From 9cba9270544a1b330a713107c9cddd0072e01b8f Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Tue, 19 Nov 2024 16:34:26 -0600 Subject: [PATCH 01/52] Reduce the log noise caused by core report summary (#1426) Signed-off-by: Ahmed Hussein (amahussein) Contributes to #1416 This change set the log level of ToolTextFileWriter to debug and summarize the logging into a single message to show the final directory --- .../com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala | 2 +- .../com/nvidia/spark/rapids/tool/profiling/Profiler.scala | 1 + .../spark/rapids/tool/qualification/Qualification.scala | 8 +------- .../spark/sql/rapids/tool/util/RuntimeReporter.scala | 3 +++ 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala index 87701419a..a05f7c74a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/ToolTextFileWriter.scala @@ -70,7 +70,7 @@ class ToolTextFileWriter( // No need to close the outputStream. // Java should handle nested streams automatically. utf8Writer.foreach { writer => - logInfo(s"$finalLocationText output location: $textOutputLoc") + logDebug(s"$finalLocationText output location: $textOutputLoc") writer.flush() writer.close() } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index bad5524e3..a69701c64 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -119,6 +119,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea progressBar.foreach(_.finishAll()) // Write status reports for all event logs to a CSV file + logOutputPath() val reportResults = generateStatusResults(appStatusReporter.asScala.values.toSeq) ProfileOutputWriter.writeCSVTable("Profiling Status", reportResults, outputDir) } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index 23d0defe7..c50464233 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -237,13 +237,6 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, } } - /** - * The outputPath of the current instance of the provider - */ - def getReportOutputPath: String = { - s"$outputDir/rapids_4_spark_qualification_output" - } - /** * Generates a qualification report based on the provided summary information. */ @@ -263,6 +256,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, qWriter.writeStageReport(allAppsSum, order) qWriter.writeUnsupportedOpsSummaryCSVReport(allAppsSum) val appStatusResult = generateStatusResults(appStatusReporter.asScala.values.toSeq) + logOutputPath() qWriter.writeStatusReport(appStatusResult, order) if (mlOpsEnabled) { if (allAppsSum.exists(x => x.mlFunctions.nonEmpty)) { diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala index 57a8ad074..0ae7bfa6b 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/RuntimeReporter.scala @@ -27,6 +27,9 @@ trait RuntimeReporter extends Logging { def generateRuntimeReport(hadoopConf: Option[Configuration] = None): Unit = { RuntimeUtil.generateReport(outputDir, hadoopConf) } + def logOutputPath(): Unit = { + logInfo(s"Tools output directory: $outputDir") + } /** * Updates the status of "SUCCESS" applications to "SKIPPED" if newer attempts with From de40e8d2bbe5eac932efd78d76b763512e0a90c0 Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Fri, 22 Nov 2024 11:55:02 -0800 Subject: [PATCH 02/52] [FEA] Add stage/task level diagnostic output for GPU slowness in Profiler tool (#1375) * initial implementation Signed-off-by: cindyyuanjiang * updated output schema based on offline discussion Signed-off-by: cindyyuanjiang * address feedback to merge two tables together Signed-off-by: cindyyuanjiang * update order of columns Signed-off-by: cindyyuanjiang * get gpu semaphore time Signed-off-by: cindyyuanjiang * add benchmark Signed-off-by: cindyyuanjiang * clean up code Signed-off-by: cindyyuanjiang * add unit test Signed-off-by: cindyyuanjiang * new expectation file Signed-off-by: cindyyuanjiang * address review feedback Signed-off-by: cindyyuanjiang * address review feedback Signed-off-by: cindyyuanjiang * add new file Signed-off-by: cindyyuanjiang * remove unnecessary comment Signed-off-by: cindyyuanjiang * address review feedback Signed-off-by: cindyyuanjiang * refactored for memory optimization Signed-off-by: cindyyuanjiang * addressed review feedback Signed-off-by: cindyyuanjiang * refactor stageDiagnosticResults Signed-off-by: cindyyuanjiang * change num attemps to tasks Signed-off-by: cindyyuanjiang * remove diagnostic from applicationsummaryinfo Signed-off-by: cindyyuanjiang * remove unused import Signed-off-by: cindyyuanjiang * new file Signed-off-by: cindyyuanjiang * add diagnostic view in qual tool output Signed-off-by: cindyyuanjiang * remove diagnostic vire from qual tool profile.log file Signed-off-by: cindyyuanjiang * address review feedback Signed-off-by: cindyyuanjiang * add profile benchmark class Signed-off-by: cindyyuanjiang * fix profiler benchmark Signed-off-by: cindyyuanjiang --------- Signed-off-by: cindyyuanjiang --- .../tool/analysis/AggRawMetricsResult.scala | 6 +- .../rapids/tool/analysis/AnalysisUtils.scala | 36 ++++ .../tool/analysis/AppSQLPlanAnalyzer.scala | 28 ++- .../analysis/AppSparkMetricsAggTrait.scala | 12 +- .../analysis/AppSparkMetricsAnalyzer.scala | 91 +++++++- .../profiling/DiagnosticSummaryInfo.scala | 25 +++ .../profiling/ProfileClassWarehouse.scala | 204 ++++++++++++++---- .../tool/profiling/ProfileOutputWriter.scala | 6 + .../rapids/tool/profiling/Profiler.scala | 51 +++-- .../rapids/tool/tuning/TunerContext.scala | 6 +- .../tool/views/AggMetricsResultSorter.scala | 15 +- .../tool/views/QualRawReportGenerator.scala | 18 +- .../rapids/tool/views/RawMetricProfView.scala | 8 +- .../spark/rapids/tool/views/package.scala | 1 + .../SingleThreadedProfileToolBenchmark.scala | 36 ++++ .../sql/rapids/tool/store/AccumNameRef.scala | 4 + ..._stagediagnosticmetricsagg_expectation.csv | 5 + .../rapids/tool/profiling/AnalysisSuite.scala | 87 ++++++++ .../tool/profiling/ApplicationInfoSuite.scala | 8 +- 19 files changed, 560 insertions(+), 87 deletions(-) create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala create mode 100644 core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/SingleThreadedProfileToolBenchmark.scala create mode 100644 core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala index b21e36054..2584d702f 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AggRawMetricsResult.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tool.analysis -import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult} +import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult, StageDiagnosticResult} /** * The result of the aggregation of the raw metrics. It contains the aggregated metrics for an @@ -32,6 +32,7 @@ import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobAggTa * @param ioAggs lists the SQLs along their IO metrics * @param sqlDurAggs the aggregated duration and CPU time for SQLs * @param maxTaskInputSizes a sequence of SQLMaxTaskInputSizes that contains the maximum input size + * @param stageDiagnostics the stage level Spark metrics for diagnostic purposes */ case class AggRawMetricsResult( jobAggs: Seq[JobAggTaskMetricsProfileResult], @@ -40,4 +41,5 @@ case class AggRawMetricsResult( sqlAggs: Seq[SQLTaskAggMetricsProfileResult], ioAggs: Seq[IOAnalysisProfileResult], sqlDurAggs: Seq[SQLDurationExecutorTimeProfileResult], - maxTaskInputSizes: Seq[SQLMaxTaskInputSizes]) + maxTaskInputSizes: Seq[SQLMaxTaskInputSizes], + stageDiagnostics: Seq[StageDiagnosticResult]) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala new file mode 100644 index 000000000..01f6989c7 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis + +object StageAccumDiagnosticMetrics { + val MEMORY_SPILLED_METRIC = "internal.metrics.memoryBytesSpilled" + val DISK_SPILLED_METRIC = "internal.metrics.diskBytesSpilled" + val INPUT_BYTES_READ_METRIC = "internal.metrics.input.bytesRead" + val OUTPUT_BYTES_WRITTEN_METRIC = "internal.metrics.output.bytesWritten" + val SW_TOTAL_BYTES_METRIC = "internal.metrics.shuffle.write.bytesWritten" + val SR_FETCH_WAIT_TIME_METRIC = "internal.metrics.shuffle.read.fetchWaitTime" + val SW_WRITE_TIME_METRIC = "internal.metrics.shuffle.write.writeTime" + val GPU_SEMAPHORE_WAIT_METRIC = "gpuSemaphoreWait" + + /** + * Get all diagnostic metrics + */ + def getAllDiagnosticMetrics: Set[String] = Set(MEMORY_SPILLED_METRIC, + DISK_SPILLED_METRIC, INPUT_BYTES_READ_METRIC, OUTPUT_BYTES_WRITTEN_METRIC, + SW_TOTAL_BYTES_METRIC, SR_FETCH_WAIT_TIME_METRIC, SW_WRITE_TIME_METRIC, + GPU_SEMAPHORE_WAIT_METRIC) +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index e32061726..7e17767b9 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -56,6 +56,24 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // SQLPlanParser. var unsupportedSQLPlan: ArrayBuffer[UnsupportedSQLPlan] = ArrayBuffer[UnsupportedSQLPlan]() var allSQLMetrics: ArrayBuffer[SQLMetricInfoCase] = ArrayBuffer[SQLMetricInfoCase]() + // A map between stage ID and a set of node names + val stageToNodeNames: HashMap[Long, Seq[String]] = HashMap.empty[Long, Seq[String]] + // A map between stage ID and diagnostic metrics results (stored as a map between metric name + // and AccumProfileResults) + val stageToDiagnosticMetrics: HashMap[Long, HashMap[String, AccumProfileResults]] = + HashMap.empty[Long, HashMap[String, AccumProfileResults]] + + /** + * Given an input diagnostic metric result, update stageToDiagnosticMetrics mapping + * @param accum AccumProfileResults to be analyzed + */ + private def updateStageDiagnosticMetrics(accum: AccumProfileResults): Unit = { + val stageId = accum.stageId + if (!stageToDiagnosticMetrics.contains(stageId)) { + stageToDiagnosticMetrics(stageId) = HashMap.empty[String, AccumProfileResults] + } + stageToDiagnosticMetrics(stageId)(accum.accMetaRef.getName()) = accum + } /** * Connects Operators to Stages using AccumulatorIDs. @@ -261,6 +279,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } validNodes.map(n => s"${n.name}(${n.id.toString})") }.getOrElse(Seq.empty) + stageToNodeNames(sModel.stageInfo.stageId) = nodeNames SQLStageInfoProfileResult(appIndex, j.sqlID.get, jobId, sModel.stageInfo.stageId, sModel.stageInfo.attemptNumber(), sModel.duration, nodeNames) } @@ -339,14 +358,19 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } else { taskUpatesSubset(taskUpatesSubset.size / 2) } - Some(AccumProfileResults( + // reuse AccumProfileResults to avoid generating extra memory from allocating new objects + val accumProfileResults = AccumProfileResults( appIndex, stageId, accumInfo.infoRef, min = min, median = median, max = max, - total = sum)) + total = sum) + if (accumInfo.infoRef.name.isDiagnosticMetrics()) { + updateStageDiagnosticMetrics(accumProfileResults) + } + Some(accumProfileResults) } }) } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala index e219ee46b..0f43ae8b2 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala @@ -26,10 +26,14 @@ trait AppSparkMetricsAggTrait extends AppIndexMapperTrait { * object to aggregate the Raw metrics and returns the result * @param app the AppBase to be analyzed * @param index the application index + * @param sqlAnalyzer optional AppSQLPlanAnalyzer used to aggregate diagnostic metrics, + * this is already present in ApplicationInfo for Profiler, but for + * Qualification this argument needs to be provided. * @return a single record of AggRawMetricsResult containing all the raw aggregated Spark * metrics */ - def getAggRawMetrics(app: AppBase, index: Int): AggRawMetricsResult = { + def getAggRawMetrics(app: AppBase, index: Int, sqlAnalyzer: Option[AppSQLPlanAnalyzer] = None): + AggRawMetricsResult = { val analysisObj = new AppSparkMetricsAnalyzer(app) AggRawMetricsResult( analysisObj.aggregateSparkMetricsByJob(index), @@ -38,7 +42,8 @@ trait AppSparkMetricsAggTrait extends AppIndexMapperTrait { analysisObj.aggregateSparkMetricsBySql(index), analysisObj.aggregateIOMetricsBySql(analysisObj.aggregateSparkMetricsBySql(index)), analysisObj.aggregateDurationAndCPUTimeBySql(index), - Seq(analysisObj.maxTaskInputSizeBytesPerSQL(index))) + Seq(analysisObj.maxTaskInputSizeBytesPerSQL(index)), + analysisObj.aggregateDiagnosticMetricsByStage(index, sqlAnalyzer)) } /** @@ -59,7 +64,8 @@ trait AppSparkMetricsAggTrait extends AppIndexMapperTrait { agg1.sqlAggs ++ agg2.sqlAggs, agg1.ioAggs ++ agg2.ioAggs, agg1.sqlDurAggs ++ agg2.sqlDurAggs, - agg1.maxTaskInputSizes ++ agg2.maxTaskInputSizes) + agg1.maxTaskInputSizes ++ agg2.maxTaskInputSizes, + agg1.stageDiagnostics ++ agg2.stageDiagnostics) } } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 48723739a..69dcbb787 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -18,13 +18,15 @@ package com.nvidia.spark.rapids.tool.analysis import java.util.concurrent.TimeUnit -import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap, LinkedHashMap} +import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper -import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult} +import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult, StageDiagnosticResult} import org.apache.spark.sql.rapids.tool.{AppBase, ToolUtils} -import org.apache.spark.sql.rapids.tool.store.{AccumInfo, TaskModel} +import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo +import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef, AccumNameRef, TaskModel} /** * Does analysis on the DataFrames from object of AppBase. @@ -50,14 +52,14 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { // Hashmap to cache the stage level metrics. It is initialized to None just in case the caller // does not call methods in order starting with stage level metrics. private var stageLevelCache: - Option[mutable.LinkedHashMap[Int, StageAggTaskMetricsProfileResult]] = None + Option[LinkedHashMap[Int, StageAggTaskMetricsProfileResult]] = None // Getter method used to protect the cache from out-of-order calls. // If the stage-level metrics are not generated yet, generates and add them to the cache private def stageLevelSparkMetrics( - index: Int): mutable.LinkedHashMap[Int, StageAggTaskMetricsProfileResult] = { + index: Int): LinkedHashMap[Int, StageAggTaskMetricsProfileResult] = { if (stageLevelCache.isEmpty) { - stageLevelCache = Some(mutable.LinkedHashMap[Int, StageAggTaskMetricsProfileResult]()) + stageLevelCache = Some(LinkedHashMap[Int, StageAggTaskMetricsProfileResult]()) aggregateSparkMetricsByStageInternal(index) } stageLevelCache.get @@ -320,6 +322,62 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { sqlRows.toSeq } + /** + * Aggregates the diagnostic SparkMetrics by stage. + * @param index the App-index (used by the profiler tool) + * @param analyzer optional AppSQLPlanAnalyzer which is used to pull stage level + * information like node names and diagnostic metrics results, only + * Qualification needs to provide this argument. + * @return sequence of StageDiagnosticAggTaskMetricsProfileResult + */ + def aggregateDiagnosticMetricsByStage(index: Int, analyzer: Option[AppSQLPlanAnalyzer] = None): + Seq[StageDiagnosticResult] = { + val sqlAnalyzer = analyzer match { + case Some(res) => res + case None => + // for Profiler this is present in ApplicationInfo + app.asInstanceOf[ApplicationInfo].planMetricProcessor + } + val zeroAccumProfileResults = + AccumProfileResults(0, 0, AccumMetaRef(0L, AccumNameRef("")), 0L, 0L, 0L, 0L) + + // TODO: this has stage attempts. we should handle different attempts + app.stageManager.getAllStages.map { sm => + // TODO: Should we only consider successful tasks? + val tasksInStage = app.taskManager.getTasks(sm.stageInfo.stageId, + sm.stageInfo.attemptNumber()) + // count duplicate task attempts + val numTasks = tasksInStage.size + val nodeNames = sqlAnalyzer.stageToNodeNames. + getOrElse(sm.stageInfo.stageId, Seq.empty[String]) + val diagnosticMetricsMap = sqlAnalyzer.stageToDiagnosticMetrics. + getOrElse(sm.stageInfo.stageId, HashMap.empty[String, AccumProfileResults]). + withDefaultValue(zeroAccumProfileResults) + val srTotalBytesMetrics = + AppSparkMetricsAnalyzer.getStatistics(tasksInStage.map(_.sr_totalBytesRead)) + + StageDiagnosticResult(index, + app.getAppName, + app.appId, + sm.stageInfo.stageId, + sm.duration, + numTasks, + srTotalBytesMetrics.min, + srTotalBytesMetrics.med, + srTotalBytesMetrics.max, + srTotalBytesMetrics.total, + diagnosticMetricsMap(MEMORY_SPILLED_METRIC), + diagnosticMetricsMap(DISK_SPILLED_METRIC), + diagnosticMetricsMap(INPUT_BYTES_READ_METRIC), + diagnosticMetricsMap(OUTPUT_BYTES_WRITTEN_METRIC), + diagnosticMetricsMap(SW_TOTAL_BYTES_METRIC), + diagnosticMetricsMap(SR_FETCH_WAIT_TIME_METRIC), + diagnosticMetricsMap(SW_WRITE_TIME_METRIC), + diagnosticMetricsMap(GPU_SEMAPHORE_WAIT_METRIC), + nodeNames) + }.toSeq + } + /** * Aggregates the SparkMetrics by stage. This is an internal method to populate the cached metrics * to be used by other aggregators. @@ -336,8 +394,8 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { // Note: // - A HashMap could be used instead of separate mutable.ArrayBuffer for each metric type, // but avoiding it for readability. - val photonPeakMemoryAccumInfos = mutable.ArrayBuffer[AccumInfo]() - val photonShuffleWriteTimeAccumInfos = mutable.ArrayBuffer[AccumInfo]() + val photonPeakMemoryAccumInfos = ArrayBuffer[AccumInfo]() + val photonShuffleWriteTimeAccumInfos = ArrayBuffer[AccumInfo]() if (app.isPhoton) { app.accumManager.applyToAccumInfoMap { accumInfo => @@ -434,6 +492,23 @@ object AppSparkMetricsAnalyzer { } } + /** + * Given an input iterable, returns its min, median, max and sum. + */ + def getStatistics(arr: Iterable[Long]): StatisticsMetrics = { + if (arr.isEmpty) { + StatisticsMetrics(0L, 0L, 0L, 0L) + } + val sortedArr = arr.toSeq.sorted + val len = sortedArr.size + val med = if (len % 2 == 0) { + (sortedArr(len / 2) + sortedArr(len / 2 - 1)) / 2 + } else { + sortedArr(len / 2) + } + StatisticsMetrics(sortedArr.head, med, sortedArr(len - 1), sortedArr.sum) + } + def maxWithEmptyHandling(arr: Iterable[Long]): Long = { if (arr.isEmpty) { 0L diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala new file mode 100644 index 000000000..6ba34f7f5 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala @@ -0,0 +1,25 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.profiling + +/** + * Stores Profiler diagnostic info. + * TODO: We plan to add two more fields/views in upcoming PRs. + */ +case class DiagnosticSummaryInfo( + stageDiagnostics: Seq[StageDiagnosticResult] +) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 24b6cbd73..ab5abdd2b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -96,9 +96,7 @@ case class SQLCleanAndAlignIdsProfileResult( override def convertToSeq: Seq[String] = { Seq(appIndex.toString, sqlID.toString) } - override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, sqlID.toString) - } + override def convertToCSVSeq: Seq[String] = convertToSeq } case class SQLStageInfoProfileResult( @@ -185,9 +183,7 @@ case class AppStatusResult( Seq(path, status, appId, message) } - override def convertToCSVSeq: Seq[String] = { - Seq(path, status, appId, message) - } + override def convertToCSVSeq: Seq[String] = convertToSeq } // note that some things might not be set until after sqlMetricsAggregation called @@ -501,40 +497,8 @@ trait BaseJobStageAggTaskMetricsProfileResult extends ProfileResult { swRecordsWrittenSum.toString, swWriteTimeSum.toString) } - override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, - id.toString, - numTasks.toString, - durStr, - diskBytesSpilledSum.toString, - durationSum.toString, - durationMax.toString, - durationMin.toString, - durationAvg.toString, - executorCPUTimeSum.toString, - executorDeserializeCpuTimeSum.toString, - executorDeserializeTimeSum.toString, - executorRunTimeSum.toString, - inputBytesReadSum.toString, - inputRecordsReadSum.toString, - jvmGCTimeSum.toString, - memoryBytesSpilledSum.toString, - outputBytesWrittenSum.toString, - outputRecordsWrittenSum.toString, - peakExecutionMemoryMax.toString, - resultSerializationTimeSum.toString, - resultSizeMax.toString, - srFetchWaitTimeSum.toString, - srLocalBlocksFetchedSum.toString, - srcLocalBytesReadSum.toString, - srRemoteBlocksFetchSum.toString, - srRemoteBytesReadSum.toString, - srRemoteBytesReadToDiskSum.toString, - srTotalBytesReadSum.toString, - swBytesWrittenSum.toString, - swRecordsWrittenSum.toString, - swWriteTimeSum.toString) - } + + override def convertToCSVSeq: Seq[String] = convertToSeq } case class JobAggTaskMetricsProfileResult( @@ -609,6 +573,166 @@ case class StageAggTaskMetricsProfileResult( override def idHeader = "stageId" } +case class StageDiagnosticResult( + appIndex: Int, + appName: String, + appId: String, + stageId: Long, + duration: Option[Long], + numTasks: Int, + srTotalBytesReadMin: Long, + srTotalBytesReadMed: Long, + srTotalBytesReadMax: Long, + srTotalBytesReadSum: Long, + memoryBytesSpilled: AccumProfileResults, + diskBytesSpilled: AccumProfileResults, + inputBytesRead: AccumProfileResults, + outputBytesWritten: AccumProfileResults, + swBytesWritten: AccumProfileResults, + srFetchWaitTime: AccumProfileResults, + swWriteTime: AccumProfileResults, + gpuSemaphoreWait: AccumProfileResults, + nodeNames: Seq[String]) extends ProfileResult { + + def bytesToMB(numBytes: Long): Long = numBytes / (1024 * 1024) + + def nanoToMilliSec(numNano: Long): Long = numNano / 1000000 + + val durStr = duration match { + case Some(dur) => dur.toString + case None => "null" + } + + override val outputHeaders = { + Seq("appIndex", + "appName", + "appId", + "stageId", + "stageDurationMs", + "numTasks", + "memoryBytesSpilledMBMin", + "memoryBytesSpilledMBMedian", + "memoryBytesSpilledMBMax", + "memoryBytesSpilledMBTotal", + "diskBytesSpilledMBMin", + "diskBytesSpilledMBMedian", + "diskBytesSpilledMBMax", + "diskBytesSpilledMBTotal", + "inputBytesReadMin", + "inputBytesReadMedian", + "inputBytesReadMax", + "inputBytesReadTotal", + "outputBytesWrittenMin", + "outputBytesWrittenMedian", + "outputBytesWrittenMax", + "outputBytesWrittenTotal", + "shuffleReadBytesMin", + "shuffleReadBytesMedian", + "shuffleReadBytesMax", + "shuffleReadBytesTotal", + "shuffleWriteBytesMin", + "shuffleWriteBytesMedian", + "shuffleWriteBytesMax", + "shuffleWriteBytesTotal", + "shuffleReadFetchWaitTimeMin", + "shuffleReadFetchWaitTimeMedian", + "shuffleReadFetchWaitTimeMax", + "shuffleReadFetchWaitTimeTotal", + "shuffleWriteWriteTimeMin", + "shuffleWriteWriteTimeMedian", + "shuffleWriteWriteTimeMax", + "shuffleWriteWriteTimeTotal", + "gpuSemaphoreWaitTimeTotal", + "SQL Nodes(IDs)") + } + + override def convertToSeq: Seq[String] = { + Seq(appIndex.toString, + appName, + appId, + stageId.toString, + durStr, + numTasks.toString, + bytesToMB(memoryBytesSpilled.min).toString, + bytesToMB(memoryBytesSpilled.median).toString, + bytesToMB(memoryBytesSpilled.max).toString, + bytesToMB(memoryBytesSpilled.total).toString, + bytesToMB(diskBytesSpilled.min).toString, + bytesToMB(diskBytesSpilled.median).toString, + bytesToMB(diskBytesSpilled.max).toString, + bytesToMB(diskBytesSpilled.total).toString, + inputBytesRead.min.toString, + inputBytesRead.median.toString, + inputBytesRead.max.toString, + inputBytesRead.total.toString, + outputBytesWritten.min.toString, + outputBytesWritten.median.toString, + outputBytesWritten.max.toString, + outputBytesWritten.total.toString, + srTotalBytesReadMin.toString, + srTotalBytesReadMed.toString, + srTotalBytesReadMax.toString, + srTotalBytesReadSum.toString, + swBytesWritten.min.toString, + swBytesWritten.median.toString, + swBytesWritten.max.toString, + swBytesWritten.total.toString, + nanoToMilliSec(srFetchWaitTime.min).toString, + nanoToMilliSec(srFetchWaitTime.median).toString, + nanoToMilliSec(srFetchWaitTime.max).toString, + nanoToMilliSec(srFetchWaitTime.total).toString, + nanoToMilliSec(swWriteTime.min).toString, + nanoToMilliSec(swWriteTime.median).toString, + nanoToMilliSec(swWriteTime.max).toString, + nanoToMilliSec(swWriteTime.total).toString, + gpuSemaphoreWait.total.toString, + nodeNames.mkString(",")) + } + + override def convertToCSVSeq: Seq[String] = { + Seq(appIndex.toString, + appName, + appId, + stageId.toString, + durStr, + numTasks.toString, + bytesToMB(memoryBytesSpilled.min).toString, + bytesToMB(memoryBytesSpilled.median).toString, + bytesToMB(memoryBytesSpilled.max).toString, + bytesToMB(memoryBytesSpilled.total).toString, + bytesToMB(diskBytesSpilled.min).toString, + bytesToMB(diskBytesSpilled.median).toString, + bytesToMB(diskBytesSpilled.max).toString, + bytesToMB(diskBytesSpilled.total).toString, + inputBytesRead.min.toString, + inputBytesRead.median.toString, + inputBytesRead.max.toString, + inputBytesRead.total.toString, + outputBytesWritten.min.toString, + outputBytesWritten.median.toString, + outputBytesWritten.max.toString, + outputBytesWritten.total.toString, + srTotalBytesReadMin.toString, + srTotalBytesReadMed.toString, + srTotalBytesReadMax.toString, + srTotalBytesReadSum.toString, + swBytesWritten.min.toString, + swBytesWritten.median.toString, + swBytesWritten.max.toString, + swBytesWritten.total.toString, + nanoToMilliSec(srFetchWaitTime.min).toString, + nanoToMilliSec(srFetchWaitTime.median).toString, + nanoToMilliSec(srFetchWaitTime.max).toString, + nanoToMilliSec(srFetchWaitTime.total).toString, + nanoToMilliSec(swWriteTime.min).toString, + nanoToMilliSec(swWriteTime.median).toString, + nanoToMilliSec(swWriteTime.max).toString, + nanoToMilliSec(swWriteTime.total).toString, + gpuSemaphoreWait.total.toString, + StringUtils.reformatCSVString(nodeNames.mkString(","))) + } +} + case class SQLMaxTaskInputSizes( appIndex: Int, appId: String, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileOutputWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileOutputWriter.scala index e42ec24fa..3c87919ee 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileOutputWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileOutputWriter.scala @@ -74,6 +74,12 @@ class ProfileOutputWriter(outputDir: String, filePrefix: String, numOutputRows: } } + def writeCSVTable(headerText: String, outRows: Seq[ProfileResult]): Unit = { + if (outputCSV) { + ProfileOutputWriter.writeCSVTable(headerText, outRows, outputDir) + } + } + def close(): Unit = { textFileWriter.close() } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index a69701c64..75b4c4590 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -76,9 +76,11 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea Profiler.COMPARE_LOG_FILE_NAME_PREFIX, numOutputRows, outputCSV = outputCSV) try { // we need the info for all of the apps to be able to compare so this happens serially - val (sums, comparedRes) = processApps(apps, printPlans = false, profileOutputWriter) + val (sums, comparedRes, diagnostics) = + processApps(apps, printPlans = false, profileOutputWriter) progressBar.foreach(_.reportSuccessfulProcesses(apps.size)) - writeSafelyToOutput(profileOutputWriter, Seq(sums), false, comparedRes) + writeSafelyToOutput(profileOutputWriter, Seq(sums), false, comparedRes, + Seq(diagnostics)) } catch { case _: Exception => progressBar.foreach(_.reportFailedProcesses(apps.size)) @@ -97,8 +99,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea // combine them into single tables in the output. val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/combined", Profiler.COMBINED_LOG_FILE_NAME_PREFIX, numOutputRows, outputCSV = outputCSV) - val sums = createAppsAndSummarize(eventLogInfos, profileOutputWriter) - writeSafelyToOutput(profileOutputWriter, sums, outputCombined) + val (sums, diagnostics) = createAppsAndSummarize(eventLogInfos, profileOutputWriter) + writeSafelyToOutput(profileOutputWriter, sums, outputCombined, diagnosticSum = diagnostics) profileOutputWriter.close() } } else { @@ -227,14 +229,18 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea allApps.asScala.toSeq } - private def createAppsAndSummarize(allPaths: Seq[EventLogInfo], - profileOutputWriter: ProfileOutputWriter): Seq[ApplicationSummaryInfo] = { + private def createAppsAndSummarize( + allPaths: Seq[EventLogInfo], + profileOutputWriter: ProfileOutputWriter) + : (Seq[ApplicationSummaryInfo], Seq[DiagnosticSummaryInfo]) = { val allApps = new ConcurrentLinkedQueue[ApplicationSummaryInfo]() + val allDiagnostics = new ConcurrentLinkedQueue[DiagnosticSummaryInfo]() class ProfileThread(path: EventLogInfo, index: Int) extends Runnable { def run: Unit = profileApp(path, index, { app => - val (s, _) = processApps(Seq(app), false, profileOutputWriter) + val (s, _, d) = processApps(Seq(app), false, profileOutputWriter) allApps.add(s) + allDiagnostics.add(d) }) } @@ -255,7 +261,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea " stopping processing any more event logs") threadPool.shutdownNow() } - allApps.asScala.toSeq + (allApps.asScala.toSeq, allDiagnostics.asScala.toSeq) } private def createAppAndProcess( @@ -266,8 +272,10 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea val profileOutputWriter = new ProfileOutputWriter(s"$outputDir/${app.appId}", Profiler.PROFILE_LOG_NAME, numOutputRows, outputCSV = outputCSV) try { - val (sum, _) = processApps(Seq(app), appArgs.printPlans(), profileOutputWriter) - writeSafelyToOutput(profileOutputWriter, Seq(sum), false) + val (sum, _, diagnostics) = + processApps(Seq(app), appArgs.printPlans(), profileOutputWriter) + writeSafelyToOutput(profileOutputWriter, Seq(sum), false, + diagnosticSum = Seq(diagnostics)) } finally { profileOutputWriter.close() } @@ -312,7 +320,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea apps: Seq[ApplicationInfo], printPlans: Boolean, profileOutputWriter: ProfileOutputWriter) - : (ApplicationSummaryInfo, Option[CompareSummaryInfo]) = { + : (ApplicationSummaryInfo, Option[CompareSummaryInfo], DiagnosticSummaryInfo) = { val startTime = System.currentTimeMillis() val collect = new CollectInformation(apps) @@ -327,6 +335,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea val stageMetrics = collect.getStageLevelMetrics val wholeStage = collect.getWholeStageCodeGenMapping val sparkRapidsBuildInfo = collect.getSparkRapidsInfo + // for compare mode we just add in extra tables for matching across applications // the rest of the tables simply list all applications specified val compareRes = if (appArgs.compare()) { @@ -393,7 +402,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea analysis.sqlAggs, analysis.sqlDurAggs, analysis.taskShuffleSkew, failedTasks, failedStages, failedJobs, removedBMs, removedExecutors, unsupportedOps, sparkProps, collect.getSQLToStage, wholeStage, maxTaskInputInfo, - appLogPath, analysis.ioAggs, systemProps, sqlIdAlign, sparkRapidsBuildInfo), compareRes) + appLogPath, analysis.ioAggs, systemProps, sqlIdAlign, sparkRapidsBuildInfo), + compareRes, DiagnosticSummaryInfo(analysis.stageDiagnostics)) } /** @@ -427,7 +437,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea def writeOutput(profileOutputWriter: ProfileOutputWriter, appsSum: Seq[ApplicationSummaryInfo], outputCombined: Boolean, - comparedRes: Option[CompareSummaryInfo] = None): Unit = { + comparedRes: Option[CompareSummaryInfo] = None, + diagnosticSum: Seq[DiagnosticSummaryInfo]): Unit = { val sums = if (outputCombined) { // the properties table here has the column names as the app indexes so we have to @@ -557,6 +568,15 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea profileOutputWriter.writeSparkRapidsBuildInfo("Spark Rapids Build Info", app.sparkRapidsBuildInfo) } + // Write diagnostic related results to CSV files + val diagnostics = if (outputCombined) { + Seq(DiagnosticSummaryInfo(diagnosticSum.flatMap(_.stageDiagnostics))) + } else { + diagnosticSum + } + diagnostics.foreach { diagnostoic => + profileOutputWriter.writeCSVTable(STAGE_DIAGNOSTICS_LABEL, diagnostoic.stageDiagnostics) + } } /** @@ -566,9 +586,10 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea */ private def writeSafelyToOutput(profileOutputWriter: ProfileOutputWriter, appsSum: Seq[ApplicationSummaryInfo], outputCombined: Boolean, - comparedRes: Option[CompareSummaryInfo] = None): Unit = { + comparedRes: Option[CompareSummaryInfo] = None, + diagnosticSum: Seq[DiagnosticSummaryInfo]): Unit = { try { - writeOutput(profileOutputWriter, appsSum, outputCombined, comparedRes) + writeOutput(profileOutputWriter, appsSum, outputCombined, comparedRes, diagnosticSum) } catch { case e: Exception => logError("Exception thrown while writing", e) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala index c795d0434..b9b23ec07 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids.tool.tuning import scala.util.{Failure, Success, Try} import com.nvidia.spark.rapids.tool.Platform -import com.nvidia.spark.rapids.tool.analysis.QualSparkMetricsAnalyzer +import com.nvidia.spark.rapids.tool.analysis.{AppSQLPlanAnalyzer, QualSparkMetricsAnalyzer} import com.nvidia.spark.rapids.tool.profiling.{DataSourceProfileResult, RecommendedCommentResult, RecommendedPropertyResult} import org.apache.hadoop.conf.Configuration @@ -53,7 +53,9 @@ case class TunerContext ( appIndex: Int = 1, dsInfo: Seq[DataSourceProfileResult], platform: Platform): Option[TuningResult] = { - val rawAggMetrics = QualSparkMetricsAnalyzer.getAggRawMetrics(appInfo, appIndex) + val sqlAnalyzer = AppSQLPlanAnalyzer(appInfo, appIndex) + val rawAggMetrics = + QualSparkMetricsAnalyzer.getAggRawMetrics(appInfo, appIndex, Some(sqlAnalyzer)) QualificationAutoTuner(appInfo, appAggStats, this, rawAggMetrics, dsInfo).collect { case qualTuner => Try { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala index b6cbff4b0..440a4ac64 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tool.views -import com.nvidia.spark.rapids.tool.profiling.{BaseJobStageAggTaskMetricsProfileResult, IOAnalysisProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLTaskAggMetricsProfileResult} +import com.nvidia.spark.rapids.tool.profiling.{BaseJobStageAggTaskMetricsProfileResult, IOAnalysisProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLTaskAggMetricsProfileResult, StageDiagnosticResult} /** * Contains the sort logic for the aggregated Spark RawMetrics. @@ -91,4 +91,17 @@ object AggMetricsResultSorter { } } } + + def sortStageDiagnostics( + rows: Seq[StageDiagnosticResult]): + Seq[StageDiagnosticResult] = { + if (rows.isEmpty) { + Seq.empty + } else { + rows.sortBy { cols => + val sortDur = cols.duration.getOrElse(0L) + (cols.appIndex, -sortDur, -cols.memoryBytesSpilled.total) + } + } + } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala index d1252d4ec..3a2cb8b48 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala @@ -36,14 +36,16 @@ object QualRawReportGenerator { AggMetricsResultSorter.sortSqlAgg(aggRawResult.sqlAggs), AggMetricsResultSorter.sortIO(aggRawResult.ioAggs), AggMetricsResultSorter.sortSqlDurationAgg(aggRawResult.sqlDurAggs), - aggRawResult.maxTaskInputSizes) + aggRawResult.maxTaskInputSizes, + AggMetricsResultSorter.sortStageDiagnostics(aggRawResult.stageDiagnostics)) Map( STAGE_AGG_LABEL -> sortedRes.stageAggs, JOB_AGG_LABEL -> sortedRes.jobAggs, TASK_SHUFFLE_SKEW -> sortedRes.taskShuffleSkew, SQL_AGG_LABEL -> sortedRes.sqlAggs, IO_LABEL -> sortedRes.ioAggs, - SQL_DUR_LABEL -> sortedRes.sqlDurAggs) + SQL_DUR_LABEL -> sortedRes.sqlDurAggs, + STAGE_DIAGNOSTICS_LABEL -> sortedRes.stageDiagnostics) } private def generateSQLProcessingView( @@ -92,11 +94,13 @@ object QualRawReportGenerator { SystemQualPropertiesView.getRawView(Seq(app)), Some(SystemQualPropertiesView.getDescription)) pWriter.writeText("\n### B. Analysis ###\n") - constructLabelsMaps( - QualSparkMetricsAnalyzer.getAggRawMetrics(app, appIndex)).foreach { case (label, metrics) => - pWriter.write(label, - metrics, - AGG_DESCRIPTION.get(label)) + constructLabelsMaps(QualSparkMetricsAnalyzer. + getAggRawMetrics(app, appIndex, Some(sqlPlanAnalyzer))).foreach { case (label, metrics) => + if (label == STAGE_DIAGNOSTICS_LABEL) { + pWriter.writeCSVTable(label, metrics) + } else { + pWriter.write(label, metrics, AGG_DESCRIPTION.get(label)) + } } pWriter.writeText("\n### C. Health Check###\n") pWriter.write(QualFailedTaskView.getLabel, QualFailedTaskView.getRawView(Seq(app))) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala index 5caf7436b..3936d0222 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/RawMetricProfView.scala @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids.tool.views import com.nvidia.spark.rapids.tool.analysis.ProfSparkMetricsAnalyzer -import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult} +import com.nvidia.spark.rapids.tool.profiling.{IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult, StageDiagnosticResult} import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo @@ -30,7 +30,8 @@ case class ProfilerAggregatedView( sqlAggs: Seq[SQLTaskAggMetricsProfileResult], ioAggs: Seq[IOAnalysisProfileResult], sqlDurAggs: Seq[SQLDurationExecutorTimeProfileResult], - maxTaskInputSizes: Seq[SQLMaxTaskInputSizes]) + maxTaskInputSizes: Seq[SQLMaxTaskInputSizes], + stageDiagnostics: Seq[StageDiagnosticResult]) object RawMetricProfilerView { def getAggMetrics(apps: Seq[ApplicationInfo]): ProfilerAggregatedView = { @@ -42,6 +43,7 @@ object RawMetricProfilerView { AggMetricsResultSorter.sortSqlAgg(aggMetricsResults.sqlAggs), AggMetricsResultSorter.sortIO(aggMetricsResults.ioAggs), AggMetricsResultSorter.sortSqlDurationAgg(aggMetricsResults.sqlDurAggs), - aggMetricsResults.maxTaskInputSizes) + aggMetricsResults.maxTaskInputSizes, + AggMetricsResultSorter.sortStageDiagnostics(aggMetricsResults.stageDiagnostics)) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala index c588aad8d..a509ae3f7 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/package.scala @@ -28,6 +28,7 @@ package object views { val IO_LABEL = "IO Metrics" val SQL_DUR_LABEL = "SQL Duration and Executor CPU Time Percent" val SQL_MAX_INPUT_SIZE = "SQL Max Task Input Size" + val STAGE_DIAGNOSTICS_LABEL = "Stage Level Diagnostic Metrics" val AGG_DESCRIPTION = Map( STAGE_AGG_LABEL -> "Stage metrics", diff --git a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/SingleThreadedProfileToolBenchmark.scala b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/SingleThreadedProfileToolBenchmark.scala new file mode 100644 index 000000000..575329277 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/SingleThreadedProfileToolBenchmark.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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.spark.rapids.tool.benchmarks + +import com.nvidia.spark.rapids.tool.profiling.{ProfileArgs, ProfileMain} + +object ProfToolBenchmark extends BenchmarkBase { + override def runBenchmarkSuite(inputArgs: Array[String]): Unit = { + // Currently the input arguments are assumed to be common across cases + // This will be improved in a follow up PR to enable passing as a config + // file with argument support for different cases + runBenchmark("Benchmark_Profiling_CSV") { + val (prefix, suffix) = inputArgs.splitAt(inputArgs.length - 1) + addCase("Profiling_CSV") { _ => + ProfileMain.mainInternal(new ProfileArgs(prefix :+ "--num-threads" + :+ "1" :+ "--csv" :+ suffix.head), + enablePB = true) + } + run() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala index e7349d064..0172f5229 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.rapids.tool.store import java.util.concurrent.ConcurrentHashMap +import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics.getAllDiagnosticMetrics + import org.apache.spark.sql.rapids.tool.util.EventUtils.normalizeMetricName import org.apache.spark.sql.rapids.tool.util.StringUtils @@ -33,6 +35,8 @@ case class AccumNameRef(value: String) { // We opt to use this optimization because while writing the CSV files, each row is going to // create a new CSV string even though they represent the same AccumulatorName. val csvValue: String = StringUtils.reformatCSVString(value) + + def isDiagnosticMetrics(): Boolean = getAllDiagnosticMetrics.contains(value) } object AccumNameRef { diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv new file mode 100644 index 000000000..e847d1b06 --- /dev/null +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv @@ -0,0 +1,5 @@ +appIndex,appName,appId,stageId,stageDurationMs,numTasks,memoryBytesSpilledMBMin,memoryBytesSpilledMBMedian,memoryBytesSpilledMBMax,memoryBytesSpilledMBTotal,diskBytesSpilledMBMin,diskBytesSpilledMBMedian,diskBytesSpilledMBMax,diskBytesSpilledMBTotal,inputBytesReadMin,inputBytesReadMedian,inputBytesReadMax,inputBytesReadTotal,outputBytesWrittenMin,outputBytesWrittenMedian,outputBytesWrittenMax,outputBytesWrittenTotal,shuffleReadBytesMin,shuffleReadBytesMedian,shuffleReadBytesMax,shuffleReadBytesTotal,shuffleWriteBytesMin,shuffleWriteBytesMedian,shuffleWriteBytesMax,shuffleWriteBytesTotal,shuffleReadFetchWaitTimeMin,shuffleReadFetchWaitTimeMedian,shuffleReadFetchWaitTimeMax,shuffleReadFetchWaitTimeTotal,shuffleWriteWriteTimeMin,shuffleWriteWriteTimeMedian,shuffleWriteWriteTimeMax,shuffleWriteWriteTimeTotal,gpuSemaphoreWaitTimeTotal,SQL Nodes(IDs) +1,Spark shell,local-1622814619968,0,1743,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,6688608,6688702,6688825,40132250,0,0,0,0,41,60,100,400,0,"GpuColumnarExchange(16),GpuProject(17),GpuRowToColumnar(18),WholeStageCodegen (2)(19),Scan(21)" +1,Spark shell,local-1622814619968,1,1631,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,6688602,6688708,6688833,40132258,0,0,0,0,37,92,108,508,0,"GpuColumnarExchange(8),GpuProject(9),GpuRowToColumnar(10),WholeStageCodegen (1)(11),Scan(13)" +1,Spark shell,local-1622814619968,2,688,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,397220,401479,405854,80264508,77,77,77,15400,0,0,0,0,0,0,9,93,0,"GpuColumnarExchange(3),GpuHashAggregate(4),GpuProject(5),GpuShuffledHashJoin(6),GpuShuffleCoalesce(7),GpuColumnarExchange(8),GpuCoalesceBatches(14),GpuShuffleCoalesce(15),GpuColumnarExchange(16)" +1,Spark shell,local-1622814619968,3,83,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,15400,15400,15400,15400,0,0,0,0,0,0,0,0,0,0,0,0,0,"GpuColumnarToRow(0),GpuHashAggregate(1),GpuShuffleCoalesce(2),GpuColumnarExchange(3)" diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index b247433a4..2b8c3bf12 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -25,8 +25,75 @@ import org.scalatest.FunSuite import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.types._ +case class TestStageDiagnosticResult( + appIndex: Int, + appName: String, + appId: String, + stageId: Long, + duration: Option[Long], + numTasks: Int, + memoryBytesSpilledMBMin: Long, + memoryBytesSpilledMBMed: Long, + memoryBytesSpilledMBMax: Long, + memoryBytesSpilledMBSum: Long, + diskBytesSpilledMBMin: Long, + diskBytesSpilledMBMed: Long, + diskBytesSpilledMBMax: Long, + diskBytesSpilledMBSum: Long, + inputBytesReadMin: Long, + inputBytesReadMed: Long, + inputBytesReadMax: Long, + inputBytesReadSum: Long, + outputBytesWrittenMin: Long, + outputBytesWrittenMed: Long, + outputBytesWrittenMax: Long, + outputBytesWrittenSum: Long, + srTotalBytesReadMin: Long, + srTotalBytesReadMed: Long, + srTotalBytesReadMax: Long, + srTotalBytesReadSum: Long, + swBytesWrittenMin: Long, + swBytesWrittenMed: Long, + swBytesWrittenMax: Long, + swBytesWrittenSum: Long, + srFetchWaitTimeMin: Long, + srFetchWaitTimeMed: Long, + srFetchWaitTimeMax: Long, + srFetchWaitTimeSum: Long, + swWriteTimeMin: Long, + swWriteTimeMed: Long, + swWriteTimeMax: Long, + swWriteTimeSum: Long, + gpuSemaphoreWaitSum: Long, + nodeNames: Seq[String]) + class AnalysisSuite extends FunSuite { + private def createTestStageDiagnosticResult(diagnosticsResults: Seq[StageDiagnosticResult]): + Seq[TestStageDiagnosticResult] = { + def bytesToMB(numBytes: Long): Long = numBytes / (1024 * 1024) + def nanoToMilliSec(numNano: Long): Long = numNano / 1000000 + diagnosticsResults.map {result => + TestStageDiagnosticResult(result.appIndex, result.appName, result.appId, result.stageId, + result.duration, result.numTasks, bytesToMB(result.memoryBytesSpilled.min), + bytesToMB(result.memoryBytesSpilled.median), bytesToMB(result.memoryBytesSpilled.max), + bytesToMB(result.memoryBytesSpilled.total), bytesToMB(result.diskBytesSpilled.min), + bytesToMB(result.diskBytesSpilled.median), bytesToMB(result.diskBytesSpilled.max), + bytesToMB(result.diskBytesSpilled.total), result.inputBytesRead.min, + result.inputBytesRead.median, result.inputBytesRead.max, result.inputBytesRead.total, + result.outputBytesWritten.min, result.outputBytesWritten.median, + result.outputBytesWritten.max, result.outputBytesWritten.total, + result.srTotalBytesReadMin, result.srTotalBytesReadMed, result.srTotalBytesReadMax, + result.srTotalBytesReadSum, result.swBytesWritten.min, result.swBytesWritten.median, + result.swBytesWritten.max, result.swBytesWritten.total, + nanoToMilliSec(result.srFetchWaitTime.min), nanoToMilliSec(result.srFetchWaitTime.median), + nanoToMilliSec(result.srFetchWaitTime.max), nanoToMilliSec(result.srFetchWaitTime.total), + nanoToMilliSec(result.swWriteTime.min), nanoToMilliSec(result.swWriteTime.median), + nanoToMilliSec(result.swWriteTime.max), nanoToMilliSec(result.swWriteTime.total), + result.gpuSemaphoreWait.total, result.nodeNames) + } + } + lazy val sparkSession = { SparkSession .builder() @@ -75,6 +142,26 @@ class AnalysisSuite extends FunSuite { expectFile("sql"), expectFile("job"), expectFile("stage")) } + test("test stage-level diagnostic aggregation simple") { + val expectFile = "rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv" + val logs = Array(s"$logDir/rapids_join_eventlog.zstd") + val apps = ToolTestUtils.processProfileApps(logs, sparkSession) + assert(apps.size == logs.size) + + // This step is to compute stage to node names and diagnostic metrics mappings, + // which is used in collecting diagnostic metrics. + val collect = new CollectInformation(apps) + collect.getSQLToStage + collect.getStageLevelMetrics + + val aggResults = RawMetricProfilerView.getAggMetrics(apps) + import org.apache.spark.sql.functions._ + import sparkSession.implicits._ + val actualDf = createTestStageDiagnosticResult(aggResults.stageDiagnostics).toDF. + withColumn("nodeNames", concat_ws(",", col("nodeNames"))) + compareMetrics(actualDf, expectFile) + } + private def testSqlMetricsAggregation(logs: Array[String], expectFileSQL: String, expectFileJob: String, expectFileStage: String): Unit = { val apps = ToolTestUtils.processProfileApps(logs, sparkSession) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala index de9921cec..7ff03a943 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala @@ -841,7 +841,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 20) + assert(dotDirs.length === 21) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -875,7 +875,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 16) + assert(dotDirs.length === 17) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -912,7 +912,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 20) + assert(dotDirs.length === 21) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -949,7 +949,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 18) + assert(dotDirs.length === 19) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly From 3cd1fe72ac16825d7438890cb167bbca4ada590b Mon Sep 17 00:00:00 2001 From: Niranjan Artal <50492963+nartal1@users.noreply.github.com> Date: Mon, 25 Nov 2024 08:59:52 -0800 Subject: [PATCH 03/52] Include expression parsers for HashAggregate and ObjectHashAggregate (#1432) * Include expression parsers for HashAggregate and ObjectHashAggregate Signed-off-by: Niranjan Artal * address review comments --------- Signed-off-by: Niranjan Artal --- .../spark/rapids/tool/planparser/GenericExecParser.scala | 3 ++- .../rapids/tool/planparser/HashAggregateExecParser.scala | 4 +++- .../tool/planparser/ObjectHashAggregateExecParser.scala | 4 +++- .../nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala | 6 ++++-- 4 files changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala index cd67502c5..62ffb5eaa 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala @@ -35,6 +35,7 @@ class GenericExecParser( override def parse: ExecInfo = { val duration = computeDuration val expressions = parseExpressions() + val notSupportedExprs = getNotSupportedExprs(expressions) val isExecSupported = checker.isExecSupported(fullExecName) && notSupportedExprs.isEmpty @@ -59,7 +60,7 @@ class GenericExecParser( } protected def getExprString: String = { - node.desc.replaceFirst(s"${node.name} ", "") + node.desc.replaceFirst(s"^${node.name}\\s*", "") } protected def getNotSupportedExprs(expressions: Array[String]): Seq[UnsupportedExpr] = { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/HashAggregateExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/HashAggregateExecParser.scala index 9dc5e0d1c..99e051caf 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/HashAggregateExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/HashAggregateExecParser.scala @@ -26,8 +26,10 @@ case class HashAggregateExecParser( override val node: SparkPlanGraphNode, override val checker: PluginTypeChecker, override val sqlID: Long, + override val expressionFunction: Option[String => Array[String]], appBase: AppBase) extends - GenericExecParser(node, checker, sqlID, app = Some(appBase)) with Logging { + GenericExecParser(node, checker, sqlID, + expressionFunction = expressionFunction, app = Some(appBase)) with Logging { override def getDurationMetricIds: Seq[Long] = { node.metrics.find(_.name == "time in aggregation build").map(_.accumulatorId).toSeq diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ObjectHashAggregateExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ObjectHashAggregateExecParser.scala index c66b60f2f..717b22d3a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ObjectHashAggregateExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ObjectHashAggregateExecParser.scala @@ -26,8 +26,10 @@ case class ObjectHashAggregateExecParser( override val node: SparkPlanGraphNode, override val checker: PluginTypeChecker, override val sqlID: Long, + override val expressionFunction: Option[String => Array[String]], appBase: AppBase) extends - GenericExecParser(node, checker, sqlID, app = Some(appBase)) with Logging { + GenericExecParser(node, checker, sqlID, + expressionFunction = expressionFunction, app = Some(appBase)) with Logging { override def getDurationMetricIds: Seq[Long] = { node.metrics.find(_.name == "time in aggregation build").map(_.accumulatorId).toSeq diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index c4b31f78d..1b8d63a5f 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -491,11 +491,13 @@ object SQLPlanParser extends Logging { GenericExecParser( node, checker, sqlID, expressionFunction = Some(parseGenerateExpressions)).parse case "HashAggregate" => - HashAggregateExecParser(node, checker, sqlID, app).parse + HashAggregateExecParser( + node, checker, sqlID, Some(parseAggregateExpressions), app).parse case i if DataWritingCommandExecParser.isWritingCmdExec(i) => DataWritingCommandExecParser.parseNode(node, checker, sqlID) case "ObjectHashAggregate" => - ObjectHashAggregateExecParser(node, checker, sqlID, app).parse + ObjectHashAggregateExecParser( + node, checker, sqlID, Some(parseAggregateExpressions), app).parse case "Project" => GenericExecParser( node, checker, sqlID, expressionFunction = Some(parseProjectExpressions)).parse From 5f10bbd1f9a31660b44f33088d7b05d096b88205 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Wed, 27 Nov 2024 09:08:43 -0600 Subject: [PATCH 04/52] Sort Qual execs report by sqlId and nodeId (#1436) * Sort Qual execs report by sqlId and nodeId Signed-off-by: Ahmed Hussein (amahussein) * handling children execs smarter way Signed-off-by: Ahmed Hussein (amahussein) --------- Signed-off-by: Ahmed Hussein (amahussein) --- .../tool/qualification/QualOutputWriter.scala | 80 +++++++++---------- .../tool/qualification/Qualification.scala | 2 +- 2 files changed, 38 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala index d0193ca8e..ed101d820 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala @@ -255,17 +255,50 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, } } - def writeExecReport(sums: Seq[QualificationSummaryInfo], order: String) : Unit = { + def writeExecReport(sums: Seq[QualificationSummaryInfo]) : Unit = { val csvFileWriter = new ToolTextFileWriter(outputDir, s"${QualOutputWriter.LOGFILE_NAME}_execs.csv", "Plan Exec Info", hadoopConf) try { - val headersAndSizes = QualOutputWriter.getDetailedExecsHeaderStrings csvFileWriter.write(QualOutputWriter.constructDetailedHeader( QualOutputWriter.getDetailedExecsHeaderStrings, ",", false)) + val delimiter = "," + val booleanTrue = "true" + val booleanFalse = "false" + val zeroDurationStr = "0" sums.foreach { sumInfo => - val appRows = QualOutputWriter.constructExecsInfo(sumInfo, headersAndSizes, ",", false) - appRows.foreach(csvFileWriter.write(_)) + val appIDCSVStr = StringUtils.reformatCSVString(sumInfo.appId) + sumInfo.planInfo.foreach { pInfo => + val execInfos = pInfo.execInfo.flatMap { eInfo => + eInfo.children.getOrElse(Seq.empty) :+ eInfo + }.sortBy(eInfo => eInfo.nodeId) + val sqlIDStr = pInfo.sqlID.toString + val rows = execInfos.map { info => + val (childrenExecsStr, nodeIdsStr) = if (info.children.isDefined) { + (StringUtils.reformatCSVString( + info.children.get.map(_.exec).mkString(":")), + StringUtils.reformatCSVString( + info.children.get.map(_.nodeId).mkString(":"))) + } else { + ("", "") + } + Seq(appIDCSVStr, sqlIDStr, StringUtils.reformatCSVString(info.exec), + StringUtils.reformatCSVString(info.expr), + if (info.duration.isDefined) info.duration.get.toString else zeroDurationStr, + info.nodeId.toString, + if (info.isSupported) booleanTrue else booleanFalse, + StringUtils.reformatCSVString(info.stages.mkString(":")), + childrenExecsStr, + nodeIdsStr, + if (info.shouldRemove) booleanTrue else booleanFalse, + if (info.shouldIgnore) booleanTrue else booleanFalse, + StringUtils.reformatCSVString(info.getOpAction.toString) + ).mkString(delimiter) + } + if (rows.nonEmpty) { + csvFileWriter.write(s"${rows.mkString("\n")}\n") + } + } } } finally { csvFileWriter.close() @@ -941,33 +974,6 @@ object QualOutputWriter { detailedHeadersAndFields } - private def constructExecInfoBuffer( - info: ExecInfo, - appId: String, - delimiter: String, - prettyPrint: Boolean, - headersAndSizes: LinkedHashMap[String, Int], - reformatCSV: Boolean = true): String = { - val reformatCSVFunc = getReformatCSVFunc(reformatCSV) - val data = ListBuffer[(String, Int)]( - reformatCSVFunc(appId) -> headersAndSizes(APP_ID_STR), - info.sqlID.toString -> headersAndSizes(SQL_ID_STR), - reformatCSVFunc(info.exec) -> headersAndSizes(EXEC_STR), - reformatCSVFunc(info.expr) -> headersAndSizes(EXEC_STR), - info.duration.getOrElse(0).toString -> headersAndSizes(EXEC_DURATION), - info.nodeId.toString -> headersAndSizes(EXEC_NODEID), - info.isSupported.toString -> headersAndSizes(EXEC_IS_SUPPORTED), - reformatCSVFunc(info.stages.mkString(":")) -> headersAndSizes(EXEC_STAGES), - reformatCSVFunc(info.children.getOrElse(Seq.empty).map(_.exec).mkString(":")) -> - headersAndSizes(EXEC_CHILDREN), - reformatCSVFunc(info.children.getOrElse(Seq.empty).map(_.nodeId).mkString(":")) -> - headersAndSizes(EXEC_CHILDREN_NODE_IDS), - info.shouldRemove.toString -> headersAndSizes(EXEC_SHOULD_REMOVE), - info.shouldIgnore.toString -> headersAndSizes(EXEC_SHOULD_IGNORE), - reformatCSVFunc(info.getOpAction.toString) -> headersAndSizes(EXEC_ACTION) - ) - constructOutputRow(data, delimiter, prettyPrint) - } private def getDetailedStagesHeaderStrings: LinkedHashMap[String, Int] = { val detailedHeadersAndFields = LinkedHashMap[String, Int]( @@ -1077,18 +1083,6 @@ object QualOutputWriter { }.toSet } - private def constructExecsInfo( - sumInfo: QualificationSummaryInfo, - headersAndSizes: LinkedHashMap[String, Int], - delimiter: String, - prettyPrint: Boolean): Set[String] = { - // No need to visit the execInfo children because the result returned from - // "getAllExecsFromPlan" is already flattened - getAllExecsFromPlan(sumInfo.planInfo).collect { case info => - constructExecInfoBuffer(info, sumInfo.appId, delimiter, prettyPrint, headersAndSizes) - } - } - def createFormattedQualSummaryInfo( appInfo: QualificationSummaryInfo, delimiter: String = TEXT_DELIMITER) : FormattedQualificationSummaryInfo = { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index c50464233..49d70b80e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -252,7 +252,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, qWriter.writePerSqlTextReport(allAppsSum, numRows, maxSQLDescLength) qWriter.writePerSqlCSVReport(allAppsSum, maxSQLDescLength) } - qWriter.writeExecReport(allAppsSum, order) + qWriter.writeExecReport(allAppsSum) qWriter.writeStageReport(allAppsSum, order) qWriter.writeUnsupportedOpsSummaryCSVReport(allAppsSum) val appStatusResult = generateStatusResults(appStatusReporter.asScala.values.toSeq) From 9f519d2329333b6242f58ca3882f10af52bff77b Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Wed, 27 Nov 2024 11:25:38 -0800 Subject: [PATCH 05/52] fix index out of bound bug (#1439) Signed-off-by: cindyyuanjiang --- .../tool/analysis/AppSparkMetricsAnalyzer.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 69dcbb787..33194644e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -498,15 +498,16 @@ object AppSparkMetricsAnalyzer { def getStatistics(arr: Iterable[Long]): StatisticsMetrics = { if (arr.isEmpty) { StatisticsMetrics(0L, 0L, 0L, 0L) - } - val sortedArr = arr.toSeq.sorted - val len = sortedArr.size - val med = if (len % 2 == 0) { - (sortedArr(len / 2) + sortedArr(len / 2 - 1)) / 2 } else { - sortedArr(len / 2) + val sortedArr = arr.toSeq.sorted + val len = sortedArr.size + val med = if (len % 2 == 0) { + (sortedArr(len / 2) + sortedArr(len / 2 - 1)) / 2 + } else { + sortedArr(len / 2) + } + StatisticsMetrics(sortedArr.head, med, sortedArr(len - 1), sortedArr.sum) } - StatisticsMetrics(sortedArr.head, med, sortedArr(len - 1), sortedArr.sum) } def maxWithEmptyHandling(arr: Iterable[Long]): Long = { From fc49eb6af1f864095fa9c585ec9ccc6ec384c16f Mon Sep 17 00:00:00 2001 From: Lee Yang Date: Wed, 27 Nov 2024 14:58:51 -0800 Subject: [PATCH 06/52] update models for latest code and dataset JSON (#1442) Signed-off-by: Lee Yang --- .../qualx/models/xgboost/combined.cfg | 2 +- .../qualx/models/xgboost/combined.json | 2 +- .../qualx/models/xgboost/combined.metrics | 176 +++++++------- .../qualx/models/xgboost/databricks-aws.json | 2 +- .../qualx/models/xgboost/databricks-azure.cfg | 2 +- .../models/xgboost/databricks-azure.json | 2 +- .../models/xgboost/databricks-azure.metrics | 204 ++++++++--------- .../xgboost/databricks-azure_photon.cfg | 2 +- .../xgboost/databricks-azure_photon.json | 2 +- .../xgboost/databricks-azure_photon.metrics | 212 ++++++++--------- .../qualx/models/xgboost/dataproc.cfg | 2 +- .../qualx/models/xgboost/dataproc.json | 2 +- .../qualx/models/xgboost/dataproc.metrics | 214 +++++++++--------- .../resources/qualx/models/xgboost/emr.json | 2 +- .../qualx/models/xgboost/emr.metrics | 118 +++++----- .../resources/qualx/models/xgboost/onprem.cfg | 2 +- .../qualx/models/xgboost/onprem.json | 2 +- .../qualx/models/xgboost/onprem.metrics | 212 ++++++++--------- 18 files changed, 580 insertions(+), 580 deletions(-) diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg index 9e52614e1..f84abdfcb 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"93"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0499666668","gamma":"0.00690161297","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0499666668","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00690161297","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.768415093"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0478905365","gamma":"0.100665279","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0478905365","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.100665279","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.799700022"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json index 630bdbde5..d08771d24 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"93"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-5.3942828E-3,-6.013364E-1,2.8103882E-1,-9.758406E-1,-2.5355244E-1,6.4240746E-2,7.559256E-1,-1.1340183E0,-5.623425E-1,-4.2454568E-1,4.6896246E-1,-4.189193E-1,1.641902E-1,-1.6076164E-2,8.64832E-1,-1.2221249E0,-6.753662E-1,-6.7967623E-1,-2.8814146E-1,-4.6328908E-1,1.1505639E-1,1.8659306E-1,6.7526746E-1,-6.660216E-1,-2.9592174E-1,3.488807E-1,-1.0145351E-1,-3.5760877E-1,3.2592925E-1,7.363453E-1,1.2714392E0,-1.0873616E0,-1.7580476E0,-1.9639263E-1,-8.668144E-1,-7.515067E-1,-4.3738413E-1,-3.5920876E-1,5.7570897E-3,-8.137134E-1,-2.995749E-1,-8.7252796E-2,2.9527426E-1,-2.838043E-3,7.4468756E-1,-3.650947E-2,-5.034992E-1,-7.4428156E-2,-3.7491515E-1,2.689827E-1,8.24014E-1,-2.6542816E-1,1.3512862E-1,-4.9546224E-1,-2.4327647E-2,-1.2075904E-2,4.3212655E-1,6.4796597E-1,1.255649E0,6.6986495E-1,1.4126163E0,-1.1479855E0,-7.217265E-1,-9.139684E-2,-4.8923936E-2,1.0711046E-2,-2.1146255E-2,-9.344209E-1,-1.4381034E-2,-4.0184088E-2,-5.0718164E-1,-5.308838E-1,-2.5337273E-3,-2.4033272E-1,-2.3087729E-2,-5.668785E-1,-1.0246204E0,-4.8323047E-1,-8.4527425E-2,-1.31485E-2,2.5407795E-3,4.1870114E-1,1.1638447E-1,4.0257087E-1,8.7951905E-1,-1.491089E-2,-2.7294973E-2,-2.171636E-1,8.830168E-2,-2.4232392E-1,-4.366045E-1,2.3298116E-2,3.5622793E-1,7.97453E-2,6.819368E-1,-1.9126049E-1,-8.837529E-1,4.835528E-2,3.3227056E-2,-5.72348E-1,-1.3498356E-2,-7.772946E-3,3.6419206E-3,9.3160905E-2,6.0562634E-1,4.2685813E-1,7.4309343E-1,7.8872633E-1,1.5314466E0,8.293188E-1,-3.4993028E-3,1.4620222E0,2.4258742E-2,-1.1866535E0,-6.51686E-1,-1.0786903E-2,-7.912455E-1,-1.0071902E0,-2.375808E-2,-1.3285383E-2,-2.9224597E-2,-6.962585E-1,-1.3914558E-2,-2.8427798E-1,-4.3369858E-3,-7.1367776E-1,-3.3425784E-1,-1.0983397E0,-1.0740288E-2,-3.9486137E-1,-9.062732E-1,-1.20240636E-1,2.7411183E-2,1.1336014E-2,2.3376042E-2,-1.251354E-3,1.104175E-2,2.4829738E-2,1.3475447E-2,9.750247E-1,2.130706E-2,-3.005614E-3,-2.5265688E-1,-2.6049752E-2,2.0554753E-1,-4.2170128E-1,-1.6088003E-1,-4.696368E-1,-6.7098485E-3,4.616749E-2,-4.3335546E-2,1.9609366E-1,5.290399E-1,4.503203E-1,6.055764E-2,-2.7695972E-1,1.729231E-1,-5.009728E-2,-2.5275452E-2,9.977265E-3,4.579915E-2,-1.5294177E-2,-3.151543E-2,-4.4416063E-3,1.6459117E-2,3.4666564E-2,1.0992598E-2,1.3396789E-1,5.735013E-1,8.0252916E-1,1.5863493E-1,9.6363527E-1,9.536046E-3,8.705642E-2,1.1903944E0,1.903923E-2,5.0311774E-2,1.5150632E0,3.841186E-2,-1.2033799E0,-2.3030318E-2,-1.570208E-2,-3.7736572E-2,-8.775817E-1,-1.2647421E-2,-4.0247016E-2,-7.553468E-2,-2.0691983E-2,-4.1929506E-2,-5.426505E-3,-1.7915068E-2,-5.100176E-1,-4.778995E-2,-2.1433203E-1,-2.4167344E-2,-3.0077217E-2,-1.1621801E0,-4.523634E-1,2.413416E-2,-9.5655685E-3,-1.1176939E0,-7.608647E-2,-4.5241535E-1,2.6125893E-2,5.3289175E-2,-1.599842E-2,-7.250623E-3,8.950633E-3,-9.628226E-3,3.6305587E-3,1.4856386E-2,-2.5356526E-2,-1.1714184E-2,-8.844902E-2,-1.1039417E-2,-5.059151E-1,-2.823641E-1,-5.531336E-1,2.2540683E-2,1.275454E-1,4.2065048E-1,6.9868994E-1,3.9950383E-1,5.416455E-1,-3.6046997E-2,-6.984102E-1,-2.0389739E-1,-2.4835081E-1,3.3566856E-1,-4.2017713E-1,5.846262E-2,2.7536598E-1,-1.5310653E-2,7.0599824E-1,2.2950846E-1,9.1421825E-1,4.9603498E-1,-9.459799E-3,3.461365E-1,2.418417E-2,5.4122783E-2,3.267176E-2,6.625834E-2,6.279618E-2,8.179622E-2,-6.0725357E-2,-3.1163009E-2,-4.950091E-2,-2.4668831E-2,-1.4232031E-2,-2.9582268E-2,-1.30415205E-2,-3.903438E-3,-6.0272504E-2,-3.3710252E-2,-2.7234368E-2,-1.1910553E-2,-6.831992E-2,-3.4834977E-2,-8.882006E-3,6.6451663E-3,-1.0973817E-2,-3.6695544E-2,-8.787243E-3,-1.501925E-3,-1.729637E-2,-2.8625073E-2,-4.687908E-3,-2.0168293E-2,-1.6089858E-2,-4.035445E-2,-6.9141556E-3,8.5827205E-3,-3.554454E-3,9.056692E-3,5.8022114E-3,2.2395078E-2,2.8782705E-2,4.451331E-2,6.41355E-3,2.1380223E-2,1.1977256E-2,3.0170463E-2,-2.347332E-2,2.0231254E-2,-1.855634E-2,-4.855372E-2,-1.2787229E-2,8.1318375E-3,-3.379269E-2,2.506536E-3,2.1654505E-2,6.391953E-4,-3.762228E-2,-1.0581651E-2,1.3546817E-4,2.0446828E-2,6.2884954E-3,1.8359708E-2,-9.69795E-3,3.7651935E-3,4.086767E-2,1.4278957E-2,1.7789224E-2,6.297824E-3,4.9513184E-2,3.8168367E-2,1.8047953E-2,4.1098896E-2,2.7615251E-2,-1.0672044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,83,-1,85,87,89,91,93,95,97,99,101,-1,103,105,107,109,111,113,115,-1,-1,-1,-1,117,-1,-1,119,121,-1,123,-1,125,127,129,131,-1,-1,133,135,137,139,-1,-1,141,143,145,147,149,151,-1,153,155,157,159,-1,161,-1,-1,-1,163,165,167,169,171,173,175,-1,177,-1,179,181,-1,183,185,-1,-1,-1,187,-1,189,-1,191,193,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,-1,203,-1,-1,205,207,209,211,213,215,-1,-1,217,219,221,223,-1,225,227,-1,-1,229,-1,-1,-1,-1,-1,-1,-1,231,233,235,237,239,-1,-1,241,-1,-1,243,-1,245,-1,-1,-1,247,-1,-1,-1,-1,-1,-1,-1,249,-1,251,-1,-1,253,255,-1,-1,257,259,261,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,263,-1,265,267,269,271,273,275,277,279,281,283,285,287,289,291,293,295,297,299,301,303,305,307,-1,309,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3066824E2,1.0642453E2,1.7538599E2,2.5132324E1,5.2829338E1,5.661823E1,4.491043E1,1.0651154E1,3.3753662E0,3.7519955E1,4.6508465E0,5.9118958E0,4.7674477E1,7.9425573E0,2.3513794E1,1.520517E1,4.2312965E0,1.0114021E0,1.0446873E0,1.9319199E1,0E0,1.097192E0,2.4308434E0,3.2562447E-1,2.3516245E0,2.1530693E1,1.5514461E1,1.5500417E0,2.1539042E0,1.561705E1,8.480804E0,3.4273987E0,7.568512E-1,1.4882038E0,1.0619678E0,3.39077E-1,6.961608E-1,2.557571E-1,0E0,5.0415573E0,9.200907E0,2.8058374E-1,5.253446E-1,0E0,1.6294384E0,0E0,2.9931545E-2,8.7858665E-1,7.121477E-1,1.0537621E1,7.905197E0,1.0653864E1,7.508132E0,2.6114607E-1,1.5167607E-1,0E0,1.6490827E0,6.161087E0,5.359688E0,2.7716093E0,3.3668365E0,2.354782E0,8.841591E-1,0E0,0E0,0E0,0E0,5.2301025E-1,0E0,0E0,8.822322E-2,4.7397614E-1,0E0,8.327961E-2,0E0,1.5371742E0,3.166153E0,4.35252E0,2.5305216E0,0E0,0E0,3.5235167E-2,1.8305132E-1,4.151821E-2,8.8241386E-1,0E0,0E0,9.6259E-2,2.4668464E-1,4.2886925E-1,6.060133E-1,7.841962E0,9.941608E0,0E0,8.170353E0,6.6644645E0,6.157913E-1,4.722477E0,0E0,7.119942E-2,0E0,0E0,0E0,5.023714E-1,4.933629E-1,4.003454E0,7.3065643E0,2.0121508E0,1.55159E-1,1.1289043E0,0E0,1.9241333E0,0E0,1.2661896E0,2.518406E-1,0E0,9.858265E-1,1.3265171E0,0E0,0E0,0E0,1.5336037E-2,0E0,1.1761457E-1,0E0,1.0448494E0,2.9053664E-1,9.4830704E-1,0E0,5.5692806E0,2.9597778E0,1.4909346E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3497658E-1,0E0,0E0,6.973314E-2,3.764391E-1,8.81626E-2,4.1638374E-2,8.920562E-2,2.6478386E-1,0E0,0E0,4.109664E0,2.871015E0,3.5446548E0,2.2848234E0,0E0,5.1538486E0,2.8943405E0,0E0,0E0,2.8818626E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.007701E-1,2.709381E0,6.187538E0,1.4356241E0,3.9311314E-1,0E0,0E0,2.2467613E-1,0E0,0E0,3.6820984E-1,0E0,2.5239563E-1,0E0,0E0,0E0,4.6603012E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8230677E-1,0E0,6.861234E-2,0E0,0E0,1.5312195E-1,1.8535271E0,0E0,0E0,7.2629356E-1,2.000584E0,6.2643766E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1067022E-2,0E0,3.2592583E-1,2.3650855E-1,5.6340075E-1,2.6412094E0,1.5815396E0,3.17101E-1,1.2310143E0,7.2245026E-1,6.623955E-1,1.9114697E0,1.8736963E0,2.864295E0,1.5669336E0,9.4090724E-1,8.114495E-1,2.4236658E0,1.8777943E-1,2.9017568E-1,1.8823414E0,2.0383835E-1,8.690796E-1,2.0801182E0,0E0,1.0744218E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,67,67,70,70,71,71,73,73,75,75,76,76,77,77,78,78,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,99,99,103,103,104,104,105,105,106,106,107,107,108,108,109,109,111,111,113,113,114,114,116,116,117,117,121,121,123,123,125,125,126,126,127,127,129,129,130,130,131,131,139,139,142,142,143,143,144,144,145,145,146,146,147,147,150,150,151,151,152,152,153,153,155,155,156,156,159,159,167,167,168,168,169,169,170,170,171,171,174,174,177,177,179,179,183,183,191,191,193,193,196,196,197,197,200,200,201,201,202,202,213,213,215,215,216,216,217,217,218,218,219,219,220,220,221,221,222,222,223,223,224,224,225,225,226,226,227,227,228,228,229,229,230,230,231,231,232,232,233,233,234,234,235,235,236,236,238,238],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,84,-1,86,88,90,92,94,96,98,100,102,-1,104,106,108,110,112,114,116,-1,-1,-1,-1,118,-1,-1,120,122,-1,124,-1,126,128,130,132,-1,-1,134,136,138,140,-1,-1,142,144,146,148,150,152,-1,154,156,158,160,-1,162,-1,-1,-1,164,166,168,170,172,174,176,-1,178,-1,180,182,-1,184,186,-1,-1,-1,188,-1,190,-1,192,194,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,-1,204,-1,-1,206,208,210,212,214,216,-1,-1,218,220,222,224,-1,226,228,-1,-1,230,-1,-1,-1,-1,-1,-1,-1,232,234,236,238,240,-1,-1,242,-1,-1,244,-1,246,-1,-1,-1,248,-1,-1,-1,-1,-1,-1,-1,250,-1,252,-1,-1,254,256,-1,-1,258,260,262,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,264,-1,266,268,270,272,274,276,278,280,282,284,286,288,290,292,294,296,298,300,302,304,306,308,-1,310,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.4166665E0,3.443721E6,1E0,1.5560952E0,4.16E2,3.7609413E-1,5.012E3,7.804304E-1,2.14099E5,1.289627E3,7.3E1,2E1,2.0261088E-1,2.3314082E6,4.0844156E7,4.0844156E7,1.8441667E2,2.3871907E2,1E0,1.1505639E-1,2.1924436E5,1E0,4.1E1,1.5043668E0,2.732E3,1.023747E6,4.88954E5,1E0,6.7652373E3,3.171E3,1.5E1,7.621583E2,1.6666666E0,3.4409692E3,1.5669752E-2,1E0,2.21875E0,5.7570897E-3,1.5445488E8,3.4976208E-1,7.5569354E2,1.4187837E-1,-2.838043E-3,1.948181E0,-3.650947E-2,1.1772152E0,3.5E2,5.4444447E0,3.2627738E5,4.277001E3,3.8879236E7,1.5963264E9,1.6E1,2.3175376E3,-1.2075904E-2,4.7121394E-2,9.923929E2,2.1150263E-1,1.72797E5,5.2409735E-7,7.5569354E2,3.188889E1,-9.139684E-2,-4.8923936E-2,1.0711046E-2,-2.1146255E-2,2.203288E2,-1.4381034E-2,-4.0184088E-2,2.5327688E-1,1.5958088E10,-2.5337273E-3,2.6883545E2,-2.3087729E-2,1.221875E1,5.7950187E-1,7.218466E7,2.5016512E12,-1.31485E-2,2.5407795E-3,6.2827226E-3,5.294838E2,2.5757682E-1,3.07125E0,-1.491089E-2,-2.7294973E-2,3.9333332E1,1.4925373E-1,9.204625E-4,1.8122449E2,3.92382E5,9.27E2,7.97453E-2,2.7371603E-1,8.594445E5,7.868429E-2,6.08767E3,3.3227056E-2,2.8454319E1,-1.3498356E-2,-7.772946E-3,3.6419206E-3,2.3357454E8,2.7485228E6,1E0,3.5301748E7,1.3651616E5,8.2474226E-1,3.5677967E0,-3.4993028E-3,4.4263797E0,2.4258742E-2,3.30399E6,6.9966E4,-1.0786903E-2,4.4444447E0,1.0017953E8,-2.375808E-2,-1.3285383E-2,-2.9224597E-2,1.19216E8,-1.3914558E-2,2.5040618E7,-4.3369858E-3,5.116254E-1,5.59E2,2.62888E-3,-1.0740288E-2,1.6140062E3,2.060792E8,7.218466E7,2.7411183E-2,1.1336014E-2,2.3376042E-2,-1.251354E-3,1.104175E-2,2.4829738E-2,1.3475447E-2,4.7233105E6,2.130706E-2,-3.005614E-3,1.85209E5,1.7E1,5.635115E6,9.21641E3,1.44E0,3.30399E6,-6.7098485E-3,4.616749E-2,1.0853132E4,3.1100148E6,3.5419354E0,1.3793921E3,6.055764E-2,1E0,1.5120704E-4,-5.009728E-2,-2.5275452E-2,1.2E1,4.579915E-2,-1.5294177E-2,-3.151543E-2,-4.4416063E-3,1.6459117E-2,3.4666564E-2,1.0992598E-2,1.136E3,1E0,1.0033929E1,4.470588E0,1.381E3,9.536046E-3,8.705642E-2,8.848604E1,1.903923E-2,5.0311774E-2,5.964E3,3.841186E-2,1.3439535E1,-2.3030318E-2,-1.570208E-2,-3.7736572E-2,6.1E1,-1.2647421E-2,-4.0247016E-2,-7.553468E-2,-2.0691983E-2,-4.1929506E-2,-5.426505E-3,-1.7915068E-2,1.2342285E1,-4.778995E-2,1.6E1,-2.4167344E-2,-3.0077217E-2,4.8083666E-1,2.155E3,2.413416E-2,-9.5655685E-3,9.075E3,5.460954E9,2.603E3,2.6125893E-2,5.3289175E-2,-1.599842E-2,-7.250623E-3,8.950633E-3,-9.628226E-3,3.6305587E-3,1.4856386E-2,-2.5356526E-2,-1.1714184E-2,1.56E2,-1.1039417E-2,1E0,4.2285586E3,5.4468E4,1.273801E6,1E0,1.198E5,4.198242E2,2.691875E2,1E0,7.57657E4,4.4E1,1.2817779E7,4.230769E0,4.552196E6,3.0795444E3,2.5682794E5,5.23E2,7.940488E2,2.983871E0,6E0,5.2107143E0,1.3359244E9,-9.459799E-3,2.6422684E9,2.418417E-2,5.4122783E-2,3.267176E-2,6.625834E-2,6.279618E-2,8.179622E-2,-6.0725357E-2,-3.1163009E-2,-4.950091E-2,-2.4668831E-2,-1.4232031E-2,-2.9582268E-2,-1.30415205E-2,-3.903438E-3,-6.0272504E-2,-3.3710252E-2,-2.7234368E-2,-1.1910553E-2,-6.831992E-2,-3.4834977E-2,-8.882006E-3,6.6451663E-3,-1.0973817E-2,-3.6695544E-2,-8.787243E-3,-1.501925E-3,-1.729637E-2,-2.8625073E-2,-4.687908E-3,-2.0168293E-2,-1.6089858E-2,-4.035445E-2,-6.9141556E-3,8.5827205E-3,-3.554454E-3,9.056692E-3,5.8022114E-3,2.2395078E-2,2.8782705E-2,4.451331E-2,6.41355E-3,2.1380223E-2,1.1977256E-2,3.0170463E-2,-2.347332E-2,2.0231254E-2,-1.855634E-2,-4.855372E-2,-1.2787229E-2,8.1318375E-3,-3.379269E-2,2.506536E-3,2.1654505E-2,6.391953E-4,-3.762228E-2,-1.0581651E-2,1.3546817E-4,2.0446828E-2,6.2884954E-3,1.8359708E-2,-9.69795E-3,3.7651935E-3,4.086767E-2,1.4278957E-2,1.7789224E-2,6.297824E-3,4.9513184E-2,3.8168367E-2,1.8047953E-2,4.1098896E-2,2.7615251E-2,-1.0672044E-3],"split_indices":[20,56,50,102,42,2,42,2,39,2,4,2,3,42,28,45,45,52,52,16,0,28,16,2,53,2,1,32,8,52,2,3,52,53,52,38,8,53,0,7,39,4,27,0,42,0,53,2,58,28,47,45,7,8,4,0,38,52,38,1,37,4,52,0,0,0,0,57,0,0,42,31,0,4,0,58,39,45,31,0,0,57,52,27,57,0,0,50,56,38,52,5,2,0,35,28,57,4,0,56,0,0,0,32,28,100,45,28,57,56,0,42,0,9,29,0,56,45,0,0,0,5,0,5,0,27,0,27,0,52,7,45,0,0,0,0,0,0,0,45,0,0,12,0,5,28,58,9,0,0,28,28,54,4,0,102,38,0,0,18,0,0,0,0,0,0,0,29,6,53,54,0,0,0,56,0,0,2,0,54,0,0,0,3,0,0,0,0,0,0,0,56,0,8,0,0,42,0,0,0,2,5,0,0,0,0,0,0,0,0,0,0,0,2,0,8,33,1,9,102,7,52,55,100,28,10,9,56,12,4,28,29,4,53,8,53,7,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.521E3,8.18E2,1.703E3,3.93E2,4.25E2,1.17E3,5.33E2,2.83E2,1.1E2,3.44E2,8.1E1,2E2,9.7E2,6.6E1,4.67E2,2.36E2,4.7E1,7.6E1,3.4E1,3.4E2,4E0,3.5E1,4.6E1,6.5E1,1.35E2,5.72E2,3.98E2,3.3E1,3.3E1,3.57E2,1.1E2,1.91E2,4.5E1,1.4E1,3.3E1,5.7E1,1.9E1,2.9E1,5E0,1.07E2,2.33E2,1E1,2.5E1,4E0,4.2E1,4.4E1,2.1E1,3.6E1,9.9E1,4.91E2,8.1E1,2.35E2,1.63E2,2.3E1,1E1,5E0,2.8E1,3.07E2,5E1,2.2E1,8.8E1,1.62E2,2.9E1,4E1,5E0,5E0,9E0,2.9E1,4E0,4.5E1,1.2E1,1.5E1,4E0,1.5E1,1.4E1,5.1E1,5.6E1,1.25E2,1.08E2,4E0,6E0,1.4E1,1.1E1,1.3E1,2.9E1,5E0,1.6E1,1.9E1,1.7E1,3.3E1,6.6E1,1.29E2,3.62E2,1.1E1,7E1,2.11E2,2.4E1,1.41E2,2.2E1,1.6E1,7E0,4E0,6E0,1E1,1.8E1,9.4E1,2.13E2,2E1,3E1,1.8E1,4E0,8.3E1,5E0,1.49E2,1.3E1,4E0,2.5E1,2.4E1,5E0,4E0,8E0,8E0,7E0,1.1E1,4E0,3E1,2.1E1,5.1E1,5E0,1.05E2,2E1,1.03E2,5E0,4E0,1E1,5E0,6E0,6E0,7E0,2.3E1,6E0,4E0,1.5E1,9E0,8E0,9E0,2.4E1,5.9E1,7E0,8E0,1.21E2,1.89E2,1.73E2,5E1,2E1,1.71E2,4E1,1.7E1,7E0,1.36E2,5E0,4E0,1.2E1,6E0,4E0,1.4E1,4E0,3.2E1,6.2E1,1.93E2,2E1,1.5E1,5E0,1.6E1,1.4E1,6E0,1.2E1,7.6E1,7E0,1.45E2,4E0,4E0,9E0,2.1E1,4E0,1.9E1,5E0,4E0,4E0,4E0,7E0,1.8E1,1.2E1,1.3E1,8E0,7E0,4.4E1,9.9E1,6E0,5E0,1.5E1,9.2E1,1.1E1,5E0,1.8E1,8E0,7E0,4E0,5E0,4E0,4E0,5E0,4E0,1.2E1,1.2E1,4.8E1,1.1E1,1.3E1,1.08E2,1.46E2,4.3E1,7.3E1,1E2,4.2E1,8E0,2.4E1,1.47E2,1.1E1,2.9E1,1.3E1,1.23E2,1.6E1,1.6E1,4.4E1,1.8E1,1.4E2,5.3E1,7E0,1.3E1,4E0,1.1E1,4E0,1E1,2.8E1,4.8E1,1.41E2,4E0,1.5E1,6E0,6E0,1.2E1,9E0,4E0,3.9E1,5E0,6.8E1,3.1E1,8E0,7E0,6.2E1,3E1,7E0,4E0,4E0,8E0,1.6E1,3.2E1,5E0,6E0,8E0,5E0,5.2E1,5.6E1,3.1E1,1.15E2,4E0,3.9E1,4.7E1,2.6E1,1E1,9E1,8E0,3.4E1,4E0,4E0,1.2E1,1.2E1,1.29E2,1.8E1,4E0,7E0,2.2E1,7E0,4E0,9E0,1.07E2,1.6E1,7E0,9E0,5E0,1.1E1,3.4E1,1E1,7E0,1.1E1,8.9E1,5.1E1,3.9E1,1.4E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"311","size_leaf_vector":"1"}},{"base_weights":[-5.630656E-3,-5.5759704E-1,2.7148098E-1,-8.9754194E-1,-2.4868973E-1,6.172165E-2,7.2123635E-1,-1.0647084E0,-4.941926E-1,-4.1144085E-1,4.3103302E-1,-1.9087502E-1,2.5325418E-1,5.8200467E-1,1.2323364E0,-1.1401552E0,-5.8053803E-1,-6.477654E-1,-2.5218076E-1,-4.5909762E-1,1.048429E-1,1.371317E-1,6.3186604E-1,-3.8715476E-1,4.260691E-2,2.2370598E-1,9.231099E-2,-1.3745931E-1,6.641594E-1,9.9192446E-1,1.4899961E0,-1.0132887E0,-1.6779944E0,-1.0195744E-1,-8.2491475E-1,-7.053305E-1,-3.5067707E-1,-4.316483E-1,-1.3404965E-1,-3.651817E-1,-1.0340726E0,-2.4366109E-2,3.2856244E-1,-1.0679361E-3,7.1753573E-1,-6.5257424E-1,-2.4344687E-1,7.891385E-2,-1.9326637E-2,2.7872697E-1,-5.479206E-1,1.16478436E-1,-5.625614E-1,5.5923593E-1,1.0041931E0,1.0504472E0,1.30355E-2,7.689651E-2,3.5096355E-2,-1.0993186E0,-7.602385E-1,-8.6834185E-2,-4.9435407E-2,1.2900729E-2,-1.852469E-2,-9.2577267E-1,-2.021132E-2,-6.001386E-1,-4.9362563E-2,-5.367929E-3,-2.1847067E-2,-2.3569373E-2,-1.1250385E-2,-2.010192E-1,1.2711309E-2,-5.1436806E-1,-1.4787339E-1,-7.085087E-2,-8.7552947E-1,9.016656E-3,-1.18324146E-1,2.3026355E-2,1.597836E-1,3.45914E-1,8.385224E-1,-5.805932E-1,-8.820019E-1,2.7177693E-2,-3.3120173E-1,-8.4264606E-2,3.5907993E-1,9.1681056E-2,4.452843E-1,-6.8850476E-1,-2.588667E-1,1.4407126E-3,2.784224E-2,-3.30202E-2,-1.6376605E-2,4.574619E-1,8.1735593E-1,8.1299114E-1,1.4961236E0,1.0968724E0,2.3248635E-2,-1.1152937E0,-2.0635244E-2,-1.0567457E-2,-8.108836E-1,-2.0276671E-2,-5.5298436E-2,-3.7932135E-2,-3.8493842E-1,-2.4408612E-1,-4.7987822E-4,-3.705656E-1,-7.4866825E-1,-1.01520754E-1,-4.9029526E-1,-2.8395997E-2,-9.5128036E-1,-1.8895203E-1,3.0340762E-3,1.2496798E-2,1.8741395E-3,8.104553E-3,2.0463493E-2,5.2057955E-2,5.739056E-1,-3.541497E-2,-3.2172182E-1,-5.3019263E-2,-5.788927E-1,-1.7459853E-2,2.271292E-1,-3.598957E-1,1.751252E-2,1.7603628E-1,-1.2792619E-1,5.128303E-1,8.3092436E-2,-1.2345891E-2,2.894557E-1,6.307022E-1,2.91562E-1,-4.4843727E-1,-4.5653977E-2,-2.0774854E-2,-9.063033E-2,1.0328839E-1,-1.0751113E-2,-1.9897016E-2,4.796256E-1,9.3208474E-1,-1.8331159E-2,3.7237048E-1,8.837602E-1,5.959527E-2,9.3403004E-2,7.525226E-1,1.1899731E0,-4.0724743E-2,-1.1459788E0,-8.6237127E-1,-1.8557923E-2,-2.183687E-2,-6.017266E-3,-2.9242942E-1,-3.3194069E-3,-4.646442E-1,1.0424385E-1,-9.2320687E-1,-3.3382645E-1,-1.4877209E-1,2.7139695E-2,-4.452982E-3,-6.280758E-1,-5.0199594E-2,-2.4689406E-2,-3.7228619E-3,-1.2177146E-2,3.6182925E-1,3.8337562E-2,4.6821013E-3,-3.98044E-1,-1.295178E-2,-3.7415747E-2,4.1009482E-2,1.0423816E-1,-4.7038126E-1,-1.986724E-1,2.271275E-2,6.1550073E-2,-2.991985E-1,-2.4591466E-2,6.661538E-1,9.2417636E-4,1.4842626E-1,-2.554643E-3,-2.1233332E-1,8.541809E-2,-8.178016E-3,3.253218E-1,5.511337E-1,4.5969125E-2,3.765792E-1,8.2340375E-2,-2.845326E-2,-3.4143094E-3,-7.1048248E-3,-1.046467E-3,2.3821631E-2,1.4452573E-2,2.8476605E-1,5.6442773E-1,4.9872782E-2,5.5052304E-1,9.755781E-3,-1.5673175E-2,2.316738E-2,8.499117E-3,9.952715E-1,3.1364113E-2,1.2866532E-2,4.6208195E-2,3.454456E-2,6.2031206E-2,-5.9248462E-2,-4.48097E-2,-4.592417E-2,-2.1547625E-2,-1.6694909E-2,-6.273093E-3,-2.8545562E-2,-1.5242072E-2,2.1274926E-2,-1.3751732E-4,-3.3147506E-2,-5.2813888E-2,-7.717381E-3,-3.9294798E-2,-1.7899696E-2,-3.8531888E-3,-3.7404962E-2,-1.6675325E-2,2.4335762E-2,9.852534E-3,-2.4888156E-2,-1.2172674E-2,-1.4700858E-2,9.496526E-3,-2.5689356E-2,-1.0765562E-2,-1.29311085E-2,1.2519918E-2,-2.6623963E-3,9.653325E-3,-1.6732035E-2,-7.6448107E-3,-3.213851E-3,7.821097E-3,3.90427E-2,1.7183011E-2,1.1104388E-5,1.33383535E-2,-6.864249E-3,-2.3227973E-2,-1.5178634E-2,6.0297847E-3,1.808273E-2,7.636125E-3,2.9557696E-2,9.10959E-3,1.6024638E-2,4.0205352E-2,-5.051845E-3,1.0782224E-2,-4.176458E-3,5.0665606E-3,1.9211315E-2,3.390559E-3,2.2942364E-2,4.026906E-2,1.0629563E-2,3.6728002E-2,3.3810798E-2,5.3505078E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,-1,89,91,93,95,97,99,101,103,-1,-1,-1,105,107,-1,-1,-1,-1,109,-1,111,-1,-1,-1,-1,-1,113,-1,115,117,-1,119,-1,121,-1,123,125,127,129,131,133,135,137,139,141,143,145,147,149,-1,-1,-1,151,153,155,157,159,-1,161,-1,-1,163,-1,-1,-1,165,167,-1,169,171,173,175,-1,177,179,-1,-1,-1,-1,-1,-1,181,-1,183,-1,185,-1,187,189,-1,191,193,195,197,199,201,203,205,207,-1,-1,209,211,-1,-1,213,215,217,219,221,-1,-1,223,225,-1,227,229,-1,-1,-1,231,-1,233,235,237,239,241,-1,-1,243,-1,-1,-1,-1,245,-1,-1,247,-1,-1,-1,249,251,253,-1,255,257,259,261,-1,263,-1,265,267,-1,269,271,-1,273,275,-1,-1,-1,-1,277,-1,279,281,-1,283,-1,-1,-1,-1,285,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9447272E2,9.029669E1,1.6193793E2,2.7118683E1,5.0290764E1,5.675026E1,3.813687E1,9.89502E0,4.3709526E0,4.4738415E1,5.1075573E0,2.320535E1,3.1006195E1,2.5566559E1,5.7075806E0,1.5289917E1,4.711006E0,1.018198E0,9.809723E-1,1.9062126E1,0E0,1.1518846E0,2.9253998E0,1.0330421E1,2.2111053E1,2.8003624E1,0E0,4.923252E0,1.3245621E1,2.4212494E0,1.0013428E0,3.6053162E0,1.7640686E-1,1.5477324E0,7.703953E-1,1.2203445E0,2.4778974E-1,5.0935745E-2,8.575727E-1,1.0006378E1,1.8016891E0,4.2548525E-1,3.0588603E-1,0E0,1.7781639E0,1.0733032E0,4.2837534E0,0E0,5.5345993E0,1.90565E1,1.6097698E0,1.46954E0,1.7573261E-1,7.454521E0,7.248886E0,1.0652962E0,0E0,0E0,0E0,1.1840668E0,1.357029E0,0E0,0E0,0E0,0E0,1.5265732E0,0E0,1.3160686E0,0E0,0E0,0E0,0E0,0E0,2.2023475E-1,0E0,5.9220695E0,1.9818611E0,0E0,3.4376144E-1,0E0,2.0115829E-1,0E0,9.001376E-2,8.553696E-2,1.2314396E0,2.293789E0,5.2570724E-1,3.468013E0,2.786336E0,2.204628E0,1.3576832E0,5.970003E0,9.051041E0,1.1579885E0,3.8467908E-1,5.5168474E-1,0E0,0E0,0E0,4.287384E0,7.9749107E0,1.760128E0,4.8804855E-1,8.0365753E-1,0E0,4.111328E-1,0E0,0E0,8.110962E-1,0E0,0E0,0E0,2.5049877E-1,1.6951716E-1,0E0,5.2300863E0,4.6878204E0,3.5534568E0,7.7314043E-1,0E0,2.1129417E-1,5.0771058E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.1949272E-1,0E0,8.933666E-1,0E0,3.4911346E-1,0E0,2.1029196E0,2.2400303E0,0E0,8.648552E-1,2.9128747E0,1.5793338E0,1.2158644E-1,3.7524867E0,1.6784277E0,2.8536835E0,3.1529417E0,6.9701886E-1,0E0,0E0,2.936022E-2,2.4432404E-1,0E0,0E0,3.3259392E0,1.2898827E0,7.5510263E-1,1.2424779E-1,1.0028191E0,0E0,0E0,9.556627E-1,1.684494E-1,0E0,2.2528076E-1,6.797104E-1,0E0,0E0,0E0,7.633257E-2,0E0,1.4767609E0,6.892388E-1,9.6713257E-1,1.6645937E0,1.5783701E0,0E0,0E0,1.4040041E-1,0E0,0E0,0E0,0E0,1.1012018E-1,0E0,0E0,2.3402262E-1,0E0,0E0,0E0,9.449802E-1,7.4798393E-1,1.5503514E0,0E0,3.277571E-1,2.7205276E-1,7.584024E-1,3.2539988E-1,0E0,1.580253E-1,0E0,1.1286831E0,1.8068215E0,0E0,5.3142357E-1,1.6361198E0,0E0,2.7794151E0,1.3192742E0,0E0,0E0,0E0,0E0,1.16795234E-1,0E0,1.4014559E0,3.34169E0,0E0,6.539383E-1,0E0,0E0,0E0,0E0,1.900177E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,59,59,60,60,65,65,67,67,73,73,75,75,76,76,78,78,80,80,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,99,99,100,100,101,101,102,102,103,103,105,105,108,108,112,112,113,113,115,115,116,116,117,117,118,118,120,120,121,121,128,128,130,130,132,132,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,143,143,144,144,145,145,148,148,149,149,152,152,153,153,154,154,155,155,156,156,159,159,160,160,162,162,163,163,167,167,169,169,170,170,171,171,172,172,173,173,176,176,181,181,184,184,188,188,189,189,190,190,192,192,193,193,194,194,195,195,197,197,199,199,200,200,202,202,203,203,205,205,206,206,211,211,213,213,214,214,216,216,221,221],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,-1,90,92,94,96,98,100,102,104,-1,-1,-1,106,108,-1,-1,-1,-1,110,-1,112,-1,-1,-1,-1,-1,114,-1,116,118,-1,120,-1,122,-1,124,126,128,130,132,134,136,138,140,142,144,146,148,150,-1,-1,-1,152,154,156,158,160,-1,162,-1,-1,164,-1,-1,-1,166,168,-1,170,172,174,176,-1,178,180,-1,-1,-1,-1,-1,-1,182,-1,184,-1,186,-1,188,190,-1,192,194,196,198,200,202,204,206,208,-1,-1,210,212,-1,-1,214,216,218,220,222,-1,-1,224,226,-1,228,230,-1,-1,-1,232,-1,234,236,238,240,242,-1,-1,244,-1,-1,-1,-1,246,-1,-1,248,-1,-1,-1,250,252,254,-1,256,258,260,262,-1,264,-1,266,268,-1,270,272,-1,274,276,-1,-1,-1,-1,278,-1,280,282,-1,284,-1,-1,-1,-1,286,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.126489E7,1E0,1.5560952E0,2.091623E5,2.3920168E6,5.012E3,4.6816683E-1,1.3144558E4,1.3097832E3,2.857E3,3.7002478E3,1.5833762E-1,7.431E3,4.0844156E7,5.12861E5,1.998E3,2.028994E8,1E0,1.048429E-1,2.0783027E5,1E0,9.6E1,1.7E1,3.0820766E-1,9.231099E-2,8.657441E7,2.043739E0,1.4274633E7,1.8222668E2,1.1E1,1E0,1E0,2.203288E2,1.98714E5,9E0,2.8331464E2,2.695E3,4.8046228E-2,6.6992125E4,1.0275E4,3.178082E6,-1.0679361E-3,1.9032773E0,1.4504054E8,1.4286339E0,7.891385E-2,8.426E3,9.27E2,2.6476662E6,1.2502964E9,1.6213043E3,1.0693399E-1,8.015419E3,1E0,1.30355E-2,7.689651E-2,3.5096355E-2,3.097E3,1.0710737E2,-8.6834185E-2,-4.9435407E-2,1.2900729E-2,-1.852469E-2,5.5544252E7,-2.021132E-2,1.09E3,-4.9362563E-2,-5.367929E-3,-2.1847067E-2,-2.3569373E-2,-1.1250385E-2,1.5377966E6,1.2711309E-2,3.791269E7,8.229907E7,-7.085087E-2,2.3770695E5,9.016656E-3,1.249246E6,2.3026355E-2,1.5E1,1.5552E4,2.4489644E0,5.222222E1,8.874974E7,8.1445E1,2.1102592E8,5.107429E1,3.0161016E7,1.5859042E6,3.71E0,4.1E2,6.554713E4,1.0033929E1,2.784224E-2,-3.30202E-2,-1.6376605E-2,1.5964827E4,4.2620764E2,8.8543115E2,2.0455818E1,2.9251662E6,2.3248635E-2,2.0304577E-1,-2.0635244E-2,-1.0567457E-2,5.6903213E-1,-2.0276671E-2,-5.5298436E-2,-3.7932135E-2,9.280322E-1,2.04946E6,-4.7987822E-4,5.351875E2,4.0647778E1,4.77721E3,1.272E3,-2.8395997E-2,5.7950187E-1,1.7560185E0,3.0340762E-3,1.2496798E-2,1.8741395E-3,8.104553E-3,2.0463493E-2,5.2057955E-2,1.4777102E9,-3.541497E-2,4E0,-5.3019263E-2,5.9E1,-1.7459853E-2,2E0,1.273801E6,1.751252E-2,2.5870967E0,7.44393E5,2.773036E1,3.3E2,4.67E2,5.8631687E1,5.427071E2,2.3E1,1.415178E3,-4.5653977E-2,-2.0774854E-2,1.0573252E8,9.4615E4,-1.0751113E-2,-1.9897016E-2,5.704E3,2.6307288E7,2.3580047E10,4.1844E4,7.771383E0,5.959527E-2,9.3403004E-2,1.1902924E0,4.6253732E2,-4.0724743E-2,1.1164689E0,1E0,-1.8557923E-2,-2.183687E-2,-6.017266E-3,2.2178302E5,-3.3194069E-3,2.785857E2,8.628E3,1.683653E8,1.6622689E5,3.4614954E8,2.7139695E-2,-4.452982E-3,8.755996E-1,-5.0199594E-2,-2.4689406E-2,-3.7228619E-3,-1.2177146E-2,2.9E1,3.8337562E-2,4.6821013E-3,2.4222221E0,-1.295178E-2,-3.7415747E-2,4.1009482E-2,1.62848E5,3.7186194E5,3.8172052E2,2.271275E-2,7.364257E4,2.3258256E7,6.797394E8,1.7181714E7,9.2417636E-4,4.2E1,-2.554643E-3,2E0,2.86968E5,-8.178016E-3,3.502483E-2,4.5158855E6,4.5969125E-2,8.5023944E2,4.05E2,-2.845326E-2,-3.4143094E-3,-7.1048248E-3,-1.046467E-3,6.274915E-2,1.4452573E-2,9.1E2,3.202814E5,4.9872782E-2,3.29774E5,9.755781E-3,-1.5673175E-2,2.316738E-2,8.499117E-3,3.1790106E8,3.1364113E-2,1.2866532E-2,4.6208195E-2,3.454456E-2,6.2031206E-2,-5.9248462E-2,-4.48097E-2,-4.592417E-2,-2.1547625E-2,-1.6694909E-2,-6.273093E-3,-2.8545562E-2,-1.5242072E-2,2.1274926E-2,-1.3751732E-4,-3.3147506E-2,-5.2813888E-2,-7.717381E-3,-3.9294798E-2,-1.7899696E-2,-3.8531888E-3,-3.7404962E-2,-1.6675325E-2,2.4335762E-2,9.852534E-3,-2.4888156E-2,-1.2172674E-2,-1.4700858E-2,9.496526E-3,-2.5689356E-2,-1.0765562E-2,-1.29311085E-2,1.2519918E-2,-2.6623963E-3,9.653325E-3,-1.6732035E-2,-7.6448107E-3,-3.213851E-3,7.821097E-3,3.90427E-2,1.7183011E-2,1.1104388E-5,1.33383535E-2,-6.864249E-3,-2.3227973E-2,-1.5178634E-2,6.0297847E-3,1.808273E-2,7.636125E-3,2.9557696E-2,9.10959E-3,1.6024638E-2,4.0205352E-2,-5.051845E-3,1.0782224E-2,-4.176458E-3,5.0665606E-3,1.9211315E-2,3.390559E-3,2.2942364E-2,4.026906E-2,1.0629563E-2,3.6728002E-2,3.3810798E-2,5.3505078E-2],"split_indices":[20,56,12,102,42,28,28,2,41,52,4,2,52,42,2,45,29,10,7,104,0,33,16,29,0,27,0,45,41,45,58,3,105,75,57,1,3,4,2,38,33,9,45,0,42,31,53,0,2,2,32,7,4,35,4,6,0,0,0,2,4,0,0,0,0,45,0,2,0,0,0,0,0,33,0,45,45,0,28,0,1,0,3,9,42,4,7,52,7,48,45,28,54,11,33,53,0,0,0,33,58,52,54,28,0,27,0,0,39,0,0,0,27,9,0,52,56,52,0,0,39,39,0,0,0,0,0,0,7,0,10,0,0,0,10,9,0,53,1,58,3,2,47,52,3,52,0,0,7,10,0,0,2,43,5,9,53,0,0,53,4,0,39,59,0,0,0,28,0,52,9,7,33,12,0,0,42,0,0,0,0,3,0,0,53,0,0,0,7,47,52,0,28,12,7,12,0,8,0,6,7,0,38,47,0,52,10,0,0,0,0,41,0,11,28,0,11,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.577E3,8.61E2,1.716E3,4.09E2,4.52E2,1.171E3,5.45E2,2.88E2,1.21E2,3.65E2,8.7E1,5.05E2,6.66E2,4.3E2,1.15E2,2.48E2,4E1,7.3E1,4.8E1,3.59E2,6E0,3.6E1,5.1E1,2.74E2,2.31E2,6.55E2,1.1E1,4.4E1,3.86E2,6.2E1,5.3E1,2.03E2,4.5E1,1.4E1,2.6E1,6E1,1.3E1,1.8E1,3E1,3.1E2,4.9E1,2E1,1.6E1,6E0,4.5E1,9.5E1,1.79E2,8E0,2.23E2,6.12E2,4.3E1,2.8E1,1.6E1,2.97E2,8.9E1,5.7E1,5E0,4.9E1,4E0,1.49E2,5.4E1,4E1,5E0,6E0,8E0,2E1,6E0,4.6E1,1.4E1,4E0,9E0,1.4E1,4E0,2.6E1,4E0,1.83E2,1.27E2,1.2E1,3.7E1,6E0,1.4E1,8E0,8E0,1.2E1,3.3E1,7.5E1,2E1,4.4E1,1.35E2,1.91E2,3.2E1,2.89E2,3.23E2,2.8E1,1.5E1,2.3E1,5E0,1E1,6E0,2.15E2,8.2E1,6.6E1,2.3E1,5.2E1,5E0,1.45E2,4E0,5E0,4.9E1,6E0,1.4E1,2.5E1,2.1E1,2.1E1,5E0,1.15E2,6.8E1,1.13E2,1.4E1,9E0,2.8E1,1E1,4E0,4E0,4E0,4E0,8E0,1.7E1,1.6E1,4.9E1,2.6E1,1.1E1,9E0,1.5E1,2.9E1,1.3E2,5E0,2.7E1,1.64E2,2E1,1.2E1,1.9E2,9.9E1,1.45E2,1.78E2,1.5E1,1.3E1,7E0,8E0,1.6E1,7E0,5E0,2.1E2,7.2E1,1E1,1E1,5.6E1,1.5E1,8E0,1.3E1,3.9E1,1.6E1,1.29E2,4.3E1,6E0,1.7E1,4E0,1.6E1,5E0,9.6E1,1.9E1,4.7E1,2.1E1,1.06E2,7E0,4E0,1E1,2.4E1,4E0,4E0,6E0,9E0,7E0,4E0,2.2E1,4E0,5E0,4E0,2.5E1,7.6E1,5.4E1,7E0,2E1,6.1E1,1.03E2,1.5E1,5E0,8E0,4E0,6.2E1,1.28E2,7E0,9.2E1,1.16E2,2.9E1,1.26E2,5.2E1,1.1E1,4E0,4E0,4E0,1.2E1,4E0,6.5E1,1.45E2,6E1,1.2E1,6E0,4E0,6E0,4E0,3.7E1,1.9E1,4E0,9E0,5E0,3.4E1,1.08E2,2.1E1,3.7E1,6E0,1.2E1,4E0,5.6E1,4E1,4E0,1.5E1,1.8E1,2.9E1,1.6E1,5E0,2.6E1,8E1,6E0,4E0,4E0,5E0,1.2E1,1E1,4E0,2.1E1,6.4E1,1.2E1,4.8E1,6E0,1.1E1,9E0,4.8E1,1.3E1,8.5E1,1.8E1,1E1,5E0,4E0,4E0,4.9E1,1.3E1,1E1,1.18E2,7.5E1,1.7E1,1.04E2,1.2E1,1.13E2,1.3E1,2.2E1,3E1,5E0,7E0,4.4E1,2.1E1,1.03E2,4.2E1,5E0,7E0,9E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"287","size_leaf_vector":"1"}},{"base_weights":[1.0596133E-2,-5.4173553E-1,2.632191E-1,-8.31292E-1,-1.868726E-1,6.666947E-2,6.7344254E-1,-8.6302143E-1,1.05969965E-1,-3.969972E-1,3.1293085E-1,-1.7750679E-1,2.3998451E-1,-3.7204584E-1,7.404549E-1,-9.8205346E-1,-3.6301708E-1,-5.319442E-1,-2.5573653E-1,1.5420464E-1,7.02212E-1,-3.730432E-1,4.576528E-2,2.8342596E-1,-4.619917E-1,-5.106712E-1,-6.379135E-2,6.4811456E-1,1.216095E0,-7.510709E-1,-1.1986773E0,-1.16148144E-1,-6.71795E-1,-5.881084E-1,-6.2144592E-2,-3.2742506E-1,-4.161081E-2,3.3234209E-3,3.213075E-1,9.6438336E-1,5.690922E-1,-6.040635E-1,-2.5007832E-1,7.560434E-2,-6.6098683E-3,7.513296E-2,4.6259293E-1,-5.9088254E-1,1.2414789E-2,-2.913663E-2,-1.2562506E-2,-1.057034E-2,3.5961065E-3,5.666854E-1,1.0302479E0,1.287801E0,4.5952103E-1,-9.000284E-1,-4.674698E-1,-1.3034296E0,-6.2797636E-1,-2.589007E-1,2.5922617E-1,-7.774284E-1,-3.4183782E-1,-7.1642756E-1,-4.6878558E-1,-1.672725E-2,6.742472E-2,-3.9940238E-1,-1.9836774E-1,-1.6929434E-1,7.4197374E-2,-6.758923E-2,2.3700283E-1,1.3349754E-1,4.9220657E-1,5.8284517E-2,2.8452123E-2,1.3207421E-2,6.1934197E-1,-5.5349207E-1,-4.4513706E-2,2.1347922E-1,-3.049342E-1,-6.513718E-2,5.965519E-1,-2.1697854E-1,1.3884212E-1,3.5265034E-1,7.946858E-1,-7.201133E-1,-2.4345969E-1,6.049616E-1,-1.8031518E-1,1.2607119E0,5.7367975E-1,7.4124837E-1,1.3957775E0,1.2188469E-2,2.914085E-2,-1.4197066E-2,-9.380204E-1,-1.10355414E-1,-7.5831825E-1,-1.1827921E0,-1.754596E0,-7.0965195E-1,-1.2903138E-2,-7.552869E-2,-4.0182188E-1,4.188351E-2,3.208191E-2,-2.356723E-2,-8.974423E-1,-4.2657247E-3,-2.3793833E-2,-1.2125694E-2,-7.4947417E-1,-4.1233334E-1,-4.678677E-2,-6.3783093E-3,1.6341409E-2,-4.4928735E-1,-2.3413824E-1,1.8956681E-3,-2.3770551E-1,2.8665771E-3,-3.110298E-1,-2.5070135E-2,8.806648E-3,-2.6234657E-1,4.171428E-2,2.120104E-3,2.114036E-2,-1.3570258E-3,2.0973566E-1,2.9680872E-2,1.1633313E-2,1.6019335E-2,3.3859063E-2,-3.385638E-2,-3.354966E-1,3.8014825E-2,2.9346237E-2,-3.2614177E-1,2.002479E-2,-3.44579E-1,-1.9244194E-2,3.393286E-2,1.9378373E-2,-8.563553E-2,-2.734837E-1,1.5520108E-1,-3.4829695E-2,3.094661E-1,1.0575148E0,4.440335E-1,9.28916E-1,-1.6083665E-2,-3.9005864E-2,-2.394059E-3,-1.9908201E-2,6.79396E-1,3.596543E-1,-3.8155016E-2,2.6137233E-2,1.4873708E0,1.0041496E0,7.186714E-1,-1.1842902E-2,4.7214255E-2,1.8662479E-2,5.1064175E-2,1.5092423E0,-1.8572195E-2,-9.6828324E-1,1.2733825E-1,-3.741228E-1,-1.0449004E-2,-8.245741E-1,-1.2528073E0,-7.336137E-1,-5.796309E-2,-9.583962E-2,-4.622153E-2,-5.4413885E-1,-1.4650951E-2,3.9608266E-2,-2.3545818E-2,-8.061125E-3,9.797586E-3,-4.6766894E-3,-2.5552765E-2,-9.987876E-1,-6.782241E-1,-4.9901694E-2,-3.0614865E-1,-6.141783E-1,-9.035688E-3,-4.7106728E-1,-1.7179694E-2,-2.1983965E-3,-2.7239114E-1,-2.3875865E-3,-1.9314745E-2,-7.904549E-3,5.012538E-4,-3.1068467E-3,-1.7731788E-2,-3.089743E-3,2.0485146E-1,-6.6139214E-2,5.216782E-3,1.2239265E-2,7.7752606E-4,-4.2037284E-1,-9.989654E-3,1.9241841E-1,-2.4988624E-1,-5.563985E-1,-4.731587E-1,-9.116148E-2,-3.4342557E-2,2.5484381E-2,4.1598976E-2,-1.9158302E-1,-1.929812E-1,-1.8593261E-2,1.11775324E-1,3.7713495E-1,3.7168962E-1,2.8833479E-2,8.011824E-2,3.1189786E-2,3.100967E-2,2.9488707E-1,7.310067E-1,5.5745427E-2,5.05307E-1,8.42335E-1,2.8568438E-1,3.569526E-2,-1.20578386E-1,1.0544851E-2,5.420075E-2,8.4117725E-2,5.8136806E-2,3.5103376E-3,8.483583E-1,1.7650869E-2,8.407847E-2,6.0677916E-2,-5.0198916E-2,-2.9116927E-2,2.1883514E-2,-6.3457416E-4,-5.714141E-3,-2.526961E-2,-2.9476976E-2,-4.6437524E-2,-5.690455E-2,-7.493692E-2,-4.1455727E-2,-2.2506278E-2,-3.312151E-2,-1.1780705E-2,1.0985354E-2,-6.0232864E-3,-6.1964158E-2,-3.632225E-2,-3.53137E-2,-1.6280105E-2,-1.9165386E-2,-5.7890997E-3,-3.311543E-2,-1.7269786E-2,-2.4440184E-2,-1.2151474E-2,-1.9274484E-2,-1.0509291E-2,1.6967178E-3,1.6727623E-2,1.2456676E-3,-8.033132E-3,-2.4220945E-2,-3.2951776E-3,4.1720085E-3,1.3134104E-2,-1.46561E-2,5.835422E-3,-5.3672414E-2,-2.1062715E-2,-3.6224958E-2,-1.7871754E-2,3.3028296E-3,-9.517076E-3,-4.090238E-3,8.947592E-3,-3.0775634E-3,6.8189753E-3,-1.4470284E-2,-3.8960394E-3,-1.7479647E-2,-6.494586E-3,7.342753E-4,9.425868E-3,1.4927432E-2,3.0676581E-2,2.6488658E-2,1.5745316E-2,-7.366165E-3,7.161057E-3,1.3727555E-3,1.9074049E-2,1.9221589E-2,3.892955E-2,1.3064409E-2,2.9590989E-2,4.5713555E-2,2.608596E-2,9.82246E-3,2.3332834E-2,-9.294088E-3,-1.5507324E-3,4.7446873E-2,2.076288E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,85,87,89,91,-1,-1,-1,-1,-1,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,121,123,125,127,129,131,133,135,137,-1,-1,-1,139,141,-1,143,145,147,149,151,153,155,157,159,161,163,165,167,169,171,173,-1,-1,-1,175,177,179,181,183,185,-1,187,189,191,-1,-1,193,-1,-1,-1,195,197,-1,-1,-1,199,201,-1,203,-1,205,207,-1,209,211,-1,-1,-1,213,-1,-1,-1,-1,-1,215,217,-1,219,-1,221,223,-1,-1,225,227,229,-1,231,233,235,237,-1,-1,-1,-1,239,241,-1,243,245,247,249,-1,-1,-1,-1,251,-1,253,255,257,-1,259,261,263,-1,-1,-1,265,-1,267,-1,-1,-1,-1,-1,269,271,-1,273,275,-1,277,-1,-1,279,-1,-1,-1,-1,-1,-1,-1,281,283,-1,-1,-1,285,-1,287,289,291,293,295,297,-1,299,301,303,-1,305,307,309,311,-1,-1,-1,313,315,-1,317,319,321,-1,323,-1,-1,-1,-1,-1,325,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5790176E2,8.2560684E1,1.4186261E2,4.332553E1,3.8246857E1,5.0445026E1,4.0210083E1,2.577127E1,0E0,4.7440186E0,6.5649405E0,2.1627697E1,2.1385864E1,1.4630399E0,2.2753357E1,1.681253E1,6.4775543E0,3.3936234E0,1.9358721E0,1.9655217E0,5.3308296E-1,7.348484E0,1.7839584E1,2.448397E1,3.9012184E0,3.0418396E-1,2.5594077E-1,1.344986E1,4.0738144E0,6.929817E0,1.0002075E1,2.6928105E0,1.0587139E0,1.4402733E0,5.524264E-1,8.032446E-1,5.0052553E-1,7.119936E-1,1.1218641E0,5.4127693E-2,1.8852615E-1,8.2678986E-1,4.4897175E0,0E0,7.9816794E0,5.70623E0,1.2608192E1,1.3961029E0,0E0,0E0,0E0,0E0,0E0,1.0802612E1,7.484108E0,3.5685272E0,9.7723365E-2,2.3775864E0,6.356017E0,6.1976624E0,7.227087E-1,9.1312957E-1,1.1380129E0,6.042175E-1,3.383832E-1,6.635399E-1,1.2768316E0,0E0,5.98853E-1,3.8945675E-1,3.4366572E-1,5.318652E-1,1.8676859E-1,7.134364E-1,3.441211E-1,2.384105E-1,3.634367E-1,0E0,0E0,0E0,1.1361551E-1,1.9367142E0,0E0,1.2239801E0,2.4654207E0,2.635199E0,8.126354E-2,3.8476562E-1,3.5404744E0,7.799961E0,3.682457E0,3.5128117E-1,3.142522E-1,6.255966E0,2.3079858E0,1.4722137E0,3.4615717E0,7.705507E-1,1.172142E0,0E0,0E0,0E0,1.4567871E0,1.8658193E0,1.064745E0,2.9492035E0,6.908493E-1,3.522644E-1,0E0,4.376963E-1,2.578206E-1,2.272608E-1,0E0,0E0,2.3257923E-1,0E0,0E0,0E0,2.105236E-1,1.0550747E0,0E0,0E0,0E0,1.7792511E-1,3.185621E-1,0E0,1.9119859E-1,0E0,5.427289E-2,1.364946E-2,0E0,2.0068479E-1,4.0063685E-1,0E0,0E0,0E0,2.264291E-2,0E0,0E0,0E0,0E0,0E0,9.3023086E-1,5.5281514E-1,0E0,2.5298328E0,0E0,9.018123E-1,1.4399381E0,0E0,0E0,2.5061285E-1,2.2990632E-1,2.3383436E0,0E0,4.407154E0,2.5473404E0,5.01709E-1,1.4757729E0,0E0,0E0,0E0,0E0,7.289879E0,2.0262384E0,0E0,4.3236342E-1,1.4783859E-1,3.9366074E0,8.376751E-1,0E0,0E0,0E0,0E0,3.6044312E-1,0E0,8.1663513E-1,7.0154977E-1,4.2077792E-1,0E0,1.8941689E-1,1.2466583E0,1.4312077E-1,0E0,0E0,0E0,4.2952967E-1,0E0,3.7290746E-1,0E0,0E0,0E0,0E0,0E0,1.700306E-2,1.9186783E-1,0E0,5.448096E-1,1.8774986E-2,0E0,4.9803734E-2,0E0,0E0,1.0178864E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8846196E-1,1.2465905E-1,0E0,0E0,0E0,5.3838444E-1,0E0,4.710397E-2,1.8815632E0,2.1308212E0,2.9324913E-1,1.8437967E-1,1.7791462E0,0E0,9.651478E-2,8.877507E-2,1.8180871E-1,0E0,1.549392E0,5.5908823E-1,1.6883507E0,9.695055E-1,0E0,0E0,0E0,3.6110365E-1,2.5003815E-1,0E0,2.6848793E0,2.751915E0,1.0712276E0,0E0,4.695435E-2,0E0,0E0,0E0,0E0,0E0,3.5013962E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,80,80,81,81,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,102,102,103,103,104,104,105,105,106,106,107,107,109,109,110,110,111,111,114,114,118,118,119,119,123,123,124,124,126,126,128,128,129,129,131,131,132,132,136,136,142,142,143,143,145,145,147,147,148,148,151,151,152,152,153,153,155,155,156,156,157,157,158,158,163,163,164,164,166,166,167,167,168,168,169,169,174,174,176,176,177,177,178,178,180,180,181,181,182,182,186,186,188,188,194,194,195,195,197,197,198,198,200,200,203,203,211,211,212,212,216,216,218,218,219,219,220,220,221,221,222,222,223,223,225,225,226,226,227,227,229,229,230,230,231,231,232,232,236,236,237,237,239,239,240,240,241,241,243,243,249,249],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,86,88,90,92,-1,-1,-1,-1,-1,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,122,124,126,128,130,132,134,136,138,-1,-1,-1,140,142,-1,144,146,148,150,152,154,156,158,160,162,164,166,168,170,172,174,-1,-1,-1,176,178,180,182,184,186,-1,188,190,192,-1,-1,194,-1,-1,-1,196,198,-1,-1,-1,200,202,-1,204,-1,206,208,-1,210,212,-1,-1,-1,214,-1,-1,-1,-1,-1,216,218,-1,220,-1,222,224,-1,-1,226,228,230,-1,232,234,236,238,-1,-1,-1,-1,240,242,-1,244,246,248,250,-1,-1,-1,-1,252,-1,254,256,258,-1,260,262,264,-1,-1,-1,266,-1,268,-1,-1,-1,-1,-1,270,272,-1,274,276,-1,278,-1,-1,280,-1,-1,-1,-1,-1,-1,-1,282,284,-1,-1,-1,286,-1,288,290,292,294,296,298,-1,300,302,304,-1,306,308,310,312,-1,-1,-1,314,316,-1,318,320,322,-1,324,-1,-1,-1,-1,-1,326,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,3.099779E6,2.14099E5,4.7931265E6,1.8108038E5,2.1328075E-1,2.496172E3,1.05969965E-1,1.5669752E-2,2.496172E3,2.857E3,4.898012E-1,7.2530017E0,3.0121632E6,6.748543E-1,8.229907E7,1.4225797E3,3.449702E2,2.4347392E-1,1.4075E4,1.14E2,1.7E1,9.27E2,2.5422776E3,2.3898147E5,2.3175376E3,5.185489E3,1E0,2.914E3,4.7777777E0,7.0764465E1,8.61491E3,1.752512E2,4.6547272E2,9.06727E5,2.0967E4,5.159139E8,4.123E3,1.1034263E10,3.130603E3,3.1066042E7,9.928469E-1,7.560434E-2,1.5963264E9,3.97E2,2.2515285E6,2.783636E6,1.2414789E-2,-2.913663E-2,-1.2562506E-2,-1.057034E-2,3.5961065E-3,3.791269E7,4.0844156E7,1E0,5.012E3,1.7515824E8,2.2075728E7,9.6910744E7,5.387818E8,5.418831E11,9.536863E4,3.046556E5,1.1855755E8,9E0,2.065904E6,-1.672725E-2,1.125584E-3,1.3572786E0,1.246372E0,1.2023678E-1,5.674727E2,6.7699623E-1,3.6482175E5,2.539E3,6.722251E1,5.8284517E-2,2.8452123E-2,1.3207421E-2,5.7064955E6,4.3875E1,-4.4513706E-2,1.8255814E0,2.8366232E0,3.689E3,2.2E1,3.5791788E0,9.669789E0,1.0872E4,5.3038636E7,3.6873734E5,4.854E4,9.070543E0,1.4387881E5,1.417988E6,2.2500049E10,1E0,1.2165793E3,1.2188469E-2,2.914085E-2,-1.4197066E-2,1E0,2.0939393E0,1.1472081E0,6.7628815E2,2.8681312E8,1.2E1,-1.2903138E-2,3.072617E11,2.8541E4,2.8091298E7,3.208191E-2,-2.356723E-2,1.8E1,-4.2657247E-3,-2.3793833E-2,-1.2125694E-2,1.0258198E-2,9.5411836E4,-4.678677E-2,-6.3783093E-3,1.6341409E-2,2.22E2,3.2704837E2,1.8956681E-3,5.5814706E5,2.8665771E-3,4.459525E5,9E0,8.806648E-3,1.5856144E3,2.7682406E8,2.120104E-3,2.114036E-2,-1.3570258E-3,2.720437E6,2.9680872E-2,1.1633313E-2,1.6019335E-2,3.3859063E-2,-3.385638E-2,4E0,4.97E2,2.9346237E-2,4.602015E0,2.002479E-2,3.788185E6,1.3664E4,3.393286E-2,1.9378373E-2,3.4078704E7,5.827505E5,3.321532E6,-3.4829695E-2,4E1,1.57383175E1,3.535297E3,3.9813398E6,-1.6083665E-2,-3.9005864E-2,-2.394059E-3,-1.9908201E-2,1.2236866E3,1.3359244E9,-3.8155016E-2,8.891209E6,3.72424E5,1.921032E4,8.4441924E-1,-1.1842902E-2,4.7214255E-2,1.8662479E-2,5.1064175E-2,1.5846036E7,-1.8572195E-2,3.3E1,3.68E2,3.1790106E8,-1.0449004E-2,8.24666E5,4.987639E7,4.5402945E6,-5.796309E-2,-9.583962E-2,-4.622153E-2,1.7039736E4,-1.4650951E-2,2.8636363E0,-2.3545818E-2,-8.061125E-3,9.797586E-3,-4.6766894E-3,-2.5552765E-2,1.819149E1,7.84892E5,-4.9901694E-2,1.9E1,1.636E3,-9.035688E-3,1.1400756E6,-1.7179694E-2,-2.1983965E-3,2.0368582E8,-2.3875865E-3,-1.9314745E-2,-7.904549E-3,5.012538E-4,-3.1068467E-3,-1.7731788E-2,-3.089743E-3,6.6E2,2.4019E6,5.216782E-3,1.2239265E-2,7.7752606E-4,2.6E2,-9.989654E-3,1.028E3,3.3030225E2,8E0,8.6E1,2.5E1,7.06699E8,2.5484381E-2,8.1445E1,2.68E0,1.21E2,-1.8593261E-2,2.6883545E2,7.780377E6,1.3878379E0,3.2E1,8.011824E-2,3.1189786E-2,3.100967E-2,2.34E2,4.5454E4,5.5745427E-2,3.9134903E5,1.4521701E2,3.164136E6,3.569526E-2,2.1996392E-1,1.0544851E-2,5.420075E-2,8.4117725E-2,5.8136806E-2,3.5103376E-3,1.1095187E1,1.7650869E-2,8.407847E-2,6.0677916E-2,-5.0198916E-2,-2.9116927E-2,2.1883514E-2,-6.3457416E-4,-5.714141E-3,-2.526961E-2,-2.9476976E-2,-4.6437524E-2,-5.690455E-2,-7.493692E-2,-4.1455727E-2,-2.2506278E-2,-3.312151E-2,-1.1780705E-2,1.0985354E-2,-6.0232864E-3,-6.1964158E-2,-3.632225E-2,-3.53137E-2,-1.6280105E-2,-1.9165386E-2,-5.7890997E-3,-3.311543E-2,-1.7269786E-2,-2.4440184E-2,-1.2151474E-2,-1.9274484E-2,-1.0509291E-2,1.6967178E-3,1.6727623E-2,1.2456676E-3,-8.033132E-3,-2.4220945E-2,-3.2951776E-3,4.1720085E-3,1.3134104E-2,-1.46561E-2,5.835422E-3,-5.3672414E-2,-2.1062715E-2,-3.6224958E-2,-1.7871754E-2,3.3028296E-3,-9.517076E-3,-4.090238E-3,8.947592E-3,-3.0775634E-3,6.8189753E-3,-1.4470284E-2,-3.8960394E-3,-1.7479647E-2,-6.494586E-3,7.342753E-4,9.425868E-3,1.4927432E-2,3.0676581E-2,2.6488658E-2,1.5745316E-2,-7.366165E-3,7.161057E-3,1.3727555E-3,1.9074049E-2,1.9221589E-2,3.892955E-2,1.3064409E-2,2.9590989E-2,4.5713555E-2,2.608596E-2,9.82246E-3,2.3332834E-2,-9.294088E-3,-1.5507324E-3,4.7446873E-2,2.076288E-2],"split_indices":[20,102,51,2,51,28,42,4,0,38,4,2,27,53,28,27,45,4,52,38,9,29,0,2,52,33,4,52,6,2,54,56,4,52,52,9,10,7,2,5,4,45,53,0,7,2,28,12,0,0,0,0,0,45,45,100,2,31,45,45,7,31,28,32,45,10,28,0,38,42,53,57,52,42,28,2,58,0,0,0,32,4,0,54,38,2,58,54,54,2,7,28,29,53,32,29,5,16,4,0,0,0,79,53,53,52,7,3,0,31,2,45,0,0,3,0,0,0,38,33,0,0,0,11,4,0,28,0,28,3,0,4,7,0,0,0,32,0,0,0,0,0,10,2,0,53,0,12,2,0,0,7,45,28,0,3,58,33,28,0,0,0,0,52,7,0,32,9,4,27,0,0,0,0,47,0,3,0,7,0,29,45,32,0,0,0,51,0,54,0,0,0,0,0,56,9,0,8,0,0,33,0,0,7,0,0,0,0,0,0,0,0,32,0,0,0,10,0,2,52,17,11,3,7,0,52,54,2,0,4,28,53,8,0,0,0,10,1,0,28,58,1,0,39,0,0,0,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.563E3,8.04E2,1.759E3,4.42E2,3.62E2,1.19E3,5.69E2,4.38E2,4E0,2.55E2,1.07E2,4.94E2,6.96E2,3.4E1,5.35E2,3.53E2,8.5E1,1.29E2,1.26E2,7.7E1,3E1,2.63E2,2.31E2,6.56E2,4E1,2.3E1,1.1E1,4.5E2,8.5E1,1.73E2,1.8E2,4.8E1,3.7E1,1.15E2,1.4E1,9.4E1,3.2E1,4.1E1,3.6E1,8E0,2.2E1,9E1,1.73E2,7E0,2.24E2,3.04E2,3.52E2,3.4E1,6E0,1.7E1,6E0,5E0,6E0,3.73E2,7.7E1,7.7E1,8E0,1.12E2,6.1E1,1.51E2,2.9E1,3.5E1,1.3E1,2.7E1,1E1,5.3E1,6.2E1,4E0,1E1,5.9E1,3.5E1,1.5E1,1.7E1,3.2E1,9E0,1.8E1,1.8E1,4E0,4E0,4E0,1.8E1,7.9E1,1.1E1,1.8E1,1.55E2,2.05E2,1.9E1,5.4E1,2.5E2,2.66E2,8.6E1,2.4E1,1E1,3.55E2,1.8E1,5E1,2.7E1,1.4E1,6.3E1,4E0,4E0,7E0,1.05E2,2.8E1,3.3E1,1.22E2,2.9E1,2.3E1,6E0,1.6E1,1.9E1,9E0,4E0,9E0,1.8E1,4E0,6E0,4E0,4.9E1,5.7E1,5E0,6E0,4E0,4.4E1,1.5E1,5E0,3E1,6E0,9E0,9E0,8E0,1.1E1,2.1E1,5E0,4E0,6E0,1.2E1,1.2E1,6E0,4E0,1.4E1,4.9E1,3E1,1.3E1,5E0,1.51E2,4E0,2.8E1,1.77E2,1.2E1,7E0,1.7E1,3.7E1,2.46E2,4E0,2.52E2,1.4E1,2.5E1,6.1E1,4E0,2E1,5E0,5E0,2.71E2,8.4E1,4E0,1.4E1,2.4E1,2.6E1,2.3E1,4E0,8E0,6E0,1.7E1,4.6E1,6E0,9.9E1,1.5E1,1.3E1,4E0,2.9E1,1.04E2,1.8E1,8E0,2.1E1,8E0,1.5E1,5E0,1.1E1,1.4E1,5E0,4E0,5E0,5E0,1.3E1,4.1E1,8E0,3.9E1,1.8E1,4E0,4E1,9E0,6E0,2.5E1,5E0,5E0,4E0,5E0,4E0,7E0,4E0,8E0,1.3E1,4E0,8E0,6E0,2.4E1,5E0,8E0,1.15E2,3.6E1,1.8E1,1E1,1.73E2,4E0,8E0,9E0,2.2E1,1.5E1,2.07E2,3.9E1,2.06E2,4.6E1,5E0,9E0,1E1,1.5E1,3.2E1,2.9E1,1.33E2,1.38E2,7.1E1,1.3E1,8E0,6E0,1E1,1.4E1,2.2E1,4E0,1.6E1,7E0,2.6E1,2E1,8.9E1,1E1,4E0,1.1E1,5E0,8E0,1.1E1,1.8E1,7.5E1,2.9E1,1.2E1,6E0,1E1,5E0,5E0,6E0,5E0,8E0,3.7E1,4E0,2.7E1,1.2E1,1.4E1,4E0,3.6E1,4E0,7E0,1.8E1,4E0,4E0,7E0,6E0,2E1,4E0,4E0,4E0,1.03E2,1.2E1,6E0,3E1,4E0,1.4E1,4E0,6E0,1.42E2,3.1E1,4E0,4E0,4E0,5E0,5E0,1.7E1,9.2E1,1.15E2,3.1E1,8E0,5.2E1,1.54E2,1.8E1,2.8E1,4E0,1.1E1,5E0,2.7E1,3.6E1,9.7E1,1.11E2,2.7E1,4.9E1,2.2E1,4E0,4E0,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"327","size_leaf_vector":"1"}},{"base_weights":[-1.06815165E-2,-5.256525E-1,2.3660535E-1,-7.972246E-1,-1.9766805E-1,5.5900708E-2,6.5051484E-1,-9.5078135E-1,-2.0208442E-1,-3.9977667E-1,3.0205107E-1,-1.292619E-1,2.6639214E-1,4.4428298E-1,9.520044E-1,-4.9485835E-1,-1.0430484E0,1.01238035E-1,-3.2229206E-1,-5.3467226E-1,-2.6194155E-1,9.64903E-2,6.325664E-1,-4.431254E-1,-2.4158487E-2,3.2715303E-1,-3.428342E-1,6.162502E-1,6.667118E-2,5.098701E-1,1.0385604E0,-5.935519E-1,-2.5948135E-2,-9.7530615E-1,-1.6015992E0,-8.129917E-2,-6.142029E-1,-6.0796666E-1,-3.5615703E-1,-3.3431625E-1,-2.5913974E-2,-1.4450757E-1,2.5485817E-1,7.135228E-1,2.5644597E-1,-5.231236E-1,-2.5875425E-1,1.2161926E0,-7.1812674E-2,1.3151051E-1,5.613645E-1,-4.593976E-1,1.7772608E-2,4.9523714E-1,1.0105292E0,-1.6644062E-1,4.3554965E-1,1.9059479E-1,6.2106115E-1,9.0881664E-1,1.2554661E0,-2.1767366E-1,-6.671619E-1,-5.386576E-3,5.5068214E-3,-1.0723746E0,-6.6975814E-1,-3.592903E-2,-8.522884E-2,-1.6808559E-1,2.9860744E-2,-7.4909467E-1,-2.6604414E-1,-6.9035953E-1,-3.966462E-1,-1.9279571E-1,-4.5953092E-1,-3.6884275E-1,-1.0486948E-1,-1.677069E-1,8.7391704E-2,1.37998555E-2,-2.2612992E-1,1.288359E-1,4.0063056E-1,1.6037034E-2,7.701395E-1,3.92703E-3,1.9137722E-2,-4.870804E-1,-5.9811763E-2,-2.9700738E-1,1.2053689E-3,3.2857336E-2,6.639692E-2,-2.715264E-2,-6.410863E-1,-2.6299515E-1,1.7932934E-1,5.21758E-1,1.3204623E0,-7.324338E-1,-2.674673E-1,3.0493543E-1,7.5730336E-1,5.207191E-2,2.5866145E-2,-3.0555911E-2,-2.3883946E-2,5.332167E-1,-4.7780346E-4,1.5715947E-2,-1.050875E-3,2.1083094E-2,3.655478E-2,6.480044E-1,1.1539596E0,6.557876E-2,2.427774E-2,-1.7484343E-2,-7.7164464E-4,-4.0608756E-2,-4.4052827E-1,-9.762218E-1,-1.3032209E0,-2.6349968E-1,-8.2190937E-1,-2.1861983E-2,-1.0850478E-1,-2.064122E-2,-8.7963736E-1,-2.7201911E-3,-3.5404146E-1,-5.986351E-1,-4.5309596E-2,-3.186932E-2,-2.562312E-1,-8.099228E-3,-1.8817062E-2,-3.3754494E-2,-1.8575188E-2,-4.1756076E-1,-2.0877023E-1,-1.0247543E-2,-1.5269889E-2,-1.63963E-2,-2.4287773E-2,1.0022545E-2,-3.4540392E-2,7.802023E-4,-2.9056868E-1,-4.357924E-3,1.9028217E-1,4.9375013E-1,5.825106E-3,5.6497883E-2,6.4790744E-1,-6.010665E-1,-3.4789854E-1,-1.7573293E-1,-3.6436212E-1,-7.912433E-2,2.404082E-1,-4.7507737E-2,-4.5162913E-1,-8.250111E-2,-3.7264922E-1,-1.8131796E-1,2.0772496E-1,4.301873E-1,7.3128027E-1,7.784331E-2,3.6173396E-2,-2.1244816E-2,-4.086785E-2,-6.2102216E-1,-3.6546685E-2,1.6353266E-1,5.0098E-1,9.118363E-1,3.5330567E-1,-1.3172854E-1,3.990984E-1,6.4330196E-1,1.8040979E-1,2.702393E-1,7.118748E-1,5.149837E-2,7.502515E-2,-3.9926525E-3,-5.6361425E-1,-1.0310508E0,-5.737635E-1,-1.3455225E0,-3.594617E-2,-1.363818E-4,-3.5024112E-1,-9.206089E-1,-2.8021486E-2,1.0526646E-2,-1.5100153E-1,-3.0640615E-2,-5.4200415E-2,-2.1448627E-2,-1.0393862E-2,-6.2789583E-1,-1.0478476E-2,-1.2544455E-1,-1.6637329E-2,4.2020534E-3,-4.176197E-3,-4.470475E-1,-1.7458695E-1,-3.4268034E-1,-7.9402424E-2,2.5362452E-3,-3.909619E-3,-5.6379777E-3,3.4535355E-3,-8.220069E-3,3.9736102E-3,-5.363599E-3,-3.3227247E-1,6.259742E-4,2.3684815E-1,2.8309083E-2,1.2460766E-2,2.1329958E-2,3.656256E-2,-1.6660128E-2,-3.0715374E-2,-2.9704875E-1,-2.6316334E-2,-3.5105022E-3,-1.1815471E-2,-7.172531E-3,-3.9518365E-1,-9.737114E-2,5.783745E-1,3.2029662E-1,2.0677892E-2,-2.6744375E-2,-9.196122E-3,-9.685656E-3,-1.1478083E-2,-4.1750252E-1,-8.172523E-3,6.7218095E-2,-4.649378E-1,1.6730398E-1,4.5891732E-1,1.843824E-1,4.8131827E-1,5.791761E-3,7.790371E-1,-4.1674163E-2,-1.5214918E-2,2.2816932E-1,-3.2661483E-1,-4.4150893E-2,2.017987E-1,8.349282E-3,5.3255934E-1,1.0865482E0,5.9544104E-1,1.0029491E-1,5.347058E-1,1.0410563E-1,-3.854543E-1,1.2500774E-3,3.2194298E-2,3.8937074E-1,4.1383155E-2,1.4543548E-2,1.6825091E-3,3.9593508E-3,1.9205468E-2,7.832829E-1,2.576483E-2,-1.4958356E-2,-3.1947516E-2,-5.234894E-2,-3.2409154E-2,-1.877974E-2,-3.5053052E-2,-6.9702074E-2,-3.6671482E-2,-7.606359E-3,-2.0406721E-2,-2.9451407E-2,-4.9199276E-2,4.613313E-3,-1.0219645E-2,-1.1579908E-2,-3.298329E-2,-1.9718907E-3,-9.310592E-3,-2.2998897E-2,-1.0462027E-2,-2.5943888E-3,-1.3107961E-2,-9.53547E-3,-2.0095883E-2,-5.1230454E-4,-9.393285E-3,-9.078747E-3,-1.8513136E-2,1.3710081E-2,5.341358E-3,-1.768375E-2,-3.982272E-3,-1.18420655E-2,-2.2303984E-2,-6.606387E-3,7.96174E-3,4.9393706E-2,7.0043216E-3,1.1866686E-2,2.2030767E-2,8.665964E-3,-2.0887784E-3,-2.2060503E-3,1.1737116E-3,-1.28122065E-2,-2.4803633E-2,-6.3899514E-4,8.283654E-3,-3.438147E-2,-7.4350364E-3,4.5314096E-3,1.2791238E-2,2.5517017E-2,1.3485534E-2,1.2308599E-2,-8.107749E-3,1.8092971E-2,2.8829038E-2,2.717511E-2,4.242235E-2,1.70856E-2,2.299001E-3,-5.611154E-3,-2.3764584E-2,-7.307593E-3,2.4128726E-3,1.422435E-2,6.321443E-3,1.44490795E-2,2.8058857E-2,3.3529475E-2,6.0054854E-2,1.795974E-2,3.352328E-2,1.090031E-2,-3.057568E-3,3.2512307E-2,1.2102233E-2,-2.3481387E-3,1.227887E-2,-3.167357E-2,-1.2564003E-2,9.276183E-3,2.5401372E-2,4.1435752E-2,2.4356024E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,107,109,111,113,115,117,119,121,-1,-1,123,125,-1,-1,127,-1,129,131,133,135,137,139,141,143,145,147,-1,149,151,153,-1,155,-1,-1,157,-1,159,-1,-1,-1,161,163,165,167,169,171,173,175,177,179,-1,-1,-1,181,183,-1,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,189,191,193,195,197,-1,199,-1,201,-1,203,205,-1,-1,207,209,-1,-1,-1,211,213,-1,215,-1,217,-1,219,-1,221,-1,223,225,-1,-1,227,229,231,233,235,237,239,-1,241,243,245,247,249,251,253,-1,-1,-1,-1,255,257,259,261,263,265,267,269,271,273,275,277,-1,-1,-1,279,281,283,285,-1,-1,287,289,-1,-1,291,-1,-1,-1,-1,293,-1,295,-1,-1,-1,297,299,301,303,-1,-1,-1,-1,-1,-1,-1,305,-1,307,-1,-1,-1,-1,-1,-1,309,-1,-1,-1,-1,311,313,315,317,319,-1,-1,-1,321,323,-1,325,327,329,331,333,335,-1,337,-1,-1,339,341,343,345,-1,347,349,351,353,355,357,359,-1,-1,361,-1,-1,-1,-1,-1,363,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2816653E2,7.430614E1,1.3018237E2,4.1572052E1,3.849982E1,4.7311287E1,3.246852E1,1.4754028E1,2.6098383E1,4.8987465E0,7.412471E0,2.1299623E1,2.113789E1,2.0478962E1,7.660141E0,2.900138E0,9.826996E0,0E0,6.3517733E0,1.5629158E0,2.3145466E0,2.6703048E0,1.095892E0,2.254034E0,2.875468E1,2.3615993E1,4.3973618E0,9.853256E0,8.671485E0,1.1951494E0,3.8683014E0,1.2935333E0,1.45624E-1,7.319397E0,1.621933E0,3.1053896E0,1.726223E0,1.3802452E0,6.206708E-1,7.8520584E-1,5.448323E-1,1.0239059E0,7.199483E-1,5.0850296E-1,1.7248583E-1,2.1374168E0,5.590334E-1,2.5868988E-1,1.1880921E1,5.385902E0,6.3763504E0,2.1622744E0,0E0,8.147507E0,2.3933792E-1,3.9192872E0,1.7005382E0,2.951193E-1,2.8550148E-1,6.6048965E0,2.1689758E0,2.620387E-1,1.0604439E0,0E0,0E0,3.2558594E0,3.9866905E0,0E0,0E0,6.9100046E-1,0E0,9.086094E-1,2.2279358E-1,7.8728104E-1,8.1860685E-1,6.0798836E-1,2.2289562E-1,6.1468315E-1,1.3156088E-1,3.3717805E-1,2.6916343E-1,0E0,3.7840188E-1,3.3521077E-1,4.4500184E-1,0E0,5.4553413E-1,0E0,0E0,1.5042896E0,0E0,3.0354834E-1,0E0,0E0,0E0,6.0679526E0,1.5655565E0,5.8280516E-1,2.6285877E0,4.002285E0,4.3016243E-1,1.249342E-1,2.2633102E0,2.6624765E0,4.0480156E0,0E0,0E0,0E0,2.242158E0,1.1267853E0,0E0,0E0,0E0,0E0,0E0,1.2070484E0,9.3800354E-1,0E0,0E0,0E0,0E0,0E0,7.98326E-1,2.6742554E0,4.3972015E-1,4.514358E-1,7.464981E-1,0E0,5.6080925E-1,0E0,4.2762852E-1,0E0,9.560108E-3,4.3733788E-1,0E0,0E0,1.5895534E-1,7.661902E-2,0E0,0E0,0E0,4.0913773E-1,3.7123454E-1,0E0,4.1394766E-2,0E0,8.223502E-2,0E0,1.6133459E-1,0E0,1.0951519E-1,0E0,1.5430158E-1,1.0780263E-1,0E0,0E0,1.3232327E-1,6.9389343E-3,3.0718613E-1,9.86945E-2,1.338377E-1,4.435315E0,1.2397046E0,0E0,4.1609478E-1,1.00576356E-1,1.0990214E-1,1.3705577E0,2.3092804E0,1.909605E0,1.8787003E0,0E0,0E0,0E0,0E0,4.877019E-1,1.4566658E0,4.7779632E-1,3.4330654E-1,2.1169167E0,9.1019154E-1,2.3737788E0,9.0118885E-1,8.0444336E-1,1.3633415E-1,1.8752354E-1,2.82732E-1,0E0,0E0,0E0,8.887243E-2,1.407013E-1,2.4207544E-1,5.48851E-1,0E0,0E0,1.1059153E-1,1.4067078E-1,0E0,0E0,4.7475392E-1,0E0,0E0,0E0,0E0,4.339695E-1,0E0,3.9765716E-2,0E0,0E0,0E0,9.239006E-2,8.3250165E-2,2.2639036E-2,9.568755E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.904428E-2,0E0,4.3836296E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.9087214E-1,0E0,0E0,0E0,0E0,8.8561535E-2,3.2055128E0,1.6587205E0,4.1462612E-1,1.981394E-1,0E0,0E0,0E0,1.1319466E-2,6.3364506E-2,0E0,8.886484E-2,5.3252876E-1,1.3629885E0,1.75282E-1,6.627757E-1,1.2848454E0,0E0,5.585518E-1,0E0,0E0,1.9151014E-1,2.339772E-1,1.0144271E-1,2.7596068E-1,0E0,9.8695755E-2,6.792946E-1,1.1508703E-1,2.0367256E-1,2.4947E-1,4.6097144E-1,4.7316742E-1,0E0,0E0,1.9652939E-1,0E0,0E0,0E0,0E0,0E0,5.8315277E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,82,82,83,83,84,84,86,86,89,89,91,91,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,108,108,109,109,115,115,116,116,122,122,123,123,124,124,125,125,126,126,128,128,130,130,132,132,133,133,136,136,137,137,141,141,142,142,144,144,146,146,148,148,150,150,152,152,153,153,156,156,157,157,158,158,159,159,160,160,161,161,162,162,164,164,165,165,166,166,167,167,168,168,169,169,170,170,175,175,176,176,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,185,185,186,186,190,190,191,191,192,192,193,193,196,196,197,197,200,200,205,205,207,207,211,211,212,212,213,213,214,214,222,222,224,224,231,231,236,236,237,237,238,238,239,239,240,240,244,244,245,245,247,247,248,248,249,249,250,250,251,251,252,252,254,254,257,257,258,258,259,259,260,260,262,262,263,263,264,264,265,265,266,266,267,267,268,268,271,271,277,277],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,108,110,112,114,116,118,120,122,-1,-1,124,126,-1,-1,128,-1,130,132,134,136,138,140,142,144,146,148,-1,150,152,154,-1,156,-1,-1,158,-1,160,-1,-1,-1,162,164,166,168,170,172,174,176,178,180,-1,-1,-1,182,184,-1,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,190,192,194,196,198,-1,200,-1,202,-1,204,206,-1,-1,208,210,-1,-1,-1,212,214,-1,216,-1,218,-1,220,-1,222,-1,224,226,-1,-1,228,230,232,234,236,238,240,-1,242,244,246,248,250,252,254,-1,-1,-1,-1,256,258,260,262,264,266,268,270,272,274,276,278,-1,-1,-1,280,282,284,286,-1,-1,288,290,-1,-1,292,-1,-1,-1,-1,294,-1,296,-1,-1,-1,298,300,302,304,-1,-1,-1,-1,-1,-1,-1,306,-1,308,-1,-1,-1,-1,-1,-1,310,-1,-1,-1,-1,312,314,316,318,320,-1,-1,-1,322,324,-1,326,328,330,332,334,336,-1,338,-1,-1,340,342,344,346,-1,348,350,352,354,356,358,360,-1,-1,362,-1,-1,-1,-1,-1,364,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.496172E3,4.7931265E6,3.0990322E5,1.5283889E0,1.2478469E-1,1.4E1,1.5669752E-2,1.2060912E3,4.54E2,5.9572783E0,2.7246006E7,1.067536E3,5.7704294E5,1.0017953E8,1.01238035E-1,7.76264E7,2.728E3,3.449702E2,1.8140173E-1,4.8765745E0,7.364257E4,8E0,1.06E3,3.0795444E3,2.7702793E-1,7.418546E1,1.7049885E0,1.7730331E6,2.22E2,1.5851064E0,3.83E3,1.8722344E8,1.5137865E2,3.970405E3,1.5345133E1,6.822312E4,9.705292E6,2.914E3,1.6254681E0,4.123E3,2.1924436E5,4.0601485E6,2.0116506E4,7.997723E6,3.833E3,1E0,3.5E2,1.415178E3,3.054842E-4,1.7772608E-2,7.733016E5,2.589369E7,1.0972222E0,1.1159378E12,3.8E1,7.298614E2,2.6312E4,4.4263797E0,3.3720784E5,7.449623E2,-5.386576E-3,5.5068214E-3,2.036986E11,2.7246006E7,-3.592903E-2,-8.522884E-2,6.938E3,2.9860744E-2,3.046556E5,2.2980049E10,5.408015E0,1.06E3,4.8992978E5,1.16072E6,1.5083523E0,7E0,5.723202E5,2.3E1,1.37998555E-2,1.06158945E5,2.4239502E8,6.426937E6,1.6037034E-2,4.7777777E0,3.92703E-3,1.9137722E-2,3.79E2,-5.9811763E-2,9.25E0,1.2053689E-3,3.2857336E-2,6.639692E-2,1.2677199E7,5.8810627E-3,7.3066666E1,1.62848E5,2.298425E6,1.059448E7,4.4E1,6.2671E4,9.075E3,1E0,5.207191E-2,2.5866145E-2,-3.0555911E-2,3.8085933E3,2.6284674E2,-4.7780346E-4,1.5715947E-2,-1.050875E-3,2.1083094E-2,3.655478E-2,6.884212E4,2.257944E1,6.557876E-2,2.427774E-2,-1.7484343E-2,-7.7164464E-4,-4.0608756E-2,1.0119178E10,2.074E3,7.471519E2,5.49E2,4.7272725E0,-2.1861983E-2,1.3888E4,-2.064122E-2,1.2427474E8,-2.7201911E-3,6.862E3,5.723202E5,-4.5309596E-2,-3.186932E-2,2.8E2,8.986545E6,-1.8817062E-2,-3.3754494E-2,-1.8575188E-2,4.1899456E7,4.138602E7,-1.0247543E-2,1.43709E0,-1.63963E-2,9.69239E5,1.0022545E-2,2.6011018E8,7.802023E-4,2.4343E5,-4.357924E-3,2.1924436E5,4.4E0,5.825106E-3,5.6497883E-2,1.2679E4,7E0,1.6908121E1,3.1E1,2.9104478E0,9.433E3,2.773036E1,-4.7507737E-2,5.7E1,4.7E2,7.733016E5,8E0,4.1973075E6,5.1586456E7,2.9328E4,7.784331E-2,3.6173396E-2,-2.1244816E-2,-4.086785E-2,7.771383E0,4.898012E-1,1.8174828E6,1.1925198E3,2E0,9.58059E2,3.9712732E7,5.0279167E1,2.8541E4,1.784E3,5.387818E8,6.38041E5,5.149837E-2,7.502515E-2,-3.9926525E-3,4.5479352E2,4.3978744E0,1.216918E7,6.3844144E-1,-3.594617E-2,-1.363818E-4,1.152553E6,5.42376E5,-2.8021486E-2,1.0526646E-2,2.2668628E7,-3.0640615E-2,-5.4200415E-2,-2.1448627E-2,-1.0393862E-2,2.3629358E6,-1.0478476E-2,6.654321E0,-1.6637329E-2,4.2020534E-3,-4.176197E-3,5.95477E5,7.7628184E7,1.5E1,5.03E2,2.5362452E-3,-3.909619E-3,-5.6379777E-3,3.4535355E-3,-8.220069E-3,3.9736102E-3,-5.363599E-3,1.0908286E3,6.259742E-4,8.962557E8,2.8309083E-2,1.2460766E-2,2.1329958E-2,3.656256E-2,-1.6660128E-2,-3.0715374E-2,2.73E2,-2.6316334E-2,-3.5105022E-3,-1.1815471E-2,-7.172531E-3,1E0,1.2693982E6,7.969174E7,6.222E3,1.0594161E1,-2.6744375E-2,-9.196122E-3,-9.685656E-3,2E0,1E0,-8.172523E-3,4.320895E8,7.88E2,7.08E2,3.8575E4,2.857E3,3.7810526E2,5.791761E-3,5.3038636E7,-4.1674163E-2,-1.5214918E-2,5E1,2.78876E5,6.884212E4,4.4768806E1,8.349282E-3,2.34768E5,5.8084745E6,7.27E2,4.94374E1,7.6008E4,4E0,3.0379269E1,1.2500774E-3,3.2194298E-2,3.01E2,4.1383155E-2,1.4543548E-2,1.6825091E-3,3.9593508E-3,1.9205468E-2,1.7291568E2,2.576483E-2,-1.4958356E-2,-3.1947516E-2,-5.234894E-2,-3.2409154E-2,-1.877974E-2,-3.5053052E-2,-6.9702074E-2,-3.6671482E-2,-7.606359E-3,-2.0406721E-2,-2.9451407E-2,-4.9199276E-2,4.613313E-3,-1.0219645E-2,-1.1579908E-2,-3.298329E-2,-1.9718907E-3,-9.310592E-3,-2.2998897E-2,-1.0462027E-2,-2.5943888E-3,-1.3107961E-2,-9.53547E-3,-2.0095883E-2,-5.1230454E-4,-9.393285E-3,-9.078747E-3,-1.8513136E-2,1.3710081E-2,5.341358E-3,-1.768375E-2,-3.982272E-3,-1.18420655E-2,-2.2303984E-2,-6.606387E-3,7.96174E-3,4.9393706E-2,7.0043216E-3,1.1866686E-2,2.2030767E-2,8.665964E-3,-2.0887784E-3,-2.2060503E-3,1.1737116E-3,-1.28122065E-2,-2.4803633E-2,-6.3899514E-4,8.283654E-3,-3.438147E-2,-7.4350364E-3,4.5314096E-3,1.2791238E-2,2.5517017E-2,1.3485534E-2,1.2308599E-2,-8.107749E-3,1.8092971E-2,2.8829038E-2,2.717511E-2,4.242235E-2,1.70856E-2,2.299001E-3,-5.611154E-3,-2.3764584E-2,-7.307593E-3,2.4128726E-3,1.422435E-2,6.321443E-3,1.44490795E-2,2.8058857E-2,3.3529475E-2,6.0054854E-2,1.795974E-2,3.352328E-2,1.090031E-2,-3.057568E-3,3.2512307E-2,1.2102233E-2,-2.3481387E-3,1.227887E-2,-3.167357E-2,-1.2564003E-2,9.276183E-3,2.5401372E-2,4.1435752E-2,2.4356024E-2],"split_indices":[20,102,12,4,51,28,41,27,0,38,52,2,53,45,52,33,45,0,45,10,52,38,57,28,18,2,4,35,56,38,28,11,53,2,7,56,52,58,33,9,2,53,2,28,47,33,9,1,73,2,52,42,0,28,45,53,31,3,52,2,42,28,4,0,0,31,45,0,0,2,0,32,5,54,2,51,1,42,3,28,8,0,33,5,45,0,54,0,0,1,0,58,0,0,0,9,42,4,7,28,12,10,1,2,6,0,0,0,52,58,0,0,0,0,0,28,54,0,0,0,0,0,19,2,52,0,54,0,10,0,45,0,9,28,0,0,0,45,0,0,0,32,12,0,53,0,1,0,7,0,29,0,28,54,0,0,2,3,58,10,54,2,58,0,6,1,28,8,28,7,1,0,0,0,0,53,27,51,52,17,4,45,56,2,0,7,9,0,0,0,52,53,45,41,0,0,32,29,0,0,45,0,0,0,0,45,0,54,0,0,0,29,12,3,0,0,0,0,0,0,0,0,4,0,12,0,0,0,0,0,0,2,0,0,0,0,8,47,7,2,56,0,0,0,8,16,0,5,0,2,9,2,52,0,7,0,0,8,1,28,58,0,29,45,0,58,10,8,56,0,0,8,0,0,0,0,0,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.575E3,8.35E2,1.74E3,4.56E2,3.79E2,1.212E3,5.28E2,3.62E2,9.4E1,2.7E2,1.09E2,6.45E2,5.67E2,3.15E2,2.13E2,6.2E1,3E2,4E0,9E1,1.35E2,1.35E2,6.8E1,4.1E1,1.61E2,4.84E2,5.16E2,5.1E1,2.16E2,9.9E1,3.6E1,1.77E2,5.1E1,1.1E1,2.7E2,3E1,5E1,4E1,9.4E1,4.1E1,1.03E2,3.2E1,2.7E1,4.1E1,3.3E1,8E0,1.11E2,5E1,1.7E1,4.67E2,2.82E2,2.34E2,4.4E1,7E0,1.67E2,4.9E1,6.1E1,3.8E1,1E1,2.6E1,1.14E2,6.3E1,9E0,4.2E1,7E0,4E0,2.03E2,6.7E1,4E0,2.6E1,4.5E1,5E0,2.8E1,1.2E1,6.6E1,2.8E1,1.7E1,2.4E1,8.9E1,1.4E1,1.4E1,1.8E1,4E0,2.3E1,2.3E1,1.8E1,5E0,2.8E1,4E0,4E0,1.07E2,4E0,4.4E1,6E0,4E0,1.3E1,4.34E2,3.3E1,3E1,2.52E2,2.24E2,1E1,1.7E1,2.7E1,9.8E1,6.9E1,4.5E1,4E0,1.4E1,4.7E1,3.1E1,7E0,6E0,4E0,1.1E1,1.5E1,5.7E1,5.7E1,5.8E1,5E0,5E0,4E0,2.4E1,1.8E1,1.47E2,5.6E1,1.9E1,4.8E1,7E0,3.8E1,9E0,1.9E1,4E0,8E0,4.9E1,1.7E1,9E0,1.9E1,9E0,8E0,5E0,1.9E1,6.7E1,2.2E1,6E0,8E0,6E0,8E0,9E0,9E0,5E0,1.8E1,5E0,1.8E1,1.3E1,5E0,5E0,2.3E1,5.7E1,5E1,1.7E1,2.7E1,3.64E2,7E1,1.1E1,2.2E1,1.2E1,1.8E1,1.8E1,2.34E2,1.58E2,6.6E1,6E0,4E0,5E0,1.2E1,1E1,1.7E1,5.8E1,4E1,4.9E1,2E1,3.8E1,9E0,2.3E1,8E0,9E0,4.8E1,4.5E1,1.2E1,5E0,1.3E1,1.28E2,1.9E1,5.1E1,5E0,5E0,1.4E1,3.3E1,1.5E1,4E0,3.4E1,1E1,9E0,4E0,4E0,4.5E1,4E0,8E0,1.1E1,4E0,5E0,5.9E1,8E0,1E1,1.2E1,4E0,4E0,4E0,4E0,4E0,5E0,4E0,1.4E1,4E0,1.4E1,9E0,4E0,8E0,1.5E1,4E0,5.3E1,4.1E1,9E0,7E0,1E1,4E0,2.3E1,3.55E2,9E0,5.1E1,1.9E1,1.6E1,6E0,4E0,8E0,1.4E1,4E0,1E1,8E0,2.03E2,3.1E1,2.8E1,1.3E2,5E0,6.1E1,5E0,5E0,9E0,8E0,9E0,4.9E1,4E0,3.6E1,3E1,1.9E1,9E0,1.1E1,2E1,1.8E1,4E0,5E0,1.1E1,1.2E1,4E0,4E0,4E0,5E0,3.3E1,1.5E1,4E0,9E0,1.21E2,7E0,9E0,1E1,4.6E1,5E0,4E0,1E1,7E0,2.6E1,6E0,2.8E1,4E0,4.1E1,4E0,4E0,5.5E1,4E0,4E0,4E0,4E0,6E0,8E0,4E0,4E0,1E1,1E1,4E0,3.2E1,9E0,7E0,1.6E1,3.13E2,4.2E1,4E0,5E0,3.2E1,1.9E1,5E0,1.4E1,4E0,4E0,6E0,8E0,6E0,4E0,4E0,4E0,1.1E2,9.3E1,2.3E1,8E0,2.4E1,4E0,6E1,7E1,1.6E1,4.5E1,5E0,4E0,4E0,4E0,4E0,5E0,2.2E1,2.7E1,5E0,3.1E1,8E0,2.2E1,6E0,1.3E1,5E0,4E0,7E0,4E0,1E1,1E1,5E0,1.3E1,5E0,6E0,2.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"365","size_leaf_vector":"1"}},{"base_weights":[1.1340561E-3,-4.896459E-1,2.3746632E-1,-7.512483E-1,-1.682982E-1,5.613248E-2,6.319141E-1,-7.86206E-1,1.01178445E-1,-3.7575814E-1,2.8848505E-1,-1.7134483E-1,2.2202997E-1,4.7864023E-1,9.681042E-1,-9.3572444E-1,-4.697337E-1,-4.9023208E-1,-2.510338E-1,6.827935E-2,6.441903E-1,-3.3266208E-1,2.5549045E-2,2.0020455E-1,1.4348447E0,2.5145078E-1,7.2854E-1,3.9044866E-1,1.0650072E0,-7.934368E-1,-1.2937992E0,-2.574339E-1,-7.2255236E-1,-5.3001046E-1,5.6020938E-2,-2.8686494E-1,1.13267034E-1,-1.4910388E-1,1.9633149E-1,1.2058402E-2,6.9998103E-1,-5.5388594E-1,-2.1536028E-1,6.990168E-2,-3.1009423E-2,6.797404E-2,3.8405916E-1,8.771706E-2,3.171928E-2,3.3332986E-1,-2.8249472E-1,5.811858E-1,1.0678879E0,5.226442E-1,-2.1501344E-3,1.1009464E0,3.9959812E-1,-8.750682E-1,-1.4126588E-1,-5.164608E-2,-1.4691739E0,-3.6498478E-1,7.235607E-2,-8.164836E-1,-3.0863518E-1,-5.743344E-1,-1.9215795E-1,-6.187949E-3,9.821923E-3,-3.6957052E-1,-2.1755953E-1,-1.9907788E-3,9.7336015E-3,7.9390565E-3,-2.0705599E-1,6.931942E-2,3.643318E-1,5.493865E-1,4.40768E-2,-3.9026225E-1,-6.2704337E-1,-1.39537575E-2,-3.2713354E-1,-1.3381042E-1,1.944103E-1,1.1463599E-1,-4.6492988E-1,2.341042E-1,5.7172936E-1,2.630354E-1,5.397309E-1,-4.0474284E-1,-3.4198664E-2,6.769194E-1,3.516524E-2,7.90675E-1,6.844977E-2,1.2595124E-2,3.2425754E-2,1.1283972E0,2.5845416E-2,8.0134785E-3,2.7926376E-2,-9.277379E-1,-5.4919314E-1,-3.9710665E-1,1.7538434E-2,-7.989286E-2,-4.676232E-2,-1.9588563E-1,-5.6332356E-1,2.7986783E-1,-1.4981707E-2,-5.0594276E-1,-9.339234E-1,-1.6393253E-1,-2.6797798E-2,-6.5498006E-1,-4.168299E-1,-7.436186E-3,-1.8784977E-2,-8.137056E-3,-3.9351442E-1,-2.715631E-1,-7.252124E-2,-2.5398716E-1,1.1032565E-3,1.392113E-1,-7.04071E-2,4.5456276E-3,5.04527E-1,6.3582313E-1,3.6479288E-1,-2.625436E-2,-2.0203447E-1,-6.5755874E-1,-1.1116002E-2,-4.4802478E-1,1.9856882E-1,-4.527543E-1,-1.6689798E-1,-2.0378815E-1,5.890512E-2,1.3792384E-1,3.3728275E-2,-8.031779E-2,2.5960323E-1,-6.271999E-1,-2.11497E-1,2.5135452E-1,-1.7362246E-2,6.112367E-1,-4.120718E-3,5.0612886E-2,3.0085373E-1,3.868647E-2,3.998188E-1,-2.5507199E-2,-2.4228087E-1,-5.8006E-3,3.5431334E-3,5.580895E-1,8.64978E-1,-2.4619237E-2,2.4397227E-1,2.0005904E-2,4.1738164E-2,8.917209E-1,1.2236786E0,-9.697746E-1,-7.4549747E-1,-3.2094094E-1,-6.6444755E-1,-6.0562966E-3,-2.8023247E-2,1.31920045E-2,-1.07161924E-1,9.125767E-3,-2.506188E-1,-3.3042055E-2,-4.1813076E-1,4.1632572E-1,-2.4494405E-3,-3.6950413E-2,-3.113198E-1,-3.1868737E-2,-1.0169032E0,-1.0971338E-2,-4.509156E-3,-5.703312E-1,-4.548539E-2,-3.2880094E-2,-3.144917E-1,-4.676428E-3,4.007618E-3,-4.1007352E-1,-8.945375E-3,-3.3086377E-1,-9.1018006E-2,4.242541E-3,-1.5171196E-1,-3.2072297E-1,-5.68586E-3,6.3108536E-4,2.119978E-1,-9.445307E-3,4.298352E-3,2.976826E-2,1.4929637E-2,3.507783E-2,1.8138662E-2,1.1059109E-2,2.3184039E-2,-1.1861889E-2,-4.1888133E-3,-2.6928592E-2,-7.6169306E-1,-3.199388E-2,-3.1560728E-1,7.2938484E-1,5.381734E-3,-3.673918E-2,-4.0598E-1,-2.2137126E-1,1.2401265E-1,-2.9606318E-1,-5.6549728E-2,1.4262346E-2,-2.3377394E-2,5.441649E-2,4.5492008E-1,-5.5625796E-2,-4.071738E-2,3.3975804E-1,4.506645E-2,-2.1766262E-2,-3.7778005E-2,-3.0354667E-1,-1.9073776E-3,1.8192111E-1,3.5536796E-1,4.037745E-1,6.7737406E-1,9.578995E-2,-5.6518153E-3,2.0314212E-1,4.3450835E-1,2.321124E-2,2.0439118E-1,-5.946018E-3,-1.5844723E-2,6.495204E-1,4.1757858E-1,9.218806E-1,1.4104783E-2,2.3585042E-2,3.94973E-2,9.485774E-1,2.6912823E-2,2.5152417E-2,1.2513274E0,-4.91727E-2,-2.4979958E-2,-1.7875783E-2,-3.979826E-2,-2.2379959E-2,-5.9627444E-3,-3.9346963E-2,-1.3324477E-2,-1.4688113E-3,-1.050076E-2,-1.830804E-2,-6.6811484E-3,-1.1864026E-2,-2.3317628E-2,2.6480518E-2,9.828193E-3,-4.839521E-3,-2.3679104E-2,-3.9348774E-2,-5.946313E-2,-3.085311E-2,-1.728941E-2,-8.414134E-3,-1.7986173E-2,-2.2655388E-2,-1.6100328E-2,-6.9667823E-3,-1.867469E-2,8.2687344E-5,-1.197329E-2,-1.2666691E-2,-1.4114565E-3,-1.9011734E-2,-9.372845E-3,1.4306063E-2,5.896787E-3,-4.342217E-2,-2.1317016E-2,-8.182457E-3,-1.9918941E-2,2.345027E-2,4.9947396E-2,-1.5083355E-2,7.2941566E-3,-1.7044455E-2,-2.4814192E-2,-1.8246314E-2,-6.467792E-3,1.3775871E-3,9.776111E-3,2.0141785E-3,-1.5875228E-2,-9.210581E-3,-1.1508678E-3,-1.1023749E-2,3.3648491E-3,9.6250616E-4,1.5732376E-2,2.661882E-2,1.1186309E-2,-1.2419378E-2,2.3575283E-3,1.4930977E-2,4.184009E-2,-5.8939087E-4,1.5948573E-2,-2.0997591E-2,-6.3033947E-3,-1.2570679E-2,1.0180248E-2,2.195407E-2,1.1657674E-2,2.2382341E-2,4.222011E-3,3.0154E-2,4.489165E-2,9.649152E-3,4.1739616E-4,7.43575E-3,2.304728E-2,2.514167E-2,9.895383E-3,2.679108E-3,1.5703835E-2,2.509044E-2,3.6806386E-2,2.3091547E-2,5.1140096E-3,4.8020147E-2,2.4326835E-2,1.3511713E-2,-9.959325E-3,5.1217727E-2,3.310634E-2,3.327607E-2,6.391931E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,-1,83,85,87,-1,-1,89,91,93,95,97,-1,99,101,103,105,-1,107,109,111,113,115,117,119,-1,-1,121,123,-1,-1,-1,125,127,129,131,-1,133,135,137,139,141,143,145,147,149,151,153,155,157,159,161,163,165,-1,-1,-1,167,-1,-1,-1,169,171,173,175,-1,-1,177,179,181,-1,183,185,187,-1,189,191,193,-1,-1,195,197,199,201,-1,203,205,-1,207,209,211,-1,213,215,-1,217,219,221,223,225,227,229,-1,231,233,235,237,239,-1,241,-1,243,245,-1,247,-1,249,-1,-1,251,253,-1,255,-1,-1,257,259,261,263,265,267,-1,-1,-1,269,-1,271,-1,273,275,-1,-1,277,-1,279,-1,-1,281,-1,-1,283,-1,-1,285,-1,287,289,-1,291,293,-1,-1,295,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,297,-1,299,301,303,-1,305,307,309,311,313,-1,315,317,319,321,-1,323,325,-1,-1,327,-1,329,331,333,335,337,-1,339,341,-1,343,-1,-1,345,347,349,-1,-1,351,353,-1,-1,355,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9797125E2,7.0068115E1,1.2399186E2,4.6234283E1,3.5739746E1,4.4908424E1,2.7671555E1,2.1090332E1,0E0,3.5748558E0,9.198757E0,1.5953215E1,1.7924225E1,2.1220825E1,9.100174E0,1.4665955E1,7.7576447E0,2.9413872E0,1.677207E0,2.0891664E0,8.4249115E-1,7.042902E0,1.7676373E1,1.643257E1,1.9641266E0,8.794237E0,8.330139E0,1.5123978E0,3.0276337E0,1.1748367E1,2.4056854E0,2.9348836E0,2.352951E0,1.7615013E0,2.7713764E-1,5.7877064E-1,1.5756615E-1,5.3266287E-1,9.776341E-1,0E0,5.851345E-1,8.924656E-1,4.092594E0,0E0,5.1009827E0,9.875501E0,7.830364E0,0E0,0E0,2.368063E0,8.057089E-1,6.6148033E0,3.3542213E0,5.1990795E-1,0E0,1.6114655E0,2.533419E-1,2.888916E0,1.0633087E0,0E0,1.8941956E0,1.9690657E0,1.7058241E0,1.4968605E0,3.7331998E-1,1.1242104E0,5.404113E-1,0E0,0E0,1.8835115E-1,4.89424E-1,0E0,0E0,0E0,2.6369464E-1,2.8324246E-1,7.2057295E-1,1.5099192E-1,0E0,7.080436E-1,6.438427E-1,6.177457E0,2.279519E0,2.0483594E0,1.7800894E0,1.03081875E1,1.1833344E0,1.6824989E0,3.2811127E0,1.037157E0,1.1013336E0,1.8554091E-1,9.43061E-2,1.9307404E0,2.3247032E0,1.8828773E-1,0E0,0E0,0E0,1.8404236E0,0E0,0E0,0E0,5.717163E-1,5.75963E-1,3.8375938E-1,5.5080193E-1,0E0,0E0,7.708932E-1,1.2173748E-1,6.487225E-1,0E0,5.4252815E-1,2.8516388E-1,2.1186113E-2,0E0,1.0153961E0,7.8045464E-1,7.546355E-2,0E0,0E0,8.641386E-2,4.7308826E-1,2.4975237E-1,1.506232E-1,0E0,1.7237702E-1,2.018283E-1,0E0,6.600499E-2,1.6058445E-2,3.9616585E-2,0E0,4.2795837E-2,3.183403E-1,0E0,3.580613E-1,4.6106806E0,5.604458E-1,8.673364E-1,1.4913511E0,7.7089757E-1,1.6429504E0,0E0,2.7842102E0,3.578683E0,1.3972616E-1,2.1177459E-1,1.0547628E0,0E0,1.3455734E0,0E0,1.6616349E-1,1.3613491E0,0E0,2.7182293E-1,0E0,4.5285165E-2,0E0,0E0,6.15221E-1,1.066042E0,0E0,6.807936E-1,0E0,0E0,3.1221008E-1,1.1624756E0,4.1055298E-1,4.2693138E-1,2.536943E-1,7.197485E-1,0E0,0E0,0E0,8.993502E-2,0E0,3.7062943E-1,0E0,1.19690895E-2,1.4534271E-1,0E0,0E0,3.3764088E-1,0E0,1.750412E-1,0E0,0E0,3.7852478E-1,0E0,0E0,1.3868237E-1,0E0,0E0,9.996414E-3,0E0,2.3519015E-1,1.7943105E-1,0E0,1.5621665E-1,3.2071114E-2,0E0,0E0,5.002278E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.7217255E-1,0E0,1.2385547E-1,3.7419462E-1,1.5128316E0,0E0,1.7157269E-1,5.475986E-1,5.6788117E-2,5.0809E-1,1.842047E-1,0E0,5.6956774E-1,4.608612E-1,1.14447355E-1,3.026564E0,0E0,2.8462467E0,9.078064E-1,0E0,0E0,1.3328111E-1,0E0,9.498513E-1,5.2296734E-1,3.993373E-1,8.9523697E-1,1.4304796E-1,0E0,8.5685253E-1,6.472101E-1,0E0,1.322717E-1,0E0,0E0,1.4356232E-1,3.7410498E-1,1.4542389E-1,0E0,0E0,5.502208E-1,6.436348E-2,0E0,0E0,5.531769E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,74,74,75,75,76,76,77,77,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,99,99,103,103,104,104,105,105,106,106,109,109,110,110,111,111,113,113,114,114,115,115,117,117,118,118,119,119,122,122,123,123,124,124,125,125,127,127,128,128,130,130,131,131,132,132,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,151,151,153,153,154,154,156,156,158,158,161,161,162,162,164,164,167,167,168,168,169,169,170,170,171,171,172,172,176,176,178,178,180,180,181,181,184,184,186,186,189,189,192,192,195,195,197,197,198,198,200,200,201,201,204,204,216,216,218,218,219,219,220,220,222,222,223,223,224,224,225,225,226,226,228,228,229,229,230,230,231,231,233,233,234,234,237,237,239,239,240,240,241,241,242,242,243,243,245,245,246,246,248,248,251,251,252,252,253,253,256,256,257,257,260,260],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,-1,84,86,88,-1,-1,90,92,94,96,98,-1,100,102,104,106,-1,108,110,112,114,116,118,120,-1,-1,122,124,-1,-1,-1,126,128,130,132,-1,134,136,138,140,142,144,146,148,150,152,154,156,158,160,162,164,166,-1,-1,-1,168,-1,-1,-1,170,172,174,176,-1,-1,178,180,182,-1,184,186,188,-1,190,192,194,-1,-1,196,198,200,202,-1,204,206,-1,208,210,212,-1,214,216,-1,218,220,222,224,226,228,230,-1,232,234,236,238,240,-1,242,-1,244,246,-1,248,-1,250,-1,-1,252,254,-1,256,-1,-1,258,260,262,264,266,268,-1,-1,-1,270,-1,272,-1,274,276,-1,-1,278,-1,280,-1,-1,282,-1,-1,284,-1,-1,286,-1,288,290,-1,292,294,-1,-1,296,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,298,-1,300,302,304,-1,306,308,310,312,314,-1,316,318,320,322,-1,324,326,-1,-1,328,-1,330,332,334,336,338,-1,340,342,-1,344,-1,-1,346,348,350,-1,-1,352,354,-1,-1,356,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.14099E5,4.7931265E6,1.8108038E5,1.2974394E6,5.121E3,1.01178445E-1,1.5669752E-2,1.2060912E3,2.857E3,3.7002478E3,1.3664E4,3.564E3,3.653602E7,6.747114E7,1E0,5.4878284E2,1.482958E0,2.1924436E5,8.2E1,1.7E1,1.1896622E6,1.7324902E8,3.9712732E7,1.6706144E0,3.0823356E5,1.708775E7,8.9910974E2,8.294758E-1,2.1169072E3,3.970405E3,2.262586E6,4.2156538E5,2.1102592E8,1.9939552E8,6E0,3.647E3,1.2058402E-2,6.117756E9,1E0,1.8736842E0,6.990168E-2,6.938E3,4.898012E-1,1.172E3,8.771706E-2,3.171928E-2,2.043739E0,8.538E3,4.2620764E2,7.7101436E3,2.065904E6,-2.1501344E-3,4.4263797E0,4.760537E8,1.5E1,1E0,-5.164608E-2,7.621583E2,3.388159E7,8.0708375E5,1.0972E4,1.3521131E8,1.221875E1,1.0503613E5,-6.187949E-3,9.821923E-3,4.8E1,1.2690893E0,-1.9907788E-3,9.7336015E-3,7.9390565E-3,4.3906153E5,1.4187837E-1,2.1924436E5,3.1E1,4.40768E-2,4.3875E1,1.75E2,2.73E2,1.273801E6,2.1975278E8,1.3664E4,8.46E2,2.1337201E4,4.2E1,5.511673E0,3.647E3,1.3E1,1.6347875E3,5.8426323E0,6.4683E4,2.9816154E1,1E0,6.844977E-2,1.2595124E-2,3.2425754E-2,1.0989723E3,2.5845416E-2,8.0134785E-3,2.7926376E-2,4.1384277E2,3.5006908E6,5.5759512E-2,9.610984E-1,-7.989286E-2,-4.676232E-2,4.85E2,6.823E3,1.4777102E9,-1.4981707E-2,2.1969697E0,1E1,5.1232258E1,-2.6797798E-2,5.408015E0,1.06E3,6.072483E2,-1.8784977E-2,-8.137056E-3,3.7311627E2,5.723202E5,2.1761298E0,1.3411796E3,1.1032565E-3,1.9563605E5,7.9203456E8,4.5456276E-3,4.25E0,2.3887E4,1.3137E4,-2.625436E-2,2.3E1,9.82542E5,-1.1116002E-2,3.6E1,2.3E1,1E0,3.2485715E2,1.101214E6,6.4E1,1.2502964E9,3.3728275E-2,1.062363E1,2E1,4.2343444E7,3.44875E5,3.2196458E6,-1.7362246E-2,5.3038636E7,-4.120718E-3,1.2152382E5,1.4571098E5,3.868647E-2,4.8E1,-2.5507199E-2,3.94E2,-5.8006E-3,3.5431334E-3,2.3058404E7,6.061914E-1,-2.4619237E-2,6.5321615E6,2.0005904E-2,4.1738164E-2,2E0,1E0,4.1179886E0,1.8357558E5,8.70203E5,2.046E3,-6.0562966E-3,-2.8023247E-2,1.31920045E-2,1.2224265E1,9.125767E-3,7.95629E5,-3.3042055E-2,1.29E3,1.942E3,-2.4494405E-3,-3.6950413E-2,3.307766E6,-3.1868737E-2,1.873E3,-1.0971338E-2,-4.509156E-3,1.9E1,-4.548539E-2,-3.2880094E-2,2.7123368E10,-4.676428E-3,4.007618E-3,1.4887473E-1,-8.945375E-3,4.44E2,8.52E0,4.242541E-3,5.146E3,1.19507775E-1,-5.68586E-3,6.3108536E-4,2.862988E5,-9.445307E-3,4.298352E-3,2.976826E-2,1.4929637E-2,3.507783E-2,1.8138662E-2,1.1059109E-2,2.3184039E-2,-1.1861889E-2,-4.1888133E-3,-2.6928592E-2,8.489483E7,-3.199388E-2,1.24E2,2E1,4.5031875E-2,-3.673918E-2,7.382963E0,6.6126086E4,7.55E2,5.6440186E2,1.65E2,1.4262346E-2,1.4685706E3,7.9138456E5,2.805891E4,3.97E2,-4.071738E-2,8.631404E2,1.1851E4,-2.1766262E-2,-3.7778005E-2,6.0479047E4,-1.9073776E-3,1.4286021E2,3.8575E4,3.37E2,3.5861215E6,5.03E2,-5.6518153E-3,3.78E2,1.2478469E-1,2.321124E-2,1.9768514E3,-5.946018E-3,-1.5844723E-2,1.2219736E5,2.4832896E9,2.8290488E5,1.4104783E-2,2.3585042E-2,1.784E3,6.1473445E6,2.6912823E-2,2.5152417E-2,1.3860265E6,-4.91727E-2,-2.4979958E-2,-1.7875783E-2,-3.979826E-2,-2.2379959E-2,-5.9627444E-3,-3.9346963E-2,-1.3324477E-2,-1.4688113E-3,-1.050076E-2,-1.830804E-2,-6.6811484E-3,-1.1864026E-2,-2.3317628E-2,2.6480518E-2,9.828193E-3,-4.839521E-3,-2.3679104E-2,-3.9348774E-2,-5.946313E-2,-3.085311E-2,-1.728941E-2,-8.414134E-3,-1.7986173E-2,-2.2655388E-2,-1.6100328E-2,-6.9667823E-3,-1.867469E-2,8.2687344E-5,-1.197329E-2,-1.2666691E-2,-1.4114565E-3,-1.9011734E-2,-9.372845E-3,1.4306063E-2,5.896787E-3,-4.342217E-2,-2.1317016E-2,-8.182457E-3,-1.9918941E-2,2.345027E-2,4.9947396E-2,-1.5083355E-2,7.2941566E-3,-1.7044455E-2,-2.4814192E-2,-1.8246314E-2,-6.467792E-3,1.3775871E-3,9.776111E-3,2.0141785E-3,-1.5875228E-2,-9.210581E-3,-1.1508678E-3,-1.1023749E-2,3.3648491E-3,9.6250616E-4,1.5732376E-2,2.661882E-2,1.1186309E-2,-1.2419378E-2,2.3575283E-3,1.4930977E-2,4.184009E-2,-5.8939087E-4,1.5948573E-2,-2.0997591E-2,-6.3033947E-3,-1.2570679E-2,1.0180248E-2,2.195407E-2,1.1657674E-2,2.2382341E-2,4.222011E-3,3.0154E-2,4.489165E-2,9.649152E-3,4.1739616E-4,7.43575E-3,2.304728E-2,2.514167E-2,9.895383E-3,2.679108E-3,1.5703835E-2,2.509044E-2,3.6806386E-2,2.3091547E-2,5.1140096E-3,4.8020147E-2,2.4326835E-2,1.3511713E-2,-9.959325E-3,5.1217727E-2,3.310634E-2,3.327607E-2,6.391931E-2],"split_indices":[20,102,12,2,51,28,28,2,0,38,52,2,52,2,2,45,45,89,52,41,28,29,0,28,7,45,42,33,45,4,27,52,52,1,28,7,12,3,2,0,12,75,53,0,2,27,2,0,0,41,2,58,52,28,0,42,7,3,100,0,52,45,28,9,45,58,33,0,0,10,42,0,0,0,28,27,28,3,0,4,6,2,9,7,2,2,51,3,53,2,8,4,53,2,56,16,0,0,0,52,0,0,0,4,45,27,57,0,0,0,2,7,0,53,3,56,0,54,2,4,0,0,52,28,58,4,0,28,12,0,54,9,2,0,3,31,0,2,0,64,52,1,11,7,0,54,3,45,1,28,0,7,0,33,28,0,3,0,0,0,0,45,38,0,47,0,0,17,67,53,32,5,2,0,0,0,56,0,29,0,0,0,0,0,1,0,0,0,0,3,0,0,31,0,0,38,0,0,54,0,10,38,0,0,28,0,0,0,0,0,0,0,0,0,0,0,7,0,29,0,38,0,54,28,10,4,3,0,55,47,48,2,0,52,10,0,0,33,0,4,9,0,28,0,0,8,27,0,4,0,0,28,7,33,0,0,0,51,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.567E3,8.34E2,1.733E3,4.59E2,3.75E2,1.188E3,5.45E2,4.54E2,5E0,2.58E2,1.17E2,5.01E2,6.87E2,3.76E2,1.69E2,3.07E2,1.47E2,1.33E2,1.25E2,7.3E1,4.4E1,2.75E2,2.26E2,6.76E2,1.1E1,1.98E2,1.78E2,2.5E1,1.44E2,2.22E2,8.5E1,8.1E1,6.6E1,1.24E2,9E0,1.14E2,1.1E1,2.7E1,4.6E1,6E0,3.8E1,9.4E1,1.81E2,8E0,2.18E2,3.94E2,2.82E2,7E0,4E0,1.72E2,2.6E1,1.26E2,5.2E1,1.9E1,6E0,1.36E2,8E0,1.97E2,2.5E1,3.7E1,4.8E1,6.1E1,2E1,5.3E1,1.3E1,1.09E2,1.5E1,4E0,5E0,5E1,6.4E1,4E0,7E0,4E0,2.3E1,2.7E1,1.9E1,2.3E1,1.5E1,3.1E1,6.3E1,6.5E1,1.16E2,1.5E2,6.8E1,3.63E2,3.1E1,1.58E2,1.24E2,1.3E2,4.2E1,1.7E1,9E0,1.07E2,1.9E1,2.9E1,2.3E1,7E0,1.2E1,1.29E2,7E0,4E0,4E0,1.68E2,2.9E1,9E0,1.6E1,3.7E1,1.1E1,3.4E1,2.7E1,1.3E1,7E0,1.6E1,3.7E1,9E0,4E0,7E1,3.9E1,8E0,7E0,6E0,4.4E1,4.6E1,1.8E1,1.9E1,4E0,1.8E1,9E0,7E0,1.2E1,1.4E1,9E0,1.7E1,1.4E1,5.8E1,5E0,2.1E1,4.4E1,6.4E1,5.2E1,1.1E2,4E1,6.2E1,6E0,1.55E2,2.08E2,1.8E1,1.3E1,1.54E2,4E0,1.17E2,7E0,2E1,1.1E2,1.4E1,2.8E1,9E0,8E0,5E0,4E0,6.8E1,3.9E1,5E0,1.4E1,4E0,2.5E1,4E1,8.9E1,1.33E2,3.5E1,1.1E1,1.8E1,4E0,5E0,5E0,1.1E1,4E0,3E1,1.4E1,1.3E1,9E0,4E0,6E0,1E1,1E1,2.7E1,4E0,5E0,5.5E1,1.5E1,1E1,2.9E1,4E0,4E0,4E1,4E0,3.4E1,1.2E1,6E0,1.2E1,1.2E1,7E0,7E0,1.1E1,5E0,4E0,7E0,5E0,1E1,4E0,5E0,4E0,1E1,4E0,3E1,2.8E1,7E0,1.4E1,1.1E1,3.3E1,7E0,5.7E1,4.4E1,8E0,6.7E1,4.3E1,1E1,3E1,5E1,1.2E1,1.51E2,4E0,1.51E2,5.7E1,9E0,9E0,8E0,5E0,9.4E1,6E1,3E1,8.7E1,1.6E1,4E0,6.5E1,4.5E1,2E1,8E0,4E0,4E0,3.9E1,2.9E1,3.5E1,4E0,6E0,8E0,3.3E1,7E0,4E0,8.5E1,1.28E2,5E0,5E0,3E1,6E0,5E0,1.3E1,5E0,7E0,4E0,1.4E1,1.6E1,4E0,9E0,5E0,4E0,5E0,5E0,1.4E1,1.3E1,4.4E1,1.1E1,8E0,2.1E1,2.4E1,1.6E1,7E0,2.7E1,8E0,4E0,6E0,6E0,7E0,5E0,5E0,6E0,2E1,8E0,6E0,8E0,7E0,4E0,1E1,2.3E1,3.6E1,2.1E1,1.6E1,2.8E1,4E0,4E0,4E0,6.3E1,8E0,3.5E1,9E0,2.1E1,4.5E1,5E0,8E0,4E0,5.2E1,9.9E1,1.41E2,1E1,4.8E1,9E0,4E0,4E0,4E0,9E1,3.4E1,2.6E1,2.6E1,4E0,6.8E1,1.9E1,7E0,9E0,5.5E1,1E1,3.4E1,1.1E1,4E0,4E0,1.7E1,2.2E1,2.5E1,4E0,3.1E1,4E0,4E0,4E0,2.4E1,9E0,5E0,8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"357","size_leaf_vector":"1"}},{"base_weights":[6.8392474E-3,-4.68947E-1,2.2175017E-1,-7.4015933E-1,-1.9242775E-1,4.9492285E-2,6.0263896E-1,-8.8834167E-1,-4.096951E-1,-2.247502E-1,9.6965484E-2,-9.4877966E-2,2.8193063E-1,4.6169636E-1,9.1067207E-1,-8.790864E-2,-8.523927E-1,-5.169268E-1,-1.8292376E-1,-3.6105195E-1,3.431419E-1,-4.9139196E-1,-3.3729848E-2,3.3801502E-1,-3.9579782E-1,2.2621864E-1,6.81523E-1,6.6175365E-1,1.1075335E0,-9.043488E-1,-5.049804E-1,-5.416697E-1,-6.349393E-4,-2.4937358E-1,9.24134E-3,-2.7704844E-1,-8.2254976E-1,1.10147595E-1,5.012014E-1,-5.261601E-1,1.3712632E-2,-2.5277808E-1,4.4376355E-2,1.338905E-1,5.1002634E-1,-1.1939491E-1,-6.655677E-1,3.155755E-1,-3.2893375E-1,-1.4772204E-1,7.319003E-1,7.631701E-1,3.689131E-1,1.3046563E0,8.9144367E-1,-8.3487844E-1,-1.2823129E0,3.816318E-3,-5.982043E-1,-6.2173784E-1,-3.8482928E-1,-6.438926E-2,-3.0865023E-1,-4.0637428E-1,-4.8560996E-2,-7.24578E-1,-6.105108E-2,-3.9426535E-2,2.6218587E-1,6.716953E-1,2.743657E-1,-4.2744398E-1,-7.754943E-1,-4.4938156E-1,-1.5568694E-1,5.7356223E-2,1.119956E-2,1.9021308E-1,-1.082628E-1,4.289883E-1,8.683425E-1,3.7593972E-2,-2.8305247E-2,-1.7477503E-2,-3.7980504E-2,1.8709198E-1,4.4997236E-1,-1.4062655E-1,-5.064235E-1,-2.8827947E-2,1.2056223E-2,5.7510465E-1,9.117153E-1,8.4708244E-1,4.4493353E-1,3.2734256E-3,2.3913745E-2,3.6069732E-2,6.857098E-2,1.1416004E-2,9.465213E-1,-9.098843E-1,-5.8152944E-1,-7.001936E-2,-4.2799454E-2,-7.125956E-1,-3.1503966E-1,-6.4953446E-1,-1.750105E-2,-2.1042898E-2,-1.26572605E-2,1.7476627E-3,-8.531828E-3,-3.5863718E-1,-7.250243E-3,-3.2706177E-1,-7.163375E-1,-9.419027E-2,2.381122E-2,-8.3738613E-1,-4.4720632E-1,-1.1221303E-1,7.879926E-3,1.950158E-2,1.1918216E-1,8.236291E-3,7.603907E-1,3.777885E-2,4.1705152E-1,-4.6906683E-1,1.0052045E-3,-6.223817E-1,-5.9067555E-2,-3.4274152E-1,-7.7574927E-1,-2.7790707E-1,-5.5515654E-2,-1.5246766E-2,4.9998993E-1,2.0568305E-1,-1.666822E-2,-3.2719237E-1,1.8668273E-2,3.9011165E-1,7.624381E-1,6.53073E-2,7.090974E-1,1.9247027E-1,-1.5522279E-2,5.2270405E-2,2.843104E-1,1.79582E-1,5.235899E-1,-1.0371088E-2,-8.3886157E-4,-2.9264627E-2,-1.3114839E-2,6.251129E-1,3.0978268E-1,5.744878E-1,1.0529813E0,2.0820746E-2,4.58317E-2,9.526829E-3,2.8231652E-2,1.0146574E0,1.7282614E-2,-5.968733E-1,-9.477589E-1,-3.173955E-1,-7.294299E-1,-2.2037037E-2,-4.0469084E-2,-7.918204E-3,-1.9080754E-2,-2.2114241E-2,-6.879132E-1,-2.054786E-2,-1.0104008E-2,-3.6530107E-1,2.6162386E-1,-2.3092756E-1,-8.81848E-1,-3.306409E-1,-6.0916845E-2,-2.7005967E-2,-4.8582964E-2,-2.5790464E-2,-1.09862145E-2,-2.4432365E-3,-8.890659E-3,9.235412E-3,1.4838321E-3,4.2731274E-2,2.241555E-2,-7.0183687E-3,1.04161985E-2,2.423614E-2,1.0555499E-2,-5.4221237E-1,-3.384942E-1,-1.4674844E-2,-3.668941E-2,-1.0025931E-1,-4.0222955E-1,-1.6243069E-2,-5.080531E-2,-1.4310017E-1,-3.7199545E-1,8.1011415E-2,-1.4861092E-1,1.5588135E-1,-1.2610666E-1,5.460708E-2,3.267704E-1,1.3606614E-1,3.0471912E-1,-2.6095172E-2,-1.6079818E-1,-8.094865E-2,1.1684918E-1,2.6638085E-1,4.7083068E-1,1.6126806E-2,4.2879954E-2,4.183715E-1,8.1730986E-1,1.2141201E-2,4.1599334E-3,-2.3409922E-2,1.8796714E-1,2.7334237E-2,2.3375128E-1,-3.188515E-3,2.781041E-1,4.077754E-1,6.333304E-1,7.3393697E-1,5.095015E-1,4.2483066E-3,4.150774E-1,7.5658566E-1,2.3700253E-3,6.310982E-2,4.3348953E-2,3.3053808E-2,1.0844511E0,-1.0940394E-2,-3.7697602E-2,-4.802414E-2,-2.0656154E-2,-2.2204557E-2,-6.34162E-3,-1.8505545E-2,-4.0161528E-2,-3.0363137E-2,-4.3473333E-2,-2.5000747E-2,-1.2808534E-2,-5.340187E-3,2.8870638E-2,-1.8335072E-2,-1.0752752E-3,-5.1521573E-2,-1.752361E-2,-7.558031E-3,-2.0563016E-2,-9.0821314E-4,-1.5247818E-2,-2.8202167E-2,-1.6104328E-2,-4.099808E-3,-1.9056821E-2,-1.565715E-2,8.769333E-3,-1.0171308E-2,-2.2216499E-2,-4.9322145E-3,-1.26126055E-2,-2.024907E-2,-8.565736E-3,-3.4312916E-3,7.978079E-3,-1.0440493E-2,-5.9267477E-4,1.8845637E-3,1.4939871E-2,-1.4910512E-2,-9.596495E-6,5.23155E-3,2.1900011E-2,2.3730586E-3,1.4191457E-2,2.0925313E-2,9.581364E-3,-1.1872923E-2,-2.5892654E-3,-8.820982E-3,2.9977178E-3,-3.5408516E-3,1.0400621E-2,1.6581023E-2,7.011328E-3,2.6801364E-2,1.630262E-2,4.960699E-3,2.9306775E-2,2.4136E-2,4.4136874E-2,-4.1760267E-3,6.7278994E-3,1.2243859E-2,2.3804908E-3,8.299445E-3,1.8582081E-2,2.1360684E-2,8.204858E-3,7.3804175E-3,2.5946414E-2,1.8264726E-2,3.4341905E-2,3.8976297E-2,2.2296716E-2,1.0756037E-2,2.7067173E-2,1.2445498E-2,2.4886772E-2,2.2843042E-2,4.587449E-2,5.7590224E-2,3.4289002E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,-1,63,65,67,69,71,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,-1,121,123,125,127,129,131,133,135,-1,137,139,141,143,145,147,-1,-1,-1,149,151,153,155,-1,-1,157,159,161,163,-1,-1,-1,-1,-1,165,167,169,-1,-1,171,173,175,-1,-1,-1,-1,-1,177,-1,179,181,183,-1,185,187,189,-1,-1,191,-1,193,195,197,199,-1,201,-1,203,205,207,209,211,213,215,-1,217,219,221,223,-1,225,227,-1,229,231,233,235,-1,-1,-1,-1,237,239,241,243,-1,-1,-1,-1,245,-1,247,249,251,253,-1,-1,-1,-1,-1,255,-1,-1,257,259,261,263,265,267,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,269,271,-1,-1,273,275,-1,-1,277,279,281,283,285,287,-1,289,291,293,-1,295,297,299,301,303,-1,-1,305,307,-1,-1,309,311,-1,313,-1,315,317,319,321,323,-1,325,327,-1,-1,-1,-1,329,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.612547E2,5.9478928E1,1.154457E2,1.9272614E1,2.7669352E1,4.0731613E1,2.3370972E1,7.1423798E0,2.9693966E0,3.0311722E1,0E0,1.8143744E1,1.7808434E1,1.9412727E1,7.6464996E0,0E0,4.362732E0,1.0535622E0,1.0728179E0,1.1983856E1,2.7436438E0,2.8070393E0,1.1132084E1,1.5029755E1,2.6048803E0,9.231434E0,8.254257E0,2.0349655E0,2.8410187E0,4.9948425E0,2.0509577E0,7.779293E-1,0E0,3.748753E-1,0E0,7.8994865E0,9.9238205E-1,7.380489E-1,1.5560446E0,2.0313969E0,0E0,3.1941643E0,1.7560442E1,2.7210548E0,6.3931694E0,1.3548644E0,2.7155304E-1,2.6613894E0,7.9777074E-1,2.142852E0,4.6634064E0,1.1452293E0,7.007463E-1,8.6309814E-1,1.5364571E0,3.2333984E0,5.2240753E-1,0E0,8.1203365E-1,1.6490555E-1,3.6576748E-2,1.116589E-1,1.6869426E-1,3.9695702E0,2.3929727E0,9.0839577E-1,0E0,2.6161945E-1,2.4659812E-1,9.9856186E-1,6.9315374E-1,1.3320808E0,8.8308144E-1,1.6831341E0,1.4047539E0,0E0,6.044396E0,1.3732238E0,1.0620888E0,2.1915398E0,2.007187E0,8.6881864E-1,0E0,0E0,0E0,1.0730317E0,1.438694E0,1.1256039E-1,9.7635984E-2,0E0,0E0,1.1464462E0,3.3967361E0,9.464817E-1,3.1971765E-1,0E0,0E0,0E0,0E0,0E0,1.4497566E0,1.2244492E0,1.6436291E0,0E0,0E0,2.2425556E-1,5.4584503E-2,6.0905457E-2,0E0,0E0,0E0,0E0,0E0,8.1380844E-2,0E0,3.2020807E0,2.5279331E0,6.958029E-1,0E0,5.961399E-1,1.05950356E-1,4.539299E-2,0E0,0E0,4.7383234E-2,0E0,2.370758E-1,3.030853E-1,7.944727E-2,4.3726635E-1,0E0,5.567074E-1,0E0,5.9436274E-1,1.0816226E0,6.0632277E-1,8.381907E-1,8.44289E0,2.1076446E0,1.040988E0,0E0,3.70417E-1,2.5397077E-1,1.6123924E0,5.227823E-1,0E0,6.962452E-1,4.0458858E-2,0E0,3.7410492E-1,5.0975204E-1,4.493823E-1,5.0431824E-1,0E0,0E0,0E0,0E0,6.8288803E-1,3.9069748E-1,2.555458E0,1.1633148E0,0E0,0E0,0E0,0E0,2.438469E-1,0E0,9.6632004E-1,5.5249023E-1,4.0319502E-1,5.030842E-1,0E0,0E0,0E0,0E0,0E0,1.2067795E-2,0E0,0E0,1.7846508E0,1.1103401E0,2.7693653E-1,1.6204815E0,9.736812E-2,8.4001696E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3430595E-2,2.5471973E-1,0E0,0E0,6.42625E-1,2.042923E-1,0E0,0E0,8.6696446E-2,1.3063002E-1,3.2885078E-1,3.1931943E-1,2.9481864E0,5.8577003E0,0E0,4.4942355E-1,1.2183462E0,7.4436045E-1,0E0,6.303088E-2,1.8573442E-1,2.375525E-1,5.3933096E-1,8.105583E-1,0E0,0E0,5.319946E-1,8.0863E-2,0E0,0E0,2.3948734E-1,9.244102E-2,0E0,3.3073092E-1,0E0,1.5100563E-1,8.5121584E-1,1.678648E-1,1.9581604E-1,3.0030632E-1,0E0,4.1260958E-2,5.474987E-1,0E0,0E0,0E0,0E0,4.9999237E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,78,78,79,79,80,80,81,81,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,100,100,101,101,102,102,105,105,106,106,107,107,113,113,115,115,116,116,117,117,119,119,120,120,121,121,124,124,126,126,127,127,128,128,129,129,131,131,133,133,134,134,135,135,136,136,137,137,138,138,139,139,141,141,142,142,143,143,144,144,146,146,147,147,149,149,150,150,151,151,152,152,157,157,158,158,159,159,160,160,165,165,167,167,168,168,169,169,170,170,176,176,179,179,180,180,181,181,182,182,183,183,184,184,199,199,200,200,203,203,204,204,207,207,208,208,209,209,210,210,211,211,212,212,214,214,215,215,216,216,218,218,219,219,220,220,221,221,222,222,225,225,226,226,229,229,230,230,232,232,234,234,235,235,236,236,237,237,238,238,240,240,241,241,246,246],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,-1,64,66,68,70,72,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,-1,122,124,126,128,130,132,134,136,-1,138,140,142,144,146,148,-1,-1,-1,150,152,154,156,-1,-1,158,160,162,164,-1,-1,-1,-1,-1,166,168,170,-1,-1,172,174,176,-1,-1,-1,-1,-1,178,-1,180,182,184,-1,186,188,190,-1,-1,192,-1,194,196,198,200,-1,202,-1,204,206,208,210,212,214,216,-1,218,220,222,224,-1,226,228,-1,230,232,234,236,-1,-1,-1,-1,238,240,242,244,-1,-1,-1,-1,246,-1,248,250,252,254,-1,-1,-1,-1,-1,256,-1,-1,258,260,262,264,266,268,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,270,272,-1,-1,274,276,-1,-1,278,280,282,284,286,288,-1,290,292,294,-1,296,298,300,302,304,-1,-1,306,308,-1,-1,310,312,-1,314,-1,316,318,320,322,324,-1,326,328,-1,-1,-1,-1,330,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.126489E7,1E0,2.62797E5,4.930349E5,1.2974394E6,3E0,1.19507775E-1,1.5343539E0,9.6965484E-2,8.2E1,5.9572783E0,1.2679E4,5.964E3,-8.790864E-2,5.121E3,3.771981E6,3.1294638E2,1E0,1.3097832E3,1.5141565E3,6.76E2,9.27E2,7.5179994E-1,3.9712732E7,2.7165637E4,1E0,2.1226158E0,4.987639E7,1.84442E5,1.5669752E-2,-6.349393E-4,1.4615384E0,9.24134E-3,3.4976208E-1,1.873E3,2.0783027E5,8.7643677E-1,1.1146532E8,1.3712632E-2,7.805608E4,2.255481E6,4.12E0,2.539E3,4.5085418E4,3.928E4,2.4513194E5,8.965631E7,3.3743455E0,6.4683E4,2E0,7.697189E2,1E0,1.2693982E6,5.0633432E2,1.5981E4,3.816318E-3,1.1879E4,4.653159E9,4.7436612E2,2.1166662E6,1.4000518E6,1.2199979E10,2.4435181E12,1.9768514E3,-6.105108E-2,5.148149E5,3.178082E6,2.1924436E5,1.6194E4,4E1,3.5612745E0,1.2513207E2,1.5409575E2,5.7356223E-2,9.075E3,4.2E1,1E0,5.5669255E6,6.8E1,4.38E2,-2.8305247E-2,-1.7477503E-2,-3.7980504E-2,9.923929E2,5.8018835E2,1.6E1,9.2593566E-2,-2.8827947E-2,1.2056223E-2,2.4285715E0,1E0,8.490231E-1,6.445E3,3.2734256E-3,2.3913745E-2,3.6069732E-2,6.857098E-2,1.1416004E-2,6.292039E0,1.345454E8,1.708775E7,-7.001936E-2,-4.2799454E-2,5.5544252E7,8.081268E-1,1.0376E4,-1.750105E-2,-2.1042898E-2,-1.26572605E-2,1.7476627E-3,-8.531828E-3,3.6E1,-7.250243E-3,3.960925E3,4.987639E7,2.161716E8,2.381122E-2,2.9910075E5,2.0601392E3,5.884028E0,7.879926E-3,1.950158E-2,1.5E1,8.236291E-3,3.1E1,3.5E0,6.7995167E0,4.3875E1,1.0052045E-3,4.0844156E7,-5.9067555E-2,2.183847E0,1.9E1,9.25E0,1.8480903E0,2.1E1,3.0654462E2,2.3314082E6,-1.666822E-2,1.901875E2,6.0559247E2,1.273801E6,1E0,6.53073E-2,7.5773336E7,3.7173E4,-1.5522279E-2,8.360387E-1,3.353E3,1.0396807E8,1.6143107E0,-1.0371088E-2,-8.3886157E-4,-2.9264627E-2,-1.3114839E-2,1.251806E6,4.7279915E6,3.666495E7,3.275343E6,2.0820746E-2,4.58317E-2,9.526829E-3,2.8231652E-2,1.9121015E0,1.7282614E-2,2.127E3,2.183E3,2.7272727E0,1.988805E8,-2.2037037E-2,-4.0469084E-2,-7.918204E-3,-1.9080754E-2,-2.2114241E-2,5.244755E-3,-2.054786E-2,-1.0104008E-2,1E0,5.84E2,1.7243155E12,3.0387878E3,7.7628184E7,2.2950048E10,-2.7005967E-2,-4.8582964E-2,-2.5790464E-2,-1.09862145E-2,-2.4432365E-3,-8.890659E-3,9.235412E-3,1.4838321E-3,4.2731274E-2,2.241555E-2,-7.0183687E-3,1.04161985E-2,2.423614E-2,1.0555499E-2,8E0,5.7018555E4,-1.4674844E-2,-3.668941E-2,4.7E2,5.836513E3,-1.6243069E-2,-5.080531E-2,6.763314E7,1.28078E6,3.5476844E7,2.165022E2,3.3030225E2,7.85939E5,5.460708E-2,1.9221336E1,7E2,3.4653E4,-2.6095172E-2,7.2565425E5,1.1490676E6,2.5464671E2,1.8955729E1,3.7333333E0,1.6126806E-2,4.2879954E-2,4.0417146E8,6.2671E4,1.2141201E-2,4.1599334E-3,5.332332E-2,5.893994E4,2.7334237E-2,1.249E3,-3.188515E-3,1.1906E4,6.171E3,8.5023944E2,3.3406114E11,6.316106E6,4.2483066E-3,8.791777E3,7.3610186E3,2.3700253E-3,6.310982E-2,4.3348953E-2,3.3053808E-2,1.784E3,-1.0940394E-2,-3.7697602E-2,-4.802414E-2,-2.0656154E-2,-2.2204557E-2,-6.34162E-3,-1.8505545E-2,-4.0161528E-2,-3.0363137E-2,-4.3473333E-2,-2.5000747E-2,-1.2808534E-2,-5.340187E-3,2.8870638E-2,-1.8335072E-2,-1.0752752E-3,-5.1521573E-2,-1.752361E-2,-7.558031E-3,-2.0563016E-2,-9.0821314E-4,-1.5247818E-2,-2.8202167E-2,-1.6104328E-2,-4.099808E-3,-1.9056821E-2,-1.565715E-2,8.769333E-3,-1.0171308E-2,-2.2216499E-2,-4.9322145E-3,-1.26126055E-2,-2.024907E-2,-8.565736E-3,-3.4312916E-3,7.978079E-3,-1.0440493E-2,-5.9267477E-4,1.8845637E-3,1.4939871E-2,-1.4910512E-2,-9.596495E-6,5.23155E-3,2.1900011E-2,2.3730586E-3,1.4191457E-2,2.0925313E-2,9.581364E-3,-1.1872923E-2,-2.5892654E-3,-8.820982E-3,2.9977178E-3,-3.5408516E-3,1.0400621E-2,1.6581023E-2,7.011328E-3,2.6801364E-2,1.630262E-2,4.960699E-3,2.9306775E-2,2.4136E-2,4.4136874E-2,-4.1760267E-3,6.7278994E-3,1.2243859E-2,2.3804908E-3,8.299445E-3,1.8582081E-2,2.1360684E-2,8.204858E-3,7.3804175E-3,2.5946414E-2,1.8264726E-2,3.4341905E-2,3.8976297E-2,2.2296716E-2,1.0756037E-2,2.7067173E-2,1.2445498E-2,2.4886772E-2,2.2843042E-2,4.587449E-2,5.7590224E-2,3.4289002E-2],"split_indices":[20,56,12,102,2,28,28,3,38,42,0,29,53,2,2,0,2,1,52,104,4,55,2,2,27,45,33,6,54,45,29,38,0,53,0,39,0,33,57,31,0,28,31,54,2,33,12,28,45,54,2,17,4,102,47,55,9,0,2,5,4,48,33,5,31,4,0,28,45,28,9,3,53,52,55,0,2,3,102,28,0,0,0,0,0,52,52,8,38,0,0,57,102,53,29,0,0,0,0,0,38,7,45,0,0,45,27,9,0,0,0,0,0,3,0,4,45,12,0,28,52,53,0,0,3,0,3,53,54,4,0,45,0,54,3,58,53,3,33,28,0,4,33,9,102,0,7,29,0,39,10,7,42,0,0,0,0,29,47,51,29,0,0,0,0,53,0,1,2,56,7,0,0,0,0,0,57,0,0,59,0,31,4,12,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,18,45,0,0,1,28,0,0,7,12,7,52,52,1,0,53,2,9,0,28,28,4,58,54,0,0,5,1,0,0,57,28,0,0,0,29,10,52,31,1,0,52,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.553E3,7.94E2,1.759E3,4E2,3.94E2,1.212E3,5.47E2,2.75E2,1.25E2,3.89E2,5E0,7.48E2,4.64E2,3.77E2,1.7E2,9E0,2.66E2,8.4E1,4.1E1,3.14E2,7.5E1,9.9E1,6.49E2,4.29E2,3.5E1,1.83E2,1.94E2,7.7E1,9.3E1,2.3E2,3.6E1,8E1,4E0,3.5E1,6E0,2.67E2,4.7E1,3.1E1,4.4E1,9.5E1,4E0,1.7E2,4.79E2,1.97E2,2.32E2,1.8E1,1.7E1,1.58E2,2.5E1,1.1E1,1.83E2,5.6E1,2.1E1,4.6E1,4.7E1,1.97E2,3.3E1,5E0,3.1E1,5.1E1,2.9E1,9E0,2.6E1,1.7E2,9.7E1,4E1,7E0,1.6E1,1.5E1,2.4E1,2E1,7E1,2.5E1,5.5E1,1.15E2,1.3E1,4.66E2,1.6E2,3.7E1,1.91E2,4.1E1,1.4E1,4E0,5E0,1.2E1,8.2E1,7.6E1,1.3E1,1.2E1,5E0,6E0,1E2,8.3E1,4.3E1,1.3E1,6E0,1.5E1,6E0,4E1,4E0,4.3E1,1.5E2,4.7E1,2.4E1,9E0,2.1E1,1E1,4.5E1,6E0,2.1E1,8E0,5E0,4E0,1.9E1,7E0,1.37E2,3.3E1,9E1,7E0,2.7E1,1.3E1,1.2E1,4E0,7E0,8E0,4E0,2E1,8E0,1.2E1,6.4E1,6E0,2E1,5E0,4.3E1,1.2E1,5.1E1,6.4E1,4.43E2,2.3E1,1.56E2,4E0,1.3E1,2.4E1,1.73E2,1.8E1,9E0,3.2E1,1E1,4E0,3.5E1,4.7E1,1.7E1,5.9E1,8E0,5E0,8E0,4E0,8.3E1,1.7E1,2.6E1,5.7E1,7E0,3.6E1,5E0,8E0,3.8E1,5E0,1.8E1,1.32E2,1.8E1,2.9E1,7E0,1.4E1,4E0,6E0,9E0,3.6E1,1.3E1,6E0,1.29E2,8E0,9E0,2.4E1,1E1,8E1,1E1,1.7E1,9E0,4E0,7E0,5E0,4E0,4E0,1.4E1,6E0,4E0,4E0,8E0,4E0,3.9E1,2.5E1,6E0,1.4E1,9E0,3.4E1,5E0,7E0,2.2E1,2.9E1,2.6E1,3.8E1,1.74E2,2.69E2,4E0,1.9E1,9.3E1,6.3E1,5E0,8E0,1.2E1,1.2E1,7E1,1.03E2,4E0,1.4E1,1E1,2.2E1,6E0,4E0,2.3E1,1.2E1,6E0,4.1E1,5E0,1.2E1,3.1E1,2.8E1,4E1,4.3E1,6E0,1.1E1,1.9E1,7E0,2.4E1,3.3E1,8E0,3E1,6E0,1.2E1,1.28E2,4E0,1E1,8E0,6E0,2.3E1,2.8E1,8E0,5.6E1,7.3E1,4E0,4E0,5E0,4E0,1.8E1,6E0,4E0,6E0,6.9E1,1.1E1,3.4E1,5E0,4E0,2.1E1,5E0,4E0,7E0,2.7E1,1.7E1,5E0,2.4E1,5E0,9E0,1.7E1,2.6E1,1.2E1,9.6E1,7.8E1,1.13E2,1.56E2,7E0,1.2E1,5.9E1,3.4E1,3E1,3.3E1,4E0,4E0,7E0,5E0,4E0,8E0,4.5E1,2.5E1,6.9E1,3.4E1,4E0,6E0,5E0,1.7E1,1.7E1,6E0,8E0,4E0,2.9E1,1.2E1,4E0,8E0,1E1,2.1E1,6E0,2.2E1,3.3E1,7E0,5E0,3.8E1,5E0,6E0,8E0,1.1E1,2.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"331","size_leaf_vector":"1"}},{"base_weights":[4.153013E-3,-4.3273845E-1,2.1360524E-1,-6.994161E-1,-1.9081418E-1,1.1674788E-2,5.001998E-1,-8.218031E-1,-3.9641276E-1,-3.2348228E-1,3.435763E-1,-2.9068905E-1,9.88729E-2,2.9006276E-1,6.898961E-1,-8.8036597E-1,-4.603801E-1,-5.583068E-1,-2.4901451E-1,-3.5629463E-1,8.0529936E-2,1.0898223E-1,4.9938038E-1,-4.1801804E-1,-1.6740522E-1,1.4397107E-1,-4.600328E-1,7.714198E-2,4.857065E-1,5.4220146E-1,1.0732266E0,-8.074587E-1,-1.2335455E0,-3.2136142E-1,-5.444892E-2,-5.9610593E-1,-7.822923E-3,-2.993001E-1,-5.5228163E-2,-2.4874265E-1,-7.1392155E-1,-1.585566E-2,2.215138E-1,5.787563E-1,1.0812604E-1,-3.939431E-1,-4.245257E-2,-2.2733799E-1,6.2240902E-2,1.1087369E0,1.22150026E-1,-7.544266E-1,-1.5030456E-1,1.7545813E-1,-3.6785212E-1,3.8828182E-1,7.3227954E-1,2.7803355E-1,6.990667E-1,1.1418579E0,7.313141E-1,-5.4031044E-1,-8.644353E-1,-4.6167966E-2,-6.938012E-2,5.0723553E-2,-4.3348747E-1,-3.1310484E-2,-3.9082938E-1,-3.727601E-1,-2.1386813E-1,-1.4918713E-1,7.1819783E-3,-3.494461E-1,-1.8512454E-2,-8.3890545E-1,-2.750827E-1,1.0922807E-2,-9.845795E-2,1.4993535E-1,1.756884E-2,2.5285363E-1,6.828836E-1,-4.823524E-3,1.3024099E-2,-4.823201E-1,-2.7949035E-1,-1.378649E-1,-3.119741E-1,-6.915388E-3,1.3288541E-1,3.3562448E-2,6.51461E-2,-6.491289E-2,2.524929E-1,-1.991034E-2,-3.9666682E-2,-5.397343E-1,4.2295545E-2,2.7263084E-1,-7.072678E-2,-8.742842E-2,-5.256012E-1,4.130802E-1,-9.440265E-3,8.101595E-1,3.8958877E-1,4.3363035E-1,-9.7340636E-2,6.2069845E-1,5.8608655E-2,9.8941255E-1,6.583781E-2,2.1424038E-2,4.3033406E-2,-3.143575E-2,-2.9543063E-1,-1.4293025E-2,-8.8458884E-1,-6.321011E-3,1.0883087E-2,-5.115191E-1,-3.753677E-3,-2.684927E-2,-8.301923E-3,-2.0931E-2,-1.1110184E-2,-4.588295E-3,-2.3978703E-1,-3.5094274E-3,-9.499449E-3,-1.8285613E-1,-4.9869412E-1,-6.993598E-2,3.2203924E-2,-7.408482E-1,-5.5721827E-2,-3.6139052E-2,-1.07350364E-1,-3.30912E-2,-1.046801E-2,9.082243E-3,2.8124698E-3,1.7533164E-2,4.813148E-3,5.40025E-1,4.712706E-2,-5.182605E-1,-1.4908322E-2,3.4048995E-3,-3.205429E-1,-2.0410602E-1,-6.587943E-2,-3.4491625E-1,-3.7204907E-3,1.9340563E-1,7.53043E-5,-1.2549068E-1,2.7307287E-1,2.2464116E-1,7.9968816E-1,-3.7954215E-2,-1.3319363E-2,1.434655E-1,-1.54967345E-2,-8.872951E-3,2.9902887E-1,-2.9327092E-1,8.058292E-2,-1.1945389E-2,8.628206E-3,-3.5907976E-2,-3.198796E-1,5.4867727E-1,2.8953224E-1,4.366442E-2,2.268705E-2,7.28184E-3,2.5388824E-2,6.681609E-1,3.0512232E-1,1.1550248E-2,-1.6347421E-2,6.812128E-1,3.703312E-1,3.209068E-2,1.0514699E0,-1.9447744E-2,-4.311654E-3,-9.3137366E-1,-6.791503E-1,-2.991986E-2,-1.7333768E-2,-6.9668656E-3,-1.31883975E-2,-3.4071648E-1,-3.454012E-2,-6.3006055E-1,-3.116652E-1,-2.5290513E-2,-1.9084126E-2,-3.933997E-1,-8.109414E-1,4.900166E-3,-1.2751141E-2,3.7542894E-4,-3.7574286E-3,3.3004382E-1,6.5143836E-1,-1.8712481E-2,-2.752557E-2,-2.4726798E-1,-2.2979597E-2,-2.4871434E-1,7.569175E-4,-2.8050367E-2,-8.728755E-3,-3.6874917E-1,-4.975152E-3,5.3993445E-3,1.1654222E-2,-2.3161295E-1,-2.2533406E-2,1.3552561E-1,4.2381614E-1,1.4996633E-1,4.578974E-1,5.734977E-1,6.813254E-2,-8.5236935E-4,2.448361E-2,3.334257E-1,-5.5233564E-2,-2.703161E-2,-1.6561419E-1,-5.1271178E-2,1.960957E-1,-4.7149383E-3,-2.1748828E-2,6.362229E-1,3.3549285E-1,2.1741581E-1,5.7236856E-1,3.9358106E-2,2.3635902E-2,3.6906564E-1,1.1890221E-1,9.8626865E-3,-4.0712286E-2,5.988961E-1,9.786657E-1,4.7866818E-1,-3.4722376E-3,5.802374E-2,3.952805E-2,-4.2580985E-2,-5.457986E-2,-1.1505644E-2,-3.752918E-2,-1.87427E-2,-2.1736277E-4,-4.88105E-3,1.04364725E-2,-2.5228322E-2,-4.1477278E-2,-2.8592682E-2,-1.0030595E-2,1.2418705E-3,-1.5971765E-2,-2.6173525E-2,-7.905513E-3,-4.7093533E-2,-3.023466E-2,2.1153476E-2,9.857418E-3,3.6220554E-2,1.8699456E-2,-1.40711125E-2,-5.662896E-3,-1.5600285E-2,-2.424043E-3,-4.1361577E-3,9.632126E-4,-1.009765E-2,-1.9711213E-2,-1.4235286E-2,-1.0927678E-3,-3.635271E-3,1.6666748E-2,1.4143884E-2,3.6993104E-3,2.5373686E-2,9.157852E-3,3.1645496E-3,1.4556261E-2,1.47771E-2,3.3138197E-2,3.207614E-2,1.4715729E-2,3.4851786E-3,-6.384055E-3,1.2603003E-2,2.8231535E-2,-7.606981E-3,2.639274E-3,-1.961323E-3,-1.1186957E-2,2.337221E-4,-6.5223947E-3,1.5637218E-3,1.3284291E-2,1.453949E-2,3.44065E-2,2.0276146E-2,7.845119E-3,-1.7162608E-3,1.336212E-2,3.514547E-2,1.510463E-2,8.620801E-3,2.0938858E-2,9.585478E-3,-5.1554124E-4,4.2124093E-3,-7.452831E-3,2.294527E-2,3.3105813E-2,6.3674964E-2,3.579108E-2,1.0404585E-2,2.9664414E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,-1,69,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,-1,-1,119,121,-1,123,125,127,129,-1,131,133,135,137,-1,139,141,-1,143,145,-1,-1,147,149,151,153,-1,155,-1,-1,157,159,-1,-1,161,163,165,167,169,171,173,-1,175,177,179,181,183,-1,185,-1,-1,-1,-1,187,-1,189,-1,-1,191,-1,-1,-1,-1,-1,-1,193,-1,-1,195,197,199,-1,201,-1,-1,203,205,-1,-1,-1,-1,-1,207,-1,209,-1,-1,211,213,215,217,-1,219,-1,221,223,225,227,-1,-1,229,-1,-1,231,233,235,-1,-1,-1,237,239,241,-1,-1,-1,-1,243,245,247,-1,249,251,-1,253,-1,-1,255,257,-1,-1,-1,-1,259,261,263,265,-1,267,269,271,-1,-1,-1,-1,273,275,-1,-1,277,-1,279,-1,281,-1,283,-1,-1,-1,285,287,289,291,293,295,297,-1,299,-1,301,303,-1,305,307,309,-1,-1,311,313,315,317,-1,-1,319,321,-1,323,325,327,329,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3352905E2,5.3221115E1,9.982367E1,1.4248108E1,3.0951145E1,2.6737047E1,2.8221634E1,5.5E0,2.6052074E0,2.2734684E1,3.1254873E0,3.4933796E0,1.9903055E1,1.4125782E1,2.0592438E1,5.158066E0,3.2368145E0,7.266922E-1,5.8763885E-1,1.3053856E1,0E0,5.0924754E-1,1.5523415E0,8.186703E-1,1.624459E0,1.521162E1,5.26635E0,7.2395606E0,3.9676132E0,1.1076973E1,1.5819321E0,2.5924072E0,5.881729E-1,1.4822028E0,0E0,6.2099457E-2,0E0,2.2779417E-1,2.831652E-1,6.1607323E0,4.102207E0,3.7012422E-1,1.24212325E-1,1.2865524E0,3.37331E-1,9.475002E-1,0E0,6.612463E-1,3.727458E-1,4.881935E-1,1.7425362E1,1.0479164E-1,2.2729545E0,3.248814E0,1.2811456E0,1.9245415E0,9.9211884E-1,6.058133E0,5.5853806E0,9.472046E-1,4.1691685E-1,6.406803E-1,1.6586227E0,0E0,0E0,2.9406044E-1,7.165928E-1,0E0,2.0699024E-1,8.034992E-2,5.706036E-2,1.8960997E-2,0E0,4.5085087E0,2.8408523E0,8.26355E-1,1.3506703E0,0E0,9.468566E-2,3.1501442E-2,0E0,1.5035605E-1,6.805563E-1,0E0,0E0,2.3544121E-1,7.211342E-1,2.1066839E-1,3.4205675E-1,0E0,1.5077835E-1,0E0,0E0,6.0537505E0,6.23654E0,0E0,0E0,3.9810014E-1,7.9327136E-1,1.2059073E0,1.336883E0,5.1459074E-1,5.2464914E-1,1.9184799E0,0E0,4.880829E-1,2.4388611E-1,1.9962807E0,7.795831E-1,1.9900398E0,0E0,3.8038254E-1,0E0,0E0,0E0,0E0,2.0178163E-1,0E0,8.8607025E-1,0E0,0E0,1.08345985E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.16842985E-2,0E0,0E0,2.0720077E0,2.1868572E0,1.698368E0,0E0,8.499584E-1,0E0,0E0,4.8057157E-1,1.7662676E-2,0E0,0E0,0E0,0E0,0E0,3.230753E-1,0E0,1.8779755E-2,0E0,0E0,3.3557796E-1,2.3850137E-1,9.389009E-2,2.0302486E-1,0E0,1.8992752E-2,0E0,2.7262957E0,8.756063E-1,6.9335155E0,1.7688494E0,0E0,0E0,8.5060394E-1,0E0,0E0,1.1394825E0,4.1637778E-1,3.7559408E-1,0E0,0E0,0E0,2.4516809E-1,8.690262E-1,1.2421665E0,0E0,0E0,0E0,0E0,2.1046925E-1,5.3958607E-1,2.2170466E-1,0E0,2.3031502E0,1.5103827E0,0E0,2.2472763E-1,0E0,0E0,6.750107E-1,8.92622E-1,0E0,0E0,0E0,0E0,4.9574137E-1,7.428074E-1,9.8291206E-1,1.0990219E0,0E0,9.323802E-1,2.2385037E-1,4.8583412E-1,0E0,0E0,0E0,0E0,4.036057E-2,1.3783932E-2,0E0,0E0,1.1067116E-1,0E0,2.391069E-1,0E0,5.080832E-2,0E0,7.899189E-2,0E0,0E0,0E0,1.3683777E0,2.3132994E0,2.0585027E-1,3.312161E-1,3.7287827E0,3.0332336E0,1.1649847E-1,0E0,1.2539732E-1,0E0,1.4370136E0,1.02380194E-1,0E0,7.957524E-2,5.3827062E-2,1.4114973E-1,0E0,0E0,5.227852E-1,1.7469335E-1,7.016034E-1,2.4734402E-1,0E0,0E0,2.8139782E-1,1.3392492E-1,0E0,2.5660557E-1,5.405388E-1,9.4904137E-1,6.539931E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,68,68,69,69,70,70,71,71,73,73,74,74,75,75,76,76,78,78,79,79,81,81,82,82,85,85,86,86,87,87,88,88,90,90,93,93,94,94,97,97,98,98,99,99,100,100,101,101,102,102,103,103,105,105,106,106,107,107,108,108,109,109,111,111,116,116,118,118,121,121,128,128,131,131,132,132,133,133,135,135,138,138,139,139,145,145,147,147,150,150,151,151,152,152,153,153,155,155,157,157,158,158,159,159,160,160,163,163,166,166,167,167,168,168,172,172,173,173,174,174,179,179,180,180,181,181,183,183,184,184,186,186,189,189,190,190,195,195,196,196,197,197,198,198,200,200,201,201,202,202,207,207,208,208,211,211,213,213,215,215,217,217,221,221,222,222,223,223,224,224,225,225,226,226,227,227,229,229,231,231,232,232,234,234,235,235,236,236,239,239,240,240,241,241,242,242,245,245,246,246,248,248,249,249,250,250,251,251],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,-1,70,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,-1,-1,120,122,-1,124,126,128,130,-1,132,134,136,138,-1,140,142,-1,144,146,-1,-1,148,150,152,154,-1,156,-1,-1,158,160,-1,-1,162,164,166,168,170,172,174,-1,176,178,180,182,184,-1,186,-1,-1,-1,-1,188,-1,190,-1,-1,192,-1,-1,-1,-1,-1,-1,194,-1,-1,196,198,200,-1,202,-1,-1,204,206,-1,-1,-1,-1,-1,208,-1,210,-1,-1,212,214,216,218,-1,220,-1,222,224,226,228,-1,-1,230,-1,-1,232,234,236,-1,-1,-1,238,240,242,-1,-1,-1,-1,244,246,248,-1,250,252,-1,254,-1,-1,256,258,-1,-1,-1,-1,260,262,264,266,-1,268,270,272,-1,-1,-1,-1,274,276,-1,-1,278,-1,280,-1,282,-1,284,-1,-1,-1,286,288,290,292,294,296,298,-1,300,-1,302,304,-1,306,308,310,-1,-1,312,314,316,318,-1,-1,320,322,-1,324,326,328,330,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,7.7344055E4,1E0,1.5343539E0,4.67E2,5.704E3,5.121E3,1.2616388E-1,1.3144558E4,1.3097832E3,7.805608E4,4.0844156E7,7.475293E5,1.2727361E6,4.0844156E7,1.0828989E8,1.436943E6,2.728889E2,6.747114E7,8.0529936E-2,2.822937E5,3.9715407E0,2.6904478E5,3.1961178E8,8E0,5.1232258E1,1E0,3.171E3,1.2679E4,2.3977574E7,1.3062E4,3.3487454E11,2.1693E5,-5.444892E-2,2.4859156E1,-7.822923E-3,9.490825E5,2.579E3,3.164136E6,3.0497742E3,1.1E1,3.333E3,2.7271206E5,2E1,3.3151623E1,-4.245257E-2,9.25E0,3.6E2,7.425E3,2.091623E5,3.7263157E1,6.624E3,9.213145E0,1.980293E7,1.4955851E7,1E0,2.1690162E7,7.096131E0,4.1973075E6,6.72E2,1.9121015E0,1E0,-4.6167966E-2,-6.938012E-2,1.4783E4,3.4409692E3,-3.1310484E-2,1.06E3,1E0,1.85221E5,2.6680525E6,7.1819783E-3,1.3304372E7,7.3610186E3,1.2427474E8,2.7635896E-1,1.0922807E-2,2.8216E4,2.9290114E7,1.756884E-2,1.6081998E5,1.3185129E0,-4.823524E-3,1.3024099E-2,1.14E2,9.928469E-1,1.1805985E3,1.8441667E2,-6.915388E-3,1.3170732E0,3.3562448E-2,6.51461E-2,7.06699E8,9.908038E1,-1.991034E-2,-3.9666682E-2,7.854E3,1.066317E12,5.52E2,1.2165793E3,9.721307E6,1.5824156E5,2.9375E0,-9.440265E-3,2.637815E1,8.319452E2,1.3035088E1,8.657441E7,3.5301748E7,5.8608655E-2,4.2121835E6,6.583781E-2,2.1424038E-2,4.3033406E-2,-3.143575E-2,1E0,-1.4293025E-2,4.6253732E2,-6.321011E-3,1.0883087E-2,1.4157E4,-3.753677E-3,-2.684927E-2,-8.301923E-3,-2.0931E-2,-1.1110184E-2,-4.588295E-3,1.227133E6,-3.5094274E-3,-9.499449E-3,3.098804E2,1E0,1E0,3.2203924E-2,2.060792E8,-5.5721827E-2,-3.6139052E-2,9.35E2,1E0,-1.046801E-2,9.082243E-3,2.8124698E-3,1.7533164E-2,4.813148E-3,5.0079144E5,4.712706E-2,1.901E3,-1.4908322E-2,3.4048995E-3,1.1E2,2.1434232E2,3.3502305E0,3.30399E6,-3.7204907E-3,6.9E1,7.53043E-5,7.805608E4,1.4777102E9,2.3314082E6,9.287868E6,-3.7954215E-2,-1.3319363E-2,5.094239E3,-1.54967345E-2,-8.872951E-3,1.216918E7,2.253E3,8.817919E-1,-1.1945389E-2,8.628206E-3,-3.5907976E-2,1.8722344E8,9.073025E6,6.860185E2,4.366442E-2,2.268705E-2,7.28184E-3,2.5388824E-2,4.29E2,1.54E2,4.12204E5,-1.6347421E-2,5.185489E3,6.061914E-1,3.209068E-2,1.8262731E0,-1.9447744E-2,-4.311654E-3,9.517604E6,5.5700205E8,-2.991986E-2,-1.7333768E-2,-6.9668656E-3,-1.31883975E-2,1.7022566E7,6.6473596E2,5.3421542E-2,9.969697E0,-2.5290513E-2,1.401692E1,5.012E3,2.0401025E3,4.900166E-3,-1.2751141E-2,3.7542894E-4,-3.7574286E-3,2.2797407E9,1.6527965E7,-1.8712481E-2,-2.752557E-2,3.0404909E0,-2.2979597E-2,4E0,7.569175E-4,1E0,-8.728755E-3,3E1,-4.975152E-3,5.3993445E-3,1.1654222E-2,9.51668E5,1.1925198E3,8.289561E8,1.1317E4,1.838444E6,1.048E3,3.0161016E7,6.813254E-2,3.6158625E4,2.448361E-2,6.4446027E2,1.4571098E5,-2.703161E-2,4.822581E0,2.073E3,1.2152382E5,-4.7149383E-3,-2.1748828E-2,9.27E2,1.9473622E7,2.0697437E2,7.6842415E6,3.9358106E-2,2.3635902E-2,3.3737933E8,3.266054E3,9.8626865E-3,1.1457406E-1,2.4911E4,1.5693306E11,4.7279915E6,-3.4722376E-3,5.802374E-2,3.952805E-2,-4.2580985E-2,-5.457986E-2,-1.1505644E-2,-3.752918E-2,-1.87427E-2,-2.1736277E-4,-4.88105E-3,1.04364725E-2,-2.5228322E-2,-4.1477278E-2,-2.8592682E-2,-1.0030595E-2,1.2418705E-3,-1.5971765E-2,-2.6173525E-2,-7.905513E-3,-4.7093533E-2,-3.023466E-2,2.1153476E-2,9.857418E-3,3.6220554E-2,1.8699456E-2,-1.40711125E-2,-5.662896E-3,-1.5600285E-2,-2.424043E-3,-4.1361577E-3,9.632126E-4,-1.009765E-2,-1.9711213E-2,-1.4235286E-2,-1.0927678E-3,-3.635271E-3,1.6666748E-2,1.4143884E-2,3.6993104E-3,2.5373686E-2,9.157852E-3,3.1645496E-3,1.4556261E-2,1.47771E-2,3.3138197E-2,3.207614E-2,1.4715729E-2,3.4851786E-3,-6.384055E-3,1.2603003E-2,2.8231535E-2,-7.606981E-3,2.639274E-3,-1.961323E-3,-1.1186957E-2,2.337221E-4,-6.5223947E-3,1.5637218E-3,1.3284291E-2,1.453949E-2,3.44065E-2,2.0276146E-2,7.845119E-3,-1.7162608E-3,1.336212E-2,3.514547E-2,1.510463E-2,8.620801E-3,2.0938858E-2,9.585478E-3,-5.1554124E-4,4.2124093E-3,-7.452831E-3,2.294527E-2,3.3105813E-2,6.3674964E-2,3.579108E-2,1.0404585E-2,2.9664414E-2],"split_indices":[20,56,33,102,42,2,2,2,39,52,4,28,45,28,28,45,45,1,52,45,0,28,57,47,31,18,56,6,2,2,47,9,31,29,0,58,0,33,2,1,52,3,2,28,3,55,0,58,2,1,28,47,2,53,5,47,6,45,35,28,0,53,79,0,0,9,52,0,2,8,9,45,0,45,4,45,27,0,9,32,0,28,34,0,0,29,53,33,52,0,53,0,0,7,56,0,0,9,31,2,4,12,33,54,0,56,52,54,45,45,0,50,0,0,0,0,8,0,4,0,0,9,0,0,0,0,0,0,45,0,0,52,59,16,0,7,0,0,0,8,0,0,0,0,0,28,0,9,0,0,10,4,53,9,0,10,0,28,7,28,1,0,0,4,0,0,45,2,39,0,0,0,7,45,52,0,0,0,0,0,3,29,0,52,38,0,38,0,0,45,5,0,0,0,0,9,52,27,58,0,57,2,4,0,0,0,0,12,45,0,0,56,0,8,0,8,0,0,0,0,0,1,52,7,32,9,2,45,0,33,0,52,28,0,56,10,33,0,0,2,45,4,47,0,0,7,4,0,42,10,31,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E3,8.26E2,1.724E3,3.92E2,4.34E2,1.012E3,7.12E2,2.78E2,1.14E2,3.48E2,8.6E1,2.26E2,7.86E2,3.39E2,3.73E2,2.38E2,4E1,5.3E1,6.1E1,3.43E2,5E0,3.5E1,5.1E1,1.1E2,1.16E2,7.28E2,5.8E1,1.63E2,1.76E2,2.71E2,1.02E2,2E2,3.8E1,3.4E1,6E0,4.8E1,5E0,4.8E1,1.3E1,2.65E2,7.8E1,1.7E1,1.8E1,4.2E1,9E0,1.06E2,4E0,9.2E1,2.4E1,1.5E1,7.13E2,2.9E1,2.9E1,1.34E2,2.9E1,1.28E2,4.8E1,1.02E2,1.69E2,8.3E1,1.9E1,3.7E1,1.63E2,1.5E1,2.3E1,8E0,2.6E1,4E1,8E0,2.4E1,2.4E1,9E0,4E0,1.84E2,8.1E1,6E1,1.8E1,4E0,1.3E1,1.3E1,5E0,1.1E1,3.1E1,4E0,5E0,5.8E1,4.8E1,4.6E1,4.6E1,6E0,1.8E1,6E0,9E0,2.93E2,4.2E2,4E0,2.5E1,9E0,2E1,9.6E1,3.8E1,1.1E1,1.8E1,1.23E2,5E0,3.8E1,1E1,7.2E1,3E1,1.47E2,2.2E1,4.8E1,3.5E1,7E0,1.2E1,2.6E1,1.1E1,6E0,1.57E2,4E0,4E0,2.1E1,5E0,4E0,4E0,1.7E1,7E0,5E0,1.9E1,4E0,5E0,8.8E1,9.6E1,7.6E1,5E0,4.7E1,1.3E1,4E0,1.4E1,9E0,4E0,9E0,4E0,6E0,5E0,2.2E1,9E0,4.7E1,1.1E1,5E0,4.3E1,2.3E1,2.3E1,4E1,6E0,1.2E1,6E0,2.49E2,4.4E1,4.01E2,1.9E1,4E0,5E0,1.6E1,4E0,5E0,9.1E1,1.5E1,2.3E1,7E0,4E0,8E0,1E1,5.7E1,6.6E1,3.1E1,7E0,4E0,6E0,2.4E1,4.8E1,2.1E1,9E0,1.17E2,3E1,9E0,3.9E1,7E0,4E0,1.25E2,3.2E1,1.2E1,9E0,5E0,1.4E1,4.2E1,4.6E1,5.5E1,4.1E1,7E0,6.9E1,9E0,3.8E1,6E0,8E0,5E0,4E0,9E0,1.3E1,1.1E1,3.6E1,3E1,1.3E1,1.9E1,4E0,1.8E1,5E0,3.6E1,4E0,5E0,7E0,1.22E2,1.27E2,2.4E1,2E1,3.05E2,9.6E1,1.5E1,4E0,1.2E1,4E0,8.3E1,8E0,4E0,1.1E1,1.1E1,1.2E1,4E0,6E0,3.9E1,1.8E1,5.4E1,1.2E1,1.3E1,1.1E1,3.5E1,1.3E1,4E0,1.7E1,9.4E1,2.3E1,2.4E1,6E0,2.5E1,1.4E1,8.8E1,3.7E1,5E0,2.7E1,3.8E1,4E0,3.7E1,9E0,3.6E1,1.9E1,1.1E1,3E1,6.1E1,8E0,5E0,4E0,2.1E1,1.7E1,4E0,5E0,9E0,4E0,2.3E1,7E0,1.4E1,5E0,8E0,1E1,6E0,3E1,9.7E1,2.5E1,1.12E2,1.5E1,6E0,1.8E1,1.4E1,6E0,1.9E2,1.15E2,5.5E1,4.1E1,1.1E1,4E0,8E0,4E0,6.3E1,2E1,4E0,4E0,4E0,7E0,7E0,4E0,4E0,8E0,6E0,3.3E1,1.2E1,6E0,9E0,4.5E1,7E0,5E0,8E0,2.7E1,8E0,5E0,8E0,9E0,3.2E1,6.2E1,9E0,1.4E1,8E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"331","size_leaf_vector":"1"}},{"base_weights":[-1.0566941E-3,-4.3251652E-1,2.1039946E-1,-6.6559243E-1,-1.4110592E-1,3.045465E-2,5.084033E-1,-6.960775E-1,8.957477E-2,-2.5734702E-1,3.6904958E-1,-3.1566164E-1,9.979884E-2,-3.4170855E-2,5.808065E-1,-8.2566726E-1,-3.4009823E-1,-3.4664673E-1,-7.423114E-4,2.0425327E-1,6.763986E-1,-4.8545888E-1,-2.3235098E-1,2.3105378E-1,-5.8211528E-2,1.4984202E-1,-4.2291132E-1,3.597002E-1,7.359603E-1,-7.804736E-1,-7.711689E-2,-1.1740916E-1,-5.505142E-1,-4.2828184E-1,-2.2658554E-1,-1.7883608E-1,1.3477996E-1,3.092565E-1,-9.41595E-2,7.384209E-1,1.4641719E-2,-5.0237656E-1,-1.0275956E-2,-1.5089576E-1,-2.9917818E-1,1.0081799E-1,4.1494793E-1,-1.9639038E-1,9.682317E-2,8.340637E-2,2.8706048E-2,-5.097114E-1,-2.1196547E-1,4.3799752E-1,3.3860028E-2,1.0381421E0,6.150431E-1,-8.426038E-1,-4.7902027E-1,-4.2512238E-2,-1.854573E-2,-6.7834795E-1,-3.0919355E-1,-4.6521083E-1,-2.7186456E-1,-9.211266E-2,-3.9816967E-1,-2.4141839E-1,-5.4356914E-2,3.0803068E-3,2.19334E-1,5.102902E-1,1.9585395E-1,-1.9248216E-1,5.079271E-3,4.0826295E-2,2.0922365E-2,-1.7855396E-2,-5.31167E-1,-2.4826045E-1,-7.772963E-2,-1.0154098E-1,-3.457394E-1,1.7545554E-1,-6.450483E-2,5.8553483E-2,3.6942253E-1,-1.4743565E-1,-6.41618E-1,6.916658E-2,5.583008E-2,-8.0137976E-2,2.7289486E-1,-2.7941985E-2,-1.39440065E-2,-1.7280018E-2,-1.784157E-3,2.4243255E-1,5.8925945E-1,-3.9568324E-2,1.2909053E-1,1.3012768E0,8.4450597E-1,4.272455E-1,7.694744E-1,-7.6351917E-1,-1.0550383E0,-2.3071504E-1,-7.1080756E-1,-1.3365136E-1,2.4963766E-1,-5.679347E-1,-4.63613E-2,1.9013657E-3,-4.2705634E-1,-4.7867155E-1,-7.936878E-3,4.471136E-3,-3.5430878E-1,-1.9331321E-1,1.7951676E-3,-3.3140826E-1,-4.0601425E-2,-1.2952456E-1,-1.6648594E-2,4.4384906E-3,-7.7424226E-3,6.4489044E-2,-7.031316E-3,3.0738303E-1,7.632006E-2,1.2654669E-2,3.0337084E-2,6.958475E-2,2.810759E-1,-5.0025135E-3,-1.2309332E-2,-1.4598453E-2,-2.7391164E-2,-1.2012477E-1,-1.8307075E-2,4.360663E-2,-1.3010335E-1,-9.99397E-3,1.2743511E-3,-2.5612435E-1,-4.166055E-1,2.1264787E-1,-2.0828141E-1,3.254057E-2,-2.5025386E-1,2.1033333E-1,5.992409E-1,8.9199446E-2,-2.2108942E-1,-3.5887264E-2,-1.9277258E-2,-3.0663118E-1,1.13230415E-1,-1.9231275E-1,6.675865E-2,3.5169074E-1,-3.265285E-3,-5.643094E-3,2.7577263E-1,6.465741E-1,-1.0667502E-1,-8.8427365E-3,1.740192E-1,4.1873142E-2,6.9797315E-2,8.7987566E-1,1.50242755E-2,4.7907326E-1,1.1248766E-1,6.2778074E-1,9.4803226E-1,-8.130618E-1,-4.6228144E-1,-1.1110315E0,-2.7896505E-2,1.0583815E-2,-3.3173504E-1,-3.738298E-1,-4.104155E-2,2.5997268E-2,-2.3890035E-1,2.3451831E-2,1.2816807E-3,-3.3675395E-2,-2.0482015E-2,-2.8579482E-1,-3.3075284E-2,-5.136727E-1,-1.9289842E-2,-4.5068467E-1,-1.5431465E-1,1.8484294E-3,-2.4990967E-1,1.3925205E-1,-1.185918E-1,-3.508207E-1,-8.626269E-3,-9.018677E-3,-2.4220315E-3,1.2022764E-1,-1.4923558E-3,7.3613822E-3,1.8077757E-2,6.318556E-3,2.5526888E-4,-1.9283096E-4,5.6014643E-3,6.674077E-3,1.7397499E-2,-9.962792E-3,-2.281537E-3,4.8958194E-3,-2.0606157E-3,-1.0176419E-2,-7.238924E-2,-1.4316249E-2,-6.630674E-3,-4.4620246E-1,-9.554574E-3,5.7209283E-1,1.8828212E-1,-3.3983508E-1,5.569537E-3,3.19138E-1,-3.8310643E-2,-3.025944E-1,4.1190446E-3,-2.924597E-2,2.442782E-1,2.9031113E-1,6.5656054E-1,1.419126E-1,-1.9085944E-2,-7.256851E-1,-1.549299E-1,-4.8104486E-1,-4.0009465E-2,6.2821954E-2,3.446955E-1,-1.4592893E-2,-1.8289089E-3,-2.4661296E-3,7.83218E-3,2.0732608E-2,7.7977916E-3,3.012565E-1,-4.0095043E-3,7.052827E-1,2.707871E-1,4.310085E-3,-1.39044365E-2,8.6632125E-2,2.336356E-1,9.2576617E-1,1.8293083E-2,5.746051E-1,2.9888707E-1,-1.7499449E-2,1.5024149E-2,6.824879E-1,1.446843E-1,9.9263346E-1,1.4718829E-2,-4.1608464E-2,-1.8754894E-2,-1.087061E-2,-3.0646773E-2,-4.4375535E-2,-6.305507E-2,-4.2286217E-3,-2.2484943E-2,-2.5097236E-2,-8.525018E-3,-4.3641264E-3,6.1885514E-3,-7.7864025E-3,-2.2399498E-2,-1.915106E-2,-2.656648E-3,-3.1894967E-2,-2.259859E-2,-1.17425965E-2,-2.8398886E-2,-1.3150671E-2,-7.2839006E-4,-8.903496E-3,-1.6276084E-2,3.2433227E-3,1.2995644E-2,1.8591072E-3,-9.633167E-3,-7.569781E-3,-1.8681787E-2,2.2437267E-3,8.569548E-3,-5.9951483E-3,2.3381799E-3,-2.4721699E-2,-1.2870772E-2,3.4903694E-2,1.7503185E-2,8.239477E-3,2.9522685E-2,-6.899534E-3,-2.0694207E-2,2.1735363E-2,8.706136E-3,-5.5981884E-3,3.707358E-3,-1.8379994E-2,-5.0787805E-3,9.777097E-3,2.4210727E-2,1.9820504E-2,9.75451E-4,2.854847E-2,4.2337365E-2,1.2765138E-2,1.6017723E-3,-5.4283805E-2,-2.1494044E-2,2.5432442E-5,-1.1202883E-2,-2.8307397E-2,-1.2395062E-2,2.9702228E-3,-6.5686745E-3,7.540203E-4,1.7411204E-2,2.1659385E-2,-4.1310033E-3,1.8086221E-2,9.53168E-3,3.7624158E-2,1.36017185E-2,3.5402859E-3,2.1829082E-2,8.938997E-3,-1.9752197E-3,1.478259E-2,5.541288E-3,3.2674775E-2,4.987737E-2,2.9831884E-2,1.0191956E-2,1.0767033E-2,2.2024574E-2,-4.73619E-3,3.1331435E-3,4.200475E-2,2.8529964E-2,5.3570507E-4,1.24772005E-2,2.3339434E-2,5.189553E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,73,75,-1,77,-1,79,81,83,85,87,89,91,-1,93,95,97,99,101,103,105,107,109,-1,111,113,115,117,119,121,123,125,127,129,131,133,135,-1,-1,-1,-1,137,139,141,143,145,147,149,-1,151,153,155,157,-1,159,161,-1,-1,-1,-1,163,165,-1,167,169,171,173,175,177,179,181,183,185,187,189,-1,-1,191,193,-1,-1,195,197,199,201,-1,203,-1,-1,-1,205,-1,207,209,-1,-1,211,213,-1,-1,-1,-1,215,-1,217,219,-1,-1,221,223,225,227,229,231,233,235,237,239,-1,-1,241,243,245,247,249,-1,-1,251,253,255,-1,257,-1,-1,259,-1,261,263,265,267,269,271,273,-1,-1,275,277,-1,279,281,-1,-1,-1,-1,283,-1,285,-1,287,289,-1,291,293,295,297,-1,-1,-1,299,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,301,-1,-1,303,-1,305,307,309,-1,311,313,315,-1,-1,317,319,321,323,-1,325,327,329,331,333,335,-1,-1,-1,-1,-1,-1,337,-1,339,341,-1,-1,343,345,347,-1,349,351,353,-1,355,357,359,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3328722E2,5.7027466E1,9.2003716E1,3.6245285E1,2.2332863E1,2.5736738E1,2.539737E1,2.103183E1,0E0,7.011648E0,3.407343E0,2.419323E0,1.853395E1,5.57143E0,1.9289413E1,9.758606E0,5.792533E0,2.1278324E0,1.9549571E0,1.5018318E0,3.542776E-1,1.7123222E-1,6.1895275E-1,1.1645569E1,8.716557E0,1.4473554E0,3.4971142E-1,6.0316124E0,1.1565506E1,5.5897675E0,0E0,1.1625501E0,1.7645893E0,6.475525E-1,2.132224E0,2.59534E-1,5.115684E-1,6.979785E-1,2.6985314E-1,1.6074944E-1,0E0,3.5666466E-2,0E0,3.8675928E-1,5.7224035E-1,3.5526943E0,6.5276756E0,4.5915422E0,5.341764E0,1.4783883E0,0E0,3.8034916E-2,2.000058E-1,5.5072174E0,3.7981658E0,3.69059E0,6.6606903E0,3.6297302E0,3.1165829E0,1.3380523E0,0E0,6.025028E-1,9.985087E-1,3.6522865E-1,8.658514E-1,5.135373E-1,8.6501217E-1,1.930331E-1,2.0101319E-1,1.7685637E-1,3.2912314E-1,1.4496994E-1,2.3870307E-1,2.0116389E-2,0E0,0E0,0E0,0E0,1.47743225E-2,3.1925368E-1,2.217376E-1,1.80246E-1,2.2988224E-1,2.8701606E0,1.6304915E0,0E0,6.893793E0,3.4183898E0,1.3444519E-1,3.1478288E0,0E0,4.4126898E-1,6.0933197E-1,0E0,0E0,0E0,0E0,1.038229E0,4.3369064E0,0E0,6.220902E-1,5.123062E-1,8.318138E-1,1.7568722E0,2.7192078E0,2.549263E0,1.2988663E0,1.3667022E0,7.7348423E-1,6.447193E-1,5.7851195E-1,2.4310303E-1,0E0,0E0,4.0807724E-1,1.3234901E-1,0E0,0E0,3.737688E-1,3.4814268E-1,4.964066E-1,5.347109E-2,0E0,4.082468E-2,0E0,0E0,0E0,7.722069E-2,0E0,9.475529E-2,4.21964E-2,0E0,0E0,3.4783248E-2,9.207249E-2,0E0,0E0,0E0,0E0,6.762533E-2,0E0,5.4789588E-2,8.7148875E-2,0E0,0E0,5.9504986E-2,1.0145903E-1,1.4775085E0,7.950283E-1,1.2305728E0,5.647527E-1,3.248899E0,1.1905956E0,1.2215464E0,4.861757E0,0E0,0E0,9.108032E-1,1.9502037E0,2.2226769E-1,1.3312496E-1,1.579299E-1,0E0,0E0,7.31843E-1,1.9931488E0,3.2197E-1,0E0,1.777283E-1,0E0,0E0,9.106598E-1,0E0,1.4635429E0,4.2113322E-1,1.9098473E0,1.3397179E0,1.1247406E0,9.756255E-1,9.314499E-1,0E0,0E0,6.632862E-1,1.4923108E-1,0E0,1.8827096E-1,3.3019435E-1,0E0,0E0,0E0,0E0,2.7442122E-1,0E0,2.5470734E-1,0E0,2.4711704E-1,1.3308729E-1,0E0,5.616641E-2,1.08881235E-1,1.9213764E-1,9.047055E-2,0E0,0E0,0E0,2.7060196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.0700746E-2,0E0,0E0,9.2381E-2,0E0,6.3802004E-2,1.5122008E0,1.3775027E-1,0E0,1.2566149E-1,4.1476372E-1,3.1664014E-1,0E0,0E0,1.201448E0,3.9548492E-1,5.7239914E-1,5.2824736E-1,0E0,1.3448257E0,1.4323082E0,9.844589E-2,8.9671314E-2,1.9034673E0,1.1741121E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2064285E-1,0E0,1.5509872E0,4.671396E-1,0E0,0E0,2.0496361E-1,1.3517857E-1,2.9252625E-1,0E0,3.8534164E-1,3.1923246E-1,7.413016E-2,0E0,7.193146E-1,1.2394947E-1,7.7111053E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,88,88,89,89,91,91,92,92,97,97,98,98,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,114,114,115,115,118,118,119,119,120,120,121,121,123,123,127,127,129,129,130,130,133,133,134,134,139,139,141,141,142,142,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,157,157,158,158,159,159,160,160,161,161,164,164,165,165,166,166,168,168,171,171,173,173,174,174,175,175,176,176,177,177,178,178,179,179,182,182,183,183,185,185,186,186,191,191,193,193,195,195,196,196,198,198,199,199,200,200,201,201,205,205,220,220,223,223,225,225,226,226,227,227,229,229,230,230,231,231,234,234,235,235,236,236,237,237,239,239,240,240,241,241,242,242,243,243,244,244,251,251,253,253,254,254,257,257,258,258,259,259,261,261,262,262,263,263,265,265,266,266,267,267],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,74,76,-1,78,-1,80,82,84,86,88,90,92,-1,94,96,98,100,102,104,106,108,110,-1,112,114,116,118,120,122,124,126,128,130,132,134,136,-1,-1,-1,-1,138,140,142,144,146,148,150,-1,152,154,156,158,-1,160,162,-1,-1,-1,-1,164,166,-1,168,170,172,174,176,178,180,182,184,186,188,190,-1,-1,192,194,-1,-1,196,198,200,202,-1,204,-1,-1,-1,206,-1,208,210,-1,-1,212,214,-1,-1,-1,-1,216,-1,218,220,-1,-1,222,224,226,228,230,232,234,236,238,240,-1,-1,242,244,246,248,250,-1,-1,252,254,256,-1,258,-1,-1,260,-1,262,264,266,268,270,272,274,-1,-1,276,278,-1,280,282,-1,-1,-1,-1,284,-1,286,-1,288,290,-1,292,294,296,298,-1,-1,-1,300,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,302,-1,-1,304,-1,306,308,310,-1,312,314,316,-1,-1,318,320,322,324,-1,326,328,330,332,334,336,-1,-1,-1,-1,-1,-1,338,-1,340,342,-1,-1,344,346,348,-1,350,352,354,-1,356,358,360,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,2.14099E5,9.026015E2,3.97E2,2.1328075E-1,1.9125667E3,8.957477E-2,1.2690893E0,1.3664E4,3.79E2,2E1,5.9767612E7,1.1410706E3,1.1855755E8,6.747114E7,2.785857E2,3.1294638E2,4.0844156E7,3.9715407E0,2.4636364E1,1.0412E4,1.13E3,1.023747E6,1.2204E4,2.3898147E5,7.6188426E0,2E0,3.83E3,-7.711689E-2,1.4843444E1,3.0497742E3,3.011152E6,1.2471935E7,1.752512E2,2.695E3,6.5346925E6,6.624E3,1.0881593E10,1.4641719E-2,1.901E3,-1.0275956E-2,1.56E2,1.43709E0,8E0,2.04E5,3.8879236E7,6.7652373E3,3.202814E5,2.8706048E-2,9.245905E-2,2.51343E5,2.6563762E7,2.960909E2,1.4307411E7,2.3292E4,2.6944466E7,4.987639E7,4.9560537E3,-1.854573E-2,1.3521131E8,9.64E2,1.4255319E0,1.2245615E0,7.524668E10,1.4136576E6,4.05E2,4.5576923E1,5.2107143E0,1.4187837E-1,5.0079144E5,1.3569831E7,8.9960505E-4,5.079271E-3,4.0826295E-2,2.0922365E-2,-1.7855396E-2,1.6695403E0,1.24E2,1.2E1,1.0784314E0,4.9E1,2E0,5.372E3,5.8553483E-2,1.1490676E6,3.7307863E0,7.868429E-2,1.2E1,5.583008E-2,9.08912E5,1E0,-2.7941985E-2,-1.39440065E-2,-1.7280018E-2,-1.784157E-3,5.52E2,1.5846036E7,-3.9568324E-2,1.993E3,1.363881E3,2.0746333E2,2.589369E7,1.6990049E0,5.492982E2,7.471519E2,1.683274E0,2.4684112E8,2.2668628E7,1.942E3,2.6835732E3,-4.63613E-2,1.9013657E-3,5.784E4,1.509434E-2,-7.936878E-3,4.471136E-3,2.480402E-2,4.7931477E4,5.7E1,1E0,-4.0601425E-2,1.67E2,-1.6648594E-2,4.4384906E-3,-7.7424226E-3,1.2903092E7,-7.031316E-3,2.3140822E8,3.275E3,1.2654669E-2,3.0337084E-2,2.6432162E5,1.5106794E3,-5.0025135E-3,-1.2309332E-2,-1.4598453E-2,-2.7391164E-2,1E0,-1.8307075E-2,1.8903887E0,8.204587E1,-9.99397E-3,1.2743511E-3,3E0,1.287E3,4.360204E1,3.29E2,2.1515152E0,4.264897E6,3.983889E3,3.7990784E7,1.84442E5,1.04E3,-3.5887264E-2,-1.9277258E-2,2.8326238E3,1.4777102E9,8.7217E4,1E0,5.500988E2,-3.265285E-3,-5.643094E-3,1.1860938E8,2E0,1.7730331E6,-8.8427365E-3,4.759E3,4.1873142E-2,6.9797315E-2,1.5548611E7,1.50242755E-2,1.1727273E1,5.3534385E3,3.7284137E2,2.586199E7,3.3E1,1.216918E7,8.091679E-1,-2.7896505E-2,1.0583815E-2,1.89777E5,4.24566E5,-4.104155E-2,4E1,1E0,2.3451831E-2,1.2816807E-3,-3.3675395E-2,-2.0482015E-2,6.53143E6,-3.3075284E-2,1.4E1,-1.9289842E-2,2.9005406E5,3.4015749E0,1.8484294E-3,5.98405E5,3.956E3,1.6644816E6,6.929741E8,-8.626269E-3,-9.018677E-3,-2.4220315E-3,6.47E2,-1.4923558E-3,7.3613822E-3,1.8077757E-2,6.318556E-3,2.5526888E-4,-1.9283096E-4,5.6014643E-3,6.674077E-3,1.7397499E-2,-9.962792E-3,-2.281537E-3,4.8958194E-3,-2.0606157E-3,-1.0176419E-2,6.647279E5,-1.4316249E-2,-6.630674E-3,1.6968552E2,-9.554574E-3,2.6393727E4,7.780377E6,4.82E2,5.569537E-3,9.232009E2,7.68E2,4.1032645E6,4.1190446E-3,-2.924597E-2,8.027061E2,1E0,3.7891315E6,1.59551E5,-1.9085944E-2,8E0,8.5534485E1,3.960925E3,5.460954E9,2.5682794E5,1.3674345E4,-1.4592893E-2,-1.8289089E-3,-2.4661296E-3,7.83218E-3,2.0732608E-2,7.7977916E-3,1.5851064E0,-4.0095043E-3,8E0,7.697189E2,4.310085E-3,-1.39044365E-2,1.259E3,4.396066E1,5.685779E9,1.8293083E-2,1.7291568E2,9.099893E8,8.891209E6,1.5024149E-2,4.556146E6,1E1,3.0497742E3,1.4718829E-2,-4.1608464E-2,-1.8754894E-2,-1.087061E-2,-3.0646773E-2,-4.4375535E-2,-6.305507E-2,-4.2286217E-3,-2.2484943E-2,-2.5097236E-2,-8.525018E-3,-4.3641264E-3,6.1885514E-3,-7.7864025E-3,-2.2399498E-2,-1.915106E-2,-2.656648E-3,-3.1894967E-2,-2.259859E-2,-1.17425965E-2,-2.8398886E-2,-1.3150671E-2,-7.2839006E-4,-8.903496E-3,-1.6276084E-2,3.2433227E-3,1.2995644E-2,1.8591072E-3,-9.633167E-3,-7.569781E-3,-1.8681787E-2,2.2437267E-3,8.569548E-3,-5.9951483E-3,2.3381799E-3,-2.4721699E-2,-1.2870772E-2,3.4903694E-2,1.7503185E-2,8.239477E-3,2.9522685E-2,-6.899534E-3,-2.0694207E-2,2.1735363E-2,8.706136E-3,-5.5981884E-3,3.707358E-3,-1.8379994E-2,-5.0787805E-3,9.777097E-3,2.4210727E-2,1.9820504E-2,9.75451E-4,2.854847E-2,4.2337365E-2,1.2765138E-2,1.6017723E-3,-5.4283805E-2,-2.1494044E-2,2.5432442E-5,-1.1202883E-2,-2.8307397E-2,-1.2395062E-2,2.9702228E-3,-6.5686745E-3,7.540203E-4,1.7411204E-2,2.1659385E-2,-4.1310033E-3,1.8086221E-2,9.53168E-3,3.7624158E-2,1.36017185E-2,3.5402859E-3,2.1829082E-2,8.938997E-3,-1.9752197E-3,1.478259E-2,5.541288E-3,3.2674775E-2,4.987737E-2,2.9831884E-2,1.0191956E-2,1.0767033E-2,2.2024574E-2,-4.73619E-3,3.1331435E-3,4.200475E-2,2.8529964E-2,5.3570507E-4,1.24772005E-2,2.3339434E-2,5.189553E-2],"split_indices":[20,102,33,2,52,2,42,4,0,42,2,1,3,45,52,45,45,52,52,45,57,52,12,2,1,2,33,54,17,2,0,57,52,9,45,52,2,45,2,12,0,9,0,2,53,17,5,45,52,28,0,41,1,12,52,51,2,45,45,55,0,45,0,57,53,31,28,10,58,53,27,28,45,42,0,0,0,0,54,29,10,56,0,6,29,0,28,53,57,18,0,1,74,0,0,0,0,2,47,0,2,4,58,45,41,55,52,53,7,45,0,4,0,0,10,57,0,0,38,47,8,80,0,0,0,0,0,32,0,7,2,0,0,28,52,0,0,0,0,13,0,53,47,0,0,8,10,47,10,53,9,28,7,29,2,0,0,52,7,9,64,4,0,0,5,6,28,0,2,0,0,45,0,53,4,58,9,3,45,27,0,0,29,12,0,8,104,0,0,0,0,29,0,3,0,33,56,0,1,2,32,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,45,0,0,4,0,28,28,2,0,33,2,45,0,0,52,16,28,9,0,17,54,4,5,28,4,0,0,0,0,0,0,53,0,17,4,0,0,11,58,31,0,58,7,32,0,48,3,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.555E3,8.4E2,1.715E3,4.66E2,3.74E2,1.07E3,6.45E2,4.61E2,5E0,3.05E2,6.9E1,1.78E2,8.92E2,7.6E1,5.69E2,3.37E2,1.24E2,2.26E2,7.9E1,4.6E1,2.3E1,5.7E1,1.21E2,4.87E2,4.05E2,5.2E1,2.4E1,2.36E2,3.33E2,3.19E2,1.8E1,6.1E1,6.3E1,1.33E2,9.3E1,3.4E1,4.5E1,3.4E1,1.2E1,1.9E1,4E0,5.3E1,4E0,5.6E1,6.5E1,2.86E2,2.01E2,2.14E2,1.91E2,4.6E1,6E0,1.6E1,8E0,1.9E2,4.6E1,9.3E1,2.4E2,2.63E2,5.6E1,4.8E1,1.3E1,4E1,2.3E1,1.06E2,2.7E1,5.3E1,4E1,2.2E1,1.2E1,1.8E1,2.7E1,1.1E1,2.3E1,8E0,4E0,1.4E1,5E0,1.1E1,4.2E1,2.3E1,3.3E1,1.3E1,5.2E1,1.97E2,8.9E1,1E1,1.91E2,1.94E2,2E1,1.87E2,4E0,2.5E1,2.1E1,1.2E1,4E0,4E0,4E0,8.4E1,1.06E2,4E0,4.2E1,3.7E1,5.6E1,1.1E2,1.3E2,1.95E2,6.8E1,2.8E1,2.8E1,3.7E1,1.1E1,3E1,1E1,6E0,1.7E1,1.01E2,5E0,5E0,2.2E1,2.5E1,2.8E1,3.6E1,4E0,1.1E1,1.1E1,5E0,7E0,1.3E1,5E0,1.6E1,1.1E1,4E0,7E0,1E1,1.3E1,4E0,4E0,4E0,3.8E1,1.2E1,1.1E1,1E1,2.3E1,7E0,6E0,2.5E1,2.7E1,1.8E2,1.7E1,5.9E1,3E1,1.14E2,7.7E1,4.6E1,1.48E2,1.4E1,6E0,1.9E1,1.68E2,1.4E1,1.1E1,1.7E1,4E0,7E0,7.7E1,9.8E1,8E0,5E0,3.7E1,8E0,2.9E1,5.2E1,4E0,9.4E1,1.6E1,7.5E1,5.5E1,1.66E2,2.9E1,6E1,8E0,5E0,2.3E1,8E0,2E1,1.5E1,2.2E1,5E0,6E0,1.6E1,1.4E1,1.2E1,5E0,7E1,3.1E1,1.4E1,8E0,5E0,2E1,1.3E1,1.5E1,3.1E1,5E0,6E0,5E0,8E0,5E0,5E0,1.1E1,6E0,5E0,4E0,6E0,5E0,8E0,5E0,7E0,6E0,4E0,9E0,1.4E1,1.9E1,6E0,2.3E1,4E0,1E1,1.7E2,1.2E1,5E0,1.1E1,4.8E1,2.6E1,4E0,4E0,1.1E2,1.3E1,6.4E1,4.2E1,4E0,1.6E1,1.32E2,1.1E1,8E0,1.39E2,2.9E1,8E0,6E0,5E0,6E0,1.2E1,5E0,7.2E1,5E0,8.4E1,1.4E1,4E0,4E0,1.6E1,2.1E1,4.7E1,5E0,6E1,3.4E1,1E1,6E0,6.7E1,8E0,5.1E1,4E0,1.58E2,8E0,1.2E1,1.7E1,2.7E1,3.3E1,8E0,1.5E1,4E0,4E0,7E0,8E0,1.7E1,5E0,8E0,4E0,2E1,5E1,6E0,8E0,4E0,4E0,1.2E1,8E0,9E0,4E0,5E0,1E1,4E0,2.7E1,4E0,4E0,1E1,4E0,1.7E1,6E0,5E0,5E0,1.62E2,8E0,4E0,8E0,5E0,6E0,2.9E1,1.9E1,1.9E1,7E0,9.3E1,1.7E1,9E0,4E0,4.7E1,1.7E1,2E1,2.2E1,6E0,1E1,4.1E1,9.1E1,7E0,4E0,4E0,4E0,1.2E2,1.9E1,2.4E1,5E0,4.5E1,2.7E1,7.5E1,9E0,7E0,7E0,9E0,7E0,1.3E1,8E0,1.2E1,3.5E1,5.6E1,4E0,2.3E1,1.1E1,5E0,5E0,2.5E1,4.2E1,4E0,4E0,5E0,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"361","size_leaf_vector":"1"}},{"base_weights":[5.6424607E-3,-3.9374077E-1,1.9405335E-1,-6.086584E-1,-1.3052028E-1,4.8971716E-2,4.9585947E-1,1.1234723E0,-6.5010726E-1,-3.2414904E-1,1.7617974E-1,-1.3841562E-1,1.9075833E-1,3.6221406E-1,7.946828E-1,9.544102E-3,8.1396766E-2,-7.997513E-1,-4.0098605E-1,-4.3713826E-1,-1.9574678E-1,8.027744E-2,5.9233505E-1,-2.6595333E-1,1.3695039E-2,2.3082018E-1,-3.890109E-1,4.4333652E-1,-4.8886403E-2,8.6566514E-1,3.7353265E-1,-7.014458E-1,-1.1510926E0,-1.9436646E-1,-6.1273205E-1,-5.393431E-1,-3.3448392E-1,-3.0364427E-1,-1.2879686E-1,-3.0711258E-2,2.2387272E-1,6.5749836E-1,1.7757395E-2,-4.5642138E-1,-1.7384708E-1,5.772501E-2,-2.6597017E-2,6.0266115E-2,3.707269E-1,-4.7691247E-1,1.235098E-2,3.097098E-1,7.041882E-1,-2.65211E-1,5.001908E-1,1.0383952E0,7.473607E-1,1.3601291E-3,4.9338078E-1,-7.359254E-1,-2.7553967E-1,-1.3180226E0,-4.260835E-2,-2.5093323E-1,3.2376024E-1,-7.2550666E-1,-2.2121675E-1,-8.318492E-3,-5.6343347E-1,-2.2740363E-1,-4.376828E-1,-3.264911E-1,-8.423871E-3,-1.9084603E-1,1.2437556E-2,-1.0376284E-1,1.779744E-1,1.1567695E-1,3.1248063E-1,1.576794E-2,3.6138773E-2,-5.292722E-1,-3.1454962E-1,-1.2516896E-1,-4.276688E-1,-8.564768E-2,2.280767E-1,-3.0779282E-3,2.7145955E-1,3.4050864E-1,7.333894E-2,-5.89799E-1,-2.057703E-1,1.9544393E-1,3.9184588E-1,5.496128E-1,9.4504756E-1,-5.1743764E-1,-3.1887017E-2,3.479215E-1,3.8580295E-2,1.5933352E-2,1.1002574E0,2.9313913E-1,7.930818E-1,1.4001437E-2,2.8217271E-2,-7.6495785E-1,-4.3348163E-1,-1.2762012E-1,-2.4471335E-2,-5.533115E-2,-8.0223955E-2,-1.1638237E-1,-3.728003E-1,3.325157E-2,-4.1325726E-3,-6.1379784E-1,-9.768849E-1,-5.074152E-2,-3.3063847E-1,-5.14084E-1,-4.1391853E-2,-2.8133288E-1,-1.1846125E-3,-5.476691E-1,-3.0034134E-1,-8.239105E-3,-1.7115643E-2,-2.5886038E-1,-7.341043E-2,-1.2934902E-1,1.4137213E-1,-1.7116044E-1,4.110063E-2,2.630061E-1,-1.7949671E-3,1.557781E-1,-1.862408E-3,3.982687E-1,1.649075E-1,-5.617497E-1,-3.172052E-1,-3.4841695E-1,-2.7586168E-3,-2.4083771E-1,1.5981965E-2,-6.94381E-1,-2.5959367E-1,2.6707277E-1,-1.13859534E-1,3.0010113E-1,1.7005159E-2,1.5252181E-2,-3.9872855E-2,2.3202023E-1,2.6401991E-2,3.976809E-1,8.3727196E-2,-6.331552E-1,-1.3610939E-2,-1.5613477E-2,-4.997582E-3,9.023959E-2,3.414952E-1,5.240508E-1,3.2272434E-1,3.473371E-2,2.9210532E-1,7.0790255E-1,1.1705118E0,-6.844853E-1,-3.0727014E-1,-1.1734343E-1,8.747058E-3,1.1969853E-2,2.1089124E-2,3.5532564E-2,1.1636907E0,2.3974158E-2,5.253821E-4,5.2431357E-1,8.6468005E-1,-8.1697166E-1,-6.356471E-1,-2.5136814E-2,-6.2007206E-3,-1.213859E-2,-5.2706926E-4,-8.73233E-3,-2.7058354E-1,-4.497498E-1,-1.9227266E-1,-7.339634E-1,-4.1487613E-1,-5.7944648E-2,-3.3240423E-2,2.6933656E-3,-7.257058E-3,-9.598631E-3,-2.18434E-2,-5.338651E-1,-1.1654072E-2,-2.1245155E-1,-2.3760589E-2,-3.0464254E-2,-1.4454429E-2,-8.001631E-3,-1.7463917E-2,-3.0132818E-1,-1.2628792E-1,-9.739989E-3,3.2688137E-2,-1.9681135E-3,-8.750567E-3,1.19514E-2,1.8448765E-4,-2.1113211E-1,-4.928993E-4,-1.1101526E-1,1.1297859E-2,5.0454545E-3,1.7570907E-2,9.675544E-2,1.0851076E-2,1.0982091E-2,2.376619E-2,1.1013051E-2,2.1548262E-3,-3.5379693E-2,-2.4133539E-2,-5.9255497E-3,-2.4588713E-2,-2.9361355E-1,-2.7619105E-2,-3.0702326E-1,-1.0264486E-1,3.2401234E-2,-7.183737E-2,-1.4784137E-2,-4.396204E-2,-1.6740995E-2,1.6870629E-3,2.8193703E-2,3.92797E-2,-2.123346E-1,-1.7447708E-2,2.188863E-1,2.2829028E-2,-4.1678282E-3,5.029037E-3,-1.8185247E-1,7.231062E-2,1.975477E-3,2.5717127E-1,3.0606487E-1,5.004182E-1,-8.2973994E-2,2.1105792E-1,-1.7413946E-2,-3.3714388E-2,1.5122098E-1,-4.6900067E-2,1.4933854E-1,4.686764E-1,2.2810723E-1,3.0449927E-2,2.3970583E-1,4.2128178E-1,3.9417005E-1,2.150349E-5,3.852911E-2,1.6298441E-2,4.5528222E-2,7.223308E-2,-3.9040964E-2,-1.9617936E-2,-1.920528E-2,-6.1193877E-3,-1.5120777E-2,-4.776259E-2,6.1451524E-2,3.8461987E-2,3.24601E-1,3.455525E-2,9.625306E-1,7.133938E-1,-3.8255863E-2,-5.1949255E-2,-3.4047272E-2,-1.761176E-2,-4.7483905E-3,7.2780484E-3,-1.6575463E-2,-3.483392E-3,-1.1285754E-2,-2.4413714E-2,-4.5126122E-3,-1.7875003E-2,-1.6442295E-2,-3.936051E-2,-2.7571223E-3,-2.5050871E-2,-2.750955E-2,-1.45008415E-2,-1.2266369E-2,-2.8794282E-3,-1.6947476E-2,-7.188819E-3,-9.357237E-3,-2.001098E-3,-1.472446E-3,4.4668606E-3,-1.2432578E-2,5.987959E-4,-9.856476E-3,7.336461E-4,6.3270056E-3,1.4797025E-3,-2.1359576E-2,-6.8536755E-3,-1.3350857E-2,-2.20228E-2,6.3245893E-3,-7.134138E-3,2.885556E-4,-1.5483975E-2,8.342529E-3,-4.8097125E-3,-1.1784895E-2,1.0551216E-2,1.6420102E-3,-5.546946E-3,1.252376E-2,4.353961E-3,-4.4136485E-3,-1.165749E-2,5.2182595E-3,-1.4530794E-2,-5.5917836E-4,1.3979073E-2,2.6429722E-2,1.2263377E-2,1.3398423E-2,2.739607E-2,2.3395396E-3,-1.1809137E-2,1.4504242E-2,1.4204131E-3,3.378108E-3,1.2242553E-2,-8.963961E-3,5.273077E-3,-5.9384573E-3,1.5311123E-2,3.154864E-2,1.6978716E-2,3.158981E-3,1.5654348E-2,2.2888241E-2,9.290088E-3,2.3942392E-2,1.5576672E-2,2.6803238E-2,8.02982E-3,-7.592222E-3,9.755273E-4,7.3496904E-3,2.0237582E-2,3.562256E-2,5.359496E-2,1.7622434E-2,3.782477E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,-1,81,83,-1,85,87,89,91,-1,93,95,97,99,101,103,-1,105,107,109,111,-1,113,115,117,119,-1,121,123,125,127,-1,129,131,133,135,137,139,-1,-1,141,143,145,147,149,151,153,155,157,-1,159,161,163,165,167,169,171,173,175,-1,-1,177,179,181,-1,-1,183,185,187,-1,-1,-1,189,191,-1,-1,193,195,197,199,201,-1,203,-1,205,207,-1,-1,209,211,213,215,217,219,221,-1,223,-1,225,227,229,231,233,-1,235,237,239,241,243,245,247,249,251,-1,253,-1,255,257,259,-1,-1,-1,261,263,265,267,-1,269,271,273,275,277,279,-1,-1,-1,-1,281,-1,-1,283,285,287,289,-1,-1,-1,-1,291,293,295,297,299,301,-1,-1,-1,-1,-1,-1,303,-1,305,-1,-1,-1,-1,-1,307,309,-1,311,-1,-1,-1,-1,313,-1,315,-1,-1,-1,317,-1,-1,-1,-1,-1,-1,-1,-1,-1,319,-1,321,323,-1,325,-1,-1,-1,-1,-1,327,329,331,333,-1,-1,-1,335,337,-1,339,341,343,345,347,-1,-1,349,351,353,355,357,-1,359,361,363,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,365,-1,-1,367,-1,369,371,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9414137E2,4.6704544E1,7.67335E1,3.339711E1,2.2208065E1,3.1510687E1,2.2435944E1,4.874303E0,1.6268661E1,3.2367783E0,5.7394347E0,9.916964E0,1.5767763E1,1.3194721E1,4.908142E0,0E0,0E0,8.759827E0,7.2748013E0,1.0891018E0,7.4827814E-1,1.9034326E0,1.8322754E-1,4.7976875E0,1.0793887E1,1.5058231E1,2.5784774E0,1.128463E1,7.972064E0,2.268715E0,1.1071999E0,3.0132523E0,1.8145523E0,2.6311035E0,3.5042038E0,4.3615913E-1,5.9665537E-1,7.1465254E-2,6.070535E-1,1.0551147E0,4.6237707E-1,2.2146225E-1,0E0,7.6371E-1,2.28167E0,0E0,3.4333081E0,3.8268998E0,1.1003399E1,1.07236E0,0E0,1.9881935E0,3.6132202E0,2.8083255E0,5.180874E-1,2.2307816E0,1.6781311E0,0E0,1.459055E-1,1.4421082E0,5.078522E-1,4.0262222E-1,0E0,1.2520003E0,1.3051007E0,1.1975822E0,3.562966E-1,0E0,2.9328728E-1,3.5584033E-1,3.0378866E-1,3.1702995E-2,0E0,3.6438763E-1,4.2047256E-1,5.0497085E-1,3.3683342E-1,1.5914297E-1,2.8880286E-1,0E0,0E0,2.2128105E-1,2.7060866E-1,2.6161292E0,1.1645741E0,1.8713247E0,6.4859605E-1,3.2323196E0,5.4619646E-1,4.943489E0,0E0,1.9908142E-1,1.15445435E-1,1.4135153E0,9.914284E-1,2.420107E0,1.417038E0,5.9716225E-1,4.8034585E-1,1.137352E-2,0E0,0E0,4.807129E-1,5.2251285E-1,1.1438293E0,0E0,0E0,7.115402E-1,3.6817193E-1,1.59437E-1,0E0,0E0,0E0,6.450279E-1,4.8307753E-1,0E0,0E0,8.18964E-1,2.5382042E-1,9.682582E-2,9.310758E-2,1.4770222E-1,0E0,2.4676132E-1,0E0,8.004093E-2,5.4329157E-2,0E0,0E0,1.3084769E-1,2.5133935E-1,3.8403332E-2,1.6072336E-1,2.2357154E-1,5.035247E-1,1.512152E-1,0E0,5.145389E-2,0E0,1.302197E-1,7.160246E-2,2.226572E-1,2.8901362E-1,1.6245246E-1,0E0,7.7336645E-1,4.101262E0,5.5258274E-1,4.539863E-1,9.255661E-1,1.629291E0,3.0745578E-1,1.0918536E-1,2.4536366E0,0E0,2.7546263E-1,0E0,2.4540672E0,1.3547618E0,3.7077904E-2,0E0,0E0,0E0,4.754185E-1,8.8311815E-1,9.219084E-1,6.0267544E-1,0E0,8.0361295E-1,3.0328655E-1,1.09745026E-1,3.245163E-2,1.23886585E-1,2.304849E-1,0E0,0E0,0E0,0E0,2.0008469E-1,0E0,0E0,4.4356585E-1,2.6075745E-1,6.5618896E-1,4.9832153E-1,0E0,0E0,0E0,0E0,3.3361113E-1,1.7339408E-1,1.3318443E-1,2.038849E-1,3.504038E-1,5.9443474E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.3249626E-2,0E0,9.327495E-2,0E0,0E0,0E0,0E0,0E0,8.230686E-2,3.9831236E-2,0E0,4.1407466E-2,0E0,0E0,0E0,0E0,2.384113E-1,0E0,1.1130551E-1,0E0,0E0,0E0,2.03157E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6493196E-1,0E0,1.9647121E-1,2.9405272E-1,0E0,1.2026982E0,0E0,0E0,0E0,0E0,0E0,1.7182373E-1,8.964844E-1,4.1788602E-1,7.438755E-2,0E0,0E0,0E0,2.1517503E-1,2.0017474E0,0E0,3.184638E-1,1.8837929E0,1.2872696E0,5.684432E-1,5.12599E-1,0E0,0E0,2.8928947E-1,3.824791E-1,7.526113E-1,2.6776838E-1,1.3470286E-1,0E0,4.9027252E-1,9.157467E-2,5.780182E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0811777E-1,0E0,0E0,9.745395E-2,0E0,2.2449112E-1,1.8916512E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,102,102,103,103,104,104,107,107,108,108,109,109,113,113,114,114,117,117,118,118,119,119,120,120,121,121,123,123,125,125,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,137,137,139,139,140,140,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,155,155,157,157,158,158,159,159,163,163,164,164,165,165,166,166,168,168,169,169,170,170,171,171,172,172,173,173,178,178,181,181,182,182,183,183,184,184,189,189,190,190,191,191,192,192,193,193,194,194,201,201,203,203,209,209,210,210,212,212,217,217,219,219,223,223,233,233,235,235,236,236,238,238,244,244,245,245,246,246,247,247,251,251,252,252,254,254,255,255,256,256,257,257,258,258,261,261,262,262,263,263,264,264,265,265,267,267,268,268,269,269,280,280,283,283,285,285,286,286],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,-1,82,84,-1,86,88,90,92,-1,94,96,98,100,102,104,-1,106,108,110,112,-1,114,116,118,120,-1,122,124,126,128,-1,130,132,134,136,138,140,-1,-1,142,144,146,148,150,152,154,156,158,-1,160,162,164,166,168,170,172,174,176,-1,-1,178,180,182,-1,-1,184,186,188,-1,-1,-1,190,192,-1,-1,194,196,198,200,202,-1,204,-1,206,208,-1,-1,210,212,214,216,218,220,222,-1,224,-1,226,228,230,232,234,-1,236,238,240,242,244,246,248,250,252,-1,254,-1,256,258,260,-1,-1,-1,262,264,266,268,-1,270,272,274,276,278,280,-1,-1,-1,-1,282,-1,-1,284,286,288,290,-1,-1,-1,-1,292,294,296,298,300,302,-1,-1,-1,-1,-1,-1,304,-1,306,-1,-1,-1,-1,-1,308,310,-1,312,-1,-1,-1,-1,314,-1,316,-1,-1,-1,318,-1,-1,-1,-1,-1,-1,-1,-1,-1,320,-1,322,324,-1,326,-1,-1,-1,-1,-1,328,330,332,334,-1,-1,-1,336,338,-1,340,342,344,346,348,-1,-1,350,352,354,356,358,-1,360,362,364,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,366,-1,-1,368,-1,370,372,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2E0,4.273439E2,2.1592189E5,1.2974394E6,2.9328E4,4.423E3,1.7801818E-1,1.4101E4,2.695E3,3.0820766E-1,4.0844156E7,1E0,9.544102E-3,8.1396766E-2,4.3702424E7,6.747114E7,1.8122449E2,2.1102592E8,1.502494E0,1.5609541E0,9.6E1,1.7E1,9.27E2,2.5422776E3,2.6312E4,6.7652373E3,1.6954315E0,8.319452E2,8.9910974E2,4.822581E0,7.023838E3,3.970405E3,2.2539302E6,1.6256282E7,4.4E1,1.5083523E0,3.368326E11,4.123E3,2.1924436E5,1.7757395E-2,2.1515152E0,1.044E1,5.772501E-2,1.2817779E7,2.3314082E6,1.0409E4,2.6476662E6,1.235098E-2,1.0485785E3,1.4847565E0,1.2542373E0,7.9023E4,2.979E3,1E0,1.3601291E-3,3.596E3,2.1E1,2.2075728E7,2.0469315E0,-4.260835E-2,2.8839298E7,1.00663277E9,2.603E3,1.272E3,-8.318492E-3,1E0,2.835051E2,2.728E3,3.6148317E-2,-8.423871E-3,5.23896E5,1.428111E5,6.604754E5,5.7E1,2.26182E5,6.5346925E6,1.576794E-2,3.6138773E-2,8.07966E5,4E1,5.44E2,4.4866666E2,2.3E1,2.836111E1,1.1095187E1,7.780377E6,2.4E1,7.333894E-2,5.1E1,7.006513E4,3.202814E5,1.3035088E1,1.4521701E2,7.0764465E1,1.7495675E5,7.683389E7,1.1803382E4,3.8580295E-2,1.5933352E-2,1.711119E8,2.0288463E0,2.5220325E6,1.4001437E-2,2.8217271E-2,2.308943E0,6.825774E3,7.3376025E6,-2.4471335E-2,-5.533115E-2,-8.0223955E-2,2.46875E0,3.266054E3,3.325157E-2,-4.1325726E-3,1.0052E4,2.8312179E12,2.310848E7,3.5491096E7,9.4538574E2,-4.1391853E-2,9.60094E5,-1.1846125E-3,1.0201212E3,2.450543E-2,-8.239105E-3,-1.7115643E-2,4.3906153E5,6.689559E0,2.4684112E8,3.7684184E-1,6.553948E8,6.72E2,4.4067603E-2,-1.7949671E-3,9.6771875E2,-1.862408E-3,1.8991614E5,4.735985E6,2.5086691E0,2.0189162E7,1.9051096E7,-2.7586168E-3,1.325204E6,2.7E1,1E0,7.364257E4,5.0740743E0,9.51668E5,3.4540886E3,2.407741E7,3.79E2,-3.9872855E-2,5.9860466E1,2.6401991E-2,4.198242E2,4.05E2,2.07E2,-1.3610939E-2,-1.5613477E-2,-4.997582E-3,1.2673605E7,3.131E3,1E0,1.0872E4,3.473371E-2,2.6025698E7,2.62797E5,1.038946E6,4.1043E4,1.6347875E3,3.0200147E8,8.747058E-3,1.1969853E-2,2.1089124E-2,3.5532564E-2,2.3977574E7,2.3974158E-2,5.253821E-4,6.626653E8,2.1298597E0,1.9577E4,3.557179E1,-2.5136814E-2,-6.2007206E-3,-1.213859E-2,-5.2706926E-4,1.0855529E3,2.2603108E3,1.3617021E0,1.1658537E1,1.10280626E-1,8.24666E5,-5.7944648E-2,-3.3240423E-2,2.6933656E-3,-7.257058E-3,-9.598631E-3,-2.18434E-2,6.126259E5,-1.1654072E-2,1.9E1,-2.3760589E-2,-3.0464254E-2,-1.4454429E-2,-8.001631E-3,-1.7463917E-2,1.1323028E0,1.2015E4,-9.739989E-3,3.202814E5,-1.9681135E-3,-8.750567E-3,1.19514E-2,1.8448765E-4,1E0,-4.928993E-4,1.0376E4,1.1297859E-2,5.0454545E-3,1.7570907E-2,1.6607648E6,1.0851076E-2,1.0982091E-2,2.376619E-2,1.1013051E-2,2.1548262E-3,-3.5379693E-2,-2.4133539E-2,-5.9255497E-3,-2.4588713E-2,4.3875E1,-2.7619105E-2,1E1,1.3717948E0,3.2401234E-2,1.7417E4,-1.4784137E-2,-4.396204E-2,-1.6740995E-2,1.6870629E-3,2.8193703E-2,1.24181E5,8.32E3,1E0,1.3071896E-3,2.2829028E-2,-4.1678282E-3,5.029037E-3,3.5791788E0,1.226617E10,1.975477E-3,4.3698645E6,1.8557197E4,5.1586456E7,1.708775E7,2.6979439E1,-1.7413946E-2,-3.3714388E-2,1.648053E6,4.735985E6,9.112E3,8.25E2,5.332332E-2,3.0449927E-2,1.2516333E7,1.9521575E5,6.702495E-1,2.150349E-5,3.852911E-2,1.6298441E-2,4.5528222E-2,7.223308E-2,-3.9040964E-2,-1.9617936E-2,-1.920528E-2,-6.1193877E-3,-1.5120777E-2,1.6E1,6.1451524E-2,3.8461987E-2,4.8E1,3.455525E-2,9.966725E2,2.2425473E0,-3.8255863E-2,-5.1949255E-2,-3.4047272E-2,-1.761176E-2,-4.7483905E-3,7.2780484E-3,-1.6575463E-2,-3.483392E-3,-1.1285754E-2,-2.4413714E-2,-4.5126122E-3,-1.7875003E-2,-1.6442295E-2,-3.936051E-2,-2.7571223E-3,-2.5050871E-2,-2.750955E-2,-1.45008415E-2,-1.2266369E-2,-2.8794282E-3,-1.6947476E-2,-7.188819E-3,-9.357237E-3,-2.001098E-3,-1.472446E-3,4.4668606E-3,-1.2432578E-2,5.987959E-4,-9.856476E-3,7.336461E-4,6.3270056E-3,1.4797025E-3,-2.1359576E-2,-6.8536755E-3,-1.3350857E-2,-2.20228E-2,6.3245893E-3,-7.134138E-3,2.885556E-4,-1.5483975E-2,8.342529E-3,-4.8097125E-3,-1.1784895E-2,1.0551216E-2,1.6420102E-3,-5.546946E-3,1.252376E-2,4.353961E-3,-4.4136485E-3,-1.165749E-2,5.2182595E-3,-1.4530794E-2,-5.5917836E-4,1.3979073E-2,2.6429722E-2,1.2263377E-2,1.3398423E-2,2.739607E-2,2.3395396E-3,-1.1809137E-2,1.4504242E-2,1.4204131E-3,3.378108E-3,1.2242553E-2,-8.963961E-3,5.273077E-3,-5.9384573E-3,1.5311123E-2,3.154864E-2,1.6978716E-2,3.158981E-3,1.5654348E-2,2.2888241E-2,9.290088E-3,2.3942392E-2,1.5576672E-2,2.6803238E-2,8.02982E-3,-7.592222E-3,9.755273E-4,7.3496904E-3,2.0237582E-2,3.562256E-2,5.359496E-2,1.7622434E-2,3.782477E-2],"split_indices":[20,102,12,17,52,28,28,1,2,39,2,2,27,45,6,0,0,45,45,52,7,41,57,29,0,2,52,2,52,54,52,4,56,4,52,45,45,8,42,31,2,28,0,53,54,0,9,28,2,32,0,52,39,53,2,2,100,0,2,3,45,56,0,45,7,0,0,0,77,52,10,38,0,9,28,28,8,29,45,0,0,9,3,2,4,3,58,54,28,3,0,3,33,28,54,58,56,33,32,4,0,0,12,53,50,0,0,54,48,32,0,0,0,53,4,0,0,2,31,32,32,4,0,9,0,4,39,0,0,28,56,7,38,12,0,39,0,4,0,28,1,54,45,45,0,9,0,8,28,54,1,4,9,2,0,47,0,52,10,11,0,0,0,45,2,102,2,0,43,2,9,10,4,7,0,0,0,0,47,0,0,7,38,9,58,0,0,0,0,52,4,53,57,27,29,0,0,0,0,0,0,33,0,8,0,0,0,0,0,42,9,0,28,0,0,0,0,105,0,9,0,0,0,45,0,0,0,0,0,0,0,0,0,4,0,8,53,0,29,0,0,0,0,0,1,2,15,57,0,0,0,54,31,0,45,47,7,45,58,0,0,1,1,10,0,57,0,32,33,39,0,0,0,0,0,0,0,0,0,0,8,0,0,8,0,4,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.578E3,8.26E2,1.752E3,4.54E2,3.72E2,1.184E3,5.68E2,1E1,4.44E2,2.28E2,1.44E2,5.1E2,6.74E2,3.94E2,1.74E2,4E0,6E0,2.76E2,1.68E2,1.2E2,1.08E2,1.18E2,2.6E1,2.77E2,2.33E2,6.31E2,4.3E1,3.29E2,6.5E1,1.48E2,2.6E1,2.18E2,5.8E1,8.6E1,8.2E1,5.8E1,6.2E1,4E1,6.8E1,6.7E1,5.1E1,1.9E1,7E0,8.9E1,1.88E2,7E0,2.26E2,2.85E2,3.46E2,3.8E1,5E0,2.19E2,1.1E2,4.7E1,1.8E1,5.7E1,9.1E1,7E0,1.9E1,2.01E2,1.7E1,3.5E1,2.3E1,7.8E1,8E0,6.3E1,1.9E1,4E0,5.4E1,3.2E1,3E1,3.3E1,7E0,4.7E1,2.1E1,5E1,1.7E1,2.4E1,2.7E1,4E0,1.5E1,5.7E1,3.2E1,1.59E2,2.9E1,1.84E2,4.2E1,2.2E2,6.5E1,3.38E2,8E0,2.6E1,1.2E1,9.3E1,1.26E2,6.9E1,4.1E1,2.2E1,2.5E1,1.3E1,5E0,5E0,5.2E1,9E0,8.2E1,6E0,1.3E1,1.82E2,1.9E1,1.1E1,6E0,2.3E1,1.2E1,3.8E1,4E1,4E0,4E0,4.6E1,1.7E1,8E0,1.1E1,4.8E1,6E0,2.5E1,7E0,1.5E1,1.5E1,4E0,2.9E1,2.9E1,1.8E1,1E1,1.1E1,3.4E1,1.6E1,1.2E1,5E0,1.9E1,5E0,1.6E1,1.1E1,4.8E1,9E0,2.8E1,4E0,8.7E1,7.2E1,1E1,1.9E1,1.3E1,1.71E2,3.1E1,1.1E1,2.16E2,4E0,5.8E1,7E0,2.76E2,6.2E1,2.2E1,4E0,5E0,7E0,5.5E1,3.8E1,4.1E1,8.5E1,4.3E1,2.6E1,2.2E1,1.9E1,1.1E1,1.1E1,1.8E1,7E0,7E0,6E0,9E0,4.3E1,5E0,4E0,1.9E1,6.3E1,1.26E2,5.6E1,1.5E1,4E0,5E0,6E0,2.3E1,1.5E1,2.7E1,1.3E1,2.7E1,1.9E1,9E0,8E0,4E0,4E0,6E0,5E0,4.4E1,4E0,2E1,5E0,1.1E1,4E0,5E0,1E1,2.1E1,8E0,8E0,1E1,4E0,6E0,6E0,5E0,2.7E1,7E0,9E0,7E0,5E0,7E0,1.1E1,8E0,6E0,1E1,7E0,4E0,1.4E1,3.4E1,5E0,4E0,2.4E1,4E0,5.8E1,2.9E1,8E0,6.4E1,4E0,6E0,1.5E1,4E0,5E0,8E0,8.4E1,8.7E1,2.2E1,9E0,5E0,6E0,4.8E1,1.68E2,7E0,5.1E1,1.48E2,1.28E2,2.7E1,3.5E1,4E0,1.8E1,3.8E1,1.7E1,1.6E1,2.2E1,1E1,3.1E1,4.8E1,3.7E1,1.9E1,7E0,1.8E1,4E0,1.2E1,7E0,7E0,4E0,7E0,4E0,4E0,1.4E1,3.5E1,8E0,1E1,9E0,3.5E1,2.8E1,1.06E2,2E1,4.7E1,9E0,1.5E1,8E0,1.1E1,4E0,5E0,2.2E1,9E0,4E0,4E0,2.3E1,4E0,1.5E1,4E1,4E0,1.6E1,4E0,1.6E1,5E0,4E0,4E0,5E0,5E0,2.3E1,4E0,5E0,4E0,7E0,4E0,1.2E1,1.2E1,4.7E1,1.1E1,4E0,2.5E1,4.9E1,1.5E1,4E0,4E0,8E1,4E0,5.7E1,3E1,1.7E1,5E0,1.8E1,3E1,1.55E2,1.3E1,4E0,4.7E1,3E1,1.18E2,2.3E1,1.05E2,1.5E1,1.2E1,2.4E1,1.1E1,2.1E1,1.7E1,9E0,8E0,6E0,1E1,8E0,1.4E1,4E0,6E0,8E0,4E1,2.2E1,1.5E1,1.1E1,8E0,5E0,9E0,4E0,6E0,1.3E1,2.2E1,4E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"373","size_leaf_vector":"1"}},{"base_weights":[-4.239018E-3,-3.9003804E-1,1.8612792E-1,-6.169906E-1,-1.795957E-1,1.2676618E-2,4.3902725E-1,-7.3034364E-1,-3.492479E-1,-2.9124606E-1,2.5146064E-1,-6.9590606E-2,2.8318185E-1,2.8959906E-1,6.599312E-1,-6.953038E-1,-1.3414297E0,-4.7192392E-1,-2.2687478E-1,-3.2810757E-1,7.593498E-2,1.5646157E-1,6.446061E-1,-2.556809E-1,2.3226112E-2,3.0821532E-1,-2.2798888E-2,4.628181E-2,4.54624E-1,4.3420735E-1,8.1098115E-1,-7.4634707E-1,-3.803775E-1,-8.1369616E-2,-3.4341387E-2,-5.033483E-1,-1.5924771E-3,-3.3934838E-1,-1.4690863E-1,-2.2966857E-1,-5.8668846E-1,-1.1742371E-2,2.675901E-1,7.826495E-1,1.676852E-2,-3.8826683E-1,-1.5004939E-1,8.8393956E-1,-1.0660695E-3,1.7415562E-1,4.570733E-1,1.4872639E-1,-3.027085E-1,2.9143414E-1,6.663266E-1,3.127369E-1,6.625251E-1,9.397122E-1,5.917301E-1,-6.758655E-1,-9.763256E-1,-5.689133E-2,-4.620226E-1,-4.4843656E-1,-3.8469348E-2,-3.874889E-1,-9.424393E-3,-1.1081614E-2,-2.1709539E-1,-2.9708838E-1,6.0676306E-2,-6.8474525E-1,-2.225561E-1,1.7018089E-1,-1.2602325E-1,3.3350345E-2,3.2687032E-1,4.792064E-2,2.37744E-2,-4.053362E-1,-9.569438E-4,-8.4483616E-2,-2.36802E-1,3.0821139E-2,5.3110868E-2,2.6672272E-2,-6.310719E-1,1.9281939E-1,-1.4673955E-2,3.0764493E-1,6.0069704E-1,7.1998914E-3,2.1696676E-1,-4.1155356E-1,-1.2732443E-1,-1.0054758E-2,3.2798564E-1,5.9451866E-3,6.98297E-1,-5.5911466E-3,3.580948E-1,4.1985473E-1,3.8570248E-2,6.961271E-1,1.0425904E0,6.4269704E-1,2.1380576E-5,-4.0849864E-1,-7.233673E-1,-7.299092E-1,-1.0998256E0,-9.059226E-3,3.94242E-3,-5.309765E-1,-1.44094555E-2,-4.692645E-1,-8.767315E-3,-1.114989E-2,-2.4815852E-2,4.315305E-3,-7.481386E-2,-1.3643004E-1,-1.46459155E-2,-6.3507324E-1,-2.6202893E-1,1.2054166E-1,-1.8412411E-2,-4.2412415E-1,-7.637648E-1,-6.423332E-2,-3.5793237E-2,2.3795983E-3,1.1267809E-2,-1.3378035E-2,-5.6286186E-2,6.8108365E-3,-2.898354E-3,4.552624E-1,1.9478951E-1,-3.8534573E-1,-3.6779102E-2,-1.6064769E-1,-9.8973075E-3,-1.1771818E-1,-3.0095908E-1,1.3593293E-2,6.288852E-2,-1.1705174E-2,-3.550929E-2,7.4664794E-2,2.4942464E-1,3.9794357E-3,3.3990833E-1,3.2814488E-1,6.489885E-1,-1.2931967E-2,6.611362E-2,3.1148466E-1,1.2182817E-1,-6.8296716E-3,-2.3999615E-2,-1.9928206E-2,-1.3119169E-2,1.07053155E-2,-1.02676824E-1,3.4736845E-1,-1.013318E-2,5.2994263E-1,8.0345356E-1,2.2595175E-2,2.8091258E-1,2.532747E-2,1.0959189E-2,1.9728744E-2,3.739788E-2,1.120072E0,6.226643E-1,1.3699905E-2,6.879016E-1,-4.6865663E-1,-7.6189074E-3,-7.8040695E-1,-5.2264816E-1,-4.082755E-2,-1.5236366E-2,-5.9269316E-2,-3.4601856E-2,-3.3396527E-2,-1.8599026E-2,-2.5152927E-2,-3.0288437E-1,-6.604629E-3,-1.241288E-4,-5.873317E-4,-9.25007E-3,-1.47886425E-2,-3.5517264E-2,-3.052697E-1,-1.1515145E-1,1.7304946E-1,-1.512121E-2,-1.9923203E-1,-5.6957006E-1,-8.6752367E-1,-6.1865824E-1,9.584866E-3,-1.5777634E-1,-1.5461348E-1,5.0259368E-3,1.0074322E-2,2.7189093E-2,9.207071E-2,1.6215134E-2,-4.645014E-1,-2.921507E-1,-7.5821325E-2,-2.2373697E-1,2.7128898E-2,-1.3479668E-1,-1.604501E-1,3.4080618E-3,-2.0386491E-2,-1.996987E-1,3.1842805E-2,-2.5465444E-2,2.8133115E-1,-5.2321605E-2,2.1048127E-1,4.4607306E-1,4.7143182E-1,2.9162005E-1,8.925985E-3,1.9888856E-2,1.9493064E-2,7.0440805E-1,1.9918546E-1,-6.34341E-2,1.9917123E-1,4.8856536E-1,3.895695E-2,2.066955E-1,3.2844187E-3,-5.9327944E-3,-8.074361E-3,-8.407493E-4,4.0103024E-1,1.7301542E-1,6.01572E-1,3.067073E-1,8.6397886E-1,1.8547837E-2,3.308121E-1,7.227956E-2,4.493771E-2,1.2238421E0,3.522516E-2,1.7487105E-2,1.270949E-2,7.1938694E-1,-2.563339E-2,-1.03122825E-2,-1.8343952E-2,-3.9773308E-2,-1.9801399E-2,-3.2769382E-2,-1.798974E-2,-8.680558E-3,-1.1735681E-2,-1.9616537E-2,-1.2611744E-2,-1.2226274E-3,1.605106E-2,3.6587038E-3,-1.3499806E-2,-4.4191238E-3,-1.23173455E-2,-3.592399E-2,-2.663645E-2,-4.65485E-2,-2.2426354E-2,-4.044526E-2,-1.1710591E-2,5.818114E-4,-1.0160264E-2,-3.7456737E-3,4.1332658E-4,6.6423705E-3,-1.6044365E-2,-2.4473403E-2,-1.0645946E-2,-2.622483E-2,-1.11656375E-2,-1.0769371E-3,-1.5396483E-2,-8.78533E-3,3.946024E-3,-2.6270093E-3,-1.1814726E-2,7.069902E-4,-3.112952E-3,-1.0041177E-2,-3.3293485E-3,-1.1270855E-2,-2.575399E-3,7.1736937E-3,5.5668154E-3,1.663851E-2,-1.1240375E-2,6.2128054E-3,6.43884E-3,1.4061728E-2,1.2162211E-2,2.5831794E-2,1.4734336E-2,2.8451437E-2,1.570081E-2,6.1185863E-3,3.7792135E-2,2.1714054E-2,1.620244E-2,1.902556E-3,4.2670127E-3,-6.130307E-3,1.4420584E-2,4.6627337E-4,2.8714348E-2,9.203907E-3,-1.550095E-3,7.946522E-3,2.0783115E-2,6.030281E-3,1.2148213E-2,2.4981586E-2,4.897639E-3,1.5649969E-2,3.358934E-2,1.9638028E-2,5.500307E-3,2.1557122E-2,2.5843466E-2,4.490607E-2,1.8133828E-2,4.239868E-3,-5.866905E-5,6.5594926E-3,6.36457E-2,3.295699E-2,3.004485E-2,4.5499813E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,53,55,57,59,61,-1,-1,63,-1,65,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,-1,119,121,123,125,127,129,131,133,135,137,-1,-1,139,-1,141,143,-1,-1,145,147,149,-1,151,153,155,157,159,161,163,165,-1,167,-1,169,171,-1,173,175,177,-1,179,181,183,185,-1,-1,187,-1,189,-1,-1,-1,-1,191,193,-1,195,197,199,-1,201,203,205,-1,-1,-1,-1,207,-1,-1,209,211,213,-1,215,217,219,221,223,-1,-1,-1,225,227,-1,229,231,233,-1,235,237,239,-1,-1,241,-1,-1,243,245,-1,247,249,-1,251,-1,-1,-1,-1,253,255,-1,257,259,-1,261,263,-1,-1,-1,-1,-1,-1,-1,265,-1,-1,-1,-1,-1,-1,267,269,271,-1,273,275,277,279,-1,281,283,-1,-1,-1,285,-1,287,289,291,293,295,297,299,-1,-1,301,303,-1,305,307,309,311,313,315,-1,-1,-1,317,319,321,323,325,327,329,-1,-1,-1,-1,331,333,335,337,339,-1,341,343,-1,345,-1,-1,-1,347,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8360759E2,3.933931E1,7.342566E1,1.1793655E1,2.07681E1,2.2139898E1,2.2288193E1,5.019928E0,1.6944017E0,2.3328722E1,3.223157E0,1.3184695E1,4.4352627E0,1.6361553E1,8.990059E0,3.9759521E0,1.5095615E0,7.998333E-1,5.077796E-1,8.41795E0,0E0,1.3676771E0,4.6251726E-1,3.499075E0,1.0664331E1,4.410467E0,0E0,5.97988E0,8.200901E0,2.8989964E0,4.0445175E0,3.019638E0,1.003746E0,0E0,0E0,4.5977497E-1,0E0,1.0832596E-1,3.5985816E-1,4.8139E0,3.1325302E0,6.451572E-1,6.000731E-1,1.3908768E-1,0E0,6.979008E-1,7.933786E-1,1.3667107E-2,8.702066E0,1.1044037E0,2.0901127E0,1.244297E0,6.730075E-1,1.5350599E0,1.7542953E0,1.4846096E0,6.8468475E-1,1.8052444E0,1.8983498E0,1.9524765E0,7.4928284E-1,1.6635546E-1,2.1804619E-1,1.905632E-1,0E0,2.1208525E-1,0E0,9.340691E-2,1.1331153E-1,2.1916542E0,1.2519629E0,1.1588478E0,1.5957165E0,7.062718E-2,1.7270419E-1,1.0196435E-1,5.048542E-1,0E0,0E0,4.237976E-1,0E0,4.6991622E-1,4.3061566E-1,0E0,0E0,7.6685452E0,4.967289E-1,7.57205E-1,0E0,3.7016153E-1,4.8462105E-1,6.909961E-1,7.443824E-1,3.6904144E-1,2.2714558E-1,3.5440627E-1,1.3461695E0,0E0,1.32687E0,0E0,3.6689758E-1,1.3315535E-1,0E0,2.2476959E-1,1.5475845E0,7.775879E-1,0E0,3.75185E-1,1.2980194E0,4.918909E-1,3.041191E-1,0E0,0E0,2.0926142E-1,0E0,9.73835E-2,0E0,0E0,0E0,0E0,3.7015542E-2,8.281866E-2,0E0,2.5196075E-1,1.1207466E0,9.855925E-1,0E0,5.080066E-1,2.7632713E-1,4.3745783E-1,0E0,0E0,0E0,0E0,2.3149909E-1,0E0,0E0,2.7354097E-1,2.2330886E-1,6.2411785E-1,0E0,2.024529E-1,2.0251252E-1,1.936771E-1,3.412509E-1,4.532863E0,0E0,0E0,0E0,1.0382285E0,5.034456E-1,0E0,1.6102028E-1,3.339541E-2,3.5643005E-1,0E0,6.3324463E-1,7.578993E-1,3.0843544E-1,0E0,0E0,9.244306E-2,0E0,0E0,5.797887E-2,1.0533466E0,0E0,4.7109222E-1,1.222847E0,0E0,3.925166E-1,0E0,0E0,0E0,0E0,2.048645E-1,4.505825E-2,0E0,5.131779E-1,1.616745E-1,0E0,4.4226837E-1,3.4589767E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1266232E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.6843834E-1,5.290165E-1,5.386081E-1,0E0,4.684481E-2,3.536446E-1,1.2475777E-1,4.791422E-1,0E0,1.7141947E-1,1.9687071E-2,0E0,0E0,0E0,3.9993875E-2,0E0,3.8586617E-2,8.083687E-1,1.4741512E-1,3.8829565E-2,1.4464682E-1,1.5475728E-1,6.302014E-2,0E0,0E0,6.0700417E-2,4.27429E0,0E0,9.457517E-2,7.9041994E-1,3.5167146E-1,4.6610594E-2,2.2364616E-2,1.0301447E-1,0E0,0E0,0E0,1.6184044E-1,3.5037893E-1,1.7803484E-1,4.7268784E-1,3.134606E-1,2.066083E-1,3.45735E-1,0E0,0E0,0E0,0E0,1.300065E0,2.8472233E-1,1.778984E-1,2.467959E-1,1.4957428E-1,0E0,2.2190356E-1,3.9156623E-2,0E0,1.7391968E-1,0E0,0E0,0E0,5.1374435E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,79,79,81,81,82,82,85,85,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,98,98,100,100,101,101,103,103,104,104,105,105,107,107,108,108,109,109,110,110,113,113,115,115,120,120,121,121,123,123,124,124,125,125,127,127,128,128,129,129,134,134,137,137,138,138,139,139,141,141,142,142,143,143,144,144,145,145,149,149,150,150,152,152,153,153,154,154,156,156,157,157,158,158,161,161,164,164,165,165,167,167,168,168,170,170,175,175,176,176,178,178,179,179,181,181,182,182,190,190,197,197,198,198,199,199,201,201,202,202,203,203,204,204,206,206,207,207,211,211,213,213,214,214,215,215,216,216,217,217,218,218,219,219,222,222,223,223,225,225,226,226,227,227,228,228,229,229,230,230,234,234,235,235,236,236,237,237,238,238,239,239,240,240,245,245,246,246,247,247,248,248,249,249,251,251,252,252,254,254,258,258],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,54,56,58,60,62,-1,-1,64,-1,66,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,-1,120,122,124,126,128,130,132,134,136,138,-1,-1,140,-1,142,144,-1,-1,146,148,150,-1,152,154,156,158,160,162,164,166,-1,168,-1,170,172,-1,174,176,178,-1,180,182,184,186,-1,-1,188,-1,190,-1,-1,-1,-1,192,194,-1,196,198,200,-1,202,204,206,-1,-1,-1,-1,208,-1,-1,210,212,214,-1,216,218,220,222,224,-1,-1,-1,226,228,-1,230,232,234,-1,236,238,240,-1,-1,242,-1,-1,244,246,-1,248,250,-1,252,-1,-1,-1,-1,254,256,-1,258,260,-1,262,264,-1,-1,-1,-1,-1,-1,-1,266,-1,-1,-1,-1,-1,-1,268,270,272,-1,274,276,278,280,-1,282,284,-1,-1,-1,286,-1,288,290,292,294,296,298,300,-1,-1,302,304,-1,306,308,310,312,314,316,-1,-1,-1,318,320,322,324,326,328,330,-1,-1,-1,-1,332,334,336,338,340,-1,342,344,-1,346,-1,-1,-1,348,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,7.7344055E4,1E0,1.4580295E0,1.022332E6,8.631E3,1.2427474E8,2.3859661E-2,1.3144558E4,1.8274589E-3,6.76E2,3E0,4.930349E5,2.1150263E-1,5.121E3,7.10309E2,7.6E1,2.028994E8,6.1061732E7,7.593498E-2,1.5190727E5,2.1367595E0,6.6126086E4,8E0,9.19E2,-2.2798888E-2,2.7246006E7,3.171E3,2.1019447E-1,8.5E1,3.0161016E7,1.9051096E7,-8.1369616E-2,-3.4341387E-2,2.0728285E6,-1.5924771E-3,3.7382904E7,3.154992E7,1.1210787E3,3.0497742E3,1.0275E4,2.091623E5,8.639937E9,1.676852E-2,4.564527E6,4.4E0,7.425E3,1E0,6.3251557E0,2.3314082E6,8.756142E7,1.6213043E3,1.8122449E2,1.683653E8,3.0324321E-2,1E0,3.4867987E-1,3.7284137E2,1.3062E4,2.8035452E11,1.4783E4,2.6835732E3,2.315878E6,-3.8469348E-2,1.3607962E5,-9.424393E-3,5.895285E6,5.03E2,1E0,1E0,2.060792E8,5.784E4,7.5569354E2,1.3E1,2.1111E4,4.9716983E0,4.792064E-2,2.37744E-2,1.919676E-5,-9.569438E-4,3.6E2,9.25E0,3.0821139E-2,5.3110868E-2,8.791777E3,1.9051096E7,1E0,-1.4673955E-2,2.7547675E2,5.0338E4,1.431413E-1,9.076336E6,7.298614E2,4.396066E1,1.0346457E0,2.5506506E0,5.9451866E-3,1.5787431E0,-5.5911466E-3,1.3181542E6,1.5012488E1,3.8570248E-2,1.8008E4,3.1029554E7,1E0,2.1380576E-5,7.718584E5,4.1384277E2,2.183E3,1.527638E12,-9.059226E-3,3.94242E-3,1.2151E4,-1.44094555E-2,1E0,-8.767315E-3,-1.114989E-2,-2.4815852E-2,4.315305E-3,2.011E3,2.047619E0,-1.46459155E-2,5.503109E8,4.41E4,1.5793961E1,-1.8412411E-2,5.37456E5,2.3023027E3,9.11898E5,-3.5793237E-2,2.3795983E-3,1.1267809E-2,-1.3378035E-2,9.2339675E5,6.8108365E-3,-2.898354E-3,3.904E3,2.2981277E3,3.79E2,-3.6779102E-2,3.1E1,3.627907E0,1.4E1,1.5782692E2,6.856226E-1,6.288852E-2,-1.1705174E-2,-3.550929E-2,4.144144E-1,6.789622E7,3.9794357E-3,8E0,7.4521235E6,1E0,-1.2931967E-2,1E0,7.2041174E2,7.089473E6,-6.8296716E-3,-2.3999615E-2,2.376302E8,-1.3119169E-2,1.07053155E-2,1.1521033E0,1E0,-1.013318E-2,1E0,2.9375E0,2.2595175E-2,2.0074467E5,2.532747E-2,1.0959189E-2,1.9728744E-2,3.739788E-2,1.2679E4,4.514673E-3,1.3699905E-2,4.423676E1,6.3251557E0,-7.6189074E-3,1.4462532E7,9.619E3,-4.082755E-2,-1.5236366E-2,-5.9269316E-2,-3.4601856E-2,-3.3396527E-2,-1.8599026E-2,-2.5152927E-2,3.7037696E7,-6.604629E-3,-1.241288E-4,-5.873317E-4,-9.25007E-3,-1.47886425E-2,-3.5517264E-2,1.6256282E7,1.462136E6,1.1317E4,-1.512121E-2,6.015E3,3.7E1,2.7864855E11,1.2E1,9.584866E-3,1.7994144E0,1.82E4,5.0259368E-3,1.0074322E-2,2.7189093E-2,3.8657358E6,1.6215134E-2,1.901E3,6E0,8.9E1,2.08E2,1E0,4E0,1.07882355E2,3.4080618E-3,-2.0386491E-2,1.3984146E2,1.7669278E5,-2.5465444E-2,4.56E2,1.2673605E7,2.6682352E2,1.359E3,4.1384277E2,7.382963E0,8.925985E-3,1.9888856E-2,1.9493064E-2,2E0,1.8662969E6,4.7779464E7,2.0666666E0,4.4993465E1,2.8721272E1,9.536135E6,3.2844187E-3,-5.9327944E-3,-8.074361E-3,-8.407493E-4,4.3957013E-1,2.632E5,2E0,2.2243262E5,9.966725E2,1.8547837E-2,3.13E2,4.798624E3,4.493771E-2,7.6499896E9,3.522516E-2,1.7487105E-2,1.270949E-2,4.9807886E3,-2.563339E-2,-1.03122825E-2,-1.8343952E-2,-3.9773308E-2,-1.9801399E-2,-3.2769382E-2,-1.798974E-2,-8.680558E-3,-1.1735681E-2,-1.9616537E-2,-1.2611744E-2,-1.2226274E-3,1.605106E-2,3.6587038E-3,-1.3499806E-2,-4.4191238E-3,-1.23173455E-2,-3.592399E-2,-2.663645E-2,-4.65485E-2,-2.2426354E-2,-4.044526E-2,-1.1710591E-2,5.818114E-4,-1.0160264E-2,-3.7456737E-3,4.1332658E-4,6.6423705E-3,-1.6044365E-2,-2.4473403E-2,-1.0645946E-2,-2.622483E-2,-1.11656375E-2,-1.0769371E-3,-1.5396483E-2,-8.78533E-3,3.946024E-3,-2.6270093E-3,-1.1814726E-2,7.069902E-4,-3.112952E-3,-1.0041177E-2,-3.3293485E-3,-1.1270855E-2,-2.575399E-3,7.1736937E-3,5.5668154E-3,1.663851E-2,-1.1240375E-2,6.2128054E-3,6.43884E-3,1.4061728E-2,1.2162211E-2,2.5831794E-2,1.4734336E-2,2.8451437E-2,1.570081E-2,6.1185863E-3,3.7792135E-2,2.1714054E-2,1.620244E-2,1.902556E-3,4.2670127E-3,-6.130307E-3,1.4420584E-2,4.6627337E-4,2.8714348E-2,9.203907E-3,-1.550095E-3,7.946522E-3,2.0783115E-2,6.030281E-3,1.2148213E-2,2.4981586E-2,4.897639E-3,1.5649969E-2,3.358934E-2,1.9638028E-2,5.500307E-3,2.1557122E-2,2.5843466E-2,4.490607E-2,1.8133828E-2,4.239868E-3,-5.866905E-5,6.5594926E-3,6.36457E-2,3.295699E-2,3.004485E-2,4.5499813E-2],"split_indices":[20,56,33,102,42,28,2,45,38,52,34,2,11,28,38,2,52,8,7,45,0,33,39,28,18,2,0,45,2,35,8,45,45,0,0,32,0,12,12,52,52,9,28,5,0,9,54,1,73,54,28,7,4,52,7,38,102,38,58,9,31,9,4,9,0,47,0,32,0,75,104,7,10,4,3,9,54,0,0,37,0,2,58,0,0,52,45,102,0,55,1,38,32,52,58,53,41,0,38,0,47,54,0,2,47,16,0,47,4,2,31,0,0,9,0,105,0,0,0,0,2,58,0,5,10,57,0,28,55,29,0,0,0,0,47,0,0,2,52,1,0,10,56,8,4,27,0,0,0,53,7,0,3,45,102,0,16,52,48,0,0,7,0,0,42,6,0,6,54,0,28,0,0,0,0,2,57,0,56,54,0,5,10,0,0,0,0,0,0,0,7,0,0,0,0,0,0,45,32,9,0,2,3,31,18,0,57,10,0,0,0,32,0,9,8,12,2,16,8,52,0,0,52,28,0,0,45,55,29,4,54,0,0,0,17,47,7,53,56,53,48,0,0,0,0,38,33,17,33,4,0,3,4,0,5,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.498E3,8.25E2,1.673E3,3.96E2,4.29E2,9.93E2,6.8E2,2.77E2,1.19E2,3.41E2,8.8E1,7.62E2,2.31E2,4.07E2,2.73E2,2.64E2,1.3E1,5.8E1,6.1E1,3.35E2,6E0,7.2E1,1.6E1,2.53E2,5.09E2,2.24E2,7E0,1.65E2,2.42E2,1.11E2,1.62E2,2.26E2,3.8E1,8E0,5E0,5.4E1,4E0,2.4E1,3.7E1,2.44E2,9.1E1,2.9E1,4.3E1,1E1,6E0,1.11E2,1.42E2,1.3E1,4.96E2,1.19E2,1.05E2,1.28E2,3.7E1,1.38E2,1.04E2,7.4E1,3.7E1,1E2,6.2E1,1.76E2,5E1,8E0,3E1,4.7E1,7E0,1.7E1,7E0,1.3E1,2.4E1,1.98E2,4.6E1,7.1E1,2E1,1.1E1,1.8E1,9E0,3.4E1,5E0,5E0,1.06E2,5E0,8.2E1,6E1,7E0,6E0,4.76E2,2E1,1.15E2,4E0,5.3E1,5.2E1,4.2E1,8.6E1,2.2E1,1.5E1,1.5E1,1.23E2,6E0,9.8E1,7E0,6.7E1,1.3E1,2.4E1,3.2E1,6.8E1,5.7E1,5E0,2.8E1,1.48E2,1.9E1,3.1E1,4E0,4E0,2E1,1E1,4.3E1,4E0,8E0,9E0,5E0,8E0,1.3E1,1.1E1,1.7E1,1.81E2,4.1E1,5E0,1.8E1,5.3E1,1.6E1,4E0,4E0,7E0,5E0,1.3E1,4E0,5E0,1.6E1,1.8E1,1.02E2,4E0,4E1,4.2E1,2.2E1,3.8E1,4.72E2,4E0,4E0,1.6E1,3.8E1,7.7E1,7E0,4.6E1,9E0,4.3E1,7E0,3.5E1,4.2E1,4.4E1,5E0,1.7E1,9E0,6E0,4E0,1.1E1,1.19E2,4E0,4E1,5.8E1,2.8E1,3.9E1,8E0,5E0,6E0,2.6E1,5.6E1,1.2E1,7E0,5E1,2.2E1,6E0,1.13E2,3.5E1,1.5E1,4E0,2.4E1,7E0,9E0,1.1E1,3.4E1,9E0,4E0,4E0,4E0,9E0,4E0,1.3E1,1.39E2,4.2E1,3.7E1,4E0,8E0,1E1,2.8E1,2.5E1,4E0,1.2E1,8E0,5E0,5E0,1.1E1,1.1E1,7E0,5.3E1,4.9E1,1.8E1,2.2E1,3.3E1,9E0,1.8E1,4E0,1.7E1,2.1E1,4.57E2,1.5E1,1.4E1,2.4E1,6.6E1,1.1E1,1E1,3.6E1,4E0,5E0,9E0,3.4E1,1.7E1,1.8E1,2.7E1,1.5E1,2.3E1,2.1E1,5E0,4E0,6E0,5E0,9E1,2.9E1,2.9E1,1.1E1,5E1,8E0,3.1E1,8E0,2.1E1,3.5E1,8E0,4E0,4E0,4.6E1,1.8E1,4E0,5E0,1.08E2,2E1,1.5E1,5E0,4E0,7.9E1,6E1,1.6E1,2.6E1,1.4E1,2.3E1,4E0,4E0,4E0,6E0,6E0,2.2E1,1.5E1,1E1,8E0,4E0,4E0,4E0,4E0,7E0,1E1,4.3E1,3.8E1,1.1E1,4E0,1.4E1,6E0,1.6E1,2E1,1.3E1,5E0,4E0,6E0,1.2E1,4E0,1.7E1,2.62E2,1.95E2,4E0,1E1,1.2E1,1.2E1,3.2E1,3.4E1,4E0,7E0,5E0,5E0,3.1E1,5E0,2.7E1,7E0,9E0,8E0,5E0,1.3E1,1.8E1,9E0,1.1E1,4E0,1.5E1,8E0,5E0,1.6E1,3.6E1,5.4E1,2E1,9E0,2E1,9E0,5E0,6E0,6E0,4.4E1,2.7E1,4E0,4E0,4E0,3.1E1,4E0,3.1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"349","size_leaf_vector":"1"}},{"base_weights":[2.6021511E-3,-3.7055874E-1,1.701766E-1,-5.7357776E-1,-1.2195901E-1,9.276306E-3,4.0971124E-1,-6.830447E-1,-2.51726E-1,-2.7290267E-1,2.1039487E-1,-1.3280487E-1,1.6563138E-1,2.9475272E-1,6.7515105E-1,-2.5835243E-1,-7.447244E-1,-2.728902E-3,-4.7162795E-1,-3.3794832E-1,-1.4419223E-1,4.0363673E-2,4.4830328E-1,-3.619783E-1,-6.85212E-2,8.468348E-2,3.523691E-1,7.503329E-2,4.6159905E-1,1.0838221E0,5.9776694E-1,-4.7637865E-1,1.8038844E-1,-6.865338E-1,-1.0478469E0,-1.05570935E-1,5.1277656E-2,-4.0068355E-1,-4.754517E-2,-4.1472036E-1,-2.2346543E-1,-3.04214E-2,-1.0754171E-1,-8.996105E-2,1.4846374E-1,5.1502657E-1,1.4992444E-1,-4.3268058E-1,-2.7650198E-1,7.317276E-1,-9.870002E-2,6.635635E-2,6.767042E-2,2.5760058E-1,6.2277967E-1,1.7025837E-1,-2.3825341E-1,2.7944857E-1,6.294573E-1,5.779945E-2,2.2186037E-2,4.8838738E-1,7.935135E-1,-6.8964423E-3,-5.58645E-1,2.841463E-2,-1.25028975E-2,-7.500178E-1,-3.6637136E-1,-1.1039522E0,-1.9458815E-2,-1.5649952E-1,1.4689633E-2,-5.1064503E-1,-1.523915E-1,-3.4724066E-1,-5.2433467E-1,-2.6020876E-1,-9.412804E-3,-1.705E-1,9.057353E-3,8.4491484E-2,-1.6516852E-1,2.2544943E-1,-3.0710556E-2,3.621756E-1,6.097284E-1,-6.95739E-4,1.3065157E-2,-8.958579E-3,-4.508082E-1,-2.9727116E-1,-4.1330587E-3,1.1895564E-2,4.6692587E-2,-6.3039936E-2,-5.464436E-1,9.748535E-2,-3.8180873E-1,-1.0868647E-1,3.0534977E-1,6.774128E-1,1.0462782E-2,1.8144326E-2,2.825704E-1,-3.061313E-1,3.640534E-2,3.1250006E-1,-1.9173676E-2,7.1221375E-1,3.7333456E-1,5.302625E-1,1.5991075E-1,5.544559E-1,9.424883E-1,-6.4266735E-1,-1.6027793E-2,-5.2560023E-3,4.1106674E-3,-7.7940285E-1,-4.0797028E-1,-3.747091E-2,-4.6798182E-1,-2.3708763E-2,-1.1607119E0,-2.0649513E-2,-3.4131274E-1,-3.623494E-1,-5.9528524E-1,3.313261E-3,-2.1961387E-1,-4.8093846E-1,-2.5263336E-1,-5.8802265E-1,-1.8553747E-2,-8.2375735E-2,-2.9119083E-1,3.1756763E-3,-4.0379413E-3,-7.849747E-2,-3.5529947E-1,-7.086622E-2,1.137816E-2,-5.1354608E-3,1.1315813E-2,2.504895E-3,-2.15798E-1,3.641203E-4,2.770458E-1,3.0697957E-3,-6.650437E-3,2.419065E-2,1.9997372E-1,3.3362508E-2,1.7181918E-2,-2.8635068E-2,-1.9852487E-2,-2.0025924E-1,-3.348248E-1,-1.525441E-1,3.0988064E-2,-3.25112E-3,-6.593716E-1,-5.1287606E-2,1.8541573E-1,-2.449261E-2,-3.2018484E-3,7.5676725E-3,-1.9510169E-2,2.6105103E-1,2.4862127E-2,7.127615E-1,1.648091E-2,1.1131081E-1,-2.1341765E-1,6.265718E-1,2.1619734E-1,-3.5235804E-1,-3.5676777E-3,-2.5844066E-3,7.6200864E-3,1.3401926E-1,3.636805E-1,6.114765E-2,6.655666E-1,4.3956268E-1,1.4393948E-1,4.0398E-1,6.689181E-1,2.9757628E-1,-8.740945E-3,6.1942893E-1,1.4007094E-2,3.9679848E-2,5.829676E-2,-4.0341064E-2,-1.8840194E-2,-7.527305E-1,-6.067929E-2,-2.8648067E-1,-2.6183644E-2,-4.7901566E-3,2.587167E-3,-5.757561E-3,-5.2903885E-1,-6.1045364E-2,-2.6887396E-2,-8.602374E-2,1.1965713E-1,-9.978941E-3,-1.8868377E-2,-2.1841172E-2,-1.1639464E-2,-2.1382231E-2,-3.4931056E-2,-1.4448694E-2,-1.8280568E-3,-2.6409207E-2,-1.5675817E-2,-2.7768734E-1,-1.3355507E-3,-6.3837343E-1,-1.3944961E-2,-7.660774E-3,9.608472E-4,-3.1576407E-1,-1.7759566E-1,-3.7085183E-2,-1.0932948E-2,-2.0699076E-2,-1.1851555E-2,3.774433E-2,-1.3273781E-2,-2.6835868E-1,-1.2725774E-3,4.8896177E-3,3.26926E-1,1.2631686E-2,5.353951E-3,-3.866705E-3,-2.4500775E-1,-3.630558E-1,-1.0482622E-2,-1.3593008E-1,-3.88055E-2,-2.3602441E-2,2.206994E-1,-1.4965116E-2,-3.669372E-2,-1.677535E-2,-2.6549488E-1,2.0729569E-1,-5.584029E-2,2.1797541E-1,3.819494E-1,3.932624E-2,2.5627704E-2,-4.3060213E-2,2.0149595E-1,-2.0756967E-2,-8.274574E-2,3.815348E-2,9.985453E-3,3.175835E-2,2.8329566E-1,-7.317911E-3,-3.8280824E-1,5.375374E-2,2.5662684E-1,4.3674406E-1,1.9020575E-1,4.6374165E-3,6.877708E-1,9.7879935E-3,4.9461237E-1,1.0044316E-2,2.3311735E-3,4.2667958E-1,6.8023116E-3,7.305312E-1,5.2222145E-1,5.612611E-3,1.9356398E-2,3.3849873E-2,1.8061467E-2,-3.953039E-2,-3.032876E-2,-1.8602887E-2,-5.4478208E-3,-1.8466355E-2,-2.9392378E-2,5.6307483E-3,-7.321454E-3,1.2741362E-2,-1.9793995E-3,-1.5928622E-2,-7.946667E-3,-1.7246049E-2,-3.465468E-2,-1.7790752E-2,-9.7657E-3,-4.1905814E-3,-1.095137E-2,4.311717E-3,-4.0105185E-3,4.2837085E-3,-3.7662832E-3,-7.3988843E-3,-1.5925838E-2,1.8658074E-2,9.354607E-3,-7.854806E-3,-1.681364E-2,-1.9550666E-2,-1.1117803E-2,-1.0168702E-2,2.5743993E-3,1.221328E-3,-6.8080234E-3,6.911668E-3,1.905981E-2,-3.489801E-5,-1.721876E-2,-1.81717E-2,-8.9380285E-3,8.0037396E-4,1.1669384E-2,-1.706488E-2,4.0128096E-3,-2.81046E-3,1.1836067E-2,2.298517E-2,8.000278E-3,7.414992E-3,-9.147203E-3,1.5996065E-2,6.7952154E-3,-8.001097E-3,4.4255126E-3,-6.911886E-4,9.926068E-3,1.0036481E-2,2.0813528E-2,-2.1212488E-2,-7.600358E-3,4.673397E-3,-4.21129E-3,7.4591683E-3,1.6943214E-2,2.4457501E-2,8.185784E-3,-1.4417393E-3,1.2085327E-2,3.690999E-2,1.9921552E-2,2.6355503E-2,1.3534089E-2,1.8097505E-2,2.9003348E-2,1.9281657E-2,3.8977068E-2,8.240536E-3,3.0605951E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,-1,79,81,83,85,87,89,91,93,95,97,-1,99,101,103,105,107,109,-1,-1,111,113,-1,115,-1,117,119,121,123,-1,125,-1,127,129,131,133,135,137,139,141,143,145,147,149,151,153,-1,-1,-1,155,157,-1,-1,-1,159,161,163,165,167,169,171,-1,173,175,177,179,181,-1,183,185,187,189,191,193,195,-1,-1,-1,197,199,201,203,-1,205,207,209,211,213,-1,215,217,219,221,-1,223,225,-1,-1,227,229,231,-1,-1,-1,-1,233,-1,235,-1,-1,-1,237,-1,-1,-1,-1,239,241,243,245,-1,247,249,251,-1,-1,-1,-1,253,-1,255,-1,257,259,261,263,265,-1,-1,-1,267,269,-1,271,273,275,277,279,281,-1,283,-1,-1,-1,-1,-1,285,-1,287,-1,-1,-1,-1,289,-1,-1,291,293,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,295,-1,297,-1,-1,-1,299,301,303,-1,-1,-1,305,-1,307,-1,-1,309,-1,-1,-1,311,313,-1,315,-1,317,319,-1,-1,321,323,325,327,329,331,-1,-1,333,335,-1,337,-1,-1,339,341,-1,343,345,347,349,351,-1,353,-1,355,-1,-1,357,-1,359,361,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6102103E2,4.020549E1,6.848392E1,1.5294968E1,1.8117434E1,2.3658876E1,2.1588402E1,8.373322E0,6.1860223E0,2.0275402E0,4.5529647E0,8.195715E0,7.6339855E0,1.8295052E1,6.151428E0,4.1991844E0,4.3076935E0,5.822673E0,1.6615477E0,1.3487339E0,1.3820206E0,9.572594E-1,8.584995E-1,6.1765385E-1,1.0626017E1,8.170496E0,3.7604141E0,6.5142717E0,8.507526E0,1.0263672E0,3.5306091E0,7.30124E-1,1.1305572E0,4.6439056E0,1.1954536E0,1.0634593E0,0E0,1.3965206E0,0E0,5.3541374E-1,5.3237104E-1,0E0,5.888146E-1,4.2067948E-1,5.172964E-1,3.2923126E-1,1.864188E-1,2.1774101E-1,2.0884705E-1,1.378932E0,6.703444E0,4.933997E0,0E0,2.0445275E0,7.389469E-1,2.8479404E0,9.619508E-1,3.144741E0,2.8498955E0,0E0,0E0,1.5658035E0,1.7567024E0,0E0,2.535491E-1,0E0,1.05278626E-1,1.702301E0,1.3860922E0,8.783722E-1,0E0,1.1227615E0,0E0,2.4871445E-1,2.7919114E-1,6.753726E-1,1.227293E-1,2.9557228E-1,6.2445655E-2,8.5621464E-1,5.229384E-1,2.8915268E-1,2.5062966E-1,2.906468E-1,1.213934E-1,2.394824E-1,1.0944557E-1,0E0,0E0,0E0,1.647625E-1,1.2239218E-1,0E0,0E0,0E0,3.303759E0,1.6422892E0,4.313377E0,7.6053405E-1,1.0853623E0,7.386675E-1,1.8264008E-1,0E0,1.5770702E0,2.0564494E0,4.1972423E-1,1.2043421E-1,1.1570435E0,0E0,1.810379E0,5.1404095E-1,1.585804E0,7.4350846E-1,3.0725002E-1,3.3760452E-1,3.9548874E-1,0E0,0E0,0E0,1.2207108E0,1.03662014E-1,6.2293403E-2,6.2825966E-1,0E0,7.1004105E-1,2.5794917E-1,2.226448E-2,5.297661E-2,8.301926E-2,0E0,1.676426E-1,3.3159733E-2,2.1331024E-1,1.7960978E-1,0E0,7.5029194E-2,8.07004E-2,0E0,0E0,2.0106177E-1,2.0123482E-2,4.7545597E-1,0E0,0E0,0E0,0E0,1.7323446E-1,0E0,1.5570462E-1,0E0,0E0,0E0,1.7045587E-2,0E0,0E0,0E0,0E0,7.451284E-2,5.090618E-2,1.978509E0,1.9936448E0,0E0,4.547434E-1,9.0485775E-1,1.10426E0,0E0,0E0,0E0,0E0,3.5940552E-1,0E0,4.1752815E-2,0E0,7.3261905E-1,5.1993257E-1,7.2437763E-1,1.0057979E0,1.5710735E-1,0E0,0E0,0E0,2.912814E-1,1.2097454E0,0E0,1.2460823E0,2.9211807E-1,4.6152562E-2,2.9454803E-1,6.138611E-2,1.3949537E-1,0E0,6.942558E-2,0E0,0E0,0E0,0E0,0E0,5.049591E-1,0E0,1.2693596E-1,0E0,0E0,0E0,0E0,3.7495136E-2,0E0,0E0,2.449451E-1,2.0410398E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1996579E-1,0E0,6.8053246E-2,0E0,0E0,0E0,1.19463205E-1,2.4485618E-2,1.6141665E-1,0E0,0E0,0E0,8.788657E-2,0E0,3.7625313E-2,0E0,0E0,3.399992E-2,0E0,0E0,0E0,4.3503046E-2,2.3991585E-2,0E0,2.5060751E0,0E0,8.151676E-1,5.1662827E-1,0E0,0E0,5.637018E-1,7.951355E-2,9.501724E-1,7.275562E-1,3.5027432E-1,2.8999877E-1,0E0,0E0,5.632745E-1,2.1290481E-1,0E0,1.9977856E-1,0E0,0E0,1.7775467E-1,5.753889E-1,0E0,2.1457434E-1,1.17694624E-1,5.5603087E-2,9.416199E-1,3.6912513E-1,0E0,1.1627731E0,0E0,2.6181221E-2,0E0,0E0,3.1534576E-1,0E0,2.452259E-1,4.797759E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,64,64,66,66,67,67,68,68,69,69,71,71,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,90,90,91,91,95,95,96,96,97,97,98,98,99,99,100,100,101,101,103,103,104,104,105,105,106,106,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,119,119,120,120,121,121,122,122,124,124,125,125,126,126,127,127,128,128,130,130,131,131,132,132,133,133,135,135,136,136,139,139,140,140,141,141,146,146,148,148,152,152,157,157,158,158,159,159,160,160,162,162,163,163,164,164,169,169,171,171,173,173,174,174,175,175,176,176,177,177,181,181,182,182,184,184,185,185,186,186,187,187,188,188,189,189,191,191,197,197,199,199,204,204,207,207,208,208,219,219,221,221,225,225,226,226,227,227,231,231,233,233,236,236,240,240,241,241,243,243,245,245,246,246,249,249,250,250,251,251,252,252,253,253,254,254,257,257,258,258,260,260,263,263,264,264,266,266,267,267,268,268,269,269,270,270,272,272,274,274,277,277,279,279,280,280],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,-1,80,82,84,86,88,90,92,94,96,98,-1,100,102,104,106,108,110,-1,-1,112,114,-1,116,-1,118,120,122,124,-1,126,-1,128,130,132,134,136,138,140,142,144,146,148,150,152,154,-1,-1,-1,156,158,-1,-1,-1,160,162,164,166,168,170,172,-1,174,176,178,180,182,-1,184,186,188,190,192,194,196,-1,-1,-1,198,200,202,204,-1,206,208,210,212,214,-1,216,218,220,222,-1,224,226,-1,-1,228,230,232,-1,-1,-1,-1,234,-1,236,-1,-1,-1,238,-1,-1,-1,-1,240,242,244,246,-1,248,250,252,-1,-1,-1,-1,254,-1,256,-1,258,260,262,264,266,-1,-1,-1,268,270,-1,272,274,276,278,280,282,-1,284,-1,-1,-1,-1,-1,286,-1,288,-1,-1,-1,-1,290,-1,-1,292,294,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,296,-1,298,-1,-1,-1,300,302,304,-1,-1,-1,306,-1,308,-1,-1,310,-1,-1,-1,312,314,-1,316,-1,318,320,-1,-1,322,324,326,328,330,332,-1,-1,334,336,-1,338,-1,-1,340,342,-1,344,346,348,350,352,-1,354,-1,356,-1,-1,358,-1,360,362,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,7.7344055E4,1.983922E3,4.438435E6,2.3770695E5,1.3137E4,8.2608955E-3,6.747114E7,2.7604443E2,1.0855529E3,2.73E2,1.9925156E6,4.930349E5,2E0,1.7495675E5,1.2199979E10,7.9023E4,7.6008E4,1.8367827E-1,1.5944675E-4,1.4126303E0,5.16184E0,3.75E1,8E0,6.7652373E3,1E0,2.7246006E7,3.097E3,4.4263797E0,1.7654233E0,2.127E3,2.2241135E6,3.956E3,8.289561E8,1.3238013E2,5.1277656E-2,3.4409692E3,-4.754517E-2,2.878424E5,9.705292E6,-3.04214E-2,5.4878284E2,1.6254681E0,5.6075206E0,3.3480989E3,2.3175376E3,7E0,3.30399E6,3.192E3,1E0,4.898012E-1,6.767042E-2,1.1317E4,1.5E1,6.308896E2,1.6E1,5.2107143E0,4.072289E0,5.779945E-2,2.2186037E-2,2.851369E2,6.513793E1,-6.8964423E-3,3.3333335E-2,2.841463E-2,2.0368582E8,4.22088E0,1.89777E5,3.9343938E-1,-1.9458815E-2,5.418831E11,1.4689633E-2,5.198287E5,7.6499896E9,1.074E3,2.2348747E2,3.87E2,1.43709E0,1.6256282E7,1.484789E6,2.4513194E5,1.06158945E5,2.091623E5,7.7854166E0,2.2100918E1,5.9124074E0,-6.95739E-4,1.3065157E-2,-8.958579E-3,9.2058825E-1,1.00033E5,-4.1330587E-3,1.1895564E-2,4.6692587E-2,1.477425E6,1.834412E6,7.33E2,4.5709072E1,1.08504E5,3.958068E2,5.954224E0,1.0462782E-2,5.024605E0,2.984735E9,4.760537E8,1E0,9.27E2,-1.9173676E-2,1.0918E4,1.3446785E7,4.7692296E-1,1.8834772E7,1.04E2,2.0455818E1,3.3E1,-1.6027793E-2,-5.2560023E-3,4.1106674E-3,4.5723195E6,7.2896E5,9.62689E6,1E0,-2.3708763E-2,4.524781E0,3.3487454E11,2.8839298E7,1.2179555E-1,1.2E1,3.313261E-3,6.53143E6,2.23329E5,3.30399E6,1.2570965E6,-1.8553747E-2,2.241E2,1.1400756E6,3.1756763E-3,-4.0379413E-3,7.2530017E0,2.6652452E-2,6.1061732E7,1.137816E-2,-5.1354608E-3,1.1315813E-2,2.504895E-3,1.4563726E-1,3.641203E-4,2.695E3,3.0697957E-3,-6.650437E-3,2.419065E-2,1.2516333E7,3.3362508E-2,1.7181918E-2,-2.8635068E-2,-1.9852487E-2,5.836513E3,3.2794893E-1,7.074777E6,7.2456576E8,-3.25112E-3,1.5E1,6.4153387E-6,1.4383178E1,-2.449261E-2,-3.2018484E-3,7.5676725E-3,-1.9510169E-2,5.0083565E6,2.4862127E-2,1.3E1,1.648091E-2,9.024187E6,3.2485715E2,2.970914E1,2.8884522E8,1.3794063E1,-3.5676777E-3,-2.5844066E-3,7.6200864E-3,1.822948E6,1.1E1,6.114765E-2,1.6777211E8,6.445E3,2.028994E8,4.43575E7,2.3907686E7,7.2132964E0,-8.740945E-3,2.1196339E1,1.4007094E-2,3.9679848E-2,5.829676E-2,-4.0341064E-2,-1.8840194E-2,1.2592834E-1,-6.067929E-2,6.3E0,-2.6183644E-2,-4.7901566E-3,2.587167E-3,-5.757561E-3,5.28587E5,-6.1045364E-2,-2.6887396E-2,1.4540612E7,2.2100918E1,-9.978941E-3,-1.8868377E-2,-2.1841172E-2,-1.1639464E-2,-2.1382231E-2,-3.4931056E-2,-1.4448694E-2,-1.8280568E-3,-2.6409207E-2,-1.5675817E-2,1.4015416E6,-1.3355507E-3,2.51343E5,-1.3944961E-2,-7.660774E-3,9.608472E-4,3.5E1,1.3414634E0,1.1317E4,-1.0932948E-2,-2.0699076E-2,-1.1851555E-2,3.5188188E5,-1.3273781E-2,3.8773947E0,-1.2725774E-3,4.8896177E-3,4.524781E0,1.2631686E-2,5.353951E-3,-3.866705E-3,9.232009E2,7.1308413E0,-1.0482622E-2,1.02E3,-3.88055E-2,3.4306667E1,1.5963264E9,-1.4965116E-2,-3.669372E-2,6.4547E4,1.5288235E2,2.1989189E2,4.745907E7,1.315782E-5,2.2174756E4,3.932624E-2,2.5627704E-2,9.403226E0,1.2375429E8,-2.0756967E-2,3.44E2,3.815348E-2,9.985453E-3,2.59E2,1.5166431E5,-7.317911E-3,1.9589581E6,4.230769E0,6.763314E7,3.9888642E0,2.3871907E2,4.6374165E-3,2E0,9.7879935E-3,1.1349584E2,1.0044316E-2,2.3311735E-3,1.9283049E-1,6.8023116E-3,8.079531E4,2.2153166E7,5.612611E-3,1.9356398E-2,3.3849873E-2,1.8061467E-2,-3.953039E-2,-3.032876E-2,-1.8602887E-2,-5.4478208E-3,-1.8466355E-2,-2.9392378E-2,5.6307483E-3,-7.321454E-3,1.2741362E-2,-1.9793995E-3,-1.5928622E-2,-7.946667E-3,-1.7246049E-2,-3.465468E-2,-1.7790752E-2,-9.7657E-3,-4.1905814E-3,-1.095137E-2,4.311717E-3,-4.0105185E-3,4.2837085E-3,-3.7662832E-3,-7.3988843E-3,-1.5925838E-2,1.8658074E-2,9.354607E-3,-7.854806E-3,-1.681364E-2,-1.9550666E-2,-1.1117803E-2,-1.0168702E-2,2.5743993E-3,1.221328E-3,-6.8080234E-3,6.911668E-3,1.905981E-2,-3.489801E-5,-1.721876E-2,-1.81717E-2,-8.9380285E-3,8.0037396E-4,1.1669384E-2,-1.706488E-2,4.0128096E-3,-2.81046E-3,1.1836067E-2,2.298517E-2,8.000278E-3,7.414992E-3,-9.147203E-3,1.5996065E-2,6.7952154E-3,-8.001097E-3,4.4255126E-3,-6.911886E-4,9.926068E-3,1.0036481E-2,2.0813528E-2,-2.1212488E-2,-7.600358E-3,4.673397E-3,-4.21129E-3,7.4591683E-3,1.6943214E-2,2.4457501E-2,8.185784E-3,-1.4417393E-3,1.2085327E-2,3.690999E-2,1.9921552E-2,2.6355503E-2,1.3534089E-2,1.8097505E-2,2.9003348E-2,1.9281657E-2,3.8977068E-2,8.240536E-3,3.0605951E-2],"split_indices":[20,102,33,55,51,28,2,27,45,52,52,2,28,28,17,33,5,2,10,42,42,41,57,4,18,52,100,45,2,42,41,1,32,2,7,56,0,52,0,33,9,0,52,53,53,4,4,3,9,1,6,27,0,9,3,52,18,53,54,0,0,58,56,0,57,0,7,53,29,27,0,31,0,32,5,2,52,11,53,45,48,28,33,28,54,56,35,0,0,0,53,5,0,0,0,9,5,2,56,7,52,54,0,53,31,7,64,2,0,9,45,35,43,8,54,3,0,0,0,28,9,45,8,0,54,31,45,27,18,0,29,12,9,33,0,4,33,0,0,53,57,45,0,0,0,0,38,0,2,0,0,0,32,0,0,0,0,28,38,47,7,0,3,37,53,0,0,0,0,28,0,3,0,5,52,58,12,56,0,0,0,47,8,0,7,29,7,1,45,54,0,54,0,0,0,0,0,38,0,58,0,0,0,0,29,0,0,45,56,0,0,0,0,0,0,0,0,0,0,32,0,1,0,0,0,3,53,9,0,0,0,48,0,54,0,0,54,0,0,0,33,54,0,0,0,53,7,0,0,1,52,52,7,38,47,0,0,58,7,0,0,0,0,8,28,0,32,56,7,46,52,0,6,0,56,0,0,38,0,28,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.573E3,7.97E2,1.776E3,4.38E2,3.59E2,1.063E3,7.13E2,3.26E2,1.12E2,2.47E2,1.12E2,5.57E2,5.06E2,4.99E2,2.14E2,4.2E1,2.84E2,5.3E1,5.9E1,1.63E2,8.4E1,6.6E1,4.6E1,1.21E2,4.36E2,3.54E2,1.52E2,2.16E2,2.83E2,3.2E1,1.82E2,2.8E1,1.4E1,2.41E2,4.3E1,4.9E1,4E0,5.3E1,6E0,9.6E1,6.7E1,5E0,7.9E1,3E1,3.6E1,3.7E1,9E0,6.4E1,5.7E1,1.5E1,4.21E2,3.5E2,4E0,1.14E2,3.8E1,1.66E2,5E1,1.37E2,1.46E2,2.8E1,4E0,1.19E2,6.3E1,6E0,2.2E1,4E0,1E1,2E2,4.1E1,3.9E1,4E0,4.4E1,5E0,3.6E1,1.7E1,6.2E1,3.4E1,5.7E1,1E1,5.1E1,2.8E1,9E0,2.1E1,2.5E1,1.1E1,1.6E1,2.1E1,4E0,5E0,5E0,5.9E1,5.1E1,6E0,5E0,1E1,3.91E2,3E1,3.28E2,2.2E1,1.3E1,1.01E2,3.3E1,5E0,7.1E1,9.5E1,4E1,1E1,1.31E2,6E0,1.09E2,3.7E1,1.05E2,1.4E1,2.6E1,3.7E1,1.5E1,7E0,5E0,5E0,1.83E2,1.7E1,1E1,3.1E1,4E0,3.5E1,2.6E1,1.8E1,1.5E1,2.1E1,4E0,1.3E1,2.4E1,3.8E1,2.2E1,1.2E1,9E0,4.8E1,5E0,5E0,3.5E1,1.6E1,2.1E1,7E0,4E0,5E0,4E0,1.7E1,5E0,2E1,6E0,5E0,8E0,8E0,1.6E1,5E0,1.5E1,4.4E1,1.6E1,3.5E1,2E2,1.91E2,6E0,2.4E1,1.22E2,2.06E2,1.6E1,6E0,7E0,6E0,8.4E1,1.7E1,2.9E1,4E0,5.1E1,2E1,1.4E1,8.1E1,3.3E1,7E0,6E0,4E0,3E1,1.01E2,7E0,1.02E2,2.8E1,9E0,5.7E1,4.8E1,1E1,4E0,2E1,6E0,2.5E1,1.2E1,8E0,7E0,1.75E2,8E0,1E1,7E0,6E0,4E0,5E0,2.6E1,3.1E1,4E0,1.8E1,8E0,5E0,1.3E1,8E0,7E0,1E1,1.1E1,9E0,4E0,1.7E1,7E0,3.4E1,4E0,1.8E1,4E0,5E0,4E0,3.8E1,1E1,2.8E1,7E0,9E0,7E0,1.4E1,7E0,1.3E1,4E0,5E0,1.5E1,4E0,4E0,5E0,1.1E1,2.7E1,8E0,1.96E2,4E0,1.49E2,4.2E1,5E0,1.9E1,1.06E2,1.6E1,1.89E2,1.7E1,6.4E1,2E1,1.9E1,1E1,1.9E1,3.2E1,7E0,1.3E1,1E1,4E0,2.2E1,5.9E1,5E0,2.8E1,1.9E1,1.1E1,7E1,3.1E1,4E0,9.8E1,6E0,2.2E1,5E0,4E0,5.2E1,5E0,3.1E1,1.7E1,4E0,6E0,1.5E1,5E0,1.35E2,4E1,6E0,4E0,9E0,1.7E1,4E0,1.4E1,4E0,4E0,2.4E1,1E1,4E0,1.4E1,2.7E1,1.1E1,4E0,6E0,7E0,2.1E1,1E1,4E0,5E0,8E0,1E1,5E0,7E0,4E0,2.1E1,6E0,1.44E2,5.2E1,1.05E2,4.4E1,2.9E1,1.3E1,1.02E2,4E0,6E0,1E1,2.3E1,1.66E2,5E0,1.2E1,4E0,6E1,1.4E1,6E0,8E0,1.1E1,1E1,2.2E1,9E0,4E0,1.8E1,4E0,3.8E1,2.1E1,2.3E1,5E0,1.5E1,4E0,6E0,5E0,5.8E1,1.2E1,6E0,2.5E1,8.2E1,1.6E1,1.8E1,4E0,3.9E1,1.3E1,5E0,2.6E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"363","size_leaf_vector":"1"}},{"base_weights":[1.2212639E-3,-3.4708545E-1,1.6477159E-1,-5.3325963E-1,-1.1938639E-1,3.871841E-2,4.2959186E-1,-5.568181E-1,6.6669755E-2,-2.408135E-1,2.2752738E-1,-1.1937688E-1,1.6680057E-1,3.2497853E-1,6.61215E-1,-6.7002887E-1,-3.2822305E-1,-2.9218316E-1,-3.1034661E-2,1.11762635E-1,5.2780646E-1,-2.3168764E-1,2.2568364E-2,1.464352E-1,6.309857E-2,1.5312761E-1,4.8273847E-1,4.8242295E-1,8.104626E-1,-6.2111515E-1,-1.3286561E0,-1.6267338E-1,-5.147984E-1,-3.1707847E-1,2.9605722E-2,-2.031718E-1,7.024074E-2,-5.2900888E-2,1.7179108E-1,6.1316204E-1,1.6058577E-2,-3.9244226E-1,-1.6015851E-1,5.1472303E-2,-2.1336906E-2,1.7745593E-1,-3.61097E-1,2.129979E-1,-3.1646672E-1,-2.1905959E-1,5.2345043E-1,5.614654E-1,2.518138E-1,8.431283E-1,2.6909376E-3,-3.9030713E-1,-6.96573E-1,-7.335784E-2,-3.7900437E-2,-2.107283E-1,2.4041142E-2,-6.1139417E-1,-2.182523E-1,-3.0414915E-2,-2.9289567E-1,-1.0038132E-2,1.7359757E-1,-1.3753965E-1,-1.927148E-2,1.4027165E-1,-6.445022E-2,1.5436335E-2,-1.3041291E-2,4.992909E-2,2.5666317E-1,3.617244E-2,2.1083726E-2,-3.4140962E-1,-5.3930026E-1,1.3294095E-1,-1.969191E-1,-7.272746E-2,2.2458641E-1,2.8313544E-2,2.8384838E-1,-1.8997537E-1,-5.0560766E-1,1.270825E-1,2.9543754E-1,-3.3885043E-3,-3.679551E-1,-3.390156E-2,1.0294463E-2,4.6351135E-1,8.7923783E-1,6.259408E-1,3.00302E-1,2.197247E-3,3.399952E-1,8.743555E-1,6.5531125E-3,-5.0779283E-1,-1.4912227E-1,-7.7678555E-1,-5.073637E-1,-2.9629394E-1,-8.114242E-2,-4.885956E-1,-8.648287E-1,-1.8060323E-3,-3.2146966E-1,-2.3603734E-1,-3.9959005E-1,1.0840499E-3,1.2856395E-2,-3.4219414E-4,-8.855369E-3,4.9019758E-2,2.2743857E-1,3.2656577E-3,-6.465889E-3,7.4787766E-2,-6.5003824E-3,1.606669E-1,-8.5481875E-2,3.2523152E-1,8.01465E-2,-4.24388E-1,-1.6122745E-1,-2.9619062E-2,-1.4885128E-2,-1.3539083E-1,3.3427727E-1,-2.920014E-1,-7.426175E-2,-8.869312E-2,2.1701984E-2,3.6259514E-1,1.1747246E-1,-2.8864613E-2,1.9853276E-1,2.2278737E-1,4.6727252E-1,-3.477195E-1,3.5170398E-3,-2.7994292E-2,-1.619293E-2,9.962143E-2,1.8934501E-2,1.00622766E-1,3.4552643E-1,-2.1807186E-2,-9.893225E-3,4.8997754E-1,1.3549426E-1,3.3349607E-2,5.276428E-2,1.4222282E-2,6.679217E-1,7.939166E-3,2.0936847E-2,1.9668322E-2,7.1589933E-3,9.068331E-1,2.1999564E-2,-7.216126E-3,-5.6630987E-1,-1.2820451E-2,-1.9487604E-2,-8.257523E-1,-5.0479734E-1,-5.313118E-1,-5.443066E-3,-3.591605E-1,-1.0908184E-1,-2.1654712E-1,1.0927765E-2,-6.173766E-1,-2.4997292E-1,-2.9390065E-2,-5.336139E-2,-2.447028E-2,-8.800224E-3,-2.5912783E-1,1.0688987E-2,-5.375644E-1,-2.837783E-1,-1.650306E-3,5.2179294E-3,4.7606053E-3,1.4071142E-2,3.0135972E-5,5.586508E-3,1.4124217E-2,7.84906E-4,6.7949906E-5,-7.898564E-3,4.495441E-3,3.7251386E-1,8.804343E-3,-2.5559044E-3,-3.6530802E-1,-2.774893E-2,5.7531935E-3,-2.222581E-1,-1.3141332E-2,7.387796E-4,3.0932667E-2,1.5845513E-1,-1.0007871E-1,-3.5715312E-1,-1.8047403E-1,4.4428315E-2,-7.332749E-2,-2.354124E-2,1.1001694E-2,2.0165851E-2,3.4629796E-2,1.8490627E-1,-1.8086703E-1,4.0545925E-2,2.4360244E-1,-4.4864073E-2,1.359237E-1,3.2847282E-1,2.587694E-1,5.31874E-1,-2.6415026E-2,-9.552184E-3,-4.5571618E-2,1.2413927E-1,-7.2472505E-3,2.2516342E-1,4.201901E-1,2.165857E-1,5.250548E-1,2.094176E-1,-7.907773E-3,2.6882246E-1,2.1503227E-2,7.1116316E-1,4.725725E-2,6.750311E-1,-2.3646154E-2,-4.034844E-2,3.1838394E-3,-8.182698E-3,-4.3432582E-2,-2.931219E-2,-1.7065782E-2,-3.4053575E-2,-2.2201614E-2,-3.135965E-2,-1.3585307E-2,-2.7053319E-2,8.849752E-4,-1.2989288E-2,-8.713381E-4,-1.692922E-2,1.205342E-2,-3.0841636E-3,-3.3949886E-2,-1.227206E-2,-6.9953995E-3,-1.6423939E-2,-1.8632375E-2,-1.0326139E-2,4.3087825E-3,-5.6122285E-3,-2.9877165E-2,-1.5206385E-2,-2.1149373E-2,-1.0556551E-2,2.0380944E-2,9.941151E-3,-2.1058325E-2,-1.4278218E-2,-1.4209627E-2,-1.4942522E-3,3.8668131E-3,9.973448E-3,-1.0225634E-2,2.107969E-2,-2.3997635E-2,-1.1708742E-2,-2.9236833E-3,-1.353218E-2,4.0648314E-3,-1.1507307E-2,5.373258E-3,-5.5071893E-3,-2.3397713E-3,4.05786E-3,1.3278976E-2,3.8794458E-3,-6.497492E-3,-3.618393E-2,-4.3375124E-3,4.779943E-3,1.0295665E-2,2.3567501E-2,8.340824E-3,-1.6608912E-2,7.617298E-3,-1.4430964E-2,1.8452143E-2,3.6040698E-3,9.683365E-3,1.6181929E-2,2.2724103E-2,3.4491654E-2,1.0855148E-3,-7.201772E-3,6.9270185E-3,-4.1625667E-3,1.6038226E-2,2.9921394E-3,1.761085E-2,2.7191184E-2,-4.969068E-3,1.5570439E-2,1.960051E-2,3.026708E-2,1.7063035E-2,-3.2864816E-3,6.112689E-3,1.729303E-2,3.74221E-2,2.0488443E-2,1.991875E-2,3.8642336E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,-1,81,83,85,87,89,91,93,95,97,99,-1,101,103,-1,-1,105,-1,107,109,-1,111,-1,113,115,-1,117,119,121,-1,123,125,-1,-1,127,129,131,133,135,137,139,141,143,145,147,149,-1,151,-1,-1,153,155,157,159,-1,161,163,-1,165,167,169,171,173,175,177,179,-1,181,183,185,-1,-1,-1,-1,187,189,-1,-1,191,-1,193,195,197,199,201,203,-1,-1,205,207,209,211,213,-1,215,217,219,221,223,225,227,-1,-1,-1,229,-1,231,233,-1,-1,235,237,-1,-1,-1,239,-1,-1,-1,-1,241,-1,-1,243,245,-1,247,249,251,-1,253,255,257,259,261,263,-1,-1,-1,-1,265,267,269,271,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,273,-1,-1,275,-1,-1,277,-1,-1,-1,279,281,283,285,287,289,-1,-1,-1,291,293,295,297,299,301,303,305,307,309,-1,-1,311,313,-1,315,317,319,321,323,-1,325,-1,327,-1,329,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4850974E2,3.524862E1,5.9202045E1,2.0893387E1,1.5893694E1,2.4379684E1,1.3645172E1,1.15034485E1,0E0,2.997509E0,3.34766E0,8.597956E0,1.4658352E1,1.0662109E1,4.3321304E0,8.903946E0,4.6111546E0,1.8038597E0,9.905334E-1,7.207716E-1,2.78656E-1,3.3982372E0,1.0594715E1,1.0360664E1,0E0,5.437322E0,6.000412E0,1.3715324E0,2.3025322E0,4.656929E0,4.7686005E-1,2.6383533E0,1.900362E0,1.2885952E0,5.991128E-1,1.9850856E-1,3.4861162E-1,2.8844696E-1,5.353472E-1,7.576895E-2,0E0,5.0537586E-1,2.288825E0,0E0,2.9231048E0,9.782288E0,8.436451E-1,1.1665378E0,2.5670528E-1,2.4833229E0,3.7509575E0,8.391857E-1,4.0658343E-1,1.9085388E0,0E0,1.9644527E0,2.82341E0,0E0,0E0,8.297417E-1,0E0,1.2268829E0,3.4551722E-1,0E0,1.0825768E0,0E0,1.342771E-1,8.695841E-2,0E0,1.7975059E-1,1.180664E-1,1.4792876E-1,0E0,3.5862172E-1,3.4914088E-1,0E0,0E0,9.931145E-1,1.0741186E-1,1.3418565E0,2.1592112E0,1.5847615E0,5.487113E-1,2.5167983E0,3.9279213E0,8.048194E-1,1.6856194E-2,5.5013883E-1,8.0442476E-1,0E0,1.2308407E-1,0E0,0E0,1.4007607E0,3.0596924E-1,4.9764442E-1,1.6845095E-1,0E0,1.1241567E-1,8.2722473E-1,0E0,9.388647E-1,8.158429E-1,1.5409622E0,5.718727E-1,5.0221395E-1,4.0130192E-1,1.0134144E0,2.7084827E-1,0E0,2.0722628E-1,7.376871E-1,9.0206814E-1,0E0,0E0,0E0,0E0,6.254898E-2,5.90837E-2,0E0,0E0,3.2885626E-2,0E0,2.2547463E-1,6.956186E-2,2.1120858E-1,1.3513589E-1,1.6851044E-1,4.303971E-1,0E0,0E0,2.1469444E-1,6.0291624E-1,1.2771873E0,1.0543447E0,1.0584856E0,0E0,1.0379314E-2,1.2767023E-1,2.054144E0,7.239673E-1,2.4523735E0,1.061163E0,2.3883486E-1,0E0,0E0,0E0,2.8332323E-1,0E0,6.139672E-1,5.6364155E-1,0E0,0E0,1.4252853E0,5.908661E-1,0E0,0E0,0E0,1.2541008E-1,0E0,0E0,0E0,0E0,8.3724976E-2,0E0,0E0,5.382471E-1,2.0683773E-1,0E0,7.688446E-1,4.8852015E-1,2.404747E-1,0E0,3.936305E-1,2.5329682E-1,3.1055927E-1,3.495278E-1,3.86981E-1,8.384454E-2,0E0,0E0,0E0,0E0,6.2729263E-1,1.2119646E-1,2.2640896E-1,2.949078E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.936697E-2,0E0,0E0,4.0348053E-2,0E0,0E0,2.3049057E-1,0E0,0E0,0E0,1.7909855E-2,1.6146021E0,1.0669584E0,4.6899927E-1,4.2448133E-1,1.2128208E0,0E0,0E0,0E0,4.933526E-2,9.210861E-2,1.5904042E0,9.488953E-1,3.7384057E-1,7.253618E-1,1.10379E0,1.2498474E0,3.4499764E-2,5.0429153E-1,0E0,0E0,8.2723945E-2,2.1299279E-1,0E0,1.8665195E-1,1.6097021E-1,8.418783E-1,1.240963E0,7.130102E-1,0E0,7.084972E-2,0E0,7.967949E-2,0E0,9.9372864E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,59,59,61,61,62,62,64,64,66,66,67,67,69,69,70,70,71,71,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,90,90,93,93,94,94,95,95,96,96,98,98,99,99,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,110,110,111,111,112,112,117,117,118,118,121,121,123,123,124,124,125,125,126,126,127,127,128,128,131,131,132,132,133,133,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,143,143,147,147,149,149,150,150,153,153,154,154,158,158,163,163,166,166,167,167,169,169,170,170,171,171,173,173,174,174,175,175,176,176,177,177,178,178,183,183,184,184,185,185,186,186,198,198,201,201,204,204,208,208,209,209,210,210,211,211,212,212,213,213,217,217,218,218,219,219,220,220,221,221,222,222,223,223,224,224,225,225,226,226,229,229,230,230,232,232,233,233,234,234,235,235,236,236,238,238,240,240,242,242],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,-1,82,84,86,88,90,92,94,96,98,100,-1,102,104,-1,-1,106,-1,108,110,-1,112,-1,114,116,-1,118,120,122,-1,124,126,-1,-1,128,130,132,134,136,138,140,142,144,146,148,150,-1,152,-1,-1,154,156,158,160,-1,162,164,-1,166,168,170,172,174,176,178,180,-1,182,184,186,-1,-1,-1,-1,188,190,-1,-1,192,-1,194,196,198,200,202,204,-1,-1,206,208,210,212,214,-1,216,218,220,222,224,226,228,-1,-1,-1,230,-1,232,234,-1,-1,236,238,-1,-1,-1,240,-1,-1,-1,-1,242,-1,-1,244,246,-1,248,250,252,-1,254,256,258,260,262,264,-1,-1,-1,-1,266,268,270,272,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,274,-1,-1,276,-1,-1,278,-1,-1,-1,280,282,284,286,288,290,-1,-1,-1,292,294,296,298,300,302,304,306,308,310,-1,-1,312,314,-1,316,318,320,322,324,-1,326,-1,328,-1,330,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,1.9013363E4,5.1180666E8,2.1592189E5,1.2974394E6,5.121E3,6.6669755E-2,1.2690893E0,1.4101E4,2.857E3,4.9807886E3,1.2679E4,5.964E3,9.6910744E7,6.747114E7,9.026015E2,1.752512E2,2.4832625E6,1.5609541E0,9.6E1,2.3E1,3.0820766E-1,6.309857E-2,4.987639E7,2.7165637E4,1E0,7.6247287E-1,2.0304577E-1,5.4803147E0,1.5137865E2,3.970405E3,3.0192846E-4,7.549744E7,7.44E2,6.92E2,5.884028E0,3.3159972E5,1.7366706E8,1.6058577E-2,2.1506184E7,9.2058825E-1,5.1472303E-2,6.932264E8,8.46E2,2.8839298E7,1.8108038E5,2.37E3,3.3743455E0,7.096131E0,2E0,8.319452E2,6.292039E0,2.6909376E-3,2.695E3,2.040457E0,-7.335784E-2,-3.7900437E-2,2.6036014E5,2.4041142E-2,4.29E4,1.272E3,-3.0414915E-2,3.3709784E5,-1.0038132E-2,6.4506575E5,1.9566955E5,-1.927148E-2,2.0503453E8,1.3E1,1.0975045E7,-1.3041291E-2,8.315035E6,3E1,3.617244E-2,2.1083726E-2,5.222222E1,1.5824156E5,4.97E2,1.273801E6,1.3137E4,1.3632204E-3,2.3314082E6,2.2214102E6,2.342E3,3.054842E-4,1.3359244E9,3.131E3,-3.3885043E-3,5.911314E1,-3.390156E-2,1.0294463E-2,7.218466E7,1.6828056E1,3.90106E0,1.65177E10,2.197247E-3,8.640299E8,1E0,6.5531125E-3,1.7515824E8,2.7448502E8,1.1164689E0,4.759E3,3.0795444E3,2.562723E-1,2.867292E3,3.2216358E-1,-1.8060323E-3,8.755996E-1,3.730007E6,1.16544716E-1,1.0840499E-3,1.2856395E-2,-3.4219414E-4,-8.855369E-3,2.3990374E8,2.539E3,3.2656577E-3,-6.465889E-3,4.0895187E8,-6.5003824E-3,1.0119178E10,5.8805E4,4.123E3,2.4E1,9.82542E5,2E0,-2.9619062E-2,-1.4885128E-2,4.7E2,2.2E1,3.44E2,1.5288235E2,8.229907E7,2.1701984E-2,5.88E3,4.79E2,1E0,1.8E1,1.645614E6,4.2945E4,1.71E2,3.5170398E-3,-2.7994292E-2,-1.619293E-2,6.6473596E2,1.8934501E-2,9.112E3,9.457831E0,-2.1807186E-2,-9.893225E-3,5.21929E5,5.528107E3,3.3349607E-2,5.276428E-2,1.4222282E-2,4.6253732E2,7.939166E-3,2.0936847E-2,1.9668322E-2,7.1589933E-3,6.2E1,2.1999564E-2,-7.216126E-3,8.7217E4,8.253647E6,-1.9487604E-2,1E0,3.73802E5,1.74123E5,-5.443066E-3,5.097268E7,1.3498145E5,6.439778E5,1.8989474E0,5.0740743E0,1.141E4,-2.9390065E-2,-5.336139E-2,-2.447028E-2,-8.800224E-3,1.287E3,2.0783027E5,4.653159E9,1.3651616E5,-1.650306E-3,5.2179294E-3,4.7606053E-3,1.4071142E-2,3.0135972E-5,5.586508E-3,1.4124217E-2,7.84906E-4,6.7949906E-5,-7.898564E-3,4.495441E-3,1.25E2,8.804343E-3,-2.5559044E-3,1.0853132E4,-2.774893E-2,5.7531935E-3,8.314423E6,-1.3141332E-2,7.387796E-4,3.0932667E-2,7.108182E4,2.8080197E2,6.3039363E4,6.953307E4,4.3E1,5.1547173E1,-2.354124E-2,1.1001694E-2,2.0165851E-2,4.004366E1,1.5548611E7,3.5612745E0,1.5561539E2,6.789622E7,6.23E2,2.7183437E1,4.06E2,2.001177E6,5.351875E2,-2.6415026E-2,-9.552184E-3,1.6627079E-2,1.1392E4,-7.2472505E-3,1.2694E4,2.0605305E5,4.492E3,2.6312E4,2.6025698E7,-7.907773E-3,8.791777E3,2.1503227E-2,2.1708366E7,4.725725E-2,1.9125667E3,-2.3646154E-2,-4.034844E-2,3.1838394E-3,-8.182698E-3,-4.3432582E-2,-2.931219E-2,-1.7065782E-2,-3.4053575E-2,-2.2201614E-2,-3.135965E-2,-1.3585307E-2,-2.7053319E-2,8.849752E-4,-1.2989288E-2,-8.713381E-4,-1.692922E-2,1.205342E-2,-3.0841636E-3,-3.3949886E-2,-1.227206E-2,-6.9953995E-3,-1.6423939E-2,-1.8632375E-2,-1.0326139E-2,4.3087825E-3,-5.6122285E-3,-2.9877165E-2,-1.5206385E-2,-2.1149373E-2,-1.0556551E-2,2.0380944E-2,9.941151E-3,-2.1058325E-2,-1.4278218E-2,-1.4209627E-2,-1.4942522E-3,3.8668131E-3,9.973448E-3,-1.0225634E-2,2.107969E-2,-2.3997635E-2,-1.1708742E-2,-2.9236833E-3,-1.353218E-2,4.0648314E-3,-1.1507307E-2,5.373258E-3,-5.5071893E-3,-2.3397713E-3,4.05786E-3,1.3278976E-2,3.8794458E-3,-6.497492E-3,-3.618393E-2,-4.3375124E-3,4.779943E-3,1.0295665E-2,2.3567501E-2,8.340824E-3,-1.6608912E-2,7.617298E-3,-1.4430964E-2,1.8452143E-2,3.6040698E-3,9.683365E-3,1.6181929E-2,2.2724103E-2,3.4491654E-2,1.0855148E-3,-7.201772E-3,6.9270185E-3,-4.1625667E-3,1.6038226E-2,2.9921394E-3,1.761085E-2,2.7191184E-2,-4.969068E-3,1.5570439E-2,1.960051E-2,3.026708E-2,1.7063035E-2,-3.2864816E-3,6.112689E-3,1.729303E-2,3.74221E-2,2.0488443E-2,1.991875E-2,3.8642336E-2],"split_indices":[20,102,12,52,12,28,28,2,0,42,2,2,4,2,2,45,45,52,52,32,57,29,0,27,0,45,33,6,57,27,56,56,52,41,7,0,0,53,28,32,0,45,53,0,7,2,45,28,10,54,35,17,52,38,0,2,53,0,0,33,0,10,0,0,33,0,28,47,0,5,3,45,0,32,3,0,0,4,33,2,9,2,39,28,28,2,42,7,2,0,58,0,0,45,35,56,31,0,5,6,0,31,7,39,2,4,39,4,42,0,42,9,39,0,0,0,0,7,2,0,0,7,0,19,9,2,8,31,10,0,0,1,0,29,52,45,0,2,8,102,3,9,1,0,0,0,0,52,0,10,54,0,0,10,52,0,0,0,4,0,0,0,0,8,0,0,9,1,0,105,33,29,0,45,33,51,54,54,9,0,0,0,0,2,33,5,28,0,0,0,0,0,0,0,0,0,0,0,8,0,0,28,0,0,5,0,0,0,28,4,28,47,3,48,0,0,0,53,45,53,52,7,2,53,0,9,52,0,0,57,2,0,29,33,2,2,43,0,52,0,47,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.605E3,8.32E2,1.773E3,4.57E2,3.75E2,1.202E3,5.71E2,4.52E2,5E0,2.78E2,9.7E1,5.38E2,6.64E2,3.95E2,1.76E2,3.01E2,1.51E2,2.23E2,5.5E1,7.1E1,2.6E1,3E2,2.38E2,6.53E2,1.1E1,1.9E2,2.05E2,8.2E1,9.4E1,2.82E2,1.9E1,8.1E1,7E1,2.07E2,1.6E1,2E1,3.5E1,1.9E1,5.2E1,1.7E1,9E0,9.1E1,2.09E2,9E0,2.29E2,6.16E2,3.7E1,1.69E2,2.1E1,1.1E1,1.94E2,6E1,2.2E1,9E1,4E0,7.1E1,2.11E2,1.4E1,5E0,7.6E1,5E0,5.2E1,1.8E1,1.4E1,1.93E2,6E0,1E1,1.6E1,4E0,2.3E1,1.2E1,1.5E1,4E0,2.2E1,3E1,9E0,8E0,7E1,2.1E1,2.3E1,1.86E2,1.9E2,3.9E1,2.57E2,3.59E2,1.8E1,1.9E1,8.4E1,8.5E1,4E0,1.7E1,5E0,6E0,1.68E2,2.6E1,4.7E1,1.3E1,7E0,1.5E1,8.6E1,4E0,4.7E1,2.4E1,1.46E2,6.5E1,4.5E1,3.1E1,3.7E1,1.5E1,7E0,1.1E1,1.28E2,6.5E1,4E0,6E0,4E0,1.2E1,1.2E1,1.1E1,4E0,8E0,1.1E1,4E0,1.2E1,1E1,2.1E1,9E0,4.7E1,2.3E1,1.6E1,5E0,1E1,1.3E1,1.04E2,8.2E1,1.85E2,5E0,1.6E1,2.3E1,1.93E2,6.4E1,2.71E2,8.8E1,1.1E1,7E0,1.3E1,6E0,7.7E1,7E0,1.8E1,6.7E1,1.1E1,6E0,1.55E2,1.3E1,1.4E1,1.2E1,6E0,4.1E1,7E0,6E0,1.1E1,4E0,7.9E1,7E0,7E0,4E1,1.6E1,8E0,1.22E2,2.4E1,6.1E1,4E0,3.3E1,1.2E1,1.2E1,1.9E1,2.3E1,1.4E1,8E0,7E0,4E0,7E0,1.17E2,1.1E1,2.8E1,3.7E1,5E0,7E0,4E0,7E0,4E0,7E0,6E0,6E0,5E0,5E0,4E0,1.7E1,5E0,4E0,3.5E1,1.2E1,4E0,1.9E1,5E0,5E0,4E0,9E0,2.7E1,7.7E1,4.3E1,3.9E1,1.79E2,6E0,5E0,1.1E1,1.1E1,1.2E1,6E1,1.33E2,5.4E1,1E1,1.5E2,1.21E2,2.2E1,6.6E1,4E0,7E0,1.1E1,6.6E1,6E0,1.2E1,4.1E1,2.6E1,1.37E2,1.8E1,4E0,9E0,8E0,3.3E1,6.5E1,1.4E1,3.1E1,9E0,1.1E1,5E0,1.01E2,2.1E1,1.4E1,1E1,3.5E1,2.6E1,2.4E1,9E0,7E0,5E0,5E0,7E0,4E0,1.5E1,1.9E1,4E0,7E0,7E0,3.5E1,8.2E1,7E0,4E0,2.1E1,7E0,1.1E1,2.6E1,1.3E1,4E0,1.8E1,1.7E1,1.4E1,5E0,4E0,5E0,2.3E1,4E0,3.7E1,4E1,1.9E1,2.4E1,3.5E1,4E0,3E1,1.49E2,4E0,7E0,6E0,6E0,5.6E1,4E0,4E1,9.3E1,4.8E1,6E0,6E0,4E0,1.45E2,5E0,1.04E2,1.7E1,1.3E1,9E0,4.7E1,1.9E1,7E0,4E0,6.2E1,4E0,7E0,5E0,2.9E1,1.2E1,6E0,2E1,5.4E1,8.3E1,1.2E1,6E0,4E0,5E0,2.8E1,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"331","size_leaf_vector":"1"}},{"base_weights":[7.845864E-3,-3.2550657E-1,1.6121751E-1,-5.345998E-1,-1.4258283E-1,3.29729E-3,3.7337914E-1,-6.3007605E-1,-3.0427435E-1,-2.0584138E-1,4.703369E-1,-1.6817978E-1,1.13447934E-1,1.0236822E-1,4.7575006E-1,-6.567799E-2,-6.038625E-1,-3.8502887E-1,-1.4117531E-1,-8.838336E-2,-4.742795E-1,6.3240066E-2,2.6937214E-1,-1.9481276E-1,3.7192953E-1,-3.0031186E-1,1.4803116E-1,1.3497678E-2,4.1505367E-1,4.3335477E-1,8.9778084E-1,-6.270213E-1,-2.1294923E-1,-4.0226874E-1,-3.158742E-3,5.618961E-3,-1.9069748E-1,-1.7371655E-1,2.0288147E-1,-5.782296E-1,-1.8270718E-1,-6.3830145E-2,4.5402023E-1,-1.6260055E-1,-5.06767E-1,5.1897928E-2,1.2489373E-1,-3.5007903E-1,9.8784845E-3,1.283869E-1,9.2042154E-1,8.478084E-2,-3.0535036E-1,3.256405E-2,3.3987334E-1,3.6015344E-1,6.2414837E-1,7.5635105E-1,5.707232E-2,-5.515189E-1,-7.777349E-1,3.5617957E-3,-3.9061716E-1,-3.3353576E-1,-5.6086105E-1,-2.7617377E-1,-1.0412423E-1,-3.0474278E-1,-7.978756E-2,5.451344E-2,3.8639462E-1,-6.806305E-1,-3.8693622E-1,-2.3821035E-1,8.781778E-3,-1.0969583E-2,5.04645E-3,1.2359964E-2,2.5785599E-2,-3.174582E-1,-1.0334127E-1,-3.8157028E-1,-4.4051096E-2,-2.528701E-3,2.1770525E-1,-2.1138358E-1,-5.059444E-1,6.588412E-2,3.3573425E-1,5.5007413E-2,1.875829E-2,3.525306E-2,2.2941393E-1,-4.2782786E-1,2.7966907E-2,3.8102368E-1,3.1858361E-3,3.0248433E-1,6.058873E-1,3.5200703E-1,6.971599E-1,4.126261E-2,1.4720657E-2,-6.0403514E-1,-1.6014352E-1,-9.60835E-1,-6.583694E-1,-1.11162625E-2,-2.3266178E-2,-4.0934002E-1,-2.1155995E-1,-6.6455895E-1,-1.6101472E-2,-1.6044412E-2,-8.621087E-3,1.206287E-3,-1.8794282E-1,3.2181055E-3,-3.3220413E-1,-1.02766596E-1,2.3035835E-1,1.4317903E-1,-3.2354668E-2,5.697802E-1,2.3421687E-1,-5.582485E-1,-4.287226E-2,-4.8000693E-1,1.3654592E-3,-6.666186E-2,-3.249905E-1,-3.66271E-1,-2.442348E-1,5.780144E-1,-1.3401169E-1,-5.6338534E-3,-4.6580714E-1,1.4735709E-2,5.8502513E-3,-5.4839972E-2,-3.1209E-1,-1.42177E-2,-2.7157411E-2,-5.762615E-2,1.4452991E-1,2.5753295E-1,5.965135E-1,2.0587486E-1,-2.3581412E-2,1.5308245E-1,2.290252E-2,-2.4433104E-2,-9.1556925E-3,-4.246721E-3,6.7620645E-3,3.2049018E-1,2.947636E-2,1.2479034E-2,3.3882537E-1,7.000323E-1,1.9275436E-1,4.170196E-1,5.6446563E-3,7.4282724E-1,3.9100027E-1,-6.162226E-1,-6.85057E-3,2.5678167E-4,-2.2659917E-1,-2.4254225E-2,-5.3152274E-2,-5.418187E-1,-8.119322E-1,-2.1719329E-2,-9.526435E-3,-2.4436595E-1,-2.555614E-3,-2.3445284E-2,-4.4133563E-2,-1.2138927E-2,-6.086428E-3,-3.954745E-1,-2.506818E-1,-2.4723204E-2,-1.9480462E-1,5.436617E-3,1.5281812E-2,8.493514E-4,1.9040248E-1,-8.4872656E-2,6.579123E-3,3.4007292E-2,1.7264858E-2,3.0767483E-1,4.2633247E-3,-2.9903362E-2,-1.2862201E-2,-3.7525794E-1,-4.326368E-2,-7.117797E-3,2.637E-3,-2.0066963E-1,-2.3526706E-2,-2.7876377E-1,-4.1616175E-1,-4.9975007E-3,-2.661003E-1,9.2564365E-3,3.8773395E-2,-1.81402E-1,-1.0777451E-2,-2.53904E-2,-1.1426926E-2,-1.10531235E-2,9.445998E-3,-2.5335886E-2,-6.2844716E-3,-1.00032166E-1,2.1406531E-1,7.917184E-2,2.2795446E-1,1.8153182E-1,4.399657E-1,1.3145007E-2,3.358056E-2,-3.071215E-3,2.7245417E-1,-1.4354147E-1,4.9385738E-2,2.3463771E-1,8.234833E-2,3.720781E-1,5.993112E-3,7.954682E-2,-1.9608092E-2,3.6315155E-1,4.1202173E-2,8.434989E-1,4.464915E-1,1.9714376E-2,-3.2681308E-3,4.7174904E-1,8.704517E-3,9.238188E-1,6.403504E-1,6.667247E-3,2.3271287E-2,-2.0732557E-2,-3.2762542E-2,-1.4937637E-2,-3.4099515E-3,-3.2277763E-2,-1.8075315E-2,-2.5321612E-2,-4.86247E-2,-1.361645E-2,-5.830487E-3,-2.5103481E-2,-1.4039493E-2,-1.6613716E-2,-8.23062E-3,-3.47182E-3,5.2836663E-3,-1.6542563E-2,-4.8995586E-3,5.090248E-3,1.0914207E-2,-7.549979E-4,-9.34368E-3,1.9710544E-2,8.474179E-3,-2.33963E-2,-1.0371071E-2,-1.358055E-2,-2.3354213E-3,-1.5753636E-2,-6.1331927E-3,-9.8488135E-3,-2.1964312E-2,-1.4839064E-2,-5.004596E-3,-1.2282898E-2,-2.4145916E-3,2.9285373E-3,-3.3527252E-3,-1.0547989E-2,-1.4279176E-3,1.9117948E-3,1.7306251E-2,-1.1202204E-2,5.6108893E-3,1.7117174E-2,8.2260175E-3,9.949015E-3,-4.8665223E-3,1.1336489E-2,2.8832387E-2,1.7522098E-2,8.467553E-3,-1.0763338E-2,-1.188025E-3,1.2751066E-2,4.0093233E-4,5.677605E-3,1.4037593E-2,7.2452333E-3,2.0330313E-3,2.314325E-2,1.4010176E-2,8.400453E-3,-2.4590374E-3,1.4308474E-2,2.2969628E-2,1.6011517E-2,-2.1661266E-3,5.3700022E-2,3.0199567E-2,1.5162307E-2,2.9370325E-2,2.8482262E-2,1.630402E-2,2.8308291E-2,4.931226E-2,3.361333E-2,6.8378937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,-1,65,67,69,71,73,75,77,79,81,-1,83,85,-1,87,89,91,93,-1,95,97,99,101,-1,103,105,-1,107,109,111,113,115,117,119,121,123,125,127,129,-1,-1,-1,-1,-1,131,133,135,-1,-1,137,139,141,143,145,-1,-1,147,149,151,153,155,-1,157,159,161,163,-1,-1,165,167,169,171,-1,-1,173,175,177,-1,-1,-1,-1,179,-1,181,183,185,187,189,191,193,195,-1,197,-1,199,201,203,205,207,209,-1,211,-1,-1,213,215,-1,-1,217,219,221,223,225,227,229,-1,-1,-1,-1,-1,231,-1,233,235,237,239,241,-1,243,245,247,-1,-1,249,-1,-1,251,253,-1,-1,255,-1,-1,-1,-1,-1,257,259,261,263,-1,-1,-1,265,267,-1,-1,-1,269,-1,-1,-1,271,-1,-1,-1,273,-1,275,277,-1,279,-1,-1,281,283,-1,-1,-1,-1,-1,-1,285,287,289,291,293,295,-1,-1,-1,297,299,301,303,305,307,-1,309,-1,311,313,315,317,-1,-1,319,-1,321,323,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3190854E2,3.1019485E1,5.9201416E1,8.142731E0,1.6962814E1,1.9171825E1,2.087001E1,3.9961166E0,1.4339647E0,1.2412514E1,6.212055E0,5.78319E0,8.88751E0,5.7744107E0,9.397934E0,0E0,2.2112198E0,3.9066982E-1,5.112957E-1,6.8951006E0,3.5314293E0,0E0,2.1208355E0,3.7260714E0,2.9996562E0,1.2588768E0,8.545633E0,3.7306328E0,5.969453E-1,6.7444153E0,8.2997894E-1,2.3476028E0,8.358291E-1,5.8686924E-1,0E0,0E0,2.154423E-1,2.6094184E0,1.6942677E0,1.4604378E0,6.9842315E-1,3.5581467E-1,1.5438747E-1,3.133174E0,1.3061209E0,0E0,2.55346E-1,8.442354E-1,0E0,7.209443E0,9.636402E-1,9.518022E-1,1.2366815E0,0E0,3.9610767E-1,4.9608955E0,2.4767113E0,8.470478E-1,0E0,3.315155E0,1.0940208E0,0E0,2.2544742E-2,3.94207E-1,2.8073072E-1,1.9098163E-2,1.9805673E-1,9.2361546E-1,9.216565E-1,2.819121E-1,6.479492E-1,7.367153E-1,1.2854595E0,4.100585E-1,0E0,0E0,0E0,0E0,0E0,2.5009632E-1,5.3227234E0,5.8152246E-1,0E0,0E0,4.3156832E-2,3.7700868E-1,1.169157E-2,4.185161E0,2.4771109E0,0E0,0E0,1.0187376E0,5.240003E-1,2.3950434E-1,1.2065118E-1,2.1558428E-1,0E0,3.115345E0,2.5098839E0,4.3464613E-1,1.205574E0,0E0,0E0,7.463646E-1,2.3182768E-1,9.026394E-1,4.7287178E-1,0E0,0E0,8.888674E-2,1.0327709E-1,1.4093351E-1,0E0,0E0,0E0,0E0,1.0416299E-2,0E0,3.289709E-1,8.427731E-1,4.9298763E-2,1.0401985E-1,1.748461E-1,6.305289E-2,1.5968412E-1,2.8150272E-1,0E0,7.663355E-1,0E0,1.070901E-1,2.5004482E-1,1.2410879E-1,1.0267472E-1,7.115567E-1,1.4055114E0,0E0,9.46188E-2,0E0,0E0,4.8814714E-1,4.8110664E-1,0E0,0E0,1.9570003E0,1.4159517E0,1.3205066E0,5.5370235E-1,4.7952902E-1,6.723278E-1,1.3643396E-1,0E0,0E0,0E0,0E0,0E0,2.3470998E-1,0E0,9.54517E-1,1.8891621E0,1.623991E0,7.551477E-1,2.367549E-1,0E0,1.1147995E0,2.4584675E-1,8.5601425E-1,0E0,0E0,1.514594E-1,0E0,0E0,3.6655807E-1,4.9074745E-1,0E0,0E0,3.2884E-2,0E0,0E0,0E0,0E0,0E0,4.097805E-1,2.195065E-1,3.8692406E-1,6.79451E-1,0E0,0E0,0E0,1.0004103E-2,1.040427E-1,0E0,0E0,0E0,6.4934134E-2,0E0,0E0,0E0,2.6144266E-1,0E0,0E0,0E0,1.1867142E-1,0E0,9.234619E-2,8.263588E-2,0E0,1.4959884E-1,0E0,0E0,1.4797983E0,2.696777E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1526762E0,5.2166355E-1,1.5222836E0,7.7828074E-1,3.7737608E-1,7.160802E-1,0E0,0E0,0E0,1.0975015E-1,2.4449253E-1,4.0655148E-1,3.5312593E-2,3.5862796E-2,4.5529842E-2,0E0,3.504198E-1,0E0,1.6699638E0,5.097207E-1,1.2301483E0,2.663188E-1,0E0,0E0,8.559513E-2,0E0,1.3957024E-1,9.18272E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,79,79,80,80,81,81,84,84,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,95,95,97,97,98,98,99,99,100,100,103,103,104,104,105,105,106,106,109,109,110,110,111,111,116,116,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,127,127,129,129,130,130,131,131,132,132,133,133,134,134,136,136,139,139,140,140,143,143,144,144,145,145,146,146,147,147,148,148,149,149,155,155,157,157,158,158,159,159,160,160,161,161,163,163,164,164,165,165,168,168,171,171,172,172,175,175,181,181,182,182,183,183,184,184,188,188,189,189,193,193,197,197,201,201,203,203,204,204,206,206,209,209,210,210,217,217,218,218,219,219,220,220,221,221,222,222,226,226,227,227,228,228,229,229,230,230,231,231,233,233,235,235,236,236,237,237,238,238,241,241,243,243,244,244],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,-1,66,68,70,72,74,76,78,80,82,-1,84,86,-1,88,90,92,94,-1,96,98,100,102,-1,104,106,-1,108,110,112,114,116,118,120,122,124,126,128,130,-1,-1,-1,-1,-1,132,134,136,-1,-1,138,140,142,144,146,-1,-1,148,150,152,154,156,-1,158,160,162,164,-1,-1,166,168,170,172,-1,-1,174,176,178,-1,-1,-1,-1,180,-1,182,184,186,188,190,192,194,196,-1,198,-1,200,202,204,206,208,210,-1,212,-1,-1,214,216,-1,-1,218,220,222,224,226,228,230,-1,-1,-1,-1,-1,232,-1,234,236,238,240,242,-1,244,246,248,-1,-1,250,-1,-1,252,254,-1,-1,256,-1,-1,-1,-1,-1,258,260,262,264,-1,-1,-1,266,268,-1,-1,-1,270,-1,-1,-1,272,-1,-1,-1,274,-1,276,278,-1,280,-1,-1,282,284,-1,-1,-1,-1,-1,-1,286,288,290,292,294,296,-1,-1,-1,298,300,302,304,306,308,-1,310,-1,312,314,316,318,-1,-1,320,-1,322,324,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.4166665E0,1.5938586E8,1E0,4.4839956E3,1.7669278E5,1.0976147E-1,3E0,1.1411955E-1,3.791269E7,1.5151515E-3,1.5106794E3,1.2E1,1.5963264E9,7.096131E0,-6.567799E-2,9.711E3,5.505334E9,2.1111E4,1.5083523E0,3.6709285E6,6.3240066E-2,1E0,1E0,6.7388856E7,2.8326238E3,1.4915413E3,5.9767612E7,1.6E1,1.8137958E6,1.8114515E1,2.1506184E7,8.081268E-1,9.2611E4,-3.158742E-3,5.618961E-3,2.028994E8,3.1294638E2,1.289627E3,2.3023027E3,9.246483E-1,9.877E3,1.91E4,2.73E2,4E0,5.1897928E-2,1.1004604E9,7.946217E-1,9.8784845E-3,2.3314082E6,1.1032986E8,4.664004E8,4.0728608E3,3.256405E-2,3.224158E6,3.970405E3,2.979E3,1.5005797E4,5.707232E-2,5.492982E2,3.342857E2,3.5617957E-3,1.4102349E7,1.172E3,6.5981384E2,3.25855E7,2.7682406E8,1.2245615E0,1.8361508E3,1.5211788E-1,3E1,6.9514094E11,1E0,1.2414683E5,8.781778E-3,-1.0969583E-2,5.04645E-3,1.2359964E-2,2.5785599E-2,4.7E2,8E0,1.0013907E1,-4.4051096E-2,-2.528701E-3,7.431E3,6.83E2,1.5733662E0,2.2746666E2,6.789622E7,5.5007413E-2,1.875829E-2,4.839905E1,3.0497742E3,9.245905E-2,4.798624E3,5.8349776E-3,3.1858361E-3,2.943086E4,2.6974544E7,1.4753189E8,1E0,4.126261E-2,1.4720657E-2,3.647E3,3.86569E5,5.0939494E5,1.3E1,-1.11162625E-2,-2.3266178E-2,1.945345E6,8.4441924E-1,1.25615805E5,-1.6101472E-2,-1.6044412E-2,-8.621087E-3,1.206287E-3,2.3028378E0,3.2181055E-3,2.011E3,1.6837112E7,1.903E3,1.9563605E5,5.148149E5,4.3800272E8,1.2516333E7,6.1707315E0,-4.287226E-2,6.174736E2,1.3654592E-3,1E1,1.8145794E7,6.7E1,6.661E3,3.192E3,1.101214E6,-5.6338534E-3,1.829E3,1.4735709E-2,5.8502513E-3,4.3161097E5,2.6476662E6,-1.42177E-2,-2.7157411E-2,6.54925E6,1.273801E6,3.1517188E2,1E0,2E0,1.7735592E3,1.4274633E7,2.290252E-2,-2.4433104E-2,-9.1556925E-3,-4.246721E-3,6.7620645E-3,1.6007329E7,2.947636E-2,2.711499E6,1E0,1.6144724E-1,1.35E2,3.8466454E0,5.6446563E-3,1.6954315E0,5.012E3,1.3177E4,-6.85057E-3,2.5678167E-4,2.7272727E0,-2.4254225E-2,-5.3152274E-2,1.0201212E3,6.1061732E7,-2.1719329E-2,-9.526435E-3,3.30399E6,-2.555614E-3,-2.3445284E-2,-4.4133563E-2,-1.2138927E-2,-6.086428E-3,1.9E1,2E0,1.3411796E3,5.5871747E2,5.436617E-3,1.5281812E-2,8.493514E-4,2.9956698E-1,2.8216E4,6.579123E-3,3.4007292E-2,1.7264858E-2,5.8426323E0,4.2633247E-3,-2.9903362E-2,-1.2862201E-2,3.3480989E3,-4.326368E-2,-7.117797E-3,2.637E-3,1.5694646E3,-2.3526706E-2,1.6E1,6E0,-4.9975007E-3,3.30399E6,9.2564365E-3,3.8773395E-2,1.445113E6,1.84556E7,-2.53904E-2,-1.1426926E-2,-1.10531235E-2,9.445998E-3,-2.5335886E-2,-6.2844716E-3,1E0,1.372176E6,1.0195312E0,1.5043668E0,2.8E1,7.3E1,1.3145007E-2,3.358056E-2,-3.071215E-3,2.2417E0,3.084419E6,1.954095E3,1.108E3,4.3800272E8,2.025313E6,5.993112E-3,2.6666667E0,-1.9608092E-2,3.7253947E5,1E0,1.447E3,5.57E4,1.9714376E-2,-3.2681308E-3,9E0,8.704517E-3,6.189387E6,4.4263797E0,6.667247E-3,2.3271287E-2,-2.0732557E-2,-3.2762542E-2,-1.4937637E-2,-3.4099515E-3,-3.2277763E-2,-1.8075315E-2,-2.5321612E-2,-4.86247E-2,-1.361645E-2,-5.830487E-3,-2.5103481E-2,-1.4039493E-2,-1.6613716E-2,-8.23062E-3,-3.47182E-3,5.2836663E-3,-1.6542563E-2,-4.8995586E-3,5.090248E-3,1.0914207E-2,-7.549979E-4,-9.34368E-3,1.9710544E-2,8.474179E-3,-2.33963E-2,-1.0371071E-2,-1.358055E-2,-2.3354213E-3,-1.5753636E-2,-6.1331927E-3,-9.8488135E-3,-2.1964312E-2,-1.4839064E-2,-5.004596E-3,-1.2282898E-2,-2.4145916E-3,2.9285373E-3,-3.3527252E-3,-1.0547989E-2,-1.4279176E-3,1.9117948E-3,1.7306251E-2,-1.1202204E-2,5.6108893E-3,1.7117174E-2,8.2260175E-3,9.949015E-3,-4.8665223E-3,1.1336489E-2,2.8832387E-2,1.7522098E-2,8.467553E-3,-1.0763338E-2,-1.188025E-3,1.2751066E-2,4.0093233E-4,5.677605E-3,1.4037593E-2,7.2452333E-3,2.0330313E-3,2.314325E-2,1.4010176E-2,8.400453E-3,-2.4590374E-3,1.4308474E-2,2.2969628E-2,1.6011517E-2,-2.1661266E-3,5.3700022E-2,3.0199567E-2,1.5162307E-2,2.9370325E-2,2.8482262E-2,1.630402E-2,2.8308291E-2,4.931226E-2,3.361333E-2,6.8378937E-3],"split_indices":[20,56,7,102,52,28,38,3,38,45,57,52,18,7,35,0,2,5,9,42,32,0,102,6,7,52,55,45,3,28,35,45,27,29,0,0,7,52,4,55,27,9,2,2,53,0,5,27,0,28,7,12,4,0,47,52,2,4,0,55,52,0,47,2,4,5,7,53,52,27,3,31,102,28,0,0,0,0,0,1,18,56,0,0,2,11,56,52,7,0,0,48,52,41,4,34,0,33,43,12,6,0,0,2,1,28,3,0,0,12,27,47,0,0,0,0,39,0,2,45,0,28,28,7,32,54,0,58,0,8,32,29,9,1,1,0,2,0,0,32,32,0,0,9,9,55,102,8,4,45,0,0,0,0,0,45,0,29,6,57,8,54,0,54,2,9,0,0,56,0,0,4,45,0,0,9,0,0,0,0,0,3,8,4,52,0,0,0,38,9,0,0,0,53,0,0,0,4,0,0,0,52,0,3,3,0,9,0,0,9,45,0,0,0,0,0,0,102,12,53,53,3,0,0,0,0,53,9,55,0,7,1,0,53,0,28,66,0,2,0,0,8,0,45,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.578E3,8.12E2,1.766E3,3.78E2,4.34E2,1.013E3,7.53E2,2.66E2,1.12E2,3.94E2,4E1,3.96E2,6.17E2,2.07E2,5.46E2,8E0,2.58E2,7.4E1,3.8E1,2.75E2,1.19E2,7E0,3.3E1,3.78E2,1.8E1,4.7E1,5.7E2,1.62E2,4.5E1,4.98E2,4.8E1,2.43E2,1.5E1,7E1,4E0,6E0,3.2E1,2.13E2,6.2E1,8.7E1,3.2E1,1.2E1,2.1E1,3.44E2,3.4E1,4E0,1.4E1,4.3E1,4E0,5.57E2,1.3E1,1.33E2,2.9E1,9E0,3.6E1,3.62E2,1.36E2,3.3E1,1.5E1,1.65E2,7.8E1,6E0,9E0,5.1E1,1.9E1,1.5E1,1.7E1,8.8E1,1.25E2,3.5E1,2.7E1,5.5E1,3.2E1,2.8E1,4E0,6E0,6E0,6E0,1.5E1,9.4E1,2.5E2,2.7E1,7E0,5E0,9E0,2.4E1,1.9E1,4.29E2,1.28E2,9E0,4E0,1E2,3.3E1,2.1E1,8E0,3.1E1,5E0,2.95E2,6.7E1,3E1,1.06E2,2.8E1,5E0,1.45E2,2E1,2.8E1,5E1,4E0,5E0,3E1,2.1E1,1.2E1,7E0,9E0,6E0,7E0,1E1,6E0,8.2E1,1.17E2,8E0,1.7E1,1.8E1,1.1E1,1.6E1,3.5E1,2E1,2.6E1,6E0,1E1,1.8E1,5.4E1,4E1,1E1,2.4E2,7E0,2E1,4E0,5E0,1E1,1.4E1,4E0,1.5E1,1.67E2,2.62E2,1E2,2.8E1,2.5E1,7.5E1,2.6E1,7E0,1.6E1,5E0,4E0,4E0,2.6E1,5E0,3.3E1,2.62E2,5.4E1,1.3E1,2.3E1,7E0,9.1E1,1.5E1,1.41E2,4E0,6E0,1.4E1,6E0,2.2E1,3.1E1,1.9E1,2.6E1,4E0,1.7E1,4E0,8E0,4E0,4E0,6E0,4.4E1,3.8E1,6.4E1,5.3E1,4E0,4E0,5E0,1.2E1,1.4E1,4E0,6E0,5E0,1E1,6E0,3E1,5E0,2.2E1,4E0,6E0,4E0,1.1E1,7E0,2.2E1,3.2E1,6E0,3.4E1,4E0,6E0,1.73E2,6.7E1,1.6E1,4E0,6E0,4E0,6E0,8E0,1.45E2,2.2E1,1.48E2,1.14E2,7.2E1,2.8E1,6E0,2.2E1,5E0,2E1,2.8E1,4.7E1,1.1E1,1.5E1,2E1,6E0,2.9E1,4E0,2.42E2,2E1,3.3E1,2.1E1,7E0,6E0,1.8E1,5E0,3E1,6.1E1,4E0,1.1E1,2.5E1,1.16E2,9E0,5E0,1.8E1,1.3E1,8E0,1.1E1,1.3E1,4E0,2.1E1,2.3E1,1.8E1,2E1,4.8E1,1.6E1,2.1E1,3.2E1,4E0,8E0,9E0,5E0,5E0,5E0,1.3E1,9E0,7E0,4E0,1.7E1,5E0,4E0,2.8E1,2.8E1,6E0,1.16E2,5.7E1,3E1,3.7E1,5.6E1,8.9E1,1E1,1.2E1,1.4E1,1.34E2,3.9E1,7.5E1,6.8E1,4E0,1.2E1,1.6E1,1E1,1E1,1.7E1,1.1E1,7E0,4E1,4E0,7E0,5E0,1E1,8E0,1.2E1,1.7E1,1.2E1,1.37E2,1.05E2,4E0,1.6E1,1.5E1,1.8E1,1.2E1,9E0,9E0,9E0,6E0,2.4E1,5.7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"325","size_leaf_vector":"1"}},{"base_weights":[-2.177305E-3,-3.109974E-1,1.4706323E-1,-5.0266206E-1,-1.167484E-1,1.2943312E-2,3.7195837E-1,-6.007198E-1,-2.6831493E-1,-2.2008273E-1,2.72082E-1,-1.21201485E-1,1.3549297E-1,2.0304774E-1,5.1206225E-1,-5.039592E-1,-7.465211E-1,-3.3198023E-1,-9.9929936E-2,-2.465935E-1,6.2588155E-2,1.9634297E-1,5.7685417E-1,-2.2798415E-1,8.816633E-3,1.6379312E-1,-3.138032E-1,-1.5125993E-1,2.9039416E-1,4.2445165E-1,7.6336384E-1,-5.798715E-1,-3.5956374E-1,-9.207706E-1,-5.365214E-1,-3.6952734E-1,-1.9742216E-1,-1.5819725E-1,8.1976745E-3,-1.8209194E-1,-5.0179833E-1,8.899562E-2,3.5244438E-1,6.2727155E-3,7.060639E-1,-4.0791413E-1,-1.5941432E-1,4.8116002E-2,-2.4700822E-2,5.4393936E-2,2.882906E-1,-4.1360053E-1,6.5064193E-3,-3.3403382E-1,-1.6385455E-2,2.3135933E-1,5.841772E-1,3.4333298E-1,6.887756E-1,8.2143813E-1,4.8543113E-1,-4.9355963E-1,-7.336828E-1,-3.4495864E-2,-4.7397327E-1,-8.1148225E-1,-6.763986E-2,-1.2239581E-2,-5.958705E-1,-3.149512E-1,-5.845958E-1,-1.1501239E-2,-3.5830906E-3,-1.9911705E-1,-4.059363E-2,-2.2543395E-1,2.7620506E-1,-3.060266E-1,-6.4167887E-1,1.9844528E-1,-6.546157E-2,1.8288922E-1,4.6137217E-1,4.2244073E-2,2.0113833E-2,-3.4424448E-1,-4.6068918E-2,2.6757216E-1,-1.8233067E-1,-9.279407E-3,-3.5508316E-2,-1.7871484E-3,2.1834405E-1,3.43223E-1,3.0415831E-2,-4.6849307E-1,-6.70781E-3,-6.8614976E-3,-2.0131772E-2,-1.5673976E-2,5.5942405E-2,-9.903871E-2,2.5883222E-1,1.1192671E-2,6.5596527E-1,3.9420572E-1,9.48256E-3,8.239889E-1,3.703889E-1,2.6674857E-2,8.806303E-1,9.33588E-3,5.573964E-1,-5.206115E-1,-4.1056746E-3,-5.3402495E-1,-8.8889587E-1,-7.419551E-3,7.154846E-2,-5.333068E-1,-2.7320558E-1,-8.932874E-1,-4.9735233E-1,-7.1691173E-1,-4.3674052E-1,-3.8372436E-1,-2.1554866E-1,-3.379543E-2,-1.5115008E-2,-2.3299405E-1,-1.8655766E-3,2.0065366E-3,-5.657528E-3,-3.979946E-1,-1.4865334E-1,1.3503914E-1,2.6183097E-2,-4.6663165E-1,-1.0061898E-1,-6.9284E-1,-8.778133E-3,8.573821E-2,2.6352853E-1,-1.78063E-1,5.5774436E-3,1.1394168E-2,3.2502466E-3,2.7717268E-2,9.882949E-3,-4.2078564E-1,-2.5814098E-1,-6.529906E-3,3.1041004E-2,-3.3105686E-1,-1.0198882E-1,-6.503887E-2,2.0209217E-1,-1.3488965E-1,5.8480784E-2,2.3398428E-1,-2.0566338E-3,3.2412097E-1,5.228016E-2,-1.2525055E-1,1.9885115E-1,-1.610768E-2,-2.8759642E-2,-1.1751574E-2,1.3509673E-2,-1.4390664E-2,1.3037489E-2,3.1165037E-1,8.875724E-2,3.5141297E-2,1.2977691E-2,4.7140098E-1,2.0964244E-1,9.483056E-3,-5.965665E-2,5.2226648E-2,6.317359E-1,2.075274E-1,5.4638326E-1,4.5354176E-2,2.0329544E-2,3.13023E-2,1.53788775E-2,-5.4616654E-1,-1.2227148E-2,-1.631163E-2,-3.101353E-2,-4.940112E-2,-2.7385956E-2,-2.5766671E-3,1.0242077E-2,-1.0502715E-2,-5.671272E-1,-8.012925E-3,-1.7339252E-2,-9.5855373E-1,-2.560307E-2,-2.977537E-2,-1.4143422E-2,-4.950856E-1,-4.3297477E-2,-2.5733938E-2,-1.1760652E-2,-4.3180752E-1,-1.2406867E-2,-2.720943E-4,-2.535017E-1,-6.0103433E-3,-1.3956871E-2,-2.8624007E-1,-5.576829E-1,-9.976408E-2,-3.9845824E-1,1.29873855E-2,3.411794E-2,-7.774564E-3,-2.7848635E-2,5.460957E-3,-1.9551258E-1,-7.648658E-1,-1.5791915E-2,-2.2476893E-3,1.0101786E-2,1.4645549E-2,6.98714E-3,-1.1747974E-2,-4.0884595E-3,-3.6651602E-1,-2.9643096E-2,-3.097334E-1,-6.8948895E-2,-3.0167338E-1,-2.5852596E-2,1.0255003E-1,-1.4010999E-1,-8.258962E-2,1.8762449E-2,2.410224E-1,-8.6505385E-3,-3.1090986E-2,-2.7835596E-1,-1.6563079E-1,8.313148E-2,1.3667963E-1,2.656471E-1,2.638264E-1,4.80475E-1,-4.7419965E-3,-2.4536392E-1,1.7108005E-2,1.2362545E-1,-9.23721E-2,1.02292314E-1,3.7155796E-3,-3.7686443E-3,3.594104E-1,1.2454985E-1,1.3836655E-1,-1.6778333E-2,5.0928706E-1,2.6132706E-1,1.3109222E-1,4.247079E-1,-1.368565E-1,5.7108887E-3,9.3585905E-3,3.535002E-2,4.5470186E-3,1.2712292E-2,1.6049072E-2,3.3092637E-2,-2.3479452E-2,-3.2600448E-2,-2.9925006E-2,-1.6470583E-2,-5.219069E-2,-3.4561224E-2,-3.3435423E-2,-1.3748683E-2,-2.29585E-2,-1.2007215E-2,-7.6592276E-3,-1.5544131E-2,-1.6388398E-2,-2.6064522E-3,-9.48867E-3,-3.0426309E-2,-7.115538E-3,3.489239E-3,-3.621891E-2,-1.54984975E-2,-2.5583718E-3,6.4795655E-3,-1.4439862E-2,-2.210388E-3,-2.3891175E-2,-4.4053815E-2,-1.9459734E-2,-1.04857E-2,-1.71291E-2,-6.5613887E-3,-1.1289443E-2,5.0881784E-3,-2.5087299E-2,-1.3125525E-2,1.2909881E-2,2.5327262E-3,-9.759612E-3,-8.103023E-4,-3.245234E-3,-2.17917E-2,8.223555E-3,1.9367574E-2,9.0942474E-4,-1.8975962E-2,-9.899479E-3,-3.0370206E-2,4.67109E-4,-1.2694293E-2,9.529876E-3,1.5959232E-3,1.5775592E-3,9.76155E-3,1.0126574E-2,1.6391853E-2,1.0765654E-2,1.9438943E-2,8.499407E-3,2.6466614E-2,-7.3542446E-3,6.9142105E-3,-3.7244526E-3,-2.3464918E-2,-1.5791834E-3,8.894196E-3,-3.383141E-4,-6.292579E-3,9.43261E-3,6.581684E-4,1.2094253E-2,2.350214E-2,-9.303911E-5,1.1058987E-2,3.7303853E-3,1.2786538E-2,2.1604678E-2,3.2105677E-2,5.9894617E-3,1.7859655E-2,1.2580176E-3,9.886944E-3,2.393401E-2,1.1551445E-2,-2.4068798E-3,-1.0414863E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,83,85,87,-1,89,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,117,119,121,-1,-1,123,125,127,-1,-1,129,131,133,135,137,139,141,143,145,147,-1,-1,149,-1,151,153,155,-1,157,159,161,163,165,-1,-1,-1,-1,167,169,171,-1,173,175,177,179,181,-1,183,-1,185,187,-1,189,191,-1,193,195,197,199,201,203,205,207,209,-1,-1,211,-1,-1,-1,213,215,217,-1,219,221,223,-1,225,227,229,-1,-1,-1,-1,-1,231,233,-1,-1,235,237,239,241,243,245,247,-1,249,-1,251,253,-1,-1,255,-1,-1,257,259,261,-1,-1,263,265,-1,267,-1,269,271,273,-1,-1,-1,-1,275,-1,-1,-1,-1,-1,-1,-1,-1,277,-1,-1,279,-1,-1,-1,281,-1,-1,-1,283,-1,-1,285,-1,-1,287,289,291,293,-1,295,-1,-1,-1,297,299,-1,-1,-1,-1,-1,-1,-1,301,-1,303,305,307,-1,309,311,313,-1,315,-1,317,319,321,323,325,327,329,331,333,335,-1,337,339,341,-1,-1,343,345,347,-1,349,351,353,355,357,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1840094E2,3.1103386E1,5.2237362E1,9.513214E0,1.6814743E1,1.786961E1,1.5207306E1,3.7996979E0,1.3175812E0,1.3192714E1,1.918055E0,7.207641E0,7.2736015E0,9.173442E0,7.4478836E0,1.7755127E0,3.7506943E0,3.8680363E-1,5.797118E-1,5.2584877E0,0E0,1.1751432E0,9.027076E-1,3.4512367E0,7.53921E0,7.270975E0,1.5604606E0,1.448967E0,3.9752388E0,5.4241066E0,1.0539131E0,1.1911469E0,2.4010553E0,1.8349724E0,8.057699E-1,6.375122E-1,7.4086666E-2,1.3628656E-1,0E0,5.246642E0,1.5836315E0,7.56823E-1,4.5122838E-1,0E0,2.0963287E-1,2.2356434E0,2.0840607E0,0E0,2.405225E0,2.634706E0,3.530981E0,3.6647224E-1,0E0,2.8336334E-1,7.701642E-1,1.831954E0,8.6145496E-1,3.4609776E0,2.32086E0,7.7253723E-1,2.826395E-1,8.252773E-1,7.6488876E-1,2.2821604E-1,4.2608643E-1,8.9199066E-1,0E0,0E0,5.536537E-1,3.1378984E-1,1.3191795E-1,0E0,0E0,1.11875E-1,5.7333212E-2,3.1074219E0,7.3381937E-1,9.097433E-1,7.6039886E-1,1.6812456E-1,3.9873615E-1,5.4327488E-2,3.0854154E-1,0E0,0E0,3.593006E-1,0E0,1.6305199E0,2.3321252E0,2.6532001E0,0E0,1.7245562E0,3.0591655E-1,2.4274979E0,1.2047935E0,1.743455E-1,0E0,0E0,0E0,0E0,4.2413285E-1,3.426134E-1,1.6252184E0,0E0,4.1745567E-1,2.4229622E0,3.6037433E-1,1.1261292E0,4.5986462E-1,0E0,3.8703537E-1,0E0,4.854679E-2,3.8105583E-1,0E0,1.571436E-1,1.7228699E-1,0E0,1.7484382E-1,2.7419376E-1,5.26253E-2,5.03191E-1,1.0587549E-1,3.6752224E-1,2.2557878E-1,1.0766029E-1,2.0594513E-1,0E0,0E0,6.039095E-2,0E0,0E0,0E0,1.1431198E0,1.9997818E0,1.9278362E-1,0E0,3.6471915E-1,3.0023688E-1,6.53842E-1,0E0,1.7651725E-1,1.7549992E-2,4.6651006E-2,0E0,0E0,0E0,0E0,0E0,1.1952114E-1,3.3003616E-1,0E0,0E0,2.2171211E-1,1.0280118E0,1.407268E0,7.282722E-1,9.8908365E-1,8.313959E-1,1.762762E-1,0E0,1.7775517E0,0E0,3.4622896E-1,1.9522113E-1,0E0,0E0,2.2091736E-1,0E0,0E0,6.528062E-2,1.2062654E0,9.942306E-1,0E0,0E0,8.5381126E-1,8.565695E-1,0E0,2.9765004E-1,0E0,7.089958E-1,4.241228E-2,2.2189617E-2,0E0,0E0,0E0,0E0,2.2835732E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.404831E-2,0E0,0E0,3.451538E-2,0E0,0E0,0E0,2.7527E-1,0E0,0E0,0E0,1.1454582E-2,0E0,0E0,7.9280615E-2,0E0,0E0,4.226253E-1,4.309721E-1,1.0296316E0,5.5317974E-1,0E0,8.817087E-2,0E0,0E0,0E0,1.2862408E-1,4.826374E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6077747E-2,0E0,1.2022734E-1,2.6430592E-1,3.5254335E-1,0E0,1.566064E-1,7.4928164E-1,1.0444283E0,0E0,4.180379E-1,0E0,6.9439495E-1,6.197164E-1,2.3446158E-1,7.319643E-1,1.1102158E-1,1.3397121E-1,8.1253433E-1,7.4994755E-1,2.8538388E-1,4.246812E-1,0E0,1.4068128E-1,3.9370358E-2,7.458963E-2,0E0,0E0,1.3314896E0,3.674266E-1,2.879045E-1,0E0,7.879677E-1,2.4264419E-1,2.850054E-1,3.6358356E-2,8.518231E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,45,45,46,46,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,68,68,69,69,70,70,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,87,87,88,88,89,89,91,91,92,92,93,93,94,94,95,95,100,100,101,101,102,102,104,104,105,105,106,106,107,107,108,108,110,110,112,112,113,113,115,115,116,116,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,126,126,129,129,133,133,134,134,135,135,137,137,138,138,139,139,141,141,142,142,143,143,149,149,150,150,153,153,154,154,155,155,156,156,157,157,158,158,159,159,161,161,163,163,164,164,167,167,170,170,171,171,172,172,175,175,176,176,178,178,180,180,181,181,182,182,187,187,196,196,199,199,203,203,207,207,210,210,213,213,214,214,215,215,216,216,218,218,222,222,223,223,231,231,233,233,234,234,235,235,237,237,238,238,239,239,241,241,243,243,244,244,245,245,246,246,247,247,248,248,249,249,250,250,251,251,252,252,254,254,255,255,256,256,259,259,260,260,261,261,263,263,264,264,265,265,266,266,267,267],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,84,86,88,-1,90,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,118,120,122,-1,-1,124,126,128,-1,-1,130,132,134,136,138,140,142,144,146,148,-1,-1,150,-1,152,154,156,-1,158,160,162,164,166,-1,-1,-1,-1,168,170,172,-1,174,176,178,180,182,-1,184,-1,186,188,-1,190,192,-1,194,196,198,200,202,204,206,208,210,-1,-1,212,-1,-1,-1,214,216,218,-1,220,222,224,-1,226,228,230,-1,-1,-1,-1,-1,232,234,-1,-1,236,238,240,242,244,246,248,-1,250,-1,252,254,-1,-1,256,-1,-1,258,260,262,-1,-1,264,266,-1,268,-1,270,272,274,-1,-1,-1,-1,276,-1,-1,-1,-1,-1,-1,-1,-1,278,-1,-1,280,-1,-1,-1,282,-1,-1,-1,284,-1,-1,286,-1,-1,288,290,292,294,-1,296,-1,-1,-1,298,300,-1,-1,-1,-1,-1,-1,-1,302,-1,304,306,308,-1,310,312,314,-1,316,-1,318,320,322,324,326,328,330,332,334,336,-1,338,340,342,-1,-1,344,346,348,-1,350,352,354,356,358,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.896243E4,1E0,1.5343539E0,2.2178302E5,5.964E3,8.294758E-1,2.0802219E8,1.3144558E4,7.75541E-1,3.051E3,4.898012E-1,1.5988182E-1,1.3565588E6,5.7118463E2,1.36054E-1,1.998E3,2.437E3,7.218466E7,6.2588155E-2,4.7519747E5,2.1924436E5,3.3870136E7,1.7E1,1.009E3,7.299E3,1.901E3,3.2196458E6,5.528107E3,2.3977574E7,1.25218E5,1.6256282E7,1.3593E4,1.2E1,1.98714E5,5.666351E7,2.04946E6,8.1976745E-3,2.1169072E3,1.3649979E10,1.4187837E-1,1.51529E3,6.2727155E-3,2.093563E0,2.6320755E1,6E0,4.8116002E-2,5.388794E6,2.3314082E6,1.044E1,1.6644816E6,6.5064193E-3,1.980293E7,2.3859661E-2,6.62E2,6.2671E4,3.5301748E7,1.2695038E-3,4.6816683E-1,1.0052E4,2.145E3,1.9303116E5,7.718584E5,1.1766373E3,1E0,-6.763986E-2,-1.2239581E-2,1.4563726E-1,1.09E3,1.0715278E7,-1.1501239E-2,-3.5830906E-3,2.1592189E5,2.1373269E8,4.296196E0,5.185489E3,2.6835732E3,8.755996E-1,2.091623E5,2.106062E1,1.5485274E6,6.4512783E-1,4.2244073E-2,2.0113833E-2,1.62848E5,-4.6068918E-2,3.79E2,2.102858E4,7.2456576E8,-3.5508316E-2,1E0,2.8E1,6.082581E3,4.179E3,3.44875E5,-6.70781E-3,-6.8614976E-3,-2.0131772E-2,-1.5673976E-2,3.084419E6,1.63E2,8E0,1.1192671E-2,3.2E1,1.0938637E1,1.1510106E5,1.417988E6,1.5343539E0,2.6674857E-2,1E0,9.33588E-3,2.2644286E2,2.22747E5,-4.1056746E-3,2.1818182E0,2.3361E4,-7.419551E-3,1.4075E4,1.64354E9,8.081268E-1,2.3902743E0,1.45E2,5.1270317E-2,1.646E3,1.4622253E5,1.2738854E0,-3.379543E-2,-1.5115008E-2,2.241E2,-1.8655766E-3,2.0065366E-3,-5.657528E-3,2.006639E0,3.791269E7,4.0844156E7,2.6183097E-2,1.0376E4,7.398769E4,3.582157E12,-8.778133E-3,9.923929E2,6.1161733E0,1.29E3,5.5774436E-3,1.1394168E-2,3.2502466E-3,2.7717268E-2,9.882949E-3,5.148E1,1.7E1,-6.529906E-3,3.1041004E-2,1.509E3,1.2839224E0,1.3137E4,3.653602E7,3.8773947E0,3.79E2,1.002E3,-2.0566338E-3,2.3314082E6,5.228016E-2,8.503172E2,1.03E2,-1.610768E-2,-2.8759642E-2,2.472E3,1.3509673E-2,-1.4390664E-2,1.42E1,1.4569893E0,1E0,3.5141297E-2,1.2977691E-2,2.3907686E7,1.3359244E9,9.483056E-3,4.9807886E3,5.2226648E-2,5.3473564E1,1.4689211E7,1.1458888E4,4.5354176E-2,2.0329544E-2,3.13023E-2,1.53788775E-2,4.854E4,-1.2227148E-2,-1.631163E-2,-3.101353E-2,-4.940112E-2,-2.7385956E-2,-2.5766671E-3,1.0242077E-2,-1.0502715E-2,5.041533E6,-8.012925E-3,-1.7339252E-2,4.877E4,-2.560307E-2,-2.977537E-2,-1.4143422E-2,3E0,-4.3297477E-2,-2.5733938E-2,-1.1760652E-2,1E0,-1.2406867E-2,-2.720943E-4,2.8E2,-6.0103433E-3,-1.3956871E-2,1.529448E-1,1.6649964E5,7.3582644E5,3.488E3,1.29873855E-2,3.8562928E6,-7.774564E-3,-2.7848635E-2,5.460957E-3,3.7002478E3,1.2E1,-1.5791915E-2,-2.2476893E-3,1.0101786E-2,1.4645549E-2,6.98714E-3,-1.1747974E-2,-4.0884595E-3,3.79E2,-2.9643096E-2,2.5870967E0,8.209663E3,3.26985E4,-2.5852596E-2,1.5597535E3,8.01E2,8.229907E7,1.8762449E-2,3.1998687E3,-8.6505385E-3,2.1827742E6,8.2E1,3.2E1,1.5851064E0,2.8080197E2,1.7640048E0,3.1930302E1,5.0338E4,1.5988282E1,3.3720784E5,1.7108005E-2,1.576081E6,1.2327E4,3.670339E6,3.7155796E-3,-3.7686443E-3,2.0131812E7,1.5092975E6,4.832E3,-1.6778333E-2,2.6396152E-3,1.4563726E-1,1.753919E3,7.56298E5,7.41688E-2,5.7108887E-3,9.3585905E-3,3.535002E-2,4.5470186E-3,1.2712292E-2,1.6049072E-2,3.3092637E-2,-2.3479452E-2,-3.2600448E-2,-2.9925006E-2,-1.6470583E-2,-5.219069E-2,-3.4561224E-2,-3.3435423E-2,-1.3748683E-2,-2.29585E-2,-1.2007215E-2,-7.6592276E-3,-1.5544131E-2,-1.6388398E-2,-2.6064522E-3,-9.48867E-3,-3.0426309E-2,-7.115538E-3,3.489239E-3,-3.621891E-2,-1.54984975E-2,-2.5583718E-3,6.4795655E-3,-1.4439862E-2,-2.210388E-3,-2.3891175E-2,-4.4053815E-2,-1.9459734E-2,-1.04857E-2,-1.71291E-2,-6.5613887E-3,-1.1289443E-2,5.0881784E-3,-2.5087299E-2,-1.3125525E-2,1.2909881E-2,2.5327262E-3,-9.759612E-3,-8.103023E-4,-3.245234E-3,-2.17917E-2,8.223555E-3,1.9367574E-2,9.0942474E-4,-1.8975962E-2,-9.899479E-3,-3.0370206E-2,4.67109E-4,-1.2694293E-2,9.529876E-3,1.5959232E-3,1.5775592E-3,9.76155E-3,1.0126574E-2,1.6391853E-2,1.0765654E-2,1.9438943E-2,8.499407E-3,2.6466614E-2,-7.3542446E-3,6.9142105E-3,-3.7244526E-3,-2.3464918E-2,-1.5791834E-3,8.894196E-3,-3.383141E-4,-6.292579E-3,9.43261E-3,6.581684E-4,1.2094253E-2,2.350214E-2,-9.303911E-5,1.1058987E-2,3.7303853E-3,1.2786538E-2,2.1604678E-2,3.2105677E-2,5.9894617E-3,1.7859655E-2,1.2580176E-3,9.886944E-3,2.393401E-2,1.1551445E-2,-2.4068798E-3,-1.0414863E-2],"split_indices":[20,56,33,102,42,28,2,27,7,52,34,2,27,38,28,55,42,10,2,45,0,28,28,7,0,2,2,29,28,52,47,29,45,10,18,1,12,9,0,52,5,27,4,0,39,46,3,0,47,28,54,32,0,5,38,2,1,45,27,41,2,2,32,47,52,16,0,0,38,2,45,0,0,28,5,58,52,4,42,28,56,47,57,0,0,7,0,2,28,7,0,102,3,52,2,1,0,0,0,0,9,10,17,0,3,53,33,29,42,0,6,0,58,1,0,56,10,0,9,5,27,53,10,38,2,33,53,0,0,4,0,0,0,53,45,45,0,9,33,31,0,52,54,0,0,0,0,0,0,33,3,0,0,10,53,2,45,54,2,29,0,28,0,4,3,0,0,10,0,0,58,46,105,0,0,45,7,0,4,0,56,1,52,0,0,0,0,29,0,0,0,0,0,0,0,0,1,0,0,12,0,0,0,8,0,0,0,105,0,0,0,0,0,39,28,28,2,0,47,0,0,0,52,18,0,0,0,0,0,0,0,1,0,53,28,45,0,33,2,45,0,4,0,28,0,10,53,4,54,56,1,54,28,0,1,9,48,0,0,12,28,2,0,34,38,52,29,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.567E3,8.36E2,1.731E3,4.2E2,4.16E2,1.085E3,6.46E2,2.95E2,1.25E2,3.29E2,8.7E1,5.18E2,5.67E2,2.94E2,3.52E2,1.8E2,1.15E2,9E1,3.5E1,3.24E2,5E0,7.1E1,1.6E1,2.84E2,2.34E2,5.34E2,3.3E1,5.8E1,2.36E2,2.63E2,8.9E1,1.16E2,6.4E1,6.1E1,5.4E1,6.9E1,2.1E1,2.9E1,6E0,2.6E2,6.4E1,4.3E1,2.8E1,4E0,1.2E1,7.7E1,2.07E2,7E0,2.27E2,2.85E2,2.49E2,2.7E1,6E0,2.4E1,3.4E1,1.98E2,3.8E1,2.03E2,6E1,7.2E1,1.7E1,7.7E1,3.9E1,1.7E1,4.7E1,5.1E1,1E1,1E1,4.4E1,5.7E1,1.2E1,1.6E1,5E0,2.1E1,8E0,2.38E2,2.2E1,2.8E1,3.6E1,2.5E1,1.8E1,1.2E1,1.6E1,7E0,5E0,7E1,7E0,1E1,1.97E2,2.23E2,4E0,2.13E2,7.2E1,2.05E2,4.4E1,2.2E1,5E0,7E0,1.7E1,6E0,2.8E1,1.5E1,1.83E2,7E0,3.1E1,1.76E2,2.7E1,4.1E1,1.9E1,1.4E1,5.8E1,4E0,1.3E1,7.2E1,5E0,1.9E1,2E1,8E0,9E0,3.5E1,1.2E1,3.9E1,1.2E1,2.3E1,2.1E1,3.2E1,2.5E1,8E0,4E0,1.7E1,4E0,4E0,4E0,7.2E1,1.66E2,1.5E1,7E0,1.5E1,1.3E1,3.2E1,4E0,1E1,1.5E1,1.1E1,7E0,8E0,4E0,1.1E1,5E0,3.5E1,3.5E1,5E0,5E0,6.8E1,1.29E2,1.77E2,4.6E1,6.6E1,1.47E2,6.8E1,4E0,2.01E2,4E0,2.3E1,2.1E1,1.1E1,1.1E1,2.2E1,6E0,5E0,1E1,1.39E2,4.4E1,2.7E1,4E0,1.23E2,5.3E1,7E0,2E1,1.7E1,2.4E1,1.1E1,8E0,5.4E1,4E0,9E0,4E0,6.5E1,7E0,7E0,1.2E1,1.4E1,6E0,5E0,4E0,4E0,3.1E1,6E0,6E0,3.2E1,7E0,7E0,5E0,1.1E1,1.2E1,1.4E1,7E0,2.2E1,1E1,4E0,2.1E1,6E0,1.1E1,4.4E1,2.8E1,1.4E2,2.6E1,6E0,9E0,4E0,1.1E1,4E0,9E0,2.6E1,6E0,5E0,5E0,1.1E1,4E0,6E0,5E0,2.9E1,6E0,2.7E1,8E0,6.1E1,7E0,2E1,1.09E2,1.71E2,6E0,4.2E1,4E0,3.9E1,2.7E1,1.4E1,1.33E2,1.8E1,5E1,1.47E2,5.4E1,1.2E1,1.1E1,6E0,1.5E1,1.3E1,9E0,6E0,4E0,1.1E2,2.9E1,4E1,4E0,1.03E2,2E1,4E1,1.3E1,1.4E1,6E0,4E0,2E1,4E0,7E0,4E0,4E0,4.1E1,2.4E1,2.6E1,5E0,2.2E1,1E1,5E0,6E0,1.8E1,4E0,9E0,1.2E1,3.7E1,7E0,4E0,2.4E1,1.12E2,2.8E1,4E0,2.2E1,5E0,4E0,5E0,4E0,9E0,1.7E1,2.4E1,5E0,2.2E1,5E0,4E0,4E0,8E0,5.3E1,4E0,1.6E1,7.5E1,3.4E1,1.64E2,7E0,2.9E1,1.3E1,3.5E1,4E0,2.3E1,4E0,5E0,9E0,4.2E1,9.1E1,7E0,1.1E1,2.7E1,2.3E1,1.08E2,3.9E1,8E0,4.6E1,6E0,6E0,7E0,4E0,4E0,1.1E1,4E0,9E0,4E0,5E0,5.5E1,5.5E1,1.3E1,1.6E1,2.7E1,1.3E1,6.8E1,3.5E1,9E0,1.1E1,1.6E1,2.4E1,9E0,4E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"359","size_leaf_vector":"1"}},{"base_weights":[-5.077089E-3,-2.9380485E-1,1.3516971E-1,-4.7116825E-1,-1.2784077E-1,9.516694E-3,3.578309E-1,-5.571684E-1,-2.652721E-1,-2.2117162E-1,2.4377309E-1,-2.6135033E-1,5.521559E-2,2.6042405E-1,5.5094874E-1,-6.046116E-1,-2.7168876E-1,-3.7204185E-1,-1.6427693E-1,-2.4298294E-1,6.722976E-2,1.5746242E-1,5.15757E-1,-1.9331139E-2,-2.1702167E-1,1.5259972E-1,-5.925857E-2,5.1914003E-2,3.78826E-1,7.508217E-1,4.6274102E-1,-5.38698E-1,-9.5295995E-1,4.747494E-3,-3.4563118E-1,-3.9776513E-1,-7.3337834E-4,-2.4930646E-1,-8.427144E-2,-4.1985604E-1,-1.5426387E-1,-9.827114E-3,2.5273663E-1,9.668541E-3,6.3175994E-1,-2.452518E-1,-1.427705E-1,9.250264E-2,3.582965E-1,-1.5716843E-1,6.2191814E-2,1.5344298E-1,-2.1374087E-1,2.141627E-1,5.382823E-1,7.8011024E-1,1.711669E-2,3.3214912E-1,5.906092E-1,-3.6658612E-1,-5.834159E-1,-7.2480553E-1,-1.1429667E0,-3.690289E-1,-6.333752E-3,-3.6399588E-1,-2.884707E-2,-4.36041E-3,-2.700803E-1,-1.4682122E-1,7.2544953E-3,-2.7271712E-1,-5.7263494E-1,-1.1765269E-1,-4.30592E-1,8.322541E-3,-7.10719E-2,1.1762676E-1,3.7098363E-1,3.668151E-2,1.7781489E-2,-2.6486564E-1,-1.0059318E-1,-9.362256E-2,-1.5144171E-2,4.717443E-2,2.7211383E-1,3.9368546E-1,-1.8088613E-2,-1.1867266E-1,-5.621389E-1,4.0611025E-2,4.432483E-2,6.9895886E-2,2.5200135E-1,-2.7482268E-1,4.7548447E-2,2.4144198E-1,-1.5752135E-2,5.8560246E-1,2.3454534E-1,2.1486826E-2,8.275237E-1,3.9775747E-1,2.3940474E-1,6.680384E-1,3.1260496E-1,-1.1293899E-1,-4.170134E-1,-6.461851E-1,-4.3043798E-1,-3.93794E-2,-1.8850397E-2,-2.9204514E-2,-6.4990535E-2,-2.53355E-1,-4.688681E-1,-1.9979443E-2,-2.3393443E-1,-2.9712814E-1,-7.512683E-3,-1.3230706E-3,-1.7183128E-1,-4.5413804E-1,-1.9230135E-1,-7.889213E-1,-4.203225E-1,-1.5530291E-1,1.9193837E-1,-4.6321113E-2,-2.7562466E-1,5.2876E-3,-1.2190942E-1,1.9032778E-1,-1.6684375E-3,4.901698E-1,1.9640148E-1,-2.0238903E-1,-3.220162E-1,2.8983344E-4,-8.864953E-3,-3.4913626E-2,-1.4608999E-1,7.0083864E-2,-2.4115911E-1,2.1181427E-1,4.8666432E-1,6.0837287E-1,2.6114243E-1,-3.7187812E-1,-7.931125E-2,-3.3322144E-2,-1.5417795E-2,3.2614595E-1,-5.5202474E-3,-8.570746E-2,1.1292753E-1,2.489793E-2,2.0149416E-1,-1.8450553E-2,-1.5311214E-1,-1.2334088E-3,5.5099158E-3,2.6387838E-1,-1.2928595E-2,4.7026012E-2,5.4725933E-1,2.9341555E-1,2.412335E-3,5.044488E-2,3.6008913E-2,1.1140009E-2,4.3134376E-1,2.7225816E-1,-2.7965606E-4,6.118161E-1,4.907847E-2,-7.1323514E-3,4.5290002E-1,-1.1605137E-2,1.4474037E-3,-4.4873387E-1,-5.891656E-3,-6.8847394E-1,-2.0530775E-2,-2.0937034E-1,-5.4467577E-1,-5.260505E-3,-1.4531204E-2,-1.2931997E-2,-2.5848703E-2,-4.5688516E-3,-1.5567862E-2,-1.6376376E-2,-8.292424E-3,-4.1450094E-3,-9.909363E-3,-1.1285587E-2,-2.7454387E-2,-1.1057317E-1,-2.6211518E-1,-4.3902546E-2,-2.01486E-2,-1.5854086E-1,-5.186935E-1,-2.0766957E-1,-3.6154605E-2,2.1838408E-2,7.971565E-2,-4.6201893E-3,-3.2405588E-1,-1.8477276E-1,2.665446E-3,1.655056E-3,1.2215945E-2,2.982918E-2,1.3294087E-2,1.5607623E-2,4.272954E-3,-2.3686276E-3,-2.1874718E-1,-2.6732057E-2,-2.8122434E-1,-7.5477674E-3,2.1052355E-2,-9.635405E-3,-2.1900653E-3,-2.8891856E-2,1.5123911E-1,3.9299927E-4,-3.2460418E-1,2.4108034E-1,4.6228107E-2,2.9947506E-2,1.7380886E-2,4.8241E-1,4.4124525E-2,9.673005E-2,3.7236214E-1,-5.751878E-1,-1.8981285E-2,-1.9327228E-1,2.6375726E-3,1.9564035E-3,3.8959163E-1,1.51449125E-2,-3.4088932E-2,-7.937582E-3,-5.490185E-4,1.8161437E-2,8.2579866E-2,2.3912258E-1,2.8592195E-2,-6.4622716E-4,-1.9524929E-1,1.2101702E-1,3.3915365E-1,5.861843E-1,3.4668016E-1,1.7114066E-2,6.332046E-3,2.2567568E-2,1.0950745E-2,3.107228E-1,6.3777487E-3,5.006412E-1,3.4949664E-2,3.346535E-2,2.6050902E-3,-2.4079083E-2,-1.0674264E-2,-4.1123405E-2,-3.0396866E-2,-1.626101E-2,-3.6159141E-3,-9.017344E-3,-2.9414799E-2,-9.312868E-3,-2.1862087E-3,-5.9835934E-3,-1.6491944E-2,-1.4907494E-2,3.4426244E-3,-1.1189912E-2,-2.8189765E-2,-1.6109638E-2,-7.618884E-3,-3.7857275E-3,4.6842336E-3,-2.3574657E-3,1.3377893E-2,-1.916022E-2,-7.5568706E-3,-1.3834174E-2,-5.2023605E-3,-4.489057E-3,-1.2107896E-2,-8.351368E-3,-1.5769958E-2,-8.4093644E-4,2.2536798E-3,1.3336002E-2,-3.1141276E-3,1.3328015E-2,5.4779365E-3,-2.2842448E-2,-5.0852587E-3,1.3445846E-2,6.489042E-3,-6.2663727E-3,8.105767E-3,1.791591E-2,3.129498E-2,3.104001E-4,9.335463E-3,1.2162039E-2,2.3910746E-2,-4.285757E-2,-1.4576909E-2,-7.375894E-3,6.708347E-3,-1.1933443E-2,4.782881E-4,4.2060195E-3,-5.441783E-3,2.1479296E-2,8.327552E-3,-1.3785913E-2,2.532326E-3,-6.9663404E-3,5.378355E-3,1.8359773E-2,9.460562E-3,-2.9384252E-3,5.510007E-3,-1.2359356E-2,-3.118644E-3,1.3958055E-2,2.6439133E-3,1.9114243E-2,4.303201E-3,3.0342719E-2,1.6557887E-2,3.1656164E-3,2.066381E-2,9.745133E-3,1.922281E-2,2.754305E-2,1.5885225E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,-1,45,47,49,51,53,55,57,59,61,-1,63,65,-1,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,107,109,111,113,115,-1,117,-1,-1,119,121,-1,123,125,127,129,-1,131,133,135,-1,-1,137,139,141,-1,143,145,147,-1,149,151,153,-1,155,157,159,161,163,-1,165,167,-1,169,171,173,175,177,179,181,183,185,-1,-1,-1,-1,187,189,-1,191,193,-1,-1,195,197,199,201,203,205,207,-1,209,-1,211,213,-1,215,217,219,221,-1,-1,223,225,227,229,231,233,235,237,239,241,-1,-1,243,245,247,249,-1,251,-1,253,-1,-1,255,-1,-1,257,259,-1,-1,-1,-1,261,263,-1,265,-1,-1,267,-1,-1,269,-1,271,-1,273,275,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,277,279,-1,-1,281,283,285,287,-1,289,-1,291,293,-1,-1,-1,-1,-1,-1,-1,-1,295,-1,297,-1,299,-1,-1,301,303,-1,305,307,309,-1,-1,311,-1,313,315,317,319,321,323,-1,325,327,-1,-1,-1,-1,329,331,333,-1,335,337,339,341,343,-1,-1,-1,-1,345,-1,347,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.05402885E2,2.4989769E1,4.901414E1,7.129822E0,1.5349136E1,1.38905325E1,1.1745064E1,3.7499619E0,1.2652197E0,1.238929E1,2.0080729E0,8.17502E-1,1.071109E1,1.0398666E1,3.3725662E0,5.1579895E0,1.198777E0,5.365238E-1,4.211744E-1,5.411747E0,0E0,1.1022319E0,6.6279316E-1,0E0,2.2071218E-1,6.382619E0,5.2656803E0,4.186434E0,6.9477463E0,4.8607635E-1,2.291914E0,1.4023895E0,8.1890106E-1,0E0,1.3758183E-1,1.23355865E-1,0E0,8.078313E-2,5.2365583E-1,2.451479E0,2.316647E0,2.9271337E-1,6.5932107E-1,0E0,1.623559E-1,2.2020245E-1,2.5527382E-1,3.2681687E0,3.1249704E0,3.7527542E0,3.4887524E0,9.04624E-1,6.993481E-1,2.011353E0,1.8173599E0,6.068764E-1,0E0,3.6721992E-1,1.3840656E0,5.412774E-1,1.3062363E0,1.4272404E-1,9.1329E-1,2.5077105E-1,0E0,1.3499355E-1,0E0,0E0,4.276085E-2,7.972354E-2,0E0,8.026037E-1,1.5132122E0,2.4542925E0,1.8216834E0,0E0,1.9275826E-1,2.4789649E-1,3.8230824E-1,0E0,0E0,2.0619154E-1,9.911241E-2,8.422834E-2,0E0,2.150497E0,9.448395E-1,3.0281239E0,0E0,2.2129736E0,3.5202265E-1,2.557186E0,0E0,4.2482263E-1,5.3400636E-1,3.4574127E-1,4.3498352E-2,1.5047426E0,0E0,1.0799751E0,2.0514524E-1,0E0,2.4516678E-1,1.5629339E-1,2.75195E-1,4.0592384E-1,1.2145412E0,1.591165E-1,3.076191E-1,8.614044E-1,1.1739302E0,0E0,0E0,0E0,0E0,6.368697E-2,2.894926E-2,0E0,1.15318E-1,2.9946089E-2,0E0,0E0,3.7693977E-2,2.6754904E-1,2.2207046E-1,4.0692997E-1,8.1907654E-1,1.1530199E0,5.978473E-1,0E0,1.6518974E-1,0E0,1.8612935E-1,1.1960101E-1,0E0,1.5235758E-1,1.1472815E-1,8.8505864E-2,1.7968655E-1,0E0,0E0,1.0068047E-1,6.325218E-2,2.4140892E0,5.012697E-1,3.0384898E-1,5.7436466E-2,1.0273123E0,1.2622805E0,2.1415024E0,1.8309484E0,0E0,0E0,4.7637224E-1,2.3597093E0,7.477513E-2,3.425517E-1,0E0,2.7942824E-1,0E0,9.788942E-2,0E0,0E0,1.2816887E0,0E0,0E0,6.399994E-1,7.934797E-2,0E0,0E0,0E0,0E0,5.1835537E-2,1.2721968E-1,0E0,1.4675331E-1,0E0,0E0,1.1817782E0,0E0,0E0,1.621933E-1,0E0,5.343056E-1,0E0,2.7857947E-1,4.227438E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.03286356E-1,1.8294704E-1,0E0,0E0,3.7033176E-1,2.0876646E-1,7.677226E-1,3.055399E-1,0E0,4.1823757E-1,0E0,1.165297E-1,6.0310006E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.458102E-2,0E0,8.0626726E-2,0E0,1.0781402E-2,0E0,0E0,1.3615268E0,7.6368093E-1,0E0,4.8512506E-1,1.3641882E-1,2.3934746E-1,0E0,0E0,3.2795095E-1,0E0,2.4351948E-1,4.5857906E-1,1.2714095E0,2.5635928E-1,7.600267E-1,1.0551304E0,0E0,1.2591195E-1,1.7101485E0,0E0,0E0,0E0,0E0,2.6534578E-1,1.6040897E-1,7.073551E-2,0E0,7.453537E-2,4.5966393E-1,8.453045E-1,2.7669525E-1,3.5272908E-1,0E0,0E0,0E0,0E0,1.2184596E-1,0E0,6.508732E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,68,68,69,69,71,71,72,72,73,73,74,74,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86,87,87,89,89,90,90,91,91,93,93,94,94,95,95,96,96,97,97,99,99,100,100,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,115,115,116,116,118,118,119,119,122,122,123,123,124,124,125,125,126,126,127,127,128,128,130,130,132,132,133,133,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144,145,145,146,146,147,147,148,148,149,149,150,150,153,153,154,154,155,155,156,156,158,158,160,160,163,163,166,166,167,167,172,172,173,173,175,175,178,178,181,181,183,183,185,185,186,186,199,199,200,200,203,203,204,204,205,205,206,206,208,208,210,210,211,211,220,220,222,222,224,224,227,227,228,228,230,230,231,231,232,232,235,235,237,237,238,238,239,239,240,240,241,241,242,242,244,244,245,245,250,250,251,251,252,252,254,254,255,255,256,256,257,257,258,258,263,263,265,265],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,-1,46,48,50,52,54,56,58,60,62,-1,64,66,-1,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,108,110,112,114,116,-1,118,-1,-1,120,122,-1,124,126,128,130,-1,132,134,136,-1,-1,138,140,142,-1,144,146,148,-1,150,152,154,-1,156,158,160,162,164,-1,166,168,-1,170,172,174,176,178,180,182,184,186,-1,-1,-1,-1,188,190,-1,192,194,-1,-1,196,198,200,202,204,206,208,-1,210,-1,212,214,-1,216,218,220,222,-1,-1,224,226,228,230,232,234,236,238,240,242,-1,-1,244,246,248,250,-1,252,-1,254,-1,-1,256,-1,-1,258,260,-1,-1,-1,-1,262,264,-1,266,-1,-1,268,-1,-1,270,-1,272,-1,274,276,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,278,280,-1,-1,282,284,286,288,-1,290,-1,292,294,-1,-1,-1,-1,-1,-1,-1,-1,296,-1,298,-1,300,-1,-1,302,304,-1,306,308,310,-1,-1,312,-1,314,316,318,320,322,324,-1,326,328,-1,-1,-1,-1,330,332,334,-1,336,338,340,342,344,-1,-1,-1,-1,346,-1,348,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,8.896243E4,1E0,1.5283889E0,3.5E2,1.3664E4,5.121E3,2.3859661E-2,2.14099E5,1.0346507E7,4.1E1,1.9E1,3.2627738E5,4.68646E5,4.987639E7,2.1693E5,8.962E3,2.060792E8,1E0,6.722976E-2,1.5190727E5,2.7271206E5,-1.9331139E-2,2.5682794E5,4.3256033E2,1.023747E6,2.7246006E7,3.171E3,4.5481584E7,6.4683E4,2.0304577E-1,1E0,4.747494E-3,1.6912074E2,3.6709285E6,-7.3337834E-4,2.22E2,2.437E3,1.7232166E8,1.0828989E8,5.743459E-3,1.51529E3,9.668541E-3,8.931957E9,1.5782692E2,1.83096E5,2.657819E6,3.8879236E7,4.719849E7,6.7652373E3,6.222E3,1.6E1,3.4463277E0,8.3769E0,1.8382221E-1,1.711669E-2,2.3058404E7,1.96E2,1.75E2,3.8410318E2,9.195571E2,1.2920592E0,9.982716E0,-6.333752E-3,1.305E3,-2.884707E-2,-4.36041E-3,1.774E3,1.0214149E6,7.2544953E-3,1.0135518E4,1.0503613E5,2.0601392E3,1.3528846E1,8.322541E-3,1.0275E4,1.1620979E3,3.1E1,3.668151E-2,1.7781489E-2,2.00059E5,1.1347137E2,6.0875E1,-1.5144171E-2,2.9676E4,6.789622E7,8E0,-1.8088613E-2,1.9598837E2,1.5675918E4,2.89E2,4.432483E-2,2.2580828E-1,1.2737473E1,1.6347875E3,1E0,1.4955851E7,-1.5752135E-2,1.0918E4,1.708775E7,2.1486826E-2,4.8279068E1,2.0201733E9,2.869537E-1,1.921032E4,2.42E2,3.9794872E0,2.742277E6,2.765625E0,1.775044E7,-3.93794E-2,-1.8850397E-2,-2.9204514E-2,-6.4990535E-2,9E0,1.7560976E0,-1.9979443E-2,1.69E2,1.1400756E6,-7.512683E-3,-1.3230706E-3,1E0,3.0990322E5,3.9986582E4,2.4559366E1,1.3498145E5,5.0818306E-1,2.1853803E4,-4.6321113E-2,2.771816E11,5.2876E-3,1.873E3,6.58542E5,-1.6684375E-3,2.2056E4,1.2903092E7,8E0,1E0,2.8983344E-4,-8.864953E-3,9E0,1.638058E2,7.08E2,1.0195312E0,1E0,7E0,7.2041174E2,1.954E3,1E0,7.57657E4,-3.3322144E-2,-1.5417795E-2,7.2210526E-1,7.074777E6,1.45E2,1.0506024E1,2.489793E-2,2.03E2,-1.8450553E-2,1.5451318E8,-1.2334088E-3,5.5099158E-3,2.960909E2,-1.2928595E-2,4.7026012E-2,2E0,4.607796E-1,2.412335E-3,5.044488E-2,3.6008913E-2,1.1140009E-2,2.6085916E7,7.224059E-1,-2.7965606E-4,1.28715766E5,4.907847E-2,-7.1323514E-3,3.1917114E8,-1.1605137E-2,1.4474037E-3,5.1E1,-5.891656E-3,2.3033286E5,-2.0530775E-2,1E0,1.1253046E9,-5.260505E-3,-1.4531204E-2,-1.2931997E-2,-2.5848703E-2,-4.5688516E-3,-1.5567862E-2,-1.6376376E-2,-8.292424E-3,-4.1450094E-3,-9.909363E-3,-1.1285587E-2,-2.7454387E-2,1.2015E4,1.2342285E1,-4.3902546E-2,-2.01486E-2,4.1667834E-2,3.940884E5,2.728889E2,4.5808794E5,2.1838408E-2,1E0,-4.6201893E-3,8.755996E-1,1.7148E4,2.665446E-3,1.655056E-3,1.2215945E-2,2.982918E-2,1.3294087E-2,1.5607623E-2,4.272954E-3,-2.3686276E-3,5.836513E3,-2.6732057E-2,8E0,-7.5477674E-3,8.112033E-1,-9.635405E-3,-2.1900653E-3,8.695652E-1,1.19839E5,3.9299927E-4,1.287E3,2.2174756E4,3.928E4,2.9947506E-2,1.7380886E-2,1.01E2,4.4124525E-2,2.4125186E3,7.621583E2,8E0,2.091623E5,3.474851E8,4.4E1,1.9564035E-3,7.879123E7,1.2E1,-3.4088932E-2,-7.937582E-3,-5.490185E-4,1.8161437E-2,6.480713E6,9.076336E6,9.8E1,-6.4622716E-4,2.0676695E-1,2.819136E0,2.03353E-6,2.3977574E7,7.7329254E2,1.7114066E-2,6.332046E-3,2.2567568E-2,1.0950745E-2,3.8134545E6,6.3777487E-3,9.82576E5,3.4949664E-2,3.346535E-2,2.6050902E-3,-2.4079083E-2,-1.0674264E-2,-4.1123405E-2,-3.0396866E-2,-1.626101E-2,-3.6159141E-3,-9.017344E-3,-2.9414799E-2,-9.312868E-3,-2.1862087E-3,-5.9835934E-3,-1.6491944E-2,-1.4907494E-2,3.4426244E-3,-1.1189912E-2,-2.8189765E-2,-1.6109638E-2,-7.618884E-3,-3.7857275E-3,4.6842336E-3,-2.3574657E-3,1.3377893E-2,-1.916022E-2,-7.5568706E-3,-1.3834174E-2,-5.2023605E-3,-4.489057E-3,-1.2107896E-2,-8.351368E-3,-1.5769958E-2,-8.4093644E-4,2.2536798E-3,1.3336002E-2,-3.1141276E-3,1.3328015E-2,5.4779365E-3,-2.2842448E-2,-5.0852587E-3,1.3445846E-2,6.489042E-3,-6.2663727E-3,8.105767E-3,1.791591E-2,3.129498E-2,3.104001E-4,9.335463E-3,1.2162039E-2,2.3910746E-2,-4.285757E-2,-1.4576909E-2,-7.375894E-3,6.708347E-3,-1.1933443E-2,4.782881E-4,4.2060195E-3,-5.441783E-3,2.1479296E-2,8.327552E-3,-1.3785913E-2,2.532326E-3,-6.9663404E-3,5.378355E-3,1.8359773E-2,9.460562E-3,-2.9384252E-3,5.510007E-3,-1.2359356E-2,-3.118644E-3,1.3958055E-2,2.6439133E-3,1.9114243E-2,4.303201E-3,3.0342719E-2,1.6557887E-2,3.1656164E-3,2.066381E-2,9.745133E-3,1.922281E-2,2.754305E-2,1.5885225E-2],"split_indices":[20,56,33,102,41,2,2,2,38,2,44,2,3,28,29,45,29,0,7,16,0,33,28,0,28,52,1,45,2,47,2,27,100,0,57,32,0,11,2,7,45,57,4,0,12,4,47,28,45,45,52,2,18,53,54,38,0,45,8,11,4,4,56,57,0,2,0,0,10,48,0,51,33,52,56,0,9,55,3,0,0,5,52,52,0,29,7,17,0,52,48,11,0,42,54,4,64,47,0,9,45,0,58,5,34,4,8,54,9,54,45,0,0,0,0,3,56,0,10,33,0,0,100,28,33,56,33,39,33,0,31,0,0,1,0,9,32,3,13,0,0,0,52,2,53,105,3,52,2,16,28,0,0,58,47,10,58,0,3,0,12,0,0,52,0,0,6,38,0,0,0,0,29,27,0,28,0,0,32,0,0,3,0,33,0,100,5,0,0,0,0,0,0,0,0,0,0,0,0,9,56,0,0,38,32,52,28,0,102,0,42,9,0,0,0,0,0,0,0,0,28,0,17,0,58,0,0,53,12,0,2,47,12,0,0,0,0,33,52,17,28,7,3,0,32,18,0,0,0,0,12,32,8,0,39,54,37,47,52,0,0,0,0,47,0,11,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.601E3,8.5E2,1.751E3,4.1E2,4.4E2,1.12E3,6.31E2,2.88E2,1.22E2,3.52E2,8.8E1,1.61E2,9.59E2,4.21E2,2.1E2,2.46E2,4.2E1,5.8E1,6.4E1,3.48E2,4E0,6.8E1,2E1,4E1,1.21E2,5.18E2,4.41E2,1.53E2,2.68E2,6.2E1,1.48E2,2.09E2,3.7E1,7E0,3.5E1,5.4E1,4E0,3E1,3.4E1,1.15E2,2.33E2,2.5E1,4.3E1,6E0,1.4E1,8.6E1,3.5E1,4.02E2,1.16E2,2.44E2,1.97E2,1.11E2,4.2E1,1.33E2,1.35E2,5.7E1,5E0,7.5E1,7.3E1,4.5E1,1.64E2,1.9E1,1.8E1,3.1E1,4E0,4.8E1,6E0,4E0,2.6E1,2.7E1,7E0,6E1,5.5E1,2.07E2,2.6E1,6E0,1.9E1,2.1E1,2.2E1,9E0,5E0,7.5E1,1.1E1,2.8E1,7E0,3.22E2,8E1,1.11E2,5E0,2.24E2,2E1,1.93E2,4E0,6.1E1,5E1,3.4E1,8E0,1.27E2,6E0,1.16E2,1.9E1,8E0,4.9E1,4.2E1,3.3E1,5.6E1,1.7E1,8E0,3.7E1,1.14E2,5E1,1.5E1,4E0,5E0,1.3E1,1.6E1,1.5E1,3.6E1,1.2E1,2E1,6E0,5E0,2.2E1,1.7E1,4.3E1,2.1E1,3.4E1,1.85E2,2.2E1,5E0,2.1E1,4E0,1.5E1,1.4E1,7E0,1.2E1,1E1,3.8E1,3.7E1,5E0,6E0,1.4E1,1.4E1,2.99E2,2.3E1,6.4E1,1.6E1,4.1E1,7E1,2.9E1,1.95E2,1.3E1,7E0,2.6E1,1.67E2,1.3E1,4.8E1,7E0,4.3E1,1.8E1,1.6E1,4E0,4E0,1.22E2,5E0,9E0,1.07E2,1.4E1,5E0,1.5E1,3.4E1,8E0,3.4E1,2.9E1,4E0,5E1,6E0,4E0,1.3E1,4E0,4E0,3.3E1,4E0,9.5E1,1.9E1,1.8E1,3.2E1,4E0,1.2E1,4E0,1.1E1,5E0,7E0,1.5E1,5E0,6E0,1.6E1,6E0,1.1E1,2.1E1,2.2E1,1.6E1,5E0,1E1,2.4E1,1.28E2,5.7E1,6E0,1.6E1,5E0,1.6E1,1.1E1,4E0,4E0,1E1,7E0,5E0,4E0,6E0,4E0,3.4E1,4E0,3.3E1,4E0,1E1,9E0,5E0,1.35E2,1.64E2,6E0,1.7E1,5.4E1,1E1,7E0,9E0,3E1,1.1E1,2.9E1,4.1E1,1.8E1,1.1E1,8.1E1,1.14E2,5E0,2.1E1,1.63E2,4E0,6E0,7E0,4E0,4.4E1,3.5E1,8E0,4E0,1.2E1,4.3E1,7.9E1,8.8E1,1.9E1,1E1,4E0,3E1,4E0,2.2E1,7E0,2.5E1,2.5E1,8E0,5E0,2.8E1,5E0,3.2E1,6.3E1,9E0,9E0,4E0,2.8E1,9E0,1.2E1,8E0,1.4E1,6E0,4E0,4E0,2E1,4E1,8.8E1,4.4E1,1.3E1,1E1,6E0,1.1E1,5E0,4E0,7E0,6E0,2.8E1,9E0,2.4E1,4E0,6E0,1.3E1,1.22E2,4.2E1,1.22E2,1E1,7E0,4.2E1,1.2E1,4E0,6E0,1.8E1,1.2E1,1.5E1,1.4E1,2E1,2.1E1,8E0,1E1,6E0,5E0,6.6E1,1.5E1,6.6E1,4.8E1,1.7E1,4E0,1.7E1,1.46E2,4E0,4E1,8E0,2.7E1,4E0,4E0,8E0,4E0,1.2E1,3.1E1,6.7E1,1.2E1,8E1,8E0,4E0,1.5E1,1E1,1.2E1,1.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"349","size_leaf_vector":"1"}},{"base_weights":[-8.2452025E-4,-2.929924E-1,1.3822374E-1,-4.4537944E-1,-1.0959473E-1,1.9181045E-2,3.5026762E-1,-5.2057475E-2,-5.099567E-1,-2.1229686E-1,1.7804015E-1,-4.8712045E-2,2.003148E-1,3.8165545E-1,-2.0839854E-1,-1.4669605E-1,3.762726E-2,-5.7278925E-1,-2.784991E-1,-2.6210067E-1,-8.944991E-2,5.3240452E-2,3.9842737E-1,-1.9009273E-1,1.7608114E-2,2.2699755E-1,-3.7051937E-1,2.2274482E-1,5.09849E-1,-3.2121786E-1,5.669807E-2,-3.4558117E-1,-4.4391688E-4,-6.229086E-1,-3.3881402E-1,-1.8119697E-1,-5.6641054E-1,-3.2464755E-1,-1.9226284E-1,-3.365478E-2,-5.0861157E-2,-4.9549088E-2,1.7481321E-1,5.2036587E-3,4.4253322E-1,-2.965727E-1,-9.5267914E-2,7.1392024E-1,-3.055963E-3,1.2235744E-1,3.4129667E-1,-2.4013897E-2,-7.1775946E-3,9.377097E-2,3.2739168E-1,3.4106702E-1,6.030643E-1,-2.079398E-2,-2.2367868E-1,-6.080276E-3,1.4745017E-2,-1.2471662E-3,-4.0546334E-1,6.800011E-2,-1.7835398E-1,-5.604461E-1,-8.676947E-1,-2.7624014E-1,-4.6611652E-1,-2.3962368E-1,1.2740903E-1,-3.6717525E-1,-4.2185403E-2,-3.438757E-1,-5.9068925E-4,-2.3216191E-1,-1.1267744E-1,6.258056E-2,-1.3544244E-1,1.08364604E-1,-1.233657E-1,-6.236405E-4,2.323585E-1,3.2354023E-2,3.5576805E-1,-2.6107052E-1,-4.752912E-1,-1.4628667E-1,1.4555407E-2,1.943138E-2,3.946654E-2,-1.3774538E-2,5.729111E-2,7.777419E-2,2.774226E-1,2.8613883E-1,5.406493E-1,-1.9346073E-1,1.393821E-1,3.804044E-1,5.347346E-2,2.5996566E-1,5.4454434E-1,8.8985646E-1,5.612747E-1,-1.3736463E-2,-3.8212107E-3,-7.128201E-3,-4.589079E-1,2.3812298E-2,1.2433028E-2,-1.2798878E-2,-2.464853E-3,-3.6819406E-2,-5.2035314E-1,-9.435245E-1,-2.1277664E-2,-3.3767378E-1,-1.3802949E-1,-2.6236763E-2,-1.3515623E-2,-3.4910765E-2,-2.8433964E-1,1.593118E-2,-4.2598057E-3,-3.0468455E-2,-2.0915756E-1,-3.2811306E-2,-3.2336158E-1,-2.4977005E-1,-2.7789155E-3,-1.7073397E-1,7.987358E-3,1.4341576E-1,-3.732252E-2,-1.9174509E-1,1.1107324E-1,8.0682345E-5,1.2865992E-2,2.5813054E-2,-2.0889206E-1,1.06337324E-1,1.60452E-2,7.284325E-3,4.3365666E-1,-3.4536573E-1,-1.5149198E-1,-2.5930554E-1,-3.216593E-2,-2.8240573E-2,-1.754266E-1,1.4599521E-1,-4.340437E-2,1.5451021E-2,-3.3300123E-1,-3.4682903E-2,1.3584198E-1,3.954107E-1,1.8555331E-1,8.7098636E-2,3.2892677E-1,1.4862201E-2,6.191357E-1,-2.6409066E-1,-2.739853E-3,-2.3282102E-1,1.7670135E-1,4.0588814E-1,-4.990738E-3,1.22520074E-1,-6.2348455E-2,1.75895E-1,3.131691E-1,6.1623615E-1,9.904823E-4,2.9015813E-2,5.002569E-2,5.2968055E-1,4.8729997E-2,-1.0710474E-2,-2.58661E-2,6.4336546E-2,-3.6084086E-3,-4.0037954E-1,-5.9217745E-1,-7.945482E-1,-5.8664985E-2,-2.5021142E-1,-2.1293333E-2,-1.1745214E-2,-1.7924649E-3,3.3625897E-3,-6.560609E-3,-3.0872238E-1,-3.7174274E-3,-4.7916723E-3,-1.3425117E-2,-2.0578215E-2,-2.8092632E-1,-2.8230247E-1,-1.0424827E-1,-3.6558046E-3,-1.9377178E-1,-2.0582373E-3,4.2510238E-3,2.0512035E-3,1.8750304E-1,3.2664845E-3,-1.05477326E-1,-2.2235967E-1,2.431387E-3,-2.1951473E-3,1.2185075E-2,-2.1859428E-3,3.971273E-3,-3.7649826E-3,-2.6274306E-1,7.994347E-3,4.972213E-4,1.6375402E-2,2.8642936E-2,-3.748568E-1,-1.6632295E-1,-1.0434832E-1,-2.6313903E-2,-7.366748E-3,-1.707317E-2,-1.1098916E-2,5.119646E-2,-2.59144E-1,-1.24007955E-1,3.991864E-3,9.273184E-3,-1.2599917E-3,-1.35996E-1,-2.3973865E-2,1.7419979E-1,-5.300314E-1,-1.5417583E-1,2.3787118E-2,-1.6510546E-2,1.1480129E-1,1.8797977E-2,1.35102235E-2,2.4074456E-2,-1.166818E-3,2.3608112E-1,1.5005295E-2,1.926711E-2,3.6014482E-1,1.7185147E-1,1.879202E-2,3.3556685E-2,-4.616203E-3,-1.9575968E-2,-3.5246909E-3,-1.9036952E-2,2.0257896E-1,-1.672819E-2,1.7839792E-1,4.7545362E-1,1.2854133E-3,1.7802432E-1,-5.742292E-3,-4.0699757E-4,2.5210014E-1,3.8136896E-2,3.0057877E-3,3.3300027E-1,1.6965777E-2,3.296467E-2,4.457734E-1,6.648192E-1,-1.7835952E-4,4.3605603E-3,-2.1759437E-2,5.7427E-3,-3.363548E-2,-2.1462262E-2,-4.2965278E-2,-2.6169393E-2,-5.7323948E-3,-1.4281144E-2,-1.0038699E-2,-1.8971154E-2,-3.4114014E-4,-1.491505E-2,-1.2394227E-2,-2.1870537E-2,-8.138765E-3,5.205708E-4,-1.0632323E-2,-5.0851647E-3,1.2926117E-2,5.5298777E-3,-7.7699386E-3,-1.2167741E-3,-1.5402019E-2,-8.264758E-3,-1.6685007E-2,-8.712472E-3,-1.0413487E-2,-1.9868547E-2,-1.0119685E-2,-4.4775847E-3,1.0036436E-2,-7.0354226E-3,-2.4430495E-4,8.21182E-3,-5.5402173E-3,-1.4318193E-2,-7.8065377E-3,7.618257E-5,-3.8280047E-3,2.200268E-3,-1.0225036E-2,-2.8045815E-3,-3.1783194E-3,8.722383E-3,1.1937111E-2,1.9745636E-3,-2.8443402E-2,-1.4524818E-2,-1.4306455E-2,2.0005011E-3,-3.915002E-3,7.1228417E-3,4.4033905E-3,1.3090677E-2,1.3214813E-2,5.748519E-3,-3.863225E-3,6.874797E-3,1.0618069E-2,1.9370804E-2,1.0725289E-2,1.3095238E-3,2.2771774E-2,8.30186E-3,1.4012731E-2,2.4500596E-3,2.5767243E-2,1.11462595E-2,1.2385227E-2,4.5044473E-3,3.3687213E-3,1.4972754E-2,5.2264314E-3,-3.3933085E-3,2.0695899E-2,1.37465075E-2,1.9161822E-2,3.547527E-2,3.4951936E-2,5.430749E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,-1,83,85,87,89,91,93,95,-1,-1,97,99,101,103,-1,105,-1,-1,-1,107,109,111,113,115,117,119,121,123,125,-1,127,-1,129,131,133,135,137,139,-1,141,-1,143,145,147,149,151,-1,-1,153,-1,155,157,159,161,163,165,167,169,171,173,175,177,-1,-1,-1,179,181,-1,-1,-1,-1,183,185,-1,187,189,-1,-1,191,193,-1,-1,-1,195,-1,197,199,-1,201,203,205,207,209,211,-1,-1,213,215,217,-1,-1,219,221,223,225,-1,227,229,231,233,235,237,239,241,243,245,247,249,-1,251,253,-1,255,257,259,-1,261,263,265,267,269,-1,-1,-1,271,-1,-1,-1,273,-1,275,277,279,-1,281,-1,-1,-1,-1,-1,283,-1,-1,-1,-1,285,287,289,-1,291,-1,-1,-1,293,-1,295,297,-1,-1,-1,-1,-1,-1,299,-1,-1,-1,-1,301,303,305,-1,-1,-1,-1,307,309,311,-1,-1,313,315,317,319,321,323,325,-1,327,-1,-1,-1,-1,329,-1,331,333,335,-1,-1,-1,-1,-1,-1,337,-1,339,341,-1,343,-1,-1,345,347,-1,349,-1,-1,351,353,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.04569756E2,2.314756E1,4.4015804E1,1.1479485E1,1.1204187E1,1.375862E1,1.1074181E1,5.0630765E0,5.4977417E0,1.6835451E0,2.7292137E0,7.635728E0,4.7296906E0,1.1987007E1,1.0350852E0,1.7160017E0,0E0,3.365593E0,2.2785916E0,8.02474E-1,1.8139808E0,8.2092845E-1,4.2118216E-1,2.5969505E0,7.9878807E0,3.4506378E0,2.8059638E-1,3.5729208E0,4.9547577E0,1.2916136E-1,5.0370336E-1,4.6986675E-1,4.3831894E-1,3.3361893E0,3.1732607E-1,1.2025344E0,8.525562E-1,6.1548615E-1,2.7867746E-1,0E0,7.568896E-1,4.3300796E-1,3.2623726E-1,0E0,3.209715E-1,6.4999294E-1,7.8095305E-1,8.774853E-2,6.6699595E0,1.0430737E0,1.3737259E0,0E0,0E0,1.6121813E0,2.1142673E0,1.8222094E0,2.0173874E0,0E0,8.456135E-2,0E0,0E0,0E0,2.2948742E-1,2.0182319E-1,8.773762E-2,1.0454559E0,1.2400017E0,2.9302216E-1,5.4075003E-2,4.9354005E-1,4.749837E-1,4.0915275E-1,0E0,4.1006947E-1,0E0,1.8195415E-1,2.395466E-1,2.7992594E-1,6.4587474E-1,1.9061556E-1,3.2436007E-1,0E0,2.2312176E-1,0E0,3.2930946E-1,9.0832186E-1,5.243726E-1,3.2183218E-1,3.491568E-1,0E0,0E0,5.005799E0,0E0,7.9485786E-1,2.951727E-1,9.166851E-1,3.6940765E-1,1.5196973E-1,1.4996262E0,1.5459957E0,2.0756671E-1,3.3405304E-1,1.229186E0,2.790203E-1,1.9161606E0,0E0,0E0,0E0,1.3332486E-1,9.025889E-2,0E0,0E0,0E0,0E0,1.2105637E0,4.753914E-1,0E0,1.0230923E-1,1.3034093E-1,0E0,0E0,1.1720436E-1,2.0738435E-1,0E0,0E0,0E0,4.165104E-2,0E0,2.3347664E-1,2.4041843E-1,0E0,3.75216E-2,4.899945E-2,7.8049034E-2,1.18882775E-1,2.831366E-1,1.9596326E-1,0E0,0E0,4.0807243E-2,9.5264375E-2,5.5296913E-2,0E0,0E0,6.3498974E-2,2.439022E-1,7.7625823E-1,4.179001E-2,0E0,3.1940636E-1,2.950163E-1,2.1814555E-2,1.2469956E-1,3.0834332E0,1.5038118E0,7.341412E-1,3.6141193E-1,2.0956516E-2,2.2717905E-1,2.979199E-1,3.8433456E-1,0E0,1.5104294E-2,2.0498008E-1,0E0,2.1357828E-1,1.3384864E0,1.766758E0,0E0,8.13805E-2,2.7560636E-2,3.6830914E-1,2.4088049E-1,1.2501907E-1,0E0,0E0,0E0,1.6879807E0,0E0,0E0,0E0,2.5227994E-2,0E0,1.2438011E0,1.0674896E0,4.50325E-2,0E0,4.4387937E-2,0E0,0E0,0E0,0E0,0E0,2.2971368E-1,0E0,0E0,0E0,0E0,3.1587982E-1,1.362791E-1,8.2379565E-2,0E0,9.835899E-3,0E0,0E0,0E0,4.24771E-2,0E0,3.6796443E-2,1.104368E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0972798E-2,0E0,0E0,0E0,0E0,1.02915764E-1,9.625107E-3,5.0670695E-1,0E0,0E0,0E0,0E0,9.5994785E-2,8.202195E-2,1.9846934E-1,0E0,0E0,8.187277E-2,4.3335497E-2,3.119373E0,8.445473E-1,3.1632423E-2,6.5306467E-1,4.4786936E-1,0E0,2.7137452E-1,0E0,0E0,0E0,0E0,2.7990818E-2,0E0,2.0490988E-1,2.2245884E-1,1.0033834E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.7278566E-1,0E0,3.6907148E-1,7.8871536E-1,0E0,3.9031655E-2,0E0,0E0,1.8179357E-1,1.0577722E-1,0E0,1.116519E-1,0E0,0E0,1.5542393E0,1.1778164E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,58,58,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,73,73,75,75,76,76,77,77,78,78,79,79,80,80,82,82,84,84,85,85,86,86,87,87,88,88,91,91,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,108,108,109,109,114,114,115,115,117,117,118,118,121,121,122,122,126,126,128,128,129,129,131,131,132,132,133,133,134,134,135,135,136,136,139,139,140,140,141,141,144,144,145,145,146,146,147,147,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,160,160,162,162,163,163,165,165,166,166,167,167,169,169,170,170,171,171,172,172,173,173,177,177,181,181,183,183,184,184,185,185,187,187,193,193,198,198,199,199,200,200,202,202,206,206,208,208,209,209,216,216,221,221,222,222,223,223,228,228,229,229,230,230,233,233,234,234,235,235,236,236,237,237,238,238,239,239,241,241,246,246,248,248,249,249,250,250,257,257,259,259,260,260,262,262,265,265,266,266,268,268,271,271,272,272],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,-1,84,86,88,90,92,94,96,-1,-1,98,100,102,104,-1,106,-1,-1,-1,108,110,112,114,116,118,120,122,124,126,-1,128,-1,130,132,134,136,138,140,-1,142,-1,144,146,148,150,152,-1,-1,154,-1,156,158,160,162,164,166,168,170,172,174,176,178,-1,-1,-1,180,182,-1,-1,-1,-1,184,186,-1,188,190,-1,-1,192,194,-1,-1,-1,196,-1,198,200,-1,202,204,206,208,210,212,-1,-1,214,216,218,-1,-1,220,222,224,226,-1,228,230,232,234,236,238,240,242,244,246,248,250,-1,252,254,-1,256,258,260,-1,262,264,266,268,270,-1,-1,-1,272,-1,-1,-1,274,-1,276,278,280,-1,282,-1,-1,-1,-1,-1,284,-1,-1,-1,-1,286,288,290,-1,292,-1,-1,-1,294,-1,296,298,-1,-1,-1,-1,-1,-1,300,-1,-1,-1,-1,302,304,306,-1,-1,-1,-1,308,310,312,-1,-1,314,316,318,320,322,324,326,-1,328,-1,-1,-1,-1,330,-1,332,334,336,-1,-1,-1,-1,-1,-1,338,-1,340,342,-1,344,-1,-1,346,348,-1,350,-1,-1,352,354,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,9.772148E4,1.0119178E10,4.995702E8,8.0708375E5,6.747114E7,2.0601392E3,2.116711E3,3.3926086E2,1.1210787E3,6.76E2,3E0,1.1410706E3,2.0676695E-1,2.368E0,3.762726E-2,4.214286E0,1.3521131E8,1.8329959E-1,5.2143492E-5,1.6394367E0,1.4008022E11,7.805608E4,8E0,9.99E2,1.3715873E3,1.7324902E8,2.3421426E-1,1.7495675E5,2.3276744E5,2.7539404E-6,1.3983857E10,7.6599895E9,1.568E3,8.015419E3,5.784E4,8.982707E9,7.84892E5,-3.365478E-2,1.10597E5,5.7E1,2.1924436E5,5.2036587E-3,9.397667E6,1.2E1,1.325204E6,3.833E3,8.791777E3,4.1973075E6,4.391553E6,-2.4013897E-2,-7.1775946E-3,1.5988182E-1,1.5572634E6,5.57E4,3.4526646E1,-2.079398E-2,1.6347875E3,-6.080276E-3,1.4745017E-2,-1.2471662E-3,1.1843003E0,2.1262943E1,2.5269838E6,1.5804776E4,4.099E3,1.401805E1,5.748E3,3.9382784E9,1.6904226E7,2.1384616E0,-4.2185403E-2,1E0,-5.9068925E-4,1.3046789E6,6.6911E4,4.7937644E-1,2.7565938E3,4.1401203E5,7E0,-6.236405E-4,3.904E3,3.2354023E-2,2.7067092E3,4.7E2,1.4E1,1.7E1,1.43709E0,1.943138E-2,3.946654E-2,4.719849E7,5.729111E-2,1E0,5.894636E6,5.033165E7,1E0,5.61E2,5.08E2,1.5846036E7,8.985661E8,1.2204E4,4.42384E5,7.21E3,1.6828056E1,-1.3736463E-2,-3.8212107E-3,-7.128201E-3,9.27E2,3.3743455E0,1.2433028E-2,-1.2798878E-2,-2.464853E-3,-3.6819406E-2,4.7E2,8.965631E7,-2.1277664E-2,3.5188188E5,1.2E1,-2.6236763E-2,-1.3515623E-2,4.0895187E8,2.649435E5,1.593118E-2,-4.2598057E-3,-3.0468455E-2,1.6081998E5,-3.2811306E-2,1.074E3,3.8E1,-2.7789155E-3,1.3617021E0,2.183E3,2.43406E5,2.9810298E-2,2.194E3,1.0800328E8,8.0682345E-5,1.2865992E-2,3.624992E5,4.9008E4,1.1660184E3,1.60452E-2,7.284325E-3,9.8163464E7,1.4925373E-1,2.4114338E8,3.5029E4,-3.216593E-2,9E0,3.5E2,7.1047956E-1,2.05156E5,1.2817779E7,1E0,3.4E1,2.1E1,5.3379045E6,7.358598E6,1.7446084E0,1.6E1,1.4862201E-2,1.77E2,1.0303075E1,-2.739853E-3,4.6E0,1.1160287E7,2.9686664E7,-4.990738E-3,1.3498145E5,1.6840342E8,6.852143E1,1.0057E4,4.1667834E-2,9.904823E-4,2.9015813E-2,5.002569E-2,1.7730331E6,4.8729997E-2,-1.0710474E-2,-2.58661E-2,1.5E1,-3.6084086E-3,4.179E3,3.1294638E2,2.316491E-2,-5.8664985E-2,5.507653E6,-2.1293333E-2,-1.1745214E-2,-1.7924649E-3,3.3625897E-3,-6.560609E-3,1.8489E4,-3.7174274E-3,-4.7916723E-3,-1.3425117E-2,-2.0578215E-2,4.7905406E-1,2.695E3,4.762529E-1,-3.6558046E-3,2.6883545E2,-2.0582373E-3,4.2510238E-3,2.0512035E-3,1.558E3,3.2664845E-3,4.925824E8,2.1E1,2.431387E-3,-2.1951473E-3,1.2185075E-2,-2.1859428E-3,3.971273E-3,-3.7649826E-3,5.012E3,7.994347E-3,4.972213E-4,1.6375402E-2,2.8642936E-2,2.6E1,1.1E1,8.490231E-1,-2.6313903E-2,-7.366748E-3,-1.707317E-2,-1.1098916E-2,3.192E3,2.7913043E0,4.1032645E6,3.991864E-3,9.273184E-3,5.862E3,4.3467668E4,2.0967E4,2.9206951E1,1.527638E12,7.431E3,7.79E2,-1.6510546E-2,4.83368E5,1.8797977E-2,1.35102235E-2,2.4074456E-2,-1.166818E-3,3.38464E5,1.5005295E-2,1.2868687E1,1.21E3,6.6992125E4,1.879202E-2,3.3556685E-2,-4.616203E-3,-1.9575968E-2,-3.5246909E-3,-1.9036952E-2,7E0,-1.672819E-2,3.5636363E0,2E0,1.2854133E-3,3.8343322E6,-5.742292E-3,-4.0699757E-4,1.7725672E8,7.160724E1,3.0057877E-3,9.7888E4,1.6965777E-2,3.296467E-2,1.0746194E4,2.0746333E2,-1.7835952E-4,4.3605603E-3,-2.1759437E-2,5.7427E-3,-3.363548E-2,-2.1462262E-2,-4.2965278E-2,-2.6169393E-2,-5.7323948E-3,-1.4281144E-2,-1.0038699E-2,-1.8971154E-2,-3.4114014E-4,-1.491505E-2,-1.2394227E-2,-2.1870537E-2,-8.138765E-3,5.205708E-4,-1.0632323E-2,-5.0851647E-3,1.2926117E-2,5.5298777E-3,-7.7699386E-3,-1.2167741E-3,-1.5402019E-2,-8.264758E-3,-1.6685007E-2,-8.712472E-3,-1.0413487E-2,-1.9868547E-2,-1.0119685E-2,-4.4775847E-3,1.0036436E-2,-7.0354226E-3,-2.4430495E-4,8.21182E-3,-5.5402173E-3,-1.4318193E-2,-7.8065377E-3,7.618257E-5,-3.8280047E-3,2.200268E-3,-1.0225036E-2,-2.8045815E-3,-3.1783194E-3,8.722383E-3,1.1937111E-2,1.9745636E-3,-2.8443402E-2,-1.4524818E-2,-1.4306455E-2,2.0005011E-3,-3.915002E-3,7.1228417E-3,4.4033905E-3,1.3090677E-2,1.3214813E-2,5.748519E-3,-3.863225E-3,6.874797E-3,1.0618069E-2,1.9370804E-2,1.0725289E-2,1.3095238E-3,2.2771774E-2,8.30186E-3,1.4012731E-2,2.4500596E-3,2.5767243E-2,1.11462595E-2,1.2385227E-2,4.5044473E-3,3.3687213E-3,1.4972754E-2,5.2264314E-3,-3.3933085E-3,2.0695899E-2,1.37465075E-2,1.9161822E-2,3.547527E-2,3.4951936E-2,5.430749E-3],"split_indices":[20,102,33,19,12,28,45,52,4,52,52,2,11,52,39,58,0,54,45,41,42,42,31,28,18,2,52,7,38,33,33,39,5,5,0,4,10,5,9,0,29,8,28,0,45,8,9,1,52,28,28,0,0,38,48,2,58,0,4,0,0,0,54,58,32,51,0,58,2,5,47,53,0,67,0,33,29,27,4,33,3,0,2,0,52,1,3,0,53,0,0,45,0,102,45,7,102,10,2,47,5,2,2,2,35,0,0,0,2,54,0,0,0,0,0,45,0,48,18,0,0,7,33,0,0,0,28,0,2,3,0,53,2,1,57,0,7,0,0,47,9,4,0,0,32,56,31,1,0,0,2,56,12,9,102,3,3,45,45,53,3,0,0,58,0,54,45,12,0,33,32,58,9,38,0,0,0,28,0,0,0,3,0,2,52,38,0,45,0,0,0,0,0,10,0,0,0,0,56,2,27,0,4,0,0,0,10,0,5,3,0,0,0,0,0,0,2,0,0,0,0,29,0,53,0,0,0,0,1,54,45,0,0,1,33,10,58,31,2,2,0,12,0,0,0,0,9,0,58,2,33,0,0,0,0,0,0,3,0,54,6,0,45,0,0,12,56,0,11,0,0,4,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.572E3,8.29E2,1.743E3,4.52E2,3.77E2,1.117E3,6.26E2,6.4E1,3.88E2,2.78E2,9.9E1,8.13E2,3.04E2,5.93E2,3.3E1,5.8E1,6E0,3.04E2,8.4E1,1.97E2,8.1E1,6.4E1,3.5E1,2.59E2,5.54E2,2.91E2,1.3E1,2.66E2,3.27E2,2.3E1,1E1,2.4E1,3.4E1,2.49E2,5.5E1,6.4E1,2E1,1.02E2,9.5E1,4E0,7.7E1,3.5E1,2.9E1,5E0,3E1,1.21E2,1.38E2,1.5E1,5.39E2,1.53E2,1.38E2,8E0,5E0,1.2E2,1.46E2,1.18E2,2.09E2,1E1,1.3E1,6E0,4E0,4E0,2E1,2.5E1,9E0,2.01E2,4.8E1,3.9E1,1.6E1,5.4E1,1E1,1.3E1,7E0,9.6E1,6E0,6.2E1,3.3E1,3.3E1,4.4E1,1.1E1,2.4E1,7E0,2.2E1,7E0,2.3E1,1.03E2,1.8E1,9.4E1,4.4E1,4E0,1.1E1,5.35E2,4E0,1.2E2,3.3E1,1.1E2,2.8E1,1.6E1,1.04E2,1.22E2,2.4E1,8.6E1,3.2E1,2.4E1,1.85E2,9E0,4E0,4E0,1.6E1,2.1E1,4E0,5E0,4E0,3.4E1,1.67E2,4E1,8E0,2.6E1,1.3E1,1.1E1,5E0,1E1,4.4E1,5E0,5E0,4E0,9E0,4E0,9.2E1,5.6E1,6E0,2.2E1,1.1E1,1.8E1,1.5E1,3.6E1,8E0,7E0,4E0,9E0,1.5E1,1E1,1.2E1,7E0,1.6E1,5.7E1,4.6E1,9E0,9E0,1.9E1,7.5E1,1.3E1,3.1E1,4.91E2,4.4E1,4.1E1,7.9E1,1.3E1,2E1,2E1,9E1,8E0,2E1,1E1,6E0,9E0,9.5E1,1.16E2,6E0,1.5E1,9E0,3.5E1,5.1E1,2.8E1,4E0,8E0,1.6E1,1.74E2,1.1E1,4E0,1.2E1,1.5E1,6E0,6.5E1,1.02E2,2.7E1,1.3E1,1.5E1,1.1E1,6E0,7E0,5E0,5E0,3.9E1,5E0,4E0,5E0,2.7E1,6.5E1,4.5E1,1.1E1,5E0,1.7E1,7E0,4E0,6E0,1.2E1,6E0,9E0,3.2E1,4E0,4E0,4E0,4E0,5E0,5E0,1E1,6E0,4E0,1.1E1,5E0,4.8E1,9E0,4.2E1,4E0,5E0,4E0,5E0,1.4E1,2.7E1,4.8E1,6E0,7E0,2.2E1,9E0,3.94E2,9.7E1,2E1,2.4E1,3.5E1,6E0,7.4E1,5E0,7E0,6E0,4E0,1.6E1,4E0,1.6E1,7.4E1,1.6E1,5E0,1.5E1,5E0,5E0,5E0,4E0,9.1E1,4E0,2.8E1,8.8E1,6E0,9E0,4E0,5E0,2.2E1,1.3E1,4E0,4.7E1,5E0,2.3E1,1.1E2,6.4E1,4E0,1.1E1,6.1E1,4E0,6.6E1,3.6E1,2E1,7E0,4E0,1.1E1,1.7E1,2.2E1,4E0,6.1E1,3.9E1,6E0,7E0,4E0,1.3E1,4E0,5E0,7E0,5E0,4E0,1.1E1,2.1E1,4E0,6E0,7E0,4.1E1,5E0,4E0,4E0,3.8E1,1E1,4E0,5E0,2.2E1,3.8E1,1E1,8E0,1.4E1,4E0,5E0,3.29E2,6.5E1,6.5E1,3.2E1,1.6E1,4E0,1.4E1,1E1,1.9E1,1.6E1,6.4E1,1E1,1.2E1,4E0,9E0,7E0,1.3E1,6.1E1,1.2E1,4E0,1E1,8.1E1,1.5E1,1.3E1,7.5E1,1.3E1,4E0,5E0,5E0,1.7E1,8E0,5E0,1.7E1,3E1,9.1E1,1.9E1,6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"355","size_leaf_vector":"1"}},{"base_weights":[3.0482307E-3,-2.687524E-1,1.348427E-1,-4.4551784E-1,-1.2587185E-1,3.2604385E-2,3.599687E-1,-5.188257E-1,-2.6318774E-1,5.9669062E-2,-1.4831169E-1,-1.03233255E-1,1.2835363E-1,-5.7358004E-2,4.0290835E-1,-1.0635083E0,-4.9378598E-1,-3.606299E-1,-1.6454731E-1,-2.701948E-1,4.9725663E-2,-2.1791486E-1,9.3958713E-4,1.5504558E-1,-2.785826E-1,-1.7930242E-1,1.8294272E-1,3.2157642E-1,5.507215E-1,-3.1765044E-2,-6.565822E-2,-5.257633E-1,-2.7319577E-1,-3.8321012E-1,-2.9073397E-3,-2.2124824E-1,-8.312017E-2,-3.915404E-1,-1.5594657E-1,-1.4359759E-1,1.3064954E-1,2.3993987E-1,-2.3695694E-1,4.1452758E-2,-3.168886E-2,5.64315E-2,2.4774505E-1,-3.474345E-1,6.744621E-3,-2.3121981E-1,5.504737E-3,1.6768035E-2,1.735114E-2,2.5189632E-1,5.791725E-1,3.1709865E-1,6.496143E-1,-4.9417195E-1,-8.633644E-1,7.819521E-3,-3.3368793E-1,-3.2696408E-1,-2.6106108E-2,-1.8062435E-1,-1.7305132E-2,4.204299E-3,-1.18986405E-1,-4.3999642E-1,7.2530836E-2,-1.7815158E-1,1.614525E-2,2.1834206E-2,-2.8125957E-1,-5.279419E-4,3.438239E-1,-5.1136906E-3,2.4243068E-2,-2.869457E-1,-7.359345E-2,-6.774263E-2,3.0127338E-1,-8.306257E-2,1.0327134E-1,2.2052532E-1,6.546131E-1,-4.2531773E-1,-1.1760131E-1,-2.8532226E-2,-1.5697955E-1,-3.6942777E-3,4.4749696E-3,2.6703724E-1,-1.2534242E-2,8.407393E-1,4.17392E-1,-6.0527795E-3,3.744017E-1,7.165353E-1,3.8761324E-1,-5.398881E-1,-3.6879382E-1,-4.6749312E-2,-2.4403911E-2,-3.7444142E-1,-5.0703594E-3,-4.1020417E-1,-1.6643299E-1,-9.1449596E-2,-2.1335848E-1,-1.6351418E-3,-1.597798E-1,-2.9235768E-1,-5.364927E-1,2.202085E-2,-1.1881223E-1,-8.672069E-2,-2.4819387E-1,-5.031211E-2,1.6013412E-2,-2.166998E-1,-2.7372068E-2,-1.2763251E-1,5.076913E-2,9.973319E-2,3.975953E-1,-2.5669014E-1,-4.5821142E-1,1.2821291E-1,-1.21288486E-1,-5.2906744E-2,-2.6622914E-2,3.4802416E-1,4.758085E-3,-1.4523892E-1,6.450908E-2,1.4463864E-1,-3.1169849E-2,1.5803541E-1,3.3525848E-1,8.0871403E-1,1.795243E-2,-2.0470499E-1,-5.033276E-1,-1.0714398E-2,-5.8535923E-5,-1.1888483E-2,-2.4965072E-2,1.3466375E-1,3.121034E-1,1.7554117E-2,4.7244944E-2,3.2896575E-1,3.4234393E-2,4.8502126E-1,1.6309012E-1,6.329043E-1,8.640189E-1,4.3173242E-1,5.353547E-3,-4.6164986E-1,-6.336548E-1,-1.3310418E-1,-4.7741878E-1,-4.31301E-1,-5.6475294E-3,-2.2171892E-2,-1.1495922E-2,-3.566278E-3,-1.2026413E-2,-7.4169924E-3,-8.079836E-4,-4.124984E-3,-1.21287005E-2,-3.5924597E-3,-1.00240605E-2,-2.7731553E-2,-3.6267856E-1,-6.191748E-1,-3.0915964E-1,6.0603383E-4,-1.2600544E-2,-1.9641224E-1,5.2438505E-2,-3.3316696E-1,-1.3272287E-1,4.296673E-3,-1.1056598E-1,-2.6241392E-1,-3.081017E-3,-2.2703837E-1,2.4298927E-2,-1.6703721E-2,1.4424029E-1,-1.2713015E-3,1.1492232E-2,3.141615E-1,5.1014525E-1,-2.7919468E-1,-9.532564E-2,-4.2888414E-2,-3.152621E-1,3.437202E-3,8.307814E-3,-1.5096079E-1,4.8338566E-3,1.0409711E-1,-7.678987E-2,1.9005183E-2,8.741842E-3,-4.996752E-2,-2.1101968E-1,1.4091362E-1,-4.554339E-2,-1.2128072E-1,1.5727141E-1,6.767328E-2,-1.8452562E-1,2.0359813E-1,2.828865E-2,1.94353E-1,3.751342E-1,5.0121542E-2,2.8073048E-2,-4.9164672E-3,-1.3494699E-2,-2.968467E-2,-1.4636416E-2,1.8121322E-3,-4.0990715E-3,3.174185E-1,5.2277684E-2,4.407545E-1,2.4836396E-1,3.729928E-1,-4.8943533E-4,9.214648E-3,2.747117E-2,2.4611792E-1,-2.1977904E-3,6.779365E-1,3.5020638E-1,4.9632303E-2,3.1321064E-2,2.73156E-1,2.769808E-2,-1.4613854E-3,-2.4103222E-2,-2.7227031E-2,-3.7436865E-2,-1.1879001E-2,-2.1753695E-3,-2.6019217E-2,-9.935373E-3,-1.4371253E-2,-2.6770942E-2,3.953829E-3,-7.384107E-3,-8.532653E-3,-2.1795548E-2,-3.421869E-2,-1.6604474E-2,-1.9142497E-2,-7.98377E-3,-1.1238199E-2,-2.3086243E-3,5.520535E-4,1.0455976E-2,-1.3095238E-2,-2.3830736E-2,1.2953117E-4,-1.2047197E-2,-9.512011E-3,-8.466396E-4,-1.4821097E-2,-5.3621163E-3,-6.1558033E-3,-1.4466285E-2,-2.3087724E-3,5.1987995E-3,-4.9214275E-3,1.6182344E-3,1.0937071E-2,3.209854E-3,1.8755993E-2,9.234421E-3,2.8658208E-2,1.3422716E-2,-1.2841588E-2,-2.2379113E-2,1.7872568E-3,-1.7183384E-2,-5.1687984E-3,-1.8935943E-2,-2.4031552E-3,-1.0869444E-2,1.11056045E-2,2.549597E-3,-8.331542E-3,-1.3094775E-3,3.303713E-4,-8.545555E-3,-8.4573245E-3,-1.7384173E-2,8.5652847E-4,9.212011E-3,-4.7950945E-3,6.230668E-4,-1.4465523E-2,3.557534E-3,6.1807586E-3,1.367888E-2,8.409209E-3,-2.1135702E-3,-2.7993629E-3,-1.3943847E-2,1.5493931E-2,8.267089E-3,-1.3537231E-3,9.811822E-3,3.8771892E-3,1.347971E-2,2.709524E-2,1.648066E-2,5.2916827E-3,1.9030452E-2,6.7876774E-4,1.328683E-2,2.8026866E-2,1.5893841E-2,-6.6772285E-3,1.3419921E-2,2.499966E-2,9.125681E-3,6.2592044E-3,1.42859975E-2,3.5182282E-2,1.3757242E-2,9.460327E-3,2.2620523E-2,1.6941521E-2,7.5322804E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,59,61,-1,63,65,67,69,71,73,75,77,-1,79,81,83,85,-1,87,-1,89,-1,91,93,95,97,99,101,-1,103,105,-1,107,-1,-1,109,111,113,115,-1,117,119,121,123,-1,-1,125,127,129,131,133,135,137,139,141,143,-1,145,-1,-1,147,-1,149,151,-1,153,155,157,159,161,-1,-1,163,-1,165,167,169,171,-1,173,175,177,-1,179,181,183,185,-1,187,-1,189,191,193,195,197,199,201,203,205,-1,207,-1,209,211,213,215,217,219,221,-1,223,225,-1,-1,-1,227,229,231,-1,-1,233,-1,235,237,239,241,243,-1,245,247,249,251,253,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,255,257,259,261,-1,-1,263,265,267,269,-1,271,273,-1,275,277,279,281,-1,-1,283,285,287,289,-1,291,-1,-1,293,-1,295,297,-1,-1,299,301,303,305,307,309,311,313,315,317,319,321,-1,-1,-1,-1,-1,-1,-1,-1,323,325,327,329,331,-1,-1,-1,333,-1,335,337,-1,-1,339,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.335207E1,2.1437073E1,4.038222E1,4.9426193E0,1.4158546E1,1.5724923E1,9.843193E0,3.1573029E0,1.0075617E0,0E0,1.1237331E1,5.9735594E0,7.744297E0,1.5565754E0,5.797798E0,3.4354115E-1,1.6903725E0,3.5331202E-1,2.4477851E-1,3.931635E0,2.8034759E0,2.1318579E0,7.1373286E0,6.0742397E0,1.2997856E0,5.5147254E-1,4.856568E-1,5.651741E0,3.8371391E0,0E0,0E0,1.9837036E0,9.6197176E-1,2.2864628E-1,0E0,1.05131626E-1,1.5998843E-1,3.1576118E0,1.6532178E0,1.2131455E0,3.5236046E0,8.89084E-1,1.8489752E0,0E0,3.0695632E0,2.125555E0,3.6543121E0,6.301875E-1,0E0,6.671779E-1,0E0,7.2645135E-2,0E0,2.0156784E0,2.5266E0,1.4014115E0,1.8613319E0,9.9884033E-1,3.0652046E-2,0E0,2.5537205E-1,4.5287085E-1,0E0,5.9302986E-2,0E0,0E0,6.8668365E-2,1.6382217E0,1.0386434E0,9.006562E-1,0E0,5.5478233E-1,3.803289E-1,5.2156717E-1,5.903516E-1,0E0,0E0,7.891855E-1,5.4493463E-1,1.5587506E0,2.0768213E-1,7.6175743E-1,1.3559453E0,2.2628183E0,6.651459E-1,3.788457E-1,1.2376116E-1,0E0,2.719066E-1,0E0,0E0,1.4433975E0,0E0,9.499836E-1,8.1637E-1,0E0,1.0429077E0,6.3724136E-1,2.8784513E-1,8.152275E-1,1.4469595E0,0E0,0E0,3.3771515E-1,0E0,4.890299E-2,8.870238E-2,3.6210574E-2,5.297166E-2,0E0,3.581819E-2,9.575968E-1,1.2183037E0,0E0,1.7769039E-1,9.8723316E-1,7.505474E-1,1.8197599E-1,0E0,1.6100526E-1,0E0,3.5288295E-1,3.6189502E-1,1.689123E-1,2.0596075E-1,5.2173233E-1,1.1593523E0,1.343675E-2,3.0572975E-1,8.4793586E-1,0E0,4.537344E-2,0E0,3.5294342E-1,2.1663037E-1,6.439042E-1,8.91661E-1,1.241591E0,5.6200695E-1,6.028843E-2,0E0,3.597054E-2,2.0791054E-1,0E0,0E0,0E0,4.141544E-2,9.620749E-1,1.4075603E0,0E0,0E0,5.256665E-1,0E0,5.021739E-1,3.2029954E-1,5.900669E-1,3.5862923E-1,2.957301E-1,0E0,7.5699806E-1,2.687664E-1,1.7708695E-1,3.7909794E-1,1.611414E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6563644E-1,5.0401974E-1,7.894707E-1,1.8179512E-1,0E0,0E0,1.4373171E-1,1.8592158E-1,3.451233E-1,5.285904E-1,0E0,1.0884142E-1,6.896949E-2,0E0,5.5163383E-2,6.1032902E-2,1.3970311E-1,1.292617E-1,0E0,0E0,1.2856007E-1,7.560849E-2,3.790245E-1,6.6402805E-1,0E0,2.3069203E-1,0E0,0E0,2.6346105E-1,0E0,1.7679062E-1,8.82887E-1,0E0,0E0,1.7236483E-1,1.3548362E-1,7.75069E-2,3.337343E-2,3.1202596E-1,6.622882E-1,4.05488E-1,2.6377302E-1,5.823779E-1,5.265903E-1,2.1682799E-1,4.7984314E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2805333E-1,3.766368E-1,7.059269E-1,1.0088005E0,6.6003084E-1,0E0,0E0,0E0,2.6701152E-2,0E0,3.8581848E-1,9.5682144E-2,0E0,0E0,4.5690954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,49,49,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,63,63,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,91,91,93,93,94,94,96,96,97,97,98,98,99,99,100,100,103,103,105,105,106,106,107,107,108,108,110,110,111,111,112,112,114,114,115,115,116,116,117,117,119,119,121,121,122,122,123,123,124,124,125,125,126,126,127,127,128,128,129,129,131,131,133,133,134,134,135,135,136,136,137,137,138,138,139,139,141,141,142,142,146,146,147,147,148,148,151,151,153,153,154,154,155,155,156,156,157,157,159,159,160,160,161,161,162,162,163,163,175,175,176,176,177,177,178,178,181,181,182,182,183,183,184,184,186,186,187,187,189,189,190,190,191,191,192,192,195,195,196,196,197,197,198,198,200,200,203,203,205,205,206,206,209,209,210,210,211,211,212,212,213,213,214,214,215,215,216,216,217,217,218,218,219,219,220,220,229,229,230,230,231,231,232,232,233,233,237,237,239,239,240,240,243,243],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,60,62,-1,64,66,68,70,72,74,76,78,-1,80,82,84,86,-1,88,-1,90,-1,92,94,96,98,100,102,-1,104,106,-1,108,-1,-1,110,112,114,116,-1,118,120,122,124,-1,-1,126,128,130,132,134,136,138,140,142,144,-1,146,-1,-1,148,-1,150,152,-1,154,156,158,160,162,-1,-1,164,-1,166,168,170,172,-1,174,176,178,-1,180,182,184,186,-1,188,-1,190,192,194,196,198,200,202,204,206,-1,208,-1,210,212,214,216,218,220,222,-1,224,226,-1,-1,-1,228,230,232,-1,-1,234,-1,236,238,240,242,244,-1,246,248,250,252,254,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,256,258,260,262,-1,-1,264,266,268,270,-1,272,274,-1,276,278,280,282,-1,-1,284,286,288,290,-1,292,-1,-1,294,-1,296,298,-1,-1,300,302,304,306,308,310,312,314,316,318,320,322,-1,-1,-1,-1,-1,-1,-1,-1,324,326,328,330,332,-1,-1,-1,334,-1,336,338,-1,-1,340,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.4803147E0,3.443721E6,1E0,2.5E1,1.7669278E5,3.4453398E-1,3E0,2.3859661E-2,5.9669062E-2,5.057096E-1,2.41E3,3.956147E-1,2.8E1,1.2974394E6,8.229907E7,5.121E3,5.501614E9,2.877528E2,1E0,1E0,6E0,2.04E5,9.6E2,3.0795444E3,1.8137958E6,2.2580828E-1,6.082581E3,3.904E3,-3.1765044E-2,-6.565822E-2,6.496696E7,1.6527965E7,2.67162E5,-2.9073397E-3,2.3428571E0,1.822542E0,4.6726016E3,1.1392E4,3.3517068E7,1.51529E3,2.127E3,1.445113E6,4.1452758E-2,1.4777102E9,4.67E2,1.3664E4,2.6476662E6,6.744621E-3,1.44657E5,5.504737E-3,2.8122288E3,1.735114E-2,4.8297736E7,1.1336898E-1,1.3E0,2.3977574E7,4.4866666E2,1.5595E4,7.819521E-3,2.203288E2,1.11E3,-2.6106108E-2,1.0579121E6,-1.7305132E-2,4.204299E-3,2.8945708E6,2.9166096E-1,4.784E3,1.3304372E7,1.614525E-2,2.8541E4,1.4299594E8,1.1590965E0,3.767388E6,-5.1136906E-3,2.4243068E-2,9.837297E0,1.5555556E0,1E0,2.6185484E1,8.594445E5,2E0,1.546644E6,2.2686274E1,4.4E1,7.006513E4,-2.8532226E-2,2.0676695E-1,-3.6942777E-3,4.4749696E-3,9.923929E2,-1.2534242E-2,6.3039363E4,3.3968747E-1,-6.0527795E-3,1E0,6.626653E8,2.2644286E2,6.8503425E6,1.74123E5,-4.6749312E-2,-2.4403911E-2,1.1896622E6,-5.0703594E-3,2.23329E5,2.53692E5,1.96E2,2.62E2,-1.6351418E-3,1.9711539E0,1.7026253E8,4.888E0,2.202085E-2,1.1171382E7,2.9348E2,1.0506024E1,1.460806E6,1.6013412E-2,1.9333144E7,-2.7372068E-2,4.459525E5,2.0783027E5,1.7231706E-1,7.3963095E9,1.0573252E8,2E0,1.3781E2,1.060324E6,5.511673E0,-2.6622914E-2,1.3152658E7,4.758085E-3,3.1E1,1.3170732E0,1.198E5,4.274E3,2.2E1,5.1586456E7,1.5736016E6,1.795243E-2,8.5959184E-1,1.3849624E1,-1.0714398E-2,-5.8535923E-5,-1.1888483E-2,3.881848E-1,3.1753032E0,6.3697716E1,1.7554117E-2,4.7244944E-2,6.13912E5,3.4234393E-2,4.8237036E2,1.4E1,3.1615953E1,8.182648E6,1.0052E4,5.353547E-3,3E0,1.8949672E0,1.9217945E0,3.4E1,1.7735592E3,-5.6475294E-3,-2.2171892E-2,-1.1495922E-2,-3.566278E-3,-1.2026413E-2,-7.4169924E-3,-8.079836E-4,-4.124984E-3,-1.21287005E-2,-3.5924597E-3,-1.00240605E-2,2.7324794E11,1.857051E5,1.0052E4,1.42E1,6.0603383E-4,-1.2600544E-2,1.1394967E7,5.4E1,5.408015E0,2.1688406E1,4.296673E-3,9.7637427E-1,1.56E2,-3.081017E-3,1.7E1,5.4878284E2,4.4844016E2,3.8434E4,-1.2713015E-3,1.1492232E-2,3E1,3.07125E0,7.2419E4,4.3360384E7,-4.2888414E-2,1.3E1,3.437202E-3,8.307814E-3,2.8800002E-1,4.8338566E-3,7.85939E5,4.759E3,1.9005183E-2,8.741842E-3,8.749618E5,6.285983E5,7.8E3,1E0,1.8523488E2,8.2253466E9,1E0,8E0,1.3878379E0,1.2905E4,2.065904E6,1.125584E-3,5.0121542E-2,2.8073048E-2,-4.9164672E-3,-1.3494699E-2,-2.968467E-2,-1.4636416E-2,1.8121322E-3,-4.0990715E-3,1.1280869E9,2.8729E4,4.8718534E0,2.6956522E0,3.5296965E-1,-4.8943533E-4,9.214648E-3,2.747117E-2,2.5239267E5,-2.1977904E-3,1E0,1.9125667E3,4.9632303E-2,3.1321064E-2,1.7440945E0,2.769808E-2,-1.4613854E-3,-2.4103222E-2,-2.7227031E-2,-3.7436865E-2,-1.1879001E-2,-2.1753695E-3,-2.6019217E-2,-9.935373E-3,-1.4371253E-2,-2.6770942E-2,3.953829E-3,-7.384107E-3,-8.532653E-3,-2.1795548E-2,-3.421869E-2,-1.6604474E-2,-1.9142497E-2,-7.98377E-3,-1.1238199E-2,-2.3086243E-3,5.520535E-4,1.0455976E-2,-1.3095238E-2,-2.3830736E-2,1.2953117E-4,-1.2047197E-2,-9.512011E-3,-8.466396E-4,-1.4821097E-2,-5.3621163E-3,-6.1558033E-3,-1.4466285E-2,-2.3087724E-3,5.1987995E-3,-4.9214275E-3,1.6182344E-3,1.0937071E-2,3.209854E-3,1.8755993E-2,9.234421E-3,2.8658208E-2,1.3422716E-2,-1.2841588E-2,-2.2379113E-2,1.7872568E-3,-1.7183384E-2,-5.1687984E-3,-1.8935943E-2,-2.4031552E-3,-1.0869444E-2,1.11056045E-2,2.549597E-3,-8.331542E-3,-1.3094775E-3,3.303713E-4,-8.545555E-3,-8.4573245E-3,-1.7384173E-2,8.5652847E-4,9.212011E-3,-4.7950945E-3,6.230668E-4,-1.4465523E-2,3.557534E-3,6.1807586E-3,1.367888E-2,8.409209E-3,-2.1135702E-3,-2.7993629E-3,-1.3943847E-2,1.5493931E-2,8.267089E-3,-1.3537231E-3,9.811822E-3,3.8771892E-3,1.347971E-2,2.709524E-2,1.648066E-2,5.2916827E-3,1.9030452E-2,6.7876774E-4,1.328683E-2,2.8026866E-2,1.5893841E-2,-6.6772285E-3,1.3419921E-2,2.499966E-2,9.125681E-3,6.2592044E-3,1.42859975E-2,3.5182282E-2,1.3757242E-2,9.460327E-3,2.2620523E-2,1.6941521E-2,7.5322804E-3],"split_indices":[20,56,50,102,0,28,42,3,38,0,39,2,27,8,28,45,2,5,4,102,102,3,5,2,4,28,42,52,2,0,0,45,45,1,0,56,58,4,2,51,4,1,9,0,7,2,2,32,0,1,0,4,0,45,57,54,47,4,9,0,57,2,0,45,0,0,45,27,10,45,0,2,45,42,32,0,0,54,53,6,58,28,17,28,58,10,33,0,39,0,0,52,0,28,38,0,6,7,58,45,29,0,0,28,0,12,12,0,11,0,53,7,54,0,32,52,58,47,0,32,0,28,33,38,5,7,17,52,51,53,0,45,0,10,53,7,29,3,7,50,0,27,54,0,0,0,27,54,58,0,0,2,0,4,8,56,45,2,0,11,58,53,3,4,0,0,0,0,0,0,0,0,0,0,0,31,28,2,58,0,0,45,3,54,58,0,42,8,0,3,52,52,9,0,0,3,57,1,5,0,0,0,0,41,0,1,2,0,0,45,28,1,8,4,31,16,8,53,10,28,38,0,0,0,0,0,0,0,0,31,10,53,54,27,0,0,0,33,0,6,4,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.604E3,8.5E2,1.754E3,3.79E2,4.71E2,1.207E3,5.47E2,2.69E2,1.1E2,7E0,4.64E2,4.99E2,7.08E2,5.1E1,4.96E2,1E1,2.59E2,5.4E1,5.6E1,2.87E2,1.77E2,2.37E2,2.62E2,6.65E2,4.3E1,3.4E1,1.7E1,3.22E2,1.74E2,5E0,5E0,2.25E2,3.4E1,5E1,4E0,3.2E1,2.4E1,1.38E2,1.49E2,5.2E1,1.25E2,9E0,2.28E2,9E0,2.53E2,3.23E2,3.42E2,3.7E1,6E0,2.9E1,5E0,9E0,8E0,2.55E2,6.7E1,5.3E1,1.21E2,2.08E2,1.7E1,4E0,3E1,3.8E1,1.2E1,2.6E1,6E0,4E0,2E1,1.25E2,1.3E1,1.43E2,6E0,2.4E1,2.8E1,7.8E1,4.7E1,4E0,5E0,1.74E2,5.4E1,2.29E2,2.4E1,8.1E1,2.42E2,3.22E2,2E1,2.7E1,1E1,4E0,2.5E1,4E0,5E0,2.48E2,7E0,2.4E1,4.3E1,6E0,4.7E1,9.5E1,2.6E1,1.5E2,5.8E1,1.3E1,4E0,2.5E1,5E0,2.4E1,1.4E1,8E0,1.8E1,7E0,1.3E1,5.1E1,7.4E1,4E0,9E0,6.3E1,8E1,2E1,4E0,2.4E1,4E0,2.2E1,5.6E1,9E0,3.8E1,1.5E2,2.4E1,1E1,4.4E1,2.23E2,6E0,1.9E1,5E0,5.7E1,2.4E1,1.85E2,5.7E1,2.1E2,1.12E2,1.2E1,8E0,8E0,1.9E1,5E0,5E0,1.5E1,1E1,6.4E1,1.84E2,5E0,1.9E1,3.4E1,9E0,3E1,1.7E1,6.4E1,3.1E1,2.2E1,4E0,8.5E1,6.5E1,1.9E1,3.9E1,2E1,5E0,1.9E1,5E0,7E0,7E0,4E0,4E0,4E0,1.4E1,5E0,8E0,1.1E1,4E1,5.3E1,2.1E1,5E0,4E0,3.5E1,2.8E1,4.5E1,3.5E1,6E0,1.4E1,1.8E1,6E0,1.3E1,9E0,3.3E1,2.3E1,5E0,4E0,2.4E1,1.4E1,1.31E2,1.9E1,5E0,1.9E1,5E0,5E0,3.9E1,5E0,2.9E1,1.94E2,1.5E1,4E0,2.4E1,3.3E1,1.4E1,1E1,8E0,1.77E2,3.5E1,2.2E1,1.55E2,5.5E1,2.6E1,8.6E1,5E0,7E0,4E0,4E0,1.2E1,7E0,5E0,5E0,1.9E1,4.5E1,5.9E1,1.25E2,3E1,4E0,6E0,2.4E1,1.2E1,5E0,5.4E1,1E1,1.8E1,1.3E1,1.1E1,1.1E1,4E0,8.1E1,4E1,2.5E1,8E0,1.1E1,3.3E1,6E0,1E1,1E1,6E0,5E0,1.2E1,2.8E1,4.2E1,1.1E1,1.3E1,8E0,2.9E1,6E0,2.3E1,5E0,3.2E1,1.3E1,1.6E1,1.9E1,7E0,7E0,1.4E1,4E0,6E0,7E0,5E0,4E0,1.2E1,2.1E1,1.1E1,1.2E1,1.5E1,9E0,1E1,4E0,1.18E2,1.3E1,1.3E1,6E0,5E0,1.4E1,1.6E1,2.3E1,8E0,2.1E1,6.9E1,1.25E2,1.7E1,7E0,2.7E1,6E0,4E0,1E1,5E0,5E0,4E0,4E0,1.39E2,3.8E1,1.8E1,1.7E1,1E1,1.2E1,3.9E1,1.16E2,4.2E1,1.3E1,1.1E1,1.5E1,1.6E1,7E1,5E0,1.4E1,3.9E1,6E0,2.8E1,3.1E1,6E0,1.19E2,1.7E1,1.3E1,4E0,8E0,5E1,4E0,5E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"341","size_leaf_vector":"1"}},{"base_weights":[1.0322615E-3,-2.5919566E-1,1.2199299E-1,-4.0412647E-1,-7.503884E-2,2.7816614E-2,3.2240042E-1,-4.3379834E-1,2.0902927E-1,-1.9258851E-1,1.6067554E-1,-8.3285145E-2,1.2177037E-1,-1.9324285E-1,3.5973567E-1,-1.704647E-1,-4.805173E-1,4.3938313E-2,-1.6241173E-1,-2.4174449E-1,-5.912055E-2,5.075454E-2,3.54382E-1,-3.150517E-1,-3.4124706E-2,1.466199E-1,-2.5454885E-1,-2.5230798E-1,-6.9747396E-2,3.14648E-1,6.16979E-1,-3.636287E-1,-4.2229168E-2,-4.4837433E-1,-8.474464E-1,-1.9855862E-3,-1.3230362E-2,-2.1946973E-1,-4.397896E-1,9.4419534E-5,-2.0872124E-1,-5.0610054E-2,1.2846579E-1,2.134905E-1,4.2726678E-1,-1.8513705E-1,-4.5506284E-1,-7.478425E-2,1.9677396E-1,4.5714617E-2,2.513833E-1,-3.2883564E-1,4.6463367E-3,-1.43079255E-2,-6.99139E-3,3.585674E-2,-1.0819235E-2,2.862506E-1,6.356334E-1,4.6213865E-1,7.0927507E-1,-2.2543813E-3,-4.1706026E-1,1.4666446E-2,-9.553439E-2,-5.41823E-1,-3.3242768E-1,-9.521908E-1,-1.435181E-2,-2.5456253E-1,-1.403395E-1,-2.5785297E-2,-1.0691396E-2,-9.49424E-2,1.21046074E-1,-2.213617E-2,-3.512946E-2,-9.0147E-2,7.7703004E-3,1.8665653E-2,1.8274367E-1,3.4079885E-3,1.4317276E-2,6.1533326E-1,3.0579668E-1,-2.4919622E-1,1.4861189E-1,-3.877117E-1,-3.872004E-2,-1.070126E-1,1.4216968E-1,3.5374856E-1,1.05193086E-1,9.772257E-2,-7.948418E-2,2.2115938E-1,6.4527667E-1,-3.8940108E-1,-1.1733156E-1,5.029091E-3,-1.804136E-3,4.4819653E-2,2.7517915E-1,5.0722677E-1,8.23797E-1,7.907392E-3,4.9989894E-1,7.513949E-1,1.0252612E-2,-1.2320756E-2,-5.2923167E-1,4.9382515E-2,-1.4387849E-1,-5.012498E-1,-7.682001E-1,-4.343356E-1,-2.3392248E-1,-5.1274467E-2,-2.7942056E-2,-3.1923035E-1,-1.9632283E-1,-4.9548086E-2,-2.720055E-1,-1.6232543E-2,-2.3499689E-1,1.6885023E-1,-1.7275156E-3,-6.942975E-3,4.589533E-3,-3.4359097E-4,-2.075558E-1,7.956093E-2,-6.5564336E-3,2.8503658E-3,2.1179137E-1,1.8103069E-2,3.6157697E-2,4.8507596E-3,3.734041E-1,-2.8102982E-1,-2.0379298E-3,1.3416565E-2,-5.0411865E-5,-2.4762988E-2,-2.5505376E-1,2.2582503E-1,-1.19838975E-1,4.249045E-2,1.0173789E-2,2.0865837E-2,9.685741E-3,1.7061198E-2,1.3184496E-2,8.504224E-2,1.9476185E-2,-2.2780505E-1,-8.716841E-3,3.3119717E-1,1.5696464E-1,4.933488E-1,5.1604375E-2,-2.4478418E-1,-2.32089E-2,-7.470144E-4,-9.805786E-3,1.7946444E-1,3.1344053E-1,2.7126048E-2,1.3149087E-2,4.6475578E-2,2.3366174E-2,5.8116424E-1,2.637651E-1,7.8378946E-1,1.7410321E-2,-3.2266397E-2,-1.0674733E-2,-6.439032E-4,4.649052E-3,-9.714845E-2,-1.2419716E-2,-4.7721255E-1,-4.85775E-2,-8.478519E-1,-2.0961113E-2,-4.5266557E-1,-4.173063E-3,-8.37378E-2,-2.8759456E-1,-3.4269205E-1,-1.7410634E-3,-2.2268976E-1,-8.183803E-2,4.5773797E-2,-1.09029904E-1,-3.3432543E-1,-6.4429464E-3,1.2345782E-2,-4.93276E-3,-1.5942784E-2,-6.2843645E-3,4.9042706E-3,1.27828615E-2,-5.216897E-3,4.771236E-2,-1.2980343E-2,-3.5996481E-3,7.938044E-3,1.0924205E-5,2.4343291E-3,2.4196987E-1,2.1274671E-2,9.692379E-3,-2.9539505E-1,-7.7539627E-3,-2.1928037E-3,-1.5244981E-2,-2.5595208E-3,2.0255692E-2,-1.8465704E-1,-7.973118E-2,2.8658065E-1,-6.909028E-2,8.9035854E-2,-5.148137E-2,-4.547627E-3,1.2304422E-1,-1.5284877E-1,-2.8581766E-2,-1.6308095E-1,5.5231743E-2,2.8943077E-1,5.402127E-1,6.1713386E-2,2.1026568E-1,1.1965308E-2,2.893959E-2,-1.565504E-2,-4.991252E-3,2.1382007E-1,-9.592967E-2,-3.767089E-2,3.3119404E-1,3.272769E-2,1.9414604E-2,5.9770606E-3,1.7745972E-2,2.2641562E-2,4.0927093E-2,-3.2230784E-4,-6.6454476E-3,-3.7693392E-2,-2.2552008E-2,-1.933847E-2,-4.6644624E-2,-2.8780786E-2,-1.8937292E-2,-1.24196345E-2,-5.4687244E-4,-1.119883E-2,-1.9338824E-2,-1.835417E-2,-7.4669146E-3,-9.676034E-3,-1.5260083E-2,3.1373118E-5,-8.901707E-3,6.693503E-3,-3.969149E-3,-8.484794E-3,-2.1917534E-3,-1.15662655E-2,-2.3641827E-2,-1.3313253E-3,3.0335844E-3,-2.073608E-3,4.5956E-3,1.460217E-2,6.497543E-3,-1.5460038E-2,-7.7471607E-3,-7.3822285E-3,-1.8009672E-2,3.2810834E-3,-6.804072E-3,1.7326986E-2,7.846576E-3,-9.196051E-3,-5.5596232E-5,1.1831365E-3,7.1584103E-3,-4.001418E-3,1.3718763E-3,8.8328775E-3,-3.365895E-3,5.4122936E-3,1.8624282E-2,-1.0664533E-2,-1.7273878E-3,-3.396919E-3,-1.2085442E-2,8.59911E-3,-2.4892837E-3,1.0098805E-2,2.0483602E-2,1.4051813E-2,3.0135248E-2,1.8421818E-3,1.7860627E-2,8.47101E-3,1.5306905E-2,4.3592197E-3,1.5031903E-2,3.1068376E-3,-1.5844591E-2,3.121104E-3,-9.907562E-3,2.066357E-2,1.1532922E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,-1,-1,-1,99,-1,101,103,105,107,-1,109,-1,111,113,115,117,-1,119,121,-1,-1,123,125,-1,127,129,-1,131,133,-1,-1,135,137,139,141,143,-1,145,147,149,151,153,155,157,159,161,163,-1,-1,-1,165,167,169,-1,171,173,-1,-1,175,177,179,181,183,185,187,-1,-1,189,191,193,195,197,199,201,-1,-1,-1,203,205,207,-1,-1,209,-1,-1,-1,211,213,-1,-1,-1,-1,215,217,219,-1,221,-1,-1,223,-1,225,-1,227,229,231,233,235,-1,237,-1,-1,-1,239,241,-1,-1,-1,-1,243,245,247,-1,-1,-1,-1,-1,249,-1,251,-1,253,-1,255,-1,257,259,261,-1,263,265,267,269,271,-1,273,-1,-1,-1,-1,-1,-1,275,-1,-1,-1,-1,-1,277,-1,-1,279,-1,-1,-1,-1,-1,281,283,285,287,289,291,293,295,297,-1,299,301,303,305,307,309,-1,-1,-1,-1,311,313,315,317,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.193571E1,2.201561E1,3.3529503E1,8.496071E0,1.0144418E1,1.2640638E1,1.1004463E1,5.343567E0,5.620499E0,1.5865307E0,2.5796878E0,6.3127785E0,6.171814E0,2.6687896E-1,5.953079E0,1.6690375E0,4.0310516E0,0E0,1.7785305E-1,6.887102E-1,5.9354585E-1,6.28798E-1,3.6061144E-1,1.6801319E0,4.3239155E0,6.5003557E0,1.0890939E0,5.940473E-2,2.2477448E-1,3.947216E0,7.869282E-1,4.4068408E-1,7.797338E-1,3.5688171E0,1.4282169E0,0E0,0E0,4.1413307E-1,1.8233824E-1,5.747406E-1,7.5720716E-1,2.992458E-1,2.647763E-1,1.6086942E-1,4.51478E-1,1.1487769E0,6.905136E-1,2.7491124E0,9.547236E-1,2.0588377E0,3.4526005E0,3.8984513E-1,0E0,0E0,0E0,4.5598175E-2,0E0,2.5795898E0,4.5196152E-1,2.8808308E-1,8.437271E-1,0E0,3.0933237E-1,0E0,2.6632252E-1,1.3600578E0,1.4744377E0,1.2391281E-1,0E0,3.5538006E-1,6.081382E-1,0E0,0E0,3.0484813E-1,1.6950133E-1,0E0,1.7096555E-1,3.1624E-1,0E0,1.5647317E-1,9.745562E-2,0E0,0E0,1.1864185E-2,2.3185515E-1,2.8045392E-1,1.617246E-1,4.460349E-1,0E0,1.482825E0,4.7588444E0,1.7428541E-1,6.247839E-1,7.927165E-1,9.7416764E-1,1.9400177E0,7.2679234E-1,1.6862941E-1,6.978085E-2,0E0,0E0,0E0,1.4525623E0,6.599808E-2,5.7377815E-2,0E0,3.840146E-1,2.8820038E-1,0E0,0E0,3.4011984E-1,2.8386079E-2,1.1282247E-1,1.3925858E0,3.9423275E-1,4.5091248E-1,6.368971E-1,0E0,0E0,3.283391E-1,1.659112E-1,1.8642813E-1,1.441164E-1,4.600905E-2,5.169815E-2,7.942763E-2,0E0,0E0,0E0,9.499759E-2,7.328832E-2,7.547297E-2,0E0,0E0,1.0632217E-1,0E0,0E0,0E0,4.8969626E-2,2.285695E-2,0E0,0E0,0E0,0E0,1.9798684E-1,6.6747355E-1,8.405814E-1,0E0,9.838017E-1,0E0,0E0,1.5258352E-1,0E0,7.325469E-1,0E0,6.8659246E-1,6.4077467E-1,7.4281883E-1,9.0088224E-1,2.3174572E-1,0E0,7.792026E-2,0E0,0E0,0E0,1.1558728E0,1.8388824E0,0E0,0E0,0E0,0E0,2.990055E-2,7.607728E-2,8.403969E-2,0E0,0E0,0E0,0E0,0E0,6.742248E-2,0E0,7.924614E-1,0E0,4.6769142E-1,0E0,4.3586445E-1,0E0,2.7322885E-1,2.91934E-1,1.6887712E-1,0E0,5.855894E-2,1.0336689E-1,1.5418318E-1,7.176566E-2,9.0363145E-2,0E0,2.9904746E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.514889E-2,0E0,0E0,0E0,0E0,0E0,7.45312E-2,0E0,0E0,1.6924143E-2,0E0,0E0,0E0,0E0,0E0,7.5782585E-1,1.6912409E0,2.3517609E-2,2.7329433E-1,4.9770556E-2,3.832086E-2,7.522476E-1,5.1484823E-1,1.789316E-1,0E0,1.2591678E-1,5.7330465E-1,8.431978E-1,9.101486E-2,4.717662E-1,3.981104E-1,0E0,0E0,0E0,0E0,1.1523156E0,5.0928676E-1,2.5362864E-1,2.21097E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,55,55,57,57,58,58,59,59,60,60,62,62,64,64,65,65,66,66,67,67,69,69,70,70,73,73,74,74,76,76,77,77,79,79,80,80,83,83,84,84,85,85,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,102,102,103,103,104,104,106,106,107,107,110,110,111,111,112,112,113,113,114,114,115,115,116,116,119,119,120,120,121,121,122,122,123,123,124,124,125,125,129,129,130,130,131,131,134,134,138,138,139,139,144,144,145,145,146,146,148,148,151,151,153,153,155,155,156,156,157,157,158,158,159,159,161,161,165,165,166,166,171,171,172,172,173,173,179,179,181,181,183,183,185,185,187,187,188,188,189,189,191,191,192,192,193,193,194,194,195,195,197,197,204,204,210,210,213,213,219,219,220,220,221,221,222,222,223,223,224,224,225,225,226,226,227,227,229,229,230,230,231,231,232,232,233,233,234,234,239,239,240,240,241,241,242,242],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,-1,-1,-1,100,-1,102,104,106,108,-1,110,-1,112,114,116,118,-1,120,122,-1,-1,124,126,-1,128,130,-1,132,134,-1,-1,136,138,140,142,144,-1,146,148,150,152,154,156,158,160,162,164,-1,-1,-1,166,168,170,-1,172,174,-1,-1,176,178,180,182,184,186,188,-1,-1,190,192,194,196,198,200,202,-1,-1,-1,204,206,208,-1,-1,210,-1,-1,-1,212,214,-1,-1,-1,-1,216,218,220,-1,222,-1,-1,224,-1,226,-1,228,230,232,234,236,-1,238,-1,-1,-1,240,242,-1,-1,-1,-1,244,246,248,-1,-1,-1,-1,-1,250,-1,252,-1,254,-1,256,-1,258,260,262,-1,264,266,268,270,272,-1,274,-1,-1,-1,-1,-1,-1,276,-1,-1,-1,-1,-1,278,-1,-1,280,-1,-1,-1,-1,-1,282,284,286,288,290,292,294,296,298,-1,300,302,304,306,308,310,-1,-1,-1,-1,312,314,316,318,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,2.8357262E6,6.7652373E3,4.197889E6,2.2178302E5,2.1328075E-1,8.2608955E-3,3.3481183E9,3.449702E2,1.2060912E3,8.2E1,4.898012E-1,1.5106794E3,3.0121632E6,2.373E3,1.2427474E8,4.3938313E-2,1.4752766E8,1.5988282E1,4.2343444E7,1.502494E0,1.948181E0,1.139605E6,6.077036E8,9.99E2,2.5422776E3,3.0820766E-1,4.396066E1,5.2076015E0,5.202E3,2.127E3,1.1E1,2.040457E0,2.5422776E3,-1.9855862E-3,-1.3230362E-2,7.84892E5,1.5669752E-2,5.4878284E2,1.7488463E3,3.650307E9,2.821E3,6.19999E9,2.9956698E-1,1.13E3,3.7832818E0,1.1925198E3,1.6526531E1,4.072289E0,1.0409E4,1.945345E6,4.6463367E-3,-1.43079255E-2,-6.99139E-3,4.152361E8,-1.0819235E-2,3.8E1,2.1196339E1,3.5044186E2,6.292039E0,-2.2543813E-3,1.5756359E0,1.4666446E-2,1.9051096E7,2.0354E4,1.6526531E1,4.242625E6,-1.435181E-2,1.9E1,1.6E1,-2.5785297E-2,-1.0691396E-2,1.945711E1,6.788246E0,-2.213617E-2,4.759E3,5.12861E5,7.7703004E-3,1.4187837E-1,2.091623E5,3.4079885E-3,1.4317276E-2,4.382419E6,4.0434834E5,3.1530054E0,5.37798E5,1.5043668E0,-3.872004E-2,7E0,2.3E1,1.07E2,1.5963264E9,7.780377E6,1E0,3.1753032E0,2.496172E3,6.5E1,3.3E1,5.029091E-3,-1.804136E-3,4.4819653E-2,1E0,1.921032E4,1.772114E6,7.907392E-3,6.6985805E8,3.5E1,1.0252612E-2,-1.2320756E-2,2.909019E3,3.6709285E6,3.4692544E7,4.5723195E6,5.4379158E1,3.266054E3,2.99665E5,-5.1274467E-2,-2.7942056E-2,1.19507775E-1,2E0,1.246372E0,3.7382904E7,1.3653578E7,2.1E1,7.298614E2,-1.7275156E-3,-6.942975E-3,4.589533E-3,4.45006E8,6.071E3,3.4048372E6,-6.5564336E-3,2.8503658E-3,1.5834932E6,1.8103069E-2,3.6157697E-2,4.8507596E-3,3.07125E0,1.8731416E4,-2.0379298E-3,1.3416565E-2,-5.0411865E-5,-2.4762988E-2,1.5E1,3.79E2,7.79E2,4.249045E-2,5.0279167E1,2.0865837E-2,9.685741E-3,2.8169013E-3,1.3184496E-2,1.5288235E2,1.9476185E-2,7.1E1,6.1E2,1.9121015E0,6.999242E7,1E0,5.1604375E-2,6.6404694E2,-2.32089E-2,-7.470144E-4,-9.805786E-3,2.992587E7,2.9E2,2.7126048E-2,1.3149087E-2,4.6475578E-2,2.3366174E-2,1.0594161E1,6.97E2,1.1328599E3,1.7410321E-2,-3.2266397E-2,-1.0674733E-2,-6.439032E-4,4.649052E-3,1.3225807E0,-1.2419716E-2,5.85E2,-4.85775E-2,1.4642603E2,-2.0961113E-2,1.2E1,-4.173063E-3,3.684685E2,1.873E3,4E1,-1.7410634E-3,1.0993947E1,8.347515E7,6.392807E7,2.028994E8,1.059448E7,-6.4429464E-3,2.437E3,-4.93276E-3,-1.5942784E-2,-6.2843645E-3,4.9042706E-3,1.27828615E-2,-5.216897E-3,1.0693E4,-1.2980343E-2,-3.5996481E-3,7.938044E-3,1.0924205E-5,2.4343291E-3,2.2729468E0,2.1274671E-2,9.692379E-3,1.8388145E-7,-7.7539627E-3,-2.1928037E-3,-1.5244981E-2,-2.5595208E-3,2.0255692E-2,3.1E1,2.1E1,1.79E2,1.415178E3,1.0134536E7,9.262083E2,5.8365756E-1,1.902353E1,2.1573659E2,-2.8581766E-2,1E0,8.134936E0,4.1365924E2,1E0,6.6692764E1,1.2727361E6,1.1965308E-2,2.893959E-2,-1.565504E-2,-4.991252E-3,5.077728E8,2.6572757E10,3.2627738E5,6.9309726E0,3.272769E-2,1.9414604E-2,5.9770606E-3,1.7745972E-2,2.2641562E-2,4.0927093E-2,-3.2230784E-4,-6.6454476E-3,-3.7693392E-2,-2.2552008E-2,-1.933847E-2,-4.6644624E-2,-2.8780786E-2,-1.8937292E-2,-1.24196345E-2,-5.4687244E-4,-1.119883E-2,-1.9338824E-2,-1.835417E-2,-7.4669146E-3,-9.676034E-3,-1.5260083E-2,3.1373118E-5,-8.901707E-3,6.693503E-3,-3.969149E-3,-8.484794E-3,-2.1917534E-3,-1.15662655E-2,-2.3641827E-2,-1.3313253E-3,3.0335844E-3,-2.073608E-3,4.5956E-3,1.460217E-2,6.497543E-3,-1.5460038E-2,-7.7471607E-3,-7.3822285E-3,-1.8009672E-2,3.2810834E-3,-6.804072E-3,1.7326986E-2,7.846576E-3,-9.196051E-3,-5.5596232E-5,1.1831365E-3,7.1584103E-3,-4.001418E-3,1.3718763E-3,8.8328775E-3,-3.365895E-3,5.4122936E-3,1.8624282E-2,-1.0664533E-2,-1.7273878E-3,-3.396919E-3,-1.2085442E-2,8.59911E-3,-2.4892837E-3,1.0098805E-2,2.0483602E-2,1.4051813E-2,3.0135248E-2,1.8421818E-3,1.7860627E-2,8.47101E-3,1.5306905E-2,4.3592197E-3,1.5031903E-2,3.1068376E-3,-1.5844591E-2,3.121104E-3,-9.907562E-3,2.066357E-2,1.1532922E-2],"split_indices":[20,102,51,52,51,28,42,27,12,52,52,29,27,52,28,2,45,0,48,54,45,41,42,31,7,2,52,27,58,35,2,1,8,53,52,0,0,9,38,52,4,12,2,5,38,2,53,52,58,54,2,12,0,0,0,7,0,0,54,4,38,0,54,0,45,9,58,29,0,3,3,0,0,56,53,0,2,29,0,27,28,0,0,1,28,53,5,53,0,3,3,3,7,28,102,54,4,10,3,0,0,0,102,4,29,0,5,8,0,0,32,32,45,28,58,4,29,0,0,38,8,53,12,32,3,52,0,0,0,12,2,45,0,0,45,0,0,0,57,28,0,0,0,0,3,2,2,0,56,0,0,57,0,52,0,0,2,53,7,101,0,33,0,0,0,43,10,0,0,0,0,56,0,4,0,0,0,0,0,57,0,2,0,4,0,18,0,52,0,8,0,56,7,12,7,12,0,2,0,0,0,0,0,0,9,0,0,0,0,0,53,0,0,37,0,0,0,0,0,8,3,3,52,45,52,54,58,55,0,16,58,52,102,56,28,0,0,0,0,7,12,28,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.601E3,8.25E2,1.776E3,4.61E2,3.64E2,1.209E3,5.67E2,4.4E2,2.1E1,2.43E2,1.21E2,5.54E2,6.55E2,3.8E1,5.29E2,6.7E1,3.73E2,7E0,1.4E1,1.77E2,6.6E1,7.8E1,4.3E1,9.6E1,4.58E2,6.15E2,4E1,2.5E1,1.3E1,4.52E2,7.7E1,2.6E1,4.1E1,3.45E2,2.8E1,7E0,7E0,1.61E2,1.6E1,4.8E1,1.8E1,3.4E1,4.4E1,1.6E1,2.7E1,5.1E1,4.5E1,3.9E2,6.8E1,3.14E2,3.01E2,3.3E1,7E0,1.8E1,7E0,8E0,5E0,4.17E2,3.5E1,3.1E1,4.6E1,4E0,2.2E1,5E0,3.6E1,1.89E2,1.56E2,2.3E1,5E0,1.1E2,5.1E1,1.1E1,5E0,2.7E1,2.1E1,7E0,1.1E1,2.9E1,5E0,1.5E1,2.9E1,6E0,1E1,9E0,1.8E1,4.3E1,8E0,3.9E1,6E0,3.4E2,5E1,2.4E1,4.4E1,2.22E2,9.2E1,2.81E2,2E1,2.5E1,8E0,4E0,4E0,6E0,4.11E2,2.3E1,1.2E1,4E0,2.7E1,4.2E1,4E0,1E1,1.2E1,9E0,2.7E1,1.63E2,2.6E1,7.5E1,8.1E1,1.8E1,5E0,5E1,6E1,3.1E1,2E1,1.8E1,9E0,1.6E1,5E0,6E0,5E0,1.7E1,1.2E1,1.1E1,4E0,6E0,2.3E1,4E0,5E0,5E0,1.3E1,3.7E1,6E0,4E0,4E0,2E1,1.9E1,1.2E1,3.28E2,7E0,4.3E1,1.6E1,8E0,2.9E1,1.5E1,2.14E2,8E0,2.9E1,6.3E1,1.02E2,1.79E2,1.6E1,4E0,1E1,1.5E1,4E0,4E0,1.19E2,2.92E2,1.9E1,4E0,8E0,4E0,1.9E1,8E0,3.8E1,4E0,8E0,4E0,4E0,5E0,2E1,7E0,1.57E2,6E0,2E1,6E0,7.1E1,4E0,2.2E1,5.9E1,4.6E1,4E0,4.8E1,1.2E1,1.2E1,1.9E1,1.3E1,7E0,1.4E1,4E0,4E0,5E0,1E1,6E0,5E0,1.2E1,8E0,4E0,5E0,6E0,4E0,1.9E1,9E0,4E0,3.2E1,5E0,4E0,1.5E1,5E0,7E0,1.24E2,2.04E2,9E0,3.4E1,1.4E1,1.5E1,6.4E1,1.5E2,2.5E1,4E0,1.8E1,4.5E1,8.7E1,1.5E1,6.5E1,1.14E2,5E0,1.1E1,6E0,4E0,1.06E2,1.3E1,1.4E1,2.78E2,1.2E1,7E0,4E0,4E0,5E0,3.3E1,6E0,1.4E1,1.1E1,1.46E2,4E0,1.6E1,2.4E1,4.7E1,6E0,1.6E1,3.8E1,2.1E1,4E1,6E0,3.8E1,1E1,7E0,5E0,7E0,5E0,9E0,1E1,9E0,4E0,8E0,6E0,4E0,8E0,1.2E1,7E0,2.8E1,4E0,1.04E2,2E1,5.7E1,1.47E2,5E0,4E0,1.2E1,2.2E1,7E0,7E0,1.1E1,4E0,1.6E1,4.8E1,1.43E2,7E0,1.6E1,9E0,9E0,9E0,2.1E1,2.4E1,5.2E1,3.5E1,4E0,1.1E1,6.1E1,4E0,8.2E1,3.2E1,4.4E1,6.2E1,8E0,5E0,9E0,5E0,1.51E2,1.27E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"319","size_leaf_vector":"1"}},{"base_weights":[-1.5463125E-3,-2.4762502E-1,1.18728206E-1,-3.866081E-1,-7.129009E-2,1.6670007E-2,2.9375032E-1,-3.162362E-3,-4.4430342E-1,-1.5033558E-1,2.4689637E-1,-8.783238E-2,1.2423222E-1,2.1092398E-1,4.928229E-1,5.910323E-1,-1.1313985E-1,-4.8141867E-1,-3.0764228E-2,-2.0564221E-1,1.610696E-2,1.5197743E-1,4.1001236E-1,-1.1735036E-1,1.4548881E-1,3.3905465E-2,2.2100458E-1,1.3393365E-1,3.6555153E-1,2.69844E-1,5.7223237E-1,-1.6338467E-3,5.0581396E-2,-3.38526E-1,-4.3469586E-4,-4.5054558E-1,-8.1246454E-1,1.5326305E-2,-1.260926E-1,-3.0770403E-1,-1.634221E-1,-1.3786037E-1,8.5524105E-2,3.6277235E-1,5.7294555E-2,4.505283E-1,7.0771086E-3,2.4714676E-1,-1.2847324E-1,6.974666E-1,1.5311459E-2,8.995988E-2,-1.3315633E-1,-1.2286909E-1,2.645184E-1,1.8102056E-1,-1.7252672E-1,2.8827292E-1,5.6396854E-1,2.9727665E-1,-2.0981957E-3,2.992043E-3,5.871912E-1,-2.0265345E-2,-1.0193937E-2,-1.2425981E-1,1.1102964E-1,-5.283109E-1,-3.4559146E-1,-9.005812E-1,-1.475091E-2,-3.2489784E-2,-2.4818887E-1,-3.7721008E-1,-1.5470539E-1,-1.12756476E-1,-2.3180166E-1,-5.061007E-2,-1.8683654E-1,1.5721755E-1,-1.4824622E-2,2.1352146E-2,9.7294E-3,9.689678E-2,-4.420804E-3,2.4732316E-2,1.2096739E-2,-6.8519375E-4,3.7894794E-1,-2.0781067E-1,-6.828662E-2,1.4075539E-2,4.347791E-2,1.5143107E-1,-1.6290157E-1,-9.3041375E-2,1.0855397E-1,-8.8701524E-2,-2.9978324E-2,2.2327404E-2,-2.6955408E-1,2.4349849E-1,4.523846E-2,1.2391064E-1,3.375264E-1,-2.5239065E-1,1.6759384E-2,3.5037056E-1,3.814843E-2,4.601353E-1,3.8801827E-2,3.3676627E-1,3.1313638E-3,6.0223114E-1,4.083415E-3,-1.1363848E-2,-5.372303E-2,4.77764E-2,1.2524535E-2,-5.602982E-1,-3.3050534E-1,-4.1958278E-1,-2.563113E-1,-1.937563E-2,-9.761934E-1,-8.313549E-3,3.7163757E-2,-6.6512036E-3,-1.9120881E-2,-3.236492E-1,-2.8869634E-2,-1.8702966E-1,-2.9340722E-3,-1.7144918E-1,-3.4609742E-2,-1.923446E-2,-1.9549409E-1,-5.00332E-3,9.463511E-4,-5.383875E-3,-1.1371873E-2,2.719019E-2,2.0917465E-1,-8.576105E-2,4.6338864E-2,1.5918843E-2,1.4835787E-1,2.4316054E-2,8.690268E-3,-2.2018719E-1,-6.1775288E-3,-8.894742E-2,7.592884E-2,-8.259676E-3,2.2139017E-1,3.6856963E-3,-2.5199383E-1,-1.475215E-1,2.931336E-3,6.703637E-2,1.5894735E-1,-1.6202937E-1,4.637345E-2,-5.134751E-3,1.0765831E-1,-1.8696608E-2,7.2637975E-4,2.1853141E-1,2.6861673E-2,2.7611363E-1,7.371028E-2,3.666774E-1,3.2863936E-3,-2.9635282E-2,-1.6447853E-1,2.543235E-1,4.7005552E-1,-1.1332174E-1,2.8636E-1,5.154237E-1,3.177804E-1,1.6480628E-1,4.3335E-1,4.8296466E-1,6.899426E-1,-4.44065E-3,-1.8796328E-4,7.11481E-3,5.375563E-3,-5.3515846E-1,-8.4461296E-1,-2.2632958E-1,-2.5793985E-2,-5.223261E-1,-3.418254E-1,-1.3139147E-1,-3.3459884E-1,-5.2660346E-2,-2.5042128E-2,5.5345045E-3,-6.736416E-4,-1.7359255E-2,-9.696775E-3,-4.157149E-3,-1.1325553E-2,-1.8881582E-1,-5.230643E-5,-1.73685E-1,1.4278933E-2,-3.6848735E-2,-2.2846441E-1,4.3873377E-3,-1.3917763E-3,3.9741243E-3,1.2123174E-2,-1.0981158E-3,-6.758073E-3,4.7182865E-3,-4.1826075E-4,3.8895833E-3,-2.9470243E-3,8.981549E-3,3.8890115E-3,-2.2991247E-1,6.3479776E-3,2.3320497E-3,-3.7781857E-3,-1.2543675E-1,-2.848714E-2,2.211638E-2,1.6605739E-2,2.6443616E-1,-1.839281E-4,2.2577427E-3,-1.9262511E-3,-8.654195E-3,-1.7054573E-2,-2.0408342E-3,-1.1780087E-2,1.598863E-1,2.1545882E-3,-4.7329538E-2,1.836723E-1,4.6728007E-3,-1.981151E-1,9.563096E-3,-3.8096413E-2,8.472942E-3,5.91123E-4,1.7037238E-1,3.1241557E-1,1.15686394E-1,4.1907924E-1,4.0448926E-2,1.8404396E-1,3.2010993E-1,3.0505363E-2,-1.4585149E-2,-6.331597E-2,2.7999574E-1,2.6460218E-3,1.4189831E-2,5.029104E-1,5.114165E-3,-1.7549112E-2,6.2325764E-3,1.952264E-2,3.0635346E-2,1.780913E-2,8.982835E-3,2.0127615E-2,2.207634E-1,3.0641453E-3,4.989953E-1,7.1106222E-3,5.1638204E-1,8.280802E-3,7.0741093E-1,1.8981822E-2,3.0787995E-3,-1.9731605E-3,-2.1097157E-2,-2.9631326E-2,-4.997875E-2,-2.1319982E-2,-3.087875E-3,-1.43915E-2,-1.171799E-2,-3.0232064E-2,-7.540163E-3,-1.8757377E-2,-1.4112784E-3,-9.636405E-3,-2.0085772E-2,-9.094698E-3,-7.747992E-3,-1.7109334E-2,-1.2599825E-2,-4.6377066E-3,-3.0445727E-3,4.6618E-3,-5.398713E-3,9.6289604E-4,-8.350884E-3,-1.5889274E-2,-2.9528367E-3,-1.2215135E-2,-2.9483435E-3,3.5192806E-3,-9.316482E-3,-1.4073234E-3,7.534801E-3,-3.7042527E-3,-2.005417E-3,5.3269197E-3,1.62993E-2,6.1603333E-3,1.1063789E-2,-2.2412245E-3,-1.9723292E-2,1.7584425E-3,3.8095568E-3,-9.301267E-3,1.22277355E-2,5.167837E-3,-1.2969393E-2,-4.3268194E-3,2.7301938E-3,-3.6756485E-3,5.990219E-3,1.3430931E-2,2.4927577E-2,1.3642036E-2,-7.278657E-4,7.9903435E-3,2.329793E-2,6.848657E-3,1.9380015E-5,5.3723613E-3,6.4377068E-3,1.5172349E-2,1.9231508E-2,8.273232E-3,4.4052275E-3,-5.640894E-3,1.6535504E-2,1.0578105E-2,2.6422927E-2,1.3114465E-2,1.095123E-2,-7.4207475E-3,6.878363E-3,1.3807588E-2,1.6926179E-2,2.8860586E-2,1.9085787E-2,2.9090378E-2,3.6975704E-2,2.3693781E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,-1,63,65,67,69,-1,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,-1,-1,113,-1,-1,115,117,119,121,123,-1,125,127,129,131,133,135,137,139,141,143,-1,-1,145,-1,-1,-1,-1,147,149,151,-1,-1,153,155,157,159,161,-1,163,165,167,-1,169,171,173,-1,175,177,179,-1,181,-1,183,-1,-1,185,187,-1,189,191,193,195,-1,197,-1,199,-1,-1,201,-1,203,-1,205,207,-1,209,-1,-1,-1,-1,211,213,215,217,219,221,-1,-1,223,225,227,229,-1,231,233,235,237,-1,239,241,243,245,-1,247,-1,-1,249,-1,251,253,255,-1,-1,257,259,261,263,265,267,269,271,273,275,277,-1,-1,279,-1,281,283,285,-1,287,289,291,293,-1,-1,-1,-1,-1,-1,-1,-1,295,-1,297,299,301,303,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,305,307,-1,-1,309,311,313,-1,315,-1,-1,-1,-1,-1,-1,-1,317,319,321,323,-1,325,-1,327,-1,-1,329,331,333,335,337,339,341,-1,-1,343,345,-1,-1,347,349,-1,-1,-1,-1,-1,-1,-1,351,-1,353,-1,355,-1,357,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.644885E1,2.075486E1,3.0986433E1,1.0484901E1,9.468821E0,1.2341747E1,1.04321785E1,4.183796E0,6.3088303E0,2.7741308E0,1.098897E0,3.8532276E0,4.7214603E0,5.3439274E0,3.1567268E0,2.6606958E0,1.3715706E0,3.497261E0,1.1647257E0,9.2356205E-1,8.2407635E-1,9.571897E-1,2.2977304E-1,2.0386095E0,4.516186E0,2.6434176E0,3.937911E0,4.418032E0,2.137432E0,4.4876242E-1,1.0148354E0,0E0,0E0,7.960081E-2,5.244736E-1,2.6577225E0,1.1470623E0,0E0,3.1193602E-1,6.3287497E-1,5.351105E-1,9.310064E-2,3.8363102E-1,7.952428E-2,2.10621E-1,9.324932E-2,0E0,5.20921E-1,2.282609E0,6.1637974E-1,1.2860993E0,7.280785E-1,1.4201709E0,6.4496285E-1,2.904745E0,2.3178196E0,1.7478576E0,1.6951685E0,5.4769135E-1,4.1432953E-1,0E0,0E0,9.618759E-1,0E0,0E0,1.18771315E-1,1.6466314E-1,1.0676193E0,8.899536E-1,6.2960243E-1,0E0,1.647264E-1,1.3477176E-1,2.8567648E-1,5.7275593E-2,4.297974E-1,3.0284548E-1,3.6339656E-2,2.3822606E-2,2.0374364E-1,1.0376461E-1,0E0,0E0,1.1432028E-1,0E0,0E0,0E0,0E0,1.3617074E-1,5.1521206E-1,8.2614577E-1,0E0,0E0,7.013511E-1,3.422864E-1,1.7395502E-1,3.9104939E-1,6.612279E-1,0E0,1.818493E-1,4.512962E-1,1.5419598E0,0E0,1.4695735E0,5.3041697E-1,9.807401E-1,0E0,8.920479E-1,8.9717364E-1,7.232618E-2,0E0,5.9292316E-1,0E0,1.0162582E0,0E0,0E0,2.073887E-2,3.571203E-2,0E0,7.559395E-1,4.6358418E-1,4.6856976E-1,6.50393E-1,0E0,2.8396797E-1,0E0,4.6223573E-2,0E0,0E0,4.6535015E-2,0E0,4.698348E-2,0E0,1.5960526E-1,2.8804666E-1,0E0,2.8972483E-1,0E0,0E0,0E0,0E0,3.5810888E-2,7.610583E-2,3.17514E-2,3.4993153E-2,6.0248133E-2,2.3647785E-2,0E0,0E0,4.3106747E-1,5.1235154E-2,5.2811015E-1,4.7557703E-1,0E0,2.4318361E-1,1.7516943E-2,4.645872E-2,1.31565E-1,0E0,6.4425224E-1,4.4730926E-1,4.1505718E-1,3.0127892E-1,0E0,5.7324447E-2,0E0,0E0,8.962612E-1,0E0,1.0524266E0,5.356931E-1,5.1992416E-1,0E0,0E0,3.682192E-1,2.4882054E-1,8.948612E-2,4.2442513E-1,1.0305971E-1,9.21402E-2,5.6677222E-2,7.911131E-2,4.066286E-1,5.29294E-1,1.2122345E-1,0E0,0E0,2.783365E-2,0E0,7.821007E-1,3.5371494E-1,1.953572E-1,0E0,6.437454E-1,2.5742626E-1,1.7841154E-1,3.689494E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0093822E-1,0E0,4.6407342E-2,1.9586465E-1,4.6033423E-2,2.0495582E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5042038E-1,4.1849725E-2,0E0,0E0,8.850336E-1,7.6409084E-1,1.6207339E-1,0E0,1.4286685E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.6429434E-1,8.5306776E-1,1.8573424E-1,3.5188985E-1,0E0,2.3791838E-1,0E0,5.664286E-2,0E0,0E0,6.6844845E-1,3.7797976E-1,1.4058691E-1,2.687993E-1,3.089512E-1,1.8536925E-1,4.868374E-1,0E0,0E0,1.4513151E-1,8.647227E-2,0E0,0E0,5.42655E-2,3.942861E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.143067E-3,0E0,5.6250572E-2,0E0,2.3761654E-1,0E0,1.5843582E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,62,62,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,83,83,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,103,103,104,104,105,105,107,107,108,108,109,109,111,111,113,113,116,116,117,117,119,119,120,120,121,121,122,122,124,124,126,126,129,129,131,131,133,133,134,134,136,136,141,141,142,142,143,143,144,144,145,145,146,146,149,149,150,150,151,151,152,152,154,154,155,155,156,156,157,157,159,159,160,160,161,161,162,162,164,164,167,167,169,169,170,170,171,171,174,174,175,175,176,176,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,187,187,189,189,190,190,191,191,193,193,194,194,195,195,196,196,205,205,207,207,208,208,209,209,210,210,223,223,224,224,227,227,228,228,229,229,231,231,239,239,240,240,241,241,242,242,244,244,246,246,249,249,250,250,251,251,252,252,253,253,254,254,255,255,258,258,259,259,262,262,263,263,271,271,273,273,275,275,277,277],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,-1,64,66,68,70,-1,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,-1,-1,114,-1,-1,116,118,120,122,124,-1,126,128,130,132,134,136,138,140,142,144,-1,-1,146,-1,-1,-1,-1,148,150,152,-1,-1,154,156,158,160,162,-1,164,166,168,-1,170,172,174,-1,176,178,180,-1,182,-1,184,-1,-1,186,188,-1,190,192,194,196,-1,198,-1,200,-1,-1,202,-1,204,-1,206,208,-1,210,-1,-1,-1,-1,212,214,216,218,220,222,-1,-1,224,226,228,230,-1,232,234,236,238,-1,240,242,244,246,-1,248,-1,-1,250,-1,252,254,256,-1,-1,258,260,262,264,266,268,270,272,274,276,278,-1,-1,280,-1,282,284,286,-1,288,290,292,294,-1,-1,-1,-1,-1,-1,-1,-1,296,-1,298,300,302,304,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,306,308,-1,-1,310,312,314,-1,316,-1,-1,-1,-1,-1,-1,-1,318,320,322,324,-1,326,-1,328,-1,-1,330,332,334,336,338,340,342,-1,-1,344,346,-1,-1,348,350,-1,-1,-1,-1,-1,-1,-1,352,-1,354,-1,356,-1,358,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,2.264894E8,1.0119178E10,9.026015E2,3.2627738E5,1.2974394E6,2E0,2.9283184E3,1.2690893E0,1.3664E4,1.1925198E3,9.99E2,1.3359244E9,3.904E3,4.676E3,1.7019386E2,1.2427474E8,3.8134545E6,1.5288235E2,1.6708623E2,6.5346925E6,4.8765745E0,7E0,1E0,4.072289E0,1.2E1,4.0844156E7,1.8777761E0,3.0596492E0,1.545974E-1,-1.6338467E-3,5.0581396E-2,7.5646E4,6.3039363E4,2.040457E0,4.8083666E-1,1.5326305E-2,1.1855755E8,1.5669752E-2,1.3697374E7,2.909019E3,2.46875E0,2.6757258E-1,6.1061732E7,4.7741196E-1,7.0771086E-3,3.79E2,6.061028E4,5.0475483E3,2.7246006E7,6.647279E5,1.1095187E1,8.601996E-1,8.426E3,1.9214156E3,3.970405E3,2.6307288E7,1.1458888E4,3.1984328E5,-2.0981957E-3,2.992043E-3,2.851369E2,-2.0265345E-2,-1.0193937E-2,2.5269838E6,3.0387878E3,1.0118026E0,1.6526531E1,1.0804272E3,-1.475091E-2,7.8599895E9,2.9273078E0,3.788185E6,6.9966E4,4.6506356E5,5.492982E2,1.566593E6,2.241E2,1.9563605E5,2.821E3,2.1352146E-2,9.7294E-3,3.6482175E5,-4.420804E-3,2.4732316E-2,1.2096739E-2,-6.8519375E-4,7.123717E7,6.47E2,1.3593E4,1.4075539E-2,4.347791E-2,8.9E1,1E0,1.325204E6,2.6682352E2,2.7604443E2,-2.9978324E-2,6.4506575E5,1.0979587E8,6.057671E6,4.523846E-2,8E0,2.8091298E7,2.9816154E1,1.6759384E-2,4.1331047E-1,1.1424924E0,3.1933654E7,3.8801827E-2,1.0061821E5,3.1313638E-3,7.123E3,4.083415E-3,-1.1363848E-2,2.5853658E0,1.4615384E0,1.2524535E-2,3.321532E6,3.73802E5,1.2E1,1.6194E4,-1.937563E-2,2.4019E6,-8.313549E-3,1.0040462E8,-6.6512036E-3,-1.9120881E-2,8.2E2,-2.8869634E-2,6.953307E4,-2.9340722E-3,7.8351054E11,1.503E3,-1.923446E-2,1.2342285E1,-5.00332E-3,9.463511E-4,-5.383875E-3,-1.1371873E-2,6.782E4,3.6512393E2,5.8426323E0,3.488E3,2.980289E-1,2.2247495E-1,2.4316054E-2,8.690268E-3,1.3645918E0,5.704E3,3.8637936E2,2.5682794E5,-8.259676E-3,4.768812E1,8.426E3,9.302862E7,1.7E1,2.931336E-3,4.7048388E0,5.884286E2,5.827505E5,3.8977896E2,-5.134751E-3,9E0,-1.8696608E-2,7.2637975E-4,2.45009E6,2.6861673E-2,2.6563762E7,3.2627738E5,4.0889E4,3.2863936E-3,-2.9635282E-2,1.5106794E3,2.8363848E7,1.6195753E9,5.3E1,2.3898147E5,5.23896E5,2.9956698E-1,3.342857E2,4E0,2.7496939E1,1.9213884E2,-4.44065E-3,-1.8796328E-4,1.0744694E9,5.375563E-3,4.44E2,1.4E1,2.84755E5,-2.5793985E-2,3.3539724E-1,1.04E2,2.2075728E7,5.6903213E-1,-5.2660346E-2,-2.5042128E-2,5.5345045E-3,-6.736416E-4,-1.7359255E-2,-9.696775E-3,-4.157149E-3,-1.1325553E-2,3.1790106E8,-5.230643E-5,2.0008473E2,1.2059859E1,2.2746666E2,2.2783158E5,4.3873377E-3,-1.3917763E-3,3.9741243E-3,1.2123174E-2,-1.0981158E-3,-6.758073E-3,4.7182865E-3,-4.1826075E-4,3.8895833E-3,-2.9470243E-3,8.981549E-3,3.8890115E-3,2.34E0,1.11E2,2.3320497E-3,-3.7781857E-3,1.365493E6,7.2530017E0,1.4921534E5,1.6605739E-2,3.51E2,-1.839281E-4,2.2577427E-3,-1.9262511E-3,-8.654195E-3,-1.7054573E-2,-2.0408342E-3,-1.1780087E-2,2.0469315E0,8.2884795E3,1.3312868E6,1E0,4.6728007E-3,8.594445E5,9.563096E-3,3.4653E4,8.472942E-3,5.91123E-4,5.862126E2,7.696402E3,5.79E2,1.4955851E7,7.06699E8,1.0409E4,1.0658642E1,3.0505363E-2,-1.4585149E-2,2.37E3,1.0317661E5,2.6460218E-3,1.4189831E-2,3.9512978E5,1.3683E4,-1.7549112E-2,6.2325764E-3,1.952264E-2,3.0635346E-2,1.780913E-2,8.982835E-3,2.0127615E-2,1.803E3,3.0641453E-3,1.0346457E0,7.1106222E-3,1.00860974E3,8.280802E-3,1.4E2,1.8981822E-2,3.0787995E-3,-1.9731605E-3,-2.1097157E-2,-2.9631326E-2,-4.997875E-2,-2.1319982E-2,-3.087875E-3,-1.43915E-2,-1.171799E-2,-3.0232064E-2,-7.540163E-3,-1.8757377E-2,-1.4112784E-3,-9.636405E-3,-2.0085772E-2,-9.094698E-3,-7.747992E-3,-1.7109334E-2,-1.2599825E-2,-4.6377066E-3,-3.0445727E-3,4.6618E-3,-5.398713E-3,9.6289604E-4,-8.350884E-3,-1.5889274E-2,-2.9528367E-3,-1.2215135E-2,-2.9483435E-3,3.5192806E-3,-9.316482E-3,-1.4073234E-3,7.534801E-3,-3.7042527E-3,-2.005417E-3,5.3269197E-3,1.62993E-2,6.1603333E-3,1.1063789E-2,-2.2412245E-3,-1.9723292E-2,1.7584425E-3,3.8095568E-3,-9.301267E-3,1.22277355E-2,5.167837E-3,-1.2969393E-2,-4.3268194E-3,2.7301938E-3,-3.6756485E-3,5.990219E-3,1.3430931E-2,2.4927577E-2,1.3642036E-2,-7.278657E-4,7.9903435E-3,2.329793E-2,6.848657E-3,1.9380015E-5,5.3723613E-3,6.4377068E-3,1.5172349E-2,1.9231508E-2,8.273232E-3,4.4052275E-3,-5.640894E-3,1.6535504E-2,1.0578105E-2,2.6422927E-2,1.3114465E-2,1.095123E-2,-7.4207475E-3,6.878363E-3,1.3807588E-2,1.6926179E-2,2.8860586E-2,1.9085787E-2,2.9090378E-2,3.6975704E-2,2.3693781E-2],"split_indices":[20,102,7,19,52,28,28,17,52,42,2,52,2,7,2,1,52,45,47,52,52,45,57,3,8,54,18,45,41,53,38,0,0,9,28,53,42,0,45,38,45,32,53,38,45,38,0,2,28,28,45,45,54,27,2,52,52,43,52,33,0,0,58,0,0,32,4,39,58,33,0,5,53,12,29,28,55,9,4,28,2,0,0,28,0,0,0,0,7,0,10,0,0,10,64,9,55,52,0,28,7,28,0,17,45,56,0,35,34,48,0,33,0,2,0,0,54,53,0,28,33,18,9,0,32,0,51,0,0,10,0,47,0,31,2,0,56,0,0,0,0,9,52,53,2,27,38,0,0,38,2,52,28,0,58,2,45,0,0,58,47,45,4,0,17,0,0,9,0,12,28,2,0,0,52,1,7,3,33,9,38,52,6,56,58,0,0,12,0,0,8,1,0,27,10,45,39,0,0,0,0,0,0,0,0,7,0,52,56,52,28,0,0,0,0,0,0,0,0,0,0,0,0,54,0,0,0,9,53,28,0,3,0,0,0,0,0,0,0,56,47,28,16,0,28,0,9,0,0,52,47,29,47,7,2,53,0,0,10,28,0,0,28,9,0,0,0,0,0,0,0,2,0,53,0,4,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.581E3,8.47E2,1.734E3,4.73E2,3.74E2,1.096E3,6.38E2,6.2E1,4.11E2,3E2,7.4E1,5.56E2,5.4E2,4.52E2,1.86E2,9E0,5.3E1,3.77E2,3.4E1,2.25E2,7.5E1,4.8E1,2.6E1,4.94E2,6.2E1,2.8E2,2.6E2,3.03E2,1.49E2,5E1,1.36E2,4E0,5E0,1.7E1,3.6E1,3.47E2,3E1,7E0,2.7E1,6.4E1,1.61E2,2.3E1,5.2E1,1.4E1,3.4E1,2.2E1,4E0,1.4E1,4.8E2,1.1E1,5.1E1,2.1E2,7E1,2.9E1,2.31E2,2.63E2,4E1,1.09E2,4E1,4.6E1,4E0,4E0,1.32E2,1E1,7E0,1.7E1,1.9E1,1.97E2,1.5E2,2.5E1,5E0,1.6E1,1.1E1,4.3E1,2.1E1,9.4E1,6.7E1,9E0,1.4E1,3E1,2.2E1,9E0,5E0,2.7E1,7E0,1.7E1,5E0,5E0,9E0,2.06E2,2.74E2,4E0,7E0,2.9E1,2.2E1,1.9E1,1.91E2,6.5E1,5E0,1.5E1,1.4E1,2.25E2,6E0,1.94E2,6.9E1,3.5E1,5E0,8.7E1,2.2E1,2.9E1,1.1E1,3.9E1,7E0,1.28E2,4E0,6E0,1.1E1,1.4E1,5E0,1.68E2,2.9E1,8E1,7E1,4E0,2.1E1,5E0,1.1E1,7E0,4E0,3.6E1,7E0,1.5E1,6E0,5.3E1,4.1E1,1.1E1,5.6E1,5E0,4E0,6E0,8E0,9E0,2.1E1,1E1,1.2E1,1.1E1,1.6E1,5E0,4E0,1.94E2,1.2E1,2.4E2,3.4E1,5E0,2.4E1,8E0,1.4E1,1.4E1,5E0,1.06E2,8.5E1,4.2E1,2.3E1,6E0,9E0,1E1,4E0,2.09E2,1.6E1,4.7E1,1.47E2,6.2E1,7E0,6E0,2.9E1,5E1,3.7E1,1.4E1,8E0,1.9E1,1E1,1.5E1,2.4E1,5.7E1,7.1E1,6E0,5E0,9E0,5E0,1.57E2,1.1E1,2E1,9E0,3.2E1,4.8E1,2.8E1,4.2E1,1.7E1,4E0,4E0,7E0,2.9E1,7E0,5E0,1E1,4.8E1,5E0,1E1,3.1E1,1E1,4.6E1,4E0,5E0,5E0,1.6E1,5E0,5E0,6E0,6E0,6E0,5E0,1E1,6E0,1.86E2,8E0,7E0,5E0,1.49E2,9.1E1,2.9E1,5E0,2E1,4E0,4E0,4E0,9E0,5E0,7E0,7E0,4.3E1,6.3E1,9E0,7.6E1,5E0,3.7E1,8E0,1.5E1,5E0,4E0,1.4E2,6.9E1,2.3E1,2.4E1,1.14E2,3.3E1,5.4E1,8E0,1.2E1,1.7E1,4.4E1,6E0,7E0,3E1,1E1,4E0,4E0,4E0,1E1,9E0,5E0,5E0,9E0,6E0,1.9E1,5E0,5.1E1,6E0,6.6E1,5E0,4E0,5E0,5.6E1,1.01E2,7E0,4E0,6E0,1.4E1,8E0,2.4E1,8E0,4E1,1.1E1,1.7E1,2.8E1,1.4E1,4.1E1,7E0,4E0,6E0,1.6E1,1.5E1,4E0,6E0,2.9E1,1.7E1,1.5E1,1.71E2,4E0,4E0,9.1E1,5.8E1,1.8E1,7.3E1,1.7E1,1.2E1,1.3E1,7E0,3.3E1,1E1,4E0,5.9E1,5E0,4E0,4.2E1,3.4E1,2.3E1,1.4E1,4E0,1.1E1,9.4E1,4.6E1,1E1,5.9E1,6E0,1.7E1,2E1,4E0,7.2E1,4.2E1,2.4E1,9E0,3.7E1,1.7E1,4E0,1.3E1,2.3E1,2.1E1,2.6E1,4E0,4E0,6E0,5E0,4E0,8E0,1.1E1,1.9E1,3.2E1,5.6E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"359","size_leaf_vector":"1"}},{"base_weights":[-2.0614485E-3,-2.3991194E-1,1.15550525E-1,-3.7470463E-1,-6.899058E-2,-2.4700002E-3,2.6412627E-1,-4.8248923E-1,-2.2664677E-1,-1.5786293E-1,2.1518001E-1,-1.3596451E-1,9.288763E-2,1.8261427E-1,4.1688704E-1,-4.1812336E-1,-7.844664E-1,4.1526157E-2,-2.522123E-1,-2.4801622E-1,-7.616088E-2,1.22045755E-1,3.9738032E-1,-2.915138E-1,-8.855526E-2,1.1798001E-1,-2.5490713E-1,1.0528034E-1,3.5467657E-1,2.5098976E-1,4.9426287E-1,-4.576195E-1,-2.7564383E-1,-7.026328E-1,-5.4742284E-2,-6.0638584E-2,-3.391015E-1,-2.6563498E-1,3.0119048E-2,-1.36505E-1,-5.394979E-3,1.6758016E-1,-1.0021676E-2,4.692539E-1,2.3057562E-1,-2.631269E-1,-3.6900274E-2,1.9292302E-1,-1.1850434E-1,5.5509023E-2,2.2450806E-1,-3.3012873E-1,6.264739E-3,1.486319E-1,-1.7274943E-1,2.7690297E-1,5.694088E-1,2.730325E-1,-8.030081E-3,5.208595E-1,8.072167E-2,-3.8827595E-1,-5.350109E-1,-1.2473033E-1,-4.3364117E-1,-1.7532203E-2,-7.666148E-1,-2.0610934E-2,-2.1300563E-1,-4.1781288E-1,-3.427623E-2,-2.913882E-2,-2.5068912E-1,3.96891E-3,-1.2600024E-3,-1.632451E-1,5.0358674E-3,2.7463928E-2,-1.1732567E-2,2.759935E-1,1.0570701E-1,5.09021E-1,1.2399153E-2,7.143546E-3,1.3978462E-2,-2.783757E-1,2.4347645E-3,-9.934966E-3,5.979334E-1,-1.5732764E-1,2.5822816E-2,-2.5403852E-2,1.7615679E-1,2.0645903E-1,4.306429E-2,-3.802057E-1,-6.6141933E-3,9.60782E-2,3.25747E-1,-2.2899878E-1,1.7265726E-2,3.0962136E-1,-1.3597258E-2,4.6671748E-1,4.6060983E-2,2.4696916E-1,2.5832828E-2,4.8731276E-1,3.7889145E-2,9.522833E-3,-5.5563897E-3,-2.5063053E-1,-4.4913015E-1,-7.4391556E-1,-4.7214335E-1,3.263624E-2,-2.009901E-1,-7.045545E-3,-5.186313E-1,-8.7558866E-1,-2.885443E-2,-5.7973493E-2,1.1118651E-2,-5.229612E-3,-1.4536296E-2,-3.626556E-1,-5.864289E-1,1.037391E-2,-1.2067925E-1,-2.1012677E-2,-2.2345789E-1,-1.22603506E-1,-2.2574453E-1,7.583714E-2,-5.0968677E-3,1.2865469E-1,-2.6194658E-2,3.3851704E-1,1.6600982E-3,2.4885602E-2,1.6911986E-1,2.9100168E-2,1.4473905E-2,-2.3798116E-1,-4.2595485E-1,1.5691288E-2,3.3608604E-2,-2.0300207E-1,-5.655384E-2,7.43275E-2,-9.148408E-2,-7.077714E-2,9.774251E-2,1.3201873E-1,3.2329682E-1,2.3267229E-1,-9.151389E-2,-3.0504632E-1,-2.8349554E-2,1.6586138E-1,2.6070273E-2,2.7431408E-1,3.3837665E-2,-1.7135677E-1,-2.7776392E-2,3.45732E-1,1.07854605E-1,1.9329874E-2,2.8692767E-2,2.8180993E-1,1.5472802E-2,3.1658554E-1,5.157391E-1,-7.3895796E-4,-2.83794E-1,-5.1435566E-1,-2.9529935E-1,-4.4921596E-2,-2.5290256E-2,-5.044959E-1,-3.1825218E-1,6.493177E-3,-4.5303633E-3,-1.553064E-2,-9.285742E-2,-1.3322953E-2,-2.8383765E-2,-4.7880437E-2,-2.260964E-2,4.2588707E-2,-1.0015778E-1,-4.082149E-1,-2.4532275E-1,-4.0179056E-1,-4.3472566E-2,-4.260752E-2,-1.2590543E-2,-2.0157991E-1,-2.3707671E-2,2.730837E-3,-1.416955E-1,-2.6281688E-1,-4.3757986E-3,5.1619047E-3,1.6588877E-3,1.1681323E-2,5.1883023E-2,-1.0677165E-1,2.252484E-2,1.0522276E-2,2.0006955E-2,-4.3381085E-3,7.866463E-2,4.17936E-3,2.0871182E-1,-2.7883542E-1,-1.380273E-1,-2.3524055E-2,-1.1201505E-2,-3.648278E-1,-1.7974934E-1,-2.4996E-3,-1.6005366E-1,5.378497E-3,1.611379E-1,8.65407E-4,-1.6229735E-1,-4.538071E-2,-4.1163045E-1,1.2377588E-1,-9.100913E-3,5.39397E-2,2.4152762E-1,1.80235E-1,2.0419294E-2,1.8714282E-1,3.6293793E-1,-4.3956363E-3,-1.4105337E-2,-8.244446E-3,-1.6476296E-2,2.8717995E-1,1.1205637E-1,-6.5241344E-2,6.2719375E-2,3.6047918E-1,1.5988813E-1,-2.5981602E-1,-7.875264E-2,2.6646912E-1,4.793696E-1,6.8720877E-3,1.8439436E-2,1.5396911E-1,3.3804744E-1,-1.0365203E-2,8.618626E-3,7.141209E-3,3.7968808E-1,6.518581E-1,4.6692023E-1,-1.1356622E-2,-2.4897562E-2,-2.928445E-2,-1.6580714E-2,-1.8244823E-2,-2.4084537E-3,-2.63212E-2,-9.479447E-3,-8.689405E-3,-2.187325E-2,-6.740613E-4,-7.832195E-3,-3.4162996E-3,4.6873717E-3,1.382182E-3,-6.531573E-3,-2.1748198E-2,-7.2538117E-3,-1.3517577E-2,-4.2640558E-3,-7.8749675E-3,-2.6015412E-2,-4.229721E-3,2.3792914E-4,-1.6671801E-2,-8.6506E-3,-8.11019E-3,-1.9457281E-3,-7.911925E-3,-1.6683856E-2,5.9818584E-3,6.384614E-4,-9.2397066E-4,-7.889258E-3,-7.1449496E-4,4.324917E-3,1.2699928E-4,9.2302635E-3,1.2502129E-2,4.6106167E-3,-8.311054E-3,-1.5379086E-2,-9.268369E-3,-3.6825796E-3,1.9262414E-3,-2.4125524E-2,-1.2692009E-2,-6.7628976E-3,-5.789337E-3,4.672895E-3,6.455285E-4,-1.1594151E-2,4.4539776E-3,-2.818341E-3,-2.4622185E-3,1.1019335E-2,-1.0091959E-2,-3.7120658E-3,6.905319E-4,-4.5814686E-3,-2.6202451E-2,-9.69266E-3,1.0139487E-2,-1.4094257E-3,-9.381188E-3,5.1979744E-3,2.090845E-3,1.3038049E-2,3.6162036E-3,1.1922178E-2,5.007785E-3,1.1007153E-2,2.0088378E-2,2.4140615E-3,-2.0584892E-3,2.7664583E-3,2.8857836E-2,1.1546474E-2,7.7127703E-3,-1.58321E-4,-4.9969424E-3,3.4111048E-4,2.338601E-3,9.501342E-3,1.198688E-2,2.0753482E-2,4.9503343E-3,1.1628207E-2,-1.4268944E-2,-6.5253046E-3,8.53921E-3,-7.587261E-3,1.4327594E-2,1.8094603E-3,2.5595518E-2,1.2293011E-2,-3.1129064E-3,5.470569E-3,3.87794E-3,1.0036439E-2,1.898511E-2,6.405621E-3,2.0878177E-2,1.0220685E-2,3.4608014E-2,1.5058012E-2,1.723457E-2,2.7132254E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,-1,81,83,85,-1,87,89,91,93,95,-1,97,99,101,103,105,-1,107,109,111,113,115,117,-1,119,121,123,125,127,-1,129,-1,-1,131,133,135,-1,137,139,141,-1,-1,-1,143,-1,-1,145,147,149,151,153,155,-1,157,-1,159,161,163,-1,165,-1,167,-1,169,-1,171,-1,-1,-1,173,175,177,179,181,183,-1,185,187,-1,189,-1,-1,-1,191,193,-1,195,-1,197,199,201,203,-1,205,207,209,-1,211,213,-1,-1,215,217,-1,-1,219,221,223,225,227,229,231,233,235,237,239,-1,241,243,245,-1,247,-1,249,251,-1,-1,253,255,257,259,-1,261,263,265,-1,-1,267,269,-1,-1,-1,271,-1,-1,-1,-1,273,275,277,279,281,-1,283,-1,285,-1,-1,287,289,-1,-1,-1,-1,291,293,295,-1,-1,-1,297,-1,299,301,303,-1,-1,305,307,309,311,313,315,-1,317,319,321,323,-1,325,327,329,-1,331,333,335,-1,-1,-1,337,339,341,343,345,347,349,351,353,355,357,-1,359,361,-1,-1,-1,363,365,367,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.186513E1,1.9557259E1,3.0143295E1,7.4708633E0,9.529814E0,1.2220431E1,9.399719E0,4.9977455E0,5.6719446E0,2.09507E0,1.4787197E0,2.9239745E0,4.917192E0,6.5891743E0,3.2647629E0,1.1569023E0,4.2679787E-1,0E0,3.2752838E0,6.744547E-1,6.4835006E-1,9.297006E-1,2.5146723E-1,9.857116E-1,2.619307E0,3.4680572E0,1.128407E0,4.18397E0,2.4142056E0,8.243251E-1,1.9267426E0,7.42157E-1,1.1857963E0,6.312504E-1,0E0,3.7984014E-1,3.2487564E0,4.6126366E-1,2.6561573E-2,3.1117702E-1,5.41665E-1,3.3305085E-1,0E0,4.9541473E-2,7.3994994E-3,4.367957E-1,0E0,4.878832E0,1.5673714E0,3.2356827E0,2.0422564E0,2.6656866E-1,0E0,2.7518115E0,1.4493543E0,2.1500435E0,1.0174046E0,3.9372492E-1,0E0,9.685898E-1,2.742147E-1,6.999359E-1,7.1256256E-1,3.4258017E-1,5.4865026E-1,0E0,1.745491E-1,4.761867E-1,7.929134E-2,7.9177094E-1,6.3116294E-1,0E0,4.7953892E-1,0E0,0E0,1.4758825E-1,1.1383697E-1,3.454445E-1,0E0,2.7715027E-1,1.8029553E-1,8.8799E-2,0E0,0E0,0E0,4.053483E-1,0E0,0E0,1.0682678E-1,1.001132E0,3.4819576E-1,1.1188414E0,8.21084E-1,1.4975414E0,0E0,1.7860389E-1,0E0,1.1290832E0,1.0434351E0,7.149985E-1,0E0,7.5687695E-1,0E0,6.1217785E-2,0E0,6.0861826E-1,0E0,5.6469727E-1,0E0,0E0,0E0,2.4608517E-1,5.103359E-1,1.7196178E-1,1.6306305E-1,1.317742E-1,1.9678599E-1,0E0,8.5166454E-2,2.3087502E-1,0E0,1.951621E-1,0E0,0E0,0E0,3.4888744E-1,9.5952606E-1,0E0,2.147811E-1,0E0,5.02944E-1,1.5664816E-1,1.1016357E-1,7.111363E-3,0E0,1.6542876E-1,1.6728318E-1,3.444314E-2,0E0,1.0895583E-1,5.022669E-2,0E0,0E0,2.4327755E-1,4.8267126E-2,0E0,0E0,5.026188E-1,3.9122218E-1,2.565094E-1,1.4149645E-1,1.2451625E0,4.1600347E-1,8.762082E-1,2.9270482E-1,9.640608E-1,2.646873E-1,1.4773965E-2,0E0,7.2501516E-1,3.9563107E-1,5.4379416E-1,0E0,2.908231E-1,0E0,8.4677315E-1,4.7220722E-1,0E0,0E0,4.2245483E-1,4.2079177E-1,2.0874572E-1,5.3956985E-1,0E0,2.2613072E-1,3.9461613E-1,3.6206412E-1,0E0,0E0,2.6474285E-1,1.6537619E-1,0E0,0E0,0E0,5.366475E-2,0E0,0E0,0E0,0E0,8.59833E-2,1.2861693E-1,3.5644722E-1,8.177328E-2,4.1137004E-1,0E0,3.027653E-2,0E0,3.2554674E-1,0E0,0E0,7.841742E-2,9.5653534E-2,0E0,0E0,0E0,0E0,3.5607602E-2,6.9284484E-2,6.490107E-2,0E0,0E0,0E0,9.641714E-2,0E0,3.9726794E-2,1.05148315E-1,5.3499103E-2,0E0,0E0,8.75422E-1,4.074502E-1,5.2244145E-1,3.0182868E-1,1.34402E-1,2.4832514E-1,0E0,2.3404807E-2,3.7686166E-1,1.437136E-1,6.0719097E-1,0E0,7.701018E-1,1.5571976E-1,6.457573E-2,0E0,3.5535336E-1,5.2182436E-1,2.867286E-2,0E0,0E0,0E0,4.3662882E-1,4.001764E-1,8.5865915E-2,1.6147423E-1,1.2530375E-1,1.00963056E-1,2.5936961E-2,3.909089E-1,2.6970768E-1,1.271801E-1,1.0619103E-1,0E0,6.318694E-2,3.4703827E-1,0E0,0E0,0E0,2.8128147E-2,2.3418808E-1,7.1310997E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,69,69,70,70,72,72,75,75,76,76,77,77,79,79,80,80,81,81,85,85,88,88,89,89,90,90,91,91,92,92,93,93,95,95,97,97,98,98,99,99,101,101,103,103,105,105,107,107,111,111,112,112,113,113,114,114,115,115,116,116,118,118,119,119,121,121,125,125,126,126,128,128,130,130,131,131,132,132,133,133,135,135,136,136,137,137,139,139,140,140,143,143,144,144,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,159,159,160,160,161,161,163,163,165,165,166,166,169,169,170,170,171,171,172,172,174,174,175,175,176,176,179,179,180,180,184,184,189,189,190,190,191,191,192,192,193,193,195,195,197,197,200,200,201,201,206,206,207,207,208,208,212,212,214,214,215,215,216,216,219,219,220,220,221,221,222,222,223,223,224,224,226,226,227,227,228,228,229,229,231,231,232,232,233,233,235,235,236,236,237,237,241,241,242,242,243,243,244,244,245,245,246,246,247,247,248,248,249,249,250,250,251,251,253,253,254,254,258,258,259,259,260,260],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,-1,82,84,86,-1,88,90,92,94,96,-1,98,100,102,104,106,-1,108,110,112,114,116,118,-1,120,122,124,126,128,-1,130,-1,-1,132,134,136,-1,138,140,142,-1,-1,-1,144,-1,-1,146,148,150,152,154,156,-1,158,-1,160,162,164,-1,166,-1,168,-1,170,-1,172,-1,-1,-1,174,176,178,180,182,184,-1,186,188,-1,190,-1,-1,-1,192,194,-1,196,-1,198,200,202,204,-1,206,208,210,-1,212,214,-1,-1,216,218,-1,-1,220,222,224,226,228,230,232,234,236,238,240,-1,242,244,246,-1,248,-1,250,252,-1,-1,254,256,258,260,-1,262,264,266,-1,-1,268,270,-1,-1,-1,272,-1,-1,-1,-1,274,276,278,280,282,-1,284,-1,286,-1,-1,288,290,-1,-1,-1,-1,292,294,296,-1,-1,-1,298,-1,300,302,304,-1,-1,306,308,310,312,314,316,-1,318,320,322,324,-1,326,328,330,-1,332,334,336,-1,-1,-1,338,340,342,344,346,348,350,352,354,356,358,-1,360,362,-1,-1,-1,364,366,368,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5125709E8,1.0443182E3,6.860185E2,2.0407547E5,8.3956606E5,4.3702424E7,2.66E2,1.529448E-1,1.3664E4,1.1317E4,8.35471E-2,1.3359244E9,3.689E3,2.074E3,3.4231E4,4.1526157E-2,4.0844156E7,1E0,3.2485715E2,6.1061732E7,2.4848485E0,3.7852024E1,2.04E5,3.3926086E2,2.200423E3,3.9712732E7,1.6706144E0,1.9761398E7,2.545455E7,7.667683E6,1.9E1,6.5029144E5,-5.4742284E-2,5.6202265E6,3.4409692E3,1E0,2.4013288E1,9.705292E6,1.164249E7,6.426937E6,-1.0021676E-2,4.7741196E-1,6.08767E3,3.83E3,-3.6900274E-2,1.91E2,1.101214E6,1.5859042E6,1.1392E4,2.6476662E6,6.264739E-3,1.9214156E3,3.970405E3,1.921032E4,2.4507338E4,1.116567E3,-8.030081E-3,7.780377E6,1.7243155E12,1.6904992E-1,1E0,7.5677685E-2,7.9484855E6,-1.7532203E-2,4.6547272E2,1.9214156E3,2.5003334E7,1.1855755E8,3.3517068E7,-2.913882E-2,1.4123257E-4,3.96891E-3,-1.2600024E-3,6.0941644E0,2.8807552E7,1.683274E0,-1.1732567E-2,2.5482938E0,3.6482175E5,1.1034263E10,1.2399153E-2,7.143546E-3,1.3978462E-2,4.67E2,2.4347645E-3,-9.934966E-3,1.5561539E2,1.566593E6,3.841808E-2,3.422351E6,6.789622E7,1.4383178E1,4.306429E-2,8.4503644E5,-6.6141933E-3,9.161318E0,5.185489E3,1.3664E4,1.7265726E-2,2.851369E2,-1.3597258E-2,7.466243E0,4.6060983E-2,1.531154E8,2.5832828E-2,7.387217E2,3.7889145E-2,9.522833E-3,-5.5563897E-3,1.2E1,1.048E3,9E0,2.165022E2,1.1E1,1.710643E0,-7.045545E-3,1.2E1,9.3299186E-1,-2.885443E-2,1.6837112E7,1.1118651E-2,-5.229612E-3,-1.4536296E-2,2.6835732E3,9.190713E-1,1.037391E-2,9.123214E-1,-2.1012677E-2,1.5988282E1,1.8544E4,3.8E1,4.5880157E2,-5.0968677E-3,3.2931E4,9.7637427E-1,9.910549E8,1.6600982E-3,1E0,1.25218E5,2.9100168E-2,1.4473905E-2,3.3151623E1,7.88E2,1.5691288E-2,3.3608604E-2,3.3820656E7,2.1E1,9.262083E2,1E0,9.457831E0,1E0,2.4757753E2,2.1827742E6,4.264897E6,3.2627738E5,3.6873734E5,-2.8349554E-2,2.72258E5,1.4685706E3,5.8E1,3.3837665E-2,1.5106794E3,-2.7776392E-2,6.4683E4,4.2892E5,1.9329874E-2,2.8692767E-2,1.513E3,3.14088E5,6.077036E8,1.0993947E1,-7.3895796E-4,1.627E3,1.7314286E0,2.7302878E2,-4.4921596E-2,-2.5290256E-2,3.1530054E0,2.2007043E0,6.493177E-3,-4.5303633E-3,-1.553064E-2,1.5316E4,-1.3322953E-2,-2.8383765E-2,-4.7880437E-2,-2.260964E-2,5.5949634E-1,5.012E3,1.1392E4,2.6036014E5,1.2E1,-4.3472566E-2,3E1,-1.2590543E-2,1.04E3,-2.3707671E-2,2.730837E-3,3.6482175E5,2.4434292E7,-4.3757986E-3,5.1619047E-3,1.6588877E-3,1.1681323E-2,5.48744E8,2.1975278E8,2E0,1.0522276E-2,2.0006955E-2,-4.3381085E-3,1.5311552E9,4.17936E-3,2.109E3,1.901E3,8.209663E3,-2.3524055E-2,-1.1201505E-2,1.61E2,1.9676556E3,1.2513207E2,1.04E2,1.7488463E3,4.6647886E10,8.65407E-4,4.70092E1,6.3E0,7.696629E0,1.1304221E0,-9.100913E-3,2.5247778E6,5.26E2,1.813943E6,2.0419294E-2,1E0,1.363176E7,2.5332516E5,-1.4105337E-2,-8.244446E-3,-1.6476296E-2,1.752E3,4.271705E1,5.121E3,5.54E2,2.091623E5,1.517E3,6.884212E4,2.37E3,4.2343444E7,2.878424E5,9.9668306E-1,1.8439436E-2,1.0927339E7,5.077728E8,-1.0365203E-2,8.618626E-3,7.141209E-3,7.697189E2,2E1,7.883E3,-1.1356622E-2,-2.4897562E-2,-2.928445E-2,-1.6580714E-2,-1.8244823E-2,-2.4084537E-3,-2.63212E-2,-9.479447E-3,-8.689405E-3,-2.187325E-2,-6.740613E-4,-7.832195E-3,-3.4162996E-3,4.6873717E-3,1.382182E-3,-6.531573E-3,-2.1748198E-2,-7.2538117E-3,-1.3517577E-2,-4.2640558E-3,-7.8749675E-3,-2.6015412E-2,-4.229721E-3,2.3792914E-4,-1.6671801E-2,-8.6506E-3,-8.11019E-3,-1.9457281E-3,-7.911925E-3,-1.6683856E-2,5.9818584E-3,6.384614E-4,-9.2397066E-4,-7.889258E-3,-7.1449496E-4,4.324917E-3,1.2699928E-4,9.2302635E-3,1.2502129E-2,4.6106167E-3,-8.311054E-3,-1.5379086E-2,-9.268369E-3,-3.6825796E-3,1.9262414E-3,-2.4125524E-2,-1.2692009E-2,-6.7628976E-3,-5.789337E-3,4.672895E-3,6.455285E-4,-1.1594151E-2,4.4539776E-3,-2.818341E-3,-2.4622185E-3,1.1019335E-2,-1.0091959E-2,-3.7120658E-3,6.905319E-4,-4.5814686E-3,-2.6202451E-2,-9.69266E-3,1.0139487E-2,-1.4094257E-3,-9.381188E-3,5.1979744E-3,2.090845E-3,1.3038049E-2,3.6162036E-3,1.1922178E-2,5.007785E-3,1.1007153E-2,2.0088378E-2,2.4140615E-3,-2.0584892E-3,2.7664583E-3,2.8857836E-2,1.1546474E-2,7.7127703E-3,-1.58321E-4,-4.9969424E-3,3.4111048E-4,2.338601E-3,9.501342E-3,1.198688E-2,2.0753482E-2,4.9503343E-3,1.1628207E-2,-1.4268944E-2,-6.5253046E-3,8.53921E-3,-7.587261E-3,1.4327594E-2,1.8094603E-3,2.5595518E-2,1.2293011E-2,-3.1129064E-3,5.470569E-3,3.87794E-3,1.0036439E-2,1.898511E-2,6.405621E-3,2.0878177E-2,1.0220685E-2,3.4608014E-2,1.5058012E-2,1.723457E-2,2.7132254E-2],"split_indices":[20,102,7,4,52,28,28,45,0,39,2,9,27,7,2,2,10,0,45,74,52,45,57,53,5,52,52,45,42,45,45,45,8,28,0,47,52,67,58,9,45,45,0,38,4,2,0,2,1,28,2,32,0,52,52,4,4,4,0,28,31,27,64,39,45,0,52,52,45,45,51,0,42,0,0,53,9,53,0,42,28,5,0,0,0,2,0,0,52,9,57,9,7,53,0,32,0,54,52,2,0,58,0,35,0,12,0,52,0,0,0,18,2,8,52,3,53,0,18,27,0,45,0,0,0,4,27,0,27,0,54,9,3,4,0,9,42,5,0,57,29,0,0,55,0,0,0,7,3,52,15,54,74,55,28,9,28,28,0,9,55,3,0,52,0,2,11,0,0,2,1,7,56,0,2,53,4,0,0,53,54,0,0,0,9,0,0,0,0,39,2,2,33,18,0,8,0,2,0,0,28,12,0,0,0,0,5,7,8,0,0,0,12,0,0,9,28,0,0,12,33,52,10,4,31,0,58,58,56,38,0,28,2,9,0,102,9,28,0,0,0,29,58,2,8,28,0,28,10,45,33,39,0,9,7,0,0,0,4,3,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.567E3,8.49E2,1.718E3,4.74E2,3.75E2,9.58E2,7.6E2,2.73E2,2.01E2,2.86E2,8.9E1,3.99E2,5.59E2,4.97E2,2.63E2,2.27E2,4.6E1,4E0,1.97E2,1.35E2,1.51E2,6E1,2.9E1,9.2E1,3.07E2,5.22E2,3.7E1,3.44E2,1.53E2,8.5E1,1.78E2,1.76E2,5.1E1,3.9E1,7E0,6.2E1,1.35E2,1.27E2,8E0,8.1E1,7E1,5.3E1,7E0,1.9E1,1E1,8.8E1,4E0,2.9E1,2.78E2,3.3E2,1.92E2,3.1E1,6E0,2.98E2,4.6E1,1.14E2,3.9E1,8.1E1,4E0,1.67E2,1.1E1,9.6E1,8E1,2.7E1,2.4E1,7E0,3.2E1,5E1,1.2E1,1.07E2,2.8E1,4E0,1.23E2,4E0,4E0,6.8E1,1.3E1,6.2E1,8E0,1.8E1,3.5E1,1.5E1,4E0,5E0,5E0,8.4E1,4E0,1.5E1,1.4E1,2.19E2,5.9E1,1.98E2,1.32E2,1.88E2,4E0,2.4E1,7E0,2.31E2,6.7E1,4.2E1,4E0,1.08E2,6E0,3.2E1,7E0,7.5E1,6E0,1.49E2,1.8E1,7E0,4E0,3.1E1,6.5E1,1.6E1,6.4E1,9E0,1.8E1,6E0,1.8E1,1.8E1,1.4E1,4.4E1,6E0,6E0,6E0,8.3E1,2.4E1,7E0,2.1E1,1.5E1,1.08E2,4.3E1,2.5E1,8E0,5E0,2.1E1,4.1E1,1.4E1,4E0,1.6E1,1.9E1,1E1,5E0,6.8E1,1.6E1,4E0,1E1,1.5E2,6.9E1,4.2E1,1.7E1,1.45E2,5.3E1,1.03E2,2.9E1,1.73E2,1.5E1,1.9E1,5E0,1.15E2,1.16E2,6E1,7E0,3.7E1,5E0,9.1E1,1.7E1,2.1E1,1.1E1,6.5E1,1E1,2.3E1,1.26E2,4E0,2.7E1,4.4E1,2.1E1,8E0,8E0,5.1E1,1.3E1,5E0,4E0,8E0,1E1,4E0,1.4E1,1.4E1,4E0,1.3E1,3.1E1,5.8E1,2.5E1,1.6E1,8E0,1.4E1,7E0,1.01E2,7E0,4E0,3.9E1,1.9E1,6E0,4E0,4E0,8E0,1.3E1,1.5E1,2.6E1,6E0,8E0,5E0,1.1E1,7E0,1.2E1,4.7E1,2.1E1,1.2E1,4E0,1.7E1,1.33E2,4.6E1,2.3E1,2.4E1,1.8E1,7E0,1E1,1.36E2,9E0,4.9E1,4E0,6.1E1,4.2E1,1.2E1,1.7E1,1.3E2,4.3E1,1.1E1,4E0,4E0,1.5E1,3.4E1,8.1E1,3.3E1,8.3E1,3.3E1,2.7E1,1.8E1,1.9E1,5.9E1,3.2E1,1.3E1,4E0,2.1E1,4.4E1,4E0,6E0,7E0,1.6E1,3E1,9.6E1,2.3E1,4E0,3E1,1.4E1,1.6E1,5E0,4.7E1,4E0,7E0,6E0,5E0,5E0,4E0,9E0,6E0,2.5E1,5.2E1,6E0,2.1E1,4E0,6E0,1E1,7E0,7E0,1.6E1,8.5E1,3.2E1,7E0,9E0,1E1,4E0,9E0,6E0,9E0,1.7E1,9E0,7E0,4E0,8E0,4E0,1.1E1,3.6E1,1.1E1,1E1,4E0,1.3E1,4.8E1,8.5E1,2.1E1,2.5E1,7E0,1.6E1,1E1,1.4E1,4E0,1.4E1,6E0,4E0,6E1,7.6E1,5E0,4E0,3.2E1,1.7E1,1E1,5.1E1,4E0,3.8E1,5E0,7E0,3.7E1,9.3E1,3.8E1,5E0,7E0,4E0,4E0,3E1,5.9E1,2.2E1,2.2E1,1.1E1,7.5E1,8E0,1.2E1,2.1E1,1.6E1,1.1E1,1.4E1,4E0,4E0,1.5E1,5.4E1,5E0,2.7E1,5E0,8E0,5E0,9E0,1.2E1,3.6E1,8E0,1.2E1,4E0,2.6E1,4E0,3.9E1,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"369","size_leaf_vector":"1"}},{"base_weights":[3.1077727E-3,-2.1360423E-1,1.03060395E-1,-3.3536246E-1,-6.0813073E-2,-1.0090546E-3,2.4566503E-1,-3.6314452E-1,2.8010872E-1,-1.4756806E-1,1.7593054E-1,-6.558675E-2,1.5458758E-1,1.1543905E-1,3.382441E-1,-1.362961E-1,-4.0600705E-1,4.5180824E-2,-5.1954318E-2,-1.8564479E-1,-1.5013108E-2,9.312576E-2,4.2391485E-1,-7.349046E-2,3.7360214E-2,1.7026913E-1,-3.1406772E-1,1.7021877E-1,-1.1956344E-1,1.433977E-1,4.027503E-1,-2.3560749E-1,5.1888708E-2,-4.5476776E-1,-2.6214918E-1,3.5694934E-2,-8.732631E-3,-1.3051736E-1,-2.5327125E-1,7.381684E-2,-1.12514675E-1,1.3428691E-1,-1.8373427E-1,4.5334482E-1,1.1063214E-2,-2.2120829E-1,-3.6949024E-2,1.4488116E-1,4.1126046E-1,-2.0691572E-2,-6.5559484E-3,1.2300799E-1,4.0902984E-1,-2.93455E-1,3.6328048E-2,1.0096378E-1,4.0258604E-1,3.5434732E-1,5.483755E-1,-7.465838E-4,-2.6215354E-1,-1.989452E-2,1.7138593E-2,-3.585561E-1,-5.3457254E-1,-3.2741964E-1,-1.1199155E-1,-2.5292337E-3,5.7396363E-3,-1.6751805E-1,-2.485364E-2,-1.7984231E-1,-3.7108326E-1,1.1099054E-2,3.1047499E-2,-2.3871714E-1,-3.352826E-2,2.319455E-1,8.890737E-2,-1.4292294E-2,-1.2104245E-3,2.497929E-2,1.3058877E-2,-2.4810492E-1,6.270497E-2,-1.4520457E-1,6.1792815E-3,9.728852E-2,2.674197E-1,9.424398E-3,2.7608668E-2,9.013547E-2,2.457256E-1,5.413058E-4,4.64034E-1,-3.410666E-1,-3.0996678E-3,1.0027111E-1,-1.2864939E-2,2.1348658E-1,4.0420655E-2,2.302883E-2,1.1908323E-2,3.103545E-1,6.0235107E-1,5.780304E-1,4.558724E-3,-3.517084E-1,-1.2818292E-1,7.186908E-2,-1.09849535E-1,-3.0829722E-1,-5.295588E-1,-5.641378E-1,-2.40731E-1,-2.8974998E-1,-2.7961204E-2,4.853875E-2,-1.9468363E-1,3.3415486E-3,-1.830036E-1,6.0816883E-4,-7.863588E-3,-2.8532586E-1,-1.12261765E-1,-5.510068E-1,-2.1758643E-1,6.527754E-3,-5.3630136E-3,-5.1401635E-3,-1.5072297E-2,-6.669757E-2,1.6419858E-3,1.345139E-1,1.967056E-2,1.6966477E-2,1.4865905E-1,-2.2141139E-1,-2.7162708E-2,9.95965E-3,-2.0162754E-2,-1.1754589E-1,-6.0541487E-1,2.8897377E-2,-1.4012991E-2,1.06410064E-1,-1.31569635E-2,1.8482426E-1,4.096182E-1,3.919294E-2,1.7230469E-1,1.8277122E-2,1.4853796E-1,1.0808066E-2,2.5801374E-2,-1.8505141E-2,-8.073482E-3,4.5824596E-3,2.0296794E-1,2.3796009E-1,1.4411849E-3,8.169882E-2,-5.9470806E-2,3.3137685E-1,-1.9829997E-1,7.9634476E-1,3.8686487E-1,6.185844E-1,8.301306E-3,-3.9923537E-1,-8.9566335E-3,-2.535685E-2,-1.0388474E-2,6.1479243E-3,4.356833E-4,-8.1276195E-3,-1.9352075E-3,-4.218871E-1,-2.0582625E-1,-6.2857455E-1,-3.2839274E-1,-5.151699E-1,-7.9610443E-1,-4.155476E-3,-3.1119752E-1,-3.6716416E-1,-1.99675E-1,-4.1822044E-4,5.921211E-3,-2.4656187E-1,5.128314E-4,-3.08455E-1,-1.6067547E-1,-7.129296E-2,4.0519454E-2,-1.6549338E-2,-1.822514E-1,5.4401974E-3,-1.689763E-1,-1.0568104E-2,-3.273992E-2,-2.854969E-1,-3.999012E-3,2.4935003E-2,-5.0588287E-3,8.442403E-5,-4.860737E-3,9.438967E-3,-5.854367E-5,-9.51611E-2,1.0191335E-1,6.272642E-2,1.9307575E-1,-2.3302536E-1,-3.1958916E-5,1.9217595E-3,-3.7351975E-3,-1.7018054E-1,-4.770182E-2,-1.1772707E-2,-4.2678308E-2,-1.2300495E-1,4.3251034E-2,-7.488707E-2,1.2381147E-1,2.3018995E-1,1.13042155E-2,4.7239262E-1,6.0778502E-3,1.1318266E-2,1.1771609E-2,2.0570065E-1,2.546662E-2,1.8722668E-1,-2.568286E-3,4.7251936E-3,-4.9866464E-2,1.198004E-2,4.8041707E-3,5.837389E-3,2.866293E-1,-4.0383036E-3,1.13763325E-1,-9.472239E-2,1.6336445E-3,9.330021E-2,3.5263494E-1,-2.0061426E-2,2.226326E-3,4.506053E-2,2.5072306E-2,1.5441628E-1,2.7075965E-2,1.7973125E-2,6.576918E-1,-2.2583338E-2,-1.258175E-2,1.308073E-3,-3.588668E-3,-1.6043756E-2,-2.8023586E-2,-1.4449921E-3,-1.30519625E-2,-1.9349203E-2,-3.9976593E-2,-8.11384E-3,-1.9989487E-2,-2.7164273E-2,-1.0335945E-2,-2.0625731E-2,-4.3071985E-2,-8.0963755E-3,-1.916886E-2,-1.4866783E-2,-2.6055383E-2,2.6677984E-3,-1.2722604E-2,-8.228849E-3,-1.7766777E-2,-9.0850815E-3,-1.917111E-2,-1.0514836E-2,-5.5266162E-3,-6.4426702E-3,6.0666056E-4,-7.071642E-4,3.4494456E-3,-1.2103436E-2,-4.288254E-3,-3.25981E-3,3.764634E-3,-1.044186E-2,-2.8907617E-3,-2.044184E-2,-9.248897E-3,4.5503657E-3,-6.697717E-4,-1.5679386E-3,-6.618189E-3,7.476116E-3,-2.1710634E-4,6.058815E-3,1.7470513E-4,1.1208675E-2,3.970706E-3,-9.846567E-3,-1.6570836E-2,-3.07845E-4,-9.319058E-3,-7.005788E-3,2.7312213E-3,-8.668174E-3,-2.9494073E-5,7.107689E-3,-8.0795004E-4,3.6914723E-3,-2.2320142E-2,6.787478E-3,-5.386278E-3,8.664719E-3,1.857925E-2,3.1997685E-3,-3.2370412E-3,2.56858E-2,1.1804593E-2,-4.320999E-3,2.526296E-3,1.1022412E-2,2.7352972E-3,-2.154117E-3,7.1858643E-3,5.7245875E-3,1.1020769E-2,-3.741437E-3,-4.935971E-4,1.567474E-2,8.552851E-3,6.7198128E-3,-2.493483E-3,-1.7894421E-3,-5.731426E-3,-1.17736515E-2,1.619122E-2,1.9466313E-2,8.126256E-3,1.4384771E-2,-4.965706E-4,2.0873813E-2,3.4394033E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,79,81,-1,83,85,87,89,-1,-1,91,93,95,97,99,101,103,105,-1,107,109,-1,111,113,115,117,-1,-1,119,121,123,125,-1,127,129,131,133,135,-1,-1,-1,-1,137,139,141,143,145,147,-1,-1,149,151,-1,153,155,-1,157,-1,159,161,-1,-1,163,165,167,-1,169,171,173,175,177,179,181,183,185,-1,187,189,-1,191,193,-1,195,197,199,201,-1,203,-1,-1,205,-1,207,-1,209,211,213,-1,-1,215,217,219,-1,221,223,-1,225,227,229,231,-1,233,-1,-1,-1,-1,235,237,239,-1,241,243,245,247,249,251,253,-1,255,-1,257,-1,-1,-1,-1,-1,259,261,263,265,267,269,-1,271,273,275,-1,-1,277,-1,279,281,283,285,-1,287,289,291,-1,-1,293,-1,295,-1,-1,-1,-1,-1,297,299,301,303,305,-1,-1,-1,307,309,-1,-1,311,313,315,317,319,321,323,-1,325,-1,327,329,331,-1,-1,333,-1,-1,-1,335,-1,337,339,-1,341,343,-1,-1,-1,-1,345,-1,-1,347,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.551732E1,1.5029903E1,2.6030987E1,7.7962914E0,7.4198904E0,1.0208933E1,8.882538E0,4.1354065E0,4.1918583E0,1.3296895E0,1.9523482E0,4.670254E0,2.2435484E0,3.9982228E0,5.371971E0,1.3204483E0,2.4200706E0,0E0,1.5694314E-1,7.311983E-1,5.2796906E-1,8.679679E-1,4.2655945E-2,3.8350763E0,0E0,1.7085991E0,1.2860048E-1,2.7795753E0,1.6100452E0,1.1581769E0,2.0894623E0,2.655418E-1,5.2924746E-1,1.8737526E0,8.8461256E-1,6.731875E-2,0E0,4.454441E-1,7.204876E-1,1.960865E-1,2.810763E-1,2.674389E-1,1.5625292E-1,4.8183918E-2,0E0,1.0922766E0,2.671496E0,1.5077815E0,7.4509E-1,0E0,0E0,8.290329E-1,8.929739E-1,2.8735447E-1,6.2698084E-1,6.385565E-1,3.3800125E-2,2.4970112E0,1.0182247E0,0E0,4.4682336E-1,1.812413E-1,0E0,9.1103935E-1,1.1238785E0,4.0634966E-1,4.0768725E-1,0E0,0E0,3.2076406E-1,1.0469827E-1,3.8142967E-1,8.191099E-1,0E0,9.498426E-2,6.300503E-2,4.4031806E-2,2.6219404E-1,1.9521287E-1,0E0,0E0,0E0,0E0,8.972211E-1,1.5119803E-1,2.0036325E0,4.7448697E0,6.5626776E-1,7.8167343E-1,0E0,0E0,6.9647956E-1,4.5907712E-1,0E0,3.5506535E-1,6.104803E-2,0E0,2.643968E-1,0E0,1.4225948E-1,2.6462883E-1,0E0,0E0,2.3219604E0,1.1968288E0,1.1333199E0,0E0,1.2514806E-1,1.4228895E-1,3.4473166E-2,3.501524E-2,1.0654917E0,3.3138323E-1,1.0778885E0,1.4066619E-1,3.3112526E-1,0E0,4.4789363E-2,2.1517646E-1,0E0,1.7452645E-1,8.036974E-2,0E0,4.3679953E-2,2.4790958E-1,3.922038E-1,1.6397619E-1,0E0,6.062464E-2,0E0,0E0,2.735281E-2,0E0,1.035431E-1,0E0,2.192457E-1,8.331013E-2,3.0753517E-1,0E0,0E0,3.1677987E-2,5.6169605E-1,6.2655663E-1,0E0,2.4831235E0,5.992241E-1,0E0,3.75247E-1,4.1256142E-1,5.721542E-1,3.0860472E-1,0E0,2.1074241E-1,0E0,0E0,0E0,0E0,7.820908E-2,2.804041E-2,1.3748431E-1,0E0,2.4369317E-1,6.482753E-2,9.944916E-1,4.6201628E-1,1.0669041E-1,6.00955E-1,4.4174767E-1,0E0,2.6134253E-2,0E0,2.3431404E-2,0E0,0E0,0E0,0E0,0E0,4.6422672E-1,5.2155876E-1,3.9374685E-1,6.590068E-2,8.6213875E-1,1.7471886E-1,0E0,4.5884013E-2,1.6714859E-1,4.1684568E-1,0E0,0E0,8.424616E-2,0E0,3.502083E-2,1.5113163E-1,4.9662333E-2,2.9039089E-2,0E0,3.1264573E-2,6.914465E-2,9.9033E-2,0E0,0E0,8.560538E-2,0E0,4.178869E-2,0E0,0E0,0E0,0E0,0E0,1.9628406E-2,6.655298E-2,3.4243762E-2,3.8973093E-2,3.3893156E-1,0E0,0E0,0E0,2.3448062E-1,6.515769E-1,0E0,0E0,8.463907E-1,1.537637E0,9.7332484E-1,4.9103904E-1,2.4113488E-1,4.828532E-2,8.5176945E-2,0E0,3.612732E-1,0E0,1.0484457E-1,1.1170708E-1,2.8017342E-2,0E0,0E0,9.2621725E-3,0E0,0E0,0E0,8.822799E-3,0E0,1.355916E-1,1.2372449E-2,0E0,1.4425079E0,1.2424984E0,0E0,0E0,0E0,0E0,2.0028992E-1,0E0,0E0,1.3159561E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,94,94,95,95,97,97,99,99,100,100,103,103,104,104,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,117,117,118,118,120,120,121,121,123,123,124,124,125,125,126,126,128,128,131,131,133,133,135,135,136,136,137,137,140,140,141,141,142,142,144,144,145,145,147,147,148,148,149,149,150,150,152,152,157,157,158,158,159,159,161,161,162,162,163,163,164,164,165,165,166,166,167,167,169,169,171,171,177,177,178,178,179,179,180,180,181,181,182,182,184,184,185,185,186,186,189,189,191,191,192,192,193,193,194,194,196,196,197,197,198,198,201,201,203,203,209,209,210,210,211,211,212,212,213,213,217,217,218,218,221,221,222,222,223,223,224,224,225,225,226,226,227,227,229,229,231,231,232,232,233,233,236,236,240,240,242,242,243,243,245,245,246,246,251,251,254,254],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,80,82,-1,84,86,88,90,-1,-1,92,94,96,98,100,102,104,106,-1,108,110,-1,112,114,116,118,-1,-1,120,122,124,126,-1,128,130,132,134,136,-1,-1,-1,-1,138,140,142,144,146,148,-1,-1,150,152,-1,154,156,-1,158,-1,160,162,-1,-1,164,166,168,-1,170,172,174,176,178,180,182,184,186,-1,188,190,-1,192,194,-1,196,198,200,202,-1,204,-1,-1,206,-1,208,-1,210,212,214,-1,-1,216,218,220,-1,222,224,-1,226,228,230,232,-1,234,-1,-1,-1,-1,236,238,240,-1,242,244,246,248,250,252,254,-1,256,-1,258,-1,-1,-1,-1,-1,260,262,264,266,268,270,-1,272,274,276,-1,-1,278,-1,280,282,284,286,-1,288,290,292,-1,-1,294,-1,296,-1,-1,-1,-1,-1,298,300,302,304,306,-1,-1,-1,308,310,-1,-1,312,314,316,318,320,322,324,-1,326,-1,328,330,332,-1,-1,334,-1,-1,-1,336,-1,338,340,-1,342,344,-1,-1,-1,-1,346,-1,-1,348,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5938586E8,6.7652373E3,6.6473596E2,8.0708375E5,2.1446484E-1,8.2608955E-3,1.1297775E2,2.3544883E8,1.6173E4,1.3664E4,4.898012E-1,4.0844156E7,6.308896E2,7.2974384E-2,2.720437E6,4.5180824E-2,9.123214E-1,8.3306855E6,1.9711539E0,6.496696E7,3.07125E0,1.2015E4,3.7360214E-2,7.780377E6,1.7735592E3,4.1331047E-1,8.32E3,2.086819E10,2.9251662E6,2.127E3,1.4387654E0,6.748543E-1,1E0,4.5575E4,-8.732631E-3,2.835051E2,1E0,2.3887E4,9.451483E-1,6.426937E6,5.012E3,4.7741196E-1,1.1063214E-2,1.91E2,5.52E2,4.1724915E2,1E0,-2.0691572E-2,-6.5559484E-3,1.4777102E9,3.8833074E4,9.2593566E-2,2.2980049E10,1.3414634E0,1.817757E1,6.7652373E3,6.292039E0,-7.465838E-4,2.692487E6,2.7524946E0,1.7138593E-2,2.3887E4,1.5106794E3,1.1855755E8,8.91E2,-2.5292337E-3,5.7396363E-3,1.59E2,8.012376E0,1.305E3,2.2348747E2,1.1099054E-2,2.7309215E6,6.782E4,7.204208E8,1.9776634E9,5.0079144E5,-1.4292294E-2,-1.2104245E-3,2.497929E-2,1.3058877E-2,1E0,3.4206028E-6,4.3978744E0,8E0,1.45752E8,2.228899E6,9.424398E-3,2.7608668E-2,1.428111E5,5.12861E5,5.413058E-4,2.8681637E11,3.3611E4,-3.0996678E-3,6.7652373E3,-1.2864939E-2,7.6E0,2.3E1,2.302883E-2,1.1908323E-2,5.1797544E7,1.041044E-1,1E0,4.558724E-3,7.932662E-5,1.1355622E7,2.2100918E1,3.4986075E6,1.445E3,2.6740572E5,1.0017953E8,4.1485362E5,3.74E3,-2.7961204E-2,2.8710976E1,2.0255396E-1,3.3415486E-3,1.8631586E4,1.732E3,-7.863588E-3,4.4444447E0,1.3697374E7,4.282564E2,3.3382E4,6.527754E-3,2.106062E1,-5.1401635E-3,-1.5072297E-2,6.72E2,1.6419858E-3,1.68635E5,1.967056E-2,9.451483E-1,1.4977E5,1.405E3,-2.7162708E-2,9.95965E-3,8.190217E0,3.5E2,1E0,2.8897377E-2,9.391714E4,1.198E5,-1.31569635E-2,1.3295082E1,2E0,2.4345527E3,2.358119E-1,1.8277122E-2,2.560372E7,1.0808066E-2,2.5801374E-2,-1.8505141E-2,-8.073482E-3,5.4106476E7,7.1277297E-1,4.0906172E5,1.4411849E-3,1.23E3,1.7653874E6,1.0449754E0,1.1225189E5,2.0967E4,1.8066783E0,1.31903E5,8.301306E-3,3.8E1,-8.9566335E-3,7.115465E6,-1.0388474E-2,6.1479243E-3,4.356833E-4,-8.1276195E-3,-1.9352075E-3,5.18E2,1.1662405E9,1.2E1,1.074E3,6.3251557E0,1.0804272E3,-4.155476E-3,8E0,2.01351E5,1.84442E5,-4.1822044E-4,5.921211E-3,7.6599895E9,5.128314E-4,7.118233E7,4.7325465E-1,3.5E1,2.1340163E8,-1.6549338E-2,8.640299E8,5.5356906E4,2.9810298E-2,-1.0568104E-2,-3.273992E-2,3.8657358E6,-3.999012E-3,5.3E1,-5.0588287E-3,8.442403E-5,-4.860737E-3,9.438967E-3,-5.854367E-5,1.241112E8,2.264389E-1,1.1317E4,2.512E3,5.61E2,-3.1958916E-5,1.9217595E-3,-3.7351975E-3,1.4396E4,8.07966E5,-1.1772707E-2,-4.2678308E-2,9.51668E5,5.116E3,1.4286339E0,6.346204E-7,1E0,7.104169E7,1.032E3,6.0778502E-3,1.6006084E3,1.1771609E-2,1.00663277E9,3.6842105E-1,1.2060912E3,-2.568286E-3,4.7251936E-3,1.0258741E2,1.198004E-2,4.8041707E-3,5.837389E-3,1.16E0,-4.0383036E-3,2.5239267E5,3.9556694E-1,1.6336445E-3,3.564E3,9.070543E0,-2.0061426E-2,2.226326E-3,4.506053E-2,2.5072306E-2,1.4285097E1,2.7075965E-2,1.7973125E-2,3.0597075E8,-2.2583338E-2,-1.258175E-2,1.308073E-3,-3.588668E-3,-1.6043756E-2,-2.8023586E-2,-1.4449921E-3,-1.30519625E-2,-1.9349203E-2,-3.9976593E-2,-8.11384E-3,-1.9989487E-2,-2.7164273E-2,-1.0335945E-2,-2.0625731E-2,-4.3071985E-2,-8.0963755E-3,-1.916886E-2,-1.4866783E-2,-2.6055383E-2,2.6677984E-3,-1.2722604E-2,-8.228849E-3,-1.7766777E-2,-9.0850815E-3,-1.917111E-2,-1.0514836E-2,-5.5266162E-3,-6.4426702E-3,6.0666056E-4,-7.071642E-4,3.4494456E-3,-1.2103436E-2,-4.288254E-3,-3.25981E-3,3.764634E-3,-1.044186E-2,-2.8907617E-3,-2.044184E-2,-9.248897E-3,4.5503657E-3,-6.697717E-4,-1.5679386E-3,-6.618189E-3,7.476116E-3,-2.1710634E-4,6.058815E-3,1.7470513E-4,1.1208675E-2,3.970706E-3,-9.846567E-3,-1.6570836E-2,-3.07845E-4,-9.319058E-3,-7.005788E-3,2.7312213E-3,-8.668174E-3,-2.9494073E-5,7.107689E-3,-8.0795004E-4,3.6914723E-3,-2.2320142E-2,6.787478E-3,-5.386278E-3,8.664719E-3,1.857925E-2,3.1997685E-3,-3.2370412E-3,2.56858E-2,1.1804593E-2,-4.320999E-3,2.526296E-3,1.1022412E-2,2.7352972E-3,-2.154117E-3,7.1858643E-3,5.7245875E-3,1.1020769E-2,-3.741437E-3,-4.935971E-4,1.567474E-2,8.552851E-3,6.7198128E-3,-2.493483E-3,-1.7894421E-3,-5.731426E-3,-1.17736515E-2,1.619122E-2,1.9466313E-2,8.126256E-3,1.4384771E-2,-4.965706E-4,2.0873813E-2,3.4394033E-2],"split_indices":[20,102,7,52,52,28,38,27,58,12,2,2,27,45,52,38,32,0,27,45,53,45,57,9,0,28,4,35,2,31,28,1,41,27,59,2,0,52,100,9,42,45,2,38,0,3,2,4,102,0,0,7,33,38,5,53,58,52,38,0,1,57,0,9,52,45,0,0,0,11,53,2,52,0,45,9,12,5,28,0,0,0,0,73,41,53,18,7,9,0,0,28,29,0,31,11,0,52,0,56,3,0,0,50,57,6,0,27,29,56,32,2,33,45,28,2,0,58,34,0,33,2,0,56,45,4,9,0,56,0,0,0,0,29,0,41,29,10,0,0,56,2,6,0,28,7,0,46,17,52,27,0,1,0,0,0,0,45,27,28,0,2,51,53,33,10,34,1,0,3,0,1,0,0,0,0,0,0,5,18,2,54,33,0,3,29,29,0,0,5,0,7,39,3,7,0,5,33,57,0,0,32,0,8,0,0,0,0,0,12,38,9,0,2,0,0,0,9,9,0,0,1,29,53,37,100,5,10,0,55,0,7,57,52,0,0,56,0,0,0,53,0,33,38,0,2,53,0,0,0,0,54,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.561E3,8.08E2,1.753E3,4.49E2,3.59E2,1.014E3,7.39E2,4.3E2,1.9E1,2.63E2,9.6E1,7.17E2,2.97E2,3.08E2,4.31E2,6.9E1,3.61E2,6E0,1.3E1,2.04E2,5.9E1,7.3E1,2.3E1,7.11E2,6E0,2.88E2,9E0,2.5E2,5.8E1,1.08E2,3.23E2,4.5E1,2.4E1,2.68E2,9.3E1,8E0,5E0,1.14E2,9E1,3.1E1,2.8E1,6.4E1,9E0,1.9E1,4E0,1.4E2,5.71E2,2.62E2,2.6E1,5E0,4E0,2.1E2,4E1,2.7E1,3.1E1,9.4E1,1.4E1,2.45E2,7.8E1,5E0,4E1,2E1,4E0,1.24E2,1.44E2,6.4E1,2.9E1,4E0,4E0,8.4E1,3E1,5.7E1,3.3E1,6E0,2.5E1,1E1,1.8E1,1.9E1,4.5E1,5E0,4E0,1.4E1,5E0,1.28E2,1.2E1,1.62E2,4.09E2,1.9E2,7.2E1,1.1E1,1.5E1,1.67E2,4.3E1,5E0,3.5E1,2.2E1,5E0,2.6E1,5E0,3.2E1,6.2E1,9E0,5E0,2.1E2,3.5E1,7.3E1,5E0,2.3E1,1.7E1,1E1,1E1,9.8E1,2.6E1,1.3E2,1.4E1,5.7E1,7E0,1E1,1.9E1,5E0,7.9E1,2.6E1,4E0,2.1E1,3.6E1,1.4E1,1.9E1,6E0,1.9E1,4E0,6E0,1.2E1,6E0,1.3E1,6E0,2.1E1,2.4E1,1.19E2,9E0,4E0,8E0,1.54E2,8E0,1.3E1,3.96E2,1.86E2,4E0,4.7E1,2.5E1,1.04E2,6.3E1,1.8E1,2.5E1,7E0,2.8E1,1.8E1,4E0,1.4E1,1.2E1,2.8E1,4E0,4.4E1,1.8E1,2.02E2,8E0,1.7E1,1.8E1,6.6E1,7E0,1.7E1,6E0,8E0,9E0,5E0,5E0,5E0,5E0,4.5E1,5.3E1,1.6E1,1E1,1.1E2,2E1,5E0,9E0,2.9E1,2.8E1,6E0,4E0,1.5E1,4E0,1E1,6.9E1,9E0,1.7E1,1.3E1,8E0,1.2E1,2.4E1,4E0,1E1,1.2E1,7E0,1.5E1,4E0,4E0,8E0,9E0,4E0,9E0,1.2E1,9E0,1.5E1,1.13E2,6E0,4E0,4E0,8.7E1,6.7E1,4E0,4E0,1.36E2,2.6E2,1.6E1,1.7E2,3.7E1,1E1,2E1,5E0,9.2E1,1.2E1,5.1E1,1.2E1,2.1E1,4E0,5E0,9E0,8E0,4E0,9E0,1.9E1,7E0,3.7E1,1.3E1,5E0,1.7E1,1.85E2,4E0,4E0,1.1E1,6E0,8E0,1E1,1E1,5.6E1,1.1E1,6E0,4E0,4E0,2.8E1,1.7E1,1.3E1,4E1,8E0,8E0,4E0,6E0,1E2,1E1,4E0,1.6E1,4E0,5E0,2.2E1,7E0,5E0,2.3E1,1E1,5E0,5E0,5E0,3.3E1,3.6E1,5E0,4E0,6E0,1.1E1,4E0,4E0,6E0,6E0,1.7E1,7E0,4E0,8E0,5E0,1E1,4E0,5E0,8E0,4E0,4E0,5E0,1.1E1,4E0,8.5E1,2.8E1,8E0,7.9E1,3.5E1,3.2E1,9.6E1,4E1,9.7E1,1.63E2,1.2E1,4E0,1.62E2,8E0,2.8E1,9E0,6E0,4E0,1.6E1,4E0,2.6E1,6.6E1,4.6E1,5E0,8E0,4E0,8E0,1.3E1,5E0,4E0,1.4E1,5E0,3.3E1,4E0,4E0,9E0,7E0,1E1,1.54E2,3.1E1,4E0,4E0,8E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"349","size_leaf_vector":"1"}},{"base_weights":[-1.9482358E-3,-2.0547229E-1,9.688696E-2,-3.2795328E-1,-5.9118025E-2,-3.0078327E-3,2.2908473E-1,-3.4265065E-1,5.2397203E-2,-1.4037478E-1,2.0246331E-1,-1.18085794E-1,7.2474316E-2,1.3827251E-1,3.471627E-1,-4.5140913E-1,-2.1259114E-1,-1.82585E-1,-1.0621947E-2,1.3140242E-1,3.6340532E-1,-1.4193876E-1,2.6351056E-1,6.2204074E-2,4.569707E-2,6.898382E-2,3.0087987E-1,1.7118002E-1,4.2249224E-1,-3.9011443E-1,-7.604529E-1,-7.442275E-2,-3.513845E-1,-2.0084646E-1,-1.9027209E-2,-1.2912203E-1,4.410053E-2,-4.2669527E-2,1.8099596E-1,4.2249063E-1,2.2071123E-1,-1.1975778E-1,-3.9327303E-1,3.7115186E-2,5.0655887E-2,-2.3095341E-1,8.724706E-2,1.15569524E-1,-1.5973406E-1,3.206297E-1,-7.0438767E-3,7.0100897E-3,2.3119986E-1,4.829647E-1,2.3461023E-1,-4.4258508E-1,-2.8989196E-1,-8.1142443E-1,-1.7232114E-2,2.0189436E-2,-9.760256E-2,-4.8134622E-1,-2.1198459E-1,-1.8863875E-1,-3.7231138E-1,7.351205E-3,-6.17493E-2,-6.589416E-2,-1.1963895E-2,1.335533E-1,-3.704639E-2,6.950206E-3,-1.2141156E-1,3.2277358E-1,1.3273115E-1,2.6867938E-1,2.7304215E-2,5.1505975E-3,1.4088176E-2,-2.1571712E-1,-6.7267135E-2,-2.6298314E-1,-3.8233817E-2,8.7514585E-3,-1.5810452E-2,-1.6991068E-1,-3.7283793E-1,6.0385358E-2,2.9934347E-1,8.111481E-2,2.7473885E-1,-2.0952946E-1,6.3898857E-3,2.3241422E-1,4.2609775E-1,-5.8664023E-3,8.7102465E-2,2.9582608E-1,1.0045457E-1,3.886618E-1,6.213879E-1,1.6386843E-1,4.170466E-1,-3.393206E-2,-4.1104132E-1,-3.0088872E-2,-2.580741E-1,-4.2660885E-2,-2.3556044E-2,-1.2349813E-1,1.642293E-2,-3.6006916E-1,-6.432281E-1,-1.58816E-1,-2.9194508E-2,-3.6441037E-1,-1.7434008E-1,-9.68565E-3,-2.1851206E-2,1.3957056E-2,-1.4741826E-1,-9.887453E-2,1.3209983E-3,4.0892374E-2,1.9681878E-1,4.5297644E-3,-6.319701E-2,-8.688553E-3,-2.4334213E-3,1.9888157E-2,9.05299E-3,2.0498979E-1,5.3835943E-2,7.391622E-3,1.6215337E-2,-2.6646987E-1,-1.6557802E-1,2.3540671E-1,-1.2060568E-1,-1.0299423E-3,-3.1292367E-1,1.5892097E-3,-4.438725E-3,-9.8816745E-2,-1.5468597E-2,-4.6977136E-1,-6.1853845E-3,-2.7393974E-2,1.2952788E-1,1.9930975E-1,2.2565031E-2,2.0681448E-2,1.4016046E-1,3.464458E-1,1.8311986E-1,-2.6195878E-1,2.6279278E-3,3.8710263E-1,1.9848463E-1,4.8195645E-1,2.5806807E-3,-3.5672002E-3,7.063364E-3,1.61233E-1,3.2346043E-1,-1.356479E-2,1.2331937E-2,4.5012543E-1,1.916863E-1,6.7827564E-1,1.8573476E-1,1.9810596E-1,-6.4512244E-3,2.5537787E-2,6.332156E-3,-3.2190254E-1,-4.9924067E-1,-3.8301833E-2,-2.953055E-1,-1.8634088E-1,-5.270217E-3,-4.1363448E-1,-8.855123E-3,-4.0171298E-1,-4.3407217E-2,2.6067453E-3,-1.980229E-1,-1.03866905E-2,-2.3428857E-2,-2.578248E-1,-1.5052055E-1,-1.9948822E-3,4.3270877E-3,-3.7635902E-3,-9.495209E-3,-6.1444184E-3,-2.0259644E-3,-1.3052196E-3,4.14313E-3,1.1166703E-2,5.2028527E-3,-9.212973E-2,1.283177E-4,1.1950358E-2,4.2662947E-3,-3.5452114E-3,1.1238425E-1,-1.883212E-2,-2.3791346E-1,-4.1397102E-2,-2.00935E-1,5.3140694E-1,6.3251734E-2,-1.6238517E-1,-3.232994E-2,-1.7529162E-2,-6.8283714E-3,-1.7573032E-1,4.8898146E-2,-1.3934994E-2,-2.831624E-2,-1.2627411E-1,2.6937446E-2,7.563784E-2,2.0694979E-1,9.155942E-2,2.2499119E-1,6.60391E-3,1.2115676E-2,2.4720185E-2,1.7159753E-1,8.150138E-3,4.0155724E-1,1.2338727E-3,2.1894397E-1,-3.40338E-1,-1.5433857E-1,1.2136728E-2,2.3324024E-2,2.758708E-2,2.2995318E-1,5.044297E-1,9.832922E-3,9.985074E-3,4.13046E-3,2.1563502E-1,3.7090424E-1,6.814154E-2,-5.8300085E-3,4.8815355E-1,2.034603E-1,-6.0421377E-3,3.1322268E-1,3.840143E-2,5.748659E-1,1.5244247E-2,1.4607365E-3,-2.6481033E-3,2.2920628E-1,-8.104413E-3,-1.7463155E-2,-2.8930131E-2,-1.4483186E-2,-7.015987E-3,3.5073264E-3,-1.2241372E-2,-2.2516299E-2,-6.834011E-4,-1.13677485E-2,2.821109E-3,-6.969092E-3,-2.1933887E-2,-1.1470549E-2,-1.2389652E-2,-2.8349977E-2,-1.1589418E-2,-4.9091916E-4,-1.3706747E-2,-6.3403314E-3,-5.3880657E-3,-1.1241645E-2,-7.747487E-3,-2.469975E-3,1.7015223E-3,7.600822E-3,-5.601662E-3,-1.2488412E-2,-5.249283E-3,2.3166307E-3,-1.160795E-2,-5.8168173E-3,1.4858059E-2,3.0542664E-2,1.186411E-3,9.635025E-3,3.5402293E-3,-9.259367E-3,1.5563491E-3,-6.24502E-3,-1.1372294E-2,-2.5658845E-3,7.203996E-3,-2.8060968E-3,-4.224994E-3,-2.3239082E-2,-4.6494626E-3,3.866252E-3,5.5386326E-3,-5.026598E-3,1.879474E-2,8.229606E-3,-1.1725087E-3,9.407363E-3,1.3818587E-2,5.535458E-3,-2.7222184E-3,2.654742E-3,-1.4007507E-2,5.1814103E-3,1.5193472E-2,6.546825E-3,2.6844855E-2,1.2633726E-2,1.2619984E-2,6.7369267E-3,-1.2688767E-2,-2.4963267E-2,-1.1158178E-2,-2.4398195E-3,4.096725E-3,-2.159201E-3,1.2504181E-2,2.306659E-3,1.7361468E-2,2.7427696E-2,3.5853176E-3,1.4370219E-2,2.0074718E-2,1.170023E-2,-6.800406E-5,6.1966544E-3,2.5628172E-2,7.752393E-3,1.7097555E-2,4.563378E-3,2.1819618E-2,9.180602E-3,2.0903809E-2,3.7944287E-2,1.3002698E-2,3.6164816E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,-1,95,97,99,101,103,105,107,-1,-1,109,111,113,115,117,-1,119,121,-1,123,125,-1,127,129,131,133,-1,-1,-1,135,137,139,-1,-1,141,143,145,147,149,151,153,155,-1,157,159,-1,161,163,165,167,169,171,173,-1,175,-1,177,-1,-1,179,-1,181,183,185,-1,187,189,-1,-1,191,193,195,-1,197,199,-1,201,-1,-1,-1,-1,203,205,-1,-1,207,209,211,213,-1,215,-1,-1,217,-1,219,-1,221,223,225,-1,227,229,231,233,235,-1,237,239,241,-1,-1,-1,243,245,247,-1,249,251,253,255,257,-1,-1,-1,259,261,263,265,267,269,271,-1,273,-1,-1,275,-1,-1,277,279,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,281,-1,-1,-1,-1,283,-1,285,287,289,291,293,295,297,-1,-1,299,301,-1,-1,303,305,307,309,311,313,315,-1,317,319,-1,321,-1,323,325,327,-1,-1,329,331,333,-1,-1,-1,335,337,339,-1,341,343,-1,345,-1,347,-1,-1,-1,349,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.0891605E1,1.4805096E1,2.2490284E1,9.463871E0,8.062905E0,8.442987E0,7.811901E0,6.212738E0,0E0,1.580884E0,9.8154926E-1,3.5417523E0,5.019683E0,4.666253E0,4.1433067E0,4.287483E0,3.9051037E0,6.473017E-1,4.7258684E-1,5.602251E-1,1.3439202E-1,1.9765701E0,2.3088045E0,4.2855835E0,0E0,3.14037E0,1.127284E0,9.54164E-1,2.408989E0,9.44479E-1,5.4629135E-1,1.1929114E0,1.745861E0,3.4003353E-1,1.7234445E-1,1.446478E-1,3.678885E-1,2.315778E-1,2.9930115E-1,1.963706E-1,4.297352E-2,1.6728735E0,1.204009E0,0E0,1.434339E-1,3.3505774E-1,3.0356612E0,1.3158431E0,7.4289536E-1,1.0081348E0,0E0,2.792761E-1,5.7020307E-1,1.9384613E0,6.543884E-1,7.056351E-1,5.6994295E-1,8.718872E-2,0E0,0E0,1.1444681E0,7.8928757E-1,9.167216E-1,4.0142918E-1,5.444491E-2,0E0,1.2528554E-1,5.1336177E-2,0E0,1.3264534E-1,9.5895365E-2,0E0,3.350231E-2,5.57518E-2,2.1129775E-1,1.9423306E-2,0E0,0E0,0E0,2.561531E-1,3.5478234E0,2.7194452E-1,0E0,0E0,4.455017E-2,3.0701232E-1,2.5568032E-1,2.8985975E0,8.238487E-1,7.1790326E-1,2.190566E-1,6.047236E-1,0E0,2.748165E-1,1.0836706E0,0E0,1.5628758E-1,1.2265873E-1,4.205125E-1,1.148613E0,1.5097198E0,4.468745E-1,3.447969E-1,0E0,7.748699E-1,0E0,5.450945E-1,0E0,0E0,7.048458E-1,0E0,2.4011421E-1,7.9469395E-1,3.908471E-1,0E0,1.15145564E-1,3.052144E-1,0E0,0E0,4.6512187E-2,1.3336375E-2,1.1879601E-2,0E0,3.3145316E-2,1.1331499E-2,0E0,4.400015E-2,0E0,0E0,0E0,0E0,6.1335027E-2,1.5409574E-1,0E0,0E0,9.007025E-2,2.639029E-1,1.647999E0,6.807394E-1,0E0,7.896447E-2,0E0,0E0,2.8126365E-1,0E0,8.479834E-3,0E0,1.1369566E0,1.0983424E0,8.4552884E-2,0E0,3.13752E-1,3.6592674E-1,1.6688919E-1,1.1377829E-1,2.5113964E-1,0E0,1.3025999E-2,3.0019665E-1,1.9553089E-1,0E0,0E0,0E0,1.3809681E-2,1.1441469E-1,1.343256E-1,0E0,6.229563E-1,1.0257056E0,9.958267E-2,1.5919107E-1,3.089043E-1,0E0,0E0,0E0,2.165389E-1,8.006544E-1,1.3172074E-1,3.4286594E-1,4.3757415E-1,2.8954622E-1,1.1154652E-2,0E0,1.6497481E-1,0E0,0E0,2.4668527E-1,0E0,0E0,4.509282E-2,4.1120744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.532186E-2,0E0,0E0,0E0,0E0,3.839317E-2,0E0,4.4573307E-2,8.8671505E-2,1.002599E-1,4.2510986E-2,1.08046025E-1,6.868677E-1,3.6266127E-1,0E0,0E0,9.15311E-2,9.8183766E-2,0E0,0E0,1.0207795E0,8.363422E-1,9.9775743E-1,7.171979E-1,1.0453827E-1,1.4603639E-1,2.7573076E-1,0E0,5.8319885E-1,3.901577E-1,0E0,2.0722246E-1,0E0,1.4027894E-2,1.4831042E-1,1.12811774E-1,0E0,0E0,4.2059608E-2,1.6750741E-1,1.02581024E-1,0E0,0E0,0E0,1.2431723E-1,1.1413813E-2,3.5119377E-2,0E0,4.6442127E-1,1.5540704E-1,0E0,2.2351384E-1,0E0,5.775938E-1,0E0,0E0,0E0,1.4657521E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,63,63,64,64,66,66,67,67,69,69,70,70,72,72,73,73,74,74,75,75,79,79,80,80,81,81,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,96,96,97,97,98,98,99,99,100,100,101,101,102,102,104,104,106,106,109,109,111,111,112,112,113,113,115,115,116,116,119,119,120,120,121,121,123,123,124,124,126,126,131,131,132,132,135,135,136,136,137,137,138,138,140,140,143,143,145,145,147,147,148,148,149,149,151,151,152,152,153,153,154,154,155,155,157,157,158,158,159,159,163,163,164,164,165,165,167,167,168,168,169,169,170,170,171,171,175,175,176,176,177,177,178,178,179,179,180,180,181,181,183,183,186,186,189,189,190,190,201,201,206,206,208,208,209,209,210,210,211,211,212,212,213,213,214,214,217,217,218,218,221,221,222,222,223,223,224,224,225,225,226,226,227,227,229,229,230,230,232,232,234,234,235,235,236,236,239,239,240,240,241,241,245,245,246,246,247,247,249,249,250,250,252,252,254,254,258,258],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,-1,96,98,100,102,104,106,108,-1,-1,110,112,114,116,118,-1,120,122,-1,124,126,-1,128,130,132,134,-1,-1,-1,136,138,140,-1,-1,142,144,146,148,150,152,154,156,-1,158,160,-1,162,164,166,168,170,172,174,-1,176,-1,178,-1,-1,180,-1,182,184,186,-1,188,190,-1,-1,192,194,196,-1,198,200,-1,202,-1,-1,-1,-1,204,206,-1,-1,208,210,212,214,-1,216,-1,-1,218,-1,220,-1,222,224,226,-1,228,230,232,234,236,-1,238,240,242,-1,-1,-1,244,246,248,-1,250,252,254,256,258,-1,-1,-1,260,262,264,266,268,270,272,-1,274,-1,-1,276,-1,-1,278,280,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,282,-1,-1,-1,-1,284,-1,286,288,290,292,294,296,298,-1,-1,300,302,-1,-1,304,306,308,310,312,314,316,-1,318,320,-1,322,-1,324,326,328,-1,-1,330,332,334,-1,-1,-1,336,338,340,-1,342,344,-1,346,-1,348,-1,-1,-1,350,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5125709E8,1.6956E5,7.885718E2,1.7669278E5,3.9556694E-1,1.0097089E3,5.2397203E-2,1.2690893E0,1.91E4,1.4845161E3,4.4839956E3,1.3359244E9,6.9664386E2,4.3702424E7,6.747114E7,3.9516E4,1.752512E2,1.5938586E8,6.9514094E11,1E0,6.7388856E7,1.2E1,4.569707E-2,3.9712732E7,8.6206274E2,5.3134368E8,1.04E2,2.308943E0,2.262586E6,7.1913286E8,1.0052E4,1E0,3.0996E4,6.685704E6,2.46875E0,9.49E2,2.6E1,4.2472876E11,7.3610186E3,3.5E2,3.7832818E0,3.7115186E-2,3.130603E3,1E0,4.5723195E6,1.2679E4,3.4409692E3,6.4683E4,-7.0438767E-3,1.2739522E0,1.0406779E1,1.6503105E3,3.832274E2,5.85E2,1.2859046E-4,1E0,-1.7232114E-2,2.0189436E-2,1.5137865E2,1.3888E4,7.6008E4,2.750917E-4,3.6020834E2,7.351205E-3,8E0,2.04946E6,-1.1963895E-2,2.695E3,7.576E3,6.950206E-3,9.34731E5,1.6957658E6,1.6667E4,8.3769E0,2.7304215E-2,5.1505975E-3,1.4088176E-2,3.79E2,1.102E3,1.3446785E7,-3.8233817E-2,8.7514585E-3,1.0617118E8,4.352552E9,2.6476662E6,2.5275471E2,3.7990784E7,1.5825138E5,3.5151117E-2,6E1,6.3898857E-3,5.4E2,1.96E2,-5.8664023E-3,2.1515152E0,1.513E3,3.773E3,1.6015803E7,4.728721E0,6.2395184E7,2.4350044E2,-3.393206E-2,1.4103535E0,-3.0088872E-2,3.07E2,-4.2660885E-2,-2.3556044E-2,2.6036014E5,1.642293E-2,2.1944155E3,1.0828989E8,9.35E2,-2.9194508E-2,1.01346857E3,1.11E3,-9.68565E-3,-2.1851206E-2,4.7121394E-2,2.6E1,2.801121E0,1.3209983E-3,1.7446084E0,1.5609541E0,4.5297644E-3,3.222E3,-8.688553E-3,-2.4334213E-3,1.9888157E-2,9.05299E-3,8.4342186E1,2.4513194E5,7.391622E-3,1.6215337E-2,6E0,6.661E3,8E0,4.709E3,-1.0299423E-3,1.829E3,1.5892097E-3,-4.438725E-3,1.0323588E3,-1.5468597E-2,7.4456794E5,-6.1853845E-3,1E0,6.9432104E7,1.1E1,2.2565031E-2,2.200423E3,5E0,1.0931163E9,5E0,1.7495675E5,2.6279278E-3,1E0,1.466657E6,7.2501016E7,2.5806807E-3,-3.5672002E-3,7.063364E-3,2.5291866E1,6.4506575E5,9.561769E-1,1.2331937E-2,2.6E1,1.1660184E3,1.7384394E7,3.2E2,4.961E3,-6.4512244E-3,2.5537787E-2,6.332156E-3,1.3177E4,7.44393E5,3.3720784E5,3.1648794E-1,2.467284E7,9.599108E6,8.8132757E-1,-8.855123E-3,1.0192391E2,-4.3407217E-2,2.6067453E-3,2.6036014E5,-1.03866905E-2,-2.3428857E-2,1.4E1,1.6256282E7,-1.9948822E-3,4.3270877E-3,-3.7635902E-3,-9.495209E-3,-6.1444184E-3,-2.0259644E-3,-1.3052196E-3,4.14313E-3,1.1166703E-2,5.2028527E-3,3E1,1.283177E-4,1.1950358E-2,4.2662947E-3,-3.5452114E-3,1E0,-1.883212E-2,2.3E1,4.974E3,1.838444E6,1.5E1,4.5184364E0,9.928469E-1,9.159699E7,-1.7529162E-2,-6.8283714E-3,2.8E2,1.48E2,-1.3934994E-2,-2.831624E-2,3.5612745E0,4.67E2,9.457831E0,2.880727E-2,1.408E3,2.2174756E4,6.808926E6,1.2115676E-2,1.683653E8,1.0666667E1,8.150138E-3,1.27734E5,1.2338727E-3,3.64299E5,1E0,1.6826648E3,1.2136728E-2,2.3324024E-2,5.96E2,1.1064664E7,9.3058E4,9.832922E-3,9.985074E-3,4.13046E-3,2.652E3,1.4771426E0,1.5481343E1,-5.8300085E-3,5.2409735E-7,1.388867E0,-6.0421377E-3,1.0993947E1,3.840143E-2,8.015419E3,1.5244247E-2,1.4607365E-3,-2.6481033E-3,1.8222668E2,-8.104413E-3,-1.7463155E-2,-2.8930131E-2,-1.4483186E-2,-7.015987E-3,3.5073264E-3,-1.2241372E-2,-2.2516299E-2,-6.834011E-4,-1.13677485E-2,2.821109E-3,-6.969092E-3,-2.1933887E-2,-1.1470549E-2,-1.2389652E-2,-2.8349977E-2,-1.1589418E-2,-4.9091916E-4,-1.3706747E-2,-6.3403314E-3,-5.3880657E-3,-1.1241645E-2,-7.747487E-3,-2.469975E-3,1.7015223E-3,7.600822E-3,-5.601662E-3,-1.2488412E-2,-5.249283E-3,2.3166307E-3,-1.160795E-2,-5.8168173E-3,1.4858059E-2,3.0542664E-2,1.186411E-3,9.635025E-3,3.5402293E-3,-9.259367E-3,1.5563491E-3,-6.24502E-3,-1.1372294E-2,-2.5658845E-3,7.203996E-3,-2.8060968E-3,-4.224994E-3,-2.3239082E-2,-4.6494626E-3,3.866252E-3,5.5386326E-3,-5.026598E-3,1.879474E-2,8.229606E-3,-1.1725087E-3,9.407363E-3,1.3818587E-2,5.535458E-3,-2.7222184E-3,2.654742E-3,-1.4007507E-2,5.1814103E-3,1.5193472E-2,6.546825E-3,2.6844855E-2,1.2633726E-2,1.2619984E-2,6.7369267E-3,-1.2688767E-2,-2.4963267E-2,-1.1158178E-2,-2.4398195E-3,4.096725E-3,-2.159201E-3,1.2504181E-2,2.306659E-3,1.7361468E-2,2.7427696E-2,3.5853176E-3,1.4370219E-2,2.0074718E-2,1.170023E-2,-6.800406E-5,6.1966544E-3,2.5628172E-2,7.752393E-3,1.7097555E-2,4.563378E-3,2.1819618E-2,9.180602E-3,2.0903809E-2,3.7944287E-2,1.3002698E-2,3.6164816E-3],"split_indices":[20,102,7,2,52,28,38,55,0,42,2,52,52,7,52,45,45,10,52,7,31,6,7,18,0,45,58,31,8,54,1,5,2,77,9,32,53,11,3,31,4,2,53,0,4,105,28,2,52,2,0,38,56,52,56,2,41,16,0,0,56,10,10,39,4,0,3,9,0,2,9,0,1,47,9,54,0,0,0,1,29,45,0,0,7,5,32,52,7,28,57,3,0,0,8,0,53,2,2,51,38,44,58,0,56,0,11,0,0,33,0,4,45,0,0,48,2,0,0,38,3,39,0,53,57,0,2,0,0,0,0,58,28,0,0,52,9,18,2,0,2,0,0,52,0,32,0,102,7,3,0,52,8,5,8,33,0,16,50,43,0,0,0,58,28,38,0,8,4,51,0,10,0,0,0,9,1,28,38,45,29,27,0,58,0,0,33,0,0,8,45,0,0,0,0,0,0,0,0,0,0,3,0,0,0,0,64,0,2,9,9,0,53,53,7,0,0,0,10,0,0,53,2,54,41,0,47,9,0,7,58,0,11,0,9,66,4,0,0,8,43,2,0,0,0,10,38,56,0,37,42,0,56,0,4,0,0,0,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.528E3,8.26E2,1.702E3,4.49E2,3.77E2,9.7E2,7.32E2,4.45E2,4E0,2.88E2,8.9E1,3.84E2,5.86E2,4.15E2,3.17E2,2.41E2,2.04E2,2.17E2,7.1E1,6.3E1,2.6E1,3.62E2,2.2E1,5.8E2,6E0,2.92E2,1.23E2,9.6E1,2.21E2,2.03E2,3.8E1,1.03E2,1.01E2,1.95E2,2.2E1,2.2E1,4.9E1,1.4E1,4.9E1,1.7E1,9E0,3.34E2,2.8E1,6E0,1.6E1,4.5E1,5.35E2,2.43E2,4.9E1,1.18E2,5E0,2.6E1,7E1,1.66E2,5.5E1,1.31E2,7.2E1,3.3E1,5E0,4E0,9.9E1,5.1E1,5E1,1.84E2,1.1E1,4E0,1.8E1,1.5E1,7E0,2.3E1,2.6E1,4E0,1E1,1.1E1,3.8E1,9E0,8E0,4E0,5E0,1.17E2,2.17E2,2.2E1,6E0,5E0,1.1E1,3.3E1,1.2E1,4.76E2,5.9E1,2.01E2,4.2E1,4.2E1,7E0,6.6E1,5.2E1,1E1,1.6E1,4.6E1,2.4E1,1.01E2,6.5E1,4.1E1,1.4E1,1.3E1,1.18E2,5E0,6.7E1,2.8E1,5E0,9.4E1,5E0,3.1E1,2E1,4.5E1,5E0,1.2E1,1.72E2,4E0,7E0,1E1,8E0,1.1E1,4E0,1E1,1.3E1,4E0,2.2E1,5E0,5E0,6E0,5E0,1.9E1,1.9E1,4E0,5E0,5.6E1,6.1E1,3.2E1,1.85E2,4E0,1.8E1,7E0,4E0,2.3E1,1E1,8E0,4E0,2.1E2,2.66E2,3.7E1,2.2E1,1E2,1.01E2,2.2E1,2E1,3.5E1,7E0,1E1,5.6E1,4.5E1,7E0,4E0,1.2E1,9E0,3.7E1,1.4E1,1E1,7.6E1,2.5E1,5.7E1,8E0,3.7E1,4E0,1E1,4E0,6.1E1,5.7E1,1E1,5.7E1,6.1E1,3.3E1,2.3E1,8E0,1.1E1,9E0,7E0,3.8E1,6E0,6E0,3.6E1,1.36E2,6E0,4E0,4E0,4E0,7E0,4E0,4E0,6E0,9E0,4E0,1.5E1,7E0,1.4E1,5E0,6E0,1.3E1,9E0,4.7E1,1.4E1,4.7E1,1.1E1,2.1E1,1.25E2,6E1,1.4E1,4E0,1.5E1,8E0,4E0,4E0,7.4E1,1.36E2,1.58E2,1.08E2,8E0,2.9E1,9.5E1,5E0,2.2E1,7.9E1,6E0,1.6E1,4E0,1.6E1,1.9E1,1.6E1,5E0,5E0,9E0,4.7E1,4.1E1,4E0,5E0,4E0,1.3E1,2.4E1,8E0,6E0,6.5E1,1.1E1,7E0,1.8E1,2.7E1,3E1,4E0,4E0,4E0,3.3E1,1E1,5.1E1,4E1,1.7E1,5E0,5E0,4.5E1,1.2E1,1.2E1,4.9E1,2.3E1,1E1,1.9E1,4E0,7E0,4E0,3.2E1,6E0,3.1E1,5E0,8.8E1,4.8E1,5E0,1E1,5E0,8E0,5E0,4.2E1,8E0,6E0,3.3E1,1.4E1,4E0,7E0,1.7E1,4E0,1.1E1,1.14E2,3.6E1,2.4E1,1E1,5E0,4E0,4E0,6.7E1,7E0,4E1,9.6E1,1.32E2,2.6E1,2E1,8.8E1,4E0,4E0,1.9E1,1E1,4.1E1,5.4E1,4E0,1.8E1,1.7E1,6.2E1,7E0,9E0,1E1,6E0,1.4E1,5E0,9E0,7E0,5E0,4E0,4.2E1,5E0,1.1E1,3E1,5E0,8E0,1.8E1,6E0,4E0,4E0,6E1,5E0,4E0,7E0,8E0,1E1,1.8E1,1.2E1,2.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"351","size_leaf_vector":"1"}},{"base_weights":[-9.540211E-4,-2.1021914E-1,9.758938E-2,-3.4094965E-1,-8.96193E-2,1.9241882E-2,2.6830703E-1,-3.1969693E-1,-8.097816E-1,-1.0598754E-1,5.25648E-2,-1.11852385E-1,6.534537E-2,-9.580468E-2,3.0770582E-1,-3.6781433E-1,-2.0454282E-1,-4.8666324E-2,-1.2679512E-2,-1.6879347E-1,1.4575647E-1,-1.6967353E-1,5.4975856E-2,5.262072E-2,4.2991307E-1,-1.801036E-1,9.3305E-2,2.8337514E-1,6.1104184E-1,-4.063043E-1,-1.8075636E-1,-2.5220284E-1,-1.024065E-1,-1.0687212E-1,-3.0482453E-1,1.8287341E-1,-6.817116E-2,2.1443701E-1,-1.8659256E-1,3.242453E-2,1.4646356E-2,6.795566E-2,-2.7120122E-1,3.477091E-2,2.754329E-1,7.2903293E-3,-2.3560175E-1,-9.618578E-4,1.1397864E-2,1.9325958E-1,3.631694E-1,3.264273E-2,1.7424444E-2,-3.568547E-1,-4.9503523E-1,6.7400925E-2,-2.7032366E-1,-2.8749174E-1,-1.14894085E-1,-4.66373E-2,-1.4747846E-1,-1.5669304E-1,3.606835E-2,-4.4745126E-1,-1.818722E-1,6.9011234E-2,2.8563836E-1,-7.362505E-3,4.3961657E-3,-2.9201508E-3,2.4933586E-2,-1.6614795E-1,-2.3542892E-2,-1.4696452E-1,7.210299E-2,3.943973E-2,2.1306403E-1,-3.315322E-1,-1.1285645E-1,3.2270004E-3,1.9187177E-2,-2.3783693E-2,-1.7345588E-1,-2.9602647E-3,4.0386366E-3,1.1937816E-1,3.1960076E-1,2.5352746E-1,4.4134948E-1,-2.824065E-1,-4.52653E-1,-5.407159E-1,-2.9564825E-1,-2.4649773E-3,1.0583536E-2,-3.6099592E-1,-1.4575511E-1,-2.3539357E-1,-4.0235817E-1,-9.382154E-3,-5.80747E-2,-9.375388E-2,2.0384628E-3,-2.1246274E-3,-8.547313E-3,-4.3750176E-1,-1.2868775E-1,1.376761E-1,-3.5494484E-2,-2.1314055E-1,-4.9733695E-1,-1.3046686E-1,-3.4496706E-2,1.0536173E-1,-4.7708195E-2,1.4304003E-1,3.9969492E-1,-1.8043293E-1,1.6755255E-2,-1.0759848E-2,-5.7054467E-2,4.1910753E-2,2.2511679E-1,-1.8687522E-2,1.0601062E-1,1.6771004E-1,4.3014926E-1,-1.8348831E-1,-4.2184436E-1,-1.0636496E-2,1.131124E-3,-2.1102086E-1,1.7005731E-3,1.3637011E-1,-1.7642468E-2,3.4541264E-1,-6.3125407E-3,4.2102966E-1,1.7333323E-1,5.228283E-1,2.7420223E-1,1.9957742E-3,-2.994493E-1,-5.528598E-1,-3.0034944E-1,-6.18909E-1,-3.1112647E-1,-7.646253E-3,-3.3043638E-1,-2.3313838E-1,-2.7325513E-2,-9.820285E-3,-1.4172044E-3,-2.8318578E-1,-1.2730235E-1,-4.8493865E-1,-1.0748288E-2,-4.7245852E-4,-5.7225167E-3,-5.2769383E-4,-6.697045E-3,-2.5568468E-2,-1.3970194E-2,-1.7150225E-1,-3.976064E-2,1.6857912E-1,-9.81934E-4,-7.729971E-2,1.1949919E-2,-2.506031E-3,-1.5661512E-2,-1.5493808E-2,-5.621411E-1,-2.1070492E-1,1.8342258E-3,4.2434465E-5,1.35182E-1,-4.8914845E-3,6.0060935E-4,2.1819782E-1,1.0901757E-3,4.7869483E-1,9.223637E-3,-2.1437973E-1,-7.213904E-2,-8.2442E-4,-4.712339E-3,7.8228705E-2,-1.7639214E-2,1.4591788E-2,5.655215E-3,1.949531E-2,-1.0843879E-1,1.2354309E-1,-5.1259227E-2,8.016405E-2,2.6035073E-1,7.815058E-3,2.5056375E-2,-1.0697771E-2,-4.8874845E-3,-4.6786705E-1,-1.1343487E-2,-8.623223E-3,-1.4549031E-2,9.6988745E-2,2.7289736E-1,3.7363055E-1,1.0968299E-3,4.5729893E-1,1.2744472E-2,2.0815518E-1,-8.484389E-3,6.103691E-1,3.7213066E-1,3.431869E-1,1.04155816E-1,-1.7893642E-2,-1.0189764E-2,-8.771097E-3,-3.0474436E-2,-5.481744E-3,-1.8936573E-2,-2.510723E-2,-4.1182168E-2,-5.6775566E-3,-1.8145647E-2,-8.304614E-3,-1.9575823E-2,-1.418636E-2,-4.7523365E-3,-8.590756E-3,-1.6221905E-2,-2.0143678E-3,-9.912272E-3,-1.4811314E-2,-2.9611759E-2,-1.3164437E-2,-6.280496E-3,2.09074E-3,-5.0465255E-3,3.5059343E-3,1.1819227E-2,-9.204519E-3,-1.6981836E-3,-3.1148445E-2,-1.692696E-2,-1.1964336E-2,-2.0716542E-3,8.116935E-3,-5.471367E-3,2.8129478E-3,7.8105493E-3,5.4602996E-3,1.634134E-2,2.8571073E-2,1.5184915E-2,-1.7845651E-2,-9.734162E-3,-7.605254E-3,5.403416E-4,5.2596494E-3,-1.2809653E-3,-3.4337747E-3,6.952108E-3,-1.355499E-3,8.617694E-3,-8.454104E-3,-1.2088332E-3,-2.8839806E-4,7.2466703E-3,6.625821E-3,-7.689045E-3,6.7280387E-3,-1.9368822E-3,8.801096E-3,1.594144E-2,-2.97095E-2,-1.6498199E-2,1.4107329E-2,4.0324396E-3,1.6060986E-2,3.719245E-3,1.6654013E-2,3.67157E-2,1.1905089E-2,2.4367483E-2,6.536869E-4,1.2637589E-2,3.2746147E-2,1.8324662E-2,5.7576955E-3,2.0969493E-2,1.0161635E-2,2.176086E-2,7.922581E-3,-1.8167228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,-1,79,-1,81,83,-1,85,87,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,-1,-1,-1,117,-1,119,121,123,125,127,129,-1,-1,-1,131,-1,-1,133,135,137,139,141,143,145,147,-1,-1,149,151,153,155,-1,157,159,-1,-1,-1,161,163,165,167,169,171,173,-1,175,177,179,181,183,-1,-1,185,187,189,191,193,195,197,199,201,-1,-1,203,-1,205,-1,207,-1,209,211,213,215,-1,217,219,221,223,225,-1,227,229,-1,-1,-1,231,233,235,-1,-1,-1,-1,-1,-1,-1,237,239,241,-1,243,-1,-1,-1,-1,245,247,249,-1,251,-1,-1,253,-1,255,-1,257,259,-1,-1,261,263,-1,-1,265,267,269,271,273,275,-1,-1,-1,-1,277,-1,-1,-1,279,281,283,-1,285,-1,287,-1,289,291,293,295,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.159538E1,1.2598072E1,2.2746223E1,3.5672798E0,7.922351E0,7.060544E0,7.7043533E0,1.9693642E0,1.2135735E0,6.5545526E0,0E0,2.9417727E0,3.9854715E0,8.6171687E-1,3.3915634E0,1.7913208E0,5.145006E-1,0E0,0E0,2.7555113E0,6.7151475E-1,1.5087366E0,1.8828299E0,4.1698294E0,9.998517E-1,7.0380926E-1,2.162483E-1,3.1584396E0,1.6007423E-1,7.6231766E-1,1.0406518E0,3.303795E-1,8.612618E-2,1.6364579E0,1.7248592E0,8.110616E-1,1.7369288E-1,8.056873E-1,1.184248E0,0E0,7.0701784E-1,3.2977784E0,3.2560182E-1,0E0,4.3302524E-1,0E0,3.9549124E-1,5.7224147E-2,0E0,1.9500451E0,1.9196262E0,0E0,0E0,8.777809E-1,5.141392E-1,2.2959769E-1,3.3140993E-1,2.5836277E-1,5.9486717E-2,7.797726E-2,4.2228222E-2,1.2806005E0,4.4116706E-1,4.433689E-1,1.3999496E0,1.5352497E-1,5.4460573E-1,0E0,0E0,0E0,0E0,1.5218763E0,0E0,1.10667616E-1,2.4927562E-1,2.5907316E0,1.2176437E0,2.8359127E-1,1.6743675E-1,0E0,0E0,0E0,2.1762961E-1,0E0,0E0,1.1399667E0,9.414525E-1,1.2837257E0,1.7224388E0,4.6244574E-1,7.4965954E-1,8.6662674E-1,2.8818607E-2,0E0,0E0,3.2405686E-1,8.980739E-2,1.8608427E-1,1.5855813E-1,0E0,2.9108267E-2,3.8793042E-2,0E0,0E0,0E0,3.2706976E-2,5.8713055E-1,1.24860525E-1,4.3183526E-1,1.47753E-1,2.8681374E-1,5.630123E-1,0E0,8.403313E-2,2.8154885E-2,1.5923327E-1,2.5149107E-1,7.1610546E-1,0E0,0E0,1.3552811E-2,1.05191074E-1,3.4359097E-2,1.2289648E0,8.666196E-1,8.7126446E-1,3.5103893E-1,1.0527402E-2,3.7244797E-2,0E0,0E0,2.0617366E-2,0E0,6.8324685E-1,0E0,6.6722584E-1,0E0,7.598925E-2,8.717475E-1,9.7335243E-1,5.155933E-1,0E0,3.6402798E-1,6.46163E-1,3.4818888E-1,6.136074E-1,1.3342118E-1,0E0,5.5776715E-2,6.411123E-2,0E0,0E0,0E0,7.7977896E-2,8.47432E-2,4.7646284E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0921235E-1,2.6407835E-1,1.2386137E-1,0E0,1.4192435E-1,0E0,0E0,0E0,0E0,1.5818691E-1,1.5059507E-1,3.9330485E-1,0E0,2.6209146E-2,0E0,0E0,9.467149E-2,0E0,5.4074526E-2,0E0,3.5224771E-1,3.2709655E-1,0E0,0E0,8.693667E-2,1.617202E-1,0E0,0E0,1.8019788E0,5.435816E-1,7.81621E-1,6.2421083E-1,3.8105625E-1,2.0575309E-1,0E0,0E0,0E0,0E0,1.30422115E-2,0E0,0E0,0E0,2.911467E-1,2.5542307E-1,7.4553776E-1,0E0,5.0143242E-2,0E0,5.555694E-1,0E0,3.7347984E-1,3.913002E-1,3.374002E-1,1.2042798E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,44,44,46,46,47,47,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,71,71,73,73,74,74,75,75,76,76,77,77,78,78,82,82,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,95,95,96,96,97,97,98,98,100,100,101,101,105,105,106,106,107,107,108,108,109,109,110,110,111,111,113,113,114,114,115,115,116,116,117,117,120,120,121,121,122,122,123,123,124,124,125,125,126,126,127,127,128,128,131,131,133,133,135,135,137,137,138,138,139,139,140,140,142,142,143,143,144,144,145,145,146,146,148,148,149,149,153,153,154,154,155,155,163,163,164,164,165,165,167,167,172,172,173,173,174,174,176,176,179,179,181,181,183,183,184,184,187,187,188,188,191,191,192,192,193,193,194,194,195,195,196,196,201,201,205,205,206,206,207,207,209,209,211,211,213,213,214,214,215,215,216,216],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,-1,80,-1,82,84,-1,86,88,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,-1,-1,-1,118,-1,120,122,124,126,128,130,-1,-1,-1,132,-1,-1,134,136,138,140,142,144,146,148,-1,-1,150,152,154,156,-1,158,160,-1,-1,-1,162,164,166,168,170,172,174,-1,176,178,180,182,184,-1,-1,186,188,190,192,194,196,198,200,202,-1,-1,204,-1,206,-1,208,-1,210,212,214,216,-1,218,220,222,224,226,-1,228,230,-1,-1,-1,232,234,236,-1,-1,-1,-1,-1,-1,-1,238,240,242,-1,244,-1,-1,-1,-1,246,248,250,-1,252,-1,-1,254,-1,256,-1,258,260,-1,-1,262,264,-1,-1,266,268,270,272,274,276,-1,-1,-1,-1,278,-1,-1,-1,280,282,284,-1,286,-1,288,-1,290,292,294,296,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,3.3633875E6,1.2427474E8,2.14099E5,7.805608E4,2.5998926E-1,1E0,1.5683041E3,1.4580295E0,5.25648E-2,1.6342836E3,1.3664E4,3.1E1,7.096131E0,3.74E3,1.0976147E-1,-4.8666324E-2,-1.2679512E-2,4.987639E7,6.298144E0,7E0,1.5E1,3.0820766E-1,6.2686886E-5,1.8955729E1,2.2580828E-1,4.0434834E5,9.2668225E11,8.5315496E-1,2.1693E5,1.884E3,1.8E1,6.7628815E2,1.067536E3,1.4564946E3,2.106062E1,2.127E3,4.3702424E7,3.242453E-2,8.874974E7,2.3314082E6,2.6476662E6,3.477091E-2,1.307E3,7.2903293E-3,1.40673E5,9.826286E-2,1.1397864E-2,1.3359244E9,1.1660184E3,3.264273E-2,1.7424444E-2,2.2444444E0,8.69032E-2,5.704E3,2.0184135E0,1.98714E5,1.31625E2,2.6883545E2,9.877E3,1E0,2.3058404E7,7.279E3,2.6572757E10,5.884028E0,4.0434834E5,-7.362505E-3,4.3961657E-3,-2.9201508E-3,2.4933586E-2,6.7628815E2,-2.3542892E-2,7.44393E5,1.5963264E9,6.999242E7,7.780377E6,6.9821E4,2.0289855E-1,3.2270004E-3,1.9187177E-2,-2.3783693E-2,3.307766E6,-2.9602647E-3,4.0386366E-3,1.1206505E2,1.3861362E8,8E0,1.0989011E-2,9E0,1.993E3,3.449702E2,1.305E3,-2.4649773E-3,1.0583536E-2,1.5E1,1.4101E4,1.09E3,1.0715278E7,-9.382154E-3,2.3E1,1.1052309E6,2.0384628E-3,-2.1246274E-3,-8.547313E-3,2.0447222E8,2.674421E9,7.4259315E0,1.3554651E2,6.4506575E5,2.8277853E11,3.317425E3,-3.4496706E-2,2.1924436E5,6.072483E2,2.79495E-1,6.4512783E-1,2.3170732E4,1.6755255E-2,-1.0759848E-2,7.55E2,7.64E2,2.130881E6,3.0273972E0,1E0,2.9254214E2,6.87E2,3.956E3,9.26E2,-1.0636496E-2,1.131124E-3,1.0828989E8,1.7005731E-3,1.4101E4,-1.7642468E-2,5.68E2,-6.3125407E-3,2.6998322E5,1.4955851E7,2.3523688E0,2.8091298E7,1.9957742E-3,2.877528E2,1.0655738E0,1.6527965E7,2.467284E7,1.9217945E0,-7.646253E-3,1.4580295E0,1.2974394E6,-2.7325513E-2,-9.820285E-3,-1.4172044E-3,2.372255E2,1.69E2,4.282564E2,-1.0748288E-2,-4.7245852E-4,-5.7225167E-3,-5.2769383E-4,-6.697045E-3,-2.5568468E-2,-1.3970194E-2,4.296196E0,1.164249E7,1E0,-9.81934E-4,2.5269838E6,1.1949919E-2,-2.506031E-3,-1.5661512E-2,-1.5493808E-2,4.982003E6,6.162955E0,9.342223E6,4.2434465E-5,8.3785375E4,-4.8914845E-3,6.0060935E-4,2.3828971E-1,1.0901757E-3,4.5184364E0,9.223637E-3,3.983889E3,1.3528182E-1,-8.2442E-4,-4.712339E-3,9.0754684E1,7.879123E7,1.4591788E-2,5.655215E-3,3.6512393E2,5.77E2,7E2,1.6915509E3,8E0,3E0,7.815058E-3,2.5056375E-2,-1.0697771E-2,-4.8874845E-3,2.3E1,-1.1343487E-2,-8.623223E-3,-1.4549031E-2,3.0791914E0,1.2238596E9,1.921032E4,1.0968299E-3,1.993E3,1.2744472E-2,4.29E2,-8.484389E-3,1.7398448E7,3.0597075E8,1.6332837E9,7.612907E-1,-1.7893642E-2,-1.0189764E-2,-8.771097E-3,-3.0474436E-2,-5.481744E-3,-1.8936573E-2,-2.510723E-2,-4.1182168E-2,-5.6775566E-3,-1.8145647E-2,-8.304614E-3,-1.9575823E-2,-1.418636E-2,-4.7523365E-3,-8.590756E-3,-1.6221905E-2,-2.0143678E-3,-9.912272E-3,-1.4811314E-2,-2.9611759E-2,-1.3164437E-2,-6.280496E-3,2.09074E-3,-5.0465255E-3,3.5059343E-3,1.1819227E-2,-9.204519E-3,-1.6981836E-3,-3.1148445E-2,-1.692696E-2,-1.1964336E-2,-2.0716542E-3,8.116935E-3,-5.471367E-3,2.8129478E-3,7.8105493E-3,5.4602996E-3,1.634134E-2,2.8571073E-2,1.5184915E-2,-1.7845651E-2,-9.734162E-3,-7.605254E-3,5.403416E-4,5.2596494E-3,-1.2809653E-3,-3.4337747E-3,6.952108E-3,-1.355499E-3,8.617694E-3,-8.454104E-3,-1.2088332E-3,-2.8839806E-4,7.2466703E-3,6.625821E-3,-7.689045E-3,6.7280387E-3,-1.9368822E-3,8.801096E-3,1.594144E-2,-2.97095E-2,-1.6498199E-2,1.4107329E-2,4.0324396E-3,1.6060986E-2,3.719245E-3,1.6654013E-2,3.67157E-2,1.1905089E-2,2.4367483E-2,6.536869E-4,1.2637589E-2,3.2746147E-2,1.8324662E-2,5.7576955E-3,2.0969493E-2,1.0161635E-2,2.176086E-2,7.922581E-3,-1.8167228E-3],"split_indices":[20,56,50,45,2,28,42,102,4,42,0,55,2,8,35,2,38,0,0,45,57,3,0,27,39,58,42,28,31,27,29,10,3,52,52,4,56,1,45,0,7,28,32,0,0,0,1,39,0,7,4,0,0,56,38,2,53,1,52,4,9,75,45,9,12,53,28,0,0,0,0,52,0,1,7,7,28,12,57,0,0,0,1,0,0,57,44,3,57,10,2,52,11,0,0,3,2,2,45,0,3,45,0,0,0,7,5,53,56,28,31,52,0,28,4,38,57,33,0,0,10,0,1,53,74,55,2,2,0,0,0,45,0,2,0,3,0,33,47,41,45,0,4,53,45,45,53,0,42,28,0,0,0,4,10,4,0,0,0,0,0,0,0,58,45,102,0,32,0,0,0,0,29,53,32,0,33,0,0,38,0,53,0,28,38,0,0,56,32,0,0,52,10,2,47,8,8,0,0,0,0,3,0,0,0,54,7,4,0,2,0,10,0,50,7,12,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E3,8E2,1.7E3,3.83E2,4.17E2,1.166E3,5.34E2,3.68E2,1.5E1,4.12E2,5E0,3.03E2,8.63E2,5.2E1,4.82E2,2.58E2,1.1E2,1.1E1,4E0,3.3E2,8.2E1,2.25E2,7.8E1,8.35E2,2.8E1,3.6E1,1.6E1,4.48E2,3.4E1,2.13E2,4.5E1,7.4E1,3.6E1,2.28E2,1.02E2,7E1,1.2E1,9E0,2.16E2,4E0,7.4E1,7.98E2,3.7E1,9E0,1.9E1,5E0,3.1E1,1E1,6E0,2.12E2,2.36E2,2.8E1,6E0,1.4E2,7.3E1,1.2E1,3.3E1,5.8E1,1.6E1,1.7E1,1.9E1,1.69E2,5.9E1,4.6E1,5.6E1,3.4E1,3.6E1,8E0,4E0,5E0,4E0,2.03E2,1.3E1,1.9E1,5.5E1,6.68E2,1.3E2,2.6E1,1.1E1,7E0,1.2E1,5E0,2.6E1,6E0,4E0,1.35E2,7.7E1,1E2,1.36E2,8.1E1,5.9E1,5.8E1,1.5E1,7E0,5E0,1.8E1,1.5E1,4.2E1,1.6E1,6E0,1E1,1.1E1,6E0,4E0,1.5E1,1.4E1,1.55E2,2.4E1,3.5E1,9E0,3.7E1,5.2E1,4E0,2.6E1,8E0,1.7E1,1.9E1,1.98E2,5E0,1E1,9E0,4.7E1,8E0,3.57E2,3.11E2,1.09E2,2.1E1,1.1E1,1.5E1,6E0,5E0,2.2E1,4E0,1.31E2,4E0,7.3E1,4E0,3.1E1,6.9E1,9E1,4.6E1,4E0,7.7E1,3.4E1,2.5E1,4.2E1,1.6E1,4E0,1.1E1,1.2E1,6E0,1E1,5E0,2.8E1,1.4E1,1E1,6E0,6E0,4E0,4E0,7E0,8E0,6E0,1.04E2,5.1E1,2E1,4E0,3.1E1,4E0,4E0,5E0,1.1E1,2.6E1,3.2E1,2E1,6E0,2E1,4E0,4E0,1E1,7E0,1.3E1,6E0,1.5E2,4.8E1,5E0,4E0,2.9E1,1.8E1,4E0,4E0,2.51E2,1.06E2,2.8E2,3.1E1,5.7E1,5.2E1,5E0,1.6E1,7E0,4E0,1.1E1,4E0,1.7E1,5E0,1.03E2,2.8E1,6.7E1,6E0,2.4E1,7E0,6.3E1,6E0,5.5E1,3.5E1,3.2E1,1.4E1,4.6E1,3.1E1,5E0,2.9E1,8E0,1.7E1,2.9E1,1.3E1,4E0,1.2E1,4E0,7E0,8E0,4E0,9E0,1.9E1,7E0,7E0,5E0,5E0,3.3E1,7.1E1,2.2E1,2.9E1,9E0,1.1E1,8E0,2.3E1,1.9E1,7E0,2.7E1,5E0,8E0,1.2E1,5E0,1.5E1,6E0,4E0,7E0,6E0,1.6E1,1.34E2,2.4E1,2.4E1,2.3E1,6E0,1.4E1,4E0,1.93E2,5.8E1,6.1E1,4.5E1,4E1,2.4E2,1.1E1,2E1,3.9E1,1.8E1,2.3E1,2.9E1,4E0,7E0,7E0,9.6E1,2.2E1,6E0,6.2E1,5E0,4E0,2E1,1.2E1,5.1E1,4.5E1,1E1,6E0,2.9E1,1.4E1,1.8E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"297","size_leaf_vector":"1"}},{"base_weights":[4.862418E-3,-1.9179456E-1,9.648262E-2,-3.105474E-1,-8.264949E-2,1.2012146E-2,2.3069014E-1,-2.5082594E-1,-4.9272937E-1,-1.0570442E-1,6.561455E-1,-1.3410296E-1,4.818715E-2,-1.4305459E-1,2.5512394E-1,-3.0187643E-1,-1.2927227E-1,-5.9344494E-1,-2.7590042E-1,-1.570374E-1,1.4807065E-1,4.893653E-2,1.1950649E-2,-1.5695055E-1,6.1462075E-2,6.694045E-2,-2.275101E-1,-2.8893796E-1,-5.9793696E-2,2.298053E-1,5.45743E-1,-3.8857976E-1,-1.9351415E-1,-1.7186011E-1,9.228514E-2,-4.772192E-1,-7.5566834E-1,-2.1429308E-1,-2.3250155E-2,-1.2330598E-1,-3.6756784E-1,1.2598301E-2,2.1409757E-1,-2.3148051E-1,-1.184594E-1,1.2734108E-1,-9.538767E-3,6.1227366E-2,4.846425E-2,-3.3230308E-1,-2.7507244E-2,-1.7520031E-2,-6.893117E-3,-1.04420885E-1,3.0977342E-2,1.7120364E-1,3.2885256E-1,4.308494E-1,3.812276E-2,-2.6919094E-1,-4.4245243E-1,-2.1271679E-1,-1.0940361E-2,-8.133552E-2,-2.5786206E-1,1.4402951E-2,-3.4523E-2,-6.060777E-1,-3.317721E-1,-5.745493E-1,-4.6666455E-2,-2.6740897E-1,-1.4060725E-1,-1.6886091E-1,-1.0881236E-2,-2.9620275E-1,-3.662825E-2,1.06364556E-1,-7.492426E-2,1.6863677E-1,2.481546E-2,-2.0574808E-1,-2.4682205E-2,-4.34008E-2,-1.8404934E-1,2.4031487E-3,8.042245E-3,-3.3350154E-3,4.1921358E-3,1.3572513E-1,1.8076938E-2,-4.028018E-1,-2.1900454E-1,-8.996094E-3,3.534044E-2,8.7446417E-4,-8.096535E-3,4.177163E-3,-1.9169262E-3,3.2460026E-2,1.5880178E-1,2.0123918E-1,4.133574E-1,4.679758E-1,5.720985E-3,-1.678749E-1,-3.4321463E-1,-4.8701978E-1,-2.4210061E-1,1.897012E-3,-2.2556487E-1,4.0199556E-3,-5.9172534E-3,-1.24798864E-1,2.0872678E-3,-1.1282197E-1,-4.1274983E-1,2.6423284E-3,-5.0769392E-3,-6.6279113E-1,-1.5158841E-2,-1.9422695E-2,-3.5885256E-3,-3.175321E-2,-1.6267877E-2,-1.9379172E-2,-1.574082E-1,-3.5338532E-3,-8.184947E-3,-1.1742487E-1,-3.8052285E-1,6.889215E-2,-2.3585467E-1,-3.2327244E-1,-1.831102E-3,8.359139E-3,-6.193851E-4,9.40182E-4,-1.1867737E-1,7.676027E-2,2.1683629E-1,-2.4097788E-1,-7.278033E-2,-1.3114972E-1,-5.8224937E-4,-4.7921962E-1,-1.3670631E-1,1.5970436E-1,-3.204673E-2,-1.1644583E-2,2.4496388E-1,-4.5080552E-1,-1.0375992E-2,-1.2524825E-2,-6.218959E-3,-6.412157E-4,3.952173E-3,2.280563E-2,1.8264864E-1,4.4676557E-2,2.327976E-1,3.5883617E-2,3.817096E-1,2.57802E-2,1.465937E-2,-3.7739205E-3,-9.872551E-3,-3.8094366E-1,-1.0659692E-2,-4.4104543E-1,-3.298193E-2,-1.750291E-2,-4.503148E-3,-2.7511734E-1,-1.5920201E-1,-1.3681407E-1,-2.467115E-3,6.424256E-2,-6.1279507E-3,-2.401544E-1,2.690221E-3,-2.6573654E-2,-9.864628E-3,-3.7120294E-2,-2.0259362E-2,-4.3360605E-3,-9.821233E-3,-1.7667316E-1,-5.886204E-2,-2.2814734E-1,-5.449499E-1,3.1168658E-2,1.8837713E-2,-4.145728E-3,-2.9755616E-1,-3.455366E-3,-3.519792E-1,-7.976625E-3,-2.6972177E-3,-1.8600554E-3,1.1785783E-1,2.504179E-1,2.876297E-3,-1.3777028E-1,-2.7242705E-1,-1.4505237E-4,-7.107407E-3,-3.9965276E-2,-1.8860891E-1,1.0260325E-1,-4.050624E-2,-1.0678059E-2,-3.2422952E-2,1.3051605E-2,-1.6339009E-1,4.7428638E-1,1.3039E-1,-3.5816185E-2,1.08094454E-1,2.0923056E-1,2.2315608E-2,-1.5977178E-2,-2.7076174E-2,2.0223114E-1,-8.550127E-2,-1.9398453E-2,1.9831333E-1,9.548259E-2,-4.306788E-3,2.5025082E-1,-2.7290026E-3,4.1333285E-1,1.3489962E-1,-1.0964235E-2,-2.1973185E-2,-2.8696395E-2,-1.7668296E-2,-2.1532612E-2,-1.1996745E-2,-2.857248E-3,-1.0287325E-2,-7.7038608E-3,-4.000156E-3,4.8531382E-3,5.962076E-4,-5.5491556E-3,-1.5375228E-2,-4.4075297E-3,4.0083188E-3,-1.1372013E-2,-4.67693E-3,-4.806391E-3,2.2331283E-3,-1.8434266E-2,-7.1700253E-3,-3.2168493E-2,-1.1166398E-2,4.4657565E-3,-5.8906637E-3,-2.2990242E-2,-1.0244585E-2,-2.4123231E-2,-1.4530776E-2,8.873433E-3,3.287549E-3,1.4001298E-2,4.6505416E-3,-4.277309E-3,-8.507117E-3,-8.106929E-3,-1.46990735E-2,1.0083113E-3,-4.602788E-3,-4.305385E-3,-1.1445371E-2,2.2868887E-3,8.64516E-3,2.6394078E-3,-3.0222794E-3,-2.783761E-3,3.4067095E-3,-9.1456575E-3,7.222121E-4,1.7398117E-2,2.9562779E-2,4.3594805E-3,1.1572079E-2,-4.594634E-3,1.4584038E-3,9.339027E-3,7.6624873E-4,2.7933982E-3,1.3203574E-2,-1.2463009E-3,1.3729935E-2,-1.1911267E-2,1.3876923E-3,-1.305917E-2,1.3406896E-2,1.0567759E-2,-3.596401E-3,1.510233E-3,7.779955E-3,1.4042293E-2,2.0252098E-3,2.9701877E-3,2.1507297E-2,2.1788115E-3,1.5428066E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,101,103,-1,105,107,109,111,113,115,-1,117,119,121,123,-1,125,127,129,131,133,-1,135,137,139,-1,141,-1,143,145,-1,-1,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,-1,157,159,161,163,-1,165,167,169,171,-1,173,-1,-1,175,177,179,181,-1,-1,183,-1,-1,-1,-1,-1,-1,185,-1,-1,187,189,191,193,195,-1,-1,-1,-1,197,199,201,203,205,207,209,211,213,215,-1,217,219,221,-1,-1,-1,-1,-1,223,225,227,229,-1,231,-1,-1,-1,-1,233,-1,235,-1,-1,-1,237,239,241,-1,243,-1,245,247,-1,-1,-1,-1,-1,-1,249,251,253,255,257,-1,-1,259,-1,261,-1,-1,-1,263,265,-1,267,269,-1,-1,271,273,275,277,-1,-1,279,281,283,285,287,289,291,-1,-1,-1,293,295,297,299,301,-1,303,-1,305,307,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6089897E1,1.051186E1,1.977955E1,4.108799E0,7.308326E0,5.6727476E0,6.1914635E0,1.7915611E0,1.910883E0,5.403417E0,1.5179186E0,9.6137166E-1,4.461631E0,4.9287128E-1,4.5322456E0,1.8696461E0,8.5563326E-1,8.4606934E-1,2.7370167E-1,2.3975906E0,6.234404E-1,0E0,0E0,5.2050877E-1,1.08796366E-1,4.0986614E0,1.1435983E0,9.135568E-2,1.16658665E-1,3.3334332E0,9.1913986E-1,6.141205E-1,3.2722712E-1,5.6292367E-1,3.8270268E-1,5.48604E-1,2.7066135E-1,6.416154E-2,0E0,1.5243707E0,1.0087652E0,2.049292E-1,5.199454E-1,3.3065915E-1,6.2216735E-1,2.4221584E-2,6.974202E-2,2.5756636E0,0E0,1.9933128E-1,1.9809178E-1,0E0,0E0,1.3630842E-1,3.9695807E-2,2.0862703E0,2.246149E0,3.574953E-1,0E0,2.2737479E-1,5.696373E-1,2.8233075E-1,1.077602E-1,1.3804215E-1,8.0710006E-1,0E0,6.4007364E-2,1.4226532E-1,2.829256E-1,2.0683289E-2,0E0,1.2441862E-1,9.492338E-3,2.263373E0,1.5776283E0,2.762239E-1,0E0,8.890636E-2,5.555173E-2,1.693486E-1,0E0,2.6352692E-1,0E0,2.2911784E-1,8.8473725E-1,0E0,0E0,0E0,0E0,5.580813E0,3.4352558E0,1.0310602E-1,1.6691864E-2,0E0,3.263292E-2,0E0,0E0,0E0,0E0,0E0,1.1665869E0,4.2439556E-1,9.7021484E-1,1.0827255E-1,0E0,3.445804E-2,2.1634817E-2,2.0216465E-1,2.3151255E-1,0E0,2.225647E-1,0E0,0E0,1.5900582E-2,1.1641387E-1,3.0952343E-1,3.5653353E-1,0E0,0E0,2.8625011E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.827669E-3,0E0,0E0,5.8639693E-1,9.0291166E-1,7.4375176E-1,1.9227576E-1,2.4557161E-1,0E0,0E0,0E0,0E0,1.5233442E-2,7.934024E-2,1.2999737E-1,1.15020275E-1,6.754118E-2,9.7249866E-2,1.771783E-1,2.667849E-1,2.4189901E-1,2.575582E0,0E0,1.3091937E0,3.316791E-1,1.7702103E-2,0E0,0E0,0E0,0E0,0E0,1.0861071E0,9.7550297E-1,1.1461355E-1,3.7985802E-1,0E0,8.730278E-1,0E0,0E0,0E0,0E0,4.598999E-2,0E0,4.028902E-1,0E0,0E0,0E0,1.4211035E-1,1.6443658E-1,8.355498E-3,0E0,1.6043331E-2,0E0,5.3038716E-2,9.164512E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.3824182E-1,3.4447846E-1,2.2116292E-1,4.5989513E-1,5.224338E-1,0E0,0E0,1.4631844E-1,0E0,1.2364912E-1,0E0,0E0,0E0,2.563186E-2,7.705736E-2,0E0,7.4563324E-3,3.6147594E-2,0E0,0E0,3.0088965E-2,2.9990733E-2,4.0105864E-2,6.1000608E-2,0E0,0E0,4.170667E-2,1.8082714E-1,1.4915752E-1,1.128118E0,1.3747702E0,5.527258E-1,4.238739E-1,0E0,0E0,0E0,3.5296792E-1,6.1273694E-1,1.6712798E0,1.0307302E0,4.245203E-2,0E0,4.3405104E-1,0E0,6.098232E-1,2.2463292E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,75,75,77,77,78,78,79,79,81,81,83,83,84,84,89,89,90,90,91,91,92,92,94,94,100,100,101,101,102,102,103,103,105,105,106,106,107,107,108,108,110,110,113,113,114,114,115,115,116,116,119,119,126,126,129,129,130,130,131,131,132,132,133,133,138,138,139,139,140,140,141,141,142,142,143,143,144,144,145,145,146,146,147,147,149,149,150,150,151,151,157,157,158,158,159,159,160,160,162,162,167,167,169,169,173,173,174,174,175,175,177,177,179,179,180,180,187,187,188,188,189,189,190,190,191,191,194,194,196,196,200,200,201,201,203,203,204,204,207,207,208,208,209,209,210,210,213,213,214,214,215,215,216,216,217,217,218,218,219,219,223,223,224,224,225,225,226,226,227,227,229,229,231,231,232,232],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,102,104,-1,106,108,110,112,114,116,-1,118,120,122,124,-1,126,128,130,132,134,-1,136,138,140,-1,142,-1,144,146,-1,-1,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,-1,158,160,162,164,-1,166,168,170,172,-1,174,-1,-1,176,178,180,182,-1,-1,184,-1,-1,-1,-1,-1,-1,186,-1,-1,188,190,192,194,196,-1,-1,-1,-1,198,200,202,204,206,208,210,212,214,216,-1,218,220,222,-1,-1,-1,-1,-1,224,226,228,230,-1,232,-1,-1,-1,-1,234,-1,236,-1,-1,-1,238,240,242,-1,244,-1,246,248,-1,-1,-1,-1,-1,-1,250,252,254,256,258,-1,-1,260,-1,262,-1,-1,-1,264,266,-1,268,270,-1,-1,272,274,276,278,-1,-1,280,282,284,286,288,290,292,-1,-1,-1,294,296,298,300,302,-1,304,-1,306,308,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,1.0950326E6,2.467284E7,1.3144558E4,4.54E2,3.6E2,1.638058E2,5.121E3,1.5343539E0,1.5151515E-3,7.920064E5,3.9712732E7,7.6588124E-2,6.701152E0,2.68E0,2.821E3,6.747114E7,1.0828989E8,1E0,2.5682794E5,4.893653E-2,1.1950649E-2,4.7E2,6.714286E0,5.3534385E3,7.557E3,1.7495675E5,2.0081382E-7,1.2727361E6,1.8114515E1,2.0304577E-1,3.422351E6,7.5891815E6,2.8169732E10,1.2566517E5,7.8099336E11,1.2151E4,-2.3250155E-2,1.0323588E3,1.0104842E12,8.2608955E-3,3.3791902E0,3.09627E5,8.836364E0,3.2E1,1.4E1,8E0,4.846425E-2,1.0376E4,9.74026E0,-1.7520031E-2,-6.893117E-3,2.096295E6,1E0,4.3E1,3.131E3,1.5005797E4,3.812276E-2,1.4966443E0,1.9121015E0,3.5476844E7,1.1331109E6,7.2123044E5,3.6148317E-2,1.4402951E-2,2.821E3,2.6E1,1.3312868E6,3.0404909E0,-4.6666455E-2,1.9557823E0,1.683274E0,3.791269E7,9.302862E7,4.6726016E3,-3.662825E-2,2.9956698E-1,1.7227725E0,1.7495675E5,2.481546E-2,3.1E2,-2.4682205E-2,1.56E2,3.3820656E7,2.4031487E-3,8.042245E-3,-3.3350154E-3,4.1921358E-3,4.9E1,1.0941897E9,1.5675918E4,1.411682E7,-8.996094E-3,1.6988909E3,8.7446417E-4,-8.096535E-3,4.177163E-3,-1.9169262E-3,3.2460026E-2,6.63E2,6.1934864E7,1.0918E4,2.1362119E0,5.720985E-3,3.37E2,4.0434834E5,2.5074627E0,2.2837209E2,1.897012E-3,3.73802E5,4.0199556E-3,-5.9172534E-3,5E0,3.9473684E0,2.05156E5,6.4863464E2,2.6423284E-3,-5.0769392E-3,1.470946E10,-1.5158841E-2,-1.9422695E-2,-3.5885256E-3,-3.175321E-2,-1.6267877E-2,-1.9379172E-2,1.6622689E5,-3.5338532E-3,-8.184947E-3,3.3030225E2,8.672049E9,8.015419E3,8.4503644E5,5.503109E8,-1.831102E-3,8.359139E-3,-6.193851E-4,9.40182E-4,1.01E3,2.7E1,4.958887E-1,7E0,7E0,1.34E2,1.2E1,2.36413E5,4.1667834E-2,1.3E1,-3.204673E-2,2.092073E6,8.865177E-1,7.326E3,-1.0375992E-2,-1.2524825E-2,-6.218959E-3,-6.412157E-4,3.952173E-3,1.5851064E0,3.05713E4,1.4E1,9.966725E2,3.5883617E-2,2.1E1,2.57802E-2,1.465937E-2,-3.7739205E-3,-9.872551E-3,3.78E2,-1.0659692E-2,7.33E2,-3.298193E-2,-1.750291E-2,-4.503148E-3,1.19573875E5,1.01268125E5,3.4419507E8,-2.467115E-3,1.3198258E0,-6.1279507E-3,4.2320535E2,1.7440945E0,-2.6573654E-2,-9.864628E-3,-3.7120294E-2,-2.0259362E-2,-4.3360605E-3,-9.821233E-3,2.437E3,5.37456E5,2.1442623E1,6.654321E0,4.2145435E6,1.8837713E-2,-4.145728E-3,2.589778E6,-3.455366E-3,8.10114E4,-7.976625E-3,-2.6972177E-3,-1.8600554E-3,5.511673E0,6.5321615E6,2.876297E-3,5.797E3,1.901E3,-1.4505237E-4,-7.107407E-3,7.330957E4,5.222222E1,2.97E2,2.7258065E0,-1.0678059E-2,-3.2422952E-2,2.1384616E0,7.997723E6,1.305E3,3.2196458E6,1.273801E6,1.8071064E7,2.168033E3,2.2315608E-2,-1.5977178E-2,-2.7076174E-2,5.3134368E8,2.41E3,3.2343243E1,7.218466E7,2.9251662E6,-4.306788E-3,2E1,-2.7290026E-3,1E0,4.760537E8,-1.0964235E-2,-2.1973185E-2,-2.8696395E-2,-1.7668296E-2,-2.1532612E-2,-1.1996745E-2,-2.857248E-3,-1.0287325E-2,-7.7038608E-3,-4.000156E-3,4.8531382E-3,5.962076E-4,-5.5491556E-3,-1.5375228E-2,-4.4075297E-3,4.0083188E-3,-1.1372013E-2,-4.67693E-3,-4.806391E-3,2.2331283E-3,-1.8434266E-2,-7.1700253E-3,-3.2168493E-2,-1.1166398E-2,4.4657565E-3,-5.8906637E-3,-2.2990242E-2,-1.0244585E-2,-2.4123231E-2,-1.4530776E-2,8.873433E-3,3.287549E-3,1.4001298E-2,4.6505416E-3,-4.277309E-3,-8.507117E-3,-8.106929E-3,-1.46990735E-2,1.0083113E-3,-4.602788E-3,-4.305385E-3,-1.1445371E-2,2.2868887E-3,8.64516E-3,2.6394078E-3,-3.0222794E-3,-2.783761E-3,3.4067095E-3,-9.1456575E-3,7.222121E-4,1.7398117E-2,2.9562779E-2,4.3594805E-3,1.1572079E-2,-4.594634E-3,1.4584038E-3,9.339027E-3,7.6624873E-4,2.7933982E-3,1.3203574E-2,-1.2463009E-3,1.3729935E-2,-1.1911267E-2,1.3876923E-3,-1.305917E-2,1.3406896E-2,1.0567759E-2,-3.596401E-3,1.510233E-3,7.779955E-3,1.4042293E-2,2.0252098E-3,2.9701877E-3,2.1507297E-2,2.1788115E-3,1.5428066E-2],"split_indices":[20,56,47,45,52,2,29,52,2,42,57,28,45,42,35,54,2,45,45,104,28,0,0,1,58,4,2,33,37,28,35,27,9,45,31,33,31,9,0,52,31,27,35,9,58,10,3,17,0,9,53,0,0,12,66,0,2,4,0,54,53,7,45,47,38,0,10,3,28,56,0,53,53,45,45,4,0,38,39,33,0,1,0,2,7,0,0,0,0,3,7,48,12,0,52,0,0,0,0,0,10,7,9,34,0,0,28,56,4,0,33,0,0,8,56,12,4,0,0,5,0,0,0,0,0,0,33,0,0,52,5,4,32,5,0,0,0,0,0,3,38,0,12,29,10,9,38,10,0,1,35,9,0,0,0,0,0,53,33,8,4,0,3,0,0,0,0,0,0,2,0,0,0,28,28,7,0,39,0,4,54,0,0,0,0,0,0,2,28,56,54,47,0,0,29,0,33,0,0,0,53,47,0,9,9,0,0,45,4,2,54,0,0,53,9,2,28,9,45,4,0,0,0,31,2,56,45,28,0,6,0,100,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.556E3,8.12E2,1.744E3,3.88E2,4.24E2,1.071E3,6.73E2,2.94E2,9.4E1,4.12E2,1.2E1,2.12E2,8.59E2,4.1E1,6.32E2,2.06E2,8.8E1,6.3E1,3.1E1,3.43E2,6.9E1,6E0,6E0,1.9E2,2.2E1,8.05E2,5.4E1,1.4E1,2.7E1,5.83E2,4.9E1,1.13E2,9.3E1,7.4E1,1.4E1,3.9E1,2.4E1,2.5E1,6E0,2.97E2,4.6E1,2.3E1,4.6E1,6.3E1,1.27E2,1.1E1,1.1E1,8.01E2,4E0,3.5E1,1.9E1,9E0,5E0,1.8E1,9E0,3.68E2,2.15E2,3.4E1,1.5E1,3.7E1,7.6E1,8.4E1,9E0,3.7E1,3.7E1,5E0,9E0,1.9E1,2E1,1.4E1,1E1,1.3E1,1.2E1,2.11E2,8.6E1,4E1,6E0,1.1E1,1.2E1,4.1E1,5E0,5.9E1,4E0,6E1,6.7E1,4E0,7E0,7E0,4E0,2.93E2,5.08E2,2E1,1.5E1,5E0,1.4E1,6E0,1.2E1,5E0,4E0,8E0,3.6E2,8.7E1,1.28E2,3E1,4E0,1.7E1,2E1,6.1E1,1.5E1,4E0,8E1,5E0,4E0,2.4E1,1.3E1,2E1,1.7E1,4E0,5E0,1.5E1,4E0,1.6E1,4E0,1E1,4E0,5E0,8E0,4E0,8E0,1.71E2,4E1,6.4E1,2.2E1,3.6E1,4E0,7E0,4E0,4E0,8E0,1.5E1,2.6E1,4.6E1,1.3E1,1.9E1,4.1E1,8E0,5.9E1,2.85E2,8E0,4.5E2,5.8E1,1.5E1,5E0,1E1,5E0,7E0,7E0,5.4E1,3.06E2,1.5E1,7.2E1,1E1,1.18E2,2.2E1,8E0,5E0,1.2E1,1.4E1,6E0,5.1E1,1E1,8E0,7E0,4.4E1,3.6E1,2E1,4E0,9E0,4E0,9E0,1.1E1,1E1,7E0,1E1,5E0,4E0,4E0,8.4E1,8.7E1,2.2E1,1.8E1,5.8E1,6E0,7E0,1.5E1,4E0,3.2E1,4E0,4E0,4E0,1.1E1,2.1E1,5E0,1.2E1,3.4E1,7E0,6E0,8E0,1.1E1,1.1E1,3E1,4E0,4E0,9E0,5E1,2.3E1,2.62E2,3.75E2,7.5E1,5.1E1,7E0,8E0,7E0,2E1,3.4E1,2.2E1,2.84E2,1.1E1,4E0,6.8E1,4E0,1.04E2,1.4E1,5E0,9E0,1.8E1,3.3E1,6E0,3.8E1,1.2E1,2.4E1,1.4E1,6E0,5E0,4E0,4E0,5E0,5E0,6E0,5.1E1,3.3E1,6.4E1,2.3E1,7E0,1.5E1,1.3E1,5E0,4.2E1,1.6E1,4E0,1.1E1,8E0,2.4E1,4E0,7E0,1.7E1,4E0,6E0,6E0,7E0,2.7E1,4E0,4E0,4E0,7E0,7E0,4E0,5E0,2.5E1,4E0,5E0,4.5E1,5E0,1.3E1,1E1,1.85E2,7.7E1,2.01E2,1.74E2,4E1,3.5E1,1.4E1,3.7E1,5E0,1.5E1,1.4E1,2E1,1.2E1,1E1,2.71E2,1.3E1,6E0,5E0,5.9E1,9E0,5E0,9.9E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"309","size_leaf_vector":"1"}},{"base_weights":[2.983202E-3,-1.8349624E-1,8.967439E-2,-2.8245267E-1,-6.2706895E-2,4.0154368E-2,2.9061913E-1,-2.9507563E-1,4.50964E-2,-1.1894827E-1,1.8517493E-1,-3.698234E-2,1.4327604E-1,3.5555997E-1,5.2666392E-2,-3.8571632E-1,-1.7044207E-1,-1.5851945E-1,1.3303462E-3,9.124814E-2,3.1158262E-1,-1.9524197E-1,-1.1644163E-2,9.9095374E-2,2.899423E-1,3.160851E-1,5.785858E-1,-3.2939088E-2,3.7574562E-1,-3.267906E-1,-5.832741E-1,-4.430617E-2,-3.2573578E-1,-1.4562693E-1,-3.2024354E-1,-1.2385985E-1,4.523339E-2,1.2043578E-2,2.023579E-1,4.436063E-1,1.3844258E-1,-2.141533E-1,3.8661438E-3,-3.420884E-2,9.789156E-2,1.1671735E-1,-2.3177448E-1,3.3023003E-1,1.005949E-1,2.5137937E-1,5.09215E-1,4.2901665E-2,3.9478257E-1,-2.1095376E-2,2.5337553E-2,2.534327E-2,2.4615054E-1,-4.8901016E-1,-2.9659656E-1,-4.3791246E-1,-6.4286286E-1,-6.8579026E-2,1.8571176E-2,-2.6566648E-1,-5.5038553E-1,-3.0928117E-1,-1.3166685E-1,-4.0776885E-1,-5.3167827E-3,-1.1002511E-2,-5.504775E-2,7.6412275E-2,-1.2954426E-1,9.982063E-3,-3.3227753E-2,4.253036E-3,1.158649E-2,1.1479812E-2,2.4770781E-2,6.464225E-2,1.2909125E-2,-1.5839866E-1,-3.1661642E-1,2.321139E-1,-4.263291E-2,2.3260027E-2,7.445975E-2,3.8219556E-2,1.8300503E-1,-2.958806E-1,3.547874E-3,3.548678E-2,2.9902574E-1,2.6240832E-1,-1.6939E-2,2.104832E-1,3.5186395E-1,3.3896515E-1,5.5086577E-1,2.50817E-3,4.5515284E-1,1.2398719E-2,-7.682308E-3,1.5941989E-2,7.21388E-3,-6.1493875E-3,-5.572754E-1,-2.5139728E-1,-3.7890363E-1,-2.4799474E-2,-8.165788E-3,-3.9765142E-2,-5.140965E-1,-8.829287E-2,1.6368343E-2,-3.6870784E-1,-1.1082643E-1,-3.2639615E-2,-1.2640074E-2,-2.3351198E-1,-2.1449238E-2,-1.4364861E-1,4.353454E-3,-1.0951741E-2,-2.4194948E-2,-4.2906734E-3,-6.7655332E-3,3.0694718E-3,1.466986E-1,-1.0219977E-2,-1.431233E-3,4.4948157E-2,-1.23716064E-1,1.07534515E-5,6.4470116E-3,-1.788205E-1,4.4589173E-3,-3.5369792E-3,-3.539829E-1,3.3945013E-3,3.3041522E-1,-8.911511E-2,3.346491E-3,1.3393517E-1,9.710789E-4,6.535678E-2,-1.5978272E-1,2.9281738E-1,1.3541977E-1,-1.1208273E-2,-2.3270192E-2,3.3248615E-1,1.0035607E-1,1.7052168E-2,8.423932E-3,7.1964967E-3,-1.5813684E-1,2.4378544E-1,1.21088795E-1,3.950115E-1,2.2485226E-1,2.0223472E-2,9.1843605E-3,2.9544543E-2,1.8596606E-2,1.14339385E-2,2.7257524E-2,-3.8188044E-2,1.2875234E-1,-3.0737322E-2,-1.7857263E-2,-2.8520766E-1,-1.2599266E-1,-5.138046E-1,-2.7844456E-1,-5.701568E-1,-1.4815066E-2,1.7761087E-2,-1.2737967E-1,-7.458742E-3,-4.0949488E-1,-1.6579932E-2,-6.2194694E-2,-1.4215251E-2,-6.75117E-3,-1.5456207E-1,-1.1841343E-2,5.6907095E-2,-3.94001E-3,2.3647808E-3,-2.750684E-3,4.5200292E-2,-6.8893125E-3,2.0852098E-1,7.35917E-2,5.625804E-3,-2.071619E-3,-7.6983883E-3,-3.1252925E-3,-2.1730144E-1,-6.627019E-2,-2.809776E-1,-3.384817E-2,2.0417599E-2,9.850273E-3,-6.1716046E-2,-2.3529378E-1,4.4452393E-1,-1.5987363E-2,8.455459E-2,1.934693E-2,-4.869904E-2,2.1445549E-1,1.1336926E-1,1.7677646E-2,-1.9522844E-2,-8.2513206E-2,1.341219E-1,3.2551092E-1,8.8289596E-2,2.3502737E-1,2.798102E-1,4.1411546E-1,-4.5179743E-3,8.295051E-3,-1.129751E-3,-1.3093078E-2,2.1310061E-2,2.16635E-1,-3.3379318E-3,1.5960993E-1,8.461231E-3,4.1792896E-1,7.494421E-3,1.3571425E-2,-1.04742646E-1,5.801775E-2,1.1379896E-2,2.0008865E-4,-1.2165331E-2,-2.1801341E-2,-2.3302687E-3,-1.1490633E-2,-1.2287839E-2,-2.7735766E-2,-7.038495E-4,-1.602074E-2,-1.5878439E-2,-3.0957634E-2,7.552933E-3,-6.0013833E-4,-7.995394E-3,3.2566022E-3,-2.2518199E-2,-1.224489E-2,2.4767537E-3,-5.0015165E-3,-5.673839E-3,-9.275722E-3,-4.080836E-3,2.187863E-3,4.731453E-3,1.0886819E-3,-1.8845812E-3,4.6343696E-3,5.3795804E-3,1.2061153E-2,-5.6121213E-4,6.396248E-3,-1.3981437E-2,-9.428253E-3,-6.1990827E-6,-7.441902E-3,-1.5795594E-2,-6.7656282E-3,5.572234E-3,-3.9034325E-3,-2.2204317E-2,-5.587315E-3,1.2330573E-2,3.0037092E-2,6.2523447E-3,-2.2117412E-3,-6.4642653E-3,5.69573E-3,-4.4185966E-3,3.743687E-3,3.3997172E-3,1.5026279E-2,3.8632392E-3,1.2193073E-2,1.0489875E-2,-5.5080926E-4,-1.0387166E-2,-6.9628784E-4,1.2664074E-2,3.0502367E-3,1.7224811E-2,6.67173E-3,8.608641E-3,1.2605548E-3,5.1329224E-3,1.5794745E-2,1.5075682E-2,3.6969443E-3,2.171912E-2,1.0390658E-2,1.2151147E-2,7.4488227E-3,8.872796E-3,1.5329911E-3,1.3268396E-2,2.2483734E-2,-7.345654E-3,9.4491325E-4,-1.3495531E-3,9.161042E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,-1,99,-1,101,-1,103,105,107,109,111,113,-1,115,117,119,121,123,-1,-1,125,127,129,-1,131,-1,-1,-1,-1,133,-1,135,137,139,141,-1,143,145,147,149,-1,-1,151,153,155,157,159,161,163,-1,165,-1,167,-1,-1,-1,169,171,173,-1,-1,-1,175,177,-1,179,181,-1,-1,183,-1,185,187,-1,-1,189,-1,191,193,-1,-1,195,197,-1,-1,199,-1,-1,201,-1,203,205,207,209,211,213,215,217,219,-1,-1,221,223,-1,-1,-1,225,227,229,231,233,-1,-1,-1,-1,-1,-1,235,237,-1,-1,239,241,243,245,247,-1,249,251,-1,253,-1,255,-1,-1,257,259,261,-1,-1,-1,263,-1,265,267,-1,-1,-1,-1,269,271,273,-1,-1,-1,275,277,279,281,283,-1,285,287,289,291,-1,293,295,297,299,301,303,305,-1,-1,-1,-1,-1,307,-1,309,-1,311,-1,-1,313,315,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1822052E1,9.795605E0,1.7561422E1,7.0123634E0,5.194217E0,1.1284731E1,5.374317E0,4.983959E0,0E0,1.4423594E0,7.9081035E-1,3.2538548E0,3.9045153E0,2.2293892E0,2.1143355E0,2.812828E0,3.7073102E0,4.28751E-1,4.2331612E-1,3.583793E-1,6.0148525E-1,5.939903E-1,1.7359673E0,2.755764E0,1.0347538E0,2.8087006E0,1.719779E0,1.394432E0,1.4773226E-1,8.394737E-1,2.2087288E-1,1.0957667E0,1.0008173E0,4.4590282E-1,2.5606906E-1,1.2046194E-1,3.2051298E-1,2.191087E-1,3.9111376E-2,6.451011E-2,1.0719311E-1,5.496917E-1,0E0,1.3156254E0,9.990165E-1,2.3085356E0,4.8684895E-1,1.1511812E0,4.9892753E-1,6.5741825E-1,2.2455502E-1,0E0,5.180235E-1,0E0,3.9917883E-1,0E0,2.8134286E-2,6.808467E-1,5.446224E-1,2.395339E-1,3.59066E-1,8.177252E-1,0E0,1.0600276E0,3.5472965E-1,4.5224547E-2,3.2331777E-1,5.246687E-2,0E0,0E0,5.2439444E-2,2.5214452E-1,6.224157E-2,0E0,1.5411493E-1,0E0,0E0,0E0,0E0,4.1452926E-2,0E0,3.6996114E-1,3.056314E-1,2.6805454E-1,1.2078295E0,0E0,4.9813753E-1,1.1119554E0,1.2196798E0,1.3861823E-1,0E0,0E0,6.845703E-1,2.3850143E-2,3.8642228E-1,3.5451746E-1,1.8519306E-1,6.847799E-2,1.0494137E-1,0E0,3.250847E-1,0E0,2.0469467E-1,0E0,0E0,0E0,6.940222E-2,4.4742298E-1,6.565037E-1,0E0,0E0,0E0,1.2371588E-1,4.0435743E-1,0E0,3.0582476E-1,2.9103404E-1,0E0,0E0,2.1760404E-2,0E0,2.5997186E-1,7.882471E-2,0E0,0E0,2.6186336E-2,0E0,1.5468812E-1,9.764564E-2,0E0,0E0,7.480692E-2,8.930609E-3,0E0,0E0,2.6711702E-1,0E0,0E0,5.416417E-1,0E0,3.4104586E-2,1.110928E0,2.43806E0,7.551516E-1,5.6184113E-1,4.1234E-1,4.1386592E-1,3.2990026E-1,7.773187E-1,0E0,0E0,2.797184E-1,2.2794437E-1,0E0,0E0,0E0,1.20805666E-1,3.6892414E-1,2.7846152E-1,1.1541605E-1,1.1768222E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.6219448E-1,1.1023547E-1,0E0,0E0,4.5510674E-1,2.0379925E-1,1.2889528E-1,3.9694428E-1,2.7566433E-2,0E0,1.09046504E-1,4.565941E-1,0E0,1.1972809E-1,0E0,1.10863484E-1,0E0,0E0,1.930232E-1,6.1843276E-2,1.2533519E-2,0E0,0E0,0E0,8.235055E-2,0E0,1.7100036E-2,6.1445422E-2,0E0,0E0,0E0,0E0,3.3052444E-2,9.840751E-2,9.533429E-2,0E0,0E0,0E0,6.850761E-1,1.0862486E0,2.2330475E-1,1.0989031E0,3.547923E-1,0E0,2.1783121E-1,1.05852365E-1,4.0507722E-1,5.09978E-1,0E0,1.6905427E-1,1.0660952E-1,1.7162514E-1,6.154096E-1,5.467684E-1,2.4367476E-1,5.370283E-2,0E0,0E0,0E0,0E0,0E0,1.1262441E-1,0E0,6.635237E-2,0E0,3.677559E-2,0E0,0E0,1.2821949E-1,1.8610416E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,54,54,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,74,74,79,79,81,81,82,82,83,83,84,84,86,86,87,87,88,88,89,89,92,92,93,93,94,94,95,95,96,96,97,97,98,98,100,100,102,102,106,106,107,107,108,108,112,112,113,113,115,115,116,116,119,119,121,121,122,122,125,125,127,127,128,128,131,131,132,132,135,135,138,138,140,140,141,141,142,142,143,143,144,144,145,145,146,146,147,147,148,148,151,151,152,152,156,156,157,157,158,158,159,159,160,160,167,167,168,168,171,171,172,172,173,173,174,174,175,175,177,177,178,178,180,180,182,182,185,185,186,186,187,187,191,191,193,193,194,194,199,199,200,200,201,201,205,205,206,206,207,207,208,208,209,209,211,211,212,212,213,213,214,214,216,216,217,217,218,218,219,219,220,220,221,221,222,222,228,228,230,230,232,232,235,235,236,236],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,-1,100,-1,102,-1,104,106,108,110,112,114,-1,116,118,120,122,124,-1,-1,126,128,130,-1,132,-1,-1,-1,-1,134,-1,136,138,140,142,-1,144,146,148,150,-1,-1,152,154,156,158,160,162,164,-1,166,-1,168,-1,-1,-1,170,172,174,-1,-1,-1,176,178,-1,180,182,-1,-1,184,-1,186,188,-1,-1,190,-1,192,194,-1,-1,196,198,-1,-1,200,-1,-1,202,-1,204,206,208,210,212,214,216,218,220,-1,-1,222,224,-1,-1,-1,226,228,230,232,234,-1,-1,-1,-1,-1,-1,236,238,-1,-1,240,242,244,246,248,-1,250,252,-1,254,-1,256,-1,-1,258,260,262,-1,-1,-1,264,-1,266,268,-1,-1,-1,-1,270,272,274,-1,-1,-1,276,278,280,282,284,-1,286,288,290,292,-1,294,296,298,300,302,304,306,-1,-1,-1,-1,-1,308,-1,310,-1,312,-1,-1,314,316,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.538E3,2.14099E5,9.026015E2,4.930349E5,3.5301748E7,1.0443182E3,4.50964E-2,2.3544883E8,1.948181E0,7.3E1,2.4684112E8,6.7652373E3,8.791777E3,3.1826714E7,6.747114E7,1.5988282E1,2.1924436E5,3.368326E11,2.4489644E0,7.2E1,9.772148E4,4.898012E-1,1.4274633E7,1.4813511E6,3.2475834E7,3.0972284E1,6.496696E7,5.85E2,2.8035452E11,2.6108465E0,5.5743E4,2.750917E-4,1.5669752E-2,1.5481343E1,6.162955E0,3.6604617E-2,1.8530114E-2,6.92E2,7.492813E0,1.8E1,3.8661438E-3,7E0,3.5E1,9.27E2,3.6022064E1,1.0918E4,1E1,4.1331047E-1,4.6816683E-1,4.2901665E-2,1.5E1,-2.1095376E-2,1.6507992E9,2.534327E-2,8.3510876E9,6.977611E5,9.619E3,9.18E2,1.12E3,9.637832E1,1.8571176E-2,2.867292E3,2.8312179E12,1.845406E1,4.4E1,1.5896305E-3,-5.3167827E-3,-1.1002511E-2,1.3097832E3,4.5479352E2,1.5988182E-1,9.982063E-3,4.0844156E7,4.253036E-3,1.158649E-2,1.1479812E-2,2.4770781E-2,3.6E1,1.2909125E-2,1.6213043E3,8E0,3.830986E-1,3.3030225E2,2.3260027E-2,1.3226592E-2,1.3767015E1,9.538462E0,1.2526851E6,3.547874E-3,3.548678E-2,4.524781E0,7.1913286E8,1.819149E1,1.7548548E7,2.0306142E7,1.2E1,2.3977574E7,2.50817E-3,5.3474556E5,1.2398719E-2,3.832274E2,1.5941989E-2,7.21388E-3,-6.1493875E-3,1E0,2.074E3,4.8632206E5,-2.4799474E-2,-8.165788E-3,-3.9765142E-2,1.631E4,2.2668628E7,1.6368343E-2,1.3238162E3,1.8238512E0,-3.2639615E-2,-1.2640074E-2,1.09E3,-2.1449238E-2,4E1,4.7326977E4,-1.0951741E-2,-2.4194948E-2,6.065707E8,-6.7655332E-3,1.3E1,1.9557823E0,-1.0219977E-2,-1.431233E-3,5.88E3,5.378E3,1.07534515E-5,6.4470116E-3,9.546517E-8,4.4589173E-3,-3.5369792E-3,1.0008265E1,3.3945013E-3,9.2058825E-1,9.337562E0,3.22E2,5.748E3,3.307766E6,1E0,6.743895E5,1.5687E4,5.862126E2,-1.1208273E-2,-2.3270192E-2,6.952967E6,7.5569354E2,1.7052168E-2,8.423932E-3,7.1964967E-3,1E0,5.236559E0,3.0324321E-2,9.000245E4,4.2E1,2.0223472E-2,9.1843605E-3,2.9544543E-2,1.8596606E-2,1.14339385E-2,2.7257524E-2,1E0,3.7448005E3,-3.0737322E-2,-1.7857263E-2,3.4782608E0,1.6256282E7,1.060324E6,2.8477445E0,7.8599895E9,-1.4815066E-2,1.366E3,2.3565867E3,-7.458742E-3,3.6282136E0,-1.6579932E-2,9.35E2,-1.4215251E-2,-6.75117E-3,5.1792985E6,3.2051282E-3,4E0,-3.94001E-3,2.3647808E-3,-2.750684E-3,4.1262585E-1,-6.8893125E-3,4.304E3,2.6E1,5.625804E-3,-2.071619E-3,-7.6983883E-3,-3.1252925E-3,8.9928055E-1,1.3446785E7,4.2E1,-3.384817E-2,2.0417599E-2,9.850273E-3,7.692308E-1,1.71E2,1.1693E4,3.0273972E0,1.45E2,1.934693E-2,1.5178888E0,1E0,5.0083565E6,4.144144E-1,-1.9522844E-2,2.0697437E2,1.2993591E6,5.724E3,4.822581E0,1E0,1.9162654E7,1E0,-4.5179743E-3,8.295051E-3,-1.129751E-3,-1.3093078E-2,2.1310061E-2,1.17E2,-3.3379318E-3,9.150449E5,8.461231E-3,8.079531E4,7.494421E-3,1.3571425E-2,2.3274304E2,1.6052323E5,1.1379896E-2,2.0008865E-4,-1.2165331E-2,-2.1801341E-2,-2.3302687E-3,-1.1490633E-2,-1.2287839E-2,-2.7735766E-2,-7.038495E-4,-1.602074E-2,-1.5878439E-2,-3.0957634E-2,7.552933E-3,-6.0013833E-4,-7.995394E-3,3.2566022E-3,-2.2518199E-2,-1.224489E-2,2.4767537E-3,-5.0015165E-3,-5.673839E-3,-9.275722E-3,-4.080836E-3,2.187863E-3,4.731453E-3,1.0886819E-3,-1.8845812E-3,4.6343696E-3,5.3795804E-3,1.2061153E-2,-5.6121213E-4,6.396248E-3,-1.3981437E-2,-9.428253E-3,-6.1990827E-6,-7.441902E-3,-1.5795594E-2,-6.7656282E-3,5.572234E-3,-3.9034325E-3,-2.2204317E-2,-5.587315E-3,1.2330573E-2,3.0037092E-2,6.2523447E-3,-2.2117412E-3,-6.4642653E-3,5.69573E-3,-4.4185966E-3,3.743687E-3,3.3997172E-3,1.5026279E-2,3.8632392E-3,1.2193073E-2,1.0489875E-2,-5.5080926E-4,-1.0387166E-2,-6.9628784E-4,1.2664074E-2,3.0502367E-3,1.7224811E-2,6.67173E-3,8.608641E-3,1.2605548E-3,5.1329224E-3,1.5794745E-2,1.5075682E-2,3.6969443E-3,2.171912E-2,1.0390658E-2,1.2151147E-2,7.4488227E-3,8.872796E-3,1.5329911E-3,1.3268396E-2,2.2483734E-2,-7.345654E-3,9.4491325E-4,-1.3495531E-3,9.161042E-3],"split_indices":[20,102,2,2,52,28,45,4,0,12,42,29,7,52,52,45,45,54,28,31,42,3,33,27,45,28,50,56,45,2,31,35,10,39,38,56,53,27,39,0,54,3,0,3,0,2,56,9,8,35,41,0,3,0,5,0,5,45,10,0,0,56,0,4,31,56,3,38,0,0,4,52,38,0,45,0,0,0,0,3,0,4,0,56,52,0,57,46,58,28,0,0,54,5,56,45,43,3,47,0,32,0,56,0,0,0,16,2,28,0,0,0,9,45,0,4,53,0,0,2,0,8,33,0,0,12,0,3,53,0,0,2,2,0,0,37,0,0,54,0,53,54,29,2,1,8,28,1,52,0,0,45,4,0,0,0,6,54,38,33,8,0,0,0,0,0,0,102,4,0,0,56,45,51,57,5,0,10,52,0,53,0,0,0,0,45,57,8,0,0,0,42,0,2,3,0,0,0,0,53,45,6,0,0,0,53,0,1,53,10,0,39,15,28,53,0,4,45,10,56,100,47,6,0,0,0,0,0,3,0,28,0,28,0,0,56,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.585E3,8.2E2,1.765E3,4.5E2,3.7E2,1.417E3,3.48E2,4.46E2,4E0,3.02E2,6.8E1,8.11E2,6.06E2,2.73E2,7.5E1,2.57E2,1.89E2,2.27E2,7.5E1,4E1,2.8E1,1.11E2,7E2,4.67E2,1.39E2,2.34E2,3.9E1,6E1,1.5E1,2E2,5.7E1,1.05E2,8.4E1,2.12E2,1.5E1,1.9E1,5.6E1,2.4E1,1.6E1,1.5E1,1.3E1,1.04E2,7E0,5.81E2,1.19E2,4.44E2,2.3E1,1.14E2,2.5E1,1.77E2,5.7E1,1.4E1,2.5E1,7E0,5.3E1,6E0,9E0,2.9E1,1.71E2,1.9E1,3.8E1,1E2,5E0,6.8E1,1.6E1,1.5E1,1.97E2,1E1,5E0,7E0,1.2E1,4.8E1,8E0,4E0,2E1,4E0,1.2E1,4E0,1.1E1,9E0,4E0,6.9E1,3.5E1,1.7E1,5.64E2,6E0,1.13E2,2.04E2,2.4E2,1.9E1,4E0,7E0,1.07E2,1E1,1.5E1,1.28E2,4.9E1,1.3E1,4.4E1,4E0,2.1E1,6E0,4.7E1,4E0,5E0,5E0,2.4E1,1.13E2,5.8E1,1.5E1,4E0,1.5E1,2.3E1,9.6E1,4E0,4E1,2.8E1,1.1E1,5E0,1.1E1,4E0,1.81E2,1.6E1,4E0,6E0,8E0,4E0,2.4E1,2.4E1,4E0,4E0,1.1E1,9E0,5E0,4E0,6.4E1,5E0,5E0,3E1,7E0,1E1,2.8E2,2.84E2,6.2E1,5.1E1,1.8E2,2.4E1,7.1E1,1.69E2,1.5E1,4E0,9.1E1,1.6E1,4E0,6E0,7E0,8E0,9.2E1,3.6E1,3.5E1,1.4E1,8E0,5E0,3.4E1,1E1,7E0,1.4E1,3.9E1,8E0,1.7E1,7E0,8.8E1,2.5E1,2.3E1,3.5E1,1.7E1,6E0,2.6E1,7E1,7E0,3.3E1,4E0,2.4E1,6E0,5E0,1.67E2,1.4E1,1E1,6E0,4E0,4E0,1.9E1,5E0,1.2E1,1.2E1,6E0,5E0,5E0,4E0,4.7E1,1.7E1,2.6E1,4E0,5E0,5E0,2.37E2,4.3E1,1.1E1,2.73E2,5.3E1,9E0,4.2E1,9E0,8.9E1,9.1E1,5E0,1.9E1,1.3E1,5.8E1,1.16E2,5.3E1,5.8E1,3.3E1,4E0,1.2E1,4E0,4E0,1E1,8.2E1,6E0,3E1,4E0,3.1E1,7E0,7E0,2.3E1,1.6E1,4E0,4E0,7.1E1,1.7E1,1.5E1,1E1,4E0,1.9E1,5E0,3E1,4E0,1.3E1,4E0,2.2E1,6E1,1E1,2.5E1,8E0,6E0,1.8E1,7.4E1,9.3E1,6E0,8E0,4E0,6E0,7E0,1.2E1,4E0,8E0,5E0,7E0,1.2E1,3.5E1,1E1,7E0,2E1,6E0,2E1,2.17E2,1.5E1,2.8E1,6E0,5E0,4.5E1,2.28E2,6E0,4.7E1,3.2E1,1E1,4E0,5E0,7.1E1,1.8E1,1.1E1,8E1,6E0,1.3E1,4E0,9E0,5.2E1,6E0,4.9E1,6.7E1,2.1E1,3.2E1,5.2E1,6E0,2.9E1,4E0,5.7E1,2.5E1,2.6E1,4E0,7E0,2.4E1,1.7E1,6E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"317","size_leaf_vector":"1"}},{"base_weights":[-3.6601538E-3,-1.7632887E-1,7.943816E-2,-2.746272E-1,-5.873367E-2,-3.7184544E-3,2.1169877E-1,-3.561211E-1,-1.7027856E-1,-1.1158249E-1,1.6202538E-1,-3.8962867E-2,1.2230006E-1,-9.772049E-3,2.4972664E-1,-3.179107E-1,-5.586165E-1,3.2447137E-2,-1.9450125E-1,-1.5708476E-1,-2.2115357E-2,8.946496E-2,3.006652E-1,-4.562143E-2,5.757345E-1,-1.6160895E-1,1.5266787E-1,6.224337E-2,-1.857487E-1,1.3363671E-1,3.226473E-1,-3.311099E-2,-3.0490515E-1,-6.2040025E-1,-3.449488E-1,-9.4141304E-2,-3.32878E-1,-2.3160161E-1,-1.06129766E-1,-4.4038348E-2,1.6923006E-1,2.3121554E-1,3.341164E-2,3.4752214E-1,4.715083E-3,-1.8726502E-1,-1.98212E-2,1.2580793E-2,3.9200768E-2,4.0813297E-2,-2.4035105E-2,1.1828349E-1,2.5294048E-1,9.317371E-2,-1.8221438E-2,-2.4251616E-1,-1.0760382E-1,3.2758048E-1,8.9211814E-2,4.6369752E-1,2.6516658E-1,-3.641243E-1,-2.41046E-1,-6.620557E-1,-1.4392553E-2,-5.7790033E-3,-2.295719E-2,-1.2550448E-1,2.137927E-1,-2.8404826E-1,-3.742738E-2,-2.62825E-1,-7.461541E-2,-1.2492279E-1,3.7508156E-2,-1.0564206E-2,-3.037829E-2,1.0664191E-2,3.6730284E-3,1.363049E-2,6.511428E-3,-6.369432E-3,6.253351E-2,2.1975689E-1,2.2650475E-2,-2.0323014E-1,1.2138389E-2,-1.2594979E-1,3.962506E-3,-2.3259357E-2,7.0352484E-3,9.697996E-2,2.7977332E-1,2.8313527E-1,-4.674717E-3,2.1396367E-2,2.3901844E-1,-4.3686107E-3,-1.4336711E-2,-6.877208E-3,-1.3000758E-3,5.4336037E-3,3.6198822E-1,-2.2591446E-1,1.0884656E-1,5.5860287E-1,3.3021337E-1,2.323566E-1,5.181761E-1,-2.065735E-1,-4.173502E-1,-1.0008561E-1,-2.7550483E-1,-7.3679805E-1,-1.8218048E-2,-6.749951E-2,-2.751899E-1,5.2339685E-5,1.6749414E-2,-4.4439813E-1,-1.7906542E-1,-1.7661504E-1,-3.4890786E-1,-6.851493E-3,-1.3903837E-4,-1.8909743E-1,-7.857229E-2,8.043471E-3,-2.2023754E-2,-1.3088493E-1,-1.1464193E-2,-1.0500441E-2,1.13056764E-1,1.4522286E-2,5.6086644E-3,-4.244687E-1,-1.7548841E-1,-1.05438344E-1,-2.625462E-2,2.526127E-2,-9.528465E-3,2.68175E-3,-5.5424855E-3,-7.345372E-2,1.0770327E-1,7.1171257E-3,2.1667467E-2,3.9419636E-1,2.1181542E-1,-6.1301764E-2,1.3196902E-1,2.9412442E-1,3.943189E-3,7.090379E-3,3.858089E-1,-2.2279495E-2,-1.8246623E-3,1.87282E-1,5.9448566E-2,4.3249193E-1,3.511284E-2,3.597147E-1,1.4070631E-3,2.5458944E-1,2.699192E-2,2.0942478E-2,3.564636E-2,-2.1209002E-3,-2.568664E-1,-4.8766634E-1,-2.6386482E-1,-1.8175025E-1,-1.6885873E-2,-2.914876E-1,1.6051322E-3,-2.0759288E-2,-4.106684E-2,-1.14702106E-1,4.1518804E-2,-3.14886E-2,-1.9318187E-1,-3.20087E-1,-3.4231156E-2,-1.942546E-2,-1.3889502E-1,-1.3571951E-2,-1.1779503E-1,-2.670529E-1,-2.8547104E-2,-2.6761928E-1,-1.4500308E-1,-2.12504E-1,-5.6046717E-2,-2.716947E-3,6.9944863E-4,-3.0387866E-3,-8.147294E-3,-3.6502823E-2,6.5572895E-2,3.713906E-2,-4.0078075E-3,5.2687526E-2,8.708167E-3,-1.2969812E-2,-3.1798482E-2,-1.8608546E-1,4.2657368E-4,-2.5157356E-1,-8.2812406E-2,-3.338159E-2,1.0005266E-1,-1.1203052E-2,4.596624E-3,1.5790685E-1,6.610437E-2,1.0638019E-2,2.1525009E-2,6.0125027E-2,2.7803266E-1,8.589181E-2,-1.2779863E-1,1.9350176E-1,5.108611E-2,1.0191604E-2,1.903238E-2,2.1434408E-2,1.0397424E-2,5.4406654E-2,2.3227127E-1,2.2208538E-2,1.4603588E-1,3.020876E-1,2.6201498E-2,2.3881088E-1,2.0767732E-2,2.1692224E-1,4.222514E-1,2.165444E-1,-1.3547495E-1,-1.5002856E-2,-2.4626562E-3,-2.0841511E-2,-3.0064438E-2,-1.7165028E-2,-5.834051E-3,-1.2813209E-2,-5.118565E-3,-5.133945E-3,1.9431193E-3,-1.1414423E-2,-1.819884E-2,-8.447219E-3,-2.4815337E-3,1.0572076E-2,-7.180582E-4,-1.1406187E-2,-3.5728372E-4,-8.564892E-3,-1.9906403E-2,-1.0250313E-2,-3.2994477E-3,5.951244E-4,-7.6226597E-3,-1.4999964E-2,-3.3876216E-3,-1.6206618E-2,-4.4615725E-3,-4.700478E-3,-1.144499E-2,-5.1528993E-3,-1.395971E-2,2.7645861E-3,-5.2347304E-3,2.5466478E-3,-3.4077668E-3,7.1684207E-4,6.7063267E-3,4.7246777E-3,-1.3843906E-3,4.5048078E-3,-5.1496644E-4,-8.586559E-3,-1.6851822E-2,-6.8617873E-3,-2.3756644E-2,5.975227E-3,-5.0066593E-3,-5.657754E-3,1.3710435E-4,9.558774E-3,1.095022E-3,9.104321E-3,-2.6040869E-3,-1.0592794E-2,4.2350716E-3,-1.5866784E-3,8.830694E-3,1.5549415E-2,5.148511E-3,6.725584E-3,9.995258E-4,-8.282413E-3,5.8168586E-4,4.6240217E-3,1.1890701E-2,4.497698E-3,-2.9203246E-4,6.746265E-3,-9.1028735E-3,4.637137E-3,1.3057846E-2,3.33802E-3,-3.8574534E-3,4.0417407E-3,1.4054159E-2,1.8651348E-2,6.0573556E-3,5.313949E-3,1.4710507E-2,1.33186625E-2,6.842266E-3,2.2412527E-2,8.468997E-3,5.6099603E-3,1.2995747E-2,1.5843238E-3,-2.0451786E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,79,81,83,-1,85,87,-1,-1,89,-1,91,93,95,-1,97,99,101,103,105,107,109,111,113,-1,-1,-1,115,117,119,-1,121,123,125,127,-1,129,-1,-1,-1,-1,-1,131,133,-1,135,-1,137,139,141,-1,143,145,147,-1,149,151,-1,-1,-1,-1,-1,153,155,157,159,161,163,165,167,169,171,173,175,-1,177,179,-1,-1,181,183,185,187,-1,-1,189,191,-1,193,195,197,199,201,-1,-1,203,205,207,-1,-1,209,-1,-1,211,213,-1,-1,215,217,219,221,223,-1,-1,225,-1,-1,227,229,231,-1,233,-1,235,237,-1,-1,-1,239,241,243,245,247,249,-1,-1,-1,251,253,-1,255,257,-1,-1,259,-1,261,263,-1,265,267,269,271,-1,-1,-1,-1,273,275,277,-1,279,-1,-1,-1,281,-1,283,285,287,289,-1,-1,291,293,-1,-1,295,297,299,301,303,305,-1,-1,-1,-1,307,309,311,313,315,-1,317,-1,319,321,323,325,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6832203E1,9.624422E0,1.9060917E1,3.8000717E0,4.4631505E0,4.7349076E0,5.636799E0,1.7877159E0,4.1170034E0,1.2503812E0,7.1752346E-1,3.4357352E0,2.0335267E0,1.2661917E0,4.7906E0,7.664261E-1,3.0084038E-1,0E0,2.6906095E0,7.5000954E-1,4.4791335E-1,3.8955423E-1,2.0887136E-1,3.0111985E0,4.1124558E-1,1.5067308E0,6.9756985E-1,9.715069E-1,1.02550864E-1,1.883513E0,2.7147255E0,0E0,7.0638275E-1,2.1078873E-1,2.3565745E-1,1.1374773E0,1.4535904E0,3.8232374E-1,3.367107E-1,2.1079595E-1,3.0846268E-2,2.3181558E-2,1.8132672E-1,1.7070651E-1,0E0,9.2162037E-1,1.7648668E0,0E0,0E0,9.936997E-2,0E0,5.198755E-1,5.809972E-1,6.9626516E-1,0E0,8.511001E-2,3.0721217E-2,2.69351E-1,1.1507015E0,1.0823803E0,1.9632092E0,8.110666E-1,4.7758818E-1,2.871666E-1,0E0,0E0,0E0,8.9308715E-1,2.8379345E-1,1.1829724E0,0E0,4.430189E-1,6.696799E-2,3.0936563E-1,1.13800116E-1,0E0,1.6768779E-1,0E0,0E0,0E0,0E0,0E0,1.19264364E-1,5.12017E-2,0E0,6.80428E-1,0E0,9.934528E-1,3.8755813E0,7.339363E-2,0E0,2.674321E-1,3.1859517E-1,3.0462503E-1,0E0,4.2872095E-1,1.7088556E-1,0E0,0E0,0E0,0E0,0E0,1.287446E-1,4.4200087E-1,6.5528965E-1,7.3207474E-1,3.8195324E-1,1.0158758E0,2.4241877E-1,2.2847557E-1,7.239294E-1,1.464095E-1,4.174552E-1,1.7802334E-1,0E0,4.010002E-1,7.2253656E-1,0E0,0E0,6.483774E-1,3.4192276E-1,1.6917968E-1,4.482174E-1,0E0,0E0,1.1700106E-1,1.8418756E-1,0E0,1.3580177E-2,2.1640852E-2,1.4899023E-1,4.9471658E-2,6.10411E-2,0E0,0E0,2.89114E-1,2.180531E-1,3.858595E-1,0E0,0E0,1.4618696E0,0E0,0E0,2.45107E-1,2.6940906E-1,0E0,0E0,2.7691603E-2,3.0470788E-1,2.7525043E-1,8.965367E-2,4.7484398E-2,0E0,0E0,1.4854908E-1,0E0,0E0,3.8190866E-1,3.4177974E-1,2.3701525E-1,0E0,1.7116642E-1,0E0,1.1841049E0,7.376443E-1,0E0,0E0,0E0,1.833576E-1,1.7542934E-1,2.7630723E-1,3.8887262E-2,6.146195E-2,2.7905416E-1,0E0,0E0,0E0,1.8250245E-1,2.3115548E-1,0E0,1.6158259E-1,1.7619729E-1,0E0,0E0,1.8435949E-1,0E0,1.100221E-1,1.5136719E-1,0E0,1.2278533E-1,1.1115438E-1,3.702128E-2,3.1507578E-1,0E0,0E0,0E0,0E0,1.6428764E-1,6.488506E-2,3.612915E-2,0E0,2.6892414E-2,0E0,0E0,0E0,1.6807675E-1,0E0,3.4394026E-1,3.9205462E-1,1.3243351E0,7.0881605E-1,0E0,0E0,3.157233E-1,4.0703744E-1,0E0,0E0,1.23463795E-1,9.5608115E-2,2.6191846E-2,1.0249981E-1,2.736491E-2,2.2687975E-2,0E0,0E0,0E0,0E0,3.6787215E-1,1.7433858E-1,3.4208715E-1,2.591735E-1,1.6074967E-1,0E0,7.915354E-2,0E0,6.264205E-1,1.6326904E-1,2.0742595E-2,6.081209E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,49,49,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,67,67,68,68,69,69,71,71,72,72,73,73,74,74,76,76,82,82,83,83,85,85,87,87,88,88,89,89,91,91,92,92,93,93,95,95,96,96,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,113,113,115,115,116,116,119,119,120,120,121,121,122,122,125,125,126,126,128,128,129,129,130,130,131,131,132,132,135,135,136,136,137,137,140,140,143,143,144,144,147,147,148,148,149,149,150,150,151,151,154,154,157,157,158,158,159,159,161,161,163,163,164,164,168,168,169,169,170,170,171,171,172,172,173,173,177,177,178,178,180,180,181,181,184,184,186,186,187,187,189,189,190,190,191,191,192,192,197,197,198,198,199,199,201,201,205,205,207,207,208,208,209,209,210,210,213,213,214,214,217,217,218,218,219,219,220,220,221,221,222,222,227,227,228,228,229,229,230,230,231,231,233,233,235,235,236,236,237,237,238,238],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,80,82,84,-1,86,88,-1,-1,90,-1,92,94,96,-1,98,100,102,104,106,108,110,112,114,-1,-1,-1,116,118,120,-1,122,124,126,128,-1,130,-1,-1,-1,-1,-1,132,134,-1,136,-1,138,140,142,-1,144,146,148,-1,150,152,-1,-1,-1,-1,-1,154,156,158,160,162,164,166,168,170,172,174,176,-1,178,180,-1,-1,182,184,186,188,-1,-1,190,192,-1,194,196,198,200,202,-1,-1,204,206,208,-1,-1,210,-1,-1,212,214,-1,-1,216,218,220,222,224,-1,-1,226,-1,-1,228,230,232,-1,234,-1,236,238,-1,-1,-1,240,242,244,246,248,250,-1,-1,-1,252,254,-1,256,258,-1,-1,260,-1,262,264,-1,266,268,270,272,-1,-1,-1,-1,274,276,278,-1,280,-1,-1,-1,282,-1,284,286,288,290,-1,-1,292,294,-1,-1,296,298,300,302,304,306,-1,-1,-1,-1,308,310,312,314,316,-1,318,-1,320,322,324,326,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,3.386E3,9.026015E2,1.022332E6,2.5998926E-1,4.8297736E7,2.66E2,6.7699623E-1,1.3664E4,2.03148E2,1.1317E4,5.9767612E7,1.0855529E3,2.9143541E-6,1E0,3.2447137E-2,7.218466E7,1.1185682E-3,1.6525185E9,6.5346925E6,1.0881593E10,1.0526E4,8.631404E2,1.3414634E0,4.273439E2,1.992945E5,9.245905E-2,7E0,2E0,-3.311099E-2,1.05E2,2.262586E6,2.6608378E8,7.0764465E1,9.60094E5,2.728E3,1.066317E12,2.5534486E4,6.071E3,2.6757258E-1,7.261386E7,4.2472876E11,4.715083E-3,1.2424786E2,4.54E2,1.2580793E-2,3.9200768E-2,2.2415E4,-2.4035105E-2,7.064848E6,3.6616542E0,1.5988182E-1,-1.8221438E-2,1E0,5.557511E8,5.0079144E5,4.97E2,1.7398448E7,7.1463747E0,1.6904992E-1,3.8609805E10,9.58059E2,-1.4392553E-2,-5.7790033E-3,-2.295719E-2,6.9514094E11,7.57657E4,2.3023027E3,-3.742738E-2,1.10597E5,1.1214854E1,2.4E1,1.8000048E10,-1.0564206E-2,1.96E2,1.0664191E-2,3.6730284E-3,1.363049E-2,6.511428E-3,-6.369432E-3,3.6482175E5,5.5012004E9,2.2650475E-2,3.983889E3,1.2138389E-2,4.3978744E0,3.92382E5,2.3314082E6,7.0352484E-3,1.1556145E2,1E0,2.5853658E0,-4.674717E-3,7.883E3,6.4863464E2,-4.3686107E-3,-1.4336711E-2,-6.877208E-3,-1.3000758E-3,5.4336037E-3,6.5E1,5.033165E7,3.1979167E0,1.4845161E3,5.847845E3,2.6025698E7,5.686354E-1,1.6633E4,1.9121015E0,2.755479E8,4.9363803E5,1.2920592E0,-1.8218048E-2,2.6036014E5,9.58059E2,5.2339685E-5,1.6749414E-2,2.8207E4,1.9469026E0,1.5697703E4,2.9138756E0,-6.851493E-3,-1.3903837E-4,6.172708E2,1.06E3,8.043471E-3,3.582157E12,1.22E2,2.3799367E0,1.775044E7,2.0676143E3,1.4522286E-2,5.6086644E-3,1.9044118E0,6.0078123E-6,1.76421E6,-2.625462E-2,2.526127E-2,1.2817779E7,2.68175E-3,-5.5424855E-3,7E2,1.3E1,7.1171257E-3,2.1667467E-2,7.417E3,5E0,3.71E0,3.1126543E1,4.5880157E2,3.943189E-3,7.090379E-3,9.6603775E0,-2.2279495E-2,-1.8246623E-3,1.3314917E0,3.3737933E8,1.7725672E8,3.511284E-2,1.6826648E3,1.4070631E-3,6.4683E4,1.35E2,2.0942478E-2,3.564636E-2,-2.1209002E-3,2.6563762E7,5.8372E4,2.2837209E2,5.818182E0,1.2352941E0,5.1100562E5,1.6051322E-3,-2.0759288E-2,-4.106684E-2,1.5E1,1E1,-3.14886E-2,4.0281758E6,1.0972E4,-3.4231156E-2,-1.942546E-2,4.73699E0,-1.3571951E-2,2.7523333E3,6.808926E6,-2.8547104E-2,6.126259E5,6.9983456E5,3.2E1,4.5033856E5,-2.716947E-3,6.9944863E-4,-3.0387866E-3,-8.147294E-3,4.61E2,1.2909952E1,7.3376025E6,-4.0078075E-3,2.9129332E1,8.708167E-3,-1.2969812E-2,-3.1798482E-2,1.9E1,4.2657368E-4,1.748435E4,9.928469E-1,8.5518695E4,1.6908121E1,-1.1203052E-2,4.596624E-3,2.1446484E-1,1.654762E2,1.0638019E-2,2.1525009E-2,1.61225E5,2.3821254E7,1.1508651E1,1.5121101E9,8.652122E8,5.159139E8,1.0191604E-2,1.903238E-2,2.1434408E-2,1.0397424E-2,2.298425E6,6.441311E-1,2E1,3.6873734E5,2.3370175E0,2.6201498E-2,6.852143E1,2.0767732E-2,1.3152658E7,5.54E2,8.3E1,2.14099E5,-1.5002856E-2,-2.4626562E-3,-2.0841511E-2,-3.0064438E-2,-1.7165028E-2,-5.834051E-3,-1.2813209E-2,-5.118565E-3,-5.133945E-3,1.9431193E-3,-1.1414423E-2,-1.819884E-2,-8.447219E-3,-2.4815337E-3,1.0572076E-2,-7.180582E-4,-1.1406187E-2,-3.5728372E-4,-8.564892E-3,-1.9906403E-2,-1.0250313E-2,-3.2994477E-3,5.951244E-4,-7.6226597E-3,-1.4999964E-2,-3.3876216E-3,-1.6206618E-2,-4.4615725E-3,-4.700478E-3,-1.144499E-2,-5.1528993E-3,-1.395971E-2,2.7645861E-3,-5.2347304E-3,2.5466478E-3,-3.4077668E-3,7.1684207E-4,6.7063267E-3,4.7246777E-3,-1.3843906E-3,4.5048078E-3,-5.1496644E-4,-8.586559E-3,-1.6851822E-2,-6.8617873E-3,-2.3756644E-2,5.975227E-3,-5.0066593E-3,-5.657754E-3,1.3710435E-4,9.558774E-3,1.095022E-3,9.104321E-3,-2.6040869E-3,-1.0592794E-2,4.2350716E-3,-1.5866784E-3,8.830694E-3,1.5549415E-2,5.148511E-3,6.725584E-3,9.995258E-4,-8.282413E-3,5.8168586E-4,4.6240217E-3,1.1890701E-2,4.497698E-3,-2.9203246E-4,6.746265E-3,-9.1028735E-3,4.637137E-3,1.3057846E-2,3.33802E-3,-3.8574534E-3,4.0417407E-3,1.4054159E-2,1.8651348E-2,6.0573556E-3,5.313949E-3,1.4710507E-2,1.33186625E-2,6.842266E-3,2.2412527E-2,8.468997E-3,5.6099603E-3,1.2995747E-2,1.5843238E-3,-2.0451786E-2],"split_indices":[20,102,33,2,52,28,42,45,0,42,2,56,9,45,52,42,16,0,45,57,5,45,12,9,52,53,52,33,41,3,17,0,52,1,7,56,9,10,31,33,2,38,7,31,0,56,2,0,0,29,0,28,53,38,0,8,7,28,2,50,35,27,31,4,0,0,0,31,28,55,0,29,56,3,5,0,0,0,0,0,0,0,28,5,0,28,0,53,5,28,0,52,102,54,0,2,4,0,0,0,0,0,0,7,54,52,4,43,38,1,53,5,33,56,0,33,4,0,0,10,53,50,58,0,0,55,2,0,31,10,39,45,4,0,0,53,37,31,0,0,9,0,0,2,3,0,0,12,8,54,58,4,0,0,56,0,0,54,7,12,0,4,0,2,8,0,0,0,12,29,4,58,56,47,0,0,0,3,3,0,32,9,0,0,57,0,47,9,0,33,33,3,48,0,0,0,0,0,56,32,0,56,0,0,0,8,0,28,53,28,58,0,0,38,4,0,0,1,9,56,5,7,7,0,0,0,0,28,42,3,28,38,0,58,0,45,8,3,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.565E3,8.33E2,1.732E3,4.53E2,3.8E2,1.064E3,6.68E2,2.53E2,2E2,3.07E2,7.3E1,8.32E2,2.32E2,9.8E1,5.7E2,2.15E2,3.8E1,5E0,1.95E2,2.03E2,1.04E2,4.9E1,2.4E1,8.24E2,8E0,2.2E1,2.1E2,7E1,2.8E1,2.21E2,3.49E2,6E0,2.09E2,2.8E1,1E1,1.14E2,8.1E1,8.1E1,1.22E2,9.4E1,1E1,1.3E1,3.6E1,1.9E1,5E0,1.26E2,6.98E2,4E0,4E0,1.4E1,8E0,1.58E2,5.2E1,6.6E1,4E0,1.5E1,1.3E1,4E1,1.81E2,9.9E1,2.5E2,1.06E2,1.03E2,2.4E1,4E0,4E0,6E0,1.04E2,1E1,7.4E1,7E0,6.7E1,1.4E1,1.08E2,1.4E1,6E0,8.8E1,6E0,4E0,8E0,5E0,5E0,3.1E1,1E1,9E0,1.22E2,4E0,1.27E2,5.71E2,9E0,5E0,1.41E2,1.7E1,4.8E1,4E0,4.5E1,2.1E1,4E0,1.1E1,9E0,4E0,6E0,3.4E1,1E1,1.71E2,5.6E1,4.3E1,2.23E2,2.7E1,2.8E1,7.8E1,2.1E1,8.2E1,1.8E1,6E0,7.6E1,2.8E1,4E0,6E0,2.8E1,4.6E1,3.5E1,3.2E1,7E0,7E0,4.4E1,6.4E1,4E0,1E1,1.3E1,7.5E1,1.3E1,1.8E1,5E0,5E0,1.2E1,1.1E2,1.22E2,5E0,1.4E1,5.57E2,5E0,4E0,8E0,1.33E2,1E1,7E0,1.7E1,3.1E1,2.6E1,1.9E1,1.5E1,6E0,4E0,3E1,4E0,6E0,6.5E1,1.06E2,3.2E1,2.4E1,3.9E1,4E0,2.01E2,2.2E1,2E1,7E0,7E0,2.1E1,5.2E1,2.6E1,1E1,1.1E1,7.8E1,4E0,5E0,1.3E1,5.3E1,2.3E1,4E0,2.4E1,2E1,8E0,6E0,4E1,1.2E1,2.3E1,2.5E1,7E0,1.4E1,3E1,8E0,5.6E1,5E0,5E0,5E0,8E0,5.7E1,1.8E1,8E0,5E0,1E1,8E0,8E0,4E0,1.04E2,6E0,1.5E1,1.07E2,4.58E2,9.9E1,4E0,4E0,5.9E1,7.4E1,4E0,1.3E1,1E1,2.1E1,8E0,1.8E1,1E1,9E0,9E0,6E0,2.3E1,7E0,1.7E1,4.8E1,7.5E1,3.1E1,1.5E1,1.7E1,1.4E1,2.5E1,1.66E2,3.5E1,1E1,1.2E1,1.7E1,4E0,3.5E1,1.7E1,1.6E1,1E1,4E0,6E0,4E0,7E0,4.4E1,3.4E1,2.8E1,2.5E1,5E0,1.8E1,2E1,4E0,8E0,1.2E1,2E1,2E1,5E0,1.8E1,2.1E1,4E0,1E1,4E0,2E1,1E1,4E0,4E0,1.7E1,3.9E1,1.5E1,4.2E1,1.1E1,7E0,4E0,4E0,6E0,4E0,9.7E1,7E0,1.1E1,4E0,8E0,9.9E1,1.42E2,3.16E2,4.5E1,5.4E1,5.3E1,6E0,4E0,7E1,6E0,4E0,1.7E1,4E0,4E0,4E0,1.4E1,4E0,4E0,6E0,5E0,4E0,1.3E1,4E0,9E0,3.9E1,5.2E1,2.3E1,2.2E1,9E0,1E1,5E0,5E0,9E0,1.01E2,6.5E1,3.1E1,4E0,4E0,6E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"327","size_leaf_vector":"1"}},{"base_weights":[1.577101E-3,-1.6978101E-1,8.287676E-2,-2.6273403E-1,-5.6215484E-2,1.761421E-2,2.15439E-1,-2.759793E-1,3.803493E-2,-1.1126443E-1,1.6473997E-1,-1.21687226E-1,5.6229733E-2,-1.1914405E-1,2.4033104E-1,-8.3898515E-2,-3.129307E-1,-1.5179178E-1,1.2368872E-2,1.021658E-1,2.9904935E-1,-1.0129387E-1,-4.3578976E-1,1.2227655E-1,-2.6345408E-2,-1.7771776E-1,-5.1684447E-2,2.1031667E-1,4.1646507E-1,-1.6976643E-1,9.070355E-3,-3.5555553E-1,-1.554074E-1,-3.497114E-1,-1.3686074E-1,-9.842515E-2,5.812547E-2,2.7068615E-1,3.2956045E-2,3.5047817E-1,7.9352325E-3,-1.5073597E-1,-1.58665E-2,-1.363436E-1,-3.7901532E-2,2.8578268E-2,1.0823294E-1,-4.0248428E-2,5.8677E-1,-1.0325963E-2,-3.5439746E-3,2.6541417E-2,-1.060468E-1,1.7160554E-1,3.3274674E-1,4.420401E-1,-1.7542336E-3,-2.1192482E-1,7.404143E-4,9.2857905E-2,-1.01467706E-1,-3.345058E-1,-7.21974E-1,-4.7958292E-2,-2.95256E-1,-2.5725895E-1,-2.4142116E-2,-1.5604773E-1,-3.3514544E-2,-1.2671122E-4,-1.430044E-1,3.7296377E-2,9.477392E-3,3.1383395E-1,6.0771345E-3,2.3613984E-3,1.1712608E-2,2.1970583E-2,1.0307547E-2,-1.9051374E-1,-9.861554E-2,9.534807E-2,-8.112033E-2,-9.852216E-3,-2.4105283E-3,5.6669243E-2,1.956636E-1,-7.083494E-2,1.3869107E-1,1.212897E-2,4.0645126E-2,4.0813773E-3,-1.6942421E-3,-7.156329E-3,-1.2670064E-3,2.013853E-1,6.632166E-2,3.491293E-2,2.9241824E-1,3.1576225E-1,5.351683E-1,-1.6428962E-1,-1.9138392E-2,4.0155843E-2,1.0888758E-2,-1.390949E-1,4.562517E-4,-2.8684556E-1,-4.102998E-1,-4.4775467E-2,-1.5560812E-2,6.194762E-2,-1.3166444E-1,-3.3996367E-1,-3.390689E-3,-1.5928993E-2,-7.6373573E-3,-1.3978389E-1,-2.9104236E-1,3.955033E-2,-1.2870008E-1,-9.0684675E-2,-1.0311615E-2,1.9291487E-2,6.128851E-3,8.537463E-3,1.8543756E-2,3.6261756E-2,-6.6744154E-3,-1.2254991E-1,-2.0719251E-1,-6.749693E-2,-2.019936E-1,1.1237656E-2,6.972314E-2,-5.2049827E-2,-1.834923E-2,-7.8332014E-2,9.2254594E-2,1.6464101E-1,3.9174297E-1,-4.8122484E-2,-2.437299E-1,6.837017E-2,2.2261025E-1,2.0948592E-1,-1.348323E-2,1.7033488E-2,1.642429E-1,1.9881482E-1,3.7319794E-1,3.5564977E-1,1.062398E-3,3.2223895E-2,1.8984072E-2,-1.9893435E-1,-7.8205176E-4,5.9688273E-3,3.7677113E-3,-8.716248E-3,-3.7312617E-3,-3.2845423E-1,-9.911003E-2,-5.4276794E-1,-3.5007566E-1,1.5026386E-2,-1.1925941E-2,-1.90789E-1,3.223772E-3,-6.1026295E-3,-3.9147863E-1,-2.2103485E-1,-1.1757078E-1,-2.123093E-2,-1.8870412E-1,1.20895386E-1,-4.875706E-3,-1.0099994E-3,-9.040071E-3,-6.5645203E-3,-1.1849968E-3,5.9200346E-2,-1.6052142E-2,6.336896E-2,-3.6821847E-3,-1.05344E-3,-8.993106E-3,-1.7051931E-2,-1.8003587E-1,-1.24477096E-1,-2.3044E-2,-4.033401E-3,-1.2719381E-2,-1.6247189E-3,9.780797E-2,1.8314958E-2,-8.383181E-2,-1.2974061E-1,1.8583211E-1,3.6140393E-2,1.8518212E-1,1.4771056E-1,3.4211838E-1,6.4334604E-3,2.2693548E-2,-3.0775967E-1,-2.8477829E-2,-2.8626293E-1,5.1339353E-3,1.22819126E-1,2.8419606E-3,2.6023012E-1,4.1633933E-3,1.1885416E-2,2.2292899E-1,-5.8011845E-2,6.0676817E-2,9.985589E-2,2.0480296E-1,2.2950731E-1,-5.548396E-3,3.9930215E-1,2.3963028E-3,8.104521E-3,4.0982977E-1,-1.4443942E-2,-7.629885E-3,-2.3225625E-3,2.3134993E-3,-3.0060286E-2,-1.5588591E-2,3.3255816E-3,-8.695823E-3,-1.7139593E-2,-3.0424248E-2,-2.5780708E-2,-1.5338365E-2,-3.8455154E-3,6.0026483E-3,-1.14690615E-2,-4.953504E-3,-2.1689603E-2,-7.5595663E-3,-1.3413088E-2,-8.025993E-3,-2.0848701E-3,-8.001084E-3,-1.2300759E-2,-5.6203487E-3,3.1991457E-3,8.01937E-3,4.334783E-3,-7.473547E-4,-2.31092E-3,9.1685966E-4,-4.901177E-4,4.209246E-3,-7.139132E-3,-1.1430023E-2,-3.3675174E-3,-9.025047E-3,9.2615094E-4,-3.1570503E-3,2.6776223E-3,8.02665E-3,2.2533922E-3,-2.7473525E-3,-9.984308E-3,-2.445593E-3,4.3157903E-3,-8.4125735E-3,2.588213E-3,1.4435041E-2,2.7760782E-3,-1.1059977E-2,4.5066033E-3,1.1591124E-2,-2.245416E-3,7.881521E-3,6.189502E-3,2.1253465E-2,-2.136703E-2,3.1567093E-3,4.0923033E-4,-5.630976E-3,-8.0352165E-3,-1.7642414E-2,7.6495563E-3,6.7394593E-4,-2.893111E-3,5.6827124E-3,1.5260476E-2,6.629998E-3,4.1584605E-3,-9.50574E-3,2.0904195E-2,1.0333363E-2,-5.0941585E-3,-1.3246081E-3,1.1603471E-2,1.1538275E-3,1.9480962E-3,6.883574E-3,5.2555744E-3,1.1565632E-2,1.4512688E-2,-3.3646313E-4,1.738208E-2,2.9155897E-2,2.3186147E-2,1.2840002E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,-1,-1,85,87,89,-1,-1,91,93,95,97,99,-1,101,-1,103,105,107,109,111,113,115,-1,117,119,-1,121,123,-1,125,-1,127,-1,-1,-1,129,131,133,135,-1,-1,137,139,141,143,-1,-1,-1,-1,-1,-1,145,147,-1,149,151,153,155,-1,157,-1,159,-1,161,163,-1,-1,165,167,169,-1,-1,-1,171,173,175,177,179,-1,181,-1,-1,-1,183,-1,185,187,189,191,-1,193,195,-1,197,199,201,203,205,207,209,211,213,-1,215,217,219,221,223,-1,-1,-1,225,-1,-1,227,-1,-1,229,231,233,235,-1,237,239,-1,-1,241,243,245,-1,247,249,-1,-1,-1,-1,-1,251,253,255,-1,-1,-1,-1,257,259,261,-1,-1,-1,263,265,267,269,271,273,275,277,279,-1,-1,281,283,285,-1,287,289,291,-1,293,295,297,299,301,303,305,-1,307,-1,-1,309,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.537185E1,8.609694E0,1.4893829E1,6.2995834E0,4.506611E0,6.2198157E0,4.7591934E0,3.125103E0,0E0,1.4849946E0,5.9136534E-1,1.5636644E0,4.938934E0,1.4886922E-1,2.708994E0,5.843018E-1,2.442505E0,6.11135E-1,3.8086826E-1,5.9756845E-1,1.1022377E-1,9.986272E-1,1.3462572E0,3.1145382E0,3.4588518E0,5.0242007E-2,8.786549E-2,2.0945892E0,8.9324284E-1,2.988485E-1,3.4284377E-1,2.0106964E0,1.2029953E0,6.077242E-2,4.0919685E-1,9.3680054E-2,1.3814807E-1,5.830598E-2,2.3365435E-1,1.1514294E-1,0E0,2.9219794E-1,6.463661E-1,3.872201E-2,0E0,0E0,2.1934452E0,2.1717129E0,5.043876E-1,0E0,0E0,3.2768294E-2,3.2364264E-2,1.0746031E0,1.3896894E0,6.777363E-1,0E0,1.8973267E-1,0E0,1.2966135E-1,6.7854494E-2,8.865452E-1,7.932844E-1,4.405469E-1,3.3275414E-1,2.1944582E-2,0E0,3.457303E-1,2.4155058E-1,0E0,3.1014413E-2,6.986996E-2,0E0,2.6685238E-2,0E0,1.6152842E-1,0E0,0E0,0E0,6.98266E-2,2.0190388E-1,9.387368E-2,4.4665653E-1,0E0,0E0,1.4916834E0,1.0365057E0,1.3155532E0,3.270539E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0871868E0,3.736584E-1,0E0,6.7436695E-1,3.8008547E-1,4.363861E-1,1.3077414E-1,0E0,4.5719385E-2,0E0,1.4297694E-2,0E0,1.3266344E0,6.476555E-1,0E0,0E0,3.740175E-1,3.2548854E-1,2.7760983E-1,0E0,0E0,0E0,2.609818E-1,1.6640031E-1,2.3489302E-1,8.13237E-2,2.4136156E-2,0E0,5.7371248E-2,0E0,0E0,0E0,8.812556E-2,0E0,1.06927544E-1,1.788652E-1,1.3137712E-1,7.506907E-2,0E0,8.6450085E-2,1.1782466E-1,0E0,9.068301E-1,1.2708733E0,4.242196E-1,3.355677E-1,1.5213805E0,6.041446E-1,1.1739306E-1,1.145556E-1,7.0394135E-1,0E0,1.7710708E-1,4.744929E-2,4.8197794E-1,4.2233658E-1,2.374804E-1,0E0,0E0,0E0,4.337603E-2,0E0,0E0,2.7799627E-2,0E0,0E0,4.5131397E-1,4.211824E-1,2.0058346E-1,3.7787724E-1,0E0,1.5525092E-1,5.1818192E-2,0E0,0E0,1.6967726E-1,5.138302E-2,4.0325046E-1,0E0,2.4879813E-2,1.9000098E-2,0E0,0E0,0E0,0E0,0E0,4.0119715E-2,2.3702847E-2,3.5955966E-2,0E0,0E0,0E0,0E0,7.049942E-2,6.1537713E-2,5.2908882E-2,0E0,0E0,0E0,5.529535E-2,3.5853278E-2,1.3589376E-1,4.736268E-1,1.3698584E-1,7.822895E-1,3.8802266E-1,2.8905988E-1,1.8346298E-1,0E0,0E0,9.607308E-1,8.6557806E-1,2.3661661E-1,0E0,5.7513267E-2,1.1435548E-1,7.472944E-2,0E0,2.7538556E-1,6.981163E-1,2.4980813E-2,2.1385545E-1,2.231709E-2,1.5498459E-2,6.354041E-1,0E0,2.5173712E-1,0E0,0E0,6.303382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,70,70,71,71,73,73,75,75,79,79,80,80,81,81,82,82,85,85,86,86,87,87,88,88,95,95,96,96,98,98,99,99,100,100,101,101,103,103,105,105,107,107,108,108,111,111,112,112,113,113,117,117,118,118,119,119,120,120,121,121,123,123,127,127,129,129,130,130,131,131,132,132,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,143,143,144,144,145,145,147,147,148,148,149,149,150,150,151,151,155,155,158,158,161,161,162,162,163,163,164,164,166,166,167,167,170,170,171,171,172,172,174,174,175,175,181,181,182,182,183,183,188,188,189,189,190,190,194,194,195,195,196,196,197,197,198,198,199,199,200,200,201,201,202,202,205,205,206,206,207,207,209,209,210,210,211,211,213,213,214,214,215,215,216,216,217,217,218,218,219,219,221,221,224,224],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,-1,-1,86,88,90,-1,-1,92,94,96,98,100,-1,102,-1,104,106,108,110,112,114,116,-1,118,120,-1,122,124,-1,126,-1,128,-1,-1,-1,130,132,134,136,-1,-1,138,140,142,144,-1,-1,-1,-1,-1,-1,146,148,-1,150,152,154,156,-1,158,-1,160,-1,162,164,-1,-1,166,168,170,-1,-1,-1,172,174,176,178,180,-1,182,-1,-1,-1,184,-1,186,188,190,192,-1,194,196,-1,198,200,202,204,206,208,210,212,214,-1,216,218,220,222,224,-1,-1,-1,226,-1,-1,228,-1,-1,230,232,234,236,-1,238,240,-1,-1,242,244,246,-1,248,250,-1,-1,-1,-1,-1,252,254,256,-1,-1,-1,-1,258,260,262,-1,-1,-1,264,266,268,270,272,274,276,278,280,-1,-1,282,284,286,-1,288,290,292,-1,294,296,298,300,302,304,306,-1,308,-1,-1,310,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,2.8357262E6,1.9013363E4,9.026015E2,4.97E2,2.2580828E-1,1.7322648E5,3.803493E-2,1.1997242E0,1.91E4,4.602015E0,1.9E1,6.131E3,3.0121632E6,8.974133E9,1.067536E3,1.4123257E-4,1.752512E2,6.5346925E6,6.9514094E11,3.5E2,6.342E3,2.04E5,1.8552577E2,2.3898147E5,1.7725672E8,3.970405E3,5.2610065E1,1.4510472E1,9.599108E6,1.3521131E8,8.229907E7,2.22E3,4.6547272E2,2.909019E3,1.6525185E9,2.6757258E-1,1.9495119E6,2.7711287E0,7.9352325E-3,2.0550128E5,1.3717948E0,7.535486E0,-3.7901532E-2,2.8578268E-2,1.5859042E6,7.06699E8,9.287868E6,-1.0325963E-2,-3.5439746E-3,4.089454E4,2.512E3,9.070543E0,5.4379158E1,1.289627E3,-1.7542336E-3,4.597929E6,7.404143E-4,7.805608E4,1.3238013E2,2.467284E7,4.822581E0,3.0715862E7,7.342736E3,1.06E3,-2.4142116E-2,1.5988282E1,7.9240966E0,-1.2671122E-4,1.9624242E7,2.2376953E1,9.477392E-3,5.49E2,6.0771345E-3,6.496696E7,1.1712608E-2,2.1970583E-2,1.0307547E-2,2.645E3,1.83096E5,8.54127E7,2.3E2,-9.852216E-3,-2.4105283E-3,1E0,7.780377E6,3.956147E-1,1.5963264E9,1.212897E-2,4.0645126E-2,4.0813773E-3,-1.6942421E-3,-7.156329E-3,-1.2670064E-3,4.1822004E7,1.058E3,3.491293E-2,1.6990049E0,5.911314E1,1.5846036E7,3.73802E5,-1.9138392E-2,3.757042E6,1.0888758E-2,1.5324128E6,4.562517E-4,2.074E3,3.449702E2,-4.4775467E-2,-1.5560812E-2,2.0521326E0,6.316106E6,1.120848E5,-3.390689E-3,-1.5928993E-2,-7.6373573E-3,1.09E3,2.3067484E0,6.9740294E5,2.373E3,1.803E3,-1.0311615E-2,7.504E3,6.128851E-3,8.537463E-3,1.8543756E-2,2.7556E4,-6.6744154E-3,1E1,8.9928055E-1,3.3820656E7,1.6521739E0,1.1237656E-2,8.777E3,4E1,-1.834923E-2,4.9612486E-1,3.3926086E2,2.2376953E1,1E0,3.983889E3,1E0,2.5291866E1,1.3152658E7,1.55E2,-1.348323E-2,8.360387E-1,1.4563726E-1,2.4507338E4,1.9099288E7,1.136E3,1.062398E-3,3.2223895E-2,1.8984072E-2,1.11E3,-7.8205176E-4,5.9688273E-3,5.4410156E4,-8.716248E-3,-3.7312617E-3,2.35E2,2.9311974E8,1.96013E9,1.7446084E0,1.5026386E-2,2.273139E1,9.064853E-1,3.223772E-3,-6.1026295E-3,4.099E3,2.23329E5,8.781059E4,-2.123093E-2,2.9E1,1.5153E4,-4.875706E-3,-1.0099994E-3,-9.040071E-3,-6.5645203E-3,-1.1849968E-3,3.3410483E8,1.023747E6,5.1180666E8,-3.6821847E-3,-1.05344E-3,-8.993106E-3,-1.7051931E-2,3.6222222E0,1.6194E4,2.7E1,-4.033401E-3,-1.2719381E-2,-1.6247189E-3,1.5288235E2,8.3E1,8.274E3,2.241E2,1.7685275E6,2.9676E4,1.13E3,8.6906275E-6,1.9495119E6,6.4334604E-3,2.2693548E-2,6.34E2,2.9833334E1,9.2199E4,5.1339353E-3,2.867292E3,1.2219736E5,2.512112E1,4.1633933E-3,5.6202265E6,1.8601036E1,2.870791E6,8.891209E6,1.96013E9,2.613714E6,2.2418336E5,-5.548396E-3,1.9013363E4,2.3963028E-3,8.104521E-3,6.930352E10,-1.4443942E-2,-7.629885E-3,-2.3225625E-3,2.3134993E-3,-3.0060286E-2,-1.5588591E-2,3.3255816E-3,-8.695823E-3,-1.7139593E-2,-3.0424248E-2,-2.5780708E-2,-1.5338365E-2,-3.8455154E-3,6.0026483E-3,-1.14690615E-2,-4.953504E-3,-2.1689603E-2,-7.5595663E-3,-1.3413088E-2,-8.025993E-3,-2.0848701E-3,-8.001084E-3,-1.2300759E-2,-5.6203487E-3,3.1991457E-3,8.01937E-3,4.334783E-3,-7.473547E-4,-2.31092E-3,9.1685966E-4,-4.901177E-4,4.209246E-3,-7.139132E-3,-1.1430023E-2,-3.3675174E-3,-9.025047E-3,9.2615094E-4,-3.1570503E-3,2.6776223E-3,8.02665E-3,2.2533922E-3,-2.7473525E-3,-9.984308E-3,-2.445593E-3,4.3157903E-3,-8.4125735E-3,2.588213E-3,1.4435041E-2,2.7760782E-3,-1.1059977E-2,4.5066033E-3,1.1591124E-2,-2.245416E-3,7.881521E-3,6.189502E-3,2.1253465E-2,-2.136703E-2,3.1567093E-3,4.0923033E-4,-5.630976E-3,-8.0352165E-3,-1.7642414E-2,7.6495563E-3,6.7394593E-4,-2.893111E-3,5.6827124E-3,1.5260476E-2,6.629998E-3,4.1584605E-3,-9.50574E-3,2.0904195E-2,1.0333363E-2,-5.0941585E-3,-1.3246081E-3,1.1603471E-2,1.1538275E-3,1.9480962E-3,6.883574E-3,5.2555744E-3,1.1565632E-2,1.4512688E-2,-3.3646313E-4,1.738208E-2,2.9155897E-2,2.3186147E-2,1.2840002E-2],"split_indices":[20,102,51,52,52,2,42,28,0,41,2,53,3,2,28,20,52,42,52,45,31,2,12,5,56,33,12,52,56,57,29,45,45,2,52,32,5,38,28,42,0,45,53,53,0,0,28,7,1,0,0,28,0,53,58,4,0,47,0,28,56,45,56,50,55,2,0,54,54,0,32,56,0,0,0,45,0,0,0,9,47,31,0,0,0,102,28,27,7,0,0,0,0,0,0,51,0,0,41,58,47,33,0,29,0,32,0,2,52,0,0,53,1,32,0,0,0,2,58,47,2,2,0,10,0,0,0,9,0,3,53,7,53,0,1,0,0,41,52,56,102,28,89,58,45,10,0,39,38,4,9,29,0,0,0,2,0,0,28,0,0,2,7,5,53,0,56,27,0,0,0,12,33,0,3,10,0,0,0,0,0,7,1,12,0,0,0,0,58,9,10,0,0,0,52,10,1,4,45,29,2,38,28,0,0,0,53,12,0,4,28,58,0,47,58,48,32,5,9,28,0,52,0,0,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.537E3,8.16E2,1.721E3,4.48E2,3.68E2,1.154E3,5.67E2,4.43E2,5E0,2.95E2,7.3E1,2.5E2,9.04E2,3.9E1,5.28E2,7.2E1,3.71E2,2.22E2,7.3E1,5.1E1,2.2E1,2.36E2,1.4E1,5.02E2,4.02E2,2E1,1.9E1,4.53E2,7.5E1,3.7E1,3.5E1,2.91E2,8E1,1.4E1,2.08E2,2.1E1,5.2E1,1.4E1,3.7E1,1.5E1,7E0,1.49E2,8.7E1,8E0,6E0,1.4E1,4.88E2,3.94E2,8E0,1.5E1,5E0,8E0,1.1E1,3.46E2,1.07E2,7.1E1,4E0,3E1,7E0,2E1,1.5E1,2.77E2,1.4E1,4.6E1,3.4E1,1E1,4E0,1.75E2,3.3E1,7E0,1.4E1,4.6E1,6E0,1E1,4E0,3.3E1,4E0,8E0,7E0,8.3E1,6.6E1,3.2E1,5.5E1,4E0,4E0,3.08E2,1.8E2,3.37E2,5.7E1,4E0,4E0,4E0,4E0,7E0,4E0,2.69E2,7.7E1,9E0,9.8E1,3.2E1,3.9E1,2.5E1,5E0,1.5E1,5E0,1.1E1,4E0,1.73E2,1.04E2,9E0,5E0,2E1,2.6E1,2.8E1,6E0,5E0,5E0,1.58E2,1.7E1,1.9E1,1.4E1,9E0,5E0,3.9E1,7E0,4E0,6E0,2.7E1,6E0,1.8E1,6.5E1,5.2E1,1.4E1,4E0,2.8E1,5.1E1,4E0,6.4E1,2.44E2,1.57E2,2.3E1,2.99E2,3.8E1,3.2E1,2.5E1,2.65E2,4E0,5.2E1,2.5E1,4.7E1,5.1E1,2.8E1,4E0,2.1E1,1.8E1,2E1,5E0,4E0,1.1E1,6E0,5E0,1.41E2,3.2E1,3E1,7.4E1,4E0,1.6E1,2E1,6E0,6E0,2.2E1,3.2E1,1.26E2,6E0,1.1E1,1.2E1,7E0,5E0,9E0,5E0,4E0,1.8E1,2.1E1,2.2E1,5E0,7E0,1.1E1,9E0,5.6E1,2.2E1,3E1,5E0,9E0,6E0,2.2E1,1.6E1,3.5E1,5.4E1,1E1,1.53E2,9.1E1,1.45E2,1.2E1,5E0,1.8E1,2E1,2.79E2,3.4E1,4E0,1.7E1,1.5E1,1.9E1,6E0,1.7E1,2.48E2,1.9E1,3.3E1,1.1E1,1.4E1,4.3E1,4E0,4.7E1,4E0,7E0,2.1E1,5E0,1.5E1,5E0,6E0,6E0,1.35E2,1E1,2.2E1,9E0,2.1E1,1.3E1,6.1E1,1.1E1,5E0,1.3E1,7E0,1.8E1,4E0,1.6E1,1.6E1,4.6E1,8E1,5E0,6E0,6E0,6E0,1.3E1,5E0,1.1E1,1E1,5E0,1.7E1,3.4E1,2.2E1,1.2E1,1E1,1.5E1,1.5E1,1.4E1,8E0,1.2E1,4E0,7E0,2.8E1,8E0,4.6E1,5E0,5E0,1.43E2,1E1,3.1E1,6E1,7E0,1.38E2,4E0,8E0,1.5E1,5E0,1.95E2,8.4E1,1.3E1,2.1E1,1.3E1,4E0,1E1,5E0,1.3E1,6E0,1.3E1,4E0,1.7E1,2.31E2,7E0,1.2E1,5E0,2.8E1,5E0,6E0,4E0,1E1,3.4E1,9E0,3.9E1,8E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"311","size_leaf_vector":"1"}},{"base_weights":[-4.370604E-6,-1.5738964E-1,7.487176E-2,-2.428942E-1,-5.3353593E-2,3.0001001E-2,2.619E-1,-2.5233588E-1,3.256118E-2,-1.1432831E-1,1.23774804E-1,-4.396162E-2,1.3107003E-1,4.3109992E-1,2.062453E-1,-2.869776E-1,-9.1836005E-2,-1.6028099E-1,-4.87713E-2,5.7655003E-2,2.3736487E-1,-1.8625738E-1,-2.290812E-2,1.4274357E-1,-2.0934655E-1,6.0732737E-2,4.8762536E-1,1.5954632E-1,4.0852493E-1,-2.4189007E-1,-4.9736375E-1,1.5426969E-2,-1.166551E-1,-9.85381E-2,-2.0022741E-1,1.9761115E-2,-1.6557659E-1,-5.78905E-2,8.9491375E-2,1.608996E-3,2.6279885E-1,-1.587777E-1,-4.041513E-1,-9.4966E-3,-1.7520931E-1,4.9539585E-2,2.0085533E-1,-2.7659088E-1,2.3836442E-3,-4.8016957E-3,7.5529786E-3,3.7786294E-2,4.1438243E-1,-2.0029749E-1,1.825995E-1,3.6163762E-1,3.6478207E-2,-3.105236E-1,-1.4584336E-1,-5.954317E-1,-2.83078E-1,-7.413315E-2,-2.6237178E-1,-1.4008538E-1,-5.0016448E-2,-1.3922425E-1,-2.889444E-1,-9.993812E-3,9.547842E-3,-3.0594462E-1,-7.945283E-2,2.9432103E-3,-6.461733E-3,3.8494527E-2,1.6798668E-1,3.748797E-1,1.8241037E-1,-2.6476073E-1,-1.2466256E-1,-3.2432225E-2,-8.5677E-3,-3.565066E-2,7.480152E-2,-4.2170665E-1,-1.2635858E-1,7.434452E-2,-9.402628E-2,1.6153227E-1,2.9971594E-1,-9.7289095E-3,-2.0875657E-2,4.7146904E-1,1.8050808E-1,-1.9291723E-2,-2.4286216E-2,6.076621E-2,2.1344928E-1,2.940718E-1,2.3392182E-2,-2.9530907E-1,-2.9731015E-2,-5.4947738E-2,-2.7106974E-1,-2.755817E-1,-6.8400496E-1,-3.4908018E-1,-4.585753E-3,-9.377344E-2,8.224154E-3,-1.4845033E-1,-2.528779E-2,-4.885241E-2,-1.6530803E-1,4.3707653E-3,-8.510929E-2,-1.860505E-1,-9.837206E-3,-3.5913718E-1,-1.3794744E-1,4.768824E-2,-5.9150033E-2,-1.9261371E-2,-5.4529086E-3,-1.2296976E-1,2.2340459E-3,1.1189645E-1,-2.073444E-2,4.606102E-3,2.1473594E-1,2.1911228E-2,9.897606E-3,4.7466257E-3,1.0647314E-2,-1.7395418E-2,-1.8025628E-1,-9.847487E-2,-2.0513529E-1,2.08233E-2,-7.018939E-2,2.5887644E-1,1.0125011E-2,-2.8418656E-2,-9.509639E-3,-1.5159911E-1,6.2996903E-3,5.2444153E-2,1.8427405E-1,-2.1274193E-1,-2.499837E-2,7.501806E-2,1.9150524E-1,2.7966255E-1,3.1018658E-2,2.910611E-1,2.666815E-2,4.3576537E-3,1.1388264E-2,5.0916392E-3,-6.826737E-3,1.1311642E-1,-2.810077E-2,2.3635954E-1,-1.7879624E-3,1.8938474E-2,2.0136075E-1,-2.2993079E-1,-3.3877456E-1,6.56334E-2,-1.2272986E-1,-8.365381E-2,-3.1176305E-1,-1.712028E-2,-6.3253846E-3,-1.6292363E-2,-7.2213244E-1,-9.983009E-3,-1.9925445E-2,-1.5075391E-2,-1.3590549E-1,-1.603309E-2,-3.6163956E-2,-4.5148027E-3,1.2101669E-4,-9.958643E-3,-1.06202096E-1,-5.544738E-2,-9.923373E-3,-2.563893E-1,-1.4467941E-1,5.350551E-3,-5.4049585E-2,-6.9887848E-3,-4.3255872E-1,-9.388653E-3,-3.0556584E-3,1.0337E-2,1.21357016E-1,2.2748739E-2,-8.837456E-2,-1.5407133E-1,3.6823028E-4,1.9615034E-3,8.940468E-3,4.2374145E-2,-6.293727E-3,1.2927048E-2,5.5073393E-3,-1.1169207E-2,-4.6224687E-3,-1.3592711E-1,-3.2713234E-2,-1.2996436E-2,-1.0590161E-3,-8.4148236E-2,8.9101925E-2,-8.275432E-2,7.535587E-2,4.3386552E-1,1.1067299E-1,6.767831E-2,-8.781722E-2,-1.8055524E-1,2.444566E-4,-7.744838E-2,7.610985E-2,1.4491996E-1,1.7984563E-2,-3.934162E-3,-1.4850727E-2,4.7841456E-2,-7.003838E-2,1.3342065E-1,-9.1379E-2,2.1523614E-1,2.8170323E-2,2.1217644E-1,3.3138806E-1,1.7621096E-2,5.4791966E-3,5.1008232E-2,1.3711299E-2,-1.0404382E-1,3.0403594E-3,1.9746347E-1,3.5296005E-1,-1.3806398E-1,9.650936E-3,1.3130375E-2,2.7578464E-3,-1.2557547E-2,-1.5033922E-3,-1.7660756E-2,-2.4952895E-3,-9.817325E-4,7.6108207E-3,1.0551626E-3,-8.492023E-3,2.1448417E-3,-9.8079955E-3,-2.1694276E-2,-1.3385105E-2,-3.8736764E-2,-1.9451948E-2,-3.1958104E-3,5.8857263E-3,-8.1542E-3,3.5662137E-3,-1.4221246E-4,-3.1103739E-3,-7.42448E-3,-8.5658825E-4,1.3479452E-3,-3.7606212E-3,-1.4908224E-2,-5.634662E-3,-2.006634E-3,-8.193386E-3,-5.1860646E-3,2.3131547E-3,-2.5994042E-2,-1.23410355E-2,1.8657194E-3,-4.174123E-3,8.84956E-3,2.7317093E-3,-2.0806517E-3,3.628341E-3,1.117496E-3,-5.7864436E-3,-2.8176892E-3,-9.497265E-3,4.1022976E-3,-5.5142416E-4,1.1911393E-3,-7.826919E-3,2.3000222E-3,-2.9271E-3,-2.8270802E-3,-1.5388006E-2,1.4738891E-2,3.110572E-3,-5.842198E-3,-1.0781355E-3,6.2685516E-3,-3.5672043E-3,9.516608E-3,2.447397E-2,-5.3861947E-3,8.861459E-3,8.175273E-3,7.1466767E-4,-2.959914E-3,-1.46489E-2,-9.986829E-3,-6.5481476E-4,1.4535167E-3,-1.2000221E-2,-5.6865844E-3,4.4454983E-3,8.941213E-3,1.106581E-3,4.321555E-3,-1.8694447E-5,6.9242966E-4,-5.9054173E-3,-3.2094414E-3,1.0223282E-2,-1.8565994E-2,2.2468988E-3,3.4170857E-4,1.1648847E-2,-7.7295373E-3,4.699799E-3,1.2200157E-2,4.6974234E-3,5.8409628E-3,1.7574228E-2,6.57378E-3,-6.8248255E-4,-1.9827487E-3,-7.3749525E-3,1.5890066E-2,7.956431E-3,2.7022265E-2,1.1972387E-2,1.0258117E-3,-1.5028168E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,-1,75,77,79,81,83,85,87,89,-1,-1,-1,-1,91,93,95,97,-1,99,101,103,105,107,109,111,113,115,117,119,-1,121,123,-1,-1,125,127,129,131,133,135,-1,-1,137,139,141,143,145,147,149,151,-1,-1,153,155,157,-1,159,161,163,-1,165,-1,167,169,171,173,175,-1,177,-1,179,-1,181,183,-1,185,187,189,191,193,195,197,-1,-1,199,-1,201,203,-1,205,-1,-1,-1,-1,-1,207,209,211,213,215,217,219,-1,-1,221,-1,223,225,227,229,231,233,235,-1,237,-1,-1,-1,-1,-1,239,241,243,245,-1,247,249,251,253,255,257,259,-1,-1,-1,261,-1,-1,263,265,-1,267,-1,-1,-1,269,271,-1,273,275,-1,277,-1,279,-1,-1,281,283,285,287,289,-1,-1,-1,291,-1,-1,-1,-1,-1,293,295,-1,-1,297,299,301,303,305,307,309,311,313,-1,315,317,319,-1,-1,-1,321,323,325,327,329,331,333,335,-1,-1,337,-1,339,-1,341,343,345,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.012093E1,7.3169785E0,1.4527262E1,3.9820595E0,4.0426836E0,1.0456275E1,3.0657043E0,2.4645195E0,0E0,8.296635E-1,7.0732784E-1,2.4163594E0,2.378437E0,1.6870699E0,2.3341694E0,3.3704643E0,8.4132385E-1,3.8229418E-1,9.31846E-1,2.3325075E-1,1.7417157E-1,5.585458E-1,1.4383622E0,3.0881424E0,3.5889953E-1,1.8470812E-1,1.0834427E0,1.7620478E0,4.4453335E-1,1.9652805E0,1.1763783E0,0E0,4.5765364E-1,1.2604564E-1,4.901495E-1,3.7913996E-1,4.9552E-1,1.2441438E-1,1.8968108E-1,0E0,2.1092844E-1,3.0686307E-1,5.1559496E-1,1.4335419E0,6.330832E-1,7.940029E-1,1.3199873E0,1.0364354E-1,0E0,0E0,0E0,0E0,6.892595E-1,6.6301656E-1,7.237015E-1,1.7179918E-1,0E0,5.9041786E-1,1.4534516E0,1.0296974E0,2.4574196E-1,3.0144185E-1,3.9687693E-1,7.246405E-2,1.6184217E-1,3.6169243E-1,3.6381483E-1,1.8428838E-1,0E0,2.0840406E-1,1.5678816E-1,0E0,0E0,1.3709322E-1,4.7536016E-2,4.814589E-2,3.3391476E-2,9.2256546E-2,1.3363922E-1,0E0,0E0,9.6892875E-1,1.8307989E0,1.9797552E-1,3.313231E-1,4.4535303E-1,2.6510668E-1,6.4427805E-1,4.6380615E-1,0E0,0E0,3.6125183E-1,3.7623674E-2,1.4190233E-1,0E0,1.9391908E-1,7.6968E-1,1.520791E-1,0E0,4.0519333E-1,0E0,6.2922186E-1,3.8575482E-1,6.651634E-2,2.1490097E-1,3.9027452E-2,0E0,1.8694663E-1,0E0,2.4008915E-1,0E0,1.9371599E-2,3.3088982E-2,0E0,7.6513186E-2,9.094548E-2,9.3665764E-2,3.5807276E-1,4.221195E-2,8.1294134E-2,8.5781276E-2,0E0,0E0,8.626431E-2,0E0,6.1145186E-2,1.2486715E-1,0E0,2.2044241E-2,0E0,0E0,0E0,0E0,0E0,2.8686106E-2,1.3593984E-1,1.663683E-1,1.3621198E0,5.694963E-1,9.893422E-1,6.541867E-1,0E0,0E0,2.0479584E-1,0E0,4.946601E-1,1.6213644E-1,1.1243403E-1,7.5526275E-2,6.63299E-1,7.22641E-1,2.6130915E-1,0E0,1.17761135E-1,0E0,0E0,0E0,0E0,0E0,2.4646008E-1,1.14411175E-1,5.697808E-1,4.5170718E-1,0E0,1.2914187E-1,2.959051E-1,4.0624905E-1,2.0998436E-1,3.3644712E-1,1.6525838E-1,1.20049E-1,0E0,0E0,0E0,2.1091747E-1,0E0,0E0,1.4393035E-1,2.2092086E-1,0E0,7.6446915E-3,0E0,0E0,0E0,4.2751506E-2,3.7961975E-2,0E0,5.6779265E-2,5.392438E-2,0E0,6.969018E-2,0E0,1.8062282E-1,0E0,0E0,5.6350965E-2,2.7496487E-2,3.5132557E-2,8.157146E-2,4.9062878E-2,0E0,0E0,0E0,2.6367577E-2,0E0,0E0,0E0,0E0,0E0,1.2288183E-1,4.7509342E-2,0E0,0E0,4.4535518E-1,6.172958E-1,5.911162E-1,1.9275329E-1,1.5180469E-1,3.5414165E-1,3.714664E-1,2.3499873E-1,1.2028146E-1,0E0,4.4037306E-1,3.3982623E-1,1.0882133E-1,0E0,0E0,0E0,1.6802983E-2,5.8190793E-2,7.166002E-1,7.043601E-1,6.100478E-1,3.157659E-1,1.4336765E-1,1.8564796E-1,0E0,0E0,1.05982065E-1,0E0,1.937206E-2,0E0,4.4707537E-1,5.943742E-1,2.6420677E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,93,93,95,95,96,96,97,97,99,99,101,101,102,102,103,103,104,104,105,105,107,107,109,109,111,111,112,112,114,114,115,115,116,116,117,117,118,118,119,119,120,120,123,123,125,125,126,126,128,128,134,134,135,135,136,136,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,148,148,149,149,150,150,151,151,153,153,159,159,160,160,161,161,162,162,164,164,165,165,166,166,167,167,168,168,169,169,170,170,174,174,177,177,178,178,180,180,184,184,185,185,187,187,188,188,190,190,192,192,195,195,196,196,197,197,198,198,199,199,203,203,209,209,210,210,213,213,214,214,215,215,216,216,217,217,218,218,219,219,220,220,221,221,223,223,224,224,225,225,229,229,230,230,231,231,232,232,233,233,234,234,235,235,236,236,239,239,241,241,243,243,244,244,245,245],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,-1,76,78,80,82,84,86,88,90,-1,-1,-1,-1,92,94,96,98,-1,100,102,104,106,108,110,112,114,116,118,120,-1,122,124,-1,-1,126,128,130,132,134,136,-1,-1,138,140,142,144,146,148,150,152,-1,-1,154,156,158,-1,160,162,164,-1,166,-1,168,170,172,174,176,-1,178,-1,180,-1,182,184,-1,186,188,190,192,194,196,198,-1,-1,200,-1,202,204,-1,206,-1,-1,-1,-1,-1,208,210,212,214,216,218,220,-1,-1,222,-1,224,226,228,230,232,234,236,-1,238,-1,-1,-1,-1,-1,240,242,244,246,-1,248,250,252,254,256,258,260,-1,-1,-1,262,-1,-1,264,266,-1,268,-1,-1,-1,270,272,-1,274,276,-1,278,-1,280,-1,-1,282,284,286,288,290,-1,-1,-1,292,-1,-1,-1,-1,-1,294,296,-1,-1,298,300,302,304,306,308,310,312,314,-1,316,318,320,-1,-1,-1,322,324,326,328,330,332,334,336,-1,-1,338,-1,340,-1,342,344,346,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.631E3,1.1458888E4,6.553948E8,4.930349E5,5.473125E3,2.867292E3,3.256118E-2,2.785857E2,1.6503105E3,8.628E3,7.5179994E-1,6.5030734E4,4.147809E0,6.9514094E11,6.737509E7,5.1792985E6,1.6256282E7,2.1924436E5,2.5682794E5,4.9E1,1E0,9.99E2,3.0379269E1,2.613714E6,1.7324902E8,1.3676985E4,4.36E2,2.074E3,2.1907706E12,1.5426969E-2,1.3521131E8,2.0368582E8,1E0,6.860185E2,1.221875E1,1.5190727E5,1.948181E0,1.608996E-3,6.731367E0,8.333333E0,3.983889E3,8.027061E2,1E0,6.1114804E-8,2.9251662E6,1.1896622E6,2.3836442E-3,-4.8016957E-3,7.5529786E-3,3.7786294E-2,5.632258E0,7.3610186E3,1.2204E4,9.866966E0,3.6478207E-2,4.5723195E6,3.653602E7,5.2873194E-1,2.208825E-2,4.091015E9,2.603E3,3.3751E4,1.6621418E0,2.652E3,2.2348747E2,5.670103E-2,9.547842E-3,1.203975E7,1E0,2.9432103E-3,-6.461733E-3,3.0996E4,3.904E3,2.9956698E-1,4.5168175E5,1.2068493E4,8.747313E6,-3.2432225E-2,-8.5677E-3,2.4081633E0,9.213145E0,1.1146532E8,1.8314E4,4.5723195E6,8.594445E5,4.4866666E2,2.0907634E3,-9.7289095E-3,-2.0875657E-2,3.2585382E10,1.6286094E5,3.29E2,-2.4286216E-2,2.0189162E7,2.6307288E7,1.6284274E7,2.3392182E-2,4.01E2,-2.9731015E-2,2.9311974E8,2.3140822E8,3.931011E6,3.5301748E7,3.3E1,-4.585753E-3,5.418831E11,8.224154E-3,2.589778E6,-2.528779E-2,2.076424E6,3.3159972E5,4.3707653E-3,6.803565E-1,1.1347137E2,2.915E4,2.3898147E5,1.1603518E8,2.695E3,4.61E2,-1.9261371E-2,-5.4529086E-3,1.60264E6,2.2340459E-3,2.3432E4,1.5E1,4.606102E-3,4.080551E-1,2.1911228E-2,9.897606E-3,4.7466257E-3,1.0647314E-2,-1.7395418E-2,1.64E1,7.3E1,4.0677965E-2,5.52E2,5.8180006E8,1.1032986E8,2.342147E7,-2.8418656E-2,-9.509639E-3,3.8527173E-1,6.2996903E-3,1.741674E2,3.001419E2,1.2469E5,1E0,3.9552E2,8.185918E0,6.088569E6,3.1018658E-2,3.2167475E6,2.666815E-2,4.3576537E-3,1.1388264E-2,5.0916392E-3,-6.826737E-3,1.8203359E9,5.936149E1,8.015419E3,1.0604997E0,1.8938474E-2,1.612289E6,1.5E1,2.13019E6,3.3018634E5,1.0508E5,6.5029144E5,1.0721749E3,-1.712028E-2,-6.3253846E-3,-1.6292363E-2,2.6120481E0,-9.983009E-3,-1.9925445E-2,8.3510876E9,2.6281825E-1,-1.603309E-2,2.6304092E0,-4.5148027E-3,1.2101669E-4,-9.958643E-3,3.37E2,1.2738854E0,-9.923373E-3,1.9521575E5,1.8108038E5,5.350551E-3,3.906E3,-6.9887848E-3,2.862988E5,-9.388653E-3,-3.0556584E-3,1.0242964E8,3.958068E2,1.2867613E7,9.2593566E-2,1.990351E1,3.6823028E-4,1.9615034E-3,8.940468E-3,2.1252015E6,-6.293727E-3,1.2927048E-2,5.5073393E-3,-1.1169207E-2,-4.6224687E-3,3.340245E0,8.2E1,-1.2996436E-2,-1.0590161E-3,4.0233E4,1.5E1,3.1019E5,2.9536E4,1E0,1E0,4.7013435E1,6.747114E7,1.6347875E3,2.444566E-4,2E0,2.86968E5,7E2,1.7984563E-2,-3.934162E-3,-1.4850727E-2,3.3125E0,1.5139E4,5.3038636E7,7E0,1.4523809E0,8.2089216E-4,3.0052083E0,1.8344E4,1.7621096E-2,5.4791966E-3,4.8530518E8,1.3711299E-2,1.0872E4,3.0403594E-3,4.20737E5,3.3517068E7,1.6956E5,9.650936E-3,1.3130375E-2,2.7578464E-3,-1.2557547E-2,-1.5033922E-3,-1.7660756E-2,-2.4952895E-3,-9.817325E-4,7.6108207E-3,1.0551626E-3,-8.492023E-3,2.1448417E-3,-9.8079955E-3,-2.1694276E-2,-1.3385105E-2,-3.8736764E-2,-1.9451948E-2,-3.1958104E-3,5.8857263E-3,-8.1542E-3,3.5662137E-3,-1.4221246E-4,-3.1103739E-3,-7.42448E-3,-8.5658825E-4,1.3479452E-3,-3.7606212E-3,-1.4908224E-2,-5.634662E-3,-2.006634E-3,-8.193386E-3,-5.1860646E-3,2.3131547E-3,-2.5994042E-2,-1.23410355E-2,1.8657194E-3,-4.174123E-3,8.84956E-3,2.7317093E-3,-2.0806517E-3,3.628341E-3,1.117496E-3,-5.7864436E-3,-2.8176892E-3,-9.497265E-3,4.1022976E-3,-5.5142416E-4,1.1911393E-3,-7.826919E-3,2.3000222E-3,-2.9271E-3,-2.8270802E-3,-1.5388006E-2,1.4738891E-2,3.110572E-3,-5.842198E-3,-1.0781355E-3,6.2685516E-3,-3.5672043E-3,9.516608E-3,2.447397E-2,-5.3861947E-3,8.861459E-3,8.175273E-3,7.1466767E-4,-2.959914E-3,-1.46489E-2,-9.986829E-3,-6.5481476E-4,1.4535167E-3,-1.2000221E-2,-5.6865844E-3,4.4454983E-3,8.941213E-3,1.106581E-3,4.321555E-3,-1.8694447E-5,6.9242966E-4,-5.9054173E-3,-3.2094414E-3,1.0223282E-2,-1.8565994E-2,2.2468988E-3,3.4170857E-4,1.1648847E-2,-7.7295373E-3,4.699799E-3,1.2200157E-2,4.6974234E-3,5.8409628E-3,1.7574228E-2,6.57378E-3,-6.8248255E-4,-1.9827487E-3,-7.3749525E-3,1.5890066E-2,7.956431E-3,2.7022265E-2,1.1972387E-2,1.0258117E-3,-1.5028168E-2],"split_indices":[20,102,2,52,12,28,32,4,0,52,52,9,27,28,35,31,7,45,45,28,28,3,6,2,56,9,7,33,3,2,31,0,45,7,100,52,58,33,42,0,54,52,28,52,16,37,28,28,0,0,0,0,54,4,2,35,0,28,45,27,38,7,0,1,58,10,52,57,0,12,89,0,0,9,2,38,28,28,45,0,0,53,53,31,6,28,28,4,4,0,0,31,28,0,0,45,43,43,0,0,0,7,7,1,45,8,0,31,0,29,0,45,28,0,57,52,9,33,12,2,0,0,0,1,0,9,3,0,38,0,0,0,0,0,4,2,57,2,7,7,45,0,0,41,0,4,4,9,8,4,54,45,0,43,0,0,0,0,0,5,56,4,34,0,32,8,9,33,29,28,4,0,0,0,53,0,0,5,38,0,53,0,0,0,0,53,0,33,28,0,0,0,28,0,0,32,52,12,38,58,0,0,0,32,0,0,0,0,0,54,29,0,0,1,10,47,11,15,16,56,45,4,0,8,7,2,0,0,0,54,9,7,3,54,41,54,9,0,0,12,0,2,0,29,51,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.554E3,8.23E2,1.731E3,4.51E2,3.72E2,1.397E3,3.34E2,4.47E2,4E0,2.77E2,9.5E1,8.07E2,5.9E2,8.1E1,2.53E2,3.67E2,8E1,1.62E2,1.15E2,6.1E1,3.4E1,1.03E2,7.04E2,5.71E2,1.9E1,1.1E1,7E1,2.07E2,4.6E1,3.04E2,6.3E1,4E0,7.6E1,6.5E1,9.7E1,7.3E1,4.2E1,1.3E1,4.8E1,4E0,3E1,9.3E1,1E1,6.48E2,5.6E1,2.2E2,3.51E2,1.5E1,4E0,4E0,7E0,1.3E1,5.7E1,1.2E1,1.95E2,4.2E1,4E0,1.76E2,1.28E2,4.2E1,2.1E1,6E1,1.6E1,3.4E1,3.1E1,5.9E1,3.8E1,6.3E1,1E1,1.5E1,2.7E1,5E0,8E0,3E1,1.8E1,1.1E1,1.9E1,2.1E1,7.2E1,4E0,6E0,4.95E2,1.53E2,8E0,4.8E1,1.88E2,3.2E1,2.53E2,9.8E1,1.1E1,4E0,4.5E1,1.2E1,8E0,4E0,4E1,1.55E2,2.8E1,1.4E1,1.69E2,7E0,7.5E1,5.3E1,1E1,3.2E1,1.5E1,6E0,5.6E1,4E0,1.2E1,4E0,8E0,2.6E1,6E0,2.5E1,4.3E1,1.6E1,2.5E1,1.3E1,2.9E1,3.4E1,1E1,5E0,2E1,7E0,1.3E1,1.7E1,8E0,1E1,7E0,4E0,6E0,1.3E1,9E0,1.2E1,5.6E1,1.6E1,1.88E2,3.07E2,3.9E1,1.14E2,4E0,4E0,4.4E1,4E0,1.58E2,3E1,1.1E1,2.1E1,6.6E1,1.87E2,9.4E1,4E0,1.3E1,3.2E1,5E0,7E0,4E0,4E0,2.5E1,1.5E1,1.4E2,1.5E1,1.3E1,1.5E1,7E1,9.9E1,2.7E1,4.8E1,1E1,4.3E1,6E0,4E0,4E0,2.8E1,5E0,1E1,2E1,3.6E1,4E0,8E0,4E0,4E0,1.5E1,1.1E1,2.1E1,4E0,1.4E1,2.9E1,4E0,1.2E1,7E0,1.8E1,7E0,6E0,2E1,9E0,9E0,2.5E1,1.6E1,4E0,7E0,6E0,1.1E1,6E0,6E0,4E0,7E0,5E0,3.5E1,2.1E1,1.2E1,4E0,7.4E1,1.14E2,2.83E2,2.4E1,1.7E1,2.2E1,7.2E1,4.2E1,3.7E1,7E0,2.4E1,1.34E2,2.6E1,4E0,5E0,6E0,8E0,1.3E1,4.9E1,1.7E1,1.63E2,2.4E1,4.3E1,5.1E1,9E0,4E0,1.9E1,6E0,8E0,7E0,1.07E2,3.3E1,9E0,6E0,1E1,5E0,6.3E1,7E0,9.4E1,5E0,1.4E1,1.3E1,1.2E1,3.6E1,5E0,5E0,9E0,3.4E1,2.3E1,5E0,1.5E1,5E0,3.2E1,4E0,4E0,4E0,7E0,4E0,4E0,1.7E1,1E1,4E0,5E0,2.4E1,8E0,4E0,1.1E1,7E0,1.6E1,4E0,4E0,5E0,4E0,5E0,5E0,2E1,5E0,1.1E1,6E0,5E0,4E0,3.1E1,5E0,1.6E1,6.7E1,7E0,1.2E1,1.02E2,1.81E2,1.02E2,1.8E1,6E0,4E0,1.3E1,5E0,1.7E1,2.5E1,4.7E1,3.8E1,4E0,3.3E1,4E0,1.5E1,9E0,8E0,1.26E2,2E1,6E0,4E0,4E0,5E0,8E0,1.3E1,3.6E1,5E0,1.2E1,1.3E1,1.5E2,6E0,1.8E1,3.3E1,1E1,5E0,4.6E1,8E0,1.1E1,4E0,4E0,2.4E1,8.3E1,1.1E1,2.2E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"347","size_leaf_vector":"1"}},{"base_weights":[-2.5953604E-3,-1.539845E-1,7.0062496E-2,-2.3088704E-1,-5.8624864E-2,2.1977832E-3,1.8017352E-1,-2.4173796E-1,4.0543135E-2,-1.0907182E-1,1.1216866E-1,-1.1863875E-1,3.4853723E-2,1.4897819E-1,3.75307E-1,-2.796011E-1,-7.426639E-2,-1.403812E-1,3.6828658E-3,-9.692767E-2,1.541725E-1,-1.0785211E-1,-2.2328252E-2,5.1871132E-2,-1.8638602E-1,1.1637926E-1,2.9723364E-1,4.1061693E-1,1.086317E-1,-2.4062175E-1,-4.3975586E-1,-4.4937655E-2,-2.2876069E-2,-1.5099311E-1,4.592209E-2,-7.559483E-2,4.1436244E-2,1.4278571E-3,-1.8377204E-1,1.11129336E-1,2.7712813E-1,-1.9134948E-1,-7.399072E-2,2.8467506E-2,2.0284298E-1,-2.7433228E-1,-5.805896E-3,1.4087372E-1,-1.5981224E-1,3.6184958E-1,1.8191907E-1,1.9025087E-1,4.6706095E-1,1.1362789E-2,-4.344528E-4,-3.09757E-1,-1.3969511E-1,-4.9415377E-1,-2.001002E-1,1.0875488E-2,-6.507736E-2,-1.3808288E-1,-3.0293402E-1,7.2427117E-3,-1.4196358E-2,-1.0262648E-1,1.5705407E-3,1.0420808E-1,-1.0438677E-2,-4.463498E-3,-1.206496E-2,1.7804E-1,7.0869826E-2,3.1860524E-1,7.048949E-3,-1.5712577E-1,-2.4954355E-1,-1.876105E-2,-1.2315182E-1,-2.5401274E-2,9.2743725E-2,5.164941E-1,1.4949939E-1,-2.0782807E-1,-1.7376602E-2,9.050448E-3,-9.265546E-2,5.8292728E-2,1.9312145E-1,-1.990841E-1,-6.7856014E-3,3.8706997E-1,4.875986E-3,3.0960828E-1,7.963055E-2,2.6871045E-3,1.2691826E-2,2.3766913E-1,5.008916E-1,-2.6456635E-2,-2.9638088E-1,8.63935E-2,-1.7608306E-1,-5.899674E-3,-5.274054E-1,-1.3818145E-2,-1.3422194E-1,-2.1785732E-1,-3.5870638E-2,-2.674619E-1,-1.2381758E-1,-8.604018E-3,-1.9261502E-2,-3.4034015E-3,2.126581E-3,-6.807636E-2,-8.590844E-3,2.9534366E-4,1.2841298E-1,-3.481172E-3,2.7233245E-2,1.29802E-3,2.1443196E-1,-4.938271E-2,1.1278299E-1,1.872352E-2,7.688734E-3,-1.8524635E-1,-5.847989E-2,-1.5586314E-1,-1.4587376E-2,-5.423009E-2,1.00511365E-1,-1.00030914E-1,-1.347095E-2,-5.8928628E-2,7.6535456E-2,4.6674654E-2,1.6165821E-1,6.1861053E-3,3.2380167E-2,2.3715865E-2,2.2282566E-1,-1.1583916E-2,-3.5098204E-3,-8.323273E-3,7.8059814E-4,8.962602E-2,-1.5656883E-1,2.2841835E-1,5.667913E-2,-2.322138E-1,-1.1163283E-3,2.1112706E-3,-2.7215676E-3,2.1968374E-1,4.3426383E-1,3.824606E-1,2.9115898E-3,1.2569454E-2,1.3368187E-2,6.3044094E-3,1.6185777E-2,2.7173204E-2,3.931784E-1,1.4675393E-3,-3.0435917E-1,1.0880271E-2,1.9121498E-2,-4.909454E-2,-2.4714768E-1,-3.7766758E-1,-6.5760964E-1,-3.4189248E-3,-8.328601E-3,-1.5220262E-2,-4.014235E-3,-5.970772E-2,1.0120409E-1,-1.9284374E-1,-1.837922E-2,-2.0928143E-1,-1.0758012E-1,-1.4874223E-3,-5.386792E-3,9.132042E-2,9.581233E-3,-1.8636227E-3,6.968132E-2,5.9076613E-3,1.2362294E-2,6.266913E-4,-5.6870086E-3,-3.5837822E-4,1.5512598E-1,-2.8440112E-3,-2.0612931E-1,-4.7600814E-3,-5.97001E-4,-1.020776E-2,-3.81057E-3,1.9107935E-3,-1.18459344E-1,1.581935E-1,2.5252186E-2,-1.6240722E-1,-4.139147E-2,1.567653E-2,-6.816685E-2,2.223381E-1,4.3351293E-2,-1.7717771E-1,6.559529E-2,2.0238827E-1,-9.170321E-2,9.206402E-2,-8.538238E-2,2.6655373E-1,6.6355035E-2,1.730999E-1,4.1738357E-2,-3.0150995E-2,-2.3744628E-2,2.464327E-1,2.1573732E-2,2.2205578E-2,1.4942648E-1,-1.746226E-2,-1.7061754E-1,2.7132127E-3,1.6184496E-2,3.5055026E-1,3.3035293E-2,8.903719E-3,2.2657229E-2,-4.519387E-2,7.5354315E-2,1.1537263E-2,2.250074E-2,-2.8101476E-2,-1.4468082E-2,3.202526E-3,-1.4281418E-3,-4.2961612E-3,6.849409E-3,-1.3538003E-2,-2.9192076E-3,-8.376139E-3,-2.1470398E-2,-3.6109425E-2,-1.5462312E-2,-5.676848E-3,-7.8590517E-4,1.2860147E-2,-2.288737E-3,-1.1266847E-2,-4.7726477E-3,-4.7443127E-3,-1.1799476E-2,-6.6916887E-3,1.0551529E-4,9.187786E-4,6.270229E-3,5.302407E-3,6.0059835E-4,8.74218E-3,3.0295397E-3,-1.1391064E-2,-4.399027E-3,-2.5899159E-3,1.3756427E-3,-9.967677E-3,-4.0948144E-3,1.0506975E-2,4.4181896E-3,4.1695414E-3,-1.8983601E-3,-9.230445E-3,6.4722664E-4,1.1668585E-3,-5.0507104E-3,-6.984278E-3,-2.0849204E-3,4.001903E-3,1.3737498E-2,8.248408E-3,9.163369E-4,4.2415173E-3,-2.0840855E-2,5.5550416E-3,9.4079835E-5,1.3537739E-2,5.8688284E-3,-1.4075583E-2,4.420237E-3,2.5456015E-3,9.654615E-3,-7.4255452E-3,-9.6862554E-4,1.1434444E-2,1.9585734E-2,1.0285461E-2,-1.699468E-3,1.0046839E-2,1.4950425E-3,-3.589521E-3,3.5324802E-3,-7.400262E-3,1.7727428E-3,-1.9255962E-3,1.2856729E-2,-7.3497216E-3,5.904289E-3,-2.166381E-4,7.771371E-3,3.7978166E-3,9.490048E-3,-1.0214242E-2,-4.925963E-3,2.484829E-2,1.5141931E-2,-4.3637347E-3,7.201076E-4,1.6439473E-3,5.1334198E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,-1,-1,99,101,103,105,-1,107,109,111,-1,113,115,-1,117,119,-1,-1,121,123,125,-1,127,129,131,133,135,137,139,141,143,-1,-1,145,147,149,151,153,155,-1,157,159,-1,-1,161,163,-1,165,167,169,-1,171,-1,173,175,177,179,181,-1,-1,-1,-1,183,-1,-1,185,-1,187,-1,189,191,193,-1,-1,195,197,199,-1,201,203,205,-1,207,209,211,213,-1,-1,215,217,-1,-1,-1,-1,219,221,223,225,227,-1,-1,-1,229,231,233,-1,235,-1,-1,-1,-1,237,-1,239,-1,241,243,245,247,249,-1,-1,-1,-1,251,253,255,-1,257,259,-1,-1,261,-1,-1,263,-1,-1,-1,-1,-1,265,-1,267,-1,-1,-1,-1,269,271,273,275,277,279,-1,281,283,285,287,289,291,293,295,297,299,301,303,305,-1,307,309,311,313,315,-1,317,-1,-1,319,-1,-1,-1,321,323,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8389784E1,6.1245193E0,1.3031911E1,5.432585E0,3.2432394E0,4.265848E0,3.9921112E0,2.8897533E0,0E0,1.0242975E0,7.7026856E-1,7.6610565E-1,3.2136211E0,2.7443657E0,8.122759E-1,2.2289658E0,9.4686925E-1,4.5572042E-1,1.946844E-1,1.6874108E-1,3.5025573E-1,6.191652E-1,0E0,2.7890565E0,9.671693E-1,3.2224345E0,7.0164204E-1,9.06209E-1,1.6920377E-1,2.070938E0,8.3976746E-1,4.4253576E-1,0E0,3.7991285E-1,7.9452716E-2,6.398095E-2,1.4436431E-1,0E0,2.746442E-2,1.3554078E-1,5.6630373E-2,8.825374E-2,4.34829E-1,2.3776178E0,1.6962996E0,1.2794876E-1,3.5772347E-1,1.8656244E0,2.3302507E-1,3.9544392E-1,4.8473024E-1,1.42955E-1,3.6986446E-1,0E0,0E0,3.6917305E-1,1.0435755E0,6.6303635E-1,3.8424253E-2,0E0,3.313127E-1,3.360095E-1,1.0630715E-1,0E0,3.0440137E-2,2.9001981E-2,0E0,4.6826467E-2,5.7681177E-2,0E0,0E0,1.0510647E-1,1.8701544E-1,6.250727E-2,0E0,1.0844791E-1,4.1879177E-2,3.3119756E-1,2.6084054E-1,1.2827183E0,9.861753E-1,6.9459295E-1,8.432534E-1,6.2980056E-2,0E0,0E0,1.22662485E-1,1.1585273E0,1.2681065E0,1.7633033E-1,2.3346057E-2,3.70677E-1,0E0,2.8899252E-1,2.864729E-1,0E0,0E0,5.530858E-2,3.591156E-2,0E0,4.533577E-1,1.5489605E-1,9.6250033E-1,0E0,7.84956E-1,0E0,9.953603E-3,1.1868733E-1,2.1842766E-1,7.136226E-2,2.3004961E-1,0E0,0E0,0E0,0E0,1.6577166E-2,0E0,0E0,2.0079881E-2,0E0,4.64261E-2,0E0,2.150911E-2,4.132778E-2,1.3825288E-1,0E0,0E0,7.670319E-2,1.7690342E-2,1.9114673E-2,0E0,2.1657959E-1,7.414527E-2,2.6760608E-1,0E0,9.9225456E-1,4.4032693E-1,8.1211805E-1,1.3084879E0,0E0,0E0,2.688439E-1,3.7915826E-1,0E0,0E0,0E0,0E0,5.883832E-1,1.2914366E0,7.809067E-1,1.7573242E-1,1.2817204E-1,0E0,0E0,0E0,2.4096072E-1,6.2292767E-1,1.077013E-1,0E0,6.868547E-2,0E0,0E0,0E0,0E0,7.219219E-2,0E0,4.75132E-1,0E0,2.9720867E-2,2.8647494E-1,2.94003E-1,2.2300696E-1,3.879528E-1,0E0,0E0,0E0,0E0,1.309719E-1,2.4137425E-1,1.9764572E-2,0E0,6.358242E-2,4.4817162E-1,0E0,0E0,2.7595915E-2,0E0,0E0,1.9734837E-2,0E0,0E0,0E0,0E0,0E0,2.6876539E-2,0E0,5.2157044E-2,0E0,0E0,0E0,0E0,4.686588E-2,6.6480786E-2,1.7747432E-2,3.6294665E-2,1.4332885E-1,1.5694413E-1,0E0,5.1676357E-1,1.0516536E-1,2.314125E-1,9.7851413E-1,5.079891E-1,5.969472E-1,6.410788E-1,8.1662536E-2,5.5538714E-2,1.23147726E-1,2.0300606E-1,2.1024561E-1,3.2082236E-1,0E0,1.3782366E-1,6.1104393E-1,3.1063008E-1,1.4929652E-1,3.0167699E-2,0E0,2.6111364E-2,0E0,0E0,9.3315125E-2,0E0,0E0,0E0,2.6375864E-2,7.0822276E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,60,60,61,61,62,62,64,64,65,65,67,67,68,68,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,97,97,98,98,100,100,101,101,102,102,104,104,106,106,107,107,108,108,109,109,110,110,115,115,118,118,120,120,122,122,123,123,124,124,127,127,128,128,129,129,131,131,132,132,133,133,135,135,136,136,137,137,138,138,141,141,142,142,147,147,148,148,149,149,150,150,151,151,155,155,156,156,157,157,159,159,164,164,166,166,168,168,169,169,170,170,171,171,172,172,177,177,178,178,179,179,181,181,182,182,185,185,188,188,194,194,196,196,201,201,202,202,203,203,204,204,205,205,206,206,208,208,209,209,210,210,211,211,212,212,213,213,214,214,215,215,216,216,217,217,218,218,219,219,220,220,222,222,223,223,224,224,225,225,226,226,228,228,231,231,235,235,236,236],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,-1,-1,100,102,104,106,-1,108,110,112,-1,114,116,-1,118,120,-1,-1,122,124,126,-1,128,130,132,134,136,138,140,142,144,-1,-1,146,148,150,152,154,156,-1,158,160,-1,-1,162,164,-1,166,168,170,-1,172,-1,174,176,178,180,182,-1,-1,-1,-1,184,-1,-1,186,-1,188,-1,190,192,194,-1,-1,196,198,200,-1,202,204,206,-1,208,210,212,214,-1,-1,216,218,-1,-1,-1,-1,220,222,224,226,228,-1,-1,-1,230,232,234,-1,236,-1,-1,-1,-1,238,-1,240,-1,242,244,246,248,250,-1,-1,-1,-1,252,254,256,-1,258,260,-1,-1,262,-1,-1,264,-1,-1,-1,-1,-1,266,-1,268,-1,-1,-1,-1,270,272,274,276,278,280,-1,282,284,286,288,290,292,294,296,298,300,302,304,306,-1,308,310,312,314,316,-1,318,-1,-1,320,-1,-1,-1,322,324,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.0950326E6,2.14099E5,6.860185E2,4.54E2,3.2196458E6,2.867292E3,4.0543135E-2,1.294364E0,1.5896305E-3,1.5126086E2,4.0844156E7,3.970405E3,2.3487206E7,6.9514094E11,2.1196339E1,8.962E3,1.752512E2,4.57091E5,1.91E4,3.79E2,-2.2328252E-2,1.0989723E3,3.1998687E3,4.2343444E7,2.8091298E7,1.48298E5,7.131107E6,2.074E3,4.099E3,2.1853803E4,-2.2876069E-2,1E0,1.0632E4,2.04946E6,1.1601851E1,1.4278571E-3,8.317E3,3.0597075E8,1.7366706E8,6E0,8.836364E0,3.830216E5,6.7388856E7,7.526755E7,6.0559247E2,5.862126E2,2.3898147E5,2.3821254E7,1.0317661E5,5.6553E4,1E0,1.1362789E-2,-4.344528E-4,1.0203835E-5,1.0119178E10,1.0198864E0,6.3053357E2,1.0875488E-2,1.0330753E5,2.750917E-4,3.1254683E0,7.2427117E-3,3.582157E12,2E1,1.5705407E-3,1.8991614E5,1.7227725E0,-4.463498E-3,-1.206496E-2,1.5938586E8,1.0323588E3,2.4848485E0,7.048949E-3,4.0814578E2,6.7E1,3.6E2,1.9E1,9.750871E6,1.06E3,3.097E3,1.819795E6,4.423676E1,-1.7376602E-2,9.050448E-3,2.6E1,5.024605E0,8.638343E0,1.9589581E6,6.1238285E8,9.391714E4,4.875986E-3,2.7487153E5,1.4722673E0,2.6871045E-3,1.2691826E-2,1.7812634E1,1.3058E4,-2.6456635E-2,1.0155283E-5,2.4E1,2.34768E5,-5.899674E-3,8.8070023E-1,-1.3818145E-2,4.4444447E0,6.08767E3,5.57E4,9.374544E5,1.04E3,-8.604018E-3,-1.9261502E-2,-3.4034015E-3,2.126581E-3,1.227133E6,-8.590844E-3,2.9534366E-4,1.3046789E6,-3.481172E-3,2.4684112E8,1.29802E-3,1.2511433E3,1.6052323E5,1.4008022E11,1.872352E-2,7.688734E-3,8E0,4.1E1,1.64E1,-1.4587376E-2,3.4402E4,1.2944685E0,1.463E2,-1.347095E-2,8E0,5.6666665E0,1.62848E5,5.9572783E0,6.1861053E-3,3.2380167E-2,8.454878E1,1.8071064E7,-1.1583916E-2,-3.5098204E-3,-8.323273E-3,7.8059814E-4,1E1,1.363E3,2.7195522E7,1.8777761E0,1.3933473E5,-1.1163283E-3,2.1112706E-3,-2.7215676E-3,6.087505E8,2.1196339E1,1.784E3,2.9115898E-3,6.7652373E3,1.3368187E-2,6.3044094E-3,1.6185777E-2,2.7173204E-2,8.538E3,1.4675393E-3,2.35E2,1.0880271E-2,1.0882861E6,3.3018634E5,3.65625E1,8.16747E11,9.075E3,-3.4189248E-3,-8.328601E-3,-1.5220262E-2,-4.014235E-3,3.3368218E6,2.560372E7,2.49855E5,-1.837922E-2,3.312E4,5.828063E5,-1.4874223E-3,-5.386792E-3,6.678896E2,9.581233E-3,-1.8636227E-3,6E0,5.9076613E-3,1.2362294E-2,6.266913E-4,-5.6870086E-3,-3.5837822E-4,3.8879236E7,-2.8440112E-3,4.4312353E0,-4.7600814E-3,-5.97001E-4,-1.020776E-2,-3.81057E-3,4.974E3,1.76421E6,5.8084745E6,5.9E1,2.4E1,1E0,1.567653E-2,4.7019145E4,4.9375E1,1.9044118E0,8E0,1E0,9.135018E0,1.945345E6,3.2627738E5,6.222E3,6.4683E4,1.3071896E-3,1.3432479E7,1.9406E4,-3.0150995E-2,2.4559366E1,1.2309059E0,1.1736916E0,3.78E2,1.7801905E5,-1.746226E-2,6.1423077E1,2.7132127E-3,1.6184496E-2,1.505516E7,3.3035293E-2,8.903719E-3,2.2657229E-2,4.091015E9,2.711499E6,1.1537263E-2,2.250074E-2,-2.8101476E-2,-1.4468082E-2,3.202526E-3,-1.4281418E-3,-4.2961612E-3,6.849409E-3,-1.3538003E-2,-2.9192076E-3,-8.376139E-3,-2.1470398E-2,-3.6109425E-2,-1.5462312E-2,-5.676848E-3,-7.8590517E-4,1.2860147E-2,-2.288737E-3,-1.1266847E-2,-4.7726477E-3,-4.7443127E-3,-1.1799476E-2,-6.6916887E-3,1.0551529E-4,9.187786E-4,6.270229E-3,5.302407E-3,6.0059835E-4,8.74218E-3,3.0295397E-3,-1.1391064E-2,-4.399027E-3,-2.5899159E-3,1.3756427E-3,-9.967677E-3,-4.0948144E-3,1.0506975E-2,4.4181896E-3,4.1695414E-3,-1.8983601E-3,-9.230445E-3,6.4722664E-4,1.1668585E-3,-5.0507104E-3,-6.984278E-3,-2.0849204E-3,4.001903E-3,1.3737498E-2,8.248408E-3,9.163369E-4,4.2415173E-3,-2.0840855E-2,5.5550416E-3,9.4079835E-5,1.3537739E-2,5.8688284E-3,-1.4075583E-2,4.420237E-3,2.5456015E-3,9.654615E-3,-7.4255452E-3,-9.6862554E-4,1.1434444E-2,1.9585734E-2,1.0285461E-2,-1.699468E-3,1.0046839E-2,1.4950425E-3,-3.589521E-3,3.5324802E-3,-7.400262E-3,1.7727428E-3,-1.9255962E-3,1.2856729E-2,-7.3497216E-3,5.904289E-3,-2.166381E-4,7.771371E-3,3.7978166E-3,9.490048E-3,-1.0214242E-2,-4.925963E-3,2.484829E-2,1.5141931E-2,-4.3637347E-3,7.201076E-4,1.6439473E-3,5.1334198E-3],"split_indices":[20,102,47,2,52,2,28,4,0,42,38,46,45,52,50,31,54,0,52,12,2,1,0,52,4,45,45,1,45,2,0,33,0,77,0,9,58,0,9,7,32,10,58,28,7,45,33,52,33,9,28,1,102,0,0,42,19,56,4,0,33,39,53,0,31,3,0,28,39,0,0,7,52,57,0,33,29,2,8,9,2,2,1,56,0,0,8,53,53,32,7,28,0,33,42,0,0,56,29,0,39,8,29,0,27,0,56,4,2,28,2,0,0,0,0,45,0,0,33,0,7,0,4,33,31,0,0,3,2,4,0,9,53,55,0,18,58,7,53,0,0,56,45,0,0,0,0,3,2,51,41,33,0,0,0,32,54,0,0,52,0,0,0,0,2,0,2,0,33,33,56,31,2,0,0,0,0,32,1,29,0,12,28,0,0,4,0,0,8,0,0,0,0,0,45,0,54,0,0,0,0,9,31,45,0,3,16,0,28,47,53,8,16,58,12,28,2,2,57,9,1,0,56,54,41,8,33,0,58,0,0,50,0,0,0,7,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.579E3,8.36E2,1.743E3,4.62E2,3.74E2,1.079E3,6.64E2,4.58E2,4E0,2.89E2,8.5E1,2.29E2,8.5E2,5.74E2,9E1,3.73E2,8.5E1,2.26E2,6.3E1,1.4E1,7.1E1,2.23E2,6E0,7.9E2,6E1,4.72E2,1.02E2,7.9E1,1.1E1,3.02E2,7.1E1,8E1,5E0,2.14E2,1.2E1,2E1,4.3E1,6E0,8E0,5.4E1,1.7E1,6.3E1,1.6E2,6.85E2,1.05E2,4E1,2E1,4.34E2,3.8E1,6.4E1,3.8E1,1.7E1,6.2E1,5E0,6E0,1.78E2,1.24E2,5.7E1,1.4E1,5E0,7.5E1,1.99E2,1.5E1,4E0,8E0,1.6E1,4E0,1.9E1,2.4E1,4E0,4E0,1.9E1,3.5E1,1.2E1,5E0,4.2E1,2.1E1,7.6E1,8.4E1,3.73E2,3.12E2,1.4E1,9.1E1,2.3E1,1.7E1,6E0,1.4E1,1.69E2,2.65E2,3E1,8E0,5.8E1,6E0,1.6E1,2.2E1,6E0,1.1E1,9E0,5.3E1,8E0,1.7E2,1.7E1,1.07E2,5E0,5.2E1,5E0,9E0,1.1E1,6.4E1,1.8E1,1.81E2,7E0,8E0,4E0,4E0,1.2E1,4E0,4E0,1.5E1,9E0,1.5E1,4E0,1.5E1,9E0,2.6E1,8E0,4E0,3.2E1,1E1,8E0,1.3E1,5.9E1,1.7E1,7.4E1,1E1,2.81E2,9.2E1,1.88E2,1.24E2,4E0,1E1,3.4E1,5.7E1,1.9E1,4E0,8E0,6E0,1.48E2,2.1E1,2.1E2,5.5E1,2.5E1,5E0,4E0,4E0,1.4E1,4.4E1,1.2E1,4E0,1.7E1,5E0,5E0,4E0,3.5E1,1.8E1,4E0,1.66E2,5E0,1.2E1,3.9E1,6.8E1,2.6E1,2.6E1,4E0,5E0,6E0,5E0,5.5E1,9E0,1.2E1,6E0,2.7E1,1.54E2,7E0,5E0,1.1E1,4E0,6E0,9E0,5E0,1E1,5E0,4E0,7E0,1.9E1,5E0,2.7E1,5E0,5E0,4E0,4E0,3.2E1,2.7E1,9E0,8E0,3.5E1,3.9E1,6E0,2.75E2,1.6E1,7.6E1,1.4E1,1.74E2,1.07E2,1.7E1,2.1E1,1.3E1,4.4E1,1.3E1,5.3E1,9.5E1,4E0,1.7E1,1.93E2,1.7E1,4.1E1,1.4E1,7E0,1.8E1,6E0,8E0,3.4E1,1E1,4E0,8E0,9E0,8E0,6E0,1.2E1,7E0,1.59E2,6E0,6E0,3.3E1,6E0,6E1,8E0,6E0,2E1,2.1E1,5E0,2.4E1,3.1E1,4E0,5E0,8E0,4E0,6E0,2.1E1,1.24E2,3E1,4E0,7E0,5E0,4E0,1.5E1,4E0,2.2E1,5E0,1E1,2.2E1,7E0,2E1,4E0,5E0,4E0,4E0,3.1E1,4E0,1.9E1,2E1,7.3E1,2.02E2,5E0,1.1E1,1.2E1,6.4E1,7E0,7E0,1.01E2,7.3E1,5.8E1,4.9E1,8E0,9E0,1.6E1,5E0,6E0,7E0,3.6E1,8E0,5E0,8E0,4.4E1,9E0,1.9E1,7.6E1,5E0,1.2E1,7E0,1.86E2,6E0,1.1E1,3.5E1,6E0,6E0,8E0,1.1E1,7E0,6E0,2.8E1,5E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"325","size_leaf_vector":"1"}},{"base_weights":[2.318822E-3,5.632393E-2,-1.9190912E-1,-3.5278086E-2,1.2962084E-1,-2.8825212E-1,-5.1430028E-2,-7.483353E-2,8.663468E-2,5.2466225E-2,2.2280884E-1,-2.6312926E-1,-5.132009E-1,-1.19964965E-1,6.233034E-2,-1.0908528E-1,3.0866275E-2,1.03925E-1,-8.9735754E-2,6.3186187E-1,3.874521E-2,1.5540078E-1,2.9633567E-1,-2.8455982E-1,-6.607249E-3,-7.072456E-1,-2.1319734E-1,-9.993174E-2,-3.7441045E-1,-9.062245E-2,1.417383E-1,-8.775388E-2,-2.5802794E-1,7.731976E-2,-9.863809E-2,5.3402465E-2,1.6835168E-1,5.250112E-2,-2.6329765E-2,1.6529249E-2,3.809255E-2,7.272272E-2,-1.2816183E-1,2.3715268E-1,8.0176756E-2,-2.0990105E-2,3.103176E-1,-2.5292087E-2,-2.6519287E-1,9.564343E-3,-8.324863E-2,-2.0961251E-2,-3.897628E-2,-3.2201555E-2,-2.2442933E-2,-1.8675266E-1,-8.008052E-2,-2.5573578E-2,-9.868757E-3,2.3198633E-2,-1.4050761E-1,8.549304E-2,2.5885338E-1,-1.2888247E-1,-4.2755794E-2,-5.913037E-1,-1.8398906E-1,1.2987198E-1,1.5113335E-2,-1.7582762E-1,-2.0516E-2,8.155004E-2,-1.223391E-1,6.861579E-2,2.0973723E-1,-4.620106E-3,1.0852806E-1,5.5092696E-2,3.5411918E-1,-2.7478394E-1,-2.7511545E-2,2.5411093E-1,-7.2133923E-3,-2.2313742E-2,1.2037119E-1,2.9199117E-1,6.5696573E-1,-3.0681798E-1,-2.1802758E-1,2.456759E-3,-1.24593936E-1,1.7712706E-3,-4.667478E-3,-9.7593665E-2,-2.3374182E-1,-5.5703286E-2,-1.4624783E-1,-2.0996104E-3,3.6816057E-3,-2.490359E-1,-1.9205898E-2,1.0912526E-1,-5.084484E-3,3.1705448E-1,7.318627E-3,-1.11462794E-1,-2.0285589E-1,3.0717407E-3,-1.0473227E-1,-1.5253458E-2,-3.5533275E-2,-2.5349313E-1,-8.231014E-2,5.0574888E-2,2.1776952E-1,-9.7974725E-2,5.783466E-2,-4.4122584E-2,-2.63614E-1,-8.361366E-2,5.5919047E-2,9.514067E-2,-5.432359E-3,1.0960214E-2,-2.0875556E-2,1.09241055E-2,1.534435E-2,2.5664932E-1,1.2107162E-1,7.635E-3,7.986844E-4,1.10072434E-1,-1.1557798E-2,4.182778E-2,2.1861558E-1,-3.4585872E-1,-1.8286246E-1,2.3021502E-3,-1.28829945E-2,1.6224454E-1,3.4181157E-1,-7.215424E-2,4.2610988E-2,1.0774743E-2,1.6284238E-1,3.5209772E-1,1.7419936E-1,2.0746378E-2,3.943533E-2,-1.7888819E-1,-3.7240282E-1,-2.966575E-1,-1.8338321E-1,-2.211397E-3,-7.6100626E-3,-7.046838E-3,-1.2966547E-3,-1.4824035E-2,-7.074653E-3,-2.6122104E-2,-1.14993766E-1,-1.6640384E-1,-8.103721E-4,-4.8510535E-3,-1.6089024E-2,-3.3525934E-3,2.58239E-3,8.909006E-3,7.566232E-2,1.9152714E-2,9.891238E-3,-1.9762518E-1,-9.209347E-2,-8.469684E-2,-2.3609509E-1,-2.8941937E-2,1.2485183E-1,-2.265758E-1,-2.8078647E-2,-6.013666E-2,-3.8979852E-1,-8.523945E-3,-1.3371936E-2,-4.6185926E-2,9.499708E-2,1.9931244E-3,1.2124152E-2,-2.7330447E-2,-9.238773E-3,9.171241E-2,-1.6471723E-2,2.647216E-3,-5.8804443E-3,-1.7084498E-2,-8.590984E-3,-7.00955E-3,-1.5319178E-3,4.0167596E-4,4.7208196E-3,1.7931165E-1,4.031092E-2,6.095953E-3,-4.2862813E-3,-4.182665E-3,7.08503E-2,3.5693586E-1,1.713658E-1,9.252314E-3,9.297939E-2,5.2153956E-2,1.5405034E-1,-6.969768E-2,8.1465155E-2,3.1594953E-1,-6.298028E-3,-8.978944E-3,-3.727999E-1,-2.0031251E-1,-4.519476E-3,-4.7235496E-2,2.0474689E-1,1.865121E-1,-4.5616296E-3,3.6207238E-1,6.6850004E-3,1.2104352E-2,-1.17899105E-1,-1.847551E-2,6.0598846E-3,6.766848E-2,-7.5463764E-2,1.8514681E-1,6.040299E-3,4.1957068E-1,2.0278804E-1,1.142819E-1,2.524419E-1,-2.7641323E-1,-5.5500016E-2,-4.204891E-1,-1.9028929E-1,-3.2429925E-1,-5.2164607E-3,-8.017375E-2,-2.2306675E-1,1.015058E-2,-1.356652E-1,-1.4179027E-1,1.2120441E-4,-4.2421366E-3,-1.9976483E-1,6.9275573E-3,4.9835768E-2,-7.924374E-3,-1.4411678E-2,-1.04738E-3,-5.97628E-3,-3.88422E-4,-8.76665E-3,-5.5080075E-3,-1.3925713E-2,2.6318713E-3,-4.756387E-3,1.0055852E-2,-2.0895735E-3,-1.4241792E-2,-4.536692E-3,7.116965E-3,-2.359088E-3,-5.3893565E-3,1.4960763E-4,-3.0257624E-2,-1.4235477E-2,-4.2572115E-3,1.7987947E-3,3.3775358E-3,-6.8511846E-3,2.7196142E-3,8.0481935E-3,-2.899745E-3,7.4847153E-4,1.9410966E-3,9.40724E-3,3.0814498E-3,-6.5425946E-3,1.5132205E-2,7.475889E-3,4.34439E-3,-4.799948E-3,5.3366153E-3,-7.6080114E-4,1.0448889E-2,2.0913728E-2,5.130254E-3,1.0808192E-2,1.7301237E-3,5.959566E-3,1.1787064E-5,7.2751744E-3,1.1572766E-2,5.72069E-3,-4.7296775E-3,2.9113214E-3,2.433914E-3,1.0669514E-2,1.112293E-2,2.403849E-2,-1.3748947E-2,-2.3007227E-2,-6.8787285E-3,-1.2279327E-2,-4.883655E-3,2.5187896E-3,1.5049516E-2,5.3268755E-3,1.0478555E-2,1.0545629E-3,1.8889371E-2,6.3354615E-3,-2.084994E-3,3.1736593E-3,-8.697866E-3,-2.3478055E-3,9.5914013E-4,-3.3737475E-3,5.0381725E-3,-7.7253475E-4,-6.6968156E-3,2.7160606E-4,1.2845198E-2,7.280802E-3,-5.8652204E-3,5.390706E-3,2.258891E-2,1.3700842E-2,5.528457E-3,1.4147355E-2,7.152677E-3,-5.0977725E-3,1.4882519E-2,-3.0683184E-3,-5.188671E-3,-1.6138848E-2,4.8909206E-3,-6.675779E-3,-1.28377555E-2,-2.3081241E-2,-5.1743547E-3,-1.4230021E-2,-9.591525E-3,-1.8087173E-2,-3.0418258E-4,-9.026316E-3,-2.0545848E-2,-1.0160383E-2,-4.5689967E-4,4.782198E-3,-3.7796458E-3,-1.01475855E-2,-9.043667E-3,-3.5382833E-4,-1.1262926E-2,-4.6537737E-3,7.9455687E-4,3.8051794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,-1,-1,77,79,81,83,-1,85,-1,87,-1,89,-1,-1,91,-1,93,95,-1,-1,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,127,-1,129,131,133,135,137,139,-1,141,143,145,147,149,151,-1,153,-1,-1,155,157,159,161,-1,-1,163,165,167,-1,169,-1,171,173,175,177,-1,-1,179,181,183,185,187,189,191,193,195,197,199,-1,201,-1,-1,203,205,207,-1,-1,209,211,-1,213,215,217,219,-1,221,223,225,227,229,231,233,235,-1,-1,237,239,241,243,-1,-1,-1,-1,-1,-1,245,247,249,-1,-1,-1,-1,-1,-1,251,-1,-1,253,255,257,259,261,263,265,267,269,271,-1,273,275,277,-1,-1,279,-1,281,283,-1,-1,-1,-1,-1,-1,-1,-1,285,287,-1,-1,-1,289,291,293,-1,295,297,299,301,303,305,-1,-1,307,309,-1,311,313,315,-1,317,-1,319,321,323,-1,325,327,329,331,333,335,337,339,341,343,345,347,349,-1,351,353,355,357,359,-1,-1,361,-1,363,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6758526E1,1.3406004E1,7.496645E0,4.2886977E0,7.961891E0,1.7242584E0,1.7772964E0,2.432657E0,6.741171E-1,4.8090563E0,2.4420109E0,1.6363449E0,1.7120686E0,6.864104E-1,1.0573721E0,1.5881748E0,1.000255E0,6.387546E-1,1.2714154E0,2.978263E-1,3.3836162E0,1.6045084E0,2.5243988E0,1.0460892E0,3.801748E-1,7.077026E-2,5.8344704E-1,2.1427047E-1,1.4198315E-1,1.7359908E-1,3.5333848E-1,8.180547E-1,1.4457545E0,3.9686573E-1,2.6172185E-1,5.6895375E-1,3.4471178E-1,1.3998595E-1,0E0,0E0,0E0,2.4361985E0,1.4829533E0,8.518863E-1,5.753031E-1,0E0,1.3010864E0,0E0,4.337368E-1,0E0,1.046725E-1,0E0,0E0,4.057958E-2,0E0,7.77365E-2,1.6928095E-1,0E0,0E0,3.6020275E-2,2.7167502E-1,1.8811014E-1,7.129574E-2,2.7634263E-1,6.081307E-1,1.5995336E-1,3.536911E-1,4.4900763E-1,2.810488E-1,2.4273062E-1,1.1560165E-1,2.6376045E-1,6.3471276E-1,2.1241197E-1,2.2265601E-1,0E0,4.6724707E-2,1.7125592E0,1.7149713E0,2.0462084E-1,4.194621E-1,9.194088E-1,0E0,1.325098E-1,4.6371448E-1,1.5316792E0,2.4055004E-2,1.0492859E0,2.8065968E-1,0E0,2.4423927E-2,0E0,0E0,2.7459964E-2,4.4992387E-2,1.4081442E-1,7.330656E-2,0E0,0E0,9.068668E-2,4.0811423E-2,6.98407E-2,0E0,1.0920644E-2,0E0,2.988925E-1,1.4888656E-1,4.870434E-1,8.4355515E-1,0E0,0E0,7.935803E-1,1.376977E-1,1.5912732E-1,1.2749064E-1,9.319757E-2,1.07238635E-1,7.7668056E-2,3.453797E-2,3.471203E-2,1.9549519E-2,4.1735858E-1,0E0,1.393739E-1,0E0,0E0,1.2139E-1,2.7239203E-1,2.4198204E-2,0E0,0E0,6.4403176E-1,1.1522467E0,0E0,8.449551E-1,3.1492233E-2,1.4133811E-2,5.6124634E-1,0E0,3.9234507E-1,2.0865583E-1,9.007825E-2,8.874014E-2,1.4739302E-1,2.5069296E-1,1.4059219E0,3.368218E-1,0E0,0E0,5.502305E-1,6.81859E-1,1.5746069E-1,3.4253883E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.2388871E-1,8.5297585E-2,5.1016927E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.53819E-2,0E0,0E0,7.475793E-2,3.0473733E-1,7.542737E-2,1.4623463E-1,5.402969E-1,3.394892E-1,2.5189638E-1,1.9269836E-1,4.2501476E-2,2.3486733E-1,0E0,5.27159E-2,1.3342135E-1,5.824639E-2,0E0,0E0,1.3859766E-2,0E0,1.4071642E-1,1.3330081E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.674562E-2,3.708595E-1,0E0,0E0,0E0,4.4952363E-2,6.731343E-2,4.8796117E-2,0E0,2.3485899E-2,5.433411E-1,4.183817E-1,4.239061E-1,3.443135E-1,1.857847E-1,0E0,0E0,2.7346611E-2,1.1712551E-2,0E0,2.2683708E-1,7.200402E-2,2.1045923E-1,0E0,1.5868664E-1,0E0,2.755848E-2,5.157192E-2,2.1619236E-2,0E0,5.118054E-2,5.758007E-2,1.4123416E-1,1.3836499E-1,3.533516E-1,3.240484E-1,3.018334E-1,4.8857188E-1,1.7545223E-1,2.766372E-1,3.4226418E-1,1.344356E-1,8.512139E-2,0E0,1.9243512E-1,1.501472E-1,7.185946E-2,4.1292846E-2,1.149025E-1,0E0,0E0,2.5489867E-2,0E0,1.665093E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,41,41,42,42,43,43,44,44,46,46,48,48,50,50,53,53,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,78,78,79,79,80,80,81,81,83,83,84,84,85,85,86,86,87,87,88,88,90,90,93,93,94,94,95,95,96,96,99,99,100,100,101,101,103,103,105,105,106,106,107,107,108,108,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,126,126,127,127,128,128,131,131,132,132,134,134,135,135,136,136,137,137,139,139,140,140,141,141,142,142,143,143,144,144,145,145,146,146,149,149,150,150,151,151,152,152,159,159,160,160,161,161,168,168,171,171,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,180,180,182,182,183,183,184,184,187,187,189,189,190,190,199,199,200,200,204,204,205,205,206,206,208,208,209,209,210,210,211,211,212,212,213,213,216,216,217,217,219,219,220,220,221,221,223,223,225,225,226,226,227,227,229,229,230,230,231,231,232,232,233,233,234,234,235,235,236,236,237,237,238,238,239,239,240,240,241,241,243,243,244,244,245,245,246,246,247,247,250,250,252,252],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,-1,-1,78,80,82,84,-1,86,-1,88,-1,90,-1,-1,92,-1,94,96,-1,-1,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,128,-1,130,132,134,136,138,140,-1,142,144,146,148,150,152,-1,154,-1,-1,156,158,160,162,-1,-1,164,166,168,-1,170,-1,172,174,176,178,-1,-1,180,182,184,186,188,190,192,194,196,198,200,-1,202,-1,-1,204,206,208,-1,-1,210,212,-1,214,216,218,220,-1,222,224,226,228,230,232,234,236,-1,-1,238,240,242,244,-1,-1,-1,-1,-1,-1,246,248,250,-1,-1,-1,-1,-1,-1,252,-1,-1,254,256,258,260,262,264,266,268,270,272,-1,274,276,278,-1,-1,280,-1,282,284,-1,-1,-1,-1,-1,-1,-1,-1,286,288,-1,-1,-1,290,292,294,-1,296,298,300,302,304,306,-1,-1,308,310,-1,312,314,316,-1,318,-1,320,322,324,-1,326,328,330,332,334,336,338,340,342,344,346,348,350,-1,352,354,356,358,360,-1,-1,362,-1,364,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7810526E2,1E0,1.0398213E6,2.1150263E-1,1.2427474E8,1.0122174E3,4.264897E6,3.8977896E2,8E0,1.1410706E3,1.1879E4,1.0052E4,1.2199979E10,1E0,9.669789E0,9.47306E3,4.39E2,1.0828514E5,3.275E3,4.5843E7,1.2909952E1,2.943086E4,4.8543688E-2,1E0,8.8070023E-1,2.332E3,1.06E3,6.3300834E0,2E0,1.4777102E9,1.4735735E2,2E0,2.46E2,1.910669E-2,3.891E3,5.3379045E6,3.2069644E2,-2.6329765E-2,1.6529249E-2,3.809255E-2,4.247868E3,1.415178E3,1.5846036E7,1.509E3,-2.0990105E-2,1.9013363E4,-2.5292087E-2,1.9121015E0,9.564343E-3,5.3084288E8,-2.0961251E-2,-3.897628E-2,1.2068E4,-2.2442933E-2,4.3360384E7,1.044E1,-2.5573578E-2,-9.868757E-3,3.076124E5,6.1392635E-1,8.965631E7,3.6E1,3.4782608E0,2.9676E4,5.7E1,2.183E3,1.7733E4,7.158023E7,3.142868E5,4.08648E5,9.07826E0,1.8E1,4.144144E-1,2.821E3,-4.620106E-3,6.31591E3,1.0938637E1,2.909019E3,1.4956522E1,2.2980049E10,5.732733E2,-7.2133923E-3,1.477425E6,2.2783158E5,2.1938796E7,1.6956E5,1.4105052E0,1.6681991E-3,2.456759E-3,6.17296E5,1.7712706E-3,-4.667478E-3,1.227133E6,8.01E2,5.998024E-1,2.1601E4,-2.0996104E-3,3.6816057E-3,5.534958E-1,9.452547E7,1.2716E4,-5.084484E-3,5.9572783E0,7.318627E-3,3.6E1,5.5E1,1.1968015E8,4.08648E5,-1.5253458E-2,-3.5533275E-2,2.5464671E2,4.46E2,1.1E1,3.97E2,2.3E1,5.954224E0,8.286065E2,5.099062E5,1.1411955E-1,2.9311974E8,1.3414634E0,-5.432359E-3,1.62283E0,-2.0875556E-2,1.09241055E-2,1E0,1.9676556E3,9E0,7.635E-3,7.986844E-4,1.857051E5,6.624E3,4.182778E-2,2.62797E5,4.8297736E7,5.378E3,6.7652373E3,-1.28829945E-2,4.1179886E0,3.2613106E5,2.3E1,1.5166431E5,1.3651616E5,1.4569893E0,2.1E1,4.247868E3,2.0746378E-2,3.943533E-2,2.4222438E2,1.4072216E-1,2.5247778E6,1.5316E4,-2.211397E-3,-7.6100626E-3,-7.046838E-3,-1.2966547E-3,-1.4824035E-2,-7.074653E-3,1.9051096E7,4.576E3,1.902353E1,-8.103721E-4,-4.8510535E-3,-1.6089024E-2,-3.3525934E-3,2.58239E-3,8.909006E-3,3.835264E-2,1.9152714E-2,9.891238E-3,8.2E1,3.4E2,4.675E0,4.1504726E-3,1E0,1.526038E6,4.9612486E-1,9.562857E2,9.070543E0,5.542E3,-8.523945E-3,1E0,1.5964827E4,6.62E2,1.9931244E-3,1.2124152E-2,2.1975676E2,-9.238773E-3,4.6339836E0,4.1E2,2.647216E-3,-5.8804443E-3,-1.7084498E-2,-8.590984E-3,-7.00955E-3,-1.5319178E-3,4.0167596E-4,4.7208196E-3,7.692308E-1,6.896236E-2,6.095953E-3,-4.2862813E-3,-4.182665E-3,1.6073539E7,1.6954315E0,9.76E2,9.252314E-3,2.8411612E6,1E0,1.4E1,6.4625664E8,2.9138382E7,8.61491E3,-6.298028E-3,-8.978944E-3,1.363881E3,3.6022064E1,-4.519476E-3,1.20086E5,6.135771E9,3.1639907E0,-4.5616296E-3,2.9273078E0,6.6850004E-3,1.5481343E1,3.14E2,4.5154482E-1,6.0598846E-3,4E1,2.9816154E1,3.3382E4,1.4523809E0,8.2E1,1.153215E9,3.5301748E7,1.3144558E4,2.4843138E-1,7.518072E-1,4.44E2,3.3018634E5,4.5168175E5,-5.2164607E-3,1.74123E5,3.25855E7,9.457831E0,3.791269E7,6.604754E5,1.2120441E-4,-4.2421366E-3,2.870791E6,6.9275573E-3,1.4008022E11,-7.924374E-3,-1.4411678E-2,-1.04738E-3,-5.97628E-3,-3.88422E-4,-8.76665E-3,-5.5080075E-3,-1.3925713E-2,2.6318713E-3,-4.756387E-3,1.0055852E-2,-2.0895735E-3,-1.4241792E-2,-4.536692E-3,7.116965E-3,-2.359088E-3,-5.3893565E-3,1.4960763E-4,-3.0257624E-2,-1.4235477E-2,-4.2572115E-3,1.7987947E-3,3.3775358E-3,-6.8511846E-3,2.7196142E-3,8.0481935E-3,-2.899745E-3,7.4847153E-4,1.9410966E-3,9.40724E-3,3.0814498E-3,-6.5425946E-3,1.5132205E-2,7.475889E-3,4.34439E-3,-4.799948E-3,5.3366153E-3,-7.6080114E-4,1.0448889E-2,2.0913728E-2,5.130254E-3,1.0808192E-2,1.7301237E-3,5.959566E-3,1.1787064E-5,7.2751744E-3,1.1572766E-2,5.72069E-3,-4.7296775E-3,2.9113214E-3,2.433914E-3,1.0669514E-2,1.112293E-2,2.403849E-2,-1.3748947E-2,-2.3007227E-2,-6.8787285E-3,-1.2279327E-2,-4.883655E-3,2.5187896E-3,1.5049516E-2,5.3268755E-3,1.0478555E-2,1.0545629E-3,1.8889371E-2,6.3354615E-3,-2.084994E-3,3.1736593E-3,-8.697866E-3,-2.3478055E-3,9.5914013E-4,-3.3737475E-3,5.0381725E-3,-7.7253475E-4,-6.6968156E-3,2.7160606E-4,1.2845198E-2,7.280802E-3,-5.8652204E-3,5.390706E-3,2.258891E-2,1.3700842E-2,5.528457E-3,1.4147355E-2,7.152677E-3,-5.0977725E-3,1.4882519E-2,-3.0683184E-3,-5.188671E-3,-1.6138848E-2,4.8909206E-3,-6.675779E-3,-1.28377555E-2,-2.3081241E-2,-5.1743547E-3,-1.4230021E-2,-9.591525E-3,-1.8087173E-2,-3.0418258E-4,-9.026316E-3,-2.0545848E-2,-1.0160383E-2,-4.5689967E-4,4.782198E-3,-3.7796458E-3,-1.01475855E-2,-9.043667E-3,-3.5382833E-4,-1.1262926E-2,-4.6537737E-3,7.9455687E-4,3.8051794E-3],"split_indices":[19,52,59,28,38,45,52,9,4,18,52,2,2,5,102,54,48,10,33,2,45,56,33,58,64,27,0,2,53,8,7,52,17,10,38,29,45,52,0,0,0,52,52,47,10,0,52,0,53,0,7,0,0,9,0,5,54,0,0,28,42,45,3,56,29,0,2,1,7,33,1,56,3,53,10,0,33,53,32,58,5,52,0,9,28,51,2,53,41,0,29,0,0,45,2,57,10,0,0,27,48,9,0,53,0,2,0,7,1,0,0,4,11,3,2,8,54,4,33,38,7,53,0,54,0,0,102,33,3,0,0,28,2,0,2,45,2,52,0,53,33,3,28,28,46,3,52,0,0,4,38,28,9,0,0,0,0,0,0,45,2,58,0,0,0,0,0,0,27,0,0,29,12,54,38,16,9,41,33,53,29,0,16,33,2,0,0,4,0,54,0,0,0,0,0,0,0,0,0,53,38,0,0,0,12,54,2,0,28,96,3,7,9,4,0,0,4,56,0,9,5,42,0,53,0,56,0,38,0,3,56,9,54,8,7,45,52,27,56,0,33,28,0,29,5,54,45,28,0,0,48,0,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.549E3,1.995E3,5.54E2,8.87E2,1.108E3,3.28E2,2.26E2,6.7E2,2.17E2,6.07E2,5.01E2,2.97E2,3.1E1,1.41E2,8.5E1,5.06E2,1.64E2,1.98E2,1.9E1,1.3E1,5.94E2,2.63E2,2.38E2,2.74E2,2.3E1,1.8E1,1.3E1,1.32E2,9E0,2.9E1,5.6E1,4.44E2,6.2E1,1.21E2,4.3E1,1.12E2,8.6E1,1.5E1,4E0,5E0,8E0,4.94E2,1E2,1.25E2,1.38E2,4E0,2.34E2,2E1,2.54E2,6E0,1.7E1,5E0,1.3E1,8E0,5E0,2.3E1,1.09E2,4E0,5E0,9E0,2E1,3.9E1,1.7E1,2.31E2,2.13E2,1E1,5.2E1,6.5E1,5.6E1,2.1E1,2.2E1,9.7E1,1.5E1,2.6E1,6E1,4E0,1.1E1,4.66E2,2.8E1,4E1,6E1,1.2E2,5E0,3.9E1,9.9E1,2.24E2,1E1,1.32E2,1.22E2,4E0,1.3E1,4E0,4E0,9E0,1.4E1,8.1E1,2.8E1,4E0,5E0,1E1,1E1,3.5E1,4E0,1E1,7E0,1.89E2,4.2E1,1.23E2,9E1,4E0,6E0,3E1,2.2E1,3.5E1,3E1,1.5E1,4.1E1,9E0,1.2E1,1.2E1,1E1,9.1E1,6E0,1.1E1,4E0,6E0,2E1,3.8E1,2.2E1,7E0,4E0,2.55E2,2.11E2,5E0,2.3E1,2.1E1,1.9E1,5.4E1,6E0,6E1,6E1,2.2E1,1.7E1,2.8E1,7.1E1,1.47E2,7.7E1,5E0,5E0,4.6E1,8.6E1,3.5E1,8.7E1,4E0,9E0,5E0,4E0,7E0,7E0,5.5E1,2.6E1,2.4E1,4E0,4E0,6E0,6E0,4E0,1E1,2.5E1,5E0,5E0,3.3E1,1.56E2,1E1,3.2E1,9.8E1,2.5E1,3.4E1,5.6E1,1.3E1,1.7E1,9E0,1.3E1,1.1E1,2.4E1,4E0,2.6E1,9E0,6E0,2.8E1,1.3E1,4E0,5E0,5E0,7E0,5E0,7E0,5E0,5E0,3.5E1,5.6E1,5E0,6E0,7E0,1.3E1,1.6E1,2.2E1,5E0,1.7E1,1.11E2,1.44E2,1.3E2,8.1E1,1.8E1,5E0,4E0,1.7E1,1.5E1,4E0,4.4E1,1E1,5.5E1,5E0,5.4E1,6E0,8E0,1.4E1,1E1,7E0,1.7E1,1.1E1,6.2E1,9E0,1E2,4.7E1,4.5E1,3.2E1,2.5E1,2.1E1,6.7E1,1.9E1,3E1,5E0,2.5E1,6.2E1,4.2E1,1.3E1,2.1E1,5E0,8E0,1.6E1,6E0,1.9E1,2.5E1,8E0,4.4E1,1.12E2,6E0,4E0,9E0,2.3E1,4.4E1,5.4E1,1.7E1,8E0,2.3E1,1.1E1,5E0,5.1E1,7E0,6E0,4E0,1.3E1,5E0,8E0,5E0,6E0,1.6E1,8E0,5E0,4E0,1.9E1,9E0,8E0,5E0,5E0,3E1,4.2E1,1.4E1,9E0,4E0,6E0,1E1,1E1,1.2E1,6E0,1.1E1,7.2E1,3.9E1,4.7E1,9.7E1,1.09E2,2.1E1,6.6E1,1.5E1,1.3E1,5E0,1E1,7E0,8E0,7E0,2.9E1,1.5E1,4E0,6E0,4.8E1,7E0,5E1,4E0,4E0,4E0,7E0,7E0,6E0,4E0,1.2E1,5E0,6E0,5E0,2E1,4.2E1,4E0,5E0,8E1,2E1,2.3E1,2.4E1,4E1,5E0,2.8E1,4E0,6E0,1.9E1,7E0,1.4E1,1.5E1,5.2E1,1.1E1,8E0,8E0,2.2E1,1.5E1,1E1,4E0,5.8E1,3.5E1,7E0,8E0,5E0,1.6E1,5E0,1.2E1,4E0,9E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"365","size_leaf_vector":"1"}},{"base_weights":[-2.7032458E-4,-1.3984819E-1,6.422443E-2,-2.1445234E-1,-4.816451E-2,2.6187502E-2,2.2788891E-1,-2.6547816E-1,-7.801984E-2,-9.585498E-2,1.0837803E-1,-2.20806E-2,1.22463696E-1,2.716662E-1,8.646849E-2,-3.6975067E-2,-2.5258416E-1,2.7473561E-2,-1.0955083E-1,-1.557308E-1,-4.8030745E-2,-2.4050321E-2,1.6714391E-1,-1.02221176E-1,8.947416E-3,1.3385218E-1,-1.7221563E-1,1.5695477E-1,3.4215546E-1,-1.7864196E-1,1.2747931E-1,-1.1344666E-1,-2.8029728E-1,-2.0292552E-2,-2.2466365E-1,-1.21116385E-1,-2.5038064E-1,-1.8719858E-3,-8.656486E-2,-9.182159E-3,1.1716074E-2,2.3637486E-1,9.698775E-2,-1.5224193E-1,-5.650308E-2,4.1784394E-1,-2.1673418E-3,4.941783E-2,1.8172202E-1,-2.4519522E-1,-1.1245266E-3,5.013817E-2,2.376122E-1,3.2901205E-2,3.203135E-1,-1.6611993E-2,2.4739714E-4,5.4263923E-2,2.7155077E-1,-1.9906391E-1,-4.8364043E-2,-2.5772718E-1,-4.4016725E-1,-4.577529E-2,1.022669E-2,-1.8066485E-1,-3.8699228E-1,-2.8432646E-1,-9.065996E-2,-2.155242E-2,-2.0912504E-1,-4.839647E-2,6.4875506E-2,-1.3348478E-1,-2.5019582E-2,6.6095283E-3,-2.0627582E-2,1.770146E-1,1.6561845E-2,-4.609742E-3,1.4609659E-1,-1.2865824E-1,-3.0486467E-1,-7.7751085E-2,3.4516465E-2,1.3348433E-2,2.4456488E-2,1.5616426E-2,-2.1587877E-1,5.694937E-2,-1.0217562E-2,1.543495E-1,3.069707E-1,-1.5582514E-2,-6.323576E-3,9.504884E-2,-1.07571915E-1,1.7242701E-1,4.3494686E-1,2.9932392E-1,2.7516734E-2,1.0478522E-1,-7.2306655E-2,2.1125209E-2,1.2269398E-1,-9.3230896E-2,-3.2732376E-1,2.971768E-2,-1.3883784E-1,-2.4149744E-2,-2.4322537E-1,-5.3696734E-1,-2.1535185E-1,-7.2035985E-3,-2.0267607E-1,-1.4565846E-1,-2.0515336E-2,-1.0380232E-2,-2.621715E-2,-6.210844E-3,-3.3590716E-1,-1.1170661E-1,4.9601737E-2,-1.6613857E-1,-1.570704E-2,-8.065262E-2,3.224485E-2,7.5178347E-3,2.691601E-2,-6.921801E-2,-1.8090641E-1,-7.892946E-2,1.3570749E-2,-5.780194E-2,2.5661536E-3,1.103785E-2,4.175568E-3,-3.399548E-4,1.7554443E-1,-1.7114815E-1,-7.020984E-2,-2.5681742E-2,-1.9232896E-1,-6.6685885E-2,-1.3189506E-2,8.93242E-2,-5.3373105E-3,7.299587E-2,-2.857933E-2,-1.62224E-1,-1.567183E-2,7.4998416E-2,-3.2106724E-2,1.7313023E-1,3.5468366E-2,1.7755918E-1,3.7826005E-1,-2.8423233E-2,1.4767984E-1,-1.4243937E-2,2.9115968E-3,2.7423242E-1,1.3326682E-1,2.8935036E-2,1.5316867E-2,3.116936E-1,1.3411217E-3,2.0807871E-1,3.8985193E-2,-6.4587938E-3,1.2580372E-3,1.2670864E-2,-3.6114997E-3,-1.1040944E-2,-2.0040162E-2,-7.1754875E-3,-2.409997E-2,9.141283E-2,-5.441774E-3,-1.6829006E-1,-2.312226E-3,-2.5719386E-1,-7.217936E-2,-6.567521E-1,-3.0428192E-1,-1.571914E-2,-4.0082852E-3,6.4707235E-2,-5.3365313E-2,-1.2587613E-2,-4.1647316E-3,-2.0576331E-1,-1.4244282E-2,-2.0144755E-2,-9.394616E-3,-1.2843232E-1,1.5929868E-3,6.254933E-3,-3.304352E-3,-1.9271103E-1,-4.7742096E-3,-1.3349354E-1,2.755756E-2,-1.9106425E-3,4.0511293E-3,-5.277993E-3,6.223909E-2,1.2815022E-3,-1.06395386E-1,-8.800795E-2,-1.1511102E-2,-1.010768E-1,-4.99165E-4,5.904957E-2,-3.184633E-3,-6.0201937E-4,-5.4550096E-3,9.598328E-3,4.2054392E-3,-1.5590136E-1,-1.5830977E-2,-3.663375E-2,-1.7570151E-2,-5.7883533E-3,-1.3802725E-2,-4.8479542E-2,-1.3980363E-1,1.7196211E-1,1.9902071E-2,5.4220382E-2,2.698219E-1,-1.6695875E-1,1.9082244E-2,-1.9405872E-1,-7.512473E-2,3.755927E-2,1.3221446E-1,-1.4760104E-1,5.1307306E-2,8.7226324E-2,2.051922E-1,-2.4780603E-2,1.6369626E-1,1.1028273E-1,1.1121472E-2,7.8856135E-3,4.230615E-1,2.8904986E-3,-4.296407E-3,3.6082002E-3,1.7450699E-1,7.426255E-3,1.6228E-2,7.769007E-2,1.9477959E-1,3.6983365E-1,2.4196208E-1,8.921824E-2,1.9007912E-2,-9.636634E-3,1.1028746E-1,8.7733666E-4,-2.400014E-3,8.009161E-3,1.4154699E-3,-1.0661325E-2,-3.5737106E-3,-9.054313E-3,-1.50050735E-2,-5.82045E-3,-1.1164623E-4,-1.846646E-2,-3.714133E-2,-4.4178334E-3,-2.2949282E-2,1.0483824E-2,7.6531124E-4,-5.122479E-3,2.2646338E-3,-4.0926514E-3,-1.5293536E-2,-4.9765357E-3,1.7366741E-3,-9.090137E-3,-3.9488473E-3,-4.724151E-3,-1.0983287E-2,-8.324311E-3,-3.539858E-3,-2.4503737E-4,3.372363E-3,4.78478E-3,9.0333796E-4,-7.2021144E-3,-3.049183E-3,-5.9173135E-3,-1.3901812E-3,-1.9583597E-3,-6.351731E-3,5.250443E-3,-4.2804726E-4,-2.008671E-3,-8.790313E-3,7.1127038E-3,-2.9026007E-3,4.6544994E-4,-3.575706E-3,-1.0140852E-2,-4.6835095E-3,4.290685E-3,1.1175588E-2,4.240684E-3,-1.1587675E-3,3.9814557E-3,-2.5952375E-3,6.088019E-3,2.2485606E-2,-1.1337061E-2,-3.1645806E-3,-1.7881347E-4,5.8097956E-3,-1.1262475E-2,-5.6610666E-3,-5.556142E-3,-4.7961218E-4,2.885474E-3,-1.4437075E-2,2.884086E-3,8.601559E-3,-1.9265467E-3,-1.4617842E-2,-3.6375455E-3,4.751528E-3,5.919107E-3,-7.85798E-3,1.1084044E-2,-1.259158E-3,-3.4436495E-3,5.952073E-3,3.2386337E-3,1.1756853E-2,2.0518044E-3,7.4742637E-3,2.28147E-2,9.537982E-3,1.0312905E-2,4.970015E-3,5.558323E-3,8.820614E-4,5.471686E-3,1.1315718E-2,2.0812908E-2,1.45429745E-2,9.761671E-3,1.7239287E-2,9.633885E-4,7.0609003E-3,2.1218492E-3,-3.3879918E-3,7.471674E-3,2.4475798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,83,85,87,89,91,93,-1,95,97,-1,99,-1,-1,101,103,105,107,109,111,113,-1,115,117,119,121,-1,123,125,127,129,131,-1,133,135,-1,-1,137,139,141,143,145,-1,-1,147,149,151,-1,153,155,-1,-1,157,159,161,163,165,-1,167,169,-1,171,173,175,177,179,-1,181,183,185,187,189,191,-1,-1,-1,-1,193,195,197,199,-1,201,203,-1,205,207,209,211,213,215,-1,-1,-1,-1,217,219,221,-1,223,225,-1,227,-1,229,231,233,-1,235,237,239,241,243,245,247,249,-1,-1,251,253,-1,-1,255,-1,257,259,-1,-1,-1,-1,-1,261,-1,-1,263,-1,265,-1,267,269,271,273,-1,-1,275,277,-1,-1,279,281,-1,-1,283,-1,-1,-1,285,-1,287,289,-1,-1,-1,291,-1,293,295,-1,297,-1,299,-1,-1,-1,-1,-1,301,-1,303,-1,-1,-1,305,307,309,311,313,315,317,319,321,323,325,327,329,331,333,335,337,339,341,-1,-1,343,-1,-1,-1,345,-1,-1,347,349,351,353,355,-1,357,359,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2901169E1,5.4890175E0,1.0826044E1,3.0632668E0,2.7129314E0,6.5690126E0,2.0071583E0,1.7832203E0,2.4754658E0,7.885771E-1,6.655127E-1,2.3432949E0,1.6070614E0,1.9677773E0,8.816926E-1,0E0,1.1828213E0,0E0,1.1975893E0,3.7265348E-1,2.7731314E-1,1.5638895E-1,2.6363635E-1,5.928402E-1,3.09136E0,1.8300657E0,1.9114584E-1,8.237951E-1,8.530369E-1,3.1228855E-1,7.1309376E-1,2.9127103E-1,8.3192635E-1,3.9345762E-1,2.9433966E-1,4.3153608E-1,1.486907E-1,2.2672802E-1,2.4211472E-1,0E0,9.241566E-2,1.0500741E-1,3.0157867E-1,4.107027E-1,2.715074E-1,6.349468E-2,2.5231652E0,3.2908687E-1,9.46846E-1,5.5438697E-2,0E0,3.147878E-1,6.324849E-1,0E0,5.411825E-1,0E0,0E0,3.0811864E-1,4.6591187E-1,2.8168827E-1,2.3018515E-1,6.414242E-1,5.589905E-1,3.6770767E-1,0E0,2.7823305E-1,1.3782561E-1,7.380688E-2,2.3875195E-1,0E0,7.44257E-2,1.1445132E-1,9.3409196E-2,1.3376439E-1,8.012634E-2,0E0,5.3812407E-2,6.356722E-2,0E0,0E0,1.1375886E-1,2.6116526E-1,2.776698E-1,2.1523428E-1,2.1868849E-1,0E0,0E0,1.5566099E0,2.1664786E-1,2.6280314E-1,0E0,5.2970123E-1,4.0334558E-1,0E0,0E0,2.2514516E-1,3.1097677E-1,1.3488281E-1,6.30939E-2,4.5002842E-1,0E0,2.2297636E-1,8.0985986E-2,0E0,3.5009354E-1,1.2645835E-1,2.1549416E-1,1.6390127E-1,3.3104867E-2,0E0,5.0707245E-1,3.8547707E-1,1.22280836E-1,1.694295E-1,4.9515635E-2,2.9721695E-1,0E0,0E0,0E0,0E0,2.366376E-2,1.6980529E-1,1.0449691E-1,2.2853732E-2,0E0,1.8093446E-1,4.7843903E-2,0E0,1.0957709E-1,8.031916E-2,1.03806734E-1,2.3419462E-2,8.4626496E-2,2.9239915E-2,0E0,0E0,0E0,0E0,1.5591621E-2,9.0540886E-2,4.406817E-1,0E0,4.501298E-2,1.369927E-1,0E0,1.1131561E-1,0E0,9.720701E-1,2.2917833E0,8.246869E-2,0E0,2.8422022E-1,2.7732068E-1,5.5223083E-1,2.6672685E-1,3.602743E-2,2.4926996E-1,5.9600085E-2,3.598851E-2,0E0,0E0,2.156663E-2,9.7684026E-2,0E0,0E0,4.3755913E-1,0E0,2.3234707E-1,7.733342E-2,0E0,0E0,0E0,0E0,0E0,1.1368025E-2,0E0,0E0,5.1945426E-2,0E0,3.2723486E-2,0E0,6.105356E-1,5.5522986E-2,4.847765E-2,2.6054418E-1,0E0,0E0,1.4012748E-1,1.550207E-1,0E0,0E0,2.9795778E-1,5.7832446E-2,0E0,0E0,1.494658E-1,0E0,0E0,0E0,1.9123495E-2,0E0,3.1749874E-2,1.4590757E-2,0E0,0E0,0E0,2.4913609E-2,0E0,1.8603086E-2,1.6842708E-2,0E0,1.4357023E-2,0E0,4.7585122E-2,0E0,0E0,0E0,0E0,0E0,1.2960672E-1,0E0,1.7724815E-1,0E0,0E0,0E0,1.1806169E-1,4.2072505E-2,2.0851284E-2,3.556421E-2,6.6637975E-1,5.5845547E-1,5.383761E-1,5.70848E-1,3.774196E-2,2.4933066E-2,5.63778E-1,1.4772093E-1,1.783931E-1,9.93045E-2,4.615535E-1,5.8053064E-1,1.6064739E-1,6.262484E-2,2.106227E-2,0E0,0E0,1.1528826E-1,0E0,0E0,0E0,2.0150483E-2,0E0,0E0,3.630752E-2,1.5729547E-2,1.3263035E-1,2.3032904E-1,3.0065358E-2,0E0,4.5308962E-2,1.4328212E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,54,54,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,70,70,71,71,72,72,73,73,74,74,76,76,77,77,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,91,91,92,92,95,95,96,96,97,97,98,98,99,99,101,101,102,102,104,104,105,105,106,106,107,107,108,108,110,110,111,111,112,112,113,113,114,114,115,115,120,120,121,121,122,122,123,123,125,125,126,126,128,128,129,129,130,130,131,131,132,132,133,133,138,138,139,139,140,140,142,142,143,143,145,145,147,147,148,148,149,149,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,161,161,162,162,165,165,167,167,168,168,174,174,177,177,179,179,181,181,182,182,183,183,184,184,187,187,188,188,191,191,192,192,195,195,199,199,201,201,202,202,206,206,208,208,209,209,211,211,213,213,219,219,221,221,225,225,226,226,227,227,228,228,229,229,230,230,231,231,232,232,233,233,234,234,235,235,236,236,237,237,238,238,239,239,240,240,241,241,242,242,243,243,246,246,250,250,253,253,254,254,255,255,256,256,257,257,259,259,260,260],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,84,86,88,90,92,94,-1,96,98,-1,100,-1,-1,102,104,106,108,110,112,114,-1,116,118,120,122,-1,124,126,128,130,132,-1,134,136,-1,-1,138,140,142,144,146,-1,-1,148,150,152,-1,154,156,-1,-1,158,160,162,164,166,-1,168,170,-1,172,174,176,178,180,-1,182,184,186,188,190,192,-1,-1,-1,-1,194,196,198,200,-1,202,204,-1,206,208,210,212,214,216,-1,-1,-1,-1,218,220,222,-1,224,226,-1,228,-1,230,232,234,-1,236,238,240,242,244,246,248,250,-1,-1,252,254,-1,-1,256,-1,258,260,-1,-1,-1,-1,-1,262,-1,-1,264,-1,266,-1,268,270,272,274,-1,-1,276,278,-1,-1,280,282,-1,-1,284,-1,-1,-1,286,-1,288,290,-1,-1,-1,292,-1,294,296,-1,298,-1,300,-1,-1,-1,-1,-1,302,-1,304,-1,-1,-1,306,308,310,312,314,316,318,320,322,324,326,328,330,332,334,336,338,340,342,-1,-1,344,-1,-1,-1,346,-1,-1,348,350,352,354,356,-1,358,360,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.631E3,1.0122174E3,6.860185E2,7.733016E5,2.8839298E7,3E0,2.66E2,1.8329959E-1,3.767388E6,6.76E2,4.898012E-1,1.6649964E5,3.2343243E1,-3.6975067E-2,2.0339169E-2,2.7473561E-2,8.229907E7,3.002239E5,2.0521326E0,1.0376E4,9.397667E6,7.805608E4,8E0,9.99E2,3.0379269E1,5.3473564E1,1.059448E7,3.906E3,9.256843E3,1E1,1.3983857E10,3.5985E4,9.222717E-1,7.0025E4,1.0317661E5,2.437E3,1.2690893E0,-9.182159E-3,1.4E2,5.311576E6,2.1944155E3,1.2E1,5.52E2,7.425E3,5.5544252E7,4.4E1,4.7772845E6,1.6709303E1,-1.1245266E-3,1.79E2,9.3058E4,3.2901205E-2,1.7314173E1,-1.6611993E-2,2.4739714E-4,5.5544252E7,1.2695038E-3,1.2513207E2,7.6599895E9,4.36E2,1.7243155E12,1.9775333E1,1.022669E-2,3.94E2,4.956641E-1,1.9670312E2,3.730007E6,-2.155242E-2,1.6956521E0,9.705292E6,5.48744E8,1.7609E4,2.1924436E5,6.6095283E-3,4.838871E8,6.587855E1,1.6561845E-2,-4.609742E-3,4.602015E0,4.7E2,1.6344065E4,5.024605E0,2E0,1.3348433E-2,2.4456488E-2,2.2E1,1.13808104E8,1E0,-1.0217562E-2,2E0,5.9570195E6,-1.5582514E-2,-6.323576E-3,1.1925198E3,1.6108601E7,8.2654175E5,2.3033286E5,1.5005797E4,2.7516734E-2,3.5905025E5,9.74026E0,2.1125209E-2,9.213145E0,5.7616882E1,1.5497989E5,1.75E2,8.230099E4,-2.4149744E-2,1E0,2.2982106E0,5.8965025E5,3.388159E7,3.7284137E2,6.08767E3,-2.0515336E-2,-1.0380232E-2,-2.621715E-2,-6.210844E-3,8.01E2,1E0,8.286065E2,1.4811083E0,-1.570704E-2,5.4744527E-2,3.8773947E0,7.5178347E-3,2.1592189E5,7.9240966E0,3.7267575E-1,2.74872E5,6.3300834E0,1.8738E4,2.5661536E-3,1.103785E-2,4.175568E-3,-3.399548E-4,9.837297E0,3.09627E5,1.482044E6,-2.5681742E-2,2.8331464E2,1E0,-1.3189506E-2,5.85E2,-5.3373105E-3,9.662076E2,4.28586E5,3.5192856E1,-1.567183E-2,2.3314082E6,1.9826213E0,7.549744E7,1.4521204E0,4.9766545E6,3.7990784E7,1.6672761E6,3.271242E0,-1.4243937E-2,2.9115968E-3,3.61E2,2.3761498E9,2.8935036E-2,1.5316867E-2,5.7417827E0,1.3411217E-3,1.475931E9,1.066E3,-6.4587938E-3,1.2580372E-3,1.2670864E-2,-3.6114997E-3,-1.1040944E-2,2.862988E5,-7.1754875E-3,-2.409997E-2,8.84E2,-5.441774E-3,7.856E3,-2.312226E-3,3.6628513E10,1.8302105E6,2.0802219E8,1.1485E4,-1.571914E-2,-4.0082852E-3,1.5476604E0,2.9283184E3,-1.2587613E-2,-4.1647316E-3,1.8E1,1.3498145E5,-2.0144755E-2,-9.394616E-3,8.45E2,1.5929868E-3,6.254933E-3,-3.304352E-3,5.7704294E5,-4.7742096E-3,1.636099E0,1.2E1,-1.9106425E-3,4.0511293E-3,-5.277993E-3,8.397659E6,1.2815022E-3,3.5E1,2.1262943E1,-1.1511102E-2,2.241E2,-4.99165E-4,5.2773216E1,-3.184633E-3,-6.0201937E-4,-5.4550096E-3,9.598328E-3,4.2054392E-3,7E0,-1.5830977E-2,2.5378451E0,-1.7570151E-2,-5.7883533E-3,-1.3802725E-2,3E1,1.3E1,1.6633E4,2.4545714E2,1.9266E4,3.830216E5,1.0573252E8,3.307766E6,5.704E3,9E0,2.298425E6,5.08E2,1.242E3,1.5859042E6,1.0604491E0,2.0295714E7,1.057101E6,1.01E2,8E0,1.1121472E-2,7.8856135E-3,2E1,2.8904986E-3,-4.296407E-3,3.6082002E-3,4.7013435E1,7.426255E-3,1.6228E-2,1.8140173E-1,1.1693485E0,9.170864E1,5.3515047E-1,1.5161313E1,1.9007912E-2,7.6599895E9,2.3887E4,8.7733666E-4,-2.400014E-3,8.009161E-3,1.4154699E-3,-1.0661325E-2,-3.5737106E-3,-9.054313E-3,-1.50050735E-2,-5.82045E-3,-1.1164623E-4,-1.846646E-2,-3.714133E-2,-4.4178334E-3,-2.2949282E-2,1.0483824E-2,7.6531124E-4,-5.122479E-3,2.2646338E-3,-4.0926514E-3,-1.5293536E-2,-4.9765357E-3,1.7366741E-3,-9.090137E-3,-3.9488473E-3,-4.724151E-3,-1.0983287E-2,-8.324311E-3,-3.539858E-3,-2.4503737E-4,3.372363E-3,4.78478E-3,9.0333796E-4,-7.2021144E-3,-3.049183E-3,-5.9173135E-3,-1.3901812E-3,-1.9583597E-3,-6.351731E-3,5.250443E-3,-4.2804726E-4,-2.008671E-3,-8.790313E-3,7.1127038E-3,-2.9026007E-3,4.6544994E-4,-3.575706E-3,-1.0140852E-2,-4.6835095E-3,4.290685E-3,1.1175588E-2,4.240684E-3,-1.1587675E-3,3.9814557E-3,-2.5952375E-3,6.088019E-3,2.2485606E-2,-1.1337061E-2,-3.1645806E-3,-1.7881347E-4,5.8097956E-3,-1.1262475E-2,-5.6610666E-3,-5.556142E-3,-4.7961218E-4,2.885474E-3,-1.4437075E-2,2.884086E-3,8.601559E-3,-1.9265467E-3,-1.4617842E-2,-3.6375455E-3,4.751528E-3,5.919107E-3,-7.85798E-3,1.1084044E-2,-1.259158E-3,-3.4436495E-3,5.952073E-3,3.2386337E-3,1.1756853E-2,2.0518044E-3,7.4742637E-3,2.28147E-2,9.537982E-3,1.0312905E-2,4.970015E-3,5.558323E-3,8.820614E-4,5.471686E-3,1.1315718E-2,2.0812908E-2,1.45429745E-2,9.761671E-3,1.7239287E-2,9.633885E-4,7.0609003E-3,2.1218492E-3,-3.3879918E-3,7.471674E-3,2.4475798E-3],"split_indices":[20,102,2,52,52,28,45,3,0,41,32,2,27,28,56,0,27,0,45,33,53,9,45,28,18,2,56,56,12,0,4,8,5,2,27,1,28,2,42,0,10,1,4,8,2,1,45,3,28,58,0,3,2,0,35,0,0,45,27,52,5,2,31,57,0,8,57,4,9,0,53,9,5,9,28,0,7,58,0,0,53,1,28,53,17,0,0,3,45,105,0,6,45,0,0,52,32,47,33,4,0,28,53,0,53,50,28,8,32,0,59,53,47,45,58,4,0,0,0,0,2,74,4,53,0,57,54,0,28,54,39,29,53,9,0,0,0,0,54,9,12,0,4,105,0,2,0,52,1,58,0,28,56,7,38,45,7,48,53,0,0,0,12,0,0,54,0,12,8,0,0,0,0,0,28,0,0,10,0,0,0,31,32,7,9,0,0,53,52,0,0,3,33,0,0,10,0,0,0,33,0,53,3,0,0,0,9,0,3,58,0,4,0,58,0,0,0,0,0,0,0,54,0,0,0,0,3,1,52,29,28,7,1,2,8,28,2,29,28,38,50,9,0,8,0,0,3,0,0,0,56,0,0,38,42,58,35,53,0,5,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.542E3,8.03E2,1.739E3,4.42E2,3.61E2,1.412E3,3.27E2,3.21E2,1.21E2,2.77E2,8.4E1,9.41E2,4.71E2,2.49E2,7.8E1,7E0,3.14E2,5E0,1.16E2,1.22E2,1.55E2,2.6E1,5.8E1,2.62E2,6.79E2,4.54E2,1.7E1,9.6E1,1.53E2,1E1,6.8E1,5.3E1,2.61E2,6.6E1,5E1,9.1E1,3.1E1,7.1E1,8.4E1,4E0,2.2E1,2.8E1,3E1,1.24E2,1.38E2,1.7E1,6.62E2,1.65E2,2.89E2,1.1E1,6E0,4.2E1,5.4E1,8E0,1.45E2,5E0,5E0,4.6E1,2.2E1,2.2E1,3.1E1,2.31E2,3E1,6E1,6E0,4.1E1,9E0,1.3E1,7.8E1,4E0,2.7E1,4.2E1,2.9E1,4.7E1,3.7E1,4E0,1.8E1,1.9E1,9E0,6E0,2.4E1,1.09E2,1.5E1,1.12E2,2.6E1,7E0,1E1,6.12E2,5E1,1.61E2,4E0,2.39E2,5E1,6E0,5E0,3.3E1,9E0,4.2E1,1.2E1,1.35E2,1E1,3.3E1,1.3E1,1E1,1.2E1,1.3E1,9E0,1.7E1,1.4E1,1.2E1,2.19E2,2E1,1E1,4.9E1,1.1E1,3.7E1,4E0,5E0,4E0,4E0,9E0,6.8E1,1E1,2.1E1,6E0,3E1,1.2E1,8E0,2.1E1,2.1E1,2.6E1,1.5E1,2.2E1,1.2E1,6E0,1.2E1,7E0,4E0,2E1,6.2E1,4.7E1,4E0,1.1E1,1.07E2,5E0,1.9E1,7E0,2.66E2,3.46E2,3.4E1,1.6E1,1.34E2,2.7E1,2.06E2,3.3E1,1.9E1,3.1E1,1E1,2.3E1,4E0,5E0,1E1,3.2E1,4E0,8E0,1.29E2,6E0,1.2E1,2.1E1,8E0,5E0,7E0,5E0,4E0,9E0,5E0,4E0,1.2E1,5E0,1E1,4E0,2.02E2,1.7E1,1.2E1,8E0,5E0,5E0,1.9E1,3E1,7E0,4E0,2.5E1,1.2E1,5E0,4E0,6.1E1,7E0,6E0,4E0,1.4E1,7E0,2E1,1E1,5E0,7E0,4E0,1.7E1,6E0,1.5E1,1E1,1.6E1,1.1E1,4E0,1.4E1,8E0,7E0,5E0,1.6E1,4E0,5.8E1,4E0,4.3E1,4E0,7E0,4E0,8.7E1,2E1,8E0,1.1E1,2.44E2,2.2E1,8.8E1,2.58E2,2.4E1,1E1,8.2E1,5.2E1,1.1E1,1.6E1,5.7E1,1.49E2,2.3E1,1E1,9E0,1E1,6E0,2.5E1,4E0,6E0,7E0,1.6E1,4E0,6E0,1.8E1,1.4E1,6.8E1,6.1E1,8E0,4E0,1.3E1,8E0,4E0,5E0,5E0,7E0,6E0,4E0,7.5E1,1.27E2,1E1,7E0,4E0,8E0,4E0,4E0,4E0,1.5E1,2E1,1E1,1.2E1,1.3E1,4E0,8E0,2.8E1,3.3E1,4E0,1E1,1.2E1,8E0,6E0,4E0,9E0,8E0,7E0,8E0,6E0,4E0,4E0,7E0,8E0,6E0,9E0,4.9E1,4E0,3.9E1,2.5E1,6.2E1,7E0,1.3E1,4E0,4E0,4E0,7E0,1.97E2,4.7E1,1.3E1,9E0,5.5E1,3.3E1,2.1E2,4.8E1,1.6E1,8E0,6E0,4E0,7.8E1,4E0,1.9E1,3.3E1,7E0,4E0,4E0,1.2E1,5.1E1,6E0,1.39E2,1E1,1.8E1,5E0,5E0,5E0,4E0,5E0,2.1E1,4E0,1E1,6E0,1.1E1,7E0,5E0,9E0,4E1,2.8E1,4.4E1,1.7E1,4E0,4E0,7E0,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"361","size_leaf_vector":"1"}},{"base_weights":[1.0617484E-3,-1.2635158E-1,6.0345255E-2,-1.8058056E-1,-1.5803419E-2,2.5846247E-2,2.119327E-1,-1.9051827E-1,4.3133086E-1,-8.023338E-2,8.1892446E-2,-2.1637743E-2,1.1839626E-1,2.5975358E-1,2.5337182E-2,-2.3833896E-1,-1.0567113E-1,3.6737442E-2,2.0564552E-3,-1.3517936E-1,-2.0794207E-2,5.1449686E-2,2.8044432E-1,-5.8133908E-2,4.9930647E-2,8.7436356E-2,2.2608343E-1,1.7125529E-1,3.2275063E-1,7.0613064E-2,-2.232193E-2,-1.7961966E-1,-3.055841E-1,-1.5353447E-1,1.5874201E-2,-1.794004E-2,-1.691733E-1,-8.7753765E-2,6.5890774E-2,-3.5580914E-2,9.4839044E-2,1.6240647E-2,6.7547397E-3,-9.412333E-2,1.2082027E-2,3.2694433E-2,1.5113838E-1,9.5655315E-2,-2.2294328E-1,3.5827193E-1,1.4809163E-1,5.2713804E-2,2.28992E-1,4.1235352E-1,2.6275373E-1,-5.3168222E-2,1.4497557E-1,-1.4447303E-1,-3.3146283E-1,-2.7638605E-1,-4.735182E-1,-1.6358453E-1,3.0372678E-3,-1.06151225E-2,1.0430928E-1,4.5354787E-2,-1.0033981E-1,2.1998151E-3,-2.0037206E-1,2.2751864E-3,-1.1350505E-1,-2.6006834E-3,9.208675E-2,4.224531E-2,-7.2876714E-2,6.383957E-2,1.8191004E-1,-1.10147424E-1,5.7654392E-2,-4.7130536E-2,8.014415E-2,9.385044E-2,-1.005157E-3,-2.2547124E-3,1.8752205E-1,1.0231103E-1,-1.8046218E-1,-1.4059174E-2,-5.4084556E-3,3.953814E-1,4.4107046E-3,-9.799101E-2,1.8197118E-1,-2.661907E-2,1.585019E-1,1.6008864E-1,3.9198655E-1,4.567206E-1,2.0421967E-1,2.1820118E-1,4.7690997E-1,-5.0270488E-3,-5.2877585E-3,1.9928558E-2,6.659722E-2,-2.0316888E-1,-5.8328584E-2,-1.279501E-1,-3.9960942E-1,-3.1164894E-1,-1.509407E-1,-5.4270023E-1,-6.483642E-3,-1.5076075E-1,-1.7324131E-2,9.810287E-3,-8.589039E-3,3.5944432E-2,1.0318324E-2,-9.3840866E-4,3.985847E-3,-7.2479877E-3,-1.7765751E-3,-6.824136E-3,5.04711E-3,-3.6309648E-1,-1.5614225E-1,-1.4699066E-2,-1.4366211E-1,3.649802E-3,1.2560321E-1,-1.3120549E-3,6.480763E-3,-4.20607E-2,-7.3971977E-3,9.899533E-2,-1.2659153E-2,2.3637752E-1,7.398081E-4,-1.7452435E-1,-7.66008E-2,1.5878841E-1,-2.4000693E-2,1.3107208E-2,-1.2730524E-1,2.2419319E-1,3.607568E-2,5.2702963E-2,2.0418768E-1,5.164215E-2,-5.132725E-2,4.5238253E-2,2.350255E-1,7.496776E-2,1.9673361E-1,4.3382067E-3,-2.0568974E-2,9.726174E-3,4.4021264E-1,3.1827795E-4,-9.131589E-3,2.3476517E-1,1.0187291E-1,6.760669E-2,-9.991583E-3,2.5164192E-3,1.0910205E-2,2.1136448E-1,1.137317E-1,3.314252E-2,2.8913262E-1,4.8730204E-1,8.320971E-3,1.231522E-2,4.785671E-3,9.96295E-4,2.3150763E-1,2.8351368E-2,1.4611671E-2,-5.2299257E-2,4.016845E-3,1.4900047E-1,-5.896115E-2,-1.4904457E-1,-3.1769258E-1,3.084372E-2,-1.4567812E-1,3.4432255E-3,-1.3524755E-2,-4.74224E-1,-4.2118146E-3,-3.23276E-1,-6.059208E-4,-1.5638056E-3,-1.8318906E-1,-9.589058E-3,-3.143995E-2,-1.7952271E-1,-6.680828E-2,7.707641E-2,-1.5180969E-2,3.161702E-3,7.114016E-5,-2.1832574E-2,-9.34838E-3,-2.335475E-1,-1.05382554E-1,-2.9200916E-3,2.0351834E-3,-7.192615E-2,-1.72347E-1,3.7235566E-3,-3.3952936E-3,1.0782474E-2,7.514094E-2,-6.940845E-3,-1.5757204E-3,-2.8083597E-3,1.2234032E-1,-5.027781E-2,4.005201E-3,1.5474194E-2,5.568768E-3,-1.364455E-1,-3.1719807E-1,1.3085803E-1,-1.0055273E-1,-3.4504366E-4,2.4221082E-1,3.6964692E-2,-7.1418043E-3,1.8884313E-1,-2.9442314E-2,-2.685079E-1,-5.070674E-2,7.8265466E-2,2.9680362E-1,-1.9511428E-2,2.0678955E-1,8.228239E-2,-8.787152E-2,2.3816818E-1,3.8939456E-3,8.870308E-2,-3.4000773E-2,-2.5328022E-1,-1.5099403E-3,7.033022E-3,-4.873341E-3,3.0353913E-1,7.4495666E-2,6.2849715E-2,2.2935982E-1,2.2323076E-2,2.3998743E-1,2.3525437E-2,1.2721873E-2,1.6688256E-3,2.8221917E-1,1.7841503E-1,-1.8722266E-2,2.6082282E-5,9.1953665E-2,1.4961332E-2,1.6537575E-1,6.458377E-2,1.6528763E-1,1.7919455E-2,2.5291946E-3,2.1457257E-2,3.2567255E-2,2.4543117E-1,3.2561745E-3,6.456836E-6,-4.710252E-3,5.8880527E-2,1.4744179E-2,3.0323675E-2,-1.0387151E-2,-1.0190219E-2,-3.4125682E-3,-1.861942E-2,-2.7044069E-3,-1.150713E-3,1.3397718E-2,-5.3094723E-3,-1.3554759E-2,-2.9053353E-2,-1.2026786E-2,-1.7363299E-2,-9.5404545E-3,-7.541992E-4,-1.071727E-2,-1.0064496E-2,-3.421386E-3,-6.588095E-3,7.2971446E-4,5.4751714E-3,8.075131E-4,2.302467E-3,-1.7902611E-3,-6.015923E-3,-1.3523958E-2,-2.381745E-3,-7.4599003E-3,-6.4132814E-3,5.08118E-4,-9.418134E-3,-4.308391E-3,-8.3873275E-4,7.878499E-3,-1.2565411E-3,2.0570671E-3,9.010299E-3,4.113785E-3,-4.6565263E-3,-8.579932E-4,-1.2923899E-2,-6.0135364E-3,-2.2339119E-2,-7.6411488E-3,-3.4748418E-3,1.5369641E-2,-8.165522E-3,-3.2318144E-3,1.5454766E-2,4.318296E-3,-1.9532447E-3,4.322475E-3,2.4162885E-3,1.35210585E-2,-4.074923E-3,3.5397883E-3,-2.2180488E-2,-6.6196495E-3,-1.3406953E-3,-6.094033E-3,6.6749523E-3,3.642439E-4,5.8052144E-3,1.7584385E-2,8.626411E-3,-2.0236815E-3,5.953864E-3,1.453629E-2,8.12274E-4,8.113529E-3,-4.730524E-4,-8.882333E-3,4.586492E-3,1.4032695E-2,7.4697315E-4,6.970631E-3,-5.18104E-3,2.4496275E-3,-8.020157E-3,-1.7447315E-2,-1.0357671E-3,6.975286E-3,1.7809872E-2,7.7140788E-3,4.487221E-4,5.8298986E-3,3.4046674E-3,-8.658793E-3,3.2843505E-3,1.6637312E-2,4.032607E-3,-3.628102E-3,1.5058986E-2,6.251226E-3,1.7852603E-2,1.081743E-2,1.1806848E-2,5.998894E-3,3.2484496E-3,-6.6059064E-3,6.2897573E-3,1.3025005E-3,9.550602E-3,4.87596E-3,4.910469E-3,-1.7952988E-3,3.4799823E-3,9.82243E-3,1.4108761E-2,7.549261E-3,-4.729563E-4,5.1355707E-3,-2.6555262E-3,5.3828373E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,115,117,119,121,123,125,-1,127,-1,129,131,133,135,137,139,141,143,145,147,149,-1,151,153,155,-1,-1,157,-1,159,161,163,165,167,169,171,173,175,177,-1,179,-1,181,183,185,187,189,191,193,195,-1,197,-1,199,-1,201,-1,-1,-1,-1,-1,-1,-1,203,205,207,209,211,213,-1,-1,215,-1,217,219,221,-1,223,225,227,229,231,233,235,237,239,241,243,245,247,249,251,253,-1,-1,-1,255,-1,-1,257,259,261,-1,-1,-1,263,265,-1,267,269,-1,-1,-1,-1,271,-1,-1,273,-1,275,277,279,281,283,285,-1,-1,287,-1,289,-1,-1,291,-1,-1,293,295,297,299,-1,-1,-1,-1,301,303,-1,-1,305,307,-1,-1,-1,309,-1,311,-1,313,315,-1,-1,-1,317,319,321,323,-1,325,327,-1,329,331,333,335,337,339,341,343,345,347,349,-1,351,353,355,357,-1,-1,359,361,363,365,367,369,-1,-1,-1,371,373,375,-1,377,-1,379,381,383,-1,-1,-1,-1,385,-1,-1,-1,387,-1,389,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9291735E1,4.859021E0,9.109745E0,3.3902454E0,1.6935076E0,6.247351E0,2.875599E0,2.1496086E0,1.0369347E0,5.262718E-1,6.323291E-1,2.458156E0,1.587213E0,1.3775368E0,1.4635732E0,1.2955265E0,1.1368625E0,0E0,0E0,3.3174098E-1,4.6432886E-1,3.5833743E-1,4.6730995E-2,1.5754778E0,5.4978997E-1,9.805014E-1,1.0500717E0,7.3394585E-1,6.926613E-1,5.783987E-1,0E0,9.3941545E-1,6.4385605E-1,3.114941E-1,1.3229205E-1,1.0886048E-1,3.4797156E-1,1.5962335E-1,1.1292653E-1,9.5961615E-2,1.5843368E-1,0E0,0E0,1.0083756E0,8.581653E-1,5.627289E-1,3.369366E-1,6.9292116E-1,3.6561877E-2,3.5585928E-1,5.930959E-1,3.1474066E-1,7.5710106E-1,4.4231796E-1,7.737284E-1,5.3982586E-2,7.4943084E-1,7.567103E-1,4.2040586E-1,5.589876E-1,4.5278215E-1,2.6241565E-1,0E0,1.466318E-1,8.3402306E-2,2.7985059E-2,2.0915039E-2,1.6462925E-1,3.3915496E-1,0E0,1.111168E-1,0E0,8.5500285E-2,6.930796E-2,4.4732153E-2,1.3034299E-1,1.4212793E-1,7.921729E-1,3.3413535E-1,5.5223656E-1,6.203661E-1,4.296283E-1,4.7158077E-1,0E0,2.5296772E-1,9.121258E-1,5.9188586E-1,0E0,0E0,2.2027397E-1,0E0,8.077717E-2,2.3403478E-1,3.7285927E-1,9.255442E-2,1.01941586E-1,3.942232E-1,3.35948E-1,3.1372964E-2,2.0116019E-1,8.449626E-2,0E0,5.656658E-2,0E0,3.295242E-1,5.105164E-1,4.9521023E-1,2.9962248E-1,5.4820776E-1,3.6523342E-1,1.1773276E-1,4.2554426E-1,0E0,2.9466534E-1,0E0,6.841269E-2,0E0,8.401517E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1221242E-2,1.5219605E-1,2.6626334E-2,4.7278166E-2,5.0733674E-2,7.936025E-2,0E0,0E0,6.643679E-2,0E0,1.2643978E-1,5.9896328E-2,7.5390875E-2,0E0,6.4669156E-1,1.2405111E0,2.5047907E-1,1.7204845E-1,4.9907365E-1,5.1636446E-1,2.2047567E-1,7.347256E-1,3.0661434E-1,9.2986465E-2,2.7915454E-1,9.151463E-1,1.6373318E-1,2.9634368E-1,5.094451E-1,5.981567E-1,0E0,0E0,0E0,3.352213E-2,0E0,0E0,3.3457732E-1,2.433362E-1,2.445165E-2,0E0,0E0,0E0,4.767859E-2,6.5341175E-2,0E0,2.6363456E-1,1.1987877E-1,0E0,0E0,0E0,0E0,1.5307808E-1,0E0,0E0,1.9815343E-2,0E0,2.3275495E-1,1.7915611E-1,2.6533186E-1,3.8531518E-1,4.1524923E-1,1.2959963E-1,0E0,0E0,3.6087275E-1,0E0,2.6060867E-1,0E0,0E0,1.2829411E-1,0E0,0E0,2.1482944E-1,1.8196164E-1,2.0006426E-2,3.9598793E-2,0E0,0E0,0E0,0E0,3.8799107E-2,6.638664E-2,0E0,0E0,4.7627524E-2,1.154995E-2,0E0,0E0,0E0,1.16453215E-1,0E0,1.4133677E-2,0E0,5.243367E-2,1.5448879E-2,0E0,0E0,0E0,1.6948831E-1,4.8637128E-1,9.422824E-1,4.9009252E-1,0E0,9.869164E-2,6.366941E-2,0E0,1.3623023E-1,2.879771E-1,3.4678984E-1,5.286763E-2,3.4365263E-2,1.0844755E-1,2.4494946E-1,1.05413735E-1,3.1466755E-1,8.9941494E-2,9.914827E-2,0E0,2.2647378E-1,1.6130576E-1,1.0409105E-1,2.0358177E-1,0E0,0E0,1.0276544E-1,2.6586585E-2,3.3778715E-1,3.171805E-1,1.0008245E-1,4.136753E-1,0E0,0E0,0E0,6.573939E-2,2.5673807E-2,1.1312293E-1,0E0,2.1306053E-2,0E0,1.2785882E-2,6.1100155E-2,2.8208405E-2,0E0,0E0,0E0,0E0,1.9079256E-1,0E0,0E0,0E0,3.9977368E-2,0E0,6.3873954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,84,84,85,85,86,86,89,89,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,102,102,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,113,113,115,115,117,117,125,125,126,126,127,127,128,128,129,129,130,130,133,133,135,135,136,136,137,137,139,139,140,140,141,141,142,142,143,143,144,144,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,158,158,161,161,162,162,163,163,167,167,168,168,170,170,171,171,176,176,179,179,181,181,182,182,183,183,184,184,185,185,186,186,189,189,191,191,194,194,197,197,198,198,199,199,200,200,205,205,206,206,209,209,210,210,214,214,216,216,218,218,219,219,223,223,224,224,225,225,226,226,228,228,229,229,231,231,232,232,233,233,234,234,235,235,236,236,237,237,238,238,239,239,240,240,241,241,243,243,244,244,245,245,246,246,249,249,250,250,251,251,252,252,253,253,254,254,258,258,259,259,260,260,262,262,264,264,265,265,266,266,271,271,275,275,277,277],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,116,118,120,122,124,126,-1,128,-1,130,132,134,136,138,140,142,144,146,148,150,-1,152,154,156,-1,-1,158,-1,160,162,164,166,168,170,172,174,176,178,-1,180,-1,182,184,186,188,190,192,194,196,-1,198,-1,200,-1,202,-1,-1,-1,-1,-1,-1,-1,204,206,208,210,212,214,-1,-1,216,-1,218,220,222,-1,224,226,228,230,232,234,236,238,240,242,244,246,248,250,252,254,-1,-1,-1,256,-1,-1,258,260,262,-1,-1,-1,264,266,-1,268,270,-1,-1,-1,-1,272,-1,-1,274,-1,276,278,280,282,284,286,-1,-1,288,-1,290,-1,-1,292,-1,-1,294,296,298,300,-1,-1,-1,-1,302,304,-1,-1,306,308,-1,-1,-1,310,-1,312,-1,314,316,-1,-1,-1,318,320,322,324,-1,326,328,-1,330,332,334,336,338,340,342,344,346,348,350,-1,352,354,356,358,-1,-1,360,362,364,366,368,370,-1,-1,-1,372,374,376,-1,378,-1,380,382,384,-1,-1,-1,-1,386,-1,-1,-1,388,-1,390,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.0778715E6,8.631E3,8.230524E3,8.499824E-1,7.733016E5,3.5301748E7,1E0,9.773983E6,1.3528846E1,2.2720945E0,1.645614E6,3.647E3,1.6649964E5,1.2367184E3,6.748543E-1,3.4197716E5,3.6737442E-2,2.0564552E-3,5.1792985E6,3.6482175E5,9.498317E0,1.5137865E2,1.7247218E-1,1.4122984E5,4.898012E-1,1.0215407E1,5.2610065E1,5.473125E3,1.6173E4,-2.232193E-2,2.72258E5,1.2427474E8,1E0,3.6873734E5,6.731367E0,2.7394975E-2,1.0979587E8,8.17766E5,5.18E2,7.105352E5,1.6240647E-2,6.7547397E-3,3.524269E6,1E0,1.0177216E1,6.866557E-1,3.1984328E5,1.6709303E1,8.962557E8,1.5139E4,3.3487454E11,9.3058E4,2.1270284E7,6.935545E0,1E0,2.5927516E11,2.914E3,1.6621418E0,7.621583E2,3.995065E6,2.462277E6,3.0372678E-3,5.534958E-1,2.4991455E9,8.891209E6,3.5E1,2.253E3,2.222456E0,2.2751864E-3,1E0,-2.6006834E-3,1.945711E1,1.151E4,2.2182236E0,1.5211788E-1,3E1,1.2939E4,1.1805985E3,1.242E3,1.5851064E0,1.3195207E7,3.7307863E0,-2.2547124E-3,5.674727E2,5.314225E2,1.20086E5,-1.4059174E-2,-5.4084556E-3,5.4529045E6,4.4107046E-3,3.09E2,2.9375E0,1.51081E5,6.927432E0,2.5570776E-2,1.417988E6,1.694761E9,2.4708056E5,5.4091E4,4.456E3,-5.0270488E-3,1.4972717E-2,1.9928558E-2,1.5902449E0,1.2936076E7,3.653602E7,6.6911E4,1.819149E1,6.3251557E0,1E0,1.0804272E3,-6.483642E-3,5.828063E5,-1.7324131E-2,2.3887E4,-8.589039E-3,6.9740294E5,1.0318324E-2,-9.3840866E-4,3.985847E-3,-7.2479877E-3,-1.7765751E-3,-6.824136E-3,5.04711E-3,1E0,8.088206E-2,1.3683E4,7.0229135E6,7.2123044E5,3E0,-1.3120549E-3,6.480763E-3,1.5825138E5,-7.3971977E-3,1.2091255E0,3.0927758E0,2.3887E4,7.398081E-4,5E0,5.37798E5,1.650267E-6,3E1,2.264389E-1,2.3E2,1.7642576E8,1.6443022E0,8.397659E6,1.17E2,3.620103E1,2.97271E5,1.169191E0,5.1232258E1,7.780377E6,1E0,4.3382067E-3,-2.0568974E-2,9.726174E-3,2.8600824E0,3.1827795E-4,-9.131589E-3,3.7342235E6,8.349059E6,1.372E4,-9.991583E-3,2.5164192E-3,1.0910205E-2,6.2328663E1,1.2E2,3.314252E-2,3.13E2,5.3534385E3,8.320971E-3,1.231522E-2,4.785671E-3,9.96295E-4,2.8091298E7,2.8351368E-2,1.4611671E-2,3.9382784E9,4.016845E-3,5.57E4,1.56E2,1.146E3,5.3157896E-1,1.2659678E3,1.2323925E6,3.4432255E-3,-1.3524755E-2,4.1808826E-1,-4.2118146E-3,1.4072216E-1,-6.059208E-4,-1.5638056E-3,8.091679E-1,-9.589058E-3,-3.143995E-2,2.437E3,1.1532471E-3,4.38E2,2.7453625E-1,3.161702E-3,7.114016E-5,-2.1832574E-2,-9.34838E-3,5.6600465E10,2.376302E8,-2.9200916E-3,2.0351834E-3,1.0908286E3,7.4709034E-1,3.7235566E-3,-3.3952936E-3,1.0782474E-2,1.747E3,-6.940845E-3,1.022332E6,-2.8083597E-3,5.400855E5,1.49974E6,4.005201E-3,1.5474194E-2,5.568768E-3,6E0,1.7036236E5,2.53E2,8.242408E4,-3.4504366E-4,1.54E2,2.11E2,-7.1418043E-3,2.5207965E2,2.6668E4,5.542E3,1.2739522E0,3.178082E6,3.342857E2,7.5773336E7,2.7919563E1,2.0939131E2,2.3066575E6,1.5147E4,3.8939456E-3,4.564527E6,7.054E3,1.2886906E5,1.5106794E3,7.033022E-3,-4.873341E-3,2.909019E3,6.065707E8,6.6315195E6,1E0,1.1E1,2E0,2.3525437E-2,1.2721873E-2,1.6688256E-3,5.8805E4,5.5E1,1.5E1,2.6082282E-5,3.307766E6,1.4961332E-2,1.9761398E7,6.744351E1,3.7448005E3,1.7919455E-2,2.5291946E-3,2.1457257E-2,3.2567255E-2,1.2817779E7,3.2561745E-3,6.456836E-6,-4.710252E-3,7.2132964E0,1.4744179E-2,3.2282164E7,-1.0387151E-2,-1.0190219E-2,-3.4125682E-3,-1.861942E-2,-2.7044069E-3,-1.150713E-3,1.3397718E-2,-5.3094723E-3,-1.3554759E-2,-2.9053353E-2,-1.2026786E-2,-1.7363299E-2,-9.5404545E-3,-7.541992E-4,-1.071727E-2,-1.0064496E-2,-3.421386E-3,-6.588095E-3,7.2971446E-4,5.4751714E-3,8.075131E-4,2.302467E-3,-1.7902611E-3,-6.015923E-3,-1.3523958E-2,-2.381745E-3,-7.4599003E-3,-6.4132814E-3,5.08118E-4,-9.418134E-3,-4.308391E-3,-8.3873275E-4,7.878499E-3,-1.2565411E-3,2.0570671E-3,9.010299E-3,4.113785E-3,-4.6565263E-3,-8.579932E-4,-1.2923899E-2,-6.0135364E-3,-2.2339119E-2,-7.6411488E-3,-3.4748418E-3,1.5369641E-2,-8.165522E-3,-3.2318144E-3,1.5454766E-2,4.318296E-3,-1.9532447E-3,4.322475E-3,2.4162885E-3,1.35210585E-2,-4.074923E-3,3.5397883E-3,-2.2180488E-2,-6.6196495E-3,-1.3406953E-3,-6.094033E-3,6.6749523E-3,3.642439E-4,5.8052144E-3,1.7584385E-2,8.626411E-3,-2.0236815E-3,5.953864E-3,1.453629E-2,8.12274E-4,8.113529E-3,-4.730524E-4,-8.882333E-3,4.586492E-3,1.4032695E-2,7.4697315E-4,6.970631E-3,-5.18104E-3,2.4496275E-3,-8.020157E-3,-1.7447315E-2,-1.0357671E-3,6.975286E-3,1.7809872E-2,7.7140788E-3,4.487221E-4,5.8298986E-3,3.4046674E-3,-8.658793E-3,3.2843505E-3,1.6637312E-2,4.032607E-3,-3.628102E-3,1.5058986E-2,6.251226E-3,1.7852603E-2,1.081743E-2,1.1806848E-2,5.998894E-3,3.2484496E-3,-6.6059064E-3,6.2897573E-3,1.3025005E-3,9.550602E-3,4.87596E-3,4.910469E-3,-1.7952988E-3,3.4799823E-3,9.82243E-3,1.4108761E-2,7.549261E-3,-4.729563E-4,5.1355707E-3,-2.6555262E-3,5.3828373E-3],"split_indices":[20,32,2,52,39,28,45,102,1,56,35,9,2,28,58,27,47,0,0,45,28,56,56,41,33,27,56,56,32,2,0,9,45,89,28,54,41,7,1,0,28,0,0,1,100,58,41,33,58,12,9,31,2,51,35,102,31,2,58,52,29,28,0,27,12,32,3,2,54,0,67,0,56,29,57,27,3,9,33,29,53,9,53,0,52,4,9,0,0,45,0,0,54,11,54,57,29,7,33,33,0,0,38,0,35,45,45,29,56,54,67,33,0,28,0,9,0,47,0,0,0,0,0,0,0,8,39,9,32,47,8,0,0,28,0,53,42,9,0,8,5,38,8,38,0,7,41,9,3,58,1,38,56,28,102,0,0,0,53,0,0,45,47,11,0,0,0,56,8,0,3,4,0,0,0,0,45,0,0,5,0,2,8,2,27,52,32,0,0,27,0,38,0,0,27,0,0,2,39,0,39,0,0,0,0,31,7,0,0,4,41,0,0,0,0,0,28,0,33,1,0,0,0,52,28,2,28,0,3,3,0,4,1,29,38,45,52,7,56,52,45,1,0,9,29,47,52,0,0,32,12,28,102,8,17,0,0,0,9,8,3,0,1,0,45,54,4,0,0,0,0,9,0,0,0,54,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.552E3,8.1E2,1.742E3,5.43E2,2.67E2,1.42E3,3.22E2,5.35E2,8E0,1.61E2,1.06E2,9.39E2,4.81E2,2.56E2,6.6E1,3.41E2,1.94E2,4E0,4E0,8.3E1,7.8E1,9.3E1,1.3E1,6.22E2,3.17E2,3.75E2,1.06E2,1.08E2,1.48E2,6.1E1,5E0,1.84E2,1.57E2,1.39E2,5.5E1,1.9E1,6.4E1,4.4E1,3.4E1,3.1E1,6.2E1,9E0,4E0,4.11E2,2.11E2,2.72E2,4.5E1,3.66E2,9E0,3.8E1,6.8E1,3.6E1,7.2E1,5.7E1,9.1E1,2.3E1,3.8E1,1.51E2,3.3E1,1.36E2,2.1E1,1.33E2,6E0,4.3E1,1.2E1,1.1E1,8E0,1E1,5.4E1,7E0,3.7E1,6E0,2.8E1,1E1,2.1E1,4.7E1,1.5E1,3.72E2,3.9E1,1.13E2,9.8E1,9.6E1,1.76E2,7E0,3.8E1,3.58E2,8E0,5E0,4E0,3.3E1,5E0,8E0,6E1,2.1E1,1.5E1,5.2E1,2E1,4.6E1,1.1E1,7.7E1,1.4E1,1.1E1,1.2E1,8E0,3E1,8.9E1,6.2E1,9E0,2.4E1,1.05E2,3.1E1,1.7E1,4E0,1.26E2,7E0,3.9E1,4E0,8E0,4E0,4E0,7E0,4E0,4E0,4E0,6E0,1E1,4.4E1,9E0,2.8E1,8E0,2E1,6E0,4E0,1.6E1,5E0,3.2E1,1.5E1,1.1E1,4E0,1.26E2,2.46E2,1.7E1,2.2E1,6.5E1,4.8E1,2.2E1,7.6E1,7.1E1,2.5E1,8.6E1,9E1,1E1,2.8E1,2.79E2,7.9E1,4E0,4E0,7E0,2.6E1,4E0,4E0,3.5E1,2.5E1,1.4E1,7E0,6E0,9E0,2.3E1,2.9E1,4E0,1.6E1,4.1E1,5E0,7E0,4E0,5E0,7.2E1,8E0,6E0,8E0,4E0,1.8E1,1.2E1,6.2E1,2.7E1,3.1E1,3.1E1,4E0,5E0,1.9E1,5E0,1.01E2,4E0,7E0,2.4E1,4E0,1.3E1,9.3E1,3.3E1,1E1,2.9E1,4E0,4E0,6E0,4E0,1.6E1,2.8E1,5E0,4E0,9E0,1.9E1,4E0,4E0,6E0,1.4E1,4E0,1.2E1,4E0,2.8E1,1.1E1,4E0,6E0,5E0,1.01E2,2.5E1,2.5E1,2.21E2,6E0,1.1E1,1.5E1,7E0,1.2E1,5.3E1,1.6E1,3.2E1,8E0,1.4E1,5.8E1,1.8E1,5.9E1,1.2E1,1.9E1,6E0,6E1,2.6E1,1.7E1,7.3E1,6E0,4E0,1.9E1,9E0,2.6E2,1.9E1,1.6E1,6.3E1,2.1E1,5E0,7E0,2.8E1,1.5E1,1E1,4E0,1E1,6E0,1.7E1,1.6E1,1.3E1,1.2E1,4E0,3.3E1,8E0,6.6E1,6E0,4E0,4E0,1.2E1,6E0,8E0,4E0,3.6E1,2.6E1,2.2E1,5E0,2.6E1,5E0,2.5E1,6E0,1.2E1,7E0,8.4E1,1.7E1,4E0,2E1,7.7E1,1.6E1,1.8E1,1.5E1,6E0,4E0,7E0,2.2E1,5E0,1.1E1,1.3E1,1.5E1,5E0,4E0,1.5E1,4E0,7E0,7E0,8E0,4E0,1E1,1.8E1,4E0,7E0,1E1,9.1E1,1.3E1,1.2E1,1.2E1,1.3E1,7.9E1,1.42E2,7E0,4E0,6E0,9E0,5E0,7E0,3.5E1,1.8E1,6E0,1E1,2.5E1,7E0,4E0,4E0,4E0,1E1,5E0,5.3E1,1E1,8E0,3.3E1,2.6E1,7E0,5E0,5E0,1.4E1,2.5E1,3.5E1,1.4E1,1.2E1,1E1,7E0,6.5E1,8E0,1.3E1,6E0,4E0,5E0,2.55E2,5E0,8E0,1.1E1,1E1,6E0,4E1,2.3E1,1.1E1,1.7E1,6E0,9E0,6E0,4E0,6E0,4E0,1.1E1,6E0,1.2E1,4E0,4E0,9E0,4.6E1,2E1,5E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"391","size_leaf_vector":"1"}},{"base_weights":[2.6729861E-3,-1.2627089E-1,6.4192735E-2,-1.9771206E-1,-3.7444904E-2,9.7195115E-3,1.6156875E-1,-2.0722246E-1,3.5625357E-2,-8.730909E-2,1.11485064E-1,-4.6687014E-2,5.697061E-2,1.1713786E-1,2.55843E-1,-6.1353374E-1,-1.9397396E-1,-1.4600913E-1,-4.6715595E-2,6.09962E-2,2.283161E-1,-7.767216E-2,3.2273147E-2,7.025385E-2,-1.451754E-1,-2.5657147E-2,1.5105361E-1,3.836502E-1,2.07153E-1,-3.907114E-2,-1.6659744E-2,-6.644642E-2,-2.256692E-1,-1.3064422E-1,-2.2726603E-2,-1.1223487E-1,-1.441227E-2,1.2807728E-1,-3.9944276E-3,2.7727628E-1,1.2744342E-1,-1.6502045E-1,-5.3223945E-2,5.0135385E-2,-2.3878902E-2,1.595036E-2,1.3176543E-1,-2.096267E-1,5.625093E-2,-1.5856987E-1,3.109667E-2,-1.2104741E-1,1.6548146E-1,2.1848863E-1,4.3599454E-1,1.7716627E-1,2.1287894E-2,-8.68342E-2,1.4865719E-2,-2.800405E-1,-1.6847104E-1,-2.7438337E-1,-1.05060324E-1,-5.83523E-2,-2.1125698E-1,-6.5206005E-3,3.3222088E-3,1.7059018E-1,-4.244907E-3,-8.633747E-2,7.857053E-2,1.6264575E-2,7.7439137E-3,3.4797555E-3,8.19477E-3,-1.0328026E-1,-2.5440323E-1,8.030252E-2,-7.375904E-2,3.8005853E-1,2.4761396E-2,-1.7219597E-2,9.70189E-2,1.1242558E-1,3.0094376E-1,-1.3157499E-1,-1.4768608E-2,8.3495015E-3,-4.3980586E-3,-9.0386224E-4,-1.9278555E-1,8.016395E-2,-5.5069387E-2,-3.1086696E-2,-1.7424772E-2,1.3672534E-1,2.8241342E-1,1.23808E-2,5.513686E-3,5.0196016E-1,1.8790163E-1,1.9116515E-1,-1.0684795E-2,-1.387322E-1,2.5623497E-2,-2.6493812E-1,-3.0859161E-2,-2.8190324E-1,-1.351933E-1,-8.135023E-3,-1.714224E-2,-5.940808E-2,-1.7501768E-1,-1.1698643E-1,-1.757559E-2,-1.8883985E-2,-1.4161496E-1,-2.6181234E-2,6.888399E-2,1.0669926E-2,8.481003E-2,-2.3022566E-3,1.9204698E-3,-1.0871669E-1,-1.1721972E-3,-2.7560801E-3,1.2071867E-1,-1.4352295E-1,-2.4952956E-3,-6.209554E-2,-3.389933E-1,-1.2094781E-1,2.9851064E-1,-1.1440831E-1,-2.6225615E-2,3.0058114E-2,6.602001E-3,-1.6587034E-2,9.746671E-2,1.6900664E-2,-9.779345E-2,2.427225E-2,1.300129E-1,2.12188E-1,7.001093E-2,2.3625467E-2,1.8873532E-1,-3.1135597E-3,-7.399721E-3,-1.1283823E-2,-3.8999447E-3,1.0991738E-2,1.3563709E-1,-6.108453E-3,8.445474E-4,3.6069567E-3,-3.9780964E-3,1.9020943E-1,9.838155E-2,2.8442536E-2,2.4845895E-1,2.6422104E-2,1.2824549E-2,1.3593658E-2,4.3199817E-3,1.7081296E-1,2.8637497E-2,-1.9487217E-1,-4.238823E-2,6.4546816E-2,-4.0398613E-2,-2.4975507E-1,-2.7514871E-2,-8.996744E-2,-3.540935E-1,-1.1307766E-1,-3.5798427E-1,-8.251061E-2,4.905947E-3,-1.4539029E-1,-1.4051737E-2,-7.299981E-3,-3.6601855E-3,-4.875478E-2,5.2175424E-3,-1.8682455E-3,-8.799926E-3,8.2239546E-2,-4.1748542E-2,-3.4175515E-3,1.0347604E-1,6.406317E-3,7.877692E-4,-6.5970407E-3,-2.2490742E-3,7.1658543E-3,2.5576386E-3,-1.6510366E-1,-5.8455416E-4,-3.479938E-3,3.804417E-2,-7.0514367E-3,1.3631404E-3,-7.6602083E-3,-3.8458908E-1,-1.126104E-2,-5.263825E-2,2.0089973E-2,3.1752042E-3,-8.9296274E-2,-1.739329E-1,-1.0465849E-1,2.7931577E-2,4.246563E-2,-7.002197E-2,1.146776E-1,-3.6716743E-3,4.8757296E-2,-7.828575E-2,-1.1746537E-1,8.082736E-3,7.014868E-2,-5.2426E-3,1.5057516E-1,-4.1757366E-3,8.621208E-2,2.4664162E-1,-2.6213432E-2,1.272755E-1,1.7659228E-3,2.3003756E-1,-2.3897704E-2,5.320642E-3,2.5511757E-3,1.8375853E-1,3.2724682E-3,-1.7570835E-3,2.3061198E-1,7.478224E-2,-1.70742E-2,1.1212067E-1,7.9054444E-4,2.6970646E-1,1.8473805E-1,-4.2145737E-3,2.5062722E-3,-1.1297107E-2,3.9396444E-3,-7.0807734E-3,-3.297989E-4,4.264457E-3,-4.206086E-3,1.4476355E-3,-2.0980868E-2,-1.175015E-2,-7.2615277E-3,-1.4821871E-3,-4.886515E-3,-2.0264404E-2,-8.071993E-4,-7.074985E-3,-2.2804063E-2,-6.4429766E-3,-6.7508672E-3,-1.0144116E-3,-5.429846E-3,-1.18610505E-2,2.6939705E-3,-3.9837346E-3,5.9124897E-3,1.4841356E-3,-5.202229E-3,-1.0471218E-3,2.2647772E-3,-2.577844E-3,1.239909E-4,6.350438E-3,-6.466764E-3,-1.1168835E-2,4.307985E-3,-1.367701E-3,-2.6068415E-2,-1.3109444E-2,-6.096039E-3,-2.7330614E-5,-5.3147003E-3,2.6891564E-3,-1.4077142E-2,-6.846735E-3,-6.8134987E-3,3.5425648E-3,5.6327297E-3,-5.351982E-4,-2.2536116E-4,7.813111E-3,-7.474414E-3,-1.6500816E-3,4.5024226E-3,1.4405127E-2,1.3261233E-3,9.822615E-3,2.6393984E-3,-7.2887307E-3,7.14091E-4,-7.845572E-3,6.0068015E-3,9.197734E-4,8.180279E-3,-8.090255E-4,5.7291253E-3,-2.4944177E-4,1.0350608E-2,1.7027372E-2,-6.0503036E-3,7.797655E-4,1.2467442E-2,5.3172107E-3,1.2963105E-2,6.2575773E-3,1.9590955E-3,-3.8038634E-3,5.8442242E-3,1.335308E-2,1.2600693E-2,1.500583E-3,-2.0385894E-3,8.917339E-3,1.6513738E-3,7.338541E-3,1.6188938E-2,8.794866E-3,5.6451587E-3,1.2055828E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,59,61,-1,63,65,67,69,71,73,75,77,79,-1,81,83,85,87,89,91,93,95,97,99,101,-1,103,-1,105,107,109,111,113,115,-1,117,119,121,123,125,-1,-1,-1,-1,127,129,131,133,135,137,139,141,143,145,147,-1,-1,-1,-1,149,151,153,155,-1,157,159,-1,-1,161,163,165,-1,167,169,171,-1,173,175,-1,-1,177,179,181,183,-1,185,187,189,-1,191,-1,-1,193,-1,-1,195,197,199,201,203,205,207,209,211,-1,-1,213,215,217,219,221,223,225,227,-1,229,-1,-1,-1,-1,231,233,-1,235,-1,-1,237,239,-1,241,-1,-1,-1,-1,243,-1,245,247,249,251,253,-1,255,257,259,261,263,-1,265,-1,-1,-1,267,-1,-1,-1,269,271,273,275,-1,-1,-1,-1,-1,-1,277,-1,-1,279,-1,-1,-1,281,-1,283,-1,-1,285,287,289,291,293,295,297,-1,299,301,303,-1,305,-1,307,-1,309,311,313,315,-1,317,319,-1,-1,321,-1,-1,323,325,-1,327,-1,329,331,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0482134E1,5.285022E0,9.270496E0,4.1491566E0,2.7797332E0,2.9930928E0,2.5963383E0,2.3464108E0,0E0,6.6037416E-1,5.405482E-1,1.255162E0,1.6508279E0,2.0766788E0,1.1651125E0,3.9972973E-1,1.7837029E0,4.8151803E-1,3.518364E-1,2.9260758E-1,1.0294771E-1,7.770846E-1,1.3453119E0,1.9152267E0,5.0511485E-1,6.3102525E-1,1.3783693E0,3.7158537E-1,8.893933E-1,0E0,0E0,6.93663E-1,1.0599594E0,3.756882E-1,0E0,2.8112882E-1,2.3307094E-1,1.8636233E-1,2.4473634E-1,5.572605E-2,1.1824906E-2,4.152062E-1,7.9835474E-1,1.170966E0,0E0,8.245168E-1,8.4630203E-1,1.5564144E-1,1.7463313E-1,1.16678536E-1,2.538627E-1,3.578406E-1,1.0676527E0,2.0174086E-2,5.6030464E-1,7.579236E-1,0E0,5.05479E-1,0E0,7.5909233E-1,6.262479E-1,6.676948E-2,2.93005E-1,8.640745E-2,1.6702282E-1,0E0,1.9329901E-1,7.39975E-2,1.7839E-2,2.2778392E-2,1.08114295E-1,0E0,0E0,0E0,0E0,1.9837892E-1,4.9950433E-1,1.7487681E0,4.819814E-1,4.696945E-1,3.9844173E-1,6.009786E-1,2.1046233E-1,1.0133717E0,4.3149614E-1,1.0221392E-2,0E0,0E0,0E0,0E0,5.819404E-2,1.443227E-1,8.274324E-2,7.591113E-2,0E0,5.267954E-1,4.818473E-1,0E0,0E0,6.83341E-2,6.18473E-2,8.893819E-1,0E0,3.1085587E-1,7.456198E-2,6.216526E-1,0E0,5.08523E-1,6.394682E-1,0E0,0E0,2.2290757E-1,7.553601E-2,8.394808E-3,9.257698E-2,0E0,4.735309E-2,1.1989294E-1,7.788645E-2,0E0,2.794502E-2,0E0,0E0,1.336354E-2,0E0,0E0,1.4750525E-2,9.763849E-2,4.3482903E-2,8.1545E-2,1.273036E-1,1.3793209E-1,4.3676078E-1,1.8626416E-1,4.9990463E-1,0E0,0E0,2.711889E-1,1.4965186E-1,4.7079402E-1,3.508891E-1,1.7924221E-1,2.8048086E-1,2.9123998E-1,9.510986E-1,0E0,1.0590881E-1,0E0,0E0,0E0,0E0,6.251471E-2,7.608962E-2,0E0,3.2566674E-2,0E0,0E0,4.9574494E-1,9.876126E-1,0E0,2.8736806E-1,0E0,0E0,0E0,0E0,4.4906497E-1,0E0,2.943921E-1,2.8811932E-1,2.7199835E-2,3.584572E-2,3.1844807E-1,0E0,3.575018E-2,3.2966852E-1,3.4872735E-1,2.1160352E-1,1.6740039E-1,0E0,7.891965E-2,0E0,0E0,0E0,6.47187E-2,0E0,0E0,0E0,1.3549481E-2,7.691751E-2,2.8168028E-2,4.9774572E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.360957E-2,0E0,0E0,3.3873018E-2,0E0,0E0,0E0,1.4804745E-1,0E0,5.0518792E-2,0E0,0E0,2.4267453E-1,1.1655629E-1,2.768156E-1,2.3058552E-1,2.2007942E-1,1.2753148E-1,1.6170919E-1,0E0,3.7575826E-1,3.5329002E-1,3.2011968E-1,0E0,5.4778963E-2,0E0,1.2536013E-1,0E0,4.467845E-2,1.3758516E-1,2.5977865E-1,2.4884367E-1,0E0,1.0642707E-2,4.91293E-2,0E0,0E0,4.1233808E-2,0E0,0E0,3.4287453E-1,3.6679727E-1,0E0,4.1552436E-1,0E0,2.1763611E-1,4.433806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,59,59,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,69,69,70,70,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,90,90,91,91,92,92,93,93,95,95,96,96,99,99,100,100,101,101,103,103,104,104,105,105,107,107,108,108,111,111,112,112,113,113,114,114,116,116,117,117,118,118,120,120,123,123,126,126,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,137,137,138,138,139,139,140,140,141,141,142,142,143,143,144,144,146,146,151,151,152,152,154,154,157,157,158,158,160,160,165,165,167,167,168,168,169,169,170,170,171,171,173,173,174,174,175,175,176,176,177,177,179,179,183,183,187,187,188,188,189,189,190,190,197,197,200,200,204,204,206,206,209,209,210,210,211,211,212,212,213,213,214,214,215,215,217,217,218,218,219,219,221,221,223,223,225,225,226,226,227,227,228,228,230,230,231,231,234,234,237,237,238,238,240,240,242,242,243,243],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,60,62,-1,64,66,68,70,72,74,76,78,80,-1,82,84,86,88,90,92,94,96,98,100,102,-1,104,-1,106,108,110,112,114,116,-1,118,120,122,124,126,-1,-1,-1,-1,128,130,132,134,136,138,140,142,144,146,148,-1,-1,-1,-1,150,152,154,156,-1,158,160,-1,-1,162,164,166,-1,168,170,172,-1,174,176,-1,-1,178,180,182,184,-1,186,188,190,-1,192,-1,-1,194,-1,-1,196,198,200,202,204,206,208,210,212,-1,-1,214,216,218,220,222,224,226,228,-1,230,-1,-1,-1,-1,232,234,-1,236,-1,-1,238,240,-1,242,-1,-1,-1,-1,244,-1,246,248,250,252,254,-1,256,258,260,262,264,-1,266,-1,-1,-1,268,-1,-1,-1,270,272,274,276,-1,-1,-1,-1,-1,-1,278,-1,-1,280,-1,-1,-1,282,-1,284,-1,-1,286,288,290,292,294,296,298,-1,300,302,304,-1,306,-1,308,-1,310,312,314,316,-1,318,320,-1,-1,322,-1,-1,324,326,-1,328,-1,330,332,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,9.772148E4,1.22159E5,6.860185E2,2.091623E5,1.4101E4,3E0,3.5625357E-2,1.5669752E-2,1.3664E4,1.747325E3,3.0820766E-1,1.5988182E-1,4.52737E5,5.101512E0,5.5759512E-2,1.1457406E-1,1.8991614E5,9.283875E6,2.4848485E0,3.3870136E7,6.5321615E6,1.06E3,5.1232258E1,3.324E3,6.46E2,3.0720797E-1,8.625562E0,-3.907114E-2,-1.6659744E-2,9.561769E-1,2.0521326E0,2.750917E-4,-2.2726603E-2,2.695E3,1.305E3,2.9722316E0,6.4506575E5,1.5137865E2,6.08767E3,1E0,2.3067484E0,3E0,-2.3878902E-2,1.566593E6,9.0754684E1,4.987639E7,9E0,1E0,6.6322645E6,4.6E0,3.2196458E6,2.3276744E5,2.7753594E7,5.68E2,2.1287894E-2,3.008162E6,1.4865719E-2,4.5723195E6,4.2647058E-1,3.6111112E0,1.0503613E5,6.4860556E5,1.387623E8,-6.5206005E-3,2.1446484E-1,2.9956698E-1,1.038E3,4.838871E8,7.549744E7,1.6264575E-2,7.7439137E-3,3.4797555E-3,8.19477E-3,9.6E1,4.0697675E-2,2.08E2,7.57657E4,5.408015E0,6.748383E8,4.12E0,6.76E2,3.1753032E0,1.0133419E8,2.6432162E5,-1.4768608E-2,8.3495015E-3,-4.3980586E-3,-9.0386224E-4,1.0046621E7,6.3974875E5,2.23226E-1,3.921901E6,-1.7424772E-2,3.7057E5,1.0918E4,1.23808E-2,5.513686E-3,1.24681935E-1,2.0746333E2,1.9013363E4,-1.0684795E-2,3.164136E6,2.2668628E7,1.2427474E8,-3.0859161E-2,2.7487562E0,4.4337E5,-8.135023E-3,-1.714224E-2,3.730007E6,2E0,3E1,2.0503453E8,-1.8883985E-2,7.7628184E7,1.3878379E0,2.1340163E8,1.0669926E-2,1.68635E5,-2.3022566E-3,1.9204698E-3,2.3175376E3,-1.1721972E-3,-2.7560801E-3,1.6667E4,2.4434783E0,1.4925373E-1,1.62848E5,7.9037575E1,1.14E2,2.04E5,3.94E2,8.2363125E4,3.0058114E-2,6.602001E-3,2.6491638E1,3.224158E6,3.0324321E-2,2.065904E6,2E0,3.958068E2,1E0,7.056912E7,2.3625467E-2,1.819795E6,-3.1135597E-3,-7.399721E-3,-1.1283823E-2,-3.8999447E-3,1.7366706E8,4.1844E4,-6.108453E-3,1.85221E5,3.6069567E-3,-3.9780964E-3,2.1E1,3.72424E5,2.8442536E-2,5.6553E4,2.6422104E-2,1.2824549E-2,1.3593658E-2,4.3199817E-3,5.0714142E2,2.8637497E-2,1.46E2,9.773983E6,1.7148E4,3.4E1,8.764843E3,-2.7514871E-2,1E0,7.326E3,3.17441E5,9.367855E-1,5.500988E2,4.905947E-3,2.0211603E1,-1.4051737E-2,-7.299981E-3,-3.6601855E-3,1.0142518E0,5.2175424E-3,-1.8682455E-3,-8.799926E-3,2.4472452E7,5.124411E2,4.9304895E6,2.720437E6,6.406317E-3,7.877692E-4,-6.5970407E-3,-2.2490742E-3,7.1658543E-3,2.5576386E-3,1.139605E6,-5.8455416E-4,-3.479938E-3,7.2744107E0,-7.0514367E-3,1.3631404E-3,-7.6602083E-3,7.9E1,-1.126104E-2,3.938775E4,2.0089973E-2,3.1752042E-3,9.277255E6,1.9007992E1,1.76132E6,1.0545219E5,4.8215595E1,7.44393E5,1.4437085E-1,-3.6716743E-3,6.7913E4,1.2944685E0,3.5E1,8.082736E-3,1.9143396E2,-5.2426E-3,1.1304221E0,-4.1757366E-3,1.3452479E12,1.31903E5,1.4392978E8,2.599248E4,1.7659228E-3,5.4582E4,2.5086E4,5.320642E-3,2.5511757E-3,1.7322648E5,3.2724682E-3,-1.7570835E-3,1.4274633E7,1.5515003E0,-1.70742E-2,1.645614E6,7.9054444E-4,9.51E3,4.5575E4,-4.2145737E-3,2.5062722E-3,-1.1297107E-2,3.9396444E-3,-7.0807734E-3,-3.297989E-4,4.264457E-3,-4.206086E-3,1.4476355E-3,-2.0980868E-2,-1.175015E-2,-7.2615277E-3,-1.4821871E-3,-4.886515E-3,-2.0264404E-2,-8.071993E-4,-7.074985E-3,-2.2804063E-2,-6.4429766E-3,-6.7508672E-3,-1.0144116E-3,-5.429846E-3,-1.18610505E-2,2.6939705E-3,-3.9837346E-3,5.9124897E-3,1.4841356E-3,-5.202229E-3,-1.0471218E-3,2.2647772E-3,-2.577844E-3,1.239909E-4,6.350438E-3,-6.466764E-3,-1.1168835E-2,4.307985E-3,-1.367701E-3,-2.6068415E-2,-1.3109444E-2,-6.096039E-3,-2.7330614E-5,-5.3147003E-3,2.6891564E-3,-1.4077142E-2,-6.846735E-3,-6.8134987E-3,3.5425648E-3,5.6327297E-3,-5.351982E-4,-2.2536116E-4,7.813111E-3,-7.474414E-3,-1.6500816E-3,4.5024226E-3,1.4405127E-2,1.3261233E-3,9.822615E-3,2.6393984E-3,-7.2887307E-3,7.14091E-4,-7.845572E-3,6.0068015E-3,9.197734E-4,8.180279E-3,-8.090255E-4,5.7291253E-3,-2.4944177E-4,1.0350608E-2,1.7027372E-2,-6.0503036E-3,7.797655E-4,1.2467442E-2,5.3172107E-3,1.2963105E-2,6.2575773E-3,1.9590955E-3,-3.8038634E-3,5.8442242E-3,1.335308E-2,1.2600693E-2,1.500583E-3,-2.0385894E-3,8.917339E-3,1.6513738E-3,7.338541E-3,1.6188938E-2,8.794866E-3,5.6451587E-3,1.2055828E-2],"split_indices":[20,102,33,2,52,28,2,3,0,38,2,55,27,38,29,56,27,42,28,45,57,7,47,2,56,29,2,38,35,0,0,38,53,39,0,2,2,57,28,56,4,8,58,8,0,9,56,45,17,67,50,54,28,33,47,3,0,32,0,28,58,54,33,33,7,0,38,38,0,7,7,0,0,0,0,29,58,2,28,54,7,54,2,54,7,28,0,0,0,0,47,47,39,5,0,9,9,0,0,57,58,52,0,1,45,45,0,54,11,0,0,9,53,3,5,0,12,53,7,0,29,0,0,4,0,0,9,53,56,7,33,29,5,0,47,0,0,53,47,38,28,17,52,102,7,0,1,0,0,0,0,32,9,0,9,0,0,3,9,0,1,0,0,0,0,58,0,29,1,9,3,33,0,16,9,32,27,4,0,56,0,0,0,58,0,0,0,9,4,32,32,0,0,0,0,0,0,31,0,0,56,0,0,0,0,0,45,0,0,9,56,1,28,54,1,57,0,12,53,0,0,52,0,38,0,31,1,5,51,0,9,9,0,0,28,0,0,45,39,0,9,0,29,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E3,8.33E2,1.747E3,4.61E2,3.72E2,1.121E3,6.26E2,4.57E2,4E0,2.79E2,9.3E1,5.11E2,6.1E2,4.27E2,1.99E2,1.3E1,4.44E2,1.13E2,1.66E2,6.6E1,2.7E1,3.67E2,1.44E2,5.73E2,3.7E1,8.2E1,3.45E2,5.3E1,1.46E2,7E0,6E0,8.9E1,3.55E2,1.09E2,4E0,5.4E1,1.12E2,3.2E1,3.4E1,1.7E1,1E1,7.9E1,2.88E2,1.4E2,4E0,3.05E2,2.68E2,2.8E1,9E0,2.4E1,5.8E1,1.7E1,3.28E2,1.4E1,3.9E1,1.3E2,1.6E1,8.5E1,4E0,1.8E2,1.75E2,1.5E1,9.4E1,3.6E1,1.8E1,1.4E1,9.8E1,2.4E1,8E0,1.7E1,1.7E1,1.1E1,6E0,5E0,5E0,4.8E1,3.1E1,3.8E1,2.5E2,9E0,1.31E2,2.17E2,8.8E1,2.42E2,2.6E1,1.6E1,1.2E1,5E0,4E0,5E0,1.9E1,3.7E1,2.1E1,1.3E1,4E0,2.65E2,6.3E1,1E1,4E0,3E1,9E0,1.26E2,4E0,5.8E1,2.7E1,1.74E2,6E0,3.8E1,1.37E2,7E0,8E0,5.8E1,3.6E1,1.4E1,2.2E1,4E0,1.4E1,6.8E1,3E1,1.5E1,9E0,4E0,4E0,1.2E1,5E0,4E0,1.3E1,3.4E1,1.4E1,1E1,2.1E1,2E1,1.8E1,1.34E2,1.16E2,4E0,5E0,8.4E1,4.7E1,1.53E2,6.4E1,2.8E1,6E1,7.1E1,1.71E2,9E0,1.7E1,4E0,1.2E1,1.4E1,5E0,1.7E1,2E1,9E0,1.2E1,4E0,9E0,1.09E2,1.56E2,5E0,5.8E1,2.6E1,4E0,4E0,5E0,1.21E2,5E0,3.6E1,2.2E1,1.7E1,1E1,1.67E2,7E0,1.1E1,2.7E1,1.26E2,1.1E1,5.1E1,7E0,3E1,6E0,7E0,7E0,1.8E1,4E0,4E0,1E1,8E0,6E1,1E1,2E1,5E0,4E0,8E0,4E0,9E0,4E0,2.9E1,5E0,5E0,9E0,5E0,5E0,5E0,1.6E1,7E0,1.3E1,1.2E1,6E0,9.6E1,3.8E1,4.7E1,6.9E1,4E1,4.4E1,4.3E1,4E0,1.15E2,3.8E1,6E1,4E0,2.1E1,7E0,5.5E1,5E0,1.6E1,5.5E1,6.4E1,1.07E2,4E0,1.3E1,1.3E1,4E0,8E0,1.2E1,4E0,8E0,8E1,2.9E1,4E0,1.52E2,5E0,5.3E1,1.15E2,6E0,4E0,3.2E1,1E1,1.2E1,4E0,1.3E1,6E0,4E0,1.1E1,1.56E2,5E0,6E0,5E0,2.2E1,2.9E1,9.7E1,7E0,4E0,2.7E1,2.4E1,2.3E1,7E0,4E0,1.4E1,4E0,4E0,1.4E1,4.6E1,5E0,5E0,4E0,1.6E1,2E1,9E0,5E0,4E0,6E0,1E1,5E0,8E0,8.6E1,1E1,8E0,3E1,4E1,7E0,2.1E1,4.8E1,2.9E1,1.1E1,1.3E1,3.1E1,3.9E1,4E0,1.01E2,1.4E1,1.3E1,2.5E1,1.4E1,4.6E1,1E1,1.1E1,5.1E1,4E0,1.2E1,4E0,4.1E1,1.4E1,1.9E1,4.5E1,1.4E1,9.3E1,9E0,4E0,6E0,7E0,8E0,4E0,7.2E1,8E0,1.4E1,1.5E1,4.7E1,1.05E2,3.2E1,2.1E1,5.2E1,6.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"333","size_leaf_vector":"1"}},{"base_weights":[8.715238E-4,-1.1340152E-1,5.500383E-2,-1.3158675E-1,1.9544531E-1,2.7602827E-3,1.5028201E-1,-1.9985795E-1,-5.1919118E-2,3.28571E-2,9.563694E-2,-2.486405E-2,9.789981E-2,-9.608693E-2,1.6581991E-1,-5.1575154E-2,-2.2911064E-1,-7.9270564E-2,1.1108405E-1,3.425376E-2,2.0280308E-1,-3.0944277E-2,3.2296106E-1,3.154147E-2,1.6835098E-1,6.8857996E-3,-1.286138E-1,1.482254E-1,3.5238472E-1,-7.6671064E-2,1.4490627E-2,-2.1218854E-1,-4.7022048E-1,-1.1046373E-1,9.511084E-3,1.9170801E-1,3.5131082E-2,-3.4420714E-3,9.656791E-3,1.1676615E-2,5.020271E-3,-5.7955977E-2,4.7641955E-2,2.5246033E-2,9.058313E-3,-1.3288832E-1,5.394953E-2,1.9425803E-1,-1.1247839E-2,-8.809344E-3,-4.1681763E-2,1.8848751E-1,1.0149502E-1,2.894153E-1,5.0449884E-1,-1.9801402E-1,-2.7042162E-2,-2.4975918E-1,-1.4616118E-1,-2.3708317E-1,-3.0648855E-2,-1.0193648E-1,-1.8243778E-2,-9.456065E-3,8.109538E-3,1.8721491E-2,1.2723112E-1,6.2469207E-2,-2.788513E-3,-6.2541375E-3,3.056141E-2,-1.0263236E-1,-5.305351E-3,8.5501306E-2,-4.337099E-2,8.5376866E-2,-1.807922E-2,1.6048221E-1,2.1390287E-2,1.3830386E-1,2.5074005E-1,-5.481607E-3,2.7962956E-3,1.5526292E-1,3.119968E-1,3.177453E-2,1.5910184E-1,3.1417203E-1,3.404619E-3,1.369977E-2,2.8800756E-2,-2.5813365E-1,1.0966511E-3,2.0322109E-2,-1.0711162E-1,-2.1829203E-1,-3.6136946E-1,-3.1328868E-2,-1.8709719E-1,-2.9895082E-3,-2.0105101E-2,-1.2685435E-1,-4.6016917E-2,1.4594157E-3,-7.904651E-3,9.415133E-3,2.3579837E-3,-3.7086017E-3,9.696834E-2,-2.8638698E-2,9.2943594E-2,-8.229746E-2,-2.2337724E-1,2.736922E-2,-2.1265624E-1,4.5703966E-2,1.5127093E-1,-6.665555E-2,9.135966E-2,7.0314864E-3,6.473091E-4,1.959865E-1,-1.7455312E-3,-4.1775912E-2,8.006949E-2,2.880482E-4,1.8848488E-1,1.8497927E-2,1.8440898E-1,1.297317E-1,2.4919537E-1,4.082506E-1,7.0071414E-2,4.825124E-2,-1.3496101E-2,1.8202043E-1,-5.198168E-2,1.7992895E-2,1.018288E-2,-2.1517882E-2,-6.4363866E-3,6.3931495E-2,-6.9118324E-3,-1.3800521E-1,-1.9411803E-3,-4.0741113E-1,-1.965189E-1,-2.7636891E-2,-2.721835E-1,1.1157879E-2,-1.1211464E-2,-2.2262199E-1,-9.2455484E-2,-1.5539191E-2,-1.1589579E-1,-8.382164E-2,2.1184225E-2,-2.6167143E-2,4.5403216E-2,8.393224E-3,5.6736253E-2,-3.1872878E-3,9.6278486E-4,6.2441733E-3,2.1151737E-3,-9.929945E-2,4.3253034E-2,-3.2342907E-2,-1.4917342E-1,6.2722646E-2,-1.3822165E-1,-2.4690045E-1,3.8191203E-3,2.4809863E-2,1.4379881E-1,1.6826153E-1,-1.200423E-3,5.814131E-3,-8.687984E-2,8.052181E-3,-5.327429E-4,1.14638194E-1,2.4974379E-1,9.333E-2,-1.0144388E-1,1.14219144E-1,-5.0310176E-3,3.8039174E-3,-2.8903745E-2,2.2530298E-1,4.0773045E-2,1.9943061E-1,2.267492E-3,1.4908637E-1,1.010279E-2,8.222012E-2,2.8657845E-1,4.4588846E-1,5.401831E-3,1.7562571E-1,-5.4996844E-2,1.12338066E-1,-4.890517E-3,1.2432197E-1,2.7464566E-1,-1.9133372E-2,1.1868701E-1,1.1925325E-4,6.5159625E-3,-3.889209E-3,-8.487525E-3,-6.617049E-3,-2.438553E-2,-7.724452E-3,-1.4643134E-2,-5.7959855E-3,-1.7411973E-2,2.850949E-3,-3.8557416E-3,-8.7925E-3,-1.70515E-2,-6.2702512E-3,1.9272129E-3,-9.088515E-3,-4.5465105E-3,2.5818131E-3,-4.911757E-3,-2.4105315E-3,2.4634413E-3,-2.8313154E-3,2.4268404E-3,3.5723662E-3,-7.357201E-4,-5.5444567E-4,4.4029118E-3,-1.9171379E-3,-6.7506973E-3,-6.374039E-5,1.1387967E-2,-1.5389676E-3,-1.0256018E-2,9.4823464E-4,9.930024E-3,-5.511295E-3,-1.696566E-2,-1.3880889E-2,-3.9568413E-3,1.982281E-3,-4.3053874E-3,9.96591E-3,4.260981E-4,3.6999357E-3,1.0002481E-2,-9.032128E-3,-2.6661083E-3,1.8224301E-3,8.679065E-3,7.314926E-3,1.5860109E-2,1.8056383E-3,6.5809977E-3,-1.9468704E-2,-2.2144322E-3,8.416648E-3,2.2694594E-3,-1.1818704E-4,-3.5422454E-3,8.109165E-3,1.4114307E-2,5.462786E-3,-2.480757E-3,3.7243436E-3,1.06728105E-2,3.1268008E-3,8.741594E-3,3.5426698E-3,-8.246932E-3,1.293453E-3,6.664388E-3,1.5298615E-2,4.796632E-3,2.413352E-2,1.0886375E-2,1.17799025E-2,3.414979E-3,-5.97283E-3,1.0264135E-3,1.4343273E-3,8.96501E-3,-2.5750452E-3,3.6169544E-3,7.061623E-3,-5.0193924E-3,1.1600278E-2,2.8810445E-2,1.00757E-2,-2.300504E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,57,59,61,63,65,67,69,-1,-1,-1,71,73,-1,-1,75,77,79,-1,-1,81,83,85,87,89,91,93,95,97,99,-1,101,-1,103,-1,-1,105,107,-1,-1,109,111,113,115,117,119,-1,121,123,125,127,-1,-1,129,131,133,135,137,-1,-1,-1,139,-1,141,143,145,147,149,151,-1,-1,153,155,157,-1,-1,-1,-1,159,161,163,165,167,169,171,173,175,177,179,-1,-1,181,-1,183,185,187,189,-1,191,193,195,197,199,201,-1,203,205,-1,-1,-1,-1,207,-1,209,-1,211,213,-1,215,217,-1,219,221,-1,223,225,227,229,231,-1,233,-1,-1,-1,-1,235,237,-1,239,241,243,245,-1,247,249,251,-1,-1,253,-1,-1,255,257,259,261,263,-1,-1,265,267,269,271,-1,273,275,277,279,281,-1,283,285,287,289,291,293,-1,295,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5780189E1,4.6312065E0,8.615932E0,4.204853E0,2.058859E0,2.943352E0,2.3625317E0,1.7969913E0,1.6093614E0,0E0,2.4960294E-1,1.8456881E0,1.1727955E0,2.992151E-1,1.8417788E0,6.2229514E-1,1.3200932E0,8.5401976E-1,3.116938E-1,1.5651047E-1,2.076757E-2,1.8155875E0,2.9580307E-1,4.898457E-1,1.2873158E0,0E0,1.2946415E-1,9.761629E-1,3.233385E-1,3.9210498E-1,0E0,7.746811E-1,5.888758E-1,4.5788574E-1,2.3626868E-1,2.472651E-1,7.3096745E-2,9.4884224E-2,0E0,0E0,0E0,1.4954798E0,7.583146E-1,0E0,0E0,8.303962E-1,3.9892763E-1,3.2876825E-1,0E0,0E0,9.248747E-2,1.1128616E0,9.8698926E-1,1.8128324E-1,5.724597E-2,2.569217E-1,1.8421344E-1,6.4811134E-1,5.6293225E-1,2.6890278E-1,0E0,3.0349636E-1,0E0,1.187533E-1,0E0,0E0,8.907223E-2,1.1094279E-1,0E0,0E0,6.913636E-2,8.262522E-1,1.990925E0,3.987919E-1,2.1000399E-1,3.4251824E-2,0E0,1.9417548E-1,3.3693376E-1,4.1549194E-1,3.7548947E-1,0E0,0E0,5.072689E-1,1.3452811E0,5.750873E-1,6.625562E-1,9.389949E-2,0E0,0E0,0E0,2.791642E-1,0E0,2.2332239E-1,3.2598093E-2,5.95242E-1,6.018648E-1,2.731718E-1,2.786467E-1,0E0,0E0,2.7269197E-1,1.7961214E-1,8.495518E-2,0E0,0E0,0E0,0E0,4.189214E-2,1.8068302E-2,9.296723E-3,6.361389E-1,1.444782E0,1.4956378E0,4.148575E-1,1.9729355E-1,1.7678797E-1,2.1738571E-1,7.2920516E-2,0E0,0E0,7.195032E-2,0E0,3.6417353E-1,3.0162147E-1,3.9837334E-2,2.3010647E-1,0E0,7.031059E-2,4.129467E-1,2.6981378E-1,4.2625713E-1,2.472857E-1,3.6836353E-1,0E0,6.1786413E-1,8.3618575E-1,0E0,0E0,0E0,0E0,1.01968125E-1,0E0,9.781256E-3,0E0,2.9009557E-1,5.5778456E-1,0E0,3.4086466E-1,1.1791496E-1,0E0,2.941835E-1,1.17373034E-1,0E0,2.23418E-1,9.3360156E-2,5.313336E-2,1.00758985E-1,4.3411158E-2,0E0,2.8650034E-2,0E0,0E0,0E0,0E0,5.642173E-1,2.9630396E-1,0E0,2.791481E-1,1.2456893E0,2.1550488E-1,1.6457748E-1,0E0,1.386637E-1,1.2627786E-1,1.4457119E-1,0E0,0E0,1.5086025E-1,0E0,0E0,4.4444665E-2,4.444784E-2,2.5969714E-2,4.9308673E-1,1.4157891E-1,0E0,0E0,1.4318302E-2,8.024299E-2,6.741679E-2,4.4242024E-2,0E0,3.33596E-1,2.8857177E-1,2.5365807E-2,1.12538815E-1,2.0779943E-1,0E0,4.839456E-2,4.63326E-2,2.697497E-1,2.2000687E-1,3.0851126E-1,4.562416E-1,0E0,1.03212416E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,41,41,42,42,45,45,46,46,47,47,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,63,63,66,66,67,67,70,70,71,71,72,72,73,73,74,74,75,75,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,87,87,91,91,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,121,121,123,123,124,124,125,125,126,126,128,128,129,129,130,130,131,131,132,132,133,133,135,135,136,136,141,141,143,143,145,145,146,146,148,148,149,149,151,151,152,152,154,154,155,155,156,156,157,157,158,158,160,160,165,165,166,166,168,168,169,169,170,170,171,171,173,173,174,174,175,175,178,178,181,181,182,182,183,183,184,184,185,185,188,188,189,189,190,190,191,191,193,193,194,194,195,195,196,196,197,197,199,199,200,200,201,201,202,202,203,203,204,204,206,206],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,58,60,62,64,66,68,70,-1,-1,-1,72,74,-1,-1,76,78,80,-1,-1,82,84,86,88,90,92,94,96,98,100,-1,102,-1,104,-1,-1,106,108,-1,-1,110,112,114,116,118,120,-1,122,124,126,128,-1,-1,130,132,134,136,138,-1,-1,-1,140,-1,142,144,146,148,150,152,-1,-1,154,156,158,-1,-1,-1,-1,160,162,164,166,168,170,172,174,176,178,180,-1,-1,182,-1,184,186,188,190,-1,192,194,196,198,200,202,-1,204,206,-1,-1,-1,-1,208,-1,210,-1,212,214,-1,216,218,-1,220,222,-1,224,226,228,230,232,-1,234,-1,-1,-1,-1,236,238,-1,240,242,244,246,-1,248,250,252,-1,-1,254,-1,-1,256,258,260,262,264,-1,-1,266,268,270,272,-1,274,276,278,280,282,-1,284,286,288,290,292,294,-1,296,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.7002478E3,9.772148E4,1E0,1.5151515E-3,1.208686E6,2.09E2,8.2608955E-3,9.026015E2,3.28571E-2,1.5595E4,1.8552577E2,1.06E3,9.652488E5,6.701152E0,1.5283889E0,1.3521131E8,1.1323028E0,9.397667E6,7.6188426E0,2.2056E4,3.30399E6,1.27365E4,5.8631687E1,2.89E2,6.8857996E-3,8.89771E6,2E1,1.8114515E1,8E0,1.4490627E-2,3.488E3,1.2E1,4.8297736E7,1.6525185E9,2.1515152E0,4.4881E4,2.4081633E0,9.656791E-3,1.1676615E-2,5.020271E-3,3.7311627E2,2.8710976E1,2.5246033E-2,9.058313E-3,2E0,8.916961E2,5.314225E2,-1.1247839E-2,-8.809344E-3,1.2542373E0,1.5106794E3,6.748383E8,4.728721E0,5.686354E-1,3.3E1,1.3649979E10,2.8035452E11,2.7246006E7,5.980516E5,-3.0648855E-2,3.2069644E2,-1.8243778E-2,1.044E1,8.109538E-3,1.8721491E-2,8.4342186E1,7.549744E7,-2.788513E-3,-6.2541375E-3,3.0990322E5,9.337562E0,4.8297736E7,2.5682794E5,4.954957E7,9.253112E-1,-1.807922E-2,4.3E0,2.8331464E2,5.3379045E6,2.5853658E0,-5.481607E-3,2.7962956E-3,3.0121632E6,2.1938796E7,2.1381798E-6,7.065733E7,2.0550244E0,3.404619E-3,1.369977E-2,2.8800756E-2,3.4955364E7,1.0966511E-3,2.006639E0,2.5269838E6,4.36E2,2.863783E2,8.9E1,9.433E3,-2.9895082E-3,-2.0105101E-2,1E0,4.459525E5,2.821E3,-7.904651E-3,9.415133E-3,2.3579837E-3,-3.7086017E-3,9.5E2,3.494351E6,1.3498145E5,1.263E3,2E0,4.004366E1,4.4E1,1.4777102E9,1E0,1.7560976E0,1.067536E3,7.0314864E-3,6.473091E-4,2.5378451E0,-1.7455312E-3,5.088781E0,6.0479047E4,1.263E3,1.6E1,1.8497927E-2,1E0,2.3523688E0,2.342E3,8.2E1,2.4350044E2,2.7753687E-2,-1.3496101E-2,2.1150263E-1,6.43673E5,1.7992895E-2,1.018288E-2,-2.1517882E-2,-6.4363866E-3,1.4615384E0,-6.9118324E-3,2.0605305E5,-1.9411803E-3,1E0,1.8544E4,-2.7636891E-2,2.22E3,8.749966E1,-1.1211464E-2,1.7843524E7,5.632258E0,-1.5539191E-2,1.09E3,1.8780488E0,1.0376E4,1.838444E6,2.26182E5,8.393224E-3,5.1180666E8,-3.1872878E-3,9.6278486E-4,6.2441733E-3,2.1151737E-3,2E0,2.5054495E1,-3.2342907E-2,2E0,1.0485785E3,1.4571098E5,7.160724E1,3.8191203E-3,3.0161016E7,2.909019E3,7.68E2,-1.200423E-3,5.814131E-3,5.4410156E4,8.052181E-3,-5.327429E-4,6.4596E4,2.6034303E0,1.5597535E3,1.632095E2,1E0,-5.0310176E-3,3.8039174E-3,4.3531516E2,7.130317E7,1.3065217E1,1.9577E4,2.267492E-3,2.960909E2,2.281E3,3.3953488E0,9.433E3,9.099893E8,5.401831E-3,2.0666666E0,7.214244E-1,1.8019372E5,1.998175E3,3.13E2,2.4507338E4,-1.9133372E-2,9.536863E4,1.1925325E-4,6.5159625E-3,-3.889209E-3,-8.487525E-3,-6.617049E-3,-2.438553E-2,-7.724452E-3,-1.4643134E-2,-5.7959855E-3,-1.7411973E-2,2.850949E-3,-3.8557416E-3,-8.7925E-3,-1.70515E-2,-6.2702512E-3,1.9272129E-3,-9.088515E-3,-4.5465105E-3,2.5818131E-3,-4.911757E-3,-2.4105315E-3,2.4634413E-3,-2.8313154E-3,2.4268404E-3,3.5723662E-3,-7.357201E-4,-5.5444567E-4,4.4029118E-3,-1.9171379E-3,-6.7506973E-3,-6.374039E-5,1.1387967E-2,-1.5389676E-3,-1.0256018E-2,9.4823464E-4,9.930024E-3,-5.511295E-3,-1.696566E-2,-1.3880889E-2,-3.9568413E-3,1.982281E-3,-4.3053874E-3,9.96591E-3,4.260981E-4,3.6999357E-3,1.0002481E-2,-9.032128E-3,-2.6661083E-3,1.8224301E-3,8.679065E-3,7.314926E-3,1.5860109E-2,1.8056383E-3,6.5809977E-3,-1.9468704E-2,-2.2144322E-3,8.416648E-3,2.2694594E-3,-1.1818704E-4,-3.5422454E-3,8.109165E-3,1.4114307E-2,5.462786E-3,-2.480757E-3,3.7243436E-3,1.06728105E-2,3.1268008E-3,8.741594E-3,3.5426698E-3,-8.246932E-3,1.293453E-3,6.664388E-3,1.5298615E-2,4.796632E-3,2.413352E-2,1.0886375E-2,1.17799025E-2,3.414979E-3,-5.97283E-3,1.0264135E-3,1.4343273E-3,8.96501E-3,-2.5750452E-3,3.6169544E-3,7.061623E-3,-5.0193924E-3,1.1600278E-2,2.8810445E-2,1.00757E-2,-2.300504E-4],"split_indices":[20,52,33,102,57,28,29,27,52,0,9,56,2,47,35,41,45,42,45,54,9,9,33,47,11,0,51,3,35,8,0,2,18,45,5,53,9,53,0,0,0,52,58,0,0,8,33,4,0,0,53,52,7,38,38,3,5,31,45,28,0,52,0,54,0,0,58,7,0,0,28,54,45,28,32,56,0,54,4,45,54,0,0,28,51,37,1,39,0,0,0,7,0,53,32,2,52,8,2,0,0,67,28,2,0,0,0,0,0,29,33,2,17,53,8,7,74,53,52,0,0,54,0,58,33,2,3,0,6,41,2,8,58,57,0,38,11,0,0,0,0,53,0,33,0,104,9,0,2,58,0,48,54,0,2,54,9,9,29,0,12,0,0,0,0,17,56,0,8,52,28,56,0,45,32,2,0,0,28,0,0,29,58,33,4,16,0,0,4,7,56,9,0,52,29,54,2,7,0,53,34,33,52,3,4,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.549E3,8.19E2,1.73E3,7.74E2,4.5E1,1.118E3,6.12E2,4.16E2,3.58E2,7E0,3.8E1,8.67E2,2.51E2,3.6E1,5.76E2,6.9E1,3.47E2,3.07E2,5.1E1,2.5E1,1.3E1,8.53E2,1.4E1,1.3E2,1.21E2,4E0,3.2E1,5.28E2,4.8E1,6.5E1,4E0,3.26E2,2.1E1,2.27E2,8E1,2.4E1,2.7E1,2.1E1,4E0,9E0,4E0,6.35E2,2.18E2,5E0,9E0,1.5E1,1.15E2,1.14E2,7E0,2E1,1.2E1,2.82E2,2.46E2,3.6E1,1.2E1,1.8E1,4.7E1,2.06E2,1.2E2,9E0,1.2E1,2.21E2,6E0,7.2E1,8E0,5E0,1.9E1,2.1E1,6E0,4E0,1.7E1,3.43E2,2.92E2,1.54E2,6.4E1,8E0,7E0,2.6E1,8.9E1,5.9E1,5.5E1,7E0,5E0,2.24E2,5.8E1,1.12E2,1.34E2,3.2E1,4E0,4E0,8E0,1.4E1,4E0,3E1,1.7E1,1.63E2,4.3E1,3.2E1,8.8E1,5E0,4E0,1.52E2,6.9E1,6.8E1,4E0,1E1,9E0,4E0,1.7E1,9E0,8E0,2.95E2,4.8E1,2.53E2,3.9E1,9.7E1,5.7E1,5.5E1,9E0,4E0,4E0,2.2E1,4E0,4.3E1,4.6E1,1.6E1,4.3E1,1.8E1,3.7E1,1.78E2,4.6E1,4.1E1,1.7E1,1.07E2,5E0,1.21E2,1.3E1,2.1E1,1.1E1,5E0,9E0,2.4E1,6E0,1.1E1,6E0,1.5E1,1.48E2,1.2E1,3.1E1,2.7E1,5E0,6.3E1,2.5E1,7E0,1.45E2,4.4E1,2.5E1,4.2E1,2.6E1,5E0,1.2E1,5E0,4E0,4E0,4E0,2.6E2,3.5E1,6E0,4.2E1,2.09E2,4.4E1,3.5E1,4E0,8.1E1,1.6E1,5.2E1,5E0,5E0,5E1,5E0,4E0,1E1,1.2E1,1.3E1,3E1,3.9E1,7E0,4E0,1.2E1,3.4E1,9E0,3.3E1,4E0,1.53E2,2.5E1,9E0,3.7E1,3.6E1,5E0,9E0,8E0,4.8E1,5.9E1,7.6E1,4.5E1,4E0,9E0,1.3E1,1.1E1,5E0,6E0,4E0,1.1E1,1.05E2,4.3E1,1.1E1,2E1,1.8E1,9E0,4.7E1,1.6E1,2E1,5E0,3.8E1,1.07E2,4E0,4E1,7E0,1.8E1,3E1,1.2E1,1.8E1,8E0,4E0,8E0,9.7E1,1.63E2,2.9E1,6E0,1.4E1,2.8E1,1.59E2,5E1,4E1,4E0,2.9E1,6E0,7.2E1,9E0,1.1E1,5E0,1.4E1,3.8E1,1.2E1,3.8E1,5E0,5E0,6E0,6E0,6E0,7E0,4E0,2.6E1,2.1E1,1.8E1,8E0,4E0,1.8E1,1.6E1,5E0,4E0,4E0,2.9E1,3.6E1,1.17E2,1.9E1,6E0,5E0,4E0,3.3E1,4E0,3E1,6E0,5E0,4E0,4E0,4E0,2.2E1,2.6E1,3.7E1,2.2E1,7.1E1,5E0,4.1E1,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"297","size_leaf_vector":"1"}},{"base_weights":[-9.570896E-5,-1.11217044E-1,5.2807212E-2,-1.7505777E-1,-3.4517344E-2,1.3578766E-2,1.4255175E-1,-1.8257774E-1,2.6970001E-2,-8.25269E-2,1.216305E-1,-5.4149617E-2,5.5778712E-2,1.01740606E-1,2.5449702E-1,-4.7839537E-1,-1.713291E-1,-1.1136996E-1,-7.3023434E-3,8.0642596E-2,2.4465482E-1,-6.652373E-2,1.5134136E-1,-1.2047807E-1,7.2581574E-2,4.8694525E-2,1.7249462E-1,6.253478E-2,2.7500436E-1,-1.0326594E-2,-2.9136509E-2,-5.28009E-2,-1.9189772E-1,-1.0554794E-1,-1.6495377E-2,2.148421E-2,-1.18632354E-1,-3.493665E-2,1.1069798E-1,2.865632E-1,4.254096E-3,-5.077998E-2,-2.1216244E-1,1.7909545E-2,7.510512E-2,-1.4493401E-1,7.406909E-3,4.4583376E-2,1.840484E-1,8.5652344E-2,-4.828607E-2,1.593349E-1,2.846274E-2,6.7595397E-3,-9.008128E-4,2.8728187E-1,1.0862014E-3,-2.652541E-2,-2.1626507E-1,-2.0062643E-1,2.9220684E-2,-9.6118525E-2,-2.3887013E-1,-5.0052363E-2,5.8390047E-2,-1.965646E-2,-1.0214529E-2,3.81616E-3,-8.23564E-2,1.3142134E-1,4.3007047E-3,2.120232E-2,1.0569594E-2,-1.4486127E-1,-3.3197917E-2,-1.3309935E-1,-2.7053256E-2,1.3036627E-1,2.2795403E-2,-9.4883785E-2,-2.2934651E-1,-2.921511E-2,7.269984E-2,1.6338292E-1,2.7354132E-2,5.347548E-2,2.0271921E-1,-2.2218622E-1,1.5641352E-2,1.7132467E-1,-1.2801717E-2,2.3347948E-1,3.600877E-1,-1.2199793E-1,3.766604E-3,-1.6852584E-2,-2.5982966E-3,-1.8650664E-1,-3.7375188E-1,1.0837223E-2,-5.8739547E-2,-1.6706143E-1,-8.063594E-2,-1.6341954E-2,-5.8240555E-3,2.7308923E-3,-8.217389E-2,1.043457E-1,2.5590353E-2,-3.3537662E-3,1.585864E-3,-1.9201749E-3,-6.3649192E-3,1.1365804E-1,1.1537318E-2,-2.7830838E-3,3.769484E-3,-1.1923045E-1,-1.8411828E-2,1.2373442E-1,-5.7556186E-2,2.5181212E-3,-1.7147829E-1,9.317816E-3,1.8465568E-3,-2.0184373E-3,2.970039E-3,-1.5205751E-1,3.5558373E-2,-1.2638E-2,-4.9257944E-3,1.403125E-1,-5.576296E-2,6.37549E-2,1.5728036E-2,1.223346E-3,1.9940706E-1,1.9347008E-1,2.7769467E-2,3.4512606E-1,1.4774214E-1,-2.9514432E-1,-3.787319E-3,-2.4557626E-2,1.4943415E-1,1.8632688E-1,-4.8825696E-2,2.4663962E-1,2.5851052E-4,2.2136332E-2,2.594325E-1,-2.0982102E-3,-9.793705E-3,8.3466604E-2,-6.944867E-2,-2.0547046E-1,-8.472196E-2,-2.4693087E-1,-5.0113213E-1,-5.3714877E-3,1.0630402E-3,-1.9038081E-1,-2.4942355E-3,-5.5352323E-2,-1.2881768E-1,4.1150147E-4,-1.085934E-1,1.4559372E-1,1.7281697E-3,4.9157977E-2,-3.2522965E-3,4.9975786E-2,1.3534234E-1,-7.374466E-2,-1.592257E-1,8.654449E-3,2.2886379E-1,-7.214811E-2,6.4625785E-2,-9.920181E-3,-8.628414E-2,-1.0316147E-1,-1.8755664E-1,5.285822E-3,-1.9861744E-3,6.869763E-3,2.3982204E-1,-2.2564267E-1,-3.225586E-2,4.823955E-2,1.2765558E-1,9.575527E-3,-4.4171803E-2,-8.123944E-3,2.1489325E-1,2.0324134E-3,2.4168995E-1,1.7200488E-1,1.4896766E-2,2.3276186E-2,7.764454E-3,1.821623E-1,7.1011625E-2,-1.9913478E-2,-7.123381E-3,5.2965702E-3,-5.855496E-2,1.1701211E-2,4.5102548E-5,1.4630614E-1,2.5681186E-1,5.1061134E-3,-9.578818E-3,2.6069802E-1,7.271386E-4,6.1094803E-3,1.4370877E-2,8.1768595E-3,1.2971656E-3,-5.282327E-3,8.145894E-4,-8.591934E-3,-1.3437568E-2,6.443199E-3,-5.7020662E-3,-1.7430551E-2,-5.7037845E-3,-1.721999E-2,-3.2543782E-2,-7.6639764E-3,-1.3963998E-2,-3.2361702E-3,7.5716805E-3,-9.173848E-3,-1.9137093E-3,-8.173377E-3,-1.5110981E-3,8.821617E-3,3.6543685E-3,-1.418511E-3,3.6793407E-3,4.665129E-3,-1.4711097E-3,7.6119932E-3,2.0963924E-3,-8.81407E-3,-1.9634773E-3,-4.904091E-3,-1.0001297E-2,-3.8147944E-3,1.8241527E-3,1.5323709E-2,-1.9618319E-4,-1.721013E-2,-3.2759046E-3,8.148102E-3,-2.1990887E-3,-1.5002395E-3,-6.2601557E-3,-1.8253014E-3,-6.3038375E-3,-5.851355E-3,-1.1720357E-2,-2.9314565E-3,3.0149792E-3,6.390972E-3,1.5064496E-2,-3.0314478E-3,-2.1439902E-2,-1.5550773E-4,-8.330616E-3,3.748215E-3,-2.3757559E-4,7.4001164E-3,-3.102332E-3,-5.2803154E-3,8.683178E-3,1.30585255E-2,5.1071113E-3,1.4010217E-2,6.8407403E-3,2.678755E-3,1.27913635E-2,-2.6691272E-3,1.9316056E-3,5.7834215E-3,1.1178599E-2,5.608734E-3,1.2397453E-3,-7.0940885E-3,6.7902077E-4,1.6670054E-3,9.118204E-3,1.4377757E-2,6.204487E-4,3.2608337E-3,1.3785646E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,59,61,-1,63,65,67,69,71,-1,73,75,-1,77,79,-1,81,83,85,87,89,-1,-1,-1,91,-1,93,95,97,99,101,103,105,107,109,-1,-1,111,113,115,-1,-1,117,119,121,-1,123,125,127,129,131,133,135,-1,137,139,141,143,145,-1,147,149,151,153,-1,-1,155,157,-1,159,161,163,-1,-1,-1,165,167,169,-1,-1,-1,-1,171,-1,-1,-1,173,-1,175,177,-1,179,-1,-1,-1,-1,181,183,-1,-1,185,187,189,-1,191,193,195,197,199,201,203,-1,205,207,209,211,213,-1,-1,215,-1,-1,217,219,221,223,225,227,-1,-1,229,-1,231,233,-1,235,237,-1,239,-1,241,243,245,247,249,251,253,255,-1,257,259,261,-1,-1,263,265,267,269,271,273,-1,275,-1,277,-1,279,281,283,-1,-1,285,287,-1,-1,-1,289,-1,-1,291,293,-1,-1,295,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5237437E1,4.0874853E0,6.180131E0,2.549736E0,2.8662376E0,3.4985182E0,2.4137478E0,1.4220886E0,0E0,6.327411E-1,4.3409634E-1,1.2051692E0,2.2418044E0,1.4666171E0,5.418043E-1,3.3478427E-1,1.0552568E0,2.3388791E-1,2.653429E-1,2.4355814E-1,1.2267375E-1,1.0072736E0,3.993467E-1,4.552154E-1,2.142074E0,8.141631E-1,7.8724813E-1,9.034872E-2,3.917675E-1,0E0,0E0,2.7774015E-1,7.2385216E-1,2.3727036E-1,0E0,1.771703E-1,1.403387E-1,8.546482E-2,1.2070477E-1,8.596301E-2,0E0,6.601565E-1,1.0473384E0,0E0,5.773285E-2,2.359556E-1,0E0,1.1474978E0,9.4691086E-1,6.069219E-1,7.0384556E-1,8.60981E-1,0E0,0E0,0E0,3.93816E-1,0E0,1.681987E-1,1.6136229E-1,8.054218E-1,2.6035026E-1,2.0071268E-1,1.02100074E-1,8.1847906E-2,6.365232E-2,2.408477E-2,0E0,0E0,1.6861781E-2,5.504179E-2,4.6885133E-2,0E0,0E0,3.171556E-1,1.3068057E0,2.6216674E-1,0E0,4.552518E-2,3.018878E-2,3.0372152E-1,4.2907476E-2,6.964151E-1,8.4605455E-1,7.708547E-1,0E0,4.6245885E-1,2.2302926E-1,1.5919387E-1,2.5645146E-1,5.3753567E-1,0E0,2.1805525E-1,3.02454E-1,7.4662924E-2,2.683832E-1,0E0,0E0,6.253576E-1,2.9627204E-1,0E0,4.5205344E-2,8.33748E-2,1.9021952E-1,0E0,0E0,0E0,4.388862E-2,4.675819E-2,6.090148E-2,0E0,0E0,0E0,0E0,4.9853265E-2,0E0,0E0,0E0,9.377247E-2,0E0,5.5454046E-1,5.306091E-1,0E0,5.2024186E-2,0E0,0E0,0E0,0E0,2.9252827E-2,7.295943E-2,0E0,0E0,2.7720782E-1,5.238412E-1,3.7752974E-1,0E0,2.245629E-1,6.403775E-1,1.3752395E-1,2.0353128E-1,1.3373423E-1,5.837339E-2,1.16714954E-1,0E0,1.7043816E-1,1.3825089E-1,3.8555574E-1,2.5693673E-1,2.213378E-1,0E0,0E0,6.1452866E-2,0E0,0E0,9.7189486E-2,7.63649E-2,5.493183E-1,3.538884E-1,1.6315705E-1,5.757761E-2,0E0,0E0,4.9425006E-2,0E0,2.2016451E-1,2.6569813E-1,0E0,5.536516E-2,1.1761069E-2,0E0,4.38507E-2,0E0,4.409907E-2,4.1232944E-2,9.562187E-2,5.535984E-2,5.7240516E-2,4.3595243E-1,4.508617E-1,3.4940913E-1,0E0,1.5986584E-2,1.6845927E-2,2.2207022E-2,0E0,0E0,3.8583234E-2,4.4259667E-2,5.10966E-1,4.6190822E-1,4.4447488E-1,3.001504E-1,0E0,2.9987112E-1,0E0,5.1765394E-1,0E0,2.2441924E-2,7.577422E-2,1.6925865E-1,0E0,0E0,2.378881E-2,1.6266905E-2,0E0,0E0,0E0,1.824812E-1,0E0,0E0,3.9436507E-1,3.9712667E-1,0E0,0E0,1.7507267E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,47,47,48,48,49,49,50,50,51,51,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,68,68,69,69,70,70,73,73,74,74,75,75,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,91,91,92,92,93,93,94,94,97,97,98,98,100,100,101,101,102,102,106,106,107,107,108,108,113,113,117,117,119,119,120,120,122,122,127,127,128,128,131,131,132,132,133,133,135,135,136,136,137,137,138,138,139,139,140,140,141,141,143,143,144,144,145,145,146,146,147,147,150,150,153,153,154,154,155,155,156,156,157,157,158,158,161,161,163,163,164,164,166,166,167,167,169,169,171,171,172,172,173,173,174,174,175,175,176,176,177,177,178,178,180,180,181,181,182,182,185,185,186,186,187,187,188,188,189,189,190,190,192,192,194,194,196,196,197,197,198,198,201,201,202,202,206,206,209,209,210,210,213,213],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,60,62,-1,64,66,68,70,72,-1,74,76,-1,78,80,-1,82,84,86,88,90,-1,-1,-1,92,-1,94,96,98,100,102,104,106,108,110,-1,-1,112,114,116,-1,-1,118,120,122,-1,124,126,128,130,132,134,136,-1,138,140,142,144,146,-1,148,150,152,154,-1,-1,156,158,-1,160,162,164,-1,-1,-1,166,168,170,-1,-1,-1,-1,172,-1,-1,-1,174,-1,176,178,-1,180,-1,-1,-1,-1,182,184,-1,-1,186,188,190,-1,192,194,196,198,200,202,204,-1,206,208,210,212,214,-1,-1,216,-1,-1,218,220,222,224,226,228,-1,-1,230,-1,232,234,-1,236,238,-1,240,-1,242,244,246,248,250,252,254,256,-1,258,260,262,-1,-1,264,266,268,270,272,274,-1,276,-1,278,-1,280,282,284,-1,-1,286,288,-1,-1,-1,290,-1,-1,292,294,-1,-1,296,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,3.474851E8,1.3144558E4,6.860185E2,1.7669278E5,1.2974394E6,3E0,2.6970001E-2,1.6164155E8,7.683389E7,1.5694646E3,1.2E1,1.3359244E9,2.612E3,1.0503613E5,1.3651616E5,4.8297736E7,5.8426323E0,1.5896305E-3,2.2056E4,1E0,5.1E1,1.0872E4,5.862126E2,2.1506184E7,4.4137526E0,4.760537E8,1.8222668E2,-1.0326594E-2,-2.9136509E-2,9.457831E0,4.77721E3,1E0,-1.6495377E-2,2.1924436E5,9.3915343E-1,1.4E2,6.298144E0,4.7777777E0,4.254096E-3,4.7E2,4E0,1.7909545E-2,2.429717E1,3.475199E9,7.406909E-3,1E0,5.3534385E3,2.200423E3,2.222456E0,3.7826266E4,2.846274E-2,6.7595397E-3,-9.008128E-4,1.980161E0,1.0862014E-3,1E0,5.704748E4,5.5743E4,3.0612E4,1.04E3,4.8259888E-2,2.8729197E8,3.2931E4,3.097E3,-1.0214529E-2,3.81616E-3,7.55E2,1.27E2,6.932264E8,2.120232E-2,1.0569594E-2,3.09627E5,3.22E2,1.3446785E7,-2.7053256E-2,4.382419E6,8.9E1,1.1598511E3,4.0728608E3,5E-1,7.064848E6,1E0,2.7354132E-2,1.8367347E0,4.556146E6,6.6288E4,3.970405E3,3.0272608E10,-1.2801717E-2,2.185321E0,1.5846036E7,4.5216934E4,6.53143E6,-1.6852584E-2,-2.5982966E-3,1.1210787E3,1.0828989E8,1.0837223E-2,4E1,1.46617E5,3.3709784E5,-1.6341954E-2,-5.8240555E-3,2.7308923E-3,2.9788235E-2,1.0119178E10,8.874407E6,-3.3537662E-3,1.585864E-3,-1.9201749E-3,-6.3649192E-3,1.0855529E3,1.1537318E-2,-2.7830838E-3,3.769484E-3,1E0,-1.8411828E-2,3.79E2,6.726E3,2.5181212E-3,1.862E3,9.317816E-3,1.8465568E-3,-2.0184373E-3,2.970039E-3,1.2068E4,1.15062E5,-1.2638E-2,-4.9257944E-3,1.315782E-5,5.8601086E1,1.0927339E7,1.5728036E-2,1.752E3,8.9928055E-1,3.3968747E-1,1.1396227E1,1.2935566E7,7.5E1,3.4613644E1,-3.787319E-3,1.8835404E3,3.5296965E-1,3.29774E5,9.2339675E5,1E0,2.5851052E-4,2.2136332E-2,1.3078561E7,-2.0982102E-3,-9.793705E-3,2.28125E0,6.6126086E4,3.0161016E7,2.1690162E7,6.567044E2,2.894E3,-5.3714877E-3,1.0630402E-3,1.2471935E7,-2.4942355E-3,7.997723E6,1.1400756E6,4.1150147E-4,8.836364E0,3.0996E4,1.7281697E-3,2.819136E0,-3.2522965E-3,4.52737E5,3.3968747E-1,9.2E0,2.05E2,1E0,1.176582E6,3.983889E3,1.84556E7,-9.920181E-3,1.13E2,7.285643E-1,6.0292965E-1,5.285822E-3,-1.9861744E-3,4.5865917E8,1.3860265E6,8E0,1.325204E6,2E0,1.7049885E0,9.575527E-3,1.612E3,-8.123944E-3,7.2E1,2.0324134E-3,3.362766E1,6.005171E8,1.1164466E6,2.3276186E-2,7.764454E-3,1.0545219E5,1E0,-1.9913478E-2,-7.123381E-3,5.2965702E-3,5.683493E6,1.1701211E-2,4.5102548E-5,1E0,2.2576077E10,5.1061134E-3,-9.578818E-3,1E0,7.271386E-4,6.1094803E-3,1.4370877E-2,8.1768595E-3,1.2971656E-3,-5.282327E-3,8.145894E-4,-8.591934E-3,-1.3437568E-2,6.443199E-3,-5.7020662E-3,-1.7430551E-2,-5.7037845E-3,-1.721999E-2,-3.2543782E-2,-7.6639764E-3,-1.3963998E-2,-3.2361702E-3,7.5716805E-3,-9.173848E-3,-1.9137093E-3,-8.173377E-3,-1.5110981E-3,8.821617E-3,3.6543685E-3,-1.418511E-3,3.6793407E-3,4.665129E-3,-1.4711097E-3,7.6119932E-3,2.0963924E-3,-8.81407E-3,-1.9634773E-3,-4.904091E-3,-1.0001297E-2,-3.8147944E-3,1.8241527E-3,1.5323709E-2,-1.9618319E-4,-1.721013E-2,-3.2759046E-3,8.148102E-3,-2.1990887E-3,-1.5002395E-3,-6.2601557E-3,-1.8253014E-3,-6.3038375E-3,-5.851355E-3,-1.1720357E-2,-2.9314565E-3,3.0149792E-3,6.390972E-3,1.5064496E-2,-3.0314478E-3,-2.1439902E-2,-1.5550773E-4,-8.330616E-3,3.748215E-3,-2.3757559E-4,7.4001164E-3,-3.102332E-3,-5.2803154E-3,8.683178E-3,1.30585255E-2,5.1071113E-3,1.4010217E-2,6.8407403E-3,2.678755E-3,1.27913635E-2,-2.6691272E-3,1.9316056E-3,5.7834215E-3,1.1178599E-2,5.608734E-3,1.2397453E-3,-7.0940885E-3,6.7902077E-4,1.6670054E-3,9.118204E-3,1.4377757E-2,6.204487E-4,3.2608337E-3,1.3785646E-2],"split_indices":[20,102,7,52,52,28,28,3,0,12,32,52,18,7,2,33,28,45,53,38,9,6,3,2,52,45,39,7,58,0,0,54,52,77,0,28,57,10,57,54,0,1,53,0,53,5,0,102,4,52,54,52,0,0,0,42,0,8,28,10,2,2,42,5,9,2,0,0,10,8,7,0,0,9,29,45,0,1,10,52,4,53,28,102,0,53,48,10,52,5,0,41,47,28,29,0,0,52,45,0,8,9,33,0,0,0,38,19,45,0,0,0,0,52,0,0,0,13,0,2,2,0,2,0,0,0,0,9,29,0,0,38,50,9,0,29,53,38,58,47,3,56,0,4,27,11,47,104,0,0,51,0,0,54,28,45,45,52,0,0,0,45,0,9,33,0,58,9,0,54,0,29,38,55,1,13,5,28,45,0,0,27,27,0,0,5,28,8,9,17,38,0,0,0,8,0,58,7,50,0,0,28,15,0,0,0,1,0,0,102,12,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.59E3,8.35E2,1.755E3,4.55E2,3.8E2,1.222E3,5.33E2,4.51E2,4E0,2.91E2,8.9E1,4.69E2,7.53E2,3.92E2,1.41E2,1.5E1,4.36E2,2.1E2,8.1E1,6.8E1,2.1E1,4.43E2,2.6E1,6.5E1,6.88E2,2.25E2,1.67E2,1.4E1,1.27E2,5E0,1E1,6.5E1,3.71E2,2.06E2,4E0,6.5E1,1.6E1,1.4E1,5.4E1,1.6E1,5E0,4.01E2,4.2E1,6E0,2E1,6E1,5E0,5.51E2,1.37E2,1.63E2,6.2E1,1.63E2,4E0,7E0,7E0,1.21E2,6E0,5.7E1,8E0,3.57E2,1.4E1,1.94E2,1.2E1,2.2E1,4.3E1,8E0,8E0,4E0,1E1,4.5E1,9E0,4E0,1.2E1,6.2E1,3.39E2,3.5E1,7E0,9E0,1.1E1,3.9E1,2.1E1,1.52E2,3.99E2,1.31E2,6E0,1.29E2,3.4E1,1.6E1,4.6E1,1.59E2,4E0,7.2E1,4.9E1,1.3E1,4.4E1,4E0,4E0,3.32E2,2.5E1,4E0,1E1,3.3E1,1.61E2,6E0,6E0,5E0,1.7E1,1.7E1,2.6E1,4E0,4E0,6E0,4E0,4E1,5E0,5E0,4E0,5.7E1,5E0,4.5E1,2.94E2,6E0,2.9E1,5E0,4E0,4E0,7E0,2.7E1,1.2E1,1.7E1,4E0,2E1,1.32E2,3.86E2,1.3E1,2.4E1,1.07E2,1.9E1,1.1E2,8E0,2.6E1,1E1,6E0,3.6E1,1E1,1.49E2,1E1,6.8E1,4E0,2.5E1,2.4E1,7E0,6E0,2.1E1,2.3E1,2.79E2,5.3E1,1.4E1,1.1E1,6E0,4E0,2.7E1,6E0,1.07E2,5.4E1,4E0,1.3E1,1E1,7E0,2.1E1,5E0,1.1E1,2.9E1,2.8E1,2.9E1,2.2E1,2.3E1,2.63E2,3.1E1,2.1E1,8E0,1.3E1,1.4E1,6E0,6E0,9E0,1.1E1,1.5E1,1.17E2,3.12E2,7.4E1,4E0,2E1,4E0,1.03E2,5E0,1.4E1,8E0,1.02E2,4E0,4E0,1.7E1,9E0,5E0,5E0,7E0,2.9E1,6E0,4E0,9.7E1,5.2E1,5E0,5E0,6.4E1,4E0,5E0,1.9E1,8E0,1.3E1,1.6E1,7E0,1.85E2,9.4E1,6E0,4.7E1,7E0,7E0,7E0,4E0,2.1E1,6E0,1.03E2,4E0,3.3E1,2.1E1,7E0,6E0,6E0,4E0,5E0,1.6E1,7E0,4E0,2.4E1,5E0,6E0,2.2E1,1.3E1,1.6E1,5E0,1.7E1,1.7E1,6E0,5E0,2.58E2,1.6E1,1.5E1,4E0,4E0,4E0,9E0,7E0,7E0,4E0,5E0,5E0,6E0,9E0,6E0,9.7E1,2E1,2.07E2,1.05E2,6.7E1,7E0,1.6E1,4E0,7.2E1,3.1E1,9E0,5E0,4E0,4E0,2.6E1,7.6E1,8E0,9E0,4E0,5E0,1.3E1,1.6E1,2.4E1,7.3E1,4.6E1,6E0,5E0,5.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"297","size_leaf_vector":"1"}},{"base_weights":[-2.344054E-3,3.4825496E-2,-1.3240069E-1,-1.0174204E-3,1.6456091E-1,-2.1551351E-1,-2.7739972E-2,-4.578984E-2,7.99935E-2,1.3684346E-1,3.7785125E-1,-2.4350414E-1,-9.879725E-2,-5.525717E-2,1.07931405E-1,-7.966507E-2,8.965917E-4,4.6827924E-2,1.4466427E-1,9.9559315E-2,3.0213457E-1,6.801134E-1,2.1069042E-1,-2.1843538E-1,-4.3133536E-1,5.13141E-2,-1.467761E-1,-4.306332E-2,-1.7952684E-2,6.3871793E-3,2.1400693E-1,-1.13064796E-1,-1.6108835E-2,2.5420621E-2,-2.0117958E-1,6.267349E-2,-8.506865E-2,1.9330016E-1,5.7983525E-2,1.3841349E-1,7.151502E-3,1.6966294E-1,3.3222276E-1,3.754006E-2,1.7938035E-2,2.9399383E-1,-7.0313783E-3,-2.6465183E-1,-1.7350109E-1,-6.095161E-1,-2.2439899E-1,9.889729E-3,-2.8204196E-3,-1.0748032E-1,-2.4430467E-1,-5.1728908E-2,1.0403103E-2,5.30661E-2,-4.935947E-2,1.6249661E-1,1.5774474E-2,-9.4315745E-2,-2.3855697E-1,6.1138673E-3,-1.5442541E-1,7.257217E-2,-3.551825E-2,-2.883744E-1,-1.5402013E-1,4.229307E-2,1.5185444E-1,-1.3121758E-1,2.0872785E-2,2.2243474E-1,1.5846716E-2,-6.065152E-2,1.1480528E-1,7.760061E-2,1.819916E-1,-1.8654835E-1,4.338083E-2,1.170813E-2,3.112743E-3,3.7038374E-1,2.1367136E-1,5.024155E-4,3.746404E-1,-2.318213E-1,-4.0969166E-1,-1.5815006E-1,-2.4423914E-2,-1.7684946E-2,-3.7004184E-2,-1.412758E-1,-1.810743E-2,-1.3361279E-2,-7.3060274E-2,-7.288905E-3,-1.6730156E-2,-1.9908401E-1,-3.856273E-2,6.3534675E-4,4.2889346E-3,-4.257291E-3,5.342853E-4,1.0496022E-2,5.6438474E-3,-1.09437965E-1,-4.12658E-3,-1.9207224E-1,-3.0440096E-2,3.166653E-2,-7.817206E-2,-2.7845335E-1,-4.400112E-2,2.9144335E-1,5.2205402E-2,-6.1245944E-2,1.0537109E-1,-9.794725E-3,-2.1678928E-2,-1.7190465E-1,-1.1523719E-3,-3.586551E-2,7.5979374E-2,7.957754E-2,2.0327221E-1,-1.6187549E-1,2.1077874E-3,4.57007E-3,-2.0662853E-2,1.728471E-1,3.2238287E-1,-1.3049183E-2,1.4247598E-1,6.3988687E-3,-9.352303E-3,8.683967E-2,2.2323345E-1,-3.803701E-2,1.05488956E-1,1.23670526E-1,2.3838033E-1,-1.7584452E-2,-9.4418816E-2,4.474531E-3,1.59876E-1,2.7255645E-1,4.4248202E-1,4.7270753E-3,1.377336E-2,2.3547012E-2,1.1054603E-2,-2.7190682E-1,-1.1351582E-1,-7.0687737E-3,-2.6112782E-2,-8.044366E-2,-2.1145988E-1,-2.7626622E-3,-1.166225E-2,-1.2572609E-1,9.532866E-3,-1.0716316E-1,-1.6710097E-2,-1.4264898E-1,-2.3453746E-2,-7.540198E-2,-1.5725698E-1,-4.774216E-2,1.5088561E-1,-6.35782E-2,-2.6781583E-1,-6.360455E-2,5.949259E-2,8.72117E-2,-1.274116E-1,-6.0929195E-3,-1.6712157E-2,-8.227654E-2,2.8311168E-3,4.921743E-3,1.8972663E-2,4.2434737E-2,1.7796418E-2,-5.1914845E-2,-1.5223285E-2,-5.6726136E-4,1.4897214E-1,-9.929505E-3,-5.184548E-3,4.777302E-2,-8.5037224E-2,-3.7138734E-2,9.5882766E-2,8.569445E-3,1.5415172E-2,2.1194336E-3,2.3508996E-1,-2.071066E-1,-6.347843E-2,2.130589E-3,-3.989006E-3,2.0165803E-1,7.681169E-2,3.6909336E-1,2.531816E-3,9.832144E-3,8.116684E-4,5.04449E-3,-4.576007E-2,1.5141709E-1,4.872722E-2,1.4109914E-2,5.9677023E-3,1.4049292E-2,-9.328202E-3,1.9466105E-1,6.790214E-2,1.4482735E-1,-7.1746944E-3,1.1332063E-1,2.680363E-1,-1.6573597E-3,-6.229847E-3,5.710781E-2,-3.3662688E-2,2.8133875E-1,-2.8723574E-3,3.0006024E-1,6.497315E-3,2.3642926E-2,1.2166621E-2,-3.1831184E-1,-1.5289028E-1,-1.8660605E-1,3.2541638E-3,-2.583056E-1,-3.2775372E-2,-1.3202189E-3,-2.2595613E-1,-7.841399E-2,-1.075116E-2,4.4066375E-3,-2.5309494E-3,-2.101464E-3,-8.838039E-3,-7.6089725E-2,-2.3773117E-1,-6.477835E-2,2.1350361E-2,-6.4548436E-3,-5.7600802E-5,-9.048034E-3,-2.5282733E-4,-1.2460626E-3,-6.645143E-3,1.7284513E-3,1.4166522E-2,-5.065186E-4,-7.300574E-3,-2.6187945E-2,-6.950035E-3,1.4787026E-3,-5.4660505E-3,3.4439804E-3,-5.3783623E-3,7.1372944E-3,1.5060243E-4,-3.1184861E-3,-1.0319388E-2,-1.2711113E-3,-6.7392057E-3,6.181801E-4,4.9136E-3,-3.1887766E-3,2.1031948E-3,8.8599725E-3,1.7059206E-3,6.145827E-3,-9.1876285E-3,-5.8337166E-3,2.1878823E-3,1.5090061E-3,-1.8308083E-2,-1.1640081E-4,5.6096762E-3,-5.586708E-3,3.8845341E-3,1.2534115E-2,5.460056E-3,-6.2066303E-3,-1.2212841E-2,-6.8328576E-3,1.1236076E-3,-7.160637E-4,1.095388E-2,-2.6876188E-3,7.324814E-3,2.02521E-2,6.0671624E-3,-3.5899763E-3,-6.0190045E-4,2.993913E-3,9.095174E-3,4.0476783E-3,-2.5214872E-3,-2.374115E-3,5.711849E-3,4.7564544E-4,1.1755899E-2,8.381373E-4,6.0111266E-3,1.5997829E-2,6.254944E-3,-2.0200058E-3,1.707016E-3,8.39327E-3,-9.322293E-4,1.9894233E-2,1.0882357E-2,-5.2564376E-4,5.787431E-3,-5.163425E-3,-1.4441408E-4,2.9536749E-3,2.062828E-2,8.285552E-3,1.6538367E-2,-1.8828614E-2,-1.2196825E-2,2.8384877E-3,-1.1091168E-2,-1.5337195E-2,-6.510536E-3,-6.2997187E-3,-1.825368E-2,-3.7377372E-3,4.047727E-3,-1.404807E-2,-8.60597E-3,-5.364482E-3,-2.4167694E-4,-6.4950655E-3,1.320234E-3,-1.6548062E-2,-4.8334803E-3,-1.3808499E-3,-5.342785E-3,4.7837626E-3,-4.7488953E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,-1,85,-1,87,89,91,93,-1,-1,95,97,99,-1,101,103,105,-1,107,109,111,113,115,117,119,121,123,125,127,129,131,133,135,137,139,141,143,145,-1,-1,147,149,-1,151,153,155,157,-1,-1,-1,159,-1,-1,161,-1,-1,163,165,-1,-1,-1,-1,-1,-1,167,169,171,-1,173,175,177,179,181,183,185,187,-1,-1,189,-1,191,193,195,197,199,-1,-1,201,203,205,-1,207,209,-1,211,213,215,217,219,221,-1,223,225,227,229,231,-1,-1,-1,-1,233,235,-1,-1,237,239,-1,-1,241,243,245,-1,247,249,251,253,255,257,259,261,263,265,267,269,-1,-1,271,-1,-1,-1,273,-1,275,-1,-1,277,-1,-1,279,281,283,285,-1,287,-1,289,291,293,-1,-1,295,297,299,-1,-1,-1,-1,301,303,305,-1,-1,307,-1,309,311,313,315,317,319,-1,-1,321,323,325,-1,327,-1,-1,-1,329,331,333,-1,335,337,-1,339,341,-1,-1,-1,-1,-1,343,345,347,349,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2375192E1,9.258507E0,4.943552E0,5.665496E0,2.4963093E0,1.01157E0,9.5048803E-1,1.5910742E0,1.1857588E0,2.3302484E0,2.3324995E0,1.1064978E0,4.612683E-1,7.667828E-1,4.6180147E-1,1.2357478E0,2.10634E0,7.765324E-1,7.8136134E-1,1.1264052E0,2.2264242E-1,1.18896484E-1,1.0074067E0,4.254942E-1,9.3251514E-1,2.6395577E-1,1.5770018E-1,4.5494977E-1,0E0,6.2469766E-2,5.8702826E-2,8.762922E-1,6.2184525E-1,1.091582E0,1.4473927E-1,5.9336734E-1,1.9830224E-1,6.168785E-1,4.7209567E-1,5.715513E-1,6.677486E-1,8.9530975E-2,1.7420578E-1,0E0,0E0,6.1449766E-1,0E0,4.2881584E-1,4.9384212E-1,2.4033594E-1,1.2085712E-1,0E0,0E0,1.7862809E-1,6.698692E-2,3.7646317E-1,0E0,1.6098257E-2,2.4588162E-2,1.3977557E-2,0E0,4.550774E-1,7.2723913E-1,3.793781E-1,3.7122732E-1,9.387325E-1,6.0816336E-1,1.1649406E-1,7.0982814E-2,7.136009E-1,2.1051931E-1,1.5568E-1,4.0005665E-2,4.5476484E-1,6.6859245E-1,1.9631058E-1,1.2313074E-1,3.071571E-1,3.9146662E-1,1.9531107E-1,3.618763E-1,0E0,0E0,1.685791E-1,9.199762E-2,0E0,2.0697379E-1,4.054289E-1,5.014932E-1,4.5523572E-1,0E0,0E0,0E0,7.423082E-2,0E0,0E0,1.3604341E-1,0E0,0E0,1.6321236E-1,2.84447E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5362663E-1,3.3849E-1,4.2093968E-1,0E0,3.616716E-1,3.4499207E-1,7.31833E-2,6.630741E-2,2.6913357E-1,5.699555E-1,3.0445087E-1,1.3401264E-1,0E0,0E0,3.055036E-2,0E0,3.4129974E-1,4.2908192E-1,1.5660687E-1,1.6916001E-1,9.3667865E-2,0E0,0E0,3.7115168E-2,1.8095827E-1,4.001286E-1,0E0,8.590749E-2,8.353273E-2,0E0,8.86088E-2,2.1534204E-2,1.4973426E-1,2.4151081E-1,1.8097657E-1,2.0698738E-1,0E0,1.6252808E-2,1.243669E-1,5.4419947E-1,3.9380908E-2,1.9145489E-2,0E0,0E0,0E0,0E0,3.2689524E-1,3.3857864E-1,0E0,0E0,3.9849594E-1,1.7374778E-1,0E0,0E0,6.352872E-2,6.6010736E-2,4.248877E-2,0E0,1.2746885E-1,2.9947644E-1,6.7470145E-1,4.2695308E-1,7.2501436E-2,1.5986148E-1,7.790838E-2,8.508363E-1,1.3552971E-1,1.7295656E-1,4.6408124E-2,1.516679E-1,0E0,0E0,3.2442942E-2,0E0,0E0,0E0,3.2145867E-1,0E0,1.5598711E-1,0E0,0E0,5.5790126E-2,0E0,0E0,5.628178E-1,2.1756583E-1,6.5406865E-1,2.5935376E-1,0E0,1.4349419E-1,0E0,3.0090094E-2,1.9126236E-2,5.9763268E-2,0E0,0E0,2.0801759E-1,1.719292E-1,2.0195317E-1,0E0,0E0,0E0,0E0,8.8091325E-3,2.8301448E-2,8.7337516E-2,0E0,0E0,9.802006E-2,0E0,1.6318506E-1,1.4533526E-1,1.5281844E-1,1.512691E-2,1.0412255E-1,2.4669504E-1,0E0,0E0,1.0446356E-1,7.688761E-2,3.5549688E-1,0E0,1.1542201E-2,0E0,0E0,0E0,1.16447926E-1,3.1681892E-1,8.19788E-2,0E0,9.543967E-2,1.925676E-1,0E0,1.3540983E-1,2.92987E-2,0E0,0E0,0E0,0E0,0E0,8.631284E-2,8.655012E-2,1.2854898E-1,1.2948722E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84,86,86,87,87,88,88,89,89,93,93,96,96,99,99,100,100,107,107,108,108,109,109,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,121,121,123,123,124,124,125,125,126,126,127,127,130,130,131,131,132,132,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,144,144,145,145,146,146,147,147,148,148,153,153,154,154,157,157,158,158,161,161,162,162,163,163,165,165,166,166,167,167,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,179,179,183,183,185,185,188,188,191,191,192,192,193,193,194,194,196,196,198,198,199,199,200,200,203,203,204,204,205,205,210,210,211,211,212,212,215,215,217,217,218,218,219,219,220,220,221,221,222,222,225,225,226,226,227,227,229,229,233,233,234,234,235,235,237,237,238,238,240,240,241,241,247,247,248,248,249,249,250,250],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,-1,86,-1,88,90,92,94,-1,-1,96,98,100,-1,102,104,106,-1,108,110,112,114,116,118,120,122,124,126,128,130,132,134,136,138,140,142,144,146,-1,-1,148,150,-1,152,154,156,158,-1,-1,-1,160,-1,-1,162,-1,-1,164,166,-1,-1,-1,-1,-1,-1,168,170,172,-1,174,176,178,180,182,184,186,188,-1,-1,190,-1,192,194,196,198,200,-1,-1,202,204,206,-1,208,210,-1,212,214,216,218,220,222,-1,224,226,228,230,232,-1,-1,-1,-1,234,236,-1,-1,238,240,-1,-1,242,244,246,-1,248,250,252,254,256,258,260,262,264,266,268,270,-1,-1,272,-1,-1,-1,274,-1,276,-1,-1,278,-1,-1,280,282,284,286,-1,288,-1,290,292,294,-1,-1,296,298,300,-1,-1,-1,-1,302,304,306,-1,-1,308,-1,310,312,314,316,318,320,-1,-1,322,324,326,-1,328,-1,-1,-1,330,332,334,-1,336,338,-1,340,342,-1,-1,-1,-1,-1,344,346,348,350,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,7.431E3,1E0,6.5029144E5,8.791777E3,6.726E3,1.7227725E0,3.6129813E2,6.7699623E-1,1.546644E6,3.2475834E7,7.8351054E11,4.8297736E7,1.4299594E8,1.4340747E3,1.477425E6,4.0844156E7,3E0,3.2449896E0,2.6944466E7,4.7347268E-1,2.0399538E5,1.27E2,1.7314286E0,2.194E3,7.6642185E-2,2.894E3,2.3274304E2,-1.7952684E-2,2.6654E4,1.4386049E6,9.837297E0,3.3704692E10,1.0938637E1,1.1863768E-4,4.5723195E6,3.0996E4,1.9162654E7,4.1724915E2,1.2679E4,2.308943E0,1.2E1,1.7384394E7,3.754006E-2,1.7938035E-2,1.5963264E9,-7.0313783E-3,1.9577E4,3.5861215E6,1E0,1E0,9.889729E-3,-2.8204196E-3,4E0,4.7563137E1,5.2143492E-5,1.0403103E-2,4.9978513E-1,2.3361E4,1.3264E4,1.5774474E-2,5.4907064E0,2E1,1.8E1,6.783E3,3.1E1,7.06699E8,2.3067484E0,2.3276744E5,2.6682352E2,8.15E2,6.624E3,7E0,5.894636E6,5.42974E5,4.3597875E0,1.9015984E6,3.3737933E8,1.6649964E5,4.5162E4,7.3610186E3,1.170813E-2,3.112743E-3,1.5445492E3,2.5247778E6,5.024155E-4,3.6655E4,1.6772E4,5.2187E4,1.504779E7,-2.4423914E-2,-1.7684946E-2,-3.7004184E-2,6.2E1,-1.810743E-2,-1.3361279E-2,4.6726016E3,-7.288905E-3,-1.6730156E-2,8.2892E4,5.408015E0,6.3534675E-4,4.2889346E-3,-4.257291E-3,5.342853E-4,1.0496022E-2,5.6438474E-3,3.8866E4,1.763E3,2E0,-3.0440096E-2,8.242408E4,4.4E1,2.4176333E0,2.7258065E0,4.273439E2,1.8552577E2,3.2627738E5,3.9739098E4,-9.794725E-3,-2.1678928E-2,6.015E3,-1.1523719E-3,5.088781E0,1.0195312E0,4.144144E-1,4.7233105E6,1E1,2.1077874E-3,4.57007E-3,9.4446504E7,5.077728E8,1E0,-1.3049183E-2,8.640299E8,1.97E2,-9.352303E-3,4.1844E4,4.29E2,1.2138641E7,4.29E2,1.428111E5,9.389757E4,-1.7584452E-2,2E0,3.7794448E3,1.066317E12,1.533115E6,2.1708366E7,4.7270753E-3,1.377336E-2,2.3547012E-2,1.1054603E-2,7.4872833E-1,1E0,-7.0687737E-3,-2.6112782E-2,4.5555557E1,1.9928571E0,-2.7626622E-3,-1.166225E-2,5.3842196E9,1.3298E4,9.213145E0,-1.6710097E-2,3.9516E4,5.351875E2,2.73E2,3.402546E6,2.5E1,1.35E2,4.0729207E3,6.239726E0,1.7997802E0,1E0,9.25E0,8.5518695E4,-6.0929195E-3,-1.6712157E-2,2.9005406E5,2.8311168E-3,4.921743E-3,1.8972663E-2,6.567044E2,1.7796418E-2,1.3508893E3,-1.5223285E-2,-5.6726136E-4,1.5548611E7,-9.929505E-3,-5.184548E-3,1.6E1,7.549744E7,2.9407698E-1,1.7148E4,8.569445E-3,1E0,2.1194336E-3,1.6181818E1,6.9821E4,3.4955364E7,2.130589E-3,-3.989006E-3,1.6708623E2,3.640909E6,2.4235377E0,2.531816E-3,9.832144E-3,8.116684E-4,5.04449E-3,7.2896E5,1.0406779E1,2E1,1.4109914E-2,5.9677023E-3,1.0700461E6,-9.328202E-3,4.0748124E7,1.56E2,9.047619E-3,5.0279167E1,5.904E3,5.695737E0,-1.6573597E-3,-6.229847E-3,1E0,1.4396E4,6.2105E4,-2.8723574E-3,1.1399403E3,6.497315E-3,2.3642926E-2,1.2166621E-2,4.029152E8,6.92E2,7.33E2,3.2541638E-3,3.4643276E1,2.5604828E2,-1.3202189E-3,1.6667E4,7.912113E-1,-1.075116E-2,4.4066375E-3,-2.5309494E-3,-2.101464E-3,-8.838039E-3,3.9880952E-1,1.2566651E7,1.6504E4,1.58362E5,-6.4548436E-3,-5.7600802E-5,-9.048034E-3,-2.5282733E-4,-1.2460626E-3,-6.645143E-3,1.7284513E-3,1.4166522E-2,-5.065186E-4,-7.300574E-3,-2.6187945E-2,-6.950035E-3,1.4787026E-3,-5.4660505E-3,3.4439804E-3,-5.3783623E-3,7.1372944E-3,1.5060243E-4,-3.1184861E-3,-1.0319388E-2,-1.2711113E-3,-6.7392057E-3,6.181801E-4,4.9136E-3,-3.1887766E-3,2.1031948E-3,8.8599725E-3,1.7059206E-3,6.145827E-3,-9.1876285E-3,-5.8337166E-3,2.1878823E-3,1.5090061E-3,-1.8308083E-2,-1.1640081E-4,5.6096762E-3,-5.586708E-3,3.8845341E-3,1.2534115E-2,5.460056E-3,-6.2066303E-3,-1.2212841E-2,-6.8328576E-3,1.1236076E-3,-7.160637E-4,1.095388E-2,-2.6876188E-3,7.324814E-3,2.02521E-2,6.0671624E-3,-3.5899763E-3,-6.0190045E-4,2.993913E-3,9.095174E-3,4.0476783E-3,-2.5214872E-3,-2.374115E-3,5.711849E-3,4.7564544E-4,1.1755899E-2,8.381373E-4,6.0111266E-3,1.5997829E-2,6.254944E-3,-2.0200058E-3,1.707016E-3,8.39327E-3,-9.322293E-4,1.9894233E-2,1.0882357E-2,-5.2564376E-4,5.787431E-3,-5.163425E-3,-1.4441408E-4,2.9536749E-3,2.062828E-2,8.285552E-3,1.6538367E-2,-1.8828614E-2,-1.2196825E-2,2.8384877E-3,-1.1091168E-2,-1.5337195E-2,-6.510536E-3,-6.2997187E-3,-1.825368E-2,-3.7377372E-3,4.047727E-3,-1.404807E-2,-8.60597E-3,-5.364482E-3,-2.4167694E-4,-6.4950655E-3,1.320234E-3,-1.6548062E-2,-4.8334803E-3,-1.3808499E-3,-5.342785E-3,4.7837626E-3,-4.7488953E-5],"split_indices":[19,2,59,28,52,2,39,52,42,28,50,31,45,45,4,9,45,11,54,45,42,33,8,53,0,38,0,56,0,9,28,54,31,53,42,28,9,47,4,2,54,3,51,0,0,7,0,9,28,100,16,0,0,8,57,42,0,27,10,10,0,54,6,3,10,0,7,58,33,55,2,2,3,45,1,54,28,7,28,10,4,0,0,52,28,0,10,9,29,45,0,0,0,8,0,0,4,0,0,12,54,0,0,0,0,0,0,1,2,8,0,28,10,54,54,52,56,28,28,0,0,2,0,58,53,53,45,8,0,0,7,7,104,0,5,10,0,9,0,51,0,28,33,0,8,55,31,1,47,0,0,0,0,42,75,0,0,52,54,0,0,12,9,53,0,10,52,2,1,3,8,50,56,53,74,58,28,0,0,33,0,0,0,52,0,52,0,0,45,0,0,3,7,38,9,0,102,0,58,12,7,0,0,52,45,38,0,0,0,0,9,56,3,0,0,47,0,5,8,57,56,0,54,0,0,64,9,10,0,4,0,0,0,7,0,2,0,52,52,0,9,27,0,0,0,0,0,57,1,9,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.558E3,1.99E3,5.68E2,1.56E3,4.3E2,3.16E2,2.52E2,1.005E3,5.55E2,3.82E2,4.8E1,2.54E2,6.2E1,2.1E2,4.2E1,5.82E2,4.23E2,3.68E2,1.87E2,3.13E2,6.9E1,1.6E1,3.2E1,2.26E2,2.8E1,1.5E1,4.7E1,2.03E2,7E0,2.2E1,2E1,3.81E2,2.01E2,3.78E2,4.5E1,3.29E2,3.9E1,1.19E2,6.8E1,2.2E2,9.3E1,1.4E1,5.5E1,1.2E1,4E0,2.6E1,6E0,1.09E2,1.17E2,1.4E1,1.4E1,6E0,9E0,3.5E1,1.2E1,1.97E2,6E0,1.2E1,1E1,1.5E1,5E0,3.33E2,4.8E1,1.74E2,2.7E1,2.13E2,1.65E2,1.4E1,3.1E1,2.69E2,6E1,2.7E1,1.2E1,1.02E2,1.7E1,2.2E1,4.6E1,9.3E1,1.27E2,1.4E1,7.9E1,8E0,6E0,4E1,1.5E1,6E0,2E1,9.1E1,1.8E1,1.13E2,4E0,6E0,8E0,1E1,4E0,5E0,3E1,7E0,5E0,1.5E1,1.82E2,6E0,6E0,6E0,4E0,6E0,9E0,2.85E2,4.8E1,4.4E1,4E0,1.34E2,4E1,1.2E1,1.5E1,1.7E1,1.96E2,1.4E2,2.5E1,1E1,4E0,2.7E1,4E0,8.1E1,1.88E2,2.6E1,3.4E1,2.3E1,4E0,4E0,8E0,7E1,3.2E1,5E0,1.2E1,1.5E1,7E0,3.8E1,8E0,1.8E1,7.5E1,6.4E1,6.3E1,4E0,1E1,6E1,1.9E1,1.9E1,2.1E1,6E0,9E0,1.1E1,9E0,6.7E1,2.4E1,6E0,1.2E1,4.7E1,6.6E1,6E0,4E0,1.8E1,1.2E1,1E1,5E0,2.2E1,1.6E2,1.68E2,1.17E2,3.8E1,1E1,1.7E1,2.7E1,3E1,1.04E2,9E0,3.1E1,4E0,8E0,1.1E1,4E0,6E0,1.1E1,1.91E2,5E0,1.36E2,4E0,7E0,1.8E1,1.8E1,9E0,3E1,5.1E1,2.8E1,1.6E2,1E1,1.6E1,6E0,2.8E1,1.5E1,8E0,4E0,4E0,5.3E1,1.7E1,2.7E1,5E0,8E0,4E0,5E0,1E1,1.3E1,2.5E1,4E0,4E0,1.4E1,4E0,2.1E1,5.4E1,5.5E1,9E0,1.3E1,5E1,4E0,6E0,2.5E1,3.5E1,1.2E1,7E0,1.5E1,4E0,1.7E1,4E0,4.7E1,2E1,1.7E1,7E0,9E0,3.8E1,5E0,6.1E1,1.3E1,5E0,5E0,7E0,6E0,4E0,1.4E1,8E0,8.3E1,7.7E1,9.7E1,7.1E1,1.01E2,1.6E1,3.1E1,7E0,6E0,4E0,1.1E1,6E0,8E0,1.9E1,1E1,2E1,9.9E1,5E0,5E0,4E0,1.8E1,1.3E1,6E0,5E0,1.25E2,6.6E1,1.21E2,1.5E1,1.4E1,4E0,2.3E1,7E0,4.1E1,1E1,2.4E1,4E0,2.3E1,1.37E2,5E0,1.1E1,2.4E1,4E0,6E0,9E0,4E0,4E0,4E0,4.9E1,6E0,1.1E1,2.3E1,4E0,5E0,5E0,4E0,9E0,1.9E1,6E0,9E0,5E0,4E0,1.7E1,2.8E1,2.6E1,4E0,5.1E1,5E0,4E0,9E0,4E0,1.2E1,3.8E1,1.2E1,1.3E1,1E1,2.5E1,5E0,7E0,4E0,1.1E1,2.4E1,2.3E1,5E0,1.5E1,4E0,1.3E1,5E0,4E0,2.8E1,1E1,2.8E1,3.3E1,9E0,4E0,9E0,5E0,4E0,4E0,4.5E1,3.8E1,1.7E1,6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"351","size_leaf_vector":"1"}},{"base_weights":[1.1264184E-3,3.7479695E-2,-1.3155183E-1,-3.2989632E-2,9.2541665E-2,-2.0513368E-1,-3.340494E-2,-6.23918E-2,3.7521303E-2,3.7406098E-2,1.6292301E-1,-1.9551104E-1,-2.5405118E-2,-8.2914844E-2,4.6395622E-2,-9.002637E-2,2.5567321E-2,1.4920582E-2,1.1918247E-1,4.6949276E-1,2.7557198E-2,7.904614E-2,1.9243145E-1,-1.7255244E-1,-3.2102683E-1,-6.10258E-2,-3.22257E-1,9.8847315E-2,-1.4475027E-1,-7.613776E-2,-1.9183928E-1,4.5445934E-2,-1.5504467E-1,2.383913E-2,-1.7228778E-2,2.0170543E-1,5.985274E-2,9.07507E-3,2.8305048E-2,5.0207812E-2,-1.2111278E-1,1.1005009E-1,-3.7827197E-2,1.04559526E-1,2.2176842E-1,-1.9174145E-1,7.5234706E-3,-1.3208313E-1,-4.1739404E-1,-3.81865E-2,-1.1431447E-1,-2.1713115E-2,-8.992373E-3,5.0903473E-2,2.634693E-1,-8.312697E-2,-1.5639305E-2,-1.112647E-1,-4.1645844E-2,-3.2721266E-1,-1.07542604E-1,9.71704E-2,-2.5475326E-3,-1.2197825E-2,-6.853724E-2,-5.683827E-2,5.8000866E-2,9.669953E-2,1.3742871E-2,2.72457E-2,1.0366984E-2,4.04751E-2,3.1342202E-1,-1.6621943E-1,1.7826501E-2,1.3326827E-1,-6.0942538E-2,-7.816752E-3,9.159674E-3,1.9627322E-2,8.371802E-2,4.216068E-1,2.0249622E-1,-1.7537633E-1,-3.5889786E-1,8.103324E-3,-5.7988763E-2,-1.4636565E-2,-4.065447E-2,-4.6184343E-1,-8.125714E-3,-1.4569685E-1,-1.8054284E-2,-1.4093642E-1,-2.0354968E-2,-4.299863E-2,9.534544E-2,3.1464314E-1,6.2827626E-3,-1.677361E-1,1.1217102E-3,-1.1228909E-2,-1.1794468E-1,1.5616126E-2,-5.5264223E-2,-4.0298012E-1,-3.5291829E-3,-1.6007582E-2,-7.0928365E-2,-9.9062465E-2,1.3047679E-1,-5.000048E-2,8.6765915E-2,-3.7257286E-4,-5.791667E-3,5.2111182E-2,-1.7509842E-1,8.1393175E-2,-6.555066E-2,1.0796604E-2,-1.9383897E-3,-8.878319E-2,7.567495E-2,1.36933E-1,2.4631921E-2,2.2783978E-2,1.6828789E-1,-2.6343393E-1,-1.4008622E-1,7.761708E-3,-3.485436E-2,1.7568551E-1,7.72549E-2,-5.322761E-4,-5.8880053E-3,-1.6173073E-3,3.2235354E-2,2.1511978E-4,1.3885738E-1,2.6875895E-1,2.694806E-2,1.9247091E-1,2.406908E-2,-3.259331E-1,-1.5396348E-1,-6.580942E-3,-4.711003E-1,-1.7815784E-2,-8.339934E-3,1.5526233E-4,-5.1237037E-3,-4.926434E-1,-1.1997738E-2,-3.1063168E-3,-1.0543657E-2,2.0789271E-2,-5.04543E-2,-3.1674765E-3,-1.6977778E-1,-3.8502768E-3,2.5861925E-3,1.20305475E-2,-9.372795E-2,1.5162095E-1,6.47185E-2,1.975993E-2,9.063129E-3,-3.64497E-3,-1.1441215E-2,4.6017724E-3,-6.1768893E-2,-1.9326667E-1,-9.884096E-2,-1.1575528E-2,-1.02847025E-1,-9.096854E-3,-4.6144912E-1,3.8986378E-3,-9.4993986E-2,-8.761541E-3,-9.4841846E-4,1.5108883E-1,-1.7683989E-3,1.4741587E-2,-1.4618573E-1,-1.4032846E-3,1.3299018E-1,-5.7602298E-2,1.3021E-1,-2.6498422E-1,-4.408632E-2,-6.9189346E-3,9.429278E-2,-1.4308502E-1,4.9552508E-2,-1.3741088E-3,-6.611051E-3,8.5246E-3,1.4868788E-1,1.8983765E-1,1.3669081E-2,-1.692754E-2,7.604249E-2,2.2091153E-3,1.1635866E-2,-7.948353E-3,-1.7052753E-2,-1.5436746E-1,-2.199959E-3,-8.293145E-2,1.9390258E-3,2.1926415E-1,9.717543E-2,1.236238E-1,-2.9230006E-3,-1.8057638E-3,3.1578522E-3,4.1422345E-2,-8.815019E-3,1.6906485E-1,-2.6078965E-3,1.5546173E-2,7.3556965E-3,2.5852466E-1,1.5051416E-1,-1.8515345E-2,-2.004766E-1,-3.6009997E-2,-1.8497096E-1,-2.8067956E-2,-1.0679892E-2,1.653306E-3,-3.3224223E-3,-1.2748776E-2,-2.6658049E-2,4.061791E-3,-7.8992546E-4,-8.285451E-2,1.9869743E-2,-1.2764139E-2,-5.634529E-3,3.2928237E-3,-1.7421425E-3,-7.18214E-3,-7.479783E-4,4.2825956E-3,9.787108E-3,7.2902227E-3,4.4256125E-2,-9.982826E-4,-4.312074E-3,-7.625065E-3,-1.632311E-2,-4.522813E-3,-9.166188E-3,-1.8853697E-3,4.926017E-3,-1.1925369E-2,-3.8658802E-3,-2.8235571E-2,-1.0160064E-2,-6.1651478E-3,1.4825815E-3,4.9011605E-3,1.12767145E-2,2.4501076E-3,-4.9987747E-3,-1.3435299E-2,-3.4412895E-3,1.0678772E-3,8.708008E-3,-6.208462E-3,2.5969057E-3,9.186666E-3,3.175037E-3,-1.9870413E-2,-9.302703E-4,2.518922E-3,-1.026146E-2,7.60273E-4,6.7836232E-3,-4.014881E-3,-1.1988785E-2,-1.3267988E-3,5.2322885E-3,2.133918E-3,-2.391999E-3,9.1792885E-3,3.8437303E-3,6.9784215E-3,1.3022689E-2,-3.8285907E-3,2.8961056E-3,-9.19061E-5,-6.1201947E-3,4.2142575E-3,-6.6792136E-3,-2.2644405E-3,-8.513972E-3,-7.907239E-4,-7.3387474E-3,7.0583276E-3,1.3595212E-2,-1.9255265E-3,7.899239E-3,7.3279566E-3,-1.2224923E-4,-2.852302E-3,4.8397467E-3,-1.8005833E-3,5.568803E-3,-1.3034772E-3,9.668556E-3,-9.3857513E-4,1.3715565E-2,4.8244316E-3,1.0829069E-2,-5.102193E-3,-1.2928672E-2,-8.474925E-3,3.4306158E-4,-1.2176032E-2,-5.854405E-3,1.1264132E-3,-2.3672893E-3,-5.133717E-3,2.054298E-3,3.2323403E-3,-2.0283847E-3,-1.4720413E-4,4.7976607E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,-1,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,95,97,99,-1,101,103,105,107,109,111,-1,113,115,117,119,-1,121,-1,123,125,127,129,131,133,-1,135,-1,137,139,141,143,145,-1,147,-1,149,151,-1,153,155,157,159,161,163,165,-1,167,-1,169,171,-1,173,175,-1,-1,177,179,181,183,185,-1,-1,187,189,191,193,-1,-1,195,197,199,201,-1,203,205,207,-1,209,211,213,-1,-1,-1,215,217,219,221,-1,223,-1,225,227,-1,229,231,-1,-1,-1,233,-1,-1,-1,235,237,-1,239,-1,-1,241,243,245,247,-1,-1,-1,-1,-1,249,251,253,255,257,-1,259,-1,261,-1,-1,263,-1,265,267,-1,269,271,273,275,277,-1,279,281,283,-1,-1,285,287,289,291,293,295,-1,-1,-1,-1,297,-1,299,-1,301,303,305,307,-1,-1,309,-1,311,-1,-1,-1,313,315,-1,317,319,321,-1,-1,-1,-1,-1,-1,-1,323,325,327,-1,-1,-1,-1,-1,-1,-1,-1,-1,329,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2289847E1,7.7634134E0,3.9507093E0,1.822588E0,4.3517237E0,8.141222E-1,9.3626356E-1,1.5093434E0,4.7623235E-1,2.6723318E0,1.2050009E0,8.2351494E-1,0E0,7.4522376E-1,9.266674E-1,6.5282035E-1,5.4200315E-1,6.6269946E-1,2.6210696E-1,2.879858E-1,2.0871751E0,4.7522253E-1,9.151678E-1,8.9986753E-1,7.824364E-1,1.5993994E-1,1.1353564E-1,5.5485654E-1,1.813373E-1,5.0060153E-1,6.038563E-1,3.3537644E-1,9.885293E-2,5.545127E-1,0E0,1.5051848E-1,1.5801029E-1,0E0,0E0,1.3629943E0,5.1686394E-1,4.1704595E-1,1.5748292E-1,5.2581596E-1,9.6218586E-1,5.799093E-1,2.7282438E-1,2.3808816E-1,2.6462603E-1,2.0515402E-1,9.7186625E-2,0E0,0E0,2.4197178E-1,7.3207974E-2,1.4668882E-1,0E0,1.3633251E-1,1.0398142E0,3.7978864E-1,2.5780126E-1,4.3775296E-1,3.0524957E-1,0E0,2.5178153E-2,7.821872E-1,4.1284722E-1,1.8630424E-1,0E0,1.6967525E-1,0E0,7.8976285E-1,3.1638396E-1,1.2060928E-1,1.575426E-1,2.026372E-1,3.594163E-2,0E0,2.1167116E-2,0E0,4.0514827E-1,2.6887655E-1,6.07069E-1,6.461692E-1,4.4778132E-1,0E0,8.016444E-2,0E0,3.2818567E-2,5.2440166E-2,0E0,6.8719715E-2,1.04248516E-1,5.8568776E-2,4.487799E-2,5.416786E-2,5.740565E-2,4.6738863E-2,0E0,3.55407E-2,0E0,7.879786E-2,2.5861096E-1,0E0,4.2496973E-1,1.22662544E-1,0E0,0E0,1.2212066E-1,5.7762057E-2,1.9680476E-1,2.9483488E-1,1.3645674E-1,0E0,0E0,2.8169215E-1,3.2982564E-1,3.5056013E-1,2.1235259E-1,0E0,0E0,2.0367965E-2,1.01692215E-1,4.734552E-1,9.5634544E-1,0E0,7.785183E-2,4.3203294E-2,6.31665E-2,0E0,5.9880584E-2,1.5318239E-1,1.5278956E-1,0E0,0E0,0E0,3.192398E-2,2.6933584E-1,3.146944E-1,1.5999794E-2,0E0,6.3453865E-1,0E0,7.1259975E-2,6.9191027E-1,0E0,1.8316507E-1,3.9367896E-2,0E0,0E0,0E0,6.7729E-2,0E0,0E0,0E0,4.9675394E-2,1.041794E-1,0E0,7.86531E-2,0E0,0E0,2.7560845E-2,3.723591E-2,2.112928E-2,3.8395397E-2,0E0,0E0,0E0,0E0,0E0,8.526951E-3,1.5630043E-1,9.326315E-2,3.1492177E-1,3.2065248E-1,0E0,2.0610285E-1,0E0,1.0276434E-1,0E0,0E0,1.7554307E-1,0E0,1.1905852E-1,1.6172281E-1,0E0,7.9224706E-2,1.08701885E-1,5.626893E-2,5.483289E-1,2.0848176E-1,0E0,3.6850035E-1,6.578225E-2,4.4763085E-2,0E0,0E0,2.5177551E-2,1.0999382E-2,1.4475036E-1,9.642233E-2,3.947592E-1,3.593557E-1,0E0,0E0,0E0,0E0,6.9170594E-2,0E0,4.0584527E-2,0E0,9.3413115E-2,1.7142898E-1,7.58574E-2,9.199841E-2,0E0,0E0,1.6674314E-1,0E0,2.2440267E-1,0E0,0E0,0E0,4.1700888E-1,5.185764E-1,0E0,2.516377E-2,2.3602377E-1,5.7257366E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.259206E-2,8.065502E-2,4.327653E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.6477312E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,64,64,65,65,66,66,67,67,69,69,71,71,72,72,73,73,74,74,75,75,76,76,78,78,80,80,81,81,82,82,83,83,84,84,86,86,88,88,89,89,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,101,101,102,102,104,104,105,105,108,108,109,109,110,110,111,111,112,112,115,115,116,116,117,117,118,118,121,121,122,122,123,123,124,124,126,126,127,127,128,128,130,130,131,131,132,132,136,136,137,137,138,138,139,139,141,141,143,143,144,144,146,146,147,147,151,151,155,155,156,156,158,158,161,161,162,162,163,163,164,164,170,170,171,171,172,172,173,173,174,174,176,176,178,178,181,181,183,183,184,184,186,186,187,187,188,188,189,189,190,190,192,192,193,193,194,194,197,197,198,198,199,199,200,200,201,201,202,202,207,207,209,209,211,211,212,212,213,213,214,214,217,217,219,219,223,223,224,224,226,226,227,227,228,228,236,236,237,237,238,238,248,248],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,-1,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,96,98,100,-1,102,104,106,108,110,112,-1,114,116,118,120,-1,122,-1,124,126,128,130,132,134,-1,136,-1,138,140,142,144,146,-1,148,-1,150,152,-1,154,156,158,160,162,164,166,-1,168,-1,170,172,-1,174,176,-1,-1,178,180,182,184,186,-1,-1,188,190,192,194,-1,-1,196,198,200,202,-1,204,206,208,-1,210,212,214,-1,-1,-1,216,218,220,222,-1,224,-1,226,228,-1,230,232,-1,-1,-1,234,-1,-1,-1,236,238,-1,240,-1,-1,242,244,246,248,-1,-1,-1,-1,-1,250,252,254,256,258,-1,260,-1,262,-1,-1,264,-1,266,268,-1,270,272,274,276,278,-1,280,282,284,-1,-1,286,288,290,292,294,296,-1,-1,-1,-1,298,-1,300,-1,302,304,306,308,-1,-1,310,-1,312,-1,-1,-1,314,316,-1,318,320,322,-1,-1,-1,-1,-1,-1,-1,324,326,328,-1,-1,-1,-1,-1,-1,-1,-1,-1,330,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7311627E2,1E0,6.978874E5,2.1446484E-1,4.5723195E6,1.0323588E3,4.264897E6,1E0,8E0,6.218596E2,4.7022E4,-2.5405118E-2,8.672049E9,9.302862E7,5.024605E0,1.164249E7,2.465E3,7E0,3.171E3,5.7581736E7,4.1179886E0,1E0,1.2204E4,1.9826213E0,1.062363E1,6.3300834E0,5.528107E3,1.4299594E8,1.3781E2,9.74026E0,1.86E2,1.0201212E3,1.8122449E2,-1.7228778E-2,2.4222221E0,3.9387238E5,9.07507E-3,2.8305048E-2,1.5137865E2,2.5422776E3,1.3295082E1,2.22E3,5.0338E4,1.4075E4,1.0828989E8,1E0,5.913995E5,2.720437E6,1.06E3,2.0619047E1,-2.1713115E-2,-8.992373E-3,1E0,1.663473E6,7.023838E3,-1.5639305E-2,1.6093831E0,3.4206028E-6,4.2378342E-1,1E0,2.53E2,1.6259277E7,-1.2197825E-2,1.2817779E7,6.62E2,1.9968051E0,2.7487562E0,1.3742871E-2,4.624343E6,1.0366984E-2,2E0,2.9788235E-2,1.1863768E-4,5.441732E9,1.4387654E0,2.74E1,-7.816752E-3,1.4022468E6,1.9627322E-2,5.077728E8,1.0838E4,1.8114515E1,7.22E2,1.2E1,8.103324E-3,1.2367184E3,-1.4636565E-2,9.793811E-2,1.91E2,-8.125714E-3,8.601996E-1,7.2591E4,1.17E2,1.0693E4,8.081268E-1,1.2939E4,2.9956698E-1,6.2827626E-3,2.5932198E6,1.1217102E-3,2.56E2,2.765625E0,1.5616126E-2,1.3058E4,5.46E2,-3.5291829E-3,-1.6007582E-2,1.3E1,4E0,2.7309215E6,1.6E1,1.2E1,-3.7257286E-4,-5.791667E-3,1.2293E4,8.2884795E3,5.845392E1,3.0291306E2,1.0796604E-2,-1.9383897E-3,2.4968571E2,2.0827537E0,6.1114804E-8,6.624E3,2.2783978E-2,9.8E1,2.3067484E0,2.3898147E5,7.761708E-3,5.913995E5,9.6603775E0,3.8134545E6,-5.322761E-4,-5.8880053E-3,-1.6173073E-3,4.4440216E-1,1.4688152E9,3.666495E7,2.804366E-1,2.694806E-2,1.9719212E1,2.406908E-2,1E0,3.94E2,-6.580942E-3,2.0521326E0,8.4503644E5,-8.339934E-3,1.5526233E-4,-5.1237037E-3,1E0,-1.1997738E-2,-3.1063168E-3,-1.0543657E-2,8.317E3,6.341735E2,-3.1674765E-3,2.3770695E5,-3.8502768E-3,2.5861925E-3,3.1E1,9.348536E-1,8.538E3,5.1148495E6,1.975993E-2,9.063129E-3,-3.64497E-3,-1.1441215E-2,4.6017724E-3,8.204587E1,3.78E2,4.785384E5,1.032E3,9E2,-9.096854E-3,5.101512E0,3.8986378E-3,1.8976281E6,-8.761541E-3,-9.4841846E-4,3.052775E2,-1.7683989E-3,4.2037987E1,5.7075036E-1,-1.4032846E-3,8.2363125E4,6.38041E5,9.857292E-1,8.15E2,2.04425E2,-6.9189346E-3,2.6133334E2,7.980211E5,2.46875E0,-1.3741088E-3,-6.611051E-3,2.9806678E0,3.898923E0,5.201557E7,2.2884E4,5.8036587E1,1.3674345E4,2.2091153E-3,1.1635866E-2,-7.948353E-3,-1.7052753E-2,6.747114E7,-2.199959E-3,1.0192E4,1.9390258E-3,1.583E3,6.765595E7,1.963801E8,1.4E1,-1.8057638E-3,3.1578522E-3,1.0289962E3,-8.815019E-3,1.37E2,-2.6078965E-3,1.5546173E-2,7.3556965E-3,3.90106E0,1.4101E4,-1.8515345E-2,1.7642576E8,6.019075E-4,8.573816E0,-2.8067956E-2,-1.0679892E-2,1.653306E-3,-3.3224223E-3,-1.2748776E-2,-2.6658049E-2,4.061791E-3,6.162955E0,3.1E1,8.893143E5,-1.2764139E-2,-5.634529E-3,3.2928237E-3,-1.7421425E-3,-7.18214E-3,-7.479783E-4,4.2825956E-3,9.787108E-3,7.2902227E-3,2.6835732E3,-9.982826E-4,-4.312074E-3,-7.625065E-3,-1.632311E-2,-4.522813E-3,-9.166188E-3,-1.8853697E-3,4.926017E-3,-1.1925369E-2,-3.8658802E-3,-2.8235571E-2,-1.0160064E-2,-6.1651478E-3,1.4825815E-3,4.9011605E-3,1.12767145E-2,2.4501076E-3,-4.9987747E-3,-1.3435299E-2,-3.4412895E-3,1.0678772E-3,8.708008E-3,-6.208462E-3,2.5969057E-3,9.186666E-3,3.175037E-3,-1.9870413E-2,-9.302703E-4,2.518922E-3,-1.026146E-2,7.60273E-4,6.7836232E-3,-4.014881E-3,-1.1988785E-2,-1.3267988E-3,5.2322885E-3,2.133918E-3,-2.391999E-3,9.1792885E-3,3.8437303E-3,6.9784215E-3,1.3022689E-2,-3.8285907E-3,2.8961056E-3,-9.19061E-5,-6.1201947E-3,4.2142575E-3,-6.6792136E-3,-2.2644405E-3,-8.513972E-3,-7.907239E-4,-7.3387474E-3,7.0583276E-3,1.3595212E-2,-1.9255265E-3,7.899239E-3,7.3279566E-3,-1.2224923E-4,-2.852302E-3,4.8397467E-3,-1.8005833E-3,5.568803E-3,-1.3034772E-3,9.668556E-3,-9.3857513E-4,1.3715565E-2,4.8244316E-3,1.0829069E-2,-5.102193E-3,-1.2928672E-2,-8.474925E-3,3.4306158E-4,-1.2176032E-2,-5.854405E-3,1.1264132E-3,-2.3672893E-3,-5.133717E-3,2.054298E-3,3.2323403E-3,-2.0283847E-3,-1.4720413E-4,4.7976607E-3],"split_indices":[19,52,59,28,38,28,52,9,101,18,52,9,0,5,45,53,45,2,3,2,45,53,102,2,56,54,53,52,45,52,53,10,4,52,0,53,47,0,0,56,52,46,2,1,9,45,64,32,32,2,56,0,0,102,29,4,0,54,41,27,64,2,9,0,9,2,53,54,0,45,0,17,38,42,5,41,46,0,51,0,7,9,35,2,3,0,58,0,38,8,0,27,29,10,9,27,9,38,0,32,0,0,54,0,29,29,0,0,10,8,45,3,8,0,0,1,47,51,52,0,0,4,53,37,2,0,8,58,33,0,32,56,47,0,0,0,41,5,51,57,0,56,0,16,0,0,53,32,0,0,0,100,0,0,0,9,52,0,28,0,0,3,27,2,45,0,0,0,0,0,47,0,28,10,2,0,56,0,50,0,0,4,0,58,41,0,47,9,56,2,4,0,55,50,53,0,0,56,53,7,1,53,4,0,0,0,0,45,0,9,0,2,5,32,3,0,0,4,0,0,0,0,0,56,2,0,7,39,58,0,0,0,0,0,0,0,53,8,47,0,0,0,0,0,0,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.546E3,1.999E3,5.47E2,8.77E2,1.122E3,3.12E2,2.35E2,6.19E2,2.58E2,6.3E2,4.92E2,3.04E2,8E0,1.45E2,9E1,4.71E2,1.48E2,2.03E2,5.5E1,1.3E1,6.17E2,1.29E2,3.63E2,2.59E2,4.5E1,1.34E2,1.1E1,7.1E1,1.9E1,4.16E2,5.5E1,1.34E2,1.4E1,1.99E2,4E0,2.2E1,3.3E1,4E0,9E0,5.36E2,8.1E1,1.02E2,2.7E1,9.2E1,2.71E2,2.34E2,2.5E1,1.6E1,2.9E1,9.5E1,3.9E1,5E0,6E0,5.6E1,1.5E1,1.5E1,4E0,2.05E2,2.11E2,2E1,3.5E1,6.4E1,7E1,6E0,8E0,5.9E1,1.4E2,1E1,1.2E1,2.8E1,5E0,5.18E2,1.8E1,6.1E1,2E1,9E1,1.2E1,7E0,2E1,5E0,8.7E1,2.2E1,2.49E2,2.15E2,1.9E1,7E0,1.8E1,5E0,1.1E1,2.4E1,5E0,1.4E1,8.1E1,3E1,9E0,1.8E1,3.8E1,1E1,5E0,8E0,7E0,1.3E1,1.92E2,7E0,2.04E2,1.5E1,5E0,4E0,3.1E1,9E0,5.5E1,4.6E1,2.4E1,4E0,4E0,3.1E1,2.8E1,1.18E2,2.2E1,5E0,5E0,8E0,2E1,7.2E1,4.46E2,8E0,1E1,1.1E1,5E1,5E0,1.5E1,5E1,4E1,7E0,5E0,7E0,1.3E1,3.5E1,5.2E1,1.1E1,1.1E1,2.42E2,7E0,2.5E1,1.9E2,7E0,1.2E1,1.4E1,4E0,7E0,4E0,2E1,4E0,7E0,7E0,3.7E1,4.4E1,9E0,2.1E1,5E0,4E0,9E0,9E0,1.2E1,2.6E1,5E0,5E0,4E0,4E0,4E0,9E0,3.7E1,1.55E2,1.07E2,9.7E1,4E0,1.1E1,4E0,2.7E1,4E0,5E0,4.9E1,6E0,2.8E1,1.8E1,7E0,1.7E1,1.3E1,1.8E1,1.6E1,1.2E1,6E0,1.12E2,1.3E1,9E0,4E0,4E0,1.1E1,9E0,5E1,2.2E1,2.47E2,1.99E2,4E0,6E0,6E0,5E0,4.3E1,7E0,9E0,6E0,3.1E1,1.9E1,2.5E1,1.5E1,4E0,9E0,2.9E1,6E0,4.5E1,7E0,7E0,4E0,9.2E1,1.5E2,1.7E1,8E0,4E1,1.5E2,8E0,4E0,7E0,7E0,4E0,1.6E1,9E0,2.8E1,3E1,1.4E1,7E0,1.4E1,4E0,5E0,5E0,4E0,6E0,6E0,4E0,2.2E1,4E0,5E0,3E1,7E0,1.43E2,1.2E1,8.7E1,2E1,1.4E1,8.3E1,7E0,4E0,2.2E1,5E0,3E1,1.9E1,2.2E1,6E0,6E0,1.2E1,5E0,1.2E1,8E0,5E0,9E0,9E0,1E1,6E0,8E0,4E0,3.9E1,7.3E1,9E0,4E0,4E0,5E0,7E0,4E0,5E0,4E0,3.1E1,1.9E1,7E0,1.5E1,2.17E2,3E1,1.92E2,7E0,6E0,3.7E1,5E0,4E0,1.4E1,1.7E1,6E0,1.3E1,2.1E1,4E0,1E1,5E0,1.4E1,1.5E1,5E0,4E1,5E0,8.7E1,8.4E1,6.6E1,4E0,4E0,9E0,3.1E1,7.9E1,7.1E1,1.9E1,9E0,2.6E1,4E0,8E0,6E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"331","size_leaf_vector":"1"}},{"base_weights":[-3.0615563E-3,-1.0209179E-1,4.283224E-2,-1.7330277E-1,-4.5055136E-2,1.252132E-2,1.3071962E-1,-1.522984E-1,-3.3910555E-1,-8.034849E-2,9.1341116E-2,-5.524852E-2,5.3649005E-2,1.6139123E-1,-3.608317E-3,-1.6646957E-1,2.3847459E-2,-4.3527153E-1,-1.2942074E-1,-8.879795E-2,2.7017118E-2,2.924351E-1,6.92343E-2,-6.2190104E-2,1.8818238E-1,-1.1221368E-1,6.576798E-2,4.8437193E-2,1.9160496E-1,2.3925835E-2,-1.568131E-2,-1.236616E-1,-2.1929114E-1,1.383214E-1,-3.8159184E-2,-8.720072E-3,-5.1780945E-1,1.7314062E-4,-2.0363124E-1,-6.0055926E-2,-1.7630126E-1,1.987688E-2,6.424733E-3,3.0440077E-2,1.8070656E-1,-8.764827E-2,-4.534557E-3,1.6531857E-2,1.9896084E-3,-1.3790719E-1,6.240012E-3,5.572136E-3,1.00724965E-1,9.720791E-3,1.4830603E-1,1.685313E-1,3.3578327E-1,5.8451943E-2,-9.632664E-2,-1.8527108E-1,-4.0946927E-2,-1.7245057E-1,-3.2776707E-1,2.28142E-3,1.0159207E-2,1.8246623E-3,-6.3945696E-2,-1.8329255E-2,-3.7264954E-2,-4.4677146E-3,-1.384688E-2,-7.333482E-2,1.714595E-1,-2.2023016E-1,-4.257913E-2,5.8115233E-2,-1.4821814E-1,2.2605082E-1,9.009253E-2,9.856965E-3,-9.4330296E-2,-2.618117E-2,6.671959E-2,-1.6447625E-1,4.3568346E-3,1.2123327E-2,-1.7911965E-2,8.2933076E-2,1.8778193E-1,2.8494444E-2,-9.742442E-3,1.052022E-2,8.902191E-2,2.6599553E-1,1.3251713E-1,5.6200504E-1,2.1442528E-1,2.4780206E-1,2.1622637E-2,-1.5358166E-1,-2.2006981E-2,-9.937219E-2,-2.5308767E-1,-6.1821077E-2,4.388787E-2,-1.7041968E-2,-1.5302533E-1,-1.9439457E-3,-3.6910257E-1,-4.734344E-3,-6.1037415E-4,-1.17533125E-1,-3.1243669E-2,1.3882445E-2,9.048878E-2,-6.3852295E-2,-2.6110396E-1,7.505233E-2,-2.0300525E-1,-1.945001E-2,9.156671E-2,-1.0423005E-2,-2.9077344E-3,7.7717104E-3,1.4697447E-2,8.97312E-4,7.20561E-3,-1.2853445E-1,-5.89179E-2,5.268909E-2,-5.8564194E-2,1.11096844E-1,-6.3287974E-3,-1.4248125E-1,-1.514699E-2,4.521079E-3,-4.1292254E-3,1.3763802E-1,7.548518E-4,1.5509997E-3,9.610026E-2,2.84826E-1,1.5951222E-1,7.657706E-2,-1.5596677E-2,-1.3245831E-3,7.1609537E-3,2.1486802E-2,2.3969199E-1,1.06915E-1,2.0811245E-1,1.8317534E-2,3.3833858E-2,4.6838965E-2,3.2671344E-1,1.7630942E-2,3.6065562E-3,-7.4152946E-3,1.21453516E-1,-1.0680174E-2,-2.5317196E-3,-2.8084991E-3,8.291912E-4,-6.6177465E-2,-1.724128E-2,-1.5857007E-1,-3.0556592E-1,3.7360063E-3,-7.3114425E-2,8.278096E-2,-2.521043E-3,-6.420373E-2,-1.9408976E-1,-2.6641665E-2,-3.06814E-1,-7.4230514E-2,-1.983839E-1,-1.4814469E-1,-1.8425845E-2,6.2348135E-3,1.5610687E-3,-1.4273104E-1,2.3326934E-3,-1.7810424E-1,-3.258857E-1,3.9226445E-3,8.456265E-3,-1.8801788E-2,-1.2376696E-3,2.4571247E-2,-5.575667E-3,4.979252E-2,1.5111595E-1,-8.762572E-2,-1.9574225E-1,1.6256971E-2,-7.0631444E-2,-4.04137E-2,8.782264E-2,4.717604E-3,-7.410112E-2,1.3378607E-1,-1.2276137E-3,-9.146465E-2,-2.0312247E-1,2.2714254E-3,1.7512524E-1,-2.0377228E-1,1.9028531E-2,1.041982E-1,-4.1047864E-2,1.8536268E-1,7.9681866E-2,1.8343132E-2,3.6371837E-3,1.6950072E-1,2.8730897E-4,6.81069E-3,5.1843546E-2,-4.0340666E-2,3.39439E-3,2.6784793E-1,1.6998775E-1,2.3590478E-1,9.3087226E-2,1.4773448E-2,1.4126849E-1,8.834828E-3,-8.5187955E-3,8.927806E-3,1.8875422E-2,-4.1922115E-2,4.5796033E-2,7.937589E-3,1.8645639E-3,5.0695543E-3,-4.3781796E-3,-1.0393127E-2,-2.6104737E-3,-1.6918126E-2,-5.0233593E-3,-2.5463435E-3,-7.4076266E-3,5.4886594E-4,6.1078346E-3,-1.0625035E-2,-8.112221E-4,-2.0604245E-3,-1.0686532E-2,-9.569183E-3,-1.8693283E-2,-6.134389E-3,-6.755104E-4,-2.2410916E-2,-7.678304E-3,-9.872693E-4,-1.0143723E-2,-3.3315897E-3,5.002531E-4,-9.400316E-3,-3.4369146E-3,-5.476461E-3,-1.2757724E-2,-2.185806E-2,-1.07363155E-2,2.509805E-3,-2.1570022E-3,4.5076483E-3,-7.3084206E-4,3.4398555E-3,-4.511563E-4,8.605619E-3,3.508949E-3,-7.143229E-3,-1.0727411E-3,-2.316394E-2,-8.2002245E-3,-2.1768038E-3,-9.043159E-3,-4.311128E-3,2.2704345E-4,2.7007456E-3,8.341565E-3,-5.166201E-3,-9.09263E-4,2.8435024E-3,8.020726E-3,-5.5197584E-3,-4.776428E-5,-1.1946551E-2,-6.6341856E-3,1.0415591E-2,4.239215E-3,-1.3508692E-3,-1.551234E-2,-9.692966E-4,3.6924582E-3,5.6239497E-4,9.271986E-3,7.282624E-3,-3.54966E-3,5.0641717E-3,1.0940343E-2,4.9444926E-3,1.3879153E-3,3.8908767E-3,9.750413E-3,-3.2531674E-4,3.584059E-3,-5.0480287E-3,-6.4850785E-4,1.4723835E-2,9.363921E-3,5.274578E-3,1.2663116E-2,6.046056E-3,1.7477905E-2,1.0764507E-2,4.0470823E-3,3.6605126E-3,8.475732E-3,-8.580024E-3,2.7988775E-4,4.0891925E-3,-1.6949389E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,-1,59,61,63,65,-1,67,-1,69,71,73,-1,-1,75,77,79,81,-1,-1,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,-1,-1,-1,109,-1,-1,-1,-1,111,113,115,117,119,121,123,125,-1,127,129,131,133,135,137,-1,139,141,143,-1,-1,145,147,149,151,153,155,157,159,161,163,165,167,169,-1,171,-1,173,-1,-1,175,177,-1,179,181,183,185,187,189,191,-1,-1,-1,-1,-1,-1,193,195,197,199,201,-1,203,-1,-1,-1,205,207,209,211,213,215,217,219,-1,-1,-1,221,223,225,-1,-1,227,229,-1,-1,231,233,-1,-1,-1,-1,235,-1,237,239,-1,241,243,-1,245,247,-1,249,251,253,255,257,-1,-1,259,-1,261,263,265,-1,-1,-1,267,-1,269,271,273,275,-1,277,279,281,-1,283,285,-1,287,289,-1,291,293,295,297,299,301,303,-1,-1,305,-1,-1,307,309,-1,311,313,315,317,-1,319,-1,-1,-1,-1,321,323,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1725601E1,3.3105488E0,4.6949024E0,1.2125311E0,2.1976547E0,3.6597922E0,1.8628469E0,8.1273603E-1,7.5024843E-1,1.9476268E0,3.9763522E-1,8.4984183E-1,1.6496208E0,1.2466154E0,7.3356515E-1,6.512375E-1,1.8273185E-1,4.9072933E-1,1.3876697E-1,8.8979006E-1,0E0,1.0423529E-1,3.6640447E-1,7.085521E-1,2.8317532E-1,3.583665E-1,1.6028888E0,3.0401674E-1,9.081583E-1,3.3362663E-1,0E0,8.485429E-1,6.1925936E-1,4.4922814E-2,3.507126E-2,0E0,4.1484737E-1,0E0,5.0767004E-2,8.457007E-1,5.075581E-1,0E0,0E0,3.2999933E-1,7.002866E-2,6.5639377E-1,2.3156604E-1,0E0,0E0,1.9348735E-1,0E0,6.7452085E-1,7.310004E-1,2.2844747E-1,6.2431782E-2,8.4929514E-1,9.50191E-1,4.2700714E-1,7.3213264E-2,5.3408575E-1,1.3198559E-1,2.5897217E-1,4.5130968E-1,0E0,0E0,0E0,1.955821E-2,0E0,0E0,0E0,0E0,4.7443974E-1,1.0405758E-1,4.051838E-1,4.4927266E-1,1.4967725E-1,3.678292E-2,2.1470606E-2,3.2542743E-2,0E0,3.8971734E-1,2.968023E-1,3.144301E-1,8.795583E-2,7.491056E-2,3.9515033E-1,0E0,4.306233E-1,1.7678261E-1,1.15675226E-1,0E0,0E0,8.534172E-2,1.8599796E-1,3.3773446E-1,4.421997E-2,4.9278188E-1,1.5902656E-1,1.5454137E-1,5.1087067E-2,1.2814576E-2,3.360025E-1,2.1443248E-1,9.64784E-2,5.9215665E-2,0E0,3.0486298E-1,0E0,1.8391418E-1,0E0,0E0,4.2290497E-1,1.9655977E-1,0E0,1.6417667E-2,1.3744906E-1,2.2155595E-1,9.312782E-2,2.9952806E-1,7.575019E-2,9.198046E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.3924618E-1,7.566781E-1,1.1476432E-1,2.0189604E-1,9.255314E-2,0E0,1.0207611E-1,0E0,0E0,0E0,7.1516484E-2,9.569306E-1,2.535184E-1,4.9230862E-1,2.6986694E-1,9.885597E-2,3.1342596E-2,6.2679395E-2,0E0,0E0,0E0,7.222438E-2,2.2953355E-1,2.2744346E-1,0E0,0E0,3.691135E-1,5.341363E-2,0E0,0E0,7.903832E-2,3.183709E-2,0E0,0E0,0E0,0E0,1.5268525E-1,0E0,9.950501E-2,1.8939066E-1,0E0,8.399537E-2,2.9818974E-2,0E0,2.0303863E-1,1.6959524E-1,0E0,1.3787246E-1,2.4293059E-1,4.1202378E-1,8.755973E-2,1.6669461E-1,0E0,0E0,1.7274678E-2,0E0,1.0425961E-1,2.5280547E-1,2.179431E-2,0E0,0E0,0E0,3.476876E-2,0E0,2.8636117E-2,1.5228242E-2,3.8130623E-1,4.4915247E-1,0E0,4.6702206E-1,2.1012466E-2,5.7776943E-2,0E0,1.2093273E-1,4.224935E-2,0E0,3.9455384E-2,1.1271477E-2,0E0,3.0455291E-2,3.8755238E-1,4.969327E-1,1.264707E-1,2.4005914E-1,1.273731E-1,2.913736E-1,0E0,0E0,1.2810647E-1,0E0,0E0,2.3675006E-2,3.6880642E-2,0E0,3.3252954E-2,8.0765605E-2,1.277914E-1,1.7338884E-1,0E0,4.1576743E-2,0E0,0E0,0E0,0E0,1.6133568E-1,5.1620558E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,38,38,39,39,40,40,43,43,44,44,45,45,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,66,66,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,106,106,108,108,111,111,112,112,114,114,115,115,116,116,117,117,118,118,119,119,120,120,127,127,128,128,129,129,130,130,131,131,133,133,137,137,138,138,139,139,140,140,141,141,142,142,143,143,144,144,148,148,149,149,150,150,153,153,154,154,157,157,158,158,163,163,165,165,166,166,168,168,169,169,171,171,172,172,174,174,175,175,176,176,177,177,178,178,181,181,183,183,184,184,185,185,189,189,191,191,192,192,193,193,194,194,196,196,197,197,198,198,200,200,201,201,203,203,204,204,206,206,207,207,208,208,209,209,210,210,211,211,212,212,215,215,218,218,219,219,221,221,222,222,223,223,224,224,226,226,231,231,232,232],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,-1,60,62,64,66,-1,68,-1,70,72,74,-1,-1,76,78,80,82,-1,-1,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,-1,-1,-1,110,-1,-1,-1,-1,112,114,116,118,120,122,124,126,-1,128,130,132,134,136,138,-1,140,142,144,-1,-1,146,148,150,152,154,156,158,160,162,164,166,168,170,-1,172,-1,174,-1,-1,176,178,-1,180,182,184,186,188,190,192,-1,-1,-1,-1,-1,-1,194,196,198,200,202,-1,204,-1,-1,-1,206,208,210,212,214,216,218,220,-1,-1,-1,222,224,226,-1,-1,228,230,-1,-1,232,234,-1,-1,-1,-1,236,-1,238,240,-1,242,244,-1,246,248,-1,250,252,254,256,258,-1,-1,260,-1,262,264,266,-1,-1,-1,268,-1,270,272,274,276,-1,278,280,282,-1,284,286,-1,288,290,-1,292,294,296,298,300,302,304,-1,-1,306,-1,-1,308,310,-1,312,314,316,318,-1,320,-1,-1,-1,-1,322,324,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.638132E0,6.624E3,6.747114E7,1.502494E0,2.0407547E5,3.5301748E7,3.773E3,7.431E3,2.14099E5,1E0,1.5106794E3,1.2E1,7.805608E4,8.6206274E2,7.667683E6,9.34731E5,8.9639384E-1,6.258335E5,6.1061732E7,2.7017118E-2,7.83495E5,5.2610065E1,1.645614E6,1.92E2,1.7227725E0,8.46E2,3.94E2,6.7652373E3,8.229907E7,-1.568131E-2,1.1E2,2.2056E4,4E3,1.0402391E6,-8.720072E-3,8.48106E5,1.7314062E-4,1.5E1,2.5422776E3,3.0497742E3,1.987688E-2,6.424733E-3,2.1690162E7,2.7E1,6E0,6.067634E5,1.6531857E-2,1.9896084E-3,3.0200147E8,6.240012E-3,5.2107143E0,3.2196458E6,3.0769388E7,6.088569E6,2E0,3.2475834E7,1E0,2.65903E5,1.6904992E-1,2.5275471E2,5.85E2,1.5756359E0,2.28142E-3,1.0159207E-2,1.8246623E-3,4.1E1,-1.8329255E-2,-3.7264954E-2,-4.4677146E-3,-1.384688E-2,1.9E1,5.884028E0,2.060792E8,3.1570474E2,2.2783158E5,9.35E2,2.7809634E5,3.11E3,9.856965E-3,2.9348E2,9.653717E3,4.4768806E1,8.2474226E-1,1.5604764E5,5E-1,-1.7911965E-2,3.9595376E7,1.4523809E0,2.1722221E1,-9.742442E-3,1.052022E-2,6.3974875E5,7.1214952E0,8.865177E-1,1.9745072E7,1.504779E7,8.837831E-2,5.2826166E-1,1.2015E4,2.3E1,1.2342778E1,1E0,2.36026E5,4.822581E0,-1.7041968E-2,3.78E2,-1.9439457E-3,1.8991614E5,-4.734344E-3,-6.1037415E-4,2.9173258E-4,7.6256483E-6,1.3882445E-2,1.6667E4,4.5E1,1.2E1,2.5682794E5,8.755996E-1,6.782E4,1.61E9,-1.0423005E-2,-2.9077344E-3,7.7717104E-3,1.4697447E-2,8.97312E-4,7.20561E-3,1E0,7.330957E4,2.53E2,1.3797468E0,6.610734E6,-6.3287974E-3,3.99704E5,-1.514699E-2,4.521079E-3,-4.1292254E-3,4.54E2,2.2222222E-1,8.204587E1,9.25E0,7.4521235E6,5.704E3,9.433962E-3,9.099893E8,-1.3245831E-3,7.1609537E-3,2.1486802E-2,1.00663277E9,3.891716E8,1.4307411E7,1.8317534E-2,3.3833858E-2,1.2677199E7,2.8E1,1.7630942E-2,3.6065562E-3,2.33812E1,4.5505118E2,-1.0680174E-2,-2.5317196E-3,-2.8084991E-3,8.291912E-4,1.7515824E8,-1.724128E-2,1.7E1,5.387818E8,3.7360063E-3,5.1148495E6,7.948417E0,-2.521043E-3,4.681464E3,3.830986E-1,-2.6641665E-2,4.0677965E-2,1.8697916E11,1.7446084E0,2.5927516E11,3.2485715E2,6.2348135E-3,1.5610687E-3,1.0219829E1,2.3326934E-3,5.2288747E-1,3.2627738E5,7.364257E4,8.456265E-3,-1.8801788E-2,-1.2376696E-3,2.62888E-3,-5.575667E-3,4.222002E8,2.9924436E9,4.7E2,5.8601086E1,1.6256971E-2,4.0844156E7,3.1E1,6.0559247E2,4.717604E-3,9.750871E6,2.8946466E2,-1.2276137E-3,3.24756E5,1.5401015E1,2.2714254E-3,1.3E1,1E0,2.6682352E2,2.0966542E0,6.49E2,4.0906172E5,4.7575555E6,1.8343132E-2,3.6371837E-3,5.6553E4,2.8730897E-4,6.81069E-3,4.347826E-2,1.477425E6,3.39439E-3,3.4285715E0,1.6992E4,1.027E4,1.5597535E3,1.4773448E-2,4.6726016E3,8.834828E-3,-8.5187955E-3,8.927806E-3,1.8875422E-2,2.0737328E-2,8.848604E1,7.937589E-3,1.8645639E-3,5.0695543E-3,-4.3781796E-3,-1.0393127E-2,-2.6104737E-3,-1.6918126E-2,-5.0233593E-3,-2.5463435E-3,-7.4076266E-3,5.4886594E-4,6.1078346E-3,-1.0625035E-2,-8.112221E-4,-2.0604245E-3,-1.0686532E-2,-9.569183E-3,-1.8693283E-2,-6.134389E-3,-6.755104E-4,-2.2410916E-2,-7.678304E-3,-9.872693E-4,-1.0143723E-2,-3.3315897E-3,5.002531E-4,-9.400316E-3,-3.4369146E-3,-5.476461E-3,-1.2757724E-2,-2.185806E-2,-1.07363155E-2,2.509805E-3,-2.1570022E-3,4.5076483E-3,-7.3084206E-4,3.4398555E-3,-4.511563E-4,8.605619E-3,3.508949E-3,-7.143229E-3,-1.0727411E-3,-2.316394E-2,-8.2002245E-3,-2.1768038E-3,-9.043159E-3,-4.311128E-3,2.2704345E-4,2.7007456E-3,8.341565E-3,-5.166201E-3,-9.09263E-4,2.8435024E-3,8.020726E-3,-5.5197584E-3,-4.776428E-5,-1.1946551E-2,-6.6341856E-3,1.0415591E-2,4.239215E-3,-1.3508692E-3,-1.551234E-2,-9.692966E-4,3.6924582E-3,5.6239497E-4,9.271986E-3,7.282624E-3,-3.54966E-3,5.0641717E-3,1.0940343E-2,4.9444926E-3,1.3879153E-3,3.8908767E-3,9.750413E-3,-3.2531674E-4,3.584059E-3,-5.0480287E-3,-6.4850785E-4,1.4723835E-2,9.363921E-3,5.274578E-3,1.2663116E-2,6.046056E-3,1.7477905E-2,1.0764507E-2,4.0470823E-3,3.6605126E-3,8.475732E-3,-8.580024E-3,2.7988775E-4,4.0891925E-3,-1.6949389E-3],"split_indices":[20,56,2,45,41,28,45,2,2,2,67,52,18,28,58,45,1,27,32,45,0,33,56,9,0,39,2,8,52,45,0,52,9,2,47,0,29,0,3,52,52,0,0,45,8,3,47,0,0,7,0,53,28,1,45,17,50,67,32,27,52,2,54,0,0,0,8,0,0,0,0,3,53,7,58,28,0,33,0,0,52,50,58,57,33,53,0,7,54,53,0,0,47,56,35,1,45,57,39,9,3,54,104,12,56,0,0,0,28,0,0,27,38,0,9,3,18,28,42,9,5,0,0,0,0,0,0,8,45,2,53,45,0,1,0,0,0,2,58,47,58,45,2,57,7,0,0,0,7,5,51,0,0,9,8,0,0,54,56,0,0,0,0,31,0,8,7,0,45,58,0,48,56,0,57,31,53,31,52,0,0,54,0,27,28,28,0,0,0,27,0,7,5,1,50,0,45,10,33,0,9,52,0,1,54,0,3,8,55,54,29,28,50,0,0,1,0,0,57,9,0,54,2,11,33,0,4,0,0,0,0,57,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.578E3,8.16E2,1.762E3,3.62E2,4.54E2,1.311E3,4.51E2,3.23E2,3.9E1,3.61E2,9.3E1,4.95E2,8.16E2,3.67E2,8.4E1,2.99E2,2.4E1,2.6E1,1.3E1,3.57E2,4E0,8E0,8.5E1,4.82E2,1.3E1,5.5E1,7.61E2,7.8E1,2.89E2,7.8E1,6E0,1.67E2,1.32E2,8E0,1.6E1,7E0,1.9E1,5E0,8E0,2.7E2,8.7E1,4E0,4E0,6.4E1,2.1E1,3.34E2,1.48E2,6E0,7E0,5E1,5E0,2.8E2,4.81E2,5.7E1,2.1E1,2.51E2,3.8E1,6.1E1,1.7E1,9.5E1,7.2E1,9.4E1,3.8E1,4E0,4E0,4E0,1.2E1,1.3E1,6E0,4E0,4E0,2.56E2,1.4E1,6.5E1,2.2E1,5.6E1,8E0,1.3E1,8E0,7E0,3.27E2,1.14E2,3.4E1,4.2E1,8E0,2.76E2,4E0,4.01E2,8E1,5.3E1,4E0,9E0,1.2E1,6.6E1,1.85E2,1.2E1,2.6E1,9E0,5.2E1,9E0,8E0,4.3E1,5.2E1,5.8E1,1.4E1,8E0,8.6E1,5E0,3.3E1,7E0,5E0,1.24E2,1.32E2,5E0,9E0,1.4E1,5.1E1,1.3E1,9E0,1.7E1,3.9E1,4E0,4E0,8E0,5E0,4E0,4E0,1.65E2,1.62E2,3.3E1,8.1E1,2.8E1,6E0,3.8E1,4E0,4E0,4E0,2.2E1,2.54E2,5.6E1,3.45E2,1.6E1,6.4E1,2.5E1,2.8E1,4E0,8E0,7E0,5.9E1,1.4E2,4.5E1,6E0,6E0,1.1E1,1.5E1,5E0,4E0,4.1E1,1.1E1,5E0,4E0,4E0,4E0,3.9E1,4E0,2E1,3.2E1,4E0,5.4E1,1E1,4E0,2.8E1,5.8E1,7E0,2.6E1,8.2E1,4.2E1,1.2E1,1.2E2,5E0,4E0,8E0,6E0,2.4E1,2.7E1,8E0,5E0,4E0,5E0,1.2E1,5E0,2.4E1,1.5E1,1.04E2,6.1E1,4E0,1.58E2,9E0,2.4E1,7E0,7.4E1,2.4E1,4E0,2.2E1,1.6E1,7E0,1.5E1,2E1,2.34E2,1.6E1,4E1,5.2E1,2.93E2,1.1E1,5E0,6E1,4E0,6E0,1.9E1,2.2E1,6E0,4E1,1.9E1,1.2E1,1.28E2,1.8E1,2.7E1,7E0,4E0,5E0,1E1,2.5E1,1.6E1,7E0,4E0,4E0,3.5E1,1.3E1,7E0,2.7E1,5E0,4.3E1,1.1E1,4E0,6E0,6E0,2.2E1,7E0,5.1E1,1.1E1,1.5E1,4.5E1,3.7E1,5E0,3.7E1,4E0,8E0,4.4E1,7.6E1,4E0,4E0,1.4E1,1E1,1.2E1,1.5E1,4E0,4E0,4E0,8E0,1.8E1,6E0,1.1E1,4E0,5.6E1,4.8E1,5E0,5.6E1,1.28E2,3E1,4E0,5E0,1.8E1,6E0,4.8E1,2.6E1,7E0,1.7E1,1.8E1,4E0,9E0,7E0,1E1,5E0,8E0,1.2E1,1.38E2,9.6E1,8E0,8E0,5E0,3.5E1,1.6E1,3.6E1,2.13E2,8E1,1.4E1,4.6E1,5E0,1.4E1,6E0,1.6E1,2.8E1,1.2E1,1.2E1,7E0,7E0,5E0,1E1,1.18E2,9E0,1.8E1,6E0,1.9E1,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"325","size_leaf_vector":"1"}},{"base_weights":[-1.0925675E-4,-9.711368E-2,4.5033675E-2,-1.5238263E-1,-3.08569E-2,4.418029E-3,1.15976386E-1,-1.8418948E-1,-5.4420196E-2,-8.330429E-2,6.0381647E-2,-4.985169E-2,4.0159903E-2,1.3604516E-1,-1.9373974E-2,-1.6953349E-1,-4.6488124E-1,3.4917715E-1,-9.3841515E-2,-9.258795E-2,7.05213E-2,1.2174835E-2,1.3564563E-1,-6.1238233E-2,1.2069361E-1,3.246681E-2,3.3420795E-1,1.00644976E-1,2.0554315E-1,-9.6168436E-2,1.2693876E-1,-2.0692846E-1,-1.2206187E-1,-5.477972E-1,-8.639569E-3,2.6693936E-2,2.8617147E-3,-7.691567E-2,-2.0158287E-2,-6.046676E-2,-1.2117238E-1,-1.053391E-3,5.459896E-3,2.6698252E-2,-1.1182012E-1,1.6739845E-1,3.5234254E-2,-5.0272178E-2,-1.7191112E-1,1.8596292E-2,3.5191074E-2,-1.13374434E-1,4.4968925E-2,3.9349037E-1,7.991755E-3,7.834406E-2,1.9820718E-1,8.289919E-2,2.285971E-1,-2.4630962E-1,-4.8540443E-2,1.3276233E-2,7.070954E-2,-1.6904344E-1,-2.9474738E-1,6.02094E-2,-1.4389217E-1,-1.3439782E-2,-3.2657538E-2,3.7705284E-2,-1.0350911E-1,-7.42693E-2,5.49925E-2,-2.723516E-1,-1.0291462E-1,4.451088E-2,-4.2881023E-2,-2.2460578E-3,-7.8110434E-3,2.634802E-1,1.2186813E-1,-3.0339728E-3,7.968792E-2,1.3774745E-1,-5.6266643E-2,-1.15457565E-1,-3.3471578E-1,-1.8144777E-2,6.5354854E-3,-7.2541125E-2,-2.1902621E-1,6.398978E-3,9.4618104E-2,1.3660848E-2,2.5330007E-2,7.17133E-2,1.3230729E-2,2.1638887E-1,-3.705511E-3,1.4760818E-1,-6.0754553E-3,3.3676895E-1,2.0230351E-1,-4.548995E-3,-1.6587568E-2,6.2646985E-2,-7.574463E-2,1.0365601E-2,6.4661074E-3,-1.5618943E-1,-1.9991277E-2,-1.1393696E-1,-3.2994172E-1,-3.0076539E-3,8.021849E-3,-1.968259E-2,-1.2617259E-1,8.2217865E-3,-7.0884493E-3,-1.5616119E-1,-6.433832E-2,-8.0517344E-2,1.642448E-3,-1.238693E-3,4.8958715E-3,-2.9549496E-3,-1.856591E-2,2.6812296E-2,-1.2799518E-1,5.4007865E-2,-2.5755896E-3,-5.868279E-3,1.3601077E-2,1.6969154E-2,6.533108E-3,4.4857882E-2,1.5427986E-1,5.3447178E-3,1.5498188E-3,-1.9661987E-3,1.3484366E-2,-8.4112406E-2,-2.0729873E-2,-8.383534E-3,-3.8151275E-2,-2.3873264E-2,-6.2310756E-3,-3.1937168E-3,9.729451E-4,-1.0964955E-1,4.6038244E-2,-1.2448897E-2,-5.745433E-3,1.1993772E-2,-1.5388534E-2,7.635429E-2,2.0460543E-1,3.2251608E-2,9.293594E-2,1.4478968E-1,2.585732E-1,2.2270663E-1,-1.520339E-3,3.7247685E-1,4.5407573E-3,1.6186371E-1,2.9738885E-1,1.145055E-4,5.5199647E-3,-1.4181814E-1,-4.7418483E-2,-1.4143147E-3,2.7589407E-3,-1.3699883E-1,-3.0508056E-1,-9.430619E-3,-6.9358124E-5,-3.6041215E-1,-1.9334814E-1,-3.0851055E-2,-1.4704353E-1,2.2452336E-3,-2.9093325E-3,-1.12585425E-1,-1.887083E-2,-4.325703E-2,-1.1630727E-2,-9.0493046E-2,2.5981868E-4,-4.8258208E-2,7.578657E-3,-1.4657977E-1,1.7111942E-2,4.509957E-2,6.7880997E-3,2.0491222E-3,-1.0997457E-3,1.8231031E-4,4.2640255E-3,8.735362E-3,2.8822399E-3,-1.0420204E-1,-3.7910383E-2,3.654922E-2,-7.50346E-2,3.002783E-3,-8.849391E-2,-5.5687536E-2,-1.8595193E-1,-1.1674571E-3,4.8068436E-3,4.2146496E-2,-3.671755E-2,3.9192513E-2,1.5472254E-1,1.973643E-2,1.5501475E-1,1.7179517E-2,8.6048255E-3,8.5377626E-2,1.3907474E-2,2.1112327E-1,9.013598E-2,6.607011E-3,2.82079E-1,1.5553258E-2,5.4661236E-3,2.9235375E-1,2.2378774E-2,6.352605E-4,1.7399709E-1,2.3515414E-1,2.4663318E-2,-3.240891E-3,-9.109557E-3,-2.0807697E-2,-7.745945E-3,-8.734657E-3,-2.7717007E-3,-8.843325E-3,-1.8138776E-2,-1.986281E-2,-8.019074E-3,-1.4767757E-2,-3.795139E-3,5.0253497E-4,-5.4369527E-3,-7.748021E-3,1.7363621E-3,-3.9063105E-3,-9.65015E-3,-6.7450567E-3,-9.210031E-4,-4.9129557E-3,3.1702683E-4,2.4177379E-3,-2.3939372E-3,-5.821671E-3,8.5634354E-4,-1.1014608E-2,-5.411561E-3,3.5389322E-3,-1.1841953E-3,2.8087315E-3,-7.1149727E-4,-6.235967E-3,-9.452306E-4,1.0407883E-2,-3.3463368E-3,3.579703E-3,-2.4143183E-3,-4.8179533E-3,-7.0531014E-4,-4.5920807E-4,-7.4999346E-3,1.2873905E-3,-3.953656E-3,-1.12179285E-2,-4.350669E-3,2.8747532E-3,-6.1692502E-3,-2.5040768E-3,4.331282E-3,4.2892187E-3,-4.4705328E-3,4.7566467E-3,9.275135E-3,-7.042989E-4,9.801711E-3,2.6420571E-3,-1.2690607E-3,4.627358E-3,-1.2985391E-3,5.466354E-3,1.2672659E-2,1.8044986E-3,7.716532E-3,7.0029465E-3,1.4857529E-2,8.562868E-3,1.6774192E-2,1.0379751E-2,5.4228483E-3,1.3089824E-2,4.2084116E-3,-2.4606646E-3,4.2837504E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,-1,69,-1,71,73,-1,-1,75,77,79,81,83,85,-1,87,89,91,93,-1,95,97,99,101,103,105,-1,107,109,111,113,115,-1,-1,117,119,121,123,125,127,129,131,-1,-1,133,135,-1,137,139,141,143,145,147,-1,149,151,153,155,-1,-1,157,-1,159,-1,161,-1,163,165,-1,-1,167,169,-1,171,173,-1,175,177,-1,-1,-1,179,-1,181,183,185,187,-1,-1,-1,-1,-1,189,191,193,-1,-1,195,-1,-1,197,199,-1,-1,-1,-1,201,203,-1,205,-1,-1,-1,-1,207,209,-1,-1,211,-1,213,215,217,219,221,223,225,-1,227,-1,229,231,-1,-1,233,235,-1,-1,237,239,-1,-1,241,243,245,247,-1,-1,249,-1,251,-1,253,255,257,-1,259,261,263,-1,-1,-1,-1,-1,-1,-1,265,267,269,271,-1,273,275,277,-1,-1,279,281,283,285,-1,287,289,-1,291,-1,293,295,-1,297,-1,-1,299,-1,-1,301,303,-1,-1,-1,305,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1210398E1,2.972458E0,5.033265E0,1.3703089E0,1.7803483E0,2.1589499E0,1.7302122E0,1.2952003E0,1.7829046E0,3.4355235E-1,4.917886E-1,8.657206E-1,1.5070446E0,1.3422871E0,9.447962E-1,5.410309E-1,2.9263186E-1,5.0959516E-1,4.9822235E-1,1.974243E-1,5.199766E-2,1.5425517E-1,1.6308427E-1,4.950204E-1,5.889646E-1,1.1993756E0,1.1269009E-1,7.8731465E-1,5.0695133E-1,3.813448E-1,2.0748591E-1,5.3905773E-1,5.7766557E-1,1.7815852E-1,0E0,0E0,0E0,2.9957503E-1,0E0,1.7460287E-1,2.9491365E-1,0E0,0E0,9.610228E-2,1.9757025E-2,1.4287543E-1,6.455587E-2,4.3491298E-1,3.0141795E-1,0E0,1.1455609E-1,2.0844537E-1,1.1560341E0,2.489245E-2,0E0,3.5436952E-1,3.5278916E-1,4.2814893E-1,3.7556934E-1,1.4268774E-1,1.3480999E-1,0E0,1.9192195E-1,3.1171823E-1,2.933092E-1,2.0295244E-1,5.173762E-1,0E0,0E0,1.091176E-1,1.5288424E-1,6.6910625E-2,4.358766E-2,2.3187172E-1,3.4831595E-1,5.8431298E-2,6.898081E-2,0E0,0E0,7.68528E-2,6.546432E-2,0E0,9.958558E-3,2.9139662E-1,3.624102E-1,1.1090526E-1,2.1083105E-1,2.7071524E-2,0E0,1.7665946E-1,1.52208805E-2,6.034662E-1,5.1469755E-1,0E0,0E0,2.4238467E-1,0E0,1.5586495E-1,0E0,3.2766044E-1,0E0,2.2078061E-1,4.4405317E-1,0E0,0E0,2.5725804E-2,5.878149E-2,0E0,2.868421E-2,3.0393863E-1,0E0,8.392596E-2,1.09283924E-1,0E0,0E0,0E0,2.3618698E-1,0E0,4.252091E-2,2.7600753E-1,1.5681633E-1,7.335204E-2,0E0,0E0,0E0,0E0,0E0,1.7604165E-1,2.4251962E-1,3.4702525E-2,0E0,0E0,1.0663032E-2,0E0,0E0,1.6369935E-2,2.960512E-2,0E0,0E0,0E0,0E0,1.8725502E-1,5.097582E-1,0E0,6.961628E-2,0E0,0E0,0E0,0E0,1.1316773E-1,3.7062146E-2,0E0,0E0,4.9505726E-1,0E0,6.579412E-1,2.8810203E-1,2.1642798E-1,2.424841E-1,7.522994E-2,7.852459E-2,1.3655221E-1,0E0,2.0088673E-2,0E0,1.642797E-1,3.3573103E-1,0E0,0E0,1.9338429E-2,7.122329E-2,0E0,0E0,3.2631063E-1,3.2231092E-2,0E0,0E0,1.8844795E-1,9.753397E-2,7.4447006E-2,1.4973426E-1,0E0,0E0,6.52709E-2,0E0,9.543438E-2,0E0,6.2054038E-2,2.781968E-2,5.55588E-2,0E0,1.9946396E-1,2.5792245E-2,3.5089582E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4887717E-1,4.718711E-1,2.4172017E-1,1.080893E-1,0E0,4.2589925E-2,3.8311798E-2,2.6201338E-2,0E0,0E0,5.3714716E-1,2.176136E-1,9.4399285E-1,1.1661911E-1,0E0,2.1849161E-1,1.4573248E-1,0E0,1.5058517E-1,0E0,1.9808173E-2,4.9998462E-2,0E0,2.7675867E-2,0E0,0E0,2.1145105E-2,0E0,0E0,1.6518998E-1,9.935999E-2,0E0,0E0,0E0,7.087814E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,37,37,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,79,79,80,80,82,82,83,83,84,84,85,85,86,86,87,87,89,89,90,90,91,91,92,92,95,95,97,97,99,99,101,101,102,102,105,105,106,106,108,108,109,109,111,111,112,112,116,116,118,118,119,119,120,120,121,121,127,127,128,128,129,129,132,132,135,135,136,136,141,141,142,142,144,144,149,149,150,150,153,153,155,155,156,156,157,157,158,158,159,159,160,160,161,161,163,163,165,165,166,166,169,169,170,170,173,173,174,174,177,177,178,178,179,179,180,180,183,183,185,185,187,187,188,188,189,189,191,191,192,192,193,193,201,201,202,202,203,203,204,204,206,206,207,207,208,208,211,211,212,212,213,213,214,214,216,216,217,217,219,219,221,221,222,222,224,224,227,227,230,230,231,231,235,235],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,-1,70,-1,72,74,-1,-1,76,78,80,82,84,86,-1,88,90,92,94,-1,96,98,100,102,104,106,-1,108,110,112,114,116,-1,-1,118,120,122,124,126,128,130,132,-1,-1,134,136,-1,138,140,142,144,146,148,-1,150,152,154,156,-1,-1,158,-1,160,-1,162,-1,164,166,-1,-1,168,170,-1,172,174,-1,176,178,-1,-1,-1,180,-1,182,184,186,188,-1,-1,-1,-1,-1,190,192,194,-1,-1,196,-1,-1,198,200,-1,-1,-1,-1,202,204,-1,206,-1,-1,-1,-1,208,210,-1,-1,212,-1,214,216,218,220,222,224,226,-1,228,-1,230,232,-1,-1,234,236,-1,-1,238,240,-1,-1,242,244,246,248,-1,-1,250,-1,252,-1,254,256,258,-1,260,262,264,-1,-1,-1,-1,-1,-1,-1,266,268,270,272,-1,274,276,278,-1,-1,280,282,284,286,-1,288,290,-1,292,-1,294,296,-1,298,-1,-1,300,-1,-1,302,304,-1,-1,-1,306,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,2.1975278E8,1.067536E3,2.3544883E8,2.0407547E5,3.5301748E7,1.3521131E8,7.156528E7,1E0,1.7227913E3,1.5106794E3,1.0258741E2,5.7835355E6,4.247868E3,1.892E3,3.1140776E0,9.11898E5,1.067509E6,8.3306855E6,4.5859104E-1,1.164249E7,3.4570816E0,1E0,1E0,1.2E1,1.0133419E8,4.1331047E-1,3.956E3,3.0379269E1,9.042859E4,1.4540612E7,7.359468E8,1.22E2,-8.639569E-3,2.6693936E-2,2.8617147E-3,2.8839298E7,-2.0158287E-2,7.524668E10,2.8729197E8,-1.053391E-3,5.459896E-3,5.884028E0,5.386213E6,5.857143E0,6.932264E8,7E0,3.7307863E0,1.8596292E-2,7.27E2,6.747114E7,1.04E3,4.832E3,7.991755E-3,3.7307932E0,1.1803382E4,3.1984328E5,1.0215407E1,3.791269E7,1E0,1.3276233E-2,9.5E2,1.2342778E1,9.253112E-1,2.2783158E5,3E0,-1.3439782E-2,-3.2657538E-2,1.6907827E9,1.753919E3,1.5377966E6,1.732E3,1.7956694E-4,2.8169732E10,1E0,2.857E3,-2.2460578E-3,-7.8110434E-3,1.9041E4,7.6188426E0,-3.0339728E-3,2.342147E7,5.7018555E4,3.3030225E2,7E0,1.2E1,2.613714E6,6.5354854E-3,1.9214156E3,7.123E3,1.1095187E1,8.027061E2,1.3660848E-2,2.5330007E-2,7.805608E4,1.3230729E-2,1E0,-3.705511E-3,2E0,-6.0754553E-3,2E1,2.2529187E0,-4.548995E-3,-1.6587568E-2,2.8122288E3,1.6E1,1.0365601E-2,1E0,4.445375E6,-1.9991277E-2,3.830986E-1,2.4375E0,-3.0076539E-3,8.021849E-3,-1.968259E-2,1.4195741E0,8.2217865E-3,1.1953147E2,1.1855755E8,1.4964736E4,3.8E1,1.642448E-3,-1.238693E-3,4.8958715E-3,-2.9549496E-3,-1.856591E-2,3.2E2,2.175722E6,5.2E1,-2.5755896E-3,-5.868279E-3,5.513E3,1.6969154E-2,6.533108E-3,3.9E1,7.325843E0,5.3447178E-3,1.5498188E-3,-1.9661987E-3,1.3484366E-2,8.91942E5,2.1722221E1,-8.383534E-3,1.8020409E1,-2.3873264E-2,-6.2310756E-3,-3.1937168E-3,9.729451E-4,7.920064E5,6.3997424E-1,-1.2448897E-2,-5.745433E-3,1E0,-1.5388534E-2,2.0211798E6,1.296E3,1.694761E9,4.77721E3,2.2023761E0,9.391714E4,9.1E3,-1.520339E-3,5.2773216E1,4.5407573E-3,8.2187925E6,2.1196339E1,1.145055E-4,5.5199647E-3,5.961E3,8.3956606E5,-1.4143147E-3,2.7589407E-3,1.04E3,5.8372E4,-9.430619E-3,-6.9358124E-5,8.00791E5,2.25E2,1.923637E3,1.4622E4,2.2452336E-3,-2.9093325E-3,5.2228385E6,-1.887083E-2,2.775359E7,-1.1630727E-2,9.705292E6,5.4091E4,1.6621418E0,7.578657E-3,6.137763E2,8E0,8.3306855E6,6.7880997E-3,2.0491222E-3,-1.0997457E-3,1.8231031E-4,4.2640255E-3,8.735362E-3,2.8822399E-3,3.6120397E-1,1E0,8E-3,1.1494032E7,3.002783E-3,1.3597637E8,4.3531516E2,2.0754387E1,-1.1674571E-3,4.8068436E-3,2.3E1,1E0,7.2488395E6,5.3038636E7,1.973643E-2,2E0,4.155E3,8.6048255E-3,2.3058404E7,1.3907474E-2,1.2030976E0,2.5030267E5,6.607011E-3,9.000245E4,1.5553258E-2,5.4661236E-3,1.4880174E0,2.2378774E-2,6.352605E-4,6.963859E1,2.851369E2,2.4663318E-2,-3.240891E-3,-9.109557E-3,3.01E2,-7.745945E-3,-8.734657E-3,-2.7717007E-3,-8.843325E-3,-1.8138776E-2,-1.986281E-2,-8.019074E-3,-1.4767757E-2,-3.795139E-3,5.0253497E-4,-5.4369527E-3,-7.748021E-3,1.7363621E-3,-3.9063105E-3,-9.65015E-3,-6.7450567E-3,-9.210031E-4,-4.9129557E-3,3.1702683E-4,2.4177379E-3,-2.3939372E-3,-5.821671E-3,8.5634354E-4,-1.1014608E-2,-5.411561E-3,3.5389322E-3,-1.1841953E-3,2.8087315E-3,-7.1149727E-4,-6.235967E-3,-9.452306E-4,1.0407883E-2,-3.3463368E-3,3.579703E-3,-2.4143183E-3,-4.8179533E-3,-7.0531014E-4,-4.5920807E-4,-7.4999346E-3,1.2873905E-3,-3.953656E-3,-1.12179285E-2,-4.350669E-3,2.8747532E-3,-6.1692502E-3,-2.5040768E-3,4.331282E-3,4.2892187E-3,-4.4705328E-3,4.7566467E-3,9.275135E-3,-7.042989E-4,9.801711E-3,2.6420571E-3,-1.2690607E-3,4.627358E-3,-1.2985391E-3,5.466354E-3,1.2672659E-2,1.8044986E-3,7.716532E-3,7.0029465E-3,1.4857529E-2,8.562868E-3,1.6774192E-2,1.0379751E-2,5.4228483E-3,1.3089824E-2,4.2084116E-3,-2.4606646E-3,4.2837504E-3],"split_indices":[20,102,7,52,12,28,45,45,7,89,4,52,56,47,52,2,54,29,9,45,27,45,57,6,8,18,7,35,2,56,28,45,5,10,0,0,0,45,0,31,5,0,0,53,51,54,7,3,53,0,0,45,2,2,0,38,4,33,56,45,64,0,0,54,56,28,3,0,0,12,52,33,2,38,31,85,2,0,0,9,54,0,45,45,52,6,8,9,0,52,2,54,52,0,0,28,0,102,0,6,0,3,35,0,0,4,3,0,102,50,0,56,54,0,0,0,53,0,58,45,55,3,0,0,0,0,0,0,1,3,0,0,2,0,0,3,53,0,0,0,0,9,53,0,46,0,0,0,0,28,27,0,0,16,0,28,29,7,52,35,28,29,0,58,0,50,54,0,0,10,28,0,0,2,29,0,0,1,10,4,2,0,0,47,0,51,0,9,33,58,0,4,3,45,0,0,0,0,0,0,0,42,79,57,12,0,31,4,58,0,0,3,84,45,7,0,17,10,0,45,0,35,33,0,33,0,0,54,0,0,56,58,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.558E3,8.12E2,1.746E3,4.42E2,3.7E2,1.111E3,6.35E2,3.33E2,1.09E2,2.35E2,1.35E2,4.41E2,6.7E2,5.53E2,8.2E1,3.18E2,1.5E1,9E0,1E2,2.22E2,1.3E1,8.3E1,5.2E1,4.14E2,2.7E1,6.54E2,1.6E1,3.68E2,1.85E2,5.4E1,2.8E1,1.76E2,1.42E2,1.1E1,4E0,5E0,4E0,9.6E1,4E0,1.06E2,1.16E2,4E0,9E0,7.5E1,8E0,3.9E1,1.3E1,3.78E2,3.6E1,6E0,2.1E1,5.1E1,6.03E2,1.1E1,5E0,3.01E2,6.7E1,3E1,1.55E2,1.2E1,4.2E1,7E0,2.1E1,1.25E2,5.1E1,1.5E1,1.27E2,4E0,7E0,1.8E1,7.8E1,9.5E1,1.1E1,1.1E1,1.05E2,6E1,1.5E1,4E0,4E0,1.1E1,2.8E1,4E0,9E0,1.1E1,3.67E2,2.8E1,8E0,1.4E1,7E0,3.8E1,1.3E1,3.4E2,2.63E2,7E0,4E0,2.92E2,9E0,6.3E1,4E0,2.3E1,7E0,2.8E1,1.27E2,5E0,7E0,8E0,3.4E1,6E0,1.5E1,1.2E2,5E0,9E0,4.2E1,7E0,8E0,7E0,1.2E2,4E0,1.4E1,3.2E1,4.6E1,9E1,5E0,4E0,7E0,4E0,7E0,1.7E1,8.8E1,5.5E1,5E0,6E0,9E0,6E0,5E0,9E0,1.9E1,5E0,4E0,5E0,6E0,2.05E2,1.62E2,1.6E1,1.2E1,4E0,4E0,6E0,8E0,2.9E1,9E0,9E0,4E0,3.35E2,5E0,2.27E2,3.6E1,1.03E2,1.89E2,2.5E1,3.8E1,1.6E1,7E0,2.4E1,4E0,9.1E1,3.6E1,4E0,4E0,9E0,2.5E1,9E0,6E0,1.08E2,1.2E1,5E0,4E0,3.3E1,9E0,2.2E1,9.8E1,7E0,7E0,2.8E1,4E0,4.2E1,4E0,8E1,1E1,1.1E1,6E0,7.8E1,1E1,5.1E1,4E0,5E0,4E0,5E0,4E0,1.5E1,4E0,1.42E2,6.3E1,7.9E1,8.3E1,4E0,8E0,1.8E1,1.1E1,4E0,5E0,2.07E2,1.28E2,1.55E2,7.2E1,6E0,3E1,9.4E1,9E0,1.83E2,6E0,1E1,1.5E1,7E0,3.1E1,8E0,8E0,1.4E1,1E1,7E0,8.4E1,2.9E1,7E0,4E0,5E0,2.1E1,4E0,7.3E1,3.5E1,5E0,7E0,2.7E1,6E0,4E0,5E0,1.5E1,7E0,9.4E1,4E0,2.1E1,7E0,8E0,3.4E1,7.4E1,6E0,5E0,5E0,5E0,6E0,2.5E1,5.3E1,4E0,6E0,4.3E1,8E0,1.14E2,2.8E1,6E0,5.7E1,5.6E1,2.3E1,6.1E1,2.2E1,4E0,4E0,4E0,1.4E1,7E0,4E0,1.9E2,1.7E1,1.16E2,1.2E1,1.14E2,4.1E1,2.6E1,4.6E1,6E0,2.4E1,5.1E1,4.3E1,1.72E2,1.1E1,4E0,6E0,9E0,6E0,4E0,2.7E1,5E0,9E0,5.4E1,3E1,2.4E1,5E0,1.7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"307","size_leaf_vector":"1"}},{"base_weights":[-3.8834424E-3,-9.4827145E-2,3.9397966E-2,-1.09834075E-1,1.1035147E-1,1.2509035E-2,1.4516628E-1,-8.0313504E-2,-2.3090346E-1,2.8723763E-2,5.4632366E-2,-1.0320738E-2,1.0427747E-1,2.4506368E-1,1.074325E-1,-1.3906555E-1,-2.0544296E-2,-3.6176845E-1,-1.7370881E-1,8.2484394E-2,-1.1176223E-2,-1.4019615E-3,-1.5191884E-1,4.6087638E-2,1.429099E-1,2.6558557E-1,-2.6881201E-3,-1.7386438E-1,1.2428504E-1,-1.8200117E-1,-8.153215E-2,-5.581199E-2,3.8109355E-2,-2.4363466E-1,-4.5498407E-1,-1.4056094E-1,-2.886593E-1,5.4461192E-2,1.3469511E-2,-2.3450717E-2,8.178835E-2,-1.8878973E-1,-6.959573E-2,-1.0964186E-2,5.7953384E-2,1.307665E-1,1.8307613E-2,3.0157221E-2,2.342604E-1,-1.7122571E-5,-2.8950742E-1,1.072779E-1,3.0940118E-1,-1.3683125E-1,-2.7155304E-1,-5.920237E-2,-1.584573E-1,-4.3228757E-2,-2.2768404E-1,7.440959E-2,-6.4994834E-2,-3.2672333E-3,-1.3971295E-2,-5.186623E-1,-5.4600798E-3,-1.7379159E-1,-6.670599E-2,-3.567447E-1,-1.4311033E-1,-3.1815576E-3,1.622731E-1,-4.6238694E-2,4.49164E-2,1.8399276E-1,9.616127E-3,-1.538205E-1,-2.8289276E-1,-1.1684404E-3,-6.891242E-3,1.4615245E-1,2.6749343E-2,1.463756E-1,-4.135459E-2,1.3619067E-1,2.8086329E-1,-8.410149E-3,-1.7628148E-2,1.13646E-1,-9.775871E-3,2.3055092E-2,2.1129698E-1,-1.7223954E-1,-3.718975E-2,-2.0966077E-1,-4.00882E-1,-7.0764266E-2,7.455652E-2,-1.8534778E-2,-1.1150018E-1,-1.9692728E-1,-3.409457E-2,-2.813208E-3,-1.4869991E-2,9.009269E-2,-7.152087E-2,-1.2215334E-1,-1.6927399E-2,-4.1433543E-1,-3.741383E-2,-1.3512309E-1,-2.856695E-1,-9.4821855E-2,-6.6152285E-3,-9.232216E-3,-2.2528296E-2,-1.4025724E-2,1.1543805E-3,-6.722766E-3,2.9791871E-2,1.1137663E-2,4.8511163E-3,8.694719E-2,-5.4554943E-2,5.4024335E-2,-1.6247286E-1,2.6135138E-1,9.910723E-2,-1.6904198E-2,1.06829606E-1,-1.6624181E-1,-2.300679E-3,-1.7404031E-2,-6.7318147E-3,2.666355E-3,-3.2275137E-3,-4.9044737E-3,1.9096012E-1,-9.826425E-3,4.4034928E-2,7.501522E-3,1.5875609E-1,-1.4536975E-2,8.757031E-2,1.19720325E-2,9.239664E-2,2.0331236E-2,2.48072E-1,1.0381841E-1,2.2409974E-2,1.331675E-2,4.6597556E-3,-1.8622147E-1,2.60896E-3,6.1872657E-2,-8.9533344E-2,-1.3202077E-1,-2.884637E-1,-2.3574483E-2,-6.102104E-3,-8.005365E-2,4.325288E-3,8.580152E-3,-1.8745377E-3,-1.5551737E-1,-3.7338797E-2,-1.3404287E-2,-3.594456E-3,-5.0507233E-2,3.2061283E-2,1.1985527E-1,2.1838634E-3,-6.160126E-3,-2.7246092E-4,-8.00212E-3,-4.1597346E-3,-5.6823265E-2,2.0009323E-3,-1.2414682E-2,-2.334175E-2,-2.127502E-1,-9.693863E-2,-7.3298574E-3,-2.0170925E-2,-4.480411E-2,-7.894521E-3,-2.1096303E-3,1.8704745E-3,5.610035E-3,-3.9402186E-3,-2.1192336E-2,1.1750955E-1,-8.016588E-2,-6.0786973E-3,7.2729304E-2,-4.283808E-2,-1.2006507E-2,-2.6063018E-3,1.4178787E-1,3.2437778E-1,2.180033E-1,1.9134494E-2,-2.8729202E-2,1.1023233E-2,6.099976E-2,1.1772391E-2,-8.866515E-3,-3.6214248E-3,2.2285606E-1,1.2583797E-3,7.846081E-3,1.3844162E-1,-6.6955276E-3,5.6307376E-3,1.7403904E-1,5.0597187E-2,8.411772E-3,2.8285093E-4,1.396107E-3,1.148213E-1,2.9132837E-1,1.5460025E-1,3.3644315E-2,1.2642935E-1,-3.1318283E-3,-1.0711289E-2,-2.4870199E-3,6.191743E-3,-1.5614583E-3,-1.0491099E-2,-7.130929E-4,-9.686989E-3,-2.209084E-2,-1.0381758E-2,-4.7238083E-3,4.7336638E-4,-1.0088907E-2,-2.2254984E-3,1.250697E-3,-4.6711382E-3,-9.867908E-3,-1.752035E-3,-2.2148164E-3,2.4680535E-3,5.071293E-3,1.4883243E-2,1.8068096E-3,-3.5443737E-3,-6.031478E-4,-4.030889E-3,-4.5440965E-3,-1.2610537E-2,-7.614466E-3,-1.1958675E-3,-3.7074941E-3,5.5908156E-4,-2.9027294E-3,2.2580733E-3,1.5350125E-3,-3.4410513E-3,7.3387283E-3,2.803584E-3,-3.6292933E-3,-2.1239394E-2,2.224981E-3,-3.72999E-3,-3.864348E-3,4.1840198E-3,4.389093E-3,-3.7540505E-3,1.48671735E-2,2.9477673E-3,1.806359E-2,8.847464E-3,1.4196614E-2,3.6732599E-3,4.631401E-3,-1.0286092E-3,-3.4403696E-3,3.6948366E-4,1.766784E-3,6.169946E-3,7.439581E-3,1.3482879E-2,1.7452245E-3,-3.4230032E-3,1.0334561E-2,3.493432E-3,8.033045E-3,1.6726837E-2,-3.2290304E-3,6.485396E-3,3.4676616E-3,8.9419205E-3,9.059568E-3,1.639374E-2,2.6671654E-3,9.075906E-3,3.4913789E-3,-2.4643121E-3,6.999124E-3,-8.771533E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,79,81,-1,-1,83,-1,85,87,89,91,93,95,97,99,101,103,105,-1,-1,107,-1,109,111,113,115,117,119,121,123,125,127,129,131,133,-1,135,137,139,141,143,145,-1,-1,147,-1,-1,149,151,153,155,157,159,161,-1,163,165,167,-1,-1,169,171,173,175,177,-1,179,181,183,185,-1,-1,-1,-1,-1,187,-1,-1,189,191,193,195,197,199,201,203,205,-1,-1,-1,-1,-1,-1,207,-1,209,211,213,-1,215,-1,217,-1,219,221,-1,-1,-1,223,-1,225,227,229,231,-1,-1,233,-1,-1,-1,235,237,-1,-1,239,241,243,245,-1,-1,-1,-1,247,-1,-1,-1,249,251,-1,-1,253,-1,-1,-1,-1,255,257,259,261,263,265,267,-1,-1,269,271,273,275,277,-1,279,-1,-1,-1,281,-1,283,285,-1,-1,287,289,-1,-1,-1,291,293,295,297,299,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0119697E1,2.5646954E0,4.952427E0,2.7441864E0,1.4438633E0,2.9131992E0,1.3043079E0,2.1848464E0,1.067358E0,0E0,4.2152157E-1,1.4066374E0,6.161139E-1,6.0307646E-1,1.2446752E0,7.608142E-1,6.4338636E-1,3.8468695E-1,3.671348E-1,2.3697025E-1,0E0,1.9260386E0,1.8719244E-1,3.6558002E-1,4.0259337E-1,8.114157E-1,0E0,3.0089948E-1,7.349925E-1,6.869073E-1,2.2422904E-1,4.0972257E-1,4.4273293E-1,1.2280011E-1,4.7166348E-1,1.9701815E-1,1.768707E-1,2.6774302E-1,0E0,1.2951739E0,1.6210208E0,1.0456729E-1,1.026894E-1,0E0,2.9332453E-1,4.3323135E-1,0E0,0E0,3.4565687E-1,0E0,9.65184E-3,4.5627403E-1,2.0075834E-1,4.2049885E-1,3.882141E-1,1.7075214E-1,2.5797105E-1,2.4879766E-1,1.3901138E-1,2.0555535E-1,8.31036E-2,0E0,0E0,1.6989565E-1,0E0,2.1231616E-1,4.6678856E-2,1.5949643E-1,2.1231061E-1,1.2962195E-1,3.7641346E-2,6.947826E-1,4.016662E-1,5.7166123E-1,3.3696315E-1,3.9200783E-2,5.9696972E-2,4.495044E-2,0E0,3.252554E-1,3.2146198E-1,2.498312E-1,4.7595417E-1,1.09194875E-1,1.3049555E-1,0E0,0E0,6.8070316E-1,0E0,0E0,6.639278E-2,2.890315E-1,1.76654E-1,2.1888363E-1,2.9824448E-1,1.5161604E-1,1.04443595E-1,0E0,7.906696E-2,6.986642E-2,1.8946166E-1,0E0,0E0,2.0643336E-1,3.0107163E-2,7.361755E-3,4.3189093E-2,2.409482E-2,0E0,1.1276305E-1,1.7040229E-1,5.295764E-2,1.7264629E-2,0E0,0E0,0E0,0E0,0E0,6.7218706E-2,0E0,0E0,1.2477526E-1,7.2828627E-1,3.6547738E-1,6.47241E-2,3.0818295E-1,4.2370865E-1,2.9562658E-1,1.5087721E-1,1.9714057E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2128919E-1,0E0,2.5856543E-1,2.0859633E-1,2.126944E-1,0E0,6.521137E-2,0E0,2.859062E-2,0E0,1.449933E-1,3.3847404E-1,0E0,0E0,0E0,2.7866507E-1,0E0,8.929266E-2,1.4785628E-1,1.647329E-1,1.5609789E-1,0E0,0E0,1.22799635E-1,0E0,0E0,0E0,7.338849E-2,4.0859923E-2,0E0,0E0,3.0858782E-1,4.852934E-2,1.6261274E-1,5.4735538E-2,0E0,0E0,0E0,0E0,1.07661225E-2,0E0,0E0,0E0,4.462725E-2,1.2589878E-1,0E0,0E0,2.0581422E-2,0E0,0E0,0E0,0E0,5.0516844E-2,2.4390057E-2,4.4043124E-2,9.666364E-1,7.1125203E-1,2.8397548E-1,1.4585963E-1,0E0,0E0,2.1456325E-1,1.0009956E-1,1.5107393E-1,8.371422E-2,1.4400321E-1,0E0,3.0879036E-2,0E0,0E0,0E0,2.9323399E-2,0E0,1.1818601E-1,8.3952725E-2,0E0,0E0,1.9573283E-1,1.7264098E-1,0E0,0E0,0E0,3.419149E-2,6.590009E-2,3.6831945E-2,1.6575678E-1,3.2273483E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,45,45,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,81,81,82,82,83,83,84,84,87,87,90,90,91,91,92,92,93,93,94,94,95,95,96,96,98,98,99,99,100,100,103,103,104,104,105,105,106,106,107,107,109,109,110,110,111,111,112,112,118,118,121,121,122,122,123,123,124,124,125,125,126,126,127,127,128,128,129,129,136,136,138,138,139,139,140,140,142,142,144,144,146,146,147,147,151,151,153,153,154,154,155,155,156,156,159,159,163,163,164,164,167,167,168,168,169,169,170,170,175,175,179,179,180,180,183,183,188,188,189,189,190,190,191,191,192,192,193,193,194,194,197,197,198,198,199,199,200,200,201,201,203,203,207,207,209,209,210,210,213,213,214,214,218,218,219,219,220,220,221,221,222,222],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,80,82,-1,-1,84,-1,86,88,90,92,94,96,98,100,102,104,106,-1,-1,108,-1,110,112,114,116,118,120,122,124,126,128,130,132,134,-1,136,138,140,142,144,146,-1,-1,148,-1,-1,150,152,154,156,158,160,162,-1,164,166,168,-1,-1,170,172,174,176,178,-1,180,182,184,186,-1,-1,-1,-1,-1,188,-1,-1,190,192,194,196,198,200,202,204,206,-1,-1,-1,-1,-1,-1,208,-1,210,212,214,-1,216,-1,218,-1,220,222,-1,-1,-1,224,-1,226,228,230,232,-1,-1,234,-1,-1,-1,236,238,-1,-1,240,242,244,246,-1,-1,-1,-1,248,-1,-1,-1,250,252,-1,-1,254,-1,-1,-1,-1,256,258,260,262,264,266,268,-1,-1,270,272,274,276,278,-1,280,-1,-1,-1,282,-1,284,286,-1,-1,288,290,-1,-1,-1,292,294,296,298,300,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.4409692E3,8.32E3,4.987639E7,2.46E2,1.8137958E6,2.909019E3,2.074E3,1.0908286E3,2.8723763E-2,1.2342778E1,4.0844156E7,1.21E3,5.5544252E7,1.70752E4,1E0,6.057963E2,8.672049E9,1.2427474E8,1.4386049E6,-1.1176223E-2,7.298614E2,1.415178E3,5.798305E1,2.062993E3,7.969174E7,-2.6881201E-3,1.4713E4,7.765071E0,2.2444444E0,2.4343E5,2.0687214E6,2.8091298E7,6.5029144E5,3.285E3,2.6835732E3,4.551684E6,1.5595E4,1.3469511E-2,3.422351E6,8.447369E0,1.1347826E1,2.5129596E11,-1.0964186E-2,8.916961E2,1.9162654E7,1.8307613E-2,3.0157221E-2,4.6816683E-1,-1.7122571E-5,2.894E3,5.68E2,2.13537E5,2.8080197E2,4.784E3,9.705292E6,1E0,8.542683E1,2.0783027E5,7.4259315E0,4.5E0,-3.2672333E-3,-1.3971295E-2,2.9722316E0,-5.4600798E-3,2.1109243E-1,1E0,8.8070023E-1,8.629299E4,2.7258065E0,6.681133E-1,1.6093831E0,1E0,3.872302E1,1.108E3,7.221312E-2,2.0907634E3,8.657441E7,-6.891242E-3,1.4431512E-1,1E1,1E0,7E0,6.019605E2,3.3382E4,-8.410149E-3,-1.7628148E-2,1.9013363E4,-9.775871E-3,2.3055092E-2,1.0772926E2,1.113742E6,3.17441E5,2.0184135E0,1.892E3,6.9740294E5,2.8807552E7,-1.8534778E-2,1.066585E6,1.0478362E-1,1.4930757E0,-2.813208E-3,-1.4869991E-2,2.0694196E7,4.0501267E8,1.02E2,2.5629143E3,2.1818182E0,-3.741383E-2,6.7628815E2,7.6599895E9,1.1392E4,2.1897722E10,-9.232216E-3,-2.2528296E-2,-1.4025724E-2,1.1543805E-3,-6.722766E-3,9.35E2,1.1137663E-2,4.8511163E-3,1.4642603E2,3.7311627E2,3.401282E1,5.4468E4,2.914E3,5.4582E4,1.4777102E9,2.4042394E5,1.12E3,-2.300679E-3,-1.7404031E-2,-6.7318147E-3,2.666355E-3,-3.2275137E-3,-4.9044737E-3,2.5956412E2,-9.826425E-3,4.5723195E6,4E0,5.513E3,-1.4536975E-2,1.2608696E0,1.19720325E-2,1.7589172E1,2.0331236E-2,2.043739E0,1.2679E4,2.2409974E-2,1.331675E-2,4.6597556E-3,1.3177E4,2.60896E-3,3.1809045E2,1.00137E5,3.68E2,2.6216663E-5,-2.3574483E-2,-6.102104E-3,3.2485715E2,4.325288E-3,8.580152E-3,-1.8745377E-3,2.5228915E0,2.155E3,-1.3404287E-2,-3.594456E-3,2.4576474E8,8.751669E4,6.9998717E-4,1.5439131E2,-6.160126E-3,-2.7246092E-4,-8.00212E-3,-4.1597346E-3,5.765574E0,2.0009323E-3,-1.2414682E-2,-2.334175E-2,1.8159722E-1,1.4411988E5,-7.3298574E-3,-2.0170925E-2,7.883E3,-7.894521E-3,-2.1096303E-3,1.8704745E-3,5.610035E-3,2.109E3,3.5029E4,3.06E2,2.4037695E6,5.8426323E0,3.5E2,1.9557823E0,-1.2006507E-2,-2.6063018E-3,3.824611E0,1.6837112E7,3.0595E4,4.4768806E1,2.6704392E10,1.1023233E-2,1.6988909E3,1.1772391E-2,-8.866515E-3,-3.6214248E-3,1.5944675E-4,1.2583797E-3,4.2782607E0,2.1383E4,-6.6955276E-3,5.6307376E-3,6.3697716E1,5.964E3,8.411772E-3,2.8285093E-4,1.396107E-3,1.0872E4,4E0,2.3370175E0,1.03E2,2.6364462E7,-3.1318283E-3,-1.0711289E-2,-2.4870199E-3,6.191743E-3,-1.5614583E-3,-1.0491099E-2,-7.130929E-4,-9.686989E-3,-2.209084E-2,-1.0381758E-2,-4.7238083E-3,4.7336638E-4,-1.0088907E-2,-2.2254984E-3,1.250697E-3,-4.6711382E-3,-9.867908E-3,-1.752035E-3,-2.2148164E-3,2.4680535E-3,5.071293E-3,1.4883243E-2,1.8068096E-3,-3.5443737E-3,-6.031478E-4,-4.030889E-3,-4.5440965E-3,-1.2610537E-2,-7.614466E-3,-1.1958675E-3,-3.7074941E-3,5.5908156E-4,-2.9027294E-3,2.2580733E-3,1.5350125E-3,-3.4410513E-3,7.3387283E-3,2.803584E-3,-3.6292933E-3,-2.1239394E-2,2.224981E-3,-3.72999E-3,-3.864348E-3,4.1840198E-3,4.389093E-3,-3.7540505E-3,1.48671735E-2,2.9477673E-3,1.806359E-2,8.847464E-3,1.4196614E-2,3.6732599E-3,4.631401E-3,-1.0286092E-3,-3.4403696E-3,3.6948366E-4,1.766784E-3,6.169946E-3,7.439581E-3,1.3482879E-2,1.7452245E-3,-3.4230032E-3,1.0334561E-2,3.493432E-3,8.033045E-3,1.6726837E-2,-3.2290304E-3,6.485396E-3,3.4676616E-3,8.9419205E-3,9.059568E-3,1.639374E-2,2.6671654E-3,9.075906E-3,3.4913789E-3,-2.4643121E-3,6.999124E-3,-8.771533E-4],"split_indices":[20,52,2,45,10,28,32,2,4,0,54,45,2,45,33,102,52,5,45,28,0,52,52,47,55,7,0,9,35,56,29,47,45,28,0,4,29,9,0,9,53,53,31,0,33,47,0,0,41,0,0,3,11,4,10,9,67,52,33,53,54,0,0,57,0,39,16,27,33,54,27,54,74,58,0,38,4,45,0,58,3,66,8,52,9,0,0,52,0,0,58,9,32,53,2,47,9,0,1,39,42,0,0,50,7,8,4,56,0,52,5,2,5,0,0,0,0,0,0,0,0,4,52,58,1,2,9,7,28,0,0,0,0,0,0,0,52,0,28,6,2,0,53,0,56,0,41,2,0,0,0,9,0,4,29,0,38,0,0,52,0,0,0,58,0,0,0,5,28,34,58,0,0,0,0,53,0,0,0,27,33,0,0,2,0,0,0,0,0,1,0,51,53,2,53,0,0,54,45,29,58,31,0,52,0,0,0,42,0,56,9,0,0,58,2,0,0,0,2,8,38,3,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.569E3,8.28E2,1.741E3,7.72E2,5.6E1,1.389E3,3.52E2,6.22E2,1.5E2,5E0,5.1E1,1.113E3,2.76E2,9.5E1,2.57E2,3.13E2,3.09E2,4.4E1,1.06E2,4.7E1,4E0,1.048E3,6.5E1,1.11E2,1.65E2,8.9E1,6E0,1.4E1,2.43E2,1.78E2,1.35E2,1.93E2,1.16E2,2.1E1,2.3E1,8.4E1,2.2E1,4.2E1,5E0,8.29E2,2.19E2,4.4E1,2.1E1,4E0,1.07E2,1.58E2,7E0,6E0,8.3E1,6E0,8E0,2.24E2,1.9E1,1.2E2,5.8E1,1.06E2,2.9E1,1.81E2,1.2E1,8.6E1,3E1,4E0,1.7E1,1.9E1,4E0,5.7E1,2.7E1,1.4E1,8E0,2.8E1,1.4E1,6.22E2,2.07E2,9E1,1.29E2,3.4E1,1E1,1.1E1,1E1,2.7E1,8E1,1.45E2,1.3E1,2.8E1,5.5E1,4E0,4E0,2.2E2,4E0,6E0,1.3E1,8.8E1,3.2E1,4.1E1,1.7E1,9.8E1,8E0,4E0,2.5E1,9E0,1.72E2,4E0,8E0,7.8E1,8E0,1.3E1,1.7E1,1.5E1,4E0,4.4E1,1.3E1,1.8E1,9E0,6E0,8E0,4E0,4E0,5E0,2.3E1,6E0,8E0,3.6E1,5.86E2,1.99E2,8E0,4.6E1,4.4E1,1.02E2,2.7E1,3E1,4E0,6E0,4E0,6E0,5E0,4E0,2.3E1,5E0,7.5E1,1.2E1,1.33E2,4E0,9E0,7E0,2.1E1,9E0,4.6E1,2.15E2,5E0,8E0,5E0,8.3E1,5E0,1.1E1,2.1E1,2.2E1,1.9E1,1.3E1,4E0,9.3E1,5E0,4E0,4E0,1.5E1,1E1,5E0,4E0,1.38E2,3.4E1,5.8E1,2E1,4E0,4E0,5E0,8E0,1E1,7E0,5E0,1E1,1.3E1,3.1E1,7E0,6E0,1.1E1,7E0,5E0,4E0,6E0,1.7E1,8E0,2.8E1,3.83E2,2.03E2,1.67E2,3.2E1,4E0,4E0,1.7E1,2.9E1,1.7E1,2.7E1,9.8E1,4E0,2.1E1,6E0,2.6E1,4E0,1.9E1,4E0,5.5E1,2E1,5E0,7E0,1.16E2,1.7E1,4E0,5E0,6E0,1.5E1,3E1,1.6E1,5.3E1,1.62E2,1.6E1,6.7E1,4E0,7E0,1.5E1,6E0,8E0,1.4E1,5E0,1.4E1,8E1,1.3E1,1E1,5E0,5E0,5E0,1.2E1,1.26E2,6E0,2.8E1,5.4E1,4E0,1.4E1,6E0,4E0,6E0,4E0,9E0,1.7E1,1.4E1,7E0,4E0,8E0,9E0,4E0,4E0,1.8E1,1E1,3.76E2,7E0,1.17E2,8.6E1,1.1E1,1.56E2,6E0,2.6E1,5E0,1.2E1,2.2E1,7E0,1.1E1,6E0,9E0,1.8E1,4.6E1,5.2E1,1.6E1,5E0,9E0,1E1,4.1E1,1.4E1,9E0,1.1E1,1.09E2,7E0,7E0,1E1,1E1,5E0,9E0,2.1E1,4E0,1.2E1,3.7E1,1.6E1,1.48E2,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"301","size_leaf_vector":"1"}},{"base_weights":[-9.289048E-4,3.0199185E-2,-1.1058059E-1,1.0515114E-2,1.5680437E-1,-1.7561997E-1,-2.618482E-2,-2.4289414E-2,7.817616E-2,2.9511025E-2,1.362583E-1,-1.4783864E-1,-2.8617167E-1,-6.9013454E-2,4.731989E-2,-4.720935E-2,2.5573043E-2,5.1757067E-2,1.4557508E-1,3.6501747E-2,1.842288E-1,-1.8665032E-1,-7.80556E-2,-3.985762E-1,-1.9337443E-1,-5.2310202E-2,-2.3827499E-1,1.00044586E-1,-1.7792438E-1,-6.822036E-2,5.8059907E-3,-4.3651782E-2,5.759107E-2,5.6174997E-2,-1.1122874E-2,2.4606022E-1,9.26631E-2,-1.4309508E-1,7.959666E-2,1.5598474E-1,2.763589E-1,-1.4606065E-1,-2.6665E-1,-4.017186E-2,-1.4801377E-1,-3.0066106E-1,-5.718792E-1,-1.4680745E-1,-1.8586334E-2,-1.02235906E-1,-3.047485E-2,-1.762092E-2,-5.4168575E-3,6.6745624E-2,2.1168536E-1,-1.0328434E-1,-1.755487E-2,-2.7493842E-2,-9.079568E-2,2.2101229E-2,-1.1511452E-1,-2.885281E-2,-1.8308707E-2,1.3069095E-1,3.0660184E-2,-7.0027724E-2,6.564176E-2,2.0007648E-1,2.0077918E-2,1.134383E-1,-2.0977588E-2,-2.6744036E-2,-1.7585667E-2,1.0934311E-1,-5.9241064E-2,1.6507229E-1,-5.8431597E-3,3.1096238E-1,4.9504344E-3,-2.3789622E-2,-1.2958077E-1,-3.2861564E-1,-1.6614929E-1,1.02384835E-1,-6.276892E-2,-1.8491095E-1,-4.7949213E-4,-1.8957127E-2,-1.8398708E-1,-3.574384E-2,-1.5690975E-2,-6.87514E-2,-2.254838E-1,-1.258578E-1,-3.6225423E-2,-5.01213E-3,-7.389761E-2,1.0466079E-1,2.5341764E-2,2.6109257E-1,3.8769164E-3,-8.427307E-3,-6.9034024E-4,-9.157931E-2,5.1518675E-2,-1.0825119E-1,-7.008822E-3,1.3822403E-2,1.4364395E-2,-1.4325108E-1,3.6169015E-4,4.705105E-2,-7.004872E-2,2.4649294E-1,4.1230727E-2,1.7977645E-1,1.0345965E-2,4.741356E-3,-1.452255E-1,5.1099338E-2,1.4900653E-1,1.0928166E-3,2.165277E-1,1.8092838E-1,8.497257E-2,-5.8356365E-3,5.061165E-2,1.2300824E-3,-5.1752767E-3,1.400063E-1,-5.184085E-2,-8.040712E-3,2.5434236E-3,1.18995264E-1,2.2825436E-1,1.1548419E-2,1.9054646E-2,-1.7618108E-1,-5.104741E-2,-2.1621038E-1,-4.2169058E-1,-1.5485815E-2,-9.71749E-2,-4.4093546E-4,9.649428E-3,6.542449E-2,-8.9272514E-2,-2.8037669E-3,-2.2074477E-1,-4.4629006E-3,-1.3030961E-2,-1.1140695E-1,-3.4167443E-4,-1.6000831E-2,-5.124183E-3,-7.280943E-2,-1.5832198E-1,8.530843E-4,-4.2146314E-3,1.4883358E-2,-9.57821E-2,-1.0670194E-1,-2.604507E-3,7.843801E-3,7.227129E-2,-1.1088113E-2,9.931033E-2,6.5645026E-3,1.6737727E-2,-1.1256441E-1,1.1629614E-2,1.662143E-1,-1.2577624E-3,-1.7100434E-1,-8.4477395E-2,-4.2084333E-2,6.681648E-2,5.0167598E-2,-2.2663835E-2,-9.365676E-2,-1.02172345E-2,6.986802E-3,2.3133572E-2,-5.041639E-2,-1.3683787E-1,1.895407E-1,1.6112076E-2,-1.1277875E-2,9.167779E-2,1.2367697E-1,1.7277436E-2,-8.812809E-3,9.220955E-2,-1.6331673E-2,-5.078507E-2,4.553605E-2,1.2431241E-2,-2.9555534E-3,1.6976003E-1,2.3195496E-1,2.796878E-3,2.1547319E-1,-7.077002E-4,1.1766818E-1,2.4029383E-2,5.0088973E-3,-1.4488864E-3,2.4266577E-1,1.00152805E-1,-4.521727E-3,2.4544378E-4,7.685659E-2,1.6243814E-1,4.1081575E-3,2.4783845E-1,-3.1110054E-1,-1.5300679E-1,6.700646E-3,-1.8038544E-1,-2.6006275E-1,-4.6204454E-3,-1.2692009E-2,-2.4486724E-2,8.445229E-5,-1.6654946E-1,-3.2969855E-4,6.9337254E-3,-1.3847245E-1,-3.8402893E-2,-5.669094E-3,-2.697873E-1,-2.7405624E-3,-6.993921E-3,-8.3609036E-4,-6.3205566E-3,-1.1766215E-2,-5.226315E-3,2.9520363E-2,-5.3935204E-2,-9.565572E-4,-7.384515E-3,-1.414755E-1,3.9444808E-4,1.4351129E-3,-2.9215636E-3,4.618094E-2,5.092962E-3,-4.406619E-2,3.056122E-3,8.334953E-3,-7.753147E-5,-6.485108E-3,7.9699635E-4,-2.0204838E-3,4.7360184E-3,-2.3694544E-3,1.1192881E-2,2.0780785E-3,-3.9696284E-3,-1.21921655E-2,-4.409681E-3,5.503248E-3,-4.708003E-3,-2.7326734E-3,3.3566263E-3,5.3154742E-3,-4.0505063E-3,-3.6268642E-3,3.4047277E-3,-8.583869E-3,-4.64921E-4,-1.6249577E-3,-6.1962535E-3,-3.3174627E-3,2.2728588E-3,-3.6366156E-3,1.628789E-3,-8.805256E-3,-2.0899163E-3,1.0847246E-2,4.680363E-3,-5.4236823E-3,1.4181158E-3,7.500647E-3,3.3820902E-3,1.029034E-4,1.0037441E-2,-1.7202833E-3,2.0178005E-3,-3.363136E-3,6.2695513E-3,1.5678355E-3,-1.1061921E-2,3.809483E-3,1.0752855E-3,6.6046533E-3,1.2798846E-2,7.359069E-3,1.3495746E-2,1.1818445E-2,4.6157027E-3,8.527725E-3,2.5729532E-3,-5.8420347E-3,2.8203274E-3,5.240332E-3,1.57732E-2,3.211111E-3,8.950834E-3,5.3424127E-3,2.3670865E-3,1.2187785E-2,6.2271818E-3,1.4127582E-2,1.0051091E-2,-8.695978E-3,-1.9284405E-2,-3.995802E-3,-9.370063E-3,-7.36339E-3,2.9978866E-3,-3.3972844E-6,-1.3017264E-2,-7.7681127E-3,-1.561785E-2,-3.9279745E-3,-1.1930274E-2,-3.3197172E-3,-8.617314E-3,-3.2807488E-3,-1.1114745E-5,-1.5373164E-2,-7.377276E-3,-2.677164E-4,3.2294246E-3,-3.1644313E-4,-4.2279013E-3,-2.044316E-3,-8.197475E-3,3.9300933E-3,3.5527788E-4,3.253925E-4,-4.453844E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,93,95,-1,-1,97,99,101,-1,103,105,107,109,111,-1,113,115,117,119,121,-1,123,125,127,-1,129,131,133,-1,135,-1,-1,137,139,141,143,145,147,-1,-1,149,-1,-1,151,153,155,157,159,161,163,165,167,-1,-1,-1,169,171,173,175,177,-1,179,-1,181,183,185,187,189,191,-1,193,195,197,-1,199,201,203,-1,205,-1,-1,207,209,-1,-1,211,213,-1,-1,215,217,219,221,-1,223,-1,-1,225,227,-1,229,-1,-1,231,-1,-1,-1,233,235,-1,-1,237,239,241,243,-1,245,247,249,-1,-1,251,253,255,257,259,261,263,265,267,269,271,-1,-1,273,275,277,279,-1,281,283,285,-1,287,289,-1,291,293,-1,-1,295,297,-1,299,-1,301,303,-1,-1,305,307,-1,-1,309,311,-1,313,315,317,319,321,323,-1,-1,-1,-1,325,-1,-1,327,329,-1,331,-1,-1,-1,-1,-1,-1,333,335,-1,-1,337,-1,-1,-1,339,-1,341,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.860717E0,5.037389E0,3.1461158E0,4.125348E0,2.3438835E0,9.527855E-1,7.9344046E-1,1.3234783E0,1.051933E0,0E0,1.2422566E0,6.923232E-1,5.9049845E-1,4.3706346E-1,1.1222624E0,8.8372374E-1,8.1151533E-1,5.3250253E-1,8.651159E-1,6.7700493E-1,4.1486692E-1,5.033288E-1,2.4583036E-1,3.0318975E-1,2.5311565E-1,1.5604565E-1,1.6972268E-1,2.684027E-1,1.9668919E-1,5.197308E-1,4.4786105E-1,5.4351515E-1,4.881541E-1,5.099684E-1,0E0,3.2787848E-1,2.6444066E-1,4.0369174E-1,2.9557264E-1,3.590951E-1,2.1457744E-1,5.5293155E-1,2.879181E-1,2.0937246E-1,1.6676366E-1,1.13921285E-1,1.08312845E-1,1.7487377E-1,0E0,6.40496E-2,1.1351275E-1,0E0,0E0,9.153777E-2,9.2371464E-2,7.955474E-2,0E0,1.037683E0,5.330694E-1,4.354305E-1,9.398705E-2,3.5351187E-1,0E0,6.839348E-1,5.5550927E-1,3.858247E-1,4.700421E-1,1.3012254E-1,0E0,1.6521454E-1,1.293762E-1,4.9903937E-2,0E0,2.9391813E-1,1.53784E-1,3.6003375E-1,0E0,9.730601E-2,0E0,0E0,3.899113E-1,2.54076E-1,1.9796735E-1,9.251694E-2,1.927078E-1,1.07612014E-1,0E0,0E0,4.941848E-2,0E0,0E0,4.413431E-2,1.4186978E-1,4.35822E-2,3.4783877E-2,1.206158E-1,8.859357E-2,4.1187882E-2,8.194114E-2,7.099968E-2,0E0,0E0,0E0,2.4605006E-1,5.5710596E-1,4.3610168E-1,1.6853738E-1,2.5972638E-1,0E0,4.7655433E-2,0E0,8.552609E-2,8.8820666E-2,6.434405E-2,1.04341224E-1,1.6211373E-1,2.5646275E-1,0E0,3.4361017E-1,3.5930818E-1,2.6060057E-1,0E0,9.1046095E-2,1.8501598E-1,1.326676E-1,0E0,4.637032E-2,0E0,0E0,1.761719E-1,2.2381838E-2,0E0,0E0,1.3213503E-1,1.369226E-1,0E0,0E0,1.6501236E-1,3.149075E-1,6.819868E-2,7.04155E-2,0E0,1.1687976E-1,0E0,0E0,5.3739022E-2,1.0981023E-1,0E0,7.4623466E-2,0E0,0E0,8.478843E-3,0E0,0E0,0E0,4.003068E-2,5.4133266E-2,0E0,0E0,5.6993566E-2,4.4999175E-2,1.04724616E-1,2.4429936E-2,0E0,1.15412995E-2,5.273214E-2,6.8340614E-2,0E0,0E0,2.1014655E-1,9.0423E-2,3.68672E-1,2.1771152E-1,4.7382998E-1,4.2953134E-1,6.427549E-2,1.3148916E-1,2.2115049E-1,1.9364043E-1,2.1944664E-2,0E0,0E0,6.906239E-2,1.1005296E-1,5.250141E-2,2.9847026E-2,0E0,8.023203E-2,1.7877534E-2,1.6859862E-1,0E0,1.689954E-1,1.7294249E-1,0E0,2.1720205E-1,2.409631E-1,0E0,0E0,1.3819861E-1,8.294225E-2,0E0,3.7757993E-2,0E0,1.455316E-1,1.208194E-1,0E0,0E0,9.7670734E-2,9.336549E-2,0E0,0E0,3.2161757E-2,8.667469E-2,0E0,2.1331787E-2,2.51593E-2,1.3627648E-1,2.551808E-1,1.8782538E-1,1.730603E-2,0E0,0E0,0E0,0E0,4.4734538E-2,0E0,0E0,4.4853777E-2,2.4960779E-2,0E0,1.4724612E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.6678727E-2,1.4068093E-2,0E0,0E0,3.870833E-2,0E0,0E0,0E0,1.5934562E-2,0E0,3.4774072E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,73,73,74,74,75,75,77,77,80,80,81,81,82,82,83,83,84,84,85,85,88,88,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,103,103,104,104,105,105,106,106,107,107,109,109,111,111,112,112,113,113,114,114,115,115,116,116,118,118,119,119,120,120,122,122,123,123,124,124,126,126,129,129,130,130,133,133,134,134,137,137,138,138,139,139,140,140,142,142,145,145,146,146,148,148,151,151,155,155,156,156,159,159,160,160,161,161,162,162,164,164,165,165,166,166,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,182,182,183,183,184,184,185,185,187,187,188,188,189,189,191,191,192,192,194,194,195,195,198,198,199,199,201,201,203,203,204,204,207,207,208,208,211,211,212,212,214,214,215,215,216,216,217,217,218,218,219,219,224,224,227,227,228,228,230,230,237,237,238,238,241,241,245,245,247,247],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,94,96,-1,-1,98,100,102,-1,104,106,108,110,112,-1,114,116,118,120,122,-1,124,126,128,-1,130,132,134,-1,136,-1,-1,138,140,142,144,146,148,-1,-1,150,-1,-1,152,154,156,158,160,162,164,166,168,-1,-1,-1,170,172,174,176,178,-1,180,-1,182,184,186,188,190,192,-1,194,196,198,-1,200,202,204,-1,206,-1,-1,208,210,-1,-1,212,214,-1,-1,216,218,220,222,-1,224,-1,-1,226,228,-1,230,-1,-1,232,-1,-1,-1,234,236,-1,-1,238,240,242,244,-1,246,248,250,-1,-1,252,254,256,258,260,262,264,266,268,270,272,-1,-1,274,276,278,280,-1,282,284,286,-1,288,290,-1,292,294,-1,-1,296,298,-1,300,-1,302,304,-1,-1,306,308,-1,-1,310,312,-1,314,316,318,320,322,324,-1,-1,-1,-1,326,-1,-1,328,330,-1,332,-1,-1,-1,-1,-1,-1,334,336,-1,-1,338,-1,-1,-1,340,-1,342,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.2679E4,1E0,6.285983E5,7.969174E7,8.387235E11,1.067536E3,1.5107028E-1,3.386E3,2.9511025E-2,1E0,7.5569354E2,3.088934E6,8.672049E9,9.302862E7,4.759E3,2.9211267E2,1.4955851E7,1.0993947E1,3.0972284E1,6.357613E-1,1.25218E5,7.76264E7,9.9925E4,1.067509E6,4.9102457E2,6.654321E0,5.528107E3,1.4299594E8,1.1795571E6,5.9767612E7,4.5184364E0,1.4757962E1,1.1317E4,-1.1122874E-2,6.852143E1,2.2075728E7,3.9712732E7,2.3580047E10,4.42384E5,4.728721E0,2.8972292E-5,1.7314286E0,1E0,3.960925E3,2.264894E8,1.0443182E3,5.3474556E5,-1.8586334E-2,4.6187375E4,4.1026765E-1,-1.762092E-2,-5.4168575E-3,2.467284E7,2.7E1,6.08767E3,-1.755487E-2,2.53E2,2.9076396E9,1E0,1.8019372E5,1.6254681E0,-1.8308707E-2,1.2616388E-1,2.3432E4,6.62E2,4.5723195E6,8.695652E-1,2.0077918E-2,4.0208E4,3.299862E1,3.0161016E7,-1.7585667E-2,3.5296965E-1,1.3554651E2,4.5575E4,-5.8431597E-3,1.8008E4,4.9504344E-3,-2.3789622E-2,2.4973914E2,1.8255814E0,1.494E3,2.386324E6,1.0931163E9,2.2222222E-1,-4.7949213E-4,-1.8957127E-2,7.940488E2,-3.574384E-2,-1.5690975E-2,3.9516E4,8.2E1,9.232009E2,7.9240966E0,4.52737E5,4.576E3,7.576E3,2.3292E4,7.64E2,3.8769164E-3,-8.427307E-3,-6.9034024E-4,2E1,3.9E2,1E0,3.9794872E0,8E-3,1.4364395E-2,1.1855755E8,3.6169015E-4,9E0,3.2982758E1,4.433625E2,2.345785E5,5.3296334E-1,1.6259277E7,4.741356E-3,1.8122449E2,3.5330936E1,1.1860938E8,1.0928166E-3,8.962557E8,1E0,1.849676E0,-5.8356365E-3,9.8376294E5,1.2300824E-3,-5.1752767E-3,9.9286984E4,6.082581E3,-8.040712E-3,2.5434236E-3,1.7231706E-1,1.16544716E-1,1.1548419E-2,1.9054646E-2,4.26E2,1.2471935E7,6.99143E5,2.3428571E0,-1.5485815E-2,2.7524946E0,-4.4093546E-4,9.649428E-3,2.2783158E5,2.4081633E0,-2.8037669E-3,7.43361E0,-4.4629006E-3,-1.3030961E-2,6.1E1,-3.4167443E-4,-1.6000831E-2,-5.124183E-3,2.0992E4,1.8631586E4,8.530843E-4,-4.2146314E-3,1.1095187E1,1.57383175E1,6.0809356E5,4.9978513E-1,7.843801E-3,2.3033286E5,7.8599895E9,2.5727873E5,6.5645026E-3,1.6737727E-2,2.862988E5,3.7761906E1,1.5052E4,4.223261E4,3.7990784E7,1.92E2,5.125467E-1,2.797723E-3,2.98E2,4.4195585E0,1.6958537E9,-1.02172345E-2,6.986802E-3,6.1E1,1.14E3,1.7E1,1.563E3,1.6112076E-2,1E0,2.1111E4,1.8796511E5,1.7277436E-2,3.957351E6,1.4105052E0,-1.6331673E-2,8.282879E0,1.4811083E0,1.2431241E-2,-2.9555534E-3,3.001419E2,7.8040985E2,2.796878E-3,1.7560185E0,-7.077002E-4,3.8773947E0,4.70092E1,5.0088973E-3,-1.4488864E-3,3.824611E0,8.61491E3,-4.521727E-3,2.4544378E-4,1.164249E7,1.4122984E5,4.1081575E-3,1E0,2.909019E3,4.5454E4,2.7635896E-1,5.5700205E8,1.6695403E0,-4.6204454E-3,-1.2692009E-2,-2.4486724E-2,8.445229E-5,3.6327022E5,-3.2969855E-4,6.9337254E-3,1.9589581E6,4.650193E-1,-5.669094E-3,4.6E1,-2.7405624E-3,-6.993921E-3,-8.3609036E-4,-6.3205566E-3,-1.1766215E-2,-5.226315E-3,7.697189E2,2.5381777E-3,-9.565572E-4,-7.384515E-3,2.2500739E-1,3.9444808E-4,1.4351129E-3,-2.9215636E-3,1.5485274E6,5.092962E-3,5.7953E-1,3.056122E-3,8.334953E-3,-7.753147E-5,-6.485108E-3,7.9699635E-4,-2.0204838E-3,4.7360184E-3,-2.3694544E-3,1.1192881E-2,2.0780785E-3,-3.9696284E-3,-1.21921655E-2,-4.409681E-3,5.503248E-3,-4.708003E-3,-2.7326734E-3,3.3566263E-3,5.3154742E-3,-4.0505063E-3,-3.6268642E-3,3.4047277E-3,-8.583869E-3,-4.64921E-4,-1.6249577E-3,-6.1962535E-3,-3.3174627E-3,2.2728588E-3,-3.6366156E-3,1.628789E-3,-8.805256E-3,-2.0899163E-3,1.0847246E-2,4.680363E-3,-5.4236823E-3,1.4181158E-3,7.500647E-3,3.3820902E-3,1.029034E-4,1.0037441E-2,-1.7202833E-3,2.0178005E-3,-3.363136E-3,6.2695513E-3,1.5678355E-3,-1.1061921E-2,3.809483E-3,1.0752855E-3,6.6046533E-3,1.2798846E-2,7.359069E-3,1.3495746E-2,1.1818445E-2,4.6157027E-3,8.527725E-3,2.5729532E-3,-5.8420347E-3,2.8203274E-3,5.240332E-3,1.57732E-2,3.211111E-3,8.950834E-3,5.3424127E-3,2.3670865E-3,1.2187785E-2,6.2271818E-3,1.4127582E-2,1.0051091E-2,-8.695978E-3,-1.9284405E-2,-3.995802E-3,-9.370063E-3,-7.36339E-3,2.9978866E-3,-3.3972844E-6,-1.3017264E-2,-7.7681127E-3,-1.561785E-2,-3.9279745E-3,-1.1930274E-2,-3.3197172E-3,-8.617314E-3,-3.2807488E-3,-1.1114745E-5,-1.5373164E-2,-7.377276E-3,-2.677164E-4,3.2294246E-3,-3.1644313E-4,-4.2279013E-3,-2.044316E-3,-8.197475E-3,3.9300933E-3,3.5527788E-4,3.253925E-4,-4.453844E-3],"split_indices":[19,2,59,28,7,31,52,38,2,0,102,4,1,5,45,2,4,47,56,56,38,29,45,11,9,55,54,52,45,45,45,53,58,9,0,58,45,45,5,2,38,42,53,64,4,7,4,32,0,33,57,0,0,45,8,4,0,2,5,63,33,53,0,39,9,2,28,53,0,9,56,45,0,27,56,2,0,2,0,0,55,54,11,1,5,58,0,0,4,0,0,10,8,33,54,29,2,9,2,0,0,0,0,3,12,102,54,57,0,45,0,3,58,4,28,38,9,0,52,56,5,0,12,6,42,0,28,0,0,33,52,0,0,38,39,0,0,2,45,1,56,0,57,0,0,28,53,0,56,0,0,8,0,0,0,29,33,0,0,54,58,28,27,0,33,5,33,0,0,28,52,9,50,7,12,57,39,0,58,5,0,0,10,10,3,2,0,100,9,33,0,45,53,0,53,53,0,0,4,4,0,39,0,54,58,0,0,54,4,0,0,45,33,0,64,32,1,27,5,54,0,0,0,0,32,0,0,32,27,0,8,0,0,0,0,0,0,4,38,0,0,27,0,0,0,47,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.594E3,2.021E3,5.73E2,1.75E3,2.71E2,3.23E2,2.5E2,1.156E3,5.94E2,1.1E1,2.6E2,2.6E2,6.3E1,1.58E2,9.2E1,7.92E2,3.64E2,4.28E2,1.66E2,8.5E1,1.75E2,1.66E2,9.4E1,2.7E1,3.6E1,1.45E2,1.3E1,7.5E1,1.7E1,5.67E2,2.25E2,1.15E2,2.49E2,4.22E2,6E0,5.6E1,1.1E2,1.6E1,6.9E1,1.36E2,3.9E1,1.12E2,5.4E1,6.2E1,3.2E1,1.9E1,8E0,3E1,6E0,4.3E1,1.02E2,6E0,7E0,5.9E1,1.6E1,1.3E1,4E0,2.03E2,3.64E2,1.99E2,2.6E1,1.11E2,4E0,6.6E1,1.83E2,2.9E1,3.93E2,4.5E1,1.1E1,9.3E1,1.7E1,1.1E1,5E0,5.7E1,1.2E1,1.32E2,4E0,3.2E1,7E0,4E0,1.08E2,3.2E1,2.2E1,8E0,5.4E1,2.5E1,7E0,1E1,9E0,4E0,4E0,1.6E1,1.4E1,3.1E1,1.2E1,6.5E1,3.7E1,3E1,2.9E1,1.1E1,5E0,7E0,6E0,1.12E2,9.1E1,3.01E2,6.3E1,1.94E2,5E0,2.1E1,5E0,3.9E1,7.2E1,2.8E1,3.8E1,2.1E1,1.62E2,9E0,2E1,3.36E2,5.7E1,4E0,4.1E1,2.6E1,6.7E1,7E0,1E1,7E0,4E0,4.8E1,9E0,6E0,6E0,7.8E1,5.4E1,1.7E1,1.5E1,6.7E1,4.1E1,1.6E1,1.6E1,6E0,1.6E1,4E0,4E0,9E0,4.5E1,6E0,1.9E1,5E0,4E0,9E0,7E0,7E0,7E0,1.3E1,1.8E1,6E0,6E0,5.4E1,1.1E1,2.5E1,1.2E1,1E1,2E1,2E1,9E0,5E0,6E0,9.3E1,1.9E1,2.8E1,6.3E1,8.1E1,2.2E2,4.3E1,2E1,9.7E1,9.7E1,1.3E1,8E0,7E0,3.2E1,5.7E1,1.5E1,1.8E1,1E1,1.9E1,1.9E1,1.7E1,4E0,1.32E2,3E1,6E0,1.4E1,3.28E2,8E0,5E0,5.2E1,3.7E1,4E0,2.2E1,4E0,4.3E1,2.4E1,6E0,4E0,1.2E1,3.6E1,5E0,4E0,4.1E1,3.7E1,7E0,4.7E1,8E0,5.9E1,2.9E1,1.2E1,1.1E1,5E0,6E0,1E1,7E0,9E0,5E0,4E0,2.2E1,2.3E1,7E0,1.2E1,4E0,5E0,7E0,6E0,6E0,1.2E1,4.5E1,9E0,5E0,6E0,1.9E1,6E0,8E0,4E0,1.2E1,8E0,1.4E1,6E0,5E0,4E0,8.2E1,1.1E1,1.2E1,7E0,6E0,2.2E1,4.1E1,2.2E1,4.2E1,3.9E1,1E1,2.1E2,3.9E1,4E0,1.6E1,4E0,1.2E1,8.5E1,7E0,9E1,5E0,8E0,6E0,2.6E1,4.5E1,1.2E1,1E1,5E0,1.3E1,5E0,5E0,1.4E1,4E0,1.5E1,7E0,1E1,8.7E1,4.5E1,5E0,2.5E1,1E1,4E0,1.43E2,1.85E2,3.8E1,1.4E1,1.3E1,2.4E1,1.8E1,4E0,2.3E1,2E1,4E0,2E1,5E0,7E0,2.6E1,1E1,1.9E1,2.2E1,1E1,2.7E1,2.4E1,2.3E1,4E0,4E0,2E1,3.9E1,7E0,2.2E1,4E0,8E0,5E0,6E0,5E0,4E0,8E0,1.4E1,1.3E1,1E1,8E0,4E0,2.3E1,2.2E1,4E0,5E0,4E0,1.5E1,6E0,6E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"343","size_leaf_vector":"1"}},{"base_weights":[-4.2901626E-3,-8.9159176E-2,3.4841765E-2,-1.1743877E-1,-1.1694784E-2,1.5629824E-2,1.5406497E-1,-1.2518984E-1,2.1261911E-1,-7.22E-2,3.924821E-2,-3.7074126E-2,4.9250133E-2,1.8783782E-1,2.5752913E-2,-9.28829E-2,-1.8140163E-1,2.4116084E-2,8.978569E-3,-1.16740195E-2,-5.7536677E-2,1.972874E-1,2.6080636E-2,-5.2977428E-2,6.0900014E-2,1.73299E-2,9.249735E-2,4.1166973E-1,1.6938512E-1,-1.6344557E-2,6.1972864E-2,-1.1876791E-1,-4.9324296E-2,-2.9055116E-1,-1.1907245E-1,6.4683305E-3,-5.6607984E-3,1.6983012E-2,-8.827932E-2,4.1380473E-3,1.360598E-2,-3.9764054E-2,4.8772387E-2,-4.589048E-2,-2.6847583E-1,2.2001803E-1,3.8574606E-2,-5.0615068E-3,9.4622426E-2,7.792877E-2,2.0628154E-1,3.242272E-2,2.5189412E-1,1.5616618E-1,3.5827932E-1,1.3579995E-2,1.9523142E-2,-3.038131E-1,-1.10802844E-1,-3.01743E-2,-1.7744236E-1,-2.2539026E-1,-4.7417122E-1,-6.5091595E-2,-1.9274607E-1,-4.0441383E-2,1.0940629E-1,-3.238135E-2,-1.3396148E-1,-6.4252466E-2,3.2386854E-3,-5.368376E-3,6.234003E-2,1.23745136E-1,-5.0732493E-2,-1.3206588E-1,-2.7090156E-2,1.372378E-2,5.5186003E-3,1.8958835E-2,9.895143E-3,7.861827E-3,-1.2985727E-1,4.7058806E-2,1.8512163E-1,3.3015575E-2,1.1471663E-1,1.04181044E-1,2.554997E-1,5.5806194E-3,1.6326666E-2,2.181168E-1,1.2464095E-1,2.9909262E-3,2.5995374E-2,1.3031648E-2,-1.3610413E-2,-2.2275181E-2,-5.04977E-3,-8.9679666E-2,-1.8441257E-1,4.3930183E-3,-8.246766E-2,-9.5280826E-2,-1.858791E-2,-2.9467043E-1,-1.07152134E-1,-5.406195E-1,-8.993605E-3,-9.7840495E-2,-2.8154526E-2,-2.5117335E-1,-1.2767237E-1,1.2787515E-3,-6.1136097E-2,7.889382E-3,2.9552053E-3,-8.3974876E-2,4.5515455E-2,-1.9524123E-1,-6.885638E-2,-1.0332081E-1,7.75297E-4,2.5902333E-2,1.1013871E-1,1.397177E-2,3.3432283E-2,-5.973533E-2,5.00172E-2,-1.1252599E-2,-6.975181E-5,-1.7305968E-2,7.332543E-2,-2.9636335E-2,4.1911356E-2,-1.4903525E-1,1.5169831E-3,1.014669E-2,2.6903918E-2,2.2075164E-1,9.1399156E-2,5.0377797E-2,-8.347796E-2,1.7053859E-1,7.0645414E-2,7.0255073E-3,1.0014488E-3,1.4458239E-2,5.3840093E-3,2.3749124E-1,-1.3491362E-3,6.1894078E-2,1.6709991E-1,-8.709573E-3,1.1150092E-2,-1.15675755E-1,-3.0385058E-2,-6.8645254E-2,-2.7619907E-1,-3.5525233E-2,9.069952E-2,-1.3771469E-3,-1.0873007E-1,-1.5497604E-1,5.0676794E-4,-3.2536012E-1,-7.324439E-3,-9.750885E-3,-2.3279867E-1,-3.0967694E-2,-1.2909938E-2,-4.1193888E-2,-1.642446E-1,-8.58146E-3,-6.7143277E-3,-2.865422E-1,-6.1215325E-3,1.4031333E-3,-1.6129114E-1,-1.1588824E-3,-4.7850967E-3,-7.013164E-3,-5.7330197E-3,-1.9867476E-3,5.1857256E-3,-1.1069132E-2,-5.2835424E-3,7.8051357E-4,-1.0904739E-1,-1.3971417E-1,-1.2714053E-3,2.5179093E-3,-2.0336928E-3,4.2749453E-2,-3.733754E-2,9.87168E-4,1.2192395E-1,-1.4049334E-3,3.9549437E-3,-9.5987365E-2,-3.8826425E-2,1.1314868E-2,1.83428E-2,-5.442601E-2,1.7226161E-2,5.7292744E-3,1.998905E-2,-1.340252E-2,-2.073148E-1,2.9334934E-2,2.3440133E-1,-8.782061E-2,-2.1623883E-1,-1.1640817E-2,8.7889954E-2,1.2825766E-1,1.349851E-2,6.3398504E-4,7.847421E-3,5.947237E-3,1.2934525E-1,1.1215812E-3,-1.179823E-1,2.0251219E-1,9.142394E-2,8.666428E-2,-1.0932166E-2,6.4980653E-3,2.56955E-1,1.1082386E-1,1.6229497E-2,1.8500344E-1,4.4759057E-2,7.960285E-3,-2.0997087E-2,1.6946039E-4,-6.8112635E-3,-4.6052677E-3,3.7735462E-4,-9.257617E-4,-1.0370951E-2,-2.2198664E-2,-6.639763E-3,-4.8934424E-4,-6.5228706E-3,2.0798563E-3,1.1541599E-2,-3.504332E-3,3.3765393E-3,-1.1126877E-2,-4.195477E-3,-3.8587055E-3,-1.0079839E-2,-8.341088E-3,-1.7558798E-2,2.374721E-3,-4.6989266E-3,-1.5469798E-2,-4.70059E-3,5.920587E-4,-4.7580395E-3,-3.3412944E-3,-1.0766501E-2,-3.7873844E-3,1.1284363E-3,-1.753508E-2,-6.495012E-3,-1.0570254E-2,-4.5082258E-4,1.1336352E-3,-1.649263E-3,-1.820559E-3,-7.488774E-3,-3.267701E-3,-8.912001E-3,4.1357065E-3,5.1078165E-4,-3.4981202E-3,-1.9428734E-4,7.5988034E-3,4.1793217E-3,-5.5115116E-3,5.2257505E-4,5.80754E-3,-2.4589559E-3,-1.4918253E-3,3.869688E-3,-4.748311E-3,-6.278746E-4,-4.275485E-3,2.239643E-3,-2.7392895E-3,2.9645187E-3,2.5012707E-3,-2.6838172E-3,-3.8301589E-3,-1.7273003E-2,3.6454718E-3,-1.1927626E-3,4.520516E-3,1.4743056E-2,-6.665441E-3,2.0365932E-3,-1.7654616E-2,-7.050221E-3,-5.3614527E-3,1.1224428E-3,7.573651E-3,-2.0935473E-3,9.6983823E-4,9.006763E-3,1.3458681E-3,-1.460901E-2,7.603888E-3,-5.3262594E-3,-8.204486E-3,-2.5603168E-3,1.1954438E-2,6.9438717E-3,3.111343E-3,1.0064471E-2,7.5943703E-3,1.284279E-3,1.4847925E-2,1.0027746E-2,7.0435833E-3,1.8331443E-3,-1.6651131E-3,2.4625328E-3,2.3041815E-3,9.706826E-3,-1.1548992E-3,4.689851E-3,-6.255003E-3,9.83986E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,-1,37,39,41,43,45,47,49,51,53,-1,55,57,59,61,63,-1,-1,65,67,-1,-1,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,-1,95,97,99,101,103,105,107,109,111,113,115,117,119,121,-1,-1,123,125,127,129,-1,-1,-1,131,-1,133,135,137,139,141,143,145,147,-1,-1,149,151,-1,-1,-1,153,-1,-1,155,157,159,161,163,-1,165,167,169,-1,171,173,175,177,-1,179,-1,-1,181,183,185,187,189,191,193,195,-1,197,199,201,-1,-1,203,205,207,209,211,-1,-1,213,215,217,219,221,223,225,-1,-1,-1,-1,227,-1,229,231,-1,233,235,237,239,241,243,245,247,249,251,-1,253,-1,255,257,-1,-1,259,261,263,-1,265,-1,-1,267,-1,-1,-1,269,-1,-1,-1,-1,-1,271,273,-1,-1,-1,275,277,-1,279,-1,-1,281,283,-1,285,287,289,-1,291,293,295,297,299,301,303,305,307,309,-1,-1,-1,311,313,-1,315,317,319,321,-1,-1,323,325,327,329,331,-1,333,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.534982E0,1.7752528E0,4.026597E0,1.5462484E0,6.7477876E-1,2.6880612E0,1.0525284E0,1.0382652E0,7.654971E-1,2.2327644E-1,2.4249278E-1,9.240707E-1,1.2767041E0,7.2741747E-1,6.8486553E-1,4.1245008E-1,1.4008789E0,0E0,1.4723957E-1,0E0,2.1455595E-1,5.473131E-2,1.6778797E-1,7.650373E-1,2.8636E-1,9.2466646E-1,6.353903E-1,3.6717057E-1,3.9350605E-1,0E0,4.221964E-1,3.082807E-1,3.3813024E-1,8.0245495E-1,5.2832127E-1,0E0,0E0,1.5303037E-1,1.6419584E-1,0E0,0E0,7.843428E-2,1.8260978E-1,4.1203856E-1,5.257404E-1,2.9536247E-2,2.2771277E-1,6.7032355E-1,5.078248E-1,5.7450366E-1,1.9449055E-1,0E0,6.846118E-2,3.0482483E-1,5.005624E-1,0E0,3.3132774E-1,2.1403861E-1,3.3175302E-1,2.2270216E-1,2.522968E-1,4.435358E-1,2.7409935E-1,9.401721E-2,1.8465042E-1,2.6162386E-2,1.7469436E-2,1.2841025E-1,1.3018286E-1,6.105274E-2,0E0,0E0,1.3125604E-1,1.8773896E-1,4.3927073E-1,1.4572686E-1,0E0,0E0,0E0,1.3336577E-1,0E0,4.8259377E-1,1.2435758E-1,2.4608639E-1,1.16736054E-1,3.2579666E-1,4.6029925E-1,4.6215087E-2,1.1389184E-1,0E0,0E0,2.7235723E-1,2.9603672E-1,0E0,0E0,0E0,1.4916453E-1,0E0,0E0,2.677765E-1,5.0170803E-1,2.6161245E-1,1.0420224E-1,8.9677215E-2,0E0,1.1591411E-1,2.7920696E-1,1.7495918E-1,0E0,1.5122005E-1,7.993108E-2,9.6862316E-2,1.5638047E-1,0E0,1.617425E-2,0E0,0E0,8.3012536E-2,6.872277E-2,1.89268E-2,6.621255E-2,3.8957343E-2,2.2630071E-2,4.9539607E-2,3.2742023E-2,0E0,3.0365665E-2,3.313049E-1,2.1938029E-1,0E0,0E0,5.349275E-2,5.723597E-2,5.1641333E-1,4.7244924E-1,1.2499428E-1,0E0,0E0,1.7027283E-1,9.7854495E-2,6.518726E-2,4.8692173E-1,7.9400286E-2,1.9397044E-1,5.243303E-1,0E0,0E0,0E0,0E0,7.532859E-2,0E0,1.05450556E-1,1.4066207E-1,0E0,1.6052358E-1,3.0028892E-1,1.2960646E-1,1.5627939E-1,5.9131384E-1,1.2586239E-1,1.5561344E-1,6.637165E-2,8.571854E-2,1.7138347E-2,0E0,6.1022043E-2,0E0,7.205997E-2,7.741958E-2,0E0,0E0,6.9910675E-2,8.0370426E-2,7.293592E-2,0E0,1.761676E-1,0E0,0E0,1.8174553E-1,0E0,0E0,0E0,7.725278E-3,0E0,0E0,0E0,0E0,0E0,3.3079028E-2,1.7055199E-2,0E0,0E0,0E0,4.601603E-2,1.0656149E-2,0E0,2.075544E-2,0E0,0E0,2.4650502E-1,4.5983034E-1,0E0,1.0201667E-1,3.2799277E-2,6.624343E-2,0E0,4.15135E-2,4.2733365E-1,2.4470502E-1,4.3577182E-1,7.8069925E-2,1.2264304E-1,1.1496067E-1,1.4922032E-1,2.3778443E-1,6.441951E-2,0E0,0E0,0E0,5.7138443E-1,2.8559566E-1,0E0,4.0939912E-2,1.04474545E-1,7.0842445E-2,4.102506E-1,0E0,0E0,3.7629604E-2,4.6224296E-2,4.4229254E-2,6.694341E-2,3.5495155E-2,0E0,1.2116673E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,37,37,38,38,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,72,72,73,73,74,74,75,75,79,79,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,96,96,99,99,100,100,101,101,102,102,103,103,105,105,106,106,107,107,109,109,110,110,111,111,112,112,114,114,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,126,126,127,127,128,128,131,131,132,132,133,133,134,134,135,135,138,138,139,139,140,140,141,141,142,142,143,143,144,144,149,149,151,151,152,152,154,154,155,155,156,156,157,157,158,158,159,159,160,160,161,161,162,162,163,163,165,165,167,167,168,168,171,171,172,172,173,173,175,175,178,178,182,182,188,188,189,189,193,193,194,194,196,196,199,199,200,200,202,202,203,203,204,204,206,206,207,207,208,208,209,209,210,210,211,211,212,212,213,213,214,214,215,215,219,219,220,220,222,222,223,223,224,224,225,225,228,228,229,229,230,230,231,231,232,232,234,234],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,-1,38,40,42,44,46,48,50,52,54,-1,56,58,60,62,64,-1,-1,66,68,-1,-1,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,-1,96,98,100,102,104,106,108,110,112,114,116,118,120,122,-1,-1,124,126,128,130,-1,-1,-1,132,-1,134,136,138,140,142,144,146,148,-1,-1,150,152,-1,-1,-1,154,-1,-1,156,158,160,162,164,-1,166,168,170,-1,172,174,176,178,-1,180,-1,-1,182,184,186,188,190,192,194,196,-1,198,200,202,-1,-1,204,206,208,210,212,-1,-1,214,216,218,220,222,224,226,-1,-1,-1,-1,228,-1,230,232,-1,234,236,238,240,242,244,246,248,250,252,-1,254,-1,256,258,-1,-1,260,262,264,-1,266,-1,-1,268,-1,-1,-1,270,-1,-1,-1,-1,-1,272,274,-1,-1,-1,276,278,-1,280,-1,-1,282,284,-1,286,288,290,-1,292,294,296,298,300,302,304,306,308,310,-1,-1,-1,312,314,-1,316,318,320,322,-1,-1,324,326,328,330,332,-1,334,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.451483E-1,1.2679E4,6.082581E3,7.857143E0,1.7669278E5,3.5301748E7,6.748543E-1,1.2003743E7,1.1216729E2,1E0,6.4625664E8,1E0,9.532E3,2.9816154E1,3.051E3,2.1631205E0,2.4116084E-2,3E1,-1.16740195E-2,4.61E2,2.6100147E5,1E0,5.0363636E1,2.808311E0,4.264897E6,4.391553E6,5.473125E3,1.9013363E4,-1.6344557E-2,9.042859E4,8.080045E-6,1.3900395E2,5.642369E11,4.987639E7,6.4683305E-3,-5.6607984E-3,1E0,1E0,4.1380473E-3,1.360598E-2,1.10018056E8,3.7225406E4,7E0,1.1725374E-1,9.7E1,1E0,2.3936112E-1,5.9643E4,7.4709034E-1,5.8084745E6,3.242272E-2,3.8560154E8,4.35581E5,1.9624242E7,1.3579995E-2,2.5927516E11,3.3E1,7.128E3,6.9E1,1.873E3,1.6708623E2,2.9885426E1,3.098804E2,5.378E3,1.1220721E0,8.9385474E-1,4.8632206E5,5.387818E8,3.9E1,3.2386854E-3,-5.368376E-3,3.164136E6,1.3050649E4,1.5106794E3,9.438605E-6,-2.7090156E-2,1.372378E-2,5.5186003E-3,3.94E2,9.895143E-3,2.5275471E2,7.299E3,1.3E1,3.7898205E1,3.881848E-1,1.59551E5,7.074777E6,1.4593E4,5.5806194E-3,1.6326666E-2,9.256843E3,1.4689211E7,2.9909262E-3,2.5995374E-2,1.3031648E-2,1.27365E4,-2.2275181E-2,-5.04977E-3,6.0529634E-2,2.4285715E0,6.624E3,1E1,6.316106E6,-1.858791E-2,9.112E3,7.16173E5,1.8736842E0,-8.993605E-3,9.912942E6,1.5311552E9,1.001194E6,4.1100917E0,1.2787515E-3,1.3984146E2,7.889382E-3,2.9552053E-3,1.9E1,1.5277338E0,7.5569354E2,4.3161097E5,1.27E2,3.0596492E0,2.57937E5,1.4571098E5,1.397177E-2,6.363636E-1,5.0795E4,1.92E2,-1.1252599E-2,-6.975181E-5,4.3563797E4,1.5548611E7,3.5612745E0,2.109E3,2.34E2,1.5169831E-3,1.014669E-2,1.4462532E7,4.561156E7,1.3715873E3,2.539E3,2.028994E8,2.1E1,2.6481817E0,7.0255073E-3,1.0014488E-3,1.4458239E-2,5.3840093E-3,1.8382221E-1,-1.3491362E-3,1.9203481E5,1.600814E6,-8.709573E-3,2.4345527E3,2.9883E4,1.146E3,1.4448833E1,1.4633E4,7.9138456E5,4E1,2.8680122E6,4.709E3,9.64E2,5.0676794E-4,4.5168175E5,-7.324439E-3,1.5316E4,8.890291E2,-3.0967694E-2,-1.2909938E-2,2.1343573E5,1.9598837E2,6.604754E5,-6.7143277E-3,1.8302105E6,-6.1215325E-3,1.4031333E-3,5.266129E0,-1.1588824E-3,-4.7850967E-3,-7.013164E-3,3.131E3,-1.9867476E-3,5.1857256E-3,-1.1069132E-2,-5.2835424E-3,7.8051357E-4,1.5604764E5,1E0,-1.2714053E-3,2.5179093E-3,-2.0336928E-3,1.5211788E-1,2.292691E-2,9.87168E-4,3.3481183E9,-1.4049334E-3,3.9549437E-3,2.2455946E-1,7.330957E4,1.1314868E-2,1.1253046E9,7.804304E-1,1E0,5.7292744E-3,5.936149E1,4.7048388E0,1.7E1,2.44E3,1.366E3,2.36026E5,3E0,1.1E1,3.4446188E2,9E0,1.349851E-2,6.3398504E-4,7.847421E-3,2.1036203E5,8.942177E6,1.1215812E-3,2.376302E8,4.760537E8,4.838871E8,1.5182724E0,-1.0932166E-2,6.4980653E-3,4.7E1,6.332424E1,4.3525005E3,1.8238512E0,6.1707315E0,7.960285E-3,4.77721E3,1.6946039E-4,-6.8112635E-3,-4.6052677E-3,3.7735462E-4,-9.257617E-4,-1.0370951E-2,-2.2198664E-2,-6.639763E-3,-4.8934424E-4,-6.5228706E-3,2.0798563E-3,1.1541599E-2,-3.504332E-3,3.3765393E-3,-1.1126877E-2,-4.195477E-3,-3.8587055E-3,-1.0079839E-2,-8.341088E-3,-1.7558798E-2,2.374721E-3,-4.6989266E-3,-1.5469798E-2,-4.70059E-3,5.920587E-4,-4.7580395E-3,-3.3412944E-3,-1.0766501E-2,-3.7873844E-3,1.1284363E-3,-1.753508E-2,-6.495012E-3,-1.0570254E-2,-4.5082258E-4,1.1336352E-3,-1.649263E-3,-1.820559E-3,-7.488774E-3,-3.267701E-3,-8.912001E-3,4.1357065E-3,5.1078165E-4,-3.4981202E-3,-1.9428734E-4,7.5988034E-3,4.1793217E-3,-5.5115116E-3,5.2257505E-4,5.80754E-3,-2.4589559E-3,-1.4918253E-3,3.869688E-3,-4.748311E-3,-6.278746E-4,-4.275485E-3,2.239643E-3,-2.7392895E-3,2.9645187E-3,2.5012707E-3,-2.6838172E-3,-3.8301589E-3,-1.7273003E-2,3.6454718E-3,-1.1927626E-3,4.520516E-3,1.4743056E-2,-6.665441E-3,2.0365932E-3,-1.7654616E-2,-7.050221E-3,-5.3614527E-3,1.1224428E-3,7.573651E-3,-2.0935473E-3,9.6983823E-4,9.006763E-3,1.3458681E-3,-1.460901E-2,7.603888E-3,-5.3262594E-3,-8.204486E-3,-2.5603168E-3,1.1954438E-2,6.9438717E-3,3.111343E-3,1.0064471E-2,7.5943703E-3,1.284279E-3,1.4847925E-2,1.0027746E-2,7.0435833E-3,1.8331443E-3,-1.6651131E-3,2.4625328E-3,2.3041815E-3,9.706826E-3,-1.1548992E-3,4.689851E-3,-6.255003E-3,9.83986E-4],"split_indices":[20,42,2,52,56,28,45,27,1,4,67,7,100,9,56,2,54,0,3,0,0,28,102,46,58,9,28,32,52,0,28,42,58,31,45,0,0,8,104,0,0,32,33,3,38,3,101,27,1,41,45,0,32,29,32,0,31,3,10,8,0,52,57,52,2,42,56,28,7,3,0,0,1,28,52,37,0,0,0,8,0,52,2,10,58,27,9,47,29,0,0,4,1,0,0,0,33,0,0,57,57,2,3,1,0,10,1,53,0,45,12,29,56,0,52,0,0,8,53,4,32,8,53,29,28,0,53,9,0,0,0,28,45,53,0,10,0,0,5,5,52,2,7,3,41,0,0,0,0,38,0,33,11,0,52,1,2,56,10,47,8,50,2,0,0,28,0,9,4,0,0,47,52,28,0,32,0,0,54,0,0,0,2,0,0,0,0,0,33,16,0,0,0,27,27,0,12,0,0,41,45,0,5,39,64,0,56,58,3,29,10,12,8,3,4,8,0,0,0,33,47,0,7,7,7,53,0,0,8,56,4,53,54,0,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.568E3,8.1E2,1.758E3,5.93E2,2.17E2,1.515E3,2.43E2,5.8E2,1.3E1,9.9E1,1.18E2,5.9E2,9.25E2,1.92E2,5.1E1,3.7E2,2.1E2,5E0,8E0,7E0,9.2E1,8E0,1.1E2,5.08E2,8.2E1,5.33E2,3.92E2,1.3E1,1.79E2,4E0,4.7E1,2.31E2,1.39E2,7.5E1,1.35E2,4E0,4E0,2.7E1,6.5E1,4E0,4E0,2.8E1,8.2E1,4.93E2,1.5E1,9E0,7.3E1,4.14E2,1.19E2,3.49E2,4.3E1,4E0,9E0,1.69E2,1E1,7E0,4E1,8E0,2.23E2,1.22E2,1.7E1,5.7E1,1.8E1,7.9E1,5.6E1,1.7E1,1E1,3E1,3.5E1,2.3E1,5E0,6E0,7.6E1,1.3E1,4.8E2,1.1E1,4E0,5E0,4E0,6.6E1,7E0,3.76E2,3.8E1,7.9E1,4E1,1.58E2,1.91E2,1.5E1,2.8E1,4E0,5E0,5.5E1,1.14E2,4E0,6E0,4E0,3.6E1,4E0,4E0,1.75E2,4.8E1,7.4E1,4.8E1,1.3E1,4E0,3.5E1,2.2E1,1.4E1,4E0,4.1E1,3.8E1,2.8E1,2.8E1,4E0,1.3E1,4E0,6E0,1.8E1,1.2E1,1.7E1,1.8E1,1.4E1,9E0,4.4E1,3.2E1,4E0,9E0,4.41E2,3.9E1,6E0,5E0,4E1,2.6E1,1.79E2,1.97E2,3.4E1,4E0,8E0,7.1E1,2.8E1,1.2E1,1.38E2,2E1,8.3E1,1.08E2,1E1,5E0,2.2E1,6E0,5.1E1,4E0,4.7E1,6.7E1,4E0,3.2E1,1.21E2,5.4E1,2.2E1,2.6E1,5.1E1,2.3E1,1.2E1,3.6E1,8E0,5E0,2.8E1,7E0,1.3E1,9E0,1E1,4E0,2.3E1,1.8E1,3.3E1,5E0,2.1E1,7E0,5E0,2.3E1,7E0,6E0,1E1,8E0,5E0,7E0,1.2E1,5E0,6E0,1.2E1,9E0,5E0,4E0,5E0,3.5E1,9E0,4E0,2.8E1,4E0,5E0,1.6E2,2.81E2,5E0,3.4E1,1.9E1,2.1E1,1.4E1,1.2E1,1.65E2,1.4E1,1.86E2,1.1E1,1.9E1,1.5E1,4.4E1,2.7E1,1.1E1,1.7E1,6E0,6E0,8.9E1,4.9E1,5E0,1.5E1,5.8E1,2.5E1,1.03E2,5E0,9E0,4.2E1,2.2E1,2.5E1,5.8E1,9E0,5E0,2.7E1,1.8E1,1.03E2,2E1,3.4E1,1.7E1,5E0,1.1E1,1.5E1,4.1E1,1E1,1.8E1,5E0,6E0,6E0,5E0,3.1E1,4E0,4E0,5E0,2.3E1,8E0,5E0,5E0,4E0,1.2E1,1.1E1,7E0,1.1E1,1E1,2.3E1,1.4E1,7E0,1.7E1,6E0,4E0,4E0,5E0,7E0,4E0,5E0,1.5E1,2E1,4E0,5E0,1.4E1,1.4E1,1.41E2,1.9E1,1.7E1,2.64E2,1.9E1,1.5E1,9E0,1E1,4E0,1.7E1,4E0,8E0,6.4E1,1.01E2,8E0,6E0,1.02E2,8.4E1,4E0,7E0,1.4E1,5E0,4E0,1.1E1,1.1E1,3.3E1,1.8E1,9E0,4E0,7E0,8.4E1,5E0,4.5E1,4E0,8E0,7E0,3.5E1,2.3E1,2.1E1,4E0,4.9E1,5.4E1,2.2E1,2E1,1.5E1,7E0,1E1,1.5E1,4E0,5.4E1,4E0,5E0,7E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"335","size_leaf_vector":"1"}},{"base_weights":[-1.111591E-3,-7.866804E-2,3.631325E-2,-1.2007409E-1,-2.8624017E-2,-2.8653336E-3,8.323259E-2,-1.2677366E-1,2.0441579E-2,-5.5291433E-2,8.9341134E-2,-3.2040086E-2,5.0803896E-2,1.10082015E-1,-7.021526E-2,-1.1358931E-1,-2.7743566E-1,-2.3505041E-2,-9.6487656E-2,-2.1978926E-2,1.170685E-1,-5.1620644E-2,3.161501E-2,3.4409296E-2,1.199433E-1,1.4953853E-1,3.120551E-2,-1.1747456E-1,6.1942045E-2,-1.3385493E-1,-3.6487732E-2,-1.956252E-1,-2.5039842E-2,-5.7579793E-2,1.53117E-2,-1.7232098E-2,-8.2370915E-2,3.1768582E-3,-5.887106E-2,1.7810522E-1,7.361384E-2,-4.541723E-2,-2.517599E-1,4.5301296E-2,-1.1189089E-1,-4.161771E-2,5.081336E-2,-1.6803935E-3,1.3641006E-1,2.3253186E-2,1.4423428E-1,1.637832E-2,3.5131773E-1,-1.4701255E-1,1.453776E-2,1.20989665E-1,-8.006168E-2,-1.4350927E-1,-2.3566252E-2,-9.7047254E-2,8.468992E-3,-1.65034E-2,-1.1445389E-1,-7.713075E-2,7.167477E-2,-4.7738333E-3,8.922387E-2,-7.0915215E-2,-1.3667539E-2,-4.477821E-3,-2.0254987E-4,2.4713334E-1,8.033514E-2,5.4217793E-2,8.951357E-3,-6.332035E-2,7.0081046E-3,5.998698E-4,-2.584477E-2,5.349921E-2,-5.550051E-3,-4.457528E-2,-1.05270175E-2,5.936464E-2,-9.2377774E-2,6.1708488E-2,-1.0583236E-1,-1.9971575E-3,1.502154E-1,1.6461402E-2,1.3745369E-1,5.1423407E-4,9.931014E-2,2.4787767E-2,5.363031E-3,-1.3015799E-1,-1.7330037E-2,3.9119244E-2,-1.8511734E-3,1.7090179E-2,5.6452855E-2,-1.03947455E-2,1.994929E-3,-1.2982084E-1,-2.1617913E-1,4.1873947E-2,-1.45546E-1,-5.849981E-4,-1.6717504E-1,5.8200117E-2,-5.676591E-2,-1.1239119E-2,-2.0622326E-1,-1.7907591E-1,-5.9068628E-2,8.571409E-3,8.228812E-3,-2.7971093E-2,6.5098755E-2,5.688275E-3,1.0500624E-3,-1.3546748E-2,-1.057564E-1,6.571876E-3,1.6360922E-2,5.2701267E-3,1.4527971E-3,-2.2058245E-2,8.170764E-2,-6.843696E-2,1.1390627E-1,7.8693435E-2,-2.9376974E-2,-7.071219E-2,6.462485E-2,-4.6322397E-3,6.231389E-4,1.12682566E-1,-4.5537073E-3,-3.0542394E-2,-2.3154491E-1,2.3698516E-2,9.913631E-2,3.624761E-2,-1.984885E-2,2.133614E-1,1.0679538E-1,7.377542E-2,1.6039428E-1,5.127947E-2,-2.5704816E-2,2.3071717E-2,1.5533537E-1,-1.87704E-1,-9.450326E-2,3.2615238E-3,-5.2725844E-4,-1.7720445E-3,1.01604305E-1,-1.0657928E-1,-1.7040966E-1,-1.0185466E-2,-2.788387E-1,1.2861656E-2,-3.276124E-2,-3.3331432E-3,-9.343129E-3,-3.8341186E-3,6.0392324E-2,-1.3210739E-2,-1.2302969E-1,2.2300137E-2,1.1654225E-2,-9.863599E-2,1.9324875E-2,-3.7136904E-3,2.1587731E-3,-1.3629173E-2,-5.8161714E-3,-1.2574462E-2,-1.3556438E-3,-2.7390324E-2,-8.235246E-2,-1.3428229E-3,2.0829223E-3,1.9968264E-2,-5.7059094E-2,9.194345E-2,1.1694973E-3,-1.1299898E-1,1.9074155E-2,-1.4847685E-1,-4.42965E-2,1.6987877E-3,-3.6827063E-3,3.9926197E-2,1.16680115E-1,-4.5865998E-2,-1.23916365E-1,-1.2277958E-3,1.2856388E-2,1.1629993E-1,-4.5756053E-2,-7.9781115E-2,4.330095E-2,-7.667606E-3,-7.539205E-5,1.13217376E-1,1.5765132E-2,1.5109367E-3,8.945469E-3,4.102277E-3,-6.450627E-2,-2.178807E-2,-1.1258215E-3,3.5612304E-2,-1.0866257E-1,1.09470785E-1,-6.57133E-3,-2.8968132E-3,8.040117E-3,1.1746876E-2,5.98835E-3,6.0985956E-2,8.7993285E-3,3.4977112E-2,1.7029913E-1,3.018205E-1,1.4720708E-1,8.300975E-3,2.3789302E-2,-5.2222308E-2,3.9093144E-2,3.8781576E-3,-2.7974632E-2,2.1752554E-1,1.7418074E-3,-1.0405513E-2,-3.1915612E-3,-1.0528885E-1,-4.3203132E-4,1.4912209E-3,7.131431E-3,-6.8874643E-3,-3.4784477E-3,-7.674996E-3,-1.5500225E-2,4.346633E-3,-4.598143E-3,-2.6592715E-2,-1.0514202E-2,-2.6557276E-3,3.3378773E-4,5.4592746E-3,-5.159232E-4,-9.344817E-3,-4.036384E-3,2.4282816E-3,-1.6375799E-3,-6.450914E-3,-5.935236E-4,3.1105075E-3,-1.3724281E-3,-2.2702601E-3,2.1759227E-3,-2.5307466E-3,-5.998449E-3,2.805165E-3,-1.5230031E-3,-4.3710656E-3,4.6556986E-5,6.9571896E-3,1.8591881E-3,-8.206289E-3,-2.645207E-3,2.9360307E-3,-1.5525285E-3,-1.7171176E-2,-6.0486416E-3,6.052906E-3,-4.6716826E-3,4.7454544E-4,4.0286765E-3,7.0722904E-3,2.6766148E-3,-5.098424E-3,-1.2581486E-3,-2.0789921E-2,-5.161126E-3,1.4190646E-3,8.510361E-3,-3.9076265E-3,1.16596515E-4,-1.5149852E-3,-7.2470517E-3,5.914692E-3,-5.703377E-4,6.4169215E-3,1.1085774E-5,5.1906207E-3,-1.9665922E-4,-1.5042785E-3,-6.696701E-3,3.692298E-3,-1.2090751E-3,-1.0485903E-2,7.127924E-4,7.235948E-3,7.747804E-4,-1.4151669E-3,4.1175373E-3,2.9872074E-3,-5.251769E-3,1.4944228E-2,3.579353E-3,8.4039895E-3,2.0203797E-2,7.5979223E-3,-5.4284534E-3,-2.1066605E-3,2.9952205E-3,-3.3685106E-3,2.941793E-3,7.5856894E-3,1.0122595E-3,1.5335063E-3,-4.0495447E-3,1.2733769E-2,6.5711983E-3,-1.1951725E-3,-5.8236867E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,-1,67,-1,69,71,73,75,77,79,81,83,85,-1,87,-1,89,91,93,95,97,99,101,103,105,107,109,-1,111,113,115,117,119,121,-1,-1,-1,123,125,127,-1,129,131,-1,-1,133,-1,135,-1,137,139,141,143,-1,145,-1,147,149,151,-1,-1,153,-1,155,-1,-1,157,-1,-1,159,161,163,165,167,169,171,173,175,177,179,181,-1,183,185,187,-1,-1,189,191,-1,-1,-1,-1,193,195,197,199,201,203,205,207,-1,-1,209,-1,211,213,215,217,219,-1,221,223,225,227,229,231,233,235,237,239,-1,-1,-1,241,243,245,247,249,-1,251,-1,-1,-1,253,-1,255,257,-1,259,261,-1,-1,-1,-1,-1,-1,263,265,-1,-1,267,269,271,-1,273,275,277,279,-1,-1,281,283,285,287,-1,-1,289,291,293,295,-1,-1,297,299,-1,-1,-1,301,-1,-1,303,305,307,-1,-1,-1,-1,-1,309,-1,311,313,315,317,-1,319,321,323,-1,325,327,-1,-1,-1,329,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.5175266E0,1.7433648E0,3.2116323E0,1.6862283E0,1.2097912E0,1.4938222E0,3.2831192E0,8.742752E-1,0E0,4.075991E-1,2.2172242E-1,7.7190864E-1,3.7673312E-1,2.1022882E0,7.5051045E-1,6.529336E-1,5.6078815E-1,2.3631704E-1,4.4845974E-1,5.1089518E-2,1.3807178E-1,5.758172E-1,2.918219E-1,3.425871E-1,1.664344E-1,6.884222E-1,1.0716708E0,3.4528244E-1,2.7800605E-1,3.511839E-1,2.4312954E-1,2.682588E-1,0E0,2.4559814E-1,1.2512943E-1,0E0,2.6528883E-1,0E0,1.8175744E-2,1.3537091E-1,6.1929166E-2,4.3218935E-1,9.8655254E-1,1.7772514E-1,7.706019E-2,2.5665164E-1,3.9053273E-1,0E0,1.4723337E-1,0E0,5.1357174E-1,2.8675792E-1,2.9996789E-1,1.9734347E-1,2.3006294E-2,3.087942E-1,1.618553E-1,2.7632475E-1,2.2960122E-1,2.5696024E-1,1.7200243E-1,0E0,1.776494E-1,1.4225394E-1,8.09591E-2,1.1041801E-1,2.7831897E-2,2.4641669E-1,0E0,0E0,0E0,7.7622175E-2,1.110743E-2,6.680365E-2,0E0,3.188274E-1,3.100884E-1,0E0,0E0,1.8145704E-1,0E0,2.5867604E-2,0E0,1.4668281E-1,2.76908E-1,2.9782563E-1,6.354508E-1,0E0,1.252712E-1,0E0,6.177788E-1,2.4622513E-1,1.4591074E-1,0E0,0E0,1.217829E-1,0E0,1.6278515E-2,0E0,0E0,8.2457215E-2,0E0,0E0,2.27108E-1,6.038115E-1,3.140703E-1,2.019827E-2,8.3570786E-2,6.188804E-2,1.8331656E-1,7.5182445E-2,3.755628E-2,2.8019369E-2,1.255123E-1,5.0850987E-2,0E0,1.1690453E-2,7.225684E-2,1.6875118E-2,0E0,0E0,1.5787801E-1,1.955939E-1,0E0,0E0,0E0,0E0,2.8561218E-2,2.8949022E-2,4.1255903E-1,2.0489107E-1,1.9252582E-1,2.928528E-1,6.27549E-2,2.783727E-1,0E0,0E0,6.569405E-2,0E0,9.711475E-2,4.1762263E-1,1.7102343E-1,2.670642E-1,1.385344E-1,0E0,1.4503717E-2,9.720966E-2,4.2845845E-1,5.4491425E-1,1.9495569E-1,2.1167162E-1,4.671893E-2,1.4226565E-1,5.0301015E-2,3.983438E-2,0E0,0E0,0E0,3.410001E-2,1.8453872E-1,1.7087245E-1,1.0343852E-1,5.120406E-1,0E0,1.2417477E-2,0E0,0E0,0E0,3.568504E-2,0E0,3.08519E-2,3.924903E-2,0E0,3.8116887E-2,1.978743E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.294131E-2,4.2919695E-2,0E0,0E0,3.820823E-2,5.6533113E-2,2.0705067E-2,0E0,2.8232217E-2,7.545772E-2,1.9925606E-1,2.7992404E-1,0E0,0E0,1.4096161E-2,1.1245623E-2,2.7513838E-1,5.35498E-1,0E0,0E0,1.4236379E-1,1.5786873E-2,1.4730296E-1,1.369012E-1,0E0,0E0,1.0135037E-1,1.0456759E-1,0E0,0E0,0E0,4.1402347E-2,0E0,0E0,2.2617856E-1,1.1495137E-1,3.365128E-1,0E0,0E0,0E0,0E0,0E0,4.4418953E-2,0E0,2.981847E-1,3.978886E-1,2.8819966E-1,3.778162E-1,0E0,1.2651868E-1,1.5081516E-1,7.335438E-2,0E0,3.050774E-2,8.21507E-3,0E0,0E0,0E0,3.2952935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,71,71,72,72,73,73,75,75,76,76,79,79,81,81,83,83,84,84,85,85,86,86,88,88,90,90,91,91,92,92,95,95,97,97,100,100,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,116,116,117,117,118,118,121,121,122,122,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,137,137,139,139,140,140,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,158,158,159,159,160,160,161,161,162,162,164,164,168,168,170,170,171,171,173,173,174,174,181,181,182,182,185,185,186,186,187,187,189,189,190,190,191,191,192,192,195,195,196,196,197,197,198,198,201,201,202,202,203,203,204,204,207,207,208,208,212,212,215,215,216,216,217,217,223,223,225,225,226,226,227,227,228,228,230,230,231,231,232,232,234,234,235,235,239,239],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,-1,68,-1,70,72,74,76,78,80,82,84,86,-1,88,-1,90,92,94,96,98,100,102,104,106,108,110,-1,112,114,116,118,120,122,-1,-1,-1,124,126,128,-1,130,132,-1,-1,134,-1,136,-1,138,140,142,144,-1,146,-1,148,150,152,-1,-1,154,-1,156,-1,-1,158,-1,-1,160,162,164,166,168,170,172,174,176,178,180,182,-1,184,186,188,-1,-1,190,192,-1,-1,-1,-1,194,196,198,200,202,204,206,208,-1,-1,210,-1,212,214,216,218,220,-1,222,224,226,228,230,232,234,236,238,240,-1,-1,-1,242,244,246,248,250,-1,252,-1,-1,-1,254,-1,256,258,-1,260,262,-1,-1,-1,-1,-1,-1,264,266,-1,-1,268,270,272,-1,274,276,278,280,-1,-1,282,284,286,288,-1,-1,290,292,294,296,-1,-1,298,300,-1,-1,-1,302,-1,-1,304,306,308,-1,-1,-1,-1,-1,310,-1,312,314,316,318,-1,320,322,324,-1,326,328,-1,-1,-1,330,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,5.862126E2,1.9013363E4,9.026015E2,7.733016E5,4.2343444E7,1.3521131E8,2.0441579E-2,8.3306855E6,1.1543406E-2,4.264897E6,4.5723195E6,1.0033929E1,3.4409692E3,3.424835E6,2.5030267E5,2.835051E2,2.755479E8,1.4E2,1.74E4,6.958929E1,4.70092E1,1.9143396E2,6.7473096E9,2.714E2,1.571654E7,1E0,1.1159378E12,1.3508893E3,3.3905947E-1,6.6992125E4,-2.5039842E-2,9.705292E6,1.6525185E9,-1.7232098E-2,4.8297736E7,3.1768582E-3,5.012E3,2.85558E-1,7.9E1,4.941905E5,5.5E1,1E0,1.151E4,1.3553541E3,4.5158855E6,-1.6803935E-3,1.198E5,2.3253186E-2,3.8E1,1.4777102E9,1.6256282E7,1.6173E4,7.5547155E6,1.636099E0,1E0,8.7217E4,5.3474556E5,2.6011018E8,3.3517068E7,-1.65034E-2,2.6432162E5,1.8631586E4,1.6530488E8,3.3968747E-1,6.5884055E11,9.173799E4,-1.3667539E-2,-4.477821E-3,-2.0254987E-4,8.9716E4,1.5609541E0,2.6432162E5,8.951357E-3,5.3473564E1,2.0011E5,5.998698E-4,-2.584477E-2,2.35E2,-5.550051E-3,6.68E3,-1.05270175E-2,3.7939188E0,2.298425E6,1.06E3,5.8564706E0,-1.9971575E-3,2.1383E4,1.6461402E-2,1E0,2.1722221E1,9.772148E4,2.4787767E-2,5.363031E-3,1.817757E1,-1.7330037E-2,4.502969E3,-1.8511734E-3,1.7090179E-2,4.1E1,-1.03947455E-2,1.994929E-3,8.228668E-1,1.6621418E0,4.0844156E7,1.2E1,2.88E2,1E0,8.5315496E-1,1.2566651E7,1.4101E4,1.8302105E6,8.081268E-1,2.2415E4,8.571409E-3,5E0,4.005E3,4.5576923E1,5.688275E-3,1.0500624E-3,4.8237036E2,1.16544716E-1,6.571876E-3,1.6360922E-2,5.2701267E-3,1.4527971E-3,2.7556E4,3.7075245E1,1.2E1,3.63E2,2.909019E3,1.2792622E8,4E0,2.46E2,-4.6322397E-3,6.231389E-4,1.6521739E0,-4.5537073E-3,1.0346457E0,3.0121632E6,1.6122807E1,1E0,2.342E3,-1.984885E-2,1.91951E5,2.650027E2,2.8863275E0,1.3062E4,1.0526649E3,7.06699E8,8.25E2,1.1394967E7,1.7911884E3,4.2E1,3.2615238E-3,-5.2725844E-4,-1.7720445E-3,1E0,1.8122449E2,2.5629715E6,9.397667E6,2.2784335E6,1.2861656E-2,2.028E3,-3.3331432E-3,-9.343129E-3,-3.8341186E-3,4.38E2,-1.3210739E-2,1.283561E6,1.3382226E-1,1.1654225E-2,1.3565588E6,3.5491096E7,-3.7136904E-3,2.1587731E-3,-1.3629173E-2,-5.8161714E-3,-1.2574462E-2,-1.3556438E-3,1.4967E4,3.7382904E7,-1.3428229E-3,2.0829223E-3,4.5859104E-1,3.7E1,2.1111E4,1.1694973E-3,4.4444447E0,2.6185484E1,1E0,1.3878379E0,1.6987877E-3,-3.6827063E-3,4.0601485E6,6.0941644E0,4.7E2,4.86E2,-1.2277958E-3,1.2856388E-2,2.0074467E5,2.4559366E1,1.2739522E0,7.857143E0,-7.667606E-3,-7.539205E-5,1.23E2,1E0,1.5109367E-3,8.945469E-3,4.102277E-3,1.9859155E0,-2.178807E-2,-1.1258215E-3,1E0,1.7E1,1.5E1,-6.57133E-3,-2.8968132E-3,8.040117E-3,1.1746876E-2,5.98835E-3,4.7233105E6,8.7993285E-3,2.21875E0,4.396066E1,2.9347712E6,1.3144558E4,8.300975E-3,1.0989723E3,1.64354E9,4.492E3,3.8781576E-3,4.8494423E1,6.255748E6,1.7418074E-3,-1.0405513E-2,-3.1915612E-3,2.0211603E1,-4.3203132E-4,1.4912209E-3,7.131431E-3,-6.8874643E-3,-3.4784477E-3,-7.674996E-3,-1.5500225E-2,4.346633E-3,-4.598143E-3,-2.6592715E-2,-1.0514202E-2,-2.6557276E-3,3.3378773E-4,5.4592746E-3,-5.159232E-4,-9.344817E-3,-4.036384E-3,2.4282816E-3,-1.6375799E-3,-6.450914E-3,-5.935236E-4,3.1105075E-3,-1.3724281E-3,-2.2702601E-3,2.1759227E-3,-2.5307466E-3,-5.998449E-3,2.805165E-3,-1.5230031E-3,-4.3710656E-3,4.6556986E-5,6.9571896E-3,1.8591881E-3,-8.206289E-3,-2.645207E-3,2.9360307E-3,-1.5525285E-3,-1.7171176E-2,-6.0486416E-3,6.052906E-3,-4.6716826E-3,4.7454544E-4,4.0286765E-3,7.0722904E-3,2.6766148E-3,-5.098424E-3,-1.2581486E-3,-2.0789921E-2,-5.161126E-3,1.4190646E-3,8.510361E-3,-3.9076265E-3,1.16596515E-4,-1.5149852E-3,-7.2470517E-3,5.914692E-3,-5.703377E-4,6.4169215E-3,1.1085774E-5,5.1906207E-3,-1.9665922E-4,-1.5042785E-3,-6.696701E-3,3.692298E-3,-1.2090751E-3,-1.0485903E-2,7.127924E-4,7.235948E-3,7.747804E-4,-1.4151669E-3,4.1175373E-3,2.9872074E-3,-5.251769E-3,1.4944228E-2,3.579353E-3,8.4039895E-3,2.0203797E-2,7.5979223E-3,-5.4284534E-3,-2.1066605E-3,2.9952205E-3,-3.3685106E-3,2.941793E-3,7.5856894E-3,1.0122595E-3,1.5335063E-3,-4.0495447E-3,1.2733769E-2,6.5711983E-3,-1.1951725E-3,-5.8236867E-3],"split_indices":[20,102,52,52,52,28,45,45,0,45,39,9,28,53,52,32,33,52,5,10,9,46,58,52,31,33,1,96,31,52,39,33,0,9,5,0,45,0,2,57,8,47,0,74,29,33,47,0,7,0,0,7,45,2,32,53,102,9,32,7,51,0,28,33,5,38,31,33,0,0,0,29,57,28,0,56,9,0,0,2,0,29,0,54,28,2,56,0,9,0,102,53,33,0,0,58,0,4,0,0,3,0,0,27,58,45,3,0,8,27,1,2,32,27,29,0,8,10,58,0,0,4,39,0,0,0,0,9,56,8,0,32,7,8,10,0,0,53,0,53,28,56,65,2,0,1,4,53,9,33,7,0,45,52,6,0,0,0,102,52,47,45,50,0,0,0,0,0,0,0,48,27,0,28,32,0,0,0,0,0,0,29,12,0,0,27,3,9,0,56,58,67,53,0,0,47,53,1,12,0,0,28,56,38,56,0,0,3,16,0,0,0,53,0,0,16,3,3,0,0,0,0,0,45,0,53,58,47,52,0,52,5,2,0,56,45,0,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.588E3,8.42E2,1.746E3,4.6E2,3.82E2,9.52E2,7.94E2,4.55E2,5E0,3.12E2,7E1,6.17E2,3.35E2,6.76E2,1.18E2,4.2E2,3.5E1,1.77E2,1.35E2,1.4E1,5.6E1,4.72E2,1.45E2,2.72E2,6.3E1,4.5E2,2.26E2,8.7E1,3.1E1,3.32E2,8.8E1,2.7E1,8E0,9.4E1,8.3E1,6E0,1.29E2,4E0,1E1,2.2E1,3.4E1,4.59E2,1.3E1,1.33E2,1.2E1,4.8E1,2.24E2,6E0,5.7E1,6E0,4.44E2,2.17E2,9E0,7.1E1,1.6E1,2.2E1,9E0,3.05E2,2.7E1,3.7E1,5.1E1,9E0,1.8E1,8.2E1,1.2E1,6.6E1,1.7E1,1.23E2,6E0,6E0,4E0,1.2E1,1E1,3E1,4E0,3.42E2,1.17E2,7E0,6E0,1.27E2,6E0,8E0,4E0,1.6E1,3.2E1,2.1E2,1.4E1,4E0,5.3E1,1.4E1,4.3E2,1.83E2,3.4E1,5E0,4E0,6.7E1,4E0,1.1E1,5E0,4E0,1.8E1,4E0,5E0,2.59E2,4.6E1,1.8E1,9E0,1.6E1,2.1E1,2.9E1,2.2E1,9E0,9E0,1.1E1,7.1E1,4E0,8E0,5E1,1.6E1,1.2E1,5E0,4.7E1,7.6E1,6E0,6E0,6E0,4E0,8E0,2.2E1,3.33E2,9E0,3.9E1,7.8E1,1E1,1.17E2,4E0,4E0,1.2E1,4E0,2.3E1,9E0,1.05E2,1.05E2,1E1,4E0,2E1,3.3E1,1.15E2,3.15E2,6.2E1,1.21E2,1.5E1,1.9E1,2.4E1,4.3E1,7E0,4E0,6E0,1.2E1,1.67E2,9.2E1,1.1E1,3.5E1,4E0,1.4E1,4E0,5E0,7E0,9E0,5E0,1.6E1,2.5E1,4E0,1.4E1,8E0,4E0,5E0,4E0,5E0,7E0,4E0,3.1E1,4E1,4E0,4E0,1.9E1,3.1E1,9E0,7E0,1.1E1,3.6E1,4.4E1,3.2E1,4E0,4E0,1.1E1,1.1E1,2.38E2,9.5E1,5E0,4E0,3E1,9E0,4.6E1,3.2E1,4E0,6E0,5.8E1,5.9E1,6E0,6E0,5E0,1.8E1,4E0,5E0,9.7E1,8E0,1.01E2,4E0,6E0,4E0,1.5E1,5E0,2.1E1,1.2E1,8.3E1,3.2E1,2.5E1,2.9E2,1.1E1,5.1E1,8.6E1,3.5E1,7E0,8E0,1.2E1,7E0,2E1,4E0,3.8E1,5E0,5E0,7E0,8.9E1,7.8E1,8.4E1,8E0,5E0,6E0,6E0,2.9E1,9E0,5E0,5E0,4E0,5E0,1.1E1,1.7E1,8E0,1E1,4E0,4E0,4E0,2.5E1,6E0,2.3E1,1.7E1,1.1E1,8E0,2E1,1.1E1,4E0,5E0,5E0,6E0,2E1,1.6E1,4E0,4E1,7E0,2.5E1,7E0,4E0,7E0,4E0,6.3E1,1.75E2,5E0,9E1,1.2E1,1.8E1,5E0,4E0,2.7E1,1.9E1,1.3E1,1.9E1,5.1E1,7E0,1E1,4.9E1,1.3E1,5E0,5.9E1,3.8E1,4E0,4E0,7.3E1,2.8E1,4E0,1.7E1,7.1E1,1.2E1,1.3E1,1.9E1,1.2E1,1.3E1,2.85E2,5E0,1.8E1,3.3E1,7.6E1,1E1,4E0,3.1E1,4E0,4E0,7E0,5E0,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"331","size_leaf_vector":"1"}},{"base_weights":[1.5401988E-3,-7.5974114E-2,3.874076E-2,-7.9803884E-2,2.2989186E-2,2.041737E-2,1.438585E-1,-1.2281745E-1,-2.8181238E-2,5.9916894E-4,9.7699724E-2,2.64764E-2,1.3401853E-1,-1.1655926E-1,-2.4960257E-2,-6.5528445E-2,3.998572E-2,-1.9766025E-2,5.696284E-2,5.5008493E-2,1.6271755E-1,-8.982446E-2,1.4868861E-1,-1.3586104E-1,-5.724336E-2,-1.8136954E-1,-5.6524787E-2,-4.4384394E-2,6.1894942E-2,-6.5760955E-2,5.3351074E-3,1.3403656E-1,3.869176E-2,3.0898098E-2,1.5193126E-1,1.7358586E-1,-3.4521192E-3,2.645312E-2,-1.2147058E-2,1.3628674E-1,1.8377721E-2,-1.1851277E-1,-2.4671982E-1,-9.9137045E-2,-4.9881935E-3,-1.1004855E-1,-1.2384217E-2,-7.5158715E-2,-3.7182802E-3,2.8242597E-3,-6.877828E-2,2.5963273E-2,1.2820093E-1,-4.8610367E-2,-2.2612824E-1,2.4674466E-2,-1.2501107E-1,1.00925796E-1,2.4343418E-1,5.3842496E-2,-3.2198127E-2,-6.299781E-2,5.5512413E-2,-1.2699983E-5,1.8867444E-1,1.9538744E-1,8.459716E-2,-3.7327511E-3,7.12285E-3,1.424794E-1,-8.166958E-3,-1.0583595E-1,-2.930868E-1,-3.4404182E-1,-1.3375956E-1,-8.154799E-3,-1.3130507E-1,1.2198273E-2,-8.099899E-3,-8.8819275E-3,-3.3920576E-4,-1.2167491E-2,-6.737217E-2,2.016486E-2,-5.2492805E-2,-5.4703145E-3,-1.0908551E-1,4.6724193E-2,-6.0417853E-2,6.2470313E-2,1.5360735E-1,1.2723683E-1,-5.641929E-2,-2.6270287E-2,-1.2510546E-1,4.8798595E-2,-5.5174865E-2,-1.4524209E-2,-1.0269805E-1,-1.2157524E-3,1.1641666E-1,1.43029345E-2,5.8800653E-3,-1.7089944E-2,6.636225E-2,-8.042479E-3,4.46259E-3,1.0574399E-1,-2.2780225E-1,8.799893E-2,-2.4410903E-3,4.3178783E-3,2.0157932E-1,1.7855178E-1,2.0053029E-2,2.5688333E-2,1.5188138E-1,1.346351E-1,1.9977674E-2,-1.6290939E-1,-8.4367625E-2,-2.5325095E-2,-2.0229667E-1,-4.3123987E-1,-8.018893E-3,-3.4731787E-2,-2.491197E-1,7.679218E-2,-7.0192143E-3,-2.536542E-1,-9.3671836E-2,5.454949E-2,-2.5558727E-2,-5.3452644E-2,-1.2888683E-1,-3.6738333E-3,3.832683E-2,-8.39879E-2,2.622414E-4,-1.5809701E-3,1.3504972E-3,-7.0435265E-3,-1.1747371E-3,6.824098E-2,-3.1351138E-2,-4.3001547E-4,-6.3990275E-3,6.978163E-3,-6.3025416E-4,1.24669885E-2,1.18893065E-1,1.329492E-2,1.2270752E-3,-4.9469057E-2,-2.2800831E-1,-6.283629E-2,-1.4543401E-2,2.3292111E-1,3.8258344E-2,-6.766501E-2,3.0938383E-2,-1.1333264E-1,2.377593E-3,1.5037581E-1,5.1834553E-2,5.310311E-2,-7.4634716E-2,9.569786E-2,9.007681E-3,-1.3791035E-2,4.10001E-3,6.6197865E-2,8.025707E-3,8.7733805E-4,-3.1105235E-1,1.5140025E-1,5.3270258E-2,3.2693416E-2,-1.3880405E-1,2.183384E-1,4.5407363E-3,2.1919614E-1,1.1422566E-1,-3.5202663E-2,5.633414E-3,1.1074092E-2,2.8390859E-3,1.9560991E-1,1.0570997E-1,-9.399688E-3,-9.535161E-4,8.3954004E-4,-5.337185E-3,-1.2283962E-2,-3.299554E-3,-1.24304835E-2,-2.7971303E-2,1.297154E-3,-3.7929418E-3,-1.6215717E-2,-5.436502E-3,5.07507E-4,6.527083E-3,-6.564526E-3,-1.5653322E-2,-3.0696746E-3,-8.062306E-3,9.0173265E-4,8.746649E-3,1.8041958E-4,-3.8882492E-3,-1.6391019E-3,-5.512915E-3,-7.390717E-3,-3.9867772E-4,-2.2730203E-3,2.859944E-3,-2.059381E-3,-5.274932E-3,1.8086056E-3,4.4102343E-3,1.8537964E-3,-3.292626E-3,4.520594E-3,8.737498E-3,-1.9180832E-3,-6.7208665E-3,-1.4244346E-2,-5.692416E-3,3.871744E-4,-6.7096334E-3,8.002881E-3,1.5192363E-2,1.3588193E-3,8.299549E-3,-5.2454975E-3,-1.2906792E-3,3.1075252E-3,-2.1988915E-3,-7.4802E-3,-3.9996617E-3,5.600977E-3,1.2178785E-2,4.007619E-3,-4.4063537E-4,1.0515655E-3,5.6467126E-3,-1.4296558E-3,-9.422257E-3,8.858695E-3,3.900871E-3,-1.5243413E-3,2.5411346E-3,-1.853395E-3,3.4012457E-3,6.2289294E-3,-1.6483695E-4,-2.2405358E-2,-4.9449145E-3,4.0724766E-3,1.1300163E-2,-6.301848E-5,5.17932E-3,-4.358286E-4,4.421849E-3,-1.1717712E-2,-7.66324E-4,5.2711936E-3,1.1926265E-2,1.2263228E-2,-3.5588413E-3,1.4335717E-3,8.167802E-3,8.586544E-4,-4.0247818E-3,1.0677227E-2,-3.7424432E-3,2.5421216E-3,7.4562207E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,-1,69,-1,71,73,75,77,79,-1,81,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,-1,111,113,115,-1,-1,117,-1,119,121,123,125,127,129,131,-1,-1,-1,-1,133,135,137,139,141,143,145,147,149,151,153,-1,155,157,159,-1,161,-1,163,-1,-1,165,167,-1,169,171,173,175,177,-1,179,181,-1,183,185,187,-1,189,191,-1,193,195,-1,197,199,201,-1,203,205,207,209,211,213,-1,215,217,-1,-1,-1,-1,-1,219,221,-1,-1,-1,-1,-1,223,-1,-1,225,227,229,-1,231,233,235,237,239,-1,241,243,245,247,249,251,253,-1,255,-1,-1,257,259,261,263,265,267,-1,269,271,273,-1,-1,-1,275,277,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3358765E0,1.7304459E0,3.308052E0,1.817328E0,0E0,2.2437315E0,8.9759207E-1,9.956174E-1,9.547665E-1,1.3406752E0,8.2098055E-1,0E0,8.343048E-1,4.9712515E-1,0E0,2.4215257E-1,2.4859546E-1,9.909332E-1,4.3136203E-1,4.2061168E-1,3.0927467E-1,2.9159698E-1,5.8845997E-1,6.092653E-1,2.4031329E-1,5.3328693E-2,2.2210282E-1,7.28101E-2,2.4921495E-1,8.223771E-1,1.4046558E0,1.891433E-1,2.7239025E-1,3.4324116E-1,2.0135182E-1,2.027018E-1,0E0,1.2809259E-1,0E0,4.3906784E-1,0E0,6.0855055E-1,4.3769598E-1,1.7745912E-1,1.3699245E-1,6.870901E-2,0E0,2.0251274E-1,7.095185E-2,0E0,5.8093965E-2,1.283576E-1,5.2193344E-2,3.8480246E-1,8.120333E-1,9.369458E-1,2.3629797E-1,9.402862E-2,3.6309063E-2,1.8584985E-1,2.1313578E-1,8.861799E-1,2.2049156E-1,0E0,2.0850062E-2,2.403853E-1,9.122564E-2,0E0,0E0,3.9027405E-1,0E0,3.1830406E-1,2.661947E-1,2.9892588E-1,2.426725E-1,2.0187527E-1,1.8343747E-1,7.495074E-2,0E0,0E0,0E0,0E0,1.3096774E-1,7.25838E-2,3.6812045E-2,9.360037E-3,3.4878537E-2,9.7895436E-2,4.7967415E-2,7.1306616E-2,5.6890905E-2,1.645358E-1,3.035627E-1,0E0,2.1939221E-1,7.14969E-1,1.2363249E-1,0E0,1.09274745E-1,0E0,8.43178E-2,0E0,0E0,1.3321912E-1,2.96946E-1,0E0,5.2119892E-2,2.4090126E-2,3.3228254E-1,1.5707314E-1,2.105529E-1,0E0,2.6540875E-2,1.971941E-1,0E0,7.818107E-2,6.0341388E-2,3.5851836E-1,0E0,2.61096E-1,4.537207E-1,0E0,7.277143E-2,2.1259832E-1,0E0,3.4459736E-2,7.0504725E-2,3.8134776E-2,0E0,3.174478E-2,6.8635225E-2,9.361002E-2,3.838961E-2,1.5258706E-1,6.553167E-2,0E0,5.807903E-2,7.649064E-3,0E0,0E0,0E0,0E0,0E0,2.5677964E-2,3.3185758E-2,0E0,0E0,0E0,0E0,0E0,1.7586559E-2,0E0,0E0,2.35142E-1,3.2629073E-2,9.181459E-2,0E0,5.4804087E-2,4.9619383E-1,1.5183124E-1,3.7871644E-2,6.1866462E-2,0E0,6.917781E-2,2.8240234E-2,2.6428707E-2,8.898988E-2,1.5444875E-1,1.0242434E-1,5.9749886E-2,0E0,4.5104798E-2,0E0,0E0,3.0519688E-1,1.1741805E-1,1.3750981E-1,8.166067E-2,1.0275307E-1,2.3034811E-2,0E0,4.0601492E-1,1.4271936E-1,2.2764474E-2,0E0,0E0,0E0,3.4852552E-1,3.4554005E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,41,41,42,42,43,43,44,44,45,45,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,64,64,65,65,66,66,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,94,94,95,95,96,96,98,98,100,100,103,103,104,104,106,106,107,107,108,108,109,109,110,110,112,112,113,113,115,115,116,116,117,117,119,119,120,120,122,122,123,123,125,125,126,126,127,127,129,129,130,130,131,131,132,132,133,133,134,134,136,136,137,137,143,143,144,144,150,150,153,153,154,154,155,155,157,157,158,158,159,159,160,160,161,161,163,163,164,164,165,165,166,166,167,167,168,168,169,169,171,171,174,174,175,175,176,176,177,177,178,178,179,179,181,181,182,182,183,183,187,187,188,188],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,-1,70,-1,72,74,76,78,80,-1,82,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,-1,112,114,116,-1,-1,118,-1,120,122,124,126,128,130,132,-1,-1,-1,-1,134,136,138,140,142,144,146,148,150,152,154,-1,156,158,160,-1,162,-1,164,-1,-1,166,168,-1,170,172,174,176,178,-1,180,182,-1,184,186,188,-1,190,192,-1,194,196,-1,198,200,202,-1,204,206,208,210,212,214,-1,216,218,-1,-1,-1,-1,-1,220,222,-1,-1,-1,-1,-1,224,-1,-1,226,228,230,-1,232,234,236,238,240,-1,242,244,246,248,250,252,254,-1,256,-1,-1,258,260,262,264,266,268,-1,270,272,274,-1,-1,-1,276,278,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,2.14099E5,1.2679E4,1E0,2.2989186E-2,1.8137958E6,7.969174E7,4.5723195E6,2.3544883E8,4.264897E6,5.314225E2,2.64764E-2,2.943086E4,3.424835E6,-2.4960257E-2,3.3854932E-4,2.1924436E5,2.6659616E2,1.402418E0,1E0,2.909019E3,4.5843E7,1.6941456E1,8.7217E4,5.057096E-1,1.28E2,3.1294638E2,9.619E3,1.8876036E3,4.602015E0,4.987639E7,4.8237036E2,4.70092E1,2.0697437E2,2.0827537E0,1E0,-3.4521192E-3,3.2343243E1,-1.2147058E-2,3.7826266E4,1.8377721E-2,9.4801545E-1,1E0,2.014576E8,9.192915E0,1.37518E5,-1.2384217E-2,1E0,6.788246E0,2.8242597E-3,2.8216E4,5.8426323E0,1.4935554E11,7E0,2E0,2.1722221E1,2.4748051E2,1.632095E2,2.5727873E5,2.5464671E2,3.2485715E2,1.0620689E0,3.7246967E2,-1.2699983E-5,5.2369475E6,2.0676143E3,3.7057E5,-3.7327511E-3,7.12285E-3,2.4507338E4,-8.166958E-3,1.09E3,9.516945E-1,4.88954E5,4.5478998E8,1.8344E4,1E0,3.3517068E7,-8.099899E-3,-8.8819275E-3,-3.3920576E-4,-1.2167491E-2,1.0889231E1,1.3E1,2.5341389E8,5.8426323E0,3.88672E5,8.874407E6,1.68635E5,8.9716E4,2.1583429E-3,1.2068493E4,1.2471935E7,-2.6270287E-2,3.8977896E2,3.921901E6,1.61E9,-1.4524209E-2,1.17E2,-1.2157524E-3,1.5445488E8,1.43029345E-2,5.8800653E-3,2.6281825E-1,1.23E2,-8.042479E-3,8.3956606E5,6.31591E3,1E0,2.0771764E4,1.4E1,4.3178783E-3,5.70875E5,4.646E3,2.0053029E-2,2.78876E5,1E0,4.52737E5,1.9977674E-2,2.1E1,1.1004604E9,-2.5325095E-2,1.612289E6,2.03E0,-8.018893E-3,8.6337E4,9E0,6.3328077E-3,-7.0192143E-3,5.84E2,7.3582644E5,1.243766E0,1.8382221E-1,6.395763E2,5.3E1,-3.6738333E-3,5.101512E0,7.59093E5,2.622414E-4,-1.5809701E-3,1.3504972E-3,-7.0435265E-3,-1.1747371E-3,7.718584E5,4.9008E4,-4.3001547E-4,-6.3990275E-3,6.978163E-3,-6.3025416E-4,1.24669885E-2,1.4386049E6,1.329492E-2,1.2270752E-3,1.990351E1,3.3953488E0,4.0729207E3,-1.4543401E-2,1.287E3,4.894913E6,1.27365E4,1.8592099E9,1.3581437E8,2.377593E-3,9.38E2,9.3E1,6.977611E5,6.977611E5,5.088781E0,1.06E2,1.5915463E0,4.10001E-3,1.2E1,8.025707E-3,8.7733805E-4,5.2921E4,5.4444447E0,2.8331464E2,7.472532E9,9.4168E4,2.6668E4,4.5407363E-3,3.1984328E5,1.3793921E3,2.657E3,5.633414E-3,1.1074092E-2,2.8390859E-3,2.851369E2,4.1331047E-1,-9.399688E-3,-9.535161E-4,8.3954004E-4,-5.337185E-3,-1.2283962E-2,-3.299554E-3,-1.24304835E-2,-2.7971303E-2,1.297154E-3,-3.7929418E-3,-1.6215717E-2,-5.436502E-3,5.07507E-4,6.527083E-3,-6.564526E-3,-1.5653322E-2,-3.0696746E-3,-8.062306E-3,9.0173265E-4,8.746649E-3,1.8041958E-4,-3.8882492E-3,-1.6391019E-3,-5.512915E-3,-7.390717E-3,-3.9867772E-4,-2.2730203E-3,2.859944E-3,-2.059381E-3,-5.274932E-3,1.8086056E-3,4.4102343E-3,1.8537964E-3,-3.292626E-3,4.520594E-3,8.737498E-3,-1.9180832E-3,-6.7208665E-3,-1.4244346E-2,-5.692416E-3,3.871744E-4,-6.7096334E-3,8.002881E-3,1.5192363E-2,1.3588193E-3,8.299549E-3,-5.2454975E-3,-1.2906792E-3,3.1075252E-3,-2.1988915E-3,-7.4802E-3,-3.9996617E-3,5.600977E-3,1.2178785E-2,4.007619E-3,-4.4063537E-4,1.0515655E-3,5.6467126E-3,-1.4296558E-3,-9.422257E-3,8.858695E-3,3.900871E-3,-1.5243413E-3,2.5411346E-3,-1.853395E-3,3.4012457E-3,6.2289294E-3,-1.6483695E-4,-2.2405358E-2,-4.9449145E-3,4.0724766E-3,1.1300163E-2,-6.301848E-5,5.17932E-3,-4.358286E-4,4.421849E-3,-1.1717712E-2,-7.66324E-4,5.2711936E-3,1.1926265E-2,1.2263228E-2,-3.5588413E-3,1.4335717E-3,8.167802E-3,8.586544E-4,-4.0247818E-3,1.0677227E-2,-3.7424432E-3,2.5421216E-3,7.4562207E-3],"split_indices":[20,2,2,102,0,28,7,28,12,9,4,0,33,32,0,41,28,52,53,84,32,45,35,9,39,10,52,10,55,53,45,4,58,4,53,6,0,56,0,52,0,27,16,7,57,29,0,67,53,0,9,53,31,3,17,53,33,4,33,4,52,54,4,0,45,4,9,0,0,4,0,2,27,32,7,9,8,51,0,0,0,0,54,3,7,53,9,45,29,29,27,28,45,0,4,5,5,0,3,0,7,0,0,38,3,0,28,33,8,47,56,0,1,2,0,1,104,29,0,3,5,0,32,54,0,29,3,39,0,0,28,39,38,4,3,0,56,1,0,0,0,0,0,47,9,0,0,0,0,0,28,0,0,58,54,50,0,2,1,33,5,7,0,2,0,45,45,58,8,39,0,3,0,0,1,58,4,31,1,1,0,33,4,2,0,0,0,58,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.542E3,8.24E2,1.718E3,8.19E2,5E0,1.464E3,2.54E2,4.46E2,3.73E2,1.166E3,2.98E2,5E0,2.49E2,4.4E2,6E0,2.41E2,1.32E2,8.57E2,3.09E2,1.81E2,1.17E2,1.5E1,2.34E2,3.31E2,1.09E2,1.6E1,2.25E2,2.7E1,1.05E2,3.02E2,5.55E2,5.8E1,2.51E2,1.46E2,3.5E1,1.12E2,5E0,9E0,6E0,2.23E2,1.1E1,2.88E2,4.3E1,6E1,4.9E1,9E0,7E0,1.66E2,5.9E1,5E0,2.2E1,6.9E1,3.6E1,2.74E2,2.8E1,4.84E2,7.1E1,4.6E1,1.2E1,2.07E2,4.4E1,3E1,1.16E2,7E0,2.8E1,8.9E1,2.3E1,5E0,4E0,2.19E2,4E0,2.7E2,1.8E1,2.2E1,2.1E1,1.6E1,4.4E1,4.5E1,4E0,5E0,4E0,6E0,1.6E2,4E1,1.9E1,9E0,1.3E1,5.6E1,1.3E1,1.1E1,2.5E1,1.1E1,2.63E2,6E0,2.2E1,3.72E2,1.12E2,7E0,6.4E1,5E0,4.1E1,8E0,4E0,3.1E1,1.76E2,9E0,3.5E1,1.5E1,1.5E1,7.4E1,4.2E1,4E0,2.4E1,8.4E1,5E0,1.3E1,1E1,2.14E2,5E0,7.2E1,1.98E2,4E0,1.4E1,1.4E1,8E0,1.2E1,9E0,1E1,6E0,9E0,3.5E1,2.1E1,2.4E1,1.32E2,2.8E1,6E0,3.4E1,1.2E1,7E0,5E0,4E0,9E0,4E0,4.4E1,1.2E1,8E0,5E0,5E0,6E0,5E0,2E1,4E0,7E0,2.54E2,9E0,1.7E1,5E0,1.9E1,3.53E2,9.8E1,1.4E1,6E1,4E0,2.6E1,1.5E1,1.4E1,1.7E1,1.16E2,6E1,2.9E1,6E0,1E1,5E0,4E0,1.1E1,2.5E1,4.9E1,3.4E1,8E0,2E1,4E0,5E1,3.4E1,8E0,5E0,5E0,5E0,6.7E1,1.47E2,6.1E1,1.1E1,3.6E1,1.62E2,1E1,4E0,7E0,7E0,5E0,7E0,5E0,4E0,5E0,5E0,4E0,5E0,2.5E1,1E1,1.7E1,4E0,1.6E1,8E0,9.8E1,3.4E1,2.4E1,4E0,6E0,2.8E1,5E0,7E0,1.8E1,2.6E1,4E0,8E0,1.5E1,5E0,2.26E2,2.8E1,5E0,4E0,9E0,8E0,1.1E1,8E0,3.26E2,2.7E1,5.1E1,4.7E1,1E1,4E0,2.7E1,3.3E1,2E1,6E0,1E1,5E0,1E1,4E0,1.3E1,4E0,1.9E1,9.7E1,3.1E1,2.9E1,2.3E1,6E0,5E0,5E0,6E0,5E0,1.4E1,1.1E1,2.4E1,2.5E1,2E1,1.4E1,4E0,4E0,4E0,1.6E1,4.6E1,4E0,1.3E1,2.1E1,4E0,4E0,6.3E1,4E0,6.6E1,8.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"279","size_leaf_vector":"1"}},{"base_weights":[-2.484188E-3,2.7305031E-2,-8.703322E-2,1.9835092E-5,1.1420439E-1,-1.807492E-1,-4.6363182E-2,-2.7079577E-2,6.0322084E-2,1.4521728E-1,-7.7176E-3,-1.634526E-1,-4.1698083E-1,-6.8656E-2,5.8054127E-2,-7.828758E-2,-3.550419E-3,7.011663E-2,-5.2630458E-2,3.8551897E-1,1.2833588E-1,1.9396171E-2,-2.3199014E-1,-1.4242662E-1,-2.889826E-1,-2.6217774E-2,-1.2476061E-2,-4.81705E-2,-1.5326901E-1,7.887278E-2,-1.3907105E-2,-9.2820026E-2,1.2007501E-2,-2.8372934E-2,4.4291023E-2,4.355925E-2,1.2941964E-1,-9.7278364E-2,9.2121966E-2,2.7780122E-1,2.7619805E-2,3.4541834E-2,1.5561974E-1,-2.6532654E-2,2.1338141E-1,-3.241491E-3,-1.9293759E-2,-1.2901959E-1,-2.1124937E-2,-3.3241352E-1,-2.0031594E-3,-1.129006E-1,-2.6572268E-2,-3.6619322E-3,-1.7134772E-1,1.906779E-1,3.433619E-2,-8.227987E-2,-2.015183E-1,1.3289133E-1,-1.8546997E-2,-3.4071505E-2,1.6449258E-1,9.3222015E-2,2.1689441E-2,2.58487E-2,1.1212366E-1,1.8148647E-1,3.8041648E-2,-1.8791573E-1,-3.6484513E-2,9.383617E-3,-1.0981669E-3,1.5437689E-2,7.368112E-3,-5.6732237E-2,6.5420106E-2,-5.978555E-3,1.6038823E-1,-1.0670056E-1,1.962031E-2,2.652262E-1,4.1356967E-3,-2.824089E-1,-1.1082771E-1,-6.7329137E-3,-4.156742E-1,-9.3943976E-2,-1.834213E-2,-1.6621972E-2,-1.520218E-1,1.2593832E-3,-1.5887374E-3,-5.910598E-4,-1.9007316E-1,1.4088605E-2,1.0579331E-1,1.4162502E-1,1.1269015E-2,-2.1630007E-1,-7.6866165E-2,-4.2664334E-2,-3.4357905E-1,9.803762E-3,2.1484843E-3,9.772103E-3,-1.0459009E-1,9.466585E-2,-4.1488074E-2,1.0577008E-2,2.3311656E-3,1.4859225E-1,2.5338743E-2,1.07767455E-1,2.614049E-3,3.22311E-2,-7.6398235E-3,8.658869E-2,1.06606865E-2,2.0854202E-1,1.05368726E-1,1.5201748E-3,9.1782756E-2,-5.1400783E-3,-1.346084E-2,2.4250268E-3,-7.291681E-2,-4.5892154E-3,1.0792281E-3,3.8950842E-2,1.7497505E-1,1.7378497E-1,6.4592525E-2,-1.7935543E-1,-2.5837954E-2,4.437899E-2,-4.8638107E-3,8.778767E-3,1.6865158E-2,-6.533046E-3,-1.8004518E-2,-5.396707E-2,-1.4944616E-1,-2.3070812E-2,-1.1553774E-2,-1.7270857E-1,-6.502631E-2,1.3802313E-2,-5.170137E-2,-2.0453387E-1,-3.3667032E-3,-1.6043265E-1,-1.5231277E-2,3.7372395E-4,7.4279103E-3,1.574182E-3,1.0482398E-2,7.3856427E-3,-1.1756358E-2,-1.3480295E-2,-5.973732E-3,-6.472966E-2,-1.2890595E-1,-7.0647825E-3,2.4010227E-3,-2.2914704E-2,-7.6011806E-3,-4.1633278E-2,4.0863766E-3,-1.155508E-3,-8.251324E-3,1.958242E-1,-5.9930813E-3,-7.242987E-2,-1.01946695E-2,1.6957423E-1,7.1550116E-5,-6.1797716E-2,5.4526795E-2,1.3367628E-1,3.6191323E-4,-1.561531E-1,1.55006945E-2,-8.879594E-2,4.2803787E-2,1.2634696E-1,3.7879825E-2,1.1028446E-1,2.4025315E-1,3.4058332E-3,1.7067303E-1,-3.5649467E-2,5.173882E-3,1.3876867E-1,2.4157444E-2,-1.3296647E-3,-5.4366686E-3,7.705021E-2,-9.5779635E-3,2.8870984E-3,1.3141588E-2,1.6538757E-1,2.255124E-2,1.0086223E-1,-4.3333597E-2,-4.9710213E-3,-1.2789191E-2,3.6190755E-3,-3.875428E-3,1.1412472E-2,9.195551E-3,-1.0851929E-2,-3.1391956E-2,-2.7734408E-1,-1.1990451E-1,-1.5484766E-3,-2.0843412E-1,-5.2618697E-2,-8.922983E-3,-6.140466E-3,8.743736E-2,-8.450736E-2,4.4044992E-4,-1.3062952E-2,-6.147332E-3,-2.0655778E-1,-8.909642E-2,1.204254E-2,-5.7165357E-3,-5.244147E-3,-2.466809E-3,-7.963083E-3,3.6299616E-4,-4.6346514E-3,-2.6564868E-4,1.1433853E-2,5.1890747E-3,3.3409658E-3,-3.897095E-3,-6.906931E-3,-2.1326796E-3,-9.917801E-4,6.728481E-3,3.8614054E-3,1.0163264E-2,-1.0461045E-3,-4.5119817E-3,-2.0231232E-3,3.6796906E-3,7.524995E-3,2.130068E-3,-3.4448956E-3,-1.0133337E-2,-5.970816E-3,1.35942E-3,4.1719615E-3,-9.0889E-3,1.7428544E-3,1.0479134E-2,4.1466397E-3,9.364868E-3,6.7659793E-3,-6.621923E-4,8.119191E-3,3.1334418E-3,1.9104624E-2,9.835951E-3,3.516073E-3,-3.8789313E-3,5.728432E-3,1.1305931E-2,-3.9055382E-3,1.1583725E-3,3.663211E-3,8.279405E-3,2.6620796E-3,-4.893591E-4,-4.886374E-5,5.4941163E-3,8.8313065E-4,-5.2850177E-3,8.539503E-3,-5.241327E-3,2.1196492E-3,7.5721764E-3,-6.103288E-3,2.2058643E-3,1.3945069E-3,-3.9948295E-3,-4.064175E-4,-6.525085E-3,-1.8031443E-2,-1.9047037E-3,-3.7963456E-3,-1.2121094E-2,-1.1823454E-2,-5.2326764E-3,-6.0774754E-3,-1.4123345E-3,2.2731118E-3,-1.6974887E-3,1.6215835E-3,5.8130003E-3,-4.848562E-3,-8.7935984E-4,1.9873453E-3,-1.5170391E-3,-1.1630094E-2,-4.545277E-3,-7.525739E-3,-1.5192899E-3,3.5261775E-3,-7.0910226E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,-1,51,53,55,-1,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,-1,83,-1,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,-1,-1,-1,-1,127,129,-1,131,133,135,137,-1,139,141,-1,143,145,-1,147,149,-1,-1,-1,151,-1,153,155,157,159,161,163,165,-1,-1,167,169,171,173,-1,-1,175,177,179,181,183,-1,185,-1,187,189,191,193,-1,-1,-1,195,-1,-1,197,199,201,203,205,207,209,-1,-1,-1,-1,-1,211,213,-1,-1,215,217,219,221,223,-1,225,-1,-1,-1,-1,-1,-1,227,-1,-1,229,231,-1,-1,-1,-1,233,-1,-1,-1,235,237,239,241,243,-1,245,247,249,-1,251,253,255,257,259,261,263,265,267,269,271,-1,273,275,-1,-1,277,279,-1,-1,281,-1,283,285,-1,-1,-1,-1,287,-1,-1,289,291,293,-1,295,297,-1,299,301,303,305,-1,-1,307,309,311,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.399747E0,4.4552307E0,2.5147877E0,2.3401103E0,1.6988592E0,7.490449E-1,1.0817366E0,1.1901447E0,4.9489594E-1,1.4027982E0,5.639402E-1,4.5504618E-1,1.0326791E-1,6.547526E-1,5.9619963E-1,4.0921128E-1,8.0641216E-1,6.3851E-1,2.4162558E-1,2.607937E-1,8.5447025E-1,7.451512E-1,2.3254871E-1,5.5812335E-1,2.677245E-1,0E0,0E0,4.289992E-1,1.9952059E-1,3.818475E-1,0E0,2.9046607E-1,1.6498071E-1,4.9688345E-1,2.5455102E-1,3.4196687E-1,5.925741E-1,1.474294E-1,1.0237662E-1,1.9420147E-2,0E0,2.2072712E-1,3.6047316E-1,2.5369257E-1,8.639252E-2,0E0,0E0,4.068005E-1,0E0,3.0605912E-1,0E0,3.365407E-1,2.8831765E-1,8.110549E-3,1.948973E-1,1.4063632E-1,1.3947049E-1,1.6185796E-1,5.0438046E-1,4.2787388E-2,8.874193E-2,4.1926074E-1,6.241232E-2,2.7062088E-1,2.624338E-1,2.6389778E-1,1.314537E-1,1.4277864E-1,9.210758E-2,4.851246E-2,5.92917E-2,0E0,0E0,0E0,0E0,5.6993097E-2,1.6137521E-1,0E0,3.1731272E-1,1.422424E-1,1.3173555E-1,1.19069815E-2,0E0,1.4197004E-1,3.0374825E-1,0E0,2.0640612E-2,1.550405E-1,0E0,2.3231024E-1,6.1779767E-2,0E0,0E0,0E0,1.5177894E-1,0E0,5.3671747E-2,6.845182E-2,1.526468E-1,1.4316589E-2,1.4197218E-1,1.1426699E-1,1.9448972E-1,0E0,0E0,1.07052416E-1,4.0572636E-2,2.4555337E-1,3.9819056E-1,0E0,0E0,1.2326586E-1,8.984331E-2,7.456136E-2,2.724893E-1,2.8530866E-1,0E0,8.816984E-2,0E0,1.5422368E-1,1.5256688E-1,1.1366775E-1,5.701901E-2,0E0,0E0,0E0,1.87089E-2,0E0,0E0,8.9485995E-2,9.828898E-2,4.4741488E-1,1.3354751E-1,5.406812E-2,7.169192E-2,1.6824509E-1,0E0,0E0,0E0,0E0,0E0,2.105752E-1,2.8412366E-1,0E0,0E0,8.9737E-2,6.809375E-2,1.7234994E-1,1.728042E-1,1.42122805E-2,0E0,1.428256E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0398257E-1,0E0,0E0,1.152997E-1,1.8367094E-1,0E0,0E0,0E0,0E0,3.1068562E-2,0E0,0E0,0E0,1.2663335E-2,7.3409885E-2,3.992889E-1,2.9075256E-1,9.3551755E-2,0E0,8.591279E-3,5.0206646E-2,2.9019982E-2,0E0,2.6700124E-2,1.9762184E-1,3.0392295E-1,2.611668E-1,5.3298175E-2,1.1649451E-1,2.954951E-2,1.8992877E-1,5.9726626E-2,1.7668694E-2,5.6837656E-2,0E0,7.2419494E-3,9.419598E-3,0E0,0E0,6.943135E-2,5.9542082E-2,0E0,0E0,3.4445715E-1,0E0,6.761938E-2,6.744414E-2,0E0,0E0,0E0,0E0,4.8952594E-2,0E0,0E0,1.2121609E-1,2.8596365E-1,3.5879338E-1,0E0,1.9078016E-2,8.2275584E-2,0E0,1.3519098E-1,3.595072E-2,5.0264806E-2,4.96681E-2,0E0,0E0,6.665099E-2,7.124211E-2,5.474147E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,47,47,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,75,75,76,76,78,78,79,79,80,80,81,81,83,83,84,84,86,86,87,87,89,89,90,90,94,94,96,96,97,97,98,98,99,99,100,100,101,101,102,102,105,105,106,106,107,107,108,108,111,111,112,112,113,113,114,114,115,115,117,117,119,119,120,120,121,121,122,122,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,141,141,142,142,145,145,146,146,147,147,148,148,149,149,151,151,158,158,161,161,162,162,167,167,171,171,172,172,173,173,174,174,175,175,177,177,178,178,179,179,181,181,182,182,183,183,184,184,185,185,186,186,187,187,188,188,189,189,190,190,191,191,193,193,194,194,197,197,198,198,201,201,203,203,204,204,209,209,212,212,213,213,214,214,216,216,217,217,219,219,220,220,221,221,222,222,225,225,226,226,227,227],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,-1,52,54,56,-1,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,-1,84,-1,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,-1,-1,-1,-1,128,130,-1,132,134,136,138,-1,140,142,-1,144,146,-1,148,150,-1,-1,-1,152,-1,154,156,158,160,162,164,166,-1,-1,168,170,172,174,-1,-1,176,178,180,182,184,-1,186,-1,188,190,192,194,-1,-1,-1,196,-1,-1,198,200,202,204,206,208,210,-1,-1,-1,-1,-1,212,214,-1,-1,216,218,220,222,224,-1,226,-1,-1,-1,-1,-1,-1,228,-1,-1,230,232,-1,-1,-1,-1,234,-1,-1,-1,236,238,240,242,244,-1,246,248,250,-1,252,254,256,258,260,262,264,266,268,270,272,-1,274,276,-1,-1,278,280,-1,-1,282,-1,284,286,-1,-1,-1,-1,288,-1,-1,290,292,294,-1,296,298,-1,300,302,304,306,-1,-1,308,310,312,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,1.0989723E3,2.0827537E0,7.733016E5,2.8839298E7,1.2427474E8,3.4409692E3,1.5288235E2,2E0,6.7388856E7,2.3580047E10,2.0354E4,3.2049225E1,7.218466E7,1.5265896E8,4.264897E6,1.7427321E-1,4.8582852E2,1.5E1,7.123E3,8.079531E4,4.247868E3,1.3674345E4,4.5723195E6,8.89771E6,-2.6217774E-2,-1.2476061E-2,1.646E3,2.765625E0,9.160584E-1,-1.3907105E-2,4.5184364E0,1.1902924E0,6.65E2,1.38797655E1,4.5723195E6,2E0,5E0,1.3035715E0,3.1E1,2.7619805E-2,1.13814086E5,2.979E3,1.4593E4,1.30399E5,-3.241491E-3,-1.9293759E-2,8.764843E3,-2.1124937E-2,2.2837209E2,-2.0031594E-3,2.0801504E0,9.8E1,2.5378451E0,3.835264E-2,6.862E3,2E0,6E0,3.97E2,9.505727E-2,8.588109E0,2.831341E6,8.631E3,1.9203481E5,3.1415796E0,3.8003298E6,2.5870967E0,5.077728E8,9.013492E2,9.822E3,2E1,9.383617E-3,-1.0981669E-3,1.5437689E-2,7.368112E-3,5.5303917E8,1.8777761E0,-5.978555E-3,2.3821254E7,1.8796511E5,7.704183E-1,1.5005797E4,4.1356967E-3,5.5402948E-6,2.3067484E0,-6.7329137E-3,1.479E3,1.2E1,-1.834213E-2,1.9051096E7,3.6709285E6,1.2593832E-3,-1.5887374E-3,-5.910598E-4,1.5265896E8,1.4088605E-2,1.2939E4,6.82E2,2.8091298E7,1.2068493E4,3.627907E0,9.070543E0,6.239726E0,9.803762E-3,2.1484843E-3,2.7272727E0,5.0338E4,1.3E1,3.386E3,1.0577008E-2,2.3311656E-3,1E0,9.950143E7,5.8084745E6,7.106795E6,5.8631687E1,-7.6398235E-3,1.111E3,1.06606865E-2,4.7798E4,1.0407268E10,1.4E2,5.9272277E4,-5.1400783E-3,-1.346084E-2,2.4250268E-3,2.1E1,-4.5892154E-3,1.0792281E-3,5.2350176E4,1.6195753E9,1.9013363E4,1.0192391E2,7E0,2.37E3,1.8222668E2,-4.8638107E-3,8.778767E-3,1.6865158E-2,-6.533046E-3,-1.8004518E-2,4.82E2,1.3456146E6,-2.3070812E-2,-1.1553774E-2,6.429309E2,9.655878E-1,1.0989723E3,3.299862E1,1.308827E6,-3.3667032E-3,8.7643677E-1,-1.5231277E-2,3.7372395E-4,7.4279103E-3,1.574182E-3,1.0482398E-2,7.3856427E-3,1.6755487E0,-1.3480295E-2,-5.973732E-3,4.7E2,5.314225E2,-7.0647825E-3,2.4010227E-3,-2.2914704E-2,-7.6011806E-3,4.921397E0,4.0863766E-3,-1.155508E-3,-8.251324E-3,4.564527E6,1.83E2,1E0,3.7253947E5,5.0702117E4,7.1550116E-5,1.918E3,2.028994E8,1E0,3.6191323E-4,3.90106E0,2.7157144E2,2.9328E4,3.5E1,3.1665432E0,5.8365756E-1,4.752366E0,1.822948E6,8.190217E0,2.9409406E-1,4.7741196E-1,5.173882E-3,2.7203197E0,4.29E4,-1.3296647E-3,-5.4366686E-3,1.2204E4,1.1297775E2,2.8870984E-3,1.3141588E-2,1.5005797E4,2.255124E-2,1.5107028E-1,5.0279167E1,-4.9710213E-3,-1.2789191E-2,3.6190755E-3,-3.875428E-3,2.4861076E-1,9.195551E-3,-1.0851929E-2,1.7609E4,1.263E3,1.3E1,-1.5484766E-3,1.34E3,1.8243062E-3,-8.922983E-3,5.9572783E0,5.0079144E5,1E0,4.0844156E7,-1.3062952E-2,-6.147332E-3,6.513793E1,4.921397E0,5.7581736E7,-5.7165357E-3,-5.244147E-3,-2.466809E-3,-7.963083E-3,3.6299616E-4,-4.6346514E-3,-2.6564868E-4,1.1433853E-2,5.1890747E-3,3.3409658E-3,-3.897095E-3,-6.906931E-3,-2.1326796E-3,-9.917801E-4,6.728481E-3,3.8614054E-3,1.0163264E-2,-1.0461045E-3,-4.5119817E-3,-2.0231232E-3,3.6796906E-3,7.524995E-3,2.130068E-3,-3.4448956E-3,-1.0133337E-2,-5.970816E-3,1.35942E-3,4.1719615E-3,-9.0889E-3,1.7428544E-3,1.0479134E-2,4.1466397E-3,9.364868E-3,6.7659793E-3,-6.621923E-4,8.119191E-3,3.1334418E-3,1.9104624E-2,9.835951E-3,3.516073E-3,-3.8789313E-3,5.728432E-3,1.1305931E-2,-3.9055382E-3,1.1583725E-3,3.663211E-3,8.279405E-3,2.6620796E-3,-4.893591E-4,-4.886374E-5,5.4941163E-3,8.8313065E-4,-5.2850177E-3,8.539503E-3,-5.241327E-3,2.1196492E-3,7.5721764E-3,-6.103288E-3,2.2058643E-3,1.3945069E-3,-3.9948295E-3,-4.064175E-4,-6.525085E-3,-1.8031443E-2,-1.9047037E-3,-3.7963456E-3,-1.2121094E-2,-1.1823454E-2,-5.2326764E-3,-6.0774754E-3,-1.4123345E-3,2.2731118E-3,-1.6974887E-3,1.6215835E-3,5.8130003E-3,-4.848562E-3,-8.7935984E-4,1.9873453E-3,-1.5170391E-3,-1.1630094E-2,-4.545277E-3,-7.525739E-3,-1.5192899E-3,3.5261775E-3,-7.0910226E-4],"split_indices":[27,52,53,28,45,45,52,52,6,7,5,9,58,45,45,9,38,55,8,2,28,52,4,28,51,0,0,2,54,57,0,53,53,8,58,28,17,6,53,0,0,33,2,29,10,0,0,33,0,4,0,42,8,54,27,9,8,52,2,38,56,5,2,33,54,51,53,7,55,29,6,0,0,0,0,5,41,0,9,33,41,4,0,39,58,0,0,18,0,45,32,0,0,0,45,0,9,0,45,28,56,53,56,0,0,56,1,3,2,0,0,105,12,45,5,47,0,10,0,1,31,8,33,0,0,0,3,0,0,28,7,52,58,8,10,58,0,0,0,0,0,2,51,0,0,33,27,52,56,1,0,57,0,0,0,0,0,0,39,0,0,1,4,0,0,0,0,54,0,0,0,9,29,102,28,33,0,2,7,101,0,56,55,1,3,56,54,56,47,56,34,38,0,53,10,0,0,2,58,0,0,4,0,38,56,0,0,0,0,39,0,0,9,2,3,0,2,39,0,53,28,104,45,0,0,56,54,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.539E3,1.878E3,6.61E2,1.43E3,4.48E2,1.99E2,4.62E2,9.87E2,4.43E2,3.57E2,9.1E1,1.87E2,1.2E1,3.81E2,8.1E1,3.1E2,6.77E2,4.08E2,3.5E1,2.2E1,3.35E2,8.2E1,9E0,1.62E2,2.5E1,6E0,6E0,3.08E2,7.3E1,7.7E1,4E0,2.67E2,4.3E1,4.46E2,2.31E2,2.83E2,1.25E2,2.7E1,8E0,1.5E1,7E0,7.6E1,2.59E2,6.7E1,1.5E1,5E0,4E0,1.56E2,6E0,2.1E1,4E0,7.6E1,2.32E2,8E0,6.5E1,2.1E1,5.6E1,2.45E2,2.2E1,8E0,3.5E1,4.34E2,1.2E1,7.2E1,1.59E2,2.26E2,5.7E1,7.9E1,4.6E1,1E1,1.7E1,4E0,4E0,1.1E1,4E0,1.9E1,5.7E1,4E0,2.55E2,2.4E1,4.3E1,1E1,5E0,1.5E1,1.41E2,7E0,1.4E1,7.2E1,4E0,2.16E2,1.6E1,4E0,4E0,7E0,5.8E1,9E0,1.2E1,9E0,4.7E1,8E0,2.37E2,1.1E1,1.1E1,4E0,4E0,2.7E1,8E0,2.3E1,4.11E2,8E0,4E0,3.9E1,3.3E1,2.8E1,1.31E2,2.19E2,7E0,4.7E1,1E1,5.7E1,2.2E1,2.8E1,1.8E1,6E0,4E0,5E0,1.2E1,1.3E1,6E0,4.7E1,1E1,2.23E2,3.2E1,1.2E1,1.2E1,3.6E1,7E0,6E0,4E0,6E0,9E0,5.8E1,8.3E1,1E1,4E0,1.8E1,5.4E1,1.16E2,1E2,9E0,7E0,4.8E1,1E1,4E0,8E0,4E0,5E0,6E0,4.1E1,4E0,4E0,1.94E2,4.3E1,5E0,6E0,6E0,5E0,1.6E1,1.1E1,4E0,4E0,1.1E1,1.2E1,2.06E2,2.05E2,3.4E1,5E0,8E0,2.5E1,2.2E1,6E0,9E0,1.22E2,1.7E1,2.02E2,2.5E1,2.2E1,1.5E1,4.2E1,9E0,1.3E1,2.1E1,7E0,1E1,8E0,6E0,6E0,2.6E1,2.1E1,5E0,5E0,2.18E2,5E0,2.4E1,8E0,7E0,5E0,4E0,8E0,3E1,6E0,6E0,5.2E1,1.4E1,6.9E1,4E0,1.4E1,5E1,4E0,9.2E1,2.4E1,6.1E1,3.9E1,4E0,5E0,2.8E1,2E1,3.4E1,7E0,5.2E1,1.42E2,3.5E1,8E0,6E0,1E1,7E0,4E0,6E0,6E0,6.3E1,1.43E2,1.93E2,1.2E1,1E1,2.4E1,4E0,4E0,4E0,2.1E1,1.8E1,4E0,4E0,5E0,9E0,1.13E2,6E0,1.1E1,1.94E2,8E0,1.6E1,9E0,7E0,1.5E1,6E0,9E0,8E0,3.4E1,5E0,4E0,8E0,5E0,1.2E1,9E0,4E0,6E0,4E0,4E0,8E0,1.8E1,1.7E1,4E0,2.14E2,4E0,1.2E1,1.2E1,4E0,4E0,2.6E1,4E0,4.3E1,9E0,1E1,4E0,5.2E1,1.7E1,1E1,4E0,1.2E1,3.8E1,3.2E1,6E1,9E0,1.5E1,5.1E1,1E1,1.7E1,2.2E1,2.2E1,6E0,9E0,1.1E1,1E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"313","size_leaf_vector":"1"}},{"base_weights":[-2.4191253E-3,2.0690402E-2,-8.601567E-2,-2.4950616E-2,5.7038333E-2,-1.3426827E-1,-1.9815043E-2,-5.1336784E-2,1.5069314E-2,8.09289E-2,-4.0179867E-2,-4.750069E-1,-1.24074265E-1,-4.296707E-3,-1.4499019E-1,-7.3094144E-2,-6.800258E-3,2.383548E-2,-1.8580389E-1,4.459182E-2,1.3466102E-1,-8.466572E-2,3.5340615E-2,-1.2711669E-2,-3.0010454E-2,-2.183672E-1,-1.0365047E-1,-1.7262539E-2,2.1215856E-1,-2.0026991E-1,2.2099855E-4,-7.8144595E-2,9.516523E-2,-4.1562192E-2,6.055562E-2,2.8529188E-2,-1.0236548E-2,-6.702341E-2,-1.7628442E-2,7.484045E-2,-2.0385427E-2,3.173093E-1,1.184477E-1,-1.4035904E-1,-4.58659E-2,1.0459599E-2,1.6851072E-1,-2.5300676E-1,-1.2605488E-1,-1.4099798E-1,-6.748044E-2,-6.8718945E-3,-1.1043475E-1,1.4341142E-2,2.496226E-3,-2.4403389E-1,-1.445233E-3,8.515789E-3,-8.624499E-2,7.85686E-3,-5.3840695E-4,5.6310665E-2,-7.5567834E-2,2.085911E-1,3.4274958E-2,-4.881797E-2,4.417297E-2,-6.4605386E-3,-1.9777832E-4,8.0397494E-2,-1.5575708E-1,-3.340384E-2,7.481381E-2,2.1482546E-2,2.539301E-1,1.0746162E-1,3.4565338E-1,-1.8328485E-1,-7.947794E-3,-5.9165925E-2,8.799514E-3,2.7044194E-2,-8.67992E-3,1.3598546E-2,1.11762136E-4,-1.9012462E-2,-1.8932956E-1,-5.6596406E-2,-1.4062417E-2,-1.9373798E-1,-8.771154E-2,-5.3800095E-2,-1.5389496E-2,-1.4047411E-2,7.497727E-3,-1.0611048E-2,-6.230132E-2,-7.4670347E-3,-1.5395891E-2,1.2310394E-2,-4.6701666E-2,-8.2876764E-2,-1.4151326E-2,-1.7184706E-2,1.3585138E-1,-4.392826E-2,-2.5472474E-1,5.7841404E-3,1.2976543E-2,6.347776E-2,-1.3247302E-2,9.178808E-2,-1.2153647E-1,1.3883606E-1,3.0963585E-2,5.4058667E-2,1.1154599E-1,-1.6439762E-2,2.4309708E-3,-2.5890558E-3,-6.3384615E-2,1.08764E-2,1.13323264E-1,1.3838764E-2,7.5047673E-3,1.608117E-1,7.1245946E-2,8.701854E-3,2.2017136E-2,-1.5783234E-1,-1.7246367E-2,-2.5594903E-3,1.8148811E-3,-3.1248818E-3,-8.0368996E-2,-2.5915678E-3,1.2797855E-1,-9.3673386E-2,-2.659867E-1,-9.484221E-2,1.1774809E-3,-9.8884344E-2,-2.6213142E-1,-3.3202767E-2,-2.525695E-1,-1.0716348E-2,-9.67164E-2,-2.8760439E-2,1.5117989E-2,-6.19157E-3,3.5471472E-4,-5.440247E-3,-1.0369117E-3,-7.856817E-2,-1.1096375E-2,1.7699591E-3,-3.609646E-2,2.0731452E-3,9.700955E-3,1.1484182E-2,-7.297242E-2,-2.264751E-2,-3.6969548E-3,-8.983016E-3,9.537301E-2,-5.7955403E-2,3.3301676E-3,1.5770288E-3,1.2930892E-1,-1.6032954E-1,2.1277783E-2,5.694887E-2,2.0048651E-1,-5.900265E-2,5.075431E-2,1.2158873E-1,3.326629E-2,1.2944406E-1,-2.3792379E-2,-3.1050824E-2,6.037189E-2,-1.2693551E-2,-4.9457718E-2,2.6108138E-3,-1.6325904E-3,9.087211E-3,2.385968E-3,9.5298514E-2,2.0381674E-1,1.7669092E-1,5.4503E-2,-1.4387652E-2,-1.254499E-1,1.1405595E-1,-7.322505E-2,-1.6690253E-1,-4.9042363E-2,8.703241E-3,-5.5214874E-3,2.6866237E-3,9.303367E-3,3.3041014E-4,-8.480066E-3,-6.3673295E-3,-1.52291255E-2,-2.3153839E-3,-6.214725E-3,-1.9779839E-1,2.7711689E-5,-2.02703E-1,-2.088078E-2,8.1135765E-2,-8.452316E-2,-2.533246E-3,-3.1563488E-1,-6.989768E-3,7.2852816E-3,-1.813688E-1,-6.2238533E-2,-3.0074434E-3,-5.8273E-2,6.880003E-2,-3.9067984E-2,2.0862427E-3,-3.962941E-3,-4.5421487E-3,-1.8252942E-3,-2.6393016E-3,6.133101E-5,4.8046634E-3,-2.1975287E-3,-2.8023834E-3,-8.429221E-3,-2.0919885E-3,1.9413108E-3,2.243015E-3,6.2268353E-3,-4.045279E-3,-1.778906E-5,8.0400985E-3,2.6425691E-3,6.6295755E-4,-1.0207278E-2,4.5263693E-3,-2.612645E-3,5.862533E-3,-3.1598562E-3,1.4133798E-2,7.538031E-3,-1.3218403E-2,-1.0291957E-3,2.1764322E-3,1.2293668E-2,1.0597541E-2,3.3103146E-3,4.9594427E-3,1.005228E-3,9.161267E-3,4.7201426E-3,-6.488452E-3,1.0102798E-3,-8.040521E-4,-6.3484823E-3,1.0079075E-2,1.1001298E-3,4.1262405E-3,-3.017026E-3,-8.213942E-4,6.415874E-3,1.2757661E-2,7.484504E-3,2.7995705E-4,1.0850729E-2,-1.3222108E-3,3.6123805E-3,-3.8727205E-3,-7.829659E-3,9.526789E-3,7.3140324E-4,1.6923215E-4,-5.589757E-3,-9.981696E-3,-3.0589593E-3,-9.459661E-4,-4.571121E-3,3.4571064E-3,-1.6543893E-4,-2.5057725E-3,-1.2583771E-2,-3.2333354E-3,2.5171242E-3,-1.590305E-2,-6.0367472E-3,1.5083957E-3,9.352918E-3,-8.118939E-3,3.7899506E-4,-8.843468E-3,-1.8129667E-2,2.7874422E-3,-1.1943872E-3,-1.0422353E-2,-3.4420334E-3,3.5047347E-3,-4.355023E-3,9.169624E-4,-2.490674E-3,-6.10645E-4,-4.4746054E-3,7.3619056E-3,2.3583926E-3,-2.2677584E-4,-3.6622996E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,51,53,55,-1,57,59,61,63,65,-1,67,-1,69,71,73,75,77,79,81,83,85,87,89,91,93,95,-1,-1,97,-1,99,101,-1,-1,103,105,107,109,111,113,-1,-1,115,117,119,121,-1,123,125,127,129,131,133,-1,135,-1,-1,-1,-1,137,139,-1,141,143,145,-1,147,-1,-1,149,-1,-1,-1,151,153,-1,155,157,159,161,-1,-1,163,165,167,169,171,173,175,177,-1,-1,179,181,183,185,-1,-1,187,189,-1,-1,191,-1,-1,-1,193,195,197,199,201,203,205,-1,207,209,211,213,215,217,219,221,-1,-1,-1,223,225,-1,-1,227,-1,-1,229,231,-1,-1,233,235,237,-1,-1,239,241,243,245,247,249,251,253,255,257,259,261,263,-1,265,-1,-1,-1,-1,267,269,271,273,-1,275,277,279,281,283,285,-1,-1,-1,-1,-1,-1,-1,-1,-1,287,289,291,-1,293,295,-1,297,-1,299,301,303,305,307,309,311,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.968635E0,3.3442206E0,1.7765927E0,9.448131E-1,2.6092072E0,1.0595274E0,4.578093E-1,5.2179265E-1,6.3130486E-1,1.7531686E0,7.488218E-1,9.657884E-2,5.820737E-1,5.958387E-1,2.1563321E-1,3.1505895E-1,4.1928968E-1,3.729698E-1,2.7389932E-1,1.0609154E0,1.0373802E0,2.9608703E-1,2.7325025E-1,0E0,0E0,1.4397812E-1,3.4170914E-1,1.9286497E-1,1.3136107E-1,1.3541842E-1,0E0,2.4783278E-1,7.4035294E-2,3.969863E-1,2.301692E-1,4.087171E-1,0E0,3.8761165E-2,0E0,4.8409295E-1,2.1566606E-1,9.7836256E-2,7.990494E-1,3.2252324E-1,2.5840896E-1,2.2156377E-1,2.2341913E-1,2.445848E-1,1.6754478E-1,3.399186E-1,4.2433345E-1,2.0591904E-1,8.489223E-2,0E0,0E0,4.4890642E-2,0E0,4.1865444E-1,1.9015598E-1,0E0,0E0,1.8355668E-1,4.8763382E-1,1.2640476E-2,7.4393414E-2,5.9396136E-1,3.4582466E-1,0E0,0E0,2.8931928E-1,3.347457E-1,1.4026125E-1,5.047352E-2,0E0,8.570671E-3,6.0874987E-1,1.67606E-1,1.2268579E-1,3.0597841E-2,9.480822E-2,0E0,1.9775526E-1,0E0,0E0,0E0,0E0,1.8040061E-1,4.2091046E-2,0E0,3.8278365E-1,5.757174E-1,2.3563495E-1,0E0,7.510018E-2,0E0,0E0,6.501879E-2,0E0,0E0,0E0,7.370746E-2,1.7327309E-1,0E0,1.8191943E-2,7.4459165E-2,1.23153E-1,4.3291938E-1,0E0,0E0,7.7124864E-2,7.887371E-2,4.111378E-2,2.1393913E-1,1.6022754E-1,4.427774E-1,2.72565E-1,4.0033674E-1,0E0,0E0,1.3806507E-1,1.9188946E-1,1.7924046E-2,4.9743637E-2,0E0,0E0,3.4402084E-1,3.3110356E-1,0E0,0E0,1.2819862E-1,0E0,0E0,0E0,1.9728288E-1,1.4904135E-1,6.4341605E-2,5.4788724E-2,1.07884236E-1,4.1797757E-2,7.1296096E-3,0E0,2.7395165E-1,2.4853396E-1,3.0025277E-1,1.8769467E-1,1.5216437E-1,1.7550445E-1,8.7989025E-2,1.7430788E-1,0E0,0E0,0E0,5.6901023E-2,1.5743124E-1,0E0,0E0,8.179879E-3,0E0,0E0,1.3102964E-1,7.136527E-2,0E0,0E0,1.8930115E-2,2.836433E-2,1.8521532E-2,0E0,0E0,2.046965E-2,2.316103E-1,5.0626073E-2,1.235312E-1,3.499937E-2,3.4977347E-1,2.745753E-1,2.1633083E-1,1.2988457E-1,2.560699E-1,9.59855E-2,7.4069925E-2,1.2566754E-1,0E0,1.1027075E-1,0E0,0E0,0E0,0E0,1.9815868E-1,1.7602634E-1,1.7879862E-1,2.4393257E-1,0E0,3.7293434E-2,6.575418E-2,4.478854E-2,3.9868385E-2,5.449643E-2,3.510975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.9086165E-2,5.2151278E-2,2.2653282E-1,0E0,8.079528E-2,2.5257137E-1,0E0,1.4313221E-2,0E0,8.906939E-2,4.073751E-2,1.6116239E-1,6.3942246E-2,7.6689884E-2,4.393904E-2,3.512239E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,79,79,81,81,86,86,87,87,89,89,90,90,91,91,93,93,96,96,100,100,101,101,103,103,104,104,105,105,106,106,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,119,119,120,120,121,121,122,122,125,125,126,126,129,129,133,133,134,134,135,135,136,136,137,137,138,138,139,139,141,141,142,142,143,143,144,144,145,145,146,146,147,147,148,148,152,152,153,153,156,156,159,159,160,160,163,163,164,164,165,165,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,180,180,182,182,187,187,188,188,189,189,190,190,192,192,193,193,194,194,195,195,196,196,197,197,207,207,208,208,209,209,211,211,212,212,214,214,216,216,217,217,218,218,219,219,220,220,221,221,222,222],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,52,54,56,-1,58,60,62,64,66,-1,68,-1,70,72,74,76,78,80,82,84,86,88,90,92,94,96,-1,-1,98,-1,100,102,-1,-1,104,106,108,110,112,114,-1,-1,116,118,120,122,-1,124,126,128,130,132,134,-1,136,-1,-1,-1,-1,138,140,-1,142,144,146,-1,148,-1,-1,150,-1,-1,-1,152,154,-1,156,158,160,162,-1,-1,164,166,168,170,172,174,176,178,-1,-1,180,182,184,186,-1,-1,188,190,-1,-1,192,-1,-1,-1,194,196,198,200,202,204,206,-1,208,210,212,214,216,218,220,222,-1,-1,-1,224,226,-1,-1,228,-1,-1,230,232,-1,-1,234,236,238,-1,-1,240,242,244,246,248,250,252,254,256,258,260,262,264,-1,266,-1,-1,-1,-1,268,270,272,274,-1,276,278,280,282,284,286,-1,-1,-1,-1,-1,-1,-1,-1,-1,288,290,292,-1,294,296,-1,298,-1,300,302,304,306,308,310,312,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7311627E2,1E0,4.459525E5,2.8091298E7,3E0,1.8000048E10,1.5107028E-1,2E0,1.1410706E3,7.160724E1,1.3565588E6,1.5804776E4,9.256843E3,1.0526E4,4.222002E8,1.1968015E8,3.6512393E2,5.69815E5,8.447369E0,1.296E3,2.7165637E4,4.5505118E2,-1.2711669E-2,-3.0010454E-2,4.1365924E2,1.8903887E0,1E0,5.9767612E7,3.166E3,2.2099855E-4,7E0,1.01E3,9.658451E0,1.387623E8,4.8404854E-5,-1.0236548E-2,2.241E2,-1.7628442E-2,2.0230594E7,4.4854636E7,6.7370984E7,3.832274E2,1.32E2,2.203288E2,2.6572757E10,3.5905025E5,1.381869E2,7.7660305E-5,1.638058E2,2.3977574E7,2.3292E4,9.877E3,1.4341142E-2,2.496226E-3,1.4991512E5,-1.445233E-3,1.3050649E4,5.1984615E1,7.85686E-3,-5.3840695E-4,1.4268628E2,4.602015E0,5E0,4.29E2,1.0620689E0,2.295E3,-6.4605386E-3,-1.9777832E-4,5.862126E2,1.4E1,6.486406E6,1.9768514E3,2.1482546E-2,2.1298597E0,5.8805E4,4.1179886E0,1.6338193E-1,1E0,3.53562E5,8.799514E-3,2.2153166E7,-8.67992E-3,1.3598546E-2,1.11762136E-4,-1.9012462E-2,1.36E2,1E0,-1.4062417E-2,8.0274E4,1.3E1,3.28825E0,-1.5389496E-2,8.631404E2,7.497727E-3,-1.0611048E-2,5.012E3,-7.4670347E-3,-1.5395891E-2,1.2310394E-2,1.85221E5,1E0,-1.4151326E-2,6.4953816E-1,1.145677E6,1.3E1,1.1563158E1,5.7841404E-3,1.2976543E-2,1.57E2,2.373E3,3.4663152E6,6.22198E5,1.6194E4,9.887052E3,1.325E0,2E0,-1.6439762E-2,2.4309708E-3,2.1373269E8,6.203E3,9.042859E4,1.23E2,1.3838764E-2,7.5047673E-3,1.3717406E0,1.3794063E1,8.701854E-3,2.2017136E-2,2.4748051E2,-1.7246367E-2,-2.5594903E-3,1.8148811E-3,1.3035715E0,1.279076E6,2.846201E-1,7.420515E-1,2.0943393E-4,1.2209878E3,3.6E1,1.1774809E-3,9E2,1.8544E4,1.3046257E11,3.097E3,2.1129233E-1,4E0,9.360022E5,2.3058404E7,-6.19157E-3,3.5471472E-4,-5.440247E-3,2.4545714E2,1.273801E6,-1.1096375E-2,1.7699591E-3,2.6668E4,2.0731452E-3,9.700955E-3,5.175119E-1,1.1416009E6,-2.264751E-2,-3.6969548E-3,2.8080197E2,2.8591623E2,1.243766E0,3.3301676E-3,1.5770288E-3,1.2252E4,2.7E1,3E0,1.4286021E2,2.873721E2,1.1978022E0,3.5E1,8.629299E4,4.273927E8,1.0993947E1,1.1092533E0,1.4992306E8,5.4E1,-1.2693551E-2,2.3502353E-6,2.6108138E-3,-1.6325904E-3,9.087211E-3,2.385968E-3,7.112637E10,2.1351435E0,9.60094E5,6.823E3,-1.4387652E-2,3.3012173E9,2.56E3,1.440286E7,5.964E3,1.2562189E0,1E0,-5.5214874E-3,2.6866237E-3,9.303367E-3,3.3041014E-4,-8.480066E-3,-6.3673295E-3,-1.52291255E-2,-2.3153839E-3,-6.214725E-3,9.422E3,1.1902924E0,9.253112E-1,-2.088078E-2,1.9589581E6,2.8277853E11,-2.533246E-3,2.1872402E5,-6.989768E-3,5.3474556E5,2.878424E5,5.1454154E8,5.93E2,5.9572783E0,7.2123044E5,2.720437E6,2.0862427E-3,-3.962941E-3,-4.5421487E-3,-1.8252942E-3,-2.6393016E-3,6.133101E-5,4.8046634E-3,-2.1975287E-3,-2.8023834E-3,-8.429221E-3,-2.0919885E-3,1.9413108E-3,2.243015E-3,6.2268353E-3,-4.045279E-3,-1.778906E-5,8.0400985E-3,2.6425691E-3,6.6295755E-4,-1.0207278E-2,4.5263693E-3,-2.612645E-3,5.862533E-3,-3.1598562E-3,1.4133798E-2,7.538031E-3,-1.3218403E-2,-1.0291957E-3,2.1764322E-3,1.2293668E-2,1.0597541E-2,3.3103146E-3,4.9594427E-3,1.005228E-3,9.161267E-3,4.7201426E-3,-6.488452E-3,1.0102798E-3,-8.040521E-4,-6.3484823E-3,1.0079075E-2,1.1001298E-3,4.1262405E-3,-3.017026E-3,-8.213942E-4,6.415874E-3,1.2757661E-2,7.484504E-3,2.7995705E-4,1.0850729E-2,-1.3222108E-3,3.6123805E-3,-3.8727205E-3,-7.829659E-3,9.526789E-3,7.3140324E-4,1.6923215E-4,-5.589757E-3,-9.981696E-3,-3.0589593E-3,-9.459661E-4,-4.571121E-3,3.4571064E-3,-1.6543893E-4,-2.5057725E-3,-1.2583771E-2,-3.2333354E-3,2.5171242E-3,-1.590305E-2,-6.0367472E-3,1.5083957E-3,9.352918E-3,-8.118939E-3,3.7899506E-4,-8.843468E-3,-1.8129667E-2,2.7874422E-3,-1.1943872E-3,-1.0422353E-2,-3.4420334E-3,3.5047347E-3,-4.355023E-3,9.169624E-4,-2.490674E-3,-6.10645E-4,-4.4746054E-3,7.3619056E-3,2.3583926E-3,-2.2677584E-4,-3.6622996E-3],"split_indices":[19,52,59,28,45,3,5,38,6,52,56,28,51,4,9,7,7,52,9,53,29,33,56,0,0,52,53,84,45,0,0,3,0,58,7,38,0,4,0,51,9,7,56,3,57,12,28,47,42,52,47,2,9,0,0,33,0,28,46,0,0,52,53,8,0,54,50,0,0,52,8,45,4,0,38,9,53,38,96,1,0,1,0,0,0,0,10,16,0,1,3,54,0,52,0,0,2,0,0,0,9,77,0,41,12,3,54,0,0,0,2,28,1,9,47,53,6,0,0,5,29,28,3,0,0,41,56,0,0,33,0,0,0,53,1,34,39,42,48,8,0,2,9,31,2,27,8,48,45,0,0,0,52,9,0,0,1,0,0,38,47,0,0,4,55,39,0,0,9,0,11,4,33,53,3,33,31,56,53,5,3,0,39,0,0,0,0,31,41,9,2,0,5,10,12,2,57,67,0,0,0,0,0,0,0,0,0,1,53,56,0,32,31,0,33,0,32,33,5,0,53,47,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E3,2.014E3,5.56E2,8.93E2,1.121E3,3.21E2,2.35E2,5.38E2,3.55E2,9E2,2.21E2,8E0,3.13E2,2.1E2,2.5E1,3.61E2,1.77E2,3.41E2,1.4E1,5.38E2,3.62E2,1.39E2,8.2E1,4E0,4E0,5.4E1,2.59E2,1.99E2,1.1E1,1.8E1,7E0,3.51E2,1E1,1.17E2,6E1,3.35E2,6E0,9E0,5E0,3.67E2,1.71E2,2.8E1,3.34E2,5.6E1,8.3E1,7E1,1.2E1,3.8E1,1.6E1,1.26E2,1.33E2,1.8E2,1.9E1,7E0,4E0,1.4E1,4E0,3E1,3.21E2,6E0,4E0,3E1,8.7E1,8E0,5.2E1,5.6E1,2.79E2,4E0,5E0,3.59E2,8E0,1.51E2,2E1,8E0,2E1,3.2E2,1.4E1,4.2E1,1.4E1,7.9E1,4E0,6.5E1,5E0,7E0,5E0,1.1E1,2.7E1,1.2E1,4E0,6.2E1,6.4E1,1.27E2,6E0,1.73E2,7E0,5E0,1.4E1,7E0,7E0,5E0,2.5E1,3.17E2,4E0,1.6E1,1.4E1,7.5E1,1.2E1,4E0,4E0,3.2E1,2E1,1.9E1,3.7E1,3.3E1,2.46E2,1.96E2,1.63E2,4E0,4E0,7.5E1,7.6E1,8E0,1.2E1,1.5E1,5E0,1.28E2,1.92E2,6E0,8E0,3.8E1,4E0,7E0,7E0,2.2E1,5.7E1,5.1E1,1.4E1,1.3E1,1.4E1,8E0,4E0,2.7E1,3.5E1,4.9E1,1.5E1,6.4E1,6.3E1,1.15E2,5.8E1,7E0,7E0,1E1,1.5E1,3.09E2,8E0,4E0,1.2E1,6E0,8E0,2.6E1,4.9E1,5E0,7E0,1E1,2.2E1,1.3E1,7E0,8E0,1.1E1,2.9E1,8E0,1.5E1,1.8E1,4.4E1,2.02E2,4.5E1,1.51E2,1.44E2,1.9E1,5.2E1,2.3E1,4E0,7.2E1,4E0,4E0,5E0,7E0,5.2E1,7.6E1,2.5E1,1.67E2,6E0,3.2E1,8E0,1.4E1,1.4E1,4.3E1,4.7E1,4E0,7E0,7E0,6E0,7E0,4E0,1E1,4E0,4E0,1.3E1,1.4E1,2.7E1,8E0,1.5E1,3.4E1,4E0,1.1E1,7E0,5.7E1,1.7E1,4.6E1,6.2E1,5.3E1,2.9E1,2.9E1,1E1,5E0,2.38E2,7.1E1,8E0,4E0,1E1,1.6E1,4.3E1,6E0,6E0,4E0,9E0,1.3E1,9E0,4E0,7E0,4E0,6E0,2.3E1,4E0,4E0,1E1,5E0,5E0,1.3E1,6E0,3.8E1,1.96E2,6E0,1.6E1,2.9E1,2.4E1,1.27E2,5.5E1,8.9E1,5E0,1.4E1,4.6E1,6E0,4E0,1.9E1,5E0,6.7E1,1.2E1,4E1,3.7E1,3.9E1,5E0,2E1,3E1,1.37E2,1.4E1,1.8E1,4E0,4E0,5E0,9E0,1E1,4E0,2.6E1,1.7E1,7E0,4E1,4E0,9E0,6E0,8E0,1E1,1.7E1,1.1E1,4E0,1.8E1,1.6E1,4E0,7E0,2.2E1,3.5E1,1.3E1,4E0,7E0,3.9E1,4.3E1,1.9E1,2.2E1,3.1E1,5E0,2.4E1,1.5E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"313","size_leaf_vector":"1"}},{"base_weights":[1.2332895E-3,-6.458208E-2,3.3407256E-2,-7.296622E-2,1.0819633E-1,1.4072748E-2,1.1249244E-1,-5.1484574E-2,-1.456719E-1,2.2337181E-2,5.6698535E-2,-1.7474487E-2,4.997143E-2,1.9256708E-1,7.904787E-2,-7.885405E-2,1.2212893E-2,-2.7314526E-1,-1.01484805E-1,1.2948027E-1,-7.643058E-2,-2.0000411E-2,1.2323549E-2,-9.121394E-3,7.811244E-2,2.107491E-2,1.7030044E-1,-7.072626E-2,9.463008E-2,-1.7510926E-2,-7.484362E-2,1.0899014E-2,6.488844E-3,-2.0467308E-1,-3.3853883E-1,-5.6248076E-2,-1.5784721E-1,-1.3326898E-3,1.6375811E-1,-8.696972E-3,3.321503E-3,-1.2440235E-1,-1.4204954E-2,6.0863183E-3,-1.3797234E-1,8.750762E-2,-9.398859E-2,6.6856705E-2,2.0430556E-1,-1.3848439E-1,-7.9519795E-3,8.7508984E-2,1.886934E-2,-1.0174315E-1,-4.3058164E-2,1.0887628E-2,-7.565236E-3,-3.1514087E-1,-1.22838646E-1,-2.0704414E-1,-2.1528842E-2,-6.8901785E-2,6.982627E-3,-1.2838371E-1,-2.279824E-2,3.3591785E-3,2.1446475E-1,-1.0520287E-1,-9.713503E-3,4.861946E-2,-2.8103618E-2,2.9979696E-2,-7.224852E-2,-4.7661733E-2,-1.449324E-2,7.6336935E-2,1.5883067E-1,-1.4224419E-1,-3.1194026E-3,-3.7969763E-3,9.922559E-2,2.2642513E-1,5.336957E-2,-2.0209341E-3,-9.430138E-3,-4.610712E-2,3.1138062E-3,1.2722683E-1,5.658707E-2,-6.9003776E-2,-1.5889597E-1,-1.5929738E-2,-1.0779819E-1,-1.5005398E-2,3.6949255E-2,-2.0460617E-2,-6.940338E-3,1.0241525E-3,-1.7232516E-1,-4.1697943E-3,-1.3278485E-2,2.4979839E-3,-7.6518774E-2,-9.93028E-2,-1.5877604E-2,1.2908854E-2,6.3718497E-3,-6.8819836E-2,-1.235917E-1,9.030474E-2,-1.5714556E-2,-1.600407E-1,-2.1430004E-2,2.7162363E-3,7.816281E-2,-2.590517E-1,5.898048E-3,-8.0625564E-2,7.605729E-4,2.271151E-2,9.212962E-2,1.6461074E-2,1.8271357E-1,-8.9298915E-3,-3.7490001E-3,-2.1325655E-3,1.8520064E-3,9.750316E-3,5.424622E-2,2.4134117E-1,8.5261604E-4,4.7178343E-3,-3.280025E-4,-4.9049145E-4,-3.656383E-3,8.3133996E-2,2.0354027E-1,6.86253E-2,-9.272972E-3,-7.895982E-2,9.353516E-2,-1.19749926E-1,-2.619941E-1,-1.17700055E-1,-9.0701046E-4,-1.3192439E-1,3.8213875E-2,3.8495802E-3,-8.566608E-2,4.5713577E-2,-3.610044E-3,-5.7461294E-3,-1.1471425E-2,-1.462008E-1,-6.078251E-2,-1.5265104E-1,-3.0039595E-2,-4.540667E-3,-1.3962894E-3,-8.523861E-3,-3.8995463E-3,-2.3900603E-3,1.9895837E-1,-1.2620221E-1,1.8684583E-2,-2.1986587E-1,-1.5960846E-2,-3.8688764E-2,5.9232046E-3,-2.4837771E-2,8.765616E-2,9.597141E-2,-5.4179993E-3,-2.2466788E-2,-1.5719788E-3,-5.5590164E-2,1.4673112E-1,-1.4864013E-3,-6.2734997E-3,7.259484E-2,-2.7712567E-2,2.2454578E-3,1.0427688E-1,-1.4684912E-3,2.949E-3,4.164253E-3,1.9802031E-1,9.0248394E-4,4.1797087E-3,9.16029E-2,2.6864642E-1,6.571993E-2,8.719756E-3,2.1199819E-2,1.4896977E-1,9.23552E-2,-2.1445757E-2,-6.8429136E-3,-2.876562E-3,8.000297E-3,4.1225544E-4,-8.454456E-3,-1.718895E-3,-1.7774956E-2,-6.7910817E-3,-1.1351949E-3,-1.799334E-2,2.5019445E-3,-1.0465444E-3,-5.492545E-3,-1.2559873E-2,-1.623073E-3,5.0600288E-3,1.072891E-3,-3.2252287E-3,9.78461E-4,-6.228521E-3,3.2375657E-3,1.8680091E-4,-3.6330845E-3,-9.409071E-3,-3.4145506E-3,1.6467914E-3,-2.3205052E-3,-1.0924878E-2,1.5450874E-3,-4.964422E-3,1.1158584E-3,-5.5088536E-3,-3.8527537E-3,1.1845444E-2,-6.8773335E-4,-1.18094515E-2,2.3134323E-3,-2.3637882E-3,-6.78392E-3,-1.8959742E-2,-4.064754E-3,2.6292356E-3,-2.1705434E-3,6.775109E-3,2.238922E-3,-6.365526E-4,-2.3701044E-3,3.5250538E-3,5.843547E-3,9.050787E-4,6.6699735E-3,2.6428541E-3,3.4735957E-3,-7.0444304E-3,3.20226E-3,1.0820619E-2,5.55011E-3,1.0076253E-3,-2.4953943E-3,5.695333E-3,3.8778908E-3,-1.810735E-2,1.2559995E-2,4.8094844E-3,1.1839699E-2,6.2365085E-3,6.4647365E-3,1.3964992E-3,1.4401708E-2,8.461165E-3,4.024021E-3,-4.4790185E-3,2.1683364E-3,8.4960265E-3,2.6329039E-3,8.829133E-3,9.878981E-4,-9.507037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,-1,47,49,51,-1,53,-1,55,57,59,61,63,-1,65,-1,-1,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,-1,95,97,99,-1,101,-1,103,-1,-1,105,107,-1,109,111,113,115,117,-1,119,121,123,125,-1,127,129,131,-1,-1,133,-1,135,137,139,141,143,145,147,149,-1,-1,-1,151,-1,-1,-1,153,155,-1,-1,-1,157,159,161,163,165,167,169,171,173,175,177,-1,179,181,183,185,-1,-1,-1,-1,-1,187,189,-1,-1,-1,-1,-1,191,193,195,-1,197,199,201,203,205,207,209,211,213,215,217,-1,-1,-1,219,221,223,225,-1,-1,-1,-1,227,229,231,233,235,237,239,241,243,245,247,-1,-1,-1,249,251,-1,-1,253,255,257,259,-1,-1,-1,261,-1,-1,263,265,267,-1,-1,269,271,273,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3785954E0,1.2160261E0,2.6070156E0,1.2357235E0,6.551951E-1,1.5546691E0,8.819289E-1,1.0748767E0,9.9179435E-1,0E0,3.4902418E-1,4.925986E-1,1.0683143E0,4.2834187E-1,5.621401E-1,4.432268E-1,2.1781443E-1,1.366818E-1,3.3787382E-1,1.274763E-1,1.9293429E-1,4.3689916E-1,0E0,4.0842155E-1,7.115853E-1,0E0,3.0644417E-1,9.672854E-2,4.022647E-1,0E0,3.5985494E-1,0E0,1.2805545E-1,1.8729019E-1,1.7787814E-1,2.0059246E-1,4.6897554E-1,0E0,7.8581095E-2,0E0,0E0,3.0617177E-2,6.021111E-1,3.5217085E-1,2.883445E-1,3.1692362E-1,1.0041009E-1,1.1851124E-1,2.1494317E-1,4.6551406E-2,3.7728216E-2,2.5316882E-1,0E0,4.1939974E-1,3.4427288E-1,1.206714E-1,0E0,1.09396756E-1,1.1702278E-1,5.763045E-2,0E0,6.993565E-2,0E0,2.768566E-1,0E0,0E0,1.2840092E-2,1.290378E-2,0E0,3.3664182E-1,4.9317083E-1,1.8895161E-1,6.4384115E-1,3.2320388E-2,0E0,3.0194092E-1,1.8400311E-1,2.3363203E-2,1.5889248E-2,0E0,7.569477E-2,1.8012953E-1,2.522276E-2,0E0,0E0,8.122968E-3,0E0,2.9264748E-1,3.6548695E-1,2.4688584E-1,3.0360436E-1,2.1393472E-1,2.0935893E-1,1.20237276E-1,8.848014E-2,0E0,0E0,0E0,1.2985587E-2,0E0,0E0,0E0,6.677443E-2,1.8016791E-1,0E0,0E0,0E0,9.179689E-3,2.9501706E-2,7.6591915E-1,1.9239202E-1,2.2922003E-1,2.5420952E-1,2.1983768E-1,1.832023E-1,5.497317E-1,2.8526318E-1,1.9123457E-2,0E0,2.107834E-1,3.0220723E-1,1.9296408E-2,5.498743E-2,0E0,0E0,0E0,0E0,0E0,1.444852E-2,2.002666E-1,0E0,0E0,0E0,0E0,0E0,8.400628E-2,3.343078E-1,2.4991643E-1,0E0,1.6577357E-1,4.9781278E-2,2.558617E-1,2.1167326E-1,4.0625155E-1,1.2669508E-1,1.0375494E-1,4.340919E-2,8.8169955E-2,8.040491E-2,6.6234425E-2,0E0,0E0,0E0,2.3441046E-2,4.3323174E-2,1.8609917E-1,1.002607E-1,0E0,0E0,0E0,0E0,1.1445681E-1,3.8649464E-1,1.4536047E-1,7.32624E-2,2.0346957E-1,4.4640027E-2,2.791375E-1,1.5224276E-1,1.5637508E-1,4.4734716E-2,7.051706E-2,0E0,0E0,0E0,2.5551707E-1,4.3827444E-2,0E0,0E0,8.249663E-2,1.3797934E-1,9.6245563E-1,2.6375222E-1,0E0,0E0,0E0,8.442783E-2,0E0,0E0,2.0431347E-2,4.0638924E-2,1.2689869E-1,0E0,0E0,5.4888368E-2,2.984568E-1,1.777196E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,27,27,28,28,30,30,32,32,33,33,34,34,35,35,36,36,38,38,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,57,57,58,58,59,59,61,61,63,63,66,66,67,67,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,80,80,81,81,82,82,85,85,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,98,98,102,102,103,103,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,119,119,120,120,121,121,122,122,128,128,129,129,135,135,136,136,137,137,139,139,140,140,141,141,142,142,143,143,144,144,145,145,146,146,147,147,148,148,149,149,153,153,154,154,155,155,156,156,161,161,162,162,163,163,164,164,165,165,166,166,167,167,168,168,169,169,170,170,171,171,175,175,176,176,179,179,180,180,181,181,182,182,186,186,189,189,190,190,191,191,194,194,195,195,196,196],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,-1,48,50,52,-1,54,-1,56,58,60,62,64,-1,66,-1,-1,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,-1,96,98,100,-1,102,-1,104,-1,-1,106,108,-1,110,112,114,116,118,-1,120,122,124,126,-1,128,130,132,-1,-1,134,-1,136,138,140,142,144,146,148,150,-1,-1,-1,152,-1,-1,-1,154,156,-1,-1,-1,158,160,162,164,166,168,170,172,174,176,178,-1,180,182,184,186,-1,-1,-1,-1,-1,188,190,-1,-1,-1,-1,-1,192,194,196,-1,198,200,202,204,206,208,210,212,214,216,218,-1,-1,-1,220,222,224,226,-1,-1,-1,-1,228,230,232,234,236,238,240,242,244,246,248,-1,-1,-1,250,252,-1,-1,254,256,258,260,-1,-1,-1,262,-1,-1,264,266,268,-1,-1,270,272,274,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.185489E3,8.538E3,4.3702424E7,2.46E2,3.9512978E5,5.473125E3,9.357879E2,1.0908286E3,2.2337181E-2,2.0568174E7,1.4777102E9,8.15E2,7.576E3,5.4091E4,3.3795454E-6,9.3262803E8,2.41E3,1.3649979E10,5.684E3,1E0,3.6E1,1.2323549E-2,2E0,7.5179994E-1,2.107491E-2,1.1698891E6,1.70752E4,5.3039246E0,-1.7510926E-2,1.638058E2,1.0899014E-2,4.6577052E7,3.1294638E2,6.5884055E11,2.3292E4,1.6597747E8,-1.3326898E-3,6.7652373E3,-8.696972E-3,3.321503E-3,8.2E1,2.7460318E0,3.7934E4,5.5E1,4.5723195E6,2.285677E6,1.4E1,8.2619705E1,2.57636E6,1.5963264E9,1.3152658E7,1.886934E-2,3.6628513E10,1.6256282E7,6.7628815E2,-7.565236E-3,9.064853E-1,3.6512393E2,6.747114E7,-2.1528842E-2,5.684E3,6.982627E-3,9.361616E-1,-2.279824E-2,3.3591785E-3,1.0660479E2,2.645E3,-9.713503E-3,1.37E2,3.3831816E7,1.102E3,5.8036E4,2.1573659E2,-1.449324E-2,5.3038636E7,4.9766545E6,3E0,2.5629495E6,-3.7969763E-3,5.8018835E2,2.2576077E10,2.3033286E5,-2.0209341E-3,-9.430138E-3,4.3800272E8,3.1138062E-3,3.2703E4,4.92E2,9.544947E6,4.1401203E5,1.5497989E5,1E0,3.2343243E1,2.2500049E10,-2.0460617E-2,-6.940338E-3,1.0241525E-3,3.6E1,-4.1697943E-3,-1.3278485E-2,2.4979839E-3,1E0,1E0,-1.5877604E-2,1.2908854E-2,6.3718497E-3,1.48E2,5.0666668E1,3.79E2,2.108E3,2.2783158E5,1.645614E6,2.6973193E2,2.7523365E0,7.029851E0,1.2471935E7,6.285983E5,7.605729E-4,6.4547E4,1.4523809E0,1.3E1,1.77E2,-8.9298915E-3,-3.7490001E-3,-2.1325655E-3,1.8520064E-3,9.750316E-3,8.5E1,4.387838E-1,8.5261604E-4,4.7178343E-3,-3.280025E-4,-4.9049145E-4,-3.656383E-3,2.03148E2,2.3761498E9,1.0033929E1,-9.272972E-3,1.7440945E0,1.2062E5,1.287E3,1.03999E5,8.182648E6,3.6222222E0,7.1E1,5.0647113E2,2.0754387E1,5.5700205E8,2.1270284E7,-3.610044E-3,-5.7461294E-3,-1.1471425E-2,3.6E1,2.1178565E6,1.2E1,1.25E2,-4.540667E-3,-1.3962894E-3,-8.523861E-3,-3.8995463E-3,7.056912E7,1.198E5,5E0,2.6944466E7,3.1494663E0,1.4E1,6.520433E6,2.4E1,7.130317E7,1E0,4.877E4,-5.4179993E-3,-2.2466788E-2,-1.5719788E-3,1.9826213E0,6.23E2,-1.4864013E-3,-6.2734997E-3,1E0,9.5822406E2,2.7487153E5,1.0918E4,-1.4684912E-3,2.949E-3,4.164253E-3,9.309785E8,9.0248394E-4,4.1797087E-3,5.783955E4,1.5846036E7,1.6762007E7,8.719756E-3,2.1199819E-2,1.4E1,8.61491E3,1.71E2,-6.8429136E-3,-2.876562E-3,8.000297E-3,4.1225544E-4,-8.454456E-3,-1.718895E-3,-1.7774956E-2,-6.7910817E-3,-1.1351949E-3,-1.799334E-2,2.5019445E-3,-1.0465444E-3,-5.492545E-3,-1.2559873E-2,-1.623073E-3,5.0600288E-3,1.072891E-3,-3.2252287E-3,9.78461E-4,-6.228521E-3,3.2375657E-3,1.8680091E-4,-3.6330845E-3,-9.409071E-3,-3.4145506E-3,1.6467914E-3,-2.3205052E-3,-1.0924878E-2,1.5450874E-3,-4.964422E-3,1.1158584E-3,-5.5088536E-3,-3.8527537E-3,1.1845444E-2,-6.8773335E-4,-1.18094515E-2,2.3134323E-3,-2.3637882E-3,-6.78392E-3,-1.8959742E-2,-4.064754E-3,2.6292356E-3,-2.1705434E-3,6.775109E-3,2.238922E-3,-6.365526E-4,-2.3701044E-3,3.5250538E-3,5.843547E-3,9.050787E-4,6.6699735E-3,2.6428541E-3,3.4735957E-3,-7.0444304E-3,3.20226E-3,1.0820619E-2,5.55011E-3,1.0076253E-3,-2.4953943E-3,5.695333E-3,3.8778908E-3,-1.810735E-2,1.2559995E-2,4.8094844E-3,1.1839699E-2,6.2365085E-3,6.4647365E-3,1.3964992E-3,1.4401708E-2,8.461165E-3,4.024021E-3,-4.4790185E-3,2.1683364E-3,8.4960265E-3,2.6329039E-3,8.829133E-3,9.878981E-4,-9.507037E-3],"split_indices":[20,52,2,45,10,28,32,55,4,0,47,7,2,9,33,42,31,2,5,9,102,2,0,6,27,0,47,33,39,0,52,0,48,52,31,2,45,0,52,0,0,29,58,1,0,28,32,8,56,1,7,45,0,31,45,52,0,27,52,45,0,9,0,27,0,0,56,9,0,0,7,29,1,55,0,7,45,8,48,0,52,12,33,0,0,7,0,2,3,9,33,28,105,56,5,0,0,0,8,0,0,0,64,101,0,0,0,1,50,2,2,28,9,52,53,58,45,28,0,1,54,8,0,0,0,0,0,0,8,42,0,0,0,0,0,56,12,53,0,54,1,2,11,45,58,8,4,58,5,51,0,0,0,3,28,18,8,0,0,0,0,7,7,8,45,46,3,1,8,7,16,12,0,0,0,56,2,0,0,16,55,33,9,0,0,0,5,0,0,43,47,51,0,0,8,4,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.538E3,8.33E2,1.705E3,7.95E2,3.8E1,1.371E3,3.34E2,6.15E2,1.8E2,4E0,3.4E1,7.3E2,6.41E2,9.7E1,2.37E2,4.3E2,1.85E2,4.5E1,1.35E2,2.2E1,1.2E1,7.24E2,6E0,2.07E2,4.34E2,7E0,9E1,2.2E1,2.15E2,5E0,4.25E2,4E0,1.81E2,2.4E1,2.1E1,7.6E1,5.9E1,4E0,1.8E1,7E0,5E0,3.7E1,6.87E2,1.86E2,2.1E1,4.12E2,2.2E1,2.3E1,6.7E1,1E1,1.2E1,2.11E2,4E0,2.29E2,1.96E2,1.77E2,4E0,9E0,1.5E1,1E1,1.1E1,7.2E1,4E0,5.5E1,4E0,7E0,1.1E1,3.1E1,6E0,1.24E2,5.63E2,1.43E2,4.3E1,1.4E1,7E0,3.58E2,5.4E1,1.4E1,8E0,4E0,1.9E1,5.8E1,9E0,4E0,6E0,8E0,4E0,9.1E1,1.2E2,1.47E2,8.2E1,1.39E2,5.7E1,8.9E1,8.8E1,5E0,4E0,4E0,1.1E1,4E0,6E0,4E0,6.8E1,4.9E1,6E0,6E0,5E0,1.2E1,1.9E1,7.5E1,4.9E1,2.6E1,5.37E2,9.2E1,5.1E1,1.2E1,3.1E1,9E0,5E0,8.2E1,2.76E2,8E0,4.6E1,8E0,6E0,4E0,4E0,5E0,1.4E1,5.4E1,4E0,5E0,4E0,4E0,4E0,5.9E1,3.2E1,1.15E2,5E0,1.39E2,8E0,6.1E1,2.1E1,1.7E1,1.22E2,4.9E1,8E0,7.1E1,1.8E1,8.2E1,6E0,7E0,4E0,1.1E1,5.7E1,2.7E1,2.2E1,7E0,5E0,8E0,1.1E1,4.1E1,3.4E1,1.1E1,3.8E1,1.8E1,8E0,3.29E2,2.08E2,7E1,2.2E1,4.7E1,4E0,6E0,6E0,2.2E1,9E0,5E0,4E0,4.1E1,4.1E1,3.3E1,2.43E2,4E0,4E0,7E0,3.9E1,7E0,7E0,9E0,4.5E1,5.1E1,8E0,5E0,2.7E1,9.1E1,2.4E1,3.6E1,1.03E2,4E0,4E0,3.8E1,2.3E1,1.1E1,1E1,1.3E1,4E0,3.4E1,8.8E1,4.3E1,6E0,4E0,4E0,5.7E1,1.4E1,5E0,1.3E1,5.6E1,2.6E1,5E0,6E0,5.3E1,4E0,1.1E1,1.6E1,1.2E1,1E1,3.4E1,7E0,4E0,3E1,6E0,5E0,2.7E1,1.1E1,1.3E1,5E0,4E0,4E0,3.21E2,8E0,6.7E1,1.41E2,5.7E1,1.3E1,1.5E1,7E0,2.4E1,2.3E1,9E0,1.3E1,5E0,4E0,2.3E1,1.8E1,3.6E1,5E0,2.8E1,5E0,1.1E1,2.32E2,2.4E1,1.5E1,5E0,4E0,3.6E1,9E0,4.7E1,4E0,5E0,2.2E1,6.3E1,2.8E1,2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"275","size_leaf_vector":"1"}},{"base_weights":[-4.053314E-3,2.026082E-2,-7.210556E-2,3.618337E-4,1.1111053E-1,-1.5450147E-1,-3.5371423E-2,-2.2798644E-2,4.8129614E-2,5.2198753E-2,1.5502805E-1,-1.4446029E-1,-1.8123075E-2,-5.7581503E-2,4.8047993E-2,-5.1520698E-2,2.5398158E-3,2.5459876E-2,1.0877963E-1,7.034742E-2,-1.0192449E-1,2.2442782E-2,1.4175797E-1,-1.24114625E-1,-2.569716E-1,-2.719555E-2,-1.3944456E-1,6.5144025E-2,-7.829747E-3,-3.919439E-2,-1.7098415E-1,2.672122E-2,-4.7686815E-2,2.984308E-2,-1.5189327E-2,1.23406604E-1,-9.227485E-3,3.5205077E-2,1.4834476E-1,1.2379672E-2,-1.7781615E-2,2.1260212E-1,1.1879422E-1,-9.967781E-2,-2.3727417E-1,-3.2716173E-1,-5.6843292E-2,-1.1001866E-1,-1.0439835E-2,2.9470898E-2,-1.5628923E-1,1.3365781E-1,3.0396862E-2,-5.407093E-2,2.3498205E-2,-2.3000244E-2,-1.1607721E-1,4.7374662E-2,-1.4796067E-2,-1.245193E-1,-2.3198843E-2,-1.223865E-2,5.084831E-2,1.3652547E-1,-5.1607884E-2,1.06413495E-2,2.4822779E-2,1.8907344E-1,-9.129733E-3,-2.6797184E-3,4.36348E-3,2.4338114E-1,1.0767805E-1,8.106355E-2,1.8674752E-1,-1.6479585E-1,-5.512007E-2,-2.795204E-1,-1.0844063E-3,-5.6050527E-3,-3.6870438E-1,-7.955671E-3,2.843186E-3,-1.2872691E-2,-1.6140077E-1,-2.7497998E-2,3.6990814E-2,-4.783068E-4,3.5190934E-3,-1.2347249E-1,-2.3929924E-1,1.6639061E-1,-1.1687262E-4,4.2779515E-3,1.1369218E-1,-3.6825273E-2,-9.460856E-2,7.375553E-2,-2.0843253E-2,-1.8025747E-2,-7.886975E-2,2.8082784E-2,1.2361326E-1,-5.4807577E-2,3.530241E-2,-9.0176195E-2,-1.8705536E-2,6.0849372E-2,-4.0241223E-2,-2.077038E-3,-1.0579818E-2,7.632479E-2,-2.1660069E-2,1.6772433E-1,6.1503712E-2,1.3784675E-3,-6.8115094E-3,-8.38541E-2,3.6591534E-2,1.2496632E-1,2.027852E-2,7.2563863E-3,2.592281E-1,2.0382078E-3,7.4763587E-3,9.844469E-2,-2.6530506E-2,2.9442024E-1,1.1361621E-1,-1.5024118E-2,-1.4333421E-1,3.886839E-2,-1.0038355E-1,-2.9778518E-3,-3.617353E-1,-6.024506E-3,-2.1535013E-2,-3.9116773E-3,2.0906553E-2,-1.1023189E-1,-2.5271532E-1,-2.0095164E-2,-9.790968E-3,1.2380042E-1,-2.2271725E-3,-1.08974755E-1,-1.2747494E-2,-3.0621553E-3,-2.8073886E-1,3.3403216E-3,2.0930037E-1,4.710277E-2,-4.2293854E-2,1.5513912E-1,1.2824067E-3,-6.562841E-2,6.179615E-3,-1.6749825E-1,-6.5731354E-2,2.4233377E-2,1.07824E-1,-3.5938239E-3,-5.940938E-3,-1.5999159E-2,-1.8887164E-1,3.3115514E-2,-8.936185E-3,1.6868581E-1,2.3545472E-2,-6.696139E-2,4.3980163E-3,-5.715225E-2,6.122704E-2,2.4839062E-3,-1.07100114E-1,-3.3762418E-3,9.8200224E-2,-4.7731183E-2,4.2110356E-3,7.674881E-2,-4.44951E-2,6.57439E-2,1.9044787E-1,-6.0683824E-3,-1.3085408E-3,-3.161458E-3,1.7961895E-1,8.893291E-3,3.4101557E-2,-8.56062E-3,1.0187825E-3,6.0629793E-2,-2.6662763E-2,-9.84122E-4,1.4797953E-1,1.8733779E-1,3.0143726E-1,1.26352E-1,3.6916785E-2,-7.451426E-3,4.2001256E-3,1.766638E-2,8.070125E-3,6.481162E-2,1.2588031E-2,-3.949859E-2,-1.843622E-1,6.1904892E-2,-2.985792E-3,-1.2609287E-1,-1.656336E-2,-1.9906132E-2,-9.331776E-3,3.4084485E-3,-2.9464038E-3,-8.108659E-3,-7.261563E-2,-5.9808274E-3,-1.8077692E-2,6.574808E-3,-2.4738561E-2,1.7406687E-1,1.6451869E-3,1.5653908E-2,-4.117935E-3,-1.2798387E-1,-4.5136463E-2,-1.5523442E-2,-6.435325E-3,1.4900788E-2,7.944829E-3,-1.6898585E-3,9.193349E-2,-6.8626017E-3,-1.0052151E-2,4.080652E-3,1.0606787E-2,-2.6790402E-3,-7.1665714E-3,1.244115E-2,-3.0093172E-4,-1.4308069E-2,-4.6845553E-3,-7.1733044E-3,-2.4807882E-3,-2.16788E-3,2.3310643E-3,8.189773E-3,1.3314284E-3,6.4870866E-4,-6.2729293E-3,-2.7913628E-3,7.3282127E-3,-1.2650966E-2,-3.5794987E-3,3.8447592E-3,8.1830553E-4,8.8662916E-4,9.656119E-3,-4.529489E-3,3.2805966E-3,-4.857393E-3,-1.9512803E-3,-1.4487756E-3,-4.817043E-3,1.7461226E-3,6.9595184E-3,-2.2324203E-3,-6.027322E-3,6.9188587E-3,1.2752147E-3,-5.4969685E-3,-1.7235382E-3,-3.550204E-3,5.413334E-3,-1.1935035E-2,-1.3945888E-3,3.9059587E-3,-6.293497E-4,1.280018E-2,3.93415E-3,2.4431269E-3,-3.6617077E-3,7.5238906E-3,1.379694E-2,-4.6024485E-3,3.336279E-3,5.536021E-3,1.602108E-3,2.2714771E-3,-4.2790757E-3,8.08957E-3,2.8549163E-3,1.1522359E-2,5.861696E-3,1.663522E-2,8.273277E-3,6.9387006E-3,-1.4525391E-3,-2.56973E-3,3.5181139E-3,6.6340705E-3,-9.9304E-4,-4.199324E-3,1.4879195E-3,-2.3514226E-3,-1.0734377E-2,4.4881884E-4,6.750406E-3,-7.908116E-3,-1.7347686E-3,6.035475E-3,-3.8719648E-3,-1.0047487E-3,-5.4704766E-3,-4.1754945E-4,-2.865333E-3,1.0500142E-2,4.0738713E-3,-1.1860946E-4,3.4977016E-3,-9.535982E-3,-5.083021E-3,8.920327E-4,-3.7670266E-3,7.816179E-3,1.556194E-3,-1.7225081E-3,2.4286562E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,-1,53,55,57,59,61,-1,63,-1,65,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,95,97,-1,99,101,103,105,107,109,111,113,115,-1,117,119,-1,-1,-1,121,123,125,127,129,131,133,-1,-1,135,-1,-1,137,139,141,143,-1,-1,145,147,149,-1,151,153,155,157,159,161,-1,163,165,167,169,171,173,-1,175,177,179,-1,181,183,185,187,-1,-1,189,191,193,-1,-1,195,-1,-1,197,199,201,203,-1,205,207,209,-1,211,-1,-1,-1,213,215,217,219,-1,221,223,225,-1,-1,227,-1,229,231,233,235,-1,237,239,241,243,245,247,249,-1,251,253,255,-1,257,259,261,-1,263,265,-1,267,-1,269,271,-1,273,275,277,279,-1,281,-1,283,-1,285,-1,-1,287,289,-1,291,293,295,297,299,-1,-1,-1,-1,301,-1,303,305,307,-1,309,311,-1,-1,-1,-1,-1,313,-1,-1,-1,315,317,-1,319,-1,321,323,-1,-1,-1,-1,-1,325,-1,327,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2869563E0,3.4510827E0,2.058773E0,1.7347081E0,8.793373E-1,3.8662815E-1,8.791183E-1,7.6865876E-1,7.011895E-1,4.2114562E-1,7.01334E-1,4.3050766E-1,0E0,9.2652786E-1,3.6024487E-1,7.222073E-1,6.838516E-1,5.491439E-1,6.1942255E-1,3.593318E-1,4.6883726E-1,0E0,2.822187E-1,4.5473886E-1,4.0281403E-1,3.7909782E-1,2.919842E-1,2.1733856E-1,0E0,4.209625E-1,6.773288E-1,3.2654142E-1,3.4142068E-1,3.2767877E-1,0E0,3.124125E-1,0E0,1.6635409E-1,5.873781E-1,6.405114E-2,0E0,1.1911869E-1,3.5663056E-1,4.087342E-1,2.671461E-1,1.5512419E-1,1.1386293E-1,2.275343E-1,1.8629642E-1,1.6665554E-2,2.2107172E-1,1.3838744E-1,1.3657536E-1,2.509879E-1,1.9541535E-1,0E0,3.3570153E-1,3.701082E-1,2.5650835E-1,3.437755E-1,2.0298207E-1,2.5093672E-1,4.5792603E-1,2.7991772E-1,7.084992E-2,0E0,1.168986E-1,4.6344817E-1,0E0,0E0,0E0,1.5240192E-2,2.7566373E-2,1.8027198E-1,3.6813557E-1,1.2605047E-1,3.7402105E-1,4.304669E-1,0E0,0E0,2.2621775E-1,0E0,0E0,3.9199937E-2,1.1231661E-1,2.0680912E-1,2.079501E-1,0E0,0E0,1.0198259E-1,1.6721737E-1,9.278864E-2,0E0,9.971919E-2,5.074504E-2,3.1874517E-1,2.1631384E-1,6.6559955E-2,7.9067975E-2,0E0,2.4595E-1,2.1836828E-1,2.2605401E-1,1.2914193E-1,1.4011985E-1,1.0002184E-1,0E0,1.21753015E-1,1.13141045E-1,4.0130806E-1,0E0,2.0845604E-1,1.318463E-1,2.5040674E-1,1.1521976E-1,0E0,0E0,8.5559085E-2,1.24778934E-1,1.0278857E-1,0E0,0E0,2.8493285E-2,0E0,0E0,1.3613582E-1,2.0234655E-1,1.0011029E-1,1.9968101E-1,0E0,2.1424663E-1,6.958583E-2,1.2505412E-1,0E0,4.103732E-2,0E0,0E0,0E0,4.9366444E-2,2.9331863E-2,1.017912E-1,1.1805707E-1,0E0,8.105978E-2,6.294438E-2,7.067442E-2,0E0,0E0,5.0964117E-2,0E0,2.8610349E-2,9.7432286E-2,7.2168976E-2,2.4644181E-2,0E0,1.3622391E-1,3.096333E-1,2.3609889E-1,9.175721E-2,2.9311568E-2,1.0489261E-1,8.451129E-2,0E0,1.6382223E-1,7.988313E-2,1.4547265E-1,0E0,1.2634867E-1,8.77014E-2,5.1927865E-2,0E0,1.1573117E-2,8.744642E-2,0E0,2.5007725E-2,0E0,5.2073866E-2,8.7718844E-2,0E0,2.0379665E-1,2.4476516E-1,1.65515E-1,8.996695E-2,0E0,2.0231923E-1,0E0,1.9103336E-1,0E0,1.3809133E-1,0E0,0E0,8.099522E-2,1.01723425E-1,0E0,2.3627162E-2,1.2331575E-2,1.332581E-2,1.12959385E-1,8.3511665E-2,0E0,0E0,0E0,0E0,1.4651605E-1,0E0,5.1876426E-2,1.4467609E-1,9.175228E-2,0E0,1.268037E-1,1.348893E-1,0E0,0E0,0E0,0E0,0E0,2.4899386E-2,0E0,0E0,0E0,8.4451765E-2,2.29263E-2,0E0,3.5437044E-2,0E0,5.9809804E-2,3.1836964E-2,0E0,0E0,0E0,0E0,0E0,6.1777264E-2,0E0,2.8892681E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,35,35,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,66,66,67,67,71,71,72,72,73,73,74,74,75,75,76,76,77,77,80,80,83,83,84,84,85,85,86,86,89,89,90,90,91,91,93,93,94,94,95,95,96,96,97,97,98,98,100,100,101,101,102,102,103,103,104,104,105,105,107,107,108,108,109,109,111,111,112,112,113,113,114,114,117,117,118,118,119,119,122,122,125,125,126,126,127,127,128,128,130,130,131,131,132,132,134,134,138,138,139,139,140,140,141,141,143,143,144,144,145,145,148,148,150,150,151,151,152,152,153,153,155,155,156,156,157,157,158,158,159,159,160,160,161,161,163,163,164,164,165,165,167,167,168,168,169,169,171,171,172,172,174,174,176,176,177,177,179,179,180,180,181,181,182,182,184,184,186,186,188,188,191,191,192,192,194,194,195,195,196,196,197,197,198,198,203,203,205,205,206,206,207,207,209,209,210,210,216,216,220,220,221,221,223,223,225,225,226,226,232,232,234,234],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,-1,54,56,58,60,62,-1,64,-1,66,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,96,98,-1,100,102,104,106,108,110,112,114,116,-1,118,120,-1,-1,-1,122,124,126,128,130,132,134,-1,-1,136,-1,-1,138,140,142,144,-1,-1,146,148,150,-1,152,154,156,158,160,162,-1,164,166,168,170,172,174,-1,176,178,180,-1,182,184,186,188,-1,-1,190,192,194,-1,-1,196,-1,-1,198,200,202,204,-1,206,208,210,-1,212,-1,-1,-1,214,216,218,220,-1,222,224,226,-1,-1,228,-1,230,232,234,236,-1,238,240,242,244,246,248,250,-1,252,254,256,-1,258,260,262,-1,264,266,-1,268,-1,270,272,-1,274,276,278,280,-1,282,-1,284,-1,286,-1,-1,288,290,-1,292,294,296,298,300,-1,-1,-1,-1,302,-1,304,306,308,-1,310,312,-1,-1,-1,-1,-1,314,-1,-1,-1,316,318,-1,320,-1,322,324,-1,-1,-1,-1,-1,326,-1,328,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,1.3508893E3,2.0827537E0,6.978874E5,1.3339816E5,4.5723195E6,3.0497742E3,2.6659616E2,2.539E3,2.3580047E10,6.737099E7,1.2199979E10,-1.8123075E-2,5.097268E7,1.5265896E8,4.602015E0,1.3152658E7,2.507E3,1.8648378E7,1.6195753E9,1.3674345E4,2.2442782E-2,2.2579439E0,1.8738E4,9.099893E8,1.16146E5,2.7487562E0,9.160584E-1,-7.829747E-3,4.264897E6,1E0,1.4383178E1,6.203E3,2.5644116E2,-1.5189327E-2,1E0,-9.227485E-3,1.5E1,3.7826266E4,4.8E1,-1.7781615E-2,2.3977574E7,4.9807886E3,1.048E3,4.70092E1,3.4526074E-1,9.9625344E5,1.0376E4,1.0485785E3,8.8070023E-1,9.9625344E5,1.515252E10,9.609747E3,2.5369E4,1E0,-2.3000244E-2,5.1212654E0,7.298614E2,4.832E3,7.2530017E0,5.1547173E1,3.4E1,2E0,1.845406E1,3.131E3,1.06413495E-2,1.2542373E0,4.42384E5,-9.129733E-3,-2.6797184E-3,4.36348E-3,1E0,9.170864E1,2.3058404E7,1.1297775E2,3.409091E-2,1.25218E5,4.854E4,-1.0844063E-3,-5.6050527E-3,2.0469315E0,-7.955671E-3,2.843186E-3,6.825774E3,2.299E2,7.362237E6,7.667683E6,-4.783068E-4,3.5190934E-3,9.60094E5,7.549744E7,7.6309943E-1,-1.1687262E-4,7.6842415E6,2.560372E7,1.5409575E2,1.06E3,5.08E2,6.1161733E0,-1.8025747E-2,7.207554E4,1E0,7.2201815E6,7.06699E8,1.4343751E6,1.0013907E1,-1.8705536E-2,1.6007329E7,7.5935423E-1,5.088781E0,-1.0579818E-2,1.945711E1,1.01813E5,7.549744E7,7.0277144E7,1.3784675E-3,-6.8115094E-3,2E0,6.744351E1,1.4991512E5,2.027852E-2,7.2563863E-3,1.6503105E3,2.0382078E-3,7.4763587E-3,8.606353E1,6.65E2,2.330807E0,4.6577052E7,-1.5024118E-2,1.9024893E8,4.8052647E5,5.121E3,-2.9778518E-3,6.99143E5,-6.024506E-3,-2.1535013E-2,-3.9116773E-3,2.8880222E6,4.5555557E1,1.27E2,2.5853658E0,-9.790968E-3,7.6247287E-1,1.2679E4,1.4972717E-2,-1.2747494E-2,-3.0621553E-3,1.3715873E3,3.3403216E-3,6.473E3,1.7945665E9,6.1392635E-1,8.06E2,1.2824067E-3,2.1358025E0,6E0,9.251576E3,3.56E2,7.805608E4,1.2561812E6,2.60656E5,-5.940938E-3,2.3770695E5,5.492982E2,1.4343751E6,-8.936185E-3,2.41E3,1E0,1.3097832E3,4.3980163E-3,9.433962E-3,8.048E3,2.4839062E-3,4.91271E5,-3.3762418E-3,2.0237808E8,7.16173E5,4.2110356E-3,9.8376294E5,6.387417E-4,1E0,1.2867613E7,-6.0683824E-3,2.0992E4,-3.161458E-3,4.94374E1,8.893291E-3,6.783E3,-8.56062E-3,1.0187825E-3,1.1851E4,1E0,-9.84122E-4,1.01268125E5,5.685916E8,1.5277338E0,5.7249427E0,8.631E3,-7.451426E-3,4.2001256E-3,1.766638E-2,8.070125E-3,2.6284674E2,1.2588031E-2,1.5998206E-3,1E0,1.40673E5,-2.985792E-3,1.6667E4,3.0491832E6,-1.9906132E-2,-9.331776E-3,3.4084485E-3,-2.9464038E-3,-8.108659E-3,5.2921E4,-5.9808274E-3,-1.8077692E-2,6.574808E-3,1.6256282E7,5.961E3,1.6451869E-3,2.7306658E3,-4.117935E-3,3.6282136E0,5.198287E5,-1.5523442E-2,-6.435325E-3,1.4900788E-2,7.944829E-3,-1.6898585E-3,2.6621832E7,-6.8626017E-3,2.5503985E9,4.080652E-3,1.0606787E-2,-2.6790402E-3,-7.1665714E-3,1.244115E-2,-3.0093172E-4,-1.4308069E-2,-4.6845553E-3,-7.1733044E-3,-2.4807882E-3,-2.16788E-3,2.3310643E-3,8.189773E-3,1.3314284E-3,6.4870866E-4,-6.2729293E-3,-2.7913628E-3,7.3282127E-3,-1.2650966E-2,-3.5794987E-3,3.8447592E-3,8.1830553E-4,8.8662916E-4,9.656119E-3,-4.529489E-3,3.2805966E-3,-4.857393E-3,-1.9512803E-3,-1.4487756E-3,-4.817043E-3,1.7461226E-3,6.9595184E-3,-2.2324203E-3,-6.027322E-3,6.9188587E-3,1.2752147E-3,-5.4969685E-3,-1.7235382E-3,-3.550204E-3,5.413334E-3,-1.1935035E-2,-1.3945888E-3,3.9059587E-3,-6.293497E-4,1.280018E-2,3.93415E-3,2.4431269E-3,-3.6617077E-3,7.5238906E-3,1.379694E-2,-4.6024485E-3,3.336279E-3,5.536021E-3,1.602108E-3,2.2714771E-3,-4.2790757E-3,8.08957E-3,2.8549163E-3,1.1522359E-2,5.861696E-3,1.663522E-2,8.273277E-3,6.9387006E-3,-1.4525391E-3,-2.56973E-3,3.5181139E-3,6.6340705E-3,-9.9304E-4,-4.199324E-3,1.4879195E-3,-2.3514226E-3,-1.0734377E-2,4.4881884E-4,6.750406E-3,-7.908116E-3,-1.7347686E-3,6.035475E-3,-3.8719648E-3,-1.0047487E-3,-5.4704766E-3,-4.1754945E-4,-2.865333E-3,1.0500142E-2,4.0738713E-3,-1.1860946E-4,3.4977016E-3,-9.535982E-3,-5.083021E-3,8.920327E-4,-3.7670266E-3,7.816179E-3,1.556194E-3,-1.7225081E-3,2.4286562E-3],"split_indices":[27,52,53,28,28,28,52,52,2,5,7,5,0,45,45,53,45,2,50,7,4,0,54,9,7,1,54,57,0,9,64,53,29,55,0,104,0,0,52,3,0,47,4,2,58,27,28,9,52,27,28,5,52,29,16,0,53,52,2,53,48,3,17,56,2,0,53,2,0,0,0,102,58,45,58,58,29,29,0,0,56,0,0,48,52,47,45,0,0,9,7,27,0,47,1,55,2,2,54,0,33,68,51,7,45,56,0,45,42,58,0,56,1,7,12,0,0,6,54,33,0,0,52,0,0,58,8,41,48,0,7,47,2,0,1,0,0,0,32,52,0,54,0,57,2,38,0,0,52,0,9,12,42,0,0,56,3,33,0,28,45,1,0,28,55,45,0,2,16,4,0,57,2,0,31,0,7,1,0,28,38,6,12,0,29,0,58,0,10,0,0,10,64,0,28,7,53,41,2,0,0,0,0,58,0,39,104,1,0,9,47,0,0,0,0,0,1,0,0,0,45,10,0,55,0,53,32,0,0,0,0,0,48,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.589E3,1.908E3,6.81E2,1.566E3,3.42E2,2.09E2,4.72E2,1.055E3,5.11E2,1.47E2,1.95E2,2.01E2,8E0,3.73E2,9.9E1,4.94E2,5.61E2,3.73E2,1.38E2,1.32E2,1.5E1,7E0,1.88E2,1.72E2,2.9E1,2.73E2,1E2,9.2E1,7E0,4.49E2,4.5E1,3.79E2,1.82E2,3.69E2,4E0,1.32E2,6E0,9.2E1,4E1,1.1E1,4E0,4.4E1,1.44E2,1.43E2,2.9E1,2.1E1,8E0,4.5E1,2.28E2,9E0,9.1E1,3E1,6.2E1,3.63E2,8.6E1,6E0,3.9E1,2.53E2,1.26E2,4.3E1,1.39E2,1.23E2,2.46E2,1.23E2,9E0,4E0,8.8E1,3.6E1,4E0,6E0,5E0,3.3E1,1.1E1,9.4E1,5E1,5.7E1,8.6E1,2.4E1,5E0,4E0,1.7E1,4E0,4E0,1.6E1,2.9E1,1.68E2,6E1,5E0,4E0,6.7E1,2.4E1,2.4E1,6E0,4.8E1,1.4E1,2.56E2,1.07E2,4E1,4.6E1,4E0,3.5E1,2.03E2,5E1,7E1,5.6E1,3.9E1,4E0,2.3E1,1.16E2,1.18E2,5E0,1.82E2,6.4E1,8.6E1,3.7E1,5E0,4E0,8E0,8E1,2.9E1,7E0,6E0,2.7E1,5E0,6E0,8.1E1,1.3E1,1.9E1,3.1E1,6E0,5.1E1,2.8E1,5.8E1,7E0,1.7E1,4E0,1.3E1,5E0,1.1E1,2E1,9E0,1.62E2,6E0,1.8E1,4.2E1,6.2E1,5E0,5E0,1.9E1,8E0,1.6E1,2.5E1,2.3E1,9E0,5E0,1.53E2,1.03E2,2.9E1,7.8E1,1.7E1,2.3E1,4E1,6E0,2.3E1,1.2E1,1.99E2,4E0,3.4E1,1.6E1,6.5E1,5E0,1.2E1,4.4E1,4E0,3.5E1,5E0,1.8E1,1.1E2,6E0,4.1E1,7.7E1,1.68E2,1.4E1,1E1,5.4E1,4E0,8.2E1,6E0,3.1E1,4E0,4E0,5.8E1,2.2E1,4E0,2.5E1,1.2E1,1.5E1,5.5E1,2.6E1,6E0,7E0,1.2E1,7E0,2.4E1,7E0,1.5E1,3.6E1,2.3E1,5E0,4.4E1,1.4E1,1.3E1,4E0,7E0,4E0,7E0,1.3E1,5E0,4E0,4E0,1.58E2,1.1E1,7E0,3.5E1,7E0,4.7E1,1.5E1,1.5E1,4E0,4E0,1.2E1,9E0,1.6E1,5E0,1.8E1,5E0,4E0,1.34E2,1.9E1,4E0,9.9E1,1E1,1.9E1,1.2E1,6.6E1,4E0,1.3E1,1.3E1,1E1,3.6E1,4E0,1.9E1,4E0,7E0,5E0,5.4E1,1.45E2,5E0,2.9E1,4E0,1.2E1,3E1,3.5E1,8E0,4E0,3.4E1,1E1,7E0,2.8E1,1.1E1,7E0,1.8E1,9.2E1,7E0,3.4E1,5E0,7.2E1,1.45E2,2.3E1,8E0,6E0,3.2E1,2.2E1,6.5E1,1.7E1,6E0,2.5E1,2E1,3.8E1,1E1,1.2E1,2.1E1,4E0,6E0,6E0,1.1E1,4E0,5.1E1,4E0,7E0,1.9E1,1.3E1,1.1E1,9E0,6E0,7E0,2.9E1,1.4E1,9E0,3.2E1,1.2E1,4E0,1E1,6E0,7E0,1.06E2,5.2E1,7E0,4E0,2.7E1,8E0,1.2E1,3.5E1,5E0,1E1,7E0,9E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"329","size_leaf_vector":"1"}},{"base_weights":[-1.3622434E-3,-6.547659E-2,2.8361622E-2,-5.0282557E-2,-1.5569156E-1,1.3287424E-2,1.1839517E-1,-8.150237E-2,9.386231E-3,-3.3166522E-1,-1.2855276E-1,-2.3135744E-2,3.7541438E-2,2.1984559E-2,1.0876297E-1,-6.918373E-2,-1.880151E-1,1.44938715E-2,9.854727E-4,-1.7845547E-1,-2.404738E-2,-1.1426106E-1,-1.9278038E-2,-4.6990134E-2,2.0445324E-2,8.855934E-3,7.0352435E-2,1.3764066E-1,2.7920192E-2,-1.1048753E-1,-4.289321E-2,-2.569877E-1,-1.1120856E-1,1.1705207E-2,-9.833143E-2,-1.1712047E-2,-4.3382384E-3,-1.7469797E-1,-4.367772E-2,-5.623597E-2,3.0957168E-2,1.0670985E-2,1.1908935E-2,1.2495006E-2,-1.7101903E-2,1.1401268E-2,8.838576E-2,1.1493082E-1,2.0785056E-1,6.0141765E-2,-1.585348E-1,-7.418699E-2,-1.8661639E-1,-1.6482876E-1,-3.6165707E-2,-7.0438827E-3,-2.9646656E-1,-1.343526E-2,-1.9290633E-1,-1.4156115E-4,1.1456312E-1,1.2057995E-3,-1.2538333E-1,-4.3083563E-2,-1.9713867E-1,-8.2803145E-3,-8.3525665E-2,-4.515181E-2,-1.11069396E-1,1.08125985E-1,-1.148888E-2,-3.8819257E-2,3.2051194E-2,5.8263645E-2,-1.1886705E-2,6.735335E-2,-2.7022677E-2,1.2957752E-1,4.9131323E-2,1.6123001E-1,6.83171E-2,1.6265948E-1,2.2955636E-2,-1.1047206E-3,1.3858145E-1,-1.3559392E-3,-1.4215783E-2,-8.390386E-2,5.407527E-3,-2.5833836E-1,-5.222474E-2,-2.1263317E-3,-1.5009437E-2,-2.999244E-2,-1.5198725E-1,-6.3170744E-3,-3.3906177E-1,4.0671406E-3,-4.6369466E-3,-1.2306019E-2,-4.476229E-3,-1.5588922E-2,4.4040725E-2,1.22038135E-2,6.2056124E-2,-1.5615858E-1,-2.5141926E-3,2.1450568E-3,-6.0199928E-3,-1.4316198E-1,-2.634667E-1,2.4180463E-2,-3.6116824E-3,-1.3569309E-1,-1.9022508E-2,-4.8783567E-2,8.64762E-3,-1.2429392E-1,2.6966678E-3,1.3616338E-3,1.5901487E-1,-7.3617278E-3,1.7290635E-2,4.108149E-3,-5.4072846E-2,6.440701E-2,-1.1749878E-3,6.6774964E-2,-1.1535207E-2,-1.527055E-1,-4.806142E-3,3.8239673E-2,1.3457218E-2,-1.1670504E-2,-8.09878E-3,1.6537242E-2,1.165435E-1,5.928248E-2,-8.692804E-2,1.7764258E-1,-4.2161653E-3,7.823549E-2,-2.6068275E-3,2.2122645E-1,5.8108367E-2,6.853535E-3,-3.691077E-2,-1.680098E-3,1.9473195E-1,1.9815236E-2,-1.0007839E-1,-5.1188545E-4,-2.9220492E-1,2.184821E-3,-8.072653E-2,-2.8032202E-3,-5.223556E-2,-1.1533574E-2,-3.1328737E-3,-8.622864E-3,-1.9680988E-2,4.3727387E-2,-2.7919333E-2,9.898594E-3,1.1597408E-1,1.6360447E-2,5.5672214E-3,-9.889187E-3,-4.601585E-3,-1.06412224E-1,-1.416947E-2,-4.761176E-3,-1.5825672E-2,-1.4965168E-2,3.247278E-3,-8.8821683E-4,-9.615062E-3,-2.5723213E-3,8.29754E-4,-1.5523177E-1,-4.3592952E-2,-1.735127E-1,-6.4615525E-2,3.2367664E-3,1.1065031E-2,5.5870242E-2,-4.2307843E-2,-8.186858E-3,-3.9224632E-2,3.864994E-2,1.2879902E-1,-1.9190056E-2,6.5489346E-3,-1.1788708E-2,8.6808726E-2,-2.4694854E-3,-1.1837113E-2,3.9258823E-2,-4.0613156E-2,1.1963553E-1,-7.984054E-4,8.620106E-3,-7.438014E-3,2.6718056E-1,1.0017981E-1,4.7260188E-2,1.6914138E-1,-8.364535E-4,-5.992473E-3,8.4523916E-2,1.992968E-1,4.0114842E-2,1.15140535E-1,1.524149E-2,8.433109E-3,-5.161998E-3,1.4914784E-1,-6.2360045E-2,3.125667E-3,1.2278566E-1,1.5016364E-2,-4.432616E-3,3.3649635E-3,-1.3631102E-2,-4.1190987E-3,-1.0876681E-2,-1.8969536E-2,-6.455458E-3,-3.4363038E-4,4.058736E-3,-8.8496524E-4,-1.9234343E-3,-8.483055E-3,7.284623E-3,-2.0301029E-4,-4.4267816E-3,-8.1089546E-4,-6.230184E-3,1.5632873E-3,7.6258034E-3,2.4155418E-4,-1.1922271E-3,2.3559842E-3,-6.784629E-3,-2.8191946E-5,-2.2053744E-3,1.15093E-3,-3.4536456E-3,-9.285018E-3,5.233488E-4,-2.7928704E-3,-7.0129745E-3,-1.3852382E-2,-5.427931E-3,2.6388254E-4,-7.5436314E-4,4.1496875E-3,-4.074692E-3,-1.2772653E-4,-6.8107457E-4,-3.5717275E-3,2.8587894E-3,-9.89477E-4,8.6355E-3,8.023279E-4,-2.3413491E-3,6.7753554E-4,-1.77737E-3,6.6587306E-3,4.700132E-3,-5.667213E-3,-1.3096094E-4,4.151328E-3,-4.0495214E-3,9.4173785E-4,2.5844863E-3,7.351379E-3,-5.8735996E-3,2.0038981E-3,4.861129E-3,-9.091733E-4,1.3063162E-3,1.8037012E-2,-1.2427411E-3,5.511418E-3,3.4667403E-3,-1.1166942E-3,1.0820319E-2,3.0947148E-3,5.3183273E-3,1.1888049E-3,1.0640224E-2,4.0536993E-3,-1.0172825E-3,3.6988382E-3,1.7526778E-3,6.577277E-3,3.6075427E-3,9.414416E-3,-5.6847357E-3,-2.0259945E-3,7.721271E-3,2.0532054E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,33,35,-1,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,-1,-1,63,65,67,69,-1,71,73,-1,75,77,79,81,83,85,87,89,91,93,-1,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,121,123,125,127,129,131,133,135,137,139,141,143,-1,145,147,-1,-1,149,-1,151,153,-1,-1,155,157,-1,159,-1,-1,-1,-1,161,163,-1,165,167,-1,-1,-1,169,171,173,-1,175,177,179,-1,181,-1,-1,183,-1,185,-1,187,189,191,193,-1,195,197,199,-1,-1,201,-1,203,205,207,209,-1,211,-1,213,215,-1,217,-1,219,221,223,-1,225,-1,227,229,231,-1,-1,-1,-1,233,235,237,239,241,-1,-1,-1,243,-1,-1,-1,245,-1,-1,-1,-1,-1,247,249,251,253,-1,-1,255,257,-1,259,261,263,265,-1,267,269,-1,-1,271,273,275,277,279,-1,281,283,285,287,-1,-1,289,291,293,295,-1,-1,-1,297,299,-1,301,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.903385E0,1.1093156E0,2.3843558E0,1.3028963E0,5.161221E-1,1.3321644E0,7.327442E-1,5.892427E-1,5.6798327E-1,2.579249E-1,3.3462226E-1,6.278394E-1,8.511628E-1,0E0,5.7045674E-1,4.437368E-1,2.2040546E-1,0E0,2.513439E-1,2.5783926E-2,0E0,4.1468763E-1,0E0,2.8280926E-1,3.502399E-1,6.22712E-1,4.4761443E-1,2.6470828E-1,4.0605417E-1,4.2831838E-1,2.0240057E-1,6.813097E-2,1.8911687E-1,2.5956392E-1,7.9221785E-2,0E0,0E0,1.4804316E-1,6.560253E-2,2.071011E-1,1.3881372E-1,0E0,2.1173097E-1,5.363331E-1,0E0,2.1681003E-1,5.159116E-1,2.8897178E-1,4.1412306E-1,2.7398992E-1,1.5780205E-1,2.0404994E-1,4.7808886E-1,2.0269793E-1,1.6847086E-1,0E0,8.0263495E-2,9.787075E-2,4.963371E-2,1.317265E-1,1.3463914E-1,0E0,3.6035776E-2,6.508644E-2,1.23389125E-1,5.5763826E-2,7.144798E-2,2.3985887E-1,1.3329649E-1,5.7401404E-2,1.1201407E-1,1.1337446E-1,1.5918462E-1,4.263702E-1,3.1234804E-1,2.3078549E-1,2.3262441E-1,3.735366E-1,2.3580709E-1,2.9313457E-1,8.893025E-2,2.1982801E-1,0E0,1.684388E-1,2.482838E-1,0E0,0E0,1.7813188E-1,0E0,2.742753E-1,5.549018E-2,0E0,0E0,1.3955615E-1,6.999895E-2,0E0,5.0678372E-2,0E0,0E0,0E0,0E0,1.0581024E-1,1.2164991E-1,0E0,3.6523305E-2,1.655081E-2,0E0,0E0,0E0,1.09151065E-1,1.400013E-1,2.972706E-2,0E0,7.686153E-2,1.3575777E-2,1.5236217E-1,0E0,1.474024E-1,0E0,0E0,3.860636E-2,0E0,5.754661E-2,0E0,7.996005E-2,1.1941314E-1,1.788077E-1,2.5757337E-1,0E0,1.1872977E-1,4.7494483E-1,1.1766442E-1,0E0,0E0,1.330715E-1,0E0,3.4051585E-1,1.980443E-1,2.5787883E-2,1.14221334E-1,0E0,8.8232875E-2,0E0,5.4017544E-2,2.3583837E-1,0E0,7.2246924E-2,0E0,1.0772002E-1,8.334656E-2,2.547996E-1,0E0,1.06337786E-1,0E0,5.2802287E-2,1.329765E-1,1.9858536E-1,0E0,0E0,0E0,0E0,1.2253199E-1,8.297269E-2,1.04649015E-1,6.442343E-2,1.3509331E-2,0E0,0E0,0E0,7.142693E-2,0E0,0E0,0E0,1.2101654E-2,0E0,0E0,0E0,0E0,0E0,2.1403283E-2,1.8415743E-1,6.082952E-2,8.056371E-2,0E0,0E0,3.0186102E-2,1.5390219E-2,0E0,3.8654543E-2,6.0814314E-2,1.0146037E-1,6.0439765E-2,0E0,1.2202976E-1,1.9955659E-1,0E0,0E0,2.479837E-1,4.0060222E-1,1.479274E-2,1.2887779E-1,1.23031706E-1,0E0,3.0709332E-1,1.7488945E-1,2.202037E-1,6.282619E-2,0E0,0E0,1.6099624E-2,6.659794E-2,7.123269E-2,3.895822E-2,0E0,0E0,0E0,1.6611487E-2,2.005829E-2,0E0,2.7361795E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,83,83,84,84,87,87,89,89,90,90,93,93,94,94,96,96,101,101,102,102,104,104,105,105,109,109,110,110,111,111,113,113,114,114,115,115,117,117,120,120,122,122,124,124,125,125,126,126,127,127,129,129,130,130,131,131,134,134,136,136,137,137,138,138,139,139,141,141,143,143,144,144,146,146,148,148,149,149,150,150,152,152,154,154,155,155,156,156,161,161,162,162,163,163,164,164,165,165,169,169,173,173,179,179,180,180,181,181,182,182,185,185,186,186,188,188,189,189,190,190,191,191,193,193,194,194,197,197,198,198,199,199,200,200,201,201,203,203,204,204,205,205,206,206,209,209,210,210,211,211,212,212,216,216,217,217,219,219],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,34,36,-1,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,-1,-1,64,66,68,70,-1,72,74,-1,76,78,80,82,84,86,88,90,92,94,-1,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,122,124,126,128,130,132,134,136,138,140,142,144,-1,146,148,-1,-1,150,-1,152,154,-1,-1,156,158,-1,160,-1,-1,-1,-1,162,164,-1,166,168,-1,-1,-1,170,172,174,-1,176,178,180,-1,182,-1,-1,184,-1,186,-1,188,190,192,194,-1,196,198,200,-1,-1,202,-1,204,206,208,210,-1,212,-1,214,216,-1,218,-1,220,222,224,-1,226,-1,228,230,232,-1,-1,-1,-1,234,236,238,240,242,-1,-1,-1,244,-1,-1,-1,246,-1,-1,-1,-1,-1,248,250,252,254,-1,-1,256,258,-1,260,262,264,266,-1,268,270,-1,-1,272,274,276,278,280,-1,282,284,286,288,-1,-1,290,292,294,296,-1,-1,-1,298,300,-1,302,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.747114E7,1.2679E4,9.357879E2,8.9910974E2,1.9563605E5,7.969174E7,2.467284E7,9.3262803E8,1.7560976E0,1.1563158E1,1.5141565E3,8.05E2,2.1984559E-2,2.8091298E7,1.05E2,2.2007043E0,1.44938715E-2,1.5793961E1,1.683274E0,-2.404738E-2,2.3023027E3,-1.9278038E-2,1.0604491E0,5.322348E7,7.55E2,3.0291306E2,9.3058E4,2.3580047E10,2.047619E0,3.565157E7,1.8255814E0,3.627907E0,3.7002478E3,1.8E9,-1.1712047E-2,-4.3382384E-3,2.2222222E-1,3.9E1,1.3304372E7,4.822785E0,1.0670985E-2,2.36413E5,1.4811083E0,-1.7101903E-2,1.59551E5,2.4081633E0,4.68646E5,4.516183E1,2.1325744E7,8.942177E6,9.705292E6,7.348312E-1,7.9484855E6,1.1087069E7,-7.0438827E-3,2.1647402E3,6.003201E2,7.73756E0,1.2554759E0,6.862E3,1.2057995E-3,7.419886E1,1.9968051E0,1.0017953E8,1.25E2,6.08767E3,8.8543115E2,1E0,1.4E1,3.7037696E7,2.3012722E1,5.244755E-3,4.304E3,3.6887734E-5,1.8817276E1,7E2,1.5208336E4,7.987328E-1,4.5481584E7,6.90802E5,5.23896E5,2.2955636E-2,8.471288E10,5.0279167E1,-1.3559392E-3,-1.4215783E-2,1.0668695E9,5.407527E-3,6.0599957E4,1.3563734E0,-2.1263317E-3,-1.5009437E-2,2.1383E4,6.137763E2,-6.3170744E-3,1.84442E5,4.0671406E-3,-4.6369466E-3,-1.2306019E-2,-4.476229E-3,1E0,5.3E1,1.22038135E-2,4.5575E4,4.456E3,-2.5141926E-3,2.1450568E-3,-6.0199928E-3,3.4E1,4.6263345E-2,3.7930825E5,-3.6116824E-3,9.251576E3,1.2427474E8,6E0,8.64762E-3,1E0,2.6966678E-3,1.3616338E-3,4.6683E7,-7.3617278E-3,2.67E2,4.108149E-3,1.43E2,5.683E3,7.673114E8,4.67E2,-1.1535207E-2,2.124059E-6,9.25E0,5E-1,1.3457218E-2,-1.1670504E-2,7.4569E4,1.6537242E-2,1.0918E4,3.771981E6,1.4977E5,1.8382221E-1,-4.2161653E-3,1.5166431E5,-2.6068275E-3,1.5552E4,1.2414683E5,6.853535E-3,1.0746194E4,-1.680098E-3,1.6956E5,4.8285713E1,4.36E2,-5.1188545E-4,3.6411794E5,2.184821E-3,4.020408E0,3.3654268E0,1.7291142E7,-1.1533574E-2,-3.1328737E-3,-8.622864E-3,-1.9680988E-2,2.7324794E11,2.047619E0,2.0447762E0,2.1367595E0,3.9E1,5.5672214E-3,-9.889187E-3,-4.601585E-3,8.853684E-1,-1.416947E-2,-4.761176E-3,-1.5825672E-2,2.028E3,3.247278E-3,-8.8821683E-4,-9.615062E-3,-2.5723213E-3,8.29754E-4,2.645E3,1.4286339E0,7.063291E0,8.482736E-2,3.2367664E-3,1.1065031E-2,6.1E2,2.5932198E6,-8.186858E-3,3.3382E4,7.64E2,7.879123E7,4.0465977E1,6.5489346E-3,2.1989189E2,2E0,-2.4694854E-3,-1.1837113E-2,1.6499501E7,4.3698645E6,4.36E2,1.198E5,1E0,-7.438014E-3,7.854E3,1.9577E4,1.3226592E-2,9.313E3,-8.364535E-4,-5.992473E-3,1.7427321E-1,2.3977574E7,7.555942E6,1E1,1.524149E-2,8.433109E-3,-5.161998E-3,2.957696E5,2.1E1,3.125667E-3,1.150124E0,1.5016364E-2,-4.432616E-3,3.3649635E-3,-1.3631102E-2,-4.1190987E-3,-1.0876681E-2,-1.8969536E-2,-6.455458E-3,-3.4363038E-4,4.058736E-3,-8.8496524E-4,-1.9234343E-3,-8.483055E-3,7.284623E-3,-2.0301029E-4,-4.4267816E-3,-8.1089546E-4,-6.230184E-3,1.5632873E-3,7.6258034E-3,2.4155418E-4,-1.1922271E-3,2.3559842E-3,-6.784629E-3,-2.8191946E-5,-2.2053744E-3,1.15093E-3,-3.4536456E-3,-9.285018E-3,5.233488E-4,-2.7928704E-3,-7.0129745E-3,-1.3852382E-2,-5.427931E-3,2.6388254E-4,-7.5436314E-4,4.1496875E-3,-4.074692E-3,-1.2772653E-4,-6.8107457E-4,-3.5717275E-3,2.8587894E-3,-9.89477E-4,8.6355E-3,8.023279E-4,-2.3413491E-3,6.7753554E-4,-1.77737E-3,6.6587306E-3,4.700132E-3,-5.667213E-3,-1.3096094E-4,4.151328E-3,-4.0495214E-3,9.4173785E-4,2.5844863E-3,7.351379E-3,-5.8735996E-3,2.0038981E-3,4.861129E-3,-9.091733E-4,1.3063162E-3,1.8037012E-2,-1.2427411E-3,5.511418E-3,3.4667403E-3,-1.1166942E-3,1.0820319E-2,3.0947148E-3,5.3183273E-3,1.1888049E-3,1.0640224E-2,4.0536993E-3,-1.0172825E-3,3.6988382E-3,1.7526778E-3,6.577277E-3,3.6075427E-3,9.414416E-3,-5.6847357E-3,-2.0259945E-3,7.721271E-3,2.0532054E-3],"split_indices":[20,45,2,55,4,28,7,45,31,56,54,55,10,0,45,52,54,0,57,53,0,55,0,38,5,10,52,2,5,58,7,54,56,52,5,0,0,58,3,45,54,0,9,53,0,9,53,29,54,1,47,9,27,45,51,0,33,55,58,39,9,0,57,53,45,8,4,52,96,3,7,56,57,2,41,56,2,33,27,47,10,9,0,31,56,0,0,31,0,32,54,0,0,9,4,0,29,0,0,0,0,67,8,0,2,0,0,0,0,3,57,28,0,33,45,52,0,100,0,0,7,0,8,0,0,11,12,2,0,38,58,53,0,0,29,0,9,1,29,38,0,28,0,9,28,0,4,0,2,55,2,0,33,0,54,58,45,0,0,0,0,31,58,54,39,3,0,0,0,27,0,0,0,0,0,0,0,0,0,9,53,46,41,0,0,2,32,0,9,0,32,58,0,52,6,0,0,5,45,2,7,16,0,9,9,57,2,0,0,38,47,1,8,0,0,0,32,3,0,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.571E3,8.14E2,1.757E3,6.98E2,1.16E2,1.506E3,2.51E2,4.58E2,2.4E2,1.4E1,1.02E2,6.02E2,9.04E2,6E0,2.45E2,4.12E2,4.6E1,6E0,2.34E2,8E0,6E0,9.8E1,4E0,3.89E2,2.13E2,4.83E2,4.21E2,1.8E2,6.5E1,1.59E2,2.53E2,2.3E1,2.3E1,2.12E2,2.2E1,4E0,4E0,5.2E1,4.6E1,3.48E2,4.1E1,8E0,2.05E2,4.79E2,4E0,9.9E1,3.22E2,1.38E2,4.2E1,5.6E1,9E0,1.09E2,5E1,1.2E1,2.41E2,7E0,1.6E1,1.1E1,1.2E1,1.91E2,2.1E1,4E0,1.8E1,8E0,4.4E1,2.5E1,2.1E1,2.91E2,5.7E1,1.4E1,2.7E1,5.8E1,1.47E2,1.66E2,3.13E2,4E1,5.9E1,1.56E2,1.66E2,6.8E1,7E1,3.7E1,5E0,3.2E1,2.4E1,5E0,4E0,1.04E2,5E0,3.2E1,1.8E1,7E0,5E0,2.3E2,1.1E1,4E0,1.2E1,5E0,6E0,7E0,5E0,1.42E2,4.9E1,5E0,1.6E1,1.2E1,6E0,4E0,4E0,2.6E1,1.8E1,1.7E1,8E0,1.1E1,1E1,2.87E2,4E0,5.3E1,4E0,6E0,8E0,4E0,2.3E1,6E0,5.2E1,7.4E1,7.3E1,1.62E2,4E0,1.4E1,2.99E2,3.6E1,4E0,4E0,5.5E1,8E0,1.48E2,1.55E2,1.1E1,6.4E1,4E0,6.5E1,5E0,2.3E1,1.4E1,6E0,2.6E1,6E0,1.8E1,1.4E1,9E1,4E0,2.8E1,4E0,1.4E1,1.04E2,1.26E2,5E0,6E0,4E0,8E0,2.4E1,1.18E2,3.4E1,1.5E1,9E0,7E0,6E0,6E0,2.2E1,4E0,5E0,1.3E1,9E0,8E0,4E0,7E0,5E0,5E0,1.2E1,2.75E2,2.8E1,2.5E1,4E0,4E0,1.4E1,9E0,5E0,4.7E1,5.4E1,2E1,6.5E1,8E0,3.3E1,1.29E2,7E0,7E0,1.34E2,1.65E2,1.1E1,2.5E1,5E1,5E0,1.3E1,1.35E2,1.41E2,1.4E1,4E0,7E0,1.3E1,5.1E1,3.3E1,3.2E1,7E0,1.6E1,5E0,9E0,2.1E1,5E0,1.2E1,6E0,4E0,1E1,7E0,8.3E1,1.7E1,1.1E1,8E0,6E0,1.5E1,8.9E1,1.14E2,1.2E1,7E0,1.7E1,1.8E1,1E2,4E0,3E1,1.1E1,4E0,4E0,5E0,1.7E1,5E0,5E0,4E0,4E0,8E0,5.1E1,2.24E2,2.3E1,5E0,1.5E1,1E1,4E0,1E1,4E0,5E0,2.7E1,2E1,4.1E1,1.3E1,1.4E1,6E0,3.5E1,3E1,2.9E1,4E0,1.25E2,4E0,6.9E1,6.5E1,9.8E1,6.7E1,4E0,7E0,6E0,1.9E1,1.1E1,3.9E1,4E0,9E0,1E1,1.25E2,1.07E2,3.4E1,9E0,5E0,9E0,4E0,4.5E1,6E0,1.2E1,2.1E1,6E0,2.6E1,4E0,5E0,5E0,1.6E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"303","size_leaf_vector":"1"}},{"base_weights":[-7.682293E-4,-5.923355E-2,2.7003234E-2,-7.825658E-2,1.3066324E-2,-1.1071141E-3,6.867472E-2,-6.1229676E-2,-1.8442287E-1,1.4846835E-1,-2.4312975E-2,-3.2240204E-3,2.2126133E-2,-2.4450615E-3,8.4198184E-2,-9.2041925E-2,-2.2291346E-2,-3.1779566E-1,-1.5173271E-1,2.1849132E-1,5.8526646E-2,7.7108354E-3,-1.0296408E-1,-3.2268833E-2,2.1950442E-2,8.473214E-3,-1.3627373E-2,7.513433E-2,1.8817845E-1,-6.962191E-2,-1.3581221E-1,2.3923183E-2,-4.1062526E-2,-3.6662048E-1,-8.3534345E-3,-1.0643255E-1,-2.6315063E-1,1.0654709E-1,1.55182835E-2,-1.3146185E-3,1.0313742E-1,-1.4836788E-2,1.0342908E-1,-5.4317594E-2,-2.1194609E-1,-2.9337544E-2,-1.5438576E-2,2.8317014E-2,-1.1593623E-1,3.8333196E-2,-7.1695365E-2,2.7805865E-2,9.003033E-2,1.5032417E-1,2.8290522E-1,-1.2888284E-1,-2.7680775E-2,-1.6841368E-1,-6.958201E-2,-2.3471552E-3,1.1214415E-1,-1.7231862E-1,-3.237902E-2,-1.1395418E-2,-2.163616E-2,-2.8947972E-2,-1.3747884E-1,-1.8958192E-1,-2.3017354E-2,7.7746427E-3,2.8126594E-3,1.3394274E-3,6.8930076E-3,-4.4768646E-2,3.4884553E-2,1.9259083E-1,-2.1606674E-3,-1.8867573E-2,-1.1027852E-2,-4.1153054E-3,-1.8832086E-2,-4.613261E-2,2.670801E-2,4.555444E-3,6.1861135E-2,-1.4272101E-1,-1.3327283E-3,5.6419276E-2,-3.317296E-2,-5.2907066E-3,-3.1277496E-2,8.372664E-2,-9.354716E-3,1.7973071E-1,7.671254E-2,1.3934822E-2,1.2473981E-1,7.3487326E-3,1.6610788E-2,-1.7240629E-1,-7.4785076E-2,-9.597408E-3,-2.0716365E-2,-2.614747E-1,-9.7931676E-2,-1.0380858E-2,-4.6538062E-2,-4.4337634E-2,3.3155657E-2,8.480763E-3,2.1035676E-3,-1.429786E-3,-1.6940711E-2,-1.14292316E-1,-2.1387935E-2,-8.771765E-2,4.667689E-3,-1.3576278E-2,-1.0974808E-1,-7.6987505E-2,-2.811117E-1,-5.4850418E-2,2.9457211E-3,7.5004174E-4,8.886836E-2,1.3088871E-2,3.9758952E-3,-5.81903E-3,5.2667586E-3,-4.9914826E-2,7.905407E-3,1.6056366E-1,1.3453014E-2,4.2571448E-2,-1.8624637E-2,6.813028E-2,-1.12474635E-2,-1.65299E-1,-2.3205937E-3,-1.112904E-2,7.712375E-2,-9.029275E-3,-4.9451725E-3,-3.2080186E-3,5.1234565E-3,-1.8603647E-2,1.4320782E-1,-3.493183E-2,8.598133E-2,1.3852295E-2,1.4354995E-1,8.03716E-2,-7.5318627E-3,2.2723933E-3,6.907056E-3,-2.7641911E-2,-2.0977475E-1,-1.161978E-2,-4.641345E-2,-1.4404455E-1,-1.07982075E-2,-1.2979007E-1,-3.0346048E-1,-1.0792191E-2,-1.4594279E-1,-9.798296E-3,-9.0739615E-3,-7.623859E-3,-1.2801937E-2,5.968426E-3,1.2751562E-2,6.7382096E-4,-9.200514E-3,3.0160233E-2,-3.4843747E-2,-6.2292526E-3,-1.4984566E-3,-8.249514E-2,-1.2557427E-2,-6.981569E-3,5.731246E-5,-1.7907493E-2,-7.375691E-3,-5.2612377E-3,-4.3390248E-2,4.135389E-3,-3.2105125E-2,6.1147683E-3,8.734643E-4,2.9710783E-2,-2.391689E-3,-6.5923125E-2,-7.835278E-3,1.2840048E-2,6.376044E-4,4.6301432E-2,-1.840967E-2,7.622765E-2,-2.4137512E-2,-4.3757334E-3,-1.1648207E-1,1.3147557E-1,5.0056413E-2,-4.3466915E-3,-9.287104E-3,-4.949139E-3,3.0305393E-2,9.688748E-2,4.2985403E-3,-3.9005342E-3,2.3170412E-2,1.9323828E-3,-1.4715791E-3,-1.3955434E-1,7.365968E-2,2.2655487E-1,9.7119756E-2,-1.0351098E-2,-8.726285E-2,1.2786861E-1,1.1429664E-3,1.6839634E-1,-1.2647714E-3,6.1535027E-2,1.2593639E-1,-7.5279935E-3,3.206722E-3,-1.1633119E-2,-4.7099525E-3,8.307365E-5,-6.099943E-3,-1.0899588E-2,-2.0557789E-3,4.0051998E-3,-9.6724526E-4,-3.1536368E-3,-8.519681E-3,-4.0906738E-3,-1.7396146E-2,4.501463E-3,-4.013824E-3,-5.638536E-3,-1.3441018E-2,2.0692798E-3,-2.116527E-3,-1.9583364E-3,3.203163E-3,-1.9929647E-3,1.8494638E-3,-1.7958195E-3,3.8613514E-3,-6.803352E-5,-2.7307183E-3,-5.493844E-3,-3.10628E-4,1.0040028E-3,-2.6499047E-3,2.1811627E-4,-4.3739923E-3,-3.1639292E-4,3.587967E-3,-7.5413203E-3,-2.6990462E-3,2.3760162E-5,-8.452355E-3,-2.0236892E-4,5.1523447E-3,-1.8229525E-3,3.697752E-3,1.3630036E-3,6.6144643E-3,1.0089256E-3,-4.730061E-3,-1.3538573E-3,2.4489565E-3,5.3544686E-4,-8.84262E-3,1.1829227E-3,7.955759E-3,8.508683E-3,2.1172476E-3,4.9306443E-3,-5.51983E-4,5.355528E-3,6.907916E-4,-4.1517336E-3,3.4628238E-3,-1.0213446E-3,2.5488506E-3,-1.1251811E-2,-1.2997069E-3,1.3202136E-6,7.359517E-3,6.9313427E-3,1.3822342E-2,7.028391E-3,-2.4829626E-3,-6.7878375E-3,4.6076384E-4,-7.1655363E-3,7.434121E-4,8.212823E-3,2.9229478E-3,1.0122563E-2,3.1310488E-3,-6.836858E-3,3.471563E-3,1.0871984E-2,5.1819016E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,-1,65,67,69,-1,-1,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,115,117,119,-1,-1,-1,-1,-1,121,123,125,-1,127,-1,-1,-1,129,131,133,135,137,-1,139,141,-1,143,145,147,149,151,-1,153,-1,-1,155,157,-1,159,161,163,-1,165,167,169,-1,-1,-1,-1,171,173,175,-1,-1,177,179,181,183,-1,185,187,-1,-1,-1,189,191,-1,193,195,197,199,201,-1,203,-1,205,207,209,-1,-1,211,213,215,217,219,-1,221,223,-1,-1,-1,225,227,-1,229,231,233,235,237,239,241,243,-1,-1,245,-1,247,-1,-1,249,251,-1,-1,253,-1,-1,-1,-1,-1,-1,255,-1,257,-1,-1,259,-1,261,263,-1,-1,265,267,269,271,273,275,277,279,-1,-1,-1,281,283,285,-1,287,-1,-1,289,291,293,295,297,299,301,-1,303,-1,305,307,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.215017E0,1.1508257E0,2.0617402E0,1.1853938E0,8.8932544E-1,9.9004215E-1,7.8295994E-1,6.842208E-1,3.4853792E-1,2.2653586E-1,3.4837562E-1,7.6699287E-1,0E0,3.8004363E-1,5.3437424E-1,3.0358887E-1,2.2132765E-1,6.372249E-2,3.50536E-1,1.8337703E-1,7.083678E-2,2.1519823E-1,2.005513E-1,3.859474E-1,4.9664336E-1,2.9958355E-1,0E0,3.760841E-1,1.22419834E-1,5.2583766E-1,2.19316E-1,1.7176521E-1,2.007176E-1,1.1637449E-2,0E0,1.2832874E-1,2.1780002E-1,1.8356912E-2,0E0,0E0,2.8191283E-2,1.2211016E-1,2.567999E-1,1.668897E-1,2.2545457E-1,4.5587462E-1,0E0,4.28537E-1,5.596617E-2,1.1955881E-1,4.3671474E-2,2.7092952E-1,4.736135E-1,8.0330074E-2,3.1844556E-2,1.9430435E-1,1.4028043E-1,4.400916E-1,1.0814187E-1,8.7935284E-2,6.1423615E-2,2.5384235E-1,1.5150955E-1,0E0,0E0,1.3044626E-1,1.153481E-1,1.5355545E-1,0E0,0E0,0E0,0E0,0E0,5.65646E-2,5.705738E-2,7.32266E-2,0E0,5.9495497E-2,0E0,0E0,0E0,2.9540908E-1,1.9655666E-1,2.7925655E-1,4.1651195E-1,3.362462E-2,0E0,1.02952614E-1,2.9289195E-2,0E0,2.0703882E-2,3.1917393E-1,1.9563876E-1,1.4364004E-1,3.0922985E-1,0E0,2.6868314E-2,0E0,0E0,2.5386345E-1,1.7272519E-1,0E0,1.4767358E-1,1.2648034E-1,1.7413884E-1,0E0,1.6197175E-1,9.021023E-2,5.48199E-2,0E0,0E0,0E0,0E0,1.8096492E-1,1.06481515E-1,2.1858491E-2,0E0,0E0,1.10444516E-1,4.4277605E-2,3.9945543E-2,2.2602066E-2,0E0,5.6548078E-2,2.6565835E-2,0E0,0E0,0E0,2.8696315E-2,2.4602908E-1,0E0,1.3921577E-1,1.0865618E-1,2.7126032E-1,2.7417946E-1,2.4415052E-1,0E0,7.989347E-3,0E0,6.857402E-2,8.018419E-2,3.5151582E-2,0E0,0E0,1.15787955E-2,2.3346579E-1,1.0476005E-1,8.0135815E-2,4.2012334E-2,0E0,1.7208719E-1,2.943666E-1,0E0,0E0,0E0,1.3391884E-1,7.9200506E-2,0E0,1.3116232E-1,5.9643388E-2,9.03057E-2,1.3671964E-2,1.8938446E-1,1.19478196E-1,7.960993E-2,4.6775937E-2,0E0,0E0,4.7165263E-2,0E0,3.595831E-2,0E0,0E0,1.0244294E-1,8.017075E-2,0E0,0E0,6.2143266E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.5553063E-2,0E0,3.0707091E-2,0E0,0E0,2.3213826E-2,0E0,2.6004648E-1,1.3683553E-1,0E0,0E0,1.4631236E-1,9.1223925E-2,2.1626914E-1,1.3009459E-1,2.1116605E-1,1.9424647E-1,1.3979995E-1,1.5167096E-1,0E0,0E0,0E0,3.7846312E-2,3.614214E-2,7.960644E-2,0E0,1.4398392E-2,0E0,0E0,8.164868E-2,6.50545E-2,1.0009289E-2,1.5401663E-1,1.0932891E-1,1.1682466E-1,1.570344E-2,0E0,1.138314E-1,0E0,4.0496027E-1,1.8523669E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,73,73,74,74,75,75,77,77,81,81,82,82,83,83,84,84,85,85,87,87,88,88,90,90,91,91,92,92,93,93,94,94,96,96,99,99,100,100,102,102,103,103,104,104,106,106,107,107,108,108,113,113,114,114,115,115,118,118,119,119,120,120,121,121,123,123,124,124,128,128,129,129,131,131,132,132,133,133,134,134,135,135,137,137,139,139,140,140,141,141,144,144,145,145,146,146,147,147,148,148,150,150,151,151,155,155,156,156,158,158,159,159,160,160,161,161,162,162,163,163,164,164,165,165,168,168,170,170,173,173,174,174,177,177,184,184,186,186,189,189,191,191,192,192,195,195,196,196,197,197,198,198,199,199,200,200,201,201,202,202,206,206,207,207,208,208,210,210,213,213,214,214,215,215,216,216,217,217,218,218,219,219,221,221,223,223,224,224],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,-1,66,68,70,-1,-1,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,116,118,120,-1,-1,-1,-1,-1,122,124,126,-1,128,-1,-1,-1,130,132,134,136,138,-1,140,142,-1,144,146,148,150,152,-1,154,-1,-1,156,158,-1,160,162,164,-1,166,168,170,-1,-1,-1,-1,172,174,176,-1,-1,178,180,182,184,-1,186,188,-1,-1,-1,190,192,-1,194,196,198,200,202,-1,204,-1,206,208,210,-1,-1,212,214,216,218,220,-1,222,224,-1,-1,-1,226,228,-1,230,232,234,236,238,240,242,244,-1,-1,246,-1,248,-1,-1,250,252,-1,-1,254,-1,-1,-1,-1,-1,-1,256,-1,258,-1,-1,260,-1,262,264,-1,-1,266,268,270,272,274,276,278,280,-1,-1,-1,282,284,286,-1,288,-1,-1,290,292,294,296,298,300,302,-1,304,-1,306,308,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,8.10114E4,4.8297736E7,1.504779E7,8.791777E3,3.1164652E-1,2.046E3,3.449702E2,5.380323E-1,9.302862E7,2.0407547E5,2.2126133E-2,2.9407698E-1,7.096131E0,1.00137E5,1.15062E5,3.476768E-3,1.13808104E8,4.0434834E5,1.3565588E6,5.528107E3,6.0435E4,6.5321615E6,4.2343444E7,4.2343444E7,-1.3627373E-2,5.862126E2,1.9483356E1,9E2,6.81011E5,8.6337E4,3.24756E5,6.8651576E9,-8.3534345E-3,4.57091E5,2.5030267E5,1.8238512E0,1.55182835E-2,-1.3146185E-3,3.088934E6,4.602015E0,1.0040462E8,9.172968E-1,3.5481934E1,5.513E3,-1.5438576E-2,1.172E3,9.222717E-1,4.1899456E7,5.964E3,1.402418E0,2.7465258E1,1.0620689E0,3.224158E6,1.9121015E0,4.361399E5,2.308943E0,1E0,5.314225E2,4.25E0,2.46E2,1.9143396E2,-1.1395418E-2,-2.163616E-2,1E0,5.5987104E-5,8.756944E5,-2.3017354E-2,7.7746427E-3,2.8126594E-3,1.3394274E-3,6.8930076E-3,3.88672E5,1.001194E6,1.671E3,-2.1606674E-3,2.3983052E0,-1.1027852E-2,-4.1153054E-3,-1.8832086E-2,1.415178E3,3E0,1.1333333E0,3.619075E1,7.160724E1,-1.3327283E-3,1.33027E5,4.217926E-2,-5.2907066E-3,3.0324321E-2,6.4506575E5,1.998E3,6.8E1,3.0272608E10,1.3934822E-2,5.073276E-1,7.3487326E-3,1.6610788E-2,2.3369722E-3,1.8991614E5,-9.597408E-3,3.6355584E7,1.6093831E0,5.188105E10,-1.0380858E-2,7.0248324E-1,1.346757E6,1.4E2,8.480763E-3,2.1035676E-3,-1.429786E-3,-1.6940711E-2,1.0700461E6,9E0,2.579E3,4.667689E-3,-1.3576278E-2,4.1043E4,8.625696E-1,2.2673786E5,1.4492002E0,2.9457211E-3,1E0,4.6E1,1.3088871E-2,3.9758952E-3,-5.81903E-3,1.1159378E12,2.001177E6,7.905407E-3,2.909019E3,1.6837112E7,8.573816E0,2.983871E0,3.1140776E0,-1.12474635E-2,3.768116E-1,-2.3205937E-3,4.9595168E8,1E0,3.7832818E0,-4.9451725E-3,-3.2080186E-3,8.657441E7,1.2792622E8,8E-1,1.296E3,1.2E1,1.3852295E-2,3.0811846E0,5.5743E4,-7.5318627E-3,2.2723933E-3,6.907056E-3,7.518072E-1,1.3593E4,-1.161978E-2,8.595346E-1,5.8805E4,4.5555557E1,7E0,4.38E2,1.7036236E5,1.7257734E6,1.1E1,-9.0739615E-3,-7.623859E-3,2E0,5.968426E-3,1.7835504E-1,6.7382096E-4,-9.200514E-3,4.1026765E-1,1.9E1,-6.2292526E-3,-1.4984566E-3,2.7102E4,-1.2557427E-2,-6.981569E-3,5.731246E-5,-1.7907493E-2,-7.375691E-3,-5.2612377E-3,2.4394053E9,4.135389E-3,8.67822E-1,6.1147683E-3,8.734643E-4,3E1,-2.391689E-3,3.983889E3,8.81356E5,1.2840048E-2,6.376044E-4,7.108182E4,1.19573875E5,5.44E2,1E0,1.838444E6,5.8891744E8,1E0,6E0,-4.3466915E-3,-9.287104E-3,-4.949139E-3,1.6908121E1,2.748204E-1,6.47E2,-3.9005342E-3,2.1690162E7,1.9323828E-3,-1.4715791E-3,1.1472081E0,6.933837E-1,9.561769E-1,4.4821704E7,4.2344445E-1,1.0013907E1,1.6709303E1,1.1429664E-3,1.4450867E-2,-1.2647714E-3,1.4523809E0,1.3332031E1,-7.5279935E-3,3.206722E-3,-1.1633119E-2,-4.7099525E-3,8.307365E-5,-6.099943E-3,-1.0899588E-2,-2.0557789E-3,4.0051998E-3,-9.6724526E-4,-3.1536368E-3,-8.519681E-3,-4.0906738E-3,-1.7396146E-2,4.501463E-3,-4.013824E-3,-5.638536E-3,-1.3441018E-2,2.0692798E-3,-2.116527E-3,-1.9583364E-3,3.203163E-3,-1.9929647E-3,1.8494638E-3,-1.7958195E-3,3.8613514E-3,-6.803352E-5,-2.7307183E-3,-5.493844E-3,-3.10628E-4,1.0040028E-3,-2.6499047E-3,2.1811627E-4,-4.3739923E-3,-3.1639292E-4,3.587967E-3,-7.5413203E-3,-2.6990462E-3,2.3760162E-5,-8.452355E-3,-2.0236892E-4,5.1523447E-3,-1.8229525E-3,3.697752E-3,1.3630036E-3,6.6144643E-3,1.0089256E-3,-4.730061E-3,-1.3538573E-3,2.4489565E-3,5.3544686E-4,-8.84262E-3,1.1829227E-3,7.955759E-3,8.508683E-3,2.1172476E-3,4.9306443E-3,-5.51983E-4,5.355528E-3,6.907916E-4,-4.1517336E-3,3.4628238E-3,-1.0213446E-3,2.5488506E-3,-1.1251811E-2,-1.2997069E-3,1.3202136E-6,7.359517E-3,6.9313427E-3,1.3822342E-2,7.028391E-3,-2.4829626E-3,-6.7878375E-3,4.6076384E-4,-7.1655363E-3,7.434121E-4,8.212823E-3,2.9229478E-3,1.0122563E-2,3.1310488E-3,-6.836858E-3,3.471563E-3,1.0871984E-2,5.1819016E-3],"split_indices":[20,52,33,45,45,52,42,2,52,27,45,28,0,38,35,29,29,38,45,28,28,52,10,47,45,45,0,52,54,2,1,29,1,5,0,12,33,53,0,0,1,53,51,27,57,2,0,2,27,32,2,53,58,54,47,53,45,54,67,4,54,10,52,0,0,16,38,28,0,0,0,0,0,9,29,0,0,54,0,0,0,52,8,56,53,56,0,9,57,0,38,28,10,0,5,0,42,0,0,41,28,0,7,54,31,0,27,50,10,0,0,0,0,50,3,2,0,0,10,27,32,53,0,8,8,0,0,0,31,9,0,32,45,58,53,54,0,58,0,31,74,53,0,0,45,7,53,29,3,0,38,10,0,0,0,56,10,0,27,9,52,8,0,28,32,8,0,0,8,0,38,0,0,57,8,0,0,10,0,0,0,0,0,0,5,0,27,0,0,8,0,28,29,0,0,28,28,2,16,9,31,102,3,0,0,0,58,42,0,0,45,0,0,53,42,38,12,38,56,58,0,57,0,54,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.594E3,8.35E2,1.759E3,6.61E2,1.74E2,1.051E3,7.08E2,5.71E2,9E1,3.7E1,1.37E2,1.047E3,4E0,1.27E2,5.81E2,3.18E2,2.53E2,1.6E1,7.4E1,2E1,1.7E1,9.8E1,3.9E1,4.86E2,5.61E2,1.23E2,4E0,5.36E2,4.5E1,2.12E2,1.06E2,7.3E1,1.8E2,1.1E1,5E0,5.4E1,2E1,1E1,1E1,6E0,1.1E1,8E1,1.8E1,2.8E1,1.1E1,4.82E2,4E0,5.37E2,2.4E1,9E1,3.3E1,1.29E2,4.07E2,3.4E1,1.1E1,8.7E1,1.25E2,7E1,3.6E1,5.7E1,1.6E1,1E1,1.7E2,5E0,6E0,1.6E1,3.8E1,1.6E1,4E0,4E0,6E0,4E0,7E0,5E1,3E1,1.1E1,7E0,2.4E1,4E0,7E0,4E0,3.71E2,1.11E2,3.15E2,2.22E2,1.8E1,6E0,7.2E1,1.8E1,1.7E1,1.6E1,5.1E1,7.8E1,5.1E1,3.56E2,4E0,3E1,4E0,7E0,4.7E1,4E1,4E0,1.21E2,2.9E1,4.1E1,4E0,3.2E1,2.6E1,3.1E1,8E0,8E0,6E0,4E0,1.9E1,1.51E2,1.1E1,5E0,5E0,3.3E1,8E0,8E0,4.6E1,4E0,1.9E1,1.1E1,6E0,5E0,4E0,2E1,3.65E2,6E0,9E0,1.02E2,1.19E2,1.96E2,2.18E2,4E0,1.4E1,4E0,1.7E1,5.5E1,1.4E1,4E0,8E0,8E0,1.9E1,3.2E1,6.2E1,1.6E1,1.2E1,3.9E1,3.51E2,5E0,5E0,2.5E1,1E1,3.7E1,5E0,3.5E1,8E0,1.13E2,8E0,2.1E1,1.5E1,2.6E1,2.6E1,6E0,5E0,2.1E1,5E0,2.6E1,7E0,1.2E1,3.1E1,1.2E2,6E0,5E0,2.9E1,4E0,4E0,4E0,4E0,4E0,7E0,3.9E1,5E0,1.4E1,7E0,4E0,1.4E1,6E0,2.64E2,1.01E2,5E0,4E0,5E1,5.2E1,7.9E1,4E1,1.72E2,2.4E1,4.7E1,1.71E2,4E0,1E1,5E0,1.2E1,4.3E1,1.2E1,4E0,1E1,4E0,4E0,8E0,1.1E1,1E1,2.2E1,4.3E1,1.9E1,9E0,7E0,3.4E1,5E0,2.5E2,1.01E2,4E0,6E0,3E1,7E0,2.2E1,1.3E1,4E0,4E0,9E0,1.04E2,4E0,4E0,4E0,1.7E1,6E0,9E0,2.2E1,4E0,1E1,1.6E1,1.6E1,5E0,8E0,1.8E1,1.3E1,1.8E1,4.5E1,7.5E1,2.1E1,8E0,5E0,3.4E1,9E0,5E0,8E0,6E0,3.1E1,2.33E2,9.7E1,4E0,2.7E1,2.3E1,4.4E1,8E0,4.3E1,3.6E1,2.5E1,1.5E1,1.21E2,5.1E1,8E0,1.6E1,1E1,3.7E1,9E0,1.62E2,4E0,8E0,3.8E1,5E0,5E0,7E0,4E0,6E0,4E0,4E0,6E0,5E0,5E0,5E0,1.7E1,5E0,5E0,3.8E1,1.2E1,7E0,5E0,4E0,2.5E1,9E0,9E0,2.41E2,1.8E1,8.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"309","size_leaf_vector":"1"}},{"base_weights":[-7.741158E-4,2.0555275E-2,-6.0773745E-2,-7.597551E-3,6.812636E-2,-9.6408255E-2,-1.0957347E-3,-2.9763427E-2,2.7990911E-2,1.7757501E-2,1.1115825E-1,-1.1184235E-1,1.3158214E-2,-2.8529875E-2,7.247545E-2,-4.775919E-2,5.7676067E-3,3.5932645E-2,-1.0011354E-1,-6.4154137E-3,8.747489E-2,9.88205E-2,2.5180474E-1,-9.7164206E-2,-1.9780095E-1,5.371398E-2,-1.1576061E-1,-1.9802954E-2,-1.4247046E-1,1.6311395E-1,4.8521623E-2,-5.3939503E-2,5.19202E-2,-6.085112E-2,2.7352579E-2,4.4170577E-2,-4.16141E-2,-4.9849246E-2,-1.5554954E-2,1.3575573E-2,-8.791162E-2,7.0534E-2,2.2081189E-1,1.345247E-1,6.642416E-2,-2.956138E-3,3.1636462E-1,-9.09648E-2,-2.7900952E-1,-3.4649277E-1,-1.4583036E-1,1.6626793E-1,1.6862314E-2,-1.1503458E-2,-3.9308306E-2,-8.066465E-2,-9.933063E-3,-1.8503878E-1,-1.8664249E-3,4.7309687E-3,1.0123358E-2,2.367868E-2,1.0412139E-1,-5.073325E-2,-1.802007E-1,-5.0057406E-3,8.0263756E-2,-4.3339845E-2,-1.2407195E-2,5.072337E-2,-1.1004854E-2,1.3354123E-2,7.066783E-2,-1.0677694E-1,5.120174E-2,-8.154816E-2,4.0261387E-3,1.7402658E-1,1.2264502E-3,-1.8842037E-1,-5.6503523E-2,1.4642605E-1,4.9471594E-2,1.4512526E-3,1.8408569E-2,1.4979574E-2,1.1852302E-1,5.682073E-2,1.8035057E-1,1.9109257E-2,1.1290139E-2,-2.12763E-1,-8.2699746E-2,-2.060812E-2,-3.1526084E-3,-9.047092E-3,-1.9981632E-2,-2.9196404E-2,-1.8762292E-1,2.1483086E-3,1.2056166E-2,1.13106094E-1,-1.99317E-2,4.0741867E-4,-3.9428077E-3,-3.257449E-2,-1.18850596E-1,-1.1584191E-3,-8.8827625E-2,-5.2466434E-3,-1.1395744E-2,6.0225368E-2,-2.4335345E-2,7.667818E-3,3.536439E-3,2.525449E-2,-5.545244E-2,-4.9538156E-3,-1.2873521E-2,7.4970815E-3,5.0331697E-2,-5.2399136E-2,3.0657873E-3,1.1715842E-1,3.2319073E-2,-1.0312157E-2,6.655744E-3,5.697731E-2,-1.9178856E-2,1.05276525E-1,2.3353178E-2,1.1670706E-3,-1.4047827E-1,8.674598E-3,6.736019E-3,-1.1383548E-1,-4.96482E-4,4.9778344E-3,1.020782E-2,-5.6401766E-3,8.728118E-3,-1.3604277E-2,-1.6664361E-3,-1.0980049E-1,8.815351E-3,6.5809295E-2,1.5520698E-2,6.646466E-2,-7.7020503E-3,-9.220333E-3,1.2789711E-1,7.023095E-2,-3.841449E-2,1.5854122E-2,6.9016024E-2,-3.5264692E-3,-2.5850272E-1,-3.4068756E-2,-1.0916211E-1,1.6602717E-3,-6.157668E-3,-2.3059885E-1,-1.13685265E-1,1.464889E-3,8.707873E-3,-6.594758E-2,3.1579807E-2,-3.2231293E-3,-2.7555152E-5,-7.7438834E-3,-1.5012881E-3,4.1871503E-2,-1.7067906E-2,-1.4249119E-3,-1.08166195E-1,1.0280757E-1,1.262439E-2,-6.937754E-3,-2.8967794E-3,6.230655E-2,-2.6252797E-2,-5.7514597E-2,6.438674E-3,-1.6005174E-3,7.443072E-2,1.365173E-3,-6.7178875E-2,2.1332705E-4,1.5258288E-1,5.6429487E-2,-3.9349604E-2,-4.834553E-2,4.054568E-2,-9.826468E-3,7.865694E-2,-9.3196265E-2,2.0789592E-2,-6.060146E-2,1.22415245E-1,-6.0972064E-3,3.7084214E-2,-7.7589355E-2,-1.1248145E-2,2.5800187E-3,-2.5009033E-3,-3.3803608E-3,-8.704097E-3,-1.1180263E-1,4.1639768E-3,-6.861636E-2,-7.849194E-3,-2.9303753E-3,4.7433548E-2,4.740524E-3,3.0701153E-4,8.302736E-3,9.309574E-2,4.889712E-2,1.5643933E-1,1.2451191E-1,5.0394945E-2,6.2041795E-3,-8.14417E-2,6.1472137E-3,6.0087295E-5,-1.6382147E-2,-5.565324E-3,-9.3454435E-2,1.6537936E-2,-1.9469309E-1,-7.414111E-2,-1.3308455E-2,-5.7317824E-3,-2.8494E-2,-1.2206482E-2,-1.6426519E-3,-4.476969E-3,-3.95296E-4,4.419524E-3,1.5241505E-2,1.2302688E-1,-3.0717922E-2,4.6822052E-2,-2.3775047E-3,-6.7948895E-3,2.3481594E-3,6.2378272E-3,2.7677333E-3,-2.9144564E-3,-1.4171602E-3,1.6495816E-3,1.3212076E-3,6.791503E-3,-3.427422E-3,6.890505E-4,-1.8552571E-3,-3.943189E-3,4.7682407E-3,6.6703465E-4,-3.844464E-3,1.314265E-3,2.0412593E-3,8.97672E-3,5.518008E-3,1.0996663E-3,-4.1554514E-3,2.9318288E-3,-2.4832378E-4,-5.3485553E-3,-1.06268E-3,4.1686962E-3,1.1971354E-3,-3.7971162E-3,6.4959084E-3,2.4479285E-3,4.393313E-3,-8.171281E-3,-4.9851238E-5,8.301911E-3,-6.574952E-3,1.614671E-3,6.7210975E-3,1.0898049E-3,2.5326803E-3,-4.835849E-3,-5.7361173E-3,-7.517533E-4,-6.549023E-3,-2.3513623E-3,1.8311111E-3,-7.551844E-4,-4.6402593E-3,-1.2414708E-3,3.930305E-3,1.5483625E-4,-1.9255723E-3,2.7138065E-3,2.5110622E-3,7.0095723E-3,5.2300603E-3,-1.7987783E-3,5.800494E-3,1.0167263E-2,-1.1796041E-3,7.0765507E-3,1.2552691E-2,2.034662E-3,-8.44564E-3,-1.7365252E-3,-2.7609807E-3,-7.437458E-3,1.65991E-3,-8.174706E-3,-2.2848962E-4,-1.0692856E-2,-1.6922134E-3,-5.1534674E-3,1.3362907E-3,-3.899041E-3,-1.9337133E-3,3.4060767E-3,8.582391E-3,2.482647E-3,-3.1526906E-3,-5.596412E-4,-7.118462E-4,4.3460475E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,83,85,87,-1,89,91,93,95,97,99,101,-1,103,105,107,109,-1,-1,-1,111,113,115,117,-1,119,121,-1,123,125,127,129,131,133,135,-1,137,139,141,143,145,147,-1,-1,-1,149,151,153,-1,-1,155,157,-1,-1,-1,-1,159,161,-1,-1,163,165,-1,-1,167,169,171,173,-1,-1,175,177,-1,-1,179,181,-1,-1,-1,183,185,-1,187,189,-1,191,193,195,197,199,-1,201,203,-1,205,-1,-1,-1,207,-1,-1,-1,209,211,213,-1,215,-1,-1,217,219,221,-1,223,-1,225,227,229,-1,-1,231,233,-1,-1,235,237,-1,-1,-1,-1,239,241,-1,243,245,247,249,-1,251,253,255,-1,-1,257,-1,259,-1,261,263,265,267,269,271,273,275,277,279,281,-1,283,285,-1,-1,-1,-1,-1,287,289,291,-1,-1,293,-1,-1,295,297,299,301,303,305,-1,307,-1,-1,-1,-1,309,311,313,315,-1,-1,317,-1,-1,-1,-1,-1,319,321,323,325,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2889411E0,2.539755E0,1.4332206E0,9.411416E-1,1.5260904E0,7.150984E-1,5.1271343E-1,4.7024035E-1,4.6955356E-1,5.499601E-1,6.3451624E-1,4.476142E-1,2.8332326E-1,1.8113354E-1,1.4189404E-1,3.0343282E-1,3.5827932E-1,2.7781034E-1,2.7109933E-1,3.9708653E-1,1.7416847E-1,3.9706063E-1,6.209965E-1,3.33112E-1,3.5903358E-1,1.6725615E-1,1.04711875E-1,1.03119574E-1,5.1258713E-2,1.8648475E-2,7.4885204E-2,1.7707741E-1,1.3306299E-1,1.8947901E-1,1.6908959E-1,3.1832796E-1,2.5948918E-1,1.0166102E-1,0E0,3.881467E-1,1.4087948E-1,1.1423972E-1,2.440522E-1,4.0346193E-1,1.9325203E-1,0E0,6.293559E-2,2.9346514E-1,2.6206857E-1,4.244888E-2,1.9437373E-1,8.21009E-2,1.1557264E-1,0E0,1.75591E-2,4.020132E-2,1.03737846E-1,7.046491E-3,0E0,0E0,0E0,7.165575E-2,1.7106742E-2,1.6277051E-1,4.350674E-2,0E0,4.466307E-2,5.7278194E-2,0E0,1.3926834E-1,2.4980633E-1,2.5938573E-1,3.41416E-1,1.12923294E-1,6.283193E-2,4.2799845E-2,0E0,1.1174291E-2,2.1894418E-1,1.3394222E-1,1.3325724E-1,1.9134814E-1,2.213082E-1,0E0,0E0,0E0,4.5825005E-1,2.2364068E-1,1.9155306E-1,0E0,0E0,1.0532373E-1,3.7076652E-1,0E0,0E0,0E0,0E0,7.453837E-2,6.911063E-2,0E0,0E0,4.1017137E-2,5.8970943E-2,0E0,0E0,1.2234791E-2,3.7054658E-2,9.3818076E-2,1.44101605E-2,0E0,0E0,4.5322075E-2,1.0132032E-2,0E0,0E0,5.3163894E-2,1.6856527E-1,0E0,0E0,0E0,4.0096816E-2,6.428282E-2,0E0,9.9324405E-2,1.6292092E-1,0E0,1.2684752E-1,1.13738865E-1,3.125654E-1,3.5141397E-1,1.8418318E-1,0E0,8.8985294E-2,3.538186E-2,0E0,2.4455667E-2,0E0,0E0,0E0,1.8469296E-1,0E0,0E0,0E0,3.0119091E-2,4.9676478E-2,2.0226609E-2,0E0,8.7515205E-2,0E0,0E0,3.2832098E-1,1.573863E-1,1.6401114E-1,0E0,3.2815672E-2,0E0,1.0634142E-1,3.140981E-1,5.486965E-1,0E0,0E0,4.401529E-2,1.3768487E-1,0E0,0E0,7.2232746E-3,2.7649812E-2,0E0,0E0,0E0,0E0,7.8088075E-2,8.864817E-2,0E0,1.2070939E-2,8.889437E-3,3.9644115E-2,1.2063466E-2,0E0,3.864848E-2,2.1291632E-2,1.8030298E-1,0E0,0E0,1.7964527E-2,0E0,4.308808E-2,0E0,5.012974E-2,1.2769316E-1,1.0795891E-1,6.5664634E-2,1.1310796E-1,4.6529606E-2,8.350396E-2,4.8575017E-1,2.1868782E-1,8.340062E-2,1.3093066E-1,0E0,1.5604854E-1,2.8526984E-2,0E0,0E0,0E0,0E0,0E0,1.2541756E-2,1.0262326E-1,1.1491656E-2,0E0,0E0,1.6368486E-2,0E0,0E0,4.3039635E-2,7.1793705E-2,1.9790381E-1,1.83285E-1,1.0473746E-1,2.0931852E-1,0E0,6.77298E-2,0E0,0E0,0E0,0E0,9.334016E-2,1.7673066E-1,1.9723344E-1,1.5416443E-1,0E0,0E0,2.6714563E-2,0E0,0E0,0E0,0E0,0E0,8.542968E-2,2.3634478E-2,5.17444E-2,4.5380894E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,61,61,62,62,63,63,64,64,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,77,77,78,78,79,79,80,80,81,81,82,82,86,86,87,87,88,88,91,91,92,92,97,97,98,98,101,101,102,102,105,105,106,106,107,107,108,108,111,111,112,112,115,115,116,116,120,120,121,121,123,123,124,124,126,126,127,127,128,128,129,129,130,130,132,132,133,133,135,135,139,139,143,143,144,144,145,145,147,147,150,150,151,151,152,152,154,154,156,156,157,157,158,158,161,161,162,162,165,165,166,166,171,171,172,172,174,174,175,175,176,176,177,177,179,179,180,180,181,181,184,184,186,186,188,188,189,189,190,190,191,191,192,192,193,193,194,194,195,195,196,196,197,197,198,198,200,200,201,201,207,207,208,208,209,209,212,212,215,215,216,216,217,217,218,218,219,219,220,220,222,222,227,227,228,228,229,229,230,230,233,233,239,239,240,240,241,241,242,242],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,84,86,88,-1,90,92,94,96,98,100,102,-1,104,106,108,110,-1,-1,-1,112,114,116,118,-1,120,122,-1,124,126,128,130,132,134,136,-1,138,140,142,144,146,148,-1,-1,-1,150,152,154,-1,-1,156,158,-1,-1,-1,-1,160,162,-1,-1,164,166,-1,-1,168,170,172,174,-1,-1,176,178,-1,-1,180,182,-1,-1,-1,184,186,-1,188,190,-1,192,194,196,198,200,-1,202,204,-1,206,-1,-1,-1,208,-1,-1,-1,210,212,214,-1,216,-1,-1,218,220,222,-1,224,-1,226,228,230,-1,-1,232,234,-1,-1,236,238,-1,-1,-1,-1,240,242,-1,244,246,248,250,-1,252,254,256,-1,-1,258,-1,260,-1,262,264,266,268,270,272,274,276,278,280,282,-1,284,286,-1,-1,-1,-1,-1,288,290,292,-1,-1,294,-1,-1,296,298,300,302,304,306,-1,308,-1,-1,-1,-1,310,312,314,316,-1,-1,318,-1,-1,-1,-1,-1,320,322,324,326,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,6.218596E2,1E0,4.930349E5,1.5166431E5,3.0497742E3,7.883E3,1.5107028E-1,2E0,1.3359244E9,6.7652373E3,1.0017953E8,2.0568174E7,3.0121632E6,1.0526E4,4.3800272E8,1.3781E2,1E0,1.325204E6,4.0844156E7,2.9956698E-1,3.824611E0,2.64E0,4.5723195E6,1.9044118E0,2E0,8.755996E-1,1.246E3,6.63E2,6.1428223E1,2.8541E4,1E0,1E0,4.3978744E0,2.2415E4,6.1934864E7,8.874407E6,1.9E1,-1.5554954E-2,1.6E1,5.4106476E7,2.909019E3,4.90027E1,1.0918E4,3.832274E2,-2.956138E-3,9.994E3,1.1331109E6,3E0,2.1818182E0,2.7487562E0,5.441732E9,6.135771E9,-1.1503458E-2,3.7930825E5,1.7491536E5,6.747114E7,1.17E2,-1.8664249E-3,4.7309687E-3,1.0123358E-2,1.2E1,1E0,1.552356E0,9.379509E-2,-5.0057406E-3,6.4E1,2.3E1,-1.2407195E-2,2.0011E5,3.6579834E5,1E0,1.5445488E8,1.4966443E0,9.7043E4,1.038946E6,4.0261387E-3,5.4582E4,3.970405E3,1.32E2,1.3715873E3,7.704183E-1,1.5005797E4,1.4512526E-3,1.8408569E-2,1.4979574E-2,3.90106E0,2.7246006E7,5.7719237E-1,1.9109257E-2,1.1290139E-2,1.3563734E0,1.25218E5,-2.060812E-2,-3.1526084E-3,-9.047092E-3,-1.9981632E-2,3.8E1,4.242625E6,2.1483086E-3,1.2056166E-2,7.27E2,8.61491E3,4.0741867E-4,-3.9428077E-3,3.04386E-4,4.5085418E4,7.0076494E5,1.8466014E-1,-5.2466434E-3,-1.1395744E-2,4.9807886E3,9E0,7.667818E-3,3.536439E-3,9.2199E4,7.019E3,-4.9538156E-3,-1.2873521E-2,7.4970815E-3,8.2253466E9,2.9104478E0,3.0657873E-3,4.8810664E4,2.4037695E6,-1.0312157E-2,2.437E3,4.97E2,1.8122449E2,1.018711E-1,1.6358411E8,1.1670706E-3,1.0303075E1,1.11E2,6.736019E-3,5.23E2,-4.96482E-4,4.9778344E-3,1.020782E-2,1.71E2,8.728118E-3,-1.3604277E-2,-1.6664361E-3,1.0828989E8,1.1472081E0,4.699263E-3,1.5520698E-2,4.9E1,-7.7020503E-3,-9.220333E-3,1E0,4.0208E4,7.621583E2,1.5854122E-2,4.413664E7,-3.5264692E-3,5.5814706E5,9E2,3.1294638E2,1.6602717E-3,-6.157668E-3,3.6282136E0,1.2E1,1.464889E-3,8.707873E-3,1.1171382E7,1.0192E4,-3.2231293E-3,-2.7555152E-5,-7.7438834E-3,-1.5012881E-3,1.8048264E4,2.6777263E0,-1.4249119E-3,2.98E2,5.125467E-1,9.213145E0,1.2166998E0,-2.8967794E-3,1.35108685E-5,7.2896E5,8.5539725E8,6.438674E-3,-1.6005174E-3,3.402546E6,1.365173E-3,2.1434232E2,2.1332705E-4,1E0,1.7997802E0,4.823012E2,1.0950326E6,4.3467668E4,5.0795E4,5.76E3,1.1216729E2,3.5E1,1.09E3,1.3261502E0,-6.0972064E-3,5.015707E0,1.19E3,-1.1248145E-2,2.5800187E-3,-2.5009033E-3,-3.3803608E-3,-8.704097E-3,1.71525E4,2.5291866E1,1.803E3,-7.849194E-3,-2.9303753E-3,7.883E3,4.740524E-3,3.0701153E-4,3.7832818E0,2.0579956E9,1.3E1,6.610734E6,2.398451E-4,4.5680464E3,6.2041795E-3,2.7753687E-2,6.1472137E-3,6.0087295E-5,-1.6382147E-2,-5.565324E-3,4.87514E6,1.0013907E1,4.302613E6,1.02E3,-1.3308455E-2,-5.7317824E-3,1.7243155E12,-1.2206482E-2,-1.6426519E-3,-4.476969E-3,-3.95296E-4,4.419524E-3,6.5029144E5,1.1107701E-2,2.1789778E10,5.432393E-1,-2.3775047E-3,-6.7948895E-3,2.3481594E-3,6.2378272E-3,2.7677333E-3,-2.9144564E-3,-1.4171602E-3,1.6495816E-3,1.3212076E-3,6.791503E-3,-3.427422E-3,6.890505E-4,-1.8552571E-3,-3.943189E-3,4.7682407E-3,6.6703465E-4,-3.844464E-3,1.314265E-3,2.0412593E-3,8.97672E-3,5.518008E-3,1.0996663E-3,-4.1554514E-3,2.9318288E-3,-2.4832378E-4,-5.3485553E-3,-1.06268E-3,4.1686962E-3,1.1971354E-3,-3.7971162E-3,6.4959084E-3,2.4479285E-3,4.393313E-3,-8.171281E-3,-4.9851238E-5,8.301911E-3,-6.574952E-3,1.614671E-3,6.7210975E-3,1.0898049E-3,2.5326803E-3,-4.835849E-3,-5.7361173E-3,-7.517533E-4,-6.549023E-3,-2.3513623E-3,1.8311111E-3,-7.551844E-4,-4.6402593E-3,-1.2414708E-3,3.930305E-3,1.5483625E-4,-1.9255723E-3,2.7138065E-3,2.5110622E-3,7.0095723E-3,5.2300603E-3,-1.7987783E-3,5.800494E-3,1.0167263E-2,-1.1796041E-3,7.0765507E-3,1.2552691E-2,2.034662E-3,-8.44564E-3,-1.7365252E-3,-2.7609807E-3,-7.437458E-3,1.65991E-3,-8.174706E-3,-2.2848962E-4,-1.0692856E-2,-1.6922134E-3,-5.1534674E-3,1.3362907E-3,-3.899041E-3,-1.9337133E-3,3.4060767E-3,8.582391E-3,2.482647E-3,-3.1526906E-3,-5.596412E-4,-7.118462E-4,4.3460475E-3],"split_indices":[27,52,102,28,28,52,2,38,6,7,52,45,47,28,9,7,52,74,9,45,38,54,54,28,53,8,42,2,10,56,2,85,102,53,29,7,45,8,0,18,45,32,56,9,56,0,10,45,8,56,54,5,5,0,28,32,45,10,0,0,0,8,64,54,57,0,11,3,0,9,50,8,7,54,1,9,0,9,52,3,52,42,4,0,0,0,56,45,42,0,0,54,29,0,0,0,0,3,29,0,0,0,4,0,0,38,33,48,27,0,0,4,17,0,0,12,2,0,0,0,31,54,0,28,51,0,2,2,52,58,7,0,58,0,0,29,0,0,0,0,0,0,0,45,53,39,0,3,0,0,102,9,52,0,44,0,28,2,52,0,0,53,18,0,0,32,9,0,0,0,0,33,41,0,0,57,53,57,0,41,9,31,0,0,1,0,4,0,102,53,52,47,33,9,12,4,3,2,38,0,54,29,0,0,0,0,0,48,58,2,0,0,2,0,0,53,7,8,45,41,50,0,57,0,0,0,0,12,56,45,0,0,0,31,0,0,0,0,0,28,38,31,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.568E3,1.895E3,6.73E2,1.191E3,7.04E2,4.21E2,2.52E2,7.34E2,4.57E2,3.25E2,3.79E2,3.69E2,5.2E1,1.84E2,6.8E1,4.87E2,2.47E2,4.31E2,2.6E1,2.42E2,8.3E1,3.5E2,2.9E1,3.17E2,5.2E1,4E1,1.2E1,1.72E2,1.2E1,1.3E1,5.5E1,4.59E2,2.8E1,6E1,1.87E2,3.9E2,4.1E1,2.2E1,4E0,1.95E2,4.7E1,7.5E1,8E0,1.65E2,1.85E2,5E0,2.4E1,3.08E2,9E0,1.2E1,4E1,9E0,3.1E1,4E0,8E0,2.3E1,1.49E2,8E0,4E0,6E0,7E0,3.9E1,1.6E1,4.49E2,1E1,4E0,2.4E1,5.6E1,4E0,1.16E2,7.1E1,1.81E2,2.09E2,2.4E1,1.7E1,1.8E1,4E0,1.3E1,1.82E2,1E1,3.7E1,1.5E1,6E1,4E0,4E0,1.3E1,1.52E2,1.72E2,1.3E1,1.2E1,1.2E1,1.8E1,2.9E2,5E0,4E0,4E0,8E0,1.1E1,2.9E1,4E0,5E0,8E0,2.3E1,4E0,4E0,1.1E1,1.2E1,1.35E2,1.4E1,4E0,4E0,2.2E1,1.7E1,5E0,1.1E1,2.6E1,4.23E2,6E0,4E0,6E0,1.8E1,5.2E1,4E0,2.4E1,9.2E1,5E0,6.6E1,7.7E1,1.04E2,1.2E2,8.9E1,5E0,1.9E1,1.2E1,5E0,1.2E1,6E0,5E0,8E0,1.76E2,6E0,6E0,4E0,2E1,1.7E1,1.1E1,4E0,5.6E1,4E0,4E0,1.48E2,1.51E2,2.1E1,5E0,8E0,5E0,1.3E1,1.03E2,1.87E2,7E0,4E0,1.7E1,1.2E1,4E0,4E0,1.2E1,1.1E1,5E0,6E0,8E0,4E0,3.6E1,9.9E1,4E0,1E1,1.1E1,1.1E1,1.2E1,5E0,1.5E1,1.1E1,4.19E2,4E0,4E0,1.4E1,8E0,4.4E1,6E0,1.8E1,6.9E1,2.3E1,2.5E1,4.1E1,1.9E1,5.8E1,3.6E1,6.8E1,1.1E1,1.09E2,7E0,8.2E1,1.2E1,7E0,7E0,5E0,8E0,4E0,1.4E1,1.62E2,1.2E1,8E0,6E0,1.1E1,7E0,4E0,1.8E1,3.8E1,4E1,1.08E2,3.9E1,1.12E2,4E0,1.7E1,4E0,4E0,8E0,5E0,4.7E1,5.6E1,5.3E1,1.34E2,1.2E1,5E0,8E0,4E0,6E0,6E0,7E0,4E0,2.8E1,8E0,8.2E1,1.7E1,4E0,6E0,4E0,7E0,7E0,4E0,8E0,4E0,1.1E1,4E0,5E0,6E0,2.16E2,2.03E2,1E1,4E0,4E1,4E0,4E0,1.4E1,2.6E1,4.3E1,1.6E1,7E0,1.5E1,1E1,1.7E1,2.4E1,1.3E1,6E0,2E1,3.8E1,1E1,2.6E1,6E1,8E0,6E0,5E0,9.7E1,1.2E1,7.5E1,7E0,7E0,5E0,1E1,4E0,6E1,1.02E2,7E0,5E0,6E0,5E0,9E0,9E0,2.1E1,1.7E1,2.4E1,1.6E1,6E1,4.8E1,4E0,3.5E1,4E0,1.08E2,5E0,1.2E1,2.9E1,1.8E1,5.2E1,4E0,5E0,4.8E1,5.7E1,7.7E1,4E0,4E0,1.4E1,1.4E1,4E0,4E0,3E1,5.2E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"327","size_leaf_vector":"1"}},{"base_weights":[-8.8725955E-4,-1.6643835E-2,6.7079015E-2,2.6674091E-3,-7.428838E-2,1.6732606E-1,4.223082E-2,-1.8239222E-2,4.4475574E-2,-1.13321245E-1,-3.3763044E-2,1.3628587E-1,1.7739277E-2,-4.7156024E-3,7.6679155E-2,-5.523243E-2,-3.148925E-3,8.24317E-3,7.20186E-2,-1.02215044E-1,-2.8116122E-1,-2.439035E-2,-1.1339544E-1,1.5324937E-1,-3.607861E-3,2.0621948E-2,-1.03691444E-1,3.514931E-2,1.2641466E-1,-7.147923E-2,1.6139349E-2,2.0913789E-1,-7.80385E-3,1.9504992E-2,-8.782579E-2,1.7970142E-1,5.9695832E-2,-2.6514032E-1,-9.508967E-2,-1.2420527E-1,-2.1115491E-2,-9.9267915E-2,-1.1975673E-2,-1.4633457E-1,-8.4309635E-4,1.8099827E-1,6.372767E-2,4.232747E-3,1.5962783E-1,-4.7048885E-2,-1.5761917E-2,-6.417434E-2,4.3480504E-2,1.0473559E-1,2.962383E-1,-5.2773435E-2,-1.5899572E-1,5.7460256E-2,-4.7442008E-2,7.2097788E-3,1.2788845E-2,-2.3465106E-2,2.1399682E-2,9.873048E-3,1.60261E-1,-1.8425281E-1,-5.7884264E-3,3.3321977E-3,1.9963647E-1,-5.9753857E-3,8.205392E-2,-1.8134596E-2,-6.9681355E-3,-8.860523E-2,-1.3715563E-2,-2.1138033E-3,-9.057218E-3,-6.739479E-2,-1.1565509E-2,5.742959E-3,-5.179233E-2,-9.525105E-3,-4.635087E-3,2.3744056E-2,1.5096195E-1,9.89382E-2,-1.7448326E-3,-1.2153498E-2,8.6771645E-2,1.3497518E-2,5.264045E-4,-1.0858851E-2,-6.803792E-3,-6.8627734E-4,-5.589623E-3,6.949657E-2,1.5788775E-2,1.18483216E-1,-5.5993116E-3,6.7385165E-3,2.3130566E-2,-3.639035E-2,-1.0153039E-1,-1.9758299E-1,1.7249234E-3,1.792561E-2,9.426161E-3,-9.211474E-2,9.665952E-3,-2.7457515E-2,1.8630375E-1,6.951937E-2,-2.6446045E-3,1.8280342E-2,-1.0739396E-2,9.0379745E-2,1.2691288E-2,-3.5881756E-3,-1.1904374E-2,5.3740074E-3,-6.187972E-2,4.6257544E-3,1.0773832E-2,2.2069234E-2,-1.17727056E-1,9.386237E-2,-4.255716E-2,-5.273794E-2,-1.2402558E-1,-1.0007707E-1,1.4410063E-3,-1.2326912E-2,5.6378733E-2,-1.477935E-2,-9.620883E-2,5.8246184E-2,1.7281713E-1,5.0675873E-2,9.2602065E-3,5.3448096E-2,-3.218279E-2,1.3644499E-1,-1.2477168E-3,2.4943321E-3,-3.2972213E-2,7.124621E-3,8.5269734E-2,3.5964392E-2,-3.3798814E-2,1.3053203E-1,-3.294355E-3,-4.8772257E-2,2.9946446E-2,-2.2332948E-2,-6.262267E-2,-1.3860504E-1,-3.190233E-1,-5.5974327E-2,5.5876862E-2,-3.0398325E-4,-6.4794165E-3,-1.161234E-3,1.5884143E-3,-4.1442543E-2,2.2993244E-2,3.7070073E-3,1.3049153E-2,1.1481051E-1,1.3155193E-2,-7.537265E-2,2.1618579E-2,7.635139E-3,1.0155765E-1,6.2356945E-3,1.8930591E-3,2.0646437E-3,-6.873742E-3,-4.5652394E-3,3.498179E-2,2.6233755E-3,-1.9106914E-1,8.462738E-2,1.3141355E-2,-1.4684712E-2,5.024442E-2,-9.3348935E-2,-5.3836303E-3,-1.9164658E-1,-8.553069E-2,-4.872014E-2,-8.224655E-3,1.659512E-3,-1.8474099E-3,4.7012684E-3,-2.0976325E-2,8.711288E-2,-3.531476E-2,-5.113605E-2,1.1893501E-2,-1.2884548E-1,-2.4578318E-2,6.11899E-3,-2.5564685E-4,1.9659778E-1,1.1373468E-1,9.279562E-4,4.271491E-3,9.630257E-2,2.5684033E-3,-4.7452867E-2,5.5757046E-2,2.8511574E-3,9.436214E-3,-6.1400343E-2,6.1707414E-4,6.647071E-2,-5.0527924E-3,9.6259E-2,-1.3137184E-3,7.2547324E-2,1.6388485E-2,-3.9102933E-3,-1.2161692E-4,1.1120316E-2,1.1466006E-1,-1.8221177E-3,-7.877945E-3,6.1739874E-3,-2.6199818E-3,7.400375E-4,-4.3318225E-3,-2.853723E-4,-8.903441E-3,-2.148434E-2,-7.7399686E-3,-4.4816425E-3,-2.1573009E-4,1.7374457E-3,4.024785E-3,-3.095891E-3,-2.081116E-4,2.8366179E-3,-1.6501581E-3,1.5276785E-3,8.79791E-3,-1.6241533E-3,3.1443548E-3,-7.838683E-4,-7.4919187E-3,1.7230952E-4,4.3489747E-3,1.1386231E-3,-3.2345033E-3,2.3454686E-3,9.2159845E-3,4.0057325E-3,-9.590169E-4,-1.3063165E-2,-3.4183776E-3,5.133486E-3,4.6212686E-4,5.349431E-3,-2.166206E-3,-1.1784552E-3,-6.0027563E-3,2.223338E-3,-6.1175083E-3,-4.6917973E-3,-1.2709485E-2,2.4633252E-3,-5.3159734E-3,-4.2053894E-3,-5.686088E-4,7.6983165E-4,-2.283804E-3,5.335454E-3,2.0503108E-3,1.3215428E-4,-3.0510633E-3,-3.9415075E-3,1.0060228E-3,4.0688333E-3,-4.626068E-4,-8.666668E-3,-3.5654928E-3,-3.2828157E-3,6.888521E-4,4.7657513E-3,1.1155394E-2,6.7802905E-3,2.9068717E-3,1.0578835E-3,6.362836E-3,2.7120912E-3,-2.1046475E-3,-6.858167E-4,-3.8922622E-3,4.0058456E-3,2.7703223E-4,-4.3496428E-4,-4.750814E-3,4.799639E-4,5.4984116E-3,5.545523E-3,8.975062E-4,1.282429E-3,4.4337125E-3,4.1941865E-3,8.156868E-5,6.6736815E-3,3.1176007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,-1,83,85,87,89,91,-1,93,95,97,99,101,103,105,107,-1,-1,109,111,113,115,117,119,-1,121,123,125,-1,-1,127,-1,-1,-1,129,-1,131,133,-1,-1,-1,135,137,-1,139,141,-1,-1,143,-1,-1,-1,145,147,149,-1,-1,-1,151,153,155,-1,157,-1,159,161,163,165,167,169,171,-1,173,-1,-1,-1,-1,175,-1,-1,177,179,181,183,185,187,189,191,193,195,197,199,201,203,205,-1,207,209,211,-1,-1,213,215,217,219,221,223,-1,225,227,-1,229,231,233,235,237,-1,-1,-1,-1,239,241,-1,-1,243,245,247,249,251,253,-1,-1,-1,-1,-1,255,-1,257,259,-1,-1,261,263,265,267,269,271,-1,-1,-1,-1,273,275,277,279,281,283,285,-1,-1,287,289,-1,-1,291,293,295,297,-1,-1,299,-1,301,-1,303,-1,305,307,-1,-1,-1,309,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7597952E0,2.3289871E0,1.2010481E0,1.3713497E0,8.2662964E-1,5.109236E-1,6.3107985E-1,5.8374363E-1,5.213343E-1,4.6896577E-1,1.9054407E-1,3.081516E-1,0E0,4.1843405E-1,4.602958E-1,3.5250127E-1,7.368374E-1,2.47307E-1,3.8337696E-1,2.6733613E-1,3.0268526E-1,2.1540353E-1,8.3373606E-2,1.8222725E-1,0E0,3.023243E-1,3.929704E-1,1.0539842E-1,3.4449852E-1,3.9596438E-1,1.5247867E-1,1.1635661E-2,3.3392048E-1,2.7499154E-1,1.8878835E-1,5.427426E-2,3.939901E-1,7.229829E-2,2.5836945E-1,3.4391105E-2,0E0,1.23446554E-1,1.4210229E-1,2.9774696E-2,0E0,4.4962668E-1,7.292361E-2,1.633549E-1,2.2780615E-1,8.882625E-2,0E0,2.2519566E-2,8.175218E-2,2.874776E-1,2.3345691E-1,1.6005707E-1,3.2791483E-1,1.7814586E-1,5.9567258E-2,0E0,0E0,4.0291986E-1,2.9521978E-1,3.6680812E-1,6.2197328E-2,4.967332E-2,9.565892E-2,0E0,2.2819221E-2,2.1884644E-1,2.982874E-1,0E0,0E0,2.932812E-1,0E0,0E0,0E0,6.3136116E-2,0E0,1.2879243E-1,1.0050528E-1,0E0,0E0,0E0,1.0138166E-1,5.3148717E-2,0E0,1.3442941E-1,1.1441974E-1,0E0,0E0,2.9870454E-2,0E0,0E0,0E0,5.7436854E-2,5.8241785E-2,2.0191765E-1,0E0,0E0,0E0,1.2877385E-1,6.5743816E-1,2.1362448E-1,0E0,8.165632E-2,0E0,4.1218378E-2,8.746788E-3,3.3044717E-1,5.6153953E-2,2.154775E-1,3.033784E-1,1.641143E-1,0E0,1.1532009E-2,0E0,0E0,0E0,0E0,8.3793215E-2,0E0,0E0,8.522129E-2,1.8482144E-1,2.6358986E-1,4.3460017E-1,2.28347E-1,2.9076958E-1,5.4943576E-2,1.3432566E-2,9.769808E-2,1.0734687E-1,3.4548976E-2,6.2291026E-2,4.9452644E-2,3.946078E-2,1.0705797E-2,0E0,5.20828E-2,1.07387975E-1,4.746574E-2,0E0,0E0,2.1372959E-2,9.0234876E-2,6.0151935E-2,2.817297E-2,2.3515295E-2,9.4249606E-2,0E0,1.6978756E-1,1.9928284E-1,0E0,8.888924E-2,1.3574955E-1,1.3369739E-1,1.7030574E-2,7.092584E-3,0E0,0E0,0E0,0E0,2.796616E-1,1.9487232E-1,0E0,0E0,2.3717237E-1,9.067966E-2,1.890167E-1,1.5303802E-1,1.8350454E-1,8.640766E-2,0E0,0E0,0E0,0E0,0E0,1.2638487E-1,0E0,6.842536E-2,2.3795462E-1,0E0,0E0,7.9090476E-2,1.16764784E-1,3.3239642E-1,2.3357785E-1,2.2180861E-1,1.4924033E-2,0E0,0E0,0E0,0E0,8.798143E-2,2.099368E-2,9.935042E-3,3.1250786E-2,3.1641617E-2,3.6342353E-2,1.668195E-2,0E0,0E0,5.965805E-2,1.0408014E-2,0E0,0E0,2.7619801E-2,3.001672E-2,6.7726895E-2,1.42658055E-2,0E0,0E0,1.6919725E-2,0E0,2.1241914E-2,0E0,4.758361E-2,0E0,8.346215E-3,2.7773015E-2,0E0,0E0,0E0,6.280053E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,68,68,69,69,70,70,73,73,77,77,79,79,80,80,84,84,85,85,87,87,88,88,91,91,95,95,96,96,97,97,101,101,102,102,103,103,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,115,115,120,120,123,123,124,124,125,125,126,126,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,137,137,139,139,140,140,141,141,144,144,145,145,146,146,147,147,148,148,149,149,151,151,152,152,154,154,155,155,156,156,157,157,158,158,163,163,164,164,167,167,168,168,169,169,170,170,171,171,172,172,178,178,180,180,181,181,184,184,185,185,186,186,187,187,188,188,189,189,194,194,195,195,196,196,197,197,198,198,199,199,200,200,203,203,204,204,207,207,208,208,209,209,210,210,213,213,215,215,217,217,219,219,220,220,224,224],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,-1,84,86,88,90,92,-1,94,96,98,100,102,104,106,108,-1,-1,110,112,114,116,118,120,-1,122,124,126,-1,-1,128,-1,-1,-1,130,-1,132,134,-1,-1,-1,136,138,-1,140,142,-1,-1,144,-1,-1,-1,146,148,150,-1,-1,-1,152,154,156,-1,158,-1,160,162,164,166,168,170,172,-1,174,-1,-1,-1,-1,176,-1,-1,178,180,182,184,186,188,190,192,194,196,198,200,202,204,206,-1,208,210,212,-1,-1,214,216,218,220,222,224,-1,226,228,-1,230,232,234,236,238,-1,-1,-1,-1,240,242,-1,-1,244,246,248,250,252,254,-1,-1,-1,-1,-1,256,-1,258,260,-1,-1,262,264,266,268,270,272,-1,-1,-1,-1,274,276,278,280,282,284,286,-1,-1,288,290,-1,-1,292,294,296,298,-1,-1,300,-1,302,-1,304,-1,306,308,-1,-1,-1,310,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.631E3,2.461362E-3,5.473125E3,6.184386E5,4.3978744E0,8.791777E3,1E0,1.4735735E2,6.9432104E7,1.2427474E8,6.747114E7,5.5544252E7,1.7739277E-2,2.3580047E10,2.6312E4,4.264897E6,8E0,3.0273972E0,1.0258198E-2,8.494517E5,9.112E3,1.188E3,6.0218544E1,4.8E1,-3.607861E-3,4.5505118E2,3.869321E2,1.588909E5,4.516183E1,4.230769E0,1.89E2,1.46E2,1.4887473E-1,2.0619047E1,1.246E3,1E0,1.260173E6,1.5981E4,9.6819663E-1,8.756944E5,-2.1115491E-2,8.522844E5,1.2342778E1,2.09633E1,-8.4309635E-4,1.4563726E-1,5.12544E-1,6.7652373E3,3.2475834E7,5.3411217E0,-1.5761917E-2,7.44E2,1.3439535E1,5.68E2,1.6956E5,3.72424E5,1.16072E6,1.86E2,5.1086234E5,7.2097788E-3,1.2788845E-2,1.5137865E2,1.3642105E1,1.819149E1,1.7331858E1,1.7685275E6,8E0,3.3321977E-3,1.3707545E-4,1.0288468E6,2.0295714E7,-1.8134596E-2,-6.9681355E-3,2.1358025E0,-1.3715563E-2,-2.1138033E-3,-9.057218E-3,7.2419E4,-1.1565509E-2,9.557794E2,6.13E2,-9.525105E-3,-4.635087E-3,2.3744056E-2,1E0,8.1572165E6,-1.7448326E-3,1.1600905E7,1.0040462E8,1.3497518E-2,5.264045E-4,3.0161016E7,-6.803792E-3,-6.8627734E-4,-5.589623E-3,1.5121101E9,8.238194E1,2.095128E6,-5.5993116E-3,6.7385165E-3,2.3130566E-2,1.5272727E0,8.2E1,1.2335714E2,1.7249234E-3,9.51505E-1,9.426161E-3,1.9557823E0,3.4015749E0,2.777588E6,2.829581E10,1.1543406E-2,3.64E2,6.057671E6,-1.0739396E-2,4.86E2,1.2691288E-2,-3.5881756E-3,-1.1904374E-2,5.3740074E-3,1.954E3,4.6257544E-3,1.0773832E-2,4.6E1,1.1217151E0,1.7488463E3,9.07826E0,2.877528E2,2.863783E2,1.42E1,5.0939494E5,4.1639212E7,4.20737E5,1.2068E4,5.704E3,8E0,1.5846036E7,5.8036587E1,9.2602065E-3,8.24666E5,1.2427485E10,3.3487454E11,-1.2477168E-3,2.4943321E-3,9.6910744E7,2.1298597E0,9.462212E9,2.5570776E-2,7.2683E4,1.2068E4,-3.294355E-3,2.7661E4,2.14E2,-2.2332948E-2,7.108182E4,6.7E1,1E0,5.28806E5,1.411E3,-3.0398325E-4,-6.4794165E-3,-1.161234E-3,1.5884143E-3,1.645614E6,7.8E1,3.7070073E-3,1.3049153E-2,2.8331464E2,2.345785E5,1.857051E5,6.015E3,8.009E3,1.5E1,6.2356945E-3,1.8930591E-3,2.0646437E-3,-6.873742E-3,-4.5652394E-3,1.8722344E8,2.6233755E-3,2.1328075E-1,1E0,1.3141355E-2,-1.4684712E-2,1.7321888E1,6.072427E-1,1.1E1,4.44E2,1.6837112E7,6.7909607E2,-8.224655E-3,1.659512E-3,-1.8474099E-3,4.7012684E-3,9.360022E5,7.370838E-1,2.980289E-1,1.1347826E1,5.5759512E-2,3.202814E5,5.549173E6,6.11899E-3,-2.5564685E-4,9.966725E2,6E0,9.279562E-4,4.271491E-3,4.2037987E1,6.135771E9,5.2228385E6,1.5874315E10,2.8511574E-3,9.436214E-3,1.3E1,6.1707414E-4,2.8980975E5,-5.0527924E-3,3.666495E7,-1.3137184E-3,4.7687202E1,9.313E3,-3.9102933E-3,-1.2161692E-4,1.1120316E-2,8.2608955E-3,-1.8221177E-3,-7.877945E-3,6.1739874E-3,-2.6199818E-3,7.400375E-4,-4.3318225E-3,-2.853723E-4,-8.903441E-3,-2.148434E-2,-7.7399686E-3,-4.4816425E-3,-2.1573009E-4,1.7374457E-3,4.024785E-3,-3.095891E-3,-2.081116E-4,2.8366179E-3,-1.6501581E-3,1.5276785E-3,8.79791E-3,-1.6241533E-3,3.1443548E-3,-7.838683E-4,-7.4919187E-3,1.7230952E-4,4.3489747E-3,1.1386231E-3,-3.2345033E-3,2.3454686E-3,9.2159845E-3,4.0057325E-3,-9.590169E-4,-1.3063165E-2,-3.4183776E-3,5.133486E-3,4.6212686E-4,5.349431E-3,-2.166206E-3,-1.1784552E-3,-6.0027563E-3,2.223338E-3,-6.1175083E-3,-4.6917973E-3,-1.2709485E-2,2.4633252E-3,-5.3159734E-3,-4.2053894E-3,-5.686088E-4,7.6983165E-4,-2.283804E-3,5.335454E-3,2.0503108E-3,1.3215428E-4,-3.0510633E-3,-3.9415075E-3,1.0060228E-3,4.0688333E-3,-4.626068E-4,-8.666668E-3,-3.5654928E-3,-3.2828157E-3,6.888521E-4,4.7657513E-3,1.1155394E-2,6.7802905E-3,2.9068717E-3,1.0578835E-3,6.362836E-3,2.7120912E-3,-2.1046475E-3,-6.858167E-4,-3.8922622E-3,4.0058456E-3,2.7703223E-4,-4.3496428E-4,-4.750814E-3,4.799639E-4,5.4984116E-3,5.545523E-3,8.975062E-4,1.282429E-3,4.4337125E-3,4.1941865E-3,8.156868E-5,6.6736815E-3,3.1176007E-3],"split_indices":[2,27,32,28,53,52,102,52,7,45,45,45,0,5,2,9,18,53,38,45,10,2,56,3,0,56,57,47,54,56,10,29,38,56,2,102,50,9,27,28,0,48,54,58,0,38,39,52,50,54,0,0,54,3,2,9,1,0,51,0,0,56,58,56,58,45,8,0,38,51,50,0,0,56,0,0,0,1,0,52,0,0,0,0,102,51,0,51,51,0,0,45,0,0,0,5,58,11,0,0,0,56,29,52,0,58,0,53,56,1,31,39,10,28,0,12,0,0,0,0,2,0,0,0,53,4,56,4,52,58,28,5,29,9,2,3,47,53,0,29,12,31,0,0,45,38,12,57,11,9,0,29,0,0,28,10,102,45,29,0,0,0,0,9,8,0,0,4,28,28,2,29,3,0,0,0,0,0,7,0,42,6,0,0,56,27,3,0,45,47,0,0,0,0,48,57,27,53,27,28,51,0,0,4,8,0,0,58,5,47,12,0,0,3,0,28,0,51,0,56,2,0,0,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.575E3,2.091E3,4.84E2,1.567E3,5.24E2,9.5E1,3.89E2,1.045E3,5.22E2,2.66E2,2.58E2,8.3E1,1.2E1,1.65E2,2.24E2,3.02E2,7.43E2,2.26E2,2.96E2,2.51E2,1.5E1,2.32E2,2.6E1,7.7E1,6E0,1.32E2,3.3E1,1.23E2,1.01E2,2.46E2,5.6E1,1.5E1,7.28E2,2.03E2,2.3E1,2.9E1,2.67E2,9E0,2.42E2,8E0,7E0,3.2E1,2E2,1.9E1,7E0,5.8E1,1.9E1,1.19E2,1.3E1,2.7E1,6E0,9E0,1.14E2,9.1E1,1E1,2.04E2,4.2E1,3.4E1,2.2E1,8E0,7E0,4.74E2,2.54E2,1.91E2,1.2E1,1E1,1.3E1,5E0,2.4E1,6.8E1,1.99E2,4E0,5E0,2.35E2,7E0,4E0,4E0,2.7E1,5E0,1.39E2,6.1E1,9E0,1E1,4E0,5.4E1,1.4E1,5E0,1E2,1.9E1,7E0,6E0,2E1,7E0,5E0,4E0,5.8E1,5.6E1,8.6E1,5E0,6E0,4E0,1.54E2,5E1,3.5E1,7E0,2.7E1,7E0,1.2E1,1E1,4.66E2,8E0,8.4E1,1.7E2,1.85E2,6E0,8E0,4E0,4E0,6E0,4E0,9E0,4E0,2E1,5.5E1,1.3E1,1.82E2,1.7E1,1.18E2,1.17E2,1.8E1,9E0,1.03E2,3.6E1,3.4E1,2.7E1,1.1E1,4.3E1,1E1,4E0,2.3E1,7.7E1,1.3E1,6E0,5E0,1.5E1,1.2E1,4.6E1,4E1,1.6E1,8.1E1,5E0,1.3E2,2.4E1,4E0,4.6E1,2.5E1,1E1,9E0,1.8E1,4E0,8E0,4E0,6E0,3.65E2,1.01E2,4E0,4E0,4.6E1,3.8E1,4.2E1,1.28E2,1.65E2,2E1,4E0,4E0,4E0,5E0,5E0,5E1,4E0,9E0,1.74E2,8E0,4E0,1.3E1,6.3E1,5.5E1,4.1E1,7.6E1,1.1E1,7E0,5E0,4E0,7E0,9.6E1,2.7E1,9E0,1.4E1,2E1,1.8E1,9E0,5E0,6E0,2.9E1,1.4E1,6E0,4E0,1.2E1,1.1E1,6.6E1,1.1E1,6E0,7E0,9E0,6E0,8E0,4E0,4.2E1,4E0,1.3E1,2.7E1,6E0,1E1,1E1,7.1E1,1.18E2,1.2E1,1.1E1,1.3E1,1.1E1,3.5E1,6E0,1.9E1,5E0,5E0,5E0,4E0,1.1E1,7E0,2.35E2,1.3E2,6.3E1,3.8E1,2E1,2.6E1,2E1,1.8E1,2.4E1,1.8E1,1.01E2,2.7E1,1.37E2,2.8E1,1.3E1,7E0,2.7E1,2.3E1,5E0,4E0,1.4E2,3.4E1,8E0,5E0,1.8E1,4.5E1,3.9E1,1.6E1,1.7E1,2.4E1,1E1,6.6E1,5E0,6E0,3.9E1,5.7E1,1.8E1,9E0,4E0,5E0,1E1,4E0,4E0,1.6E1,9E0,9E0,4E0,5E0,7E0,2.2E1,9E0,5E0,4E0,8E0,5E0,6E0,3.2E1,3.4E1,7E0,4E0,4E0,5E0,4E0,4E0,3.5E1,7E0,4E0,9E0,4E0,2.3E1,5.1E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"311","size_leaf_vector":"1"}},{"base_weights":[-4.4161707E-4,-4.9960535E-2,2.284483E-2,-6.815056E-2,1.3132654E-2,9.784168E-3,1.03165254E-1,-4.9330093E-2,-1.425994E-1,5.438832E-2,-9.471315E-2,-1.0365369E-2,4.9821354E-2,1.7132303E-1,7.329065E-2,-1.3259563E-1,-3.4533083E-2,-1.2281374E-1,-2.935255E-1,1.4158013E-2,3.746803E-2,-4.9714576E-2,-1.9939542E-1,-6.232974E-2,-6.678694E-4,8.572898E-2,2.0967703E-2,1.1702107E-1,2.2693598E-1,9.0485215E-2,-2.5420979E-2,-1.642637E-2,-1.1713243E-1,-6.749957E-2,-1.5397179E-2,-9.878249E-2,-2.0035098E-1,-2.180449E-2,-4.413565E-3,5.343015E-2,-1.265679E-2,-8.358995E-3,-2.6115336E-2,-9.347533E-2,-1.961394E-2,-5.2741036E-2,-1.4653081E-2,5.5655185E-2,-6.8587908E-3,7.425369E-2,2.1973956E-1,-1.8191753E-1,2.9216845E-2,5.417221E-2,1.3149264E-1,1.9613432E-2,1.929067E-1,1.7501941E-2,8.063117E-2,4.368464E-2,-1.0954393E-2,3.4055035E-3,-1.3197714E-1,-8.607692E-3,-1.03467986E-1,-2.4760494E-2,6.1583135E-2,-3.1475846E-2,-1.1310345E-1,-2.3045416E-1,-1.3904127E-3,1.6436534E-2,8.243572E-2,4.0973462E-2,-8.070591E-2,6.616622E-3,-6.252743E-3,-7.882031E-3,-1.7095713E-3,-2.378692E-2,-9.3920544E-2,-6.744003E-2,1.2301076E-1,-2.440325E-2,1.3840179E-2,8.110186E-2,-6.808053E-2,1.3128172E-2,4.8526744E-3,-1.1537077E-3,-1.3459915E-2,-2.6725207E-2,5.579784E-2,8.761616E-4,3.996087E-3,1.42894E-1,3.0065586E-3,2.3167449E-1,4.4134003E-3,6.295146E-2,1.734009E-1,-2.4043165E-2,1.1381675E-1,-1.4070317E-2,-1.1536942E-1,5.73639E-3,-1.9282296E-2,-7.2634086E-2,-1.9551276E-1,-4.3045253E-2,1.4205516E-2,-2.3171802E-3,8.502965E-2,5.2383835E-3,-1.153593E-1,-1.9253449E-1,-8.742922E-2,-1.7693028E-1,-1.8839896E-2,8.103764E-2,-3.0472192E-3,9.4121225E-2,-2.4268476E-3,9.509952E-2,-1.1837343E-2,-6.810887E-3,-2.8133977E-4,-2.3535956E-3,2.240177E-2,-6.143845E-2,2.4813453E-2,-1.2596181E-1,2.4932485E-2,3.3477058E-3,-9.06998E-2,1.2694819E-2,7.078987E-2,-3.5633814E-2,4.8528634E-2,-2.6356418E-2,3.413958E-2,5.5133168E-2,1.2742887E-1,-1.0231466E-2,2.8566257E-3,8.49575E-3,-3.825584E-2,8.8455126E-2,6.9405204E-3,8.325876E-3,1.0131792E-1,4.819928E-3,1.3279557E-2,1.0773143E-1,2.8291892E-2,9.392516E-3,4.3097087E-3,1.7393024E-3,-3.1302052E-3,2.0813812E-3,8.876431E-3,-9.0043686E-5,-1.2750517E-1,-4.421253E-2,2.2389103E-2,-1.0269335E-2,-5.8099862E-2,-1.2930837E-1,-1.8616308E-2,-1.3126951E-1,-2.8754193E-2,-8.361499E-2,2.640522E-2,5.963393E-4,1.0748411E-1,-1.630765E-3,-1.0397991E-2,-1.2467575E-2,-3.771698E-3,-4.8407484E-2,-1.0419803E-1,-2.0833725E-1,-2.6784483E-3,5.350584E-3,1.6776605E-3,-1.6513385E-2,2.1376736E-3,1.0335653E-2,7.660987E-2,1.8812474E-3,6.672004E-3,-1.8341681E-3,1.0180566E-3,-4.4373027E-3,4.4764403E-2,-2.7883494E-2,-1.4011143E-1,-2.5334237E-3,4.236837E-2,-2.0836402E-1,-8.347953E-2,3.016177E-3,-2.5441335E-3,-1.6856985E-1,-4.6725906E-2,1.1806847E-1,-1.20489036E-4,-3.765718E-2,5.672868E-3,1.2491925E-1,-6.183872E-3,4.4924304E-2,-5.7328973E-2,9.4130844E-2,5.263885E-3,6.566549E-2,-7.57148E-2,1.600283E-1,8.1254475E-2,-6.9972076E-2,1.7202755E-2,9.690035E-2,-5.3257262E-3,5.749606E-2,-3.9904095E-2,6.764707E-3,2.2328799E-3,8.064045E-2,1.3509372E-2,7.834185E-3,1.58033E-2,-9.098585E-3,-4.7244527E-3,2.6993905E-3,-3.0879304E-3,-7.3902804E-4,3.884221E-3,-1.8072488E-3,-7.078925E-3,-1.2349257E-3,-8.778199E-3,-8.657669E-3,8.297899E-4,4.930735E-4,-2.9254344E-3,-1.7393472E-3,-5.7809846E-3,2.860496E-3,-3.4749473E-4,6.1871917E-3,2.5701905E-3,-3.0083387E-4,-3.6291485E-3,-7.031591E-3,-2.686427E-3,-1.2405585E-2,-4.3365993E-3,1.6947484E-3,-1.4748499E-3,2.3473527E-3,8.056305E-3,-1.5677069E-3,1.4378137E-3,3.4302825E-3,1.1840655E-4,-2.860163E-3,2.3340299E-3,-9.969445E-3,-3.914157E-3,3.5710048E-3,-7.2507473E-4,-4.898672E-3,-1.2471641E-2,1.705544E-3,-5.2432367E-3,-1.514514E-3,-1.3646657E-2,-3.4071289E-3,1.1349047E-3,9.6217E-4,7.491807E-3,3.157118E-3,-2.5359292E-3,-2.486703E-3,-4.4025955E-4,9.217178E-3,5.892995E-4,-6.9663757E-3,1.428446E-3,6.4128386E-3,1.0174873E-3,-2.1399471E-3,-9.299507E-3,-1.5289444E-3,5.9147873E-3,7.683432E-3,-1.8695933E-4,4.4138883E-3,1.5727165E-3,-6.1163367E-3,-2.2682793E-4,1.0735324E-2,5.266912E-3,5.9497583E-4,7.7163926E-3,-2.6212544E-3,-1.13721825E-2,-3.605413E-3,2.312075E-3,4.3510064E-3,1.3787213E-2,-9.86551E-4,4.010817E-3,-1.0474977E-3,-8.219737E-3,3.6791805E-4,5.1695537E-3,1.5668306E-3,-6.98436E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,-1,-1,71,73,-1,75,77,-1,79,-1,81,83,85,87,89,91,93,95,-1,97,-1,99,101,-1,-1,103,105,107,109,111,113,115,117,-1,119,121,123,125,127,-1,-1,-1,129,131,133,135,137,139,141,143,-1,-1,-1,-1,145,147,-1,-1,149,-1,151,-1,153,155,157,159,-1,161,-1,163,165,167,169,171,-1,173,-1,175,177,179,181,-1,183,185,187,-1,189,191,-1,-1,-1,193,195,197,199,201,-1,203,-1,205,207,209,211,213,215,217,-1,-1,-1,219,221,223,-1,225,-1,-1,227,229,-1,-1,-1,-1,-1,-1,-1,231,233,235,-1,237,239,-1,241,243,245,247,-1,249,-1,-1,-1,-1,251,253,255,-1,-1,-1,257,-1,-1,259,-1,-1,-1,-1,261,263,265,267,-1,269,271,273,-1,-1,275,277,279,281,283,-1,285,287,289,291,293,295,297,299,301,303,305,307,309,-1,311,313,-1,-1,315,-1,-1,317,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9346778E0,9.350953E-1,1.8148751E0,8.784995E-1,8.193641E-1,1.2025452E0,4.80201E-1,6.188539E-1,3.4319496E-1,5.046365E-1,2.3032531E-1,4.9985147E-1,5.1519954E-1,1.9389558E-1,2.9084176E-1,1.920104E-1,2.708543E-1,1.9262695E-1,3.718394E-1,0E0,1.0068856E-1,9.829594E-2,2.701764E-1,3.2996315E-1,2.9223663E-1,3.2507718E-1,4.7058395E-1,2.8348088E-2,1.2150526E-1,3.4295058E-1,3.5780996E-1,0E0,2.0703411E-1,3.3379245E-1,1.9902155E-1,8.2226634E-2,1.2861896E-1,0E0,0E0,1.00582E-1,1.16441056E-1,0E0,1.04546875E-1,3.649895E-2,0E0,1.7780566E-1,0E0,6.9646126E-1,2.745177E-1,2.0631552E-1,6.411749E-2,1.4658284E-1,4.0000603E-1,7.1057603E-3,1.6609907E-2,0E0,1.0522115E-1,0E0,2.2034407E-1,9.7734414E-2,0E0,0E0,1.339941E-1,8.257355E-2,2.6393294E-1,1.7517006E-1,7.9880126E-2,2.0647946E-1,1.32038E-1,1.05908036E-1,0E0,5.4369837E-2,8.570698E-2,5.2577656E-2,5.7674013E-2,2.2254627E-2,0E0,0E0,0E0,1.6621354E-1,2.3924899E-1,9.988725E-2,3.5158038E-1,3.3672732E-1,2.8404027E-1,2.299757E-1,1.829065E-1,0E0,0E0,0E0,0E0,2.0388085E-1,2.897758E-1,0E0,0E0,8.774102E-3,0E0,7.578063E-2,0E0,1.8319857E-1,1.1730671E-2,2.7171148E-2,3.866084E-2,0E0,8.522016E-2,0E0,6.020592E-2,1.3291767E-1,2.3315746E-1,2.0678204E-1,9.648498E-2,0E0,3.974986E-2,0E0,7.7595875E-2,9.455395E-2,3.225386E-2,5.8551252E-2,0E0,8.766323E-3,2.1689361E-2,8.0881596E-2,0E0,1.4842458E-2,8.759208E-3,0E0,0E0,0E0,1.2202481E-2,1.307265E-1,5.5175066E-2,1.5413696E-1,4.069757E-2,0E0,8.1153914E-2,0E0,1.3411593E-1,1.1142099E-1,2.3064615E-1,2.6088175E-1,3.9966336E-1,1.8137291E-1,9.2365146E-2,0E0,0E0,0E0,1.4719558E-1,1.8986887E-1,1.7754517E-1,0E0,1.7100446E-2,0E0,0E0,2.0730817E-1,1.08249165E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.351779E-2,6.553622E-2,4.6420917E-2,0E0,1.2134059E-1,8.781338E-2,0E0,1.4572394E-1,1.6742818E-1,1.0064311E-2,7.3450774E-2,0E0,1.1033058E-2,0E0,0E0,0E0,0E0,1.9071408E-2,6.428057E-2,4.641646E-2,0E0,0E0,0E0,1.8706884E-2,0E0,0E0,1.0275674E-1,0E0,0E0,0E0,0E0,9.849602E-3,1.0976825E-2,8.3702415E-2,3.942257E-2,0E0,5.5657316E-2,4.8318148E-2,8.937773E-2,0E0,0E0,1.2181115E-1,2.8374381E-2,7.268664E-2,5.7696067E-2,1.2217566E-1,0E0,1.5275961E-1,1.5687089E-1,7.114434E-2,1.4517048E-1,2.3041731E-1,2.1054591E-1,8.9460194E-2,3.2678552E-2,9.7126245E-2,1.554725E-1,1.3435632E-1,8.3865464E-2,1.6060919E-1,0E0,6.491077E-2,8.858939E-2,0E0,0E0,7.615736E-2,0E0,0E0,1.5926203E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,39,39,40,40,42,42,43,43,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,58,58,59,59,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,75,75,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,91,91,92,92,95,95,97,97,99,99,100,100,101,101,102,102,104,104,106,106,107,107,108,108,109,109,110,110,112,112,114,114,115,115,116,116,117,117,119,119,120,120,121,121,123,123,124,124,128,128,129,129,130,130,131,131,132,132,134,134,136,136,137,137,138,138,139,139,140,140,141,141,142,142,146,146,147,147,148,148,150,150,153,153,154,154,162,162,163,163,164,164,166,166,167,167,169,169,170,170,171,171,172,172,174,174,179,179,180,180,181,181,185,185,188,188,193,193,194,194,195,195,196,196,198,198,199,199,200,200,203,203,204,204,205,205,206,206,207,207,209,209,210,210,211,211,212,212,213,213,214,214,215,215,216,216,217,217,218,218,219,219,220,220,221,221,223,223,224,224,227,227,230,230],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,-1,-1,72,74,-1,76,78,-1,80,-1,82,84,86,88,90,92,94,96,-1,98,-1,100,102,-1,-1,104,106,108,110,112,114,116,118,-1,120,122,124,126,128,-1,-1,-1,130,132,134,136,138,140,142,144,-1,-1,-1,-1,146,148,-1,-1,150,-1,152,-1,154,156,158,160,-1,162,-1,164,166,168,170,172,-1,174,-1,176,178,180,182,-1,184,186,188,-1,190,192,-1,-1,-1,194,196,198,200,202,-1,204,-1,206,208,210,212,214,216,218,-1,-1,-1,220,222,224,-1,226,-1,-1,228,230,-1,-1,-1,-1,-1,-1,-1,232,234,236,-1,238,240,-1,242,244,246,248,-1,250,-1,-1,-1,-1,252,254,256,-1,-1,-1,258,-1,-1,260,-1,-1,-1,-1,262,264,266,268,-1,270,272,274,-1,-1,276,278,280,282,284,-1,286,288,290,292,294,296,298,300,302,304,306,308,310,-1,312,314,-1,-1,316,-1,-1,318,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,1.2679E4,3.1066042E7,9.302862E7,7.733016E5,4.52737E5,8.87E2,2.3920168E6,2.66E2,6.0435E4,3.3870136E7,1.4492002E0,2.0718E4,4.719849E7,7.7660305E-5,1.638058E2,9.6910744E7,2.142857E0,1.4158013E-2,7.2234793E0,1.9469026E0,1.4299594E8,7.221312E-2,3.0769231E0,8.072373E1,4.8590577E-5,1.8382221E-1,2.1592189E5,1.27365E4,2.1897722E10,-1.642637E-2,4E0,1.2062E5,1E0,2.3428571E0,4.982003E6,-2.180449E-2,-4.413565E-3,3.130603E3,5.6202265E6,-8.358995E-3,9.190713E-1,2.822137E6,-1.961394E-2,3.1E1,-1.4653081E-2,2.35E2,1.545974E-1,3.3709784E5,4E0,7E0,3.136758E9,8.3304755E8,4.347826E-2,1.9613432E-2,1.7384394E7,1.7501941E-2,4.091015E9,1.4689211E7,-1.0954393E-2,3.4055035E-3,2.35E2,1.1E1,4.9363803E5,1.1908646E0,1.1376085E-4,9.0763354E-1,7.080933E2,1.97729E5,-1.3904127E-3,1.18827E5,7.083E3,1.462136E6,7.203918E9,1.0017953E8,-6.252743E-3,-7.882031E-3,-1.7095713E-3,7.3E1,1.79E2,2.645E3,2.04E5,1.903E3,2.7547675E2,7.704183E-1,3.5551153E5,1.3128172E-2,4.8526744E-3,-1.1537077E-3,-1.3459915E-2,2.8373447E3,2E0,8.761616E-4,3.996087E-3,2.200423E3,3.0065586E-3,3.8758245E-1,4.4134003E-3,1.2469E5,6.3340454E8,2.325945E9,2.5932198E6,-1.4070317E-2,6.451758E-5,5.73639E-3,1.0797824E6,1.2561812E6,2.8107285E-1,1E0,1E0,-2.3171802E-3,5.650399E6,5.2383835E-3,4.67E2,2.7538432E8,1.1214854E1,1.191572E6,-1.8839896E-2,6.722251E1,8.185918E0,1.5476604E0,-2.4268476E-3,2.6573448E9,1.1925198E3,-6.810887E-3,-2.8133977E-4,-2.3535956E-3,2.1E1,1.139605E6,3.2E1,1.62848E5,9.00042E-6,3.3477058E-3,1.0972222E0,1.2694819E-2,1.07E2,9.438605E-6,1.8071064E7,1.6254681E0,5.839E3,1E0,7.857143E0,-1.0231466E-2,2.8566257E-3,8.49575E-3,1.2577342E6,2.0319266E-7,1.08103E5,8.325876E-3,7E0,4.819928E-3,1.3279557E-2,1.22159E5,7.673114E8,9.392516E-3,4.3097087E-3,1.7393024E-3,-3.1302052E-3,2.0813812E-3,8.876431E-3,-9.0043686E-5,1.6254681E0,1.6633E4,1E0,-1.0269335E-2,4.50847E5,5.84E2,-1.8616308E-2,5.5871747E2,1.8344E4,1.6880698E7,4.768812E1,5.963393E-4,8.63332E5,-1.630765E-3,-1.0397991E-2,-1.2467575E-2,-3.771698E-3,3.998805E-3,4.05E0,2.035483E6,-2.6784483E-3,5.350584E-3,1.6776605E-3,1.8576577E3,2.1376736E-3,1.0335653E-2,8.985569E-1,1.8812474E-3,6.672004E-3,-1.8341681E-3,1.0180566E-3,2.5682794E5,1.5265896E8,4.4312353E0,2.6E1,-2.5334237E-3,1.2266316E3,6E0,1.01644066E2,3.016177E-3,-2.5441335E-3,6E0,3.0596492E0,1.2E1,7.102631E1,2.302366E6,5.672868E-3,1E0,1.7321888E1,9E0,6.480713E6,1.2354571E0,1.2245615E0,1.0894445E-2,2.45009E6,1.9701008E5,1.0989723E3,2.4E1,1.6118722E0,3.5E1,-5.3257262E-3,5.9674234E0,2.1827742E6,6.764707E-3,2.2328799E-3,6.4341316E0,1.3509372E-2,7.834185E-3,1.0746194E4,-9.098585E-3,-4.7244527E-3,2.6993905E-3,-3.0879304E-3,-7.3902804E-4,3.884221E-3,-1.8072488E-3,-7.078925E-3,-1.2349257E-3,-8.778199E-3,-8.657669E-3,8.297899E-4,4.930735E-4,-2.9254344E-3,-1.7393472E-3,-5.7809846E-3,2.860496E-3,-3.4749473E-4,6.1871917E-3,2.5701905E-3,-3.0083387E-4,-3.6291485E-3,-7.031591E-3,-2.686427E-3,-1.2405585E-2,-4.3365993E-3,1.6947484E-3,-1.4748499E-3,2.3473527E-3,8.056305E-3,-1.5677069E-3,1.4378137E-3,3.4302825E-3,1.1840655E-4,-2.860163E-3,2.3340299E-3,-9.969445E-3,-3.914157E-3,3.5710048E-3,-7.2507473E-4,-4.898672E-3,-1.2471641E-2,1.705544E-3,-5.2432367E-3,-1.514514E-3,-1.3646657E-2,-3.4071289E-3,1.1349047E-3,9.6217E-4,7.491807E-3,3.157118E-3,-2.5359292E-3,-2.486703E-3,-4.4025955E-4,9.217178E-3,5.892995E-4,-6.9663757E-3,1.428446E-3,6.4128386E-3,1.0174873E-3,-2.1399471E-3,-9.299507E-3,-1.5289444E-3,5.9147873E-3,7.683432E-3,-1.8695933E-4,4.4138883E-3,1.5727165E-3,-6.1163367E-3,-2.2682793E-4,1.0735324E-2,5.266912E-3,5.9497583E-4,7.7163926E-3,-2.6212544E-3,-1.13721825E-2,-3.605413E-3,2.312075E-3,4.3510064E-3,1.3787213E-2,-9.86551E-4,4.010817E-3,-1.0474977E-3,-8.219737E-3,3.6791805E-4,5.1695537E-3,1.5668306E-3,-6.98436E-3],"split_indices":[20,52,2,45,45,28,29,2,28,0,10,7,53,2,45,42,52,45,54,0,57,53,45,38,58,58,41,38,28,33,5,0,11,1,105,56,29,0,0,4,47,0,27,29,0,10,0,2,38,33,6,8,31,5,57,0,51,0,7,1,0,0,2,10,33,42,38,27,4,11,0,29,0,32,12,45,0,0,0,29,3,9,5,0,55,41,33,0,0,0,0,47,17,0,0,52,0,38,0,9,32,5,32,0,38,0,51,45,27,64,100,0,45,0,10,7,56,29,0,58,54,53,0,12,52,0,0,0,8,31,4,7,39,0,53,0,3,37,45,53,29,74,56,0,0,0,51,37,9,0,8,0,0,2,12,0,0,0,0,0,0,0,53,1,8,0,32,0,0,52,9,32,58,0,1,0,0,0,0,39,54,32,0,0,0,55,0,0,27,0,0,0,0,28,45,54,29,0,47,8,33,0,0,3,53,3,51,9,0,100,56,3,12,53,53,38,9,33,52,8,53,3,0,58,28,0,0,54,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.543E3,8.13E2,1.73E3,6.31E2,1.82E2,1.489E3,2.41E2,5.05E2,1.26E2,1.32E2,5E1,9.91E2,4.98E2,7.2E1,1.69E2,7.5E1,4.3E2,1.13E2,1.3E1,8E0,1.24E2,3.6E1,1.4E1,1.55E2,8.36E2,2.21E2,2.77E2,3.8E1,3.4E1,1.44E2,2.5E1,4E0,7.1E1,1.57E2,2.73E2,8.8E1,2.5E1,7E0,6E0,9.4E1,3E1,5E0,3.1E1,1E1,4E0,1.5E2,5E0,8.2E1,7.54E2,2.05E2,1.6E1,1E1,2.67E2,8E0,3E1,4E0,3E1,4E0,1.4E2,1.9E1,6E0,5E0,6.6E1,6E1,9.7E1,2.44E2,2.9E1,1.6E1,7.2E1,2.1E1,4E0,4.2E1,5.2E1,1.7E1,1.3E1,2.4E1,7E0,4E0,6E0,8.9E1,6.1E1,2.9E1,5.3E1,4.08E2,3.46E2,1.96E2,9E0,1.1E1,5E0,4E0,6E0,8.6E1,1.81E2,4E0,4E0,2.5E1,5E0,2.1E1,9E0,1.19E2,2.1E1,1E1,9E0,5E0,6.1E1,4E0,5.6E1,7.4E1,2.3E1,1.66E2,7.8E1,5E0,2.4E1,6E0,1E1,1.6E1,5.6E1,1.7E1,4E0,9E0,3.3E1,4.8E1,4E0,8E0,9E0,7E0,6E0,5E0,1.9E1,5E1,3.9E1,4.8E1,1.3E1,4E0,2.5E1,1.4E1,3.9E1,3.54E2,5.4E1,1.16E2,2.3E2,1.27E2,6.9E1,4E0,5E0,4E0,8.2E1,1.08E2,7.3E1,1.4E1,1.1E1,5E0,1.6E1,5.1E1,6.8E1,1.7E1,4E0,4E0,6E0,5E0,4E0,6E0,5.5E1,3.5E1,2.1E1,6E0,6.8E1,1.8E1,5E0,2.2E1,1.44E2,8E0,7E1,6E0,1.8E1,6E0,4E0,1E1,6E0,1.8E1,3.8E1,1.3E1,4E0,5E0,4E0,2.6E1,7E0,5E0,4.3E1,4E0,4E0,5E0,4E0,9E0,1E1,3.6E1,1.4E1,7E0,3.2E1,1.5E1,3.3E1,9E0,4E0,8E0,1.7E1,2.3E1,1.6E1,3.5E2,4E0,2.2E1,3.2E1,3.5E1,8.1E1,7.4E1,1.56E2,1.18E2,9E0,3.9E1,3E1,5.2E1,3E1,1.04E2,4E0,3.5E1,3.8E1,6E0,5E0,4.5E1,6E0,5E0,6.3E1,1.9E1,3.6E1,5E0,3E1,1.3E1,8E0,5.5E1,1.3E1,6E0,1.2E1,1.7E1,5E0,6.3E1,8.1E1,4E0,4E0,3.6E1,3.4E1,1.3E1,5E0,7E0,1.1E1,2.1E1,1.7E1,9E0,4E0,5E0,2.1E1,3.3E1,1E1,5E0,4E0,6E0,4E0,2.6E1,1E1,6E0,8E0,2.1E1,1.1E1,5E0,1E1,5E0,2.8E1,4E0,4E0,1.3E1,4E0,6E0,1.7E1,7E0,9E0,2.46E2,1.04E2,1.4E1,8E0,6E0,2.6E1,7E0,2.8E1,7.4E1,7E0,1.2E1,6.2E1,8E0,1.48E2,7E1,4.8E1,5E0,4E0,1.8E1,2.1E1,1.6E1,1.4E1,4.8E1,4E0,7E0,2.3E1,1E2,4E0,8E0,2.7E1,3.4E1,4E0,1.1E1,3.4E1,5.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"319","size_leaf_vector":"1"}},{"base_weights":[-1.0729837E-3,1.8125245E-2,-5.406302E-2,-1.4425238E-2,5.1388185E-2,-9.769929E-2,-1.52064245E-2,-2.5600262E-2,3.552973E-2,9.687363E-3,8.485895E-2,-1.116853E-1,-1.9381687E-2,-2.3471829E-2,1.2771885E-1,-3.859442E-2,2.4679072E-2,-1.1059393E-2,4.2832628E-2,2.2390634E-2,-7.0936635E-2,7.554139E-2,2.339065E-1,-9.237217E-2,-2.0748577E-1,1.04543656E-1,-4.9134657E-2,-4.840101E-2,6.529175E-3,9.275625E-3,1.8140225E-3,-3.0134374E-2,-1.0268374E-1,1.2895982E-1,7.246437E-3,1.1157153E-1,2.4322093E-2,1.2529346E-1,1.1655613E-2,-1.0029205E-1,1.7944658E-2,1.0494383E-1,5.110466E-2,3.0401635E-1,1.550132E-4,-2.5863054E-1,-8.35995E-2,-7.930685E-2,-2.981047E-1,1.061002E-2,-1.3551998E-4,-1.030901E-1,-1.04795685E-2,-3.8896136E-2,-1.3089718E-1,-5.192943E-2,2.8330833E-2,-5.889388E-2,-1.5750885E-2,-8.0586866E-2,-1.9120466E-2,1.1136853E-3,1.6529742E-1,-5.4759637E-2,3.5590403E-2,2.6505236E-2,1.5850008E-1,4.5548316E-2,-7.174427E-2,1.0581966E-2,7.472225E-2,6.447738E-3,1.0950928E-2,-6.785643E-2,-2.1528703E-1,-1.479911E-3,5.294396E-2,1.187545E-1,2.2666195E-2,1.8844387E-2,8.114057E-2,1.6867567E-2,7.6282537E-3,-1.6931437E-2,-4.7263824E-3,-6.709875E-2,-1.4957702E-1,-4.359915E-2,-8.879582E-3,-1.0990712E-1,-2.0429322E-2,-1.3904916E-1,5.507063E-4,-4.313115E-2,2.848724E-2,-1.3341854E-2,-7.6108314E-2,-7.2815314E-2,-1.3938869E-2,-1.9342903E-2,-7.942454E-2,-8.951364E-3,6.3869044E-2,-9.992392E-3,-5.2981425E-2,1.3060057E-1,-2.1050937E-2,2.0305606E-2,-1.1202119E-1,4.9591814E-3,9.608624E-3,-2.2164715E-2,-1.7905492E-1,5.8803868E-2,-3.6500145E-2,-1.8356926E-3,3.923893E-3,1.2375361E-1,1.2933204E-2,1.1804443E-1,2.7521254E-2,-2.0512752E-1,1.6104273E-2,-1.7004186E-6,1.0110883E-1,-2.315143E-2,2.947003E-2,-9.156622E-2,5.7467225E-4,-4.2548054E-3,-1.51081085E-2,3.9086433E-4,4.3709152E-3,1.0141693E-1,2.1482348E-1,-1.461773E-2,7.6232165E-2,2.5728125E-2,-7.983246E-3,9.6180364E-2,-2.5303647E-2,-1.8838835E-1,-5.1320966E-2,-1.832057E-1,-2.02818E-2,-5.693644E-3,-1.0653516E-2,-1.046977E-2,2.5759551E-3,-1.2967442E-3,-9.315331E-3,2.429742E-3,-5.2107223E-3,-8.134115E-4,2.643501E-3,9.238041E-3,-5.8773354E-2,-1.1267988E-1,-3.279753E-2,-5.56044E-4,-9.423529E-2,-2.6969027E-3,3.2829054E-2,-1.1058835E-1,-1.8659905E-2,5.2297458E-2,-2.1525048E-2,7.6509E-2,-7.422019E-4,9.387378E-3,-6.628841E-2,-1.3626121E-4,9.501677E-3,4.2348497E-2,-2.9896062E-2,-1.563259E-3,2.9754331E-3,-1.779809E-2,-6.6122405E-2,2.1919128E-3,-7.2108484E-3,-1.1710766E-2,-4.393433E-3,1.1334708E-1,3.1447046E-2,-6.975383E-2,3.741831E-3,-2.467125E-4,1.5840042E-1,1.623662E-1,4.8621677E-2,8.285155E-3,1.925708E-2,-1.7706901E-3,-1.5606989E-2,2.7700036E-3,-2.0647539E-2,5.281641E-2,7.4368264E-3,-3.0464664E-2,8.74943E-2,3.5617266E-2,-9.196796E-2,-5.033729E-3,-1.6429373E-3,4.1632894E-3,-3.4215506E-3,6.6200584E-2,1.2885445E-1,2.6363394E-1,2.9901622E-3,1.5132886E-1,2.1932576E-2,7.849798E-3,1.6755553E-2,1.4652802E-1,6.0378138E-2,1.7606974E-2,-7.0170596E-2,-8.150344E-4,-2.3038806E-1,-3.642173E-2,-1.22260906E-1,-9.496161E-2,-2.478741E-1,-4.1896184E-3,1.8023255E-3,2.1365122E-3,-2.133801E-3,2.3865E-3,-2.167969E-3,-3.543148E-2,5.9042107E-2,-7.2106724E-3,-3.529023E-2,-1.3426705E-3,-1.2524773E-1,2.0566424E-3,-5.0757054E-2,-1.4767585E-3,-7.155718E-3,2.9361222E-3,1.6523007E-5,-7.5486884E-3,-2.8120577E-3,7.042485E-4,-2.3825201E-3,-2.9728355E-4,4.6029524E-3,-5.1389027E-2,8.005376E-3,4.2938698E-2,1.0970457E-1,-1.3256328E-3,1.04255015E-2,-1.1793712E-2,-2.8644775E-3,4.44191E-3,4.8704472E-4,-4.1895328E-4,-3.0440325E-3,-1.7034044E-3,-7.5633735E-3,5.27045E-3,-1.339918E-3,-1.20597106E-4,7.192074E-3,3.602608E-3,-2.3868326E-5,-3.2552163E-4,-6.2966743E-3,3.8064937E-3,9.062603E-3,3.913606E-3,1.0248254E-2,-2.2499204E-4,5.128917E-3,-1.2953518E-3,2.2350664E-3,1.1545424E-3,-3.011582E-3,7.066394E-4,3.809567E-3,-1.0933199E-3,-8.087082E-3,7.620183E-4,7.1072187E-3,2.0750645E-3,-4.5720916E-3,-8.287052E-3,1.545352E-5,4.450862E-3,-8.105242E-3,8.676769E-3,3.2888828E-3,1.4357705E-2,6.5100216E-3,1.0404493E-2,2.9842227E-3,-3.1628744E-3,2.977185E-3,-3.8301789E-3,1.2963478E-3,8.568062E-3,1.4785284E-3,8.437003E-3,2.022639E-3,-1.3170134E-3,2.5637809E-3,-5.2735074E-3,-1.0376362E-3,-5.114139E-3,-1.3954795E-2,-3.930275E-3,-4.8511164E-4,-8.004384E-3,-3.7772E-3,-7.1019838E-3,-3.3673937E-3,-1.4821889E-2,-2.5990165E-3,-6.5627805E-4,-5.7270243E-3,5.732E-3,3.134742E-4,4.0820558E-4,-3.192462E-3,-1.1033227E-2,-5.1820227E-3,-6.6729233E-4,-3.4335235E-3,-5.4825707E-3,-1.2915405E-3,-1.5094458E-3,2.0080998E-3,3.6408294E-3,-1.2496847E-3,6.3973665E-3,1.5153263E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,-1,-1,91,93,95,97,99,101,103,105,107,-1,-1,109,111,113,115,117,119,121,-1,123,125,-1,127,129,-1,131,133,135,137,139,-1,-1,-1,-1,141,143,145,-1,147,-1,149,-1,151,153,155,157,159,-1,161,163,165,167,-1,169,171,173,175,177,-1,-1,179,181,183,185,-1,-1,187,-1,189,191,193,195,-1,197,199,201,203,205,-1,-1,-1,-1,207,209,-1,211,213,-1,215,217,219,221,223,225,-1,227,-1,-1,-1,-1,229,-1,-1,-1,231,233,235,237,-1,239,-1,241,243,245,247,249,251,-1,253,255,-1,-1,257,259,-1,-1,-1,261,263,-1,-1,-1,265,267,269,-1,-1,271,273,275,-1,277,-1,-1,-1,279,281,-1,283,285,287,289,-1,-1,-1,-1,291,293,295,-1,297,299,-1,301,303,305,307,309,-1,311,313,315,317,319,-1,-1,-1,-1,-1,-1,321,323,-1,325,-1,327,-1,329,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,331,333,335,337,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6164963E0,2.0449255E0,1.1582752E0,5.340364E-1,1.3028162E0,3.5053825E-1,4.3182248E-1,5.1120925E-1,3.3775598E-1,4.2867494E-1,7.017076E-1,4.8594046E-1,1.8949404E-1,2.5759375E-1,9.915003E-2,3.3361095E-1,2.9174554E-1,0E0,2.1486568E-1,3.9732334E-1,1.5052113E-1,3.4598303E-1,4.8450613E-1,3.1238186E-1,4.9905288E-1,1.1619592E-1,8.443142E-2,1.4232236E-1,2.0156683E-1,0E0,0E0,2.2658652E-1,4.1460437E-1,8.5467786E-2,2.4624115E-1,1.3951224E-1,2.8058898E-1,1.3288295E-1,3.530503E-1,1.457785E-1,2.6537972E-2,2.4843287E-1,2.5912297E-1,6.528711E-2,0E0,1.1270696E-1,2.2838414E-1,6.2527075E-2,4.9699974E-1,0E0,0E0,7.1289405E-2,3.3036884E-2,1.6064277E-1,1.4308643E-1,3.699042E-2,1.5294012E-1,1.4205825E-1,2.881959E-1,2.1793577E-1,0E0,0E0,1.1233717E-2,1.7429268E-1,1.6279957E-1,4.894851E-2,4.8153818E-2,1.4312316E-1,2.9624152E-1,0E0,4.5372367E-2,2.1942472E-1,0E0,5.683969E-2,7.623774E-2,0E0,1.3339188E-2,2.9301238E-1,5.7950145E-1,1.6518094E-1,2.2495878E-1,0E0,0E0,0E0,0E0,3.2905334E-1,1.8324941E-1,3.586948E-2,0E0,1.8774816E-1,0E0,6.482521E-2,0E0,3.925985E-2,1.3978523E-2,1.05178475E-1,1.0559645E-1,1.8771335E-2,0E0,3.971417E-2,4.1490555E-2,4.5022734E-2,5.9844702E-2,0E0,1.475218E-1,1.0375102E-1,2.0090228E-1,3.6281202E-2,5.592361E-1,0E0,0E0,1.0742082E-1,2.4760187E-2,1.05361015E-1,9.4120055E-2,0E0,0E0,8.5498124E-2,0E0,5.979815E-2,1.00808285E-1,1.7087376E-1,2.4273438E-2,0E0,3.132145E-2,1.1675959E-1,1.3839361E-1,1.0687783E-2,6.284325E-2,0E0,0E0,0E0,0E0,1.4800191E-1,1.9603372E-1,0E0,1.1493485E-1,1.4706892E-1,0E0,2.1106493E-1,3.589033E-2,1.4078552E-1,1.6254535E-1,1.704861E-1,3.8845655E-2,0E0,2.2368975E-2,0E0,0E0,0E0,0E0,2.076556E-2,0E0,0E0,0E0,1.555979E-1,6.320168E-2,3.6536336E-2,4.5921337E-2,0E0,3.0612081E-2,0E0,7.565465E-3,2.4479568E-2,9.227485E-3,2.3743825E-2,4.2742394E-2,5.21231E-2,0E0,2.3456043E-1,2.0853615E-1,0E0,0E0,6.590556E-2,2.0838404E-1,0E0,0E0,0E0,1.17715105E-1,9.571004E-2,0E0,0E0,0E0,8.5335225E-2,6.4926535E-2,6.678219E-2,0E0,0E0,1.4514893E-2,3.0626088E-2,2.9162774E-2,0E0,1.01292625E-1,0E0,0E0,0E0,1.699607E-2,7.98149E-3,0E0,1.4645137E-1,3.3425577E-2,1.3380899E-1,6.14113E-2,0E0,0E0,0E0,0E0,3.9588875E-1,2.427001E-1,2.8962016E-2,0E0,5.0111115E-2,6.153482E-2,0E0,1.04986966E-1,1.4079142E-1,1.5073287E-1,1.616981E-2,1.353503E-2,0E0,7.157922E-2,1.4816621E-1,3.5686344E-2,1.1229128E-2,1.6602743E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.3822165E-2,9.622934E-2,0E0,3.5326514E-2,0E0,4.296434E-2,0E0,1.721865E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2922573E-2,3.1960838E-2,5.672622E-2,3.2488316E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,73,73,74,74,76,76,77,77,78,78,79,79,80,80,85,85,86,86,87,87,89,89,91,91,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,104,104,105,105,106,106,107,107,108,108,111,111,112,112,113,113,114,114,117,117,119,119,120,120,121,121,122,122,124,124,125,125,126,126,127,127,128,128,133,133,134,134,136,136,137,137,139,139,140,140,141,141,142,142,143,143,144,144,146,146,151,151,155,155,156,156,157,157,158,158,160,160,162,162,163,163,164,164,165,165,166,166,167,167,169,169,170,170,173,173,174,174,178,178,179,179,183,183,184,184,185,185,188,188,189,189,190,190,192,192,196,196,197,197,199,199,200,200,201,201,202,202,207,207,208,208,209,209,211,211,212,212,214,214,215,215,216,216,217,217,218,218,220,220,221,221,222,222,223,223,224,224,231,231,232,232,234,234,236,236,238,238,249,249,250,250,251,251,252,252],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,-1,-1,92,94,96,98,100,102,104,106,108,-1,-1,110,112,114,116,118,120,122,-1,124,126,-1,128,130,-1,132,134,136,138,140,-1,-1,-1,-1,142,144,146,-1,148,-1,150,-1,152,154,156,158,160,-1,162,164,166,168,-1,170,172,174,176,178,-1,-1,180,182,184,186,-1,-1,188,-1,190,192,194,196,-1,198,200,202,204,206,-1,-1,-1,-1,208,210,-1,212,214,-1,216,218,220,222,224,226,-1,228,-1,-1,-1,-1,230,-1,-1,-1,232,234,236,238,-1,240,-1,242,244,246,248,250,252,-1,254,256,-1,-1,258,260,-1,-1,-1,262,264,-1,-1,-1,266,268,270,-1,-1,272,274,276,-1,278,-1,-1,-1,280,282,-1,284,286,288,290,-1,-1,-1,-1,292,294,296,-1,298,300,-1,302,304,306,308,310,-1,312,314,316,318,320,-1,-1,-1,-1,-1,-1,322,324,-1,326,-1,328,-1,330,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,332,334,336,338,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,4.3256033E2,3.6616542E0,1.5859042E6,1.5166431E5,1.0052E4,1.1803382E4,7.997723E6,5.798305E1,4.2343444E7,6.082581E3,7.8351054E11,4.9122E5,4.0465977E1,3.832274E2,9.337562E0,2.4946861E4,-1.1059393E-2,8.916961E2,4.1E0,1E0,3.642857E0,2.7753594E7,1.381869E2,6.439228E-1,1.0935709E6,2.7E1,1E0,1E0,9.275625E-3,1.8140225E-3,1.05E2,2E1,1.5147E4,1.7247218E-1,6.7909607E2,1.8E1,1.9E1,1.8552577E2,1.9125667E3,1.2792622E8,2.0295714E7,7.885718E2,1.10032365E-1,1.550132E-4,3.958068E2,1.9041E4,1.0828989E8,2.0469315E0,1.061002E-2,-1.3551998E-4,2.098129E-1,1.0017953E8,1.2342778E1,7.398769E4,8.352216E-2,1.0323588E3,6E0,1.2E1,2.5207965E2,-1.9120466E-2,1.1136853E-3,4.64109E2,1.1151842E6,1.7E1,2.5464671E2,7.427971E2,3.4078704E7,4.39E2,1.0581966E-2,1.4E1,5.8965025E5,1.0950928E-2,1.3715873E3,5E0,-1.479911E-3,3.373528E4,1.6503105E3,3.171E3,9.0510735E1,2.7246006E7,1.6867567E-2,7.6282537E-3,-1.6931437E-2,-4.7263824E-3,5.85E2,7.806314E6,3.74E3,-8.879582E-3,2.1226158E0,-2.0429322E-2,8.35338E-1,5.507063E-4,1.5190727E5,1.6052323E5,3.1826714E7,1.6410715E1,3E1,-1.3938869E-2,4.3359457E10,2.3033286E5,9.360022E5,7.978062E6,-9.992392E-3,3.938775E4,3.6E2,1.5756359E0,1.56E2,4.0724753E2,4.9591814E-3,9.608624E-3,1E0,6.34E2,2.25E2,3.05402E5,-1.8356926E-3,3.923893E-3,2.0211798E6,1.2933204E-2,2.0588236E0,1.0620689E0,5.3E1,3.627907E0,-1.7004186E-6,5.445593E2,6.520433E6,1.428111E5,1.0963991E7,2.652E3,-4.2548054E-3,-1.51081085E-2,3.9086433E-4,4.3709152E-3,3.324E3,1.4543829E7,-1.461773E-2,2.53383E5,8E0,-7.983246E-3,8.4413904E-1,3.7448005E3,1E0,6.971004E7,6.6911E4,4.47682E5,-5.693644E-3,1.747E3,-1.046977E-2,2.5759551E-3,-1.2967442E-3,-9.315331E-3,2.512E3,-5.2107223E-3,-8.134115E-4,2.643501E-3,6.072483E2,2.0619047E1,2.545E3,3.8910418E6,-5.56044E-4,1.71E2,-2.6969027E-3,3.0972284E1,9.2312753E-1,3.544243E8,1.479E3,1.1323028E0,2.6646748E3,-7.422019E-4,2.4360857E4,6.9436204E-1,-1.3626121E-4,9.501677E-3,2.66E2,2.3416922E9,-1.563259E-3,2.9754331E-3,-1.779809E-2,2.7465258E1,1.1E1,-7.2108484E-3,-1.1710766E-2,-4.393433E-3,5.7E1,2.4251762E5,1.729767E2,3.741831E-3,-2.467125E-4,2.183847E0,1.7486666E3,2.9806678E0,8.285155E-3,5.23896E5,-1.7706901E-3,-1.5606989E-2,2.7700036E-3,2.2214102E6,3.1183178E8,7.4368264E-3,2.002E3,2.0676143E3,5.68E2,4.6368E4,-5.033729E-3,-1.6429373E-3,4.1632894E-3,-3.4215506E-3,2.653E3,7.1913286E8,8.2619705E1,2.9901622E-3,2.3977574E7,4.9422E5,7.849798E-3,4.551E4,1.4383178E1,3.4653E4,1E0,3.555E3,-8.150344E-4,8E-3,1.048E3,7.298614E2,1.7440945E0,3.3012173E9,-4.1896184E-3,1.8023255E-3,2.1365122E-3,-2.133801E-3,2.3865E-3,-2.167969E-3,1.2471935E7,1.4560842E-2,-7.2106724E-3,9.71709E5,-1.3426705E-3,8.542683E1,2.0566424E-3,7.326E3,-1.4767585E-3,-7.155718E-3,2.9361222E-3,1.6523007E-5,-7.5486884E-3,-2.8120577E-3,7.042485E-4,-2.3825201E-3,-2.9728355E-4,4.6029524E-3,6E2,2.821E3,8.749966E1,3.8879236E7,-1.3256328E-3,1.04255015E-2,-1.1793712E-2,-2.8644775E-3,4.44191E-3,4.8704472E-4,-4.1895328E-4,-3.0440325E-3,-1.7034044E-3,-7.5633735E-3,5.27045E-3,-1.339918E-3,-1.20597106E-4,7.192074E-3,3.602608E-3,-2.3868326E-5,-3.2552163E-4,-6.2966743E-3,3.8064937E-3,9.062603E-3,3.913606E-3,1.0248254E-2,-2.2499204E-4,5.128917E-3,-1.2953518E-3,2.2350664E-3,1.1545424E-3,-3.011582E-3,7.066394E-4,3.809567E-3,-1.0933199E-3,-8.087082E-3,7.620183E-4,7.1072187E-3,2.0750645E-3,-4.5720916E-3,-8.287052E-3,1.545352E-5,4.450862E-3,-8.105242E-3,8.676769E-3,3.2888828E-3,1.4357705E-2,6.5100216E-3,1.0404493E-2,2.9842227E-3,-3.1628744E-3,2.977185E-3,-3.8301789E-3,1.2963478E-3,8.568062E-3,1.4785284E-3,8.437003E-3,2.022639E-3,-1.3170134E-3,2.5637809E-3,-5.2735074E-3,-1.0376362E-3,-5.114139E-3,-1.3954795E-2,-3.930275E-3,-4.8511164E-4,-8.004384E-3,-3.7772E-3,-7.1019838E-3,-3.3673937E-3,-1.4821889E-2,-2.5990165E-3,-6.5627805E-4,-5.7270243E-3,5.732E-3,3.134742E-4,4.0820558E-4,-3.192462E-3,-1.1033227E-2,-5.1820227E-3,-6.6729233E-4,-3.4335235E-3,-5.4825707E-3,-1.2915405E-3,-1.5094458E-3,2.0080998E-3,3.6408294E-3,-1.2496847E-3,6.3973665E-3,1.5153263E-3],"split_indices":[27,52,53,28,28,2,4,9,47,45,52,31,29,58,56,54,50,0,33,58,102,54,47,47,27,28,8,105,102,0,0,52,6,1,41,47,3,3,56,4,7,50,52,57,0,52,9,45,56,0,0,34,45,54,33,38,52,52,10,4,0,0,33,51,3,4,33,7,10,0,56,47,0,52,8,0,28,52,2,54,45,0,0,0,0,2,48,2,0,54,0,27,0,33,33,45,54,3,0,19,33,48,47,0,45,2,54,2,55,0,0,74,0,10,1,0,0,28,0,54,54,0,56,0,33,1,28,47,10,0,0,0,0,29,47,0,9,3,0,41,4,104,45,29,9,0,0,0,0,0,0,0,0,0,0,4,56,29,45,0,0,0,56,27,7,2,42,55,0,28,53,0,0,0,31,0,0,0,58,3,0,0,0,0,28,52,0,0,54,33,56,0,9,0,0,0,28,7,0,0,4,3,11,0,0,0,0,29,5,56,0,47,9,0,12,53,9,66,0,0,57,2,52,54,5,0,0,0,0,0,0,45,38,0,29,0,52,0,9,0,0,0,0,0,0,0,0,0,0,0,2,58,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E3,1.887E3,6.83E2,9.54E2,9.33E2,3.21E2,3.62E2,7.8E2,1.74E2,4.16E2,5.17E2,2.72E2,4.9E1,3.43E2,1.9E1,6.2E2,1.6E2,4E0,1.7E2,3.6E2,5.6E1,4.88E2,2.9E1,2.28E2,4.4E1,9E0,4E1,1.87E2,1.56E2,1.1E1,8E0,5.49E2,7.1E1,2.2E1,1.38E2,3.5E1,1.35E2,3.3E1,3.27E2,4.2E1,1.4E1,2.2E2,2.68E2,2.2E1,7E0,1E1,2.18E2,1.9E1,2.5E1,4E0,5E0,1.6E1,2.4E1,1.69E2,1.8E1,4.2E1,1.14E2,1.82E2,3.67E2,6.7E1,4E0,6E0,1.6E1,4.3E1,9.5E1,1.3E1,2.2E1,1.11E2,2.4E1,1.1E1,2.2E1,3.2E2,7E0,3.4E1,8E0,6E0,8E0,1.88E2,3.2E1,1.3E2,1.38E2,1.7E1,5E0,6E0,4E0,1.76E2,4.2E1,1.5E1,4E0,1E1,1.5E1,1.2E1,4E0,1.3E1,1.1E1,1.01E2,6.8E1,1.4E1,4E0,2E1,2.2E1,5.6E1,5.8E1,6E0,1.76E2,1.2E1,3.55E2,1.6E1,5.1E1,6E0,1E1,3.5E1,8E0,7.2E1,2.3E1,6E0,7E0,1.8E1,4E0,2.1E1,9E1,9E0,1.5E1,6E0,1.6E1,1.4E2,1.8E2,2.5E1,9E0,4E0,4E0,4E0,4E0,1.61E2,2.7E1,4E0,2.8E1,1.26E2,4E0,1.21E2,1.7E1,1.9E1,1.57E2,3.3E1,9E0,4E0,1.1E1,6E0,4E0,4E0,8E0,8E0,5E0,4E0,7E0,6.8E1,3.3E1,3.6E1,3.2E1,4E0,1E1,1.2E1,8E0,1.4E1,8E0,9E0,4.7E1,5E1,8E0,3.1E1,1.45E2,4E0,8E0,4.3E1,3.12E2,7E0,9E0,7E0,4.4E1,3E1,5E0,4E0,4E0,2.3E1,4.9E1,1.8E1,5E0,4E0,1.4E1,1.2E1,9E0,4E0,8.6E1,4E0,5E0,7E0,8E0,9E0,7E0,1.32E2,8E0,1.72E2,8E0,2.1E1,4E0,4E0,5E0,7.2E1,8.9E1,2E1,7E0,1.1E1,1.7E1,7E0,1.19E2,4.9E1,7.2E1,9E0,8E0,4E0,1.5E1,1.31E2,2.6E1,1.5E1,1.8E1,4E0,5E0,4E0,7E0,4E0,4E0,3.6E1,3.2E1,6E0,2.7E1,5E0,3.1E1,6E0,2.6E1,5E0,5E0,4E0,4E0,7E0,7E0,4E0,4E0,4E0,5E0,2.3E1,2.4E1,2.6E1,2.4E1,2.7E1,4E0,6E0,1.39E2,1.7E1,2.6E1,1.85E2,1.27E2,3.3E1,1.1E1,6E0,2.4E1,5E0,1.8E1,2.1E1,2.8E1,9E0,9E0,4E0,1E1,5E0,7E0,5E0,4E0,3.1E1,5.5E1,4E0,4E0,4E0,5E0,1.25E2,7E0,4E0,4E0,1.65E2,7E0,4E0,4E0,6.6E1,6E0,5.1E1,3.8E1,1.6E1,4E0,6E0,5E0,5E0,1.2E1,1E1,1.09E2,4E1,9E0,1E1,6.2E1,4E0,5E0,4E0,4E0,5E0,1E1,5E1,8.1E1,1.3E1,1.3E1,4E0,1.1E1,1.4E1,4E0,2.9E1,7E0,1.5E1,1.7E1,1.1E1,1.6E1,4E0,2.7E1,9E0,1.7E1,6E0,1.7E1,1.1E1,1.3E1,1.8E1,8E0,1.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"339","size_leaf_vector":"1"}},{"base_weights":[1.34153E-3,-4.5859776E-2,2.2866493E-2,-6.0784165E-2,9.332063E-3,2.360403E-3,6.0479175E-2,-6.626139E-2,1.4158252E-1,-1.0333706E-2,1.7041836E-2,-2.4195805E-2,2.5236418E-2,5.5840753E-2,2.4445792E-1,-5.86805E-2,-1.5695131E-1,2.303745E-1,-5.875736E-3,8.2109645E-2,2.773168E-3,-2.0978536E-2,-1.5046861E-2,2.2520944E-2,1.7562138E-2,6.476114E-2,-2.3760764E-2,1.1675908E-3,1.7621566E-2,-6.895544E-2,-3.725308E-3,-1.0495615E-1,-3.648246E-1,1.53414905E-2,4.48528E-3,4.1226167E-2,1.05205625E-2,-4.7314268E-2,1.8340264E-2,-4.9529307E-2,-4.722939E-3,2.9348683E-2,-8.7158814E-2,3.7537966E-2,8.806021E-2,-4.7060154E-2,8.295904E-3,-1.620953E-1,-6.1732866E-2,6.466225E-2,-2.3900144E-2,-1.4342903E-1,3.1006755E-3,-1.0069925E-2,-2.27424E-2,1.0775331E-1,1.2280149E-3,-8.301438E-2,4.309742E-2,-3.0524325E-2,3.3014342E-2,-3.5157397E-2,-1.2810586E-1,4.5228142E-2,-3.1387568E-2,3.5307694E-3,6.5371476E-2,-5.7717156E-2,-1.1229888E-2,5.1396894E-3,8.498457E-2,2.516846E-1,7.83611E-2,-6.157259E-3,-1.4053403E-1,-2.437195E-1,-5.5035602E-2,-8.438863E-2,-3.4948193E-2,7.4219937E-3,2.120899E-2,6.514775E-2,-4.311578E-2,-3.8703915E-2,-1.8927209E-1,7.7551194E-3,1.9362133E-3,2.2364493E-2,-1.8850856E-3,-2.4029477E-2,-1.1680335E-1,4.362904E-3,-4.669591E-5,-6.4765452E-3,-4.568765E-3,5.155736E-3,2.4034115E-2,-2.5258876E-2,-1.1294229E-1,-1.906361E-2,-7.22975E-2,-6.383989E-2,6.975178E-2,1.895222E-2,-4.4152725E-2,1.974441E-2,-4.3100733E-2,9.086156E-2,2.8149627E-2,-8.206035E-2,2.8675655E-3,-1.4953052E-1,1.4770885E-2,9.691958E-2,-5.0438777E-2,6.4997324E-3,1.5571535E-2,-1.9183751E-2,8.596759E-2,-5.2765522E-2,9.4703406E-2,-3.0752692E-2,-1.4638797E-2,-2.3799154E-3,-1.550712E-2,1.6271604E-2,-8.823757E-3,-5.768001E-2,-1.2582688E-1,-1.9633288E-2,-8.666631E-2,-2.3038131E-3,3.558092E-3,6.5618358E-3,3.6834902E-4,-5.5614427E-2,3.2500535E-2,-4.3745423E-3,8.290487E-4,-1.115104E-2,-3.0821362E-3,-7.6007585E-5,2.5648647E-3,1.3530992E-3,-3.1287041E-3,-2.4070274E-3,-6.8644523E-3,-1.910491E-3,3.305934E-2,1.698308E-2,4.4518514E-3,-7.223983E-2,-8.494662E-3,-1.0118862E-2,-3.7533022E-2,1.1562149E-3,-9.15779E-2,-1.0311181E-2,-2.0703984E-2,1.7925687E-1,3.6700252E-2,-6.6512994E-2,4.2166106E-2,-7.439527E-2,-1.6060047E-2,-1.5174481E-2,6.2213987E-2,2.2948718E-2,-7.652113E-2,4.4720303E-2,1.2407808E-1,1.1604886E-2,1.7111057E-2,-4.9258158E-2,-6.4765825E-3,1.9836163E-3,-1.543239E-2,5.8093075E-2,-1.1099106E-2,8.241329E-2,1.820813E-1,2.6654713E-3,-7.201935E-3,6.3382564E-3,-7.2747305E-2,9.752246E-2,9.230085E-3,-7.7216886E-2,-1.0313854E-3,4.0317062E-4,8.3848545E-3,2.479695E-3,-4.1858824E-3,-1.8568782E-3,4.3883123E-3,-4.2186347E-3,-1.4328045E-3,-4.702554E-3,-9.85432E-3,-1.8259232E-4,-3.651853E-3,-1.2210787E-3,-6.794858E-3,-2.2034112E-3,-7.6323836E-3,-8.234283E-4,3.7465263E-3,3.1616178E-3,-1.8826085E-4,5.0048233E-4,4.362266E-3,-1.7420162E-3,-6.6866977E-3,2.0653321E-3,-1.5474919E-3,2.4577118E-3,-4.7025657E-3,-5.590315E-3,-1.1079417E-3,-3.8503956E-3,1.1074546E-3,1.0541033E-2,6.414429E-3,2.867584E-3,-7.870605E-4,-5.4420047E-3,-1.1645982E-4,2.9070182E-3,-2.3781944E-3,-2.8055648E-3,-6.904389E-3,-2.3339964E-3,4.2051703E-4,-2.5177107E-4,-1.4179358E-2,2.465007E-3,1.0272109E-2,-7.4152863E-3,2.7490985E-3,-1.4375433E-3,-6.0024858E-3,3.093676E-3,-3.0732031E-3,6.882451E-3,-1.0226335E-4,-9.010014E-3,1.5031492E-3,-4.198394E-4,-3.2768736E-3,2.0604827E-3,6.884948E-3,-4.054457E-3,4.2530373E-4,9.8232216E-5,5.3097266E-3,1.1667224E-2,2.653786E-3,-5.9415847E-3,2.6279937E-3,3.6521854E-3,7.4161068E-3,2.064034E-3,-5.2063847E-3,-4.5747296E-3,-1.5434865E-3,-1.4432909E-3,1.6288793E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,35,37,39,-1,41,-1,43,45,-1,-1,47,49,51,53,-1,-1,55,-1,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,83,-1,-1,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,-1,111,113,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,-1,-1,141,-1,143,145,-1,-1,147,-1,-1,149,151,153,-1,155,157,159,161,163,165,167,169,171,173,-1,175,177,179,181,-1,-1,183,185,187,189,191,-1,-1,-1,193,-1,195,197,199,201,-1,-1,-1,-1,203,205,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,207,209,-1,211,213,-1,215,-1,217,-1,219,221,223,225,227,229,231,233,235,237,239,241,243,-1,245,247,-1,-1,-1,249,251,253,255,-1,-1,-1,257,259,261,263,265,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6081212E0,6.628257E-1,1.3596398E0,7.11004E-1,2.9014897E-1,6.943759E-1,5.171461E-1,4.1498375E-1,4.1830668E-1,0E0,1.5482028E-1,4.650202E-1,5.36026E-1,4.3326557E-1,3.506173E-1,3.221643E-1,4.697044E-1,1.1256707E-1,0E0,1.4885442E-1,1.08468555E-1,2.4295965E-1,0E0,4.5936558E-1,0E0,3.4364223E-1,2.8197613E-1,0E0,0E0,3.1356668E-1,1.2709884E-1,2.5949094E-1,4.1018605E-2,0E0,0E0,6.3730784E-2,0E0,1.10092044E-1,7.701428E-2,2.1006224E-1,4.4761515E-1,5.345384E-1,1.3285848E-1,3.9000908E-1,4.4685888E-1,2.1324238E-1,0E0,2.8370124E-1,2.6890934E-1,7.338862E-2,1.2429783E-1,1.4630389E-1,0E0,0E0,0E0,2.3455143E-2,1.4017645E-2,4.5026228E-2,1.955204E-2,3.6027722E-2,4.9191147E-2,1.2187995E-1,3.825634E-1,3.1739804E-1,1.4156519E-1,2.548893E-1,2.251513E-1,9.157348E-2,0E0,2.2863686E-1,1.7136514E-1,7.611096E-2,2.0865989E-1,1.930347E-1,2.7548137E-1,2.3346066E-1,1.4156E-1,2.6057768E-1,1.6225445E-1,0E0,5.3770564E-2,4.8736148E-2,5.7568505E-2,3.1163039E-2,8.1382215E-2,0E0,0E0,7.6889447E-3,0E0,2.1416934E-2,1.4565483E-2,0E0,0E0,2.512064E-2,0E0,0E0,3.243485E-2,1.1358817E-1,1.13978416E-1,0E0,4.8120797E-2,1.3097878E-1,3.4087497E-1,9.2083685E-2,1.4748418E-1,3.7169173E-1,1.9379976E-1,2.1208787E-1,2.1907333E-1,3.4175187E-2,0E0,2.8359872E-1,1.6203201E-1,1.0235518E-1,9.520599E-2,0E0,0E0,1.7349146E-1,2.2875321E-1,3.533952E-2,8.098113E-2,5.1037177E-2,0E0,0E0,0E0,4.532083E-2,0E0,1.131804E-1,1.9425786E-1,1.3724586E-1,1.4092919E-1,0E0,0E0,0E0,0E0,5.0317094E-2,1.9961704E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0253036E-2,3.2439105E-2,0E0,8.280113E-2,1.2223415E-1,0E0,5.8602996E-2,0E0,2.7110592E-2,0E0,4.5192882E-2,8.408606E-3,8.2295075E-2,2.695924E-2,5.451396E-2,9.000465E-2,6.941283E-2,3.7427068E-1,1.985333E-1,1.7302373E-1,1.17320806E-1,1.1522405E-1,1.4158523E-1,0E0,2.48727E-1,1.0786094E-2,0E0,0E0,0E0,6.686042E-2,1.2569053E-1,1.5905458E-1,7.3808014E-2,0E0,0E0,0E0,9.883421E-2,2.6892853E-1,1.316529E-1,9.792857E-3,1.0309025E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,25,25,26,26,29,29,30,30,31,31,32,32,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,87,87,89,89,90,90,93,93,96,96,97,97,98,98,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,111,111,112,112,113,113,114,114,117,117,118,118,119,119,120,120,121,121,125,125,127,127,128,128,129,129,130,130,135,135,136,136,148,148,149,149,151,151,152,152,154,154,156,156,158,158,159,159,160,160,161,161,162,162,163,163,164,164,165,165,166,166,167,167,168,168,169,169,170,170,172,172,173,173,177,177,178,178,179,179,180,180,184,184,185,185,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,36,38,40,-1,42,-1,44,46,-1,-1,48,50,52,54,-1,-1,56,-1,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,84,-1,-1,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,-1,112,114,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,-1,-1,142,-1,144,146,-1,-1,148,-1,-1,150,152,154,-1,156,158,160,162,164,166,168,170,172,174,-1,176,178,180,182,-1,-1,184,186,188,190,192,-1,-1,-1,194,-1,196,198,200,202,-1,-1,-1,-1,204,206,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,208,210,-1,212,214,-1,216,-1,218,-1,220,222,224,226,228,230,232,234,236,238,240,242,244,-1,246,248,-1,-1,-1,250,252,254,256,-1,-1,-1,258,260,262,264,266,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.294364E0,9.772148E4,6.082581E3,1.1216729E2,2.091623E5,1.9013363E4,1.0828989E8,1.1855755E8,-1.0333706E-2,1E0,6.9934395E-5,4.5505118E2,2E0,1.0238709E1,6.494E3,2.7504245E10,1.100592E6,-5.875736E-3,2.9005406E5,2.9104478E0,1.20086E5,-1.5046861E-2,3.0820766E-1,1.7562138E-2,3.9556694E-1,3.9E1,1.1675908E-3,1.7621566E-2,2.8972292E-5,1.9051096E7,6.5E1,2.915E4,1.53414905E-2,4.48528E-3,2.5E-1,1.05205625E-2,5.3E1,2.3140822E8,4.1585636E1,6.8671523E3,1.246E3,1.74E4,1.3359244E9,1.1485E4,1.2E1,8.295904E-3,1.159E3,1.9121015E0,2.2007043E0,2.3140822E8,5.34E2,3.1006755E-3,-1.0069925E-2,-2.27424E-2,5.015707E0,8.037839E-1,2E0,6.126259E5,2.3567346E7,1E1,1.5825138E5,1.752E3,2.08E2,1.5E1,2E0,3.1530054E0,7.299E3,-1.1229888E-2,4E0,3.0272608E10,1.0838E4,1.63E2,3.324E3,1.6E1,8.628E3,2.41E3,2.0304577E-1,5.0795E4,7.4219937E-3,1.7227913E3,5.1160636E11,7.137324E0,1.2427474E8,2.6120481E0,7.7551194E-3,1.9362133E-3,4.9304895E6,-1.8850856E-3,6.585265E-1,1.990351E1,4.362904E-3,-4.669591E-5,5.551823E-1,-4.568765E-3,5.155736E-3,1E0,4.1E1,6.239726E0,-1.906361E-2,1.01644066E2,8.2E1,9.8E1,3.247716E5,3.6022064E1,2.6682352E2,1.04E2,5.314225E2,7E0,2.4079119E1,2.8675655E-3,3.5259784E2,1.7770598E-2,2.14099E5,5.6143098E-2,6.4997324E-3,1.5571535E-2,1E0,2.3977574E7,1.8480903E0,2.2243262E5,9.4538574E2,-1.4638797E-2,-2.3799154E-3,-1.550712E-2,1.513E3,-8.823757E-3,1.7642576E8,4.6816683E-1,3.0161016E7,4.071191E7,-2.3038131E-3,3.558092E-3,6.5618358E-3,3.6834902E-4,2.6312E4,2.028E3,-4.3745423E-3,8.290487E-4,-1.115104E-2,-3.0821362E-3,-7.6007585E-5,2.5648647E-3,1.3530992E-3,-3.1287041E-3,-2.4070274E-3,-6.8644523E-3,-1.910491E-3,1.98714E5,3.011152E6,4.4518514E-3,6.48334E5,2.419355E0,-1.0118862E-2,1.2E1,1.1562149E-3,1.9897084E4,-1.0311181E-2,1.53962E4,1.6344065E4,2.7629982E7,1.6968552E2,2.6642E7,9.8E1,1.3581108E7,3.4E1,1.4229508E1,1.56E2,2.3446269E2,3.0595E4,1.40501E5,1.1604886E-2,4E0,5.053386E3,-6.4765825E-3,1.9836163E-3,-1.543239E-2,1.9026287E9,1.0485785E3,1E0,2.2576077E10,2.6654713E-3,-7.201935E-3,6.3382564E-3,8.756142E7,1.3508893E3,7.134432E7,3.794679E8,5.3E1,4.0317062E-4,8.3848545E-3,2.479695E-3,-4.1858824E-3,-1.8568782E-3,4.3883123E-3,-4.2186347E-3,-1.4328045E-3,-4.702554E-3,-9.85432E-3,-1.8259232E-4,-3.651853E-3,-1.2210787E-3,-6.794858E-3,-2.2034112E-3,-7.6323836E-3,-8.234283E-4,3.7465263E-3,3.1616178E-3,-1.8826085E-4,5.0048233E-4,4.362266E-3,-1.7420162E-3,-6.6866977E-3,2.0653321E-3,-1.5474919E-3,2.4577118E-3,-4.7025657E-3,-5.590315E-3,-1.1079417E-3,-3.8503956E-3,1.1074546E-3,1.0541033E-2,6.414429E-3,2.867584E-3,-7.870605E-4,-5.4420047E-3,-1.1645982E-4,2.9070182E-3,-2.3781944E-3,-2.8055648E-3,-6.904389E-3,-2.3339964E-3,4.2051703E-4,-2.5177107E-4,-1.4179358E-2,2.465007E-3,1.0272109E-2,-7.4152863E-3,2.7490985E-3,-1.4375433E-3,-6.0024858E-3,3.093676E-3,-3.0732031E-3,6.882451E-3,-1.0226335E-4,-9.010014E-3,1.5031492E-3,-4.198394E-4,-3.2768736E-3,2.0604827E-3,6.884948E-3,-4.054457E-3,4.2530373E-4,9.8232216E-5,5.3097266E-3,1.1667224E-2,2.653786E-3,-5.9415847E-3,2.6279937E-3,3.6521854E-3,7.4161068E-3,2.064034E-3,-5.2063847E-3,-4.5747296E-3,-1.5434865E-3,-1.4432909E-3,1.6288793E-3],"split_indices":[20,42,33,52,4,28,52,45,45,0,64,37,56,6,53,2,5,29,0,33,54,9,0,27,0,38,3,0,0,42,45,8,9,0,0,57,0,8,7,53,47,2,9,7,9,8,0,11,53,54,7,0,0,0,0,54,27,8,33,32,3,28,29,2,3,17,53,2,0,3,5,9,10,29,3,9,2,27,9,0,4,31,53,45,53,0,0,32,0,27,58,0,0,38,0,0,74,2,56,0,33,29,0,45,56,55,10,4,3,56,0,52,57,2,38,0,0,102,47,53,33,4,0,0,0,2,0,7,41,45,12,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,1,9,0,31,56,0,8,0,48,0,28,28,45,4,5,8,12,3,56,2,52,29,10,0,58,48,0,0,0,12,52,102,12,0,0,0,7,52,44,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.565E3,8.03E2,1.762E3,6.32E2,1.71E2,1.141E3,6.21E2,6.16E2,1.6E1,5E0,1.66E2,5.28E2,6.13E2,6.07E2,1.4E1,5.7E2,4.6E1,1.2E1,4E0,2.9E1,1.37E2,5.23E2,5E0,6.09E2,4E0,5.46E2,6.1E1,5E0,9E0,4.8E2,9E1,3.8E1,8E0,7E0,5E0,2.3E1,6E0,3.2E1,1.05E2,1.89E2,3.34E2,5.74E2,3.5E1,2.53E2,2.93E2,5.5E1,6E0,3.3E1,4.47E2,2E1,7E1,3.1E1,7E0,4E0,4E0,8E0,1.5E1,2.3E1,9E0,2.4E1,8.1E1,1.61E2,2.8E1,1.16E2,2.18E2,3.35E2,2.39E2,3E1,5E0,1.51E2,1.02E2,1.5E1,2.78E2,3.9E1,1.6E1,1.8E1,1.5E1,2.41E2,2.06E2,6E0,1.4E1,1.2E1,5.8E1,1E1,2.1E1,4E0,4E0,1E1,5E0,9E0,1.4E1,4E0,5E0,1.8E1,6E0,8E0,7.3E1,1.44E2,1.7E1,4E0,2.4E1,2.1E1,9.5E1,4.4E1,1.74E2,2.49E2,8.6E1,1.41E2,9.8E1,2.5E1,5E0,8E0,1.43E2,9.4E1,8E0,6E0,9E0,2E1,2.58E2,2.7E1,1.2E1,1E1,6E0,5E0,1.3E1,1E1,5E0,1.48E2,9.3E1,1.6E2,4.6E1,6E0,8E0,5E0,7E0,5E1,8E0,5E0,5E0,1.6E1,5E0,6E0,4E0,4E0,5E0,4E0,1E1,1E1,8E0,6.7E1,6E0,3.7E1,1.07E2,7E0,1E1,4E0,2E1,4E0,1.7E1,2.1E1,7.4E1,9E0,3.5E1,8.3E1,9.1E1,1.37E2,1.12E2,2.9E1,5.7E1,6E1,8.1E1,4E0,9.4E1,1.6E1,9E0,4E0,4E0,5.3E1,9E1,8.2E1,1.2E1,4E0,4E0,5E0,1.5E1,2.24E2,3.4E1,1.8E1,9E0,6E0,6E0,4E0,6E0,6E0,4E0,7.6E1,7.2E1,6.6E1,2.7E1,1.24E2,3.6E1,2.1E1,2.5E1,4.6E1,4E0,4E0,4E0,4E0,4E0,6.2E1,5E0,2.4E1,1.3E1,3.3E1,7.4E1,4E0,6E0,1.5E1,5E0,7E0,1E1,1.1E1,1E1,5.3E1,2.1E1,5E0,4E0,3E1,5E0,6.6E1,1.7E1,4E1,5.1E1,1.33E2,4E0,1.04E2,8E0,4E0,2.5E1,2.8E1,2.9E1,5.2E1,8E0,7.3E1,8E0,5E0,8.9E1,5E0,1.1E1,4.5E1,8E0,1.9E1,7.1E1,1.9E1,6.3E1,8E0,4E0,1.1E1,4E0,1.53E2,7.1E1,2.7E1,7E0,1.3E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"267","size_leaf_vector":"1"}},{"base_weights":[2.0196496E-3,-3.791477E-2,2.6806131E-2,-7.646038E-3,-7.546078E-2,6.5508354E-3,7.29367E-2,-4.1975174E-2,3.7821706E-2,-2.4840678E-1,-6.925229E-2,4.3496504E-2,-4.5103845E-3,1.8695611E-1,6.0675193E-2,-3.0022003E-2,-1.5882291E-1,1.1521174E-1,1.3932834E-2,-1.8648652E-2,-4.6239565E-3,-9.133931E-2,-3.4934845E-2,3.102538E-2,1.3682717E-1,-1.3249688E-2,3.4347218E-2,2.9344708E-1,1.2786826E-1,4.9502358E-2,1.4950363E-1,1.22490905E-1,-3.8291864E-2,2.7004927E-2,-3.1149584E-1,1.0420793E-2,8.5014634E-2,2.1324247E-2,-1.0376471E-2,-6.77971E-2,-1.533847E-1,-1.336121E-3,-7.747541E-2,4.772935E-2,-6.9126256E-2,2.0152964E-1,8.249611E-2,1.0593793E-1,-1.6362283E-2,6.0203463E-2,-3.5733052E-3,8.690725E-3,1.6586982E-2,1.1339193E-3,1.5640944E-1,3.33141E-2,1.1812807E-1,1.13556795E-1,1.482261E-2,4.750389E-2,1.0063491E-2,-7.828479E-3,-6.983352E-2,5.8622118E-2,-2.0801343E-3,-2.3075933E-2,-5.1683923E-3,1.0573576E-1,1.9932861E-4,7.8004426E-3,1.5776524E-1,-1.2396556E-2,-6.0608488E-2,-1.7749843E-1,-7.252218E-2,8.236176E-3,-1.0352551E-2,-4.5788936E-2,-1.3543113E-1,1.2633511E-1,3.1040238E-2,-1.12299696E-1,-2.5123316E-3,1.3263099E-2,6.4160423E-3,1.2456083E-1,7.1677635E-4,3.387932E-2,8.7058E-3,-2.9437156E-2,1.3149222E-2,5.015017E-2,8.8934265E-3,1.9097548E-2,-7.419639E-2,1.7803518E-1,4.905828E-3,-3.2623198E-2,4.4743575E-2,9.180274E-4,1.281268E-1,1.4328854E-1,7.0595383E-3,3.9424384E-3,3.3006168E-4,4.302863E-2,-4.764256E-2,-7.5563155E-2,4.652087E-3,3.228486E-4,5.470886E-3,5.923378E-2,1.7125773E-1,-6.444798E-2,2.2617761E-2,1.6523173E-2,4.9103662E-2,-4.788225E-2,-1.4816362E-1,-1.3103949E-1,-2.1910405E-1,2.097056E-3,-1.0823177E-1,6.447939E-2,-2.8174182E-2,-1.0204231E-1,-4.8852777E-3,-5.163057E-2,-1.9344282E-1,8.118598E-2,9.64473E-3,1.1906656E-2,1.00674294E-1,-7.218773E-4,-7.6551205E-3,1.442089E-3,-1.4815103E-3,1.7200832E-3,8.551767E-3,-1.7474081E-4,2.967012E-3,-2.5983242E-2,-1.5018523E-1,2.5304144E-2,-8.7304235E-2,6.9437367E-3,3.9531358E-2,8.167961E-2,-7.5087994E-3,-8.003654E-3,-6.588316E-3,4.0591997E-3,1.0071771E-2,-2.6803434E-1,3.6380637E-2,9.0387024E-2,2.1174433E-2,1.5064012E-1,6.656287E-2,1.7916317E-1,8.4366634E-2,2.3995326E-3,-2.173956E-3,6.455435E-3,1.2545979E-1,-1.3022423E-1,-2.2898806E-2,-6.0703415E-2,-2.0463134E-1,8.40016E-2,-1.040006E-3,3.913846E-3,1.0701936E-2,5.94622E-4,-8.581118E-2,-1.0364981E-3,6.206185E-2,-2.3218782E-3,5.5140574E-3,-8.759728E-2,-1.2181801E-2,-1.25180865E-2,-5.0149776E-2,-1.7574376E-1,-4.597437E-2,-1.2492129E-1,-1.2954126E-2,-5.2416794E-2,-8.689983E-3,6.6263564E-3,-3.2910854E-3,-8.241481E-3,-9.569508E-2,2.6428353E-3,-1.5376502E-1,-2.6157241E-2,4.027437E-2,-5.932054E-3,-1.621776E-4,-2.603972E-1,-3.9835405E-3,3.718638E-2,6.509001E-3,3.37628E-2,-6.81374E-2,3.272495E-4,1.16143465E-1,-8.813402E-3,-2.3287624E-2,-9.367476E-3,-2.6496076E-3,1.0974663E-1,1.7921621E-2,4.7862786E-4,-1.0820485E-1,7.681971E-3,2.9867431E-2,5.9733973E-3,1.6789614E-3,-6.392708E-2,1.9872326E-2,-1.7230371E-3,8.8720396E-4,5.1786227E-4,-2.5071295E-2,-3.331814E-2,6.983077E-2,-4.266628E-3,1.0134466E-1,-8.1346616E-2,2.9542008E-2,1.308276E-1,1.1200494E-2,-1.1201103E-2,1.1933894E-1,6.181187E-3,1.0827968E-2,5.4981112E-3,2.1474834E-3,2.0037864E-3,-3.4232065E-3,7.710213E-3,1.8607276E-3,-1.3528208E-2,-2.7657251E-3,1.8675097E-3,-2.418674E-3,-8.203474E-4,-4.2908816E-3,-1.7430415E-2,-5.0855135E-3,4.9881735E-3,1.3015004E-3,-5.9756376E-3,-2.0565197E-3,-2.4758927E-3,1.3595381E-3,7.92215E-3,1.8793027E-3,-3.5138568E-3,-1.138184E-2,2.2061768E-3,-3.05125E-3,6.434857E-4,-4.5545674E-3,-1.0631988E-2,-1.8432217E-3,-3.6336053E-3,-5.7790225E-4,-8.597331E-3,-2.1670044E-3,-7.114873E-4,-4.002879E-3,-1.7819722E-3,1.2109026E-3,4.053812E-3,-1.4058795E-3,-6.8038106E-3,-2.2280326E-3,-4.1328827E-3,-9.985216E-3,-1.9008216E-3,9.734346E-4,3.9050162E-3,-6.612674E-4,-6.8308157E-3,-1.6589306E-2,5.3393785E-3,-2.8242898E-4,2.0693394E-3,-6.143949E-3,-6.234797E-3,3.4061464E-4,8.461625E-3,3.8189236E-3,-2.8366186E-3,-6.4254744E-4,1.0107099E-2,2.1612504E-3,-1.9541853E-3,1.4285598E-3,-2.6445428E-3,-6.478021E-3,-9.833188E-4,2.349903E-3,-5.034634E-3,3.89279E-4,2.3104711E-3,-8.939292E-4,2.8855342E-3,-7.454437E-3,5.3597875E-3,-3.9691076E-5,6.3627125E-3,1.8937436E-3,-6.503792E-3,1.6266492E-3,5.010069E-3,1.0216705E-3,7.5574936E-3,3.0668613E-3,-2.619293E-3,1.6118657E-3,7.613925E-3,2.1288462E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,-1,95,97,99,101,-1,103,-1,105,107,109,-1,-1,-1,111,-1,113,115,-1,117,119,121,-1,123,125,127,129,131,133,135,-1,-1,137,-1,139,-1,141,143,145,-1,147,149,151,-1,153,155,-1,157,159,161,-1,-1,163,165,167,-1,-1,-1,169,171,173,175,-1,177,179,181,183,185,-1,187,189,191,193,195,197,199,201,-1,203,205,-1,-1,-1,-1,-1,-1,-1,-1,207,209,211,213,-1,215,217,219,-1,221,-1,-1,223,225,227,229,231,233,235,237,-1,-1,239,241,243,245,247,249,251,-1,-1,-1,-1,253,255,257,-1,-1,259,261,-1,263,265,267,269,-1,271,-1,-1,273,275,277,-1,279,281,283,-1,-1,285,-1,287,-1,289,291,-1,293,-1,295,-1,-1,297,299,-1,301,-1,303,-1,-1,305,307,-1,-1,-1,-1,309,311,-1,313,315,317,319,-1,321,323,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.571615E0,1.1302106E0,1.4973468E0,8.6318684E-1,4.5953488E-1,4.558567E-1,6.7230296E-1,4.3519253E-1,4.383998E-1,2.5727814E-1,3.2273626E-1,2.948871E-1,2.921615E-1,2.5821662E-1,4.3226147E-1,3.666721E-1,8.3116734E-1,1.4011663E-1,3.0417177E-1,0E0,0E0,3.7084007E-1,2.4287903E-1,3.8472742E-1,8.883524E-2,2.6216325E-1,1.5513133E-1,1.5910625E-2,9.2508554E-2,4.3488222E-1,2.2549629E-1,7.9194486E-2,2.6176962E-1,3.2821268E-2,4.5267284E-1,0E0,7.345143E-2,3.2721516E-1,0E0,2.3251885E-1,1.2557018E-1,1.4553902E-1,1.3152966E-1,2.532032E-1,9.462428E-2,2.6668191E-2,4.981474E-2,8.267853E-2,2.6470065E-1,1.0238063E-1,1.0548753E-1,0E0,0E0,0E0,1.4000416E-2,2.4337447E-1,7.245302E-2,1.2880874E-1,0E0,1.1035722E-2,0E0,2.8552604E-1,1.321339E-1,2.5826033E-2,0E0,0E0,0E0,9.649733E-2,0E0,1.759537E-1,2.8439602E-1,0E0,1.9904137E-1,7.56681E-2,7.813423E-2,0E0,1.2451036E-1,1.1454994E-1,1.1640912E-1,8.670026E-2,2.1597417E-1,7.7839494E-2,1.2775311E-2,0E0,0E0,4.0300757E-2,0E0,9.738849E-3,0E0,1.941981E-1,2.5984097E-1,7.750219E-2,0E0,8.386732E-2,9.195552E-2,2.2352219E-2,0E0,7.894762E-1,2.92849E-1,0E0,8.422148E-2,5.2526295E-2,2.2803472E-2,0E0,0E0,1.8610588E-1,1.5844508E-1,2.3690772E-1,0E0,0E0,0E0,4.5365803E-2,3.906715E-2,4.669045E-2,1.2695514E-1,0E0,6.9520906E-2,2.300415E-1,2.1819955E-1,9.839794E-2,6.971562E-2,0E0,4.220988E-2,8.3553605E-2,1.0015187E-1,1.7641896E-1,2.98725E-2,3.8568154E-2,9.344971E-2,4.2665705E-2,0E0,2.2798014E-1,5.020058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8360534E-1,3.715691E-2,1.1585547E-1,4.7609583E-2,0E0,8.370592E-2,2.329228E-2,5.688304E-2,0E0,7.403215E-3,0E0,0E0,7.209592E-1,9.0921104E-2,1.8778068E-1,1.5853234E-1,4.2047143E-2,8.341828E-2,1.2748301E-2,1.0144375E-2,0E0,0E0,1.13731146E-1,4.2607278E-2,1.7439482E-1,9.712818E-2,1.3012844E-1,1.5731698E-1,1.2994193E-2,0E0,0E0,0E0,0E0,2.8263688E-2,1.1919474E-1,1.1435464E-1,0E0,0E0,1.7044133E-1,2.422824E-1,0E0,3.4769766E-2,8.472258E-2,9.282233E-3,3.0981645E-2,0E0,8.37495E-3,0E0,0E0,9.774795E-3,1.0718708E-1,2.9089436E-2,0E0,3.5847753E-2,1.2131478E-2,2.1302273E-2,0E0,0E0,3.4329236E-2,0E0,3.9404824E-2,0E0,1.2740934E-1,1.1972566E-1,0E0,5.0442964E-2,0E0,1.5848689E-1,0E0,0E0,8.353962E-2,1.07596405E-1,0E0,1.5658736E-2,0E0,6.469977E-2,0E0,0E0,3.2483805E-2,2.5720455E-2,0E0,0E0,0E0,0E0,1.4579119E-1,6.882809E-2,0E0,1.4030397E-1,8.208026E-2,1.066595E-1,4.886979E-2,0E0,1.781368E-2,2.3932174E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,54,54,55,55,56,56,57,57,59,59,61,61,62,62,63,63,67,67,69,69,70,70,72,72,73,73,74,74,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,87,87,89,89,90,90,91,91,93,93,94,94,95,95,97,97,98,98,100,100,101,101,102,102,105,105,106,106,107,107,111,111,112,112,113,113,114,114,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,125,125,126,126,127,127,128,128,129,129,131,131,132,132,141,141,142,142,143,143,144,144,146,146,147,147,148,148,150,150,153,153,154,154,155,155,156,156,157,157,158,158,159,159,160,160,163,163,164,164,165,165,166,166,167,167,168,168,169,169,174,174,175,175,176,176,179,179,180,180,182,182,183,183,184,184,185,185,187,187,190,190,191,191,192,192,194,194,195,195,196,196,199,199,201,201,203,203,204,204,206,206,208,208,211,211,212,212,214,214,216,216,219,219,220,220,225,225,226,226,228,228,229,229,230,230,231,231,233,233,234,234],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,-1,96,98,100,102,-1,104,-1,106,108,110,-1,-1,-1,112,-1,114,116,-1,118,120,122,-1,124,126,128,130,132,134,136,-1,-1,138,-1,140,-1,142,144,146,-1,148,150,152,-1,154,156,-1,158,160,162,-1,-1,164,166,168,-1,-1,-1,170,172,174,176,-1,178,180,182,184,186,-1,188,190,192,194,196,198,200,202,-1,204,206,-1,-1,-1,-1,-1,-1,-1,-1,208,210,212,214,-1,216,218,220,-1,222,-1,-1,224,226,228,230,232,234,236,238,-1,-1,240,242,244,246,248,250,252,-1,-1,-1,-1,254,256,258,-1,-1,260,262,-1,264,266,268,270,-1,272,-1,-1,274,276,278,-1,280,282,284,-1,-1,286,-1,288,-1,290,292,-1,294,-1,296,-1,-1,298,300,-1,302,-1,304,-1,-1,306,308,-1,-1,-1,-1,310,312,-1,314,316,318,320,-1,322,324,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.9173258E-4,1.460806E6,2.539E3,3E0,2E0,1.5552E4,9.837297E0,9.073025E6,2.4991455E9,7.621583E2,4.750825E2,6.624E3,4.7777777E0,6.082581E3,7E0,1E0,5E1,4.6577052E7,-1.8648652E-2,-4.6239565E-3,2.85745E5,7.76264E7,1E0,2.3663366E0,3.277292E2,1.5548611E7,9.6E1,1.7801905E5,3.1100148E6,2.4536058E1,1.3984146E2,2E0,2.106062E1,2.6659616E2,1.0420793E-2,2.6878788E1,8.61491E3,-1.0376471E-2,2.35E2,5.378E3,1.7387315E2,4.29E4,2.6654E4,2.1E2,4.7646948E2,7.73756E0,1.3989231E5,3.624992E5,6.9673E4,9.12179E6,8.690725E-3,1.6586982E-2,1.1339193E-3,4.3359457E10,1E0,3.0923E4,7.771383E0,1.482261E-2,5.714286E-1,1.0063491E-2,6.1E2,4.564527E6,9.213145E0,-2.0801343E-3,-2.3075933E-2,-5.1683923E-3,3.7057E5,1.9932861E-4,4.5216934E4,1.8738E4,-1.2396556E-2,3.03157E5,3.940884E5,2.3140822E8,8.236176E-3,1E0,3.0387878E3,2.1036203E5,4.877E4,4E0,3.5E1,3.6158625E4,1.3263099E-2,6.4160423E-3,8.690019E-4,7.1677635E-4,5.664E1,8.7058E-3,2.9251662E6,1.3446785E7,5.6666665E0,8.8934265E-3,3.95376E5,2.7565938E3,8.0469E4,4.905828E-3,1.0449754E0,2E0,9.180274E-4,1.845406E1,2.185321E0,1.4285097E1,3.9424384E-3,3.3006168E-4,5E0,7.68E2,3.64299E5,4.652087E-3,3.228486E-4,5.470886E-3,1.0908286E3,5.9693484E0,4.4463258E8,2.0605305E5,1.6523173E-2,2.943086E4,1.048E3,8E0,4.1380672E0,4.0021616E-1,2.097056E-3,5.820743E-1,1E0,4.5698788E7,2.34E0,3.3E1,3.3125E0,1.0526E4,5.85E2,9.64473E-3,1.6118722E0,6.977611E5,-7.218773E-4,-7.6551205E-3,1.442089E-3,-1.4815103E-3,1.7200832E-3,8.551767E-3,-1.7474081E-4,2.967012E-3,1E0,5.46E2,3.9387238E5,2.909019E3,6.9437367E-3,1.683653E8,3.6407896E3,9.115134E5,-8.003654E-3,7.683389E7,4.0591997E-3,1.0071771E-2,3.324E3,4.238096E6,7.106795E6,5.31E2,1.8E1,1.4101E4,2.4480248E5,2.4708056E5,2.3995326E-3,-2.173956E-3,2.727619E1,1.372176E6,1.5597535E3,1.3E1,1.2748036E-5,5.69815E5,9.00042E-6,-1.040006E-3,3.913846E-3,1.0701936E-2,5.94622E-4,3.4348795E6,5.794E3,1.2684739E1,-2.3218782E-3,5.5140574E-3,3.1066042E7,1.1004604E9,-1.25180865E-2,2.9446045E-1,2.88001E10,4.68646E5,8.48106E5,-1.2954126E-2,1.62E2,-8.689983E-3,6.6263564E-3,2.2603108E3,1E0,7.8599895E9,2.6428353E-3,1.5E1,1.1855755E8,1.25E2,-5.932054E-3,-1.621776E-4,6.473E3,-3.9835405E-3,1.7E1,6.509001E-3,1.0779661E1,1.113742E6,3.272495E-4,8.9E1,-8.813402E-3,8.242408E4,-9.367476E-3,-2.6496076E-3,2.2783158E5,2.691875E2,4.7862786E-4,1.7302E4,7.681971E-3,1.983922E3,5.9733973E-3,1.6789614E-3,1.517E3,1.29E3,-1.7230371E-3,8.8720396E-4,5.1786227E-4,-2.5071295E-2,9.52E2,3E0,-4.266628E-3,1.5822886E0,1.2982227E1,4.383864E7,9.1E3,1.1200494E-2,3.8807898E6,1.5449402E7,6.181187E-3,1.0827968E-2,5.4981112E-3,2.1474834E-3,2.0037864E-3,-3.4232065E-3,7.710213E-3,1.8607276E-3,-1.3528208E-2,-2.7657251E-3,1.8675097E-3,-2.418674E-3,-8.203474E-4,-4.2908816E-3,-1.7430415E-2,-5.0855135E-3,4.9881735E-3,1.3015004E-3,-5.9756376E-3,-2.0565197E-3,-2.4758927E-3,1.3595381E-3,7.92215E-3,1.8793027E-3,-3.5138568E-3,-1.138184E-2,2.2061768E-3,-3.05125E-3,6.434857E-4,-4.5545674E-3,-1.0631988E-2,-1.8432217E-3,-3.6336053E-3,-5.7790225E-4,-8.597331E-3,-2.1670044E-3,-7.114873E-4,-4.002879E-3,-1.7819722E-3,1.2109026E-3,4.053812E-3,-1.4058795E-3,-6.8038106E-3,-2.2280326E-3,-4.1328827E-3,-9.985216E-3,-1.9008216E-3,9.734346E-4,3.9050162E-3,-6.612674E-4,-6.8308157E-3,-1.6589306E-2,5.3393785E-3,-2.8242898E-4,2.0693394E-3,-6.143949E-3,-6.234797E-3,3.4061464E-4,8.461625E-3,3.8189236E-3,-2.8366186E-3,-6.4254744E-4,1.0107099E-2,2.1612504E-3,-1.9541853E-3,1.4285598E-3,-2.6445428E-3,-6.478021E-3,-9.833188E-4,2.349903E-3,-5.034634E-3,3.89279E-4,2.3104711E-3,-8.939292E-4,2.8855342E-3,-7.454437E-3,5.3597875E-3,-3.9691076E-5,6.3627125E-3,1.8937436E-3,-6.503792E-3,1.6266492E-3,5.010069E-3,1.0216705E-3,7.5574936E-3,3.0668613E-3,-2.619293E-3,1.6118657E-3,7.613925E-3,2.1288462E-3],"split_indices":[102,27,47,2,3,17,9,54,45,12,52,55,2,54,52,3,66,0,48,0,0,29,45,74,54,50,45,0,33,28,54,52,17,56,52,0,56,4,0,2,2,33,10,9,10,52,58,28,47,2,9,0,0,0,19,8,1,53,0,53,0,2,9,53,0,0,0,9,0,28,9,0,9,32,7,0,64,4,33,12,54,0,33,0,0,38,0,50,0,28,45,58,0,12,4,29,0,53,17,0,56,41,54,0,0,8,2,9,0,0,0,4,56,31,33,0,33,2,3,57,27,0,27,75,51,54,8,54,9,2,0,53,45,0,0,0,0,0,0,0,0,67,10,47,32,0,7,4,32,0,32,0,0,29,45,5,10,8,2,33,33,0,0,46,12,33,3,42,9,39,0,0,0,0,47,2,56,0,0,45,5,0,39,5,29,29,0,8,0,0,4,8,5,0,3,45,8,0,0,9,0,10,0,56,9,0,10,0,28,0,0,28,55,0,9,0,55,0,0,0,0,0,0,0,0,29,11,0,42,53,12,29,0,28,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.596E3,9.94E2,1.602E3,5.51E2,4.43E2,1.114E3,4.88E2,3.14E2,2.37E2,1.4E1,4.29E2,2.56E2,8.58E2,4.6E1,4.42E2,2.86E2,2.8E1,5.5E1,1.82E2,7E0,7E0,2.6E2,1.69E2,2.27E2,2.9E1,7.01E2,1.57E2,1.5E1,3.1E1,3.94E2,4.8E1,1.4E1,2.72E2,1.3E1,1.5E1,1.2E1,4.3E1,1.77E2,5E0,1.9E2,7E1,9.5E1,7.4E1,1.95E2,3.2E1,1.2E1,1.7E1,1.7E1,6.84E2,9.3E1,6.4E1,5E0,1E1,7E0,2.4E1,3.2E2,7.4E1,4E1,8E0,8E0,6E0,1.39E2,1.33E2,9E0,4E0,8E0,7E0,3.4E1,9E0,1.62E2,1.5E1,6E0,1.84E2,5.3E1,1.7E1,4E0,9.1E1,4.9E1,2.5E1,3.3E1,1.62E2,1.9E1,1.3E1,5E0,7E0,1E1,7E0,9E0,8E0,4.74E2,2.1E2,8.7E1,6E0,4.9E1,1.5E1,1.6E1,8E0,4.7E1,2.73E2,7E0,6.7E1,3.1E1,9E0,4E0,4E0,6.1E1,7.8E1,1.29E2,4E0,5E0,4E0,2.1E1,1.3E1,2.7E1,1.35E2,5E0,1E1,1.62E2,2.2E1,2.7E1,2.6E1,4E0,1.3E1,1.7E1,7.4E1,2E1,2.9E1,1.1E1,1.4E1,2.1E1,1.2E1,1.28E2,3.4E1,6E0,1.3E1,6E0,7E0,4E0,6E0,4E0,5E0,4.62E2,1.2E1,1.88E2,2.2E1,8E0,7.9E1,1.4E1,3.5E1,6E0,9E0,4E0,1.2E1,1E1,3.7E1,9.2E1,1.81E2,4.8E1,1.9E1,1.8E1,1.3E1,5E0,4E0,4.3E1,1.8E1,1.7E1,6.1E1,1.17E2,1.2E1,1.6E1,5E0,5E0,8E0,6E0,2.1E1,8.5E1,5E1,4E0,6E0,7.6E1,8.6E1,1E1,1.2E1,1.7E1,1E1,9E0,1.7E1,8E0,5E0,8E0,9E0,5.8E1,1.6E1,5E0,1.5E1,2E1,9E0,4E0,7E0,8E0,6E0,1.2E1,9E0,1.01E2,2.7E1,5E0,2.9E1,7E0,4.55E2,8E0,4E0,1.4E1,1.74E2,4E0,1.8E1,5E0,7.4E1,7E0,7E0,1.1E1,2.4E1,4E0,5E0,5E0,5E0,1.2E1,2.5E1,5E0,8.7E1,1.3E1,1.68E2,4E1,8E0,8E0,1.1E1,9E0,9E0,7E0,6E0,3E1,1.3E1,1.3E1,5E0,5E0,1.2E1,1.8E1,4.3E1,4.3E1,7.4E1,4E0,8E0,1.2E1,4E0,1.1E1,1E1,3.1E1,5.4E1,9E0,4.1E1,6.9E1,7E0,4E1,4.6E1,5E0,7E0,1.3E1,4E0,5E0,5E0,5E0,4E0,4E0,4E0,4E0,5E0,1E1,4.8E1,8E0,8E0,7E0,8E0,1.6E1,4E0,5E0,4E0,4E0,4E0,4E0,8E0,9.7E1,4E0,1.5E1,1.2E1,1.1E1,1.8E1,1.07E2,3.48E2,5E0,9E0,2.7E1,1.47E2,6E0,1.2E1,1.9E1,5.5E1,7E0,4E0,1.4E1,1E1,7E0,5E0,1.6E1,9E0,6.1E1,2.6E1,9E0,4E0,1.8E1,1.5E2,3E1,1E1,4E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"325","size_leaf_vector":"1"}},{"base_weights":[1.9233439E-3,-3.714102E-2,2.0026332E-2,-4.5344785E-2,6.09872E-2,1.2277968E-3,6.2779486E-2,-2.5023464E-2,-8.157842E-2,1.2298523E-2,4.1124973E-2,-2.1885267E-2,2.2439744E-2,3.821258E-2,1.0472067E-1,-5.964733E-2,1.0338044E-3,-1.3229804E-1,-5.332202E-2,3.0828053E-2,7.3034335E-3,-3.5304263E-2,1.1087306E-2,1.9544926E-2,1.4165031E-2,7.669602E-2,2.027653E-2,4.810833E-3,1.1579504E-1,-5.1749296E-2,-1.596713E-2,3.119035E-2,-3.0671133E-2,-1.04912385E-1,-2.3920166E-1,-1.1462668E-2,-7.8108095E-2,9.1113575E-2,3.759284E-3,-4.275457E-2,2.4890812E-2,4.7311194E-2,-1.74086E-2,7.1301134E-3,6.141734E-2,-1.5600726E-3,9.3234085E-2,-2.2141922E-2,5.162938E-2,3.8705922E-3,-2.9414458E-2,2.1097483E-1,1.0189723E-1,6.3444026E-2,-5.987928E-2,1.8248005E-2,1.2077299E-1,-1.8847078E-2,-1.5226778E-1,-1.2257401E-2,-8.785536E-2,-3.3197755E-1,-1.6355057E-3,3.717053E-2,-4.3209832E-2,-2.399243E-2,-1.03763714E-1,9.484561E-4,1.3970765E-1,-5.2300405E-2,2.7650604E-2,-3.72715E-2,-1.4943144E-1,-5.8909883E-3,4.0915348E-2,1.9104887E-2,1.6074319E-1,-2.720016E-2,6.776985E-3,1.0038273E-2,-1.1545963E-2,6.854104E-2,-5.8074207E-3,-5.8485027E-3,3.4627438E-2,-1.3671145E-3,1.0217371E-1,-5.2349918E-2,1.8180931E-2,5.9384134E-2,-6.310177E-3,-6.6213817E-3,2.012391E-2,2.3963553E-1,3.277883E-3,9.209768E-2,2.1426697E-1,7.6170675E-3,-1.1172827E-3,-5.085275E-2,-1.2731828E-1,2.5157526E-2,-5.1862826E-3,9.341645E-3,4.9129594E-2,-4.082612E-2,2.3041727E-2,-1.4518492E-2,-2.3384131E-3,-2.2142626E-1,-6.52591E-2,-1.8940294E-2,-9.035781E-3,-2.386345E-3,5.8643486E-2,-1.0373967E-1,-2.0337634E-2,-6.6202716E-3,5.7410058E-3,-1.4324808E-1,-3.6193766E-2,2.9558253E-3,9.1713555E-3,-5.6014527E-3,-4.190075E-4,5.00414E-2,-1.7279253E-3,-2.0549675E-2,-6.1411753E-2,-6.0344845E-2,-1.2653887E-2,9.440992E-3,2.0887975E-2,-3.9461166E-2,4.033354E-2,-3.70567E-4,2.2256629E-1,5.3604734E-3,-3.4532044E-2,-2.2118606E-2,2.618456E-2,9.56921E-3,8.719032E-2,3.844605E-3,1.5435189E-3,9.675237E-3,1.13144524E-1,-8.642938E-3,-3.510532E-2,3.574783E-2,-5.410676E-2,7.203262E-2,-1.896654E-2,2.4444838E-3,-1.2101227E-3,1.3493607E-2,6.685216E-3,1.5683647E-1,7.215667E-2,2.5895438E-3,1.4440845E-2,-8.743467E-2,-2.8154826E-2,-1.9188188E-1,-5.7525117E-2,4.624591E-2,-1.1486096E-2,5.5851797E-3,-1.7925315E-3,6.2836915E-2,-5.4282933E-2,3.975772E-2,-1.56181E-2,-4.0650074E-3,-1.7249856E-2,-7.716464E-2,4.410963E-3,6.3495394E-3,2.503926E-2,-8.38588E-3,-2.1556257E-3,-5.7152424E-2,9.4259605E-3,6.771756E-3,-1.9438352E-2,-1.9525316E-1,-8.5274324E-2,-6.485054E-2,1.6594635E-2,-4.3942026E-5,6.600913E-2,-5.3640854E-2,6.0403375E-3,-2.5614062E-1,-4.6749707E-2,-6.101747E-3,-3.798289E-4,-2.2189321E-2,8.4198415E-2,-3.7927732E-3,1.1715537E-3,5.4707196E-2,-4.4206455E-3,6.1099976E-3,1.311141E-2,-1.3355961E-1,-2.295979E-2,-6.913468E-3,-1.5319166E-1,4.8695248E-2,-9.933159E-3,3.457228E-2,-7.363271E-3,1.0536512E-1,-8.527554E-3,-1.617231E-3,1.4762334E-3,-1.3896493E-3,2.634458E-3,1.2537165E-1,9.548757E-3,-7.0371E-2,-1.0865173E-2,5.7263535E-2,-1.2055546E-2,-6.437417E-3,1.5710552E-3,2.511373E-2,1.14549674E-1,3.346915E-2,-6.950558E-2,1.924913E-1,1.046941E-1,1.20719E-1,5.661352E-2,2.7597793E-3,-5.1405374E-3,-4.952281E-3,-9.489638E-4,-1.2144649E-2,-4.0904265E-3,1.564168E-3,-7.57354E-3,1.5165204E-4,3.285599E-3,1.6002905E-3,-2.2243191E-3,6.699882E-3,-1.7603558E-3,-7.0891134E-3,-1.9972587E-3,4.5876262E-5,4.3398296E-3,-2.762868E-3,2.8889588E-4,-4.717153E-3,2.226006E-3,3.2041396E-3,-2.4109244E-4,-4.8406646E-3,-7.6192175E-4,-2.4999322E-3,1.4983836E-3,-2.623625E-3,3.0744828E-3,-4.863346E-3,-1.0491543E-2,-9.114803E-4,-7.024862E-3,-1.3190843E-3,-4.8376094E-3,3.1661415E-3,-1.6459808E-3,4.1909446E-3,1.111951E-3,-1.8560204E-3,-8.805679E-3,7.0068073E-3,-7.3262374E-4,-1.9807143E-2,-1.8284149E-3,-3.3932412E-3,-6.9787237E-4,-7.219561E-5,-5.0331047E-3,5.498066E-3,2.6740655E-4,-2.2936517E-3,3.323073E-3,-2.4790594E-3,-9.533293E-3,-4.1555893E-4,-3.8342448E-3,-1.0995913E-3,4.8327004E-3,-1.2613647E-2,-4.9870386E-4,1.0833817E-3,5.24058E-3,-3.8413291E-3,1.5423622E-3,5.2597024E-3,4.7807037E-4,6.52622E-3,1.9085632E-3,-1.9273827E-3,2.2084157E-3,3.6140021E-3,7.0250393E-3,1.9134601E-3,-1.0546449E-3,-4.715256E-3,-9.783391E-4,-4.796749E-3,4.225113E-4,-1.9883267E-4,3.8380804E-3,-3.8667384E-3,2.2817035E-3,2.05718E-3,-7.179893E-3,6.8060784E-3,1.370045E-3,2.8174743E-3,-3.6285157E-5,1.8998407E-4,-5.9465906E-3,1.0961569E-2,3.6642568E-3,6.7738993E-3,4.8252937E-4,6.949714E-3,6.0371653E-4,-4.5450707E-3,3.4165988E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,-1,45,47,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,97,99,101,103,105,107,-1,109,111,-1,113,115,117,119,-1,121,123,125,127,129,-1,131,133,135,137,-1,139,-1,141,-1,-1,143,-1,145,147,149,151,-1,-1,153,155,-1,157,159,-1,-1,161,163,165,-1,-1,167,169,171,-1,-1,173,175,-1,-1,-1,177,179,181,-1,183,185,187,-1,-1,-1,-1,189,-1,191,193,195,-1,-1,197,199,201,-1,203,-1,205,207,209,211,213,-1,215,217,219,-1,221,223,225,227,229,-1,-1,-1,-1,231,233,-1,-1,235,237,239,241,243,245,-1,-1,247,249,251,253,-1,-1,255,-1,-1,257,-1,-1,259,261,-1,263,265,267,269,271,-1,273,275,277,279,281,-1,-1,283,285,-1,-1,287,-1,-1,-1,289,291,293,295,297,299,301,-1,303,305,-1,-1,-1,-1,307,309,311,313,315,317,-1,-1,319,321,323,325,327,329,331,333,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8401235E0,6.661025E-1,1.4288957E0,5.5829465E-1,2.2523381E-1,6.064686E-1,5.5548644E-1,4.4123915E-1,3.8419652E-1,0E0,5.9308387E-2,2.6232702E-1,4.818805E-1,2.3588783E-1,2.2073054E-1,4.1454375E-1,2.6866785E-1,2.5888443E-1,1.827473E-1,8.940949E-2,0E0,1.8987083E-1,1.7836761E-1,3.317387E-1,0E0,1.4121372E-1,3.1500885E-1,5.4496266E-2,2.1660447E-1,1.9578558E-1,0E0,1.647511E-1,1.9410408E-1,1.6763675E-1,3.530866E-1,1.0505185E-1,1.5161967E-1,5.693613E-2,5.3680398E-2,2.1377414E-1,1.12365976E-1,2.4009083E-1,1.490425E-1,3.4453002E-1,1.9272596E-1,8.763754E-2,9.9821806E-2,1.2397711E-1,1.951392E-1,0E0,7.927895E-2,7.5942636E-2,1.5535283E-1,1.1031998E-1,1.1040372E-1,1.0970493E-1,7.61652E-2,1.1694585E-1,1.6143167E-1,0E0,2.0286053E-1,2.395904E-2,0E0,5.204017E-2,5.465264E-2,1.2012701E-1,1.9491202E-1,0E0,2.4457306E-2,3.0574873E-2,4.0746488E-2,1.4325151E-1,1.5119159E-1,0E0,1.2353373E-1,7.867487E-2,1.5759277E-1,9.441662E-2,0E0,2.5447494E-1,0E0,1.54235E-1,0E0,0E0,2.2280881E-2,0E0,8.4147155E-2,1.1457436E-1,5.7808504E-2,1.311895E-1,0E0,0E0,1.5231745E-2,2.1817327E-2,0E0,1.7974508E-1,1.3071764E-1,0E0,0E0,1.3952532E-1,8.809614E-2,9.403452E-2,0E0,0E0,5.726383E-2,1.1919381E-1,2.905539E-2,0E0,0E0,1.4532459E-1,1.2197241E-1,0E0,0E0,0E0,4.7409445E-2,3.5480767E-2,3.450667E-2,0E0,1.0067108E-1,1.2505168E-1,4.4945784E-2,0E0,0E0,0E0,0E0,1.7132934E-2,0E0,1.8774159E-1,4.0487355E-1,3.4910005E-2,0E0,0E0,1.0823077E-1,4.0755153E-2,9.000469E-2,0E0,1.1858225E-2,0E0,9.777904E-2,3.2579026E-1,2.6553303E-1,1.4242782E-1,1.8704134E-1,0E0,1.0451247E-2,1.7604047E-2,9.380382E-2,0E0,4.368736E-2,3.771718E-2,6.158644E-2,2.2254163E-1,5.2677236E-2,0E0,0E0,0E0,0E0,4.305756E-2,7.996899E-2,0E0,0E0,1.4890617E-1,6.768585E-2,4.2032808E-2,1.0499415E-1,6.471448E-2,6.610158E-2,0E0,0E0,7.4597195E-2,8.709282E-2,5.662605E-2,1.22065935E-2,0E0,0E0,1.2801865E-1,0E0,0E0,1.8985977E-2,0E0,0E0,2.1827828E-2,2.3506368E-2,0E0,7.284782E-2,1.4151335E-2,8.642776E-2,2.0797774E-2,2.7572481E-2,0E0,1.0640949E-2,1.8556377E-1,3.3264592E-1,2.934466E-1,9.4088405E-2,0E0,0E0,3.8956866E-2,3.2004803E-2,0E0,0E0,5.2909672E-2,0E0,0E0,0E0,3.3774287E-2,6.235777E-2,2.336182E-1,2.3911205E-1,3.0279747E-1,3.465305E-1,5.3788956E-2,0E0,1.476292E-1,3.0220019E-2,0E0,0E0,0E0,0E0,4.3931603E-2,8.739345E-3,2.3747876E-2,5.34477E-2,3.0236997E-2,4.8784595E-2,0E0,0E0,1.5499867E-1,1.0762656E-1,7.87781E-3,3.6743946E-2,4.478246E-2,4.563801E-2,5.2196324E-2,1.6073135E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,79,79,81,81,84,84,86,86,87,87,88,88,89,89,92,92,93,93,95,95,96,96,99,99,100,100,101,101,104,104,105,105,106,106,109,109,110,110,114,114,115,115,116,116,118,118,119,119,120,120,125,125,127,127,128,128,129,129,132,132,133,133,134,134,136,136,138,138,139,139,140,140,141,141,142,142,144,144,145,145,146,146,148,148,149,149,150,150,151,151,152,152,157,157,158,158,161,161,162,162,163,163,164,164,165,165,166,166,169,169,170,170,171,171,172,172,175,175,178,178,181,181,182,182,184,184,185,185,186,186,187,187,188,188,190,190,191,191,192,192,193,193,194,194,197,197,198,198,201,201,205,205,206,206,207,207,208,208,209,209,210,210,211,211,213,213,214,214,219,219,220,220,221,221,222,222,223,223,224,224,227,227,228,228,229,229,230,230,231,231,232,232,233,233,234,234],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,-1,46,48,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,98,100,102,104,106,108,-1,110,112,-1,114,116,118,120,-1,122,124,126,128,130,-1,132,134,136,138,-1,140,-1,142,-1,-1,144,-1,146,148,150,152,-1,-1,154,156,-1,158,160,-1,-1,162,164,166,-1,-1,168,170,172,-1,-1,174,176,-1,-1,-1,178,180,182,-1,184,186,188,-1,-1,-1,-1,190,-1,192,194,196,-1,-1,198,200,202,-1,204,-1,206,208,210,212,214,-1,216,218,220,-1,222,224,226,228,230,-1,-1,-1,-1,232,234,-1,-1,236,238,240,242,244,246,-1,-1,248,250,252,254,-1,-1,256,-1,-1,258,-1,-1,260,262,-1,264,266,268,270,272,-1,274,276,278,280,282,-1,-1,284,286,-1,-1,288,-1,-1,-1,290,292,294,296,298,300,302,-1,304,306,-1,-1,-1,-1,308,310,312,314,316,318,-1,-1,320,322,324,326,328,330,332,334,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.4409692E3,3.3737933E8,6.7033327E-1,2.1583429E-3,2.9910075E5,1.5283889E0,2.2348747E2,1.9121015E0,1.2298523E-2,1.9204996E0,1.445113E6,6.5317163E3,5.59E2,9.5E2,2.165022E2,1.775044E7,1.2199979E10,1.504779E7,4.0601485E6,7.3034335E-3,1.3264E4,2.7E1,4.264897E6,1.4165031E-2,1.4343751E6,1.2204E4,7.1E1,1.3062E4,9E0,-1.596713E-2,1.0122174E3,3.653309E7,7E0,9.099893E8,4.842894E-2,2.819136E0,7.3582644E5,9.877E3,1E0,2.6393727E4,1.860326E7,8.3588595E8,1.766486E7,1E0,1.1E1,1.002E3,6.748383E8,6.3751144E7,3.8705922E-3,1.5481343E1,2.03148E2,4.5505118E2,1.6E1,1.2737473E1,1E0,1.5595E4,1.0323588E3,5.7950187E-1,-1.2257401E-2,4.26E2,4.832E3,-1.6355057E-3,1.1E2,6.23699E4,3.7246967E2,6.654321E0,9.484561E-4,1.8677662E7,7.023838E3,1E0,4.3978744E0,6.2671E4,-5.8909883E-3,8.581E3,1.654762E2,5.2350176E4,1.3E1,6.776985E-3,1E0,-1.1545963E-2,8.31E2,-5.8074207E-3,-5.8485027E-3,4.0895187E8,-1.3671145E-3,1.2151E4,3.936726E7,1.0409E4,5.21929E5,-6.310177E-3,-6.6213817E-3,2.3E1,2.0136518E0,3.277883E-3,1.710933E0,2.72258E5,7.6170675E-3,-1.1172827E-3,1.13E3,8.447369E0,2.322963E0,-5.1862826E-3,9.341645E-3,3.1648794E-1,2.4198477E2,7.628589E6,-1.4518492E-2,-2.3384131E-3,2.6610387E-3,1E0,-1.8940294E-2,-9.035781E-3,-2.386345E-3,1.69E2,5.03E2,1.9589581E6,-6.6202716E-3,1.17E2,7.298614E2,1.4032121E-1,2.9558253E-3,9.1713555E-3,-5.6014527E-3,-4.190075E-4,1.91E4,-1.7279253E-3,2.73E2,2E0,1.2897E4,-1.2653887E-2,9.440992E-3,2.653986E6,6.54925E6,2.145E3,-3.70567E-4,1.63E2,5.3604734E-3,2.9221216E8,1.325204E6,3.38464E5,3.6581E4,1.7049885E0,3.844605E-3,2.2746666E2,1.7892157E-1,3.91014E5,-8.642938E-3,5.23896E5,4.9217E4,7.0217915E-2,4.7658E4,4.25E0,2.4444838E-3,-1.2101227E-3,1.3493607E-2,6.685216E-3,5.077728E8,1.6394367E0,2.5895438E-3,1.4440845E-2,1.0307031E9,3.4955364E7,3.424835E6,4.6568102E8,2.1592189E5,1.7148E4,5.5851797E-3,-1.7925315E-3,1.5148404E1,8.4569194E10,3.130603E3,1.8600049E10,-4.0650074E-3,-1.7249856E-2,3.9401836E11,4.410963E-3,6.3495394E-3,5.511673E0,-8.38588E-3,-2.1556257E-3,1.208686E6,2.9910075E5,6.771756E-3,2.028E3,4.433625E2,7.218466E7,1.216E3,4.345542E1,-4.3942026E-5,1.6904226E7,4.13568E5,1.1E1,1.2E1,1E0,-6.101747E-3,-3.798289E-4,4.6187375E4,1.06E2,-3.7927732E-3,1.1715537E-3,4.07882E7,-4.4206455E-3,6.1099976E-3,1.311141E-2,1.8E1,7.5E2,2.894E3,5.2921E4,1E0,1.196966E6,2.7009541E-2,-7.363271E-3,1.176E3,4.709E3,-1.617231E-3,1.4762334E-3,-1.3896493E-3,2.634458E-3,1.592E1,3.080528E1,1.479E3,3.83E3,2.6704392E10,1.0931163E9,-6.437417E-3,1.5710552E-3,5.707753E9,1.277094E6,3.6111112E0,4.215533E3,2.3977574E7,1.4966443E0,8.520668E0,1.6990049E0,2.7597793E-3,-5.1405374E-3,-4.952281E-3,-9.489638E-4,-1.2144649E-2,-4.0904265E-3,1.564168E-3,-7.57354E-3,1.5165204E-4,3.285599E-3,1.6002905E-3,-2.2243191E-3,6.699882E-3,-1.7603558E-3,-7.0891134E-3,-1.9972587E-3,4.5876262E-5,4.3398296E-3,-2.762868E-3,2.8889588E-4,-4.717153E-3,2.226006E-3,3.2041396E-3,-2.4109244E-4,-4.8406646E-3,-7.6192175E-4,-2.4999322E-3,1.4983836E-3,-2.623625E-3,3.0744828E-3,-4.863346E-3,-1.0491543E-2,-9.114803E-4,-7.024862E-3,-1.3190843E-3,-4.8376094E-3,3.1661415E-3,-1.6459808E-3,4.1909446E-3,1.111951E-3,-1.8560204E-3,-8.805679E-3,7.0068073E-3,-7.3262374E-4,-1.9807143E-2,-1.8284149E-3,-3.3932412E-3,-6.9787237E-4,-7.219561E-5,-5.0331047E-3,5.498066E-3,2.6740655E-4,-2.2936517E-3,3.323073E-3,-2.4790594E-3,-9.533293E-3,-4.1555893E-4,-3.8342448E-3,-1.0995913E-3,4.8327004E-3,-1.2613647E-2,-4.9870386E-4,1.0833817E-3,5.24058E-3,-3.8413291E-3,1.5423622E-3,5.2597024E-3,4.7807037E-4,6.52622E-3,1.9085632E-3,-1.9273827E-3,2.2084157E-3,3.6140021E-3,7.0250393E-3,1.9134601E-3,-1.0546449E-3,-4.715256E-3,-9.783391E-4,-4.796749E-3,4.225113E-4,-1.9883267E-4,3.8380804E-3,-3.8667384E-3,2.2817035E-3,2.05718E-3,-7.179893E-3,6.8060784E-3,1.370045E-3,2.8174743E-3,-3.6285157E-5,1.8998407E-4,-5.9465906E-3,1.0961569E-2,3.6642568E-3,6.7738993E-3,4.8252937E-4,6.949714E-3,6.0371653E-4,-4.5450707E-3,3.4165988E-3],"split_indices":[20,52,7,27,27,28,41,52,53,0,34,9,4,0,10,52,45,5,45,47,0,10,8,9,0,45,2,0,9,10,0,52,48,3,7,57,54,28,9,105,28,5,12,47,65,3,29,7,48,0,56,56,56,3,54,85,9,52,39,0,2,2,0,52,33,4,54,0,51,4,104,53,1,0,29,4,28,10,0,102,0,2,0,0,7,0,9,12,2,10,0,0,8,39,0,54,9,0,0,2,53,53,0,0,38,52,29,0,0,39,105,0,0,0,10,0,32,0,10,52,57,0,0,0,0,2,0,2,17,1,0,0,1,9,2,0,0,0,31,9,9,1,38,0,52,57,29,0,9,11,57,10,54,0,0,0,0,7,42,0,0,31,7,32,5,28,9,0,0,56,31,4,5,0,0,31,0,0,53,0,0,28,28,0,0,4,45,10,58,0,47,9,10,8,16,0,0,33,8,0,0,7,0,0,0,3,0,0,1,100,9,38,0,10,2,0,0,0,0,58,53,0,2,31,5,0,0,12,11,54,4,47,54,35,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E3,8.23E2,1.777E3,7.6E2,6.3E1,1.235E3,5.42E2,4.88E2,2.72E2,5E0,5.8E1,5.91E2,6.44E2,3.43E2,1.99E2,2.09E2,2.79E2,9.6E1,1.76E2,5.4E1,4E0,4.2E2,1.71E2,6.38E2,6E0,1.08E2,2.35E2,2E1,1.79E2,2.04E2,5E0,1.43E2,1.36E2,7.8E1,1.8E1,6.6E1,1.1E2,1.6E1,3.8E1,3.74E2,4.6E1,7.5E1,9.6E1,4.93E2,1.45E2,1.9E1,8.9E1,1E2,1.35E2,6E0,1.4E1,2.1E1,1.58E2,1.3E1,1.91E2,1.26E2,1.7E1,1.25E2,1.1E1,7E0,7.1E1,1.2E1,6E0,2.6E1,4E1,3.6E1,7.4E1,7E0,9E0,1.1E1,2.7E1,3.57E2,1.7E1,4E0,4.2E1,6.1E1,1.4E1,9.1E1,5E0,4.88E2,5E0,1.4E2,5E0,4E0,1.5E1,6E0,8.3E1,5.7E1,4.3E1,1.3E2,5E0,4E0,1E1,1.7E1,4E0,1.47E2,1.1E1,6E0,7E0,1.7E2,2.1E1,1.2E2,6E0,8E0,9E0,8.2E1,4.3E1,4E0,7E0,9E0,6.2E1,8E0,4E0,5E0,2.1E1,1E1,3E1,7E0,2.9E1,4.6E1,2.8E1,4E0,5E0,4E0,7E0,2E1,7E0,2.12E2,1.45E2,1E1,7E0,4E0,3.8E1,1.6E1,4.5E1,4E0,1E1,4E0,8.7E1,1.63E2,3.25E2,3.4E1,1.06E2,6E0,9E0,9E0,7.4E1,6E0,5.1E1,3.5E1,8E0,1.12E2,1.8E1,6E0,4E0,1.2E1,5E0,3.3E1,1.14E2,4E0,7E0,6.4E1,1.06E2,1E1,1.1E1,7.6E1,4.4E1,5E0,4E0,9E0,7.3E1,3E1,1.3E1,5E0,4E0,5.8E1,4E0,6E0,1.5E1,4E0,6E0,1.3E1,1.7E1,4E0,2.5E1,2.3E1,2.3E1,1.8E1,1E1,5E0,1.5E1,9.4E1,1.18E2,9E0,1.36E2,4E0,6E0,2.3E1,1.5E1,1E1,6E0,4.1E1,4E0,4E0,6E0,8E0,7.9E1,1.47E2,1.6E1,2E2,1.25E2,3E1,4E0,8.9E1,1.7E1,4E0,5E0,5E0,4E0,6.6E1,8E0,2E1,3.1E1,2.4E1,1.1E1,4E0,4E0,5.4E1,5.8E1,9E0,9E0,1.8E1,1.5E1,2.6E1,8.8E1,6E0,5.8E1,1.1E1,9.5E1,6E0,4E0,6E0,5E0,2.4E1,5.2E1,1.9E1,2.5E1,5E0,4E0,9E0,6.4E1,1.7E1,1.3E1,4E0,9E0,5.1E1,7E0,6E0,9E0,6E0,7E0,4E0,1.3E1,1.8E1,7E0,4E0,1.9E1,1.1E1,1.2E1,9E0,9E0,5E0,5E0,1E1,5E0,8.4E1,1E1,1.5E1,1.03E2,5E0,4E0,8.2E1,5.4E1,1.9E1,4E0,1.1E1,4E0,4E0,3.7E1,4E0,4E0,6.3E1,1.6E1,1.29E2,1.8E1,9E0,7E0,1.36E2,6.4E1,4.7E1,7.8E1,7E0,2.3E1,6.4E1,2.5E1,1.1E1,6E0,1.6E1,5E1,4E0,4E0,1.3E1,7E0,5E0,2.6E1,6E0,1.8E1,5E0,6E0,5E1,4E0,4.6E1,1.2E1,5E0,4E0,4E0,5E0,1.4E1,4E0,1.1E1,4E0,2.2E1,4E0,6E0,8.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"335","size_leaf_vector":"1"}},{"base_weights":[7.158245E-5,-3.8673557E-2,1.864859E-2,-3.300868E-2,-1.5904038E-1,-2.4399733E-2,3.2091126E-2,-3.776123E-2,1.2084426E-1,-1.1367525E-1,-1.8711565E-2,-5.1992506E-2,1.9832741E-2,1.8220365E-2,6.615591E-2,-7.162564E-2,-2.2752324E-2,1.5101807E-2,4.358883E-2,-1.771255E-1,6.8356167E-3,-2.7097916E-2,-7.272203E-2,1.4102218E-1,1.1621228E-2,2.2283198E-2,-6.898396E-2,4.4551875E-2,9.00329E-2,-1.0268381E-1,-4.2745013E-2,9.80051E-3,-5.0081164E-2,9.86854E-2,-1.8796243E-3,-1.2612672E-1,-1.3046545E-2,2.4251267E-3,-1.0033748E-3,-6.689961E-2,1.7452138E-2,-6.25729E-2,-1.4269188E-2,9.34232E-3,2.8820322E-3,1.3101876E-3,9.221289E-2,-6.090038E-3,3.8029343E-2,-9.496545E-2,2.663627E-3,5.1024575E-2,-9.992093E-2,1.3698976E-1,5.5187628E-2,-6.7062795E-2,-1.4318033E-1,-8.884881E-3,-7.0499174E-2,-3.9150015E-2,2.1617807E-2,-9.40528E-2,-2.927994E-2,6.298523E-3,2.0302248E-3,-1.0953376E-2,-5.7661876E-2,-4.809129E-2,-1.0383529E-2,4.0008523E-2,-1.1023881E-2,-8.755711E-2,-3.3422865E-2,7.6856394E-3,-6.7944867E-3,6.8908846E-3,1.1774821E-3,-1.851808E-3,-8.578244E-3,7.241487E-2,2.4316167E-2,-5.747934E-3,-4.691183E-2,1.026002E-2,4.500748E-2,2.4272366E-3,-1.1414125E-2,1.5658578E-1,6.959937E-3,4.862319E-3,7.718143E-2,-1.4146852E-1,-3.810714E-2,-6.870502E-2,-1.8758036E-1,-5.6042638E-2,1.8120464E-2,-9.817436E-2,-2.7456954E-2,-8.697593E-3,-1.4955089E-2,2.7700283E-2,-7.496892E-2,-5.993459E-2,-1.6035585E-1,-4.4273812E-2,6.1937566E-3,5.545978E-4,-5.264118E-3,-5.323016E-3,-2.9731927E-2,9.433553E-2,-1.3788175E-2,-4.8241716E-2,-1.09200634E-1,2.397908E-3,-4.445646E-2,-3.8477767E-2,2.2063214E-2,8.872749E-3,-7.151734E-2,1.4394906E-1,4.8944604E-2,7.725807E-3,6.8981916E-2,-3.8645093E-4,-3.2745095E-3,1.8376349E-1,3.781964E-2,7.478299E-2,1.7523962E-1,4.2650956E-3,-3.6275268E-3,-1.0546221E-2,6.066343E-2,1.4965862E-1,5.7446864E-2,-1.8425553E-1,-1.6147449E-3,-5.1344536E-2,4.0327297E-3,-1.0750339E-1,3.8449562E-3,-2.4230585E-1,-7.617907E-2,-1.1216289E-1,3.124691E-2,6.048188E-3,-9.929949E-3,-1.27395E-1,-4.942787E-2,-3.2553945E-3,-5.778817E-3,1.9890398E-2,-4.7360234E-2,6.226474E-2,1.6946556E-2,-5.0335465E-4,-5.3043235E-3,-7.995306E-2,4.1966636E-2,-2.4828124E-1,-9.958282E-2,6.904781E-3,-5.8892537E-2,6.8391957E-3,-5.5848295E-3,1.4033654E-2,-6.057093E-2,-7.515622E-4,1.2790065E-1,-4.9575556E-2,5.9825215E-3,-8.708344E-2,1.6172007E-2,-1.8686172E-1,-7.529776E-2,-7.149646E-2,1.1963347E-2,-5.803222E-2,2.6318016E-3,1.531064E-3,5.3611882E-2,-3.1959888E-2,2.5665214E-2,4.4221547E-3,-9.069049E-2,1.7296751E-1,-8.7750476E-4,3.7161417E-2,1.328772E-1,1.9290239E-2,-4.979433E-2,-3.3485936E-3,7.868673E-2,4.4224644E-3,1.2105224E-2,-4.593401E-3,4.2822067E-2,5.7724537E-3,1.4882523E-3,1.9477545E-1,9.226098E-2,7.722969E-3,3.229718E-2,9.033557E-3,2.5600004E-3,-4.7540274E-3,7.027427E-2,-1.1771179E-2,-4.6802275E-3,-3.3799016E-3,1.9892599E-3,-9.0878345E-3,-3.0658299E-3,-1.419675E-2,-4.517256E-3,5.198021E-4,-6.0344976E-3,-2.3577794E-4,-7.9642655E-3,4.0604235E-3,-1.2500763E-3,-4.31988E-3,1.0906331E-3,-1.0066476E-2,-4.0317243E-3,-5.8527277E-3,-9.786187E-4,9.2865026E-4,-4.2771124E-3,-7.4512215E-4,3.8507357E-3,-9.671825E-4,-6.090898E-3,-4.1151052E-5,3.922932E-3,-3.4141514E-4,2.5643315E-3,-1.5352208E-3,-5.017178E-3,-1.0695143E-3,4.913891E-3,-7.2842427E-3,-1.6313288E-2,-1.6636248E-3,-1.0889383E-2,-1.1019894E-3,5.670287E-3,-3.372421E-3,2.9045732E-3,-2.2931867E-3,9.036062E-4,7.0014372E-3,-1.4456068E-3,-1.9177634E-3,-6.402781E-3,3.508548E-3,7.3753954E-3,7.7447604E-4,-3.4790358E-3,-1.1070374E-3,-5.264337E-3,-1.7249046E-3,2.5018854E-3,-1.09210545E-2,-4.61793E-3,2.7601025E-4,-4.328915E-3,-2.5191586E-3,-6.2697106E-3,-2.8876052E-3,2.428526E-3,-1.05826846E-4,-3.7325402E-3,2.600966E-3,-1.0407376E-3,-9.1154897E-4,3.6347553E-3,1.9373035E-3,-4.2513027E-3,8.55669E-3,1.0047328E-3,-2.574448E-3,-9.101138E-3,7.115033E-3,1.6079547E-2,2.9017695E-3,-1.5015573E-3,3.672177E-3,1.1839504E-2,1.2975176E-3,-4.059396E-3,2.968671E-3,-3.4236424E-3,1.0862426E-2,3.4765538E-3,5.143366E-3,1.2046688E-3,7.3851803E-3,1.2184812E-2,6.185197E-3,1.1676211E-3,-1.5362941E-3,3.5235614E-3,6.2265065E-3,4.6506364E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,-1,67,69,71,-1,-1,-1,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,99,101,103,105,-1,-1,-1,107,109,-1,111,-1,113,115,117,-1,-1,-1,119,-1,121,123,-1,125,-1,127,-1,-1,129,131,133,135,137,139,141,143,145,147,149,151,153,-1,155,157,159,161,163,165,-1,-1,-1,167,169,171,173,175,-1,177,179,181,183,185,187,189,191,193,-1,-1,195,197,199,201,-1,-1,-1,203,205,207,209,-1,211,-1,213,-1,215,217,219,221,-1,223,225,227,229,-1,231,233,235,237,-1,-1,239,241,243,245,247,249,-1,251,253,255,-1,257,259,-1,261,263,265,267,269,271,273,-1,275,277,279,281,-1,283,285,-1,287,289,291,293,-1,295,-1,-1,-1,297,-1,-1,299,301,-1,303,-1,-1,-1,305,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8411698E0,5.600146E-1,1.0013185E0,5.8439964E-1,3.1904984E-1,5.0387573E-1,6.2155545E-1,3.8973653E-1,3.231528E-1,2.4589697E-1,0E0,1.2910688E-1,1.565898E-1,3.3386737E-1,1.9252372E-1,2.0729804E-1,4.764435E-1,0E0,8.424779E-2,5.700755E-2,1.4443408E-2,2.0862818E-1,2.802472E-1,2.7510896E-2,1.2452679E-1,4.0098348E-1,1.3851027E-1,1.9399449E-1,2.8696012E-1,1.534034E-1,1.1590381E-1,1.4256975E-1,2.6341325E-1,1.2355901E-2,0E0,8.295748E-2,0E0,0E0,0E0,1.5432301E-1,3.0955544E-1,9.3936265E-2,0E0,0E0,0E0,1.1833388E-1,5.007857E-2,2.2496164E-1,2.7031457E-1,2.8358638E-2,0E0,1.7088997E-1,1.8285322E-1,1.9256353E-1,1.15843475E-1,1.2818602E-1,1.5906096E-1,7.360567E-2,7.830614E-2,3.776642E-1,1.1839895E-1,2.0076829E-1,1.0611993E-1,0E0,0E0,0E0,3.3962067E-2,5.691625E-2,0E0,1.5327664E-1,0E0,5.514753E-2,5.8725007E-2,8.646854E-2,0E0,0E0,0E0,2.3523399E-1,0E0,2.6829845E-1,3.0624968E-1,0E0,8.24797E-3,0E0,1.8055314E-1,0E0,0E0,8.760679E-2,7.4807554E-2,4.104207E-1,9.581488E-2,7.344043E-2,7.649312E-2,1.2694749E-1,1.7662895E-1,1.063777E-1,1.0802321E-1,5.21037E-2,5.9389293E-2,4.9574655E-2,0E0,6.8452105E-2,2.3218222E-2,1.3186945E-1,1.4092374E-1,1.05149895E-1,9.311896E-2,0E0,0E0,0E0,6.0236253E-2,9.737612E-2,1.349491E-1,6.8960704E-2,1.0441178E-1,0E0,8.628846E-2,5.8310002E-2,6.404634E-2,1.8801104E-1,1.3688555E-1,1.9050926E-1,1.1868626E-1,2.0251243E-1,1.5309542E-1,0E0,0E0,2.8711438E-2,1.20767415E-1,2.222614E-2,6.794572E-2,0E0,0E0,0E0,6.692426E-2,3.6701113E-2,1.2216951E-1,3.3727467E-2,0E0,6.50932E-2,0E0,4.7185764E-2,0E0,1.0771096E-1,4.758615E-2,6.521565E-2,2.7360315E-2,0E0,7.479387E-2,7.0232034E-2,3.2053985E-2,4.295539E-2,0E0,5.288594E-2,3.7300408E-2,4.5162365E-2,1.1805633E-1,0E0,0E0,5.0290316E-2,4.1404277E-2,4.8592508E-2,1.4964816E-1,1.01266965E-1,1.13631696E-1,0E0,5.425877E-2,1.0698458E-1,3.4065865E-2,0E0,1.2369007E-2,2.9786233E-2,0E0,1.7835632E-2,2.0631457E-2,1.8725574E-2,3.0719087E-2,3.6914393E-2,5.1368162E-2,2.4073794E-2,0E0,6.999459E-2,5.4940782E-2,3.0389705E-1,1.5350749E-1,0E0,1.2673071E-1,1.0888529E-1,0E0,1.5776637E-1,7.528925E-2,1.719478E-1,1.083567E-1,0E0,1.1754352E-1,0E0,0E0,0E0,1.9324878E-1,0E0,0E0,6.0792685E-2,2.3169972E-2,0E0,5.5179197E-2,0E0,0E0,0E0,1.8104804E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,39,39,40,40,41,41,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,66,66,67,67,69,69,71,71,72,72,73,73,77,77,79,79,80,80,82,82,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,106,106,110,110,111,111,112,112,113,113,114,114,116,116,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,127,127,128,128,129,129,130,130,134,134,135,135,136,136,137,137,139,139,141,141,143,143,144,144,145,145,146,146,148,148,149,149,150,150,151,151,153,153,154,154,155,155,156,156,159,159,160,160,161,161,162,162,163,163,164,164,166,166,167,167,168,168,170,170,171,171,173,173,174,174,175,175,176,176,177,177,178,178,179,179,181,181,182,182,183,183,184,184,186,186,187,187,189,189,190,190,191,191,192,192,194,194,198,198,201,201,202,202,204,204,208,208],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,-1,68,70,72,-1,-1,-1,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,100,102,104,106,-1,-1,-1,108,110,-1,112,-1,114,116,118,-1,-1,-1,120,-1,122,124,-1,126,-1,128,-1,-1,130,132,134,136,138,140,142,144,146,148,150,152,154,-1,156,158,160,162,164,166,-1,-1,-1,168,170,172,174,176,-1,178,180,182,184,186,188,190,192,194,-1,-1,196,198,200,202,-1,-1,-1,204,206,208,210,-1,212,-1,214,-1,216,218,220,222,-1,224,226,228,230,-1,232,234,236,238,-1,-1,240,242,244,246,248,250,-1,252,254,256,-1,258,260,-1,262,264,266,268,270,272,274,-1,276,278,280,282,-1,284,286,-1,288,290,292,294,-1,296,-1,-1,-1,298,-1,-1,300,302,-1,304,-1,-1,-1,306,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.2427474E8,7.805608E4,6.7652373E3,5.39978E5,1.6006084E3,2.8781365E6,1.638058E2,1.5151515E-3,4.6726016E3,-1.8711565E-2,4.869229E7,5.322348E7,4.2343444E7,1.2974394E6,2.308943E0,1.6256282E7,1.5101807E-2,1.4955851E7,9.112E3,5.198287E5,5.222222E1,9.438605E-6,2.368E0,4.982003E6,8.61E2,3.220188E3,1E0,1.9521575E5,1.710933E0,2.25E2,2.4576474E8,9.5822406E2,8.931957E9,-1.8796243E-3,3.051E3,-1.3046545E-2,2.4251267E-3,-1.0033748E-3,2.0223091E1,9.03E2,8.69032E-2,-1.4269188E-2,9.34232E-3,2.8820322E-3,1.2003743E7,4.13568E5,5.432133E0,1.3E1,1.1336898E-1,2.663627E-3,3.8E1,1E1,2E1,3.0597075E8,2.1155348E6,1.16146E5,7.771773E3,5E0,7.8091515E6,1.1900813E1,4.3702424E7,6.654321E0,6.298523E-3,2.0302248E-3,-1.0953376E-2,2.2222222E-1,1.64E1,-1.0383529E-2,4.1E0,-1.1023881E-2,9.5E1,1.1092533E0,2.36413E5,-6.7944867E-3,6.8908846E-3,1.1774821E-3,1.8E1,-8.578244E-3,1.0258198E-2,1.0989723E3,-5.747934E-3,3.63E2,1.026002E-2,9.188401E4,2.4272366E-3,-1.1414125E-2,4.607796E-1,6.6288E4,1.3E0,9.489796E-1,1.5915463E0,1.6695403E0,6.29E2,1.2429E4,6.4506575E5,9.7549E4,7.12E2,3.95221E5,4.61E2,-1.4955089E-2,2.6414216E-1,2.9786E4,1E0,1.7314286E0,1E1,1.9051096E7,5.545978E-4,-5.264118E-3,-5.323016E-3,3.5229592E0,5.214E3,7E0,5.9E1,4.2285586E3,2.397908E-3,7.5E1,2.4708056E5,1.0743855E-1,1.654762E2,1.1216729E2,1E0,8.579284E-1,5.2879925E5,2.006639E0,-3.8645093E-4,-3.2745095E-3,1.22159E5,6.14E2,3.565214E6,1.5449402E7,4.2650956E-3,-3.6275268E-3,-1.0546221E-2,5.388794E6,6.1095314E-8,1.0195312E0,9.908197E1,-1.6147449E-3,3.8E1,4.0327297E-3,1.9360614E0,3.8449562E-3,1.9051096E7,2.4284733E5,8.628E3,3.8E1,6.048188E-3,3.9838778E5,2.2837209E2,5.1086234E5,1.2335714E2,-5.778817E-3,1.4563726E-1,6.949118E-1,1.08511E5,1.6525185E9,-5.0335465E-4,-5.3043235E-3,9E0,5.500988E2,3.325328E11,2.5073426E0,6.2105E4,3.4409692E3,6.8391957E-3,4.888E3,3E1,6.233195E-2,-7.515622E-4,1.279012E7,7.73756E0,5.9825215E-3,1.4733E4,2.0011E5,3.596E3,9E0,3.925E4,2.1196338E4,1E0,2.6318016E-3,7.474142E-4,1.06158945E5,4.611111E0,9.391714E4,4.4221547E-3,1.5837E4,3.535297E3,-8.7750476E-4,3.2251606E7,9.4538574E2,4.004366E1,1.912E3,-3.3485936E-3,8.117555E-5,4.4224644E-3,1.2105224E-2,-4.593401E-3,3.14E2,5.7724537E-3,1.4882523E-3,5.8084745E6,1.5353E4,7.722969E-3,2.5030267E5,9.033557E-3,2.5600004E-3,-4.7540274E-3,2.1298597E0,-1.1771179E-2,-4.6802275E-3,-3.3799016E-3,1.9892599E-3,-9.0878345E-3,-3.0658299E-3,-1.419675E-2,-4.517256E-3,5.198021E-4,-6.0344976E-3,-2.3577794E-4,-7.9642655E-3,4.0604235E-3,-1.2500763E-3,-4.31988E-3,1.0906331E-3,-1.0066476E-2,-4.0317243E-3,-5.8527277E-3,-9.786187E-4,9.2865026E-4,-4.2771124E-3,-7.4512215E-4,3.8507357E-3,-9.671825E-4,-6.090898E-3,-4.1151052E-5,3.922932E-3,-3.4141514E-4,2.5643315E-3,-1.5352208E-3,-5.017178E-3,-1.0695143E-3,4.913891E-3,-7.2842427E-3,-1.6313288E-2,-1.6636248E-3,-1.0889383E-2,-1.1019894E-3,5.670287E-3,-3.372421E-3,2.9045732E-3,-2.2931867E-3,9.036062E-4,7.0014372E-3,-1.4456068E-3,-1.9177634E-3,-6.402781E-3,3.508548E-3,7.3753954E-3,7.7447604E-4,-3.4790358E-3,-1.1070374E-3,-5.264337E-3,-1.7249046E-3,2.5018854E-3,-1.09210545E-2,-4.61793E-3,2.7601025E-4,-4.328915E-3,-2.5191586E-3,-6.2697106E-3,-2.8876052E-3,2.428526E-3,-1.05826846E-4,-3.7325402E-3,2.600966E-3,-1.0407376E-3,-9.1154897E-4,3.6347553E-3,1.9373035E-3,-4.2513027E-3,8.55669E-3,1.0047328E-3,-2.574448E-3,-9.101138E-3,7.115033E-3,1.6079547E-2,2.9017695E-3,-1.5015573E-3,3.672177E-3,1.1839504E-2,1.2975176E-3,-4.059396E-3,2.968671E-3,-3.4236424E-3,1.0862426E-2,3.4765538E-3,5.143366E-3,1.2046688E-3,7.3851803E-3,1.2184812E-2,6.185197E-3,1.1676211E-3,-1.5362941E-3,3.5235614E-3,6.2265065E-3,4.6506364E-4],"split_indices":[20,45,28,52,11,55,47,52,57,4,0,31,5,45,28,54,45,0,47,10,32,4,37,58,29,2,52,6,33,54,10,5,55,12,0,2,0,0,0,52,29,38,0,0,0,1,9,53,3,57,0,0,8,3,7,45,1,48,8,45,54,45,54,0,0,0,58,4,0,58,0,10,53,9,0,0,0,8,0,38,52,0,0,0,28,0,0,38,10,54,53,39,54,10,10,28,12,10,1,0,0,41,9,105,53,3,45,0,0,0,54,9,8,10,33,0,3,33,38,4,4,74,35,48,53,0,0,2,29,47,51,0,0,0,47,37,53,4,0,3,0,54,0,45,47,9,10,0,48,4,51,52,0,38,27,12,5,0,0,3,4,31,53,10,52,0,2,2,38,0,7,58,0,1,9,2,58,29,33,64,0,39,33,58,28,0,29,33,0,12,4,53,11,0,41,0,0,0,0,0,0,45,2,0,33,0,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.556E3,8.28E2,1.728E3,7.92E2,3.6E1,4.11E2,1.317E3,7.69E2,2.3E1,3.1E1,5E0,2.53E2,1.58E2,9.37E2,3.8E2,2.35E2,5.34E2,6E0,1.7E1,2E1,1.1E1,1.16E2,1.37E2,9E0,1.49E2,8.96E2,4.1E1,2.01E2,1.79E2,1.12E2,1.23E2,2.44E2,2.9E2,1E1,7E0,1.4E1,6E0,4E0,7E0,6.1E1,5.5E1,1.32E2,5E0,5E0,4E0,1.33E2,1.6E1,3.2E2,5.76E2,3.4E1,7E0,1.93E2,8E0,7.5E1,1.04E2,6.1E1,5.1E1,5.6E1,6.7E1,4.7E1,1.97E2,9.2E1,1.98E2,6E0,4E0,5E0,9E0,5.5E1,6E0,5.1E1,4E0,7E1,6.2E1,1.28E2,5E0,9E0,7E0,3.13E2,7E0,1.63E2,4.13E2,2.3E1,1.1E1,6E0,1.87E2,4E0,4E0,6.5E1,1E1,3.2E1,7.2E1,1.6E1,4.5E1,2E1,3.1E1,2E1,3.6E1,4E1,2.7E1,4.3E1,4E0,1.86E2,1.1E1,6.2E1,3E1,1.39E2,5.9E1,4E0,5E0,1.2E1,4.3E1,2.5E1,2.6E1,2.6E1,4.4E1,7E0,5.5E1,3E1,9.8E1,2.72E2,4.1E1,3.9E1,1.24E2,3.02E2,1.11E2,4E0,7E0,8E0,1.79E2,1.3E1,5.2E1,5E0,5E0,6E0,2.6E1,1.4E1,5.8E1,1.1E1,5E0,4.1E1,4E0,1.6E1,4E0,2E1,1.1E1,1.2E1,8E0,7E0,2.9E1,2.4E1,1.6E1,2.2E1,5E0,2.5E1,1.8E1,4.3E1,1.43E2,4E0,7E0,5.2E1,1E1,1.1E1,1.9E1,3.1E1,1.08E2,4E0,5.5E1,1.8E1,2.5E1,6E0,1.9E1,2.1E1,5E0,1.6E1,1E1,1.2E1,3.2E1,3.7E1,1.8E1,2.5E1,5E0,6E1,3.8E1,7.9E1,1.93E2,4E0,3.7E1,3.3E1,6E0,1.1E2,1.4E1,2.52E2,5E1,7E0,1.04E2,4E0,4E0,6E0,1.73E2,6E0,7E0,4.1E1,1.1E1,5E0,2.1E1,1E1,4E0,4E0,5.4E1,6E0,5E0,3.5E1,6E0,5E0,1.1E1,1.5E1,5E0,4E0,7E0,4E0,8E0,4E0,4E0,8E0,2.1E1,8E0,1.6E1,4E0,1.2E1,1.8E1,4E0,1.6E1,9E0,1.4E1,4E0,9E0,3.4E1,8.5E1,5.8E1,1.6E1,3.6E1,5E0,5E0,6E0,5E0,1.3E1,6E0,2.5E1,6E0,1.01E2,7E0,2E1,3.5E1,4E0,1.4E1,2E1,5E0,6E0,1.3E1,5E0,1.6E1,4E0,1.2E1,4E0,6E0,8E0,4E0,4E0,2.8E1,2.8E1,9E0,6E0,1.2E1,6E0,1.9E1,1.8E1,4.2E1,8E0,3E1,3.4E1,4.5E1,6E0,1.87E2,2.7E1,1E1,2.9E1,4E0,8.4E1,2.6E1,1E1,4E0,2.37E2,1.5E1,7E0,4.3E1,5E0,9.9E1,4E1,1.33E2,2.3E1,1.8E1,7E0,4E0,8E0,1.3E1,2.8E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"307","size_leaf_vector":"1"}},{"base_weights":[-3.0451847E-4,-2.099458E-2,3.322723E-2,-2.4398612E-2,1.5626106E-1,6.4880056E-3,6.307778E-2,-5.4601267E-2,-6.0461247E-3,2.4131514E-1,-1.2588276E-2,-3.7289653E-2,1.9814476E-2,9.257768E-2,3.7024915E-2,-8.846736E-2,-3.731204E-2,-3.9073676E-2,2.757618E-3,1.5860373E-2,1.334693E-1,2.504732E-3,-4.890412E-3,-2.9132642E-2,-8.292998E-3,4.307965E-2,-1.717589E-2,1.1856149E-1,5.2687176E-2,2.4876108E-2,1.6014333E-1,-7.931855E-2,-2.7177927E-1,-1.0370131E-1,-2.8159551E-2,-4.5361247E-2,1.0219716E-1,3.5227746E-2,-1.8107042E-2,1.0411509E-2,2.4387697E-3,-3.9951183E-2,2.425298E-2,-1.816515E-2,4.9230777E-2,-2.6069483E-2,5.037178E-3,1.4371157E-1,6.046228E-2,-1.664685E-2,1.1204904E-1,-2.0447433E-2,4.7976352E-2,2.1535282E-1,5.5326096E-4,-9.8817356E-2,3.982402E-3,-1.8069142E-2,-6.3746856E-3,-1.3134948E-2,-8.278117E-2,-4.7952216E-2,-7.263329E-3,-2.489225E-2,-9.484017E-2,2.2152779E-3,6.9763353E-3,-6.956962E-2,4.756339E-2,-5.491739E-2,8.25239E-3,-2.0883316E-2,-7.572064E-2,-1.5736207E-3,4.0157914E-2,3.2862935E-2,-5.893602E-3,-3.3126934E-3,5.3300545E-2,3.453551E-2,-4.439831E-2,8.776629E-3,1.5859598E-1,-2.5264332E-3,7.38315E-2,-1.2149491E-2,2.156641E-2,1.5713735E-1,5.3848503E-3,6.0513724E-2,-3.7838195E-2,5.3883687E-2,-6.701185E-3,1.6529258E-2,1.32801E-1,-8.468962E-2,-2.3404552E-1,-3.5454985E-3,3.8416777E-2,-5.7435926E-2,-7.168721E-3,-8.5731195E-3,-4.261114E-2,-3.960006E-2,9.5473295E-3,-1.0144594E-1,-1.2765918E-2,-2.7326757E-1,-3.0392472E-2,-1.206269E-2,-2.8805165E-2,1.7709915E-1,3.812209E-2,-3.595004E-2,-1.29865E-1,-1.044749E-1,1.2935841E-2,1.4114096E-2,-5.0284434E-2,2.3997213E-4,-9.0271324E-2,6.8197586E-2,1.7955739E-4,-1.9238207E-2,4.3655303E-3,7.591459E-2,2.4477767E-2,-5.481136E-3,6.9359936E-2,-1.7473225E-2,-8.350015E-2,-4.557179E-3,6.3456926E-3,1.3024922E-1,2.2049247E-1,-4.9839527E-3,9.934303E-2,8.744793E-3,-8.828532E-3,1.7311713E-1,3.6502073E-3,4.496052E-2,-4.135171E-3,-1.6197938E-3,5.0430335E-3,4.5842946E-2,-5.4183647E-2,2.6715117E-2,1.05513036E-1,3.2408142E-3,8.1125405E-3,-1.2860884E-1,-5.3356238E-2,-1.6533546E-2,-5.9984773E-3,5.941327E-3,2.5047201E-3,-3.8571026E-2,-7.574555E-3,-6.267734E-3,-6.539588E-2,1.6238064E-2,-9.225691E-2,-1.9075525E-4,7.2533E-2,-6.0322516E-2,-8.1608435E-3,-3.6906367E-3,-7.076646E-3,-1.7510417E-1,-2.171419E-2,-8.088052E-2,1.7026722E-2,-6.286806E-3,-1.0372478E-3,5.428993E-3,1.1665894E-2,5.1416438E-2,-1.945005E-2,2.9811931E-3,-6.49653E-2,-7.384621E-2,-2.14572E-1,-1.5620927E-3,-7.087489E-3,2.2017624E-2,-3.67757E-2,4.612071E-2,-3.980804E-3,-7.461656E-2,9.587171E-3,-1.0285917E-1,-5.8937626E-4,4.75138E-3,1.3823111E-3,-3.956622E-3,2.2263378E-3,8.364869E-2,-2.8766908E-2,-3.7980232E-2,4.4676594E-2,1.2184396E-1,-3.8514608E-3,-7.1515734E-3,-7.2233574E-3,-2.02105E-1,-5.239356E-2,1.5072788E-1,4.7767118E-2,1.4747342E-2,8.613872E-3,-2.7287346E-3,2.7764186E-3,1.6887282E-3,5.6023984E-3,-5.390959E-3,2.4560181E-2,4.5999405E-3,9.361485E-3,-1.3098334E-3,5.428467E-3,5.292664E-3,-1.809297E-4,-9.212623E-2,-2.7246661E-2,3.987279E-2,-6.228251E-3,1.3454956E-1,6.2441584E-2,-7.614293E-3,3.833925E-5,-4.248504E-3,1.7156713E-3,8.6301495E-4,-1.2630423E-3,-3.0623819E-3,4.2218165E-4,2.4200417E-3,-1.9977142E-3,1.4922319E-3,-4.0162955E-3,7.0792628E-3,-7.8942E-4,-9.639675E-3,-2.6598135E-3,3.2806608E-3,-7.405676E-4,6.603351E-3,9.044469E-4,-6.5206466E-3,-7.451756E-6,4.5105154E-4,-2.084431E-3,-2.2669458E-3,-1.2129765E-2,-1.2320245E-3,-6.483242E-3,2.9089565E-3,-2.1157714E-3,1.2481594E-3,-4.721416E-3,1.3715175E-3,4.715826E-3,-4.5037894E-3,1.9990468E-4,-3.9936148E-4,5.8815125E-3,-6.2856376E-3,-1.5425179E-3,-8.117172E-3,-1.8645277E-3,-7.439171E-3,-1.4405E-2,-5.2923583E-3,1.3012873E-3,-1.0080261E-3,-5.1333797E-3,1.5036795E-4,5.63866E-3,-1.094634E-3,-4.903342E-3,-1.1914308E-3,2.721889E-3,-5.9382906E-3,-2.7997224E-3,2.8738931E-3,6.458155E-3,-4.04482E-3,1.4575247E-3,3.2193218E-3,-3.0821252E-3,9.7104517E-4,4.6151853E-3,3.92362E-3,8.5382E-3,2.701659E-4,-2.80899E-3,-3.9898395E-3,-1.4187486E-2,1.4820679E-3,-4.4095493E-3,9.1619175E-3,4.9535614E-3,-1.5169196E-4,6.478631E-3,-3.5968528E-3,2.3718416E-3,-2.7990618E-3,-7.4106706E-3,-2.0840568E-3,3.102125E-3,2.651376E-3,-2.512563E-3,7.759161E-3,2.358013E-3,1.0895716E-3,5.447552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,-1,-1,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,73,75,77,79,-1,81,83,85,87,89,91,93,-1,95,97,-1,-1,-1,99,101,103,105,107,-1,-1,109,111,113,115,117,119,-1,121,123,-1,-1,125,127,129,131,133,-1,135,-1,137,139,141,143,145,147,-1,-1,149,151,153,-1,155,157,-1,-1,159,161,163,165,167,169,171,-1,173,175,177,179,181,183,185,187,189,-1,191,193,-1,195,-1,197,199,-1,201,203,205,-1,-1,207,209,211,213,-1,215,217,-1,219,-1,-1,-1,221,223,225,227,-1,-1,229,231,-1,-1,-1,233,235,-1,237,239,241,243,245,247,249,-1,251,-1,253,-1,255,257,-1,259,-1,-1,261,263,265,267,269,271,-1,-1,273,275,277,-1,279,281,283,-1,-1,-1,-1,-1,285,287,289,291,293,-1,-1,295,297,299,301,303,-1,-1,-1,-1,-1,-1,-1,305,-1,-1,-1,-1,-1,-1,307,309,311,-1,313,315,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7781464E0,9.6017617E-1,7.802112E-1,8.621511E-1,4.3387747E-1,3.0235755E-1,3.516457E-1,3.409841E-1,2.8186005E-1,1.2178409E-1,6.374274E-2,1.2217277E-1,3.4239304E-1,2.1763909E-1,3.6546448E-1,3.10506E-1,2.3466188E-1,1.8583891E-1,5.1958615E-1,0E0,5.324112E-2,0E0,0E0,6.73877E-2,0E0,9.281513E-2,1.6422418E-1,1.8126667E-1,3.5993963E-1,2.3759957E-1,1.7867815E-1,3.0900455E-1,7.046527E-2,1.3474512E-1,1.4234096E-1,1.9614482E-1,1.3298079E-2,3.9113188E-1,4.5358324E-1,0E0,0E0,6.3895226E-2,1.8997563E-2,1.20989084E-1,1.0791266E-1,1.6149119E-1,0E0,1.7936659E-1,6.458992E-2,3.6011928E-1,2.251913E-1,1.1070389E-1,1.68576E-1,1.0856217E-1,0E0,2.7439106E-1,9.8204486E-2,0E0,0E0,0E0,5.7901174E-2,1.0955995E-1,9.224677E-2,1.2866911E-1,6.4715105E-1,0E0,0E0,2.1721156E-1,3.2223177E-1,2.7253515E-1,1.454826E-1,6.656335E-2,3.909862E-2,0E0,1.615898E-2,4.711795E-2,0E0,0E0,1.3827878E-1,1.7838284E-1,1.1540404E-1,1.2978227E-1,1.13400936E-1,0E0,7.489112E-2,0E0,1.6944861E-1,3.1693757E-2,5.6045905E-2,5.585225E-2,9.002352E-2,2.0197475E-1,0E0,0E0,1.1559874E-2,1.8717325E-1,1.1483729E-1,0E0,7.485588E-2,5.099287E-2,0E0,0E0,1.4119089E-1,1.7203313E-1,6.8710126E-2,3.849727E-2,1.4215067E-1,1.6141951E-1,1.0432354E-1,0E0,7.260479E-2,4.4088244E-2,1.938768E-1,1.7754088E-1,1.6921687E-1,2.5660455E-2,1.1954616E-1,9.3724966E-2,5.170349E-2,0E0,2.6446313E-2,7.179953E-3,0E0,3.7946925E-2,0E0,9.956038E-2,1.2280834E-1,0E0,2.2459383E-1,8.525768E-2,1.5543371E-1,0E0,0E0,9.076923E-2,3.5297036E-2,3.3083413E-2,1.9664943E-2,0E0,1.05184555E-1,1.3911426E-2,0E0,5.2705348E-2,0E0,0E0,0E0,3.3073805E-2,5.2681044E-2,1.9815412E-1,5.4130733E-2,0E0,0E0,1.7854726E-1,2.3621011E-1,0E0,0E0,0E0,8.231637E-3,3.0426934E-2,0E0,1.2549384E-1,1.5227199E-1,1.18782826E-1,1.07584685E-1,9.538474E-2,4.6288133E-2,5.4877102E-2,0E0,5.5990588E-2,0E0,8.5532874E-2,0E0,5.450228E-2,5.8712542E-2,0E0,5.585124E-2,0E0,0E0,2.0850366E-1,7.992734E-2,8.655221E-2,1.8277693E-1,7.485877E-2,2.9717684E-2,0E0,0E0,1.1760385E-1,4.282472E-2,6.582499E-2,0E0,2.8837577E-2,1.7806007E-2,1.1306733E-2,0E0,0E0,0E0,0E0,0E0,1.2603158E-1,2.9571427E-2,6.2233176E-2,8.608188E-2,3.118822E-2,0E0,0E0,3.92888E-2,6.7368984E-2,1.11478046E-1,5.92649E-2,5.4478236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.64975E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.6324173E-2,4.5222923E-2,1.0969515E-1,0E0,4.5206904E-2,3.9266296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,60,60,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,74,74,75,75,78,78,79,79,80,80,81,81,82,82,84,84,86,86,87,87,88,88,89,89,90,90,91,91,94,94,95,95,96,96,98,98,99,99,102,102,103,103,104,104,105,105,106,106,107,107,108,108,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,120,120,121,121,123,123,125,125,126,126,128,128,129,129,130,130,133,133,134,134,135,135,136,136,138,138,139,139,141,141,145,145,146,146,147,147,148,148,151,151,152,152,156,156,157,157,159,159,160,160,161,161,162,162,163,163,164,164,165,165,167,167,169,169,171,171,172,172,174,174,177,177,178,178,179,179,180,180,181,181,182,182,185,185,186,186,187,187,189,189,190,190,191,191,197,197,198,198,199,199,200,200,201,201,204,204,205,205,206,206,207,207,208,208,216,216,223,223,224,224,225,225,227,227,228,228],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,-1,-1,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,74,76,78,80,-1,82,84,86,88,90,92,94,-1,96,98,-1,-1,-1,100,102,104,106,108,-1,-1,110,112,114,116,118,120,-1,122,124,-1,-1,126,128,130,132,134,-1,136,-1,138,140,142,144,146,148,-1,-1,150,152,154,-1,156,158,-1,-1,160,162,164,166,168,170,172,-1,174,176,178,180,182,184,186,188,190,-1,192,194,-1,196,-1,198,200,-1,202,204,206,-1,-1,208,210,212,214,-1,216,218,-1,220,-1,-1,-1,222,224,226,228,-1,-1,230,232,-1,-1,-1,234,236,-1,238,240,242,244,246,248,250,-1,252,-1,254,-1,256,258,-1,260,-1,-1,262,264,266,268,270,272,-1,-1,274,276,278,-1,280,282,284,-1,-1,-1,-1,-1,286,288,290,292,294,-1,-1,296,298,300,302,304,-1,-1,-1,-1,-1,-1,-1,306,-1,-1,-1,-1,-1,-1,308,310,312,-1,314,316,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5988182E-1,9.256843E3,6.860185E2,1.0119178E10,5.5544252E7,1.5288235E2,1.3226592E-2,2.308943E0,1.3062E4,8.0886055E3,5.8E1,1.044E1,2.5291866E1,2.1298597E0,4.5505118E2,1.2427474E8,1.5944675E-4,4.11E2,9.87156E0,1.5860373E-2,3.224158E6,2.504732E-3,-4.890412E-3,1.188E3,-8.292998E-3,3.9595376E7,2.2425473E0,2.2E1,2.9251662E6,1E0,2.6104508E10,3.402546E6,4.230839E6,3.5521298E2,4.0434834E5,8E0,2.2E1,2.35E2,6.29E2,1.0411509E-2,2.4387697E-3,6.704299E5,6.439778E5,3.4015749E0,5.0972332E7,1.7710843E0,5.037178E-3,1E0,1.517647E0,1.3E0,1.5277338E0,1E0,4.42384E5,1.381E3,5.5326096E-4,7.8351054E11,6.6126086E4,-1.8069142E-2,-6.3746856E-3,-1.3134948E-2,1.5626E5,5.684E3,1.513E3,8.333333E0,3.05402E5,2.2152779E-3,6.9763353E-3,6.9436204E-1,2.04E5,6.896236E-2,1.4343751E6,4.3304763E0,2.3067484E0,-1.5736207E-3,1.9802104E6,2.3871907E2,-5.893602E-3,-3.3126934E-3,1.6358411E8,6.0607E4,9.2882293E-1,1.8796511E5,1.8361508E3,-2.5264332E-3,5.68197E6,-1.2149491E-2,5.388794E6,2.878424E5,6.5346925E6,2.3487206E7,7.197394E1,8.253647E6,-6.701185E-3,1.6529258E-2,1.1E1,4.5089152E-1,1.066317E12,-3.5454985E-3,1.8000048E10,2.881356E-2,-7.168721E-3,-8.5731195E-3,2.0354E4,1.0989011E-2,4.2E1,3.0052083E0,1.1855755E8,4.9E1,9.757E3,-1.206269E-2,3.3820656E7,1.139605E6,3.2884402E4,4.0233E4,3.5905025E5,1.2224265E1,4.2343444E7,2.6E1,3.1019E5,2.3997213E-4,2.983871E0,1E0,1.7955739E-4,1.2939234E5,4.3655303E-3,2.51343E5,1.8114872E7,-5.481136E-3,1.0692383E7,2.0184135E0,6.63E2,-4.557179E-3,6.3456926E-3,5.685916E8,7.557E3,1.2414683E5,3.9739098E4,8.744793E-3,7.2201815E6,2.1871264E0,3.6502073E-3,2.8363478E0,-4.135171E-3,-1.6197938E-3,5.0430335E-3,4.1045683E8,6.77603E5,5.7136595E8,1.9610367E0,3.2408142E-3,8.1125405E-3,4.222002E8,4.6847186E2,-1.6533546E-2,-5.9984773E-3,5.941327E-3,5.641203E5,2.9143541E-6,-7.574555E-3,1.7669278E5,1.2839224E0,1E0,3.680591E-3,5.3474556E5,7.64E2,9E0,-8.1608435E-3,2.589369E7,-7.076646E-3,8.317E3,-2.171419E-2,2.2E1,1.0617118E8,-6.286806E-3,3.4643276E1,5.428993E-3,1.1665894E-2,1.2747E4,1.0115874E4,3.5E1,1.8817276E1,7.9E1,3.17E2,-1.5620927E-3,-7.087489E-3,1.0460138E4,5.3641737E-1,8E0,-3.980804E-3,3E1,5.3038636E7,1.1255588E1,-5.8937626E-4,4.75138E-3,1.3823111E-3,-3.956622E-3,2.2263378E-3,1.059448E7,3.956E3,1.0545219E5,1.592E1,1.2738854E0,-3.8514608E-3,-7.1515734E-3,6.978874E5,2.6877522E5,1.5418775E9,5.59277E5,9.97958E5,1.4747342E-2,8.613872E-3,-2.7287346E-3,2.7764186E-3,1.6887282E-3,5.6023984E-3,-5.390959E-3,1.2710136E5,4.5999405E-3,9.361485E-3,-1.3098334E-3,5.428467E-3,5.292664E-3,-1.809297E-4,9.3262426E8,9.609747E3,9.71709E5,-6.228251E-3,1.9099288E7,6.513793E1,-7.614293E-3,3.833925E-5,-4.248504E-3,1.7156713E-3,8.6301495E-4,-1.2630423E-3,-3.0623819E-3,4.2218165E-4,2.4200417E-3,-1.9977142E-3,1.4922319E-3,-4.0162955E-3,7.0792628E-3,-7.8942E-4,-9.639675E-3,-2.6598135E-3,3.2806608E-3,-7.405676E-4,6.603351E-3,9.044469E-4,-6.5206466E-3,-7.451756E-6,4.5105154E-4,-2.084431E-3,-2.2669458E-3,-1.2129765E-2,-1.2320245E-3,-6.483242E-3,2.9089565E-3,-2.1157714E-3,1.2481594E-3,-4.721416E-3,1.3715175E-3,4.715826E-3,-4.5037894E-3,1.9990468E-4,-3.9936148E-4,5.8815125E-3,-6.2856376E-3,-1.5425179E-3,-8.117172E-3,-1.8645277E-3,-7.439171E-3,-1.4405E-2,-5.2923583E-3,1.3012873E-3,-1.0080261E-3,-5.1333797E-3,1.5036795E-4,5.63866E-3,-1.094634E-3,-4.903342E-3,-1.1914308E-3,2.721889E-3,-5.9382906E-3,-2.7997224E-3,2.8738931E-3,6.458155E-3,-4.04482E-3,1.4575247E-3,3.2193218E-3,-3.0821252E-3,9.7104517E-4,4.6151853E-3,3.92362E-3,8.5382E-3,2.701659E-4,-2.80899E-3,-3.9898395E-3,-1.4187486E-2,1.4820679E-3,-4.4095493E-3,9.1619175E-3,4.9535614E-3,-1.5169196E-4,6.478631E-3,-3.5968528E-3,2.3718416E-3,-2.7990618E-3,-7.4106706E-3,-2.0840568E-3,3.102125E-3,2.651376E-3,-2.512563E-3,7.759161E-3,2.358013E-3,1.0895716E-3,5.447552E-3],"split_indices":[38,4,52,20,45,52,57,54,9,33,3,54,58,38,56,45,42,8,58,0,47,0,0,2,0,7,38,3,28,102,19,1,1,4,28,8,3,2,10,0,0,50,51,56,31,53,0,67,54,54,53,64,2,0,0,31,28,0,0,0,12,9,2,52,1,0,0,53,5,38,45,54,58,0,50,52,0,0,7,1,38,33,52,0,50,0,47,33,45,50,58,1,0,0,3,27,31,0,5,57,0,0,9,57,3,54,45,3,9,0,7,31,50,1,28,56,45,8,47,0,53,16,0,33,0,1,12,0,50,53,10,0,0,7,2,28,28,0,51,42,0,42,0,0,0,7,29,32,39,0,0,7,4,0,0,0,28,38,0,28,53,102,39,32,0,3,0,45,0,9,0,8,7,0,52,0,0,12,33,3,56,0,10,0,0,50,27,17,0,0,7,58,0,0,0,0,0,12,2,28,58,53,0,0,28,28,31,1,1,0,0,0,0,0,0,0,33,0,0,0,0,0,0,7,52,29,0,9,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.561E3,1.584E3,9.77E2,1.555E3,2.9E1,5.16E2,4.61E2,5.87E2,9.68E2,1.9E1,1E1,1.2E2,3.96E2,2.15E2,2.46E2,1.97E2,3.9E2,2.03E2,7.65E2,1E1,9E0,6E0,4E0,1.14E2,6E0,2.43E2,1.53E2,1.29E2,8.6E1,2.25E2,2.1E1,1.89E2,8E0,4.6E1,3.44E2,1.95E2,8E0,2.99E2,4.66E2,4E0,5E0,9.5E1,1.9E1,2.2E1,2.21E2,1.43E2,1E1,8.9E1,4E1,4E1,4.6E1,7.6E1,1.49E2,1.5E1,6E0,1.53E2,3.6E1,4E0,4E0,4E0,4.2E1,1.76E2,1.68E2,1.39E2,5.6E1,4E0,4E0,3.1E1,2.68E2,1.94E2,2.72E2,6.3E1,3.2E1,4E0,1.5E1,1.5E1,7E0,7E0,2.14E2,3.3E1,1.1E2,9E0,8E1,4E0,3.6E1,5E0,3.5E1,3.2E1,1.4E1,1.3E1,6.3E1,1.45E2,4E0,5E0,1E1,1.4E2,1.3E1,1.1E1,2.5E1,3.1E1,1.1E1,6E0,1.7E2,5.7E1,1.11E2,1.8E1,1.21E2,1.4E1,4.2E1,5E0,2.6E1,1.7E1,2.51E2,1.56E2,3.8E1,1E1,2.62E2,2.9E1,3.4E1,5E0,2.7E1,8E0,7E0,8E0,7E0,1.19E2,9.5E1,6E0,2.7E1,6.6E1,4.4E1,5E0,4E0,5.7E1,2.3E1,9E0,2.7E1,5E0,3E1,2.6E1,6E0,1E1,4E0,4E0,9E0,1E1,5.3E1,9.6E1,4.9E1,4E0,6E0,5.7E1,8.3E1,6E0,7E0,7E0,1.8E1,2.7E1,4E0,6.6E1,1.04E2,2.8E1,2.9E1,9.7E1,1.4E1,1.2E1,6E0,1.14E2,7E0,1E1,4E0,2E1,2.2E1,5E0,2.1E1,9E0,8E0,2.04E2,4.7E1,6.7E1,8.9E1,2.4E1,1.4E1,4E0,6E0,2.22E2,4E1,2.2E1,7E0,2.4E1,1E1,2.3E1,4E0,4E0,4E0,4E0,4E0,1.11E2,8E0,2.3E1,7.2E1,2E1,7E0,4E0,6.2E1,8E0,3.6E1,4.5E1,1.2E1,7E0,1.6E1,5E0,4E0,5E0,2.2E1,7E0,2.3E1,5E0,2.1E1,5E0,5E0,4E0,6E0,2.1E1,3.2E1,8.9E1,7E0,2.8E1,2.1E1,4.8E1,9E0,6.1E1,2.2E1,1.2E1,6E0,1.8E1,9E0,2.5E1,4.1E1,1.4E1,9E1,5E0,2.3E1,7E0,2.2E1,1.7E1,8E1,6E0,8E0,5E0,7E0,8.6E1,2.8E1,4E0,6E0,1E1,1E1,1.3E1,9E0,1.7E1,4E0,1.32E2,7.2E1,1.1E1,3.6E1,6.2E1,5E0,3.1E1,5.8E1,6E0,1.8E1,9E0,5E0,6E0,2.16E2,3.3E1,7E0,1.4E1,8E0,8E0,1.6E1,6E0,4E0,1.6E1,7E0,7.2E1,3.9E1,4E0,4E0,4E0,1.9E1,4.8E1,2.4E1,1.2E1,8E0,5E1,1.2E1,4E0,4E0,1.1E1,2.5E1,2.6E1,1.9E1,8E0,4E0,4E0,1.9E1,1.4E1,7E0,2.8E1,4E0,7.8E1,1.1E1,2.2E1,6E0,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"317","size_leaf_vector":"1"}},{"base_weights":[3.8353584E-4,1.4196014E-2,-3.82799E-2,3.8445294E-3,7.318968E-2,-6.209081E-2,-4.476114E-3,-2.8216725E-3,7.459247E-2,2.1871735E-1,5.9682664E-2,-5.511349E-2,-2.0826605E-1,-1.5140644E-1,1.4173753E-3,-3.3437677E-2,8.06965E-3,4.1051947E-2,9.201023E-2,1.3200958E-1,2.819309E-1,7.058053E-2,-7.6800324E-2,-6.734064E-2,2.1307882E-2,-6.0635034E-2,-1.7446354E-2,-2.1070284E-3,-1.0383269E-2,-1.4366623E-2,4.219138E-2,-4.006748E-2,6.1685584E-2,-1.0519971E-2,2.1245178E-2,1.0907219E-2,1.0442779E-1,5.5717338E-2,1.06228545E-1,8.821868E-3,1.7155993E-3,2.0811472E-2,5.3503737E-3,5.9843745E-2,2.0265867E-1,-9.838918E-3,-2.3636581E-2,-5.7701614E-2,-1.702764E-1,1.08175464E-1,-8.873923E-3,-5.8984966E-3,1.0187349E-3,-3.698562E-2,1.1811862E-2,2.6193164E-2,1.1231241E-1,-2.1944683E-2,-7.733838E-2,1.4248574E-1,1.0527521E-2,8.791542E-3,-1.2961136E-2,2.3597933E-2,-1.16942255E-2,4.829504E-2,-4.216288E-2,1.5527712E-1,-1.9235236E-3,-1.9650403E-3,7.808902E-2,4.169661E-2,1.2312561E-1,1.6456282E-1,5.1164176E-2,-4.948938E-4,3.097656E-1,-6.265039E-2,1.4940373E-3,-4.838519E-2,-1.04434006E-1,-1.11398555E-1,-1.5167193E-2,3.1457925E-3,8.842883E-3,-3.2768562E-2,7.972575E-2,4.644068E-3,-4.4265892E-2,-4.8138155E-3,6.262105E-2,-1.0257261E-2,4.299466E-2,7.889432E-3,2.2072599E-3,-1.479665E-2,-1.9130298E-1,-1.305277E-2,-6.874533E-2,2.40197E-3,9.864253E-3,-2.3518796E-3,3.187021E-2,-3.0256486E-2,1.1670882E-2,1.9254725E-3,4.1901894E-2,4.699153E-3,6.1448566E-3,-6.8442695E-2,4.3290932E-4,1.0273401E-2,3.5317256E-3,9.614453E-2,1.153449E-3,-2.556273E-3,8.011336E-2,1.3096951E-1,1.2375658E-3,9.280631E-3,4.736262E-3,6.354061E-2,-2.0438904E-2,8.448458E-3,1.7936647E-2,-5.024521E-4,-5.1323236E-3,-9.028861E-2,-3.1904377E-2,-4.1040476E-2,-1.4892447E-1,-9.423739E-3,-6.794441E-2,-4.988249E-2,2.626141E-3,6.321798E-3,8.487356E-4,-3.3886805E-2,-1.2704866E-1,5.0825667E-2,-3.0832067E-2,6.8218485E-3,2.1753762E-2,3.904951E-2,-5.0491467E-2,5.817449E-2,-1.6282332E-3,-6.0692903E-2,2.5973455E-3,-3.3244519E-3,-1.5128203E-2,-1.3505894E-2,-9.876281E-2,2.9324354E-3,2.496828E-5,-4.4048976E-2,7.000948E-3,3.9742712E-2,-1.7431986E-2,6.5970668E-3,-1.1201985E-2,4.8012245E-2,-6.6430636E-2,-2.5772539E-3,2.7354257E-3,-5.178076E-3,-6.260153E-4,6.406782E-3,6.0868807E-2,1.7304664E-3,5.6083626E-3,9.330211E-3,1.1654344E-1,1.6684377E-1,5.251216E-2,-1.16742425E-1,6.0949523E-2,-6.403437E-2,-1.2925832E-1,2.1444522E-2,-6.2121358E-2,-1.0498095E-2,8.201079E-3,-2.6357454E-1,-6.4217515E-2,-1.1324778E-5,-1.0178062E-1,-1.1456847E-3,-9.610591E-2,2.608193E-3,-4.049973E-2,-8.740093E-3,-2.8982698E-3,2.4952132E-2,5.9947674E-3,-8.203737E-2,1.3929189E-2,-7.485498E-4,3.524709E-2,-2.792434E-4,3.4846591E-3,-3.8946278E-3,1.764676E-4,8.191795E-2,1.8824538E-3,-2.0637095E-3,-1.2377796E-2,4.817719E-3,-1.0348123E-3,4.6060006E-3,-1.3519335E-3,-6.7836135E-3,-1.5892923E-3,-2.428718E-3,5.3775315E-3,3.235287E-3,-1.5174866E-3,-6.8529067E-4,3.0409563E-3,-8.211383E-3,-4.488515E-4,7.232002E-5,6.8494626E-3,5.717206E-3,1.9216156E-3,-1.0001055E-2,-8.707837E-4,1.2986584E-3,4.5244326E-3,1.9822633E-3,6.7215976E-3,9.946108E-3,2.4846501E-3,1.502574E-3,4.3699057E-3,-2.199086E-3,-9.672382E-3,5.851113E-3,-4.1034745E-4,-4.036481E-3,1.9202112E-3,-8.669049E-3,-2.2543764E-3,-1.5492949E-3,2.8351957E-3,-4.349811E-3,-5.347694E-4,-1.8981268E-3,3.5319307E-3,-1.5829764E-2,-6.280235E-3,-4.5993817E-3,1.3268224E-3,-7.7450606E-3,-2.0218466E-3,-6.7965393E-3,-1.4483371E-3,-4.376259E-3,-1.423376E-3,4.288108E-3,-6.312339E-4,-2.4850096E-3,-5.149613E-3,1.889646E-3,-9.5353904E-4,3.089086E-3,3.7137742E-4,5.20164E-3,2.5845678E-3,1.9372271E-3,-1.0696771E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,53,55,57,59,61,63,65,67,69,71,-1,-1,-1,-1,73,75,-1,77,79,81,83,85,-1,-1,87,89,91,93,95,97,99,101,-1,103,105,-1,107,109,111,-1,-1,113,115,117,119,121,-1,123,125,-1,127,129,131,-1,-1,-1,133,135,-1,137,139,141,143,145,-1,-1,147,149,-1,151,-1,-1,-1,153,155,157,159,161,-1,163,165,-1,-1,-1,167,-1,-1,169,171,-1,-1,-1,173,175,-1,-1,-1,-1,177,179,181,183,-1,185,187,-1,-1,-1,189,191,193,195,-1,197,199,201,203,-1,205,207,-1,-1,209,211,-1,-1,213,215,217,219,221,-1,223,225,-1,-1,-1,-1,-1,227,-1,-1,-1,229,231,233,235,237,239,241,243,245,-1,247,249,251,-1,253,-1,255,-1,257,-1,-1,259,-1,261,263,-1,265,-1,-1,-1,-1,267,269,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3938482E0,1.174041E0,5.526875E-1,7.7225244E-1,5.5108654E-1,3.995459E-1,2.4700533E-1,4.9944267E-1,7.85951E-2,9.4329E-2,3.980872E-1,3.6211944E-1,3.5323393E-1,5.900526E-2,1.7758977E-1,2.503164E-1,2.7086902E-1,9.438336E-2,4.1555345E-2,4.6149597E-2,2.6130295E-1,3.3530533E-1,1.2305495E-1,3.205775E-1,1.4253825E-1,4.892522E-2,0E0,0E0,0E0,1.1827093E-1,8.343765E-2,2.468695E-1,1.0597303E-1,2.1121696E-1,3.941272E-1,7.148239E-2,1.222589E-1,6.260749E-2,6.5742135E-2,0E0,0E0,0E0,0E0,1.9881314E-1,4.128738E-1,0E0,3.3203673E-2,1.2817276E-1,1.8589139E-1,3.2411724E-2,8.941185E-2,0E0,0E0,1.0532747E-1,7.8862E-2,3.963599E-2,3.650658E-2,2.9860348E-1,1.7166412E-1,4.237257E-2,2.2397585E-2,0E0,1.9376591E-1,2.545955E-1,0E0,3.991853E-2,2.0628786E-2,3.6557734E-2,0E0,0E0,2.0622715E-2,5.7607055E-2,3.610313E-2,7.509768E-3,1.8917114E-1,0E0,2.2375226E-2,1.7931752E-2,0E0,1.7460424E-1,1.349085E-1,5.7635307E-2,0E0,0E0,0E0,5.081063E-2,2.4273816E-2,0E0,8.33341E-2,1.0434964E-1,6.6498645E-2,4.363186E-2,5.2377246E-2,0E0,0E0,1.9175427E-1,1.1892778E-1,0E0,1.9101137E-1,0E0,0E0,0E0,1.0907345E-2,1.3751245E-1,1.542833E-1,3.1309217E-1,2.3354799E-1,0E0,3.648865E-2,1.8376634E-2,0E0,0E0,0E0,1.1431247E-2,0E0,0E0,1.2186475E-2,1.60442E-2,0E0,0E0,0E0,1.9744724E-1,2.5793117E-1,0E0,0E0,0E0,0E0,6.5387666E-2,3.0117208E-1,1.848698E-1,2.6471198E-1,0E0,3.4346618E-2,3.2681122E-2,0E0,0E0,0E0,5.448749E-2,2.621068E-2,3.8156725E-2,1.1345139E-1,0E0,8.458821E-3,1.4089255E-2,1.8072486E-2,4.9298793E-2,0E0,2.2913063E-1,3.8467473E-1,0E0,0E0,6.2644415E-2,1.814838E-1,0E0,0E0,1.4003181E-1,1.5950735E-1,1.0920574E-1,1.1395511E-1,1.9466753E-1,0E0,2.0497906E-1,1.2101729E-1,0E0,0E0,0E0,0E0,0E0,7.998537E-3,0E0,0E0,0E0,4.9943864E-2,5.4234028E-2,1.2675834E-1,7.5304404E-2,7.086466E-2,8.033277E-2,9.6395046E-2,1.2765306E-1,1.5162873E-1,0E0,5.4635003E-2,4.826039E-2,4.7917873E-2,0E0,2.6363075E-2,0E0,2.2848524E-2,0E0,4.6149105E-2,0E0,0E0,4.2262927E-2,0E0,1.0494798E-2,2.2014942E-2,0E0,8.410853E-3,0E0,0E0,0E0,0E0,1.2029737E-2,1.1139548E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,43,43,44,44,46,46,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,65,65,66,66,67,67,70,70,71,71,72,72,73,73,74,74,76,76,77,77,79,79,80,80,81,81,85,85,86,86,88,88,89,89,90,90,91,91,92,92,95,95,96,96,98,98,102,102,103,103,104,104,105,105,106,106,108,108,109,109,113,113,116,116,117,117,121,121,122,122,127,127,128,128,129,129,130,130,132,132,133,133,137,137,138,138,139,139,140,140,142,142,143,143,144,144,145,145,147,147,148,148,151,151,152,152,155,155,156,156,157,157,158,158,159,159,161,161,162,162,168,168,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,180,180,182,182,183,183,184,184,186,186,188,188,190,190,193,193,195,195,196,196,198,198,203,203,204,204],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,54,56,58,60,62,64,66,68,70,72,-1,-1,-1,-1,74,76,-1,78,80,82,84,86,-1,-1,88,90,92,94,96,98,100,102,-1,104,106,-1,108,110,112,-1,-1,114,116,118,120,122,-1,124,126,-1,128,130,132,-1,-1,-1,134,136,-1,138,140,142,144,146,-1,-1,148,150,-1,152,-1,-1,-1,154,156,158,160,162,-1,164,166,-1,-1,-1,168,-1,-1,170,172,-1,-1,-1,174,176,-1,-1,-1,-1,178,180,182,184,-1,186,188,-1,-1,-1,190,192,194,196,-1,198,200,202,204,-1,206,208,-1,-1,210,212,-1,-1,214,216,218,220,222,-1,224,226,-1,-1,-1,-1,-1,228,-1,-1,-1,230,232,234,236,238,240,242,244,246,-1,248,250,252,-1,254,-1,256,-1,258,-1,-1,260,-1,262,264,-1,266,-1,-1,-1,-1,268,270,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,1.5445492E3,2.6476662E6,3.4663152E6,1.1485E4,4.4337E5,1.5953778E0,1.8122449E2,4.5723195E6,3.1140776E0,2.3580047E10,1.998175E3,1E1,1.3717948E0,1.243766E0,1.3195207E7,1.7322648E5,1.4521204E0,2.47E2,3.392E3,1.044E1,1.921032E4,3.7225406E4,1.0828989E8,2.5927516E11,1.4510472E1,-1.7446354E-2,-2.1070284E-3,-1.0383269E-2,2.2100918E1,9.408768E1,2.9806678E0,1.22E2,7.330957E4,3.7852024E1,3.9813398E6,1.819149E1,5E0,7.45E2,8.821868E-3,1.7155993E-3,2.0811472E-2,5.3503737E-3,3.61565E5,5.0279167E1,-9.838918E-3,3.8085933E3,2.8216E4,4.784E3,1.2E1,5.57E4,-5.8984966E-3,1.0187349E-3,2.0966542E0,6.604754E5,4.664004E8,3.1699734E2,1E0,4.034E3,1.2484015E7,2.0368582E8,8.791542E-3,5.121E3,3.7810526E2,-1.16942255E-2,2.1283955E-3,9.283875E6,2.52E2,-1.9235236E-3,-1.9650403E-3,1.425263E10,8E0,1.4440248E9,2.040457E0,3.388159E7,-4.948938E-4,3.2917362E7,7.160724E1,1.4940373E-3,1.04E3,2.8591623E2,3.488E3,-1.5167193E-2,3.1457925E-3,8.842883E-3,5.2E1,3.4E1,4.644068E-3,7.362237E6,1.5497989E5,3E0,3.099779E6,1.1708007E7,7.889432E-3,2.2072599E-3,4.7E2,5.3E1,-1.305277E-2,2.0616984E4,2.40197E-3,9.864253E-3,-2.3518796E-3,1.216E3,1.4887473E-1,8E-3,2.9347712E6,1.6015803E7,4.699153E-3,1E1,2.5086691E0,4.3290932E-4,1.0273401E-2,3.5317256E-3,9.25E0,1.153449E-3,-2.556273E-3,1.1216729E2,1.1057693E0,1.2375658E-3,9.280631E-3,4.736262E-3,1.6470588E0,3.4E1,8.448458E-3,1.7936647E-2,-5.024521E-4,-5.1323236E-3,4.6229E4,1.25218E5,4.0492815E-1,5.638132E0,-9.423739E-3,2.2222222E-1,5.913995E5,2.626141E-3,6.321798E-3,8.487356E-4,3.7936268E-3,4.759E3,3.0612E4,3.3720784E5,6.8218485E-3,1.7911884E3,2.631294E2,5.3788322E1,4.29E4,-1.6282332E-3,3.09627E5,3.9E2,-3.3244519E-3,-1.5128203E-2,2.3E2,5.3871745E-1,2.9324354E-3,2.496828E-5,1.0427401E-5,1.48298E5,3.7510395E4,7.118233E7,2.9816154E1,-1.1201985E-2,8E0,1.4523809E0,-2.5772539E-3,2.7354257E-3,-5.178076E-3,-6.260153E-4,6.406782E-3,3.6355584E7,1.7304664E-3,5.6083626E-3,9.330211E-3,1E0,1.0941897E9,6.2105E4,1.01268125E5,8.282879E0,3.925E4,1.56221E5,3.1672379E4,1.6504E4,-1.0498095E-2,3.6411794E5,2.970914E1,1.0219829E1,-1.1324778E-5,6.6404694E2,-1.1456847E-3,6.332424E1,2.608193E-3,8.7404094E5,-8.740093E-3,-2.8982698E-3,4.2864155E1,5.9947674E-3,1.0015482E7,1.0743855E-1,-7.485498E-4,9.536135E6,-2.792434E-4,3.4846591E-3,-3.8946278E-3,1.764676E-4,2.8E1,1.3062E4,-2.0637095E-3,-1.2377796E-2,4.817719E-3,-1.0348123E-3,4.6060006E-3,-1.3519335E-3,-6.7836135E-3,-1.5892923E-3,-2.428718E-3,5.3775315E-3,3.235287E-3,-1.5174866E-3,-6.8529067E-4,3.0409563E-3,-8.211383E-3,-4.488515E-4,7.232002E-5,6.8494626E-3,5.717206E-3,1.9216156E-3,-1.0001055E-2,-8.707837E-4,1.2986584E-3,4.5244326E-3,1.9822633E-3,6.7215976E-3,9.946108E-3,2.4846501E-3,1.502574E-3,4.3699057E-3,-2.199086E-3,-9.672382E-3,5.851113E-3,-4.1034745E-4,-4.036481E-3,1.9202112E-3,-8.669049E-3,-2.2543764E-3,-1.5492949E-3,2.8351957E-3,-4.349811E-3,-5.347694E-4,-1.8981268E-3,3.5319307E-3,-1.5829764E-2,-6.280235E-3,-4.5993817E-3,1.3268224E-3,-7.7450606E-3,-2.0218466E-3,-6.7965393E-3,-1.4483371E-3,-4.376259E-3,-1.423376E-3,4.288108E-3,-6.312339E-4,-2.4850096E-3,-5.149613E-3,1.889646E-3,-9.5353904E-4,3.089086E-3,3.7137742E-4,5.20164E-3,2.5845678E-3,1.9372271E-3,-1.0696771E-3],"split_indices":[27,52,32,28,9,11,53,52,28,54,5,52,3,53,39,9,28,38,0,11,54,4,33,45,31,57,0,0,0,56,56,56,10,45,53,28,56,3,2,0,0,0,0,1,56,0,52,9,10,18,2,0,0,54,28,12,56,6,9,5,7,0,2,52,0,38,45,0,0,0,31,3,12,53,45,0,1,56,0,2,55,2,0,0,0,8,8,0,47,28,8,51,47,0,0,1,0,0,50,0,0,0,10,38,57,47,51,0,8,54,0,0,0,58,0,0,4,54,0,0,0,54,3,0,0,0,0,29,29,27,56,0,58,32,0,0,0,57,2,2,28,0,52,52,58,10,0,9,12,0,0,29,42,0,0,37,1,28,7,56,0,3,54,0,0,0,0,0,7,0,0,0,102,7,10,28,53,29,1,33,9,0,33,58,54,0,33,0,56,0,50,0,0,56,0,32,38,0,48,0,0,0,0,3,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.608E3,1.922E3,6.86E2,1.636E3,2.86E2,4.02E2,2.84E2,1.496E3,1.4E2,2.3E1,2.63E2,3.85E2,1.7E1,1E1,2.74E2,3.92E2,1.104E3,4.9E1,9.1E1,1.1E1,1.2E1,2.44E2,1.9E1,3.32E2,5.3E1,9E0,8E0,4E0,6E0,1.98E2,7.6E1,3.67E2,2.5E1,4.58E2,6.46E2,3.4E1,1.5E1,2.7E1,6.4E1,7E0,4E0,6E0,6E0,2.27E2,1.7E1,5E0,1.4E1,3.05E2,2.7E1,1.3E1,4E1,5E0,4E0,1.06E2,9.2E1,6.3E1,1.3E1,2.48E2,1.19E2,9E0,1.6E1,5E0,4.53E2,6.41E2,5E0,2E1,1.4E1,1.1E1,4E0,5E0,2.2E1,1.4E1,5E1,1.6E1,2.11E2,6E0,1.1E1,8E0,6E0,2.56E2,4.9E1,2E1,7E0,9E0,4E0,3.2E1,8E0,5E0,1.01E2,7E1,2.2E1,2E1,4.3E1,7E0,6E0,2.39E2,9E0,4E0,1.15E2,4E0,5E0,4E0,1.2E1,2.66E2,1.87E2,2.94E2,3.47E2,9E0,1.1E1,9E0,5E0,6E0,5E0,1.6E1,6E0,4E0,1E1,4.6E1,4E0,1.1E1,5E0,1.8E2,3.1E1,4E0,7E0,4E0,4E0,7.1E1,1.85E2,2.1E1,2.8E1,6E0,1.4E1,2.7E1,5E0,4E0,4E0,9.1E1,1E1,2.2E1,4.8E1,7E0,1.5E1,9E0,1.1E1,3.6E1,7E0,6.5E1,1.74E2,5E0,4E0,4.1E1,7.4E1,6E0,6E0,1.94E2,7.2E1,9.5E1,9.2E1,2.89E2,5E0,3.29E2,1.8E1,5E0,6E0,5E0,4E0,7E0,9E0,5E0,5E0,7E0,3.9E1,1.6E1,1.64E2,1.4E1,1.7E1,4.4E1,2.7E1,6.7E1,1.18E2,4E0,1.7E1,1.1E1,1.7E1,5E0,9E0,1.8E1,9E0,6E0,8.5E1,5E0,5E0,1.7E1,5E0,2.2E1,2.6E1,4E0,1.1E1,4E0,5E0,7E0,4E0,2.5E1,1.1E1,6E1,5E0,3.4E1,1.4E2,4E0,3.7E1,4.7E1,2.7E1,1.89E2,5E0,2.8E1,4.4E1,2.7E1,6.8E1,4E0,8.8E1,2.79E2,1E1,4E1,2.89E2,4E0,1.4E1,5E0,4E0,8E0,3.1E1,1.2E1,4E0,1.01E2,6.3E1,8E0,6E0,9E0,8E0,3.8E1,6E0,1.7E1,1E1,2.7E1,4E1,7.9E1,3.9E1,1E1,7E0,7E0,4E0,1.3E1,4E0,4E0,5E0,5E0,4E0,1.6E1,6.9E1,6E0,1.1E1,1E1,1.2E1,1.5E1,1.1E1,5E0,6E0,1.3E1,1.2E1,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"271","size_leaf_vector":"1"}},{"base_weights":[1.1223223E-3,1.4879626E-2,-3.860177E-2,-1.1497493E-2,3.616812E-2,-9.537993E-2,-2.0582665E-2,-5.4718694E-3,-8.053447E-2,-1.0113043E-3,6.257079E-2,-7.512084E-2,-1.8290761E-1,-3.3135585E-2,4.880721E-2,-3.4550186E-3,-1.6062556E-1,-1.6307225E-2,-4.697455E-2,8.706072E-3,-6.86612E-2,4.6264537E-2,1.12263046E-1,-4.868274E-2,-1.7417422E-1,-1.629311E-2,-1.225311E-1,-1.9029934E-2,-9.1734394E-2,1.255705E-1,3.270367E-2,-1.6595306E-2,2.4877742E-2,1.5314462E-3,-1.7889583E-2,-6.5065615E-2,5.0906134E-3,-4.696705E-3,5.7206433E-2,-5.0226904E-2,-1.2139839E-2,5.2805636E-2,-3.5419393E-2,1.8596503E-1,6.810082E-2,-1.6863498E-1,-2.1615587E-2,-9.4726967E-4,-2.1761893E-1,-1.7837006E-1,-3.7687354E-2,-7.058803E-2,-7.6269843E-3,-7.659335E-2,-2.0603472E-1,8.843958E-3,3.5628323E-3,9.202738E-2,1.7535523E-2,-1.0063334E-2,-7.039346E-2,-7.430016E-2,3.121017E-2,-8.261917E-2,2.417832E-3,-1.4218034E-2,5.276192E-2,3.8878586E-2,1.6240633E-1,-1.00611985E-1,2.0763064E-2,7.541178E-2,3.0268673E-2,-6.709177E-2,3.9783236E-2,2.2196612E-1,4.9810167E-2,1.6016959E-1,4.480072E-2,-1.3414547E-1,-1.1910879E-2,7.555066E-2,-4.490431E-2,-1.4115593E-2,-9.9721655E-2,-4.7805877E-3,-1.2936423E-2,1.423635E-3,-5.474585E-3,4.74668E-3,-9.002565E-2,-1.6891876E-2,3.0065112E-2,-1.6757008E-1,-5.4457814E-2,-5.3536207E-3,-1.31771425E-2,-1.1151683E-4,1.3416834E-1,7.2979983E-3,3.7079938E-3,-2.4383325E-2,1.2086668E-2,-1.1779427E-2,-4.2529233E-2,1.8982225E-3,-1.3424411E-1,1.1763729E-1,1.5176564E-2,9.183251E-3,-1.1015998E-1,-1.7215762E-2,7.0707677E-3,-4.069628E-2,8.231581E-2,5.6058563E-2,-6.428619E-3,1.018311E-2,3.1827895E-3,-7.943552E-2,-9.524006E-3,-2.2985935E-3,3.815556E-2,8.7405235E-2,1.1576556E-2,7.9973E-2,8.954623E-3,-9.322691E-2,1.5593255E-3,-1.2112149E-3,3.988895E-3,1.9009879E-2,1.9660634E-1,2.3598128E-4,3.463902E-3,1.10485125E-2,1.2041442E-3,-6.2168803E-2,5.9654266E-2,-2.4279587E-3,-1.6083856E-1,-1.6132127E-3,1.5426475E-1,7.55057E-3,-1.1509584E-1,-7.837203E-3,-1.1317625E-3,-1.7251493E-2,-1.24043636E-1,8.100159E-2,-2.2252198E-2,-2.9408112E-3,3.9936844E-2,-4.698867E-3,-1.1375703E-2,-6.521922E-2,3.9530536E-3,9.34538E-3,2.7217208E-3,3.4476996E-3,-3.6822997E-3,-9.991781E-3,-1.0155581E-1,1.0293368E-1,-7.277704E-4,-8.417678E-2,-3.9340327E-3,-7.053112E-4,-1.0591792E-2,9.958291E-4,1.3651444E-1,-8.334905E-2,2.5608081E-2,2.64716E-3,-9.661903E-4,-2.3943804E-1,-6.0054954E-2,-6.450135E-2,-1.0383821E-2,-4.387462E-3,1.6688404E-3,1.4572774E-1,-3.4936846E-3,1.2517783E-1,3.1436842E-2,-4.71271E-3,-1.2775903E-3,9.09573E-4,3.748777E-3,3.6516666E-2,1.0563353E-1,-9.404977E-3,5.1150985E-2,3.083828E-2,1.2120738E-1,-3.933181E-3,1.0005456E-1,-5.2031286E-2,-9.027885E-3,2.3253813E-1,1.1326597E-1,2.6231098E-3,-7.363205E-3,3.521472E-2,9.636709E-2,-1.0620912E-2,-4.1127945E-3,1.015699E-2,3.2278034E-3,-7.034984E-2,3.1658493E-2,-1.7974614E-1,-3.846246E-2,3.4102615E-2,-5.7700176E-2,-8.9613885E-2,-1.3177023E-2,5.882675E-3,3.0142965E-4,-1.7652101E-3,-4.7842547E-2,6.467049E-2,-1.6338236E-3,-7.039509E-3,-5.355504E-2,-3.098189E-2,3.1405084E-2,-4.6505905E-3,-1.4647991E-4,-1.5389213E-2,-3.6927362E-3,-9.96959E-4,6.488401E-3,3.119722E-4,-4.106424E-3,-8.512408E-3,-2.6247553E-3,-1.2668942E-3,4.766232E-3,3.8102232E-3,9.217613E-3,-1.3626713E-2,1.2506298E-3,-8.864874E-4,2.3884384E-3,-5.982359E-3,-1.495457E-2,-5.166432E-3,-1.7029771E-3,-7.2733883E-4,-8.640457E-3,7.332595E-4,-1.349738E-3,3.655115E-3,8.004898E-3,-4.1117324E-3,1.6309222E-3,3.6683995E-3,9.425404E-3,-6.186816E-4,3.0510675E-3,-1.0085318E-3,4.584779E-3,6.212128E-3,2.7122225E-3,-1.7698518E-3,4.209272E-3,4.2895875E-3,2.2143625E-4,9.013454E-3,3.6556995E-3,2.3483771E-3,-1.2177113E-3,7.000607E-3,-5.033386E-4,-4.2445012E-3,5.7459384E-4,6.8329684E-3,1.3274762E-2,7.779097E-3,1.5369069E-3,-1.3808387E-3,2.6314575E-3,6.5359897E-3,-8.886786E-5,-1.6703968E-3,-5.025818E-3,4.1056206E-3,-6.0913834E-4,-2.575503E-3,-1.2567253E-2,-3.5892932E-3,4.965748E-4,-1.531015E-3,4.5982045E-3,-4.554298E-3,3.322888E-5,-3.150773E-3,-9.702625E-3,1.0097817E-3,-1.2591296E-3,-9.180626E-3,-1.9165645E-3,4.8934403E-3,1.4769853E-3,-2.9286558E-3,1.0626133E-3,-2.0197795E-3,-7.2607426E-3,-2.4964411E-3,2.3653915E-3,2.440373E-3,-5.798527E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,-1,-1,63,-1,65,67,69,-1,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,-1,-1,97,99,101,103,105,107,109,-1,111,113,115,117,119,121,123,125,127,129,131,133,135,137,139,-1,141,143,-1,145,-1,-1,-1,-1,-1,147,149,151,153,155,-1,-1,-1,157,159,-1,161,163,-1,165,-1,167,169,171,173,175,177,-1,179,181,183,-1,-1,-1,185,-1,-1,187,189,191,193,195,197,-1,-1,-1,-1,199,-1,-1,-1,-1,201,203,-1,205,-1,207,209,211,-1,-1,213,215,217,219,-1,221,-1,-1,223,-1,-1,-1,-1,225,227,229,231,233,235,237,-1,-1,-1,239,241,243,-1,-1,245,247,249,251,-1,-1,253,255,257,259,-1,-1,-1,-1,261,263,-1,265,267,269,271,273,275,-1,277,279,-1,-1,281,283,-1,-1,-1,-1,285,287,289,291,293,295,297,-1,-1,-1,299,301,303,305,-1,307,309,311,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3957797E0,1.0656366E0,6.7020804E-1,3.5231957E-1,1.0307815E0,2.6643062E-1,4.3714E-1,2.4383695E-1,5.456662E-1,2.878726E-1,4.9232674E-1,3.3196247E-1,2.1072763E-1,3.4872183E-1,9.107776E-2,2.8782892E-1,3.8855922E-1,0E0,1.7089827E-1,2.4923384E-1,1.6451481E-1,2.4978691E-1,4.788171E-1,3.3401853E-1,1.7794406E-1,0E0,9.7504586E-2,2.0111625E-1,1.261304E-1,2.3658946E-2,5.6875244E-2,1.8483503E-1,1.5618798E-1,0E0,0E0,1.1353199E-1,0E0,1.65422E-1,1.5371713E-1,1.8450764E-1,0E0,2.1679115E-1,8.603373E-2,2.634405E-1,1.9876459E-1,1.3724506E-2,1.9800007E-1,0E0,1.3246447E-1,6.1802655E-2,5.068946E-2,2.0810398E-1,9.898598E-2,1.3989711E-1,2.3083895E-2,0E0,0E0,5.1937684E-2,3.0209007E-2,1.5000737E-1,2.5259644E-1,1.0606643E-1,3.1832024E-1,1.2237066E-1,0E0,1.23345E-1,1.2211823E-1,2.1444836E-1,3.6046803E-2,4.203239E-2,2.7302105E-2,1.6282761E-1,2.2917378E-1,6.713587E-2,3.0139295E-2,1.0204339E-1,1.1110507E-2,1.5224293E-1,1.2764099E-1,2.6440084E-2,0E0,1.4928077E-1,2.586012E-1,0E0,3.6072977E-2,0E0,0E0,0E0,0E0,0E0,1.3649535E-1,1.20918564E-1,5.15073E-2,4.045412E-2,9.2842296E-2,0E0,0E0,0E0,2.773039E-2,3.1419367E-2,0E0,3.1773645E-1,2.1660745E-1,0E0,8.003862E-2,0E0,9.0387344E-2,6.367904E-2,2.0357764E-1,1.6012896E-2,2.2528738E-1,8.170859E-2,0E0,4.1330867E-2,1.800735E-1,1.0838592E-1,0E0,0E0,0E0,1.8430546E-2,0E0,0E0,1.1875587E-2,1.6311264E-1,2.8637406E-1,1.2795475E-1,1.8002748E-1,6.265128E-2,0E0,0E0,0E0,0E0,9.344411E-2,0E0,0E0,0E0,0E0,1.0542412E-1,5.8516353E-2,0E0,2.7181238E-2,0E0,3.0814186E-2,7.9225875E-2,1.4171764E-1,0E0,0E0,4.1434642E-2,1.5910542E-1,3.2333463E-2,1.13157675E-1,0E0,5.254356E-2,0E0,0E0,4.27614E-2,0E0,0E0,0E0,0E0,3.928784E-2,1.4226569E-1,2.3411044E-1,7.867965E-2,9.364166E-2,5.6510046E-2,5.9862603E-2,0E0,0E0,0E0,7.2635055E-2,3.967682E-1,1.720911E-1,0E0,0E0,3.5817325E-2,2.799625E-2,1.7076847E-1,9.3737505E-2,0E0,0E0,8.244067E-3,4.539394E-2,4.1988254E-2,6.5663025E-2,0E0,0E0,0E0,0E0,1.5532762E-1,1.19956255E-1,0E0,8.892519E-2,4.465156E-2,8.632034E-2,1.4166866E-1,8.521536E-2,3.234395E-2,0E0,4.9363017E-2,4.421051E-2,0E0,0E0,4.861358E-2,9.149605E-2,0E0,0E0,0E0,0E0,7.7995956E-3,7.1874514E-2,1.3424551E-1,2.494174E-2,3.657078E-2,2.1533873E-2,7.661778E-2,0E0,0E0,0E0,6.281823E-2,1.1669968E-1,3.447859E-2,2.734287E-2,0E0,5.558087E-2,3.655598E-2,1.4064517E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,82,82,84,84,90,90,91,91,92,92,93,93,94,94,98,98,99,99,101,101,102,102,104,104,106,106,107,107,108,108,109,109,110,110,111,111,113,113,114,114,115,115,119,119,122,122,123,123,124,124,125,125,126,126,127,127,132,132,137,137,138,138,140,140,142,142,143,143,144,144,147,147,148,148,149,149,150,150,152,152,155,155,160,160,161,161,162,162,163,163,164,164,165,165,166,166,170,170,171,171,172,172,175,175,176,176,177,177,178,178,181,181,182,182,183,183,184,184,189,189,190,190,192,192,193,193,194,194,195,195,196,196,197,197,199,199,200,200,203,203,204,204,209,209,210,210,211,211,212,212,213,213,214,214,215,215,219,219,220,220,221,221,222,222,224,224,225,225,226,226],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,-1,-1,64,-1,66,68,70,-1,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,-1,-1,98,100,102,104,106,108,110,-1,112,114,116,118,120,122,124,126,128,130,132,134,136,138,140,-1,142,144,-1,146,-1,-1,-1,-1,-1,148,150,152,154,156,-1,-1,-1,158,160,-1,162,164,-1,166,-1,168,170,172,174,176,178,-1,180,182,184,-1,-1,-1,186,-1,-1,188,190,192,194,196,198,-1,-1,-1,-1,200,-1,-1,-1,-1,202,204,-1,206,-1,208,210,212,-1,-1,214,216,218,220,-1,222,-1,-1,224,-1,-1,-1,-1,226,228,230,232,234,236,238,-1,-1,-1,240,242,244,-1,-1,246,248,250,252,-1,-1,254,256,258,260,-1,-1,-1,-1,262,264,-1,266,268,270,272,274,276,-1,278,280,-1,-1,282,284,-1,-1,-1,-1,286,288,290,292,294,296,298,-1,-1,-1,300,302,304,306,-1,308,310,312,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,3.7311627E2,1.9121015E0,1.0008265E1,1.5166431E5,2.0354E4,3.4409692E3,3.684685E2,2E0,4.719849E7,1.5106794E3,1.3E1,1.3456146E6,7.6599895E9,6.504065E-2,6.978874E5,3.0812828E6,-1.6307225E-2,1.1994927E7,1.3359244E9,8.360387E-1,6.323775E-7,1.4497429E7,7.33E2,1.8255814E0,-1.629311E-2,1.038946E6,1.16146E5,1.5265896E8,8.9712226E-1,2E0,1E0,1.459995E2,1.5314462E-3,-1.7889583E-2,1.2511433E3,5.0906134E-3,1E0,5.388794E6,1.3715873E3,-1.2139839E-2,2E0,1.057101E6,5.057096E-1,1.5277338E0,1.636099E0,6.028929E5,-9.4726967E-4,2.1631205E0,4.47682E5,6.77603E5,1.9557823E0,1.6394367E0,9.3343524E2,7.7329254E2,8.843958E-3,3.5628323E-3,1.6173E4,6.2105E4,8.58183E5,1.1E1,5E-1,8.916961E2,2.5207965E2,2.417832E-3,3.970405E3,1.6812695E-2,9.70107E7,9.042859E4,1.4813794E3,1.5E1,2.8290488E5,3.3382E4,1.5038776E2,7.1E1,7.576E3,2.4519731E3,2.8721826E7,1.1997242E0,5.4849293E-3,-1.1910879E-2,1.8379223E-3,3.73802E5,-1.4115593E-2,1.8238512E0,-4.7805877E-3,-1.2936423E-2,1.423635E-3,-5.474585E-3,4.74668E-3,1.0918E4,8E0,7.108182E4,7.43361E0,2.6740572E5,-5.3536207E-3,-1.31771425E-2,-1.1151683E-4,3.1570474E2,2.8091298E7,3.7079938E-3,3.64299E5,2.7E1,-1.1779427E-2,3.9238465E4,1.8982225E-3,1.5552E4,7.102631E1,6.6861836E-4,1E0,8.315271E0,1.7812634E1,7.0707677E-3,1.028E3,1.5481343E1,1.1317E4,-6.428619E-3,1.018311E-2,3.1827895E-3,1.0046621E7,-9.524006E-3,-2.2985935E-3,5.8180006E8,4.750825E2,1.4523809E0,1.7E1,1.602114E6,8.890291E2,1.5593255E-3,-1.2112149E-3,3.988895E-3,1.9009879E-2,9.195004E6,2.3598128E-4,3.463902E-3,1.10485125E-2,1.2041442E-3,2.8326238E3,2.3553647E5,-2.4279587E-3,7.469927E6,-1.6132127E-3,1.0894445E-2,1.4283712E6,1.5052E4,-7.837203E-3,-1.1317625E-3,8.595346E-1,2.5275471E2,4.071191E7,1.6504E4,-2.9408112E-3,1.3593E4,-4.698867E-3,-1.1375703E-2,3.5315048E2,3.9530536E-3,9.34538E-3,2.7217208E-3,3.4476996E-3,2.8E1,8.333333E0,2.102858E4,3.1964528E7,1.9719212E1,2.8331464E2,2.165022E2,-7.053112E-4,-1.0591792E-2,9.958291E-4,3.591E3,1.3058E4,1.4733E4,2.64716E-3,-9.661903E-4,1.6410715E1,2.2589723E2,5.961E3,2.4013288E1,-4.387462E-3,1.6688404E-3,1.00729517E3,5.987692E2,6.720137E7,1.0409588E-1,-4.71271E-3,-1.2775903E-3,9.09573E-4,3.748777E-3,2.7487562E0,5.077728E8,-9.404977E-3,3.7342235E6,6.4506575E5,2.558106E10,2.4042394E5,1.5470392E5,7.697189E2,-9.027885E-3,1.8361508E3,6.7995167E0,2.6231098E-3,-7.363205E-3,2.345785E5,3.7891315E6,-1.0620912E-2,-4.1127945E-3,1.015699E-2,3.2278034E-3,9E0,1.3446785E7,7.518072E-1,4.4705097E5,8.294758E-1,3.6E1,2.7056329E10,-1.3177023E-2,5.882675E-3,3.0142965E-4,2.8839298E7,3.1743118E-1,4.4844124E-1,2.0185E4,-7.039509E-3,1.0941897E9,9.609747E3,1E0,-4.6505905E-3,-1.4647991E-4,-1.5389213E-2,-3.6927362E-3,-9.96959E-4,6.488401E-3,3.119722E-4,-4.106424E-3,-8.512408E-3,-2.6247553E-3,-1.2668942E-3,4.766232E-3,3.8102232E-3,9.217613E-3,-1.3626713E-2,1.2506298E-3,-8.864874E-4,2.3884384E-3,-5.982359E-3,-1.495457E-2,-5.166432E-3,-1.7029771E-3,-7.2733883E-4,-8.640457E-3,7.332595E-4,-1.349738E-3,3.655115E-3,8.004898E-3,-4.1117324E-3,1.6309222E-3,3.6683995E-3,9.425404E-3,-6.186816E-4,3.0510675E-3,-1.0085318E-3,4.584779E-3,6.212128E-3,2.7122225E-3,-1.7698518E-3,4.209272E-3,4.2895875E-3,2.2143625E-4,9.013454E-3,3.6556995E-3,2.3483771E-3,-1.2177113E-3,7.000607E-3,-5.033386E-4,-4.2445012E-3,5.7459384E-4,6.8329684E-3,1.3274762E-2,7.779097E-3,1.5369069E-3,-1.3808387E-3,2.6314575E-3,6.5359897E-3,-8.886786E-5,-1.6703968E-3,-5.025818E-3,4.1056206E-3,-6.0913834E-4,-2.575503E-3,-1.2567253E-2,-3.5892932E-3,4.965748E-4,-1.531015E-3,4.5982045E-3,-4.554298E-3,3.322888E-5,-3.150773E-3,-9.702625E-3,1.0097817E-3,-1.2591296E-3,-9.180626E-3,-1.9165645E-3,4.8934403E-3,1.4769853E-3,-2.9286558E-3,1.0626133E-3,-2.0197795E-3,-7.2607426E-3,-2.4964411E-3,2.3653915E-3,2.440373E-3,-5.798527E-4],"split_indices":[27,52,53,54,28,9,52,52,17,45,52,3,51,5,57,28,50,0,9,7,39,37,50,2,54,0,9,1,45,27,8,105,33,0,0,4,0,63,47,52,0,17,9,39,53,53,51,0,54,9,29,53,42,4,52,0,0,2,10,9,3,53,33,4,0,52,38,1,28,4,3,33,9,46,0,9,4,50,41,39,0,38,33,0,53,0,0,0,0,0,9,3,28,56,33,0,0,0,58,45,0,9,10,0,50,0,9,47,41,75,56,56,0,2,56,32,0,0,0,47,0,0,7,55,54,3,29,4,0,0,0,0,45,0,0,0,0,52,33,0,12,0,38,48,9,0,0,27,52,12,9,0,10,0,0,33,0,0,0,0,8,52,28,7,56,4,52,0,0,0,29,29,1,0,0,54,52,10,58,0,0,33,4,12,38,0,0,0,0,54,7,0,45,28,19,28,33,4,0,52,54,0,0,28,28,0,0,0,0,3,45,56,33,27,10,31,0,0,0,45,58,57,10,0,7,52,101,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.552E3,1.896E3,6.56E2,8.47E2,1.049E3,1.57E2,4.99E2,7.8E2,6.7E1,4.36E2,6.13E2,1.29E2,2.8E1,4.23E2,7.6E1,7.71E2,9E0,7E0,6E1,3.82E2,5.4E1,4.63E2,1.5E2,1.03E2,2.6E1,7E0,2.1E1,3.42E2,8.1E1,1.2E1,6.4E1,5.27E2,2.44E2,5E0,4E0,5.4E1,6E0,3E2,8.2E1,5E1,4E0,4.29E2,3.4E1,5.5E1,9.5E1,1.8E1,8.5E1,6E0,2E1,1.2E1,9E0,6.1E1,2.81E2,7.3E1,8E0,5E0,7E0,1.2E1,5.2E1,4.71E2,5.6E1,1.4E1,2.3E2,4.7E1,7E0,2.58E2,4.2E1,7.1E1,1.1E1,2.9E1,2.1E1,2.13E2,2.16E2,2.4E1,1E1,4.3E1,1.2E1,1.8E1,7.7E1,1.4E1,4E0,1.6E1,6.9E1,1.2E1,8E0,7E0,5E0,5E0,4E0,6E0,5.5E1,2.26E2,5.5E1,1.3E1,6E1,4E0,4E0,4E0,8E0,4.5E1,7E0,2.86E2,1.85E2,7E0,4.9E1,5E0,9E0,3.5E1,1.95E2,1.1E1,3.6E1,2.54E2,4E0,1E1,3.2E1,6.5E1,6E0,7E0,4E0,2.5E1,4E0,4E0,1.7E1,1.79E2,3.4E1,6.4E1,1.52E2,1.9E1,5E0,4E0,6E0,4E0,3.9E1,4E0,8E0,1.2E1,6E0,9E0,6.8E1,4E0,1E1,7E0,9E0,4E1,2.9E1,4E0,4E0,1.8E1,3.7E1,1.1E1,2.15E2,5E0,5E1,7E0,6E0,5.6E1,4E0,4E0,4E0,6E0,3.9E1,2.42E2,4.4E1,2.2E1,1.63E2,2.3E1,2.6E1,4E0,5E0,6E0,2.9E1,1.8E1,1.77E2,4E0,7E0,9E0,2.7E1,3.1E1,2.23E2,6E0,4E0,1.8E1,1.4E1,1.6E1,4.9E1,1.9E1,6E0,1.2E1,5E0,4.8E1,1.31E2,5E0,2.9E1,3E1,3.4E1,1.34E2,1.8E1,1.4E1,5E0,2.6E1,1.3E1,4E0,5E0,4.2E1,2.6E1,5E0,5E0,5E0,4E0,9E0,3.1E1,1.5E1,1.4E1,8E0,1E1,3.1E1,6E0,7E0,4E0,1.2E2,9.5E1,3.1E1,1.9E1,6E0,5E1,2.2E1,1.7E1,1.8E1,2.24E2,4E0,4E1,4E0,1.8E1,1.51E2,1.2E1,5E0,1.8E1,2.2E1,4E0,1.4E1,1.5E1,6E0,1.2E1,6E1,1.17E2,4E0,5E0,9E0,1.8E1,2.2E1,9E0,8.9E1,1.34E2,4E0,1.4E1,4E0,1E1,1E1,6E0,2E1,2.9E1,2.4E1,2.4E1,9.5E1,3.6E1,8E0,2.1E1,9E0,2.1E1,1.4E1,2E1,3.8E1,9.6E1,1.3E1,5E0,9E0,5E0,8E0,1.8E1,8E0,5E0,9E0,3.3E1,1.9E1,7E0,5E0,4E0,1.4E1,1.7E1,6E0,9E0,8E0,6E0,4E0,4E0,6E0,4E0,2.6E1,5E0,6.2E1,5.8E1,5E0,9E1,1.5E1,1.6E1,5E0,1.4E1,4.5E1,5E0,1.8E1,4E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"313","size_leaf_vector":"1"}},{"base_weights":[-2.154297E-3,9.715117E-3,-4.184277E-2,-9.796612E-3,3.5082687E-2,-6.841955E-2,-6.9011054E-3,-3.0516598E-2,1.3754953E-2,7.36322E-2,1.6772654E-2,-1.9104668E-1,-6.3272275E-2,-2.0830678E-3,-1.3889802E-1,-1.8933492E-2,-7.052321E-2,1.63277E-2,-8.9053465E-3,1.883333E-1,6.2460426E-2,4.892478E-3,6.354714E-2,-1.2193615E-2,-2.8710042E-3,-1.3580896E-2,-8.040062E-2,-2.4315197E-2,2.6491072E-2,-1.5924168E-3,-1.0900071E-2,-1.2944766E-3,-4.1432064E-2,-1.640244E-1,-5.5406444E-2,9.623645E-3,7.8751095E-2,7.1497604E-2,1.4075334E-2,1.737651E-1,5.3706426E-2,7.505367E-3,-1.2861814E-1,4.835829E-2,1.4524305E-1,-5.4881953E-2,2.5212836E-2,-9.302963E-2,-1.2927251E-2,-1.8168764E-2,-7.922585E-3,6.398237E-2,1.0770397E-2,-4.554515E-2,2.0083942E-2,5.401187E-3,-4.7323853E-2,-8.1248E-2,-1.813757E-2,-4.7010392E-2,-1.1612088E-2,1.2192055E-2,-8.057933E-3,1.0434462E-1,2.2931622E-2,7.5507313E-3,-1.6641922E-3,2.0983328E-1,1.8997894E-3,6.484321E-2,-1.4366299E-2,1.2101129E-2,-5.618791E-2,-1.5091712E-3,-1.0058744E-2,2.753811E-2,9.9100195E-2,2.051284E-1,-2.5725113E-3,-7.3175214E-2,1.485242E-3,5.042354E-2,-5.7585137E-3,3.4197976E-3,-1.0248349E-1,2.124564E-2,-1.0214141E-2,-1.6218334E-3,-4.3739073E-2,7.983615E-3,4.4590775E-2,-4.9850927E-3,6.7598045E-2,1.8072516E-2,-6.29062E-2,6.186737E-2,-1.0836069E-2,-6.337353E-2,-2.8242858E-3,9.072641E-5,-1.2759627E-1,-5.8591027E-2,7.6971166E-2,2.9995387E-2,-1.6643794E-2,3.4192298E-2,1.3640912E-1,-2.235816E-3,2.358305E-3,7.3090284E-3,1.3500686E-2,-6.1760895E-2,7.527311E-2,7.723594E-2,-7.475035E-2,-4.9376577E-2,1.6859189E-2,-7.915795E-2,2.6339216E-3,5.288233E-2,2.0929923E-5,1.17240086E-1,-5.7058415E-4,6.691553E-3,1.53157255E-2,-2.3205018E-2,-1.0300414E-1,8.044542E-3,2.879108E-2,-9.37267E-2,8.2592405E-2,-9.3602106E-2,-1.4299148E-2,8.962198E-2,-3.2402345E-3,-3.1392537E-2,3.6238838E-2,-7.684328E-2,-3.5886336E-3,6.1675582E-2,-7.732061E-4,8.441107E-3,-7.564544E-2,1.2781538E-1,-1.3018482E-4,-3.6972451E-3,6.6256724E-2,-8.622985E-2,1.0765126E-2,1.2743312E-1,3.0416502E-2,6.248782E-2,-1.860384E-2,-7.1987115E-2,4.5997235E-3,2.1425093E-2,-9.736055E-2,-3.3777573E-3,-8.098251E-3,-4.205397E-2,-1.698434E-1,6.1059557E-3,3.6483884E-4,3.7023977E-3,5.3150374E-2,-6.8211425E-3,-1.030316E-1,3.6379613E-3,-1.7203978E-3,2.036224E-3,8.030475E-3,-8.705488E-3,5.198289E-3,1.2814096E-1,5.724451E-2,1.4916083E-1,-4.1003167E-3,-1.2684548E-1,8.803759E-3,-5.568679E-3,-2.6628787E-2,-6.2768594E-3,3.4676135E-2,-9.5362745E-2,-1.0767935E-3,-1.5839018E-2,8.5209064E-2,-2.7935593E-3,2.68662E-2,8.969431E-3,7.737809E-2,-4.5348364E-3,6.4414967E-4,-1.3957001E-1,-5.7074256E-2,-2.6534768E-2,8.923321E-2,-1.8378848E-3,-6.591895E-3,1.9589435E-4,7.3700286E-3,-7.582178E-2,-1.66154E-1,6.5910416E-3,1.4695596E-3,2.448376E-3,-2.5065662E-2,-5.5949176E-3,-8.605446E-2,1.7349089E-2,5.123859E-3,-2.9996097E-2,-5.1868227E-3,-3.148511E-3,2.3021223E-2,5.3274822E-3,4.57644E-2,-2.1977413E-3,1.737504E-2,-6.7556323E-3,-6.6989055E-4,2.9571976E-3,8.538518E-3,1.366454E-3,-1.3781628E-3,1.939892E-4,5.2342783E-3,-2.774979E-3,-9.061698E-3,2.9484853E-3,-1.4443627E-3,-8.881314E-5,8.524416E-3,3.1371773E-3,-1.2248335E-3,6.835829E-3,-4.926775E-4,-5.3039184E-3,-2.7808745E-4,-3.0804381E-3,-9.233397E-3,3.286145E-3,-9.989481E-4,-7.046169E-3,-8.3788414E-4,-2.5861E-3,2.603145E-3,-1.4476953E-2,-4.2155394E-3,1.680387E-3,-3.2844448E-3,3.739786E-3,3.6951134E-4,-2.7375815E-3,6.802756E-4,-1.983018E-3,-1.2980874E-2,2.6484504E-3,-3.1364E-3,6.8753264E-3,1.0582873E-3,4.9057133E-3,1.5108632E-3,9.349321E-3,3.6869547E-3,-7.406942E-4,-8.7434575E-3,-2.096811E-3,2.888621E-3,7.6712057E-4,-2.6402962E-3,1.6114682E-5,-6.360812E-3,1.4751805E-3,9.540277E-3,-6.0617137E-3,-2.062112E-3,2.50174E-3,-3.9715255E-3,5.850148E-3,2.6997442E-3,2.85762E-3,-1.6687973E-3,2.663322E-3,6.220868E-3,-8.643075E-3,-3.2421635E-3,-4.5748223E-3,1.3076821E-4,8.5328834E-4,-2.4865323E-3,6.1053643E-3,2.0488375E-3,-1.0056826E-2,-3.2961825E-3,-4.3593245E-3,-1.0844181E-2,-3.2881044E-3,2.0902744E-4,-1.3720581E-3,2.0301703E-3,-1.0627246E-3,-6.33986E-3,-1.1469375E-3,2.7006466E-3,-2.7844086E-3,-1.8274409E-4,2.0495611E-3,-5.5563496E-4,1.2820917E-3,3.6822003E-3,2.3111824E-3,-1.4201838E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,-1,-1,45,47,49,51,-1,-1,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,95,-1,97,99,-1,101,-1,103,-1,105,107,-1,-1,109,-1,111,113,115,117,-1,-1,119,121,123,-1,125,-1,127,-1,129,131,133,-1,135,137,-1,139,141,143,145,147,149,151,153,155,-1,157,159,161,163,165,167,169,-1,-1,-1,-1,171,173,175,177,179,181,183,-1,185,187,189,-1,-1,-1,191,193,-1,195,197,199,201,-1,203,205,207,209,211,213,215,-1,217,219,221,223,-1,225,227,229,231,233,235,237,239,-1,241,243,-1,-1,245,247,-1,-1,249,251,253,255,-1,-1,-1,-1,-1,257,259,261,263,-1,265,267,-1,269,271,273,275,-1,277,279,-1,281,-1,283,-1,-1,285,287,289,291,-1,-1,-1,-1,293,295,-1,-1,-1,297,299,301,303,-1,305,-1,-1,307,-1,309,-1,311,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2073107E0,9.7691E-1,5.474237E-1,5.4501444E-1,6.043781E-1,1.99363E-1,1.6251798E-1,2.7392483E-1,2.6122898E-1,3.4190702E-1,3.2365584E-1,7.800016E-2,2.7382338E-1,1.5818961E-1,6.938465E-2,1.832923E-1,1.7889893E-1,2.1560138E-1,0E0,2.4164963E-1,2.3718107E-1,1.6352867E-1,1.3939852E-1,0E0,0E0,1.3597038E-1,2.033068E-1,1.1263474E-1,6.355424E-2,0E0,0E0,2.4687716E-1,1.8395582E-1,2.669664E-1,1.6145316E-1,2.0737839E-1,6.903803E-2,1.0516383E-1,0E0,8.0150604E-2,1.7984569E-1,1.3474531E-1,5.8304384E-2,1.03973016E-1,2.226207E-1,6.563207E-2,1.616193E-1,1.8453777E-1,2.5955236E-1,5.704893E-2,0E0,5.2320004E-2,7.017408E-2,9.501679E-2,2.2805253E-1,0E0,1.3980496E-1,5.4129727E-2,0E0,1.6531771E-1,0E0,2.3767814E-1,0E0,7.181543E-2,3.0030414E-2,0E0,0E0,1.41837E-2,0E0,2.7270406E-1,1.9383809E-1,1.2602869E-1,8.17183E-2,0E0,0E0,5.09092E-2,5.9478343E-2,6.0028374E-2,0E0,4.8288748E-2,0E0,8.626075E-2,0E0,1.538766E-1,2.7339554E-1,5.721505E-2,0E0,9.4690144E-2,7.016909E-2,0E0,2.9688071E-2,5.9457928E-2,6.934966E-2,8.918973E-2,1.1605331E-1,1.5078142E-1,5.9257973E-2,1.9941461E-1,1.2358558E-1,0E0,7.660091E-3,1.8235552E-1,3.0618843E-2,1.7392397E-1,1.4949186E-1,3.430584E-2,4.6930045E-2,0E0,0E0,0E0,0E0,1.2139473E-1,1.7185378E-1,1.7264417E-1,9.252471E-2,4.080382E-2,1.6427875E-1,2.2216469E-2,0E0,8.576804E-2,5.5553876E-2,4.8163325E-2,0E0,0E0,0E0,3.5307683E-2,2.730295E-2,0E0,1.1272018E-1,1.4724553E-2,5.5591233E-2,2.1573114E-1,0E0,1.9035697E-2,3.0872634E-2,6.579546E-2,4.456405E-2,3.351648E-2,4.0981952E-2,1.0853432E-2,0E0,2.5587505E-2,3.5255082E-2,1.6489625E-2,7.54296E-3,0E0,3.0810587E-2,1.3883266E-1,3.424232E-2,1.3417241E-1,9.399547E-2,5.5117913E-2,1.0556398E-1,1.4836115E-1,0E0,8.0284975E-2,3.6337458E-2,0E0,0E0,8.582574E-2,1.08775675E-1,0E0,0E0,2.827457E-1,1.5033248E-1,1.5758447E-1,1.6842738E-1,0E0,0E0,0E0,0E0,0E0,3.906924E-2,4.4629216E-2,1.5401813E-1,1.4797524E-2,0E0,6.7511976E-2,2.4817951E-2,0E0,2.7221635E-2,1.3901201E-1,1.7552549E-1,2.3070201E-2,0E0,5.850978E-2,1.9510433E-2,0E0,4.7597002E-2,0E0,1.2650751E-2,0E0,0E0,1.622042E-2,2.2881474E-2,1.9084973E-2,2.0420432E-2,0E0,0E0,0E0,0E0,1.647346E-1,1.1755705E-1,0E0,0E0,0E0,2.291593E-2,3.446872E-2,3.6976866E-2,4.5614254E-2,0E0,7.928309E-3,0E0,0E0,1.15162935E-2,0E0,8.2164705E-3,0E0,2.6989233E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,56,56,57,57,59,59,61,61,63,63,64,64,67,67,69,69,70,70,71,71,72,72,75,75,76,76,77,77,79,79,81,81,83,83,84,84,85,85,87,87,88,88,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,100,100,101,101,102,102,103,103,104,104,105,105,106,106,111,111,112,112,113,113,114,114,115,115,116,116,117,117,119,119,120,120,121,121,125,125,126,126,128,128,129,129,130,130,131,131,133,133,134,134,135,135,136,136,137,137,138,138,139,139,141,141,142,142,143,143,144,144,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,155,155,156,156,159,159,160,160,163,163,164,164,165,165,166,166,172,172,173,173,174,174,175,175,177,177,178,178,180,180,181,181,182,182,183,183,185,185,186,186,188,188,190,190,193,193,194,194,195,195,196,196,201,201,202,202,206,206,207,207,208,208,209,209,211,211,214,214,216,216,218,218],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,-1,-1,46,48,50,52,-1,-1,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,96,-1,98,100,-1,102,-1,104,-1,106,108,-1,-1,110,-1,112,114,116,118,-1,-1,120,122,124,-1,126,-1,128,-1,130,132,134,-1,136,138,-1,140,142,144,146,148,150,152,154,156,-1,158,160,162,164,166,168,170,-1,-1,-1,-1,172,174,176,178,180,182,184,-1,186,188,190,-1,-1,-1,192,194,-1,196,198,200,202,-1,204,206,208,210,212,214,216,-1,218,220,222,224,-1,226,228,230,232,234,236,238,240,-1,242,244,-1,-1,246,248,-1,-1,250,252,254,256,-1,-1,-1,-1,-1,258,260,262,264,-1,266,268,-1,270,272,274,276,-1,278,280,-1,282,-1,284,-1,-1,286,288,290,292,-1,-1,-1,-1,294,296,-1,-1,-1,298,300,302,304,-1,306,-1,-1,308,-1,310,-1,312,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.386E3,1E0,4.459525E5,6.4E1,4.36E2,1.4299594E8,7.2488395E6,1.4955851E7,5.857681E6,1.4847565E0,8E0,3.6628513E10,1.3097832E3,2.298146E6,9.403226E0,5.4E1,2.539E3,-8.9053465E-3,2.5332516E5,1.0918E4,3.869321E2,1.1458888E4,-1.2193615E-2,-2.8710042E-3,1.06E3,1.0409E4,4.7687202E1,2.99665E5,-1.5924168E-3,-1.0900071E-2,3.7761906E1,7E0,3.7832818E0,3.0574578E-1,2.507E3,2.909019E3,3.37E2,1.4075334E-2,1.0872E4,2E0,7.1277297E-1,3.791269E7,2.6100147E5,4.29E4,1.2526851E6,4.7022E4,7.518072E-1,2.2813266E9,7.27E2,-7.922585E-3,5.695737E0,5.528107E3,2.645E3,1.543E3,5.401187E-3,1.2375429E8,7.948417E0,-1.813757E-2,2.0757051E-1,-1.1612088E-2,3.77451E0,-8.057933E-3,1E0,3.07E2,7.5507313E-3,-1.6641922E-3,2.5002517E8,1.8997894E-3,9.4168E4,1.7321888E1,9.996076E2,4.4839956E3,-1.5091712E-3,-1.0058744E-2,8.4342186E1,1.4274633E7,4.43575E7,-2.5725113E-3,3.788185E6,1.485242E-3,5.348301E6,-5.7585137E-3,7.2591E4,4.7790206E5,9.11898E5,-1.0214141E-2,5.500988E2,4.2648572E5,7.983615E-3,2.3058404E7,1.5961998E7,4.5E1,8.333333E0,1.4925373E-1,2.4626505E0,1.517647E0,5.141184E2,4.152361E8,9.072641E-5,1.2224265E1,1.0074342E-1,1.9563605E5,8.15E2,1E0,7.2041174E2,1.5834932E6,-2.235816E-3,2.358305E-3,7.3090284E-3,1.3500686E-2,6.072483E2,1.6954315E0,1E0,4.6E1,2.2746666E2,7.805608E4,1.8496495E5,2.6339216E-3,3.8935616E8,1.6052323E5,2.8524646E-1,-5.7058415E-4,6.691553E-3,1.53157255E-2,4.2371875E4,2.7999267E8,8.044542E-3,1E0,1.9836358E5,3.3333334E-1,9.18292E-1,-1.4299148E-2,8.7022424E-1,5.3084288E8,8.601996E-1,1E0,7.9240966E0,3.2785368E7,6.554713E4,-7.732061E-4,5.012E3,7.808427E9,3.7E1,1.2424786E2,-3.6972451E-3,1.3E1,2.0223091E1,3.565157E7,1.279012E7,2.6304092E0,5.61E2,2.402216E6,7.629372E4,4.5997235E-3,2.3449652E0,1.9358906E6,-3.3777573E-3,-8.098251E-3,7.6588124E-2,1.9051096E7,6.1059557E-3,3.6483884E-4,3.7934E4,1.216E3,2.0008473E2,4.949877E6,3.6379613E-3,-1.7203978E-3,2.036224E-3,8.030475E-3,-8.705488E-3,9.330579E0,5.99999E9,1.1323028E0,1.0201212E3,-4.1003167E-3,7E0,1.4543829E7,-5.568679E-3,9.87156E0,1.4930757E0,8.791777E3,1.5401015E1,-1.0767935E-3,2.6E1,1.7227725E0,-2.7935593E-3,2.4536058E1,8.969431E-3,7.1277297E-1,-4.5348364E-3,6.4414967E-4,1.440286E7,1.1905188E6,6.472E1,3.2801925E6,-1.8378848E-3,-6.591895E-3,1.9589435E-4,7.3700286E-3,6.9121815E-2,2.579E3,6.5910416E-3,1.4695596E-3,2.448376E-3,3.4540886E3,7.1974045E-1,7.1117523E2,3.424835E6,5.123859E-3,2.3421426E-1,-5.1868227E-3,-3.148511E-3,6.6992125E4,5.3274822E-3,3.596463E-1,-2.1977413E-3,5.4379158E1,-6.7556323E-3,-6.6989055E-4,2.9571976E-3,8.538518E-3,1.366454E-3,-1.3781628E-3,1.939892E-4,5.2342783E-3,-2.774979E-3,-9.061698E-3,2.9484853E-3,-1.4443627E-3,-8.881314E-5,8.524416E-3,3.1371773E-3,-1.2248335E-3,6.835829E-3,-4.926775E-4,-5.3039184E-3,-2.7808745E-4,-3.0804381E-3,-9.233397E-3,3.286145E-3,-9.989481E-4,-7.046169E-3,-8.3788414E-4,-2.5861E-3,2.603145E-3,-1.4476953E-2,-4.2155394E-3,1.680387E-3,-3.2844448E-3,3.739786E-3,3.6951134E-4,-2.7375815E-3,6.802756E-4,-1.983018E-3,-1.2980874E-2,2.6484504E-3,-3.1364E-3,6.8753264E-3,1.0582873E-3,4.9057133E-3,1.5108632E-3,9.349321E-3,3.6869547E-3,-7.406942E-4,-8.7434575E-3,-2.096811E-3,2.888621E-3,7.6712057E-4,-2.6402962E-3,1.6114682E-5,-6.360812E-3,1.4751805E-3,9.540277E-3,-6.0617137E-3,-2.062112E-3,2.50174E-3,-3.9715255E-3,5.850148E-3,2.6997442E-3,2.85762E-3,-1.6687973E-3,2.663322E-3,6.220868E-3,-8.643075E-3,-3.2421635E-3,-4.5748223E-3,1.3076821E-4,8.5328834E-4,-2.4865323E-3,6.1053643E-3,2.0488375E-3,-1.0056826E-2,-3.2961825E-3,-4.3593245E-3,-1.0844181E-2,-3.2881044E-3,2.0902744E-4,-1.3720581E-3,2.0301703E-3,-1.0627246E-3,-6.33986E-3,-1.1469375E-3,2.7006466E-3,-2.7844086E-3,-1.8274409E-4,2.0495611E-3,-5.5563496E-4,1.2820917E-3,3.6822003E-3,2.3111824E-3,-1.4201838E-4],"split_indices":[19,2,59,28,11,2,45,45,47,12,39,3,31,4,29,58,0,2,0,28,9,57,52,0,0,2,2,56,29,0,0,52,3,53,39,2,32,0,0,2,6,27,45,28,10,28,9,56,7,0,0,54,52,9,29,0,7,58,0,41,0,54,0,102,11,0,0,12,0,1,56,55,52,0,0,58,45,1,0,12,0,12,0,29,33,29,0,4,28,0,45,43,3,52,56,58,54,52,7,0,56,42,28,2,74,52,45,0,0,0,0,4,54,104,6,52,28,33,0,7,33,38,0,0,0,32,7,0,8,32,56,27,0,27,7,27,74,54,12,33,0,2,12,3,56,0,5,52,7,7,53,2,7,33,0,53,45,0,0,42,45,0,0,1,10,52,9,0,0,0,0,0,56,5,42,4,0,8,47,0,58,42,52,54,0,3,39,0,54,0,27,0,0,12,48,52,45,0,0,0,0,58,2,0,0,0,4,27,33,32,0,38,0,0,33,0,38,0,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.561E3,1.972E3,5.89E2,1.115E3,8.57E2,3.34E2,2.55E2,5.93E2,5.22E2,2.75E2,5.82E2,1.2E1,3.22E2,2.47E2,8E0,4.61E2,1.32E2,5.16E2,6E0,2.3E1,2.52E2,4.65E2,1.17E2,8E0,4E0,8.3E1,2.39E2,1.39E2,1.08E2,4E0,4E0,2.59E2,2.02E2,1.7E1,1.15E2,4.67E2,4.9E1,1.1E1,1.2E1,1.7E1,2.35E2,4.57E2,8E0,1E2,1.7E1,4E1,4.3E1,2.01E2,3.8E1,1.34E2,5E0,3.1E1,7.7E1,8.4E1,1.75E2,7E0,1.95E2,1.3E1,4E0,1.11E2,4E0,4.61E2,6E0,3.3E1,1.6E1,6E0,5E0,1.3E1,4E0,2.02E2,3.3E1,4.27E2,3E1,4E0,4E0,7.2E1,2.8E1,1.3E1,4E0,3.3E1,7E0,3.7E1,6E0,1.8E1,1.83E2,3.3E1,5E0,8.2E1,5.2E1,4E0,2.7E1,6.1E1,1.6E1,1.8E1,6.6E1,7.4E1,1.01E2,1.43E2,5.2E1,5E0,8E0,1.02E2,9E0,2.85E2,1.76E2,1.1E1,2.2E1,4E0,1.2E1,8E0,5E0,1.5E1,1.87E2,1.3E1,2E1,3E1,3.97E2,2.5E1,5E0,3.7E1,3.5E1,2.4E1,4E0,9E0,4E0,1.3E1,2E1,5E0,3.2E1,8E0,1E1,1.76E2,7E0,8E0,2.5E1,4.6E1,3.6E1,2.8E1,2.4E1,2.1E1,6E0,5.2E1,9E0,8E0,8E0,6E0,1.2E1,5E1,1.6E1,2.3E1,5.1E1,9E0,9.2E1,1.36E2,7E0,4.2E1,1E1,4E0,4E0,9E1,1.2E1,5E0,4E0,1.34E2,1.51E2,1.59E2,1.7E1,7E0,4E0,5E0,1.7E1,5E0,1E1,4.6E1,1.41E2,9E0,4E0,1.2E1,8E0,7E0,2.3E1,1.73E2,2.24E2,1.9E1,6E0,1.2E1,2.5E1,1.1E1,2.4E1,8E0,1.6E1,4E0,9E0,1E1,1E1,1.7E1,1.5E1,4E0,4E0,5E0,5E0,1.43E2,3.3E1,4E0,4E0,7E0,1.8E1,3.2E1,1.4E1,2.9E1,7E0,1.1E1,1.7E1,7E0,1.7E1,4E0,1.7E1,7E0,4.5E1,4E0,5E0,4E0,4E0,4E0,4E0,5E0,7E0,3.9E1,1.1E1,7E0,9E0,6E0,1.7E1,3.2E1,1.9E1,4E0,5E0,1.1E1,8.1E1,1.26E2,1E1,2E1,2.2E1,6E0,4E0,8.2E1,8E0,4E0,8E0,9.4E1,4E1,1.02E2,4.9E1,4.7E1,1.12E2,1.3E1,4E0,6E0,4E0,4.2E1,4E0,5.5E1,8.6E1,5E0,4E0,4E0,8E0,4E0,4E0,9E0,1.4E1,1.65E2,8E0,2.18E2,6E0,1.2E1,7E0,6E0,6E0,1.1E1,1.4E1,1.6E1,8E0,1.2E1,4E0,6E0,4E0,6E0,4E0,6E0,1.1E1,8E0,7E0,9E0,1.34E2,1.4E1,1.9E1,7E0,1.1E1,2.2E1,1E1,6E0,8E0,1.4E1,1.5E1,5E0,6E0,1.1E1,6E0,1.1E1,6E0,1.8E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"313","size_leaf_vector":"1"}},{"base_weights":[-3.4638303E-3,-1.2002759E-2,4.7732543E-2,-4.481376E-3,-7.438575E-2,2.5442898E-1,4.000716E-2,-3.278287E-2,7.680595E-3,-9.622138E-2,-2.5283143E-2,1.5619021E-2,4.9394774E-3,7.4430294E-2,5.735504E-4,-4.2160146E-2,2.6548179E-2,4.8692273E-3,1.0891019E-1,-1.09671436E-1,-3.029107E-2,4.0390436E-2,-4.7113765E-2,3.3340376E-2,9.832066E-2,2.0039205E-2,-8.15651E-2,-4.952439E-2,3.3039194E-2,6.52259E-2,-1.6777212E-2,1.434624E-2,-1.7240189E-2,8.256709E-3,1.4588344E-1,-9.28457E-2,-1.9187424E-1,1.7700725E-3,-4.5457974E-2,7.685374E-2,3.440189E-4,-5.8564235E-2,1.444515E-3,1.1851153E-1,6.408559E-3,1.0531696E-1,-2.5480865E-3,-1.0802188E-2,5.763044E-2,-1.6872151E-1,-1.305169E-2,-3.2916524E-2,-8.407074E-2,5.8016624E-2,-9.16284E-3,4.027019E-2,1.4355272E-1,-4.5207418E-3,-3.2782455E-3,1.2130114E-2,1.8379237E-1,-2.1675836E-1,-1.1056962E-2,-3.035986E-3,2.8168706E-3,9.952486E-3,8.216415E-2,-1.2213405E-1,-5.4271195E-2,-2.4207928E-3,-2.3120849E-1,-1.0161484E-2,-7.990768E-2,5.7134205E-3,1.1988052E-3,-2.9918732E-2,-9.236013E-2,4.1466832E-2,1.4746434E-2,2.7434211E-2,-8.488002E-3,9.505487E-2,1.0120106E-2,4.513084E-3,-6.755932E-2,3.0636404E-2,1.3836971E-1,-1.6845757E-2,-6.764016E-2,-5.421938E-2,2.9421167E-2,-2.5301548E-2,-1.331841E-1,-2.40122E-1,-6.876105E-2,1.1186457E-1,2.3957966E-2,1.9669471E-2,5.2347006E-3,3.1116593E-3,9.361702E-3,-1.8243857E-2,2.8507244E-3,1.14341654E-1,9.043431E-3,1.1273832E-2,4.00225E-3,-1.5752573E-2,-3.2833826E-3,-2.2078523E-2,3.0610347E-2,1.9356213E-3,5.998799E-3,-6.370649E-2,-1.6043432E-1,-6.684695E-2,2.5576996E-3,-2.5611588E-1,-6.0063717E-3,9.027759E-4,-1.8457131E-3,-4.4319927E-4,-5.7120807E-3,-4.271749E-2,1.8737023E-3,-1.1787916E-3,-1.063314E-1,4.479014E-3,-6.310894E-4,4.2493816E-2,-2.8451239E-3,1.4189492E-1,7.062591E-2,2.8043635E-2,-2.9551432E-2,-6.688979E-3,-3.4300417E-2,1.1464992E-2,7.981186E-3,2.3592673E-3,8.355615E-3,-9.996957E-4,-4.799843E-3,-4.6176114E-3,1.2281022E-4,2.7950322E-3,-4.138835E-4,-7.239366E-2,-8.552031E-3,-6.72668E-2,-1.4877698E-2,-1.7878741E-2,-6.0878512E-3,-8.155038E-2,1.0773513E-2,4.3066381E-4,7.2732233E-3,-4.4377907E-3,4.7432184E-2,3.0348284E-2,-1.6682349E-3,-3.4238442E-3,-4.2836997E-3,1.7210165E-1,7.590046E-2,1.372005E-2,-5.0118197E-2,-1.1400531E-2,-6.047664E-2,5.05382E-2,-2.2706244E-2,4.3062535E-3,-9.352121E-2,-9.465334E-2,-2.0397261E-1,-1.242091E-1,-4.9536992E-2,-6.74639E-3,-1.5735751E-2,-2.4358697E-2,-4.6835793E-3,-6.1927224E-3,-2.521512E-3,2.5211353E-2,6.384412E-3,8.3563976E-2,2.1409857E-1,1.9748908E-2,7.864265E-2,-2.8583305E-2,5.7548076E-2,-4.872505E-2,1.3347535E-3,8.07282E-5,-3.0072713E-3,-4.4761016E-3,2.4458906E-2,-2.9825121E-3,-1.2161283E-2,4.2161504E-3,-9.176472E-4,-5.631594E-3,-1.02839884E-4,-6.2596984E-3,-2.7095315E-3,-1.0191204E-3,5.4144515E-3,3.0652676E-3,-7.458873E-4,-3.481034E-4,2.0536599E-3,1.1007979E-3,-1.0075902E-3,1.0768778E-2,4.276157E-3,5.017865E-3,5.0543854E-4,1.7375402E-4,1.9274506E-3,-7.677217E-3,-1.330058E-3,2.8981903E-4,-2.5363967E-3,-2.2974599E-3,-8.9023765E-3,-1.2856742E-3,3.313119E-3,-5.0770915E-3,5.555805E-4,-3.2162664E-3,-9.273725E-3,-1.6209612E-3,-5.889661E-3,-1.1563351E-2,-5.6009535E-3,-3.313627E-3,-8.436277E-3,-4.792313E-4,-3.0092846E-3,5.482493E-4,-1.7244387E-3,-1.1226097E-3,2.0655717E-3,6.176549E-3,1.5149925E-4,1.278304E-2,6.4885025E-3,2.7387517E-3,-4.0558574E-4,1.317985E-3,4.415732E-3,-3.0578708E-3,1.7908147E-3,6.6751393E-4,6.48127E-3,-3.0354224E-3,-1.4532081E-4,2.1945885E-3,-3.397985E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,-1,77,79,81,-1,83,85,87,89,91,93,95,-1,97,99,-1,101,103,105,107,109,-1,-1,-1,111,113,115,-1,117,119,121,-1,-1,123,125,127,-1,129,-1,131,-1,133,135,137,139,-1,141,143,145,147,149,151,153,155,157,159,-1,-1,-1,161,-1,163,165,-1,-1,-1,-1,167,169,-1,-1,171,173,175,-1,177,-1,-1,-1,-1,-1,179,-1,-1,181,-1,-1,183,-1,185,187,189,191,-1,193,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,197,199,201,-1,-1,-1,203,205,-1,-1,-1,207,209,-1,211,-1,213,215,217,219,221,223,225,227,-1,229,231,233,235,237,-1,-1,239,-1,-1,-1,241,-1,243,245,247,249,251,253,255,-1,-1,-1,-1,257,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1192709E0,1.0291053E0,5.7425326E-1,6.7454344E-1,2.506032E-1,8.672464E-2,4.80487E-1,3.2881743E-1,3.8964745E-1,1.4122093E-1,1.0799638E-1,0E0,0E0,1.8225431E-1,2.67049E-1,2.835135E-1,1.3682824E-1,2.7997148E-1,1.3649052E-1,1.6890669E-1,3.051496E-2,2.2683546E-2,5.0441228E-2,1.621013E-1,1.2968552E-1,1.5713693E-1,1.8888214E-1,2.634486E-1,2.5929534E-1,7.8271255E-2,3.850239E-2,3.48876E-1,4.923033E-1,3.99561E-2,7.533115E-2,1.2216699E-1,1.12912655E-1,0E0,2.8370906E-2,1.5093103E-2,0E0,4.467362E-2,0E0,2.190839E-1,2.0851459E-1,9.4483495E-2,0E0,6.5498725E-2,1.2870906E-1,2.1553126E-1,3.4816876E-2,2.3714387E-1,3.4185338E-1,7.434748E-2,0E0,4.311228E-2,2.3937121E-2,0E0,3.173901E-2,2.9048303E-1,3.3983737E-2,1.5782708E-1,1.7977896E-1,0E0,0E0,0E0,1.8386073E-2,1.3299793E-1,7.105951E-2,0E0,1.6213536E-2,1.0494104E-2,2.831906E-2,0E0,0E0,2.5762046E-2,1.8628731E-2,3.501053E-2,0E0,6.5952055E-2,0E0,1.10125184E-1,0E0,4.8922136E-2,3.0694343E-2,1.1448948E-1,3.155619E-2,0E0,1.1615705E-2,2.1874696E-2,1.0461323E-2,2.312793E-1,2.1862498E-1,1.3737726E-1,1.4125139E-1,5.441034E-2,7.568865E-2,1.6310886E-2,0E0,0E0,0E0,2.7711019E-2,0E0,4.688719E-2,2.4914303E-1,0E0,0E0,0E0,0E0,1.2600513E-1,8.8271335E-2,0E0,0E0,1.2872016E-1,8.9524984E-2,3.952214E-2,0E0,5.0061226E-2,0E0,0E0,0E0,0E0,0E0,2.019523E-2,0E0,0E0,1.1450186E-2,0E0,0E0,6.0153835E-2,0E0,1.2951887E-1,2.740553E-2,6.1791778E-2,2.8271345E-2,0E0,1.125354E-2,5.733676E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5273526E-1,2.0035458E-1,5.0140634E-2,0E0,0E0,0E0,1.3538563E-1,6.3045405E-2,0E0,0E0,0E0,2.0872388E-2,9.514457E-3,0E0,1.0671707E-2,0E0,1.9121379E-2,2.8023481E-2,2.1192919E-1,1.5647805E-1,1.6481613E-1,1.0611813E-1,7.386427E-2,6.2515095E-2,0E0,4.9792677E-2,2.0732269E-2,2.9542089E-2,1.4639959E-2,1.52205005E-2,0E0,0E0,7.194328E-3,0E0,0E0,0E0,2.9379625E-2,0E0,6.735562E-2,1.9513369E-2,1.1192545E-2,2.8692901E-2,2.9508514E-2,7.479926E-2,1.0290328E-2,0E0,0E0,0E0,0E0,7.136962E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,58,58,59,59,60,60,61,61,62,62,66,66,67,67,68,68,70,70,71,71,72,72,75,75,76,76,77,77,79,79,81,81,83,83,84,84,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,101,101,103,103,104,104,109,109,110,110,113,113,114,114,115,115,117,117,123,123,126,126,129,129,131,131,132,132,133,133,134,134,136,136,137,137,147,147,148,148,149,149,153,153,154,154,158,158,159,159,161,161,163,163,164,164,165,165,166,166,167,167,168,168,169,169,170,170,172,172,173,173,174,174,175,175,176,176,179,179,183,183,185,185,186,186,187,187,188,188,189,189,190,190,191,191,196,196],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,-1,78,80,82,-1,84,86,88,90,92,94,96,-1,98,100,-1,102,104,106,108,110,-1,-1,-1,112,114,116,-1,118,120,122,-1,-1,124,126,128,-1,130,-1,132,-1,134,136,138,140,-1,142,144,146,148,150,152,154,156,158,160,-1,-1,-1,162,-1,164,166,-1,-1,-1,-1,168,170,-1,-1,172,174,176,-1,178,-1,-1,-1,-1,-1,180,-1,-1,182,-1,-1,184,-1,186,188,190,192,-1,194,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,198,200,202,-1,-1,-1,204,206,-1,-1,-1,208,210,-1,212,-1,214,216,218,220,222,224,226,228,-1,230,232,234,236,238,-1,-1,240,-1,-1,-1,242,-1,244,246,248,250,252,254,256,-1,-1,-1,-1,258,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,4.0844156E7,7.969174E7,1.8122449E2,1.3715873E3,5.8E1,2.545455E7,4.264897E6,6.057671E6,2.6476662E6,1E0,1.5619021E-2,4.9394774E-3,1.6649964E5,2.3580047E10,2.5247778E6,2.15E2,9.669789E0,4.9766545E6,8.988928E5,4.020325E0,2.37E3,3.4409692E3,9.71709E5,4.5481584E7,8.015419E3,7.025225E-2,3.84E2,1E0,1.86E2,1.4912975E-1,1.5137865E2,4.1724915E2,1.4956522E1,7.0988875E6,8.690019E-4,1.5953778E0,1.7700725E-3,6.1061732E7,2.39E2,3.440189E-4,3.6E1,1.444515E-3,4.894913E6,4.92E2,1.8114515E1,-2.5480865E-3,5.31972E7,1.10018056E8,1.11696E5,3.11E3,1E0,7E2,6.429309E2,-9.16284E-3,2.686063E6,6.763314E7,-4.5207418E-3,1.322E3,4.135203E5,5.911314E1,3.9595376E7,6.28205E8,-3.035986E-3,2.8168706E-3,9.952486E-3,1.3E1,6.5029144E5,8.63558E-1,-2.4207928E-3,1.7243155E12,7.217221E7,1.8159722E-1,5.7134205E-3,1.1988052E-3,6.408734E-2,3.956E3,3.80458E5,1.4746434E-2,1.428111E5,-8.488002E-3,7.704183E-1,1.0120106E-2,5.7719237E-1,6.976043E9,1.1355622E7,2E0,-1.6845757E-2,8.498798E-1,2.2153166E7,5.5064187E0,4.7E2,5.5E1,4.076E4,7.830236E-1,6E2,1.198E5,3.05713E4,5.2347006E-3,3.1116593E-3,9.361702E-3,6.3251557E0,2.8507244E-3,1.82407E5,7.224059E-1,1.1273832E-2,4.00225E-3,-1.5752573E-2,-3.2833826E-3,4.387561E1,6.564E3,1.9356213E-3,5.998799E-3,1.1508651E1,2.1862803E8,1.2939E4,2.5576996E-3,1.97729E5,-6.0063717E-3,9.027759E-4,-1.8457131E-3,-4.4319927E-4,-5.7120807E-3,9.2057055E-1,1.8737023E-3,-1.1787916E-3,1E0,4.479014E-3,-6.310894E-4,7.466243E0,-2.8451239E-3,8.422367E6,1E0,9.173799E4,1.1693485E0,-6.688979E-3,3.71E0,1.70752E4,7.981186E-3,2.3592673E-3,8.355615E-3,-9.996957E-4,-4.799843E-3,-4.6176114E-3,1.2281022E-4,2.7950322E-3,-4.138835E-4,1.56E2,9.2058825E-1,1.1255588E1,-1.4877698E-2,-1.7878741E-2,-6.0878512E-3,9E0,2.372255E2,4.3066381E-4,7.2732233E-3,-4.4377907E-3,1.4540612E7,7.026578E7,-1.6682349E-3,2.2444444E0,-4.2836997E-3,2.88E2,1.7E1,5.862126E2,1.6254681E0,8.7E1,1.4571098E5,6.1099392E1,5.8965025E5,4.3062535E-3,2.1196339E1,2.5927516E11,9.11898E5,1E0,1E0,-6.74639E-3,-1.5735751E-2,1E0,-4.6835793E-3,-6.1927224E-3,-2.521512E-3,3.2049225E1,6.384412E-3,1.11E2,2.1036203E5,1.504779E7,4.651395E1,5.094239E3,3.051499E5,3.166E3,1.3347535E-3,8.07282E-5,-3.0072713E-3,-4.4761016E-3,9.123214E-1,-2.9825121E-3,-1.2161283E-2,4.2161504E-3,-9.176472E-4,-5.631594E-3,-1.02839884E-4,-6.2596984E-3,-2.7095315E-3,-1.0191204E-3,5.4144515E-3,3.0652676E-3,-7.458873E-4,-3.481034E-4,2.0536599E-3,1.1007979E-3,-1.0075902E-3,1.0768778E-2,4.276157E-3,5.017865E-3,5.0543854E-4,1.7375402E-4,1.9274506E-3,-7.677217E-3,-1.330058E-3,2.8981903E-4,-2.5363967E-3,-2.2974599E-3,-8.9023765E-3,-1.2856742E-3,3.313119E-3,-5.0770915E-3,5.555805E-4,-3.2162664E-3,-9.273725E-3,-1.6209612E-3,-5.889661E-3,-1.1563351E-2,-5.6009535E-3,-3.313627E-3,-8.436277E-3,-4.792313E-4,-3.0092846E-3,5.482493E-4,-1.7244387E-3,-1.1226097E-3,2.0655717E-3,6.176549E-3,1.5149925E-4,1.278304E-2,6.4885025E-3,2.7387517E-3,-4.0558574E-4,1.317985E-3,4.415732E-3,-3.0578708E-3,1.7908147E-3,6.6751393E-4,6.48127E-3,-3.0354224E-3,-1.4532081E-4,2.1945885E-3,-3.397985E-3],"split_indices":[2,45,7,52,52,3,45,9,28,32,15,0,0,28,5,28,10,54,45,32,56,10,52,29,47,4,27,0,105,0,41,56,4,58,45,38,53,0,45,3,0,3,0,1,3,35,0,44,32,10,0,14,2,33,0,12,7,0,2,45,58,7,7,0,0,0,3,28,27,0,31,12,27,0,0,38,2,29,0,28,0,42,0,42,12,29,8,0,34,1,57,1,0,29,27,2,7,33,0,0,0,54,0,9,27,0,0,0,0,53,10,0,0,56,7,9,0,11,0,0,0,0,0,27,0,0,101,0,0,35,0,50,102,33,42,0,54,33,0,0,0,0,0,0,0,0,0,2,53,58,0,0,0,3,4,0,0,0,45,31,0,56,0,0,3,52,53,8,28,54,47,0,54,31,29,67,75,0,0,75,0,0,0,58,0,8,33,45,58,4,28,0,0,0,0,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.558E3,2.193E3,3.65E2,1.958E3,2.35E2,1.2E1,3.53E2,5.88E2,1.37E3,1.62E2,7.3E1,8E0,4E0,1.88E2,1.65E2,5.08E2,8E1,1.334E3,3.6E1,1.34E2,2.8E1,1.8E1,5.5E1,7E1,1.18E2,1.34E2,3.1E1,4.63E2,4.5E1,4.2E1,3.8E1,9.34E2,4E2,1E1,2.6E1,1.13E2,2.1E1,5E0,2.3E1,8E0,1E1,4.8E1,7E0,1.6E1,5.4E1,1.13E2,5E0,7.4E1,6E1,1.3E1,1.8E1,3.14E2,1.49E2,4.1E1,4E0,3.3E1,9E0,5E0,3.3E1,9.23E2,1.1E1,1.1E1,3.89E2,4E0,6E0,1.3E1,1.3E1,6.3E1,5E1,5E0,1.6E1,1.2E1,1.1E1,4E0,4E0,2.7E1,2.1E1,1.2E1,4E0,4.9E1,5E0,1.04E2,9E0,5.9E1,1.5E1,4.6E1,1.4E1,4E0,9E0,9E0,9E0,2.93E2,2.1E1,1.2E1,1.37E2,1.5E1,2.6E1,2.6E1,7E0,4E0,5E0,2.7E1,6E0,2.6E1,8.97E2,7E0,4E0,6E0,5E0,3.08E2,8.1E1,7E0,6E0,2.6E1,3.7E1,4.5E1,5E0,1.2E1,4E0,6E0,6E0,4E0,7E0,2.3E1,4E0,4E0,1.7E1,6E0,6E0,4.2E1,7E0,3.4E1,7E1,3.5E1,2.4E1,4E0,1.1E1,4.1E1,5E0,4E0,1E1,4E0,5E0,5E0,4E0,5E0,4E0,7.6E1,2.17E2,1.6E1,5E0,5E0,7E0,1.18E2,1.9E1,4E0,1.1E1,4E0,2.2E1,2.2E1,4E0,2.3E1,4E0,9E0,1.7E1,8.32E2,6.5E1,2.42E2,6.6E1,5.9E1,2.2E1,4E0,2.2E1,1.6E1,2.1E1,9E0,3.6E1,5E0,7E0,1.8E1,5E0,1.2E1,5E0,3.6E1,6E0,2E1,1.4E1,1E1,6E1,1.2E1,2.3E1,1.8E1,6E0,5E0,6E0,4E0,3.7E1,7.2E1,4E0,2E1,1.97E2,9E0,7E0,4.4E1,7.4E1,1.5E1,4E0,1.8E1,4E0,5E0,1.7E1,9E0,1.4E1,5E0,4E0,1.2E1,5E0,5.9E2,2.42E2,1.1E1,5.4E1,1.69E2,7.3E1,6E1,6E0,1E1,4.9E1,6E0,1.6E1,1.8E1,4E0,5E0,1.1E1,1.5E1,6E0,5E0,4E0,8E0,2.8E1,4E0,1.4E1,9E0,2.7E1,1.3E1,7E0,8E0,6E0,4E0,6E0,1E1,5E1,8E0,4E0,1.5E1,8E0,1.4E1,4E0,3.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"259","size_leaf_vector":"1"}},{"base_weights":[-1.126241E-3,-1.357514E-2,3.3620875E-2,-7.798633E-3,-9.850678E-2,6.1784882E-2,-1.1879201E-2,-3.5267234E-2,3.2918176E-3,-2.6892012E-1,-8.380204E-2,1.3609679E-2,5.749371E-2,-3.4467563E-2,3.852839E-2,-2.2045676E-2,-7.017335E-2,3.473517E-2,-7.53094E-3,-7.792386E-3,-1.639429E-2,-6.7180365E-2,-1.7175831E-1,5.0619327E-2,1.3386945E-1,-1.7889353E-2,-9.50744E-2,9.467601E-2,8.138546E-3,-6.0714256E-2,-9.631519E-3,-4.5813203E-2,-1.433393E-1,6.514253E-2,6.984731E-4,-3.642153E-2,5.2265655E-3,-7.4541636E-2,4.0088636E-3,-1.3585344E-2,-1.07380204E-1,5.5141695E-2,-1.1996133E-1,1.088477E-1,1.260268E-2,-6.639145E-2,3.3579161E-3,-7.447895E-2,-1.0798227E-2,1.0260784E-2,5.5157285E-2,4.848371E-2,-4.6593573E-2,-4.619843E-2,-1.8461066E-1,8.299469E-2,-1.5958391E-2,-9.2545405E-2,-1.6796514E-2,-2.0385902E-1,-8.483117E-2,8.2658194E-2,6.588031E-3,-6.53706E-2,1.748079E-2,-1.6871446E-1,-3.099737E-2,-2.806942E-3,6.462751E-2,-6.157411E-2,-1.3613881E-1,-7.675498E-3,-5.962373E-4,5.038816E-2,1.4576869E-2,6.4222957E-4,-1.2758811E-2,1.0072046E-2,8.594827E-2,-4.870707E-2,-7.514064E-3,4.908336E-2,-1.5365968E-2,4.618778E-3,-1.0313888E-1,4.6666763E-3,-1.0453413E-2,8.293101E-3,1.11427754E-1,-7.305256E-2,1.2151999E-3,-8.1008896E-2,1.5679652E-2,-1.6016606E-2,-5.8727694E-4,9.050854E-3,4.2015377E-2,-9.2646223E-4,-6.881334E-2,-1.0453429E-1,6.1377365E-4,-4.282505E-3,-9.557608E-2,-3.541705E-3,-1.3818655E-2,-4.1183198E-4,-1.2882127E-1,1.5946414E-1,7.349603E-2,-5.495313E-3,3.0393481E-2,3.1307895E-2,-1.7350163E-2,4.0036812E-2,-5.002356E-2,-1.3371128E-2,1.733595E-4,-1.7757298E-2,-8.4854744E-2,3.5492033E-2,-1.0638574E-2,7.9178564E-2,-1.3099113E-2,-1.68304E-2,-9.301785E-2,-1.0911087E-2,-5.705678E-2,6.297275E-2,4.0826965E-3,1.8592386E-3,1.02835014E-1,-5.9548255E-2,1.6282217E-3,-1.9046153E-3,6.883849E-2,-8.108431E-2,7.620106E-4,-1.5759581E-3,1.6979399E-3,-8.7534515E-3,-6.538808E-2,-2.616103E-3,1.6759231E-3,4.782568E-2,-3.6022045E-2,7.779067E-3,2.876812E-3,-1.6989501E-2,-8.644396E-3,-6.301007E-2,-1.1871874E-2,6.4069204E-2,-5.2527394E-2,-1.1657681E-3,5.5310433E-3,-8.165595E-3,4.7940756E-3,-1.4267946E-2,-3.346626E-2,-1.2308049E-1,-3.522007E-2,8.03569E-3,-4.1110413E-3,-7.934647E-3,-6.6146546E-4,-8.033165E-3,-2.5951832E-3,4.0736697E-3,9.246026E-3,1.11024745E-1,4.7977347E-2,-9.0209715E-4,4.3622598E-2,-1.9494969E-3,5.691673E-2,1.3942607E-2,1.2759615E-1,-3.2568786E-3,-9.197397E-3,-3.211667E-2,1.742021E-2,-8.124617E-3,-1.2627612E-1,6.251246E-2,-6.295067E-3,-2.7476493E-2,9.600591E-3,5.4212503E-2,1.2890688E-1,-3.1912578E-3,1.9757224E-3,2.5853641E-2,-5.852442E-2,-1.2788008E-1,-2.8075337E-2,-8.041931E-4,-4.3274937E-3,8.595709E-2,4.450724E-2,2.8056532E-2,-6.6844255E-2,2.8808196E-3,6.313023E-3,-7.836762E-2,-2.0083725E-2,-1.0347493E-3,1.06929064E-1,-1.5298782E-3,-7.1587125E-3,1.7458478E-2,-7.216458E-2,-2.6515734E-4,-4.106018E-3,3.9016267E-3,-2.0495987E-4,-3.3153107E-3,-2.3707689E-4,-2.2171796E-3,1.5101064E-3,-3.8889642E-3,1.0405242E-3,1.570284E-3,6.2016915E-3,-6.667513E-3,7.356293E-4,-3.2522646E-4,2.7211125E-3,3.4931174E-3,-2.438531E-3,-7.1053766E-3,-2.2685046E-3,-3.004673E-3,-1.6302052E-4,-1.6560499E-4,3.9899386E-3,6.2159905E-3,-5.6232253E-5,1.4884925E-3,5.751383E-3,4.512738E-3,5.418312E-4,4.0006447E-3,-1.362458E-4,4.7630076E-3,-2.8003706E-4,-1.3909467E-3,8.284787E-3,1.8249346E-3,-8.080868E-3,-3.2796052E-3,1.9967621E-4,2.807783E-3,-1.6313803E-3,-2.2344661E-3,2.6161077E-3,-8.436173E-3,-9.028667E-4,6.2055485E-3,1.5044459E-3,-1.0558618E-3,3.9386842E-3,-8.0764265E-4,-2.9807216E-3,3.2786224E-3,-1.5552339E-4,6.789468E-4,4.305557E-3,8.375374E-3,2.250844E-3,-5.6350726E-4,5.857308E-3,-4.4505238E-3,6.15615E-4,-1.2128717E-3,-7.4177203E-3,1.5814447E-4,-3.1786961E-3,3.631824E-3,8.294404E-3,1.4210481E-3,3.99715E-3,2.114578E-3,-3.4058068E-3,-7.638307E-3,-1.7880099E-3,-1.1412945E-3,-4.884866E-3,-2.9735768E-3,8.284655E-4,8.884819E-4,6.6451314E-3,4.3960885E-4,4.5059253E-3,-6.3546854E-3,-1.1272329E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,73,75,77,-1,79,81,83,-1,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,-1,-1,127,-1,-1,-1,-1,129,131,-1,133,135,137,139,-1,141,143,145,147,-1,149,151,-1,-1,-1,153,155,157,159,-1,161,163,-1,-1,-1,165,167,169,-1,171,173,-1,175,177,-1,-1,179,181,183,185,187,189,191,193,-1,195,197,199,-1,201,203,-1,-1,205,207,209,-1,-1,-1,211,-1,-1,213,215,-1,-1,217,-1,219,-1,221,223,-1,-1,-1,225,-1,227,229,231,233,-1,-1,-1,-1,-1,-1,-1,235,237,-1,239,-1,241,243,245,247,-1,249,251,253,255,257,259,261,263,265,267,-1,-1,269,271,273,275,-1,-1,277,279,281,283,-1,-1,285,287,-1,289,-1,-1,291,293,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0987347E0,9.1666913E-1,8.593152E-1,5.3384393E-1,2.7566433E-1,3.584633E-1,2.9395458E-1,2.3105568E-1,4.2530105E-1,9.005249E-3,1.5074098E-1,0E0,2.0691335E-1,1.7714521E-1,1.3573489E-1,1.7558911E-1,2.393952E-1,3.3114356E-1,3.4296218E-1,0E0,0E0,1.09068155E-1,8.038944E-2,2.9590058E-1,7.027185E-2,1.4552867E-1,7.862911E-2,1.11139596E-1,1.1929238E-1,1.5153414E-1,1.6482799E-1,1.4082702E-1,1.0309684E-1,1.7278546E-1,1.6967916E-1,1.9928005E-1,3.084513E-1,6.430483E-2,0E0,0E0,5.1119924E-2,3.9691842E-1,1.8309307E-1,4.4253767E-2,0E0,5.6543782E-2,8.551385E-2,7.7552184E-2,0E0,0E0,5.6171104E-2,7.749847E-2,4.4931725E-2,1.7574456E-1,2.077089E-1,6.4782515E-2,2.0809834E-1,5.1555395E-2,6.4656675E-2,1.3507354E-1,6.2952265E-2,7.976216E-2,1.1434044E-1,8.556274E-1,1.877636E-1,1.8822181E-1,1.9458899E-1,1.7134538E-1,8.8124484E-2,1.0676649E-1,8.508262E-2,0E0,0E0,2.0952755E-1,0E0,0E0,0E0,0E0,1.6749486E-2,3.4549102E-2,0E0,5.2581355E-2,7.531498E-2,1.1689024E-2,5.6540668E-2,0E0,1.8347725E-2,3.6760617E-2,1.9637257E-2,9.232099E-2,0E0,1.3158497E-1,1.0238064E-1,0E0,0E0,0E0,6.541373E-2,1.9134228E-1,4.3478066E-1,4.194367E-2,0E0,5.638377E-2,4.474847E-2,0E0,0E0,0E0,2.1126047E-2,1.0830492E-2,1.0767996E-1,0E0,2.2578746E-2,4.5442544E-2,0E0,2.084218E-1,1.9373757E-1,0E0,0E0,1.1269767E-1,1.7060047E-1,1.09916724E-1,1.617961E-1,7.371047E-2,3.7267312E-2,6.024215E-2,9.8291546E-2,0E0,9.500291E-3,1.1628449E-1,1.3450591E-1,0E0,1.1834711E-2,2.29351E-2,0E0,0E0,8.4076144E-2,3.9549187E-2,7.188164E-2,0E0,0E0,0E0,1.7718546E-2,0E0,0E0,1.7604297E-2,9.171106E-3,0E0,0E0,1.685504E-2,0E0,6.12617E-2,0E0,3.138125E-2,7.270149E-2,0E0,0E0,0E0,1.1206408E-1,0E0,8.426997E-2,3.5134763E-2,6.969357E-3,4.1835528E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.065618E-2,8.447459E-2,0E0,4.03446E-2,0E0,2.507126E-2,1.1506919E-1,1.2933755E-1,1.5719925E-1,0E0,1.9151714E-1,1.2802199E-1,4.6607327E-2,1.5813369E-1,1.1418328E-1,5.0918084E-2,9.331748E-2,1.5402515E-1,5.7248145E-2,6.0624003E-2,0E0,0E0,5.8998115E-2,3.78852E-2,5.821079E-2,1.9076664E-2,0E0,0E0,1.20725036E-1,8.838901E-2,8.385511E-2,4.796809E-2,0E0,0E0,2.1459445E-2,1.833359E-2,0E0,3.713274E-2,0E0,0E0,2.9436436E-2,2.7149431E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,40,40,41,41,42,42,43,43,45,45,46,46,47,47,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,73,73,78,78,79,79,81,81,82,82,83,83,84,84,86,86,87,87,88,88,89,89,91,91,92,92,96,96,97,97,98,98,99,99,101,101,102,102,106,106,107,107,108,108,110,110,111,111,113,113,114,114,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,126,126,127,127,128,128,130,130,131,131,134,134,135,135,136,136,140,140,143,143,144,144,147,147,149,149,151,151,152,152,156,156,158,158,159,159,160,160,161,161,169,169,170,170,172,172,174,174,175,175,176,176,177,177,179,179,180,180,181,181,182,182,183,183,184,184,185,185,186,186,187,187,188,188,191,191,192,192,193,193,194,194,197,197,198,198,199,199,200,200,203,203,204,204,206,206,209,209,210,210],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,74,76,78,-1,80,82,84,-1,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,-1,-1,128,-1,-1,-1,-1,130,132,-1,134,136,138,140,-1,142,144,146,148,-1,150,152,-1,-1,-1,154,156,158,160,-1,162,164,-1,-1,-1,166,168,170,-1,172,174,-1,176,178,-1,-1,180,182,184,186,188,190,192,194,-1,196,198,200,-1,202,204,-1,-1,206,208,210,-1,-1,-1,212,-1,-1,214,216,-1,-1,218,-1,220,-1,222,224,-1,-1,-1,226,-1,228,230,232,234,-1,-1,-1,-1,-1,-1,-1,236,238,-1,240,-1,242,244,246,248,-1,250,252,254,256,258,260,262,264,266,268,-1,-1,270,272,274,276,-1,-1,278,280,282,284,-1,-1,286,288,-1,290,-1,-1,292,294,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0855529E3,4.2343444E7,2.8839298E7,1.5561539E2,2.3446269E2,2.8972292E-5,4.247868E3,5.901422E5,1.4702222E0,6.8651576E9,1.13808104E8,1.3609679E-2,1.8552577E2,4.1822004E7,5.499284E7,2.102858E4,6.699017E6,5.9693484E0,3.0291306E2,-7.792386E-3,-1.639429E-2,1.4622E4,2.0288463E0,5.68E2,1.4009782E9,6.015E3,8.91942E5,1.5277338E0,2.289485E1,4.3E2,7.635605E-6,9E2,1.05E2,1.3058E4,5.9643E4,8.080045E-6,1.9925156E6,2.2797407E9,4.0088636E-3,-1.3585344E-2,1.515252E10,1.9013363E4,9.773983E6,7.318337E-1,1.260268E-2,3.6672198E-8,2.322963E0,5.198287E5,-1.0798227E-2,1.0260784E-2,5.5544252E7,9.609747E3,1E0,4.7E2,6.46E2,3.938775E4,8.635138E6,1.6709303E1,1.2737473E1,1.30727E5,1.937716E6,1.15E0,1.394E3,1.0700461E6,7.465873E6,2.194E3,8.5599425E5,1.5834932E6,4E0,2.8636363E0,4.8767575E9,-7.675498E-3,-5.962373E-4,7.418546E1,1.4576869E-2,6.4222957E-4,-1.2758811E-2,1.0072046E-2,4.6726016E3,8.474894E-9,-7.514064E-3,1.1472081E0,3.6616542E0,2.3E1,1.0052E4,4.6666763E-3,1.671E3,2.1150263E-1,1.9162654E7,7.446158E-1,1.2151999E-3,6.737101E7,3.8452727E3,-1.6016606E-2,-5.8727694E-4,9.050854E-3,6.6126086E4,1.198E5,1.2694E4,3.3253515E6,6.1377365E-4,3.8992128E6,3.25855E7,-3.541705E-3,-1.3818655E-2,-4.1183198E-4,1.568E3,7.920064E5,1.9358906E6,-5.495313E-3,3.1809045E2,1.208686E6,-1.7350163E-2,2.2E1,2.185321E0,-1.3371128E-2,1.733595E-4,1.526038E6,1.8298306E7,2.2E1,3.904E3,2.612E3,3.14088E5,6.1061732E7,1.221875E1,-1.0911087E-2,8.027061E2,8E-3,1.3195207E7,1.8592386E-3,3.1E1,1.6E1,1.6282217E-3,-1.9046153E-3,3.291238E7,4.822785E0,9E0,-1.5759581E-3,1.6979399E-3,-8.7534515E-3,2.0447762E0,-2.616103E-3,1.6759231E-3,8.230524E3,4.2343444E7,7.779067E-3,2.876812E-3,3.6E1,-8.644396E-3,2.7523365E0,-1.1871874E-2,7.9037575E1,9.6E1,-1.1657681E-3,5.5310433E-3,-8.165595E-3,9.8376294E5,-1.4267946E-2,7.9E1,2.21875E0,3.290375E-2,3.6E1,-4.1110413E-3,-7.934647E-3,-6.6146546E-4,-8.033165E-3,-2.5951832E-3,4.0736697E-3,9.246026E-3,2.1146836E7,1.5E1,-9.0209715E-4,9E0,-1.9494969E-3,9.342733E7,1E0,1.1485E4,1.8817276E1,-9.197397E-3,4.624343E6,3.8410318E2,9.024187E6,2.183E3,2.53383E5,1.7921995E3,1.5548611E7,2.9034E5,1.1860938E8,6.3639E4,-3.1912578E-3,1.9757224E-3,1.8E1,1.8571428E0,7.483E3,5.2288747E-1,-8.041931E-4,-4.3274937E-3,7.2E1,1.6195753E9,5.4210526E1,2.5369E4,2.8808196E-3,6.313023E-3,6.747114E7,2.05156E5,-1.0347493E-3,1.3999657E3,-1.5298782E-3,-7.1587125E-3,2.6312E4,3.266054E3,-2.6515734E-4,-4.106018E-3,3.9016267E-3,-2.0495987E-4,-3.3153107E-3,-2.3707689E-4,-2.2171796E-3,1.5101064E-3,-3.8889642E-3,1.0405242E-3,1.570284E-3,6.2016915E-3,-6.667513E-3,7.356293E-4,-3.2522646E-4,2.7211125E-3,3.4931174E-3,-2.438531E-3,-7.1053766E-3,-2.2685046E-3,-3.004673E-3,-1.6302052E-4,-1.6560499E-4,3.9899386E-3,6.2159905E-3,-5.6232253E-5,1.4884925E-3,5.751383E-3,4.512738E-3,5.418312E-4,4.0006447E-3,-1.362458E-4,4.7630076E-3,-2.8003706E-4,-1.3909467E-3,8.284787E-3,1.8249346E-3,-8.080868E-3,-3.2796052E-3,1.9967621E-4,2.807783E-3,-1.6313803E-3,-2.2344661E-3,2.6161077E-3,-8.436173E-3,-9.028667E-4,6.2055485E-3,1.5044459E-3,-1.0558618E-3,3.9386842E-3,-8.0764265E-4,-2.9807216E-3,3.2786224E-3,-1.5552339E-4,6.789468E-4,4.305557E-3,8.375374E-3,2.250844E-3,-5.6350726E-4,5.857308E-3,-4.4505238E-3,6.15615E-4,-1.2128717E-3,-7.4177203E-3,1.5814447E-4,-3.1786961E-3,3.631824E-3,8.294404E-3,1.4210481E-3,3.99715E-3,2.114578E-3,-3.4058068E-3,-7.638307E-3,-1.7880099E-3,-1.1412945E-3,-4.884866E-3,-2.9735768E-3,8.284655E-4,8.884819E-4,6.6451314E-3,4.3960885E-4,4.5059253E-3,-6.3546854E-3,-1.1272329E-3],"split_indices":[52,45,45,52,52,42,52,50,53,5,45,0,56,51,50,28,45,56,52,0,0,2,53,3,32,2,9,53,57,29,41,2,52,29,1,42,28,12,0,0,5,52,1,42,0,37,53,32,0,0,45,52,102,1,2,45,45,58,54,29,50,54,12,47,50,0,47,45,53,54,12,0,0,56,0,0,0,0,4,37,0,53,53,8,2,0,0,38,47,34,0,7,47,0,0,0,28,7,29,45,0,51,5,0,0,0,0,28,45,0,4,28,0,58,41,0,0,9,5,3,2,2,1,45,58,0,52,57,9,0,8,18,0,0,45,54,17,0,0,0,54,0,0,52,45,0,0,8,0,53,0,33,29,0,0,0,28,0,11,53,38,8,0,0,0,0,0,0,0,9,3,0,3,0,5,8,9,56,0,45,4,5,2,9,4,45,12,5,10,0,0,3,53,9,27,0,0,3,7,53,29,0,0,45,12,0,52,0,0,2,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.538E3,1.869E3,6.69E2,1.751E3,1.18E2,4.13E2,2.56E2,5.03E2,1.248E3,8E0,1.1E2,7E0,4.06E2,1.77E2,7.9E1,3.66E2,1.37E2,3.19E2,9.29E2,4E0,4E0,9.4E1,1.6E1,3.74E2,3.2E1,1.4E2,3.7E1,2.7E1,5.2E1,8.8E1,2.78E2,1.04E2,3.3E1,1.68E2,1.51E2,2.84E2,6.45E2,9E1,4E0,5E0,1.1E1,3.65E2,9E0,2.8E1,4E0,4.2E1,9.8E1,3.3E1,4E0,6E0,2.1E1,3E1,2.2E1,8E1,8E0,1.7E1,2.61E2,3.9E1,6.5E1,1.5E1,1.8E1,1.29E2,3.9E1,3E1,1.21E2,1E1,2.74E2,5.69E2,7.6E1,7.6E1,1.4E1,7E0,4E0,3.59E2,6E0,5E0,4E0,4E0,2.4E1,3.6E1,6E0,2.8E1,7E1,9E0,2.4E1,1.3E1,8E0,1.9E1,1.1E1,1.6E1,6E0,5.1E1,2.9E1,4E0,4E0,4E0,1.3E1,2.04E2,5.7E1,3.5E1,4E0,5.7E1,8E0,6E0,9E0,7E0,1.1E1,1.2E1,1.17E2,6E0,3.3E1,2.3E1,7E0,9.1E1,3E1,6E0,4E0,2.21E2,5.3E1,9.6E1,4.73E2,6.4E1,1.2E1,3.2E1,4.4E1,6E0,8E0,2.82E2,7.7E1,7E0,1.7E1,3.2E1,4E0,5E0,2.3E1,1.3E1,5.7E1,4E0,5E0,7E0,1.7E1,4E0,4E0,1E1,9E0,5E0,6E0,1.1E1,5E0,4.7E1,4E0,1.7E1,1.2E1,7E0,6E0,6E0,1.98E2,7E0,5E1,2.7E1,8E0,5E1,7E0,4E0,4E0,7E0,4E0,4E0,8E0,4.6E1,7.1E1,7E0,2.6E1,6E0,1.7E1,7.1E1,2E1,2.3E1,7E0,1.57E2,6.4E1,1.9E1,3.4E1,5.8E1,3.8E1,2.58E2,2.15E2,4.4E1,2E1,6E0,6E0,1.6E1,1.6E1,2.8E1,1.6E1,4E0,4E0,1.24E2,1.58E2,5.8E1,1.9E1,7E0,1E1,2.1E1,1.1E1,7E0,1.6E1,8E0,5E0,4.7E1,1E1,4E0,1.3E1,6E0,4E0,4E0,5E0,7E0,4E0,4E1,7E0,1.2E1,5E0,5E0,7E0,1.62E2,3.6E1,6E0,4.4E1,2.1E1,6E0,4E0,4E0,4.4E1,6E0,4.1E1,5E0,5.7E1,1.4E1,1E1,1.6E1,1.2E1,5E0,1.3E1,5.8E1,4E0,1.6E1,1.9E1,4E0,8.1E1,7.6E1,3.6E1,2.8E1,1.2E1,7E0,2.4E1,1E1,1.9E1,3.9E1,3.3E1,5E0,1.92E2,6.6E1,3.9E1,1.76E2,2E1,2.4E1,1.3E1,7E0,1.2E1,4E0,1.1E1,5E0,5E0,2.3E1,9E0,7E0,1.08E2,1.6E1,1.1E2,4.8E1,5.1E1,7E0,4E0,1.5E1,6E0,1.5E1,5E0,6E0,4E0,1.2E1,4.3E1,4E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"295","size_leaf_vector":"1"}},{"base_weights":[-1.9480054E-3,9.260051E-3,-3.2566287E-2,1.9535697E-3,5.262993E-2,-6.860838E-2,-1.8638017E-2,-1.6680319E-3,6.64377E-2,6.3988306E-2,-7.780222E-2,-5.3823017E-2,-1.4566958E-1,-6.251108E-2,-8.576081E-3,2.6849527E-3,-4.6152852E-2,-2.4529265E-3,7.27489E-2,1.6695312E-1,5.2211035E-2,-1.1735112E-2,-2.9853038E-2,-2.865469E-2,-1.2445102E-1,1.7232958E-3,-1.7471276E-1,-2.8002873E-2,-1.06671676E-1,-6.778661E-2,-1.6826605E-3,-8.618916E-3,1.8763468E-2,-2.9593412E-2,-1.9928043E-1,3.1842172E-2,9.150767E-2,1.1649693E-1,1.3993424E-2,1.576952E-1,4.1096434E-2,-6.2366664E-2,2.276914E-3,-1.7450276E-1,-1.3809386E-2,-6.2703E-2,-1.7273404E-1,-1.1588523E-2,-5.9980564E-2,-6.7103943E-3,-1.0019298E-1,-1.5282793E-1,-5.287402E-2,-4.61058E-2,-1.0942874E-2,-9.353962E-3,4.3218963E-2,8.434792E-3,-2.1913875E-2,5.8538493E-2,7.4591595E-3,-3.8497724E-2,8.747209E-2,-1.3702532E-2,-3.3042787E-3,1.0437131E-1,-1.02208005E-2,1.0158853E-1,9.785502E-4,1.3174948E-3,8.338195E-3,3.8945659E-3,8.588044E-3,3.457005E-2,9.827024E-3,-3.469507E-5,-5.7526343E-3,-1.111755E-2,-3.6179374E-3,-6.7028165E-2,2.406559E-2,-1.0419463E-1,2.5721132E-3,-1.293412E-2,-1.1887768E-1,-5.8929184E-3,-8.428283E-5,-1.8840194E-2,4.3380144E-3,-8.203131E-3,-4.4229833E-4,-8.295304E-3,-3.8684362E-3,-4.7765397E-3,-5.552751E-3,-6.390842E-2,1.4311753E-3,5.3889185E-2,-1.708289E-2,2.2346327E-2,9.9768E-2,1.9466045E-3,9.810695E-2,-5.610946E-2,-2.6988953E-3,8.065492E-2,-3.6781677E-3,-1.4184008E-2,2.721253E-2,-4.8497237E-2,6.045197E-2,7.592507E-3,2.7473256E-4,7.929255E-3,9.1506646E-4,-4.6183784E-3,3.8585827E-2,1.2858356E-3,1.0981038E-1,3.946512E-2,-8.059309E-3,3.557311E-3,-8.188724E-2,1.1007203E-1,-3.9377846E-3,-6.532749E-3,-2.12773E-3,9.802597E-6,-1.5858531E-1,-2.750333E-2,2.1903974E-3,9.711382E-4,-1.6103301E-3,-8.9761876E-2,-2.6103377E-2,2.3728457E-5,7.941451E-2,-1.4420339E-2,-6.7424052E-3,3.320807E-2,-2.2354198E-3,7.585486E-3,2.0865796E-3,1.631158E-2,-4.6024114E-2,1.5147072E-1,8.2506E-3,-3.4498073E-2,-1.5149036E-1,-1.14473365E-2,1.4947397E-1,2.0817136E-2,9.1895305E-2,7.211396E-2,-3.826297E-2,1.341304E-2,-4.4919476E-2,3.7700016E-2,-1.0952612E-2,-1.2868966E-2,-8.456319E-2,2.0927595E-4,4.5971484E-3,3.915518E-3,-5.1006704E-5,1.1670869E-1,1.7009415E-3,3.3705793E-2,1.0660211E-2,-1.841953E-1,-5.239725E-2,1.3653941E-1,1.2259774E-3,9.943917E-2,-3.2268003E-2,-9.32E-3,-4.105551E-3,-7.1775755E-3,-2.63678E-3,-5.5921584E-2,-7.401173E-3,-1.8449589E-3,-2.905084E-3,6.9915717E-3,4.9827103E-2,-3.1927507E-2,6.839026E-4,-9.6273795E-4,4.4729505E-2,-5.373285E-4,3.9254255E-3,1.8517071E-3,-3.4189138E-3,2.9257345E-3,9.597289E-3,-2.0000204E-3,3.2245344E-3,-2.7777292E-3,1.2518376E-3,-1.0405482E-2,-3.5850618E-3,-9.934165E-4,5.225073E-3,3.2934397E-3,1.2366436E-2,2.418625E-3,-1.6603778E-3,6.3934433E-3,3.1993287E-3,3.1750213E-4,5.4355324E-3,-3.6671252E-3,-1.4457334E-4,-8.8749407E-4,4.167005E-3,-8.81086E-3,-9.967253E-4,1.3507442E-3,4.26554E-3,-3.9625964E-3,1.276947E-3,4.6105497E-4,-3.0731254E-3,-5.0249584E-3,1.5338177E-4,2.0786705E-3,6.2010535E-3,2.2479605E-3,-1.2436935E-3,-5.15844E-3,-1.1408084E-2,4.0363564E-4,-4.0808506E-3,2.531215E-3,8.65162E-3,7.0928136E-4,7.3012393E-3,-5.180152E-3,9.820672E-4,6.1183644E-4,-1.8908331E-3,-3.904893E-3,-8.4760884E-4,-1.7266184E-3,1.5606829E-3,3.7278438E-3,9.749121E-4,-2.5224644E-3,-8.175533E-4,1.2002611E-3,-9.847427E-4,1.4318633E-3,4.554527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,-1,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,-1,77,79,81,83,-1,85,87,89,91,93,95,-1,97,99,101,103,105,107,109,111,-1,-1,113,115,117,-1,-1,-1,-1,-1,119,-1,-1,-1,-1,-1,121,123,125,-1,-1,127,-1,-1,129,-1,-1,-1,-1,-1,131,-1,133,-1,135,137,139,141,143,145,147,149,151,153,155,157,159,161,-1,-1,-1,-1,-1,163,-1,165,167,-1,-1,169,171,173,-1,-1,-1,175,177,-1,-1,-1,179,181,-1,183,185,-1,187,-1,-1,-1,189,191,193,195,197,199,201,203,205,207,209,211,213,215,217,219,221,223,-1,-1,-1,-1,225,-1,227,-1,229,231,233,-1,235,237,-1,-1,239,-1,241,-1,243,-1,-1,245,247,249,-1,251,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7470233E-1,5.911975E-1,3.415901E-1,3.733011E-1,4.0350127E-1,2.0864159E-1,2.1753915E-1,2.9329753E-1,6.554708E-2,2.92019E-1,1.5866493E-1,2.8262562E-1,1.6299689E-1,1.3721198E-1,1.6437566E-1,2.509759E-1,3.364231E-1,0E0,5.924231E-2,1.1004192E-1,2.5614017E-1,0E0,4.6982147E-2,2.574093E-1,1.14387095E-1,0E0,1.567555E-1,8.079429E-2,9.124324E-2,1.2759988E-1,1.2509659E-1,1.8406154E-1,2.5585192E-1,1.3158119E-1,1.11605704E-1,8.343811E-2,3.7888378E-2,8.1870615E-2,0E0,1.0352373E-2,2.0033109E-1,4.2222083E-2,0E0,3.7792176E-2,2.2333567E-1,1.00113526E-1,7.590556E-2,0E0,3.358606E-2,4.9283944E-2,6.867384E-2,8.0322325E-3,5.5507645E-2,5.2397765E-2,0E0,1.5244915E-1,6.032791E-2,2.069333E-1,2.994713E-1,1.7367396E-1,1.9063485E-1,1.1671835E-1,4.673507E-2,0E0,0E0,4.3840036E-2,7.578812E-2,2.711007E-2,0E0,0E0,0E0,0E0,0E0,1.9524255E-1,0E0,0E0,0E0,0E0,0E0,1.0030691E-1,1.5698402E-1,1.8967152E-2,0E0,0E0,7.568136E-2,0E0,0E0,2.1839008E-2,0E0,0E0,0E0,0E0,0E0,8.6023435E-3,0E0,2.7956039E-2,0E0,4.631848E-2,8.497548E-2,3.0631876E-2,3.5925552E-2,2.302204E-1,1.1383203E-1,3.3282918E-1,3.9177474E-1,6.0638726E-2,9.188199E-2,1.8127087E-1,9.367263E-2,1.3383336E-1,1.91424E-2,0E0,0E0,0E0,0E0,0E0,1.8985087E-2,0E0,1.8502116E-2,1.8608671E-1,0E0,0E0,1.1706534E-1,3.2872558E-2,1.4964329E-1,0E0,0E0,0E0,1.0295153E-2,1.7082956E-2,0E0,0E0,0E0,2.7265534E-2,1.0773091E-2,0E0,3.420578E-2,7.194747E-2,0E0,2.2022657E-2,0E0,0E0,0E0,4.3383768E-1,1.4583577E-1,4.5344174E-2,2.9755281E-2,1.709736E-1,1.2083733E-1,2.7411008E-1,1.1473742E-1,2.5464864E-2,7.018298E-2,2.5833663E-2,2.9607136E-2,2.4801256E-1,3.279301E-1,9.074351E-2,1.2929426E-1,5.859804E-2,7.323301E-2,0E0,0E0,0E0,0E0,1.6498923E-2,0E0,1.2541892E-1,0E0,8.575082E-3,6.0852274E-2,2.8956413E-2,0E0,4.1699745E-2,1.5033677E-1,0E0,0E0,1.2374904E-2,0E0,9.931985E-3,0E0,1.0817708E-2,0E0,0E0,1.1314932E-2,3.5565987E-2,7.042959E-2,0E0,1.9026104E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,43,43,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,73,73,79,79,80,80,81,81,84,84,87,87,93,93,95,95,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,116,116,118,118,119,119,122,122,123,123,124,124,128,128,129,129,133,133,134,134,136,136,137,137,139,139,143,143,144,144,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,160,160,165,165,167,167,169,169,170,170,171,171,173,173,174,174,177,177,179,179,181,181,184,184,185,185,186,186,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,-1,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,-1,78,80,82,84,-1,86,88,90,92,94,96,-1,98,100,102,104,106,108,110,112,-1,-1,114,116,118,-1,-1,-1,-1,-1,120,-1,-1,-1,-1,-1,122,124,126,-1,-1,128,-1,-1,130,-1,-1,-1,-1,-1,132,-1,134,-1,136,138,140,142,144,146,148,150,152,154,156,158,160,162,-1,-1,-1,-1,-1,164,-1,166,168,-1,-1,170,172,174,-1,-1,-1,176,178,-1,-1,-1,180,182,-1,184,186,-1,188,-1,-1,-1,190,192,194,196,198,200,202,204,206,208,210,212,214,216,218,220,222,224,-1,-1,-1,-1,226,-1,228,-1,230,232,234,-1,236,238,-1,-1,240,-1,242,-1,244,-1,-1,246,248,250,-1,252,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,1.5106794E3,2.0666666E0,4.5723195E6,2.3580047E10,2.0354E4,1.6681991E-3,1E0,8.6E1,1.1485E4,1.562E5,1E1,4.854E4,9.2199E4,1.07279E5,3.5150378E5,9.457831E0,-2.4529265E-3,1E0,1.633077E2,1.5789307E8,-1.1735112E-2,1.1056608E8,1.3676985E4,9.83E2,1.7232958E-3,1.038946E6,1.9E1,3.488E3,5.551823E-1,2.1103458E0,5.116E3,4.0208E4,6.3251557E0,1.3E1,5.8365756E-1,2.1146836E7,2.5629715E6,1.3993424E-2,1.3E1,7.0568744E7,5.21929E5,2.276914E-3,2.18988E5,2.4757753E2,7.17E2,1.3238162E3,-1.1588523E-2,7.016421E5,1E0,2.130881E6,1.3316584E-1,8.282879E0,3.624992E5,-1.0942874E-2,8.963196E6,6.1428223E1,3.591E3,8.684849E0,1.8297922E7,2.228899E6,1.4521204E0,7.366243E7,-1.3702532E-2,-3.3042787E-3,7.780377E6,1.287E3,5.72E2,9.785502E-4,1.3174948E-3,8.338195E-3,3.8945659E-3,8.588044E-3,3.7826266E4,9.827024E-3,-3.469507E-5,-5.7526343E-3,-1.111755E-2,-3.6179374E-3,1.0668695E9,2.014576E8,1.9944625E6,2.5721132E-3,-1.293412E-2,3.650307E9,-5.8929184E-3,-8.428283E-5,3.8114656E5,4.3380144E-3,-8.203131E-3,-4.4229833E-4,-8.295304E-3,-3.8684362E-3,6.938E3,-5.552751E-3,1.3448806E8,1.4311753E-3,7.158023E7,1.6597747E8,2.589369E7,1E0,1E0,1.09E3,3.1530054E0,1.9495119E6,2.6973193E2,2.3303845E6,3.8244262E6,9.433962E-3,2.20524E5,1.01E2,7.592507E-3,2.7473256E-4,7.929255E-3,9.1506646E-4,-4.6183784E-3,6.6315195E6,1.2858356E-3,1.3912211E9,4.42384E5,-8.059309E-3,3.557311E-3,4.82E2,1.0931163E9,6.8503425E6,-6.532749E-3,-2.12773E-3,9.802597E-6,8.4441924E-1,2.772925E3,2.1903974E-3,9.711382E-4,-1.6103301E-3,2.0939131E2,4.116997E6,2.3728457E-5,2.3871907E2,3.2627738E5,-6.7424052E-3,1.5825138E5,-2.2354198E-3,7.585486E-3,2.0865796E-3,6E2,1.5953778E0,1.2E1,9.750871E6,1.838444E6,6.137763E2,7.3963095E9,2.2376953E1,4.3597875E0,4.657871E6,3.1826714E7,8.920204E6,9.772148E4,1.0449754E0,8.874407E6,1.19507775E-1,1.0201212E3,3.4613644E1,2.0927595E-4,4.5971484E-3,3.915518E-3,-5.1006704E-5,1.19216E8,1.7009415E-3,3.388159E7,1.0660211E-2,1.6695403E0,4.2857143E-1,4.7937644E-1,1.2259774E-3,5.59E2,1.4713E4,-9.32E-3,-4.105551E-3,1.348185E6,-2.63678E-3,7.8704453E0,-7.401173E-3,1.4191E4,-2.905084E-3,6.9915717E-3,4.24566E5,2.7453625E-1,7.224059E-1,-9.6273795E-4,7.8704453E0,-5.373285E-4,3.9254255E-3,1.8517071E-3,-3.4189138E-3,2.9257345E-3,9.597289E-3,-2.0000204E-3,3.2245344E-3,-2.7777292E-3,1.2518376E-3,-1.0405482E-2,-3.5850618E-3,-9.934165E-4,5.225073E-3,3.2934397E-3,1.2366436E-2,2.418625E-3,-1.6603778E-3,6.3934433E-3,3.1993287E-3,3.1750213E-4,5.4355324E-3,-3.6671252E-3,-1.4457334E-4,-8.8749407E-4,4.167005E-3,-8.81086E-3,-9.967253E-4,1.3507442E-3,4.26554E-3,-3.9625964E-3,1.276947E-3,4.6105497E-4,-3.0731254E-3,-5.0249584E-3,1.5338177E-4,2.0786705E-3,6.2010535E-3,2.2479605E-3,-1.2436935E-3,-5.15844E-3,-1.1408084E-2,4.0363564E-4,-4.0808506E-3,2.531215E-3,8.65162E-3,7.0928136E-4,7.3012393E-3,-5.180152E-3,9.820672E-4,6.1183644E-4,-1.8908331E-3,-3.904893E-3,-8.4760884E-4,-1.7266184E-3,1.5606829E-3,3.7278438E-3,9.749121E-4,-2.5224644E-3,-8.175533E-4,1.2002611E-3,-9.847427E-4,1.4318633E-3,4.554527E-3],"split_indices":[27,52,53,28,5,9,41,6,0,9,10,3,29,12,1,47,54,0,102,56,5,0,44,33,0,0,9,8,2,38,42,29,9,54,8,54,9,47,0,0,48,10,0,1,55,0,4,0,32,74,1,57,53,47,0,12,56,29,56,45,9,38,7,0,0,28,2,2,0,0,0,0,0,52,0,0,0,0,0,31,7,48,0,0,12,0,0,32,0,0,0,0,0,2,0,7,0,7,45,45,64,16,2,53,28,52,51,50,57,9,0,0,0,0,0,0,28,0,12,2,0,0,2,5,45,0,0,0,27,50,0,0,0,52,12,0,52,28,0,28,0,0,0,2,53,3,9,9,4,5,56,54,51,45,50,33,53,45,38,4,56,0,0,0,0,5,0,45,0,54,56,27,0,0,9,0,0,1,0,53,0,1,0,0,12,39,27,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.547E3,1.865E3,6.82E2,1.597E3,2.68E2,1.89E2,4.93E2,1.513E3,8.4E1,2.47E2,2.1E1,1.6E2,2.9E1,9.1E1,4.02E2,1.379E3,1.34E2,4E0,8E1,2.4E1,2.23E2,4E0,1.7E1,1.19E2,4.1E1,4E0,2.5E1,5.2E1,3.9E1,4.1E1,3.61E2,8.1E2,5.69E2,1.22E2,1.2E1,2.6E1,5.4E1,1.8E1,6E0,2E1,2.03E2,1.2E1,5E0,1E1,1.09E2,1.9E1,2.2E1,1.6E1,9E0,4.1E1,1.1E1,2E1,1.9E1,3.7E1,4E0,3.09E2,5.2E1,3.55E2,4.55E2,1.25E2,4.44E2,1.14E2,8E0,7E0,5E0,9E0,1.7E1,4.7E1,7E0,7E0,1.1E1,4E0,1.6E1,1.96E2,7E0,6E0,6E0,6E0,4E0,4.5E1,6.4E1,1.4E1,5E0,7E0,1.5E1,4E0,5E0,3.7E1,4E0,6E0,5E0,1.6E1,4E0,1.1E1,8E0,3E1,7E0,3.3E1,2.76E2,3.9E1,1.3E1,3.32E2,2.3E1,1.63E2,2.92E2,9.2E1,3.3E1,2.12E2,2.32E2,1.04E2,1E1,4E0,4E0,5E0,4E0,6E0,1.1E1,5E0,4.2E1,1.92E2,4E0,4E0,4.1E1,1.5E1,4.9E1,9E0,5E0,4E0,1.1E1,3.3E1,4E0,6E0,5E0,1.7E1,1.3E1,1.1E1,2.2E1,2.71E2,5E0,3.4E1,5E0,6E0,7E0,2.56E2,7.6E1,1.4E1,9E0,1.34E2,2.9E1,2.77E2,1.5E1,1.5E1,7.7E1,1E1,2.3E1,1.12E2,1E2,1.82E2,5E1,5.3E1,5.1E1,4E0,6E0,5E0,6E0,3.8E1,4E0,1.87E2,5E0,8E0,3.3E1,1.1E1,4E0,1E1,3.9E1,7E0,4E0,1.9E1,1.4E1,1.2E1,5E0,8E0,5E0,6E0,1.6E1,1.25E2,1.46E2,6E0,2.8E1,1.79E2,7.7E1,1.6E1,6E1,5E0,9E0,5E0,4E0,9.9E1,3.5E1,1.6E1,1.3E1,2.59E2,1.8E1,9E0,6E0,1E1,5E0,3.2E1,4.5E1,4E0,6E0,1.1E1,1.2E1,7.8E1,3.4E1,1.5E1,8.5E1,1.5E2,3.2E1,1.7E1,3.3E1,3.7E1,1.6E1,4.3E1,8E0,4E0,3.4E1,1.57E2,3E1,4E0,4E0,1.1E1,2.2E1,4E0,7E0,4E0,6E0,1.6E1,2.3E1,1.2E1,7E0,7E0,5E0,4E0,4E0,8E0,8E0,5.6E1,6.9E1,6.8E1,7.8E1,2.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"253","size_leaf_vector":"1"}},{"base_weights":[-1.3059168E-3,4.1939463E-3,-7.0352845E-2,-1.2160607E-2,3.210566E-2,-1.544824E-1,-5.222146E-2,1.2705002E-2,-2.4272675E-2,4.530002E-2,-1.7313819E-3,-1.2003964E-1,-1.615692E-2,-4.454299E-2,-1.0512823E-2,1.9275945E-2,-1.6103563E-1,-3.530272E-2,-6.0571562E-3,5.567115E-2,3.7038038E-3,-1.6700394E-2,8.367924E-2,-1.53525E-1,-5.980829E-2,-6.526793E-2,-2.6260903E-3,9.185714E-3,6.669877E-2,-1.4512168E-2,-5.2285958E-2,-7.5116446E-3,-5.3958297E-2,4.4790458E-2,-1.4773136E-2,8.670243E-2,4.1552003E-2,-3.1847548E-2,4.654675E-2,-1.1283792E-2,-8.641419E-3,1.1736741E-2,5.8232564E-2,-9.986716E-3,-5.2896957E-3,-2.4945472E-4,-4.3267226E-3,-3.4626875E-2,-9.462922E-2,-7.165234E-2,1.3871257E-2,1.43527705E-2,-1.3135001E-1,4.1862838E-2,1.5405834E-1,1.9511422E-3,-8.479221E-3,-4.7705363E-2,1.5360786E-2,-5.023537E-2,-1.8482158E-1,1.0929773E-1,5.917034E-3,6.216607E-2,-2.0004999E-2,1.5103821E-1,7.156723E-2,7.115308E-2,2.4507662E-2,-6.571343E-2,5.9431912E-3,2.1826865E-2,1.3043658E-1,-2.4317143E-2,5.7384495E-2,8.554361E-2,3.3096075E-3,-5.646231E-3,-2.3651542E-2,-1.1558194E-1,-7.930104E-3,-6.1187213E-3,-8.681133E-4,5.2729435E-3,-4.336437E-3,4.0743258E-2,-1.0757962E-2,-1.9415151E-2,1.0391032E-2,1.1082067E-1,2.1166103E-2,1.9076669E-1,2.4407317E-3,-1.5264712E-1,-2.5279732E-2,1.40777E-1,5.292207E-3,-7.815592E-2,-3.1137299E-2,-2.1455197E-3,-1.3603598E-2,6.1376E-2,1.093634E-2,-2.3880158E-2,4.4656035E-2,2.5201445E-2,8.764781E-3,-7.214381E-2,-1.2119016E-2,1.9390766E-1,3.4227498E-2,2.5775148E-2,9.0405956E-2,9.627465E-2,1.8019551E-2,4.607251E-2,-4.9124222E-2,-8.861103E-2,-3.536284E-3,4.8948288E-2,-3.0584428E-2,3.4857227E-3,8.297671E-4,3.0010699E-3,8.340105E-3,-1.8123329E-2,-1.2346397E-1,7.8919284E-2,-3.9470764E-3,1.2559888E-1,4.815159E-2,-1.483146E-3,1.1750213E-3,-3.845947E-2,2.1509213E-2,-6.2474344E-2,-1.9465789E-1,1.9003669E-3,-1.980069E-3,-3.411199E-3,8.177588E-3,4.5350242E-2,-6.7044413E-3,-3.632236E-2,2.2752695E-2,-9.753943E-4,2.2088836E-3,1.0543495E-2,5.52052E-2,2.6954303E-3,5.0584406E-2,1.1972668E-2,4.689057E-3,-3.0040748E-3,-2.1793127E-1,-1.507862E-2,-5.030614E-3,9.330543E-3,4.0609473E-3,-8.6209895E-3,1.1304744E-2,-3.3165164E-2,-1.12990074E-1,-1.4041543E-2,-7.324676E-2,1.1649675E-4,4.040465E-3,3.2105204E-3,-4.839703E-2,7.615883E-2,-1.0264036E-3,2.8404198E-3,-2.1130147E-3,-2.2701507E-2,-1.623437E-1,1.0270854E-3,-4.8424125E-2,2.7127003E-3,1.1170812E-2,-3.2662738E-3,6.3446946E-3,7.544041E-2,-8.408989E-3,1.5155657E-1,6.361407E-2,1.3179357E-1,6.252356E-2,2.9786954E-2,-2.8755497E-3,3.3227175E-2,1.4267796E-1,-1.4547188E-2,-2.2332897E-2,-6.047941E-2,-5.516984E-3,1.588053E-3,-1.9119962E-3,-5.644855E-4,3.8654036E-3,8.401365E-4,-2.434047E-3,-1.96753E-2,5.769609E-2,-2.2492595E-2,5.030782E-3,-9.230068E-3,-2.5900817E-3,1.1098235E-1,-9.553154E-4,3.0880177E-3,7.886247E-3,3.67119E-3,3.1802448E-4,-5.1406684E-4,-3.5611168E-3,3.2224627E-3,-1.6100525E-3,-6.9721253E-3,-4.21207E-2,-1.0845668E-2,-5.094896E-3,2.0015053E-3,-8.910462E-3,5.6598197E-5,3.1323305E-3,-2.830369E-3,7.804631E-4,1.5895612E-3,-6.423995E-3,-2.8547287E-4,4.538569E-3,1.3008061E-3,-3.1195171E-3,1.3600456E-4,3.3156439E-3,-1.698196E-2,-2.6187792E-3,-2.3069913E-3,1.3431032E-3,1.4031911E-3,-2.1578893E-3,-3.5386004E-3,5.812107E-4,-5.2781517E-5,-6.5902825E-3,5.174934E-3,-9.28962E-4,-6.7870417E-3,-6.1136956E-4,-5.518346E-3,3.036717E-4,5.2622723E-4,6.450345E-3,-2.2661418E-3,1.8370287E-3,-1.40890265E-2,-4.31197E-3,6.805348E-4,-1.4319371E-3,-9.28376E-4,-5.925899E-3,6.0638143E-3,7.172203E-4,-4.4025565E-3,1.1518799E-3,9.160502E-3,4.8898644E-4,1.7405546E-3,5.2686813E-3,1.5089674E-3,7.093294E-3,3.608929E-3,-1.5733415E-3,-7.817726E-4,2.4788615E-3,5.275358E-3,5.920096E-4,1.2938416E-2,3.0058094E-3,6.5125218E-3,-1.9769752E-3,-4.2638173E-3,-1.170975E-3,-4.9012764E-3,-3.397469E-6,5.367693E-3,-1.785065E-4,-6.42457E-3,-7.250111E-4,6.4767175E-3,1.0836567E-3,-1.0796371E-3,-4.146812E-3,1.6088943E-3,-1.3741339E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,-1,-1,75,-1,-1,-1,-1,77,79,81,83,85,87,89,91,-1,-1,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,127,129,131,-1,133,135,137,-1,-1,-1,139,141,143,-1,145,147,149,151,-1,153,155,157,159,161,163,-1,-1,165,-1,167,169,171,-1,173,175,177,179,181,183,185,187,189,191,193,195,197,199,-1,201,-1,-1,203,205,207,-1,209,211,-1,-1,213,215,217,219,-1,-1,-1,221,223,-1,225,227,-1,-1,-1,229,231,233,-1,-1,-1,235,237,-1,-1,-1,-1,239,241,243,245,247,-1,-1,-1,249,251,-1,-1,-1,253,255,257,259,-1,-1,-1,-1,261,263,265,267,269,271,273,-1,275,277,-1,279,281,-1,-1,-1,-1,-1,-1,-1,283,285,287,-1,-1,-1,289,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,291,-1,-1,-1,293,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.728314E-1,1.083626E0,2.8023052E-1,4.511658E-1,3.916222E-1,1.53126E-1,1.813116E-1,5.636492E-1,2.0211059E-1,2.720071E-1,3.1719005E-1,4.9307108E-2,0E0,1.3100612E-1,0E0,2.2619975E-1,2.3811656E-1,3.2466817E-1,1.6951583E-1,2.1765554E-1,1.9491923E-1,1.7661947E-1,1.28692E-1,1.859793E-2,1.7185878E-2,8.7589055E-2,5.9070162E-2,2.865384E-1,1.7363214E-1,0E0,1.2458794E-1,2.3332278E-1,1.7341328E-1,1.3977991E-1,1.3247018E-1,1.4206696E-1,1.7433745E-1,8.997772E-2,1.17430314E-1,1.848659E-1,0E0,0E0,4.9217E-2,0E0,0E0,0E0,0E0,4.119608E-2,9.1729075E-2,2.5447957E-2,7.055058E-2,2.5159574E-1,5.144427E-1,9.1849715E-2,6.0199857E-2,0E0,0E0,2.1265258E-1,2.0382258E-1,1.9272816E-1,1.123603E-1,9.683797E-2,4.2621676E-2,8.348774E-2,1.24942414E-1,1.3858378E-1,1.08953476E-1,1.6778457E-1,3.5635614E-1,5.2363813E-2,5.4906767E-2,4.624006E-2,2.3346722E-2,1.0400118E-1,1.03151426E-1,2.6902646E-2,8.69793E-3,0E0,3.158915E-2,1.6001976E-1,1.7533213E-2,0E0,0E0,0E0,2.9360436E-2,1.5516934E-1,1.6787899E-1,0E0,1.0978015E-2,7.193275E-2,2.8119195E-2,3.9060265E-2,0E0,8.354485E-2,5.7707258E-2,1.763983E-2,1.6339836E-1,2.282005E-1,1.5662812E-1,0E0,0E0,1.8221274E-2,0E0,4.848552E-2,3.442704E-2,3.871099E-2,0E0,1.7470215E-1,1.27481E-1,7.481772E-2,9.1439635E-2,6.684717E-2,1.3944781E-1,9.485674E-2,3.9062712E-2,2.0993996E-1,3.2417518E-1,9.882256E-3,1.470857E-2,2.8260466E-2,1.7145466E-2,0E0,3.9614763E-2,0E0,0E0,8.739626E-2,3.4304157E-2,9.328075E-2,0E0,1.0808557E-2,1.3287576E-2,0E0,0E0,3.1617615E-2,2.9792745E-2,3.4903057E-2,1.108855E-2,0E0,0E0,0E0,1.724429E-2,1.386927E-1,0E0,1.1805712E-1,1.2061685E-1,0E0,0E0,0E0,2.4392534E-2,5.1783543E-2,1.4917802E-2,0E0,0E0,0E0,1.6383165E-1,9.125683E-2,0E0,0E0,0E0,0E0,1.3560672E-1,1.1215215E-1,1.754837E-1,8.594137E-2,2.3889494E-1,0E0,0E0,0E0,5.8127634E-2,3.6853597E-2,0E0,0E0,0E0,3.8036708E-2,1.0302609E-1,7.403151E-2,1.4648865E-1,0E0,0E0,0E0,0E0,4.313632E-2,6.2212285E-2,1.1983687E-1,7.457739E-2,3.8061857E-2,4.423587E-2,3.409954E-2,0E0,2.3695393E-1,1.7630738E-1,0E0,1.288122E-1,1.0889895E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.843131E-2,2.7805427E-2,1.3278206E-1,0E0,0E0,0E0,3.38037E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6651735E-2,0E0,0E0,0E0,1.6934851E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,42,42,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,79,79,80,80,84,84,85,85,86,86,88,88,89,89,90,90,91,91,93,93,94,94,95,95,96,96,97,97,98,98,101,101,103,103,104,104,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,122,122,125,125,126,126,127,127,129,129,130,130,133,133,134,134,135,135,136,136,140,140,141,141,143,143,144,144,148,148,149,149,150,150,154,154,155,155,160,160,161,161,162,162,163,163,164,164,168,168,169,169,173,173,174,174,175,175,176,176,181,181,182,182,183,183,184,184,185,185,186,186,187,187,189,189,190,190,192,192,193,193,201,201,202,202,203,203,207,207,218,218,222,222],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,-1,-1,76,-1,-1,-1,-1,78,80,82,84,86,88,90,92,-1,-1,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,128,130,132,-1,134,136,138,-1,-1,-1,140,142,144,-1,146,148,150,152,-1,154,156,158,160,162,164,-1,-1,166,-1,168,170,172,-1,174,176,178,180,182,184,186,188,190,192,194,196,198,200,-1,202,-1,-1,204,206,208,-1,210,212,-1,-1,214,216,218,220,-1,-1,-1,222,224,-1,226,228,-1,-1,-1,230,232,234,-1,-1,-1,236,238,-1,-1,-1,-1,240,242,244,246,248,-1,-1,-1,250,252,-1,-1,-1,254,256,258,260,-1,-1,-1,-1,262,264,266,268,270,272,274,-1,276,278,-1,280,282,-1,-1,-1,-1,-1,-1,-1,284,286,288,-1,-1,-1,290,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,292,-1,-1,-1,294,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.747114E7,5.862126E2,1.116567E3,2E0,2.342147E7,3.7697E4,2.6572757E10,9.669789E0,2.046E3,1.9221336E1,1.5137865E2,1.4713E4,-1.615692E-2,1.2679E4,-1.0512823E-2,7.5408E4,2.6659616E2,5.116E3,4.8237036E2,5.454369E6,7.06699E8,1.8147379E3,6.485856E7,5.46E2,1.0558809E-4,4.842894E-2,4.956641E-1,2.465E3,4.681779E2,-1.4512168E-2,1.42E1,4.0701206E1,1.5677104E0,4.38E2,8.417E3,2.2805734E9,4.0208E4,8.079531E4,1.058542E0,7.023838E3,-8.641419E-3,1.1736741E-2,4.6506356E5,-9.986716E-3,-5.2896957E-3,-2.4945472E-4,-4.3267226E-3,3.5315048E2,2.822137E6,1.1E1,7.6499896E9,1.5277338E0,5.0647113E2,8.916961E2,4.9355304E2,1.9511422E-3,-8.479221E-3,2.0042918E0,1.1092533E0,5.524425E-4,1E0,3.9552E2,3.63825E2,2.4898794E5,2.2348747E2,1.8601036E1,3.09E2,4.4844124E-1,2.5482938E0,3.92E2,3.050899E4,1.3195207E7,5.704748E4,1.7225842E2,9.123214E-1,3.9712732E7,2.408E3,-5.646231E-3,7.285643E-1,9.6910744E7,6.520433E6,-6.1187213E-3,-8.681133E-4,5.2729435E-3,1E0,1.8817276E1,2.0288463E0,-1.9415151E-2,1.6332493E1,9.9341E4,1.7314286E0,2.3140822E8,2.4407317E-3,1.925E1,2.1592189E5,7.33E2,3.2351852E1,2.0081E4,7.274E3,-2.1455197E-3,-1.3603598E-2,3.6401E4,1.093634E-2,1.8949672E0,1.8174828E6,1.4540612E7,8.764781E-3,1.0942544E6,1.7291142E7,1.139605E6,1.1906538E9,1.9968051E0,1.8722344E8,2.558106E10,9.408306E-1,3.970405E3,6.972973E0,7.667683E6,5.3379045E6,1.5781393E-4,9.536863E4,3.4857227E-3,5.7315452E7,3.0010699E-3,8.340105E-3,2.6037296E-7,2.93E2,7.771383E0,-3.9470764E-3,7.6767676E-2,2.3887E4,-1.483146E-3,1.1750213E-3,1.0828989E8,1.1863768E-4,1E0,9.277796E-1,1.9003669E-3,-1.980069E-3,-3.411199E-3,3.7284137E2,5.08E2,-6.7044413E-3,1.3510204E1,2.067698E0,-9.753943E-4,2.2088836E-3,1.0543495E-2,1.408E3,1.1217151E0,2.960909E2,1.1972668E-2,4.689057E-3,-3.0040748E-3,3.4893394E-4,1.6344065E4,-5.030614E-3,9.330543E-3,4.0609473E-3,-8.6209895E-3,1E0,7.696629E0,1.5756359E0,9.4596675E-5,4.8632206E5,1.1649675E-4,4.040465E-3,3.2105204E-3,6.0231606E5,1.752512E2,-1.0264036E-3,2.8404198E-3,-2.1130147E-3,4E0,6.720137E7,1.9333144E7,7.8E1,2.7127003E-3,1.1170812E-2,-3.2662738E-3,6.3446946E-3,4.052E3,7.857143E0,2.6591174E7,1.4622E4,1.07279E5,1.2E2,1.0855529E3,-2.8755497E-3,9E0,2.06267E5,-1.4547188E-2,1.3865336E0,4.4648915E6,-5.516984E-3,1.588053E-3,-1.9119962E-3,-5.644855E-4,3.8654036E-3,8.401365E-4,-2.434047E-3,6.823E3,6.823E3,9.357879E2,5.030782E-3,-9.230068E-3,-2.5900817E-3,2.2576077E10,-9.553154E-4,3.0880177E-3,7.886247E-3,3.67119E-3,3.1802448E-4,-5.1406684E-4,-3.5611168E-3,3.2224627E-3,-1.6100525E-3,-6.9721253E-3,9.075E3,-1.0845668E-2,-5.094896E-3,2.0015053E-3,1.7452E4,5.6598197E-5,3.1323305E-3,-2.830369E-3,7.804631E-4,1.5895612E-3,-6.423995E-3,-2.8547287E-4,4.538569E-3,1.3008061E-3,-3.1195171E-3,1.3600456E-4,3.3156439E-3,-1.698196E-2,-2.6187792E-3,-2.3069913E-3,1.3431032E-3,1.4031911E-3,-2.1578893E-3,-3.5386004E-3,5.812107E-4,-5.2781517E-5,-6.5902825E-3,5.174934E-3,-9.28962E-4,-6.7870417E-3,-6.1136956E-4,-5.518346E-3,3.036717E-4,5.2622723E-4,6.450345E-3,-2.2661418E-3,1.8370287E-3,-1.40890265E-2,-4.31197E-3,6.805348E-4,-1.4319371E-3,-9.28376E-4,-5.925899E-3,6.0638143E-3,7.172203E-4,-4.4025565E-3,1.1518799E-3,9.160502E-3,4.8898644E-4,1.7405546E-3,5.2686813E-3,1.5089674E-3,7.093294E-3,3.608929E-3,-1.5733415E-3,-7.817726E-4,2.4788615E-3,5.275358E-3,5.920096E-4,1.2938416E-2,3.0058094E-3,6.5125218E-3,-1.9769752E-3,-4.2638173E-3,-1.170975E-3,-4.9012764E-3,-3.397469E-6,5.367693E-3,-1.785065E-4,-6.42457E-3,-7.250111E-4,6.4767175E-3,1.0836567E-3,-1.0796371E-3,-4.146812E-3,1.6088943E-3,-1.3741339E-3],"split_indices":[45,52,4,17,45,10,12,54,2,53,56,9,0,2,0,1,52,29,4,50,7,58,7,10,39,57,57,2,55,0,58,52,39,0,12,31,9,28,39,4,0,0,28,0,0,0,0,33,29,8,5,53,4,33,52,0,0,54,53,39,8,4,55,28,52,58,0,57,42,3,48,9,28,54,27,45,0,0,27,45,1,0,0,0,64,56,53,0,58,1,53,7,0,55,28,2,50,29,10,0,0,9,0,58,51,45,0,50,45,31,31,53,7,19,27,52,56,45,45,39,28,0,9,0,0,37,3,53,0,57,9,0,0,45,42,64,27,0,0,0,58,2,0,58,38,0,0,0,0,53,52,0,0,0,38,28,0,0,0,0,105,56,54,38,28,0,0,0,48,52,0,0,0,58,12,32,8,0,0,0,0,2,56,32,2,1,8,52,0,3,11,0,54,45,0,0,0,0,0,0,0,2,2,55,0,0,0,12,0,0,0,0,0,0,0,0,0,0,2,0,0,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.56E3,2.372E3,1.88E2,1.496E3,8.76E2,3.2E1,1.56E2,4.9E2,1.006E3,6.3E2,2.46E2,2.8E1,4E0,1.5E2,6E0,4.73E2,1.7E1,6.26E2,3.8E2,5.04E2,1.26E2,2.1E2,3.6E1,1.7E1,1.1E1,1E2,5E1,3.91E2,8.2E1,7E0,1E1,2.52E2,3.74E2,5.5E1,3.25E2,1.56E2,3.48E2,6.9E1,5.7E1,2.04E2,6E0,4E0,3.2E1,7E0,1E1,4E0,7E0,5E1,5E1,9E0,4.1E1,3.78E2,1.3E1,6.5E1,1.7E1,6E0,4E0,9.1E1,1.61E2,3.65E2,9E0,2E1,3.5E1,2E1,3.05E2,2.8E1,1.28E2,1.26E2,2.22E2,3.6E1,3.3E1,4.5E1,1.2E1,1.72E2,3.2E1,2.1E1,1.1E1,5E0,4.5E1,4E1,1E1,4E0,5E0,6E0,3.5E1,1.84E2,1.94E2,4E0,9E0,1.4E1,5.1E1,1.2E1,5E0,1.5E1,7.6E1,1.1E1,1.5E2,1.47E2,2.18E2,4E0,5E0,1.5E1,5E0,2E1,1.5E1,1.6E1,4E0,3.9E1,2.66E2,2E1,8E0,3.8E1,9E1,8.5E1,4.1E1,1.72E2,5E1,2.6E1,1E1,1.5E1,1.8E1,1.3E1,3.2E1,5E0,7E0,1.63E2,9E0,2.8E1,4E0,9E0,1.2E1,4E0,7E0,3.4E1,1.1E1,2.5E1,1.5E1,4E0,6E0,5E0,3E1,1.8E2,4E0,1.1E2,8.4E1,5E0,4E0,4E0,1E1,3.2E1,1.9E1,7E0,5E0,7E0,8E0,6.8E1,8E0,5E0,6E0,4E0,1.46E2,6.5E1,8.2E1,1.56E2,6.2E1,4E0,1.1E1,4E0,1.6E1,1E1,5E0,1.1E1,5E0,2.6E1,1.3E1,1.96E2,7E1,4E0,1.6E1,4E0,4E0,1.5E1,2.3E1,2.6E1,6.4E1,4E1,4.5E1,3.6E1,5E0,1.53E2,1.9E1,4E0,4.6E1,1.3E1,1.3E1,5E0,5E0,5E0,1E1,5E0,1.3E1,2.4E1,8E0,1.58E2,5E0,4E0,5E0,2.1E1,7E0,4E0,5E0,7E0,5E0,1.9E1,1.5E1,6E0,5E0,4E0,2.1E1,1.1E1,4E0,1E1,2E1,5.1E1,1.29E2,7.9E1,3.1E1,8E1,4E0,4E0,6E0,2.4E1,8E0,5E0,1.4E1,4E0,4E0,3.9E1,2.9E1,1.12E2,3.4E1,3.5E1,3E1,1.2E1,7E1,5E0,1.51E2,3E1,3.2E1,7E0,9E0,5E0,5E0,1.9E1,7E0,4E0,9E0,1.38E2,5.8E1,5E1,2E1,8E0,7E0,6E0,1.7E1,2.1E1,5E0,3.9E1,2.5E1,4E0,3.6E1,4.1E1,4E0,1.1E1,2.5E1,3.4E1,1.19E2,7E0,1.2E1,4E0,4.2E1,7E0,6E0,4E0,2E1,4E0,4E0,1E1,1.48E2,1.7E1,4E0,1.5E1,6E0,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"295","size_leaf_vector":"1"}},{"base_weights":[1.3650148E-3,-2.126035E-2,1.6515698E-2,-1.2144839E-2,-5.49864E-2,3.144024E-2,-2.4217438E-2,-7.2898925E-3,-7.70051E-2,-4.3959256E-2,-1.2942493E-1,1.3548526E-2,4.9981017E-2,-4.708844E-2,1.4349362E-2,-3.7970024E-3,-8.707109E-2,-2.1323311E-1,-4.348354E-2,-3.953712E-2,-1.0153657E-2,-1.7806014E-1,-2.407552E-2,3.2441836E-2,-1.3543778E-2,2.311688E-1,4.6252828E-2,-3.4937292E-2,-9.7910576E-2,2.9850747E-2,-6.430568E-2,3.7960712E-2,-9.187543E-3,-1.1273338E-1,-8.9637805E-3,1.3720779E-3,-2.0905416E-2,5.216277E-3,-6.357374E-2,-4.516043E-2,5.6074984E-2,-2.018461E-1,-3.5521437E-3,-4.8429645E-3,2.0308476E-3,5.2355647E-2,9.462655E-3,-9.212829E-3,-9.003666E-3,6.428797E-3,1.3559158E-2,7.164246E-3,5.52842E-2,-8.371602E-3,-3.0944137E-2,-6.926339E-2,-1.9318119E-1,3.5917333E-3,7.829476E-2,-5.0125187E-3,-1.3047002E-2,8.976617E-2,-4.086397E-2,5.064548E-2,-1.2367643E-2,-1.7444359E-4,-1.3440725E-1,2.8195917E-3,-3.625794E-3,4.2074956E-3,-8.134546E-2,-1.1249926E-1,-3.5072118E-2,-9.052428E-4,7.431476E-3,-4.6605016E-3,-2.3284821E-1,1.0334202E-1,3.49808E-2,2.3372795E-2,-5.6948468E-2,2.8459292E-3,-5.0448325E-2,-3.0439834E-3,6.4978404E-3,6.474627E-2,2.3237854E-2,2.5133135E-2,-3.909327E-2,-4.3736394E-2,-1.1354665E-1,-1.2391002E-2,-6.317687E-3,-3.9614774E-3,6.3330773E-3,1.0969809E-2,5.9615385E-2,2.2748316E-2,-2.6466053E-3,2.2089388E-2,1.12337455E-1,-1.2672786E-1,-4.88385E-3,7.625227E-3,3.1319637E-2,-2.4664603E-2,4.21489E-3,-3.381884E-3,-1.5213655E-1,-1.1815017E-1,-3.3887446E-2,-5.2018344E-2,-1.5980685E-2,5.126483E-2,-4.3166224E-2,-7.708699E-3,-1.48040475E-2,2.1964523E-4,1.20988876E-1,4.4131554E-3,8.869159E-2,-2.5012374E-2,4.0242575E-2,-8.3950885E-2,4.199839E-3,9.133634E-2,-4.501477E-3,-2.60966E-2,-1.1834809E-2,6.72227E-3,-9.198239E-3,5.934693E-2,1.788553E-1,-8.146101E-3,3.644035E-2,4.618908E-3,4.151499E-3,3.7300815E-3,-4.2728156E-2,-3.7138571E-3,-5.075561E-3,-7.218176E-3,-1.4495805E-3,6.2120058E-2,-2.2200033E-2,4.3450795E-2,7.463078E-3,-1.5055982E-3,2.3441196E-3,4.0679462E-2,-9.747815E-4,1.3919793E-1,3.3752833E-2,-2.8867552E-3,-9.200231E-3,-2.4580562E-2,3.9383285E-3,-1.37511E-2,5.2289024E-2,-1.9124525E-2,-1.11867554E-1,-9.480486E-2,8.894819E-3,-4.616409E-3,-9.079053E-3,-7.3458916E-3,-7.169635E-2,-3.6813493E-3,-8.622587E-4,-8.765643E-3,1.84354E-2,1.1658795E-1,-2.7613572E-3,-7.153023E-2,-1.9226339E-2,1.310119E-1,1.3306073E-3,-1.659004E-2,8.266651E-2,2.5663419E-2,1.1759711E-1,6.2503573E-3,-4.8415672E-2,9.855177E-2,3.106715E-2,-9.886215E-3,-5.1277004E-2,6.827096E-3,4.0815736E-4,-6.999885E-2,1.4053484E-3,3.384501E-2,-6.6535756E-2,8.2557485E-2,-6.6329213E-3,4.946396E-2,1.1224892E-1,5.22954E-3,1.2426598E-2,7.862335E-2,-1.5761531E-3,-3.294308E-2,3.026806E-2,-5.5505324E-2,-2.1924688E-2,1.8840434E-3,-1.825147E-3,4.590492E-3,1.0768466E-1,-8.2922235E-2,-9.651956E-3,6.956699E-2,-1.675518E-2,-5.238206E-6,4.0715197E-3,7.929274E-3,2.062315E-3,-1.2557913E-3,5.468444E-3,-3.1708535E-3,1.2257856E-3,-2.0638977E-3,1.102484E-3,4.894757E-3,3.0059947E-4,-6.1393675E-4,-4.8302137E-3,-8.059466E-3,-5.6876184E-4,-1.3282083E-2,1.1956773E-3,-6.158906E-5,2.225698E-3,-1.5037563E-3,-5.360347E-3,3.0301453E-3,-3.0592573E-3,7.982292E-3,2.5036281E-3,-2.2370701E-3,-5.820766E-3,1.0876285E-3,-3.0361714E-3,5.160576E-3,8.772013E-3,2.4266793E-4,-3.7745675E-3,7.3705073E-3,1.6263975E-3,3.159442E-3,-1.8506426E-3,7.1058366E-3,9.905283E-4,-4.272453E-4,-6.5594865E-3,1.438706E-3,6.9229635E-3,2.6913467E-3,6.955328E-4,1.0671045E-4,-4.350268E-3,-5.197513E-3,-3.6481905E-4,1.6610274E-3,-4.5614966E-4,-3.6171633E-3,3.701695E-3,-1.700457E-3,-8.186002E-3,6.447165E-3,2.1187472E-3,-1.7364405E-3,9.179243E-4,7.729361E-3,2.1855307E-3,3.7430145E-3,1.0272098E-2,2.4519523E-3,6.30321E-3,-7.794772E-4,6.663928E-3,4.4601786E-4,-3.4089184E-3,2.2578193E-4,3.525817E-3,-2.3626932E-3,-7.663452E-3,1.0283369E-3,-2.1797789E-3,3.1328816E-3,-2.7200128E-3,7.457064E-3,2.753519E-3,-1.5636919E-3,-6.032446E-3,-3.914265E-3,1.8517442E-4,2.0806475E-3,4.9305893E-3,-2.7424502E-3,4.582306E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,-1,69,71,73,75,-1,-1,-1,77,79,81,-1,-1,-1,83,85,-1,87,89,91,93,95,97,-1,99,101,103,105,-1,107,-1,-1,-1,109,111,113,-1,-1,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,-1,-1,143,-1,-1,145,147,-1,149,151,153,155,-1,157,159,161,-1,163,165,167,169,-1,171,173,-1,-1,-1,175,177,179,181,183,185,-1,187,189,191,-1,193,-1,195,197,-1,199,201,-1,-1,203,-1,205,-1,-1,207,209,211,-1,-1,-1,213,-1,215,217,-1,-1,219,-1,221,223,225,227,229,231,-1,-1,-1,233,-1,-1,-1,235,237,-1,239,241,243,-1,245,247,249,251,-1,253,255,257,-1,259,-1,-1,261,263,265,267,269,271,273,275,-1,-1,277,279,281,283,285,287,-1,-1,289,291,293,295,297,299,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.8886213E-1,3.1905958E-1,9.448663E-1,2.5765103E-1,1.7571706E-1,3.764956E-1,3.6849535E-1,2.1257094E-1,2.5102273E-1,1.2858486E-1,1.3917649E-1,2.973201E-1,3.6623788E-1,1.583767E-1,1.9212446E-1,1.653446E-1,6.313068E-2,5.546596E-1,1.4626968E-1,1.04687095E-1,0E0,3.399396E-2,5.1086538E-2,1.5600482E-1,1.7083843E-1,1.0409176E-2,1.9306087E-1,1.07602805E-1,1.2308133E-1,1.6636285E-1,3.0209538E-1,3.4668624E-1,1.24305144E-1,5.6355596E-2,4.1526128E-2,0E0,0E0,0E0,1.17154405E-1,1.1833477E-1,7.8310214E-2,2.8771996E-2,0E0,0E0,0E0,1.5850109E-1,1.4913715E-1,1.164998E-1,0E0,0E0,0E0,1.3085133E-1,1.3343048E-1,0E0,9.6034616E-2,4.025063E-2,8.881092E-3,8.050021E-2,1.0827923E-1,2.9161546E-2,0E0,7.542372E-2,1.0442043E-1,6.024623E-2,1.2612328E-1,0E0,1.3362497E-2,0E0,0E0,0E0,6.2369972E-2,2.7176854E-1,1.1262652E-1,0E0,0E0,0E0,9.112895E-3,7.97883E-2,2.2617875E-1,1.0956744E-1,1.136628E-1,1.1874299E-1,2.3440191E-1,1.7274483E-1,0E0,2.007122E-1,2.592395E-1,3.171506E-2,7.843742E-2,3.1554207E-2,3.2394826E-2,0E0,0E0,1.00208074E-1,0E0,0E0,5.443971E-2,1.9303821E-2,0E0,1.1619944E-2,7.551283E-2,2.8941706E-2,4.3154247E-2,0E0,2.8127097E-2,1.6936505E-1,1.2318231E-1,0E0,9.709358E-3,1.7618835E-2,1.0639481E-2,1.6843534E-1,0E0,1.03866234E-1,9.7084224E-2,0E0,0E0,0E0,3.3395886E-2,1.4764921E-1,8.807513E-2,1.2900794E-1,4.990442E-2,7.917604E-2,0E0,5.1625006E-2,6.5481365E-2,1.18603334E-1,0E0,9.473743E-2,0E0,1.6645837E-1,5.1700264E-2,0E0,1.5567796E-1,2.122922E-2,0E0,0E0,4.5831323E-2,0E0,1.8869504E-2,0E0,0E0,4.6689287E-2,4.872619E-2,5.9342586E-2,0E0,0E0,0E0,1.6650682E-2,0E0,4.6898544E-2,5.1777314E-2,0E0,0E0,4.165504E-2,0E0,1.0780118E-2,4.167776E-2,1.7643458E-1,1.024687E-1,2.5002378E-1,9.1395706E-2,0E0,0E0,0E0,1.1574011E-2,0E0,0E0,0E0,4.7423385E-2,1.7822325E-2,0E0,7.419285E-2,1.375441E-1,2.5104344E-2,0E0,9.013325E-2,5.6942508E-2,4.2385813E-2,7.9006374E-2,0E0,1.0127923E-1,3.228493E-2,3.3257023E-2,0E0,3.8414855E-2,0E0,0E0,2.9102795E-2,5.2624065E-2,9.085851E-2,8.613481E-2,2.0653337E-2,5.6884754E-2,1.6128546E-1,1.6232604E-1,0E0,0E0,5.8413833E-2,1.0048643E-1,1.3903647E-2,1.3169698E-2,8.043298E-2,6.45407E-2,0E0,0E0,3.4283124E-2,1.3624944E-2,1.7692596E-2,5.075259E-2,1.7027706E-2,1.2366128E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,39,39,40,40,41,41,45,45,46,46,47,47,51,51,52,52,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,66,66,70,70,71,71,72,72,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,90,90,93,93,96,96,97,97,99,99,100,100,101,101,102,102,104,104,105,105,106,106,108,108,109,109,110,110,111,111,113,113,114,114,118,118,119,119,120,120,121,121,122,122,123,123,125,125,126,126,127,127,129,129,131,131,132,132,134,134,135,135,138,138,140,140,143,143,144,144,145,145,149,149,151,151,152,152,155,155,157,157,158,158,159,159,160,160,161,161,162,162,166,166,170,170,171,171,173,173,174,174,175,175,177,177,178,178,179,179,180,180,182,182,183,183,184,184,186,186,189,189,190,190,191,191,192,192,193,193,194,194,195,195,196,196,199,199,200,200,201,201,202,202,203,203,204,204,207,207,208,208,209,209,210,210,211,211,212,212],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,-1,70,72,74,76,-1,-1,-1,78,80,82,-1,-1,-1,84,86,-1,88,90,92,94,96,98,-1,100,102,104,106,-1,108,-1,-1,-1,110,112,114,-1,-1,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,-1,-1,144,-1,-1,146,148,-1,150,152,154,156,-1,158,160,162,-1,164,166,168,170,-1,172,174,-1,-1,-1,176,178,180,182,184,186,-1,188,190,192,-1,194,-1,196,198,-1,200,202,-1,-1,204,-1,206,-1,-1,208,210,212,-1,-1,-1,214,-1,216,218,-1,-1,220,-1,222,224,226,228,230,232,-1,-1,-1,234,-1,-1,-1,236,238,-1,240,242,244,-1,246,248,250,252,-1,254,256,258,-1,260,-1,-1,262,264,266,268,270,272,274,276,-1,-1,278,280,282,284,286,288,-1,-1,290,292,294,296,298,300,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2485715E2,2.0304577E-1,3.0161016E7,1.2737473E1,2.589369E7,8.027061E2,1.0052E4,1E0,8E0,7.780377E6,1.631E4,3.3135505E0,3.4955364E7,1.0828989E8,2.3580047E10,7E0,2.5544708E0,1.3E1,1.3E1,3.426E3,-1.0153657E-2,3.6E1,1.9701008E5,8E0,5.7835355E6,2.653986E6,8.079531E4,3E0,1.515252E10,9.256843E3,1.5005797E4,1.59E2,5E-1,9.6590906E-1,1.6908121E1,1.3720779E-3,-2.0905416E-2,5.216277E-3,3.114E3,4.2378342E-1,1.3989231E5,1.4410892E-5,-3.5521437E-3,-4.8429645E-3,2.0308476E-3,1E0,1.7291142E7,1.2673605E7,-9.003666E-3,6.428797E-3,1.3559158E-2,9.492855E8,2.5506506E0,-8.371602E-3,1E1,4.6263345E-2,3.088934E6,4.5505118E2,2.909019E3,3.21E2,-1.3047002E-2,3.830986E-1,4.0208E4,9E0,1.8122449E2,-1.7444359E-4,6.763312E7,2.8195917E-3,-3.625794E-3,4.2074956E-3,2.4559366E1,5.1100562E5,2.1924436E5,-9.052428E-4,7.431476E-3,-4.6605016E-3,3.9712732E7,3.5902756E-6,3.114E3,1.646E3,1E0,1.116942E-3,3.4306667E1,1.4521701E2,6.4978404E-3,8.791777E3,6.972973E0,1E0,3.5734247E2,3.689E3,1.5265896E8,-1.2391002E-2,-6.317687E-3,3.307766E6,6.3330773E-3,1.0969809E-2,6.318545E8,3.5791788E0,-2.6466053E-3,1.0066326E8,1.5445488E8,3.2704837E2,1.22E2,7.625227E-3,1.1E1,1E0,8E0,-3.381884E-3,1.752E3,1.588909E5,8.0146375E2,8.085719E-2,-1.5980685E-2,2.0131812E7,1.845044E10,-7.708699E-3,-1.48040475E-2,2.1964523E-4,1E0,7.246204E2,4.5638318E2,7.598633E8,9.577834E7,3.7931025E6,4.199839E-3,1.23E2,1.2030203E7,2.057307E6,-1.1834809E-2,1.4685535E9,-9.198239E-3,1.6368244E0,3.7137984E7,-8.146101E-3,3.7129E4,1.7994144E0,4.151499E-3,3.7300815E-3,6.071E3,-3.7138571E-3,1E0,-7.218176E-3,-1.4495805E-3,2.9049585E3,6.473E3,1.0040462E8,7.463078E-3,-1.5055982E-3,2.3441196E-3,1.85221E5,-9.747815E-4,7.128E3,1.0012501E7,-2.8867552E-3,-9.200231E-3,3.3743455E0,3.9383285E-3,1.4642603E2,4.3440155E6,1.4813511E6,5.2E1,2.4576474E8,1.9E1,-4.616409E-3,-9.079053E-3,-7.3458916E-3,1.03270106E6,-3.6813493E-3,-8.622587E-4,-8.765643E-3,5.9572783E0,1.7232166E8,-2.7613572E-3,1E0,5.188105E10,1.068E3,1.3306073E-3,1.8053533E8,4.6330696E7,7.43361E0,1.531154E8,6.2503573E-3,1.4E1,3.3870136E7,4.9008E4,-9.886215E-3,3.474851E8,6.827096E-3,4.0815736E-4,8.588109E0,8.347515E7,2.2690948E2,1.837E3,2.21875E0,7.06699E8,7.696629E0,2.0676143E3,5.22954E-3,1.2426598E-2,3.15856E0,4.5505118E2,4.67E2,1.8137958E6,7.341606E1,8.3333335E-4,1.8840434E-3,-1.825147E-3,1.2355374E3,1.3298E4,1.195E3,1.5391705E0,3.159399E6,3.898923E0,-5.238206E-6,4.0715197E-3,7.929274E-3,2.062315E-3,-1.2557913E-3,5.468444E-3,-3.1708535E-3,1.2257856E-3,-2.0638977E-3,1.102484E-3,4.894757E-3,3.0059947E-4,-6.1393675E-4,-4.8302137E-3,-8.059466E-3,-5.6876184E-4,-1.3282083E-2,1.1956773E-3,-6.158906E-5,2.225698E-3,-1.5037563E-3,-5.360347E-3,3.0301453E-3,-3.0592573E-3,7.982292E-3,2.5036281E-3,-2.2370701E-3,-5.820766E-3,1.0876285E-3,-3.0361714E-3,5.160576E-3,8.772013E-3,2.4266793E-4,-3.7745675E-3,7.3705073E-3,1.6263975E-3,3.159442E-3,-1.8506426E-3,7.1058366E-3,9.905283E-4,-4.272453E-4,-6.5594865E-3,1.438706E-3,6.9229635E-3,2.6913467E-3,6.955328E-4,1.0671045E-4,-4.350268E-3,-5.197513E-3,-3.6481905E-4,1.6610274E-3,-4.5614966E-4,-3.6171633E-3,3.701695E-3,-1.700457E-3,-8.186002E-3,6.447165E-3,2.1187472E-3,-1.7364405E-3,9.179243E-4,7.729361E-3,2.1855307E-3,3.7430145E-3,1.0272098E-2,2.4519523E-3,6.30321E-3,-7.794772E-4,6.663928E-3,4.4601786E-4,-3.4089184E-3,2.2578193E-4,3.525817E-3,-2.3626932E-3,-7.663452E-3,1.0283369E-3,-2.1797789E-3,3.1328816E-3,-2.7200128E-3,7.457064E-3,2.753519E-3,-1.5636919E-3,-6.032446E-3,-3.914265E-3,1.8517442E-4,2.0806475E-3,4.9305893E-3,-2.7424502E-3,4.582306E-4],"split_indices":[52,27,45,54,45,52,2,74,17,28,9,53,7,45,5,3,53,0,10,2,0,3,33,17,47,1,28,3,5,4,4,11,53,53,58,0,0,0,29,27,28,39,0,0,0,16,45,45,0,0,0,32,41,0,3,57,1,56,32,8,0,56,9,3,52,0,7,0,0,0,56,47,28,0,0,0,45,38,29,2,105,41,53,58,0,52,56,16,52,2,45,0,0,1,0,0,44,54,0,7,7,4,8,0,3,14,3,0,29,47,4,42,0,12,31,0,0,0,74,4,55,31,7,48,0,3,5,12,0,31,0,38,1,0,9,57,0,0,2,0,8,0,0,4,9,51,0,0,0,9,0,10,45,0,0,54,0,4,45,28,10,5,3,0,0,0,50,0,0,0,53,7,0,104,31,29,0,7,5,56,12,0,8,7,9,0,7,0,0,56,7,33,0,53,7,56,4,0,0,38,56,10,28,58,57,0,0,52,9,0,53,32,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.591E3,1.039E3,1.552E3,8.19E2,2.2E2,1.136E3,4.16E2,7.63E2,5.6E1,1.93E2,2.7E1,5.79E2,5.57E2,2.61E2,1.55E2,7.32E2,3.1E1,1E1,4.6E1,1.89E2,4E0,1.8E1,9E0,3.41E2,2.38E2,1E1,5.47E2,2.12E2,4.9E1,1.3E2,2.5E1,8.3E1,6.49E2,2.3E1,8E0,5E0,5E0,5E0,4.1E1,1.79E2,1E1,1.4E1,4E0,4E0,5E0,1.82E2,1.59E2,2.33E2,5E0,4E0,6E0,1.03E2,4.44E2,5E0,2.07E2,3.9E1,1E1,8.5E1,4.5E1,2E1,5E0,5E1,3.3E1,3.2E1,6.17E2,4E0,1.9E1,4E0,4E0,4E0,3.7E1,2.2E1,1.57E2,6E0,4E0,4E0,1E1,4.5E1,1.37E2,1.32E2,2.7E1,1.81E2,5.2E1,9.6E1,7E0,3.42E2,1.02E2,2.6E1,1.81E2,2.6E1,1.3E1,4E0,6E0,8.1E1,4E0,4E0,4.1E1,1.3E1,7E0,1.3E1,3.7E1,9E0,2.4E1,4E0,2.8E1,3.54E2,2.63E2,5E0,1.4E1,2E1,1.7E1,1.8E1,4E0,1.3E1,1.44E2,6E0,4E0,7E0,3.8E1,8.8E1,4.9E1,3.4E1,9.8E1,2.3E1,4E0,1.3E1,1.68E2,4.7E1,5E0,9.2E1,4E0,3.28E2,1.4E1,6E0,9.6E1,2E1,6E0,5E0,1.76E2,1.4E1,1.2E1,9E0,4E0,1.7E1,6.4E1,3.6E1,5E0,4E0,9E0,9E0,4E0,2.7E1,1E1,5E0,4E0,2E1,4E0,9E0,1.9E1,3.34E2,2E1,1.1E1,2.52E2,6E0,8E0,1.1E1,9E0,4E0,1.3E1,6E0,1.2E1,8E0,5E0,6.5E1,7.9E1,3.4E1,4E0,7E1,1.8E1,1.6E1,3.3E1,4E0,3E1,1.2E1,8.6E1,4E0,1.9E1,8E0,5E0,1.3E1,1.55E2,1.9E1,2.8E1,1.3E1,7.9E1,2.78E2,5E1,8E0,6E0,4.5E1,5.1E1,8E0,1.2E1,1.08E2,6.8E1,5E0,7E0,8E0,9E0,1E1,5.4E1,2.5E1,1.1E1,5E0,4E0,2.2E1,5E0,6E0,4E0,1.1E1,9E0,5E0,4E0,9E0,1E1,3.08E2,2.6E1,1.3E1,7E0,4E0,7E0,1.97E2,5.5E1,5E0,4E0,8E0,4E0,4E0,4E0,4.2E1,2.3E1,4E1,3.9E1,2.3E1,1.1E1,5.2E1,1.8E1,7E0,1.1E1,1E1,6E0,2.6E1,7E0,2.1E1,9E0,5E0,7E0,3.6E1,5E1,8E0,1.1E1,8E0,5E0,3.8E1,1.17E2,5E0,1.4E1,2.2E1,6E0,5E0,8E0,3.7E1,4.2E1,1.3E1,2.65E2,3.7E1,1.3E1,2.9E1,1.6E1,4.7E1,4E0,4E0,4E0,8E0,4E0,1.01E2,7E0,2.3E1,4.5E1,4E0,4E0,4E0,5E0,5E0,5E0,8E0,4.6E1,1.4E1,1.1E1,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"301","size_leaf_vector":"1"}},{"base_weights":[3.401948E-5,1.0091209E-2,-2.803433E-2,2.4452463E-3,5.7111867E-2,-5.208378E-2,-6.196773E-3,-7.6650046E-3,1.8032013E-2,2.1310033E-1,4.887091E-2,-4.463651E-2,-1.430045E-1,9.577254E-3,-8.981905E-3,-1.8295618E-2,1.0352759E-2,-8.18362E-3,2.8824333E-2,6.92129E-3,1.3842091E-2,1.1834595E-1,3.6422193E-2,-8.995227E-2,-3.0453661E-2,-2.80663E-1,-4.169852E-2,2.9181074E-3,-4.7727946E-2,-7.6035624E-3,-6.3049786E-2,8.210519E-3,6.9070314E-3,-1.3955073E-2,5.404132E-3,3.1381622E-2,-6.9616167E-3,8.9989754E-4,1.404215E-1,2.7269417E-2,1.2475594E-1,-1.0555566E-2,-7.955747E-2,-1.8908694E-2,-9.423536E-2,-8.042567E-3,-1.6670855E-2,2.3852503E-3,-7.856017E-2,-1.8475521E-3,6.3700792E-3,-2.2155078E-2,-1.2717472E-1,-2.2070952E-2,1.3956807E-2,-5.1406108E-2,-1.0974455E-2,2.0059077E-2,-1.719166E-2,-4.8648817E-3,-1.1362286E-1,1.9952636E-2,6.440187E-2,8.6362725E-3,8.223799E-2,4.1839838E-2,-1.8831264E-2,6.0533468E-2,1.0669031E-2,-6.508937E-2,-8.27269E-3,1.55883385E-2,-3.498634E-2,-1.8229759E-1,-3.3912335E-2,-7.0866533E-3,-1.106575E-3,-2.1089379E-2,1.697414E-2,-6.708841E-2,2.7841317E-2,-1.6247597E-1,5.608364E-4,-8.94772E-3,-1.07804425E-1,4.8339043E-2,-1.7342573E-2,-2.05142E-1,-3.773916E-2,7.5721435E-2,1.0249801E-2,-1.14211E-1,-2.755564E-3,-1.4721324E-2,8.238989E-2,-1.22825345E-2,1.6374926E-3,2.4790658E-2,-7.200509E-2,9.24254E-2,3.623674E-2,1.5092685E-3,5.5454304E-3,2.8964864E-2,9.0328775E-2,1.9124202E-2,-1.6497445E-1,4.500512E-3,-2.3681698E-4,5.8096885E-3,-9.891166E-2,1.03657536E-1,-1.1689749E-2,-5.1241327E-2,1.8427724E-2,-2.9186625E-3,-1.2223475E-2,-4.8152492E-3,-4.467925E-3,-1.2290267E-2,-1.3802296E-1,9.2207365E-2,2.84253E-3,-2.148714E-2,-8.7963484E-2,6.3021414E-2,-3.3112288E-2,-1.1083612E-2,-3.7677016E-3,-2.003082E-2,2.655468E-2,-1.2793033E-1,2.6735387E-4,5.7109073E-2,-2.186433E-3,1.0032543E-2,-4.875086E-2,-4.472748E-3,-1.3977723E-2,-1.0543278E-1,-2.2521716E-2,3.658103E-2,1.3068864E-1,5.7571614E-4,7.127676E-2,-1.4287325E-3,-1.4274512E-1,5.418312E-3,-7.086838E-2,-8.410836E-2,-4.62227E-3,6.9410764E-3,2.893151E-2,3.617004E-2,-2.1114228E-2,-1.5888045E-2,2.8136749E-2,1.413247E-1,4.9806416E-2,-2.5842132E-2,9.31416E-2,2.1141699E-2,7.606569E-3,1.434018E-1,3.1610213E-2,-1.9674234E-2,8.251332E-2,-1.2173275E-2,-3.5943096E-3,3.5342716E-2,-3.4213204E-2,-1.3247144E-1,-1.9020472E-2,-3.604064E-5,1.4167905E-1,-3.253502E-2,7.286031E-2,-6.7438155E-2,-6.8001606E-4,-3.5597016E-3,5.128587E-2,2.1213975E-3,-1.1815786E-3,-2.1857545E-3,-7.4571915E-2,-8.989608E-3,-3.4241774E-3,-3.0009664E-4,1.1566967E-1,3.95457E-2,-1.0299585E-2,3.9049936E-4,-3.2015368E-3,-6.374055E-3,-4.768712E-2,9.692327E-2,2.6593173E-6,-3.7598019E-3,-1.3188993E-4,-6.181067E-4,-5.705973E-3,-7.732726E-4,2.823509E-3,-7.3985783E-3,-2.1968705E-3,3.2132107E-3,-3.8313896E-3,4.294567E-3,-3.784428E-4,-7.16302E-3,-1.6285173E-3,-2.7320846E-3,-1.1822619E-2,4.784235E-4,-3.021712E-3,-4.7087995E-4,2.666841E-3,9.541849E-3,2.7020422E-3,3.3811734E-3,-4.2388195E-4,1.3475397E-3,7.061741E-3,-1.1001666E-2,-3.3498784E-3,9.7107026E-4,-2.1433067E-3,7.2937936E-4,-8.811456E-3,-9.792972E-4,-1.1023252E-2,6.73419E-5,-5.567389E-3,3.1150978E-3,-8.468951E-4,2.7344981E-3,-5.7340905E-5,-5.343015E-3,6.096548E-4,3.1812508E-3,-5.7029445E-4,4.081217E-3,8.399263E-3,-3.3027574E-3,3.4339617E-3,1.8535258E-3,-3.962217E-3,5.4766424E-3,-7.1922125E-4,5.7507114E-4,3.1605074E-3,8.682724E-3,2.0904532E-3,3.4095803E-3,-3.1287298E-3,-5.4102195E-3,1.0391866E-3,5.4633436E-3,3.4938988E-4,1.9884032E-4,3.9201537E-3,4.805199E-4,-3.5576553E-3,-7.5925067E-3,-6.2324316E-4,2.223325E-3,-2.6079654E-3,9.558794E-3,3.4197937E-3,-3.0228388E-3,1.5086792E-3,5.406365E-3,7.935358E-4,-5.1033366E-3,-1.9140157E-3,1.2180306E-3,-3.606292E-3,5.06704E-3,-8.578674E-4,3.0493995E-4,-3.3788395E-3,-9.884604E-3,-1.0708346E-3,3.751693E-3,8.14072E-3,3.0529196E-3,3.3280405E-4,-2.848628E-3,2.592464E-4,-8.6752494E-4,-4.633648E-3,2.4244129E-3,6.5696053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,55,57,-1,59,-1,61,-1,-1,63,65,67,-1,69,71,73,-1,-1,-1,75,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,101,103,105,107,-1,109,-1,111,113,115,117,-1,-1,119,121,123,125,127,-1,129,131,133,135,137,139,141,143,145,147,149,151,-1,-1,153,155,157,159,-1,-1,161,163,165,167,-1,-1,169,171,173,175,177,179,-1,-1,-1,181,183,185,187,189,191,193,195,197,-1,-1,199,201,203,-1,205,-1,207,209,-1,-1,211,213,215,217,219,221,-1,223,225,227,229,231,-1,233,235,237,-1,239,241,243,245,247,249,-1,251,253,255,257,-1,-1,259,261,263,265,-1,267,269,271,273,275,-1,277,-1,-1,279,281,-1,-1,-1,283,285,287,-1,-1,-1,289,291,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.2661173E-1,6.811473E-1,3.5627526E-1,2.5716117E-1,3.302415E-1,2.1232373E-1,1.9846176E-1,1.8981387E-1,1.818754E-1,2.3607612E-2,2.1450299E-1,1.900726E-1,3.2299194E-1,0E0,1.6289993E-1,2.9763484E-1,9.967852E-2,1.2774031E-1,2.0002538E-1,0E0,0E0,8.1639946E-2,1.712338E-1,7.2716355E-2,1.6758034E-1,9.713054E-3,5.3201236E-2,1.6111493E-1,1.6581202E-1,1.5751684E-1,2.0749745E-1,1.096003E-1,0E0,1.6252711E-1,0E0,1.6816226E-1,0E0,0E0,4.4260025E-2,1.3311405E-1,1.01091474E-1,0E0,7.259521E-2,1.09253064E-1,1.7974418E-1,0E0,0E0,0E0,3.6119856E-2,9.524487E-2,0E0,1.4564413E-1,9.953678E-2,3.3881566E-1,2.1929528E-1,2.2968245E-1,0E0,1.346337E-1,1.6215262E-1,1.4397648E-1,2.9833794E-1,1.5082437E-1,8.7418735E-2,0E0,1.6091213E-2,9.102774E-2,2.6940778E-1,2.5490317E-2,0E0,1.4205481E-1,0E0,1.5256037E-1,1.1728908E-1,9.382403E-2,4.0027574E-2,0E0,0E0,1.3206217E-1,1.4113906E-1,3.0475527E-2,6.8561755E-2,6.59489E-2,0E0,1.0401629E-1,9.177843E-2,8.084512E-2,9.2488274E-2,5.259019E-2,1.05788484E-1,7.48225E-2,1.2511952E-1,3.2563612E-2,5.7172053E-2,1.0472274E-1,4.7349803E-2,0E0,0E0,1.6739951E-1,4.2768505E-1,1.1175132E-1,2.1082973E-1,0E0,0E0,1.1250307E-1,9.207454E-2,9.753993E-2,5.5654258E-2,0E0,0E0,2.4849132E-2,1.0472554E-1,5.9631795E-2,8.873149E-2,8.430421E-2,9.793738E-2,0E0,0E0,0E0,1.5403629E-2,7.661474E-2,1.3870031E-2,4.9085215E-2,5.4913674E-2,1.5475288E-2,3.0602917E-2,4.2689607E-2,1.5208607E-2,0E0,0E0,1.4357743E-1,8.018806E-2,5.098462E-2,0E0,9.063864E-2,0E0,7.81745E-2,7.176292E-2,0E0,0E0,1.135619E-1,1.06558606E-1,1.8270142E-2,5.894059E-2,1.1240833E-1,8.620091E-2,0E0,4.9721166E-2,6.313216E-2,1.01736754E-1,1.6298586E-1,8.431277E-2,0E0,1.638634E-2,1.7733794E-1,1.8428354E-1,0E0,1.8994896E-2,2.7788818E-2,7.471152E-2,1.0050689E-1,5.6463182E-2,4.5514394E-2,0E0,4.2080015E-2,5.949235E-2,9.189937E-2,2.9624797E-2,0E0,0E0,1.5913635E-2,1.5275135E-2,6.355649E-2,2.9714443E-2,0E0,2.6884958E-2,7.198769E-2,1.841753E-2,7.4929565E-2,4.8336167E-2,0E0,8.408798E-2,0E0,0E0,5.7904463E-2,1.0710585E-1,0E0,0E0,0E0,2.0369932E-2,2.085669E-2,6.1066292E-2,0E0,0E0,0E0,1.6148869E-2,1.5364654E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,38,38,39,39,40,40,42,42,43,43,44,44,48,48,49,49,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,64,64,65,65,66,66,67,67,69,69,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,97,97,98,98,99,99,100,100,103,103,104,104,105,105,106,106,109,109,110,110,111,111,112,112,113,113,114,114,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,126,126,129,129,130,130,131,131,133,133,135,135,136,136,139,139,140,140,141,141,142,142,143,143,144,144,146,146,147,147,148,148,149,149,150,150,152,152,153,153,154,154,156,156,157,157,158,158,159,159,160,160,161,161,163,163,164,164,165,165,166,166,169,169,170,170,171,171,172,172,174,174,175,175,176,176,177,177,178,178,180,180,183,183,184,184,188,188,189,189,190,190,194,194,195,195],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,56,58,-1,60,-1,62,-1,-1,64,66,68,-1,70,72,74,-1,-1,-1,76,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,102,104,106,108,-1,110,-1,112,114,116,118,-1,-1,120,122,124,126,128,-1,130,132,134,136,138,140,142,144,146,148,150,152,-1,-1,154,156,158,160,-1,-1,162,164,166,168,-1,-1,170,172,174,176,178,180,-1,-1,-1,182,184,186,188,190,192,194,196,198,-1,-1,200,202,204,-1,206,-1,208,210,-1,-1,212,214,216,218,220,222,-1,224,226,228,230,232,-1,234,236,238,-1,240,242,244,246,248,250,-1,252,254,256,258,-1,-1,260,262,264,266,-1,268,270,272,274,276,-1,278,-1,-1,280,282,-1,-1,-1,284,286,288,-1,-1,-1,290,292,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9173258E-4,1.5445492E3,1E0,4.5168175E5,6.737099E7,3.0996E4,4.107E3,3.386E3,1E0,1.0119178E10,2.5659466E1,1.04E3,6.058777E6,9.577254E-3,6.747114E7,7.2488395E6,2.03148E2,3.422351E6,1.9162654E7,6.92129E-3,1.3842091E-2,1.0937254E5,3.832274E2,8.494517E5,6.2E1,7.3946733E0,2.7772428E5,6.4683E4,4.5162E4,8.87E2,4.1585636E1,1.8071064E7,6.9070314E-3,1.445113E6,5.404132E-3,5.862126E2,-6.9616167E-3,8.9989754E-4,1.7398448E7,9.496754E6,3.6282136E0,-1.0555566E-2,2.0724944E7,1.1662405E9,4.179E3,-8.042567E-3,-1.6670855E-2,2.3852503E-3,4.7519747E5,5.8018835E2,6.3700792E-3,1.3664E4,8.755996E-1,2.4E1,8.581E3,9.073025E6,-1.0974455E-2,1E0,1.9473622E7,1.038946E6,5.2921E4,4.597929E6,5.6202265E6,8.6362725E-3,1.2E1,8.61491E3,3.291238E7,3.2475834E7,1.0669031E-2,2.8373447E3,-8.27269E-3,2.4626505E0,2.9E1,3.9343938E-1,1.1492701E6,-7.0866533E-3,-1.106575E-3,5.3084288E8,2.9600988E5,2.7864855E11,5.2E1,1.0526E4,5.608364E-4,1.365493E6,6.067634E5,3.8003298E6,2.3449652E0,8.0391425E6,3.5476844E7,7.298614E2,1.5435694E9,4.3978744E0,1E0,5.798305E1,1.143316E6,-1.22825345E-2,1.6374926E-3,1E0,2.148972E5,3.824611E0,6.088569E6,1.5092685E-3,5.5454304E-3,3.7253947E5,2.8159826E0,2.4131306E5,1.6E1,4.500512E-3,-2.3681698E-4,8.595346E-1,1.69383E5,1.21E3,3.44875E5,4.832493E-1,3.7382904E7,-2.9186625E-3,-1.2223475E-2,-4.8152492E-3,3.940884E5,2.74872E5,4.459525E5,7.483E3,7.667683E6,4.1E2,1.3740173E5,1.431413E-1,4.8916136E7,-1.1083612E-2,-3.7677016E-3,4.22088E0,1.6983333E2,3.557179E1,2.6735387E-4,4.70092E1,-2.186433E-3,1.59E2,1.04E3,-4.472748E-3,-1.3977723E-2,1.9044118E0,1.470375E1,3.2485715E2,3.844034E4,3.2829E4,2.1143505E6,-1.4287325E-3,1.7921995E3,1.4400111E10,9.214292E7,8E0,1.4964847E3,6.9410764E-3,1.2727361E6,2E0,1.9826213E0,-1.5888045E-2,5.7835355E6,1.3325451E1,4.020408E0,2.185321E0,5.685916E8,2.26E2,7.606569E-3,2.0620978E7,6.13912E5,7.27E2,1.2677199E7,-1.2173275E-2,-3.5943096E-3,6.2539073E-4,9.232009E2,5E0,9.18E2,-3.604064E-5,1.0894445E-2,2.7538432E8,4.96576E5,1.1508651E1,1.7897441E0,-3.5597016E-3,3.6148317E-2,2.1213975E-3,-1.1815786E-3,2.065904E6,1E0,-8.989608E-3,-3.4241774E-3,-3.0009664E-4,7.469927E6,4.4844124E-1,1.0598885E11,3.9049936E-4,-3.2015368E-3,-6.374055E-3,3.3487454E11,2.9964134E9,2.6593173E-6,-3.7598019E-3,-1.3188993E-4,-6.181067E-4,-5.705973E-3,-7.732726E-4,2.823509E-3,-7.3985783E-3,-2.1968705E-3,3.2132107E-3,-3.8313896E-3,4.294567E-3,-3.784428E-4,-7.16302E-3,-1.6285173E-3,-2.7320846E-3,-1.1822619E-2,4.784235E-4,-3.021712E-3,-4.7087995E-4,2.666841E-3,9.541849E-3,2.7020422E-3,3.3811734E-3,-4.2388195E-4,1.3475397E-3,7.061741E-3,-1.1001666E-2,-3.3498784E-3,9.7107026E-4,-2.1433067E-3,7.2937936E-4,-8.811456E-3,-9.792972E-4,-1.1023252E-2,6.73419E-5,-5.567389E-3,3.1150978E-3,-8.468951E-4,2.7344981E-3,-5.7340905E-5,-5.343015E-3,6.096548E-4,3.1812508E-3,-5.7029445E-4,4.081217E-3,8.399263E-3,-3.3027574E-3,3.4339617E-3,1.8535258E-3,-3.962217E-3,5.4766424E-3,-7.1922125E-4,5.7507114E-4,3.1605074E-3,8.682724E-3,2.0904532E-3,3.4095803E-3,-3.1287298E-3,-5.4102195E-3,1.0391866E-3,5.4633436E-3,3.4938988E-4,1.9884032E-4,3.9201537E-3,4.805199E-4,-3.5576553E-3,-7.5925067E-3,-6.2324316E-4,2.223325E-3,-2.6079654E-3,9.558794E-3,3.4197937E-3,-3.0228388E-3,1.5086792E-3,5.406365E-3,7.935358E-4,-5.1033366E-3,-1.9140157E-3,1.2180306E-3,-3.606292E-3,5.06704E-3,-8.578674E-4,3.0493995E-4,-3.3788395E-3,-9.884604E-3,-1.0708346E-3,3.751693E-3,8.14072E-3,3.0529196E-3,3.3280405E-4,-2.848628E-3,2.592464E-4,-8.6752494E-4,-4.633648E-3,2.4244129E-3,6.5696053E-3],"split_indices":[27,52,16,28,7,9,12,2,102,19,56,2,51,0,45,45,56,9,47,0,0,33,56,45,8,57,28,2,10,2,53,45,0,9,0,52,0,0,50,47,53,0,45,5,2,0,0,0,28,52,0,2,42,8,29,45,0,102,45,9,1,47,47,0,3,4,45,50,0,47,0,58,8,27,50,0,0,7,47,31,3,9,0,9,47,51,53,45,7,52,7,53,74,47,9,0,0,105,33,54,45,0,0,28,35,33,8,0,0,27,1,2,1,39,12,0,0,0,32,29,28,9,45,0,33,38,48,0,0,53,55,58,0,58,0,11,2,0,0,53,56,52,48,9,50,0,4,5,7,8,4,0,28,17,56,0,47,58,54,41,7,8,0,29,2,0,9,0,0,39,33,8,0,0,38,7,1,56,41,0,38,0,0,28,100,0,0,0,12,57,31,0,0,0,31,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.572E3,1.894E3,6.78E2,1.63E3,2.64E2,3.22E2,3.56E2,9.89E2,6.41E2,1.2E1,2.52E2,2.99E2,2.3E1,4E0,3.52E2,6.22E2,3.67E2,1.87E2,4.54E2,7E0,5E0,3.7E1,2.15E2,7E1,2.29E2,9E0,1.4E1,2.7E2,8.2E1,5.03E2,1.19E2,3.62E2,5E0,1.79E2,8E0,4.48E2,6E0,7E0,3E1,1.96E2,1.9E1,4E0,6.6E1,1.95E2,3.4E1,4E0,5E0,4E0,1E1,2.61E2,9E0,6.3E1,1.9E1,3.01E2,2.02E2,1.12E2,7E0,2.47E2,1.15E2,1.65E2,1.4E1,3.34E2,1.14E2,1.8E1,1.2E1,1.49E2,4.7E1,1.2E1,7E0,5.8E1,8E0,6.2E1,1.33E2,1.3E1,2.1E1,4E0,6E0,1.29E2,1.32E2,3.3E1,3E1,1.5E1,4E0,2.62E2,3.9E1,9.6E1,1.06E2,8E0,1.04E2,3.6E1,2.11E2,1.4E1,1.01E2,1.49E2,1.6E1,7E0,7E0,3.18E2,1.6E1,5.6E1,5.8E1,5E0,7E0,1.19E2,3E1,3.8E1,9E0,8E0,4E0,1.9E1,3.9E1,1.4E1,4.8E1,1.02E2,3.1E1,5E0,8E0,6E0,1.5E1,1.21E2,8E0,2E1,1.12E2,1.1E1,2.2E1,1.9E1,1.1E1,8E0,7E0,2E2,6.2E1,3.3E1,6E0,8.8E1,8E0,5.7E1,4.9E1,4E0,4E0,1.8E1,8.6E1,2.2E1,1.4E1,1.83E2,2.8E1,4E0,1E1,9.1E1,1E1,1.8E1,1.31E2,7E0,9E0,2.55E2,6.3E1,4E0,1.2E1,2.5E1,3.1E1,2.8E1,3E1,1.13E2,6E0,1.5E1,1.5E1,2.4E1,1.4E1,4E0,5E0,1.1E1,8E0,2.7E1,1.2E1,4E0,1E1,3.9E1,9E0,7.7E1,2.5E1,8E0,2.3E1,4E0,1.1E1,1.05E2,1.6E1,4E0,4E0,4E0,1.6E1,2.9E1,8.3E1,7E0,4E0,1E1,1.2E1,1.2E1,7E0,4E0,7E0,1.86E2,1.4E1,2.6E1,3.6E1,2.6E1,7E0,8.4E1,4E0,1E1,4.7E1,6E0,4.3E1,1.4E1,4E0,4.7E1,3.9E1,6E0,1.6E1,7E0,7E0,2.1E1,1.62E2,1.8E1,1E1,4E0,6E0,7.1E1,2E1,6E0,4E0,1.3E1,5E0,1.25E2,6E0,5E0,4E0,1.7E2,8.5E1,1.7E1,4.6E1,6E0,6E0,9E0,1.6E1,4E0,2.7E1,1.3E1,1.5E1,2.6E1,4E0,9.3E1,2E1,1.1E1,4E0,1.1E1,4E0,7E0,1.7E1,1E1,4E0,7E0,4E0,4E0,4E0,2.3E1,4E0,4E0,8E0,5E0,5E0,2.7E1,1.2E1,5E0,4E0,3.4E1,4.3E1,1.9E1,6E0,1.3E1,1E1,9.4E1,1.1E1,4E0,1.2E1,1E1,6E0,1.7E1,1.2E1,2E1,6.3E1,8E0,4E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"293","size_leaf_vector":"1"}},{"base_weights":[-1.0205011E-3,-1.2043649E-2,2.9758025E-2,-7.526074E-3,-8.05538E-2,3.9493445E-2,-5.273027E-2,-2.596746E-2,1.8260231E-3,-1.2035569E-2,-6.8161994E-2,6.816295E-2,2.513203E-2,-3.9102692E-2,-1.136838E-2,-2.3971293E-2,-1.2887333E-2,1.1088389E-2,-1.8481016E-2,-5.7943292E-2,-1.0505255E-2,5.7987988E-2,1.6895215E-1,1.6680356E-2,8.1108645E-2,-5.432232E-2,2.946668E-2,-7.500619E-3,-4.53924E-2,2.2926519E-2,-1.2982623E-2,-2.7568158E-2,8.932185E-2,-1.0233323E-1,-4.1834682E-2,-7.260861E-2,6.430087E-2,2.2512726E-3,2.2714587E-1,2.4815774E-2,-4.6031874E-2,1.6744672E-1,5.4509465E-2,-6.818085E-2,8.569939E-3,3.1213164E-3,-9.716597E-4,-1.4584395E-2,8.9842424E-2,6.298153E-2,-5.139992E-2,1.7172044E-2,7.482505E-2,-4.697629E-2,5.0537065E-3,-4.941629E-2,3.9481936E-4,5.156218E-2,1.6750309E-1,-9.262707E-3,-7.503322E-2,1.3688358E-2,-5.6383986E-2,-8.222619E-3,1.6922009E-3,7.1378514E-2,-3.8964886E-2,1.3277876E-2,5.9947316E-3,1.6412184E-2,7.220612E-2,-1.17988944E-1,1.9856658E-5,3.7020622E-3,1.1169811E-2,7.984404E-2,3.5842815E-3,-8.918503E-2,-1.3800954E-2,2.6141403E-3,-1.1943396E-3,-9.459103E-3,-8.2414E-2,4.285743E-2,7.903141E-3,-4.5292085E-4,9.2706375E-2,-5.7293486E-2,4.9936254E-2,1.1392198E-1,1.388387E-2,1.001163E-1,-1.2640914E-2,-8.53157E-3,-9.345604E-2,4.7635775E-2,-1.4695464E-2,-4.3548852E-2,-1.2998765E-2,7.6072454E-2,-2.2247164E-2,2.65633E-4,6.213087E-2,1.0830256E-2,4.234972E-3,-3.7501343E-2,-7.500063E-3,6.7518447E-3,-3.407176E-2,-8.076152E-3,-3.546225E-2,8.848385E-2,3.4262195E-2,-2.9215761E-3,1.8529155E-6,4.502625E-2,3.9964844E-3,1.5452085E-1,1.5785912E-2,-2.432914E-3,-8.318325E-3,-4.7141906E-2,4.00697E-3,3.807095E-2,1.00936264E-1,-2.3978145E-3,2.6700513E-2,-5.989661E-2,-1.3544837E-1,-1.8317539E-3,3.9600613E-4,-2.4199493E-2,1.730407E-2,-7.0782066E-3,-4.5322627E-2,-1.547253E-4,3.3049998E-3,6.683974E-3,1.2436885E-3,-5.2269887E-2,-1.3736376E-2,5.573504E-3,1.41385365E-2,6.886693E-3,2.553867E-3,-1.03330895E-1,1.8185537E-2,1.2861247E-1,3.393734E-2,2.1206704E-3,-2.0904962E-3,4.287677E-2,-6.5767646E-2,-2.7037482E-3,-1.3873562E-1,1.3438596E-1,9.5760515E-3,-8.989755E-3,-7.258655E-3,-6.251274E-2,1.4672871E-2,9.073822E-3,5.2431695E-2,1.2671086E-2,-4.587329E-2,1.0740844E-3,3.8609605E-3,-6.342334E-2,1.6095127E-3,-3.8147569E-3,1.3976961E-3,-2.4626752E-2,-4.6635685E-3,1.0433243E-1,1.4083773E-2,7.555788E-2,5.693488E-3,8.402765E-2,1.6462214E-2,-7.5101894E-3,3.771408E-2,9.0488635E-2,1.5775142E-2,-7.4659236E-2,7.520568E-2,7.2850566E-4,-4.1039498E-3,3.0323244E-3,-6.0241564E-5,2.5155204E-3,5.8222273E-3,2.4306453E-3,-4.6780493E-4,-4.8419973E-3,-1.3401828E-3,-3.154875E-3,-8.180059E-3,-9.2142436E-4,-5.8077765E-3,1.7284243E-3,-2.6728702E-3,2.1139665E-3,-4.049013E-3,-3.812528E-3,-1.5918126E-3,-5.561845E-4,2.0803323E-3,-7.84014E-3,-2.1873851E-4,1.2116913E-3,-1.6003324E-3,-9.903084E-4,7.652612E-3,4.5034653E-3,1.2229718E-4,5.995813E-3,5.724265E-4,-5.6432504E-3,-1.3385997E-3,-1.9634606E-3,3.9143213E-3,-1.2589162E-2,-4.817102E-3,8.34592E-3,1.1309922E-3,2.2835154E-3,-5.1649646E-3,-3.084992E-3,1.5952936E-4,-2.6069004E-3,-1.6758459E-2,4.243002E-3,-7.3364004E-4,-2.6180975E-3,4.3278215E-3,-1.4816507E-4,7.3819193E-3,-7.4028936E-3,-1.6973317E-3,-4.5967223E-3,-1.4926149E-3,2.1188175E-3,-1.886008E-3,8.366236E-3,4.6881405E-3,-1.9977249E-3,2.8962793E-3,-2.9958172E-5,4.8963535E-3,2.4888744E-3,-1.470796E-3,6.2293666E-3,2.2674683E-3,-3.9068237E-4,2.4010516E-3,1.6514269E-4,-2.016623E-3,2.5407013E-3,-3.1327852E-3,7.6870727E-3,2.1698792E-3,-7.1434333E-4,-7.8101866E-3,4.8840037E-3,1.7394942E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,-1,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,-1,-1,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,107,109,-1,-1,111,113,-1,-1,115,117,119,121,-1,-1,123,125,127,129,-1,-1,131,133,135,-1,-1,137,139,141,143,145,147,149,151,153,155,157,159,-1,161,163,-1,165,-1,-1,167,-1,-1,169,-1,171,173,175,-1,-1,177,179,181,183,-1,-1,185,-1,187,189,-1,191,193,195,-1,-1,197,199,-1,201,-1,-1,-1,-1,203,-1,-1,205,-1,-1,207,209,211,213,-1,-1,215,217,219,221,223,225,227,-1,229,231,-1,233,235,237,-1,-1,239,-1,-1,-1,241,-1,243,245,247,249,251,253,255,257,259,-1,261,263,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7229836E-1,5.8527684E-1,5.465558E-1,3.0651885E-1,2.1601772E-1,2.4820995E-1,1.6260386E-1,2.7069923E-1,2.2215258E-1,0E0,1.4752734E-1,1.973592E-1,1.907241E-1,7.256552E-2,0E0,2.0923465E-1,0E0,2.3139676E-1,3.6492178E-1,7.008523E-2,0E0,1.571905E-1,1.1954594E-1,1.8174966E-1,1.1258823E-1,4.9393594E-2,2.2199191E-2,2.3369077E-1,1.7121035E-1,1.6118005E-1,1.6455384E-1,2.0896056E-1,7.6974005E-2,4.7302663E-2,6.378189E-2,9.369006E-2,1.3214666E-1,0E0,1.7745972E-2,1.2386514E-1,1.35866E-1,4.6283036E-2,5.3900868E-2,5.1310584E-2,1.68772E-2,0E0,0E0,1.08729094E-1,6.705795E-2,3.082212E-2,1.4918268E-1,1.5462047E-1,1.2072021E-1,1.6538446E-1,1.4870909E-1,2.2607109E-1,2.6042965E-1,9.933639E-3,1.8304259E-2,0E0,5.635482E-2,1.032027E-1,1.3078882E-1,0E0,0E0,1.0235965E-1,9.13126E-3,0E0,0E0,9.503071E-2,2.1583879E-1,4.565677E-2,1.02089055E-1,0E0,0E0,2.082771E-2,1.9163761E-2,3.6601454E-2,7.2434074E-3,0E0,0E0,1.1640164E-1,4.192385E-2,1.625655E-2,0E0,0E0,2.4517253E-2,2.3925781E-1,2.9298972E-2,1.6978085E-2,2.4188736E-1,7.4840814E-2,2.2682814E-2,1.5582532E-1,1.7221239E-1,1.8360765E-1,8.884146E-2,2.0848995E-1,0E0,7.779276E-2,9.710426E-2,0E0,8.797482E-3,0E0,0E0,3.0956611E-2,0E0,0E0,3.6233023E-2,0E0,3.0935936E-2,1.3177288E-1,6.327709E-2,0E0,0E0,8.885373E-2,7.322298E-2,1.6754025E-1,1.6146748E-1,0E0,0E0,3.7913375E-2,0E0,9.844236E-3,8.32735E-3,0E0,9.160756E-3,2.4329744E-2,1.4186442E-2,0E0,0E0,9.855556E-2,1.3036032E-1,0E0,5.0369635E-2,0E0,0E0,0E0,0E0,1.0941261E-1,0E0,0E0,7.4115456E-3,0E0,0E0,8.9439705E-2,1.4085487E-1,1.0867652E-1,2.4552045E-2,0E0,0E0,6.647324E-2,4.294151E-2,4.7560487E-2,1.09895766E-1,5.823487E-2,1.6792724E-1,7.538606E-2,0E0,3.8316393E-1,9.81626E-2,0E0,1.12037316E-1,1.01811096E-1,8.019799E-2,0E0,0E0,8.755531E-3,0E0,0E0,0E0,4.2050935E-2,0E0,4.5952678E-2,5.213774E-2,3.7646994E-2,5.2609432E-2,4.774849E-2,3.719459E-2,5.0226733E-2,6.648997E-2,3.8043655E-2,0E0,5.6907922E-2,1.2984939E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,81,81,82,82,83,83,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,102,102,105,105,108,108,110,110,111,111,112,112,115,115,116,116,117,117,118,118,121,121,123,123,124,124,126,126,127,127,128,128,131,131,132,132,134,134,139,139,142,142,145,145,146,146,147,147,148,148,151,151,152,152,153,153,154,154,155,155,156,156,157,157,159,159,160,160,162,162,163,163,164,164,167,167,171,171,173,173,174,174,175,175,176,176,177,177,178,178,179,179,180,180,181,181,183,183,184,184],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,-1,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,-1,-1,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,108,110,-1,-1,112,114,-1,-1,116,118,120,122,-1,-1,124,126,128,130,-1,-1,132,134,136,-1,-1,138,140,142,144,146,148,150,152,154,156,158,160,-1,162,164,-1,166,-1,-1,168,-1,-1,170,-1,172,174,176,-1,-1,178,180,182,184,-1,-1,186,-1,188,190,-1,192,194,196,-1,-1,198,200,-1,202,-1,-1,-1,-1,204,-1,-1,206,-1,-1,208,210,212,214,-1,-1,216,218,220,222,224,226,228,-1,230,232,-1,234,236,238,-1,-1,240,-1,-1,-1,242,-1,244,246,248,250,252,254,256,258,260,-1,262,264,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.067536E3,4.3702424E7,9.302862E7,1.8122449E2,3E0,5.244755E-3,5.39978E5,4.5E1,1.0715278E7,-1.2035569E-2,4.4337E5,5.847845E3,2.03148E2,5.2E1,-1.136838E-2,2.53E2,-1.2887333E-2,2.6979439E1,1E0,1.45E2,-1.0505255E-2,2.979E3,2.7777777E0,9.496754E6,1.6507992E9,5E0,3.7307863E0,9.705292E6,5.8365756E-1,6.9664386E2,2.465E3,1.945711E1,1.1E1,4.0621606E2,2.294901E0,4.911E3,4.9610714E1,2.2512726E-3,1.701E3,9.3058E4,2.4907975E0,3.832274E2,3.9712732E7,6.08767E3,5.2217197E-2,3.1213164E-3,-9.716597E-4,5.5700205E8,1.2484015E7,4.54E2,1E0,1.0203835E-5,2.0650282E1,1.0950326E6,3.0596492E0,1.8817276E1,1.3944331E-6,6.8671523E3,1.3569831E7,-9.262707E-3,9.662076E2,9.1569895E-1,7.8040985E2,-8.222619E-3,1.6922009E-3,1.6918114E7,5.9E1,1.3277876E-2,5.9947316E-3,9.62689E6,2.7125356E0,1E0,1.6992E4,3.7020622E-3,1.1169811E-2,6.4341316E0,1.001194E6,1.97E4,6.7652373E3,2.6141403E-3,-1.1943396E-3,4.54E2,8.1445E1,2.419355E0,7.903141E-3,-4.5292085E-4,1.2252E4,6.057671E6,4E0,1E0,2.86968E5,1.241112E8,1.79E2,6.654321E0,5.116E3,7.217221E7,7.362237E6,2.1872402E5,-1.2998765E-2,3E0,1.1027811E-2,2.65633E-4,1.5675966E0,1.0830256E-2,4.234972E-3,8.319452E2,-7.500063E-3,6.7518447E-3,1.3298E4,-8.076152E-3,2.6740572E5,2.7246006E7,1.710933E0,-2.9215761E-3,1.8529155E-6,1.3035088E1,3.4409692E3,2.4507338E4,5.0279167E1,-2.432914E-3,-8.318325E-3,4.5709072E1,4.00697E-3,2.545455E7,3.3E1,-2.3978145E-3,6.3300834E0,3.26E2,5.913995E5,-1.8317539E-3,3.9600613E-4,1E0,9E0,-7.0782066E-3,1.0134536E7,-1.547253E-4,3.3049998E-3,6.683974E-3,1.2436885E-3,1.216E3,-1.3736376E-2,5.573504E-3,1E0,6.886693E-3,2.553867E-3,7.2744107E0,1E0,1.378E3,1.2E1,2.1206704E-3,-2.0904962E-3,2.3449652E0,1.8631586E4,4.345542E1,4.561156E7,1.7049885E0,8.5023944E2,9.624275E5,-7.258655E-3,2.0399538E5,1.2542373E0,9.073822E-3,2.5054495E1,2.109E3,2.41E3,1.0740844E-3,3.8609605E-3,3.07E2,1.6095127E-3,-3.8147569E-3,1.3976961E-3,1E0,-4.6635685E-3,1.0918E4,4.9008E4,7.341606E1,5E0,2.79495E-1,1.307E3,3.8546875E3,1.6284274E7,2.1642606E9,1.5775142E-2,7.2896E5,2.9956698E-1,7.2850566E-4,-4.1039498E-3,3.0323244E-3,-6.0241564E-5,2.5155204E-3,5.8222273E-3,2.4306453E-3,-4.6780493E-4,-4.8419973E-3,-1.3401828E-3,-3.154875E-3,-8.180059E-3,-9.2142436E-4,-5.8077765E-3,1.7284243E-3,-2.6728702E-3,2.1139665E-3,-4.049013E-3,-3.812528E-3,-1.5918126E-3,-5.561845E-4,2.0803323E-3,-7.84014E-3,-2.1873851E-4,1.2116913E-3,-1.6003324E-3,-9.903084E-4,7.652612E-3,4.5034653E-3,1.2229718E-4,5.995813E-3,5.724265E-4,-5.6432504E-3,-1.3385997E-3,-1.9634606E-3,3.9143213E-3,-1.2589162E-2,-4.817102E-3,8.34592E-3,1.1309922E-3,2.2835154E-3,-5.1649646E-3,-3.084992E-3,1.5952936E-4,-2.6069004E-3,-1.6758459E-2,4.243002E-3,-7.3364004E-4,-2.6180975E-3,4.3278215E-3,-1.4816507E-4,7.3819193E-3,-7.4028936E-3,-1.6973317E-3,-4.5967223E-3,-1.4926149E-3,2.1188175E-3,-1.886008E-3,8.366236E-3,4.6881405E-3,-1.9977249E-3,2.8962793E-3,-2.9958172E-5,4.8963535E-3,2.4888744E-3,-1.470796E-3,6.2293666E-3,2.2674683E-3,-3.9068237E-4,2.4010516E-3,1.6514269E-4,-2.016623E-3,2.5407013E-3,-3.1327852E-3,7.6870727E-3,2.1698792E-3,-7.1434333E-4,-7.8101866E-3,4.8840037E-3,1.7394942E-3],"split_indices":[52,45,45,52,3,57,11,6,45,0,11,4,56,8,0,10,0,58,105,10,0,2,53,47,5,6,53,9,54,52,2,56,8,52,56,29,53,0,11,2,53,56,45,4,38,0,0,5,5,2,84,41,53,47,53,56,39,47,45,0,52,27,4,0,0,50,10,0,0,45,35,16,2,0,0,54,29,10,52,0,0,2,52,56,0,0,9,28,3,68,7,12,3,54,29,12,47,33,0,8,42,0,53,0,0,52,0,0,9,0,33,45,54,0,0,54,52,4,56,0,0,56,0,45,8,0,53,0,32,0,0,104,8,0,45,0,0,0,0,10,0,0,8,0,0,56,74,2,8,0,0,53,33,58,5,38,52,50,0,33,53,0,56,0,2,0,0,11,0,0,0,67,0,9,9,58,8,38,0,4,43,7,0,9,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.569E3,1.892E3,6.77E2,1.776E3,1.16E2,6.06E2,7.1E1,5.97E2,1.179E3,7E0,1.09E2,2.01E2,4.05E2,6.7E1,4E0,5.93E2,4E0,8.1E2,3.69E2,1.03E2,6E0,1.84E2,1.7E1,3.53E2,5.2E1,5.5E1,1.2E1,3.36E2,2.57E2,5.43E2,2.67E2,3.41E2,2.8E1,2.6E1,7.7E1,8E0,1.76E2,6E0,1.1E1,3.13E2,4E1,1.1E1,4.1E1,4.5E1,1E1,7E0,5E0,3.14E2,2.2E1,1.3E1,2.44E2,4.9E2,5.3E1,9.2E1,1.75E2,1.91E2,1.5E2,2E1,8E0,5E0,2.1E1,1.6E1,6.1E1,4E0,4E0,1.65E2,1.1E1,7E0,4E0,2.67E2,4.6E1,1.5E1,2.5E1,5E0,6E0,2.7E1,1.4E1,3.2E1,1.3E1,4E0,6E0,2.93E2,2.1E1,1.4E1,8E0,4E0,9E0,2.31E2,1.3E1,1.5E1,4.75E2,4.1E1,1.2E1,5.1E1,4.1E1,5.5E1,1.2E2,1.87E2,4E0,3.4E1,1.16E2,4E0,1.6E1,4E0,4E0,1.5E1,6E0,4E0,1.2E1,9E0,5.2E1,1.12E2,5.3E1,7E0,4E0,8E1,1.87E2,1.8E1,2.8E1,7E0,8E0,1.6E1,9E0,1E1,1.7E1,4E0,1E1,2.1E1,1.1E1,6E0,7E0,1.89E2,1.04E2,7E0,1.4E1,5E0,9E0,5E0,4E0,2.27E2,4E0,4E0,9E0,1E1,5E0,1.6E1,4.59E2,2.8E1,1.3E1,4E0,8E0,2.7E1,2.4E1,1.4E1,2.7E1,1.6E1,3.9E1,1.16E2,4E0,1.41E2,4.6E1,5E0,2.9E1,4.7E1,6.9E1,5E0,1.1E1,1.1E1,4E0,7E0,5E0,4.5E1,7E0,9.2E1,2E1,2.1E1,3.2E1,3.3E1,4.7E1,1.4E2,4.7E1,1.4E1,4E0,1.1E1,1.7E1,6E0,1E1,6E0,4E0,5E0,1.2E1,6E0,4E0,9E0,1.2E1,4E0,7E0,1.79E2,1E1,8.4E1,2E1,4E0,1E1,1.03E2,1.24E2,5E0,4E0,1E1,6E0,4.1E2,4.9E1,4E0,2.4E1,4E0,9E0,7E0,2E1,1E1,1.4E1,1E1,4E0,6E0,2.1E1,1.2E1,4E0,3E1,9E0,2.1E1,9.5E1,1.37E2,4E0,1.3E1,3.3E1,7E0,2.2E1,4.3E1,4E0,6E0,6.3E1,5E0,6E0,7E0,3.8E1,1.1E1,8.1E1,9E0,1.1E1,5E0,1.6E1,1.4E1,1.8E1,1.5E1,1.8E1,2.7E1,2E1,1.06E2,3.4E1,4.2E1,5E0,5E0,9E0,7E0,4E0,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"265","size_leaf_vector":"1"}},{"base_weights":[-1.2962357E-3,9.018811E-3,-2.9013291E-2,1.4666681E-3,5.3465556E-2,-4.594633E-2,-2.8616926E-3,-4.8141177E-3,2.8299805E-2,1.6350727E-1,3.8112048E-2,-4.1420855E-2,-1.4546841E-1,-2.1159628E-2,1.8944276E-2,-2.4030115E-3,-8.9337006E-2,3.3942062E-2,-6.3235044E-2,1.8431045E-1,5.070272E-4,4.3459617E-2,-8.9227974E-2,-4.9260646E-2,1.4116148E-2,-2.2199583E-1,-1.1022473E-3,-3.2645797E-3,-5.7978716E-2,3.260172E-2,-4.363479E-2,-9.928109E-3,2.1916881E-2,-1.3963209E-1,3.766385E-2,1.668956E-2,6.5548755E-2,-1.2777638E-2,1.0027528E-2,9.316147E-2,2.3483697E-1,1.105565E-1,3.2114606E-2,-7.7089635E-3,-1.0065711E-3,-4.432259E-2,-1.2221461E-1,9.45557E-2,-1.036862E-2,-5.737719E-3,-1.4598333E-2,-1.1857367E-2,7.69748E-2,-6.31557E-3,-4.1546248E-2,4.605842E-2,-9.539856E-3,-1.1999448E-2,-5.7076896E-3,-1.5404627E-2,2.3608025E-2,-9.76894E-3,4.0946726E-2,-9.889496E-4,-1.7459446E-1,-1.4590427E-3,4.9092607E-3,-6.2326035E-3,2.2882385E-2,7.2390206E-2,-4.0115844E-3,2.3920373E-3,-3.3811566E-3,6.8956357E-3,1.3890656E-4,1.4221278E-2,2.4228944E-3,1.4244752E-1,4.28766E-2,2.517482E-2,1.0991499E-1,-4.0726744E-2,-1.37717E-1,-1.7949948E-1,-1.952541E-2,1.4488795E-3,6.8574743E-3,-3.3442338E-3,8.211231E-3,3.2510285E-3,-1.8874962E-2,5.5997036E-3,1.7438709E-3,-7.741109E-2,-2.4036277E-2,1.9493518E-2,7.420757E-2,2.8211776E-2,-3.4897864E-2,-3.231956E-3,9.186198E-3,-1.676746E-2,-1.3450279E-2,5.079849E-2,-1.7359726E-2,-4.0853724E-2,4.1784283E-2,8.7092176E-2,2.5274504E-2,-3.907953E-3,-2.4634339E-1,1.6688019E-2,1.3872115E-1,1.3378745E-1,5.470537E-2,3.2791365E-3,7.7884514E-3,3.6281E-3,-6.6319807E-4,2.9070586E-2,-3.2061527E-3,6.024729E-2,9.126575E-3,-1.4169613E-1,-3.7059896E-2,-2.832642E-3,-1.0457769E-2,-3.7173198E-3,-1.14735495E-2,6.880426E-4,-2.444158E-3,-4.5505012E-3,1.994279E-3,-1.0944487E-2,-4.738291E-3,-9.919561E-2,-1.13508E-3,-7.257186E-3,-3.7841669E-3,7.7599077E-3,4.4271396E-3,9.342544E-4,8.6138934E-2,2.4474286E-3,-3.2517363E-4,-3.4049922E-3,3.4315643E-4,1.5787228E-3,-1.110456E-3,-2.186668E-2,4.1864156E-3,1.07196964E-1,2.9625297E-2,-5.926049E-2,2.516103E-2,-9.515696E-3,-2.9080745E-2,6.37516E-2,-2.7916253E-2,5.589568E-2,1.3885684E-1,-6.2608615E-2,3.3935014E-2,-1.446447E-2,-6.8294504E-3,2.3055144E-2,-1.0022563E-2,2.4234073E-3,1.0053173E-2,8.8076614E-2,9.846331E-3,6.2991925E-2,-2.6907448E-3,1.7692389E-2,6.2053468E-2,4.5450493E-3,2.5971493E-4,-2.323711E-3,-1.23229865E-2,-2.003535E-2,-5.353014E-2,5.979969E-4,-3.1575587E-2,-2.9503861E-2,2.589223E-2,-2.3067128E-3,-6.61494E-3,-3.2861255E-2,7.478032E-4,3.8292124E-3,-2.9109118E-3,5.1746987E-2,5.753522E-3,-9.085624E-5,-2.2235122E-3,1.1120606E-3,-1.4200617E-3,8.8827836E-4,6.299814E-3,-1.1255634E-3,2.431811E-3,-5.080247E-3,7.75915E-4,1.8834835E-3,-8.906686E-4,-2.0424586E-3,3.8906317E-3,2.3701656E-4,5.6293197E-3,1.2586317E-3,-3.09098E-3,4.5121904E-3,-1.2687644E-3,8.579739E-3,2.498794E-3,-5.572205E-4,-5.252561E-3,1.178143E-3,4.1588857E-3,1.7090933E-3,-1.5772058E-3,6.4004436E-3,1.4869971E-3,2.05967E-3,5.2373293E-3,1.5917488E-3,-4.8908574E-4,5.600875E-3,4.6552604E-4,-2.4218652E-3,1.1920267E-3,-3.9658737E-3,-1.5040047E-3,-2.9925941E-3,1.5268571E-4,-4.2910798E-4,-3.4825122E-3,-8.6980494E-4,3.3648908E-3,4.3055884E-4,-2.8877766E-3,-9.062167E-4,1.4584712E-3,3.8573262E-3,8.317569E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,-1,-1,81,83,85,87,-1,-1,89,91,-1,93,95,97,99,-1,101,103,105,107,-1,109,-1,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1,115,117,119,121,123,125,127,129,-1,-1,-1,131,-1,133,-1,-1,135,137,139,141,143,145,-1,147,-1,149,151,153,155,157,159,161,-1,163,165,167,169,171,-1,-1,-1,-1,173,-1,175,-1,177,179,-1,-1,-1,-1,-1,-1,181,-1,183,-1,185,-1,187,-1,189,-1,-1,191,-1,-1,-1,-1,-1,-1,193,195,197,199,201,203,-1,205,207,209,211,213,215,217,-1,-1,219,-1,-1,-1,221,-1,223,-1,225,227,-1,-1,-1,-1,229,231,-1,233,235,237,-1,-1,239,-1,-1,241,243,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3474574E-1,6.2862813E-1,3.085183E-1,2.701264E-1,4.533195E-1,1.844874E-1,1.1012702E-1,2.6451325E-1,1.5915108E-1,1.0341221E-1,1.6730809E-1,1.7746717E-1,1.6509765E-1,9.864264E-2,1.0911101E-1,2.315417E-1,2.3520976E-1,1.5540555E-1,2.5640348E-1,1.1168003E-1,0E0,1.7229015E-1,4.1833438E-2,1.228323E-1,1.0142715E-1,4.9162984E-2,0E0,7.123968E-2,4.983741E-2,5.941639E-2,4.9995247E-2,1.7778042E-1,1.806899E-1,1.05481505E-1,4.743094E-2,1.6663241E-1,1.0658124E-1,0E0,4.440516E-2,4.8252337E-2,1.5316749E-1,6.511891E-2,1.0467659E-1,0E0,0E0,1.0689503E-1,1.25911E-1,2.9601164E-2,4.258513E-2,0E0,0E0,5.142972E-2,1.0854203E-2,0E0,2.3887016E-2,5.762799E-2,2.5874488E-2,1.9568207E-2,0E0,5.1546013E-1,1.5237662E-1,1.8279189E-1,1.3303754E-1,0E0,1.19041085E-1,0E0,0E0,0E0,1.2684363E-1,9.8003745E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2460411E-2,2.0998174E-2,6.590658E-2,4.6799526E-2,1.1470628E-1,5.7731315E-2,5.6672633E-2,9.480657E-3,0E0,0E0,0E0,1.2722952E-2,0E0,5.076441E-2,0E0,0E0,1.3235971E-2,2.4606088E-2,3.336986E-2,2.3557216E-2,8.250936E-3,2.2685554E-2,0E0,9.852616E-3,0E0,1.2284933E-1,9.4743416E-2,9.9481925E-2,1.1949896E-1,6.761847E-2,6.8137676E-2,1.0944153E-1,0E0,1.2471855E-2,2.4257006E-1,4.097183E-2,4.337889E-2,7.2658E-2,0E0,0E0,0E0,0E0,6.5327704E-2,0E0,1.8126108E-2,0E0,9.840082E-2,8.6972654E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0645362E-2,0E0,5.477357E-2,0E0,9.729803E-3,0E0,1.3578919E-2,0E0,2.7416497E-2,0E0,0E0,2.4968639E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.5402388E-1,1.5853524E-1,3.4906566E-2,6.167798E-2,9.039412E-2,1.5757767E-2,0E0,8.770278E-2,9.470625E-2,2.1387395E-2,8.899941E-2,4.1033536E-2,2.7266055E-2,6.389888E-2,0E0,0E0,1.03390075E-1,0E0,0E0,0E0,2.885978E-2,0E0,6.1239958E-2,0E0,5.207951E-2,1.17705286E-1,0E0,0E0,0E0,0E0,1.9584909E-1,9.387529E-2,0E0,9.008672E-3,4.3957107E-2,4.963198E-2,0E0,0E0,1.1873848E-2,0E0,0E0,1.6657665E-2,1.3008535E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,51,51,52,52,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,64,64,68,68,69,69,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,88,88,90,90,93,93,94,94,95,95,96,96,97,97,98,98,100,100,102,102,103,103,104,104,105,105,106,106,107,107,108,108,110,110,111,111,112,112,113,113,114,114,119,119,121,121,123,123,124,124,131,131,133,133,135,135,137,137,139,139,142,142,149,149,150,150,151,151,152,152,153,153,154,154,156,156,157,157,158,158,159,159,160,160,161,161,162,162,165,165,169,169,171,171,173,173,174,174,179,179,180,180,182,182,183,183,184,184,187,187,190,190,191,191],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,-1,-1,82,84,86,88,-1,-1,90,92,-1,94,96,98,100,-1,102,104,106,108,-1,110,-1,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1,116,118,120,122,124,126,128,130,-1,-1,-1,132,-1,134,-1,-1,136,138,140,142,144,146,-1,148,-1,150,152,154,156,158,160,162,-1,164,166,168,170,172,-1,-1,-1,-1,174,-1,176,-1,178,180,-1,-1,-1,-1,-1,-1,182,-1,184,-1,186,-1,188,-1,190,-1,-1,192,-1,-1,-1,-1,-1,-1,194,196,198,200,202,204,-1,206,208,210,212,214,216,218,-1,-1,220,-1,-1,-1,222,-1,224,-1,226,228,-1,-1,-1,-1,230,232,-1,234,236,238,-1,-1,240,-1,-1,242,244,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9173258E-4,1.5445492E3,3.008162E6,1.8137958E6,1.4075E4,4.4337E5,2.2100918E1,3.842105E1,1.6218617E7,2.4536058E1,1.2367184E3,1.998175E3,3.285E3,2.65851E5,7.76264E7,8.992681E-1,1.75E2,2.342E3,1.1660184E3,2.2007043E0,5.070272E-4,2.3012722E1,1.3676985E4,1.2427474E8,2.5927516E11,1.3E1,-1.1022473E-3,1.4963856E0,1.484789E6,1.3849624E1,4.5162E4,1.2817779E7,1E0,1.01644066E2,7E0,2.4943195E6,1.4421636E3,-1.2777638E-2,1.2608696E0,4.1391224E1,1E0,1.2471131E7,4.5505118E2,-7.7089635E-3,-1.0065711E-3,4.5723195E6,7.4456794E5,2.0201733E9,7.315484E9,-5.737719E-3,-1.4598333E-2,4.514673E-3,1.5E1,-6.31557E-3,3E0,6.607201E0,5.11E2,8.657441E7,-5.7076896E-3,4E0,2.22E2,3.77E2,5.372E3,-9.889496E-4,3.2188842E0,-1.4590427E-3,4.9092607E-3,-6.2326035E-3,2.2376953E1,1.2358527E8,-4.0115844E-3,2.3920373E-3,-3.3811566E-3,6.8956357E-3,1.3890656E-4,1.4221278E-2,2.4228944E-3,1.3E1,1.819149E1,4.5481584E7,7.1463747E0,7.233065E5,1.0503613E5,5.980516E5,3.1372E4,1.4488795E-3,6.8574743E-3,-3.3442338E-3,1.1855755E8,3.2510285E-3,1.9051096E7,5.5997036E-3,1.7438709E-3,1.058E3,1.2199979E10,9.3902206E-1,2.014576E8,3.979829E1,1.058E3,-3.231956E-3,1.1855755E8,-1.676746E-2,3.131E3,5.088781E0,2.407741E7,2.3983052E0,1.1185682E-3,6.072483E2,1.421E3,-3.907953E-3,1.131645E6,1.845406E1,2.0354E4,1.06E2,3.2E1,3.2791365E-3,7.7884514E-3,3.6281E-3,-6.6319807E-4,4.247868E3,-3.2061527E-3,4.860679E7,9.126575E-3,1E0,1.10597E5,-2.832642E-3,-1.0457769E-2,-3.7173198E-3,-1.14735495E-2,6.880426E-4,-2.444158E-3,5.4E1,1.994279E-3,3.451532E8,-4.738291E-3,2.3859661E-2,-1.13508E-3,2.5129596E11,-3.7841669E-3,1.241112E8,4.4271396E-3,9.342544E-4,1.2903092E7,2.4474286E-3,-3.2517363E-4,-3.4049922E-3,3.4315643E-4,1.5787228E-3,-1.110456E-3,3.627907E0,8.9E1,1.459995E2,1.9090909E0,2.316491E-2,9.173084E2,-9.515696E-3,9.277255E6,7.5773336E7,5.1612E7,1.5445488E8,3.2613106E5,1.9E1,1.954095E3,-1.446447E-2,-6.8294504E-3,1.8E1,-1.0022563E-2,2.4234073E-3,1.0053173E-2,5.666351E7,9.846331E-3,3.2049225E1,-2.6907448E-3,7.418546E1,2.5954134E0,4.5450493E-3,2.5971493E-4,-2.323711E-3,-1.23229865E-2,1.246E3,8.890291E2,5.979969E-4,7.8E1,1.6126542E8,2.695E3,-2.3067128E-3,-6.61494E-3,3.512733E-1,7.478032E-4,3.8292124E-3,1.8738E4,7.4259315E0,5.753522E-3,-9.085624E-5,-2.2235122E-3,1.1120606E-3,-1.4200617E-3,8.8827836E-4,6.299814E-3,-1.1255634E-3,2.431811E-3,-5.080247E-3,7.75915E-4,1.8834835E-3,-8.906686E-4,-2.0424586E-3,3.8906317E-3,2.3701656E-4,5.6293197E-3,1.2586317E-3,-3.09098E-3,4.5121904E-3,-1.2687644E-3,8.579739E-3,2.498794E-3,-5.572205E-4,-5.252561E-3,1.178143E-3,4.1588857E-3,1.7090933E-3,-1.5772058E-3,6.4004436E-3,1.4869971E-3,2.05967E-3,5.2373293E-3,1.5917488E-3,-4.8908574E-4,5.600875E-3,4.6552604E-4,-2.4218652E-3,1.1920267E-3,-3.9658737E-3,-1.5040047E-3,-2.9925941E-3,1.5268571E-4,-4.2910798E-4,-3.4825122E-3,-8.6980494E-4,3.3648908E-3,4.3055884E-4,-2.8877766E-3,-9.062167E-4,1.4584712E-3,3.8573262E-3,8.317569E-4],"split_indices":[27,52,32,28,9,11,56,46,50,54,58,52,0,29,45,42,6,2,4,54,0,56,33,45,31,3,0,57,48,54,10,9,100,33,8,45,52,0,53,56,101,51,56,0,0,28,32,5,5,0,0,57,3,0,8,53,0,45,0,3,11,10,29,0,53,0,0,0,56,5,0,0,0,0,0,0,0,0,56,47,35,45,33,28,11,0,0,0,45,0,45,0,0,0,5,27,7,56,0,0,45,0,2,58,9,54,57,4,2,0,9,56,9,0,3,0,0,0,0,52,0,50,0,8,29,0,0,0,0,0,0,3,0,5,0,38,0,31,0,12,0,0,32,0,0,0,0,0,0,56,8,33,56,38,52,0,9,7,5,7,33,8,55,0,0,3,0,0,0,12,0,58,0,56,35,0,0,0,0,2,4,0,3,5,2,0,0,27,0,0,9,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.568E3,1.872E3,6.96E2,1.601E3,2.71E2,4.22E2,2.74E2,1.298E3,3.03E2,3.2E1,2.39E2,4.05E2,1.7E1,1.49E2,1.25E2,1.263E3,3.5E1,2.86E2,1.7E1,2.8E1,4E0,2.3E2,9E0,3.55E2,5E1,1E1,7E0,1.01E2,4.8E1,1.03E2,2.2E1,9.65E2,2.98E2,2.5E1,1E1,1.86E2,1E2,4E0,1.3E1,1.1E1,1.7E1,3.2E1,1.98E2,4E0,5E0,3.34E2,2.1E1,1.1E1,3.9E1,5E0,5E0,9.2E1,9E0,8E0,4E1,7.8E1,2.5E1,1.6E1,6E0,8.3E2,1.35E2,1.12E2,1.86E2,6E0,1.9E1,5E0,5E0,7E0,1.79E2,9.6E1,4E0,9E0,4E0,7E0,4E0,1.3E1,4E0,2.1E1,1.1E1,1.83E2,1.5E1,3.23E2,1.1E1,1.3E1,8E0,5E0,6E0,9E0,3E1,7E0,8.5E1,4E0,5E0,1.2E1,2.8E1,4.1E1,3.7E1,1E1,1.5E1,4E0,1.2E1,4E0,8.26E2,8.1E1,5.4E1,7E1,4.2E1,4.6E1,1.4E2,9E0,1E1,1.71E2,8E0,2E1,7.6E1,4E0,1.7E1,7E0,4E0,1.76E2,7E0,1E1,5E0,1E1,3.13E2,6E0,5E0,5E0,8E0,4E0,4E0,2.2E1,8E0,7.8E1,7E0,8E0,4E0,2.2E1,6E0,3.6E1,5E0,7E0,3E1,6E0,4E0,8E0,7E0,7E0,5E0,5.59E2,2.67E2,2.1E1,6E1,2.7E1,2.7E1,4E0,6.6E1,3.2E1,1E1,3E1,1.6E1,1.2E1,1.28E2,6E0,4E0,1.67E2,4E0,4E0,4E0,1.3E1,7E0,7.1E1,5E0,1.32E2,4.4E1,6E0,4E0,6E0,4E0,1.55E2,1.58E2,1.4E1,8E0,5.2E1,2.6E1,4E0,4E0,1E1,1.2E1,4E0,3.2E1,1.5E1,1.5E1,2.97E2,2.62E2,1.72E2,9.5E1,4E0,1.7E1,1.6E1,4.4E1,1.7E1,1E1,2.1E1,6E0,6E1,6E0,1.5E1,1.7E1,4E0,6E0,2.1E1,9E0,1.1E1,5E0,6E0,6E0,1.07E2,2.1E1,1.39E2,2.8E1,7E0,6E0,4.8E1,2.3E1,8.7E1,4.5E1,2.2E1,2.2E1,9.4E1,6.1E1,7.4E1,8.4E1,4E0,4E0,3.5E1,1.7E1,1.3E1,1.3E1,4E0,6E0,2.2E1,1E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"245","size_leaf_vector":"1"}},{"base_weights":[5.0148217E-4,-1.6372547E-2,1.4105366E-2,-2.6185015E-2,1.2082557E-2,2.4788903E-2,-2.9445184E-2,-2.3839075E-2,-1.3945784E-1,1.5679786E-2,-8.124489E-3,3.2827515E-2,-2.0192214E-3,-2.5283013E-2,-9.371919E-3,-2.2079071E-2,-1.8232834E-1,-1.8385965E-1,-2.486391E-3,2.0970996E-2,-8.201831E-2,1.3765659E-2,4.583945E-2,-7.6176478E-3,1.2692815E-1,-4.2475723E-2,2.4229703E-2,-1.7542202E-2,-8.1694596E-2,4.6069585E-5,-1.644468E-2,-5.165684E-3,-1.2979121E-2,3.270078E-2,-1.8833535E-2,-6.8344837E-3,-4.095895E-2,2.7231464E-2,-2.5171341E-2,1.0030128E-1,3.4941852E-2,-1.971477E-2,4.873605E-2,3.2677458E-4,1.130055E-2,-3.0555127E-2,-9.882157E-2,1.0069528E-1,3.6821014E-3,-1.6023021E-2,-1.2284842E-2,-2.4119712E-1,-5.1350247E-2,6.153466E-3,5.3002443E-2,-5.8664706E-2,1.47075355E-2,-5.107824E-3,8.455499E-4,6.630084E-2,1.8061558E-2,-2.9243103E-3,-7.591726E-2,1.6395776E-1,6.5038204E-2,1.0107406E-1,2.5871836E-2,9.596354E-3,-3.301213E-2,-1.0185313E-3,8.858046E-2,4.6111783E-3,-4.2592924E-2,-7.6271325E-2,-1.0936145E-2,1.1238521E-2,1.7525916E-3,4.600266E-3,-6.5869307E-3,-5.379863E-2,-9.028724E-3,-4.0184893E-4,-2.1291422E-2,-9.020172E-2,-3.0414207E-2,1.5725665E-2,-8.267603E-3,-2.085052E-3,6.4040795E-2,-1.1842435E-2,-1.24009386E-1,-2.9821396E-3,3.8959518E-2,1.3125925E-1,3.9930064E-2,8.6469255E-2,1.14059625E-2,-5.4338314E-3,5.7977946E-3,8.634881E-4,-9.21704E-2,1.2689085E-1,1.2493186E-2,4.537021E-2,8.985675E-3,1.2938687E-1,2.9283572E-2,2.8754048E-2,-5.637549E-3,-2.2186648E-2,6.998139E-2,-7.964862E-3,-2.7318887E-2,2.4963863E-2,-2.8255943E-3,6.990561E-3,5.0966658E-2,-4.427673E-2,2.8901426E-2,-8.292532E-2,-2.216288E-2,-9.909871E-2,8.070104E-4,-3.4354213E-3,2.6060501E-5,4.8553664E-3,-6.410191E-2,9.927161E-3,-2.7265768E-2,-1.1669625E-1,-1.6366992E-4,4.7466666E-3,-5.5264153E-2,-6.491871E-2,2.675557E-2,3.4582752E-3,-2.2932021E-2,2.7488645E-2,8.695957E-2,-3.9242793E-2,2.4732873E-3,-8.87943E-3,-3.0113498E-3,7.050523E-2,9.022596E-3,1.7819483E-3,9.0142805E-3,-3.013076E-3,5.3906698E-2,8.054156E-3,4.8269458E-2,-2.5053553E-3,5.263357E-2,-3.5977274E-2,3.580763E-2,-7.798969E-3,-6.2194757E-2,2.1085679E-3,1.4807181E-1,1.065188E-1,2.9349329E-2,7.624504E-2,1.5184514E-1,-1.4273443E-2,5.2094E-3,-2.5932703E-2,3.570847E-2,6.3493126E-3,-6.788196E-2,8.694753E-2,-4.2402965E-4,-1.941618E-2,-9.649191E-2,-1.0545211E-3,2.1802746E-3,1.0738702E-5,8.1396215E-2,-6.237057E-2,-3.368191E-4,-2.9771277E-3,7.696735E-2,-6.120767E-2,-9.132387E-3,-8.2325526E-2,-1.0156771E-2,-1.323587E-3,-1.2586404E-1,-1.623248E-2,1.1791955E-3,-2.829713E-3,-9.325347E-3,-1.8497674E-4,2.5158133E-3,-2.890582E-3,2.8972724E-4,-2.038203E-3,-7.290109E-3,3.1230946E-3,-1.5180941E-3,-1.16110496E-4,-3.6353762E-3,-1.1113473E-3,-5.5804113E-3,2.7246838E-3,-1.5635873E-4,-2.719524E-3,5.551661E-4,5.2771764E-3,-1.4488065E-3,4.7649434E-3,-1.3103809E-3,7.2088913E-4,-3.7270181E-3,3.298665E-4,4.9053906E-3,-1.4252346E-3,1.8598975E-3,3.741843E-3,2.5676048E-4,-7.030684E-5,3.4117629E-3,-8.782336E-4,1.6347128E-3,6.810221E-4,4.2648767E-3,-2.5922898E-4,-3.8719648E-3,5.041022E-3,8.3043415E-4,-4.3677147E-3,1.4038784E-4,8.454038E-3,2.9638521E-3,2.8077462E-3,7.2754826E-3,5.9914055E-5,2.9403954E-3,5.0381697E-3,1.5368133E-3,9.226776E-3,4.2051477E-3,-3.440102E-3,3.2471106E-3,-1.5934957E-2,6.1427307E-4,-2.8311526E-3,1.9799687E-3,3.5781697E-3,-7.347308E-4,-6.9744084E-3,-1.8659329E-4,5.4474087E-3,1.6679187E-4,-1.3750872E-3,9.029862E-4,-1.084517E-3,-9.623109E-3,1.3618411E-3,6.4999857E-3,-7.597838E-4,-4.5609293E-3,-6.96409E-4,1.52395E-3,2.0152766E-3,6.0054827E-3,-2.1654442E-3,-5.912809E-3,-5.5658575E-3,-1.8660846E-3,-1.0988372E-4,-4.8629646E-3,-2.3957684E-3,-7.602116E-3,-1.462162E-3,1.4440462E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,57,59,61,63,65,67,69,-1,-1,71,73,75,77,79,-1,81,83,85,87,89,91,-1,-1,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,-1,-1,-1,-1,123,125,127,-1,-1,129,131,133,-1,135,137,139,141,-1,143,145,147,149,151,-1,153,-1,155,157,-1,159,-1,161,163,165,-1,167,169,-1,171,173,-1,-1,175,177,179,181,183,185,-1,-1,187,-1,189,191,193,195,-1,197,199,201,203,-1,205,207,209,211,-1,-1,-1,213,215,-1,-1,-1,217,-1,219,221,223,225,227,-1,229,-1,231,233,235,237,239,241,-1,243,245,247,249,251,-1,253,255,-1,-1,-1,257,259,-1,261,263,265,-1,267,269,-1,271,273,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8627796E-1,3.1862134E-1,6.586273E-1,2.2201556E-1,1.866609E-1,2.4487448E-1,1.7887196E-1,2.2805372E-1,5.8556914E-2,1.5059906E-1,0E0,2.1614873E-1,1.908272E-1,2.3342253E-1,0E0,2.2143242E-1,2.4238855E-1,4.0332466E-2,0E0,1.2854609E-1,2.8125323E-2,1.8734267E-1,3.0452335E-1,1.7343539E-1,1.2993316E-1,1.3333073E-1,1.1118333E-1,2.6210004E-1,2.6833448E-1,0E0,0E0,0E0,0E0,1.1393224E-1,8.502956E-2,0E0,3.719121E-2,9.374207E-2,1.03641026E-1,1.8238276E-1,2.5770706E-1,8.177675E-2,8.9300096E-2,0E0,0E0,7.173829E-2,7.8054905E-2,1.1438571E-1,5.2341018E-2,2.011094E-1,0E0,3.8459814E-1,3.792636E-2,1.553769E-1,7.3102534E-2,8.725941E-2,6.5468974E-2,0E0,0E0,8.040896E-2,9.7357206E-2,6.0626484E-2,4.4006363E-2,6.39869E-2,1.20535016E-1,1.0190576E-1,1.5635127E-1,1.28221E-1,9.926431E-2,3.1711124E-2,4.1131824E-2,5.3529635E-2,1.0198519E-1,6.700307E-2,0E0,0E0,0E0,0E0,2.1379616E-2,1.9133264E-1,2.2324674E-1,0E0,0E0,3.8769916E-2,2.9969677E-2,8.090243E-2,0E0,3.2799885E-2,8.138269E-2,3.219633E-2,3.0363813E-2,0E0,2.4862159E-2,5.93417E-2,5.508407E-2,4.645185E-2,1.1369086E-1,0E0,7.7761695E-2,0E0,3.6272466E-2,3.500986E-2,0E0,4.5856453E-2,0E0,3.258109E-2,5.3738367E-2,1.4392433E-1,0E0,5.7805225E-2,3.1357616E-2,0E0,7.411143E-2,1.3822824E-2,0E0,0E0,2.4692152E-2,9.767083E-3,4.619731E-2,7.9885155E-2,6.0560986E-2,4.5251235E-2,0E0,0E0,1.8780136E-2,0E0,9.194231E-2,1.7483033E-1,3.3303642E-1,2.2865266E-2,0E0,3.233295E-2,1.8351477E-2,1.9469153E-2,6.574341E-2,0E0,1.8831573E-2,1.7957579E-1,6.0516536E-2,2.573839E-2,0E0,0E0,0E0,2.2336975E-2,1.6943913E-2,0E0,0E0,0E0,3.337942E-2,0E0,1.4013376E-2,7.96677E-2,6.264897E-2,3.2830518E-2,4.3278176E-2,0E0,2.9752433E-2,0E0,2.3707986E-2,1.1489563E-2,3.397464E-2,1.1437394E-2,3.9984703E-2,5.205483E-2,0E0,4.855001E-1,1.2310004E-1,3.9525136E-2,7.766747E-2,3.4701318E-2,0E0,3.8776014E-2,9.643704E-2,0E0,0E0,0E0,2.2815973E-2,1.2246817E-2,0E0,7.396441E-3,1.41538605E-2,3.162262E-2,0E0,1.4142849E-2,4.9728483E-2,0E0,2.9562414E-2,1.7900333E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,78,78,79,79,80,80,83,83,84,84,85,85,87,87,88,88,89,89,90,90,92,92,93,93,94,94,95,95,96,96,98,98,100,100,101,101,103,103,105,105,106,106,107,107,109,109,110,110,112,112,113,113,116,116,117,117,118,118,119,119,120,120,121,121,124,124,126,126,127,127,128,128,129,129,131,131,132,132,133,133,134,134,136,136,137,137,138,138,139,139,143,143,144,144,148,148,150,150,151,151,152,152,153,153,154,154,156,156,158,158,159,159,160,160,161,161,162,162,163,163,165,165,166,166,167,167,168,168,169,169,171,171,172,172,176,176,177,177,179,179,180,180,181,181,183,183,184,184,186,186,187,187],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,58,60,62,64,66,68,70,-1,-1,72,74,76,78,80,-1,82,84,86,88,90,92,-1,-1,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,-1,-1,-1,-1,124,126,128,-1,-1,130,132,134,-1,136,138,140,142,-1,144,146,148,150,152,-1,154,-1,156,158,-1,160,-1,162,164,166,-1,168,170,-1,172,174,-1,-1,176,178,180,182,184,186,-1,-1,188,-1,190,192,194,196,-1,198,200,202,204,-1,206,208,210,212,-1,-1,-1,214,216,-1,-1,-1,218,-1,220,222,224,226,228,-1,230,-1,232,234,236,238,240,242,-1,244,246,248,250,252,-1,254,256,-1,-1,-1,258,260,-1,262,264,266,-1,268,270,-1,272,274,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7311627E2,1.838444E6,4.5843E7,4.8297736E7,1E0,1.0938637E1,5.794597E7,3.684685E2,6.9666364E2,4.70092E1,-8.124489E-3,6.9664386E2,1.5439131E2,3.220188E3,-9.371919E-3,1.2342778E1,1.4396E4,8.3510876E9,-2.486391E-3,9.1E2,1.0315458E8,5.1861375E6,1.4075E4,3.33E2,1.3532244E7,1.2427474E8,2E0,1.813943E6,8E0,4.6069585E-5,-1.644468E-2,-5.165684E-3,-1.2979121E-2,7.79E2,1.314546E-1,-6.8344837E-3,1.2E1,1.325E0,2.3902743E0,5.473125E3,2.072E3,4.24566E5,2.6904478E5,3.2677458E-4,1.130055E-2,1.3E1,7.6008E4,6.862E3,8.837831E-2,4.0701206E1,-1.2284842E-2,9E0,7.8704453E0,3.5521298E2,2.742277E6,1.0241028E5,2.8E2,-5.107824E-3,8.455499E-4,1.5208336E4,1.13929E7,2.8599728E7,1.5552E4,4.0728608E3,1E0,1.7589172E1,1.3144558E4,5.72E2,6.75784E5,7.38E2,4.65E3,6.7628815E2,1.0323588E3,2.589778E6,-1.0936145E-2,1.1238521E-2,1.7525916E-3,4.600266E-3,2.7258065E0,2E0,4.204986E-2,-4.0184893E-4,-2.1291422E-2,2.862988E5,4.1828512E6,4.07882E7,-8.267603E-3,8.31E2,1.752E3,6.97E3,9.69239E5,-2.9821396E-3,3.051499E5,6.2E1,3.5029E4,3.958068E2,5.8018835E2,-5.4338314E-3,9.451483E-1,8.634881E-4,8.169439E5,3.3159972E5,1.2493186E-2,4.155E3,8.985675E-3,3.564E3,2.07298E5,3.647E3,-5.637549E-3,1.0148889E3,2.9154164E-3,-7.964862E-3,1.417988E6,2.8199516E4,-2.8255943E-3,6.990561E-3,4.94374E1,2.175722E6,1.5092975E6,1.8600049E10,1.9928571E0,8.7757526E2,8.070104E-4,-3.4354213E-3,3.6E1,4.8553664E-3,3.64299E5,9.76E2,1.4887473E-1,2.434002E7,-1.6366992E-4,2.1E1,8.956858E-4,7.1E1,1E0,3.4582752E-3,1.4813511E6,8E0,1.4521204E0,5.4E2,2.4732873E-3,-8.87943E-3,-3.0113498E-3,1.2460085E7,7E0,1.7819483E-3,9.0142805E-3,-3.013076E-3,2.462277E6,8.054156E-3,8.01E2,2.857E3,5.751617E6,6.1392635E-1,2.2772677E5,-7.798969E-3,9.161318E0,2.1085679E-3,7.727022E8,3.4E1,4.7013435E1,1.6E1,6.7507215E6,1.411E3,5.2094E-3,1.0449754E0,3.324E3,1.79E2,7.474142E-4,8.2619705E1,-4.2402965E-4,4.92E2,1.71E2,-1.0545211E-3,2.1802746E-3,1.0738702E-5,1.138E3,6.5981384E2,-3.368191E-4,3.4736843E0,1.6841E4,4.5E1,-9.132387E-3,5E0,1.0941897E9,-1.323587E-3,1.3992102E3,9.609747E3,1.1791955E-3,-2.829713E-3,-9.325347E-3,-1.8497674E-4,2.5158133E-3,-2.890582E-3,2.8972724E-4,-2.038203E-3,-7.290109E-3,3.1230946E-3,-1.5180941E-3,-1.16110496E-4,-3.6353762E-3,-1.1113473E-3,-5.5804113E-3,2.7246838E-3,-1.5635873E-4,-2.719524E-3,5.551661E-4,5.2771764E-3,-1.4488065E-3,4.7649434E-3,-1.3103809E-3,7.2088913E-4,-3.7270181E-3,3.298665E-4,4.9053906E-3,-1.4252346E-3,1.8598975E-3,3.741843E-3,2.5676048E-4,-7.030684E-5,3.4117629E-3,-8.782336E-4,1.6347128E-3,6.810221E-4,4.2648767E-3,-2.5922898E-4,-3.8719648E-3,5.041022E-3,8.3043415E-4,-4.3677147E-3,1.4038784E-4,8.454038E-3,2.9638521E-3,2.8077462E-3,7.2754826E-3,5.9914055E-5,2.9403954E-3,5.0381697E-3,1.5368133E-3,9.226776E-3,4.2051477E-3,-3.440102E-3,3.2471106E-3,-1.5934957E-2,6.1427307E-4,-2.8311526E-3,1.9799687E-3,3.5781697E-3,-7.347308E-4,-6.9744084E-3,-1.8659329E-4,5.4474087E-3,1.6679187E-4,-1.3750872E-3,9.029862E-4,-1.084517E-3,-9.623109E-3,1.3618411E-3,6.4999857E-3,-7.597838E-4,-4.5609293E-3,-6.96409E-4,1.52395E-3,2.0152766E-3,6.0054827E-3,-2.1654442E-3,-5.912809E-3,-5.5658575E-3,-1.8660846E-3,-1.0988372E-4,-4.8629646E-3,-2.3957684E-3,-7.602116E-3,-1.462162E-3,1.4440462E-3],"split_indices":[52,9,45,45,65,53,1,52,4,58,0,52,58,52,0,54,9,5,0,11,7,50,9,8,1,45,8,9,17,0,0,0,0,2,39,0,3,53,53,32,29,12,47,0,0,3,10,9,57,52,0,0,53,4,9,47,0,0,0,33,5,5,9,4,66,56,52,33,12,0,10,52,52,29,0,0,0,0,54,17,41,0,0,28,45,7,0,2,29,10,1,0,28,12,1,52,52,0,41,0,48,28,0,10,0,2,1,2,0,47,39,0,29,28,0,0,58,1,28,5,54,52,0,0,8,0,9,10,38,5,0,3,39,0,16,0,28,8,38,0,0,0,0,12,3,0,0,0,28,0,2,2,45,42,33,0,54,0,7,3,56,3,47,29,0,53,29,3,39,56,0,3,8,0,0,0,0,4,0,54,9,3,0,6,7,0,4,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.552E3,1.139E3,1.413E3,8.47E2,2.92E2,1.135E3,2.78E2,8.31E2,1.6E1,2.87E2,5E0,8.73E2,2.62E2,2.72E2,6E0,8.23E2,8E0,1E1,6E0,2.73E2,1.4E1,3.55E2,5.18E2,2.52E2,1E1,2.02E2,7E1,7.66E2,5.7E1,4E0,4E0,6E0,4E0,2.11E2,6.2E1,5E0,9E0,2.64E2,9.1E1,8.5E1,4.33E2,2.08E2,4.4E1,5E0,5E0,1.68E2,3.4E1,1.4E1,5.6E1,7.62E2,4E0,8E0,4.9E1,9.2E1,1.19E2,2.8E1,3.4E1,4E0,5E0,4.9E1,2.15E2,6.4E1,2.7E1,2.9E1,5.6E1,5.1E1,3.82E2,6.5E1,1.43E2,2E1,2.4E1,4.3E1,1.25E2,3E1,4E0,4E0,1E1,5E0,5.1E1,1.18E2,6.44E2,4E0,4E0,1.6E1,3.3E1,8.8E1,4E0,2E1,9.9E1,1.7E1,1.1E1,8E0,2.6E1,1.3E1,3.6E1,1.8E1,1.97E2,4E0,6E1,4E0,2.3E1,2.2E1,7E0,4.9E1,7E0,3.6E1,1.5E1,3.75E2,7E0,4.3E1,2.2E1,5E0,1.38E2,1.4E1,6E0,9E0,1.5E1,1.4E1,2.9E1,4.1E1,8.4E1,2.4E1,6E0,4E0,4.7E1,7E0,1.11E2,3.16E2,3.28E2,1.2E1,4E0,1.4E1,1.9E1,1E1,7.8E1,4E0,1.6E1,3.9E1,6E1,1.2E1,5E0,5E0,6E0,1.2E1,1.4E1,5E0,8E0,4E0,3.2E1,5E0,1.3E1,1.48E2,4.9E1,2.5E1,3.5E1,6E0,1.7E1,5E0,1.7E1,9E0,4E1,1.2E1,2.4E1,1E1,5E0,4.2E1,3.33E2,2.7E1,1.6E1,1.8E1,4E0,1.25E2,1.3E1,4E0,1E1,6E0,9E0,9E0,5E0,1.8E1,1.1E1,3.5E1,6E0,1.3E1,7.1E1,7E0,1.7E1,2.8E1,1.9E1,1.06E2,5E0,2.37E2,7.9E1,1.7E2,1.58E2,4E0,8E0,5E0,9E0,5E0,1.4E1,6E0,4E0,4E1,3.8E1,8E0,8E0,1.6E1,2.3E1,5.6E1,4E0,5E0,7E0,4E0,8E0,6E0,8E0,2.2E1,1E1,4E0,9E0,1.04E2,4.4E1,2.3E1,2.6E1,1.5E1,1E1,7E0,2.8E1,1.2E1,5E0,1.3E1,4E0,5E0,4E0,2.1E1,1.9E1,7E0,5E0,1.5E1,9E0,6E0,4E0,4E0,3.8E1,1.3E1,3.2E2,6E0,2.1E1,7E0,9E0,1.4E1,4E0,1.03E2,2.2E1,8E0,5E0,5E0,4E0,4E0,5E0,1.4E1,4E0,7E0,4E0,2.8E1,7E0,7E0,6E0,6.6E1,5E0,5E0,1.2E1,2.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"275","size_leaf_vector":"1"}},{"base_weights":[-7.3425507E-4,-1.2174737E-2,1.6416114E-2,-1.4359694E-2,8.281298E-2,2.0976897E-2,-4.30343E-2,-3.2962497E-2,-2.7001032E-3,1.1982006E-1,-8.154238E-3,-2.6972683E-3,3.2524977E-2,-9.215734E-2,-2.047922E-2,-8.209129E-2,-2.7699409E-2,7.450326E-2,-5.4360316E-3,1.7015561E-1,1.2832192E-2,1.9770737E-3,-1.4682042E-2,5.2129637E-2,2.0961158E-2,-1.1225693E-1,1.3253996E-4,4.6634874E-3,-4.0257227E-2,-2.2815257E-1,-4.92678E-2,-2.2627119E-2,-7.479603E-2,-8.228211E-3,1.182306E-1,-9.397854E-3,3.131762E-2,2.1025902E-1,9.332229E-2,-2.0770417E-3,2.4911726E-3,4.2644966E-2,-2.4009626E-2,5.5281892E-2,-5.319065E-3,5.238642E-2,7.214865E-3,-1.2809226E-1,-2.113568E-3,-5.1076073E-2,2.8235542E-3,-6.1353776E-3,-1.5437594E-2,-7.827816E-2,6.531555E-3,-3.4241926E-2,-1.09395645E-4,-1.6456395E-2,-1.08920924E-1,2.3424819E-3,-4.7346978E-3,1.3533178E-3,1.4502102E-1,-2.6182873E-2,-1.3532115E-3,3.635496E-4,6.226343E-2,5.0100116E-3,1.239191E-2,6.8995873E-3,1.4938192E-3,3.4252696E-2,1.1671973E-1,-7.696107E-3,-1.9686505E-2,4.5924608E-2,1.0743994E-1,8.541807E-2,3.2489683E-2,1.0313134E-2,-6.924054E-3,-3.3643064E-3,-8.636337E-3,-5.0912034E-3,-3.1397004E-2,-1.3602588E-1,-3.4646656E-2,4.091596E-3,-2.295355E-2,-3.1198166E-2,-8.201797E-3,3.2129423E-3,-4.4037183E-3,-5.48915E-3,2.2000637E-2,-3.498525E-2,-1.9038233E-1,9.313987E-3,4.0141987E-3,-2.1531712E-2,-1.2206294E-1,4.6035413E-2,-9.718962E-3,3.995026E-2,-3.2903828E-2,8.5369036E-2,-6.4656213E-3,5.021368E-2,8.807248E-3,7.2433734E-3,2.4076325E-3,1.1633055E-2,-3.1286586E-2,1.1922343E-1,3.761746E-2,9.181189E-3,8.180099E-2,1.1941971E-1,7.186703E-3,-3.4279579E-3,4.0247492E-2,6.514964E-3,1.088821E-2,-5.627195E-3,-7.173026E-2,-1.9383542E-3,-8.740679E-3,1.3406327E-3,-7.2744235E-2,2.6439866E-3,-3.8467227E-3,-4.425566E-2,-4.13724E-3,4.4737373E-2,-4.5005665E-3,4.447218E-2,-9.552982E-4,7.34379E-4,-3.3946154E-3,-4.787712E-3,-1.0795726E-2,-2.6680576E-2,8.409802E-2,-1.361244E-2,-6.4084254E-4,5.7266645E-2,-3.268417E-2,-7.359007E-2,-3.7686992E-3,-1.7550908E-3,5.968127E-2,-8.517595E-2,1.4251115E-2,5.5694934E-2,1.2673728E-1,2.8003405E-3,-2.2348112E-3,1.0581821E-1,2.9603966E-2,8.2158566E-2,-2.126695E-2,1.1396747E-1,-1.30092595E-2,-6.2025115E-2,1.29526E-3,7.342142E-3,2.3362804E-3,-5.0109994E-2,4.5226354E-2,6.0224798E-2,6.8917037E-3,4.9091324E-2,1.9585244E-1,-2.0513583E-2,3.3322766E-3,1.3668934E-2,5.5225197E-2,-1.1983575E-2,2.592443E-2,-1.9962445E-3,1.1936912E-2,-5.292031E-3,-1.3464405E-4,-4.943885E-3,-8.13273E-4,3.8092036E-4,-2.7171173E-3,-9.806415E-4,2.825083E-3,3.7067942E-3,-7.9234096E-4,-5.0718025E-3,7.981128E-5,3.3243913E-3,4.54984E-4,-9.472964E-4,-4.644365E-3,1.5572374E-3,6.5874187E-3,4.0252106E-3,1.5644169E-4,-2.9648996E-3,5.579975E-4,-5.0774124E-3,2.5590747E-3,-6.4849836E-4,1.808168E-3,6.4964517E-4,4.9831616E-3,-7.365496E-4,-5.9235934E-3,2.1881682E-3,-1.0766023E-3,8.14001E-4,4.4987896E-3,3.8341836E-3,7.926432E-3,1.1343554E-3,7.2693676E-3,-2.656339E-3,1.9864745E-3,7.803632E-3,8.182089E-4,4.953068E-4,-4.081926E-3,1.6037418E-3,8.154507E-3,1.9637628E-3,-2.89463E-3,-2.3233402E-3,-1.0987679E-2,1.7481197E-3,-1.7226654E-3,-7.53341E-3,7.6430175E-4,2.737716E-3,-8.7210577E-4,4.607365E-3,-3.6342078E-4,3.155153E-5,3.9909384E-3,1.300884E-2,6.1921696E-3,8.9943805E-4,-2.7785962E-3,-1.9730998E-3,1.4970659E-3,3.162015E-3,-5.186564E-4,-3.7236544E-3,1.3372418E-4,2.0168677E-3,-1.0900316E-3,-9.5612276E-4,2.2856072E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,-1,-1,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,73,75,-1,77,79,81,-1,83,-1,-1,-1,85,87,89,91,93,95,-1,-1,-1,97,99,101,103,105,-1,-1,-1,-1,107,109,-1,111,113,115,117,119,121,-1,-1,-1,-1,123,125,127,-1,129,131,-1,133,-1,-1,135,137,139,-1,-1,141,143,145,147,149,151,153,155,157,159,-1,-1,161,163,165,167,-1,169,171,173,-1,175,177,-1,179,181,-1,-1,-1,183,-1,-1,185,187,189,191,193,-1,-1,-1,-1,-1,195,197,-1,-1,199,201,203,205,-1,207,209,211,213,215,-1,-1,217,219,221,223,225,227,229,231,-1,-1,233,235,237,-1,239,241,243,-1,245,247,249,251,-1,253,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.070063E-1,3.2288253E-1,2.8126207E-1,3.2873303E-1,3.381933E-1,2.6281664E-1,8.010279E-2,1.4873052E-1,1.9747593E-1,1.6476059E-1,0E0,4.3062073E-1,1.453585E-1,4.4127494E-2,1.2080713E-1,2.5723693E-1,1.2467921E-1,1.1673693E-1,1.3163583E-1,4.5085013E-2,2.4228234E-2,3.3075312E-1,0E0,1.253587E-1,1.7587024E-1,1.5629917E-2,0E0,0E0,4.9996585E-2,4.720223E-2,7.6592974E-2,1.2527345E-1,9.974006E-2,6.166467E-2,4.7350913E-2,1.1001359E-1,8.427245E-2,2.9104173E-2,2.1423183E-2,0E0,0E0,7.184401E-2,1.0432348E-1,1.0975945E-1,0E0,7.87203E-2,1.3016117E-1,2.9148087E-2,0E0,3.7801303E-2,0E0,0E0,0E0,7.339539E-2,3.965102E-2,1.1982381E-1,4.824407E-2,7.425453E-2,1.8607232E-1,0E0,0E0,0E0,2.785346E-2,1.1510351E-1,2.1928307E-1,6.0577083E-2,7.042512E-2,0E0,0E0,0E0,0E0,4.4638664E-2,1.6243681E-2,0E0,6.793952E-2,1.179049E-1,5.344391E-2,1.21501535E-1,6.471869E-2,2.1935642E-1,0E0,0E0,0E0,0E0,3.1817153E-2,4.2390093E-2,4.6452284E-2,0E0,5.736521E-2,1.09354764E-1,0E0,5.1128898E-2,0E0,0E0,1.47104785E-2,3.0467369E-2,1.571852E-2,0E0,0E0,1.398437E-1,1.9361524E-1,7.525352E-2,1.783461E-1,3.320394E-2,6.2836066E-2,3.345105E-2,3.1206448E-2,7.479493E-2,9.881048E-2,0E0,0E0,1.3009225E-1,1.3628682E-1,3.3301204E-2,1.2375292E-1,0E0,2.5600448E-2,1.6239783E-1,2.6091922E-2,0E0,2.8703474E-2,9.940757E-2,0E0,1.2534422E-2,2.8031033E-2,0E0,0E0,0E0,1.6142689E-2,0E0,0E0,1.1017504E-1,9.696923E-2,4.5786206E-2,8.0016166E-2,7.1960073E-3,0E0,0E0,0E0,0E0,0E0,1.2207584E-1,2.6214108E-2,0E0,0E0,9.179275E-2,1.3519363E-2,1.4511617E-1,1.5907475E-1,0E0,3.0484006E-2,2.646169E-2,1.574872E-2,2.69498E-2,8.561358E-3,0E0,0E0,5.6041196E-2,4.496598E-2,6.0741857E-2,6.109989E-2,3.5068333E-2,1.00972585E-1,1.6137141E-1,8.1951894E-2,0E0,0E0,1.0106179E-1,1.0167286E-1,4.8165493E-2,0E0,2.6798878E-2,3.7322402E-2,1.586945E-2,0E0,2.5432719E-2,2.5501102E-2,1.3038716E-1,9.394309E-2,0E0,1.5581115E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56,57,57,58,58,62,62,63,63,64,64,65,65,66,66,71,71,72,72,74,74,75,75,76,76,77,77,78,78,79,79,84,84,85,85,86,86,88,88,89,89,91,91,94,94,95,95,96,96,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,111,111,112,112,113,113,114,114,116,116,117,117,118,118,120,120,121,121,123,123,124,124,128,128,131,131,132,132,133,133,134,134,135,135,141,141,142,142,145,145,146,146,147,147,148,148,150,150,151,151,152,152,153,153,154,154,157,157,158,158,159,159,160,160,161,161,162,162,163,163,164,164,167,167,168,168,169,169,171,171,172,172,173,173,175,175,176,176,177,177,178,178,180,180],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,-1,-1,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,74,76,-1,78,80,82,-1,84,-1,-1,-1,86,88,90,92,94,96,-1,-1,-1,98,100,102,104,106,-1,-1,-1,-1,108,110,-1,112,114,116,118,120,122,-1,-1,-1,-1,124,126,128,-1,130,132,-1,134,-1,-1,136,138,140,-1,-1,142,144,146,148,150,152,154,156,158,160,-1,-1,162,164,166,168,-1,170,172,174,-1,176,178,-1,180,182,-1,-1,-1,184,-1,-1,186,188,190,192,194,-1,-1,-1,-1,-1,196,198,-1,-1,200,202,204,206,-1,208,210,212,214,216,-1,-1,218,220,222,224,226,228,230,232,-1,-1,234,236,238,-1,240,242,244,-1,246,248,250,252,-1,254,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5107028E-1,8.61491E3,1E0,1.0119178E10,5.301666E9,3.7810526E2,1.172E3,2.8972292E-5,6E0,5.5544252E7,-8.154238E-3,3.684685E2,8E0,1.5915463E0,3.957351E6,3.1517188E2,9.6910744E7,3.830986E-1,2.512E3,2.909019E3,3.0612E4,1.7710843E0,-1.4682042E-2,5.683493E6,2.3887E4,5.8372E4,1.3253996E-4,4.6634874E-3,1.038946E6,1.650267E-6,5.116325E3,1.9E1,1.2E1,2.714E2,1.552356E0,1E0,4.7E1,2.692487E6,1.0746194E4,-2.0770417E-3,2.4911726E-3,6.4166665E0,3.3820656E7,8.072373E1,-5.319065E-3,2.1583429E-3,3.7826266E4,8.9E1,-2.113568E-3,1.0012501E7,2.8235542E-3,-6.1353776E-3,-1.5437594E-2,2E0,6.135771E9,9.697451E-1,1.7851065E1,1.125584E-3,4.8765745E0,2.3424819E-3,-4.7346978E-3,1.3533178E-3,1.00737E5,4.1585636E1,4.296196E0,1.4400111E10,3.1827815E12,5.0100116E-3,1.239191E-2,6.8995873E-3,1.4938192E-3,7.997723E6,8.629299E4,-7.696107E-3,5.2770585E-1,5E1,4.247E3,8.2E1,1.0966866E0,1.3144558E4,-6.924054E-3,-3.3643064E-3,-8.636337E-3,-5.0912034E-3,1.9589581E6,5.2E1,1.5E1,4.091596E-3,3.4015749E0,6.285983E5,-8.201797E-3,2.322963E0,-4.4037183E-3,-5.48915E-3,4.5402945E6,2.707947E8,9E0,9.313987E-3,4.0141987E-3,6.1E1,1.752E3,1E0,1.8273557E2,4.5510144E9,1.8215276E7,4.3978744E0,2.9E1,1E0,1.0506024E1,7.2433734E-3,2.4076325E-3,1.1E2,1.032E3,2.7627052E7,1.55E2,9.181189E-3,2.642E4,2.192768E6,6.4683E4,-3.4279579E-3,3.076124E5,1.02E2,1.088821E-2,1.7848537E0,9.9625344E5,-1.9383542E-3,-8.740679E-3,1.3406327E-3,2.88001E10,2.6439866E-3,-3.8467227E-3,9.9E1,1E0,2.155E3,4.020408E0,1.0347911E7,-9.552982E-4,7.34379E-4,-3.3946154E-3,-4.787712E-3,-1.0795726E-2,1.057101E6,4.277001E3,-1.361244E-2,-6.4084254E-4,1.03E2,2.4945455E-4,1.131645E6,1.7730331E6,-1.7550908E-3,2.0288463E0,4.1E1,2.0907634E3,1.4400111E10,7.315484E9,2.8003405E-3,-2.2348112E-3,1.632095E2,3.97E2,2.1288E2,1.4622253E5,3.4463277E0,5.117647E0,2.835051E2,6.977611E5,7.342142E-3,2.3362804E-3,2.0939393E0,1.6218617E7,8.962557E8,6.8917037E-3,1.6943169E5,2.7523365E0,6.44833E-1,3.3322766E-3,4.304E3,2.778E3,1.9121015E0,1.0046621E7,-1.9962445E-3,1.5190727E5,-5.292031E-3,-1.3464405E-4,-4.943885E-3,-8.13273E-4,3.8092036E-4,-2.7171173E-3,-9.806415E-4,2.825083E-3,3.7067942E-3,-7.9234096E-4,-5.0718025E-3,7.981128E-5,3.3243913E-3,4.54984E-4,-9.472964E-4,-4.644365E-3,1.5572374E-3,6.5874187E-3,4.0252106E-3,1.5644169E-4,-2.9648996E-3,5.579975E-4,-5.0774124E-3,2.5590747E-3,-6.4849836E-4,1.808168E-3,6.4964517E-4,4.9831616E-3,-7.365496E-4,-5.9235934E-3,2.1881682E-3,-1.0766023E-3,8.14001E-4,4.4987896E-3,3.8341836E-3,7.926432E-3,1.1343554E-3,7.2693676E-3,-2.656339E-3,1.9864745E-3,7.803632E-3,8.182089E-4,4.953068E-4,-4.081926E-3,1.6037418E-3,8.154507E-3,1.9637628E-3,-2.89463E-3,-2.3233402E-3,-1.0987679E-2,1.7481197E-3,-1.7226654E-3,-7.53341E-3,7.6430175E-4,2.737716E-3,-8.7210577E-4,4.607365E-3,-3.6342078E-4,3.155153E-5,3.9909384E-3,1.300884E-2,6.1921696E-3,8.9943805E-4,-2.7785962E-3,-1.9730998E-3,1.4970659E-3,3.162015E-3,-5.186564E-4,-3.7236544E-3,1.3372418E-4,2.0168677E-3,-1.0900316E-3,-9.5612276E-4,2.2856072E-3],"split_indices":[38,4,104,20,7,52,2,42,3,45,0,52,17,39,45,55,45,56,0,32,2,53,0,1,9,29,0,0,9,38,33,3,18,33,54,102,8,1,4,0,0,56,7,58,0,27,52,0,0,45,0,0,0,8,5,27,57,38,57,0,0,0,12,53,58,5,31,0,0,0,0,9,33,0,41,0,2,8,41,52,0,0,0,0,32,10,8,0,56,28,0,53,0,0,32,7,3,0,0,3,29,74,55,5,1,53,3,16,58,0,0,10,10,5,10,0,29,47,2,0,28,8,0,54,28,0,0,0,5,0,0,10,105,0,54,1,0,0,0,0,0,9,47,0,0,11,38,9,28,0,53,3,4,5,5,0,0,4,2,52,33,53,54,52,45,0,0,53,50,12,0,33,53,35,0,2,0,53,47,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.582E3,1.549E3,1.033E3,1.515E3,3.4E1,9.6E2,7.3E1,5.83E2,9.32E2,3E1,4E0,3.15E2,6.45E2,2.2E1,5.1E1,5.5E1,5.28E2,3.1E1,9.01E2,2E1,1E1,3.11E2,4E0,2.38E2,4.07E2,1.8E1,4E0,7E0,4.4E1,9E0,4.6E1,4.78E2,5E1,1.1E1,2E1,8.14E2,8.7E1,1.2E1,8E0,4E0,6E0,1.21E2,1.9E2,2.34E2,4E0,1.23E2,2.84E2,1.4E1,4E0,4E1,4E0,5E0,4E0,3E1,1.6E1,3.15E2,1.63E2,1.9E1,3.1E1,7E0,4E0,5E0,1.5E1,2.63E2,5.51E2,4.4E1,4.3E1,4E0,8E0,4E0,4E0,1.1E2,1.1E1,5E0,1.85E2,2E2,3.4E1,4.5E1,7.8E1,2.79E2,5E0,7E0,7E0,1E1,3E1,1.2E1,1.8E1,4E0,1.2E1,3.09E2,6E0,1.58E2,5E0,5E0,1.4E1,1.7E1,1.4E1,8E0,7E0,2.52E2,1.1E1,8.2E1,4.69E2,2E1,2.4E1,3.2E1,1.1E1,6.7E1,4.3E1,7E0,4E0,5E1,1.35E2,1.9E1,1.81E2,7E0,2.7E1,3.1E1,1.4E1,5E0,7.3E1,2.75E2,4E0,1.9E1,1.1E1,4E0,8E0,7E0,1.1E1,5E0,7E0,2.08E2,1.01E2,2.4E1,1.34E2,9E0,5E0,7E0,1E1,4E0,1E1,2.41E2,1.1E1,4E0,7E0,7.2E1,1E1,3.9E1,4.3E2,4E0,1.6E1,1.1E1,1.3E1,2E1,1.2E1,4E0,7E0,1.7E1,5E1,1.2E1,3.1E1,9E0,4.1E1,6.9E1,6.6E1,1.3E1,6E0,1.4E1,1.67E2,2.1E1,6E0,1.7E1,1.4E1,1E1,4E0,2.7E1,4.6E1,1.41E2,1.34E2,6E0,1.3E1,7E0,4E0,7E0,4E0,3.4E1,1.74E2,8.1E1,2E1,1.6E1,8E0,7E0,1.27E2,5E0,4E0,2.17E2,2.4E1,6E0,5E0,5E1,2.2E1,6E0,4E0,3.2E1,7E0,3.5E2,8E1,8E0,8E0,4E0,7E0,7E0,6E0,1E1,1E1,6E0,6E0,6E0,1.1E1,5E0,4.5E1,5E0,7E0,2.1E1,1E1,4E0,5E0,1.9E1,2.2E1,6.4E1,5E0,3.4E1,3.2E1,5E0,9E0,1.45E2,2.2E1,1.4E1,7E0,7E0,1E1,6E0,8E0,5E0,5E0,6E0,2.1E1,4.1E1,5E0,2.6E1,1.15E2,1.03E2,3.1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"255","size_leaf_vector":"1"}},{"base_weights":[-1.7994246E-3,-1.981202E-2,9.841743E-3,-5.3462265E-3,-3.8951367E-2,2.3642584E-4,3.6659963E-2,-2.6411531E-2,1.9822225E-2,-1.3830346E-1,-3.6230635E-2,1.8983912E-2,-8.927677E-3,4.25854E-2,-3.385568E-3,-3.8916068E-3,-5.0731104E-2,7.934104E-2,-1.8968304E-3,-1.0739538E-2,-1.9298273E-3,1.380735E-2,-4.2794846E-2,2.8214049E-2,-4.371501E-2,-1.0799894E-2,8.465433E-2,5.104204E-2,1.2224359E-2,-2.3164555E-2,7.765349E-3,1.9489819E-3,-5.787963E-3,-2.6578281E-2,-1.0370957E-1,6.426742E-2,7.2744032E-3,-8.344023E-3,2.5514038E-3,-2.5259707E-2,5.906836E-2,-7.113445E-2,-2.8795306E-2,2.002984E-2,1.3444959E-1,-1.67995E-2,-1.8845478E-3,2.9006688E-4,-2.9754702E-2,9.076291E-3,3.287094E-2,4.817574E-2,8.570203E-3,5.6333005E-2,-3.108718E-2,-4.860891E-2,1.4887989E-2,-3.13204E-2,1.7772982E-2,-3.2600358E-2,4.895461E-3,-2.1773203E-1,-5.7062473E-2,7.642179E-2,-8.572969E-5,6.6054384E-3,-7.271436E-3,1.1841906E-2,-6.6126205E-2,-6.571299E-4,7.496469E-2,-1.3422681E-1,-5.2454147E-2,-2.407158E-2,-1.3881284E-1,9.809098E-3,7.669724E-2,1.68512E-1,4.9237546E-4,4.817067E-2,-7.408184E-2,3.6094584E-2,-6.4747413E-3,-7.432675E-2,-2.0476792E-2,3.4173052E-3,-1.170834E-3,5.595192E-2,6.9628158E-3,9.341193E-2,3.420078E-2,1.263626E-2,-8.0078155E-2,-6.803842E-2,-7.701813E-4,3.4006394E-2,-1.768288E-3,-7.0785165E-2,-4.848923E-3,8.685041E-3,9.872076E-3,-2.1914897E-2,-1.1609179E-1,-1.2899893E-2,-5.066486E-3,-8.558656E-2,1.0359401E-2,8.8325165E-2,-3.8056404E-4,2.4966092E-5,8.556932E-2,4.02231E-3,-3.0537058E-2,-5.534171E-3,-1.6980583E-3,-7.907197E-5,9.5482774E-2,-5.2865703E-2,-1.8685848E-1,-2.9336898E-2,-1.3251793E-1,-6.496541E-2,-1.1202616E-2,-1.1486006E-2,-8.8822744E-5,5.014602E-3,1.2773171E-1,8.334962E-2,2.7370313E-4,4.4212565E-3,1.9248548E-1,1.4000106E-2,8.686966E-2,-1.3138001E-2,-1.1487747E-2,9.011924E-2,5.5892714E-3,-2.088208E-2,1.568222E-2,-5.0903037E-2,-9.1016535E-3,4.0123004E-2,-2.552547E-2,9.8235615E-2,4.8252966E-2,4.264322E-3,-3.00882E-3,1.8946395E-3,5.5871564E-3,-1.4585257E-4,6.8558626E-2,-2.8808594E-2,3.837679E-2,-1.0263757E-2,-3.296164E-2,-4.406066E-3,9.823385E-5,-7.338784E-4,2.6506532E-3,-7.2370493E-3,-1.9769462E-2,3.046934E-2,-4.4553027E-2,6.9556735E-2,-4.8055486E-3,-4.4385742E-2,3.73859E-2,-2.0665694E-3,-7.6392954E-3,1.9702417E-4,-1.0935453E-1,-1.3699406E-3,3.0566913E-3,2.594481E-2,1.1754517E-1,-1.2446854E-2,3.1723656E-2,-8.777152E-4,6.424717E-3,-3.108588E-3,4.7431768E-5,5.546759E-3,2.1099972E-3,2.949782E-5,-5.31746E-3,-1.1708139E-2,-3.432931E-3,-7.959621E-2,5.922159E-4,-1.7882587E-1,-7.172974E-4,-3.1256936E-2,-1.1343634E-1,8.698727E-2,-1.8601833E-2,9.161922E-2,-9.5401873E-4,8.789455E-3,2.2173098E-3,6.755244E-4,9.034439E-2,1.1857399E-2,5.028828E-3,3.5047608E-3,-2.193126E-3,5.911614E-3,9.5882703E-4,2.9264165E-2,-3.725094E-3,-3.4006723E-4,1.2438333E-1,-3.8692257E-3,4.476908E-3,-1.667807E-2,-1.2739895E-1,3.818331E-3,1.012883E-2,2.351772E-3,-6.624349E-2,-1.5454758E-2,4.867651E-3,-2.9719876E-2,4.0967368E-2,1.294562E-1,4.438788E-2,9.709297E-2,4.1441493E-2,-1.5118445E-2,2.3304898E-2,9.690017E-4,-1.9584089E-3,4.9371948E-3,1.4247195E-3,-3.0420192E-3,4.5097424E-4,5.00778E-4,3.7231403E-3,-6.6549834E-3,1.5592521E-2,-2.6017944E-3,1.3287953E-3,3.5496475E-3,-1.880636E-3,-4.0983753E-3,4.9898354E-4,1.2394859E-3,5.5876016E-3,-5.7937936E-3,6.11696E-4,-4.437261E-3,-1.0576763E-3,6.818537E-4,3.9010781E-3,-6.6124746E-3,-2.5507687E-3,3.4256203E-3,2.1068502E-4,3.935903E-3,7.201515E-3,-1.0250513E-3,3.99556E-3,-1.8068812E-3,3.5035447E-3,-6.551438E-3,-9.349162E-4,1.4441081E-3,-2.4081657E-3,-1.0537692E-2,-4.7755106E-3,3.145772E-3,-2.851048E-3,-7.7836956E-3,-2.1393087E-3,8.048996E-3,1.8065609E-3,-1.35882E-4,-2.1899415E-3,1.9938757E-3,7.6153944E-3,-2.841962E-4,5.255304E-3,6.6150776E-3,3.2901496E-3,-1.0529885E-3,3.6850076E-3,2.666974E-3,7.219904E-3,-1.0899663E-3,1.0910071E-3,-1.1182482E-3,2.8523458E-3,-8.8423E-3,-2.6159622E-3,-2.182505E-4,1.6612072E-3,-1.8646432E-3,-5.924783E-3,-2.817199E-3,1.8361933E-3,-1.8824022E-3,3.7765992E-4,-1.7222449E-4,3.0524621E-3,7.3769055E-3,2.9679853E-3,8.445193E-4,4.5211217E-3,5.5627017E-3,1.1782296E-3,-4.320671E-3,2.2485869E-3,3.8083372E-4,-1.5130423E-3,2.8757313E-3,-2.645378E-4,-1.6797791E-3,2.424152E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,-1,57,-1,59,61,63,-1,-1,65,67,69,71,73,75,77,-1,79,81,83,-1,85,87,-1,89,91,93,95,97,99,101,-1,103,105,107,-1,109,-1,111,113,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,-1,-1,143,145,147,149,151,153,155,-1,157,-1,159,161,-1,163,165,167,-1,-1,169,171,173,-1,175,177,-1,179,-1,-1,-1,181,183,185,187,189,191,193,-1,-1,195,197,199,-1,-1,201,203,205,207,-1,209,211,213,215,217,-1,219,221,223,225,-1,227,-1,-1,229,231,233,235,-1,237,-1,-1,-1,-1,-1,239,241,243,245,247,249,251,-1,-1,-1,253,-1,-1,255,257,259,261,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,263,265,267,-1,269,271,273,275,277,279,-1,-1,-1,281,-1,-1,-1,-1,-1,-1,283,-1,-1,285,287,-1,289,291,-1,293,-1,295,297,-1,299,301,303,305,307,309,311,313,-1,-1,-1,-1,-1,-1,-1,-1,-1,315,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.288176E-1,2.7388895E-1,3.9463076E-1,3.0007586E-1,1.1014134E-1,1.9413605E-1,9.600818E-2,1.6848645E-1,3.3366513E-1,7.572363E-2,1.3723022E-1,2.1593839E-1,1.3389553E-1,8.975285E-2,1.7006385E-1,1.0549142E-1,1.8662137E-1,5.9542775E-2,1.3995844E-1,0E0,0E0,8.8134564E-2,1.4395249E-1,2.7901006E-1,5.8666444E-1,1.5661366E-1,6.9368176E-2,8.670032E-2,1.5076348E-1,4.719837E-2,0E0,8.163068E-2,0E0,8.0450885E-2,2.3224124E-1,4.6646193E-2,0E0,0E0,1.126189E-1,4.1720815E-2,2.739054E-2,1.3556796E-1,1.2554233E-1,1.7416084E-1,9.5952064E-2,0E0,1.5896416E-1,1.1431494E-1,1.12200275E-1,0E0,2.3601845E-2,8.613974E-2,0E0,2.881018E-2,8.6694054E-2,1.7655179E-2,2.0427916E-2,5.1891305E-2,1.2773716E-1,8.577817E-2,0E0,3.4978688E-2,6.6273645E-2,4.9689263E-2,0E0,9.456863E-2,0E0,4.6103023E-2,1.488518E-2,0E0,2.9992536E-2,1.0574019E-1,1.7194355E-1,1.2504868E-1,1.2437633E-1,1.4466202E-1,2.13373E-2,1.750195E-2,0E0,3.316534E-2,1.6791926E-1,1.2309922E-1,1.2705895E-1,1.0943693E-1,7.1086444E-2,0E0,0E0,6.915754E-2,3.4646392E-2,1.0059774E-2,3.0691981E-2,2.4575459E-2,1.0542993E-1,2.5553495E-2,0E0,1.4687562E-2,0E0,7.269332E-2,4.4828176E-2,0E0,8.8813186E-2,1.2025535E-1,2.392532E-2,0E0,0E0,5.150172E-2,2.279305E-2,7.532874E-2,0E0,6.720738E-2,6.436161E-2,0E0,1.0032723E-2,0E0,0E0,0E0,7.715106E-3,3.4416527E-2,7.362372E-2,1.12863384E-1,1.1234185E-1,8.958556E-2,1.3457277E-1,0E0,0E0,1.2828112E-1,3.2350436E-2,1.9293308E-2,0E0,0E0,2.9630035E-2,5.183104E-2,2.3265332E-2,3.8638655E-2,0E0,9.051491E-2,3.9281845E-2,1.05794504E-1,5.2180056E-2,6.323398E-2,0E0,5.882343E-2,6.0343936E-2,5.141312E-2,6.051472E-2,0E0,1.3098305E-2,0E0,0E0,1.1441541E-2,1.2930982E-2,1.1470346E-2,1.3597773E-2,0E0,7.507318E-2,0E0,0E0,0E0,0E0,0E0,2.08533E-2,4.9647756E-2,3.147225E-2,3.5057925E-2,1.567934E-1,6.5155864E-2,2.3082476E-2,0E0,0E0,0E0,1.870495E-2,0E0,0E0,1.3599643E-2,1.8536657E-2,9.2598855E-2,1.276423E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.53775E-2,6.768765E-2,1.9890577E-2,0E0,8.8674664E-2,6.2105477E-2,4.314229E-2,6.839238E-2,4.4218615E-2,1.17654696E-1,0E0,0E0,0E0,3.0422837E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1707814E-2,0E0,0E0,2.0848662E-2,2.1232447E-2,0E0,9.9273436E-2,2.4211586E-2,0E0,4.8738822E-2,0E0,4.855524E-2,2.3437003E-2,0E0,5.9581444E-2,1.18290335E-2,1.7977297E-2,1.6208332E-2,2.144374E-2,8.22202E-2,9.885201E-3,1.3037782E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9452656E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,34,34,35,35,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,65,65,67,67,68,68,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,97,97,98,98,100,100,101,101,102,102,105,105,106,106,107,107,109,109,110,110,112,112,116,116,117,117,118,118,119,119,120,120,121,121,122,122,125,125,126,126,127,127,130,130,131,131,132,132,133,133,135,135,136,136,137,137,138,138,139,139,141,141,142,142,143,143,144,144,146,146,149,149,150,150,151,151,152,152,154,154,160,160,161,161,162,162,163,163,164,164,165,165,166,166,170,170,173,173,174,174,175,175,176,176,187,187,188,188,189,189,191,191,192,192,193,193,194,194,195,195,196,196,200,200,207,207,210,210,211,211,213,213,214,214,216,216,218,218,219,219,221,221,222,222,223,223,224,224,225,225,226,226,227,227,228,228,238,238],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,-1,58,-1,60,62,64,-1,-1,66,68,70,72,74,76,78,-1,80,82,84,-1,86,88,-1,90,92,94,96,98,100,102,-1,104,106,108,-1,110,-1,112,114,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,-1,-1,144,146,148,150,152,154,156,-1,158,-1,160,162,-1,164,166,168,-1,-1,170,172,174,-1,176,178,-1,180,-1,-1,-1,182,184,186,188,190,192,194,-1,-1,196,198,200,-1,-1,202,204,206,208,-1,210,212,214,216,218,-1,220,222,224,226,-1,228,-1,-1,230,232,234,236,-1,238,-1,-1,-1,-1,-1,240,242,244,246,248,250,252,-1,-1,-1,254,-1,-1,256,258,260,262,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,264,266,268,-1,270,272,274,276,278,280,-1,-1,-1,282,-1,-1,-1,-1,-1,-1,284,-1,-1,286,288,-1,290,292,-1,294,-1,296,298,-1,300,302,304,306,308,310,312,314,-1,-1,-1,-1,-1,-1,-1,-1,-1,316,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.9173258E-4,1.0989723E3,2.41E3,8.494517E5,2E0,2.429717E1,1.7560976E0,9.073025E6,4.838871E8,4.1032645E6,2.1212247E5,1.1037509E2,2.5482938E0,2.5503985E9,1.1846624E-5,1.068E3,1.7911884E3,6.2E1,-1.0739538E-2,-1.9298273E-3,8.61E2,1.9121015E0,2.4013288E1,2.1872402E5,6.8503425E6,4.6E1,1.921032E4,3.7129E4,1.16E2,7.765349E-3,2.4360857E4,-5.787963E-3,6.3686957E2,8.61E2,2.6878788E1,7.2744032E-3,-8.344023E-3,4.6577052E7,4.97E2,1.3325E2,1.2939E4,7.6008E4,4.5723195E6,1.4329944E0,-1.67995E-2,2.185321E0,2.7460318E0,3.494398E8,9.076291E-3,2.6838362E-1,4.0844156E7,8.570203E-3,2.4505E4,6.8503425E6,2.93E2,9.07E2,8.9928055E-1,3.938775E4,6.03368E5,4.895461E-3,2.9E2,1.1490676E6,2.878424E5,-8.572969E-5,4.5505118E2,-7.271436E-3,8.595346E-1,2.2847826E0,-6.571299E-4,1.259E3,2.1818182E0,2.0354E4,6.019075E-4,3.166E3,1.8496495E5,1E0,9.150449E5,4.9237546E-4,2.0447222E8,5.322348E7,9.47306E3,2.579E3,1.1040155E-1,6.1224E4,3.4173052E-3,-1.170834E-3,1.4075E4,1.13E2,1.5227116E6,1.062363E1,1.3860265E6,9.1E3,9.099893E8,-7.701813E-4,4.238096E6,-1.768288E-3,1.7E1,2.8878505E0,8.685041E-3,9.3191874E-1,2.7777777E0,3.97E2,-1.2899893E-2,-5.066486E-3,6.7E1,1.544E3,3.1609525E2,-3.8056404E-4,3.41466E5,1E1,4.02231E-3,6.4506575E5,-5.534171E-3,-1.6980583E-3,-7.907197E-5,3.2801925E6,3.097E3,9.173084E2,1.4713E4,5.4379158E1,1.5595E4,1.48E2,-1.1486006E-2,-8.8822744E-5,1.3E1,4.1384277E2,6.122449E-1,2.7370313E-4,4.4212565E-3,1.3569831E7,5.7835355E6,1.72797E5,4E0,-1.1487747E-2,5.664E1,5.7082694E5,4.4200982E8,1.85369E5,1.4E1,-9.1016535E-3,1.98714E5,2.4472452E7,4.898012E-1,3.154992E7,4.264322E-3,4.995702E8,1.8946395E-3,5.5871564E-3,2.3033286E5,1.903E3,1.838444E6,2.34E0,-1.0263757E-2,5.6202265E6,-4.406066E-3,9.823385E-5,-7.338784E-4,2.6506532E-3,-7.2370493E-3,1.24E2,8.0274E4,3.642857E0,5.08E2,1.3563734E0,1.00011E5,1.91E2,-2.0665694E-3,-7.6392954E-3,1.9702417E-4,3.8758245E-1,-1.3699406E-3,3.0566913E-3,1.0203835E-5,4.784047E6,3.2613106E5,2.222456E0,-8.777152E-4,6.424717E-3,-3.108588E-3,4.7431768E-5,5.546759E-3,2.1099972E-3,2.949782E-5,-5.31746E-3,-1.1708139E-2,-3.432931E-3,1.763E3,7.686371E5,4.222002E8,-7.172974E-4,2.7487562E0,1.02E3,4.1602106E5,1.8012E4,3.6648486E2,5.862126E2,8.789455E-3,2.2173098E-3,6.755244E-4,3.4653E4,1.1857399E-2,5.028828E-3,3.5047608E-3,-2.193126E-3,5.911614E-3,9.5882703E-4,1.48E2,-3.725094E-3,-3.4006723E-4,1.92E2,1.6666666E0,4.476908E-3,1.7997152E7,3.6482175E5,3.818331E-3,1.2287474E3,2.351772E-3,1.2471935E7,2.62E2,4.867651E-3,5.505334E9,2.67E2,2.5E-1,2.8034655E6,7.1E1,3.17E2,1.5694646E3,7.535486E0,9.690017E-4,-1.9584089E-3,4.9371948E-3,1.4247195E-3,-3.0420192E-3,4.5097424E-4,5.00778E-4,3.7231403E-3,-6.6549834E-3,2.0605305E5,-2.6017944E-3,1.3287953E-3,3.5496475E-3,-1.880636E-3,-4.0983753E-3,4.9898354E-4,1.2394859E-3,5.5876016E-3,-5.7937936E-3,6.11696E-4,-4.437261E-3,-1.0576763E-3,6.818537E-4,3.9010781E-3,-6.6124746E-3,-2.5507687E-3,3.4256203E-3,2.1068502E-4,3.935903E-3,7.201515E-3,-1.0250513E-3,3.99556E-3,-1.8068812E-3,3.5035447E-3,-6.551438E-3,-9.349162E-4,1.4441081E-3,-2.4081657E-3,-1.0537692E-2,-4.7755106E-3,3.145772E-3,-2.851048E-3,-7.7836956E-3,-2.1393087E-3,8.048996E-3,1.8065609E-3,-1.35882E-4,-2.1899415E-3,1.9938757E-3,7.6153944E-3,-2.841962E-4,5.255304E-3,6.6150776E-3,3.2901496E-3,-1.0529885E-3,3.6850076E-3,2.666974E-3,7.219904E-3,-1.0899663E-3,1.0910071E-3,-1.1182482E-3,2.8523458E-3,-8.8423E-3,-2.6159622E-3,-2.182505E-4,1.6612072E-3,-1.8646432E-3,-5.924783E-3,-2.817199E-3,1.8361933E-3,-1.8824022E-3,3.7765992E-4,-1.7222449E-4,3.0524621E-3,7.3769055E-3,2.9679853E-3,8.445193E-4,4.5211217E-3,5.5627017E-3,1.1782296E-3,-4.320671E-3,2.2485869E-3,3.8083372E-4,-1.5130423E-3,2.8757313E-3,-2.645378E-4,-1.6797791E-3,2.424152E-3],"split_indices":[102,27,52,2,45,17,53,53,45,7,45,33,56,42,7,39,29,52,0,0,0,2,53,58,33,45,8,4,9,8,0,28,0,46,2,56,0,0,48,2,55,9,10,28,38,0,41,58,5,0,41,45,0,29,45,3,0,53,45,9,0,10,28,33,0,56,0,27,57,0,11,56,9,39,0,33,106,28,0,7,5,48,2,42,29,0,0,9,10,45,54,28,29,7,0,45,0,0,54,0,54,53,2,0,0,10,2,52,0,11,3,0,28,0,0,0,45,2,52,9,58,9,10,0,0,10,4,56,0,0,45,47,1,8,0,47,47,7,1,3,0,1,9,27,12,0,12,0,0,33,0,9,54,0,47,0,0,0,0,0,29,1,54,2,54,5,2,0,0,0,38,0,0,41,45,33,54,0,0,0,0,0,0,0,0,0,0,2,32,7,0,54,0,32,9,4,52,0,0,0,9,0,0,0,0,0,0,10,0,0,12,53,0,9,28,0,55,0,45,11,0,5,8,57,47,8,10,52,53,0,0,0,0,0,0,0,0,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E3,9.89E2,1.531E3,5.64E2,4.25E2,1.128E3,4.03E2,3.07E2,2.57E2,1E1,4.15E2,3.7E2,7.58E2,3.51E2,5.2E1,1.6E2,1.47E2,6.8E1,1.89E2,5E0,5E0,4.8E1,3.67E2,3.23E2,4.7E1,7.44E2,1.4E1,2.74E2,7.7E1,4.7E1,5E0,1.53E2,7E0,1.02E2,4.5E1,5.7E1,1.1E1,4E0,1.85E2,2.6E1,2.2E1,1.2E2,2.47E2,3.01E2,2.2E1,5E0,4.2E1,4.7E2,2.74E2,4E0,1E1,2.69E2,5E0,3.8E1,3.9E1,2.8E1,1.9E1,4.9E1,1.04E2,9.8E1,4E0,1.2E1,3.3E1,4.8E1,9E0,1.81E2,4E0,1.4E1,1.2E1,4E0,1.8E1,2.6E1,9.4E1,2.38E2,9E0,2.56E2,4.5E1,1.7E1,5E0,2.5E1,1.7E1,7.4E1,3.96E2,4.6E1,2.28E2,6E0,4E0,2.26E2,4.3E1,1.3E1,2.5E1,2.1E1,1.8E1,1.7E1,1.1E1,1.4E1,5E0,1.9E1,3E1,4E0,1E2,8.8E1,1E1,8E0,4E0,2.3E1,1E1,4.2E1,6E0,1.68E2,1.3E1,5E0,9E0,4E0,8E0,4E0,1.4E1,1.1E1,1.5E1,7.4E1,2E1,5.6E1,1.82E2,5E0,4E0,2.47E2,9E0,4.1E1,4E0,5E0,1.2E1,1.4E1,1.1E1,1.3E1,4E0,2.6E1,4.8E1,2.4E2,1.56E2,3.9E1,7E0,1.7E1,2.11E2,3.3E1,1.93E2,4E0,3.9E1,4E0,9E0,1.3E1,1.2E1,8E0,1.3E1,4E0,1.4E1,1.3E1,4E0,4E0,1E1,7E0,1.2E1,1.6E1,1.4E1,1.9E1,8.1E1,6.4E1,2.4E1,4E0,6E0,5E0,1.8E1,6E0,4E0,1.4E1,2.8E1,1.21E2,4.7E1,4E0,9E0,4E0,5E0,1E1,4E0,6E0,5E0,1E1,5E0,2.7E1,4.7E1,1.4E1,6E0,3.4E1,2.2E1,1.2E1,1.7E2,1.5E1,2.32E2,5E0,4E0,4E0,3.7E1,7E0,5E0,7E0,7E0,7E0,4E0,8E0,5E0,7E0,1.9E1,4.4E1,4E0,2.32E2,8E0,1.2E1,1.44E2,5E0,3.4E1,9E0,8E0,1.99E2,1.2E1,2E1,1.3E1,2.2E1,1.71E2,2.7E1,1.2E1,9E0,4E0,6E0,6E0,4E0,4E0,8E0,5E0,4E0,1E1,7E0,5E0,1E1,6E0,8E0,6E0,1E1,9E0,1E1,7.1E1,2.1E1,4.3E1,1.6E1,8E0,1.2E1,6E0,4E0,1E1,1.3E1,1.5E1,1.12E2,9E0,1.7E1,3E1,1.4E1,1.3E1,3E1,1.7E1,9E0,5E0,7E0,2.7E1,1.3E1,9E0,4E0,8E0,1.05E2,6.5E1,9E0,6E0,2.23E2,9E0,1.2E1,2.5E1,4E0,4E0,5E0,1.4E1,2.6E1,1.8E1,2.16E2,1.6E1,4E0,4E0,8.9E1,5.5E1,2.3E1,1.1E1,5E0,4E0,1.64E2,3.5E1,4E0,8E0,1.5E1,5E0,9E0,4E0,1.8E1,4E0,4E0,1.67E2,1.1E1,1.6E1,5E0,7E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"317","size_leaf_vector":"1"}},{"base_weights":[-1.3157332E-3,-1.3560789E-2,1.649545E-2,-6.2217796E-3,-4.0042613E-2,3.306455E-2,-1.3731765E-2,-1.3784908E-2,1.803963E-2,-1.790182E-2,-3.5023518E-2,7.145681E-2,2.5203977E-2,-2.8628262E-2,2.9083531E-2,-2.4251716E-2,6.4503695E-3,-3.2810777E-2,3.2159835E-2,-3.044345E-2,-1.0628075E-2,3.760576E-2,1.1559381E-1,7.4042305E-2,1.9641621E-2,-9.989522E-3,-2.536219E-2,3.7885625E-2,-3.6213442E-3,-2.2467209E-2,-9.822375E-3,1.1142905E-2,-8.657902E-2,4.9226386E-3,-5.131665E-2,-3.9505013E-3,3.6148768E-2,1.7999645E-2,-4.369197E-2,6.5901406E-2,-1.4892238E-1,8.0797955E-2,1.514153E-1,3.595704E-2,1.3761911E-1,9.739843E-3,3.9096657E-2,-3.4101952E-2,1.0390227E-2,7.1708085E-3,2.9889874E-2,-2.4484806E-2,8.650795E-2,3.7795953E-2,-5.2861036E-3,-3.0884402E-2,-8.243178E-3,-2.4779454E-2,-1.20430544E-1,6.463072E-2,2.0428935E-2,4.3255646E-2,-7.5805664E-2,-8.859052E-2,-2.8732171E-2,1.1440288E-1,4.6321753E-2,7.971125E-4,-1.4191192E-2,5.8711893E-3,4.821996E-2,8.820758E-3,3.4369482E-3,5.563144E-2,-1.5706263E-3,1.9337869E-1,7.9684526E-2,5.0826534E-3,6.8971813E-3,6.981719E-2,1.3516045E-2,-2.7337693E-2,-8.300196E-2,-2.1217506E-2,2.5486145E-2,8.026084E-2,1.5911259E-2,-2.059922E-2,-7.161507E-2,2.0207716E-4,6.648181E-3,4.4004373E-2,-5.251836E-3,-2.8273625E-2,8.433078E-3,-4.2995913E-3,1.0110085E-3,2.2295002E-2,-4.74317E-2,-6.259068E-2,-1.1172154E-2,1.1542855E-1,4.6875723E-2,6.6351816E-2,7.3013157E-3,9.263947E-2,1.2389473E-2,-7.044791E-3,-5.7252488E-5,-1.9995582E-1,-5.7863727E-2,-3.5565794E-2,3.8163103E-2,6.602978E-3,2.4451122E-3,-2.5111279E-3,5.751579E-2,1.5433044E-4,8.297889E-2,7.587595E-2,-9.410521E-4,3.7347476E-3,1.484328E-2,1.4976797E-3,5.036305E-3,2.3621095E-2,-1.2043456E-2,1.6630441E-1,4.7619715E-2,2.3829501E-2,-8.735642E-3,-7.4357297E-3,-2.3979396E-2,-7.08712E-3,-1.2277528E-1,1.4085531E-3,-2.7592226E-3,3.7482157E-2,-1.1334772E-3,9.7551085E-3,3.3466686E-2,-1.5044788E-2,5.956592E-2,-5.9470665E-3,-3.408307E-2,-4.769819E-2,-8.658162E-3,5.777361E-2,-2.0157395E-2,-6.317941E-3,-7.4891776E-2,-3.6814556E-2,1.882271E-2,4.4298094E-2,-1.3076368E-3,-5.5467924E-3,-1.6449949E-2,-6.0294936E-3,1.5609062E-3,6.398136E-2,7.761467E-3,-3.825437E-2,6.329256E-2,1.1551285E-1,2.4429249E-2,1.2598929E-2,-5.319869E-3,1.1821166E-1,4.500536E-4,-4.014887E-3,3.0524647E-2,-1.2149479E-2,-4.107865E-3,2.2489908E-3,-1.180664E-1,-6.379433E-2,-1.4106422E-2,-2.0557014E-2,6.6642687E-3,6.857762E-2,-7.3720247E-4,1.7385215E-3,5.7246E-3,6.582279E-3,1.5268845E-3,5.3545035E-2,4.3832525E-3,-6.853799E-2,4.534478E-3,3.0737356E-3,1.1852416E-2,8.954873E-2,1.2520797E-2,-4.056616E-2,3.238604E-2,7.078352E-2,-2.966885E-2,-2.1694605E-3,1.217684E-3,-9.872923E-3,-8.455607E-2,4.489235E-3,2.5970835E-2,2.8585042E-3,4.2519145E-4,-7.45203E-2,4.626176E-3,-8.4042567E-4,7.299768E-2,3.6599187E-4,-3.0896922E-3,-3.971002E-3,-9.317854E-4,-4.6268846E-3,8.505399E-4,1.7955775E-3,5.238805E-3,-3.3946857E-3,5.3950225E-4,4.32633E-4,-3.6064743E-3,-6.3888314E-3,-2.081025E-3,-4.3659015E-3,2.4538473E-4,5.5315094E-3,5.124567E-4,-2.1537361E-4,3.868525E-3,-3.23727E-3,4.4442207E-4,5.082221E-3,1.0930414E-3,2.197807E-3,-5.0172447E-3,1.1660332E-3,4.388091E-3,2.156365E-3,7.540766E-3,-1.556197E-3,4.170404E-3,9.3347434E-4,-2.38172E-3,1.3473659E-3,7.3141204E-3,-4.1695757E-4,4.0970524E-3,1.6303301E-3,-3.5068162E-3,-3.2000714E-3,-1.1235416E-2,-1.638921E-3,-5.1690023E-3,4.786837E-4,-2.4625314E-3,1.5532052E-3,-4.9503255E-3,5.790194E-3,2.3440914E-3,-1.0055493E-3,3.7362368E-3,-5.5307295E-4,3.4072434E-3,-4.4031916E-3,1.1187188E-3,-3.925411E-3,5.581913E-4,-3.6280154E-4,5.2629244E-3,-3.6357408E-3,1.2719098E-3,1.6576307E-3,-4.3692384E-3,4.8055714E-3,1.146508E-3,6.049701E-4,5.1257326E-3,-2.4001906E-3,-5.885046E-4,-5.1995968E-3,-1.6256872E-3,1.8967991E-3,-1.0155342E-3,-1.1701569E-3,-5.2307537E-3,-3.3801544E-4,3.2584418E-3,4.923923E-3,2.1885962E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,-1,47,49,-1,51,-1,53,55,-1,57,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,-1,117,-1,-1,119,-1,121,123,-1,125,127,129,131,133,135,137,139,141,143,145,-1,-1,147,-1,149,151,-1,-1,153,155,157,-1,159,161,163,165,167,169,-1,-1,171,173,175,177,-1,-1,-1,179,-1,181,183,-1,-1,-1,-1,-1,185,187,189,191,193,-1,-1,195,197,199,-1,-1,201,-1,-1,203,205,207,209,211,213,-1,215,217,219,221,223,225,227,-1,-1,229,-1,-1,231,-1,233,235,237,239,241,-1,243,-1,-1,245,-1,-1,247,249,251,253,255,-1,257,-1,-1,-1,-1,-1,259,261,263,265,-1,-1,267,269,271,273,275,277,-1,-1,-1,279,-1,281,-1,-1,283,285,-1,287,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.583388E-1,2.9458028E-1,5.2282786E-1,2.1842085E-1,5.129439E-1,2.0160228E-1,2.3681784E-1,1.9225787E-1,2.0425433E-1,0E0,2.5800484E-1,1.6597861E-1,1.5094483E-1,1.4878191E-1,8.8563E-2,1.7947951E-1,1.3629328E-1,1.5573809E-1,1.0135041E-1,2.048786E-1,0E0,3.581579E-1,4.8841476E-2,1.3306797E-1,9.690213E-2,0E0,8.503306E-2,7.0876375E-2,0E0,1.324999E-1,0E0,1.2981321E-1,6.039717E-2,0E0,9.789069E-2,0E0,9.494197E-2,1.6675016E-1,1.6537291E-1,4.9779892E-2,2.0498903E-1,2.661924E-2,3.5593152E-2,5.0805815E-2,5.242926E-2,8.715265E-2,1.3265085E-1,6.98393E-2,2.6306732E-2,0E0,5.7868734E-2,1.052888E-1,4.168234E-2,1.048788E-1,5.826651E-2,2.994026E-2,0E0,4.477752E-2,7.550801E-2,6.313473E-2,8.391568E-2,8.2688175E-2,7.283802E-2,2.0058748E-1,8.775617E-2,1.2120813E-2,4.937762E-2,0E0,0E0,0E0,2.4843153E-2,0E0,0E0,4.5146763E-2,0E0,1.0080156E-1,9.570904E-3,0E0,1.0386552E-1,1.5699708E-1,1.867604E-1,7.5166434E-2,7.763387E-2,3.179733E-2,2.2231003E-2,9.030611E-2,9.143833E-2,1.0648371E-1,9.871997E-2,0E0,0E0,9.746151E-2,0E0,7.015064E-2,5.5225268E-2,0E0,0E0,1.6088666E-2,5.3327248E-2,6.3715115E-2,0E0,2.9067963E-2,8.348012E-2,6.106776E-2,6.763925E-2,4.3847173E-2,6.131816E-2,0E0,0E0,4.612857E-2,1.8121287E-1,1.0329193E-1,1.03376396E-1,0E0,0E0,0E0,3.218974E-2,0E0,9.7130835E-3,5.4195344E-2,0E0,0E0,0E0,0E0,0E0,9.9932246E-2,1.4454144E-1,8.942431E-2,9.306522E-2,5.090411E-2,0E0,0E0,1.0399931E-1,1.2486767E-2,3.1901777E-2,0E0,0E0,1.5749495E-2,0E0,0E0,7.8084935E-3,4.7142364E-2,2.995564E-2,1.9330364E-1,1.953839E-1,1.0909908E-1,0E0,8.874977E-2,3.0314803E-2,4.7969215E-2,3.3414833E-2,4.684282E-2,7.36413E-2,1.6434781E-2,0E0,0E0,2.5078457E-2,0E0,0E0,1.366194E-2,0E0,5.5534855E-2,4.5972645E-2,2.9350206E-2,6.1653472E-2,3.974939E-2,0E0,3.717695E-2,0E0,0E0,6.0703658E-2,0E0,0E0,5.9501432E-2,1.2982756E-1,8.755723E-2,8.2961015E-2,5.17359E-2,0E0,2.9407114E-2,0E0,0E0,0E0,0E0,0E0,1.07854664E-1,1.0625174E-1,6.4300075E-2,6.705583E-2,0E0,0E0,4.5066684E-2,4.1563667E-2,4.0925086E-2,4.597E-2,1.9282553E-2,5.825399E-2,0E0,0E0,0E0,9.805091E-3,0E0,1.52041055E-2,0E0,0E0,1.2962971E-2,2.1835376E-2,0E0,1.35845095E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,26,26,27,27,29,29,31,31,32,32,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,70,70,73,73,75,75,76,76,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,91,91,93,93,94,94,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,106,106,109,109,110,110,111,111,112,112,116,116,118,118,119,119,125,125,126,126,127,127,128,128,129,129,132,132,133,133,134,134,137,137,140,140,141,141,142,142,143,143,144,144,145,145,147,147,148,148,149,149,150,150,151,151,152,152,153,153,156,156,159,159,161,161,162,162,163,163,164,164,165,165,167,167,170,170,173,173,174,174,175,175,176,176,177,177,179,179,185,185,186,186,187,187,188,188,191,191,192,192,193,193,194,194,195,195,196,196,200,200,202,202,205,205,206,206,208,208],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,-1,48,50,-1,52,-1,54,56,-1,58,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,-1,118,-1,-1,120,-1,122,124,-1,126,128,130,132,134,136,138,140,142,144,146,-1,-1,148,-1,150,152,-1,-1,154,156,158,-1,160,162,164,166,168,170,-1,-1,172,174,176,178,-1,-1,-1,180,-1,182,184,-1,-1,-1,-1,-1,186,188,190,192,194,-1,-1,196,198,200,-1,-1,202,-1,-1,204,206,208,210,212,214,-1,216,218,220,222,224,226,228,-1,-1,230,-1,-1,232,-1,234,236,238,240,242,-1,244,-1,-1,246,-1,-1,248,250,252,254,256,-1,258,-1,-1,-1,-1,-1,260,262,264,266,-1,-1,268,270,272,274,276,278,-1,-1,-1,280,-1,282,-1,-1,284,286,-1,288,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.862126E2,1.2471935E7,2.8091298E7,9.8376294E5,1.2673605E7,1.0779661E1,3.4409692E3,1.143316E6,1.6633E4,-1.790182E-2,2.79495E-1,9.60094E5,1.1317E4,8.890291E2,1.4299594E8,1.113742E6,1E0,1.1251919E6,1.198E5,1.3617021E0,-1.0628075E-2,3.64299E5,1.0201212E3,2.1169072E3,2.0201733E9,-9.989522E-3,9.5E1,2.909019E3,-3.6213442E-3,1.1884672E7,-9.822375E-3,1E0,3.051499E5,4.9226386E-3,1.13E2,-3.9505013E-3,3.3057434E4,1.30024E6,2.1631205E0,2.072E3,9.9625344E5,7.115101E2,8.606353E1,9.075E3,3.2983315E-1,1.15E0,1.2936076E7,4.1822004E7,1.0855529E3,7.1708085E-3,3.0715862E7,1E0,1.0119178E10,2.0316172E6,1.1803029E8,1.24E2,-8.243178E-3,6.342E3,2.065904E6,1.06E2,1.1376811E0,1.4E1,1.1E1,1.4713E4,1E0,3.0121632E6,3.928E4,7.971125E-4,-1.4191192E-2,5.8711893E-3,7.697189E2,8.820758E-3,3.4369482E-3,1.1900813E1,-1.5706263E-3,3.3743455E0,1.8677662E7,5.0826534E-3,6.8247475E6,2.231E3,1.921032E4,3E0,1.6E1,1.7921995E3,7.67436E5,4.20737E5,1E0,7.451E3,1.8302105E6,2.0207716E-4,6.648181E-3,1.0770323E2,-5.251836E-3,2.1934064E7,2E0,-4.2995913E-3,1.0110085E-3,1.2897E4,1.4315012E3,1.6254681E0,-1.1172154E-2,2.9375E0,1.77E2,4.29E2,1E0,1.6709303E1,6.6126086E4,-7.044791E-3,-5.7252488E-5,7.5677685E-2,1.9928571E0,5.61E2,2.2E1,6.602978E-3,2.4451122E-3,-2.5111279E-3,7.9527026E-1,1.5433044E-4,7.2201815E6,4.3359457E10,-9.410521E-4,3.7347476E-3,1.484328E-2,1.4976797E-3,5.036305E-3,6.162955E0,4.576E3,9.000245E4,5.057096E-1,2.0907634E3,-8.735642E-3,-7.4357297E-3,8E0,6.131E3,2.980289E-1,1.4085531E-3,-2.7592226E-3,1.06E2,-1.1334772E-3,9.7551085E-3,1.348651E6,6.77603E5,3.5985E4,4.13568E5,1.06E3,1.70752E4,-8.658162E-3,3.9552E2,5.8892828E7,2.1E1,5.236559E0,4.2986497E5,1.328903E8,5.85E2,-1.3076368E-3,-5.5467924E-3,4.09E2,-6.0294936E-3,1.5609062E-3,1E0,7.761467E-3,4.7048388E0,7.22E2,5E0,2.342E3,1.9407146E7,-5.319869E-3,1.6559546E0,4.500536E-4,-4.014887E-3,1E0,-1.2149479E-2,-4.107865E-3,7.686371E5,2.5247778E6,1.0115874E4,2.8276239E-2,1.945711E1,6.6642687E-3,1E1,-7.3720247E-4,1.7385215E-3,5.7246E-3,6.582279E-3,1.5268845E-3,1E0,7.804304E-1,5E0,8.422367E6,3.0737356E-3,1.1852416E-2,8.027061E2,2.637815E1,1.249E3,5.244755E-3,5.1160636E11,6.015E3,-2.1694605E-3,1.217684E-3,-9.872923E-3,9.287868E6,4.489235E-3,5.980516E5,2.8585042E-3,4.2519145E-4,2.4848E4,9.74354E7,-8.4042567E-4,4.719849E7,3.6599187E-4,-3.0896922E-3,-3.971002E-3,-9.317854E-4,-4.6268846E-3,8.505399E-4,1.7955775E-3,5.238805E-3,-3.3946857E-3,5.3950225E-4,4.32633E-4,-3.6064743E-3,-6.3888314E-3,-2.081025E-3,-4.3659015E-3,2.4538473E-4,5.5315094E-3,5.124567E-4,-2.1537361E-4,3.868525E-3,-3.23727E-3,4.4442207E-4,5.082221E-3,1.0930414E-3,2.197807E-3,-5.0172447E-3,1.1660332E-3,4.388091E-3,2.156365E-3,7.540766E-3,-1.556197E-3,4.170404E-3,9.3347434E-4,-2.38172E-3,1.3473659E-3,7.3141204E-3,-4.1695757E-4,4.0970524E-3,1.6303301E-3,-3.5068162E-3,-3.2000714E-3,-1.1235416E-2,-1.638921E-3,-5.1690023E-3,4.786837E-4,-2.4625314E-3,1.5532052E-3,-4.9503255E-3,5.790194E-3,2.3440914E-3,-1.0055493E-3,3.7362368E-3,-5.5307295E-4,3.4072434E-3,-4.4031916E-3,1.1187188E-3,-3.925411E-3,5.581913E-4,-3.6280154E-4,5.2629244E-3,-3.6357408E-3,1.2719098E-3,1.6576307E-3,-4.3692384E-3,4.8055714E-3,1.146508E-3,6.049701E-4,5.1257326E-3,-2.4001906E-3,-5.885046E-4,-5.1995968E-3,-1.6256872E-3,1.8967991E-3,-1.0155342E-3,-1.1701569E-3,-5.2307537E-3,-3.3801544E-4,3.2584418E-3,4.923923E-3,2.1885962E-3],"split_indices":[52,45,45,28,45,56,52,9,1,0,38,9,9,4,45,9,74,28,7,53,0,9,4,52,5,0,8,32,0,45,0,16,28,0,10,0,51,9,54,29,28,55,58,2,39,54,45,51,52,0,50,105,19,47,7,0,0,12,28,0,53,3,8,9,105,28,12,0,0,0,4,0,0,54,0,54,51,0,51,0,4,3,3,4,11,29,102,29,32,0,0,54,0,5,8,0,0,1,33,53,0,54,0,10,74,58,28,0,0,39,54,10,3,0,0,0,27,0,51,19,0,0,0,0,0,53,2,33,39,4,0,0,3,2,27,0,0,8,0,0,29,29,2,9,2,33,0,4,5,3,54,33,7,2,0,0,29,0,0,16,0,58,2,3,2,9,0,56,0,0,8,0,0,32,28,33,39,56,0,3,0,0,0,0,0,102,39,6,50,0,0,52,56,0,57,31,2,0,0,0,1,0,28,0,0,2,43,0,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.558E3,1.516E3,1.042E3,1.188E3,3.28E2,6.73E2,3.69E2,9.06E2,2.82E2,4E0,3.24E2,1.13E2,5.6E2,2.74E2,9.5E1,5.97E2,3.09E2,6.1E1,2.21E2,3.17E2,7E0,6.5E1,4.8E1,5.6E1,5.04E2,4E0,2.7E2,8.8E1,7E0,5.92E2,5E0,2.95E2,1.4E1,7E0,5.4E1,7E0,2.14E2,6.8E1,2.49E2,5.7E1,8E0,2.6E1,2.2E1,3.6E1,2E1,3.35E2,1.69E2,2.17E2,5.3E1,5E0,8.3E1,5.82E2,1E1,1.12E2,1.83E2,9E0,5E0,4E1,1.4E1,7.5E1,1.39E2,5.4E1,1.4E1,6.1E1,1.88E2,1.5E1,4.2E1,4E0,4E0,1.1E1,1.5E1,1.6E1,6E0,2.8E1,8E0,9E0,1.1E1,9E0,3.26E2,7.6E1,9.3E1,1.92E2,2.5E1,1.7E1,3.6E1,1.7E1,6.6E1,5.39E2,4.3E1,4E0,6E0,1.08E2,4E0,6.8E1,1.15E2,4E0,5E0,1.3E1,2.7E1,1E1,4E0,1.8E1,5.7E1,3E1,1.09E2,2E1,3.4E1,7E0,7E0,1.2E1,4.9E1,1.71E2,1.7E1,1.1E1,4E0,4E0,3.8E1,7E0,8E0,2.2E1,6E0,5E0,4E0,4E0,7E0,1.73E2,1.53E2,1.3E1,6.3E1,8.9E1,4E0,4E0,1.88E2,9E0,1.6E1,7E0,1E1,2.9E1,7E0,4E0,1.3E1,3.9E1,2.7E1,2.59E2,2.8E2,3.6E1,7E0,8.9E1,1.9E1,4.7E1,2.1E1,2.1E1,9.4E1,9E0,4E0,8E0,1.9E1,6E0,4E0,9E0,9E0,9E0,4.8E1,1.3E1,1.7E1,1.05E2,4E0,1.5E1,5E0,5E0,2.9E1,8E0,4E0,2.5E1,2.4E1,7.3E1,9.8E1,1.1E1,6E0,3.3E1,5E0,4E0,4E0,9E0,1.3E1,6.7E1,1.06E2,3.4E1,1.19E2,6E0,7E0,2.8E1,3.5E1,1E1,7.9E1,1E1,1.78E2,4E0,5E0,4E0,1.2E1,4E0,2.5E1,6E0,7E0,9E0,3E1,4E0,2.3E1,2.1E2,4.9E1,7E1,2.1E2,2.1E1,1.5E1,6.2E1,2.7E1,7E0,1.2E1,3.9E1,8E0,7E0,1.4E1,9E0,1.2E1,7E0,8.7E1,4E0,5E0,6E0,1.3E1,4E0,5E0,4E0,5E0,1.9E1,2.9E1,5E0,8E0,9E0,8E0,9.6E1,9E0,4E0,1.1E1,1.7E1,1.2E1,1.8E1,7E0,1.7E1,7E0,4.2E1,3.1E1,5.9E1,3.9E1,7E0,4E0,9E0,2.4E1,1.5E1,5.2E1,8.6E1,2E1,2.8E1,6E0,8E0,1.11E2,4E0,2.4E1,4E0,3.1E1,4E0,6E0,9E0,7E1,4E0,6E0,8.7E1,9.1E1,8E0,4E0,2E1,5E0,4E0,5E0,2.6E1,4E0,1.1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"289","size_leaf_vector":"1"}},{"base_weights":[-1.7950897E-3,6.1761327E-3,-2.391905E-2,-7.7816662E-3,1.8520996E-2,-3.820143E-2,-9.5371425E-4,-4.9862843E-3,-7.3489934E-2,7.0396264E-4,3.1657897E-2,-1.4390041E-1,-3.4182403E-2,-7.822158E-3,4.844303E-2,2.0723052E-3,-2.6205124E-2,5.115913E-3,-1.0515989E-1,-1.5141218E-3,5.82166E-3,6.861111E-3,2.916201E-2,-1.8790364E-1,-2.3273625E-3,-2.762517E-2,-9.004176E-2,6.8629226E-3,-2.6981393E-2,-1.1677323E-3,5.9536718E-2,5.965327E-3,-6.948053E-2,-4.8560776E-2,7.7681546E-3,-1.9536408E-2,-1.4500268E-1,4.0391242E-4,-7.995892E-3,5.598504E-2,1.9723434E-2,-1.1993005E-2,-4.3862313E-3,-3.605358E-2,2.8437385E-2,-1.4845595E-2,-1.3126756E-1,-2.9863639E-2,1.9534761E-2,-9.704058E-2,-1.811881E-2,6.2356363E-3,3.992402E-2,8.020774E-3,-1.3127056E-1,-2.760962E-2,-1.4911837E-2,-9.78792E-2,-2.2459937E-2,-5.5451173E-4,7.047385E-3,-3.572982E-3,2.8546024E-3,-1.9296733E-1,-2.1455288E-3,-1.8021126E-3,6.442E-3,4.1025512E-2,8.476155E-2,7.435026E-2,1.3906029E-2,-2.3675177E-2,-6.417413E-2,7.328872E-2,3.7214488E-3,2.5329864E-2,-5.411269E-3,-8.9833036E-2,-1.4770791E-2,-1.2906465E-2,-5.33874E-3,9.072175E-2,5.576609E-3,-1.1269398E-3,-6.814577E-3,-1.0662514E-2,-6.569619E-2,7.015255E-2,5.699437E-4,3.8616334E-3,7.4813195E-2,-1.2660663E-2,8.541898E-4,-1.7304072E-1,4.6486408E-2,-6.549101E-2,-2.1655649E-1,-3.890785E-2,5.0903432E-2,1.1964029E-2,-5.643369E-2,-1.2383022E-2,-3.923312E-3,7.6404057E-4,-1.0401141E-1,6.864626E-2,9.516376E-4,9.764398E-2,-1.1910317E-3,5.1698864E-2,1.3713457E-1,-4.191893E-2,2.1363394E-2,-4.292044E-2,1.2158067E-2,-1.2132009E-1,-4.005441E-2,3.5731126E-2,7.8088264E-3,3.996413E-3,-1.002934E-2,2.130245E-3,-3.7083504E-4,8.4804196E-4,-1.3762106E-1,1.7999328E-3,-3.32615E-2,1.1840221E-1,3.0700982E-4,-3.8318377E-2,1.7463475E-2,2.2330236E-2,-2.461186E-2,-4.635586E-3,-4.6134897E-4,5.3864536E-3,2.0707212E-3,-1.2091578E-3,1.7554727E-3,2.2808824E-2,-8.142113E-3,1.4609037E-1,1.4050793E-2,-1.996904E-3,-1.489625E-2,-1.9267824E-3,7.015844E-2,-9.489353E-2,1.2218183E-3,-1.6441338E-2,-3.035753E-3,-1.0145591E-2,-8.719223E-2,6.2171803E-3,-2.0106396E-2,2.4355764E-2,-5.333533E-2,-5.0457087E-3,-2.4061637E-2,-6.872334E-3,3.738543E-2,-7.717964E-3,-2.230203E-3,-8.2559476E-4,8.462678E-2,-5.399513E-3,2.2727236E-2,5.8557656E-2,1.12146065E-1,5.2873143E-3,2.8274488E-2,8.850521E-3,3.0836896E-3,-6.258034E-2,1.3753792E-2,5.6029286E-2,7.204586E-3,-2.8845578E-2,-1.1209285E-1,-8.267564E-3,2.3995096E-2,-3.9097053E-4,-1.3990998E-1,4.9997922E-2,-5.4395556E-2,3.3156553E-3,4.1447702E-4,-1.813346E-2,1.5427859E-3,-3.4949454E-3,-8.095799E-3,-2.5275722E-3,1.6134871E-3,2.5631702E-3,7.944993E-3,-4.039461E-3,-3.907809E-3,1.1264876E-2,4.5386427E-3,-2.2986382E-3,5.3173713E-2,-4.6310037E-2,-1.2597557E-3,7.9825986E-4,6.4884857E-3,1.0853407E-3,-1.203942E-3,2.1854455E-3,1.0149627E-2,2.2983884E-3,-3.4057335E-3,4.8489417E-3,4.5849493E-4,-6.3295085E-3,2.461179E-4,-2.1077935E-3,1.9113376E-3,-1.155965E-3,3.1814678E-3,-5.9360033E-3,7.7796227E-4,9.734032E-4,-2.7817725E-3,-1.5053791E-3,1.7587801E-3,-4.2272597E-3,5.5188673E-5,1.6343058E-3,-3.365721E-3,3.0661759E-4,-1.1691245E-3,4.4356957E-3,8.7481446E-4,1.8515665E-3,5.803948E-3,6.5518645E-5,3.8042662E-3,4.864408E-3,1.3085094E-3,6.1538205E-3,1.5114726E-3,3.092883E-4,3.094962E-3,-2.0605584E-3,-6.2017753E-3,3.4128495E-3,-1.4428865E-3,1.8559133E-3,5.932339E-3,-3.1288448E-5,2.720083E-3,-3.1089634E-3,-1.819043E-4,-6.5667927E-3,-6.171802E-4,3.235425E-3,2.1710726E-4,-9.793117E-3,-4.656592E-3,3.5079027E-4,3.8713906E-3,-7.3437463E-3,-2.1639164E-3,-1.3790268E-3,7.426437E-4,1.3544954E-3,-1.2743342E-3,1.1961415E-3,-1.9160949E-3,4.0741316E-3,5.532758E-4,2.9709566E-4,-2.7436106E-3,-2.1405006E-3,4.8176176E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,-1,-1,39,41,-1,43,45,47,49,-1,51,53,55,57,59,61,63,65,-1,67,69,-1,-1,71,73,75,77,79,81,83,85,-1,87,89,91,93,-1,95,97,99,-1,-1,-1,101,-1,103,-1,105,107,109,111,113,115,117,119,121,-1,123,-1,125,-1,127,129,-1,-1,131,133,135,137,139,141,-1,-1,143,145,147,149,151,153,155,157,-1,-1,159,161,163,165,167,-1,169,171,173,175,177,179,181,183,185,-1,-1,187,-1,-1,-1,189,-1,191,193,-1,195,197,199,201,-1,-1,-1,-1,-1,-1,203,205,207,209,-1,-1,-1,211,213,215,-1,-1,217,219,-1,221,223,225,-1,227,229,231,-1,-1,-1,233,-1,235,237,239,-1,241,-1,-1,243,245,247,249,251,253,-1,255,-1,257,259,261,-1,-1,263,-1,-1,-1,-1,-1,-1,-1,-1,265,267,-1,-1,269,271,273,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.514402E-1,3.244261E-1,2.2234729E-1,1.6200641E-1,2.3380426E-1,1.7265046E-1,8.893068E-2,1.2721334E-1,2.1128698E-1,1.0934595E-1,1.476261E-1,5.548507E-2,1.4549077E-1,6.489681E-2,2.688495E-2,1.7813367E-1,1.6139013E-1,0E0,1.02922946E-1,1.271485E-1,0E0,0E0,1.4143118E-1,3.1109542E-2,0E0,1.7254937E-1,1.2825221E-1,6.154987E-2,6.033226E-2,0E0,3.0855104E-2,1.7199406E-1,3.0811682E-1,1.6256991E-1,9.435948E-2,4.7913946E-2,9.451836E-2,1.176108E-1,0E0,5.937314E-2,1.3180077E-1,0E0,0E0,1.0847634E-1,5.2937314E-2,6.281478E-2,1.5730265E-1,4.300472E-2,9.6870676E-2,2.9159002E-2,3.1227108E-2,0E0,2.7321182E-2,1.6583496E-1,1.673863E-1,3.2054415E-1,0E0,1.5494215E-1,1.04547866E-1,5.7336755E-2,0E0,0E0,0E0,6.84396E-2,0E0,1.07091635E-1,0E0,1.08407974E-1,7.087886E-2,4.9796402E-2,1.5867592E-1,1.5292789E-1,1.2746167E-1,4.740455E-2,3.4358542E-2,7.116984E-3,0E0,1.1972797E-1,0E0,3.0080836E-2,0E0,3.6730796E-2,4.39764E-2,0E0,0E0,3.6907546E-2,1.748987E-2,9.584263E-3,1.0267261E-2,1.2846033E-1,1.4953138E-1,0E0,0E0,1.5454108E-1,4.2970456E-2,7.0730835E-2,1.3774252E-1,9.6329115E-2,8.6035594E-2,5.551357E-2,1.9201785E-2,0E0,0E0,1.1183914E-1,2.3062259E-2,8.0339044E-2,9.978355E-2,1.8373668E-2,0E0,3.630615E-2,1.9231811E-2,5.2623183E-2,1.6402362E-1,1.369639E-1,1.6796441E-1,5.798155E-2,9.1829985E-2,1.0372976E-2,0E0,0E0,9.842013E-3,0E0,0E0,0E0,1.5090823E-2,0E0,2.5481362E-2,2.453968E-2,0E0,2.6023487E-2,2.899297E-2,5.3029723E-2,2.8316751E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5811357E-1,1.653594E-1,8.4522426E-2,5.5664144E-2,0E0,0E0,0E0,2.5341824E-2,7.993302E-2,2.0901665E-2,0E0,0E0,4.4620108E-2,8.62976E-2,0E0,1.3756325E-2,3.473495E-2,1.8816885E-2,0E0,2.6783094E-2,7.130191E-2,6.892627E-2,0E0,0E0,0E0,6.9190174E-2,0E0,3.993328E-2,1.5031856E-2,2.4232775E-2,0E0,1.6717121E-2,0E0,0E0,3.822115E-2,3.2203283E-2,1.1044833E-1,8.843392E-2,1.0156153E-1,4.465002E-2,0E0,5.8982946E-2,0E0,4.4614047E-2,1.1266917E-2,5.5226848E-2,0E0,0E0,7.905999E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.059299E-3,3.99025E-2,0E0,0E0,1.939714E-2,1.3357263E-2,1.3104541E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,22,22,23,23,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,57,57,58,58,59,59,63,63,65,65,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,77,77,79,79,81,81,82,82,85,85,86,86,87,87,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,103,103,104,104,105,105,106,106,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,120,120,124,124,126,126,127,127,129,129,130,130,131,131,132,132,139,139,140,140,141,141,142,142,146,146,147,147,148,148,151,151,152,152,154,154,155,155,156,156,158,158,159,159,160,160,164,164,166,166,167,167,168,168,170,170,173,173,174,174,175,175,176,176,177,177,178,178,180,180,182,182,183,183,184,184,187,187,196,196,197,197,200,200,201,201,202,202],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,-1,-1,40,42,-1,44,46,48,50,-1,52,54,56,58,60,62,64,66,-1,68,70,-1,-1,72,74,76,78,80,82,84,86,-1,88,90,92,94,-1,96,98,100,-1,-1,-1,102,-1,104,-1,106,108,110,112,114,116,118,120,122,-1,124,-1,126,-1,128,130,-1,-1,132,134,136,138,140,142,-1,-1,144,146,148,150,152,154,156,158,-1,-1,160,162,164,166,168,-1,170,172,174,176,178,180,182,184,186,-1,-1,188,-1,-1,-1,190,-1,192,194,-1,196,198,200,202,-1,-1,-1,-1,-1,-1,204,206,208,210,-1,-1,-1,212,214,216,-1,-1,218,220,-1,222,224,226,-1,228,230,232,-1,-1,-1,234,-1,236,238,240,-1,242,-1,-1,244,246,248,250,252,254,-1,256,-1,258,260,262,-1,-1,264,-1,-1,-1,-1,-1,-1,-1,-1,266,268,-1,-1,270,272,274,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,3.8637936E2,3.008162E6,2E0,1.5166431E5,8.494517E5,9.637832E1,5.2187E4,3.5E1,4.9722066E0,1.9E1,2.5506506E0,4.1043E4,2.99665E5,6.624E3,1.3569831E7,1.023747E6,5.115913E-3,2.20524E5,3.7826266E4,5.82166E-3,6.861111E-3,2.2579439E0,2.5369E4,-2.3273625E-3,1.998175E3,1.2E1,2.92E2,1.9711539E0,-1.1677323E-3,3.4526074E-1,2.1196339E1,3.6129813E2,4.9008E4,3.7075245E1,9E0,1.0406779E1,1.1458888E4,-7.995892E-3,1.3715873E3,1E0,-1.1993005E-2,-4.3862313E-3,2.8035452E11,2.5927516E11,3.3502305E0,1.5265896E8,1.3446785E7,2.6540773E1,2.090909E0,1.5988282E1,6.2356363E-3,1.772114E6,1.4E1,6.558106E-3,7.029851E0,-1.4911837E-2,1.1457406E-1,1.8797805E6,2.8863275E0,7.047385E-3,-3.572982E-3,2.8546024E-3,2.3189322E2,-2.1455288E-3,2.851369E2,6.442E-3,7.6842415E6,2.609323E3,1.10032365E-1,9.7549E4,1E0,9.08912E5,9.5E2,2.322963E0,1.75E2,-5.411269E-3,5.3474556E5,-1.4770791E-2,9.58851E4,-5.33874E-3,9.2057055E-1,2.342E3,-1.1269398E-3,-6.814577E-3,7.88E2,3.0715862E7,1.5137865E2,3.0769388E7,1.5182724E0,1.59551E5,-1.2660663E-2,8.541898E-4,6.62E2,2.2589723E2,4.6253732E2,6.4863464E2,1.0819753E5,2.6624084E-1,2.011653E7,2.064332E2,-1.2383022E-2,-3.923312E-3,1.3359244E9,1.0216753E0,2.0354E4,3.114E3,1.3E1,-1.1910317E-3,2.2729468E0,6.4683E4,2.43406E5,1.34748E8,1.9041E4,2.1934064E7,6.7033327E-1,7.080933E2,6.822511E5,7.8088264E-3,3.996413E-3,1.921032E4,2.130245E-3,-3.7083504E-4,8.4804196E-4,8.988928E5,1.7999328E-3,1.4229508E1,4.116997E6,3.0700982E-4,2.9E1,3.6207435E0,2.37E3,1.5119754E0,-4.635586E-3,-4.6134897E-4,5.3864536E-3,2.0707212E-3,-1.2091578E-3,1.7554727E-3,2.8034655E6,4.4195585E0,5.555794E0,3.0273972E0,-1.996904E-3,-1.489625E-2,-1.9267824E-3,2.56E2,8.0708375E5,1.0134536E7,-1.6441338E-2,-3.035753E-3,1.3411796E3,6.685704E6,6.2171803E-3,5.48744E8,9.095E3,4.456E3,-5.0457087E-3,3.342857E2,1.07E2,1.1795429E9,-7.717964E-3,-2.230203E-3,-8.2559476E-4,1.2062E5,-5.399513E-3,1.6E1,6E0,3.73802E5,5.2873143E-3,3.3487454E11,8.850521E-3,3.0836896E-3,7.621583E2,2.74872E5,8.027061E2,5.3534385E3,1.04E3,1.2323925E6,-8.267564E-3,1.5773E4,-3.9097053E-4,1.2E1,1E0,8.890291E2,3.3156553E-3,4.1447702E-4,4.8E1,1.5427859E-3,-3.4949454E-3,-8.095799E-3,-2.5275722E-3,1.6134871E-3,2.5631702E-3,7.944993E-3,-4.039461E-3,1.563E3,9.170864E1,4.5386427E-3,-2.2986382E-3,7.137324E0,2.6575577E-1,9.64E2,7.9825986E-4,6.4884857E-3,1.0853407E-3,-1.203942E-3,2.1854455E-3,1.0149627E-2,2.2983884E-3,-3.4057335E-3,4.8489417E-3,4.5849493E-4,-6.3295085E-3,2.461179E-4,-2.1077935E-3,1.9113376E-3,-1.155965E-3,3.1814678E-3,-5.9360033E-3,7.7796227E-4,9.734032E-4,-2.7817725E-3,-1.5053791E-3,1.7587801E-3,-4.2272597E-3,5.5188673E-5,1.6343058E-3,-3.365721E-3,3.0661759E-4,-1.1691245E-3,4.4356957E-3,8.7481446E-4,1.8515665E-3,5.803948E-3,6.5518645E-5,3.8042662E-3,4.864408E-3,1.3085094E-3,6.1538205E-3,1.5114726E-3,3.092883E-4,3.094962E-3,-2.0605584E-3,-6.2017753E-3,3.4128495E-3,-1.4428865E-3,1.8559133E-3,5.932339E-3,-3.1288448E-5,2.720083E-3,-3.1089634E-3,-1.819043E-4,-6.5667927E-3,-6.171802E-4,3.235425E-3,2.1710726E-4,-9.793117E-3,-4.656592E-3,3.5079027E-4,3.8713906E-3,-7.3437463E-3,-2.1639164E-3,-1.3790268E-3,7.426437E-4,1.3544954E-3,-1.2743342E-3,1.1961415E-3,-1.9160949E-3,4.0741316E-3,5.532758E-4,2.9709566E-4,-2.7436106E-3,-2.1405006E-3,4.8176176E-4],"split_indices":[27,52,32,6,28,45,56,29,0,41,10,41,10,29,2,45,1,0,9,52,0,0,54,29,0,52,18,0,53,0,27,54,52,9,56,8,56,52,0,52,64,0,0,31,31,53,45,45,58,56,54,0,29,56,38,58,0,42,48,53,0,0,0,52,0,58,0,47,52,57,12,16,1,0,53,8,0,32,0,47,0,27,2,0,0,0,50,56,1,53,9,0,0,2,52,4,4,47,42,32,52,0,0,7,39,9,29,3,0,53,2,1,7,9,5,27,4,32,0,0,4,0,0,0,32,0,56,12,0,3,57,10,57,0,0,0,0,0,0,47,58,58,53,0,0,0,0,28,45,0,0,4,32,0,5,9,0,0,52,3,5,0,0,0,1,0,8,8,33,0,31,0,0,52,29,52,4,2,32,0,9,0,3,101,4,0,0,8,0,0,0,0,0,0,0,0,2,58,0,0,53,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.558E3,1.881E3,6.77E2,8.83E2,9.98E2,4.17E2,2.6E2,8.48E2,3.5E1,4.24E2,5.74E2,1.4E1,4.03E2,2.29E2,3.1E1,6.37E2,2.11E2,5E0,3E1,4.17E2,7E0,1.2E1,5.62E2,9E0,5E0,3.62E2,4.1E1,1.3E2,9.9E1,4E0,2.7E1,6.05E2,3.2E1,1.27E2,8.4E1,1E1,2E1,4.13E2,4E0,1.45E2,4.17E2,5E0,4E0,3.15E2,4.7E1,1.5E1,2.6E1,3.3E1,9.7E1,1E1,8.9E1,5E0,2.2E1,5.97E2,8E0,2.8E1,4E0,4.3E1,8.4E1,8E1,4E0,6E0,4E0,1.3E1,7E0,4.07E2,6E0,9.7E1,4.8E1,3.9E1,3.78E2,2.2E2,9.5E1,1.6E1,3.1E1,1.1E1,4E0,2.2E1,4E0,2.8E1,5E0,1.5E1,8.2E1,4E0,6E0,7.8E1,1.1E1,1.2E1,1E1,5.63E2,3.4E1,4E0,4E0,9E0,1.9E1,3.5E1,8E0,6.9E1,1.5E1,6.6E1,1.4E1,8E0,5E0,3.98E2,9E0,5.7E1,4E1,4.3E1,5E0,3E1,9E0,4.4E1,3.34E2,1.43E2,7.7E1,2.7E1,6.8E1,1.2E1,4E0,4E0,2.7E1,7E0,4E0,7E0,1.5E1,8E0,2E1,1.1E1,4E0,1.7E1,6.5E1,2.3E1,5.5E1,7E0,4E0,4E0,8E0,6E0,4E0,2.18E2,3.45E2,1.5E1,1.9E1,5E0,4E0,4E0,1.5E1,2.4E1,1.1E1,4E0,4E0,4.4E1,2.5E1,7E0,8E0,5.6E1,1E1,5E0,9E0,3.3E2,6.8E1,4E0,5E0,9E0,4.8E1,6E0,3.4E1,1.3E1,3E1,8E0,2.2E1,5E0,4E0,3.2E1,1.2E1,9.6E1,2.38E2,1.2E2,2.3E1,4E0,7.3E1,4E0,2.3E1,9E0,5.9E1,5E0,7E0,2.3E1,4E0,5E0,1E1,1.6E1,4E0,5E0,6E0,7E0,1E1,6.1E1,4E0,7E0,1.6E1,2.8E1,2.7E1,2.06E2,1.2E1,1.2E2,2.25E2,6E0,9E0,1.4E1,5E0,1E1,5E0,1.8E1,6E0,5E0,6E0,3.8E1,6E0,1.9E1,6E0,4E0,4E0,9E0,4.7E1,6E0,4E0,4E0,5E0,1.85E2,1.45E2,1.8E1,5E1,2E1,2.8E1,2.5E1,9E0,5E0,8E0,2.6E1,4E0,1.4E1,8E0,2.5E1,7E0,5E0,7E0,7.5E1,2.1E1,2.05E2,3.3E1,5.1E1,6.9E1,1.9E1,4E0,2.3E1,5E1,9E0,1.4E1,4E0,5E0,5E0,5.4E1,1.8E1,5E0,4E0,6E0,4.9E1,1.2E1,9E0,7E0,4E0,2.4E1,5E0,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"275","size_leaf_vector":"1"}},{"base_weights":[9.6448004E-4,8.781637E-3,-1.9826734E-2,7.2373133E-3,1.4360207E-1,-5.2058414E-2,-1.12966765E-2,2.1731234E-3,3.154496E-2,1.7950258E-1,-3.5876362E-4,-1.364673E-1,-3.9876875E-2,-1.9174964E-3,-4.669464E-2,-1.5952536E-3,3.520496E-2,5.66059E-3,2.4462694E-2,1.2720711E-2,5.7664085E-3,-8.064581E-2,-1.1725642E-2,-4.7077283E-2,3.7431999E-3,-1.1884151E-2,2.1352557E-2,-3.8003303E-2,-1.1594347E-2,-2.7576104E-2,2.44025E-3,2.4867576E-2,7.961807E-2,2.7310789E-2,-5.812273E-3,-6.775079E-3,-7.084678E-4,-2.4039192E-2,-9.549067E-2,9.152154E-3,-3.2979887E-2,7.0159644E-2,8.133977E-3,-3.0205946E-2,-7.5274035E-3,-1.2215166E-2,-1.3457158E-1,1.1837767E-2,-9.683267E-3,3.0998234E-2,-5.2442765E-3,9.117189E-2,2.9606855E-4,3.0358894E-2,-6.0462408E-2,-8.741643E-3,-7.5912826E-2,-1.3279338E-1,-1.9393465E-2,7.644136E-2,-4.376088E-3,3.825073E-2,-4.0376168E-2,1.14960514E-1,2.3668181E-2,3.9556656E-2,-6.715117E-4,-8.4048934E-2,-1.16563635E-2,-3.0341327E-2,3.2781642E-2,5.120119E-4,-1.9857018E-1,1.3431109E-2,-7.0222123E-3,-4.1482914E-2,3.391272E-3,2.2540988E-2,7.100291E-2,6.4314617E-4,1.05473325E-1,3.2976642E-2,-3.8009756E-3,-7.2123454E-4,-4.434146E-3,1.2882552E-2,-6.29111E-2,-1.2660855E-1,-1.7625814E-2,-1.6732106E-1,-7.5811235E-4,-3.1675035E-3,1.2093945E-2,2.6721766E-2,7.660916E-3,2.2347385E-2,-1.927641E-2,2.8169337E-3,8.3264174E-5,-2.8290508E-2,-8.028118E-2,7.833923E-3,2.5353322E-3,2.6082024E-3,-5.7568366E-5,9.937381E-3,7.706797E-2,-1.1746558E-2,4.6439406E-2,-1.1411431E-1,-4.2892814E-2,-2.771148E-2,5.4296076E-2,-2.3683945E-3,-6.241454E-2,8.489739E-3,1.2181271E-1,-1.5502116E-2,-7.723662E-2,4.7564384E-2,6.9841105E-3,-1.19802415E-1,-2.5739053E-2,-1.0017769E-2,3.619293E-2,5.7864424E-2,1.2232593E-2,-1.9863885E-4,4.4409735E-3,2.1896644E-3,5.866167E-3,2.5788533E-2,8.274633E-2,2.2400659E-2,-3.670216E-3,-5.5804006E-3,-7.848632E-3,-3.257494E-3,-7.829101E-3,-2.9110927E-3,1.731905E-3,-1.9384575E-1,-2.4472594E-3,2.4956074E-3,-1.4078779E-3,2.9187917E-3,-8.1119494E-4,4.5314398E-2,-2.0848192E-2,-9.784765E-3,-8.13769E-2,-3.3614185E-2,3.4327924E-3,-1.8721271E-2,-1.07411996E-1,-6.490038E-4,2.6883136E-3,1.6099453E-3,5.0764284E-3,-1.7735088E-2,2.6527687E-3,3.6174557E-3,5.712067E-4,-7.19526E-3,-1.2761628E-3,1.2904765E-3,-3.8126805E-3,-5.133964E-3,-1.5377478E-2,-1.3906554E-4,4.9467143E-3,-1.4938331E-3,3.691452E-3,1.3902122E-3,-3.933018E-3,2.8810385E-3,-1.3033419E-3,2.102049E-3,8.392584E-3,-7.555536E-3,6.088742E-4,9.521653E-4,4.667591E-3,-7.718569E-4,9.069495E-4,-7.5956457E-3,3.766205E-4,-2.6669496E-4,-3.95804E-3,-1.0939474E-3,1.8053651E-3,2.7122856E-3,-1.1117043E-3,4.252017E-3,1.2896265E-3,-2.3033668E-3,1.151489E-3,8.15056E-4,2.8664668E-3,5.1589687E-3,-5.537356E-4,2.3602196E-3,-6.37743E-4,1.7435853E-3,-2.8766422E-3,-1.1934343E-2,-3.9492426E-3,3.5450975E-3,-7.8016525E-4,-2.1594367E-3,1.9175326E-4,-3.9265854E-3,-1.828983E-4,-6.0333754E-3,-4.987659E-4,-4.884323E-3,-9.057739E-4,-2.679845E-3,1.6938164E-3,-1.967293E-3,-6.822406E-3,-1.1162269E-3,2.00582E-3,-3.888266E-3,-4.595333E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,-1,-1,35,-1,37,-1,39,41,43,-1,45,47,49,51,53,-1,-1,-1,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,79,-1,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,119,-1,121,123,125,127,-1,129,131,-1,-1,-1,133,135,137,139,141,-1,-1,143,145,-1,147,149,-1,-1,151,153,-1,-1,-1,-1,155,157,159,161,163,165,167,169,171,173,175,177,-1,179,181,183,185,187,189,191,193,195,-1,-1,-1,-1,197,199,201,-1,-1,203,-1,-1,-1,-1,205,-1,-1,-1,-1,-1,207,209,211,213,215,-1,217,219,-1,-1,-1,-1,221,-1,-1,-1,-1,-1,-1,-1,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1575867E-1,3.8601628E-1,1.9159359E-1,2.2630966E-1,1.1496514E-1,1.4467967E-1,1.8384063E-1,1.8963373E-1,1.8074667E-1,5.2432597E-2,0E0,7.974699E-2,1.1015047E-1,1.0206865E-1,1.7804754E-1,1.4349636E-1,6.96421E-2,0E0,1.2066141E-1,0E0,0E0,4.5554847E-2,0E0,1.3377744E-1,0E0,1.3698494E-1,8.459048E-2,9.382303E-2,0E0,3.0076957E-1,1.3512282E-1,1.0512874E-1,2.4151355E-2,7.888763E-2,0E0,0E0,0E0,6.59937E-2,1.0813132E-1,1.4143175E-1,8.267802E-2,5.559911E-2,2.9079225E-2,1.0488938E-1,0E0,1.3312498E-1,2.152031E-1,1.6406894E-1,2.1550733E-1,4.001405E-2,0E0,2.6637599E-2,0E0,8.023918E-2,1.1735663E-2,7.812094E-2,5.391158E-2,1.0305205E-1,2.0162411E-2,9.5285445E-2,5.2195266E-2,9.852709E-3,6.524779E-2,2.9646546E-2,1.0682243E-2,2.4788484E-2,4.3851215E-2,2.8514415E-2,8.621456E-2,1.0392526E-1,1.0061433E-1,0E0,1.9284397E-1,1.4534566E-1,0E0,1.8310714E-1,1.6222323E-1,3.679735E-2,2.8499603E-2,0E0,1.0298312E-2,9.553185E-2,0E0,0E0,0E0,4.108958E-2,5.0147407E-2,8.257493E-3,2.3266153E-2,5.3286195E-2,0E0,0E0,1.5125898E-2,2.4092494E-2,0E0,4.7572173E-2,4.8716363E-2,0E0,0E0,5.859205E-2,5.1434547E-2,0E0,0E0,0E0,0E0,1.4777902E-2,7.7268854E-3,2.7344633E-2,1.3712015E-2,3.6030218E-2,3.1819154E-2,5.8577366E-2,4.1102447E-2,1.3444155E-1,8.160862E-2,6.610337E-2,2.97364E-2,0E0,6.137761E-2,1.3546547E-1,1.400553E-1,1.03692055E-1,1.3812992E-1,1.4438199E-1,1.1432305E-1,1.7881155E-2,5.2149296E-2,0E0,0E0,0E0,0E0,7.088663E-2,6.654863E-2,3.9028376E-2,0E0,0E0,2.3262173E-2,0E0,0E0,0E0,0E0,6.378472E-2,0E0,0E0,0E0,0E0,0E0,4.9526162E-2,9.553349E-3,3.0892387E-2,2.971457E-2,1.0152832E-1,0E0,2.2025576E-2,3.8223952E-2,0E0,0E0,0E0,0E0,1.7685723E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.1281766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,77,77,78,78,80,80,81,81,85,85,86,86,87,87,88,88,89,89,92,92,93,93,95,95,96,96,99,99,100,100,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,126,126,131,131,132,132,133,133,136,136,141,141,147,147,148,148,149,149,150,150,151,151,153,153,154,154,159,159,168,168],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,-1,-1,36,-1,38,-1,40,42,44,-1,46,48,50,52,54,-1,-1,-1,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,80,-1,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,120,-1,122,124,126,128,-1,130,132,-1,-1,-1,134,136,138,140,142,-1,-1,144,146,-1,148,150,-1,-1,152,154,-1,-1,-1,-1,156,158,160,162,164,166,168,170,172,174,176,178,-1,180,182,184,186,188,190,192,194,196,-1,-1,-1,-1,198,200,202,-1,-1,204,-1,-1,-1,-1,206,-1,-1,-1,-1,-1,208,210,212,214,216,-1,218,220,-1,-1,-1,-1,222,-1,-1,-1,-1,-1,-1,-1,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,4.5505118E2,1.263E3,1.3508893E3,3.430961E7,1.1795571E6,1.3649979E10,2.9251662E6,1.2059859E1,1.6907827E9,-3.5876362E-4,1E0,2.0185E4,6.624E3,3.5551153E5,3.3820656E7,2.0827537E0,5.66059E-3,3.7826266E4,1.2720711E-2,5.7664085E-3,1.74E4,-1.1725642E-2,6.6911E4,3.7431999E-3,1.4490348E9,6.504065E-2,1.6597747E8,-1.1594347E-2,4.5184364E0,2.0992E4,3.3743455E0,5.70875E5,4.0950096E10,-5.812273E-3,-6.775079E-3,-7.084678E-4,1.2471935E7,1.7019386E2,1.1107701E-2,8.3306855E6,2.7864855E11,1.4540612E7,2.376302E8,-7.5274035E-3,4.0058823E2,3.92382E5,1.6015803E7,2.437E3,1.7E1,-5.2442765E-3,6E2,2.9606855E-4,4.5481584E7,7.41688E-2,3.532346E-1,7.59E2,1.4E1,6.0292965E-1,1.3676985E4,1.2716E4,6.567044E2,1.1355795E7,4.5168175E5,3.8874447E1,2.6295085E3,9.6910744E7,2.262586E6,1.2727361E6,2.4E1,4.459525E5,5.120119E-4,1.2E1,4.780434E-3,-7.0222123E-3,2.3170732E4,2.1592189E5,9.489796E-1,7.81893E-2,6.4314617E-4,3.6355584E7,9.256843E3,-3.8009756E-3,-7.2123454E-4,-4.434146E-3,2.4752695E6,6.702495E-1,1.25623764E2,3.52E2,1.5153E4,-7.5811235E-4,-3.1675035E-3,1.8203359E9,8.977316E0,7.660916E-3,1.1231086E1,5.557511E8,2.8169337E-3,8.3264174E-5,2.462277E6,1.8989474E0,7.833923E-3,2.5353322E-3,2.6082024E-3,-5.7568366E-5,1.0134536E7,1.0219829E1,1.9745072E7,8.352216E-2,8.5023944E2,1.4431512E-1,3.402546E6,9.0206254E-1,4.6E0,1.3314917E0,1.0975045E7,1.6E1,-1.5502116E-2,1.2737473E1,5E0,7.92582E5,3.4175084E0,1.5818182E1,2.67E2,2.02E2,4.624343E6,1E1,-1.9863885E-4,4.4409735E-3,2.1896644E-3,5.866167E-3,9.9625344E5,5.2686436E7,8.5315496E-1,-3.670216E-3,-5.5804006E-3,1E0,-3.257494E-3,-7.829101E-3,-2.9110927E-3,1.731905E-3,2.2007043E0,-2.4472594E-3,2.4956074E-3,-1.4078779E-3,2.9187917E-3,-8.1119494E-4,6.587855E1,1.0238709E1,5.6553E4,1.3225807E0,1.892E3,3.4327924E-3,1.4633E4,1.2E1,-6.490038E-4,2.6883136E-3,1.6099453E-3,5.0764284E-3,9.06727E5,2.6527687E-3,3.6174557E-3,5.712067E-4,-7.19526E-3,-1.2761628E-3,1.2904765E-3,-3.8126805E-3,-5.133964E-3,2.5943396E0,-1.3906554E-4,4.9467143E-3,-1.4938331E-3,3.691452E-3,1.3902122E-3,-3.933018E-3,2.8810385E-3,-1.3033419E-3,2.102049E-3,8.392584E-3,-7.555536E-3,6.088742E-4,9.521653E-4,4.667591E-3,-7.718569E-4,9.069495E-4,-7.5956457E-3,3.766205E-4,-2.6669496E-4,-3.95804E-3,-1.0939474E-3,1.8053651E-3,2.7122856E-3,-1.1117043E-3,4.252017E-3,1.2896265E-3,-2.3033668E-3,1.151489E-3,8.15056E-4,2.8664668E-3,5.1589687E-3,-5.537356E-4,2.3602196E-3,-6.37743E-4,1.7435853E-3,-2.8766422E-3,-1.1934343E-2,-3.9492426E-3,3.5450975E-3,-7.8016525E-4,-2.1594367E-3,1.9175326E-4,-3.9265854E-3,-1.828983E-4,-6.0333754E-3,-4.987659E-4,-4.884323E-3,-9.057739E-4,-2.679845E-3,1.6938164E-3,-1.967293E-3,-6.822406E-3,-1.1162269E-3,2.00582E-3,-3.888266E-3,-4.595333E-4],"split_indices":[27,56,2,52,47,45,5,28,56,12,0,8,10,2,33,7,53,0,52,0,0,9,0,29,0,5,57,45,0,53,29,54,1,5,0,0,0,45,52,38,45,31,45,7,0,4,5,51,2,8,0,2,0,47,38,42,2,8,27,33,9,52,50,28,56,4,45,1,28,10,28,0,8,41,0,33,28,53,58,0,7,4,0,0,0,50,39,4,0,10,0,0,5,54,0,54,7,0,0,28,54,0,0,0,0,45,54,1,38,52,58,1,27,54,54,45,3,0,54,8,12,53,56,8,8,45,3,0,0,0,0,28,43,27,0,0,75,0,0,0,0,54,0,0,0,0,0,58,53,1,57,2,0,10,18,0,0,0,0,9,0,0,0,0,0,0,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.556E3,1.858E3,6.98E2,1.838E3,2E1,1.45E2,5.53E2,1.522E3,3.16E2,1.6E1,4E0,1.7E1,1.28E2,4.38E2,1.15E2,1.367E3,1.55E2,2.4E1,2.92E2,6E0,1E1,1.2E1,5E0,1.21E2,7E0,3.07E2,1.31E2,1.11E2,4E0,1.83E2,1.184E3,1.27E2,2.8E1,2.87E2,5E0,6E0,6E0,8.3E1,3.8E1,1.54E2,1.53E2,2.7E1,1.04E2,1.05E2,6E0,1.61E2,2.2E1,6.67E2,5.17E2,1.22E2,5E0,2.4E1,4E0,2.78E2,9E0,6.5E1,1.8E1,2.5E1,1.3E1,2.5E1,1.29E2,1.4E1,1.39E2,1.3E1,1.4E1,2.2E1,8.2E1,2.6E1,7.9E1,1.15E2,4.6E1,7E0,1.5E1,6.61E2,6E0,1.5E2,3.67E2,1.02E2,2E1,4E0,2E1,2.72E2,6E0,4E0,5E0,4.7E1,1.8E1,9E0,9E0,1.9E1,6E0,5E0,8E0,1.6E1,9E0,4.6E1,8.3E1,9E0,5E0,1.08E2,3.1E1,7E0,6E0,6E0,8E0,1.3E1,9E0,6.7E1,1.5E1,1.4E1,1.2E1,6.4E1,1.5E1,6.2E1,5.3E1,3.7E1,9E0,7E0,8E0,1.04E2,5.57E2,2.4E1,1.26E2,2.61E2,1.06E2,2.2E1,8E1,4E0,1.6E1,4E0,1.6E1,2.39E2,3.3E1,4.3E1,4E0,9E0,9E0,4E0,5E0,5E0,4E0,1.5E1,4E0,4E0,4E0,9E0,7E0,3E1,1.6E1,7.3E1,1E1,1.03E2,5E0,1E1,2.1E1,9E0,4E0,4E0,5E0,6.2E1,5E0,8E0,7E0,1E1,4E0,4E0,8E0,8E0,5.6E1,7E0,8E0,4.6E1,1.6E1,8E0,4.5E1,1.5E1,2.2E1,4E0,5E0,4E0,4E0,6.5E1,3.9E1,1.85E2,3.72E2,1.9E1,5E0,9.2E1,3.4E1,2.08E2,5.3E1,8.1E1,2.5E1,1.1E1,1.1E1,1.2E1,6.8E1,1.85E2,5.4E1,2.7E1,6E0,2.5E1,1.8E1,5E0,4E0,1E1,5E0,2.1E1,9E0,8E0,8E0,5E0,6.8E1,6E0,4E0,1.9E1,8.4E1,6E0,4E0,7E0,1.4E1,5.8E1,4E0,4E0,5.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[-1.3380749E-3,-1.6486444E-2,8.051542E-3,-1.8721327E-2,1.0598233E-1,3.2578448E-3,4.2544644E-2,-2.8848521E-2,-1.2280458E-3,1.0724082E-2,3.377575E-2,1.4385504E-2,-8.775969E-3,3.643041E-2,1.4206308E-1,-1.4549402E-2,-4.6851974E-2,3.4396935E-2,-2.5693674E-2,3.0072648E-3,-7.6123537E-4,-2.2870218E-2,2.148727E-2,-1.1485417E-2,6.900158E-2,4.117363E-2,-3.577742E-2,3.8355244E-3,1.0246786E-2,-2.7090156E-2,2.4424339E-2,-3.1603526E-2,-8.84045E-2,1.0027958E-1,1.592589E-2,-1.99185E-2,-1.0677342E-1,-1.3021932E-2,-6.3922117E-3,2.311073E-2,-9.0862615E-3,-3.9196387E-2,-6.960869E-3,6.8693254E-3,3.674642E-2,2.4993684E-2,7.51097E-2,3.2257466E-3,-5.8295173E-3,-1.8559523E-2,-9.528357E-2,3.681012E-2,-4.7005363E-2,1.8532705E-2,-4.448625E-2,-1.4245268E-2,-6.653523E-2,4.673964E-2,1.4477131E-1,8.768109E-3,5.569281E-3,-2.3668375E-2,5.291329E-3,-3.792051E-2,-1.1148791E-2,5.974734E-3,-5.510207E-2,5.6275696E-2,1.5007682E-2,-3.1498235E-2,-7.209534E-3,-2.720269E-3,-4.018057E-2,-2.0151965E-3,3.1761245E-3,7.601275E-2,9.364354E-3,9.9074105E-3,5.9146293E-2,-2.2328269E-2,5.365604E-3,-1.7253906E-1,1.3072348E-2,4.485206E-3,7.177759E-2,-5.589133E-3,-5.7537276E-3,-2.0208787E-3,3.5843E-2,-8.244918E-2,-1.9878665E-2,-4.463892E-2,-1.7394926E-1,1.9618709E-2,5.120681E-3,9.183332E-3,3.892531E-3,5.3770967E-2,-3.2299813E-3,5.5895005E-3,-2.7074605E-2,5.886931E-4,-4.4959546E-3,3.4112185E-2,-1.0225742E-2,-7.316108E-2,1.2494515E-3,-6.0146227E-2,6.5128565E-2,4.4045106E-2,1.2220929E-3,-5.3921074E-2,-1.126771E-2,-2.0034006E-2,1.070136E-2,-8.220363E-2,-1.4601697E-2,8.824038E-2,8.230213E-4,-5.5098576E-3,1.5980206E-2,-3.5831612E-4,6.814797E-2,-1.7189806E-2,-1.063835E-1,-1.4956931E-2,-1.1049794E-1,4.3876138E-2,-2.2479475E-3,-4.751829E-3,2.1927144E-2,6.444782E-3,4.541808E-3,-1.696408E-3,1.1789178E-3,2.0304998E-2,5.9357807E-3,-6.462659E-2,-1.8220401E-1,-4.6072587E-2,1.3910843E-2,2.9241391E-3,-5.7922963E-2,-1.1744754E-2,-2.6790057E-3,1.8584791E-3,-6.208912E-4,-1.0237772E-3,8.118568E-2,-1.1645866E-2,4.557166E-2,-2.3575164E-2,-6.5770173E-3,-2.9445186E-3,5.2074563E-2,4.0117025E-2,-2.585436E-2,-1.8694159E-2,-9.401314E-2,-6.39671E-3,9.871585E-4,3.7298065E-2,8.4678344E-2,-7.9692245E-2,5.1128488E-2,-6.4498987E-3,1.16609104E-1,-2.6752945E-2,-9.922897E-2,4.9602766E-2,-3.355481E-2,4.54177E-3,-2.3568941E-2,-3.7622725E-4,4.2142842E-2,-1.054276E-1,-5.0194014E-2,-3.1904217E-2,1.4427501E-2,5.875489E-2,6.8045254E-3,4.4605234E-3,2.773154E-3,7.849543E-2,3.6639195E-2,2.41453E-3,-1.1315028E-3,-1.0235437E-2,-9.6131896E-4,-6.793415E-3,-2.1270085E-3,4.5572026E-3,-6.109824E-4,4.830662E-3,-4.4221315E-4,2.5999905E-3,-2.0203467E-3,2.4038397E-3,-8.8020414E-4,1.9341658E-3,-3.7841925E-3,-5.1577096E-3,-1.1229719E-2,-6.933404E-5,-4.532377E-3,1.4729829E-3,-4.477147E-3,-3.4910047E-3,3.2944274E-3,5.0728107E-3,6.0305774E-4,-2.69758E-3,4.0575076E-4,-7.289298E-5,4.301726E-3,-1.6794042E-3,4.898493E-4,4.0868097E-3,-3.2679393E-4,4.4119806E-5,3.9575547E-3,-5.284625E-4,-4.597105E-3,-2.7044727E-3,1.0231201E-3,-5.570642E-3,-9.640385E-4,6.653756E-4,4.455527E-3,4.589442E-3,-9.869634E-4,2.6644347E-3,-9.831955E-3,3.180701E-3,-8.9029147E-4,-2.0572115E-3,1.888929E-4,3.5606972E-3,8.452097E-3,-2.4113064E-3,3.8735298E-3,-1.6882733E-3,-6.4242473E-3,1.0869894E-3,3.920731E-3,-2.8647534E-3,-6.422857E-4,-9.418204E-3,-8.534972E-4,-3.5284946E-4,1.7937425E-3,6.51773E-3,1.3945732E-3,-6.7706592E-3,-2.3861642E-3,-5.570458E-4,-3.8220081E-3,-1.6316488E-4,-3.0125787E-3,1.9919581E-3,-1.1978964E-3,3.7975954E-3,1.285232E-3,-5.8140804E-4,1.5253102E-3,3.2307983E-3,5.6787934E-3,3.2352305E-3,5.485273E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,-1,49,51,53,55,57,59,61,63,65,-1,67,-1,69,71,-1,73,75,77,-1,-1,79,81,83,85,87,89,-1,91,93,95,97,-1,99,-1,101,-1,103,105,107,109,111,-1,113,115,-1,-1,117,119,-1,121,123,-1,125,127,129,131,-1,133,-1,135,137,139,141,143,145,-1,-1,-1,147,149,-1,151,-1,-1,153,155,157,-1,159,161,163,165,167,169,171,173,175,177,179,-1,-1,181,-1,183,185,187,-1,189,191,-1,-1,193,195,-1,-1,-1,197,-1,199,201,203,205,-1,207,-1,-1,-1,-1,-1,209,211,213,215,-1,-1,217,219,221,223,225,-1,-1,227,229,231,233,235,237,239,241,243,245,-1,247,249,251,253,255,257,259,261,-1,263,-1,265,267,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.702344E-1,2.7459285E-1,2.6557797E-1,1.7340216E-1,1.339554E-1,1.892046E-1,1.1408648E-1,1.5892708E-1,3.1465533E-1,0E0,1.6449165E-2,1.9469419E-1,1.4388108E-1,6.517765E-2,2.9518887E-2,1.7026141E-1,1.7081565E-1,1.7729037E-1,9.807867E-2,0E0,0E0,1.2042329E-1,2.0787698E-1,8.2101576E-2,4.5748785E-2,9.421241E-2,1.0492626E-1,0E0,0E0,1.512274E-1,7.6930836E-2,1.3129447E-1,2.950396E-1,6.9460124E-2,7.834487E-2,9.733683E-2,1.0369462E-1,8.746189E-2,0E0,1.638914E-1,0E0,6.988345E-2,7.962396E-2,0E0,3.7704177E-2,9.497544E-2,9.838101E-2,0E0,0E0,1.1393994E-1,2.4501705E-1,8.2191445E-2,3.4134604E-2,4.432802E-2,1.4876339E-1,0E0,1.4981717E-1,2.2177964E-2,3.0036658E-2,5.902805E-2,0E0,9.3513414E-2,0E0,2.6934372E-2,0E0,3.499267E-2,5.101613E-2,1.2759745E-1,1.9769335E-1,3.885787E-2,0E0,1.1710082E-1,6.759344E-2,0E0,0E0,1.893194E-2,7.537048E-2,0E0,3.0934036E-2,9.707973E-2,0E0,1.00658715E-1,2.522454E-2,6.9826655E-2,4.3133005E-2,0E0,8.248737E-3,0E0,4.0121753E-2,1.002318E-1,8.8263795E-2,8.150204E-2,6.827694E-2,7.220691E-3,0E0,0E0,0E0,4.8487328E-2,3.6262397E-2,0E0,6.714557E-2,0E0,0E0,4.9863804E-2,3.9589804E-2,3.0043423E-2,0E0,5.1338468E-2,5.7778597E-2,1.4343476E-1,2.981955E-1,4.7813497E-2,6.542585E-2,8.841098E-2,9.893185E-2,1.2549534E-2,2.1053705E-2,2.4510577E-2,0E0,0E0,4.027026E-2,0E0,1.179561E-2,7.864878E-2,1.0760395E-1,0E0,1.6696587E-2,2.5013179E-2,0E0,0E0,7.9076715E-2,2.1338625E-2,0E0,0E0,0E0,3.1423498E-2,0E0,6.606214E-2,7.040024E-3,1.1156799E-1,7.3549494E-2,0E0,8.005047E-2,0E0,0E0,0E0,0E0,0E0,2.2671685E-2,6.3208826E-2,2.489953E-2,6.296815E-2,0E0,0E0,4.2338304E-2,1.8332498E-2,3.743086E-2,1.3598971E-2,2.396202E-2,0E0,0E0,5.852341E-2,5.0495982E-2,1.5065242E-1,1.315614E-1,1.1240309E-1,3.7985027E-2,6.428793E-2,2.445808E-2,8.742712E-3,1.6490966E-2,0E0,2.2574794E-1,5.1424276E-2,8.935778E-2,1.6078189E-2,1.1468722E-2,2.09407E-2,1.6585564E-2,7.1609877E-3,0E0,2.9725129E-2,0E0,8.703694E-3,8.666154E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,59,59,61,61,63,63,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,76,76,78,78,79,79,81,81,82,82,83,83,84,84,86,86,88,88,89,89,90,90,91,91,92,92,93,93,97,97,98,98,100,100,103,103,104,104,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,120,120,122,122,123,123,124,124,126,126,127,127,130,130,131,131,135,135,137,137,138,138,139,139,140,140,142,142,148,148,149,149,150,150,151,151,154,154,155,155,156,156,157,157,158,158,161,161,162,162,163,163,164,164,165,165,166,166,167,167,168,168,169,169,170,170,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,181,181,183,183,184,184],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,-1,50,52,54,56,58,60,62,64,66,-1,68,-1,70,72,-1,74,76,78,-1,-1,80,82,84,86,88,90,-1,92,94,96,98,-1,100,-1,102,-1,104,106,108,110,112,-1,114,116,-1,-1,118,120,-1,122,124,-1,126,128,130,132,-1,134,-1,136,138,140,142,144,146,-1,-1,-1,148,150,-1,152,-1,-1,154,156,158,-1,160,162,164,166,168,170,172,174,176,178,180,-1,-1,182,-1,184,186,188,-1,190,192,-1,-1,194,196,-1,-1,-1,198,-1,200,202,204,206,-1,208,-1,-1,-1,-1,-1,210,212,214,216,-1,-1,218,220,222,224,226,-1,-1,228,230,232,234,236,238,240,242,244,246,-1,248,250,252,254,256,258,260,262,-1,264,-1,266,268,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,4.5505118E2,1.3137E4,3.83E3,1.1781829E9,5.107429E1,7.273333E1,7.667683E6,2.8091298E7,1.0724082E-2,2.21875E0,5.08E2,1.5435694E9,7.065733E7,1.96E4,1.263E3,3.03157E5,1.7605528E7,2.6572757E10,3.0072648E-3,-7.6123537E-4,1E0,3.7852024E1,2.47E2,8.538E3,4.5575E4,1.003252E5,3.8355244E-3,1.0246786E-2,3.363849E2,1.3264E4,1.3617021E0,7E2,4.4844016E2,1.6956E5,2.4137697E0,6.211896E0,1.3325451E1,-6.3922117E-3,4.0208E4,-9.0862615E-3,7.612907E-1,3.4692544E7,6.8693254E-3,5.3473564E1,2.0288463E0,2.3761498E9,3.2257466E-3,-5.8295173E-3,8.684849E0,1.9676556E3,3E0,3.2126E4,6.6126086E4,1.4075E4,-1.4245268E-2,6.5260305E0,3.4016838E5,6.823E3,9.58059E2,5.569281E-3,3.53562E5,5.291329E-3,7.644773E-1,-1.1148791E-2,3.2E1,5.015707E0,1.5944675E-4,1E0,9.42812E3,-7.209534E-3,1.5166431E5,1.958583E3,-2.0151965E-3,3.1761245E-3,1.7655972E0,5.839E3,9.9074105E-3,1.1041459E-1,5.8372E4,5.365604E-3,2.6659616E2,2.2333652E1,3.4955364E7,7.232191E0,-5.589133E-3,4.256737E-1,-2.0208787E-3,1.1835411E3,1.775894E10,1.4437085E-1,2.2837209E2,2.142857E0,2.8E2,5.120681E-3,9.183332E-3,3.892531E-3,1.5451318E8,1.4771426E0,5.5895005E-3,1.8147379E3,5.886931E-4,-4.4959546E-3,1.91E2,8.751669E4,2.1337201E4,1.2494515E-3,1.048E3,1.3E1,6.363636E-1,1.3697374E7,1.7956694E-4,4.777882E5,7.1047956E-1,1.2287474E3,2.821E3,7.678E3,3.3734077E-1,8.230213E-4,-5.5098576E-3,1.208686E6,-3.5831612E-4,2.409695E0,5E-1,7.38E2,-1.4956931E-2,7.696629E0,1E0,-2.2479475E-3,-4.751829E-3,2.3067484E0,9.00042E-6,4.541808E-3,-1.696408E-3,1.1789178E-3,4.282564E2,5.9357807E-3,1.01644066E2,2.0802219E8,1.0989011E-2,1.822948E6,2.9241391E-3,1.0772926E2,-1.1744754E-2,-2.6790057E-3,1.8584791E-3,-6.208912E-4,-1.0237772E-3,2.5882E4,1.5E1,4.70092E1,5.094239E3,-6.5770173E-3,-2.9445186E-3,8.494517E5,2.4481E4,6.025862E0,7.860526E1,6.63E2,-6.39671E-3,9.871585E-4,6E0,3.33E2,2.1934064E7,1E0,3E0,2.106062E1,6.6404694E2,7.427971E2,3.0062932E4,5.857681E6,4.54177E-3,1.3797468E0,9.490825E5,1.1031587E-2,6.5928856E5,6.4562735E6,2.3616017E11,2.028994E8,6E0,6.8045254E-3,2.525229E8,2.773154E-3,2.7487153E5,1E0,2.41453E-3,-1.1315028E-3,-1.0235437E-2,-9.6131896E-4,-6.793415E-3,-2.1270085E-3,4.5572026E-3,-6.109824E-4,4.830662E-3,-4.4221315E-4,2.5999905E-3,-2.0203467E-3,2.4038397E-3,-8.8020414E-4,1.9341658E-3,-3.7841925E-3,-5.1577096E-3,-1.1229719E-2,-6.933404E-5,-4.532377E-3,1.4729829E-3,-4.477147E-3,-3.4910047E-3,3.2944274E-3,5.0728107E-3,6.0305774E-4,-2.69758E-3,4.0575076E-4,-7.289298E-5,4.301726E-3,-1.6794042E-3,4.898493E-4,4.0868097E-3,-3.2679393E-4,4.4119806E-5,3.9575547E-3,-5.284625E-4,-4.597105E-3,-2.7044727E-3,1.0231201E-3,-5.570642E-3,-9.640385E-4,6.653756E-4,4.455527E-3,4.589442E-3,-9.869634E-4,2.6644347E-3,-9.831955E-3,3.180701E-3,-8.9029147E-4,-2.0572115E-3,1.888929E-4,3.5606972E-3,8.452097E-3,-2.4113064E-3,3.8735298E-3,-1.6882733E-3,-6.4242473E-3,1.0869894E-3,3.920731E-3,-2.8647534E-3,-6.422857E-4,-9.418204E-3,-8.534972E-4,-3.5284946E-4,1.7937425E-3,6.51773E-3,1.3945732E-3,-6.7706592E-3,-2.3861642E-3,-5.570458E-4,-3.8220081E-3,-1.6316488E-4,-3.0125787E-3,1.9919581E-3,-1.1978964E-3,3.7975954E-3,1.285232E-3,-5.8140804E-4,1.5253102E-3,3.2307983E-3,5.6787934E-3,3.2352305E-3,5.485273E-4],"split_indices":[102,56,2,2,7,48,54,45,45,0,53,2,7,1,2,2,9,12,12,0,0,68,53,0,2,2,28,0,0,4,10,53,2,52,2,35,54,58,0,9,0,38,45,0,56,53,12,0,0,56,33,8,29,28,9,0,57,32,2,4,0,1,0,34,0,10,54,42,16,47,0,28,4,0,0,34,29,0,39,29,0,52,58,7,47,0,42,0,4,5,57,4,54,3,0,0,0,12,38,0,58,0,0,2,28,51,0,2,3,53,45,38,48,56,55,10,2,34,0,0,28,0,41,53,0,0,56,16,0,0,58,39,0,0,0,4,0,33,7,57,47,0,58,0,0,0,0,0,10,3,58,4,0,0,45,12,54,52,10,0,0,8,8,5,74,8,56,33,33,28,12,0,53,33,41,32,32,31,7,8,0,32,0,33,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.601E3,9.95E2,1.606E3,9.78E2,1.7E1,1.411E3,1.95E2,6.19E2,3.59E2,6E0,1.1E1,7.33E2,6.78E2,1.85E2,1E1,3.46E2,2.73E2,1.46E2,2.13E2,7E0,4E0,1.17E2,6.16E2,6.56E2,2.2E1,1.74E2,1.1E1,6E0,4E0,2.62E2,8.4E1,2.01E2,7.2E1,3.1E1,1.15E2,2E2,1.3E1,1.08E2,9E0,6.12E2,4E0,9.1E1,5.65E2,6E0,1.6E1,1.19E2,5.5E1,5E0,6E0,2.34E2,2.8E1,7.2E1,1.2E1,4.1E1,1.6E2,6E0,6.6E1,1.5E1,1.6E1,1.08E2,7E0,1.95E2,5E0,9E0,4E0,7.5E1,3.3E1,1.19E2,4.93E2,8.6E1,5E0,5.02E2,6.3E1,4E0,1.2E1,2.7E1,9.2E1,5E0,5E1,2.28E2,6E0,1.6E1,1.2E1,3.8E1,3.4E1,4E0,8E0,9E0,3.2E1,6.2E1,9.8E1,5.6E1,1E1,1.1E1,4E0,9E0,7E0,2.2E1,8.6E1,4E0,1.91E2,5E0,4E0,2.7E1,4.8E1,2.7E1,6E0,8E0,1.11E2,1.58E2,3.35E2,4E1,4.6E1,2.19E2,2.83E2,2.3E1,4E1,2.2E1,5E0,4E0,8.8E1,6E0,4.4E1,2.16E2,1.2E1,4E0,1.2E1,8E0,4E0,5E0,3.3E1,8E0,2.6E1,4E0,4E0,2.8E1,4E0,5.4E1,8E0,5.5E1,4.3E1,6E0,5E1,6E0,4E0,7E0,4E0,6E0,1.6E1,7.4E1,1.2E1,1.86E2,5E0,4E0,2.3E1,1.1E1,3.7E1,8E0,1.9E1,4E0,4E0,4.7E1,6.4E1,8E0,1.5E2,3.15E2,2E1,2.6E1,1.4E1,1.2E1,3.4E1,6E0,2.13E2,2.1E2,7.3E1,1.2E1,1.1E1,2.5E1,1.5E1,1.5E1,7E0,6.9E1,1.9E1,3.2E1,1.2E1,1.6E1,2E2,5E0,7E0,8E0,4E0,4E0,4E0,9E0,2.4E1,4E0,4E0,1.6E1,1.2E1,5E0,4.9E1,4E0,4E0,2.8E1,2.7E1,3.8E1,5E0,4.6E1,4E0,1.2E1,4E0,2.3E1,5.1E1,6E0,6E0,1.43E2,4.3E1,1.5E1,8E0,6E0,5E0,3.1E1,6E0,4E0,4E0,1.5E1,4E0,3.3E1,1.4E1,6E1,4E0,4E0,4E0,1.27E2,2.3E1,7.1E1,2.44E2,1.2E1,8E0,2.2E1,4E0,5E0,9E0,7E0,5E0,1.5E1,1.9E1,7E0,2.06E2,1.78E2,3.2E1,9E0,6.4E1,7E0,5E0,5E0,6E0,1.3E1,1.2E1,9E0,6E0,9E0,6E0,4.3E1,2.6E1,2.5E1,7E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"269","size_leaf_vector":"1"}},{"base_weights":[-1.9942132E-4,1.4934615E-3,-8.030415E-2,-4.020447E-3,2.8074455E-2,-4.819627E-2,-2.1354842E-1,6.05801E-3,-1.5809517E-2,4.240404E-2,5.167778E-3,-7.1958065E-2,5.2206047E-2,-3.799704E-3,-1.4617419E-2,-1.0114221E-2,1.5615119E-2,-4.3841038E-2,-1.0302337E-2,4.9369756E-2,-2.3525767E-2,-5.9513203E-3,3.948382E-2,-8.721326E-2,1.8492942E-4,3.938164E-3,7.5724773E-4,-2.9066226E-3,-9.12202E-2,1.1411188E-2,6.413191E-2,-9.549862E-3,-3.5693638E-2,-6.5712247E-4,-2.8783686E-2,4.367448E-2,1.3818814E-1,-4.8652785E-3,3.49617E-3,5.209663E-3,-1.0383183E-2,9.555762E-2,6.8693436E-3,-5.7473563E-2,-1.3335696E-1,7.634558E-2,-7.241246E-3,-1.2216201E-2,-1.6950561E-1,1.3373214E-2,-1.1549578E-1,3.9738826E-2,8.7105736E-2,-2.8878456E-2,-7.6475656E-3,-2.0361427E-2,1.4041986E-2,-1.9842608E-2,-6.7665234E-2,9.488831E-2,3.3844415E-2,1.9627123E-1,-2.80596E-4,-1.9008223E-2,2.9567506E-3,-5.813159E-3,-5.8439313E-3,6.43413E-3,1.2131901E-3,-1.8529559E-2,4.1014994E-3,3.4413626E-4,-8.554874E-2,-8.145056E-3,-3.2564076E-3,7.47996E-5,9.496416E-2,-1.4100712E-2,3.0860107E-2,3.7109263E-3,-8.883039E-2,-1.5615933E-2,-1.0145848E-1,1.5319361E-4,3.5608757E-2,-1.1759075E-2,1.8099691E-4,3.4131266E-3,-6.7778735E-4,1.1478032E-1,2.7716048E-3,2.7366884E-2,-3.9331056E-2,-2.772267E-2,6.1296463E-2,8.623695E-3,5.049853E-2,6.0002983E-2,-2.6656926E-2,-4.600655E-2,-1.6362725E-1,1.1021277E-1,-1.1075577E-3,8.408247E-2,2.3271292E-2,3.718561E-3,1.3246231E-2,2.0189108E-3,-2.212838E-3,3.4700483E-4,-8.154949E-2,-4.2243578E-2,2.3882112E-3,-1.5513661E-3,-6.12308E-3,1.2039486E-1,7.4638525E-4,-1.6673878E-2,3.6880432E-3,1.1574367E-1,7.5387126E-3,7.0522673E-4,-9.7233895E-3,-1.2679992E-3,-9.293342E-3,2.9892183E-3,-1.1302071E-2,4.936132E-2,-1.8724868E-2,-1.6544919E-3,1.5915965E-3,2.9149756E-3,6.97809E-3,5.998401E-2,-2.1033777E-2,-7.22334E-2,-8.898757E-3,-4.238347E-2,-4.6782303E-3,8.815661E-2,6.667644E-4,2.271853E-2,-1.3414643E-2,6.202985E-3,8.104313E-2,6.1872876E-3,8.129753E-3,-7.6593846E-2,-1.9416396E-2,-7.4696094E-3,-3.155955E-2,-1.0533228E-2,-4.183408E-3,8.968361E-2,1.0149906E-2,1.1649223E-1,4.2366333E-2,6.907001E-2,1.3869525E-2,5.889386E-2,-6.032821E-3,-1.0141865E-3,-6.3203746E-3,-8.729138E-5,-3.0225988E-3,7.7475943E-3,3.6718848E-3,1.8921733E-5,-1.51287E-3,1.3124167E-3,7.854727E-3,-2.2007085E-3,1.4925224E-3,3.9029977E-4,-4.972187E-3,4.750148E-3,1.99426E-3,1.9308426E-3,-2.9790704E-3,1.3324546E-3,4.9947873E-3,-3.502214E-3,1.1668653E-3,5.7755224E-4,-4.4935476E-3,1.1901173E-3,-1.8331588E-3,-1.1467208E-3,-4.6374444E-3,-7.987305E-4,1.973427E-3,5.735562E-3,1.3948416E-3,-9.854106E-4,1.440782E-3,-1.7739551E-3,9.366412E-4,1.7936066E-3,-1.5833805E-3,2.153605E-3,5.6078504E-3,-1.4754912E-3,2.9593643E-3,-6.795884E-3,-3.3563387E-4,4.1628195E-5,-1.9497823E-3,-2.149862E-3,2.7490142E-3,5.21916E-3,9.054447E-4,6.6156583E-3,3.1282578E-3,5.1181535E-3,-7.8608404E-4,7.0279674E-3,1.2723708E-3,-1.9393159E-3,1.01628E-3,7.986532E-4,4.5963516E-3,-1.1038361E-3,1.3854321E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,47,49,51,-1,53,55,57,59,61,-1,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,93,95,97,99,101,103,105,-1,107,-1,-1,109,-1,-1,111,-1,-1,113,-1,-1,-1,115,117,119,-1,121,-1,123,125,127,-1,-1,-1,129,131,-1,133,135,137,139,141,143,145,147,149,151,153,-1,155,157,-1,-1,-1,-1,159,161,163,-1,-1,-1,165,-1,167,-1,169,171,-1,-1,-1,-1,173,-1,175,177,-1,-1,-1,-1,179,181,183,185,187,189,191,-1,193,195,197,199,-1,201,203,205,-1,207,-1,-1,209,-1,211,213,215,217,219,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4727627E-1,3.6769828E-1,2.1645191E-1,2.4699466E-1,1.4125368E-1,1.0872917E-1,8.637455E-2,1.7341943E-1,1.4743923E-1,1.2286222E-1,6.395608E-2,4.1873813E-2,7.6788086E-3,0E0,0E0,2.4350443E-1,1.4306083E-1,1.8157467E-1,1.4294288E-1,1.1536747E-1,5.222153E-2,6.311621E-2,7.432511E-2,3.350447E-2,0E0,0E0,0E0,1.3246776E-1,2.0821455E-1,1.6339113E-1,2.7921066E-2,0E0,1.1555423E-1,1.5321358E-1,9.425916E-2,1.1106837E-1,1.1803818E-1,0E0,2.6184922E-2,0E0,5.8479182E-2,3.238134E-2,5.310147E-2,3.672807E-2,1.1617169E-2,2.772934E-2,9.58297E-2,1.258638E-1,1.3373879E-1,1.8841925E-1,1.4360675E-1,3.3531792E-2,1.8148124E-2,8.514751E-2,0E0,1.3754869E-1,5.9416767E-2,1.2402047E-1,9.847103E-2,6.701952E-2,1.0002853E-1,6.414071E-2,0E0,2.4317218E-2,0E0,0E0,5.5802204E-2,0E0,0E0,3.504888E-2,0E0,0E0,2.442249E-2,0E0,0E0,0E0,3.0763641E-2,7.135551E-2,1.099852E-1,0E0,1.1162782E-1,0E0,7.8887746E-2,2.5937036E-1,1.7951143E-1,0E0,0E0,0E0,1.4771188E-2,1.1508316E-2,0E0,3.757157E-2,1.2051377E-1,7.00745E-2,2.329573E-2,8.262086E-2,5.2263893E-2,5.8588162E-2,7.371934E-2,5.9569366E-2,1.6277522E-2,4.2825013E-2,0E0,3.9680973E-2,6.793626E-2,0E0,0E0,0E0,0E0,4.1815877E-2,2.2207912E-2,1.1718497E-2,0E0,0E0,0E0,8.115783E-3,0E0,7.006883E-2,0E0,4.0383384E-2,5.315184E-2,0E0,0E0,0E0,0E0,1.9798304E-1,0E0,7.869676E-2,1.17263645E-1,0E0,0E0,0E0,0E0,1.598183E-2,2.341169E-2,8.704838E-2,5.9060603E-2,1.2232891E-1,4.1587614E-2,1.6047142E-2,0E0,4.2620055E-2,7.454225E-2,2.026457E-2,2.2380248E-2,0E0,2.2915972E-2,1.0703035E-1,7.268298E-2,0E0,4.1083176E-2,0E0,0E0,2.7701125E-2,0E0,7.1416795E-3,5.7446472E-2,8.069685E-2,4.646083E-2,1.4150668E-2,5.536851E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,66,66,69,69,72,72,76,76,77,77,78,78,80,80,82,82,83,83,84,84,88,88,89,89,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,103,103,104,104,109,109,110,110,111,111,115,115,117,117,119,119,120,120,125,125,127,127,128,128,133,133,134,134,135,135,136,136,137,137,138,138,139,139,141,141,142,142,143,143,144,144,146,146,147,147,148,148,150,150,153,153,155,155,156,156,157,157,158,158,159,159,160,160],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,48,50,52,-1,54,56,58,60,62,-1,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,94,96,98,100,102,104,106,-1,108,-1,-1,110,-1,-1,112,-1,-1,114,-1,-1,-1,116,118,120,-1,122,-1,124,126,128,-1,-1,-1,130,132,-1,134,136,138,140,142,144,146,148,150,152,154,-1,156,158,-1,-1,-1,-1,160,162,164,-1,-1,-1,166,-1,168,-1,170,172,-1,-1,-1,-1,174,-1,176,178,-1,-1,-1,-1,180,182,184,186,188,190,192,-1,194,196,198,200,-1,202,204,206,-1,208,-1,-1,210,-1,212,214,216,218,220,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2427474E8,1.5106794E3,3.6600095E10,1.1846624E-5,2.8839298E7,6.08767E3,1.4633E4,1E0,1.172E3,2.3821254E7,6.2105E4,4.2E1,1.7322648E5,-3.799704E-3,-1.4617419E-2,4.5184364E0,5.0083565E6,4.39688E-5,1.8071064E7,1.9013363E4,1.9483356E1,1.9125667E3,1.30399E5,9.1002154E-1,1.8492942E-4,3.938164E-3,7.5724773E-4,6E0,1E0,1.6762007E7,2.831692E10,-9.549862E-3,3.9556694E-1,1.5497989E5,1.0131627E1,1.4157E4,1.10032365E-1,-4.8652785E-3,2.6312E4,5.209663E-3,6.33771E5,1.8880464E-1,3.832274E2,8.7757526E2,1.2962776E3,2.127E3,3.596E3,1.01644066E2,8.9E1,3.7810526E2,2.0568174E7,7.2419E4,1.00999994E9,4.3360384E7,-7.6475656E-3,4.4854636E7,1.292689E6,2.1358025E0,9.222717E-1,1E0,2.1944155E3,2.692487E6,-2.80596E-4,7.274E3,2.9567506E-3,-5.813159E-3,1.2367184E3,6.43413E-3,1.2131901E-3,6.082581E3,4.1014994E-3,3.4413626E-4,2.9166096E-1,-8.145056E-3,-3.2564076E-3,7.47996E-5,2.5882E4,4.564527E6,6.98E2,3.7109263E-3,2.9E1,-1.5615933E-2,4.607796E-1,4.0718698E6,5.49E2,-1.1759075E-2,1.8099691E-4,3.4131266E-3,2.37E3,5.152518E0,2.7716048E-3,2.0521326E0,2.1789778E10,5.012E3,6.6692764E1,7.94E3,3.8172052E2,6.5029144E5,1.9761398E7,1.2428079E1,3.650307E9,3.7137984E7,-1.1075577E-3,2.040457E0,7.318337E-1,3.718561E-3,1.3246231E-2,2.0189108E-3,-2.212838E-3,1E0,6.9673E4,4.0728608E3,2.3882112E-3,-1.5513661E-3,-6.12308E-3,1.5470392E5,7.4638525E-4,6E2,3.6880432E-3,1.0804272E3,1.0258E4,7.0522673E-4,-9.7233895E-3,-1.2679992E-3,-9.293342E-3,3.2126E4,-1.1302071E-2,2.4898794E5,3.64299E5,-1.6544919E-3,1.5915965E-3,2.9149756E-3,6.97809E-3,4.6268812E5,6.3E0,2.0714398E-3,1.6219908E5,7.2488395E6,3.337364E-1,9.18E2,6.667644E-4,1.44E0,5.2153603E5,3.222E3,5.3E1,6.1872876E-3,1.2526851E6,9.815951E-3,1.7322648E5,-7.4696094E-3,1.2974394E6,-1.0533228E-2,-4.183408E-3,8.639937E9,1.0149906E-2,5.0083565E6,6.2880285E6,5.033057E-4,1.9214156E3,4.894913E6,1.2566651E7,-1.0141865E-3,-6.3203746E-3,-8.729138E-5,-3.0225988E-3,7.7475943E-3,3.6718848E-3,1.8921733E-5,-1.51287E-3,1.3124167E-3,7.854727E-3,-2.2007085E-3,1.4925224E-3,3.9029977E-4,-4.972187E-3,4.750148E-3,1.99426E-3,1.9308426E-3,-2.9790704E-3,1.3324546E-3,4.9947873E-3,-3.502214E-3,1.1668653E-3,5.7755224E-4,-4.4935476E-3,1.1901173E-3,-1.8331588E-3,-1.1467208E-3,-4.6374444E-3,-7.987305E-4,1.973427E-3,5.735562E-3,1.3948416E-3,-9.854106E-4,1.440782E-3,-1.7739551E-3,9.366412E-4,1.7936066E-3,-1.5833805E-3,2.153605E-3,5.6078504E-3,-1.4754912E-3,2.9593643E-3,-6.795884E-3,-3.3563387E-4,4.1628195E-5,-1.9497823E-3,-2.149862E-3,2.7490142E-3,5.21916E-3,9.054447E-4,6.6156583E-3,3.1282578E-3,5.1181535E-3,-7.8608404E-4,7.0279674E-3,1.2723708E-3,-1.9393159E-3,1.01628E-3,7.986532E-4,4.5963516E-3,-1.1038361E-3,1.3854321E-3],"split_indices":[45,52,5,39,45,4,10,102,2,9,10,8,28,0,0,53,28,42,45,52,54,4,10,27,0,0,0,3,66,51,31,0,38,28,57,9,57,0,2,0,1,38,56,52,52,1,2,33,12,52,47,1,5,5,0,9,29,56,27,101,4,1,0,10,0,0,58,0,0,52,0,0,27,0,0,0,10,9,10,0,3,0,38,51,0,0,0,0,10,56,0,53,31,2,56,10,52,28,45,57,12,1,0,53,42,0,0,0,0,75,2,4,0,0,0,33,0,2,0,33,29,0,0,0,0,29,0,28,9,0,0,0,0,32,58,41,32,45,57,0,0,58,28,2,8,0,28,57,28,0,28,0,0,5,0,28,50,39,52,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.559E3,2.507E3,5.2E1,2.077E3,4.3E2,4.3E1,9E0,1.12E3,9.57E2,2.64E2,1.66E2,3.5E1,8E0,4E0,5E0,4.16E2,7.04E2,1.56E2,8.01E2,2.39E2,2.5E1,1.26E2,4E1,2.9E1,6E0,4E0,4E0,3.83E2,3.3E1,6.49E2,5.5E1,7E0,1.49E2,5.27E2,2.74E2,2.26E2,1.3E1,6E0,1.9E1,4E0,1.22E2,1.4E1,2.6E1,1.9E1,1E1,1.9E1,3.64E2,1.7E1,1.6E1,6.4E2,9E0,2.8E1,2.7E1,1.42E2,7E0,2.25E2,3.02E2,2.24E2,5E1,3.5E1,1.91E2,9E0,4E0,1.4E1,5E0,4E0,1.18E2,9E0,5E0,2E1,6E0,6E0,1.3E1,6E0,4E0,4E0,1.5E1,3.09E2,5.5E1,8E0,9E0,4E0,1.2E1,4.02E2,2.38E2,4E0,5E0,1.6E1,1.2E1,1.3E1,1.4E1,2.2E1,1.2E2,2.07E2,1.8E1,2.64E2,3.8E1,1.7E1,2.07E2,4.2E1,8E0,3.1E1,4E0,3.2E1,1.59E2,4E0,5E0,4E0,1E1,1.1E2,8E0,1.5E1,5E0,6E0,7E0,1.1E1,4E0,3.01E2,8E0,1.1E1,4.4E1,5E0,4E0,7E0,5E0,3.98E2,4E0,1.9E2,4.8E1,6E0,6E0,5E0,8E0,1.3E1,9E0,5.7E1,6.3E1,1.26E2,8.1E1,1.1E1,7E0,1.61E2,1.03E2,1.6E1,2.2E1,7E0,1E1,2.5E1,1.82E2,4E0,3.8E1,4E0,4E0,2.7E1,4E0,1.7E1,1.5E1,2.6E1,1.33E2,1E1,1E2,4E0,4E0,5E0,1E1,5E0,6E0,1.34E2,1.67E2,4E0,7E0,1.3E1,3.1E1,3.81E2,1.7E1,3.1E1,1.59E2,2E1,2.8E1,8E0,5E0,4E0,5E0,1E1,4.7E1,2.9E1,3.4E1,9.2E1,3.4E1,6.5E1,1.6E1,7E0,4E0,2E1,1.41E2,6.1E1,4.2E1,9E0,7E0,1.1E1,1.1E1,6E0,4E0,1.3E1,1.2E1,9E1,9.2E1,3.4E1,4E0,2.2E1,5E0,1.2E1,5E0,7E0,8E0,9E0,1.7E1,1.4E1,1.19E2,5E0,5E0,6.8E1,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[2.3541178E-3,-1.0076356E-2,1.1024005E-2,-1.7554654E-2,1.3774852E-2,2.0662153E-2,-1.594192E-2,5.9360176E-2,-1.9919986E-2,1.778151E-2,-7.6012365E-3,9.3990706E-2,1.8102398E-2,-2.2331562E-2,2.9632527E-2,7.263993E-2,-2.8135424E-4,-2.2711974E-2,4.3376457E-2,3.3544432E-2,-1.1751618E-2,5.5772346E-2,1.1236847E-1,6.3707076E-2,1.419489E-2,-1.8810397E-2,-1.2817712E-1,5.5979E-2,-1.8851893E-2,6.05469E-3,4.318403E-2,-3.934291E-4,-3.2883897E-2,8.664072E-2,-8.006873E-3,5.5824418E-2,9.406185E-3,-8.217176E-3,1.7566617E-3,4.086485E-3,-3.70047E-4,6.378951E-3,2.8473067E-3,3.8516387E-2,1.3294828E-1,2.375635E-2,2.5864076E-3,-3.359089E-3,-3.0044891E-2,-2.06209E-3,-1.1203157E-2,8.6434454E-2,2.0121386E-2,-3.9430847E-3,5.156243E-4,3.4603255E-3,3.6086634E-4,-4.7173824E-2,1.3982633E-2,-1.2140564E-1,-2.9578459E-2,2.1555688E-2,1.4933069E-1,-2.694468E-2,1.9509605E-3,1.0362575E-2,7.414173E-2,-3.2938323E-3,1.7761227E-2,-1.2743787E-2,4.5895196E-2,2.756352E-2,7.997512E-3,6.2164288E-2,1.0576577E-2,7.7370787E-3,3.939521E-2,-9.945947E-3,2.8145047E-2,1.818462E-4,-5.0528925E-3,-2.2988424E-2,-9.815637E-2,6.2060244E-5,1.21827796E-1,-1.4480948E-3,2.252268E-3,-3.1150779E-2,-1.0052E-2,8.305158E-2,-4.812588E-4,-8.658323E-3,5.609944E-4,-3.5435792E-2,1.4009023E-2,-9.705359E-4,3.318775E-3,9.927503E-3,3.5032989E-3,-2.2431042E-3,3.5776081E-4,-2.2846484E-3,4.476868E-2,8.9205176E-2,-1.9455744E-2,2.6893059E-2,-3.449359E-2,-3.0944305E-2,3.618369E-2,8.631593E-2,1.1755049E-2,3.4020243E-3,4.9277063E-2,9.23953E-4,4.3545165E-3,1.1270808E-2,-9.013551E-3,2.347199E-2,6.3879155E-2,6.432535E-3,-1.2254955E-2,2.4167547E-2,6.8090837E-3,-1.9443158E-2,1.3263017E-2,5.233703E-3,-2.7285565E-2,-9.359561E-3,-3.796683E-2,7.4099507E-3,2.753669E-3,-4.969813E-2,4.0812142E-2,3.6897026E-2,1.6361466E-1,-8.696618E-3,5.507133E-3,-3.1076832E-2,-2.171481E-1,-5.097794E-2,3.168608E-2,6.7442295E-4,4.1495482E-3,6.154908E-2,1.1150878E-1,2.7997603E-3,-4.54961E-3,-2.4478936E-3,3.6062006E-2,-3.4018483E-3,9.7082194E-4,2.8606886E-3,-4.0945206E-2,5.002678E-3,8.8093225E-3,8.38877E-4,6.9243773E-3,-1.31397E-3,2.7076863E-3,-2.0770583E-2,1.36648E-3,6.337588E-2,-1.8343739E-3,1.9948907E-2,-5.00804E-2,-3.615403E-2,3.3731982E-2,1.441144E-1,5.4121763E-2,-6.707179E-2,-6.9505544E-3,2.8547486E-2,-5.118E-3,4.987576E-3,-4.58683E-2,-2.1077362E-3,2.132574E-2,-1.2650791E-2,-5.3282484E-2,-3.272055E-4,-4.0294537E-3,-1.2752682E-3,-6.6366E-3,5.2282754E-3,-1.4896564E-3,3.2286567E-3,-1.387686E-3,1.0961866E-2,5.016963E-3,-4.7070677E-3,6.00465E-4,3.027627E-4,-1.957751E-3,-2.5966128E-3,-1.5919765E-2,-5.9829936E-3,2.4583961E-3,2.6761503E-3,-1.1840336E-3,5.4103956E-3,2.0532783E-3,6.6272663E-3,1.6091286E-3,7.103065E-4,3.6766685E-3,-3.0813601E-3,4.213638E-4,1.4784675E-3,-1.516793E-3,2.1390848E-4,-2.951106E-3,5.430015E-3,2.1030142E-3,1.4126837E-3,-1.8507243E-3,-9.4428373E-4,-1.1038669E-2,2.195196E-3,-4.6206377E-3,3.9133267E-3,1.2258588E-3,3.349139E-3,8.923484E-3,-6.6134875E-4,3.233029E-3,-6.585553E-3,-1.8131984E-3,-7.746082E-4,9.916979E-4,1.1343823E-3,9.02623E-3,7.9944445E-4,-2.084956E-3,-3.6738154E-3,-9.414216E-4,3.887844E-3,6.1498757E-4,-1.763547E-3,4.9055147E-4,1.9410152E-3,-3.2835219E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,-1,55,57,59,61,63,65,67,-1,69,-1,-1,-1,-1,71,73,75,77,79,81,-1,-1,83,85,-1,-1,-1,-1,87,89,91,93,95,97,99,-1,101,103,-1,105,107,109,111,-1,113,-1,115,117,119,121,123,-1,125,127,-1,129,-1,-1,131,-1,133,135,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,141,143,145,147,149,151,153,155,157,159,161,-1,-1,163,-1,165,167,-1,169,171,-1,173,175,-1,177,-1,179,-1,-1,181,183,185,187,-1,189,191,193,195,197,-1,-1,199,201,-1,-1,-1,203,-1,-1,-1,205,-1,207,-1,-1,-1,-1,209,-1,211,-1,213,215,217,219,221,223,225,227,229,-1,231,233,-1,235,237,239,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7406448E-1,1.8663478E-1,3.896619E-1,1.4599799E-1,1.6888216E-1,2.0552284E-1,1.1568749E-1,2.1122567E-2,1.3759193E-1,1.1444867E-1,0E0,1.9720554E-2,1.89414E-1,1.2691572E-1,6.364759E-2,2.3359835E-2,0E0,1.6821074E-1,7.405476E-2,8.552286E-2,1.7768411E-1,2.3613054E-2,1.3056517E-2,1.4139494E-1,1.0926767E-1,5.839812E-2,8.255547E-2,3.320674E-2,3.235208E-2,0E0,1.2624536E-2,1.5735626E-1,1.4534402E-1,7.02243E-2,1.5309082E-2,6.834903E-2,5.0260782E-2,0E0,5.178716E-2,0E0,0E0,0E0,0E0,7.945497E-2,1.0943481E-1,1.349777E-1,1.4313816E-1,4.9522657E-2,9.146571E-2,0E0,0E0,5.110784E-2,2.0911261E-2,0E0,0E0,0E0,0E0,1.294487E-1,1.7886198E-1,1.2579477E-1,1.2610719E-1,1.9675128E-2,2.1255791E-2,7.6664817E-3,0E0,5.0289515E-2,8.491963E-2,0E0,3.4807578E-2,5.5440508E-2,2.6584744E-2,3.0785628E-2,0E0,1.2573194E-2,0E0,1.8019816E-1,1.0493326E-1,9.7643346E-2,5.9927948E-2,3.594168E-2,0E0,1.0055931E-1,8.9484885E-2,0E0,1.3019547E-2,0E0,0E0,7.0099086E-2,0E0,1.0923204E-1,1.558273E-1,0E0,0E0,3.3701313E-1,6.93932E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8030904E-2,2.4595141E-2,5.3744666E-2,4.4208825E-2,2.1246072E-2,4.2036075E-2,2.8871821E-2,3.037735E-2,2.0801764E-2,1.7354175E-2,3.9909296E-2,0E0,0E0,1.4155579E-1,0E0,1.0338386E-1,7.596916E-2,0E0,8.5601866E-2,8.2215E-2,0E0,3.6381565E-2,3.8451325E-2,0E0,6.514536E-2,0E0,1.5561631E-2,0E0,0E0,7.900153E-2,5.270218E-2,3.9438702E-2,1.674825E-2,0E0,9.525713E-2,1.2814659E-1,1.5103516E-1,9.544127E-2,5.8203172E-2,0E0,0E0,1.9448832E-2,4.096201E-2,0E0,0E0,0E0,4.4036724E-2,0E0,0E0,0E0,4.2474017E-2,0E0,1.1478229E-2,0E0,0E0,0E0,0E0,1.4641576E-2,0E0,2.146113E-2,0E0,1.1109567E-1,1.7162137E-1,1.1682629E-1,5.6727037E-2,1.726237E-2,6.999555E-2,4.7023207E-2,6.2413517E-2,1.1946505E-1,0E0,1.610019E-2,1.879707E-2,0E0,3.6734343E-2,5.7328343E-2,7.356271E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,43,43,44,44,45,45,46,46,47,47,48,48,51,51,52,52,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,68,68,69,69,70,70,71,71,73,73,75,75,76,76,77,77,78,78,79,79,81,81,82,82,84,84,87,87,89,89,90,90,93,93,94,94,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,115,115,117,117,118,118,120,120,121,121,123,123,124,124,126,126,128,128,131,131,132,132,133,133,134,134,136,136,137,137,138,138,139,139,140,140,143,143,144,144,148,148,152,152,154,154,159,159,161,161,163,163,164,164,165,165,166,166,167,167,168,168,169,169,170,170,171,171,173,173,174,174,176,176,177,177,178,178],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,-1,56,58,60,62,64,66,68,-1,70,-1,-1,-1,-1,72,74,76,78,80,82,-1,-1,84,86,-1,-1,-1,-1,88,90,92,94,96,98,100,-1,102,104,-1,106,108,110,112,-1,114,-1,116,118,120,122,124,-1,126,128,-1,130,-1,-1,132,-1,134,136,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,142,144,146,148,150,152,154,156,158,160,162,-1,-1,164,-1,166,168,-1,170,172,-1,174,176,-1,178,-1,180,-1,-1,182,184,186,188,-1,190,192,194,196,198,-1,-1,200,202,-1,-1,-1,204,-1,-1,-1,206,-1,208,-1,-1,-1,-1,210,-1,212,-1,214,216,218,220,222,224,226,228,230,-1,232,234,-1,236,238,240,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.494881E2,2.13019E6,3.0161016E7,5E-1,1E0,3.79E2,6.082581E3,3.8575E4,2.273139E1,7.2744107E0,-7.6012365E-3,1.2993591E6,1.1317E4,1.2367184E3,6.747114E7,6.285983E5,-2.8135424E-4,4.8E1,1.2375429E8,2.53E2,4.1323944E-4,1.6344065E4,1.0013907E1,1E0,8.1980075E5,2.585911E5,8.91942E5,8.639937E9,1.4563726E-1,6.05469E-3,1.2252E4,3.75E1,7.93E2,2.8E1,2.2784335E6,1.79325E2,1.2897E4,-8.217176E-3,4.1891047E5,4.086485E-3,-3.70047E-4,6.378951E-3,2.8473067E-3,3.0769388E7,1.9711539E0,2.98E2,1.2204E4,5.4210526E1,5.23896E5,-2.06209E-3,-1.1203157E-2,5.501614E9,3.0769388E7,-3.9430847E-3,5.156243E-4,3.4603255E-3,3.6086634E-4,2.0011E5,1.7723127E5,7.2744107E0,2.7485228E6,2.6666667E0,4.4866666E2,3.979829E1,1.9509605E-3,9.51505E-1,3.0608664E2,-3.2938323E-3,3.8338925E8,9.756616E2,2.7604443E2,1.6764174E1,7.997512E-3,1.2399164E5,1.0576577E-2,9.216E3,9.9625344E5,1.610753E3,4.4543375E4,6.015E3,-5.0528925E-3,3.907936E2,5.3E1,6.2060244E-5,2.2153166E7,-1.4480948E-3,2.252268E-3,3.1E2,-1.0052E-2,1.452881E6,4.30858E-6,-8.658323E-3,5.609944E-4,2.298425E6,1.6954315E0,-9.705359E-4,3.318775E-3,9.927503E-3,3.5032989E-3,-2.2431042E-3,3.5776081E-4,-2.2846484E-3,1.411E3,7.59E2,1.3676985E4,1.44E0,3.4047124E-1,7.860526E1,3.084419E6,9.27E2,1.627E3,4.3359457E10,3.6E1,9.23953E-4,4.3545165E-3,5.8E1,-9.013551E-3,2.579E3,1.4643513E-1,6.432535E-3,6.9434594E5,5.68E2,6.8090837E-3,1.990351E1,2.308943E0,5.233703E-3,3.6E1,-9.359561E-3,3.653309E7,7.4099507E-3,2.753669E-3,8.2E1,1.24E2,9.232009E2,7.056912E7,-8.696618E-3,1.56E2,1.710933E0,1.631E4,8.459239E6,2.7302878E2,6.7442295E-4,4.1495482E-3,4.914413E-3,1.681727E6,2.7997603E-3,-4.54961E-3,-2.4478936E-3,2.6680525E6,-3.4018483E-3,9.7082194E-4,2.8606886E-3,1.7022566E7,5.002678E-3,3.4692544E7,8.38877E-4,6.9243773E-3,-1.31397E-3,2.7076863E-3,4.9473104E-1,1.36648E-3,1.509E3,-1.8343739E-3,5.077728E8,1.45E2,1.6711E4,5.59277E5,7.5773336E7,1.5116882E8,4.516183E1,3.771981E6,2.62797E5,-5.118E-3,5.441E3,4.587916E6,-2.1077362E-3,1.958583E3,8.496E3,3.23153E5,-3.272055E-4,-4.0294537E-3,-1.2752682E-3,-6.6366E-3,5.2282754E-3,-1.4896564E-3,3.2286567E-3,-1.387686E-3,1.0961866E-2,5.016963E-3,-4.7070677E-3,6.00465E-4,3.027627E-4,-1.957751E-3,-2.5966128E-3,-1.5919765E-2,-5.9829936E-3,2.4583961E-3,2.6761503E-3,-1.1840336E-3,5.4103956E-3,2.0532783E-3,6.6272663E-3,1.6091286E-3,7.103065E-4,3.6766685E-3,-3.0813601E-3,4.213638E-4,1.4784675E-3,-1.516793E-3,2.1390848E-4,-2.951106E-3,5.430015E-3,2.1030142E-3,1.4126837E-3,-1.8507243E-3,-9.4428373E-4,-1.1038669E-2,2.195196E-3,-4.6206377E-3,3.9133267E-3,1.2258588E-3,3.349139E-3,8.923484E-3,-6.6134875E-4,3.233029E-3,-6.585553E-3,-1.8131984E-3,-7.746082E-4,9.916979E-4,1.1343823E-3,9.02623E-3,7.9944445E-4,-2.084956E-3,-3.6738154E-3,-9.414216E-4,3.887844E-3,6.1498757E-4,-1.763547E-3,4.9055147E-4,1.9410152E-3,-3.2835219E-3],"split_indices":[52,9,45,53,65,29,52,9,56,56,0,45,9,58,45,28,0,10,7,10,41,28,56,63,32,28,9,5,38,0,9,4,12,8,50,55,1,0,47,0,0,0,0,1,53,0,2,53,9,0,0,5,1,0,0,0,0,9,45,56,28,53,4,56,0,58,52,0,7,55,52,56,0,33,0,10,28,33,4,2,0,52,8,0,1,0,0,1,0,31,41,0,0,28,54,0,0,0,0,0,0,0,29,2,33,58,41,52,9,2,2,19,8,0,0,3,0,2,42,0,50,3,0,58,54,0,3,0,48,0,0,29,29,33,7,0,2,54,9,45,4,0,0,38,28,0,0,0,45,0,0,0,9,0,45,0,0,0,0,42,0,10,0,7,10,29,1,7,12,54,1,2,0,2,50,0,4,10,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.541E3,1.044E3,1.497E3,7.95E2,2.49E2,1.103E3,3.94E2,2.3E1,7.72E2,2.44E2,5E0,3.6E1,1.067E3,3.46E2,4.8E1,1.9E1,4E0,7.4E2,3.2E1,1.59E2,8.5E1,1.3E1,2.3E1,8.3E1,9.84E2,3.36E2,1E1,3.1E1,1.7E1,6E0,1.3E1,2.32E2,5.08E2,1.7E1,1.5E1,8.2E1,7.7E1,6E0,7.9E1,9E0,4E0,1.7E1,6E0,6.2E1,2.1E1,5.39E2,4.45E2,1.42E2,1.94E2,6E0,4E0,1.6E1,1.5E1,5E0,1.2E1,7E0,6E0,5.4E1,1.78E2,1.7E1,4.91E2,9E0,8E0,1.1E1,4E0,2.4E1,5.8E1,7E0,7E1,6E1,1.9E1,5.8E1,4E0,1.2E1,9E0,2.67E2,2.72E2,2.99E2,1.46E2,1.38E2,4E0,1.77E2,1.7E1,5E0,1.1E1,5E0,1E1,5E1,4E0,3E1,1.48E2,1.2E1,5E0,4.33E2,5.8E1,5E0,4E0,4E0,4E0,7E0,4E0,9E0,1.5E1,5E1,8E0,6E1,1E1,4.4E1,1.6E1,8E0,1.1E1,2.8E1,3E1,5E0,7E0,2.63E2,4E0,1.66E2,1.06E2,4E0,2.95E2,1.42E2,4E0,5.5E1,8.3E1,5E0,1.72E2,6E0,1.1E1,7E0,4E0,4E1,1E1,2E1,1E1,4E0,1.44E2,4.24E2,9E0,1.2E1,4.6E1,9E0,6E0,2.4E1,2.6E1,4E0,4E0,6E0,5.4E1,6E0,4E0,4E0,4E1,4E0,1.2E1,4E0,4E0,6E0,5E0,1.4E1,1.4E1,2.6E1,4E0,2.31E2,3.2E1,2.4E1,1.42E2,1E1,9.6E1,2.5E1,2.7E2,1.38E2,4E0,2.9E1,2.6E1,1E1,7.3E1,1.11E2,6.1E1,7E0,4E0,3.2E1,8E0,5E0,5E0,1.4E1,6E0,4E0,6E0,8E0,1.36E2,7.6E1,3.48E2,4E0,5E0,7E0,5E0,3.3E1,1.3E1,6E0,1.8E1,2E1,6E0,3.5E1,1.9E1,2.8E1,1.2E1,8E0,4E0,9E0,5E0,7E0,1.9E1,2.02E2,2.9E1,2.8E1,4E0,1E1,1.4E1,2.3E1,1.19E2,4E0,6E0,1.3E1,8.3E1,7E0,1.8E1,2.05E2,6.5E1,1.34E2,4E0,2.4E1,5E0,1.2E1,1.4E1,9E0,6.4E1,5.5E1,5.6E1,7E0,5.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"241","size_leaf_vector":"1"}},{"base_weights":[-2.1596793E-3,1.2236774E-3,-4.771495E-2,-1.0007034E-2,1.6408268E-2,-4.221578E-2,-8.59891E-3,-5.6353128E-3,-4.5997184E-2,1.398199E-2,9.9355586E-2,-3.530677E-2,-1.4715657E-1,5.6321E-4,-2.5542412E-2,-1.0858856E-1,-2.872475E-2,6.481589E-2,1.0355628E-2,1.690394E-1,3.6527764E-2,-4.971524E-2,-3.1901712E-3,-9.13471E-5,-1.217875E-2,-1.9791925E-3,6.6350006E-2,-1.0635577E-1,-1.6288E-2,3.9901376E-2,-1.7114814E-1,4.1741364E-2,-3.9712083E-2,9.970901E-2,3.305561E-2,-8.567704E-2,1.2188623E-2,1.19585255E-2,4.756401E-3,6.3530942E-3,-2.1356178E-4,-3.8691334E-2,-9.989735E-2,-3.690547E-3,6.30339E-3,1.2120379E-3,-4.9869686E-2,9.6715376E-2,-5.9335423E-3,-9.648783E-3,-1.4939147E-1,-4.472853E-2,-5.7630804E-3,-2.762173E-4,4.318938E-3,-1.6595105E-2,-9.002465E-2,-4.6703542E-4,4.123012E-3,-2.8247215E-2,-1.4007115E-1,1.4258544E-1,5.853377E-2,-2.5602505E-3,5.2262776E-2,-1.7735778E-1,1.5523363E-2,2.1617277E-2,-2.9163782E-3,-4.50379E-2,-5.6824303E-4,-7.628058E-3,-1.214508E-3,3.6135104E-2,-1.688652E-2,2.218266E-3,-8.217232E-3,-2.9229557E-2,-7.6007266E-3,1.1675198E-3,1.2554361E-1,-3.3456127E-3,1.9238275E-3,1.9973933E-3,-2.4680237E-3,-1.4557225E-2,-8.265314E-2,-1.0301943E-2,-8.913969E-2,4.5861036E-2,-1.2529353E-2,3.0344243E-3,-1.6406985E-1,-5.872594E-2,-1.3768958E-2,-1.1850404E-2,-1.7894789E-3,8.427973E-3,3.0927074E-3,-9.4329036E-5,5.0475164E-3,-1.3799809E-3,7.0055366E-2,-1.3349129E-2,-2.6024296E-3,3.9397026E-3,-2.5435314E-3,4.3509394E-2,1.3270219E-2,5.9601287E-3,-2.8504064E-2,-5.2665036E-2,1.3001133E-3,-1.852043E-3,2.059834E-3,3.806388E-3,2.1363083E-2,-2.3796193E-3,2.1877547E-3,-3.730652E-2,5.6046685E-3,-6.0869837E-3,-4.094218E-3,3.109147E-3,8.176775E-3,-5.3570555E-3,-1.3963294E-3,-2.8340945E-2,3.936763E-3,-4.267579E-2,-1.4661743E-1,4.654388E-3,1.7582929E-2,-8.482658E-2,-5.875518E-3,-1.5234887E-2,-2.0005456E-3,-6.297531E-3,-3.9451845E-2,-2.6466362E-2,2.8627161E-2,9.080357E-2,1.8627518E-3,7.902535E-2,3.2418292E-2,5.774825E-2,8.110473E-3,-7.17186E-2,1.0790216E-2,-1.632694E-2,-8.2280695E-2,-6.365969E-2,-2.0980262E-3,-1.3746496E-3,2.355349E-3,2.7511236E-3,-5.477686E-4,-1.2302841E-3,9.518491E-4,-4.2007264E-4,-4.1576955E-3,6.957818E-3,1.6330254E-4,-3.311901E-3,1.1840364E-3,-1.5378959E-4,-2.5296148E-3,1.8186624E-3,-4.259437E-3,-4.2810137E-3,-9.324386E-3,-2.9436251E-4,3.2832786E-3,-7.897193E-3,-5.0733425E-4,-5.2656344E-4,3.5185483E-3,7.8255474E-4,-3.397103E-3,1.4149846E-3,-1.8910787E-3,2.560566E-3,-1.3420333E-3,1.1250555E-3,5.789485E-3,2.4785479E-3,6.2458124E-3,2.448265E-4,2.2607257E-3,1.497611E-3,7.0381435E-3,5.085775E-4,-4.8188497E-3,8.3917484E-4,-7.558312E-3,3.0699442E-3,1.0654479E-4,-1.1355041E-3,2.3607383E-3,-9.2592783E-4,-8.074485E-3,-3.5461083E-3,-1.7280608E-3,8.828309E-4,-1.8616572E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,69,71,-1,73,75,77,79,81,83,85,87,89,-1,-1,-1,91,-1,-1,93,95,97,99,-1,101,103,105,107,109,111,113,-1,-1,115,117,119,-1,121,-1,-1,123,-1,-1,-1,-1,-1,125,127,129,131,133,-1,135,137,139,-1,-1,-1,-1,-1,-1,-1,141,-1,-1,-1,-1,143,145,147,149,151,153,-1,-1,-1,155,-1,157,159,161,-1,163,-1,-1,-1,-1,165,-1,167,169,-1,171,173,175,-1,-1,-1,177,179,181,183,-1,185,187,189,191,193,195,197,199,201,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8569093E-1,3.9750785E-1,1.11062646E-1,2.1057491E-1,1.9805291E-1,1.15885556E-1,0E0,1.4759433E-1,1.536411E-1,1.7703453E-1,1.16411686E-1,7.295604E-2,1.3991395E-1,1.5283652E-1,2.1066564E-1,2.9480255E-1,9.06946E-2,6.7629814E-2,1.5964487E-1,4.4698507E-2,5.970213E-2,5.6433916E-2,3.4011073E-2,0E0,0E0,1.3455047E-1,7.520105E-2,1.19252294E-1,7.63718E-2,2.1618403E-2,2.5739616E-1,3.410037E-2,1.1110373E-1,4.4617355E-2,5.9345134E-2,1.6048135E-1,1.2610461E-1,0E0,0E0,0E0,0E0,2.1991909E-2,7.207276E-2,0E0,3.174096E-2,1.3805744E-1,1.1162439E-1,4.7815338E-2,3.2405797E-2,2.1697067E-2,1.6538638E-1,1.0457751E-1,6.624354E-2,0E0,0E0,0E0,1.8856508E-1,0E0,0E0,3.9347768E-2,9.273529E-2,1.9028842E-2,4.3828793E-2,0E0,4.3152086E-2,8.73355E-2,4.1871548E-2,9.901303E-2,7.7620976E-2,2.7635723E-2,2.2882354E-2,0E0,0E0,1.0203041E-2,1.5347929E-2,1.1023119E-1,0E0,1.0888502E-1,0E0,0E0,3.0830204E-2,0E0,0E0,0E0,0E0,0E0,1.715786E-2,6.651651E-2,7.3862195E-2,2.77814E-2,7.983759E-2,0E0,1.7993614E-1,3.2649815E-2,3.4671858E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2935646E-2,0E0,0E0,0E0,0E0,5.6524843E-2,9.041179E-2,9.60734E-2,5.6282938E-2,3.711623E-2,1.714485E-2,0E0,0E0,0E0,1.803163E-2,0E0,8.15416E-3,8.552738E-2,2.362321E-1,0E0,6.7244165E-2,0E0,0E0,0E0,0E0,1.8985868E-2,0E0,6.369312E-2,1.5696168E-2,0E0,1.7332986E-2,7.494392E-2,5.5525914E-2,0E0,0E0,0E0,3.8809348E-2,3.162212E-2,2.0201288E-2,2.1352306E-2,0E0,4.11413E-2,4.0721998E-2,9.042369E-2,7.7948354E-2,1.0815695E-1,1.0511363E-1,3.0767154E-2,7.790823E-2,9.23498E-3,9.693114E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,56,56,59,59,60,60,61,61,62,62,64,64,65,65,66,66,67,67,68,68,69,69,70,70,73,73,74,74,75,75,77,77,80,80,86,86,87,87,88,88,89,89,90,90,92,92,93,93,94,94,102,102,107,107,108,108,109,109,110,110,111,111,112,112,116,116,118,118,119,119,120,120,122,122,127,127,129,129,130,130,132,132,133,133,134,134,138,138,139,139,140,140,141,141,143,143,144,144,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,70,72,-1,74,76,78,80,82,84,86,88,90,-1,-1,-1,92,-1,-1,94,96,98,100,-1,102,104,106,108,110,112,114,-1,-1,116,118,120,-1,122,-1,-1,124,-1,-1,-1,-1,-1,126,128,130,132,134,-1,136,138,140,-1,-1,-1,-1,-1,-1,-1,142,-1,-1,-1,-1,144,146,148,150,152,154,-1,-1,-1,156,-1,158,160,162,-1,164,-1,-1,-1,-1,166,-1,168,170,-1,172,174,176,-1,-1,-1,178,180,182,184,-1,186,188,190,192,194,196,198,200,202,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,3.171E3,2.462277E6,9.837297E0,4.5505118E2,5.39978E5,-8.59891E-3,5.2187E4,8E0,2.139E3,3.5905025E5,1.0052E4,1.2E1,3.1615953E1,2.6446635E8,1.1880085E2,3.114E3,3.0200147E8,1.43E2,1.942E3,1.4307411E7,1.515252E10,1.7892157E-1,-9.13471E-5,-1.217875E-2,1E0,8.959817E7,2.5797562E2,3.7246967E2,1.56E2,4.4195585E0,4E0,1.948181E0,8.901186E0,1.6908121E1,1.8E1,1.775044E7,1.19585255E-2,4.756401E-3,6.3530942E-3,-2.1356178E-4,5E0,2.974979E6,-3.690547E-3,4.9473104E-1,1.6904226E7,5.485425E6,1.945345E6,1.1997242E0,2.3066575E6,1.3797468E0,2.0469315E0,2.7123368E10,-2.762173E-4,4.318938E-3,-1.6595105E-2,3.1E1,-4.6703542E-4,4.123012E-3,2.558106E10,1.692E3,1.7605528E7,2.0754387E1,-2.5602505E-3,3.544243E8,9.60094E5,2.22747E5,3.3382E4,1.0033929E1,1E0,1.3715873E3,-7.628058E-3,-1.214508E-3,2.7324794E11,6.08767E3,4.7E2,-8.217232E-3,2.8331464E2,-7.6007266E-3,1.1675198E-3,1.892E3,-3.3456127E-3,1.9238275E-3,1.9973933E-3,-2.4680237E-3,-1.4557225E-2,1.862E3,2.9845978E2,2.909019E3,1.5E1,1.5825138E5,3.0344243E-3,2.853839E4,1.1924399E0,1.74123E5,-1.1850404E-2,-1.7894789E-3,8.427973E-3,3.0927074E-3,-9.4329036E-5,5.0475164E-3,-1.3799809E-3,5.597E3,-1.3349129E-2,-2.6024296E-3,3.9397026E-3,-2.5435314E-3,2.558106E10,8.756142E7,8.890291E2,1.91E4,6.298144E0,6.015E3,-1.852043E-3,2.059834E-3,3.806388E-3,3.7075245E1,-2.3796193E-3,1.0881593E10,1.5444445E1,6.377113E4,-6.0869837E-3,5.76E3,3.109147E-3,8.176775E-3,-5.3570555E-3,-1.3963294E-3,9E0,3.936763E-3,1.04E3,7.680554E10,4.654388E-3,3E0,1.8E1,7.2E1,-1.5234887E-2,-2.0005456E-3,-6.297531E-3,2.3362158E1,1.773747E2,8.91E2,7.8E1,1.8627518E-3,1.8E1,1.2287474E3,2.130881E6,7.065733E7,1.6E1,1.1317E4,1.5153E4,1.49E2,9.0206254E-1,7.6599895E9,-1.3746496E-3,2.355349E-3,2.7511236E-3,-5.477686E-4,-1.2302841E-3,9.518491E-4,-4.2007264E-4,-4.1576955E-3,6.957818E-3,1.6330254E-4,-3.311901E-3,1.1840364E-3,-1.5378959E-4,-2.5296148E-3,1.8186624E-3,-4.259437E-3,-4.2810137E-3,-9.324386E-3,-2.9436251E-4,3.2832786E-3,-7.897193E-3,-5.0733425E-4,-5.2656344E-4,3.5185483E-3,7.8255474E-4,-3.397103E-3,1.4149846E-3,-1.8910787E-3,2.560566E-3,-1.3420333E-3,1.1250555E-3,5.789485E-3,2.4785479E-3,6.2458124E-3,2.448265E-4,2.2607257E-3,1.497611E-3,7.0381435E-3,5.085775E-4,-4.8188497E-3,8.3917484E-4,-7.558312E-3,3.0699442E-3,1.0654479E-4,-1.1355041E-3,2.3607383E-3,-9.2592783E-4,-8.074485E-3,-3.5461083E-3,-1.7280608E-3,8.828309E-4,-1.8616572E-3],"split_indices":[45,2,28,54,56,11,0,29,17,29,28,2,3,56,5,52,29,7,0,0,51,5,57,0,0,74,12,4,4,2,58,8,42,56,58,3,45,0,0,0,0,6,29,0,42,47,9,12,41,45,53,56,31,0,0,0,0,0,0,19,2,12,58,0,7,9,1,9,53,59,52,0,0,31,4,1,0,4,0,0,2,0,0,0,0,0,2,55,32,3,28,0,33,58,29,0,0,0,0,0,0,0,2,0,0,0,0,19,7,4,10,57,2,0,0,0,56,0,12,52,45,0,12,0,0,0,0,3,0,2,31,0,8,3,3,0,0,0,58,52,0,0,0,3,55,1,1,3,32,10,8,27,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.501E3,2.329E3,1.72E2,1.339E3,9.9E2,1.66E2,6E0,1.195E3,1.44E2,9.63E2,2.7E1,1.57E2,9E0,9.12E2,2.83E2,3E1,1.14E2,6.3E1,9E2,1.2E1,1.5E1,1.08E2,4.9E1,4E0,5E0,8.79E2,3.3E1,2.8E1,2.55E2,9E0,2.1E1,1.5E1,9.9E1,2.9E1,3.4E1,1.6E1,8.84E2,5E0,7E0,4E0,1.1E1,9E1,1.8E1,5E0,4.4E1,8.25E2,5.4E1,2.3E1,1E1,9E0,1.9E1,6.8E1,1.87E2,5E0,4E0,6E0,1.5E1,7E0,8E0,9E1,9E0,1.3E1,1.6E1,6E0,2.8E1,8E0,8E0,5.44E2,3.4E2,7.7E1,1.3E1,1E1,8E0,1.9E1,2.5E1,8.21E2,4E0,4.6E1,8E0,7E0,1.6E1,4E0,6E0,4E0,5E0,5E0,1.4E1,3.9E1,2.9E1,2.1E1,1.66E2,5E0,1E1,2.8E1,6.2E1,4E0,5E0,9E0,4E0,7E0,9E0,5E0,2.3E1,4E0,4E0,4E0,4E0,1.49E2,3.95E2,2.53E2,8.7E1,6.6E1,1.1E1,7E0,6E0,4E0,1.5E1,9E0,1.6E1,6.4E1,7.57E2,9E0,3.7E1,7E0,9E0,9E0,5E0,3.3E1,6E0,1.7E1,1.2E1,7E0,1.4E1,1.3E1,1.53E2,4E0,6E0,5E0,2.3E1,4.8E1,1.4E1,1.3E1,1E1,3.4E1,1.15E2,4E1,3.55E2,1.4E1,2.39E2,7.2E1,1.5E1,5.4E1,1.2E1,7E0,4E0,7E0,8E0,6E0,1E1,4E1,2.4E1,1.2E1,7.45E2,1.1E1,2.6E1,1.6E1,1.7E1,6E0,1.1E1,6E0,6E0,1E1,4E0,6E0,7E0,1.45E2,8E0,8E0,1.5E1,8E0,4E1,1E1,4E0,4E0,9E0,2.2E1,1.2E1,3.7E1,7.8E1,3.1E1,9E0,3.49E2,6E0,7E0,7E0,3.4E1,2.05E2,6.6E1,6E0,9E0,6E0,4.2E1,1.2E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"205","size_leaf_vector":"1"}},{"base_weights":[-1.1599025E-3,-7.469967E-3,1.6915768E-2,-5.3156884E-3,-7.060152E-2,2.3460612E-2,-3.3729948E-2,9.70033E-3,-1.1828884E-2,-8.212988E-2,1.7490867E-3,3.8191568E-2,1.0142682E-2,-2.1617066E-2,-7.669935E-3,1.48459785E-2,-9.833554E-2,-3.0472692E-2,-5.9019034E-3,-5.9462167E-2,-1.8498868E-1,3.4141853E-2,1.5858932E-1,5.7718074E-3,8.2427114E-2,-2.7541708E-2,3.1619638E-3,1.34666255E-2,6.809631E-3,-1.8456286E-2,-1.6548517E-1,-1.5522653E-2,-7.438127E-2,-7.140185E-3,8.146472E-2,-1.0953591E-2,-7.489863E-2,-1.3277728E-2,-2.5532618E-3,4.57801E-2,6.387636E-3,3.81911E-3,1.0444413E-2,8.496846E-3,-7.830018E-3,1.2699482E-1,-2.8730623E-4,1.4313653E-3,-3.5510894E-2,1.891808E-2,-3.120484E-2,3.743203E-3,-6.9890353E-3,-1.6266875E-2,-4.038501E-2,-4.1647684E-2,-9.2484916E-5,-1.09712124E-1,-3.809797E-2,-8.1559387E-4,-2.63969E-2,6.186127E-3,2.1869157E-4,-1.722106E-3,1.6767734E-3,-8.490168E-2,3.0485707E-4,9.943371E-2,3.9080065E-2,8.041447E-2,-8.508982E-3,5.1264797E-4,3.8622487E-2,9.330165E-3,1.8058228E-3,1.0278236E-2,-4.967016E-2,1.603974E-2,8.446182E-2,-6.6173063E-3,-7.881966E-2,-5.1270784E-3,1.4948504E-3,-1.2099651E-1,-3.2200344E-2,5.5856332E-2,-2.309861E-2,-1.4016679E-1,-4.314153E-2,2.4844147E-3,-5.7640832E-2,1.7312372E-3,-5.0739124E-2,-7.60158E-2,-1.8237665E-2,-1.1263927E-3,-1.0082355E-1,7.6878285E-3,6.123107E-2,3.0980295E-2,1.0395289E-1,5.6189317E-3,2.5043343E-3,3.7349926E-3,-1.606349E-2,4.861317E-3,-4.1811343E-2,5.3580295E-2,-4.4516154E-2,1.7345198E-3,-2.2752362E-3,-6.5603174E-2,-4.1784174E-3,1.9964118E-2,-4.5501083E-2,1.0445616E-1,1.7926E-4,-3.9066657E-2,4.4498388E-2,3.2156461E-3,-1.1863577E-1,1.659195E-3,-1.2541621E-2,-2.400482E-2,-5.14454E-3,7.7242223E-3,3.299942E-2,-9.34544E-3,-1.4916348E-2,-1.6134112E-1,-7.3608855E-4,-5.2439147E-3,-7.92401E-3,-1.2051307E-2,-9.749533E-2,1.5652485E-2,-7.7857375E-3,4.8121944E-4,-6.564327E-2,2.9399822E-2,-1.0958213E-1,-2.236019E-2,2.5806446E-3,-2.390238E-3,-5.554372E-3,1.1470976E-3,4.3504364E-3,8.082368E-2,2.394429E-2,8.851055E-3,4.6499293E-2,-5.508617E-2,9.702839E-3,4.3384116E-2,-2.9134434E-3,-2.5094273E-3,-1.5842568E-4,6.7741126E-2,-1.408537E-2,1.3423538E-3,-6.0594724E-3,-3.8668064E-3,-8.809084E-4,9.91741E-4,-1.7746814E-3,1.2997438E-3,-1.114042E-3,3.12771E-5,-3.766728E-3,2.7809588E-3,6.40425E-3,-2.9107665E-3,4.4912886E-4,-1.3970486E-3,3.5466852E-3,-1.0038761E-2,-3.1466165E-3,-4.688619E-3,-6.3551555E-4,-2.8049073E-4,3.4914499E-3,2.6833252E-3,-1.4542772E-3,-9.357185E-3,-2.831541E-3,1.2190612E-3,-1.6757783E-3,-3.3809147E-3,2.2434941E-3,-6.3862777E-3,3.4411644E-4,1.2060121E-3,-1.6513846E-3,-2.0941498E-3,-1.2635367E-4,-4.091744E-3,1.4348609E-3,-1.4867083E-3,4.1309283E-3,-7.774239E-3,-2.145685E-3,-7.738829E-4,-3.9909775E-3,6.69044E-4,5.194075E-3,2.6184585E-5,2.194665E-3,3.4305647E-3,8.729208E-5,-1.6058389E-3,-7.247166E-3,-6.13655E-4,2.356518E-3,3.3332375E-3,-4.2151683E-3,-8.575231E-6,-4.911454E-3,4.8271124E-3,1.2329576E-3,-1.9248216E-3,9.0218795E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,51,53,55,57,59,61,63,65,-1,-1,67,69,-1,-1,71,-1,73,-1,-1,75,77,79,-1,-1,-1,81,83,85,87,89,91,93,-1,-1,-1,-1,95,-1,97,99,101,103,105,107,-1,-1,109,111,113,115,117,119,-1,-1,121,123,125,127,129,131,-1,133,135,137,139,141,-1,143,-1,145,147,149,-1,-1,-1,151,153,155,157,159,-1,-1,161,163,165,167,169,-1,171,173,-1,175,-1,-1,177,-1,-1,179,-1,181,183,-1,-1,185,187,189,191,193,-1,195,197,199,201,-1,-1,-1,-1,-1,203,205,-1,207,209,211,213,215,-1,-1,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9884595E-1,2.6374245E-1,2.2552681E-1,1.8388519E-1,8.079308E-2,1.1752027E-1,1.09842055E-1,3.1777528E-1,1.4468758E-1,1.2423146E-1,0E0,1.3354483E-1,9.96792E-2,3.7958395E-2,0E0,8.928749E-2,1.3640615E-1,2.0627886E-1,1.08333424E-1,3.587082E-2,9.452766E-2,8.918145E-2,2.131772E-2,1.3415802E-1,7.146749E-2,3.146586E-2,0E0,1.3185003E-1,0E0,1.5785694E-1,2.6719838E-1,9.553407E-2,9.83738E-2,1.1969889E-1,4.5153186E-2,1.4761474E-2,3.0497104E-2,0E0,0E0,6.621295E-2,9.221244E-2,0E0,0E0,7.117681E-2,0E0,5.8375493E-2,0E0,0E0,3.941094E-2,8.9389324E-2,6.855156E-2,0E0,0E0,0E0,4.2440612E-2,6.2071756E-2,1.9436502E-1,7.504842E-2,7.3231936E-2,9.430362E-2,9.6992746E-2,0E0,0E0,0E0,0E0,3.04918E-2,0E0,3.424214E-2,8.873412E-2,7.952288E-3,4.5229223E-2,4.345572E-2,7.948663E-2,0E0,0E0,2.2037914E-2,3.3067636E-2,1.1248569E-1,3.1965777E-2,6.8085246E-2,1.2164874E-1,0E0,0E0,1.8875618E-1,4.3921113E-2,9.4193086E-2,1.4073548E-1,6.933528E-2,2.9641904E-2,0E0,6.060128E-2,9.3658514E-2,3.3023648E-2,1.23527855E-1,6.165638E-2,0E0,8.924663E-3,0E0,1.1844818E-2,5.312437E-2,7.165465E-2,0E0,0E0,0E0,6.5890126E-2,6.4220585E-2,6.990649E-3,5.173886E-2,5.8045905E-2,0E0,0E0,1.7857328E-2,1.0493466E-2,1.117623E-1,3.8643736E-2,1.138258E-2,0E0,2.357513E-2,3.2686662E-2,0E0,6.061922E-2,0E0,0E0,5.6338187E-2,0E0,0E0,5.307649E-2,0E0,1.0184406E-1,5.1807225E-2,0E0,0E0,9.097042E-3,5.6879986E-2,5.783929E-2,1.1938096E-1,7.52401E-2,0E0,4.680814E-2,3.0822044E-2,7.303092E-2,7.7415384E-2,0E0,0E0,0E0,0E0,0E0,2.8239578E-2,6.501461E-2,0E0,1.1779385E-2,4.9009368E-2,3.351931E-2,1.1289445E-1,4.679493E-2,0E0,0E0,5.2312985E-2,8.549802E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,43,43,45,45,48,48,49,49,50,50,54,54,55,55,56,56,57,57,58,58,59,59,60,60,65,65,67,67,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91,92,92,93,93,94,94,96,96,98,98,99,99,100,100,104,104,105,105,106,106,107,107,108,108,111,111,112,112,113,113,114,114,115,115,117,117,118,118,120,120,123,123,126,126,128,128,129,129,132,132,133,133,134,134,135,135,136,136,138,138,139,139,140,140,141,141,147,147,148,148,150,150,151,151,152,152,153,153,154,154,157,157,158,158],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,52,54,56,58,60,62,64,66,-1,-1,68,70,-1,-1,72,-1,74,-1,-1,76,78,80,-1,-1,-1,82,84,86,88,90,92,94,-1,-1,-1,-1,96,-1,98,100,102,104,106,108,-1,-1,110,112,114,116,118,120,-1,-1,122,124,126,128,130,132,-1,134,136,138,140,142,-1,144,-1,146,148,150,-1,-1,-1,152,154,156,158,160,-1,-1,162,164,166,168,170,-1,172,174,-1,176,-1,-1,178,-1,-1,180,-1,182,184,-1,-1,186,188,190,192,194,-1,196,198,200,202,-1,-1,-1,-1,-1,204,206,-1,208,210,212,214,216,-1,-1,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.067536E3,6.971004E7,9.302862E7,2E0,1.0052E4,1.4540612E7,1.5988282E1,9.669789E0,8.87E2,1.903E3,1.7490867E-3,1.921032E4,6.2277466E2,9.609747E3,-7.669935E-3,1.03992E3,8.07966E5,9.1E3,1.3238013E2,1.8159722E-1,4.230839E6,8.606353E1,7.8E1,1.9013363E4,2.6104508E10,5.26337E5,3.1619638E-3,2E0,6.809631E-3,2.2401E4,2.853839E4,3.7761906E1,2.8729197E8,1.7291142E7,1.8631586E4,9.38E2,8.012376E0,-1.3277728E-2,-2.5532618E-3,1.4157E4,1.4811083E0,3.81911E-3,1.0444413E-2,7.9771865E3,-7.830018E-3,2.9964134E9,-2.8730623E-4,1.4313653E-3,1E0,3.557179E1,1.823E3,3.743203E-3,-6.9890353E-3,-1.6266875E-2,3.3751E4,7.2727275E-1,4.162121E0,1.1383217E5,1.9266E4,1E0,1.9473622E7,6.186127E-3,2.1869157E-4,-1.722106E-3,1.6767734E-3,7E0,3.0485707E-4,1.6943169E5,1.216918E7,2.3920168E6,4.911E3,3.4306667E1,2.3580047E10,9.330165E-3,1.8058228E-3,9.115134E5,3.757042E6,1.4433751E0,1.4E1,2.9221216E8,1.2715517E0,-5.1270784E-3,1.4948504E-3,1.8731416E4,4.296196E0,4.0340412E7,1.2E1,8.230099E4,2.7302878E2,2.4844147E-3,9.069307E1,1.2982479E-2,2E0,5.5303917E8,8.0752426E1,-1.1263927E-3,1.8367347E0,7.6878285E-3,1.318688E7,9.330579E0,8.0663525E-4,5.6189317E-3,2.5043343E-3,3.7349926E-3,3.5151117E-2,1.3206031E5,9.3E1,9.74026E0,1.3674345E4,1.7345198E-3,-2.2752362E-3,9.2593566E-2,6.1707315E0,1E0,5.2228385E6,1E0,1.7926E-4,4.1221875E-1,7E0,3.2156461E-3,8.113553E0,1.659195E-3,-1.2541621E-2,6E0,-5.14454E-3,7.7242223E-3,3.7333333E0,-9.34544E-3,5.314985E3,8.588109E0,-7.3608855E-4,-5.2439147E-3,2.3449652E0,1.9670312E2,3.8977896E2,4.70092E1,8.088206E-2,4.8121944E-4,2.7102E4,1.99E2,4.349136E6,1.62E2,2.5806446E-3,-2.390238E-3,-5.554372E-3,1.1470976E-3,4.3504364E-3,1.1925198E3,8.18E3,8.851055E-3,3.6930276E3,1.9213884E2,3.0497742E3,7.562698E1,1.2709078E0,-2.5094273E-3,-1.5842568E-4,1.4930757E0,4.9E1,1.3423538E-3,-6.0594724E-3,-3.8668064E-3,-8.809084E-4,9.91741E-4,-1.7746814E-3,1.2997438E-3,-1.114042E-3,3.12771E-5,-3.766728E-3,2.7809588E-3,6.40425E-3,-2.9107665E-3,4.4912886E-4,-1.3970486E-3,3.5466852E-3,-1.0038761E-2,-3.1466165E-3,-4.688619E-3,-6.3551555E-4,-2.8049073E-4,3.4914499E-3,2.6833252E-3,-1.4542772E-3,-9.357185E-3,-2.831541E-3,1.2190612E-3,-1.6757783E-3,-3.3809147E-3,2.2434941E-3,-6.3862777E-3,3.4411644E-4,1.2060121E-3,-1.6513846E-3,-2.0941498E-3,-1.2635367E-4,-4.091744E-3,1.4348609E-3,-1.4867083E-3,4.1309283E-3,-7.774239E-3,-2.145685E-3,-7.738829E-4,-3.9909775E-3,6.69044E-4,5.194075E-3,2.6184585E-5,2.194665E-3,3.4305647E-3,8.729208E-5,-1.6058389E-3,-7.247166E-3,-6.13655E-4,2.356518E-3,3.3332375E-3,-4.2151683E-3,-8.575231E-6,-4.911454E-3,4.8271124E-3,1.2329576E-3,-1.9248216E-3,9.0218795E-4],"split_indices":[52,45,45,17,2,45,54,54,2,0,0,4,56,52,0,46,9,29,56,27,1,58,3,52,19,1,0,6,0,1,33,52,5,45,33,11,53,0,0,9,53,0,0,55,0,12,0,0,75,58,29,0,0,0,1,53,54,47,29,74,45,0,0,0,0,3,0,33,45,28,29,53,5,0,0,32,29,42,8,31,54,0,0,28,58,31,12,32,4,0,52,39,8,5,54,0,53,0,50,56,39,0,0,0,57,32,8,53,4,0,0,38,54,74,47,8,0,38,6,0,46,0,0,52,0,0,54,0,50,56,0,0,53,4,4,58,39,0,10,0,32,8,0,0,0,0,0,52,2,0,55,58,52,58,34,0,0,42,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.618E3,1.941E3,6.77E2,1.878E3,6.3E1,6E2,7.7E1,5.68E2,1.31E3,5.7E1,6E0,2.84E2,3.16E2,7.1E1,6E0,5.43E2,2.5E1,3.15E2,9.95E2,4.8E1,9E0,2.76E2,8E0,2.99E2,1.7E1,6.7E1,4E0,5.38E2,5E0,1.2E1,1.3E1,2.36E2,7.9E1,9.82E2,1.3E1,1.2E1,3.6E1,5E0,4E0,1.94E2,8.2E1,4E0,4E0,2.95E2,4E0,1.1E1,6E0,8E0,5.9E1,4.8E2,5.8E1,7E0,5E0,5E0,8E0,8.7E1,1.49E2,3.9E1,4E1,7.4E2,2.42E2,8E0,5E0,8E0,4E0,3.2E1,4E0,2E1,1.74E2,1.3E1,6.9E1,2.34E2,6.1E1,6E0,5E0,1.4E1,4.5E1,4.61E2,1.9E1,3.9E1,1.9E1,4E0,4E0,8E0,7.9E1,4.3E1,1.06E2,2.6E1,1.3E1,7E0,3.3E1,7.05E2,3.5E1,3.3E1,2.09E2,7E0,2.5E1,7E0,1.3E1,1.56E2,1.8E1,5E0,8E0,5E0,6.4E1,2.13E2,2.1E1,5.2E1,9E0,1E1,4E0,3.3E1,1.2E1,4.34E2,2.7E1,1.5E1,4E0,2.4E1,1.5E1,4E0,1.5E1,4E0,4E0,7.2E1,7E0,7E0,3.6E1,4E0,1.02E2,2.2E1,4E0,4E0,9E0,1.6E1,1.7E1,2.86E2,4.19E2,7E0,2.8E1,8E0,2.5E1,1.98E2,1.1E1,5E0,2E1,6E0,7E0,1.8E1,1.38E2,7E0,1.1E1,2.5E1,3.9E1,3.5E1,1.78E2,1.7E1,4E0,4.3E1,9E0,5E0,4E0,2.6E1,7E0,7E0,5E0,3.8E2,5.4E1,1.1E1,1.6E1,6E0,9E0,1.7E1,7E0,4E0,1.1E1,5E0,1E1,9E0,6.3E1,1.8E1,1.8E1,1.7E1,8.5E1,1.7E1,5E0,4E0,5E0,8E0,8E0,1.3E1,4E0,2.44E2,4.2E1,5.5E1,3.64E2,2.4E1,4E0,4E0,4E0,1.4E1,1.1E1,1.78E2,2E1,5E0,1.3E1,6.4E1,7.4E1,7E0,4E0,2.1E1,4E0,2.5E1,1.4E1,3E1,5E0,1.74E2,4E0,2.5E1,1.8E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-4.7107582E-4,-1.2032748E-2,8.825353E-3,-1.822235E-2,9.296241E-3,1.4513272E-2,-2.9416772E-2,-5.3729404E-2,-1.28065795E-2,2.934415E-2,-1.8308582E-2,5.7329223E-3,2.8473428E-2,-1.7975336E-2,-7.350512E-2,-3.2432698E-2,-1.17969885E-1,2.1487537E-3,-2.2115218E-2,3.5999715E-2,-2.523498E-2,-6.968451E-3,-1.2421075E-2,1.2812262E-3,3.2070607E-2,3.0529954E-2,-8.685465E-3,-3.3065077E-2,3.0308155E-2,-1.2056799E-2,-4.666795E-2,-4.3287177E-2,5.1192977E-3,-3.248723E-1,-2.3679819E-2,-1.4218243E-2,2.960274E-2,-3.1718038E-2,4.4653094E-3,2.6043204E-3,4.6778575E-2,8.506525E-3,-5.392198E-3,3.3260796E-3,-2.2267638E-2,9.644553E-3,-2.6623301E-2,4.238207E-2,-6.354903E-3,3.281912E-2,-6.9639E-2,-2.4014631E-2,-7.7053376E-2,5.5869743E-3,1.0279846E-2,-1.6169025E-2,-1.10090505E-2,3.32366E-3,-5.1078837E-2,-5.365377E-3,-2.3853635E-2,-8.535093E-2,2.9686093E-2,-1.9009117E-2,6.7091812E-3,3.551816E-2,-5.201073E-3,-2.3204243E-2,-7.827186E-2,1.6615536E-2,-6.1976694E-2,1.9128563E-2,-2.6727438E-3,6.1538476E-2,2.6715174E-2,2.8827435E-3,-9.875735E-4,-5.0430936E-3,-6.546119E-2,4.519167E-3,4.92442E-2,-1.9333292E-2,-9.7101E-2,8.263221E-2,1.6341489E-2,2.784601E-2,8.533521E-2,-4.8446083E-3,-8.727314E-4,1.4306476E-2,-3.244204E-2,-2.3210682E-2,-6.1554303E-3,2.3446696E-2,-3.0101785E-3,-4.880047E-2,1.16550485E-2,-1.2692383E-1,-3.560027E-2,-3.8493643E-4,-6.7870556E-3,3.8985831E-3,-2.1164173E-4,-7.7768816E-3,-7.17682E-2,2.8327279E-2,6.756769E-3,-1.164496E-1,-1.8899735E-2,-9.872217E-2,2.581161E-3,-8.883279E-3,4.9205348E-2,-9.926187E-3,-3.1731247E-3,-9.939688E-4,4.3252364E-2,3.7121754E-2,9.9805765E-2,-1.9424522E-3,5.4211274E-2,-6.6006154E-2,8.949238E-3,-1.14961796E-1,-1.1284789E-2,7.445434E-3,-4.160144E-2,5.9339374E-2,-2.4052565E-3,-6.9139902E-3,-1.478124E-2,-7.380337E-3,-2.0328788E-3,6.2101915E-2,1.0894229E-2,-1.42686E-2,4.298638E-2,3.4385126E-2,-1.5600048E-3,1.071848E-1,2.5420755E-2,4.9416903E-2,-9.3555375E-4,-4.654041E-3,-2.5246914E-2,-2.251833E-3,3.826786E-4,7.493211E-3,5.2748475E-2,-8.4832996E-2,6.686409E-4,4.0048383E-2,-9.010635E-4,-3.9068093E-3,-8.531837E-3,-1.1626651E-2,-8.410874E-2,-2.7597854E-2,2.3934105E-2,-3.7000507E-2,-1.3477921E-1,3.693923E-2,-4.627516E-2,-1.5621658E-2,2.261021E-3,3.5213407E-2,-2.4425233E-2,-1.3393992E-1,-4.874113E-2,8.133593E-2,-2.5738163E-2,1.9469604E-2,8.346261E-2,1.3490033E-3,-2.0958524E-2,-5.567661E-4,6.0846455E-2,2.1499017E-2,5.046885E-3,1.2211257E-1,2.2737223E-3,-3.2382227E-2,1.993899E-2,5.9262495E-3,3.4412578E-2,-7.2586135E-4,-6.1780848E-3,6.938403E-2,-8.708575E-3,-6.9873272E-3,-2.922277E-3,1.9907455E-3,-1.7847815E-3,8.724225E-3,-5.0697206E-3,-6.166163E-2,-7.6223956E-4,1.23457536E-1,4.519709E-2,-3.5231907E-2,4.149387E-3,1.07151255E-1,3.3235554E-2,3.4565812E-3,-4.4320244E-3,-1.6717134E-3,6.945124E-2,2.9440517E-2,1.0361953E-1,-7.7822566E-2,1.3260492E-2,1.2817565E-1,1.1443738E-3,-3.6377183E-4,2.4892737E-3,4.2271503E-4,4.021841E-3,-1.8232407E-2,-4.3859244E-3,1.758377E-3,-6.661474E-4,3.944861E-3,7.993372E-4,-1.6488907E-3,-5.980989E-3,4.2435586E-5,3.5595158E-3,1.5246442E-3,-2.4106258E-3,-3.857529E-4,-7.351221E-3,-1.9141524E-3,2.1028502E-3,3.2432799E-3,-2.0884723E-5,-4.243319E-3,9.314909E-4,-9.598796E-3,-1.3775147E-3,2.270669E-3,-1.6256837E-4,6.669192E-4,-6.0205613E-3,2.8069385E-3,-1.6341856E-3,4.491136E-5,-1.6576843E-3,-8.12331E-3,-1.4441193E-3,-4.038623E-3,-2.6805865E-4,7.6673757E-3,-1.0726808E-3,-4.222189E-3,-2.262504E-4,-4.4865764E-4,3.0812384E-3,5.3782146E-3,8.1374816E-4,-2.145886E-3,4.806078E-4,4.14655E-3,1.0585716E-3,-2.1259452E-4,2.6519974E-3,4.093706E-3,8.431052E-3,-2.9154827E-3,2.6251565E-4,-1.268928E-4,3.0433831E-3,1.8738545E-4,4.3191635E-3,2.2932852E-4,7.205821E-3,-1.5384805E-3,3.1150535E-3,1.3420444E-4,1.4953497E-3,-8.0778234E-4,-4.13687E-3,2.0450575E-3,9.058713E-3,-1.5478925E-3,2.667442E-3,-2.1676465E-3,1.9233986E-3,1.5798933E-3,-4.832695E-4,2.067333E-3,6.4619216E-3,-4.2679545E-4,2.232466E-3,-4.6603344E-4,2.3795436E-3,-3.3691209E-3,1.5676185E-3,-3.501559E-4,4.338679E-3,2.3587325E-3,6.6673197E-4,6.446218E-3,3.2074717E-3,-8.243757E-4,-8.626313E-3,2.7338734E-3,-5.570652E-4,7.087504E-3,2.5282314E-3,-1.5242352E-3,3.8471603E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,-1,51,53,-1,55,57,-1,59,61,63,65,67,69,71,73,75,-1,-1,77,79,81,83,-1,85,87,89,91,-1,93,95,-1,-1,97,-1,-1,99,101,103,-1,105,-1,107,109,111,113,115,-1,117,119,-1,-1,121,123,125,127,129,131,133,135,137,139,-1,-1,141,143,145,-1,147,-1,149,151,153,155,-1,-1,-1,-1,157,159,161,-1,163,165,167,-1,169,171,-1,173,-1,175,177,179,181,183,185,187,189,191,193,195,197,-1,-1,199,-1,-1,201,-1,203,205,207,209,211,213,215,-1,-1,217,-1,-1,219,221,223,-1,225,-1,-1,-1,227,229,231,233,235,237,239,241,-1,-1,243,245,247,249,251,253,255,257,-1,259,-1,261,263,-1,265,-1,267,269,-1,271,-1,-1,273,275,-1,-1,-1,-1,277,-1,279,-1,281,283,285,287,289,291,293,-1,295,297,299,301,303,305,307,-1,-1,-1,-1,-1,309,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7805534E-1,1.5238608E-1,3.123479E-1,1.7122644E-1,1.444262E-1,1.5293303E-1,9.250343E-2,1.5761095E-1,1.0821664E-1,5.575995E-2,7.672979E-2,9.000489E-2,2.0583454E-1,1.0956136E-1,1.5985526E-1,1.362064E-1,5.580635E-1,1.3477334E-1,1.2240541E-1,4.847336E-2,4.834423E-2,0E0,8.392386E-2,1.5381731E-1,1.8766737E-1,1.118924E-1,0E0,4.3534555E-2,5.761787E-2,0E0,1.765165E-1,7.526478E-2,0E0,2.4728417E-1,7.164758E-2,1.3629629E-1,9.2272714E-2,1.3769886E-1,1.0436231E-1,3.2538645E-2,2.8561637E-2,1.9217683E-2,0E0,0E0,7.034886E-2,1.02679946E-1,7.621774E-2,1.0870406E-1,0E0,1.1985046E-1,1.3984207E-2,3.1467106E-2,4.433968E-2,0E0,2.9269716E-2,2.7806962E-2,0E0,0E0,8.822526E-2,0E0,0E0,3.8150378E-2,2.0006828E-2,1.07888125E-1,0E0,7.3603675E-2,0E0,1.178104E-1,1.4858308E-1,9.1016285E-2,1.6011447E-1,2.6276577E-2,0E0,5.0488025E-2,3.5564084E-2,0E0,0E0,5.9448767E-2,7.1110316E-2,6.1042294E-2,5.9875235E-2,7.327472E-2,3.39593E-2,1.0042986E-1,5.362752E-2,8.254573E-2,4.9639523E-2,0E0,0E0,2.1799836E-2,3.2151714E-2,7.091966E-3,0E0,1.1758847E-2,0E0,3.2503363E-2,1.5052371E-2,1.44162625E-2,7.693797E-2,0E0,0E0,0E0,0E0,9.619899E-2,6.511702E-2,6.7241564E-2,0E0,4.265628E-1,8.6673684E-2,7.6201946E-2,0E0,9.637275E-2,4.7270946E-2,0E0,8.61494E-3,0E0,1.6947553E-2,3.4755662E-2,2.1091476E-2,1.596519E-2,2.524107E-2,3.6776904E-2,6.1431576E-2,1.1492103E-2,1.8912649E-2,5.9935596E-2,1.3329264E-2,4.2479277E-2,0E0,0E0,5.2410617E-2,0E0,0E0,4.4553965E-2,0E0,4.1205E-2,4.1481562E-2,1.17109746E-1,9.0321645E-2,4.8369825E-2,9.936748E-3,1.0773171E-2,0E0,0E0,2.9980097E-2,0E0,0E0,1.08517E-2,7.4033607E-3,1.2315154E-2,0E0,1.0942878E-2,0E0,0E0,0E0,7.2453745E-2,1.04761824E-1,7.1769595E-2,5.8898557E-2,6.024702E-2,6.230797E-2,3.1595036E-2,5.0282825E-2,0E0,0E0,4.0279116E-2,5.7867423E-2,7.651019E-2,2.8792951E-2,7.7430114E-2,6.567387E-2,3.296534E-2,3.3972725E-2,0E0,7.1363198E-3,0E0,9.657018E-3,2.5435276E-2,0E0,1.4276996E-2,0E0,1.0079251E-2,1.3047692E-2,0E0,2.998675E-2,0E0,0E0,6.236823E-2,7.2521016E-2,0E0,0E0,0E0,0E0,5.369195E-2,0E0,1.3081897E-2,0E0,3.553945E-2,2.9719152E-2,4.0725783E-2,2.7261881E-2,1.5068263E-2,1.1855526E-2,1.5123645E-2,0E0,3.253596E-2,2.9843733E-2,9.345505E-2,1.4743686E-2,7.274241E-2,6.840373E-2,1.7212152E-2,0E0,0E0,0E0,0E0,0E0,2.1239517E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,27,27,28,28,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,54,54,55,55,58,58,61,61,62,62,63,63,65,65,67,67,68,68,69,69,70,70,71,71,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,89,89,90,90,91,91,93,93,95,95,96,96,97,97,98,98,103,103,104,104,105,105,107,107,108,108,109,109,111,111,112,112,114,114,116,116,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,126,126,127,127,130,130,133,133,135,135,136,136,137,137,138,138,139,139,140,140,141,141,144,144,147,147,148,148,149,149,151,151,155,155,156,156,157,157,158,158,159,159,160,160,161,161,162,162,165,165,166,166,167,167,168,168,169,169,170,170,171,171,172,172,174,174,176,176,177,177,179,179,181,181,182,182,184,184,187,187,188,188,193,193,195,195,197,197,198,198,199,199,200,200,201,201,202,202,203,203,205,205,206,206,207,207,208,208,209,209,210,210,211,211,217,217],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,-1,52,54,-1,56,58,-1,60,62,64,66,68,70,72,74,76,-1,-1,78,80,82,84,-1,86,88,90,92,-1,94,96,-1,-1,98,-1,-1,100,102,104,-1,106,-1,108,110,112,114,116,-1,118,120,-1,-1,122,124,126,128,130,132,134,136,138,140,-1,-1,142,144,146,-1,148,-1,150,152,154,156,-1,-1,-1,-1,158,160,162,-1,164,166,168,-1,170,172,-1,174,-1,176,178,180,182,184,186,188,190,192,194,196,198,-1,-1,200,-1,-1,202,-1,204,206,208,210,212,214,216,-1,-1,218,-1,-1,220,222,224,-1,226,-1,-1,-1,228,230,232,234,236,238,240,242,-1,-1,244,246,248,250,252,254,256,258,-1,260,-1,262,264,-1,266,-1,268,270,-1,272,-1,-1,274,276,-1,-1,-1,-1,278,-1,280,-1,282,284,286,288,290,292,294,-1,296,298,300,302,304,306,308,-1,-1,-1,-1,-1,310,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7311627E2,2.13019E6,6.747114E7,8E0,3.7246967E2,1E0,2.8493E5,3.8977896E2,2.9138756E0,1.8E1,7.79E2,3.970405E3,3.080528E1,1.3664E4,4.230839E6,2.046E3,1.7232166E8,6.851852E-1,2.728889E2,1.4925373E-1,7.954694E4,-6.968451E-3,3.928E4,2.1616615E6,5.68E2,2.851369E2,-8.685465E-3,9.2057055E-1,3.307766E6,-1.2056799E-2,1.5988282E1,4.004E3,5.1192977E-3,1.260173E6,5.815045E2,6.38041E5,1.1658537E1,2.4E1,1E0,1.4260634E8,3.8866E4,9.25E0,-5.392198E-3,3.3260796E-3,3.1530054E0,2.160255E9,9.123214E-1,1.1336898E-1,-6.354903E-3,2.0718E4,4.5505118E2,2.2222222E-1,7.6499896E9,5.5869743E-3,9.377185E-1,3E1,-1.10090505E-2,3.32366E-3,4.36E2,-5.365377E-3,-2.3853635E-2,7.27E2,4.1891047E5,2.5363636E0,6.7091812E-3,5.5325594E9,-5.201073E-3,6.6861836E-4,1.3065217E1,6.0809356E5,1.2920592E0,2.82E2,-2.6727438E-3,1.411E3,9.705292E6,2.8827435E-3,-9.875735E-4,3.5677967E0,2.9816154E1,7.806529E1,6.90802E5,1.70752E4,7.621583E2,1.5005797E4,1.9745072E7,2.330807E0,4.35581E5,-4.8446083E-3,-8.727314E-4,2.7487562E0,1.0526649E3,9.442914E-1,-6.1554303E-3,1.5005797E4,-3.0101785E-3,9.287868E6,2.7777777E0,2.6610387E-3,1.85221E5,-3.8493643E-4,-6.7870556E-3,3.8985831E-3,-2.1164173E-4,5.116325E3,1E0,1.7291142E7,6.756769E-3,5.2921E4,2.0260927E-3,1.7593515E1,2.581161E-3,6.647279E5,3.627907E0,-9.926187E-3,1.6748592E0,-9.939688E-4,1.28715766E5,1E0,4.085E3,3.61E2,1.13742945E5,1.4811083E0,3.01E2,2.429717E1,1.0506024E1,1E0,5.10008E2,1.5818182E1,-2.4052565E-3,-6.9139902E-3,3.088954E-1,-7.380337E-3,-2.0328788E-3,2.5E1,1.0894229E-2,3.036676E9,2.4832896E9,1.980161E0,2.03E2,2.3977574E7,2.1270284E7,4.5808794E5,-9.3555375E-4,-4.654041E-3,4.1822004E7,-2.251833E-3,3.826786E-4,8.352216E-2,1.9745072E7,1.5E1,6.686409E-4,5.5743E4,-9.010635E-4,-3.9068093E-3,-8.531837E-3,7.5939475E5,5.724838E5,6.576923E0,5.5306E4,5.4907064E0,5.28E2,1E0,1.6592433E9,-1.5621658E-2,2.261021E-3,6.737509E7,1.5139E4,1E0,1.5361997E7,2.36118E5,1.13E3,3.2485715E2,2.8209653E-2,1.3490033E-3,4.25E0,-5.567661E-4,6.49E2,2.1818182E0,5.046885E-3,2.6668E4,2.2737223E-3,4.7777777E0,5.901422E5,5.9262495E-3,1.544E3,-7.2586135E-4,-6.1780848E-3,1.4757962E1,1.0712973E7,-6.9873272E-3,-2.922277E-3,1.9907455E-3,-1.7847815E-3,2.804366E-1,-5.0697206E-3,1.7387315E2,-7.6223956E-4,1.7227913E3,1.4912975E-1,1.91E2,1.775044E7,2.958763E0,4.1179886E0,1E0,-4.4320244E-3,1.003252E5,3.1415796E0,3.68E0,4.1844E4,3.2627738E5,1.6947798E6,3.8E1,1.1443738E-3,-3.6377183E-4,2.4892737E-3,4.2271503E-4,4.021841E-3,1.992945E5,-4.3859244E-3,1.758377E-3,-6.661474E-4,3.944861E-3,7.993372E-4,-1.6488907E-3,-5.980989E-3,4.2435586E-5,3.5595158E-3,1.5246442E-3,-2.4106258E-3,-3.857529E-4,-7.351221E-3,-1.9141524E-3,2.1028502E-3,3.2432799E-3,-2.0884723E-5,-4.243319E-3,9.314909E-4,-9.598796E-3,-1.3775147E-3,2.270669E-3,-1.6256837E-4,6.669192E-4,-6.0205613E-3,2.8069385E-3,-1.6341856E-3,4.491136E-5,-1.6576843E-3,-8.12331E-3,-1.4441193E-3,-4.038623E-3,-2.6805865E-4,7.6673757E-3,-1.0726808E-3,-4.222189E-3,-2.262504E-4,-4.4865764E-4,3.0812384E-3,5.3782146E-3,8.1374816E-4,-2.145886E-3,4.806078E-4,4.14655E-3,1.0585716E-3,-2.1259452E-4,2.6519974E-3,4.093706E-3,8.431052E-3,-2.9154827E-3,2.6251565E-4,-1.268928E-4,3.0433831E-3,1.8738545E-4,4.3191635E-3,2.2932852E-4,7.205821E-3,-1.5384805E-3,3.1150535E-3,1.3420444E-4,1.4953497E-3,-8.0778234E-4,-4.13687E-3,2.0450575E-3,9.058713E-3,-1.5478925E-3,2.667442E-3,-2.1676465E-3,1.9233986E-3,1.5798933E-3,-4.832695E-4,2.067333E-3,6.4619216E-3,-4.2679545E-4,2.232466E-3,-4.6603344E-4,2.3795436E-3,-3.3691209E-3,1.5676185E-3,-3.501559E-4,4.338679E-3,2.3587325E-3,6.6673197E-4,6.446218E-3,3.2074717E-3,-8.243757E-4,-8.626313E-3,2.7338734E-3,-5.570652E-4,7.087504E-3,2.5282314E-3,-1.5242352E-3,3.8471603E-4],"split_indices":[52,9,45,3,4,63,11,4,58,3,2,52,53,2,1,2,7,58,52,56,33,0,12,47,3,58,0,27,1,0,54,9,0,50,4,9,57,8,105,7,1,58,0,0,53,12,27,57,0,2,56,58,5,0,27,8,0,0,2,0,0,0,47,53,0,5,0,41,56,28,56,29,0,29,9,0,0,56,56,53,10,33,52,4,1,41,29,0,0,54,33,27,0,4,0,1,53,39,9,0,0,0,0,33,66,45,0,1,42,58,0,45,56,0,53,0,28,100,29,0,50,53,0,53,58,85,33,56,0,0,57,0,0,8,0,7,7,42,10,47,51,28,0,0,51,0,0,38,1,3,0,10,0,0,0,28,48,56,29,54,0,105,5,0,0,7,9,101,12,1,2,52,38,0,54,0,29,56,0,1,0,54,50,0,2,0,0,58,32,0,0,0,0,57,0,33,0,4,41,8,45,53,53,74,0,28,54,54,9,28,45,3,0,0,0,0,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.585E3,1.152E3,1.433E3,8.93E2,2.59E2,1.248E3,1.85E2,1.17E2,7.76E2,1.5E2,1.09E2,7.67E2,4.81E2,1.48E2,3.7E1,8.9E1,2.8E1,2.98E2,4.78E2,1.34E2,1.6E1,4E0,1.05E2,6.57E2,1.1E2,4.77E2,4E0,1.13E2,3.5E1,4E0,3.3E1,8.3E1,6E0,8E0,2E1,1.87E2,1.11E2,3.51E2,1.27E2,3.3E1,1.01E2,1.2E1,4E0,1.1E1,9.4E1,5.06E2,1.51E2,1.04E2,6E0,4.67E2,1E1,9.5E1,1.8E1,6E0,2.9E1,2.9E1,4E0,5E0,7.8E1,4E0,4E0,9E0,1.1E1,1.82E2,5E0,1.07E2,4E0,2.98E2,5.3E1,1.08E2,1.9E1,2.6E1,7E0,5.7E1,4.4E1,4E0,8E0,6.8E1,2.6E1,4.49E2,5.7E1,1.38E2,1.3E1,4E1,6.4E1,4.28E2,3.9E1,6E0,4E0,1.7E1,7.8E1,9E0,9E0,2.5E1,4E0,1.3E1,1.6E1,1.2E1,6.6E1,4E0,5E0,4E0,7E0,1.51E2,3.1E1,1.01E2,6E0,1.2E1,2.86E2,4.6E1,7E0,6.1E1,4.7E1,5E0,1.4E1,1E1,1.6E1,3.6E1,2.1E1,2.2E1,2.2E1,1.2E1,5.6E1,1.3E1,1.3E1,4.23E2,2.6E1,5.2E1,5E0,4E0,1.34E2,6E0,7E0,3.6E1,4E0,3E1,3.4E1,3.5E2,7.8E1,2.8E1,1.1E1,8E0,9E0,7E0,7.1E1,5E0,4E0,1.7E1,8E0,8E0,5E0,8E0,8E0,7E0,5E0,4.5E1,2.1E1,9.3E1,5.8E1,2.1E1,1E1,9.1E1,1E1,5E0,7E0,2.6E1,2.6E2,2.6E1,2E1,9E0,5.2E1,2.6E1,2.1E1,5E0,9E0,4E0,1.2E1,3E1,6E0,1.4E1,7E0,9E0,1.3E1,4E0,1.8E1,7E0,5E0,1.2E1,4.4E1,8E0,5E0,4E0,9E0,4.19E2,4E0,1.4E1,1.2E1,8E0,4.4E1,6.4E1,7E1,1.3E1,2.3E1,2.5E1,5E0,1.3E1,2.1E1,3.28E2,2.2E1,1.2E1,6.6E1,2.2E1,6E0,5E0,6E0,4E0,4E0,6.5E1,6E0,7E0,1E1,4E0,4E0,4E0,4E0,4E0,4E0,2.1E1,2.4E1,1E1,1.1E1,8.1E1,1.2E1,2.1E1,3.7E1,1.1E1,1E1,6E0,4E0,7.5E1,1.6E1,6E0,4E0,2E1,6E0,6.7E1,1.93E2,2E1,6E0,1.1E1,9E0,5E0,4E0,1.3E1,3.9E1,1.6E1,1E1,1.5E1,6E0,5E0,4E0,7E0,5E0,1.7E1,1.3E1,9E0,5E0,5E0,4E0,9E0,4E0,1.2E1,6E0,7E0,5E0,3.4E1,1E1,3.27E2,9.2E1,5E0,9E0,4E0,4E0,4E0,4E1,5.8E1,6E0,2.3E1,4.7E1,4E0,9E0,5E0,1.8E1,2E1,5E0,4E0,9E0,4E0,1.7E1,1.55E2,1.73E2,1.2E1,1E1,8E0,4E0,2.4E1,4.2E1,1.8E1,4E0,4.4E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"311","size_leaf_vector":"1"}},{"base_weights":[-3.3631343E-3,-2.1090899E-2,2.172843E-3,5.0782505E-2,-2.450662E-2,1.0880924E-2,-1.0919936E-2,1.5507836E-2,8.032003E-2,-1.5950372E-2,-5.1467095E-2,6.2692486E-2,8.375389E-3,-1.405064E-2,8.053133E-2,-1.7065309E-3,4.6892226E-2,6.2127435E-3,2.2923402E-3,-1.434486E-2,-7.6971385E-3,-1.1269166E-2,-4.1989982E-2,8.41571E-2,-5.3981405E-2,3.5061154E-3,3.4151897E-2,-8.787683E-2,-1.2335646E-2,1.0103639E-1,-1.289E-3,3.6037462E-3,6.137407E-4,-1.1259077E-2,-8.606382E-2,-1.623318E-3,-6.10465E-2,9.919629E-2,1.1307521E-2,-6.0160365E-3,1.1609487E-3,-6.1863016E-2,5.5718343E-3,3.0257259E-2,6.2443307E-3,1.7569128E-2,-1.0428026E-2,-9.553131E-3,-4.9739573E-2,9.500915E-3,6.364295E-2,-7.923568E-3,-8.5867845E-2,-1.279991E-1,2.4267668E-3,-6.4329065E-2,2.6747772E-2,-8.312018E-2,1.1962775E-2,1.10540584E-1,1.0128234E-3,-9.961084E-4,2.0131068E-3,-1.00269474E-1,2.572665E-3,3.149172E-2,-2.0708946E-4,-1.0878949E-3,4.239964E-2,-2.8172093E-3,5.1363925E-3,-8.4298635E-3,-8.509231E-2,-2.976341E-2,-8.546904E-3,4.215036E-3,5.800019E-4,-8.0046505E-2,-3.979253E-3,-1.3048567E-1,-6.8878283E-4,-8.622045E-3,-2.9376117E-3,-1.9072585E-2,-6.0705645E-3,5.859384E-2,-2.1853996E-2,-1.0946604E-1,-3.432465E-2,3.8547893E-3,-6.808612E-3,1.660023E-3,1.2005952E-1,-1.3385987E-1,-4.5805456E-4,1.8215774E-2,8.1040554E-2,9.856245E-2,-2.4943852E-3,-3.727115E-2,3.5138875E-2,5.621529E-2,2.2189217E-2,-1.19057875E-2,4.0961105E-2,-1.0290238E-2,1.4888993E-3,-6.562577E-2,-5.6460556E-3,-6.1926073E-3,-2.8881293E-2,-2.0436144E-3,-6.307247E-3,-8.747408E-3,-2.097483E-3,-4.4573345E-3,2.7419461E-3,2.1462241E-2,4.490307E-3,1.4307039E-3,-2.3721275E-3,-3.9280066E-4,-1.3464886E-1,-6.367364E-4,-5.498475E-3,-3.3165186E-3,1.49512915E-2,2.9655898E-3,6.8293423E-3,-9.159693E-3,-1.0433147E-3,2.4194593E-2,-4.6398705E-3,1.0547841E-1,-1.277537E-3,6.59471E-3,7.642845E-4,3.303098E-2,-6.7082117E-3,-1.0860929E-3,-7.05892E-2,4.7337175E-3,1.9275447E-2,3.503727E-2,8.377785E-2,3.1705786E-2,-3.6779302E-3,-8.95795E-3,-4.489668E-2,6.586874E-2,6.533116E-3,-3.2445304E-2,-4.7568865E-3,2.5966885E-2,-3.9113775E-2,-3.1652425E-3,1.256515E-3,-2.6754357E-4,2.3698206E-3,2.7266839E-3,-4.8490876E-4,-7.7902917E-3,-1.2139446E-3,2.1314446E-3,-9.566546E-4,-1.5224996E-3,1.8071425E-3,6.8324246E-4,4.5701955E-3,2.033703E-3,7.0547173E-3,-2.5480248E-3,2.433123E-3,5.0675175E-3,5.408684E-4,-2.2970617E-3,-1.7329729E-4,1.9957325E-3,-2.7785583E-3,-4.459213E-3,-1.1438946E-3,-5.3157896E-4,2.1284597E-3,3.1772314E-3,1.04933206E-4,-1.222391E-4,5.019328E-3,3.4739956E-4,2.3147522E-3,6.171076E-4,-7.0886937E-4,-4.367289E-3,-6.6365127E-4,4.5548514E-3,1.6772585E-3,1.1739277E-3,-1.8466923E-3,-3.488796E-3,2.0535698E-4,2.7760717E-3,-3.432992E-4,-4.3791765E-3,-1.207034E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,-1,-1,-1,51,53,55,57,59,61,-1,-1,63,65,67,-1,69,-1,71,73,-1,75,77,79,81,-1,83,85,87,89,91,-1,-1,-1,93,-1,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,109,111,113,-1,-1,-1,115,-1,117,119,121,123,-1,125,-1,127,129,-1,131,133,135,137,139,141,143,145,147,149,-1,-1,151,153,-1,155,157,-1,-1,-1,-1,-1,159,-1,-1,-1,-1,161,163,-1,-1,165,-1,-1,-1,-1,167,-1,169,171,-1,-1,173,175,177,179,-1,181,183,185,187,-1,189,191,193,195,197,-1,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.515799E-1,1.5099916E-1,2.2290018E-1,2.7928345E-2,1.3366577E-1,1.5187855E-1,2.2473724E-1,2.342175E-2,1.703702E-2,9.6231036E-2,2.2148064E-1,1.3977839E-1,1.4058019E-1,9.4634265E-2,5.929354E-2,0E0,6.973073E-3,0E0,0E0,9.640702E-2,0E0,0E0,1.0298017E-1,4.9275875E-2,4.895527E-2,1.2781703E-1,5.9177965E-2,2.2025141E-1,7.6541826E-2,6.120439E-2,0E0,0E0,0E0,1.0458666E-1,1.0784121E-1,7.9998E-2,1.476484E-1,3.1783104E-2,8.952389E-3,0E0,0E0,1.3141116E-1,1.372133E-1,6.5510154E-2,0E0,6.882216E-2,0E0,5.7829663E-2,1.1850023E-1,0E0,1.734437E-2,1.1517854E-1,5.6092635E-2,3.3026636E-2,0E0,3.390092E-2,4.9261533E-2,8.623186E-2,2.7444517E-2,2.0298839E-2,0E0,0E0,0E0,6.6096425E-2,0E0,1.08377665E-1,1.696822E-1,6.553899E-2,3.337592E-2,0E0,0E0,1.1740125E-1,1.449811E-1,3.855021E-2,0E0,0E0,0E0,4.3577194E-2,9.108179E-2,3.6052987E-2,0E0,0E0,0E0,5.157143E-2,0E0,2.013383E-2,1.8174723E-2,1.1408281E-1,6.6250436E-2,0E0,2.4323096E-2,0E0,2.1145344E-2,8.557302E-2,0E0,9.057169E-2,7.054351E-2,4.5287907E-2,1.100997E-1,3.0063722E-2,2.1809861E-2,3.9676264E-2,4.9739987E-2,6.154719E-2,3.8158067E-2,0E0,0E0,1.4575034E-2,3.0640025E-2,0E0,2.207642E-2,6.265872E-2,0E0,0E0,0E0,0E0,0E0,1.0848258E-2,0E0,0E0,0E0,0E0,8.067614E-2,1.6042078E-2,0E0,0E0,1.4593111E-2,0E0,0E0,0E0,0E0,8.863047E-2,0E0,5.6709796E-2,2.484348E-2,0E0,0E0,1.17355205E-1,8.341915E-2,3.130077E-2,9.52366E-3,0E0,1.5104071E-2,4.030146E-2,4.4819027E-2,1.7048862E-2,0E0,6.546676E-2,6.871993E-2,1.8178597E-2,1.5621619E-2,1.3950252E-2,0E0,1.4957402E-2,2.472077E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,45,45,47,47,48,48,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,63,63,65,65,66,66,67,67,68,68,71,71,72,72,73,73,77,77,78,78,79,79,83,83,85,85,86,86,87,87,88,88,90,90,92,92,93,93,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,107,107,108,108,110,110,111,111,117,117,122,122,123,123,126,126,131,131,133,133,134,134,137,137,138,138,139,139,140,140,142,142,143,143,144,144,145,145,147,147,148,148,149,149,150,150,151,151,153,153,154,154],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,-1,-1,-1,52,54,56,58,60,62,-1,-1,64,66,68,-1,70,-1,72,74,-1,76,78,80,82,-1,84,86,88,90,92,-1,-1,-1,94,-1,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,110,112,114,-1,-1,-1,116,-1,118,120,122,124,-1,126,-1,128,130,-1,132,134,136,138,140,142,144,146,148,150,-1,-1,152,154,-1,156,158,-1,-1,-1,-1,-1,160,-1,-1,-1,-1,162,164,-1,-1,166,-1,-1,-1,-1,168,-1,170,172,-1,-1,174,176,178,180,-1,182,184,186,188,-1,190,192,194,196,198,-1,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8122449E2,5E-1,1.4274633E7,1.315782E-5,2.8169732E10,3.02E2,4.5505118E2,1.1251919E6,1.2716E4,3.7852024E1,2.946089E10,1.5787431E0,1.8592099E9,4E0,8.9712226E-1,-1.7065309E-3,8.339375E2,6.2127435E-3,2.2923402E-3,6.97E3,-7.6971385E-3,-1.1269166E-2,1.4103535E0,1.0937254E5,1.172E3,1.727279E-4,4.92E2,2.183847E0,1.2427474E8,1.6907827E9,-1.289E-3,3.6037462E-3,6.137407E-4,1E0,1.6755487E0,1.4431512E-1,8.92686E5,1.1251919E6,6.1E2,-6.0160365E-3,1.1609487E-3,9.06727E5,1.325E0,2.1924436E5,6.2443307E-3,4.8046228E-2,-1.0428026E-2,2.4507338E4,7.6008E4,9.500915E-3,4.5843E7,8.333333E0,5.2E1,4.296196E0,2.4267668E-3,8.2892E4,2.1949357E-1,4.0208E4,1.903E3,2.0473914E2,1.0128234E-3,-9.961084E-4,2.0131068E-3,6.3157897E0,2.572665E-3,1.2542373E0,2.1761298E0,3.1930302E1,6.6E1,-2.8172093E-3,5.1363925E-3,6.7652373E3,1.26248375E5,2.0104182E-4,-8.546904E-3,4.215036E-3,5.800019E-4,1.2068493E4,1.773747E2,7.733016E5,-6.8878283E-4,-8.622045E-3,-2.9376117E-3,3.87E2,-6.0705645E-3,9.417789E1,3.3410483E8,4.01E2,9.379509E-2,3.8547893E-3,1.8991614E5,1.660023E-3,1.139605E6,1.4386049E6,-4.5805456E-4,1.9162654E7,1.0258E4,5.1E1,1.3358E4,3.3546498E-1,3.8874447E1,4.735985E6,6.4860556E5,1.0963991E7,4.5843E7,-1.0290238E-2,1.4888993E-3,1.5683041E3,1.2E1,-6.1926073E-3,9.2E0,9.750871E6,-6.307247E-3,-8.747408E-3,-2.097483E-3,-4.4573345E-3,2.7419461E-3,7.07541E1,4.490307E-3,1.4307039E-3,-2.3721275E-3,-3.9280066E-4,2.5544708E0,1.23E3,-5.498475E-3,-3.3165186E-3,5.505334E9,2.9655898E-3,6.8293423E-3,-9.159693E-3,-1.0433147E-3,4.5576923E1,-4.6398705E-3,4.5154482E-1,4.93E2,6.59471E-3,7.642845E-4,4.6E0,1.3333334E0,6.5E1,2.0579956E9,4.7337175E-3,2.4848E4,3.9314518E1,1.1772152E0,1.8367347E0,-3.6779302E-3,5.107429E1,1.1736916E0,4E1,2.289485E1,1.289627E3,-4.7568865E-3,5.7417827E0,2.6120481E0,-3.1652425E-3,1.256515E-3,-2.6754357E-4,2.3698206E-3,2.7266839E-3,-4.8490876E-4,-7.7902917E-3,-1.2139446E-3,2.1314446E-3,-9.566546E-4,-1.5224996E-3,1.8071425E-3,6.8324246E-4,4.5701955E-3,2.033703E-3,7.0547173E-3,-2.5480248E-3,2.433123E-3,5.0675175E-3,5.408684E-4,-2.2970617E-3,-1.7329729E-4,1.9957325E-3,-2.7785583E-3,-4.459213E-3,-1.1438946E-3,-5.3157896E-4,2.1284597E-3,3.1772314E-3,1.04933206E-4,-1.222391E-4,5.019328E-3,3.4739956E-4,2.3147522E-3,6.171076E-4,-7.0886937E-4,-4.367289E-3,-6.6365127E-4,4.5548514E-3,1.6772585E-3,1.1739277E-3,-1.8466923E-3,-3.488796E-3,2.0535698E-4,2.7760717E-3,-3.432992E-4,-4.3791765E-3,-1.207034E-4],"split_indices":[52,53,45,38,31,29,56,28,9,53,31,38,5,3,27,0,33,0,0,10,0,0,56,33,2,41,3,54,45,12,0,0,0,6,39,58,1,28,2,0,0,9,53,28,0,38,0,4,10,0,45,52,10,58,0,12,42,9,0,52,0,0,0,56,0,53,58,56,8,0,0,52,28,39,0,0,0,28,52,28,0,0,0,11,0,52,7,0,57,0,28,0,31,28,0,47,29,3,9,42,56,1,33,47,45,0,0,4,18,0,55,9,0,0,0,0,0,52,0,0,0,0,53,2,0,0,5,0,0,0,0,58,0,38,0,0,0,54,56,8,7,0,2,58,53,53,0,48,41,8,57,4,0,54,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.562E3,6.09E2,1.953E3,2.7E1,5.82E2,1.173E3,7.8E2,1.3E1,1.4E1,4.43E2,1.39E2,5.3E1,1.12E3,7.55E2,2.5E1,5E0,8E0,5E0,9E0,4.39E2,4E0,6E0,1.33E2,4.5E1,8E0,9.43E2,1.77E2,1.6E1,7.39E2,2.1E1,4E0,4E0,4E0,4.22E2,1.7E1,4.3E1,9E1,3.7E1,8E0,4E0,4E0,2.8E1,9.15E2,1.71E2,6E0,9E0,7E0,6.89E2,5E1,5E0,1.6E1,4.05E2,1.7E1,1.3E1,4E0,1.3E1,3E1,6.9E1,2.1E1,3.2E1,5E0,4E0,4E0,2.1E1,7E0,1.66E2,7.49E2,4.8E1,1.23E2,5E0,4E0,6.8E2,9E0,4.4E1,6E0,1.1E1,5E0,2E1,3.85E2,1E1,7E0,7E0,6E0,8E0,5E0,1.8E1,1.2E1,4.4E1,2.5E1,4E0,1.7E1,4E0,2.8E1,1.5E1,6E0,1.32E2,3.4E1,1.6E1,7.33E2,2.4E1,2.4E1,7.2E1,5.1E1,6.36E2,4.4E1,4E0,5E0,1.7E1,2.7E1,1E1,1E1,3.8E2,5E0,6E0,4E0,4E0,4E0,9E0,9E0,4E0,8E0,9E0,3.5E1,1.8E1,7E0,4E0,1.3E1,7E0,2.1E1,1E1,5E0,1.26E2,6E0,2.6E1,8E0,1.1E1,5E0,7.7E1,6.56E2,1.2E1,1.2E1,4E0,2E1,4.2E1,3E1,4.7E1,4E0,5.85E2,5.1E1,2.5E1,1.9E1,9E0,8E0,1.4E1,1.3E1,6E0,4E0,3.57E2,2.3E1,4E0,5E0,2.9E1,6E0,5E0,1.3E1,4E0,9E0,1.1E2,1.6E1,1E1,1.6E1,4E0,4E0,1.8E1,5.9E1,4.9E1,6.07E2,7E0,5E0,8E0,4E0,9E0,1.1E1,2.2E1,2E1,5E0,2.5E1,1.8E1,2.9E1,1.15E2,4.7E2,2.1E1,3E1,1.3E1,1.2E1,1.4E1,5E0,4E0,5E0,7E0,7E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"203","size_leaf_vector":"1"}},{"base_weights":[1.6208467E-4,-2.1783741E-2,5.325731E-3,6.001849E-3,-2.3193005E-2,9.14698E-3,-3.2300577E-2,-7.5305165E-3,-2.1023909E-2,1.4382324E-3,2.173986E-2,-2.798586E-2,-8.100175E-3,-1.5320087E-2,-5.2692935E-2,2.1535626E-2,-3.6409795E-3,2.6947878E-2,-7.7487235E-3,-4.8469648E-2,1.5229401E-3,-2.7300982E-2,8.560865E-4,-7.876935E-3,-4.265767E-2,7.7252224E-2,1.4039889E-2,-5.2828575E-3,5.9380963E-2,5.2802533E-2,1.769943E-2,-1.7597204E-2,6.0337197E-2,-5.9058838E-2,1.3556404E-2,3.6468601E-3,-4.3715867E-3,-1.9823465E-2,-7.9733565E-2,1.7663006E-2,-1.9791093E-2,-1.0595181E-2,-7.0863344E-2,6.3533904E-3,5.0934453E-2,2.12304E-2,-2.401749E-2,2.4604096E-4,-1.803203E-2,9.10325E-2,6.844896E-4,8.1647724E-2,2.9716698E-2,-7.606624E-2,2.1084249E-2,-2.6524294E-2,4.143916E-2,4.3337105E-4,4.0041027E-3,-4.278981E-2,-1.048121E-2,3.071742E-2,-1.150484E-3,-2.493498E-2,1.5224565E-2,-2.8003694E-2,2.26231E-2,-3.034955E-2,-1.8610364E-1,-1.0593191E-2,7.033164E-2,4.2466637E-2,-3.0486707E-2,1.926997E-2,-6.446502E-2,-9.444504E-2,1.0375622E-2,5.793604E-3,4.771614E-3,3.4385942E-2,-8.815373E-3,-3.8522612E-2,3.3885762E-3,-4.7660153E-3,1.4504401E-3,-1.2559743E-2,-1.0130891E-1,5.754175E-3,1.9821152E-3,9.633593E-2,-7.227605E-3,6.837282E-2,-3.3487834E-3,4.8417966E-3,-1.0578939E-2,2.3398755E-2,-6.234131E-3,-3.7501283E-2,2.2471735E-2,3.447674E-3,-8.223138E-4,-2.4711324E-2,-6.1571248E-2,-4.641155E-4,2.592343E-3,-1.557258E-2,-3.990705E-3,3.731013E-3,-6.073176E-4,3.0996315E-2,-3.5867307E-2,4.719247E-3,5.03779E-4,-7.229756E-2,3.1916054E-3,-4.744196E-3,-1.1993966E-2,-2.4563437E-2,6.730248E-2,-1.4545255E-3,8.8480525E-2,3.6218774E-3,1.8308701E-5,-8.139373E-3,-6.8598546E-2,-1.691E-2,7.340881E-2,-1.3003272E-3,-4.407324E-3,-1.3355549E-1,-6.529726E-2,3.2721707E-3,-2.3389873E-3,1.8506249E-3,-1.9540035E-3,4.033063E-2,-3.5344816E-3,-5.666886E-2,2.0148585E-2,1.6553467E-2,-7.1279995E-2,7.286631E-2,-1.5932304E-4,-3.296712E-2,8.735983E-3,4.087031E-3,-1.9544831E-1,1.514415E-1,8.256148E-2,-4.283365E-3,2.4920418E-3,7.9803444E-2,-1.607649E-3,-5.6744453E-2,1.1418745E-2,2.0244673E-3,-2.302012E-3,3.3675484E-2,5.1836185E-3,-2.9956434E-2,-5.5827484E-3,-9.386963E-4,2.4536278E-3,-1.3699994E-2,-4.7886483E-3,-5.188733E-2,-6.1386432E-3,1.4235852E-3,-2.4787657E-2,-1.4223884E-2,2.0145709E-3,1.9787822E-5,4.0309257E-3,-1.0246137E-3,-3.1551416E-3,-1.0700583E-3,2.3345281E-3,-5.674394E-3,4.2615266E-4,-5.27678E-4,-5.2719843E-3,5.5152006E-3,1.0752062E-4,6.9861775E-3,3.287355E-3,-1.4220974E-3,2.4195442E-3,-5.6649097E-3,-9.267066E-4,-1.8919113E-3,8.663489E-4,5.450645E-3,1.151744E-3,-8.0966465E-3,-3.3460052E-3,-7.930775E-4,-4.1515306E-3,2.3725007E-3,-2.575891E-3,1.310789E-3,-4.2079603E-3,1.8563262E-3,-3.4088786E-3,-1.0537438E-3,2.3211315E-3,-3.321291E-4,-5.4178187E-3,4.9810032E-3,2.013174E-3,-6.1607105E-3,4.1831565E-5,8.1926107E-4,-2.3975612E-3,1.284907E-3,-1.7077521E-3,2.7549765E-3,-2.3873888E-3,-1.7035276E-2,-5.4334453E-4,3.436044E-3,9.436725E-3,8.071879E-4,4.9149357E-3,5.214987E-3,1.7577182E-3,-5.664985E-4,-5.4446305E-3,2.706564E-3,-4.41538E-4,8.9417175E-5,2.0433876E-3,-3.0694525E-3,8.827112E-4,-2.1109679E-3,1.899338E-4,2.391863E-3,-1.1782623E-3,-1.5771586E-3,-3.7542772E-3,6.846683E-4,-1.7067743E-3,1.2117662E-3,-1.5736123E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,-1,7,9,11,-1,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,-1,77,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,101,-1,103,-1,105,107,109,111,113,115,117,119,121,123,125,127,129,131,133,-1,135,137,139,-1,-1,141,143,145,-1,-1,147,149,151,153,155,-1,157,-1,159,161,-1,-1,163,165,-1,-1,167,-1,-1,169,171,173,-1,175,177,-1,-1,-1,179,181,-1,183,-1,-1,185,187,189,191,-1,-1,193,195,-1,-1,-1,-1,197,-1,199,201,203,205,207,209,211,213,215,217,219,221,-1,-1,223,-1,225,227,-1,-1,229,231,233,-1,-1,-1,235,-1,237,-1,-1,239,241,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9270953E-1,1.0117382E-1,3.0093834E-1,0E0,1.318115E-1,1.8431477E-1,1.0356203E-1,0E0,8.60779E-2,1.20429456E-1,1.1093789E-1,1.1371079E-1,0E0,7.928964E-2,7.01596E-2,9.831298E-2,9.7786516E-2,1.456432E-1,7.415094E-2,7.370606E-2,3.3175796E-2,9.0554774E-2,6.1077278E-2,0E0,6.0744584E-2,3.0295E-2,5.8250476E-2,6.478371E-2,3.345471E-2,1.0478604E-1,1.453769E-1,5.1693343E-2,1.3624985E-2,2.2394788E-1,1.1379437E-2,0E0,2.979595E-2,7.2508596E-2,1.466883E-1,1.451019E-1,5.389954E-2,5.4333664E-2,6.994702E-2,0E0,3.96776E-2,7.0660904E-2,4.7902737E-2,7.4081734E-2,1.2541634E-1,1.2781836E-2,0E0,9.338111E-2,1.16543666E-1,1.766327E-1,1.5077533E-1,4.6020348E-2,2.1878907E-2,0E0,0E0,2.801846E-2,0E0,1.06421355E-2,0E0,1.7271012E-2,3.580843E-2,8.1839636E-2,5.3968772E-2,8.76209E-2,2.1458209E-2,7.129937E-2,6.393729E-2,1.5139235E-2,5.735711E-2,4.4333663E-2,8.655779E-3,2.3198709E-2,3.143023E-2,1.6879499E-2,0E0,8.034316E-2,7.737325E-2,5.5304386E-2,0E0,0E0,7.301289E-2,1.1415057E-1,1.6947128E-1,0E0,0E0,3.3798754E-2,5.3580754E-2,5.0838947E-2,4.0074054E-2,2.1845818E-2,0E0,8.0692396E-2,0E0,3.5295762E-2,1.850415E-2,0E0,0E0,3.4713626E-2,1.8550918E-2,0E0,0E0,1.3699707E-2,0E0,0E0,1.7848738E-2,3.189014E-2,6.323443E-2,0E0,2.8353216E-2,5.099926E-2,0E0,0E0,0E0,6.1227255E-2,2.7826186E-2,0E0,2.4580434E-2,0E0,0E0,5.1953457E-2,5.3924628E-2,1.0703255E-2,1.3655242E-2,0E0,0E0,1.0014504E-2,1.45070255E-2,0E0,0E0,0E0,0E0,7.998063E-2,0E0,5.0166167E-2,5.5124074E-2,1.448581E-2,4.4898808E-2,8.215316E-3,7.642737E-2,1.0011336E-1,9.4874285E-2,2.6464183E-2,2.3796678E-1,2.1042287E-2,5.1931173E-2,0E0,0E0,3.798665E-2,0E0,2.4847649E-2,3.511429E-2,0E0,0E0,6.3462466E-2,1.3163021E-1,2.6715357E-2,0E0,0E0,0E0,2.6071694E-2,0E0,1.5582569E-2,0E0,0E0,1.0206306E-2,1.671044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,59,59,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,81,81,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,95,95,97,97,98,98,101,101,102,102,105,105,108,108,109,109,110,110,112,112,113,113,117,117,118,118,120,120,123,123,124,124,125,125,126,126,129,129,130,130,135,135,137,137,138,138,139,139,140,140,141,141,142,142,143,143,144,144,145,145,146,146,147,147,148,148,151,151,153,153,154,154,157,157,158,158,159,159,163,163,165,165,168,168,169,169],"right_children":[2,4,6,-1,8,10,12,-1,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,-1,78,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,102,-1,104,-1,106,108,110,112,114,116,118,120,122,124,126,128,130,132,134,-1,136,138,140,-1,-1,142,144,146,-1,-1,148,150,152,154,156,-1,158,-1,160,162,-1,-1,164,166,-1,-1,168,-1,-1,170,172,174,-1,176,178,-1,-1,-1,180,182,-1,184,-1,-1,186,188,190,192,-1,-1,194,196,-1,-1,-1,-1,198,-1,200,202,204,206,208,210,212,214,216,218,220,222,-1,-1,224,-1,226,228,-1,-1,230,232,234,-1,-1,-1,236,-1,238,-1,-1,240,242,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5561539E2,8.974133E9,6.747114E7,6.001849E-3,3.983889E3,8.027061E2,2.6572757E10,-7.5305165E-3,1.27027025E1,2.294901E0,2.1722221E1,1.3715873E3,-8.100175E-3,1.8903887E0,3.4078704E7,2.1975676E2,6.057671E6,5.386213E6,4.4854636E7,3.138649E6,1.92E2,1.6956521E0,4.7272725E0,-7.876935E-3,2.090909E0,5.555794E0,1E0,1.2471935E7,2.1949357E-1,2.2779044E-3,3.171E3,1.64354E9,4.01E2,2.55298E5,1.6122807E1,3.6468601E-3,2.3405604E7,5.339161E0,3.7991562E0,1E0,1E0,8.536472E4,2.8340038E5,6.3533904E-3,1.1255588E1,7.5891815E6,1.1192772E8,7E0,2.1446484E-1,7.358598E6,6.844896E-4,3.0121632E6,1.1394967E7,2.821E3,3.7826266E4,3.94E2,3.011152E6,4.3337105E-4,4.0041027E-3,3.264E3,-1.048121E-2,6.019605E2,-1.150484E-3,1.4101E4,3.666495E7,1.08504E5,5.818182E0,3E0,4.2857143E-1,3.124379E2,9.162573E1,4.22088E0,1.55E2,1.2352941E0,1.6667E4,2.4013288E1,6.480713E6,2.46E2,4.771614E-3,1.5088776E0,2.728889E2,1.5834932E6,3.3885762E-3,-4.7660153E-3,1.0203835E-5,1E0,2.3983052E0,5.754175E-3,1.9821152E-3,3.7333333E0,5.3850017E0,5.608469E1,1.2060912E3,7.85939E5,-1.0578939E-2,4.68418E5,-6.234131E-3,2.47E2,2.175722E6,3.447674E-3,-8.223138E-4,1.1347826E1,3.8E1,-4.641155E-4,2.592343E-3,1E0,-3.990705E-3,3.731013E-3,1.5005797E4,1.0449754E0,4.785384E5,4.719247E-3,2.294901E0,1.4713E4,3.1916054E-3,-4.744196E-3,-1.1993966E-2,1.8523488E2,4.1602106E5,-1.4545255E-3,1.4925373E-1,3.6218774E-3,1.8308701E-5,1.1880085E2,6.98E2,5E0,1.4E1,-1.3003272E-3,-4.407324E-3,4.577553E3,3.1547058E1,3.2721707E-3,-2.3389873E-3,1.8506249E-3,-1.9540035E-3,2.4E1,-3.5344816E-3,4.459525E5,1.7851065E1,1.5437563E8,9.03E2,1E0,1.7348832E-5,1E0,1.1040155E-1,1E0,4.878616E2,2.808311E0,4.888E0,-4.283365E-3,2.4920418E-3,5.057096E-1,-1.607649E-3,1.4260634E8,2.5129596E11,2.0244673E-3,-2.302012E-3,1.2659678E3,4E0,1.06E2,-5.5827484E-3,-9.386963E-4,2.4536278E-3,1.0804272E3,-4.7886483E-3,9.302862E7,-6.1386432E-3,1.4235852E-3,2.9414747E1,3.046556E5,2.0145709E-3,1.9787822E-5,4.0309257E-3,-1.0246137E-3,-3.1551416E-3,-1.0700583E-3,2.3345281E-3,-5.674394E-3,4.2615266E-4,-5.27678E-4,-5.2719843E-3,5.5152006E-3,1.0752062E-4,6.9861775E-3,3.287355E-3,-1.4220974E-3,2.4195442E-3,-5.6649097E-3,-9.267066E-4,-1.8919113E-3,8.663489E-4,5.450645E-3,1.151744E-3,-8.0966465E-3,-3.3460052E-3,-7.930775E-4,-4.1515306E-3,2.3725007E-3,-2.575891E-3,1.310789E-3,-4.2079603E-3,1.8563262E-3,-3.4088786E-3,-1.0537438E-3,2.3211315E-3,-3.321291E-4,-5.4178187E-3,4.9810032E-3,2.013174E-3,-6.1607105E-3,4.1831565E-5,8.1926107E-4,-2.3975612E-3,1.284907E-3,-1.7077521E-3,2.7549765E-3,-2.3873888E-3,-1.7035276E-2,-5.4334453E-4,3.436044E-3,9.436725E-3,8.071879E-4,4.9149357E-3,5.214987E-3,1.7577182E-3,-5.664985E-4,-5.4446305E-3,2.706564E-3,-4.41538E-4,8.9417175E-5,2.0433876E-3,-3.0694525E-3,8.827112E-4,-2.1109679E-3,1.899338E-4,2.391863E-3,-1.1782623E-3,-1.5771586E-3,-3.7542772E-3,6.846683E-4,-1.7067743E-3,1.2117662E-3,-1.5736123E-3],"split_indices":[52,19,45,0,28,52,12,0,58,56,53,52,0,53,7,4,28,51,9,29,0,53,54,0,56,58,105,45,42,57,2,5,0,11,56,0,48,58,58,102,75,33,47,0,58,45,5,10,38,45,0,28,45,2,52,8,9,0,0,10,0,52,0,2,51,7,58,8,56,4,4,53,10,56,9,58,12,10,0,38,52,45,0,0,41,16,54,0,0,54,35,58,52,1,0,9,0,8,1,0,0,53,3,0,0,8,0,0,4,53,28,0,56,9,0,0,0,4,32,0,56,0,0,52,10,8,3,0,0,33,58,0,0,0,0,8,0,28,57,31,29,68,41,8,42,8,52,58,54,0,0,39,0,7,31,0,0,52,8,8,0,0,0,33,0,45,0,0,56,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.581E3,4.91E2,2.09E3,4E0,4.87E2,1.898E3,1.92E2,7E0,4.8E2,1.178E3,7.2E2,1.87E2,5E0,4.08E2,7.2E1,2.37E2,9.41E2,6.12E2,1.08E2,1.1E2,7.7E1,2.34E2,1.74E2,5E0,6.7E1,2.7E1,2.1E2,9.18E2,2.3E1,1.6E2,4.52E2,9.5E1,1.3E1,9.4E1,1.6E1,5E0,7.2E1,2.06E2,2.8E1,9.6E1,7.8E1,3.2E1,3.5E1,8E0,1.9E1,1.77E2,3.3E1,6.41E2,2.77E2,1.3E1,1E1,7E1,9E1,1.5E1,4.37E2,8.3E1,1.2E1,4E0,9E0,8.6E1,8E0,1.1E1,5E0,3.5E1,3.7E1,1.73E2,3.3E1,2E1,8E0,6.3E1,3.3E1,1.1E1,6.7E1,2.1E1,1.1E1,2.7E1,8E0,1E1,9E0,1.23E2,5.4E1,2.9E1,4E0,7E0,6.34E2,2.61E2,1.6E1,8E0,5E0,6E1,1E1,4.1E1,4.9E1,1E1,5E0,4.31E2,6E0,6.8E1,1.5E1,8E0,4E0,4.5E1,4.1E1,4E0,7E0,3.1E1,4E0,7E0,3E1,2E1,1.53E2,7E0,2.6E1,1.4E1,6E0,4E0,4E0,5.4E1,9E0,5E0,2.8E1,6E0,5E0,4.3E1,2.4E1,1.3E1,8E0,5E0,6E0,1E1,1.7E1,4E0,4E0,6E0,4E0,1.17E2,6E0,2E1,3.4E1,1.1E1,1.8E1,1.3E1,6.21E2,1.33E2,1.28E2,8E0,8E0,1E1,5E1,4E0,6E0,3.7E1,4E0,1E1,3.9E1,6E0,4E0,2.75E2,1.56E2,6.3E1,5E0,6E0,9E0,4E1,5E0,3.7E1,4E0,5E0,2.6E1,2.3E1,7E0,1.3E1,7E0,9.9E1,5.4E1,1.8E1,8E0,9E0,5E0,4.7E1,7E0,5E0,4E0,7E0,2.1E1,3.2E1,1.1E1,1.2E1,1.2E1,8E0,5E0,4E0,4E0,6E0,4E0,5E0,1.2E1,1.09E2,8E0,5E0,1.5E1,2.9E1,5E0,5E0,6E0,7E0,1.1E1,6E0,7E0,4E0,6.17E2,3.1E1,1.02E2,9.2E1,3.6E1,4E0,4E0,4E0,4E0,4E0,6E0,1E1,4E1,2.3E1,1.4E1,6E0,4E0,1.2E1,2.7E1,5.1E1,2.24E2,2.4E1,1.32E2,4.6E1,1.7E1,5E0,3.5E1,2.1E1,1.6E1,5E0,2.1E1,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"243","size_leaf_vector":"1"}},{"base_weights":[-1.7317076E-3,-9.350527E-3,9.174596E-3,2.1697655E-2,-1.2589226E-2,2.0780334E-2,-1.2821952E-2,-5.4096024E-2,3.235312E-2,-1.5101836E-2,2.6492778E-2,3.3239175E-2,7.208154E-3,-2.2585642E-2,8.331016E-3,1.4935683E-2,-1.1081496E-1,6.6926284E-3,2.1471225E-2,-9.15959E-3,-3.7639327E-2,4.255782E-2,-3.8380783E-2,4.534628E-2,4.188303E-3,-1.15232365E-2,2.9197972E-2,-1.6642451E-2,-5.6349345E-2,1.3181741E-2,-3.4519387E-3,-2.6804334E-3,4.023749E-3,-1.9117193E-3,-8.7800445E-3,-2.6984574E-2,3.454731E-2,-1.065363E-2,7.003481E-2,-3.2954264E-2,-1.0373957E-2,3.1881027E-2,6.4716195E-3,-4.8963716E-3,-5.884279E-3,1.8527221E-2,5.9951086E-2,-7.3471E-2,1.6668962E-2,-4.6862214E-4,-5.7307534E-2,4.9516715E-2,6.8371976E-3,-5.1041157E-3,-1.4108048E-2,5.2303565E-3,-7.935036E-2,5.557786E-3,5.183986E-2,-3.7562347E-3,-5.953363E-3,4.2725176E-2,-3.5965912E-2,-1.0773124E-3,-2.177886E-2,9.555351E-2,6.31477E-4,-1.0017325E-2,-2.8244749E-2,1.47389835E-2,7.917184E-2,2.7412513E-2,-3.1015857E-3,-2.0071331E-2,4.190305E-2,6.745476E-2,-1.9082459E-2,2.6045997E-2,-9.539914E-3,8.4569424E-2,2.5269573E-3,-2.6447758E-2,2.315486E-2,-6.8858206E-2,4.99521E-4,4.320353E-2,6.77417E-3,2.8874055E-2,-5.015459E-2,-1.7582988E-2,4.431078E-3,-1.680184E-3,2.9272132E-3,-1.0009256E-1,-8.7821664E-4,1.9111892E-2,-1.852469E-2,7.3081255E-2,8.071097E-4,2.4991546E-2,-3.9868876E-3,-1.2751678E-2,5.7430685E-2,-5.8397753E-3,1.871318E-3,1.03128E-2,-1.3158892E-2,-4.81877E-2,-8.07175E-3,5.3265445E-2,7.6229805E-3,-1.2099056E-2,-6.361403E-2,2.35654E-2,-3.1154575E-3,8.839784E-3,3.1848438E-2,-8.04162E-4,3.2696438E-3,-5.9888316E-3,4.706867E-3,5.8213983E-2,-8.884667E-4,8.055155E-2,3.9650157E-2,-3.9908573E-2,1.5265557E-3,2.7562054E-3,-4.136284E-4,1.0819929E-1,1.3567497E-3,1.1625353E-2,-3.924241E-3,-4.8553623E-2,-5.419456E-3,9.111708E-2,1.2819904E-2,2.773604E-4,-8.051926E-2,8.4984265E-2,3.2398455E-2,1.61494E-2,7.5122784E-3,-4.255023E-3,-9.802571E-3,-1.4803843E-2,-6.229666E-3,-5.0159577E-2,-7.245646E-3,7.080256E-3,3.9254837E-3,-1.6527103E-3,1.4908243E-2,5.689931E-3,1.6740338E-3,5.4424633E-3,2.8314248E-3,1.9097507E-3,-5.7351466E-2,9.0766855E-2,1.7940473E-2,-3.0714436E-2,1.9444713E-2,-6.411962E-2,1.9682145E-3,-8.345127E-2,-2.198284E-2,-3.981724E-2,1.5130019E-3,6.476566E-4,4.1430374E-3,-4.1904986E-2,1.2915165E-2,-4.2925548E-2,-1.1620525E-1,4.0414464E-2,-9.457212E-3,8.8110115E-5,2.6525233E-3,6.369299E-2,-5.459305E-2,4.1814018E-2,6.4375577E-3,9.117938E-2,1.3540079E-2,-1.9294086E-3,5.1200118E-2,-1.7231065E-4,-4.1457815E-3,2.283658E-3,6.864521E-3,-1.25664845E-2,3.686743E-2,1.1258653E-3,-7.2362326E-2,-3.0145615E-2,5.57647E-2,8.8845525E-5,7.5139897E-3,4.572246E-2,-8.636975E-3,-9.666968E-2,-1.6037853E-3,5.146072E-3,1.4177843E-3,-1.4336009E-3,4.1980084E-2,4.4974187E-4,4.4034913E-2,3.046345E-3,-2.9013315E-2,-1.7413985E-2,2.9618412E-3,-3.7440686E-3,-2.7217923E-4,2.7905994E-3,-7.72744E-3,1.993633E-3,-6.279551E-4,1.8388141E-3,-8.861016E-4,-4.662017E-3,-8.9108845E-4,2.8696628E-3,6.564879E-3,-8.8995024E-5,3.31512E-3,-3.7921532E-6,-3.7215257E-3,1.450448E-3,-1.8033937E-3,-5.7734475E-3,-6.137809E-4,4.010774E-4,-3.7226386E-3,-3.53383E-3,-1.134882E-2,1.430056E-4,-3.5458063E-3,-1.0993568E-3,-8.862266E-3,6.2839413E-4,-1.3883427E-3,2.5772986E-3,-3.024768E-3,2.4390745E-3,-7.11506E-4,-5.015276E-3,-1.3052629E-3,-9.364448E-3,-2.0609694E-3,4.7111258E-4,3.7695947E-3,-1.5567152E-3,9.888938E-4,-1.8371263E-4,4.4233142E-3,-3.111977E-4,-7.498863E-3,2.780397E-3,-3.0279648E-3,1.8311976E-3,5.4499744E-3,2.3606597E-3,-1.5853783E-3,-5.320678E-5,3.6551633E-3,-3.4110674E-3,1.0217687E-3,2.7037014E-3,-2.834131E-4,1.181153E-3,-1.7548309E-3,3.9166054E-5,-4.9079624E-3,1.6013628E-5,-3.1468964E-3,4.451517E-3,1.198936E-3,3.0296836E-3,-1.5997106E-3,1.1222052E-4,-4.477987E-3,-2.159854E-3,-5.6297034E-3,3.9167786E-3,1.4845843E-3,-2.0707853E-3,1.8818969E-3,7.9075544E-4,3.5219386E-3,1.4887839E-4,-3.339518E-3,-3.7218447E-4,-1.8009776E-3,1.5201542E-4,-2.9447286E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,61,63,65,67,-1,69,-1,71,-1,73,75,77,79,81,83,85,87,-1,89,91,93,95,97,99,-1,101,103,105,107,109,-1,-1,111,113,115,117,-1,119,121,123,125,127,-1,129,131,133,135,137,-1,139,-1,141,143,145,-1,-1,-1,147,-1,149,151,153,-1,155,-1,157,159,-1,-1,161,163,165,167,169,-1,171,173,175,-1,-1,177,-1,-1,-1,179,181,-1,183,185,187,-1,-1,-1,189,-1,191,-1,193,195,197,199,-1,201,203,205,207,-1,209,-1,211,-1,213,-1,215,-1,-1,217,-1,-1,219,-1,-1,221,223,225,227,229,231,233,235,237,239,241,-1,-1,243,245,247,249,251,253,-1,-1,255,257,259,-1,261,263,-1,265,-1,-1,-1,-1,267,269,271,273,275,277,-1,-1,279,281,283,-1,-1,-1,-1,285,287,289,-1,291,293,-1,-1,-1,-1,295,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1313663E-1,1.5211228E-1,2.6959333E-1,1.172394E-1,1.3473476E-1,1.1660412E-1,7.5571984E-2,7.2132386E-2,1.3641433E-1,1.7188063E-1,8.8135704E-2,1.2642998E-1,1.3708721E-1,4.9228236E-2,4.447134E-2,4.480528E-2,4.0367536E-2,0E0,7.379765E-2,1.21313795E-1,2.0658225E-1,5.8077864E-2,4.461114E-2,9.8377824E-2,1.0493329E-1,9.1076136E-2,6.91759E-2,4.493515E-2,5.2821666E-2,3.213194E-2,0E0,0E0,0E0,0E0,0E0,5.3071782E-2,5.4011583E-2,1.0662182E-1,2.6621558E-2,2.0104647E-1,0E0,4.8283424E-2,0E0,2.5716688E-2,0E0,8.2947895E-2,9.854615E-2,1.8030374E-1,8.908942E-2,9.021517E-2,2.7897857E-2,3.7606254E-2,9.4386235E-2,0E0,7.62109E-2,2.4774102E-2,3.283684E-2,3.0710772E-2,1.2464229E-2,4.789651E-2,0E0,6.76292E-2,6.377713E-2,7.43078E-2,1.6721188E-1,2.3213513E-2,0E0,0E0,1.4542212E-1,3.276891E-2,6.8373166E-2,1.594174E-2,0E0,8.710157E-2,5.6524932E-2,5.1428795E-2,1.6724497E-2,9.450702E-3,0E0,1.8769905E-2,5.8442198E-2,3.2276284E-2,5.2915648E-2,2.6477605E-2,0E0,3.1556368E-2,0E0,8.103906E-2,1.399199E-1,5.8572873E-2,0E0,0E0,0E0,3.7583202E-2,0E0,4.2360723E-2,1.7034162E-2,1.3944652E-2,0E0,9.598987E-3,0E0,4.311227E-2,8.394101E-2,0E0,0E0,1.0475192E-1,2.0209897E-1,1.4416963E-1,9.316768E-2,9.680508E-3,0E0,1.3250747E-1,8.2054436E-2,2.375101E-2,0E0,0E0,7.5735357E-3,0E0,0E0,0E0,1.0491548E-1,4.67325E-2,0E0,7.0696175E-2,4.7174387E-2,1.6984494E-2,0E0,0E0,0E0,1.3782859E-2,0E0,4.38196E-2,0E0,4.0296823E-2,5.765648E-2,5.267755E-2,4.840196E-2,0E0,1.7230183E-2,1.2367256E-2,3.7495404E-2,2.186985E-2,0E0,2.9220456E-2,0E0,3.951084E-2,0E0,1.1775468E-2,0E0,3.726153E-2,0E0,0E0,8.054542E-3,0E0,0E0,8.647104E-3,0E0,0E0,1.2543149E-2,3.984517E-2,2.9712163E-2,6.838116E-2,1.223986E-1,1.5852427E-1,9.465029E-2,1.0645601E-1,1.12159364E-1,1.7007947E-1,7.684547E-2,0E0,0E0,1.4420018E-1,9.52532E-2,5.4329112E-2,1.096988E-1,2.9678952E-2,1.013597E-2,0E0,0E0,2.5419667E-2,6.833666E-2,5.7198532E-2,0E0,8.088976E-2,2.4385521E-2,0E0,5.055979E-2,0E0,0E0,0E0,0E0,6.9438085E-2,2.597921E-2,1.0622223E-2,4.3586917E-2,2.7029179E-2,9.508412E-3,0E0,0E0,3.3239655E-2,3.7216853E-2,1.1376172E-2,0E0,0E0,0E0,0E0,2.1791257E-2,5.3007215E-2,1.2315374E-2,0E0,1.592942E-2,3.5561986E-2,0E0,0E0,0E0,0E0,2.2238566E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,35,35,36,36,37,37,38,38,39,39,41,41,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,68,68,69,69,70,70,71,71,73,73,74,74,75,75,76,76,77,77,79,79,80,80,81,81,82,82,83,83,85,85,87,87,88,88,89,89,93,93,95,95,96,96,97,97,99,99,101,101,102,102,105,105,106,106,107,107,108,108,109,109,111,111,112,112,113,113,116,116,120,120,121,121,123,123,124,124,125,125,129,129,131,131,133,133,134,134,135,135,136,136,138,138,139,139,140,140,141,141,143,143,145,145,147,147,149,149,152,152,155,155,158,158,159,159,160,160,161,161,162,162,163,163,164,164,165,165,166,166,167,167,168,168,171,171,172,172,173,173,174,174,175,175,176,176,179,179,180,180,181,181,183,183,184,184,186,186,191,191,192,192,193,193,194,194,195,195,196,196,199,199,200,200,201,201,206,206,207,207,208,208,210,210,211,211,216,216],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,62,64,66,68,-1,70,-1,72,-1,74,76,78,80,82,84,86,88,-1,90,92,94,96,98,100,-1,102,104,106,108,110,-1,-1,112,114,116,118,-1,120,122,124,126,128,-1,130,132,134,136,138,-1,140,-1,142,144,146,-1,-1,-1,148,-1,150,152,154,-1,156,-1,158,160,-1,-1,162,164,166,168,170,-1,172,174,176,-1,-1,178,-1,-1,-1,180,182,-1,184,186,188,-1,-1,-1,190,-1,192,-1,194,196,198,200,-1,202,204,206,208,-1,210,-1,212,-1,214,-1,216,-1,-1,218,-1,-1,220,-1,-1,222,224,226,228,230,232,234,236,238,240,242,-1,-1,244,246,248,250,252,254,-1,-1,256,258,260,-1,262,264,-1,266,-1,-1,-1,-1,268,270,272,274,276,278,-1,-1,280,282,284,-1,-1,-1,-1,286,288,290,-1,292,294,-1,-1,-1,-1,296,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.862126E2,1E0,2.8091298E7,2.35E2,1.289627E3,4.8718534E0,6.963859E1,1.00033E5,4.9069305E1,1.2471935E7,3.368326E11,2.185321E0,2.4408699E3,1.2427474E8,3.5551153E5,1.2909091E1,1.748435E4,6.6926284E-3,4.86E2,1.164249E7,2.2927427E-1,7.883E3,2.9138756E0,1E0,6.3157897E0,1.8662969E6,1.32E2,2.2690948E2,8.7757526E2,9.609747E3,-3.4519387E-3,-2.6804334E-3,4.023749E-3,-1.9117193E-3,-8.7800445E-3,1.0779661E1,4.502827E6,5.372E3,1.42E1,1.2673605E7,-1.0373957E-2,2.3616017E11,6.4716195E-3,1.7735592E3,-5.884279E-3,9.923929E2,3.51E2,2.632E5,1.418705E0,4.439941E10,9.9625344E5,1.19783E5,1.417988E6,-5.1041157E-3,2.6037296E-7,5.748E3,4.8083666E-1,2.9166096E-1,2.9885426E1,1.1906E4,-5.953363E-3,1.1347137E2,9.342733E7,3.38464E5,1.21E3,1.8441667E2,6.31477E-4,-1.0017325E-2,3.73524E5,7.418546E1,1.2283395E-2,1.216E3,-3.1015857E-3,1.4523809E0,5.0818306E-1,5.557511E8,3.8398104E0,4E3,-9.539914E-3,1.9007992E1,4.1822004E7,5.4091E4,1E0,6.920598E7,4.99521E-4,2.2241135E6,6.77417E-3,1.6956E5,1.08874E5,4.3400474E0,4.431078E-3,-1.680184E-3,2.9272132E-3,9.757E3,-8.7821664E-4,5.528107E3,1E0,3.5491096E7,8.071097E-4,2E0,-3.9868876E-3,4.144144E-1,1.5938586E8,-5.8397753E-3,1.871318E-3,3.79E2,1.196966E6,1.6E1,9.536863E4,9E2,7.6229805E-3,3.9E2,3.9509638E5,1.32E2,-3.1154575E-3,8.839784E-3,5.125467E-1,-8.04162E-4,3.2696438E-3,-5.9888316E-3,2.7258065E0,4.9807886E3,-8.884667E-4,3.4246575E-3,5.88E3,1.3E1,1.5265557E-3,2.7562054E-3,-4.136284E-4,6.47E2,1.3567497E-3,3.15856E0,-3.924241E-3,6.9664386E2,6.4625664E8,1.108E3,1.819795E6,2.773604E-4,4.0492815E-1,7.9023E4,1.0409588E-1,2.32E2,7.5122784E-3,1.902843E-2,-9.802571E-3,5E0,-6.229666E-3,8E0,-7.245646E-3,1.1766373E3,3.9254837E-3,-1.6527103E-3,7.2530017E0,5.689931E-3,1.6740338E-3,1.860326E7,2.8314248E-3,1.9097507E-3,6.806472E7,2.5464671E2,5.532928E-1,6.7E1,1E0,6.8E1,1E0,1.1698891E6,2.9722316E0,6.486406E6,7.7854166E0,6.476566E-4,4.1430374E-3,1.3414634E0,5.5303917E8,1.214E3,5.400855E5,2.7183437E1,1.4685706E3,8.8110115E-5,2.6525233E-3,7.940488E2,8.319452E2,2.545455E7,6.4375577E-3,1.892E3,1.2693982E6,-1.9294086E-3,1.4956522E1,-1.7231065E-4,-4.1457815E-3,2.283658E-3,6.864521E-3,1.2224206E7,3.7891315E6,2.7123368E10,2.1111E4,3.6401E4,2.652E3,8.8845525E-5,7.5139897E-3,5.72143E5,2.6491638E1,1.0677447E0,-1.6037853E-3,5.146072E-3,1.4177843E-3,-1.4336009E-3,2.1169072E3,2.93E2,2.2333652E1,3.046345E-3,1.7548548E7,7.218466E7,2.9618412E-3,-3.7440686E-3,-2.7217923E-4,2.7905994E-3,2.6364462E7,1.993633E-3,-6.279551E-4,1.8388141E-3,-8.861016E-4,-4.662017E-3,-8.9108845E-4,2.8696628E-3,6.564879E-3,-8.8995024E-5,3.31512E-3,-3.7921532E-6,-3.7215257E-3,1.450448E-3,-1.8033937E-3,-5.7734475E-3,-6.137809E-4,4.010774E-4,-3.7226386E-3,-3.53383E-3,-1.134882E-2,1.430056E-4,-3.5458063E-3,-1.0993568E-3,-8.862266E-3,6.2839413E-4,-1.3883427E-3,2.5772986E-3,-3.024768E-3,2.4390745E-3,-7.11506E-4,-5.015276E-3,-1.3052629E-3,-9.364448E-3,-2.0609694E-3,4.7111258E-4,3.7695947E-3,-1.5567152E-3,9.888938E-4,-1.8371263E-4,4.4233142E-3,-3.111977E-4,-7.498863E-3,2.780397E-3,-3.0279648E-3,1.8311976E-3,5.4499744E-3,2.3606597E-3,-1.5853783E-3,-5.320678E-5,3.6551633E-3,-3.4110674E-3,1.0217687E-3,2.7037014E-3,-2.834131E-4,1.181153E-3,-1.7548309E-3,3.9166054E-5,-4.9079624E-3,1.6013628E-5,-3.1468964E-3,4.451517E-3,1.198936E-3,3.0296836E-3,-1.5997106E-3,1.1222052E-4,-4.477987E-3,-2.159854E-3,-5.6297034E-3,3.9167786E-3,1.4845843E-3,-2.0707853E-3,1.8818969E-3,7.9075544E-4,3.5219386E-3,1.4887839E-4,-3.339518E-3,-3.7218447E-4,-1.8009776E-3,1.5201542E-4,-2.9447286E-3],"split_indices":[52,53,45,2,4,53,56,5,50,45,31,41,55,45,33,55,28,0,12,45,38,2,58,102,56,47,3,33,52,52,0,0,0,0,0,56,51,29,58,45,0,31,0,4,0,52,8,33,54,31,28,10,29,0,37,2,42,27,57,29,0,52,5,9,2,52,0,0,29,56,57,10,0,54,39,7,53,2,0,56,51,33,102,7,0,32,0,2,2,46,0,0,0,9,0,52,101,32,0,8,0,53,7,0,0,1,9,18,28,2,0,10,33,3,0,0,57,0,0,0,54,4,0,57,2,3,0,0,0,0,0,38,0,52,7,0,1,0,27,2,38,8,0,57,0,6,0,8,0,52,0,0,53,0,0,5,0,0,7,4,41,29,105,0,74,47,57,45,54,0,0,53,5,0,33,53,55,0,0,4,52,45,0,2,47,0,58,0,0,0,0,50,28,31,9,9,10,0,0,29,53,41,0,0,0,0,52,3,58,0,45,45,0,0,0,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.563E3,1.509E3,1.054E3,1.42E2,1.367E3,6.9E2,3.64E2,1.7E1,1.25E2,1.285E3,8.2E1,3.59E2,3.31E2,2.49E2,1.15E2,8E0,9E0,1.1E1,1.14E2,1.018E3,2.67E2,6.6E1,1.6E1,2.53E2,1.06E2,1.79E2,1.52E2,2.13E2,3.6E1,1.09E2,6E0,4E0,4E0,5E0,4E0,2.4E1,9E1,1E3,1.8E1,2.61E2,6E0,6E1,6E0,1.2E1,4E0,9E1,1.63E2,1.4E1,9.2E1,1.45E2,3.4E1,7.9E1,7.3E1,5E0,2.08E2,1E1,2.6E1,9.2E1,1.7E1,2E1,4E0,8.1E1,9E0,5.38E2,4.62E2,1.2E1,6E0,6E0,2.55E2,4.5E1,1.5E1,8E0,4E0,3.4E1,5.6E1,1.49E2,1.4E1,8E0,6E0,1.5E1,7.7E1,6.9E1,7.6E1,2.9E1,5E0,7.5E1,4E0,5.3E1,2E1,2.02E2,6E0,6E0,4E0,1.9E1,7E0,5.9E1,3.3E1,1E1,7E0,1.5E1,5E0,1.7E1,6.4E1,4E0,5E0,2.77E2,2.61E2,1.57E2,3.05E2,8E0,4E0,1.76E2,7.9E1,4.1E1,4E0,4E0,1.1E1,4E0,4E0,6E0,2.8E1,4.4E1,1.2E1,1E2,4.9E1,1E1,4E0,4E0,4E0,1E1,5E0,7E1,7E0,3.3E1,3.6E1,9E0,6.7E1,4E0,2.5E1,1.4E1,6.1E1,4.9E1,4E0,1.6E1,4E0,1.98E2,4E0,1E1,9E0,5E1,9E0,2.3E1,1E1,4E0,6E0,1E1,5E0,8E0,9E0,3.4E1,3E1,5E1,2.27E2,5.9E1,2.02E2,6.6E1,9.1E1,7E1,2.35E2,4E0,4E0,8E1,9.6E1,5.8E1,2.1E1,2.7E1,1.4E1,5E0,6E0,1.4E1,1.4E1,3.7E1,7E0,8.6E1,1.4E1,6E0,4.3E1,6E0,4E0,4E0,6E0,3.6E1,3.4E1,1.1E1,2.2E1,2.6E1,1E1,4E0,5E0,2.6E1,4.1E1,1.8E1,7E0,1E1,4E0,8E0,5.3E1,3.2E1,1.7E1,4E0,1.2E1,1.92E2,6E0,6E0,4E0,1.1E1,3.9E1,5E0,5E0,4E0,6E0,4E0,5E0,2E1,1.4E1,2.2E1,8E0,3E1,2E1,1.94E2,3.3E1,2.9E1,3E1,1.88E2,1.4E1,6.2E1,4E0,6.1E1,3E1,6.3E1,7E0,1.71E2,6.4E1,1.3E1,6.7E1,4.1E1,5.5E1,1.2E1,4.6E1,1E1,1.1E1,1.5E1,1.2E1,8E0,6E0,4E0,1E1,1E1,4E0,3.3E1,4E0,2.2E1,6.4E1,8E0,6E0,1.3E1,3E1,1.3E1,2.3E1,2.4E1,1E1,7E0,4E0,6E0,1.6E1,1.4E1,1.2E1,4E0,6E0,2.2E1,4E0,3.7E1,4E0,5E0,1.3E1,1.2E1,4.1E1,1.5E1,1.7E1,9E0,8E0,7E0,5E0,1.26E2,6.6E1,3.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"297","size_leaf_vector":"1"}},{"base_weights":[-8.638155E-4,-7.831099E-3,9.319454E-3,-5.965889E-3,-6.3589536E-2,4.0344786E-2,5.181602E-3,-7.110545E-3,8.1763685E-2,-3.5181146E-2,-1.8594E-1,7.407917E-2,1.9667821E-2,7.907858E-3,-3.769004E-2,-9.699754E-3,1.3408486E-2,8.51927E-3,1.579312E-2,-5.6595635E-2,2.3608077E-3,-5.2541015E-3,-1.1469342E-2,-3.7549585E-3,9.411067E-2,5.815328E-3,6.164831E-2,6.2942365E-3,6.9306105E-2,-6.4353814E-4,-6.0552776E-2,-4.8185126E-3,-2.9570887E-2,9.8009E-2,7.2384584E-3,-3.870286E-4,2.5131567E-3,6.909939E-4,-6.966537E-2,1.2414806E-1,1.1174629E-2,-1.8491372E-2,2.8366264E-2,8.058028E-2,3.597095E-4,7.08403E-3,-6.362457E-3,1.3845238E-3,4.7954344E-3,1.9176817E-2,-2.6345644E-2,-2.6706458E-4,-8.224495E-2,3.6006805E-3,-1.9005239E-2,-2.5540536E-2,-8.792605E-2,1.0942006E-3,7.883968E-3,4.686806E-2,5.7868793E-4,-8.099688E-2,-6.700963E-5,2.8184077E-3,1.5519893E-1,-1.085738E-3,3.0772423E-3,-3.939706E-2,1.9299487E-2,-7.5393883E-3,4.1182987E-2,4.9569085E-3,1.359902E-3,6.1631408E-3,5.7724644E-3,-2.515964E-4,2.4115653E-3,-2.8352228E-3,1.6867879E-4,-1.1562706E-3,1.3608361E-3,-1.2460959E-1,-2.5965791E-2,7.686675E-4,5.0576013E-2,-1.499613E-2,-6.7841634E-2,-6.2528E-3,-2.3312729E-2,1.2541985E-3,-1.2769337E-1,-8.0293516E-4,7.1473956E-2,3.89188E-2,-7.443142E-3,-1.2632349E-3,-1.0245133E-1,3.137829E-3,9.5093325E-3,-6.356376E-2,-2.7394432E-4,3.0859988E-3,-6.889074E-4,1.6787035E-3,-2.356796E-3,2.2060847E-2,3.4215841E-3,-5.2772867E-3,6.9666626E-3,-7.1705906E-3,-3.0925316E-3,5.5623555E-4,-4.0037944E-3,-5.7706836E-4,5.143994E-3,7.184294E-2,2.1330051E-2,-1.1533317E-2,-8.829293E-2,-8.820816E-2,4.3813884E-4,-3.30331E-2,-6.3972347E-3,-8.14689E-3,-2.277963E-3,1.0569931E-3,4.9742092E-3,6.105087E-2,-5.1016896E-4,-3.6818758E-2,6.1830375E-3,-2.7317768E-3,-6.173013E-3,-1.1763935E-3,-4.6464E-3,2.783709E-3,2.3782367E-4,1.932305E-2,-6.7206085E-5,2.947383E-4,-1.4964145E-3,4.326942E-3,3.6535613E-4,2.5581868E-4,2.9585627E-3,1.7156215E-4,-1.4530069E-3,-1.2122101E-2,-8.313959E-4,5.976028E-5,-5.351082E-3,-1.1605261E-3,-4.1544265E-3,5.8621686E-4,-1.7489744E-3,1.6875543E-3,4.561799E-3,3.4929486E-4,-2.6108776E-3,-2.806504E-4,1.5980978E-3,1.1367776E-3,-2.6144756E-3,-1.3026695E-3,4.9664755E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,-1,-1,-1,-1,39,41,43,45,47,49,51,53,55,57,59,-1,-1,-1,61,63,65,67,69,71,-1,73,-1,-1,-1,75,77,79,81,83,85,87,89,-1,-1,91,93,95,-1,-1,97,-1,-1,99,101,103,105,-1,-1,107,-1,-1,-1,-1,-1,-1,-1,109,111,113,115,117,119,-1,121,-1,123,-1,125,127,129,-1,131,-1,-1,133,-1,-1,-1,-1,-1,135,-1,-1,137,-1,-1,-1,-1,139,-1,141,143,145,147,149,-1,151,153,-1,-1,-1,-1,155,-1,157,159,-1,-1,-1,-1,-1,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8127859E-1,1.5739049E-1,1.3304757E-1,1.4809863E-1,1.6377158E-1,8.364102E-2,1.07476145E-1,7.72189E-2,1.0853185E-1,7.5046E-2,7.564485E-3,1.4457875E-1,4.4419646E-2,8.506422E-2,4.6545163E-2,1.2488511E-1,8.4374525E-2,0E0,1.0135671E-2,3.13377E-2,0E0,0E0,0E0,0E0,1.0075116E-1,3.2864723E-2,1.9005723E-2,8.976567E-2,2.1744281E-2,1.1712573E-2,4.4440806E-2,1.23809084E-1,5.7493E-2,4.658968E-2,4.035326E-2,0E0,0E0,0E0,2.156943E-2,5.43496E-2,2.124293E-2,2.3671616E-2,1.45764705E-2,1.12139955E-2,0E0,8.2973465E-2,0E0,0E0,0E0,9.398261E-3,9.225881E-3,6.920971E-3,5.6370556E-2,8.6566746E-2,7.455772E-2,5.0113723E-2,7.5121514E-2,0E0,0E0,3.5218466E-2,4.0887315E-2,2.6098087E-2,0E0,0E0,5.2940756E-2,0E0,0E0,1.5223945E-2,1.6305659E-2,1.6255835E-2,1.049364E-2,0E0,0E0,7.5463615E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.7112913E-3,2.500426E-2,8.4626265E-2,2.1480538E-2,8.984679E-2,4.602173E-2,0E0,3.8428515E-2,0E0,2.7398273E-2,0E0,2.049914E-2,2.5631841E-2,4.4258166E-2,0E0,1.0377049E-2,0E0,0E0,1.0764759E-2,0E0,0E0,0E0,0E0,0E0,8.467738E-3,0E0,0E0,7.1878515E-2,0E0,0E0,0E0,0E0,1.1581884E-1,0E0,1.9360185E-2,1.01358015E-2,9.0175554E-2,1.7287493E-1,3.8960963E-2,0E0,7.129376E-2,4.5529027E-2,0E0,0E0,0E0,0E0,1.011702E-2,0E0,2.403006E-2,2.3316888E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.486555E-2,1.376511E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,39,39,40,40,41,41,42,42,43,43,45,45,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,64,64,67,67,68,68,69,69,70,70,73,73,81,81,82,82,83,83,84,84,85,85,86,86,88,88,90,90,92,92,93,93,94,94,96,96,99,99,105,105,108,108,113,113,115,115,116,116,117,117,118,118,119,119,121,121,122,122,127,127,129,129,130,130,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,-1,-1,-1,-1,40,42,44,46,48,50,52,54,56,58,60,-1,-1,-1,62,64,66,68,70,72,-1,74,-1,-1,-1,76,78,80,82,84,86,88,90,-1,-1,92,94,96,-1,-1,98,-1,-1,100,102,104,106,-1,-1,108,-1,-1,-1,-1,-1,-1,-1,110,112,114,116,118,120,-1,122,-1,124,-1,126,128,130,-1,132,-1,-1,134,-1,-1,-1,-1,-1,136,-1,-1,138,-1,-1,-1,-1,140,-1,142,144,146,148,150,-1,152,154,-1,-1,-1,-1,156,-1,158,160,-1,-1,-1,-1,-1,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4887473E-1,1.2427474E8,1.3543E4,1.8552577E2,3.6600095E10,2.5369E4,1E0,1.46705E3,1.4622E4,6.08767E3,1.2E1,2.73E2,2.2021123E8,1.386749E1,2.8524646E-1,5.2217197E-2,1.3E1,8.51927E-3,5.2646267E9,8E0,2.3608077E-3,-5.2541015E-3,-1.1469342E-2,-3.7549585E-3,2.2515285E6,1.4100353E0,2.0550244E0,3.7826266E4,5.686354E-1,9E0,8.4413904E-1,1.4540612E7,1.7851065E1,1.6E1,7.970349E2,-3.870286E-4,2.5131567E-3,6.909939E-4,1.6667E4,1.0838E4,2.2603108E3,3.5799E4,3.0990322E5,6.504065E-2,3.597095E-4,1.9013363E4,-6.362457E-3,1.3845238E-3,4.7954344E-3,6.5928856E5,1.4977E5,1.544E3,4.838871E8,2.495333E6,5.485425E6,1.00011E5,8.91E2,1.0942006E-3,7.883968E-3,4.6647886E10,9.517604E6,9.61E2,-6.700963E-5,2.8184077E-3,1.1902924E0,-1.085738E-3,3.0772423E-3,5.2610065E1,1.1408248E0,4.3359457E10,3.3996282E0,4.9569085E-3,1.359902E-3,3.3820656E7,5.7724644E-3,-2.515964E-4,2.4115653E-3,-2.8352228E-3,1.6867879E-4,-1.1562706E-3,1.3608361E-3,1.2593641E6,2.2241135E6,7.883E3,7.555942E6,4.387561E1,3.2053032E1,-6.2528E-3,4.273439E2,1.2541985E-3,1.5E1,-8.0293516E-4,8.417E3,1.54E2,3.401282E1,-1.2632349E-3,2.771816E11,3.137829E-3,9.5093325E-3,9.51668E5,-2.7394432E-4,3.0859988E-3,-6.889074E-4,1.6787035E-3,-2.356796E-3,1E0,3.4215841E-3,-5.2772867E-3,3.64299E5,-7.1705906E-3,-3.0925316E-3,5.5623555E-4,-4.0037944E-3,7.325843E0,5.143994E-3,1.0061821E5,2.2153166E7,1.3382226E-1,1.43E2,1.6122807E1,4.3813884E-4,2.285677E6,6.7699623E-1,-8.14689E-3,-2.277963E-3,1.0569931E-3,4.9742092E-3,1.06158945E5,-5.1016896E-4,1E0,2.7608706E7,-2.7317768E-3,-6.173013E-3,-1.1763935E-3,-4.6464E-3,2.783709E-3,2.3782367E-4,1.0417911E1,1.325204E6,2.947383E-4,-1.4964145E-3,4.326942E-3,3.6535613E-4,2.5581868E-4,2.9585627E-3,1.7156215E-4,-1.4530069E-3,-1.2122101E-2,-8.313959E-4,5.976028E-5,-5.351082E-3,-1.1605261E-3,-4.1544265E-3,5.8621686E-4,-1.7489744E-3,1.6875543E-3,4.561799E-3,3.4929486E-4,-2.6108776E-3,-2.806504E-4,1.5980978E-3,1.1367776E-3,-2.6144756E-3,-1.3026695E-3,4.9664755E-4],"split_indices":[38,45,9,56,5,29,104,52,2,4,18,2,32,35,38,38,3,0,12,3,0,0,0,0,28,42,39,52,38,3,41,45,57,18,33,0,0,0,9,9,4,10,28,57,0,52,0,0,0,32,29,2,7,1,9,5,0,0,0,31,45,12,0,0,53,0,0,56,42,19,57,0,0,7,0,0,0,0,0,0,0,32,32,2,1,53,53,0,52,0,8,0,12,3,58,0,31,0,0,1,0,0,0,0,0,64,0,0,9,0,0,0,0,53,0,33,1,27,0,56,0,32,42,0,0,0,0,33,0,64,50,0,0,0,0,0,0,53,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.553E3,1.516E3,1.037E3,1.468E3,4.8E1,1.21E2,9.16E2,1.45E3,1.8E1,4E1,8E0,4.5E1,7.6E1,8.62E2,5.4E1,1.288E3,1.62E2,7E0,1.1E1,3.2E1,8E0,4E0,4E0,5E0,4E1,5.8E1,1.8E1,8.41E2,2.1E1,2.1E1,3.3E1,1.035E3,2.53E2,1E1,1.52E2,7E0,4E0,5E0,2.7E1,2.9E1,1.1E1,2.8E1,3E1,1.3E1,5E0,8.37E2,4E0,9E0,1.2E1,1.2E1,9E0,9E0,2.4E1,6.5E2,3.85E2,2.38E2,1.5E1,5E0,5E0,2.1E1,1.31E2,2.3E1,4E0,1E1,1.9E1,7E0,4E0,1.8E1,1E1,8E0,2.2E1,9E0,4E0,8.31E2,6E0,7E0,5E0,4E0,5E0,5E0,4E0,1.3E1,1.1E1,6.14E2,3.6E1,3.57E2,2.8E1,4E0,2.34E2,4E0,1.1E1,6E0,1.5E1,2.2E1,1.09E2,7E0,1.6E1,6E0,1.3E1,1E1,8E0,4E0,6E0,4E0,4E0,1.4E1,8E0,5E0,8.26E2,9E0,4E0,7E0,4E0,6.07E2,7E0,2E1,1.6E1,3.42E2,1.5E1,2.2E1,6E0,1.48E2,8.6E1,7E0,4E0,6E0,9E0,1.5E1,7E0,3.4E1,7.5E1,6E0,1E1,5E0,5E0,4E0,1E1,2.99E2,5.27E2,4.98E2,1.09E2,1.6E1,4E0,1.2E1,4E0,1.85E2,1.57E2,4E0,1.1E1,4E0,1.8E1,1.25E2,2.3E1,5.3E1,3.3E1,9E0,6E0,9E0,2.5E1,5.2E1,2.3E1,2.86E2,1.3E1,1.46E2,3.81E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-1.868574E-4,5.1846057E-3,-1.5682431E-2,3.4569E-3,6.789013E-2,-4.376924E-2,-8.973241E-3,-8.332875E-4,2.2224221E-2,1.5426062E-1,4.30617E-2,-9.973898E-2,-3.538277E-2,-5.6854202E-3,-7.860254E-2,-9.832983E-5,-1.2339696E-1,-1.1474951E-2,3.069059E-2,9.768656E-3,3.281238E-3,-5.7390127E-3,6.3453E-2,-4.912603E-2,-1.0056403E-2,-5.188995E-2,2.4155965E-3,-2.362811E-3,-3.7685532E-2,-1.4291245E-1,-2.3832668E-3,-1.0209039E-1,5.8487104E-4,-1.3723548E-2,2.625225E-3,5.6443654E-2,-3.7884697E-2,2.619476E-2,1.2741594E-1,-2.2040727E-3,1.1870705E-3,4.29471E-3,1.4565728E-3,2.0609704E-4,-3.8108064E-3,-3.9302323E-2,-8.691694E-3,-4.282046E-3,1.6549554E-2,2.213806E-2,-6.196622E-3,-2.1541623E-2,-8.605001E-3,-3.8904794E-3,-9.371115E-3,2.5489908E-3,-4.3641864E-3,-8.0252E-3,-1.8141939E-3,9.213205E-2,-1.567963E-4,3.474297E-4,7.247691E-2,1.1031087E-2,-8.177396E-2,1.3012693E-2,4.85253E-2,1.3160171E-3,9.78607E-3,-1.29686585E-2,-6.835263E-2,4.795118E-2,-3.716562E-2,-2.7640513E-3,3.1261805E-2,-1.7520497E-2,9.519255E-3,-5.9313864E-2,9.537833E-3,6.8564108E-3,2.6200402E-3,3.773118E-4,-4.927382E-3,5.3342474E-3,9.018888E-4,2.9469136E-2,-3.4284596E-3,-2.0666516E-1,-1.8311074E-2,2.8692923E-2,-4.3279283E-2,7.6254033E-3,4.0078375E-2,3.61206E-2,-4.515447E-2,-9.632367E-2,-1.1110313E-3,3.1627745E-3,7.959894E-5,7.486496E-4,-3.2534692E-3,4.6646312E-2,-3.0317275E-2,-8.980606E-4,-3.3304326E-2,4.528842E-2,1.7525452E-4,-8.954834E-2,-2.0124553E-2,2.4414514E-3,-1.1841683E-2,5.017564E-3,-1.10986475E-2,-1.9431364E-3,4.784089E-2,-1.648209E-2,-2.1053734E-3,3.0383167E-3,-5.9504498E-2,1.3335319E-2,5.133284E-2,-1.6744198E-1,2.6102257E-3,4.8847716E-2,-3.671499E-2,-2.9096182E-3,8.4301524E-2,2.1330148E-3,-8.383632E-2,-1.2647136E-1,-1.0102945E-3,1.5050116E-3,-2.2291483E-3,5.627791E-2,-3.067215E-4,-2.6254205E-3,5.6502526E-4,-5.1940205E-3,4.7294376E-3,-2.3986144E-2,-8.8350594E-2,1.8407283E-2,6.965353E-2,6.191996E-3,-3.688443E-2,-6.5048155E-3,-1.69831E-3,1.2599309E-3,-2.7258706E-3,-2.5285159E-3,2.0352382E-2,7.2936054E-5,2.2963705E-3,-4.4364035E-3,-3.918586E-4,7.016321E-4,3.2478385E-3,-9.861656E-4,-5.9553296E-3,1.3927537E-3,-1.739211E-3,3.9780806E-3,3.432994E-5,-3.2813053E-3,-1.2795469E-2,1.0947586E-3,-4.524465E-3,4.243535E-3,1.590534E-3,-3.076344E-4,-3.29989E-3,9.533334E-4,5.9383227E-3,-2.0991494E-3,-7.670595E-3,-7.2946483E-3,-2.5197128E-3,-3.1866645E-4,3.3908917E-3,-5.410726E-4,2.1083052E-3,-4.665669E-3,-9.386236E-4,-5.7565323E-3,-4.2604097E-5,2.0316583E-3,-1.6426746E-3,4.74047E-3,1.693521E-3,1.9933556E-3,6.115236E-5,9.816249E-4,-2.661665E-3,3.0975507E-3,-8.86391E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,51,53,55,57,59,-1,-1,61,63,65,67,-1,-1,-1,-1,-1,-1,69,-1,-1,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,79,81,-1,83,85,87,89,91,-1,-1,93,95,97,99,-1,101,103,105,107,109,-1,-1,111,-1,-1,-1,113,-1,115,117,119,121,-1,123,125,127,129,131,-1,-1,-1,-1,133,135,137,139,141,143,145,147,-1,149,151,153,-1,155,-1,-1,-1,157,159,161,163,165,167,169,-1,171,-1,173,175,-1,-1,-1,177,-1,-1,-1,179,-1,181,183,185,187,189,191,-1,-1,-1,-1,-1,193,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1340981E-1,2.0603834E-1,1.2397745E-1,1.4929393E-1,1.027239E-1,5.6083336E-2,1.21801004E-1,1.3600732E-1,9.8783985E-2,2.7353257E-2,4.116366E-2,7.232705E-2,7.000713E-2,5.4242454E-2,1.1729908E-1,1.04720674E-1,2.5393498E-1,1.2769827E-1,1.166839E-1,0E0,0E0,1.576169E-2,1.9875064E-2,1.7657882E-2,0E0,1.13005996E-1,4.5007072E-2,4.372149E-2,1.0053797E-1,2.3143351E-2,5.8894217E-2,3.266477E-2,1.013855E-1,0E0,0E0,1.5365906E-2,1.103928E-1,7.737467E-2,7.785022E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.419363E-2,0E0,0E0,5.4071195E-2,4.6250723E-2,7.170856E-2,5.221912E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4282525E-2,7.797034E-2,0E0,2.6058838E-2,3.8734954E-2,2.1021652E-1,1.4953056E-1,8.004461E-2,0E0,0E0,6.325014E-2,6.384078E-2,1.4127873E-2,1.8464454E-2,0E0,5.5413913E-2,6.1363652E-2,5.647645E-2,2.1895245E-2,2.1596681E-2,0E0,0E0,7.860142E-2,0E0,0E0,0E0,2.8233372E-2,0E0,1.6852644E-1,6.554193E-2,4.521118E-2,2.1125437E-1,0E0,6.374875E-2,7.764427E-2,8.5129365E-2,5.2507713E-2,1.628852E-2,0E0,0E0,0E0,0E0,2.3731977E-2,1.1696218E-2,4.769666E-2,5.943255E-2,2.164153E-2,3.0330492E-2,2.040933E-2,1.6985804E-2,0E0,2.2244526E-2,1.5302254E-1,1.0681735E-1,0E0,8.946631E-3,0E0,0E0,0E0,2.8502122E-2,5.6963436E-2,7.58602E-2,7.339212E-2,5.2326366E-2,4.8252285E-2,8.555146E-3,0E0,2.251599E-2,0E0,4.3093838E-2,1.7870903E-2,0E0,0E0,0E0,2.8741911E-2,0E0,0E0,0E0,3.0095715E-2,0E0,3.5780065E-2,3.98504E-2,2.1878202E-2,1.3017751E-2,1.9541023E-2,1.864448E-2,0E0,0E0,0E0,0E0,0E0,1.6961228E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,45,45,48,48,49,49,50,50,51,51,59,59,60,60,62,62,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,81,81,85,85,87,87,88,88,89,89,90,90,92,92,93,93,94,94,95,95,96,96,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,110,110,111,111,112,112,114,114,118,118,119,119,120,120,121,121,122,122,123,123,124,124,126,126,128,128,129,129,133,133,137,137,139,139,140,140,141,141,142,142,143,143,144,144,150,150],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,52,54,56,58,60,-1,-1,62,64,66,68,-1,-1,-1,-1,-1,-1,70,-1,-1,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,80,82,-1,84,86,88,90,92,-1,-1,94,96,98,100,-1,102,104,106,108,110,-1,-1,112,-1,-1,-1,114,-1,116,118,120,122,-1,124,126,128,130,132,-1,-1,-1,-1,134,136,138,140,142,144,146,148,-1,150,152,154,-1,156,-1,-1,-1,158,160,162,164,166,168,170,-1,172,-1,174,176,-1,-1,-1,178,-1,-1,-1,180,-1,182,184,186,188,190,192,-1,-1,-1,-1,-1,194,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,2.03148E2,1.263E3,1.8137958E6,4.8083666E-1,1.2E1,3.9509638E5,1.7730331E6,2.4757753E2,1.942E3,1E1,1.5139E4,1E0,1.2427474E8,1.0001285E-1,6E0,2E0,8.339375E2,3.8E1,9.768656E-3,3.281238E-3,3.4922136E11,6.4506575E5,6.999242E7,-1.0056403E-2,4.9363803E5,4.0677965E-2,2.2444444E0,5.39978E5,1E0,6.129333E-2,1.2068493E4,8.974133E9,-1.3723548E-2,2.625225E-3,6E2,6.23E2,2.831692E10,1.7812634E1,-2.2040727E-3,1.1870705E-3,4.29471E-3,1.4565728E-3,2.0609704E-4,-3.8108064E-3,1.5316E4,-8.691694E-3,-4.282046E-3,1.2125011E6,2.7482478E-2,6.726E3,8.32E3,-8.605001E-3,-3.8904794E-3,-9.371115E-3,2.5489908E-3,-4.3641864E-3,-8.0252E-3,-1.8141939E-3,7.1308413E0,1E0,3.474297E-4,1.04E3,6.763314E7,8.01E2,1.4952381E1,2.85898E10,1.3160171E-3,9.78607E-3,1E0,3.6411794E5,4.7022E4,8.988928E5,-2.7640513E-3,1.1953147E2,1.4400709E9,2E0,1E1,3.2343243E1,6.8564108E-3,2.6200402E-3,1.1394967E7,-4.927382E-3,5.3342474E-3,9.018888E-4,3.9E1,-3.4284596E-3,1.5597535E3,1.3333334E0,5.815045E2,1.8722344E8,7.6254033E-3,8.65832E-8,1.5222145E2,4.5808794E5,3.8977896E2,9E0,3.1627745E-3,7.959894E-5,7.486496E-4,-3.2534692E-3,3.3018634E5,1.022332E6,7.524668E10,7.1E1,5.530237E9,1E0,1.5265896E8,4.6263345E-2,2.4414514E-3,2.0718E4,3.771981E6,3.7246967E2,-1.9431364E-3,4.36E2,-1.648209E-2,-2.1053734E-3,3.0383167E-3,5E0,4.1724915E2,1.4753189E8,4.47682E5,9.750871E6,1.4523809E0,3.173913E0,-2.9096182E-3,5.4347825E-1,2.1330148E-3,1E0,6E2,-1.0102945E-3,1.5050116E-3,-2.2291483E-3,1.4123257E-4,-3.067215E-4,-2.6254205E-3,5.6502526E-4,1.6594656E3,4.7294376E-3,1.503E3,2.3033286E5,3.5301748E7,9.11898E5,2.65851E5,1.3E1,-6.5048155E-3,-1.69831E-3,1.2599309E-3,-2.7258706E-3,-2.5285159E-3,6.9998717E-4,7.2936054E-5,2.2963705E-3,-4.4364035E-3,-3.918586E-4,7.016321E-4,3.2478385E-3,-9.861656E-4,-5.9553296E-3,1.3927537E-3,-1.739211E-3,3.9780806E-3,3.432994E-5,-3.2813053E-3,-1.2795469E-2,1.0947586E-3,-4.524465E-3,4.243535E-3,1.590534E-3,-3.076344E-4,-3.29989E-3,9.533334E-4,5.9383227E-3,-2.0991494E-3,-7.670595E-3,-7.2946483E-3,-2.5197128E-3,-3.1866645E-4,3.3908917E-3,-5.410726E-4,2.1083052E-3,-4.665669E-3,-9.386236E-4,-5.7565323E-3,-4.2604097E-5,2.0316583E-3,-1.6426746E-3,4.74047E-3,1.693521E-3,1.9933556E-3,6.115236E-5,9.816249E-4,-2.661665E-3,3.0975507E-3,-8.86391E-4],"split_indices":[27,56,2,28,42,18,33,28,55,0,3,9,16,45,39,52,17,33,3,0,0,31,28,7,0,33,57,56,11,100,38,28,19,0,0,2,2,31,56,0,0,0,0,0,0,9,0,0,48,27,2,2,0,0,0,0,0,0,0,54,65,0,2,7,2,56,31,0,0,75,33,9,32,0,58,5,8,8,56,0,0,45,0,0,0,10,0,33,56,4,7,0,37,4,28,4,3,0,0,0,0,33,28,31,8,5,84,45,57,0,2,1,4,0,2,0,0,0,8,4,12,9,9,54,46,0,56,0,104,0,0,0,0,42,0,0,0,4,0,2,33,45,29,29,8,0,0,0,0,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.562E3,1.903E3,6.59E2,1.853E3,5E1,1.26E2,5.33E2,1.509E3,3.44E2,1E1,4E1,1.5E1,1.11E2,5.1E2,2.3E1,1.501E3,8E0,6.9E1,2.75E2,6E0,4E0,1.2E1,2.8E1,1.1E1,4E0,7.7E1,3.4E1,4.63E2,4.7E1,1.2E1,1.1E1,9E0,1.492E3,4E0,4E0,1.9E1,5E1,2.64E2,1.1E1,5E0,7E0,1.6E1,1.2E1,4E0,7E0,7.1E1,6E0,4E0,3E1,6.2E1,4.01E2,4.3E1,4E0,6E0,6E0,7E0,4E0,4E0,5E0,1.1E1,1.481E3,5E0,1.4E1,2.4E1,2.6E1,1.67E2,9.7E1,5E0,6E0,3.8E1,3.3E1,1.9E1,1.1E1,6E0,5.6E1,2.33E2,1.68E2,1.9E1,2.4E1,4E0,7E0,1.474E3,7E0,8E0,6E0,2E1,4E0,8E0,1.8E1,1.31E2,3.6E1,6E0,9.1E1,1.5E1,2.3E1,2.3E1,1E1,1.4E1,5E0,4E0,7E0,4.5E1,1.1E1,1.14E2,1.19E2,3.4E1,1.34E2,1E1,9E0,8E0,1.6E1,1.05E3,4.24E2,4E0,1.6E1,4E0,4E0,6E0,1.2E1,7.9E1,5.2E1,9E0,2.7E1,8.2E1,9E0,5E0,1E1,7E0,1.6E1,1.6E1,7E0,6E0,4E0,3.8E1,7E0,7E0,4E0,1.1E2,4E0,1.03E2,1.6E1,1.7E1,1.7E1,1.16E2,1.8E1,5E0,5E0,4E0,5E0,7E0,9E0,9.67E2,8.3E1,1.6E1,4.08E2,6E0,1E1,8E0,4E0,6.1E1,1.8E1,3.3E1,1.9E1,5E0,4E0,2.3E1,4E0,2.5E1,5.7E1,5E0,4E0,4E0,6E0,1.1E1,5E0,1.2E1,4E0,6E0,3.2E1,9.9E1,1.1E1,6E0,9.7E1,1.2E1,4E0,1.2E1,5E0,9E0,8E0,1.4E1,1.02E2,4E0,1.4E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"195","size_leaf_vector":"1"}},{"base_weights":[3.889371E-4,-2.008915E-3,3.244792E-2,6.2861154E-3,-1.1749007E-2,5.7043523E-2,1.3984677E-2,-1.1643112E-3,2.2379702E-2,-3.793577E-2,-5.9263585E-3,8.9741334E-2,1.6176341E-2,2.0851586E-2,-6.0053203E-2,1.66624E-2,-8.9561455E-3,2.8140534E-2,-9.522943E-3,-2.5516024E-2,-1.478167E-1,-3.1228175E-3,-6.1533365E-2,6.052667E-2,1.9013658E-1,-4.0098317E-3,3.1207182E-2,9.5848165E-2,8.853909E-3,1.7431491E-3,-7.1443343E-3,-1.8458167E-2,3.1705506E-2,-7.78716E-2,-6.500529E-3,5.768308E-3,2.6254814E-2,-1.9685347E-2,5.3638738E-2,-3.134936E-2,5.8763146E-2,-2.7339486E-3,-1.9373927E-1,1.2051965E-3,-2.4148619E-2,-4.2742122E-2,-9.783393E-3,3.9864786E-2,6.721892E-3,1.4243284E-2,2.8575992E-3,1.740494E-2,4.9715755E-3,2.523957E-3,7.908832E-3,3.9770205E-2,-8.485772E-3,-3.1779336E-3,-1.0605468E-1,4.708651E-2,-5.3905523E-3,-1.5321062E-2,-8.874135E-3,1.6215177E-2,-1.3296046E-2,2.3151418E-2,6.991959E-2,1.9371103E-2,-3.62052E-2,4.406599E-3,4.1766913E-4,-2.460684E-2,-1.1036678E-1,4.946381E-3,-8.673326E-4,-1.5535235E-2,-5.352266E-3,2.8639615E-2,-3.3917096E-3,1.0763345E-2,-3.44394E-2,-5.2838653E-2,1.8228844E-3,4.4973847E-3,2.209819E-2,4.2395983E-2,-2.931943E-2,9.186647E-5,4.728299E-2,-4.1474435E-3,2.558287E-3,-1.3924793E-2,7.1209095E-2,-1.0479748E-2,-1.3989555E-3,1.2200255E-1,4.0003885E-2,2.5851525E-2,-2.9629972E-2,1.7515825E-3,-1.9475507E-3,2.2551365E-2,-6.650956E-2,1.9760886E-2,-1.9529449E-2,2.1600984E-2,5.790556E-3,7.3413257E-3,3.1859986E-2,-8.530916E-3,4.399172E-3,-4.252733E-2,2.8253152E-4,-6.193687E-2,-9.52314E-3,-3.9993034E-4,-9.841582E-3,1.73528E-2,6.459873E-2,-7.3094286E-2,1.6367568E-3,7.8670256E-2,-1.6829953E-2,-5.8860283E-2,-1.1886614E-2,-3.2100685E-2,-8.9911364E-2,3.8250326E-3,2.6781175E-3,-6.688186E-4,2.962503E-3,-2.9786362E-3,4.0539875E-5,3.2758915E-3,2.2017768E-2,3.9662246E-2,-8.249656E-3,-7.213555E-2,4.5561674E-3,6.9967234E-3,-5.9217756E-4,3.782868E-3,8.115319E-3,4.71759E-2,-1.4779682E-2,-7.0448295E-4,4.708811E-3,-9.265439E-2,1.8368548E-2,3.3357643E-2,-3.409764E-2,-5.49915E-4,-5.0805057E-3,-4.139644E-3,3.1898934E-2,-3.5590652E-2,-5.5464306E-3,5.389844E-2,1.695895E-2,3.2907657E-3,7.514307E-4,1.375986E-3,-2.7582145E-3,-5.6791995E-2,-5.817506E-4,-3.3153575E-2,-1.733863E-1,1.2605024E-3,-1.0963151E-1,5.683696E-2,7.974105E-3,7.813887E-2,-1.2399716E-4,-2.1528026E-2,-1.4146268E-2,4.719387E-2,-1.9902044E-3,6.4986744E-3,1.1359222E-3,3.7825964E-3,-3.806004E-2,-4.496573E-2,-7.2815735E-3,1.6499853E-2,-3.7240393E-2,-7.375562E-2,-2.1906814E-3,-5.968658E-3,-1.2323188E-3,1.2257248E-3,-8.673671E-4,2.5184331E-3,-2.7267795E-4,3.4544624E-3,1.7881981E-4,-1.756639E-2,1.1853469E-3,-1.4593705E-3,-7.602504E-3,7.186283E-4,-3.3289175E-3,1.7551433E-3,4.9487175E-3,-3.6067117E-3,1.0562899E-3,2.1216339E-3,-1.470157E-3,-4.9183157E-4,-6.873761E-3,2.6109335E-3,-2.357664E-3,9.201131E-4,3.986223E-3,7.5019394E-5,-2.5984864E-3,3.2976686E-3,8.246045E-5,-1.0563724E-3,-4.8262826E-3,-1.0009444E-4,-7.409623E-3,-1.1776903E-3,4.0684994E-3,1.1370084E-3,-4.221043E-4,-1.4768878E-3,-4.4590826E-3,3.9689546E-4,-4.6947883E-3,-1.531258E-2,-2.817849E-4,4.102507E-4,-5.619552E-3,-8.577556E-3,-1.4652989E-3,4.213705E-3,7.704175E-5,-1.1392321E-3,7.680864E-4,1.5377224E-3,5.060385E-3,-2.414568E-3,3.577512E-3,1.5475996E-3,6.9633126E-3,-5.772851E-4,5.318265E-4,-3.3456169E-3,3.9517984E-4,-1.7892073E-3,-5.8241356E-3,1.7960896E-3,-2.021671E-4,-7.8833476E-4,-3.3675546E-3,-9.142478E-4,-5.3803315E-3,-1.8002954E-3,1.2177946E-3,1.3272956E-3,-1.2680192E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,-1,-1,57,59,61,63,-1,65,67,69,71,73,-1,75,77,79,81,-1,83,-1,-1,-1,85,-1,-1,-1,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,-1,-1,113,115,-1,-1,-1,-1,117,119,121,123,125,-1,-1,127,129,131,-1,133,-1,135,137,139,-1,-1,141,143,145,147,-1,-1,149,151,153,155,157,-1,-1,159,161,-1,163,-1,165,167,-1,-1,169,171,173,175,177,179,181,183,185,187,189,-1,-1,-1,-1,-1,-1,191,193,195,197,199,-1,-1,-1,-1,201,203,205,-1,207,209,211,213,-1,-1,-1,215,217,219,221,223,-1,-1,-1,-1,225,-1,227,229,231,233,235,237,239,-1,241,-1,243,245,-1,-1,-1,247,249,-1,251,253,255,257,-1,-1,-1,-1,-1,-1,-1,-1,259,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9662866E-1,1.9236441E-1,8.003466E-2,1.5422271E-1,1.6673674E-1,1.0010463E-1,5.361857E-2,1.2238368E-1,7.506879E-2,2.678023E-1,1.3964704E-1,1.1167824E-1,5.3232364E-2,8.463092E-2,7.584703E-2,1.421756E-1,1.0323176E-1,5.3765774E-2,4.1493986E-2,9.033537E-2,7.4908376E-2,7.783407E-2,1.00120455E-1,4.6622977E-2,9.727013E-2,0E0,2.7487732E-2,2.8800786E-2,4.4832014E-2,0E0,0E0,1.08081065E-1,1.0752183E-1,1.2827978E-1,9.175402E-2,0E0,4.4318825E-2,3.618771E-2,1.3343396E-2,8.732347E-2,3.8467128E-2,0E0,9.864348E-2,8.963769E-2,5.2824304E-2,3.312332E-2,0E0,2.2955325E-2,0E0,0E0,0E0,3.2770675E-2,0E0,0E0,0E0,8.462243E-3,4.465359E-2,5.695146E-2,9.124328E-2,6.5689534E-2,4.3174207E-2,1.8050047E-2,0E0,7.330975E-2,9.455405E-2,4.321955E-2,5.9796035E-2,3.33241E-2,1.05615035E-2,0E0,0E0,8.781294E-2,1.1365846E-1,0E0,0E0,0E0,0E0,4.027714E-2,2.1357235E-1,6.503239E-2,6.1536103E-2,2.4022229E-2,0E0,0E0,1.245824E-2,1.7755851E-2,9.17589E-3,0E0,1.0101773E-2,0E0,1.9576041E-2,6.702769E-2,5.3104624E-2,0E0,0E0,8.283481E-3,4.9025E-2,4.546888E-2,9.850445E-2,0E0,0E0,7.949451E-2,1.8400777E-2,9.3562275E-2,8.6267486E-2,4.6162665E-2,0E0,0E0,8.160174E-3,2.3403635E-2,0E0,1.43177435E-2,0E0,1.3860632E-1,1.2212543E-1,0E0,0E0,2.8801257E-2,2.1992564E-2,4.3793252E-1,9.411201E-2,2.5789082E-2,5.1992975E-2,5.8625028E-2,4.379671E-2,2.9740013E-2,2.0184383E-2,7.0052934E-3,0E0,0E0,0E0,0E0,0E0,0E0,9.633413E-3,1.1450982E-2,1.1736104E-2,4.7080807E-2,3.4529537E-2,0E0,0E0,0E0,0E0,6.464028E-2,3.245412E-2,2.4802435E-2,0E0,5.0715543E-2,4.4669706E-2,7.34461E-2,1.3464803E-2,0E0,0E0,0E0,6.7965515E-2,1.5572618E-1,1.0439827E-1,8.508156E-2,4.0837906E-2,0E0,0E0,0E0,0E0,1.7688222E-2,0E0,9.320499E-2,1.9916981E-1,8.2334355E-2,4.9764574E-2,2.2673685E-2,1.5148489E-2,1.8799819E-2,0E0,8.8302195E-2,0E0,5.847513E-2,6.391584E-2,0E0,0E0,0E0,2.8425124E-2,2.729708E-2,0E0,1.1731907E-2,1.9648224E-2,1.684285E-2,1.4361112E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1014979E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,47,47,51,51,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,71,71,72,72,77,77,78,78,79,79,80,80,81,81,84,84,85,85,86,86,88,88,90,90,91,91,92,92,95,95,96,96,97,97,98,98,101,101,102,102,103,103,104,104,105,105,108,108,109,109,111,111,113,113,114,114,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,126,126,127,127,134,134,135,135,136,136,137,137,138,138,143,143,144,144,145,145,147,147,148,148,149,149,150,150,154,154,155,155,156,156,157,157,158,158,163,163,165,165,166,166,167,167,168,168,169,169,170,170,171,171,173,173,175,175,176,176,180,180,181,181,183,183,184,184,185,185,186,186,195,195],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,-1,-1,58,60,62,64,-1,66,68,70,72,74,-1,76,78,80,82,-1,84,-1,-1,-1,86,-1,-1,-1,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,-1,-1,114,116,-1,-1,-1,-1,118,120,122,124,126,-1,-1,128,130,132,-1,134,-1,136,138,140,-1,-1,142,144,146,148,-1,-1,150,152,154,156,158,-1,-1,160,162,-1,164,-1,166,168,-1,-1,170,172,174,176,178,180,182,184,186,188,190,-1,-1,-1,-1,-1,-1,192,194,196,198,200,-1,-1,-1,-1,202,204,206,-1,208,210,212,214,-1,-1,-1,216,218,220,222,224,-1,-1,-1,-1,226,-1,228,230,232,234,236,238,240,-1,242,-1,244,246,-1,-1,-1,248,250,-1,252,254,256,258,-1,-1,-1,-1,-1,-1,-1,-1,260,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4409692E3,7.358598E6,3.2475834E7,2.465E3,2.3871907E2,1.417988E6,1.3205668E7,1.402418E0,1.32E2,4.7121394E-2,1.2427474E8,1.62E2,3.4058E4,9.042859E4,8.848604E1,9.417789E1,4.974E3,6.3552374E2,3.266054E3,3.0324321E-2,2.6006784E2,7.1277297E-1,3.6600095E10,7.6696295E-1,2.3761498E9,-4.0098317E-3,8.61491E3,1.921032E4,2.8091298E7,1.7431491E-3,-7.1443343E-3,1.2062E5,3.9473684E0,2.645E3,3.182857E0,5.768308E-3,2.0718E4,7.969174E7,6.963859E1,1.7653874E6,2.808311E0,-2.7339486E-3,3.932E3,7.2744107E0,2.41E3,4.551684E6,-9.783393E-3,1.945711E1,6.721892E-3,1.4243284E-2,2.8575992E-3,4.77721E3,4.9715755E-3,2.523957E-3,7.908832E-3,3.6022064E1,3.33231E5,1.3E1,2.14306E5,6.977611E5,3.5044186E2,1E0,-8.874135E-3,5.8E1,1.5316E4,3.3480989E3,1.00663277E9,1.2019585E3,7.7938065E1,4.406599E-3,4.1766913E-4,1.9826213E0,1.7640048E0,4.946381E-3,-8.673326E-4,-1.5535235E-2,-5.352266E-3,1.9283049E-1,5.4468E4,3.7936268E-3,5.121E3,9.112E3,1.8228844E-3,4.4973847E-3,2.8E1,3.2E1,9.391714E4,9.186647E-5,6.447073E7,-4.1474435E-3,1.91E4,9.6E1,2E1,-1.0479748E-2,-1.3989555E-3,7.08E2,1.2852783E0,4.624343E6,1.8114872E7,1.7515825E-3,-1.9475507E-3,8.4503644E5,2.6877522E5,3.4643276E1,2.80307E2,4.752366E0,5.790556E-3,7.3413257E-3,1.8601036E1,3.333E3,4.399172E-3,5.695621E8,2.8253152E-4,4.1E0,1.5012488E1,-3.9993034E-4,-9.841582E-3,1.8481675E0,2.8800161E9,4.5454E4,6.5749304E2,9.087248E-1,9.253112E-1,3.5E1,2.777588E6,1.28E2,1E0,5.4E2,2.6781175E-3,-6.688186E-4,2.962503E-3,-2.9786362E-3,4.0539875E-5,3.2758915E-3,2.0783027E5,8E0,4.930349E5,1.08504E5,1E0,6.9967234E-3,-5.9217756E-4,3.782868E-3,8.115319E-3,1.4171931E6,1.6789798E0,2.2178302E5,4.708811E-3,3.830216E5,3.8244262E6,3.2188842E0,1.85221E5,-5.49915E-4,-5.0805057E-3,-4.139644E-3,5.024605E0,2.7E1,2.6104508E10,1.0223611E8,7.019902E1,3.2907657E-3,7.514307E-4,1.375986E-3,-2.7582145E-3,1.088E3,-5.817506E-4,1.4540612E7,6.387417E-4,2.6608378E8,2.2783158E5,4.1100917E0,2.785857E2,3.9473684E0,-1.2399716E-4,3.5E1,-1.4146268E-2,2.8454319E1,2.692487E6,6.4986744E-3,1.1359222E-3,3.7825964E-3,1E0,9.633465E-1,-7.2815735E-3,2.3140822E8,3.6E1,5.9E1,4.3359457E10,-5.968658E-3,-1.2323188E-3,1.2257248E-3,-8.673671E-4,2.5184331E-3,-2.7267795E-4,3.4544624E-3,1.7881981E-4,3.653602E7,1.1853469E-3,-1.4593705E-3,-7.602504E-3,7.186283E-4,-3.3289175E-3,1.7551433E-3,4.9487175E-3,-3.6067117E-3,1.0562899E-3,2.1216339E-3,-1.470157E-3,-4.9183157E-4,-6.873761E-3,2.6109335E-3,-2.357664E-3,9.201131E-4,3.986223E-3,7.5019394E-5,-2.5984864E-3,3.2976686E-3,8.246045E-5,-1.0563724E-3,-4.8262826E-3,-1.0009444E-4,-7.409623E-3,-1.1776903E-3,4.0684994E-3,1.1370084E-3,-4.221043E-4,-1.4768878E-3,-4.4590826E-3,3.9689546E-4,-4.6947883E-3,-1.531258E-2,-2.817849E-4,4.102507E-4,-5.619552E-3,-8.577556E-3,-1.4652989E-3,4.213705E-3,7.704175E-5,-1.1392321E-3,7.680864E-4,1.5377224E-3,5.060385E-3,-2.414568E-3,3.577512E-3,1.5475996E-3,6.9633126E-3,-5.772851E-4,5.318265E-4,-3.3456169E-3,3.9517984E-4,-1.7892073E-3,-5.8241356E-3,1.7960896E-3,-2.021671E-4,-7.8833476E-4,-3.3675546E-3,-9.142478E-4,-5.3803315E-3,-1.8002954E-3,1.2177946E-3,1.3272956E-3,-1.2680192E-3],"split_indices":[52,45,50,2,52,29,9,53,3,38,45,8,11,28,56,52,9,47,4,38,4,27,5,27,12,0,4,4,45,0,0,1,56,9,58,0,2,7,56,51,58,0,11,56,2,29,0,56,0,0,0,52,0,0,0,56,29,3,1,45,4,75,0,3,9,4,7,4,56,0,0,56,54,0,0,0,0,38,1,57,2,10,0,0,8,3,28,0,44,0,10,29,0,0,0,2,42,45,12,0,0,32,28,52,55,56,0,0,58,2,0,5,0,58,54,0,0,54,5,1,55,27,56,8,1,10,67,0,0,0,0,0,0,0,33,8,28,7,68,0,0,0,0,47,41,28,0,28,50,53,9,0,0,0,53,8,20,7,58,0,0,0,0,10,0,45,38,7,28,56,52,56,0,3,0,56,1,0,0,0,100,27,0,7,3,10,19,0,0,0,0,0,0,0,0,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.556E3,2.379E3,1.77E2,1.285E3,1.094E3,7.5E1,1.02E2,8.79E2,4.06E2,1.98E2,8.96E2,4.1E1,3.4E1,9.4E1,8E0,2.67E2,6.12E2,3.44E2,6.2E1,1.79E2,1.9E1,8.54E2,4.2E1,3.3E1,8E0,4E0,3E1,1.2E1,8.2E1,4E0,4E0,8E1,1.87E2,2E1,5.92E2,6E0,3.38E2,5.4E1,8E0,1.68E2,1.1E1,7E0,1.2E1,7.09E2,1.45E2,3.8E1,4E0,2.7E1,6E0,4E0,4E0,2.6E1,4E0,8E0,4E0,2.9E1,5.3E1,6.9E1,1.1E1,1.32E2,5.5E1,1.3E1,7E0,1.36E2,4.56E2,3.17E2,2.1E1,1.6E1,3.8E1,4E0,4E0,1.56E2,1.2E1,7E0,4E0,4E0,8E0,1.01E2,6.08E2,3.3E1,1.12E2,3.4E1,4E0,6E0,2.1E1,1.7E1,9E0,5E0,2.4E1,6E0,4.7E1,6.1E1,8E0,4E0,7E0,1E1,1.22E2,2.4E1,3.1E1,4E0,9E0,1.27E2,9E0,7.2E1,3.84E2,3.13E2,4E0,6E0,1.5E1,1.2E1,4E0,3.3E1,5E0,4.4E1,1.12E2,6E0,6E0,7.8E1,2.3E1,4E1,5.68E2,9E0,2.4E1,5.3E1,5.9E1,2.3E1,1.1E1,1.4E1,7E0,4E0,1.3E1,4E0,5E0,1.3E1,1.1E1,1E1,3.7E1,1.4E1,4.7E1,4E0,4E0,6E0,4E0,1.08E2,1.4E1,1.8E1,6E0,1.3E1,1.8E1,1.07E2,2E1,4E0,5E0,7E0,6.5E1,1.78E2,2.06E2,3.8E1,2.75E2,4E0,1.1E1,7E0,5E0,2.2E1,1.1E1,3.6E1,8E0,1.02E2,1E1,1.4E1,6.4E1,1.9E1,4E0,3.3E1,7E0,4.1E1,5.27E2,4E0,5E0,4E0,2E1,4.7E1,6E0,2.8E1,3.1E1,9E0,1.4E1,7E0,4E0,7E0,7E0,5E0,6E0,5E0,5E0,2.9E1,8E0,1E1,4E0,4.2E1,5E0,8.9E1,1.9E1,5E0,9E0,7E0,1.1E1,5E0,8E0,1.2E1,6E0,8.2E1,2.5E1,7E0,1.3E1,3E1,3.5E1,1.45E2,3.3E1,2.02E2,4E0,1E1,2.8E1,2.24E2,5.1E1,1.3E1,9E0,2.2E1,1.4E1,4E0,4E0,9.7E1,5E0,5E0,5E0,9E0,5E0,1.2E1,5.2E1,7E0,1.2E1,2.6E1,7E0,3.6E1,5E0,3E2,2.27E2,1.2E1,8E0,4.3E1,4E0,1.4E1,1.4E1,1.9E1,1.2E1,4E0,5E0,6E0,8E0,4E0,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"261","size_leaf_vector":"1"}},{"base_weights":[-1.6913296E-3,-1.1352046E-2,3.9748955E-3,-1.0170598E-2,-1.131061E-1,1.0178507E-2,-1.548549E-2,-2.7339081E-3,-2.4292024E-2,-1.5019242E-3,-1.0330683E-2,7.850542E-3,5.7927005E-2,-2.2717187E-2,1.3634865E-2,-9.747497E-3,1.4374258E-2,-2.8485565E-2,2.8008595E-2,2.2582613E-2,9.4541913E-4,-2.1062752E-3,7.028542E-2,-2.0067545E-2,-6.1731217E-3,5.340992E-2,-3.3517345E-3,-7.836465E-3,-8.830767E-3,6.6512704E-2,4.526585E-3,-2.3319928E-2,-9.103144E-2,7.7707484E-2,-2.3844799E-2,2.4484454E-2,-5.0148126E-3,-2.6600935E-2,5.1071625E-3,7.5123366E-3,5.6574192E-2,-1.6319118E-2,-6.6620775E-2,5.6078103E-3,2.7567826E-2,7.3319073E-3,-4.766054E-3,-2.7010923E-3,-6.397498E-2,7.852918E-2,3.6782224E-4,8.525729E-3,-4.059528E-3,2.473419E-2,-2.8464658E-2,-8.170082E-3,-6.7166835E-2,4.931666E-3,1.218239E-3,-2.3537024E-3,5.568053E-4,3.88562E-2,7.3958915E-3,-9.499925E-3,-7.017686E-2,9.473104E-3,-2.5760528E-2,3.0939259E-2,9.848442E-2,-6.7575306E-2,-1.3288619E-2,-1.0345705E-1,2.175331E-3,4.240837E-3,5.1324475E-3,-2.0450177E-3,1.4516361E-2,-2.3165973E-2,6.804917E-3,-9.530541E-2,-5.6960112E-3,6.759938E-3,2.602924E-3,1.5344132E-2,-4.9111247E-2,4.767655E-2,-2.899707E-2,-6.646687E-3,-2.4571145E-2,-1.0783404E-1,1.11416906E-4,1.4955573E-2,5.3533133E-2,-1.1648365E-1,1.41081335E-2,-4.0876433E-2,1.3215831E-2,-7.4340943E-3,-2.34357E-2,7.5256773E-3,7.8824395E-3,-1.9125773E-2,-6.5992125E-3,1.8416425E-2,4.4905115E-3,6.6951546E-3,1.4298923E-3,-2.177281E-2,-8.193846E-3,-2.4563915E-4,-2.5417784E-2,-6.9059315E-4,-6.735281E-3,-1.6070564E-3,1.8027057E-3,1.3766519E-3,-1.0604139E-3,-9.221789E-4,2.6696444E-2,-9.9487575E-3,-9.916307E-2,1.5021439E-2,-4.3932777E-2,-1.3344221E-1,-1.17500065E-4,-2.7112937E-2,1.9125761E-3,-5.0758064E-2,2.1668378E-2,1.1751029E-3,-8.426239E-2,4.3020346E-3,1.0021184E-1,-3.275494E-3,6.6749746E-4,-5.218583E-2,-1.49372155E-2,-7.405068E-3,-2.889555E-3,2.490532E-2,-6.859511E-3,6.497419E-2,1.4755943E-2,3.1202268E-3,-1.3596766E-2,-4.854422E-3,1.8192258E-2,-8.7744646E-2,3.0551117E-3,3.3981245E-2,-2.4206422E-2,-5.2578617E-2,2.6645996E-3,-2.0290213E-3,1.9005436E-2,-5.004657E-2,5.1074643E-3,-2.6812672E-3,2.1612502E-3,-2.3522086E-3,1.1525382E-3,1.1918319E-2,-2.3015484E-2,1.7095151E-3,-3.3571515E-2,6.581212E-2,7.6560755E-3,3.960516E-4,-3.036244E-3,-1.1975813E-2,-2.2842032E-3,5.21404E-3,5.1340007E-4,-7.783046E-3,-1.3341879E-3,-7.4251993E-3,-3.3645337E-3,4.3019514E-5,-2.7611095E-3,-3.8532624E-3,7.8312965E-5,3.2529232E-4,2.6461782E-3,-6.7706783E-3,-1.7800044E-3,1.2712689E-3,-1.5181303E-3,6.5985024E-3,2.4145506E-3,-1.5927091E-3,-8.632401E-3,1.4346679E-3,-1.2793747E-3,5.4335305E-3,5.9086946E-4,2.8909473E-3,7.405507E-3,-1.7873519E-3,1.7366058E-3,3.3070876E-3,4.8226962E-4,8.125745E-5,-6.421497E-3,2.2700857E-3,-3.4990371E-3,2.884795E-3,-9.6428086E-4,1.9908631E-4,-2.2154432E-3,-3.8549798E-3,5.9942075E-4,-3.0145372E-4,2.2956645E-3,2.7993668E-3,1.6549062E-4,-9.71788E-4,-4.678221E-3,9.877533E-4,-2.6272633E-3,7.1974605E-4,-1.9853634E-3,2.84214E-4,4.802647E-3,-1.6006221E-3,1.246819E-3,-2.3495988E-3,1.0560091E-3,-2.937076E-3,-6.174827E-4,6.002198E-3,8.8022044E-4,-4.901037E-4,1.8914249E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,45,47,-1,49,51,53,55,57,59,61,-1,63,65,-1,67,69,71,-1,73,75,-1,77,79,81,-1,83,-1,85,87,-1,89,-1,-1,-1,-1,91,93,95,97,99,101,103,105,107,109,111,113,-1,115,-1,117,119,121,123,125,-1,-1,127,129,131,133,-1,135,137,-1,139,141,143,145,147,149,-1,151,153,-1,155,-1,157,-1,-1,-1,159,-1,161,163,-1,-1,-1,-1,-1,-1,-1,165,167,169,171,173,175,-1,177,-1,179,181,-1,183,185,187,-1,-1,189,191,-1,-1,193,-1,195,197,-1,-1,-1,199,201,203,205,207,209,-1,211,213,215,217,219,-1,-1,-1,221,223,225,227,229,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4318968E-1,1.1509969E-1,1.9923262E-1,1.0043454E-1,7.9332605E-2,1.3860251E-1,8.433498E-2,7.5493604E-2,7.317501E-2,0E0,0E0,1.2144038E-1,7.469934E-2,8.3011016E-2,5.4230455E-2,1.4094026E-1,9.365888E-2,9.6742466E-2,6.6279314E-2,9.1544E-2,9.345351E-2,0E0,4.8576802E-2,5.3531736E-2,0E0,3.338749E-2,5.674443E-2,1.2725236E-1,0E0,2.0034865E-2,5.3859513E-2,7.091606E-2,2.8800324E-2,1.21459365E-2,1.110499E-2,9.2055395E-2,0E0,7.906889E-2,9.5658734E-2,0E0,4.6373785E-2,4.4303603E-2,5.8511473E-2,0E0,2.2706458E-2,1.862348E-2,0E0,7.9140626E-2,6.701563E-2,2.8964356E-2,0E0,5.9309386E-2,0E0,3.5898693E-2,1.0135539E-1,0E0,5.386164E-2,0E0,0E0,0E0,0E0,7.07601E-2,1.454945E-1,5.6134023E-2,1.067348E-1,1.7813653E-1,5.9545033E-2,2.0484364E-2,3.8340077E-2,6.708335E-2,4.3657072E-2,4.0272743E-2,1.1637737E-2,0E0,8.856591E-3,0E0,1.8966522E-2,1.2859006E-1,1.1655462E-1,8.4765255E-2,1.422255E-2,0E0,0E0,5.7118077E-2,4.282954E-2,4.5142733E-2,1.4811184E-2,0E0,6.546393E-2,1.5651435E-2,0E0,1.2308215E-1,5.4941416E-2,2.6761925E-1,7.674429E-2,6.8214834E-2,3.656864E-2,0E0,4.754598E-2,6.738555E-2,0E0,6.319774E-2,0E0,1.3740291E-2,0E0,0E0,0E0,1.4700433E-2,0E0,3.7388485E-2,3.168738E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4818545E-2,1.01308614E-1,1.3173735E-1,1.00689664E-1,7.09939E-2,7.031679E-3,0E0,7.921137E-3,0E0,1.6671548E-2,5.799502E-2,0E0,2.2588275E-2,9.572002E-3,8.490637E-3,0E0,0E0,1.5122746E-1,8.734174E-2,0E0,0E0,8.0959246E-2,0E0,4.888341E-2,3.1558327E-2,0E0,0E0,0E0,6.4942986E-2,5.850879E-2,5.892768E-2,3.9081022E-2,9.824795E-3,2.4726998E-2,0E0,6.48185E-2,1.6190189E-1,4.7517188E-2,4.1635316E-2,1.0068413E-2,0E0,0E0,0E0,4.561621E-2,2.1131301E-2,3.3179E-2,5.809921E-2,2.6677646E-2,1.3028717E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,44,44,45,45,47,47,48,48,49,49,51,51,53,53,54,54,56,56,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,74,74,76,76,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,88,88,89,89,91,91,92,92,93,93,94,94,95,95,96,96,98,98,99,99,101,101,103,103,107,107,109,109,110,110,118,118,119,119,120,120,121,121,122,122,123,123,125,125,127,127,128,128,130,130,131,131,132,132,135,135,136,136,139,139,141,141,142,142,146,146,147,147,148,148,149,149,150,150,151,151,153,153,154,154,155,155,156,156,157,157,161,161,162,162,163,163,164,164,165,165,166,166],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,46,48,-1,50,52,54,56,58,60,62,-1,64,66,-1,68,70,72,-1,74,76,-1,78,80,82,-1,84,-1,86,88,-1,90,-1,-1,-1,-1,92,94,96,98,100,102,104,106,108,110,112,114,-1,116,-1,118,120,122,124,126,-1,-1,128,130,132,134,-1,136,138,-1,140,142,144,146,148,150,-1,152,154,-1,156,-1,158,-1,-1,-1,160,-1,162,164,-1,-1,-1,-1,-1,-1,-1,166,168,170,172,174,176,-1,178,-1,180,182,-1,184,186,188,-1,-1,190,192,-1,-1,194,-1,196,198,-1,-1,-1,200,202,204,206,208,210,-1,212,214,216,218,220,-1,-1,-1,222,224,226,228,230,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.960909E2,2.6104508E10,3.388159E7,2.4505E4,3.8833074E4,1.5137865E2,4.247868E3,8.46E2,9.5822406E2,-1.5019242E-3,-1.0330683E-2,8E0,9.822E3,8.015419E3,4.987639E7,3.8977896E2,2.7393186E4,2.589369E7,1.2792622E8,5.4E1,1.993E3,-2.1062752E-3,7.056912E7,2.2515285E6,-6.1731217E-3,9.042859E4,1.1355622E7,2.4E1,-8.830767E-3,1.0066326E8,1E0,1.0195312E0,5.486433E5,3.002239E5,1.010368E6,1.7248153E5,-5.0148126E-3,3.1E1,4.2145435E6,7.5123366E-3,1.4329944E0,1.34E2,2.262586E6,5.6078103E-3,1.00663277E9,1.3E1,-4.766054E-3,2.53E2,5.0740743E0,1.5277338E0,3.6782224E-4,7.629E3,-4.059528E-3,1E0,3.217E3,-8.170082E-3,2.264894E8,4.931666E-3,1.218239E-3,-2.3537024E-3,5.568053E-4,2.9883E4,3.7311627E2,4.470588E0,2.317155E-1,9.256843E3,1.5005797E4,1.445113E6,3.2295492E0,1.5213392E-7,1.038E3,9.86818E5,7.207554E4,4.240837E-3,1.957417E6,-2.0450177E-3,2.8541E4,6.769866E7,2.983871E0,7.759459E7,7.68E2,6.759938E-3,2.602924E-3,7.92582E5,1.7181714E7,1.1347137E2,7E0,-6.646687E-3,2.8729197E8,2.1631205E0,1.11416906E-4,1.1075305E0,1.3432479E7,3.7246967E2,1.22E2,2.37E2,1.2375429E8,-7.4340943E-3,2.067698E0,1.5166431E5,7.8824395E-3,1.7495675E5,-6.5992125E-3,4.0889E4,4.4905115E-3,6.6951546E-3,1.4298923E-3,3.1494663E0,-8.193846E-3,9.302862E7,1.2E1,-6.9059315E-4,-6.735281E-3,-1.6070564E-3,1.8027057E-3,1.3766519E-3,-1.0604139E-3,-9.221789E-4,9.5E2,4.296196E0,1.24E2,1.1E1,1.5944675E-4,2.8331464E2,-1.17500065E-4,1.0980226E4,1.9125761E-3,1.6358411E8,2.315878E6,1.1751029E-3,2.877528E2,1.1779856E2,3.61565E5,-3.275494E-3,6.6749746E-4,9.2611E4,4.85E2,-7.405068E-3,-2.889555E-3,6.46E2,-6.859511E-3,2.3012722E1,1.3642105E1,3.1202268E-3,-1.3596766E-2,-4.854422E-3,9.489796E-1,6.4547E4,1.5294166E5,1E0,6.227466E2,6.072483E2,2.6645996E-3,4.4854636E7,8.561815E7,1.0061821E5,5.2768206E5,1.3489793E7,2.1612502E-3,-2.3522086E-3,1.1525382E-3,1.3137E4,1E0,7E0,3.8466454E0,4.7777777E0,3.7930825E5,3.960516E-4,-3.036244E-3,-1.1975813E-2,-2.2842032E-3,5.21404E-3,5.1340007E-4,-7.783046E-3,-1.3341879E-3,-7.4251993E-3,-3.3645337E-3,4.3019514E-5,-2.7611095E-3,-3.8532624E-3,7.8312965E-5,3.2529232E-4,2.6461782E-3,-6.7706783E-3,-1.7800044E-3,1.2712689E-3,-1.5181303E-3,6.5985024E-3,2.4145506E-3,-1.5927091E-3,-8.632401E-3,1.4346679E-3,-1.2793747E-3,5.4335305E-3,5.9086946E-4,2.8909473E-3,7.405507E-3,-1.7873519E-3,1.7366058E-3,3.3070876E-3,4.8226962E-4,8.125745E-5,-6.421497E-3,2.2700857E-3,-3.4990371E-3,2.884795E-3,-9.6428086E-4,1.9908631E-4,-2.2154432E-3,-3.8549798E-3,5.9942075E-4,-3.0145372E-4,2.2956645E-3,2.7993668E-3,1.6549062E-4,-9.71788E-4,-4.678221E-3,9.877533E-4,-2.6272633E-3,7.1974605E-4,-1.9853634E-3,2.84214E-4,4.802647E-3,-1.6006221E-3,1.246819E-3,-2.3495988E-3,1.0560091E-3,-2.937076E-3,-6.174827E-4,6.002198E-3,8.8022044E-4,-4.901037E-4,1.8914249E-3],"split_indices":[52,20,45,29,33,56,52,2,55,0,0,17,29,4,45,4,50,45,7,3,2,0,7,28,0,28,29,8,0,7,65,53,28,33,9,33,0,8,47,0,38,0,1,0,7,3,0,2,54,53,0,10,0,16,12,0,7,0,0,0,0,1,52,54,41,4,4,9,38,37,0,12,33,0,29,0,2,7,53,7,2,0,0,12,12,52,8,0,5,54,0,38,9,4,10,0,7,0,38,28,0,33,0,2,0,0,0,46,0,45,3,0,0,0,0,0,0,0,0,58,29,10,42,4,0,33,0,7,9,0,4,4,1,0,0,29,0,0,0,2,0,56,58,0,0,0,53,1,33,105,55,4,0,9,12,33,28,51,0,0,0,2,59,3,54,54,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.614E3,9.66E2,1.648E3,9.56E2,1E1,1.25E3,3.98E2,6.27E2,3.29E2,6E0,4E0,1.193E3,5.7E1,3.19E2,7.9E1,4.45E2,1.82E2,3.05E2,2.4E1,3.8E2,8.13E2,6E0,5.1E1,3.12E2,7E0,2.3E1,5.6E1,4.41E2,4E0,2.8E1,1.54E2,2.83E2,2.2E1,1.2E1,1.2E1,3.75E2,5E0,1.06E2,7.07E2,6E0,4.5E1,2.9E2,2.2E1,6E0,1.7E1,5.1E1,5E0,4.05E2,3.6E1,2.3E1,5E0,1.48E2,6E0,2.7E1,2.56E2,4E0,1.8E1,8E0,4E0,7E0,5E0,2.03E2,1.72E2,7.7E1,2.9E1,6.2E2,8.7E1,2.9E1,1.6E1,1.5E1,2.75E2,1.4E1,8E0,4E0,1.3E1,6E0,4.5E1,1.28E2,2.77E2,2.3E1,1.3E1,6E0,1.7E1,1.33E2,1.5E1,1.9E1,8E0,8E0,2.48E2,1.1E1,7E0,7.8E1,1.25E2,8E0,1.64E2,3.2E1,4.5E1,1E1,1.9E1,6.13E2,7E0,8.3E1,4E0,2.5E1,4E0,1E1,6E0,1.1E1,4E0,1.33E2,1.42E2,4E0,1E1,4E0,4E0,7E0,6E0,1.2E1,3.3E1,1.1E2,1.8E1,2.39E2,3.8E1,1.6E1,7E0,9E0,4E0,1.1E1,1.22E2,5E0,1E1,1.1E1,8E0,4E0,4E0,6.3E1,1.85E2,5E0,6E0,7.4E1,4E0,9.6E1,2.9E1,4E0,4E0,5E0,1.59E2,1.5E1,1.7E1,2.9E1,1.6E1,1.4E1,5E0,3.35E2,2.78E2,3.6E1,4.7E1,1.4E1,1.1E1,7E0,4E0,8.7E1,4.6E1,3.3E1,1.09E2,1E1,2.3E1,8.2E1,2.8E1,4E0,1.4E1,1.1E1,2.28E2,4E0,3.4E1,1.2E1,4E0,5E0,4E0,7E0,4E0,8.3E1,3.9E1,4E0,6E0,7E0,4E0,4E0,4E0,5.5E1,8E0,3.6E1,1.49E2,9E0,6.5E1,9E1,6E0,8E0,2.1E1,2.3E1,1.36E2,5E0,1E1,1.1E1,6E0,2E1,9E0,7E0,9E0,1E1,4E0,3.1E2,2.5E1,8.2E1,1.96E2,2.2E1,1.4E1,3.8E1,9E0,1E1,4E0,8.2E1,5E0,3.9E1,7E0,9E0,2.4E1,4.9E1,6E1,4E0,6E0,1.5E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[-7.310812E-4,-9.952547E-3,5.566542E-3,2.0241972E-2,-1.389437E-2,9.34232E-3,-2.2351118E-2,4.07455E-2,-3.8540799E-3,-3.21282E-3,-2.3077074E-2,4.4535138E-2,7.823808E-3,-1.8027682E-2,-5.756821E-3,-4.3544765E-3,6.6620596E-2,3.091315E-2,-3.848841E-2,1.3232243E-3,-9.672149E-2,-1.3102712E-1,-1.9704739E-2,-2.6899986E-3,5.968683E-2,6.194871E-3,4.4015214E-2,-3.232286E-2,8.352848E-3,3.099354E-2,-5.2160453E-2,2.2537192E-2,1.0205744E-1,6.9657345E-3,4.642123E-3,-5.498667E-2,1.0602182E-3,6.264921E-3,-5.0601345E-2,-1.21161096E-1,-1.7001126E-3,-8.396778E-3,-1.1680522E-3,-5.124807E-2,-1.1626347E-2,7.205074E-2,-1.2386561E-3,1.2627659E-2,-1.6403913E-3,5.4694694E-3,3.0826887E-2,-2.0964205E-2,-6.528425E-2,-9.834258E-3,3.0014774E-2,3.4984727E-3,-5.9476536E-4,5.1966903E-4,-5.2978564E-3,3.131127E-3,-3.8900513E-3,9.394719E-3,6.773349E-2,-6.4188475E-3,2.6860621E-3,-7.9014555E-2,1.3121953E-3,3.517471E-3,6.6403095E-3,-1.3500261E-1,-1.2946911E-2,-3.388307E-3,-7.629759E-3,-1.07013E-2,-4.213384E-2,1.6991638E-2,-2.0297283E-2,8.1425615E-2,3.338263E-5,1.0012986E-2,6.247336E-2,-3.0734172E-2,4.1694934E-3,3.6819319E-3,1.4493048E-2,-2.7563144E-2,1.2548239E-2,-8.793392E-2,9.22505E-4,4.9694262E-2,-2.9312868E-2,7.442951E-2,6.3066664E-3,-1.9039055E-3,2.1173994E-3,1.9209932E-5,9.360739E-2,1.3038052E-3,-1.2416457E-3,-4.6056397E-3,-1.3173454E-3,-5.037335E-3,2.8424459E-2,-3.0492267E-3,-8.422257E-3,1.10825505E-2,-3.9849896E-3,-1.178041E-1,-3.0822212E-2,7.017567E-2,-5.3616003E-3,-1.8143084E-2,-6.1194296E-3,5.099351E-3,4.993895E-2,1.6387008E-2,-1.1470481E-2,7.95128E-2,1.2295959E-3,9.27208E-3,-5.9270825E-2,3.0595494E-2,-7.1001E-3,-1.3180799E-3,2.3546802E-2,7.568314E-3,-3.3918213E-2,2.1436564E-3,-9.680431E-4,-1.0481669E-1,-1.1307434E-3,4.8683733E-3,-3.9887105E-4,-3.293419E-3,-8.9933974E-4,4.9536503E-3,1.4936089E-3,1.0463105E-3,-2.0877013E-3,1.9863518E-3,6.699968E-3,1.2550266E-3,-7.142161E-2,-4.100023E-3,3.679449E-2,2.6377696E-3,-1.0563078E-2,-3.7339749E-3,-8.152708E-3,-9.953468E-3,-7.699274E-2,3.5947103E-2,1.24769524E-1,-4.10611E-2,1.14173135E-2,8.216338E-3,-2.8725978E-2,1.3215949E-3,4.4267788E-3,3.232895E-2,8.248153E-3,-6.020539E-3,-5.2912124E-3,1.0109382E-1,1.7343264E-3,-3.9004032E-3,5.109568E-3,-1.78126E-2,-2.2419149E-2,5.460565E-3,4.232061E-2,2.345192E-3,-3.0245816E-2,4.4575345E-2,-4.534324E-4,-1.3806466E-3,1.4301493E-3,-4.0604062E-2,9.965586E-3,-6.9006765E-3,-3.5515204E-3,-7.2856416E-4,1.5636185E-3,-2.0511653E-3,-7.7794413E-3,-1.6832007E-4,2.5820665E-3,9.403638E-4,-2.1303052E-3,-2.287789E-3,8.864517E-4,-8.3752885E-4,-5.9982324E-3,-9.4080047E-4,3.2430356E-3,3.1162782E-3,8.729101E-3,-3.444143E-3,3.0930038E-3,-3.4027858E-4,2.4146978E-3,8.214856E-4,-5.1371157E-3,-4.709498E-3,-1.0271448E-3,9.4596745E-4,3.510795E-3,4.9684366E-4,-5.278158E-3,2.6061533E-3,-6.4654765E-4,2.8278546E-3,6.2194266E-3,-1.1684418E-3,3.358596E-3,5.8647125E-5,-3.41203E-3,-2.0989645E-3,6.81092E-4,7.636824E-3,1.5944822E-3,-2.7461431E-3,4.8476516E-4,-9.465172E-4,-1.1600711E-2,7.434839E-4,3.6181435E-3,1.6412301E-3,-7.769089E-4,-2.277298E-3,-3.1561038E-4,2.2158464E-3,-1.3195416E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,55,57,59,61,63,-1,65,-1,67,69,71,-1,-1,-1,73,75,77,-1,79,81,-1,83,85,87,89,91,-1,-1,-1,-1,-1,93,-1,95,97,-1,99,-1,101,-1,103,105,-1,-1,-1,107,109,111,113,-1,115,117,119,121,-1,123,125,127,129,-1,131,133,135,137,-1,-1,-1,139,-1,-1,-1,-1,141,143,-1,-1,145,-1,147,149,151,153,155,-1,-1,157,159,161,163,-1,165,167,169,171,-1,173,175,177,-1,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,183,-1,185,-1,187,-1,-1,189,191,193,195,197,199,201,203,-1,-1,205,207,-1,209,211,-1,213,-1,-1,215,217,219,221,223,225,227,-1,-1,229,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5069759E-1,1.2561083E-1,1.627524E-1,6.0431644E-2,9.134157E-2,7.227751E-2,7.179591E-2,7.730986E-2,6.982576E-2,1.8336487E-1,1.7980412E-1,8.668056E-2,7.660273E-2,6.701924E-2,0E0,4.38628E-2,6.331134E-2,4.2404227E-2,2.9733483E-2,1.06290944E-1,2.6523083E-2,5.5850685E-2,1.2350123E-1,0E0,5.2964658E-2,6.297354E-2,4.4961356E-2,4.168705E-2,2.5115142E-2,2.5942575E-2,3.8172536E-2,2.1452751E-2,5.5127755E-2,1.4123276E-2,0E0,4.6976082E-2,0E0,1.3104223E-1,1.1266308E-1,1.3199449E-2,0E0,0E0,0E0,1.3798097E-1,9.6873194E-2,2.8721347E-2,0E0,8.878099E-2,9.546192E-2,0E0,3.2536E-2,1.958562E-2,5.6631655E-2,4.2040333E-2,3.0067287E-2,0E0,0E0,0E0,0E0,0E0,2.2192774E-2,0E0,3.133049E-2,1.14344265E-2,0E0,1.0041326E-2,0E0,7.919381E-2,0E0,1.7019093E-2,4.2622007E-2,0E0,0E0,0E0,7.768464E-2,1.0856028E-1,6.358824E-2,1.9586474E-2,0E0,8.960724E-2,1.9980088E-2,1.0786414E-1,1.4048587E-1,0E0,1.39428405E-2,1.6554896E-2,1.5365336E-2,2.2985771E-2,0E0,2.555832E-2,9.994058E-3,8.041535E-3,1.4949E-2,0E0,0E0,0E0,2.3010522E-2,0E0,0E0,0E0,0E0,1.1579637E-1,9.0867385E-2,0E0,0E0,1.8613562E-2,0E0,1.12528205E-2,8.0169424E-2,4.596898E-2,3.9403733E-2,8.2249954E-2,0E0,0E0,1.3092708E-2,6.512691E-2,1.00412615E-1,1.8729702E-2,0E0,4.9393885E-2,5.944234E-1,4.132302E-2,7.243352E-2,0E0,1.5160911E-2,9.683888E-3,1.8595472E-2,0E0,0E0,1.0400116E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2128051E-1,5.503405E-2,0E0,5.3367533E-2,0E0,1.3995303E-2,0E0,0E0,5.9361897E-2,6.540741E-2,2.977243E-2,2.0261407E-2,6.452267E-2,3.060301E-2,7.930589E-2,1.1047825E-1,0E0,0E0,8.506349E-2,6.572858E-2,0E0,6.350787E-2,8.559823E-3,0E0,5.1455148E-2,0E0,0E0,5.4186836E-2,1.841276E-2,1.05566576E-1,1.000229E-1,2.1470344E-1,1.2148228E-2,8.050708E-3,0E0,0E0,8.918457E-3,1.2426268E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,37,37,38,38,39,39,43,43,44,44,45,45,47,47,48,48,50,50,51,51,52,52,53,53,54,54,60,60,62,62,63,63,65,65,67,67,69,69,70,70,74,74,75,75,76,76,77,77,79,79,80,80,81,81,82,82,84,84,85,85,86,86,87,87,89,89,90,90,91,91,92,92,96,96,101,101,102,102,105,105,107,107,108,108,109,109,110,110,111,111,114,114,115,115,116,116,117,117,119,119,120,120,121,121,122,122,124,124,125,125,126,126,129,129,141,141,142,142,144,144,146,146,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,159,159,160,160,162,162,163,163,165,165,168,168,169,169,170,170,171,171,172,172,173,173,174,174,177,177,178,178],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,56,58,60,62,64,-1,66,-1,68,70,72,-1,-1,-1,74,76,78,-1,80,82,-1,84,86,88,90,92,-1,-1,-1,-1,-1,94,-1,96,98,-1,100,-1,102,-1,104,106,-1,-1,-1,108,110,112,114,-1,116,118,120,122,-1,124,126,128,130,-1,132,134,136,138,-1,-1,-1,140,-1,-1,-1,-1,142,144,-1,-1,146,-1,148,150,152,154,156,-1,-1,158,160,162,164,-1,166,168,170,172,-1,174,176,178,-1,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,184,-1,186,-1,188,-1,-1,190,192,194,196,198,200,202,204,-1,-1,206,208,-1,210,212,-1,214,-1,-1,216,218,220,222,224,226,228,-1,-1,230,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3030225E2,1.0195312E0,6.971004E7,2.53E2,4.085E3,3.5E1,1.2864323E7,3.6E2,6.9436204E-1,1E0,9.61E2,1E0,2.3274304E2,1.0052E4,-5.756821E-3,2.222456E0,1.7848537E0,4.6187375E4,1E0,2.4E1,1.5E1,1.2684739E1,8.87E2,-2.6899986E-3,1.1490676E6,1.0715278E7,4.70092E1,6.01324E11,3.6E1,6.377113E4,5.9E1,6.62E2,8.490231E-1,3.86569E5,4.642123E-3,2.4E1,1.0602182E-3,3.731E3,2.2401E4,3.0439E4,-1.7001126E-3,-8.396778E-3,-1.1680522E-3,1.3314917E0,2.62E2,5.1E1,-1.2386561E-3,1.437152E6,2.92E2,5.4694694E-3,2.2418336E5,5E0,4.0950096E10,3E0,2.1169072E3,3.4984727E-3,-5.9476536E-4,5.1966903E-4,-5.2978564E-3,3.131127E-3,4.3360384E7,9.394719E-3,8.9928055E-1,5.8826897E1,2.6860621E-3,3.99704E5,1.3121953E-3,5E0,6.6403095E-3,1.4735735E2,3.7423557E1,-3.388307E-3,-7.629759E-3,-1.07013E-2,4.54E2,1.1E1,2.6104508E10,3.38464E5,3.338263E-5,2.185321E0,1.2988522E7,3.114E3,5.244755E-3,3.6819319E-3,1.7997802E0,1E1,3.6672198E-8,2.974979E6,9.22505E-4,7.6499896E9,2.867292E3,6.773463E6,5.5743E4,-1.9039055E-3,2.1173994E-3,1.9209932E-5,7.79E2,1.3038052E-3,-1.2416457E-3,-4.6056397E-3,-1.3173454E-3,2E0,4E0,-3.0492267E-3,-8.422257E-3,2.322963E0,-3.9849896E-3,1.3058E4,7.89E2,6.03368E5,6.973054E7,2.2444444E0,-6.1194296E-3,5.099351E-3,7.661245E6,4.12E0,1.3E0,3.5E0,1.2295959E-3,3.5E1,7.792289E9,6.973054E7,5.69815E5,-1.3180799E-3,2.6666667E0,9.195571E2,8.32E3,2.1436564E-3,-9.680431E-4,1.13808104E8,-1.1307434E-3,4.8683733E-3,-3.9887105E-4,-3.293419E-3,-8.9933974E-4,4.9536503E-3,1.4936089E-3,1.0463105E-3,-2.0877013E-3,1.9863518E-3,6.699968E-3,5.5E1,7.29E2,-4.100023E-3,1.8991614E5,2.6377696E-3,5.827505E5,-3.7339749E-3,-8.152708E-3,1.7142857E0,1.3333334E0,3.5006908E6,3.78E2,8.92686E5,3.9556694E-1,1.6018559E1,9.417789E1,1.3215949E-3,4.4267788E-3,3.557179E1,8.7690773E-7,-6.020539E-3,1.1376811E0,1E1,1.7343264E-3,9.9783656E7,5.109568E-3,-1.78126E-2,1.4813511E6,1.3304372E7,8.756142E7,1.9682497E-3,1.5005797E4,7.26E2,2.3292E4,-1.3806466E-3,1.4301493E-3,7.160724E1,1.0972E4,-6.9006765E-3,-3.5515204E-3,-7.2856416E-4,1.5636185E-3,-2.0511653E-3,-7.7794413E-3,-1.6832007E-4,2.5820665E-3,9.403638E-4,-2.1303052E-3,-2.287789E-3,8.864517E-4,-8.3752885E-4,-5.9982324E-3,-9.4080047E-4,3.2430356E-3,3.1162782E-3,8.729101E-3,-3.444143E-3,3.0930038E-3,-3.4027858E-4,2.4146978E-3,8.214856E-4,-5.1371157E-3,-4.709498E-3,-1.0271448E-3,9.4596745E-4,3.510795E-3,4.9684366E-4,-5.278158E-3,2.6061533E-3,-6.4654765E-4,2.8278546E-3,6.2194266E-3,-1.1684418E-3,3.358596E-3,5.8647125E-5,-3.41203E-3,-2.0989645E-3,6.81092E-4,7.636824E-3,1.5944822E-3,-2.7461431E-3,4.8476516E-4,-9.465172E-4,-1.1600711E-2,7.434839E-4,3.6181435E-3,1.6412301E-3,-7.769089E-4,-2.277298E-3,-3.1561038E-4,2.2158464E-3,-1.3195416E-3],"split_indices":[52,53,45,10,29,0,29,2,53,74,12,15,56,2,0,54,54,33,74,8,3,56,2,0,28,45,58,31,8,45,29,2,53,1,0,8,0,29,1,12,0,0,0,54,11,3,0,43,0,0,28,6,5,8,52,0,0,0,0,0,5,0,53,50,0,1,0,8,0,52,58,0,0,0,2,3,20,9,0,41,43,29,57,0,53,3,37,29,0,5,4,1,10,0,0,0,2,0,0,0,0,6,3,0,0,53,0,29,2,9,7,56,0,0,5,54,54,53,0,3,31,7,9,0,53,4,2,0,0,45,0,0,0,0,0,0,0,0,0,0,0,0,29,0,28,0,45,0,0,56,56,45,0,1,38,57,52,0,0,58,37,0,53,8,0,5,0,0,28,45,7,39,4,8,2,0,0,56,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.593E3,1.052E3,1.541E3,1.21E2,9.31E2,1.358E3,1.83E2,6.5E1,5.6E1,4.31E2,5E2,5.5E1,1.303E3,1.76E2,7E0,2.4E1,4.1E1,2.8E1,2.8E1,4.12E2,1.9E1,1.4E1,4.86E2,7E0,4.8E1,1.248E3,5.5E1,1.14E2,6.2E1,1.4E1,1E1,1.9E1,2.2E1,2.1E1,7E0,2.2E1,6E0,3.77E2,3.5E1,1.3E1,6E0,1E1,4E0,9.8E1,3.88E2,4.2E1,6E0,6.85E2,5.63E2,8E0,4.7E1,8.6E1,2.8E1,3.4E1,2.8E1,7E0,7E0,5E0,5E0,7E0,1.2E1,5E0,1.7E1,1.7E1,4E0,1.7E1,5E0,3.7E2,7E0,1E1,2.5E1,6E0,7E0,4E0,9.4E1,9E1,2.98E2,3.7E1,5E0,6.52E2,3.3E1,9.3E1,4.7E2,1.2E1,3.5E1,7.2E1,1.4E1,2.2E1,6E0,8E0,2.6E1,9E0,1.9E1,7E0,5E0,5E0,1.2E1,6E0,1.1E1,1.3E1,4E0,2.76E2,9.4E1,4E0,6E0,1.9E1,6E0,1.1E1,8.3E1,2.6E1,6.4E1,2.93E2,5E0,2.1E1,1.6E1,5.03E2,1.49E2,2.2E1,1.1E1,3.9E1,5.4E1,1.4E2,3.3E2,6E0,2.9E1,1.1E1,6.1E1,7E0,7E0,1.7E1,5E0,4E0,4E0,5E0,2.1E1,5E0,4E0,1.5E1,4E0,6E0,6E0,2.53E2,2.3E1,6E0,8.8E1,6E0,1.3E1,7E0,4E0,5.8E1,2.5E1,1.7E1,9E0,2E1,4.4E1,8.4E1,2.09E2,1.1E1,5E0,1.69E2,3.34E2,7E0,1.42E2,1.4E1,8E0,3.5E1,4E0,5E0,4.9E1,4.5E1,9.5E1,2.35E2,9.5E1,1.5E1,1.4E1,4E0,7E0,5.3E1,8E0,7E0,1E1,1.66E2,8.7E1,1.8E1,5E0,2.4E1,6.4E1,7E0,6E0,2.5E1,3.3E1,1.1E1,1.4E1,6E0,1.1E1,5E0,4E0,1.6E1,4E0,3E1,1.4E1,7.9E1,5E0,2.2E1,1.87E2,1.26E2,4.3E1,3.3E2,4E0,1.6E1,1.26E2,6E0,8E0,2.8E1,7E0,3.3E1,1.6E1,6E0,3.9E1,7E0,8.8E1,2.6E1,2.09E2,9.1E1,4E0,8E0,7E0,4E0,1E1,4.6E1,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[1.2642146E-3,-4.9544303E-3,1.193854E-2,2.861656E-2,-8.06708E-3,2.1137634E-2,-8.315044E-3,-4.3773826E-2,3.855061E-2,-6.7405486E-3,-6.771618E-2,4.9543284E-2,1.6150195E-2,2.4709545E-5,-3.5510253E-2,3.618695E-3,-9.8224014E-2,6.3907905E-3,3.1942237E-2,-1.051716E-2,9.293421E-3,1.7353505E-4,-1.00232765E-1,7.6664366E-2,1.7102404E-2,3.6792196E-2,8.687429E-3,-7.3219393E-3,3.3352807E-2,-2.4497556E-2,-8.572945E-3,-9.610864E-4,-6.7612105E-3,5.6579474E-2,1.6784692E-2,-6.802898E-3,-2.8071934E-2,3.1931374E-2,-1.6515476E-3,-1.3643937E-3,-1.2587364E-1,1.2380621E-1,4.8065558E-2,4.0693276E-2,-1.7321596E-2,7.2971605E-2,1.3469769E-2,-5.210919E-2,1.1642193E-2,3.4190854E-3,-2.8346783E-2,6.76017E-3,1.8675597E-2,-7.535674E-2,-1.2741856E-2,8.2531966E-2,2.2607509E-2,4.207902E-2,-1.1157284E-2,-1.1856485E-2,9.078024E-3,-6.558411E-2,-1.1986417E-2,-2.4537489E-2,4.0933475E-2,1.0824531E-2,-3.5133336E-2,-1.4968275E-1,-2.1764017E-3,1.8263322E-3,1.52358E-1,1.43424915E-2,9.521691E-2,2.2453723E-2,5.8189463E-3,2.1375213E-2,-5.23287E-3,1.8451685E-3,8.993999E-2,-3.1049327E-5,8.676214E-2,-1.7003583E-2,-7.3453085E-3,1.3349057E-2,-4.9593425E-3,6.58836E-2,-2.6381842E-3,-6.9332756E-3,-1.6436193E-2,4.7317866E-2,-3.004036E-3,-6.020617E-3,-1.8851019E-3,-1.3631885E-3,6.5700198E-3,-4.0284343E-4,1.0105392E-1,3.9596757E-4,3.715943E-3,1.3039592E-2,9.092949E-2,-3.9987173E-2,1.2464492E-2,-2.0017696E-3,-2.3782972E-2,6.1852396E-3,7.011846E-3,-4.79548E-2,-1.0952375E-2,-6.993589E-4,-4.84828E-2,-3.1231649E-3,1.5077642E-3,4.6357483E-2,-1.5929792E-3,2.5674997E-2,-1.1114088E-2,-4.7990937E-2,3.4625472E-3,-3.9616562E-3,-8.742687E-3,3.6763141E-3,8.819792E-3,-2.5201676E-2,6.0361244E-2,6.817102E-3,2.3609612E-3,1.9571233E-4,3.2110657E-3,-6.283776E-4,1.8094563E-3,-4.3647424E-3,2.8662598E-3,9.9382736E-2,5.2088225E-4,-1.4497713E-2,6.610449E-2,2.5260842E-3,6.3263746E-3,1.948481E-3,-3.5415043E-3,1.1560686E-2,6.1854552E-3,1.4231523E-3,5.2499515E-3,-3.1971965E-2,7.284116E-3,-4.121137E-3,-7.229908E-3,1.695345E-5,3.7710383E-3,-8.827833E-4,1.5460423E-3,3.1269614E-2,-6.64861E-4,1.22900456E-1,2.1447982E-3,-1.1640894E-3,1.5390832E-3,-5.719084E-3,3.10259E-3,2.2362089E-3,7.2470554E-3,-5.512095E-2,2.1642195E-4,2.9096575E-3,-2.6168767E-3,-1.19094765E-2,2.8542193E-2,-4.502756E-2,-5.1498595E-3,5.4804888E-2,-1.3817319E-3,-2.4929075E-2,-1.6543037E-1,-1.229052E-2,2.1891305E-2,-3.88007E-3,-1.11923836E-1,9.638281E-3,5.9098713E-2,-1.99455E-2,4.170739E-2,-5.9710268E-2,6.506311E-3,-8.5253246E-2,-2.9024512E-2,6.581379E-4,-2.722822E-3,1.2352492E-3,4.5498013E-3,3.2005075E-3,1.299801E-1,-3.0023364E-2,4.87201E-2,4.6776547E-3,6.905674E-4,1.9063039E-2,-6.2020277E-3,-1.2474776E-2,-4.076917E-3,3.9318956E-2,-8.042319E-3,-1.6223367E-2,1.6071756E-3,2.9849743E-3,3.2313558E-4,8.624973E-3,4.2848103E-3,1.316876E-3,-1.1267415E-3,-8.9417846E-4,-3.9550643E-3,-1.3686582E-3,2.1605224E-3,-7.800695E-4,3.3058473E-3,-1.3267822E-3,2.530512E-3,-8.717602E-3,-1.5963699E-3,4.1754742E-4,-1.7350272E-3,-9.337458E-4,4.8047947E-3,-4.038135E-3,1.611125E-4,-6.337861E-3,-5.0610997E-4,-1.1852573E-2,-3.0262347E-3,-1.3052693E-3,5.679661E-4,-9.426167E-4,1.958413E-3,-1.4954724E-3,4.2703343E-3,-3.1059983E-3,-7.0690583E-3,2.035477E-3,-3.4833835E-3,3.6190292E-3,3.201583E-4,-1.9588224E-3,9.3211373E-4,9.997363E-4,3.5050856E-3,-1.0969837E-3,-6.5372046E-3,4.8883134E-3,-2.8387955E-4,-6.7117293E-3,-2.3626932E-3,-1.9782109E-3,7.1788946E-4,7.102067E-3,3.1305982E-3,2.258986E-3,-2.2349905E-3,-2.769682E-4,3.6701995E-3,7.74097E-4,4.108775E-3,-4.310487E-3,3.5964633E-5,-1.4824753E-3,1.065503E-3,-1.6027392E-3,3.01041E-3,-1.096068E-3,1.1173558E-3,-2.8866576E-3,8.611494E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,37,-1,39,41,43,45,47,49,51,53,-1,-1,-1,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,-1,121,123,125,127,-1,129,-1,131,133,135,137,139,-1,141,-1,143,145,-1,147,149,151,-1,-1,-1,153,-1,155,157,-1,159,161,163,165,167,169,171,-1,173,-1,175,177,-1,-1,179,-1,181,183,185,-1,-1,-1,-1,-1,187,189,-1,-1,-1,-1,-1,-1,-1,-1,191,-1,193,195,-1,-1,-1,-1,197,-1,-1,-1,199,201,-1,203,-1,-1,-1,-1,205,-1,207,-1,-1,-1,209,-1,-1,-1,211,-1,-1,213,215,217,219,221,223,225,227,229,231,233,235,237,239,241,243,245,247,249,251,253,-1,-1,-1,-1,-1,255,257,259,-1,-1,261,263,265,-1,267,269,271,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6932774E-1,1.6867639E-1,1.7512521E-1,1.0020828E-1,1.1633415E-1,9.071964E-2,6.6680476E-2,1.1467082E-1,6.736036E-2,8.759836E-2,7.442266E-2,8.31527E-2,8.455996E-2,5.558027E-2,9.918795E-2,0E0,3.2554306E-2,0E0,4.1622497E-2,7.609658E-2,6.8437725E-2,0E0,3.7700757E-2,6.415558E-2,3.724902E-2,1.2221065E-1,7.354037E-2,4.2100996E-2,5.8817778E-2,3.790236E-2,0E0,0E0,0E0,3.6045417E-2,5.1357172E-2,7.7714816E-2,1.2230803E-1,4.6754852E-2,7.849421E-2,0E0,2.4838388E-2,4.176575E-2,5.2697346E-2,3.4696106E-2,6.650263E-2,6.862742E-2,8.9058325E-2,6.079598E-2,7.4766785E-2,4.7091E-2,8.05683E-2,0E0,2.3117438E-2,1.5835486E-2,1.5342261E-2,4.1086286E-2,2.2964355E-2,5.2720733E-2,2.4333881E-2,8.622098E-2,8.797831E-2,1.546784E-1,5.9284993E-2,2.889115E-2,3.228365E-2,4.491978E-2,7.180462E-2,7.4893236E-3,0E0,0E0,1.3655454E-2,3.9736442E-2,2.123113E-2,1.7574076E-2,0E0,7.508182E-3,0E0,6.443646E-2,3.3502758E-2,7.4628934E-2,1.3798706E-2,4.8017208E-2,0E0,7.421613E-2,0E0,1.3128996E-2,3.3413563E-2,0E0,3.466753E-2,2.1135952E-2,1.14973085E-2,0E0,0E0,0E0,1.2198439E-2,0E0,2.012305E-2,1.1523963E-2,0E0,2.3359634E-2,2.8487638E-2,1.0970049E-2,1.395711E-2,1.22296035E-1,1.3119125E-1,8.4693536E-2,0E0,1.4735372E-1,0E0,2.9334633E-2,9.577362E-2,0E0,0E0,3.3583283E-2,0E0,6.0807146E-2,4.83404E-2,3.003528E-2,0E0,0E0,0E0,0E0,0E0,1.4205672E-2,8.688748E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.5995513E-2,0E0,6.42786E-2,1.884355E-2,0E0,0E0,0E0,0E0,5.049132E-2,0E0,0E0,0E0,2.703865E-2,4.2540487E-2,0E0,1.894522E-2,0E0,0E0,0E0,0E0,7.381011E-3,0E0,1.1051968E-2,0E0,0E0,0E0,1.0851138E-2,0E0,0E0,0E0,9.319216E-3,0E0,0E0,1.9361429E-2,8.919078E-2,1.2189817E-1,2.568314E-1,7.138966E-2,9.6014686E-2,7.3438056E-2,7.1114615E-2,5.3377762E-2,2.4512403E-2,2.7537312E-2,5.1432103E-2,1.1793539E-2,5.2194625E-2,3.746459E-2,1.7084513E-2,3.6242753E-2,3.740634E-2,4.7303617E-2,2.1595165E-2,1.5240267E-2,0E0,0E0,0E0,0E0,0E0,8.28144E-3,5.9555E-2,1.785423E-2,0E0,0E0,5.872394E-2,6.2686175E-2,1.3418155E-2,0E0,4.3771747E-2,2.5413768E-2,3.2162927E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,73,73,75,75,77,77,78,78,79,79,80,80,81,81,83,83,85,85,86,86,88,88,89,89,90,90,94,94,96,96,97,97,99,99,100,100,101,101,102,102,103,103,104,104,105,105,107,107,109,109,110,110,113,113,115,115,116,116,117,117,123,123,124,124,133,133,135,135,136,136,141,141,145,145,146,146,148,148,153,153,155,155,159,159,163,163,166,166,167,167,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,185,185,186,186,192,192,193,193,194,194,197,197,198,198,199,199,201,201,202,202,203,203],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,38,-1,40,42,44,46,48,50,52,54,-1,-1,-1,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,-1,122,124,126,128,-1,130,-1,132,134,136,138,140,-1,142,-1,144,146,-1,148,150,152,-1,-1,-1,154,-1,156,158,-1,160,162,164,166,168,170,172,-1,174,-1,176,178,-1,-1,180,-1,182,184,186,-1,-1,-1,-1,-1,188,190,-1,-1,-1,-1,-1,-1,-1,-1,192,-1,194,196,-1,-1,-1,-1,198,-1,-1,-1,200,202,-1,204,-1,-1,-1,-1,206,-1,208,-1,-1,-1,210,-1,-1,-1,212,-1,-1,214,216,218,220,222,224,226,228,230,232,234,236,238,240,242,244,246,248,250,252,254,-1,-1,-1,-1,-1,256,258,260,-1,-1,262,264,266,-1,268,270,272,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.860185E2,1E0,3.5301748E7,3.4643276E1,6.971004E7,1.4157E4,1.1900813E1,2.645E3,2.04E5,4.264897E6,7.285643E-1,7.5939475E5,1.845406E1,9.256843E3,6.07372E5,3.618695E-3,6.363636E-1,6.3907905E-3,9E0,9.669789E0,5.0723374E-2,1.7353505E-4,2.0469315E0,2.4832625E6,9E0,7.465873E6,1.43E2,1.6225381E7,2.909019E3,4.30453E2,-8.572945E-3,-9.610864E-4,-6.7612105E-3,1.5938586E8,2E0,5.1538463E0,1.646E3,1.6791044E-1,1.7E1,-1.3643937E-3,3.7E1,1.0119178E10,3.5905025E5,3.7E1,2.545455E7,2.579E3,1.3432479E7,8.99653E9,3.7826266E4,6.3157897E0,9.173084E2,6.76017E-3,8.931957E9,1.0323588E3,2.0455818E1,1.5687E4,2.8524646E-1,3.72424E5,6.445E3,3.4015749E0,1.1037509E2,5.4E1,6.17296E5,7E0,1.23E2,4.3579788E2,1.6006084E3,8.853684E-1,-2.1764017E-3,1.8263322E-3,1.5754185E7,1.753919E3,2.6E1,3.2196458E6,5.8189463E-3,1.7735592E3,-5.23287E-3,2.5933685E-2,7.064848E6,2.0568174E7,7.996879E0,2.3902743E0,-7.3453085E-3,1.9013363E4,-4.9593425E-3,6E0,3.488E3,-6.9332756E-3,1E0,1E0,6.9E1,-6.020617E-3,-1.8851019E-3,-1.3631885E-3,1.59551E5,-4.0284343E-4,5.68E2,1.1E1,3.715943E-3,1.2339E4,3.17E2,1.1E1,9.822E3,3.1517188E2,1.088E3,3E1,7.011846E-3,8.1407714E-1,-1.0952375E-2,1.9283049E-1,7.8E1,-3.1231649E-3,1.5077642E-3,1.6633E4,-1.5929792E-3,1.145677E6,5.0647113E2,3.7225406E4,3.4625472E-3,-3.9616562E-3,-8.742687E-3,3.6763141E-3,8.819792E-3,1.067536E3,2.2243262E5,6.817102E-3,2.3609612E-3,1.9571233E-4,3.2110657E-3,-6.283776E-4,1.8094563E-3,-4.3647424E-3,2.8662598E-3,4E0,5.2088225E-4,8.631E3,1.8E1,2.5260842E-3,6.3263746E-3,1.948481E-3,-3.5415043E-3,1.2817779E7,6.1854552E-3,1.4231523E-3,5.2499515E-3,8.471288E10,2.7864855E11,-4.121137E-3,9.557264E5,1.695345E-5,3.7710383E-3,-8.827833E-4,1.5460423E-3,1.7322648E5,-6.64861E-4,2.5797562E2,2.1447982E-3,-1.1640894E-3,1.5390832E-3,1.6713532E9,3.10259E-3,2.2362089E-3,7.2470554E-3,2.5207965E2,2.1642195E-4,2.9096575E-3,2.14306E5,2.4E1,4.577553E3,8E0,5.7E1,1.0853132E4,1.4181934E-5,1.954516E-4,1.1075305E0,1.1394967E7,5.667654E5,3.51E2,1.3446785E7,1.362E4,1.9407146E7,1.0604491E0,1.1E1,1.8238512E0,1.0980226E4,1.24E2,8.9655056E5,6.581379E-4,-2.722822E-3,1.2352492E-3,4.5498013E-3,3.2005075E-3,6.4E1,9.4538574E2,3.3720784E5,4.6776547E-3,6.905674E-4,9.750042E6,2.7661E4,1.5043668E0,-4.076917E-3,1.0346457E0,4.099E3,1.9557823E0,1.6071756E-3,2.9849743E-3,3.2313558E-4,8.624973E-3,4.2848103E-3,1.316876E-3,-1.1267415E-3,-8.9417846E-4,-3.9550643E-3,-1.3686582E-3,2.1605224E-3,-7.800695E-4,3.3058473E-3,-1.3267822E-3,2.530512E-3,-8.717602E-3,-1.5963699E-3,4.1754742E-4,-1.7350272E-3,-9.337458E-4,4.8047947E-3,-4.038135E-3,1.611125E-4,-6.337861E-3,-5.0610997E-4,-1.1852573E-2,-3.0262347E-3,-1.3052693E-3,5.679661E-4,-9.426167E-4,1.958413E-3,-1.4954724E-3,4.2703343E-3,-3.1059983E-3,-7.0690583E-3,2.035477E-3,-3.4833835E-3,3.6190292E-3,3.201583E-4,-1.9588224E-3,9.3211373E-4,9.997363E-4,3.5050856E-3,-1.0969837E-3,-6.5372046E-3,4.8883134E-3,-2.8387955E-4,-6.7117293E-3,-2.3626932E-3,-1.9782109E-3,7.1788946E-4,7.102067E-3,3.1305982E-3,2.258986E-3,-2.2349905E-3,-2.769682E-4,3.6701995E-3,7.74097E-4,4.108775E-3,-4.310487E-3,3.5964633E-5,-1.4824753E-3,1.065503E-3,-1.6027392E-3,3.01041E-3,-1.096068E-3,1.1173558E-3,-2.8866576E-3,8.611494E-5],"split_indices":[52,53,45,52,45,9,54,9,5,9,27,28,56,4,11,0,53,0,3,54,41,0,56,32,8,50,0,48,32,33,0,0,0,7,8,54,2,58,3,0,8,20,28,3,45,2,9,31,52,56,52,0,12,52,54,1,38,9,29,56,56,3,29,8,3,55,55,27,0,0,45,52,3,28,0,4,0,38,28,47,56,53,0,52,0,8,2,0,8,8,8,0,0,0,9,0,0,8,0,29,10,8,29,55,10,0,0,41,0,38,8,0,0,1,0,12,4,33,0,0,0,0,0,52,33,0,0,0,0,0,0,0,0,8,0,2,3,0,0,0,0,9,0,0,0,31,31,0,28,0,0,0,0,28,0,4,0,0,0,31,0,0,0,4,0,0,1,3,33,3,8,28,38,38,38,45,45,3,45,1,9,38,3,53,33,0,28,0,0,0,0,0,3,4,28,0,0,9,29,53,0,53,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.549E3,1.611E3,9.38E2,1.36E2,1.475E3,6.45E2,2.93E2,1.6E1,1.2E2,1.444E3,3.1E1,9.5E1,5.5E2,2.25E2,6.8E1,5E0,1.1E1,7E0,1.13E2,1.169E3,2.75E2,1E1,2.1E1,5.1E1,4.4E1,1.45E2,4.05E2,1.85E2,4E1,6.4E1,4E0,4E0,7E0,4.2E1,7.1E1,9.66E2,2.03E2,8.9E1,1.86E2,6E0,1.5E1,1.8E1,3.3E1,2.6E1,1.8E1,5.6E1,8.9E1,1.8E1,3.87E2,1.23E2,6.2E1,4E0,3.6E1,1.1E1,5.3E1,2.3E1,1.9E1,3.7E1,3.4E1,7.33E2,2.33E2,6E1,1.43E2,1.2E1,7.7E1,1.36E2,5E1,1.1E1,4E0,5E0,1.3E1,2E1,1.3E1,2.2E1,4E0,1.3E1,5E0,1.1E1,4.5E1,7.6E1,1.3E1,1.4E1,4E0,3.82E2,5E0,1E1,1.13E2,5E0,5.7E1,1.5E1,2.1E1,4E0,7E0,3E1,2.3E1,4E0,1.9E1,1.4E1,5E0,2.4E1,1.3E1,1.5E1,1.9E1,4.02E2,3.31E2,2.29E2,4E0,5.5E1,5E0,1.1E2,3.3E1,7E0,5E0,7.2E1,5E0,8.1E1,5.5E1,4.5E1,5E0,4E0,7E0,4E0,9E0,1.1E1,9E0,6E0,7E0,1.6E1,6E0,4E0,9E0,4E0,7E0,4E1,5E0,6.3E1,1.3E1,8E0,5E0,7E0,7E0,3.77E2,5E0,6E0,4E0,2.8E1,8.5E1,6E0,5.1E1,6E0,9E0,1.5E1,6E0,1E1,1.3E1,1.3E1,6E0,8E0,6E0,1.8E1,6E0,8E0,5E0,1.1E1,4E0,4E0,1.5E1,3.04E2,9.8E1,1.54E2,1.77E2,3E1,1.99E2,4.7E1,8E0,7.3E1,3.7E1,2E1,1.3E1,1.9E1,5.3E1,2.1E1,6E1,1.4E1,4.1E1,1.4E1,3.1E1,5E0,6E0,5E0,4E0,2E1,2E1,5.1E1,1.2E1,8E0,5E0,2.65E2,1.12E2,2.1E1,7E0,2.7E1,5.8E1,4.2E1,9E0,4E0,6E0,4E0,9E0,6E0,1.2E1,5E0,6E0,1E1,5E0,2.91E2,1.3E1,2.8E1,7E1,1.3E1,1.41E2,1.22E2,5.5E1,1.1E1,1.9E1,1E1,1.89E2,5E0,4.2E1,4E0,4E0,4.6E1,2.7E1,1.1E1,2.6E1,1.6E1,4E0,6E0,7E0,1.4E1,5E0,4.2E1,1.1E1,1.4E1,7E0,3.5E1,2.5E1,1E1,4E0,4E0,3.7E1,5E0,9E0,2.5E1,6E0,1.6E1,4E0,8E0,4.3E1,4E0,8E0,2.52E2,1.3E1,8E0,1.04E2,1.4E1,7E0,6E0,2.1E1,4E1,1.8E1,1.2E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"273","size_leaf_vector":"1"}},{"base_weights":[-6.468919E-5,-5.1803463E-3,1.5666468E-2,-2.0878983E-3,-2.8773434E-2,1.9054184E-2,-3.725539E-2,-1.4635512E-3,-7.730638E-2,-2.2015374E-2,-8.3891705E-2,2.6128486E-2,-1.2760424E-3,-1.4025663E-2,-7.93519E-3,-1.22666955E-2,2.9199885E-3,-8.9665607E-4,-5.9752245E-3,-1.6739111E-2,-9.0918645E-2,-4.8644543E-2,-1.0398164E-2,2.169742E-2,1.1287258E-1,-6.623043E-3,3.137662E-3,-5.251072E-2,4.990045E-3,-6.8178484E-3,-8.049062E-2,7.921079E-3,-1.2996397E-2,-4.6117E-2,-7.377265E-3,-8.3585015E-5,-1.403138E-1,-6.0748667E-3,-8.70301E-3,1.2879474E-2,4.712437E-2,6.778571E-2,1.2171551E-2,-1.7987987E-3,4.5153694E-3,6.6562236E-4,-4.5985365E-3,-7.535834E-4,2.1589998E-2,-3.0647367E-3,-5.297929E-2,-1.3753484E-2,-4.019817E-2,6.43777E-3,8.598325E-2,-1.033123E-1,-5.870911E-3,-5.969064E-2,3.6774468E-4,4.4594206E-2,-1.4743212E-2,-9.2775915E-3,-3.342231E-3,-2.6012757E-3,2.8872392E-3,1.0024968E-2,5.8787555E-2,1.1847566E-1,3.5109386E-2,9.547047E-2,7.293842E-4,1.5807007E-3,-4.732504E-3,2.4709178E-3,-1.00170306E-4,4.2749904E-3,-2.821933E-2,-1.4236862E-2,-1.2555331E-2,-1.6556483E-2,-6.7927577E-3,5.2354578E-2,3.959863E-3,7.952398E-3,3.1603348E-2,-6.089294E-2,-8.966619E-3,-9.870584E-3,3.716346E-2,-3.0129943E-2,-4.088907E-3,6.9117304E-3,7.932971E-3,-9.8720856E-2,-5.852125E-3,2.416627E-2,2.157275E-3,5.459962E-4,8.180713E-2,2.539184E-3,7.891469E-3,-2.7798756E-3,4.0897507E-2,6.1923647E-3,1.8254406E-3,5.128582E-3,-4.687805E-3,6.5643457E-3,-7.195924E-3,-8.233664E-3,-1.7161286E-2,3.1582627E-2,-1.0398951E-1,-3.819178E-2,1.8436154E-3,1.3638406E-2,7.064826E-2,1.4172266E-3,3.0003237E-2,5.298538E-3,-2.4034958E-3,-5.3485944E-3,-1.8232366E-2,-6.8547116E-3,-8.03597E-2,9.021621E-3,6.828932E-3,-4.4963844E-2,7.227272E-4,-1.0059795E-3,1.9424591E-3,-1.6841083E-3,-7.012625E-3,3.6975592E-3,-5.429777E-2,3.2118496E-2,-2.2075396E-2,-4.4296496E-3,4.586789E-3,8.9667406E-4,5.571023E-3,9.9485464E-2,3.0337756E-2,-8.700615E-3,2.23296E-2,6.6815974E-4,-1.1685084E-3,7.264505E-4,-2.1074966E-3,-1.6117712E-3,3.28813E-3,-2.480533E-3,-7.4153794E-3,-4.859569E-3,-1.7238614E-4,3.2633115E-3,-1.6892267E-3,1.0530059E-3,4.3458585E-3,8.300825E-4,-2.2330908E-4,1.8253928E-3,-3.498399E-3,-2.330052E-3,6.9023285E-4,4.084739E-4,-9.744019E-4,-1.0029146E-3,-5.8564153E-3,1.5987876E-3,-3.0833979E-3,-3.2174815E-3,-4.9935945E-5,-2.8430045E-4,1.8958257E-3,5.377059E-4,-4.6541784E-3,1.8850828E-3,-1.9871546E-3,-4.0822406E-3,6.2226347E-4,-3.415501E-5,2.0081652E-3,2.522554E-3,8.383892E-3,1.0189387E-3,3.3157098E-3,2.2710138E-4,-2.1474347E-3,-1.7702775E-3,1.4691869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,35,37,-1,39,41,-1,43,45,47,49,51,53,55,57,59,-1,61,-1,63,65,67,69,-1,71,-1,-1,-1,-1,73,75,77,-1,79,81,83,85,87,89,-1,91,93,-1,-1,-1,-1,95,97,99,101,103,-1,105,-1,-1,-1,107,109,-1,111,113,-1,115,117,-1,119,121,-1,123,125,127,-1,-1,129,131,133,135,137,-1,139,-1,-1,-1,141,-1,-1,143,-1,145,-1,-1,147,149,151,153,-1,155,157,159,161,-1,-1,-1,163,165,167,169,-1,171,-1,-1,-1,-1,-1,173,175,177,179,-1,181,-1,-1,183,185,187,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0529293E-1,1.403887E-1,1.1294177E-1,7.986389E-2,8.121893E-2,8.47446E-2,1.0507216E-1,8.006748E-2,3.298939E-2,7.1188465E-2,9.495044E-2,1.655201E-1,8.9064255E-2,2.4412112E-2,0E0,1.808612E-1,9.586089E-2,0E0,0E0,5.1090628E-2,6.148164E-2,5.720369E-2,0E0,9.2890084E-2,1.07257485E-1,0E0,6.432304E-2,2.9845482E-2,7.988993E-3,7.8319445E-2,2.7144387E-1,1.0551038E-1,1.8473434E-1,3.3103675E-2,5.5400588E-2,0E0,1.7556354E-2,0E0,4.3365467E-2,4.016562E-2,8.756985E-2,2.3638971E-2,0E0,4.4738315E-2,0E0,0E0,0E0,0E0,8.645086E-3,7.766434E-2,3.1521523E-1,0E0,6.792191E-2,1.02192536E-1,6.261356E-2,5.6898385E-2,4.6495903E-2,2.1097645E-2,0E0,6.0757015E-2,9.350048E-2,0E0,0E0,0E0,0E0,3.2658137E-2,1.8937118E-2,3.2019973E-2,5.1122367E-2,1.3922982E-2,0E0,4.7189403E-2,0E0,0E0,0E0,1.1161822E-1,1.406343E-1,0E0,1.2435579E-1,3.1477638E-2,0E0,3.1616494E-2,5.657458E-2,0E0,7.0365064E-2,2.7580336E-2,0E0,5.1745355E-2,6.31737E-2,1.1895949E-2,0E0,0E0,1.2940569E-2,2.7745463E-2,5.340444E-2,3.9320063E-2,4.2324927E-2,0E0,2.075003E-2,0E0,0E0,0E0,5.1425174E-2,0E0,0E0,3.20697E-2,0E0,6.596634E-2,0E0,0E0,7.119299E-2,4.8004247E-2,1.677131E-2,3.865251E-2,0E0,4.1433096E-2,2.3258194E-2,6.9842584E-2,5.2186877E-2,0E0,0E0,0E0,8.835118E-3,4.5205943E-2,1.9861646E-2,3.285086E-2,0E0,1.10399E-2,0E0,0E0,0E0,0E0,0E0,3.139781E-2,4.902311E-2,3.7785754E-2,3.3957463E-2,0E0,3.492883E-2,0E0,0E0,3.5039604E-2,2.5873005E-2,3.4246713E-2,2.5555953E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,38,38,39,39,40,40,41,41,43,43,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,65,65,66,66,67,67,68,68,69,69,71,71,75,75,76,76,78,78,79,79,81,81,82,82,84,84,85,85,87,87,88,88,89,89,92,92,93,93,94,94,95,95,96,96,98,98,102,102,105,105,107,107,110,110,111,111,112,112,113,113,115,115,116,116,117,117,118,118,122,122,123,123,124,124,125,125,127,127,133,133,134,134,135,135,136,136,138,138,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,36,38,-1,40,42,-1,44,46,48,50,52,54,56,58,60,-1,62,-1,64,66,68,70,-1,72,-1,-1,-1,-1,74,76,78,-1,80,82,84,86,88,90,-1,92,94,-1,-1,-1,-1,96,98,100,102,104,-1,106,-1,-1,-1,108,110,-1,112,114,-1,116,118,-1,120,122,-1,124,126,128,-1,-1,130,132,134,136,138,-1,140,-1,-1,-1,142,-1,-1,144,-1,146,-1,-1,148,150,152,154,-1,156,158,160,162,-1,-1,-1,164,166,168,170,-1,172,-1,-1,-1,-1,-1,174,176,178,180,-1,182,-1,-1,184,186,188,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,3.1066042E7,3.6600095E10,1E0,2.1827742E6,3.2475834E7,1.0828989E8,1.5561539E2,4.47682E5,7.341606E1,2.1383E4,8.230524E3,1.3676985E4,1.980198E-2,-7.93519E-3,4.0448148E2,9.457831E0,-8.9665607E-4,-5.9752245E-3,1.2E1,1.1E1,1.11E3,-1.0398164E-2,2.0718E4,1.62E2,-6.623043E-3,4.2482185E0,1E0,5.108923E6,1.2471935E7,4.0724753E2,8.848604E1,1.13E3,5.258106E7,2.090909E0,-8.3585015E-5,2.3405604E7,-6.0748667E-3,1.3298E4,4.4854636E7,8.05953E8,3.3135505E0,1.2171551E-2,4.92E2,4.5153694E-3,6.6562236E-4,-4.5985365E-3,-7.535834E-4,1.5824156E5,2.5943396E0,1.2694E4,-1.3753484E-2,4.4914685E-2,2.831341E6,3.3382E4,8.132321E0,2.3616017E11,1.0979587E8,3.6774468E-4,8.3956606E5,3.4782608E0,-9.2775915E-3,-3.342231E-3,-2.6012757E-3,2.8872392E-3,8E-3,2.3362158E1,2.7465258E1,3.1546112E-2,3.0491832E6,7.293842E-4,4.7772845E6,-4.732504E-3,2.4709178E-3,-1.00170306E-4,9.5992924E-5,1.198E5,-1.4236862E-2,1.8251354E8,1.0119178E10,-6.7927577E-3,4.07882E7,3.8E1,7.952398E-3,4.5085418E4,1E0,-8.966619E-3,1.517E3,4.36E2,1.3707545E-4,-4.088907E-3,6.9117304E-3,1.3565588E6,1E1,1.143316E6,2.3523688E0,9.433962E-3,5.459962E-4,7.8E1,2.539184E-3,7.891469E-3,-2.7798756E-3,3.5229592E0,6.1923647E-3,1.8254406E-3,7.3610186E3,-4.687805E-3,2.9446045E-1,-7.195924E-3,-8.233664E-3,1.75E2,1.2920592E0,1.8071064E7,7.5646E4,1.8436154E-3,3.5452E4,5.26E2,2.3663366E0,3.6193964E7,5.298538E-3,-2.4034958E-3,-5.3485944E-3,1.2737473E1,2.6540773E1,9.289174E2,2.001177E6,6.828932E-3,7.2565425E5,7.227272E-4,-1.0059795E-3,1.9424591E-3,-1.6841083E-3,-7.012625E-3,2.653986E6,2.34E0,4.6726016E3,2.6186154E0,-4.4296496E-3,2.42708E5,8.9667406E-4,5.571023E-3,2.6835732E3,3.565214E6,2.7132668E-2,1.3E1,6.6815974E-4,-1.1685084E-3,7.264505E-4,-2.1074966E-3,-1.6117712E-3,3.28813E-3,-2.480533E-3,-7.4153794E-3,-4.859569E-3,-1.7238614E-4,3.2633115E-3,-1.6892267E-3,1.0530059E-3,4.3458585E-3,8.300825E-4,-2.2330908E-4,1.8253928E-3,-3.498399E-3,-2.330052E-3,6.9023285E-4,4.084739E-4,-9.744019E-4,-1.0029146E-3,-5.8564153E-3,1.5987876E-3,-3.0833979E-3,-3.2174815E-3,-4.9935945E-5,-2.8430045E-4,1.8958257E-3,5.377059E-4,-4.6541784E-3,1.8850828E-3,-1.9871546E-3,-4.0822406E-3,6.2226347E-4,-3.415501E-5,2.0081652E-3,2.522554E-3,8.383892E-3,1.0189387E-3,3.3157098E-3,2.2710138E-4,-2.1474347E-3,-1.7702775E-3,1.4691869E-3],"split_indices":[2,45,5,65,28,50,45,52,9,58,9,52,33,57,0,4,54,0,0,18,3,2,0,2,8,0,39,64,32,45,55,56,2,12,56,0,48,0,9,9,7,53,0,3,0,0,0,0,33,56,29,0,42,5,9,53,31,7,0,28,56,0,0,0,0,57,58,58,38,47,0,28,0,0,0,37,7,0,7,19,0,7,3,0,33,102,0,0,3,38,0,0,28,3,9,41,57,0,3,0,0,0,54,0,0,4,0,39,0,0,10,56,45,9,0,9,2,54,29,0,0,0,54,58,55,9,0,28,0,0,0,0,0,1,54,4,42,0,10,0,0,4,47,34,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.549E3,1.924E3,6.25E2,1.702E3,2.22E2,5.88E2,3.7E1,1.689E3,1.3E1,1.99E2,2.3E1,4.36E2,1.52E2,3.2E1,5E0,4.87E2,1.202E3,6E0,7E0,1.86E2,1.3E1,1.9E1,4E0,4.16E2,2E1,4E0,1.48E2,1E1,2.2E1,4.52E2,3.5E1,9.15E2,2.87E2,4.4E1,1.42E2,5E0,8E0,6E0,1.3E1,3.1E2,1.06E2,1.6E1,4E0,1.41E2,7E0,4E0,6E0,1E1,1.2E1,4.19E2,3.3E1,5E0,3E1,8.99E2,1.6E1,2E1,2.67E2,3.5E1,9E0,1.7E1,1.25E2,4E0,4E0,8E0,5E0,2.93E2,1.7E1,1.4E1,9.2E1,1E1,6E0,1.37E2,4E0,5E0,7E0,3.25E2,9.4E1,4E0,2.9E1,2.5E1,5E0,4.5E1,8.54E2,6E0,1E1,1.4E1,6E0,2.45E2,2.2E1,1.6E1,1.9E1,4E0,1.3E1,1.1E1,1.14E2,1.04E2,1.89E2,6E0,1.1E1,6E0,8E0,5E0,8.7E1,6E0,4E0,1.33E2,4E0,3.21E2,4E0,6E0,8.8E1,2E1,9E0,1.8E1,7E0,1.5E1,3E1,7.79E2,7.5E1,5E0,5E0,6E0,8E0,2.36E2,9E0,1.8E1,4E0,1.2E1,4E0,7E0,6E0,5E0,6E0,9.6E1,1.8E1,8.9E1,1.5E1,4E0,1.85E2,4E0,7E0,1.2E1,7.5E1,7.4E1,5.9E1,2.62E2,5.9E1,3.9E1,4.9E1,7E0,1.3E1,5E0,4E0,6E0,1.2E1,7E0,8E0,8E0,2.2E1,2.17E2,5.62E2,7.1E1,4E0,4E0,4E0,1.08E2,1.28E2,4E0,5E0,1.4E1,4E0,8E0,4E0,7.6E1,2E1,7E0,1.1E1,8.3E1,6E0,5E0,1E1,1.62E2,2.3E1,8E0,4E0,6E1,1.5E1,5.4E1,2E1,6E0,5.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-3.4978133E-4,-1.0757522E-3,6.15388E-2,-8.157883E-3,4.0834453E-3,1.3652545E-1,2.2900451E-2,-9.369923E-3,4.326803E-2,1.009083E-2,-1.1941398E-2,8.440225E-3,3.5151732E-3,4.6910793E-2,-4.098268E-4,-4.1344976E-3,-2.5424086E-2,6.491632E-2,-1.8040098E-3,6.708786E-2,8.055305E-3,-8.016932E-2,-9.827717E-3,3.494713E-4,3.7186975E-3,1.11993365E-2,-1.0852886E-2,-9.238126E-3,-2.2197684E-2,9.129457E-2,1.442059E-4,8.308882E-4,8.51605E-2,3.9497565E-2,5.7131425E-3,-8.460085E-3,-7.211373E-4,2.0185465E-2,-1.416153E-2,-1.6198797E-2,1.8561335E-2,1.771995E-2,-1.7732112E-2,-1.18576586E-1,-1.6530318E-2,6.631743E-3,2.1801747E-3,6.5694717E-3,6.2375836E-2,6.7655E-3,2.3647971E-2,3.425635E-2,3.2829782E-3,3.125055E-2,-3.369444E-2,-2.9543018E-2,-3.0715007E-3,2.3837127E-2,-6.771527E-2,6.8710595E-3,1.464545E-2,-5.224649E-3,2.5938172E-2,-1.0358059E-1,-1.5328639E-2,-1.23060625E-2,-4.28645E-2,-3.7838485E-2,1.8135378E-3,3.9960053E-3,3.990293E-4,9.651421E-3,5.9294356E-3,1.2133597E-2,7.3955655E-2,1.3808304E-2,-3.991649E-3,4.624654E-2,-5.7594595E-3,-3.1493243E-3,1.1884658E-4,-2.0110883E-2,-8.093437E-2,2.6648212E-4,-4.1788947E-3,-1.2016005E-2,7.8618154E-2,-6.0903123E-3,-1.4721869E-1,3.6218293E-2,4.4187615E-3,-4.28946E-3,3.1376794E-2,-6.739621E-3,-1.6379894E-3,4.5871146E-2,-1.822712E-2,-2.9049595E-4,-3.3275806E-3,-1.7091063E-1,-2.5958797E-2,2.284468E-2,-2.1249356E-2,-3.5524708E-3,1.896197E-2,2.673518E-2,-2.7322982E-2,7.0350966E-3,4.8153915E-2,4.8516646E-2,1.0451662E-2,-4.5114502E-2,2.913746E-4,5.5254836E-2,-4.9196713E-4,-3.1187346E-3,1.3169467E-3,3.0481527E-2,-2.43724E-2,-1.1212607E-1,9.608623E-5,-5.976911E-3,3.328745E-2,-2.4353084E-3,2.8930593E-3,5.9709568E-3,8.9045794E-4,1.3179457E-3,-2.1148487E-3,-1.1080822E-2,-3.0260263E-3,7.1196485E-4,3.6734461E-3,-4.869139E-5,4.2691096E-3,7.689499E-4,3.1913794E-3,-8.499615E-4,3.6993069E-3,-1.7296333E-3,-2.3743937E-4,-1.5396567E-2,2.4864776E-5,-5.781764E-4,-4.2875377E-3,4.4694528E-4,5.7219933E-3,-3.3974878E-3,2.4066067E-5,2.0509067E-3,-6.0029136E-4,2.3607456E-3,-3.2466926E-4,-1.2560746E-4,-2.9971448E-3,4.907442E-3,1.5288643E-3,-1.4704159E-3,3.1999028E-3,-1.9338068E-3,7.351502E-4,-4.835854E-3,-1.00964E-3,6.754877E-4,-4.38436E-4,3.3015008E-3,-1.699241E-4,-1.8366637E-3,5.250117E-3,-8.9298066E-4,-4.1781897E-3,-2.1163803E-3,-7.482639E-3,2.2300445E-3,-4.950231E-4,3.0496735E-3,-4.909725E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,-1,31,33,35,37,-1,-1,39,41,-1,43,45,-1,-1,47,49,51,-1,-1,53,55,57,59,61,63,65,67,-1,-1,-1,69,-1,71,73,75,77,79,81,83,85,87,-1,89,-1,91,93,95,-1,97,99,101,-1,-1,103,-1,105,107,109,111,113,115,-1,-1,117,119,121,-1,123,125,127,129,131,133,-1,135,-1,-1,137,139,-1,-1,141,143,145,147,-1,149,151,153,-1,155,157,159,161,163,165,-1,-1,-1,167,169,171,-1,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.162534E-1,9.346126E-2,8.379431E-2,6.759743E-2,1.4259624E-1,9.551838E-3,1.6067157E-2,8.848396E-2,4.5300607E-2,1.24411896E-1,5.7473782E-2,0E0,0E0,1.2658106E-2,0E0,8.21244E-2,1.2914823E-1,3.246154E-2,0E0,3.232518E-2,7.643661E-2,6.7878045E-2,5.140396E-2,0E0,0E0,4.931499E-2,1.0920788E-1,0E0,1.3732456E-1,2.185303E-2,0E0,0E0,2.0292327E-2,1.0623943E-1,6.7171276E-2,0E0,0E0,3.0862257E-2,5.8589175E-2,1.0886463E-1,8.741976E-2,1.1124629E-1,9.064372E-2,1.2481056E-1,9.4660126E-2,0E0,0E0,0E0,1.8503964E-2,0E0,8.264154E-2,6.585025E-2,6.8583325E-2,2.357642E-2,9.673412E-3,6.817348E-2,5.4074213E-2,5.9866823E-2,1.1178948E-1,0E0,4.1095667E-2,0E0,6.4704895E-2,2.217193E-2,7.817662E-2,0E0,8.405514E-3,1.7261924E-1,6.4025536E-2,0E0,0E0,4.344952E-2,0E0,2.9536908E-2,3.978412E-2,4.2133197E-2,9.3362115E-2,1.5702955E-2,2.5299735E-2,0E0,0E0,2.7261768E-2,5.7073787E-2,4.019456E-2,0E0,5.160433E-2,2.7695514E-2,1.7586423E-2,5.116932E-2,4.8119716E-2,5.6014437E-2,0E0,4.992702E-2,0E0,0E0,3.667984E-2,9.192555E-2,0E0,0E0,2.1184856E-1,8.8559724E-2,8.689665E-2,6.403013E-2,0E0,3.5379924E-2,2.5527949E-2,1.1192919E-2,0E0,1.5451182E-2,4.0605225E-2,7.073544E-2,6.2391214E-2,5.7799555E-2,1.672452E-2,0E0,0E0,0E0,5.401661E-2,4.2413615E-2,3.5660505E-2,0E0,3.3061028E-2,3.7683666E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,62,62,63,63,64,64,66,66,67,67,68,68,71,71,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,90,90,92,92,95,95,96,96,99,99,100,100,101,101,102,102,104,104,105,105,106,106,108,108,109,109,110,110,111,111,112,112,113,113,117,117,118,118,119,119,121,121,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,-1,32,34,36,38,-1,-1,40,42,-1,44,46,-1,-1,48,50,52,-1,-1,54,56,58,60,62,64,66,68,-1,-1,-1,70,-1,72,74,76,78,80,82,84,86,88,-1,90,-1,92,94,96,-1,98,100,102,-1,-1,104,-1,106,108,110,112,114,116,-1,-1,118,120,122,-1,124,126,128,130,132,134,-1,136,-1,-1,138,140,-1,-1,142,144,146,148,-1,150,152,154,-1,156,158,160,162,164,166,-1,-1,-1,168,170,172,-1,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.5505118E2,3.449702E2,1.6907827E9,1.0711212E3,2.8091298E7,1.4689211E7,3.9712732E7,8.3306855E6,3.653602E7,3.79E2,3E0,8.440225E-3,3.5151732E-3,4.8858527E9,-4.098268E-4,1.3970588E0,5.664E1,1.923637E3,-1.8040098E-3,1.139605E6,6E0,2.4991455E9,1.1E1,3.494713E-4,3.7186975E-3,3.6E2,3.076635E3,-9.238126E-3,1.003252E5,7.57657E4,1.442059E-4,8.308882E-4,1.83E2,1.5938586E8,1.1317E4,-8.460085E-3,-7.211373E-4,2.4347392E-1,3.0379269E1,1.8083334E1,2.04E5,8.333333E0,6.2285713E1,3.3654268E0,1E0,6.631743E-3,2.1801747E-3,6.5694717E-3,7.857143E0,6.7655E-3,7.197394E1,1.753919E3,2.773036E1,4.3E1,7.6599895E9,6.03368E5,3.4867987E-1,1.0882353E1,2E0,6.8710595E-3,1E2,-5.224649E-3,2.819136E0,5.041306E3,3.833E3,-1.23060625E-2,1.6436782E0,1.2694E4,4.0058823E2,3.9960053E-3,3.990293E-4,2.0757051E-1,5.9294356E-3,8E0,1.8E1,4.433625E2,3.1547058E1,2.4079119E1,3.6E3,-3.1493243E-3,1.1884658E-4,1.72797E5,2.5269838E6,4.9386E4,-4.1788947E-3,1.1E1,6.377113E4,3.6873734E5,1.1472081E0,2.339665E6,2.991942E2,-4.28946E-3,1.6772E4,-6.739621E-3,-1.6379894E-3,1.8E1,1.86E2,-2.9049595E-4,-3.3275806E-3,1.662639E7,6.3157897E0,4E1,1.701E3,-3.5524708E-3,1.4387654E0,2.6277744E8,2.558106E10,7.0350966E-3,2.2178302E5,1.8481675E0,1.06E3,2.3279E4,5.8805E4,9.377185E-1,-4.9196713E-4,-3.1187346E-3,1.3169467E-3,1.445E3,1.5859042E6,4.3563797E4,9.608623E-5,3.3341297E1,9.3161255E-1,-2.4353084E-3,2.8930593E-3,5.9709568E-3,8.9045794E-4,1.3179457E-3,-2.1148487E-3,-1.1080822E-2,-3.0260263E-3,7.1196485E-4,3.6734461E-3,-4.869139E-5,4.2691096E-3,7.689499E-4,3.1913794E-3,-8.499615E-4,3.6993069E-3,-1.7296333E-3,-2.3743937E-4,-1.5396567E-2,2.4864776E-5,-5.781764E-4,-4.2875377E-3,4.4694528E-4,5.7219933E-3,-3.3974878E-3,2.4066067E-5,2.0509067E-3,-6.0029136E-4,2.3607456E-3,-3.2466926E-4,-1.2560746E-4,-2.9971448E-3,4.907442E-3,1.5288643E-3,-1.4704159E-3,3.1999028E-3,-1.9338068E-3,7.351502E-4,-4.835854E-3,-1.00964E-3,6.754877E-4,-4.38436E-4,3.3015008E-3,-1.699241E-4,-1.8366637E-3,5.250117E-3,-8.9298066E-4,-4.1781897E-3,-2.1163803E-3,-7.482639E-3,2.2300445E-3,-4.950231E-4,3.0496735E-3,-4.909725E-4],"split_indices":[56,52,12,55,45,1,45,45,45,29,3,0,0,5,0,53,50,4,0,31,3,12,3,0,0,2,50,0,28,28,0,0,29,7,9,0,0,38,56,52,5,52,4,58,102,0,0,0,56,0,58,52,58,8,5,9,38,52,17,0,0,0,54,50,1,0,53,29,4,0,0,41,0,8,3,4,58,56,10,0,0,1,32,2,0,3,45,28,53,12,52,0,9,0,0,3,0,0,0,45,56,8,11,0,41,12,20,0,28,54,2,29,9,27,0,0,0,2,28,28,0,56,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.585E3,2.556E3,2.9E1,1.077E3,1.479E3,9E0,2E1,1.053E3,2.4E1,1.076E3,4.03E2,5E0,4E0,1.1E1,9E0,7.95E2,2.58E2,1.9E1,5E0,3.6E1,1.04E3,1.1E1,3.92E2,5E0,6E0,2.42E2,5.53E2,4E0,2.54E2,1.3E1,6E0,1E1,2.6E1,7.1E1,9.69E2,4E0,7E0,4.9E1,3.43E2,5.1E1,1.91E2,1.07E2,4.46E2,1.3E1,2.41E2,6E0,7E0,7E0,1.9E1,9E0,6.2E1,7.5E1,8.94E2,4.1E1,8E0,1.43E2,2E2,2.9E1,2.2E1,5E0,1.86E2,6E0,1.01E2,1.1E1,4.35E2,4E0,9E0,1.11E2,1.3E2,1.4E1,5E0,5.5E1,7E0,4.9E1,2.6E1,3.65E2,5.29E2,2.9E1,1.2E1,4E0,4E0,1.22E2,2.1E1,1.93E2,7E0,1.8E1,1.1E1,1.3E1,9E0,5.9E1,1.27E2,4E0,9.7E1,7E0,4E0,1.9E1,4.16E2,4E0,5E0,8E0,1.03E2,6.8E1,6.2E1,5E0,5E1,3.6E1,1.3E1,6E0,2E1,3.1E1,3.34E2,4.9E1,4.8E2,2.5E1,4E0,4E0,8E0,9E0,1.13E2,1.5E1,6E0,1.63E2,3E1,1.2E1,6E0,6E0,5E0,7E0,6E0,4E0,5E0,3.8E1,2.1E1,1.2E2,7E0,6.6E1,3.1E1,6E0,1.3E1,1.87E2,2.29E2,4E0,4E0,8.4E1,1.9E1,6E1,8E0,1.9E1,4.3E1,2.9E1,2.1E1,2.2E1,1.4E1,8E0,5E0,4E0,1.6E1,5E0,2.6E1,2.6E1,3.08E2,1.5E1,3.4E1,1.95E2,2.85E2,2.1E1,4E0,5E0,4E0,1.03E2,1E1,6E0,9E0,1.1E1,1.52E2,1.8E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-2.3933945E-3,-5.9263664E-1,2.6889783E-1,-9.1858613E-1,-2.1448609E-1,4.9850196E-2,7.607807E-1,-9.524582E-1,1.082049E-1,-4.2231518E-1,3.841283E-1,-2.0045005E-1,2.5073895E-1,5.913415E-1,1.2293434E0,-1.190826E0,-6.499661E-1,-5.2304554E-1,-1.684352E-1,1.654881E-1,7.682251E-1,-6.403749E-1,-1.0299838E-1,2.2019845E-1,1.7205881E0,3.6116946E-1,8.6512417E-1,9.544972E-1,1.447596E0,-1.0850258E0,-1.5838625E0,-3.0255994E-1,-1.0274159E0,-6.542544E-1,-3.6114734E-1,2.1063672E-2,-3.827689E-1,-1.6058318E-1,3.2476264E-1,1.0358812E0,6.4064044E-1,-6.7629725E-1,1.9147405E-2,-3.798304E-1,3.208436E-3,2.7569512E-1,-5.907691E-1,9.582441E-2,3.9070837E-2,4.5508566E-1,-3.4440452E-1,6.501266E-1,1.1846459E0,1.0611738E0,4.215443E-1,9.425399E-3,1.4771441E0,-6.6020197E-1,-1.1486143E0,-8.24079E-2,-5.5061117E-2,4.055765E-2,-3.4508678E-1,-1.3218734E0,-6.474342E-1,-7.689773E-1,-5.070937E-1,-4.139987E-1,-2.1466441E-1,-6.077923E-2,2.2719845E-1,-4.9959147E-1,1.3236758E-1,-3.058238E-1,-1.4064024E-2,1.03090085E-1,4.891102E-1,5.986518E-2,1.9258194E-2,1.481983E-2,7.018445E-1,-6.000221E-1,-8.9893275E-1,-4.98677E-1,-2.3440248E-1,7.2169036E-2,-5.789142E-2,6.985793E-2,4.658221E-1,-7.8216726E-1,-2.4077396E-1,2.9188117E-1,6.169904E-1,-2.985779E-2,-1.7024064E-1,7.72275E-1,-1.2616562E-3,1.0792186E0,7.851262E-2,8.0599916E-1,1.2145987E0,1.318254E-2,2.8760185E-2,5.3556785E-2,1.5661538E0,-3.5902992E-2,-4.7232355E-3,-5.6290977E-2,-2.8847083E-2,-2.3156878E-2,-3.9114323E-3,-5.3385925E-2,-9.192074E-2,-1.2863264E-2,-3.9320957E-2,-3.7759576E-2,-6.2052947E-3,-2.586171E-2,-4.2914865E-3,-2.1443246E-2,-1.0495426E-2,-3.7772711E-3,-1.5231074E-2,-6.4352313E-3,1.1099501E-2,2.5020547E-3,1.4326472E-2,-3.558328E-2,-1.7009383E-2,-3.764246E-3,1.3154044E-2,-1.3399529E-3,-1.7454095E-2,7.5460942E-3,-4.2518647E-3,-1.3406342E-3,1.0195194E-2,4.4437973E-3,2.4880974E-2,1.3480483E-2,3.6080826E-2,-3.3894286E-2,-1.9205628E-2,-4.6135005E-2,-7.1078218E-3,-2.7161289E-2,-1.3822419E-2,9.33951E-3,-1.5613237E-2,-5.1571312E-3,1.618328E-2,-7.5569376E-3,8.243618E-3,3.0937996E-2,1.4483788E-2,-3.968076E-2,-5.711906E-3,-1.3762629E-2,3.4719768E-3,5.2895495E-3,2.007394E-2,1.8349549E-2,3.8173135E-2,-1.20216245E-2,-1.8143258E-3,3.894546E-2,1.0547749E-2,-3.2969955E-2,1.0913452E-2,5.2875895E-2,1.022835E-2,4.322535E-2,1.2377302E-2,3.060715E-2,6.0913827E-2,7.693966E-2,3.8323175E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,-1,-1,91,93,95,97,99,101,-1,103,105,107,-1,-1,-1,109,111,113,115,117,119,121,123,125,127,129,131,133,135,137,-1,-1,-1,139,141,143,145,147,-1,149,151,153,155,157,159,161,-1,163,165,167,169,-1,171,173,-1,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2257657E2,1.0223035E2,1.9476643E2,5.0161987E1,4.8184288E1,6.300396E1,4.3494873E1,3.109616E1,0E0,7.2514725E0,8.270768E0,2.3856287E1,3.0819637E1,2.5587585E1,7.4362183E0,8.510925E0,2.5521439E1,4.1184235E0,3.3791897E0,3.4279172E0,5.6572723E-1,4.0443726E0,1.3467082E1,3.085696E1,1.9182358E0,1.506632E1,1.207605E1,3.4187164E0,2.6659546E0,4.563431E0,9.930954E-1,5.375991E0,9.66169E0,1.5009193E0,6.3885975E-1,7.7298903E-1,2.4131331E0,4.6387058E-1,1.5525565E0,8.5456944E-1,3.3010578E-1,1.1248093E0,0E0,2.078125E0,3.059103E1,2.4974857E1,2.7591639E0,0E0,0E0,5.078205E0,1.2060947E0,9.0715065E0,1.7786636E0,1.21344E0,1.7507887E-1,0E0,7.3979187E-1,1.3331614E0,1.7802887E0,0E0,0E0,0E0,3.6439438E0,4.2959213E0,2.607813E0,6.575699E-1,6.663475E-1,3.4912968E-1,3.5392582E-1,7.416297E-1,1.4801759E-1,8.610544E-1,2.6175827E-1,1.6900599E-1,1.6788095E-1,2.938071E-1,2.5327396E-1,0E0,0E0,0E0,2.647257E-1,1.345295E0,9.688587E-1,8.287754E-1,2.3847637E0,0E0,6.3433666E0,7.198537E0,9.579071E0,7.625027E-1,2.6902348E-1,2.2860737E0,3.8638573E0,0E0,1.829462E-1,1.9906616E0,3.1487076E0,1.1210556E0,0E0,1.0359983E0,2.7605438E-1,0E0,0E0,0E0,5.718994E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,80,80,81,81,82,82,83,83,84,84,86,86,87,87,88,88,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,99,99,100,100,104,104],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,-1,-1,92,94,96,98,100,102,-1,104,106,108,-1,-1,-1,110,112,114,116,118,120,122,124,126,128,130,132,134,136,138,-1,-1,-1,140,142,144,146,148,-1,150,152,154,156,158,160,162,-1,164,166,168,170,-1,172,174,-1,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.14099E5,5.549173E6,2.2178302E5,1.8137958E6,3.386E3,1.082049E-1,3.3926086E2,2.496172E3,8.2E1,3.7002478E3,1.3664E4,5.964E3,4.0844156E7,6.747114E7,1.913925E-1,1.3304372E7,1.8140173E-1,5.015707E0,1.5427417E3,6.76E2,3.0820766E-1,1.7324902E8,3.9712732E7,1.5103707E0,1E0,1.545974E-1,2.461362E-3,9.6771875E2,2.66E2,1.0052E4,1.8122449E2,2.7538432E8,5.5871747E2,6.306E3,7.0014185E6,3.647E3,2.9956698E-1,1.3137E4,1.4504054E8,1.9147405E-2,7.2896E5,2.255481E6,9.52E2,2.6476662E6,9.582441E-2,3.9070837E-2,2.5682794E5,1.4813794E3,4.2620764E2,1.386749E1,2.9251662E6,1.3170732E0,9.425399E-3,2.3920168E6,2.701189E8,5.8426323E0,-8.24079E-2,-5.5061117E-2,4.055765E-2,6.316106E6,9.222717E-1,1.272E3,1.1994927E7,8.982707E9,5.426937E5,1E1,6.4860556E5,2.028994E8,1.221875E1,3.3E1,1.06158945E5,5.911314E1,2.4513194E5,2.028994E8,5.986518E-2,1.9258194E-2,1.481983E-2,5.7064955E6,5.222222E1,1E0,2.2247495E-1,3.28825E0,7.2169036E-2,8.426E3,4.97E2,3.71E0,1E0,3.989822E1,1.2939234E5,8.8543115E2,-2.985779E-2,3.4540886E3,6.971004E7,2.222456E0,6.13912E5,7.851262E-2,1.766486E7,1.460806E6,1.318254E-2,2.8760185E-2,5.3556785E-2,6.292039E0,-3.5902992E-2,-4.7232355E-3,-5.6290977E-2,-2.8847083E-2,-2.3156878E-2,-3.9114323E-3,-5.3385925E-2,-9.192074E-2,-1.2863264E-2,-3.9320957E-2,-3.7759576E-2,-6.2052947E-3,-2.586171E-2,-4.2914865E-3,-2.1443246E-2,-1.0495426E-2,-3.7772711E-3,-1.5231074E-2,-6.4352313E-3,1.1099501E-2,2.5020547E-3,1.4326472E-2,-3.558328E-2,-1.7009383E-2,-3.764246E-3,1.3154044E-2,-1.3399529E-3,-1.7454095E-2,7.5460942E-3,-4.2518647E-3,-1.3406342E-3,1.0195194E-2,4.4437973E-3,2.4880974E-2,1.3480483E-2,3.6080826E-2,-3.3894286E-2,-1.9205628E-2,-4.6135005E-2,-7.1078218E-3,-2.7161289E-2,-1.3822419E-2,9.33951E-3,-1.5613237E-2,-5.1571312E-3,1.618328E-2,-7.5569376E-3,8.243618E-3,3.0937996E-2,1.4483788E-2,-3.968076E-2,-5.711906E-3,-1.3762629E-2,3.4719768E-3,5.2895495E-3,2.007394E-2,1.8349549E-2,3.8173135E-2,-1.20216245E-2,-1.8143258E-3,3.894546E-2,1.0547749E-2,-3.2969955E-2,1.0913452E-2,5.2875895E-2,1.022835E-2,4.322535E-2,1.2377302E-2,3.060715E-2,6.0913827E-2,7.693966E-2,3.8323175E-2],"split_indices":[20,102,12,2,51,28,28,2,0,52,4,29,52,2,2,45,45,41,45,38,54,4,2,27,7,45,41,6,38,27,4,0,2,52,7,52,2,51,2,38,2,31,0,9,31,2,32,0,0,28,4,58,35,28,53,0,28,12,53,0,0,0,1,27,0,9,5,47,3,33,7,58,3,33,58,28,7,0,0,0,32,4,68,38,54,0,2,2,54,74,58,33,52,0,4,45,54,2,0,47,47,0,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.637E3,8.3E2,1.807E3,4.45E2,3.85E2,1.251E3,5.56E2,4.41E2,4E0,2.86E2,9.9E1,5.57E2,6.94E2,4.1E2,1.46E2,2.45E2,1.96E2,2.04E2,8.2E1,6.4E1,3.5E1,1E2,4.57E2,6.81E2,1.3E1,2.24E2,1.86E2,6.7E1,7.9E1,1.96E2,4.9E1,1.03E2,9.3E1,1.11E2,9.3E1,4.4E1,3.8E1,2.1E1,4.3E1,9E0,2.6E1,9.7E1,3E0,1.26E2,3.31E2,6.38E2,4.3E1,9E0,4E0,1.98E2,2.6E1,1.13E2,7.3E1,5.5E1,1.2E1,2E0,7.7E1,2.7E1,1.69E2,3.5E1,1.4E1,3E0,1E2,5.1E1,4.2E1,6E1,5.1E1,6.7E1,2.6E1,3.2E1,1.2E1,3.1E1,7E0,1E1,1.1E1,1.9E1,2.4E1,6E0,3E0,5E0,2.1E1,7.5E1,2.2E1,6.8E1,5.8E1,1.2E1,3.19E2,3.07E2,3.31E2,2.7E1,1.6E1,1E2,9.8E1,9E0,1.7E1,9.5E1,1.8E1,6.2E1,1.1E1,2.3E1,3.2E1,8E0,4E0,1.8E1,5.9E1,2.3E1,4E0,1.6E2,9E0,6.5E1,3.5E1,4E1,1.1E1,1.4E1,2.8E1,5.8E1,2E0,4.7E1,4E0,5.6E1,1.1E1,1.2E1,1.4E1,2.6E1,6E0,4E0,8E0,1E1,2.1E1,3E0,4E0,2E0,8E0,3E0,8E0,9E0,1E1,2E0,2.2E1,3E0,1.8E1,4.7E1,2.8E1,2E1,2E0,5E1,1.8E1,1E1,4.8E1,2.84E2,3.5E1,9.5E1,2.12E2,1.56E2,1.75E2,2.5E1,2E0,1.4E1,2E0,4.2E1,5.8E1,4.4E1,5.4E1,1E1,7E0,8.8E1,7E0,4E0,1.4E1,6E1,2E0,1.9E1,4E0,4E0,2.8E1,5.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-2.3714243E-3,-5.6937283E-1,2.632269E-1,-8.611649E-1,-2.1411869E-1,5.843058E-2,7.091735E-1,-9.001495E-1,1.0970145E-1,-4.0091532E-1,3.5433906E-1,-1.7773639E-1,2.4572168E-1,8.125841E-1,-2.1678058E-2,-1.1528195E0,-5.955599E-1,-4.9700186E-1,-1.660928E-1,1.5033236E-1,6.9017434E-1,-4.293215E-1,-1.3976596E-2,3.0475536E-1,-5.242693E-1,6.9015497E-1,1.2918309E0,-3.9108118E-1,5.0852233E-1,-1.047478E0,-1.5900015E0,-2.785633E-1,-9.6850884E-1,-6.040674E-1,-3.522944E-1,-5.0136417E-2,-1.10359766E-1,-1.9317546E-1,3.103282E-1,8.4424776E-1,4.6831298E-1,-6.551769E-1,-3.1109655E-1,1.366152E0,-7.375315E-2,7.9924315E-2,5.056863E-1,-6.0004544E-1,2.3809645E-2,4.9802318E-1,9.267325E-1,1.3728242E0,4.1750902E-1,-5.172177E-1,1.4290325E-1,6.738396E-1,-2.6505345E-1,-1.1137598E0,-6.214992E-1,-8.415815E-2,-5.3442948E-2,-3.4109232E-1,6.082151E-1,-1.257475E0,-5.9185696E-1,-7.346779E-1,-4.6774274E-1,-4.6425366E-1,-2.6155004E-1,2.4459746E-2,-2.3934168E-1,-2.3751712E-1,6.5737586E-3,1.5395092E-3,4.4827098E-1,5.2593436E-2,3.146254E-2,2.4698006E-2,3.600609E-3,-5.8996284E-1,-6.88253E-2,-6.0252374E-1,-2.3930793E-1,2.1151744E-2,7.104212E-2,-1.564757E-1,2.7012524E-1,-2.5477833E-1,1.5236986E-1,4.733558E-1,8.602649E-2,-3.6391646E-1,-3.977876E-2,2.5720194E-1,6.832873E-1,8.212044E-1,1.2729123E0,1.4012694E0,2.9624726E-3,2.8644571E-2,1.2195575E-2,-6.585593E-1,-2.8485823E-1,-3.0871911E-3,2.0029142E-2,6.137451E-3,8.038528E-1,2.440875E-3,-2.0870985E-2,-5.3829897E-2,-6.897391E-3,-3.3818252E-2,-2.7286527E-3,3.9100908E-2,-1.8272914E-2,2.4593093E-3,3.7607342E-2,-4.8421297E-2,-7.940924E-2,-8.505313E-4,-3.2663096E-2,-3.6118884E-2,-6.0071778E-3,-2.3973383E-2,-7.0663393E-3,-2.3054577E-2,-4.693853E-3,-1.621451E-2,-6.3083717E-3,-6.4284313E-3,5.80315E-3,-1.5587123E-2,1.13891335E-2,-2.1632304E-3,-1.475134E-2,1.2172776E-2,2.767851E-2,-2.8860107E-2,-5.0891167E-3,-2.229037E-2,-4.2480927E-2,-1.0742514E-3,-1.453044E-2,-5.429632E-3,-3.5057046E-2,8.080705E-3,2.9354332E-2,-7.101869E-3,-1.7226916E-2,7.898379E-3,-4.544447E-2,1.831193E-2,3.5269514E-2,-3.2701466E-2,-8.317099E-3,3.5670513E-4,1.553319E-2,1.7630273E-2,4.005947E-2,4.3862082E-2,2.4472235E-2,3.050482E-2,6.543778E-2,3.5831463E-2,6.891499E-2,-1.0401175E-2,-3.4483846E-2,-1.8825023E-2,-5.029285E-3,2.8034959E-2,5.8235735E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,87,89,91,-1,93,95,97,99,101,103,105,107,109,111,-1,-1,113,115,117,119,121,123,125,127,129,131,133,-1,-1,135,-1,-1,-1,-1,137,-1,139,141,-1,-1,143,145,147,149,151,-1,153,-1,155,157,159,161,163,-1,-1,-1,165,167,-1,-1,-1,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.026893E2,8.8237885E1,1.6626724E2,5.9566467E1,4.112986E1,5.528941E1,4.33237E1,3.4946228E1,0E0,6.483368E0,6.4734583E0,2.277734E1,3.1842194E1,2.8562683E1,1.4300626E1,9.939972E0,2.4793594E1,2.9787827E0,4.106191E0,3.412496E0,8.352127E-1,5.617153E0,2.7820534E1,2.923331E1,4.168356E0,1.781099E1,6.579071E0,2.9701748E0,4.0151677E0,5.1241455E0,1.766571E0,6.6395864E0,9.81321E0,1.7395973E0,7.929449E-1,0E0,1.4317751E0,3.2564998E-1,1.5972629E0,2.2972393E-1,2.7804232E-1,2.9257603E0,2.8893795E0,8.626785E-1,9.230486E0,7.482868E0,1.3420235E1,2.2566376E0,0E0,9.818138E0,5.493866E0,3.3440552E0,1.418258E-1,9.6187687E-1,5.407983E-1,1.6706333E0,3.459807E-1,1.5991211E0,1.3935089E0,0E0,0E0,5.416959E0,7.4846697E-1,3.6544037E0,2.2658472E0,5.9511185E-1,5.5712414E-1,1.9774723E-1,4.7538614E-1,6.4553094E-1,1.8035471E0,2.3019171E-1,0E0,0E0,5.578661E-1,0E0,0E0,0E0,0E0,3.6473846E-1,0E0,6.7917824E-1,1.6336422E0,0E0,0E0,6.391383E0,2.0864224E0,5.612302E-1,3.6548967E0,7.7378616E0,0E0,1.3982625E0,0E0,1.6578064E0,5.775181E0,3.6436539E0,1.5078278E0,1.1416321E0,0E0,0E0,0E0,4.136114E-1,2.506739E-1,0E0,0E0,0E0,1.0889463E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,74,74,79,79,81,81,82,82,85,85,86,86,87,87,88,88,89,89,91,91,93,93,94,94,95,95,96,96,97,97,101,101,102,102,106,106],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,88,90,92,-1,94,96,98,100,102,104,106,108,110,112,-1,-1,114,116,118,120,122,124,126,128,130,132,134,-1,-1,136,-1,-1,-1,-1,138,-1,140,142,-1,-1,144,146,148,150,152,-1,154,-1,156,158,160,162,164,-1,-1,-1,166,168,-1,-1,-1,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.14099E5,5.549173E6,2.1592189E5,3.9712732E7,3.386E3,1.0970145E-1,3.3926086E2,1.2060912E3,9.99E2,3.0820766E-1,2.9251662E6,3.970405E3,4.8297736E7,6.747114E7,1.6659623E-1,5.2143492E-5,1.4126303E0,3.1E1,3.439199E7,8E0,9.52E2,3.1998687E3,1.2679E4,1E0,7.418546E1,2.2576077E10,1.5E1,9.6771875E2,7.3610186E3,1.1879E4,1.8122449E2,2.1102592E8,-5.0136417E-2,1.1884672E7,1.0323588E3,2.091623E5,5.015707E0,3.2829E4,4E0,1.7E1,2.127E3,1.2817779E7,3.79E2,1.0409E4,4.2343444E7,2.3809645E-2,3.202814E5,6.082581E3,9.256843E3,2.84E2,2.28125E0,1.3664E4,7.3610186E3,2.14099E5,9.12179E6,9.7637427E-1,-8.415815E-2,-5.3442948E-2,2.66E2,8.06E2,1.0828989E8,7.6599895E9,1.1994927E7,3.906E3,5.1100562E5,1.2690893E0,2.612E3,6.624E3,7.44393E5,6.5737586E-3,1.5395092E-3,3.904E3,5.2593436E-2,3.146254E-2,2.4698006E-2,3.600609E-3,9.213145E0,-6.88253E-2,1.00033E5,1.3797468E0,2.1151744E-2,7.104212E-2,1E0,1.2502964E9,3.824611E0,5.432133E0,2.2214102E6,8.602649E-2,9.4168E4,-3.977876E-2,4.423E3,3.596E3,2.3058404E7,5.0279167E1,4.6253732E2,2.9624726E-3,2.8644571E-2,1.2195575E-2,2.652E3,3.474851E8,-3.0871911E-3,2.0029142E-2,6.137451E-3,2.3828971E-1,2.440875E-3,-2.0870985E-2,-5.3829897E-2,-6.897391E-3,-3.3818252E-2,-2.7286527E-3,3.9100908E-2,-1.8272914E-2,2.4593093E-3,3.7607342E-2,-4.8421297E-2,-7.940924E-2,-8.505313E-4,-3.2663096E-2,-3.6118884E-2,-6.0071778E-3,-2.3973383E-2,-7.0663393E-3,-2.3054577E-2,-4.693853E-3,-1.621451E-2,-6.3083717E-3,-6.4284313E-3,5.80315E-3,-1.5587123E-2,1.13891335E-2,-2.1632304E-3,-1.475134E-2,1.2172776E-2,2.767851E-2,-2.8860107E-2,-5.0891167E-3,-2.229037E-2,-4.2480927E-2,-1.0742514E-3,-1.453044E-2,-5.429632E-3,-3.5057046E-2,8.080705E-3,2.9354332E-2,-7.101869E-3,-1.7226916E-2,7.898379E-3,-4.544447E-2,1.831193E-2,3.5269514E-2,-3.2701466E-2,-8.317099E-3,3.5670513E-4,1.553319E-2,1.7630273E-2,4.005947E-2,4.3862082E-2,2.4472235E-2,3.050482E-2,6.543778E-2,3.5831463E-2,6.891499E-2,-1.0401175E-2,-3.4483846E-2,-1.8825023E-2,-5.029285E-3,2.8034959E-2,5.8235735E-2],"split_indices":[20,102,12,2,51,28,45,2,0,52,52,2,27,28,52,45,45,39,42,41,3,7,18,2,4,2,6,56,12,3,4,4,2,52,7,0,45,52,28,54,9,6,0,1,9,2,2,45,0,28,52,4,0,54,2,4,2,9,42,0,0,0,0,45,5,9,0,47,42,2,2,1,0,0,2,0,0,0,0,53,0,5,53,0,0,14,7,54,53,28,0,1,0,2,2,45,56,4,0,0,0,10,7,0,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.672E3,8.52E2,1.82E3,4.67E2,3.85E2,1.248E3,5.72E2,4.62E2,5E0,2.9E2,9.5E1,5.52E2,6.96E2,5.01E2,7.1E1,2.51E2,2.11E2,2.05E2,8.5E1,6E1,3.5E1,2.17E2,3.35E2,6.47E2,4.9E1,4.01E2,1E2,4.2E1,2.9E1,2.05E2,4.6E1,1.15E2,9.6E1,1.16E2,8.9E1,4E0,8.1E1,1.9E1,4.1E1,1.9E1,1.6E1,7.3E1,1.44E2,1.3E1,3.22E2,3.06E2,3.41E2,4.6E1,3E0,2.23E2,1.78E2,9.1E1,9E0,3.4E1,8E0,2.4E1,5E0,1.76E2,2.9E1,3.2E1,1.4E1,1.08E2,7E0,5.3E1,4.3E1,5.7E1,5.9E1,3.8E1,5.1E1,4E1,4.1E1,1.7E1,2E0,1.4E1,2.7E1,6E0,1.3E1,1.4E1,2E0,6.9E1,4E0,2.7E1,1.17E2,2E0,1.1E1,2.6E2,6.2E1,5.4E1,2.52E2,3.34E2,7E0,2.4E1,2.2E1,9.8E1,1.25E2,1.39E2,3.9E1,8.9E1,2E0,3E0,6E0,2E1,1.4E1,5E0,3E0,5E0,1.9E1,2E0,3E0,1.74E2,2E0,2.5E1,4E0,3E0,1.05E2,2E0,5E0,3.5E1,1.8E1,6E0,3.7E1,5.5E1,2E0,5.3E1,6E0,3.6E1,2E0,3.1E1,2E1,1.5E1,2.5E1,3.5E1,6E0,5E0,1.2E1,1.2E1,1.5E1,6.7E1,2E0,2E1,7E0,2.7E1,9E1,2.43E2,1.7E1,4.9E1,1.3E1,2.8E1,2.6E1,2.5E2,2E0,2.5E2,8.4E1,8E0,1.6E1,2.1E1,7.7E1,4.2E1,8.3E1,1.05E2,3.4E1,6E0,3.3E1,6E0,8.3E1,3E0,1.7E1,8E0,6E0,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[1.1071307E-3,-5.423456E-1,2.477801E-1,-8.41369E-1,-2.0129503E-1,1.2935925E-2,6.322977E-1,-1.017471E0,-3.7494534E-1,-4.0880737E-1,2.7471045E-1,-2.020943E-1,2.133659E-1,-4.2133304E-1,6.871309E-1,-8.673482E-1,-1.3602864E0,9.552879E-2,-4.5081005E-1,-5.1557684E-1,-2.1397609E-1,1.659792E-1,8.755924E-1,-3.7122184E-1,-2.086004E-3,2.631779E-1,-4.971364E-1,-1.14391066E-1,-5.080421E-1,4.3807676E-1,8.842364E-1,-9.792807E-1,-1.8303722E-1,-7.906136E-2,-1.0933759E0,-1.8364438E-1,-6.817106E-1,-6.383278E-1,-3.5833102E-1,-4.4184007E-2,-1.7400676E-1,5.937926E-2,4.08212E-1,4.4052668E-2,1.6647978E-2,-6.254603E-1,-2.617054E-1,7.01121E-2,-4.3626767E-2,9.80237E-2,4.5375454E-1,-6.410622E-1,2.5239038E-1,-1.3428412E-2,3.5675636E-3,-5.421011E-1,-2.6277422E-3,5.6580454E-1,2.8934479E-2,5.7499295E-1,1.0716008E0,-1.0271367E0,-5.5844575E-1,8.560279E-2,-4.738174E-1,-8.5298234E-1,-1.4473763E0,-2.5177908E-1,3.7454486E-2,-8.6696786E-1,-3.957296E-1,-7.214756E-1,-4.5520848E-1,-4.507801E-1,-2.412056E-1,-3.286371E-1,-8.4966786E-2,-1.4040019E-1,2.2271283E-1,2.6220378E-1,2.8265363E-2,-5.502463E-1,-4.5966916E-2,-2.0271505E-1,-5.411945E-1,-9.6603416E-2,4.7651988E-1,-2.3675628E-1,1.6351043E-1,3.493417E-1,8.2898444E-1,-6.9881845E-1,-8.517995E-3,-7.0675695E-4,2.9381186E-2,-2.7989402E-2,-8.317738E-3,3.3609986E-1,7.483398E-1,-7.860974E-1,1.3036136E-1,4.3768713E-1,9.2287767E-1,1.2026979E0,7.386259E-1,-5.167524E-2,-3.7510544E-2,-3.2135352E-2,-7.012149E-3,3.089976E-2,-1.1816963E-3,6.715307E-4,-2.6076894E-2,-4.425726E-2,-1.2234967E-2,-7.9474166E-2,-3.236669E-2,-3.444067E-3,-1.8337598E-2,-3.3463713E-2,-6.072954E-2,-1.3819378E-2,-4.500773E-2,-3.6008183E-2,-2.294122E-2,-2.9578067E-3,-2.4297133E-2,-5.169944E-4,-2.260335E-2,-2.353277E-3,-1.400775E-2,-4.175997E-3,-2.0382361E-2,-1.0091006E-2,2.084057E-3,-1.122155E-2,4.743048E-3,1.3938326E-2,7.7124435E-4,-9.910276E-4,1.515929E-2,-3.2590084E-2,-1.3340968E-2,-1.3620337E-2,1.8964648E-3,-3.496037E-2,-2.0541077E-2,-3.8837392E-3,-4.468688E-2,-9.579503E-3,2.6362702E-2,-3.6163644E-3,-1.46387955E-2,4.6837595E-3,1.9107463E-2,1.4221742E-2,3.8845684E-2,2.584793E-2,4.380022E-2,-1.3829837E-2,-3.5200674E-2,1.6466076E-3,1.9209769E-2,4.8949435E-2,2.8015452E-2,-4.0800646E-3,-4.8155468E-2,-1.5659416E-4,1.3060679E-2,-1.3518512E-2,2.3233196E-2,4.808888E-2,-1.878078E-3,6.864334E-2,5.0247166E-2,3.8324706E-2,-1.0213288E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,75,77,79,-1,-1,81,83,-1,85,87,89,91,93,-1,-1,95,-1,97,99,101,103,105,107,109,111,113,115,117,-1,119,121,123,125,127,129,131,133,135,137,139,-1,141,-1,143,145,147,149,151,153,155,157,159,-1,-1,-1,-1,-1,161,163,165,167,169,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5725714E2,8.467975E1,1.655149E2,3.6005707E1,3.863533E1,4.9132317E1,4.0442444E1,1.53402405E1,2.3011557E1,5.5478325E0,7.629492E0,1.8604027E1,2.1011847E1,8.7339735E-1,3.2061096E1,1.7131592E1,5.6408844E0,0E0,7.2774506E0,3.149128E0,2.6151524E0,2.6096046E0,1.0094166E-1,8.13361E0,1.54722395E1,1.7329464E1,4.368972E0,3.0309436E-1,3.8698864E-1,1.5347729E1,2.0706177E1,3.3334808E0,2.6183615E0,0E0,3.2756844E0,3.9474967E0,3.047638E0,1.1485023E0,7.466917E-1,0E0,1.266295E0,2.3806784E0,6.6900444E-1,0E0,0E0,1.7101631E0,3.3525963E0,0E0,6.852273E0,6.5500345E0,9.740181E0,7.5854015E-1,6.843557E-1,0E0,0E0,2.8850317E-1,0E0,9.140625E0,5.971773E0,6.3311577E0,9.064331E0,1.344223E0,9.2215633E-1,1.1267533E0,5.5942464E-1,1.2093525E0,2.1515617E0,1.2460558E0,0E0,1.6779404E0,1.3031058E0,1.1903381E-1,6.456218E-1,3.959837E-1,3.5011506E-1,7.588725E-1,9.9468243E-1,7.65288E-1,5.600376E-1,2.9861236E-1,0E0,2.4646645E0,0E0,3.4753609E0,4.5005894E-1,2.8429904E0,1.2609515E0,5.2141476E-1,3.8136616E0,4.645075E0,8.682747E-1,2.1825981E-1,0E0,0E0,0E0,0E0,0E0,1.9675417E0,4.947426E0,1.1447501E0,1.2183943E0,3.6552372E0,3.0885353E0,4.2310486E0,3.1550903E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,45,45,46,46,48,48,49,49,50,50,51,51,52,52,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,76,78,80,-1,-1,82,84,-1,86,88,90,92,94,-1,-1,96,-1,98,100,102,104,106,108,110,112,114,116,118,-1,120,122,124,126,128,130,132,134,136,138,140,-1,142,-1,144,146,148,150,152,154,156,158,160,-1,-1,-1,-1,-1,162,164,166,168,170,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.781059E4,1.983922E3,4.7931265E6,2.2178302E5,5.836513E3,3.9712732E7,1.4E1,2.728889E2,1.91E4,2.914E3,4.898012E-1,2.1506184E7,1.1410706E3,3.386E3,1.0443182E3,9.552879E-2,6.747114E7,1.7751849E-1,5.2143492E-5,4.7519747E5,3.07125E0,9.6E1,1.7E1,9.99E2,2.5422776E3,1.4E2,1E0,3.8398104E0,2.3421426E-1,2.1E1,2.1515152E0,-7.906136E-2,1.775894E10,1.5137865E2,3.0497742E3,1.998E3,2.264894E8,-4.4184007E-2,2.822937E5,1.7427321E-1,1.8835404E3,4.4052668E-2,1.6647978E-2,1.3597637E8,1.044E1,7.01121E-2,1.4777102E9,3.97E2,2.3314082E6,3.2177323E-3,8.426E3,-1.3428412E-2,3.5675636E-3,5.964E3,-2.6277422E-3,2.4434292E7,2.960909E2,5.57E4,8.5E1,2.5604828E2,6.3E0,2.66E2,1.0508E5,1.1879E4,8.289561E8,2.8839298E7,3.7454486E-2,1.2427474E8,2.12E2,2.4859156E1,2.8169732E10,1.59E2,1.822542E0,1.3062E4,7.59093E5,8.631E3,1.4187837E-1,6.143686E6,2.8265363E-2,4.3875E1,-4.5966916E-2,8.61E2,4.0911578E2,5.388794E6,2.142857E0,3.5791788E0,3.321532E6,9.908038E1,5.3038636E7,3.051499E5,-8.517995E-3,-7.0675695E-4,2.9381186E-2,-2.7989402E-2,-8.317738E-3,2.3161087E2,7E0,2.5672606E8,5.441E3,1.4972717E-2,7.218466E7,1.4307411E7,3.7284137E2,-5.167524E-2,-3.7510544E-2,-3.2135352E-2,-7.012149E-3,3.089976E-2,-1.1816963E-3,6.715307E-4,-2.6076894E-2,-4.425726E-2,-1.2234967E-2,-7.9474166E-2,-3.236669E-2,-3.444067E-3,-1.8337598E-2,-3.3463713E-2,-6.072954E-2,-1.3819378E-2,-4.500773E-2,-3.6008183E-2,-2.294122E-2,-2.9578067E-3,-2.4297133E-2,-5.169944E-4,-2.260335E-2,-2.353277E-3,-1.400775E-2,-4.175997E-3,-2.0382361E-2,-1.0091006E-2,2.084057E-3,-1.122155E-2,4.743048E-3,1.3938326E-2,7.7124435E-4,-9.910276E-4,1.515929E-2,-3.2590084E-2,-1.3340968E-2,-1.3620337E-2,1.8964648E-3,-3.496037E-2,-2.0541077E-2,-3.8837392E-3,-4.468688E-2,-9.579503E-3,2.6362702E-2,-3.6163644E-3,-1.46387955E-2,4.6837595E-3,1.9107463E-2,1.4221742E-2,3.8845684E-2,2.584793E-2,4.380022E-2,-1.3829837E-2,-3.5200674E-2,1.6466076E-3,1.9209769E-2,4.8949435E-2,2.8015452E-2,-4.0800646E-3,-4.8155468E-2,-1.5659416E-4,1.3060679E-2,-1.3518512E-2,2.3233196E-2,4.808888E-2,-1.878078E-3,6.864334E-2,5.0247166E-2,3.8324706E-2,-1.0213288E-3],"split_indices":[20,102,33,55,51,28,28,45,0,52,2,2,27,45,52,2,4,0,45,41,42,28,57,29,0,2,52,10,68,53,38,3,53,0,5,56,52,10,7,0,28,38,4,0,0,31,54,0,7,2,28,38,2,0,0,2,0,12,52,2,8,52,58,0,29,2,7,45,0,45,8,58,31,11,58,9,1,2,27,50,0,4,0,2,4,47,54,54,28,56,7,28,0,0,0,0,0,4,3,31,2,38,45,51,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.663E3,8.31E2,1.832E3,4.42E2,3.89E2,1.138E3,6.94E2,3.2E2,1.22E2,2.71E2,1.18E2,5.49E2,5.89E2,3.4E1,6.6E2,2.25E2,9.5E1,3E0,1.19E2,1.74E2,9.7E1,1.01E2,1.7E1,2.97E2,2.52E2,5.51E2,3.8E1,8E0,2.6E1,2.93E2,3.67E2,1.93E2,3.2E1,4.3E1,5.2E1,5.6E1,6.3E1,9.6E1,7.8E1,4E0,9.3E1,7.1E1,3E1,1.5E1,2E0,8.8E1,2.09E2,6E0,2.46E2,2.96E2,2.55E2,3.2E1,6E0,4E0,4E0,2.4E1,2E0,2.23E2,7E1,1.4E2,2.27E2,1.72E2,2.1E1,1.7E1,1.5E1,3.3E1,1.9E1,5.3E1,3E0,3.7E1,2.6E1,6.4E1,3.2E1,4.2E1,3.6E1,3.3E1,6E1,3.2E1,3.9E1,1.8E1,1.2E1,7.4E1,1.4E1,1.74E2,3.5E1,2.24E2,2.2E1,4.8E1,2.48E2,2.01E2,5.4E1,2.8E1,4E0,4E0,2E0,2.1E1,3E0,1E2,1.23E2,7E0,6.3E1,1.02E2,3.8E1,1.61E2,6.6E1,1.39E2,3.3E1,1.6E1,5E0,2E0,1.5E1,2E0,1.3E1,2.9E1,4E0,1.4E1,5E0,2.3E1,3E1,2.8E1,9E0,2.3E1,3E0,5.5E1,9E0,4E0,2.8E1,2E0,4E1,8E0,2.8E1,1E1,2.3E1,3E1,3E1,2.3E1,9E0,2.9E1,1E1,3E0,1.5E1,4.9E1,2.5E1,1.3E2,4.4E1,1.1E1,2.4E1,2.21E2,3E0,2E0,2E1,1.5E1,3.3E1,1.95E2,5.3E1,1.82E2,1.9E1,1.4E1,4E1,3E0,2.5E1,1.8E1,8.2E1,4.4E1,7.9E1,2E0,5E0,3.3E1,3E1,6E0,9.6E1,3.5E1,3E0,6.1E1,1E2,6.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-4.919165E-3,-5.127597E-1,2.3879103E-1,-7.9191464E-1,-1.7278804E-1,1.0360127E-2,5.744305E-1,-9.3548167E-1,-2.1089284E-1,-3.733383E-1,3.110595E-1,-3.844976E-1,9.87894E-2,-5.0353825E-2,6.7176384E-1,-5.1685107E-1,-1.0391344E0,9.2097595E-2,-3.004092E-1,-4.4944045E-1,-1.4406104E-1,1.22466266E-1,6.40427E-1,-5.205553E-1,-2.5133985E-1,2.6613197E-1,-8.421548E-2,-4.7398776E-1,2.2826439E-1,4.2708096E-1,8.6057603E-1,-7.2208554E-1,-1.9926143E-1,-9.6880615E-1,-1.6317617E0,-3.0516336E-2,-6.135284E-1,-5.636162E-1,-3.0649674E-1,-4.206998E-2,-9.874249E-2,-9.315113E-2,2.32138E-1,8.1261045E-1,4.103946E-1,-5.969056E-1,-3.8630342E-1,-2.7414072E-1,-1.4157817E-2,1.2882808E0,2.3131421E-1,-1.10023074E-1,8.274286E-2,-5.2044743E-1,-2.2309323E-1,4.899046E-1,-5.8427315E-2,8.5922915E-1,3.1470641E-1,5.6706154E-1,1.0399625E0,-1.4199626E-2,-7.7713436E-1,-4.8259678E-1,1.1423624E-1,-1.0583283E0,-6.140792E-1,-9.734037E-1,-9.093996E-2,-1.4891629E-1,4.698151E-1,-5.237221E-1,-5.7052564E-2,-5.186377E-1,-8.766293E-1,-4.1703E-1,-2.070744E-1,-1.3190204E-1,1.0012114E-2,-1.3954552E-2,-4.7818264E-3,2.5336344E-2,3.4767216E-1,1.32296225E-2,4.0732723E-2,3.2350381E-3,2.236119E-2,-2.9924694E-2,-1.15515925E-2,-1.9704202E-2,-1.3809703E-1,-1.8779661E-1,-3.5827935E-1,-5.774251E-3,5.5730506E-3,2.1002414E-2,6.634151E-2,-1.4144881E-1,3.2282695E-1,-4.6645034E-2,-5.456144E-1,-1.7495487E-2,-3.0887377E-2,2.9410354E-3,-1.5715923E-2,3.3676726E-1,3.5996564E-2,3.8872745E-2,-1.5845139E-2,4.7522017E-1,9.510711E-1,1.4229986E-1,4.537174E-1,4.4853738E-1,8.564297E-1,1.1075033E0,5.902776E-1,-4.0111583E-2,-1.3649273E-2,-7.0425444E-3,-2.9220473E-2,2.3889191E-2,1.1332703E-3,-5.2024394E-2,-2.5440563E-2,-1.2739209E-2,-3.600956E-2,-1.3584908E-2,-5.244782E-2,-1.1711691E-2,6.8403166E-3,3.4070555E-2,-3.374226E-3,-3.20843E-2,-1.1737632E-2,-2.8140582E-2,-1.7909383E-2,-4.5041524E-2,-1.6770821E-2,-2.0713795E-2,-2.9264675E-3,-1.0821908E-2,3.6486143E-3,-1.2323991E-2,-1.6360426E-3,8.529433E-3,-6.048879E-3,-7.3080612E-3,4.778744E-3,5.3772903E-3,2.018379E-2,1.7572939E-3,-1.2779862E-2,-1.315036E-2,-4.6435213E-3,2.2946522E-3,-1.8094562E-2,-1.24904215E-2,4.8844684E-3,8.581845E-3,2.4710488E-2,-9.423928E-3,4.403247E-3,-4.1492075E-2,-7.264296E-3,1.8601334E-2,-5.7371575E-3,-1.5103351E-2,4.1965568E-3,2.6947062E-2,1.0667516E-2,9.782774E-3,4.6983264E-2,8.852071E-3,-2.1807175E-2,2.488308E-2,5.222937E-3,2.3928111E-2,1.5389657E-3,2.7770624E-2,4.6568908E-2,5.967394E-2,4.4764254E-2,3.6715064E-2,3.5426198E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,83,85,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,113,115,-1,117,119,121,123,125,127,-1,129,131,133,-1,135,137,139,141,143,-1,-1,145,147,149,-1,-1,-1,-1,-1,-1,-1,151,153,155,-1,-1,-1,-1,157,159,161,163,-1,-1,-1,-1,165,-1,167,-1,169,171,173,175,177,179,181,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3243393E2,8.252524E1,1.3915088E2,3.9675354E1,3.834822E1,3.778324E1,4.4730896E1,1.6082123E1,1.8916203E1,4.803383E0,7.127082E0,3.456356E0,2.7094479E1,1.1911408E1,2.9015076E1,4.9401817E0,1.11727295E1,0E0,7.840581E0,3.2382812E0,2.2762773E0,1.7951586E0,1.3300343E0,7.636776E-1,5.4773045E-1,1.6144096E1,1.9970615E1,3.501501E-1,4.6042395E0,1.3278519E1,1.8259888E1,8.706894E-1,2.8975291E0,8.114746E0,3.6039124E0,3.0911632E0,1.6072922E0,1.1644135E0,9.5764446E-1,0E0,7.216298E-1,4.5406762E-1,1.1858242E0,2.3293018E-1,3.560667E-1,4.4579697E-1,1.8028927E-1,6.083498E-1,1.529316E-1,6.2875557E-1,1.53280945E1,1.1514244E1,0E0,4.0176678E-1,2.5929692E-1,1.0294905E0,8.080976E-1,1.5583191E0,5.2799416E0,4.3488655E0,6.0584717E0,0E0,9.806614E-1,6.37872E-1,5.455323E-1,2.6540527E0,2.5826187E0,6.13122E-1,0E0,1.2067871E0,1.3554792E0,1.424799E0,0E0,8.0791855E-1,1.0782242E-1,2.1621132E-1,2.955656E-1,7.5132704E-1,0E0,0E0,4.4476962E-1,2.6692042E-1,5.084567E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2232128E-1,3.51812E-1,3.8343716E-1,0E0,0E0,0E0,0E0,2.612753E0,9.894775E0,7.633507E0,6.565894E0,0E0,0E0,0E0,0E0,5.6528807E-1,0E0,4.2487645E-1,0E0,1.6172051E-1,8.0724335E-1,2.6520016E0,2.7355404E0,2.1119423E0,6.869602E-1,3.4075012E0,2.7661333E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,77,77,80,80,81,81,82,82,90,90,91,91,92,92,97,97,98,98,99,99,100,100,105,105,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,84,86,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,114,116,-1,118,120,122,124,126,128,-1,130,132,134,-1,136,138,140,142,144,-1,-1,146,148,150,-1,-1,-1,-1,-1,-1,-1,152,154,156,-1,-1,-1,-1,158,160,162,164,-1,-1,-1,-1,166,-1,168,-1,170,172,174,176,178,180,182,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,7.7344055E4,2.4519731E3,4.7931265E6,3.97E2,2.1328075E-1,1.2478469E-1,1.4E1,3.5734247E2,1.2060912E3,2.0550128E5,1.9E1,2.6393727E4,1.1410706E3,2.373E3,1.0017953E8,9.2097595E-2,8.229907E7,1.8329959E-1,5.2143492E-5,1.502494E0,8.7643677E-1,9.6E1,8.397659E6,2.04E5,6.7652373E3,1.6213043E3,3.5E0,8E0,2.3421426E-1,1.7515824E8,1E0,3.83E3,2.8681312E8,8.015419E3,9.22E4,1.5988282E1,2.1102592E8,-4.206998E-2,2.1446484E-1,3.4614954E8,2.821E3,2.6882867E8,1.4008022E11,6.2285713E1,2.73E2,9.25E0,3.598933E6,7.03E2,2.0407547E5,4.8297736E7,8.274286E-2,7.218466E7,1.92E2,2.3990374E8,3.9343938E-1,2.0131812E7,1E0,3.166E3,2.8091298E7,-1.4199626E-2,3.008162E6,1.68635E5,1.376072E8,5.8426323E0,2.7246006E7,2.7324794E11,-9.093996E-2,7.0764465E1,1.4433751E0,6.5317163E3,-5.7052564E-2,2.728E3,6.895858E-1,1.1075305E0,3.9382784E9,5.4878284E2,1.0012114E-2,-1.3954552E-2,9.035491E5,2.0783027E5,2.1924436E5,1.32296225E-2,4.0732723E-2,3.2350381E-3,2.236119E-2,-2.9924694E-2,-1.15515925E-2,-1.9704202E-2,3.1E1,1.109125E3,1.1485E4,-5.774251E-3,5.5730506E-3,2.1002414E-2,6.634151E-2,2.001177E6,1.009E3,7.59093E5,3.1998687E3,-1.7495487E-2,-3.0887377E-2,2.9410354E-3,-1.5715923E-2,1E0,3.5996564E-2,4.238096E6,-1.5845139E-2,1.3878379E0,1.5166431E5,4.7058824E-1,8.447369E0,3.5481934E1,5.57E4,2E1,7.4709034E-1,-4.0111583E-2,-1.3649273E-2,-7.0425444E-3,-2.9220473E-2,2.3889191E-2,1.1332703E-3,-5.2024394E-2,-2.5440563E-2,-1.2739209E-2,-3.600956E-2,-1.3584908E-2,-5.244782E-2,-1.1711691E-2,6.8403166E-3,3.4070555E-2,-3.374226E-3,-3.20843E-2,-1.1737632E-2,-2.8140582E-2,-1.7909383E-2,-4.5041524E-2,-1.6770821E-2,-2.0713795E-2,-2.9264675E-3,-1.0821908E-2,3.6486143E-3,-1.2323991E-2,-1.6360426E-3,8.529433E-3,-6.048879E-3,-7.3080612E-3,4.778744E-3,5.3772903E-3,2.018379E-2,1.7572939E-3,-1.2779862E-2,-1.315036E-2,-4.6435213E-3,2.2946522E-3,-1.8094562E-2,-1.24904215E-2,4.8844684E-3,8.581845E-3,2.4710488E-2,-9.423928E-3,4.403247E-3,-4.1492075E-2,-7.264296E-3,1.8601334E-2,-5.7371575E-3,-1.5103351E-2,4.1965568E-3,2.6947062E-2,1.0667516E-2,9.782774E-3,4.6983264E-2,8.852071E-3,-2.1807175E-2,2.488308E-2,5.222937E-3,2.3928111E-2,1.5389657E-3,2.7770624E-2,4.6568908E-2,5.967394E-2,4.4764254E-2,3.6715064E-2,3.5426198E-3],"split_indices":[20,102,33,4,51,2,42,27,0,52,52,45,3,28,52,2,45,0,45,41,42,41,57,29,9,5,52,4,53,3,38,31,16,2,7,4,10,54,7,0,38,12,2,7,31,4,2,58,5,1,28,45,0,45,0,7,27,12,100,0,45,0,32,29,12,53,45,31,0,56,42,4,0,10,27,38,5,52,0,0,47,33,28,0,0,0,0,0,0,0,0,33,9,0,0,0,0,9,2,1,4,0,0,0,0,74,0,45,0,53,28,57,53,57,2,3,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.684E3,8.7E2,1.814E3,4.77E2,3.93E2,1.08E3,7.34E2,3.82E2,9.5E1,2.78E2,1.15E2,1.97E2,8.83E2,9.9E1,6.35E2,7.7E1,3.05E2,3E0,9.2E1,2.08E2,7E1,7.4E1,4.1E1,9.6E1,1.01E2,4.61E2,4.22E2,3.9E1,6E1,2.78E2,3.57E2,4.6E1,3.1E1,2.75E2,3E1,5E1,4.2E1,1.14E2,9.4E1,3E0,6.7E1,2.5E1,4.9E1,2.2E1,1.9E1,5.9E1,3.7E1,9.2E1,9E0,1.4E1,4.47E2,4.17E2,5E0,3.2E1,7E0,3.1E1,2.9E1,5.6E1,2.22E2,1.37E2,2.2E2,6E0,4E1,1.6E1,1.5E1,2.18E2,5.7E1,1E1,2E1,4.1E1,9E0,3.8E1,4E0,1.02E2,1.2E1,4.3E1,5.1E1,6.1E1,6E0,7E0,1.8E1,1.8E1,3.1E1,2E0,2E1,3E0,1.6E1,5.4E1,5E0,3.3E1,4E0,4.7E1,4.5E1,5E0,4E0,2E0,1.2E1,8.8E1,3.59E2,3.65E2,5.2E1,1.6E1,1.6E1,2E0,5E0,2.1E1,1E1,2.2E1,7E0,1.2E1,4.4E1,1E2,1.22E2,9.9E1,3.8E1,1.9E2,3E1,3.5E1,5E0,5E0,1.1E1,2E0,1.3E1,2.06E2,1.2E1,1.7E1,4E1,2E0,8E0,3.1E1,1E1,6E0,3E0,2.4E1,1.4E1,6.7E1,3.5E1,1E1,2E0,4.1E1,2E0,4.8E1,3E0,2.6E1,3.5E1,7E0,1.1E1,5E0,1.3E1,8E0,2.3E1,2E0,2E0,2.3E1,2.4E1,2E0,4.3E1,5.9E1,2.9E1,2.07E2,1.52E2,1.75E2,1.9E2,2.8E1,2.4E1,1.9E1,2E0,2E0,2E1,8E0,4E0,2E0,4.2E1,9.4E1,6E0,1.02E2,2E1,8.8E1,1.1E1,1.3E1,2.5E1,1.02E2,8.8E1,2.2E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[1.7515587E-2,-4.6577635E-1,2.4062452E-1,-7.76435E-1,-1.8065654E-1,2.9554037E-2,5.8481735E-1,-9.203482E-1,-4.3783516E-1,-3.3206242E-1,4.0224496E-1,-7.918369E-2,3.9286548E-1,-3.128538E-2,6.600262E-1,-9.911532E-1,-5.4222184E-1,-2.9339272E-1,-6.0541105E-1,-3.7570697E-1,8.998138E-2,1.10121034E-1,6.057744E-1,-5.34453E-1,-2.2682665E-2,3.3058777E-1,8.8398254E-1,-4.3784672E-1,1.7259692E-1,5.795156E-1,1.1303889E0,-8.783423E-1,-1.4966913E0,6.742432E-2,-6.690063E-1,-3.5230425E-1,1.0886428E-2,-6.979792E-1,-3.0585936E-1,-2.740333E-1,-7.346481E-1,3.601082E-1,-2.8722864E-2,6.762017E-1,1.1629193E-1,-5.77323E-1,4.795034E-2,-3.5621222E-2,7.944069E-2,3.6245024E-1,-4.225629E-1,2.2578161E-2,4.8785433E-2,-5.0905085E-1,-2.1400604E-1,3.2812512E-1,-1.9784933E-1,5.1131433E-1,1.0553775E0,6.180007E-1,1.2166445E0,-9.576119E-1,-6.388863E-1,-7.4846774E-2,-4.0186353E-2,-6.3578864E-3,1.2170062E-2,-7.243893E-1,-1.1669412E-2,-2.3466805E-2,-2.7618378E-1,-1.22268766E-1,1.4738983E-2,-7.5107783E-1,-3.2518956E-1,-1.7730802E-2,3.5946374E-3,-3.2666022E-1,3.3884788E-1,-5.068287E-1,-1.0011654E0,6.2573394E-3,1.967377E-2,-1.7891224E-2,1.5581794E-2,3.160023E-1,7.650123E-1,-5.546192E-3,1.33478055E-2,-4.8409703E-1,-8.481132E-1,-1.1109948E-1,1.6773078E-1,2.156619E-1,5.625935E-1,-8.9622214E-2,-3.532306E-2,-2.3697845E-3,-5.6020683E-1,2.8471376E-3,-1.5088721E-2,7.415243E-2,4.3272308E-1,-1.3762907E-2,2.7106209E-3,2.4210797E-1,5.873564E-1,5.1964873E-1,1.2608923E0,7.1616924E-1,-6.4156565E-4,1.2823193E0,4.981187E-1,-4.640231E-2,-4.5691077E-3,4.3105887E-4,-3.2505058E-2,-1.7590953E-2,-3.847332E-2,-6.4417283E-4,-1.54947825E-2,-1.1906166E-2,-8.458317E-4,-3.7037075E-2,-1.1880717E-2,-3.48473E-3,-1.9712064E-2,-3.897682E-2,-1.3119226E-2,8.864441E-3,3.0739471E-2,-3.4407414E-2,-1.4815022E-2,-5.1590413E-2,-2.2342328E-2,-4.0430613E-3,4.455926E-3,1.9970935E-2,5.3344383E-3,2.938278E-2,4.7768764E-2,-2.537272E-2,1.0806537E-3,-4.3794144E-2,-4.8348783E-3,-1.0770009E-3,-1.2388245E-2,6.208467E-3,3.1455684E-2,1.1108762E-2,-2.694045E-2,1.908882E-2,3.362981E-2,4.281918E-3,-1.4293335E-2,-7.31503E-3,-2.8942747E-2,5.8901254E-3,-5.428953E-3,1.2272494E-2,2.7090317E-2,5.605291E-3,2.8689759E-2,2.9984437E-2,4.7241547E-3,3.1016782E-2,2.9132725E-4,6.332771E-2,1.291666E-2,1.2427995E-2,3.9615847E-2,3.2217834E-2,6.318827E-2,3.3706088E-2,1.1203404E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,-1,93,95,-1,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,-1,-1,117,-1,-1,119,121,-1,123,125,-1,-1,127,129,131,133,-1,-1,-1,135,137,139,-1,-1,141,143,145,147,149,151,153,-1,-1,155,-1,-1,157,159,-1,-1,161,163,165,167,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8984253E2,7.5059875E1,1.3358415E2,1.9380997E1,3.9318665E1,4.5107067E1,3.240976E1,7.066162E0,2.7996006E0,3.4769104E1,5.403226E0,2.2598555E1,7.751793E0,6.4581523E0,2.2937439E1,1.2258606E1,3.677909E0,1.2227731E0,1.3393364E0,1.2462399E1,0E0,1.3640018E0,1.7829018E0,6.964094E0,1.7093975E1,5.792507E0,1.1312428E0,3.0545807E-1,3.060891E0,1.6791E1,3.225792E0,3.1275635E0,5.162964E-1,3.7010124E-1,7.421913E-1,4.6215057E-1,5.119994E-1,5.871315E-1,3.9562404E-1,8.883785E0,4.107136E0,1.0918355E-1,4.0307605E-1,1.2617626E0,3.338671E-1,1.9876747E0,0E0,1.1959888E1,0E0,6.518255E0,9.827137E-1,0E0,0E0,4.18962E-1,2.3981732E-1,9.5081854E-1,3.807733E-1,9.458702E0,6.611389E0,9.393444E-1,2.9476242E0,1.3315887E0,1.348669E0,0E0,0E0,0E0,0E0,6.636505E-1,0E0,0E0,4.8094654E-1,1.1455661E-1,0E0,2.117939E-1,1.2274653E-1,0E0,0E0,6.2239876E0,9.0616107E-1,1.5748081E0,8.8058853E-1,0E0,0E0,0E0,1.9377315E-1,1.8089831E-1,6.907463E-1,0E0,0E0,1.7025833E0,1.0478802E0,7.421955E0,3.8459063E0,1.8048329E0,1.8955288E0,2.510165E-1,0E0,0E0,2.1347618E-1,0E0,0E0,1.2383856E-1,4.6500874E-1,0E0,0E0,4.6174064E0,6.849945E0,1.3109026E0,2.5060577E0,4.445734E-1,0E0,6.733322E-1,2.7007997E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,67,67,70,70,71,71,73,73,74,74,77,77,78,78,79,79,80,80,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,98,98,101,101,102,102,105,105,106,106,107,107,108,108,109,109,111,111,112,112],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,-1,94,96,-1,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,-1,-1,118,-1,-1,120,122,-1,124,126,-1,-1,128,130,132,134,-1,-1,-1,136,138,140,-1,-1,142,144,146,148,150,152,154,-1,-1,156,-1,-1,158,160,-1,-1,162,164,166,168,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,1.8532948E6,1E0,1.6990049E0,1.2727361E6,2.5998926E-1,5.121E3,5.1148495E6,1.3144558E4,1.3097832E3,8.2E1,7.064848E6,3.731E3,3.2196458E6,4.0844156E7,2.1693E5,2.3871907E2,1.998E3,6.747114E7,8.998138E-2,1.3510204E1,7.2234793E0,1.5445492E3,9.256843E3,6.3300834E0,1E0,1.6213043E3,1.8159722E-1,6.082581E3,1E0,1.1E1,1E0,1.4783E4,3.4409692E3,1.009E3,2.695E3,2.4859156E1,3.771981E6,2.0601392E3,1.4396E4,6.58542E5,2.5534486E4,2.7271206E5,1E0,1.4504054E8,4.795034E-2,3.422351E6,7.944069E-2,1.06E3,1E0,2.2578161E-2,4.8785433E-2,3.7246967E2,1.92E2,3.502483E-2,3.4540886E3,1E0,2.1150263E-1,2.777588E6,1E0,9.12179E6,2.08E2,-7.4846774E-2,-4.0186353E-2,-6.3578864E-3,1.2170062E-2,1.8357558E5,-1.1669412E-2,-2.3466805E-2,1.246372E0,1.732E3,1.4738983E-2,2.1989189E2,2.8E1,-1.7730802E-2,3.5946374E-3,1E0,7.3610186E3,2.8207336E3,3.4986075E6,6.2573394E-3,1.967377E-2,-1.7891224E-2,2.821E3,2.011653E7,3.3791902E0,-5.546192E-3,1.33478055E-2,6.8E1,1E0,5.024605E0,1.5963264E9,4.6E1,3.6129813E2,5E0,-3.532306E-2,-2.3697845E-3,1.401E3,2.8471376E-3,-1.5088721E-2,6.513793E1,4.3800272E8,-1.3762907E-2,2.7106209E-3,1.3508893E3,3.5301748E7,1.921032E4,1.515252E10,9.8E1,-6.4156565E-4,1.2062E5,1.85369E5,-4.640231E-2,-4.5691077E-3,4.3105887E-4,-3.2505058E-2,-1.7590953E-2,-3.847332E-2,-6.4417283E-4,-1.54947825E-2,-1.1906166E-2,-8.458317E-4,-3.7037075E-2,-1.1880717E-2,-3.48473E-3,-1.9712064E-2,-3.897682E-2,-1.3119226E-2,8.864441E-3,3.0739471E-2,-3.4407414E-2,-1.4815022E-2,-5.1590413E-2,-2.2342328E-2,-4.0430613E-3,4.455926E-3,1.9970935E-2,5.3344383E-3,2.938278E-2,4.7768764E-2,-2.537272E-2,1.0806537E-3,-4.3794144E-2,-4.8348783E-3,-1.0770009E-3,-1.2388245E-2,6.208467E-3,3.1455684E-2,1.1108762E-2,-2.694045E-2,1.908882E-2,3.362981E-2,4.281918E-3,-1.4293335E-2,-7.31503E-3,-2.8942747E-2,5.8901254E-3,-5.428953E-3,1.2272494E-2,2.7090317E-2,5.605291E-3,2.8689759E-2,2.9984437E-2,4.7241547E-3,3.1016782E-2,2.9132725E-4,6.332771E-2,1.291666E-2,1.2427995E-2,3.9615847E-2,3.2217834E-2,6.318827E-2,3.3706088E-2,1.1203404E-2],"split_indices":[20,56,51,102,41,28,42,2,45,52,4,29,28,29,28,45,29,52,10,45,0,58,57,52,4,53,102,4,27,52,100,3,105,9,52,2,2,58,1,52,9,1,33,28,102,31,0,9,0,2,84,0,0,4,0,38,4,100,38,1,6,9,2,0,0,0,0,32,0,0,53,2,0,52,3,0,0,75,4,55,32,0,0,0,2,32,35,0,0,0,68,53,7,3,52,8,0,0,2,0,0,56,7,0,0,52,45,4,5,0,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.686E3,8.48E2,1.838E3,4.05E2,4.43E2,1.14E3,6.98E2,2.83E2,1.22E2,3.52E2,9.1E1,8.78E2,2.62E2,7.6E1,6.22E2,2.37E2,4.6E1,6.7E1,5.5E1,3.46E2,6E0,3.8E1,5.3E1,9.6E1,7.82E2,2.34E2,2.8E1,2.5E1,5.1E1,5.33E2,8.9E1,1.96E2,4.1E1,8E0,3.8E1,5.6E1,1.1E1,4.1E1,1.4E1,2.71E2,7.5E1,1.3E1,2.5E1,4.6E1,7E0,9.4E1,2E0,7.77E2,5E0,2.25E2,9E0,8E0,2E1,1.8E1,7E0,3.6E1,1.5E1,4.68E2,6.5E1,1.4E1,7.5E1,1.45E2,5.1E1,3.6E1,5E0,4E0,4E0,3.3E1,5E0,1.8E1,3.8E1,8E0,3E0,3.5E1,6E0,1.2E1,2E0,2.5E2,2.1E1,4.2E1,3.3E1,3E0,1E1,2E0,2.3E1,1E1,3.6E1,3E0,4E0,7.2E1,2.2E1,5.67E2,2.1E2,1.31E2,9.4E1,5E0,4E0,2E0,1.6E1,2E0,5E0,1.1E1,2.5E1,1.1E1,4E0,1.04E2,3.64E2,1.9E1,4.6E1,1.2E1,2E0,6.8E1,7E0,1.43E2,2E0,3E0,4.8E1,7E0,2.6E1,6E0,3.2E1,3E0,5E0,3.3E1,2E0,2E0,4E0,2.3E1,2.27E2,1.5E1,6E0,1.9E1,2.3E1,2.8E1,5E0,1E1,1.3E1,6E0,4E0,2.4E1,1.2E1,6.6E1,6E0,2E1,2E0,3.55E2,2.12E2,1.96E2,1.4E1,1.29E2,2E0,4.5E1,4.9E1,3E0,2E0,2E0,1.4E1,9E0,2E0,1.2E1,1.3E1,7.8E1,2.6E1,3.37E2,2.7E1,1.5E1,4E0,4.3E1,3E0,3E0,9E0,5E0,6.3E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-4.4346023E-3,-4.9924287E-1,2.2344005E-1,-7.523059E-1,-1.8343851E-1,5.5417992E-2,5.948842E-1,-8.762053E-1,-1.7842771E-1,-3.6012796E-1,2.510525E-1,-1.6846073E-1,2.1308942E-1,4.954327E-1,1.0726805E0,-5.171616E-1,-9.753441E-1,-2.867368E-1,7.6155685E-2,-4.754012E-1,-2.4041353E-1,5.2564174E-2,5.664371E-1,-3.3135834E-1,2.7452286E-2,2.5753817E-1,-4.171702E-1,4.2617947E-1,1.04659E0,1.1405934E0,3.67778E-1,-6.154318E-1,-5.490702E-2,-1.048101E0,-5.6294364E-1,-1.8221585E-2,-5.4170924E-1,-5.681259E-1,-2.565554E-1,-2.683761E-1,6.2431926E-3,-1.5978965E-1,1.7202511E-1,6.069159E-1,1.4443612E-3,-5.2767235E-1,-2.4052514E-1,6.52581E-2,-2.514266E-2,5.6089576E-2,4.2269495E-1,-5.049324E-1,2.959479E-1,-9.540178E-2,4.8730674E-1,8.661617E-1,6.697101E-2,1.1675713E0,-6.2108174E-4,2.604694E-2,7.6493616E-3,-6.5836084E-1,-9.599027E-3,1.0822739E-1,-2.7363077E-2,-9.637717E-1,-1.4375284E0,-3.025072E-1,-6.755699E-1,-1.5328455E-1,4.640269E-1,-7.024536E-1,-2.466429E-1,-6.275368E-1,-1.4967823E-1,-3.3063972E-1,1.6110031E-1,-3.6441445E-1,-1.8654244E-1,1.9013456E-1,-2.3549592E-1,4.184815E-2,2.7876225E-1,4.4638932E-1,3.3484027E-2,-4.927532E-1,-4.1489117E-2,-1.8446854E-1,-5.118208E-1,-9.3524195E-2,3.210257E-1,-3.921947E-2,2.5558886E-1,3.909797E-1,7.441659E-2,-6.1830604E-1,-2.1782424E-1,1.6337793E-3,2.6167838E-2,1.2928998E-1,-4.7308764E-1,3.2622993E-1,5.9587157E-1,8.87934E-1,1.5489248E-2,2.7664749E-2,1.2045933E0,-3.422607E-2,-1.38134975E-2,2.1688746E-2,2.1033087E-4,-4.866716E-2,-2.8714808E-2,-7.213576E-2,-2.9399473E-2,-2.4058742E-2,-4.3695294E-3,-3.3436537E-2,-1.07376035E-2,1.9643265E-3,-1.3022219E-2,-4.7177344E-4,2.8773973E-2,-2.7872162E-2,-5.8400787E-2,-5.440206E-3,-2.474146E-2,-2.4827808E-2,-3.7059445E-2,2.475866E-3,-1.3505178E-2,-2.8926613E-2,-1.2251873E-2,-3.9099203E-3,1.7411487E-2,-1.478637E-2,-2.59938E-2,-1.14788655E-2,-2.5692945E-3,-1.5492676E-3,1.6725345E-2,-3.1319007E-3,-1.4327683E-2,7.6043764E-3,-4.9165785E-3,8.540398E-3,2.1901878E-2,2.8067967E-2,1.0342919E-2,-2.4330894E-2,-3.0451086E-3,-1.4212215E-2,-6.335672E-4,-4.2702634E-2,-1.7971318E-2,1.3271941E-2,-5.8422377E-3,4.5248043E-2,1.2457539E-2,-8.447121E-4,-4.4497415E-2,-2.314539E-3,1.4044928E-2,7.8089884E-3,2.2793485E-2,-3.14628E-2,-1.8400025E-3,-1.660995E-2,-3.7218041E-3,-8.3835813E-4,1.963284E-2,-7.1223923E-3,-2.5476398E-2,5.5143614E-3,2.0148948E-2,1.2880463E-2,3.2400552E-2,4.4973023E-2,2.3189435E-2,6.1774302E-2,4.1320153E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,-1,85,87,-1,89,91,93,95,97,99,101,103,-1,105,-1,-1,-1,107,-1,109,-1,111,113,115,117,119,121,123,125,127,129,131,133,135,137,139,141,143,145,147,-1,149,-1,151,153,155,157,159,161,163,-1,165,167,-1,-1,169,171,173,175,177,-1,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0037616E2,6.6993454E1,1.1380436E2,3.2977722E1,2.8879148E1,4.440709E1,2.6458542E1,1.3128571E1,1.6546911E1,3.5702744E0,6.793643E0,1.6604252E1,2.0781948E1,1.7569923E1,4.224098E0,3.827404E0,8.381439E0,5.467248E0,0E0,2.5937176E0,1.4139161E0,1.7525111E0,8.9342594E-1,4.9373226E0,1.667752E1,2.293301E1,3.2160912E0,1.3469162E1,2.089695E0,2.753769E0,2.7952147E-1,1.1151142E0,1.409876E0,6.9230957E0,1.1742392E0,2.6764843E0,1.7713852E0,2.2429447E0,1.352819E0,9.06559E-1,0E0,7.070623E-1,5.99702E-1,2.655058E-1,0E0,6.256027E-1,2.8863363E0,0E0,5.4508023E0,5.9424195E0,1.2989502E1,1.2975397E0,3.7483382E-1,3.8799963E0,6.3983154E0,1.5002823E-1,0E0,1.1823349E0,0E0,0E0,0E0,1.1062202E0,0E0,4.6324646E-1,0E0,3.420639E0,1.5096817E0,6.297536E-1,1.7012405E-1,7.48764E-1,5.8936167E-1,9.09482E-1,5.1603514E-1,9.077492E-1,3.609862E-1,5.817082E-1,3.73717E-1,3.8836956E-1,4.7448087E-1,1.882919E-1,2.079035E-1,3.711783E-1,3.488561E-1,4.2063236E-1,0E0,5.126629E-1,0E0,3.1497064E0,1.3772144E0,2.0597034E0,1.197828E0,4.0552034E0,1.1708708E0,7.087963E0,0E0,6.69857E-1,2.26098E-1,0E0,0E0,1.2019776E0,2.4558163E-1,3.023449E0,5.7263565E0,2.6797104E-1,0E0,0E0,1.2197952E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,61,61,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,96,96,99,99,100,100,101,101,102,102,103,103,106,106],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,-1,86,88,-1,90,92,94,96,98,100,102,104,-1,106,-1,-1,-1,108,-1,110,-1,112,114,116,118,120,122,124,126,128,130,132,134,136,138,140,142,144,146,148,-1,150,-1,152,154,156,158,160,162,164,-1,166,168,-1,-1,170,172,174,176,178,-1,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.867292E3,4.7931265E6,1.8108038E5,2.9251662E6,2.0304577E-1,1.9013363E4,1.5669752E-2,1.2060912E3,2.857E3,3.956147E-1,3.6966505E0,1E0,6.2880285E6,4.888E0,6.971004E7,7.6155685E-2,1.5345133E1,6.341735E2,1.482958E0,1E0,1.141E4,2.3E1,9.27E2,3.0795444E3,1.5833762E-1,1.146155E1,9.256843E3,2.84E2,3.4986075E6,8.749966E1,1.1034263E10,3E0,9.908038E1,3.970405E3,2.262586E6,4.3E1,2.1787034E8,6.2431926E-3,1.6254681E0,2.4513194E5,1.3137E4,1.4443612E-3,2.545455E7,1E0,6.52581E-2,8.32E3,1.5859042E6,4.4839956E3,2.6476662E6,3.09E2,6.496696E7,1.8108038E5,1.314806E8,6.697101E-2,4.6253732E2,-6.2108174E-4,2.604694E-2,7.6493616E-3,3.164136E6,-9.599027E-3,1.1766268E6,-2.7363077E-2,3.956E3,1.3999657E3,6.1463413E0,9.5732903E-1,5.418831E11,5.015233E4,1.5874315E10,5.5743E4,9.577834E7,6.822312E4,9.19E4,3.52E2,2.612E3,1.2690893E0,3.051499E5,5.48744E8,1.0824E4,2.106062E1,1.744993E6,3.3484027E-2,1.27384615E1,-4.1489117E-2,6.87E2,1E0,2.4E1,7.969174E7,1.062363E1,1.198E5,3.830216E5,7.441659E-2,1E0,8.896243E4,1.6337793E-3,2.6167838E-2,3.5799E4,8.657441E7,1.3933473E5,3.773E3,3.1066042E7,1.5489248E-2,2.7664749E-2,2.1226158E0,-3.422607E-2,-1.38134975E-2,2.1688746E-2,2.1033087E-4,-4.866716E-2,-2.8714808E-2,-7.213576E-2,-2.9399473E-2,-2.4058742E-2,-4.3695294E-3,-3.3436537E-2,-1.07376035E-2,1.9643265E-3,-1.3022219E-2,-4.7177344E-4,2.8773973E-2,-2.7872162E-2,-5.8400787E-2,-5.440206E-3,-2.474146E-2,-2.4827808E-2,-3.7059445E-2,2.475866E-3,-1.3505178E-2,-2.8926613E-2,-1.2251873E-2,-3.9099203E-3,1.7411487E-2,-1.478637E-2,-2.59938E-2,-1.14788655E-2,-2.5692945E-3,-1.5492676E-3,1.6725345E-2,-3.1319007E-3,-1.4327683E-2,7.6043764E-3,-4.9165785E-3,8.540398E-3,2.1901878E-2,2.8067967E-2,1.0342919E-2,-2.4330894E-2,-3.0451086E-3,-1.4212215E-2,-6.335672E-4,-4.2702634E-2,-1.7971318E-2,1.3271941E-2,-5.8422377E-3,4.5248043E-2,1.2457539E-2,-8.447121E-4,-4.4497415E-2,-2.314539E-3,1.4044928E-2,7.8089884E-3,2.2793485E-2,-3.14628E-2,-1.8400025E-3,-1.660995E-2,-3.7218041E-3,-8.3835813E-4,1.963284E-2,-7.1223923E-3,-2.5476398E-2,5.5143614E-3,2.0148948E-2,1.2880463E-2,3.2400552E-2,4.4973023E-2,2.3189435E-2,6.1774302E-2,4.1320153E-2],"split_indices":[20,102,12,4,51,28,28,27,52,38,52,2,27,35,6,50,54,45,0,58,52,41,84,9,0,2,4,42,35,4,0,32,58,5,8,56,52,1,3,7,0,53,28,2,0,45,6,0,2,28,52,32,0,45,28,1,0,4,0,0,0,1,0,32,0,2,52,58,27,31,28,12,10,7,33,1,0,2,42,28,5,10,56,47,0,56,0,2,16,3,7,54,7,28,0,74,33,0,0,10,45,33,2,45,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.662E3,8.39E2,1.823E3,4.65E2,3.74E2,1.256E3,5.67E2,3.82E2,8.3E1,2.66E2,1.08E2,5.19E2,7.37E2,4.71E2,9.6E1,8.4E1,2.98E2,7.9E1,4E0,1.34E2,1.32E2,6.7E1,4.1E1,2.83E2,2.36E2,6.89E2,4.8E1,4.2E2,5.1E1,8.7E1,9E0,6.9E1,1.5E1,2.52E2,4.6E1,3.9E1,4E1,9.3E1,4.1E1,1.23E2,9E0,2.4E1,4.3E1,3.8E1,3E0,8.8E1,1.95E2,8E0,2.28E2,3.11E2,3.78E2,4.3E1,5E0,4.4E1,3.76E2,3.6E1,1.5E1,8.5E1,2E0,4E0,5E0,6.2E1,7E0,1.2E1,3E0,2.1E2,4.2E1,1.5E1,3.1E1,3.1E1,8E0,2.5E1,1.5E1,8.1E1,1.2E1,3.5E1,6E0,5.5E1,6.8E1,4E0,2E1,2E1,2.3E1,1.6E1,2.2E1,8.2E1,6E0,1.63E2,3.2E1,1.91E2,3.7E1,2.11E2,1E2,3.69E2,9E0,3E1,1.3E1,3E0,2E0,2.8E1,1.6E1,1.53E2,2.23E2,3.4E1,2E0,6E0,7.9E1,5.3E1,9E0,2E0,1E1,1.82E2,2.8E1,3.8E1,4E0,7E0,8E0,2.9E1,2E0,1.2E1,1.9E1,2E0,6E0,2.2E1,3E0,1.1E1,4E0,4.9E1,3.2E1,5E0,7E0,6E0,2.9E1,3E0,3E0,4.4E1,1.1E1,4.8E1,2E1,2E0,2E0,6E0,1.4E1,1.1E1,9E0,1.6E1,7E0,9E0,7E0,7.9E1,3E0,9.8E1,6.5E1,7E0,2.5E1,1.3E1,1.78E2,2E0,3.5E1,2.07E2,4E0,1.1E1,8.9E1,1.01E2,2.68E2,2.8E1,2E0,6E0,7E0,1.9E1,9E0,3E0,1.3E1,4.8E1,1.05E2,4.5E1,1.78E2,2.9E1,5E0,6.1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[1.8860749E-4,-4.441118E-1,2.1098435E-1,-7.091485E-1,-2.0233929E-1,3.833865E-2,5.3233534E-1,-8.467057E-1,-3.8465324E-1,-3.2853097E-1,3.4043851E-1,-1.8696646E-1,2.0979714E-1,3.932558E-1,8.7977976E-1,-9.193671E-1,-3.918579E-1,-5.089913E-1,-2.1635105E-1,-3.6799085E-1,8.3161615E-2,2.4440913E-1,7.3928964E-1,-2.2847822E-1,3.9725527E-1,-3.4646764E-1,2.535628E-1,2.344202E-1,6.950682E-1,4.2269012E-1,9.999692E-1,-7.9712665E-1,-1.3801624E0,8.4622346E-2,-5.099326E-1,-5.600769E-1,-1.8407054E-1,-2.609325E-1,5.7592033E-3,-2.8763703E-1,-8.1502664E-1,6.661717E-2,3.923756E-1,1.2650997E-2,3.968274E-2,-4.495674E-1,-1.5057354E-1,5.7468224E-2,1.4469442E-1,-4.7625467E-1,2.0732722E-1,6.434489E-2,4.2175263E-1,3.0443656E-1,-2.9840046E-1,5.953819E-1,1.1456037E0,5.341327E-1,-8.133385E-3,1.0522008E0,5.261475E-1,-8.636042E-1,-4.0049723E-1,-1.8480714E-2,-1.4135507E0,-5.4962873E-3,1.27909845E-2,-5.8353454E-1,1.8945217E-2,-3.5019055E-1,-6.284403E-1,-1.1907787E-2,3.4008247E-3,-3.0906764E-1,-2.2562281E-3,-4.145218E-1,-1.15938716E-1,-7.00872E-1,-1.1561619E0,-2.2724554E-2,3.073867E-1,8.500362E-2,4.71508E-1,-3.8424906E-1,-7.1685374E-1,3.3643937E-1,-1.7629193E-1,-4.1071237E-3,2.1462257E-1,-6.267079E-1,-1.8371882E-1,-5.3983373E-3,2.1046968E-2,1.51384E-1,-1.6425556E-1,3.889705E-1,7.359985E-2,1.82212E-1,5.222423E-1,-3.0668985E-2,-3.9907258E-2,6.531286E-1,2.9862763E-2,5.765489E-2,2.4399783E-2,2.7261978E-1,6.849504E-1,1.35738505E-2,-2.1237178E-2,4.2470154E-1,1.091144E0,7.300193E-3,3.05549E-2,-2.5234437E-2,-4.3630842E-2,-8.500939E-3,-2.9292386E-2,-5.0847054E-2,-7.698113E-2,-3.441194E-2,-2.1357201E-2,-7.6197176E-3,9.131878E-3,3.8244042E-3,-1.9771801E-2,-1.8318975E-2,-3.3021484E-2,-1.6819876E-2,-3.171332E-3,-1.47554865E-2,-3.614761E-2,-3.6377327E-3,-1.9086443E-2,-3.5690863E-2,-1.4508022E-2,-2.7694875E-2,-6.1516788E-2,1.0454507E-2,-5.2836225E-3,6.4858077E-3,2.0011839E-2,8.006469E-3,-2.8502487E-3,8.727613E-3,2.6417486E-2,-2.5190484E-2,-9.394082E-3,-3.7013404E-2,-4.106744E-3,-5.6144674E-3,3.0448768E-2,-1.6382191E-2,-5.7627195E-3,7.1954224E-3,1.87048E-2,-1.5031501E-2,-3.4741882E-2,5.4933242E-3,-1.2829099E-2,-4.9090735E-3,9.933474E-3,-2.0423489E-2,-8.4694475E-4,1.5097896E-2,2.9911326E-2,6.607695E-3,2.0749427E-2,1.0217073E-2,2.7914017E-2,-6.7159845E-3,2.9156309E-2,2.5535481E-2,3.846386E-2,2.087439E-2,-1.3331809E-2,1.6034933E-2,-2.219152E-3,4.0116373E-2,1.4533908E-2,2.4894895E-2,2.9294665E-3,5.3171974E-2,-6.012525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,-1,83,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,-1,-1,119,121,123,125,-1,-1,127,-1,129,131,133,135,137,139,141,143,145,147,149,151,-1,153,155,157,-1,-1,159,161,163,-1,165,167,-1,169,171,173,-1,-1,175,177,-1,-1,179,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4856476E2,5.457962E1,9.9843376E1,1.784143E1,3.0797182E1,4.5314144E1,3.0048813E1,9.058945E0,2.4638233E0,3.0299782E1,3.074932E0,1.2386562E1,1.6301819E1,2.1457344E1,9.411118E0,1.270015E1,2.349822E0,1.0588207E0,8.4173036E-1,1.2625309E1,0E0,1.8153687E0,4.6014786E-1,8.096428E0,6.7195163E0,3.6206346E0,1.9640419E1,1.1170774E1,6.2793274E0,1.8768163E0,2.9359283E0,4.822052E0,1.1143723E0,3.5808948E-1,1.2956285E0,6.4084053E-1,1.9884986E-1,4.794681E-1,0E0,6.5966415E0,1.2925568E0,7.168405E-1,8.8419724E-1,0E0,0E0,1.8719711E0,4.4681764E0,0E0,4.5351923E-1,1.645381E0,7.9254586E-1,5.834842E0,1.1355621E1,6.9062214E0,3.068581E0,4.1932297E0,3.8768387E-1,1.0359316E0,1.2680173E0,2.6156158E0,5.7164717E-1,2.187851E0,1.3039918E0,0E0,1.567543E0,0E0,0E0,2.1824074E-1,1.8322968E-1,4.910903E-1,3.971653E-1,0E0,0E0,3.7592125E-1,0E0,6.104145E0,1.4498109E0,5.337391E-1,1.4731216E-1,5.519434E-1,1.1438781E-1,1.16838366E-1,5.947046E-1,2.5897245E0,7.468929E-1,2.5191545E0,3.0667686E0,0E0,1.7918271E-1,5.753937E-1,4.0748137E-1,0E0,0E0,3.037404E0,3.105428E0,5.3431816E0,0E0,1.8282342E0,1.6361084E0,0E0,1.4521673E0,1.6768799E0,1.7483073E0,0E0,0E0,2.7344447E-1,8.526659E-1,0E0,0E0,2.7943885E-1,2.5063477E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,64,64,67,67,68,68,69,69,70,70,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,89,89,90,90,93,93,94,94,95,95,97,97,98,98,100,100,101,101,102,102,105,105,106,106,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,-1,84,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,-1,-1,120,122,124,126,-1,-1,128,-1,130,132,134,136,138,140,142,144,146,148,150,152,-1,154,156,158,-1,-1,160,162,164,-1,166,168,-1,170,172,174,-1,-1,176,178,-1,-1,180,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,2.376302E8,1E0,1.5343539E0,2.3770695E5,1.2727361E6,5.121E3,1.5445488E8,1.3144558E4,2.5727358E0,1.5106794E3,1.2E1,1.3359244E9,3.564E3,3.1826714E7,2.1693E5,1.774E3,2.437E3,1E0,8.3161615E-2,2.0783027E5,1.8061392E0,1.24E2,1E0,2.5422776E3,9.27E2,4.3702424E7,7.466243E0,3.1984328E5,2E0,5.492982E2,1.0198864E0,1.4783E4,3.2703E4,3.3449508E9,3.771981E6,3.30399E6,5.7592033E-3,2.8593832E-1,1.3983857E10,2.1442623E1,2.4042394E5,1.2650997E-2,3.968274E-2,1.4504054E8,1E0,5.7468224E-2,5.6158E4,2.6476662E6,6.3997424E-1,4.12E0,8.426E3,2.6477592E6,1.2354571E0,5.0714142E2,9E0,1.0061821E5,1.20086E5,1.2045951E6,5.012E3,1.2327E4,1.6256282E7,-1.8480714E-2,4.3927447E11,-5.4962873E-3,1.27909845E-2,1.2068E4,4.9386E4,1.36E2,1.0376E4,-1.1907787E-2,3.4008247E-3,1.5377966E6,-2.2562281E-3,7.8099336E11,7.76264E7,2.9799202E3,1.5485038E5,1.0275E4,2.6976675E-1,1.0824E4,3.333E3,3.3151623E1,1E0,3.97E2,1.4735735E2,-4.1071237E-3,1.2204E4,4.877E4,1.8955729E1,-5.3983373E-3,2.1046968E-2,1.0046401E6,1E0,2.3314082E6,7.359985E-2,2.0601392E3,2.428818E6,-3.0668985E-2,5.094239E3,6.4683E4,1.9307388E7,5.765489E-2,2.4399783E-2,2.6666667E0,2E0,1.35738505E-2,-2.1237178E-2,1E1,1.3674345E4,7.300193E-3,3.05549E-2,-2.5234437E-2,-4.3630842E-2,-8.500939E-3,-2.9292386E-2,-5.0847054E-2,-7.698113E-2,-3.441194E-2,-2.1357201E-2,-7.6197176E-3,9.131878E-3,3.8244042E-3,-1.9771801E-2,-1.8318975E-2,-3.3021484E-2,-1.6819876E-2,-3.171332E-3,-1.47554865E-2,-3.614761E-2,-3.6377327E-3,-1.9086443E-2,-3.5690863E-2,-1.4508022E-2,-2.7694875E-2,-6.1516788E-2,1.0454507E-2,-5.2836225E-3,6.4858077E-3,2.0011839E-2,8.006469E-3,-2.8502487E-3,8.727613E-3,2.6417486E-2,-2.5190484E-2,-9.394082E-3,-3.7013404E-2,-4.106744E-3,-5.6144674E-3,3.0448768E-2,-1.6382191E-2,-5.7627195E-3,7.1954224E-3,1.87048E-2,-1.5031501E-2,-3.4741882E-2,5.4933242E-3,-1.2829099E-2,-4.9090735E-3,9.933474E-3,-2.0423489E-2,-8.4694475E-4,1.5097896E-2,2.9911326E-2,6.607695E-3,2.0749427E-2,1.0217073E-2,2.7914017E-2,-6.7159845E-3,2.9156309E-2,2.5535481E-2,3.846386E-2,2.087439E-2,-1.3331809E-2,1.6034933E-2,-2.219152E-3,4.0116373E-2,1.4533908E-2,2.4894895E-2,2.9294665E-3,5.3171974E-2,-6.012525E-4],"split_indices":[20,56,7,102,42,28,28,2,7,52,35,52,18,7,2,45,29,10,2,104,0,33,42,29,8,52,2,45,35,33,6,55,56,9,2,31,1,9,0,42,5,56,28,0,0,31,79,0,12,32,27,54,2,47,53,58,17,33,9,47,2,9,45,0,31,0,0,9,2,11,9,0,0,33,0,31,45,4,32,9,38,10,2,55,68,2,52,0,2,12,58,0,0,45,102,28,0,52,45,0,4,2,43,0,0,53,6,0,0,3,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.652E3,8.53E2,1.799E3,4.06E2,4.47E2,1.171E3,6.28E2,2.84E2,1.22E2,3.63E2,8.4E1,5.06E2,6.65E2,4.5E2,1.78E2,2.44E2,4E1,6.9E1,5.3E1,3.57E2,6E0,6.9E1,1.5E1,4.73E2,3.3E1,4.8E1,6.17E2,2.96E2,1.54E2,3.8E1,1.4E2,1.95E2,4.9E1,8E0,3.2E1,5.9E1,1E1,4.7E1,6E0,3.04E2,5.3E1,3.2E1,3.7E1,3E0,1.2E1,1.22E2,3.51E2,7E0,2.6E1,3.9E1,9E0,2.91E2,3.26E2,2.62E2,3.4E1,1.28E2,2.6E1,3E1,8E0,1.25E2,1.5E1,1.66E2,2.9E1,2E0,4.7E1,4E0,4E0,2.8E1,4E0,1.6E1,4.3E1,8E0,2E0,3.8E1,9E0,1.74E2,1.3E2,4.2E1,1.1E1,2.4E1,8E0,8E0,2.9E1,1E2,2.2E1,1.7E1,3.34E2,6E0,2E1,2.5E1,1.4E1,4E0,5E0,2.11E2,8E1,3.18E2,8E0,1.69E2,9.3E1,1.4E1,2E1,1.16E2,1.2E1,2.3E1,3E0,1.2E1,1.8E1,5E0,3E0,8E0,1.17E2,4E0,1.1E1,2.2E1,1.44E2,1.5E1,1.4E1,1.9E1,2.8E1,1.2E1,1.6E1,2E0,2E0,2E0,1.4E1,1E1,3.3E1,3.2E1,6E0,1.34E2,4E1,1.15E2,1.5E1,3.7E1,5E0,3E0,8E0,6E0,1.8E1,4E0,4E0,5E0,3E0,7E0,2.2E1,5.6E1,4.4E1,2E1,2E0,7E0,1E1,8.3E1,2.51E2,1.6E1,4E0,7E0,1.8E1,3E0,1.1E1,3.8E1,1.73E2,2.8E1,5.2E1,2.44E2,7.4E1,1.45E2,2.4E1,1.6E1,7.7E1,1.8E1,2E0,6.7E1,4.9E1,5E0,7E0,1E1,2E0,1.2E1,6E0,6E0,2E0,1.15E2,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[-1.0558123E-3,-4.143971E-1,1.9785956E-1,-6.403888E-1,-1.398694E-1,2.093154E-2,4.9939817E-1,-7.546526E-1,-1.3248697E-1,-3.2637885E-1,2.3420908E-1,-1.4117453E-1,1.8300101E-1,3.6262414E-1,7.9521173E-1,-2.7618825E-1,-8.307267E-1,8.0190375E-2,-2.1499006E-1,-3.6164734E-1,-3.9229605E-2,4.894443E-2,5.370642E-1,-4.9477413E-1,-7.929514E-2,2.2702083E-1,-4.08138E-1,4.4598362E-1,-1.8994002E-1,1.1016883E0,6.6323334E-1,-4.501161E-1,2.176271E-1,-7.8177416E-1,-1.3843228E0,2.1360764E-2,-4.4108316E-1,-4.6004504E-1,-2.4167927E-1,-1.9486237E-1,9.124618E-2,-1.9222733E-1,1.3373208E-1,3.4146073E-1,6.560845E-1,-5.3371245E-1,2.3498158E-1,-2.9008928E-1,-4.3340805E-3,6.1461017E-2,3.5540074E-1,-5.317152E-1,1.5263522E-1,3.1257325E-1,6.970049E-1,-1.1616825E-2,-4.140649E-1,1.166689E0,4.2429814E-1,4.58092E-1,8.3232784E-1,-5.623407E-1,-1.651446E-1,3.387619E-2,3.624483E-2,-8.504915E-1,-4.3898076E-1,-7.3438324E-2,-4.1045845E-2,-9.511301E-2,4.719409E-1,-3.549145E-1,-4.991228E-2,-5.6477106E-1,-2.7373558E-1,-2.6465765E-1,-1.4991928E-2,-7.134505E-4,-1.404091E-2,3.4773387E-2,1.6435781E-2,9.663403E-3,-2.6505923E-1,2.3398513E-1,-2.4001842E-2,4.2059473E-3,2.0931056E-2,8.351251E-1,4.472286E-1,-4.4688103E-1,-8.1279576E-1,2.1603137E-2,-2.847481E-3,-3.7856635E-1,-1.1257467E-1,1.1313146E0,-5.68371E-2,-2.5072476E-2,3.072867E-1,2.87979E-1,6.9488806E-1,-3.2454398E-1,-3.3887777E-2,-5.464966E-3,2.1450773E-2,1.6604684E-1,4.1044426E-1,3.4868482E-1,8.0315614E-1,-1.2776924E-2,1.1645597E-1,-2.6567418E-2,-2.1988136E-1,1.3049594E0,7.7271825E-1,2.8797045E-2,-5.82057E-4,4.8199612E-1,-2.7213732E-3,9.2071515E-1,1.1270674E-1,4.2334455E-3,-2.9396053E-2,3.4408124E-5,-1.4722298E-2,-9.859364E-3,5.0407504E-3,-5.138792E-2,-3.758987E-2,-2.3569454E-2,-5.672048E-3,1.1335611E-2,-7.5813043E-3,2.8859437E-2,4.6582654E-4,-2.4400096E-2,-5.545852E-3,-2.243816E-2,-3.3757295E-2,-2.9116474E-2,-1.0570923E-2,-1.4962618E-2,-6.98966E-3,3.8287563E-3,-5.1450375E-3,-2.4265123E-3,5.54898E-3,4.7967347E-5,-1.5085914E-2,1.6452364E-3,1.6111292E-2,3.6997618E-3,-5.910264E-3,2.0549277E-2,4.5197498E-2,4.894936E-3,2.4854504E-2,-2.4199603E-2,-1.0647177E-2,-2.8591735E-2,-5.4826915E-2,-1.4884704E-2,-3.0234348E-2,-2.488721E-4,-1.28053315E-2,1.906669E-2,5.783344E-2,-4.5019253E-3,1.7654154E-2,-6.1487914E-3,6.4180307E-3,1.683762E-2,6.3912785E-3,1.5927361E-2,-1.9491625E-3,2.493711E-2,4.2382292E-2,-1.7435376E-2,-8.97773E-4,1.1015172E-2,-7.805209E-3,1.8117836E-2,4.0461455E-2,2.0523792E-2,-4.5723403E-3,4.2356595E-2,1.0133508E-2,-1.7275623E-3,1.7943593E-2,-1.2590871E-2,5.3550483E-4,4.098259E-2,6.8299994E-2,1.6829154E-2,4.0127397E-2,2.503365E-2,1.087695E-2,4.5116853E-2,5.518465E-3,2.1798437E-2,-4.745412E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,-1,125,127,129,-1,-1,131,133,135,-1,137,139,141,143,-1,-1,145,-1,-1,147,149,151,-1,-1,153,155,157,159,-1,-1,161,163,165,167,169,171,173,175,177,-1,-1,-1,179,181,183,185,-1,187,-1,189,191,193,-1,-1,195,-1,197,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2067807E2,5.4010193E1,9.666088E1,2.7640625E1,2.763526E1,3.005524E1,2.681958E1,1.396608E1,1.36279335E1,2.6594276E0,7.380015E0,1.2476428E1,1.4977144E1,2.127298E1,7.8039246E0,4.821811E0,8.023346E0,0E0,4.6050286E0,2.6628933E0,6.2696964E-1,1.722923E0,9.4001293E-1,2.5407276E0,7.7098026E0,1.1306835E1,2.8531098E0,1.3177673E1,2.4331565E0,2.2348862E0,4.821953E0,1.2183456E0,1.3068138E0,6.9525146E0,7.301674E-1,2.3015606E0,1.9092503E0,2.3428688E0,5.585947E-1,2.4336326E-1,2.29946E-1,6.848045E-1,9.8604107E-1,4.7081447E-1,7.483177E-1,1.5996666E0,3.4498167E-1,1.9677477E0,2.1588919E1,4.9853888E0,6.6665344E0,9.7287464E-1,6.8123215E-1,3.7034492E0,4.8224487E0,1.1749153E0,6.156416E-1,2.0422058E0,5.481075E-1,8.889942E-1,5.0482635E0,1.0257034E0,3.0698845E-1,0E0,2.2217424E-1,2.9229736E0,8.6207867E-1,0E0,0E0,7.7134246E-1,5.377228E-1,1.4195013E0,0E0,7.619953E-1,7.16192E-1,5.101919E-1,1.05129294E-1,0E0,0E0,1.0980317E-1,0E0,0E0,2.5281549E-1,7.6550436E-1,2.6118183E-1,0E0,0E0,2.5672817E-1,3.4307337E-1,7.2739315E-1,6.0613155E-1,0E0,0E0,1.2595434E0,7.3004407E-1,4.1431427E-1,5.506197E0,2.877227E0,4.2280912E-1,3.7075863E0,1.1615791E0,1.9863915E-1,0E0,0E0,0E0,2.289271E0,1.8715801E0,1.2470903E0,4.862068E0,0E0,9.5879376E-1,0E0,1.3218832E-1,9.574661E-1,1.7239189E-1,0E0,0E0,5.768137E-1,0E0,1.130661E0,7.7042854E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,64,64,65,65,66,66,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,82,82,83,83,84,84,87,87,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,105,105,106,106,107,107,108,108,110,110,112,112,113,113,114,114,117,117,119,119,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,-1,126,128,130,-1,-1,132,134,136,-1,138,140,142,144,-1,-1,146,-1,-1,148,150,152,-1,-1,154,156,158,160,-1,-1,162,164,166,168,170,172,174,176,178,-1,-1,-1,180,182,184,186,-1,188,-1,190,192,194,-1,-1,196,-1,198,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,2.9799202E3,4.197889E6,2.2178302E5,1.3664E4,8.2608955E-3,1.4E1,2.262586E6,1.1925198E3,8.2E1,4.898012E-1,2.2075728E7,4.52737E5,6.325655E6,1.1855755E8,8.0190375E-2,7.218466E7,1.8329959E-1,2.34E0,6.7699623E-1,1.948181E0,1.6006084E3,6.76E2,8.46E2,2.5422776E3,6.167018E6,5.9767612E7,4.5481584E7,6.4683E4,2.692487E6,2.66E2,4.7777777E0,1.3528846E1,9.908038E1,9.22E4,1.4757962E1,9.705292E6,9.912942E6,9.075E3,6E0,1.4187837E-1,3.368326E11,2.4489644E0,1.4504054E8,5.202E3,1.325204E6,8E0,2.3314082E6,2.9251662E6,4.352552E9,8.32E3,1E0,3.171E3,4.492E3,9.53882E-2,7.8E1,5.5743E4,2.3817926E7,7.26E2,1.21E2,4.76E3,3.387619E-2,1.4E1,8.857896E4,3.65625E1,-7.3438324E-2,-4.1045845E-2,1E0,1.7390422E7,3.4409692E3,-4.991228E-2,1E0,1.06E3,1.4930757E0,1.43709E0,-7.134505E-4,-1.404091E-2,3.624992E5,1.6435781E-2,9.663403E-3,9.5411836E4,2.695E3,5.884028E0,4.2059473E-3,2.0931056E-2,1E0,2.862988E5,2.7523365E0,2.5544708E0,2.1603137E-2,-2.847481E-3,1.5E1,5.3296334E-1,1.279012E7,1.4777102E9,2.6883545E2,1.6E1,1.0238709E1,7.130317E7,1.6503105E3,-3.3887777E-2,-5.464966E-3,2.1450773E-2,5.743459E-3,1.7474695E0,2.067698E0,5.3411217E0,-1.2776924E-2,2.1872402E5,-2.6567418E-2,3.1E1,3.0720797E-1,2.73E2,2.8797045E-2,-5.82057E-4,3.5296965E-1,-2.7213732E-3,1.43761E8,4.0728608E3,4.2334455E-3,-2.9396053E-2,3.4408124E-5,-1.4722298E-2,-9.859364E-3,5.0407504E-3,-5.138792E-2,-3.758987E-2,-2.3569454E-2,-5.672048E-3,1.1335611E-2,-7.5813043E-3,2.8859437E-2,4.6582654E-4,-2.4400096E-2,-5.545852E-3,-2.243816E-2,-3.3757295E-2,-2.9116474E-2,-1.0570923E-2,-1.4962618E-2,-6.98966E-3,3.8287563E-3,-5.1450375E-3,-2.4265123E-3,5.54898E-3,4.7967347E-5,-1.5085914E-2,1.6452364E-3,1.6111292E-2,3.6997618E-3,-5.910264E-3,2.0549277E-2,4.5197498E-2,4.894936E-3,2.4854504E-2,-2.4199603E-2,-1.0647177E-2,-2.8591735E-2,-5.4826915E-2,-1.4884704E-2,-3.0234348E-2,-2.488721E-4,-1.28053315E-2,1.906669E-2,5.783344E-2,-4.5019253E-3,1.7654154E-2,-6.1487914E-3,6.4180307E-3,1.683762E-2,6.3912785E-3,1.5927361E-2,-1.9491625E-3,2.493711E-2,4.2382292E-2,-1.7435376E-2,-8.97773E-4,1.1015172E-2,-7.805209E-3,1.8117836E-2,4.0461455E-2,2.0523792E-2,-4.5723403E-3,4.2356595E-2,1.0133508E-2,-1.7275623E-3,1.7943593E-2,-1.2590871E-2,5.3550483E-4,4.098259E-2,6.8299994E-2,1.6829154E-2,4.0127397E-2,2.503365E-2,1.087695E-2,4.5116853E-2,5.518465E-3,2.1798437E-2,-4.745412E-3],"split_indices":[20,102,33,4,51,28,2,27,0,1,52,29,27,45,29,51,45,0,45,41,54,42,42,55,2,2,52,47,45,47,2,1,0,54,56,56,10,58,9,45,2,3,27,31,42,31,2,9,18,28,28,5,2,100,2,2,42,8,10,43,8,2,0,0,3,32,56,0,0,75,32,52,0,100,2,42,53,0,0,47,0,0,33,2,53,0,0,8,28,53,53,0,0,8,38,7,7,4,3,53,7,52,0,0,0,57,38,38,54,0,33,0,3,38,0,0,0,27,0,43,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.682E3,8.71E2,1.811E3,4.77E2,3.94E2,1.142E3,6.69E2,3.89E2,8.8E1,2.63E2,1.31E2,5.71E2,5.71E2,4.59E2,2.1E2,5.4E1,3.35E2,3E0,8.5E1,2.34E2,2.9E1,8.2E1,4.9E1,8.4E1,4.87E2,5.32E2,3.9E1,3.99E2,6E1,6.1E1,1.49E2,4E1,1.4E1,3.1E2,2.5E1,4.2E1,4.3E1,1.27E2,1.07E2,1.3E1,1.6E1,2.1E1,6.1E1,2E1,2.9E1,8E1,4E0,1.27E2,3.6E2,2.33E2,2.99E2,3.2E1,7E0,2.62E2,1.37E2,3.4E1,2.6E1,5.5E1,6E0,6.9E1,8E1,2.8E1,1.2E1,3E0,1.1E1,2.57E2,5.3E1,1.8E1,7E0,3.4E1,8E0,3.9E1,4E0,8E1,4.7E1,9.7E1,1E1,5E0,8E0,1.4E1,2E0,3E0,1.8E1,3.7E1,2.4E1,6E0,1.4E1,1.4E1,1.5E1,6.3E1,1.7E1,2E0,2E0,8.4E1,4.3E1,1.5E1,3.45E2,1.73E2,6E1,2.51E2,4.8E1,1.6E1,1.6E1,4E0,3E0,1.06E2,1.56E2,3.3E1,1.04E2,1.1E1,2.3E1,1.4E1,1.2E1,3.9E1,1.6E1,4E0,2E0,6.6E1,3E0,7.1E1,9E0,2E0,2.6E1,6E0,6E0,2E0,9E0,5.5E1,2.02E2,4.5E1,8E0,5E0,2.9E1,6E0,2E0,2.3E1,1.6E1,5E1,3E1,5E0,4.2E1,6.8E1,2.9E1,5E0,5E0,7E0,7E0,3E0,1.5E1,1.3E1,2.4E1,1.2E1,1.2E1,4E0,1E1,3E0,1.2E1,4.9E1,1.4E1,1.2E1,5E0,6.8E1,1.6E1,2.6E1,1.7E1,2E0,1.3E1,3.18E2,2.7E1,1.05E2,6.8E1,4.7E1,1.3E1,2.21E2,3E1,2.7E1,2.1E1,1.4E1,2E0,8.9E1,1.7E1,1.47E2,9E0,2.8E1,5E0,9.1E1,1.3E1,1.5E1,8E0,1E1,2E0,1E1,2.9E1,3E0,1.3E1,5.6E1,1E1,6.9E1,2E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[3.5006138E-3,-3.969561E-1,1.9679224E-1,-6.2508416E-1,-1.5343739E-1,4.1693296E-2,5.272067E-1,-7.534093E-1,-3.267084E-1,-1.9846436E-1,1.1858184E0,-1.0494695E-1,2.1011755E-1,3.9100263E-1,8.236596E-1,-8.107093E-1,-4.1659498E-1,-4.236018E-1,-1.3620822E-1,-3.8936618E-1,1.02756895E-1,8.169178E-2,2.790128E-2,-3.9404657E-1,-4.7355063E-2,2.4980399E-1,-3.0136E-1,1.8116954E-1,6.020342E-1,8.882204E-1,2.5630575E-1,-7.271806E-1,-1.1708956E0,-1.2824127E-1,-5.853325E-1,-4.7424787E-1,-2.0671944E-1,-2.752362E-1,-4.3616913E-2,-3.094976E-1,-7.947965E-1,-7.3902115E-2,2.8705993E-1,-2.509585E-2,-3.1526917E-1,9.063E-1,-8.476967E-2,6.858692E-2,4.2808235E-1,-4.4440764E-1,9.329305E-2,2.5746968E-1,-3.0217668E-1,-7.235705E-1,6.469938E-1,3.7384248E-1,9.4949603E-1,1.23258114E-1,2.3264548E-2,-8.129806E-1,-4.990952E-1,-8.2292154E-2,-1.0545468E0,1.9247614E-1,-2.9107055E-1,-1.1946179E-2,-7.174048E-1,-5.1234645E-1,-2.7709347E-1,1.2152324E-3,-3.442226E-1,-1.489933E-2,-3.013421E-3,-1.2386474E-1,1.062534E-1,-3.4466365E-1,2.580313E-1,-6.9528145E-1,-1.1183861E0,-3.0659411E-2,-3.2719985E-1,1.00163855E-1,4.189601E-1,-3.3341008E-1,-1.01779506E-1,1.0792022E0,-2.3506928E-2,-4.4020485E-2,-4.9643812E-1,1.4205422E-1,-1.4020038E-1,4.0156215E-1,7.3178805E-2,-2.2734977E-1,-2.8185548E-2,-7.2610513E-3,1.1239938E-2,2.1631588E-1,5.338431E-1,-3.4070706E-1,6.344477E-3,-4.2585578E-2,-9.879125E-3,5.5545664E-1,9.7280335E-1,5.2993053E-1,-1.6472928E-2,9.8488903E-1,3.0523643E-1,1.4037281E-2,-8.846869E-2,-4.1305944E-2,-2.4568625E-2,-1.3011487E-2,-3.1203024E-2,-6.1274856E-2,-4.379243E-2,-3.6355834E-3,1.9766707E-2,-1.6897934E-2,1.5544799E-3,-2.4753502E-2,-4.441104E-2,-1.2627219E-2,-2.63906E-2,-1.5624962E-2,-3.4768959E-3,-2.2406809E-2,-9.054172E-3,5.626869E-3,-9.173333E-3,8.435628E-3,-6.6489354E-3,-1.1311458E-2,-2.6499942E-2,-5.2402285E-3,2.4010714E-2,-3.616597E-2,-9.4264895E-3,-6.018833E-2,-2.2450307E-2,-4.5309556E-3,5.6982473E-3,-3.5137131E-3,-2.1308593E-2,-3.2591121E-3,9.339231E-3,-1.9096164E-3,2.340804E-2,-1.6591502E-2,-5.1571047E-3,-1.2474722E-2,6.6084776E-4,2.205711E-2,5.811348E-2,-4.5268578E-3,7.5387587E-3,-3.85709E-2,-1.4416413E-2,4.192362E-3,1.4656126E-2,-1.7306225E-2,-6.2065425E-5,9.208459E-3,2.2508822E-2,-2.0943552E-2,6.0102227E-3,5.4899757E-3,1.3670353E-2,3.6588386E-2,1.7983042E-2,-2.0728165E-2,-1.0310648E-2,2.9113524E-2,9.42248E-3,3.8484946E-2,7.520287E-2,2.9169668E-2,6.7132157E-3,3.7481792E-2,5.091276E-2,2.1644149E-2,4.2413422E-4,2.7449136E-3,-1.0159337E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,99,101,103,105,107,109,-1,111,113,-1,115,117,119,-1,121,123,125,-1,127,-1,-1,129,131,133,135,137,139,141,143,145,147,149,151,153,-1,155,157,159,161,163,-1,165,-1,-1,-1,167,169,171,-1,-1,-1,173,175,177,-1,179,181,-1,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.087618E2,4.8680023E1,9.319619E1,1.7098831E1,2.592427E1,3.0622591E1,2.3137619E1,5.7608795E0,2.4895363E0,2.3789032E1,3.3974476E0,1.1020151E1,1.1787647E1,1.76035E1,6.4388046E0,7.2416534E0,2.2604294E0,9.015293E-1,6.0588837E-1,7.9464417E0,5.26318E0,0E0,0E0,9.611225E-1,1.9843895E1,1.7287235E1,2.4125354E0,7.527437E0,1.232827E1,4.782608E0,5.507587E-1,3.9191513E0,1.3089523E0,1.0482457E0,1.1518774E0,4.063568E-1,6.1509585E-1,1.2514591E-1,3.7396407E-1,4.3399353E0,5.4268837E-1,8.990904E-1,1.9055133E0,0E0,2.4104118E-1,5.6026897E0,8.936079E0,4.1154394E0,7.2626076E0,8.3604E-1,4.5117146E-1,1.874939E0,5.1406455E-1,4.1640782E-1,5.215622E0,2.311079E0,3.0057068E0,5.294074E-1,0E0,1.9196701E0,2.0070906E0,0E0,1.8524551E-1,4.451652E-1,2.7126873E-1,0E0,4.0475273E-1,4.3234825E-1,1.1708343E-1,0E0,1.4177966E-1,0E0,0E0,3.4922487E-1,2.1287902E-1,4.419035E0,1.2172432E0,8.664665E-1,1.3009548E-1,7.002927E-1,3.2599795E-1,5.557446E-1,1.4931831E0,1.5396309E-1,1.3199444E-1,1.019228E0,0E0,4.969372E0,2.6554832E0,1.7431588E0,2.1500363E0,3.7054596E0,0E0,1.0921187E0,0E0,0E0,0E0,1.0498285E0,5.1486444E-1,2.0717835E-1,0E0,0E0,0E0,2.7595863E0,2.8831291E0,4.0826225E-1,0E0,1.3350525E0,3.8734543E-1,0E0,1.3836148E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,62,62,63,63,64,64,66,66,67,67,68,68,70,70,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,90,90,91,91,93,93,97,97,98,98,99,99,103,103,104,104,105,105,107,107,108,108,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,100,102,104,106,108,110,-1,112,114,-1,116,118,120,-1,122,124,126,-1,128,-1,-1,130,132,134,136,138,140,142,144,146,148,150,152,154,-1,156,158,160,162,164,-1,166,-1,-1,-1,168,170,172,-1,-1,-1,174,176,178,-1,180,182,-1,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.126489E7,1E0,1.3144558E4,3.2627738E5,1.2974394E6,5.121E3,1.19507775E-1,6.1392635E-1,1.5151515E-3,2.73E2,7.325843E0,1.2679E4,1.708775E7,4.0844156E7,5.9767612E7,3.011152E6,2.028994E8,1E0,1.5343539E0,8.169178E-2,2.790128E-2,4.1E1,8E0,9.52E2,1.945345E6,3.9712732E7,1.70752E4,3.333E3,1.7708E4,4.4866666E2,3E0,1.6956521E0,4.88954E5,5.638132E0,1.2608696E0,1.638058E2,2.437E3,1.1037509E2,1.0104842E12,1.9756216E-1,1.3097832E3,-2.509585E-2,1.8991614E5,1.9E1,1E0,4.12E0,5.847845E3,7.5179994E-1,4.366359E0,1.9725596E0,3.202814E5,3E1,8.791777E3,1.0963991E7,4.4263797E0,1.7321888E1,2.3264548E-2,3.340245E0,1.6256282E7,-8.2292154E-2,1.141E4,2.632E5,1.0409E4,-1.1946179E-2,1.6E1,2.200144E9,4.785384E5,1.2152324E-3,1.322E3,-1.489933E-2,-3.013421E-3,1.822542E0,4.768812E1,3.791269E7,1.003252E5,2.1169072E3,3.285E3,5.432133E0,4.651395E1,2.4513194E5,1E0,9.213145E0,3.8E1,3.192E3,-2.3506928E-2,2.1975278E8,3.3820656E7,3.321532E6,1E0,5.1586456E7,7.3178805E-2,2.4601164E-5,-2.8185548E-2,-7.2610513E-3,1.1239938E-2,5.794E3,4.1E2,1.6347875E3,6.344477E-3,-4.2585578E-2,-9.879125E-3,3.791269E7,4.516183E1,1.537007E7,-1.6472928E-2,1.067536E3,3.84E2,1.4037281E-2,4.06E2,-4.1305944E-2,-2.4568625E-2,-1.3011487E-2,-3.1203024E-2,-6.1274856E-2,-4.379243E-2,-3.6355834E-3,1.9766707E-2,-1.6897934E-2,1.5544799E-3,-2.4753502E-2,-4.441104E-2,-1.2627219E-2,-2.63906E-2,-1.5624962E-2,-3.4768959E-3,-2.2406809E-2,-9.054172E-3,5.626869E-3,-9.173333E-3,8.435628E-3,-6.6489354E-3,-1.1311458E-2,-2.6499942E-2,-5.2402285E-3,2.4010714E-2,-3.616597E-2,-9.4264895E-3,-6.018833E-2,-2.2450307E-2,-4.5309556E-3,5.6982473E-3,-3.5137131E-3,-2.1308593E-2,-3.2591121E-3,9.339231E-3,-1.9096164E-3,2.340804E-2,-1.6591502E-2,-5.1571047E-3,-1.2474722E-2,6.6084776E-4,2.205711E-2,5.811348E-2,-4.5268578E-3,7.5387587E-3,-3.85709E-2,-1.4416413E-2,4.192362E-3,1.4656126E-2,-1.7306225E-2,-6.2065425E-5,9.208459E-3,2.2508822E-2,-2.0943552E-2,6.0102227E-3,5.4899757E-3,1.3670353E-2,3.6588386E-2,1.7983042E-2,-2.0728165E-2,-1.0310648E-2,2.9113524E-2,9.42248E-3,3.8484946E-2,7.520287E-2,2.9169668E-2,6.7132157E-3,3.7481792E-2,5.091276E-2,2.1644149E-2,4.2413422E-4,2.7449136E-3,-1.0159337E-2],"split_indices":[20,56,12,102,52,28,28,2,38,42,57,2,53,2,45,45,45,9,7,104,42,0,0,2,18,2,12,45,33,2,10,4,3,53,32,56,53,52,2,56,31,38,4,0,28,3,6,54,4,27,56,38,28,3,52,47,42,56,0,54,45,0,9,33,2,0,3,31,28,0,2,0,0,58,58,45,28,52,0,53,58,28,16,53,0,1,0,7,7,28,102,7,0,39,0,0,0,2,0,4,0,0,0,45,54,9,0,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.695E3,8.77E2,1.818E3,4.52E2,4.25E2,1.238E3,5.8E2,3.15E2,1.37E2,4.12E2,1.3E1,6.62E2,5.76E2,3.99E2,1.81E2,2.68E2,4.7E1,9E1,4.7E1,2.52E2,1.6E2,6E0,7E0,1.09E2,5.53E2,5.35E2,4.1E1,2.01E2,1.98E2,1.62E2,1.9E1,2.2E2,4.8E1,1.8E1,2.9E1,7.2E1,1.8E1,1.8E1,2.9E1,2.12E2,4E1,8.2E1,7.8E1,3.9E1,7E1,2E1,5.33E2,2.66E2,2.69E2,3E1,1.1E1,1.74E2,2.7E1,6E0,1.92E2,1.8E1,1.44E2,1.3E1,6E0,1.58E2,6.2E1,6E0,4.2E1,6E0,1.2E1,9E0,2E1,5.9E1,1.3E1,7E0,1.1E1,1.5E1,3E0,1.9E1,1E1,2E2,1.2E1,3.3E1,7E0,7.1E1,1.1E1,3.3E1,4.5E1,6.4E1,6E0,1.8E1,2E0,4.86E2,4.7E1,1.97E2,6.9E1,2.64E2,5E0,1.3E1,1.7E1,4E0,7E0,1.53E2,2.1E1,2.5E1,2E0,4E0,2E0,1.52E2,4E1,1.5E1,3E0,1.36E2,8E0,7E0,6E0,1.34E2,2.4E1,2.6E1,3.6E1,1.3E1,2.9E1,3E0,3E0,1E1,2E0,1.2E1,8E0,9E0,5E1,1E1,3E0,5E0,6E0,4E0,1.5E1,8E0,2E0,1.33E2,6.7E1,5E0,7E0,2.9E1,4E0,5E0,2E0,5E1,2.1E1,4E0,7E0,1.2E1,2.1E1,6E0,3.9E1,6E1,4E0,2E0,4E0,4E0,1.4E1,3.89E2,9.7E1,1.7E1,3E1,1.49E2,4.8E1,2.6E1,4.3E1,6.6E1,1.98E2,8E0,5E0,6.3E1,9E1,7E0,1.4E1,1.3E1,1.2E1,1.32E2,2E1,3.3E1,7E0,1.2E1,3E0,4.1E1,9.5E1,5E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[-3.3234335E-3,-4.003951E-1,1.827159E-1,-6.184015E-1,-1.8858416E-1,1.0493759E-2,4.5210552E-1,-7.303263E-1,-3.3480394E-1,-2.915597E-1,2.9219142E-1,-1.3600777E-1,1.6380881E-1,2.9205087E-1,6.8141043E-1,-6.973475E-1,-1.394606E0,-4.8029998E-1,-1.8372782E-1,-3.1853017E-1,7.898548E-2,4.374563E-2,2.3557529E-1,-2.645788E-1,6.5557584E-3,2.0476851E-1,-4.2835072E-1,-1.3189842E-1,3.9556977E-1,7.987184E-1,3.7321103E-1,-7.577371E-1,-4.065924E-1,-7.445402E-2,-1.6418459E-2,-5.021286E-1,4.87623E-3,-2.1747012E-1,1.1059439E-2,-2.4748187E-1,-6.8584573E-1,1.3946673E-1,5.108134E-1,-4.6650845E-1,-1.8856515E-1,5.481633E-2,-3.301609E-2,-2.0880313E-1,2.4694636E-1,-5.3670335E-1,1.8480417E-1,1.5242167E-2,-4.1621393E-1,2.1529351E-1,5.403843E-1,6.5922534E-1,1.007569E0,2.0209235E-1,6.426898E-1,-6.7556447E-1,-9.0747494E-1,-7.862383E-2,-5.9886676E-1,-5.541575E-1,-1.9697428E-1,-1.1856262E-2,-2.837202E-4,-3.3748367E-1,-7.981044E-2,-9.450117E-1,-5.769568E-1,1.9328591E-1,-1.5110531E-1,5.682029E-1,1.6505509E-3,-3.7037188E-1,-6.415035E-1,-1.2850253E-1,-5.177456E-1,-7.8479916E-2,3.9566454E-1,-8.00756E-2,-3.1888035E-1,2.3583508E-1,6.9133006E-2,-3.1529808E-1,-3.1046988E-2,-3.7625376E-3,2.271746E-2,-2.9632473E-1,1.1873489E-1,-1.0991432E-2,-4.6751893E-1,-4.183315E-1,2.4459964E-1,2.646944E-1,6.5893066E-1,4.5572096E-1,8.463261E-1,1.0476203E0,3.323233E-1,2.869257E-1,-9.752654E-2,4.3432415E-1,9.820525E-1,-2.0829827E-2,-3.506599E-2,-4.4434495E-2,-1.3825222E-2,7.96538E-3,-1.06946E-2,-3.546534E-2,-1.7422864E-2,-6.277315E-3,-2.7267532E-2,-1.3810012E-2,-2.2943935E-3,-1.7560048E-2,1.4054912E-2,-2.1815042E-3,-1.7494887E-2,-4.9511332E-2,-2.1180192E-2,-9.699638E-3,-2.9263008E-2,4.9635353E-3,1.7797928E-2,1.8917877E-3,-1.2115964E-2,7.3791808E-3,3.1220611E-2,-2.1107988E-2,-6.5990756E-3,-3.271612E-2,-3.8631996E-3,-1.6484188E-2,-1.6219511E-3,-4.041705E-2,-1.8179415E-2,-3.0936569E-3,-4.1376576E-2,2.2838445E-2,-1.7189847E-2,-1.1571315E-2,-4.236853E-4,-8.802553E-3,-1.7797694E-2,8.224837E-3,2.0140592E-2,-1.7521737E-2,-1.9933828E-3,-1.7260598E-2,2.8170303E-3,-9.426454E-3,8.978151E-3,-2.4869086E-2,-1.2046495E-2,-9.204677E-3,-3.1405147E-2,1.3242372E-2,-1.268284E-2,2.3406796E-2,9.294755E-3,3.653343E-2,1.5484214E-2,1.7550675E-2,2.7231995E-2,3.490982E-2,5.1433202E-2,5.110159E-2,2.064687E-2,-5.770159E-4,2.4305474E-2,1.630929E-2,-6.964002E-4,-1.0069781E-2,-7.1228476E-4,2.44764E-2,-9.437193E-4,1.6457073E-2,5.199667E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,43,45,47,49,51,53,55,57,59,61,-1,-1,63,-1,65,-1,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,143,145,147,-1,149,-1,-1,-1,151,153,-1,155,157,159,161,163,165,167,169,171,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.947231E2,3.871753E1,8.327402E1,1.28965E1,2.127467E1,2.4639414E1,2.5491913E1,5.511978E0,2.5274916E0,1.9003387E1,2.457982E0,1.0284604E1,1.30882225E1,1.8221294E1,1.0078186E1,4.678543E0,1.6555805E0,8.1354046E-1,8.897147E-1,8.938171E0,0E0,0E0,1.8069031E0,4.440111E0,1.2062186E1,8.808229E0,2.424477E0,3.4367604E0,8.57716E0,5.3663635E0,3.60526E0,2.266159E0,3.168971E0,0E0,0E0,8.117027E-1,0E0,3.5511398E-1,0E0,4.4097385E0,1.0351067E0,8.7367725E-1,4.7248983E-1,1.1082783E0,4.204099E0,0E0,5.0788226E0,6.4045405E-1,5.7225113E0,5.367756E-1,4.94833E-1,1.8086538E0,1.6090441E-1,2.8876882E0,5.8390846E0,4.4499435E0,1.8424606E0,1.3242841E0,1.7655277E0,1.7942123E0,7.4002075E-1,7.4597263E-1,7.9631805E-1,2.4078274E-1,1.22418106E-1,0E0,0E0,3.6250343E0,1.0005627E0,2.1267414E-1,4.1221142E-1,6.96926E-1,1.8477874E-1,4.5422506E-1,0E0,8.232169E-1,5.754328E-1,3.74433E0,1.1666842E0,2.506602E0,1.7034743E0,2.6251405E-1,1.09829664E-1,5.290922E0,0E0,1.5085602E-1,0E0,0E0,0E0,3.3987474E-1,9.479378E-1,0E0,1.0418606E-1,2.498008E-1,2.4143496E0,8.069513E-1,4.2175865E0,4.2644405E-1,9.539528E-1,4.5030975E-1,3.6811483E-1,6.5163326E-1,1.0869281E-1,7.4201035E-1,3.5504723E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,44,46,48,50,52,54,56,58,60,62,-1,-1,64,-1,66,-1,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,144,146,148,-1,150,-1,-1,-1,152,154,-1,156,158,160,162,164,166,168,170,172,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.629299E4,1E0,1.6990049E0,2.3770695E5,8.631E3,1.4299594E8,2.3859661E-2,2.14099E5,1E0,2.657E3,4.898012E-1,1.545974E-1,2.5003334E7,3.773E3,7.431E3,6.54925E6,2.579E3,1E0,7.898548E-2,4.374563E-2,5.2610065E1,8.2E1,1.2E2,3.97E2,2.5422776E3,3.9712732E7,2.0802219E8,6.5029144E5,8.61491E3,6.4876E4,4.0844156E7,-7.445402E-2,-1.6418459E-2,2.064332E2,4.87623E-3,7.2123044E5,1.1059439E-2,4.0823767E-1,8.10114E4,6.298144E0,5.8E1,1.452881E6,4.602015E0,5.481633E-2,1.4777102E9,3.68E0,6.082581E3,1.0508E5,7.431E3,1E0,7.218466E7,4.54E2,3.7253947E5,4.0889E4,2.6284674E2,5.4106476E7,2.4347392E-1,1.2716E4,6.4446027E2,1E0,2.6835732E3,4E0,1.17E2,-1.1856262E-2,-2.837202E-4,1.1037509E2,7.218466E7,1.819149E1,5.503109E8,1.9776634E9,1.4307411E7,2.3292E4,1.6505509E-3,4.3875E1,1.75E2,2.73E2,8E0,5.388794E6,1.30399E5,3.3720784E5,1.6194E4,2.3314082E6,6.9133006E-2,2.8E2,-3.1046988E-2,-3.7625376E-3,2.271746E-2,1E0,4.1891047E5,-1.0991432E-2,9.2593566E-2,5.266129E0,2.015E3,1.2E1,1E0,1.6649964E5,6.7652373E3,1E0,5E0,2.6364462E7,6.0218544E1,1.17684984E8,1.2238596E9,-2.0829827E-2,-3.506599E-2,-4.4434495E-2,-1.3825222E-2,7.96538E-3,-1.06946E-2,-3.546534E-2,-1.7422864E-2,-6.277315E-3,-2.7267532E-2,-1.3810012E-2,-2.2943935E-3,-1.7560048E-2,1.4054912E-2,-2.1815042E-3,-1.7494887E-2,-4.9511332E-2,-2.1180192E-2,-9.699638E-3,-2.9263008E-2,4.9635353E-3,1.7797928E-2,1.8917877E-3,-1.2115964E-2,7.3791808E-3,3.1220611E-2,-2.1107988E-2,-6.5990756E-3,-3.271612E-2,-3.8631996E-3,-1.6484188E-2,-1.6219511E-3,-4.041705E-2,-1.8179415E-2,-3.0936569E-3,-4.1376576E-2,2.2838445E-2,-1.7189847E-2,-1.1571315E-2,-4.236853E-4,-8.802553E-3,-1.7797694E-2,8.224837E-3,2.0140592E-2,-1.7521737E-2,-1.9933828E-3,-1.7260598E-2,2.8170303E-3,-9.426454E-3,8.978151E-3,-2.4869086E-2,-1.2046495E-2,-9.204677E-3,-3.1405147E-2,1.3242372E-2,-1.268284E-2,2.3406796E-2,9.294755E-3,3.653343E-2,1.5484214E-2,1.7550675E-2,2.7231995E-2,3.490982E-2,5.1433202E-2,5.110159E-2,2.064687E-2,-5.770159E-4,2.4305474E-2,1.630929E-2,-6.964002E-4,-1.0069781E-2,-7.1228476E-4,2.44764E-2,-9.437193E-4,1.6457073E-2,5.199667E-2],"split_indices":[20,56,33,102,41,28,2,45,38,2,67,2,27,38,45,2,2,9,2,104,0,0,56,29,0,2,52,45,7,28,4,11,45,0,0,52,0,47,0,42,33,57,3,31,53,0,7,54,52,29,2,8,45,2,28,2,58,45,38,9,52,75,4,10,10,0,0,56,45,56,5,5,51,2,0,4,6,2,17,47,10,28,9,28,0,0,0,0,0,101,47,0,38,54,11,18,6,28,52,6,8,47,56,48,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.634E3,8.4E2,1.794E3,4.13E2,4.27E2,1.095E3,6.99E2,2.95E2,1.18E2,3.52E2,7.5E1,5.6E2,5.35E2,4.13E2,2.86E2,2.83E2,1.2E1,5.9E1,5.9E1,3.48E2,4E0,5E0,7E1,2.94E2,2.66E2,5.01E2,3.4E1,8.1E1,3.32E2,2.06E2,8E1,2.33E2,5E1,1E1,2E0,5.7E1,2E0,5.5E1,4E0,2.93E2,5.5E1,5.3E1,1.7E1,7.9E1,2.15E2,8E0,2.58E2,4.6E1,4.55E2,2.9E1,5E0,5.4E1,2.7E1,1.49E2,1.83E2,1.26E2,8E1,5E1,3E1,1.54E2,7.9E1,1.9E1,3.1E1,4.8E1,9E0,4.8E1,7E0,1.9E2,1.03E2,1.4E1,4.1E1,4.5E1,8E0,1.5E1,2E0,5.3E1,2.6E1,1.83E2,3.2E1,2.34E2,2.4E1,2.2E1,2.4E1,4.52E2,3E0,1.1E1,1.8E1,3E0,2E0,1.3E1,4.1E1,7E0,2E1,6E0,1.43E2,5.6E1,1.27E2,6.2E1,6.4E1,7.5E1,5E0,3.9E1,1.1E1,2E1,1E1,3.1E1,1.23E2,7.6E1,3E0,7E0,1.2E1,1.8E1,1.3E1,2E0,4.6E1,5E0,4E0,1.82E2,8E0,9.3E1,1E1,1.1E1,3E0,4E0,3.7E1,3.1E1,1.4E1,3E0,5E0,3E0,1.2E1,4E1,1.3E1,2.4E1,2E0,5.5E1,1.28E2,8E0,2.4E1,2.31E2,3E0,2.2E1,2E0,6E0,1.6E1,8E0,1.6E1,3.37E2,1.15E2,9E0,2E0,1.1E1,2E0,7E0,3.4E1,1.5E1,5E0,4E0,2E0,1.35E2,8E0,1.2E1,4.4E1,9.6E1,3.1E1,3.7E1,2.5E1,4.5E1,1.9E1,7.2E1,3E0,2E0,3E0,3.3E1,6E0,4E0,7E0,1.7E1,3E0,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[-3.940342E-3,-3.6695775E-1,1.6809095E-1,-5.6298876E-1,-1.3593155E-1,1.564209E-2,4.2950606E-1,8.910751E-1,-5.968302E-1,-3.0664882E-1,1.3832769E-1,-4.1079283E-2,4.251397E-1,5.829088E-3,5.047003E-1,1.0903615E-2,6.668626E-2,-6.972369E-1,-2.7466702E-1,-4.0406275E-1,-2.018088E-1,5.6622706E-2,5.175204E-1,-2.927398E-1,2.9284887E-2,3.3002976E-1,6.5466374E-1,-1.5693237E-1,4.209504E-1,3.0830398E-1,6.360723E-1,-6.7291516E-3,7.599452E-3,-5.989176E-1,-9.480622E-1,-6.0031887E-2,-4.9613988E-1,-3.3868358E-1,-5.465544E-1,-2.5439456E-1,-7.6970585E-2,1.328865E-1,-2.9399994E-1,1.0017676E-2,5.6205034E-1,-3.877959E-1,-1.73844E-1,6.728925E-2,-3.9622977E-1,2.3494028E-1,4.6916774E-1,7.094587E-1,-7.5932906E-4,-2.5370944E-1,2.0974137E-1,6.311919E-1,1.8264557E-1,6.446502E-1,2.2071388E-1,8.776205E-1,5.259085E-1,-7.13896E-1,-2.773012E-1,-1.0215458E0,-4.4469056E-1,-1.4809892E-1,4.2611438E-1,-4.001835E-1,-8.7832963E-1,-2.4752684E-2,-2.5275436E-1,-5.770845E-1,-1.0997533E-2,-2.8429908E-1,6.5235704E-2,-1.8614653E-1,2.9893992E-3,1.8249609E-1,-8.7079085E-2,-6.167658E-1,-8.0740415E-2,2.9726518E-2,1.08569395E-2,-3.567573E-1,-5.8150847E-2,-8.735131E-2,-2.6208434E-1,1.2578143E-1,-1.2658308E-1,-5.849912E-1,-1.4173315E-1,-1.1494045E-2,2.5477195E-1,5.151328E-1,-1.2903897E-2,2.7038762E-2,4.3442495E-2,1.1366823E-2,-2.7510363E-1,-2.273375E-2,3.4142634E-1,2.1020358E-2,4.053637E-2,-9.134987E-3,2.9188937E-1,3.196937E-1,7.2383595E-1,2.654588E-1,7.9278415E-3,4.9981683E-2,9.185026E-1,3.6276573E-1,6.8873954E-1,-3.543028E-2,-1.8861957E-2,8.036962E-4,-1.7629035E-2,-4.466214E-2,-7.040693E-2,-2.9637398E-2,-8.721627E-3,-1.9847872E-3,-1.3808355E-2,2.9624062E-2,1.4793124E-3,-2.4099316E-2,-7.6897545E-3,-1.650373E-2,-4.6390697E-2,-1.3556897E-2,-2.045462E-3,-1.9556813E-2,-3.3732682E-2,1.8238957E-3,-1.4265794E-2,-6.6158082E-3,7.473468E-3,-1.13970395E-2,-2.3217446E-3,2.4009808E-3,1.3477319E-2,-1.09154135E-2,3.666427E-3,-3.746451E-2,-1.5702527E-2,-7.675417E-3,8.526117E-3,-2.0725144E-2,-1.0639216E-2,-9.884179E-3,-2.5906113E-3,-1.7709967E-2,-4.829884E-3,3.5465024E-3,2.2730246E-2,-1.049206E-2,1.18358815E-2,-3.0183103E-2,-8.106523E-3,-1.7917014E-3,-3.7564598E-2,8.360126E-3,1.5763534E-2,2.942993E-2,1.3252408E-2,-1.7089332E-2,-6.645239E-3,1.886281E-2,8.538547E-4,1.702046E-2,-8.199157E-4,2.1661012E-2,6.407878E-3,2.7836595E-2,4.0326063E-2,9.415532E-3,2.2025635E-2,-7.3179584E-3,8.054962E-3,-7.926686E-3,9.535489E-3,4.5518387E-2,1.7147176E-2,1.1750841E-2,2.4499182E-2,3.9547026E-2,1.6478214E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,-1,61,63,65,67,69,71,73,75,77,79,-1,81,83,85,87,89,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,127,-1,129,131,-1,133,135,137,-1,139,141,143,145,-1,-1,147,-1,149,151,153,155,157,159,-1,161,163,-1,-1,-1,-1,165,-1,167,-1,-1,-1,169,171,173,175,177,179,181,183,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6742863E2,3.8961975E1,7.248487E1,2.34626E1,1.8630272E1,2.672882E1,2.1341965E1,4.8392897E0,1.4558884E1,2.4206104E0,4.710932E0,1.7911243E1,2.8401413E0,6.957768E0,1.448465E1,1.341773E-1,0E0,7.9895325E0,5.1991606E0,9.887047E-1,7.6818514E-1,3.4350245E0,2.5315285E-1,2.430275E0,1.2818249E1,1.2260599E0,1.4808922E0,2.6791487E0,1.3376608E0,6.6336536E0,8.585815E0,0E0,0E0,9.120728E0,3.0967255E0,2.5034742E0,1.6288776E0,1.2319984E0,2.3514557E-1,8.214512E-1,5.746952E-1,1.161694E0,1.5333585E0,0E0,3.1178427E-1,2.696331E0,7.4032116E-1,8.260114E0,3.0396318E0,6.451242E-1,1.4968157E0,5.4730225E-1,0E0,6.8513966E-1,1.6041712E0,2.4924755E-1,6.675398E-1,9.77993E-1,1.7497768E0,3.5068283E0,6.019417E0,1.243393E0,1.8221617E0,1.9413147E0,5.275402E-1,7.2091746E-1,6.654923E-1,1.030168E0,1.4840412E-1,0E0,3.8284183E-1,4.425316E-1,0E0,3.485546E-1,1.6932149E-1,1.3831574E-1,0E0,1.1119406E0,4.7840068E-1,1.7386341E-1,3.3723605E-1,0E0,0E0,1.1212406E0,0E0,1.9469771E-1,8.1370425E-1,1.0051939E1,5.896514E0,5.894995E-1,1.9148684E0,0E0,3.0050397E-1,7.303982E-1,0E0,0E0,0E0,0E0,5.857229E-1,0E0,2.3058152E-1,0E0,0E0,0E0,2.4160242E-1,2.1063948E-1,1.3417816E-1,1.9447184E0,8.757941E-1,2.2577769E-1,1.451149E0,1.970725E0,5.242012E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,73,73,74,74,75,75,77,77,78,78,79,79,80,80,83,83,85,85,86,86,87,87,88,88,89,89,90,90,92,92,93,93,98,98,100,100,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,-1,62,64,66,68,70,72,74,76,78,80,-1,82,84,86,88,90,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,128,-1,130,132,-1,134,136,138,-1,140,142,144,146,-1,-1,148,-1,150,152,154,156,158,160,-1,162,164,-1,-1,-1,-1,166,-1,168,-1,-1,-1,170,172,174,176,178,180,182,184,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,2E0,4.273439E2,2.3314082E6,2.5998926E-1,4.676E3,1.983922E3,1.913925E-1,1.4101E4,4.54E2,1E0,2.001177E6,1.0855529E3,5.9E1,6.668626E-2,3.9712732E7,6.747114E7,2.9005406E5,9.36494E5,4.0844156E7,2.1924436E5,7.805608E4,4.0844156E7,3.0872485E2,2.0189162E7,3.202814E5,4.4312353E0,8E0,2.1402586E4,-6.7291516E-3,7.599452E-3,2.291E3,4.7272725E0,9.637832E1,6.0435E4,1.074E3,1.16544716E-1,1.4000518E6,2.046E3,6.788246E0,1.7488463E3,1.0017676E-2,4.8765745E0,2.0116506E4,9.25E0,2.429717E1,3.0387878E3,2.657819E6,1.1328599E3,5.9570195E6,-7.5932906E-4,1.2897E4,4E0,5.8349776E-3,6.4625664E8,1.915148E7,6.205317E-2,3.051E3,2.6312E4,2.1E1,8.2608955E-3,1.2427474E8,6.823E3,5.418831E11,1.49E2,3.4409692E3,2.1897722E10,-2.4752684E-2,3.730007E6,4.914413E-3,-1.0997533E-2,2.28125E0,1.646E3,2.7738693E0,2.9893992E-3,3.386E3,4.0501267E8,1.116567E3,5.5325594E9,2.9726518E-2,1.08569395E-2,5.222222E1,-5.8150847E-2,1.56E2,1.5046341E2,8.5023944E2,7.06699E8,6.8E1,1.6338193E-1,-1.1494045E-2,4.5723195E6,2.0771764E4,-1.2903897E-2,2.7038762E-2,4.3442495E-2,1.1366823E-2,1.6213043E3,-2.273375E-2,8.48E2,2.1020358E-2,4.053637E-2,-9.134987E-3,2.342147E7,2.2444444E0,1.2774637E10,3.794679E8,4.646E3,1.2633843E3,1.4122614E2,2.4042394E5,3.5301748E7,-3.543028E-2,-1.8861957E-2,8.036962E-4,-1.7629035E-2,-4.466214E-2,-7.040693E-2,-2.9637398E-2,-8.721627E-3,-1.9847872E-3,-1.3808355E-2,2.9624062E-2,1.4793124E-3,-2.4099316E-2,-7.6897545E-3,-1.650373E-2,-4.6390697E-2,-1.3556897E-2,-2.045462E-3,-1.9556813E-2,-3.3732682E-2,1.8238957E-3,-1.4265794E-2,-6.6158082E-3,7.473468E-3,-1.13970395E-2,-2.3217446E-3,2.4009808E-3,1.3477319E-2,-1.09154135E-2,3.666427E-3,-3.746451E-2,-1.5702527E-2,-7.675417E-3,8.526117E-3,-2.0725144E-2,-1.0639216E-2,-9.884179E-3,-2.5906113E-3,-1.7709967E-2,-4.829884E-3,3.5465024E-3,2.2730246E-2,-1.049206E-2,1.18358815E-2,-3.0183103E-2,-8.106523E-3,-1.7917014E-3,-3.7564598E-2,8.360126E-3,1.5763534E-2,2.942993E-2,1.3252408E-2,-1.7089332E-2,-6.645239E-3,1.886281E-2,8.538547E-4,1.702046E-2,-8.199157E-4,2.1661012E-2,6.407878E-3,2.7836595E-2,4.0326063E-2,9.415532E-3,2.2025635E-2,-7.3179584E-3,8.054962E-3,-7.926686E-3,9.535489E-3,4.5518387E-2,1.7147176E-2,1.1750841E-2,2.4499182E-2,3.9547026E-2,1.6478214E-2],"split_indices":[20,102,33,17,52,28,42,1,55,41,2,2,100,9,52,29,0,45,45,33,9,45,28,28,45,4,45,28,54,3,32,0,0,2,54,56,10,2,39,33,2,53,4,0,57,33,58,53,4,28,4,45,0,1,3,34,7,12,39,2,2,3,27,45,2,31,8,52,5,0,9,38,0,54,2,56,0,2,7,4,5,0,0,4,0,2,55,52,7,3,38,0,28,47,0,0,0,0,4,0,0,0,0,0,45,56,31,7,2,4,56,28,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.679E3,8.61E2,1.818E3,4.65E2,3.96E2,1.149E3,6.69E2,1E1,4.55E2,2.44E2,1.52E2,1.01E3,1.39E2,1.01E2,5.68E2,4E0,6E0,3.46E2,1.09E2,1.25E2,1.19E2,1.26E2,2.6E1,2.2E2,7.9E2,1E2,3.9E1,7.3E1,2.8E1,2.29E2,3.39E2,2E0,2E0,2.51E2,9.5E1,5.6E1,5.3E1,8.8E1,3.7E1,8.3E1,3.6E1,1.04E2,2.2E1,4E0,2.2E1,1.21E2,9.9E1,7.26E2,6.4E1,6.1E1,3.9E1,3.6E1,3E0,5.8E1,1.5E1,1.4E1,1.4E1,4.6E1,1.83E2,1.04E2,2.35E2,1.84E2,6.7E1,8.2E1,1.3E1,4.8E1,8E0,4.4E1,9E0,2.7E1,6.1E1,3.3E1,4E0,7.6E1,7E0,2E1,1.6E1,8.5E1,1.9E1,8E0,1.4E1,1.8E1,4E0,1.18E2,3E0,5.1E1,4.8E1,5.58E2,1.68E2,3.6E1,2.8E1,2E0,5.9E1,3.7E1,2E0,2.3E1,1.3E1,2E0,5.6E1,2E0,1.3E1,9E0,5E0,3E0,1.1E1,1E1,3.6E1,1.51E2,3.2E1,5E0,9.9E1,1.19E2,1.16E2,1.69E2,1.5E1,1.6E1,5.1E1,7.1E1,1.1E1,7E0,6E0,2.8E1,2E1,5E0,3E0,3E1,1.4E1,2E0,7E0,5.3E1,8E0,1.6E1,1.7E1,3E0,7.3E1,2E0,5E0,1.4E1,6E0,3.7E1,4.8E1,1E1,9E0,4E0,4E0,1.1E1,3E0,7.4E1,4.4E1,1E1,4.1E1,2.8E1,2E1,4.87E2,7.1E1,1.35E2,3.3E1,3.2E1,4E0,2.5E1,3E0,3E1,2.9E1,2.5E1,1.2E1,3.4E1,2.2E1,1.1E1,2E0,9E0,2E0,5E0,5E0,1.9E1,1.7E1,1.13E2,3.8E1,1.6E1,1.6E1,2E0,3E0,9.3E1,6E0,6.8E1,5.1E1,8.2E1,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[4.1392664E-3,-3.3872607E-1,1.652893E-1,-5.57618E-1,-1.4421193E-1,1.926719E-2,4.1324338E-1,-6.5777576E-1,-3.1092346E-1,-2.491947E-1,3.047966E-1,-2.7809837E-1,9.0273544E-2,3.0791126E-3,4.8405278E-1,-5.0943136E-1,-7.7347136E-1,-3.9476103E-1,-1.3337931E-1,-2.6916647E-1,6.142199E-2,2.2677243E-1,5.9562546E-1,-4.024755E-1,-1.5346384E-1,2.1885608E-1,-5.881808E-2,1.4628533E-1,-3.6244982E-1,4.4217822E-1,1.001584E0,-5.823936E-1,-7.901949E-2,-9.6072274E-1,-6.072205E-1,-4.3235815E-1,-4.050857E-3,-3.961166E-2,-2.8701794E-1,-2.0821539E-1,-6.2311614E-1,1.2306538E-1,4.015321E-1,9.960072E-3,6.915271E-1,-3.707553E-1,-4.8232306E-2,5.581945E-2,-2.0989293E-1,5.3908084E-2,1.9266136E-1,-1.9294934E-1,8.057769E-2,1.8580418E-2,3.813336E-1,-1.9697133E-2,-5.10806E-3,1.5266545E-1,4.933439E-1,1.0429623E0,1.615854E-2,-8.7981194E-2,-6.1994034E-1,-1.844539E-1,1.8621355E-2,-8.2348293E-1,-1.4067496E0,-6.905495E-1,-3.8195306E-1,-4.835756E-1,-2.6403704E-1,-7.718894E-2,1.3132087E-2,-1.5709234E-2,-6.538714E-4,-2.941284E-1,-6.207406E-2,-8.669495E-1,-5.161007E-1,1.6730997E-1,-7.0562693E-3,4.6479002E-1,5.421565E-3,3.985905E-2,2.068201E-2,-4.3931797E-1,-2.4799885E-1,-1.8314597E-1,7.611014E-3,-9.0402186E-2,-2.8787932E-1,1.5248914E-2,2.9286507E-1,-4.4649893E-1,-1.092915E-1,6.1810523E-2,5.0528973E-2,-1.555497E-1,1.7860128E-1,2.7916614E-2,1.9463786E-1,-3.5069543E-1,2.0364206E-1,4.1897413E-1,7.446038E-1,5.9212983E-2,4.2020526E-2,-9.865256E-3,5.3175846E-3,-3.2971278E-2,-2.172833E-2,-1.9469347E-2,-1.2623376E-3,-4.178595E-2,-1.601854E-2,-4.2116594E-2,-7.831036E-2,-1.7896062E-2,-3.7041195E-2,-2.3268951E-2,-9.38974E-3,-1.0504885E-2,-2.4819799E-2,-2.1493426E-3,-1.4478454E-2,-6.2234136E-3,2.1348954E-3,-1.5292731E-2,1.3223458E-2,-1.324724E-3,-1.5238406E-2,-4.545038E-2,-1.0641752E-2,-7.231449E-3,-2.5938587E-2,3.9575254E-3,1.4507128E-2,8.22095E-3,2.5078539E-2,-2.2074101E-2,-1.1801965E-2,6.3827382E-3,-1.4590142E-2,-1.4283853E-2,3.128987E-4,-1.574744E-2,-2.858776E-3,-1.9985612E-2,-1.013208E-2,-1.688613E-2,2.4444999E-3,7.499787E-3,1.983767E-2,-2.6241193E-2,-1.7880421E-3,-7.779946E-3,1.3079105E-2,5.9997174E-3,-9.485768E-3,-1.4606297E-2,-1.2939525E-3,-2.2007927E-2,1.2017079E-2,1.4505352E-2,-8.561137E-3,-6.894331E-3,-3.0997986E-2,-1.385513E-2,1.1610181E-2,1.6169662E-2,2.5808292E-2,3.89333E-2,1.6615016E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,-1,83,85,-1,87,89,-1,91,93,95,97,99,-1,-1,101,103,105,-1,107,109,111,-1,113,115,117,119,121,123,125,-1,-1,-1,127,129,131,133,135,-1,137,-1,-1,-1,139,141,143,-1,145,147,149,151,153,155,157,-1,159,161,-1,163,165,167,169,171,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.503435E2,3.6962082E1,6.701078E1,9.901207E0,2.1853785E1,2.4645014E1,1.992234E1,4.5831833E0,1.73353E0,1.1817085E1,1.8617964E0,3.426115E0,1.8071558E1,5.3923144E0,1.2234894E1,4.0364723E0,4.428673E0,9.080267E-1,5.6488407E-1,7.874069E0,0E0,1.2373757E0,5.2535105E-1,1.7747555E0,1.3580179E0,1.1802286E1,8.186281E0,2.2141595E0,3.184576E-1,7.9682007E0,8.298454E-1,2.0060158E0,1.0582736E0,3.3712997E0,1.3881187E0,4.930277E-1,0E0,3.2777315E-1,1.6366434E-1,3.974372E0,9.616375E-1,5.749979E-1,4.2159176E-1,0E0,1.523037E-1,8.0003166E-1,0E0,6.450743E-1,8.126538E-1,0E0,8.759783E0,4.681346E0,3.8668952E0,1.3931395E0,8.650851E-1,0E0,0E0,2.2011654E0,8.302345E0,1.9610214E-1,0E0,2.3425594E-1,8.6101913E-1,5.8616227E-1,0E0,1.12323E0,4.89357E-1,1.3659668E0,4.0700865E-1,3.8121128E-1,1.4585733E-1,1.5998547E-1,0E0,0E0,0E0,2.9620323E0,1.0400273E0,5.83086E-1,2.9093552E-1,4.3427718E-1,0E0,2.7251625E-1,0E0,0E0,0E0,1.698513E-1,9.262886E-1,1.766277E-1,0E0,2.5126445E-1,4.4510698E-1,2.3748798E0,5.144413E0,2.262392E0,3.4799013E0,3.5216599E0,0E0,4.5275396E-1,1.3153472E0,0E0,6.604448E-1,3.973145E-1,1.5233591E0,3.3101387E0,2.5451584E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,47,47,48,48,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,75,75,76,76,77,77,78,78,79,79,81,81,85,85,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,98,98,100,100,101,101,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,-1,84,86,-1,88,90,-1,92,94,96,98,100,-1,-1,102,104,106,-1,108,110,112,-1,114,116,118,120,122,124,126,-1,-1,-1,128,130,132,134,136,-1,138,-1,-1,-1,140,142,144,-1,146,148,150,152,154,156,158,-1,160,162,-1,164,166,168,170,172,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.896243E4,1E0,1.5560952E0,4.54E2,2.5998926E-1,6.7033327E-1,2.0368582E8,1.3144558E4,3.3060927E0,6.6126086E4,2E1,5.9767612E7,7.0484486E0,3.74E3,1.36054E-1,3.7382904E7,5.59E2,1E0,6.142199E-2,5.148149E5,2.1924436E5,2.0116506E4,1.3170732E0,2.04E5,9.51668E5,2.001177E6,5.464732E8,1.0617118E8,6.292039E0,7.9E1,1.3141646E3,2.8681312E8,1.3928872E0,1E0,-4.050857E-3,2.67162E5,1.243766E0,3.4976208E-1,8.10114E4,4.5510144E9,2.8477445E0,9.960072E-3,2.093563E0,4.7E2,-4.8232306E-2,3.25E2,1.2747E4,5.3908084E-2,4.2648572E5,1.5139E4,8.791777E3,1.857051E5,4.822785E0,-1.9697133E-2,-5.10806E-3,4.97E2,2.2214102E6,2.231E3,1.615854E-2,1.4925373E-1,1.5053125E2,3.5912186E-1,1.8621355E-2,2.074E3,5.1160636E11,1.2E1,1.152553E6,2.9883E4,1.0376E4,1E0,1.3132087E-2,-1.5709234E-2,-6.538714E-4,3.0497742E3,7.526755E7,3.4613644E1,5.503109E8,1.4564946E3,-7.0562693E-3,1.308827E6,5.421565E-3,3.985905E-2,2.068201E-2,1.14E2,9.928469E-1,3.3820656E7,7.611014E-3,1.21E2,1.3325E2,1.8731416E4,9.99E2,6.650572E-3,9.9625344E5,4.0844156E7,5.0528973E-2,8.05E2,4E0,2.7916614E-2,1E0,4.6E0,1.2E1,1.3664E4,1E0,5.9212983E-2,4.2020526E-2,-9.865256E-3,5.3175846E-3,-3.2971278E-2,-2.172833E-2,-1.9469347E-2,-1.2623376E-3,-4.178595E-2,-1.601854E-2,-4.2116594E-2,-7.831036E-2,-1.7896062E-2,-3.7041195E-2,-2.3268951E-2,-9.38974E-3,-1.0504885E-2,-2.4819799E-2,-2.1493426E-3,-1.4478454E-2,-6.2234136E-3,2.1348954E-3,-1.5292731E-2,1.3223458E-2,-1.324724E-3,-1.5238406E-2,-4.545038E-2,-1.0641752E-2,-7.231449E-3,-2.5938587E-2,3.9575254E-3,1.4507128E-2,8.22095E-3,2.5078539E-2,-2.2074101E-2,-1.1801965E-2,6.3827382E-3,-1.4590142E-2,-1.4283853E-2,3.128987E-4,-1.574744E-2,-2.858776E-3,-1.9985612E-2,-1.013208E-2,-1.688613E-2,2.4444999E-3,7.499787E-3,1.983767E-2,-2.6241193E-2,-1.7880421E-3,-7.779946E-3,1.3079105E-2,5.9997174E-3,-9.485768E-3,-1.4606297E-2,-1.2939525E-3,-2.2007927E-2,1.2017079E-2,1.4505352E-2,-8.561137E-3,-6.894331E-3,-3.0997986E-2,-1.385513E-2,1.1610181E-2,1.6169662E-2,2.5808292E-2,3.89333E-2,1.6615016E-2],"split_indices":[20,56,33,102,42,2,42,27,7,52,35,28,3,45,35,2,42,12,0,104,0,28,28,33,53,5,1,9,7,7,38,11,52,7,41,105,0,1,39,39,33,5,57,0,39,1,0,2,12,0,28,9,52,28,54,0,0,2,28,0,0,56,52,27,0,2,31,18,32,1,9,8,0,0,0,52,45,56,5,4,0,1,0,0,0,29,53,7,0,2,55,28,2,41,28,45,0,10,3,0,74,54,18,2,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.719E3,8.69E2,1.85E3,4.08E2,4.61E2,1.165E3,6.85E2,2.89E2,1.19E2,3.74E2,8.7E1,2.24E2,9.41E2,1.01E2,5.84E2,1.29E2,1.6E2,8E1,3.9E1,3.7E2,4E0,7E1,1.7E1,1.11E2,1.13E2,5.05E2,4.36E2,7.3E1,2.8E1,5.42E2,4.2E1,1.1E2,1.9E1,7.3E1,8.7E1,7.1E1,9E0,2.5E1,1.4E1,3.17E2,5.3E1,4.5E1,2.5E1,4E0,1.3E1,1.07E2,4E0,2.4E1,8.9E1,1.3E1,4.92E2,2.22E2,2.14E2,4.8E1,2.5E1,2.3E1,5E0,8.2E1,4.6E2,3.9E1,3E0,8E0,1.02E2,1.6E1,3E0,5.8E1,1.5E1,6.2E1,2.5E1,5.3E1,1.8E1,2.3E1,2E0,1.2E1,2E0,1.99E2,1.18E2,1.4E1,3.9E1,3.9E1,6E0,2E1,5E0,7E0,6E0,6.7E1,4E1,7E0,1.7E1,3.6E1,5.3E1,1.78E2,3.14E2,5.4E1,1.68E2,2.11E2,3E0,2.3E1,2.5E1,1.1E1,1.4E1,7E0,7.5E1,3.57E2,1.03E2,1.5E1,2.4E1,5E0,3E0,7E1,3.2E1,6E0,1E1,5.2E1,6E0,6E0,9E0,1.4E1,4.8E1,1.5E1,1E1,7E0,4.6E1,3E0,1.5E1,1.6E1,7E0,1.91E2,8E0,1.05E2,1.3E1,1.2E1,2E0,3E0,3.6E1,2.5E1,1.4E1,4E0,1.6E1,5.9E1,8E0,5E0,3.5E1,4E0,3E0,3E0,3.3E1,1.8E1,3.5E1,1.5E1,1.63E2,1.49E2,1.65E2,4.3E1,1.1E1,1.48E2,2E1,1.7E2,4.1E1,1E1,1.3E1,2E0,2.3E1,1.1E1,3E0,5E0,2E0,5E0,7E1,2.15E2,1.42E2,8.7E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"173","size_leaf_vector":"1"}},{"base_weights":[-4.66578E-3,-3.454011E-1,1.5617731E-1,-5.204079E-1,-1.3271241E-1,7.499103E-3,3.8000122E-1,-6.225245E-1,-2.9486296E-1,-2.3698321E-1,2.4995245E-1,-2.9048836E-1,5.8320526E-2,2.520562E-1,5.814602E-1,-1.2812632E0,-5.90869E-1,-3.857814E-1,-1.22901596E-1,-2.5281426E-1,4.910802E-2,7.1488485E-2,3.7596777E-1,-4.3122056E-1,-2.3508063E-1,1.8283927E-1,-6.138081E-2,3.204046E-1,-1.08374886E-1,1.850741E-1,6.681728E-1,-2.1471651E-2,-6.62192E-2,-6.04459E-1,2.8891351E-2,-3.593749E-1,-3.478693E-2,-1.7141923E-1,4.926316E-3,-2.0135568E-1,-6.425572E-1,1.3153975E-1,-1.2328083E-1,5.1727355E-1,2.0722115E-1,-4.3892553E-3,-2.1321265E-2,-1.8168747E-1,-3.135421E-1,8.92581E-1,1.5690958E-1,-1.18823946E-1,4.45137E-1,2.0745718E-1,4.930029E-1,-3.5799405E-1,1.5118857E-1,5.4417275E-2,2.790752E-1,6.3266456E-1,6.0508516E-2,-4.842061E-1,-7.030904E-1,-3.9791638E-1,-1.9035153E-1,-1.7805612E-2,-2.1607602E-1,-2.7094245E-1,-4.67146E-2,-5.255081E-1,-4.654526E-2,2.00839E-2,1.0861377E-2,1.991162E-3,-1.0183419E-2,7.2082746E-1,3.1994554E-1,1.2434062E-3,2.8032938E-1,-2.9073268E-1,-1.0979963E-1,-3.35321E-1,-3.2513223E-3,2.335298E-2,4.72645E-2,9.465136E-2,3.9153588E-1,-2.0452768E-1,3.8235947E-2,6.392268E-2,3.1370714E-1,2.6482764E-1,-1.1810554E-1,9.123972E-1,3.599169E-1,-5.1740825E-1,-1.9662157E-1,2.5160512E-1,-6.4678583E-3,-4.714877E-3,1.425105E-1,2.662728E-3,1.5399018E-2,6.976434E-1,3.7170902E-1,-2.7662793E-2,-1.3851189E-2,-3.6894362E-2,-1.83417E-2,-1.9801801E-2,-1.0619635E-3,-1.0737708E-2,3.112108E-3,7.1985847E-3,-4.771783E-3,-1.2324916E-2,-2.3823907E-3,-1.0306834E-2,-2.9002203E-2,5.225078E-3,-1.7614838E-2,-1.6823642E-2,-3.0372806E-2,4.7458243E-3,-3.5001414E-3,1.16110025E-2,3.7941676E-2,1.8020436E-2,-1.4767415E-3,2.0525556E-4,1.6243292E-2,-1.001888E-2,-2.4205467E-2,-2.4753315E-3,-8.9364415E-3,-1.6837103E-2,-4.454629E-3,-6.087485E-3,7.2758067E-3,1.5741266E-2,3.6060594E-2,-2.1874592E-2,-6.798603E-3,-1.2197411E-3,9.1803605E-3,1.953457E-2,-4.689556E-3,1.5821604E-2,4.193816E-3,-4.0387303E-2,-2.3273611E-3,5.1823054E-2,3.3139817E-2,1.0876971E-2,2.2583893E-2,-2.670788E-2,-6.231729E-3,1.4904204E-3,-1.3151253E-2,1.833648E-4,2.481432E-2,-4.9353323E-5,1.0266969E-2,4.856328E-2,3.1363934E-2,6.8832855E-3,3.047126E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,-1,61,-1,63,-1,65,-1,67,69,71,73,75,77,-1,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,121,-1,123,-1,-1,-1,125,127,-1,129,131,133,135,-1,-1,-1,137,139,141,143,-1,145,147,149,151,153,155,157,159,-1,-1,161,-1,-1,163,165,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4715038E2,3.1979042E1,6.069575E1,1.0687485E1,1.5620537E1,1.6630074E1,1.8618118E1,5.9992294E0,2.2821217E0,6.394697E0,1.862215E0,1.1462517E0,1.3992168E1,1.10465145E1,9.569153E0,5.631275E-1,5.411705E0,6.113863E-1,5.981803E-1,5.9675255E0,0E0,4.372775E-1,1.0612082E0,1.6818619E-1,4.3318224E-1,8.325497E0,1.3989862E1,7.227665E0,4.7191496E0,6.234716E-1,4.0835495E0,0E0,0E0,3.3126678E0,0E0,5.235815E-1,0E0,2.9798782E-1,0E0,2.892805E0,8.385172E-1,2.9271695E-1,1.4142455E-1,8.1371355E-1,3.1075597E-1,0E0,0E0,5.3248763E-1,2.2333527E-1,1.3003063E-1,6.4632425E0,5.8093476E0,5.310542E0,4.308467E0,7.932312E0,8.500943E-1,1.0687503E0,3.237638E-1,2.6685762E-1,3.4711304E0,0E0,2.4194527E0,3.3025055E0,4.235878E-1,1.7884749E-1,1.6631295E-1,2.20644E-1,3.3383322E0,4.2929816E0,3.074684E-1,0E0,1.1030639E-1,0E0,0E0,0E0,2.179699E-1,3.0803967E-1,0E0,2.7469528E-1,3.84382E-1,1.9119602E-1,1.3270807E-1,0E0,0E0,0E0,4.502903E0,1.8649292E0,4.33817E0,1.500631E0,0E0,1.7459431E0,2.2365503E0,1.7233174E0,5.604229E-1,1.5832233E0,2.0821142E-1,3.637755E-1,1.7811681E0,0E0,0E0,1.5497303E-1,0E0,0E0,1.7021332E0,2.683004E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,37,37,39,39,40,40,41,41,42,42,43,43,44,44,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,75,75,76,76,78,78,79,79,80,80,81,81,85,85,86,86,87,87,88,88,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,100,100,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,-1,62,-1,64,-1,66,-1,68,70,72,74,76,78,-1,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,122,-1,124,-1,-1,-1,126,128,-1,130,132,134,136,-1,-1,-1,138,140,142,144,-1,146,148,150,152,154,156,158,160,-1,-1,162,-1,-1,164,166,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.330579E0,1.0402391E6,1E0,1.4930757E0,3.5E2,1.5283889E0,3E0,2.0368582E8,1.3144558E4,1.3097832E3,4.1E1,1.8E1,3.9712732E7,6.4446027E2,1.9041E4,4.9386E4,1E0,2.437E3,1E0,4.910802E-2,5.8426323E0,7.6247287E-1,4E0,9.25E0,3.92382E5,1.0409E4,6.071E3,6.513793E1,3.7253947E5,1.8114515E1,-2.1471651E-2,-6.62192E-2,6.748543E-1,2.8891351E-2,1.998E3,-3.478693E-2,2.25E2,4.926316E-3,1.0323588E3,1.3649979E10,3.1648794E-1,3.2794893E-1,1.4075E4,1.4157E4,-4.3892553E-3,-2.1321265E-2,2.5167784E-1,1.8224286E2,2.127E3,2.3314082E6,1.645614E6,7.969174E7,4.22088E0,2E0,1.7495675E5,1.5902449E0,3.101286E0,3.0291306E2,1.4285097E1,6.0508516E-2,2.046E3,1.0118026E0,3.098804E2,3.771981E6,1.0214149E6,2.04946E6,4.8297736E7,8.965631E7,2.9910075E5,-4.654526E-2,1.15062E5,1.0861377E-2,1.991162E-3,-1.0183419E-2,3.8910418E6,2.077501E6,1.2434062E-3,1.9776634E9,3.64299E5,3.9E2,2.4E1,-3.2513223E-3,2.335298E-2,4.72645E-2,1E0,2.5870967E0,6.061028E4,6.932264E8,6.392268E-2,2.342147E7,1E0,9.3E1,6.797394E8,4.7658E4,7.2530017E0,1E0,2.8541E4,-6.4678583E-3,-4.714877E-3,2.1515152E0,2.662728E-3,1.5399018E-2,1.0918E4,9.287868E6,-2.7662793E-2,-1.3851189E-2,-3.6894362E-2,-1.83417E-2,-1.9801801E-2,-1.0619635E-3,-1.0737708E-2,3.112108E-3,7.1985847E-3,-4.771783E-3,-1.2324916E-2,-2.3823907E-3,-1.0306834E-2,-2.9002203E-2,5.225078E-3,-1.7614838E-2,-1.6823642E-2,-3.0372806E-2,4.7458243E-3,-3.5001414E-3,1.16110025E-2,3.7941676E-2,1.8020436E-2,-1.4767415E-3,2.0525556E-4,1.6243292E-2,-1.001888E-2,-2.4205467E-2,-2.4753315E-3,-8.9364415E-3,-1.6837103E-2,-4.454629E-3,-6.087485E-3,7.2758067E-3,1.5741266E-2,3.6060594E-2,-2.1874592E-2,-6.798603E-3,-1.2197411E-3,9.1803605E-3,1.953457E-2,-4.689556E-3,1.5821604E-2,4.193816E-3,-4.0387303E-2,-2.3273611E-3,5.1823054E-2,3.3139817E-2,1.0876971E-2,2.2583893E-2,-2.670788E-2,-6.231729E-3,1.4904204E-3,-1.3151253E-2,1.833648E-4,2.481432E-2,-4.9353323E-5,1.0266969E-2,4.856328E-2,3.1363934E-2,6.8832855E-3,3.047126E-2],"split_indices":[20,56,47,102,42,2,41,3,7,52,4,2,3,45,52,9,2,77,2,104,0,53,57,0,58,5,2,2,56,28,35,0,0,27,0,10,0,10,0,52,5,38,38,9,9,0,0,58,4,1,28,9,7,53,17,33,35,41,52,54,0,2,39,52,1,48,9,45,45,28,0,29,0,0,0,45,29,0,5,9,12,3,0,0,0,102,53,28,7,0,45,14,0,7,10,53,67,2,0,0,53,0,0,9,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,8.6E2,1.823E3,4.71E2,3.89E2,1.096E3,7.27E2,3.23E2,1.48E2,3.06E2,8.3E1,1.59E2,9.37E2,4.46E2,2.81E2,1.3E1,3.1E2,9.6E1,5.2E1,3.03E2,3E0,3.5E1,4.8E1,4.3E1,1.16E2,4.59E2,4.78E2,3.75E2,7.1E1,5.1E1,2.3E2,2E0,1.1E1,3.07E2,3E0,9.1E1,5E0,4.3E1,9E0,2.69E2,3.4E1,2.7E1,8E0,2.5E1,2.3E1,2E0,4.1E1,7.1E1,4.5E1,1.5E1,4.44E2,4.3E2,4.8E1,2.28E2,1.47E2,3.6E1,3.5E1,2.2E1,2.9E1,2.19E2,1.1E1,1.41E2,1.66E2,7.3E1,1.8E1,1E1,3.3E1,1.85E2,8.4E1,2.7E1,7E0,1.3E1,1.4E1,3E0,5E0,1.1E1,1.4E1,7E0,1.6E1,2.7E1,4.4E1,4.1E1,4E0,4E0,1.1E1,3.52E2,9.2E1,2.78E2,1.52E2,5E0,4.3E1,1.94E2,3.4E1,3.4E1,1.13E2,1.7E1,1.9E1,2.6E1,9E0,8E0,1.4E1,5E0,2.4E1,1.74E2,4.5E1,9.4E1,4.7E1,1.36E2,3E1,7E1,3E0,1.6E1,2E0,3E0,7E0,2.6E1,7E0,1.6E2,2.5E1,5.7E1,2.7E1,1.2E1,1.5E1,7E0,6E0,2E0,9E0,1.2E1,2E0,3E0,1.3E1,2.1E1,6E0,2.6E1,1.8E1,3.8E1,3E0,7.2E1,2.8E2,8E1,1.2E1,5.4E1,2.24E2,1.08E2,4.4E1,3.5E1,8E0,1.41E2,5.3E1,2E0,3.2E1,1.7E1,1.7E1,5.3E1,6E1,1.5E1,2E0,5E0,1.4E1,1.4E1,1.2E1,5E0,9E0,1.8E1,1.56E2,2.5E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"167","size_leaf_vector":"1"}},{"base_weights":[2.6937397E-3,-3.194539E-1,1.5231116E-1,-4.913423E-1,-1.18760586E-1,6.9139555E-2,5.200848E-1,-5.9034604E-1,-2.7204713E-1,-2.3735517E-1,2.949747E-1,-5.319983E-2,3.0064428E-1,6.0685134E-1,1.6792245E-1,-6.339281E-1,-1.2254589E-1,-3.395377E-1,-1.0005719E-1,-1.441261E-1,-4.8610166E-1,9.58362E-2,4.1858348E-1,-3.1337887E-1,-1.2541638E-2,3.2553402E-1,-3.6755174E-1,9.192817E-1,5.263071E-1,-3.8451374E-2,4.6411547E-1,-5.5510205E-1,-8.616017E-1,3.1575095E-2,-2.8032812E-1,-4.1089502E-1,-2.0114641E-1,-1.856754E-1,3.6774024E-2,-2.1052109E-1,4.1793716E-1,-6.018875E-1,-3.3314762E-1,2.4706568E-1,-4.7569303E-3,-3.3420238E-3,4.6984115E-1,-3.394702E-1,4.1723487E-1,-9.149867E-2,1.0757618E-1,2.4472596E-1,5.8084095E-1,-4.9893895E-1,2.7701756E-4,1.0257202E0,5.0468916E-1,4.5080528E-1,9.336263E-1,-4.825508E-1,3.6633287E-2,3.1382203E-1,4.167263E-2,-5.891922E-1,-1.2138844E-1,-6.7874014E-1,-1.057143E0,-3.276434E-1,7.6669473E-3,-2.538854E-1,-4.871981E-1,-3.4732756E-1,-1.0749297E-1,-7.7762246E-2,-1.3773647E-2,6.223159E-3,-1.148917E-1,-4.6709093E-1,-1.5548837E-1,5.327621E-2,2.234101E-1,-3.6035302E-1,-6.962979E-1,-1.4601439E-2,-4.623849E-1,1.5146034E-2,4.889199E-3,8.138211E-3,-5.3017806E-2,2.287168E-1,5.505606E-1,-2.673376E-1,-4.8601162E-1,3.4250062E-2,-9.473923E-4,-2.4089362E-2,-2.132216E-1,2.1752708E-1,1.665665E-2,2.2128503E-1,6.161153E-1,6.3161933E-1,2.6082778E-1,-2.7001845E-2,-3.5000786E-3,8.009486E-3,-5.987215E-3,1.0762048E0,2.0923112E-2,2.8408049E-2,7.5318483E-3,3.5010093E-1,6.0399264E-1,9.851632E-1,4.30389E-3,-2.9383853E-2,-7.0407623E-3,2.1347733E-1,-9.984948E-2,5.3211357E-3,4.2172077E-1,-1.7243562E-2,-3.036037E-2,3.0039465E-3,-1.2285865E-2,-6.551776E-3,-3.3698786E-2,-3.5198424E-2,-5.776744E-2,-1.761137E-3,-1.7797781E-2,-1.5294387E-2,-2.0260303E-4,-2.4285527E-2,-7.7273E-3,-1.03474455E-2,-2.9762484E-2,4.0359926E-4,-7.948868E-3,5.157741E-3,-5.63419E-3,2.924477E-3,-9.4578015E-3,-7.666724E-3,-2.6441582E-2,-1.085656E-2,-2.1946235E-3,1.380741E-2,-1.1516257E-2,-1.9913303E-2,3.4777306E-6,-3.5714786E-2,-1.341211E-2,-6.0083307E-3,1.1420933E-2,-2.5039101E-2,-7.837343E-3,1.8195275E-4,-8.011385E-3,-2.954847E-3,1.3517644E-2,2.9255943E-2,1.2597094E-2,-1.8326944E-2,-1.5123065E-3,-2.9279768E-2,-1.6758831E-2,-5.0736316E-3,1.2241925E-2,-2.6952317E-2,-6.4601973E-3,7.910658E-3,2.0348242E-2,-6.9334256E-3,5.0431653E-3,6.0268547E-3,1.4960297E-2,1.5487161E-2,3.501578E-2,3.335104E-2,1.4940443E-2,5.4204254E-3,2.2828024E-2,3.1745672E-2,5.425941E-2,1.1852521E-2,1.9575668E-2,3.5509393E-2,2.3659457E-2,4.8983056E-2,1.4023721E-2,1.6855178E-2,1.4850127E-3,-8.805697E-3,1.5617014E-3,3.1090884E-2,1.4398778E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,121,123,125,127,129,-1,131,133,135,137,139,-1,-1,141,143,145,-1,147,149,151,153,155,-1,-1,-1,157,159,161,163,165,-1,-1,167,169,171,173,175,177,179,181,-1,-1,-1,-1,183,-1,-1,-1,185,187,189,-1,-1,-1,191,193,-1,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2902888E2,2.9229485E1,5.5879494E1,9.761345E0,1.9353254E1,4.227264E1,1.0195435E1,6.3240433E0,1.637536E0,7.026203E0,2.1260276E0,1.0331178E1,8.701355E0,6.31057E0,4.174447E0,4.6724014E0,3.6390243E0,9.3449783E-1,4.9886823E-1,8.447258E0,1.2694054E0,5.3361785E-1,1.3845491E0,2.6579113E0,8.03343E0,1.0132015E1,9.1837E-1,1.8652763E0,6.25259E0,1.3848246E0,1.4356647E0,3.1221085E0,1.8569756E0,0E0,5.5258095E-1,6.7898846E-1,4.6920836E-1,2.6362848E-1,2.5515524E-1,2.76544E0,2.9939399E0,7.980423E-1,1.5576339E0,1.0815942E-1,1.9448532E-1,0E0,8.2778454E-1,1.2230844E0,6.6517043E-1,4.187751E0,3.3552623E0,3.1718311E0,1.7689743E0,3.4563613E-1,1.4643167E-1,7.7833176E-1,3.064022E-1,2.6424713E0,1.3464565E0,1.7354846E-1,8.9021754E-1,4.3433404E-1,0E0,1.8229675E0,4.3910864E-1,4.439888E-1,6.98555E-1,2.6783872E-1,0E0,3.9096236E-1,2.0358467E-1,3.6950743E-1,1.6411495E-1,1.1622991E-1,0E0,0E0,1.1379375E-1,8.2531357E-1,1.2936702E0,0E0,6.714636E-1,2.9984498E-1,4.499483E-1,3.6659348E-1,4.0205765E-1,0E0,0E0,0E0,1.17497966E-1,2.3511845E-1,4.6378708E-1,2.3809853E0,4.8552322E-1,0E0,0E0,7.5825195E0,4.9073286E0,1.5796914E0,2.6627736E0,3.0820065E0,5.044613E-1,1.8949165E0,5.1967204E-1,0E0,0E0,0E0,0E0,1.4081955E-1,0E0,0E0,0E0,5.8497906E-1,7.221203E-1,5.08543E-1,0E0,0E0,0E0,3.9336407E-1,2.39715E-1,0E0,1.9036412E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,88,88,89,89,90,90,91,91,92,92,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,107,107,111,111,112,112,113,113,117,117,118,118,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,122,124,126,128,130,-1,132,134,136,138,140,-1,-1,142,144,146,-1,148,150,152,154,156,-1,-1,-1,158,160,162,164,166,-1,-1,168,170,172,174,176,178,180,182,-1,-1,-1,-1,184,-1,-1,-1,186,188,190,-1,-1,-1,192,194,-1,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.330579E0,9.075E3,1E0,1.4722673E0,7.733016E5,3.5301748E7,1.3999657E3,1.4887473E-1,5.9767612E7,1.289627E3,9.6E1,4.898012E-1,2E0,5.528107E3,3.9712732E7,6.2186885E-1,1.8122449E2,1.752512E2,2.0601392E3,2.8207336E3,1.3510204E1,1E0,1.6140062E3,1.1032986E8,3.171E3,5.98E2,2.3977574E7,6.7652373E3,2.9816154E1,2.4347392E-1,3.386E3,9.1569895E-1,3.1575095E-2,5.212854E8,3.5685265E6,2.8729197E8,4.05E2,7E0,2.368E0,8.417E3,2.8035452E11,9.35E2,4.571429E0,7.576E3,-3.3420238E-3,2.7271206E5,2.0189162E7,1.3E1,4.602015E0,2.1E1,7.064848E6,2E0,1.3715873E3,8E0,1E0,5.847845E3,1.6382656E-1,1.6259277E7,3.906E3,1.0503613E5,4.4E1,4.167263E-2,1.3062E4,2.1515152E0,7E0,8.91E2,7.9527026E-1,7.6669473E-3,9.705292E6,2.4906608E5,8.182648E6,1.6621418E0,3.9794872E0,-1.3773647E-2,6.223159E-3,1.9358906E6,6.699017E6,5.4878284E2,5.327621E-2,1.2342778E1,4.1E1,2.208825E-2,2.7271206E5,8.755996E-1,1.5146034E-2,4.889199E-3,8.138211E-3,5.884028E0,2.028994E8,3.07125E0,5.222222E1,7.7344055E4,3.4250062E-2,-9.473923E-4,3.5734247E2,8.8365E4,7.10309E2,1.6006084E3,1.06E3,1E0,2E0,9.015703E-7,-2.7001845E-2,-3.5000786E-3,8.009486E-3,-5.987215E-3,8.7757526E2,2.0923112E-2,2.8408049E-2,7.5318483E-3,5.3473564E1,1.0715278E7,2.4435181E12,4.30389E-3,-2.9383853E-2,-7.0407623E-3,1.7911884E3,5.094239E3,5.3211357E-3,1.3235373E1,-1.7243562E-2,-3.036037E-2,3.0039465E-3,-1.2285865E-2,-6.551776E-3,-3.3698786E-2,-3.5198424E-2,-5.776744E-2,-1.761137E-3,-1.7797781E-2,-1.5294387E-2,-2.0260303E-4,-2.4285527E-2,-7.7273E-3,-1.03474455E-2,-2.9762484E-2,4.0359926E-4,-7.948868E-3,5.157741E-3,-5.63419E-3,2.924477E-3,-9.4578015E-3,-7.666724E-3,-2.6441582E-2,-1.085656E-2,-2.1946235E-3,1.380741E-2,-1.1516257E-2,-1.9913303E-2,3.4777306E-6,-3.5714786E-2,-1.341211E-2,-6.0083307E-3,1.1420933E-2,-2.5039101E-2,-7.837343E-3,1.8195275E-4,-8.011385E-3,-2.954847E-3,1.3517644E-2,2.9255943E-2,1.2597094E-2,-1.8326944E-2,-1.5123065E-3,-2.9279768E-2,-1.6758831E-2,-5.0736316E-3,1.2241925E-2,-2.6952317E-2,-6.4601973E-3,7.910658E-3,2.0348242E-2,-6.9334256E-3,5.0431653E-3,6.0268547E-3,1.4960297E-2,1.5487161E-2,3.501578E-2,3.335104E-2,1.4940443E-2,5.4204254E-3,2.2828024E-2,3.1745672E-2,5.425941E-2,1.1852521E-2,1.9575668E-2,3.5509393E-2,2.3659457E-2,4.8983056E-2,1.4023721E-2,1.6855178E-2,1.4850127E-3,-8.805697E-3,1.5617014E-3,3.1090884E-2,1.4398778E-2],"split_indices":[20,56,2,102,42,28,45,52,38,45,4,29,27,17,52,45,27,52,52,52,55,58,16,52,7,2,10,47,52,56,38,2,27,0,32,45,5,10,54,58,12,31,0,54,9,0,28,45,0,53,3,28,6,52,8,6,4,35,9,0,33,3,0,9,53,3,0,27,0,9,47,45,58,54,0,0,45,45,52,0,54,3,38,28,42,0,0,0,53,7,57,4,33,0,0,52,1,52,55,2,102,17,37,0,0,0,0,52,0,0,0,56,45,31,0,0,0,52,4,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.675E3,8.48E2,1.827E3,4.56E2,3.92E2,1.491E3,3.36E2,3.13E2,1.43E2,3.05E2,8.7E1,9.76E2,5.15E2,2.69E2,6.7E1,2.86E2,2.7E1,1.02E2,4.1E1,2.23E2,8.2E1,3.4E1,5.3E1,1.31E2,8.45E2,4.97E2,1.8E1,5.3E1,2.16E2,4E1,2.7E1,2.15E2,7.1E1,4E0,2.3E1,6.6E1,3.6E1,2.5E1,1.6E1,2E2,2.3E1,4.5E1,3.7E1,1.3E1,2.1E1,5E0,4.8E1,1.27E2,4E0,5.1E2,3.35E2,3.79E2,1.18E2,1.3E1,5E0,4.1E1,1.2E1,1.84E2,3.2E1,5E0,3.5E1,2.1E1,6E0,1.99E2,1.6E1,3.9E1,3.2E1,2.1E1,2E0,2.3E1,4.3E1,1.3E1,2.3E1,1.3E1,1.2E1,1E1,6E0,3.4E1,1.66E2,4E0,1.9E1,1.4E1,3.1E1,1.1E1,2.6E1,8E0,5E0,4E0,1.7E1,1.3E1,3.5E1,8.7E1,4E1,2E0,2E0,3.29E2,1.81E2,1.51E2,1.84E2,3.58E2,2.1E1,1.01E2,1.7E1,1.1E1,2E0,2E0,3E0,3.7E1,4E0,9E0,3E0,1.13E2,7.1E1,3E1,2E0,3E0,2E0,1.5E1,2E1,8E0,1.3E1,3.4E1,1.65E2,7E0,9E0,2E0,3.7E1,1.2E1,2E1,3E0,1.8E1,1.8E1,5E0,4E1,3E0,1E1,3E0,8E0,1.5E1,2E0,1.1E1,2E0,4E0,8E0,2.6E1,1E2,6.6E1,1.7E1,2E0,1.2E1,2E0,2.7E1,4E0,8E0,3E0,2.1E1,5E0,1.2E1,5E0,2E0,1.1E1,2.8E1,7E0,5.8E1,2.9E1,1.9E1,2.1E1,2.55E2,7.4E1,3.2E1,1.49E2,1.22E2,2.9E1,6.5E1,1.19E2,1.76E2,1.82E2,7E0,1.4E1,8.3E1,1.8E1,1.1E1,6E0,6E0,3.1E1,4.3E1,7E1,2.9E1,4.2E1,2.8E1,2E0,8E0,7E0,1.2E1,8E0,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-4.247643E-3,-3.058124E-1,1.3960832E-1,-4.83575E-1,-1.4032428E-1,-5.5578197E-3,3.4042555E-1,-6.204453E-1,-3.3837742E-1,-2.2793765E-1,2.0594054E-1,-1.677511E-1,1.02495E-1,1.7850184E-1,4.6826115E-1,-5.861073E-1,-6.06181E-2,-2.3077953E-1,-4.7181672E-1,-2.5280342E-1,6.175792E-2,1.5057157E-1,5.6251943E-1,-1.9253857E-1,3.4382218E-1,-2.9788864E-1,1.3338059E-1,4.558468E-2,3.4065822E-1,2.992651E-1,6.070901E-1,-6.245098E-1,-1.2242591E-1,-2.6355356E-1,-9.469802E-3,-5.521135E-1,-3.275672E-1,-4.4889247E-1,-1.5776566E-1,2.2195434E-2,2.6350406E-1,3.672236E-2,1.4705444E-2,-1.6261622E-1,-5.0506926E-1,5.021334E-2,1.02120735E-1,-3.4369522E-1,2.2663586E-2,6.6342577E-3,2.4003159E-1,1.13434315E-1,-1.9297439E-1,2.513278E-1,5.1439106E-1,3.9056975E-1,8.21803E-2,5.7588106E-1,5.9795406E-2,-5.613824E-1,-8.3359253E-1,2.3499297E-1,-2.909332E-1,6.4825253E-3,-2.7714646E-1,8.840951E-3,-2.4127911E-3,-7.927887E-1,-4.553172E-1,-1.7926198E-1,-4.1790104E-1,-2.7021593E-1,-6.0556453E-1,-1.848087E-1,3.3282402E-1,1.8410621E-2,-3.0186702E-2,1.0764347E-1,3.6127278E-1,-3.093565E-1,-9.739997E-2,-3.3089048E-1,-9.074272E-1,1.1576631E-2,-2.7865483E-4,-2.3088285E-1,-2.3649285E-2,-4.8656188E-2,1.8346162E-1,1.8445863E-1,4.304313E-1,2.582306E-1,2.6582377E-2,-1.8688168E-2,-2.877568E-2,3.676333E-1,7.365476E-2,5.594331E-1,-7.409193E-3,4.4755977E-1,8.863831E-2,-6.2615713E-3,1.5133004E-1,7.2443664E-1,3.992038E-1,-2.9152242E-2,-1.3196349E-2,-4.2941295E-2,-1.4643722E-2,1.6191078E-2,9.197765E-4,-1.6214507E-2,-1.6834256E-3,-1.5316843E-2,-7.880858E-3,-4.2317837E-2,-1.5423025E-2,-2.4033567E-2,-1.0717409E-2,-3.558471E-4,-1.4031484E-2,-7.3647606E-3,-2.255646E-2,-1.8288078E-2,-4.4734506E-3,-2.1846795E-2,-3.842872E-2,-1.2852492E-2,-3.7025856E-3,-9.845759E-6,2.5908455E-2,6.641948E-3,-4.946377E-3,7.5539984E-3,-2.5928232E-3,2.1970824E-3,1.9176168E-2,-1.900498E-2,-1.1592528E-2,2.6534004E-2,-6.421102E-3,-2.4568362E-2,-7.812615E-3,-5.0317384E-2,-1.0677549E-2,-1.8034786E-2,-2.1553861E-3,-5.7003503E-3,4.5416444E-3,1.0764772E-2,-4.721906E-3,7.6967296E-3,3.546809E-2,1.0244192E-2,2.9621884E-2,2.1285497E-2,9.382852E-3,-5.95372E-3,5.1617357E-3,-5.894312E-3,8.391165E-3,1.1162952E-2,2.410681E-2,6.320504E-3,-1.8405957E-2,2.9070454E-2,1.3788845E-3,1.0779137E-2,2.4901416E-2,-9.139686E-3,7.543788E-3,4.830767E-3,1.5221984E-2,4.1526545E-2,2.2230864E-2,2.174772E-2,4.226578E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,-1,79,81,-1,83,85,-1,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,-1,113,-1,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,143,-1,-1,145,-1,147,149,151,153,155,157,-1,159,161,163,165,-1,167,169,-1,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.17130394E2,2.558429E1,5.3291496E1,8.13681E0,1.3789798E1,1.8629211E1,1.5793022E1,4.0223007E0,2.8488197E0,1.3985649E1,1.731694E0,5.4554577E0,7.9292946E0,7.3126154E0,9.859856E0,3.59169E0,0E0,8.3887625E-1,8.731766E-1,6.582184E0,0E0,1.1684762E0,3.907535E-1,3.7146397E0,3.29937E0,1.7874904E0,8.0143585E0,3.0648854E0,2.245222E0,3.8329697E0,3.9657516E0,2.0398178E0,1.092238E0,5.851793E-1,1.3634089E-1,9.669266E-1,3.939805E-1,3.0836277E0,3.272132E0,7.516176E-1,6.1635804E-1,0E0,0E0,3.5298529E0,2.1526184E0,0E0,2.4251765E-1,6.3116455E-1,0E0,2.6682334E0,3.3277092E0,1.8417839E0,1.3490914E0,2.1013503E0,1.6235743E0,2.3177013E0,8.884964E-1,5.6162415E0,0E0,1.848587E0,1.1238155E0,1.2698254E-1,1.3432741E-1,0E0,4.1832018E-1,0E0,0E0,3.5691643E-1,3.5089684E-1,2.909558E-1,2.220819E-1,1.0698562E0,1.4278355E0,2.0452685E0,9.0135574E-1,0E0,4.4638655E-1,1.5441114E-1,3.0262113E-1,5.750809E-1,6.2629795E0,7.1065474E-1,7.4623203E-1,0E0,0E0,7.1427345E-1,0E0,2.111622E0,7.7966475E-1,3.1954193E0,2.80013E0,5.5111885E-1,1.1529428E0,0E0,4.8325378E-1,1.0149403E0,1.1726044E0,1.1945648E0,0E0,1.7354832E0,4.7285166E-1,0E0,3.460467E-1,4.090164E0,1.7526894E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,87,87,88,88,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,99,99,100,100,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,-1,80,82,-1,84,86,-1,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,-1,114,-1,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,144,-1,-1,146,-1,148,150,152,154,156,158,-1,160,162,164,166,-1,168,170,-1,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,1.5938586E8,2.308943E0,1.4930757E0,1.8108038E5,2.1446484E-1,1.1855755E8,7.5891815E6,1.22159E5,7.75541E-1,1.5106794E3,1.2E1,9.099893E8,1.0855529E3,5.35E3,-6.06181E-2,2.728889E2,7.2764084E-2,1E0,6.175792E-2,2.0783027E5,2.093563E0,1E0,6.7388856E7,1.0409E4,8.15E2,4.0844156E7,3.166E3,5.236559E0,1.9013363E4,2.589369E7,1.5062694E9,2.08E2,1.04E2,1.2E1,1.9521575E5,1.7232166E8,1.3554651E2,6.862E3,3.0990322E5,3.672236E-2,1.4705444E-2,2.73E2,7.492813E0,5.021334E-2,6.1428223E1,5.097268E7,2.2663586E-2,2.3314082E6,2.2214102E6,6.5369595E3,1.46705E3,1.9761398E7,5.2686436E7,2E0,2.7165637E4,1.778482E7,5.9795406E-2,5.551643E2,9.173084E2,2.1358025E0,1.8489688E7,6.4825253E-3,9.490825E5,8.840951E-3,-2.4127911E-3,1.108E3,1E0,1.0830879E6,2.808311E0,1.221875E1,8.387235E11,3.7609413E-1,9.536863E4,1.8410621E-2,7.11E2,9.47817E7,3.333E3,4.1E1,8E0,1.5043668E0,1.752E3,1.1576631E-2,-2.7865483E-4,2.291E3,-2.3649285E-2,2.3446269E2,7.89E2,1.4685706E3,5.3038636E7,6E0,1.6342836E3,-1.8688168E-2,4.798624E3,3.0795444E3,3.0769388E7,3.00624E5,-7.409193E-3,2.0131812E7,1.2974394E6,-6.2615713E-3,8.985661E8,3.7E1,3.5301748E7,-2.9152242E-2,-1.3196349E-2,-4.2941295E-2,-1.4643722E-2,1.6191078E-2,9.197765E-4,-1.6214507E-2,-1.6834256E-3,-1.5316843E-2,-7.880858E-3,-4.2317837E-2,-1.5423025E-2,-2.4033567E-2,-1.0717409E-2,-3.558471E-4,-1.4031484E-2,-7.3647606E-3,-2.255646E-2,-1.8288078E-2,-4.4734506E-3,-2.1846795E-2,-3.842872E-2,-1.2852492E-2,-3.7025856E-3,-9.845759E-6,2.5908455E-2,6.641948E-3,-4.946377E-3,7.5539984E-3,-2.5928232E-3,2.1970824E-3,1.9176168E-2,-1.900498E-2,-1.1592528E-2,2.6534004E-2,-6.421102E-3,-2.4568362E-2,-7.812615E-3,-5.0317384E-2,-1.0677549E-2,-1.8034786E-2,-2.1553861E-3,-5.7003503E-3,4.5416444E-3,1.0764772E-2,-4.721906E-3,7.6967296E-3,3.546809E-2,1.0244192E-2,2.9621884E-2,2.1285497E-2,9.382852E-3,-5.95372E-3,5.1617357E-3,-5.894312E-3,8.391165E-3,1.1162952E-2,2.410681E-2,6.320504E-3,-1.8405957E-2,2.9070454E-2,1.3788845E-3,1.0779137E-2,2.4901416E-2,-9.139686E-3,7.543788E-3,4.830767E-3,1.5221984E-2,4.1526545E-2,2.2230864E-2,2.174772E-2,4.226578E-3],"split_indices":[20,56,7,54,42,28,38,45,45,2,34,52,18,7,52,2,0,52,39,16,0,33,39,6,7,2,2,45,0,54,52,45,12,2,10,18,33,7,56,9,28,0,0,2,54,0,56,45,0,28,28,48,52,45,43,6,33,51,0,4,52,56,32,0,33,0,0,0,105,32,58,58,31,42,28,0,0,32,2,2,18,53,29,0,0,2,0,52,2,55,7,3,55,0,4,4,1,10,0,12,28,0,5,8,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.698E3,8.71E2,1.827E3,4.19E2,4.52E2,1.061E3,7.66E2,2.14E2,2.05E2,3.61E2,9.1E1,4.24E2,6.37E2,3.39E2,4.27E2,2.05E2,9E0,1.15E2,9E1,3.56E2,5E0,8E1,1.1E1,4.05E2,1.9E1,4.5E1,5.92E2,1.87E2,1.52E2,1.94E2,2.33E2,1.89E2,1.6E1,1E2,1.5E1,5.6E1,3.4E1,1.15E2,2.41E2,3.8E1,4.2E1,5E0,6E0,3.71E2,3.4E1,4E0,1.5E1,4.3E1,2E0,2.71E2,3.21E2,1.46E2,4.1E1,1.02E2,5E1,1.36E2,5.8E1,2.24E2,9E0,1.48E2,4.1E1,5E0,1.1E1,3E0,9.7E1,2E0,1.3E1,1.4E1,4.2E1,1.4E1,2E1,5.5E1,6E1,2.29E2,1.2E1,4E0,3.4E1,1.7E1,2.5E1,1.13E2,2.58E2,2.5E1,9E0,6E0,9E0,2.6E1,1.7E1,2.07E2,6.4E1,2.5E2,7.1E1,5.4E1,9.2E1,1.8E1,2.3E1,6.1E1,4.1E1,4.7E1,3E0,1.14E2,2.2E1,1.4E1,4.4E1,1.2E2,1.04E2,1.26E2,2.2E1,3.6E1,5E0,3E0,2E0,9E0,2E0,6.9E1,2.8E1,1.1E1,3E0,3.4E1,8E0,6E0,8E0,4E0,1.6E1,3.3E1,2.2E1,3.6E1,2.4E1,1.28E2,1.01E2,5E0,7E0,1E1,2.4E1,1.3E1,4E0,3E0,2.2E1,4.7E1,6.6E1,1.3E1,2.45E2,1.1E1,1.4E1,7E0,2E0,1.4E1,1.2E1,1.39E2,6.8E1,5.6E1,8E0,2.41E2,9E0,3.4E1,3.7E1,1.2E1,4.2E1,3.2E1,6E1,1.6E1,7E0,3.2E1,2.9E1,3.7E1,4E0,4.3E1,4E0,2.9E1,8.5E1,4E0,1.8E1,3.5E1,9E0,7.6E1,4.4E1,8.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-9.633271E-4,-2.8656363E-1,1.3522477E-1,-4.427301E-1,-9.8753646E-2,-9.263564E-4,3.3237875E-1,-4.8258913E-1,4.3627083E-1,-2.2861637E-1,1.549249E-1,-5.902825E-2,2.369929E-1,2.4496846E-1,5.4095954E-1,-1.20156206E-1,-5.4981375E-1,6.432092E-2,6.89497E-2,-2.5363678E-1,1.03413925E-1,7.143452E-2,4.684444E-1,-6.6681206E-2,6.077724E-2,1.6654457E-1,4.860674E-1,1.5139186E-1,4.7727683E-1,7.4349344E-1,4.461697E-1,-2.763765E-1,1.911675E-2,-6.0303396E-1,-2.7678502E-1,2.0186953E-2,-6.208101E-3,-2.0026304E-1,-3.9671773E-1,-5.725977E-2,1.853529E-2,-1.2567244E-1,1.330607E-1,8.251597E-3,5.106224E-1,-3.1394044E-1,-2.3808453E-2,1.17333665E-1,5.411061E-1,3.1367743E-1,5.8568674E-1,2.1691048E-1,-6.799056E-2,2.68717E-1,6.252142E-1,7.851519E-1,1.8216412E-1,4.061159E-1,9.70317E-1,8.623198E-3,-3.105307E-1,1.2650074E-1,-2.0538205E-1,-5.4167235E-1,-9.065054E-1,-1.9976299E-1,-5.150728E-1,-2.4798037E-1,-2.2611583E-2,-4.4193938E-1,-2.0041464E-1,-9.394587E-3,6.584681E-3,-1.3730908E-2,1.1539308E-2,9.198243E-2,2.5580418E-1,5.3598255E-1,7.396243E-3,-3.4010336E-1,1.8794918E-2,-2.0184338E-1,1.48444995E-2,-1.544517E-1,1.5615357E-1,1.487895E-2,3.2586623E-2,3.40232E-1,2.8084107E-3,1.5138577E-2,3.0881722E-2,1.0328209E-1,3.3381253E-1,-2.923784E-1,2.2131456E-2,3.238783E-1,-1.5789692E-1,5.5347953E-2,5.7164955E-1,2.3442192E-2,8.753861E-1,1.6115185E-2,-4.0390394E-3,2.354291E-1,4.9632144E-1,3.0084045E-3,5.69581E-2,-1.7265059E-2,-5.532618E-3,1.7177178E-2,2.7503646E-3,-1.6294865E-2,4.154639E-4,-2.8388292E-2,-1.4751882E-2,-6.758118E-2,-3.8151737E-2,-2.5196272E-4,-1.2187545E-2,-7.535172E-3,-3.247957E-2,-1.07381465E-2,-2.3212058E-2,4.5194463E-3,-3.4452036E-3,-1.752935E-2,-3.468031E-2,-1.2019997E-2,-5.378835E-4,6.4026155E-3,-2.156702E-3,-2.4276385E-4,8.6377645E-3,9.229632E-3,2.2027062E-2,1.5012921E-2,2.964023E-2,-1.5120021E-2,-3.1183576E-2,-8.489763E-3,-3.276444E-2,4.361947E-2,-3.8118946E-4,1.6319944E-3,-2.0015331E-2,3.3912018E-3,1.1270735E-2,1.8272504E-2,5.295288E-3,1.311894E-2,5.139978E-4,2.2507358E-2,7.915439E-3,-3.4143146E-2,-1.0766284E-2,2.1613581E-2,-1.0846308E-3,8.454472E-3,2.6352236E-2,7.130458E-3,-2.4036609E-2,3.2023266E-3,2.8590823E-2,5.1839583E-2,3.3979844E-2,1.4570054E-2,-2.032371E-3,2.4907017E-2,-7.4918596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,-1,-1,67,69,71,-1,73,75,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,-1,107,109,111,113,115,117,119,121,123,125,127,-1,-1,-1,129,131,133,135,-1,137,-1,139,141,143,145,-1,-1,147,-1,-1,-1,149,151,153,155,157,159,-1,161,-1,163,-1,-1,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0420053E2,2.5326408E1,4.869303E1,1.6747597E1,1.3015112E1,1.4860492E1,1.3377472E1,1.0946648E1,6.902304E0,2.1944962E0,3.4741032E0,8.866329E0,3.621439E0,1.1318783E1,3.8534698E0,1.5718062E0,5.3692627E0,0E0,1.1748701E0,1.7762861E0,8.879984E-1,1.3164456E0,2.617588E-1,9.109114E0,0E0,2.9963074E0,6.0625935E-1,5.4048605E0,4.460064E0,1.447094E0,2.7760487E0,5.859487E-1,9.661764E-1,5.430603E0,1.0717263E0,0E0,0E0,1.511786E0,4.8844433E-1,3.7866026E-1,0E0,5.7782674E-1,3.8992167E-1,0E0,1.3400984E-1,2.4806118E0,5.0540733E0,1.5856192E0,3.9859486E-1,1.0872269E-1,2.4265957E-1,3.8044243E0,1.7658097E0,1.5572243E0,1.8115501E0,1.2522392E0,2.751667E-1,2.0695095E0,1.9129114E0,0E0,2.7023506E-1,4.123853E-1,3.7826127E-1,3.0052414E0,1.7587776E0,5.3086233E-1,7.686484E-1,6.842575E-1,2.318358E-1,8.4894085E-1,1.3113046E-1,0E0,0E0,0E0,1.0968002E-1,5.413058E-1,1.6890585E-1,2.1193743E-1,0E0,7.272587E-1,0E0,1.4310508E0,1.2348071E1,9.580694E-1,8.6190724E-1,0E0,0E0,1.1911678E-1,0E0,0E0,0E0,2.3345575E0,3.1808739E0,5.501399E-1,1.2209888E0,1.8041077E0,9.190344E-1,0E0,9.9673843E-1,0E0,8.46138E-1,0E0,0E0,9.879968E-1,1.5082893E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,74,74,75,75,76,76,77,77,79,79,81,81,82,82,83,83,84,84,87,87,91,91,92,92,93,93,94,94,95,95,96,96,98,98,100,100,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,-1,-1,68,70,72,-1,74,76,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,-1,108,110,112,114,116,118,120,122,124,126,128,-1,-1,-1,130,132,134,136,-1,138,-1,140,142,144,146,-1,-1,148,-1,-1,-1,150,152,154,156,158,160,-1,162,-1,164,-1,-1,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,7.7344055E4,6.7652373E3,2.3544883E8,1.2526851E6,1.3137E4,8.2608955E-3,2.46E2,9.026015E2,1.4101E4,8.791777E3,3.084419E6,1.8137958E6,4.68646E5,2.0605305E5,2.3023027E3,6.432092E-2,1.0040462E8,1.4274633E7,1.125584E-3,2.1592189E5,2.1924436E5,1.0376E4,6.077724E-2,7.780377E6,3.9892856E2,1.4274633E7,3.564E3,4.5481584E7,1.9013363E4,2.08E2,1.3983857E10,7.8351054E11,1.2427474E8,2.0186953E-2,-6.208101E-3,2.835051E2,7.07089E5,1E0,1.853529E-2,1.4887473E-1,8.715555E5,8.251597E-3,3.430961E7,3.78E2,4.54E2,1.1317E4,1E0,3.684685E2,3E0,6.308896E2,3.731E3,3.1984328E5,1.0918E4,3.0720797E-1,3.2E2,1.59769E5,5.673198E5,8.623198E-3,2.262586E6,6.4860556E5,2.5269838E6,3.83E3,3E0,9.07E2,3.046556E5,1E0,1.0001285E-1,1.3046257E11,2.184578E1,-9.394587E-3,6.584681E-3,-1.3730908E-2,9.619E3,5.2768206E5,1.2783833E3,2.4347392E-1,7.396243E-3,1.1855755E8,1.8794918E-2,4.5184364E0,8E0,1.4615384E0,2.3314082E6,1.487895E-2,3.2586623E-2,2.9138382E7,2.8084107E-3,1.5138577E-2,3.0881722E-2,3.1753032E0,2.4E1,4.0208153E-10,2.22747E5,5.314225E2,2.779E3,5.5347953E-2,1.9577E4,2.3442192E-2,5.4379158E1,1.6115185E-2,-4.0390394E-3,2.6025698E7,7.218466E7,3.0084045E-3,5.69581E-2,-1.7265059E-2,-5.532618E-3,1.7177178E-2,2.7503646E-3,-1.6294865E-2,4.154639E-4,-2.8388292E-2,-1.4751882E-2,-6.758118E-2,-3.8151737E-2,-2.5196272E-4,-1.2187545E-2,-7.535172E-3,-3.247957E-2,-1.07381465E-2,-2.3212058E-2,4.5194463E-3,-3.4452036E-3,-1.752935E-2,-3.468031E-2,-1.2019997E-2,-5.378835E-4,6.4026155E-3,-2.156702E-3,-2.4276385E-4,8.6377645E-3,9.229632E-3,2.2027062E-2,1.5012921E-2,2.964023E-2,-1.5120021E-2,-3.1183576E-2,-8.489763E-3,-3.276444E-2,4.361947E-2,-3.8118946E-4,1.6319944E-3,-2.0015331E-2,3.3912018E-3,1.1270735E-2,1.8272504E-2,5.295288E-3,1.311894E-2,5.139978E-4,2.2507358E-2,7.915439E-3,-3.4143146E-2,-1.0766284E-2,2.1613581E-2,-1.0846308E-3,8.454472E-3,2.6352236E-2,7.130458E-3,-2.4036609E-2,3.2023266E-3,2.8590823E-2,5.1839583E-2,3.3979844E-2,1.4570054E-2,-2.032371E-3,2.4907017E-2,-7.4918596E-3],"split_indices":[20,102,33,52,12,28,2,27,10,52,2,52,9,28,29,33,55,0,51,45,38,28,28,9,0,28,52,45,2,47,52,2,5,31,45,0,0,52,29,74,0,38,33,0,47,8,2,9,102,52,8,52,29,33,9,38,0,11,32,0,1,33,32,2,3,0,32,77,39,31,56,0,0,0,10,28,4,38,0,45,0,53,18,53,28,0,0,9,0,0,0,54,3,37,1,4,2,0,9,0,58,0,0,43,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.677E3,8.64E2,1.813E3,4.71E2,3.93E2,1.073E3,7.4E2,4.51E2,2E1,2.6E2,1.33E2,8.63E2,2.1E2,5.23E2,2.17E2,7.1E1,3.8E2,5E0,1.5E1,2.42E2,1.8E1,1.06E2,2.7E1,8.59E2,4E0,1.65E2,4.5E1,3.74E2,1.49E2,6.7E1,1.5E2,3.3E1,3.8E1,3.17E2,6.3E1,5E0,1E1,1.78E2,6.4E1,1.2E1,6E0,2.5E1,8.1E1,4E0,2.3E1,1.26E2,7.33E2,1.47E2,1.8E1,1.8E1,2.7E1,2.88E2,8.6E1,6.3E1,8.6E1,6.2E1,5E0,1.41E2,9E0,2E0,3.1E1,2.6E1,1.2E1,2.66E2,5.1E1,4.9E1,1.4E1,1.4E2,3.8E1,5.1E1,1.3E1,7E0,5E0,1.1E1,1.4E1,6.2E1,1.9E1,2.1E1,2E0,1.22E2,4E0,1.3E2,6.03E2,1.8E1,1.29E2,8E0,1E1,1.6E1,2E0,6E0,2.1E1,1.47E2,1.41E2,2.4E1,6.2E1,5.6E1,7E0,6E0,8E1,1.6E1,4.6E1,3E0,2E0,5E1,9.1E1,2E0,7E0,2.4E1,7E0,5E0,2.1E1,7E0,5E0,2.17E2,4.9E1,7E0,4.4E1,1.1E1,3.8E1,5E0,9E0,1.29E2,1.1E1,1.1E1,2.7E1,4.2E1,9E0,1E1,3E0,4E0,1E1,3E1,3.2E1,1.6E1,3E0,7E0,1.4E1,1.15E2,7E0,1.25E2,5E0,1.4E1,5.89E2,1.1E1,7E0,6.3E1,6.6E1,1.3E1,3E0,5.1E1,9.6E1,7.7E1,6.4E1,2E0,2.2E1,5E0,5.7E1,3.5E1,2.1E1,4E0,3E0,4E0,7.6E1,1.8E1,2.8E1,4E1,1E1,8.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[5.3309975E-3,-2.6363635E-1,1.317033E-1,-4.380478E-1,-1.08409576E-1,4.523528E-3,3.433563E-1,-5.462082E-1,-2.3884617E-1,-1.2755385E-1,6.504693E-2,-1.253692E-1,1.17934704E-1,1.9513133E-1,4.6135095E-1,-4.4440767E-1,-7.6162606E-1,-1.3689595E-1,-3.938413E-1,-2.0843865E-1,2.2098157E-1,-2.4990152E-1,-4.0793244E-2,1.4988588E-1,-3.1057602E-1,-1.7015627E-1,2.7698156E-1,3.827645E-1,7.3746014E-1,-4.6342984E-1,9.463305E-2,-9.117233E-1,-5.065781E-1,-1.7044578E-1,2.6828043E-2,-2.7030838E-1,-4.6945745E-1,-3.8242882E-1,-1.2715292E-1,1.4256208E-1,5.8266306E-1,-2.9542986E-1,-8.3449095E-2,8.262254E-1,-7.132833E-2,4.6901345E-2,2.3588608E-1,-3.9428294E-1,1.7624289E-1,-2.2792435E-1,2.7503353E-1,-2.5256416E-1,3.021997E-1,2.3394586E-1,5.107189E-1,8.157198E-1,3.538744E-1,-5.328269E-1,-3.142525E-1,8.679864E-3,-3.8917093E-3,-1.0345267E0,-4.7318056E-1,-5.9796566E-1,-2.6944238E-1,-9.145912E-2,-2.284485E-1,-3.1287079E-3,1.1620289E-2,-9.142162E-2,-1.7793521E-2,-5.1376194E-1,-8.480281E-3,-2.2164418E-1,-5.264009E-1,-2.7201477E-1,-7.025486E-2,1.12606116E-1,2.4972554E-2,3.5317462E-2,1.4386224E-2,-2.559147E-1,-5.2515066E-1,-1.4868414E-1,3.3536207E-2,1.4605067E-2,4.349299E-2,-1.0024972E-1,2.4955039E-1,-2.0992028E-2,1.7910972E-1,3.8047147E-1,1.4718372E-1,-2.3860666E-1,-2.534403E-2,-5.0408836E-3,1.4841042E-2,-1.5203847E-1,-1.7755546E-2,4.222626E-3,1.8827507E-2,-5.9512886E-3,-2.8544102E-2,2.0289478E-1,3.9739963E-1,3.240895E-1,8.271278E-2,6.181312E-1,3.2923582E-1,1.3939724E-2,8.4352756E-1,4.219235E-1,-3.9100843E-3,-1.8477492E-2,-2.9917318E-2,-2.714549E-3,-2.1977397E-2,-4.172783E-2,-6.365318E-2,-3.1380527E-2,-1.5298745E-2,-3.191983E-2,-1.0434379E-2,-1.7331542E-2,-8.222017E-4,-6.486354E-3,6.179507E-3,-1.6990216E-2,-7.642719E-3,-1.03235645E-2,1.3059198E-3,-1.5117565E-2,-2.9561939E-2,-5.7718926E-3,-1.8641692E-2,-2.954147E-2,-6.9426806E-3,-7.270126E-3,-1.5811894E-2,-2.7213676E-4,-1.5559657E-2,4.4568034E-4,1.0326746E-2,6.082619E-3,-1.3079393E-2,-3.36598E-2,-1.53597165E-2,-9.2995865E-3,1.6788843E-3,5.4491423E-3,-6.988274E-3,-4.3347143E-3,-3.6591392E-2,1.606166E-2,-1.4672387E-2,-4.3428224E-3,7.358545E-3,9.380508E-3,-2.1672035E-3,1.276732E-2,2.5908256E-2,5.6628776E-3,2.6014706E-2,-1.34601435E-2,3.9031385E-3,-1.5421444E-2,-3.8702958E-3,2.9167082E-2,8.437186E-3,9.918803E-3,2.2161607E-2,2.3269286E-2,1.1517048E-2,6.0425694E-3,-5.801302E-3,2.3799367E-2,4.235028E-2,8.48224E-3,2.4310464E-2,2.5109528E-2,4.3384533E-2,4.7504413E-3,2.3989476E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,-1,131,-1,133,135,137,139,141,-1,-1,-1,143,145,147,149,-1,-1,151,153,155,157,159,161,163,-1,-1,-1,165,-1,-1,-1,-1,-1,167,169,171,173,175,177,-1,179,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.197774E1,2.3370945E1,4.955445E1,8.638649E0,1.3061037E1,1.6970497E1,1.1995392E1,5.4290314E0,2.2373447E0,1.2844875E1,0E0,5.645726E0,8.466776E0,9.255985E0,8.049927E0,1.8997765E0,2.7155418E0,4.9344182E-1,4.0554333E-1,5.170109E0,2.3561425E0,1.6198368E0,8.557244E0,5.06382E0,1.8217444E0,1.5285993E0,3.4415722E0,5.6121635E0,2.232956E0,1.6442451E0,1.2136955E-1,2.2570648E0,5.5403423E-1,3.1813312E-1,3.340782E-1,4.006728E-1,3.4945345E-1,2.5916996E0,2.066206E0,7.5847924E-1,4.3388367E-1,1.4087753E0,3.7130028E-1,1.9292259E-1,2.911178E0,2.3554907E0,3.9454212E0,6.508956E-1,2.95982E-1,4.9640274E-1,1.1981785E-1,4.5471108E-1,2.1977386E0,1.8954039E0,2.934948E0,7.6112366E-1,5.0863695E-1,1.3526306E0,2.1350212E0,0E0,0E0,7.171631E-1,1.5454745E-1,4.540491E-1,2.5035542E-1,3.375024E-1,3.0873203E-1,0E0,0E0,1.532256E-1,0E0,3.9983082E-1,0E0,9.191897E-1,2.0151749E0,4.4619656E-1,3.0072365E0,7.216896E-1,0E0,0E0,0E0,1.0026236E0,6.464009E-1,2.6597434E-1,2.758211E-1,0E0,0E0,1.7793725E0,1.3433751E0,2.1317024E0,3.4361696E-1,2.012741E0,2.170474E0,2.8111386E-1,0E0,0E0,0E0,3.9769322E-1,0E0,0E0,0E0,0E0,0E0,1.1948719E0,1.4213829E0,1.0734844E0,4.937066E-1,2.7860794E0,1.5925484E0,0E0,8.194084E-1,3.0724907E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,69,69,71,71,73,73,74,74,75,75,76,76,77,77,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,97,97,103,103,104,104,105,105,106,106,107,107,108,108,110,110,111,111],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,-1,132,-1,134,136,138,140,142,-1,-1,-1,144,146,148,150,-1,-1,152,154,156,158,160,162,164,-1,-1,-1,166,-1,-1,-1,-1,-1,168,170,172,174,176,178,-1,180,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.896243E4,3.1140776E0,2.14099E5,2.0407547E5,5.964E3,8.592097E-1,5.751617E6,1.6018981E0,6.504693E-2,9.99E2,4.898012E-1,1.5988182E-1,1.546644E6,3.730007E6,1.36054E-1,2.728889E2,2.2056E4,1E0,7.75541E-1,1.838444E6,8E0,9E2,7.160724E1,9.750871E6,5.26E2,2.3292E4,2.4268942E7,1.729767E2,1E0,1E0,9.9668306E-1,2.21875E0,2.695E3,1.6007329E7,3.386E3,1.7232166E8,3.3926086E2,1.6316416E6,2.093563E0,1.5E1,7.997723E6,2.127E3,1.4777102E9,1.5859042E6,4.020408E0,1.37518E5,6.306E3,4.8297736E7,1.8465776E8,4.888E0,1.2974394E6,1.504779E7,2.5003334E7,1E0,9.256843E3,2.0304577E-1,1.9051096E7,8.679864E-3,-3.8917093E-3,2.7682406E8,8E0,1.9557823E0,1.5851064E0,1.6559546E0,8.542683E1,-3.1287079E-3,1.1620289E-2,1.9840922E8,-1.7793521E-2,3.1809045E2,-8.480281E-3,6.2186885E-1,5.676415E-1,6.952967E6,7.218466E7,2.3033286E5,2.4972554E-2,3.5317462E-2,1.4386224E-2,7E0,6.3039363E4,1.5561539E2,6.576923E0,1.4605067E-2,4.349299E-2,5.2228385E6,1.30399E5,1.566593E6,1.662639E7,4.3689142E2,9.908038E1,3.86569E5,-2.534403E-2,-5.0408836E-3,1.4841042E-2,3.270662E6,-1.7755546E-2,4.222626E-3,1.8827507E-2,-5.9512886E-3,-2.8544102E-2,3.5E1,1.8114872E7,1.0219829E1,5.9767612E7,6.7652373E3,1E0,1.3939724E-2,8.7757526E2,7.883E3,-3.9100843E-3,-1.8477492E-2,-2.9917318E-2,-2.714549E-3,-2.1977397E-2,-4.172783E-2,-6.365318E-2,-3.1380527E-2,-1.5298745E-2,-3.191983E-2,-1.0434379E-2,-1.7331542E-2,-8.222017E-4,-6.486354E-3,6.179507E-3,-1.6990216E-2,-7.642719E-3,-1.03235645E-2,1.3059198E-3,-1.5117565E-2,-2.9561939E-2,-5.7718926E-3,-1.8641692E-2,-2.954147E-2,-6.9426806E-3,-7.270126E-3,-1.5811894E-2,-2.7213676E-4,-1.5559657E-2,4.4568034E-4,1.0326746E-2,6.082619E-3,-1.3079393E-2,-3.36598E-2,-1.53597165E-2,-9.2995865E-3,1.6788843E-3,5.4491423E-3,-6.988274E-3,-4.3347143E-3,-3.6591392E-2,1.606166E-2,-1.4672387E-2,-4.3428224E-3,7.358545E-3,9.380508E-3,-2.1672035E-3,1.276732E-2,2.5908256E-2,5.6628776E-3,2.6014706E-2,-1.34601435E-2,3.9031385E-3,-1.5421444E-2,-3.8702958E-3,2.9167082E-2,8.437186E-3,9.918803E-3,2.2161607E-2,2.3269286E-2,1.1517048E-2,6.0425694E-3,-5.801302E-3,2.3799367E-2,4.235028E-2,8.48224E-3,2.4310464E-2,2.5109528E-2,4.3384533E-2,4.7504413E-3,2.3989476E-2],"split_indices":[20,56,33,54,2,28,2,27,45,41,0,2,27,38,28,9,42,52,9,16,34,9,18,2,56,9,2,2,50,52,77,16,39,53,2,45,2,7,52,28,39,8,9,1,7,28,54,29,2,45,5,54,28,45,45,67,4,27,45,0,0,7,3,53,53,56,52,0,0,5,0,4,0,27,42,45,45,33,0,0,0,3,28,52,56,0,0,47,10,9,45,52,56,1,0,0,0,12,0,0,0,0,0,0,12,54,45,52,102,0,52,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.704E3,8.64E2,1.84E3,4.06E2,4.58E2,1.15E3,6.9E2,2.62E2,1.44E2,4.53E2,5E0,5.36E2,6.14E2,3.07E2,3.83E2,1.8E2,8.2E1,8.8E1,5.6E1,3.68E2,8.5E1,2.16E2,3.2E2,5.72E2,4.2E1,5.6E1,2.51E2,3E2,8.3E1,1.74E2,6E0,5E1,3.2E1,7.3E1,1.5E1,2.3E1,3.3E1,1.16E2,2.52E2,7.1E1,1.4E1,1.69E2,4.7E1,1E1,3.1E2,2.61E2,3.11E2,3.6E1,6E0,5E1,6E0,1.1E1,2.4E2,1.4E2,1.6E2,6.8E1,1.5E1,1.17E2,5.7E1,4E0,2E0,3.8E1,1.2E1,2.2E1,1E1,3.2E1,4.1E1,1.1E1,4E0,9E0,1.4E1,2.8E1,5E0,5.6E1,6E1,7E1,1.82E2,6.7E1,4E0,8E0,6E0,1.46E2,2.3E1,3E1,1.7E1,2E0,8E0,2.85E2,2.5E1,1.73E2,8.8E1,1.17E2,1.94E2,1.8E1,1.8E1,2E0,4E0,3.4E1,1.6E1,3E0,3E0,9E0,2E0,1.19E2,1.21E2,8.7E1,5.3E1,9.9E1,6.1E1,4E0,6.4E1,1.3E1,2E0,4.7E1,7E1,2.1E1,3.6E1,2.7E1,1.1E1,4E0,8E0,1.8E1,4E0,7E0,3E0,2.7E1,5E0,1.3E1,2.8E1,4E0,5E0,1.1E1,1.7E1,3.6E1,2E1,4.8E1,1.2E1,2.4E1,4.6E1,1.46E2,3.6E1,3.4E1,3.3E1,6E0,1.4E2,1.1E1,1.2E1,2.4E1,6E0,1.2E1,5E0,2.82E2,3E0,2.2E1,3E0,1.24E2,4.9E1,8.2E1,6E0,7E1,4.7E1,1.82E2,1.2E1,1.6E1,2E0,9E0,2.5E1,6E0,1.13E2,3.2E1,8.9E1,2.8E1,5.9E1,4.4E1,9E0,7E1,2.9E1,3.4E1,2.7E1,1.2E1,5.2E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[-1.8346848E-3,-2.7554688E-1,1.2622665E-1,-4.229637E-1,-8.861101E-2,-7.9248287E-4,3.016821E-1,-4.3935767E-1,5.853826E-2,-2.0995888E-1,1.5193294E-1,-1.5124889E-1,9.422172E-2,1.504549E-1,4.1390675E-1,-5.0994503E-1,-2.026519E-1,-2.3025903E-1,1.2748733E-1,5.138487E-2,3.427205E-1,-1.7443928E-1,3.6734274E-1,1.21811755E-1,-3.212541E-1,6.1054513E-2,3.645707E-1,-7.6224945E-2,4.5234686E-1,-4.4532838E-1,-8.2572794E-1,-1.1482691E-1,-5.16233E-1,-4.5554867E-1,-2.0873152E-1,-2.931076E-2,1.7603105E-2,9.207189E-2,-1.61533E-1,2.546432E-2,2.5522974E-1,-3.2903945E-1,-1.18968785E-1,4.517485E-2,1.3540716E-1,1.994482E-2,2.1115692E-1,-3.5882297E-1,1.3132824E-2,1.1649986E-1,-2.4713102E-1,3.9903194E-1,-2.2459665E-1,1.43685E-2,-3.5277106E-2,3.1117746E-1,5.614743E-1,-5.0500077E-1,-1.7579465E-1,-8.5740876E-1,3.475467E-3,-1.5527041E-1,3.0901697E-1,-8.649813E-2,-6.5467256E-1,-1.5137451E-2,-2.9942818E-2,-2.832092E-1,-1.4878742E-1,1.0848282E-2,-5.822741E-3,1.6009694E-1,2.0375298E-3,-1.0867766E-2,7.3701893E-3,1.2997694E-1,1.599966E-2,-2.8618035E-1,-3.186528E-2,2.1560621E-1,-1.4826816E-1,-3.7199253E-4,1.1673757E-2,-1.6012955E-1,6.5071955E-2,1.9856204E-1,5.6441534E-2,-4.2865124E-1,-3.756253E-3,2.2968031E-1,2.2838583E-2,-3.1518298E-1,6.975698E-3,3.3517888E-1,5.869653E-1,9.261569E-3,-2.3080258E-2,1.3832317E-1,-1.0273704E-1,3.7621644E-1,1.2312745E-1,6.74767E-1,3.6750162E-1,-2.2046011E-2,-3.2298822E-2,-1.2424464E-2,3.624977E-3,-5.4098353E-2,-3.495284E-2,4.2476607E-3,-9.94074E-3,3.0624181E-2,4.4198935E-3,3.9982665E-3,-8.198379E-3,-8.504614E-3,-3.9486114E-2,-1.058987E-2,-2.266847E-2,-4.2960816E-3,-1.2355452E-2,1.4220296E-2,4.465118E-3,-5.197592E-3,3.53456E-3,1.2550784E-2,2.439139E-3,-1.50182415E-2,4.757736E-3,-1.0363526E-2,3.57183E-2,-1.1076832E-2,-2.8597994E-3,-1.1948968E-3,-9.899202E-3,3.748914E-3,-3.3719998E-2,8.0983555E-3,2.6059516E-2,-8.217187E-3,-2.2047872E-2,1.4118726E-2,3.5884478E-3,-1.080528E-2,3.2340689E-3,-1.7587384E-2,-8.100029E-3,3.6031646E-3,1.8522771E-2,3.117383E-2,1.1458572E-2,-1.5627183E-3,8.80137E-3,-9.345598E-3,-1.3302928E-3,2.0819508E-2,6.664944E-3,7.6534064E-3,-1.0742229E-2,3.8508825E-2,2.7432324E-2,1.9476097E-2,-3.3626049E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,-1,81,83,85,87,-1,89,91,93,95,97,-1,99,101,103,105,107,-1,109,111,113,115,-1,-1,117,119,-1,-1,121,123,-1,-1,125,-1,127,-1,129,131,-1,-1,133,135,137,-1,139,-1,141,143,145,-1,147,149,-1,-1,151,153,155,157,159,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.4464714E1,2.3634415E1,4.0917706E1,1.3484062E1,1.1123909E1,1.5253248E1,1.302269E1,7.849884E0,0E0,1.7624578E0,2.4378545E0,5.030961E0,7.535932E0,6.294613E0,8.362076E0,7.100441E0,2.9976916E0,1.064044E0,5.754404E-1,7.5122714E-1,5.918579E-1,3.3568516E0,2.2767751E0,5.583997E0,1.0156302E0,4.027255E0,2.0590544E0,1.9848891E0,6.1415176E0,4.8328476E0,1.798214E0,1.5564537E0,1.3711872E0,2.756753E-1,9.4005203E-1,2.6360813E-1,0E0,4.4064438E-1,3.2371464E-1,0E0,2.7264762E-1,1.3087568E0,2.9041991E0,0E0,2.1907917E-1,2.3507771E0,3.750229E0,7.299628E-1,0E0,2.1042788E0,1.0081942E0,9.072733E-1,7.385937E-1,4.4968432E-1,0E0,2.1674118E0,4.8064346E0,1.581173E0,1.2185994E0,1.2242622E0,0E0,1.0519558E0,5.139098E-1,1.15067065E-1,1.3120537E0,0E0,0E0,1.0402985E0,7.769079E-1,0E0,0E0,3.4742498E-1,2.6193646E-1,0E0,0E0,1.328337E-1,0E0,1.024653E0,0E0,5.658863E0,1.9712939E0,0E0,0E0,3.600098E-1,2.4020832E0,3.206685E0,0E0,1.7386103E-1,0E0,8.850384E-1,1.236707E0,1.7357731E-1,0E0,9.2311525E-1,3.3361435E-1,0E0,0E0,1.2256503E-1,1.0540861E-1,1.7981682E0,6.4760464E-1,1.4381943E0,1.5204964E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,68,68,71,71,72,72,75,75,77,77,79,79,80,80,83,83,84,84,85,85,87,87,89,89,90,90,91,91,93,93,94,94,97,97,98,98,99,99,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,-1,82,84,86,88,-1,90,92,94,96,98,-1,100,102,104,106,108,-1,110,112,114,116,-1,-1,118,120,-1,-1,122,124,-1,-1,126,-1,128,-1,130,132,-1,-1,134,136,138,-1,140,-1,142,144,146,-1,148,150,-1,-1,152,154,156,158,160,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5938586E8,2.14099E5,2.3544883E8,1.8108038E5,2.1446484E-1,2.0163035E3,5.853826E-2,9.026015E2,1.2060912E3,1.46705E3,3.956147E-1,1.4777102E9,2.943086E4,6.971004E7,1.2427474E8,2.750917E-4,1.125584E-3,1.1160287E7,1.4075E4,1.141E4,6.7388856E7,8.46E2,3.1998687E3,5.9767612E7,1.921032E4,2.1506184E7,5.964E3,3.904E3,1.1879E4,1.3554651E2,3.046556E5,9.374544E5,1.8122449E2,1.4E2,1.7603105E-2,2.3686748E0,1E0,2.546432E-2,3.3480989E3,1E0,7.330957E4,4.517485E-2,5.695621E8,3.97E2,1.3664E4,1.9505986E6,1.3132824E-2,9.74026E0,4.6E1,1.08874E5,5.1678925E1,2.6666667E0,-3.5277106E-2,4.020408E0,8.5E1,1E0,3.002239E5,1.8736842E0,3.475467E-3,2.467284E7,6.135771E9,1.141E4,1.5119754E0,-1.5137451E-2,-2.9942818E-2,2.01351E5,1.6256282E7,1.0848282E-2,-5.822741E-3,3.8434E4,6.019605E2,-1.0867766E-2,7.3701893E-3,3E0,1.599966E-2,1.91E2,-3.186528E-2,3.7761906E1,7.364257E4,-3.7199253E-4,1.1673757E-2,1.4396E4,5.8E1,5.5669255E6,5.6441534E-2,1.1880085E2,-3.756253E-3,2.5003334E7,1.4813794E3,1.5694646E3,6.975698E-3,1E0,2.7809634E5,9.261569E-3,-2.3080258E-2,1.4763578E3,1.4126303E0,1E0,2.1381798E-6,1.8903887E0,3.1570474E2,-2.2046011E-2,-3.2298822E-2,-1.2424464E-2,3.624977E-3,-5.4098353E-2,-3.495284E-2,4.2476607E-3,-9.94074E-3,3.0624181E-2,4.4198935E-3,3.9982665E-3,-8.198379E-3,-8.504614E-3,-3.9486114E-2,-1.058987E-2,-2.266847E-2,-4.2960816E-3,-1.2355452E-2,1.4220296E-2,4.465118E-3,-5.197592E-3,3.53456E-3,1.2550784E-2,2.439139E-3,-1.50182415E-2,4.757736E-3,-1.0363526E-2,3.57183E-2,-1.1076832E-2,-2.8597994E-3,-1.1948968E-3,-9.899202E-3,3.748914E-3,-3.3719998E-2,8.0983555E-3,2.6059516E-2,-8.217187E-3,-2.2047872E-2,1.4118726E-2,3.5884478E-3,-1.080528E-2,3.2340689E-3,-1.7587384E-2,-8.100029E-3,3.6031646E-3,1.8522771E-2,3.117383E-2,1.1458572E-2,-1.5627183E-3,8.80137E-3,-9.345598E-3,-1.3302928E-3,2.0819508E-2,6.664944E-3,7.6534064E-3,-1.0742229E-2,3.8508825E-2,2.7432324E-2,1.9476097E-2,-3.3626049E-3],"split_indices":[20,102,7,2,12,28,38,55,0,52,52,52,27,7,33,45,45,39,38,45,9,9,7,2,4,45,4,45,2,2,2,56,32,28,52,10,0,53,80,0,4,73,45,0,5,2,2,48,0,53,3,2,56,53,0,54,8,101,33,53,0,45,5,9,57,0,0,29,45,0,0,9,52,0,0,8,0,3,0,52,28,0,0,9,3,28,0,52,0,45,4,52,0,102,33,0,0,33,41,6,37,53,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.693E3,8.58E2,1.835E3,4.79E2,3.79E2,1.065E3,7.7E2,4.75E2,4E0,2.52E2,1.27E2,4.12E2,6.53E2,3.29E2,4.41E2,3.65E2,1.1E2,2.38E2,1.4E1,8.4E1,4.3E1,3.95E2,1.7E1,6.13E2,4E1,2.33E2,9.6E1,3.2E1,4.09E2,3.05E2,6E1,8.7E1,2.3E1,1.9E1,2.19E2,9E0,5E0,7.1E1,1.3E1,1.2E1,3.1E1,1.03E2,2.92E2,4E0,1.3E1,2.87E2,3.26E2,3.8E1,2E0,1.98E2,3.5E1,9.1E1,5E0,2.9E1,3E0,1.8E2,2.29E2,2.49E2,5.6E1,5.8E1,2E0,8E1,7E0,6E0,1.7E1,1.2E1,7E0,9.6E1,1.23E2,2E0,7E0,4E1,3.1E1,1.1E1,2E0,1.3E1,1.8E1,9.3E1,1E1,2.3E1,2.69E2,6E0,7E0,5.7E1,2.3E2,3.23E2,3E0,3E1,8E0,8.9E1,1.09E2,3E1,5E0,7E1,2.1E1,2E0,3E0,1.4E1,1.5E1,1.33E2,4.7E1,1.43E2,8.6E1,2E2,4.9E1,4.2E1,1.4E1,1.6E1,4.2E1,1.4E1,6.6E1,2E0,5E0,2E0,4E0,5E0,1.2E1,7.4E1,2.2E1,8.1E1,4.2E1,1.2E1,2.8E1,1.2E1,1.9E1,4E0,9E0,8.7E1,6E0,1.3E1,1E1,1.38E2,1.31E2,1.5E1,4.2E1,2.27E2,3E0,2.99E2,2.4E1,4E0,2.6E1,6.2E1,2.7E1,1.6E1,9.3E1,2.1E1,9E0,1.2E1,5.8E1,1.7E1,4E0,3E0,1.1E1,6E0,9E0,1.06E2,2.7E1,4.3E1,4E0,6E1,8.3E1,7.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[7.38637E-3,9.797268E-2,-3.0374405E-1,2.2218784E-2,4.0858358E-1,-4.8061123E-1,-6.537279E-2,-5.7118237E-2,2.4897574E-1,7.6588136E-1,3.38231E-1,-5.010012E-1,8.022406E-2,-1.753539E-1,1.251775E-1,-1.4859292E-1,3.807346E-2,1.8957101E-1,4.3930897E-1,-1.2470475E-2,8.020212E-1,2.7486053E-1,6.3646173E-1,-5.4284924E-1,-2.8811055E-1,1.5190777E-2,-5.1498762E-3,-1.4508761E-1,-2.6178703E-2,2.2252588E-1,-3.3456957E-1,-1.990346E-1,1.702818E-2,8.613689E-2,-3.4345978E-1,5.0753217E-2,2.4441539E-1,-1.753276E-1,5.0067514E-1,6.883975E-1,5.722066E-2,3.7108693E-1,6.775551E-2,9.0885025E-1,3.7823236E-1,-4.9316275E-1,-7.765605E-1,-2.3894012E-3,-3.416335E-1,-3.2083705E-1,-1.0990294E-1,1.4805111E-1,4.348239E-1,-4.1206703E-1,7.875164E-3,-1.7395695E-1,-4.080462E-1,8.9714095E-2,-1.6254705E-1,1.6209288E-1,-9.4796926E-2,-2.9368865E-1,-2.7521055E-2,7.3506035E-2,-2.3238877E-2,1.7324558E-1,3.5873446E-1,2.1934551E-3,-1.9425211E-2,6.515506E-1,3.6918586E-1,7.396438E-1,1.9117853E-2,2.5364107E-1,4.611726E-1,-3.49597E-1,1.3557611E-1,3.138944E-1,9.949306E-1,4.6753854E-1,-1.9752875E-1,-7.0240694E-1,-4.5242184E-1,-6.186981E-1,-1.1854341E0,-7.2022905E-3,8.4138885E-3,-4.0942836E-1,-1.2761447E-1,-1.6486576E-2,-1.645069E-3,-1.3379458E-1,4.4861846E-2,-9.201413E-3,2.104347E-1,2.3324244E-2,8.65822E-3,-1.0821477E-2,-6.53563E-1,-1.1240039E-2,-4.689002E-3,-4.793625E-3,-2.347739E-2,-5.644859E-3,6.8189558E-3,-1.15500195E-2,-7.0821797E-4,3.9879967E-2,6.6070436E-3,-1.1256724E-2,4.2018414E-4,-1.5014447E-2,2.507491E-3,1.1694624E-2,9.377558E-4,5.8075506E-3,1.679558E-2,1.8086782E-2,-1.5146022E-2,1.4626165E-4,3.2752294E-2,2.2956595E-2,1.1944999E-2,3.6738936E-2,1.3396466E-2,1.9379811E-2,9.357921E-3,1.7417908E-2,2.7773544E-2,-6.8150945E-3,-3.1263042E-2,2.8248017E-3,1.5867896E-2,2.6278581E-2,-1.2242954E-3,3.4389954E-2,5.4661196E-2,1.1415201E-2,2.6988776E-2,3.5388148E-3,-1.9305076E-2,-1.5133599E-2,-3.613515E-2,-2.6053755E-2,-1.7493192E-2,-3.0447835E-2,-1.0481941E-2,-2.3212695E-2,-6.116148E-2,-1.4548871E-2,-2.572024E-2,1.0482011E-3,-1.463924E-2,-1.4363955E-2,-5.3987317E-3,4.485082E-3,-4.8237806E-3,4.3340423E-3,-6.950795E-3,5.40154E-3,1.3909321E-2,-1.2239776E-2,-3.7769392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,-1,-1,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,-1,99,101,103,105,107,109,111,-1,113,-1,115,117,-1,-1,119,121,123,-1,125,127,129,131,133,135,137,139,141,143,145,147,-1,-1,149,151,-1,-1,153,155,157,159,-1,-1,-1,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.536602E1,4.8721527E1,2.5391102E1,2.9985928E1,9.946518E0,3.9968338E0,5.428839E0,1.0759581E1,4.8012104E0,2.656746E0,6.2696266E0,2.8192291E0,6.1252934E-1,1.7636127E0,4.326236E0,5.2665377E0,1.1142289E1,2.5076218E0,3.935007E0,0E0,2.0339012E0,5.6147537E0,3.8062286E0,2.845253E0,8.708639E-1,0E0,0E0,9.1084313E-1,0E0,1.1846805E0,7.251853E-1,2.4606724E0,1.9463661E0,7.422846E0,6.137295E-1,1.1942489E0,1.8660975E0,5.2781445E-1,1.6044407E0,4.753933E-1,0E0,1.925003E0,2.6231608E0,1.1216888E0,1.7377605E0,1.6599693E0,2.3002243E0,2.8998196E-1,6.4420176E-1,1.5590072E-1,4.8299074E-1,5.905349E-1,1.7720151E-1,5.2647066E-1,0E0,1.9772549E0,1.2416706E0,1.1757776E0,4.9600863E-1,6.034857E0,2.3262599E0,4.1436386E-1,0E0,8.420196E-1,0E0,1.3226519E0,1.2697268E0,0E0,0E0,8.7898254E-1,5.6653786E-1,3.2925606E-1,0E0,6.773553E-1,1.0392933E0,7.241758E-1,1.1649519E0,4.1259894E-1,1.5338516E-1,4.8178577E-1,3.0878198E-1,4.867096E-1,1.3739624E0,1.06713295E-1,1.518631E-1,0E0,0E0,3.2221603E-1,3.5276777E-1,0E0,0E0,3.5813546E-1,1.3729304E-1,2.5708976E-1,3.031633E-1,0E0,0E0,0E0,1.2103701E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,65,65,66,66,69,69,70,70,71,71,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,87,87,88,88,91,91,92,92,93,93,94,94,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,-1,-1,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,-1,100,102,104,106,108,110,112,-1,114,-1,116,118,-1,-1,120,122,124,-1,126,128,130,132,134,136,138,140,142,144,146,148,-1,-1,150,152,-1,-1,154,156,158,160,-1,-1,-1,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,8.426E3,1E0,1.022332E6,2E0,3.970405E3,1.067536E3,3.3030225E2,3.097E3,1E0,7.7101436E3,9.3026364E-1,1.1708007E7,1.2199979E10,9.302862E7,3.422351E6,4.0844156E7,1E0,1E0,-1.2470475E-2,4.162121E0,2.3907686E7,8.2474226E-1,9.222717E-1,3.94E2,1.5190777E-2,-5.1498762E-3,1.06E3,-2.6178703E-2,5.185489E3,3.3047943E0,5.024605E0,2.4505E4,2.2893274E1,1.2427474E8,3.7832818E0,7.472532E9,3.3012173E9,1.0993947E1,4.728721E0,5.722066E-2,1.5353E4,2.308943E0,6.3039363E4,1.56E2,1.5804776E4,7.95629E5,2.56E2,2.8880222E6,6.1E1,7.9203456E8,1E0,5.9767612E7,2.010856E0,7.875164E-3,1.4735735E2,3.97E2,4.26E2,5.4121188E7,8E0,7.44393E5,1E0,-2.7521055E-2,5.714286E-1,-2.3238877E-2,4.3689142E2,1.0289962E3,2.1934551E-3,-1.9425211E-2,7.549744E7,1.59551E5,1E0,1.9117853E-2,8.6E1,2.1924436E5,1.7231706E-1,7.023838E3,1.5435694E9,7.137324E0,2.1325744E7,9.188401E4,3.7930825E5,1.8903887E0,5.5E1,1.992945E5,-7.2022905E-3,8.4138885E-3,6.5928856E5,1.157E3,-1.6486576E-2,-1.645069E-3,2.9030122E-4,8.182648E6,8.1480905E-2,6.4506575E5,2.3324244E-2,8.65822E-3,-1.0821477E-2,6E0,-1.1240039E-2,-4.689002E-3,-4.793625E-3,-2.347739E-2,-5.644859E-3,6.8189558E-3,-1.15500195E-2,-7.0821797E-4,3.9879967E-2,6.6070436E-3,-1.1256724E-2,4.2018414E-4,-1.5014447E-2,2.507491E-3,1.1694624E-2,9.377558E-4,5.8075506E-3,1.679558E-2,1.8086782E-2,-1.5146022E-2,1.4626165E-4,3.2752294E-2,2.2956595E-2,1.1944999E-2,3.6738936E-2,1.3396466E-2,1.9379811E-2,9.357921E-3,1.7417908E-2,2.7773544E-2,-6.8150945E-3,-3.1263042E-2,2.8248017E-3,1.5867896E-2,2.6278581E-2,-1.2242954E-3,3.4389954E-2,5.4661196E-2,1.1415201E-2,2.6988776E-2,3.5388148E-3,-1.9305076E-2,-1.5133599E-2,-3.613515E-2,-2.6053755E-2,-1.7493192E-2,-3.0447835E-2,-1.0481941E-2,-2.3212695E-2,-6.116148E-2,-1.4548871E-2,-2.572024E-2,1.0482011E-3,-1.463924E-2,-1.4363955E-2,-5.3987317E-3,4.485082E-3,-4.8237806E-3,4.3340423E-3,-6.950795E-3,5.40154E-3,1.3909321E-2,-1.2239776E-2,-3.7769392E-2],"split_indices":[19,2,59,28,17,52,52,52,2,15,52,39,47,5,45,9,45,102,66,0,54,45,57,27,0,0,0,2,0,52,35,53,29,53,45,53,31,5,56,38,0,2,54,28,8,51,29,0,32,3,12,102,45,57,0,52,2,2,12,18,1,96,0,53,0,52,4,0,0,7,9,6,0,11,28,38,4,7,53,1,28,28,53,8,33,0,0,32,0,0,0,42,45,38,28,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.672E3,2.07E3,6.02E2,1.665E3,4.05E2,3.45E2,2.57E2,1.234E3,4.31E2,6.5E1,3.4E2,3.33E2,1.2E1,1.63E2,9.4E1,6.29E2,6.05E2,3.3E2,1.01E2,2E0,6.3E1,2.82E2,5.8E1,2.77E2,5.6E1,5E0,7E0,1.52E2,1.1E1,7.8E1,1.6E1,4.82E2,1.47E2,5.38E2,6.7E1,9.4E1,2.36E2,9E0,9.2E1,5.1E1,1.2E1,1.92E2,9E1,2.7E1,3.1E1,2.31E2,4.6E1,9E0,4.7E1,2.4E1,1.28E2,5.9E1,1.9E1,1.4E1,2E0,4.32E2,5E1,1.05E2,4.2E1,3.79E2,1.59E2,5.7E1,1E1,9.1E1,3E0,1.47E2,8.9E1,5E0,4E0,4.1E1,5.1E1,4.2E1,9E0,8.5E1,1.07E2,1.2E1,7.8E1,4E0,2.3E1,2.7E1,4E0,3.5E1,1.96E2,3.5E1,1.1E1,5E0,4E0,3.5E1,1.2E1,2.2E1,2E0,1.11E2,1.7E1,1.7E1,4.2E1,1.5E1,4E0,9E0,5E0,2.39E2,1.93E2,1.1E1,3.9E1,2.1E1,8.4E1,2.7E1,1.5E1,1.2E1,3.67E2,6.7E1,9.2E1,5.4E1,3E0,2.1E1,7E1,1.15E2,3.2E1,8.7E1,2E0,2E0,3.9E1,2.5E1,2.6E1,3.9E1,3E0,2.2E1,6.3E1,6.1E1,4.6E1,8E0,4E0,5.7E1,2.1E1,2E0,2E0,1E1,1.3E1,9E0,1.8E1,2E0,2E0,5E0,3E1,9.3E1,1.03E2,3.3E1,2E0,2E0,9E0,2.1E1,1.4E1,7E0,5E0,1.1E1,1E2,1.3E1,4E0,1E1,7E0,2E1,2.2E1,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[6.8415073E-3,-2.3900324E-1,1.2297631E-1,-3.7571606E-1,-7.774172E-2,5.3328514E-2,3.906239E-1,-4.2016906E-1,1.7988554E-1,-1.692995E-1,2.3792598E-1,-5.045602E-2,2.1727946E-1,7.023055E-1,3.264362E-1,-1.9315444E-1,-4.9880263E-1,5.37293E-2,-1.4517334E-2,-2.2276229E-1,1.0792277E-2,1.4221674E-1,4.2141357E-1,-2.6582333E-1,-1.7170008E-2,1.4972404E-1,3.7727678E-1,7.4605554E-1,4.5594167E-2,2.6415056E-1,5.684199E-1,-2.1588987E-1,2.2732444E-2,-5.5888563E-1,-3.1610426E-1,2.9770127E-1,-1.2248392E-1,-2.4003726E-1,6.8315E-2,-1.19798474E-1,6.4353324E-2,1.8888097E-1,-1.4320059E-1,4.503752E-1,5.774121E-3,-2.9944035E-1,8.84004E-2,-1.602112E-1,2.2433339E-2,1.7311618E-1,-2.5368458E-1,4.4319814E-1,1.0016107E-1,7.841902E-1,2.3024652E-2,8.910671E-3,-5.2714553E-3,3.2983577E-1,-5.0871864E-2,8.6411256E-1,3.5114542E-1,-2.9500383E-1,-2.4748562E-2,-6.543193E-1,-4.3387097E-1,-5.532174E-1,-2.1831264E-1,6.342485E-3,2.058604E-2,-7.385668E-2,-2.021204E-2,-1.722193E-1,-2.9578283E-1,1.0050049E-2,-4.447256E-4,-4.3488745E-2,-1.0378487E-2,1.5050128E-1,-2.4269756E-2,3.159909E-1,1.1811354E-1,-9.754068E-3,2.1856343E-3,2.4952112E-2,1.33217145E-2,-2.0226671E-1,-3.8940284E-1,2.2388028E-2,-5.372893E-3,-1.3021599E-1,-4.1545063E-1,1.591351E-1,-8.04573E-2,1.4886717E-1,4.7834903E-1,-2.1667829E-2,1.4401774E-3,5.063877E-1,1.8815206E-1,-7.6076365E-3,1.6360363E-1,8.075759E-1,1.5007019E-2,2.5759277E-1,4.7058856E-1,2.4739679E-2,-3.476656E-1,2.8670388E-1,9.56064E-1,2.3009215E-1,3.1947073E-2,-1.1084582E-2,-2.968877E-2,2.8764794E-3,-1.6682466E-2,-2.7135054E-2,-4.190036E-2,-6.0230624E-3,-2.2423416E-2,-1.416683E-2,-3.001126E-2,-6.677955E-4,-1.4735691E-2,6.7610373E-3,-4.9768877E-3,-1.0897801E-2,-2.295959E-3,-1.7846253E-2,-9.423455E-3,-4.5614108E-3,4.6347543E-3,9.785591E-4,9.027367E-3,5.5191903E-3,-2.6279788E-3,1.9164782E-2,8.17318E-4,2.9037767E-5,9.406794E-3,-1.0560492E-2,8.688622E-3,-1.6451137E-2,-3.3853706E-2,-7.821845E-3,2.4148838E-3,-3.179421E-3,-2.191309E-2,3.7854288E-2,5.8623464E-3,-1.0637188E-2,2.860109E-4,3.7811992E-3,1.1300808E-2,1.3559695E-2,2.848814E-2,1.2363486E-2,2.5925787E-2,-9.695655E-3,1.2532416E-2,-2.1256553E-3,1.1427601E-2,1.3736075E-2,3.9478403E-2,1.5234678E-2,7.193999E-3,2.6801009E-2,1.7358784E-2,-1.0702049E-3,2.6346546E-2,-2.4879666E-2,-5.090142E-3,2.5439585E-2,-2.5555792E-3,5.324557E-2,3.1510875E-2,2.2711856E-2,3.6698652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,99,101,-1,-1,-1,103,105,107,109,111,113,115,117,119,121,-1,-1,123,-1,125,127,-1,-1,129,-1,131,133,135,137,-1,-1,-1,-1,139,141,-1,-1,143,145,147,149,151,153,-1,-1,155,157,-1,159,161,-1,163,165,167,169,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.6175255E1,1.8842617E1,3.3756775E1,1.1510826E1,1.1427604E1,2.4496372E1,7.2530365E0,7.561722E0,6.425977E0,2.948635E0,1.5030379E0,6.318246E0,5.974758E0,1.7760468E0,4.552349E0,1.8030186E0,3.3140411E0,0E0,1.0477389E0,1.2030897E0,5.0432444E-1,8.261744E-1,2.0708704E-1,1.4316092E0,4.33725E0,3.7619314E0,2.9698048E0,2.19841E-1,1.298127E-1,5.1898575E0,3.7440987E0,1.6409903E0,0E0,2.5555878E0,1.7574272E0,1.1779481E-1,3.0386797E-1,7.921238E-1,1.5571994E-1,1.4588088E-1,3.9302218E-1,4.3042982E-1,1.120732E-1,1.8355083E-1,0E0,8.6537075E-1,8.888966E-1,1.2167907E0,8.4520035E0,2.6895847E0,1.2543973E0,2.0481815E0,5.682658E-1,2.974701E-1,0E0,0E0,0E0,1.9836407E0,9.960066E-1,1.1063251E0,1.29984E0,1.4464202E0,9.2660755E-1,2.0777664E0,1.0629921E0,2.494402E-1,1.0734615E0,0E0,0E0,1.5043825E-1,0E0,6.9743824E-1,8.4753895E-1,0E0,0E0,1.03596605E-1,0E0,1.2465125E-1,1.17340714E-1,4.461558E-1,3.219476E-1,0E0,0E0,0E0,0E0,4.180627E-1,5.5846596E-1,0E0,0E0,9.054322E-1,2.1953797E-1,5.8846197E0,4.200802E0,2.09448E0,4.4565868E-1,0E0,0E0,7.8030777E-1,8.572498E-1,0E0,4.3185705E-1,1.7403221E-1,0E0,8.5551167E-1,4.614334E-1,9.029104E-1,3.1810105E-1,4.4407624E-1,2.8987122E-1,1.0502359E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,71,71,72,72,75,75,77,77,78,78,79,79,80,80,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,97,97,98,98,100,100,101,101,103,103,104,104,105,105,106,106,107,107,108,108,109,109],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,100,102,-1,-1,-1,104,106,108,110,112,114,116,118,120,122,-1,-1,124,-1,126,128,-1,-1,130,-1,132,134,136,138,-1,-1,-1,-1,140,142,-1,-1,144,146,148,150,152,154,-1,-1,156,158,-1,160,162,-1,164,166,168,170,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.426E3,3.317425E3,7.2041174E2,6.0809356E5,2E0,2.0304577E-1,2.46E2,1.2690893E0,1.3664E4,8.2E1,2.9892595E6,4.5481584E7,6.7652373E3,1.7474695E0,5.676415E-1,5.37293E-2,3.3517068E7,1.1159378E12,1.752512E2,6.1061732E7,2.970339E0,1.6006084E3,5.52E2,9.837297E0,1E0,2.3977574E7,3.01E2,3.5301748E7,1.1336898E-1,3.4986075E6,2.2732444E-2,2.0521326E0,1.1880085E2,4.8765745E0,1.921032E4,8.3306855E6,1.8000048E10,1.2062E5,1.3510204E1,6.5346925E6,1.60264E6,1.5609541E0,5.774121E-3,9.82542E5,1.83E2,2.4E1,2.2E1,7.064848E6,3.1E2,2E0,7.3308685E2,1E0,2.3024652E-2,8.910671E-3,-5.2714553E-3,2.1019447E-1,1.1623708E0,6.3039363E4,3.2794893E-1,5.69815E5,1E0,7.8599895E9,2.5332516E5,1.3177E4,1.3264E4,6.342485E-3,2.058604E-2,7.364257E4,-2.021204E-2,5.980516E5,2.728E3,1.0050049E-2,-4.447256E-4,1E0,-1.0378487E-2,1.9563605E5,7.576E3,2.5482938E0,5.0079144E5,-9.754068E-3,2.1856343E-3,2.4952112E-2,1.33217145E-2,1.038E3,7.492813E0,2.2388028E-2,-5.372893E-3,3.422351E6,2.873721E2,2.04E5,7.44393E5,1.06E3,1E0,-2.1667829E-2,1.4401774E-3,8.490231E-1,3.14088E5,-7.6076365E-3,1.4523809E0,4.785384E5,1.5007019E-2,1.3152658E7,1.9493858E7,6.702495E-1,2.4832896E9,3.5762883E4,2.0967E4,4.3978744E0,3.1947073E-2,-1.1084582E-2,-2.968877E-2,2.8764794E-3,-1.6682466E-2,-2.7135054E-2,-4.190036E-2,-6.0230624E-3,-2.2423416E-2,-1.416683E-2,-3.001126E-2,-6.677955E-4,-1.4735691E-2,6.7610373E-3,-4.9768877E-3,-1.0897801E-2,-2.295959E-3,-1.7846253E-2,-9.423455E-3,-4.5614108E-3,4.6347543E-3,9.785591E-4,9.027367E-3,5.5191903E-3,-2.6279788E-3,1.9164782E-2,8.17318E-4,2.9037767E-5,9.406794E-3,-1.0560492E-2,8.688622E-3,-1.6451137E-2,-3.3853706E-2,-7.821845E-3,2.4148838E-3,-3.179421E-3,-2.191309E-2,3.7854288E-2,5.8623464E-3,-1.0637188E-2,2.860109E-4,3.7811992E-3,1.1300808E-2,1.3559695E-2,2.848814E-2,1.2363486E-2,2.5925787E-2,-9.695655E-3,1.2532416E-2,-2.1256553E-3,1.1427601E-2,1.3736075E-2,3.9478403E-2,1.5234678E-2,7.193999E-3,2.6801009E-2,1.7358784E-2,-1.0702049E-3,2.6346546E-2,-2.4879666E-2,-5.090142E-3,2.5439585E-2,-2.5555792E-3,5.324557E-2,3.1510875E-2,2.2711856E-2,3.6698652E-3],"split_indices":[20,102,2,52,52,28,17,27,10,42,2,29,51,47,52,38,42,0,51,31,52,45,34,55,2,54,6,47,0,45,57,32,0,53,52,57,4,45,5,1,58,45,1,57,0,31,0,8,3,28,10,17,4,6,0,0,0,35,41,28,38,9,104,5,28,9,10,0,0,28,0,28,10,0,0,8,0,28,9,42,28,0,0,0,0,0,54,0,0,9,33,5,1,2,102,0,0,53,1,0,54,28,0,45,51,39,7,28,10,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.666E3,8.55E2,1.811E3,4.62E2,3.93E2,1.438E3,3.73E2,4.28E2,3.4E1,3.05E2,8.8E1,8.81E2,5.57E2,6.2E1,3.11E2,1.11E2,3.17E2,5E0,2.9E1,2.35E2,7E1,5.9E1,2.9E1,1.17E2,7.64E2,3.93E2,1.64E2,5.8E1,4E0,2.49E2,6.2E1,1.08E2,3E0,2.37E2,8E1,7E0,2.2E1,2.22E2,1.3E1,2E1,5E1,5.1E1,8E0,2.6E1,3E0,1.07E2,1E1,1.65E2,5.99E2,3.72E2,2.1E1,1.32E2,3.2E1,4.9E1,9E0,2E0,2E0,2.06E2,4.3E1,2.5E1,3.7E1,7.6E1,3.2E1,1.32E2,1.05E2,2.2E1,5.8E1,4E0,3E0,2E1,2E0,1.02E2,1.2E2,4E0,9E0,1.2E1,8E0,2.5E1,2.5E1,1.7E1,3.4E1,6E0,2E0,1.7E1,9E0,5.3E1,5.4E1,3E0,7E0,1.49E2,1.6E1,2.57E2,3.42E2,3.46E2,2.6E1,1.2E1,9E0,1.05E2,2.7E1,6E0,2.6E1,4.6E1,3E0,1.38E2,6.8E1,3.5E1,8E0,4E0,2.1E1,2.8E1,9E0,6.5E1,1.1E1,2.6E1,6E0,9.7E1,3.5E1,1.1E1,9.4E1,6E0,1.6E1,1.8E1,4E1,2E0,1.8E1,7E1,3.2E1,6.6E1,5.4E1,9E0,3E0,6E0,1.9E1,4E0,2.1E1,1.3E1,4E0,1.4E1,2E1,5.1E1,2E0,4.9E1,5E0,1.26E2,2.3E1,2E0,1.4E1,1.3E1,2.44E2,1.29E2,2.13E2,1.93E2,1.53E2,1.1E1,1.5E1,1.4E1,9.1E1,4E0,2.3E1,7E0,1.9E1,2E0,4.4E1,8.7E1,5.1E1,3.5E1,3.3E1,3.3E1,2E0,4E0,4E0,2E0,2E0,1.2E1,9E0,1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-1.1700931E-3,-2.2083169E-1,1.02506E-1,-3.720529E-1,-8.384257E-2,4.123486E-2,3.690944E-1,-4.5417738E-1,-2.3221032E-1,-1.0038841E-1,5.5686913E-2,-4.622238E-2,2.0783722E-1,4.3822956E-1,1.05671436E-1,-3.3931053E-1,-5.5303556E-1,-1.4410281E-1,-3.5920516E-1,-2.0287484E-1,1.305622E-1,-1.03566065E-1,6.720343E-2,2.2560506E-1,-2.9035056E-1,6.212498E-1,3.6558983E-1,-2.5730671E-2,4.4860345E-1,-3.7221906E-1,1.7204954E-1,-6.011704E-1,-2.7628335E-1,-2.1252732E-1,-1.532787E-2,-6.1976504E-1,-3.1477454E-1,-3.136656E-1,-1.133257E-1,8.937629E-2,4.3901873E-1,-7.8104414E-2,-3.652995E-1,3.527504E-3,1.7417094E-1,1.6358863E-1,4.1913912E-1,-4.0033913E-1,-1.7481353E-4,4.7435302E-2,5.351954E-1,3.131524E-1,7.6370424E-1,-2.096719E-2,4.6495512E-2,3.2184553E-1,3.307948E-2,-2.7165666E-1,-4.581976E-1,1.507775E-2,-8.183383E-3,-5.5537945E-1,-8.694184E-1,-1.518201E-2,-5.499286E-3,-1.6607599E-1,-4.572986E-1,-4.7694817E-2,5.349649E-3,-5.8775647E-3,-3.6635727E-2,-3.0747766E-2,-2.6920688E-1,-9.51993E-3,-3.7441197E-1,-1.5878254E-1,1.6213012E-1,-5.486665E-2,1.42875E-1,4.830728E-3,5.0929374E-1,-1.7302787E-1,3.581178E-2,-4.3912455E-1,-9.902181E-3,-1.2418658E-2,2.196395E-2,-2.1266206E-1,2.0686261E-1,1.7208724E-1,-2.3932112E-2,5.390719E-1,2.9626742E-1,-2.2260321E-2,-1.4594613E-3,2.6001596E-1,6.2334126E-1,1.1662375E-1,3.6537823E-1,9.087846E-3,3.886845E-2,3.3189926E-1,-1.3241038E-2,2.4056094E-2,8.686377E-3,-1.5566328E-2,-5.8053113E-3,-1.5621327E-2,-2.774462E-2,-4.001646E-2,-2.416033E-2,-4.611535E-2,-8.351378E-3,6.4470153E-3,-9.287821E-3,-2.5623959E-2,-5.9115887E-3,-4.4842646E-3,2.6143896E-3,-7.2903335E-3,-1.508508E-2,-7.446386E-3,5.069164E-3,-2.3715764E-2,-1.0465999E-2,-3.5979275E-3,-1.1838676E-2,-1.0360867E-3,1.2779582E-2,-2.16084E-4,-1.657113E-2,-1.5609409E-3,1.0280591E-2,3.0951483E-2,1.434488E-2,1.9488862E-2,-9.056114E-3,6.256312E-3,-8.537145E-3,-1.6985603E-2,-2.674975E-2,-2.6305504E-3,2.9071986E-3,-2.102909E-2,5.500749E-4,1.4881802E-2,5.722099E-3,6.9257254E-3,1.674554E-2,4.0641394E-2,2.269407E-2,1.9985855E-2,8.996729E-3,1.5797975E-2,1.2630989E-3,3.2295052E-2,1.3906097E-2,8.794423E-3,-1.262952E-3,1.8084612E-2,-1.4724937E-2,1.4353474E-3,2.0475438E-2,5.3934744E-3,-6.0574627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,95,97,99,-1,101,103,-1,105,107,-1,-1,109,111,-1,-1,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,131,-1,133,135,137,139,-1,141,-1,143,145,147,-1,149,151,-1,-1,153,155,157,159,-1,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1010494E1,1.7763557E1,2.971298E1,4.5802116E0,9.439987E0,2.1588642E1,6.1456985E0,2.7193604E0,1.6601849E0,1.0606667E1,0E0,6.3290944E0,4.579878E0,3.3491135E0,3.2560062E0,2.1002417E0,1.6456223E0,8.073877E-1,5.226779E-1,3.0400305E0,1.7139225E0,4.276519E0,2.226666E0,5.848564E0,5.66146E-1,1.8530273E0,3.8316803E0,1.5978097E0,3.9152288E-1,8.611946E-1,4.4552532E-1,9.2972565E-1,1.08273864E-1,6.0625076E-1,1.4101359E-1,4.835651E-1,6.4736414E-1,2.5460873E0,2.1919372E0,9.570069E-1,3.2884192E-1,6.3860536E0,5.76313E-1,1.50066E0,1.5868309E0,2.1999617E0,1.593914E0,2.9585814E-1,0E0,0E0,1.366045E0,1.7433033E0,4.5088196E-1,0E0,7.886503E-1,2.891351E-1,0E0,4.0228987E-1,7.704973E-1,0E0,0E0,1.0186291E0,8.070116E-1,0E0,0E0,4.6177208E-1,1.6758335E-1,1.3198736E-1,0E0,0E0,0E0,0E0,2.1983433E-1,4.2062145E-1,2.0570736E0,1.0835431E0,4.868676E-1,4.969115E-1,1.1407752E0,0E0,1.7999959E-1,3.0785017E0,5.483966E0,1.8516874E-1,0E0,6.250286E-1,0E0,5.1261675E-1,9.886079E-1,1.7592697E0,0E0,7.8863335E-1,7.221265E-1,0E0,0E0,2.6877987E-1,6.024723E-1,3.6897516E-1,1.2449188E0,0E0,0E0,2.1821475E-1,5.7011884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51,52,52,54,54,55,55,57,57,58,58,61,61,62,62,65,65,66,66,67,67,72,72,73,73,74,74,75,75,76,76,77,77,78,78,80,80,81,81,82,82,83,83,85,85,87,87,88,88,89,89,91,91,92,92,95,95,96,96,97,97,98,98,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,96,98,100,-1,102,104,-1,106,108,-1,-1,110,112,-1,-1,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,132,-1,134,136,138,140,-1,142,-1,144,146,148,-1,150,152,-1,-1,154,156,158,160,-1,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.631E3,2.8636363E0,2.62797E5,7.733016E5,3.5301748E7,6.748543E-1,5.751617E6,9.451483E-1,5.5686913E-2,1.645614E6,4.898012E-1,4.4365574E-4,7.7101436E3,5.378E3,1.0118026E0,2.877528E2,1.9646037E-4,1E0,1.121807E0,4.0844156E7,1.8108038E5,3.097E3,5.98E2,1.7324902E8,7.765071E0,3.0972284E1,2.4347392E-1,1.6904992E-1,1.447E3,1.0017953E8,1.681727E6,1.2342778E1,8.2474226E-1,7.279E3,1.4921534E5,2.9173258E-4,6.624E3,8.890291E2,1.6081998E5,3.5259784E2,5.1861375E6,1.5435694E9,4.54E2,4.9E1,1.2909952E1,1.3715873E3,-1.7481353E-4,4.7435302E-2,1E0,1.2204E4,2.3936528E5,-2.096719E-2,1.6507992E9,1.19507775E-1,3.307948E-2,7.5646E4,1.8333334E0,1.507775E-2,-8.183383E-3,4.8543688E-2,3.724138E0,-1.518201E-2,-5.499286E-3,1.2E1,2.8107285E-1,3.1851864E7,5.349649E-3,-5.8775647E-3,-3.6635727E-2,-3.0747766E-2,3.5229592E0,5.2350176E4,1.067536E3,1.3304372E7,3.6873734E5,4.616371E8,1E0,4.830728E-3,2.093563E0,6E0,2.429717E1,1.13808104E8,-9.902181E-3,3.474851E8,2.196395E-2,5.033165E7,1E0,5.0083565E6,-2.3932112E-2,2.5737E4,1.46617E5,-2.2260321E-2,-1.4594613E-3,3.84E2,5.632258E0,1.4274633E7,1.314806E8,9.087846E-3,3.886845E-2,9.313E3,4.5843E7,2.4056094E-2,8.686377E-3,-1.5566328E-2,-5.8053113E-3,-1.5621327E-2,-2.774462E-2,-4.001646E-2,-2.416033E-2,-4.611535E-2,-8.351378E-3,6.4470153E-3,-9.287821E-3,-2.5623959E-2,-5.9115887E-3,-4.4842646E-3,2.6143896E-3,-7.2903335E-3,-1.508508E-2,-7.446386E-3,5.069164E-3,-2.3715764E-2,-1.0465999E-2,-3.5979275E-3,-1.1838676E-2,-1.0360867E-3,1.2779582E-2,-2.16084E-4,-1.657113E-2,-1.5609409E-3,1.0280591E-2,3.0951483E-2,1.434488E-2,1.9488862E-2,-9.056114E-3,6.256312E-3,-8.537145E-3,-1.6985603E-2,-2.674975E-2,-2.6305504E-3,2.9071986E-3,-2.102909E-2,5.500749E-4,1.4881802E-2,5.722099E-3,6.9257254E-3,1.674554E-2,4.0641394E-2,2.269407E-2,1.9985855E-2,8.996729E-3,1.5797975E-2,1.2630989E-3,3.2295052E-2,1.3906097E-2,8.794423E-3,-1.262952E-3,1.8084612E-2,-1.4724937E-2,1.4353474E-3,2.0475438E-2,5.3934744E-3,-6.0574627E-3],"split_indices":[20,56,2,54,2,28,45,27,45,42,0,9,27,57,52,2,39,4,42,102,34,45,28,2,10,7,35,56,38,27,0,45,28,54,57,9,28,27,2,4,28,52,50,7,2,3,56,52,0,0,100,2,32,0,5,38,0,9,56,0,0,58,56,0,0,18,27,32,0,0,0,0,54,28,52,45,28,7,102,0,39,3,53,45,0,7,0,7,16,28,0,9,9,0,0,0,54,45,1,0,0,2,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.677E3,8.58E2,1.819E3,4.07E2,4.51E2,1.48E3,3.39E2,2.55E2,1.52E2,4.46E2,5E0,9.71E2,5.09E2,2.68E2,7.1E1,1.2E2,1.35E2,9.1E1,6.1E1,3.09E2,1.37E2,6.45E2,3.26E2,4.92E2,1.7E1,7.4E1,1.94E2,5.2E1,1.9E1,1.13E2,7E0,1.14E2,2.1E1,5.9E1,3.2E1,7E0,5.4E1,1.37E2,1.72E2,1.22E2,1.5E1,5.89E2,5.6E1,2.05E2,1.21E2,3.74E2,1.18E2,1.2E1,5E0,1.2E1,6.2E1,1.73E2,2.1E1,7E0,4.5E1,1.4E1,5E0,5.4E1,5.9E1,5E0,2E0,1E2,1.4E1,1.6E1,5E0,5.1E1,8E0,2.6E1,6E0,2E0,5E0,5E0,4.9E1,2.3E1,1.14E2,1.48E2,2.4E1,3.3E1,8.9E1,3E0,1.2E1,3.21E2,2.68E2,3.7E1,1.9E1,1.99E2,6E0,9E0,1.12E2,3.7E2,4E0,5.8E1,6E1,1E1,2E0,1.6E1,4.6E1,3.7E1,1.36E2,2E0,1.9E1,7E0,3.8E1,5E0,9E0,3.9E1,1.5E1,3E1,2.9E1,1.3E1,8.7E1,1.2E1,2E0,4E0,4.7E1,6E0,2E0,1.8E1,8E0,1.5E1,3.4E1,1E1,1.3E1,6.3E1,5.1E1,7.7E1,7.1E1,9E0,1.5E1,2.9E1,4E0,2.6E1,6.3E1,6E0,6E0,8E0,3.13E2,1.86E2,8.2E1,2.4E1,1.3E1,1.26E2,7.3E1,4E0,5E0,5E1,6.2E1,3.22E2,4.8E1,8E0,5E1,2.7E1,3.3E1,1.2E1,4E0,3.9E1,7E0,2.5E1,1.2E1,1.34E2,2E0,2E0,5E0,1.8E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-1.8533156E-4,7.718929E-2,-2.7878785E-1,-1.8487187E-2,2.4208145E-1,-4.4018972E-1,-5.9117075E-2,-3.4032863E-2,5.139266E-1,1.9574815E-1,5.302279E-1,-8.904963E-1,-4.143258E-1,-1.526585E-1,8.821755E-2,-8.014152E-2,1.5181763E-1,3.463094E-1,5.1637992E-2,-2.3115821E-1,2.1888073E-1,6.236E-3,5.7139736E-1,-9.9007004E-1,-1.5294529E-2,-4.71162E-1,-2.4290088E-1,-1.0945926E-1,-4.4247124E-1,1.7611788E-1,-2.7265316E-1,-1.2911242E-1,5.5460073E-2,1.7094187E-1,-2.1686712E-2,3.837709E-1,-5.687864E-3,-1.5979981E-2,-1.09011136E-1,1.8838236E-1,4.9477762E-1,6.2277055E-1,2.59314E-1,-1.832884E-2,-5.3681314E-2,-4.278674E-1,-7.8724E-1,1.00098915E-1,-3.109471E-1,-7.179725E-2,-2.4543749E-1,-2.971236E-2,-2.566502E-1,1.1371355E-1,3.7434798E-1,-1.3172558E-1,-3.0693294E-2,-2.0500316E-1,-3.4374062E-2,1.755707E-1,1.0060853E-2,1.3205121E-1,4.4525653E-1,4.1548496E-1,4.291729E-3,-9.367991E-3,2.0425245E-3,2.105947E-1,-9.005501E-2,2.1238561E-1,6.6011E-1,6.3987553E-1,-1.4151166E-3,8.720725E-2,2.1241812E-2,-6.4799404E-1,-4.010029E-1,-2.470162E-2,-8.9430815E-1,1.8032188E-2,-4.564451E-3,-7.648353E-2,-3.6597568E-1,-2.2085959E-1,-4.4192813E-2,-3.0012348E-1,-1.8440108E-3,-1.4794427E-2,-6.841471E-4,1.5143657E-1,-4.9798228E-2,2.1796564E-2,3.5094381E-3,-2.904395E-1,4.3454006E-2,-8.659856E-3,-1.9866806E-2,3.760486E-2,-2.9778187E-3,9.84364E-3,-3.0528118E-3,1.2138577E-3,-1.6964309E-2,6.895193E-3,-2.5672827E-2,1.2013704E-2,2.6081199E-2,2.132424E-2,7.0779156E-3,4.6586744E-3,1.3347322E-2,2.756037E-3,-1.4870225E-2,1.6509587E-2,-4.5690886E-3,3.8387332E-2,1.7257964E-2,2.477213E-2,3.533877E-2,-2.5544963E-3,1.007202E-2,-3.3627503E-2,-1.1598347E-2,-1.3438389E-2,-2.1737168E-2,-1.3411587E-2,-4.5667835E-2,-9.312574E-3,8.369203E-3,-1.3977025E-2,-2.8786132E-2,-6.8995254E-3,-1.728781E-2,-3.4483615E-3,1.855611E-3,-6.2171645E-3,-1.7042968E-2,1.3834342E-2,5.2099684E-3,-1.0221809E-2,2.8113348E-3,-2.431578E-3,-2.0309621E-2,6.43387E-3,-3.880267E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,43,-1,45,47,49,51,53,55,57,59,61,-1,63,-1,-1,65,67,69,71,73,-1,-1,75,77,79,81,83,85,-1,87,89,91,93,-1,95,97,99,101,103,105,107,-1,-1,-1,109,111,113,115,117,-1,119,-1,121,123,-1,125,-1,-1,127,129,131,133,135,-1,-1,-1,137,139,-1,-1,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.662958E1,3.244055E1,2.0218937E1,1.0793796E1,9.976044E0,3.4628067E0,3.3624666E0,1.0860923E1,3.1264362E0,6.4893875E0,1.6383076E0,6.505928E-1,2.935154E0,1.8170011E0,3.0614812E0,6.747301E0,2.9914365E0,5.6825495E-1,0E0,3.893869E-1,5.1153107E0,0E0,1.345581E0,6.835699E-1,0E0,2.8653297E0,1.8983126E0,6.5325725E-1,4.6904206E-1,9.0995383E-1,9.016168E-1,5.356515E0,1.4705243E0,2.5549312E0,0E0,2.2397423E-1,0E0,0E0,2.3162185E-1,3.4762383E0,2.7078028E0,9.11541E-1,4.3694115E-1,0E0,0E0,9.701805E-1,3.2343102E-1,7.9212475E-1,8.364558E-1,4.181769E-1,3.0530095E-1,0E0,1.3494325E-1,3.7763065E-1,4.0443897E-1,4.2967233E-1,0E0,2.0252914E0,7.637721E0,5.439222E-1,1.1059606E0,1.7984223E0,4.4275188E-1,1.3352442E-1,0E0,0E0,0E0,3.9708347E0,1.3845136E0,1.0105644E0,1.3223648E0,5.5583763E-1,0E0,1.6693854E-1,0E0,2.9166794E-1,1.0697727E0,0E0,3.8838863E-1,0E0,0E0,4.489327E-1,7.656059E-1,1.201601E-1,2.0808992E-1,1.6267681E-1,0E0,0E0,0E0,2.557807E-1,2.255958E-1,0E0,0E0,2.3470825E-1,1.01396695E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,67,67,68,68,69,69,70,70,71,71,73,73,75,75,76,76,78,78,81,81,82,82,83,83,84,84,85,85,89,89,90,90,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,44,-1,46,48,50,52,54,56,58,60,62,-1,64,-1,-1,66,68,70,72,74,-1,-1,76,78,80,82,84,86,-1,88,90,92,94,-1,96,98,100,102,104,106,108,-1,-1,-1,110,112,114,116,118,-1,120,-1,122,124,-1,126,-1,-1,128,130,132,134,136,-1,-1,-1,138,140,-1,-1,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.0950326E6,1E0,1.6173E4,3.1100148E6,6E0,1.0323588E3,9.9625344E5,5.3534385E3,1.83E2,6.2671E4,1.9125667E3,5.121E3,4.8297736E7,9.6910744E7,4.264897E6,6.975062E-1,2.2668628E7,5.1637992E-2,9.53882E-2,6.7652373E3,6.236E-3,2.3487206E7,1.9041E4,-1.5294529E-2,1.1034263E10,1E0,1.1095187E1,6.654321E0,5.185489E3,1.4299594E8,2.6659616E2,1.07E2,6.057671E6,-2.1686712E-2,1.65E2,-5.687864E-3,-1.5979981E-2,8.829276E-8,3.5301748E7,2.1150263E-1,1E0,1.5311552E9,-1.832884E-2,-5.3681314E-2,5.85E2,1E1,1.447E3,1E1,1.287E3,2.2376953E1,-2.971236E-2,6.823E3,5.976267E6,5.9767612E7,6.08767E3,-3.0693294E-2,4.602015E0,8E0,9.47306E3,1E0,4.9E1,1E0,5.794597E7,4.291729E-3,-9.367991E-3,2.0425245E-3,5.862126E2,2.3432E4,1.5401015E1,2.60332E5,3.1547058E1,-1.4151166E-3,1.3948507E7,2.1241812E-2,1.74E4,3.8609805E10,-2.470162E-2,2.8107285E-1,1.8032188E-2,-4.564451E-3,2.6740572E5,1.2E1,8.595346E-1,2.491016E-1,2.69E2,-1.8440108E-3,-1.4794427E-2,-6.841471E-4,1.18827E5,9.877E3,2.1796564E-2,3.5094381E-3,1.747E3,1.10018056E8,-8.659856E-3,-1.9866806E-2,3.760486E-2,-2.9778187E-3,9.84364E-3,-3.0528118E-3,1.2138577E-3,-1.6964309E-2,6.895193E-3,-2.5672827E-2,1.2013704E-2,2.6081199E-2,2.132424E-2,7.0779156E-3,4.6586744E-3,1.3347322E-2,2.756037E-3,-1.4870225E-2,1.6509587E-2,-4.5690886E-3,3.8387332E-2,1.7257964E-2,2.477213E-2,3.533877E-2,-2.5544963E-3,1.007202E-2,-3.3627503E-2,-1.1598347E-2,-1.3438389E-2,-2.1737168E-2,-1.3411587E-2,-4.5667835E-2,-9.312574E-3,8.369203E-3,-1.3977025E-2,-2.8786132E-2,-6.8995254E-3,-1.728781E-2,-3.4483615E-3,1.855611E-3,-6.2171645E-3,-1.7042968E-2,1.3834342E-2,5.2099684E-3,-1.0221809E-2,2.8113348E-3,-2.431578E-3,-2.0309621E-2,6.43387E-3,-3.880267E-3],"split_indices":[19,47,59,2,28,3,52,28,4,29,1,4,2,45,45,9,27,45,0,42,52,0,50,9,0,5,64,54,54,52,45,52,10,28,0,3,0,0,37,45,38,104,12,0,0,2,3,0,3,2,56,0,2,47,45,4,0,53,18,48,85,3,102,1,0,0,0,52,9,54,11,58,0,9,0,9,31,0,27,0,0,33,18,27,38,0,0,0,0,29,9,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.625E3,2.055E3,5.7E2,1.301E3,7.54E2,3.28E2,2.42E2,1.265E3,3.6E1,6.51E2,1.03E2,1.6E1,3.12E2,1.48E2,9.4E1,1.014E3,2.51E2,2.9E1,7E0,3.3E1,6.18E2,1E1,9.3E1,1.3E1,3E0,2.33E2,7.9E1,1.3E2,1.8E1,7.6E1,1.8E1,7.45E2,2.69E2,2.44E2,7E0,2.7E1,2E0,1.7E1,1.6E1,5.58E2,6E1,7.9E1,1.4E1,3E0,1E1,2.07E2,2.6E1,1.3E1,6.6E1,1.03E2,2.7E1,8E0,1E1,5.9E1,1.7E1,1.4E1,4E0,4.13E2,3.32E2,7.3E1,1.96E2,2.15E2,2.9E1,2.4E1,3E0,1E1,6E0,5.17E2,4.1E1,2.3E1,3.7E1,7.7E1,2E0,8E0,6E0,2E1,1.87E2,9E0,1.7E1,5E0,8E0,1.3E1,5.3E1,1.5E1,8.8E1,2.1E1,6E0,8E0,2E0,4.8E1,1.1E1,1.3E1,4E0,7E0,7E0,3.72E2,4.1E1,1E1,3.22E2,6.5E1,8E0,1.89E2,7E0,2.12E2,3E0,1.1E1,1.8E1,2.1E1,3E0,1.95E2,3.22E2,2.5E1,1.6E1,1.6E1,7E0,2.4E1,1.3E1,3.7E1,4E1,4E0,4E0,1.7E1,3E0,5.9E1,1.28E2,2E0,1.5E1,9E0,4E0,4.2E1,1.1E1,1.1E1,4E0,6.6E1,2.2E1,6E0,1.5E1,1E1,3.8E1,4E0,7E0,3E0,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[-1.1812591E-4,-2.1217752E-1,9.950555E-2,-3.3008397E-1,-6.9691785E-2,1.2290686E-2,2.8372344E-1,-3.4806302E-1,5.140371E-2,-1.4491461E-1,1.8358131E-1,-1.5911955E-1,5.697012E-2,-1.6482951E-1,3.1343165E-1,-4.1546497E-1,-1.6751058E-1,-1.9065656E-1,8.568481E-3,-9.287021E-2,2.4718828E-1,-1.3739425E-1,-5.7674456E-1,1.4927594E-1,-5.2786626E-2,-2.2034866E-1,2.7312504E-3,2.4332842E-1,4.411571E-1,-3.023017E-1,-5.2885777E-1,-2.7848902E-1,-4.83281E-2,-2.2789668E-1,-7.950142E-2,6.396678E-2,-1.488694E-1,-1.8736912E-1,8.208487E-3,1.7515372E-1,4.2211872E-1,-1.9731843E-1,-2.5184227E-2,-5.4248464E-3,-4.8912223E-2,3.7033524E-2,1.3044512E-1,-8.483824E-2,3.296145E-1,-1.134739E-2,1.1789459E-4,1.726036E-1,4.52706E-1,3.2321626E-1,5.802711E-1,-3.9668986E-1,-1.6043806E-1,-5.4921764E-1,9.203449E-2,-2.2308059E-1,-4.7870055E-1,7.05412E-2,-1.3864648E-1,-2.0584533E-1,-4.5575833E-1,-1.9969797E-1,-5.008102E-4,-2.3493841E-2,1.4257969E-1,-1.190015E-2,-2.1228474E-3,-5.493766E-3,-1.8740091E-2,3.2574403E-1,7.8579165E-2,2.2445519E-2,9.684837E-5,-3.001944E-1,-1.5768589E-1,1.09362975E-1,-1.2837303E-1,-1.7208385E-1,1.5264909E-1,-5.5696737E-2,-3.640512E-1,3.9571773E-2,2.1548674E-1,1.8689074E-1,-2.2304924E-2,5.415034E-1,1.9705012E-1,3.748758E-1,3.6789384E-2,6.722104E-1,2.6806518E-1,-1.4159009E-2,-2.5497314E-2,2.1292619E-5,-1.6351983E-2,-2.8597225E-2,-1.1688683E-2,-5.369946E-3,1.597509E-2,-1.48337865E-2,-1.2660619E-3,-2.536114E-2,-6.3517992E-3,1.5126711E-3,1.3727201E-2,-1.1317879E-2,-1.5929495E-3,-1.0287399E-2,5.0801346E-3,-2.8260436E-2,-1.1355379E-2,-4.698423E-3,-1.2944702E-2,-2.591937E-3,5.3279162E-3,2.3805564E-3,-4.0496425E-3,7.998828E-3,-1.9024961E-3,1.8258393E-2,3.9733727E-3,-2.4188573E-4,7.5469175E-3,-6.9455393E-3,-1.6192617E-2,-4.960649E-3,-1.1414312E-2,1.993946E-2,3.3537608E-3,2.741936E-5,-8.691419E-3,8.416972E-3,-1.1298114E-2,3.9031713E-3,1.14904875E-2,-6.5663285E-3,1.5550986E-3,-2.3483902E-2,-3.9948937E-3,1.3105262E-2,-1.036583E-2,1.1249707E-2,4.1080574E-3,3.495401E-2,2.103797E-2,1.3737437E-2,-5.7545435E-3,3.8826056E-3,1.9912638E-2,-2.686017E-2,7.368813E-3,2.488727E-2,3.5861753E-2,4.460515E-3,1.8749028E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,-1,-1,-1,81,83,85,-1,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,-1,-1,-1,-1,123,125,-1,-1,127,129,131,133,135,137,139,141,-1,143,145,-1,147,149,151,153,155,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4991318E1,1.3954586E1,2.845035E1,1.1854408E1,7.2282295E0,9.222255E0,7.6339912E0,5.406723E0,0E0,2.0516534E0,1.5527854E0,2.1799421E0,9.683165E0,4.5599717E-1,4.6740723E0,4.036148E0,1.6266501E0,9.1325283E-1,6.026044E-1,4.5666212E-1,8.2073784E-1,1.595365E0,2.3569455E0,5.9977903E0,5.384086E0,1.0781622E-1,0E0,5.0603027E0,2.897007E0,2.1591387E0,2.0984383E0,6.03189E-1,6.646813E-1,7.5032806E-1,5.5064476E-1,3.54002E-1,1.7644098E-1,1.4773437E-1,0E0,7.3064935E-1,3.902881E-1,5.8463526E-1,1.1802753E0,0E0,0E0,0E0,3.4185266E0,3.2669442E0,1.7492402E0,0E0,0E0,2.467421E0,1.8683357E0,1.5250626E0,2.2596416E0,1.2001257E0,1.9803716E0,2.149994E0,3.3328712E-1,8.779831E-1,1.585958E-1,2.1823375E-1,3.5243243E-1,4.562459E-1,2.7132225E-1,1.3844365E-1,2.2174504E-1,1.1595838E-1,1.2695193E-1,0E0,0E0,0E0,0E0,2.3789167E-1,2.1884044E-1,0E0,0E0,1.981039E-1,4.727745E-1,4.1761756E-1,3.288352E-1,8.185698E-1,2.9053335E0,2.638853E0,1.3061304E0,0E0,7.877954E-1,1.2226553E0,0E0,9.1158867E-1,7.181251E-1,1.0242395E0,1.2758327E0,3.7052727E-1,4.1695678E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,-1,-1,-1,82,84,86,-1,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,-1,-1,-1,-1,124,126,-1,-1,128,130,132,134,136,138,140,142,-1,144,146,-1,148,150,152,154,156,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,3.3633875E6,2.14099E5,6.860185E2,4.97E2,2.1328075E-1,2.720437E6,5.140371E-2,1.2690893E0,1.5896305E-3,4.2E1,1.9E1,6E1,1.2974394E6,6.7033327E-1,5.4786605E-1,1.187875E6,5.9572783E0,1E0,1.3664E4,3.5E2,5.5E1,2.04E5,1.0052E4,4.0728608E3,2.7312504E-3,4.247868E3,5.964E3,7.5569354E2,6.08767E3,9.6910744E7,1.1297775E2,4.12204E5,2.1592189E5,2.3125623E5,5.124411E2,5.5325594E9,8.208487E-3,6.5346925E6,1.0881593E10,4.1E1,1.4492002E0,-5.4248464E-3,-4.8912223E-2,3.7033524E-2,7.805608E4,4.719849E7,7.969174E7,-1.134739E-2,1.1789459E-4,4.1822004E7,3.6604617E-2,1.9162654E7,2.3977574E7,2.5943396E0,7.6499896E9,3.6282136E0,1.3E1,6.494E3,3.757042E6,2.4137697E0,1.2103871E0,1E0,7.115101E2,2.0728285E6,3.673719E8,6.822511E5,5.4379158E1,-1.190015E-2,-2.1228474E-3,-5.493766E-3,-1.8740091E-2,2.5482938E0,3.6482175E5,2.2445519E-2,9.684837E-5,1.901E3,9.25E0,1.021793E6,9E0,6.9436204E-1,1.172E3,7.44393E5,3.1998687E3,3.9571773E-2,1E0,9.070543E0,-2.2304924E-2,1.417988E6,1.775894E10,1.8114872E7,1E0,1.6140062E3,6.72E2,-1.4159009E-2,-2.5497314E-2,2.1292619E-5,-1.6351983E-2,-2.8597225E-2,-1.1688683E-2,-5.369946E-3,1.597509E-2,-1.48337865E-2,-1.2660619E-3,-2.536114E-2,-6.3517992E-3,1.5126711E-3,1.3727201E-2,-1.1317879E-2,-1.5929495E-3,-1.0287399E-2,5.0801346E-3,-2.8260436E-2,-1.1355379E-2,-4.698423E-3,-1.2944702E-2,-2.591937E-3,5.3279162E-3,2.3805564E-3,-4.0496425E-3,7.998828E-3,-1.9024961E-3,1.8258393E-2,3.9733727E-3,-2.4188573E-4,7.5469175E-3,-6.9455393E-3,-1.6192617E-2,-4.960649E-3,-1.1414312E-2,1.993946E-2,3.3537608E-3,2.741936E-5,-8.691419E-3,8.416972E-3,-1.1298114E-2,3.9031713E-3,1.14904875E-2,-6.5663285E-3,1.5550986E-3,-2.3483902E-2,-3.9948937E-3,1.3105262E-2,-1.036583E-2,1.1249707E-2,4.1080574E-3,3.495401E-2,2.103797E-2,1.3737437E-2,-5.7545435E-3,3.8826056E-3,1.9912638E-2,-2.686017E-2,7.368813E-3,2.488727E-2,3.5861753E-2,4.460515E-3,1.8749028E-2],"split_indices":[20,102,50,2,52,2,42,32,0,42,38,6,3,3,28,27,39,1,53,89,2,2,0,5,2,4,0,52,2,4,4,45,58,29,28,28,4,5,0,45,12,2,53,0,0,0,28,45,7,0,0,51,27,47,47,56,5,53,3,2,29,35,39,89,55,32,7,32,58,0,0,0,0,42,28,0,0,9,58,5,58,53,2,1,4,0,84,53,0,29,5,12,8,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.601E3,8.31E2,1.77E3,4.54E2,3.77E2,1.202E3,5.68E2,4.49E2,5E0,2.91E2,8.6E1,2.48E2,9.54E2,3.5E1,5.33E2,3.26E2,1.23E2,2.24E2,6.7E1,1.6E1,7E1,2.37E2,1.1E1,5.18E2,4.36E2,2.8E1,7E0,3.46E2,1.87E2,1.65E2,1.61E2,6.3E1,6E1,1.67E2,5.7E1,5E1,1.7E1,1.2E1,4E0,5.1E1,1.9E1,1.54E2,8.3E1,6E0,5E0,1.4E1,5.04E2,4.03E2,3.3E1,2.6E1,2E0,2.6E2,8.6E1,1.03E2,8.4E1,9.8E1,6.7E1,1.56E2,5E0,5.1E1,1.2E1,2.6E1,3.4E1,1.54E2,1.3E1,2.2E1,3.5E1,2.4E1,2.6E1,8E0,9E0,1E1,2E0,1.9E1,3.2E1,1.7E1,2E0,4.1E1,1.13E2,3.6E1,4.7E1,3.4E1,4.7E2,3.66E2,3.7E1,5E0,2.8E1,2.55E2,5E0,6.3E1,2.3E1,8.7E1,1.6E1,6.4E1,2E1,5.8E1,4E1,3.6E1,3.1E1,1.34E2,2.2E1,3E0,2E0,3.5E1,1.6E1,1E1,2E0,2.3E1,3E0,1.7E1,1.7E1,1.5E2,4E0,7E0,6E0,1E1,1.2E1,2.4E1,1.1E1,1.1E1,1.3E1,2.3E1,3E0,1.5E1,4E0,1.6E1,1.6E1,9E0,3.2E1,6.9E1,4.4E1,3E0,3.3E1,1.4E1,3.3E1,5E0,2.9E1,2.6E2,2.1E2,1.9E2,1.76E2,2.5E1,1.2E1,2.5E1,3E0,1.72E2,8.3E1,2E1,4.3E1,1.8E1,5E0,1.1E1,7.6E1,2E0,1.4E1,2.4E1,4E1,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[5.063741E-3,-1.9028682E-1,9.715479E-2,-3.0770007E-1,-5.123238E-2,4.0889755E-2,3.3335114E-1,-3.1787968E-1,4.5255132E-2,-1.3289656E-1,1.9678284E-1,-5.3676285E-2,1.7596121E-1,3.9855924E-1,6.682498E-2,-3.740894E-1,-1.106596E-1,-1.7535107E-1,1.2653947E-2,1.10195026E-1,3.1447336E-1,-2.3090102E-1,-2.430489E-2,1.1812779E-1,3.188687E-1,6.002395E-1,3.2356128E-1,-4.1215703E-1,1.2658738E-1,-3.4826675E-1,-7.585101E-1,-5.296035E-3,-2.6857254E-1,-2.0605093E-1,-2.4375435E-2,8.423543E-2,-5.331391E-2,1.6784361E-1,-3.871292E-2,3.5326666E-1,3.619664E-3,-2.5422713E-1,1.9196434E-1,-6.651609E-2,4.4299565E-2,1.3478264E-1,-1.6999839E-2,7.8164E-1,2.828434E-1,7.7163047E-1,4.3929014E-1,2.104338E-1,4.5997095E-1,-2.78277E-2,-9.319314E-3,1.066173E-2,3.023679E-1,-3.9859042E-1,-2.1125762E-1,-4.055045E-2,-3.821295E-1,-4.61906E-2,2.2123571E-2,-2.0258436E-1,-4.4627866E-1,-2.012045E-2,-1.8541807E-1,3.0427357E-2,-1.4053719E-1,-4.5638094E-3,1.1969819E-1,5.1831612E-3,-3.6822548E-3,1.3388498E-2,5.3571593E-2,-7.18669E-3,6.790842E-3,4.4458872E-1,1.8452723E-1,-2.006734E-1,-3.902082E-1,1.9596996E-2,-3.5087622E-3,-1.0742121E-1,3.517041E-2,8.781587E-2,-3.072172E-2,-1.3778908E-2,1.8262824E-1,4.23824E-2,8.457966E-3,3.3276123E-1,6.809356E-2,7.9821265E-1,1.0632489E-2,1.527415E-2,3.227701E-2,1.7764118E-1,3.6677328E-1,7.369334E-1,3.6377224E-1,1.069791E-2,-4.285165E-2,3.9866355E-1,-1.6254738E-1,-1.6244465E-2,-2.8825775E-2,-8.6132187E-4,-1.3834382E-2,-2.6657287E-2,-1.0577224E-3,4.0802546E-3,-6.588688E-3,-1.2281722E-2,8.9785364E-4,-2.6249254E-2,-1.2437388E-3,-8.28177E-3,-2.0402577E-2,-2.790003E-3,8.274651E-3,1.8810356E-3,-9.501936E-3,5.973803E-4,9.019937E-3,5.160228E-3,-2.6201033E-3,1.2511615E-2,2.5415035E-2,-1.7566555E-3,1.1886012E-2,-1.2323977E-2,-4.3983194E-3,-1.348573E-2,-3.8979713E-2,1.1513348E-3,-7.781715E-3,8.143433E-3,-2.5627918E-3,2.4361168E-3,1.2971507E-2,-8.105712E-3,1.1669343E-3,-4.2938446E-3,6.8412223E-3,8.046124E-3,2.4497425E-2,1.7818317E-2,6.7237653E-3,-8.30653E-3,6.2499577E-3,3.1132214E-2,4.6746843E-2,-2.670691E-3,9.273692E-3,1.9320628E-2,1.7612483E-3,2.023467E-2,4.740859E-2,1.9158298E-2,-9.3394555E-3,9.158682E-3,-4.991569E-3,2.4183936E-2,5.453911E-3,-2.1601168E-2,8.627033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,101,103,105,107,-1,109,111,-1,113,115,-1,117,119,121,-1,123,-1,-1,-1,125,-1,-1,127,129,131,133,-1,-1,135,137,139,141,143,145,-1,-1,147,149,151,-1,-1,-1,153,155,157,159,-1,161,163,165,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.817782E1,1.39926605E1,2.4173431E1,6.229927E0,8.006211E0,1.879878E1,6.0562515E0,5.339878E0,0E0,1.8374634E0,9.6402216E-1,4.50221E0,4.9708176E0,4.05608E0,2.0559316E0,3.3392525E0,1.6624049E0,1.0610118E0,3.255922E-1,5.0438887E-1,3.5326242E-1,1.2620511E0,2.1574786E0,3.4596329E0,2.6797562E0,1.7299252E0,3.0975628E0,1.8091524E-1,1.280557E0,2.273819E0,4.0694904E-1,1.1869437E0,3.786509E-1,7.657747E-1,2.5891572E-1,2.2150809E-1,1.4542383E-1,5.2121973E-1,3.6298576E-1,4.5729828E-1,0E0,7.7372456E-1,4.3331096E-1,1.9213073E0,9.298584E-1,2.9807754E0,0E0,5.940223E-1,1.7246752E0,3.3413315E-1,9.4307184E-1,5.2442646E-1,2.2097626E0,0E0,0E0,4.5352525E-1,1.1840343E0,2.8025627E0,1.4049277E0,0E0,3.6467153E-1,6.961445E-1,0E0,3.7442434E-1,4.137684E-1,0E0,4.551053E-1,3.635608E-1,1.4307505E-1,0E0,2.037473E-1,0E0,0E0,0E0,1.3496E-1,0E0,0E0,2.0863199E-1,2.0626563E-1,5.0670767E-1,1.2435117E0,0E0,0E0,2.3856735E0,1.5999128E0,1.2035466E0,8.0543995E-1,1.2369834E0,1.4381056E0,0E0,0E0,9.4571877E-1,5.050356E-1,2.3518181E-1,0E0,0E0,0E0,3.7310076E-1,2.150178E-1,1.4113312E0,1.5170412E0,0E0,4.797084E-1,5.783272E-1,5.7559E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,60,60,61,61,63,63,64,64,66,66,67,67,68,68,70,70,74,74,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,93,93,97,97,98,98,99,99,100,100,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,102,104,106,108,-1,110,112,-1,114,116,-1,118,120,122,-1,124,-1,-1,-1,126,-1,-1,128,130,132,134,-1,-1,136,138,140,142,144,146,-1,-1,148,150,152,-1,-1,-1,154,156,158,160,-1,162,164,166,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.631E3,2.14099E5,6.860185E2,4.930349E5,3.5947604E7,1.945711E1,4.5255132E-2,1.1693485E0,2.160255E9,8.2E1,2.9892595E6,4.4365574E-4,3.0972284E1,1.2427474E8,7.218466E7,2.231E3,2.3686748E0,3.1826714E7,4.607796E-1,1.5694646E3,1.645614E6,7.5179994E-1,1.0918E4,7.06699E8,4.0889E4,4.1043E4,9.256843E3,9.357879E2,4.8083666E-1,8.230524E3,5.784E4,2.750917E-4,1.164249E7,1.8053533E8,7.576E3,6.4512783E-1,5.794E3,2.9722316E0,3.619664E-3,1.4504054E8,1.92E2,1.1968015E8,1.07E2,1E0,-1.6999839E-2,2E1,2E0,1.8222668E2,4.391553E6,1.0665628E6,1.417988E6,-2.78277E-2,-9.319314E-3,3.9382784E9,2.3580047E10,3.0161016E7,3.653602E7,-4.055045E-2,2.6E1,2.8091298E7,2.2123571E-2,9.408768E1,9.367855E-1,-2.012045E-2,1E0,5.810811E-1,1.0503613E5,-4.5638094E-3,1.8991614E5,5.1831612E-3,-3.6822548E-3,1.3388498E-2,2.8216E4,-7.18669E-3,6.790842E-3,1.948181E0,7.8377126E8,5.222222E1,3.5612745E0,1.9596996E-2,-3.5087622E-3,1E0,2.558106E10,7.298614E2,1.5676449E3,1.405E3,7.064848E6,4.23824E-2,8.457966E-3,2E0,7.697189E2,2.6295085E3,1.0632489E-2,1.527415E-2,3.227701E-2,1.260173E6,3.3517068E7,6.162955E0,1.5005797E4,1.069791E-2,3.7448005E3,1.0008265E1,9.627E3,-1.6244465E-2,-2.8825775E-2,-8.6132187E-4,-1.3834382E-2,-2.6657287E-2,-1.0577224E-3,4.0802546E-3,-6.588688E-3,-1.2281722E-2,8.9785364E-4,-2.6249254E-2,-1.2437388E-3,-8.28177E-3,-2.0402577E-2,-2.790003E-3,8.274651E-3,1.8810356E-3,-9.501936E-3,5.973803E-4,9.019937E-3,5.160228E-3,-2.6201033E-3,1.2511615E-2,2.5415035E-2,-1.7566555E-3,1.1886012E-2,-1.2323977E-2,-4.3983194E-3,-1.348573E-2,-3.8979713E-2,1.1513348E-3,-7.781715E-3,8.143433E-3,-2.5627918E-3,2.4361168E-3,1.2971507E-2,-8.105712E-3,1.1669343E-3,-4.2938446E-3,6.8412223E-3,8.046124E-3,2.4497425E-2,1.7818317E-2,6.7237653E-3,-8.30653E-3,6.2499577E-3,3.1132214E-2,4.6746843E-2,-2.670691E-3,9.273692E-3,1.9320628E-2,1.7612483E-3,2.023467E-2,4.740859E-2,1.9158298E-2,-9.3394555E-3,9.158682E-3,-4.991569E-3,2.4183936E-2,5.453911E-3,-2.1601168E-2,8.627033E-3],"split_indices":[20,102,2,2,52,28,45,56,0,42,12,29,51,57,56,45,45,0,53,45,38,52,9,27,9,7,2,10,4,55,42,52,10,39,45,7,9,57,2,57,0,31,0,7,3,102,0,3,6,58,28,28,29,0,0,5,5,45,45,0,8,45,0,56,27,0,77,57,33,0,28,0,0,0,9,0,0,42,7,4,53,0,0,8,19,52,55,10,28,0,0,17,4,4,0,0,0,50,51,53,4,0,4,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.676E3,8.57E2,1.819E3,4.64E2,3.93E2,1.47E3,3.49E2,4.61E2,3E0,2.96E2,9.7E1,8.65E2,6.05E2,2.8E2,6.9E1,3.62E2,9.9E1,2.29E2,6.7E1,5.7E1,4E1,1.22E2,7.43E2,4.32E2,1.73E2,7.4E1,2.06E2,7E0,6.2E1,3.41E2,2.1E1,6E1,3.9E1,1.9E2,3.9E1,3.2E1,3.5E1,4.1E1,1.6E1,3.4E1,6E0,1.16E2,6E0,4.6E2,2.83E2,4.18E2,1.4E1,1.1E1,1.62E2,3.4E1,4E1,1.14E2,9.2E1,3E0,4E0,3.8E1,2.4E1,2.48E2,9.3E1,1.6E1,5E0,5.6E1,4E0,3E1,9E0,1.5E1,1.75E2,2.7E1,1.2E1,5E0,2.7E1,4E0,3.1E1,2E1,2.1E1,1E1,6E0,2.1E1,1.3E1,8.5E1,3.1E1,3E0,3E0,3.28E2,1.32E2,1.79E2,1.04E2,1.02E2,3.16E2,9E0,2E0,1.31E2,3.1E1,3.2E1,2E0,2.8E1,1.2E1,9.6E1,1.8E1,2.2E1,7E1,7E0,3.1E1,2E1,4E0,1.94E2,5.4E1,2.7E1,6.6E1,3E0,2E0,2.3E1,3.3E1,2.4E1,6E0,7E0,2E0,1.68E2,7E0,1.7E1,1E1,3E0,9E0,1.1E1,1.6E1,1.4E1,7E0,8E0,1.3E1,3E0,1E1,5.5E1,3E1,2.6E1,5E0,9.7E1,2.31E2,5.2E1,8E1,1.5E2,2.9E1,2.9E1,7.5E1,6.9E1,3.3E1,3.04E2,1.2E1,1.08E2,2.3E1,6E0,2.5E1,2E1,1.2E1,6E0,9E1,1.6E1,2E0,1.1E1,1.1E1,6.6E1,4E0,6E0,2.5E1,1.4E1,6E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"167","size_leaf_vector":"1"}},{"base_weights":[2.534354E-4,-1.817672E-1,8.404214E-2,-3.109616E-1,-7.187962E-2,2.8869577E-2,2.9664963E-1,-2.77736E-1,-6.1055225E-1,-1.19388856E-1,2.5864786E-1,-4.3913096E-2,1.7333625E-1,3.5595453E-1,6.323017E-2,-3.7522808E-1,-1.9453834E-1,-8.328086E-1,-3.4446386E-1,-4.1534215E-2,-3.6376137E-1,4.559558E-2,1.7526606E-1,-2.115718E-2,-2.6970044E-1,1.8964878E-1,-2.6495045E-1,5.084102E-1,2.9189843E-1,-4.2525625E-1,1.2513228E-1,-7.0113844E-1,-3.442746E-1,-7.879981E-2,-3.2212946E-1,-4.2002924E-2,-1.387348E-2,-1.9726798E-1,-2.7231444E-2,-1.0362253E-1,1.2652984E-1,-3.909306E-1,-1.2637248E-2,2.468904E-1,-1.5740766E-1,-2.0026104E-1,9.947142E-3,-1.7888477E-1,-7.868624E-1,4.2348534E-2,2.5271308E-1,-3.225862E-1,6.2620286E-3,5.876966E-1,3.0751517E-1,1.997716E-1,4.2258406E-1,-2.8320055E-2,-8.338294E-3,6.199665E-3,3.38963E-1,-7.1059936E-3,-4.0808868E-2,-4.1303667E-1,-2.1193138E-1,-1.1484453E-1,1.7680207E-1,-3.7580168E-1,-9.896091E-2,-1.2855413E-2,-1.4231578E-3,-4.66029E-2,-2.185731E-1,2.683018E-1,3.231407E-2,-5.757786E-1,-3.4318265E-1,1.025269E-2,-5.9332848E-3,1.772871E-2,1.4728934E-1,8.637621E-4,-2.7737623E-1,-2.3111896E-1,-5.3056203E-2,5.7410216E-1,-6.975689E-3,-2.7371916E-1,-4.4144414E-2,-4.3356832E-2,-4.319082E-3,-2.7111015E-1,7.0523955E-2,2.8781077E-1,3.4509905E-2,-2.0273535E-2,-8.089927E-3,5.1899564E-1,4.4758625E-2,5.9207836E-3,3.9258856E-1,1.391346E-1,3.4514546E-1,1.4747322E-1,4.6830365E-1,-9.775152E-2,1.0556123E-1,2.3733936E-1,2.9750304E-2,-2.0808822E-2,-3.5871996E-3,1.1896432E-3,-1.2814185E-2,-4.351967E-3,-1.672296E-2,1.2292709E-2,-1.1229675E-4,-8.200372E-3,-2.1054527E-2,-1.03594195E-2,1.3547657E-3,-7.110817E-4,-1.1612182E-2,-5.4985033E-3,-1.5030434E-2,2.3009578E-2,7.4311094E-3,9.579997E-3,-1.1696282E-3,-1.2166969E-2,-3.0006858E-2,-1.4278742E-2,-2.8808668E-2,-6.5609803E-3,9.8274E-3,-4.3682167E-3,-2.0743106E-2,-1.5474554E-2,-9.761322E-3,2.0288483E-3,-1.0382548E-2,-1.7220875E-2,3.28266E-2,4.0469943E-3,-2.303593E-3,-1.4086619E-2,4.3822938E-4,-4.760197E-3,5.4710703E-3,4.2146565E-3,-2.587018E-2,-2.8679082E-3,5.6604687E-3,9.774477E-3,1.842159E-2,-1.6164748E-3,8.7346975E-3,2.6857993E-2,1.3355934E-2,2.0775834E-2,6.6963662E-3,2.833423E-3,8.859438E-3,1.2243242E-2,3.05354E-2,1.1947385E-2,-1.4830871E-2,1.9344917E-2,3.1805784E-2,-9.738684E-3,4.267631E-3,3.2484306E-3,1.6621247E-2,1.4479285E-2,-4.705413E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,95,-1,97,99,101,103,-1,-1,105,107,-1,-1,109,111,113,115,117,119,-1,-1,121,123,125,127,129,131,-1,-1,-1,133,-1,135,137,139,141,143,145,147,-1,-1,149,151,153,155,-1,-1,157,-1,-1,159,161,163,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0980194E1,1.200239E1,2.157482E1,3.7025604E0,7.2386065E0,1.538006E1,5.223957E0,2.7821178E0,1.9604416E0,7.633156E0,3.1866672E0,4.9935656E0,3.5535326E0,2.8008423E0,2.4118037E0,1.3827915E0,2.8099203E0,2.2660732E-1,5.1472807E-1,3.2061648E0,9.204178E-1,0E0,1.3017219E0,4.932243E0,4.0392523E0,4.3831787E0,4.527049E-1,1.2209587E0,2.5041046E0,2.7246082E-1,1.7781001E0,9.585514E-1,1.2683907E0,9.600407E-1,1.052804E0,0E0,0E0,1.4703268E-1,0E0,1.458008E0,1.0999739E0,5.967531E-1,2.2831708E-1,4.8583865E-1,2.154803E-1,5.827966E-1,7.2120647E0,9.718244E-1,9.913044E-1,1.2820715E0,2.5250053E0,1.7617643E-1,0E0,9.925785E-1,3.7384105E-1,1.0803032E0,1.0377979E0,0E0,0E0,4.8544133E-1,5.6964755E-1,0E0,0E0,6.716881E-1,7.048819E-1,4.7385192E-1,1.8663853E-1,8.780365E-1,2.8985032E-1,0E0,0E0,9.3559504E-1,6.955166E-1,7.1752477E-1,4.9021372E-1,1.1861515E-1,6.8156815E-1,0E0,0E0,0E0,4.5620066E-1,0E0,1.3447323E-1,2.05338E-1,3.8416702E-1,2.53371E0,2.7655163E0,2.6265717E-1,3.001476E-1,0E0,0E0,1.2071288E0,8.277011E-1,2.2320747E0,4.806082E-1,0E0,0E0,3.8589573E-1,0E0,0E0,1.2692833E-1,3.2645E-1,8.054304E-1,7.3007315E-1,6.7705727E-1,4.7198218E-1,1.9515228E-1,4.6598947E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,59,59,60,60,63,63,64,64,65,65,66,66,67,67,68,68,71,71,72,72,73,73,74,74,75,75,76,76,80,80,82,82,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,97,97,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,96,-1,98,100,102,104,-1,-1,106,108,-1,-1,110,112,114,116,118,120,-1,-1,122,124,126,128,130,132,-1,-1,-1,134,-1,136,138,140,142,144,146,148,-1,-1,150,152,154,156,-1,-1,158,-1,-1,160,162,164,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.382114E0,8.32E3,6.747114E7,2.9283184E3,7.733016E5,3.5301748E7,1.146E3,6.567044E2,4.0844156E7,6.737509E7,1E0,4.898012E-1,1.1317E4,3.2343243E1,4.39688E-5,7.44E2,2.7504245E10,1.0828989E8,6.860185E2,9.408768E1,4.559558E-2,1.1855755E8,2.73E2,9.457831E0,8.31E2,1E0,6.4E1,2.1019447E-1,4.1043E4,9.256843E3,1.9523809E0,2.765625E0,2.695E3,6.057963E2,-4.2002924E-2,-1.387348E-2,9.6910744E7,-2.7231444E-2,1.6256282E7,9.397667E6,3.131E3,1.1349584E2,8.7643677E-1,2.7523365E0,1.8108038E5,8E0,4.2E1,1.752E3,1.198E5,1E0,1.6709303E1,6.2620286E-3,1.5548611E7,9.433E3,1.8777761E0,5.673198E5,-2.8320055E-2,-8.338294E-3,1E0,2.4507338E4,-7.1059936E-3,-4.0808868E-2,3.86569E5,5.2E1,1.2342778E1,5.5814706E5,1.1978022E0,2.3E1,-1.2855413E-2,-1.4231578E-3,1.1231086E1,1.9563605E5,4.9716983E0,1.1317E4,5.5759512E-2,6.2105E4,1.025269E-2,-5.9332848E-3,1.772871E-2,5.505334E9,8.637621E-4,2.109E3,1.0972222E0,3.1E1,1.0005E5,5.372E3,1E0,1.8796511E5,-4.3356832E-2,-4.319082E-3,1.08504E5,5.79E2,4.7646948E2,1.4521204E0,-2.0273535E-2,-8.089927E-3,4.728721E0,4.4758625E-2,5.9207836E-3,2.79495E-1,1.2204E4,1.6195753E9,1.314806E8,7.983011E7,2.3274304E2,2.6312E4,1.5988282E1,2.9750304E-2,-2.0808822E-2,-3.5871996E-3,1.1896432E-3,-1.2814185E-2,-4.351967E-3,-1.672296E-2,1.2292709E-2,-1.1229675E-4,-8.200372E-3,-2.1054527E-2,-1.03594195E-2,1.3547657E-3,-7.110817E-4,-1.1612182E-2,-5.4985033E-3,-1.5030434E-2,2.3009578E-2,7.4311094E-3,9.579997E-3,-1.1696282E-3,-1.2166969E-2,-3.0006858E-2,-1.4278742E-2,-2.8808668E-2,-6.5609803E-3,9.8274E-3,-4.3682167E-3,-2.0743106E-2,-1.5474554E-2,-9.761322E-3,2.0288483E-3,-1.0382548E-2,-1.7220875E-2,3.28266E-2,4.0469943E-3,-2.303593E-3,-1.4086619E-2,4.3822938E-4,-4.760197E-3,5.4710703E-3,4.2146565E-3,-2.587018E-2,-2.8679082E-3,5.6604687E-3,9.774477E-3,1.842159E-2,-1.6164748E-3,8.7346975E-3,2.6857993E-2,1.3355934E-2,2.0775834E-2,6.6963662E-3,2.833423E-3,8.859438E-3,1.2243242E-2,3.05354E-2,1.1947385E-2,-1.4830871E-2,1.9344917E-2,3.1805784E-2,-9.738684E-3,4.267631E-3,3.2484306E-3,1.6621247E-2,1.4479285E-2,-4.705413E-3],"split_indices":[20,56,2,45,52,28,45,2,52,45,7,6,27,32,56,42,0,5,45,52,56,0,45,2,54,2,89,11,35,10,4,54,54,2,52,0,0,45,0,45,45,2,56,57,53,28,18,6,29,7,6,58,0,45,2,41,32,0,0,102,4,0,0,1,11,54,28,53,8,0,0,54,28,54,9,27,10,0,0,0,5,0,0,53,10,5,29,68,33,0,0,7,29,52,38,0,0,38,0,0,38,2,7,1,44,56,2,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.685E3,8.46E2,1.839E3,3.88E2,4.58E2,1.461E3,3.78E2,3.51E2,3.7E1,4.01E2,5.7E1,9.72E2,4.89E2,3.01E2,7.7E1,1.6E2,1.91E2,1.9E1,1.8E1,3.05E2,9.6E1,5E0,5.2E1,8.84E2,8.8E1,4.72E2,1.7E1,8.7E1,2.14E2,8E0,6.9E1,1.2E1,1.48E2,1.01E2,9E1,1.7E1,2E0,1.2E1,6E0,2.23E2,8.2E1,8.9E1,7E0,4.3E1,9E0,1.3E2,7.54E2,7.6E1,1.2E1,1.42E2,3.3E2,1.5E1,2E0,6.1E1,2.6E1,1.27E2,8.7E1,4E0,4E0,4.5E1,2.4E1,3E0,9E0,9.6E1,5.2E1,8.9E1,1.2E1,7.2E1,1.8E1,8E0,4E0,1.5E2,7.3E1,3.2E1,5E1,1.6E1,7.3E1,2E0,5E0,1.8E1,2.5E1,4E0,5E0,1.07E2,2.3E1,2.1E1,7.33E2,4.4E1,3.2E1,1E1,2E0,1.1E1,1.31E2,2.84E2,4.6E1,8E0,7E0,5.3E1,8E0,9E0,1.7E1,9.1E1,3.6E1,1.3E1,7.4E1,2.2E1,2.3E1,1.9E1,5E0,9E1,6E0,1E1,4.2E1,8.2E1,7E0,8E0,4E0,1.8E1,5.4E1,9E0,9E0,1.3E2,2E1,3.6E1,3.7E1,1E1,2.2E1,1.2E1,3.8E1,3E0,1.3E1,6.4E1,9E0,4E0,2.1E1,3E0,2E0,2.2E1,8.5E1,1.5E1,8E0,2E0,1.9E1,2.27E2,5.06E2,4.1E1,3E0,2.4E1,8E0,5E0,6E0,3.5E1,9.6E1,1.54E2,1.3E2,3.2E1,1.4E1,4.4E1,9E0,1.4E1,3E0,3.4E1,5.7E1,2.9E1,7E0,1.1E1,2E0,5.8E1,1.6E1,1.4E1,8E0,2.1E1,2E0,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-1.537491E-3,6.8239585E-2,-2.3639442E-1,-5.7615686E-2,1.6996337E-1,-3.746522E-1,-4.2344075E-2,-1.1156281E-1,8.400984E-2,4.9860086E-2,2.770806E-1,-5.7519734E-1,-3.3568943E-1,-8.887326E-2,1.8491934E-1,-1.4450718E-1,6.343844E-2,-2.271651E-1,1.070941E-1,9.3518734E-2,-2.1849242E-1,2.4747835E-1,5.323474E-1,-7.144298E-1,-4.741331E-1,-7.5807554E-1,-3.095004E-1,-5.172515E-2,-3.6764398E-1,2.5746548E-1,-2.3531961E-1,-1.1564013E-1,-3.2595497E-1,1.7792185E-1,-1.9342387E-2,6.2915278E-3,-4.1738197E-1,6.572452E-2,2.2413751E-1,1.977243E-1,-1.8016933E-2,-2.8789404E-1,8.293377E-2,2.0863147E-1,4.3856636E-1,5.845373E-1,1.069984E-1,-4.1911047E-2,-1.8440304E-2,-4.8672482E-3,-4.9305567E-1,-4.5865297E-2,-4.1377163E-1,-1.7508686E-1,-3.642573E-1,-1.0363793E-1,5.209747E-2,-4.9183795E-1,-4.225204E-2,2.875755E-1,-3.3697519E-3,-2.8361029E-3,-1.6885666E-2,-1.7697051E-1,-4.5807354E-2,-7.9863715E-1,-2.2640385E-1,2.1616499E-1,-5.1092533E-3,-1.2240728E-1,4.7220808E-2,-7.829505E-3,-5.2389324E-1,-3.9004873E-2,1.2343719E-1,8.13886E-2,2.9017538E-1,1.0389114E-1,3.5968548E-1,-1.702875E-1,4.0587112E-2,-1.8415479E-2,-1.7885107E-1,1.1892553E-2,-2.6301926E-3,9.0585425E-2,2.6206723E-1,6.4369905E-1,3.3223495E-1,9.816511E-3,6.268307E-1,-4.348767E-3,1.1677751E-2,-2.4563512E-2,-5.5998266E-3,-3.7726574E-3,-2.4534665E-2,-3.0466324E-1,-9.1103695E-2,-7.287307E-1,-3.20442E-1,-2.2872247E-1,-7.812004E-2,2.0821896E-1,-8.802197E-3,-6.0429436E-1,-1.0957017E-2,-6.612545E-3,4.2187353E-3,2.0904373E-1,4.0439516E-1,-7.2786696E-3,-1.3235975E-2,2.9199202E-2,-2.878566E-3,8.715093E-5,-4.5224957E-2,-6.868306E-4,-1.3616994E-2,4.2539425E-3,1.3841089E-2,-1.3838199E-3,-1.1436041E-2,-6.411396E-4,6.948284E-3,-2.9958751E-2,-6.9879084E-3,8.202395E-3,-4.112801E-3,7.0046107E-3,-1.511462E-2,-3.0721114E-3,8.423202E-3,1.6643105E-2,4.6436493E-3,1.3614329E-2,1.0534972E-3,3.3336535E-2,1.2306028E-2,-9.242589E-3,1.6674273E-2,5.562477E-4,1.0874078E-2,-1.1041956E-2,1.5036275E-3,5.1610977E-3,-1.5311847E-2,8.601215E-3,1.53613705E-2,3.4646858E-2,1.8064663E-2,1.2750096E-2,4.328682E-2,3.0667381E-2,9.858946E-3,-8.018865E-3,-1.7963847E-2,8.80625E-3,-5.918266E-3,-1.0597752E-2,-3.83608E-2,-1.3183085E-2,-2.552682E-2,-1.2210845E-2,6.5985403E-4,-9.302131E-4,-6.1906786E-3,1.3772639E-2,3.4989864E-3,-6.5293247E-3,1.6260952E-3,-3.294881E-2,-5.930562E-3,1.2114551E-3,1.2388291E-2,2.4783622E-2,8.916356E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,-1,-1,-1,93,-1,95,97,99,101,103,105,107,109,-1,-1,-1,111,113,115,117,119,-1,121,123,-1,125,127,129,131,133,135,137,139,141,-1,143,-1,-1,145,147,149,151,-1,153,-1,-1,-1,-1,-1,-1,155,157,159,161,163,165,167,169,171,-1,-1,-1,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4000534E1,2.6510914E1,1.6483955E1,7.08428E0,1.4716763E1,2.6120605E0,2.7340508E0,3.8776875E0,1.8632387E0,6.358739E0,4.4399185E0,4.6333885E-1,3.1171265E0,2.197126E0,1.4020021E0,2.9216986E0,1.0174838E0,1.2824328E0,1.1425157E0,5.4197116E0,1.6137555E0,3.9467087E0,1.3137875E0,8.2760334E-1,2.0364094E-1,7.7222633E-1,2.048563E0,1.0294461E0,9.895117E-1,3.9533067E-1,1.2367651E-1,2.0862179E0,3.442501E0,5.128554E-1,4.3798086E-1,0E0,2.3886156E-1,1.0806847E0,5.608468E-1,3.6251879E0,2.0236614E0,5.8456755E-1,3.556436E-1,2.8400383E0,1.7710361E0,7.544193E-1,2.3868991E-1,0E0,0E0,0E0,1.9392204E-1,0E0,2.237134E-1,8.972342E-1,3.0082932E0,3.8750255E-1,6.1032856E-1,3.9420176E-1,1.120836E-1,2.3655581E-1,0E0,0E0,0E0,6.037655E-1,2.1926115E0,1.5178947E0,7.9445815E-1,3.4539318E-1,0E0,2.6519156E-1,2.3327088E-1,0E0,2.1616602E-1,6.4553595E-1,1.1986372E0,3.0008873E-1,4.31679E-1,2.2687297E0,2.8622036E0,8.08862E-1,8.8338053E-1,0E0,3.443048E-1,0E0,0E0,1.0426731E0,1.4525166E0,3.6476898E-1,2.098517E0,0E0,1.4732933E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.5616264E-1,4.9602896E-1,5.9296894E-1,1.6103668E0,1.3719988E-1,3.187195E-1,1.7207384E-1,2.609213E-1,3.9741993E-1,0E0,0E0,0E0,2.0322573E-1,2.2451806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,63,63,64,64,65,65,66,66,67,67,69,69,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,82,82,85,85,86,86,87,87,88,88,90,90,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,-1,-1,-1,94,-1,96,98,100,102,104,106,108,110,-1,-1,-1,112,114,116,118,120,-1,122,124,-1,126,128,130,132,134,136,138,140,142,-1,144,-1,-1,146,148,150,152,-1,154,-1,-1,-1,-1,-1,-1,156,158,160,162,164,166,168,170,172,-1,-1,-1,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7810526E2,1E0,7.733016E5,1.4887473E-1,1.5804776E4,3.0497742E3,7.997723E6,5.8631687E1,5.7581736E7,4.391553E6,1.381869E2,6E0,8.672049E9,1.1855755E8,9.669789E0,2.25E2,3.0039525E-1,1E0,1.0238709E1,4.6726016E3,3.970405E3,4.728721E0,4.6547272E2,1.8108038E5,1.7997802E0,1.2E1,6.9664386E2,1.2151E4,1.2428079E1,2.603E3,1.4735735E2,2E0,2.0783027E5,1.9987492E5,6.2915278E-3,1.3860265E6,2.4757753E2,1E0,9.557794E2,7.44393E5,1.884E3,5.441732E9,2.2783158E5,3.2475834E7,1.752E3,8E0,-4.1911047E-2,-1.8440304E-2,-4.8672482E-3,1.8302105E6,-4.5865297E-2,2.1132307E0,3.9314518E1,1.2068E4,1.06E3,1.18827E5,8.977316E0,1E0,8.63558E-1,-3.3697519E-3,-2.8361029E-3,-1.6885666E-2,5.8036E4,2.04E5,8E0,2.4558064E2,1.5687E4,-5.1092533E-3,3.0273972E0,1.70244E7,-7.829505E-3,4.38E2,1.1779856E2,2.507E3,1.9925156E6,1.3581108E7,8E0,9.653717E3,6.071E3,1.5963264E9,-1.8415479E-2,4.6E1,1.1892553E-2,-2.6301926E-3,3.0161016E7,3.83E3,3.275343E6,4.2482185E0,9.816511E-3,2E1,-4.348767E-3,1.1677751E-2,-2.4563512E-2,-5.5998266E-3,-3.7726574E-3,-2.4534665E-2,3.512733E-1,1E0,1.7848537E0,2.72258E5,6.1E1,5.68E2,6.722251E1,5.378E3,6E1,-1.0957017E-2,-6.612545E-3,4.2187353E-3,1E0,1.0057E4,-7.2786696E-3,-1.3235975E-2,2.9199202E-2,-2.878566E-3,8.715093E-5,-4.5224957E-2,-6.868306E-4,-1.3616994E-2,4.2539425E-3,1.3841089E-2,-1.3838199E-3,-1.1436041E-2,-6.411396E-4,6.948284E-3,-2.9958751E-2,-6.9879084E-3,8.202395E-3,-4.112801E-3,7.0046107E-3,-1.511462E-2,-3.0721114E-3,8.423202E-3,1.6643105E-2,4.6436493E-3,1.3614329E-2,1.0534972E-3,3.3336535E-2,1.2306028E-2,-9.242589E-3,1.6674273E-2,5.562477E-4,1.0874078E-2,-1.1041956E-2,1.5036275E-3,5.1610977E-3,-1.5311847E-2,8.601215E-3,1.53613705E-2,3.4646858E-2,1.8064663E-2,1.2750096E-2,4.328682E-2,3.0667381E-2,9.858946E-3,-8.018865E-3,-1.7963847E-2,8.80625E-3,-5.918266E-3,-1.0597752E-2,-3.83608E-2,-1.3183085E-2,-2.552682E-2,-1.2210845E-2,6.5985403E-4,-9.302131E-4,-6.1906786E-3,1.3772639E-2,3.4989864E-3,-6.5293247E-3,1.6260952E-3,-3.294881E-2,-5.930562E-3,1.2114551E-3,1.2388291E-2,2.4783622E-2,8.916356E-3],"split_indices":[19,52,59,28,38,51,52,9,47,45,28,47,3,5,45,54,10,53,101,53,4,52,38,52,28,53,18,52,9,57,0,52,17,33,47,0,28,55,16,52,1,10,5,28,50,29,8,0,0,0,32,0,53,58,9,2,29,54,102,27,0,0,0,1,5,0,4,1,0,53,9,0,0,4,2,28,12,17,50,2,7,0,3,0,0,45,2,29,39,0,3,0,0,0,0,0,0,27,64,54,9,3,0,58,2,3,0,0,0,102,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,2.069E3,6.14E2,9.25E2,1.144E3,3.58E2,2.56E2,6.7E2,2.55E2,5.4E2,6.04E2,5.6E1,3.02E2,2.13E2,4.3E1,5.64E2,1.06E2,1.7E1,2.38E2,4.65E2,7.5E1,5.43E2,6.1E1,2.1E1,3.5E1,1.6E1,2.86E2,1.89E2,2.4E1,3.7E1,6E0,4.88E2,7.6E1,4.4E1,6.2E1,6E0,1.1E1,1.77E2,6.1E1,2.4E2,2.25E2,6.1E1,1.4E1,4.53E2,9E1,5.4E1,7E0,1.3E1,8E0,2E0,3.3E1,9E0,7E0,8.4E1,2.02E2,1.26E2,6.3E1,1.7E1,7E0,3.4E1,3E0,3E0,3E0,2.59E2,2.29E2,1.2E1,6.4E1,3.9E1,5E0,2.4E1,3.8E1,4E0,7E0,6.3E1,1.14E2,2E1,4.1E1,1.53E2,8.7E1,6.2E1,1.63E2,3.1E1,3E1,6E0,8E0,1.42E2,3.11E2,2.9E1,6.1E1,6E0,4.8E1,3E0,4E0,3.1E1,2E0,2E0,5E0,3.2E1,5.2E1,2E1,1.82E2,2E1,1.06E2,1.7E1,4.6E1,1.1E1,6E0,4E0,3E0,2.2E1,1.2E1,2.09E2,5E1,4E0,2.25E2,2E0,1E1,1.4E1,5E1,1.5E1,2.4E1,1.4E1,1E1,2.4E1,1.4E1,5E0,2E0,1.1E1,5.2E1,1.09E2,5E0,8E0,1.2E1,3.1E1,1E1,4.7E1,1.06E2,1.9E1,6.8E1,6E1,2E0,1.42E2,2.1E1,2.4E1,6E0,1.37E2,5E0,1.31E2,1.8E2,2.1E1,8E0,5.6E1,5E0,4.6E1,2E0,1.2E1,2E1,5E0,4.7E1,3E0,1.7E1,1.52E2,3E1,1.8E1,2E0,5E1,5.6E1,1E1,7E0,1.1E1,3.5E1,9E0,2E0,5E0,1.7E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[2.8231882E-3,6.418889E-2,-2.194715E-1,1.12586515E-2,2.538697E-1,-3.312235E-1,-6.4612046E-2,-5.092457E-2,1.5077528E-1,3.2521424E-1,4.441304E-2,-5.233784E-1,-2.8863823E-1,-1.3849673E-1,5.2404385E-2,-9.6434295E-2,4.1458823E-2,1.9353819E-1,9.043456E-4,4.880579E-1,2.3524341E-1,-4.0031947E-2,2.4925189E-1,-5.828715E-1,-2.1078362E-1,-3.0565947E-1,1.0846398E-1,-1.05985194E-1,-4.679638E-1,1.3483164E-1,-2.4054901E-1,-7.664969E-2,-3.2333928E-1,-1.9504818E-1,9.0468526E-2,7.910176E-2,2.7836934E-1,4.593802E-2,-2.5345665E-1,6.1077523E-1,3.002827E-1,1.24052174E-1,3.6632046E-1,-3.241094E-1,1.7355429E-2,2.9335463E-1,-7.4162004E-3,-1.1665747E-2,-6.531196E-1,-4.694509E-3,-1.5635677E-2,-2.2150363E-1,-3.8261747E-1,1.4311201E-2,-3.3620608E-3,-1.9982952E-1,-5.8931064E-2,-2.9841432E-2,-1.4036863E-2,-4.783056E-4,1.8845405E-1,-1.4084247E-1,-2.545102E-2,-1.0054427E-1,2.2611235E-1,-4.043859E-1,-1.0357394E-2,-1.3020298E-1,-3.8321268E-2,1.8889357E-1,4.6352413E-2,-3.1877365E-2,1.3537842E-1,3.2344848E-1,5.039328E-2,-7.822408E-2,1.0924132E-1,-3.2374144E-1,2.6772846E-3,4.5356223E-1,7.882047E-1,2.1442097E-2,1.9197415E-1,7.5913206E-2,2.1495621E-1,5.5645907E-1,2.883725E-1,-5.6801313E-3,-2.2192989E-2,6.804548E-2,-1.9026518E-1,1.4181435E-1,4.6723825E-1,-5.7816416E-1,-5.0868127E-2,-3.4390637E-1,-1.7536472E-1,-4.6278292E-1,-2.523147E-1,-1.2634554E-2,-1.209369E-1,-9.589752E-2,7.0617296E-2,1.9543554E-1,-1.04933806E-1,1.1726349E-1,1.3952951E-2,-2.529828E-1,8.111831E-4,-5.649903E-3,3.3903683E-3,3.207168E-2,4.865177E-3,-2.046667E-2,-6.2715844E-3,4.1759796E-3,-7.950544E-3,1.9628948E-2,6.974768E-3,-4.409762E-4,6.2632128E-3,7.926493E-3,-5.091463E-3,5.5325073E-3,2.220092E-2,1.251162E-2,2.1536443E-2,-2.4689494E-3,8.589683E-3,5.711626E-3,-5.2476726E-3,-7.939319E-3,6.731021E-3,-7.2629643E-3,-2.077717E-2,1.1725272E-2,2.385394E-2,3.9952338E-2,1.7851202E-2,-9.963025E-4,1.360835E-2,4.671612E-3,-9.921493E-3,1.3550245E-2,5.8112345E-3,1.8949054E-2,4.0799376E-2,1.4899093E-2,-1.6560933E-2,4.6695487E-4,2.0300016E-2,-5.1410017E-3,-1.77539E-2,9.432625E-3,1.29381E-3,2.457955E-2,6.1044954E-3,-2.9950356E-2,-1.1388044E-2,-1.7730225E-2,2.4639869E-3,-6.0007675E-3,-1.6081035E-2,-2.7156882E-2,-1.5220247E-2,-1.6891837E-2,-5.8301245E-3,-8.438276E-3,1.0849E-3,2.974882E-3,-6.155658E-3,6.7558414E-3,-2.9452348E-3,2.5408536E-3,1.5542958E-2,3.339106E-3,-7.394705E-3,1.3534941E-2,2.0821819E-3,-1.1062532E-3,-1.6675686E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,-1,93,-1,-1,95,97,-1,-1,99,101,-1,-1,103,105,107,-1,109,111,113,-1,115,-1,117,119,121,123,125,127,129,131,133,-1,135,137,-1,139,141,143,145,147,-1,-1,149,151,153,155,157,-1,159,161,163,165,-1,167,169,171,173,175,177,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6067963E1,2.0799715E1,9.877546E0,1.4070814E1,6.740059E0,2.5656776E0,2.090877E0,4.7212706E0,3.2042837E0,4.8226357E0,2.0165014E0,9.743767E-1,1.8851662E0,1.5357018E0,2.302284E0,3.3511138E0,4.3179584E0,3.754302E0,1.294582E0,2.5587444E0,3.1463976E0,1.3587292E0,6.51387E-1,1.0087605E0,1.1150786E-1,1.6185608E0,4.2912754E-1,5.8876765E-1,1.700561E-1,5.370077E-1,5.3423214E-1,5.0361543E0,4.3318796E-1,2.4071856E0,1.3286204E0,1.0473197E0,2.275465E0,7.6309294E-1,3.8773572E-1,1.6229916E0,6.9657516E-1,5.0880516E-1,1.3216238E0,3.4581196E-1,7.496476E-1,7.506006E-1,0E0,0E0,6.1232567E-1,0E0,0E0,6.6257477E-1,1.3038578E0,0E0,0E0,1.952765E-1,4.473548E-1,0E0,0E0,4.7072193E-1,3.540752E-1,3.050761E-1,0E0,1.9287453E0,2.7489617E0,1.5136576E-1,0E0,4.7382498E-1,0E0,8.483274E-1,1.0050641E0,8.5510725E-1,6.697409E-1,1.3430157E0,5.088664E-1,2.1739514E-1,5.7437956E-1,1.9946742E-1,0E0,2.4984074E-1,3.0109215E-1,0E0,5.9433496E-1,5.1015836E-1,2.266028E-1,9.931154E-1,1.1600943E0,0E0,0E0,1.1782517E0,1.6967851E-1,1.086348E-1,1.5341926E-1,4.29986E-1,0E0,3.7281275E-1,7.208929E-1,1.0745125E0,6.650131E-1,0E0,1.7856503E-1,3.7931722E-1,2.0413858E-1,1.2984943E-1,1.4038283E-1,3.9034855E-1,0E0,2.1085465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,51,51,52,52,55,55,56,56,59,59,60,60,61,61,63,63,64,64,65,65,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,82,82,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,95,95,96,96,97,97,98,98,100,100,101,101,102,102,103,103,104,104,105,105,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,-1,94,-1,-1,96,98,-1,-1,100,102,-1,-1,104,106,108,-1,110,112,114,-1,116,-1,118,120,122,124,126,128,130,132,134,-1,136,138,-1,140,142,144,146,148,-1,-1,150,152,154,156,158,-1,160,162,164,166,-1,168,170,172,174,176,178,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,7.431E3,1E0,7.733016E5,2.8091298E7,8.230099E4,1.0323588E3,1.5107028E-1,2E0,1.4232318E3,6.082581E3,1E0,3.970405E3,8.672049E9,9.6910744E7,4.0844156E7,5.08E2,1.06E3,4.898012E-1,8.606353E1,2.3292E4,2.308943E0,1.56E2,5.020181E-1,2.5E1,8.15E2,1.8140173E-1,5.124411E2,6.3300834E0,1E0,1.4299594E8,1.1210787E3,6.4587574E-3,4.5184364E0,8E0,1E0,1E0,5.033165E7,2.4519731E3,1.6503105E3,5.2610065E1,3.5432583E3,1.417988E6,1.2679E4,1.066317E12,1.3674345E4,-7.4162004E-3,-1.1665747E-2,7.0277144E7,-4.694509E-3,-1.5635677E-2,9.38E2,5.676415E-1,1.4311201E-2,-3.3620608E-3,1.8631586E4,8.0708375E5,-2.9841432E-2,-1.4036863E-2,1E0,1.3137E4,7.023838E3,-2.545102E-2,1.2817779E7,1.2E2,1.5324128E6,-1.0357394E-2,2.7913043E0,-3.8321268E-2,3.5E1,3.131E3,5E-1,7.064848E6,3.01E2,8.677148E2,1.07E2,1.7956694E-4,9.9625344E5,2.6772846E-3,1E0,1.766486E7,2.1442097E-2,2.7565938E3,2.1690162E7,8.5837976E7,1.921032E4,1.3144558E4,-5.6801313E-3,-2.2192989E-2,7.023838E3,7.644773E-1,2.6100147E5,1.1159378E12,2.5167784E-1,-5.0868127E-2,1.593E2,1.9041E4,2.0939393E0,6.057963E2,-1.2634554E-2,2.2927427E-1,1.8466014E-1,6.4888096E0,8.35338E-1,2.57636E6,6.5346925E6,1.3952951E-2,1.19839E5,8.111831E-4,-5.649903E-3,3.3903683E-3,3.207168E-2,4.865177E-3,-2.046667E-2,-6.2715844E-3,4.1759796E-3,-7.950544E-3,1.9628948E-2,6.974768E-3,-4.409762E-4,6.2632128E-3,7.926493E-3,-5.091463E-3,5.5325073E-3,2.220092E-2,1.251162E-2,2.1536443E-2,-2.4689494E-3,8.589683E-3,5.711626E-3,-5.2476726E-3,-7.939319E-3,6.731021E-3,-7.2629643E-3,-2.077717E-2,1.1725272E-2,2.385394E-2,3.9952338E-2,1.7851202E-2,-9.963025E-4,1.360835E-2,4.671612E-3,-9.921493E-3,1.3550245E-2,5.8112345E-3,1.8949054E-2,4.0799376E-2,1.4899093E-2,-1.6560933E-2,4.6695487E-4,2.0300016E-2,-5.1410017E-3,-1.77539E-2,9.432625E-3,1.29381E-3,2.457955E-2,6.1044954E-3,-2.9950356E-2,-1.1388044E-2,-1.7730225E-2,2.4639869E-3,-6.0007675E-3,-1.6081035E-2,-2.7156882E-2,-1.5220247E-2,-1.6891837E-2,-5.8301245E-3,-8.438276E-3,1.0849E-3,2.974882E-3,-6.155658E-3,6.7558414E-3,-2.9452348E-3,2.5408536E-3,1.5542958E-2,3.339106E-3,-7.394705E-3,1.3534941E-2,2.0821819E-3,-1.1062532E-3,-1.6675686E-2],"split_indices":[19,2,59,28,45,32,52,38,17,48,52,16,52,5,45,45,2,2,27,58,2,54,8,27,8,0,38,4,53,102,45,52,42,53,17,102,6,7,4,52,56,4,29,2,31,4,0,0,12,0,0,2,42,0,0,33,28,0,0,8,2,4,0,9,0,32,0,54,0,0,2,53,28,0,55,0,38,28,0,102,47,0,4,45,32,4,52,0,0,4,34,28,31,58,0,52,9,53,52,0,38,27,53,27,1,45,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.642E3,2.071E3,5.71E2,1.62E3,4.51E2,3.31E2,2.4E2,1.121E3,4.99E2,3.36E2,1.15E2,5.8E1,2.73E2,1.47E2,9.3E1,7.51E2,3.7E2,3.88E2,1.11E2,1.18E2,2.18E2,8.2E1,3.3E1,4.8E1,1E1,2.62E2,1.1E1,1.35E2,1.2E1,7.3E1,2E1,6.92E2,5.9E1,6.3E1,3.07E2,1.66E2,2.22E2,9.5E1,1.6E1,7E1,4.8E1,1.19E2,9.9E1,1.3E1,6.9E1,3E1,3E0,9E0,3.9E1,6E0,4E0,1.27E2,1.35E2,5E0,6E0,4.4E1,9.1E1,5E0,7E0,2.1E1,5.2E1,1.6E1,4E0,6.42E2,5E1,3.2E1,2.7E1,5.8E1,5E0,9.4E1,2.13E2,5.6E1,1.1E2,1.85E2,3.7E1,3.2E1,6.3E1,1.3E1,3E0,3.9E1,3.1E1,1.9E1,2.9E1,7.9E1,4E1,2.7E1,7.2E1,6E0,7E0,5.6E1,1.3E1,1.7E1,1.3E1,3.5E1,4E0,3.3E1,9.4E1,8.2E1,5.3E1,2.3E1,2.1E1,7.1E1,2E1,7E0,1.4E1,3.2E1,2E1,9E0,7E0,5.83E2,5.9E1,1E1,4E1,2.9E1,3E0,8E0,5E1,1.4E1,8E1,1.29E2,8.4E1,1.5E1,4.1E1,1.05E2,5E0,1.26E2,5.9E1,2.1E1,1.6E1,4E0,2.8E1,6E0,5.7E1,6E0,7E0,8E0,3.1E1,2.7E1,4E0,9E0,2E1,7.4E1,5E0,2.2E1,1.8E1,1.9E1,8E0,7E1,2E0,4.9E1,7E0,1E1,3E0,1.1E1,6E0,1.1E1,2E0,3E1,5E0,3.1E1,2E0,7.3E1,2.1E1,4.6E1,3.6E1,2.9E1,2.4E1,1.5E1,6E0,1.2E1,5.9E1,1.3E1,7E0,4E0,3E0,3E0,1.1E1,9E0,2.3E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[1.5107727E-3,6.2422026E-2,-2.1350959E-1,-1.00000445E-2,1.9007328E-1,-3.328192E-1,-4.5204706E-2,-1.5767956E-2,4.4709764E-2,1.5834345E-1,4.0813473E-1,-6.568949E-1,-3.028608E-1,-1.1140232E-1,6.276351E-2,-3.72615E-2,1.6657418E-1,1.2049971E-1,3.845171E-1,9.0460374E-4,4.3923622E-1,-1.9046584E-2,-3.729672E-2,-3.334063E-1,-7.223023E-2,-8.367229E-2,-4.2563576E-1,-1.0553695E-1,1.8096268E-1,-5.9970755E-2,1.6194424E-1,1.7866638E-1,-2.0934422E-2,1.4383371E-1,-1.7838275E-1,5.064117E-1,2.4896368E-1,4.9928537E-1,1.6260529E-1,-2.9447743E-1,-6.8295217E-1,2.1391213E-1,-1.5566526E-1,-2.2162172E-1,-5.32205E-2,-2.2608304E-2,-6.1746906E-3,-6.267876E-2,-2.2263179E-2,-1.3083662E-3,2.119064E-1,-1.0342472E-1,1.6715804E-2,9.091881E-2,2.7829954E-1,2.4257691E-1,6.0242068E-2,5.371444E-2,1.9908185E-1,-2.0731106E-1,2.4917552E-2,3.1085178E-1,6.133859E-1,3.2134047E-1,4.051953E-2,4.334938E-1,3.3444386E-2,2.2594444E-1,-4.7762725E-3,-2.3382933E-1,-3.779528E-1,-1.7781088E-2,-4.3462228E-2,1.641716E-2,2.0226995E-3,-1.6774682E-2,-2.4732706E-1,-1.1672439E-2,1.0421706E-3,-1.0437492E-2,-3.732832E-2,3.743706E-3,-1.19858816E-1,1.2219479E-2,6.0899656E-2,-5.7840534E-3,3.4516363E-3,3.7767633E-3,-3.1605663E-3,5.2367053E-3,-9.225471E-3,1.687312E-2,6.496214E-3,7.6703634E-3,1.8991891E-2,4.309809E-3,-1.0515035E-2,4.215987E-3,-9.035498E-3,2.998568E-3,1.2380063E-2,-1.6155442E-2,-5.231133E-3,-4.7013615E-3,8.655112E-3,1.7601652E-2,-5.459258E-3,1.9979272E-2,3.4529496E-2,-8.4089884E-4,1.6986944E-2,-2.368349E-3,9.3083335E-3,2.1377819E-2,2.7239432E-3,-8.5398984E-5,1.2645383E-2,-1.5937015E-2,-5.690159E-3,-2.2719162E-2,-1.4334592E-2,3.0329346E-3,-7.547988E-3,-1.4504537E-2,-4.107378E-3,3.7144008E-3,-3.6910535E-3,-7.7957194E-3,4.45263E-3,-2.0412346E-3,6.749374E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,79,-1,-1,81,-1,-1,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,115,-1,117,119,-1,-1,-1,-1,121,123,-1,-1,-1,125,-1,127,-1,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.458983E1,1.9027634E1,1.1691822E1,7.1661654E0,5.090067E0,3.1117516E0,1.743771E0,5.128574E0,0E0,5.5343018E0,1.1296806E0,5.7488537E-1,2.1879597E0,1.280155E0,1.8691449E0,5.3025746E0,1.0855181E0,3.9311676E0,1.4105825E0,0E0,1.364008E0,0E0,0E0,3.568142E0,9.40522E-1,5.7631826E-1,1.04504585E-1,5.7148695E-1,3.6026645E-1,3.5040128E0,9.6639943E-1,1.0147104E0,0E0,2.5783453E0,2.4629211E-1,8.134823E-1,6.80959E-1,6.2714577E-1,3.0501118E-1,1.1823425E0,1.5528879E0,1.8467337E-1,3.777799E-1,1.3983357E-1,2.9553246E-1,0E0,0E0,2.9930863E-1,0E0,0E0,2.979362E-1,2.052042E0,1.9636341E0,4.1820425E-1,4.274156E-1,1.0532775E0,4.2765802E-1,1.6719861E0,2.5951529E0,4.222721E-1,1.3280997E-1,4.9948335E-1,3.3267307E-1,3.906572E-1,1.8955931E-1,2.4657917E-1,0E0,1.21950984E-1,0E0,1.6447983E0,6.467123E-1,0E0,0E0,0E0,0E0,1.5664203E-1,1.3554454E-1,0E0,0E0,0E0,4.9455845E-1,0E0,2.5265825E-1,0E0,1.0543051E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,69,69,70,70,75,75,76,76,80,80,82,82,84,84],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,80,-1,-1,82,-1,-1,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,116,-1,118,120,-1,-1,-1,-1,122,124,-1,-1,-1,126,-1,128,-1,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.0950326E6,1E0,7.3610186E3,3.2196458E6,6.9121815E-2,1.067536E3,2.065904E6,4.4709764E-2,4.4839956E3,5.6553E4,1.2920592E0,1.1879E4,1.2199979E10,1E0,6.28205E8,4.4E1,4.2343444E7,2.5003334E7,9.0460374E-4,2.3487206E7,-1.9046584E-2,-3.729672E-2,1.0017953E8,1E0,1.06E3,1.2151E4,1.4299594E8,2.4418726E-3,1.273801E6,1.5435694E9,1.7463945E4,-2.0934422E-2,5.862126E2,3.202814E5,2.79495E-1,3.0990322E5,1.7921995E3,2.0288463E0,8.73E2,1.992945E5,1.447E3,4.928836E9,6.1E1,1.9646037E-4,-2.2608304E-2,-6.1746906E-3,1.4302E4,-2.2263179E-2,-1.3083662E-3,3.4570816E0,3.402546E6,5.107429E1,1E0,9.397667E6,1.048E3,2.84755E5,5.024605E0,1.5166431E5,1.7495675E5,4.286E3,5.68E2,6.7652373E3,5.0279167E1,4.8E1,5.6E1,3.3444386E-2,2.612E3,-4.7762725E-3,1.421E3,3.0608664E2,-1.7781088E-2,-4.3462228E-2,1.641716E-2,2.0226995E-3,1.16E2,3.970405E3,-1.1672439E-2,1.0421706E-3,-1.0437492E-2,5.2879925E5,3.743706E-3,6.9309726E0,1.2219479E-2,3.633605E6,-5.7840534E-3,3.4516363E-3,3.7767633E-3,-3.1605663E-3,5.2367053E-3,-9.225471E-3,1.687312E-2,6.496214E-3,7.6703634E-3,1.8991891E-2,4.309809E-3,-1.0515035E-2,4.215987E-3,-9.035498E-3,2.998568E-3,1.2380063E-2,-1.6155442E-2,-5.231133E-3,-4.7013615E-3,8.655112E-3,1.7601652E-2,-5.459258E-3,1.9979272E-2,3.4529496E-2,-8.4089884E-4,1.6986944E-2,-2.368349E-3,9.3083335E-3,2.1377819E-2,2.7239432E-3,-8.5398984E-5,1.2645383E-2,-1.5937015E-2,-5.690159E-3,-2.2719162E-2,-1.4334592E-2,3.0329346E-3,-7.547988E-3,-1.4504537E-2,-4.107378E-3,3.7144008E-3,-3.6910535E-3,-7.7957194E-3,4.45263E-3,-2.0412346E-3,6.749374E-3],"split_indices":[19,47,59,4,28,58,52,28,0,52,1,56,2,5,102,7,3,45,45,0,50,0,0,45,64,2,9,45,39,9,7,50,0,52,28,38,28,4,53,0,33,0,12,3,42,0,0,10,0,0,57,1,48,74,45,2,1,53,28,33,0,3,52,56,3,8,0,2,0,2,52,0,0,0,0,8,52,0,0,0,48,0,53,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.639E3,2.057E3,5.82E2,1.313E3,7.44E2,3.4E2,2.42E2,1.306E3,7E0,6.51E2,9.3E1,2.7E1,3.13E2,1.5E2,9.2E1,1.169E3,1.37E2,5.59E2,9.2E1,7E0,8.6E1,1E1,1.7E1,2.76E2,3.7E1,1.39E2,1.1E1,3.8E1,5.4E1,1.05E3,1.19E2,1.35E2,2E0,5.19E2,4E1,4.7E1,4.5E1,7E1,1.6E1,2.5E2,2.6E1,8E0,2.9E1,2.4E1,1.15E2,9E0,2E0,3.5E1,3E0,7E0,4.7E1,6.7E2,3.8E2,7.5E1,4.4E1,8.7E1,4.8E1,1.98E2,3.21E2,3.5E1,5E0,1.8E1,2.9E1,3.3E1,1.2E1,5.5E1,1.5E1,1.3E1,3E0,1.47E2,1.03E2,1.2E1,1.4E1,4E0,4E0,1.2E1,1.7E1,2.2E1,2E0,9E0,1.06E2,1E1,2.5E1,3.6E1,1.1E1,6.1E2,6E1,2.17E2,1.63E2,7.1E1,4E0,2.8E1,1.6E1,5.8E1,2.9E1,4.4E1,4E0,1.74E2,2.4E1,9.8E1,2.23E2,1.4E1,2.1E1,3E0,2E0,1.6E1,2E0,1.2E1,1.7E1,3E0,3E1,8E0,4E0,5.3E1,2E0,2E0,1.1E1,7.8E1,6.9E1,4.4E1,5.9E1,8E0,4E0,1.2E1,5E0,2.7E1,7.9E1,2.1E1,4E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-5.3433225E-5,-1.5606336E-1,7.3864155E-2,-2.7044863E-1,-6.729937E-2,1.4702409E-2,2.221406E-1,-1.8964024E-1,-3.4172866E-1,6.9088924E-1,-8.146373E-2,-9.232414E-2,8.218058E-2,1.5090178E-1,3.4294137E-1,-2.3640485E-1,1.3869165E-3,-3.8782865E-1,-9.947628E-2,7.990749E-3,3.9115895E-2,-1.6133048E-1,4.0026207E-2,-1.1340608E-1,2.1073145E-1,7.3953636E-2,3.799462E-2,1.0658173E-1,2.8640768E-1,1.4029399E-2,3.7708235E-1,-3.2219544E-1,-1.4733085E-1,-3.2927968E-2,8.546034E-3,-3.2338172E-1,-6.2852305E-1,2.9404778E-2,-1.847192E-1,-6.3770436E-2,-2.2038229E-1,-2.2137025E-2,1.138762E-1,-2.2129531E-1,-6.7894E-2,3.100706E-2,1.09621294E-1,1.124494E-1,-8.148124E-2,1.6623518E-1,1.6632285E-2,3.603062E-1,-3.1705517E-2,8.196518E-3,-1.8811304E-3,3.427386E-1,3.3171814E-2,-4.8889685E-1,-2.4396746E-1,-1.0648112E-1,-3.6418104E-1,-2.0523336E-4,-8.668755E-3,-4.0112618E-1,-1.9051564E-1,-4.0924692E-1,-7.2952604E-1,7.134041E-3,-1.5955258E-3,-3.5210484E-1,-3.2327783E-3,-8.8373445E-2,2.609646E-1,-3.0881885E-1,-1.1437352E-1,-6.4291414E-3,-2.0982799E-1,-7.090155E-2,1.4729528E-1,-2.9242274E-1,-1.6357294E-1,2.1406561E-1,-9.155216E-2,1.4698546E-1,-1.0355264E-4,1.6177587E-2,1.801334E-1,-1.7810671E-1,3.806661E-2,-3.739704E-3,1.9890027E-1,-2.2508079E-1,5.5337463E-2,-2.7964246E-3,3.7452504E-1,-1.0497735E-2,2.4067695E-1,3.5375687E-1,-1.6496254E-2,-2.4744457E-2,-6.736191E-3,-4.9003786E-3,-1.5235247E-2,-6.2534437E-3,4.8611E-3,-2.1608597E-2,-4.7450764E-3,-6.4338106E-3,-2.0677082E-2,1.5840215E-3,-1.2459256E-2,-1.7645089E-3,-2.4772966E-2,-3.7082773E-2,-8.989092E-3,-2.4174046E-2,-7.640748E-3,-7.2868373E-3,-4.3770584E-4,2.2648782E-2,1.8773497E-3,-1.5723255E-2,5.5362433E-3,-3.4656918E-3,-1.436615E-2,-1.3850961E-3,6.759027E-3,-1.94962E-3,-1.4518345E-2,-6.35474E-3,3.2020428E-3,1.8818734E-3,9.985261E-3,-1.0560763E-2,-2.5906444E-2,-8.920375E-3,5.5722246E-4,-8.974266E-3,2.4949055E-2,-7.0280256E-3,-9.643518E-4,8.134615E-3,-2.2447424E-3,1.7948869E-3,-1.043556E-2,1.9050927E-3,1.0468E-2,-1.7160157E-2,-5.344521E-3,7.7851377E-3,-2.7293684E-3,5.387367E-3,-6.6583795E-3,1.325398E-2,6.6879336E-3,-1.624695E-2,2.3120295E-3,3.8442214E-4,8.581635E-3,1.8562635E-2,4.05572E-3,-3.1724956E-3,1.8040104E-2,2.6949257E-2,1.5185237E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,-1,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,-1,-1,97,-1,99,101,103,105,-1,-1,107,109,111,113,-1,-1,115,-1,117,119,121,123,125,127,129,131,133,135,137,139,141,-1,143,145,147,149,151,153,155,157,-1,159,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0813173E1,8.703476E0,1.5900902E1,2.0952053E0,5.26482E0,9.374464E0,4.3974E0,1.5904565E0,2.1726513E0,4.574442E-1,4.6347914E0,3.231176E0,4.598614E0,1.9333153E0,2.1395187E0,1.0452957E0,2.246075E-1,2.3717403E0,3.6797616E-1,0E0,0E0,1.6451416E0,8.747642E-1,2.2927985E0,1.3862183E0,4.724227E0,0E0,1.3279517E0,1.8920703E0,1.6586736E-1,1.6375885E0,8.196564E-1,5.9901714E-1,1.3071056E-1,0E0,1.2974157E0,4.5446014E-1,1.0994836E-1,3.686362E-1,9.0190274E-1,1.6428127E0,3.0526668E-1,5.476774E-1,5.228524E-1,2.234253E0,0E0,1.172654E-1,4.115237E0,1.8203446E0,8.280287E-1,9.4801015E-1,4.088421E-1,8.698921E-1,0E0,0E0,1.3181839E0,0E0,1.4023733E-1,5.031569E-1,3.239842E-1,2.0907247E-1,0E0,0E0,6.202564E-1,8.027842E-1,5.039692E-1,3.8865185E-1,0E0,0E0,1.5465176E-1,0E0,5.191444E-1,3.575074E-1,8.4065056E-1,6.2541294E-1,3.2604E-1,1.13927275E-1,1.280484E-1,4.8096895E-1,9.7079086E-1,3.1627107E-1,3.30095E0,1.2072756E0,1.0107821E-1,0E0,1.3145682E0,1.9933529E0,1.0027645E0,8.4949625E-1,4.0878004E-1,5.3796625E-1,4.5331228E-1,5.0835E-1,0E0,2.0602036E-1,0E0,3.1718135E-1,1.0171356E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,57,57,58,58,59,59,60,60,63,63,64,64,65,65,66,66,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,94,94,96,96,97,97],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,-1,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,-1,-1,98,-1,100,102,104,106,-1,-1,108,110,112,114,-1,-1,116,-1,118,120,122,124,126,128,130,132,134,136,138,140,142,-1,144,146,148,150,152,154,156,158,-1,160,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.382114E0,3.794679E8,6.8503425E6,2.5E1,1.8108038E5,1.5589395E0,1.638058E2,7.621583E2,2.9328E4,5.517536E-1,1.5694646E3,7.3610186E3,4.7692296E-1,6.860185E2,2.68E0,2.579E3,4.3702424E7,4.422732E6,7.990749E-3,3.9115895E-2,1.683653E8,1.3640075E0,2.1525E4,1.5E1,3.7832818E0,3.799462E-2,1.5548611E7,2.6307288E7,4.6847186E2,1.6828056E1,7.08E2,1.2342778E1,4.822581E0,8.546034E-3,4.6253732E2,9.251576E3,2.2444444E0,2.116711E3,2.1169072E3,2.7009541E-2,2.0620978E7,2.821E3,3.5315048E2,7.330957E4,3.100706E-2,9.408768E1,8.01E2,1.176E3,1.4931408E6,1.0824417E0,2.1853803E4,6.43673E5,8.196518E-3,-1.8811304E-3,3.7826266E4,3.3171814E-2,9.824211E5,7.5408E4,9.544947E6,2.8107285E-1,-2.0523336E-4,-8.668755E-3,1.1317E4,1.18827E5,6.5029144E5,2.1907706E12,7.134041E-3,-1.5955258E-3,1.2068E4,-3.2327783E-3,3.24645E5,2.5927516E11,2.0601392E3,3.1933654E7,2.6312E4,3.4692544E7,3.3968747E-1,1.6052323E5,3E0,2.7E2,1.91E2,7.805608E4,1.7427321E-1,-1.0355264E-4,2E0,1E0,1E0,9.283875E6,3.37E2,1.03E2,2.2214102E6,2.2772677E5,-2.7964246E-3,8.476884E-1,-1.0497735E-2,5.0279167E1,1.0594161E1,-1.6496254E-2,-2.4744457E-2,-6.736191E-3,-4.9003786E-3,-1.5235247E-2,-6.2534437E-3,4.8611E-3,-2.1608597E-2,-4.7450764E-3,-6.4338106E-3,-2.0677082E-2,1.5840215E-3,-1.2459256E-2,-1.7645089E-3,-2.4772966E-2,-3.7082773E-2,-8.989092E-3,-2.4174046E-2,-7.640748E-3,-7.2868373E-3,-4.3770584E-4,2.2648782E-2,1.8773497E-3,-1.5723255E-2,5.5362433E-3,-3.4656918E-3,-1.436615E-2,-1.3850961E-3,6.759027E-3,-1.94962E-3,-1.4518345E-2,-6.35474E-3,3.2020428E-3,1.8818734E-3,9.985261E-3,-1.0560763E-2,-2.5906444E-2,-8.920375E-3,5.5722246E-4,-8.974266E-3,2.4949055E-2,-7.0280256E-3,-9.643518E-4,8.134615E-3,-2.2447424E-3,1.7948869E-3,-1.043556E-2,1.9050927E-3,1.0468E-2,-1.7160157E-2,-5.344521E-3,7.7851377E-3,-2.7293684E-3,5.387367E-3,-6.6583795E-3,1.325398E-2,6.6879336E-3,-1.624695E-2,2.3120295E-3,3.8442214E-4,8.581635E-3,1.8562635E-2,4.05572E-3,-3.1724956E-3,1.8040104E-2,2.6949257E-2,1.5185237E-2],"split_indices":[20,56,7,45,0,28,41,52,52,1,42,52,4,35,52,54,2,45,47,0,0,7,39,9,0,53,0,45,43,4,35,2,54,56,0,4,33,56,4,52,38,29,2,33,45,0,56,2,10,45,53,33,11,0,0,52,0,32,1,9,27,0,0,9,29,28,31,0,0,9,0,32,31,52,48,2,45,38,33,8,3,2,28,38,0,6,102,102,45,0,11,28,33,0,34,0,56,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E3,8.58E2,1.812E3,3.74E2,4.84E2,1.296E3,5.16E2,1.77E2,1.97E2,8E0,4.76E2,5.01E2,7.95E2,3.26E2,1.9E2,1.42E2,3.5E1,1.65E2,3.2E1,2E0,6E0,2.87E2,1.89E2,4.69E2,3.2E1,7.87E2,8E0,2.47E2,7.9E1,1.8E1,1.72E2,7.1E1,7.1E1,3E1,5E0,1.32E2,3.3E1,1.3E1,1.9E1,1.09E2,1.78E2,1.03E2,8.6E1,1.38E2,3.31E2,5E0,2.7E1,6.31E2,1.56E2,1.48E2,9.9E1,6.4E1,1.5E1,4E0,1.4E1,1.57E2,1.5E1,2.1E1,5E1,6.1E1,1E1,2.6E1,4E0,8.2E1,5E1,1.2E1,2.1E1,4E0,9E0,7E0,1.2E1,1.02E2,7E0,9.6E1,8.2E1,9.6E1,7E0,1.3E1,7.3E1,6E1,7.8E1,2.5E1,3.06E2,2E1,7E0,2.61E2,3.7E2,8.6E1,7E1,2.4E1,1.24E2,1.3E1,8.6E1,2E0,6.2E1,9E0,6E0,1.55E2,2E0,1.9E1,2E0,1.8E1,3.2E1,5.5E1,6E0,7E0,3E0,9E0,7.3E1,1.2E1,3.8E1,3E0,9E0,1.9E1,2E0,3E0,4E0,5.6E1,4.6E1,3E0,4E0,9.2E1,4E0,6.8E1,1.4E1,8.4E1,1.2E1,3E0,4E0,9E0,4E0,2.7E1,4.6E1,4.8E1,1.2E1,6.9E1,9E0,1.1E1,1.4E1,1.72E2,1.34E2,1.8E1,2E0,2.4E2,2.1E1,8E1,2.9E2,2.2E1,6.4E1,3E1,4E1,1.3E1,1.1E1,5.2E1,7.2E1,9E0,4E0,6.3E1,2.3E1,5.9E1,3E0,2E0,4E0,2.1E1,1.34E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[1.2119557E-3,-1.540139E-1,7.6362364E-2,-1.7639807E-1,2.5078934E-1,1.71523E-2,2.1235107E-1,-2.5985917E-1,-7.406544E-2,3.997826E-2,1.4232053E-1,-9.186061E-2,8.405321E-2,1.4797324E-1,3.5971838E-1,-2.3324619E-1,-6.0352886E-1,-1.5483174E-1,3.3917613E-2,2.2796384E-1,-3.301447E-2,-1.15178585E-1,1.9698398E-1,1.0628826E-1,-2.0412114E-1,1.2223314E-1,4.5618987E-1,3.9698562E-1,9.943642E-2,-3.019184E-1,-1.1961865E-1,-6.822117E-1,-2.2506E-1,-1.3383795E-1,-3.6214522E-1,-2.3699392E-2,5.064838E-2,2.8102335E-1,-2.194184E-4,-1.2456429E-2,2.6508642E-2,-9.159425E-2,-3.4361082E-1,2.9769607E-2,5.193369E-3,4.2846348E-2,1.9239607E-1,-2.8201234E-1,-6.087331E-2,3.489819E-2,1.7615661E-1,3.357693E-1,2.982226E-2,1.7352588E-1,4.4247276E-1,-6.3184793E-3,1.5818225E-1,-2.4881501E-1,-5.076333E-1,1.6581057E-2,-2.0508291E-1,-4.7581366E-1,-3.951628E-2,2.7363605E-3,-1.6731359E-2,-5.6544937E-2,-1.6535358E-1,-2.4306362E-2,-1.1256031E-2,-6.996627E-2,9.4258524E-2,1.8107957E-1,1.9408436E-2,-7.910122E-3,3.8964525E-3,2.3566937E-1,-1.0140389E-1,-2.215539E-1,-3.3515327E-2,-9.452073E-3,1.3040622E-1,1.7504188E-1,3.5708874E-2,-1.5014423E-1,-4.0553913E-1,-1.10911205E-1,5.8153574E-3,-2.5529439E-2,1.6182709E-1,2.0000014E-1,-8.409927E-2,1.7599303E-2,2.1117046E-3,1.12859376E-1,1.7093938E-2,4.5768884E-1,-5.2397903E-3,1.0657383E-2,-2.5691148E-3,-1.4343081E-2,-5.419665E-3,-1.48444865E-2,-2.9164245E-2,1.41752735E-2,-8.6316053E-4,-1.2008255E-2,-4.63748E-3,-5.770687E-3,-2.7314786E-2,-1.5841426E-2,-1.1700669E-3,-1.0532126E-2,-4.732732E-3,-5.443011E-3,4.925782E-3,1.6829729E-3,8.880654E-3,1.1461196E-2,1.3592008E-3,2.3971599E-2,1.9472501E-3,-1.1030624E-2,-3.8127773E-3,-1.5555846E-2,-5.8996133E-3,-2.0172992E-3,8.54556E-3,4.7921035E-3,1.6470768E-2,1.3572843E-2,5.0935447E-3,-1.039082E-2,3.8486072E-3,-3.1682875E-2,-1.5908632E-2,-6.78267E-3,3.8142996E-3,-3.5286003E-3,2.6995398E-3,1.7673096E-2,3.4765357E-3,7.505292E-3,1.6551051E-2,9.859318E-3,-9.691396E-3,6.5784464E-3,-3.8292196E-3,3.3776365E-2,2.0481696E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,-1,-1,73,75,77,-1,-1,79,81,83,85,87,89,91,-1,93,95,-1,97,99,101,103,105,107,-1,-1,-1,109,111,-1,-1,113,115,117,-1,-1,-1,119,121,123,-1,125,127,129,-1,131,133,135,-1,137,139,141,143,-1,-1,145,-1,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1111414E1,7.9274673E0,1.4465624E1,7.026909E0,2.7950737E0,9.145928E0,5.1172867E0,4.011549E0,3.2496092E0,0E0,6.081769E-1,3.2354126E0,5.00038E0,2.9669132E0,1.5610256E0,3.264574E0,7.993889E-1,8.785701E-1,1.4405768E0,3.344339E-1,1.961324E-1,2.3497667E0,1.2861078E0,3.9359312E0,6.076665E-1,1.6562715E0,3.724575E-1,1.3869114E0,3.1989732E-1,2.746748E0,1.8774741E0,3.743782E-1,2.6551554E-1,4.6565795E-1,2.3184705E-1,0E0,8.2715845E-1,1.9979608E-1,0E0,0E0,1.3960893E-1,1.3144538E0,1.6275992E0,0E0,0E0,1.9107769E0,2.807394E0,5.255234E-1,2.0503175E-1,1.047253E0,1.3696017E0,1.5958738E-1,0E0,2.4901712E-1,1.0493832E0,0E0,2.573137E-1,1.4167099E0,8.729477E-1,6.144797E-1,4.6504593E-1,3.679874E-1,0E0,0E0,0E0,5.027007E-1,4.792533E-1,0E0,0E0,3.2880622E-1,6.1334705E-1,1.1771414E-1,0E0,0E0,0E0,5.9949607E-1,1.0800714E0,2.8503466E-1,0E0,1.615833E0,9.762647E-1,2.1879911E0,0E0,3.1010932E-1,1.1827898E-1,1.10788494E-1,0E0,3.7089854E-1,7.8675246E-1,1.2062378E0,6.93244E-1,0E0,0E0,1.1020735E-1,0E0,5.603237E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,65,65,66,66,69,69,70,70,71,71,75,75,76,76,77,77,79,79,80,80,81,81,83,83,84,84,85,85,87,87,88,88,89,89,90,90,93,93,95,95],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,-1,-1,74,76,78,-1,-1,80,82,84,86,88,90,92,-1,94,96,-1,98,100,102,104,106,108,-1,-1,-1,110,112,-1,-1,114,116,118,-1,-1,-1,120,122,124,-1,126,128,130,-1,132,134,136,-1,138,140,142,144,-1,-1,146,-1,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.4839956E3,3.474851E8,1E0,2.46E2,1.8108038E5,1.1251919E6,1.2427474E8,3.3926086E2,3.997826E-2,6.747114E7,1.4845161E3,3.956147E-1,6.8545566E0,2E0,9.357879E2,4.8083666E-1,1.5988282E1,5.2143492E-5,2.7711287E0,5.704748E4,1E0,1.5E1,3.5259784E2,2.6476662E6,7.2456576E8,2.257944E1,1.460806E6,8.319452E2,3.0161016E7,3.791269E7,3.1372E4,1.7322648E5,2.8945708E6,2.3067484E0,-2.3699392E-2,2.3125623E5,3.5985E4,-2.194184E-4,-1.2456429E-2,2.9104478E0,7E0,7.492813E0,2.9769607E-2,5.193369E-3,1.5859042E6,4.4839956E3,7.31E2,2.0550244E0,4.5154482E-1,2.6974544E7,7.771383E0,2.982226E-2,8.32E3,5.1797544E7,-6.3184793E-3,3.8807898E6,4.31375E2,2.8035452E11,7.131107E6,3.0387878E3,5.980516E5,-3.951628E-2,2.7363605E-3,-1.6731359E-2,1.8631586E4,1.1771E4,-2.4306362E-2,-1.1256031E-2,1.0323588E3,1.6394367E0,1.0881593E10,1.9408436E-2,-7.910122E-3,3.8964525E-3,1.2068493E4,3.75E1,5E0,-3.3515327E-2,9.750871E6,7.780377E6,3.68E0,3.5708874E-2,1.415178E3,8.8365E4,1E0,5.8153574E-3,1.4571098E5,2.53383E5,6.4683E4,9.391714E4,1.7599303E-2,2.1117046E-3,1.964127E6,1.7093938E-2,1.0918E4,-5.2397903E-3,1.0657383E-2,-2.5691148E-3,-1.4343081E-2,-5.419665E-3,-1.48444865E-2,-2.9164245E-2,1.41752735E-2,-8.6316053E-4,-1.2008255E-2,-4.63748E-3,-5.770687E-3,-2.7314786E-2,-1.5841426E-2,-1.1700669E-3,-1.0532126E-2,-4.732732E-3,-5.443011E-3,4.925782E-3,1.6829729E-3,8.880654E-3,1.1461196E-2,1.3592008E-3,2.3971599E-2,1.9472501E-3,-1.1030624E-2,-3.8127773E-3,-1.5555846E-2,-5.8996133E-3,-2.0172992E-3,8.54556E-3,4.7921035E-3,1.6470768E-2,1.3572843E-2,5.0935447E-3,-1.039082E-2,3.8486072E-3,-3.1682875E-2,-1.5908632E-2,-6.78267E-3,3.8142996E-3,-3.5286003E-3,2.6995398E-3,1.7673096E-2,3.4765357E-3,7.505292E-3,1.6551051E-2,9.859318E-3,-9.691396E-3,6.5784464E-3,-3.8292196E-3,3.3776365E-2,2.0481696E-2],"split_indices":[20,52,7,102,10,28,28,45,52,0,45,52,27,35,6,55,42,54,42,42,28,6,0,52,32,7,54,47,52,45,45,11,28,45,58,0,28,2,0,0,54,3,54,0,0,28,52,11,39,38,43,53,0,2,50,0,28,55,31,45,4,28,0,0,0,33,11,0,0,52,42,12,0,0,0,28,4,6,0,9,28,54,0,52,1,89,0,28,9,2,28,0,0,1,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.665E3,8.69E2,1.796E3,8.24E2,4.5E1,1.252E3,5.44E2,4.53E2,3.71E2,6E0,3.9E1,4.76E2,7.76E2,3.8E2,1.64E2,4.22E2,3.1E1,2.12E2,1.59E2,2.6E1,1.3E1,4.41E2,3.5E1,7.21E2,5.5E1,3.52E2,2.8E1,1.43E2,2.1E1,2.62E2,1.6E2,2.5E1,6E0,1.94E2,1.8E1,4E0,1.55E2,2.1E1,5E0,2E0,1.1E1,4.01E2,4E1,5E0,3E1,4.16E2,3.05E2,3.5E1,2E1,1.35E2,2.17E2,1.8E1,1E1,2.5E1,1.18E2,4E0,1.7E1,2.1E2,5.2E1,6.2E1,9.8E1,1.2E1,1.3E1,2E0,4E0,5.7E1,1.37E2,7E0,1.1E1,4.1E1,1.14E2,1.3E1,8E0,2E0,9E0,1.1E1,3.9E2,3.1E1,9E0,2.61E2,1.55E2,2.97E2,8E0,1.8E1,1.7E1,1.6E1,4E0,9.2E1,4.3E1,1.99E2,1.8E1,1.6E1,2E0,2E1,5E0,1.15E2,3E0,1.3E1,4E0,1.52E2,5.8E1,1.9E1,3.3E1,6E0,5.6E1,6.8E1,3E1,3E0,9E0,5E0,5.2E1,7.4E1,6.3E1,3.3E1,8E0,7E1,4.4E1,9E0,4E0,4E0,7E0,5.5E1,3.35E2,1.4E1,1.7E1,2.23E2,3.8E1,1.37E2,1.8E1,1.14E2,1.83E2,1.4E1,4E0,2E0,1.5E1,1.4E1,2E0,5.8E1,3.4E1,1.2E1,3.1E1,1.55E2,4.4E1,5E0,1.3E1,1.8E1,2E0,1E1,1.05E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[-9.025267E-4,-1.3896793E-1,6.3401364E-2,-2.3047473E-1,-5.0123435E-2,2.4148004E-2,2.3427945E-1,-7.497217E-1,-2.1711773E-1,3.6045685E-2,-6.32899E-2,-1.48147E-2,1.8111321E-1,2.8776106E-1,3.6191937E-2,-4.3232553E-2,-1.4104466E-2,-2.8571758E-1,-1.5108955E-1,-1.039998E-1,1.09880276E-1,-6.32237E-2,7.2402485E-2,9.866355E-2,2.942659E-1,2.630818E-1,6.5693223E-1,-3.9383966E-1,9.517797E-2,-5.8497024E-1,-2.5799212E-1,-7.683761E-2,-2.1111427E-1,-6.1867006E-2,-2.3735094E-1,-1.0717568E-2,1.6390999E-1,-4.40953E-2,-2.60416E-1,-7.533637E-2,1.0049781E-1,6.1613254E-2,2.7795768E-1,3.3515242E-1,-2.2460647E-2,4.4898596E-1,2.1105194E-1,1.7014723E-2,4.0806178E-2,-2.5476264E-2,-8.473882E-3,1.4346276E-1,-1.8473464E-1,-8.07251E-3,-3.3989172E-2,-3.3321697E-1,-1.3463435E-1,1.1846715E-1,-1.0382356E-1,-1.9780806E-1,-3.074304E-2,-8.637672E-2,1.9657634E-1,-3.4236902E-1,-1.1278308E-1,1.1810268E-3,-6.3577644E-3,1.22817546E-1,1.7473219E-2,-8.554943E-2,1.0941487E-1,-3.7090725E-1,-1.6633303E-1,-4.9658455E-2,-2.9459309E-2,1.9624798E-1,3.422164E-2,-1.1727213E-1,1.1424032E-1,-1.2262575E-3,3.1396902E-1,3.8565356E-1,9.1139704E-2,-2.899652E-2,1.13808855E-1,4.9838895E-1,8.705254E-3,2.2165413E-1,-1.717199E-2,-1.2298125E-3,2.5202063E-1,-3.2229635E-1,4.8935506E-3,-1.4815088E-2,-3.3869617E-2,2.7267411E-3,-1.0900609E-2,1.0358575E-3,1.1958922E-2,-1.3460235E-2,-3.6432345E-3,-4.2113075E-3,-1.1780219E-2,-1.4100601E-2,-3.2161924E-3,1.2550069E-2,-4.385187E-3,-9.69781E-3,-1.9877557E-2,5.4907165E-3,-9.346425E-3,8.992355E-3,1.9796053E-4,-6.7965193E-3,-1.6988249E-3,3.0361772E-2,3.0012364E-3,-1.5751198E-2,-3.061828E-2,-1.0338508E-2,-3.1597777E-3,5.0719344E-4,-6.836698E-3,8.051416E-3,1.6193341E-2,9.442629E-4,1.0045677E-2,1.0519773E-2,-1.5337736E-2,-9.595826E-3,6.123958E-3,5.749683E-4,1.6217286E-2,2.3616776E-2,1.4429935E-2,-7.6268674E-3,7.839307E-3,1.5715433E-2,-8.580432E-4,2.521508E-2,8.04935E-3,7.982369E-3,1.7341759E-2,4.140153E-3,-4.9528796E-3,2.0140136E-2,6.882822E-3,-5.4836664E-3,-2.1527488E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,-1,-1,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,-1,-1,-1,89,91,-1,-1,93,95,97,99,101,-1,103,105,107,109,-1,-1,111,-1,113,115,117,119,121,-1,123,125,127,129,-1,131,133,135,-1,137,139,-1,141,-1,143,145,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3553644E1,6.8419323E0,1.2134988E1,2.7154903E0,4.5821385E0,9.01124E0,3.5703697E0,4.3066692E-1,1.7958202E0,0E0,2.9934678E0,4.9908733E0,2.7044792E0,2.2506676E0,1.8891926E0,0E0,0E0,1.4918709E0,9.152808E-1,1.912446E0,5.2954197E-1,2.8552167E0,1.7591968E0,1.1188104E0,1.6003599E0,2.3330536E0,6.39091E-1,1.7550969E-1,9.0501153E-1,7.115588E-1,1.6533003E0,5.129957E-1,5.322809E-1,1.676585E0,1.0341005E0,1.1553011E-1,4.1760468E-1,4.4257197E0,6.3479376E-1,9.1200054E-1,2.2460978E0,1.3596296E0,3.2505918E-1,1.3080502E0,1.2606144E0,6.2226295E-1,1.2752409E0,0E0,0E0,0E0,0E0,8.79931E-1,4.276203E-1,0E0,0E0,7.8978443E-1,1.2776433E0,1.4711373E-1,3.9175272E-1,5.764766E-1,0E0,9.3993115E-1,4.5795792E-1,3.633237E-1,7.530744E-1,0E0,0E0,3.6924082E-1,0E0,1.5389943E0,3.5894587E0,1.280036E-1,1.7469323E-1,3.71712E-1,0E0,5.21811E-1,5.331017E-1,2.3274164E0,5.017365E-1,0E0,1.8956709E-1,6.6733074E-1,3.894061E-1,0E0,3.729297E-1,3.2956886E-1,0E0,1.445365E0,0E0,2.3294386E-1,5.2360225E-1,1.3357317E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,51,51,52,52,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,80,80,81,81,82,82,84,84,85,85,87,87,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,-1,-1,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,-1,-1,-1,90,92,-1,-1,94,96,98,100,102,-1,104,106,108,110,-1,-1,112,-1,114,116,118,120,122,-1,124,126,128,130,-1,132,134,136,-1,138,140,-1,142,-1,144,146,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,9.075E3,3E0,2.5E1,1.8137958E6,3.5301748E7,1.36054E-1,2.308943E0,3.6045685E-2,1.5083523E0,1.645614E6,4.930435E2,4.5505118E2,3.0972284E1,-4.3232553E-2,-1.4104466E-2,1.0693E4,5.1148495E6,6.747114E7,2.3125623E5,4.0844156E7,6.1E2,1E0,2.0295714E7,2E0,4.22088E0,4.1043E4,2.3580047E10,1.7848537E0,1.752512E2,1.36E2,9.5032835E0,2.1169072E3,2.867292E3,1.3653578E7,7.6008E4,6.6473596E2,1.817757E1,1.744993E6,2.4E1,2.1033548E2,1.1693E4,2E0,1E0,4.728721E0,3.7826266E4,1.7014723E-2,4.0806178E-2,-2.5476264E-2,-8.473882E-3,7.023838E3,1E0,-8.07251E-3,-3.3989172E-2,4.5723195E6,6.748543E-1,2.72E-1,4.5555557E1,1.2E1,-3.074304E-2,1E0,8.996408E0,2.8277853E11,9.64E2,1.1810268E-3,-6.3577644E-3,8.7643677E-1,1.7473219E-2,1.7669278E5,2.04E5,1.6355225E1,5E0,5.8019036E-1,-2.9459309E-2,7.298614E2,1.2502964E9,5E-1,3.2801925E6,-1.2262575E-3,1.459995E2,7.857143E0,1.4523809E0,-2.899652E-2,1.3865336E0,1E0,8.705254E-3,6.9673E4,-1.717199E-2,6.8743766E-4,1.2695038E-3,2.14099E5,4.8935506E-3,-1.4815088E-2,-3.3869617E-2,2.7267411E-3,-1.0900609E-2,1.0358575E-3,1.1958922E-2,-1.3460235E-2,-3.6432345E-3,-4.2113075E-3,-1.1780219E-2,-1.4100601E-2,-3.2161924E-3,1.2550069E-2,-4.385187E-3,-9.69781E-3,-1.9877557E-2,5.4907165E-3,-9.346425E-3,8.992355E-3,1.9796053E-4,-6.7965193E-3,-1.6988249E-3,3.0361772E-2,3.0012364E-3,-1.5751198E-2,-3.061828E-2,-1.0338508E-2,-3.1597777E-3,5.0719344E-4,-6.836698E-3,8.051416E-3,1.6193341E-2,9.442629E-4,1.0045677E-2,1.0519773E-2,-1.5337736E-2,-9.595826E-3,6.123958E-3,5.749683E-4,1.6217286E-2,2.3616776E-2,1.4429935E-2,-7.6268674E-3,7.839307E-3,1.5715433E-2,-8.580432E-4,2.521508E-2,8.04935E-3,7.982369E-3,1.7341759E-2,4.140153E-3,-4.9528796E-3,2.0140136E-2,6.882822E-3,-5.4836664E-3,-2.1527488E-2],"split_indices":[20,56,2,3,0,28,45,42,54,0,42,9,4,56,56,0,0,9,45,45,28,45,2,84,50,17,53,10,5,54,52,11,53,52,4,32,10,52,58,47,8,4,1,6,8,38,52,0,0,0,0,4,102,0,0,28,27,56,52,18,0,75,57,31,0,0,0,57,0,28,5,53,6,41,0,52,7,53,45,0,33,56,54,0,54,6,0,2,0,39,27,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.651E3,8.42E2,1.809E3,4.14E2,4.28E2,1.472E3,3.37E2,9E0,4.05E2,6E0,4.22E2,1.18E3,2.92E2,2.65E2,7.2E1,6E0,3E0,1.97E2,2.08E2,3.42E2,8E1,7.59E2,4.21E2,1.7E2,1.22E2,2.5E2,1.5E1,8E0,6.4E1,1.5E1,1.82E2,9.4E1,1.14E2,2.61E2,8.1E1,2.5E1,5.5E1,6.93E2,6.6E1,6.7E1,3.54E2,1.42E2,2.8E1,1.08E2,1.4E1,5.3E1,1.97E2,7E0,8E0,4E0,4E0,5.5E1,9E0,4E0,1.1E1,1.12E2,7E1,1.1E1,8.3E1,1.12E2,2E0,2.39E2,2.2E1,4.3E1,3.8E1,2E1,5E0,4.7E1,8E0,5.46E2,1.47E2,2.9E1,3.7E1,6.5E1,2E0,1.44E2,2.1E2,3.2E1,1.1E2,3E0,2.5E1,8.9E1,1.9E1,2E0,1.2E1,4.4E1,9E0,1.94E2,3E0,2.4E1,3.1E1,6E0,3E0,1.07E2,5E0,2.3E1,4.7E1,7E0,4E0,1E1,7.3E1,3.5E1,7.7E1,1.9E1,2.2E2,1.8E1,4E0,1.6E1,2.7E1,1E1,2.8E1,3E1,1.7E1,2.56E2,2.9E2,1.1E1,1.36E2,2.7E1,2E0,2.4E1,1.3E1,4E1,2.5E1,1.22E2,2.2E1,1.95E2,1.5E1,1.2E1,2E1,4E0,1.06E2,2E0,2.3E1,3.7E1,5.2E1,4E0,1.5E1,4E0,8E0,4E1,4E0,1.41E2,5.3E1,1.3E1,1.1E1,1.1E1,2E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[-1.3425478E-3,4.8757233E-2,-1.7652026E-1,-4.3651097E-2,1.2142991E-1,-2.7242485E-1,-4.5142222E-2,-8.06858E-2,6.7806914E-2,1.5672283E-1,-1.3868158E-1,-4.6507066E-1,-2.3862034E-1,-7.999441E-2,1.078331E-1,-1.15254015E-1,2.983426E-2,7.533232E-2,-2.4984965E-2,2.087838E-1,1.4028188E-2,-2.125899E-1,7.081138E-2,-4.7950563E-1,-3.7875737E-3,-2.0027818E-1,-3.8116303E-1,-6.4674936E-2,-4.0643236E-1,8.05217E-3,3.201623E-1,-9.052396E-2,-2.9399326E-1,1.2081901E-1,-5.4091666E-2,3.457673E-2,1.6027793E-1,1.10237435E-1,2.7129313E-1,-1.842489E-2,6.1804426E-1,-2.5536036E-1,2.6509364E-3,1.3243365E-1,-2.6964265E-1,-1.7777385E-2,-5.642904E-1,-2.2838044E-1,5.28191E-3,-6.124084E-1,-2.7012056E-1,-7.6627746E-2,2.2728874E-1,-3.4849874E-3,-2.3531916E-2,5.021013E-3,-6.010439E-2,1.8268974E-2,5.8064847E-3,3.3546217E-2,-1.0676294E-1,-6.340038E-1,-2.2558402E-1,-1.4084831E-1,1.5647407E-1,-1.5914435E-2,-2.6724732E-1,9.3876295E-2,-4.180032E-2,1.620395E-2,1.0435567E-1,2.1467651E-1,4.8671722E-2,3.1491216E-2,2.5118002E-1,-7.185794E-2,1.1258791E-1,3.5531007E-2,5.884379E-3,-3.3652577E-1,-1.5212949E-1,4.63879E-2,3.4717417E-1,-3.817188E-3,-2.0737078E-2,-3.209351E-2,-1.2240141E-2,-4.5594755E-1,-1.8829298E-1,1.1369117E-1,-7.5557926E-3,-8.444118E-3,-6.831547E-1,-2.138393E-1,-3.665556E-2,-9.361729E-2,6.1215954E-3,1.6439272E-2,1.2208688E-3,-4.7428445E-3,5.7649314E-3,-1.0842792E-2,7.1337093E-3,-6.5562846E-3,-1.2527142E-3,4.285857E-3,-3.9560676E-2,-1.4964692E-2,-4.5439205E-3,-1.1996783E-2,-1.2448498E-3,9.62783E-3,-3.6240048E-3,1.8040574E-3,-5.296071E-3,-2.0547083E-2,-5.38744E-3,3.2815791E-3,1.2586527E-2,-3.4588396E-3,1.7048124E-2,5.9721293E-3,-1.4533703E-2,-2.421101E-3,1.1460163E-2,-8.5112394E-4,4.658486E-3,1.0437517E-2,1.9314304E-2,-7.317713E-3,-4.4669711E-4,2.6530349E-3,1.1587833E-2,-1.766145E-2,-7.28712E-3,-1.1596382E-2,-5.1019005E-3,9.353475E-3,-1.9290172E-3,1.959618E-2,4.154649E-3,-1.4382444E-2,-3.762815E-2,-1.3851483E-2,-6.8218824E-3,1.1440678E-2,-4.960935E-4,-3.517574E-2,-1.0335316E-2,-5.2380366E-3,-1.3033063E-2,2.754422E-3,-5.6999354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,-1,81,83,-1,85,87,89,91,93,95,97,-1,-1,-1,99,-1,-1,101,103,105,107,109,111,113,115,117,119,-1,121,123,125,-1,127,129,131,-1,-1,133,135,137,139,-1,-1,-1,-1,141,143,145,-1,-1,147,149,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3362513E1,1.3906513E1,7.4467583E0,3.7700377E0,1.0661463E1,2.11623E0,1.3468387E0,2.6204681E0,1.0507911E0,7.5818977E0,2.167229E0,2.406578E-1,1.5250225E0,9.9888957E-1,9.93183E-1,2.280747E0,1.2588359E0,7.7687216E-1,0E0,4.5778236E0,5.382385E0,1.1945238E0,8.119532E-1,2.3126316E-1,0E0,1.3471785E0,1.4009371E0,7.049046E-1,2.193042E-1,2.2400141E-1,1.3668656E-1,9.30377E-1,1.3223505E0,7.5909066E-1,6.9558835E-1,7.014095E-1,6.916549E-1,1.8670721E0,3.385334E0,1.8351983E0,7.6760626E-1,6.945162E-1,0E0,5.751755E-1,1.5166235E-1,0E0,6.4565086E-1,1.784277E0,5.305578E-1,4.6572447E-1,1.0349567E0,6.71942E-1,1.7848161E-1,0E0,0E0,0E0,1.5244716E-1,0E0,0E0,1.6538293E0,9.8515177E-1,1.46348E0,5.836375E-1,1.1943343E-1,7.393931E-1,3.7956917E-1,2.772318E-1,3.5464084E-1,8.4864074E-1,0E0,5.108661E-1,7.174125E-1,5.9915006E-1,0E0,2.1163883E0,9.399945E-1,5.489254E-1,0E0,0E0,2.3533487E-1,1.424827E-1,3.1762725E-1,1.0984659E-1,0E0,0E0,0E0,0E0,1.3036561E0,7.732606E-1,2.820363E-1,0E0,0E0,2.2544241E-1,2.1385908E-1,0E0,6.8254125E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,72,72,74,74,75,75,76,76,79,79,80,80,81,81,82,82,87,87,88,88,89,89,92,92,93,93,95,95],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,-1,82,84,-1,86,88,90,92,94,96,98,-1,-1,-1,100,-1,-1,102,104,106,108,110,112,114,116,118,120,-1,122,124,126,-1,128,130,132,-1,-1,134,136,138,140,-1,-1,-1,-1,142,144,146,-1,-1,148,150,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7311627E2,1E0,9.9625344E5,4.0844156E7,1.5804776E4,1.7227725E0,4.264897E6,3.684685E2,1.0938637E1,3.4409692E3,1.8302105E6,4.7022E4,1.4299594E8,1.3137E4,5.024605E0,2.46E2,6.895302E7,-2.4984965E-2,6.9664386E2,5.847845E3,1E0,5.2686436E7,2.8035452E11,-3.7875737E-3,1.2204E4,1.7887324E0,2.3274304E2,4.1E2,1.5316E4,1.766486E7,9.6590906E-1,2E0,2.53E2,1.1347826E1,2E0,1.7039736E4,3.2449896E0,3.8E1,7.06699E8,1.417988E6,7.398769E4,2.6509364E-3,1.1803382E4,9.599108E6,-1.7777385E-2,1.191572E6,1.2068E4,1.1708007E7,2.2837209E2,3.6600095E10,1E0,5.1E1,-3.4849874E-3,-2.3531916E-2,5.021013E-3,3.8562928E6,1.8268974E-2,5.8064847E-3,4.0701206E1,2.2348747E2,8E0,2.373E3,4E0,3.3253515E6,1.5E1,2.558106E10,5.5669255E6,1.7331858E1,1.620395E-2,2E0,1.7148E4,1.19507775E-1,3.1491216E-2,3.2196458E6,4.646E3,2.9138382E7,3.5531007E-2,5.884379E-3,9.222717E-1,2.5933685E-2,7.0099896E9,7.7854166E0,-3.817188E-3,-2.0737078E-2,-3.209351E-2,-1.2240141E-2,9.71709E5,1.048E3,6.5884055E11,-7.5557926E-3,-8.444118E-3,9.099893E8,4.259842E8,-3.665556E-2,2.1129233E-1,6.1215954E-3,1.6439272E-2,1.2208688E-3,-4.7428445E-3,5.7649314E-3,-1.0842792E-2,7.1337093E-3,-6.5562846E-3,-1.2527142E-3,4.285857E-3,-3.9560676E-2,-1.4964692E-2,-4.5439205E-3,-1.1996783E-2,-1.2448498E-3,9.62783E-3,-3.6240048E-3,1.8040574E-3,-5.296071E-3,-2.0547083E-2,-5.38744E-3,3.2815791E-3,1.2586527E-2,-3.4588396E-3,1.7048124E-2,5.9721293E-3,-1.4533703E-2,-2.421101E-3,1.1460163E-2,-8.5112394E-4,4.658486E-3,1.0437517E-2,1.9314304E-2,-7.317713E-3,-4.4669711E-4,2.6530349E-3,1.1587833E-2,-1.766145E-2,-7.28712E-3,-1.1596382E-2,-5.1019005E-3,9.353475E-3,-1.9290172E-3,1.959618E-2,4.154649E-3,-1.4382444E-2,-3.762815E-2,-1.3851483E-2,-6.8218824E-3,1.1440678E-2,-4.960935E-4,-3.517574E-2,-1.0335316E-2,-5.2380366E-3,-1.3033063E-2,2.754422E-3,-5.6999354E-3],"split_indices":[19,52,59,28,45,51,39,9,52,53,52,32,9,45,2,53,10,7,0,52,4,96,43,31,0,2,53,56,0,9,47,53,17,2,53,8,50,54,0,7,29,33,0,4,29,0,29,9,47,4,5,89,3,0,0,0,47,0,0,52,52,0,2,8,45,3,19,28,58,0,6,9,38,0,28,2,9,0,0,27,38,5,54,0,0,0,0,29,2,31,0,0,7,7,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E3,2.069E3,5.91E2,9.11E2,1.158E3,3.41E2,2.5E2,6.84E2,2.27E2,1.02E3,1.38E2,4.9E1,2.92E2,2.04E2,4.6E1,5.21E2,1.63E2,2.25E2,2E0,7.47E2,2.73E2,1.02E2,3.6E1,4.7E1,2E0,2.32E2,6E1,1.96E2,8E0,3.2E1,1.4E1,4.59E2,6.2E1,7.8E1,8.5E1,1.53E2,7.2E1,2.91E2,4.56E2,2.6E2,1.3E1,8.8E1,1.4E1,3.1E1,5E0,2.3E1,2.4E1,2.04E2,2.8E1,1.8E1,4.2E1,1.89E2,7E0,2E0,6E0,1.3E1,1.9E1,1E1,4E0,5.3E1,4.06E2,9E0,5.3E1,9E0,6.9E1,7.3E1,1.2E1,8.6E1,6.7E1,1.6E1,5.6E1,1.07E2,1.84E2,2.1E1,4.35E2,1.85E2,7.5E1,1E1,3E0,4.8E1,4E1,2.3E1,8E0,3E0,2E0,1.7E1,7E0,2.9E1,1.75E2,1.7E1,1.1E1,3E0,1.5E1,3.9E1,3E0,1.75E2,1.4E1,4E0,3E0,1.6E1,3E0,1.6E1,3.7E1,2.95E2,1.11E2,2E0,7E0,3.1E1,2.2E1,4E0,5E0,5.8E1,1.1E1,4.7E1,2.6E1,5E0,7E0,7.6E1,1E1,6.3E1,4E0,5.4E1,2E0,9E0,9.8E1,7.8E1,1.06E2,3.59E2,7.6E1,8E1,1.05E2,5.3E1,2.2E1,4E1,8E0,1.2E1,2.8E1,8E0,1.5E1,6E0,2E0,2.1E1,8E0,5.3E1,1.22E2,8E0,9E0,1.3E1,2E0,1.5E1,2.4E1,2.5E1,1.5E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-1.0068289E-4,-1.3803886E-1,6.588948E-2,-1.8094383E-1,2.296315E-2,3.4195255E-2,2.6138955E-1,-2.7613327E-1,-1.3222906E-1,-8.3077E-2,1.01039864E-1,-1.4144013E-2,1.3907227E-1,4.5661634E-1,2.1273552E-1,-1.8471724E-1,-4.0934724E-1,-1.3992633E-1,2.8478988E-2,4.065788E-2,-1.2857856E-1,6.947225E-2,2.9002306E-1,-1.1912059E-1,2.1910315E-2,3.9241705E-2,1.971288E-1,3.0362066E-2,3.274529E-1,1.6397789E-1,3.5770518E-1,-2.5331122E-1,-1.8633373E-2,-3.4374425E-1,-6.672407E-1,-1.064535E-1,-2.6734373E-1,7.3735192E-3,-1.6803473E-3,-1.6668923E-2,-9.162848E-2,5.6957774E-2,1.3329638E-2,4.3484548E-1,2.9879345E-3,-1.0168853E-1,-3.4956014E-1,5.274668E-1,1.0975836E-2,-5.878817E-2,8.5982114E-2,2.314828E-1,1.5431064E-2,4.2184472E-2,3.9422992E-1,1.9641086E-1,-5.3872686E-2,2.8639588E-1,6.2143373E-1,-6.855983E-2,-3.252053E-1,4.693454E-2,-2.6426402E-1,-3.778572E-1,-1.4673752E-1,-7.545159E-1,-1.386058E-2,-1.4061412E-1,3.553743E-2,-4.0166473E-1,-1.3579045E-1,-4.7814015E-2,-2.1602352E-1,2.161548E-1,4.2845283E-2,2.4147045E-2,6.1318427E-3,-2.3349099E-1,-7.1223095E-2,-9.487379E-3,-2.0181065E-2,7.1193436E-3,2.7204888E-2,2.7873196E-2,-1.9451001E-1,7.0035267E-3,-1.3038078E-1,6.831947E-2,1.7909331E-2,1.9228734E-1,3.6737898E-1,1.3549495E-1,-6.0091086E-2,6.827845E-3,-6.665855E-3,2.2366568E-2,6.7188926E-3,1.1844995E-1,2.726299E-1,1.1218681E-2,-1.1490852E-1,2.1205842E-2,3.137526E-1,3.788239E-2,1.4062718E-2,1.613151E-3,-8.644862E-3,-2.18529E-2,-7.682095E-3,1.0435009E-2,-3.3175063E-3,-1.9056788E-2,-6.2790923E-3,-1.8621324E-2,-1.027549E-3,-1.0541899E-3,-1.3820314E-2,-1.5933609E-2,-4.0194903E-2,-1.8358912E-2,-5.993665E-3,-2.4255658E-3,7.4236374E-3,-6.4384355E-3,-2.136784E-2,-4.1184374E-3,-3.230067E-2,3.5648192E-3,-3.2556658E-3,-3.9216685E-3,-1.52913695E-2,2.0612013E-3,1.6054396E-2,1.0770377E-3,6.0667354E-3,-1.5448965E-2,-9.15574E-3,-2.9589625E-3,-2.1439444E-2,-1.3174274E-3,4.3732687E-3,-1.0613964E-2,4.535905E-4,-1.04925055E-2,2.1121812E-3,4.5598056E-3,-2.6140953E-3,1.0113632E-2,-7.834159E-3,3.13661E-2,1.4763115E-2,9.797779E-3,-2.8646956E-3,-5.9076822E-3,2.004941E-3,6.533834E-3,-6.657527E-3,1.4144981E-2,-5.4986967E-4,-1.045945E-2,-4.9194985E-4,-6.444248E-3,8.136847E-3,1.6316138E-2,6.980457E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,-1,-1,-1,71,73,-1,75,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,-1,119,121,123,125,127,129,131,133,-1,-1,135,137,-1,-1,-1,-1,139,141,-1,143,145,-1,147,149,151,153,-1,-1,-1,-1,155,157,-1,159,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4613325E1,6.0484657E0,1.1323956E1,3.1677475E0,1.5400584E0,7.9867105E0,2.324995E0,2.761835E0,2.651372E0,4.5000803E-1,6.1818624E-1,4.086474E0,2.8727837E0,9.3790054E-1,1.3974285E0,1.5902095E0,1.3609858E0,1.9121246E0,0E0,1.9421825E-1,4.3848556E-1,2.2723144E-1,4.6736324E-1,1.0731556E0,4.4387207E0,8.4813464E-1,1.9564304E0,0E0,5.739002E-1,1.1128054E0,7.811456E-1,1.2926745E0,6.8883365E-1,4.6203136E-1,3.7516212E-1,1.7608771E0,1.606473E0,0E0,0E0,0E0,2.6763645E-1,1.9069213E-1,0E0,1.2698376E-1,0E0,1.0193484E0,1.2324715E-1,1.7698717E-1,2.7414904E0,8.997783E-1,6.10266E-1,1.335557E0,4.7058365E-1,1.472981E-1,3.9666843E-1,7.758341E-1,3.9622873E-1,2.9804492E-1,3.236122E-1,3.380298E-1,1.4607882E0,6.9055843E-1,1.0799879E-1,2.4587631E-1,2.2363675E-1,2.2147608E-1,0E0,1.0478654E0,7.389406E-1,4.872203E-1,1.2563106E0,1.0121402E-1,1.5393978E-1,1.298689E-1,1.3879049E-1,0E0,0E0,1.1927652E-1,7.219465E-1,0E0,0E0,0E0,0E0,2.5640154E0,3.3401346E-1,0E0,7.062203E-1,3.9853263E-1,0E0,1.4264517E0,7.840328E-1,2.8212863E-1,2.1157895E-1,0E0,0E0,0E0,0E0,3.3843148E-1,4.4254112E-1,0E0,1.9268373E-1,1.3867562E-1,1.22344255E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,41,41,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,77,77,78,78,83,83,84,84,86,86,87,87,89,89,90,90,91,91,92,92,97,97,98,98,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,-1,-1,-1,72,74,-1,76,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,-1,120,122,124,126,128,130,132,134,-1,-1,136,138,-1,-1,-1,-1,140,142,-1,144,146,-1,148,150,152,154,-1,-1,-1,-1,156,158,-1,160,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.3344417E0,1.3137E4,2.308943E0,8.450237E0,8.9655056E5,1.5151515E-3,6.748543E-1,1.1458888E4,8.2608955E-3,8.498798E-1,6.76E2,1.172E3,6.797394E8,2.043739E0,3.402546E6,7.8351054E11,6.971004E7,2.8478988E-2,1.85221E5,2.2500739E-1,1.9015984E6,2.1367595E0,3.1E1,8E0,1E0,2E1,3.0362066E-2,1.4858E4,4.2620764E2,2.14099E5,4.01E2,4.0844156E7,1E0,9.824211E5,7.2041174E2,2.3023027E3,7.3735192E-3,-1.6803473E-3,-1.6668923E-2,3.2613106E5,1E0,1.3329638E-2,4.2E1,2.9879345E-3,6.1E1,3.8996825E5,1.279012E7,5.9767612E7,5.8365756E-1,6.6315195E6,1.0855529E3,2.9375E0,1.4225797E3,1.24900505E2,2.1019447E-1,1.2245615E0,6E2,6.2105E4,2.7772428E5,6.58542E5,1.8000048E10,2E1,7.8099336E11,1.9557823E0,1.4410892E-5,-1.386058E-2,1.04661034E-4,1E0,1E0,2.6572757E10,4.6263345E-2,1E0,6.6126086E4,1.02E2,2.4147045E-2,6.1318427E-3,1.0972222E0,1E0,-9.487379E-3,-2.0181065E-2,7.1193436E-3,2.7204888E-2,1.7669278E5,7.2E1,7.0035267E-3,1.2673605E7,3.9346433E0,1.7909331E-2,1.3543708E7,1.0918E4,5.5743E4,3.7E1,6.827845E-3,-6.665855E-3,2.2366568E-2,6.7188926E-3,8.4441924E-1,2.6572757E10,1.1218681E-2,1.7036236E5,2.6312E4,1.9840434E7,3.788239E-2,1.4062718E-2,1.613151E-3,-8.644862E-3,-2.18529E-2,-7.682095E-3,1.0435009E-2,-3.3175063E-3,-1.9056788E-2,-6.2790923E-3,-1.8621324E-2,-1.027549E-3,-1.0541899E-3,-1.3820314E-2,-1.5933609E-2,-4.0194903E-2,-1.8358912E-2,-5.993665E-3,-2.4255658E-3,7.4236374E-3,-6.4384355E-3,-2.136784E-2,-4.1184374E-3,-3.230067E-2,3.5648192E-3,-3.2556658E-3,-3.9216685E-3,-1.52913695E-2,2.0612013E-3,1.6054396E-2,1.0770377E-3,6.0667354E-3,-1.5448965E-2,-9.15574E-3,-2.9589625E-3,-2.1439444E-2,-1.3174274E-3,4.3732687E-3,-1.0613964E-2,4.535905E-4,-1.04925055E-2,2.1121812E-3,4.5598056E-3,-2.6140953E-3,1.0113632E-2,-7.834159E-3,3.13661E-2,1.4763115E-2,9.797779E-3,-2.8646956E-3,-5.9076822E-3,2.004941E-3,6.533834E-3,-6.657527E-3,1.4144981E-2,-5.4986967E-4,-1.045945E-2,-4.9194985E-4,-6.444248E-3,8.136847E-3,1.6316138E-2,6.980457E-3],"split_indices":[20,42,2,54,56,28,57,27,52,27,34,2,2,7,42,1,31,45,0,9,27,28,39,8,18,102,3,0,9,58,2,0,45,105,32,52,55,0,0,0,33,67,0,3,0,2,45,7,45,54,28,52,54,4,58,35,53,0,10,28,1,5,8,31,53,39,0,41,102,8,12,57,8,28,8,0,0,53,104,0,0,0,0,28,3,0,45,54,0,47,9,10,3,0,0,0,0,27,12,0,28,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.702E3,8.74E2,1.828E3,6.9E2,1.84E2,1.574E3,2.54E2,2.32E2,4.58E2,7.8E1,1.06E2,1.078E3,4.96E2,4.9E1,2.05E2,1.39E2,9.3E1,4.54E2,4E0,2.1E1,5.7E1,9.2E1,1.4E1,2.75E2,8.03E2,1.83E2,3.13E2,1.9E1,3E1,1.55E2,5E1,9.8E1,4.1E1,7.6E1,1.7E1,3.61E2,9.3E1,8E0,1.3E1,7E0,5E1,8.8E1,4E0,8E0,6E0,2.57E2,1.8E1,1.6E1,7.87E2,5.9E1,1.24E2,2.63E2,5E1,6E0,2.4E1,1.35E2,2E1,4.1E1,9E0,2.8E1,7E1,3.3E1,8E0,6.4E1,1.2E1,1.3E1,4E0,2.91E2,7E1,4.5E1,4.8E1,3.8E1,1.2E1,6E0,8.2E1,6E0,2E0,4.7E1,2.1E2,7E0,1.1E1,2E0,1.4E1,7.28E2,5.9E1,1.5E1,4.4E1,1.18E2,6E0,2.06E2,5.7E1,1.9E1,3.1E1,4E0,2E0,1.8E1,6E0,6.8E1,6.7E1,3E0,1.7E1,4E0,3.7E1,5E0,4E0,1.5E1,1.3E1,3.8E1,3.2E1,1.3E1,2E1,3E0,5E0,6.2E1,2E0,7E0,5E0,3E0,1E1,1.6E1,2.75E2,4.1E1,2.9E1,7E0,3.8E1,4.5E1,3E0,5E0,3.3E1,6E0,6E0,3E0,3E0,6.7E1,1.5E1,1.3E1,3.4E1,2.06E2,4E0,3.89E2,3.39E2,5.2E1,7E0,2.9E1,1.5E1,9.7E1,2.1E1,1.96E2,1E1,8E0,4.9E1,1.4E1,5E0,1.9E1,1.2E1,6.4E1,4E0,6.2E1,5E0,8E0,9E0,2E0,2E0,3.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-9.678152E-4,4.928742E-2,-1.3957706E-1,-1.6921453E-2,1.5478629E-1,-2.2277896E-1,-8.8423025E-3,-5.8182426E-2,8.2129434E-2,3.6538996E-2,2.6213652E-1,-2.6590958E-1,-6.486342E-2,-8.089233E-2,1.7841546E-1,-8.5551836E-2,3.4383427E-2,5.6254745E-2,2.0266624E-1,6.732515E-2,-2.1779166E-1,2.3705065E-1,5.5860996E-1,-2.4211429E-1,-5.977727E-1,-2.712564E-2,-3.791482E-1,-1.030613E-1,9.4193324E-2,8.5301645E-2,2.767736E-1,-1.2499692E-1,-2.4911646E-2,2.3563936E-1,7.271392E-3,-9.370268E-2,8.5404545E-2,1.22610964E-1,1.51450755E-2,2.4184413E-2,5.452534E-2,-1.9402E-2,-1.1952429E-1,1.9726111E-1,3.9350665E-1,6.612122E-1,3.7998985E-2,-4.779668E-1,-2.272058E-1,-1.4475487E-2,-7.15527E-1,-7.286107E-2,8.555383E-2,-4.3329066E-3,-3.1115733E-2,-1.7838975E-2,-8.650867E-2,7.0934873E-3,-6.2047946E-3,6.365458E-3,-2.9177105E-4,2.187177E-2,1.0123552E-2,-8.87959E-2,-2.3270309E-1,-7.423837E-2,5.688265E-2,-1.1886256E-3,1.2529451E-2,5.80037E-2,-8.599566E-2,3.8501327E-3,-2.718196E-1,1.5123035E-1,4.3425363E-2,1.5615915E-1,-1.3003883E-1,5.4774876E-3,1.8187405E-1,-8.121478E-3,-8.3636155E-4,2.242705E-1,1.5603318E-2,4.2339763E-1,6.412744E-3,3.430514E-2,1.75904E-2,8.568076E-3,-7.784519E-3,-3.0381117E-2,-2.5105944E-1,-3.2892048E-1,-1.9779742E-1,-3.8609985E-2,-1.7858945E-2,5.9416056E-2,-1.1352693E-1,1.921929E-2,6.298107E-3,-2.2762224E-1,-5.5652127E-2,-3.8878978E-3,-1.88534E-2,-1.5931077E-2,-6.6891075E-3,-6.0697235E-3,2.0362344E-3,6.299109E-3,-4.3006163E-4,-1.1281198E-3,4.642858E-3,-1.0475943E-2,5.303865E-6,9.091E-3,-5.6141345E-3,-3.4869347E-2,-3.6117095E-3,1.2818813E-2,3.9242506E-3,1.3128718E-3,9.489496E-3,9.580349E-3,1.4761825E-3,-1.4560707E-2,4.1813245E-3,8.647494E-4,-1.879472E-2,1.0155805E-2,-4.948288E-3,-4.0572826E-3,1.1360983E-2,-7.3753814E-3,7.359076E-3,1.6361479E-2,2.4572875E-2,-1.5204191E-3,-1.4807013E-2,-7.115673E-3,-1.9908711E-2,-1.2244772E-2,-6.78189E-3,-6.3968822E-3,6.787856E-3,5.205727E-3,-6.547938E-3,-8.030326E-3,2.1364067E-3,-8.430699E-3,-3.0322986E-2,-3.5273316E-3,3.0107903E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,-1,77,-1,79,81,83,85,87,89,91,-1,93,95,97,-1,-1,-1,99,-1,-1,-1,-1,-1,-1,101,103,105,107,-1,-1,109,111,113,115,117,119,121,123,125,127,-1,-1,129,131,133,-1,-1,-1,-1,-1,-1,135,137,139,-1,-1,141,143,-1,145,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8508152E1,1.3623296E1,7.6883373E0,4.9050145E0,9.536213E0,2.9251041E0,3.7382042E0,2.148793E0,1.0895541E0,2.8255842E0,2.7978745E0,2.5322762E0,1.104594E0,7.860018E-1,6.8159556E-1,1.5612259E0,1.0568395E0,1.2844017E0,5.249579E-1,1.7707306E0,6.695459E-1,2.1946163E0,1.5773497E0,1.0095577E0,5.492711E-1,4.4427627E-1,7.2230124E-1,7.616265E-1,2.9671097E-1,1.7923138E-1,3.3874655E-1,1.5233703E0,1.0491956E0,1.6217327E-1,8.1883234E-1,8.3508515E-1,6.704446E-1,3.4239244E-1,0E0,0E0,1.9540584E0,0E0,1.3498485E-1,1.4340944E0,5.0976944E-1,1.4183521E-1,1.9863677E-1,4.7960186E-1,8.3531094E-1,0E0,1.6541147E-1,3.348743E-1,6.231236E-1,0E0,0E0,0E0,7.232051E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.597054E-1,8.740816E-1,9.984674E-1,4.8300517E-1,0E0,0E0,3.580222E-1,6.973028E-1,7.428303E-1,1.4685639E0,7.4451065E-1,3.686471E-1,1.8085623E-1,2.1564221E-1,1.1433737E0,7.709658E-1,0E0,0E0,1.0410643E0,9.3690175E-1,3.0140877E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.04911625E-1,9.7126484E-1,7.313299E-1,0E0,0E0,2.5723752E-1,2.5718176E-1,0E0,1.4745577E-1,5.1609933E-1,3.0333337E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,56,56,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,90,90,91,91,92,92,95,95,96,96,98,98,99,99,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,-1,78,-1,80,82,84,86,88,90,92,-1,94,96,98,-1,-1,-1,100,-1,-1,-1,-1,-1,-1,102,104,106,108,-1,-1,110,112,114,116,118,120,122,124,126,128,-1,-1,130,132,134,-1,-1,-1,-1,-1,-1,136,138,140,-1,-1,142,144,-1,146,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,5.862126E2,1E0,7.733016E5,1.6649964E5,1.0052E4,6.553948E8,4.264897E6,4.5723195E6,1E0,6.7652373E3,1.2427474E8,1.5265896E8,1E0,4.7013435E1,2.631294E2,1.6E1,1.8122449E2,5.201557E7,1.102E3,1.6332493E1,3.2196458E6,2.3370175E0,7E0,8.8070023E-1,6.7652373E3,1.6992E4,3.6887734E-5,1.100592E6,1.5E1,8.628E3,1.2E1,1.19507775E-1,3.79E2,2.7661E4,1.6530488E8,1.3414634E0,1.8E1,1.51450755E-2,2.4184413E-2,1.3359244E9,-1.9402E-2,2.3898147E5,2.3907686E7,2.3487206E7,6.2105E4,5.5743E4,1.8081865E0,1.5804776E4,-1.4475487E-2,3.2049225E1,3.307766E6,3.3517068E7,-4.3329066E-3,-3.1115733E-2,-1.7838975E-2,9.4168E4,7.0934873E-3,-6.2047946E-3,6.365458E-3,-2.9177105E-4,2.187177E-2,1.0123552E-2,5.0272728E1,8.61E2,2.9076396E9,2.72258E5,-1.1886256E-3,1.2529451E-2,1.2513207E2,1.2460085E7,7.684162E-3,4.38E2,1.4136576E6,2.857E3,6.558106E-3,4.5454E4,9.192915E0,1.5005797E4,-8.121478E-3,-8.3636155E-4,1.3E0,1.3554651E2,1.3793921E3,6.412744E-3,3.430514E-2,1.75904E-2,8.568076E-3,-7.784519E-3,-3.0381117E-2,2.6011018E8,1.0376E4,1.7887324E0,-3.8609985E-2,-1.7858945E-2,4.3376322E5,2.5129596E11,1.921929E-2,6.1061732E7,1.5572634E6,7.123E3,-3.8878978E-3,-1.88534E-2,-1.5931077E-2,-6.6891075E-3,-6.0697235E-3,2.0362344E-3,6.299109E-3,-4.3006163E-4,-1.1281198E-3,4.642858E-3,-1.0475943E-2,5.303865E-6,9.091E-3,-5.6141345E-3,-3.4869347E-2,-3.6117095E-3,1.2818813E-2,3.9242506E-3,1.3128718E-3,9.489496E-3,9.580349E-3,1.4761825E-3,-1.4560707E-2,4.1813245E-3,8.647494E-4,-1.879472E-2,1.0155805E-2,-4.948288E-3,-4.0572826E-3,1.1360983E-2,-7.3753814E-3,7.359076E-3,1.6361479E-2,2.4572875E-2,-1.5204191E-3,-1.4807013E-2,-7.115673E-3,-1.9908711E-2,-1.2244772E-2,-6.78189E-3,-6.3968822E-3,6.787856E-3,5.205727E-3,-6.547938E-3,-8.030326E-3,2.1364067E-3,-8.430699E-3,-3.0322986E-2,-3.5273316E-3,3.0107903E-3],"split_indices":[27,52,102,28,28,2,12,9,28,14,52,45,45,89,56,52,10,52,7,29,58,28,38,3,27,52,2,38,29,8,9,8,38,2,29,5,53,3,0,0,7,0,33,45,50,10,10,53,51,0,58,1,51,0,0,0,1,0,0,0,0,0,0,46,2,5,9,0,0,52,12,38,0,28,2,38,1,57,4,0,0,54,56,4,0,0,0,0,0,0,7,9,53,0,0,28,31,0,45,48,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.655E3,1.949E3,7.06E2,1.198E3,7.51E2,4.31E2,2.75E2,8.46E2,3.52E2,3.58E2,3.93E2,3.38E2,9.3E1,1.99E2,7.6E1,6.53E2,1.93E2,2.91E2,6.1E1,3.2E2,3.8E1,3.64E2,2.9E1,3.17E2,2.1E1,8.4E1,9E0,1.77E2,2.2E1,4E1,3.6E1,3.95E2,2.58E2,2.2E1,1.71E2,4.7E1,2.44E2,3.7E1,2.4E1,8E0,3.12E2,1.2E1,2.6E1,2.92E2,7.2E1,2.4E1,5E0,1.7E1,3E2,7E0,1.4E1,6E1,2.4E1,5E0,4E0,9E0,1.68E2,1.8E1,4E0,2.6E1,1.4E1,8E0,2.8E1,2.97E2,9.8E1,1.61E2,9.7E1,2E0,2E1,1.11E2,6E1,3.1E1,1.6E1,9.4E1,1.5E2,3.3E1,4E0,2.26E2,8.6E1,1.7E1,9E0,2.54E2,3.8E1,6.4E1,8E0,1.9E1,5E0,3E0,2E0,9E0,8E0,6.5E1,2.35E2,1E1,4E0,1.4E1,4.6E1,4E0,2E1,2.9E1,1.39E2,2.91E2,6E0,4.6E1,5.2E1,1.11E2,5E1,4.5E1,5.2E1,3.6E1,7.5E1,2.3E1,3.7E1,1.2E1,1.9E1,4E0,1.2E1,3.4E1,6E1,1.37E2,1.3E1,2.4E1,9E0,2E0,2E0,2.2E2,6E0,7.8E1,8E0,1E1,2.44E2,1.7E1,2.1E1,3.6E1,2.8E1,2E0,6E0,2.2E1,4.3E1,1.14E2,1.21E2,4E0,1E1,4E0,4.2E1,3E0,1.7E1,2.7E1,2E0,1.21E2,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[1.142826E-3,-1.1853941E-1,5.7505704E-2,-1.3726164E-1,2.3136705E-1,-1.26056615E-2,1.2950072E-1,-2.058362E-1,-5.3220227E-2,3.3538356E-2,1.388937E-1,-9.216229E-2,4.1319672E-2,7.360901E-2,2.0648977E-1,-1.8715027E-1,-4.60578E-1,-1.1562095E-1,6.328669E-2,2.2607832E-1,-4.615969E-2,-7.102795E-2,-3.2371226E-1,3.1327907E-2,4.8833832E-1,3.631149E-2,1.7843041E-1,2.7092196E-2,1.8904985E-1,-2.4653E-1,-8.2984455E-2,-3.6778176E-1,-3.4143064E-2,-9.920873E-2,-3.2507485E-1,-2.0650366E-2,7.848238E-2,1.3667449E-2,-9.1902155E-4,-1.2871268E-1,8.225808E-3,-1.0522257E-1,7.736467E-2,-1.9778414E-1,-6.0956055E-1,5.592182E-2,-1.6311471E-1,2.7869472E-2,7.454037E-3,6.320274E-2,-1.11299224E-1,1.9976243E-1,-6.042506E-2,1.6337703E-1,4.1699335E-1,-2.2971192E-1,-6.1704946E-1,6.5067016E-2,-1.4470357E-1,-4.9703106E-1,-6.567212E-3,-6.938136E-2,-2.0713028E-1,-8.809176E-3,-2.0597063E-2,3.4751785E-1,5.973645E-2,-6.5330486E-4,-1.4267014E-2,2.1527879E-1,-1.20376654E-1,3.3336794E-1,-8.062184E-2,1.91107E-3,-1.1969005E-2,-3.5677258E-2,-4.329618E-3,8.569402E-3,1.7004418E-1,-4.4183695E-1,-9.2959076E-2,1.5395634E-2,1.1711859E-1,-2.680986E-1,-5.774945E-2,2.101925E-1,-1.4268249E-2,4.576699E-3,-8.883665E-3,2.1177794E-1,8.9343764E-2,4.6057552E-1,5.8579654E-3,-7.7388957E-3,-1.3801881E-2,-3.740023E-2,-6.464327E-3,1.1828843E-2,3.6730757E-4,-9.255528E-3,-7.798643E-4,-2.678754E-2,-8.898943E-3,-7.7473177E-3,-2.016138E-3,-1.0771496E-2,2.1443737E-3,4.903907E-3,2.2705382E-2,1.0601939E-3,6.8227123E-3,2.3272974E-2,-7.01278E-4,-6.820848E-3,-1.0245618E-3,1.9951154E-2,1.7692539E-5,-7.949823E-3,2.3790363E-3,-1.2338152E-3,6.3595483E-3,3.9244113E-3,1.2247906E-2,7.8099896E-4,-2.856019E-2,-1.4140213E-2,-1.0202689E-3,-3.40558E-3,2.8738207E-3,3.0114052E-3,9.519577E-3,-1.536612E-2,2.3895972E-3,-4.457168E-3,3.7695204E-3,9.310864E-3,2.6181804E-2,1.2208567E-2,8.763836E-4,-4.135167E-3,5.8319904E-3,4.51828E-3,2.3775013E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,-1,65,-1,-1,67,-1,69,71,73,75,77,79,-1,-1,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,109,-1,-1,111,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,131,133,135,-1,-1,-1,137,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8098612E1,5.659092E0,9.20858E0,4.69174E0,1.810791E0,3.9721665E0,3.857545E0,2.0510788E0,2.6827197E0,0E0,6.2352055E-1,1.807235E0,2.4502115E0,2.0382302E0,2.2611408E0,2.577385E0,4.4034052E-1,7.927747E-1,1.003155E0,3.837018E-1,2.5597993E-1,1.7536752E0,9.953251E-1,2.5962403E0,2.831123E-1,1.5432065E0,7.1012306E-1,0E0,2.0502806E0,1.5220451E0,1.4166068E0,6.473212E-1,0E0,7.066884E-1,1.7283428E-1,0E0,6.14699E-1,0E0,0E0,1.9063184E-1,0E0,1.3858478E0,2.6558058E0,2.9501504E-1,5.653579E-1,2.5978527E0,1.1517174E0,0E0,0E0,8.436369E-1,4.883117E-1,7.16691E-1,2.5185287E-1,1.152811E0,4.0599298E-1,9.7725296E-1,7.6372194E-1,4.784521E-1,6.765902E-1,1.8697643E-1,0E0,4.3934083E-1,2.2516513E-1,0E0,0E0,1.9969374E-1,3.6699164E-1,0E0,0E0,8.161845E-1,5.818596E-1,6.9274235E-1,4.6435565E-1,0E0,0E0,0E0,0E0,1.4577651E0,1.0436478E0,8.592746E-1,7.1850616E-1,6.796464E-1,6.695664E-1,2.676748E-1,2.2903879E-1,5.518627E-1,0E0,0E0,0E0,1.6399498E0,7.5699043E-1,3.6039162E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,-1,66,-1,-1,68,-1,70,72,74,76,78,80,-1,-1,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,110,-1,-1,112,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,132,134,136,-1,-1,-1,138,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.4839956E3,1.1803029E8,1E0,2.46E2,1.7669278E5,3.9556694E-1,1.2427474E8,4.5479352E2,3.3538356E-2,1.766486E7,1E0,9.0754684E1,1.3359244E9,1.141E4,3.83E3,7.6008E4,1.5401015E1,5.2143492E-5,5.1E1,1.4157E4,7.471519E2,3.5612745E0,5.9572783E0,2.8839298E7,3.9712732E7,3.0272608E10,2.7092196E-2,5.7039347E0,4.5723195E6,2.8091298E7,1.3528846E1,-3.4143064E-2,1.6527965E7,1.18827E5,-2.0650366E-2,1E0,1.3667449E-2,-9.1902155E-4,1.3674345E4,8.225808E-3,7E0,4.1391224E1,1.3446785E7,1.752E3,3.5734247E2,6.203E3,2.7869472E-2,7.454037E-3,2.1592189E5,3.0379269E1,3.7826266E4,9.2339675E5,8E0,4.728721E0,9.517604E6,1.4E1,3.3995522E5,3.4540886E3,9.277796E-1,-6.567212E-3,1.09E3,1E0,-8.809176E-3,-2.0597063E-2,1.16072E6,6.604754E5,-6.5330486E-4,-1.4267014E-2,1.3050649E4,3.2004956E5,2.6944466E7,4.06E2,1.91107E-3,-1.1969005E-2,-3.5677258E-2,-4.329618E-3,3.321532E6,1.273801E6,8.5539725E8,1.5944675E-4,1.6006084E3,3.596E3,1.7495675E5,6E1,1.9013363E4,-1.4268249E-2,4.576699E-3,-8.883665E-3,1E0,8.079531E4,1.401805E1,5.8579654E-3,-7.7388957E-3,-1.3801881E-2,-3.740023E-2,-6.464327E-3,1.1828843E-2,3.6730757E-4,-9.255528E-3,-7.798643E-4,-2.678754E-2,-8.898943E-3,-7.7473177E-3,-2.016138E-3,-1.0771496E-2,2.1443737E-3,4.903907E-3,2.2705382E-2,1.0601939E-3,6.8227123E-3,2.3272974E-2,-7.01278E-4,-6.820848E-3,-1.0245618E-3,1.9951154E-2,1.7692539E-5,-7.949823E-3,2.3790363E-3,-1.2338152E-3,6.3595483E-3,3.9244113E-3,1.2247906E-2,7.8099896E-4,-2.856019E-2,-1.4140213E-2,-1.0202689E-3,-3.40558E-3,2.8738207E-3,3.0114052E-3,9.519577E-3,-1.536612E-2,2.3895972E-3,-4.457168E-3,3.7695204E-3,9.310864E-3,2.6181804E-2,1.2208567E-2,8.763836E-4,-4.135167E-3,5.8319904E-3,4.51828E-3,2.3775013E-2],"split_indices":[20,52,7,102,10,28,38,45,52,0,47,6,56,7,9,2,10,54,42,3,9,52,53,53,45,45,5,0,35,28,45,56,0,45,29,0,67,0,0,4,0,3,56,45,29,52,29,0,0,28,56,52,47,17,38,45,8,48,4,27,0,2,74,0,0,1,28,0,0,28,47,45,0,0,0,0,0,28,9,31,42,55,2,33,3,52,0,0,0,6,28,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.681E3,8.58E2,1.823E3,8.15E2,4.3E1,9.24E2,8.99E2,4.48E2,3.67E2,6E0,3.7E1,3.73E2,5.51E2,5.22E2,3.77E2,4.19E2,2.9E1,2.39E2,1.28E2,2.5E1,1.2E1,3.43E2,3E1,5.4E2,1.1E1,3.86E2,1.36E2,1.6E1,3.61E2,2.66E2,1.53E2,2.3E1,6E0,2.23E2,1.6E1,3E0,1.25E2,2E1,5E0,9E0,3E0,2.79E2,6.4E1,2.2E1,8E0,4.8E2,6E1,8E0,3E0,3.27E2,5.9E1,1.25E2,1.1E1,3.26E2,3.5E1,2.56E2,1E1,4.5E1,1.08E2,1.4E1,9E0,1.76E2,4.7E1,8E0,8E0,7E0,1.18E2,6E0,3E0,1.2E1,2.67E2,2.4E1,4E1,4E0,1.8E1,6E0,2E0,3.4E2,1.4E2,1.1E1,4.9E1,1.74E2,1.53E2,1.4E1,4.5E1,1.23E2,2E0,5E0,6E0,1.96E2,1.3E2,3E1,5E0,1.2E2,1.36E2,7E0,3E0,1E1,3.5E1,7.8E1,3E1,1.1E1,3E0,3.9E1,1.37E2,4.4E1,3E0,3E0,4E0,8.2E1,3.6E1,5E0,7E0,2.18E2,4.9E1,1.9E1,5E0,2.4E1,1.6E1,2.67E2,7.3E1,7E1,7E1,3E0,8E0,1.2E1,3.7E1,5.9E1,1.15E2,9.3E1,6E1,1.2E1,2E0,3.6E1,9E0,1.19E2,4E0,1.6E2,3.6E1,2E1,1.1E2,3E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[-4.5213656E-4,4.105191E-2,-1.4387938E-1,-3.7552986E-2,1.00957386E-1,-2.3817492E-1,-1.4863122E-2,-7.9134434E-2,5.279529E-2,8.406425E-3,1.5901442E-1,-3.683517E-1,-1.9634594E-1,-4.9881138E-2,1.3646172E-1,-5.5431083E-2,-2.3011091E-1,-1.4316818E-1,6.824182E-2,-2.9016305E-2,1.688783E-1,1.8478848E-1,-3.631344E-2,-1.7717399E-1,-4.3439654E-1,-8.642746E-2,-2.3711523E-1,-3.2418083E-2,-2.8830624E-1,1.855081E-1,-4.630278E-3,-9.4142154E-2,1.4174249E-2,-6.108212E-1,-1.5068156E-1,7.448607E-2,-3.1261602E-1,9.587808E-2,-4.049624E-2,4.6638826E-1,-4.5540992E-2,1.9410118E-1,-2.7855268E-1,1.4284904E-1,3.302752E-1,-1.4746805E-1,1.0504799E-1,-2.083238E-1,1.0951897E-3,-3.3233306E-1,-5.885739E-1,-1.4820004E-1,-7.060679E-3,-5.066721E-1,-2.0282565E-1,-8.207635E-2,3.9370902E-2,-1.9382942E-2,1.0588976E-4,2.866507E-1,9.828748E-2,4.954074E-2,-1.10164106E-1,4.107475E-2,-1.3156094E-1,5.939747E-4,-7.157254E-1,3.0402867E-2,-1.9649558E-1,-1.0808054E-3,1.4772741E-2,-2.4466386E-2,-4.6939827E-3,7.660987E-2,1.6343003E-2,7.8213975E-2,-1.18409224E-1,6.400581E-3,2.4882525E-2,-1.9210975E-1,-2.3315761E-2,1.6187759E-1,3.151049E-2,-1.9583225E-2,-2.6501715E-3,1.2524231E-1,3.1274885E-1,4.9003834E-1,2.3252276E-1,3.147329E-2,-1.9538611E-1,2.3517495E-1,-1.1491689E-2,-6.4447173E-3,-1.7144002E-2,-3.6879358E-1,-5.4102684E-3,-6.31795E-1,-4.3022432E-3,8.0208825E-3,-1.701202E-1,1.9184507E-2,-9.680325E-3,-9.945265E-3,-2.811807E-2,-1.5413874E-1,-3.824723E-1,-7.264922E-2,-1.3167186E-2,6.160074E-3,1.0981631E-3,2.0335693E-2,9.622094E-3,-7.344622E-3,1.2955715E-1,4.6059564E-3,-3.6716233E-3,-2.8294143E-3,-6.976723E-3,-2.9289399E-3,4.054271E-3,-9.964525E-3,2.585566E-5,-3.84374E-2,-8.981231E-3,-1.8443529E-3,6.713946E-3,-1.7170938E-2,-4.263286E-3,4.0424117E-3,-1.5270849E-2,-7.8193017E-4,1.1766819E-2,-3.4992136E-3,-1.4859083E-2,-1.3647474E-2,-2.5140974E-3,-7.7851483E-4,-2.1719597E-2,1.06563335E-2,4.4152075E-3,1.1778935E-2,4.9965135E-3,1.699927E-2,4.7780434E-3,3.0475354E-2,1.5019408E-2,1.0059599E-2,2.2457328E-2,8.535874E-3,-3.7284812E-3,-1.2966271E-2,-4.5685284E-4,-3.0569548E-3,1.7005347E-2,-6.2989024E-3,5.2482467E-3,-1.5106979E-2,-2.4992632E-2,-3.2918274E-2,-1.0717031E-2,-6.4529106E-3,9.37213E-3,-5.709563E-3,-1.1569812E-2,6.534183E-3,-2.2156662E-3,-9.393368E-3,2.480797E-3,-1.6927895E-3,-2.2271125E-2,-2.9571084E-3,-1.2123139E-2,-2.2046359E-3,3.030028E-3,9.938658E-3,2.5757675E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,-1,95,97,99,101,103,105,107,109,-1,-1,111,113,115,117,119,121,-1,123,125,127,-1,-1,-1,-1,129,-1,131,133,-1,-1,135,137,139,-1,-1,-1,141,143,145,147,149,151,153,155,-1,-1,157,-1,159,-1,161,163,165,-1,-1,-1,167,169,171,-1,-1,173,-1,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5888006E1,9.751139E0,7.2836885E0,3.3702817E0,6.312213E0,1.8117008E0,1.3527772E0,2.1845348E0,8.663984E-1,2.7313044E0,3.6415443E0,9.710188E-1,1.1625681E0,8.525253E-1,5.66815E-1,1.4348068E0,2.4178448E0,7.9778755E-1,7.942275E-1,3.0420234E0,1.0246205E0,3.8456612E0,1.351094E0,1.480838E-1,7.6428604E-1,3.5685003E-1,1.6718893E0,6.9482523E-1,4.7443604E-1,3.246945E-1,0E0,7.927687E-1,7.5419056E-1,9.241805E-1,5.867084E-1,2.3405205E-1,4.6050203E-1,9.667405E-1,5.0871086E-1,1.4290977E-1,1.1617146E0,1.119184E0,1.2286675E-1,1.447547E0,2.1037025E0,4.1772652E-1,5.789708E-1,1.7257881E-1,0E0,2.7025986E-1,4.3903208E-1,1.4176434E-1,1.7282946E-1,3.832898E-1,1.4454336E0,1.891166E-1,2.7314848E-1,0E0,0E0,1.2760901E-1,2.0085667E-1,2.1337752E-1,5.5019736E-1,7.269821E-1,3.033148E-1,0E0,4.0017605E-1,1.1889205E-1,9.4338727E-1,0E0,0E0,0E0,0E0,6.3148844E-1,0E0,3.523727E-1,2.644127E-1,0E0,0E0,5.943862E-1,9.402103E-1,3.0906725E-1,0E0,0E0,0E0,1.1152554E0,3.7813807E-1,1.1685114E0,3.7956E-1,1.903752E-1,5.2963626E-1,6.680524E-1,3.1962562E-1,0E0,0E0,1.0485268E-1,0E0,3.2231808E-1,0E0,1.8713088E-1,1.02154136E-1,2.3654257E-1,0E0,0E0,0E0,1.1952455E0,1.015399E0,2.0278782E-1,0E0,0E0,1.6994216E-1,0E0,0E0,0E0,1.1290845E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,73,73,75,75,76,76,79,79,80,80,81,81,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,95,95,97,97,99,99,100,100,101,101,105,105,106,106,107,107,110,110,114,114],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,-1,96,98,100,102,104,106,108,110,-1,-1,112,114,116,118,120,122,-1,124,126,128,-1,-1,-1,-1,130,-1,132,134,-1,-1,136,138,140,-1,-1,-1,142,144,146,148,150,152,154,156,-1,-1,158,-1,160,-1,162,164,166,-1,-1,-1,168,170,172,-1,-1,174,-1,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.6129813E2,1E0,6.978874E5,1.5166431E5,7.2474093E-3,3.0497742E3,9.337562E0,5.8631687E1,1.3359244E9,2.8839298E7,5.0079144E5,1.2E1,1.8000048E10,1.1855755E8,1.325204E6,2E0,2.2444444E0,4.072289E0,8E0,1.314806E8,1.6503105E3,8.848604E1,1.9521575E5,8.672049E9,3.888319E-1,1.2068E4,6.218596E2,3.1827815E12,8.7643677E-1,-4.630278E-3,1.62283E0,5.894636E6,8E0,3.415965E2,7E0,2E0,2.4907975E0,9.87156E0,7.471519E2,1.54E2,2.093563E0,2.6422684E9,4.5723195E6,1.505516E7,7.621583E2,5.8019036E-1,5.065188E-4,1.0951897E-3,3.1743118E-1,1.4225797E3,1.060324E6,1E0,1.8058888E0,2.72258E5,5.077728E8,1.25218E5,-1.9382942E-2,1.0588976E-4,6.862E3,4.349136E6,3.3159972E5,3.2E1,1.8579784E2,6.97E3,5.939747E-4,3.9595376E7,9.213145E0,2.183E3,-1.0808054E-3,1.4772741E-2,-2.4466386E-2,-4.6939827E-3,2E0,1.6343003E-2,6.999242E7,5.743459E-3,6.400581E-3,2.4882525E-2,1.6848017E-2,5.1797544E7,5.670103E-2,3.151049E-2,-1.9583225E-2,-2.6501715E-3,8E0,2.5489312E5,5.473125E3,4.091015E9,5.1821535E4,1.8796511E5,4.8E1,2.3276744E5,-6.4447173E-3,-1.7144002E-2,1.1532471E-3,-5.4102684E-3,9.5082015E-1,-4.3022432E-3,6.395763E2,5.486433E5,2.46875E0,-9.680325E-3,-9.945265E-3,-2.811807E-2,3.647E3,1.411682E7,4.2343444E7,-1.3167186E-2,6.160074E-3,3.65625E1,2.0335693E-2,9.622094E-3,-7.344622E-3,5.323065E7,4.6059564E-3,-3.6716233E-3,-2.8294143E-3,-6.976723E-3,-2.9289399E-3,4.054271E-3,-9.964525E-3,2.585566E-5,-3.84374E-2,-8.981231E-3,-1.8443529E-3,6.713946E-3,-1.7170938E-2,-4.263286E-3,4.0424117E-3,-1.5270849E-2,-7.8193017E-4,1.1766819E-2,-3.4992136E-3,-1.4859083E-2,-1.3647474E-2,-2.5140974E-3,-7.7851483E-4,-2.1719597E-2,1.06563335E-2,4.4152075E-3,1.1778935E-2,4.9965135E-3,1.699927E-2,4.7780434E-3,3.0475354E-2,1.5019408E-2,1.0059599E-2,2.2457328E-2,8.535874E-3,-3.7284812E-3,-1.2966271E-2,-4.5685284E-4,-3.0569548E-3,1.7005347E-2,-6.2989024E-3,5.2482467E-3,-1.5106979E-2,-2.4992632E-2,-3.2918274E-2,-1.0717031E-2,-6.4529106E-3,9.37213E-3,-5.709563E-3,-1.1569812E-2,6.534183E-3,-2.2156662E-3,-9.393368E-3,2.480797E-3,-1.6927895E-3,-2.2271125E-2,-2.9571084E-3,-1.2123139E-2,-2.2046359E-3,3.030028E-3,9.938658E-3,2.5757675E-3],"split_indices":[19,52,59,28,28,41,52,54,47,7,45,28,18,5,45,9,17,56,54,18,1,52,56,33,5,39,9,52,31,57,0,54,45,0,4,8,17,53,58,52,0,39,7,28,50,52,41,38,0,58,4,51,105,54,9,7,29,0,0,9,32,28,10,55,10,0,7,53,2,0,0,0,0,6,0,7,57,0,0,41,50,57,0,0,0,3,33,32,7,33,33,8,33,0,0,39,0,27,0,4,28,53,0,0,0,2,12,45,0,0,56,0,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.667E3,2.069E3,5.98E2,8.95E2,1.174E3,3.45E2,2.53E2,6.13E2,2.82E2,4.53E2,7.21E2,8.2E1,2.63E2,2.06E2,4.7E1,5.31E2,8.2E1,2E1,2.62E2,3.68E2,8.5E1,6.37E2,8.4E1,2.2E1,6E1,7.2E1,1.91E2,1.93E2,1.3E1,3.9E1,8E0,3.41E2,1.9E2,1.3E1,6.9E1,9E0,1.1E1,2.09E2,5.3E1,1.1E1,3.57E2,8.1E1,4E0,4.96E2,1.41E2,4.7E1,3.7E1,1.9E1,3E0,3.8E1,2.2E1,4E1,3.2E1,2E1,1.71E2,1.14E2,7.9E1,9E0,4E0,1.7E1,2.2E1,3.4E1,3.07E2,1.61E2,2.9E1,2E0,1.1E1,1.4E1,5.5E1,7E0,2E0,5E0,6E0,1.95E2,1.4E1,2.1E1,3.2E1,2E0,9E0,4.6E1,3.11E2,7.7E1,4E0,2E0,2E0,4.51E2,4.5E1,5.2E1,8.9E1,1E1,3.7E1,1.7E1,2E1,1.4E1,5E0,3.2E1,6E0,2E1,2E0,5E0,3.5E1,2.9E1,3E0,5E0,1.5E1,1.36E2,3.5E1,1.1E2,4E0,2.3E1,5.6E1,5E0,1.2E1,2E0,2E1,2.5E1,9E0,1.27E2,1.8E2,4.8E1,1.13E2,1.8E1,1.1E1,9E0,2E0,9E0,5E0,2.1E1,3.4E1,1.92E2,3E0,1.4E1,7E0,2.7E1,5E0,2.7E1,1.9E1,3.07E2,4E0,4E1,3.7E1,6.5E1,3.86E2,3.7E1,8E0,2.7E1,2.5E1,8.3E1,6E0,4E0,6E0,2.6E1,1.1E1,5E0,1.2E1,1E1,1E1,2.6E1,6E0,1.7E1,3E0,3E0,2E0,2.2E1,1.3E1,1E1,1.9E1,1.13E2,2.3E1,7E0,2.8E1,1.05E2,5E0,3.2E1,2.4E1,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[7.4223237E-4,4.3862432E-2,-1.23617254E-1,3.4423836E-3,1.7984578E-1,-1.6719507E-1,5.0642032E-2,-7.446E-2,5.7787E-2,1.0162796E-1,2.7888247E-1,-2.2419123E-1,-6.775571E-2,1.3219929E-1,-9.3911566E-2,-1.3195291E-1,-4.3187395E-2,-2.4270918E-2,1.066255E-1,7.3395364E-2,2.9431573E-1,3.4827888E-2,2.561767E-1,-1.6783807E-1,-3.3305484E-1,-1.0919295E-1,2.7622792E-3,3.432525E-1,9.43586E-2,-6.046815E-2,-2.862429E-2,-1.541995E-1,7.93606E-3,7.811829E-2,-7.375492E-2,6.609801E-3,-1.3100047E-1,8.4829345E-2,2.1802732E-1,1.2499836E-1,9.18973E-3,3.539527E-2,1.8457709E-1,3.197639E-1,1.346694E-1,-2.1179755E-1,-5.05088E-2,-1.8805878E-1,-4.2106542E-1,-1.463907E-1,-2.2459723E-2,3.435573E-2,-5.549833E-3,-1.8415968E-3,1.9929629E-2,8.416342E-3,5.3417757E-2,-1.5524073E-1,9.096097E-4,-1.768985E-1,9.226949E-3,-5.938462E-3,7.6667905E-2,-1.4402078E-1,1.671559E-1,-3.7767002E-1,-6.486571E-2,1.4518505E-2,-2.4097411E-2,-1.0473795E-1,-1.3947306E-2,8.988294E-2,-1.9306183E-2,2.3857637E-1,-3.0000275E-3,1.943329E-1,5.364771E-2,2.3736939E-2,-1.6564261E-2,3.0457407E-1,-1.9363318E-2,-1.9709233E-3,3.4500962E-1,1.9179831E-1,9.096698E-3,-1.5406986E-1,-2.9357654E-1,8.272292E-2,-1.0045906E-1,1.7017075E-3,-2.291018E-1,-4.6124667E-1,-8.854711E-2,-9.6162826E-2,-2.3564318E-1,-7.121093E-2,1.2603986E-1,-7.7322006E-2,9.647252E-2,-1.6941411E-2,1.4286807E-1,3.1432898E-3,-9.270185E-3,-6.0414975E-3,-1.0557211E-2,5.735834E-3,-4.4788313E-3,8.851723E-3,-9.51018E-4,-1.5370347E-2,-3.8584997E-3,2.0372406E-2,1.6214466E-3,-9.257867E-3,-2.6808545E-2,-5.5061574E-3,-7.3232845E-4,3.877297E-3,-1.750448E-3,-6.719764E-3,7.284256E-4,2.9459181E-3,7.415229E-3,5.502135E-3,1.32832E-2,1.1610938E-2,4.589427E-3,2.6645002E-4,7.3917066E-3,-6.8728058E-3,2.8037634E-3,5.6780335E-3,1.616278E-2,1.7441098E-2,3.6660156E-3,-4.2355247E-3,1.0564346E-2,5.576741E-3,-4.3133283E-3,-1.3831443E-2,-4.802307E-3,-1.2023601E-2,-2.3516336E-2,7.795427E-3,-7.565273E-3,-6.526951E-3,1.3980351E-3,-1.5470681E-2,-7.833612E-3,3.3111576E-4,-2.3197426E-2,4.331042E-3,-9.248215E-3,-1.106831E-3,-7.222096E-3,-1.6203482E-2,-7.325904E-3,6.753086E-3,-4.3774694E-3,1.2384815E-2,-2.6048522E-4,-1.0631013E-2,-1.5728206E-3,3.2898919E-3,1.0231181E-2,-3.3546698E-3,8.024582E-3,-1.6604393E-3,8.719413E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,-1,-1,-1,-1,99,101,-1,103,105,-1,107,109,111,113,115,117,-1,119,-1,121,-1,123,-1,125,127,129,-1,131,-1,-1,133,135,137,139,141,143,145,-1,147,149,151,153,155,157,159,161,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4183623E1,1.0794416E1,5.1798706E0,6.418169E0,3.457553E0,3.0736866E0,1.6275619E0,1.1144259E0,3.5813618E0,1.3547378E0,1.8293943E0,2.0667458E0,5.847158E-1,6.704143E-1,8.0628264E-1,6.827636E-1,1.5075076E0,1.1016916E0,1.341259E0,7.343824E-1,1.4254563E0,0E0,1.4249372E0,1.175581E0,1.4138737E0,4.0322363E-1,2.8511903E-1,3.777318E-1,2.4383467E-1,3.644263E-1,0E0,7.027502E-1,2.9062292E-1,1.64593E0,8.5080945E-1,1.0555608E0,2.851051E-1,1.1952336E0,5.397892E-1,5.9806347E-1,5.2473384E-1,0E0,2.03114E0,1.1440439E0,4.7890186E-1,7.431612E-1,4.342276E-1,4.3027782E-1,9.211502E-1,3.7184072E-1,2.9131722E-1,4.2368317E-1,0E0,0E0,0E0,0E0,3.2969016E-1,1.989584E-1,0E0,3.390169E-1,2.8379738E-1,0E0,2.22925E-1,2.4357247E-1,2.01655E0,1.9395792E-1,7.8320444E-1,8.750166E-1,0E0,2.8378326E-1,0E0,8.491764E-1,0E0,3.7594128E-1,0E0,2.7060533E-1,2.993433E-1,5.7309794E-1,0E0,1.008265E-1,0E0,0E0,5.693226E-1,3.907112E-1,2.446836E-1,6.9586754E-1,4.4756126E-1,3.7254134E-1,2.223121E-1,0E0,1.8723512E-1,6.966057E-1,1.8590194E-1,2.2802854E-1,2.1144593E-1,1.414625E-1,1.7570427E-1,1.3427164E-1,1.122458E-1,3.0568996E-1,1.6638884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,56,56,57,57,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,69,69,71,71,73,73,75,75,76,76,77,77,79,79,82,82,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,-1,-1,-1,-1,100,102,-1,104,106,-1,108,110,112,114,116,118,-1,120,-1,122,-1,124,-1,126,128,130,-1,132,-1,-1,134,136,138,140,142,144,146,-1,148,150,152,154,156,158,160,162,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,6.624E3,1.998175E3,2.1592189E5,1.8108038E5,2.6476662E6,6.971004E7,1.20086E5,2.5275471E2,6.082581E3,6.737099E7,2.8277853E11,4.832493E-1,8.628E3,1.067509E6,1.4887473E-1,1.1558939E4,5.2E1,2.9251662E6,1.84556E7,1.417988E6,3.4827888E-2,8.2E1,2.011E3,7.5179994E-1,2.2100918E1,1.4540612E7,5.214E3,2.8091298E7,6.08767E3,-2.862429E-2,2.374722E6,2.4360857E4,2.53E2,3.3820656E7,4.4E1,2.5363636E0,3.7852024E1,2E0,5.003246E1,4.6577052E7,3.539527E-2,1.5005797E4,5.4091E4,3.5561565E8,1.411682E7,6.0910425E0,4.4444447E0,3.138649E6,2.74872E5,5.0079144E5,2.4684112E8,-5.549833E-3,-1.8415968E-3,1.9929629E-2,8.416342E-3,2.5882E4,3.9382784E9,9.096097E-4,5.61E2,9.287868E6,-5.938462E-3,3.1648794E-1,8.2E1,5.37798E5,5E0,1.19507775E-1,6.4736843E0,-2.4097411E-2,1.9143396E2,-1.3947306E-2,3.171E3,-1.9306183E-2,5.338169E9,-3.0000275E-3,1.65E2,1.3359244E9,2.496172E3,-1.6564261E-2,5.6981827E1,-1.9363318E-2,-1.9709233E-3,6.292039E0,1.0119178E10,1.19783E5,1.4431512E-1,8.7217E4,6.22198E5,8.0708375E5,1.7017075E-3,2.0939393E0,2.467284E7,4.242625E6,6.546031E5,3.2049225E1,5.3421542E-2,1.9125667E3,2.5534486E4,1.1251919E6,1E0,3.286908E-1,3.1432898E-3,-9.270185E-3,-6.0414975E-3,-1.0557211E-2,5.735834E-3,-4.4788313E-3,8.851723E-3,-9.51018E-4,-1.5370347E-2,-3.8584997E-3,2.0372406E-2,1.6214466E-3,-9.257867E-3,-2.6808545E-2,-5.5061574E-3,-7.3232845E-4,3.877297E-3,-1.750448E-3,-6.719764E-3,7.284256E-4,2.9459181E-3,7.415229E-3,5.502135E-3,1.32832E-2,1.1610938E-2,4.589427E-3,2.6645002E-4,7.3917066E-3,-6.8728058E-3,2.8037634E-3,5.6780335E-3,1.616278E-2,1.7441098E-2,3.6660156E-3,-4.2355247E-3,1.0564346E-2,5.576741E-3,-4.3133283E-3,-1.3831443E-2,-4.802307E-3,-1.2023601E-2,-2.3516336E-2,7.795427E-3,-7.565273E-3,-6.526951E-3,1.3980351E-3,-1.5470681E-2,-7.833612E-3,3.3111576E-4,-2.3197426E-2,4.331042E-3,-9.248215E-3,-1.106831E-3,-7.222096E-3,-1.6203482E-2,-7.325904E-3,6.753086E-3,-4.3774694E-3,1.2384815E-2,-2.6048522E-4,-1.0631013E-2,-1.5728206E-3,3.2898919E-3,1.0231181E-2,-3.3546698E-3,8.024582E-3,-1.6604393E-3,8.719413E-3],"split_indices":[27,2,52,28,28,32,45,9,52,52,7,31,39,9,9,38,51,11,28,45,29,0,8,2,27,56,45,9,45,4,0,29,28,2,7,3,53,53,6,58,48,0,4,33,32,12,56,56,29,29,28,7,0,0,0,0,10,5,0,2,1,0,38,29,5,6,38,58,0,52,0,2,0,31,0,3,7,4,0,56,0,0,38,20,10,58,9,1,28,0,53,45,29,48,58,27,4,33,28,89,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.643E3,1.963E3,6.8E2,1.514E3,4.49E2,5.44E2,1.36E2,6.22E2,8.92E2,2.52E2,1.97E2,3.45E2,1.99E2,8.7E1,4.9E1,2.18E2,4.04E2,3.33E2,5.59E2,2.21E2,3.1E1,8E0,1.89E2,2.29E2,1.16E2,1.25E2,7.4E1,1.2E1,7.5E1,4.7E1,2E0,1.88E2,3E1,8.1E1,3.23E2,2.59E2,7.4E1,4.69E2,9E1,1.22E2,9.9E1,5E0,2.6E1,1.23E2,6.6E1,1.66E2,6.3E1,4.5E1,7.1E1,8.7E1,3.8E1,5.9E1,1.5E1,2E0,1E1,2.4E1,5.1E1,2.1E1,2.6E1,1.65E2,2.3E1,1E1,2E1,2.3E1,5.8E1,8E0,3.15E2,2.56E2,3E0,6.5E1,9E0,4.65E2,4E0,8.4E1,6E0,6.1E1,6.1E1,9.6E1,3E0,2.2E1,4E0,8E0,1.15E2,4.5E1,2.1E1,9.9E1,6.7E1,1.7E1,4.6E1,7E0,3.8E1,6.3E1,8E0,5.7E1,3E1,2.9E1,9E0,2.1E1,3.8E1,2.9E1,2.2E1,3E0,1.8E1,7.8E1,8.7E1,1.1E1,1.2E1,9E0,1.1E1,5E0,1.8E1,1.9E1,3.9E1,5E0,3E0,1.56E2,1.59E2,1.11E2,1.45E2,5E1,1.5E1,3.25E2,1.4E2,2.1E1,6.3E1,4E1,2.1E1,4.2E1,1.9E1,1.6E1,8E1,4E0,1.8E1,1.07E2,8E0,4E0,4.1E1,1E1,1.1E1,2.7E1,7.2E1,5.7E1,1E1,1.3E1,4E0,3.6E1,1E1,1.4E1,2.4E1,3E0,6E1,3E0,5E0,2.5E1,3.2E1,1.2E1,1.8E1,2E0,2.7E1,4E0,5E0,4E0,1.7E1,3.2E1,6E0,2.3E1,6E0,4E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-1.9889919E-3,3.87132E-2,-1.1667124E-1,-4.4713423E-2,9.982473E-2,-1.4555909E-1,9.759356E-2,-6.508019E-2,1.150468E-1,1.863714E-2,1.5678556E-1,-9.81145E-2,-2.5548053E-1,2.3642708E-1,3.128151E-2,-9.8361135E-2,3.9561663E-2,7.911834E-2,1.4433761E-2,2.2502216E-2,5.844456E-3,1.4448035E-1,5.132219E-1,-1.7611371E-1,-7.659394E-3,-4.1859937E-1,-1.7829701E-1,2.9434854E-1,4.208564E-3,5.856077E-2,-2.695908E-1,-8.2598686E-2,-2.6608667E-1,1.1508546E-1,-4.24935E-2,2.9955331E-2,1.3677286E-1,-3.3627875E-2,1.5199356E-1,1.2394203E-1,3.102232E-1,5.9042376E-1,-8.262605E-3,-1.5642376E-1,-3.8965186E-1,4.9412098E-2,-8.215442E-2,-4.5626017E-1,-5.817708E-3,-1.3868155E-1,-3.3596548E-1,-2.203609E-3,1.6143795E-2,-5.691363E-3,7.908833E-2,-2.5300256E-3,-1.8988054E-2,-4.513431E-2,-1.3501313E-1,2.6033907E-3,-3.4143007E-1,1.3080752E-1,-1.2995703E-2,-2.2716054E-1,-1.2776421E-2,-7.714824E-2,4.3919086E-3,1.6992256E-1,2.05793E-3,-8.080126E-3,-1.8626769E-1,1.7368044E-1,-1.3702367E-2,8.736292E-2,1.7360179E-1,3.944957E-1,2.2212154E-1,3.149553E-2,1.2317148E-2,-2.126334E-1,-1.0569682E-1,-4.9034992E-1,-4.6106084E-4,1.7061938E-2,1.5949297E-1,-1.2741265E-1,-6.526266E-3,-1.51054375E-2,-6.074361E-1,-1.79534E-1,-4.9293123E-2,-3.6907598E-1,1.4304917E-3,2.8710915E-2,8.677636E-3,-5.0598397E-3,1.4346811E-3,-1.1604743E-2,-4.7129747E-3,-3.2481253E-2,-9.669161E-3,7.3323967E-3,-5.73213E-3,-1.7371178E-2,-3.7598596E-3,1.5589267E-3,-7.2063077E-3,9.731486E-4,-7.95197E-3,4.6239104E-3,1.2144572E-2,-6.01955E-5,-2.0438358E-2,-1.521743E-2,-5.4271477E-3,9.571553E-3,-2.0137613E-3,4.847699E-3,-4.2682453E-3,1.071768E-2,2.5033245E-3,1.9862695E-2,1.9241526E-3,5.3751995E-3,1.5339971E-2,-1.3109867E-2,-7.1731247E-3,4.7694944E-4,-7.910446E-3,-2.6578486E-2,-8.326242E-3,3.010049E-3,-3.0619202E-3,8.809152E-3,-4.2754523E-3,-7.481009E-3,1.7783456E-3,6.017749E-3,-1.8839579E-3,-3.2715898E-2,-1.5612066E-2,-2.097215E-3,-1.1669237E-2,9.051533E-3,-3.757066E-3,-5.438436E-3,-1.9238846E-2,6.4647617E-3,-1.0151259E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,-1,89,91,-1,-1,-1,93,-1,-1,95,97,-1,99,101,-1,103,105,107,-1,109,-1,111,113,115,-1,117,119,121,123,-1,-1,125,127,129,-1,131,133,135,137,-1,139,141,143,145,-1,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.25096035E1,1.0088957E1,4.359338E0,2.730918E0,5.2782707E0,3.2031279E0,7.6593655E-1,2.5914478E0,6.1218405E-1,2.7218497E0,2.8688717E0,3.0607061E0,2.2671213E0,2.0644438E-1,4.9182674E-1,1.4691052E0,1.1203716E0,2.2549498E-1,0E0,0E0,2.6587803E0,2.174201E0,1.2665772E0,9.1870975E-1,8.6287105E-1,6.0477066E-1,7.5291014E-1,2.9196537E-1,0E0,2.0604679E-1,1.16586596E-1,1.0091257E0,1.1774554E0,6.0661745E-1,4.7453332E-1,3.046983E-1,1.07749045E-1,1.4140512E0,9.80819E-1,1.0384102E0,4.395175E-1,2.7893543E-1,0E0,5.9056425E-1,7.223656E-1,4.0680912E-1,3.0037463E-1,9.156027E-1,0E0,3.709309E-1,3.1384897E-1,0E0,0E0,0E0,2.46853E-1,0E0,0E0,1.3801649E0,8.2057595E-1,0E0,1.6958523E0,5.266627E-1,0E0,2.1334726E-1,4.7926146E-1,1.506724E-1,0E0,1.412623E-1,0E0,8.907899E-1,4.6272945E-1,5.3935194E-1,0E0,8.31017E-1,1.4803481E0,2.3453617E-1,3.632605E-1,0E0,0E0,3.469453E-1,7.9276574E-1,2.1035218E-1,0E0,3.3782384E-1,1.745671E-1,2.6642317E-1,1.524097E-1,0E0,2.8864288E-1,6.074023E-1,2.5106937E-1,1.4625716E-1,0E0,1.8320295E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,54,54,57,57,58,58,60,60,61,61,63,63,64,64,65,65,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,80,80,81,81,83,83,84,84,85,85,86,86,88,88,89,89,90,90,91,91,93,93],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,-1,90,92,-1,-1,-1,94,-1,-1,96,98,-1,100,102,-1,104,106,108,-1,110,-1,112,114,116,-1,118,120,122,124,-1,-1,126,128,130,-1,132,134,136,138,-1,140,142,144,146,-1,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,3.3030225E2,3.4409692E3,2.5247778E6,1.5166431E5,4.987639E7,9.160584E-1,4.264897E6,2.0827537E0,8E0,8.791777E3,2.960909E2,6.7628815E2,1.7313588E7,3.103818E1,5.024605E0,2.53E2,4.5723195E6,1.4433761E-2,2.2502216E-2,1.3359244E9,4.391553E6,1.1896622E6,1.3142047E11,1.775044E7,3.1827815E12,1.2427474E8,8.838E3,4.208564E-3,1.3E1,1.341E3,5.372E3,1E0,1E0,2.5330253E-2,8.459239E6,3.2177323E-3,1E0,6.5880767E3,8.319452E2,1.6470588E0,4.6512E4,-8.262605E-3,1.074E3,2.0184135E0,1.51529E3,2.4559366E1,8.672049E9,-5.817708E-3,3.65625E1,6.8E1,-2.203609E-3,1.6143795E-2,-5.691363E-3,9.609747E3,-2.5300256E-3,-1.8988054E-2,4.16E2,8.74E2,2.6033907E-3,1E0,8.2465213E9,-1.2995703E-2,2.291E3,1.7E1,3E0,4.3919086E-3,1.7486666E3,2.05793E-3,2.7622065E1,7.937392E-3,2.8841177E10,-1.3702367E-2,2E0,1.9493858E7,2.1E1,1.2E1,3.149553E-2,1.2317148E-2,1.9121015E0,4.01E2,3.5799E4,-4.6106084E-4,9.669789E0,2.9933598E0,1E0,2.91E2,-1.51054375E-2,3.5791788E0,1.3E1,9.262083E2,8.7757526E2,1.4304917E-3,4.0601485E6,8.677636E-3,-5.0598397E-3,1.4346811E-3,-1.1604743E-2,-4.7129747E-3,-3.2481253E-2,-9.669161E-3,7.3323967E-3,-5.73213E-3,-1.7371178E-2,-3.7598596E-3,1.5589267E-3,-7.2063077E-3,9.731486E-4,-7.95197E-3,4.6239104E-3,1.2144572E-2,-6.01955E-5,-2.0438358E-2,-1.521743E-2,-5.4271477E-3,9.571553E-3,-2.0137613E-3,4.847699E-3,-4.2682453E-3,1.071768E-2,2.5033245E-3,1.9862695E-2,1.9241526E-3,5.3751995E-3,1.5339971E-2,-1.3109867E-2,-7.1731247E-3,4.7694944E-4,-7.910446E-3,-2.6578486E-2,-8.326242E-3,3.010049E-3,-3.0619202E-3,8.809152E-3,-4.2754523E-3,-7.481009E-3,1.7783456E-3,6.017749E-3,-1.8839579E-3,-3.2715898E-2,-1.5612066E-2,-2.097215E-3,-1.1669237E-2,9.051533E-3,-3.757066E-3,-5.438436E-3,-1.9238846E-2,6.4647617E-3,-1.0151259E-3],"split_indices":[27,52,52,28,28,45,57,9,53,18,52,52,52,1,54,53,10,28,0,0,7,28,28,31,45,31,45,2,0,3,0,29,66,65,38,45,38,14,57,52,54,10,0,2,53,4,56,5,0,56,3,0,0,0,52,0,0,2,2,0,102,31,0,2,3,8,0,33,0,57,42,5,0,6,51,3,3,0,0,53,0,10,0,54,39,105,0,0,54,3,52,52,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.678E3,1.977E3,7.01E2,8.36E2,1.141E3,6.18E2,8.3E1,7.42E2,9.4E1,4.71E2,6.7E2,4.33E2,1.85E2,2.6E1,5.7E1,5.63E2,1.79E2,8E1,1.4E1,1.2E1,4.59E2,6.49E2,2.1E1,2.32E2,2.01E2,5.8E1,1.27E2,1.8E1,8E0,5.3E1,4E0,5.16E2,4.7E1,9.3E1,8.6E1,4.4E1,3.6E1,3.62E2,9.7E1,5.79E2,7E1,1.9E1,2E0,2.14E2,1.8E1,1.14E2,8.7E1,5.1E1,7E0,1.03E2,2.4E1,2E0,1.6E1,5E0,4.8E1,2E0,2E0,3.02E2,2.14E2,9E0,3.8E1,9E1,3E0,1.1E1,7.5E1,1.6E1,2.8E1,2.6E1,1E1,3.11E2,5.1E1,9.3E1,4E0,3.35E2,2.44E2,3.4E1,3.6E1,1.5E1,4E0,1E2,1.14E2,1.4E1,4E0,8.9E1,2.5E1,5.4E1,3.3E1,2.8E1,2.3E1,7E1,3.3E1,2.2E1,2E0,3.3E1,1.5E1,1.67E2,1.35E2,5.3E1,1.61E2,1E1,2.8E1,8.3E1,7E0,5E0,6E0,5.7E1,1.8E1,8E0,8E0,1.5E1,1.1E1,3.07E2,4E0,1.7E1,3.4E1,8.3E1,1E1,3.11E2,2.4E1,1.72E2,7.2E1,3.2E1,2E0,1.8E1,1.8E1,4.9E1,5.1E1,3.9E1,7.5E1,1.1E1,3E0,5.7E1,3.2E1,2.3E1,2E0,4.6E1,8E0,6E0,2.7E1,1.7E1,6E0,2.3E1,4.7E1,3E0,3E1,3E0,1.9E1,1E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[2.3881637E-3,-9.557077E-2,4.910324E-2,-1.2084852E-1,7.416613E-2,8.779722E-3,1.4180945E-1,-8.720527E-2,-2.8043774E-1,1.4441279E-1,-1.5493113E-1,-9.081974E-2,3.9741147E-2,8.6879455E-2,2.4826741E-1,-1.5058519E-1,3.535118E-3,-2.2976854E-1,-4.8608428E-1,2.5196359E-2,1.0443652E-1,-7.8244574E-2,-2.760016E-2,2.1485205E-1,-1.03155054E-1,1.2522324E-2,1.2506633E-1,1.274118E-1,-7.0180963E-3,7.3524185E-2,2.880206E-1,-1.8908446E-1,-9.3372375E-2,5.124224E-2,-8.763057E-2,-4.63054E-1,-1.8967123E-1,-1.0670159E-2,-5.5749923E-1,2.9251019E-2,1.75241E-1,-1.7071894E-1,3.270513E-3,-1.5836745E-3,2.242638E-2,-1.21036105E-1,1.3165633E-1,8.782367E-3,3.3226635E-2,4.8096143E-2,1.655009E-1,9.487594E-2,2.5260225E-1,-8.892402E-2,5.9148263E-2,1.2534995E-1,-2.6654487E-3,1.9981887E-2,2.3814519E-1,-2.9976237E-1,-1.634389E-1,-5.3959325E-2,-2.4922591E-1,7.271923E-2,-1.1347116E-1,-2.0340782E-1,-5.3299487E-2,-2.4928471E-2,-8.374224E-3,-1.544753E-1,-3.37931E-1,-1.0755542E-2,-6.058839E-1,1.2255846E-1,-3.1635784E-2,2.694085E-1,1.3617378E-1,-1.097357E-3,-2.4233009E-1,-1.3195583E-1,8.432194E-3,1.733318E-2,2.6906496E-3,-1.205946E-1,2.4397044E-2,-4.6682186E-2,1.00836776E-1,2.3061757E-1,4.528177E-2,1.4823703E-1,5.0591905E-2,3.0478245E-1,1.4593229E-1,-1.3625433E-1,9.12702E-2,1.0224279E-1,-1.1114979E-1,1.0634754E-2,6.1085936E-2,2.0265873E-1,5.455519E-1,-1.5814949E-2,-9.786132E-4,-1.1502824E-3,-8.946975E-3,-4.359495E-3,6.2180805E-4,-1.8373866E-2,-4.272336E-3,1.0834874E-2,2.7698823E-3,-1.1349192E-3,-1.52766835E-2,-1.4361851E-3,-1.3198065E-2,-8.3153544E-4,-5.816487E-3,-1.1073316E-2,-2.9044272E-3,-5.106811E-3,-2.0373499E-2,-3.1819817E-2,-1.2650009E-2,1.4599806E-3,1.2975077E-2,-6.301207E-3,2.0957938E-3,2.4138685E-4,1.5688792E-2,1.411339E-3,8.237498E-3,-1.50540825E-2,-2.6686518E-3,-5.756288E-3,-1.40636545E-2,-7.2124125E-3,6.203269E-3,8.035083E-4,1.2694883E-2,-5.6949696E-3,1.1472185E-3,5.8515416E-3,-9.158331E-3,9.864568E-3,2.174665E-2,-4.659244E-3,3.7893988E-3,4.286363E-3,9.941699E-3,-5.4508564E-4,5.6757377E-3,1.5407113E-2,1.0809872E-3,2.5437484E-3,1.1645056E-2,8.197421E-3,-8.388856E-3,1.3458895E-2,-9.226298E-3,-3.345424E-3,7.5872005E-3,-9.338353E-3,6.046058E-3,8.501599E-3,-1.8280336E-3,1.941537E-2,8.06898E-3,1.2955044E-2,3.4944043E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,-1,-1,79,81,83,-1,85,87,89,91,93,95,97,-1,-1,99,101,103,105,107,109,111,113,115,-1,-1,117,119,-1,121,123,125,127,129,-1,131,133,-1,-1,-1,135,137,139,141,143,145,147,149,151,153,155,157,159,161,-1,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2287105E1,3.7288847E0,6.794876E0,4.029216E0,1.840919E0,3.9137928E0,3.200533E0,3.5949826E0,1.2533255E0,1.2726203E0,8.274024E-1,1.1571009E0,2.2465663E0,1.3900435E0,1.2778378E0,7.929201E-1,1.1265975E0,9.1306305E-1,3.4584284E-1,0E0,4.207515E-1,3.3687842E-1,0E0,7.6945835E-1,1.2335505E0,1.8666596E0,7.203076E-1,1.0146942E0,6.068149E-1,2.4799371E-1,8.802633E-1,5.726156E-1,9.0795135E-1,6.096347E-1,3.4291202E-1,1.574421E-1,4.330852E-1,0E0,1.6997719E-1,2.311249E-1,1.1414695E-1,1.5321466E-1,0E0,0E0,0E0,8.9920545E-1,3.44639E-1,1.4817135E0,0E0,4.1459817E-1,1.1872821E0,4.7456133E-1,2.4495745E-1,4.3886834E-1,4.6637696E-1,1.4803079E-1,0E0,0E0,1.1112213E0,3.3368373E-1,5.8230925E-1,3.014627E-1,6.111375E-1,3.3189648E-1,3.6144423E-1,2.4208409E-1,1.693358E-1,0E0,0E0,5.404495E-1,2.9863727E-1,0E0,1.4460516E-1,2.1031871E-1,1.941387E-1,1.6756356E-1,1.13459885E-1,0E0,1.168651E-1,4.609995E-1,0E0,0E0,0E0,6.124501E-1,1.1948441E0,1.5592131E-1,3.538438E-1,4.5150948E-1,2.7308688E-1,3.0214548E-1,4.770158E-1,1.5521073E-1,1.5854633E-1,5.0849515E-1,6.4680773E-1,4.9453622E-1,2.8337356E-1,0E0,2.0324592E-1,6.383705E-1,3.596351E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,72,72,73,73,74,74,75,75,76,76,78,78,79,79,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,98,98,99,99,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,-1,-1,80,82,84,-1,86,88,90,92,94,96,98,-1,-1,100,102,104,106,108,110,112,114,116,-1,-1,118,120,-1,122,124,126,128,130,-1,132,134,-1,-1,-1,136,138,140,142,144,146,148,150,152,154,156,158,160,162,-1,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.753919E3,3.474851E8,4.987639E7,1.0828989E8,7.805608E4,1.6789798E0,2.437E3,1.2427474E8,8.417E3,9.22E4,7E0,4.264897E6,2.3058404E7,1.0989723E3,3.68E0,1.9761398E7,3.2069644E2,8.8070023E-1,2.5196359E-2,1E0,3E1,-2.760016E-2,4.7E2,1.5106794E3,3.832274E2,9.52E2,4.1331047E-1,6.6692764E1,1.6474128E0,1.5277338E0,7.22E2,1.2342778E1,1.2342778E1,9.756616E2,8.92686E5,2.0295714E7,-1.0670159E-2,2.7324794E11,5.4786605E-1,6.211896E0,1.9589581E6,3.270513E-3,-1.5836745E-3,2.242638E-2,1.636E3,1.92E2,1.2E1,3.3226635E-2,5.4674416E7,2.4E1,5.68E2,5.2638875E6,6.923162E6,2.6025698E7,4.0248335E6,-2.6654487E-3,1.9981887E-2,1.9013363E4,2.5454E4,7.5408E4,1E0,2.8107285E-1,1E0,4.5154482E-1,2.65851E5,3.653602E7,-2.4928471E-2,-8.374224E-3,8.319452E2,1.2E1,-1.0755542E-2,1.4510472E1,5.033303E-1,5.4106476E7,3.7225406E4,7.6188426E0,-1.097357E-3,7.023838E3,9.55E0,8.432194E-3,1.733318E-2,2.6906496E-3,1.6943169E5,7.780377E6,2.04425E2,6.310469E6,1.4845161E3,1.1721691E5,4.1855645E-1,9.075E3,5.0738932E7,6.963859E1,9.4538574E2,1.30399E5,2.6477592E6,2.3553647E5,1.0634754E-2,2.185321E0,1.3062E4,1.6956E5,-1.5814949E-2,-9.786132E-4,-1.1502824E-3,-8.946975E-3,-4.359495E-3,6.2180805E-4,-1.8373866E-2,-4.272336E-3,1.0834874E-2,2.7698823E-3,-1.1349192E-3,-1.52766835E-2,-1.4361851E-3,-1.3198065E-2,-8.3153544E-4,-5.816487E-3,-1.1073316E-2,-2.9044272E-3,-5.106811E-3,-2.0373499E-2,-3.1819817E-2,-1.2650009E-2,1.4599806E-3,1.2975077E-2,-6.301207E-3,2.0957938E-3,2.4138685E-4,1.5688792E-2,1.411339E-3,8.237498E-3,-1.50540825E-2,-2.6686518E-3,-5.756288E-3,-1.40636545E-2,-7.2124125E-3,6.203269E-3,8.035083E-4,1.2694883E-2,-5.6949696E-3,1.1472185E-3,5.8515416E-3,-9.158331E-3,9.864568E-3,2.174665E-2,-4.659244E-3,3.7893988E-3,4.286363E-3,9.941699E-3,-5.4508564E-4,5.6757377E-3,1.5407113E-2,1.0809872E-3,2.5437484E-3,1.1645056E-2,8.197421E-3,-8.388856E-3,1.3458895E-2,-9.226298E-3,-3.345424E-3,7.5872005E-3,-9.338353E-3,6.046058E-3,8.501599E-3,-1.8280336E-3,1.941537E-2,8.06898E-3,1.2955044E-2,3.4944043E-2],"split_indices":[20,52,7,45,45,28,41,2,45,12,10,3,9,45,52,54,45,52,27,0,102,8,0,1,52,56,2,35,56,39,53,2,54,54,55,1,50,0,31,39,54,32,0,0,0,0,0,18,0,7,8,0,43,29,43,50,0,0,52,9,1,8,27,67,38,29,45,0,0,52,18,0,57,27,45,33,54,0,4,46,0,0,0,33,28,4,45,52,28,41,2,1,56,4,10,47,33,0,41,9,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,8.66E2,1.817E3,7.54E2,1.12E2,1.267E3,5.5E2,6.24E2,1.3E2,8.6E1,2.6E1,3E2,9.67E2,3.64E2,1.86E2,3.67E2,2.57E2,1.06E2,2.4E1,7E0,7.9E1,2.3E1,3E0,1.1E1,2.89E2,7.34E2,2.33E2,2.54E2,1.1E2,3.5E1,1.51E2,2.18E2,1.49E2,1.69E2,8.8E1,1.4E1,9.2E1,6E0,1.8E1,3.9E1,4E1,1.4E1,9E0,6E0,5E0,2.69E2,2E1,7.31E2,3E0,8.1E1,1.52E2,2.03E2,5.1E1,4.9E1,6.1E1,2.5E1,1E1,4E1,1.11E2,3.9E1,1.79E2,1.2E2,2.9E1,1.5E2,1.9E1,1.9E1,6.9E1,1.1E1,3E0,7.6E1,1.6E1,3E0,1.5E1,1.5E1,2.4E1,1E1,3E1,5E0,9E0,2.6E2,9E0,4E0,1.6E1,7.8E1,6.53E2,2.9E1,5.2E1,9.8E1,5.4E1,9.1E1,1.12E2,3.3E1,1.8E1,3.9E1,1E1,4.9E1,1.2E1,9E0,1.6E1,1.01E2,1E1,3.5E1,4E0,2.6E1,1.53E2,7.7E1,4.3E1,1.5E1,1.4E1,1.2E1,1.38E2,1.4E1,5E0,6E0,1.3E1,4.6E1,2.3E1,4.1E1,3.5E1,5E0,1.1E1,1.2E1,3E0,1E1,5E0,1E1,1.4E1,2E0,8E0,8E0,2.2E1,6E0,3E0,2.44E2,1.6E1,7E1,8E0,6.34E2,1.9E1,1.4E1,1.5E1,4.9E1,3E0,9E1,8E0,1E1,4.4E1,4.7E1,4.4E1,5.9E1,5.3E1,3.1E1,2E0,1E1,8E0,4E0,3.5E1,6E0,4E0,1.2E1,3.7E1,9E0,3E0,7E0,9E0,1.3E1,8.8E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-1.7043349E-3,-1.01871766E-1,4.5026742E-2,-1.4534566E-1,-2.0014042E-2,-1.0472367E-2,1.02395765E-1,-1.5360042E-1,3.198988E-1,-6.1272588E-2,7.360436E-2,-9.3126565E-2,1.9027775E-2,4.8570186E-2,1.611383E-1,-1.1913281E-1,-2.5563052E-1,2.0403912E-2,-3.3433423E-3,-4.6243764E-2,-1.2646888E-2,3.7049506E-2,1.6095517E-2,-1.2999433E-1,2.7345827E-2,1.4592828E-2,2.9614307E-2,1.03017795E-4,1.0783287E-1,2.0822385E-2,1.480638E-1,-1.5957874E-1,-4.5564314E-3,-3.9065E-1,-1.8534216E-1,-6.246214E-2,1.2533838E-1,-4.533743E-2,9.177847E-2,2.5322142E-1,-1.4630395E-1,2.2918869E-2,-6.529836E-2,-9.682538E-5,1.5125258E-1,-4.7874853E-2,1.303272E-1,1.2669975E-1,-1.3002592E-1,1.371924E-1,2.134815E-2,-2.0584288E-1,-1.0424374E-1,7.130941E-2,-9.074589E-2,-3.3783787E-1,-3.2873243E-2,-1.2154753E-1,-2.857635E-1,3.8481033E-3,-9.238765E-2,9.249305E-3,-1.4914662E-3,-4.412583E-3,3.425128E-3,1.15688734E-1,-3.266615E-3,3.5777825E-4,2.3806596E-2,-9.294487E-2,-2.3079544E-1,-1.4872979E-1,2.5394387E-3,8.470698E-3,-1.0125259E-2,1.0144756E-1,1.5755817E-2,-6.9717094E-2,7.094101E-2,1.9060145E-1,-2.9888535E-2,9.050476E-2,2.7008712E-1,3.9081164E-2,-3.5303235E-1,1.8254802E-1,7.978682E-2,-7.3868665E-3,-1.7333541E-2,-8.247547E-3,-2.9144012E-3,2.0590168E-3,1.300054E-2,-1.0256515E-3,-9.883833E-3,9.3349046E-4,-1.7080748E-2,5.5017318E-3,-6.76158E-3,-9.530306E-3,-2.4629956E-2,7.870523E-3,-1.2607207E-3,4.8867734E-3,-5.0337603E-3,4.161137E-3,1.02885775E-2,-8.52642E-3,7.726465E-4,-4.5746946E-4,-1.2054747E-2,-8.8999625E-3,-2.184501E-3,-2.022692E-3,2.393292E-3,6.6923546E-3,-8.149516E-3,-4.4535445E-3,4.304989E-3,6.8828417E-3,-3.2486126E-3,1.5161054E-2,6.8829376E-3,-3.4890994E-3,5.4396065E-3,3.3113577E-3,9.40047E-3,1.4731327E-2,-6.4694183E-3,-7.801616E-3,1.31051745E-2,-2.426871E-2,-5.3183828E-3,1.0382343E-2,8.226801E-4,3.0494467E-3,1.5187758E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,39,41,43,-1,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,-1,97,99,101,103,-1,-1,-1,-1,105,-1,-1,-1,107,109,111,-1,113,-1,115,-1,117,119,121,123,125,127,129,131,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2507357E1,3.021964E0,5.8021917E0,2.180605E0,1.1478276E0,2.2620857E0,2.8251915E0,1.8896828E0,4.4343865E-1,6.156642E-1,8.6070347E-1,1.0872927E0,1.8090976E0,1.347095E0,1.4669533E0,1.8992734E0,1.227766E0,0E0,0E0,5.4490614E-1,0E0,3.693861E-1,0E0,1.2058165E0,2.4558291E0,1.3651042E0,0E0,1.6244398E0,9.6336913E-1,0E0,1.2658014E0,7.542305E-1,7.126652E-1,4.866433E-1,5.5464983E-1,3.5158855E-1,1.8636471E-1,1.8707013E-1,1.9533414E-1,4.7576153E-1,7.9005885E-1,0E0,4.9161267E-1,1.115285E0,5.5089164E-1,4.9803475E-1,6.8167865E-1,9.9277663E-1,6.171884E-1,1.0190897E0,0E0,1.2650867E0,4.0241122E-1,3.17038E-1,4.0445212E-1,2.8273916E-1,0E0,2.8555024E-1,5.9163165E-1,2.755157E-1,3.124957E-1,0E0,0E0,0E0,0E0,1.04117334E-1,0E0,0E0,0E0,1.0424588E0,3.1694221E-1,1.0249442E-1,0E0,1.247972E0,0E0,5.7506615E-1,0E0,6.065151E-1,3.1274775E-1,2.556883E-1,1.3156761E-1,3.434503E-1,6.3439035E-1,5.183728E-1,2.0410544E-1,1.2478251E0,6.5665877E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,21,21,23,23,24,24,25,25,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,65,65,69,69,70,70,71,71,73,73,75,75,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,40,42,44,-1,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,-1,98,100,102,104,-1,-1,-1,-1,106,-1,-1,-1,108,110,112,-1,114,-1,116,-1,118,120,122,124,126,128,130,132,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.424835E6,1.1032986E8,8.230524E3,1.2554759E0,7.805608E4,2.9956698E-1,4.0844156E7,8.9712226E-1,1.1855755E8,1.19783E5,2.914E3,1.3664E4,7.06699E8,1.1485E4,2.9348E2,1.0908286E3,2.0403912E-2,-3.3433423E-3,3.0497742E3,-1.2646888E-2,7.204208E8,1.6095517E-2,7E0,3E0,4.5723195E6,2.9614307E-2,1.2677199E7,5.0714142E2,2.0822385E-2,1.6828056E1,1.1E1,1.6837112E7,1.13808104E8,1.0828989E8,9.1233586E4,5.1E1,3.3E1,1E0,4.7E2,4.9114623E0,2.2918869E-2,5.4E2,4.2343444E7,3.0608664E2,4.230839E6,3.63E2,9.3058E4,2.3580047E10,8E0,2.134815E-2,1.15062E5,1.11E3,2.8781365E6,4.1045683E8,4.107E3,-3.2873243E-2,5.684E3,5.2097E4,5.0461445E-2,2.22E2,9.249305E-3,-1.4914662E-3,-4.412583E-3,3.425128E-3,1.5859042E6,-3.266615E-3,3.5777825E-4,2.3806596E-2,3.3151623E1,1E0,7.5773336E7,2.5394387E-3,2.6659616E2,-1.0125259E-2,1.8E1,1.5755817E-2,1.8137958E6,4.0844156E7,1.5804776E4,1.4490348E9,4.1585636E1,5.68E2,4.90027E1,9.627E3,1E0,1.3674345E4,-7.3868665E-3,-1.7333541E-2,-8.247547E-3,-2.9144012E-3,2.0590168E-3,1.300054E-2,-1.0256515E-3,-9.883833E-3,9.3349046E-4,-1.7080748E-2,5.5017318E-3,-6.76158E-3,-9.530306E-3,-2.4629956E-2,7.870523E-3,-1.2607207E-3,4.8867734E-3,-5.0337603E-3,4.161137E-3,1.02885775E-2,-8.52642E-3,7.726465E-4,-4.5746946E-4,-1.2054747E-2,-8.8999625E-3,-2.184501E-3,-2.022692E-3,2.393292E-3,6.6923546E-3,-8.149516E-3,-4.4535445E-3,4.304989E-3,6.8828417E-3,-3.2486126E-3,1.5161054E-2,6.8829376E-3,-3.4890994E-3,5.4396065E-3,3.3113577E-3,9.40047E-3,1.4731327E-2,-6.4694183E-3,-7.801616E-3,1.31051745E-2,-2.426871E-2,-5.3183828E-3,1.0382343E-2,8.226801E-4,3.0494467E-3,1.5187758E-2],"split_indices":[20,32,7,52,39,28,38,45,27,45,10,2,2,7,9,52,4,0,0,52,0,12,0,3,8,28,0,9,58,0,35,3,45,45,45,33,3,3,104,1,56,0,0,45,52,1,0,2,5,17,0,29,2,47,7,12,0,9,10,27,11,0,0,0,0,28,0,0,0,55,79,7,0,52,0,3,0,28,45,47,5,53,3,56,0,6,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E3,8.49E2,1.821E3,5.54E2,2.95E2,9.26E2,8.95E2,5.45E2,9E0,2.05E2,9E1,2.43E2,6.83E2,4.68E2,4.27E2,4.09E2,1.36E2,7E0,2E0,1.92E2,1.3E1,8E1,1E1,1.86E2,5.7E1,6.79E2,4E0,2.58E2,2.1E2,1.8E1,4.09E2,3.02E2,1.07E2,4.5E1,9.1E1,1.76E2,1.6E1,3.2E1,4.8E1,7E0,1.79E2,9E0,4.8E1,6.14E2,6.5E1,1.89E2,6.9E1,1.95E2,1.5E1,3.96E2,1.3E1,1.63E2,1.39E2,5.7E1,5E1,4E1,5E0,5.7E1,3.4E1,5.5E1,1.21E2,1.1E1,5E0,2.3E1,9E0,4.2E1,6E0,4E0,3E0,1.11E2,6.8E1,2.8E1,2E1,5.91E2,2.3E1,5.2E1,1.3E1,1.6E2,2.9E1,5E1,1.9E1,1.57E2,3.8E1,9E0,6E0,2.2E2,1.76E2,1.24E2,3.9E1,5.3E1,8.6E1,5.1E1,6E0,3.2E1,1.8E1,2E0,3.8E1,4E0,5.3E1,2.6E1,8E0,8E0,4.7E1,7E0,1.14E2,3.4E1,8E0,6.2E1,4.9E1,6E0,6.2E1,2E1,8E0,2.66E2,3.25E2,4.6E1,6E0,1.4E2,2E1,1.9E1,1E1,1.2E1,3.8E1,1.5E1,4E0,1.32E2,2.5E1,3.5E1,3E0,5E0,4E0,3E0,3E0,1.82E2,3.8E1,1.66E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[3.6600014E-4,-9.0000324E-2,4.270803E-2,-1.3295661E-1,4.2422896E-4,3.193754E-3,1.15064375E-1,-1.4044258E-1,3.0853623E-1,1.0264827E-2,-1.1420575E-2,-3.3331245E-2,6.570521E-2,1.3753934E-1,-5.5307586E-2,-9.6795894E-2,-2.1410298E-1,4.1101402E-1,-3.236599E-3,-5.0370842E-2,5.1768117E-2,-3.7157886E-2,2.5992054E-2,9.161756E-2,-8.674673E-2,1.2211425E-1,3.3925453E-1,-1.0788301E-1,8.7527215E-2,-1.334115E-1,-2.3067346E-2,-2.541447E-1,-4.5978364E-2,2.3154907E-2,6.1800084E-3,5.571208E-3,-7.219319E-2,1.2606227E-1,9.0148905E-3,-6.9618985E-2,7.272048E-3,-8.873461E-3,1.3406385E-1,-2.985171E-2,-5.9383776E-2,1.6143043E-1,7.096846E-2,1.3421362E-2,2.108347E-2,-8.549321E-2,-2.274505E-2,1.5931176E-1,-5.6743626E-3,-5.3936925E-2,-1.7481942E-1,4.409454E-2,-1.2265409E-1,-2.1020396E-1,-3.7250754E-1,1.6508909E-2,-1.8664296E-1,-1.1447609E-2,-1.3657258E-1,6.673693E-2,2.201066E-1,-1.6391354E-2,1.5746386E-1,-5.577468E-2,-2.825929E-1,1.7293869E-1,-1.6763957E-2,-4.6827372E-2,1.2507924E-1,1.1566035E-1,1.6313186E-2,-1.7846546E-1,1.5709652E-3,7.308828E-2,2.2325622E-1,-1.7197456E-2,1.2861712E-1,-1.05558276E-1,6.333815E-3,2.2027093E-1,-3.7657998E-3,-1.9061929E-2,-1.5208513E-3,-1.034143E-2,-2.8651394E-3,6.4054615E-4,1.0345502E-2,-4.889726E-3,-1.9335803E-2,-2.1350281E-2,-8.8127395E-3,-2.201977E-2,-7.0106233E-3,-2.94504E-3,3.8205395E-3,-1.41563155E-2,-3.9077154E-3,2.49571E-3,-5.391128E-3,-1.077589E-2,-4.4171745E-3,4.8309406E-3,-3.3342168E-3,5.330021E-3,1.7475802E-2,3.8763832E-5,-7.2141523E-3,1.0145234E-2,-2.1568455E-3,8.29233E-3,-3.2024328E-3,-3.326295E-2,-1.0503451E-2,2.2041805E-2,5.686692E-3,-4.7360803E-4,-2.307186E-2,2.8772117E-3,-4.8078657E-3,-8.370184E-4,7.738746E-3,4.77222E-3,1.4328226E-2,-1.075212E-2,3.513805E-3,7.967701E-3,-2.0161609E-3,4.3595345E-3,-6.817637E-3,2.1918938E-2,9.661977E-3,-7.5876964E-3,1.9145287E-3,7.902323E-3,-2.665776E-3,-6.3729016E-3,2.6590808E-3,1.4639388E-2,3.3367507E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,-1,-1,61,63,65,67,69,71,73,-1,75,77,79,-1,-1,81,-1,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,-1,127,129,131,133,135,137,139,-1,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0277443E1,3.3289032E0,5.228943E0,1.9635372E0,6.536434E-1,2.7055357E0,2.481207E0,1.8182745E0,4.132166E-1,6.745343E-1,0E0,1.6690674E0,1.7342669E0,1.7352247E0,5.8017224E-1,9.7148395E-1,1.4154978E0,1.0111022E-1,0E0,4.049241E-1,5.0389755E-1,1.073536E0,0E0,1.5964761E0,9.026681E-1,1.05832E0,1.1496925E-1,4.1962034E-1,3.2943618E-1,7.836714E-1,8.147088E-1,8.1105995E-1,3.7225068E-1,0E0,0E0,0E0,3.7778062E-1,3.08905E-1,3.8727716E-1,1.2520828E0,1.2572178E0,5.752683E-1,9.604316E-1,0E0,4.5031315E-1,1.6222291E0,1.1864686E0,0E0,0E0,2.4989507E-1,0E0,2.4939978E-1,0E0,6.284517E-1,7.341747E-1,3.8266414E-1,2.4185264E-1,7.015004E-1,8.208599E-1,1.5295993E-1,1.2466821E-1,3.3366898E-1,1.6354054E-1,1.792569E-1,3.0602574E-1,2.0332743E-1,1.7471549E-1,1.040269E0,5.253737E-1,5.681915E-1,8.77515E-1,5.09962E-1,1.3295439E-1,6.8426704E-1,0E0,2.600069E-1,3.092768E-1,4.961785E-1,7.9495525E-1,7.573457E-1,9.5475316E-1,2.2898966E-1,0E0,1.4591998E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,49,49,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,-1,-1,62,64,66,68,70,72,74,-1,76,78,80,-1,-1,82,-1,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,-1,128,130,132,134,136,138,140,-1,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.0281758E6,9.772148E4,8.230524E3,1.1855755E8,4.5168175E5,1E0,6.856226E-1,8.9712226E-1,4.4844016E2,-1.1420575E-2,2.03148E2,3.7832818E0,7.096131E0,4.760537E8,2.914E3,6.654321E0,1.2003743E7,-3.236599E-3,1.8530114E-2,9.912942E6,3.131E3,2.5992054E-2,1E0,1.198E5,1.3226592E-2,1.7314173E1,1.566593E6,2.6878788E1,4.854E4,3.653602E7,1E0,1.1E1,2.3154907E-2,6.1800084E-3,5.571208E-3,5.650399E6,1.6525185E9,5.528107E3,4.004366E1,1.03E2,1.408E3,8.5023944E2,-2.985171E-2,1.6437647E-4,5.862126E2,4.894913E6,1.3421362E-2,2.108347E-2,4E1,-2.274505E-2,2.9375E0,-5.6743626E-3,1.04661034E-4,2E1,1.1598511E3,2.6572757E10,4.36E2,1.4510472E1,9.159699E7,5.37456E5,7.7854166E0,1.998E3,1.0931163E9,1.6594656E3,9.36494E5,1.766486E7,7E0,5.4210526E1,1E0,7.074777E6,2.294901E0,4.8858527E9,6.057671E6,1.6313186E-2,4.423676E1,5.6656016E3,1E0,1.0918E4,7.5935423E-1,2.6025698E7,1.5088776E0,6.333815E-3,7.1913286E8,-3.7657998E-3,-1.9061929E-2,-1.5208513E-3,-1.034143E-2,-2.8651394E-3,6.4054615E-4,1.0345502E-2,-4.889726E-3,-1.9335803E-2,-2.1350281E-2,-8.8127395E-3,-2.201977E-2,-7.0106233E-3,-2.94504E-3,3.8205395E-3,-1.41563155E-2,-3.9077154E-3,2.49571E-3,-5.391128E-3,-1.077589E-2,-4.4171745E-3,4.8309406E-3,-3.3342168E-3,5.330021E-3,1.7475802E-2,3.8763832E-5,-7.2141523E-3,1.0145234E-2,-2.1568455E-3,8.29233E-3,-3.2024328E-3,-3.326295E-2,-1.0503451E-2,2.2041805E-2,5.686692E-3,-4.7360803E-4,-2.307186E-2,2.8772117E-3,-4.8078657E-3,-8.370184E-4,7.738746E-3,4.77222E-3,1.4328226E-2,-1.075212E-2,3.513805E-3,7.967701E-3,-2.0161609E-3,4.3595345E-3,-6.817637E-3,2.1918938E-2,9.661977E-3,-7.5876964E-3,1.9145287E-3,7.902323E-3,-2.665776E-3,-6.3729016E-3,2.6590808E-3,1.4639388E-2,3.3367507E-3],"split_indices":[20,32,33,52,45,28,14,27,27,52,0,56,53,35,7,2,54,1,0,39,45,2,0,102,7,57,35,9,56,29,45,101,8,0,0,0,45,5,52,53,11,0,52,0,38,52,1,0,0,3,0,54,0,41,3,52,12,2,57,7,28,54,10,5,4,9,47,3,53,8,47,56,5,28,0,56,47,65,9,42,43,38,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.684E3,8.56E2,1.828E3,5.8E2,2.76E2,1.183E3,6.45E2,5.71E2,9E0,2.66E2,1E1,7.47E2,4.36E2,5.7E2,7.5E1,3.6E2,2.11E2,7E0,2E0,1.08E2,1.58E2,7.43E2,4E0,3.73E2,6.3E1,5.31E2,3.9E1,5.5E1,2E1,2.4E2,1.2E2,1.7E2,4.1E1,5E0,2E0,1.2E1,9.6E1,5.7E1,1.01E2,4.29E2,3.14E2,1.11E2,2.62E2,2E0,6.1E1,2.99E2,2.32E2,2.7E1,1.2E1,5.3E1,2E0,1.5E1,5E0,8.3E1,1.57E2,7.2E1,4.8E1,1.26E2,4.4E1,2.9E1,1.2E1,5E1,4.6E1,3.6E1,2.1E1,8.7E1,1.4E1,4.04E2,2.5E1,3.9E1,2.75E2,8.7E1,2.4E1,2.42E2,2E1,2E1,4.1E1,1.24E2,1.75E2,9.2E1,1.4E2,4.9E1,4E0,1.2E1,3E0,4E0,7.9E1,1.15E2,4.2E1,6.2E1,1E1,4.6E1,2E0,1.1E1,1.15E2,3.1E1,1.3E1,1.3E1,1.6E1,5E0,7E0,3.1E1,1.9E1,1.4E1,3.2E1,2.9E1,7E0,1.3E1,8E0,7.8E1,9E0,1.1E1,3E0,1.8E1,3.86E2,2E0,2.3E1,5E0,3.4E1,2.72E2,3E0,2.9E1,5.8E1,5E0,1.9E1,2.24E2,1.8E1,1.7E1,3E0,8E0,3.3E1,1.15E2,9E0,1.3E1,1.62E2,2.6E1,6.6E1,1.17E2,2.3E1,4.2E1,7E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[2.3761154E-3,3.6150422E-2,-9.07322E-2,-7.7931467E-3,1.116759E-1,-1.24647185E-1,6.4713754E-2,4.9105242E-2,-4.7877047E-2,3.223874E-2,1.820868E-1,-1.6664198E-1,-3.599859E-2,1.19691126E-1,-3.2581102E-2,6.4705454E-2,-3.386753E-1,-6.3339174E-2,4.550474E-2,4.0242793E-3,1.9204178E-1,2.9948002E-1,1.4696755E-1,-1.437786E-1,-3.6653155E-1,-1.3633028E-1,-9.846709E-3,1.6602917E-2,7.9640225E-2,-1.2695941E-1,3.5995588E-2,9.088688E-2,-6.4166844E-2,3.112114E-3,-5.163138E-1,-8.714618E-2,-1.3419466E-2,1.4166014E-1,-2.4599224E-2,1.8645272E-2,-9.702027E-3,2.4595402E-1,-1.2848076E-1,3.3136705E-1,-1.3692562E-2,1.2962659E-1,3.380972E-1,-1.3445573E-1,-4.248021E-1,-4.2190802E-1,8.4222294E-4,-8.5769184E-2,-1.163478E-2,-6.1884284E-2,2.2366336E-2,-1.0151187E-2,9.1164574E-2,-1.1953857E-2,-1.9978646E-3,7.144044E-2,-9.316321E-3,6.671786E-2,1.9161342E-1,-9.556616E-2,6.520495E-2,-3.2015383E-2,1.4994002E-3,-5.1636282E-2,-1.5779974E-1,-5.6298796E-2,6.709279E-2,1.204933E-2,6.3119225E-2,-1.003543E-2,7.2715334E-3,-8.751904E-2,3.5818603E-2,-7.8382984E-2,3.0943352E-1,-1.805706E-2,8.2451233E-4,2.2330992E-1,4.420784E-1,-1.693609E-2,1.33956E-1,1.5175901E-1,1.9973706E-2,2.0520618E-2,-4.3625194E-3,-1.8730003E-1,-1.0073802E-1,-3.010783E-2,-2.6058385E-3,-3.0291292E-1,-5.9998333E-1,-1.0750909E-1,3.4264494E-3,-4.51819E-2,-1.1009709E-2,4.31852E-2,-3.9186296E-3,9.627861E-3,7.1332775E-2,1.4402388E-2,1.8402552E-3,-1.7571129E-2,3.4371847E-3,7.0657423E-3,1.5470044E-2,-1.1654082E-3,-7.953744E-3,5.774435E-4,9.115826E-3,1.5161066E-2,-3.041008E-3,-1.2441723E-2,-5.137853E-3,2.1789765E-3,-3.8448637E-3,6.6124606E-3,-8.933309E-5,5.491202E-3,-3.5260103E-3,2.2230586E-3,-9.064561E-3,-5.696863E-3,2.4167304E-3,2.1423101E-3,-1.3485144E-2,4.403998E-3,-1.3012604E-2,1.04654785E-2,3.198155E-2,1.2918087E-2,-4.800513E-3,2.3689631E-2,1.3741009E-2,-6.599679E-4,1.1886611E-2,5.723315E-3,1.3934731E-2,-9.358705E-3,2.1337545E-3,-7.6022935E-3,-1.5586693E-2,9.958256E-4,-7.204988E-3,-4.011593E-3,-1.5880343E-2,-3.307813E-2,-1.5166525E-2,-2.6509203E-3,-1.0125017E-2,-7.1334117E-3,-1.1444285E-3,3.2335336E-3,-9.098512E-4,4.266838E-3,-2.6465668E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,55,57,59,61,63,-1,65,67,69,71,73,-1,75,77,79,81,83,85,87,89,91,93,-1,95,-1,97,99,-1,101,-1,-1,103,-1,105,107,109,111,-1,-1,113,115,117,119,-1,121,-1,123,125,127,129,131,-1,-1,133,135,-1,137,139,141,-1,-1,143,145,-1,-1,147,149,151,-1,153,-1,155,-1,-1,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.399546E0,6.5058346E0,3.756421E0,2.8304305E0,4.0268564E0,2.1681032E0,6.885736E-1,3.1252866E0,1.0540278E0,1.5322044E0,1.5308809E0,1.7570801E0,4.9456757E-1,7.178763E-1,3.0876112E-1,1.6726179E0,1.4676602E0,7.4160457E-1,7.0453566E-1,1.5375407E0,9.128847E-1,8.7333155E-1,9.372268E-1,8.7820435E-1,8.5826397E-1,1.8939507E-1,2.5450376E-1,0E0,2.5792474E-1,1.9868335E-1,2.426877E-1,9.877796E-1,3.4745502E-1,0E0,1.1877887E0,1.0531168E0,7.045966E-1,3.640545E-1,3.6108822E-1,0E0,9.985154E-1,9.301598E-1,2.962169E-1,8.463688E-1,4.9910942E-1,6.5950775E-1,9.701314E-1,5.9941196E-1,7.9644954E-1,5.599108E-1,0E0,1.0345341E-1,0E0,1.5236966E-1,2.0743275E-1,0E0,1.360107E-1,0E0,0E0,1.7769355E-1,0E0,7.560817E-1,4.0990996E-1,3.375169E-1,1.1077152E-1,0E0,0E0,1.2562664E0,6.986232E-1,3.2839704E-1,3.4778592E-1,0E0,1.987432E-1,0E0,4.1659343E-1,4.571284E-1,5.1626354E-1,2.8844914E-1,1.0594347E0,0E0,0E0,6.580069E-1,1.6809177E-1,0E0,1.2157057E-1,9.843831E-1,2.575524E-1,0E0,0E0,4.770398E-1,1.2959499E0,0E0,0E0,1.1680055E-1,1.1536217E-1,1.2378484E-1,0E0,1.15714125E-1,0E0,1.2082423E-1,0E0,0E0,1.3984582E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,56,56,59,59,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,72,72,74,74,75,75,76,76,77,77,78,78,81,81,82,82,84,84,85,85,86,86,89,89,90,90,93,93,94,94,95,95,97,97,99,99,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,56,58,60,62,64,-1,66,68,70,72,74,-1,76,78,80,82,84,86,88,90,92,94,-1,96,-1,98,100,-1,102,-1,-1,104,-1,106,108,110,112,-1,-1,114,116,118,120,-1,122,-1,124,126,128,130,132,-1,-1,134,136,-1,138,140,142,-1,-1,144,146,-1,-1,148,150,152,-1,154,-1,156,-1,-1,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,6.218596E2,2.0601392E3,2E0,1.5166431E5,1E0,6.971004E7,9.669789E0,9.750871E6,3.317425E3,1.0993947E1,1.0017953E8,2.067695E4,8.628E3,3.4570816E0,1E0,3.1E1,2.373E3,4.1E2,2.04E5,2.3580047E10,2.0295714E7,6.7652373E3,4.5723195E6,1.0052E4,1.48614E5,6.8177136E2,1.6602917E-2,9.877E3,3.0497742E3,2.12E2,7.0025E4,1.75E2,3.112114E-3,6.743895E5,2.2415E4,2.1924436E5,7.73756E0,1.322E3,1.8645272E-2,5.794E3,5.0279167E1,9.627E3,1.0201212E3,1.0855529E3,2.2807484E7,1.1896622E6,1.445E3,3E0,1.2905E4,8.4222294E-4,6.1E1,-1.163478E-2,1.9719212E1,3.0996E4,-1.0151187E-2,7.016421E5,-1.1953857E-2,-1.9978646E-3,5.491E3,-9.316321E-3,1.198E5,4.681779E2,2.20524E5,6.211896E0,-3.2015383E-2,1.4994002E-3,6E0,2.4063041E4,9.7549E4,4.7022E4,1.204933E-2,3.271242E0,-1.003543E-2,1.5548611E7,7.704183E-1,8.6206274E2,3.7137984E7,1.5005797E4,-1.805706E-2,8.2451233E-4,2E0,2.1298597E0,-1.693609E-2,1.0215407E1,5.3515047E-1,4.709E3,2.0520618E-2,-4.3625194E-3,1.087037E1,1.1662405E9,-3.010783E-2,-2.6058385E-3,1.4977E5,1.2659678E3,1.17E2,3.4264494E-3,1.009E3,-1.1009709E-2,7.2530017E0,-3.9186296E-3,9.627861E-3,1E0,1.4402388E-2,1.8402552E-3,-1.7571129E-2,3.4371847E-3,7.0657423E-3,1.5470044E-2,-1.1654082E-3,-7.953744E-3,5.774435E-4,9.115826E-3,1.5161066E-2,-3.041008E-3,-1.2441723E-2,-5.137853E-3,2.1789765E-3,-3.8448637E-3,6.6124606E-3,-8.933309E-5,5.491202E-3,-3.5260103E-3,2.2230586E-3,-9.064561E-3,-5.696863E-3,2.4167304E-3,2.1423101E-3,-1.3485144E-2,4.403998E-3,-1.3012604E-2,1.04654785E-2,3.198155E-2,1.2918087E-2,-4.800513E-3,2.3689631E-2,1.3741009E-2,-6.599679E-4,1.1886611E-2,5.723315E-3,1.3934731E-2,-9.358705E-3,2.1337545E-3,-7.6022935E-3,-1.5586693E-2,9.958256E-4,-7.204988E-3,-4.011593E-3,-1.5880343E-2,-3.307813E-2,-1.5166525E-2,-2.6509203E-3,-1.0125017E-2,-7.1334117E-3,-1.1444285E-3,3.2335336E-3,-9.098512E-4,4.266838E-3,-2.6465668E-3],"split_indices":[27,52,52,17,28,59,45,54,9,52,56,45,48,9,57,14,0,2,0,5,5,50,52,28,2,12,4,0,9,52,8,1,6,0,28,29,28,58,2,0,2,56,0,4,52,51,28,2,8,10,0,3,0,56,9,0,32,0,0,9,0,7,55,9,54,0,0,3,33,12,9,0,53,0,45,41,58,1,4,0,0,17,38,0,56,35,2,0,0,58,5,0,0,29,52,10,0,2,0,53,0,0,104,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.669E3,1.959E3,7.1E2,1.239E3,7.2E2,5.83E2,1.27E2,5.12E2,7.27E2,3.39E2,3.81E2,3.95E2,1.88E2,8.1E1,4.6E1,4.93E2,1.9E1,6.24E2,1.03E2,2.89E2,5E1,8.6E1,2.95E2,3.56E2,3.9E1,3.8E1,1.5E2,1.1E1,7E1,1.9E1,2.7E1,4.1E2,8.3E1,6E0,1.3E1,4.22E2,2.02E2,4.3E1,6E1,9E0,2.8E2,4.3E1,7E0,7.8E1,8E0,2.72E2,2.3E1,3.46E2,1E1,3.4E1,5E0,2.7E1,1.1E1,5.7E1,9.3E1,2E0,6.8E1,7E0,1.2E1,2.4E1,3E0,3.32E2,7.8E1,6.7E1,1.6E1,1E1,3E0,2.82E2,1.4E2,1.32E2,7E1,1.7E1,2.6E1,8E0,5.2E1,1.03E2,1.77E2,7E0,3.6E1,2E0,5E0,4.1E1,3.7E1,2E0,6E0,2.26E2,4.6E1,1.9E1,4E0,1.33E2,2.13E2,6E0,4E0,2.2E1,1.2E1,2.4E1,3E0,5.3E1,4E0,7.8E1,1.5E1,9E0,5.9E1,2E0,2.2E1,3E0,3.29E2,6E1,1.8E1,3.4E1,3.3E1,1.2E1,4E0,8E0,2.74E2,4.5E1,9.5E1,2.5E1,1.07E2,3.4E1,3.6E1,1.9E1,7E0,4.4E1,8E0,8.4E1,1.9E1,1.73E2,4E0,4E0,3E0,3E1,6E0,3.6E1,5E0,2.6E1,1.1E1,3E0,3E0,1.85E2,4.1E1,4E0,4.2E1,1.12E2,2.1E1,6.2E1,1.51E2,3E0,1.9E1,8E0,4E0,1.7E1,7E0,8E0,4.5E1,5.6E1,2.2E1,5.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[-2.689658E-3,-8.5344225E-2,3.592615E-2,-1.1440159E-1,2.2573067E-2,1.7378945E-4,1.0457496E-1,-6.312117E-2,-1.97686E-1,8.375912E-2,-1.4998297E-1,-4.9539793E-2,4.019781E-2,7.0687994E-2,1.81382E-1,-1.06992304E-1,3.0657876E-2,-7.701161E-2,-2.5411296E-1,4.4086424E-1,6.078568E-2,2.4764931E-3,-1.978538E-1,-3.7179343E-2,-2.5009495E-1,2.8207088E-2,2.4958497E-1,1.0112072E-1,-5.4341935E-2,6.739604E-2,2.1707827E-1,-1.1684477E-1,2.1859212E-3,4.5795154E-2,-7.7402536E-3,-9.852361E-2,1.10811956E-1,-2.874993E-1,-8.42752E-2,6.7936885E-3,2.4754122E-2,-1.0197091E-2,1.3192779E-1,-2.5078878E-1,-4.0311243E-2,-3.3013564E-2,-2.1494266E-2,-1.3010123E-1,-4.890716E-1,-1.0608999E-1,4.5123253E-2,2.261786E-2,1.3270679E-1,1.3858387E-2,1.475493E-1,-2.2095827E-2,-3.4070827E-2,1.9485065E-1,1.2932489E-2,1.9979894E-1,2.337115E-2,-8.573453E-2,-2.2559574E-1,5.3540813E-3,1.7841082E-2,-1.4515683E-1,1.3219172E-2,1.2624245E-2,-4.5589724E-4,-2.628267E-1,-4.9959093E-1,-8.308785E-3,1.6511317E-2,7.0440575E-2,-8.579145E-2,2.8853709E-1,1.01054676E-1,2.0844187E-3,-2.8642204E-1,-5.21243E-3,5.11591E-3,-6.0623802E-2,4.504721E-2,-1.068303E-2,4.8364103E-3,-2.8578794E-2,-1.862889E-3,2.9247839E-3,-1.4821883E-1,7.1820416E-3,9.90604E-2,8.438104E-3,-3.66304E-3,-2.6223424E-1,3.7955474E-2,1.8579046E-1,7.113639E-2,7.04476E-2,-6.836949E-2,2.3692087E-3,3.3316264E-1,1.0490476E-1,-3.5749655E-3,2.0932716E-1,-1.430713E-2,-7.178461E-3,-2.4162561E-3,-1.2608592E-2,-7.0005615E-4,1.2461193E-3,-1.0789694E-2,-1.1479266E-2,-4.4947234E-3,7.237082E-3,-5.003632E-3,-1.7370151E-2,-9.158028E-3,-2.8337546E-2,-1.0316811E-2,-3.5628073E-3,7.914851E-3,7.6364656E-3,7.767811E-4,-7.523199E-3,7.181083E-4,1.7752256E-2,-1.2786944E-3,2.0164715E-3,7.910838E-3,-1.6982008E-2,-8.492452E-3,9.469461E-3,-3.4136088E-3,4.8681027E-3,-1.077626E-3,-1.2106488E-2,-3.3543E-3,-1.3995846E-2,8.3427527E-4,1.0073852E-2,2.4727145E-3,-5.833648E-3,-2.1187464E-2,-1.01687275E-2,2.6969023E-3,2.510672E-2,7.941946E-3,1.5698733E-3,1.1017025E-2,-4.602676E-5,1.1930944E-2,-4.1452902E-3,5.6914943E-3,2.4354674E-3,2.0876192E-2,4.5670517E-4,9.032969E-3,7.4701076E-3,1.225837E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,59,61,-1,63,-1,65,67,69,71,-1,-1,73,75,77,79,81,-1,83,85,87,89,-1,91,93,95,-1,97,99,101,103,-1,105,107,-1,109,111,113,-1,-1,115,117,-1,119,121,123,125,127,-1,129,-1,-1,131,133,-1,-1,-1,-1,-1,135,137,139,-1,-1,141,143,145,147,149,151,-1,153,155,-1,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.560209E0,2.6805487E0,4.4865465E0,2.8618097E0,1.9341229E0,2.3956456E0,1.6165047E0,1.7230127E0,1.723608E0,1.0807326E0,4.736848E-1,1.3218992E0,1.6675988E0,1.6644757E0,7.625246E-1,4.3392944E-1,3.9742497E-1,3.4627593E-1,9.623966E-1,1.0853076E-1,6.477278E-1,0E0,3.1513143E-1,8.5458976E-1,8.253176E-1,1.4405594E0,8.743501E-1,1.4200201E0,6.9387937E-1,3.2373673E-1,5.8132076E-1,8.873749E-1,0E0,2.2863752E-1,0E0,3.9273828E-1,1.6798317E-1,6.2745667E-1,2.7263033E-1,0E0,0E0,4.0226716E-1,2.8028297E-1,3.165995E-1,1.2254145E-1,1.086835E0,0E0,4.9518853E-1,4.6153688E-1,5.0905895E-1,1.1493883E0,0E0,2.4091762E-1,8.2827574E-1,6.5594864E-1,0E0,3.023584E-1,2.650258E-1,2.817024E-1,7.1807003E-1,0E0,4.6778595E-1,4.6522617E-1,0E0,1.8285318E-1,2.3456502E-1,3.890917E-1,0E0,0E0,8.745918E-1,2.3902893E-1,0E0,2.1514085E-1,1.5186197E-1,2.466819E-1,2.6885527E-1,2.0096982E-1,0E0,1.3049841E-1,0E0,0E0,1.0381832E0,5.069903E-1,0E0,0E0,0E0,0E0,0E0,4.4829988E-1,1.0198298E0,1.2137032E0,0E0,0E0,1.9912136E-1,5.357752E-1,9.3967247E-1,4.678789E-1,2.6818764E-1,2.2480538E-1,0E0,1.8088567E-1,1.3392068E-1,0E0,2.9843283E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,53,53,54,54,56,56,57,57,58,58,59,59,61,61,62,62,64,64,65,65,66,66,69,69,70,70,72,72,73,73,74,74,75,75,76,76,78,78,81,81,82,82,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,100,100,101,101,103,103],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,60,62,-1,64,-1,66,68,70,72,-1,-1,74,76,78,80,82,-1,84,86,88,90,-1,92,94,96,-1,98,100,102,104,-1,106,108,-1,110,112,114,-1,-1,116,118,-1,120,122,124,126,128,-1,130,-1,-1,132,134,-1,-1,-1,-1,-1,136,138,140,-1,-1,142,144,146,148,150,152,-1,154,156,-1,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.0485785E3,9.772148E4,1.4540612E7,9.302862E7,1.7669278E5,1.4101E4,2.785857E2,2.6432162E5,3.541262E-5,3.046556E5,1E0,9.0754684E1,1.4274633E7,1E0,4.653159E9,1.044E1,5.189068E-1,6.568389E0,7.678E3,1E0,2.4764931E-3,8.2619705E1,7.074777E6,7.492813E0,1E0,1.8631586E4,1E0,5.1776E4,5.7075036E-1,4.2620764E2,1.0508E5,2.1859212E-3,4.1730586E-1,-7.7402536E-3,3.138649E6,9.8708276E2,1.2427474E8,9.538462E0,6.7936885E-3,2.4754122E-2,4.5158855E6,5.015707E0,1.7560976E0,4.73699E0,5.513E3,-2.1494266E-2,1.6812695E-2,1.752E3,1.6254681E0,1.146E3,2.261786E-2,4.8718534E0,4.97E2,4.4E0,-2.2095827E-2,3.53562E5,4.1043E4,2.589369E7,6.13912E5,2.337115E-2,9E2,3.2E1,5.3540813E-3,1E0,1.3E1,5.904E3,1.2624245E-2,-4.5589724E-4,3.0608664E2,4.822581E0,-8.308785E-3,9.3299186E-1,5.12861E5,8.454452E6,2.9956698E-1,3.368326E11,2.0844187E-3,5.2768206E5,-5.21243E-3,5.11591E-3,7E0,5.244755E-3,-1.068303E-2,4.8364103E-3,-2.8578794E-2,-1.862889E-3,2.9247839E-3,2.7754388E-4,1.198E5,3.1753032E0,8.438104E-3,-3.66304E-3,5.266129E0,1.5988182E-1,1.0918E4,1.00663277E9,5.012E3,1.7022566E7,2.3692087E-3,8.422367E6,1.7313588E7,-3.5749655E-3,2.1446484E-1,-1.430713E-2,-7.178461E-3,-2.4162561E-3,-1.2608592E-2,-7.0005615E-4,1.2461193E-3,-1.0789694E-2,-1.1479266E-2,-4.4947234E-3,7.237082E-3,-5.003632E-3,-1.7370151E-2,-9.158028E-3,-2.8337546E-2,-1.0316811E-2,-3.5628073E-3,7.914851E-3,7.6364656E-3,7.767811E-4,-7.523199E-3,7.181083E-4,1.7752256E-2,-1.2786944E-3,2.0164715E-3,7.910838E-3,-1.6982008E-2,-8.492452E-3,9.469461E-3,-3.4136088E-3,4.8681027E-3,-1.077626E-3,-1.2106488E-2,-3.3543E-3,-1.3995846E-2,8.3427527E-4,1.0073852E-2,2.4727145E-3,-5.833648E-3,-2.1187464E-2,-1.01687275E-2,2.6969023E-3,2.510672E-2,7.941946E-3,1.5698733E-3,1.1017025E-2,-4.602676E-5,1.1930944E-2,-4.1452902E-3,5.6914943E-3,2.4354674E-3,2.0876192E-2,4.5670517E-4,9.032969E-3,7.4701076E-3,1.225837E-2],"split_indices":[20,52,33,45,45,28,2,52,28,39,32,6,56,45,102,5,54,27,54,2,102,0,56,47,54,66,33,100,1,41,58,29,0,39,0,29,4,45,58,0,0,47,54,56,57,2,0,38,29,53,2,0,53,2,54,0,1,10,45,2,0,2,3,0,103,3,0,0,0,52,56,0,27,29,32,38,31,0,28,0,0,3,57,0,0,0,0,0,41,7,54,0,0,54,38,9,7,2,9,0,50,1,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E3,8.53E2,1.827E3,6.72E2,1.81E2,1.202E3,6.25E2,4.17E2,2.55E2,1.34E2,4.7E1,5.36E2,6.66E2,4.35E2,1.9E2,2.84E2,1.33E2,8.2E1,1.73E2,7E0,1.27E2,9E0,3.8E1,5.06E2,3E1,6.31E2,3.5E1,3.5E2,8.5E1,4.6E1,1.44E2,2.67E2,1.7E1,1.24E2,9E0,7.4E1,8E0,1.44E2,2.9E1,2E0,5E0,6.4E1,6.3E1,2.8E1,1E1,5.02E2,4E0,2.1E1,9E0,7E1,5.61E2,1.1E1,2.4E1,1.22E2,2.28E2,3E0,8.2E1,1.3E1,3.3E1,1.37E2,7E0,2.09E2,5.8E1,3.6E1,8.8E1,5.2E1,2.2E1,3E0,5E0,1.31E2,1.3E1,1.5E1,1.4E1,3.1E1,3.3E1,9E0,5.4E1,3E0,2.5E1,7E0,3E0,3.71E2,1.31E2,1.5E1,6E0,7E0,2E0,1.4E1,5.6E1,3.3E2,2.31E2,2E1,4E0,9E0,1.13E2,1.51E2,7.7E1,2E1,6.2E1,7E0,6E0,1.6E1,1.7E1,1.35E2,2E0,7.3E1,1.36E2,4.9E1,9E0,8.6E1,2E0,1.7E1,3.5E1,1E1,1.2E1,5.3E1,7.8E1,9E0,4E0,9E0,5E0,1.1E1,2E1,1.9E1,1.4E1,7E0,2E0,2.9E1,2.5E1,1.4E1,1.1E1,1.4E1,3.57E2,7.1E1,6E1,2.3E1,3.3E1,1E1,3.2E2,6.8E1,1.63E2,6E0,3E0,7E0,1.06E2,7E0,1.44E2,6.3E1,1.4E1,1.5E1,5E0,5.7E1,5E0,2E0,4E0,8E0,8E0,6.5E1,7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[5.0597626E-4,-8.41867E-2,4.101294E-2,-1.1360305E-1,2.9799454E-2,-8.410059E-2,6.045761E-2,-8.526692E-2,-2.5281742E-1,1.850927E-1,-2.4543645E-2,-7.097548E-2,-3.2975057E-1,8.762305E-2,-1.4870629E-2,-1.3224685E-1,-2.2598581E-2,-3.4586138E-1,-1.759242E-1,3.1965503E-1,8.717209E-2,-6.1389055E-2,1.5274066E-1,-1.0346897E-2,-5.467041E-2,-6.0777594E-3,-2.817622E-2,4.2938143E-2,1.4479825E-1,-5.7743948E-2,9.808434E-2,-8.080971E-2,-1.8225032E-1,-6.9750585E-2,1.040108E-2,-3.7848467E-1,-1.8249024E-3,-8.110903E-2,-2.5005007E-1,4.767929E-3,1.9806543E-2,1.228576E-3,1.0257844E-2,-4.8832316E-2,-1.8291049E-2,2.1164343E-3,1.190366E-2,-1.1119661E-2,-1.1326558E-1,-4.7590703E-2,7.5056955E-2,4.1716534E-1,1.3260593E-1,-2.9200855E-1,-4.1755397E-2,1.6251493E-1,-8.11238E-3,-5.8081806E-2,-3.3683863E-1,-2.1987046E-1,-7.465284E-2,-3.9180946E-2,-8.82817E-3,4.5421574E-2,-1.0526645E-1,-2.2318123E-1,-4.464321E-1,3.5225567E-3,-1.1498388E-1,-2.8802133E-1,6.0210535E-3,8.524108E-2,-6.9931984E-2,-6.965486E-2,6.789155E-2,-1.6562076E-1,-2.370217E-2,2.1726191E-1,-8.663913E-2,1.16041325E-1,1.0753073E-2,2.1997621E-2,9.116828E-3,1.568273E-1,2.9867036E-2,-3.035723E-2,-1.8828817E-1,-5.136941E-2,1.21288165E-1,1.3192953E-1,3.0737953E-2,-3.0320534E-2,1.0610115E-2,-4.6806918E-3,4.8982824E-4,-1.9385861E-2,1.1788341E-3,-1.3438413E-2,-6.5442445E-3,-5.1362077E-3,1.5748915E-3,7.2384044E-3,-2.5712966E-3,3.8240654E-3,-2.6755857E-3,-3.3317952E-4,-8.320269E-3,-1.6456159E-2,-6.570487E-3,-1.1288314E-3,-2.2737572E-2,-6.4813127E-3,3.8014012E-3,-1.0792331E-2,-2.550001E-2,7.62949E-3,-1.0616846E-3,-1.6603108E-3,-9.430072E-3,-9.605686E-4,-7.572817E-3,8.026952E-3,1.0983687E-3,2.2114484E-4,-8.563059E-3,-1.9437693E-3,7.868962E-3,1.6927205E-2,3.5588413E-3,4.759963E-3,-5.4982947E-3,6.0119973E-3,-9.307976E-3,1.093448E-2,-5.300097E-4,1.6704813E-2,6.6113416E-3,-1.6920137E-3,8.043709E-3,-5.221398E-3,-2.3725323E-2,-2.7548042E-3,1.081828E-2,1.1650656E-2,-5.870055E-3,3.512497E-3,9.020546E-3,-2.810305E-4,-1.6191516E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,-1,-1,49,51,53,55,57,59,61,63,65,-1,67,69,-1,-1,-1,-1,71,-1,-1,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,107,109,-1,111,113,-1,115,117,119,121,123,125,127,129,131,133,-1,-1,135,137,-1,139,141,143,145,-1,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.108379E0,2.8836598E0,4.374378E0,2.672739E0,1.4982665E0,7.56848E-1,3.1833372E0,1.6722355E0,7.6191664E-1,5.782504E-1,8.716154E-1,5.34619E-1,5.547795E-1,2.9139977E0,2.005721E0,8.211837E-1,3.8194552E-1,4.976182E-1,4.365554E-1,3.516004E-1,2.1330097E-1,4.2409977E-1,2.2823656E-1,0E0,5.3208E-1,0E0,0E0,1.87321E0,1.6083956E0,1.1114726E0,7.813425E-1,9.2311394E-1,6.474304E-1,3.4657785E-1,5.921378E-1,3.8577843E-1,0E0,1.656335E-1,5.4988337E-1,0E0,0E0,0E0,0E0,3.0865377E-1,0E0,0E0,0E0,5.643464E-1,4.1340506E-1,1.7666345E0,1.250593E0,1.1338353E-1,1.1922731E0,5.5438364E-1,4.4826642E-1,9.562019E-1,2.3113343E-1,4.0732902E-1,3.292855E-1,5.732508E-1,1.5776089E-1,2.3180953E-1,0E0,3.9503655E-1,2.2764912E-1,1.1854774E-1,3.604412E-1,0E0,1.0963377E-1,3.855195E-1,0E0,1.228696E-1,4.0959027E-1,3.0398896E-1,2.2927277E-1,1.2651408E-1,1.1460947E-1,4.0205097E-1,7.870209E-1,8.7909484E-1,8.855434E-1,0E0,0E0,1.3540936E0,8.432495E-1,0E0,3.35465E-1,4.6853864E-1,5.0579023E-1,2.1098292E-1,0E0,3.0657342E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,43,43,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,68,68,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84,86,86,87,87,88,88,89,89,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,-1,-1,50,52,54,56,58,60,62,64,66,-1,68,70,-1,-1,-1,-1,72,-1,-1,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,108,110,-1,112,114,-1,116,118,120,122,124,126,128,130,132,134,-1,-1,136,138,-1,140,142,144,146,-1,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,4.97E2,3.9712732E7,2.2668628E7,6.1313953E1,9.070543E0,2.074E3,9.966725E2,4.3800272E8,4.9386E4,8.333333E0,5.5E1,2.979E3,7.2456576E8,4.61E2,2.1592189E5,2.262586E6,6.971004E7,1E0,7.492813E0,2.6572757E10,3.5E0,-1.0346897E-2,8.836364E0,-6.0777594E-3,-2.817622E-2,1E0,6.8E1,6.2E1,1.6256282E7,9.367855E-1,1.568E3,1.8489688E7,1.9051096E7,2.8035452E11,-1.8249024E-3,1.116567E3,1.2679E4,4.767929E-3,1.9806543E-2,1.228576E-3,1.0257844E-2,3.046556E5,-1.8291049E-2,2.1164343E-3,1.190366E-2,4.0701206E1,1.8273557E2,1E0,2E0,7.8955513E-1,2.8839298E7,7.854E3,2.109E3,1.9745072E7,5.2076015E0,1.172E3,1.9E1,1E0,5.7118463E2,1.1472081E0,-8.82817E-3,5.9572783E0,2.65851E5,1.2939E4,3.830986E-1,3.5225567E-3,2.1944155E3,2.0605305E5,6.0210535E-3,3.28825E0,1.0828989E8,6.737088E7,3.4E2,1.1902924E0,9.544947E6,8E0,6.1E2,2.6204643E0,3.0769231E0,2.1997621E-2,9.116828E-3,1.0918E4,4.247868E3,-3.035723E-2,3.3720784E5,1.6173E4,2.342147E7,1.5435694E9,3.0737953E-2,9.3058E4,1.0610115E-2,-4.6806918E-3,4.8982824E-4,-1.9385861E-2,1.1788341E-3,-1.3438413E-2,-6.5442445E-3,-5.1362077E-3,1.5748915E-3,7.2384044E-3,-2.5712966E-3,3.8240654E-3,-2.6755857E-3,-3.3317952E-4,-8.320269E-3,-1.6456159E-2,-6.570487E-3,-1.1288314E-3,-2.2737572E-2,-6.4813127E-3,3.8014012E-3,-1.0792331E-2,-2.550001E-2,7.62949E-3,-1.0616846E-3,-1.6603108E-3,-9.430072E-3,-9.605686E-4,-7.572817E-3,8.026952E-3,1.0983687E-3,2.2114484E-4,-8.563059E-3,-1.9437693E-3,7.868962E-3,1.6927205E-2,3.5588413E-3,4.759963E-3,-5.4982947E-3,6.0119973E-3,-9.307976E-3,1.093448E-2,-5.300097E-4,1.6704813E-2,6.6113416E-3,-1.6920137E-3,8.043709E-3,-5.221398E-3,-2.3725323E-2,-2.7548042E-3,1.081828E-2,1.1650656E-2,-5.870055E-3,3.512497E-3,9.020546E-3,-2.810305E-4,-1.6191516E-2],"split_indices":[20,52,2,45,45,46,53,2,4,7,2,52,0,2,7,0,28,1,45,8,54,12,53,0,58,0,0,102,0,0,45,27,0,32,45,31,0,4,2,0,0,0,0,32,0,0,0,52,55,79,17,38,45,9,0,1,35,2,3,16,55,53,0,53,29,9,56,0,4,33,0,54,45,7,12,53,9,3,2,38,58,0,0,9,52,0,28,2,45,7,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.653E3,8.58E2,1.795E3,6.82E2,1.76E2,2.41E2,1.554E3,5.68E2,1.14E2,4.5E1,1.31E2,2.3E2,1.1E1,1.142E3,4.12E2,3.24E2,2.44E2,5E1,6.4E1,1.8E1,2.7E1,1.09E2,2.2E1,2.2E1,2.08E2,7E0,4E0,6.42E2,5E2,2.99E2,1.13E2,1.61E2,1.63E2,1E2,1.44E2,4.5E1,5E0,2.9E1,3.5E1,6E0,1.2E1,1.9E1,8E0,1.06E2,3E0,1.1E1,1.1E1,1.2E2,8.8E1,1.68E2,4.74E2,2E1,4.8E2,1.8E1,2.81E2,7E1,4.3E1,1.49E2,1.2E1,1.2E2,4.3E1,8E1,2E1,1.11E2,3.3E1,1.5E1,3E1,5E0,2.4E1,3.2E1,3E0,1.4E1,9.2E1,6.9E1,5.1E1,5.5E1,3.3E1,2.1E1,1.47E2,2.89E2,1.85E2,1.6E1,4E0,3.88E2,9.2E1,3E0,1.5E1,2.66E2,1.5E1,6.7E1,3E0,4E1,3E0,9.4E1,5.5E1,1E1,2E0,6.8E1,5.2E1,3.3E1,1E1,5E0,7.5E1,8.3E1,2.8E1,1.4E1,1.9E1,5E0,1E1,2E0,2.8E1,2.2E1,2E0,2.7E1,5E0,8E0,6E0,7.3E1,1.9E1,4.5E1,2.4E1,1.5E1,3.6E1,4E0,5.1E1,3.1E1,2E0,1E1,1.1E1,1.9E1,1.28E2,2.81E2,8E0,1.6E1,1.69E2,3.3E1,3.55E2,6.3E1,2.9E1,1.3E1,2E0,2.61E2,5E0,1E1,5E0,3.4E1,3.3E1,3.8E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[-2.8268914E-3,2.7982784E-2,-9.084398E-2,-1.3151068E-2,9.448543E-2,-1.8935782E-1,-5.0116386E-2,-4.360954E-2,5.6930955E-2,2.6189813E-1,6.884182E-2,-1.6766462E-1,-2.4675412E-2,-7.033677E-2,6.7127705E-2,-6.375882E-2,2.8510094E-2,2.038122E-2,1.03569865E-1,2.0987731E-1,2.3194913E-2,8.995504E-2,-1.1318532E-1,-1.3609564E-1,-3.2417434E-1,-3.6310185E-2,-1.4809369E-1,8.74176E-3,2.8137134E-2,-8.4559016E-2,8.769969E-3,1.02610774E-1,-2.4097817E-2,3.9317813E-2,-9.265528E-2,2.2212712E-1,6.513651E-2,2.3423636E-1,-2.1209913E-1,7.131105E-2,2.4271002E-1,-2.0057233E-1,-5.0745763E-2,-1.2016375E-1,-2.5014734E-2,-4.069689E-3,-4.0367797E-1,-1.5949805E-1,-1.1827354E-2,-1.282418E-1,-4.2487878E-1,7.299388E-3,5.078395E-4,-3.414583E-2,-1.1386132E-1,7.312087E-2,-5.483985E-2,-6.995687E-3,1.2568402E-1,-1.6832036E-1,2.5245606E-3,2.795005E-2,9.302387E-3,-1.4007173E-1,8.258309E-3,1.1411229E-1,3.4180486E-1,1.26976725E-2,4.7021013E-2,2.5835773E-1,-2.2667367E-3,-1.7024565E-2,9.532978E-5,3.0777E-2,1.4664218E-1,4.332064E-1,1.5060462E-1,3.2996075E-4,-2.257434E-1,-9.739994E-2,4.962522E-3,-2.75008E-1,-9.087806E-2,-5.376019E-1,-1.3561833E-1,-1.3302602E-1,-2.4531279E-2,-3.2334127E-2,4.988685E-2,-1.9282885E-2,-1.7159852E-1,-7.7844104E-3,-2.6328748E-2,-2.3619859E-2,7.916105E-3,-6.217722E-3,1.0896539E-3,-6.8739224E-3,-1.9834345E-6,5.359261E-3,-4.1615088E-3,-1.1298731E-2,3.1057285E-4,7.84211E-3,2.1557984E-4,1.6626782E-3,-1.2251022E-2,-1.2988701E-2,8.5603877E-4,3.209582E-3,-1.2365183E-3,-1.7771974E-2,-4.492304E-3,8.211742E-3,-3.6854608E-4,1.9175857E-2,7.559966E-3,3.487453E-3,-5.0034225E-3,1.3991252E-2,4.6688775E-3,1.9005117E-3,-1.2793418E-2,8.413275E-3,-1.6438293E-3,1.575776E-2,3.2561112E-2,4.456548E-3,2.392437E-2,1.609013E-4,-1.231445E-2,-5.8173263E-3,8.418222E-3,-1.6611684E-2,-5.588864E-3,-3.1400982E-3,-1.4257609E-2,-7.6651005E-3,-2.7938103E-2,3.8004331E-3,-9.979258E-3,-9.3590876E-4,-7.854351E-3,8.895771E-3,-1.9528002E-3,3.7538277E-3,-7.1213897E-3,-2.6633823E-3,7.930017E-3,-1.6724288E-2,-6.990558E-3,-3.6844598E-5,-1.1244445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,-1,83,85,87,89,91,-1,93,95,97,99,101,-1,103,105,107,109,-1,111,-1,113,115,-1,117,119,-1,-1,-1,121,123,125,127,-1,129,131,-1,133,135,137,139,141,-1,143,145,147,149,-1,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.164347E0,5.3546352E0,2.7388892E0,2.5853384E0,3.193183E0,1.3275242E0,1.1568915E0,1.2282836E0,6.23492E-1,1.0438967E0,2.5082471E0,8.8970184E-1,0E0,1.0926306E0,3.1885278E-1,9.9642634E-1,7.236548E-1,4.4673195E-1,7.1913636E-1,8.925543E-1,0E0,1.643054E0,3.608575E-1,9.156358E-1,5.552199E-1,8.7275E-1,6.3784575E-1,0E0,1.8878146E-1,7.53927E-1,6.0982555E-1,4.6018767E-1,4.189356E-1,3.080555E-1,4.0125248E-1,4.6514773E-1,4.331807E-1,5.419893E-1,1.5419926E-1,1.5863478E0,1.0401392E0,1.4777255E-1,3.0537108E-1,6.7335486E-1,0E0,0E0,7.527821E-1,3.8006902E-1,3.091439E-1,5.58859E-1,1.7274046E-1,0E0,1.8679816E-1,1.0373955E0,1.094255E0,4.6899492E-1,8.3442223E-1,0E0,3.26787E-1,3.10876E-1,3.9563566E-1,3.5255295E-1,0E0,2.4310529E-1,0E0,1.5502256E-1,1.3083482E-1,0E0,4.5319745E-1,3.629222E-1,0E0,0E0,0E0,9.1285324E-1,9.6232677E-1,2.9025817E-1,8.3711E-1,0E0,1.8067586E-1,2.3177886E-1,0E0,2.3254228E-1,6.662569E-1,1.6584682E-1,1.5730661E-1,1.5851927E-1,0E0,3.4450084E-1,3.5686076E-1,2.4428628E-1,3.3596492E-1,0E0,0E0,1.9766293E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,63,63,65,65,66,66,68,68,69,69,73,73,74,74,75,75,76,76,78,78,79,79,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,90,90,93,93],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,-1,84,86,88,90,92,-1,94,96,98,100,102,-1,104,106,108,110,-1,112,-1,114,116,-1,118,120,-1,-1,-1,122,124,126,128,-1,130,132,-1,134,136,138,140,142,-1,144,146,148,150,-1,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,5.862126E2,2.0666666E0,7.5939475E5,1.0594161E1,1.2427474E8,3.970405E3,4.264897E6,1.146E3,1.415178E3,1E0,2.0354E4,-2.4675412E-2,4.987639E7,4.0601485E6,6.067634E5,2.46E2,4.05E0,1.0437505E-2,1.9162654E7,2.3194913E-2,5.185489E3,5E0,4.5723195E6,5.2187E4,9.7043E4,8.91942E5,8.74176E-3,2.8091298E7,9.52E2,2.72258E5,2.35E2,2.5330253E-2,7.780377E6,2.1178565E6,7.130317E7,7.692308E-1,3.5E1,1.1E1,1E0,3.2475834E7,8.503172E2,1.4659271E7,7.22E2,-2.5014734E-2,-4.069689E-3,8.92686E5,5.551823E-1,8.3956606E5,4.1808826E-1,7.526755E7,7.299388E-3,5.545735E8,4.7E2,3.3012173E9,1.0715278E7,7.118233E7,-6.995687E-3,2.7309215E6,9.173799E4,2.511352E6,2.69E2,9.302387E-3,1.6437647E-4,8.258309E-3,6.445E3,1.3E1,1.26976725E-2,4.5158855E6,4.675E0,-2.2667367E-3,-1.7024565E-2,9.532978E-5,8.6206274E2,1.4521701E2,4.2E1,4.2482185E0,3.2996075E-4,1.7321888E1,1.9178673E3,4.962522E-3,1E0,5.7E1,2.2837209E2,6.77603E5,2.7627052E7,-2.4531279E-2,2.1515152E0,2.632E5,1.6992E4,8.1057306E2,-7.7844104E-3,-2.6328748E-2,7.75541E-1,7.916105E-3,-6.217722E-3,1.0896539E-3,-6.8739224E-3,-1.9834345E-6,5.359261E-3,-4.1615088E-3,-1.1298731E-2,3.1057285E-4,7.84211E-3,2.1557984E-4,1.6626782E-3,-1.2251022E-2,-1.2988701E-2,8.5603877E-4,3.209582E-3,-1.2365183E-3,-1.7771974E-2,-4.492304E-3,8.211742E-3,-3.6854608E-4,1.9175857E-2,7.559966E-3,3.487453E-3,-5.0034225E-3,1.3991252E-2,4.6688775E-3,1.9005117E-3,-1.2793418E-2,8.413275E-3,-1.6438293E-3,1.575776E-2,3.2561112E-2,4.456548E-3,2.392437E-2,1.609013E-4,-1.231445E-2,-5.8173263E-3,8.418222E-3,-1.6611684E-2,-5.588864E-3,-3.1400982E-3,-1.4257609E-2,-7.6651005E-3,-2.7938103E-2,3.8004331E-3,-9.979258E-3,-9.3590876E-4,-7.854351E-3,8.895771E-3,-1.9528002E-3,3.7538277E-3,-7.1213897E-3,-2.6633823E-3,7.930017E-3,-1.6724288E-2,-6.990558E-3,-3.6844598E-5,-1.1244445E-2],"split_indices":[27,52,53,28,56,45,52,9,2,52,14,9,0,45,47,47,10,54,41,47,0,52,6,28,29,1,9,0,45,29,9,2,38,28,28,7,53,3,8,63,50,4,50,2,0,0,1,38,28,27,45,0,44,1,5,45,7,0,45,33,5,10,0,38,0,29,3,0,47,54,0,0,0,58,58,8,39,0,56,46,0,16,8,4,29,5,0,53,33,2,4,0,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E3,1.956E3,6.84E2,1.209E3,7.47E2,1.99E2,4.85E2,8.43E2,3.66E2,9.8E1,6.49E2,1.88E2,1.1E1,4.14E2,7.1E1,6.59E2,1.84E2,2.06E2,1.6E2,8.1E1,1.7E1,5.82E2,6.7E1,1.58E2,3E1,2.89E2,1.25E2,1.7E1,5.4E1,5.12E2,1.47E2,7.6E1,1.08E2,1.77E2,2.9E1,3.8E1,1.22E2,7.7E1,4E0,5.2E2,6.2E1,2.7E1,4E1,1.53E2,5E0,8E0,2.2E1,4.7E1,2.42E2,1.18E2,7E0,9E0,4.5E1,1.89E2,3.23E2,7.3E1,7.4E1,6E0,7E1,1.6E1,9.2E1,1.66E2,1.1E1,2.5E1,4E0,2.1E1,1.7E1,9E0,1.13E2,7.1E1,6E0,2E0,2E0,3.39E2,1.81E2,1.9E1,4.3E1,3E0,2.4E1,3.1E1,9E0,2.3E1,1.3E2,1.4E1,8E0,4.5E1,2E0,1.82E2,6E1,3.4E1,8.4E1,3E0,4E0,4E1,5E0,7E1,1.19E2,2.56E2,6.7E1,5.9E1,1.4E1,1.8E1,5.6E1,5.3E1,1.7E1,5E0,1.1E1,4E0,8.8E1,9.6E1,7E1,3E0,2.2E1,1.4E1,7E0,1.2E1,5E0,9.7E1,1.6E1,5.8E1,1.3E1,3.3E2,9E0,1.56E2,2.5E1,1.5E1,4E0,3.8E1,5E0,3E0,2.1E1,2.9E1,2E0,1.5E1,8E0,1.17E2,1.3E1,2E0,1.2E1,2E0,6E0,1E1,3.5E1,6E0,1.76E2,5.3E1,7E0,2.9E1,5E0,9E0,7.5E1,3.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[7.590838E-4,-6.0613424E-2,3.8963288E-2,-7.306628E-2,2.0708703E-1,1.9886235E-2,1.790503E-1,-2.8392944E-2,-1.3904828E-1,2.465124E-2,8.336369E-2,1.01669416E-1,-1.6797764E-2,1.6225196E-1,1.7871466E-2,-7.882628E-2,5.4508377E-2,-1.22743964E-1,-3.0951062E-1,8.1719095E-3,-1.2204375E-3,6.022596E-2,1.766683E-1,-4.201876E-2,4.185695E-2,1.322178E-2,1.4261757E-1,-5.6855954E-2,-2.8497833E-1,2.084573E-1,-3.4388222E-3,-1.7096014E-1,-8.697128E-2,-2.4118785E-2,-1.9477753E-1,5.0335433E-2,3.656352E-1,2.1178412E-1,2.9586773E-2,-6.371543E-2,-2.8496743E-3,1.7469425E-1,1.6786452E-2,1.5100859E-1,-1.4602757E-1,-7.013273E-2,9.2563E-3,2.4425008E-3,-4.4109428E-1,1.8138494E-2,1.4469373E-1,-2.9542506E-2,1.4183521E-1,-3.6735502E-1,-1.5024747E-1,-1.7996979E-1,-5.9953704E-2,-1.2293349E-1,-2.419925E-2,7.4928336E-2,-1.211637E-1,2.0761304E-2,4.088355E-3,1.1542316E-1,2.6936242E-1,-2.9160032E-2,9.244815E-2,-7.582123E-2,1.5322201E-2,4.349946E-2,-3.31246E-2,2.1665667E-1,-6.2199935E-2,3.481671E-2,-1.3742635E-1,1.0386245E-1,2.2596057E-1,5.6169466E-3,-1.7272511E-2,7.999206E-3,-3.7560086E-3,-3.3741046E-2,-1.13047585E-2,5.678335E-3,2.2969475E-2,-2.1852208E-3,1.1473884E-2,3.9306437E-3,1.2382816E-2,-3.5809148E-3,-2.1088721E-2,-4.2263824E-3,-1.2045431E-2,-5.3500957E-3,-1.2411112E-2,-2.9652755E-4,-5.1761474E-3,3.0697947E-3,-9.945398E-3,-1.0154068E-3,5.466813E-3,-1.9512279E-3,-1.9038476E-2,1.1170663E-2,2.2821792E-3,8.794285E-3,1.4941511E-2,-7.0741E-3,7.0106536E-3,5.0430354E-3,-3.8941004E-3,7.4308133E-3,-1.2347978E-3,-3.4187704E-3,3.7710615E-3,-1.3014524E-2,-1.112332E-3,8.65011E-3,2.1070782E-2,3.7437095E-3,-7.7610286E-3,3.7379013E-3,-1.4790042E-3,-9.413969E-3,-8.2469475E-4,2.025146E-3,7.1890815E-3,9.3619395E-3,2.4710873E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,79,-1,-1,81,-1,83,85,87,89,91,93,95,97,-1,99,101,-1,-1,103,105,-1,107,109,111,113,115,117,119,121,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.2884703E0,3.4454403E0,4.413995E0,2.8965263E0,1.7080342E0,4.369903E0,5.851183E-1,2.462719E0,1.0653801E0,0E0,3.2733208E-1,1.3911643E0,1.4899651E0,3.7186766E-1,0E0,1.6414948E0,1.9900796E0,6.1296654E-1,6.591239E-1,0E0,0E0,8.6362755E-1,8.201165E-1,5.981046E-1,1.0061476E0,0E0,4.0715384E-1,1.1161286E0,1.8583112E0,6.075604E-1,6.223167E-1,5.712571E-1,5.193677E-1,0E0,4.5099825E-1,1.2068946E0,1.3421333E-1,6.8463945E-1,1.340626E0,4.3459845E-1,3.5505146E-1,4.906274E-1,7.165919E-1,5.242543E-1,3.248895E-1,6.348561E-1,0E0,0E0,1.1301212E0,0E0,3.456295E-1,6.127273E-1,1.531713E-1,2.660824E-1,8.6307955E-1,2.25075E-1,4.2627114E-1,3.302811E-1,0E0,9.413085E-1,7.839515E-1,0E0,0E0,3.881492E-1,2.3508358E-1,0E0,4.102611E-1,3.9944363E-1,3.5464492E-1,4.1052648E-1,3.5507125E-1,2.504648E-1,1.2480637E-1,6.557202E-1,1.8857777E-1,2.6990962E-1,4.205947E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,63,63,64,64,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,80,-1,-1,82,-1,84,86,88,90,92,94,96,98,-1,100,102,-1,-1,104,106,-1,108,110,112,114,116,118,120,122,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,6.082581E3,1.3137E4,1.6627582E-2,1.4232318E3,2E0,1.8114515E1,2.857E3,1.2427474E8,2.465124E-2,2.851369E2,5.9643E4,7.298614E2,2.386324E6,1.7871466E-2,9.837297E0,9.073025E6,1.9121015E0,2.0521326E0,8.1719095E-3,-1.2204375E-3,6.567044E2,2.3523688E0,3.5150378E5,5.1212654E0,1.322178E-2,5.68E2,4.564527E6,1.1880085E2,9.9341E4,1E0,5.85E2,1.5804776E4,-2.4118785E-2,7.6008E4,1.604E3,3.4971508E7,5.8084745E6,3.171E3,1.2817779E7,2.0282449E6,4.029152E8,6.747114E7,5.57E4,3.2917362E7,6E0,9.2563E-3,2.4425008E-3,3.475E2,1.8138494E-2,4.215533E3,4.5505118E2,1.82407E5,1E0,2.1358025E0,1.5476E4,1.6772E4,2.345785E5,-2.419925E-2,5.52E2,8.450237E0,2.0761304E-2,4.088355E-3,4.1681856E9,4.930435E2,-2.9160032E-2,1E0,3.8E2,2.4946861E4,9E2,1.074E3,1.5137865E2,7.0491E4,2.429717E1,6.1428223E1,2.7271206E5,1.0938637E1,5.6169466E-3,-1.7272511E-2,7.999206E-3,-3.7560086E-3,-3.3741046E-2,-1.13047585E-2,5.678335E-3,2.2969475E-2,-2.1852208E-3,1.1473884E-2,3.9306437E-3,1.2382816E-2,-3.5809148E-3,-2.1088721E-2,-4.2263824E-3,-1.2045431E-2,-5.3500957E-3,-1.2411112E-2,-2.9652755E-4,-5.1761474E-3,3.0697947E-3,-9.945398E-3,-1.0154068E-3,5.466813E-3,-1.9512279E-3,-1.9038476E-2,1.1170663E-2,2.2821792E-3,8.794285E-3,1.4941511E-2,-7.0741E-3,7.0106536E-3,5.0430354E-3,-3.8941004E-3,7.4308133E-3,-1.2347978E-3,-3.4187704E-3,3.7710615E-3,-1.3014524E-2,-1.112332E-3,8.65011E-3,2.1070782E-2,3.7437095E-3,-7.7610286E-3,3.7379013E-3,-1.4790042E-3,-9.413969E-3,-8.2469475E-4,2.025146E-3,7.1890815E-3,9.3619395E-3,2.4710873E-2],"split_indices":[102,52,2,27,48,17,35,2,45,0,58,1,52,1,0,54,45,53,53,0,0,52,41,47,53,0,3,9,52,1,96,2,51,0,10,2,5,45,2,9,50,7,45,2,1,3,0,0,55,0,4,56,9,104,56,9,9,28,0,2,56,0,0,31,4,0,16,12,50,2,2,56,10,53,56,28,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E3,1.028E3,1.652E3,9.83E2,4.5E1,1.455E3,1.97E2,5.87E2,3.96E2,1.2E1,3.3E1,4.5E2,1.005E3,1.83E2,1.4E1,3.65E2,2.22E2,3.63E2,3.3E1,1.8E1,1.5E1,2.91E2,1.59E2,7.03E2,3.02E2,2.5E1,1.58E2,3.31E2,3.4E1,6E1,1.62E2,1.53E2,2.1E2,1.1E1,2.2E1,2.83E2,8E0,1.28E2,3.1E1,4.52E2,2.51E2,4.7E1,2.55E2,1.54E2,4E0,3.15E2,1.6E1,1.1E1,2.3E1,1.5E1,4.5E1,1.38E2,2.4E1,1.3E1,1.4E2,4.6E1,1.64E2,1.9E1,3E0,2.48E2,3.5E1,6E0,2E0,4.9E1,7.9E1,2E0,2.9E1,3.92E2,6E1,9.9E1,1.52E2,4E1,7E0,2.29E2,2.6E1,9.6E1,5.8E1,2E0,2E0,1E1,3.05E2,9E0,1.4E1,4.3E1,2E0,1.31E2,7E0,1.7E1,7E0,3E0,1E1,8.8E1,5.2E1,2.6E1,2E1,7.8E1,8.6E1,6E0,1.3E1,7.2E1,1.76E2,2.8E1,7E0,1.7E1,3.2E1,2.8E1,5.1E1,5E0,2.4E1,1.1E1,3.81E2,1.3E1,4.7E1,2.3E1,7.6E1,5E0,1.47E2,3.6E1,4E0,3E0,4E0,1.38E2,9.1E1,1.7E1,9E0,4.2E1,5.4E1,5.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[-1.6417538E-5,-6.5911196E-2,3.1341176E-2,-8.880324E-2,1.3958083E-2,3.7770246E-3,1.08839154E-1,-5.681546E-2,-2.0157078E-1,6.581459E-2,-1.1001461E-1,-5.2184317E-2,3.6549885E-2,1.320488E-1,-1.8089226E-2,-1.1279587E-1,-1.0063984E-2,-1.6442397E-1,-3.9469168E-1,1.67138E-1,1.2169989E-2,-5.9666056E-2,-2.4701282E-1,1.9597353E-1,-5.9862524E-2,5.3300314E-2,-5.1319383E-2,1.1955666E-1,3.905179E-1,-2.5985873E-1,4.778198E-2,-5.030633E-2,-1.6384259E-1,-2.2273358E-2,-5.2048373E-3,-1.958364E-1,-6.701173E-2,-4.7418323E-1,-9.0435125E-2,1.3376762E-1,2.0147337E-2,-2.2717725E-2,1.7494553E-1,-1.3134602E-2,-1.0148311E-2,-1.8172564E-2,7.136646E-4,1.8471206E-2,2.9951928E-3,-6.969798E-2,7.062689E-2,1.4546054E-2,1.20950565E-1,-2.0676288E-1,-1.9226287E-2,2.087537E-1,9.531733E-2,4.6776813E-1,4.6029324E-3,-5.056277E-3,-1.534352E-2,3.129298E-4,2.0033874E-1,-8.773058E-2,3.3726748E-2,-1.2996623E-1,-2.6352745E-1,3.334921E-2,-3.865174E-2,-1.429598E-1,-3.0387446E-1,-4.3355394E-3,9.1236355E-3,-2.6795128E-2,-1.0070557E-2,-1.0822805E-2,1.6206236E-3,1.9425955E-1,7.778403E-2,-5.343405E-2,6.1245207E-3,1.227667E-2,1.6121519E-3,-1.1575131E-2,1.6573792E-3,-7.544189E-2,1.6378355E-1,9.3550505E-3,1.191143E-4,-1.4184625E-3,1.2689316E-1,1.3747165E-1,-3.7863847E-2,-4.896904E-2,-3.2437593E-1,2.4504896E-2,-1.4674091E-1,2.2530448E-1,-7.958212E-3,6.646334E-2,1.639498E-1,5.8351257E-3,2.4676496E-2,1.2976955E-1,-4.919439E-2,1.4598988E-2,2.5041006E-3,5.658634E-3,-5.2996897E-3,1.0443961E-2,8.989079E-5,-3.7187503E-3,-1.1862621E-2,-1.8293979E-2,-8.301669E-3,-2.5175244E-3,2.8319801E-3,-5.7597426E-3,-8.0205145E-4,-4.709894E-3,-1.3095168E-2,-4.0281406E-3,-1.6016947E-2,5.0866343E-3,1.301109E-2,-2.4264379E-4,7.713102E-3,-3.8572683E-3,1.180914E-3,-7.528414E-3,-2.8922237E-3,1.3757758E-2,6.223268E-4,-2.9050536E-3,1.1658467E-3,4.0732324E-3,1.2391443E-2,8.897558E-3,4.2133536E-3,-1.916104E-2,3.4816621E-3,5.138261E-3,-5.7937168E-3,-2.604672E-2,-7.968375E-3,4.686131E-4,1.2358862E-2,-8.26409E-3,2.7671985E-3,-4.8665362E-3,1.1265999E-2,8.466979E-3,2.5975942E-3,9.000484E-3,-6.891981E-3,1.1180475E-2,3.6117237E-4,-3.6288358E-3,4.1890126E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,-1,79,81,83,-1,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,-1,-1,-1,103,105,107,109,111,113,115,117,119,121,-1,-1,-1,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,129,131,-1,-1,133,135,137,139,141,143,145,147,149,-1,151,153,-1,-1,155,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4984283E0,1.5699813E0,3.8510962E0,2.3945394E0,1.2415684E0,2.442979E0,1.3958297E0,1.3624086E0,9.896555E-1,7.371875E-1,3.7842852E-1,9.491918E-1,1.2390875E0,1.2448449E0,1.1909146E0,7.477603E-1,6.2772375E-1,3.6937094E-1,5.13391E-1,3.4140623E-1,5.150723E-1,3.0182716E-1,5.260045E-1,3.556013E-1,6.183988E-1,1.849186E0,6.694943E-1,8.08177E-1,3.644116E-1,1.1558604E-1,4.2719913E-1,3.426053E-1,4.047556E-1,0E0,3.6622128E-1,4.9355054E-1,2.1105562E-1,2.876544E-1,1.0872464E-1,1.3051158E-1,0E0,3.537825E-1,1.7538336E-1,3.7938005E-1,0E0,0E0,0E0,0E0,0E0,6.09144E-1,2.884485E-1,8.0728537E-1,6.79518E-1,4.109637E-1,6.3372517E-1,5.400748E-1,5.891404E-1,1.6726112E-1,0E0,0E0,0E0,3.0120024E-1,2.0066291E-1,3.6718273E-1,1.9922504E-1,5.905472E-1,2.7565575E-1,2.9554182E-1,2.6972443E-1,3.514769E-1,1.7344451E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0748124E-1,1.6639714E-1,1.3535985E-1,0E0,0E0,0E0,0E0,0E0,5.2574134E-1,2.010994E-1,0E0,0E0,6.043315E-1,2.8692883E-1,5.408473E-1,1.0303489E0,1.367545E-1,3.6033487E-1,2.8970113E-1,1.6312015E-1,2.7738214E-1,0E0,2.803676E-1,6.843326E-1,0E0,0E0,1.6299799E-1,1.3013086E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,77,77,78,78,79,79,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,-1,80,82,84,-1,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,-1,-1,-1,104,106,108,110,112,114,116,118,120,122,-1,-1,-1,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,130,132,-1,-1,134,136,138,140,142,144,146,148,150,-1,152,154,-1,-1,156,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,6.624E3,2.589369E7,9.302862E7,1.8108038E5,4.0844156E7,1.638058E2,2.1827742E6,1.3977353E7,6.0435E4,7E0,7.382963E0,4.5505118E2,4.7013435E1,4.15E2,7E0,4.6E0,2.194E3,1E0,2.6108465E0,9.190713E-1,3.285E3,1.3050649E4,6.077036E8,4.5154498E2,4.745907E7,2E0,3.1029554E7,1.02E3,1.1803382E4,1.11E3,6.298144E0,-2.2273358E-2,2.1111E4,2.8636363E0,1.4622E4,1E0,2.0802219E8,2.5927516E11,2.0147337E-2,5.884028E0,3.0720797E-1,1.9469026E0,-1.0148311E-2,-1.8172564E-2,7.136646E-4,1.8471206E-2,2.9951928E-3,1.5694646E3,2.12062E5,4.5723195E6,1.6015803E7,8E0,6.856226E-1,2.2644286E2,1.4689211E7,3.4E1,4.6029324E-3,-5.056277E-3,-1.534352E-2,1.774775E11,7.7854166E0,3E0,1.71525E4,8.617448E5,9.192915E0,1E0,1.4571098E5,9.1569895E-1,1.5825138E5,-4.3355394E-3,9.1236355E-3,-2.6795128E-2,-1.0070557E-2,-1.0822805E-2,1.6206236E-3,5.59E2,1.3593E4,1.4777102E9,6.1245207E-3,1.227667E-2,1.6121519E-3,-1.1575131E-2,1.6573792E-3,3.983889E3,2.5057983E3,9.3550505E-3,1.191143E-4,2.0697437E2,2.0827537E0,3.68E0,3.171E3,9.61E2,2E0,3.9516E4,1.1364663E0,5.0040483E0,-7.958212E-3,1.7593515E1,1.314806E8,5.8351257E-3,2.4676496E-2,5.4106476E7,2.1325744E7,1.4598988E-2,2.5041006E-3,5.658634E-3,-5.2996897E-3,1.0443961E-2,8.989079E-5,-3.7187503E-3,-1.1862621E-2,-1.8293979E-2,-8.301669E-3,-2.5175244E-3,2.8319801E-3,-5.7597426E-3,-8.0205145E-4,-4.709894E-3,-1.3095168E-2,-4.0281406E-3,-1.6016947E-2,5.0866343E-3,1.301109E-2,-2.4264379E-4,7.713102E-3,-3.8572683E-3,1.180914E-3,-7.528414E-3,-2.8922237E-3,1.3757758E-2,6.223268E-4,-2.9050536E-3,1.1658467E-3,4.0732324E-3,1.2391443E-2,8.897558E-3,4.2133536E-3,-1.916104E-2,3.4816621E-3,5.138261E-3,-5.7937168E-3,-2.604672E-2,-7.968375E-3,4.686131E-4,1.2358862E-2,-8.26409E-3,2.7671985E-3,-4.8665362E-3,1.1265999E-2,8.466979E-3,2.5975942E-3,9.000484E-3,-6.891981E-3,1.1180475E-2,3.6117237E-4,-3.6288358E-3,4.1890126E-3],"split_indices":[20,52,2,45,45,28,45,52,28,50,10,3,54,56,56,0,10,54,0,66,35,27,0,28,7,55,7,17,47,11,4,2,57,0,9,54,2,16,7,31,0,53,38,53,0,0,0,0,0,52,29,28,51,8,27,58,1,8,0,0,0,31,54,11,48,32,57,64,28,27,28,0,0,0,0,0,0,0,10,7,0,0,0,0,0,28,55,0,0,4,53,54,2,12,17,10,39,56,0,58,1,0,0,45,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.659E3,8.57E2,1.802E3,6.66E2,1.91E2,1.33E3,4.72E2,5.2E2,1.46E2,1.35E2,5.6E1,4.91E2,8.39E2,3.99E2,7.3E1,2.36E2,2.84E2,1.24E2,2.2E1,4.6E1,8.9E1,4.2E1,1.4E1,1.4E1,4.77E2,7.05E2,1.34E2,3.82E2,1.7E1,1.5E1,5.8E1,1.07E2,1.29E2,2E0,2.82E2,9.3E1,3.1E1,1.7E1,5E0,4.2E1,4E0,7.4E1,1.5E1,3.3E1,9E0,9E0,5E0,5E0,9E0,4.44E2,3.3E1,4.49E2,2.56E2,2.2E1,1.12E2,8E1,3.02E2,1.3E1,4E0,5E0,1E1,4.5E1,1.3E1,7.4E1,3.3E1,9.8E1,3.1E1,1.31E2,1.51E2,6.4E1,2.9E1,2.9E1,2E0,1.2E1,5E0,2E0,3E0,1.9E1,2.3E1,6.2E1,1.2E1,9E0,6E0,5E0,2.8E1,4.34E2,1E1,1.1E1,2.2E1,3.94E2,5.5E1,2.32E2,2.4E1,1E1,1.2E1,8.4E1,2.8E1,7.7E1,3E0,2.14E2,8.8E1,2E0,1.1E1,1.2E1,3.3E1,7E0,6E0,7E0,6.7E1,4E0,2.9E1,6.9E1,2.9E1,1.2E1,1.9E1,3E1,1.01E2,3.1E1,1.2E2,4.9E1,1.5E1,4E0,2.5E1,1E1,9E0,1.2E1,1.1E1,4.6E1,1.6E1,6.6E1,3.68E2,5E0,5E0,1.19E2,2.75E2,4.3E1,1.2E1,1.16E2,1.16E2,5E0,1.9E1,3E0,7E0,4E0,8E0,8E1,4E0,2.5E1,3E0,2E0,7.5E1,2E1,1.94E2,8.2E1,6E0,6E0,6E0,2.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[-3.4797138E-3,-5.853659E-2,3.704608E-2,-3.2438904E-2,-1.3875745E-1,6.6020064E-2,-7.9248466E-2,-4.614918E-2,6.2207628E-2,-1.1811673E-1,-3.8868463E-1,2.9693453E-2,1.2439971E-1,-1.2554118E-1,6.306372E-2,-3.413721E-2,-1.7568286E-1,1.23296216E-1,-4.777981E-2,-9.760061E-2,-2.1985887E-1,-2.1758405E-2,-9.217794E-3,7.8779586E-2,-1.833606E-2,1.8346475E-2,1.0745199E-1,-9.7005E-2,-2.6222995E-1,8.82491E-2,-9.645017E-3,1.043304E-2,-6.672952E-2,-4.5982483E-1,-1.0126884E-1,4.3500736E-2,1.7424442E-1,2.883165E-2,-1.3397309E-1,-1.8481953E-2,-8.906079E-2,-6.5414324E-2,-2.9216263E-1,9.937331E-2,-2.2866428E-2,1.3372892E-2,-7.3199585E-2,9.6906304E-2,3.5516483E-1,-1.1197613E-1,3.3086605E-2,-3.1026304E-1,-1.4996774E-3,2.5030768E-1,5.1045537E-2,6.0554955E-2,-3.3689618E-2,-7.292131E-2,1.4006448E-1,1.459638E-3,-5.517417E-1,6.693444E-2,-1.5905286E-1,3.9417247E-4,8.492466E-3,1.2693601E-2,9.825902E-2,7.2399944E-2,-1.2989551E-2,-1.5539353E-2,-3.734213E-3,-2.0503278E-1,-6.9038E-2,-1.6765192E-2,2.542055E-4,-3.1722662E-1,1.4750673E-3,1.0675285E-1,-1.650324E-1,9.358433E-3,-4.837642E-2,-2.6477046E-2,7.5680286E-2,-2.939583E-2,-1.2531185E-1,8.695632E-2,3.352731E-1,8.901675E-3,4.720088E-1,-1.5354916E-1,-6.9612056E-2,-1.3314452E-4,1.1934996E-2,-1.0802491E-2,-2.310761E-2,1.4719202E-2,2.1609413E-3,1.24310724E-1,7.8414683E-4,-3.5303994E-3,3.6544942E-3,-2.9890074E-3,7.437525E-3,-3.6907475E-3,9.8273195E-3,-1.4600224E-3,9.5925825E-3,-3.0218448E-2,-6.2297494E-3,-2.8287754E-3,5.9021246E-3,-9.48032E-3,4.210484E-3,6.5397252E-3,-2.1920954E-3,6.4176233E-3,-8.51782E-4,-1.2384263E-2,-8.9633866E-4,-3.69534E-3,7.0596198E-3,-1.787673E-2,-7.836698E-3,6.811432E-3,2.5570607E-3,-1.6855827E-2,-2.937705E-3,5.970519E-3,-3.5471874E-3,-3.3738248E-3,1.2470211E-3,8.857069E-3,2.7091058E-3,-6.500371E-3,-3.0620597E-4,-7.824225E-3,-1.50835E-3,4.8974957E-3,-1.1927933E-3,1.759485E-2,3.0120611E-3,5.648826E-3,2.6137631E-2,-1.5954716E-2,-6.2940014E-3,-4.148103E-3,5.4299887E-3,1.9792689E-3,1.2356785E-2,-4.131064E-3,1.7997547E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,-1,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,95,97,99,101,103,105,-1,107,109,111,-1,-1,-1,113,115,-1,-1,-1,117,119,-1,-1,121,-1,123,125,-1,127,129,131,133,135,137,139,-1,141,143,145,-1,-1,-1,-1,-1,-1,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0309343E0,2.3947406E0,5.251569E0,1.1266184E0,1.4036779E0,2.6408625E0,2.0567906E0,1.1718599E0,7.449785E-1,5.233407E-1,1.8306017E-1,1.8167834E0,2.0545034E0,8.9119744E-1,5.3063536E-1,1.0088335E0,1.3066593E0,2.810663E-1,2.6799837E-1,5.0485206E-1,4.6106625E-1,0E0,0E0,7.9980826E-1,6.794481E-1,0E0,1.1411185E0,3.8543832E-1,4.3394732E-1,4.1529304E-1,0E0,6.5222466E-1,5.2366436E-1,6.0268974E-1,5.160581E-1,1.3542596E-1,2.7066624E-1,3.072512E-1,1.7739183E-1,0E0,4.8277068E-1,3.038345E-1,2.4451852E-1,6.3614583E-1,3.713933E-1,6.177743E-1,3.2274425E-1,9.978113E-1,2.6831317E-1,2.992828E-1,1.6348064E-1,3.7258816E-1,0E0,1.3828921E-1,2.1716471E-1,2.9754865E-1,8.6197627E-1,4.6541834E-1,1.2939002E-1,0E0,2.8549123E-1,1.0660765E-1,3.921355E-1,0E0,0E0,0E0,1.4414924E-1,1.1350362E-1,0E0,0E0,0E0,3.0452633E-1,3.3493537E-1,0E0,0E0,1.7137766E-1,0E0,5.77337E-1,1.5285826E-1,0E0,2.715453E-1,3.5283518E-1,1.9209367E-1,1.9236487E-1,2.269789E-1,7.1661973E-1,1.2564123E-1,0E0,1.9681072E-1,3.058889E-1,2.8676262E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.556293E-1,1.18547395E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,66,66,67,67,71,71,72,72,75,75,77,77,78,78,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,89,89,90,90,97,97,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,-1,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,96,98,100,102,104,106,-1,108,110,112,-1,-1,-1,114,116,-1,-1,-1,118,120,-1,-1,122,-1,124,126,-1,128,130,132,134,136,138,140,-1,142,144,146,-1,-1,-1,-1,-1,-1,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.449702E2,1.6204434E-2,4.0844156E7,9.750871E6,4.3702424E7,1.1210787E3,3.4409692E3,9.669789E0,3.61E2,5.426937E5,4.8052647E5,4.428154E-6,1.296E3,1.2427474E8,4.33E2,2E0,2E0,1.4462532E7,1.3E1,2.8972292E-5,4.302613E6,-2.1758405E-2,-9.217794E-3,1E0,1.3152658E7,1.8346475E-2,4.5505118E2,8.454878E1,4.551684E6,8.35471E-2,-9.645017E-3,1.5182724E0,1.681727E6,8E0,3.114E3,3.6648486E2,9.538462E0,4.0465977E1,4E0,-1.8481953E-2,4.0701206E1,1.2E1,9.3902206E-1,8.0752426E1,6.689559E0,2.5682794E5,1.1968433E-1,7.064848E6,3.8398104E0,1.13814086E5,1.3664E4,8.5354E4,-1.4996774E-3,5.0991332E7,9.83E2,1.02036265E-5,6.4547E4,3.65625E1,5.650399E6,1.459638E-3,2.6668E4,3.188889E1,9.756616E2,3.9417247E-4,8.492466E-3,1.2693601E-2,4.1384277E2,4.9363803E5,-1.2989551E-2,-1.5539353E-2,-3.734213E-3,4.4705097E5,5.964E3,-1.6765192E-2,2.542055E-4,1.8081865E0,1.4750673E-3,8E0,8.3E1,9.358433E-3,8.87E2,4.832E3,1.0074342E-1,3.63E2,5.6903213E-1,7.273333E1,3.7E1,8.901675E-3,3.4E1,1.7190635E0,1.681727E6,-1.3314452E-4,1.1934996E-2,-1.0802491E-2,-2.310761E-2,1.4719202E-2,2.1609413E-3,7.5E2,9.532E3,-3.5303994E-3,3.6544942E-3,-2.9890074E-3,7.437525E-3,-3.6907475E-3,9.8273195E-3,-1.4600224E-3,9.5925825E-3,-3.0218448E-2,-6.2297494E-3,-2.8287754E-3,5.9021246E-3,-9.48032E-3,4.210484E-3,6.5397252E-3,-2.1920954E-3,6.4176233E-3,-8.51782E-4,-1.2384263E-2,-8.9633866E-4,-3.69534E-3,7.0596198E-3,-1.787673E-2,-7.836698E-3,6.811432E-3,2.5570607E-3,-1.6855827E-2,-2.937705E-3,5.970519E-3,-3.5471874E-3,-3.3738248E-3,1.2470211E-3,8.857069E-3,2.7091058E-3,-6.500371E-3,-3.0620597E-4,-7.824225E-3,-1.50835E-3,4.8974957E-3,-1.1927933E-3,1.759485E-2,3.0120611E-3,5.648826E-3,2.6137631E-2,-1.5954716E-2,-6.2940014E-3,-4.148103E-3,5.4299887E-3,1.9792689E-3,1.2356785E-2,-4.131064E-3,1.7997547E-3],"split_indices":[52,27,45,9,45,52,52,54,0,47,47,39,29,45,8,17,17,5,3,42,45,0,0,14,45,0,56,56,29,27,0,53,28,0,29,4,58,58,8,0,52,18,27,54,56,28,41,28,53,33,2,11,0,43,0,38,1,56,45,0,1,52,55,0,0,0,4,33,0,0,0,33,2,0,0,53,0,17,3,0,2,2,42,0,39,54,8,0,8,53,28,0,0,0,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.701E3,1.145E3,1.556E3,8.65E2,2.8E2,1.246E3,3.1E2,7.56E2,1.09E2,2.6E2,2E1,7.69E2,4.77E2,2.34E2,7.6E1,6.93E2,6.3E1,7E1,3.9E1,2.18E2,4.2E1,1.4E1,6E0,3.8E2,3.89E2,2.8E1,4.49E2,1.95E2,3.9E1,7E1,6E0,2.93E2,4E2,1.2E1,5.1E1,2.8E1,4.2E1,2.1E1,1.8E1,5E0,2.13E2,1.4E1,2.8E1,3.16E2,6.4E1,2.47E2,1.42E2,4.32E2,1.7E1,1.75E2,2E1,3.2E1,7E0,1.2E1,5.8E1,1.37E2,1.56E2,3.89E2,1.1E1,2E0,1E1,1.3E1,3.8E1,2.3E1,5E0,1.8E1,2.4E1,1.9E1,2E0,3E0,1.5E1,3E1,1.83E2,2E0,1.2E1,2.6E1,2E0,3.08E2,8E0,6E0,5.8E1,1.51E2,9.6E1,7.8E1,6.4E1,4.16E2,1.6E1,8E0,9E0,8.7E1,8.8E1,1.8E1,2E0,2.3E1,9E0,9E0,3E0,2.3E1,3.5E1,1.4E1,1.23E2,1.36E2,2E1,3.84E2,5E0,3E0,8E0,8E0,2E0,4E0,9E0,3.3E1,5E0,1.9E1,5E0,1.1E1,8E0,2.3E1,7E0,1.77E2,6E0,1.8E1,8E0,1.84E2,1.24E2,2E0,6E0,7E0,5.1E1,8.2E1,6.9E1,1.3E1,8.3E1,1.3E1,6.5E1,4.5E1,1.9E1,3.66E2,5E1,1.4E1,2E0,2E0,7E0,8E0,7.9E1,8.1E1,7E0,1.5E1,8E0,1E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[9.887444E-4,-2.4525387E-2,7.195976E-2,-1.1019293E-2,-1.9117817E-1,1.3340597E-1,-2.9271409E-2,-6.7987114E-2,1.1136018E-2,-1.7343913E-1,-4.1863322E-1,3.5362384E-1,1.1822643E-1,-6.474622E-2,8.306102E-2,-1.4982256E-3,-9.7931124E-2,3.3972546E-2,-5.458534E-2,-1.8194837E-2,-1.562214E-1,-2.4213742E-2,-7.7473735E-3,1.9046277E-2,7.881211E-3,6.970366E-3,1.456019E-1,-8.371007E-2,5.8122E-2,1.4506757E-1,-7.4695244E-2,-3.9558712E-2,1.0298167E-1,-6.9661625E-2,-1.9670612E-1,5.482689E-2,-2.6974607E-2,-2.5557252E-2,-4.72938E-2,-1.6614743E-1,3.6687737E-3,-3.285133E-2,1.5259601E-1,2.7677628E-1,1.24673E-1,-5.927577E-2,-1.4682738E-1,1.1927485E-1,-1.5973743E-3,1.4993047E-2,8.50374E-2,6.691645E-3,-1.8273544E-2,-6.814294E-2,1.7764862E-1,1.7591112E-3,2.2419909E-1,-1.214927E-1,-3.5618484E-2,-2.5734082E-2,-1.7591558E-1,1.8430306E-1,4.4195753E-2,-3.9596695E-2,1.5105466E-2,-7.891135E-2,7.39839E-3,-1.4552577E-1,-2.613439E-1,-1.5080853E-2,-9.963273E-3,7.76593E-2,1.5894849E-2,2.5109738E-2,1.8126912E-1,1.1638704E-1,4.0116894E-1,-3.447174E-2,-1.4080681E-1,-7.1728475E-2,-2.4354361E-1,7.5916434E-3,-2.2941732E-3,1.5523177E-1,-1.2407547E-2,-2.583429E-3,5.590864E-3,5.163664E-3,-5.066285E-3,2.0692356E-2,1.5645848E-3,1.2811237E-2,1.8300931E-3,-1.3884139E-2,-5.0420095E-3,-2.4038514E-5,-5.909662E-3,-4.133331E-3,-1.2958851E-2,1.3251064E-2,4.697428E-3,3.7953406E-4,3.7476716E-3,-2.8889417E-3,4.056959E-3,-4.2277467E-3,6.8598106E-3,1.2664349E-3,-7.531224E-3,-9.270128E-3,-4.7905515E-3,-5.0881854E-3,-1.7662669E-2,-7.4840775E-3,5.9201405E-4,5.797252E-3,-5.8329953E-3,5.460449E-3,1.4918709E-2,7.4350233E-3,2.6396231E-3,7.1102446E-3,2.5051318E-2,-3.0133715E-3,3.37872E-3,-1.0752966E-2,-6.727715E-4,-6.44779E-3,2.9337471E-3,-1.9429612E-3,-1.3836312E-2,3.1966988E-3,1.1775641E-2,5.22427E-3,-2.9690512E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,-1,69,71,73,75,77,79,81,-1,-1,83,85,-1,87,89,-1,91,93,95,-1,97,99,101,103,-1,105,107,109,111,113,-1,115,-1,-1,117,119,121,123,125,127,129,-1,-1,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8618546E0,4.4432793E0,4.4202976E0,2.3079488E0,5.2454853E-1,1.4357123E0,1.0773354E0,1.0192163E0,1.9784707E0,4.348235E-1,1.4160573E-1,1.6789532E-1,1.2629447E0,4.8252642E-1,6.466608E-1,6.403227E-1,9.7198606E-1,1.2445295E0,1.162446E0,0E0,3.1316638E-1,0E0,0E0,0E0,0E0,4.8633495E-1,8.796902E-1,2.658131E-1,1.6061206E-1,4.4738E-1,4.770878E-1,7.447325E-1,3.370816E-1,4.8267245E-1,4.5788407E-1,9.953077E-1,1.0929122E0,0E0,5.816061E-1,2.0735645E-1,0E0,2.0233592E-1,2.1608049E-1,9.6883607E-1,6.1541796E-1,2.5804496E-1,3.3843172E-1,1.1913267E-1,0E0,0E0,2.4816921E-1,1.1306655E-1,0E0,7.0696527E-1,5.022956E-1,0E0,1.10607326E-1,2.6707292E-1,5.178083E-1,0E0,6.129334E-1,4.1097188E-1,8.332422E-1,6.2864506E-1,0E0,4.5592225E-1,3.9303046E-1,2.1013618E-1,3.108486E-1,2.3829067E-1,0E0,1.3591455E-1,0E0,0E0,2.622913E-1,6.629205E-1,1.6882348E-1,3.0621994E-1,3.152333E-1,2.5086E-1,1.8167233E-1,0E0,0E0,1.520341E-1,1.0316048E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51,53,53,54,54,56,56,57,57,58,58,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,71,71,74,74,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,-1,70,72,74,76,78,80,82,-1,-1,84,86,-1,88,90,-1,92,94,96,-1,98,100,102,104,-1,106,108,110,112,114,-1,116,-1,-1,118,120,122,124,126,128,130,-1,-1,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0855529E3,3.9712732E7,2.8839298E7,1.5288235E2,1.5265896E8,1.296E3,5.528107E3,2.4336365E3,6.015825E0,2.3446269E2,7.10309E2,3.4526646E1,8.079531E4,1.79E2,2.1897722E10,2.585E2,2.8169732E10,1.2471935E7,8.01E2,-1.8194837E-2,1.4622E4,-2.4213742E-2,-7.7473735E-3,1.9046277E-2,7.881211E-3,1.4847565E0,1.0918E4,1.0828989E8,8.848604E1,3.1664667E-3,1.921032E4,6.03368E5,5.52E2,4.54E2,5.2921E4,9.2058825E-1,1.4122614E2,-2.5557252E-2,4.832E3,9.6910744E7,3.6687737E-3,1.6018981E0,8.9550705E1,3.4246575E-3,2.4507338E4,1.8880464E-1,2.585911E5,1.13814086E5,-1.5973743E-3,1.4993047E-2,1.3235373E1,5.39978E5,-1.8273544E-2,5.8365756E-1,1.143316E6,1.7591112E-3,1.2094463E7,9.51505E-1,4.15E2,-2.5734082E-2,2.5943396E0,8E0,3.3030225E2,2.3314082E6,1.5105466E-2,2.3616017E11,1E0,3.488E3,4.6263345E-2,9.08912E5,-9.963273E-3,1.14E2,1.5894849E-2,2.5109738E-2,5.3534385E3,1.6015803E7,3.2917362E7,3.317425E3,7.683389E7,1.3664E4,6.99143E5,7.5916434E-3,-2.2941732E-3,9.609747E3,1.4777102E9,-2.583429E-3,5.590864E-3,5.163664E-3,-5.066285E-3,2.0692356E-2,1.5645848E-3,1.2811237E-2,1.8300931E-3,-1.3884139E-2,-5.0420095E-3,-2.4038514E-5,-5.909662E-3,-4.133331E-3,-1.2958851E-2,1.3251064E-2,4.697428E-3,3.7953406E-4,3.7476716E-3,-2.8889417E-3,4.056959E-3,-4.2277467E-3,6.8598106E-3,1.2664349E-3,-7.531224E-3,-9.270128E-3,-4.7905515E-3,-5.0881854E-3,-1.7662669E-2,-7.4840775E-3,5.9201405E-4,5.797252E-3,-5.8329953E-3,5.460449E-3,1.4918709E-2,7.4350233E-3,2.6396231E-3,7.1102446E-3,2.5051318E-2,-3.0133715E-3,3.37872E-3,-1.0752966E-2,-6.727715E-4,-6.44779E-3,2.9337471E-3,-1.9429612E-3,-1.3836312E-2,3.1966988E-3,1.1775641E-2,5.22427E-3,-2.9690512E-3],"split_indices":[52,45,45,52,45,29,52,51,53,52,52,58,28,3,5,47,31,45,2,0,2,0,0,0,0,39,9,45,56,27,4,9,2,2,1,53,56,0,2,45,0,41,58,57,4,38,28,33,0,0,57,11,0,54,9,0,45,58,0,0,56,3,52,28,0,31,74,2,57,1,0,3,0,0,4,51,1,52,32,2,1,0,0,52,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,1.974E3,7.09E2,1.827E3,1.47E2,4.41E2,2.68E2,5.11E2,1.316E3,1.38E2,9E0,2.7E1,4.14E2,2.04E2,6.4E1,1.59E2,3.52E2,9.77E2,3.39E2,9E0,1.29E2,6E0,3E0,2.1E1,6E0,8.2E1,3.32E2,1.77E2,2.7E1,4.6E1,1.8E1,1.17E2,4.2E1,2.75E2,7.7E1,7.28E2,2.49E2,4E0,3.35E2,1.24E2,5E0,6.5E1,1.7E1,4.4E1,2.88E2,1.29E2,4.8E1,1.6E1,1.1E1,1.1E1,3.5E1,1.5E1,3E0,1.04E2,1.3E1,2.8E1,1.4E1,1.08E2,1.67E2,3E0,7.4E1,5.4E1,6.74E2,2.41E2,8E0,2.12E2,1.23E2,1.04E2,2E1,6E1,5E0,1.3E1,4E0,1.1E1,3.3E1,2.81E2,7E0,1E2,2.9E1,2.8E1,2E1,1.3E1,3E0,2E1,1.5E1,1E1,5E0,1.8E1,8.6E1,4E0,9E0,1.1E1,3E0,8E0,1E2,1.2E2,4.7E1,3.9E1,3.5E1,2.5E1,2.9E1,3.27E2,3.47E2,2.07E2,3.4E1,2.04E2,8E0,1.11E2,1.2E1,4.9E1,5.5E1,9E0,1.1E1,9E0,5.1E1,1.1E1,2E0,2.3E1,1E1,1.71E2,1.1E2,3E0,4E0,7.9E1,2.1E1,1.7E1,1.2E1,1.9E1,9E0,4E0,1.6E1,1.1E1,9E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-1.6559662E-3,-1.8388318E-2,9.913462E-2,4.051779E-3,-9.596363E-2,1.2776917E-1,-9.4699524E-2,-6.961674E-3,1.17416374E-1,-1.5892436E-1,-3.8647637E-2,2.2792218E-2,1.133612E-1,-5.3493947E-2,-2.1783056E-2,-4.527749E-2,1.6807841E-2,1.0457537E-1,1.8261148E-2,-1.312889E-1,-2.9865563E-1,-4.8860487E-2,5.208041E-3,1.8454486E-1,8.333584E-2,-2.519828E-1,-5.075688E-3,-1.0027864E-2,-8.892604E-2,1.6322132E-2,1.2902867E-2,1.152017E-1,-5.890125E-2,-1.19931616E-1,-2.1084866E-2,-3.3253524E-1,4.2811506E-3,-3.704676E-2,-7.994268E-3,2.3501131E-1,8.260755E-2,5.440575E-2,1.8324074E-1,-2.0760732E-2,-6.6180197E-3,4.881125E-2,-3.937605E-3,-1.1342228E-1,1.1906468E-2,9.584322E-2,-1.0662254E-1,-6.39047E-2,2.568909E-2,4.779193E-2,1.4535975E-1,-1.3537004E-2,3.2373004E-2,-1.3231637E-1,2.67368E-3,-3.6732095E-1,-1.7452479E-3,-6.241089E-2,3.591071E-2,1.7035443E-1,3.264756E-1,1.10322244E-1,-1.1077728E-2,-4.060792E-3,1.00634925E-1,1.4375915E-1,2.4793867E-2,-8.750407E-4,6.60153E-3,-4.5219874E-3,-2.1515278E-2,7.4300603E-3,-5.661807E-4,8.693243E-3,1.2749265E-3,-1.8071787E-2,-4.560616E-3,-3.8497276E-3,9.119074E-3,4.759187E-3,4.4542472E-4,-4.4210185E-3,3.2148792E-3,9.4114365E-3,3.8121177E-3,6.9137393E-3,-5.5414536E-3,-4.543712E-3,-9.252605E-3,-1.0807908E-2,-2.09527E-2,-2.5205505E-3,-1.0417621E-2,9.208026E-3,7.095071E-5,9.077823E-3,-2.1563484E-3,1.8783804E-2,9.033643E-3,7.0402366E-3,-5.260249E-3,5.050018E-3,-1.1037405E-3,2.8665848E-3,8.697523E-3,1.1204367E-2,-2.1849193E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,-1,51,53,55,57,-1,59,-1,61,-1,63,65,67,69,-1,-1,71,-1,73,75,77,79,81,83,85,87,-1,89,91,-1,93,-1,95,97,99,101,103,-1,105,107,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5485287E0,4.0268536E0,2.1434352E0,2.241939E0,1.8667507E0,1.6264725E0,7.176807E-1,1.4916561E0,4.9892116E-1,9.145899E-1,4.162708E-1,0E0,6.7497396E-1,4.408034E-1,0E0,9.6380734E-1,1.2747858E0,2.7257037E-1,0E0,6.7711306E-1,5.556061E-1,3.508889E-1,0E0,4.6930575E-1,6.5189946E-1,1.2599498E-1,1.619839E-1,7.905754E-1,9.257548E-1,0E0,9.837203E-1,2.8579748E-1,2.1338989E-1,4.4709063E-1,0E0,3.6770678E-1,0E0,4.3525887E-1,0E0,3.1695247E-1,3.1262654E-1,4.842205E-1,5.9906685E-1,0E0,0E0,1.3989142E-1,0E0,3.4679163E-1,9.8072547E-1,1.4344007E-1,7.536564E-1,6.14693E-1,1.0351977E0,1.3082775E-1,3.1741643E-1,0E0,1.4937882E-1,4.1250682E-1,0E0,2.2934914E-1,0E0,2.525038E-1,3.2810283E-1,1.6980529E-1,1.5259004E-1,2.6651946E-1,0E0,1.6876599E-1,3.1933916E-1,8.1352985E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,35,35,37,37,39,39,40,40,41,41,42,42,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,59,59,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,-1,52,54,56,58,-1,60,-1,62,-1,64,66,68,70,-1,-1,72,-1,74,76,78,80,82,84,86,88,-1,90,92,-1,94,-1,96,98,100,102,104,-1,106,108,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,2.0304577E-1,2.3580047E10,3.2196458E6,4.7777777E0,7.969174E7,1.4299594E8,2.5275471E2,2.1944155E3,1.4400111E10,1E0,2.2792218E-2,4.52737E5,5.4091E4,-2.1783056E-2,2.5943396E0,8E0,3.2E1,1.8261148E-2,4.5723195E6,5.8530144E10,7.218466E7,5.208041E-3,2.8721826E7,1.08874E5,2.7504245E10,2.47E2,3.75E1,1.0195312E0,1.6322132E-2,3.7510395E4,5.6553E4,3.7057E5,2.3920168E6,-2.1084866E-2,9.099893E8,4.2811506E-3,1.0323588E3,-7.994268E-3,2.2216E4,5.6812387E0,1E0,4.516183E1,-2.0760732E-2,-6.6180197E-3,4.921397E0,-3.937605E-3,1.56E2,9.2058825E-1,6E0,5.7616882E1,1.2502964E9,1.823E3,4.3698645E6,9.1E3,-1.3537004E-2,2.52E2,8.15E2,2.67368E-3,8.853684E-1,-1.7452479E-3,7.4709034E-1,6.5346925E6,1E0,1.9921019E9,4.5723195E6,-1.1077728E-2,1.2059859E1,4.5575E4,4.13568E5,2.4793867E-2,-8.750407E-4,6.60153E-3,-4.5219874E-3,-2.1515278E-2,7.4300603E-3,-5.661807E-4,8.693243E-3,1.2749265E-3,-1.8071787E-2,-4.560616E-3,-3.8497276E-3,9.119074E-3,4.759187E-3,4.4542472E-4,-4.4210185E-3,3.2148792E-3,9.4114365E-3,3.8121177E-3,6.9137393E-3,-5.5414536E-3,-4.543712E-3,-9.252605E-3,-1.0807908E-2,-2.09527E-2,-2.5205505E-3,-1.0417621E-2,9.208026E-3,7.095071E-5,9.077823E-3,-2.1563484E-3,1.8783804E-2,9.033643E-3,7.0402366E-3,-5.260249E-3,5.050018E-3,-1.1037405E-3,2.8665848E-3,8.697523E-3,1.1204367E-2,-2.1849193E-3],"split_indices":[2,27,5,28,54,7,45,52,4,5,89,0,29,33,0,56,18,3,0,28,5,45,0,50,2,5,8,4,53,0,28,1,9,28,0,7,0,52,0,2,57,102,54,0,0,54,0,2,53,8,50,7,29,45,29,0,0,0,0,27,0,38,45,6,12,28,0,56,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.695E3,2.312E3,3.83E2,1.794E3,5.18E2,3.34E2,4.9E1,1.636E3,1.58E2,2.46E2,2.72E2,1.2E1,3.22E2,4.5E1,4E0,6.26E2,1.01E3,1.52E2,6E0,2.07E2,3.9E1,2.55E2,1.7E1,9.4E1,2.28E2,8E0,3.7E1,3.47E2,2.79E2,1.1E1,9.99E2,1.43E2,9E0,2.01E2,6E0,3.6E1,3E0,2.33E2,2.2E1,6.2E1,3.2E1,1.78E2,5E1,2E0,6E0,2.2E1,1.5E1,6E1,2.87E2,2.4E1,2.55E2,1.42E2,8.57E2,4.5E1,9.8E1,2E0,7E0,1.88E2,1.3E1,3.2E1,4E0,1.73E2,6E1,3.8E1,2.4E1,3E1,2E0,7.9E1,9.9E1,4.6E1,4E0,1.3E1,9E0,5.8E1,2E0,4E1,2.47E2,1E1,1.4E1,9E0,2.46E2,1.34E2,8E0,1.55E2,7.02E2,5E0,4E1,5.4E1,4.4E1,4E0,3E0,1.18E2,7E1,1.2E1,2E1,1.64E2,9E0,1E1,5E1,3.5E1,3E0,1.5E1,9E0,2.6E1,4E0,1.1E1,6.8E1,6.7E1,3.2E1,3.1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-6.4784044E-4,-2.5233878E-2,6.7146726E-2,-2.9729814E-3,-1.0441139E-1,1.2173669E-1,-2.4225794E-2,-2.8662782E-2,4.0451635E-2,-8.489691E-2,-2.6121217E-1,2.703225E-1,1.02194525E-1,-7.1208715E-2,7.001226E-2,-4.888E-2,1.6194943E-2,-3.0950438E-2,6.926519E-2,-1.6115832E-1,-5.157145E-2,-3.7481228E-4,-3.1298804E-1,2.271352E-1,2.2703102E-2,1.5177262E-1,6.309901E-2,-3.5791487E-2,-1.720627E-1,1.972347E-1,3.723425E-2,-2.778102E-2,-1.4111225E-1,1.462751E-1,-5.043668E-3,1.4437805E-1,-5.4920226E-2,2.2730062E-2,1.10283524E-1,-1.2401643E-1,-3.332386E-1,-5.7831737E-3,-9.155573E-2,-4.5460775E-1,-1.9470444E-1,1.5453053E-2,1.4858207E-1,1.0128849E-1,2.6114896E-1,3.5002645E-2,2.1221381E-1,-5.310686E-2,7.0895836E-2,-1.3940993E-1,-1.9727707E-2,1.16133345E-2,-1.0444602E-3,-8.78893E-3,5.6447662E-2,-1.2780482E-1,-1.4302397E-2,-2.0192072E-2,-1.2565279E-1,2.0587362E-1,5.9254025E-4,-1.0444039E-2,1.2944997E-2,1.2630209E-3,9.246121E-3,-1.6242872E-1,-3.3260952E-3,5.8488235E-2,-9.12574E-2,2.1339162E-1,8.904922E-2,-2.1956088E-1,-6.653874E-2,-2.1086987E-2,-1.5344176E-1,4.2145785E-2,-5.1411055E-2,-3.005143E-1,-7.012048E-2,-2.5847124E-2,-6.8202666E-3,1.640371E-3,-2.3076062E-1,-2.7464363E-3,9.033624E-3,7.130596E-2,1.1457292E-2,1.8583316E-2,2.2349165E-1,-4.1750398E-2,7.130681E-2,3.1848434E-1,6.611231E-2,-4.1562174E-2,-1.07891355E-2,1.4931704E-1,-4.0645422E-3,-1.8826324E-1,2.9039374E-3,1.971702E-1,3.402477E-2,-5.310257E-3,-2.082836E-2,7.7159996E-3,-1.5564454E-3,-1.6211657E-2,-5.027136E-3,1.22682145E-2,2.0027268E-3,1.594673E-3,-1.5081464E-3,-2.0092782E-3,-1.6175497E-2,1.1800812E-3,-7.1810232E-3,3.9254725E-3,-1.4969043E-3,-2.0730733E-3,-1.0706605E-2,-3.1575295E-3,1.3000084E-2,4.5381053E-3,-8.90347E-3,-5.0575035E-3,-1.24193635E-2,9.204534E-3,-4.6785614E-3,-1.5336561E-2,-7.982388E-4,1.0422741E-2,7.592345E-4,7.231377E-3,-3.2162883E-3,-4.489937E-3,-1.6913354E-2,-1.4330022E-2,-2.808984E-3,-6.045971E-3,-1.4596844E-2,4.0617674E-3,-4.532039E-3,5.2475873E-3,1.2153368E-2,-2.5568295E-2,-7.67144E-4,9.239419E-3,1.7663981E-3,1.872536E-2,4.027887E-3,6.738187E-3,-1.4558717E-2,-4.4571045E-3,-1.186265E-4,3.2826061E-3,1.2683579E-2,-1.2300761E-2,-2.7637053E-3,1.4255068E-2,2.7415876E-3,-6.081176E-4,6.023156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,-1,-1,-1,-1,103,105,107,-1,109,111,-1,113,-1,-1,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,-1,-1,-1,141,-1,-1,143,-1,-1,145,147,149,151,153,155,-1,157,-1,159,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.440382E0,3.4454985E0,3.5394576E0,1.7046344E0,1.2905159E0,1.2616119E0,1.1827216E0,8.7141395E-1,1.170717E0,9.6506286E-1,6.1415577E-1,3.410716E-1,7.5410604E-1,6.2965196E-1,3.637948E-1,1.2843359E0,8.267916E-1,6.9715357E-1,7.7055454E-1,7.0007396E-1,4.9012786E-1,0E0,5.647862E-1,2.8227258E-1,0E0,9.288149E-1,9.226597E-1,2.4946004E-1,3.0286455E-1,1.8375039E-1,3.1574488E-1,7.26191E-1,4.8042893E-1,3.326288E-1,3.8646358E-1,1.098209E-1,8.037467E-1,7.8385025E-1,4.5172477E-1,5.18083E-1,3.17482E-1,2.7771577E-1,6.1943614E-1,3.7481427E-1,1.9777632E-1,0E0,2.2524536E-1,4.7927403E-1,1.7130351E-1,5.268307E-1,5.22364E-1,2.1923015E-1,2.4476656E-1,4.2187965E-1,0E0,0E0,0E0,0E0,2.0470348E-1,2.8437078E-1,1.5279305E0,0E0,4.8393548E-1,2.285862E-1,0E0,2.348979E-1,0E0,0E0,0E0,9.7859466E-1,4.0965247E-1,3.1020698E-1,2.8227097E-1,6.017543E-1,3.002596E-1,1.3575447E-1,5.1854295E-1,0E0,1.9998728E-1,2.8262094E-1,2.1733229E-1,1.0841322E-1,3.2658082E-1,0E0,0E0,0E0,1.1627698E-1,0E0,0E0,2.3217052E-1,0E0,0E0,1.239779E-1,7.639273E-1,5.287598E-1,3.070519E-1,4.844643E-1,2.1920122E-1,0E0,1.03156835E-1,0E0,2.89631E-1,0E0,1.09506875E-1,2.545042E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,58,58,59,59,60,60,62,62,63,63,65,65,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,79,79,80,80,81,81,82,82,86,86,89,89,92,92,93,93,94,94,95,95,96,96,97,97,99,99,101,101,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,-1,-1,-1,-1,104,106,108,-1,110,112,-1,114,-1,-1,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,-1,-1,-1,142,-1,-1,144,-1,-1,146,148,150,152,154,156,-1,158,-1,160,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0855529E3,2.0304577E-1,2.8839298E7,4.930349E5,1.2199979E10,1.7742857E1,3.970405E3,3.386E3,1E0,1.9121015E0,2.0469315E0,6.5317163E3,5.8805E4,1.0828989E8,8.837831E-2,8.0391425E6,6.4E1,5E-1,6.9432104E7,1.9577E4,1.5316E4,-3.7481228E-4,2.4434783E0,9.397667E6,2.2703102E-2,1.3645918E0,4.4839956E3,4.559E3,2.778E3,4.1221875E-1,3.33231E5,4.7E2,9.073025E6,1.3645918E0,1.96E4,4.86E2,8.15E2,5.9378143E-2,1.3396206E-2,9.38E2,8E0,1.2471935E7,1.3441292E4,3.285E3,2.7777777E0,1.5453053E-2,2.184578E1,7.203918E9,1.0918E4,1.526038E6,3.4231E4,4.5575E4,8.962E3,2.8E1,-1.9727707E-2,1.16133345E-2,-1.0444602E-3,-8.78893E-3,2E0,1.56E2,1E0,-2.0192072E-2,1.394E3,1E0,5.9254025E-4,8.134936E0,1.2944997E-2,1.2630209E-3,9.246121E-3,2.2884E4,9.800051E6,6.710987E7,2.4E1,1.6841E4,1.3543708E7,1E0,2.014576E8,-2.1086987E-2,4.5254502E-2,1E0,5.684E3,1.2352941E0,1.71E2,-2.5847124E-2,-6.8202666E-3,1.640371E-3,1.387623E8,-2.7464363E-3,9.033624E-3,1.9221336E1,1.1457292E-2,1.8583316E-2,1.2974394E6,3.90106E0,1.5851064E0,1.7049885E0,1.5005797E4,1.2E1,-1.07891355E-2,2.8363848E7,-4.0645422E-3,1.2068E4,2.9039374E-3,1.1590965E0,1.5595E4,-5.310257E-3,-2.082836E-2,7.7159996E-3,-1.5564454E-3,-1.6211657E-2,-5.027136E-3,1.22682145E-2,2.0027268E-3,1.594673E-3,-1.5081464E-3,-2.0092782E-3,-1.6175497E-2,1.1800812E-3,-7.1810232E-3,3.9254725E-3,-1.4969043E-3,-2.0730733E-3,-1.0706605E-2,-3.1575295E-3,1.3000084E-2,4.5381053E-3,-8.90347E-3,-5.0575035E-3,-1.24193635E-2,9.204534E-3,-4.6785614E-3,-1.5336561E-2,-7.982388E-4,1.0422741E-2,7.592345E-4,7.231377E-3,-3.2162883E-3,-4.489937E-3,-1.6913354E-2,-1.4330022E-2,-2.808984E-3,-6.045971E-3,-1.4596844E-2,4.0617674E-3,-4.532039E-3,5.2475873E-3,1.2153368E-2,-2.5568295E-2,-7.67144E-4,9.239419E-3,1.7663981E-3,1.872536E-2,4.027887E-3,6.738187E-3,-1.4558717E-2,-4.4571045E-3,-1.186265E-4,3.2826061E-3,1.2683579E-2,-1.2300761E-2,-2.7637053E-3,1.4255068E-2,2.7415876E-3,-6.081176E-4,6.023156E-3],"split_indices":[52,27,45,28,5,58,52,2,102,53,56,4,9,45,57,45,11,53,7,9,9,0,53,45,0,38,52,0,0,38,29,1,45,38,2,12,2,38,41,2,3,45,51,0,53,0,56,12,9,9,10,2,0,8,0,0,0,0,8,2,53,0,12,101,0,58,0,0,0,1,50,7,8,9,47,104,7,0,39,75,9,56,0,0,0,0,7,0,0,53,0,0,28,56,53,38,4,18,0,1,0,9,0,42,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.662E3,1.954E3,7.08E2,1.526E3,4.28E2,4.43E2,2.65E2,9.59E2,5.67E2,3.82E2,4.6E1,5E1,3.93E2,1.77E2,8.8E1,6.61E2,2.98E2,1.63E2,4.04E2,1.15E2,2.67E2,8E0,3.8E1,4.3E1,7E0,1.72E2,2.21E2,1.32E2,4.5E1,1.7E1,7.1E1,5.39E2,1.22E2,4.1E1,2.57E2,1.9E1,1.44E2,1.9E2,2.14E2,9.6E1,1.9E1,1.25E2,1.42E2,1.6E1,2.2E1,1.8E1,2.5E1,1.19E2,5.3E1,1.87E2,3.4E1,1.14E2,1.8E1,4.1E1,4E0,1.4E1,3E0,5E0,6.6E1,6.3E1,4.76E2,5E0,1.17E2,2.8E1,1.3E1,2.53E2,4E0,6E0,1.3E1,4.6E1,9.8E1,1.45E2,4.5E1,3.5E1,1.79E2,3.5E1,6.1E1,1.1E1,8E0,6.1E1,6.4E1,1.2E1,1.3E2,1.2E1,4E0,3E0,1.9E1,4E0,2.1E1,9.9E1,2E1,1E1,4.3E1,6E1,1.27E2,1.9E1,1.5E1,1.08E2,6E0,1.2E1,6E0,3.3E1,8E0,8E0,5.8E1,6.1E1,2E0,4.4E1,4.32E2,9E0,1.08E2,2.1E1,7E0,8.2E1,1.71E2,2.8E1,1.8E1,8.3E1,1.5E1,1.15E2,3E1,3.4E1,1.1E1,6E0,2.9E1,1.76E2,3E0,1E1,2.5E1,6E0,5.5E1,3E0,5E0,7E0,5.4E1,4E0,6E1,3E0,9E0,5E0,1.25E2,9E0,1E1,9.2E1,7E0,1E1,3.3E1,2E0,5.8E1,2.7E1,1E2,1.4E1,5E0,1.3E1,2E0,4.6E1,6.2E1,8E0,4E0,2.1E1,1.2E1,4E0,4E0,3.9E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-2.0076171E-3,-4.870162E-2,2.691012E-2,-1.46295745E-2,-9.4747275E-2,1.2061991E-2,1.4380443E-1,-6.1810456E-2,4.7111306E-2,-7.5717464E-2,-2.0249148E-1,7.641427E-2,-1.6111728E-2,1.11326136E-1,2.3553684E-1,-4.6479728E-2,-2.3554066E-1,1.4968385E-2,2.8191643E-2,-1.8357475E-1,-6.293492E-2,-3.5883608E-1,-1.197171E-1,5.0940625E-2,1.8490209E-1,-3.0853827E-2,3.151756E-2,1.1905103E-1,-1.3463357E-2,1.4403536E-2,3.4479382E-3,1.5837513E-1,-5.7432942E-2,2.2436916E-3,-4.326127E-1,3.5272256E-2,-1.9163532E-2,-2.1806498E-1,-1.085534E-3,2.7006788E-2,-8.186498E-2,-4.4853476E-1,-2.1495116E-3,-1.9094941E-1,-3.260793E-2,5.5944543E-2,-2.7370095E-2,6.835099E-2,2.4403836E-1,-3.582238E-2,8.74366E-3,1.6838523E-2,1.8589413E-1,8.9987904E-2,1.8116753E-1,1.102255E-2,1.6582279E-2,7.476529E-2,-6.783779E-2,-2.9447448E-2,-5.2877665E-3,2.2065572E-2,2.171359E-1,-2.3357237E-3,-2.7246377E-1,-6.7885496E-2,1.41986E-1,-1.414822E-1,-5.65054E-2,-2.3962684E-2,-1.9098137E-3,-2.5674424E-1,-3.5037773E-3,1.6865251E-3,-6.0798516E-3,-8.173761E-2,7.042154E-2,9.119173E-3,7.3310995E-4,1.08240165E-1,2.9565164E-1,1.11089855E-1,-4.1332815E-2,1.9497922E-1,2.1272635E-3,1.2944463E-2,3.1620425E-3,1.4643398E-1,3.28573E-2,2.9542074E-1,1.09409705E-1,-7.828165E-3,3.5954067E-3,-5.474116E-3,5.2983453E-3,-2.0778517E-3,-8.718658E-3,3.1603104E-3,-7.1065925E-4,-1.5013763E-3,1.5944395E-2,-1.690991E-2,-7.777802E-3,1.195298E-3,-9.370812E-3,2.592198E-3,9.993042E-3,-1.0332727E-2,-2.5888206E-3,-1.461737E-3,-7.23384E-3,-1.6775817E-2,-4.7788485E-3,5.0653187E-3,-5.9470087E-3,7.377344E-3,2.451832E-3,1.4215062E-2,2.3601492E-4,1.4931703E-2,3.4709016E-3,2.494668E-3,1.2072032E-2,-4.472754E-3,-1.4506669E-3,3.479822E-3,1.2216497E-2,1.4978254E-3,-2.4273016E-3,1.146933E-2,3.956574E-3,5.33965E-3,-1.4417202E-3,8.191524E-3,2.2709826E-2,-4.4733007E-3,7.0449086E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,57,-1,59,61,-1,63,-1,65,67,69,-1,71,73,75,-1,77,79,81,-1,83,85,87,89,91,-1,93,95,-1,-1,97,99,-1,101,103,105,107,109,-1,-1,111,-1,-1,-1,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,131,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6295266E0,1.6114118E0,2.879464E0,1.7274017E0,8.80672E-1,2.6736565E0,5.309098E-1,8.839581E-1,1.2818654E0,5.026982E-1,7.982273E-1,1.2314284E0,7.213841E-1,4.5790148E-1,4.9034142E-1,7.046798E-1,1.5228555E0,0E0,7.450532E-1,2.1149719E-1,5.725825E-1,5.9941244E-1,2.6679283E-1,1.1688423E0,5.691736E-1,8.379234E-1,5.475321E-1,2.328285E-1,0E0,0E0,0E0,4.391339E-1,4.1084504E-1,0E0,8.717849E-1,5.66841E-1,0E0,2.8140962E-1,0E0,6.5336657E-1,4.1079283E-1,3.401091E-1,0E0,1.67211E-1,1.3884737E-1,7.292963E-1,0E0,1.9132736E-1,3.60502E-1,6.2551636E-1,0E0,5.839956E-1,1.9460332E-1,3.0476916E-1,3.2021618E-1,1.24032766E-1,0E0,1.6184548E-1,7.572776E-1,0E0,0E0,3.6246973E-1,4.7082555E-1,0E0,1.5562928E-1,3.96403E-1,1.4687818E-1,5.208039E-1,4.7687E-1,0E0,0E0,1.8529743E-1,0E0,0E0,0E0,2.9166663E-1,5.2103865E-1,0E0,0E0,3.2974762E-1,1.2145424E-1,2.1977279E-1,4.2255914E-1,1.0646266E-1,3.2023963E-1,0E0,0E0,2.6526618E-1,2.4597E-1,2.7466238E-1,2.2780353E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,35,35,37,37,39,39,40,40,41,41,43,43,44,44,45,45,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,57,57,58,58,61,61,62,62,64,64,65,65,66,66,67,67,68,68,71,71,75,75,76,76,79,79,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,58,-1,60,62,-1,64,-1,66,68,70,-1,72,74,76,-1,78,80,82,-1,84,86,88,90,92,-1,94,96,-1,-1,98,100,-1,102,104,106,108,110,-1,-1,112,-1,-1,-1,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,132,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.9173258E-4,1.3664E4,2.41E3,4.7022E4,2E0,6.357613E-1,4.3978744E0,8E0,7.22E2,6.258335E5,2.84E2,1.3593E4,6.13912E5,3.1029554E7,7E0,1.3E1,1.4968385E-2,4.8916136E7,9.824211E5,5.5700205E8,1E0,1.010368E6,9.216E3,6.189387E6,1.5963264E9,2.3565867E3,8.015419E3,-1.3463357E-2,1.4403536E-2,3.4479382E-3,1.3984146E2,4.144144E-1,2.2436916E-3,9.99E2,3.832274E2,-1.9163532E-2,1.6358411E8,-1.085534E-3,1.21E3,4.1365924E2,1.91E2,-2.1495116E-3,2.5030267E5,2.9964134E9,1.5288235E2,-2.7370095E-2,1.2469E5,1.44657E5,9.61E2,8.74366E-3,8.963196E6,2.0189162E7,1.0254098E7,8.454878E1,3.6E1,1.6582279E-2,1E0,3.37E2,-2.9447448E-2,-5.2877665E-3,1.6837112E7,1E1,-2.3357237E-3,5.85E2,2.047619E0,3.8977896E2,1.2123E4,1.1855755E8,-2.3962684E-2,-1.9098137E-3,9.026015E2,-3.5037773E-3,1.6865251E-3,-6.0798516E-3,7.427971E2,3.5452E4,9.119173E-3,7.3310995E-4,1.366E3,2.8863275E0,2.5682794E5,8.87E2,1.3794063E1,1.1756504E5,1.2944463E-2,3.1620425E-3,1.526038E6,1.2469E5,1.21320024E8,2.4394053E9,-7.828165E-3,3.5954067E-3,-5.474116E-3,5.2983453E-3,-2.0778517E-3,-8.718658E-3,3.1603104E-3,-7.1065925E-4,-1.5013763E-3,1.5944395E-2,-1.690991E-2,-7.777802E-3,1.195298E-3,-9.370812E-3,2.592198E-3,9.993042E-3,-1.0332727E-2,-2.5888206E-3,-1.461737E-3,-7.23384E-3,-1.6775817E-2,-4.7788485E-3,5.0653187E-3,-5.9470087E-3,7.377344E-3,2.451832E-3,1.4215062E-2,2.3601492E-4,1.4931703E-2,3.4709016E-3,2.494668E-3,1.2072032E-2,-4.472754E-3,-1.4506669E-3,3.479822E-3,1.2216497E-2,1.4978254E-3,-2.4273016E-3,1.146933E-2,3.956574E-3,5.33965E-3,-1.4417202E-3,8.191524E-3,2.2709826E-2,-4.4733007E-3,7.0449086E-3],"split_indices":[102,27,2,2,9,17,38,53,18,2,32,0,10,2,47,3,0,0,48,32,5,16,9,10,45,7,52,4,0,0,0,52,53,0,2,56,0,7,0,2,52,8,0,33,12,52,0,9,1,12,0,12,45,45,56,2,0,15,0,0,0,45,3,0,2,58,4,10,45,0,0,52,0,0,0,33,9,0,0,10,53,28,2,56,33,0,0,9,9,32,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.686E3,1.027E3,1.659E3,5.91E2,4.36E2,1.473E3,1.86E2,3.35E2,2.56E2,3.72E2,6.4E1,4.48E2,1.025E3,1.39E2,4.7E1,3.09E2,2.6E1,1.6E1,2.4E2,3.8E1,3.34E2,2.1E1,4.3E1,3.64E2,8.4E1,7.83E2,2.42E2,1.37E2,2E0,3.3E1,1.4E1,1.5E1,2.94E2,1.1E1,1.5E1,2.37E2,3E0,3.1E1,7E0,5.8E1,2.76E2,1.6E1,5E0,2.3E1,2E1,3.62E2,2E0,2.9E1,5.5E1,7.66E2,1.7E1,2.22E2,2E1,9.5E1,4.2E1,9E0,6E0,2.1E1,2.73E2,9E0,6E0,2.22E2,1.5E1,8E0,2.3E1,3.2E1,2.6E1,8.1E1,1.95E2,1.4E1,2E0,1.4E1,9E0,1.2E1,8E0,3.4E1,3.28E2,8E0,2.1E1,1.6E1,3.9E1,2.7E1,7.39E2,1.6E1,2.06E2,1.1E1,9E0,4.7E1,4.8E1,1.5E1,2.7E1,2E0,7E0,3E0,1.8E1,2.26E2,4.7E1,1.01E2,1.21E2,5E0,1E1,1.2E1,1.1E1,1.9E1,1.3E1,1.2E1,1.4E1,4.3E1,3.8E1,1.54E2,4.1E1,8E0,6E0,6E0,2.8E1,6E1,2.68E2,5E0,1.1E1,3.6E1,3E0,2E1,7E0,1.28E2,6.11E2,6E0,1E1,1.33E2,7.3E1,1.8E1,2.9E1,2.1E1,2.7E1,1E1,5E0,4E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-8.2744757E-4,1.9621057E-2,-6.8965584E-2,-2.013101E-2,5.1293805E-2,-1.5249503E-1,-4.2346045E-2,-7.2814627E-3,-1.2626216E-1,7.463365E-2,-2.0917306E-2,-3.724339E-1,-1.24889635E-1,-6.0125694E-2,5.6030158E-2,3.5122536E-2,-3.7350655E-2,-3.8979778E-1,-8.137085E-2,9.5279776E-2,-4.5441445E-2,-1.3487735E-1,8.950262E-3,-5.543325E-3,-2.1471694E-2,-2.6584837E-1,-9.608623E-2,-2.9694492E-2,-1.3897687E-1,9.067925E-3,3.640702E-2,4.712802E-2,-8.9304514E-2,-4.887578E-2,3.481073E-2,1.8499856E-3,-4.6442145E-1,-4.080366E-2,-2.0306118E-1,1.9394626E-1,8.3773814E-2,-6.781334E-2,1.0330686E-1,-3.212199E-1,-9.863815E-2,2.2511403E-3,1.7886618E-2,6.152125E-3,-3.1170252E-1,-1.7401959E-1,4.0315334E-3,-6.1633077E-2,2.8837575E-2,-1.7458285E-1,-6.400877E-2,9.983201E-3,1.8293167E-2,-9.967301E-2,5.6046546E-2,4.318558E-3,-1.8355335E-1,-2.2776458E-2,-9.74474E-2,8.5357316E-2,-2.8690418E-2,-2.6462939E-2,-7.182562E-3,-1.0006911E-3,-1.0369102E-2,-1.4492958E-2,-3.6681868E-2,1.2030374E-1,1.6252365E-2,8.669115E-2,-1.6482532E-1,-4.4527132E-2,-2.399303E-1,2.4771935E-1,-4.315996E-3,2.738769E-3,-4.375062E-1,-5.327094E-3,3.5454326E-3,3.939288E-2,-5.0425075E-2,-7.967605E-3,-1.8062398E-2,-1.3924716E-1,-3.443887E-1,8.980047E-3,-3.5381686E-2,-2.6258139E-2,-9.313039E-2,5.2965283E-3,-6.5458374E-4,-9.470135E-2,-2.0655769E-1,2.707285E-3,-1.0477255E-1,9.516381E-3,8.476554E-3,1.3646448E-3,-2.1005223E-2,7.006528E-3,1.8649291E-3,-1.1596156E-2,-3.1324178E-3,-2.9935504E-3,2.6431184E-3,-1.2093748E-2,-3.2892714E-3,6.209021E-3,1.59456E-4,5.3032767E-3,-3.2642884E-3,3.3677008E-3,-4.457775E-3,-8.027926E-3,2.7168791E-3,4.0448373E-3,1.4696961E-2,2.9768997E-3,5.7863053E-3,-1.2466742E-2,3.8839544E-3,-3.032495E-3,1.0130233E-2,-6.3031698E-3,-1.649162E-2,1.8899888E-2,6.147796E-3,-2.523449E-2,-6.8314625E-3,-6.074895E-4,4.276421E-3,3.692465E-4,-3.723094E-3,-1.4121617E-2,-5.258141E-3,-2.2867095E-2,-8.432352E-3,-4.953844E-3,1.493814E-3,-2.7949866E-3,2.790207E-3,-6.4102923E-3,-2.3739735E-4,-9.020644E-3,-1.0075304E-3,-1.2667972E-2,-6.852777E-3,-6.663238E-4,-7.7939588E-3,-8.9466624E-4,3.51884E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,-1,-1,85,87,89,91,93,95,97,-1,99,101,103,-1,105,107,109,111,113,-1,-1,115,-1,-1,117,119,-1,121,123,125,127,129,-1,-1,131,-1,-1,133,135,-1,-1,137,139,-1,141,143,145,-1,-1,147,149,-1,151,-1,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.706199E0,2.5770636E0,1.3587806E0,1.2373542E0,1.921227E0,8.5206866E-1,8.2019585E-1,1.0353805E0,1.1276667E0,2.1388226E0,9.5084214E-1,2.5988436E-1,5.144477E-1,9.460896E-1,1.814652E-1,5.0762165E-1,3.967442E-1,4.6553183E-1,4.1132838E-1,8.1816244E-1,4.296337E-1,3.5998404E-1,5.41304E-1,0E0,0E0,4.3781924E-1,8.74583E-1,5.383041E-1,2.8244948E-1,0E0,1.9564386E-1,4.0883493E-1,5.2402586E-1,5.175318E-1,2.1429354E-1,0E0,2.4974322E-1,4.5438692E-1,3.367778E-1,7.7305627E-1,4.9295187E-1,4.3419904E-1,4.9719554E-1,4.210531E-1,1.1698821E-1,4.304569E-1,0E0,0E0,1.275121E-1,3.2526588E-1,3.6821216E-1,2.047987E-1,3.559861E-1,1.6944933E-1,1.877277E-1,0E0,1.02810085E-1,7.944324E-1,4.4356042E-1,0E0,1.225943E-1,8.3365345E-1,6.2055135E-1,1.3399854E-1,1.726489E-1,0E0,0E0,3.611862E-1,0E0,0E0,1.1960067E-1,3.2184684E-1,0E0,5.392399E-1,2.0898385E-1,4.8982006E-1,1.0172337E-1,1.2469363E-1,0E0,0E0,1.09387755E-1,0E0,0E0,3.363822E-1,1.4511311E-1,0E0,0E0,2.1513426E-1,1.3994789E-1,0E0,1.9352753E-1,2.4489534E-1,3.512199E-1,0E0,0E0,1.5471806E-1,1.5343356E-1,0E0,1.4522144E-1,0E0,1.0199347E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,67,67,70,70,71,71,73,73,74,74,75,75,76,76,77,77,80,80,83,83,84,84,87,87,88,88,90,90,91,91,92,92,95,95,96,96,98,98,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,-1,-1,86,88,90,92,94,96,98,-1,100,102,104,-1,106,108,110,112,114,-1,-1,116,-1,-1,118,120,-1,122,124,126,128,130,-1,-1,132,-1,-1,134,136,-1,-1,138,140,-1,142,144,146,-1,-1,148,150,-1,152,-1,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,3.6129813E2,1.9121015E0,9.669789E0,1.2328733E1,7E0,3.4409692E3,2E0,2E0,3.5301748E7,4.052E3,8.3933276E-1,1.2068E4,4.987639E7,1.4274633E7,2E0,2.437E3,8E0,1.4773677E5,9.42812E3,3.970405E3,4.745907E7,6.249511E9,-5.543325E-3,-2.1471694E-2,5.5544252E7,1.638058E2,5.674727E2,1.1832823E-2,9.067925E-3,1.5151515E-3,1.198E5,7.106795E6,2.4505E4,1.3152658E7,1.8499856E-3,1.1160287E7,2.970914E1,2.074E3,1.0485785E3,1.5005797E4,5.860658E-1,2.6422684E9,2.1325E4,1E0,2.7753687E-2,1.7886618E-2,6.152125E-3,6.41689E5,1.4977E5,8.640299E8,1.5773E4,1.25218E5,2.8277853E11,4.88954E5,9.983201E-3,4.3525005E3,1.08504E5,8.916961E2,4.318558E-3,2.2417E0,7.22E2,9E2,3.333E3,7.57657E4,-2.6462939E-2,-7.182562E-3,9.4E1,-1.0369102E-2,-1.4492958E-2,1.0238709E1,2.3369722E-3,1.6252365E-2,1.0855529E3,4.5505118E2,2.2216E4,8.637973E6,6.082581E3,-4.315996E-3,2.738769E-3,6.203E3,-5.327094E-3,3.5454326E-3,6.222E3,1.17E2,-7.967605E-3,-1.8062398E-2,4.26E2,1.05E2,8.980047E-3,7.804559E2,1.4157E4,1.3225807E0,5.2965283E-3,-6.5458374E-4,1E0,3.6282136E0,2.707285E-3,6.1238285E8,9.516381E-3,2.2153166E7,1.3646448E-3,-2.1005223E-2,7.006528E-3,1.8649291E-3,-1.1596156E-2,-3.1324178E-3,-2.9935504E-3,2.6431184E-3,-1.2093748E-2,-3.2892714E-3,6.209021E-3,1.59456E-4,5.3032767E-3,-3.2642884E-3,3.3677008E-3,-4.457775E-3,-8.027926E-3,2.7168791E-3,4.0448373E-3,1.4696961E-2,2.9768997E-3,5.7863053E-3,-1.2466742E-2,3.8839544E-3,-3.032495E-3,1.0130233E-2,-6.3031698E-3,-1.649162E-2,1.8899888E-2,6.147796E-3,-2.523449E-2,-6.8314625E-3,-6.074895E-4,4.276421E-3,3.692465E-4,-3.723094E-3,-1.4121617E-2,-5.258141E-3,-2.2867095E-2,-8.432352E-3,-4.953844E-3,1.493814E-3,-2.7949866E-3,2.790207E-3,-6.4102923E-3,-2.3739735E-4,-9.020644E-3,-1.0075304E-3,-1.2667972E-2,-6.852777E-3,-6.663238E-4,-7.7939588E-3,-8.9466624E-4,3.51884E-3],"split_indices":[27,52,53,54,53,3,52,17,17,45,2,56,9,45,45,6,2,0,33,47,52,7,7,0,0,45,52,52,38,0,57,7,5,29,45,0,45,58,2,52,4,41,7,1,101,57,0,0,29,29,5,9,29,31,32,0,4,7,33,0,53,2,2,2,28,0,0,3,0,0,53,41,0,52,56,2,47,52,0,0,29,0,0,2,3,0,0,2,52,0,55,9,57,0,0,16,53,0,7,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.658E3,2.045E3,6.13E2,9.07E2,1.138E3,1.47E2,4.66E2,8.1E2,9.7E1,8.6E2,2.78E2,1.5E1,1.32E2,3.95E2,7.1E1,3.36E2,4.74E2,1.3E1,8.4E1,7.34E2,1.26E2,5.7E1,2.21E2,4E0,1.1E1,2.1E1,1.11E2,2.86E2,1.09E2,8E0,6.3E1,3.07E2,2.9E1,4.09E2,6.5E1,2E0,1.1E1,6.4E1,2E1,7.5E1,6.59E2,1.1E2,1.6E1,8E0,4.9E1,2.18E2,3E0,2E0,1.9E1,6.2E1,4.9E1,1.85E2,1.01E2,7.3E1,3.6E1,5E0,5.8E1,1.7E1,2.9E2,1E1,1.9E1,2.67E2,1.42E2,3.6E1,2.9E1,8E0,3E0,5.3E1,1.1E1,1.2E1,8E0,5.1E1,2.4E1,6.52E2,7E0,9.8E1,1.2E1,9E0,7E0,2E0,6E0,4.6E1,3E0,1.28E2,9E1,7E0,1.2E1,5.3E1,9E0,8E0,4.1E1,8.8E1,9.7E1,3.4E1,6.7E1,2.2E1,5.1E1,9E0,2.7E1,2E0,5.6E1,1.3E1,4E0,4.5E1,2.45E2,1.2E1,7E0,1.77E2,9E1,2.1E1,1.21E2,2.3E1,1.3E1,6E0,2.3E1,3E1,2.3E1,3E0,5E0,4.4E1,7E0,3.81E2,2.71E2,5E0,2E0,9.2E1,6E0,7E0,5E0,3E0,6E0,4E0,2E0,6.3E1,6.5E1,2.9E1,6.1E1,7E0,4.6E1,4E0,5E0,2E1,2.1E1,6.4E1,2.4E1,6.6E1,3.1E1,9E0,1.3E1,2.5E1,2.6E1,1.1E1,1.6E1,4E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"155","size_leaf_vector":"1"}},{"base_weights":[2.430006E-3,2.2013843E-2,-6.165792E-2,-1.843684E-2,5.102397E-2,-1.5407607E-1,-3.4091175E-2,-3.291846E-2,4.70343E-2,5.74579E-3,8.36082E-2,-1.2179068E-1,-3.028829E-1,-1.3823505E-1,-1.906325E-2,-5.1987424E-2,3.1201081E-2,3.613268E-2,1.767337E-2,1.3111398E-2,-1.3770073E-3,7.613562E-2,2.268644E-1,-1.8837102E-1,-4.233509E-2,-4.034121E-1,-1.8596771E-3,-1.6830054E-1,7.921827E-3,-2.6918573E-2,6.036869E-3,-2.1479659E-2,-9.2484966E-2,9.497761E-2,-4.0435057E-2,7.694717E-2,-3.520908E-2,4.598737E-3,-2.886726E-1,6.544445E-2,2.0314649E-1,1.6793769E-2,4.666612E-2,1.1328447E-3,-2.1033391E-1,1.5810223E-1,-8.9345425E-2,-6.2022954E-3,-4.4955856E-1,-1.3184215E-1,-1.4628779E-2,-1.2782974E-2,-1.004445E-1,-9.7438216E-2,-3.3090517E-3,-1.4782019E-1,-2.817866E-2,1.11501314E-1,-1.3008677E-2,5.073272E-3,-1.2989652E-1,4.8806034E-2,8.756796E-3,-1.08773865E-1,3.9953263E-3,-1.58949E-2,9.026998E-2,-7.5401925E-3,-2.1366378E-2,7.489477E-2,-1.8247152E-2,1.1399026E-1,2.6685154E-1,-2.9221678E-2,1.2579071E-2,-1.3719773E-1,-2.7196664E-1,1.4517191E-2,-3.666547E-3,2.1300158E-3,-1.3788493E-1,-2.4576794E-2,-1.0519857E-2,-3.5506224E-3,-9.434639E-3,-6.7062594E-2,4.4514327E-3,-1.6625528E-1,-3.1589944E-2,-3.4413224E-3,-1.745361E-2,5.5479766E-3,-1.5632209E-3,1.1714168E-3,-8.756439E-3,2.4712048E-3,-2.9694063E-3,2.4527514E-3,8.382178E-3,-1.0500026E-2,1.114982E-3,-1.1319804E-3,-1.0501816E-2,4.5393975E-3,-1.0005847E-4,-1.6248632E-2,-1.4353594E-4,-2.4530757E-4,-5.813163E-3,1.4523773E-2,3.3239075E-3,2.0644653E-3,5.085096E-3,-2.1986805E-3,3.9200336E-3,-2.0021747E-3,7.6428503E-3,1.4582985E-3,1.3564273E-2,3.8757117E-3,-9.559799E-3,-1.7313039E-2,-3.6110622E-3,-7.91577E-3,-1.794636E-2,-1.3317644E-2,-4.6889824E-3,-1.5758965E-2,-2.1922418E-3,-1.9221816E-3,1.7531908E-3,-7.546665E-4,-9.535746E-3,3.8925186E-3,-6.8292455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,49,-1,51,-1,53,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,79,-1,81,83,-1,85,87,89,91,93,95,97,-1,99,101,103,-1,105,-1,107,109,-1,-1,111,113,115,117,119,-1,121,123,-1,-1,-1,125,-1,-1,-1,-1,127,129,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3675022E0,2.412441E0,1.5946827E0,8.1599045E-1,1.7655191E0,6.549289E-1,7.5642186E-1,8.62398E-1,5.342803E-1,9.587665E-1,7.2779083E-1,6.268468E-1,6.4795566E-1,5.8473253E-1,4.8787424E-1,6.688844E-1,7.4397063E-1,4.452995E-1,0E0,0E0,8.4256196E-1,8.869896E-1,7.4215996E-1,3.076613E-1,5.417927E-1,1.6929102E-1,0E0,2.3976731E-1,0E0,4.1860655E-1,0E0,4.289238E-1,8.249426E-1,5.5125594E-1,3.1414467E-1,2.8015834E-1,4.9780795E-1,8.459339E-1,1.3640732E-1,4.861958E-1,2.592466E-1,0E0,2.5354832E-1,0E0,2.2863412E-1,3.9757478E-1,3.0493316E-1,0E0,1.0668731E-1,1.5835255E-1,0E0,3.183607E-1,2.8928715E-1,3.8362014E-1,8.845579E-1,7.586863E-1,2.973447E-1,3.0890357E-1,0E0,2.175051E-1,2.4178302E-1,1.8273477E-1,0E0,8.523849E-1,0E0,4.4143808E-1,3.9223915E-1,0E0,0E0,5.433972E-1,1.7821369E-1,1.7007157E-1,1.03693485E-1,2.4138471E-1,0E0,3.747279E-1,2.779231E-1,0E0,0E0,0E0,1.6332322E-1,0E0,0E0,0E0,0E0,4.4225517E-1,3.7282962E-1,1.6005915E-1,4.2507306E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,63,63,65,65,66,66,69,69,70,70,71,71,72,72,73,73,75,75,76,76,80,80,85,85,86,86,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,50,-1,52,-1,54,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,80,-1,82,84,-1,86,88,90,92,94,96,98,-1,100,102,104,-1,106,-1,108,110,-1,-1,112,114,116,118,120,-1,122,124,-1,-1,-1,126,-1,-1,-1,-1,128,130,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,3.3030225E2,1.9121015E0,1.5859042E6,1.5166431E5,1.1034263E10,1.4123257E-4,4.264897E6,1.819149E1,8E0,5.528107E3,1.4268628E2,9.099893E8,3.1998687E3,9.609747E3,2.915723E0,2.46E2,8E0,1.767337E-2,1.3111398E-2,1.2427474E8,5.0083565E6,3.526947E7,1.3177E4,2.6011018E8,4.987639E7,-1.8596771E-3,1E0,7.921827E-3,7.218466E7,6.036869E-3,3.75E1,1.363E3,1E0,1.6E1,6.789622E7,1.63E3,1.4777102E9,1.6213043E3,2.7629982E7,7E0,1.6793769E-2,1.9013363E4,1.1328447E-3,1.16146E5,2.5927516E11,1.4195741E0,-6.2022954E-3,5.3842196E9,1.2283395E-2,-1.4628779E-2,1.513E3,1.2204E4,3.09627E5,3.9E2,4.09E2,5.07E2,7.59E2,-1.3008677E-2,6.061028E4,1.1383217E5,1.7463945E4,8.756796E-3,1.8122449E2,3.9953263E-3,7.806529E1,8.538E3,-7.5401925E-3,-2.1366378E-2,6.9664386E2,8.36E2,1.86E2,5.76E3,2.6284674E2,1.2579071E-2,5.23E2,2.4968571E2,1.4517191E-2,-3.666547E-3,2.1300158E-3,3.6482175E5,-2.4576794E-2,-1.0519857E-2,-3.5506224E-3,-9.434639E-3,4.2378342E-1,3.9134903E5,6.258335E5,4.1975945E1,-3.4413224E-3,-1.745361E-2,5.5479766E-3,-1.5632209E-3,1.1714168E-3,-8.756439E-3,2.4712048E-3,-2.9694063E-3,2.4527514E-3,8.382178E-3,-1.0500026E-2,1.114982E-3,-1.1319804E-3,-1.0501816E-2,4.5393975E-3,-1.0005847E-4,-1.6248632E-2,-1.4353594E-4,-2.4530757E-4,-5.813163E-3,1.4523773E-2,3.3239075E-3,2.0644653E-3,5.085096E-3,-2.1986805E-3,3.9200336E-3,-2.0021747E-3,7.6428503E-3,1.4582985E-3,1.3564273E-2,3.8757117E-3,-9.559799E-3,-1.7313039E-2,-3.6110622E-3,-7.91577E-3,-1.794636E-2,-1.3317644E-2,-4.6889824E-3,-1.5758965E-2,-2.1922418E-3,-1.9221816E-3,1.7531908E-3,-7.546665E-4,-9.535746E-3,3.8925186E-3,-6.8292455E-3],"split_indices":[27,52,53,28,28,5,42,9,56,18,52,52,7,4,52,56,10,8,0,0,45,28,50,9,7,45,0,101,0,45,0,4,2,74,3,7,10,7,4,45,3,0,52,0,1,31,53,0,12,57,0,2,2,9,12,29,11,2,0,28,47,50,0,52,0,53,2,0,0,52,8,0,12,58,0,29,4,0,0,0,28,0,0,0,0,27,28,32,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.681E3,2.054E3,6.27E2,8.58E2,1.196E3,1.43E2,4.84E2,7.03E2,1.55E2,5.01E2,6.95E2,1.19E2,2.4E1,6E1,4.24E2,5.42E2,1.61E2,1.51E2,4E0,1.2E1,4.89E2,6.62E2,3.3E1,6.4E1,5.5E1,1.7E1,7E0,5.5E1,5E0,4.03E2,2.1E1,3.1E2,2.32E2,8.5E1,7.6E1,9.6E1,5.5E1,4.8E2,9E0,6.12E2,5E1,1.9E1,1.4E1,6E0,5.8E1,1E1,4.5E1,3E0,1.4E1,4.5E1,1E1,3.39E2,6.4E1,5.9E1,2.51E2,1.24E2,1.08E2,8.2E1,3E0,5.1E1,2.5E1,7.7E1,1.9E1,3.4E1,2.1E1,3.88E2,9.2E1,6E0,3E0,5.5E2,6.2E1,2.2E1,2.8E1,1.1E1,3E0,2.8E1,3E1,6E0,4E0,1.2E1,3.3E1,1E1,4E0,2.5E1,2E1,8.1E1,2.58E2,3.2E1,3.2E1,5.5E1,4E0,4.9E1,2.02E2,2.1E1,1.03E2,3.2E1,7.6E1,4.3E1,3.9E1,3E0,4.8E1,1.2E1,1.3E1,4E1,3.7E1,1E1,2.4E1,3.53E2,3.5E1,7E0,8.5E1,2.74E2,2.76E2,4.9E1,1.3E1,5E0,1.7E1,2E0,2.6E1,7E0,4E0,5E0,2.3E1,1.6E1,1.4E1,6E0,2.7E1,5E0,7.6E1,1.08E2,1.5E2,6E0,2.6E1,1.6E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-2.8165912E-3,-3.134089E-2,4.0566783E-2,-1.644277E-2,-1.0840408E-1,-1.5399587E-2,6.9172494E-2,-2.97465E-2,4.663593E-2,-2.058015E-1,-7.214112E-2,-9.471575E-3,-2.5748866E-2,1.1089835E-2,9.146348E-2,-4.7077443E-2,2.8751638E-2,1.2124591E-1,-8.567262E-3,-1.4605333E-1,-3.834839E-1,1.5391331E-2,-9.6917994E-2,6.146762E-2,-4.371939E-2,-1.7754085E-2,2.1026853E-2,2.6861243E-2,1.0980594E-1,-5.8710698E-2,4.8222035E-2,6.253108E-2,-7.970315E-2,2.3208415E-2,1.5718843E-1,-7.647571E-2,2.9342748E-2,-1.3010666E-2,-9.5293835E-2,-4.463491E-1,-3.2366607E-3,5.2318115E-2,-6.222081E-3,-8.330702E-2,-2.453677E-1,8.23365E-2,-1.10211715E-1,-8.841E-2,1.7079742E-3,1.2479577E-1,-7.2880653E-3,1.4538383E-1,-2.3846624E-2,1.1438519E-1,-1.2854796E-2,-4.4655234E-2,-1.2179069E-1,2.7690897E-2,8.917724E-3,4.7147576E-2,1.6398251E-2,-4.96436E-2,-1.9464694E-2,-6.285667E-3,4.699354E-2,2.1169415E-1,6.202944E-2,-1.3899727E-1,1.3505295E-2,7.4186022E-3,1.3077052E-2,1.2543705E-2,-2.1514289E-1,-2.4759423E-2,-1.0333674E-2,-9.0728066E-4,5.4272506E-3,-1.5640152E-1,-5.1876597E-2,-1.5811814E-2,3.2420421E-4,1.0035618E-1,-1.4975682E-3,-8.164625E-3,3.2577426E-3,-7.626899E-2,-1.3181363E-2,5.542723E-2,-8.002565E-2,8.648715E-2,1.598817E-2,9.169883E-2,-2.958117E-2,2.7000636E-1,3.278678E-2,-4.5948718E-2,1.517529E-2,1.4945926E-1,8.2174994E-2,-1.6612294E-3,-7.2858394E-3,-9.670467E-3,-3.3549904E-3,-3.8810528E-3,2.1755775E-3,3.2062686E-4,4.276851E-3,-8.712922E-3,-8.441064E-4,3.3039635E-3,-6.5758387E-3,1.1058788E-2,3.771312E-4,-2.9250695E-3,6.4085117E-3,2.9682626E-3,-7.881385E-3,4.7932426E-3,-3.5584883E-3,3.0525445E-3,-1.5838477E-3,-2.7941186E-3,6.8909656E-3,-5.215821E-3,-1.3529135E-2,-8.940751E-3,-3.888412E-4,3.643264E-3,-3.3817843E-3,2.0058337E-3,6.243447E-3,-2.3550924E-3,-8.74005E-3,3.385721E-3,-7.968129E-3,-7.509882E-4,-7.7586165E-3,7.2912155E-3,1.9321596E-4,8.458297E-3,-9.515826E-4,-3.9939806E-3,6.2861905E-4,1.4442653E-2,1.5649493E-3,-8.676684E-4,6.9967853E-3,-1.6346594E-2,-1.187579E-3,1.7703604E-2,6.1230226E-3,2.1020526E-3,8.330972E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,-1,77,79,81,83,85,87,89,91,93,95,97,-1,99,101,103,-1,105,-1,107,-1,-1,109,111,113,115,117,-1,119,121,123,-1,-1,-1,-1,125,127,-1,-1,129,-1,-1,-1,131,-1,133,135,137,-1,139,141,143,145,147,-1,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2942064E0,1.8420608E0,1.6916767E0,1.1317654E0,9.0345216E-1,1.1032652E0,9.039948E-1,1.129744E0,9.689353E-1,6.8712497E-1,4.157288E-1,8.651642E-1,0E0,7.4763924E-1,5.954571E-1,9.551556E-1,9.3918717E-1,3.487234E-1,3.5236374E-1,3.18442E-1,3.0743837E-1,2.3753488E-1,2.8062046E-1,4.2467415E-1,4.873934E-1,0E0,5.621073E-1,6.826001E-1,7.113757E-1,6.7371106E-1,2.6115143E-1,8.234354E-1,5.789405E-1,1.0903517E-1,3.6549854E-1,2.7753904E-1,1.7770526E-1,0E0,5.196678E-1,1.1650586E-1,0E0,1.5431353E-1,0E0,3.089264E-1,2.5880408E-1,2.1604323E-1,1.5125522E-1,2.5347167E-1,5.31316E-1,2.994753E-1,3.359281E-1,4.6823967E-1,6.1701196E-1,4.271884E-1,0E0,6.67482E-1,5.621419E-1,1.637542E-1,0E0,3.166E-1,0E0,2.3822577E-1,0E0,0E0,1.10823065E-1,1.7841864E-1,2.5562197E-1,1.5925163E-1,1.6709521E-1,0E0,1.8681486E-1,2.1323298E-1,1.0518849E-1,0E0,0E0,0E0,0E0,1.8263352E-1,2.4033573E-1,0E0,0E0,1.5248334E-1,0E0,0E0,0E0,3.2200336E-1,0E0,2.5901332E-1,2.5066203E-1,1.945819E-1,0E0,2.687884E-1,2.8609362E-1,1.1002469E-1,1.1224151E-1,4.6635655E-1,0E0,8.3288336E-1,7.126676E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,59,59,61,61,64,64,65,65,66,66,67,67,68,68,70,70,71,71,72,72,77,77,78,78,81,81,85,85,87,87,88,88,89,89,91,91,92,92,93,93,94,94,95,95,97,97,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,-1,78,80,82,84,86,88,90,92,94,96,98,-1,100,102,104,-1,106,-1,108,-1,-1,110,112,114,116,118,-1,120,122,124,-1,-1,-1,-1,126,128,-1,-1,130,-1,-1,-1,132,-1,134,136,138,-1,140,142,144,146,148,-1,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5107028E-1,6.748543E-1,3.8172052E2,4.264897E6,1.9121015E0,1.0012501E7,1E0,2.092073E6,1.03E2,5.418831E11,1.4540612E7,1.5182724E0,-2.5748866E-2,1.3676985E4,1E0,2.3920168E6,5.7581736E7,7.68E2,1.4915413E3,7.33E2,8.289561E8,2.3432E4,1.5265896E8,5.59E2,2.6133334E2,-1.7754085E-2,8.3306855E6,1.3351997E0,5.68E2,6.72E2,1.862E3,8.791777E3,2.2576077E10,1.56E2,9.538462E0,2.5341389E8,3.4613644E1,-1.3010666E-2,3.3410483E8,1.3655606E6,-3.2366607E-3,7.105352E5,-6.222081E-3,9.3343524E2,6.773463E6,1.2453214E0,3.99704E5,1.2737473E1,3.401282E1,1.912391E6,4.5397964E-1,1.7E1,1.0980621E7,4.9716983E0,-1.2854796E-2,4.004366E1,6.81011E5,1.0203835E-5,8.917724E-3,8.027061E2,1.6398251E-2,1.2909952E1,-1.9464694E-2,-6.285667E-3,2.6120481E0,1.37E2,1.07E2,2.7487562E0,7.449623E2,7.4186022E-3,4.354E3,1E0,1.2252E4,-2.4759423E-2,-1.0333674E-2,-9.0728066E-4,5.4272506E-3,1.8344E4,6.239726E0,-1.5811814E-2,3.2420421E-4,1.5053125E2,-1.4975682E-3,-8.164625E-3,3.2577426E-3,2.6E1,-1.3181363E-2,1E0,2.0081E4,1.3E1,1.598817E-2,1.5851064E0,1.9007992E1,1.14E3,9.705292E6,1.22E2,1.517529E-2,1.4075E4,1.5435694E9,-1.6612294E-3,-7.2858394E-3,-9.670467E-3,-3.3549904E-3,-3.8810528E-3,2.1755775E-3,3.2062686E-4,4.276851E-3,-8.712922E-3,-8.441064E-4,3.3039635E-3,-6.5758387E-3,1.1058788E-2,3.771312E-4,-2.9250695E-3,6.4085117E-3,2.9682626E-3,-7.881385E-3,4.7932426E-3,-3.5584883E-3,3.0525445E-3,-1.5838477E-3,-2.7941186E-3,6.8909656E-3,-5.215821E-3,-1.3529135E-2,-8.940751E-3,-3.888412E-4,3.643264E-3,-3.3817843E-3,2.0058337E-3,6.243447E-3,-2.3550924E-3,-8.74005E-3,3.385721E-3,-7.968129E-3,-7.509882E-4,-7.7586165E-3,7.2912155E-3,1.9321596E-4,8.458297E-3,-9.515826E-4,-3.9939806E-3,6.2861905E-4,1.4442653E-2,1.5649493E-3,-8.676684E-4,6.9967853E-3,-1.6346594E-2,-1.187579E-3,1.7703604E-2,6.1230226E-3,2.1020526E-3,8.330972E-3],"split_indices":[38,27,52,9,53,45,102,1,11,31,45,53,0,33,100,28,45,2,55,2,7,9,45,0,55,0,45,41,3,0,2,52,12,2,58,7,56,0,7,32,0,28,0,4,1,38,1,54,58,1,41,8,1,54,0,53,1,42,0,52,0,56,0,0,53,3,10,54,4,0,10,16,9,0,0,0,0,9,56,0,0,52,0,0,0,8,0,74,29,8,0,53,56,10,9,10,0,9,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E3,1.605E3,1.055E3,1.346E3,2.59E2,3.57E2,6.98E2,1.112E3,2.34E2,6.9E1,1.9E2,3.54E2,3E0,1.94E2,5.04E2,8.58E2,2.54E2,9.9E1,1.35E2,5.3E1,1.6E1,4.2E1,1.48E2,1.15E2,2.39E2,4E0,1.9E2,1.12E2,3.92E2,7.65E2,9.3E1,1.94E2,6E1,2.7E1,7.2E1,4.8E1,8.7E1,1.4E1,3.9E1,1.3E1,3E0,3.4E1,8E0,1.37E2,1.1E1,1.03E2,1.2E1,1.2E2,1.19E2,4E1,1.5E2,3.3E1,7.9E1,3.88E2,4E0,6.27E2,1.38E2,8.2E1,1.1E1,1.85E2,9E0,5.6E1,4E0,3E0,2.4E1,4.5E1,2.7E1,2.8E1,2E1,9E0,7.8E1,2.1E1,1.8E1,9E0,4E0,1.6E1,1.8E1,4E1,9.7E1,8E0,3E0,8.9E1,1.4E1,9E0,3E0,1.14E2,6E0,7.2E1,4.7E1,3.5E1,5E0,2.7E1,1.23E2,1.5E1,1.8E1,7.5E1,4E0,1.84E2,2.04E2,5.75E2,5.2E1,5.3E1,8.5E1,1.1E1,7.1E1,9.5E1,9E1,1E1,4.6E1,2.2E1,2E0,4.1E1,4E0,1E1,1.7E1,3E0,2.5E1,1E1,1E1,3.7E1,4.1E1,1.4E1,7E0,8E0,1E1,3.3E1,7E0,1.2E1,8.5E1,3.1E1,5.8E1,9.2E1,2.2E1,6.8E1,4E0,2.7E1,2E1,1.9E1,1.6E1,1.5E1,1.2E1,5.4E1,6.9E1,1.3E1,2E0,1.3E1,5E0,4E0,7.1E1,1.5E1,1.69E2,1.45E2,5.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-2.501018E-4,1.9820053E-2,-5.7229545E-2,-2.5346614E-2,4.3810464E-2,-9.307421E-2,-2.211632E-4,-3.7346676E-2,9.7191855E-2,4.487065E-3,7.649069E-2,-8.2929105E-2,-3.0869815E-1,-3.5351593E-2,4.732796E-2,-6.8989885E-3,-7.623325E-2,1.1382237E-1,-1.2825838E-2,1.2981693E-2,-1.026676E-3,2.3598461E-1,6.713445E-2,-7.5338006E-2,-2.938941E-1,-1.8265557E-1,-2.388193E-2,-3.7684245E-3,-1.13874845E-1,2.8438682E-2,8.760066E-3,-9.982716E-2,1.4273972E-2,2.8881935E-2,-9.51427E-2,1.4953046E-1,2.8898044E-2,-1.9263823E-2,8.6886644E-2,2.785558E-1,3.0233264E-3,5.3398646E-2,1.354593E-1,-8.940438E-2,1.4522103E-2,-1.9726804E-2,4.4317087E-5,-1.2247542E-2,-2.622803E-3,-3.804913E-2,6.4892165E-2,1.676993E-3,-1.3767919E-1,7.2570935E-2,-2.782456E-2,-7.711848E-2,-1.5040548E-2,9.0018585E-2,-2.5457624E-2,-1.3062933E-1,7.494754E-2,-3.172035E-1,-8.177081E-2,3.0314142E-3,1.0132045E-2,3.9806557E-3,-3.6362307E-3,-2.383249E-2,9.518746E-3,1.0881697E-1,-1.00442074E-1,1.6182737E-2,1.5641755E-1,9.495584E-2,2.9816676E-2,1.8444574E-1,-3.3666797E-2,-6.1011225E-2,-1.4976597E-1,1.2868324E-1,-1.1435003E-3,-9.3914114E-2,-3.5046819E-3,9.305247E-3,1.886724E-3,-1.6268125E-2,-5.6309E-3,9.036303E-3,4.138032E-2,-5.931944E-2,3.7248824E-3,-7.378593E-3,-1.4251366E-3,6.9578853E-3,-2.4091017E-3,-1.662466E-2,-7.1567355E-4,-1.1537439E-2,4.550747E-3,4.4671344E-3,-4.1114283E-3,1.8136132E-3,-1.8447675E-2,2.464835E-3,-5.281913E-3,-1.543254E-4,-3.4180225E-3,5.917607E-3,-7.979386E-3,9.88886E-4,-1.1966934E-2,1.18489135E-2,1.1581079E-3,6.09773E-3,-2.572462E-3,4.3953714E-4,4.2202207E-3,9.930993E-3,1.1444441E-3,6.717889E-3,-3.4322632E-3,-4.200115E-3,5.990488E-4,-2.8794536E-3,-9.308791E-3,8.682683E-3,-2.3078928E-3,-3.9942772E-4,-7.3669273E-3,2.9346566E-3,-1.9638774E-3,7.757997E-3,7.7692827E-4,-8.598211E-3,-8.887975E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,49,51,53,-1,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,-1,-1,-1,-1,81,83,-1,85,87,89,91,-1,93,95,97,99,101,103,-1,-1,-1,-1,105,-1,107,109,-1,111,113,115,117,119,121,123,125,-1,127,129,-1,-1,-1,-1,-1,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0819554E0,2.1606593E0,1.4344788E0,1.0211945E0,1.672854E0,9.172244E-1,4.560277E-1,7.465822E-1,4.0683973E-1,8.7044126E-1,1.045157E0,6.3926005E-1,3.63878E-1,3.8893294E-1,2.918982E-1,6.999104E-1,5.5333245E-1,1.7762798E-1,0E0,0E0,9.3318415E-1,2.6940155E-1,6.2437487E-1,5.06901E-1,4.87507E-1,1.0750151E-1,0E0,2.68441E-1,1.6431361E-1,2.5777066E-1,0E0,2.9686803E-1,8.7503344E-1,3.2575366E-1,6.7186475E-1,2.1128666E-1,1.14304446E-1,4.8484772E-1,4.2174768E-1,1.7188048E-1,0E0,5.4792905E-1,9.341166E-1,5.838568E-1,2.441777E-1,0E0,0E0,0E0,0E0,1.463376E-1,1.17721885E-1,0E0,1.1842364E-1,2.0225179E-1,1.5760863E-1,2.1773213E-1,0E0,7.8285503E-1,6.407843E-1,2.7175054E-1,1.0822317E-1,3.2847977E-1,8.530878E-1,0E0,0E0,0E0,0E0,4.6320668E-1,0E0,3.861848E-1,2.0421249E-1,0E0,1.3841617E-1,9.8379123E-1,4.32398E-1,3.152249E-1,1.8191847E-1,4.6617013E-1,4.281907E-1,1.3903548E-1,0E0,1.4740264E-1,1.1913163E-1,0E0,0E0,0E0,0E0,0E0,1.3998526E-1,1.7307523E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,49,49,50,50,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,67,67,69,69,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,82,82,88,88,89,89],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,50,52,54,-1,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,-1,-1,-1,-1,82,84,-1,86,88,90,92,-1,94,96,98,100,102,104,-1,-1,-1,-1,106,-1,108,110,-1,112,114,116,118,120,122,124,126,-1,128,130,-1,-1,-1,-1,-1,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,2.6659616E2,2.7947538E6,1.3195207E7,2.3770695E5,1.5265896E8,6.726E3,2.5074627E0,1.0430133E7,8E0,1.0918E4,4.5723195E6,2.3580047E10,2.99665E5,1.4813511E6,3.75E1,1.1376811E0,2.8807552E7,-1.2825838E-2,1.2981693E-2,1.5435694E9,1E0,1.3664E4,3.0387878E3,3.0897958E0,2.867292E3,-2.388193E-2,3.222E3,1.0134536E7,1.29E3,8.760066E-3,3.09627E5,3.5508975E3,1.9E3,1.198E5,1.4462532E7,1.216E3,1.5137865E2,2.6572757E10,3.4246575E-3,3.0233264E-3,1.4811083E0,2.6364462E7,2.8035452E11,2.8277853E11,-1.9726804E-2,4.4317087E-5,-1.2247542E-2,-2.622803E-3,4.593E3,1.7270016E5,1.676993E-3,1.0595947E7,6.504065E-2,3.8248518E-1,8.333333E0,-1.5040548E-2,8E0,6.6861836E-4,1.8836284E2,2.0316172E6,1.5E1,3.4955364E7,3.0314142E-3,1.0132045E-2,3.9806557E-3,-3.6362307E-3,2.2075728E7,9.518746E-3,3.7826266E4,1.3674345E4,1.6182737E-2,9.532E3,3.1984328E5,1.8137958E6,2.1146836E7,1.2819376E2,2.011E3,1E1,1.0238709E1,-1.1435003E-3,4.0778715E6,1.9987492E5,9.305247E-3,1.886724E-3,-1.6268125E-2,-5.6309E-3,9.036303E-3,7.576E3,4.214286E0,3.7248824E-3,-7.378593E-3,-1.4251366E-3,6.9578853E-3,-2.4091017E-3,-1.662466E-2,-7.1567355E-4,-1.1537439E-2,4.550747E-3,4.4671344E-3,-4.1114283E-3,1.8136132E-3,-1.8447675E-2,2.464835E-3,-5.281913E-3,-1.543254E-4,-3.4180225E-3,5.917607E-3,-7.979386E-3,9.88886E-4,-1.1966934E-2,1.18489135E-2,1.1581079E-3,6.09773E-3,-2.572462E-3,4.3953714E-4,4.2202207E-3,9.930993E-3,1.1444441E-3,6.717889E-3,-3.4322632E-3,-4.200115E-3,5.990488E-4,-2.8794536E-3,-9.308791E-3,8.682683E-3,-2.3078928E-3,-3.9942772E-4,-7.3669273E-3,2.9346566E-3,-1.9638774E-3,7.757997E-3,7.7692827E-4,-8.598211E-3,-8.887975E-4],"split_indices":[27,52,32,9,28,45,2,56,45,18,9,28,5,29,28,4,53,9,0,0,7,6,2,4,56,4,0,2,45,0,0,9,50,12,7,5,10,56,12,57,0,53,47,31,31,0,0,0,0,11,47,0,45,57,34,52,0,17,41,52,47,0,7,0,0,0,0,45,0,52,4,0,9,33,28,9,58,2,3,53,0,32,47,0,0,0,0,0,9,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.693E3,1.992E3,7.01E2,6.91E2,1.301E3,4.3E2,2.71E2,6.3E2,6.1E1,5.91E2,7.1E2,4.12E2,1.8E1,1.56E2,1.15E2,3.54E2,2.76E2,5.9E1,2E0,1.1E1,5.8E2,3.8E1,6.72E2,3.99E2,1.3E1,1.2E1,6E0,1.12E2,4.4E1,1.02E2,1.3E1,6.5E1,2.89E2,4.2E1,2.34E2,4.1E1,1.8E1,4.81E2,9.9E1,3E1,8E0,5.61E2,1.11E2,3.45E2,5.4E1,9E0,4E0,7E0,5E0,7.5E1,3.7E1,6E0,3.8E1,5.7E1,4.5E1,6E1,5E0,9.9E1,1.9E2,9E0,3.3E1,1.2E1,2.22E2,1.8E1,2.3E1,1.2E1,6E0,4.72E2,9E0,8.9E1,1E1,1.9E1,1.1E1,2.02E2,3.59E2,8.6E1,2.5E1,2.36E2,1.09E2,1.3E1,4.1E1,2.8E1,4.7E1,5E0,3.2E1,2E0,3.6E1,1.1E1,4.6E1,3.5E1,1E1,2.2E1,3.8E1,7.1E1,2.8E1,5E0,1.85E2,6E0,3E0,3E1,3E0,2E0,1E1,3.9E1,1.83E2,3.3E2,1.42E2,8.5E1,4E0,6E0,4E0,6E0,5E0,1.66E2,3.6E1,2.66E2,9.3E1,7.5E1,1.1E1,4E0,2.1E1,1.73E2,6.3E1,3.7E1,7.2E1,1E1,3E0,1.2E1,1.6E1,1.7E1,3E1,7E0,3.9E1,8E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[9.7434997E-4,1.9878583E-2,-5.2678537E-2,2.8047527E-3,7.50732E-2,-6.991756E-2,5.993552E-2,-2.244531E-2,4.1533615E-2,8.164391E-2,-1.3027421E-2,-5.064338E-2,-1.4520691E-1,1.03148736E-1,-6.461553E-3,-4.289559E-2,1.806712E-2,5.3273868E-2,-8.9379095E-2,5.5105793E-3,1.1254196E-1,-4.3809943E-2,-2.6320383E-1,-2.3215084E-1,-8.3323844E-2,1.7148723E-1,4.825144E-2,-4.999566E-3,1.3185056E-3,-5.4477833E-2,1.323498E-2,-3.397296E-2,4.711923E-2,3.5841953E-2,1.0928806E-1,-2.1031407E-1,-3.6506772E-2,1.018185E-2,-1.2757965E-2,2.3564896E-1,9.426156E-2,-3.2955095E-2,-1.4914241E-1,-1.9007558E-2,-2.5723656E-3,-1.8752815E-1,-4.1955367E-1,-6.390718E-2,-1.621058E-2,1.9199818E-1,-2.055747E-3,9.6032483E-4,9.681312E-3,-1.5141088E-2,-7.625876E-2,5.9081227E-2,-8.145543E-2,-9.857383E-2,1.2545432E-2,7.924086E-2,-2.2377376E-2,4.359701E-2,-1.2299005E-1,1.1806432E-1,-1.4583651E-2,-4.8307846E-3,-1.7128825E-2,-9.88281E-2,3.8961926E-3,-6.1212685E-3,1.8750887E-2,9.745947E-2,3.0881128E-1,1.5213711E-2,8.4278665E-2,-7.990434E-2,2.3427417E-3,-1.4108649E-2,-1.0332131E-1,-2.6339105E-1,-1.4182222E-1,-2.3137957E-2,-6.289382E-3,-1.03638664E-1,5.6630354E-2,-9.254465E-4,1.0202807E-2,-2.4572844E-3,4.487157E-3,-3.36846E-3,-1.1519064E-2,3.883884E-3,-2.766488E-3,-6.713301E-3,-1.4560123E-3,2.7544512E-3,-5.881868E-3,7.4717607E-3,-6.151634E-4,5.13813E-3,1.6017283E-4,3.7629586E-3,-3.52027E-3,8.099108E-3,1.7214454E-3,-1.0601172E-3,-2.4944799E-2,8.556164E-3,3.0066827E-3,-9.5239293E-4,-9.114258E-3,1.9974964E-4,7.8582E-3,-4.34095E-5,1.0730326E-2,1.7317807E-2,3.052491E-3,5.22225E-3,1.3273908E-3,-3.2942193E-3,-1.6895313E-2,1.4604506E-3,-1.9526059E-3,-1.0853752E-2,-2.1517074E-3,-1.6212106E-2,-6.185731E-3,3.8440393E-3,-7.7221645E-3,-1.4253247E-3,-7.240574E-3,1.0444362E-2,4.6930666E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,-1,79,81,83,-1,85,-1,-1,-1,87,89,91,93,95,97,99,101,103,105,107,-1,-1,-1,109,-1,-1,111,113,115,-1,117,119,121,-1,123,125,127,-1,-1,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7405813E0,1.8826716E0,1.3709054E0,1.4941949E0,1.098691E0,8.792932E-1,2.703742E-1,7.6712865E-1,9.3154156E-1,1.0908799E0,0E0,6.9729817E-1,6.478207E-1,2.0556659E-1,1.2937982E-1,4.0054345E-1,4.7179368E-1,5.3702784E-1,3.1202775E-1,5.13464E-1,7.2043467E-1,5.370413E-1,3.9878953E-1,3.4780526E-1,3.450024E-1,1.1818725E-1,1.3916361E-1,0E0,0E0,4.358089E-1,4.6497968E-1,3.3781105E-1,4.4824204E-1,5.276244E-1,5.1170444E-1,2.0582086E-1,2.7247387E-1,0E0,4.5557255E-1,3.9453435E-1,6.177921E-1,7.144433E-1,2.6282674E-1,0E0,0E0,1.11742735E-1,1.0150123E-1,3.4110856E-1,0E0,1.0640985E-1,0E0,0E0,0E0,7.2519994E-1,3.021692E-1,1.8992347E-1,1.0066961E-1,1.8493143E-1,2.4243191E-1,2.9128045E-1,3.3611637E-1,3.828944E-1,7.9102623E-1,4.2091954E-1,0E0,0E0,0E0,1.6954419E-1,0E0,0E0,2.0909514E-1,1.9945297E-1,3.2633352E-1,0E0,3.864143E-1,5.3700817E-1,3.0105463E-1,0E0,2.4183768E-1,1.07548475E-1,1.3804442E-1,0E0,0E0,1.8168569E-1,1.339644E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,67,67,70,70,71,71,72,72,74,74,75,75,76,76,78,78,79,79,80,80,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,-1,80,82,84,-1,86,-1,-1,-1,88,90,92,94,96,98,100,102,104,106,108,-1,-1,-1,110,-1,-1,112,114,116,-1,118,120,122,-1,124,126,128,-1,-1,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.0989723E3,3.4409692E3,4.7519747E5,1.2367184E3,6.747114E7,1.663473E6,1.5107028E-1,2E0,7.805608E4,-1.3027421E-2,4.391553E6,5.35E3,3.3517068E7,5.528107E3,2.777588E6,3.8833074E4,6.072483E2,3.4446188E2,1.2E1,1.5552E4,6.5E1,3.0897958E0,1.97729E5,1.2502964E9,1.2982227E1,6.707523E8,-4.999566E-3,1.3185056E-3,3.8E1,7.8E1,6.061028E4,2.0687214E6,4.5158855E6,1.5449402E7,5.69815E5,1.038946E6,1.018185E-2,6.015E3,2.0601392E3,6.6666694E8,1.646E3,1.7446084E0,-1.9007558E-2,-2.5723656E-3,3.1665432E0,3.285E3,1.2204E4,-1.621058E-2,8.838E3,-2.055747E-3,9.6032483E-4,9.681312E-3,7.55552E5,1E0,2.116711E3,1.27E2,1.5555556E0,3.7129E4,1.8777761E0,2.72258E5,1.8574098E8,8.190217E0,2E0,-1.4583651E-2,-4.8307846E-3,-1.7128825E-2,9E0,3.8961926E-3,-6.1212685E-3,9.492855E8,2.0225275E5,8.861076E0,1.5213711E-2,2.2807484E7,2.1103458E0,2.65851E5,-1.4108649E-2,2.264894E8,8.126489E7,9.090909E-2,-2.3137957E-2,-6.289382E-3,6.5928856E5,2.060792E8,-9.254465E-4,1.0202807E-2,-2.4572844E-3,4.487157E-3,-3.36846E-3,-1.1519064E-2,3.883884E-3,-2.766488E-3,-6.713301E-3,-1.4560123E-3,2.7544512E-3,-5.881868E-3,7.4717607E-3,-6.151634E-4,5.13813E-3,1.6017283E-4,3.7629586E-3,-3.52027E-3,8.099108E-3,1.7214454E-3,-1.0601172E-3,-2.4944799E-2,8.556164E-3,3.0066827E-3,-9.5239293E-4,-9.114258E-3,1.9974964E-4,7.8582E-3,-4.34095E-5,1.0730326E-2,1.7317807E-2,3.052491E-3,5.22225E-3,1.3273908E-3,-3.2942193E-3,-1.6895313E-2,1.4604506E-3,-1.9526059E-3,-1.0853752E-2,-2.1517074E-3,-1.6212106E-2,-6.185731E-3,3.8440393E-3,-7.7221645E-3,-1.4253247E-3,-7.240574E-3,1.0444362E-2,4.6930666E-4],"split_indices":[27,52,52,28,58,45,29,38,6,28,0,28,2,51,52,1,33,4,4,18,9,8,56,11,7,53,32,0,0,10,8,28,47,47,50,9,9,0,2,52,31,2,53,0,0,56,0,2,0,2,0,0,0,9,85,4,8,53,9,41,9,31,56,17,0,0,0,8,0,0,32,33,53,0,51,42,29,0,7,12,58,0,0,32,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E3,1.997E3,7.03E2,1.526E3,4.71E2,6.1E2,9.3E1,9.24E2,6.02E2,4.63E2,8E0,4.87E2,1.23E2,5.6E1,3.7E1,6.14E2,3.1E2,5.53E2,4.9E1,1.34E2,3.29E2,4.73E2,1.4E1,5E1,7.3E1,2.4E1,3.2E1,9E0,2.8E1,5.09E2,1.05E2,1.11E2,1.99E2,4.23E2,1.3E2,1.4E1,3.5E1,1E1,1.24E2,4.1E1,2.88E2,4.3E2,4.3E1,8E0,6E0,4.2E1,8E0,6.9E1,4E0,2.2E1,2E0,2.8E1,4E0,1.82E2,3.27E2,7.1E1,3.4E1,4.6E1,6.5E1,1.36E2,6.3E1,4.04E2,1.9E1,1.28E2,2E0,9E0,5E0,2.3E1,1.2E1,2.6E1,9.8E1,1.5E1,2.6E1,1.1E1,2.77E2,1.84E2,2.46E2,9E0,3.4E1,1.4E1,2.8E1,6E0,2E0,5.2E1,1.7E1,2E0,2E1,1.37E2,4.5E1,3.17E2,1E1,6E1,1.1E1,1.5E1,1.9E1,6E0,4E1,9E0,5.6E1,9.9E1,3.7E1,2.1E1,4.2E1,2.2E1,3.82E2,1.6E1,3E0,6E1,6.8E1,1.3E1,1E1,9E1,8E0,9E0,6E0,2.1E1,5E0,1.92E2,8.5E1,1.78E2,6E0,1.49E2,9.7E1,1E1,2.4E1,8E0,6E0,2E0,2.6E1,2.1E1,3.1E1,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-2.4875929E-3,-3.7208196E-2,2.570828E-2,-4.9318664E-2,3.9249163E-2,5.1845904E-2,-6.6702664E-2,-4.1313488E-2,-1.3417709E-1,4.8208714E-2,-1.2140937E-2,1.0500792E-1,3.1266578E-2,-1.14903934E-1,-6.7390045E-3,-3.220296E-2,-1.2905051E-1,1.4085165E-2,-1.673199E-1,1.0561627E-1,7.388422E-3,1.2936331E-1,-4.100771E-3,2.0665443E-2,1.9495882E-1,-9.162128E-2,-2.1899548E-1,9.591572E-3,-2.545779E-1,-4.1825395E-2,3.665212E-2,-1.9779624E-2,-9.941118E-2,4.7033476E-3,-3.4292045E-3,-2.3750818E-1,-6.682866E-2,1.4946978E-1,1.7130578E-2,-3.4164144E-3,3.2434516E-2,1.6598189E-1,3.3285856E-2,8.2438216E-2,-5.4094374E-2,3.2670046E-3,6.371778E-2,3.2829952E-1,1.22554384E-1,-1.0920754E-2,-7.566726E-2,-1.7512725E-1,-1.819377E-2,-1.3050267E-1,2.6598256E-2,-1.8924491E-2,-3.0208786E-3,-3.8205303E-2,-2.8089824E-1,5.1417876E-2,-1.6715597E-2,3.1808133E-3,-1.2641895E-1,-8.3898336E-2,-2.6583305E-1,-7.6505536E-4,-1.0106618E-2,2.3747282E-3,1.0082044E-2,5.4952065E-3,-2.5773598E-3,7.779994E-2,-4.6055995E-2,9.4766065E-2,2.0463225E-1,-5.5271015E-2,1.16305895E-1,8.139922E-3,4.197697E-4,-9.161473E-2,7.723944E-3,9.536678E-3,-5.9471065E-3,1.6408002E-1,3.3005256E-2,6.182641E-3,1.9182136E-2,7.835693E-2,1.7395195E-2,-5.6434277E-2,-1.6146702E-1,-7.355841E-2,-1.0623323E-2,-9.622743E-4,-1.4020816E-2,7.650289E-2,-1.7660333E-2,-1.7213266E-3,-1.736891E-2,-2.0595968E-2,4.197928E-3,6.4516338E-3,9.4029435E-4,-1.4073057E-2,-4.152573E-3,-7.979636E-3,5.244776E-3,-1.016112E-2,-1.8139193E-2,1.1444332E-3,6.744535E-3,-7.282183E-4,-1.0518803E-2,5.3402972E-3,-6.633595E-3,1.111334E-2,5.2501056E-3,-1.826757E-2,4.6177144E-4,7.7056135E-3,-3.401982E-3,-2.4766324E-3,-1.2573513E-2,7.2525337E-4,-5.2127736E-3,1.0919113E-2,3.523322E-3,7.230783E-3,9.14296E-4,5.554188E-3,-9.346604E-3,-4.007623E-3,1.7176705E-3,2.2844438E-4,-9.839814E-3,1.4546271E-3,-1.1352478E-2,5.762611E-3,-3.0861396E-4,8.731044E-3,-1.2814606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,-1,-1,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,-1,89,91,-1,93,95,-1,-1,97,99,101,-1,-1,103,105,107,-1,-1,-1,-1,-1,-1,109,111,113,115,117,119,-1,-1,121,-1,123,-1,125,127,-1,-1,129,-1,131,133,135,-1,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5952477E0,1.101946E0,3.537192E0,6.9142556E-1,4.3977526E-1,1.2450144E0,9.328004E-1,7.4641573E-1,4.347732E-1,3.718192E-1,0E0,8.455789E-1,1.4258776E0,4.1374707E-1,5.8930784E-1,5.6602865E-1,6.9960105E-1,1.2958038E-1,4.91966E-1,2.5354648E-1,1.8766196E-1,9.092121E-1,2.597168E-1,5.8030045E-1,4.4248843E-1,3.0484617E-1,1.5934813E-1,3.298789E-1,2.1736807E-1,6.3663614E-1,6.159216E-1,0E0,3.7117028E-1,0E0,0E0,1.6043806E-1,2.239656E-1,2.5787985E-1,1.6546384E-1,0E0,2.599575E-1,4.980812E-1,5.43124E-1,1.4038119E-1,3.2185566E-1,4.4306386E-1,6.819548E-1,1.9292998E-1,3.3320242E-1,0E0,2.1201259E-1,1.1174512E-1,0E0,2.6285177E-1,2.730222E-1,0E0,0E0,5.287143E-1,6.424293E-1,2.6740253E-1,0E0,0E0,4.3994355E-1,1.4624792E-1,1.3172793E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.52926E-1,1.412496E-1,2.8106564E-1,2.8982592E-1,7.6590455E-1,3.2889986E-1,0E0,0E0,2.1149057E-1,0E0,3.5214373E-1,0E0,2.840724E-1,2.7946675E-1,0E0,0E0,3.3133107E-1,0E0,2.8217524E-1,1.7651731E-1,1.7712149E-1,0E0,0E0,0E0,2.1161881E-1,1.2495278E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,53,53,54,54,57,57,58,58,59,59,62,62,63,63,64,64,71,71,72,72,73,73,74,74,75,75,76,76,79,79,81,81,83,83,84,84,87,87,89,89,90,90,91,91,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,-1,-1,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,-1,90,92,-1,94,96,-1,-1,98,100,102,-1,-1,104,106,108,-1,-1,-1,-1,-1,-1,110,112,114,116,118,120,-1,-1,122,-1,124,-1,126,128,-1,-1,130,-1,132,134,136,-1,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7311627E2,8.397659E6,4.0844156E7,2.1094006E7,1E0,8E0,3.0387878E3,1.062363E1,6.043677E8,2.53E2,-1.2140937E-2,1.0424884E-7,8.015419E3,1.2427474E8,2.6572757E10,2.5247778E6,2E0,3.910327E-3,7.85939E5,2.7309215E6,1.0617118E8,2.1298597E0,1.752E3,3.771981E6,4.0601485E6,4.30453E2,9.374544E5,2.2579439E0,2.862988E5,2.298425E6,9.697451E-1,-1.9779624E-2,7.29E2,4.7033476E-3,-3.4292045E-3,1.2068E4,1.8600049E10,7.59E2,1.1217151E0,-3.4164144E-3,4.0895187E8,5.314225E2,2.9251662E6,1.1214854E1,5.794E3,1.0046621E7,9.397667E6,1.9745072E7,8.724166E9,-1.0920754E-2,1.93974E7,8.7757526E2,-1.819377E-2,1.9E1,6.747114E7,-1.8924491E-2,-3.0208786E-3,4.7279915E6,1.8081865E0,8.339375E2,-1.6715597E-2,3.1808133E-3,2.5040618E7,3.3926086E2,5.59277E5,-7.6505536E-4,-1.0106618E-2,2.3747282E-3,1.0082044E-2,5.4952065E-3,-2.5773598E-3,3.624992E5,3.7898205E1,1.6259277E7,5.8180006E8,1.3E0,2.2579439E0,8.139922E-3,4.197697E-4,9.557794E2,7.723944E-3,7.806529E1,-5.9471065E-3,1.1860538E8,6.85945E4,6.182641E-3,1.9182136E-2,5.68E2,1.7395195E-2,1.9701008E5,5.5544252E7,2.894E3,-1.0623323E-2,-9.622743E-4,-1.4020816E-2,1.5609541E0,2.99665E5,-1.7213266E-3,-1.736891E-2,-2.0595968E-2,4.197928E-3,6.4516338E-3,9.4029435E-4,-1.4073057E-2,-4.152573E-3,-7.979636E-3,5.244776E-3,-1.016112E-2,-1.8139193E-2,1.1444332E-3,6.744535E-3,-7.282183E-4,-1.0518803E-2,5.3402972E-3,-6.633595E-3,1.111334E-2,5.2501056E-3,-1.826757E-2,4.6177144E-4,7.7056135E-3,-3.401982E-3,-2.4766324E-3,-1.2573513E-2,7.2525337E-4,-5.2127736E-3,1.0919113E-2,3.523322E-3,7.230783E-3,9.14296E-4,5.554188E-3,-9.346604E-3,-4.007623E-3,1.7176705E-3,2.2844438E-4,-9.839814E-3,1.4546271E-3,-1.1352478E-2,5.762611E-3,-3.0861396E-4,8.731044E-3,-1.2814606E-3],"split_indices":[52,9,45,45,74,17,4,54,5,10,0,37,4,45,12,28,17,38,1,45,7,38,29,1,47,33,28,54,28,28,27,0,29,0,0,9,5,2,53,0,7,4,28,56,2,47,45,1,7,0,50,52,0,3,45,0,0,47,53,33,0,0,5,52,1,0,0,0,0,0,0,47,58,9,7,54,54,0,0,52,0,53,0,32,32,0,0,3,0,33,45,0,0,0,0,57,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.649E3,1.187E3,1.462E3,1.025E3,1.62E2,1.14E3,3.22E2,9.38E2,8.7E1,1.58E2,4E0,3.17E2,8.23E2,1.78E2,1.44E2,8.51E2,8.7E1,1.6E1,7.1E1,6.5E1,9.3E1,2.59E2,5.8E1,7.74E2,4.9E1,1.47E2,3.1E1,1.36E2,8E0,7.47E2,1.04E2,7E0,8E1,8E0,8E0,4.1E1,3E1,4.3E1,2.2E1,2.2E1,7.1E1,1.87E2,7.2E1,2.1E1,3.7E1,5.52E2,2.22E2,1.6E1,3.3E1,1.4E1,1.33E2,2.6E1,5E0,1.4E1,1.22E2,4E0,4E0,7.37E2,1E1,1.01E2,3E0,1.1E1,6.9E1,7E0,3.4E1,2.3E1,7E0,1.7E1,2.6E1,9E0,1.3E1,4.5E1,2.6E1,6.7E1,1.2E2,3.5E1,3.7E1,9E0,1.2E1,3.2E1,5E0,5.27E2,2.5E1,5.1E1,1.71E2,5E0,1.1E1,2.9E1,4E0,1.1E2,2.3E1,9E0,1.7E1,9E0,5E0,5.7E1,6.5E1,7.33E2,4E0,7E0,3E0,2.7E1,7.4E1,1.2E1,5.7E1,5E0,2E0,2.5E1,9E0,2.5E1,2E1,2.3E1,3E0,6.3E1,4E0,9.2E1,2.8E1,5E0,3E1,3E1,7E0,2.7E1,5E0,5.04E2,2.3E1,2.9E1,2.2E1,1.7E1,1.54E2,2.6E1,3E0,8.5E1,2.5E1,5E0,1.8E1,6E0,3E0,3.7E1,2E1,2E0,6.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[3.1497396E-4,-3.3174522E-2,2.499809E-2,-2.7953394E-2,-2.1275616E-1,4.7153536E-2,-5.956037E-2,1.0292534E-2,-4.5065027E-2,-2.5134924E-1,-4.919145E-3,4.080183E-2,2.126615E-1,-1.06342115E-1,-5.848964E-4,2.8620219E-2,-1.3330327E-1,-4.3079905E-2,-1.6195986E-2,-8.951981E-3,-1.7810443E-2,5.567284E-2,-1.0505596E-2,-7.920173E-3,2.4360216E-1,-8.218959E-2,-2.1899967E-1,1.6933598E-2,-1.7707242E-1,3.96871E-2,-1.15544096E-1,-3.297028E-2,-2.754713E-1,-5.31273E-2,8.46625E-3,9.934983E-2,3.5041973E-2,-2.779531E-2,7.06485E-2,3.209605E-1,6.822484E-3,-1.8270066E-2,-1.0880092E-1,-3.193961E-1,-1.5598251E-1,8.7244227E-4,5.3428584E-3,-2.4216084E-3,-1.7689347E-2,-2.3022565E-1,4.6340708E-2,-7.338118E-3,5.0194765E-4,2.6103456E-3,-5.985803E-3,-1.8790513E-2,-6.750598E-3,-2.9849296E-2,-1.0065427E-1,1.1894266E-1,-5.418893E-3,7.5983256E-2,1.8882364E-1,5.7601742E-2,-2.3972457E-2,-5.729735E-2,-2.2742713E-3,8.8453434E-2,-1.0534724E-2,8.2052057E-4,1.7512511E-2,-8.882978E-2,2.542221E-3,-5.856492E-2,-1.8578966E-1,-4.551935E-3,-1.8756103E-2,-2.1188618E-3,-1.1306932E-2,3.155813E-2,-4.6198335E-2,-8.5888733E-4,-2.4294993E-2,4.59315E-3,1.0011031E-3,8.592818E-3,-1.6647219E-3,-7.881456E-3,-2.8217298E-3,1.0193262E-2,1.0492948E-3,2.4480538E-3,-3.0107964E-3,7.123467E-3,2.7013319E-3,7.097018E-4,1.0130081E-2,1.5785316E-3,5.629085E-3,-2.5827736E-2,-7.0836325E-4,-4.342947E-3,-5.375952E-4,9.676845E-4,-3.483459E-3,-6.9591203E-3,4.9186293E-3,-6.3045938E-3,2.6063758E-3,8.097856E-3,-3.2793558E-3,-3.6459314E-3,-1.0701635E-2,6.456327E-3,-1.0330937E-4,-4.0139705E-3,7.1279675E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,-1,-1,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,-1,-1,-1,81,83,-1,-1,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,103,105,-1,-1,-1,107,-1,109,111,-1,-1,-1,-1,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.217011E0,1.0610625E0,2.895385E0,7.249404E-1,1.1007917E-1,1.2780626E0,8.8556874E-1,9.0651524E-1,4.3057227E-1,1.0910547E-1,0E0,9.0091693E-1,5.6664634E-1,4.6905327E-1,4.4512528E-1,4.911977E-1,5.472193E-1,3.94575E-1,0E0,0E0,0E0,8.2101965E-1,3.7519655E-1,0E0,2.8289223E-1,2.5145864E-1,1.4630747E-1,1.9900605E-1,2.9757395E-1,5.207024E-1,1.060836E-1,2.0000131E-1,1.9626868E-1,7.017294E-1,1.9239213E-1,5.982268E-1,8.3081305E-1,1.6551267E-1,2.6266068E-1,3.0622125E-1,0E0,2.3127791E-1,3.952179E-1,1.4107275E-1,1.7754954E-1,1.6467506E-1,0E0,0E0,0E0,4.0264893E-1,3.485152E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.4720387E-1,5.478208E-1,1.22898474E-1,3.6698896E-1,3.2492948E-1,2.33881E-1,6.622188E-1,8.105509E-1,1.5588102E-1,1.8997227E-1,1.6485071E-1,0E0,0E0,0E0,1.4823647E-1,0E0,1.5824027E-1,1.5235615E-1,0E0,0E0,0E0,0E0,2.4050154E-1,1.04645856E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,49,49,50,50,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,71,71,73,73,74,74,79,79,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,-1,-1,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,-1,-1,-1,82,84,-1,-1,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,104,106,-1,-1,-1,108,-1,110,112,-1,-1,-1,-1,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.494881E2,3.653602E7,4.0844156E7,2E0,7.080933E2,9.256843E3,3.0387878E3,2E0,7.780377E6,1.2199979E10,-4.919145E-3,1.0938637E1,2.64E0,1.2427474E8,2.6572757E10,1E0,5.5E1,2.437E3,-1.6195986E-2,-8.951981E-3,-1.7810443E-2,2.1068998E6,3.33E2,-7.920173E-3,1.10032365E-1,1.3E1,3.773E3,8.791777E3,3.869321E2,3.5902756E-6,1.5E1,2.943086E4,6.23E2,4.44E2,1.1451E4,8.027061E2,1.5548611E7,7.57657E4,4.543313E6,6.3039363E4,6.822484E-3,8.027061E2,3.325328E11,5E-1,8.7757526E2,5.7075036E-1,5.3428584E-3,-2.4216084E-3,-1.7689347E-2,2.2299826E-6,4.7022E4,-7.338118E-3,5.0194765E-4,2.6103456E-3,-5.985803E-3,-1.8790513E-2,-6.750598E-3,6E0,3.3720784E5,1.701E3,2.1E1,2.3663366E0,1.954E3,1.9125667E3,4.1724915E2,7.6E1,3.9150736E7,1.2561812E6,-1.0534724E-2,8.2052057E-4,1.7512511E-2,1.775894E10,2.542221E-3,4.198242E2,1.387623E8,-4.551935E-3,-1.8756103E-2,-2.1188618E-3,-1.1306932E-2,2.6970406E0,7.1308413E0,-8.5888733E-4,-2.4294993E-2,4.59315E-3,1.0011031E-3,8.592818E-3,-1.6647219E-3,-7.881456E-3,-2.8217298E-3,1.0193262E-2,1.0492948E-3,2.4480538E-3,-3.0107964E-3,7.123467E-3,2.7013319E-3,7.097018E-4,1.0130081E-2,1.5785316E-3,5.629085E-3,-2.5827736E-2,-7.0836325E-4,-4.342947E-3,-5.375952E-4,9.676845E-4,-3.483459E-3,-6.9591203E-3,4.9186293E-3,-6.3045938E-3,2.6063758E-3,8.097856E-3,-3.2793558E-3,-3.6459314E-3,-1.0701635E-2,6.456327E-3,-1.0330937E-4,-4.0139705E-3,7.1279675E-4],"split_indices":[52,45,45,17,4,4,4,6,28,5,0,53,54,45,12,74,0,2,0,0,0,51,8,0,57,3,2,52,57,38,3,33,2,0,12,52,45,28,1,28,0,52,31,53,52,41,0,0,0,41,9,0,0,0,0,0,0,3,28,11,3,54,2,4,4,8,32,45,0,0,0,5,0,52,7,0,0,0,0,53,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E3,1.137E3,1.543E3,1.106E3,3.1E1,1.223E3,3.2E2,3.42E2,7.64E2,2.2E1,9E0,1.179E3,4.4E1,1.78E2,1.42E2,3.04E2,3.8E1,7.6E2,4E0,1.6E1,6E0,9.14E2,2.65E2,3E0,4.1E1,1.48E2,3E1,1.3E2,1.2E1,2.83E2,2.1E1,2.3E1,1.5E1,6.36E2,1.24E2,2.92E2,6.22E2,2.19E2,4.6E1,2.2E1,1.9E1,4.4E1,1.04E2,1E1,2E1,1.12E2,1.8E1,8E0,4E0,6E0,2.77E2,1.6E1,5E0,1.2E1,1.1E1,7E0,8E0,4.28E2,2.08E2,1.3E1,1.11E2,2.33E2,5.9E1,4.5E2,1.72E2,1.01E2,1.18E2,4.4E1,2E0,3E0,1.9E1,2.2E1,2.2E1,6.4E1,4E1,3E0,7E0,9E0,1.1E1,6.8E1,4.4E1,4E0,2E0,9.3E1,1.84E2,9E0,4.19E2,8.1E1,1.27E2,6E0,7E0,5.6E1,5.5E1,4.8E1,1.85E2,7E0,5.2E1,3.2E2,1.3E2,2E0,1.7E2,5.8E1,4.3E1,9E1,2.8E1,2E0,4.2E1,1.7E1,5E0,2E0,6.2E1,1.1E1,2.9E1,1.6E1,5.2E1,2.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-1.7627389E-3,-3.744867E-2,2.0102818E-2,-3.0324143E-1,-3.3499368E-2,-4.797342E-3,5.4007232E-2,-1.8904464E-2,-6.8136784E-3,-6.0507372E-2,5.007274E-3,3.5184257E-2,-2.6916668E-2,1.0392677E-1,1.8612986E-2,4.8116397E-2,-7.404824E-2,1.7286214E-1,-1.3068992E-2,2.9007528E-2,2.932972E-1,-2.8535971E-2,1.5296751E-2,1.5342662E-1,2.6884891E-2,-4.936788E-3,1.0475534E-1,-5.9021775E-2,1.6028972E-1,-6.181677E-2,-2.1931821E-1,1.3459782E-2,1.1259303E-1,-2.4021113E-2,1.7582552E-1,1.263238E-1,1.3351499E-2,2.2614636E-3,2.0664686E-2,-3.5450075E-2,4.833383E-2,1.8487154E-1,4.8942305E-2,1.3114576E-1,-4.4992346E-2,5.7103388E-2,-2.6694005E-2,7.9027444E-2,3.026758E-1,-8.09971E-3,-1.4402532E-2,1.5072074E-2,4.7068493E-3,7.663518E-2,-6.752821E-2,3.1055901E-3,-2.7973026E-1,3.3996522E-3,1.2131323E-2,2.5881257E-2,-5.221674E-2,3.1676063E-1,9.0543233E-4,-6.7965663E-3,1.4783849E-1,-1.22896396E-1,2.5917593E-2,-3.8613707E-2,7.0322594E-3,7.1244105E-3,-2.0801563E-5,1.1957875E-1,2.3504403E-1,2.0107695E-1,-3.9739717E-2,9.344088E-3,3.9513446E-3,-3.6294648E-1,-1.4950543E-2,1.10954516E-1,1.0057674E-2,-3.1813286E-2,8.74128E-3,4.221578E-3,-8.982221E-3,4.1197054E-3,2.0725759E-2,1.1087245E-3,-1.0858509E-2,5.249893E-3,-3.961544E-3,-7.675833E-3,-2.6565394E-3,-1.9565515E-2,-6.8258285E-3,7.619883E-3,2.2116074E-4,-1.9650375E-3,-7.7636135E-3,1.7810792E-2,2.1903557E-3,5.339959E-3,1.18149305E-2,-3.1825502E-3,-1.3914727E-2,2.2246577E-3,-2.9245003E-3,-2.376027E-3,-6.82144E-5,4.772448E-3,1.2542534E-2,1.20764505E-2,1.3305168E-3,1.4040633E-3,1.2254552E-2,-4.531189E-3,4.1483776E-3,-2.2724539E-3,-2.4368213E-2,-1.4380303E-2,1.7712743E-4,3.2866574E-3,9.270804E-3,4.883596E-3,-1.1670967E-3,-1.8728632E-3,4.719962E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,-1,-1,67,69,71,73,75,77,79,81,83,85,-1,87,-1,-1,89,91,-1,93,-1,-1,95,97,99,-1,-1,101,103,105,107,-1,-1,-1,109,111,113,115,-1,-1,117,119,121,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1215885E0,1.0741779E0,1.4236231E0,1.6476667E-1,1.0600504E0,8.614352E-1,1.2596066E0,0E0,0E0,8.846085E-1,1.2796527E0,5.447122E-1,3.6074227E-1,1.1246893E0,8.5051435E-1,8.1466955E-1,9.33512E-1,2.356987E-1,7.9243463E-1,5.1633275E-1,2.516874E-1,3.3389193E-1,0E0,5.8234215E-1,8.826965E-1,4.4696543E-1,4.3436736E-1,1.6951665E-1,2.8506285E-1,3.9501965E-1,7.2198844E-1,0E0,1.4594895E-1,5.0928867E-1,4.5843518E-1,2.9222548E-1,5.0654376E-1,0E0,0E0,3.366959E-1,2.547513E-1,4.2396832E-1,5.881298E-1,1.3324183E-1,6.632079E-1,2.1676207E-1,2.6821584E-1,2.1353227E-1,2.4463135E-1,0E0,2.1257113E-1,0E0,0E0,1.14203446E-1,5.211859E-1,0E0,5.4267454E-1,0E0,0E0,3.7037247E-1,2.772845E-1,1.4739406E-1,0E0,0E0,1.3055658E-1,2.1726173E-1,4.851237E-1,2.3106104E-1,0E0,0E0,0E0,1.497218E-1,2.6627445E-1,1.4312279E-1,2.023536E-1,0E0,0E0,2.5201178E-1,3.469784E-1,1.2660825E-1,1.512156E-1,2.3207206E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,53,53,54,54,56,56,59,59,60,60,61,61,64,64,65,65,66,66,67,67,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,-1,-1,68,70,72,74,76,78,80,82,84,86,-1,88,-1,-1,90,92,-1,94,-1,-1,96,98,100,-1,-1,102,104,106,108,-1,-1,-1,110,112,114,116,-1,-1,118,120,122,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3E0,5.862126E2,5.101512E0,3.386E3,2E0,5.511673E0,-1.8904464E-2,-6.8136784E-3,1E0,7.4521235E6,2.1442623E1,1.3137E4,2.1351435E0,7.115465E6,1.56E2,9.457831E0,3.496664E9,4.5505118E2,4.296196E0,1.1668591E6,1.8149056E7,1.5296751E-2,1.3195207E7,4.0208E4,6.486406E6,4.516183E1,1.64E1,1.2920592E0,5.8365756E-1,1.3E1,1.3459782E-2,2.262586E6,2.8091298E7,6.7321223E-1,5.714286E-1,1.632095E2,2.2614636E-3,2.0664686E-2,3.15856E0,3.37E2,2.264389E-1,1.0594161E1,2.1155348E6,3.333E3,2.881356E-2,2.6312E4,1.33832655E1,6.4683E4,-8.09971E-3,3.64299E5,1.5072074E-2,4.7068493E-3,1.3483871E2,5.8941E1,3.1055901E-3,8E0,3.3996522E-3,1.2131323E-2,7.55E2,1.5265896E8,5.5544252E7,9.0543233E-4,-6.7965663E-3,6.7317666E-4,1.6521739E0,1.604E3,1.7248E4,7.0322594E-3,7.1244105E-3,-2.0801563E-5,8.66076E1,2.02E2,1.8820395E6,6.1707315E0,9.344088E-3,3.9513446E-3,2.821E3,1.683653E8,1.4122984E5,3E0,1E0,8.74128E-3,4.221578E-3,-8.982221E-3,4.1197054E-3,2.0725759E-2,1.1087245E-3,-1.0858509E-2,5.249893E-3,-3.961544E-3,-7.675833E-3,-2.6565394E-3,-1.9565515E-2,-6.8258285E-3,7.619883E-3,2.2116074E-4,-1.9650375E-3,-7.7636135E-3,1.7810792E-2,2.1903557E-3,5.339959E-3,1.18149305E-2,-3.1825502E-3,-1.3914727E-2,2.2246577E-3,-2.9245003E-3,-2.376027E-3,-6.82144E-5,4.772448E-3,1.2542534E-2,1.20764505E-2,1.3305168E-3,1.4040633E-3,1.2254552E-2,-4.531189E-3,4.1483776E-3,-2.2724539E-3,-2.4368213E-2,-1.4380303E-2,1.7712743E-4,3.2866574E-3,9.270804E-3,4.883596E-3,-1.1670967E-3,-1.8728632E-3,4.719962E-3],"split_indices":[102,3,52,56,2,17,53,0,0,79,45,56,2,41,1,2,54,31,56,58,28,9,0,9,9,45,54,4,56,54,0,0,1,45,41,53,4,0,0,38,0,38,56,45,2,57,2,53,2,0,9,0,0,52,50,0,17,0,0,10,45,45,0,0,42,53,2,11,0,0,0,56,8,45,54,0,0,2,7,33,8,89,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.717E3,1.032E3,1.685E3,1.4E1,1.018E3,9.72E2,7.13E2,8E0,6E0,5.98E2,4.2E2,3.46E2,6.26E2,2.95E2,4.18E2,6.6E1,5.32E2,4E1,3.8E2,3.39E2,7E0,6.24E2,2E0,1.79E2,1.16E2,3.29E2,8.9E1,3.4E1,3.2E1,4.92E2,4E1,1.3E1,2.7E1,3.6E2,2E1,4.6E1,2.93E2,3E0,4E0,5.73E2,5.1E1,1.37E2,4.2E1,4.7E1,6.9E1,8.5E1,2.44E2,8E1,9E0,9E0,2.5E1,8E0,2.4E1,1.9E1,4.73E2,7E0,3.3E1,2.2E1,5E0,1.3E2,2.3E2,1E1,1E1,3E0,4.3E1,2.4E1,2.69E2,5.64E2,9E0,1.6E1,3.5E1,6.1E1,7.6E1,1.5E1,2.7E1,1.9E1,2.8E1,5E0,6.4E1,3.9E1,4.6E1,2.39E2,5E0,7.8E1,2E0,4E0,5E0,2.2E1,3E0,1.6E1,3E0,5.3E1,4.2E2,1.6E1,1.7E1,1.7E1,1.13E2,2.1E2,2E1,8E0,2E0,3.3E1,1E1,1.9E1,5E0,2.18E2,5.1E1,4.35E2,1.29E2,5.5E1,6E0,7E1,6E0,4E0,1.1E1,1.9E1,8E0,2E0,3E0,3E0,6.1E1,2.7E1,1.2E1,1.2E1,3.4E1,2.27E2,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[4.8568082E-4,-3.258395E-2,2.14746E-2,-3.751854E-2,1.9723402E-1,6.969378E-2,9.344302E-4,-5.8749013E-2,-1.5686114E-3,2.1594968E-2,7.391775E-2,4.3773275E-2,1.5361112E-1,-2.5254713E-2,5.379143E-2,5.0714944E-2,-7.271401E-2,6.937837E-2,-5.205534E-2,7.054741E-3,-1.9868636E-3,4.8532557E-2,-1.9624734E-2,1.7196076E-1,-1.7195893E-4,-3.5252433E-2,2.370557E-2,4.557284E-2,9.571321E-3,-5.7779346E-2,1.4678124E-1,-6.41821E-2,-1.7803784E-1,1.7666084E-1,3.6273204E-2,-4.132565E-2,-1.9737004E-1,1.1688049E-2,4.189867E-2,1.9445686E-1,7.4858986E-2,5.4707425E-3,-3.952174E-2,-3.2989603E-2,5.5820603E-2,7.867378E-2,4.6786005E-3,-1.4746554E-2,-1.856222E-2,1.55604975E-2,8.25096E-2,-5.4471683E-2,-1.6453518E-1,-3.6175594E-1,-8.320711E-2,2.8390616E-1,3.3789493E-2,5.8580093E-2,-3.2462347E-3,1.0958357E-2,-4.6395212E-2,-1.1872441E-2,1.7173376E-4,-5.3460016E-3,7.047921E-2,1.6799125E-1,1.4117274E-2,-1.4391664E-3,6.787559E-3,-8.801597E-2,-2.7993293E-2,3.608498E-3,-7.119545E-2,1.1746106E-1,1.7550502E-2,8.8049635E-2,-1.3008593E-1,-2.0380553E-2,9.364785E-2,-8.570586E-3,1.4382713E-3,5.1622824E-3,-3.1352933E-3,-3.0331928E-3,4.127127E-3,-6.215174E-3,-2.0285577E-2,-1.9403074E-2,-3.2362451E-3,-1.136077E-2,-2.3502524E-3,1.4919993E-2,6.816437E-4,6.2146154E-3,-3.1580215E-3,-6.3167783E-3,3.5519456E-3,-4.4889227E-3,-1.0131429E-3,2.8760877E-4,-1.4665421E-2,1.094432E-2,2.8974812E-3,9.184335E-3,2.9953283E-3,-1.5149879E-3,-7.7856192E-3,-1.2036066E-4,-2.865393E-3,-1.4624825E-4,-6.321722E-3,7.5754356E-3,5.7737326E-4,3.1520605E-5,7.1722777E-3,9.348004E-3,3.5657373E-3,-1.1006491E-2,2.4924967E-3,4.6678516E-3,-1.5465537E-3,6.7038545E-3,-4.1196668E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,-1,39,-1,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,65,67,-1,69,71,73,75,77,79,-1,-1,81,83,85,87,89,91,93,95,-1,-1,97,-1,-1,99,101,103,-1,-1,-1,105,107,-1,109,111,113,115,117,119,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8574059E0,1.1856854E0,1.6212689E0,7.7688E-1,6.549176E-1,1.0569389E0,1.5918957E0,9.8203254E-1,1.3611443E0,0E0,1.4162874E-1,8.293636E-1,3.3234596E-1,3.7725434E-1,4.482684E-1,7.693339E-1,4.989853E-1,5.547454E-1,3.3798188E-1,0E0,0E0,4.7229213E-1,0E0,2.1043754E-1,0E0,4.1232765E-1,2.4036202E-1,4.896487E-1,0E0,4.9103808E-1,4.197076E-1,5.045836E-1,7.016283E-1,5.573313E-1,2.871576E-1,2.954775E-1,1.5322721E-1,0E0,4.8782724E-1,1.7085814E-1,1.5158515E-1,0E0,3.4527344E-1,2.0404436E-1,1.9609335E-1,4.0291417E-1,3.6531612E-1,2.4068373E-1,0E0,0E0,1.1929986E-1,6.0639644E-1,3.5919118E-1,1.513598E-1,1.4080627E-1,1.520803E-1,1.712818E-1,3.0967113E-1,0E0,0E0,2.4295044E-1,0E0,0E0,4.6975923E-1,3.409667E-1,1.5856147E-1,0E0,0E0,0E0,4.9507827E-1,4.0872228E-1,0E0,1.4876528E-1,1.3544405E-1,1.1827124E-1,2.651143E-1,1.754539E-1,1.8285187E-1,3.117428E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,60,60,63,63,64,64,65,65,69,69,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,-1,40,-1,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,66,68,-1,70,72,74,76,78,80,-1,-1,82,84,86,88,90,92,94,96,-1,-1,98,-1,-1,100,102,104,-1,-1,-1,106,108,-1,110,112,114,116,118,120,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,4.5505118E2,2E0,3.83E3,1.6907827E9,1.3572903E10,9.557794E2,1E0,2.8091298E7,2.1594968E-2,5.1797544E7,1.3361491E10,2.3977574E7,1.5083523E0,4.091015E9,4.1161446E1,9.457831E0,2.0131812E7,7.756375E-1,7.054741E-3,-1.9868636E-3,1.3E1,-1.9624734E-2,1E0,-1.7195893E-4,3.4E2,1.088E3,9.213145E0,9.571321E-3,3.64299E5,7.330957E4,6.496696E7,2E0,1.65E2,1.1953147E2,3.53562E5,8.985569E-1,1.1688049E-2,1.5687E4,1.8E1,4.1365924E2,5.4707425E-3,8.87E2,4.7022E4,9.1E3,2.3821254E7,1.5435694E9,1.64E1,-1.856222E-2,1.55604975E-2,1.0715278E7,1.0289962E3,2.55298E5,6E1,4.22088E0,6.5E1,1.0545219E5,3.290375E-2,-3.2462347E-3,1.0958357E-2,2.0288463E0,-1.1872441E-2,1.7173376E-4,2.4E1,4.1E1,1.5837E4,1.4117274E-2,-1.4391664E-3,6.787559E-3,1.6908121E1,3.5188188E5,3.608498E-3,4.5154482E-1,4.949877E6,1.30399E5,5.42974E5,6.117756E9,8.573816E0,1.100592E6,-8.570586E-3,1.4382713E-3,5.1622824E-3,-3.1352933E-3,-3.0331928E-3,4.127127E-3,-6.215174E-3,-2.0285577E-2,-1.9403074E-2,-3.2362451E-3,-1.136077E-2,-2.3502524E-3,1.4919993E-2,6.816437E-4,6.2146154E-3,-3.1580215E-3,-6.3167783E-3,3.5519456E-3,-4.4889227E-3,-1.0131429E-3,2.8760877E-4,-1.4665421E-2,1.094432E-2,2.8974812E-3,9.184335E-3,2.9953283E-3,-1.5149879E-3,-7.7856192E-3,-1.2036066E-4,-2.865393E-3,-1.4624825E-4,-6.321722E-3,7.5754356E-3,5.7737326E-4,3.1520605E-5,7.1722777E-3,9.348004E-3,3.5657373E-3,-1.1006491E-2,2.4924967E-3,4.6678516E-3,-1.5465537E-3,6.7038545E-3,-4.1196668E-3],"split_indices":[102,56,17,2,12,31,52,79,45,0,50,31,47,42,7,55,54,12,34,0,0,10,0,74,0,12,10,53,0,9,45,45,17,3,58,1,27,0,1,8,52,0,2,9,29,9,7,4,0,0,45,4,11,3,53,8,28,38,0,0,53,0,0,8,0,29,0,0,0,58,48,0,38,9,10,1,12,58,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.674E3,1.038E3,1.636E3,1.017E3,2.1E1,4.88E2,1.148E3,6.39E2,3.78E2,6E0,1.5E1,3.74E2,1.14E2,7.68E2,3.8E2,7.2E1,5.67E2,1.57E2,2.21E2,9E0,6E0,3.71E2,3E0,1.02E2,1.2E1,6.38E2,1.3E2,3.61E2,1.9E1,3.4E1,3.8E1,5.26E2,4.1E1,3.6E1,1.21E2,2.07E2,1.4E1,1.1E1,3.6E2,8.2E1,2E1,1.7E1,6.21E2,4.7E1,8.3E1,1.99E2,1.62E2,3.1E1,3E0,9E0,2.9E1,4.81E2,4.5E1,1.3E1,2.8E1,2E1,1.6E1,1E2,2.1E1,3E0,2.04E2,1.1E1,3E0,1.36E2,2.24E2,6.7E1,1.5E1,8E0,1.2E1,1.18E2,5.03E2,1.2E1,3.5E1,3.1E1,5.2E1,1.91E2,8E0,1.27E2,3.5E1,6E0,2.5E1,2.5E1,4E0,4.53E2,2.8E1,4.1E1,4E0,1.1E1,2E0,4E0,2.4E1,1.8E1,2E0,8E0,8E0,7E0,9.3E1,7E1,1.34E2,1.32E2,4E0,1.2E1,2.12E2,5.4E1,1.3E1,6.8E1,5E1,2.8E2,2.23E2,1.7E1,1.8E1,2.2E1,9E0,4.7E1,5E0,2E1,1.71E2,5E0,3E0,1.1E1,1.16E2,2.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[1.122273E-3,1.6461233E-2,-4.168877E-2,4.798923E-3,8.368394E-2,-9.899803E-2,-2.475647E-2,-5.716325E-3,6.0941726E-2,9.858676E-2,-1.340662E-1,-7.43938E-2,-2.2294313E-1,-1.291578E-1,-1.5764412E-2,-2.0147132E-2,3.460444E-2,2.4177296E-2,9.235781E-2,2.705808E-1,8.236896E-2,-1.6497461E-2,-4.264539E-3,-1.05353616E-1,2.85504E-2,-3.0005276E-1,-4.3952387E-2,-2.0805192E-1,-3.0665249E-3,-9.209467E-3,-1.5836276E-1,-4.0898856E-2,5.556926E-3,8.526216E-2,4.017814E-3,-1.74771E-2,7.772601E-2,1.0861144E-1,-6.400479E-2,1.4851156E-2,3.1736067E-3,6.284748E-2,1.8034804E-1,-5.514342E-2,-1.8762335E-1,-6.381103E-3,7.918243E-2,-3.3320247E-3,-1.6355086E-2,-9.450627E-3,1.3439644E-3,-1.9333801E-3,-1.2693598E-2,-2.5268849E-2,4.1540045E-2,-7.816136E-2,-1.7202582E-2,-3.8077388E-2,-2.7114008E-2,1.9261189E-2,-9.910443E-2,9.271901E-2,-1.3705013E-2,4.9940247E-2,-2.8525937E-2,1.4451918E-2,-4.354447E-1,1.2967029E-1,2.0979121E-2,-2.115313E-3,1.2043811E-1,-1.484644E-2,1.4975125E-3,4.4093594E-2,1.4657314E-1,3.4024152E-1,1.02452755E-1,-1.2364596E-1,2.6186744E-2,1.6441878E-3,-2.2901927E-1,1.0647811E-2,-7.297423E-4,2.8201488E-3,-5.59421E-2,1.06671676E-1,1.0897415E-2,-7.036022E-3,8.407499E-4,-1.3786291E-3,-7.141334E-3,2.4715096E-3,-1.0682301E-3,-3.8040362E-3,-1.9494329E-2,4.8629413E-3,-5.6668487E-3,3.0278447E-3,-7.761693E-3,-2.2699519E-3,4.366605E-3,-1.1986471E-3,4.5281085E-3,-1.316429E-3,-3.3439707E-2,8.342582E-3,-2.9999914E-3,-2.1727125E-3,5.157528E-3,6.544728E-3,3.0906938E-4,9.184618E-3,1.3820525E-3,9.825487E-3,2.547095E-3,2.0980364E-2,7.3306053E-3,-5.1103258E-3,7.741795E-3,1.5449917E-4,-7.662981E-3,6.671737E-3,-2.0591917E-3,2.1146953E-3,-1.23466E-2,1.954326E-3,-2.2776462E-3,1.2962887E-4,-3.4331006E-3,6.44116E-3,-1.5227275E-3,7.187468E-3,-2.1574109E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,-1,-1,73,75,77,79,-1,81,-1,-1,-1,-1,-1,-1,83,85,87,-1,89,-1,91,93,95,-1,97,99,101,103,105,107,-1,109,-1,-1,111,113,115,117,119,121,-1,123,-1,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7500734E0,1.5377517E0,6.797246E-1,9.880131E-1,9.5429397E-1,4.7134614E-1,5.0866187E-1,8.211216E-1,3.026855E-1,7.364199E-1,1.4930499E-1,4.3327093E-1,3.4566152E-1,2.0761406E-1,4.677731E-1,5.5443347E-1,5.760523E-1,2.7571476E-1,3.6783898E-1,1.6518772E-1,4.668398E-1,0E0,0E0,4.192549E-1,2.7269307E-1,1.482271E-1,1.04953475E-1,1.7090607E-1,0E0,3.9306748E-1,3.2629853E-1,8.306063E-1,6.6886145E-1,4.142977E-1,3.4964034E-1,9.404637E-1,1.5645418E-1,2.384088E-1,3.3782244E-1,0E0,0E0,3.232752E-1,4.7579432E-1,3.709452E-1,3.66197E-1,0E0,3.413034E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.1549633E-1,2.3035088E-1,1.144607E-1,0E0,5.868136E-1,0E0,5.5509883E-1,2.9481292E-1,2.690066E-1,0E0,2.8343478E-1,3.1382763E-1,2.1108393E-1,5.1687753E-1,2.507E-1,1.6022544E-1,0E0,2.210052E-1,0E0,0E0,3.8372576E-1,1.9701612E-1,1.7102921E-1,3.763581E-1,1.6660672E-1,2.48613E-1,0E0,2.709161E-1,0E0,0E0,3.6939448E-1,1.6178668E-1,1.468369E-1,1.7217603E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,46,46,53,53,54,54,55,55,57,57,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,70,70,73,73,74,74,75,75,76,76,77,77,78,78,80,80,83,83,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,-1,-1,74,76,78,80,-1,82,-1,-1,-1,-1,-1,-1,84,86,88,-1,90,-1,92,94,96,-1,98,100,102,104,106,108,-1,110,-1,-1,112,114,116,118,120,122,-1,124,-1,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.5106794E3,1.8903887E0,2.065904E6,2.3580047E10,2.0354E4,1.2209878E3,4.264897E6,6.789622E7,1.2684739E1,8.962E3,1.6772E4,1.038946E6,4.3256033E2,1.5265896E8,3.7311627E2,2.69E2,5.2921E4,3.1984328E5,1.4659271E7,9.256843E3,-1.6497461E-2,-4.264539E-3,1.8481675E0,7.68E2,2.04425E2,1.4560842E-2,4.0814578E2,-3.0665249E-3,2.0601392E3,2.894E3,4.5158855E6,3.7852024E1,1E0,1.5E1,2.829581E10,1.366E3,1.5139E4,1.4523809E0,1.4851156E-2,3.1736067E-3,4.43837E0,4.0601485E6,2.8884522E8,2.0304577E-1,-6.381103E-3,6.028929E5,-3.3320247E-3,-1.6355086E-2,-9.450627E-3,1.3439644E-3,-1.9333801E-3,-1.2693598E-2,1.4540612E7,2.5927516E11,3.931011E6,-1.7202582E-2,1.0219829E1,-2.7114008E-2,1.1394967E7,3.3720784E5,3.1826714E7,-1.3705013E-2,1E0,1E0,1.164249E7,3.8866E4,2.5652175E0,1.72797E5,-2.115313E-3,2.3821254E7,-1.484644E-2,1.4975125E-3,3.53562E5,1.745982E0,3.275343E6,5.0279167E1,4.7937644E-1,7.5891815E6,1.6441878E-3,4.107E3,1.0647811E-2,-7.297423E-4,1.8344E4,2.765625E0,1.4285097E1,2.99665E5,-7.036022E-3,8.407499E-4,-1.3786291E-3,-7.141334E-3,2.4715096E-3,-1.0682301E-3,-3.8040362E-3,-1.9494329E-2,4.8629413E-3,-5.6668487E-3,3.0278447E-3,-7.761693E-3,-2.2699519E-3,4.366605E-3,-1.1986471E-3,4.5281085E-3,-1.316429E-3,-3.3439707E-2,8.342582E-3,-2.9999914E-3,-2.1727125E-3,5.157528E-3,6.544728E-3,3.0906938E-4,9.184618E-3,1.3820525E-3,9.825487E-3,2.547095E-3,2.0980364E-2,7.3306053E-3,-5.1103258E-3,7.741795E-3,1.5449917E-4,-7.662981E-3,6.671737E-3,-2.0591917E-3,2.1146953E-3,-1.23466E-2,1.954326E-3,-2.2776462E-3,1.2962887E-4,-3.4331006E-3,6.44116E-3,-1.5227275E-3,7.187468E-3,-2.1574109E-4],"split_indices":[27,52,53,28,5,9,48,9,7,56,0,9,9,52,45,52,10,1,33,50,4,0,0,54,2,4,38,33,0,52,0,47,53,65,3,31,10,9,54,0,0,35,47,12,27,0,51,0,0,0,0,0,0,45,31,1,0,54,0,45,28,45,0,85,101,45,1,54,1,0,9,0,0,1,34,29,56,27,45,0,12,0,0,9,54,54,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.663E3,1.961E3,7.02E2,1.672E3,2.89E2,1.59E2,5.43E2,1.409E3,2.63E2,2.71E2,1.8E1,1.34E2,2.5E1,4.2E1,5.01E2,1.038E3,3.71E2,1.22E2,1.41E2,2.2E1,2.49E2,2E0,1.6E1,1.03E2,3.1E1,1.7E1,8E0,1.8E1,2.4E1,4.8E2,2.1E1,5.74E2,4.64E2,1.39E2,2.32E2,6.9E1,5.3E1,1.28E2,1.3E1,1.8E1,4E0,2.09E2,4E1,6.5E1,3.8E1,7E0,2.4E1,3E0,1.4E1,2E0,6E0,5E0,1.3E1,3.65E2,1.15E2,1.6E1,5E0,5.72E2,2E0,4.11E2,5.3E1,1.37E2,2E0,9.6E1,1.36E2,6.5E1,4E0,2.7E1,2.6E1,9E0,1.19E2,3E0,1E1,1.72E2,3.7E1,1.2E1,2.8E1,3.5E1,3E1,6E0,3.2E1,9E0,1.5E1,1.91E2,1.74E2,3.6E1,7.9E1,9E0,7E0,5.29E2,4.3E1,2.31E2,1.8E2,5.1E1,2E0,1.32E2,5E0,9.1E1,5E0,1.18E2,1.8E1,4.4E1,2.1E1,2E0,2E0,2.2E1,5E0,1.5E1,1.1E1,1.04E2,1.5E1,1.5E1,1.57E2,2.2E1,1.5E1,7E0,5E0,6E0,2.2E1,8E0,2.7E1,1.1E1,1.9E1,3E0,2.9E1,1.09E2,8.2E1,3.7E1,1.37E2,3E1,6E0,7E0,7.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-4.3682673E-4,-1.1988949E-2,5.3294897E-2,-3.5073673E-3,-1.0489852E-1,6.269076E-2,-1.5192501E-1,-4.4811577E-2,1.283393E-2,-9.74755E-2,-1.9853769E-2,1.3705349E-1,3.815474E-2,-1.5931306E-2,-3.7795585E-2,-2.5494812E-2,-9.230509E-2,2.7599087E-2,-2.8364308E-2,-1.5141769E-1,-7.455758E-2,1.692949E-1,-9.4448385E-4,1.8228639E-2,9.634931E-2,-4.62945E-3,4.190758E-3,-4.556368E-2,4.2970322E-2,-6.4062715E-2,-2.555597E-1,3.8849052E-2,-1.3412082E-2,-1.0509772E-1,-7.689571E-3,-1.0829665E-1,-2.757673E-1,-1.0070515E-2,-6.38462E-2,2.2418816E-1,4.0420484E-2,5.2466318E-2,-3.3138655E-2,2.20241E-1,6.747963E-2,-2.956571E-2,-1.5214601E-1,9.680287E-2,-7.344682E-3,-2.2085331E-1,-5.051151E-2,-3.4432095E-3,-3.269561E-1,-1.2965984E-2,4.875814E-2,-2.5820075E-2,1.4357159E-1,-2.114322E-2,-8.64994E-2,-2.0533808E-2,1.0132536E-1,-8.252397E-3,1.3160965E-3,-4.291358E-3,-1.6692175E-2,3.669709E-2,-7.6288424E-2,2.452613E-1,6.9656246E-4,7.6099024E-3,-4.0812775E-2,7.824821E-2,-2.0499572E-2,3.0964212E-3,-7.1609765E-2,2.0267943E-2,5.618303E-3,9.11328E-2,-1.6517588E-1,-1.0537513E-3,-1.2043106E-2,-3.348048E-3,-9.672948E-3,1.445865E-4,7.3576714E-3,4.12602E-4,-7.1604773E-3,1.8000358E-3,-1.79064E-2,-1.5744084E-3,-8.444079E-3,-2.1234956E-2,-4.657603E-3,1.0151772E-3,-7.519133E-3,8.624463E-3,2.1188895E-3,-1.8367084E-3,4.6769585E-3,1.2908282E-2,1.3620995E-3,-2.4866106E-3,-1.153468E-2,2.1141423E-3,-1.5131911E-3,8.246741E-3,9.428899E-4,8.08588E-3,-2.6567862E-3,-8.797418E-3,-2.9794667E-3,3.760851E-3,1.2415387E-2,-9.000331E-3,1.4477052E-3,5.7559083E-3,1.3270079E-3,-4.511221E-3,2.9497847E-3,-2.3017826E-3,2.339792E-3,-5.369397E-3,-2.5101026E-4,5.092409E-4,6.5552206E-3,-1.4664816E-2,5.983849E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,37,39,-1,41,43,-1,-1,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,97,99,-1,101,103,105,-1,-1,-1,-1,107,109,111,-1,-1,113,115,117,119,121,-1,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6591905E0,1.7332764E0,9.215708E-1,1.3619782E0,3.8254452E-1,8.212987E-1,4.205997E-1,5.2241147E-1,8.803824E-1,2.1247065E-1,0E0,5.694599E-1,3.9482367E-1,0E0,1.1173403E-1,5.625571E-1,7.443932E-1,4.918844E-1,6.0521317E-1,2.5432575E-1,1.7411381E-1,6.4885163E-1,0E0,4.519651E-1,2.9631233E-1,0E0,0E0,5.343136E-1,2.5260943E-1,2.9055208E-1,2.9120016E-1,4.2986798E-1,4.5171303E-1,4.6857512E-1,4.2494515E-1,3.6113536E-1,1.4490342E-1,0E0,1.541684E-1,2.8359222E-1,2.855998E-1,2.9130867E-1,1.4380412E-1,2.8460783E-1,4.1127178E-1,4.5763236E-1,1.5390396E-1,2.393839E-1,1.1247449E-1,4.4914806E-1,2.870865E-1,0E0,4.1160405E-1,6.67661E-1,4.0816402E-1,3.349024E-1,2.3887423E-1,0E0,4.037876E-1,1.9530866E-1,1.8176073E-1,0E0,0E0,0E0,0E0,1.7988849E-1,1.5183747E-1,1.1720204E-1,0E0,0E0,1.9381587E-1,2.3830849E-1,2.5386187E-1,1.2876807E-1,1.3383615E-1,0E0,0E0,2.4159354E-1,3.2469398E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,65,65,66,66,67,67,70,70,71,71,72,72,73,73,74,74,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,38,40,-1,42,44,-1,-1,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,98,100,-1,102,104,106,-1,-1,-1,-1,108,110,112,-1,-1,114,116,118,120,122,-1,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5106794E3,4.2343444E7,1.2367184E3,1.638058E2,4.1973075E6,1.1317E4,1.5166431E5,2.5943396E0,5.884028E0,1.0275E4,-1.9853769E-2,5.5544252E7,5.528107E3,-1.5931306E-2,5.7763332E7,1.04E3,3.2785368E7,1.4540612E7,2.373E3,1.8600049E10,3E0,2.2277021E0,-9.4448385E-4,2.589369E7,2.0399538E5,-4.62945E-3,4.190758E-3,1E0,9.887221E5,1.198E5,3.8609805E10,7E2,1.3238013E2,3.1964528E7,1.2502964E9,5.35E3,6.85945E4,-1.0070515E-2,1.6559546E0,7.273333E1,2.53383E5,1.0938637E1,3.6E1,2.4394053E9,1.56E2,1E0,1.71E2,1.0119178E10,4.438435E6,7.635605E-6,7.686371E5,-3.4432095E-3,8.17766E5,1E2,1.1693E4,1E0,3.7974813E8,-2.114322E-2,8.1407714E-1,1.4858E4,2.130881E6,-8.252397E-3,1.3160965E-3,-4.291358E-3,-1.6692175E-2,9.222717E-1,7.8040985E2,3.689E3,6.9656246E-4,7.6099024E-3,7.883E3,2.1374066E7,1.04E2,1.7E1,1E0,2.0267943E-2,5.618303E-3,4.9386E4,3.555E3,-1.0537513E-3,-1.2043106E-2,-3.348048E-3,-9.672948E-3,1.445865E-4,7.3576714E-3,4.12602E-4,-7.1604773E-3,1.8000358E-3,-1.79064E-2,-1.5744084E-3,-8.444079E-3,-2.1234956E-2,-4.657603E-3,1.0151772E-3,-7.519133E-3,8.624463E-3,2.1188895E-3,-1.8367084E-3,4.6769585E-3,1.2908282E-2,1.3620995E-3,-2.4866106E-3,-1.153468E-2,2.1141423E-3,-1.5131911E-3,8.246741E-3,9.428899E-4,8.08588E-3,-2.6567862E-3,-8.797418E-3,-2.9794667E-3,3.760851E-3,1.2415387E-2,-9.000331E-3,1.4477052E-3,5.7559083E-3,1.3270079E-3,-4.511221E-3,2.9497847E-3,-2.3017826E-3,2.339792E-3,-5.369397E-3,-2.5101026E-4,5.092409E-4,6.5552206E-3,-1.4664816E-2,5.983849E-3],"split_indices":[52,45,58,52,28,32,28,56,53,9,0,45,52,0,43,2,12,45,2,5,3,41,0,45,33,0,0,104,50,7,31,2,56,7,7,2,32,0,56,54,9,53,3,5,8,14,0,19,51,41,32,0,1,0,1,105,12,0,41,9,1,0,0,0,0,27,4,2,0,0,2,50,8,3,102,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.671E3,2.199E3,4.72E2,2.016E3,1.83E2,4.52E2,2E1,5.71E2,1.445E3,1.8E2,3E0,1.11E2,3.41E2,7E0,1.3E1,4.07E2,1.64E2,1.064E3,3.81E2,5.2E1,1.28E2,9.2E1,1.9E1,2.55E2,8.6E1,9E0,4E0,3.15E2,9.2E1,1.41E2,2.3E1,8.35E2,2.29E2,8E1,3.01E2,4E1,1.2E1,8E0,1.2E2,6.4E1,2.8E1,1.53E2,1.02E2,1.5E1,7.1E1,2.75E2,4E1,4.4E1,4.8E1,1E1,1.31E2,7E0,1.6E1,1.34E2,7.01E2,2.13E2,1.6E1,3E0,7.7E1,2.7E2,3.1E1,2.7E1,1.3E1,4E0,8E0,1.3E1,1.07E2,5.8E1,6E0,1.1E1,1.7E1,1.13E2,4E1,5.3E1,4.9E1,4E0,1.1E1,6.5E1,6E0,2.67E2,8E0,1.6E1,2.4E1,1.7E1,2.7E1,4.4E1,4E0,4E0,6E0,1.16E2,1.5E1,1E1,6E0,1.09E2,2.5E1,2.2E1,6.79E2,1.94E2,1.9E1,7E0,9E0,6.4E1,1.3E1,3.9E1,2.31E2,1.6E1,1.5E1,5E0,8E0,1.1E1,9.6E1,5E0,5.3E1,5E0,1.2E1,6.1E1,5.2E1,2.1E1,1.9E1,2.5E1,2.8E1,3E1,1.9E1,2.4E1,4.1E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[4.828569E-4,-2.9840367E-2,1.8985806E-2,-3.3553567E-2,1.3759737E-1,6.1527655E-2,7.5819873E-4,-2.3596663E-2,-9.0541855E-2,2.3813145E-1,-1.6319931E-3,4.5055754E-2,1.6233203E-1,-2.3350995E-2,3.950535E-2,-5.184071E-2,1.01660695E-2,-1.746807E-1,-6.3440055E-2,1.8066917E-2,6.932481E-3,5.0337013E-2,-2.0523557E-1,1.0892338E-2,8.861718E-2,-1.9702343E-2,-1.713575E-1,3.137077E-2,1.8021442E-1,-2.1582238E-2,-1.0093017E-1,9.763381E-2,-1.252284E-2,-1.3807252E-1,-2.0299643E-2,-1.3128415E-1,-4.140249E-2,3.3047426E-2,1.1387888E-1,-2.4418661E-2,-2.1719413E-3,-1.2022199E-3,7.269195E-3,-6.922451E-2,-9.265046E-3,-8.469876E-4,-1.1202243E-2,8.417833E-2,1.1659266E-2,1.1365007E-2,2.8103844E-3,-1.6678277E-2,-1.1781997E-2,-6.301815E-2,-1.9635841E-1,9.307029E-3,6.998885E-2,-3.2874502E-3,-1.03769824E-1,3.106797E-3,-7.3638717E-3,-4.8004016E-3,-1.6582068E-2,1.3620384E-2,-6.547274E-2,4.2975508E-2,-1.2298664E-1,1.6699685E-1,9.109882E-3,-1.2946209E-2,-1.4621538E-1,8.775503E-3,-3.6246713E-2,1.2075153E-1,1.5821797E-3,1.5889054E-2,-1.1828854E-2,-3.6831878E-3,1.6655715E-4,-6.8794546E-4,-7.499469E-3,-1.20777665E-2,-1.0695722E-3,-4.0482474E-3,4.3491E-3,2.9804683E-3,-1.0347534E-3,-7.3436983E-3,4.9594096E-3,-1.6268764E-3,4.6895538E-3,-6.8528187E-3,-1.1055571E-3,1.0663287E-2,1.6770991E-3,-2.3005684E-3,-2.5462562E-2,1.25526935E-2,6.010716E-3,-9.869995E-3,3.304582E-3,1.5312432E-3,-4.182445E-3,-5.6287018E-3,-1.4832485E-2,3.1385345E-3,-4.0906045E-4,-8.837213E-3,-1.3406385E-3,9.185842E-3,2.5662754E-3,-1.7604264E-3,7.934366E-3,-5.892375E-4,3.169401E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,-1,-1,-1,-1,69,71,-1,-1,73,75,-1,-1,77,-1,79,81,-1,83,85,87,-1,-1,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4879606E0,6.2945545E-1,1.2770966E0,5.562117E-1,3.8781703E-1,8.129432E-1,1.0789461E0,8.005943E-1,3.218441E-1,1.2725759E-1,0E0,5.734404E-1,3.1101704E-1,3.8112104E-1,5.0140834E-1,6.766367E-1,7.613044E-1,2.6076472E-1,1.6201782E-1,0E0,0E0,4.5545673E-1,4.1525084E-1,0E0,2.4805948E-1,3.591054E-1,1.592854E-1,4.3620345E-1,1.5261304E-1,3.0847588E-1,6.171366E-1,1.9759923E-1,2.566828E-1,1.118837E-1,0E0,1.4549279E-1,1.1479141E-1,5.1649475E-1,4.9377394E-1,0E0,0E0,0E0,0E0,5.2238977E-1,2.8069422E-1,0E0,0E0,3.4418076E-1,3.3980227E-1,0E0,0E0,3.3897594E-1,0E0,5.7141125E-1,4.705001E-1,0E0,2.1039826E-1,3.350332E-1,3.0366218E-1,0E0,0E0,0E0,0E0,1.11651145E-1,1.9410613E-1,4.3680233E-1,5.870827E-1,1.9992173E-1,4.137878E-1,2.4003719E-1,2.0277059E-1,3.4064963E-1,2.7752355E-1,3.6675727E-1,2.3258983E-1,4.3042296E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,43,43,44,44,47,47,48,48,51,51,53,53,54,54,56,56,57,57,58,58,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,-1,-1,-1,-1,70,72,-1,-1,74,76,-1,-1,78,-1,80,82,-1,84,86,88,-1,-1,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.832274E2,2E0,7.218466E7,5.1797544E7,2.831692E10,8.027061E2,2.074E3,7.298614E2,1.381E3,-1.6319931E-3,1.3888E4,1.8428E4,1E0,4.091015E9,2.7272727E0,7.4521235E6,2.2515285E6,8.10114E4,1.8066917E-2,6.932481E-3,5.862126E2,2.75E2,1.0892338E-2,5.8084745E6,8.01E2,2.18988E5,6.486406E6,8.931957E9,2.0136518E0,4.44E2,4.6E2,1.1075305E0,1.0804272E3,-2.0299643E-2,1.4285097E1,1.4302E4,4.5158855E6,5.077728E8,-2.4418661E-2,-2.1719413E-3,-1.2022199E-3,7.269195E-3,1.57383175E1,2.6540773E1,-8.469876E-4,-1.1202243E-2,6.587855E1,4.42384E5,1.1365007E-2,2.8103844E-3,4.7E2,-1.1781997E-2,2.3902743E0,4.352552E9,9.307029E-3,1.37E2,1.5548611E7,1.4771426E0,3.106797E-3,-7.3638717E-3,-4.8004016E-3,-1.6582068E-2,2.200423E3,1.7E1,1.3E1,3.324E3,4.8958065E6,5.3084288E8,1.22E2,3.0595E4,4.389587E1,2.23E2,7.21E3,6.1238285E8,6.520433E6,-1.1828854E-2,-3.6831878E-3,1.6655715E-4,-6.8794546E-4,-7.499469E-3,-1.20777665E-2,-1.0695722E-3,-4.0482474E-3,4.3491E-3,2.9804683E-3,-1.0347534E-3,-7.3436983E-3,4.9594096E-3,-1.6268764E-3,4.6895538E-3,-6.8528187E-3,-1.1055571E-3,1.0663287E-2,1.6770991E-3,-2.3005684E-3,-2.5462562E-2,1.25526935E-2,6.010716E-3,-9.869995E-3,3.304582E-3,1.5312432E-3,-4.182445E-3,-5.6287018E-3,-1.4832485E-2,3.1385345E-3,-4.0906045E-4,-8.837213E-3,-1.3406385E-3,9.185842E-3,2.5662754E-3,-1.7604264E-3,7.934366E-3,-5.892375E-4,3.169401E-3],"split_indices":[102,56,17,45,50,31,52,2,52,0,0,10,29,77,7,56,45,28,33,0,0,52,0,0,45,2,1,45,12,39,0,29,38,33,0,54,10,47,7,0,0,0,0,58,58,0,0,58,2,0,0,1,0,53,5,0,0,45,38,0,0,0,0,52,8,10,29,47,7,10,29,48,0,2,7,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E3,1.004E3,1.646E3,9.83E2,2.1E1,4.93E2,1.153E3,8.38E2,1.45E2,1.3E1,8E0,4.25E2,6.8E1,7.11E2,4.42E2,4.56E2,3.82E2,3.4E1,1.11E2,4E0,9E0,4.17E2,8E0,3.5E1,3.3E1,6.95E2,1.6E1,4.19E2,2.3E1,2.83E2,1.73E2,7.8E1,3.04E2,3.1E1,3E0,2.6E1,8.5E1,3.29E2,8.8E1,2E0,6E0,1.2E1,2.1E1,1.2E2,5.75E2,5E0,1.1E1,1.13E2,3.06E2,1.5E1,8E0,2.78E2,5E0,1.25E2,4.8E1,1.6E1,6.2E1,2.77E2,2.7E1,2E0,2.9E1,2.4E1,2E0,2.6E1,5.9E1,3.1E2,1.9E1,5.8E1,3E1,7E1,5E1,3.45E2,2.3E2,7.8E1,3.5E1,3.02E2,4E0,6.9E1,2.09E2,8.3E1,4.2E1,3.6E1,1.2E1,7E0,5.5E1,6E1,2.17E2,2.2E1,5E0,1.7E1,9E0,2E1,3.9E1,1.2E1,2.98E2,1.7E1,2E0,1.6E1,4.2E1,6E0,2.4E1,4.4E1,2.6E1,4.4E1,6E0,8E1,2.65E2,1.1E1,2.19E2,3.7E1,4.1E1,2.9E1,6E0,1.94E2,1.08E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-2.3375808E-3,-3.0542942E-2,1.5454932E-2,-3.4638602E-2,1.4939143E-1,-2.212469E-3,4.76343E-2,4.4114396E-2,-4.2260874E-2,1.16132265E-2,-5.5490103E-2,3.5243947E-2,-2.1806026E-2,8.734131E-2,2.2669258E-2,-4.7645573E-2,1.1664008E-1,-6.31152E-2,-8.571538E-3,1.4051355E-3,-9.896806E-3,1.4917633E-2,1.2757736E-1,7.2100144E-3,-4.1520588E-2,1.5163821E-1,1.2931002E-2,9.885526E-3,1.23094015E-1,-1.3774268E-2,-1.3504645E-2,1.3480528E-2,6.8391375E-2,-5.3653162E-2,-1.859632E-1,4.755254E-2,-3.7932757E-2,3.630101E-2,-1.1223769E-1,2.5926137E-1,7.1628995E-2,8.535104E-2,-2.6145602E-2,-8.877055E-2,-1.9345265E-2,1.666799E-1,-5.403987E-3,1.112712E-1,-4.778498E-2,2.9946636E-2,-2.4491461E-2,7.1088545E-2,1.5431758E-2,-6.500738E-3,1.2413923E-3,4.71735E-3,-1.5225871E-3,8.822755E-3,-7.768028E-2,-3.3398658E-1,-7.336484E-2,7.587133E-2,-3.7110776E-2,4.128978E-2,-5.6272432E-2,5.879385E-2,-5.7095513E-2,-6.568111E-2,-2.1691134E-2,1.411271E-2,2.525947E-3,-1.4946366E-3,4.801824E-3,4.702858E-2,9.157295E-3,-8.739121E-4,-9.137662E-2,-4.2899203E-2,-1.5763474E-1,-2.5695125E-2,6.6496413E-3,1.5048133E-2,1.426418E-1,2.0557644E-3,8.192295E-3,-2.1732806E-1,-1.300755E-2,2.0575918E-2,5.916967E-3,-4.396255E-3,-1.0878928E-5,5.6267856E-3,-1.5731198E-1,-5.337589E-3,2.2702196E-3,-8.587799E-4,-4.766917E-3,-1.7586932E-2,-2.7025172E-3,2.2699737E-3,-5.1549156E-3,5.361387E-3,8.4479264E-4,7.6859468E-3,-4.355861E-3,-1.902168E-4,7.1609295E-3,-1.3035922E-3,-5.0216704E-3,3.183361E-3,-8.092054E-3,-6.5388368E-3,-3.899639E-4,8.1785135E-3,-4.6733487E-3,-4.5870897E-6,5.073401E-3,6.1964325E-4,-5.727922E-3,4.2634456E-5,-6.9022705E-3,-3.6527151E-3,3.2690032E-3,-1.2348758E-3,-8.842099E-3,2.7422336E-4,-2.4829975E-3,8.011823E-3,2.0644777E-3,3.587738E-4,-1.9440027E-2,6.748865E-3,-1.8385946E-3,3.608036E-3,5.2507065E-5,3.499556E-3,-1.3925299E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,-1,-1,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,-1,-1,-1,-1,-1,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,115,-1,117,119,121,123,125,-1,-1,127,-1,-1,129,131,133,-1,-1,-1,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3298576E0,7.6054853E-1,9.2396E-1,6.039418E-1,4.5218682E-1,7.713682E-1,5.690013E-1,5.978484E-1,6.423093E-1,0E0,1.0865055E-1,6.75621E-1,3.9500594E-1,1.0601654E0,4.5453328E-1,3.1361765E-1,3.7842947E-1,6.4677453E-1,5.8020663E-1,0E0,0E0,8.116205E-1,4.6031713E-1,7.320353E-1,4.2892158E-1,4.9221492E-1,6.262825E-1,2.1865985E-1,3.9133924E-1,1.7781556E-1,0E0,0E0,1.2379941E-1,7.902726E-1,6.4218473E-1,2.929208E-1,3.3794692E-1,5.390489E-1,6.5229714E-1,1.2346053E-1,1.4154676E-1,3.3178645E-1,3.2467425E-1,4.0705252E-1,2.8583878E-1,3.5594106E-1,0E0,1.561153E-1,3.8043076E-1,1.7288755E-1,1.9105724E-1,3.682407E-1,0E0,0E0,0E0,0E0,0E0,6.8743205E-1,4.944737E-1,1.3614285E-1,1.044417E-1,1.8921226E-1,3.4442368E-1,2.161499E-1,2.625069E-1,3.7146688E-1,1.9333227E-1,3.1086656E-1,0E0,0E0,0E0,0E0,0E0,1.7490084E-1,0E0,2.3606525E-1,2.6781428E-1,3.0198303E-1,1.8033338E-1,2.2298668E-1,0E0,0E0,2.3571157E-1,0E0,0E0,4.6944743E-1,2.2031587E-1,1.9513503E-1,0E0,0E0,0E0,0E0,2.0573406E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,73,73,75,75,76,76,77,77,78,78,79,79,82,82,85,85,86,86,87,87,92,92],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,-1,-1,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,-1,-1,-1,-1,-1,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,116,-1,118,120,122,124,126,-1,-1,128,-1,-1,130,132,134,-1,-1,-1,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,8.791777E3,7.298614E2,1E0,1.3043794E8,2E0,5.2107143E0,1.91E2,3.83E3,1.16132265E-2,5.0738932E7,6.4547E4,9.969697E0,2.1351435E0,4.5575E4,3.09627E5,8E0,9.457831E0,2.3907686E7,1.4051355E-3,-9.896806E-3,1.604E3,1.5208336E4,1.03E2,1.246E3,3.410165E6,4.0208E4,2.2075728E7,4.516183E1,1.64E1,-1.3504645E-2,1.3480528E-2,1.3905958E7,8.2E1,2E0,4.8718534E0,5.473125E3,1E0,1.2936076E7,1E0,1.2045951E6,9.705292E6,1.4274633E7,5.372E3,2.8366232E0,7.696629E0,-5.403987E-3,3.15856E0,1.1660184E3,4.95E9,1.16072E6,1.3E2,1.5431758E-2,-6.500738E-3,1.2413923E-3,4.71735E-3,-1.5225871E-3,3.79E2,1.5756359E0,6E1,1.09E3,1.2471131E7,6.426937E6,2.2876644E-5,1.775894E10,1.4329944E0,1.9826213E0,2.8216E4,-2.1691134E-2,1.411271E-2,2.525947E-3,-1.4946366E-3,4.801824E-3,1.454213E2,9.157295E-3,1E0,2.64E0,8.2E2,5.28806E5,2.9206951E1,6.6496413E-3,1.5048133E-2,1.2817779E7,2.0557644E-3,8.192295E-3,7.792289E9,1.2236866E3,1.9E1,5.916967E-3,-4.396255E-3,-1.0878928E-5,5.6267856E-3,1.8215276E7,-5.337589E-3,2.2702196E-3,-8.587799E-4,-4.766917E-3,-1.7586932E-2,-2.7025172E-3,2.2699737E-3,-5.1549156E-3,5.361387E-3,8.4479264E-4,7.6859468E-3,-4.355861E-3,-1.902168E-4,7.1609295E-3,-1.3035922E-3,-5.0216704E-3,3.183361E-3,-8.092054E-3,-6.5388368E-3,-3.899639E-4,8.1785135E-3,-4.6733487E-3,-4.5870897E-6,5.073401E-3,6.1964325E-4,-5.727922E-3,4.2634456E-5,-6.9022705E-3,-3.6527151E-3,3.2690032E-3,-1.2348758E-3,-8.842099E-3,2.7422336E-4,-2.4829975E-3,8.011823E-3,2.0644777E-3,3.587738E-4,-1.9440027E-2,6.748865E-3,-1.8385946E-3,3.608036E-3,5.2507065E-5,3.499556E-3,-1.3925299E-2],"split_indices":[102,52,52,79,50,17,53,2,2,0,1,1,58,41,2,9,18,54,45,0,0,2,33,11,2,11,9,45,54,4,0,0,45,0,17,53,32,105,45,106,47,9,45,29,38,56,0,38,4,5,1,8,0,0,0,0,0,1,54,3,2,51,45,39,5,42,56,9,0,0,0,0,0,52,0,77,54,10,45,58,0,0,9,0,0,31,52,8,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.648E3,1.024E3,1.624E3,1.002E3,2.2E1,1.049E3,5.75E2,8.8E1,9.14E2,1.5E1,7E0,3.6E2,6.89E2,2.21E2,3.54E2,3.9E1,4.9E1,5.64E2,3.5E2,5E0,2E0,2.96E2,6.4E1,2.79E2,4.1E2,1.18E2,1.03E2,3.15E2,3.9E1,3.5E1,4E0,1E1,3.9E1,5.25E2,3.9E1,1.2E2,2.3E2,2.54E2,4.2E1,1.8E1,4.6E1,8.3E1,1.96E2,1.3E2,2.8E2,1.12E2,6E0,3.9E1,6.4E1,1.99E2,1.16E2,3.2E1,7E0,8E0,2.7E1,3E1,9E0,1.46E2,3.79E2,1.6E1,2.3E1,9E1,3E1,4.3E1,1.87E2,2.05E2,4.9E1,3.8E1,4E0,1.5E1,3E0,1E1,3.6E1,6.2E1,2.1E1,1.42E2,5.4E1,7.9E1,5.1E1,2.7E2,1E1,1.4E1,9.8E1,1.9E1,2E1,1E1,5.4E1,1.82E2,1.7E1,3E1,8.6E1,2.7E1,5E0,3.5E1,1.11E2,1.02E2,2.77E2,1.4E1,2E0,5E0,1.8E1,5.5E1,3.5E1,6E0,2.4E1,3.1E1,1.2E1,1.18E2,6.9E1,1.99E2,6E0,1.8E1,3.1E1,4E0,3.4E1,3.5E1,2.7E1,1.28E2,1.4E1,2E1,3.4E1,6.1E1,1.8E1,9E0,4.2E1,1.23E2,1.47E2,7.8E1,2E1,5E0,5E0,7E0,4.7E1,4.7E1,1.35E2,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-1.3486892E-3,-1.0264289E-2,5.7059854E-2,-4.4193747E-3,-9.535855E-2,8.872511E-2,9.160142E-3,-3.7424985E-2,8.649954E-3,-1.3783325E-1,-5.385191E-2,7.107852E-2,2.7594775E-1,3.1914353E-2,-8.4621795E-2,-1.2733996E-2,-8.871857E-2,2.1366874E-2,-2.1789758E-2,-1.1518908E-1,-2.6897258E-1,1.8833174E-3,-7.70291E-2,1.4325248E-1,3.417348E-2,8.576038E-3,2.106444E-2,2.9794497E-3,1.2611035E-1,-2.9318685E-2,-2.4634275E-1,-3.5267533E-3,-1.1976595E-1,-2.2989541E-1,-7.087159E-2,1.8671693E-2,1.9165023E-1,-3.910709E-2,6.463009E-2,-1.2559208E-1,3.460283E-3,-1.9784417E-2,-2.0211805E-3,-2.7956371E-3,-9.369476E-3,4.57036E-2,1.943669E-1,4.1260477E-2,-7.311492E-3,4.2258464E-3,-3.1704076E-2,-6.369279E-3,1.6404025E-1,5.152626E-3,-3.147392E-3,3.9091977E-4,-1.6751032E-2,-4.0570132E-2,2.8190786E-2,4.7966214E-3,-1.723475E-1,-3.59599E-1,-3.8291205E-2,-1.9988716E-1,-5.3386066E-2,3.1300582E-2,-1.961302E-2,2.2761813E-1,-2.4937887E-3,-1.5878223E-2,-3.514579E-2,-1.0590588E-2,9.686465E-2,-3.8138982E-3,-1.6749498E-1,4.418951E-3,-2.823961E-3,5.3103287E-3,1.2252159E-2,4.6575665E-2,-9.288807E-3,-3.002236E-3,1.6631221E-3,1.3519164E-2,5.217181E-3,7.4645504E-5,-4.30587E-3,4.2583463E-3,-9.0419117E-4,-3.3690054E-3,-1.9434111E-2,-8.119415E-3,-2.1895196E-2,-8.592947E-3,4.1044788E-3,2.9937704E-3,-1.2705624E-2,4.400748E-3,-3.1035289E-3,-5.612811E-3,1.6760298E-3,-3.815743E-3,1.5152643E-4,2.989912E-3,1.3505614E-2,-2.2327374E-3,2.063714E-3,7.4128984E-3,-4.204454E-3,2.7944434E-3,7.889506E-3,-4.35487E-3,-1.1642092E-2,1.3756546E-3,5.0130417E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,-1,-1,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,-1,-1,-1,-1,75,77,79,-1,-1,81,-1,83,-1,-1,-1,-1,85,87,-1,89,91,93,95,97,99,101,103,-1,-1,105,107,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3920223E0,1.1529766E0,5.361191E-1,9.368087E-1,2.5526166E-1,6.841043E-1,3.0571926E-1,7.790277E-1,6.0279226E-1,1.8579376E-1,1.6931768E-1,5.1716673E-1,2.0702195E-1,3.1306273E-1,2.4320038E-1,4.105183E-1,4.8762286E-1,5.003423E-1,6.894492E-1,1.3387269E-1,3.0943978E-1,0E0,1.2575915E-1,3.2020295E-1,1.7976497E-1,0E0,0E0,2.6598176E-1,2.8717008E-1,1.1671444E-1,1.8712819E-1,4.5348978E-1,3.9887214E-1,5.329691E-1,3.931359E-1,5.2375376E-1,1.6087812E-1,4.3203914E-1,1.8772817E-1,1.0412359E-1,0E0,0E0,0E0,0E0,0E0,1.22424945E-1,1.922921E-1,1.678138E-1,0E0,0E0,1.3391566E-1,0E0,1.2562609E-1,0E0,0E0,0E0,0E0,3.7014115E-1,5.9625006E-1,0E0,6.14385E-1,1.6361654E-1,1.9038406E-1,3.770663E-1,2.697146E-1,4.5093662E-1,3.6826143E-1,1.1716527E-1,0E0,0E0,3.4229505E-1,3.2001305E-1,1.3155234E-1,0E0,1.5732998E-1,0E0,0E0,0E0,0E0,1.2663692E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,45,45,46,46,47,47,50,50,52,52,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,74,74,79,79],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,-1,-1,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,-1,-1,-1,-1,76,78,80,-1,-1,82,-1,84,-1,-1,-1,-1,86,88,-1,90,92,94,96,98,100,102,104,-1,-1,106,108,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,6.747114E7,3.2475834E7,1.8122449E2,1.067536E3,8.791777E3,1.56E2,2.5074627E0,9.669789E0,4.1043E4,1.3E1,2E1,1.341E3,1.5005797E4,1.477425E6,1.2471935E7,8E0,1.3238013E2,7.06699E8,3.582157E12,3.138649E6,1.8833174E-3,4.4967708E7,2.6838362E-1,9.70107E7,8.576038E-3,2.106444E-2,1.10032365E-1,2.7258065E0,1E0,8.015419E3,4.26E2,3.3820656E7,2.6138031E0,1.141E4,1.25218E5,1.1953147E2,3.415965E2,6.1099392E1,2.8277853E11,3.460283E-3,-1.9784417E-2,-2.0211805E-3,-2.7956371E-3,-9.369476E-3,1.2804922E3,2.0718E4,1.4299594E8,-7.311492E-3,4.2258464E-3,5.2E1,-6.369279E-3,1.0317661E5,5.152626E-3,-3.147392E-3,3.9091977E-4,-1.6751032E-2,2.772925E3,1.75E2,4.7966214E-3,2.2214102E6,5.1086234E5,9.5751E4,4.580013E-3,1.0449754E0,2.402216E6,9.51668E5,2.4042394E5,-2.4937887E-3,-1.5878223E-2,3.307766E6,1.4663157E1,1.5435694E9,-3.8138982E-3,8.8070023E-1,4.418951E-3,-2.823961E-3,5.3103287E-3,1.2252159E-2,7.9684106E9,-9.288807E-3,-3.002236E-3,1.6631221E-3,1.3519164E-2,5.217181E-3,7.4645504E-5,-4.30587E-3,4.2583463E-3,-9.0419117E-4,-3.3690054E-3,-1.9434111E-2,-8.119415E-3,-2.1895196E-2,-8.592947E-3,4.1044788E-3,2.9937704E-3,-1.2705624E-2,4.400748E-3,-3.1035289E-3,-5.612811E-3,1.6760298E-3,-3.815743E-3,1.5152643E-4,2.989912E-3,1.3505614E-2,-2.2327374E-3,2.063714E-3,7.4128984E-3,-4.204454E-3,2.7944434E-3,7.889506E-3,-4.35487E-3,-1.1642092E-2,1.3756546E-3,5.0130417E-3],"split_indices":[2,45,50,52,52,52,8,56,54,10,3,3,0,4,9,45,3,56,7,31,29,0,50,41,1,0,0,57,54,64,4,2,7,54,9,29,58,4,54,31,0,0,0,0,0,52,2,45,0,0,8,0,28,0,0,0,0,50,11,0,28,51,9,58,53,7,1,28,0,0,1,54,7,0,27,0,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.671E3,2.318E3,3.53E2,2.17E3,1.48E2,2.12E2,1.41E2,6.15E2,1.555E3,7.2E1,7.6E1,1.95E2,1.7E1,1.14E2,2.7E1,4.16E2,1.99E2,1.097E3,4.58E2,6.3E1,9E0,1.5E1,6.1E1,6.5E1,1.3E2,1.2E1,5E0,8.8E1,2.6E1,2.1E1,6E0,3.84E2,3.2E1,2.1E1,1.78E2,1.081E3,1.6E1,3.82E2,7.6E1,6E1,3E0,5E0,4E0,5.4E1,7E0,2.3E1,4.2E1,1.26E2,4E0,2.5E1,6.3E1,3E0,2.3E1,4E0,1.7E1,2E0,4E0,1.77E2,2.07E2,6E0,2.6E1,1.2E1,9E0,2E1,1.58E2,8.13E2,2.68E2,1.4E1,2E0,4E0,3.78E2,2.3E1,5.3E1,3E1,3E1,1.6E1,7E0,1.9E1,2.3E1,1.24E2,2E0,4.3E1,2E1,6E0,1.7E1,9.6E1,8.1E1,9E1,1.17E2,1.9E1,7E0,5E0,7E0,4E0,5E0,4E0,1.6E1,1.1E1,1.47E2,1.9E1,7.94E2,7.3E1,1.95E2,4E0,1E1,3.3E2,4.8E1,7E0,1.6E1,3.5E1,1.8E1,1.6E1,1.4E1,9.6E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.981525E-3,-1.1756037E-2,5.9991084E-2,-5.6886408E-3,-1.082216E-1,1.3377568E-1,3.93587E-2,-1.9043704E-2,2.6035812E-2,-8.7427296E-2,-1.9871478E-1,1.4802474E-1,-3.1911135E-3,4.986983E-2,-6.8883516E-2,1.6252391E-2,-3.573239E-2,4.837493E-2,-2.110669E-2,-1.2102827E-2,-7.9119265E-2,-1.5646175E-2,-5.090761E-3,1.069153E-1,1.15991505E-2,3.5106856E-2,5.1066293E-3,-1.1131903E-3,-1.2256078E-2,3.7120417E-4,8.710284E-2,-4.4259466E-2,2.2390107E-2,1.0691169E-1,1.4455603E-2,-3.291213E-2,5.500505E-3,-9.315754E-3,-3.3230195E-3,2.8556147E-1,7.4232645E-2,4.3557014E-2,-5.257671E-2,8.06967E-3,-1.3471876E-1,5.5238385E-2,2.0494507E-1,-4.277222E-2,-1.4687151E-2,1.14483394E-1,-1.0987378E-2,1.3169283E-1,-1.3302945E-3,5.0503258E-2,-3.725445E-2,-9.326146E-2,-1.2982337E-2,1.7302072E-2,1.864368E-3,-2.7128668E-3,4.9103266E-3,2.649916E-2,1.3535473E-1,3.481158E-2,-8.480794E-3,2.2465116E-3,-1.119031E-3,-1.2480057E-3,-1.7424354E-2,6.42248E-3,1.5578313E-3,1.535914E-2,6.074591E-3,-1.2592523E-4,-2.5111728E-3,7.412858E-3,1.0045597E-3,2.2970563E-4,-4.8249215E-3,2.4704884E-3,7.863372E-3,3.7189212E-3,-2.563152E-6,-1.0300847E-2,-5.1418657E-4,-1.6115671E-2,-3.6095078E-3,1.0948219E-4,-4.028896E-3,2.9456173E-3,8.065669E-5,-9.803738E-4,8.388705E-3,-2.2566302E-3,1.0078854E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,-1,41,-1,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,57,59,61,63,65,67,69,71,73,-1,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6362358E0,1.3648772E0,5.5496037E-1,9.309017E-1,2.4230754E-1,2.3831475E-1,3.3224446E-1,9.112449E-1,6.866034E-1,1.3655502E-1,2.6098073E-1,2.6447093E-1,0E0,2.1748006E-1,2.1598998E-1,5.591867E-1,5.213866E-1,8.760576E-1,3.4085047E-1,0E0,1.1003637E-1,0E0,0E0,2.9408813E-1,0E0,1.5862843E-1,0E0,0E0,0E0,4.2436123E-1,3.303823E-1,3.443855E-1,4.1622657E-1,5.4437494E-1,5.256728E-1,2.3212266E-1,0E0,0E0,0E0,1.3533938E-1,1.7988935E-1,2.986265E-1,2.1289887E-1,4.7238928E-1,5.382266E-1,1.2638566E-1,1.311124E-1,3.5364342E-1,0E0,1.3242736E-1,1.4216156E-1,3.4822512E-1,0E0,2.2786993E-1,5.447277E-1,1.8353757E-1,1.5987402E-1,0E0,0E0,0E0,0E0,1.4236742E-1,1.89686E-1,1.8260802E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,25,25,29,29,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,53,53,54,54,55,55,56,56,61,61,62,62,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,-1,42,-1,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,58,60,62,64,66,68,70,72,74,-1,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,6.747114E7,1.2427981E9,5.862126E2,1.31496E5,4.5481584E7,3.0272608E10,2E0,1.0238709E1,5.815045E2,1E0,2.3292E4,-3.1911135E-3,4.328683E0,1.0828989E8,7.0025E4,6.808926E6,8.959817E7,4.4854636E7,-1.2102827E-2,3.5315048E2,-1.5646175E-2,-5.090761E-3,7.969174E7,1.15991505E-2,1.2367184E3,5.1066293E-3,-1.1131903E-3,-1.2256078E-2,4.5158855E6,4.681779E2,7.780377E6,2.25E2,3.5301748E7,5.8805E4,7.44393E5,5.500505E-3,-9.315754E-3,-3.3230195E-3,5.8E1,7.398769E4,9.3058E4,1.4858E4,1.4702222E0,6.4166665E0,1.4103535E0,1.6524802E6,1.704E3,-1.4687151E-2,1.9071268E3,1.0950326E6,1.954E3,-1.3302945E-3,1.6837112E7,3.564E3,5.542E3,1.9099288E7,1.7302072E-2,1.864368E-3,-2.7128668E-3,4.9103266E-3,1.6E1,1.36882E5,3.5791788E0,-8.480794E-3,2.2465116E-3,-1.119031E-3,-1.2480057E-3,-1.7424354E-2,6.42248E-3,1.5578313E-3,1.535914E-2,6.074591E-3,-1.2592523E-4,-2.5111728E-3,7.412858E-3,1.0045597E-3,2.2970563E-4,-4.8249215E-3,2.4704884E-3,7.863372E-3,3.7189212E-3,-2.563152E-6,-1.0300847E-2,-5.1418657E-4,-1.6115671E-2,-3.6095078E-3,1.0948219E-4,-4.028896E-3,2.9456173E-3,8.065669E-5,-9.803738E-4,8.388705E-3,-2.2566302E-3,1.0078854E-2],"split_indices":[2,45,12,52,11,47,5,17,53,4,67,2,0,35,45,1,9,12,9,0,33,0,0,7,0,58,0,0,0,47,55,28,10,45,9,1,0,0,0,3,33,2,9,53,56,56,47,29,0,48,47,2,0,45,2,29,9,0,0,0,0,18,11,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.699E3,2.332E3,3.67E2,2.195E3,1.37E2,7.9E1,2.88E2,1.545E3,6.5E2,1.13E2,2.4E1,7.4E1,5E0,2.63E2,2.5E1,4.96E2,1.049E3,4.41E2,2.09E2,4E0,1.09E2,9E0,1.5E1,5.3E1,2.1E1,2.1E2,5.3E1,2.1E1,4E0,4.06E2,9E1,9.15E2,1.34E2,1.61E2,2.8E2,1.93E2,1.6E1,7E0,1.02E2,7E0,4.6E1,1.92E2,1.8E1,3.85E2,2.1E1,7.2E1,1.8E1,9.11E2,4E0,3.5E1,9.9E1,1.36E2,2.5E1,1.65E2,1.15E2,4.7E1,1.46E2,5E0,2E0,8E0,3.8E1,1.63E2,2.9E1,1.1E1,7E0,1.72E2,2.13E2,1.5E1,6E0,1.5E1,5.7E1,6E0,1.2E1,1.77E2,7.34E2,2.4E1,1.1E1,8.5E1,1.4E1,4E1,9.6E1,1.07E2,5.8E1,1.4E1,1.01E2,2E0,4.5E1,1.21E2,2.5E1,6.7E1,9.6E1,6E0,2.3E1,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-2.1274015E-3,3.3004908E-3,-7.89385E-2,-1.644372E-2,3.401707E-2,-1.4766493E-1,-4.390631E-2,1.4348372E-2,-3.0972628E-2,4.9982652E-2,-1.5085784E-2,-1.14800654E-1,-2.972198E-1,-2.487709E-2,-1.5549889E-1,2.5526572E-2,-7.391543E-2,-3.861111E-2,2.5729429E-2,9.680996E-2,2.8908765E-2,-1.587504E-1,3.1106195E-3,-6.9739427E-3,-1.5887533E-3,-1.9633874E-2,-1.2257602E-3,-2.90397E-3,2.3639968E-2,-3.6622456E-3,-1.377513E-2,2.9122543E-2,-1.126579E-2,1.1143223E-1,-1.0783486E-1,-4.4243813E-2,4.328787E-2,9.994241E-2,-6.128154E-3,8.7163515E-2,1.8625112E-2,-1.29254805E-2,5.2835498E-2,-3.4343138E-1,-5.349381E-3,-6.166045E-3,2.1509755E-1,-6.401259E-3,3.841015E-2,5.668741E-2,-1.1391798E-3,-1.6822115E-3,9.126298E-3,-1.705108E-1,1.2799782E-2,-2.356119E-2,-6.482257E-2,2.0655436E-2,7.285168E-3,7.407709E-3,-2.6984276E-2,-9.260876E-2,2.4566954E-2,1.114113E-1,-5.118009E-4,-7.766E-3,-4.4210437E-3,8.394575E-2,8.659322E-3,-4.870825E-3,-2.2541031E-2,4.992273E-2,-3.1797063E-2,2.0142922E-3,1.6527753E-2,8.829968E-3,8.451066E-4,3.2331508E-3,-2.734144E-3,-1.7703656E-3,4.4315876E-3,-3.2963266E-3,-1.3140261E-2,6.8237297E-3,-3.395028E-3,-9.83119E-4,-1.6205305E-2,-6.7488733E-3,-2.2422266E-3,3.0329938E-3,-2.8127108E-3,-4.574992E-3,4.2184885E-3,-9.919357E-4,-9.680875E-3,1.8741095E-3,-7.569317E-3,9.166963E-3,4.265126E-3,-1.977096E-6,-1.23765115E-2,7.180333E-3,1.5543675E-3,-8.053363E-3,1.3830452E-3,3.8049025E-3,-7.5318565E-4,-1.8431812E-3,3.416862E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,69,-1,71,73,-1,75,77,79,-1,-1,81,83,85,87,89,-1,-1,91,93,95,97,-1,-1,99,101,103,-1,-1,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1117755E0,1.5112877E0,4.1726077E-1,6.7911303E-1,7.6511085E-1,2.5586474E-1,2.4670695E-1,4.823758E-1,4.4753772E-1,7.2351086E-1,6.278968E-1,1.209625E-1,2.9577494E-1,1.7840323E-1,1.5552884E-1,4.128794E-1,3.577951E-1,4.2124808E-1,2.9141474E-1,6.085429E-1,5.1017E-1,1.962493E-1,4.2216405E-1,0E0,0E0,0E0,0E0,0E0,1.0736427E-1,0E0,0E0,3.5708523E-1,0E0,1.1107306E-1,3.5754436E-1,3.6067784E-1,1.4218456E-1,2.6123744E-1,2.3321933E-1,5.2792025E-1,0E0,2.3534833E-1,4.445566E-1,1.0592145E-1,0E0,2.9773557E-1,1.8796545E-1,0E0,1.21009514E-1,2.797165E-1,6.9143397E-1,0E0,0E0,3.0852294E-1,1.9461212E-1,3.997844E-1,5.770284E-1,1.7311811E-1,0E0,0E0,1.030661E-1,1.76945E-1,1.7909223E-1,3.0125093E-1,0E0,0E0,1.9690412E-1,6.40674E-1,4.8752475E-1,0E0,0E0,1.2724966E-1,1.00735724E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,28,28,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,63,63,66,66,67,67,68,68,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,70,-1,72,74,-1,76,78,80,-1,-1,82,84,86,88,90,-1,-1,92,94,96,98,-1,-1,100,102,104,-1,-1,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,5.5871747E2,1.067536E3,2E0,1.0658642E1,3.9516E4,4.03424E5,2E0,7.997723E6,4.7931265E6,2.47E2,6.2408745E-3,3.138649E6,1.1879E4,1.4299594E8,4.6E1,3.5E1,2.852738E6,2.25E2,2.3274304E2,1E0,6.203E3,2.0136518E0,-6.9739427E-3,-1.5887533E-3,-1.9633874E-2,-1.2257602E-3,-2.90397E-3,1.8736842E0,-3.6622456E-3,-1.377513E-2,1E0,-1.126579E-2,2E0,3.449702E2,2.9676E4,4.8632206E5,1.9071268E3,1.7247218E-1,3.5301748E7,1.8625112E-2,2.7165637E4,2.1871264E0,4.5E1,-5.349381E-3,4.514673E-3,3.2917362E7,-6.401259E-3,3.307766E6,1E0,4.274E3,-1.6822115E-3,9.126298E-3,5.5E1,7.122057E8,6.6315195E6,1.2866874E3,7.8E1,7.285168E-3,7.407709E-3,4.116997E6,2.9005406E5,1.460806E6,9.42812E3,-5.118009E-4,-7.766E-3,9.6615314E-1,2.7203197E0,9.22E2,-4.870825E-3,-2.2541031E-2,2.773036E1,4.4854636E7,2.0142922E-3,1.6527753E-2,8.829968E-3,8.451066E-4,3.2331508E-3,-2.734144E-3,-1.7703656E-3,4.4315876E-3,-3.2963266E-3,-1.3140261E-2,6.8237297E-3,-3.395028E-3,-9.83119E-4,-1.6205305E-2,-6.7488733E-3,-2.2422266E-3,3.0329938E-3,-2.8127108E-3,-4.574992E-3,4.2184885E-3,-9.919357E-4,-9.680875E-3,1.8741095E-3,-7.569317E-3,9.166963E-3,4.265126E-3,-1.977096E-6,-1.23765115E-2,7.180333E-3,1.5543675E-3,-8.053363E-3,1.3830452E-3,3.8049025E-3,-7.5318565E-4,-1.8431812E-3,3.416862E-3],"split_indices":[45,52,52,17,53,10,11,6,9,51,0,38,29,2,45,3,0,1,10,56,102,29,39,0,0,0,0,0,53,0,0,16,0,8,52,29,28,48,41,45,0,33,42,3,0,57,1,0,1,74,29,0,0,0,31,28,48,8,0,0,12,33,47,47,0,0,27,53,10,0,0,58,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.665E3,2.49E3,1.75E2,1.516E3,9.74E2,5.8E1,1.17E2,4.86E2,1.03E3,7.35E2,2.39E2,4.9E1,9E0,1.01E2,1.6E1,4.32E2,5.4E1,9.08E2,1.22E2,2.27E2,5.08E2,2.6E1,2.13E2,3.5E1,1.4E1,6E0,3E0,5.8E1,4.3E1,1.1E1,5E0,4.27E2,5E0,8E0,4.6E1,8.5E2,5.8E1,3.6E1,8.6E1,2.21E2,6E0,1.85E2,3.23E2,4E0,2.2E1,2.05E2,8E0,3E0,4E1,2.23E2,2.04E2,3E0,5E0,3E1,1.6E1,4.25E2,4.25E2,4.9E1,9E0,2.5E1,1.1E1,2.2E1,6.4E1,1.77E2,4.4E1,9E0,1.76E2,1.89E2,1.34E2,2E0,2E0,6.4E1,1.41E2,4E0,4E0,4E0,3.6E1,2.04E2,1.9E1,1.48E2,5.6E1,1.6E1,1.4E1,6E0,1E1,4.22E2,3E0,8E1,3.45E2,3.2E1,1.7E1,7E0,4E0,1.4E1,8E0,6E1,4E0,3.7E1,1.4E2,1.74E2,2E0,8.2E1,1.07E2,1.3E1,1.21E2,4.4E1,2E1,1.33E2,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-9.0413727E-4,9.985263E-3,-3.600035E-2,6.0698995E-3,1.0957025E-1,-9.310574E-2,-1.819122E-2,3.3371456E-3,1.2788582E-1,2.2228993E-1,5.7443533E-2,-2.0946832E-1,-7.25392E-2,-8.529567E-2,-1.0761851E-2,-3.0970782E-2,1.2080123E-2,1.8869624E-3,7.5222133E-3,2.4988648E-1,1.0457162E-3,7.007239E-2,-8.649244E-3,-5.283069E-3,-3.1742015E-1,-1.3371855E-1,-3.929674E-2,3.8473662E-3,-1.2043056E-1,-6.0931145E-4,-5.6633342E-2,-3.0745827E-3,-7.55842E-2,1.1960138E-2,1.0352274E-2,8.329059E-3,1.967437E-2,1.2646501E-3,5.563665E-3,-1.7710485E-2,-3.4434453E-3,-6.491688E-2,-2.669941E-1,-8.699467E-3,-3.6539037E-3,-1.1770845E-2,-3.8988045E-3,-3.8455743E-2,1.281776E-2,-8.789539E-2,6.900316E-4,-1.3060038E-1,1.1631498E-2,-5.8017585E-2,-4.2615685E-1,1.23154465E-2,-1.4538646E-1,2.7375005E-3,-1.0986425E-1,-1.6228132E-2,-2.6575814E-3,-1.5061362E-1,1.5367574E-2,-5.4245394E-2,2.6648925E-3,5.7670258E-2,-4.5358995E-3,-1.4851986E-1,-1.3610998E-3,-3.2045601E-3,-1.050901E-2,2.1047816E-3,-1.4527445E-3,-3.423002E-3,3.45073E-3,5.445721E-4,-2.8899176E-2,1.994154E-3,2.2777833E-5,7.1992464E-3,-1.0067941E-2,-6.67738E-3,3.1335156E-3,-2.4775513E-3,-1.4279475E-2,2.040965E-3,-3.8841204E-3,-1.214641E-3,-1.12896E-2,-7.7077965E-3,3.3127998E-3,-6.951502E-3,7.515945E-5,-3.9557123E-4,-9.443911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,-1,39,41,43,-1,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,57,59,-1,61,-1,-1,63,65,67,-1,69,71,73,75,77,79,-1,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0200069E0,7.9358464E-1,6.4104843E-1,6.5237623E-1,4.396602E-1,3.437816E-1,2.398787E-1,5.759847E-1,1.0485554E-1,1.2730801E-1,1.7499414E-1,2.0046556E-1,2.5709927E-1,2.890096E-1,2.03016E-1,4.8514017E-1,6.256915E-1,0E0,0E0,1.8529117E-1,0E0,1.01611674E-1,0E0,0E0,1.0204041E-1,3.9529872E-1,4.3062514E-1,0E0,1.7564493E-1,1.824166E-1,1.7671868E-1,4.5349163E-1,8.990316E-1,0E0,4.665181E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7638035E-1,2.091881E-1,0E0,1.9519654E-1,0E0,0E0,1.426861E-1,2.0652564E-1,1.9408685E-1,0E0,1.2675977E-1,2.9564545E-1,2.5722283E-1,5.498333E-1,5.21468E-1,3.9592013E-1,0E0,1.2832597E-1,0E0,0E0,1.0129687E-1,1.6644372E-1,4.1861936E-1,0E0,1.9724318E-1,1.6506281E-1,1.8271565E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,21,21,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,41,41,42,42,44,44,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,65,65,66,66,67,67],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,-1,40,42,44,-1,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,58,60,-1,62,-1,-1,64,66,68,-1,70,72,74,76,78,80,-1,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.759343E-1,1.5137865E2,1.9121015E0,7.780377E6,9.640999E6,8.230099E4,5.2143492E-5,1.5288235E2,1E0,2.558106E10,1.1096592E8,7.5408E4,1.05E2,1E0,8.229907E7,4.1E0,7.330957E4,1.8869624E-3,7.5222133E-3,2.9799202E3,1.0457162E-3,6.5317163E3,-8.649244E-3,-5.283069E-3,9.334426E-1,6.6911E4,1.2068E4,3.8473662E-3,5.987692E2,3.891716E8,1.4843444E1,8.333333E0,2.7056329E10,1.1960138E-2,4.2620764E2,8.329059E-3,1.967437E-2,1.2646501E-3,5.563665E-3,-1.7710485E-2,-3.4434453E-3,2.7772428E5,1.5153E4,-8.699467E-3,7E0,-1.1770845E-2,-3.8988045E-3,1.3741055E0,1.15062E5,4.3597875E0,6.900316E-4,6.7E1,1.1846624E-5,1.6713532E9,3.4893394E-4,1.5277338E0,2.2536E4,2.7375005E-3,2.220153E6,-1.6228132E-2,-2.6575814E-3,2.080593E0,2.109E3,8.7643677E-1,2.6648925E-3,2.5757682E-1,4.2647058E-1,2.5652175E0,-1.3610998E-3,-3.2045601E-3,-1.050901E-2,2.1047816E-3,-1.4527445E-3,-3.423002E-3,3.45073E-3,5.445721E-4,-2.8899176E-2,1.994154E-3,2.2777833E-5,7.1992464E-3,-1.0067941E-2,-6.67738E-3,3.1335156E-3,-2.4775513E-3,-1.4279475E-2,2.040965E-3,-3.8841204E-3,-1.214641E-3,-1.12896E-2,-7.7077965E-3,3.3127998E-3,-6.951502E-3,7.515945E-5,-3.9557123E-4,-9.443911E-3],"split_indices":[27,56,53,28,51,32,42,52,102,19,43,1,52,67,45,58,45,0,0,4,0,4,0,0,27,29,9,0,4,5,57,52,31,0,58,0,0,0,0,0,0,28,10,0,3,0,0,57,29,54,0,29,39,31,38,53,9,0,51,0,0,54,0,57,0,27,58,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.667E3,2.036E3,6.31E2,1.96E3,7.6E1,1.49E2,4.82E2,1.918E3,4.2E1,2.3E1,5.3E1,2.1E1,1.28E2,4.7E1,4.35E2,3.89E2,1.529E3,1.1E1,3.1E1,2E1,3E0,5.1E1,2E0,1.2E1,9E0,4.4E1,8.4E1,8E0,3.9E1,3.57E2,7.8E1,2.4E2,1.49E2,1E1,1.519E3,1.5E1,5E0,2.7E1,2.4E1,7E0,2E0,3E1,1.4E1,1.6E1,6.8E1,8E0,3.1E1,9.3E1,2.64E2,5.4E1,2.4E1,2.4E1,2.16E2,1.43E2,6E0,1.501E3,1.8E1,8E0,2.2E1,1E1,4E0,7E0,6.1E1,8E1,1.3E1,7.3E1,1.91E2,2.6E1,2.8E1,1.5E1,9E0,1.22E2,9.4E1,1.3E2,1.3E1,2E0,4E0,4.31E2,1.07E3,3E0,1.5E1,1.9E1,3E0,5E0,2E0,4.8E1,1.3E1,7E1,1E1,3E0,7E1,7E0,1.84E2,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-3.7045596E-4,-3.6905535E-2,1.074292E-2,-1.5716402E-2,-7.9719685E-2,2.3399413E-2,-3.266609E-2,-1.0545039E-2,-2.266878E-1,-4.3506805E-2,-1.4847243E-1,9.215458E-3,5.6995597E-2,-8.137936E-2,-7.570754E-3,-1.3643672E-3,-1.2303829E-1,-5.394868E-3,-1.7699482E-2,6.037914E-3,-5.7852626E-2,-1.8682328E-1,1.772061E-3,2.3320531E-2,-2.6835205E-2,1.7143126E-1,4.6604823E-2,-1.176125E-2,-7.098443E-2,-2.1522405E-2,6.3156635E-2,1.7024914E-2,-5.931575E-2,4.275471E-3,-1.7756474E-1,5.974491E-3,-6.793847E-2,-1.7615017E-2,-1.468878E-1,2.0309126E-2,1.8096879E-1,-1.9845191E-2,-1.5665765E-1,1.0837881E-2,5.353572E-4,1.255252E-1,3.413027E-2,1.7568696E-3,-8.2337804E-2,-1.4919146E-2,-2.025549E-1,-6.1760414E-3,3.7283131E-3,-6.618896E-2,4.0882777E-2,-1.5164492E-1,-3.3207543E-2,-4.1114238E-1,-8.21643E-2,-1.2700996E-2,-5.1625438E-2,3.3750876E-3,-7.538584E-3,1.3424848E-1,1.642536E-2,1.4490446E-2,3.1741408E-3,-2.5601132E-2,5.7904683E-3,-1.7578412E-2,-4.8073903E-3,9.058901E-2,2.5480953E-1,8.0738366E-2,2.2288065E-2,-6.885432E-2,-2.0461586E-1,9.880613E-3,-1.9355161E-2,-1.3959523E-2,-8.867553E-4,3.1658085E-3,-4.2445837E-3,5.332567E-3,9.187295E-4,4.246692E-4,-9.85301E-3,-5.718539E-4,-6.719911E-3,-2.8425375E-2,-6.0318247E-3,-9.624904E-3,-5.683564E-4,-7.2848407E-4,-5.8950796E-3,7.933593E-3,-3.8281565E-3,1.7544639E-4,2.3679947E-3,-5.91567E-3,-8.7543053E-4,2.8594022E-3,1.13757E-2,1.4902215E-2,3.078912E-3,6.3625356E-4,5.1708897E-3,-4.047322E-3,1.4837121E-3,-3.899793E-3,3.958989E-3,-1.4766823E-2,-3.4316233E-3,-1.1583843E-3,7.141098E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,47,49,51,53,55,-1,57,-1,59,-1,61,63,65,67,69,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,85,87,89,91,-1,93,-1,-1,95,97,-1,-1,99,-1,-1,-1,101,103,105,107,109,111,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0897697E0,5.6665105E-1,1.1314414E0,4.5583373E-1,5.089154E-1,7.590585E-1,5.6783533E-1,4.24298E-1,1.2132001E-1,3.4059092E-1,5.1194394E-1,5.711598E-1,5.54443E-1,2.522602E-1,3.0528256E-1,4.0703553E-1,3.7473115E-1,0E0,0E0,0E0,2.4237558E-1,3.7243128E-1,0E0,3.7944213E-1,2.8364867E-1,3.404243E-1,4.2397177E-1,0E0,1.8638241E-1,3.053924E-1,1.540129E-1,5.784445E-1,2.167348E-1,0E0,5.23304E-1,0E0,3.7569594E-1,0E0,1.2548101E-1,3.487083E-1,1.891492E-1,2.4696006E-1,1.5259412E-1,0E0,0E0,2.4340254E-1,2.0628282E-1,0E0,2.1013916E-1,2.4934453E-1,1.4190704E-1,0E0,0E0,1.9884023E-1,3.4342602E-1,1.7548373E-1,1.6461027E-1,2.8938866E-1,1.5630521E-1,0E0,2.9220864E-1,0E0,0E0,1.8817165E-1,3.234579E-1,0E0,0E0,2.056516E-1,0E0,0E0,0E0,2.036412E-1,1.0885292E-1,1.3818195E-1,2.830071E-1,2.4303865E-1,1.5720004E-1,0E0,2.0390177E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,20,20,21,21,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,60,60,63,63,64,64,67,67,71,71,72,72,73,73,74,74,75,75,76,76,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,48,50,52,54,56,-1,58,-1,60,-1,62,64,66,68,70,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,86,88,90,92,-1,94,-1,-1,96,98,-1,-1,100,-1,-1,-1,102,104,106,108,110,112,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8122449E2,2.419355E0,3.0161016E7,1E0,3.8609805E10,1.0122174E3,5.121E3,1.2471935E7,5.5E1,1E0,6.9666364E2,5.666993E0,9.1875E0,3E0,6.082581E3,6.4953816E-1,3.3820656E7,-5.394868E-3,-1.7699482E-2,6.037914E-3,1.279012E7,6.929741E8,1.772061E-3,8.848604E1,9.2882293E-1,3E0,1.4075E4,-1.176125E-2,8E0,2.6572757E10,1.1E1,3.79E2,4.5555557E1,4.275471E-3,1.2694E4,5.974491E-3,4.1E1,-1.7615017E-2,1.1216729E2,2.831341E6,1.0133419E8,6.65E2,1.2920592E0,1.0837881E-2,5.353572E-4,1E0,2.6979439E1,1.7568696E-3,1.8600049E10,3.75163E5,1.5190727E5,-6.1760414E-3,3.7283131E-3,1.901E3,3.9E2,1.43E2,8.893E3,6.4E1,3.054842E-4,-1.2700996E-2,2.6E1,3.3750876E-3,-7.538584E-3,2.9600988E5,1.8137958E6,1.4490446E-2,3.1741408E-3,4.6501E4,5.7904683E-3,-1.7578412E-2,-4.8073903E-3,5.185489E3,8.861076E0,8.5518695E4,3.3633875E6,2.653986E6,9.330579E0,9.880613E-3,4.5505118E2,-1.3959523E-2,-8.867553E-4,3.1658085E-3,-4.2445837E-3,5.332567E-3,9.187295E-4,4.246692E-4,-9.85301E-3,-5.718539E-4,-6.719911E-3,-2.8425375E-2,-6.0318247E-3,-9.624904E-3,-5.683564E-4,-7.2848407E-4,-5.8950796E-3,7.933593E-3,-3.8281565E-3,1.7544639E-4,2.3679947E-3,-5.91567E-3,-8.7543053E-4,2.8594022E-3,1.13757E-2,1.4902215E-2,3.078912E-3,6.3625356E-4,5.1708897E-3,-4.047322E-3,1.4837121E-3,-3.899793E-3,3.958989E-3,-1.4766823E-2,-3.4316233E-3,-1.1583843E-3,7.141098E-3],"split_indices":[52,56,45,14,31,52,2,45,0,53,4,53,56,3,52,41,7,0,0,0,7,5,0,56,38,11,9,0,3,12,3,1,52,0,29,0,0,0,4,5,7,8,56,0,0,66,58,0,5,1,33,0,0,9,12,0,10,11,42,0,8,0,0,47,28,0,0,1,0,0,0,52,53,28,51,1,56,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.682E3,6.25E2,2.057E3,4.19E2,2.06E2,1.593E3,4.64E2,4.1E2,9E0,1.36E2,7E1,1.121E3,4.72E2,1.57E2,3.07E2,3.8E2,3E1,6E0,3E0,1E1,1.26E2,5.8E1,1.2E1,8.06E2,3.15E2,3.8E1,4.34E2,8E0,1.49E2,2.57E2,5E1,2.89E2,9.1E1,6E0,2.4E1,6E0,1.2E2,9E0,4.9E1,7.92E2,1.4E1,3E2,1.5E1,2.8E1,1E1,5.8E1,3.76E2,1.4E1,1.35E2,2.49E2,8E0,3E0,4.7E1,6.4E1,2.25E2,1.9E1,7.2E1,6E0,1.8E1,8E0,1.12E2,2E0,4.7E1,2.5E1,7.67E2,6E0,8E0,2.89E2,1.1E1,2E0,1.3E1,4.7E1,1.1E1,7.5E1,3.01E2,1.23E2,1.2E1,4E0,2.45E2,5E0,3E0,9E0,5.5E1,5.2E1,1.73E2,5E0,1.4E1,6.1E1,1.1E1,3E0,3E0,6E0,1.2E1,7.5E1,3.7E1,2.2E1,3E0,5.54E2,2.13E2,1.9E1,2.7E2,4E1,7E0,8E0,3E0,2.2E1,5.3E1,2.2E1,2.79E2,1.14E2,9E0,6E0,6E0,2.39E2,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.2291354E-3,-2.242416E-2,1.8934418E-2,-6.969995E-3,-5.7640344E-2,3.6321364E-2,-2.4335792E-2,-1.803986E-2,3.544245E-2,-7.275407E-2,1.4736947E-3,9.81766E-3,3.1442832E-2,-4.1047584E-2,4.1761704E-2,-3.7999442E-3,-4.986822E-2,4.317684E-2,-9.759392E-3,-1.5901522E-1,-5.6529354E-2,5.4578822E-2,-3.432106E-2,1.8879361E-2,7.302315E-2,-3.4678027E-2,-1.9686581E-1,7.046844E-3,1.877496E-2,1.1248517E-1,-9.729822E-3,-9.59718E-2,-1.1353953E-2,-8.336403E-3,6.632146E-2,-7.21133E-2,-2.4653335E-1,-3.1189358E-2,-1.1075971E-1,9.943153E-2,-6.414922E-4,6.8529444E-3,-1.0561256E-1,3.3028506E-2,-1.8614754E-2,8.402982E-3,5.314058E-2,-1.9789933E-1,-2.8367296E-2,-3.6642575E-3,-1.676441E-2,7.677766E-2,-2.8761689E-2,-1.6908859E-3,2.0351447E-1,-6.51642E-3,-2.2134253E-1,-6.9907725E-2,-3.2355767E-1,1.1944948E-1,-2.55202E-2,2.044738E-2,-8.964773E-3,1.315656E-1,2.6665814E-2,-8.43176E-3,-3.220332E-4,-4.0098327E-1,-1.4424694E-1,-5.408174E-2,2.7289912E-3,3.7399686E-3,-1.465993E-1,6.2734396E-3,-2.8529726E-3,2.0822876E-3,-3.7390497E-3,-6.5015554E-3,4.327909E-3,6.357335E-2,2.7048127E-3,-3.7986785E-2,6.1834265E-2,1.00490585E-1,1.9638369E-2,-1.8085772E-2,-3.9886828E-3,-3.1510174E-2,9.780491E-3,1.7903276E-1,1.6698474E-2,-8.229478E-2,1.8705702E-3,1.1436124E-2,-8.244502E-4,-1.1340085E-3,1.1428441E-3,-1.7697668E-2,-8.5270416E-4,1.0553922E-2,-4.108393E-3,-7.501211E-3,-3.164794E-2,1.2169124E-2,9.7504724E-4,-4.0778797E-3,3.4163802E-4,-4.035697E-4,5.9185997E-3,3.991354E-3,9.493994E-3,1.9446971E-3,-5.8681034E-3,-2.436361E-3,-2.3645807E-2,2.5859496E-3,-9.297908E-3,3.9000422E-4,-4.469447E-3,1.0018104E-2,-3.5217085E-3,4.531245E-3,-7.866371E-3,1.6731343E-3,5.982061E-3,-2.042094E-2,3.6345847E-4,-1.4006087E-3,-9.649833E-3,3.7779266E-3,-5.128896E-3,6.5392E-3,5.4942863E-4,1.6160468E-3,-4.654913E-3,-5.656984E-3,-1.1433343E-3,1.0782969E-2,-9.1099844E-4,-4.375089E-3,3.3299716E-3,-8.74991E-3,-1.4343066E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,83,85,87,-1,-1,89,91,-1,93,95,97,99,101,103,105,107,-1,109,111,-1,-1,113,115,117,-1,119,121,-1,-1,-1,-1,-1,-1,123,125,127,129,131,133,-1,-1,135,-1,137,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1299555E0,7.011194E-1,1.0203004E0,4.2178094E-1,3.5122657E-1,7.893125E-1,4.3140656E-1,3.225125E-1,3.5295117E-1,4.2907405E-1,1.5584488E-1,0E0,4.901932E-1,3.0183828E-1,1.886081E-1,3.4104583E-1,3.8955235E-1,2.16351E-1,0E0,3.5489082E-1,3.6070776E-1,1.0194257E-1,1.4455622E-1,3.847038E-1,4.3468785E-1,3.0339986E-1,1.9445935E-1,0E0,1.8418702E-1,3.3006826E-1,3.1815618E-1,5.7056236E-1,2.2726062E-1,2.9727328E-1,3.199041E-1,1.7554253E-1,3.2131433E-1,3.7044823E-1,3.452524E-1,1.081796E-1,0E0,1.0293615E-1,1.1861509E-1,4.8702633E-1,3.124826E-1,0E0,2.8991127E-1,1.9516134E-1,2.1842872E-1,0E0,0E0,1.8131255E-1,1.3754904E-1,0E0,1.20930076E-1,2.4249755E-1,2.0457199E-1,4.3837348E-1,4.59499E-1,1.5493946E-1,2.1479365E-1,1.4994422E-1,0E0,1.346001E-1,1.692963E-1,0E0,0E0,2.6717854E-1,1.6876867E-1,3.5485125E-1,0E0,3.48785E-1,2.881428E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5584238E-1,5.5868274E-1,2.2420923E-1,1.2079482E-1,2.423529E-1,1.829266E-1,0E0,0E0,1.8700176E-1,0E0,1.0477331E-1,1.2053237E-1,1.049052E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,46,46,47,47,48,48,51,51,52,52,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,67,67,68,68,69,69,71,71,72,72,79,79,80,80,81,81,82,82,83,83,84,84,87,87,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,84,86,88,-1,-1,90,92,-1,94,96,98,100,102,104,106,108,-1,110,112,-1,-1,114,116,118,-1,120,122,-1,-1,-1,-1,-1,-1,124,126,128,130,132,134,-1,-1,136,-1,138,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.273439E2,5.2187E4,3.1826714E7,4.264897E6,9.5822406E2,3.79E2,4.247868E3,3.38464E5,1E0,3.451532E8,1.0931163E9,9.81766E-3,1.8839418E3,1.5874315E10,9.042859E4,7E0,1.131645E6,6.9432104E7,-9.759392E-3,1.0407268E10,1.7291142E7,2.4013288E1,2.3012722E1,1.2328733E1,1.4075E4,3E0,2.8216E4,7.046844E-3,6.01324E11,1.8255814E0,6.8E1,9.195004E6,5.6656016E3,6.763314E7,9.538462E0,8.457023E3,1.3797468E0,4.95E9,2.1358025E0,2.342147E7,-6.414922E-4,1.4015416E6,3.3546498E-1,5.68197E6,4.33E2,8.402982E-3,5.9948825E6,2.4991455E9,4.5505118E2,-3.6642575E-3,-1.676441E-2,4.7777777E0,1E0,-1.6908859E-3,1.85221E5,8.01E2,8.15E2,4.814389E4,4.6501E4,1.296E3,5.033165E7,1.57383175E1,-8.964773E-3,1.1411955E-1,3.7307863E0,-8.43176E-3,-3.220332E-4,8.646E4,4.01E2,1.0508E5,2.7289912E-3,6.5029144E5,1.941926E2,6.2734396E-3,-2.8529726E-3,2.0822876E-3,-3.7390497E-3,-6.5015554E-3,4.327909E-3,8.027061E2,1.22E2,3.5085775E5,7.494145E-2,2.3977574E7,2.3821254E7,-1.8085772E-2,-3.9886828E-3,3.5315048E2,9.780491E-3,1.9013363E4,3.368326E11,1.4E1,1.8705702E-3,1.1436124E-2,-8.244502E-4,-1.1340085E-3,1.1428441E-3,-1.7697668E-2,-8.5270416E-4,1.0553922E-2,-4.108393E-3,-7.501211E-3,-3.164794E-2,1.2169124E-2,9.7504724E-4,-4.0778797E-3,3.4163802E-4,-4.035697E-4,5.9185997E-3,3.991354E-3,9.493994E-3,1.9446971E-3,-5.8681034E-3,-2.436361E-3,-2.3645807E-2,2.5859496E-3,-9.297908E-3,3.9000422E-4,-4.469447E-3,1.0018104E-2,-3.5217085E-3,4.531245E-3,-7.866371E-3,1.6731343E-3,5.982061E-3,-2.042094E-2,3.6345847E-4,-1.4006087E-3,-9.649833E-3,3.7779266E-3,-5.128896E-3,6.5392E-3,5.4942863E-4,1.6160468E-3,-4.654913E-3,-5.656984E-3,-1.1433343E-3,1.0782969E-2,-9.1099844E-4,-4.375089E-3,3.3299716E-3,-8.74991E-3,-1.4343066E-3],"split_indices":[52,29,45,9,55,29,52,9,65,5,5,0,52,12,28,3,9,7,0,31,45,58,56,53,9,3,9,0,31,54,3,45,47,7,58,48,53,5,56,45,0,32,42,50,8,0,48,12,56,0,0,54,102,0,9,2,2,45,1,29,7,58,0,38,53,0,0,1,0,29,0,28,33,0,0,0,0,0,0,52,10,28,57,47,9,0,0,33,0,52,31,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.642E3,1.288E3,1.354E3,8.96E2,3.92E2,9.66E2,3.88E2,7.11E2,1.85E2,3.12E2,8E1,2.6E1,9.4E2,3.1E2,7.8E1,4.92E2,2.19E2,1.8E2,5E0,4.8E1,2.64E2,3.2E1,4.8E1,7.23E2,2.17E2,2.99E2,1.1E1,1.3E1,6.5E1,2.3E1,4.69E2,9.9E1,1.2E2,5.6E1,1.24E2,2.5E1,2.3E1,1.81E2,8.3E1,1.9E1,1.3E1,3.1E1,1.7E1,5.25E2,1.98E2,3.4E1,1.83E2,1E1,2.89E2,7E0,4E0,2.9E1,3.6E1,9E0,1.4E1,4.63E2,6E0,9E1,9E0,1.1E1,1.09E2,4.9E1,7E0,4.6E1,7.8E1,9E0,1.6E1,8E0,1.5E1,1.44E2,3.7E1,2E1,6.3E1,1.6E1,3E0,2.2E1,9E0,1.5E1,2E0,2.61E2,2.64E2,1.6E2,3.8E1,7.5E1,1.08E2,3E0,7E0,2.86E2,3E0,1E1,1.9E1,2E1,1.6E1,1.2E1,2E0,2.96E2,1.67E2,3E0,3E0,4E0,8.6E1,7E0,2E0,4E0,7E0,3.8E1,7.1E1,3.9E1,1E1,2.8E1,1.8E1,7.2E1,6E0,2E0,6E0,3E0,1.2E1,5.6E1,8.8E1,5E0,1.5E1,4E0,5.9E1,1.79E2,8.2E1,2E0,2.62E2,1.53E2,7E0,3.5E1,3E0,5.3E1,2.2E1,9.7E1,1.1E1,2.2E1,2.64E2,8E0,2E0,6E0,1.3E1,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[-5.281632E-4,-1.1202844E-2,3.1433403E-2,2.2004566E-3,-4.7597833E-2,4.3088675E-2,-3.6814332E-2,-1.4853295E-2,2.5956249E-2,-5.2261353E-2,1.2035309E-1,3.283489E-2,1.0124883E-1,7.6310046E-2,-6.289518E-2,-4.3192427E-3,-5.7961255E-2,1.5090963E-1,1.8195909E-2,-2.1320546E-1,-4.8756264E-2,1.0116357E-2,-2.4511605E-3,7.7917054E-2,1.4502976E-2,1.70254E-1,4.566831E-2,5.8881886E-4,7.7962787E-3,-1.4975409E-1,-2.7968906E-2,9.269684E-2,-1.2170515E-2,-1.07313454E-1,1.3369906E-2,8.464524E-3,-2.137516E-3,2.159101E-2,-1.21332906E-1,-2.1642644E-2,-1.2237837E-2,-6.690859E-2,-1.3201713E-2,6.819413E-2,1.3741599E-2,9.904998E-2,6.002954E-3,6.4217607E-3,1.4905538E-2,-1.3023481E-2,7.229387E-2,-8.179164E-3,2.2703132E-3,-4.8799335E-4,-1.0437534E-2,1.342202E-2,6.677975E-2,-6.3569043E-3,-8.479879E-3,-1.9267292E-1,-4.4730213E-2,-3.9610814E-2,8.130084E-2,6.0581405E-2,5.691981E-3,-2.3208467E-2,-2.0401338E-2,7.645307E-3,-8.670941E-3,-3.5171766E-2,-1.0072776E-1,-6.903289E-2,3.2509472E-2,8.1979826E-2,-3.393903E-2,-1.7469164E-3,1.5366836E-1,8.539461E-3,-1.2684656E-2,-6.8481714E-3,1.0203649E-1,2.8374788E-4,7.1183303E-3,3.3448618E-5,-3.0145016E-3,-5.9549934E-3,-1.6068693E-2,-9.860446E-3,-7.3809933E-4,1.4741772E-3,-5.700591E-3,9.496523E-3,2.1689641E-3,2.340815E-3,9.852083E-3,-1.3587912E-4,4.231888E-3,-6.312675E-3,3.7359537E-3,-4.0171384E-3,-7.715464E-5,-2.4696998E-3,-6.9332467E-3,1.641281E-3,-4.982478E-3,-8.220929E-3,3.1490452E-3,-8.804818E-3,4.214012E-3,-8.184943E-3,2.6966832E-3,4.425255E-3,1.4934732E-2,-2.6951492E-4,2.7730812E-3,3.6451896E-3,1.1520508E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,51,53,55,57,59,61,-1,-1,63,65,-1,67,69,71,73,-1,75,77,-1,-1,-1,79,-1,-1,-1,-1,-1,81,-1,83,85,87,89,91,93,95,-1,97,-1,-1,99,101,103,105,107,109,-1,111,113,-1,-1,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.1983086E-1,9.8597467E-1,5.385123E-1,5.9916085E-1,4.3237388E-1,3.407762E-1,2.968872E-1,3.9073104E-1,5.965167E-1,2.885903E-1,2.446648E-1,4.0547955E-1,3.220644E-1,1.03187144E-1,2.4153784E-1,5.289861E-1,5.9637386E-1,1.897744E-1,2.7879518E-1,5.2218014E-1,3.347236E-1,0E0,0E0,2.69018E-1,2.5165206E-1,1.5200162E-1,4.3403172E-1,0E0,0E0,1.0348195E-1,1.9705895E-1,2.3406693E-1,2.8454608E-1,5.25589E-1,2.5515246E-1,0E0,0E0,3.5300353E-1,5.0344586E-1,0E0,2.3202029E-1,3.6567688E-1,4.5396763E-1,1.9567513E-1,0E0,2.4247831E-1,2.2212264E-1,0E0,0E0,0E0,3.091424E-1,0E0,0E0,0E0,0E0,0E0,2.33247E-1,0E0,3.1198424E-1,3.6823893E-1,2.7560422E-1,2.2729705E-1,1.2162542E-1,2.7020717E-1,2.865406E-1,0E0,1.4185527E-1,0E0,0E0,2.924831E-1,3.519523E-1,2.928674E-1,6.7648816E-1,2.095828E-1,2.1976674E-1,0E0,1.9407809E-1,2.227239E-1,0E0,0E0,1.2890899E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,40,40,41,41,42,42,43,43,45,45,46,46,50,50,56,56,58,58,59,59,60,60,61,61,62,62,63,63,64,64,66,66,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,52,54,56,58,60,62,-1,-1,64,66,-1,68,70,72,74,-1,76,78,-1,-1,-1,80,-1,-1,-1,-1,-1,82,-1,84,86,88,90,92,94,96,-1,98,-1,-1,100,102,104,106,108,110,-1,112,114,-1,-1,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,1.2471935E7,7.218466E7,3.0291306E2,1.3238013E2,6.082581E3,8.24666E5,5.2187E4,3.79E2,1.2673605E7,1.9768514E3,2.26182E5,3.3517068E7,1E0,3.0387878E3,7E0,6.5932993E10,4.2E1,1.9162654E7,7E2,1.022332E6,1.0116357E-2,-2.4511605E-3,1.95684E5,1E0,5.0738932E7,5.388794E6,5.8881886E-4,7.7962787E-3,5.377837E6,9.22E4,1.3050649E4,8.333333E0,3.6111112E0,1.6708623E2,8.464524E-3,-2.137516E-3,3.28825E0,3.171E3,-2.1642644E-2,2.3279E4,1.2886906E5,5.065188E-4,1.3900395E2,1.3741599E-2,9.536863E4,1.3056483E8,6.4217607E-3,1.4905538E-2,-1.3023481E-2,2.7258065E0,-8.179164E-3,2.2703132E-3,-4.8799335E-4,-1.0437534E-2,1.342202E-2,1E0,-6.3569043E-3,2.4E1,3.3162162E0,3E0,2.1818182E0,1.84442E5,7.562698E1,4.41E4,-2.3208467E-2,5.11E2,7.645307E-3,-8.670941E-3,1.159E3,2.1270142E5,1.7560976E0,4E0,7.549744E7,4.4200982E8,-1.7469164E-3,4.3525005E3,4.551684E6,-1.2684656E-2,-6.8481714E-3,5.68E2,2.8374788E-4,7.1183303E-3,3.3448618E-5,-3.0145016E-3,-5.9549934E-3,-1.6068693E-2,-9.860446E-3,-7.3809933E-4,1.4741772E-3,-5.700591E-3,9.496523E-3,2.1689641E-3,2.340815E-3,9.852083E-3,-1.3587912E-4,4.231888E-3,-6.312675E-3,3.7359537E-3,-4.0171384E-3,-7.715464E-5,-2.4696998E-3,-6.9332467E-3,1.641281E-3,-4.982478E-3,-8.220929E-3,3.1490452E-3,-8.804818E-3,4.214012E-3,-8.184943E-3,2.6966832E-3,4.425255E-3,1.4934732E-2,-2.6951492E-4,2.7730812E-3,3.6451896E-3,1.1520508E-2],"split_indices":[2,45,45,52,56,52,29,29,29,45,4,29,51,59,4,3,31,0,47,2,28,0,0,29,67,1,47,0,0,29,10,28,52,54,52,0,0,54,2,0,29,47,38,58,0,28,44,0,0,0,54,0,0,0,0,0,16,0,8,56,11,56,29,58,10,0,0,0,0,11,28,53,3,7,7,0,4,29,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.694E3,2.02E3,6.74E2,1.477E3,5.43E2,5.76E2,9.8E1,8.6E2,6.17E2,5.29E2,1.4E1,4.91E2,8.5E1,1.8E1,8E1,6.92E2,1.68E2,3.5E1,5.82E2,1E1,5.19E2,9E0,5E0,1.41E2,3.5E2,3.7E1,4.8E1,1.1E1,7E0,2.2E1,5.8E1,5.1E1,6.41E2,9.9E1,6.9E1,3.1E1,4E0,5.69E2,1.3E1,4E0,6E0,3.43E2,1.76E2,1.36E2,5E0,3.1E1,3.19E2,3.1E1,6E0,3E0,4.5E1,2E1,2E0,5.4E1,4E0,5E0,4.6E1,1.8E1,6.23E2,4.1E1,5.8E1,3.9E1,3E1,1.64E2,4.05E2,2E0,1.1E1,3E0,3E0,1.78E2,1.65E2,7.9E1,9.7E1,1.2E2,1.6E1,9E0,2.2E1,3.17E2,2E0,5E0,4E1,2.7E1,1.9E1,5.34E2,8.9E1,2.9E1,1.2E1,8E0,5E1,2.1E1,1.8E1,6E0,2.4E1,1.53E2,1.1E1,3.68E2,3.7E1,5E0,6E0,7.2E1,1.06E2,7.9E1,8.6E1,2E1,5.9E1,1.3E1,8.4E1,2E0,1.18E2,6E0,1E1,1.7E1,5E0,2.47E2,7E1,3.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-1.0142139E-3,-1.4178801E-2,2.1765398E-2,-1.8516176E-3,-5.9930407E-2,5.7788864E-2,3.1822545E-3,-1.82586E-2,2.210297E-2,-2.9770514E-1,-5.071695E-2,3.5540257E-2,1.0340945E-1,-1.9315339E-3,1.1046208E-1,-1.4197727E-3,-4.2384584E-2,5.658927E-2,-4.8949798E-3,-3.6004197E-2,-1.4679648E-1,-5.955818E-2,9.460475E-2,2.8082239E-2,1.0005599E-2,1.2385368E-1,-1.4481518E-3,2.7740391E-2,-1.866459E-2,1.5376389E-2,6.392128E-2,2.7887279E-3,-1.5912023E-1,-1.9818562E-1,-3.2475E-2,1.4442478E-1,1.1182662E-2,-1.01579785E-1,8.208584E-3,1.697075E-3,-1.051501E-2,-5.352984E-2,-2.639325E-1,6.961128E-3,-9.774782E-4,-1.031019E-2,3.2562774E-2,1.3937709E-1,-9.0648764E-4,7.696125E-2,-9.35537E-4,-4.1285637E-3,-7.7698976E-2,2.7140042E-2,9.847168E-3,-6.0941588E-2,1.4888449E-2,-1.4536853E-2,-6.033029E-4,5.992135E-4,-3.771519E-1,-4.377845E-2,6.133441E-2,9.820745E-2,1.819093E-1,2.0467034E-2,-2.246749E-2,3.5180133E-2,-1.7142558E-1,2.8312683E-2,-6.768737E-2,-9.617554E-3,-7.639413E-2,-1.8933129E-2,-3.8186358E-3,2.9477695E-2,1.1332613E-2,1.1275154E-2,1.1508262E-1,2.3307134E-3,7.7019357E-3,-7.882398E-3,1.5273821E-2,-2.6288193E-2,3.8986683E-2,-1.2880665E-1,-1.5915046E-2,5.9145787E-3,-1.4301324E-3,-3.7210542E-3,5.0651054E-3,3.0504225E-3,-5.1956775E-4,-1.0325392E-2,3.9078924E-3,-2.2271471E-2,-8.1609824E-4,1.7307271E-4,-3.3029001E-3,-1.2487058E-3,5.3888247E-3,6.050218E-3,-3.094054E-4,6.802989E-4,9.518027E-3,-2.6355565E-3,2.5930281E-3,5.8529396E-3,-2.7240338E-3,-6.950938E-4,-1.0675903E-2,-1.0815797E-3,2.54794E-3,-4.0288246E-3,2.8446435E-3,2.5977574E-3,-2.9097442E-3,-8.524616E-3,-3.1500182E-3,5.4907694E-4,3.0112644E-3,-4.9246913E-3,6.2293517E-3,-8.5760606E-4,6.5994277E-3,8.065654E-3,-1.4813286E-3,2.1677134E-3,-9.27698E-3,-4.557007E-3,-1.2448348E-2,-2.42437E-3,3.235921E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,-1,47,-1,49,51,-1,53,55,57,59,61,63,65,67,69,-1,-1,71,73,-1,-1,-1,75,77,-1,79,81,83,85,87,-1,89,91,-1,-1,93,95,97,99,101,103,105,-1,107,109,111,113,115,117,-1,-1,119,-1,-1,121,-1,-1,123,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8930396E-1,9.4062173E-1,6.4514595E-1,5.172282E-1,7.604189E-1,3.2924294E-1,3.4967557E-1,3.1722364E-1,4.9839377E-1,7.804909E-1,4.4504368E-1,2.8120485E-1,2.9610515E-1,3.0288747E-1,2.6373547E-1,3.0640134E-1,4.8916346E-1,9.356385E-1,3.821754E-1,0E0,1.5364654E-1,3.8381994E-1,1.1970799E-1,2.4036077E-1,0E0,2.0876932E-1,0E0,3.105918E-1,3.343481E-1,0E0,1.2471428E-1,3.479109E-1,2.5011125E-1,6.761659E-1,3.2397726E-1,1.2072992E-1,7.052867E-1,3.4990656E-1,4.077845E-1,0E0,0E0,3.155328E-1,1.8632191E-1,0E0,0E0,0E0,1.2586622E-1,1.6893625E-1,0E0,1.8436855E-1,3.136044E-1,3.010261E-1,2.4108773E-1,1.19038574E-1,0E0,2.0935574E-1,4.767825E-1,0E0,0E0,1.9272085E-1,3.0860424E-1,3.2376313E-1,1.5058888E-1,1.1268765E-1,1.1858702E-1,3.9455205E-1,0E0,1.11113794E-1,1.906972E-1,2.685103E-1,1.219857E-1,3.592794E-1,2.1163464E-1,0E0,0E0,1.2501813E-1,0E0,0E0,2.3880023E-1,0E0,0E0,2.0401992E-1,0E0,1.9326086E-1,1.650637E-1,1.605987E-1,1.0773334E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,46,46,47,47,49,49,50,50,51,51,52,52,53,53,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71,72,72,75,75,78,78,81,81,83,83,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,-1,48,-1,50,52,-1,54,56,58,60,62,64,66,68,70,-1,-1,72,74,-1,-1,-1,76,78,-1,80,82,84,86,88,-1,90,92,-1,-1,94,96,98,100,102,104,106,-1,108,110,112,114,116,118,-1,-1,120,-1,-1,122,-1,-1,124,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.423E3,1.3905958E7,8E-3,2.7604443E2,6E0,1.5106794E3,4.5505118E2,2.294901E0,8E0,1.6856086E8,1E0,4.0602388E6,8.657441E7,1.3446785E7,1.6907827E9,6.2233735E-6,5.8601086E1,1E0,1.074E3,-3.6004197E-2,3.01E2,4.387561E1,3.476768E-3,2.8672566E0,1.0005599E-2,4.728721E0,-1.4481518E-3,4.0208E4,1.8E2,1.5376389E-2,2.62797E5,4.7E2,1.4735735E2,7.8079817E-3,1.7022566E7,3.7810526E2,1.6904226E7,1.7143776E7,1.4383178E1,1.697075E-3,-1.051501E-2,1.36E2,6.203E3,6.961128E-3,-9.774782E-4,-1.031019E-2,2.03148E2,4.899767E6,-9.0648764E-4,2.3276744E5,1.921032E4,3.4409692E3,3.555E3,3.2475834E7,9.847168E-3,3.1530054E0,2.3067484E0,-1.4536853E-2,-6.033029E-4,3.0404909E0,4.38E2,2.6195612E-2,2.691875E2,9.38E2,5.9E1,1E0,-2.246749E-2,4.930435E2,1.3E1,1.954E3,3.474851E8,1E0,1.48298E5,-1.8933129E-2,-3.8186358E-3,7.019E3,1.1332613E-2,1.1275154E-2,5.794E3,2.3307134E-3,7.7019357E-3,4.091015E9,1.5273821E-2,5.473125E3,4.4543375E4,1.8215276E7,6.5900993E-1,5.9145787E-3,-1.4301324E-3,-3.7210542E-3,5.0651054E-3,3.0504225E-3,-5.1956775E-4,-1.0325392E-2,3.9078924E-3,-2.2271471E-2,-8.1609824E-4,1.7307271E-4,-3.3029001E-3,-1.2487058E-3,5.3888247E-3,6.050218E-3,-3.094054E-4,6.802989E-4,9.518027E-3,-2.6355565E-3,2.5930281E-3,5.8529396E-3,-2.7240338E-3,-6.950938E-4,-1.0675903E-2,-1.0815797E-3,2.54794E-3,-4.0288246E-3,2.8446435E-3,2.5977574E-3,-2.9097442E-3,-8.524616E-3,-3.1500182E-3,5.4907694E-4,3.0112644E-3,-4.9246913E-3,6.2293517E-3,-8.5760606E-4,6.5994277E-3,8.065654E-3,-1.4813286E-3,2.1677134E-3,-9.27698E-3,-4.557007E-3,-1.2448348E-2,-2.42437E-3,3.235921E-3],"split_indices":[2,45,57,52,3,52,56,56,17,5,74,48,45,45,12,37,50,16,2,0,0,53,38,56,0,38,0,9,8,0,2,1,52,58,9,52,47,5,53,0,0,11,29,0,0,0,56,45,0,33,4,52,0,50,0,53,58,0,0,56,0,42,55,2,29,100,0,4,3,2,7,14,1,0,0,2,0,0,2,0,0,7,0,32,4,1,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.63E3,1.667E3,9.63E2,1.314E3,3.53E2,3.27E2,6.36E2,7.8E2,5.34E2,1.2E1,3.41E2,2.21E2,1.06E2,6.08E2,2.8E1,4.6E2,3.2E2,2.34E2,3E2,2E0,1E1,3.22E2,1.9E1,2.13E2,8E0,9.2E1,1.4E1,2.19E2,3.89E2,4E0,2.4E1,4.49E2,1.1E1,1.8E1,3.02E2,7.9E1,1.55E2,3.5E1,2.65E2,3E0,7E0,3.14E2,8E0,1.3E1,6E0,3E0,2.1E2,8.3E1,9E0,8E1,1.39E2,3.13E2,7.6E1,2E1,4E0,7.1E1,3.78E2,5E0,6E0,9E0,9E0,2.7E2,3.2E1,3.7E1,4.2E1,1.53E2,2E0,1.2E1,2.3E1,2.1E2,5.5E1,1.08E2,2.06E2,4E0,4E0,2.08E2,2E0,1.5E1,6.8E1,6.1E1,1.9E1,1.37E2,2E0,2.07E2,1.06E2,4.1E1,3.5E1,7E0,1.3E1,6.5E1,6E0,1.3E2,2.48E2,2E0,7E0,7E0,2E0,9.4E1,1.76E2,1.2E1,2E1,2.9E1,8E0,4E0,3.8E1,4.7E1,1.06E2,6E0,6E0,6E0,1.7E1,6.9E1,1.41E2,4.9E1,6E0,4.8E1,6E1,1.8E1,1.88E2,1.36E2,7.2E1,4E0,6.4E1,1.29E2,8E0,4E0,2.03E2,1.04E2,2E0,3.4E1,7E0,2.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-1.9088521E-3,-7.848197E-3,3.692166E-2,-3.4132984E-3,-7.680386E-2,5.2463256E-2,-6.260142E-2,-3.2322776E-2,6.083315E-3,-6.2375523E-2,-1.7667925E-1,2.3234911E-1,4.494253E-2,4.9292248E-2,-1.2555254E-1,5.535428E-3,-5.661081E-2,4.2398136E-2,-4.54813E-3,-8.277883E-3,-2.547168E-3,-1.41818775E-2,-2.5284023E-4,1.3345404E-2,2.461529E-5,3.2221846E-2,9.72685E-2,-4.3512685E-3,6.079188E-3,-1.7648704E-1,-1.4462447E-3,-5.429388E-2,3.5198662E-2,-4.8784E-2,-1.7392181E-1,1.1567187E-1,1.28170205E-2,-1.7096302E-2,1.8499969E-2,1.15579575E-1,2.010245E-2,7.051919E-2,1.4984204E-2,-1.20289065E-2,-4.4320114E-3,-2.8239004E-2,-2.0779923E-1,-1.4610586E-2,9.366194E-2,-8.1413776E-2,-9.791397E-3,-2.8870687E-1,-2.5746275E-3,1.4669019E-1,3.9357938E-2,2.3352988E-2,-1.6363691E-1,-3.4045547E-3,-4.8496295E-2,1.446451E-1,1.1597278E-2,2.7891665E-3,8.911542E-3,2.5243545E-2,-6.3148984E-3,1.3379712E-2,3.7202824E-2,-4.8330007E-3,3.3725626E-4,-1.8986696E-2,-3.3670547E-3,2.8038216E-3,-4.369222E-3,1.216976E-2,3.4978057E-3,-3.3300105E-3,-1.351699E-2,4.499438E-4,-5.459559E-3,-6.5066223E-3,-1.9844709E-2,8.304198E-3,2.8179528E-3,-1.3988273E-3,4.864539E-3,-8.862896E-4,2.860393E-3,-2.3833254E-2,-1.8514056E-3,-7.4159104E-4,1.5696839E-3,-9.714607E-3,-1.9054809E-3,3.8441012E-3,1.34249E-2,1.4452289E-3,-1.0906295E-3,3.5853295E-3,5.6672667E-4,3.7972329E-3,-2.234857E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,65,-1,-1,-1,67,69,71,73,75,77,79,-1,81,83,85,87,89,91,93,95,-1,-1,97,-1,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1191475E-1,7.034098E-1,5.477392E-1,5.940312E-1,1.8928355E-1,4.0218377E-1,3.4429264E-1,4.9224657E-1,6.291155E-1,1.15659624E-1,3.4649473E-1,1.2870783E-1,1.9255877E-1,2.0741484E-1,1.4523068E-1,3.7460452E-1,2.908392E-1,7.9833186E-1,3.6501196E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.3811495E-1,3.0668765E-1,0E0,0E0,1.0214484E-1,0E0,2.7410135E-1,4.1196212E-1,3.8980502E-1,2.6033294E-1,2.4514234E-1,4.9462718E-1,3.5108012E-1,3.8580215E-1,1.1043805E-1,1.6661176E-1,3.5144922E-1,0E0,0E0,0E0,1.5712643E-1,2.3151964E-1,4.369655E-1,1.965431E-1,3.788203E-1,2.8299057E-1,1.357773E-1,0E0,1.6577554E-1,1.3948232E-1,3.8176298E-1,6.069449E-1,2.4964757E-1,3.252709E-1,1.7839438E-1,2.708249E-1,0E0,0E0,1.3419287E-1,0E0,0E0,1.6607031E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,25,25,26,26,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,63,63,66,66],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,66,-1,-1,-1,68,70,72,74,76,78,80,-1,82,84,86,88,90,92,94,96,-1,-1,98,-1,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,6.747114E7,2.3580047E10,1.5288235E2,2.25215E5,7.969174E7,5.393203E-1,2.8373447E3,1.4615384E0,3.5315048E2,3.138649E6,7.5E1,9.3058E4,1.6482393E-1,1.4580295E0,3.3151623E1,2.74872E5,3.3253515E6,6.7628815E2,-8.277883E-3,-2.547168E-3,-1.41818775E-2,-2.5284023E-4,1.3345404E-2,2.461529E-5,2.386324E6,4.516183E1,-4.3512685E-3,6.079188E-3,1.031322E6,-1.4462447E-3,6.737101E7,7.6593264E-5,8.31E2,6E0,3.0805944E7,2.3523688E0,2.6540773E1,1.296E3,9.994E3,4.5481584E7,1.3683E4,1.4984204E-2,-1.20289065E-2,-4.4320114E-3,8.333333E0,1.14E2,2.0042918E0,8E0,2.2214102E6,6.5260305E0,9.0067856E5,-2.5746275E-3,8E0,7.8586104E7,3.7382904E7,1.0449754E0,1.6908307E-1,1.2920592E0,5.5607376E0,1.1347826E1,2.7891665E-3,8.911542E-3,2.805891E4,-6.3148984E-3,1.3379712E-2,6.38041E5,-4.8330007E-3,3.3725626E-4,-1.8986696E-2,-3.3670547E-3,2.8038216E-3,-4.369222E-3,1.216976E-2,3.4978057E-3,-3.3300105E-3,-1.351699E-2,4.499438E-4,-5.459559E-3,-6.5066223E-3,-1.9844709E-2,8.304198E-3,2.8179528E-3,-1.3988273E-3,4.864539E-3,-8.862896E-4,2.860393E-3,-2.3833254E-2,-1.8514056E-3,-7.4159104E-4,1.5696839E-3,-9.714607E-3,-1.9054809E-3,3.8441012E-3,1.34249E-2,1.4452289E-3,-1.0906295E-3,3.5853295E-3,5.6672667E-4,3.7972329E-3,-2.234857E-3],"split_indices":[2,45,5,52,11,7,41,47,53,33,29,3,2,41,42,55,29,45,52,0,0,0,0,0,0,1,54,0,0,11,0,7,38,2,8,5,41,58,29,10,47,9,0,0,0,52,29,54,18,28,57,50,0,8,5,12,53,38,56,54,53,0,0,48,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.651E3,2.3E3,3.51E2,2.162E3,1.38E2,3.04E2,4.7E1,5.34E2,1.628E3,1.22E2,1.6E1,1.1E1,2.93E2,1.7E1,3E1,2.09E2,3.25E2,3.68E2,1.26E3,8E0,1.14E2,9E0,7E0,9E0,2E0,2.37E2,5.6E1,6E0,1.1E1,1.9E1,1.1E1,6.9E1,1.4E2,3.06E2,1.9E1,1.05E2,2.63E2,8.16E2,4.44E2,2.9E1,2.08E2,5.1E1,5E0,9E0,1E1,6E1,9E0,7.6E1,6.4E1,1.66E2,1.4E2,9E0,1E1,7.4E1,3.1E1,2.49E2,1.4E1,5.69E2,2.47E2,2.2E1,4.22E2,1.7E1,1.2E1,2.02E2,6E0,6E0,4.5E1,1.9E1,4.1E1,3E0,6E0,3.9E1,3.7E1,6E0,5.8E1,1.58E2,8E0,1.19E2,2.1E1,5E0,4E0,5.6E1,1.8E1,1.5E1,1.6E1,1.16E2,1.33E2,3E0,1.1E1,4.27E2,1.42E2,1.2E1,2.35E2,1.6E1,6E0,2.74E2,1.48E2,4.2E1,1.6E2,3E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-1.643766E-3,-1.0015222E-2,3.597123E-2,6.320102E-4,-4.0407643E-2,9.3945056E-2,2.0066757E-2,-1.7151E-2,2.501478E-2,-2.3165508E-1,-3.552816E-2,8.332106E-3,6.675863E-2,-8.546275E-4,7.443124E-2,-1.0311707E-2,-7.09938E-2,7.277038E-3,1.9791927E-2,-2.540778E-2,-1.01910606E-1,-3.9765988E-2,1.1024873E-1,1.1838934E-1,2.3970537E-2,5.670017E-3,-1.4650047E-1,1.6297033E-1,2.9127924E-2,-3.4932964E-2,5.0727045E-3,-1.6223269E-2,-4.5678854E-2,3.9606217E-2,-1.3185554E-3,5.082975E-3,-7.6594464E-3,-3.011627E-2,-8.428325E-2,9.550045E-3,-1.5746458E-3,2.0387109E-1,2.6208095E-3,-6.866172E-4,4.2100106E-3,-6.2549626E-4,1.6815682E-1,-1.1189128E-3,-1.1068337E-2,6.713864E-3,1.7566385E-2,-9.660369E-3,4.5673512E-2,-3.0318114E-3,-5.854067E-2,1.19891666E-1,-2.2615767E-3,-6.1536536E-2,6.7244126E-3,2.563644E-2,1.0876027E-1,1.3554903E-2,-5.4844167E-2,-2.5234088E-2,-1.3308915E-1,-6.647169E-2,-2.0342486E-1,1.3314665E-3,1.1760547E-2,-9.492422E-3,3.2511787E-3,-4.474717E-3,1.1744419E-2,-7.5467816E-3,6.788527E-2,-2.138244E-3,4.968909E-3,-1.2486031E-3,-4.7845966E-3,4.4085085E-3,1.3638431E-2,2.3808531E-4,-5.3814244E-3,3.1562378E-3,-3.8256682E-3,1.4190607E-3,-7.905045E-3,7.554166E-3,1.19842356E-4,6.589851E-3,-1.2285486E-4,-4.1635838E-4,-5.736866E-3,-3.8178337E-3,-6.788315E-4,-1.922376E-2,-1.9597139E-3,-4.263264E-3,-5.6885293E-4,-1.3603877E-2,1.3823479E-3,8.19591E-3,-3.7887126E-5,2.3132898E-3,9.291863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,33,-1,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,-1,-1,63,65,-1,-1,67,-1,-1,-1,69,71,-1,-1,-1,-1,-1,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,99,-1,-1,-1,101,-1,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.458461E-1,7.1096414E-1,4.4899398E-1,7.063283E-1,5.236871E-1,2.1685743E-1,4.3795496E-1,3.4615636E-1,4.5245412E-1,4.8883134E-1,3.48486E-1,0E0,1.7389649E-1,2.6596084E-1,4.244423E-1,3.1721902E-1,7.0303833E-1,0E0,2.7654144E-1,0E0,1.4976919E-1,2.3029351E-1,2.1075077E-1,1.8476015E-1,1.1148069E-1,2.742525E-1,1.1963341E-1,1.1572492E-1,2.8421548E-1,2.4232492E-1,4.3480226E-1,0E0,2.9783574E-1,3.266405E-1,2.5629503E-1,0E0,0E0,2.2136301E-1,1.9019943E-1,0E0,0E0,1.03304446E-1,0E0,0E0,0E0,1.9903173E-1,2.245442E-1,0E0,0E0,0E0,0E0,0E0,3.176977E-1,6.179244E-1,2.4587E-1,1.16081506E-1,3.876147E-1,2.1803933E-1,0E0,2.2201325E-1,2.962497E-1,5.0427073E-1,2.0877096E-1,2.567758E-1,4.781892E-1,1.0358152E-1,2.3297313E-1,0E0,0E0,0E0,1.7319117E-1,0E0,0E0,0E0,1.4376637E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,37,37,38,38,41,41,45,45,46,46,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,70,70,74,74],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,34,-1,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,-1,-1,64,66,-1,-1,68,-1,-1,-1,70,72,-1,-1,-1,-1,-1,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,100,-1,-1,-1,102,-1,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5106794E3,1.4540612E7,8.3306855E6,3.3030225E2,4E0,2.2376953E1,6.082581E3,9.837297E0,2.1761298E0,1E0,1.4122614E2,8.332106E-3,1.813943E6,2.6572757E10,3.3517068E7,1.1556145E2,2E0,7.277038E-3,2.773036E1,-2.540778E-2,2.56E2,6.971004E7,1.6826648E3,2.8E1,1.75E2,2.62797E5,3.1415796E0,1.3948507E7,1.5439131E2,3.2E1,7.692308E-1,-1.6223269E-2,1.2677199E7,8.027061E2,2.0399538E5,5.082975E-3,-7.6594464E-3,4.387561E1,2.55298E5,9.550045E-3,-1.5746458E-3,9.97958E5,2.6208095E-3,-6.866172E-4,4.2100106E-3,8.874407E6,2.142857E0,-1.1189128E-3,-1.1068337E-2,6.713864E-3,1.7566385E-2,-9.660369E-3,2.9104478E0,4.0701206E1,2.9138756E0,7.8955513E-1,1E0,7.29E2,6.7244126E-3,1E0,1.3195207E7,2.7487562E0,2.185321E0,2.2348747E2,2.90002E5,1.9701008E5,4.242625E6,1.3314665E-3,1.1760547E-2,-9.492422E-3,3.53562E5,-4.474717E-3,1.1744419E-2,-7.5467816E-3,4.42384E5,-2.138244E-3,4.968909E-3,-1.2486031E-3,-4.7845966E-3,4.4085085E-3,1.3638431E-2,2.3808531E-4,-5.3814244E-3,3.1562378E-3,-3.8256682E-3,1.4190607E-3,-7.905045E-3,7.554166E-3,1.19842356E-4,6.589851E-3,-1.2285486E-4,-4.1635838E-4,-5.736866E-3,-3.8178337E-3,-6.788315E-4,-1.922376E-2,-1.9597139E-3,-4.263264E-3,-5.6885293E-4,-1.3603877E-2,1.3823479E-3,8.19591E-3,-3.7887126E-5,2.3132898E-3,9.291863E-3],"split_indices":[52,45,45,52,3,56,52,54,58,8,56,0,9,12,51,52,17,0,58,0,0,45,4,3,8,2,54,9,58,10,53,0,9,52,33,0,0,53,11,0,0,1,0,0,0,45,54,0,0,0,0,0,54,52,58,38,74,29,0,85,9,54,41,52,1,33,29,0,0,0,1,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.684E3,2.196E3,4.88E2,1.627E3,5.69E2,1.04E2,3.84E2,9.41E2,6.86E2,1.3E1,5.56E2,2.5E1,7.9E1,2.78E2,1.06E2,8.36E2,1.05E2,2.6E1,6.6E2,3E0,1E1,5.41E2,1.5E1,3.5E1,4.4E1,2.67E2,1.1E1,3.5E1,7.1E1,3.21E2,5.15E2,8E0,9.7E1,3.4E2,3.2E2,2E0,8E0,4.46E2,9.5E1,9E0,6E0,1.4E1,2.1E1,2.8E1,1.6E1,2.58E2,9E0,5E0,6E0,3.3E1,2E0,4E0,6.7E1,1.37E2,1.84E2,3E1,4.85E2,9E1,7E0,2.84E2,5.6E1,2.51E2,6.9E1,4.27E2,1.9E1,8.4E1,1.1E1,3E0,1.1E1,4E0,2.54E2,2E0,7E0,6E0,6.1E1,9.9E1,3.8E1,1.04E2,8E1,2.7E1,3E0,4.56E2,2.9E1,1.1E1,7.9E1,2.79E2,5E0,3.8E1,1.8E1,2.8E1,2.23E2,4.1E1,2.8E1,7.1E1,3.56E2,4E0,1.5E1,5.9E1,2.5E1,8E0,3E0,5E0,2.49E2,5.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.191869E-4,5.7219495E-3,-4.7454964E-2,-1.0996642E-2,3.648694E-2,-1.24135844E-1,-3.2339003E-2,-1.755248E-2,4.385029E-2,9.7798556E-2,2.5957005E-2,-7.240969E-2,-1.2597989E-2,-1.5674135E-2,-7.2853506E-2,-1.0120915E-2,-5.7817478E-2,7.173417E-2,-1.0708805E-2,1.2414128E-1,-8.2834415E-2,2.2127982E-2,1.255441E-1,-1.9368847E-3,-1.10282086E-1,-4.2948057E-3,5.1709507E-5,1.964069E-3,-8.6261004E-2,-1.4697574E-2,4.4655595E-2,-4.4090685E-2,-1.8739015E-1,-2.816523E-2,9.202699E-2,-7.878122E-3,1.1036517E-2,1.417953E-1,-1.2434712E-3,-2.0806082E-2,-7.525068E-5,2.9536547E-2,-1.6315576E-2,1.5478192E-2,7.491589E-2,5.3847E-3,-3.3699577E-3,-6.200131E-3,3.8283244E-3,1.6751887E-2,-9.1467336E-2,-6.977814E-2,-9.19769E-3,-1.8928327E-2,5.0551966E-2,1.9228399E-2,1.3732272E-1,-4.843383E-2,1.18347015E-2,-1.5765702E-2,-3.1327477E-3,-3.3883967E-3,7.146558E-3,-1.96626E-3,1.0732103E-1,7.014859E-3,-1.8797957E-4,1.5092318E-1,-4.961201E-3,7.8300975E-2,1.6309122E-2,-6.486333E-2,6.498581E-2,1.2615668E-3,8.392244E-3,3.406414E-2,-2.6081933E-2,-8.886817E-3,-7.8370015E-4,-9.590577E-2,3.749297E-3,9.565159E-5,-1.8297962E-3,-1.2386943E-2,3.087671E-3,1.4099659E-3,-7.6241856E-3,1.45658245E-2,1.7686107E-3,-7.82894E-3,-1.9064385E-3,-7.8194705E-4,5.8907233E-3,7.533757E-3,-1.3868688E-3,4.36843E-3,-1.7724967E-3,-1.3026588E-3,1.447021E-3,-2.2805235E-3,-8.450147E-3,6.8340497E-3,-5.819276E-4,8.016655E-4,6.9945436E-3,-4.315518E-3,9.7502884E-4,1.7219377E-3,-5.2724998E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,-1,-1,-1,69,71,-1,73,-1,-1,-1,-1,75,77,79,-1,81,83,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,-1,93,-1,95,97,99,101,-1,-1,103,105,-1,-1,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.0522984E-1,1.2096314E0,3.7153566E-1,5.487508E-1,5.3202426E-1,3.6180532E-1,1.8250397E-1,4.0696877E-1,2.487432E-1,5.8604527E-1,2.6719734E-1,1.06545374E-1,0E0,2.2578159E-1,1.24142855E-1,2.8920442E-1,3.688047E-1,2.2153401E-1,1.8909621E-1,2.845794E-1,4.5649242E-1,1.9500822E-1,2.3596749E-1,1.2531355E-1,1.0522139E-1,0E0,2.476021E-1,0E0,1.1040938E-1,2.9440302E-1,2.0706712E-1,2.5606948E-1,3.2441854E-1,1.5409806E-1,1.8759203E-1,0E0,1.0189596E-1,2.2842407E-1,0E0,0E0,0E0,3.6832392E-1,4.4233218E-1,0E0,1.0258898E-1,0E0,0E0,0E0,0E0,1.024808E-1,1.73639E-1,2.512327E-1,0E0,4.031097E-1,2.9647428E-1,1.3407022E-1,3.0698025E-1,1.8302217E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5908122E-1,0E0,0E0,1.1028576E-1,0E0,1.8507981E-1,2.7430484E-1,1.262869E-1,2.535136E-1,0E0,0E0,1.876906E-1,1.2078086E-1,0E0,0E0,1.0499188E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,41,41,42,42,44,44,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,64,64,67,67,69,69,70,70,71,71,72,72,75,75,76,76,79,79],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,-1,-1,-1,70,72,-1,74,-1,-1,-1,-1,76,78,80,-1,82,84,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,-1,94,-1,96,98,100,102,-1,-1,104,106,-1,-1,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2343444E7,5.862126E2,1.1328599E3,9.750871E6,1.0594161E1,1E0,1.0828989E8,9.837297E0,4.1E2,1.93974E7,3.832274E2,2.3428571E0,-1.2597989E-2,9.173084E2,8.7757526E2,3.8E1,9.561769E-1,4E0,3.943114E7,3.159399E6,3.14088E5,2.7183437E1,4.8083666E-1,5.7581736E7,5.441E3,-4.2948057E-3,9E0,1.964069E-3,2.88001E10,4.5723195E6,1.2224265E1,1.0409E4,6.137763E2,3.324E3,6.62E2,-7.878122E-3,1.0927339E7,5.7E1,-1.2434712E-3,-2.0806082E-2,-7.525068E-5,1.683653E8,7.06699E8,1.5478192E-2,1.0787629E0,5.3847E-3,-3.3699577E-3,-6.200131E-3,3.8283244E-3,5.019624E7,1.27365E4,7.1E1,-9.19769E-3,2.3416922E9,2.35E2,8.67822E-1,1.6681991E-3,5.664E1,1.18347015E-2,-1.5765702E-2,-3.1327477E-3,-3.3883967E-3,7.146558E-3,-1.96626E-3,1.8481675E0,7.014859E-3,-1.8797957E-4,8.5E1,-4.961201E-3,7.2201815E6,1E0,1.912391E6,4.155E3,1.2615668E-3,8.392244E-3,2.2216E4,7.808427E9,-8.886817E-3,-7.8370015E-4,1.1E1,3.749297E-3,9.565159E-5,-1.8297962E-3,-1.2386943E-2,3.087671E-3,1.4099659E-3,-7.6241856E-3,1.45658245E-2,1.7686107E-3,-7.82894E-3,-1.9064385E-3,-7.8194705E-4,5.8907233E-3,7.533757E-3,-1.3868688E-3,4.36843E-3,-1.7724967E-3,-1.3026588E-3,1.447021E-3,-2.2805235E-3,-8.450147E-3,6.8340497E-3,-5.819276E-4,8.016655E-4,6.9945436E-3,-4.315518E-3,9.7502884E-4,1.7219377E-3,-5.2724998E-3],"split_indices":[45,52,4,9,56,101,45,54,0,50,56,56,0,52,52,3,38,3,5,32,1,53,42,45,2,0,17,0,5,28,56,2,4,29,2,0,9,6,0,0,0,7,7,0,34,0,0,0,0,51,33,8,0,31,2,27,41,50,0,0,0,0,0,0,54,0,0,8,0,50,102,1,10,0,0,2,12,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.673E3,2.35E3,3.23E2,1.523E3,8.27E2,5.2E1,2.71E2,1.361E3,1.62E2,1.2E2,7.07E2,3.9E1,1.3E1,1.93E2,7.8E1,1.15E3,2.11E2,1.07E2,5.5E1,1.05E2,1.5E1,6.82E2,2.5E1,1.4E1,2.5E1,3.3E1,1.6E2,8E0,7E1,1.062E3,8.8E1,1.92E2,1.9E1,1.8E1,8.9E1,6E0,4.9E1,9.4E1,1.1E1,2E0,1.3E1,5.72E2,1.1E2,4E0,2.1E1,5E0,9E0,2.3E1,2E0,1.36E2,2.4E1,6.2E1,8E0,9.98E2,6.4E1,7E1,1.8E1,1.9E2,2E0,8E0,1.1E1,1.5E1,3E0,9E0,8E1,4E0,4.5E1,9.1E1,3E0,1.21E2,4.51E2,6.9E1,4.1E1,1.5E1,6E0,9.7E1,3.9E1,1E1,1.4E1,5.3E1,9E0,4.79E2,5.19E2,2E0,6.2E1,6.7E1,3E0,6E0,1.2E1,1.2E1,1.78E2,9E0,7.1E1,8.8E1,3E0,1.09E2,1.2E1,1.09E2,3.42E2,6.1E1,8E0,2E1,2.1E1,8.5E1,1.2E1,1.6E1,2.3E1,5E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-1.3900019E-3,-1.1255132E-2,2.73545E-2,-2.2133246E-2,1.2760066E-2,3.9756186E-2,-2.8867034E-2,-1.7614743E-2,-9.140465E-2,1.66642E-2,-7.6859975E-3,6.704436E-3,3.252154E-2,-1.0310641E-2,-8.305423E-2,-1.6482973E-2,-1.4810501E-2,-3.7816742E-1,-6.483827E-2,3.440376E-2,-8.893297E-3,1.142074E-1,2.5648283E-2,-3.4827532E-3,6.8297156E-4,-4.5929883E-2,-1.0012942E-2,-2.652305E-2,1.0224242E-2,-3.2783188E-2,-5.6849867E-3,6.8132686E-3,-8.380953E-2,1.7285544E-2,8.4284075E-2,-2.1321741E-1,-4.7712415E-3,4.0985093E-2,8.715659E-3,-1.6266054E-1,2.8307777E-2,4.037223E-3,-4.2725164E-3,-2.1719882E-2,-9.600978E-2,3.783808E-2,-1.8710366E-2,1.0441071E-3,-5.2315337E-3,2.4602009E-2,-1.10197775E-1,3.3671193E-2,1.6719481E-1,-1.1866164E-3,-1.5831877E-2,-1.2086632E-2,3.8293311E-3,5.4091145E-3,-2.3690618E-3,-1.3126214E-2,3.7005902E-3,4.7383428E-2,8.231538E-3,-8.316518E-4,-4.1234116E-3,-3.1758456E-3,-1.2461748E-2,-5.831956E-4,3.8955768E-3,-5.7977834E-3,-1.5333088E-4,2.8020244E-3,-9.778156E-5,2.3723063E-4,-8.98023E-3,2.3188407E-3,-1.2344854E-2,1.6121937E-2,5.4025436E-3,1.5649153E-3,-1.2417021E-3,5.1025758E-3,1.3025291E-3,-1.3173674E-3,2.0882448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,41,-1,43,45,-1,-1,-1,47,49,51,53,55,57,-1,59,61,-1,-1,63,65,67,69,-1,-1,71,73,75,77,-1,-1,79,-1,-1,-1,-1,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.701957E-1,5.2850986E-1,4.850335E-1,4.3409014E-1,4.2947972E-1,4.0808153E-1,1.2583902E-1,4.2663708E-1,6.230314E-1,2.8049403E-1,0E0,0E0,2.95761E-1,1.467277E-1,1.4011271E-1,3.4992906E-1,0E0,4.751855E-1,3.2231903E-1,3.1020272E-1,2.1291046E-1,1.9603598E-1,2.5100112E-1,0E0,0E0,1.540818E-1,0E0,3.1431574E-1,2.8594932E-1,0E0,0E0,0E0,1.984368E-1,2.5734562E-1,3.8463098E-1,1.0239215E-1,1.5592712E-1,1.4228132E-1,0E0,2.0832443E-1,1.8573204E-1,0E0,0E0,2.4743545E-1,2.8018236E-1,3.9918926E-1,2.7739984E-1,0E0,0E0,2.3410234E-1,1.3447797E-1,2.6608258E-1,2.8455818E-1,0E0,0E0,1.4397147E-1,0E0,0E0,0E0,0E0,0E0,2.944659E-1,3.020816E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,27,27,28,28,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,55,55,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,42,-1,44,46,-1,-1,-1,48,50,52,54,56,58,-1,60,62,-1,-1,64,66,68,70,-1,-1,72,74,76,78,-1,-1,80,-1,-1,-1,-1,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0989723E3,1.273801E6,6.747114E7,9.60094E5,1E0,2.139E3,3.01968E5,1.6904226E7,4E0,2.7E1,-7.6859975E-3,6.704436E-3,9.532E3,1.3715873E3,1.4299594E8,1.5107028E-1,-1.4810501E-2,1.964127E6,3.7344142E-3,4.3689142E2,1.5687E4,4.9807886E3,1.43E2,-3.4827532E-3,6.8297156E-4,5.3474556E5,-1.0012942E-2,9.793811E-2,2.4013288E1,-3.2783188E-2,-5.6849867E-3,6.8132686E-3,2.5378451E0,1E0,8.84E2,7.56484E5,5.96E2,2.3058404E7,8.715659E-3,1.8E1,1.778482E7,4.037223E-3,-4.2725164E-3,6.971004E7,6.949118E-1,2.8080197E2,6.63E2,1.0441071E-3,-5.2315337E-3,1E0,2.315878E6,3.7852024E1,1.526038E6,-1.1866164E-3,-1.5831877E-2,9.87156E0,3.8293311E-3,5.4091145E-3,-2.3690618E-3,-1.3126214E-2,3.7005902E-3,2.7203197E0,1.3532244E7,-8.316518E-4,-4.1234116E-3,-3.1758456E-3,-1.2461748E-2,-5.831956E-4,3.8955768E-3,-5.7977834E-3,-1.5333088E-4,2.8020244E-3,-9.778156E-5,2.3723063E-4,-8.98023E-3,2.3188407E-3,-1.2344854E-2,1.6121937E-2,5.4025436E-3,1.5649153E-3,-1.2417021E-3,5.1025758E-3,1.3025291E-3,-1.3173674E-3,2.0882448E-3],"split_indices":[52,9,45,9,85,29,11,47,3,8,0,0,9,52,45,38,0,1,42,52,1,4,0,0,0,32,0,38,58,0,0,0,54,74,10,12,8,45,0,3,50,0,0,45,27,4,10,0,0,16,9,53,9,0,0,58,0,0,0,0,0,53,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.714E3,2.021E3,6.93E2,1.391E3,6.3E2,5.68E2,1.25E2,1.307E3,8.4E1,6.17E2,1.3E1,3.7E1,5.31E2,9.4E1,3.1E1,1.303E3,4E0,6E0,7.8E1,3.64E2,2.53E2,4E1,4.91E2,2.6E1,6.8E1,2.5E1,6E0,9.47E2,3.56E2,2E0,4E0,6E0,7.2E1,2.72E2,9.2E1,4E0,2.49E2,2E1,2E1,6E0,4.85E2,6E0,1.9E1,8.87E2,6E1,1.82E2,1.74E2,1.4E1,5.8E1,2.58E2,1.4E1,5.8E1,3.4E1,2E0,2E0,2.3E2,1.9E1,1.1E1,9E0,4E0,2E0,2.48E2,2.37E2,8.32E2,5.5E1,5.2E1,8E0,8.5E1,9.7E1,2.2E1,1.52E2,1.13E2,1.45E2,6E0,8E0,5.6E1,2E0,7E0,2.7E1,5.4E1,1.76E2,6.2E1,1.86E2,1.18E2,1.19E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-9.42093E-4,-1.0269936E-2,2.6848795E-2,-3.2133204E-3,-5.5798084E-2,3.9437436E-2,-1.9263199E-2,-6.2390894E-2,1.5729344E-3,-8.365326E-2,-2.2550879E-2,3.3939965E-2,1.4890884E-1,-1.02001026E-1,1.0026623E-2,-2.1810383E-1,-4.759728E-2,1.8253697E-1,-4.754161E-4,-6.2222775E-2,-7.1759513E-3,3.9225303E-2,-5.1356718E-2,6.950085E-2,1.9002955E-2,2.117591E-1,3.5728277E-3,-1.3749518E-1,-1.1395521E-4,3.181979E-2,-8.7454304E-2,-5.468184E-3,-1.7787682E-2,6.2922486E-3,-5.9847604E-2,-2.574821E-3,2.3338163E-1,1.02103844E-1,-2.9834833E-3,-1.060101E-2,-5.112676E-2,7.430462E-3,-4.8361078E-4,-2.1008137E-1,-3.2137517E-2,2.421696E-2,9.809699E-2,4.197099E-2,-9.2188185E-3,1.2005438E-2,-7.449099E-5,2.8561943E-4,-8.090687E-3,6.63662E-5,1.0551041E-1,-1.7437096E-1,-2.3407923E-5,-5.1662844E-2,-1.2928845E-2,1.1251019E-3,1.359593E-2,2.4677124E-3,9.35786E-3,-1.1543275E-2,2.9295556E-2,-3.68488E-2,-8.836297E-3,-4.380566E-3,-1.6066197E-2,2.7728886E-3,-4.9650706E-2,8.977288E-2,-1.1703331E-2,1.1604693E-1,3.2104415E-4,1.1539923E-2,3.4458287E-2,8.729137E-2,-1.9004084E-2,6.800164E-3,-3.6453793E-3,-8.204114E-5,-1.1882718E-2,-3.0424157E-3,4.2409617E-3,-4.929855E-4,-1.5100146E-2,3.1278091E-3,-4.9669173E-4,1.518936E-3,-3.1874238E-3,-5.4462412E-3,-5.677891E-4,8.036697E-3,8.2647754E-4,6.033123E-3,-1.4864083E-3,6.2632924E-3,-3.9223552E-4,4.3917028E-4,4.223964E-3,6.3702683E-3,-6.7023453E-3,-3.6129553E-3,3.1866322E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,-1,53,55,-1,-1,-1,57,-1,59,61,63,-1,65,-1,-1,67,69,71,73,75,77,-1,-1,-1,-1,-1,79,81,-1,83,-1,-1,-1,-1,-1,85,87,89,-1,-1,-1,-1,91,93,95,97,-1,-1,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.942318E-1,6.4397633E-1,3.9142767E-1,4.9208415E-1,2.4723619E-1,3.1330657E-1,3.524123E-1,2.8908122E-1,5.9613675E-1,1.9776702E-1,2.2267433E-1,2.6692176E-1,1.0095388E-1,1.3404006E-1,2.3221995E-1,1.3285857E-1,2.7097872E-1,2.2881186E-1,4.0963024E-1,1.8645003E-1,0E0,2.3028962E-1,2.5254622E-1,1.9057107E-1,2.3179387E-1,1.0833478E-1,0E0,1.2732202E-1,0E0,1.9945413E-1,1.510898E-1,0E0,0E0,0E0,1.803872E-1,0E0,1.5236598E-1,1.6923687E-1,4.297112E-1,0E0,1.950722E-1,0E0,0E0,1.0317394E-1,1.2429257E-1,1.3998312E-1,1.4850801E-1,2.8620774E-1,1.5370372E-1,0E0,0E0,0E0,0E0,0E0,1.8309674E-1,1.2691271E-1,0E0,1.886422E-1,0E0,0E0,0E0,0E0,0E0,4.6313572E-1,4.664166E-1,1.9969256E-1,0E0,0E0,0E0,0E0,1.5605463E-1,1.1594634E-1,1.07726395E-1,1.4021432E-1,0E0,0E0,2.5876594E-1,1.7447913E-1,1.6323543E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,34,34,36,36,37,37,38,38,40,40,43,43,44,44,45,45,46,46,47,47,48,48,54,54,55,55,57,57,63,63,64,64,65,65,70,70,71,71,72,72,73,73,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,-1,54,56,-1,-1,-1,58,-1,60,62,64,-1,66,-1,-1,68,70,72,74,76,78,-1,-1,-1,-1,-1,80,82,-1,84,-1,-1,-1,-1,-1,86,88,90,-1,-1,-1,-1,92,94,96,98,-1,-1,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,2.6944466E7,3.666495E7,4.0701206E1,1.6667E4,3.1699734E2,2.8316305E0,7.2727275E-1,3.9333332E1,2.7448502E8,2.1525E4,6.85945E4,5.7719237E-1,5.2E1,1.75E2,3.79E2,2E0,9.21641E3,1.0620689E0,1.752512E2,-7.1759513E-3,3.1665432E0,3E0,1.0989723E3,1.9051096E7,5.8E1,3.5728277E-3,2.777588E6,-1.1395521E-4,1.5005797E4,3.555E3,-5.468184E-3,-1.7787682E-2,6.2922486E-3,1.1603518E8,-2.574821E-3,1.7848537E0,1.5053125E2,5.862126E2,-1.060101E-2,1.1347826E1,7.430462E-3,-4.8361078E-4,2.2515285E6,2.0892E-5,6.341735E2,4.2343444E7,6.439778E5,4.35581E5,1.2005438E-2,-7.449099E-5,2.8561943E-4,-8.090687E-3,6.63662E-5,1.066317E12,2.656E3,-2.3407923E-5,1.16146E5,-1.2928845E-2,1.1251019E-3,1.359593E-2,2.4677124E-3,9.35786E-3,5.8018835E2,5.3506675E6,1.8159722E-1,-8.836297E-3,-4.380566E-3,-1.6066197E-2,2.7728886E-3,6.63E2,1.5736016E6,8E-1,2.5492362E7,3.2104415E-4,1.1539923E-2,4.95E9,5.0818306E-1,2.3023027E3,6.800164E-3,-3.6453793E-3,-8.204114E-5,-1.1882718E-2,-3.0424157E-3,4.2409617E-3,-4.929855E-4,-1.5100146E-2,3.1278091E-3,-4.9669173E-4,1.518936E-3,-3.1874238E-3,-5.4462412E-3,-5.677891E-4,8.036697E-3,8.2647754E-4,6.033123E-3,-1.4864083E-3,6.2632924E-3,-3.9223552E-4,4.3917028E-4,4.223964E-3,6.3702683E-3,-6.7023453E-3,-3.6129553E-3,3.1866322E-5],"split_indices":[2,45,51,52,9,56,54,53,50,7,9,32,42,8,8,1,17,28,54,52,0,56,3,52,45,3,0,1,0,4,0,0,0,0,12,0,54,52,52,0,53,0,0,28,38,52,45,51,29,0,0,0,0,0,31,0,0,1,0,0,0,0,0,52,50,27,0,0,0,0,10,50,53,50,0,0,5,39,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.676E3,2.004E3,6.72E2,1.736E3,2.68E2,5.28E2,1.44E2,1.29E2,1.607E3,1.45E2,1.23E2,5.04E2,2.4E1,3.7E1,1.07E2,1E1,1.19E2,1.7E1,1.59E3,1.11E2,3.4E1,3.9E1,8.4E1,1.48E2,3.56E2,1.2E1,1.2E1,2.7E1,1E1,8.8E1,1.9E1,7E0,3E0,7E0,1.12E2,3E0,1.4E1,3.7E1,1.553E3,6E0,1.05E2,1.1E1,2.8E1,8E0,7.6E1,5.8E1,9E1,1.96E2,1.6E2,1E1,2E0,5E0,2.2E1,6.3E1,2.5E1,9E0,1E1,1.09E2,3E0,3E0,1.1E1,2.5E1,1.2E1,1.228E3,3.25E2,9.6E1,9E0,5E0,3E0,1.2E1,6.4E1,2E1,3.8E1,7.5E1,1.5E1,6E0,1.9E2,1.4E1,1.46E2,2.1E1,4E0,3E0,6E0,1.01E2,8E0,1.224E3,4E0,1.7E2,1.55E2,2.9E1,6.7E1,2.3E1,4.1E1,9E0,1.1E1,4E0,3.4E1,6.7E1,8E0,1.3E2,6E1,1.2E1,2E0,3.7E1,1.09E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-4.1469975E-5,-8.570121E-3,2.4705455E-2,-1.9341419E-2,1.6644098E-2,6.571429E-2,1.2190902E-2,-9.68403E-3,-6.3361935E-2,6.0850527E-2,-3.7579853E-3,2.6992334E-3,7.975073E-2,1.7474692E-2,-1.14948966E-1,-2.2701949E-2,2.6384395E-2,1.2796186E-2,-6.847931E-2,5.1210597E-2,8.90467E-3,1.0108443E-3,-8.385356E-3,-4.354762E-3,2.5890209E-3,1.4707677E-1,6.311959E-2,2.5137894E-2,-4.062022E-2,-2.8776708E-3,-1.5951563E-2,-2.7899805E-2,2.4038649E-3,3.7899524E-2,-6.5496616E-2,-2.6724523E-1,-6.067522E-2,7.735741E-2,-1.5078407E-3,7.8688115E-2,-7.214078E-3,1.4918168E-2,3.7717083E-3,-2.2200262E-3,7.375504E-2,1.5048365E-2,6.9619E-2,-6.1465073E-3,-7.419049E-4,3.52969E-2,-3.4762565E-2,6.7535095E-2,-4.2461865E-3,4.4878755E-2,-4.845774E-3,-1.6547771E-2,-3.975008E-3,4.203245E-2,-7.2927445E-2,-3.9415895E-3,8.983077E-2,-1.3296394E-2,1.8242907E-2,1.2480075E-3,1.1751065E-2,-3.8920145E-2,1.3956699E-2,9.949706E-4,1.0468081E-1,4.6742108E-2,-1.0318309E-2,7.7429645E-2,-7.7521E-3,-1.3868338E-3,4.155791E-3,-3.8695594E-3,-1.1875863E-3,4.7419703E-4,4.470718E-3,1.8224358E-3,-2.0540163E-3,-7.7015925E-3,6.0640266E-3,9.652642E-3,1.8516665E-4,-4.0700664E-3,3.1215508E-4,4.883569E-3,-2.7019691E-3,3.7968731E-3,-2.6453065E-3,-2.7578396E-3,1.4835856E-3,3.2092386E-3,-7.0588005E-4,6.2476736E-3,-1.1659035E-3,1.1502858E-2,1.8084472E-3,8.186828E-6,-5.090935E-3,4.6291742E-3,-2.6528517E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,41,43,45,47,-1,-1,49,-1,51,53,55,57,59,61,63,65,-1,-1,-1,67,69,71,-1,-1,73,75,77,79,81,-1,-1,-1,83,85,-1,87,-1,89,-1,-1,91,93,-1,95,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.607814E-1,5.369835E-1,3.48693E-1,5.881821E-1,5.3415537E-1,1.4042139E-1,3.5334983E-1,5.344612E-1,4.3950295E-1,2.1596909E-1,3.319087E-1,1.4918782E-1,1.3627875E-1,2.249532E-1,2.3129207E-1,3.1816733E-1,3.2200485E-1,0E0,3.6537683E-1,2.4133995E-1,0E0,2.5356793E-1,0E0,0E0,0E0,2.5631076E-1,1.2683752E-1,1.9866538E-1,1.2807095E-1,0E0,0E0,3.399498E-1,0E0,3.3627102E-1,1.3846874E-1,1.01136446E-1,3.0256402E-1,2.4040091E-1,3.2684174E-1,3.26957E-1,2.4151912E-1,0E0,0E0,0E0,1.5675181E-1,2.931255E-1,1.6079688E-1,0E0,0E0,2.5721827E-1,3.2141292E-1,2.3417419E-1,1.8471186E-1,1.7169206E-1,0E0,0E0,0E0,1.5751877E-1,2.0462227E-1,0E0,1.9932544E-1,0E0,2.5554723E-1,0E0,0E0,1.8966702E-1,3.298437E-1,0E0,2.0605266E-1,2.723514E-1,2.0440143E-1,1.281878E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,21,21,25,25,26,26,27,27,28,28,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,57,57,58,58,60,60,62,62,65,65,66,66,68,68,69,69,70,70,71,71],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,42,44,46,48,-1,-1,50,-1,52,54,56,58,60,62,64,66,-1,-1,-1,68,70,72,-1,-1,74,76,78,80,82,-1,-1,-1,84,86,-1,88,-1,90,-1,-1,92,94,-1,96,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,1.273801E6,2.909019E3,3.38464E5,1.5277338E0,7.57657E4,1.2367184E3,1.5107028E-1,3.938775E4,4.94374E1,1E0,2.5629143E3,1.6081998E5,1.9840434E7,9.599108E6,3.454E3,2.1199985E-7,1.2796186E-2,2.9143541E-6,5.4803147E0,8.90467E-3,4.5680464E3,-8.385356E-3,-4.354762E-3,2.5890209E-3,4.8E1,2.4513194E5,8.015419E3,1.6108601E7,-2.8776708E-3,-1.5951563E-2,6E0,2.4038649E-3,2.558106E10,4.0434834E5,1.4E1,4.1504726E-3,7.8E3,7.79E2,6.0559247E2,6.98E2,1.4918168E-2,3.7717083E-3,-2.2200262E-3,1.3999657E3,1.9051096E7,1E0,-6.1465073E-3,-7.419049E-4,3.79E2,2.6216663E-5,1.8991614E5,1.5773E4,3.8527173E-1,-4.845774E-3,-1.6547771E-2,-3.975008E-3,2.2444444E0,4.832E3,-3.9415895E-3,2.507E3,-1.3296394E-2,9E0,1.2480075E-3,1.1751065E-2,1.1994927E7,1.9E1,9.949706E-4,1.24900505E2,6.439778E5,1.8834772E7,6.747114E7,-7.7521E-3,-1.3868338E-3,4.155791E-3,-3.8695594E-3,-1.1875863E-3,4.7419703E-4,4.470718E-3,1.8224358E-3,-2.0540163E-3,-7.7015925E-3,6.0640266E-3,9.652642E-3,1.8516665E-4,-4.0700664E-3,3.1215508E-4,4.883569E-3,-2.7019691E-3,3.7968731E-3,-2.6453065E-3,-2.7578396E-3,1.4835856E-3,3.2092386E-3,-7.0588005E-4,6.2476736E-3,-1.1659035E-3,1.1502858E-2,1.8084472E-3,8.186828E-6,-5.090935E-3,4.6291742E-3,-2.6528517E-4],"split_indices":[2,9,32,9,53,28,58,38,45,58,85,4,28,9,29,0,37,0,38,56,0,50,0,0,0,3,28,4,32,0,0,10,0,19,28,0,41,1,2,33,10,0,0,0,52,45,105,0,0,1,38,28,9,41,0,0,0,56,2,0,2,0,8,0,0,9,8,0,58,51,43,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.655E3,1.975E3,6.8E2,1.384E3,5.91E2,1.58E2,5.22E2,1.136E3,2.48E2,1.86E2,4.05E2,2.9E1,1.29E2,5.02E2,2E1,8.35E2,3.01E2,3E0,2.45E2,1.74E2,1.2E1,3.95E2,1E1,1E1,1.9E1,2.4E1,1.05E2,4.44E2,5.8E1,1.7E1,3E0,7.8E2,5.5E1,2.68E2,3.3E1,8E0,2.37E2,1.16E2,5.8E1,3.7E1,3.58E2,6E0,1.8E1,9E0,9.6E1,3.63E2,8.1E1,1.2E1,4.6E1,7.6E1,7.04E2,1.57E2,1.11E2,8E0,2.5E1,5E0,3E0,2.5E1,2.12E2,8E0,1.08E2,3E0,5.5E1,2.9E1,8E0,1.43E2,2.15E2,3.6E1,6E1,1.61E2,2.02E2,7.9E1,2E0,3.4E1,4.2E1,1.24E2,5.8E2,4.9E1,1.08E2,5.3E1,5.8E1,2E0,6E0,4E0,2.1E1,1.84E2,2.8E1,1E2,8E0,3E1,2.5E1,1.13E2,3E1,7.5E1,1.4E2,5E1,1E1,6E0,1.55E2,1.83E2,1.9E1,6.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[9.198005E-4,4.8439465E-3,-5.11713E-2,-6.579271E-3,2.8507423E-2,-7.375152E-2,-5.913907E-3,-1.1211438E-1,-4.5418767E-3,4.7929484E-2,5.823528E-3,-5.4088093E-2,-1.5603808E-1,3.4185268E-2,-9.84639E-2,3.7861988E-3,-1.3542083E-1,4.862542E-2,-9.444326E-3,7.563755E-2,2.715072E-2,-1.04328925E-2,5.579247E-2,-4.1107688E-2,-1.6680098E-1,-2.7865607E-3,-1.1536205E-2,9.122621E-3,9.261993E-3,-1.0941584E-2,-1.7103421E-3,-2.3526622E-3,-9.021094E-3,1.030198E-2,1.01559624E-1,-1.3420478E-1,-8.069678E-3,1.1045616E-1,2.9987702E-3,-1.4942914E-1,3.5300627E-2,-7.458258E-3,-1.3391635E-2,-3.4805841E-3,6.583414E-2,-7.464208E-2,7.702907E-5,1.86752E-3,-1.0685555E-2,3.777007E-3,-2.0395743E-3,2.9705465E-2,-1.3729803E-2,1.7369556E-1,5.4655455E-2,-4.4932544E-2,-2.4236135E-2,7.659578E-3,-1.9034194E-2,1.6395171E-1,8.069363E-2,-1.5343296E-3,6.9832434E-3,-2.1145646E-3,-1.9895859E-2,1.05102025E-1,1.8215546E-2,-1.0683364E-2,8.085559E-3,8.149354E-2,-1.672021E-3,-4.2315875E-3,3.3155053E-3,9.468568E-3,3.3678385E-4,9.86362E-3,1.0918102E-3,3.8370893E-3,-1.2077263E-2,-7.175806E-3,8.326449E-3,8.605302E-4,-2.202211E-3,-5.237995E-3,-5.688547E-4,3.2000553E-3,1.140098E-2,-5.939246E-3,4.317671E-3,6.7625375E-4,7.334658E-3,5.8310974E-4,1.1457703E-2,-9.494054E-3,-2.3749942E-4,8.325102E-3,2.8963252E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,-1,-1,-1,49,-1,-1,-1,-1,51,53,55,57,59,61,63,65,67,-1,-1,69,71,-1,-1,-1,-1,-1,73,-1,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.339704E-1,6.567779E-1,1.8657556E-1,3.5190305E-1,3.4821016E-1,1.8896055E-1,2.3345907E-1,1.488294E-1,4.1984504E-1,2.4321139E-1,2.978898E-1,1.3944909E-1,1.7694426E-1,1.6964215E-1,1.468419E-1,0E0,1.1868286E-1,2.7484617E-1,2.5171053E-1,4.5997965E-1,3.5785675E-1,2.1972586E-1,1.2088978E-1,1.3047382E-1,1.246129E-1,0E0,0E0,0E0,1.4411229E-1,0E0,0E0,0E0,0E0,4.695327E-1,1.8336475E-1,5.084218E-1,2.5177032E-1,1.8260646E-1,3.0490685E-1,2.897089E-1,2.783377E-1,1.5815821E-1,0E0,0E0,1.3536748E-1,1.0676971E-1,0E0,0E0,0E0,0E0,0E0,2.881122E-1,0E0,1.0242015E-1,3.0151618E-1,3.47392E-1,0E0,3.3263847E-1,5.546659E-1,2.936312E-1,1.6809994E-1,0E0,0E0,0E0,0E0,1.9894409E-1,2.507292E-1,2.8997067E-1,0E0,1.3115239E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,44,44,45,45,51,51,53,53,54,54,55,55,57,57,58,58,59,59,60,60,65,65,66,66,67,67,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,-1,-1,-1,50,-1,-1,-1,-1,52,54,56,58,60,62,64,66,68,-1,-1,70,72,-1,-1,-1,-1,-1,74,-1,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,7.298614E2,1.0052E4,1.0882353E1,1.504779E7,9.222717E-1,6.9998717E-4,1E0,1E0,6.627257E6,9.908038E1,1.0192391E2,1.2199979E10,8.24666E5,3.4409692E3,3.7861988E-3,2.645E3,5.69815E5,1.0195312E0,5.4770195E1,3.171E3,9.195E1,1E1,1.249246E6,6.13E2,-2.7865607E-3,-1.1536205E-2,9.122621E-3,4.7777777E0,-1.0941584E-2,-1.7103421E-3,-2.3526622E-3,-9.021094E-3,4.9422E5,3.011152E6,2.8781365E6,5.372E3,1.5938586E8,4.0602388E6,2.857E3,1.5595E4,6.001692E-7,-1.3391635E-2,-3.4805841E-3,2.8721272E1,7.221312E-2,7.702907E-5,1.86752E-3,-1.0685555E-2,3.777007E-3,-2.0395743E-3,2.6307692E1,-1.3729803E-2,3.06E2,1.3794063E1,2.7460318E0,-2.4236135E-2,1E0,7.59E2,3.275E3,2.41E3,-1.5343296E-3,6.9832434E-3,-2.1145646E-3,-1.9895859E-2,1.6554546E3,1.921032E4,9.8708276E2,8.085559E-3,2.1853803E4,-1.672021E-3,-4.2315875E-3,3.3155053E-3,9.468568E-3,3.3678385E-4,9.86362E-3,1.0918102E-3,3.8370893E-3,-1.2077263E-2,-7.175806E-3,8.326449E-3,8.605302E-4,-2.202211E-3,-5.237995E-3,-5.688547E-4,3.2000553E-3,1.140098E-2,-5.939246E-3,4.317671E-3,6.7625375E-4,7.334658E-3,5.8310974E-4,1.1457703E-2,-9.494054E-3,-2.3749942E-4,8.325102E-3,2.8963252E-3],"split_indices":[45,52,2,52,45,27,34,75,53,51,56,58,5,29,52,0,9,9,53,54,2,53,3,1,0,0,0,0,54,0,0,0,0,9,9,47,29,7,48,2,9,37,0,0,53,38,0,0,0,0,0,47,0,0,56,58,0,105,2,2,2,0,0,0,0,55,4,4,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E3,2.428E3,1.82E2,1.638E3,7.9E2,1.21E2,6.1E1,3E1,1.608E3,4.25E2,3.65E2,9.9E1,2.2E1,4.3E1,1.8E1,3E0,2.7E1,1.35E2,1.473E3,1.81E2,2.44E2,2.76E2,8.9E1,9E1,9E0,1.1E1,1.1E1,5E0,3.8E1,5E0,1.3E1,1.1E1,1.6E1,7.9E1,5.6E1,1.5E1,1.458E3,1.22E2,5.9E1,1E1,2.34E2,2.74E2,2E0,6E0,8.3E1,5E1,4E1,2E0,7E0,1.6E1,2.2E1,7.5E1,4E0,2.1E1,3.5E1,1.3E1,2E0,5.99E2,8.59E2,4.2E1,8E1,4.8E1,1.1E1,8E0,2E0,4.5E1,1.89E2,2.7E2,4E0,7.2E1,1.1E1,4.6E1,4E0,8E0,6.7E1,1.7E1,4E0,3.3E1,2E0,9E0,4E0,5.03E2,9.6E1,6.2E1,7.97E2,1.9E1,2.3E1,3E0,7.7E1,1.6E1,2.9E1,1.85E2,4E0,7E0,2.63E2,1.2E1,6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[5.3759076E-4,4.010311E-3,-4.798753E-2,-1.3766378E-2,1.7333908E-2,-7.588496E-2,6.7814942E-3,3.9472856E-2,-1.8568153E-2,4.7465187E-2,6.9286674E-3,-5.710561E-2,-1.5260199E-1,2.401453E-3,-1.4560716E-3,-1.1556385E-1,8.258921E-2,-6.3261084E-2,-1.2366538E-2,5.981223E-2,-1.1581077E-2,3.117737E-2,-4.953725E-3,-3.58478E-3,2.2072897E-3,-2.0790774E-1,1.682053E-4,1.5077149E-3,-9.442308E-3,1.5370573E-2,5.9668496E-2,-1.1195473E-1,2.7670525E-2,1.6494092E-2,-3.2572985E-2,7.973781E-2,1.2961323E-3,7.6934725E-2,-8.5022554E-2,5.0297886E-2,-1.928334E-2,-8.581442E-2,4.2595747E-3,-2.8248925E-3,-1.2693039E-2,1.0557215E-1,-5.050949E-4,-2.0541437E-2,-2.1516599E-1,5.9916847E-3,-1.8025972E-3,-1.5911703E-1,2.4037817E-2,-8.077747E-2,-1.4005123E-2,8.3405204E-2,-1.0743794E-2,1.259929E-2,-1.0238417E-2,6.086133E-3,7.5613025E-5,-3.1054965E-1,-2.956548E-2,4.4659078E-2,1.0546614E-2,-9.3632685E-3,-6.21206E-3,-7.827584E-3,-1.247326E-3,8.432361E-4,1.065469E-1,3.3608207E-3,9.219241E-3,-1.8018042E-3,1.0524379E-2,-1.360826E-2,-3.1582182E-3,-4.8306882E-3,2.0917684E-3,-1.2036117E-2,-8.539927E-4,-1.9537578E-3,1.6561705E-3,-2.9522663E-3,-1.3624409E-2,6.851132E-4,-1.8854869E-3,5.179243E-3,2.09883E-3,-2.1360675E-4,7.5745326E-3,-2.8918048E-3,-1.908626E-2,3.4969053E-3,-3.0631498E-3,3.4407133E-3,9.675387E-4,-9.6753903E-4,4.3889163E-3,2.6943293E-4,-4.893813E-3,6.841654E-4,9.057139E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,73,75,-1,77,79,81,83,85,87,-1,89,-1,-1,-1,91,93,95,-1,97,-1,-1,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.566973E-1,5.992308E-1,2.770672E-1,2.7784967E-1,4.5302567E-1,1.6416907E-1,1.0149876E-1,6.1115986E-1,2.7500015E-1,2.71169E-1,3.101851E-1,1.829392E-1,1.9931269E-1,0E0,0E0,2.4618852E-1,3.6875427E-1,5.388607E-1,5.108117E-1,3.5784864E-1,4.290887E-1,3.4252146E-1,5.389961E-1,0E0,0E0,1.2901258E-1,0E0,0E0,0E0,0E0,2.1384083E-1,7.409358E-1,1.2371752E-1,4.8206395E-1,4.5982152E-1,2.7408612E-1,1.9475277E-1,1.13171145E-1,4.41073E-1,2.3641646E-1,1.0614222E-1,3.3910662E-1,2.2705145E-1,0E0,0E0,1.0957217E-1,0E0,1.8477455E-1,3.62149E-1,0E0,1.5759714E-1,1.909334E-1,2.3863739E-1,5.397769E-1,2.684239E-1,2.175597E-1,0E0,1.8943374E-1,0E0,0E0,0E0,1.3001889E-1,1.1052498E-1,1.6476512E-1,0E0,1.0148863E-1,0E0,0E0,0E0,3.1102204E-1,1.5728511E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,57,57,61,61,62,62,63,63,65,65,69,69,70,70],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,74,76,-1,78,80,82,84,86,88,-1,90,-1,-1,-1,92,94,96,-1,98,-1,-1,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,3.3030225E2,1.0052E4,8.695652E-1,8E0,1.29175E5,1.00663277E9,2.35E2,1E0,2.3523688E0,2.1111E4,9.245905E-2,3.285E3,2.401453E-3,-1.4560716E-3,1.00033E5,2.04E5,1.307E3,3.98231E-2,1E0,2.53383E5,1.1900813E1,3E0,-3.58478E-3,2.2072897E-3,8.48106E5,1.682053E-4,1.5077149E-3,-9.442308E-3,1.5370573E-2,4.3440155E6,4.3E2,1.25218E5,1.418705E0,1.7247218E-1,3.7852024E1,4.5709072E1,2.4708056E5,3.90106E0,1.22159E5,3.275343E6,5.695737E0,3.832274E2,-2.8248925E-3,-1.2693039E-2,1.3181542E6,-5.050949E-4,7.920064E5,5.750032E3,5.9916847E-3,3.2E1,1.216E3,1.4431512E-1,1E0,1.0177216E1,2.462277E6,-1.0743794E-2,2.4079119E1,-1.0238417E-2,6.086133E-3,7.5613025E-5,1E1,2.8982632E0,2.342147E7,1.0546614E-2,4.0889E4,-6.21206E-3,-7.827584E-3,-1.247326E-3,1.0963991E7,1.6E1,3.3608207E-3,9.219241E-3,-1.8018042E-3,1.0524379E-2,-1.360826E-2,-3.1582182E-3,-4.8306882E-3,2.0917684E-3,-1.2036117E-2,-8.539927E-4,-1.9537578E-3,1.6561705E-3,-2.9522663E-3,-1.3624409E-2,6.851132E-4,-1.8854869E-3,5.179243E-3,2.09883E-3,-2.1360675E-4,7.5745326E-3,-2.8918048E-3,-1.908626E-2,3.4969053E-3,-3.0631498E-3,3.4407133E-3,9.675387E-4,-9.6753903E-4,4.3889163E-3,2.6943293E-4,-4.893813E-3,6.841654E-4,9.057139E-3],"split_indices":[45,52,2,53,17,11,7,2,64,42,9,41,0,0,0,5,5,0,42,14,9,54,8,0,0,29,0,0,0,0,45,29,29,54,41,53,56,33,56,2,29,54,56,0,0,47,0,28,33,0,3,10,58,105,58,28,0,56,0,0,0,3,41,45,0,2,0,0,0,47,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.708E3,2.528E3,1.8E2,1.083E3,1.445E3,1.19E2,6.1E1,8.9E1,9.94E2,3.7E2,1.075E3,9.7E1,2.2E1,2.8E1,3.3E1,1.9E1,7E1,1.2E2,8.74E2,3.06E2,6.4E1,3.53E2,7.22E2,8.3E1,1.4E1,1.6E1,6E0,7E0,1.2E1,5E0,6.5E1,7.8E1,4.2E1,3.6E2,5.14E2,2.28E2,7.8E1,2.9E1,3.5E1,2.56E2,9.7E1,7.3E1,6.49E2,5E0,1.1E1,3.9E1,2.6E1,4.2E1,3.6E1,9E0,3.3E1,1.4E1,3.46E2,1.42E2,3.72E2,2.26E2,2E0,7.5E1,3E0,1.7E1,1.2E1,6E0,2.9E1,2.49E2,7E0,9E1,7E0,3.1E1,4.2E1,6.29E2,2E1,2.9E1,1E1,4E1,2E0,2.4E1,1.2E1,1E1,2.3E1,8E0,6E0,4.8E1,2.98E2,1.31E2,1.1E1,1.76E2,1.96E2,1.38E2,8.8E1,6.8E1,7E0,2E0,4E0,7E0,2.2E1,1.17E2,1.32E2,8.2E1,8E0,6.02E2,2.7E1,1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[2.820341E-4,-6.144251E-3,3.573406E-2,-1.6163193E-2,1.505748E-2,7.712124E-2,2.0418983E-2,-6.4341314E-3,-6.0775254E-2,9.5607325E-2,1.02976505E-2,1.14065506E-1,-1.098812E-2,4.481024E-3,7.590156E-2,7.7521903E-3,-2.3062099E-2,-4.3109935E-2,-1.7515086E-1,1.8708764E-3,1.4266303E-1,6.81587E-2,4.440545E-3,1.3519557E-1,7.134519E-4,-7.697484E-3,2.0372026E-2,1.4732771E-2,-6.444944E-2,1.315196E-2,5.224077E-2,-9.380402E-3,3.3146963E-2,-1.2642719E-1,-1.490925E-2,-7.118995E-2,2.5917547E-2,-2.1300618E-2,-1.3827862E-1,1.0179894E-2,8.655641E-4,-1.1713222E-1,8.584661E-2,-1.5330375E-2,5.97412E-3,8.032844E-3,3.279822E-3,-2.9594412E-3,4.3578353E-3,6.358082E-3,8.727579E-3,-6.265976E-3,-4.421335E-4,7.5499415E-2,-2.9911632E-3,2.1967273E-3,-8.3430566E-2,4.3483034E-2,-8.3788574E-2,-3.5728917E-1,-7.933365E-2,-1.9258768E-3,-4.7521792E-2,-4.0776905E-1,-5.874551E-2,1.1426508E-1,-3.7804913E-2,-1.6229372E-3,-1.8673094E-1,6.5310707E-4,-1.2089859E-2,1.043574E-2,3.0591777E-3,2.266394E-2,-1.2222627E-2,1.3669271E-2,-1.1096397E-1,1.7682186E-1,4.7164977E-2,2.5868847E-4,-1.3581354E-2,-5.849613E-3,7.860139E-3,6.440143E-3,1.6470643E-3,-1.2815883E-2,-2.5639995E-3,-2.9823338E-3,-2.2165678E-2,-6.4637777E-3,5.394432E-4,-3.4253318E-3,2.4319848E-4,1.5310087E-3,-2.9456834E-3,-2.8528202E-2,-1.0189542E-3,1.1650296E-2,-3.1469376E-3,2.9259992E-3,1.0272048E-2,-3.239445E-3,3.0860791E-3,4.7335844E-3,-1.0470551E-2,-4.6514568E-4,2.1747581E-3,-1.0197393E-2,-3.2358564E-4,-3.131697E-3,9.700234E-4,5.5761257E-4,-1.1325263E-2,2.5457263E-3,1.2979112E-2,-4.655867E-3,3.3189987E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,-1,-1,47,49,51,-1,53,55,57,59,61,63,65,-1,67,-1,-1,69,71,-1,73,-1,-1,-1,-1,-1,75,-1,-1,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,-1,-1,-1,-1,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.176248E-1,4.877448E-1,2.62106E-1,6.762672E-1,2.815839E-1,3.6582857E-1,2.6936495E-1,3.0235347E-1,5.5616367E-1,1.0351166E-1,2.3590939E-1,1.6352439E-1,1.6293678E-1,1.6913404E-1,3.053868E-1,3.0138507E-1,4.953198E-1,4.728501E-1,3.063407E-1,0E0,1.8775928E-1,2.1980956E-1,3.1747133E-1,1.2960124E-1,0E0,0E0,1.7391042E-1,1.4627023E-1,1.1243436E-1,0E0,1.7011577E-1,3.5510066E-1,3.4085786E-1,4.391557E-1,2.3194307E-1,7.006622E-1,4.0403003E-1,0E0,1.6564202E-1,0E0,0E0,1.0961382E-1,1.572474E-1,0E0,1.9221567E-1,0E0,0E0,0E0,0E0,0E0,1.1922107E-1,0E0,0E0,1.4094779E-1,0E0,3.303517E-1,5.5676144E-1,2.0796502E-1,1.1356409E-1,1.8915164E-1,1.880193E-1,1.9194011E-1,1.754114E-1,4.231195E-1,3.6940622E-1,1.4675647E-1,1.3249125E-1,0E0,2.3116219E-1,0E0,0E0,0E0,0E0,2.434774E-1,3.3096546E-1,1.01295575E-1,1.2586933E-1,1.1372718E-1,1.4099056E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,41,41,42,42,44,44,50,50,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,68,68,73,73,74,74,75,75,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,-1,-1,48,50,52,-1,54,56,58,60,62,64,66,-1,68,-1,-1,70,72,-1,74,-1,-1,-1,-1,-1,76,-1,-1,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,-1,-1,-1,-1,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.753919E3,1.273801E6,1.0911331E4,3.38464E5,1.9E1,1.2819376E2,9.3058E4,5.5306E4,1.4329944E0,5.72E2,1.0655738E0,5.5544252E7,4.4200982E8,1.5410214E7,2.4394053E9,1.172E3,3.451532E8,3.83E3,1.1057693E0,1.8708764E-3,7.696402E3,3.5902756E-6,1.3314917E0,4.949877E6,7.134519E-4,-7.697484E-3,1.4E1,2.0588236E0,7.203918E9,1.315196E-2,2.6572757E10,1.2E1,4.0844156E7,1.3797468E0,7.03295E-1,4E0,4.7675516E4,-2.1300618E-2,8.896243E4,1.0179894E-2,8.655641E-4,2.9143541E-6,2E0,-1.5330375E-2,2.7E1,8.032844E-3,3.279822E-3,-2.9594412E-3,4.3578353E-3,6.358082E-3,1.6616056E7,-6.265976E-3,-4.421335E-4,1.3683E4,-2.9911632E-3,2.6777263E0,1.4229508E1,3.04386E-4,1.941926E2,9.19E4,4.312482E5,9.721951E1,8E0,6.058777E6,3.938775E4,3.9759712E-4,5.2646267E9,-1.6229372E-3,9.489796E-1,6.5310707E-4,-1.2089859E-2,1.043574E-2,3.0591777E-3,4E0,5.1586456E7,3.258768E-4,3.2703E4,3.285E3,4E0,2.5868847E-4,-1.3581354E-2,-5.849613E-3,7.860139E-3,6.440143E-3,1.6470643E-3,-1.2815883E-2,-2.5639995E-3,-2.9823338E-3,-2.2165678E-2,-6.4637777E-3,5.394432E-4,-3.4253318E-3,2.4319848E-4,1.5310087E-3,-2.9456834E-3,-2.8528202E-2,-1.0189542E-3,1.1650296E-2,-3.1469376E-3,2.9259992E-3,1.0272048E-2,-3.239445E-3,3.0860791E-3,4.7335844E-3,-1.0470551E-2,-4.6514568E-4,2.1747581E-3,-1.0197393E-2,-3.2358564E-4,-3.131697E-3,9.700234E-4,5.5761257E-4,-1.1325263E-2,2.5457263E-3,1.2979112E-2,-4.655867E-3,3.3189987E-3],"split_indices":[52,9,48,9,10,58,2,29,42,33,53,45,7,43,5,2,5,2,54,0,47,38,54,9,0,0,3,54,12,0,12,8,45,53,27,3,48,0,33,0,0,38,8,0,8,0,0,0,0,0,29,0,0,9,0,41,56,38,33,1,47,52,3,51,45,39,12,0,53,0,0,0,0,8,7,38,2,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.709E3,2.294E3,4.15E2,1.558E3,7.36E2,1.11E2,3.04E2,1.28E3,2.78E2,4E1,6.96E2,7.8E1,3.3E1,2.37E2,6.7E1,6.91E2,5.89E2,2.42E2,3.6E1,1.9E1,2.1E1,6.3E1,6.33E2,6.4E1,1.4E1,5E0,2.8E1,2.07E2,3E1,6E0,6.1E1,4.13E2,2.78E2,4.2E1,5.47E2,1.72E2,7E1,3E0,3.3E1,1.3E1,8E0,5E0,5.8E1,2E0,6.31E2,4.2E1,2.2E1,1.3E1,1.5E1,9E0,1.98E2,1.3E1,1.7E1,5.1E1,1E1,3.58E2,5.5E1,2.56E2,2.2E1,6E0,3.6E1,3.92E2,1.55E2,5E0,1.67E2,2.9E1,4.1E1,1.1E1,2.2E1,3E0,2E0,7E0,5.1E1,3.29E2,3.02E2,1.91E2,7E0,1E1,4.1E1,3.55E2,3E0,4.8E1,7E0,2.2E1,2.34E2,2E0,2E1,2E0,4E0,2.2E1,1.4E1,3.5E1,3.57E2,2.3E1,1.32E2,3E0,2E0,3E0,1.64E2,2E1,9E0,3.2E1,9E0,2E0,2E1,1.36E2,1.93E2,7E0,2.95E2,1.4E1,1.77E2,4E0,3E0,5E0,5E0,5E0,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-6.294603E-5,-1.727741E-2,1.0222398E-2,-3.505E-2,-6.1009557E-4,4.878508E-3,4.6097886E-2,-1.4355834E-2,-6.650365E-2,4.3033488E-2,-2.3789892E-2,-9.208187E-3,2.306798E-2,3.7939735E-2,7.3218797E-3,-1.0714821E-2,-1.0593722E-2,-3.7416838E-2,-1.3960508E-1,9.354763E-2,3.9831316E-3,2.1153677E-2,-4.9064547E-2,-2.3547415E-2,1.3225632E-2,3.0762877E-2,-4.251068E-2,4.3311536E-2,-1.1789063E-1,-8.442494E-2,-1.1712703E-3,4.9263656E-2,-6.965884E-2,-1.2344713E-2,-9.180333E-2,1.2020343E-1,-1.7256299E-2,5.2098878E-2,-5.1232476E-2,2.8438844E-2,-9.619698E-3,-1.3522124E-1,-3.551344E-2,1.6870071E-3,-3.4993473E-2,3.889838E-2,-2.4595294E-2,1.17102884E-1,2.31101E-2,-7.555908E-3,-2.4207536E-2,2.72482E-2,8.967151E-2,2.320947E-3,-1.2201177E-2,-1.5291064E-3,-7.7933003E-3,2.5221623E-2,-3.5875697E-2,7.0254825E-2,-7.026739E-3,1.8400339E-3,-9.681972E-2,5.9584836E-3,-1.19284086E-1,6.432693E-3,-5.710416E-4,-3.6506047E-3,6.116929E-3,-9.878926E-5,1.5100491E-1,-4.368468E-3,9.307828E-3,7.954867E-2,-8.474447E-3,-5.0137364E-3,-1.4360396E-2,8.669201E-3,-4.1204523E-2,-2.5828734E-2,-8.3619595E-2,-6.572732E-3,8.491732E-2,-9.423081E-2,6.2840013E-3,2.8359238E-3,1.8939397E-1,3.1801376E-2,-3.8619224E-2,-3.3815948E-3,1.1016147E-3,4.3890295E-3,6.05959E-4,3.337167E-3,1.4468497E-2,1.870738E-3,-4.705824E-3,-1.2885048E-3,-1.2887821E-2,5.151283E-3,-3.3096222E-3,-3.688303E-3,-1.2830272E-2,-8.990728E-3,-1.9349441E-3,-4.4528903E-3,1.5840662E-3,-2.2897387E-3,9.923408E-3,4.4260547E-3,-3.2132193E-3,1.1628698E-2,3.0631202E-3,-4.817128E-3,1.3320266E-3,-2.6121137E-3,3.1843717E-4,-1.4240765E-3,4.2147706E-3,-9.330566E-3,-1.8129586E-3,3.133464E-3,-2.1417285E-3,6.5327487E-3,1.7166693E-3,-2.3256952E-3,-1.08414E-2,9.952365E-3,-2.7765996E-5,1.0641067E-2,1.263565E-4,4.0660966E-3,1.0471422E-3,-5.5458113E-3,2.135637E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,-1,75,77,-1,79,81,83,85,87,-1,89,91,93,-1,-1,-1,-1,95,97,99,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,-1,109,111,113,-1,-1,-1,115,117,119,121,123,125,127,-1,129,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.715015E-1,2.9501513E-1,3.1936345E-1,3.1278247E-1,5.220232E-1,3.722577E-1,1.8238056E-1,2.1754895E-1,4.0089732E-1,3.527219E-1,3.8384736E-1,2.6377034E-1,3.2123747E-1,1.7419815E-1,0E0,2.0249651E-1,0E0,3.891075E-1,2.8135276E-1,2.3245257E-1,2.7365667E-1,2.030436E-1,2.4738193E-1,3.3759645E-1,3.1166616E-1,3.7259465E-1,1.3752997E-1,1.4351699E-1,1.7174295E-1,1.304593E-1,2.3538809E-1,1.6664132E-1,3.0021173E-1,0E0,2.5251913E-1,1.1911464E-1,1.4653552E-1,2.8396145E-1,1.17043674E-1,2.2518337E-1,0E0,1.1437082E-1,2.391345E-1,0E0,1.8456578E-1,4.0023184E-1,2.801306E-1,1.8127847E-1,2.818211E-1,0E0,1.3087659E-1,1.3700789E-1,1.9126758E-1,0E0,0E0,0E0,0E0,2.5337556E-1,2.2499655E-1,1.8941502E-1,0E0,0E0,2.538203E-1,0E0,1.86795E-1,0E0,0E0,0E0,0E0,1.1708765E-1,2.222237E-1,0E0,1.3320154E-1,1.1171991E-1,2.3677912E-1,0E0,0E0,0E0,1.1781296E-1,1.6007634E-1,3.2852516E-1,2.6936495E-1,2.3515922E-1,2.3040414E-1,1.2599748E-1,0E0,1.2193155E-1,2.4032572E-1,4.226345E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,57,57,58,58,59,59,62,62,64,64,69,69,70,70,72,72,73,73,74,74,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,-1,76,78,-1,80,82,84,86,88,-1,90,92,94,-1,-1,-1,-1,96,98,100,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,-1,110,112,114,-1,-1,-1,116,118,120,122,124,126,128,-1,130,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.074E3,1.2679E4,2.5074627E0,2.7682406E8,4.5168175E5,7.273333E1,2.0136518E0,4.44E2,2.5073426E0,2.3907686E7,1.5107028E-1,8.0073607E-1,5.68E2,7.3218797E-3,1.0882353E1,-1.0593722E-2,1.2E1,2.84755E5,1.0347911E7,3.0161016E7,2.03353E-6,3.83E3,1.11335E3,3.2982758E1,1.5316E4,1.47E2,6.9673E4,1.25E2,8.2E1,2.0304577E-1,2.271E3,1E0,-1.2344713E-2,2.5797562E2,3.8758245E-1,5.38732E4,2.653986E6,1.6E1,1.1662405E9,-9.619698E-3,1.4136576E6,2.97271E5,1.6870071E-3,3.5551153E5,1.96E2,2.373E3,1.7835504E-1,1E0,-7.555908E-3,9.123214E-1,2.0966542E0,1.2328733E1,2.320947E-3,-1.2201177E-2,-1.5291064E-3,-7.7933003E-3,2.59698E-6,4.5723195E6,4.072289E0,-7.026739E-3,1.8400339E-3,3.7832818E0,5.9584836E-3,2.3871907E2,6.432693E-3,-5.710416E-4,-3.6506047E-3,6.116929E-3,2.41E3,5.0713895E2,-4.368468E-3,1.775894E10,1.0976147E-1,1.02942106E11,-5.0137364E-3,-1.4360396E-2,8.669201E-3,4.247868E3,2.32661E5,6.126259E5,5.2770585E-1,2.9676E4,2.4505E4,4.162121E0,2.8359238E-3,4.3359457E10,4.7048388E0,8.874407E6,-3.3815948E-3,1.1016147E-3,4.3890295E-3,6.05959E-4,3.337167E-3,1.4468497E-2,1.870738E-3,-4.705824E-3,-1.2885048E-3,-1.2887821E-2,5.151283E-3,-3.3096222E-3,-3.688303E-3,-1.2830272E-2,-8.990728E-3,-1.9349441E-3,-4.4528903E-3,1.5840662E-3,-2.2897387E-3,9.923408E-3,4.4260547E-3,-3.2132193E-3,1.1628698E-2,3.0631202E-3,-4.817128E-3,1.3320266E-3,-2.6121137E-3,3.1843717E-4,-1.4240765E-3,4.2147706E-3,-9.330566E-3,-1.8129586E-3,3.133464E-3,-2.1417285E-3,6.5327487E-3,1.7166693E-3,-2.3256952E-3,-1.08414E-2,9.952365E-3,-2.7765996E-5,1.0641067E-2,1.263565E-4,4.0660966E-3,1.0471422E-3,-5.5458113E-3,2.135637E-3],"split_indices":[102,2,2,56,7,28,54,39,0,53,45,38,27,3,0,52,0,3,1,1,45,37,2,47,58,9,0,2,8,29,27,11,67,0,4,38,32,1,18,5,0,28,1,0,33,0,2,38,74,0,27,54,53,0,0,0,0,37,28,54,0,0,53,0,52,0,0,0,0,2,52,0,5,38,31,0,0,0,52,11,33,41,29,29,54,0,19,58,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.661E3,9.95E2,1.666E3,4.81E2,5.14E2,1.451E3,2.15E2,2.91E2,1.9E2,1.78E2,3.36E2,8.18E2,6.33E2,2.01E2,1.4E1,2.87E2,4E0,1.37E2,5.3E1,7.7E1,1.01E2,1.21E2,2.15E2,4.99E2,3.19E2,5.67E2,6.6E1,1.95E2,6E0,3.2E1,2.55E2,3.7E1,1E2,1.4E1,3.9E1,6.2E1,1.5E1,5.4E1,4.7E1,1.18E2,3E0,2.8E1,1.87E2,8.1E1,4.18E2,1.9E2,1.29E2,4.5E1,5.22E2,8E0,5.8E1,1.46E2,4.9E1,3E0,3E0,2E1,1.2E1,1.45E2,1.1E2,3.4E1,3E0,2E1,8E1,4E0,3.5E1,5.6E1,6E0,1.1E1,4E0,3.6E1,1.8E1,2.8E1,1.9E1,4.9E1,6.9E1,2.5E1,3E0,4E0,1.83E2,3.53E2,6.5E1,9.6E1,9.4E1,3.9E1,9E1,2.6E1,1.9E1,4.58E2,6.4E1,2.9E1,2.9E1,2.6E1,1.2E2,4.6E1,3E0,1.31E2,1.4E1,1.07E2,3E0,2.7E1,7E0,7.3E1,7E0,1.8E1,1.7E1,9E0,2.7E1,4E0,1.4E1,9E0,1E1,3E0,4.6E1,1.9E1,5E1,1.43E2,4E1,3.42E2,1.1E1,1.8E1,4.7E1,3.3E1,6.3E1,4.5E1,4.9E1,3E1,9E0,2E0,8.8E1,1.6E1,3E0,7.1E1,3.87E2,3.3E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-7.5653064E-5,4.819218E-3,-2.911583E-2,1.7605964E-3,6.334686E-2,-9.5471395E-3,-2.4996426E-2,-1.2136161E-2,1.4768766E-2,5.012812E-3,9.8605335E-2,-2.2179363E-2,-1.1075975E-2,1.1454574E-3,-5.2187376E-2,2.1899132E-2,-2.3409583E-2,-5.3894413E-3,2.054317E-3,5.0581647E-3,-4.5775794E-3,-4.9175028E-2,-2.29692E-3,2.1003865E-2,-1.7599018E-2,-8.87032E-2,-1.2981592E-2,5.3965304E-2,9.498208E-3,-5.7410114E-2,2.8639248E-2,-6.532234E-2,2.2435838E-2,1.0213037E-1,-1.1117651E-2,-6.346866E-4,4.9990255E-2,-5.085267E-2,1.6396571E-2,-2.0732778E-1,-6.885091E-2,-2.814371E-2,9.4912216E-2,1.2285399E-1,3.3203475E-2,-7.727915E-3,3.33904E-2,-1.2561522E-2,-4.9058154E-2,4.2185374E-2,-7.59576E-3,-7.004329E-2,4.832072E-3,4.7357148E-3,-9.929467E-4,6.722082E-3,-2.7402043E-3,8.3746435E-4,-4.2586204E-2,7.209575E-4,-8.585783E-3,3.3097416E-3,-3.7083114E-3,-1.6867602E-3,-7.81771E-3,-1.3057457E-2,9.879414E-4,-1.3451629E-2,-3.4040234E-3,-1.2405337E-3,-6.1682626E-3,-2.7626224E-3,2.1657292E-3,1.2098343E-2,1.8717676E-3,9.917709E-3,2.603371E-3,4.438887E-3,8.9410605E-4,3.8883185E-3,-7.258796E-4,2.2290077E-3,-4.329813E-3,-2.6166271E-3,7.6077883E-3,-1.8263707E-3,3.3646687E-3,-4.3422813E-3,-1.2517615E-3,-5.8266544E-3,-7.664384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,-1,-1,-1,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,-1,87,-1,-1,-1,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8138103E-1,4.1090444E-1,2.6641104E-1,3.948032E-1,2.3397744E-1,0E0,2.1510935E-1,5.617936E-1,3.0758306E-1,2.007664E-1,1.0680026E-1,2.0116444E-1,0E0,2.9592457E-1,3.7536764E-1,3.776527E-1,3.1664863E-1,0E0,0E0,0E0,0E0,1.8532097E-1,2.0101957E-1,2.4214002E-1,4.6319914E-1,3.060012E-1,2.1206619E-1,3.7493843E-1,2.8300646E-1,1.719422E-1,1.8677934E-1,1.0908061E-1,1.0106006E-1,1.1160186E-1,1.8167455E-1,6.249093E-1,4.0735283E-1,3.5790867E-1,2.5397953E-1,1.7044872E-1,3.004691E-1,2.475271E-1,1.3100271E-1,3.4069216E-1,1.7549866E-1,2.6546952E-1,4.731886E-1,0E0,1.30487E-1,1.5375102E-1,0E0,1.126495E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.9985647E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,51,51,58,58],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,-1,-1,-1,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,-1,88,-1,-1,-1,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7033327E-1,4.5723195E6,3E0,2.437E3,9E0,-9.5471395E-3,7.780377E6,2.2415E4,2.8721272E1,1.752E3,2.851369E2,1E0,-1.1075975E-2,9.658451E0,3.3720784E5,1.8174828E6,1.16E2,-5.3894413E-3,2.054317E-3,5.0581647E-3,-4.5775794E-3,4.6982746E8,5.1998E4,7E2,3.9E2,9.251576E3,4.222002E8,1.6E1,1.5106794E3,5.542E3,6.1603E4,5.5669255E6,1E0,9.83E2,2.34768E5,3.3751E4,2.7E1,3.2188842E0,1.2E1,3.7342235E6,6.13E2,1.058E3,1.032E3,1.988805E8,5.1547173E1,5E0,2.3580047E10,-1.2561522E-2,1.1879E4,9.750042E6,-7.59576E-3,5.695737E0,4.832072E-3,4.7357148E-3,-9.929467E-4,6.722082E-3,-2.7402043E-3,8.3746435E-4,6.624E3,7.209575E-4,-8.585783E-3,3.3097416E-3,-3.7083114E-3,-1.6867602E-3,-7.81771E-3,-1.3057457E-2,9.879414E-4,-1.3451629E-2,-3.4040234E-3,-1.2405337E-3,-6.1682626E-3,-2.7626224E-3,2.1657292E-3,1.2098343E-2,1.8717676E-3,9.917709E-3,2.603371E-3,4.438887E-3,8.9410605E-4,3.8883185E-3,-7.258796E-4,2.2290077E-3,-4.329813E-3,-2.6166271E-3,7.6077883E-3,-1.8263707E-3,3.3646687E-3,-4.3422813E-3,-1.2517615E-3,-5.8266544E-3,-7.664384E-4],"split_indices":[27,28,3,2,3,0,28,29,53,29,58,16,0,58,28,51,8,0,0,0,0,7,12,2,10,33,7,3,52,29,11,28,104,0,29,1,3,53,18,45,0,0,10,7,48,3,5,0,2,9,0,54,0,0,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.681E3,2.295E3,3.86E2,2.182E3,1.13E2,8E0,3.78E2,1.055E3,1.127E3,4.3E1,7E1,3.74E2,4E0,7.93E2,2.62E2,9.5E2,1.77E2,1E1,3.3E1,6.8E1,2E0,1.58E2,2.16E2,3.85E2,4.08E2,1.35E2,1.27E2,2.64E2,6.86E2,1.07E2,7E1,1.29E2,2.9E1,1.6E1,2E2,2.21E2,1.64E2,2.06E2,2.02E2,1.8E1,1.17E2,1.12E2,1.5E1,6E1,2.04E2,3.99E2,2.87E2,3E0,1.04E2,6.6E1,4E0,1.26E2,3E0,1E1,1.9E1,1.3E1,3E0,1.05E2,9.5E1,2.04E2,1.7E1,1.43E2,2.1E1,1.82E2,2.4E1,2E0,2E2,1.1E1,7E0,6.9E1,4.8E1,8E1,3.2E1,3E0,1.2E1,2.6E1,3.4E1,3.9E1,1.65E2,3E1,3.69E2,2.6E2,2.7E1,1.02E2,2E0,1.7E1,4.9E1,8.5E1,4.1E1,2.3E1,7.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-6.6464423E-4,-2.7832894E-2,4.8377756E-3,-2.0860024E-2,-1.40742E-1,1.6015174E-2,-1.3154069E-2,-1.8255502E-2,-2.3598868E-1,-2.537767E-3,-9.208126E-3,-1.5211444E-1,1.811273E-2,-3.148249E-2,9.159842E-3,1.052508E-2,-2.048962E-2,-2.12041E-2,2.3680597E-3,-2.523902E-1,2.7192412E-2,9.0135604E-2,1.4764362E-2,-2.6136182E-2,-1.4403789E-1,5.715214E-2,-9.487603E-4,3.2001864E-3,-2.6451226E-2,-1.4530828E-2,-7.2688895E-4,-6.2181046E-3,5.5540255E-3,1.2773897E-1,-2.1257337E-2,-2.0602247E-2,2.1084778E-2,-4.1227727E-3,-5.5199895E-2,-1.61742E-2,-2.988368E-3,1.2242728E-1,1.9707622E-4,-4.5340493E-2,8.882038E-3,-1.484533E-2,-1.5655364E-1,1.0207913E-2,4.3207295E-3,-6.281742E-3,2.7683792E-3,-4.218454E-2,2.7071852E-2,1.7809866E-2,1.0523898E-1,2.480049E-2,-7.295156E-2,-2.589239E-1,-4.7278676E-2,4.080509E-3,1.113729E-2,-1.31654E-1,-9.5598545E-4,1.8107383E-2,-2.89478E-3,-1.123359E-3,2.626708E-3,-8.885568E-3,4.4408715E-3,1.8578715E-3,-2.997918E-3,5.3859777E-3,3.088308E-4,2.6930822E-3,5.722779E-4,3.3773833E-3,9.856174E-3,1.575608E-3,-3.9463425E-3,4.557352E-3,-4.1248524E-3,4.3303633E-5,-2.174331E-2,-9.162349E-3,-1.8639447E-3,-1.0256394E-2,1.440142E-5,5.588564E-3,4.4393106E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,27,-1,-1,29,31,33,35,37,39,41,43,-1,45,-1,-1,-1,-1,47,49,51,53,55,57,-1,-1,59,-1,61,63,65,67,-1,-1,-1,-1,69,71,73,75,77,79,81,83,-1,-1,85,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9942542E-1,3.5134926E-1,4.4705933E-1,2.3382242E-1,1.0738385E-1,4.8625377E-1,3.4873825E-1,2.283847E-1,3.1699586E-1,0E0,0E0,3.125254E-1,3.254398E-1,2.7785712E-1,1.8683122E-1,0E0,2.193462E-1,0E0,0E0,1.2854195E-1,1.1264831E-1,2.5404033E-1,2.901315E-1,2.8639898E-1,3.109951E-1,2.3144999E-1,1.396163E-1,0E0,5.895783E-1,0E0,0E0,0E0,0E0,1.2497121E-1,1.4662251E-1,2.0383856E-1,3.012831E-1,5.1144207E-1,2.9921073E-1,0E0,0E0,1.0316393E-1,0E0,1.2442188E-1,1.6866258E-1,2.1794568E-1,2.4842411E-1,0E0,0E0,0E0,0E0,2.2725919E-1,1.08460724E-1,2.3801059E-1,1.2862909E-1,1.5987693E-1,1.7658296E-1,3.5525265E-1,2.1767321E-1,0E0,0E0,1.4160173E-1,0E0,2.0105559E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,33,33,34,34,35,35,36,36,37,37,38,38,41,41,43,43,44,44,45,45,46,46,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,63,63],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,28,-1,-1,30,32,34,36,38,40,42,44,-1,46,-1,-1,-1,-1,48,50,52,54,56,58,-1,-1,60,-1,62,64,66,68,-1,-1,-1,-1,70,72,74,76,78,80,82,84,-1,-1,86,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.08E2,3.1E1,1.4540612E7,1.9483356E1,3.90106E0,1.198E5,6.624E3,8.974133E9,1.7E1,-2.537767E-3,-9.208126E-3,3.05402E5,4.09E2,4.387561E1,2.060792E8,1.052508E-2,1.1057693E0,-2.12041E-2,2.3680597E-3,1.2684739E1,1.341E3,9.887221E5,7.805608E4,8.292435E5,2.90002E5,7.535486E0,3.1546112E-2,3.2001864E-3,8.581E3,-1.4530828E-2,-7.2688895E-4,-6.2181046E-3,5.5540255E-3,1.5043668E0,3.6020834E2,9.277255E6,1.3446785E7,2.8035452E11,1.6527965E7,-1.61742E-2,-2.988368E-3,1.7324902E8,1.9707622E-4,1E0,8.342E3,2.6659616E2,7.016421E5,1.0207913E-2,4.3207295E-3,-6.281742E-3,2.7683792E-3,2.1132307E0,1.1994927E7,1.2599255E9,1.9925156E6,1E0,5.107429E1,3.5259784E2,1.8122449E2,4.080509E-3,1.113729E-2,1.43E2,-9.5598545E-4,5.473125E3,-2.89478E-3,-1.123359E-3,2.626708E-3,-8.885568E-3,4.4408715E-3,1.8578715E-3,-2.997918E-3,5.3859777E-3,3.088308E-4,2.6930822E-3,5.722779E-4,3.3773833E-3,9.856174E-3,1.575608E-3,-3.9463425E-3,4.557352E-3,-4.1248524E-3,4.3303633E-5,-2.174331E-2,-9.162349E-3,-1.8639447E-3,-1.0256394E-2,1.440142E-5,5.588564E-3,4.4393106E-4],"split_indices":[2,8,45,54,56,7,2,19,0,0,0,1,29,53,7,0,54,0,0,56,0,50,28,51,1,53,38,0,29,0,0,0,0,53,4,9,45,31,45,0,0,7,0,64,0,52,32,0,0,0,0,53,9,31,28,73,48,52,52,0,0,3,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E3,4.49E2,2.221E3,4.24E2,2.5E1,1.37E3,8.51E2,4.2E2,4E0,1E1,1.5E1,1.6E1,1.354E3,4.67E2,3.84E2,3E0,4.17E2,2E0,2E0,1E1,6E0,5.9E1,1.295E3,4.47E2,2E1,6.6E1,3.18E2,2.6E1,3.91E2,8E0,2E0,2E0,4E0,4.4E1,1.5E1,1.96E2,1.099E3,2.55E2,1.92E2,5E0,1.5E1,2.9E1,3.7E1,5.7E1,2.61E2,3.6E2,3.1E1,1.2E1,3.2E1,6E0,9E0,1.35E2,6.1E1,1.059E3,4E1,1.8E2,7.5E1,6E0,1.86E2,2.3E1,6E0,1.2E1,4.5E1,2.31E2,3E1,3.21E2,3.9E1,2.8E1,3E0,2.7E1,1.08E2,1.1E1,5E1,1.39E2,9.2E2,3.1E1,9E0,1.68E2,1.2E1,5E0,7E1,3E0,3E0,9E0,1.77E2,7E0,5E0,1.8E1,2.13E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-8.383659E-4,-2.381538E-2,6.1179753E-3,-1.3333205E-2,-8.019522E-2,1.08433785E-2,-3.7119344E-2,8.672287E-2,-1.7890887E-2,-1.1045968E-1,6.698501E-3,8.460512E-2,8.74394E-3,-1.0223128E-1,-1.0948238E-3,-2.9623266E-3,1.314313E-1,-1.3693681E-2,-5.4295813E-3,-5.5662985E-4,-1.4319001E-1,-1.9401792E-3,4.132371E-3,1.05932295E-1,-4.4540158E-3,-1.3686826E-2,1.6630298E-2,-2.4129245E-3,-9.103278E-3,9.93876E-4,9.57697E-3,-5.9933453E-3,-6.213832E-2,3.5068055E-3,-1.6676763E-1,6.617293E-3,2.236415E-2,-2.8822197E-2,3.196563E-2,3.1019554E-2,2.366742E-3,2.2252256E-3,-5.589968E-2,-1.1611877E-1,-4.6391245E-3,5.1495857E-3,-1.8072762E-1,-1.6539942E-3,8.297093E-3,-1.3377881E-2,-2.5868315E-2,3.7506938E-2,-9.15219E-3,2.809833E-2,5.9090178E-3,-6.647859E-3,4.126814E-2,-9.513866E-4,1.5535924E-3,-4.5110825E-3,2.119393E-3,-1.9175769E-3,-1.3004711E-2,-6.443644E-3,1.1189374E-3,-1.4128609E-2,-6.3748513E-3,-2.5563754E-4,-2.3212074E-3,2.3278429E-3,-4.0185205E-3,1.9349075E-3,3.481105E-4,8.588709E-4,-9.1752154E-4,1.524389E-3,7.682512E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,-1,-1,35,-1,37,39,-1,-1,-1,-1,41,43,-1,45,-1,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,-1,67,69,-1,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2642817E-1,3.6534044E-1,4.188371E-1,2.4050803E-1,2.558297E-1,2.852835E-1,1.8519777E-1,1.6091278E-1,1.996319E-1,2.3125279E-1,1.01191156E-1,2.0303515E-1,3.181116E-1,1.544016E-1,0E0,0E0,1.324057E-1,1.7943284E-1,0E0,0E0,2.877499E-1,0E0,0E0,1.2085658E-1,0E0,3.242938E-1,2.7289838E-1,0E0,0E0,0E0,0E0,1.7116058E-1,2.0431426E-1,0E0,2.0706272E-1,0E0,1.2618256E-1,2.5325692E-1,1.5179282E-1,1.7515576E-1,2.3487964E-1,2.402726E-1,2.3046452E-1,3.911749E-1,1.2319624E-1,0E0,2.1649992E-1,0E0,0E0,0E0,1.6167752E-1,1.5907702E-1,0E0,1.6450304E-1,0E0,1.6764581E-1,1.3657723E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,20,20,23,23,25,25,26,26,31,31,32,32,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,50,50,51,51,53,53,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,-1,-1,36,-1,38,40,-1,-1,-1,-1,42,44,-1,46,-1,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,-1,68,70,-1,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8122449E2,1.15062E5,6.747114E7,6E0,6.172708E2,3.921901E6,6.7628815E2,1.0119178E10,1E0,1.5756359E0,1.6219908E5,1.6427443E2,3.9E2,9.18292E-1,-1.0948238E-3,-2.9623266E-3,1.4925373E-1,1.4136576E6,-5.4295813E-3,-5.5662985E-4,3.120286E6,-1.9401792E-3,4.132371E-3,2.5534486E4,-4.4540158E-3,4.264897E6,2.0483453E-2,-2.4129245E-3,-9.103278E-3,9.93876E-4,9.57697E-3,4.5586734E0,5.2921E4,3.5068055E-3,3.84E2,6.617293E-3,5.4E1,1.1057693E0,1E0,8.581219E6,3.4409692E3,1.6171625E2,3.3025316E5,4E0,8E0,5.1495857E-3,7.359468E8,-1.6539942E-3,8.297093E-3,-1.3377881E-2,1.0666667E1,3.29E2,-9.15219E-3,8.959817E7,5.9090178E-3,1.9577E4,4.516183E1,-9.513866E-4,1.5535924E-3,-4.5110825E-3,2.119393E-3,-1.9175769E-3,-1.3004711E-2,-6.443644E-3,1.1189374E-3,-1.4128609E-2,-6.3748513E-3,-2.5563754E-4,-2.3212074E-3,2.3278429E-3,-4.0185205E-3,1.9349075E-3,3.481105E-4,8.588709E-4,-9.1752154E-4,1.524389E-3,7.682512E-3],"split_indices":[52,29,45,3,55,5,52,19,14,54,32,54,10,27,0,0,56,28,0,0,45,0,0,33,0,9,39,0,0,0,0,56,1,0,0,0,0,54,84,28,52,55,50,8,3,0,5,0,0,0,58,10,0,12,0,9,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.666E3,6.19E2,2.047E3,5.23E2,9.6E1,1.846E3,2.01E2,2.2E1,5.01E2,7.1E1,2.5E1,5E1,1.796E3,3.5E1,1.66E2,5E0,1.7E1,4.81E2,2E1,1.8E1,5.3E1,1.6E1,9E0,4.5E1,5E0,4.67E2,1.329E3,2.3E1,1.2E1,7E0,1E1,4.16E2,6.5E1,5E0,4.8E1,3.2E1,1.3E1,3.51E2,1.16E2,6.61E2,6.68E2,3.58E2,5.8E1,3.3E1,3.2E1,2E0,4.6E1,1E1,3E0,3E0,3.48E2,1.14E2,2E0,6.42E2,1.9E1,5.43E2,1.25E2,2.07E2,1.51E2,4.2E1,1.6E1,2.3E1,1E1,5E0,2.7E1,1.2E1,3.4E1,1.83E2,1.65E2,1.05E2,9E0,4.03E2,2.39E2,1.83E2,3.6E2,1.17E2,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[2.0381191E-3,-5.4262234E-3,2.3171626E-2,-1.0532526E-3,-4.1642655E-2,5.476456E-2,8.906531E-3,-2.8416352E-2,5.0133783E-3,-4.6187565E-2,5.8473013E-3,8.3553664E-2,1.7095135E-3,7.017449E-4,6.978237E-2,-1.3026168E-2,-9.329572E-2,2.8296756E-2,-7.68294E-3,1.3202975E-2,-5.5169497E-2,1.1442048E-1,3.923786E-2,-1.1847753E-2,1.1650795E-2,1.1413612E-2,-4.4799294E-2,1.8347056E-1,3.8607568E-2,-6.913069E-3,-6.619193E-3,-3.1653725E-2,-1.0913822E-2,-9.350875E-2,3.2447968E-2,4.409743E-3,-1.1562681E-2,8.295782E-3,-2.9115638E-2,-9.267178E-2,-1.4588383E-3,1.5737617E-3,6.476988E-3,-6.881474E-4,9.51889E-2,5.6499415E-3,-8.467255E-3,5.8892958E-2,-2.0430556E-3,-1.1568737E-1,-1.6405651E-2,4.0393337E-3,1.4624676E-2,-9.79164E-3,3.1906976E-3,1.0181213E-2,-1.0171157E-2,8.849488E-4,-4.595159E-3,-7.96166E-4,-1.3155796E-2,6.507694E-2,1.780852E-2,-9.987941E-2,-7.971653E-3,-7.915936E-2,6.0278913E-3,-1.3066909E-1,5.1505363E-4,-4.301403E-4,7.6312474E-3,1.508409E-3,-2.3665496E-3,-8.3361035E-3,9.828358E-2,5.066466E-3,-7.4830525E-2,-4.332845E-4,-7.7475826E-3,8.130309E-3,-2.9410856E-2,4.3592092E-5,-3.3791366E-3,3.4702176E-3,-6.067235E-3,4.6111393E-4,4.4951793E-3,2.3385885E-3,-7.3384857E-3,1.4848536E-3,-6.9951016E-4,-6.713532E-3,7.013925E-4,-3.568548E-3,-9.493471E-3,4.8502125E-3,-1.9795166E-3,1.3994111E-3,7.227478E-3,4.8624116E-4,-5.0058644E-3,1.8276964E-4,-6.287046E-3,-7.57074E-4,-9.6877925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,-1,55,57,-1,59,61,-1,63,-1,65,67,-1,-1,-1,-1,69,-1,71,73,75,77,79,-1,-1,-1,-1,-1,81,-1,-1,-1,-1,83,85,87,89,91,-1,93,-1,-1,-1,-1,-1,95,97,99,101,-1,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.200502E-1,3.1163493E-1,3.1273225E-1,2.9180768E-1,1.6340625E-1,3.2977718E-1,2.3969816E-1,3.1730798E-1,4.2560473E-1,1.111508E-1,0E0,1.8701458E-1,1.9356449E-1,2.071694E-1,1.9493642E-1,2.169666E-1,4.9888635E-1,2.5989965E-1,3.6179802E-1,1.9465563E-1,1.6396332E-1,1.3483107E-1,1.7833057E-1,0E0,1.6142835E-1,2.1990669E-1,1.611365E-1,1.1925146E-1,3.268842E-1,0E0,1.9526389E-1,1.403786E-1,0E0,2.322625E-1,2.3385477E-1,0E0,2.8400254E-1,0E0,1.8835077E-1,2.804305E-1,0E0,0E0,0E0,0E0,1.9525173E-1,0E0,1.06291115E-1,2.020821E-1,1.3958234E-1,1.1158103E-1,1.4784586E-1,0E0,0E0,0E0,0E0,0E0,1.640154E-1,0E0,0E0,0E0,0E0,2.2442752E-1,2.1124339E-1,2.8519088E-1,2.2388569E-1,1.0509431E-1,0E0,1.8251652E-1,0E0,0E0,0E0,0E0,0E0,1.0907991E-1,1.7006871E-1,1.381993E-1,1.0704996E-1,0E0,0E0,0E0,1.2849063E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,30,30,31,31,33,33,34,34,36,36,38,38,39,39,44,44,46,46,47,47,48,48,49,49,50,50,56,56,61,61,62,62,63,63,64,64,65,65,67,67,73,73,74,74,75,75,76,76,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,-1,56,58,-1,60,62,-1,64,-1,66,68,-1,-1,-1,-1,70,-1,72,74,76,78,80,-1,-1,-1,-1,-1,82,-1,-1,-1,-1,84,86,88,90,92,-1,94,-1,-1,-1,-1,-1,96,98,100,102,-1,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0989723E3,2.8091298E7,9.517604E6,9.721951E1,9.637832E1,2.3523688E0,8.791777E3,2.5943396E0,9.538462E0,2.090909E0,5.8473013E-3,6.6E1,1.2375429E8,4.1755924E7,4.0601485E6,6E0,3.4246575E-3,4.034E3,8E-1,6.5029144E5,1.1328599E3,1.1E1,2.47E2,-1.1847753E-2,1.4523809E0,3.5229592E0,2.478592E0,3.5E1,5.388794E6,-6.913069E-3,8.974133E9,2.2174756E4,-1.0913822E-2,9.51505E-1,2.3629358E6,4.409743E-3,3.3870136E7,8.295782E-3,1.775894E10,1.5669752E-2,-1.4588383E-3,1.5737617E-3,6.476988E-3,-6.881474E-4,2.5787393E3,5.6499415E-3,3.7129E4,8.079531E4,2.9833334E1,1.5E1,9.042859E4,4.0393337E-3,1.4624676E-2,-9.79164E-3,3.1906976E-3,1.0181213E-2,2.9446045E-1,8.849488E-4,-4.595159E-3,-7.96166E-4,-1.3155796E-2,1E0,4.5723195E6,1.6118722E0,1.5476E4,2.183847E0,6.0278913E-3,5.93E2,5.1505363E-4,-4.301403E-4,7.6312474E-3,1.508409E-3,-2.3665496E-3,2.01E2,1.5190727E5,1.4602E4,1E0,-4.332845E-4,-7.7475826E-3,8.130309E-3,1.2367184E3,4.3592092E-5,-3.3791366E-3,3.4702176E-3,-6.067235E-3,4.6111393E-4,4.4951793E-3,2.3385885E-3,-7.3384857E-3,1.4848536E-3,-6.9951016E-4,-6.713532E-3,7.013925E-4,-3.568548E-3,-9.493471E-3,4.8502125E-3,-1.9795166E-3,1.3994111E-3,7.227478E-3,4.8624116E-4,-5.0058644E-3,1.8276964E-4,-6.287046E-3,-7.57074E-4,-9.6877925E-3],"split_indices":[52,45,45,52,56,42,52,56,58,56,0,8,7,50,47,52,57,9,53,28,4,3,8,0,54,54,54,8,47,0,19,47,0,58,45,0,7,0,5,38,0,0,0,0,55,0,9,28,53,3,28,0,0,0,0,0,39,0,0,0,0,84,28,53,9,54,0,0,0,0,0,0,0,0,33,0,16,0,0,0,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.661E3,1.967E3,6.94E2,1.756E3,2.11E2,2.15E2,4.79E2,3.18E2,1.438E3,2.06E2,5E0,1.39E2,7.6E1,4.23E2,5.6E1,2.58E2,6E1,5.07E2,9.31E2,2.7E1,1.79E2,8.1E1,5.8E1,2E0,7.4E1,3.43E2,8E1,1.1E1,4.5E1,1.1E1,2.47E2,4.2E1,1.8E1,1.6E1,4.91E2,3.4E1,8.97E2,5E0,2.2E1,7E1,1.09E2,1.7E1,6.4E1,3E1,2.8E1,1.1E1,6.3E1,7.5E1,2.68E2,2.2E1,5.8E1,7E0,4E0,4E0,4.1E1,3E0,2.44E2,2.4E1,1.8E1,1.2E1,4E0,1.51E2,3.4E2,3.4E1,8.63E2,1.7E1,5E0,5.1E1,1.9E1,1.1E1,1.7E1,3.2E1,3.1E1,2.8E1,4.7E1,2.45E2,2.3E1,7E0,1.5E1,3E0,5.5E1,2.07E2,3.7E1,1.46E2,5E0,3.08E2,3.2E1,9E0,2.5E1,1.25E2,7.38E2,1E1,7E0,2.9E1,2.2E1,6E0,2.2E1,2.1E1,2.6E1,2.35E2,1E1,1E1,1.3E1,5.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-1.991247E-3,1.7424638E-3,-3.3713963E-2,-2.1602359E-2,7.835268E-3,-6.872339E-2,-9.823067E-3,-1.8414395E-2,-1.1464835E-2,2.5429338E-2,1.6627641E-4,-9.987942E-2,-9.366645E-3,2.3047533E-3,-9.752515E-4,-5.843912E-3,-6.9958024E-2,7.188974E-2,1.0170992E-2,-2.5457695E-2,9.561559E-3,-1.5996728E-3,-1.419777E-1,3.8983996E-3,-2.0275114E-3,-1.1750292E-2,1.5404576E-1,-3.354256E-2,-1.6042873E-1,8.458685E-2,-9.373666E-4,-1.096299E-1,1.505399E-2,-1.3602316E-1,-1.731475E-2,1.1065564E-2,-1.9677873E-1,-1.13121234E-1,-1.2513735E-2,-2.5238995E-3,-7.512916E-2,-2.7870315E-3,9.303648E-3,-9.323772E-3,-1.7910954E-2,-2.4597889E-1,-2.3646594E-3,7.326807E-2,9.3003865E-3,2.017359E-3,-7.6415986E-3,-3.1508043E-2,2.804504E-2,-9.609712E-3,-5.466474E-2,-3.6294719E-3,-8.542633E-2,9.055196E-3,5.8110207E-3,-1.2782283E-4,-1.6363896E-2,-6.594157E-3,7.357751E-4,-2.3938161E-3,4.22373E-4,3.7857096E-3,-5.1693553E-3,-3.2443067E-3,1.2123494E-3,-3.2135076E-3,-1.4603035E-2,9.367656E-3,2.9174478E-3,1.4576565E-4,-5.436594E-3,1.703158E-3,-2.8631734E-3,-8.4158825E-3,2.7257449E-3,1.2665856E-4,-8.149095E-3,9.83604E-4,-6.5864776E-3,6.037901E-4,-2.9559324E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,25,27,29,31,33,35,-1,37,-1,-1,39,41,43,45,47,-1,49,51,53,55,57,59,61,-1,63,65,-1,-1,-1,67,69,-1,71,-1,-1,-1,73,75,-1,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1381541E-1,3.3740953E-1,2.3268238E-1,3.36503E-1,2.537968E-1,2.0803773E-1,1.0299062E-1,3.1383643E-1,0E0,4.0405583E-1,3.158594E-1,2.0235068E-1,1.2315136E-1,0E0,0E0,3.710584E-1,3.0759946E-1,1.6540086E-1,2.5378E-1,3.1456053E-1,3.0010077E-1,0E0,1.2538481E-1,0E0,0E0,2.20849E-1,1.3082439E-1,1.703226E-1,2.4436927E-1,1.3961786E-1,0E0,1.3554229E-1,2.519129E-1,1.152409E-1,3.0635935E-1,2.1039492E-1,1.9599378E-1,1.2208179E-1,0E0,1.7864276E-1,2.513592E-1,0E0,0E0,0E0,1.3957675E-1,1.3770223E-1,0E0,1.6030866E-1,0E0,0E0,0E0,2.582124E-1,2.1674252E-1,0E0,1.7210701E-1,2.8778517E-1,3.064998E-1,2.3662847E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,44,44,45,45,47,47,51,51,52,52,54,54,55,55,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,26,28,30,32,34,36,-1,38,-1,-1,40,42,44,46,48,-1,50,52,54,56,58,60,62,-1,64,66,-1,-1,-1,68,70,-1,72,-1,-1,-1,74,76,-1,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.9527026E-1,1.5561539E2,1E0,5.0363636E1,1.683274E0,9.357879E2,1.9598837E2,9.0067856E5,-1.1464835E-2,3.3253515E6,1.954E3,8.3306855E6,1E0,2.3047533E-3,-9.752515E-4,2.6713815E-1,6.699017E6,2.0494426E7,3.640909E6,2.0979407E-4,2.1470589E1,-1.5996728E-3,2.5882E4,3.8983996E-3,-2.0275114E-3,6.3251557E0,7.68E2,8.732036E1,1.05E2,2.4859156E1,-9.373666E-4,4.685271E8,1.5497989E5,6.3157897E0,4.8E1,4.5505118E2,8E0,2.155E3,-1.2513735E-2,4.7E2,1E0,-2.7870315E-3,9.303648E-3,-9.323772E-3,2.028994E8,1.40673E5,-2.3646594E-3,6.977611E5,9.3003865E-3,2.017359E-3,-7.6415986E-3,9.188401E4,2.3523688E0,-9.609712E-3,5.695737E0,2.6777263E0,9.2699325E5,1.9162654E7,5.8110207E-3,-1.2782283E-4,-1.6363896E-2,-6.594157E-3,7.357751E-4,-2.3938161E-3,4.22373E-4,3.7857096E-3,-5.1693553E-3,-3.2443067E-3,1.2123494E-3,-3.2135076E-3,-1.4603035E-2,9.367656E-3,2.9174478E-3,1.4576565E-4,-5.436594E-3,1.703158E-3,-2.8631734E-3,-8.4158825E-3,2.7257449E-3,1.2665856E-4,-8.149095E-3,9.83604E-4,-6.5864776E-3,6.037901E-4,-2.9559324E-3],"split_indices":[27,52,16,46,53,55,52,50,0,45,2,45,75,0,0,57,45,9,45,38,46,0,10,0,0,54,2,55,52,58,0,31,28,56,8,56,8,0,0,1,75,0,0,0,7,1,0,45,0,0,0,28,41,0,54,41,45,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.648E3,2.37E3,2.78E2,4.9E2,1.88E3,1.12E2,1.66E2,4.84E2,6E0,5.7E2,1.31E3,7.3E1,3.9E1,2.5E1,1.41E2,3.9E2,9.4E1,1.4E2,4.3E2,3.51E2,9.59E2,2.9E1,4.4E1,1E1,2.9E1,3.77E2,1.3E1,6.8E1,2.6E1,1.23E2,1.7E1,1.6E1,4.14E2,2.3E1,3.28E2,9.53E2,6E0,3.7E1,7E0,3.3E2,4.7E1,2E0,1.1E1,5E0,6.3E1,1.4E1,1.2E1,1.13E2,1E1,4E0,1.2E1,9E1,3.24E2,1.2E1,1.1E1,2.74E2,5.4E1,9.37E2,1.6E1,3E0,3E0,3.1E1,6E0,6.3E1,2.67E2,8E0,3.9E1,2.9E1,3.4E1,4E0,1E1,9E0,1.04E2,6.4E1,2.6E1,2.99E2,2.5E1,5E0,6E0,2.65E2,9E0,1.8E1,3.6E1,8.93E2,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.3315783E-5,-1.3907322E-2,9.146604E-3,-2.8851056E-3,-3.9034557E-2,2.154681E-2,-7.703026E-3,-1.1564173E-2,4.1380424E-2,-6.0566366E-2,3.139792E-4,5.509166E-3,1.7931344E-2,-1.4888161E-2,4.7808614E-2,-5.2741193E-3,-1.5114781E-1,5.4383982E-2,-2.7638033E-1,-9.597695E-3,-9.808754E-2,3.056069E-3,-2.2315942E-2,4.049813E-2,7.697429E-3,-1.1866749E-2,-1.0935853E-1,3.463327E-3,-3.4684192E-2,-1.6731056E-3,-1.21814E-1,-9.2970945E-2,-1.3299034E-2,8.3049955E-3,4.0654626E-2,-1.937949E-2,-2.6805138E-3,3.7855743E-3,-4.627143E-2,-6.9764994E-2,-1.6380903E-1,3.6925152E-3,-1.8064347E-3,8.157249E-2,1.4448547E-2,6.0080215E-3,1.2918135E-2,6.8916874E-3,-2.8133843E-2,-1.49690155E-2,-1.6441636E-3,-6.2057124E-3,4.0781097E-3,-4.2402903E-3,1.0082969E-2,-1.9852166E-1,3.0979565E-3,-1.4670172E-1,2.855686E-4,-5.101633E-2,5.7151925E-2,-7.754389E-3,-2.230061E-2,-8.657628E-2,6.452387E-3,-2.2504702E-1,-2.4665117E-3,4.2670045E-2,6.021785E-3,2.5528062E-2,-2.5405772E-3,1.3165819E-2,-4.587088E-2,5.4543737E-2,-1.1871515E-2,5.1311734E-3,-3.1933732E-2,4.7944504E-4,-1.2528927E-3,-2.3280394E-3,-1.7495116E-2,-3.837058E-3,-1.6375158E-2,-4.7037234E-3,6.0794335E-3,1.162812E-3,5.3882776E-3,-7.1586575E-3,1.2103326E-4,-5.6969575E-3,-6.46589E-4,-1.2497771E-2,5.1752344E-4,2.6203291E-3,-4.8257895E-3,3.858433E-3,5.371406E-4,4.3935038E-4,6.2561785E-3,-6.917418E-3,-1.1128811E-3,3.986624E-3,-3.379485E-5,-1.1942681E-3,3.0379645E-3,-3.1286587E-3,-8.285648E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,53,55,57,-1,-1,59,-1,-1,-1,61,63,65,-1,-1,67,69,71,-1,73,75,-1,-1,-1,-1,77,-1,79,-1,81,-1,83,85,-1,87,89,-1,91,-1,93,-1,95,-1,97,99,101,103,-1,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4531558E-1,2.97894E-1,3.4282574E-1,2.88232E-1,2.779155E-1,3.1735605E-1,2.783174E-1,5.4970163E-1,5.232692E-1,4.0448827E-1,1.6955213E-1,0E0,2.1008062E-1,1.7481492E-1,1.6425322E-1,2.5201285E-1,1.7296183E-1,1.8742391E-1,1.187225E-1,2.994904E-1,2.1566272E-1,0E0,1.3614592E-1,3.0269286E-1,2.7639604E-1,1.8297589E-1,2.7894515E-1,0E0,2.2710933E-1,3.1905323E-1,2.7035284E-1,1.0719068E-1,0E0,0E0,1.6624661E-1,0E0,0E0,0E0,1.7658001E-1,3.1075814E-1,2.359953E-1,0E0,0E0,1.8364185E-1,1.3222757E-1,2.326906E-1,0E0,2.5010282E-1,1.684283E-1,0E0,0E0,0E0,0E0,1.8083645E-1,0E0,3.072927E-1,0E0,1.4158186E-1,0E0,1.5526962E-1,1.6468695E-1,0E0,1.7404506E-1,1.9006288E-1,0E0,1.9766366E-1,0E0,1.0917481E-1,0E0,1.1804375E-1,0E0,2.5670156E-1,1.6619587E-1,1.2532903E-1,1.9935748E-1,0E0,1.5182093E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,34,34,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,53,53,55,55,57,57,59,59,60,60,62,62,63,63,65,65,67,67,69,69,71,71,72,72,73,73,74,74,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,54,56,58,-1,-1,60,-1,-1,-1,62,64,66,-1,-1,68,70,72,-1,74,76,-1,-1,-1,-1,78,-1,80,-1,82,-1,84,86,-1,88,90,-1,92,-1,94,-1,96,-1,98,100,102,104,-1,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2485715E2,4.61E2,1.6837112E7,8.8365E4,2.074E3,2.368E0,5.528107E3,3.8977896E2,3.099779E6,2.1818182E0,1E1,5.509166E-3,4.0208E4,4.1585636E1,1.4285097E1,2.829581E10,3.8833074E4,3.114E3,8E0,1.3298E4,3.6604617E-2,3.056069E-3,1.1451E4,4.6E0,1.662639E7,5.673198E5,6.203E3,3.463327E-3,3.3047943E0,2.5551208E10,8.87E2,4.6847186E2,-1.3299034E-2,8.3049955E-3,2.66E2,-1.937949E-2,-2.6805138E-3,3.7855743E-3,2.428818E6,1.9505986E6,2.3449652E0,3.6925152E-3,-1.8064347E-3,1.5E1,1.9221336E1,2.5482938E0,1.2918135E-2,1.3332031E1,1.22E2,-1.49690155E-2,-1.6441636E-3,-6.2057124E-3,4.0781097E-3,1E0,1.0082969E-2,1E0,3.0979565E-3,3.5E1,2.855686E-4,1.30727E5,1.1160287E7,-7.754389E-3,1.3841E4,6.5029144E5,6.452387E-3,3.8935616E8,-2.4665117E-3,1.9232916E7,6.021785E-3,1.4022468E6,-2.5405772E-3,5.3534385E3,6.972973E0,2.8316305E0,1E0,5.1311734E-3,3.6709285E6,4.7944504E-4,-1.2528927E-3,-2.3280394E-3,-1.7495116E-2,-3.837058E-3,-1.6375158E-2,-4.7037234E-3,6.0794335E-3,1.162812E-3,5.3882776E-3,-7.1586575E-3,1.2103326E-4,-5.6969575E-3,-6.46589E-4,-1.2497771E-2,5.1752344E-4,2.6203291E-3,-4.8257895E-3,3.858433E-3,5.371406E-4,4.3935038E-4,6.2561785E-3,-6.917418E-3,-1.1128811E-3,3.986624E-3,-3.379485E-5,-1.1942681E-3,3.0379645E-3,-3.1286587E-3,-8.285648E-4],"split_indices":[52,0,45,1,2,58,52,4,51,56,3,0,9,53,54,31,33,29,3,9,27,0,12,54,45,32,29,0,35,31,2,4,0,0,0,0,0,0,45,48,53,0,0,3,53,42,0,56,10,0,0,0,0,16,0,8,0,3,0,29,45,0,9,28,0,7,0,48,0,51,0,4,56,54,96,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.714E3,1.075E3,1.639E3,7.48E2,3.27E2,9.44E2,6.95E2,6.26E2,1.22E2,2.11E2,1.16E2,3.4E1,9.1E2,6.16E2,7.9E1,6E2,2.6E1,1.18E2,4E0,9E1,1.21E2,3E1,8.6E1,2.83E2,6.27E2,5.98E2,1.8E1,6.1E1,1.8E1,5.83E2,1.7E1,1.9E1,7E0,1.1E1,1.07E2,2E0,2E0,2.6E1,6.4E1,8.6E1,3.5E1,1.1E1,7.5E1,1.09E2,1.74E2,6.24E2,3E0,2.78E2,3.2E2,4E0,1.4E1,1E1,8E0,5.77E2,6E0,1.2E1,5E0,1.2E1,7E0,1.6E1,9.1E1,1E1,5.4E1,8E1,6E0,2.2E1,1.3E1,5.9E1,5E1,1.5E2,2.4E1,5.49E2,7.5E1,7.8E1,2E2,8E0,3.12E2,3.5E2,2.27E2,7E0,5E0,1E1,2E0,1.3E1,3E0,5.8E1,3.3E1,8E0,4.6E1,5.5E1,2.5E1,1.9E1,3E0,5.5E1,4E0,3E1,1.2E2,5.32E2,1.7E1,1.3E1,6.2E1,5.1E1,2.7E1,1.71E2,2.9E1,9.4E1,2.18E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[-1.637545E-3,-1.0796604E-2,1.2243679E-2,-2.8717825E-2,-6.9582346E-4,2.9338853E-2,9.0742606E-4,-1.6665215E-2,-8.119836E-2,4.7403865E-2,-9.961938E-3,2.3064915E-2,1.1791871E-1,-3.8015816E-2,1.0059724E-2,1.9722485E-3,-5.8109116E-2,-9.60461E-2,5.6635416E-3,1.2265814E-1,1.0681155E-2,-2.2164252E-2,1.1685215E-2,2.6454529E-2,-1.5282848E-2,2.4513416E-1,1.0144729E-3,-7.5179E-4,-8.888109E-2,-1.20399535E-1,1.2837485E-2,1.5115305E-2,-7.748733E-2,-1.7986799E-2,-1.166554E-1,-1.1795263E-1,-2.3855878E-2,9.092356E-3,6.0330473E-2,2.5494792E-2,-1.3034606E-1,-2.434309E-3,-4.5429908E-2,6.82748E-3,5.57928E-3,2.9383184E-2,-6.748306E-3,9.037837E-4,1.3816393E-2,-3.3347495E-2,2.962606E-3,-4.6264455E-3,-1.3725114E-1,6.5675895E-3,-9.236528E-3,2.6940256E-3,3.73667E-2,-4.4029806E-2,3.2174908E-2,-1.4459307E-2,-2.649444E-3,3.3549555E-3,-3.9747346E-2,-1.4104137E-1,2.8487018E-3,-6.172395E-3,5.5805114E-3,4.55719E-4,-7.6263743E-3,-4.346701E-3,4.169664E-3,1.4890461E-2,7.225985E-3,-1.065625E-2,4.146058E-3,-5.138739E-2,1.7203258E-2,-1.5376706E-1,-3.900131E-2,1.6335666E-2,-4.412293E-2,3.156216E-2,-9.066789E-3,-7.3993853E-3,-4.2591052E-4,2.6852307E-3,-4.917407E-3,-1.4322159E-2,-5.010318E-3,1.1849131E-2,-3.5926692E-2,1.2044706E-2,3.9980677E-3,-1.3024671E-3,-1.1360616E-2,1.9481808E-3,-3.7995146E-3,-5.9472066E-3,5.1132636E-4,-1.1180117E-2,-4.354136E-3,4.288706E-3,-3.3131114E-4,-1.7230488E-2,-1.5415894E-3,-2.3342695E-3,1.4440342E-3,2.0779527E-3,-1.0127966E-2,-2.3714113E-3,1.000224E-3,4.4066127E-4,5.142512E-3,2.981119E-4,-5.3758873E-3,-1.1740755E-4,2.2152297E-3,7.5036474E-4,-6.087997E-3,4.9505937E-3,-3.1861402E-3,-1.7477366E-3,2.2101083E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,-1,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,79,81,-1,-1,-1,83,-1,85,87,-1,-1,89,91,93,95,-1,-1,-1,97,99,-1,-1,-1,-1,-1,-1,-1,101,-1,-1,-1,103,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,-1,117,119,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.425174E-1,2.9377276E-1,2.0752807E-1,3.6867535E-1,4.6355164E-1,2.3458305E-1,2.3015535E-1,3.685538E-1,3.3416277E-1,4.6296468E-1,2.3059373E-1,4.723925E-1,3.3938247E-1,2.3307288E-1,1.9107291E-1,3.4580404E-1,3.455556E-1,1.584928E-1,0E0,2.1890521E-1,2.414835E-1,2.560282E-1,2.5093338E-1,1.987324E-1,0E0,1.1229855E-1,0E0,1.4901075E-1,2.1128586E-1,2.3174436E-1,1.2753943E-1,2.879704E-1,2.1709871E-1,1.7329341E-1,2.6610893E-1,2.1497321E-1,1.1447038E-1,0E0,1.290921E-1,1.3604039E-1,2.3918398E-1,2.9216385E-1,1.7229575E-1,0E0,1.6217487E-1,1.9230881E-1,0E0,0E0,0E0,1.408415E-1,0E0,1.2478784E-1,1.4295173E-1,0E0,0E0,1.2912765E-1,1.7415974E-1,2.0167565E-1,2.1282588E-1,0E0,0E0,0E0,3.0787513E-1,2.2426343E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.585613E-1,0E0,0E0,0E0,5.0350624E-1,1.8677865E-1,1.6847938E-1,1.5406874E-1,1.607464E-1,1.8593124E-1,1.9403428E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5856686E-1,3.0528626E-1,1.6391341E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,44,44,45,45,49,49,51,51,52,52,55,55,56,56,57,57,58,58,62,62,63,63,71,71,75,75,76,76,77,77,78,78,79,79,80,80,81,81,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,-1,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,80,82,-1,-1,-1,84,-1,86,88,-1,-1,90,92,94,96,-1,-1,-1,98,100,-1,-1,-1,-1,-1,-1,-1,102,-1,-1,-1,104,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,-1,118,120,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4887473E-1,1.172E3,2.5291866E1,6.523944E-2,5.107429E1,9.923929E2,7.2243604E2,6.68E3,1.02E3,1.3E1,6.624E3,1.3697374E7,9.433E3,5.5356906E4,1.3676985E4,1E0,2.64E0,1.566593E6,5.6635416E-3,1.0258198E-2,4.004366E1,1.7291142E7,1.363881E3,1E0,-1.5282848E-2,3.773E3,1.0144729E-3,1.0277911E0,1.3E1,8.3893835E2,1.3860265E6,3.3151623E1,1.632095E2,4.144144E-1,1.563E3,1.6410715E1,1.9E1,9.092356E-3,1.5669752E-2,1.3238013E2,6.0078123E-6,1.0317661E5,1.84556E7,6.82748E-3,9.302862E7,9.414044E2,-6.748306E-3,9.037837E-4,1.3816393E-2,1.2E1,2.962606E-3,1.3264E4,2.8599728E7,6.5675895E-3,-9.236528E-3,1.1502944E0,1.5E1,2.4636364E1,1E0,-1.4459307E-2,-2.649444E-3,3.3549555E-3,7.33E2,1E0,2.8487018E-3,-6.172395E-3,5.5805114E-3,4.55719E-4,-7.6263743E-3,-4.346701E-3,4.169664E-3,7.2419E4,7.225985E-3,-1.065625E-2,4.146058E-3,1.3797468E0,8E0,1.627E3,3.138649E6,1.08874E5,1.2E1,2.372255E2,-9.066789E-3,-7.3993853E-3,-4.2591052E-4,2.6852307E-3,-4.917407E-3,-1.4322159E-2,-5.010318E-3,1E0,5.7819215E6,6.952967E6,3.9980677E-3,-1.3024671E-3,-1.1360616E-2,1.9481808E-3,-3.7995146E-3,-5.9472066E-3,5.1132636E-4,-1.1180117E-2,-4.354136E-3,4.288706E-3,-3.3131114E-4,-1.7230488E-2,-1.5415894E-3,-2.3342695E-3,1.4440342E-3,2.0779527E-3,-1.0127966E-2,-2.3714113E-3,1.000224E-3,4.4066127E-4,5.142512E-3,2.981119E-4,-5.3758873E-3,-1.1740755E-4,2.2152297E-3,7.5036474E-4,-6.087997E-3,4.9505937E-3,-3.1861402E-3,-1.7477366E-3,2.2101083E-3],"split_indices":[38,2,58,41,48,52,55,29,0,3,2,45,2,33,33,105,54,9,0,38,53,45,4,65,0,2,0,38,3,4,28,55,4,53,11,54,8,0,38,56,37,28,45,0,45,52,0,0,0,3,0,10,5,0,0,38,8,52,74,0,0,0,2,102,0,0,0,0,0,0,0,1,0,0,0,53,3,2,29,2,18,4,0,0,0,0,0,0,0,105,51,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.692E3,1.622E3,1.07E3,5.84E2,1.038E3,4.26E2,6.44E2,4.76E2,1.08E2,1.67E2,8.71E2,3.99E2,2.7E1,1.22E2,5.22E2,3.29E2,1.47E2,1.01E2,7E0,5.4E1,1.13E2,5.57E2,3.14E2,3.96E2,3E0,1.1E1,1.6E1,7.1E1,5.1E1,1E1,5.12E2,2.83E2,4.6E1,8.8E1,5.9E1,7.7E1,2.4E1,2.5E1,2.9E1,1.03E2,1E1,3.02E2,2.55E2,1.3E1,3.01E2,3.9E2,6E0,2E0,9E0,4.7E1,2.4E1,1.9E1,3.2E1,2E0,8E0,3.63E2,1.49E2,6.3E1,2.2E2,3E0,4.3E1,1.7E1,7.1E1,5.2E1,7E0,7.4E1,3E0,2E1,4E0,4E0,2.5E1,9.6E1,7E0,7E0,3E0,8.6E1,2.16E2,1.3E1,2.42E2,2.48E2,5.3E1,3.87E2,3E0,7E0,4E1,1.2E1,7E0,4E0,2.8E1,2.94E2,6.9E1,9.7E1,5.2E1,5.9E1,4E0,2.05E2,1.5E1,2.6E1,4.5E1,1.7E1,3.5E1,2.1E1,7.5E1,4E0,8.2E1,3.5E1,1.81E2,3E0,1E1,2.06E2,3.6E1,2.31E2,1.7E1,3.1E1,2.2E1,1.17E2,2.7E2,2.87E2,7E0,1.2E1,5.7E1,4E1,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[-1.7857727E-3,-1.5245068E-2,6.3086837E-3,3.66996E-2,-2.0110637E-2,3.6427625E-3,4.9310643E-2,-3.1467382E-2,8.409787E-2,-3.9379105E-2,-4.458105E-3,-6.000741E-3,1.7881362E-2,2.7711175E-3,-4.312673E-3,6.5940915E-4,-1.24488855E-2,1.1632722E-2,2.3420372E-3,-7.1505303E-3,-7.629485E-2,2.4418577E-3,-1.4224856E-2,-8.568839E-3,3.3123365E-3,-7.371797E-3,3.1257723E-2,-2.173216E-2,3.948643E-3,1.0315223E-2,-8.368596E-2,-1.3127958E-2,-1.1762654E-2,1.1242803E-2,-1.9165298E-2,1.6421094E-3,-1.7878531E-1,6.985648E-2,1.3541595E-2,-8.759423E-2,-6.294956E-3,-1.07138894E-1,-1.3538097E-2,2.7627632E-2,-2.8103953E-2,-9.245398E-3,4.813086E-2,-1.333554E-2,-1.692019E-2,-6.0112447E-2,1.3913515E-2,-1.9818211E-3,-1.6062122E-2,8.991337E-2,-2.1527794E-3,-2.1557769E-1,1.7570259E-2,-4.883165E-4,-6.819266E-3,1.6833469E-3,-2.2824197E-3,-1.30273225E-2,-4.079154E-3,1.7118638E-3,-3.079885E-3,7.369151E-3,5.623253E-4,-5.8941813E-3,-1.0549527E-3,2.2223608E-4,-5.0040283E-3,4.917619E-3,2.27898E-4,-2.4701196E-3,-4.719342E-4,3.598542E-4,-5.0550676E-3,3.6102723E-4,7.032465E-3,7.3335418E-3,2.3568645E-3,4.9603516E-3,-2.2167237E-2,1.5070465E-3,-1.4856507E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,31,33,-1,35,37,39,-1,-1,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,-1,-1,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.901096E-1,2.5347847E-1,1.904664E-1,2.805121E-1,2.7578872E-1,2.1527204E-1,1.2083137E-1,3.7565735E-1,2.7315745E-1,4.8724848E-1,2.7474016E-1,1.8107793E-1,2.1405488E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.892312E-1,4.2979968E-1,0E0,2.7258313E-1,1.9020504E-1,0E0,3.4021673E-1,2.8215572E-1,1.9262663E-1,0E0,0E0,3.0619693E-1,0E0,2.762751E-1,2.392683E-1,3.38752E-1,1.5997756E-1,2.2282183E-1,3.033932E-1,2.668555E-1,1.5123424E-1,2.6739642E-1,4.8380184E-1,1.22407295E-1,2.508902E-1,1.7212185E-1,2.7028334E-1,2.6688185E-1,0E0,1.4299533E-1,1.0835156E-1,1.4803985E-1,0E0,0E0,2.7806234E-1,0E0,4.4257003E-1,1.9074884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,19,19,20,20,22,22,23,23,25,25,26,26,27,27,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,50,50,53,53,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,32,34,-1,36,38,40,-1,-1,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,-1,-1,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1E0,2.7702793E-1,3.6E2,2.074E3,4.930349E5,1E0,3.09627E5,4.676E3,2.2444444E0,8.182648E6,4.4854636E7,7.118233E7,2.7711175E-3,-4.312673E-3,6.5940915E-4,-1.24488855E-2,1.1632722E-2,2.3420372E-3,1.5733662E0,3.192E3,2.4418577E-3,5E1,2.2415E4,3.3123365E-3,1.4329944E0,1.5445488E8,3.79E2,3.948643E-3,1.0315223E-2,1.27384615E1,-1.3127958E-2,2.7682406E8,5.4468E4,6.23E2,5.08E2,3.577537E7,6.788246E0,1.5938586E8,6.7E1,1.5316E4,5.8601086E1,2.0189162E7,3.4015749E0,1.3456146E6,1.421E3,2.4517391E1,-1.333554E-2,3.26E2,3.429012E7,1E0,-1.9818211E-3,-1.6062122E-2,1.8048264E4,-2.1527794E-3,3.7246967E2,2.1298597E0,-4.883165E-4,-6.819266E-3,1.6833469E-3,-2.2824197E-3,-1.30273225E-2,-4.079154E-3,1.7118638E-3,-3.079885E-3,7.369151E-3,5.623253E-4,-5.8941813E-3,-1.0549527E-3,2.2223608E-4,-5.0040283E-3,4.917619E-3,2.27898E-4,-2.4701196E-3,-4.719342E-4,3.598542E-4,-5.0550676E-3,3.6102723E-4,7.032465E-3,7.3335418E-3,2.3568645E-3,4.9603516E-3,-2.2167237E-2,1.5070465E-3,-1.4856507E-3],"split_indices":[102,79,35,2,2,28,106,9,1,56,45,9,7,0,0,0,0,0,0,56,1,0,0,29,0,38,7,1,0,0,56,0,7,1,2,2,7,53,7,29,9,50,45,56,51,2,58,0,0,7,100,0,0,33,0,4,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.661E3,9.99E2,1.662E3,8.5E1,9.14E2,1.566E3,9.6E1,3.5E1,5E1,4.09E2,5.05E2,9.34E2,6.32E2,9.1E1,5E0,3E1,5E0,8E0,4.2E1,2.19E2,1.9E2,7.5E1,4.3E2,9.04E2,3E1,2.19E2,4.13E2,1.89E2,3E1,4E0,1.86E2,3E0,4.27E2,3.15E2,5.89E2,2.09E2,1E1,1.29E2,2.84E2,3.5E1,1.54E2,1.39E2,4.7E1,1.25E2,3.02E2,2.03E2,1.12E2,4E0,5.85E2,3.4E1,1.75E2,6E0,4E0,1.1E2,1.9E1,4E0,2.8E2,1.5E1,2E1,7.7E1,7.7E1,1.5E1,1.24E2,2.4E1,2.3E1,1.3E1,1.12E2,1.7E1,2.85E2,1.78E2,2.5E1,4.9E1,6.3E1,9.8E1,4.87E2,1.4E1,2E1,1.68E2,7E0,4.2E1,6.8E1,2E0,2E0,2.18E2,6.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.2462559E-3,3.9307628E-2,-3.8827376E-3,-3.226818E-3,5.32079E-2,-1.0608421E-2,1.4449323E-2,1.4283804E-2,4.029864E-2,-1.1262197E-2,9.006066E-3,3.072184E-2,-5.6940853E-3,4.2204163E-3,1.4342787E-2,-1.26485005E-2,8.4186085E-2,9.1464795E-2,2.0660782E-2,-2.1914434E-2,4.1145023E-2,-3.2763116E-2,6.433448E-2,-5.7149515E-3,-3.359276E-2,6.421183E-3,-4.3354635E-4,3.7462173E-3,1.3488212E-2,5.5227272E-2,3.2800797E-3,-1.0318107E-2,-3.564003E-3,1.5522914E-1,1.229263E-3,1.2057642E-2,-1.392733E-1,1.8997518E-3,9.053429E-3,-4.9020974E-3,-1.0620517E-2,-3.6577213E-1,-2.9732876E-2,3.510824E-2,5.145175E-3,5.5486675E-2,-1.2338233E-2,-1.18834985E-4,-4.151889E-3,6.8171846E-4,1.2699493E-2,-3.2080316E-3,4.120541E-3,-1.1222906E-2,1.2713218E-3,-1.1154166E-3,2.1528079E-4,-1.9811468E-4,-2.8996926E-2,-1.0156455E-3,-3.2158718E-3,3.3741598E-3,-4.947873E-4,6.722919E-4,5.756105E-3,-1.6915234E-3,1.3450813E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,7,9,11,-1,13,15,-1,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,-1,-1,-1,-1,43,45,47,-1,49,-1,51,53,-1,-1,55,-1,57,59,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.819926E-1,2.4234384E-1,3.0543324E-1,0E0,4.403516E-1,2.3746333E-1,2.1815127E-1,0E0,1.6880892E-1,2.4016763E-1,0E0,2.2300923E-1,2.2730617E-1,0E0,2.1404913E-1,2.5858656E-1,1.2216824E-1,1.08906806E-1,1.9026923E-1,1.3440105E-1,1.319715E-1,2.260511E-1,1.2784807E-1,2.3424143E-1,5.571277E-1,0E0,0E0,0E0,0E0,1.07569695E-1,1.7354494E-1,1.09319046E-1,0E0,1.3574398E-1,0E0,2.0459409E-1,2.2692758E-1,0E0,0E0,2.3097467E-1,0E0,4.3093556E-1,1.3861415E-1,1.2601018E-1,0E0,1.2972991E-1,1.5335764E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,29,29,30,30,31,31,33,33,35,35,36,36,39,39,41,41,42,42,43,43,45,45,46,46],"right_children":[2,4,6,-1,8,10,12,-1,14,16,-1,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,-1,-1,-1,-1,44,46,48,-1,50,-1,52,54,-1,-1,56,-1,58,60,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.489796E-1,2.35E2,1.0855529E3,-3.226818E-3,2.04E5,1.03992E3,2.3907686E7,1.4283804E-2,3.640909E6,1.0660479E2,9.006066E-3,1.5552E4,8.61491E3,4.2204163E-3,3.01E2,1.3905958E7,2.0237808E8,5.24422E5,6.89132E6,6.9998717E-4,8.0886055E3,2.2515285E6,1.067509E6,1.93974E7,1.0195312E0,6.421183E-3,-4.3354635E-4,3.7462173E-3,1.3488212E-2,1.1225189E5,2.28125E0,9E0,-3.564003E-3,1.9745072E7,1.229263E-3,5.828063E5,3.0121632E6,1.8997518E-3,9.053429E-3,2.0407547E5,-1.0620517E-2,2.915E4,7.8351054E11,4.949877E6,5.145175E-3,8.874407E6,3.4409692E3,-1.18834985E-4,-4.151889E-3,6.8171846E-4,1.2699493E-2,-3.2080316E-3,4.120541E-3,-1.1222906E-2,1.2713218E-3,-1.1154166E-3,2.1528079E-4,-1.9811468E-4,-2.8996926E-2,-1.0156455E-3,-3.2158718E-3,3.3741598E-3,-4.947873E-4,6.722919E-4,5.756105E-3,-1.6915234E-3,1.3450813E-3],"split_indices":[53,2,52,0,5,46,45,0,45,56,0,9,4,0,0,45,7,11,50,34,33,28,9,50,53,0,0,0,0,33,54,17,0,1,0,28,28,0,0,28,0,9,31,9,0,45,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.635E3,1.6E2,2.475E3,1.8E1,1.42E2,1.811E3,6.64E2,6E0,1.36E2,1.806E3,5E0,3.67E2,2.97E2,4.7E1,8.9E1,1.781E3,2.5E1,5.1E1,3.16E2,2.21E2,7.6E1,4.6E1,4.3E1,1.339E3,4.42E2,1.6E1,9E0,4.9E1,2E0,1.05E2,2.11E2,1.82E2,3.9E1,8E0,6.8E1,3.3E1,1.3E1,3.7E1,6E0,1.335E3,4E0,4E0,4.38E2,7.7E1,2.8E1,4.8E1,1.63E2,1.66E2,1.6E1,4E0,4E0,1.6E1,1.7E1,8E0,5E0,4.51E2,8.84E2,2E0,2E0,3.58E2,8E1,4.3E1,3.4E1,3E1,1.8E1,1.04E2,5.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-8.450246E-4,5.2264654E-3,-1.9518176E-2,3.0008398E-3,8.22117E-2,-5.9087586E-2,-7.964739E-3,4.0684803E-3,-8.77398E-2,7.0906975E-3,3.5626392E-3,-3.0349128E-2,-1.2277698E-1,-6.0651384E-2,-2.8816962E-3,-7.248077E-3,1.2767473E-2,2.402147E-3,-5.718662E-3,-2.9443514E-3,5.760498E-3,-1.31945945E-2,-6.6896277E-3,-7.1725324E-3,2.314325E-4,-9.0103E-3,-1.5427849E-3,6.0027998E-2,-1.0550387E-2,-5.555414E-3,-5.682088E-3,4.9527483E-3,1.0225812E-2,-3.687641E-2,8.489701E-2,6.80339E-3,-1.24300895E-2,-7.4487864E-3,-1.0660419E-1,4.484137E-2,-9.847016E-3,5.4754927E-3,5.2708063E-2,-2.640837E-2,-8.431305E-3,1.0968585E-2,1.95396E-3,-6.253427E-3,2.5727274E-2,8.901867E-3,-2.260177E-2,-9.886521E-3,4.0519098E-4,-3.6246088E-4,4.831039E-3,-1.6007894E-3,-4.352661E-5,2.9634868E-3,8.9957044E-5,3.570403E-3,-3.148785E-4,-2.2300882E-3,2.756089E-3,2.741482E-3,-6.2455297E-3,1.8083701E-3,-4.2680305E-4,-2.4799057E-3,-2.1004827E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,35,37,-1,39,-1,41,43,45,-1,47,49,51,53,55,57,59,61,-1,-1,-1,-1,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.049643E-1,3.4757414E-1,3.0125678E-1,1.9159487E-1,2.946725E-1,2.6899368E-1,1.3694315E-1,1.922526E-1,1.0644063E-1,0E0,2.1237947E-1,1.9322133E-1,1.5916973E-1,1.5708388E-1,2.264129E-1,1.4393237E-1,2.533076E-1,0E0,0E0,0E0,0E0,2.1452254E-1,0E0,0E0,0E0,0E0,0E0,3.0490848E-1,1.2373671E-1,0E0,1.7677902E-1,0E0,2.1665096E-1,1.0316228E-1,1.05062425E-1,0E0,1.2958884E-1,1.0079224E-1,1.5108547E-1,1.8871908E-1,1.6330263E-1,1.9628556E-1,1.4035648E-1,1.2114979E-1,0E0,0E0,0E0,0E0,1.15730435E-1,1.0109722E-1,1.1193089E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,21,21,27,27,28,28,30,30,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,48,48,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,36,38,-1,40,-1,42,44,46,-1,48,50,52,54,56,58,60,62,-1,-1,-1,-1,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3382226E-1,1.8552577E2,1.8903887E0,1E0,3.1972716E7,1.9166666E0,4.39688E-5,1.9563605E5,9E0,7.0906975E-3,1E0,2.285677E6,8.289561E8,5.987692E2,2.64E0,6E0,1.2E1,2.402147E-3,-5.718662E-3,-2.9443514E-3,5.760498E-3,3.449702E2,-6.6896277E-3,-7.1725324E-3,2.314325E-4,-9.0103E-3,-1.5427849E-3,4.88954E5,6.59148E-1,-5.555414E-3,1E0,4.9527483E-3,3.524269E6,6.5E1,2.4684112E8,6.80339E-3,3.8977896E2,7.1277297E-1,4.9885715E2,3.6E2,6.947568E7,2.3125623E5,1.21320024E8,4.3800272E8,-8.431305E-3,1.0968585E-2,1.95396E-3,-6.253427E-3,2.6E1,1.5418326E6,6.5039682E0,-9.886521E-3,4.0519098E-4,-3.6246088E-4,4.831039E-3,-1.6007894E-3,-4.352661E-5,2.9634868E-3,8.9957044E-5,3.570403E-3,-3.148785E-4,-2.2300882E-3,2.756089E-3,2.741482E-3,-6.2455297E-3,1.8083701E-3,-4.2680305E-4,-2.4799057E-3,-2.1004827E-4],"split_indices":[27,56,53,77,51,54,42,28,3,0,61,32,7,4,54,52,10,0,0,0,0,52,0,0,0,0,0,32,38,0,79,0,1,8,7,0,4,27,52,2,7,28,32,7,0,0,0,0,3,51,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.688E3,2.029E3,6.59E2,1.973E3,5.6E1,1.48E2,5.11E2,1.951E3,2.2E1,3E1,2.6E1,1.03E2,4.5E1,4.4E1,4.67E2,8.48E2,1.103E3,4E0,1.8E1,1.7E1,9E0,9E1,1.3E1,3.7E1,8E0,7E0,3.7E1,5E1,4.17E2,1.1E1,8.37E2,2.9E1,1.074E3,7.3E1,1.7E1,2.3E1,2.7E1,4.05E2,1.2E1,6.3E1,7.74E2,9.67E2,1.07E2,6.9E1,4E0,3E0,1.4E1,6E0,2.1E1,1.95E2,2.1E2,6E0,6E0,3.3E1,3E1,2.12E2,5.62E2,5.7E1,9.1E2,7.8E1,2.9E1,5.6E1,1.3E1,1.8E1,3E0,7.4E1,1.21E2,8E1,1.3E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[2.505353E-4,-1.5475386E-2,6.7282217E-3,-2.7822427E-4,-4.7753043E-2,1.96619E-2,-1.0524958E-2,4.2476733E-3,-5.5722357E-3,-7.911929E-2,-5.0942414E-3,4.7287423E-2,1.2083955E-2,-1.7698677E-2,2.0536177E-2,1.1404028E-2,-7.841582E-2,-2.2309543E-1,-6.434043E-2,1.0973172E-1,-2.0753644E-2,6.568712E-2,-2.6925992E-2,-2.2170506E-2,2.2216989E-2,-2.0602634E-2,4.175178E-3,3.7042093E-2,-5.1318146E-2,1.650263E-2,-6.1824527E-2,-1.09231554E-1,1.3426641E-3,-1.9525543E-3,-1.3108387E-2,1.7312178E-3,-8.932993E-2,1.1217811E-2,1.3585981E-3,2.1686828E-2,-7.965146E-2,5.5673987E-2,1.767928E-1,-5.375866E-3,6.070145E-4,-6.5569236E-4,-1.0053029E-1,2.3662772E-2,-8.205319E-3,-1.700037E-2,-1.0229141E-1,3.916233E-3,1.5029411E-2,-1.3214477E-2,-1.1090149E-2,-2.3420404E-3,4.9524613E-2,-9.2841027E-4,-2.16204E-2,2.6007663E-3,-6.1310385E-3,-1.898155E-1,-6.800329E-2,3.99772E-3,6.9412286E-3,-1.7718047E-1,-1.7503047E-3,6.7515234E-3,4.7276262E-2,1.3408653E-2,2.466413E-3,-6.518708E-2,2.8299095E-2,-3.264733E-1,-6.1173845E-2,2.2012698E-2,9.492001E-3,6.108415E-3,-3.3034977E-2,-1.41859995E-2,-2.1432433E-3,9.49959E-3,3.9375166E-4,-3.5722866E-3,4.9897404E-3,7.985907E-4,-1.7868241E-3,4.677161E-3,2.0279555E-4,-1.4820773E-2,-4.990099E-3,-1.1843284E-2,-2.7133075E-3,4.272176E-3,-1.0690631E-3,-1.1149878E-2,-3.6857784E-4,1.1044326E-3,3.6499936E-3,-1.3734085E-3,-6.2293797E-3,2.3263711E-3,-2.1001028E-3,-2.3100097E-2,-4.6983524E-4,-1.3840309E-3,-8.359507E-3,2.246093E-3,4.94886E-4,1.4918715E-3,-2.0677017E-3,-1.6896473E-2,-1.451139E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,-1,-1,-1,-1,61,-1,-1,63,65,67,69,-1,-1,71,73,75,-1,77,79,-1,81,83,-1,85,87,-1,-1,-1,-1,89,91,93,-1,95,-1,-1,97,-1,-1,99,101,103,105,107,-1,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7270177E-1,3.8308966E-1,4.2329377E-1,2.799779E-1,3.3410847E-1,2.263582E-1,1.8151927E-1,3.043693E-1,0E0,2.920497E-1,1.9467466E-1,3.1993115E-1,2.9622313E-1,2.0311564E-1,1.8336219E-1,1.7762111E-1,1.3755858E-1,1.08820915E-1,3.347066E-1,1.25048E-1,2.3919426E-1,1.9819784E-1,1.6023797E-1,3.2868382E-1,1.8818504E-1,1.8776739E-1,0E0,1.2185645E-1,1.9612253E-1,2.7578974E-1,5.03842E-1,1.0528651E-1,0E0,0E0,0E0,0E0,2.1425194E-1,0E0,0E0,1.2151341E-1,1.6167775E-1,1.1726552E-1,1.7951828E-1,0E0,0E0,2.895926E-1,3.5350552E-1,1.8399084E-1,0E0,2.2919564E-1,2.8801444E-1,0E0,1.0445136E-1,1.8792477E-1,0E0,1.8882E-1,3.5060126E-1,0E0,0E0,0E0,0E0,1.5215778E-1,1.670542E-1,1.14130214E-1,0E0,1.113686E-1,0E0,0E0,1.0966161E-1,0E0,0E0,1.0998201E-1,1.5867813E-1,2.9143363E-1,1.3034733E-1,1.8815726E-1,0E0,3.1484014E-1,3.083984E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,36,36,39,39,40,40,41,41,42,42,45,45,46,46,47,47,49,49,50,50,52,52,53,53,55,55,56,56,61,61,62,62,63,63,65,65,68,68,71,71,72,72,73,73,74,74,75,75,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,-1,-1,-1,-1,62,-1,-1,64,66,68,70,-1,-1,72,74,76,-1,78,80,-1,82,84,-1,86,88,-1,-1,-1,-1,90,92,94,-1,96,-1,-1,98,-1,-1,100,102,104,106,108,-1,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.146E3,5.116E3,1.4274633E7,1E0,5.1454154E8,1.1457406E-1,3.317425E3,1E0,-5.5722357E-3,1.763213E-5,2.831692E10,2.773036E1,3.2485715E2,7.26E2,1.75E2,1.3569831E7,5.117647E0,8.9928055E-1,4.0233E4,1.3569831E7,1.9928571E0,4.8992978E5,7.57657E4,5.3379045E6,1E0,4.1585636E1,4.175178E-3,1.5479116E-1,1.5005797E4,5.85E2,8.9E1,9.6590906E-1,1.3426641E-3,-1.9525543E-3,-1.3108387E-2,1.7312178E-3,4.2197E4,1.1217811E-2,1.3585981E-3,1.074E3,2.5070414E-5,7E0,2.65903E5,-5.375866E-3,6.070145E-4,1.04856936E8,9.536863E4,1.921032E4,-8.205319E-3,1.7181714E7,2.90002E5,3.916233E-3,8.0886055E3,8.61491E3,-1.1090149E-2,3.38464E5,1E0,-9.2841027E-4,-2.16204E-2,2.6007663E-3,-6.1310385E-3,5.339161E0,4.54E2,4.144144E-1,6.9412286E-3,2.1989189E2,-1.7503047E-3,6.7515234E-3,4.4377274E2,1.3408653E-2,2.466413E-3,1.4E1,3.2049225E1,9.5E2,6.6141135E-1,4.0208E4,9.492001E-3,2.8277853E11,5.1776E4,-1.41859995E-2,-2.1432433E-3,9.49959E-3,3.9375166E-4,-3.5722866E-3,4.9897404E-3,7.985907E-4,-1.7868241E-3,4.677161E-3,2.0279555E-4,-1.4820773E-2,-4.990099E-3,-1.1843284E-2,-2.7133075E-3,4.272176E-3,-1.0690631E-3,-1.1149878E-2,-3.6857784E-4,1.1044326E-3,3.6499936E-3,-1.3734085E-3,-6.2293797E-3,2.3263711E-3,-2.1001028E-3,-2.3100097E-2,-4.6983524E-4,-1.3840309E-3,-8.359507E-3,2.246093E-3,4.94886E-4,1.4918715E-3,-2.0677017E-3,-1.6896473E-2,-1.451139E-3],"split_indices":[2,29,45,104,5,42,52,74,0,38,31,58,52,8,8,45,54,53,1,45,54,51,28,45,85,53,0,57,4,2,0,53,0,0,0,0,12,0,0,2,42,3,32,0,0,7,28,4,0,12,1,0,33,4,0,9,16,0,0,0,0,58,2,53,0,52,0,0,52,0,0,8,58,0,41,9,0,31,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.675E3,7.8E2,1.895E3,5.31E2,2.49E2,1.083E3,8.12E2,5.12E2,1.9E1,1.43E2,1.06E2,2.32E2,8.51E2,6.6E2,1.52E2,4.72E2,4E1,1.2E1,1.31E2,1.2E1,9.4E1,1.86E2,4.6E1,1.94E2,6.57E2,6.43E2,1.7E1,1.24E2,2.8E1,4.42E2,3E1,3.1E1,9E0,3E0,9E0,2.6E1,1.05E2,4E0,8E0,5.5E1,3.9E1,1.72E2,1.4E1,1.4E1,3.2E1,1.53E2,4.1E1,6.53E2,4E0,6.17E2,2.6E1,4E1,8.4E1,2.4E1,4E0,2.82E2,1.6E2,2.8E1,2E0,3E0,2.8E1,1.7E1,8.8E1,4.9E1,6E0,1.1E1,2.8E1,1.4E1,1.58E2,7E0,7E0,4.7E1,1.06E2,5E0,3.6E1,6.48E2,5E0,2.53E2,3.64E2,5E0,2.1E1,2E0,8.2E1,1.6E1,8E0,1.83E2,9.9E1,7.7E1,8.3E1,6E0,1.1E1,4E0,8.4E1,1.1E1,3.8E1,8E0,3E0,8.7E1,7.1E1,3.1E1,1.6E1,8.3E1,2.3E1,3E0,2E0,2.9E1,7E0,2.06E2,4.42E2,1.68E2,8.5E1,2E0,3.62E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-1.0351321E-3,-9.22095E-3,1.1703669E-2,-8.717156E-4,-3.1310078E-2,3.020403E-2,1.228821E-4,5.6635885E-4,-1.7047733E-1,-8.9964405E-2,-2.1326074E-2,-1.9878922E-2,4.2211216E-2,3.2005638E-3,-4.4021127E-3,5.4684356E-3,-4.915344E-2,5.4527284E-3,-3.0860084E-1,-2.1504734E-1,-2.3053824E-3,4.4316027E-2,-3.125569E-2,-1.3446878E-1,1.3709729E-2,1.0522257E-2,3.820892E-2,-4.6437304E-3,4.1012384E-2,1.0765594E-3,5.2946784E-2,-6.586064E-2,8.720801E-2,-1.9719334E-2,-1.6189173E-3,-1.3470543E-2,-6.266413E-4,-2.23503E-3,1.0452187E-1,-1.3738455E-1,-2.1128677E-2,-3.4813446E-3,-1.5923979E-2,2.926725E-3,-1.862494E-3,4.313307E-2,-2.2545496E-3,-1.196464E-2,2.917685E-2,7.5314394E-3,1.3819087E-3,1.80894E-2,-1.6428579E-2,3.9676197E-2,7.493475E-3,3.7091756E-3,-7.992898E-2,9.141252E-3,-6.757506E-4,-1.8806949E-3,6.095457E-3,-6.1728597E-2,-2.2575122E-1,5.221671E-3,-4.5370694E-2,3.8966984E-2,6.388206E-3,-5.9049954E-3,-5.9940048E-2,2.1114575E-2,9.735371E-3,-4.9291115E-4,1.729651E-3,-1.0559111E-3,4.9375794E-3,3.421892E-3,-2.4118379E-4,-1.4083701E-2,-3.030732E-3,-5.3249407E-3,3.7937127E-3,-1.314917E-2,1.3644493E-3,-5.4131616E-3,7.8956806E-4,-1.2754747E-3,-5.341214E-3,1.9811473E-3,-6.908964E-3,-4.5285383E-3,-8.4814E-5,-7.714554E-4,-5.316256E-3,1.393565E-3,-5.7679266E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,35,-1,37,39,41,43,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,61,63,-1,-1,-1,-1,65,-1,67,69,-1,-1,71,73,75,-1,-1,77,-1,-1,-1,-1,79,81,83,85,87,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8040585E-1,3.0188024E-1,2.2538595E-1,2.901002E-1,2.610181E-1,2.4438795E-1,1.8385938E-1,2.8787053E-1,4.2787087E-1,3.2970822E-1,2.5232843E-1,3.059081E-1,2.239126E-1,1.8645968E-1,0E0,2.2410095E-1,2.4723464E-1,0E0,1.8451208E-1,2.1033645E-1,0E0,2.84249E-1,3.5714465E-1,1.9603825E-1,1.5696873E-1,0E0,1.3683805E-1,1.2934734E-1,1.4809252E-1,2.936421E-1,1.1880994E-1,1.977256E-1,1.2873745E-1,0E0,0E0,0E0,0E0,0E0,1.0643956E-1,1.799432E-1,1.963441E-1,0E0,0E0,0E0,0E0,1.0911393E-1,0E0,1.24362305E-1,1.2625167E-1,0E0,0E0,2.5619668E-1,3.28739E-1,1.1686435E-1,0E0,0E0,2.9149514E-1,0E0,0E0,0E0,0E0,1.2732032E-1,1.6916668E-1,1.99023E-1,1.9537485E-1,1.3444486E-1,0E0,1.3983572E-1,1.05654866E-1,1.0522493E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,38,38,39,39,40,40,45,45,47,47,48,48,51,51,52,52,53,53,56,56,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,36,-1,38,40,42,44,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,62,64,-1,-1,-1,-1,66,-1,68,70,-1,-1,72,74,76,-1,-1,78,-1,-1,-1,-1,80,82,84,86,88,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.83E3,5.5306E4,8E-3,1.6015803E7,3.451532E8,9.996076E2,2.6572757E10,1.062363E1,2.5489312E5,2.5454E4,3.2585382E10,3.114E3,1.05842E3,8.865177E-1,-4.4021127E-3,2.539E3,3.422351E6,5.4527284E-3,5.59E2,1.9939552E8,-2.3053824E-3,9.048701E7,7.359468E8,2.139E3,4.28586E5,1.0522257E-2,2.1751701E2,1.5515003E0,4.9469455E6,1E0,1.35E2,3.97E2,1.35419E5,-1.9719334E-2,-1.6189173E-3,-1.3470543E-2,-6.266413E-4,-2.23503E-3,2.75E2,6.09603E8,1.7291142E7,-3.4813446E-3,-1.5923979E-2,2.926725E-3,-1.862494E-3,1.8552577E2,-2.2545496E-3,1.4613059E0,7.9023E4,7.5314394E-3,1.3819087E-3,1.6171625E2,3.7E1,3.7057E5,7.493475E-3,3.7091756E-3,5.664E1,9.141252E-3,-6.757506E-4,-1.8806949E-3,6.095457E-3,5.0461445E-2,1.9E1,1.8108038E5,3.6401E4,2.52331E5,6.388206E-3,1.0989011E-2,1.46617E5,1E0,9.735371E-3,-4.9291115E-4,1.729651E-3,-1.0559111E-3,4.9375794E-3,3.421892E-3,-2.4118379E-4,-1.4083701E-2,-3.030732E-3,-5.3249407E-3,3.7937127E-3,-1.314917E-2,1.3644493E-3,-5.4131616E-3,7.8956806E-4,-1.2754747E-3,-5.341214E-3,1.9811473E-3,-6.908964E-3,-4.5285383E-3,-8.4814E-5,-7.714554E-4,-5.316256E-3,1.393565E-3,-5.7679266E-3],"split_indices":[2,29,57,51,5,55,12,54,33,9,31,29,4,35,0,2,9,0,0,12,0,7,5,29,1,0,54,39,51,16,8,2,1,0,0,0,0,0,11,5,45,0,0,0,0,56,0,41,2,0,0,55,3,9,0,0,47,0,0,0,0,27,3,28,9,11,0,57,9,105,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.687E3,1.636E3,1.051E3,1.188E3,4.48E2,4.04E2,6.47E2,1.179E3,9E0,6.4E1,3.84E2,7.8E1,3.26E2,6.27E2,2E1,1.074E3,1.05E2,3E0,6E0,1.5E1,4.9E1,5E1,3.34E2,1.7E1,6.1E1,6E0,3.2E2,5.2E2,1.07E2,9.84E2,9E1,9.4E1,1.1E1,4E0,2E0,1.1E1,4E0,2E1,3E1,2.8E1,3.06E2,1.4E1,3E0,3.2E1,2.9E1,3.03E2,1.7E1,4.28E2,9.2E1,9E0,9.8E1,4.99E2,4.85E2,8.1E1,9E0,8E0,8.6E1,5E0,6E0,4E0,2.6E1,1.6E1,1.2E1,1.47E2,1.59E2,2.91E2,1.2E1,3.81E2,4.7E1,8.9E1,3E0,1.94E2,3.05E2,4.64E2,2.1E1,4.7E1,3.4E1,5E0,8.1E1,1.2E1,4E0,1E1,2E0,1.2E1,1.35E2,1.25E2,3.4E1,2.88E2,3E0,1.6E1,3.65E2,2.6E1,2.1E1,8.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[5.0632947E-4,1.3622423E-3,-8.9590184E-2,-4.6076654E-3,1.3854655E-2,-2.4001086E-3,-9.876812E-3,1.0749808E-2,-1.2399439E-2,2.4207287E-2,-5.2403696E-3,5.940075E-3,7.139234E-2,-4.2675834E-2,-5.469837E-3,7.490368E-2,1.55816665E-2,-1.443045E-2,2.8017194E-3,8.222949E-3,-1.5175004E-1,8.902365E-2,-2.6392427E-3,-1.728675E-1,-3.366444E-2,-7.976705E-3,7.275965E-2,8.4878534E-2,-4.3913145E-3,5.231528E-2,4.699666E-3,-3.149252E-3,-2.731515E-3,1.5043318E-2,-4.289893E-2,9.097752E-4,-1.2355645E-2,5.8336486E-3,2.4417273E-4,6.6346987E-3,-2.3631346E-1,-2.059524E-2,-1.2385404E-1,6.167797E-4,-3.3686347E-2,-3.3701367E-3,4.9902312E-3,6.494585E-3,2.246757E-3,1.12606525E-1,3.4676395E-2,-5.9560616E-2,1.2545879E-2,-2.1838553E-2,2.5485573E-2,-6.1286177E-4,1.5563488E-3,-4.4888537E-3,2.2913506E-3,-6.4450456E-3,-1.9450184E-2,-1.683627E-4,-4.081503E-3,-1.1058426E-2,-3.0384173E-3,4.1796328E-3,-1.7521942E-4,-8.375788E-3,-1.3055303E-3,1.3188211E-3,8.738541E-3,-6.438924E-3,2.0355212E-3,-7.6779258E-3,-5.100913E-4,3.2033145E-3,1.5940925E-5,-1.4430815E-3,5.969806E-3,1.1723967E-2,6.800888E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,-1,49,51,53,-1,55,57,-1,-1,-1,-1,-1,59,61,63,65,67,-1,-1,-1,-1,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0486172E-1,1.9628002E-1,1.04776174E-1,2.1325123E-1,1.6835573E-1,0E0,0E0,1.7458057E-1,2.4775416E-1,2.4021581E-1,1.7658502E-1,2.0176044E-1,1.0304366E-1,2.522865E-1,1.8943527E-1,1.4072657E-1,1.8882889E-1,1.2623101E-1,0E0,1.9241388E-1,1.5003285E-1,1.06698215E-1,0E0,3.0934316E-1,2.4144126E-1,2.0637447E-1,1.4233911E-1,1.400209E-1,0E0,1.1094999E-1,1.8528438E-1,1.1240602E-1,0E0,2.3658636E-1,3.0431536E-1,0E0,0E0,0E0,0E0,0E0,1.5266758E-1,2.0060405E-1,1.5349635E-1,2.600457E-1,2.0755237E-1,0E0,0E0,0E0,0E0,1.3760918E-1,1.1823745E-1,1.9542041E-1,2.1700251E-1,1.5807047E-1,2.0194659E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,40,40,41,41,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,-1,50,52,54,-1,56,58,-1,-1,-1,-1,-1,60,62,64,66,68,-1,-1,-1,-1,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5265896E8,8.027061E2,1.7243155E12,2.2444444E0,2.8839298E7,-2.4001086E-3,-9.876812E-3,1.9826213E0,1.8122449E2,1.4075E4,1E0,2E0,6.731367E0,5.8941E1,6.32E2,2.0319266E-7,2.6979439E1,1.2737473E1,2.8017194E-3,1E0,5.3E1,2.1143505E6,-2.6392427E-3,1.57E2,4.4821704E7,6.015825E0,3.0972284E1,7.4456794E5,-4.3913145E-3,1.3E1,3.557179E1,4.7272725E0,-2.731515E-3,1E0,6.68E3,9.097752E-4,-1.2355645E-2,5.8336486E-3,2.4417273E-4,6.6346987E-3,2.5363636E0,1.593E2,2.1340163E8,6.647279E5,3.6789783E2,-3.3701367E-3,4.9902312E-3,6.494585E-3,2.246757E-3,2.6956522E0,1.954E3,5.2369475E6,1.5277338E0,1.7310865E6,4.7777777E0,-6.1286177E-4,1.5563488E-3,-4.4888537E-3,2.2913506E-3,-6.4450456E-3,-1.9450184E-2,-1.683627E-4,-4.081503E-3,-1.1058426E-2,-3.0384173E-3,4.1796328E-3,-1.7521942E-4,-8.375788E-3,-1.3055303E-3,1.3188211E-3,8.738541E-3,-6.438924E-3,2.0355212E-3,-7.6779258E-3,-5.100913E-4,3.2033145E-3,1.5940925E-5,-1.4430815E-3,5.969806E-3,1.1723967E-2,6.800888E-4],"split_indices":[45,52,31,56,45,0,0,56,52,9,96,6,54,50,8,37,58,54,0,105,0,50,0,29,12,53,56,32,0,3,58,54,0,66,29,0,0,0,0,0,53,52,7,45,55,0,0,0,0,54,2,45,53,28,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.654E3,2.63E3,2.4E1,1.78E3,8.5E2,1.9E1,5E0,5.99E2,1.181E3,5.51E2,2.99E2,5.56E2,4.3E1,2.19E2,9.62E2,7.9E1,4.72E2,2.62E2,3.7E1,5.49E2,7E0,3.8E1,5E0,1.3E1,2.06E2,9.33E2,2.9E1,7.5E1,4E0,1.07E2,3.65E2,2.08E2,5.4E1,4.85E2,6.4E1,3E0,4E0,2.7E1,1.1E1,2E0,1.1E1,1.81E2,2.5E1,7E2,2.33E2,5E0,2.4E1,3.1E1,4.4E1,2.3E1,8.4E1,3.9E1,3.26E2,1.26E2,8.2E1,1.87E2,2.98E2,4.1E1,2.3E1,8E0,3E0,1.44E2,3.7E1,8E0,1.7E1,3.2E1,6.68E2,9E0,2.24E2,1.1E1,1.2E1,3E0,8.1E1,1.2E1,2.7E1,5.9E1,2.67E2,1.2E2,6E0,3E0,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[1.04701E-3,-5.101969E-4,4.8529644E-2,1.3016697E-2,-6.923786E-3,1.0006952E-1,9.419537E-3,1.4987735E-2,-2.2480424E-1,-3.6442153E-2,-2.2554586E-3,1.1942421E-1,-2.283673E-3,-5.448645E-4,6.6470536E-3,9.980535E-3,6.5060176E-2,4.6291022E-4,-1.9303402E-2,-1.6704468E-1,-2.833399E-2,-1.299561E-3,-7.0967386E-3,1.28666805E-2,4.289764E-3,8.645347E-3,8.476427E-3,9.538927E-2,-4.6274066E-3,-9.994523E-3,2.6454101E-3,4.8131095E-3,-4.967722E-2,-2.0690986E-3,1.11905495E-2,1.58717E-2,-2.0751279E-2,2.0031498E-3,6.6220053E-3,-3.8318925E-3,2.6308717E-3,-6.761639E-3,2.2559604E-2,-4.39713E-3,-6.838517E-2,3.3913122E-3,-3.9289654E-3,-7.6697243E-4,1.287678E-3,-1.4811134E-3,2.4441313E-3,2.768058E-3,-1.1529619E-3,-2.054178E-3,4.569593E-3,-2.0818287E-3,-6.7235227E-3,4.0361355E-4,-5.9861975E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,-1,-1,29,31,33,-1,-1,-1,35,-1,37,39,-1,-1,41,43,45,-1,47,49,-1,-1,-1,-1,-1,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0025504E-1,2.2765094E-1,1.7241018E-1,3.9899778E-1,2.452043E-1,1.1150542E-1,1.3611096E-1,2.094563E-1,2.9648784E-1,2.5259098E-1,2.1435118E-1,1.2294665E-1,0E0,0E0,0E0,1.6861641E-1,1.6107205E-1,0E0,0E0,1.411429E-1,1.6295631E-1,2.769533E-1,0E0,0E0,0E0,1.6135159E-1,0E0,1.1595529E-1,1.1216009E-1,0E0,0E0,2.3896922E-1,1.1829293E-1,2.0699587E-1,0E0,2.143558E-1,1.1115179E-1,0E0,0E0,0E0,0E0,0E0,1.3615075E-1,1.6547644E-1,1.71509E-1,0E0,1.5769568E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,19,19,20,20,21,21,25,25,27,27,28,28,31,31,32,32,33,33,35,35,36,36,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,-1,-1,30,32,34,-1,-1,-1,36,-1,38,40,-1,-1,42,44,46,-1,48,50,-1,-1,-1,-1,-1,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8552577E2,8E0,2.6749698E7,4.004366E1,9E2,1.701E3,2.62797E5,1E0,1E1,2.8972292E-5,1.314806E8,2.0185E4,-2.283673E-3,-5.448645E-4,6.6470536E-3,1.8924E4,4.838871E8,4.6291022E-4,-1.9303402E-2,5.750032E3,1.04E2,2.4507338E4,-7.0967386E-3,1.28666805E-2,4.289764E-3,1E0,8.476427E-3,7.2488395E6,7.73E3,-9.994523E-3,2.6454101E-3,1.56E2,5.827505E5,2.139E3,1.11905495E-2,1E1,2.632E5,2.0031498E-3,6.6220053E-3,-3.8318925E-3,2.6308717E-3,-6.761639E-3,4.6E0,3.591E3,4.752366E0,3.3913122E-3,2.1525E4,-7.6697243E-4,1.287678E-3,-1.4811134E-3,2.4441313E-3,2.768058E-3,-1.1529619E-3,-2.054178E-3,4.569593E-3,-2.0818287E-3,-6.7235227E-3,4.0361355E-4,-5.9861975E-4],"split_indices":[56,17,50,53,2,11,2,84,8,42,1,10,0,0,0,6,7,0,0,33,10,4,0,0,0,14,0,45,29,0,0,2,45,29,0,3,33,0,0,0,0,0,54,29,56,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.707E3,2.622E3,8.5E1,8.43E2,1.779E3,3.6E1,4.9E1,8.37E2,6E0,2.42E2,1.537E3,3.2E1,4E0,4.3E1,6E0,7.62E2,7.5E1,3E0,3E0,1.3E1,2.29E2,1.528E3,9E0,4E0,2.8E1,7.57E2,5E0,5.2E1,2.3E1,1.1E1,2E0,9E1,1.39E2,1.524E3,4E0,6.08E2,1.49E2,2.4E1,2.8E1,1E1,1.3E1,9E0,8.1E1,4.1E1,9.8E1,3.7E1,1.487E3,1.56E2,4.52E2,1.31E2,1.8E1,4.6E1,3.5E1,3E1,1.1E1,7.4E1,2.4E1,6.09E2,8.78E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics index 0c51ba152..4895b8807 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics @@ -1,89 +1,89 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,maxOnHeapMem,0.28238225,12891675333.097458,7443156274.854311,455501414.0,8974132838.0,10119177830.0,10119177830.0,26104509235.0 -1,duration_max,0.12174313,19727.637005649718,132756.56647030968,15.0,1000.75,2626.5,6606.5,3183955.0 -2,executorCPUTime_mean,0.10378177,1600.0121232217436,8490.83160303521,3.875,196.08367473240125,438.1275815373426,1135.2854603729604,227947.84615384616 -3,scan_bw,0.10079862,991028.5544311537,1739237.4464631807,0.0,111648.31626667196,344807.51426898653,1052087.8925818277,17385294.707427993 -4,input_bytesRead_mean,0.080574825,19051772.387111697,29839180.431289956,3240.8,2282876.103862428,7538112.224328557,20442663.2280394,240185411.32144082 -5,sqlOp_SubqueryBroadcast,0.07300111,0.6070621468926554,0.48847220931507623,0.0,0.0,1.0,1.0,1.0 -6,cache_hit_ratio,0.04790776,0.18990465580425084,0.32957735575345715,0.0,0.0,0.0,0.30864069264631583,1.0 -7,shuffle_write_bw,0.03888357,165536.28360480047,275668.61585567426,0.0,3445.5108454312553,83232.4505533861,213410.4162089666,4415434.0 -8,sw_bytesWrittenRatio,0.03190458,0.7500153334529005,1.312331627655776,0.0,0.004156014806987636,0.20888208704072536,1.1450582662697832,19.10476372587304 -9,sw_recordsWritten_sum,0.03133821,1307752725.568644,4224481965.3663344,0.0,139053.75,16308835.0,463500001.0,44204763473.0 -10,jvmGCTime_mean,0.026384914,37.707004894329714,112.05477273251786,0.0,2.6427434456928838,10.6552217453505,35.56270150609702,2226.977412731006 -11,executorDeserializeCPUTime_mean,0.02529729,7.7308427479068875,16.187183743296377,0.01225114854517611,1.5320794973899026,2.4677083333333334,6.553110708090206,245.76923076923077 -12,peakExecutionMemory_max,0.023639923,514434918.05706215,1343644906.535583,0.0,68478953.0,185118727.5,448760080.5,25501360736.0 -13,duration_mean,0.022849066,2226.988951208108,8813.182199843042,13.333333333333334,319.1502445988216,776.5181426587314,1918.973089868859,228041.3076923077 -14,resultSize_max,0.020969415,3958608.534180791,11103446.031355042,1785.0,15101.25,54348.0,1857380.5,263794922.0 -15,duration_min,0.02067177,53.92768361581921,116.25739909610253,0.0,11.0,16.0,39.0,1779.0 -16,sr_localBytesReadRatio,0.019839268,0.41861586814223417,1.0093096602351779,0.0,0.0020370799199269708,0.06772948216509181,0.3323507501682036,16.779651636748564 -17,executorDeserializeTime_mean,0.016782796,18.885789433104375,44.57893656009577,0.04395604395604396,2.56471764117941,4.46117303124429,10.272696053946055,475.6 -18,maxMem,0.015070829,21686267078.83305,13741098115.368979,455501414.0,10119177830.0,10119177830.0,26104509235.0,43359456460.0 -19,scan_time,0.013603113,1520882.0240112995,7956731.9533324735,0.0,2684.5,49494.0,516427.25,217849366.0 -20,duration_sum,0.013003219,4842764.72118644,20681577.709232815,40.0,51743.25,347622.0,2003098.25,515693115.0 -21,sw_bytesWritten_mean,0.012463557,8701193.262865013,23683912.279130522,0.0,21461.232201626925,1031447.9115545701,6399900.097575081,314037255.60981447 -22,numExecutors,0.011927418,6.029378531073446,3.152654870163696,1.0,1.0,8.0,8.0,14.0 -23,executorRunTime_mean,0.011093886,2199.359015529836,8810.870593599537,6.0,307.07921482662414,748.9813360530342,1864.906949267008,228020.38461538462 -24,sr_totalBytesRead_mean,0.009971427,9202557.355776286,24510563.626570754,0.0,23858.06464995155,1237051.5861350573,7001522.360789239,314039311.06463194 -25,sw_writeTime_mean,0.009456708,65.46063250813955,282.56757535974504,0.0,4.31657853171011,17.049135220125788,45.60306478963834,6475.538414708389 -26,numTasks_sum,0.008844041,1383.4225988700564,3662.17887830497,3.0,130.0,419.5,1177.0,86122.0 -27,sr_totalBytesReadRatio,0.00783731,0.806944875525602,1.3776145394257457,0.0,0.004556759536252387,0.2297007756287479,1.1873950080800997,19.18647529336742 -28,sr_localBytesRead_mean,0.007435422,2910251.7965611317,7109002.559062359,0.0,9821.28396762487,375550.6402773764,2249818.307875526,104601898.28278689 -29,output_recordsWritten_sum,0.0072206557,82.66638418079096,1380.6943192952579,0.0,0.0,0.0,0.0,28800.0 -30,memoryBytesSpilledRatio,0.007082802,0.38330444729758456,2.2121805527381846,0.0,0.0,0.0,0.0,43.971493342592005 -31,executorCores,0.006804198,12.945762711864408,3.103634601050894,6.0,12.0,12.0,16.0,16.0 -32,sr_remoteBytesReadRatio,0.006617698,0.38832900738336773,0.9402463724732919,0.0,0.0,0.0002836891594572681,0.31458116619117393,16.78961797640585 -33,input_recordsRead_sum,0.0065157213,4254947665.6189265,13496209064.983376,0.0,41728108.0,454111279.5,2835348935.0,230401504404.0 -34,sr_localBlocksFetched_sum,0.0061623002,44000.88644067797,208616.43447519775,0.0,145.0,1412.5,15105.5,3860026.0 -35,sr_remoteBlocksFetched_sum,0.00609993,165629.2963276836,948144.8162985918,0.0,0.0,368.0,45471.0,21016512.0 -36,sr_fetchWaitTime_mean,0.005696238,53.589055836211855,852.5182710790447,0.0,0.0,0.0,0.199677245831092,26216.59561667383 -37,shuffle_read_bw,0.0051940535,200340881.16569823,2736126536.0809875,0.0,0.0,0.0,4142971.1076581576,107487374955.0 -38,data_size,0.00502704,354067441856.6409,1436943956289.1838,0.0,1947746994.0,25418067831.0,172467501151.5,22044104226233.0 -39,resultSerializationTime_sum,0.0048711114,73.159604519774,251.12253258121743,0.0,0.0,8.0,49.0,7044.0 -40,platform_databricks-aws,0.0046606925,0.08813559322033898,0.283532043314531,0.0,0.0,0.0,0.0,1.0 -41,sqlOp_Sort,0.003654258,0.581638418079096,0.49335983507473885,0.0,0.0,1.0,1.0,1.0 -42,sqlOp_SortMergeJoin,0.0031222708,0.4788135593220339,0.49962150606394323,0.0,0.0,0.0,1.0,1.0 -43,sqlOp_TakeOrderedAndProject,0.0030879476,0.10423728813559321,0.3056112883826109,0.0,0.0,0.0,0.0,1.0 -44,sr_remoteBytesRead_mean,0.0027204687,6292305.5592151545,21016177.50230388,0.0,0.0,4620.1696005917165,2150364.996089182,274787384.05385995 -45,sqlOp_Expand,0.0014509854,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 -46,diskBytesSpilled_mean,0.0010950331,2012928.371556133,13605693.750735264,0.0,0.0,0.0,0.0,374126772.9802514 -47,output_bytesWritten_mean,0.0010565275,18.163857860424017,514.6579916903137,0.0,0.0,0.0,0.0,25987.92 -48,sqlOp_DeserializeToObject,0.0009877413,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -49,output_bytesWrittenRatio,0.0008588691,4.878133481863413e-06,0.00011009204907379928,0.0,0.0,0.0,0.0,0.004454741188373785 -50,sqlOp_Window,0.00084713375,0.08870056497175141,0.2843512182402978,0.0,0.0,0.0,0.0,1.0 -51,platform_onprem,0.0008317245,0.42994350282485877,0.49513769985720407,0.0,0.0,0.0,1.0,1.0 -52,diskBytesSpilledRatio,0.00075665826,0.07709541615220665,0.425212311809422,0.0,0.0,0.0,0.0,7.517807102047801 -53,sqlOp_AQEShuffleRead,0.00055154914,0.8387005649717514,0.3678588769077452,0.0,1.0,1.0,1.0,1.0 -54,sqlOp_Filter,0.00044267304,0.9059322033898305,0.2919642550557146,0.0,1.0,1.0,1.0,1.0 -55,sqlOp_Subquery,0.0004069503,0.16638418079096046,0.3724777539299263,0.0,0.0,0.0,0.0,1.0 -56,sqlOp_BroadcastHashJoin,0.00037806501,0.9096045197740112,0.28678802468260717,0.0,1.0,1.0,1.0,1.0 -57,sqlOp_RunningWindowFunction,0.00029167705,0.011016949152542373,0.10439662207145209,0.0,0.0,0.0,0.0,1.0 -58,sqlOp_HashAggregate,0.0002790658,0.9709039548022599,0.1680995171957513,0.0,1.0,1.0,1.0,1.0 -59,sqlOp_BroadcastExchange,0.00027373768,0.6274011299435028,0.4835648944825209,0.0,0.0,1.0,1.0,1.0 -60,sqlOp_Scan orc ,0.00025952377,0.0768361581920904,0.26636892844122007,0.0,0.0,0.0,0.0,1.0 -61,sqlOp_GenerateBloomFilter,0.00023063934,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 -62,memoryBytesSpilled_mean,0.00020367035,7570388.018003908,50085507.71137506,0.0,0.0,0.0,0.0,874196883.2566372 -63,sqlOp_ObjectHashAggregate,0.00017525726,0.06016949152542373,0.23783419152332647,0.0,0.0,0.0,0.0,1.0 -64,sqlOp_BatchEvalPython,0.00017140161,0.005367231638418079,0.07307484463606036,0.0,0.0,0.0,0.0,1.0 -65,sqlOp_Scan parquet ,0.0001439218,0.8997175141242938,0.30041871387962593,0.0,1.0,1.0,1.0,1.0 -66,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.00012968264,0.05593220338983051,0.22982322022384652,0.0,0.0,0.0,0.0,1.0 -67,sqlOp_BroadcastNestedLoopJoin,0.00010052012,0.0384180790960452,0.19223050973607964,0.0,0.0,0.0,0.0,1.0 -68,sqlOp_OutputAdapter,7.55314e-05,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 -69,sqlOp_Exchange,4.260147e-05,0.980225988700565,0.13924251056124148,0.0,1.0,1.0,1.0,1.0 -70,sqlOp_HashAggregatePrefixGroupingSets,1.8387198e-05,0.009322033898305085,0.09611317867107047,0.0,0.0,0.0,0.0,1.0 -71,sqlOp_WindowGroupLimit,6.442981e-06,0.011864406779661017,0.10829106764028926,0.0,0.0,0.0,0.0,1.0 -72,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_SubqueryOutputBroadcast,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 +0,maxOnHeapMem,0.24525835,12891675333.097458,7443156274.854312,455501414.0,8974132838.0,10119177830.0,10119177830.0,26104509235.0 +1,duration_max,0.122840814,19727.637005649718,132756.56647030966,15.0,1000.75,2626.5,6606.5,3183955.0 +2,executorCPUTime_mean,0.116636105,1600.0121232217436,8490.83160303521,3.875,196.08367473240125,438.1275815373426,1135.2854603729604,227947.84615384616 +3,scan_bw,0.09914459,991028.5544311537,1739237.4464631807,0.0,111648.31626667196,344807.51426898653,1052087.8925818277,17385294.707427993 +4,input_bytesRead_mean,0.08676977,19051772.3871117,29839180.431289956,3240.8,2282876.103862428,7538112.224328557,20442663.2280394,240185411.32144082 +5,sqlOp_SubqueryBroadcast,0.066191964,0.6070621468926554,0.48847220931507623,0.0,0.0,1.0,1.0,1.0 +6,cache_hit_ratio,0.057533674,0.18990465580425084,0.32957735575345715,0.0,0.0,0.0,0.30864069264631583,1.0 +7,shuffle_write_bw,0.042041168,165536.28360480044,275668.61585567426,0.0,3445.5108454312553,83232.4505533861,213410.4162089666,4415434.0 +8,maxMem,0.033244178,21686267078.83305,13741098115.368979,455501414.0,10119177830.0,10119177830.0,26104509235.0,43359456460.0 +9,executorDeserializeCPUTime_mean,0.02722386,7.730842747906887,16.187183743296377,0.01225114854517611,1.5320794973899026,2.4677083333333334,6.553110708090206,245.76923076923077 +10,peakExecutionMemory_max,0.026185669,514434918.05706215,1343644906.5355828,0.0,68478953.0,185118727.5,448760080.5,25501360736.0 +11,jvmGCTime_mean,0.023969578,37.70700489432971,112.05477273251788,0.0,2.6427434456928838,10.6552217453505,35.56270150609702,2226.977412731006 +12,sw_recordsWritten_sum,0.023896383,1307752725.568644,4224481965.3663344,0.0,139053.75,16308835.0,463500001.0,44204763473.0 +13,sw_bytesWrittenRatio,0.023290131,0.7500153334529006,1.312331627655776,0.0,0.004156014806987636,0.20888208704072536,1.1450582662697832,19.10476372587304 +14,resultSize_max,0.022359248,3958608.534180791,11103446.031355042,1785.0,15101.25,54348.0,1857380.5,263794922.0 +15,sr_localBytesReadRatio,0.020905973,0.4186158681422342,1.0093096602351777,0.0,0.0020370799199269708,0.06772948216509181,0.3323507501682036,16.779651636748564 +16,numExecutors,0.018349394,6.029378531073446,3.1526548701636963,1.0,1.0,8.0,8.0,14.0 +17,duration_mean,0.016111324,2226.988951208108,8813.182199843044,13.333333333333334,319.1502445988216,776.5181426587314,1918.973089868859,228041.3076923077 +18,duration_min,0.0159653,53.92768361581921,116.25739909610253,0.0,11.0,16.0,39.0,1779.0 +19,executorDeserializeTime_mean,0.01589882,18.885789433104375,44.57893656009577,0.04395604395604396,2.56471764117941,4.46117303124429,10.272696053946055,475.6 +20,scan_time,0.015392276,1520882.0240112995,7956731.9533324735,0.0,2684.5,49494.0,516427.25,217849366.0 +21,sw_bytesWritten_mean,0.012503516,8701193.262865013,23683912.279130522,0.0,21461.232201626925,1031447.9115545701,6399900.097575081,314037255.60981447 +22,sr_localBytesRead_mean,0.010507526,2910251.7965611313,7109002.559062359,0.0,9821.28396762487,375550.6402773764,2249818.307875526,104601898.28278689 +23,duration_sum,0.010165823,4842764.72118644,20681577.709232815,40.0,51743.25,347622.0,2003098.25,515693115.0 +24,platform_databricks-aws,0.009524851,0.08813559322033898,0.283532043314531,0.0,0.0,0.0,0.0,1.0 +25,executorRunTime_mean,0.009243112,2199.3590155298366,8810.870593599537,6.0,307.07921482662414,748.9813360530342,1864.906949267008,228020.38461538462 +26,numTasks_sum,0.008401272,1383.4225988700564,3662.17887830497,3.0,130.0,419.5,1177.0,86122.0 +27,sr_totalBytesReadRatio,0.0075071068,0.806944875525602,1.3776145394257457,0.0,0.004556759536252387,0.2297007756287479,1.1873950080800997,19.18647529336742 +28,output_recordsWritten_sum,0.007329679,82.66638418079096,1380.6943192952579,0.0,0.0,0.0,0.0,28800.0 +29,executorCores,0.0070164194,12.945762711864408,3.103634601050894,6.0,12.0,12.0,16.0,16.0 +30,input_recordsRead_sum,0.0068921465,4254947665.6189265,13496209064.983372,0.0,41728108.0,454111279.5,2835348935.0,230401504404.0 +31,sw_writeTime_mean,0.0067101875,65.46063250813955,282.56757535974504,0.0,4.31657853171011,17.049135220125788,45.60306478963834,6475.538414708389 +32,resultSerializationTime_sum,0.006529984,73.159604519774,251.12253258121743,0.0,0.0,8.0,49.0,7044.0 +33,sr_fetchWaitTime_mean,0.0058721746,53.589055836211855,852.5182710790448,0.0,0.0,0.0,0.199677245831092,26216.59561667383 +34,data_size,0.005667293,354067441856.6409,1436943956289.1838,0.0,1947746994.0,25418067831.0,172467501151.5,22044104226233.0 +35,memoryBytesSpilledRatio,0.0055548237,0.38330444729758456,2.2121805527381846,0.0,0.0,0.0,0.0,43.971493342592005 +36,sr_totalBytesRead_mean,0.0048567513,9202557.355776286,24510563.626570754,0.0,23858.06464995155,1237051.5861350573,7001522.360789239,314039311.06463194 +37,shuffle_read_bw,0.0046514957,200340881.16569823,2736126536.080987,0.0,0.0,0.0,4142971.1076581576,107487374955.0 +38,sr_localBlocksFetched_sum,0.004350457,44000.88644067797,208616.4344751978,0.0,145.0,1412.5,15105.5,3860026.0 +39,sr_remoteBlocksFetched_sum,0.0043445663,165629.2963276836,948144.8162985918,0.0,0.0,368.0,45471.0,21016512.0 +40,sqlOp_SortMergeJoin,0.0042423685,0.4788135593220339,0.49962150606394323,0.0,0.0,0.0,1.0,1.0 +41,sqlOp_TakeOrderedAndProject,0.0037542866,0.10423728813559321,0.3056112883826109,0.0,0.0,0.0,0.0,1.0 +42,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0032178024,0.05593220338983051,0.22982322022384652,0.0,0.0,0.0,0.0,1.0 +43,sr_remoteBytesReadRatio,0.0028581598,0.38832900738336773,0.9402463724732919,0.0,0.0,0.0002836891594572681,0.31458116619117393,16.78961797640585 +44,sr_remoteBytesRead_mean,0.0027610108,6292305.559215155,21016177.50230388,0.0,0.0,4620.1696005917165,2150364.996089182,274787384.05385995 +45,sqlOp_Sort,0.002645748,0.581638418079096,0.49335983507473885,0.0,0.0,1.0,1.0,1.0 +46,sqlOp_HashAggregate,0.0014733697,0.9709039548022599,0.1680995171957513,0.0,1.0,1.0,1.0,1.0 +47,sqlOp_BroadcastExchange,0.0011297737,0.6274011299435028,0.4835648944825208,0.0,0.0,1.0,1.0,1.0 +48,sqlOp_Expand,0.0009254089,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 +49,diskBytesSpilled_mean,0.00091879274,2012928.3715561333,13605693.750735264,0.0,0.0,0.0,0.0,374126772.9802514 +50,sqlOp_Window,0.0006066229,0.08870056497175141,0.2843512182402978,0.0,0.0,0.0,0.0,1.0 +51,sqlOp_Subquery,0.00060174114,0.16638418079096046,0.3724777539299263,0.0,0.0,0.0,0.0,1.0 +52,sqlOp_Filter,0.00053117686,0.9059322033898305,0.2919642550557146,0.0,1.0,1.0,1.0,1.0 +53,sqlOp_BroadcastHashJoin,0.00048093367,0.9096045197740112,0.28678802468260717,0.0,1.0,1.0,1.0,1.0 +54,platform_onprem,0.00047731653,0.42994350282485877,0.49513769985720407,0.0,0.0,0.0,1.0,1.0 +55,sqlOp_RunningWindowFunction,0.0004683478,0.011016949152542373,0.1043966220714521,0.0,0.0,0.0,0.0,1.0 +56,sqlOp_Scan orc ,0.00044496782,0.0768361581920904,0.26636892844122,0.0,0.0,0.0,0.0,1.0 +57,diskBytesSpilledRatio,0.0004446417,0.07709541615220665,0.425212311809422,0.0,0.0,0.0,0.0,7.517807102047801 +58,sqlOp_AQEShuffleRead,0.0004128795,0.8387005649717514,0.3678588769077452,0.0,1.0,1.0,1.0,1.0 +59,sqlOp_GenerateBloomFilter,0.00035020744,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 +60,output_bytesWrittenRatio,0.00034191555,4.878133481863412e-06,0.00011009204907379927,0.0,0.0,0.0,0.0,0.004454741188373785 +61,sqlOp_ObjectHashAggregate,0.00033151326,0.06016949152542373,0.23783419152332647,0.0,0.0,0.0,0.0,1.0 +62,output_bytesWritten_mean,0.0003017444,18.163857860424017,514.6579916903138,0.0,0.0,0.0,0.0,25987.92 +63,sqlOp_OutputAdapter,0.0003016191,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 +64,sqlOp_BroadcastNestedLoopJoin,0.00019190965,0.0384180790960452,0.19223050973607964,0.0,0.0,0.0,0.0,1.0 +65,sqlOp_Scan parquet ,0.00017240988,0.8997175141242938,0.3004187138796259,0.0,1.0,1.0,1.0,1.0 +66,sqlOp_BatchEvalPython,0.00015781133,0.005367231638418079,0.07307484463606036,0.0,0.0,0.0,0.0,1.0 +67,sqlOp_DeserializeToObject,0.00011716436,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 +68,memoryBytesSpilled_mean,8.455912e-05,7570388.018003908,50085507.71137506,0.0,0.0,0.0,0.0,874196883.2566372 +69,sqlOp_WindowGroupLimit,4.368235e-05,0.011864406779661017,0.10829106764028926,0.0,0.0,0.0,0.0,1.0 +70,platform_dataproc,2.9462633e-05,0.16751412429378532,0.3734870106579372,0.0,0.0,0.0,0.0,1.0 +71,sqlOp_HashAggregatePrefixGroupingSets,2.4900206e-05,0.009322033898305085,0.09611317867107047,0.0,0.0,0.0,0.0,1.0 +72,sqlOp_SubqueryOutputBroadcast,1.1342636e-05,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 +73,sqlOp_Project,0.0,0.9850282485875707,0.1214568451648366,0.0,1.0,1.0,1.0,1.0 +74,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_ReusedSort,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 +76,sqlOp_SerializeFromObject,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 77,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,sqlOp_Scan text ,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -80,sqlOp_SerializeFromObject,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -81,sqlOp_ReusedSort,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 -82,sqlOp_Project,0.0,0.9850282485875707,0.12145684516483662,0.0,1.0,1.0,1.0,1.0 -83,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,sqlOp_Scan text ,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 +83,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 85,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 86,sqlOp_MapElements,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 87,sqlOp_LocalTableScan,0.0,0.0011299435028248588,0.03360038173899415,0.0,0.0,0.0,0.0,1.0 @@ -95,15 +95,15 @@ 93,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 94,maxOffHeapMem,0.0,8794591745.735594,15744974458.616884,0.0,0.0,0.0,0.0,36974886912.0 95,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,platform_databricks-azure,0.0,0.07542372881355933,0.26411114073704656,0.0,0.0,0.0,0.0,1.0 -97,platform_dataproc,0.0,0.16751412429378532,0.3734870106579372,0.0,0.0,0.0,0.0,1.0 +96,sqlOp_Exchange,0.0,0.980225988700565,0.13924251056124148,0.0,1.0,1.0,1.0,1.0 +97,platform_databricks-azure,0.0,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 98,platform_emr,0.0,0.08926553672316384,0.2851669203761615,0.0,0.0,0.0,0.0,1.0 99,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 100,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_ColumnarToRow,0.0,0.9765536723163842,0.15133759346164188,0.0,1.0,1.0,1.0,1.0 +101,sqlOp_ColumnarToRow,0.0,0.9765536723163842,0.1513375934616419,0.0,1.0,1.0,1.0,1.0 102,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 103,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 104,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 105,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 106,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 +107,sqlOp_WindowSort,0.0,0.000847457627118644,0.02910289851903554,0.0,0.0,0.0,0.0,1.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json index 0b1f885b3..bfe96460c 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[1.5103106E-2,-2.2944748E-1,4.0485805E-1,-5.2396727E-1,-5.360323E-2,7.115805E-1,-2.6379624E-1,-3.949695E-1,-4.2585783E-2,-1.771069E-1,8.511205E-2,4.611889E-1,1.0766281E0,2.0126725E-2,-6.011713E-1,-4.39699E-1,2.4540755E-3,-2.5400817E-1,-1.0922299E-1,2.1650563E-1,-2.3319207E-1,-1.3541527E-2,5.946551E-1,5.5333883E-2,1.7301071E-2,-1.22435205E-1,1.9814072E-2,-4.192214E-2,-1.1715452E-2,-2.594052E-1,-5.554998E-1,-3.1053467E-4,-3.0310825E-1,8.845983E-2,-1.6904932E-1,2.8078866E-1,6.388342E-2,-1.6651804E-2,-3.3995867E-2,7.99645E-3,-1.5311554E-1,6.501042E-1,6.8630027E-3,-1.1348052E-2,5.7094876E-2,-4.0573996E-2,-3.4288028E-1,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,-3.68585E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,-1.9150509E-1,7.322452E-3,1.7035466E-2,-6.4032795E-3,1.3852504E-1,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,45,47,-1,49,51,53,55,57,-1,59,-1,61,63,-1,-1,65,67,69,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8298021E1,6.0830874E0,1.536447E1,1.6471167E0,1.2999624E0,4.1651173E0,2.3110242E0,6.9953203E-1,0E0,1.810708E-1,1.5561198E0,2.0355868E0,4.665451E-1,8.22276E-1,8.244722E-1,5.031233E-1,0E0,2.1691906E-1,2.8696674E-1,2.3714864E-1,2.271381E-1,2.2374773E-1,5.371609E-1,0E0,0E0,2.3201612E-1,0E0,0E0,0E0,2.418254E-1,3.252077E-2,0E0,1.0391843E-1,5.7198662E-2,4.2966306E-2,1.09799504E-1,1.495303E-1,0E0,2.0135637E-2,0E0,2.2688761E-2,2.5058937E-1,0E0,0E0,6.3281864E-2,1.3045962E-2,8.805108E-2,0E0,0E0,0E0,2.4855494E-2,0E0,0E0,0E0,2.7623773E-2,0E0,0E0,0E0,6.379056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,34,34,35,35,36,36,38,38,40,40,41,41,44,44,45,45,46,46,50,50,54,54,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,46,48,-1,50,52,54,56,58,-1,60,-1,62,64,-1,-1,66,68,70,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.039435E-3,6.627576E7,6.8652206E9,4.5570032E2,4.7162656E3,4E0,4.1211517E1,-4.2585783E-2,1.1383354E6,9.518453E6,1E0,7.317201E0,1.5312085E0,7.588906E-2,5.1147053E5,2.4540755E-3,4.7812922E5,1.630886E6,8.729897E0,1.2534044E3,8.78E2,2.709E4,5.5333883E-2,1.7301071E-2,6.7995167E0,1.9814072E-2,-4.192214E-2,-1.1715452E-2,2.77E2,1E0,-3.1053467E-4,2.8880866E0,4.35E2,2.6374866E1,5.716463E2,2.831E3,-1.6651804E-2,9.17E2,7.99645E-3,3.349E4,1.5145266E6,6.8630027E-3,-1.1348052E-2,6.695E3,9.513E3,1.4377E4,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,1.0491824E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,4.2658337E2,7.322452E-3,1.7035466E-2,-6.4032795E-3,5.586E3,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"split_indices":[53,39,46,5,53,53,8,59,0,52,46,102,54,58,39,29,0,52,46,55,4,0,9,0,0,55,0,0,0,0,105,0,57,0,59,53,2,0,0,0,10,49,0,0,2,9,9,0,0,0,40,0,0,0,53,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.17E2,7.3E1,4.3E1,7.4E1,5E1,2.3E1,3.3E1,1E1,3.9E1,3.5E1,3.1E1,1.9E1,1.3E1,1E1,3E1,3E0,1.7E1,2.2E1,2.5E1,1E1,7E0,2.4E1,1.7E1,2E0,1E1,3E0,5E0,5E0,1.3E1,1.7E1,3E0,1.4E1,5E0,1.7E1,1.7E1,8E0,6E0,4E0,3E0,4E0,2.1E1,3E0,6E0,4E0,4E0,9E0,1.4E1,3E0,5E0,9E0,2E0,3E0,3E0,1.4E1,7E0,1E1,2E0,6E0,2E0,2E0,2E0,2E0,5E0,1.6E1,2E0,2E0,2E0,2E0,5E0,4E0,2E0,7E0,1.2E1,2E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-3.3002432E-2,-1.7783551E-1,5.082772E-1,-2.7808794E-1,6.895006E-2,7.5295E-2,8.787938E-1,-1.8593931E-1,-9.442671E-1,3.6505783E-1,-2.5721067E-1,3.1757823E-1,-1.1269199E-1,4.4750738E-1,1.012557E0,-2.7686968E-1,-1.1995372E-3,-1.790164E-2,-4.942715E-2,1.0660491E-1,5.7554835E-1,3.059845E-2,-4.588169E-1,5.580259E-3,1.7479694E-2,-1.3313731E-2,4.1303214E-2,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,-2.2212821E-1,-4.4453266E-1,7.171647E-2,-2.8227565E-1,2.1667868E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,-7.6152295E-2,1.4846555E-2,-9.6043665E-3,-5.975643E-1,7.728357E-3,-3.7457045E-2,-2.4542458E-1,2.0091075E-3,-7.810127E-3,-2.595061E-2,-1.12920456E-1,1.1881216E-1,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,-1,-1,57,-1,59,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5220662E1,3.804525E0,6.511341E0,6.5156603E0,4.4399395E0,9.4671035E-1,8.160591E-1,1.6291182E0,1.6117287E-1,1.2442894E0,1.2905694E0,3.4778595E-2,3.0859464E-1,2.4943113E-2,1.920681E-1,4.978342E-1,6.9657713E-1,0E0,0E0,3.9868122E-1,7.959266E-1,3.1622946E-1,3.536806E-1,0E0,0E0,0E0,7.072824E-2,0E0,0E0,0E0,0E0,3.2309484E-1,2.9235983E-1,2.4819759E-1,3.8475174E-1,7.987356E-2,0E0,0E0,0E0,3.175994E-2,0E0,0E0,1.1517382E-1,0E0,1.07335895E-2,1.10358E-1,0E0,0E0,0E0,1.0522947E-2,4.5623663E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,26,26,31,31,32,32,33,33,34,34,35,35,39,39,42,42,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,-1,-1,58,-1,60,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5085194E0,1.2871602E3,1.020266E3,8.260109E9,6.971004E7,2.3238E4,1.9107767E0,4.684559E2,2.6E1,5.5326223E9,8.462096E11,5.380165E0,3.236715E0,5.8229775E3,2.8626094E5,1.03431056E6,5.5015685E9,-1.790164E-2,-4.942715E-2,6.4348924E-1,5.6759316E7,5.4827127E0,1.5214152E5,5.580259E-3,1.7479694E-2,-1.3313731E-2,5.884028E0,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,1.1556807E0,5.7625353E-1,7.1568984E7,2.533331E6,4.1808826E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,8.307621E7,1.4846555E-2,-9.6043665E-3,2.012E3,7.728357E-3,1.009E3,5.1146146E-2,2.0091075E-3,-7.810127E-3,-2.595061E-2,4.5E1,7.384995E4,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"split_indices":[43,53,53,5,46,9,43,53,3,5,32,54,57,4,29,29,5,0,0,28,46,58,34,0,0,0,54,0,0,0,0,43,28,7,1,28,0,0,0,46,0,0,0,0,0,28,0,0,0,3,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.92E2,1.52E2,4E1,1.08E2,4.4E1,1.9E1,2.1E1,9.6E1,1.2E1,2.3E1,2.1E1,8E0,1.1E1,6E0,1.5E1,6.4E1,3.2E1,2E0,1E1,1.1E1,1.2E1,9E0,1.2E1,2E0,6E0,5E0,6E0,4E0,2E0,2E0,1.3E1,5E1,1.4E1,2.6E1,6E0,8E0,3E0,9E0,3E0,7E0,2E0,5E0,7E0,2E0,4E0,4.6E1,4E0,4E0,1E1,5E0,2.1E1,4E0,2E0,5E0,3E0,4E0,3E0,2E0,5E0,2E0,2E0,6E0,4E1,3E0,2E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-7.2186072E-3,-1.7554772E-1,4.6836394E-1,-1.3343666E-1,-7.98081E-1,4.8964757E-2,7.305873E-1,-2.4706829E-1,1.5109251E-1,-1.1278819E-2,-9.1069216E-1,-2.2222637E-1,2.0841512E-1,5.310632E-1,1.0273924E0,-2.0253772E-1,-7.193482E-1,8.116926E-2,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,2.4997413E-1,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,-4.6462825E-1,-1.3208786E-1,-1.10341525E-2,-4.028151E-2,-1.6114761E-1,2.4269797E-1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,-5.287513E-1,-9.0207495E-2,-3.1034762E-1,1.1962035E-2,-2.5665852E-1,3.4205204E-1,-1.1185924E-2,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7454023E1,4.1142073E0,6.232873E0,4.9517193E0,4.2668343E-1,1.0408703E0,1.5270805E0,2.1466722E0,2.0214696E0,0E0,3.1103182E-1,1.4763036E-1,1.022948E-1,7.396126E-2,9.1228485E-3,1.8035164E0,2.927456E-1,1.6438966E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2034745E-2,0E0,0E0,0E0,0E0,2.2022247E-1,5.7671654E-1,0E0,0E0,2.841458E-1,6.34436E-1,0E0,0E0,0E0,1.3254166E-2,7.471074E-1,2.088238E-1,2.8850803E-1,9.1903985E-2,4.5164657E-1,1.8377072E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24,29,29,30,30,33,33,34,34,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,7.7567725E2,1.0523087E3,4.14E2,2.6263257E2,1.5605432E9,6.8652206E9,2.3274304E2,-1.1278819E-2,3.7284137E2,2.5546576E4,2.5086822E8,6.8734935E6,3.4456624E7,1.142E3,1.9E1,7.157224E0,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,1.630886E6,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,2.7373068E7,3.0143394E7,-1.10341525E-2,-4.028151E-2,2E0,1.0943299E1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,5.9215684E0,7.54579E5,1.5641167E5,6.802721E-3,1.0180757E0,2.4794E4,9.875E3,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"split_indices":[43,46,53,53,0,53,7,5,57,0,59,34,7,48,48,2,3,55,0,0,0,0,0,0,46,0,0,0,0,5,46,0,0,8,55,0,0,0,57,29,34,58,40,9,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.6E2,5.6E1,1.51E2,9E0,2.2E1,3.4E1,1.08E2,4.3E1,2E0,7E0,8E0,1.4E1,2.2E1,1.2E1,1E2,8E0,4E1,3E0,5E0,2E0,3E0,5E0,3E0,1.1E1,1.8E1,4E0,1E1,2E0,2E1,8E1,2E0,6E0,1.6E1,2.4E1,5E0,6E0,5E0,1.5E1,6.6E1,1.4E1,6E0,1E1,1.7E1,7E0,1.3E1,2E0,4.6E1,2E1,1.2E1,2E0,3E0,3E0,6E0,4E0,1.4E1,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-9.860434E-3,-1.3550225E-1,4.1085625E-1,-4.535459E-1,-6.382216E-2,2.3670215E-2,7.524566E-1,-5.6793636E-1,9.8671935E-2,-9.873285E-2,4.3589786E-1,2.2890316E-1,-1.3927497E-1,9.374098E-1,4.9539495E-1,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,-4.404742E-2,-6.1913097E-1,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,-2.5028974E-1,-2.472767E-2,1.5874594E-2,4.9255155E-2,5.498805E-1,6.3501373E-3,-1.7045158E-1,8.096087E-2,-4.8548445E-2,-3.0521184E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,-6.917094E-2,1.6786462E-2,3.2274876E-2,-2.1812399E-1,-5.0437325E-3,1.5704603E-1,-2.1648994E-2,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,39,-1,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1261909E1,3.7098274E0,6.463071E0,1.9510937E0,2.3981898E0,8.344817E-1,7.7176666E-1,1.549511E-1,3.8158774E-1,3.5869198E0,2.744478E-1,1.8382758E-1,1.7184111E-1,3.0896282E-1,1.867218E-1,0E0,0E0,0E0,0E0,1.8467818E0,1.121139E0,0E0,0E0,0E0,0E0,1.3394743E-2,3.954195E-2,0E0,0E0,2.6690483E-2,0E0,4.5625246E-1,4.6401808E-1,0E0,7.3043644E-2,0E0,0E0,0E0,1.0194156E-2,0E0,0E0,3.1877828E-1,1.343909E-1,8.3303446E-1,9.282285E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,25,25,26,26,29,29,31,31,32,32,34,34,38,38,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,40,-1,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.7575762E4,1.1382212E3,2.0199957E3,1.7730763E12,1.5593E4,8.812601E-1,4.6490747E2,3.20084E6,1.775894E10,4.6673E4,8.147317E0,3.2851852E2,4.558E3,1.4118051E7,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,5.470729E2,8.92851E6,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,6.314915E-1,4.1546512E2,1.5874594E-2,4.9255155E-2,3.76835E7,6.3501373E-3,3.9E1,3.291238E7,-4.8548445E-2,6.881048E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,8.27E2,1.6786462E-2,3.2274876E-2,4.7095413E2,6.9327216E0,1.4154823E-1,2.4654E4,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"split_indices":[43,52,4,4,32,9,58,53,7,5,10,54,4,2,1,0,0,0,0,53,1,0,0,0,0,39,4,0,0,49,0,3,46,0,35,0,0,0,0,0,0,4,54,39,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.63E2,4.8E1,2.9E1,1.34E2,2.3E1,2.5E1,2.4E1,5E0,1.26E2,8E0,1E1,1.3E1,1.3E1,1.2E1,1.4E1,1E1,3E0,2E0,1.15E2,1.1E1,4E0,4E0,8E0,2E0,6E0,7E0,2E0,1.1E1,1E1,2E0,5.7E1,5.8E1,4E0,7E0,4E0,2E0,2E0,5E0,5E0,5E0,4.4E1,1.3E1,3.3E1,2.5E1,5E0,2E0,3E0,2E0,1.2E1,3.2E1,7E0,6E0,1.9E1,1.4E1,2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.037411E-2,-1.7082255E-1,4.0125197E-1,-1.368692E-1,-4.7962084E-2,1.3270798E-1,7.9803574E-1,-2.2494297E-1,7.1677774E-2,2.673719E-1,-1.0539308E-1,1.9987479E-2,4.1916054E-2,-5.089811E-1,-1.4352345E-1,1.7057969E-1,-1.6686903E-2,1.1773016E-1,1.8603355E-2,-1.25300465E-2,-9.196732E-3,-3.748573E-1,-3.2896925E-2,-2.6696217E-1,-5.671464E-2,6.511845E-2,4.1393185E-1,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,-4.2942005E-1,-5.2706E-2,-3.20558E-1,5.7432473E-2,-1.8276703E-1,2.8034353E-1,-4.025757E-2,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,-1,27,-1,-1,29,31,-1,33,35,37,39,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3262604E1,4.125049E0,5.7545595E0,2.7528837E0,0E0,1.1183422E0,1.8529606E-1,2.355206E0,1.903528E0,3.3322942E-1,1.8780968E-1,0E0,0E0,2.6173544E-1,8.7425184E-1,9.226215E-1,0E0,4.9472928E-2,0E0,0E0,1.1085327E-1,8.237338E-2,0E0,3.7353468E-1,7.2992265E-1,6.2363416E-1,1.9650996E-1,0E0,0E0,0E0,0E0,0E0,7.688856E-2,1.9526544E-1,2.2509766E-1,3.0900213E-1,1.7987198E-1,2.4989247E-1,3.8682532E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,20,20,21,21,23,23,24,24,25,25,26,26,32,32,33,33,34,34,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,-1,28,-1,-1,30,32,-1,34,36,38,40,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,1.3453049E3,1.0648859E3,-4.7962084E-2,2.4608E4,2.7350403E5,1.7575762E4,9.820072E7,5.433479E2,2.5546576E4,1.9987479E-2,4.1916054E-2,1.3900659E6,2.802166E2,1.6E1,-1.6686903E-2,2.668E3,1.8603355E-2,-1.25300465E-2,1.9E1,2.9270105E2,-3.2896925E-2,2.124077E1,9.222491E0,3.10214E5,2.4986116E7,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,6.1E1,1.5497297E1,3.079376E8,6.735744E1,3.1E1,5.1E1,6.8125E0,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"split_indices":[43,46,53,53,0,9,29,52,46,53,34,0,0,29,53,8,0,2,0,0,3,34,0,59,55,30,49,0,0,0,0,0,3,59,5,59,3,3,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.53E2,5.4E1,1.48E2,5E0,3.3E1,2.1E1,1.04E2,4.4E1,2.1E1,1.2E1,4E0,1.7E1,2.2E1,8.2E1,3.6E1,8E0,1E1,1.1E1,4E0,8E0,1.4E1,8E0,3.3E1,4.9E1,2.6E1,1E1,4E0,6E0,3E0,5E0,4E0,1E1,7E0,2.6E1,2.6E1,2.3E1,8E0,1.8E1,8E0,2E0,8E0,2E0,3E0,4E0,1.3E1,1.3E1,2.1E1,5E0,9E0,1.4E1,6E0,2E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.8539885E-2,-1.315011E-1,4.7729447E-1,-3.645807E-1,-4.856526E-2,3.3575363E-2,7.048596E-1,-4.006475E-1,1.0404514E-2,-1.2890418E-1,1.0923791E-1,2.1161272E-1,-2.1542291E-1,4.2066466E-2,7.948288E-1,-5.199407E-1,-2.0412306E-1,-1.0585207E-1,-3.185486E-2,-2.5976712E-1,1.9807549E-1,1.8924989E-3,3.4906948E-1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,-5.954765E-1,-2.6221934E-1,-4.2201504E-1,1.2239022E-2,-1.8437941E-1,-8.044045E-3,-5.61047E-2,-2.8251013E-2,2.2316225E-2,1.2032818E-1,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,-1.095318E-1,-2.132905E-1,-1.3484026E-2,1.0698679E-1,-1.6088222E-1,-6.4098774E-3,3.0856705E-3,3.494343E-1,6.278915E-2,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.39668255E1,2.9398837E0,5.0250864E0,9.200578E-1,1.4593433E0,8.4354407E-1,1.9343872E0,7.847419E-1,0E0,8.516334E-1,1.3299325E0,2.4759096E-1,8.105555E-2,1.5741335E-1,1.7285347E-1,2.9159975E-1,7.59462E-1,5.668815E-1,0E0,4.9000573E-1,5.9287274E-1,0E0,1.3428587E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.4581165E-2,1.9908011E-2,1.5304077E-1,2.1783859E-1,1.9969559E-1,6.1496985E-1,6.256449E-2,0E0,0E0,3.2080543E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8989278E-2,1.5572238E-1,2.168184E-2,1.3415802E-1,2.2903234E-1,0E0,0E0,5.3615987E-2,2.547765E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,48,48,49,49,50,50,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.130317E8,6.794883E4,1E0,4.5342876E3,6.787619E2,9.820072E7,3.788E3,1.4862166E1,1.0404514E-2,4.462437E7,7.443E3,3.5073215E3,1.2895599E4,6.09E2,4.2414474E0,6.1E1,5.0548022E2,2.98022E2,-3.185486E-2,9.64E2,1.20171E5,1.8924989E-3,5.1E1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,5.0446276E7,1.0804272E3,1.0363E4,4.1749332E4,2.0890524E6,7.6420575E6,1.1484E4,-2.8251013E-2,2.2316225E-2,1.2117E4,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,1.7488E4,2.124077E1,4.35E2,1.725E3,6.9789816E7,-6.4098774E-3,3.0856705E-3,4E0,6.1108776E7,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"split_indices":[7,51,102,4,53,46,2,59,0,46,9,53,4,0,58,3,4,53,0,0,30,0,3,0,0,0,0,0,0,46,34,9,33,51,46,2,0,0,9,0,0,0,0,0,0,0,0,0,9,59,0,2,12,0,0,8,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.52E2,4.9E1,3.9E1,1.13E2,1.7E1,3.2E1,3.7E1,2E0,7.5E1,3.8E1,1E1,7E0,4E0,2.8E1,2.2E1,1.5E1,7.3E1,2E0,7E0,3.1E1,5E0,5E0,3E0,4E0,2E0,2E0,2.4E1,4E0,1.6E1,6E0,7E0,8E0,4E1,3.3E1,5E0,2E0,6E0,2.5E1,2E0,3E0,1.1E1,5E0,3E0,3E0,2E0,5E0,3E0,5E0,3.4E1,6E0,1.9E1,1.4E1,3E0,2E0,4E0,2.1E1,3E0,2E0,8E0,2.6E1,2E0,4E0,2E0,1.7E1,5E0,9E0,2E0,2E0,1.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-2.776328E-2,-1.2117824E-1,3.86292E-1,-5.909288E-2,-5.795699E-1,5.5683386E-1,-2.6300025E-1,-2.0977455E-1,9.621003E-2,-8.023927E-1,-1.1718366E-1,2.3014233E-1,6.94869E-1,-1.3781548E-3,-3.7562144E-1,-3.571755E-1,-1.0406611E-1,1.914385E-1,-8.06779E-2,-2.3914319E-2,-4.741177E-2,9.839836E-3,-3.492453E-1,3.794242E-1,-6.1298814E-3,3.9063934E-2,3.491601E-1,-2.482055E-2,-3.466254E-3,-4.0868425E-1,-5.727207E-2,-1.4600524E-1,2.0078115E-2,7.405951E-2,3.1055027E-1,-1.1359027E-1,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,-4.7098124E-1,-1.5451024E-1,2.9516693E-3,-6.394594E-3,4.4541497E-2,-1.712155E-1,7.7366033E-3,-5.9316065E-2,1.5788026E-1,-6.221433E-2,3.575082E-1,-1.2339829E-3,-7.365338E-2,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,61,63,-1,65,67,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.401466E0,5.003068E0,4.575707E0,3.694187E0,2.0696063E0,1.2831182E0,2.2723442E-1,1.2093525E0,1.3293368E0,2.6940823E-1,6.6352755E-1,6.3254225E-1,5.397568E-1,0E0,2.0912564E-1,4.8636627E-1,2.528435E-1,6.9004285E-1,3.1393635E-1,0E0,0E0,0E0,6.1558306E-2,2.0225549E-1,0E0,0E0,7.9099655E-2,0E0,0E0,3.7058783E-1,6.041512E-2,1.8056059E-1,1.5278538E-1,3.1823194E-1,4.0340137E-1,1.3923776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.6962032E-2,1.4837602E-1,0E0,0E0,2.0146374E-2,1.326319E-1,0E0,4.3555457E-2,1.6426831E-1,1.09528095E-1,6.0819864E-2,0E0,7.7050425E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,22,22,23,23,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,62,64,-1,66,68,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0105532E3,7.6599895E9,1.2055723E8,4.684559E2,1.5461028E12,8.601656E6,1.3611247E5,1.822764E4,2.2971932E7,3.6509445E1,2.5E1,2.4716E4,5.9283892E7,-1.3781548E-3,1.3382495E8,1E0,2.0806985E-1,7.7567725E2,7.4297624E0,-2.3914319E-2,-4.741177E-2,9.839836E-3,1.776E3,5.1E1,-6.1298814E-3,3.9063934E-2,1.046E4,-2.482055E-2,-3.466254E-3,1.01978E5,1.685E3,4.914413E-3,5.1E2,6.2742615E0,2.8314E4,8.539363E7,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,7.774228E-2,7.359468E8,2.9516693E-3,-6.394594E-3,3.76E2,5.2560944E1,7.7366033E-3,3.892E3,2.7883E4,4.4589956E5,7.3586698E0,-1.2339829E-3,1E0,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"split_indices":[53,5,46,53,32,1,34,34,46,57,3,2,46,0,33,105,39,53,54,0,0,0,0,3,0,0,9,0,0,30,2,39,0,54,9,46,0,0,0,0,0,0,0,58,5,0,0,0,59,0,11,9,29,54,0,84,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.76E2,3.9E1,1.56E2,2E1,3.1E1,8E0,7.9E1,7.7E1,1.3E1,7E0,1E1,2.1E1,3E0,5E0,3.2E1,4.7E1,5E1,2.7E1,6E0,7E0,3E0,4E0,7E0,3E0,1.5E1,6E0,3E0,2E0,2.7E1,5E0,3.5E1,1.2E1,2.6E1,2.4E1,2.5E1,2E0,2E0,2E0,5E0,2E0,3E0,3E0,2.1E1,6E0,2E0,3E0,4E0,3.1E1,4E0,8E0,1.6E1,1E1,2.1E1,3E0,2.1E1,4E0,1.9E1,2E0,2E0,4E0,2E0,2E0,2.3E1,8E0,3E0,5E0,1.4E1,2E0,4E0,6E0,1.8E1,3E0,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[7.874234E-3,-1.0717477E-1,4.6521157E-1,-1.982936E-1,9.943495E-2,1.02683544E-1,7.759888E-1,-1.3372196E-1,-6.0561585E-1,2.801086E-1,-2.3861834E-1,-1.4555064E-1,1.8935364E-1,4.1564103E-2,4.7552985E-1,-1.7659279E-2,-9.404589E-2,-7.024602E-3,-6.612021E-1,1.8482257E-1,3.9656345E-2,-7.397643E-2,-5.8301306E-1,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.4021983E-1,9.948502E-3,2.6430182E-2,-1.4144325E-1,2.806465E-2,-2.1147303E-2,-3.7808225E-2,2.9104125E-1,-3.4760606E-2,5.0945976E-3,-1.12476826E-1,-1.1272882E-2,-3.41051E-2,9.7476035E-2,1.4251037E-2,-8.572106E-2,-2.9373407E-1,9.719249E-2,-1.7345269E-1,5.5071485E-4,3.198904E-1,-1.1351033E-2,3.5261977E-2,-1.3395034E-1,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,35,37,-1,-1,-1,39,-1,-1,41,43,-1,-1,45,47,-1,49,-1,-1,51,-1,53,55,57,59,-1,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1257111E1,3.2389693E0,4.7649326E0,3.0488906E0,3.303937E0,4.7936463E-1,1.3339043E-1,9.1138387E-1,3.148079E-1,1.6327975E0,1.0342052E0,4.6653435E-2,1.7866707E-1,0E0,1.7190337E-2,0E0,5.248641E-1,0E0,2.9516697E-2,7.331557E-1,0E0,1.08073495E-1,8.598232E-2,0E0,0E0,0E0,6.9084704E-2,0E0,0E0,5.2640927E-1,3.790502E-1,0E0,0E0,1.658467E-1,1.6110414E-1,0E0,1.4867976E-2,0E0,0E0,1.5488528E-2,0E0,2.9356018E-1,1.905942E-2,1.5623116E-1,6.481768E-2,0E0,1.2719929E-1,0E0,4.8637956E-2,9.308532E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,18,18,19,19,21,21,22,22,26,26,29,29,30,30,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,46,46,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,36,38,-1,-1,-1,40,-1,-1,42,44,-1,-1,46,48,-1,50,-1,-1,52,-1,54,56,58,60,-1,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7987691E0,1.0648859E3,1.5262272E3,5.0446276E7,7.294459E7,1.0437421E8,2.98476E0,1.295E3,6.856226E-1,6.2150537E3,2.09E3,7.44E2,1.5675428E5,4.1564103E-2,3.145611E6,-1.7659279E-2,5.3287085E2,-7.024602E-3,9.180692E7,2.249985E5,3.9656345E-2,1.1341179E3,1.4233672E8,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.398624E8,9.948502E-3,2.6430182E-2,6.7841024E10,3.0143394E7,-2.1147303E-2,-3.7808225E-2,7.91144E5,2.3276744E5,5.0945976E-3,3.7387386E-1,-1.1272882E-2,-3.41051E-2,1.144E3,1.4251037E-2,6.0130353E0,7.273E3,9.4390506E5,2.9673992E9,5.5071485E-4,1E0,-1.1351033E-2,2.6387034E5,9.9764734E4,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"split_indices":[43,53,4,46,46,7,58,2,28,4,0,0,29,0,1,0,53,0,7,34,0,34,46,0,0,0,7,0,0,32,46,0,0,1,34,0,58,0,0,0,0,54,9,33,5,0,102,0,29,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.7E2,4.2E1,1.18E2,5.2E1,2E1,2.2E1,1.03E2,1.5E1,3.4E1,1.8E1,5E0,1.5E1,1.6E1,6E0,1.4E1,8.9E1,2E0,1.3E1,3E1,4E0,1.3E1,5E0,3E0,2E0,3E0,1.2E1,2E0,4E0,6.4E1,2.5E1,6E0,7E0,2E1,1E1,2E0,1.1E1,2E0,3E0,4E0,8E0,4.8E1,1.6E1,1.9E1,6E0,2E0,1.8E1,2E0,8E0,8E0,3E0,2E0,2E0,1.6E1,3.2E1,2E0,1.4E1,4E0,1.5E1,3E0,3E0,3E0,1.5E1,3E0,5E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.3249438E-2,-1.2597813E-1,3.6839122E-1,-3.741495E-1,-6.773875E-2,5.7275885E-1,-2.5949249E-2,-4.510603E-1,-5.7005443E-2,-1.6321269E-1,4.9919948E-2,2.9108667E-1,7.353367E-1,1.081298E-1,-2.378517E-2,-4.51294E-3,-4.978342E-1,1.5801355E-3,-4.8320345E-3,-1.921461E-2,-1.2809235E-1,1.7340758E-1,-1.6564597E-1,3.5022026E-1,-1.1079086E-3,4.0072575E-2,4.0726957E-1,-5.00003E-2,3.2730332E-1,-8.940148E-3,-5.212322E-1,-7.0953816E-3,-1.7170624E-1,2.1817629E-1,-2.174709E-2,-2.512538E-1,1.0451725E-1,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,4.3051414E-2,1.952225E-2,5.8277114E-3,-1.2334295E-2,-5.556018E-1,-1.2370801E-1,4.7271628E-2,-9.585648E-2,-2.4668616E-1,3.1131923E-2,2.7143374E-1,6.132134E-4,-4.048703E-3,-1.6482335E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,65,67,69,71,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.865889E0,2.272738E0,3.8026128E0,7.0604134E-1,1.4684818E0,1.2055569E0,1.1077032E0,3.5608482E-1,2.9706666E-2,5.3508985E-1,1.5992881E0,2.492423E-1,2.136507E-1,5.015763E-1,0E0,0E0,5.8598995E-2,0E0,0E0,0E0,3.3651924E-1,3.3674383E-1,5.350784E-1,1.20595574E-1,0E0,0E0,1.174857E-1,1.1404645E-1,4.39198E-2,0E0,3.405571E-2,1.1996572E-1,2.4365234E-1,3.003564E-1,1.7932303E-2,2.712661E-1,2.0998505E-1,0E0,0E0,0E0,0E0,0E0,1.23745585E-2,0E0,0E0,0E0,3.083849E-2,2.4845354E-2,9.429864E-2,2.3359656E-1,2.8410673E-2,1.0068264E-1,1.6567254E-1,0E0,0E0,6.312257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,42,42,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,66,68,70,72,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.4862166E1,6.627576E7,1.7497415E-3,5.3287085E2,3.5073215E3,5.7006E4,3.7453184E-3,9.096081E7,9.4418E4,3.1821228E7,5.2E1,1.0096877E8,1.0252E4,-2.378517E-2,-4.51294E-3,6.519E3,1.5801355E-3,-4.8320345E-3,-1.921461E-2,4.06E2,2.8314E4,4.4E1,1.075839E9,-1.1079086E-3,4.0072575E-2,1.5605432E9,3.9880952E-1,1E0,-8.940148E-3,2.0521739E0,1.369E3,9.258696E0,5.716463E2,1.3717402E0,7.6599895E9,1.2633E4,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,2E0,1.952225E-2,5.8277114E-3,-1.2334295E-2,1.0595825E3,1.36135E5,9.087618E6,1.1947E4,2.4734788E7,8.5093097E2,5.0691235E6,6.132134E-4,-4.048703E-3,1.8159722E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"split_indices":[53,59,46,40,53,53,10,59,7,1,46,3,52,9,0,0,9,0,0,0,0,9,3,7,0,0,7,58,8,0,57,2,55,53,43,5,2,0,0,0,0,0,8,0,0,0,34,1,33,9,12,4,46,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.58E2,4.6E1,2.9E1,1.29E2,3E1,1.6E1,2.3E1,6E0,7.1E1,5.8E1,1.2E1,1.8E1,1.3E1,3E0,3E0,2E1,2E0,4E0,8E0,6.3E1,3.7E1,2.1E1,1E1,2E0,1.3E1,5E0,8E0,5E0,2E0,1.8E1,1.7E1,4.6E1,3E1,7E0,1.6E1,5E0,8E0,2E0,2E0,3E0,3E0,5E0,3E0,2E0,3E0,1.5E1,5E0,1.2E1,2.4E1,2.2E1,7E0,2.3E1,5E0,2E0,1.3E1,3E0,3E0,2E0,3E0,2E0,9E0,6E0,3E0,2E0,9E0,3E0,3E0,2.1E1,7E0,1.5E1,5E0,2E0,7E0,1.6E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[6.0231384E-2,-6.6708066E-2,4.1548002E-1,-1.802126E-1,9.4408296E-2,6.121976E-1,4.159176E-2,-1.3381815E-1,-4.551602E-1,-2.3027116E-1,1.6258606E-1,2.7157372E-1,6.904492E-1,1.487131E-1,-1.9334765E-2,-3.371152E-1,-7.73753E-2,-1.0917176E-2,-2.8615614E-2,-1.011948E-1,-2.631605E-2,7.940243E-2,3.767569E-1,1.6384088E-2,2.4758964E-3,8.306936E-1,4.7551715E-1,3.2327557E-1,1.1648051E-2,-9.826521E-3,-2.175756E-2,-1.1204586E-1,1.6198908E-1,-1.3703959E-1,7.871614E-4,1.0334385E-1,-7.2968313E-3,4.7740215E-1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,-8.063122E-2,5.84258E-3,-3.1697118E-1,-6.857089E-2,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.4921291E-1,-1.0298919E-2,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,-1,35,37,-1,-1,39,41,43,45,-1,-1,47,49,51,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,57,-1,59,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.911444E0,2.6869793E0,3.7991571E0,1.0361109E0,1.3877299E0,7.1874046E-1,9.523439E-1,8.3451545E-1,2.3604941E-1,3.861003E-1,8.7867534E-1,9.7789764E-2,4.188528E-1,3.7905708E-1,0E0,1.4759302E-1,5.16084E-1,0E0,0E0,4.008031E-2,0E0,2.2414063E-1,3.420025E-1,0E0,0E0,4.281807E-2,4.2568207E-2,1.4053464E-1,1.0971589E-1,0E0,0E0,4.504485E-1,6.2172145E-2,1.6357675E-2,0E0,1.8398082E-1,0E0,1.7374277E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1939041E-2,0E0,2.7805746E-2,2.6995105E-1,0E0,0E0,0E0,0E0,2.1720147E-1,1.2581149E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,37,37,45,45,47,47,48,48,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,-1,36,38,-1,-1,40,42,44,46,-1,-1,48,50,52,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,58,-1,60,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0601392E3,2.1836595E8,6.627576E7,8.93182E5,1E0,1.2737E4,5.7006E4,1.042E3,4E0,7.6499896E9,1.068371E3,3.5E1,1.5213319E0,1E0,-1.9334765E-2,8.601996E-1,1E0,-1.0917176E-2,-2.8615614E-2,2.3276744E5,-2.631605E-2,4.759155E1,7.9762895E6,1.6384088E-2,2.4758964E-3,1.929E3,9.299267E9,3.6392304E-1,6.331E3,-9.826521E-3,-2.175756E-2,1.5497297E1,6.98247E5,1.1534339E9,7.871614E-4,7.183439E0,-7.2968313E-3,8.383825E1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,1.181E3,5.84258E-3,3.5E1,5.974511E0,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.8613E4,2.770862E8,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"split_indices":[53,7,46,30,102,2,10,2,8,5,53,3,58,8,0,28,89,0,0,34,0,57,46,0,0,0,12,28,9,0,0,59,29,7,0,54,0,59,0,0,0,0,0,0,0,0,0,10,54,0,0,0,0,9,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.45E2,5.1E1,8.5E1,6E1,3.3E1,1.8E1,7.4E1,1.1E1,1E1,5E1,7E0,2.6E1,1.5E1,3E0,1.5E1,5.9E1,5E0,6E0,8E0,2E0,3.7E1,1.3E1,5E0,2E0,1.4E1,1.2E1,6E0,9E0,8E0,7E0,5.2E1,7E0,6E0,2E0,3.4E1,3E0,9E0,4E0,8E0,6E0,1E1,2E0,2E0,4E0,5E0,4E0,8E0,4.4E1,3E0,4E0,4E0,2E0,2.4E1,1E1,5E0,4E0,2E0,3E0,2E0,6E0,1.7E1,2.7E1,1E1,1.4E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[6.2760315E-3,-8.942039E-2,3.3899674E-1,-3.464915E-1,-2.9866556E-2,-3.944495E-2,5.6220204E-1,-7.096159E-2,-4.272631E-1,-3.1877894E-2,-1.0096922E-2,-2.7154645E-1,1.3294205E-1,3.9848673E-1,3.567469E-2,-8.072219E-3,6.935243E-3,-2.918099E-1,-2.5652623E-2,-1.33155E-1,6.0559504E-2,-4.3664877E-3,-2.650756E-2,1.6947778E-2,8.673523E-3,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,5.782375E-2,-1.6802189E-1,2.2789724E-1,-4.5959535E-3,3.984736E-3,-5.162517E-2,6.689567E-3,-5.874931E-3,-2.1503107E-1,-5.4912195E-3,7.9473734E-2,3.573293E-1,-3.5994206E-2,2.3827569E-1,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,27,-1,29,31,-1,-1,-1,33,-1,-1,-1,-1,35,37,39,41,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5575504E0,2.4391828E0,3.907806E0,6.373303E-1,1.6124437E0,7.563031E-1,5.1900864E-1,1.857129E-1,1.5725279E-1,0E0,1.1210039E0,3.4842473E-1,2.9128677E-1,7.89125E-2,0E0,0E0,0E0,4.2960286E-2,0E0,3.2268065E-1,8.950907E-1,0E0,0E0,0E0,3.954031E-2,0E0,0E0,0E0,0E0,1.3033742E-1,3.0443656E-1,4.1344917E-1,4.6612126E-1,0E0,1.9462278E-2,0E0,0E0,1.0997474E-1,4.840184E-2,1.0358285E-1,2.309668E-2,5.0568026E-1,1.8324465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,17,17,19,19,20,20,24,24,29,29,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,28,-1,30,32,-1,-1,-1,34,-1,-1,-1,-1,36,38,40,42,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7108864E8,1.4133928E1,1E0,2.862988E5,3.673257E0,3.067586E5,1.5435694E9,5.34E2,3.203027E7,-3.1877894E-2,3.3969492E2,2.565E3,1E0,1.4008022E11,3.567469E-2,-8.072219E-3,6.935243E-3,1.3448806E8,-2.5652623E-2,2.0589474E1,1.10597E5,-4.3664877E-3,-2.650756E-2,1.6947778E-2,4.078932E5,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,7.269755E0,1.7869681E6,6.185678E2,3.1982856E3,3.984736E-3,1.743E3,6.689567E-3,-5.874931E-3,1.5373E4,7.343501E-1,4.0759E4,4.3411046E8,9.518453E6,1.4139E4,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"split_indices":[7,59,102,29,54,29,7,0,46,0,53,0,8,32,0,0,0,7,0,59,30,0,0,0,29,0,0,0,0,54,51,53,4,0,0,0,0,9,28,30,7,46,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.59E2,4.5E1,2.9E1,1.3E2,1.7E1,2.8E1,7E0,2.2E1,3E0,1.27E2,7E0,1E1,1.6E1,1.2E1,5E0,2E0,1.1E1,1.1E1,4.6E1,8.1E1,5E0,2E0,3E0,7E0,4E0,1.2E1,9E0,2E0,7E0,3.9E1,2.2E1,5.9E1,3E0,4E0,5E0,2E0,3E1,9E0,1.1E1,1.1E1,5.3E1,6E0,2E0,2E0,1.3E1,1.7E1,7E0,2E0,5E0,6E0,9E0,2E0,1.8E1,3.5E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.2222775E-2,-9.139186E-2,4.9998534E-1,-5.1995587E-2,-4.944162E-1,6.0979265E-1,-1.5150189E-2,-1.3674228E-1,1.0816623E-1,3.8813213E-3,-6.330518E-1,8.045889E-1,3.992678E-1,-3.3645314E-1,-1.01294234E-1,2.436709E-1,-8.044677E-2,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,-3.9906213E-1,-5.7864846E-3,-5.545319E-2,-2.0736766E-1,-8.008159E-2,3.159065E-1,-1.8992147E-1,-6.997158E-3,-2.360766E-2,-1.1299601E-2,-5.494948E-3,-2.1435164E-1,-2.4299191E-1,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.7862383E-1,4.077045E-1,-5.167938E-3,-1.228408E-2,-5.0322257E-2,4.2618223E-2,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,37,39,41,43,-1,-1,45,47,49,-1,-1,-1,51,53,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.480311E0,2.8481483E0,2.6842346E0,2.2679396E0,1.3240938E0,7.0399E-1,0E0,7.3756933E-1,1.5019631E0,0E0,4.9333286E-1,1.782751E-2,2.6048756E-1,1.7048275E-1,4.4557756E-1,8.2043767E-1,1.9969293E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.9644938E-2,0E0,5.306874E-1,1.2524438E-1,5.6175735E-2,2.8003812E-1,2.552101E-2,3.65381E-2,0E0,0E0,2.7235347E-1,1.0511035E-1,6.087303E-2,0E0,0E0,0E0,1.4527509E-1,1.0674596E-1,0E0,0E0,2.577142E-2,9.493223E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,23,23,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,39,39,40,40,43,43,44,44],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,38,40,42,44,-1,-1,46,48,50,-1,-1,-1,52,54,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,1.650269E10,1.2055723E8,6.978001E2,1.8159722E-1,8.812601E-1,-1.5150189E-2,1.142E3,3.291238E7,3.8813213E-3,5.535E3,1.929E3,1.7595541E0,1.3448806E8,1.06403265E1,1E0,2.1845242E3,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,1.22E2,-5.7864846E-3,1.9540394E7,9.744047E0,9.31E2,1.068371E3,5.5053414E9,7.951E3,-2.360766E-2,-1.1299601E-2,1.1888112E-1,4.22631E5,4.187504E7,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.617623E5,2.9E1,-5.167938E-3,-1.228408E-2,2.5821698E0,7.157903E7,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"split_indices":[53,5,46,53,28,58,0,2,46,0,9,0,58,7,55,102,4,0,0,0,0,0,0,10,0,46,54,0,53,5,9,0,0,58,29,33,0,0,0,34,8,0,0,54,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.8E2,2.7E1,1.65E2,1.5E1,2.4E1,3E0,1.08E2,5.7E1,3E0,1.2E1,1.1E1,1.3E1,1.5E1,9.3E1,3.3E1,2.4E1,5E0,7E0,8E0,3E0,3E0,1E1,1.1E1,4E0,6.6E1,2.7E1,6E0,2.7E1,9E0,1.5E1,6E0,5E0,5.1E1,1.5E1,2.1E1,6E0,3E0,3E0,1.2E1,1.5E1,5E0,4E0,8E0,7E0,1.9E1,3.2E1,6E0,9E0,1.8E1,3E0,1E1,2E0,1.2E1,3E0,2E0,6E0,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.469039E-3,-6.84523E-2,5.0491726E-1,-1.8420036E-1,8.1181556E-2,6.261244E-1,3.6942E-2,-2.2080618E-1,7.735341E-2,-3.1455643E-2,2.747695E-1,3.8490146E-2,4.343899E-1,5.4492303E-3,-3.6699688E-3,-1.701605E-1,-4.6023417E-1,-3.2597054E-2,1.7538356E-2,-2.7009997E-1,4.3159403E-2,1.4272796E-1,3.6220354E-1,7.502832E-3,2.3337092E-2,-2.6047137E-1,-8.6393684E-2,-3.3155903E-2,-2.9781944E-1,3.291199E-4,-5.0207926E-3,-1.3054396E-1,-2.0009596E-2,-3.049531E-3,2.1293099E-1,1.6921525E-1,-5.2545854E-4,2.1539481E-2,2.3263586E-1,-3.6815163E-1,-1.5577061E-1,-1.2372288E-2,-4.669773E-2,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,-5.099285E-2,7.831989E-2,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,-1,31,33,35,37,-1,-1,39,41,-1,43,-1,-1,45,-1,47,49,51,-1,-1,53,55,57,-1,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009293E0,3.3230407E0,1.3381958E0,1.0511143E0,1.8402805E0,2.5670147E-1,5.9022054E-2,1.0736556E0,4.4741017E-1,9.7320604E-1,3.0619597E-1,0E0,6.5217495E-2,0E0,0E0,5.8270144E-1,3.7153912E-1,3.1251714E-2,0E0,2.0223719E-1,3.3013782E-1,5.8757484E-2,8.452511E-2,0E0,0E0,3.710723E-1,2.7460602E-1,0E0,4.2746305E-2,0E0,0E0,6.497468E-2,0E0,1.3663186E-1,6.1325133E-2,2.643928E-2,0E0,0E0,1.6941309E-2,5.598998E-2,1.3381499E-1,0E0,1.9927172E-1,0E0,0E0,0E0,0E0,1.089835E-1,1.0501201E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28,31,31,33,33,34,34,35,35,38,38,39,39,40,40,42,42,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,-1,32,34,36,38,-1,-1,40,42,-1,44,-1,-1,46,-1,48,50,52,-1,-1,54,56,58,-1,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,2.2896422E6,9.298568E7,2.096851E3,1.537424E0,8.812601E-1,1.2055723E8,5.0446276E7,1.0625348E8,5.6E0,8.57703E2,3.8490146E-2,4.11964E5,5.4492303E-3,-3.6699688E-3,4.9658173E2,5.511724E0,5.9527163E3,1.7538356E-2,4.3927447E11,5.4732365E9,1.1216892E3,6.518218E6,7.502832E-3,2.3337092E-2,1.822764E4,1E0,-3.3155903E-2,2.0529972E1,3.291199E-4,-5.0207926E-3,6.4336755E8,-2.0009596E-2,7.9472524E-1,3.9952212E3,8.3154816E2,-5.2545854E-4,2.1539481E-2,7.8114974E-1,4.3117376E8,2.524E3,-1.2372288E-2,1.3474202E8,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,1.03235014E-1,9.445656E6,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"split_indices":[53,51,46,4,43,58,46,46,7,55,53,0,30,0,0,4,54,34,0,32,5,4,46,0,0,34,65,0,57,0,0,7,0,40,4,4,0,0,28,5,2,0,7,0,0,0,0,58,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.9E2,2.3E1,1.07E2,8.3E1,1.8E1,5E0,9.4E1,1.3E1,5.3E1,3E1,8E0,1E1,3E0,2E0,7.9E1,1.5E1,1E1,3E0,1.2E1,4.1E1,1.3E1,1.7E1,2E0,8E0,3.7E1,4.2E1,5E0,1E1,7E0,3E0,7E0,5E0,3.3E1,8E0,1.1E1,2E0,9E0,8E0,1.7E1,2E1,7E0,3.5E1,7E0,3E0,4E0,3E0,2.1E1,1.2E1,6E0,2E0,6E0,5E0,6E0,2E0,1.2E1,5E0,1.7E1,3E0,1.2E1,2.3E1,4E0,1.7E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-2.0219442E-2,-1.16520904E-1,2.8449756E-1,-3.0797613E-1,-5.9537902E-2,9.543412E-2,5.2078587E-1,-3.3963862E-1,7.4714734E-3,-2.8888127E-2,-3.689822E-1,2.1040678E-1,-8.497771E-2,3.1196967E-2,3.1238148E-1,-2.7664992E-1,-2.8590444E-2,-1.8818352E-1,6.0974874E-2,-4.8011044E-1,-2.6513548E-3,1.2447058E-2,4.99297E-3,2.9411163E-2,-1.22154895E-2,5.4155197E-3,1.8623266E-2,-3.116507E-1,-1.2212595E-1,-3.6836183E-1,-1.402538E-1,3.1627953E-2,3.3315614E-1,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,-3.2800922E-1,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,-1.5982786E-1,-2.9887237E-2,1.3616896E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.107501E0,1.705395E0,2.1839733E0,5.781839E-1,1.1559219E0,6.194103E-1,3.51089E-1,3.682499E-1,0E0,1.6290425E0,3.58294E-1,6.1777413E-2,2.3528156E-1,0E0,9.8772764E-2,1.18784666E-1,0E0,3.0240762E-1,5.7256365E-1,1.4015281E-1,0E0,0E0,0E0,2.9680826E-2,0E0,0E0,0E0,5.1579952E-2,1.1075765E-2,5.4989576E-2,1.3172704E-1,4.3493733E-1,5.3087175E-2,0E0,0E0,0E0,0E0,0E0,3.895402E-2,0E0,0E0,0E0,0E0,0E0,8.195257E-2,2.843659E-1,3.718426E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,23,23,27,27,28,28,29,29,30,30,31,31,32,32,38,38,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.5497297E1,1.3453049E3,4.5342876E3,9.996903E7,2.68E4,8.812601E-1,3.013037E6,7.4714734E-3,3.2384683E2,1.4539318E-1,1.5331E4,6.079755E0,3.1196967E-2,1.5545E4,1.3448806E8,-2.8590444E-2,8.2955E4,9.467578E6,2.1897722E10,-2.6513548E-3,1.2447058E-2,4.99297E-3,5.5566853E-1,-1.22154895E-2,5.4155197E-3,1.8623266E-2,3.7453184E-3,9.279E3,2.700379E2,2.0589474E1,6.5963456E5,6.2357716E-2,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,5.46562E5,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,1.2705392E1,1.617E3,7.6760286E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"split_indices":[43,59,53,4,46,9,58,29,0,53,39,9,54,0,9,7,0,1,1,5,0,0,0,28,0,0,0,59,9,4,59,29,39,0,0,0,0,0,12,0,0,0,0,0,57,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.57E2,4.9E1,3.5E1,1.22E2,2.8E1,2.1E1,3.3E1,2E0,1.12E2,1E1,1.7E1,1.1E1,1.2E1,9E0,2.8E1,5E0,4E1,7.2E1,7E0,3E0,1.1E1,6E0,7E0,4E0,3E0,6E0,2.2E1,6E0,7E0,3.3E1,6.6E1,6E0,3E0,4E0,4E0,3E0,2E0,2E1,4E0,2E0,2E0,5E0,3E0,3E1,4.2E1,2.4E1,2E0,4E0,1.7E1,3E0,2.1E1,9E0,3.5E1,7E0,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.1936052E-2,-1.5182267E-1,1.5760238E-1,-1.214642E-1,-3.187172E-2,1.07080944E-1,3.2834876E-2,-1.8811886E-1,6.19821E-2,-8.8205084E-2,1.7667614E-1,-3.115475E-1,-1.0234942E-1,1.5685846E-1,-1.4203048E-1,-1.3068446E-2,-2.3196185E-2,-3.587127E-2,2.4979593E-1,-2.4225932E-1,-2.5490196E-2,-2.0880654E-2,-1.4597873E-1,3.4969583E-2,3.1181702E-1,1.3681039E-2,-1.3338024E-2,-9.279402E-2,4.1761484E-2,2.3741236E-2,-1.5669461E-2,9.474784E-2,3.6113718E-1,5.085498E-3,-2.820379E-1,-8.3250865E-2,8.308102E-2,-1.855153E-1,-1.1149166E-2,-5.6521293E-2,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,-2.4819752E-2,-8.349065E-3,6.2872446E-3,1.8276206E-4,-3.816804E-2,4.864078E-3,1.2524661E-1,-6.326331E-3,3.7593523E-1,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,-1,-1,-1,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.107609E0,1.8600852E0,2.331173E0,1.4911933E0,0E0,1.1708164E0,0E0,9.103031E-1,6.6102564E-1,2.6339516E-1,9.880699E-1,4.0248227E-1,1.8889087E-1,4.235136E-1,2.2991076E-1,0E0,8.525165E-2,2.97587E-1,7.771559E-1,2.8676605E-1,0E0,1.3659878E-1,1.8450612E-1,2.3734151E-1,9.613538E-2,8.857786E-3,0E0,4.3926656E-2,3.2743193E-2,7.4582696E-2,0E0,1.6022247E-1,5.321765E-2,5.2886583E-2,9.587407E-2,5.3622194E-2,1.212705E-1,8.459413E-2,9.3104824E-2,2.9008571E-2,0E0,0E0,0E0,0E0,0E0,2.897961E-2,0E0,0E0,0E0,5.3322386E-2,0E0,4.675585E-2,0E0,3.5073757E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,45,45,49,49,51,51,53,53],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,-1,-1,-1,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.0728035E-1,1.3917024E8,1.1038146E0,1.0648859E3,-3.187172E-2,4.2447827E2,3.2834876E-2,6.794883E4,2.2882867E5,2.5546576E4,1E0,2.47565E6,8.623692E0,3.851071E7,4.1808826E-1,-1.3068446E-2,2.3739047E-1,1.019731E0,1.0861458E3,2.9377524E2,-2.5490196E-2,3.8E1,2.4079119E1,2.0601392E3,3.353193E6,4E1,-1.3338024E-2,6.741443E-1,4.35E2,8.836347E7,-1.5669461E-2,4.6E1,4.8420677E-1,1.4416069E9,4.2E1,1.4178E4,7.47E2,3.8718798E6,8.2092184E7,4.83E9,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,9.576E3,-8.349065E-3,6.2872446E-3,1.8276206E-4,9.31E2,4.864078E-3,6.98247E5,-6.326331E-3,1E0,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"split_indices":[40,46,35,53,0,53,0,51,34,34,102,29,55,12,28,0,39,35,53,34,0,3,57,53,1,3,0,28,0,49,0,3,39,5,3,9,0,46,7,5,0,0,0,0,0,9,0,0,0,0,0,29,0,84,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.26E2,9.1E1,1.2E2,6E0,8.4E1,7E0,8.8E1,3.2E1,2.2E1,6.2E1,3.5E1,5.3E1,2.2E1,1E1,5E0,1.7E1,1.6E1,4.6E1,2.8E1,7E0,1.9E1,3.4E1,1.3E1,9E0,5E0,5E0,8E0,9E0,1.4E1,2E0,2E1,2.6E1,4E0,2.4E1,1.2E1,7E0,2.6E1,8E0,9E0,4E0,4E0,5E0,3E0,2E0,5E0,3E0,2E0,7E0,8E0,6E0,1.8E1,2E0,2.4E1,2E0,2E0,2E0,1.7E1,7E0,7E0,5E0,5E0,2E0,4E0,2.2E1,4E0,4E0,3E0,6E0,3E0,2E0,2E0,6E0,1.2E1,6E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-1.6992955E-2,-8.875833E-2,2.2156855E-1,-6.0390815E-2,-5.453842E-1,6.565264E-2,3.5614845E-1,-8.633612E-2,3.0628803E-1,-3.5136674E-2,-2.5970185E-1,1.033548E-2,1.3761056E-3,4.3397513E-3,3.885008E-1,-1.4680506E-1,-1.1059224E-2,2.477328E-2,1.7252595E-3,-9.932688E-4,-2.007134E-2,-5.6842774E-2,1.1758452E-1,2.6560768E-1,4.3975347E-1,-1.07489556E-1,-2.2993198E-2,-1.1149532E-1,1.1053811E-1,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,-2.6751718E-1,-5.8981713E-2,-4.527185E-2,-3.2934055E-1,1.45027E-2,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,29,-1,-1,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4779978E0,1.9877847E0,9.62826E-1,1.4344559E0,2.6657557E-1,2.1661456E-1,1.8128777E-1,6.3109255E-1,6.166359E-1,0E0,1.7431176E-1,0E0,1.2169773E-1,0E0,2.1767378E-2,9.350836E-1,7.817284E-1,0E0,4.614174E-2,0E0,0E0,3.312366E-2,1.0790542E-2,2.8682768E-2,3.3157587E-2,5.27603E-1,0E0,4.9001646E-1,6.9180393E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0392995E-2,2.4817911E-1,1.5262431E-1,9.5196724E-2,2.9514036E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,39,39,40,40,41,41,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,30,-1,-1,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5064244E0,1.4233672E8,8.57703E2,4.9192E4,2.849513E3,1.1244E4,1E0,6.978001E2,7.424212E-1,-3.5136674E-2,2.09E3,1.033548E-2,1.4872659E1,4.3397513E-3,2.8626094E5,4.462437E7,1E0,2.477328E-2,1.3174513E4,-9.932688E-4,-2.007134E-2,5.9857685E6,3.261056E8,1.2519051E7,3.007E3,1.142E3,-2.2993198E-2,3.4521E4,3.212485E11,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,4.543028E2,1.06403265E1,2.7577372E11,5.4656273E-1,2.4394053E9,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"split_indices":[43,46,53,2,4,9,102,53,43,0,0,0,57,0,29,46,102,0,4,0,0,33,7,33,0,2,0,10,32,0,0,0,0,0,0,0,0,0,0,4,55,32,28,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.55E2,4.6E1,1.47E2,8E0,2.2E1,2.4E1,1.38E2,9E0,4E0,4E0,6E0,1.6E1,3E0,2.1E1,7.6E1,6.2E1,5E0,4E0,2E0,2E0,1.1E1,5E0,8E0,1.3E1,6.9E1,7E0,3.4E1,2.8E1,2E0,2E0,6E0,5E0,2E0,3E0,6E0,2E0,1.1E1,2E0,1.5E1,5.4E1,2.7E1,7E0,2.1E1,7E0,1.3E1,2E0,3.6E1,1.8E1,7E0,2E1,2E0,5E0,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.02255E-2,-6.558096E-2,3.4569636E-1,-1.2843427E-2,-4.0469694E-1,5.4956263E-1,4.3343384E-2,-1.403701E-1,1.0433215E-1,2.2149876E-2,-4.8705584E-1,6.838387E-1,3.7301254E-1,1.4347048E-1,-1.6691085E-2,-2.8438172E-1,-7.497648E-2,3.6531553E-2,2.410501E-1,-3.8605474E-3,5.6571406E-3,-5.657334E-1,-2.4888176E-1,9.962667E-3,7.5234216E-1,6.084204E-3,1.9794445E-2,7.620034E-2,1.50580425E-2,-3.147017E-1,-2.6331637E-3,-3.2688044E-2,-1.6758075E-1,3.0311698E-3,1.3236392E-1,-3.7357313E-4,2.712043E-1,-6.3281095E-1,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,2.171496E-3,8.731073E-3,-1.6129533E-2,-5.054191E-3,2.5078287E-2,-1.376335E-1,-2.738632E-3,-1.0184393E-2,6.306253E-2,-5.9866663E-2,2.3515062E-1,-2.9690387E-3,1.6760947E-2,1.7714213E-1,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,-1,41,-1,-1,43,-1,45,-1,47,49,51,53,-1,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,-1,-1,65,67,69,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7954755E0,3.3212526E0,2.565043E0,2.4354134E0,8.896537E-1,3.3758068E-1,7.454521E-1,7.132385E-1,7.791066E-1,5.6962486E-2,2.5271702E-1,2.7266836E-1,5.7875514E-2,1.438675E-1,0E0,1.5053928E-1,2.0990708E-1,1.8580317E-1,2.1208572E-1,0E0,0E0,2.0361853E-1,6.189382E-2,0E0,4.8636913E-2,0E0,0E0,9.129316E-2,0E0,3.965044E-2,0E0,2.3988187E-1,6.710768E-2,1.699124E-1,3.1248683E-1,0E0,1.11125946E-1,2.6826096E-1,0E0,0E0,0E0,0E0,0E0,3.272756E-2,0E0,0E0,0E0,2.1187598E-1,8.620715E-2,0E0,0E0,1.1212682E-1,1.685599E-1,7.086277E-2,0E0,0E0,4.61289E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,24,24,27,27,29,29,31,31,32,32,33,33,34,34,36,36,37,37,43,43,47,47,48,48,51,51,52,52,53,53,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,-1,42,-1,-1,44,-1,46,-1,48,50,52,54,-1,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,-1,-1,66,68,70,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6499896E9,6.627576E7,3.7763092E2,3.3478114E-1,8E0,5.7006E4,1.278E3,1.5022863E0,1.776E3,9.006135E0,2.6E1,1.625E3,8.129243E5,-1.6691085E-2,4.5715973E4,5.531948E8,1.8331E4,7.555544E2,-3.8605474E-3,5.6571406E-3,1.2728E4,1.2187347E3,9.962667E-3,2.2047469E4,6.084204E-3,1.9794445E-2,1.0252E4,1.50580425E-2,3.4322312E-1,-2.6331637E-3,7.269755E0,1.2421726E7,4.278E3,2.4794E4,-3.7357313E-4,1.5489E4,8.260109E9,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,4E0,8.731073E-3,-1.6129533E-2,-5.054191E-3,5.1E2,2.037744E10,-2.738632E-3,-1.0184393E-2,4.541179E1,5.382E3,5.586E3,-2.9690387E-3,1.6760947E-2,7.4353185E0,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"split_indices":[53,5,46,53,28,8,10,2,42,0,55,3,0,29,0,34,5,10,4,0,0,9,4,0,4,0,0,9,0,39,0,54,46,2,9,0,9,5,0,0,0,0,0,8,0,0,0,0,32,0,0,57,2,2,0,0,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.85E2,4.1E1,1.61E2,2.4E1,2.4E1,1.7E1,7.7E1,8.4E1,4E0,2E1,1.2E1,1.2E1,1.4E1,3E0,2.3E1,5.4E1,5.7E1,2.7E1,2E0,2E0,1.4E1,6E0,2E0,1E1,2E0,1E1,1.1E1,3E0,2E1,3E0,3.8E1,1.6E1,4.3E1,1.4E1,3E0,2.4E1,1.1E1,3E0,4E0,2E0,8E0,2E0,7E0,4E0,1.8E1,2E0,2.5E1,1.3E1,5E0,1.1E1,2.2E1,2.1E1,9E0,5E0,1.2E1,1.2E1,2E0,9E0,5E0,2E0,1.5E1,1E1,1E1,3E0,1.8E1,4E0,9E0,1.2E1,5E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-5.735387E-3,-6.410505E-2,2.797867E-1,-1.760073E-2,-3.442871E-1,5.04675E-1,2.686096E-2,-1.2676698E-1,1.1911592E-1,-4.2207205E-1,5.1267102E-2,6.152395E-1,1.3419471E-2,1.5410598E-1,-1.9511065E-1,-2.5507975E-1,-8.848773E-2,2.6512688E-1,4.352096E-2,-3.2537884E-1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,2.4225692E-1,2.5578894E-2,-4.040385E-2,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,-4.242069E-2,-1.8737647E-1,3.087647E-1,4.6277814E-2,-5.1333107E-2,1.4060691E-1,-3.4747878E-3,-3.5448268E-1,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.1831463E-3,-1.2673235E-1,-2.0977704E-1,8.250273E-4,2.3295416E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,5.527623E-2,-1.189838E-1,1.975704E-1,-1.7819712E-2,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,45,47,49,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,65,-1,-1,-1,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5344255E0,2.287163E0,2.0341249E0,2.2836764E0,7.9084253E-1,3.1127357E-1,5.384738E-1,3.9492452E-1,7.390245E-1,2.7931428E-1,7.600727E-2,2.1988153E-1,0E0,1.2976074E-1,1.931273E-1,6.892848E-2,2.9727066E-1,2.0536327E-1,4.308317E-1,1.00349784E-1,0E0,0E0,0E0,0E0,0E0,8.028874E-2,1.2697376E-2,1.2392008E-2,0E0,0E0,0E0,1.8929411E-1,9.967518E-2,7.952249E-2,1.2078894E-2,1.7966965E-1,2.110849E-1,0E0,1.2337124E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1306114E-1,7.93834E-2,1.05109274E-1,0E0,1.7682552E-2,0E0,0E0,0E0,2.9296512E-2,1.0169087E-1,1.4039552E-1,1.7305832E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,27,27,31,31,32,32,33,33,34,34,35,35,36,36,38,38,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,46,48,50,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,66,-1,-1,-1,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6599895E9,6.627576E7,5.242538E2,1.5593E4,8.812601E-1,1.3917024E8,1.056E3,1.26881E5,1.3917024E8,3.729811E6,8E0,1.3419471E-2,5.1E1,2.565E3,7.107292E6,5.4127317E10,6.3300834E0,1.5181E4,1.9E1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,1.739E3,7.7573473E9,1.3174513E4,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,1.0466548E1,1E0,4.361E3,3.098E3,5.7277904E0,3.2395E4,-3.4747878E-3,3.20084E6,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.998024E-1,2.124077E1,7.2899837E0,8.250273E-4,9.607843E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,2.7060036E11,5.36E3,1.7790995E0,4.0953446E8,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"split_indices":[53,5,46,53,9,58,46,2,30,46,1,8,0,3,0,46,32,54,10,3,0,0,0,0,0,0,5,4,0,0,0,55,105,2,2,54,9,0,7,0,0,0,0,0,0,58,59,54,0,58,0,0,0,32,2,58,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.75E2,3.5E1,1.51E2,2.4E1,1.8E1,1.7E1,8.4E1,6.7E1,2E1,4E0,1.1E1,7E0,1.1E1,6E0,1.8E1,6.6E1,2.2E1,4.5E1,1.6E1,4E0,2E0,2E0,5E0,6E0,6E0,5E0,4E0,2E0,1.1E1,7E0,4.6E1,2E1,1.8E1,4E0,2.3E1,2.2E1,2E0,1.4E1,4E0,2E0,2E0,3E0,2E0,2E0,3E1,1.6E1,1.8E1,2E0,1.3E1,5E0,2E0,2E0,9E0,1.4E1,1.6E1,6E0,2E0,1.2E1,2.2E1,8E0,7E0,9E0,1.1E1,7E0,1.1E1,2E0,5E0,4E0,9E0,5E0,7E0,9E0,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-2.2588685E-2,-1.4545202E-1,1.1374607E-1,-8.6952515E-2,-3.9759496E-1,-3.064367E-2,2.3289914E-1,-1.2433577E-1,1.7574582E-1,-5.7842267E-1,-9.222219E-2,-6.9834334E-3,-1.8266106E-2,6.878721E-2,3.451957E-1,-2.7263525E-1,-8.135537E-2,-5.932932E-4,1.1667584E-2,-6.281898E-1,-9.885883E-3,-1.32481875E-2,1.0622433E-1,-2.1569806E-1,3.714026E-2,-4.582277E-2,1.5392454E-1,2.7393732E-2,4.457966E-1,-1.735199E-1,-1.8242102E-2,-2.038975E-1,-3.7180442E-2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,-3.8278066E-2,8.7990396E-2,-4.001707E-3,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,-1.1330966E-1,2.5172912E-2,3.078277E-1,-2.0457634E-1,-1.095509E-3,-1.3675371E-1,-1.4804438E-2,3.1190168E-2,-1.298265E-1,-8.181965E-2,8.646828E-3,-2.6838033E-4,1.4474735E-1,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,61,-1,63,65,-1,67,-1,69,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6011605E0,1.6322045E0,1.7619938E0,9.3515986E-1,1.1062734E0,3.806838E-1,1.0082908E0,5.004853E-1,1.4725864E-1,1.1519289E-1,3.504039E-1,4.224278E-1,0E0,2.4123535E-1,1.0428362E0,1.1729789E-1,3.442828E-1,0E0,0E0,1.5132141E-1,0E0,0E0,1.4618695E-2,8.2455575E-2,1.4909932E-1,5.0066087E-2,1.4621168E-1,4.4345292E-1,8.724737E-2,4.6982825E-2,0E0,7.2891474E-2,3.1442845E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6498463E-1,1.1525272E-1,1.6504118E-2,0E0,0E0,0E0,0E0,9.535315E-2,0E0,1.2730312E-1,2.6230484E-2,0E0,1.0405913E-2,0E0,2.094729E-1,1.0832605E-1,9.527324E-2,0E0,3.1808536E-2,1.478824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,39,39,40,40,41,41,46,46,48,48,49,49,51,51,53,53,54,54,55,55,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,62,-1,64,66,-1,68,-1,70,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3905947E-1,1.4446067E10,4.4589956E5,1E0,1.5461028E12,1.4146589E8,1.2838955E3,6.639351E2,4.22631E5,8.92851E6,1.244125E3,2.853839E4,-1.8266106E-2,1.3012535E1,1E0,8.3949E4,4.7095413E2,-5.932932E-4,1.1667584E-2,6.1826086E0,-9.885883E-3,-1.32481875E-2,1.5011E4,1.338E3,1.0176675E0,1.1350955E0,6.695E3,1E0,3.1E1,1E0,-1.8242102E-2,7.1399055E6,7.11E2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,4.8008E4,1.2364354E3,4.1376923E2,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,2.5591E4,2.5172912E-2,5.86883E6,1.4008755E2,-1.095509E-3,6.1E1,-1.4804438E-2,6.38E2,9.388849E-1,6.8992513E-1,8.646828E-3,2.7953E4,1.1946838E7,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"split_indices":[40,5,29,89,32,46,53,34,29,1,34,34,0,57,102,12,4,0,0,54,0,0,9,2,40,58,2,8,3,105,0,46,0,0,0,0,0,0,0,2,4,53,0,0,0,0,2,0,1,34,0,3,0,0,28,28,0,9,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.12E2,1.01E2,9.2E1,2E1,4.6E1,5.5E1,8.1E1,1.1E1,1.2E1,8E0,4.4E1,2E0,2.3E1,3.2E1,1.7E1,6.4E1,3E0,8E0,1E1,2E0,4E0,4E0,7E0,3.7E1,1E1,1.3E1,8E0,2.4E1,1E1,7E0,1.6E1,4.8E1,7E0,3E0,2E0,2E0,2E0,5E0,1.5E1,2.2E1,8E0,2E0,1.1E1,2E0,2E0,6E0,1.4E1,1E1,8E0,2E0,1.1E1,5E0,2.8E1,2E1,1.3E1,2E0,9E0,1.3E1,6E0,2E0,4E0,2E0,2E0,8E0,2E0,6E0,9E0,2E0,2.4E1,4E0,1.8E1,2E0,1E1,3E0,7E0,2E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.9344994E-3,-8.175323E-2,2.3140544E-1,-4.4790298E-2,-4.8490012E-1,2.5693094E-2,3.847721E-1,-9.1722965E-2,4.259129E-2,-2.0991474E-1,-3.360973E-2,2.7427834E-2,-5.6275822E-2,3.184738E-2,3.1256717E-1,-1.0761265E-1,1.2113636E-1,-2.8683331E-3,2.0815976E-1,-1.3282049E-2,-1.7001278E-3,7.0364326E-2,-1.4656629E-1,4.4856225E-3,1.5982723E-2,-2.2537349E-1,-8.0327906E-2,8.497628E-3,1.8191043E-3,-6.4350456E-2,7.170837E-2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,-3.6205932E-2,-4.63827E-3,-1.3423674E-2,5.86282E-3,-1.1076284E-1,1.3306548E-2,-1.0665464E-1,-2.4310345E-2,1.2974514E-1,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,-1,-1,33,35,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6567056E0,2.3029141E0,1.5360355E0,5.94458E-1,5.9041786E-1,1.0017112E0,3.4057522E-1,3.3096892E-1,3.8445777E-1,6.7638606E-2,0E0,0E0,2.3900099E-1,0E0,5.3549767E-2,2.6464677E-1,2.4824701E-2,1.9260496E-1,3.0953288E-2,0E0,0E0,8.459512E-2,1.8438596E-1,0E0,0E0,7.871467E-2,1.9214371E-1,0E0,0E0,7.697916E-2,1.0903385E-1,0E0,0E0,0E0,0E0,0E0,4.2805668E-2,0E0,0E0,1.03736766E-1,1.2159997E-1,6.966427E-2,5.4593235E-2,3.1910963E-2,5.4574564E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26,29,29,30,30,36,36,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,-1,-1,34,36,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.8000048E10,1E0,2.4277832E6,6.0671224E7,1.7727281E2,1.2117E4,1E0,8.373168E5,5.441E3,-3.360973E-2,2.7427834E-2,3.1047E4,3.184738E-2,5.8E2,6.554877E-4,3.5E1,1.0415988E0,1.521636E5,-1.3282049E-2,-1.7001278E-3,3.5073215E3,5.785788E-1,4.4856225E-3,1.5982723E-2,3E2,3.7504044E5,8.497628E-3,1.8191043E-3,2E0,8.3154816E2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,6.804E3,-4.63827E-3,-1.3423674E-2,1.2599119E-3,5.948464E0,2.7954546E-1,2.1845242E3,8.412314E0,1.579932E5,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"split_indices":[4,5,102,49,46,34,9,89,29,2,0,0,10,0,0,40,3,40,34,0,0,53,43,0,0,0,49,0,0,8,4,0,0,0,0,0,9,0,0,40,54,58,4,55,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.55E2,4.8E1,1.43E2,1.2E1,2.1E1,2.7E1,9.3E1,5E1,6E0,6E0,2E0,1.9E1,4E0,2.3E1,8.7E1,6E0,4E1,1E1,4E0,2E0,8E0,1.1E1,2E0,2.1E1,1.5E1,7.2E1,3E0,3E0,2.2E1,1.8E1,7E0,3E0,5E0,3E0,4E0,7E0,5E0,1E1,1.9E1,5.3E1,8E0,1.4E1,7E0,1.1E1,5E0,2E0,7E0,1.2E1,1.2E1,4.1E1,5E0,3E0,1E1,4E0,5E0,2E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.2111462E-2,-1.5450773E-1,1.031491E-1,-1.09187275E-1,-4.5132187E-1,-2.8939374E-2,1.8243721E-1,-2.6891872E-1,-6.3274145E-2,-2.8736094E-2,-2.1376169E-1,-1.9380113E-1,1.3079939E-2,-5.3318623E-2,2.8723615E-1,-3.048581E-1,-3.2565857E-3,-2.1935193E-1,-2.3797065E-2,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,-1.15509786E-1,4.264821E-2,1.4384617E-1,-1.3136774E-1,1.7543119E-1,4.10858E-1,-3.4710836E-1,-7.3455116E-3,-2.187251E-3,-2.6718003E-1,-4.2986523E-2,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.1813E-2,-8.5979775E-2,1.9637176E-3,1.3061595E-2,-5.950475E-2,-2.1748714E-2,2.684922E-1,1.24782175E-1,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.991617E-2,-1.04395926E-1,8.574969E-3,4.3539573E-2,-5.5793906E-3,-9.2947256E-4,-9.633975E-2,3.2059713E-3,4.982372E-3,1.4664561E-2,1.5088551E-1,3.8726836E-2,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,-1,53,55,-1,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,63,65,-1,67,-1,-1,69,-1,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4133608E0,1.2017343E0,1.2096798E0,5.8435357E-1,2.704537E-1,3.0733958E-1,1.7942166E0,1.1027694E-1,3.9527723E-1,0E0,1.3330251E-2,9.3468994E-2,1.4180462E-1,3.728331E-1,6.189995E-1,4.515779E-2,0E0,9.6432924E-2,2.5955042E-1,0E0,0E0,0E0,0E0,1.570262E-2,1.1871672E-1,7.939862E-2,3.5887626E-1,9.847349E-2,1.3197088E-1,3.6898494E-2,0E0,0E0,1.9970298E-2,2.5871733E-1,0E0,0E0,0E0,6.611198E-2,9.313557E-3,0E0,0E0,7.5622745E-2,0E0,1.8241048E-2,3.9083302E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.800409E-1,1.35196E-1,0E0,3.6446035E-2,0E0,0E0,6.822489E-2,0E0,0E0,0E0,3.4981847E-2,1.2126244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,37,37,38,38,41,41,43,43,44,44,51,51,52,52,54,54,57,57,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,-1,54,56,-1,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,64,66,-1,68,-1,-1,70,-1,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8601307E-2,1.8000048E10,6.787619E2,1.142E3,1.527638E12,2.853839E4,1E0,1.3448806E8,4.059859E-5,-2.8736094E-2,2.906E3,3.6E1,1.7218965E2,2E0,1.3173E4,2.23695E5,-3.2565857E-3,1.7727281E2,3.3E1,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,3.0345E4,4.6E1,4.078932E5,8.476884E-1,9.686559E6,4.747723E6,3.7790768E2,-7.3455116E-3,-2.187251E-3,6.802721E-3,4.454691E5,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.751E3,1.251E3,1.9637176E-3,1.3061595E-2,1.257013E0,-2.1748714E-2,8.366892E4,8.442676E0,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.5E1,5.979E3,8.574969E-3,2.702E3,-5.5793906E-3,-9.2947256E-4,2.012E3,3.2059713E-3,4.982372E-3,1.4664561E-2,7.595543E-1,1.3235667E9,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"split_indices":[39,5,53,2,32,34,102,7,39,0,0,3,53,8,2,12,0,34,8,0,0,0,0,9,3,29,35,52,33,34,0,0,58,49,0,0,0,9,0,0,0,40,0,34,55,0,0,0,0,0,0,3,2,0,2,0,0,0,0,0,0,28,5,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,9.2E1,1.14E2,8.1E1,1.1E1,4.3E1,7.1E1,1.7E1,6.4E1,6E0,5E0,8E0,3.5E1,2.2E1,4.9E1,1.4E1,3E0,1.2E1,5.2E1,2E0,3E0,5E0,3E0,6E0,2.9E1,6E0,1.6E1,2.7E1,2.2E1,1E1,4E0,3E0,9E0,4.9E1,3E0,4E0,2E0,2.4E1,5E0,4E0,2E0,1.4E1,2E0,8E0,1.9E1,2E0,2E1,3E0,7E0,5E0,4E0,2.1E1,2.8E1,4E0,2E1,3E0,2E0,1.1E1,3E0,2E0,6E0,1.4E1,5E0,1E1,1.1E1,2E1,8E0,1.3E1,7E0,5E0,6E0,9E0,5E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.2747186E-3,-5.4162722E-2,1.8111008E-1,-3.108007E-2,-4.4845343E-1,7.1374275E-2,3.275183E-1,-9.3600295E-2,1.1067671E-1,-2.5658727E-2,-6.88157E-3,1.2687185E-1,-2.088327E-2,2.448392E-3,3.571717E-1,-2.3487736E-1,-5.843745E-2,1.925611E-1,-3.9538275E-2,9.78743E-3,3.3138932E-3,-7.214252E-2,6.806273E-3,1.9431083E-2,1.0387621E-2,-1.5502349E-1,-1.7014932E-2,5.5956807E-5,-1.2483689E-1,2.5137913E-1,-3.1633927E-3,-1.7023765E-1,8.103208E-2,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,2.859666E-2,-1.6140592E-1,6.8674185E-3,1.9025224E-1,2.8979616E-2,-1.5382758E-2,-9.495701E-4,-2.327769E-2,1.6913544E-1,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9040357E0,1.3148481E0,6.996037E-1,1.2422698E0,1.18970394E-1,1.4089137E-1,1.3821435E-1,4.6814162E-1,5.365249E-1,0E0,0E0,6.0488462E-2,1.007224E-1,0E0,1.7639875E-2,1.198957E-1,3.0683744E-1,4.4052112E-1,2.659302E-1,0E0,0E0,2.7664423E-2,0E0,0E0,0E0,2.1903068E-2,0E0,2.2701421E-1,1.7931145E-1,3.7047946E-1,0E0,1.6969696E-1,8.6647436E-2,0E0,0E0,0E0,0E0,0E0,1.13961965E-1,6.886476E-2,1.559542E-2,2.4640507E-1,0E0,0E0,0E0,1.2072772E-2,1.1905044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,25,25,27,27,28,28,29,29,31,31,32,32,38,38,39,39,40,40,41,41,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4436649E0,1.4233672E8,1.4008022E11,1.0648859E3,3.7284137E2,2.7883E4,1E0,6.126268E0,2.249985E5,-2.5658727E-2,-6.88157E-3,3.5037E4,5.53839E5,2.448392E-3,2.7E1,7.0712406E5,7.269755E0,9.040637E7,9.875E3,9.78743E-3,3.3138932E-3,1.009E3,6.806273E-3,1.9431083E-2,1.0387621E-2,2.44412E3,-1.7014932E-2,1.053E3,2.1102592E8,5.902126E3,-3.1633927E-3,4.6E1,3.858136E6,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,1E0,7.3376025E6,1.9519106E-1,2.8880222E6,2.8979616E-2,-1.5382758E-2,-9.495701E-4,4.895358E8,8.51E2,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"split_indices":[43,46,32,53,59,9,102,55,34,0,0,30,29,0,8,29,54,46,9,0,0,0,0,0,0,48,0,2,7,4,0,3,48,0,0,0,0,0,89,33,40,33,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.89E2,1.45E2,4.4E1,1.38E2,7E0,2.6E1,1.8E1,9.6E1,4.2E1,5E0,2E0,1.6E1,1E1,2E0,1.6E1,1.8E1,7.8E1,2.7E1,1.5E1,6E0,1E1,8E0,2E0,1.1E1,5E0,1.2E1,6E0,4.2E1,3.6E1,2.2E1,5E0,7E0,8E0,6E0,2E0,4E0,8E0,5E0,3.7E1,2.8E1,8E0,2E1,2E0,3E0,4E0,4E0,4E0,3.3E1,4E0,2.2E1,6E0,2E0,6E0,9E0,1.1E1,2E0,2E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.826838E-3,-6.721812E-2,1.9046006E-1,-7.924094E-2,2.6099376E-2,5.0246296E-3,2.6243427E-1,-2.5692692E-1,-4.875577E-2,-3.755318E-2,3.6572933E-3,2.1474557E-1,1.9588318E-2,-4.4567264E-3,-2.904459E-1,-3.542672E-2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,2.688706E-1,1.3754945E-1,-4.928484E-3,-3.0891788E-1,-1.2508754E-1,4.5123346E-2,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,-1.1071097E-1,1.8496023E-2,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,-1,21,23,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,-1,-1],"loss_changes":[2.477284E0,1.1091111E0,6.6600955E-1,7.729075E-1,0E0,4.7609035E-2,1.5403104E-1,8.979154E-2,6.413361E-1,3.7017375E-2,0E0,8.418608E-2,0E0,0E0,2.8120995E-2,8.8675094E-1,0E0,0E0,0E0,1.0490298E-2,3.518188E-2,0E0,1.5468597E-2,1.0648471E-1,4.4222242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2309903E-1,2.3909211E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,14,14,15,15,19,19,20,20,22,22,23,23,24,24,32,32,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,-1,22,24,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,7.8040985E2,5.5200674E3,2.6099376E-2,6.2958473E4,1.2790948E0,5.4E0,1.4146589E8,4.71E2,3.6572933E-3,9.6494156E-1,1.9588318E-2,-4.4567264E-3,4.146E3,5.3287085E2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,7.810231E4,1.5593E4,-4.928484E-3,1.2187347E3,6.35E2,1.2241036E12,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,4.2E1,2.122531E6,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"split_indices":[43,57,4,49,0,34,36,55,46,0,0,58,0,0,30,53,0,0,0,34,9,0,4,2,32,0,0,0,0,0,0,0,3,48,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.46E2,4.9E1,1.44E2,2E0,1.4E1,3.5E1,2E1,1.24E2,9E0,5E0,2.8E1,7E0,4E0,1.6E1,1.21E2,3E0,4E0,5E0,1.5E1,1.3E1,2E0,1.4E1,5.7E1,6.4E1,4E0,1.1E1,5E0,8E0,1.2E1,2E0,3E0,5.4E1,5.9E1,5E0,4.8E1,6E0,3.8E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.4262974E-3,-8.005432E-2,1.4594802E-1,-2.888352E-1,-4.6757724E-2,-1.8107321E-2,2.4755892E-1,-4.6657827E-3,-3.4083346E-1,-1.0410578E-1,1.8106598E-2,2.52526E-2,-7.499621E-2,7.3558204E-2,3.0511466E-1,-1.0108946E-2,-1.9372413E-2,-1.6241044E-1,-4.8085254E-2,6.1593957E-2,-1.17093466E-1,8.193563E-2,-1.12674244E-1,-1.9586612E-3,1.11724325E-1,3.5032213E-1,1.3559058E-1,-6.2673695E-2,-2.1433787E-1,7.613574E-2,-8.7488696E-2,8.202132E-2,-6.9475058E-3,-5.7769872E-2,-1.4384934E-2,7.815317E-5,7.870827E-3,-1.6748264E-1,-8.807537E-3,7.7555277E-3,2.0889724E-5,3.6501428E-1,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,-3.6418545E-3,-1.605696E-1,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,-1.1916163E-1,1.1715029E-1,1.5941553E-2,1.3056229E-3,-9.284757E-2,-2.2564225E-1,-2.0749527E-3,-2.448091E-3,2.7077156E-3,3.8617578E-1,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,53,-1,55,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,67,69,71,-1,73,75,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4140606E0,8.554564E-1,1.2713884E0,1.3789642E-1,4.0734783E-1,9.553926E-1,4.482491E-1,0E0,1.6353011E-2,1.8163341E-1,3.1294593E-1,0E0,1.7479275E-1,6.096188E-2,2.2073102E-1,0E0,0E0,1.3158071E-1,1.5839781E-1,1.786837E-1,1.20575935E-1,3.8199473E-2,1.2945855E-1,0E0,5.294875E-2,5.433297E-2,3.3570766E-2,4.5042265E-2,5.2512467E-2,7.294734E-2,5.8373034E-2,8.402106E-2,0E0,3.5133395E-2,0E0,0E0,0E0,9.929475E-2,2.6866937E-2,0E0,0E0,4.9077034E-2,0E0,0E0,0E0,0E0,2.0661239E-2,1.6366512E-2,0E0,0E0,0E0,0E0,5.8940127E-2,1.0018352E-1,5.5888433E-2,0E0,2.880384E-2,6.1261773E-2,0E0,0E0,0E0,1.6714334E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,37,37,38,38,41,41,46,46,47,47,52,52,53,53,54,54,56,56,57,57,61,61],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,54,-1,56,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,68,70,72,-1,74,76,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.6770158E-4,1E0,6.051637E-1,3.3030225E2,6.948189E7,6.201765E5,-4.6657827E-3,1.57525425E1,2.0562818E4,9.959017E0,2.52526E-2,9.31E2,1.0363E4,5.03433E0,-1.0108946E-2,-1.9372413E-2,1.0363E4,2.0589474E1,6.579211E9,3.2776688E7,8.319244E8,1.5574268E7,-1.9586612E-3,3.4E1,5.2E1,1.5E1,5.3943863E0,3.62043E1,9.075745E7,5.948464E0,4.9212202E-1,-6.9475058E-3,2E1,-1.4384934E-2,7.815317E-5,7.870827E-3,5.840393E-1,6.804E3,7.7555277E-3,2.0889724E-5,1.0528413E10,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,6.948189E7,1.2421726E7,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,2E0,1.4265206E1,2.1150263E-1,1.3056229E-3,1.5996247E1,2.6023E4,-2.0749527E-3,-2.448091E-3,2.7077156E-3,1.4456249E7,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"split_indices":[53,40,102,28,53,7,49,0,57,34,55,0,0,9,58,0,0,9,59,5,33,12,1,0,3,8,8,54,59,7,54,58,0,3,0,0,0,43,9,0,0,12,0,0,0,0,7,46,0,0,0,0,8,57,39,0,59,10,0,0,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.24E2,7.5E1,1.6E1,1.08E2,2.9E1,4.6E1,4E0,1.2E1,5.7E1,5.1E1,2E0,2.7E1,1.2E1,3.4E1,5E0,7E0,2.7E1,3E1,3.9E1,1.2E1,5E0,2.2E1,3E0,9E0,2.6E1,8E0,1E1,1.7E1,7E0,2.3E1,3.6E1,3E0,1E1,2E0,3E0,2E0,1.4E1,8E0,6E0,3E0,2.4E1,2E0,5E0,3E0,4E0,6E0,1.3E1,4E0,4E0,3E0,7E0,1.6E1,2.3E1,1.3E1,3E0,7E0,9E0,5E0,5E0,3E0,2.1E1,3E0,2E0,4E0,7E0,6E0,1.3E1,3E0,1.3E1,1E1,8E0,5E0,2E0,5E0,2E0,7E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.6450368E-2,-1.1724766E-1,1.2559639E-1,-7.511801E-2,-3.363559E-1,1.8126132E-1,-2.3499241E-1,-1.6364163E-1,-1.0940317E-2,-1.8125798E-1,-4.375745E-1,1.2667201E-1,3.6933804E-1,-4.659123E-2,-2.6914816E-2,-2.2536351E-1,-1.00311846E-1,7.6694086E-2,-4.540101E-2,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,-1.531599E-2,2.0203556E-1,4.1224214E-1,2.9349497E-3,1.9489948E-3,-3.9977366E-3,-2.5541717E-1,2.9264723E-3,1.9217292E-2,-1.2399441E-1,1.2778491E-1,-2.9145163E-2,-7.644804E-2,6.597971E-2,5.250523E-2,-1.4877042E-1,2.3447502E-1,7.5913966E-4,2.1494467E-2,8.123851E-3,-2.7855477E-1,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,-7.0292644E-2,7.10427E-2,8.864286E-3,-2.6523983E-3,1.1110839E-3,1.0315416E-2,-1.1217816E-1,-7.6833105E-4,1.1495934E-1,9.3826205E-2,-1.8227108E-2,-1.1465208E-2,-1.2050162E-3,2.9752174E-1,1.3666672E-1,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,-1,-1,65,-1,-1,-1,-1,67,69,-1,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.913867E0,1.1932194E0,1.6282953E0,6.345087E-1,2.5036097E-1,6.616366E-1,6.333728E-1,1.611085E-1,2.0279819E-1,5.5682838E-2,9.7411156E-2,5.916339E-1,1.740191E-1,2.7933039E-2,0E0,2.1270418E-1,7.315171E-2,1.06465176E-1,1.7102996E-1,0E0,0E0,0E0,0E0,1.8883207E-1,2.3487675E-1,2.0117044E-2,0E0,0E0,0E0,1.0595679E-1,0E0,2.1876162E-2,3.615102E-2,2.7157709E-2,1.046974E-2,1.1895977E-1,4.5869254E-2,4.262839E-2,6.952983E-2,1.5049982E-1,3.6906146E-2,0E0,0E0,4.3109298E-2,0E0,0E0,0E0,0E0,1.5443023E-2,1.32172E-2,0E0,0E0,0E0,9.080692E-2,1.4392808E-1,0E0,9.912826E-3,3.9224803E-2,1.2763529E-2,0E0,0E0,1.4601588E-2,1.00580364E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,48,48,49,49,53,53,54,54,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,-1,-1,66,-1,-1,-1,-1,68,70,-1,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.352552E9,1.2055723E8,5.162122E2,3.4E1,5.0561914E3,1.5119754E0,1.822764E4,5.415E4,9.285558E4,1.2E1,1E0,1.02E2,5.4438036E9,-2.6914816E-2,1E0,2.1768606E5,6.0511904E0,2.2583E4,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,3.20566E7,6.1108776E7,9.298568E7,2.9349497E-3,1.9489948E-3,-3.9977366E-3,6.1E1,2.9264723E-3,8.637E3,2.8774E4,2E0,4.411222E6,1.0092749E8,8.3154816E2,1.4364E4,5.840393E-1,8.383825E1,1.91E2,2.1494467E-2,8.123851E-3,3.5623631E0,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,5.6058385E6,6.314179E2,8.864286E-3,-2.6523983E-3,1.1110839E-3,9.075745E7,2.0053221E6,-7.6833105E-4,4.7E1,9.355911E-1,5.314E3,-1.1465208E-2,-1.2050162E-3,6.518218E6,2.6678162E9,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"split_indices":[53,5,46,56,3,53,58,34,30,34,8,102,3,5,0,105,29,54,10,0,0,0,0,51,46,46,0,0,0,3,0,9,9,8,33,7,4,10,43,59,10,0,0,57,0,0,0,0,33,4,0,0,0,7,49,0,8,28,9,0,0,46,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.31E2,7.8E1,1.11E2,2E1,6.8E1,1E1,4.6E1,6.5E1,9E0,1.1E1,5.4E1,1.4E1,7E0,3E0,2.2E1,2.4E1,1.8E1,4.7E1,7E0,2E0,7E0,4E0,1.9E1,3.5E1,1.2E1,2E0,2E0,5E0,2E1,2E0,4E0,2E1,1.2E1,6E0,3.7E1,1E1,1.3E1,6E0,3E1,5E0,1E1,2E0,1.8E1,2E0,2E0,2E0,1E1,1E1,7E0,5E0,4E0,2E0,1.1E1,2.6E1,4E0,6E0,8E0,5E0,3E0,3E0,1.7E1,1.3E1,2E0,3E0,1E1,8E0,7E0,3E0,3E0,4E0,7E0,4E0,1.2E1,1.4E1,4E0,2E0,6E0,2E0,2E0,3E0,7E0,1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.8897833E-2,-6.780625E-2,2.0897096E-1,-1.5521011E-2,-2.9606915E-1,3.5466662E-1,-4.421246E-3,-9.93627E-2,9.932131E-2,-1.4433515E-1,-4.58412E-1,4.7411785E-1,2.1561413E-1,9.5952526E-2,-1.4718074E-1,-2.0804802E-1,-6.7736566E-2,1.9759446E-1,2.8317139E-2,-1.7917114E-1,-4.480132E-2,-5.3120285E-1,-6.104539E-3,5.165622E-1,9.185563E-3,2.6545724E-1,2.4120836E-3,7.911076E-3,-3.163457E-3,-1.2183908E-2,-3.5092747E-4,-2.297472E-1,-1.1178532E-3,1.8829303E-2,-9.4082914E-2,1.3437358E-1,1.6593615E-2,-1.20509595E-1,6.209947E-2,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,-2.9204038E-1,-6.37282E-3,-4.2878713E-2,1.0429371E-1,-4.7475312E-2,-1.8176384E-1,2.5304276E-2,1.6177177E-1,6.3902856E-4,-8.5941E-3,1.6473201E-1,1.7290728E-2,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,-1,-1,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,67,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3322675E0,2.048187E0,1.1519101E0,1.3674978E0,7.220886E-1,2.631929E-1,2.4339132E-1,2.6776582E-1,4.1307282E-1,5.4385334E-2,2.9693174E-1,3.5738707E-2,8.616072E-2,6.653001E-2,9.928414E-2,6.7011595E-2,1.5001667E-1,1.8078548E-1,1.8848129E-1,1.2413353E-2,1.4049292E-2,2.2869587E-2,0E0,3.3781767E-2,0E0,5.2916467E-2,0E0,0E0,1.2511282E-2,0E0,0E0,6.0228944E-2,0E0,8.88534E-2,1.9570062E-1,5.2684218E-2,0E0,5.4782815E-2,1.3511673E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1208367E-2,0E0,1.15634445E-2,1.6972624E-2,1.08784124E-1,5.8658242E-2,1.0622809E-2,1.1181682E-2,0E0,0E0,4.1976526E-2,3.8918935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,28,28,31,31,33,33,34,34,35,35,37,37,38,38,51,51,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,-1,-1,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,68,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.2829284E7,6.627576E7,5.3287085E2,8.67241E9,8.812601E-1,3.511154E7,1.056E3,1.26881E5,2.1845242E3,5.770668E6,3.2707875E8,1.7414924E7,5.1E1,1.4539318E-1,6.1E1,2.5126263E1,1.467574E3,2.035483E6,1.0892118E3,7.6336784E1,6.1826086E0,-6.104539E-3,4.5E1,9.185563E-3,4.747723E6,2.4120836E-3,7.911076E-3,7.358531E3,-1.2183908E-2,-3.5092747E-4,2.23695E5,-1.1178532E-3,6.21E2,9.348848E0,5.9734796E2,1.6593615E-2,9.31E2,2.7E1,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,2.2290532E3,-6.37282E-3,5.093772E-1,3.396881E4,1.1556807E0,7.688406E0,3.7E1,1.617623E5,6.3902856E-4,-8.5941E-3,1.38635E5,1.8454842E-1,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"split_indices":[53,46,46,53,5,58,33,2,30,4,30,33,33,3,39,3,59,4,33,53,57,54,0,3,0,33,0,0,4,0,0,12,0,11,55,53,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,49,0,28,34,43,54,3,34,0,0,11,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.71E2,3.6E1,1.4E2,3.1E1,2.1E1,1.5E1,8.1E1,5.9E1,1.7E1,1.4E1,1E1,1.1E1,9E0,6E0,1.7E1,6.4E1,2.4E1,3.5E1,1.2E1,5E0,1.1E1,3E0,8E0,2E0,8E0,3E0,5E0,4E0,3E0,3E0,1.5E1,2E0,1.5E1,4.9E1,1.8E1,6E0,6E0,2.9E1,8E0,4E0,3E0,2E0,9E0,2E0,6E0,2E0,2E0,6E0,2E0,2E0,8E0,7E0,9E0,6E0,3.3E1,1.6E1,4E0,1.4E1,2E0,4E0,8E0,2.1E1,2E0,6E0,6E0,3E0,4E0,2E0,2E1,1.3E1,1E1,6E0,2E0,2E0,1.2E1,2E0,2E0,6E0,3E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[9.597382E-3,-7.689029E-2,1.3135731E-1,-5.166708E-2,-3.934461E-1,1.9286239E-1,-1.5746945E-1,-1.8152632E-1,-2.5963075E-2,-2.2153566E-2,-5.7512084E-3,1.2324249E-1,4.3318063E-1,-8.0194466E-2,-2.265891E-2,-2.6121805E-3,-2.3640567E-1,5.765824E-4,-9.947969E-2,-1.1233864E-2,1.7927498E-1,5.3216517E-1,2.2989117E-1,-1.9807076E-1,6.597765E-2,-4.3076635E-3,-1.2422029E-2,2.8069925E-2,-1.1729321E-1,6.657329E-2,-1.4218627E-1,9.798071E-2,-6.077645E-2,-1.3545295E-3,1.9635805E-1,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,7.5570874E-2,-6.374473E-2,-1.7920145E-1,-1.4495179E-3,2.901012E-4,4.642252E-3,-8.290076E-2,-1.9911025E-1,1.871269E-3,7.0417197E-3,-5.9855576E-3,-1.9459605E-2,2.7096123E-1,1.4672595E-1,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,63,65,-1,-1,-1,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2320848E0,9.697564E-1,1.5936292E0,3.824837E-1,9.9226475E-2,1.1679718E0,3.4362957E-1,1.2062222E-1,1.912657E-1,0E0,0E0,4.3904662E-1,1.9962025E-1,2.5429142E-1,0E0,0E0,1.3633668E-2,2.3985873E-1,1.9384623E-1,1.0334882E-1,1.5052485E-1,1.5721798E-2,2.7384043E-2,1.3692349E-2,4.27702E-2,0E0,0E0,2.664942E-1,7.050744E-2,9.9109635E-3,5.4363728E-2,1.1121236E-2,3.102351E-2,0E0,1.0094595E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2414542E-1,1.4501756E-1,3.8122535E-2,0E0,0E0,0E0,4.5270823E-2,2.6841223E-2,0E0,0E0,0E0,1.3282825E-2,1.0726178E-1,4.9301982E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,34,34,43,43,44,44,45,45,49,49,50,50,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,64,66,-1,-1,-1,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.2109607E2,8.260109E9,9.298568E7,1.295E3,9.4819355E0,5.08711E3,2.88001E10,5.7625353E-1,1.5989196E7,-2.2153566E-2,-5.7512084E-3,1E0,2.7E1,1.5574268E7,-2.265891E-2,-2.6121805E-3,2.9585883E2,1.193419E1,1.3729923E5,1E0,7.056125E7,9.886E3,1.917786E7,1.15E2,3.9E1,-4.3076635E-3,-1.2422029E-2,5.998024E-1,8.401064E0,2.68E2,1.09137096E8,1.9253E4,2.1845242E3,-1.3545295E-3,1.2951E4,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,4.1376923E2,8.05E2,2.6239506E5,-1.4495179E-3,2.901012E-4,4.642252E-3,3.1274893E2,1.0209414E1,1.871269E-3,7.0417197E-3,-5.9855576E-3,4E1,1.1865996E1,1.4456249E7,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"split_indices":[53,5,46,2,54,53,5,28,46,0,0,102,8,1,0,0,34,55,29,8,7,9,1,10,3,0,0,58,54,0,7,2,4,0,9,0,0,0,0,0,0,0,0,53,0,29,0,0,0,53,55,0,0,0,3,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.23E2,8.7E1,1.15E2,8E0,7.2E1,1.5E1,1.8E1,9.7E1,6E0,2E0,5.7E1,1.5E1,1.3E1,2E0,6E0,1.2E1,7.2E1,2.5E1,1.7E1,4E1,9E0,6E0,7E0,6E0,2E0,1E1,5.9E1,1.3E1,5E0,2E1,5E0,1.2E1,3E0,3.7E1,4E0,5E0,3E0,3E0,2E0,5E0,2E0,4E0,3.9E1,2E1,7E0,6E0,2E0,3E0,1.1E1,9E0,3E0,2E0,4E0,8E0,1.3E1,2.4E1,2.4E1,1.5E1,1.1E1,9E0,4E0,3E0,4E0,7E0,7E0,2E0,4E0,4E0,1.1E1,2E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-1.772178E-2,-7.057884E-2,8.537469E-2,-5.372641E-2,-3.503544E-1,-7.3327735E-2,1.9524743E-1,-1.9434269E-1,-2.6594644E-2,-2.087923E-2,-8.001967E-3,1.7575795E-2,-2.1872297E-1,1.2774226E-1,3.8615E-1,-2.1584399E-1,-2.1231158E-3,2.6761827E-3,-1.002963E-1,-2.3103694E-2,1.2448591E-2,-3.2898784E-1,7.034172E-5,-6.594587E-2,1.6341075E-1,2.4855476E-2,1.1777236E-2,-2.5548145E-1,-6.297792E-3,6.468007E-2,-6.0867704E-2,3.847441E-3,-1.2086692E-1,1.0591442E-1,-6.7531325E-2,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,1.8735889E-1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,1.1528385E-1,-3.7465915E-2,-1.02739446E-1,5.237328E-2,-1.373252E-1,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,-2.5229553E-2,2.3754518E-1,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,-1,-1,41,-1,43,45,-1,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,55,57,59,61,63,-1,-1,-1,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1885403E0,6.6012216E-1,1.3046603E0,5.1750946E-1,4.6478212E-2,4.1356945E-1,5.176964E-1,6.1877012E-2,2.5051618E-1,0E0,0E0,1.9972178E-1,2.9179746E-1,2.4566746E-1,7.394695E-2,3.203535E-2,0E0,3.3489424E-1,1.3130611E-1,1.10328E-1,0E0,9.977138E-2,0E0,1.4139006E-2,1.0379839E-1,0E0,0E0,6.927937E-2,0E0,2.2658505E-1,2.045279E-1,0E0,7.257894E-2,1.1413369E-2,8.147253E-2,0E0,0E0,0E0,0E0,8.3029985E-2,0E0,0E0,0E0,9.5561266E-2,5.3016253E-2,1.7131087E-1,4.7250487E-2,5.080968E-2,0E0,0E0,0E0,0E0,2.8906034E-2,3.853917E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,23,23,24,24,27,27,29,29,30,30,32,32,33,33,34,34,39,39,43,43,44,44,45,45,46,46,47,47,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,-1,-1,42,-1,44,46,-1,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,56,58,60,62,64,-1,-1,-1,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.142E3,6.645232E2,8.307621E7,1.9253E4,1.3448806E8,1.6278354E7,-2.087923E-2,-8.001967E-3,1.9142364E7,1.4539318E-1,7.5773336E7,1.2117E4,2.23695E5,-2.1231158E-3,1.643E4,5.577E3,8.87E2,1.2448591E-2,2.2982106E0,7.034172E-5,1.33511E5,7.4308796E0,2.4855476E-2,1.1777236E-2,3.7790768E2,-6.297792E-3,1.0466548E1,6.3004294E5,3.847441E-3,9.568003E2,1.4783E4,2.9E1,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,7.4009796E1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,5.3055557E1,6.693824E5,4.966E3,1.175E1,1.30468176E11,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,7.7573473E9,4.60675E5,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"split_indices":[53,5,102,2,34,46,2,7,46,0,0,1,39,7,9,12,0,9,9,0,0,54,0,12,54,0,0,34,0,55,29,0,53,9,3,0,0,0,0,59,0,0,0,59,49,2,57,32,0,0,0,0,5,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.43E2,7.3E1,1.36E2,7E0,3E1,4.3E1,2.1E1,1.15E2,4E0,3E0,1.9E1,1.1E1,3.3E1,1E1,1.8E1,3E0,8.3E1,3.2E1,1.7E1,2E0,7E0,4E0,5E0,2.8E1,4E0,6E0,1.1E1,7E0,4.2E1,4.1E1,3E0,2.9E1,4E0,1.3E1,2E0,5E0,3E0,2E0,2.4E1,4E0,4E0,7E0,2.8E1,1.4E1,3E1,1.1E1,2.6E1,3E0,2E0,2E0,2E0,1.1E1,1.4E1,1E1,1.7E1,1.1E1,4E0,1E1,2.5E1,5E0,6E0,5E0,1.3E1,1.3E1,7E0,4E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.1619442E-2,-6.1717395E-2,1.3433132E-1,-2.3387735E-1,-4.0964365E-2,1.77865E-3,2.3689537E-1,1.3427171E-2,-3.110039E-1,-1.2818594E-1,-3.7928703E-3,5.2376132E-2,-8.1358645E-3,2.3620052E-2,1.8211213E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,-1.5684637E-1,1.68092E-2,7.7523957E-3,-1.8851958E-2,-1.6274573E-2,1.226543E-1,2.2858551E-1,6.3767E-2,-1.8596564E-1,-4.8825707E-2,8.553022E-2,-6.1434526E-3,-7.6919543E-3,1.647666E-1,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,1.4144917E-1,1.4252614E-2,4.003073E-4,4.388098E-3,-1.9882518E-1,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,-2.6464459E-2,1.329478E-1,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,-1,-1,-1,-1,27,29,31,-1,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,53,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5730232E0,5.620738E-1,7.473112E-1,3.2842952E-1,4.6638432E-1,2.2247311E-1,3.3355105E-1,3.4519482E-2,7.46156E-2,1.8132824E-1,4.54711E-1,9.848905E-2,0E0,0E0,1.3392115E-1,0E0,0E0,0E0,0E0,1.0415751E-1,8.9438625E-2,2.4376561E-1,0E0,1.8215474E-2,2.6880935E-2,6.499237E-2,1.245489E-2,5.9958518E-2,5.9645988E-2,1.10448785E-2,0E0,2.4641316E-1,2.660959E-1,0E0,0E0,0E0,0E0,1.1662543E-2,0E0,0E0,0E0,2.202785E-2,0E0,0E0,0E0,0E0,0E0,1.3542011E-1,6.190428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,37,37,41,41,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,-1,-1,-1,-1,28,30,32,-1,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,54,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,6.794276E-5,6.48045E6,1.7727281E2,2.802166E2,1.0430015E1,6.677E3,1.94122E5,3.2445145E2,1.01978E5,1.0590684E1,2.806E3,-8.1358645E-3,2.3620052E-2,8.269529E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,2.1974122E5,4.87374E5,4.9192E4,-1.8851958E-2,1.5878E4,2.0837E4,1.980161E0,4.602015E0,6.1E1,3.7E1,3.52777E5,-6.1434526E-3,1E0,1.0670794E10,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,5.649635E0,1.4252614E-2,4.003073E-4,4.388098E-3,1.8E2,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,3.2441288E5,3.9360254E2,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"split_indices":[43,43,49,34,53,55,9,1,34,30,58,2,0,0,28,0,0,0,0,48,1,2,0,10,9,43,54,3,3,1,0,89,12,0,0,0,0,54,0,0,0,11,0,0,0,0,0,29,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.59E2,5.4E1,1.6E1,1.43E2,2.4E1,3E1,4E0,1.2E1,4.2E1,1.01E2,1.9E1,5E0,4E0,2.6E1,2E0,2E0,5E0,7E0,3.5E1,7E0,9.9E1,2E0,1E1,9E0,1.8E1,8E0,2.7E1,8E0,5E0,2E0,9.1E1,8E0,6E0,4E0,7E0,2E0,9E0,9E0,3E0,5E0,2.5E1,2E0,6E0,2E0,2E0,3E0,8.1E1,1E1,4E0,4E0,4E0,5E0,4E0,2.1E1,2.7E1,5.4E1,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.0766466E-3,-4.316265E-2,2.0560837E-1,-1.2818186E-2,-2.3354582E-1,2.703695E-1,-9.532744E-2,-5.3655125E-2,1.0286662E-1,-2.9023632E-1,-1.5101601E-2,2.0894068E-1,4.177894E-1,1.6216272E-2,-1.39497025E-2,-1.1859325E-1,-2.1754269E-2,2.239287E-1,4.9224846E-2,-7.534239E-3,-3.746514E-1,-5.9629036E-3,3.3697663E-3,7.089449E-2,2.384329E-1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,-1.6322732E-1,-5.6480993E-2,2.9111031E-2,-7.753057E-2,1.5182119E-2,4.083155E-3,8.881563E-2,-4.8623074E-2,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,2.5431147E-1,5.2253725E-3,6.213173E-4,-1.8242495E-1,-1.1897854E-2,-5.765586E-3,5.034584E-2,-1.3384877E-1,7.912595E-3,-1.0946924E-1,-9.919967E-3,1.289712E-1,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,37,-1,-1,39,41,-1,-1,-1,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,-1,55,-1,-1,57,59,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.055721E0,1.0079945E0,8.2482004E-1,7.235998E-1,2.9282618E-1,2.2060513E-1,1.7536506E-1,2.3150888E-1,2.487365E-1,1.5920115E-1,6.284737E-2,8.894992E-2,2.0777345E-2,1.1685234E-2,0E0,9.335673E-2,2.2071204E-1,1.2175274E-1,1.1666089E-1,0E0,1.730293E-1,0E0,0E0,3.1039445E-2,1.6396642E-2,0E0,0E0,0E0,0E0,7.328111E-2,4.5573883E-2,1.4861596E-1,1.0183704E-1,0E0,0E0,8.453986E-2,1.6937668E-2,0E0,0E0,0E0,0E0,1.6923666E-2,0E0,0E0,3.164649E-2,3.1010306E-2,0E0,1.0485908E-1,1.7051622E-2,8.5487984E-2,7.8597695E-2,2.3515034E-2,7.874721E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,20,20,23,23,24,24,29,29,30,30,31,31,32,32,35,35,36,36,41,41,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,38,-1,-1,40,42,-1,-1,-1,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,-1,56,-1,-1,58,60,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.0099896E9,1.2055723E8,8.297185E2,1.2728E4,1.4243088E6,2.374331E5,2.1015242E2,1.10597E5,5.033165E7,1.4792529E10,4.0492815E-1,1.4735E4,9.4908775E1,-1.39497025E-2,3.5E1,7.2929355E6,6.675932E1,8.7022424E-1,-7.534239E-3,8.260109E9,-5.9629036E-3,3.3697663E-3,8.601656E6,5.1E1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,7.36E2,3.5223506E5,1.996866E1,4.008916E10,1.5182119E-2,4.083155E-3,9.1014955E10,2.7577372E11,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,1.6503E4,5.2253725E-3,6.213173E-4,1.13002876E-4,6.693824E5,-5.765586E-3,2.579E3,6.09E2,1.438E3,1.8272479E3,5.0895057E10,1E0,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"split_indices":[53,5,46,53,9,29,34,53,30,7,5,28,2,57,0,3,46,59,28,0,5,0,0,1,3,0,0,0,0,30,29,57,32,0,0,32,32,0,0,0,0,2,0,0,39,49,0,2,0,2,4,32,102,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.74E2,4E1,1.51E2,2.3E1,3.3E1,7E0,1.12E2,3.9E1,1.8E1,5E0,2.5E1,8E0,5E0,2E0,3.6E1,7.6E1,1.1E1,2.8E1,8E0,1E1,2E0,3E0,5E0,2E1,2E0,6E0,2E0,3E0,2E1,1.6E1,4E1,3.6E1,6E0,5E0,2E1,8E0,3E0,7E0,3E0,2E0,1.7E1,3E0,2E0,1.8E1,1E1,6E0,3.6E1,4E0,1E1,2.6E1,6E0,1.4E1,2E0,6E0,3E0,1.4E1,2E0,1.6E1,4E0,6E0,1.8E1,1.8E1,2E0,2E0,3E0,7E0,2.4E1,2E0,2E0,4E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[5.1350137E-3,-3.3741128E-2,2.475987E-1,-2.1715815E-1,-6.9968994E-3,3.2988298E-1,2.8091062E-2,-6.1420463E-2,-2.4671237E-1,-9.5820725E-2,3.2276653E-2,2.3344913E-3,3.5703343E-1,-5.512472E-2,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,-3.0382487E-1,-1.4058068E-1,-2.9446453E-2,-2.1209379E-1,5.59181E-2,4.091285E-1,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,-6.627251E-2,-2.2693902E-1,5.2225657E-2,-9.5052674E-2,1.0859418E-2,-3.6253917E-1,2.6486173E-2,1.5046827E-1,2.1380894E-2,7.393621E-3,2.4758913E-3,-8.528379E-2,-3.1659063E-3,-2.504589E-1,-1.3437112E-3,5.355821E-3,-7.7356305E-3,-4.3233983E-2,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,1.0883494E-2,1.4663106E-1,1.9825523E-1,8.6160064E-2,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,59,-1,-1,-1,-1,61,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9783993E0,8.8634837E-1,5.145322E-1,9.070897E-2,5.577526E-1,1.4360905E-1,6.3942134E-2,9.758426E-3,6.0073018E-2,1.4143506E-1,6.5313697E-1,0E0,1.07219934E-1,1.2388382E-2,0E0,0E0,0E0,0E0,3.4586906E-2,1.7105693E-1,1.1748636E-1,3.393597E-1,2.7985966E-1,4.202509E-2,0E0,0E0,0E0,0E0,0E0,4.2199805E-2,3.3204734E-2,4.9204696E-2,3.2862037E-2,3.151442E-2,7.372153E-2,1.4649613E-1,5.6672513E-2,0E0,0E0,0E0,3.002061E-2,0E0,3.334278E-2,0E0,0E0,0E0,2.2537217E-2,0E0,0E0,0E0,0E0,1.6933751E-1,1.3314232E-2,2.8548539E-2,3.267061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,18,18,19,19,20,20,21,21,22,22,23,23,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,42,42,46,46,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,60,-1,-1,-1,-1,62,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7162656E3,2.750917E-4,9.298568E7,2.56917E-2,2.802166E2,4.6111366E1,5.547E3,6.054E3,2.799504E11,1.8716404E10,3.791641E0,2.3344913E-3,4.5E1,4.8008E4,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,6.645232E2,8.1446184E7,4.57E2,8.640031E9,1.5085194E0,3.2707875E8,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,1.5E1,2.2057E4,2.3168E5,3.1E1,8.601996E-1,8.92851E6,7.6599895E9,2.5661135E0,2.1380894E-2,7.393621E-3,2.4758913E-3,3.07E2,-3.1659063E-3,2.0562818E4,-1.3437112E-3,5.355821E-3,-7.7356305E-3,2.7898628E5,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,6.7835927E0,1.743E3,2.6198992E-1,1.51E3,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"split_indices":[53,40,46,59,53,57,9,2,32,32,54,0,3,2,0,0,0,0,34,5,0,5,43,33,0,0,0,0,0,3,12,12,3,28,1,5,43,0,0,0,11,0,34,0,0,0,29,0,0,0,0,54,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.8E2,2.8E1,2.2E1,1.58E2,2E1,8E0,4E0,1.8E1,4.8E1,1.1E2,2E0,1.8E1,4E0,4E0,2E0,2E0,8E0,1E1,2.8E1,2E1,9E0,1.01E2,1.3E1,5E0,2E0,2E0,5E0,5E0,1.6E1,1.2E1,9E0,1.1E1,4E0,5E0,7.8E1,2.3E1,1.1E1,2E0,2E0,1.4E1,2E0,1E1,4E0,5E0,4E0,7E0,2E0,2E0,3E0,2E0,7E1,8E0,1.2E1,1.1E1,5E0,9E0,5E0,5E0,2E0,5E0,4E1,3E1,2E0,6E0,6E0,6E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-6.245155E-3,-2.833662E-2,3.4589776E-1,-7.005383E-2,5.9592E-2,4.2491645E-1,-1.5961174E-3,-4.3598708E-2,-2.8613925E-1,-8.835135E-2,1.3214482E-1,5.290952E-1,1.0459326E-2,-9.431229E-2,1.7313663E-2,-4.463223E-3,-3.3082795E-1,2.9486376E-3,-2.370824E-1,-2.7359384E-3,1.4807577E-1,3.203436E-2,8.785884E-3,-1.4088191E-1,-5.170084E-2,4.7399618E-2,-1.0460413E-1,-2.1237668E-2,-1.0130584E-2,6.996463E-3,-4.852486E-2,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,1.24671705E-1,-2.0670156E-1,-8.551871E-2,5.622916E-3,-6.533785E-2,6.141754E-2,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,-8.802993E-2,6.8448015E-2,1.8364535E-1,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,51,53,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5812309E0,7.0478874E-1,3.730235E-1,7.306783E-1,6.77165E-1,1.050607E-1,0E0,3.6305583E-1,9.1302514E-2,2.8586107E-1,1.3435823E-1,1.5126264E-1,0E0,1.1971778E-1,2.0303094E-1,0E0,5.1597834E-2,1.0845146E-1,1.1068231E-1,0E0,6.660807E-2,0E0,0E0,9.164274E-2,8.737236E-2,1.02572635E-1,6.1553992E-2,0E0,0E0,0E0,2.9702732E-2,0E0,0E0,0E0,1.00164235E-1,4.0271103E-2,6.0178608E-2,0E0,5.5570483E-2,7.76602E-2,0E0,0E0,0E0,0E0,1.725753E-2,3.47196E-2,2.8324187E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,30,30,34,34,35,35,36,36,38,38,39,39,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,52,54,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,1.0648859E3,1.2055723E8,4.934166E7,1E0,3.0291426E-1,-1.5961174E-3,3.3969492E2,8.7793425E-2,5E0,7.340097E7,1.04E3,1.0459326E-2,2.0562818E4,1.2670352E1,-4.463223E-3,3E0,8.78E2,9.875E3,-2.7359384E-3,3.835264E-2,3.203436E-2,8.785884E-3,1.4357987E10,4.914413E-3,1.7328871E6,1.018E3,-2.1237668E-2,-1.0130584E-2,6.996463E-3,2E0,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,2.6673E4,7.3353534E2,5.571429E0,5.622916E-3,3.606519E-1,2.5356756E1,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,9.281264E-2,6.7835927E0,7.1847E4,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"split_indices":[4,53,46,46,102,39,0,53,59,8,7,0,0,34,55,0,8,0,9,0,28,0,0,32,39,48,0,0,0,0,8,0,0,0,10,4,57,0,39,57,0,0,0,0,39,54,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.9E2,1.1E1,1.29E2,6.1E1,9E0,2E0,1.16E2,1.3E1,2E1,4.1E1,5E0,4E0,6.3E1,5.3E1,3E0,1E1,1.3E1,7E0,3E0,3.8E1,3E0,2E0,2.9E1,3.4E1,4.3E1,1E1,4E0,6E0,3E0,1E1,4E0,3E0,5E0,3.3E1,1.2E1,1.7E1,2E0,3.2E1,4E1,3E0,8E0,2E0,4E0,6E0,1.8E1,1.5E1,1E1,2E0,6E0,1.1E1,2.6E1,6E0,3.1E1,9E0,3E0,3E0,1.3E1,5E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.3435227E-3,-3.277699E-2,1.774612E-1,-6.964357E-3,-1.5355606E-1,1.9180963E-2,2.988968E-1,-6.0709182E-2,7.332652E-2,-6.922103E-2,-2.4452102E-1,1.5958549E-1,-3.233358E-2,3.533102E-1,1.318238E-1,-1.3005522E-1,-4.060321E-2,1.0830404E-1,-3.1816754E-2,-1.1131207E-2,-1.2607405E-1,-2.9532254E-1,2.7801134E-3,1.0396815E-3,1.1904416E-2,1.8165827E-2,-1.0879546E-2,3.7512168E-1,6.5092915E-3,2.031597E-3,9.1953445E-3,-1.5207392E-1,-1.9281486E-4,-4.925895E-3,-9.33342E-2,5.5923946E-2,1.5398896E-1,-1.0388468E-1,2.1812472E-2,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,-3.3411142E-1,-2.0121949E-2,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,4.4080522E-2,-6.593762E-2,-4.5851987E-2,-1.4824677E-1,-2.6830027E-3,8.220542E-2,1.7252632E-3,1.8331738E-1,9.3954586E-4,-7.641705E-3,4.75524E-3,-9.375982E-3,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,67,69,71,73,-1,75,-1,77,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4195844E0,5.663384E-1,7.4388015E-1,6.560496E-1,2.2856313E-1,1.353517E-1,1.5342748E-1,1.21597886E-1,2.2778258E-1,5.8043107E-2,2.467323E-1,5.353722E-2,1.3925183E-1,2.644825E-2,2.8130919E-2,5.492565E-2,1.3499385E-1,1.01465404E-1,6.410514E-2,3.194621E-2,4.1086197E-2,4.452479E-2,0E0,0E0,0E0,3.9501768E-2,0E0,4.0338874E-2,0E0,0E0,0E0,3.306678E-2,0E0,1.3446508E-1,6.8816334E-2,7.1661495E-2,7.693821E-2,4.8853584E-2,2.4587668E-2,0E0,0E0,0E0,0E0,0E0,4.43269E-2,1.6359001E-2,0E0,0E0,0E0,0E0,0E0,1.0982905E-1,8.9442335E-2,3.789381E-2,2.6375294E-2,0E0,2.339761E-2,0E0,3.7862718E-2,0E0,0E0,0E0,1.0178003E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,27,27,31,31,33,33,34,34,35,35,36,36,37,37,38,38,44,44,45,45,51,51,52,52,53,53,54,54,56,56,58,58,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,68,70,72,74,-1,76,-1,78,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.075104E7,1E0,5.470729E2,7.6599895E9,1E0,3.3326385E0,1.142E3,2.7883E4,1.0363E4,7.2418816E12,3.5912186E-1,8.129243E5,4.4E1,3.6E1,1.3448806E8,5.549974E8,7.8761675E5,1.6595541E1,1.1484E4,1.0731037E-1,8.67241E9,2.7801134E-3,1.0396815E-3,1.1904416E-2,2.613E3,-1.0879546E-2,2.608036E6,6.5092915E-3,2.031597E-3,9.1953445E-3,1.8E2,-1.9281486E-4,4.61E2,1.2845426E1,3.3997994E5,3.88E2,8.45E2,1.3050505E0,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,6.1826086E0,1.970492E6,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,7.269755E0,3.6050353E5,9.5157854E-2,1E0,-2.6830027E-3,7E1,1.7252632E-3,7.3586698E0,9.3954586E-4,-7.641705E-3,4.75524E-3,1.82E9,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"split_indices":[53,46,102,53,5,8,58,2,9,9,32,28,29,3,3,7,5,29,57,2,28,5,0,0,0,0,0,1,0,0,0,11,0,0,57,48,0,0,58,0,0,0,0,0,54,30,0,0,0,0,0,54,29,39,65,0,8,0,54,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.81E2,3.8E1,1.5E2,3.1E1,1.7E1,2.1E1,9E1,6E1,1.7E1,1.4E1,4E0,1.3E1,1.5E1,6E0,1.9E1,7.1E1,4.5E1,1.5E1,9E0,8E0,1.2E1,2E0,2E0,2E0,1.1E1,2E0,1.3E1,2E0,3E0,3E0,1.6E1,3E0,4.3E1,2.8E1,2.2E1,2.3E1,6E0,9E0,7E0,2E0,2E0,6E0,3E0,9E0,8E0,3E0,2E0,1.1E1,5E0,1.1E1,2.4E1,1.9E1,1.6E1,1.2E1,4E0,1.8E1,5E0,1.8E1,2E0,4E0,2E0,7E0,5E0,4E0,4E0,4E0,1.7E1,7E0,1.3E1,6E0,1.2E1,4E0,2E0,1E1,1.6E1,2E0,1.6E1,2E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-1.0673481E-2,-2.922265E-2,3.2056907E-1,-9.558372E-2,6.192814E-3,7.489175E-3,1.9760614E-2,-6.519865E-2,-2.9253313E-1,-5.28204E-2,5.102089E-2,-1.4650565E-1,-1.8217016E-2,-8.463386E-3,-2.1037353E-2,2.8328342E-2,-8.429096E-2,-5.649434E-2,8.870596E-2,-1.687275E-1,-7.512345E-4,8.831442E-2,-5.69984E-2,7.728642E-2,-2.82216E-3,-1.0359726E-1,6.4482796E-4,4.802586E-3,-9.4108865E-2,1.5494351E-2,1.22702755E-1,-2.0045006E-1,-3.7551587E-3,5.5987155E-4,6.6966885E-3,-2.3729175E-3,-8.077933E-3,6.7342306E-3,6.569003E-4,-1.23539165E-1,-4.9365353E-2,-6.873523E-3,-9.76779E-4,4.8460853E-3,-3.942172E-2,7.549796E-2,1.7108953E-1,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,-1,39,-1,-1,41,43,45,47,-1,-1,-1,49,-1,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.158347E0,4.18929E-1,7.998216E-2,3.5195512E-1,3.1221244E-1,0E0,0E0,2.0742762E-1,5.744326E-2,1.3263142E-1,2.764765E-1,5.5647433E-2,1.537655E-1,0E0,0E0,6.787142E-2,7.104939E-2,1.143521E-1,1.2254682E-1,3.3913255E-2,0E0,3.627623E-2,1.603502E-1,3.7099797E-2,0E0,2.7427554E-2,0E0,0E0,4.950477E-2,8.2423344E-2,6.474209E-2,3.1638563E-2,0E0,0E0,0E0,9.902678E-2,0E0,0E0,0E0,5.4692864E-2,1.5442751E-2,0E0,0E0,0E0,3.445577E-2,8.0237605E-2,6.781274E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,28,28,29,29,30,30,31,31,35,35,39,39,40,40,44,44,45,45,46,46],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,-1,40,-1,-1,42,44,46,48,-1,-1,-1,50,-1,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3060927E0,1.04201E-2,5.9165E4,1.810504E10,5.470729E2,7.489175E-3,1.9760614E-2,5.162122E2,5.752E3,1.0275E4,3.8834286E0,1E0,9.8298E4,-8.463386E-3,-2.1037353E-2,2.8945708E6,4E1,1.4139E4,3.996671E5,3.5523141E-3,-7.512345E-4,2.7123368E10,5.347626E5,7.751E3,-2.82216E-3,7.2899837E0,6.4482796E-4,4.802586E-3,1.1334876E0,7.6420575E6,2.6673E4,3.0353E4,-3.7551587E-3,5.5987155E-4,6.6966885E-3,8.591E3,-8.077933E-3,6.7342306E-3,6.569003E-4,5.8910837E0,5.69434E-1,-6.873523E-3,-9.76779E-4,4.8460853E-3,5.754E3,7.6221466E8,7.6E0,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"split_indices":[36,39,2,5,53,0,0,56,2,9,54,105,30,0,0,46,3,10,29,40,0,32,29,9,0,54,0,0,40,46,10,12,0,0,0,2,0,0,0,54,28,0,0,0,2,7,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.77E2,9E0,6.1E1,1.16E2,4E0,5E0,5.4E1,7E0,5E1,6.6E1,1.9E1,3.5E1,5E0,2E0,1.4E1,3.6E1,1.7E1,4.9E1,1.6E1,3E0,9E0,2.6E1,9E0,5E0,3E1,6E0,3E0,1.4E1,1.6E1,3.3E1,1.1E1,5E0,4E0,5E0,1.8E1,8E0,4E0,5E0,2.1E1,9E0,8E0,6E0,6E0,1E1,1.8E1,1.5E1,3E0,8E0,1.5E1,3E0,1E1,1.1E1,5E0,4E0,3E0,7E0,1.1E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[3.5595038E-4,-4.4311643E-2,2.0678306E-1,-1.9426705E-2,-1.8886262E-1,3.1928593E-1,5.56452E-2,-7.896872E-2,2.8734265E-2,-9.082003E-2,-3.8585064E-1,2.895736E-2,2.4027427E-1,1.074728E-1,-3.3747476E-2,-1.00550115E-1,1.675159E-2,7.017641E-2,-3.8212094E-2,-1.407528E-1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.6446205E-1,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,-1.5302381E-1,-7.644423E-2,7.8863114E-2,-4.7753017E-2,1.0884225E-1,-1.3557769E-2,-5.902727E-2,4.1850857E-3,-9.2872605E-3,-4.909929E-2,1.4215367E-2,5.618002E-3,-3.536953E-3,-2.0744017E-1,-4.741403E-2,-1.308574E-1,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.7041807E-2,1.5840885E-1,-8.000341E-2,2.9958343E-2,-1.2175934E-1,-2.2903696E-2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9178276E0,6.114423E-1,6.1136055E-1,4.238065E-1,4.477874E-1,3.14538E-1,8.238807E-2,1.3802564E-1,2.3018244E-1,2.9552594E-1,3.0219913E-2,0E0,4.197371E-2,6.660308E-2,4.066568E-2,5.66082E-2,5.5850185E-2,1.6659597E-1,8.780883E-2,6.407347E-2,0E0,0E0,0E0,0E0,2.8056264E-2,0E0,0E0,0E0,0E0,5.4878265E-2,5.5400893E-2,1.0032747E-2,1.9345354E-2,9.9761456E-2,5.1551525E-2,6.0662225E-2,0E0,0E0,1.02921E-2,0E0,0E0,0E0,4.0799797E-2,7.5311735E-2,1.8520892E-2,0E0,0E0,0E0,0E0,3.455124E-2,3.763908E-2,3.8690723E-2,1.8420529E-2,3.3416137E-2,2.2927333E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,6.6932364E7,5.9337012E7,3.3969492E2,1.4792529E10,5.0655737E0,1.2055723E8,2E0,2.2971932E7,2E1,1.8942408E7,2.895736E-2,2.819228E-1,2.1516852E5,9.4908775E1,1.142E3,1.06403265E1,9.607843E-1,1.4792529E10,5.9862564E1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.5868E4,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,5.244755E-3,7.44E2,9.950143E7,1.2067E4,5.6426154E2,3.634E3,2.2502415E1,4.1850857E-3,-9.2872605E-3,1.2281632E3,1.4215367E-2,5.618002E-3,-3.536953E-3,5.322348E7,1.4300815E6,3.1001872E5,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.58881E5,2.94E2,5.4452E4,6.568221E0,2.88446E5,9.38E2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"split_indices":[53,46,46,53,5,55,46,8,46,8,33,0,58,34,57,2,55,58,5,57,0,0,0,0,9,0,0,0,0,58,0,12,9,53,2,57,0,0,53,0,0,0,5,33,29,0,0,0,0,1,0,30,54,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.7E2,3.6E1,1.46E2,2.4E1,2E1,1.6E1,6.5E1,8.1E1,1.7E1,7E0,3E0,1.7E1,1E1,6E0,5.3E1,1.2E1,5E1,3.1E1,1.5E1,2E0,5E0,2E0,3E0,1.4E1,3E0,7E0,3E0,3E0,1.5E1,3.8E1,6E0,6E0,3.4E1,1.6E1,2.7E1,4E0,9E0,6E0,1.1E1,3E0,7E0,8E0,2.6E1,1.2E1,4E0,2E0,3E0,3E0,1.6E1,1.8E1,6E0,1E1,9E0,1.8E1,3E0,3E0,3E0,5E0,1.4E1,1.2E1,9E0,3E0,9E0,7E0,3E0,1.5E1,2E0,4E0,6E0,4E0,3E0,6E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[2.6763344E-4,-3.562319E-2,1.7865828E-1,-1.8097268E-2,-2.4454215E-1,4.47172E-1,1.16943076E-1,-3.3486996E-2,1.414897E-1,-3.481418E-1,3.6788161E-3,2.790042E-2,6.3239834E-3,-1.0595338E-2,1.9547185E-1,-6.566554E-2,2.4707085E-2,1.8834342E-1,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,5.0848205E-2,2.2159144E-1,4.051122E-3,-5.639541E-2,-1.0586935E-2,-4.5325225E-3,4.210238E-2,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,-6.8116255E-2,7.0792675E-2,7.365901E-2,-2.802525E-2,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,37,-1,-1,-1,-1,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.299687E0,6.143675E-1,5.111331E-1,3.9033002E-1,4.5747256E-1,1.6556728E-1,2.925187E-1,2.7127695E-1,7.63599E-2,5.6399226E-2,0E0,0E0,0E0,1.5918176E-1,3.4319043E-2,1.1866668E-1,1.1051539E-1,3.6700696E-2,0E0,0E0,0E0,0E0,5.9108846E-2,4.338783E-2,0E0,1.3749906E-1,0E0,0E0,1.0386192E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3594824E-1,6.001374E-2,6.339806E-2,7.879637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,22,22,23,23,25,25,28,28,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,38,-1,-1,-1,-1,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,1.8000048E10,1.4164306E-3,7.918757E7,3.582157E12,8.50999E9,1E0,2.0053221E6,1.2117E4,6.1826086E0,3.6788161E-3,2.790042E-2,6.3239834E-3,5.9084116E3,7.1528664E0,4.187504E7,5.6865673E0,8.81832E5,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,7.644773E-1,2.819228E-1,4.051122E-3,1E0,-1.0586935E-2,-4.5325225E-3,6.773399E0,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,1.2936847E4,6.7111494E8,2.7883E4,3.4288502E8,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"split_indices":[53,5,58,33,32,5,102,49,9,54,0,0,0,4,54,33,55,1,0,0,0,0,35,58,0,89,0,0,54,0,0,0,0,0,0,52,5,9,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.68E2,3.3E1,1.56E2,1.2E1,5E0,2.8E1,1.43E2,1.3E1,9E0,3E0,3E0,2E0,1.1E1,1.7E1,9.2E1,5.1E1,9E0,4E0,6E0,3E0,2E0,9E0,1.3E1,4E0,8.8E1,4E0,6E0,4.5E1,4E0,5E0,7E0,2E0,3E0,1E1,8.1E1,7E0,3.1E1,1.4E1,2.4E1,5.7E1,4E0,3E0,2.4E1,7E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[7.671295E-3,-4.4080827E-2,1.6683623E-1,-2.5097556E-2,-3.2062513E-1,2.2227801E-1,-1.2322681E-2,-1.1087152E-1,-5.909076E-3,-4.2326204E-3,-1.9872338E-2,1.3815795E-1,3.3456573E-1,-9.755224E-2,3.63379E-3,-1.355743E-1,1.2135234E-3,-9.3863435E-2,9.225092E-3,9.646266E-2,1.2028726E-2,3.6554006E-1,3.8201683E-3,-7.6256203E-3,-6.825402E-4,-9.017518E-2,-2.0942573E-1,-7.649883E-3,-5.741548E-2,5.0508834E-2,-2.6426286E-2,1.13049895E-1,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,-1.8938167E-2,9.144381E-2,1.4660759E-2,-8.3531834E-2,1.4068319E-3,1.457002E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7124388E0,8.161584E-1,5.0931644E-1,2.4293393E-1,1.6942883E-1,3.2215738E-1,1.0366744E-1,9.3960494E-2,1.6326205E-1,0E0,0E0,8.524135E-2,9.821105E-2,3.2380693E-2,0E0,5.822915E-2,0E0,3.252116E-2,1.558761E-1,4.3112665E-2,0E0,9.481287E-2,0E0,0E0,0E0,2.1951303E-2,2.0213217E-2,0E0,1.926352E-2,1.4102952E-1,1.3496363E-1,4.2491734E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.587293E-2,1.4206815E-1,5.812499E-2,9.924857E-2,0E0,3.1057805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,19,19,21,21,25,25,26,26,28,28,29,29,30,30,31,31,41,41,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.8000048E10,9.298568E7,1.4862166E1,5.709406E7,5.8038013E3,9.4908775E1,3.068397E6,2.0293878E2,-4.2326204E-3,-1.9872338E-2,1.197355E6,5E1,6.97021E3,3.63379E-3,8.3949E4,1.2135234E-3,8.587809E4,7.9762895E6,1.127938E6,1.2028726E-2,6.677E3,3.8201683E-3,-7.6256203E-3,-6.825402E-4,1.3868161E5,1.0595825E3,-7.649883E-3,1.96897E5,2.495E3,4.497684E1,4.078932E5,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,2.7309215E6,5.511358E1,5.8454223E0,1.6594656E3,1.4068319E-3,2.819228E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"split_indices":[53,5,46,59,46,53,57,12,53,0,0,29,3,4,0,12,0,29,46,29,0,9,0,0,0,29,34,0,1,2,59,29,0,0,0,0,0,0,0,0,0,46,59,54,4,0,58,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.56E2,5E1,1.47E2,9E0,3.8E1,1.2E1,2.6E1,1.21E2,3E0,6E0,2.3E1,1.5E1,6E0,6E0,2.2E1,4E0,1.7E1,1.04E2,1.8E1,5E0,1.3E1,2E0,3E0,3E0,1.5E1,7E0,5E0,1.2E1,4.8E1,5.6E1,1.6E1,2E0,2E0,1.1E1,2E0,1.3E1,5E0,2E0,1E1,2E0,1.8E1,3E1,3.3E1,2.3E1,5E0,1.1E1,6E0,1.2E1,1.4E1,1.6E1,7E0,2.6E1,1.3E1,1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.0088654E-2,-4.6401702E-2,1.01957306E-1,-3.6083024E-2,-2.5793928E-1,3.6505505E-2,1.9830973E-1,-8.227722E-2,2.2556398E-2,-2.2164036E-3,-1.9751323E-2,1.0576863E-1,-6.8776435E-3,1.892742E-2,1.5552393E-1,-6.712239E-2,-1.4856316E-2,-5.168168E-2,6.346703E-2,6.68196E-3,1.6625825E-3,2.28483E-2,-3.8955654E-3,1.8000433E-1,1.2089454E-3,-7.764623E-2,3.5982527E-2,-1.81113E-2,-1.3062242E-1,-3.4493096E-2,1.2884454E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,-1.4065999E-1,-6.0010847E-2,-2.1526885E-4,3.771369E-3,-5.376633E-2,7.684118E-4,-8.152548E-3,-1.2452523E-3,-7.3128425E-2,5.791923E-3,1.6153444E-1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,-1,27,29,-1,-1,31,-1,33,-1,35,37,39,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.306833E-1,3.268422E-1,3.0808258E-1,4.0275842E-1,2.0142797E-1,9.3877874E-2,1.0262287E-1,2.599991E-1,2.0373048E-1,0E0,0E0,2.2404373E-2,4.5329027E-2,0E0,5.311328E-2,8.85137E-2,0E0,6.123595E-2,2.7983302E-1,0E0,0E0,9.2295585E-3,0E0,3.8849056E-2,0E0,7.1570426E-2,1.3761323E-2,2.2606054E-2,2.2841543E-2,1.1545663E-1,9.22758E-2,0E0,0E0,0E0,0E0,5.3088874E-2,9.419921E-2,0E0,0E0,2.1270107E-2,0E0,0E0,0E0,8.4302165E-2,0E0,8.3529234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,25,25,26,26,27,27,28,28,29,29,30,30,35,35,36,36,39,39,43,43,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,-1,28,30,-1,-1,32,-1,34,-1,36,38,40,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.537424E0,1.4146589E8,1.2236866E3,6.978001E2,1.5214152E5,1.1244E4,4.9695907E0,8.260109E9,3.41673E5,-2.2164036E-3,-1.9751323E-2,5.6352734E1,8.412314E0,1.892742E-2,4.8420677E-1,3.4741312E8,-1.4856316E-2,8.062193E-1,6.6217735E-2,6.68196E-3,1.6625825E-3,4.9852173E1,-3.8955654E-3,1.08E2,1.2089454E-3,8.8916E4,1.09524E5,4.737429E8,4.6E1,1E0,1.5107028E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,9.135018E0,5.580003E8,-2.1526885E-4,3.771369E-3,9.042926E3,7.684118E-4,-8.152548E-3,-1.2452523E-3,9.2312753E-1,5.791923E-3,1.234543E1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"split_indices":[43,46,53,53,34,9,55,5,29,0,0,59,55,0,39,7,0,28,58,0,0,59,0,3,0,1,30,7,3,89,39,0,0,0,0,59,5,0,0,34,0,0,0,28,0,55,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.53E2,4.9E1,1.47E2,6E0,3E1,1.9E1,8.2E1,6.5E1,3E0,3E0,1.1E1,1.9E1,2E0,1.7E1,7.8E1,4E0,2.3E1,4.2E1,7E0,4E0,1.4E1,5E0,1.4E1,3E0,7.1E1,7E0,1.7E1,6E0,1.7E1,2.5E1,4E0,1E1,1.2E1,2E0,1.4E1,5.7E1,4E0,3E0,8E0,9E0,4E0,2E0,1.4E1,3E0,1.9E1,6E0,6E0,8E0,2.4E1,3.3E1,3E0,5E0,8E0,6E0,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.7253997E-2,-2.9623577E-2,9.3434505E-2,-2.2292453E-1,-1.1781458E-2,1.2750259E-1,-1.9930653E-1,-3.3444623E-3,-1.2307007E-2,1.7842796E-2,-6.838117E-2,8.674666E-2,3.2339618E-1,-8.01149E-2,-1.8067626E-2,-4.725675E-3,3.552434E-2,-1.2093892E-1,-3.2683358E-2,2.175943E-3,1.3742693E-1,2.7282907E-2,2.0839193E-1,-5.710558E-3,-1.8373676E-4,4.8699785E-2,-4.864452E-3,-1.5385593E-1,3.9161125E-4,1.3277833E-3,-5.8980435E-2,6.1291102E-2,-4.3464635E-2,4.9143904E-3,1.6467385E-1,1.2900099E-2,5.3549507E-3,1.10007E-1,1.44394925E-2,-1.135703E-2,-1.014236E-1,-6.185148E-3,-1.7182616E-3,1.0451842E-1,7.325834E-4,-7.334734E-2,9.2549727E-4,-2.225592E-3,2.6441324E-3,2.1629092E-1,7.352542E-2,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,35,-1,-1,37,-1,39,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,-1,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.848589E-1,4.077926E-1,7.5345045E-1,2.8820753E-2,1.8565977E-1,4.9887896E-1,1.3421485E-1,0E0,0E0,1.5181659E-1,6.734279E-2,2.3972067E-1,1.832602E-1,1.6607814E-2,0E0,0E0,1.1910778E-1,6.492545E-2,3.9455958E-2,6.2021095E-2,1.2556028E-1,0E0,2.1007508E-2,0E0,0E0,1.2233533E-1,0E0,2.6357263E-2,0E0,0E0,2.2924516E-2,1.8414091E-2,2.566748E-2,2.000419E-2,1.20536804E-1,0E0,0E0,4.2832226E-2,5.8858946E-2,0E0,2.2556297E-2,0E0,0E0,1.198972E-2,0E0,1.4668021E-2,0E0,0E0,0E0,3.9754987E-2,2.4631858E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,22,22,25,25,27,27,30,30,31,31,32,32,33,33,34,34,37,37,38,38,40,40,43,43,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,36,-1,-1,38,-1,40,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,-1,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,1.6770158E-4,1.2055723E8,1.9E1,7.183439E0,8.258924E3,3.312524E6,-3.3444623E-3,-1.2307007E-2,1.056E3,7.8206635E0,1E0,1.6232848E7,1.5574268E7,-1.8067626E-2,-4.725675E-3,4.759155E1,2.2235178E1,2.7E1,9.32784E5,1.1114967E-3,2.7282907E-2,1.5307794E2,-5.710558E-3,-1.8373676E-4,1.2029E4,-4.864452E-3,2.9239766E-2,3.9161125E-4,1.3277833E-3,1.1079511E1,8.78E2,3.0558723E-1,2.69336E3,8.637931E1,1.2900099E-2,5.3549507E-3,4E1,4.711463E5,-1.135703E-2,3.4996462E-1,-6.185148E-3,-1.7182616E-3,5.88E2,7.325834E-4,9.042926E3,9.2549727E-4,-2.225592E-3,2.6441324E-3,1.5213319E0,4.6379595E6,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"split_indices":[53,40,46,3,54,53,30,0,0,2,54,102,1,1,0,0,57,57,3,30,39,0,57,0,0,9,0,58,0,0,55,0,40,4,59,0,0,10,52,0,58,0,0,0,0,34,0,0,0,58,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.18E2,7.2E1,9E0,1.09E2,6.5E1,7E0,2E0,7E0,7.2E1,3.7E1,5.5E1,1E1,5E0,2E0,9E0,6.3E1,1.4E1,2.3E1,2.1E1,3.4E1,2E0,8E0,3E0,2E0,5.8E1,5E0,1.1E1,3E0,7E0,1.6E1,9E0,1.2E1,6E0,2.8E1,4E0,4E0,2E1,3.8E1,3E0,8E0,3E0,1.3E1,4E0,5E0,8E0,4E0,3E0,3E0,1.7E1,1.1E1,3E0,1.7E1,1.1E1,2.7E1,2E0,6E0,2E0,2E0,2E0,6E0,1.2E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.821953E-2,-5.8290947E-2,5.3941235E-2,-1.5808642E-2,-4.884159E-2,-6.909937E-2,1.1836419E-1,-1.4827322E-2,-8.093995E-2,-2.6398048E-2,-1.7481435E-2,4.550153E-2,1.5699527E-1,-6.594871E-2,3.347421E-2,-9.2051856E-2,-6.509846E-3,6.4362705E-2,-7.304391E-2,8.8149734E-2,-1.3496414E-2,1.2912967E-2,1.3428916E-1,-1.2285006E-1,-2.9534832E-2,9.6844934E-2,3.1477788E-3,-6.093492E-2,-1.3656648E-1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,-1.4490403E-1,5.8896925E-2,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.758437E-2,1.9397938E-1,-1.2398881E-3,-9.721017E-3,-4.254263E-2,2.629008E-3,6.676271E-3,9.927296E-5,-2.7347464E-2,5.9932888E-2,-1.4137745E-1,-3.3839725E-2,-1.0662791E-3,-1.5445867E-1,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,1.0476206E-1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,63,65,67,69,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.496537E-1,2.9141405E-1,5.4719037E-1,0E0,1.2872437E-1,2.878313E-1,1.1952078E-1,1.4789282E-1,4.9851447E-2,9.7840816E-2,0E0,4.396437E-2,3.570825E-2,5.645959E-2,5.92712E-2,6.509176E-2,3.340755E-2,1.0004889E-2,1.5075365E-1,1.9013055E-2,1.2354755E-2,0E0,7.2952986E-2,7.7276245E-2,2.294885E-2,3.7963897E-2,3.9735876E-2,6.714282E-2,3.9696425E-2,0E0,0E0,0E0,0E0,4.4010997E-2,2.6443997E-2,0E0,0E0,0E0,0E0,4.359097E-2,9.496659E-3,0E0,0E0,1.2809919E-2,0E0,0E0,0E0,2.1849878E-2,1.7759107E-2,1.1468753E-2,4.3800417E-2,0E0,9.729177E-3,0E0,0E0,0E0,0E0,1.81911E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,39,39,40,40,43,43,47,47,48,48,49,49,50,50,52,52,57,57],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,64,66,68,70,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,5.243E3,1E0,-1.5808642E-2,7.6420575E6,6.3639E4,4.0679638E6,2.802166E2,5.754E3,1.4364E4,-1.7481435E-2,1.9071735E6,7.573E3,1.822764E4,4.585366E-2,3E0,5.178571E0,7.406885E8,1.0807868E7,9.1797715E-1,3.508923E6,1.2912967E-2,1.56E4,1.17E2,3.65102E5,1.7027842E8,2.2583E4,1.8716404E10,7E1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,2.6654E4,1.00152E5,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.8114974E-1,4.2414474E0,-1.2398881E-3,-9.721017E-3,1.0437421E8,2.629008E-3,6.676271E-3,9.927296E-5,1.8465776E8,8.3154816E2,1.1272E4,2.7048333E10,-1.0662791E-3,7.64764E5,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,3.6E1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"split_indices":[53,9,102,0,46,10,33,53,2,10,0,33,9,34,58,8,55,7,1,28,33,0,2,10,1,33,10,32,10,0,0,0,0,2,2,0,0,0,0,28,58,0,0,7,0,0,0,5,4,30,32,0,9,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,1.21E2,6.7E1,3E0,1.18E2,2.3E1,4.4E1,5.8E1,6E1,2.1E1,2E0,1.6E1,2.8E1,2.8E1,3E1,5.2E1,8E0,7E0,1.4E1,9E0,7E0,3E0,2.5E1,1E1,1.8E1,9E0,2.1E1,3.2E1,2E1,4E0,4E0,5E0,2E0,9E0,5E0,6E0,3E0,4E0,3E0,1.4E1,1.1E1,5E0,5E0,1.6E1,2E0,6E0,3E0,1.4E1,7E0,7E0,2.5E1,3E0,1.7E1,6E0,3E0,3E0,2E0,1.1E1,3E0,8E0,3E0,8E0,8E0,6E0,8E0,3E0,4E0,3E0,4E0,3E0,2.2E1,1.5E1,2E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.81344E-3,-2.7871855E-2,1.0370427E-1,-1.6869712E-1,-1.2145221E-2,1.3553725E-1,-1.1319494E-2,-6.7867875E-2,-2.5078225E-1,1.1689531E-2,-7.812123E-2,2.2571503E-1,4.7493696E-2,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,-2.2323536E-2,5.4143388E-2,-9.7720034E-2,1.1783024E-4,1.8657329E-2,1.5811518E-1,9.230261E-2,-1.6328286E-3,-4.379925E-2,5.0750427E-2,1.1329648E-1,1.5480721E-2,-1.1520765E-1,-1.8632218E-3,8.1961945E-2,9.722441E-3,1.8049882E-3,8.147564E-3,-2.9605734E-3,1.3710238E-3,-1.09546825E-1,-2.015405E-2,-1.4587996E-3,6.54756E-2,1.231898E-1,7.869888E-4,7.951029E-2,-2.6912533E-2,-1.2801106E-1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,37,39,41,43,45,-1,47,-1,49,-1,-1,-1,51,53,-1,55,57,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8306116E-1,3.744514E-1,4.811502E-1,1.2368178E-1,2.4308157E-1,2.989232E-1,0E0,2.157684E-2,3.9408505E-2,1.6582306E-1,6.4956516E-2,1.4187676E-1,4.637814E-2,0E0,0E0,0E0,0E0,1.0265903E-1,1.14883915E-1,2.9888332E-2,0E0,0E0,3.1216264E-2,7.44717E-2,2.112299E-2,7.5522915E-2,1.9793212E-2,1.7225742E-2,8.900131E-2,4.644516E-2,0E0,3.160474E-2,0E0,1.4609658E-2,0E0,0E0,0E0,6.457342E-2,9.696657E-2,0E0,1.6802516E-2,1.4416695E-2,0E0,1.5758365E-2,1.763527E-2,4.027137E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,37,37,38,38,40,40,41,41,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,38,40,42,44,46,-1,48,-1,50,-1,-1,-1,52,54,-1,56,58,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.178571E0,1.3578947E1,8.062193E-1,7.3808947E0,5.861021E7,-1.1319494E-2,3.662451E6,2.692487E6,5.716463E2,2.770862E8,6.661E3,2.1516852E5,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,3.9E1,1.26881E5,1.6259928E1,1.1783024E-4,1.8657329E-2,6.917027E3,4.078932E5,6.47008E1,1.4300815E6,2.1229417E-4,2.4025E4,3.6E1,4.2845196E1,-1.8632218E-3,5.59076E3,9.722441E-3,6.804E3,8.147564E-3,-2.9605734E-3,1.3710238E-3,2.07533E5,1.2670352E1,-1.4587996E-3,1.43E2,2.458E3,7.869888E-4,1E0,4.983287E-2,5.3E1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"split_indices":[53,55,55,28,54,46,0,1,1,53,7,9,34,0,0,0,0,3,30,55,0,0,4,29,57,33,39,9,3,57,0,4,0,9,0,0,0,1,55,0,10,2,0,84,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.69E2,4.1E1,1.6E1,1.53E2,3.8E1,3E0,8E0,8E0,1.13E2,4E1,1.8E1,2E1,6E0,2E0,4E0,4E0,6.3E1,5E1,3.2E1,8E0,4E0,1.4E1,1E1,1E1,4.9E1,1.4E1,1.9E1,3.1E1,2.4E1,8E0,6E0,8E0,5E0,5E0,3E0,7E0,1.2E1,3.7E1,2E0,1.2E1,1.7E1,2E0,1.2E1,1.9E1,2.2E1,2E0,4E0,2E0,2E0,3E0,7E0,5E0,3.5E1,2E0,4E0,8E0,4E0,1.3E1,1E1,2E0,3E0,1.6E1,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.1425839E-2,-5.037939E-2,8.030544E-2,-3.6732145E-2,-1.940263E-1,2.1132613E-2,5.40595E-2,-8.394264E-2,-1.2433366E-2,-2.411859E-3,-2.3667553E-1,-7.583577E-2,1.2632278E-1,-9.6114695E-2,3.007908E-3,3.00965E-2,-5.099469E-2,-1.4926101E-2,-5.8065807E-3,-2.905628E-2,-1.687885E-2,9.731201E-4,1.4231151E-1,-1.0521082E-1,6.9725106E-4,-3.51654E-2,6.765553E-2,-1.512059E-2,-1.1731359E-1,-7.190969E-2,1.2920251E-3,7.135396E-2,1.5594196E-1,-5.4826945E-2,-1.3231695E-1,1.7662038E-3,-8.1669055E-2,-1.3722117E-3,8.081185E-2,-6.1539885E-2,5.3886116E-2,-1.3972227E-1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,1.7186429E-1,7.422348E-2,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,33,-1,35,37,39,41,43,-1,45,47,49,51,-1,53,-1,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1869767E-1,2.692873E-1,5.338764E-1,1.4808676E-1,6.220767E-2,0E0,5.446193E-1,8.3999515E-2,1.4421546E-1,0E0,3.7372112E-2,2.5613624E-1,6.0055733E-2,4.269564E-2,0E0,1.05328426E-1,1.0766618E-1,0E0,0E0,4.71834E-2,0E0,0E0,1.9822717E-2,4.4642508E-2,0E0,5.613244E-2,3.6327116E-2,1.0261665E-1,3.450933E-2,6.169112E-2,0E0,1.8945247E-2,2.107352E-2,2.1910824E-2,4.9286067E-2,0E0,3.790599E-2,0E0,1.6026601E-2,3.391008E-2,3.6774833E-2,1.0349661E-2,0E0,0E0,0E0,0E0,0E0,1.815337E-2,3.8996745E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,19,19,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,38,38,39,39,40,40,41,41,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,34,-1,36,38,40,42,44,-1,46,48,50,52,-1,54,-1,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3453049E3,8.67241E9,1.4164306E-3,4.9054803E2,4.944487E7,2.1132613E-2,1E0,1.4864864E0,9.785595E6,-2.411859E-3,6.096518E0,4.4816768E8,1.6463401E3,9.741151E-1,3.007908E-3,2.4242616E5,8.78E2,-1.4926101E-2,-5.8065807E-3,3.0687964E3,-1.687885E-2,9.731201E-4,2.8626094E5,5.2646E4,6.9725106E-4,1.3585858E1,1.29E2,3.307175E-2,3.170081E6,7.6599895E9,1.2920251E-3,2.6295085E3,3.8014093E2,6.492337E0,2.0289855E-2,1.7662038E-3,1.1565725E0,-1.3722117E-3,1.2075471E7,1.2067E4,1.2117E4,4E1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,3.2014463E0,1.0807868E7,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"split_indices":[53,5,58,56,46,0,102,58,46,0,54,45,53,28,0,29,0,0,0,53,0,0,29,1,0,57,0,58,1,5,0,4,59,54,58,0,43,0,49,9,9,3,0,0,0,0,0,36,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.4E2,5.9E1,1.29E2,1.1E1,3E0,5.6E1,4.3E1,8.6E1,3E0,8E0,2E1,3.6E1,4E1,3E0,4.1E1,4.5E1,4E0,4E0,1.8E1,2E0,5E0,3.1E1,3.7E1,3E0,1.5E1,2.6E1,3E1,1.5E1,1E1,8E0,6E0,2.5E1,1.4E1,2.3E1,6E0,9E0,3E0,2.3E1,1.8E1,1.2E1,1.2E1,3E0,6E0,4E0,2E0,4E0,2E1,5E0,5E0,9E0,1.2E1,1.1E1,6E0,3E0,2E1,3E0,1.3E1,5E0,6E0,6E0,1E1,2E0,1.7E1,3E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[4.6584974E-3,-1.20946085E-2,1.9950889E-1,-1.0169338E-1,3.8820638E-3,3.8749582E-1,8.888968E-2,-5.8970988E-2,-1.9482663E-1,-9.452532E-2,1.551003E-2,2.2913612E-2,7.1646026E-3,1.3861427E-1,-4.1111445E-4,-8.537462E-2,2.566781E-2,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,-1.250599E-1,-3.5212412E-2,4.8972625E-2,2.1056903E-3,7.941639E-3,-4.738872E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,-8.0461E-2,-1.16186915E-2,8.0707185E-2,-5.17336E-2,5.499706E-2,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,-4.0313277E-2,4.2702768E-2,1.3941039E-1,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009837E-1,2.8382987E-1,3.1906998E-1,1.07877135E-1,1.9484903E-1,5.082476E-2,5.925162E-2,5.135636E-2,2.9909045E-2,1.1930962E-1,2.5971475E-1,0E0,0E0,1.16812885E-2,0E0,1.3007693E-2,1.56768E-2,0E0,0E0,0E0,6.1911702E-2,1.2099819E-1,1.173628E-1,0E0,0E0,1.1479644E-2,0E0,0E0,0E0,1.0476619E-2,0E0,6.582728E-2,7.438664E-2,8.7449014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.335335E-2,8.5745096E-2,9.818435E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,20,20,21,21,22,22,25,25,29,29,31,31,32,32,33,33,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1.4862166E1,2E0,1E0,4.060294E0,9.886E3,6.627576E7,1.3448806E8,8.3949E4,7.56E2,4.2658337E2,2.2913612E-2,7.1646026E-3,1.625E3,-4.1111445E-4,8.351808E-1,9.282E3,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,5.936149E1,2.124077E1,6.7948157E-1,2.1056903E-3,7.941639E-3,2.2970924E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,1E0,-1.16186915E-2,1.2623029E7,1.38E2,1.6974416E6,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,5.138372E8,3.810132E-2,1.2951E4,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"split_indices":[53,59,8,84,54,9,46,7,12,0,53,0,0,0,0,28,9,0,0,0,57,59,35,0,0,28,0,0,0,65,0,12,0,29,0,0,0,0,0,0,0,0,7,28,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.97E2,1.6E1,2.9E1,1.68E2,5E0,1.1E1,2.1E1,8E0,1.7E1,1.51E2,3E0,2E0,7E0,4E0,1.6E1,5E0,2E0,6E0,2E0,1.5E1,6E1,9.1E1,2E0,5E0,8E0,8E0,3E0,2E0,1.2E1,3E0,7E0,5.3E1,8.9E1,2E0,4E0,4E0,2E0,1E1,5E0,2E0,3E0,5E1,7.9E1,1E1,4.8E1,2E0,4E0,7.5E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-9.7492535E-4,2.4562754E-2,-1.2644652E-1,1.0812377E-2,3.050939E-1,-1.6741821E-1,3.276021E-2,-4.319679E-2,5.309041E-2,1.990497E-2,7.832562E-3,-1.8774702E-1,3.1424365E-3,3.0188295E-3,-1.7861315E-3,-2.2519317E-2,-9.010733E-2,8.511394E-2,-1.3702277E-2,-1.5522692E-1,-1.8752424E-2,-3.5709422E-2,8.45564E-2,-1.2079619E-1,5.7729695E-4,3.1312965E-2,1.5568885E-1,2.2016572E-2,-9.091352E-2,-1.9204946E-1,-1.8670933E-3,-5.38557E-2,2.1705188E-2,5.703837E-3,6.2500284E-4,-1.3545471E-1,-3.5225743E-4,8.462742E-2,-2.3649544E-2,1.15246095E-1,1.1550232E-2,-3.8249083E-2,4.199171E-2,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,27,29,-1,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.63182E-1,6.601755E-1,2.3378861E-1,3.7905455E-1,5.4006636E-2,1.4417273E-1,1.8542241E-2,6.878023E-2,2.0008749E-1,0E0,0E0,1.0724431E-1,0E0,0E0,0E0,7.6461144E-2,7.027824E-2,2.343359E-1,8.749639E-2,9.590763E-2,0E0,5.013643E-2,1.2541946E-2,2.7317584E-2,0E0,1.1042236E-1,6.1801553E-2,2.8090071E-2,2.3553073E-2,3.9512277E-2,0E0,1.5470095E-2,4.2363483E-2,0E0,0E0,2.9321939E-2,0E0,1.5620723E-2,5.383783E-2,5.4485172E-2,0E0,1.0460219E-2,2.2912873E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,37,37,38,38,39,39,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,28,30,-1,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,8.861832E3,1.4539318E-1,4.2658337E2,1.04E3,1E0,1.3507566E8,1.3839568E1,2.4934822E7,1.990497E-2,7.832562E-3,3.074E3,3.1424365E-3,3.0188295E-3,-1.7861315E-3,1.91126E5,7.407092E0,5.370622E5,1.06403265E1,8.66076E1,-1.8752424E-2,3.8E1,4.61E2,5.979E3,5.7729695E-4,9.6494156E-1,1.8907035E3,4.602015E0,1.27718E6,3.27E2,-1.8670933E-3,2.4397528E8,4.3E1,5.703837E-3,6.2500284E-4,1.7218965E2,-3.5225743E-4,8.047134E4,4.649E3,7.4245725E0,1.1550232E-2,8.062193E-1,3.706834E-4,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"split_indices":[46,4,39,53,0,89,46,57,46,0,0,0,0,0,0,30,54,29,55,57,0,3,0,2,0,58,4,54,29,0,0,32,10,0,0,53,0,34,2,55,0,28,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.71E2,3.4E1,1.64E2,7E0,2.7E1,7E0,7.2E1,9.2E1,3E0,4E0,2.5E1,2E0,5E0,2E0,5.1E1,2.1E1,6.2E1,3E1,2.3E1,2E0,4.6E1,5E0,1.6E1,5E0,3.6E1,2.6E1,2.1E1,9E0,1.7E1,6E0,3.5E1,1.1E1,3E0,2E0,1.4E1,2E0,1.8E1,1.8E1,1.9E1,7E0,5E0,1.6E1,7E0,2E0,4E0,1.3E1,2E0,3.3E1,4E0,7E0,4E0,1E1,8E0,1E1,7E0,1.1E1,8E0,1.1E1,2E0,3E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[6.475444E-3,-1.306737E-2,1.7832997E-1,2.339655E-3,-1.5614031E-1,2.8744474E-1,5.3003386E-2,-1.6489496E-2,6.7189105E-2,-2.4180317E-1,1.30160665E-2,3.55285E-1,3.888601E-3,9.1399415E-4,6.7161336E-3,-1.0044899E-2,-1.06382035E-2,1.3008821E-2,1.2666671E-1,-2.8520468E-1,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,-1.6523695E-2,1.5263823E-1,-4.6523552E-2,7.135857E-2,1.455868E-1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,-2.6452178E-2,7.834649E-2,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,-1,33,35,37,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9082713E-1,4.0816608E-1,2.7193904E-1,2.06246E-1,2.639749E-1,1.2648803E-1,2.9406311E-2,1.6842958E-1,1.2080774E-1,5.8472157E-2,2.9649243E-2,6.8154216E-2,0E0,0E0,0E0,1.3743296E-1,0E0,7.626754E-2,3.144613E-2,2.2196114E-2,0E0,0E0,0E0,0E0,0E0,1.1887014E-1,4.850761E-2,1.9805137E-2,2.4082582E-2,1.8560827E-2,0E0,0E0,0E0,7.060035E-2,5.3812273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,-1,34,36,38,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.993295E3,1.8000048E10,5.9337012E7,1.5085194E0,1.5461028E12,6.6737964E2,6.543368E6,1.3201905E0,7.7567725E2,4.577256E3,2.6023E4,1.04E3,3.888601E-3,9.1399415E-4,6.7161336E-3,2.3329E4,-1.06382035E-2,2.2906428E5,6.4487465E6,5.709406E7,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,1E0,9.39E2,3E1,5.1E2,3.5E1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,4.56567E7,3.2834557E-1,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"split_indices":[4,5,46,43,32,57,30,40,53,4,10,0,0,0,0,2,0,29,48,46,0,0,0,0,0,89,0,3,0,3,0,0,0,46,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.84E2,2E1,1.67E2,1.7E1,1E1,1E1,1.3E2,3.7E1,1.1E1,6E0,7E0,3E0,8E0,2E0,1.27E2,3E0,2E1,1.7E1,8E0,3E0,4E0,2E0,4E0,3E0,1.23E2,4E0,1E1,1E1,1.4E1,3E0,2E0,6E0,1.12E2,1.1E1,2E0,2E0,4E0,6E0,4E0,6E0,5E0,9E0,9.5E1,1.7E1,2E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0223907E-3,-2.7888758E-2,9.6585326E-2,-1.5584077E-2,-2.1336152E-1,1.575512E-1,-3.900375E-2,-3.563399E-2,2.183398E-2,-3.6633124E-3,-1.6371118E-2,1.8702927E-1,-5.3675026E-3,2.0629173E-2,-1.4183015E-1,-3.0176764E-2,-7.741039E-3,5.2294556E-2,-3.702995E-2,5.3999938E-2,2.235795E-1,4.8674736E-2,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,-4.079073E-2,3.0565633E-2,6.8358756E-3,7.473907E-2,2.9559305E-3,-5.9157714E-2,-3.6169306E-4,3.6848264E-3,1.9294271E-2,1.8000335E-1,4.2783604E-3,7.0099917E-4,-3.1242667E-2,-1.5811709E-1,-3.2913506E-2,1.07815154E-1,2.1910788E-3,-7.367315E-4,9.088779E-2,5.4238463E-4,-5.502338E-3,-4.669484E-3,8.802838E-2,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8288884E-1,3.6638862E-1,4.1119415E-1,1.1556303E-1,1.445714E-1,2.890743E-1,1.0103525E-1,6.2558055E-2,9.8760046E-2,0E0,0E0,1.3865662E-1,0E0,3.1503566E-2,8.29455E-2,6.378441E-2,0E0,3.621982E-2,4.469773E-2,1.12352E-2,1.0943866E-1,1.083583E-2,0E0,0E0,0E0,8.803986E-2,7.710456E-2,1.1590129E-2,2.3634195E-2,0E0,4.6466663E-2,0E0,0E0,0E0,4.115665E-2,0E0,0E0,7.36302E-2,2.4497524E-2,2.882525E-2,2.4574421E-2,0E0,0E0,1.8441036E-2,0E0,0E0,1.4278462E-2,2.0401236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,25,25,26,26,27,27,28,28,30,30,34,34,37,37,38,38,39,39,40,40,43,43,46,46,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.775894E10,9.040637E7,2.0053221E6,6.0671224E7,1.3578947E1,5.3868E4,5.138372E8,3.8279E5,-3.6633124E-3,-1.6371118E-2,1.2737E4,-5.3675026E-3,9.123462E-1,2.6022625E5,2.0997734E8,-7.741039E-3,5.716463E2,8.75E2,5.22E2,1.6064256E-2,3.9E1,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,5.1383884E7,4.016202E1,7.25E2,1.5562991E6,2.9559305E-3,2.7102E4,-3.6169306E-4,3.6848264E-3,1.9294271E-2,3.383436E5,4.2783604E-3,7.0099917E-4,2.47565E6,5.88E2,3.887674E6,1.631E4,2.1910788E-3,-7.367315E-4,7.3586698E0,5.4238463E-4,-5.502338E-3,1.617E3,9.851E3,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"split_indices":[4,5,46,49,46,55,10,7,30,0,0,2,0,28,34,7,0,53,0,0,58,3,0,0,0,33,59,0,48,0,10,0,0,0,29,0,0,29,0,46,9,0,0,54,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.61E2,4.8E1,1.52E2,9E0,3.3E1,1.5E1,9.9E1,5.3E1,5E0,4E0,3E1,3E0,1E1,5E0,9.6E1,3E0,3.5E1,1.8E1,7E0,2.3E1,8E0,2E0,3E0,2E0,8.2E1,1.4E1,1.2E1,2.3E1,3E0,1.5E1,2E0,5E0,3E0,2E1,3E0,5E0,7.7E1,5E0,8E0,6E0,4E0,8E0,1.8E1,5E0,7E0,8E0,6E0,1.4E1,7.3E1,4E0,2E0,3E0,4E0,4E0,4E0,2E0,1.5E1,3E0,2E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[8.825459E-3,-2.0716727E-2,9.291196E-2,-9.286735E-3,-2.2815323E-1,4.1376296E-2,1.8566836E-1,-4.1899815E-2,3.8700562E-2,-1.1017629E-3,-1.425366E-2,-6.1172745E-3,7.967702E-2,2.2176522E-1,7.0497654E-2,-1.486095E-1,-2.7861884E-2,-2.4466898E-2,7.862809E-2,2.2698525E-2,-3.884307E-3,1.145321E-1,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,-1.2128689E-2,-1.3520806E-1,1.7996103E-2,-7.628645E-2,5.7908777E-2,9.716253E-3,-1.2009995E-3,3.937936E-2,2.097745E-3,6.499496E-3,-2.237607E-2,1.0090606E-1,-8.440877E-3,5.7535426E-4,5.9123063E-3,-9.376124E-3,-4.9414644E-3,-3.266944E-4,8.224998E-3,4.116531E-2,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,-1,49,-1,-1,51,53,-1,-1,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1613337E-1,3.6301482E-1,2.5045186E-1,2.3167856E-1,9.979531E-2,6.5972775E-2,6.335485E-2,1.2768199E-1,1.5325102E-1,0E0,0E0,3.7913833E-2,3.5779223E-2,8.4248126E-2,1.1696836E-2,5.5736482E-2,1.3178307E-1,5.418279E-2,8.121562E-2,1.124549E-2,0E0,1.054576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.3339594E-2,5.8268234E-2,4.0797673E-2,1.8643163E-2,5.442866E-2,0E0,0E0,9.034543E-3,0E0,0E0,8.4523104E-2,1.790364E-2,0E0,0E0,0E0,1.3487471E-2,0E0,0E0,0E0,5.1540002E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,29,29,30,30,31,31,32,32,33,33,36,36,39,39,40,40,44,44,48,48],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,-1,50,-1,-1,52,54,-1,-1,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5585917E0,1.4233672E8,1.6059814E3,1.2534044E3,8.667E3,5.433479E2,2.7304773E0,8.35E2,7.157224E0,-1.1017629E-3,-1.425366E-2,1.1350955E0,1.2224206E7,8E0,2.8352614E9,4.1749332E4,1.448907E6,7.794012E7,5.2086E4,2.9539914E4,-3.884307E-3,7.810231E4,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,1E0,2.18E3,5.491E3,1.47245E7,6.489288E6,9.716253E-3,-1.2009995E-3,8.842206E7,2.097745E-3,6.499496E-3,1.5989196E7,2.94E2,-8.440877E-3,5.7535426E-4,5.9123063E-3,1.424506E6,-4.9414644E-3,-3.266944E-4,8.224998E-3,9.1487586E-1,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"split_indices":[42,46,53,4,2,53,43,2,55,0,0,58,51,8,5,33,1,46,2,34,0,34,0,0,0,0,0,0,0,89,0,9,1,46,0,0,5,0,0,46,0,0,0,0,30,0,0,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.53E2,5.3E1,1.46E2,7E0,3.5E1,1.8E1,8.7E1,5.9E1,2E0,5E0,1.6E1,1.9E1,1.3E1,5E0,9E0,7.8E1,2.3E1,3.6E1,1.2E1,4E0,1.1E1,8E0,9E0,4E0,2E0,3E0,6E0,3E0,6.9E1,9E0,1.3E1,1E1,3.2E1,4E0,3E0,9E0,3E0,8E0,6.4E1,5E0,7E0,2E0,2E0,1.1E1,7E0,3E0,3E0,2.9E1,4E0,5E0,4.5E1,1.9E1,2E0,3E0,9E0,2E0,2.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.4462977E-3,-2.893987E-2,6.9793515E-2,-3.6714904E-2,1.3322261E-2,2.450053E-2,1.3456057E-1,-3.0940516E-2,-1.5242134E-2,6.13726E-2,-2.383746E-2,1.9235724E-1,5.9276864E-2,-1.44404005E-2,-1.02318145E-1,6.015054E-3,1.4756152E-3,-5.1880803E-2,1.3078791E-3,2.2946529E-1,2.957695E-3,4.6630064E-3,-1.3928295E-3,-7.5536236E-2,1.9160501E-3,-1.6728845E-1,-1.4484057E-2,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,-4.1924756E-2,-1.2142303E-2,9.4770074E-2,-1.28825195E-2,-1.1275523E-1,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,27,-1,29,-1,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9038587E-1,3.7482685E-1,1.5511182E-1,2.329789E-1,0E0,6.027959E-2,8.43229E-2,1.7518334E-1,0E0,3.29639E-2,2.2726193E-2,4.4647485E-2,3.807029E-2,1.2305999E-1,1.5736201E-1,0E0,0E0,1.2245912E-2,0E0,1.3593733E-2,0E0,0E0,0E0,8.423267E-2,1.2907745E-1,5.515635E-2,1.3991938E-2,0E0,0E0,0E0,0E0,0E0,5.6269135E-2,7.5751625E-2,1.7076403E-2,0E0,2.499877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,23,23,24,24,25,25,26,26,32,32,33,33,34,34,36,36],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,28,-1,30,-1,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,1.3453049E3,2.8159826E0,1.3322261E-2,2.4608E4,8.812601E-1,7.361366E7,-1.5242134E-2,7.614E3,2.449092E5,2.5501368E9,8.2731545E-1,6.974669E7,8.66076E1,6.015054E-3,1.4756152E-3,3.239E3,1.3078791E-3,3.6E1,2.957695E-3,4.6630064E-3,-1.3928295E-3,8.35E2,5.506676E11,1E0,2.313233E6,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,3.54E2,8E0,3.3478114E-1,-1.28825195E-2,3.890129E5,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"split_indices":[43,57,53,36,0,9,58,46,0,10,29,7,28,7,57,0,0,2,0,3,0,0,0,2,32,65,1,0,0,0,0,0,0,8,28,0,29,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.54E2,5.3E1,1.51E2,3E0,3.2E1,2.1E1,1.49E2,2E0,1.8E1,1.4E1,1.1E1,1E1,1.22E2,2.7E1,5E0,1.3E1,9E0,5E0,8E0,3E0,7E0,3E0,2.5E1,9.7E1,1.5E1,1.2E1,5E0,4E0,6E0,2E0,5E0,2E1,8.5E1,1.2E1,4E0,1.1E1,3E0,9E0,1E1,1E1,6.1E1,2.4E1,3E0,9E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.4985325E-2,-3.9879307E-2,5.6348145E-2,-2.3429228E-2,-1.0598515E-1,-1.7923791E-2,9.930136E-2,-5.1214058E-2,3.0147022E-2,-3.333782E-2,-1.8125312E-1,3.6635974E-3,-3.763992E-2,1.5140602E-1,5.2652568E-2,-6.5377824E-2,1.8297803E-2,1.5232481E-2,8.540196E-3,-7.670071E-2,8.6073264E-2,-2.6167667E-1,-8.275727E-4,-9.86009E-3,-4.455716E-3,8.21394E-3,2.7782663E-3,8.663035E-2,1.0011745E-2,-1.3291118E-1,-5.3235017E-2,-3.8194293E-3,3.9224032E-2,7.2662863E-3,-3.2975117E-3,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,-1.8245183E-1,-3.2175884E-2,-1.0584831E-1,4.2874664E-3,-1.5806843E-3,9.24955E-3,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,-1,-1,-1,43,45,47,49,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.769355E-1,1.6824591E-1,1.7724186E-1,1.9059703E-1,1.6346598E-1,4.1423395E-2,7.709977E-2,8.389847E-2,9.2703044E-2,9.462805E-2,1.9369146E-1,0E0,2.6057394E-2,1.6584098E-2,2.810949E-2,5.03276E-2,3.3456847E-2,1.0235344E-1,0E0,3.1034954E-2,2.0551417E-2,4.1319788E-2,0E0,1.02055315E-2,0E0,0E0,0E0,1.1555165E-2,9.544139E-3,5.704157E-2,6.4181656E-2,0E0,4.842038E-2,0E0,6.882077E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.789392E-3,5.7711136E-2,5.315976E-2,0E0,0E0,3.5858765E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,27,27,28,28,29,29,30,30,32,32,34,34,48,48,49,49,50,50,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,-1,-1,-1,44,46,48,50,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,7.294459E7,3.239E3,1.2943141E3,1.3907746E10,7.751E3,6.4512783E-1,4.3E1,4.1983E4,1.753919E3,3.7284137E2,3.6635974E-3,8.724138E0,2.2151E4,2E0,8.35E2,3.99E2,2.73782E5,8.540196E-3,2.7864855E11,1.7837838E0,9.301866E7,-8.275727E-4,5.0900903E0,-4.455716E-3,8.21394E-3,2.7782663E-3,3.1E1,2.2621576E-1,4.569361E7,7E0,-3.8194293E-3,1.2623029E7,7.2662863E-3,1.2001309E0,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,4.1749332E4,1E0,3.677609E-1,4.2874664E-3,-1.5806843E-3,1E1,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"split_indices":[43,46,2,56,5,9,58,3,2,53,59,0,55,9,8,2,2,30,0,32,58,7,0,57,0,0,0,3,39,5,8,0,12,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,33,74,28,0,0,3,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.56E2,5.4E1,1.26E2,3E1,2E1,3.4E1,8.3E1,4.3E1,1.6E1,1.4E1,3E0,1.7E1,1.5E1,1.9E1,6.9E1,1.4E1,4E1,3E0,1.2E1,4E0,9E0,5E0,1.2E1,5E0,1.2E1,3E0,1E1,9E0,9E0,6E1,2E0,1.2E1,4E0,3.6E1,6E0,6E0,2E0,2E0,2E0,7E0,5E0,7E0,5E0,5E0,2E0,7E0,3E0,6E0,4.4E1,1.6E1,7E0,5E0,3.4E1,2E0,3E0,3E0,3.9E1,5E0,2E0,1.4E1,2E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.3111593E-4,8.332628E-3,-1.9431697E-1,-3.4451315E-3,1.5696207E-1,-1.8588053E-3,-1.294587E-2,-3.1280745E-2,2.9869221E-2,2.6512837E-1,5.53925E-2,-1.5708588E-1,-2.058258E-2,1.0180442E-2,9.531762E-2,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,2.4204077E-3,-7.460408E-2,-4.4569806E-3,1.9710528E-2,1.22437425E-1,-5.1242317E-4,-1.3157908E-2,7.7513896E-2,-1.2132418E-1,-5.7141267E-2,9.711912E-2,6.6852947E-3,1.4493192E-3,1.5084128E-1,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5398465E-1,3.5764718E-1,9.0578735E-2,1.7712615E-1,1.5010995E-1,0E0,0E0,1.367387E-1,1.1113774E-1,3.353101E-2,1.3905274E-2,3.2876134E-2,1.20718196E-1,6.775201E-2,5.870013E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.17382E-2,1.6727448E-2,0E0,6.287142E-2,3.7616774E-2,0E0,7.575147E-2,3.0201033E-2,3.716676E-2,8.603498E-3,2.5892913E-2,4.6929307E-2,0E0,2.140811E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,34,34],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4233672E8,4.9192E4,8.901E3,4.2350553E5,8.812601E-1,-1.8588053E-3,-1.294587E-2,5.160874E0,9.855582E5,1.8136424E7,6.804E3,5.3012047E-2,7.501344E0,2.884462E0,3.9247762E6,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,1E0,2.6239506E5,-4.4569806E-3,1.5324128E6,2.8244882E6,-5.1242317E-4,1.6278354E7,3.57E2,4.888E3,1.3593E4,1.7417817E-1,1.5489E4,1.4493192E-3,1.02952E5,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"split_indices":[46,2,2,48,58,0,0,55,29,1,9,58,54,54,48,0,0,0,0,0,0,74,29,0,33,33,0,46,0,2,10,39,9,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.03E2,8E0,1.89E2,1.4E1,3E0,5E0,1.03E2,8.6E1,6E0,8E0,7E0,9.6E1,6.7E1,1.9E1,3E0,3E0,2E0,6E0,5E0,2E0,6.8E1,2.8E1,5E0,6.2E1,1.5E1,4E0,5.7E1,1.1E1,6E0,2.2E1,8E0,5.4E1,4E0,1.1E1,3.5E1,2.2E1,6E0,5E0,4E0,2E0,2E1,2E0,2E0,6E0,2.7E1,2.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[7.821103E-3,-2.8981427E-3,1.7867728E-1,-5.675793E-2,1.8588716E-2,2.1517694E-1,-1.2397172E-4,-3.6930315E-2,-1.2052149E-2,-1.4890818E-2,4.163438E-2,2.7627808E-1,2.6501652E-3,-5.6514364E-2,6.0741283E-2,-4.3238487E-2,2.5888829E-2,-2.5325203E-2,7.3175E-2,1.5455141E-2,5.614201E-3,-7.2515686E-3,-4.0423278E-2,-8.0662005E-4,6.1275316E-3,1.2608571E-2,-6.090714E-2,-9.619795E-3,7.656288E-2,7.5304456E-2,-7.165898E-2,1.6621768E-1,5.4205015E-2,-1.64469E-2,-1.2827884E-1,2.7917859E-3,-9.4076915E-4,1.9582137E-3,-7.1012415E-2,3.3104515E-3,-3.469917E-2,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,-2.7355356E-2,-1.6309094E-1,1.9180217E-1,2.4875687E-3,5.2126553E-3,3.92498E-2,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,27,29,31,-1,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,-1,59,61,63,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7230206E-1,2.2325236E-1,7.992381E-2,1.9902351E-1,1.0709499E-1,8.315757E-2,0E0,1.0098582E-1,0E0,6.701185E-2,1.7503278E-1,9.8798275E-3,0E0,5.797288E-2,4.7433127E-2,3.4317195E-2,4.39213E-2,1.3128516E-1,8.98312E-2,0E0,0E0,0E0,7.7935606E-2,0E0,0E0,1.3974639E-2,2.9431812E-2,3.1568956E-2,1.9552685E-2,2.7546454E-2,7.250317E-2,1.6699761E-2,3.3773705E-2,4.245451E-2,5.6236163E-2,0E0,0E0,0E0,2.1543108E-2,0E0,1.2517829E-2,0E0,0E0,0E0,0E0,2.2041425E-2,2.2109985E-2,1.6221613E-2,0E0,0E0,3.1004522E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,40,40,45,45,46,46,47,47,50,50],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,28,30,32,-1,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,-1,60,62,64,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,4.039435E-3,4.6E1,2.0604518E8,5.470729E2,2.0867E4,-1.2397172E-4,1E0,-1.2052149E-2,3.5E1,1E0,9.299267E9,2.6501652E-3,9.342733E7,3.52777E5,6.027375E0,5.935E3,1.4364E4,7.573E3,1.5455141E-2,5.614201E-3,-7.2515686E-3,6.6932364E7,-8.0662005E-4,6.1275316E-3,1.625E3,6.558106E-3,6.693824E5,1.0275E4,9.123462E-1,5.4827127E0,1.81025E5,7.03482E5,1.3696512E7,9.64E2,2.7917859E-3,-9.4076915E-4,1.9582137E-3,4.053E3,3.3104515E-3,4E1,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,2.3541917E3,3.800374E6,2.6807916E0,2.4875687E-3,5.2126553E-3,2.1225555E-1,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"split_indices":[2,39,3,7,53,9,0,74,0,3,102,12,0,5,1,54,10,10,9,0,0,0,46,0,0,0,39,49,9,28,58,30,48,46,0,0,0,0,11,0,3,0,0,0,0,4,1,40,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.91E2,1.1E1,5.4E1,1.37E2,9E0,2E0,5E1,4E0,5.6E1,8.1E1,6E0,3E0,4.2E1,8E0,3.3E1,2.3E1,2.6E1,5.5E1,4E0,2E0,5E0,3.7E1,4E0,4E0,8E0,2.5E1,1.4E1,9E0,8E0,1.8E1,8E0,4.7E1,3E1,7E0,3E0,5E0,2E0,2.3E1,3E0,1.1E1,4E0,5E0,5E0,3E0,1.3E1,5E0,6E0,2E0,9E0,3.8E1,7E0,2.3E1,5E0,2E0,9E0,1.4E1,6E0,5E0,2E0,1.1E1,3E0,2E0,4E0,2E0,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[1.6695334E-3,-2.430477E-2,6.987881E-2,-1.628185E-1,-1.665332E-2,3.3794872E-2,1.4406632E-1,-2.0858636E-3,-2.262213E-1,-1.3216189E-3,-8.5313104E-2,7.368082E-2,5.320639E-3,2.1326382E-1,6.0462188E-2,-1.3826551E-2,-4.5224247E-3,-3.0901669E-2,5.3741764E-2,-4.580613E-2,-1.4649896E-2,6.6577974E-3,3.5978813E-2,1.8743427E-2,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,-1.9552542E-2,-8.157249E-3,-1.2044725E-2,9.7219266E-2,-6.857402E-2,-3.44251E-4,5.0435625E-2,-1.1778983E-3,-4.252783E-4,5.2595764E-2,-4.5224298E-2,2.5154488E-2,-2.6128737E-3,4.6722338E-2,-8.63488E-4,1.1892799E-1,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,-1,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,-1,49,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7906808E-1,1.6156235E-1,1.5307292E-1,5.1149473E-2,1.556826E-1,4.617256E-2,9.702435E-2,0E0,1.2178838E-2,2.0037158E-1,2.1631023E-1,3.5536163E-2,2.652885E-2,6.27754E-2,2.7581971E-2,0E0,0E0,1.2084925E-1,1.2416096E-1,2.0679474E-2,0E0,0E0,1.1661252E-2,2.092692E-2,0E0,0E0,0E0,0E0,0E0,8.721569E-2,0E0,4.656056E-2,6.699793E-2,2.1412484E-2,0E0,8.723641E-3,0E0,0E0,1.359814E-2,4.531187E-2,9.016034E-2,0E0,1.4617754E-2,0E0,5.3301275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,22,22,23,23,29,29,31,31,32,32,33,33,35,35,38,38,39,39,40,40,42,42,44,44],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,-1,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,-1,50,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.3125E0,1.3173E4,5.34E2,1.3845824E1,1.5489E4,8E0,-2.0858636E-3,4.233395E6,8.3764136E2,2.1024984E5,5.2560944E1,3.0794094E0,2.2047469E4,2.4711658E5,-1.3826551E-2,-4.5224247E-3,8.260109E9,7.157224E0,8.071663E5,-1.4649896E-2,6.6577974E-3,7.4809E4,1.5383295E6,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,3.7E1,-8.157249E-3,5E1,7.157903E7,1.17E2,-3.44251E-4,2.398624E8,-1.1778983E-3,-4.252783E-4,2.8314E4,1.4218562E5,7.47E2,-2.6128737E-3,5.5326223E9,-8.63488E-4,2.4039803E5,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"split_indices":[43,54,2,0,55,9,8,0,1,53,34,59,43,4,34,0,0,5,55,49,0,0,30,48,0,0,0,0,0,3,0,3,7,10,0,7,0,0,9,29,0,0,5,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.54E2,5.8E1,7E0,1.47E2,4E1,1.8E1,3E0,4E0,1.21E2,2.6E1,1.6E1,2.4E1,9E0,9E0,2E0,2E0,7.9E1,4.2E1,2.3E1,3E0,5E0,1.1E1,2.1E1,3E0,7E0,2E0,4E0,5E0,7.4E1,5E0,1.7E1,2.5E1,1.4E1,9E0,9E0,2E0,1.2E1,9E0,4.7E1,2.7E1,1E1,7E0,4E0,2.1E1,8E0,6E0,2E0,7E0,7E0,2E0,7E0,4E1,1.6E1,1.1E1,3E0,4E0,1.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[3.2811111E-3,-1.1759641E-2,9.609429E-2,-5.231051E-3,-1.926002E-1,4.223107E-2,1.7549825E-1,-2.3685513E-2,3.7570078E-2,-7.758012E-4,-1.34779E-2,7.0535643E-3,2.5339068E-3,1.425658E-2,4.615414E-3,-1.5669417E-2,-9.816772E-3,4.6593472E-2,-5.953628E-3,1.4607337E-3,-3.911037E-2,-1.08244635E-1,-6.703496E-3,1.3439576E-3,8.3413675E-2,-3.8560897E-3,3.702331E-5,-1.4499448E-1,1.2573712E-3,4.405412E-3,-4.881462E-2,2.0919163E-3,-1.956045E-2,9.7872205E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,23,-1,-1,25,27,29,31,33,-1,-1,35,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.80326E-1,2.0324215E-1,1.1234322E-1,1.3358834E-1,8.498618E-2,7.30211E-2,7.642406E-2,1.6523767E-1,7.928742E-2,0E0,0E0,0E0,1.7196054E-2,0E0,0E0,9.389689E-2,0E0,8.1526615E-2,0E0,0E0,9.659185E-3,5.3020775E-2,4.9334954E-2,2.086177E-2,4.4374347E-2,0E0,0E0,2.1213919E-2,0E0,4.3695264E-2,4.650379E-2,0E0,1.4095474E-2,3.8369343E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,17,17,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,32,32,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,24,-1,-1,26,28,30,32,34,-1,-1,36,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.918757E7,1.4623123E8,1.7227725E0,1.0892118E3,8.667E3,1.1944959E8,3.0291426E-1,6.6932364E7,1.8323593E1,-7.758012E-4,-1.34779E-2,7.0535643E-3,8.0756325E-1,1.425658E-2,4.615414E-3,8.35E2,-9.816772E-3,7.316901E0,-5.953628E-3,1.4607337E-3,1.467E3,6.1E1,1.1103951E1,4.8759037E1,2.3524794E7,-3.8560897E-3,3.702331E-5,1.0907971E6,1.2573712E-3,1.643E4,6.87E2,2.0919163E-3,6.879445E1,1.9652087E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"split_indices":[33,46,40,53,2,33,39,46,58,0,0,0,28,0,0,2,0,55,0,0,0,3,55,57,33,0,0,29,0,9,0,0,57,40,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.72E2,2.7E1,1.67E2,5E0,1.7E1,1E1,1.17E2,5E1,2E0,3E0,4E0,1.3E1,3E0,7E0,1.13E2,4E0,4.8E1,2E0,8E0,5E0,9E0,1.04E2,2.2E1,2.6E1,2E0,3E0,7E0,2E0,8.3E1,2.1E1,7E0,1.5E1,2.3E1,3E0,4E0,3E0,4.2E1,4.1E1,1.2E1,9E0,9E0,6E0,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.0387296E-3,-1.8574167E-2,8.5663736E-2,-8.023423E-3,-1.4978984E-1,1.4511184E-1,-3.3413577E-3,-2.7192928E-2,3.1242838E-2,-1.0438316E-2,-2.401442E-3,3.2174537E-1,9.6915334E-2,2.1752674E-2,-7.842911E-3,-2.143232E-2,-6.5535344E-3,9.6770816E-2,-2.4400586E-3,1.8892974E-2,6.1257724E-3,7.6597073E-4,1.1642241E-1,-6.7683046E-5,7.212856E-2,-7.260881E-2,-7.379258E-3,1.2818316E-1,1.6344215E-3,-7.859502E-2,1.6685827E-2,2.2511675E-3,1.3107267E-1,6.177627E-3,-1.2163549E-3,-2.2213712E-2,-1.08554E-1,2.886258E-2,-3.7549652E-2,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,4.6269923E-2,-4.119254E-2,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,-1,-1,-1,31,-1,33,35,37,39,-1,41,43,-1,45,-1,-1,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1510934E-1,2.230747E-1,2.6504415E-1,1.14509515E-1,6.739807E-2,2.2421151E-1,8.667014E-2,6.0260966E-2,1.10594735E-1,0E0,0E0,2.9250383E-2,3.775531E-2,2.2251856E-2,0E0,6.994447E-2,0E0,2.9437363E-2,5.0833475E-2,0E0,0E0,0E0,1.449576E-2,0E0,3.511203E-2,3.5629295E-2,8.637136E-2,1.2054026E-2,0E0,1.4433451E-2,4.9849793E-2,0E0,1.9013852E-2,0E0,0E0,1.4445642E-2,3.5027638E-2,4.9891964E-2,7.3443145E-2,0E0,0E0,0E0,0E0,2.8110322E-2,1.6329382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,22,22,24,24,25,25,26,26,27,27,29,29,30,30,32,32,35,35,36,36,37,37,38,38,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,-1,-1,-1,32,-1,34,36,38,40,-1,42,44,-1,46,-1,-1,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6503105E3,8.67241E9,5.9337012E7,6.90321E2,6.1826086E0,5.0655737E0,1.4623123E8,6.0073395E8,1.10597E5,-1.0438316E-2,-2.401442E-3,2E0,2.819228E-1,4.078932E5,-7.842911E-3,1.142E3,-6.5535344E-3,7.393122E1,1.6595541E1,1.8892974E-2,6.1257724E-3,7.6597073E-4,3.559158E6,-6.7683046E-5,2.1516852E5,5.12334E9,5.62E2,7.44905E0,1.6344215E-3,1E0,6.1093975E6,2.2511675E-3,6.797394E8,6.177627E-3,-1.2163549E-3,1.1086122E3,5.244755E-3,1.5989196E7,2.7816156E5,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,5.6E0,7.3376025E6,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"split_indices":[53,5,46,53,54,55,46,7,30,0,0,8,58,29,0,2,0,59,57,0,0,0,1,0,34,32,0,55,0,17,33,0,7,0,0,34,58,46,29,0,0,0,0,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.61E2,4.9E1,1.5E2,1.1E1,2.9E1,2E1,1.01E2,4.9E1,6E0,5E0,5E0,2.4E1,1.8E1,2E0,9.7E1,4E0,1.6E1,3.3E1,3E0,2E0,5E0,1.9E1,1.3E1,5E0,2E1,7.7E1,1E1,6E0,6E0,2.7E1,4E0,1.5E1,3E0,2E0,9E0,1.1E1,3.5E1,4.2E1,3E0,7E0,2E0,4E0,1.8E1,9E0,2E0,1.3E1,4E0,5E0,7E0,4E0,2.5E1,1E1,1.9E1,2.3E1,2E0,1.6E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.4796249E-3,-2.8328227E-2,4.2443145E-2,-1.2839314E-2,-1.4659679E-1,6.714284E-2,-6.827072E-2,-2.624263E-2,3.197302E-2,-5.994708E-2,-2.0733023E-1,1.3621603E-2,5.6769222E-2,-2.7255906E-2,-8.979017E-3,-2.0926751E-2,-6.972742E-3,6.93552E-2,-2.6764661E-2,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,3.61498E-2,1.4122447E-1,1.9069649E-3,-5.1106434E-2,-1.4165286E-2,-5.6748535E-3,8.369632E-2,-1.472121E-3,-3.9843167E-3,1.1822013E-3,8.772388E-2,2.698954E-3,1.7185566E-1,1.427893E-3,-4.393489E-3,-4.8403983E-4,-7.145611E-2,5.5866083E-4,1.9922133E-3,5.4703923E-3,1.7163315E-1,4.553083E-2,1.823748E-2,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,33,35,-1,37,39,-1,41,-1,-1,-1,43,45,47,-1,-1,-1,49,51,-1,-1,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6417843E-1,2.5274518E-1,2.370248E-1,7.531766E-2,7.177171E-2,1.3860276E-1,7.124599E-2,5.672677E-2,6.5532796E-2,1.3237113E-2,3.143558E-2,0E0,1.1331123E-1,2.2602845E-2,0E0,5.8513593E-2,0E0,2.8243646E-2,3.5750594E-2,0E0,0E0,0E0,0E0,9.637425E-2,3.9510548E-2,0E0,1.52250845E-2,7.427207E-2,0E0,1.5040562E-2,0E0,0E0,0E0,7.007395E-2,7.020194E-2,1.9569606E-2,0E0,0E0,0E0,2.5643677E-2,5.1912256E-2,0E0,0E0,1.0038182E-2,2.4570972E-2,2.6236098E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,18,18,23,23,24,24,26,26,27,27,29,29,33,33,34,34,35,35,39,39,40,40,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,34,36,-1,38,40,-1,42,-1,-1,-1,44,46,48,-1,-1,-1,50,52,-1,-1,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,4.56567E7,1.09495016E8,5.470729E2,7.683833E-1,1.7727281E2,1.4623123E8,5.138372E8,1.1424464E3,3.97E2,2.1E1,1.3621603E-2,4.2245E4,1E0,-8.979017E-3,9.85054E5,-6.972742E-3,2.4848485E0,1.2707859E3,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,2.1637352E7,3.832274E2,1.9069649E-3,8.427037E6,1.056E3,-5.6748535E-3,6.685237E0,-1.472121E-3,-3.9843167E-3,1.1822013E-3,7.573E3,1.28885E1,4.759155E1,1.427893E-3,-4.393489E-3,-4.8403983E-4,7.8206635E0,4.570007E7,1.9922133E-3,5.4703923E-3,5.142232E-1,2.3033286E5,2.249985E5,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"split_indices":[53,46,46,53,28,34,46,7,4,0,3,0,2,8,0,30,0,58,4,0,0,0,0,46,57,0,48,2,0,55,0,0,0,9,55,57,0,0,0,54,5,0,0,58,34,34,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.38E2,8.4E1,1.23E2,1.5E1,6.9E1,1.5E1,9.5E1,2.8E1,7E0,8E0,2E0,6.7E1,1.2E1,3E0,9.2E1,3E0,1.7E1,1.1E1,2E0,5E0,2E0,6E0,5.5E1,1.2E1,3E0,9E0,8.7E1,5E0,1.5E1,2E0,5E0,6E0,2.1E1,3.4E1,9E0,3E0,4E0,5E0,1.7E1,7E1,7E0,8E0,6E0,1.5E1,3.1E1,3E0,2E0,7E0,1.2E1,5E0,6E0,6.4E1,4E0,2E0,8E0,7E0,1.9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.5098556E-3,-3.430795E-2,4.6132654E-2,-7.989392E-2,-7.3299413E-3,6.539497E-2,-1.2817919E-1,-5.846227E-2,-1.0367309E-2,4.5335226E-2,-3.8782444E-2,4.477679E-2,1.7191277E-1,3.5083215E-4,-1.0261507E-2,1.7973488E-3,-6.9773324E-2,6.968256E-2,-1.4760661E-3,-5.74948E-2,8.712715E-3,1.1792199E-1,2.4033781E-2,2.3350367E-1,2.4967322E-3,2.0957861E-2,-8.140613E-2,-2.9711837E-3,8.504071E-2,-2.0592897E-3,-7.8736946E-2,-3.0431528E-2,3.5283656E-3,1.3522897E-3,1.7078307E-1,-1.0893706E-2,4.7292955E-2,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,-9.249218E-2,1.0750794E-3,7.799928E-4,1.05605684E-1,-2.1099192E-3,1.9261859E-3,-2.492124E-2,-1.10814184E-1,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,-3.874176E-2,3.5721097E-2,3.354856E-4,7.9686604E-2,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,29,31,33,35,37,-1,39,41,-1,43,45,47,49,-1,-1,51,53,55,-1,-1,-1,-1,57,-1,-1,59,-1,-1,61,63,-1,-1,-1,-1,65,67,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0736163E-1,1.5191185E-1,2.669213E-1,1.18796796E-1,1.3261785E-1,1.4603654E-1,9.106535E-2,4.6816275E-2,0E0,5.738108E-2,4.4938266E-2,8.8200994E-2,6.973371E-2,0E0,0E0,0E0,4.075806E-2,5.1372252E-2,0E0,4.223033E-2,3.9660476E-2,5.719474E-2,3.9509647E-2,2.0871669E-2,0E0,1.7594527E-2,3.99065E-2,0E0,2.8111845E-2,2.0630281E-2,4.2121127E-2,1.1639008E-2,0E0,0E0,1.40551925E-2,2.734653E-2,3.7402496E-2,0E0,0E0,0E0,0E0,5.2814364E-2,0E0,0E0,8.88367E-3,0E0,0E0,9.187069E-3,3.1335637E-2,0E0,0E0,0E0,0E0,2.286155E-2,2.1192856E-2,0E0,1.3821408E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,41,41,44,44,47,47,48,48,53,53,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,30,32,34,36,38,-1,40,42,-1,44,46,48,50,-1,-1,52,54,56,-1,-1,-1,-1,58,-1,-1,60,-1,-1,62,64,-1,-1,-1,-1,66,68,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,4.039435E-3,1.3917024E8,2.0604518E8,1.2903E4,5.4349E4,1.5214152E5,2.9270105E2,-1.0367309E-2,1.3845824E1,4.311809E5,1.20171E5,1.1406721E2,3.5083215E-4,-1.0261507E-2,1.7973488E-3,2.1013361E-1,9.33E2,-1.4760661E-3,5.948464E0,1.3012535E1,4.361E3,5.590909E0,1.671E3,2.4967322E-3,4E1,4.1211517E1,-2.9711837E-3,6.0775385E0,2.5848E4,1.2753862E1,5.8348556E0,3.5283656E-3,1.3522897E-3,1.57727E3,1.96E4,4.485647E5,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,6.258335E5,1.0750794E-3,7.799928E-4,3.0597075E8,-2.1099192E-3,1.9261859E-3,2E0,1.7261968E7,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,8.062193E-1,6.75741E5,3.354856E-4,1.184E3,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"split_indices":[53,39,46,7,9,2,34,34,0,55,29,30,57,0,0,0,28,2,0,54,57,2,54,0,0,3,59,0,55,9,57,55,0,0,4,2,29,0,0,0,0,33,0,0,7,0,0,8,46,0,0,0,0,28,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.23E2,7.6E1,4.5E1,7.8E1,6.9E1,7E0,4E1,5E0,2.9E1,4.9E1,5.9E1,1E1,3E0,4E0,4E0,3.6E1,2.2E1,7E0,3.5E1,1.4E1,1.2E1,4.7E1,6E0,4E0,4E0,3.2E1,2E0,2E1,1E1,2.5E1,9E0,5E0,5E0,7E0,1.9E1,2.8E1,4E0,2E0,2E0,2E0,2.9E1,3E0,5E0,1.5E1,5E0,5E0,1E1,1.5E1,3E0,6E0,3E0,4E0,1.2E1,7E0,1.3E1,1.5E1,1.5E1,1.4E1,1.1E1,4E0,6E0,4E0,9E0,6E0,5E0,7E0,3E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.6206973E-3,-1.9603977E-2,4.7420494E-2,-4.8373155E-2,9.0342695E-3,7.200922E-2,-6.0652774E-2,-2.3101505E-2,-7.6871574E-2,8.582439E-2,-5.1944912E-3,5.357398E-2,1.6620213E-1,1.5464444E-2,-1.2918839E-1,4.415707E-2,-5.9842244E-2,-5.267666E-4,-1.0224566E-1,5.1139877E-3,-2.7146392E-5,7.1116164E-2,-2.1878945E-2,1.2166778E-1,4.110889E-2,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,-3.735069E-2,-1.0302123E-2,5.6916097E-4,4.160516E-3,-7.803266E-2,1.388256E-3,-1.3427517E-1,-4.7179796E-2,5.5395655E-4,5.7911566E-3,-3.5666935E-2,2.1849289E-3,1.78514E-3,9.610851E-3,1.667288E-2,9.926257E-2,6.1937334E-4,-3.6489156E-3,-5.62302E-2,-5.528736E-3,-7.2530344E-2,-9.035339E-3,-3.0848335E-3,5.726469E-4,-9.4292425E-2,-1.1822653E-2,4.17147E-2,-1.5671453E-2,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,47,-1,49,51,-1,-1,53,-1,-1,-1,55,57,-1,-1,59,-1,61,-1,-1,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.23844E-1,1.1413506E-1,2.1164352E-1,4.791811E-2,7.69298E-2,1.0222992E-1,8.024894E-2,9.65454E-2,5.252196E-2,1.8357664E-2,7.7948034E-2,4.1682094E-2,4.1363686E-2,1.208929E-2,5.2706182E-2,1.7784836E-2,4.241229E-2,0E0,3.423077E-2,0E0,0E0,3.0106395E-2,4.7635924E-2,4.4303104E-2,6.655512E-2,0E0,0E0,0E0,0E0,1.0375681E-2,0E0,0E0,0E0,9.898692E-3,0E0,3.0951291E-2,1.0299623E-2,0E0,0E0,5.7596285E-2,0E0,0E0,0E0,2.9002367E-2,3.170462E-2,0E0,0E0,2.2735469E-2,0E0,9.1954805E-3,0E0,0E0,0E0,2.8296992E-2,2.1532139E-2,1.2805883E-2,1.26161575E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,29,29,33,33,35,35,36,36,39,39,43,43,44,44,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,48,-1,50,52,-1,-1,54,-1,-1,-1,56,58,-1,-1,60,-1,62,-1,-1,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,3.1749217E4,9.820072E7,1.4455165E7,8.637E3,1.6974416E6,1.3611247E5,1.1086122E3,6.9163686E-1,7.105731E0,3.9986582E4,6.518218E6,1.884E3,9.276886E1,2.1000721E0,1.9789175E2,9.946E3,-5.267666E-4,2.19E2,5.1139877E-3,-2.7146392E-5,3.067E3,1.3411796E3,5.62E2,1.6992E4,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,2.012E3,-1.0302123E-2,5.6916097E-4,4.160516E-3,1.6126542E8,1.388256E-3,6.7E1,7.0229135E6,5.5395655E-4,5.7911566E-3,2.4861076E-1,2.1849289E-3,1.78514E-3,9.610851E-3,6.9089854E-1,7.438459E6,6.1937334E-4,-3.6489156E-3,7.774228E-2,-5.528736E-3,7.856E3,-9.035339E-3,-3.0848335E-3,5.726469E-4,1.0092749E8,1.4519928E1,1.8156171E3,1.521636E5,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"split_indices":[53,34,46,46,9,29,34,34,28,54,34,46,0,57,58,53,10,0,10,0,0,2,4,0,2,0,0,0,0,0,0,0,0,5,0,10,33,0,0,40,0,0,0,28,1,0,0,58,0,9,0,0,0,7,57,53,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.37E2,7.7E1,6.8E1,6.9E1,6.3E1,1.4E1,3.7E1,3.1E1,1E1,5.9E1,5.4E1,9E0,7E0,7E0,1.3E1,2.4E1,9E0,2.2E1,8E0,2E0,1E1,4.9E1,7E0,4.7E1,4E0,5E0,3E0,4E0,4E0,3E0,8E0,5E0,2E1,4E0,1.3E1,9E0,5E0,5E0,4.1E1,8E0,4E0,3E0,3.4E1,1.3E1,2E0,2E0,1.4E1,6E0,7E0,6E0,7E0,2E0,1.1E1,3E1,1.9E1,1.5E1,3E0,1E1,8E0,6E0,4E0,3E0,3E0,8E0,2.1E1,9E0,1.2E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.1845648E-4,-1.3225078E-2,1.3611369E-1,-1.417116E-1,-6.7631784E-3,1.953729E-2,8.1282414E-2,-1.8065992E-1,-1.7495391E-4,-5.01936E-2,7.976252E-3,7.00604E-4,1.2666328E-1,-1.9851862E-3,-1.1117795E-2,-3.9532438E-2,-9.23816E-3,1.1314319E-2,-6.9974E-3,7.236729E-3,1.2495602E-3,-2.5365911E-2,-6.0845776E-3,-5.2135065E-4,6.29387E-2,-5.9177596E-3,-7.433232E-2,-5.97098E-3,4.050646E-3,-1.3454848E-3,8.696749E-2,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,-1,23,-1,-1,-1,25,-1,27,29,31,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8503218E-1,1.5596202E-1,2.437979E-1,4.776524E-2,1.1626564E-1,0E0,4.97777E-2,3.7470296E-2,0E0,6.0973354E-2,7.052197E-2,0E0,1.8436834E-2,0E0,0E0,5.011916E-2,0E0,8.190803E-2,0E0,0E0,0E0,3.6700055E-2,0E0,6.1938964E-2,5.6821078E-2,3.1497225E-2,1.2199353E-2,0E0,4.7399078E-2,0E0,2.0637825E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,17,17,21,21,23,23,24,24,25,25,26,26,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,-1,24,-1,-1,-1,26,-1,28,30,32,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,5.314E3,1.4164306E-3,5.3E1,3.5523141E-3,1.953729E-2,4.1151314E0,9.34E2,-1.7495391E-4,4.0515396E-3,3.167407E7,7.00604E-4,1.7652562E4,-1.9851862E-3,-1.1117795E-2,2.47565E6,-9.23816E-3,9.855582E5,-6.9974E-3,7.236729E-3,1.2495602E-3,2.2345362E1,-6.0845776E-3,1E1,2.55E2,1.4740072E1,6.2174755E-1,-5.97098E-3,5.6E0,-1.3454848E-3,1.548321E-1,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"split_indices":[53,9,58,3,40,0,54,0,0,42,44,0,4,0,0,29,0,29,0,0,0,59,0,3,0,59,28,0,55,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.88E2,1.8E1,8E0,1.8E2,2E0,1.6E1,6E0,2E0,4.5E1,1.35E2,7E0,9E0,2E0,4E0,4.3E1,2E0,1.33E2,2E0,7E0,2E0,3.8E1,5E0,1.09E2,2.4E1,2.8E1,1E1,3E0,1.06E2,5E0,1.9E1,1.6E1,1.2E1,3E0,7E0,8E0,9.8E1,9E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.666469E-3,-1.7545465E-2,1.19480304E-1,-8.49554E-3,-8.6954616E-2,1.4809049E-2,7.858761E-2,-1.9953433E-2,4.7460496E-2,-1.262078E-2,-4.3817658E-2,1.2613867E-1,2.3892207E-2,-3.333415E-2,2.116914E-2,-2.6011164E-3,9.102063E-2,7.2902715E-4,-9.959214E-2,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,-2.6154688E-2,-6.2321755E-3,6.586735E-2,-6.205673E-3,-4.2967666E-3,1.9037895E-2,1.7304541E-1,1.7110182E-2,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,-4.9978763E-2,-6.258132E-3,5.289967E-3,1.691339E-3,8.535292E-3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,37,39,-1,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9837632E-1,1.14212275E-1,1.5746966E-1,1.0545254E-1,1.447895E-1,0E0,5.1603585E-2,7.553959E-2,6.1296083E-2,0E0,4.4794835E-2,2.756174E-2,1.8413085E-2,6.669822E-2,4.2011324E-2,2.7640013E-2,8.797648E-2,1.5424975E-2,1.9613385E-2,0E0,0E0,0E0,0E0,4.566688E-2,0E0,1.3546545E-2,2.2841673E-2,0E0,3.328212E-2,1.718907E-2,1.4277115E-2,0E0,0E0,0E0,0E0,2.4923556E-2,6.463696E-2,0E0,0E0,1.3547961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,25,25,26,26,28,28,29,29,30,30,35,35,36,36,39,39],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,38,40,-1,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,7.810581E11,1.4164306E-3,4.5E1,8.0522484E11,1.4809049E-2,1.0096877E8,1.5064244E0,6.787619E2,-1.262078E-2,7.294459E7,3.2635366E8,1.1038146E0,1.4114665E0,1.5489E4,8.27725E3,5.1270317E-2,6.324075E-5,7.5213313E-1,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,3.52777E5,-6.2321755E-3,3.0597075E8,4.187504E7,-4.2967666E-3,6.58542E5,8.63332E5,2.0323718E7,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,2.1034482E-1,4.95715E5,5.289967E-3,1.691339E-3,1.46E3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"split_indices":[53,32,58,3,32,0,52,43,53,0,46,33,35,42,9,49,39,39,28,0,0,0,0,1,0,7,33,0,1,1,46,0,0,0,0,58,1,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.82E2,2.3E1,1.62E2,2E1,3E0,2E1,1.35E2,2.7E1,3E0,1.7E1,1E1,1E1,1.02E2,3.3E1,1.3E1,1.4E1,1E1,7E0,8E0,2E0,6E0,4E0,9.6E1,6E0,1.2E1,2.1E1,2E0,1.1E1,6E0,8E0,3E0,7E0,3E0,4E0,4.3E1,5.3E1,4E0,8E0,1.8E1,3E0,6E0,5E0,2E0,4E0,4E0,4E0,3E1,1.3E1,1E1,4.3E1,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-2.7402204E-3,-1.4488179E-2,8.47742E-2,-1.21823445E-1,-8.342185E-3,1.8740107E-1,-9.2469845E-3,-1.6784839E-3,-1.3934572E-1,1.2403458E-2,-4.2587325E-2,2.9215527E-1,4.7938665E-3,2.4776671E-2,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,4.4642862E-2,-1.8922342E-2,-3.3003904E-2,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,-3.79182E-2,5.552622E-2,-4.235295E-2,3.8699843E-2,1.6639564E-2,-5.707555E-2,-3.353905E-3,1.2455323E-4,7.502207E-2,-1.0837827E-2,-2.996247E-2,-1.0963698E-1,6.998143E-2,-2.54994E-4,7.42517E-2,-9.605715E-3,-8.033993E-2,-1.4847374E-3,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,-1,25,27,29,-1,-1,-1,-1,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1713255E-1,1.21684544E-1,2.4296036E-1,1.0496929E-2,1.2627566E-1,8.29688E-2,6.688262E-2,0E0,1.0810778E-2,1.1294642E-1,1.1641891E-1,3.8371354E-2,0E0,1.0111526E-2,0E0,0E0,0E0,5.1525243E-2,7.859604E-2,7.862471E-2,0E0,0E0,0E0,0E0,0E0,8.959427E-3,6.421043E-2,3.0895934E-2,2.3733113E-2,3.3978265E-2,5.6785285E-2,0E0,0E0,2.0940512E-2,1.2563176E-2,2.8267324E-2,2.118297E-2,8.720197E-3,0E0,1.4422674E-2,9.381125E-3,2.5498733E-2,8.510427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,-1,26,28,30,-1,-1,-1,-1,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,8.35E2,5.9337012E7,9.279E3,2.4934822E7,1.04E3,1.4233672E8,-1.6784839E-3,1.22E2,1.6672E4,6.3522E4,1.6232848E7,4.7938665E-3,9.5589536E-1,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,5.277751E0,1.02942106E11,6.4348924E-1,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,3.4E1,8.019512E0,4.305809E7,3.2395E4,2.7778377E11,3.9880952E-1,-3.353905E-3,1.2455323E-4,4.091697E7,6.2958473E4,2.237806E8,1.0519024E8,7.2023706E0,-2.54994E-4,7.951E3,4.83E9,4.944487E7,1.5107028E-1,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"split_indices":[53,2,46,9,46,0,46,0,10,9,10,1,0,28,0,0,0,55,32,28,0,0,0,0,0,3,54,33,9,32,58,0,0,33,34,7,7,55,0,9,5,46,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.85E2,2.4E1,9E0,1.76E2,1.1E1,1.3E1,2E0,7E0,1.1E2,6.6E1,4E0,7E0,1.1E1,2E0,5E0,2E0,5.4E1,5.6E1,6.4E1,2E0,2E0,2E0,9E0,2E0,6E0,4.8E1,4E1,1.6E1,2.1E1,4.3E1,3E0,3E0,3.7E1,1.1E1,3.5E1,5E0,9E0,7E0,6E0,1.5E1,3E1,1.3E1,2.7E1,1E1,4E0,7E0,2.1E1,1.4E1,3E0,2E0,6E0,3E0,2E0,4E0,4E0,1.1E1,1.2E1,1.8E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.9001366E-3,-1.343855E-2,1.1601315E-1,-1.0010185E-2,-1.111417E-2,2.2562835E-1,3.3268586E-2,-4.3225788E-2,2.3105354E-3,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,-5.3559467E-2,4.5828316E-3,4.5007914E-2,-1.1152626E-2,-4.4588957E-2,-8.577182E-3,-4.0048826E-3,5.9981905E-2,-3.185049E-2,2.3447553E-2,-2.0497924E-2,-8.2263514E-2,8.222998E-3,3.7578024E-2,-1.694235E-2,-1.1662346E-1,3.2336626E-2,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,23,-1,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.899919E-1,1.4066838E-1,1.6482484E-1,7.867399E-2,0E0,1.0884556E-1,1.5947495E-2,7.8872435E-2,8.1554644E-2,0E0,0E0,0E0,0E0,4.606816E-2,0E0,6.974669E-2,7.8090996E-2,4.0971868E-2,0E0,0E0,6.934259E-2,8.396302E-2,4.81951E-2,3.422363E-2,2.2953227E-2,0E0,6.309295E-2,6.78614E-2,3.393674E-2,2.4240296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,24,-1,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.5303E4,1.897887E6,3.5523141E-3,-1.111417E-2,3.0291426E-1,3.9637537E0,1E0,1.19855255E-1,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,4.0515396E-3,4.5828316E-3,5.178571E0,1.5064244E0,1.3431159E6,-8.577182E-3,-4.0048826E-3,3.849802E-1,1.4330416E0,5.46229E5,3.59E2,4.277001E3,8.222998E-3,6.448948E6,9.717001E5,4.3590187E1,1.3047E4,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"split_indices":[4,10,30,40,0,39,54,89,42,0,0,0,0,42,0,55,43,29,0,0,28,42,9,0,48,0,33,29,59,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.93E2,1.8E1,1.91E2,2E0,7E0,1.1E1,5.1E1,1.4E2,4E0,3E0,9E0,2E0,4.8E1,3E0,3.3E1,1.07E2,4.6E1,2E0,3E0,3E1,6.7E1,4E1,2.9E1,1.7E1,4E0,2.6E1,5.8E1,9E0,3.8E1,2E0,1.4E1,1.5E1,1.3E1,4E0,2E1,6E0,4.8E1,1E1,4E0,5E0,1.3E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.2056913E-3,-9.473315E-3,7.512071E-2,4.3374414E-4,-7.8392126E-2,1.5720385E-1,-1.8191284E-2,-2.5484065E-2,2.0686356E-2,-1.2111978E-2,-4.889973E-2,2.523382E-1,8.440286E-2,1.3624126E-2,-6.7835744E-3,1.453452E-2,-4.12981E-2,5.900995E-2,5.9917844E-3,-9.201298E-3,-1.0729721E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,2.635762E-2,-6.7390345E-2,-9.481888E-5,5.3193234E-3,3.617373E-2,-4.9073147E-3,6.533765E-2,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,1.4299699E-3,3.6974133E-3,-3.8031008E-2,-1.2230593E-1,7.3048964E-2,-2.9346012E-2,2.2003332E-3,-1.1724293E-3,-1.1925893E-2,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,-1,45,47,49,-1,-1,-1,-1,51,-1,53,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7272978E-1,1.1952097E-1,2.1732129E-1,8.136228E-2,9.84464E-2,8.259007E-2,5.5882495E-2,4.3634407E-2,4.864037E-2,0E0,4.537849E-2,8.354357E-2,1.8657587E-2,1.5502267E-2,0E0,1.9534279E-2,5.267284E-2,2.3609497E-2,4.1716885E-2,2.1403857E-2,2.0815924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2154378E-2,4.371713E-2,4.4934556E-2,0E0,1.089422E-2,2.8820923E-2,5.3309254E-2,0E0,0E0,0E0,0E0,1.3273055E-2,0E0,1.3264997E-2,2.0031199E-2,2.1552466E-2,2.3880532E-2,0E0,0E0,2.0580877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,28,28,29,29,30,30,32,32,33,33,34,34,39,39,41,41,42,42,43,43,44,44,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,-1,46,48,50,-1,-1,-1,-1,52,-1,54,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.8236975E11,5.9337012E7,3.9360254E2,8.0522484E11,1.04E3,3.6030095E10,1.9689655E1,1.15062E5,-1.2111978E-2,7.294459E7,7.65E2,5.8199444E0,3.6377484E7,-6.7835744E-3,1.7081589E2,4.2060562E1,1E0,8.657441E7,1.6491606E3,5.534958E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,8.351808E-1,3.2022728E1,4.497684E1,5.3193234E-3,2.6921875E1,1E0,6.695E3,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,5.093772E-1,3.6974133E-3,5.1184835E0,4.8887E5,3.7E1,2.0133628E6,2.2003332E-3,-1.1724293E-3,5.0123274E-1,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"split_indices":[53,32,46,53,32,0,5,59,30,0,46,0,54,1,0,4,59,65,46,4,28,0,0,0,0,0,0,0,28,59,59,0,57,89,2,0,0,0,0,28,0,54,1,3,46,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.74E2,2.7E1,1.53E2,2.1E1,1.4E1,1.3E1,6.7E1,8.6E1,2E0,1.9E1,5E0,9E0,1.1E1,2E0,1.9E1,4.8E1,2.3E1,6.3E1,1.2E1,7E0,2E0,3E0,4E0,5E0,8E0,3E0,2E0,1.7E1,2.9E1,1.9E1,6E0,1.7E1,5.4E1,9E0,6E0,6E0,3E0,4E0,1.2E1,5E0,2E1,9E0,5E0,1.4E1,1.5E1,2E0,5E1,4E0,4E0,5E0,7E0,5E0,4E0,1.6E1,7E0,2E0,3E0,2E0,7E0,7E0,2.4E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-3.738608E-4,-7.6801833E-3,1.01941355E-1,7.692978E-3,-4.625046E-2,1.483836E-1,7.440007E-4,-9.925026E-3,6.1333936E-2,-8.378208E-2,-1.7553208E-2,1.8119766E-1,1.0880003E-3,-9.9033825E-2,-3.6509018E-3,1.2566844E-1,3.761748E-2,-1.050452E-1,-2.485187E-3,-5.857001E-3,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,1.0020773E-4,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.9409139E-2,5.540788E-3,-1.2027661E-1,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,1.2911957E-2,-6.143175E-2,2.4736589E-2,-1.5001127E-2,2.155236E-3,-5.720792E-4,-9.470102E-4,-1.3429011E-1,3.1634532E-2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,39,-1,41,-1,-1,-1,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5702043E-1,1.1657343E-1,5.407299E-2,1.3386032E-1,5.8844402E-2,3.31683E-2,0E0,5.945997E-2,4.867518E-2,4.122594E-2,4.3614455E-2,1.10411495E-2,0E0,1.9426428E-2,4.644964E-2,1.4112622E-2,3.5269197E-2,1.4692083E-2,8.812408E-3,3.3128686E-2,0E0,0E0,0E0,0E0,0E0,3.7203256E-2,0E0,0E0,0E0,1.8351674E-2,0E0,1.8578902E-2,0E0,0E0,0E0,1.6290672E-2,2.7178083E-2,4.342707E-2,2.3945123E-2,0E0,0E0,0E0,1.3021171E-2,2.1517647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,29,29,31,31,35,35,36,36,37,37,38,38,42,42,43,43],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,40,-1,42,-1,-1,-1,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,3.0143394E7,7.7434E4,7.7567725E2,2.3443186E3,2.0867E4,7.440007E-4,1.0135136E0,5.63905E6,1.07431E1,1.3844339E1,1.04E3,1.0880003E-3,5.045749E-1,1.3237829E7,1E0,1.3526E4,4.5E1,1.476E3,1.09495016E8,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,6.0130353E0,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.7115049E6,5.540788E-3,2.95E2,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,2.249985E5,2.884462E0,2.7496E4,9.0636E0,2.155236E-3,-5.720792E-4,-9.470102E-4,2.1E1,5.57E2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"split_indices":[2,46,2,53,4,9,0,57,51,55,58,0,0,28,51,102,2,3,0,46,0,0,0,0,0,54,0,0,0,48,0,0,0,0,0,34,54,30,55,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.95E2,1.3E1,1.4E2,5.5E1,8E0,5E0,1.06E2,3.4E1,2.3E1,3.2E1,6E0,2E0,6E0,1E2,8E0,2.6E1,1.8E1,5E0,3E1,2E0,3E0,3E0,3E0,3E0,9.8E1,2E0,2E0,6E0,2.2E1,4E0,1.4E1,4E0,3E0,2E0,2.3E1,7E0,3.7E1,6.1E1,1.2E1,1E1,2E0,1.2E1,1.5E1,8E0,4E0,3E0,9E0,2.8E1,2.1E1,4E1,1E1,2E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.0470916E-4,-1.1145463E-2,9.843046E-2,-6.1118633E-2,4.996951E-3,1.2766055E-2,5.4078806E-2,-2.5317255E-2,-1.1859635E-1,-1.923276E-2,6.010194E-2,-2.131724E-4,3.8350024E-3,-5.8678132E-3,-2.225848E-3,-1.1466437E-2,-6.446723E-2,-1.1111286E-2,-6.9062617E-3,1.0727382E-1,4.0534303E-2,2.531128E-3,-8.6874526E-4,7.2951976E-4,-8.027679E-2,-5.5201775E-3,-5.2759657E-3,3.1679564E-3,8.534004E-3,2.4647668E-2,1.0526845E-1,-5.7665943E-3,-2.0202387E-3,-5.2320316E-2,9.702222E-3,3.7629638E-2,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.165946E-1,1.4873593E-1,1.3118732E-1,8.959766E-2,1.8808083E-1,0E0,2.4914969E-2,1.1248944E-2,9.248865E-2,9.6062645E-2,3.534253E-2,0E0,0E0,1.2025353E-2,0E0,0E0,1.753379E-2,4.9994655E-2,0E0,2.3044884E-2,3.0314423E-2,0E0,0E0,0E0,1.1376724E-2,6.391215E-2,0E0,0E0,0E0,1.0773087E-2,2.1219812E-2,0E0,0E0,2.3858346E-2,4.077713E-2,1.2596151E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,16,16,17,17,19,19,20,20,24,24,25,25,29,29,30,30,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1E0,1.4164306E-3,6.107924E11,8.3764136E2,1.2766055E-2,6.804E3,4.327473E5,8.0522484E11,4.462437E7,6.518218E6,-2.131724E-4,3.8350024E-3,8.6E1,-2.225848E-3,-1.1466437E-2,1E0,6.0073395E8,-6.9062617E-3,4.6008915E6,1.56E4,2.531128E-3,-8.6874526E-4,7.2951976E-4,3.7614656E5,1.292E3,-5.2759657E-3,3.1679564E-3,8.534004E-3,9.087618E6,9.56E2,-5.7665943E-3,-2.0202387E-3,2.8434807E3,7.269755E0,8.461558E10,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"split_indices":[53,102,58,32,53,0,9,29,32,46,46,0,0,10,0,0,8,7,0,46,2,0,0,0,29,2,0,0,0,33,0,0,0,48,54,32,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.83E2,1.9E1,4.4E1,1.39E2,3E0,1.6E1,2.8E1,1.6E1,9.7E1,4.2E1,5E0,1.1E1,1.5E1,1.3E1,4E0,1.2E1,9.2E1,5E0,1.1E1,3.1E1,2E0,1.3E1,2E0,1E1,8.8E1,4E0,8E0,3E0,2.6E1,5E0,4E0,6E0,2.1E1,6.7E1,1.8E1,8E0,2E0,3E0,6E0,1.5E1,4.2E1,2.5E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.050566E-3,4.2812275E-3,-1.5856923E-1,-1.1301997E-2,5.6136116E-2,-2.1941631E-4,-1.0672151E-2,-7.0011653E-3,-1.2052315E-1,8.916989E-3,1.02983706E-1,-5.7596188E-2,1.8009902E-3,-2.245222E-3,-8.737236E-3,-5.627105E-3,5.7158438E-3,1.7363462E-1,6.978703E-2,-1.5238659E-2,-8.90067E-2,1.7688595E-2,-2.4926206E-2,-3.5939007E-3,4.3866397E-3,1.0011509E-2,3.031237E-3,8.032361E-2,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,3.1169657E-2,-2.7009284E-2,-6.0450863E-2,-1.2030017E-3,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7956841E-1,1.679042E-1,6.4495996E-2,7.4314885E-2,1.0526259E-1,0E0,0E0,6.9132246E-2,1.8055126E-2,4.0123533E-2,4.4171542E-2,2.9244505E-2,5.6916848E-2,0E0,0E0,1.6036227E-2,0E0,1.2192935E-2,1.5846536E-2,1.4111153E-2,3.3006206E-2,5.1458158E-2,4.2065214E-2,0E0,8.906513E-3,0E0,0E0,9.820163E-3,0E0,0E0,0E0,0E0,0E0,2.6585214E-2,3.6565024E-2,2.0105965E-2,2.8099384E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,3.0455479E3,8.667E3,1.8600049E10,1E0,-2.1941631E-4,-1.0672151E-2,1.292E3,3.240192E6,8.129243E5,1.2117E4,8.137731E7,2.73782E5,-2.245222E-3,-8.737236E-3,2.3125E0,5.7158438E-3,1.884E3,7.317201E0,1.1086122E3,4.1749332E4,1.06403265E1,2.2009405E1,-3.5939007E-3,1.3174513E4,1.0011509E-2,3.031237E-3,3.2707875E8,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,2.0589474E1,5.3153326E2,6.704386E6,4.462437E7,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"split_indices":[46,4,2,5,102,0,0,2,7,29,9,5,30,0,0,54,0,0,54,34,33,55,57,0,4,0,0,33,0,0,0,0,0,59,4,46,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.06E2,6E0,1.59E2,4.7E1,2E0,4E0,1.54E2,5E0,2.4E1,2.3E1,2.2E1,1.32E2,3E0,2E0,2.2E1,2E0,6E0,1.7E1,1E1,1.2E1,8.3E1,4.9E1,2E0,2E1,4E0,2E0,1.5E1,2E0,3E0,7E0,6E0,6E0,6.4E1,1.9E1,1.9E1,3E1,1.5E1,5E0,1.2E1,3E0,8E0,5.6E1,1E1,9E0,3E0,1.6E1,2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.4155842E-3,-1.3810372E-3,1.2642601E-2,-1.6525343E-2,4.4375908E-2,-1.2493689E-2,-8.009961E-3,1.661861E-2,9.891294E-2,-8.0245316E-2,-4.4310438E-3,4.9835492E-2,-6.8227405E-4,3.6446485E-3,9.054623E-3,-2.6625823E-2,-1.3053659E-1,-7.4558124E-2,1.1256639E-3,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,5.806915E-3,-7.995392E-2,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,9,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[2.0164502E-1,1.420966E-1,0E0,8.987266E-2,7.556003E-2,8.2129285E-2,0E0,2.0147853E-2,2.1653563E-2,3.9669834E-2,5.307982E-2,1.9236067E-2,1.3649627E-2,0E0,0E0,9.523076E-3,1.3058096E-2,3.7478834E-2,4.8667632E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2858007E-2,2.4820045E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,29,29,30,30],"right_children":[2,4,-1,6,8,10,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.8375797E2,1.6572577E0,1.2642601E-2,7.5656E4,1.9846272E9,4.994744E-5,-8.009961E-3,1.5489E4,9.246969E-1,8.7793425E-2,1.9E1,7.614E3,5E0,3.6446485E-3,9.054623E-3,2.8800612E9,3.2445145E2,2.75108E5,1.4655151E0,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,1E0,3.14608E6,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"split_indices":[57,42,0,10,5,43,0,9,28,59,3,10,8,0,0,5,34,9,42,0,0,0,0,0,0,0,0,0,0,89,49,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,2.03E2,2E0,1.53E2,5E1,1.5E2,3E0,3.4E1,1.6E1,1.5E1,1.35E2,1.1E1,2.3E1,1.4E1,2E0,8E0,7E0,9E0,1.26E2,6E0,5E0,1.6E1,7E0,2E0,6E0,5E0,2E0,6E0,3E0,1.2E2,6E0,1.07E2,1.3E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.000657E-3,-1.0153504E-2,9.496438E-2,-3.1673333E-3,-7.777414E-2,1.5645174E-2,6.2693365E-2,5.9380285E-3,-5.026301E-2,-9.674218E-3,-2.5248412E-2,9.625793E-2,2.6888123E-2,-6.8429126E-3,4.4476364E-2,-2.1850644E-2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,8.983803E-3,-4.372294E-2,6.858251E-3,1.07610226E-1,-6.777975E-2,-4.1921283E-4,-1.2492558E-2,3.5249196E-2,-9.645313E-3,-8.684807E-2,-1.2344087E-3,1.4300514E-3,1.2848052E-1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,29,31,33,35,37,-1,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5137326E-1,8.573215E-2,1.699006E-1,7.148311E-2,1.03037156E-1,0E0,2.6254699E-2,6.926221E-2,8.881589E-2,0E0,1.071448E-2,2.5661737E-2,2.7722035E-2,6.226391E-2,8.238617E-2,1.3398929E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2746734E-2,4.618959E-2,1.7440343E-2,9.797484E-3,1.0284005E-2,0E0,3.8003672E-2,4.3830436E-2,3.493194E-2,1.7143585E-2,0E0,0E0,1.4258161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,30,32,34,36,38,-1,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.3983983E10,1.4164306E-3,1.2670352E1,5.535E3,1.5645174E-2,1.095857E10,6.695E3,2.0662694E-1,-9.674218E-3,2.1000721E0,1.7173885E8,3.6377484E7,1.6278354E7,4.485647E5,2.650027E2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,5.097052E8,7.149117E-1,4.602015E0,8.564948E6,2.3968115E2,-4.1921283E-4,1.5498047E1,6.034607E6,9.737113E0,5.8426323E0,-1.2344087E-3,1.4300514E-3,6.879445E1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"split_indices":[53,5,58,55,9,0,12,2,39,0,58,33,1,46,29,4,0,0,0,0,0,0,0,5,28,54,33,4,0,57,49,55,54,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.81E2,2.5E1,1.65E2,1.6E1,2E0,2.3E1,1.39E2,2.6E1,4E0,1.2E1,1.1E1,1.2E1,1.05E2,3.4E1,2.2E1,4E0,8E0,4E0,8E0,3E0,7E0,5E0,7.4E1,3.1E1,2.2E1,1.2E1,4E0,1.8E1,4.1E1,3.3E1,1.8E1,1.3E1,9E0,1.3E1,8E0,4E0,2E0,2E0,3.5E1,6E0,2.5E1,8E0,1.2E1,6E0,8E0,5E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.156695E-3,-7.7511407E-3,6.4903505E-2,1.10561E-3,-6.161546E-2,8.889031E-2,9.5223775E-3,-1.1973289E-2,2.9857075E-2,-4.3537878E-2,-9.037242E-3,8.965046E-4,1.1017409E-1,-6.7902607E-4,2.1986503E-3,7.537986E-3,-3.928328E-2,6.545862E-2,1.1178152E-2,5.3028204E-2,-6.55025E-2,1.2060009E-3,6.3895895E-3,3.2280452E-4,6.727481E-2,-7.918549E-2,-8.271147E-3,3.5601782E-4,7.7395335E-2,2.8093105E-2,-2.6743395E-2,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,6.4077914E-2,-8.103772E-3,-7.31776E-5,4.6275426E-3,2.5301395E-5,-9.328173E-2,-5.3631708E-2,1.3650758E-2,2.564571E-3,6.1303745E-3,2.7586694E-3,-5.2228123E-3,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,-1,-1,-1,-1,49,51,-1,-1,-1,53,55,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5891957E-1,8.784768E-2,4.6976537E-2,6.014745E-2,5.0235853E-2,3.546658E-2,1.0674542E-2,5.890991E-2,3.251822E-2,5.437839E-2,0E0,0E0,3.1012043E-2,0E0,0E0,2.7994063E-2,5.6267872E-2,1.12324655E-2,2.2560306E-2,1.894341E-2,1.4103778E-2,0E0,0E0,3.2216277E-2,1.35331005E-2,2.251944E-2,2.7580641E-2,0E0,1.0156989E-2,2.3157822E-2,1.0649132E-2,0E0,0E0,0E0,0E0,1.371366E-2,2.1627188E-2,0E0,0E0,0E0,1.5584886E-2,1.2574337E-2,9.651603E-3,0E0,0E0,0E0,1.44525645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,35,35,36,36,40,40,41,41,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,-1,-1,-1,-1,50,52,-1,-1,-1,54,56,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,5.075104E7,5.1E1,6.787619E2,6.3538E4,6.19999E9,9.532E3,1.7086E4,1E0,5.577E3,-9.037242E-3,8.965046E-4,1.5214152E5,-6.7902607E-4,2.1986503E-3,1E0,2.5259212E5,2.695E3,7.6760286E-1,2.083122E6,5.534958E-1,1.2060009E-3,6.3895895E-3,4.570007E7,2.77E2,5.8910837E0,2.368447E6,3.5601782E-4,1.4854577E5,8.729897E0,2.3117E4,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,1.297E3,2.0200117E-1,-7.31776E-5,4.6275426E-3,2.5301395E-5,1.2318777E3,2.187288E1,3.3E1,2.564571E-3,6.1303745E-3,2.7586694E-3,7.9206874E8,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"split_indices":[53,46,3,53,10,5,9,9,8,9,0,0,34,0,0,74,29,2,28,1,28,0,0,5,0,54,33,0,34,55,10,0,0,0,0,2,28,0,0,0,4,59,3,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.83E2,3.5E1,1.58E2,2.5E1,2.4E1,1.1E1,1.09E2,4.9E1,2.3E1,2E0,6E0,1.8E1,7E0,4E0,6.4E1,4.5E1,1.6E1,3.3E1,4E0,1.9E1,4E0,1.4E1,5.8E1,6E0,1.9E1,2.6E1,3E0,1.3E1,2.3E1,1E1,2E0,2E0,5E0,1.4E1,6E0,5.2E1,2E0,4E0,3E0,1.6E1,8E0,1.8E1,1E1,3E0,1.2E1,1.1E1,4E0,6E0,3E0,3E0,6E0,4.6E1,1.4E1,2E0,3E0,5E0,1.1E1,7E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-8.12835E-4,2.8342765E-3,-1.5601721E-1,-1.7404577E-2,2.4213504E-2,-2.3796102E-3,-1.0275062E-2,-1.0378984E-2,-1.0568353E-1,9.81486E-3,1.8756628E-2,-2.6070654E-2,1.303203E-2,-6.017882E-4,-7.747756E-3,5.4938868E-2,-4.513947E-3,-1.7527992E-2,-4.4134418E-3,-1.0665796E-2,3.9632928E-2,1.0806894E-1,3.8279194E-2,1.5463653E-3,-6.14344E-3,-6.364611E-2,-6.1187022E-3,2.6128873E-2,-4.657165E-2,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.8797043E-2,-8.169748E-4,-2.9431539E-2,2.5399424E-2,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,-1,-1,21,23,25,-1,27,29,31,33,35,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2327455E-1,9.2584625E-2,1.9196324E-2,6.712644E-2,9.692997E-2,0E0,0E0,3.8177617E-2,3.798466E-2,0E0,8.61301E-2,3.2634463E-2,2.6898822E-2,0E0,0E0,3.1268403E-2,4.6668507E-2,2.9076368E-2,0E0,3.1603176E-2,3.4378827E-2,1.0986567E-2,2.5717452E-2,4.5821525E-2,0E0,1.2384877E-2,2.5712674E-2,1.1580934E-2,1.3976639E-2,0E0,0E0,0E0,0E0,1.3670683E-2,1.1835324E-2,1.6557261E-2,5.3196885E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,-1,-1,22,24,26,-1,28,30,32,34,36,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,6.978001E2,2.9E1,4.462437E7,1.7727281E2,-2.3796102E-3,-1.0275062E-2,7.555544E2,9.214292E7,9.81486E-3,2.4611046E7,1.9284746E1,9.258696E0,-6.017882E-4,-7.747756E-3,7.573E3,1.3844339E1,2.547512E2,-4.4134418E-3,6.855402E6,1.2670352E1,1.884E3,6.5829754E-1,2.819228E-1,-6.14344E-3,3.7790768E2,1.3696512E7,6.735744E1,6.8539324E0,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.982359E-1,2.709E4,7.340202E1,1.3888E4,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"split_indices":[46,53,3,46,34,0,0,4,7,0,46,57,55,0,0,9,58,56,0,46,55,0,28,58,0,34,46,59,55,0,0,0,0,28,9,57,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,2.12E2,4E0,1.09E2,1.03E2,2E0,2E0,1.02E2,7E0,2E0,1.01E2,6.1E1,4.1E1,3E0,4E0,3.9E1,6.2E1,5.5E1,6E0,2.2E1,1.9E1,8E0,3.1E1,6E1,2E0,1E1,4.5E1,1.1E1,1.1E1,1E1,9E0,5E0,3E0,2E1,1.1E1,2.6E1,3.4E1,2E0,8E0,3.7E1,8E0,7E0,4E0,5E0,6E0,1.7E1,3E0,8E0,3E0,1.5E1,1.1E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.6912411E-3,-1.9064195E-2,2.4100082E-2,-6.059589E-3,-7.711967E-2,4.095765E-2,-4.2764593E-2,-5.51559E-2,1.8715139E-3,-1.06207624E-1,-1.2241886E-3,1.0488696E-2,3.2134976E-2,-7.6291934E-3,-1.7113544E-2,5.660094E-4,-7.0297986E-2,5.156652E-3,-5.481648E-3,-6.6033505E-2,-8.217553E-3,7.82574E-3,2.3170885E-2,-4.49811E-2,1.0298288E-3,-4.236804E-3,9.818355E-4,2.7860066E-2,-4.866169E-3,-4.3798788E-3,-7.048719E-4,9.212592E-4,4.4673063E-2,-4.4644894E-3,-3.072789E-4,4.526607E-2,-1.4959747E-2,-3.7038807E-2,1.067402E-2,-6.5524145E-3,3.5458216E-3,2.2589561E-4,8.1912346E-2,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,25,27,-1,29,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,43,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.82962E-2,1.1127101E-1,9.417995E-2,4.7608547E-2,3.814037E-2,9.47225E-2,4.7029093E-2,1.8009417E-2,4.045555E-2,3.1635314E-2,0E0,0E0,7.009145E-2,0E0,1.6951097E-2,0E0,2.2984944E-2,2.3801006E-2,0E0,1.3520818E-2,0E0,0E0,2.9142186E-2,1.5691848E-2,0E0,0E0,0E0,2.4527468E-2,3.6893085E-2,0E0,0E0,1.7144382E-2,4.5832694E-2,0E0,0E0,1.8606756E-2,1.2692557E-2,1.1324577E-2,3.5423E-2,2.0130176E-2,0E0,0E0,1.3577841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,16,16,17,17,19,19,22,22,23,23,27,27,28,28,31,31,32,32,35,35,36,36,37,37,38,38,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,26,28,-1,30,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,44,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,2.325945E9,1.09495016E8,1.142E3,7.888087E0,4.164E3,2.2970924E-2,4.569361E7,1.2224206E7,8.260109E9,-1.2241886E-3,1.0488696E-2,1.1307017E3,-7.6291934E-3,8.66076E1,5.660094E-4,5.9215684E0,2.6374866E1,-5.481648E-3,5.754E3,-8.217553E-3,7.82574E-3,7.6E0,2.884462E0,1.0298288E-3,-4.236804E-3,9.818355E-4,7.438972E0,4.2060562E1,-4.3798788E-3,-7.048719E-4,7.361366E7,2.6023E4,-4.4644894E-3,-3.072789E-4,6.8623977E0,7.444958E7,6.37E2,6.191606E0,8.062193E-1,3.5458216E-3,2.2589561E-4,7.3586698E0,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"split_indices":[53,5,46,2,54,12,28,5,51,5,0,0,53,0,57,0,57,59,0,2,0,0,55,54,0,0,0,54,59,0,0,46,10,0,0,54,7,0,54,28,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.47E2,8.1E1,1.21E2,2.6E1,6.5E1,1.6E1,1.6E1,1.05E2,1.6E1,1E1,2E0,6.3E1,2E0,1.4E1,3E0,1.3E1,1.03E2,2E0,1.1E1,5E0,3E0,6E1,8E0,6E0,1.1E1,2E0,3.1E1,7.2E1,7E0,4E0,3E1,3E1,3E0,5E0,2.2E1,9E0,2.3E1,4.9E1,2.8E1,2E0,1.5E1,1.5E1,1.5E1,7E0,4E0,5E0,1.1E1,1.2E1,2.7E1,2.2E1,1.4E1,1.4E1,1.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.5013376E-3,5.938352E-3,-1.24217264E-1,-2.7496833E-3,8.3399355E-2,-8.301269E-3,-2.700483E-4,-3.2061845E-2,9.8344935E-3,1.079024E-2,4.594788E-2,-1.9474141E-2,-7.3742555E-3,6.1866198E-2,-1.161049E-3,4.8218677E-3,7.569026E-4,-8.262385E-2,-3.5967962E-3,9.7572885E-2,-1.3912333E-3,3.422547E-2,-7.907227E-3,-1.2076516E-3,-6.5928353E-3,5.179689E-2,-2.5634378E-2,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,-5.0402116E-2,2.8263181E-3,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,29,31,-1,-1,33,35,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.14531964E-1,1.3348015E-1,3.7561677E-2,6.634294E-2,9.341748E-2,0E0,0E0,7.825357E-2,7.215855E-2,0E0,2.5913164E-2,4.983531E-2,0E0,7.3844664E-2,2.5336621E-2,0E0,0E0,2.7188063E-2,5.1378243E-2,3.849563E-2,0E0,1.8305652E-2,4.0737204E-2,0E0,0E0,2.3600813E-2,3.192762E-2,0E0,0E0,0E0,0E0,2.3702014E-2,4.1213304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,22,22,25,25,26,26,31,31,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,30,32,-1,-1,34,36,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,8.861832E3,6.45153E5,4.039435E-3,1.4164306E-3,-8.301269E-3,-2.700483E-4,2.0604518E8,7.573E3,1.079024E-2,3.0904205E2,7.5999E4,-7.3742555E-3,8.19162E5,1.2402962E-1,4.8218677E-3,7.569026E-4,4.1749332E4,5.4127317E10,1.946E3,-1.3912333E-3,7.317201E0,3.0738266E4,-1.2076516E-3,-6.5928353E-3,7.356E3,1.9990385E1,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,3.7E1,2.2759746E7,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"split_indices":[46,4,29,39,58,0,0,7,9,0,59,1,0,30,42,0,0,33,32,2,0,54,34,0,0,9,59,0,0,0,0,3,46,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.97E2,6E0,1.78E2,1.9E1,4E0,2E0,5.3E1,1.25E2,3E0,1.6E1,4.9E1,4E0,2.1E1,1.04E2,5E0,1.1E1,9E0,4E1,1.5E1,6E0,1.6E1,8.8E1,5E0,4E0,1.1E1,2.9E1,3E0,1.2E1,1E1,6E0,1.7E1,7.1E1,2E0,9E0,2.1E1,8E0,1.3E1,4E0,5.2E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.211318E-5,-7.6537346E-3,7.079598E-2,6.928152E-3,-3.6480606E-2,1.4365091E-2,3.16417E-2,-9.043619E-3,5.276066E-2,-6.891165E-2,-9.199198E-3,6.183768E-2,-4.7232076E-3,7.881159E-3,-3.276541E-2,7.939335E-3,8.772806E-2,-3.0282011E-2,-1.1089681E-1,-1.8124288E-2,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,1.8954586E-2,-4.7628164E-2,2.5371954E-2,2.6573949E-3,-1.2165993E-2,5.2440288E-3,4.916817E-4,-1.1942704E-2,-4.229069E-3,-3.970585E-5,-1.548476E-1,2.0617556E-3,-4.2224076E-2,6.205118E-2,4.711567E-4,-7.918416E-2,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,2.3531545E-2,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1673426E-1,8.264138E-2,1.751531E-1,9.6372604E-2,5.776053E-2,0E0,7.98365E-2,3.9603867E-2,5.2591182E-2,4.529798E-2,3.7174046E-2,1.3060965E-2,0E0,4.1106842E-2,3.6635753E-2,1.5675733E-2,2.8447852E-2,1.6613591E-2,6.760767E-2,1.7113835E-2,0E0,0E0,0E0,0E0,3.9799407E-2,3.9343596E-2,1.4240889E-2,0E0,1.0493089E-2,0E0,0E0,1.1072869E-2,0E0,0E0,1.0195777E-2,1.9533398E-2,1.2358395E-2,2.6103884E-2,2.7533289E-2,3.5623252E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1920147E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,25,25,26,26,28,28,31,31,34,34,35,35,36,36,37,37,38,38,39,39,50,50],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,2.2971932E7,1.4164306E-3,7.7567725E2,9.875E3,1.4365091E-2,1.402511E8,1.7086E4,4.5470334E5,6.107924E11,3.5632E4,8.0426145E-1,-4.7232076E-3,5.421359E0,1.5071542E0,3.4706132E-3,4.493248E6,7.951E3,2.358119E-1,7.4500206E5,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,4.573327E5,4.4589956E5,1.69228E5,2.6573949E-3,1.8590495E9,5.2440288E-3,4.916817E-4,9.1487586E-1,-4.229069E-3,-3.970585E-5,1.96995E10,1.1036333E3,1.1192292E1,1.4214917E3,4.570007E7,5.6111975E0,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,3.3478114E-1,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"split_indices":[53,46,58,53,9,0,46,9,29,32,2,28,0,55,58,28,1,9,28,49,0,0,0,0,49,29,30,0,5,0,0,28,0,0,5,34,55,49,5,54,0,0,0,0,0,0,0,0,0,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.95E2,2E1,1.3E2,6.5E1,2E0,1.8E1,9.7E1,3.3E1,2.9E1,3.6E1,1.5E1,3E0,5.7E1,4E1,1.5E1,1.8E1,1.6E1,1.3E1,3.4E1,2E0,8E0,7E0,8E0,4.9E1,3.2E1,8E0,4E0,1.1E1,1.4E1,4E0,1.3E1,3E0,4E0,9E0,1.9E1,1.5E1,1.4E1,3.5E1,1.7E1,1.5E1,2E0,6E0,5E0,6E0,9E0,4E0,5E0,4E0,6E0,1.3E1,1.2E1,3E0,2E0,1.2E1,6E0,2.9E1,3E0,1.4E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.1967006E-3,8.5786525E-3,-1.4402847E-1,8.315427E-4,8.315179E-2,-9.1837915E-3,-1.7736525E-3,-1.9747915E-2,1.3641278E-2,1.3899025E-2,4.2534754E-2,-1.29205575E-2,-7.1898274E-2,3.6052603E-2,-8.618837E-3,3.79517E-4,7.254509E-2,-1.8197084E-2,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,2.5952812E-2,-1.9911975E-2,1.2206658E-3,1.6528742E-4,4.4683916E-3,-2.9289007E-2,1.7770747E-2,5.043673E-2,-5.733453E-3,2.2968883E-3,-2.780274E-2,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.356853E-1,1.1663157E-1,2.196373E-2,4.877499E-2,1.4408445E-1,0E0,0E0,2.4350522E-2,5.7182897E-2,0E0,1.7155953E-2,2.955227E-2,1.8783536E-2,5.395387E-2,2.2573741E-2,0E0,1.1681624E-2,2.485658E-2,0E0,0E0,0E0,0E0,4.1406788E-2,2.4611853E-2,0E0,0E0,0E0,1.7023295E-2,2.698188E-2,2.68047E-2,2.1004716E-2,0E0,2.7273526E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,22,22,23,23,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,9.417025E3,1.6330187E8,8.1913336E2,1.4164306E-3,-9.1837915E-3,-1.7736525E-3,1.996866E1,4.91101E5,1.3899025E-2,8.476884E-1,1.6482397E1,2.25813E5,6.37E3,1.0807868E7,3.79517E-4,3.5766065E0,3.8E1,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,6.995449E-1,2E1,1.2206658E-3,1.6528742E-4,4.4683916E-3,1.5662762E10,6.2734294E0,7.269755E0,1E0,2.2968883E-3,1.01618E5,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"split_indices":[46,4,46,56,58,0,0,57,30,0,35,57,1,9,1,0,36,3,0,0,0,0,28,3,0,0,0,32,55,54,101,0,11,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,2.01E2,5E0,1.83E2,1.8E1,3E0,2E0,7E1,1.13E2,2E0,1.6E1,6.3E1,7E0,5.6E1,5.7E1,8E0,8E0,6E1,3E0,2E0,5E0,4E0,5.2E1,4.3E1,1.4E1,2E0,6E0,4.6E1,1.4E1,2.9E1,2.3E1,4E0,3.9E1,2.2E1,2.4E1,4E0,1E1,2.3E1,6E0,1.7E1,6E0,2.1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.9543333E-3,-2.559017E-2,1.9161552E-2,-2.0923022E-2,-5.902389E-3,2.6990097E-2,-7.9133324E-2,-8.834234E-3,-4.04624E-2,1.3379109E-2,7.079206E-2,3.3201702E-4,-8.117297E-3,8.160095E-3,-2.9381018E-2,-7.949769E-2,-1.22999055E-2,4.4339493E-2,-1.4131938E-2,1.0944012E-1,-7.471799E-4,7.50054E-2,-6.7173177E-3,-5.5795915E-2,-1.1298074E-3,-5.1226197E-3,1.4864717E-3,-1.065279E-3,-4.5908066E-3,-3.7182914E-3,6.852332E-2,-5.39476E-2,8.511279E-3,1.29912235E-2,5.2558865E-2,4.981872E-4,4.811432E-3,-3.245346E-3,1.7857874E-3,-2.4376418E-2,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,8.3817184E-2,-7.3740636E-3,-2.728049E-2,5.5922586E-2,-1.4394672E-2,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,-1,45,47,49,51,-1,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0851875E-1,4.7482766E-2,8.293395E-2,2.5152944E-2,0E0,5.747927E-2,6.141771E-2,2.3983913E-2,4.4094764E-2,6.53598E-2,7.825514E-2,0E0,0E0,3.8294274E-2,2.30708E-2,5.1408485E-2,2.2939531E-2,4.2094745E-2,3.7422903E-2,1.2931412E-1,0E0,9.805635E-3,1.6436392E-2,1.2211446E-2,2.4566848E-2,0E0,0E0,9.913108E-3,0E0,1.5312212E-2,2.8017767E-2,3.489702E-2,3.0127015E-2,0E0,2.5919788E-2,0E0,0E0,0E0,1.9405605E-2,2.0426273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.549913E-2,0E0,1.4559356E-2,1.8774323E-2,1.5052079E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,34,34,38,38,39,39,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,-1,46,48,50,52,-1,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,8.260109E9,1.4623123E8,1.7086E4,-5.902389E-3,9.276886E1,1.5214152E5,4.327473E5,2.5259212E5,2.4934822E7,5.2E1,3.3201702E-4,-8.117297E-3,4.570007E7,4.9054803E2,4.758E3,4.1969254E8,2.8626094E5,5.8816573E9,1.6064256E-2,-7.471799E-4,7.444958E7,8.8916E4,2.79E2,6.027375E0,-5.1226197E-3,1.4864717E-3,1.194E3,-4.5908066E-3,1E0,1E0,2.9E1,3.800374E6,1.29912235E-2,3.5935396E-1,4.981872E-4,4.811432E-3,-3.245346E-3,2.0558643E8,3.6E1,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,1.5262272E3,-7.3740636E-3,4E0,5.57E2,2.5848E4,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"split_indices":[53,5,46,9,0,57,34,29,29,46,3,0,0,5,56,2,7,29,5,58,0,7,1,0,54,0,0,10,0,8,102,3,1,0,58,0,0,0,7,10,0,0,0,0,0,0,0,0,4,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.11E2,1.04E2,1.07E2,4E0,9.7E1,7E0,6.7E1,4E1,7.5E1,2.2E1,4E0,3E0,3.7E1,3E1,1.6E1,2.4E1,3.5E1,4E1,1.5E1,7E0,6E0,3.1E1,1.5E1,1.5E1,1.3E1,3E0,2.2E1,2E0,1.2E1,2.3E1,1.4E1,2.6E1,3E0,1.2E1,2E0,4E0,3E0,2.8E1,8E0,7E0,7E0,8E0,1.2E1,1E1,5E0,7E0,4E0,1.9E1,2E0,1.2E1,8E0,1.8E1,4E0,8E0,1.6E1,1.2E1,4E0,4E0,5E0,1.4E1,7E0,5E0,2E0,6E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.1127363E-3,-5.290152E-3,6.0248755E-2,-5.542424E-2,-1.2838513E-3,6.840342E-3,8.6930476E-4,1.3661059E-4,-7.317434E-2,-5.0637187E-3,6.27141E-4,-3.2042078E-4,-4.331261E-3,-7.5224843E-3,3.989758E-2,6.3668867E-3,-2.6071595E-2,-3.1950418E-2,6.536836E-2,-2.7029186E-2,1.4747367E-2,-7.087237E-2,-1.15542095E-2,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,7.527232E-2,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,-1,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8252895E-2,3.5278834E-2,3.8336206E-2,1.3642367E-2,3.2272287E-2,0E0,0E0,0E0,9.951666E-3,0E0,5.21465E-2,0E0,0E0,3.4946784E-2,5.3329237E-2,2.2228872E-2,3.690375E-2,1.3936162E-2,1.901877E-2,1.1226752E-2,2.018778E-2,3.677965E-2,1.6500767E-2,0E0,0E0,0E0,1.2472153E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,-1,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8008E4,1.9388489E3,7.9684106E9,6.051637E-1,1E1,6.840342E-3,8.6930476E-4,1.3661059E-4,7.891632E1,-5.0637187E-3,9.198947E-1,-3.2042078E-4,-4.331261E-3,8.729897E0,7.5773336E7,4.817292E0,2.9E1,6.7995167E0,1.4740072E1,5.9145586E3,7.137324E0,1.6753285E-1,1.6144748E5,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,1.2633E4,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"split_indices":[2,49,12,28,3,0,0,0,52,0,28,0,0,55,7,54,3,55,59,4,54,40,29,0,0,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.75E2,1.1E1,1.2E1,1.63E2,3E0,8E0,3E0,9E0,2E0,1.61E2,2E0,7E0,1.34E2,2.7E1,7.7E1,5.7E1,7E0,2E1,1.5E1,6.2E1,1.3E1,4.4E1,4E0,3E0,2E0,1.8E1,9E0,6E0,5.5E1,7E0,5E0,8E0,1.5E1,2.9E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-9.433804E-4,-1.2022093E-2,5.917883E-2,-6.3716695E-2,-6.556876E-3,1.185642E-2,3.261322E-2,-7.952249E-2,-4.8074138E-4,-1.3831026E-2,3.4149677E-2,4.693409E-2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,-4.5358995E-4,-3.4879703E-2,-6.250601E-4,2.4170529E-3,7.417909E-2,8.9431816E-4,1.8055465E-2,-2.7939798E-2,-7.63545E-2,-1.986685E-2,4.411847E-3,-8.005692E-4,-7.7093607E-3,6.321066E-2,1.0480897E-2,-5.4052684E-2,-4.8673116E-3,-6.69903E-4,-7.4792397E-3,-7.8966774E-2,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,27,29,31,33,-1,-1,35,37,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4125362E-1,5.015369E-2,1.5375312E-1,1.3680756E-2,4.8737537E-2,0E0,6.7347795E-2,8.856602E-3,0E0,3.9123423E-2,1.886667E-2,2.0422306E-2,0E0,0E0,0E0,4.4256426E-2,3.210645E-2,0E0,0E0,2.239582E-2,0E0,6.0984924E-2,3.5681155E-2,1.9354679E-2,2.9425774E-2,0E0,0E0,1.626718E-2,2.6215322E-2,1.7650409E-2,2.1931183E-2,0E0,0E0,1.2872141E-2,1.4290784E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,15,15,16,16,19,19,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,28,30,32,34,-1,-1,36,38,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,4.060294E0,1.4164306E-3,4.668851E6,2.6493E4,1.185642E-2,1.18E2,7.7573473E9,-4.8074138E-4,1.7086E4,7.4353185E0,3.0904205E2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,1.5989196E7,2.2741E4,-6.250601E-4,2.4170529E-3,1.2003011E8,8.9431816E-4,4.984018E8,7.149117E-1,4.4589956E5,1.3000701E7,4.411847E-3,-8.005692E-4,2.80373E5,7.11E2,2.2809816E5,5.36E3,-4.8673116E-3,-6.69903E-4,1.2318777E3,1.1075563E3,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"split_indices":[53,54,58,30,10,0,3,5,0,9,55,59,0,0,0,46,9,0,0,33,0,5,28,29,51,0,0,1,0,29,2,0,0,4,53,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.78E2,3.2E1,1.6E1,1.62E2,3E0,2.9E1,1.2E1,4E0,1.38E2,2.4E1,2.7E1,2E0,8E0,4E0,8.5E1,5.3E1,6E0,1.8E1,1.3E1,1.4E1,5.1E1,3.4E1,1.3E1,4E1,1.1E1,2E0,3.3E1,1.8E1,1.4E1,2E1,9E0,4E0,3.4E1,6E0,1.9E1,1.4E1,1.3E1,5E0,4E0,1E1,1.5E1,5E0,2.8E1,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.7782936E-4,-1.382795E-2,2.3196248E-2,-7.2672334E-3,-1.0122567E-1,-7.0918067E-3,4.573724E-2,-4.658554E-3,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,2.595186E-2,-2.5460321E-2,5.915973E-3,3.5621542E-2,-5.366179E-2,-6.7920005E-4,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,1.8883888E-2,8.107459E-2,-4.6013836E-3,-9.936781E-5,2.8179128E-2,-8.45359E-3,7.830855E-4,5.3368773E-2,7.481408E-3,2.3286117E-3,-1.5134966E-3,4.5415547E-2,-1.37019865E-2,1.673101E-3,3.2098924E-3,-1.2978952E-2,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,-1,-1,31,33,35,37,-1,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.788673E-2,7.380529E-2,5.4942183E-2,3.1929776E-2,2.121576E-2,2.1904452E-2,3.0952565E-2,2.3363702E-2,0E0,0E0,0E0,9.656656E-3,1.7785855E-2,0E0,3.0421652E-2,1.8802509E-2,2.5366073E-2,0E0,0E0,0E0,0E0,1.993274E-2,1.9198395E-2,0E0,0E0,2.5934385E-2,2.0478453E-2,2.058842E-2,1.0298643E-2,0E0,0E0,0E0,1.0949407E-2,2.1243747E-2,0E0,0E0,1.2440793E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,15,15,16,16,21,21,22,22,25,25,26,26,27,27,28,28,32,32,33,33,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,-1,-1,32,34,36,38,-1,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.2224206E7,3E0,1.256E3,5.0655737E0,1.1475E2,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,1.7837838E0,2.3894766E7,5.915973E-3,4.4E1,7.8206635E0,3.06123E5,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,3.4170952E11,1.2117E4,-4.6013836E-3,-9.936781E-5,5.1184835E0,2.6493E4,2.95E2,3.5935396E-1,7.481408E-3,2.3286117E-3,-1.5134966E-3,7.856E3,2.325945E9,1.673101E-3,3.2098924E-3,3.5120192E-1,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"split_indices":[53,5,102,51,8,0,55,53,0,0,0,58,49,0,3,54,12,0,0,0,0,32,9,0,0,54,10,0,58,0,0,0,9,5,0,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.29E2,7.9E1,1.21E2,8E0,3.4E1,4.5E1,1.19E2,2E0,4E0,4E0,1.2E1,2.2E1,4E0,4.1E1,8E0,1.11E2,6E0,6E0,7E0,1.5E1,3.1E1,1E1,4E0,4E0,2.3E1,8.8E1,2.1E1,1E1,2E0,8E0,5E0,1.8E1,7.9E1,9E0,3E0,1.8E1,2E0,8E0,3E0,1.5E1,7E1,9E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.1277195E-4,4.217555E-3,-1.0916315E-1,-2.9917555E-3,5.3992495E-2,5.0713454E-4,-7.7418964E-3,3.2400577E-3,-3.849527E-2,1.5364976E-1,2.9980375E-2,-2.0002978E-2,1.4616362E-2,-1.2040921E-2,-9.3005456E-2,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,-2.5916894E-3,-7.337681E-2,2.9357852E-2,-1.9776661E-2,-3.767159E-2,6.012406E-4,-6.2919264E-3,-7.461606E-4,4.0725708E-2,-2.1920465E-2,-5.1689288E-3,-1.9261596E-3,3.854466E-2,-1.4079762E-2,-5.6224447E-3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7122716E-2,7.65951E-2,4.344309E-2,4.149871E-2,6.000404E-2,0E0,0E0,4.2599544E-2,3.9257012E-2,6.30681E-2,1.8668272E-2,4.906311E-2,5.533707E-2,1.2970728E-2,2.3380555E-2,0E0,0E0,0E0,0E0,3.5220888E-2,1.0222927E-2,3.087072E-2,1.9976046E-2,2.0381412E-2,0E0,0E0,0E0,1.7245183E-2,1.5136076E-2,0E0,0E0,2.189073E-2,2.3911959E-2,1.0170521E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,22,22,23,23,27,27,28,28,31,31,32,32,33,33],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,4.7162656E3,8.667E3,1E0,1.3277E4,5.0713454E-4,-7.7418964E-3,2.427E3,2.7577372E11,4.2E1,4.970186E7,1.3017557E7,8.40547E5,1.6391889E5,1.485E3,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,6.522946E5,6.5657372E0,2.7883E4,1.7657008E7,2.1692245E1,6.012406E-4,-6.2919264E-3,-7.461606E-4,1E0,2.85898E10,-5.1689288E-3,-1.9261596E-3,1.2670352E1,1.5324128E6,2.016E3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"split_indices":[46,53,2,84,10,0,0,2,32,3,52,46,30,29,0,0,0,0,0,49,55,9,33,57,0,0,0,69,32,0,0,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,2.12E2,6E0,1.86E2,2.6E1,2E0,4E0,1.59E2,2.7E1,4E0,2.2E1,5.2E1,1.07E2,1.9E1,8E0,2E0,2E0,6E0,1.6E1,4E1,1.2E1,7.5E1,3.2E1,9E0,1E1,5E0,3E0,1.2E1,2.8E1,5E0,7E0,6.2E1,1.3E1,2.5E1,7E0,5E0,4E0,1E1,2E0,2.3E1,5E0,5.5E1,7E0,3E0,1E1,1.4E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.3611307E-3,-9.2408955E-3,4.410103E-2,-6.9312286E-3,-7.04401E-3,1.3789503E-1,4.378376E-3,-2.0679065E-3,-5.6651037E-2,1.4739492E-3,1.1579285E-2,3.072128E-2,-8.0794595E-2,-4.1738063E-2,2.7708148E-3,-8.478865E-2,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,8.560338E-3,-6.760063E-2,-1.666495E-3,6.8934974E-3,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,-8.954448E-2,3.4993854E-3,4.3400084E-3,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,33,-1,35,-1,-1,-1,-1],"loss_changes":[6.6894464E-2,5.595534E-2,9.999141E-2,4.340747E-2,0E0,7.891193E-2,4.7356185E-2,3.178787E-2,3.738624E-2,0E0,0E0,2.251287E-2,2.4343904E-2,2.3993906E-2,2.2820415E-2,1.3125837E-2,0E0,0E0,0E0,0E0,0E0,1.6774707E-2,1.7332826E-2,0E0,2.1932926E-2,0E0,0E0,0E0,0E0,0E0,9.104654E-3,0E0,2.9089697E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,21,21,22,22,24,24,30,30,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,34,-1,36,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.4311E4,1.04E3,2.3361332E6,-7.04401E-3,7.65E2,3.093317E5,1.278E3,6.8125E0,1.4739492E-3,1.1579285E-2,5.8199444E0,9.532E3,2.8434807E3,1.0458333E1,1E0,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,1.7081589E2,8.137731E7,-1.666495E-3,2.2413793E0,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,2.1367E4,3.4993854E-3,5.3473053E0,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"split_indices":[53,10,0,29,0,0,34,2,54,0,0,54,9,48,59,84,0,0,0,0,0,4,5,0,57,0,0,0,0,0,9,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.81E2,2.6E1,1.79E2,2E0,7E0,1.9E1,1.64E2,1.5E1,4E0,3E0,1.5E1,4E0,1.7E1,1.47E2,1.1E1,4E0,1E1,5E0,2E0,2E0,6E0,1.1E1,1.4E1,1.33E2,4E0,7E0,2E0,4E0,3E0,8E0,4E0,1.29E2,6E0,2E0,7E0,1.22E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4610173E-3,9.7932376E-2,-4.4504614E-3,8.870057E-3,4.9648836E-4,-3.314924E-2,7.020033E-3,-2.3121499E-2,-1.00224335E-2,-5.438503E-3,5.8383252E-2,1.6109983E-2,-4.7826532E-2,7.8370846E-5,-3.111012E-3,7.136404E-3,4.1926984E-2,-1.6382972E-2,4.455269E-2,-6.4636454E-2,-4.9594564E-5,-5.14796E-2,5.3385887E-3,-4.3677186E-4,4.9276084E-2,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,-1.0062985E-1,-8.23799E-2,7.526664E-4,2.452092E-2,-9.486034E-3,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,-1,35,-1,-1,-1,-1,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.105056E-2,4.253041E-2,6.5741345E-2,0E0,0E0,9.4003044E-2,9.1824725E-2,5.4077312E-2,0E0,3.7433997E-2,3.4517996E-2,2.1062033E-2,2.668634E-2,2.929389E-2,0E0,0E0,9.794302E-3,1.1263341E-2,8.910602E-3,2.421417E-2,0E0,2.1960407E-2,2.8100949E-2,0E0,1.2502909E-2,0E0,0E0,0E0,0E0,0E0,1.0933511E-2,1.6032834E-2,0E0,1.9169066E-2,2.5302555E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,21,21,22,22,24,24,30,30,31,31,33,33,34,34],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,-1,36,-1,-1,-1,-1,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.813798E2,8.1356537E2,1E0,8.870057E-3,4.9648836E-4,4.4816768E8,2.7123457E3,2E0,-1.00224335E-2,4.462437E7,1.6064256E-2,6.822511E5,1.3088399E7,8.35E2,-3.111012E-3,7.136404E-3,5.4907064E0,8.901E3,5.1270317E-2,2.041E3,-4.9594564E-5,6.974669E7,1.6533424E6,-4.3677186E-4,7.317201E0,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,3.606E3,4.1749332E4,7.526664E-4,1.4653994E6,6.191606E0,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"split_indices":[34,52,102,0,0,45,4,8,0,46,58,33,1,2,0,0,55,2,39,0,0,7,51,0,54,0,0,0,0,0,10,33,0,51,54,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,5E0,1.98E2,2E0,3E0,5.6E1,1.42E2,5.4E1,2E0,1.15E2,2.7E1,2.1E1,3.3E1,1.06E2,9E0,3E0,2.4E1,1E1,1.1E1,2.4E1,9E0,9E0,9.7E1,3E0,2.1E1,8E0,2E0,4E0,7E0,1.4E1,1E1,6E0,3E0,4.2E1,5.5E1,1.8E1,3E0,3E0,7E0,3E0,3E0,3.7E1,5E0,3.7E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[4.282986E-4,4.1160197E-3,-9.1818266E-2,-1.1841053E-3,7.4616544E-2,-6.1210035E-3,3.2609148E-4,-1.09339515E-2,2.124421E-2,1.0531628E-1,7.8503106E-4,-7.0198895E-3,-9.130692E-2,-6.0027227E-4,4.235613E-2,6.323839E-3,6.2048313E-4,-3.122984E-2,2.2667213E-3,-1.7209452E-3,-6.5925727E-3,4.2787664E-2,-5.5588E-4,8.872538E-3,5.4689284E-2,3.1641412E-3,-7.729943E-2,-7.908227E-3,3.704432E-2,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,19,21,23,-1,-1,25,27,-1,-1,29,-1,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0795104E-2,7.4820675E-2,2.7605474E-2,4.112406E-2,2.3438096E-2,0E0,0E0,4.056964E-2,2.6312627E-2,1.8859334E-2,0E0,2.8399166E-2,1.00320205E-2,1.410548E-2,1.1490557E-2,0E0,0E0,5.5702664E-2,3.28481E-2,0E0,0E0,9.601074E-3,0E0,9.688978E-3,9.379521E-3,1.8371563E-2,2.2842146E-2,2.3135118E-2,2.1546472E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,20,22,24,-1,-1,26,28,-1,-1,30,-1,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,5.4349E4,6.45153E5,1.1565725E0,2.0216698E7,-6.1210035E-3,3.2609148E-4,9.964687E-1,7.6126127E0,2.3274304E2,7.8503106E-4,6.126268E0,9.31E2,7.614E3,6.314179E2,6.323839E-3,6.2048313E-4,8.351808E-1,1.0685499E0,-1.7209452E-3,-6.5925727E-3,1.3559702E8,-5.5588E-4,2.6164518E7,1.5107028E-1,1.424506E6,9.313E3,6.4745765E0,6.6352534E-1,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"split_indices":[46,2,29,43,48,0,0,40,55,57,0,55,0,10,4,0,0,28,58,0,0,7,0,33,39,30,2,55,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.99E2,7E0,1.86E2,1.3E1,5E0,2E0,1.3E2,5.6E1,8E0,5E0,1.25E2,5E0,2.8E1,2.8E1,6E0,2E0,3.4E1,9.1E1,3E0,2E0,5E0,2.3E1,8E0,2E1,2E1,1.4E1,7.1E1,2E1,3E0,2E0,6E0,2E0,2E0,1.8E1,1E1,1E1,8E0,6E0,3E0,6.8E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.1866337E-3,-1.4964924E-3,7.432874E-2,-7.1397156E-2,2.0264261E-3,1.4760831E-1,2.5388267E-2,2.4828428E-4,-5.377491E-3,-2.0973684E-2,1.2397231E-2,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,-8.560193E-2,-1.29591115E-2,2.8582712E-4,3.782403E-2,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,6.9721565E-2,-6.839452E-3,-3.537567E-3,4.555346E-2,5.6208028E-5,4.49603E-3,-2.1360328E-2,1.4701819E-2,6.0154725E-2,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.437666E-2,4.619424E-2,6.0806543E-2,2.7483527E-2,4.295682E-2,4.940191E-2,1.3439575E-2,0E0,0E0,2.7896874E-2,3.8175512E-2,0E0,0E0,0E0,0E0,1.4757402E-2,1.4061235E-2,4.253052E-2,3.747972E-2,0E0,0E0,0E0,0E0,1.2324721E-2,2.471229E-2,0E0,2.2664323E-2,0E0,0E0,2.4652388E-2,2.1451397E-2,1.7983623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,23,23,24,24,26,26,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,5.314E3,1.256E3,6.035227E9,1.05190264E8,5.3696685E0,9.532E3,2.4828428E-4,-5.377491E-3,1.2338027E5,8.1350375E5,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,1.4084E4,1.7669278E5,1.3448397E8,6.0599937E-6,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,2.8E1,1.1812092E0,-3.537567E-3,2.7991664E-1,5.6208028E-5,4.49603E-3,5.791575E-1,3.4E1,1.3839568E1,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"split_indices":[4,9,0,5,7,54,9,0,0,29,29,0,0,0,0,9,29,7,39,0,0,0,0,3,43,0,39,0,0,42,3,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.86E2,1.7E1,8E0,1.78E2,6E0,1.1E1,3E0,5E0,5.5E1,1.23E2,3E0,3E0,4E0,7E0,5E0,5E1,8.4E1,3.9E1,2E0,3E0,3E0,4.7E1,7E0,7.7E1,2E0,3.7E1,2E0,5E0,4.6E1,3.1E1,2.7E1,1E1,3E1,1.6E1,8E0,2.3E1,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4136843E-3,-1.7078152E-2,1.7051285E-2,-4.9061432E-2,-3.8076276E-3,2.0602716E-2,-4.516626E-3,-3.270243E-4,-7.544991E-2,6.5981923E-3,-3.537236E-2,-7.553134E-3,3.3188965E-2,-5.2146493E-3,-2.832102E-2,-1.7100578E-2,3.0886639E-2,1.3335078E-3,-4.885492E-2,4.1569836E-4,-4.885816E-2,2.7083127E-2,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,4.4731785E-2,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,4.3737285E-2,-3.549587E-3,6.7125736E-3,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6693766E-2,4.4803295E-2,3.7026227E-2,3.4130327E-2,2.5196122E-2,3.143581E-2,0E0,0E0,2.535192E-2,3.3913348E-2,1.7419232E-2,1.9031767E-2,2.084884E-2,0E0,1.2429329E-2,1.4870551E-2,2.5628617E-2,0E0,1.2944881E-2,0E0,1.3053814E-2,2.9234491E-2,0E0,0E0,0E0,0E0,0E0,2.8370071E-2,0E0,0E0,0E0,0E0,0E0,1.1751018E-2,1.2866924E-2,1.5719058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,27,27,33,33,34,34,35,35],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9640406E-1,5.511724E0,1.2083455E1,4.958582E-1,9.952575E5,3.225E3,-4.516626E-3,-3.270243E-4,1.0702152E-2,1.643614E6,5.916872E0,2.806E3,1.6974416E6,-5.2146493E-3,5.815166E0,1E0,5.2192265E1,1.3335078E-3,4.4589956E5,4.1569836E-4,1.8465776E8,8.062193E-1,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,1.01978E5,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,1.0252E4,1.0266666E1,7.1706E4,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"split_indices":[40,54,36,28,52,2,0,0,39,33,54,2,29,0,55,106,57,0,29,0,5,28,0,0,0,0,0,30,0,0,0,0,0,9,55,30,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.94E2,1.05E2,8.9E1,3E1,7.5E1,8.7E1,2E0,1.2E1,1.8E1,5.7E1,1.8E1,2.7E1,6E1,1E1,8E0,2.9E1,2.8E1,3E0,1.5E1,2E1,7E0,5.6E1,4E0,6E0,2E0,2.7E1,2E0,2.3E1,5E0,1.2E1,3E0,3E0,4E0,3.6E1,2E1,1.1E1,1.2E1,1E1,2.6E1,1.6E1,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.348882E-3,-1.4007735E-2,2.0349778E-2,-8.801855E-3,-7.60288E-2,3.1053293E-2,-1.0958469E-2,-5.7257973E-2,-5.0193043E-3,-4.4670764E-5,-4.9110665E-3,2.440679E-2,1.0895903E-1,-3.7627418E-2,2.2707816E-2,-4.087384E-3,3.10375E-4,3.1430066E-2,-1.0879586E-2,4.352828E-3,4.6011005E-2,6.7795296E-3,1.5659243E-3,-6.1382737E-2,-3.0169598E-4,4.8260357E-5,2.7864138E-3,6.0025465E-2,-8.0559475E-5,-1.663934E-2,1.3024335E-3,1.8343031E-2,-1.7778914E-2,-1.4250753E-4,6.0098227E-2,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,8.744232E-3,-2.3721121E-2,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,6.921383E-2,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,-1,41,43,-1,45,-1,-1,-1,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.254653E-2,3.32326E-2,3.619688E-2,1.76748E-2,1.4859837E-2,3.8764708E-2,2.5994083E-2,1.244238E-2,1.9993477E-2,0E0,0E0,3.2293484E-2,9.498283E-3,1.1564847E-2,9.696336E-3,0E0,0E0,1.2398537E-2,1.7790027E-2,1.2711223E-2,2.4995036E-2,0E0,0E0,1.1450622E-2,0E0,0E0,0E0,1.1509789E-2,0E0,1.27906855E-2,0E0,9.788271E-3,8.809424E-3,0E0,1.9199878E-2,0E0,0E0,0E0,0E0,8.899177E-3,1.8784469E-2,0E0,0E0,0E0,0E0,1.800523E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,27,27,29,29,31,31,32,32,34,34,39,39,40,40,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,-1,42,44,-1,46,-1,-1,-1,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,3.6509445E1,1.3983983E10,1.1475E2,2.1434378E5,5.2086E4,1.5461028E12,8.842206E7,4.1740056E7,-4.4670764E-5,-4.9110665E-3,4.9754506E5,1.8136424E7,1.3088399E7,6.1921E4,-4.087384E-3,3.10375E-4,4.327473E5,2.6493E4,5.0123274E-1,1E0,6.7795296E-3,1.5659243E-3,1.775894E10,-3.0169598E-4,4.8260357E-5,2.7864138E-3,1.0380544E8,-8.0559475E-5,1.43E2,1.3024335E-3,1.1201525E2,2.513265E5,-1.4250753E-4,1.3578947E1,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,6.1579504E2,2.109E3,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,2.1525126E0,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"split_indices":[53,57,5,53,29,2,32,5,5,0,0,29,1,1,10,0,0,29,10,28,17,0,0,5,0,0,0,7,0,10,0,57,34,0,55,0,0,0,0,4,2,0,0,0,0,40,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.04E2,1.06E2,9.7E1,7E0,7.9E1,2.7E1,6E0,9.1E1,2E0,5E0,7.4E1,5E0,1.5E1,1.2E1,4E0,2E0,1.2E1,7.9E1,3.9E1,3.5E1,3E0,2E0,8E0,7E0,8E0,4E0,6E0,6E0,6.9E1,1E1,2.4E1,1.5E1,8E0,2.7E1,2E0,6E0,3E0,3E0,1.5E1,5.4E1,1.9E1,5E0,1.2E1,3E0,2.4E1,3E0,8E0,7E0,1.8E1,3.6E1,1.6E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.8318873E-3,-1.0389051E-2,6.847433E-2,-9.090252E-2,-7.818896E-3,1.4557077E-1,1.5369026E-2,-1.1227432E-3,-5.793796E-3,-3.6059227E-3,-5.7291586E-2,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,-1.0661409E-2,3.3012316E-2,-1.4323252E-3,-7.734146E-3,-6.9573727E-3,-4.916107E-2,4.209554E-3,2.0994669E-2,-5.233113E-2,-3.3860938E-3,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.4740467E-2,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[8.666744E-2,4.0054876E-2,5.917617E-2,8.709792E-3,3.9811417E-2,4.4877455E-2,9.926351E-3,0E0,0E0,4.6339944E-2,3.7794035E-2,0E0,0E0,0E0,0E0,2.1048589E-2,1.682442E-2,0E0,0E0,2.2184646E-2,1.6582798E-2,0E0,2.2704527E-2,1.6044017E-2,2.14594E-2,0E0,0E0,0E0,9.11136E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,19,19,20,20,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,2.3125E0,2E0,5.34E2,7.8032995E6,6.677E3,6.881048E-1,-1.1227432E-3,-5.793796E-3,1.6503105E3,5.8199444E0,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,9.5096946E-1,6.0775385E0,-1.4323252E-3,-7.734146E-3,1.2425E2,6.128552E0,4.209554E-3,5.590909E0,7.005162E5,1.2804077E7,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.842571E7,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"split_indices":[4,54,8,0,48,9,35,0,0,53,54,0,0,0,0,28,55,0,0,53,54,0,54,29,46,0,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.96E2,1.4E1,5E0,1.91E2,5E0,9E0,2E0,3E0,1.77E2,1.4E1,2E0,3E0,7E0,2E0,1.49E2,2.8E1,1.2E1,2E0,1.37E2,1.2E1,4E0,2.4E1,9E0,1.28E2,9E0,3E0,4E0,2E1,5E0,4E0,7.1E1,5.7E1,1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6622468E-4,-4.8488043E-3,5.08311E-2,-7.834397E-4,-6.2016364E-2,7.5922403E-3,1.9395813E-2,2.0835144E-3,-6.411687E-2,-4.91167E-3,-1.6307954E-3,4.5530796E-2,-4.4596207E-4,-8.775512E-3,2.092946E-2,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,-4.312229E-3,-4.002928E-2,-6.387113E-4,3.425055E-2,-6.662406E-4,-2.56261E-3,-3.120549E-3,1.352972E-4,4.647988E-2,-1.3092252E-3,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1110663E-2,4.5503184E-2,5.6485604E-2,3.354321E-2,1.1037149E-2,0E0,1.1613244E-2,3.640647E-2,3.848807E-2,0E0,0E0,9.618994E-3,0E0,1.5590253E-2,2.9935649E-2,0E0,0E0,0E0,0E0,1.7591223E-2,1.4617277E-2,0E0,2.0499848E-2,1.4878065E-2,0E0,0E0,0E0,2.9250316E-2,9.96951E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,22,22,23,23,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.8600049E10,1.4164306E-3,7.8032995E6,2.884462E0,7.5922403E-3,5.861021E7,6.978001E2,7E0,-4.91167E-3,-1.6307954E-3,2.819228E-1,-4.4596207E-4,1E0,2.8626094E5,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,3.6509445E1,4.966E3,-6.387113E-4,5.2192265E1,4.454691E5,-2.56261E-3,-3.120549E-3,1.352972E-4,9.42191E-1,2.6673E4,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"split_indices":[53,5,58,48,54,0,46,53,8,0,0,58,0,84,29,0,0,0,0,57,2,0,57,49,0,0,0,28,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.95E2,1.7E1,1.83E2,1.2E1,3E0,1.4E1,1.76E2,7E0,4E0,8E0,7E0,7E0,1.12E2,6.4E1,5E0,2E0,2E0,5E0,9.9E1,1.3E1,1.8E1,4.6E1,9.3E1,6E0,8E0,5E0,3.4E1,1.2E1,2.7E1,6.6E1,2.9E1,5E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.3941145E-4,-2.0441902E-3,8.919721E-3,-1.1523694E-2,2.1440972E-2,-9.1706775E-3,-5.6796446E-3,5.424757E-3,1.2543296E-2,-1.505701E-2,4.4461366E-2,1.9426743E-2,-3.729582E-3,-7.865752E-3,-3.6841E-2,3.1923247E-4,3.1530529E-3,-4.811276E-3,5.0493825E-2,-1.1715324E-2,1.7492641E-3,-1.6764663E-2,-3.0829909E-3,-2.7253285E-2,1.2633276E-2,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0149213E-1,4.410473E-2,0E0,3.3826478E-2,4.4260666E-2,4.4551205E-2,0E0,0E0,3.4023255E-2,1.944866E-2,1.0361386E-2,3.789317E-2,0E0,1.6611358E-2,1.5435919E-2,0E0,0E0,1.1697531E-2,2.4396595E-2,1.604245E-2,0E0,1.6484369E-2,0E0,1.106496E-2,1.5908504E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,23,23,24,24],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9028345E2,8.1350375E5,8.919721E-3,2.61E2,5.0655737E0,3.2432E4,-5.6796446E-3,5.424757E-3,7.251039E0,1.8747724E7,1.0252E4,7.6188426E0,-3.729582E-3,2.513265E5,6.995449E-1,3.1923247E-4,3.1530529E-3,1.2153278E-3,9.161318E0,4.8E1,1.7492641E-3,4.588941E-1,-3.0829909E-3,6.809655E0,2.0350546E-3,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"split_indices":[57,29,0,3,55,2,0,0,58,33,9,55,0,34,28,0,0,40,55,10,0,39,0,55,43,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.96E2,2E0,1.4E2,5.6E1,1.38E2,2E0,4E0,5.2E1,1.25E2,1.3E1,4.9E1,3E0,9.5E1,3E1,5E0,8E0,2.8E1,2.1E1,8.8E1,7E0,1.8E1,1.2E1,1.2E1,1.6E1,7E0,1.4E1,7E0,8.1E1,1.5E1,3E0,1E1,2E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.881544E-3,-6.655184E-3,2.188296E-2,-2.8302656E-3,-2.666419E-3,-2.7059872E-2,3.2757495E-2,-9.625506E-3,2.9928401E-2,-5.4137424E-2,7.742816E-7,2.5661085E-2,7.91566E-2,1.4555868E-3,-2.7334698E-2,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,3.091641E-2,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,-5.2867554E-2,1.0829677E-2,-4.0220562E-2,6.7792195E-3,6.600478E-2,1.1288791E-3,-7.709266E-4,-4.6325806E-3,-1.4827026E-2,2.3715133E-2,-2.952948E-4,-5.428135E-2,2.0235393E-3,-9.2693866E-4,8.701906E-2,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,35,37,-1,-1,-1,39,41,-1,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3877617E-2,2.2490527E-2,5.198053E-2,2.5887422E-2,0E0,1.3197487E-2,2.3980856E-2,1.8878557E-2,3.1391144E-2,9.128876E-3,0E0,1.9829843E-2,2.8112642E-2,3.112634E-2,1.6537398E-2,0E0,0E0,0E0,0E0,1.5805755E-2,0E0,0E0,0E0,1.2595711E-2,1.7542738E-2,1.2636464E-2,1.0678305E-2,1.2879096E-2,0E0,0E0,0E0,9.937875E-3,1.8333944E-2,0E0,1.3910651E-2,0E0,0E0,1.5306905E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,19,19,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,37,37],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,36,38,-1,-1,-1,40,42,-1,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2936785E3,3.0978662E6,3.673257E0,1.0907971E6,-2.666419E-3,4.8601307E-2,5.4349E4,2.2906403E-1,1E0,3.26E2,7.742816E-7,1.1778515E7,7.65E2,9.7043E4,4.4589956E5,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,7.573E3,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,4.0289572E2,3.1E1,1.0485419E8,1.2857332E6,1.1865996E1,1.1288791E-3,-7.709266E-4,-4.6325806E-3,1.5403727E-1,1.9540394E7,-2.952948E-4,1.6680729E0,2.0235393E-3,-9.2693866E-4,1.884E3,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"split_indices":[4,29,54,29,0,39,2,58,17,0,0,48,0,1,29,0,0,0,0,9,0,0,0,4,3,7,49,55,0,0,0,58,46,0,40,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.22E2,9.5E1,1.14E2,8E0,1.7E1,7.8E1,9.5E1,1.9E1,8E0,9E0,6.9E1,9E0,5.9E1,3.6E1,1E1,9E0,2E0,6E0,6.3E1,6E0,2E0,7E0,8E0,5.1E1,2.6E1,1E1,1E1,5.3E1,5E0,3E0,1.7E1,3.4E1,8E0,1.8E1,4E0,6E0,7E0,3E0,1.5E1,2E0,2.8E1,6E0,1.3E1,5E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.403546E-3,-9.570334E-4,-4.4911288E-2,-7.611177E-2,1.5534413E-3,-2.1605039E-2,-8.161089E-3,-5.0543556E-3,-6.690376E-4,-5.41183E-3,1.55741805E-2,-2.0407424E-3,4.387262E-4,-1.5254784E-3,-6.8485916E-2,3.5775187E-3,7.775442E-3,5.1245918E-3,-2.5670715E-2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,1.9513896E-2,2.5408894E-2,-7.420587E-3,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4000924E-2,3.5022937E-2,7.077891E-2,8.954152E-3,1.7662771E-2,1.4694375E-2,0E0,0E0,0E0,2.955602E-2,2.6538467E-2,0E0,0E0,1.8596815E-2,1.0216929E-2,0E0,1.7291361E-2,2.3345338E-2,1.594018E-2,0E0,0E0,0E0,8.91481E-3,1.4992485E-2,2.6731981E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,13,13,14,14,16,16,17,17,18,18,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.361366E7,1.0135136E0,7.5656E4,1.38E2,1.500621E3,3.0687964E3,-8.161089E-3,-5.0543556E-3,-6.690376E-4,1.2473612E7,4.965E3,-2.0407424E-3,4.387262E-4,2.1067844E7,1E0,3.5775187E-3,5.1034484E0,1.2623029E7,8.103274E2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,9.43017E5,2.1768606E5,1.1565725E0,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"split_indices":[46,57,10,0,4,53,0,0,0,51,2,0,0,46,8,0,54,12,4,0,0,0,49,29,43,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.84E2,2.5E1,5E0,1.79E2,2.2E1,3E0,3E0,2E0,1.2E2,5.9E1,1.3E1,9E0,1.14E2,6E0,6E0,5.3E1,9E1,2.4E1,2E0,4E0,1.6E1,3.7E1,3.4E1,5.6E1,7E0,1.7E1,6E0,3.1E1,7E0,2.7E1,2.8E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.779959E-6,1.8453373E-3,-5.9890207E-3,-1.285875E-3,6.8091586E-2,-5.2687E-3,3.303071E-2,8.70132E-3,5.314312E-4,-5.49993E-2,-3.4504228E-3,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,-8.831604E-3,1.3855618E-2,-1.8325018E-2,8.489807E-3,-3.5646225E-3,4.3126993E-2,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.611087E-2,4.1366342E-2,0E0,2.6279328E-2,5.5009544E-2,1.53631605E-2,2.091176E-2,0E0,0E0,1.1431506E-2,1.5674567E-2,0E0,0E0,0E0,0E0,2.1203808E-2,2.0550307E-2,1.3397509E-2,1.3573993E-2,1.2377562E-2,1.1399828E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6085916E7,2.0494681E2,-5.9890207E-3,1E0,9.900047E6,2.3125E0,1.04E3,8.70132E-3,5.314312E-4,3.26E2,2.6673E4,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,3.9E1,5.8167644E0,5.540166E-3,7.31E3,3.7988422E5,2.3E1,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"split_indices":[30,57,0,74,1,54,0,0,0,0,10,0,0,0,0,3,54,58,9,29,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.98E2,2E0,1.9E2,8E0,1.71E2,1.9E1,2E0,6E0,5E0,1.66E2,1.3E1,6E0,2E0,3E0,1.27E2,3.9E1,8.2E1,4.5E1,2.5E1,1.4E1,2.1E1,6.1E1,1.2E1,3.3E1,1E1,1.5E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.3686826E-4,-3.519467E-3,5.055597E-2,-2.117268E-3,-4.640776E-3,7.0017064E-3,1.523785E-2,-4.871748E-2,3.465182E-4,-9.892045E-4,4.3544497E-2,6.1831385E-4,-3.1955007E-3,-5.2515082E-2,2.6555143E-3,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,-1.2239224E-3,2.5115369E-2,9.1643715E-3,-1.2135973E-2,5.6670893E-2,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,15,-1,-1,17,19,-1,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0816244E-2,2.5682297E-2,4.724356E-2,2.2864658E-2,0E0,0E0,1.2965487E-2,1.1241557E-2,2.333033E-2,0E0,8.747165E-3,0E0,0E0,2.0471422E-2,1.5975818E-2,0E0,0E0,0E0,0E0,1.790836E-2,3.341707E-2,2.1425955E-2,2.8729388E-2,1.4656175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,16,-1,-1,18,20,-1,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,6.7948157E-1,1.0038867E4,1.2425E2,-4.640776E-3,7.0017064E-3,6.881048E-1,3.7790768E2,2.3125E0,-9.892045E-4,2.7445236E-1,6.1831385E-4,-3.1955007E-3,8.591E3,4.4E1,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,3.3E1,8.87E2,1.127938E6,4.570007E7,1.7E1,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"split_indices":[4,35,4,53,0,0,35,34,54,0,39,0,0,2,3,0,0,0,0,3,0,29,5,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,2E2,1.4E1,1.98E2,2E0,3E0,1.1E1,9E0,1.89E2,5E0,6E0,2E0,7E0,7E0,1.82E2,3E0,3E0,3E0,4E0,1.56E2,2.6E1,8E1,7.6E1,1.4E1,1.2E1,5.3E1,2.7E1,7E0,6.9E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.8720286E-3,2.203888E-3,-3.8517896E-2,7.5318073E-3,-6.002819E-4,-2.1684619E-2,-8.622627E-3,1.0079312E-3,-4.2458405E-3,-2.1083932E-3,1.5030096E-3,-9.4883353E-4,3.8221332E-3,2.3723133E-3,-7.695643E-4,2.9969336E-3,-3.9027993E-2,-1.1395355E-2,1.3584136E-2,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,13,15,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0922936E-2,7.110839E-2,6.3847005E-2,0E0,2.2939872E-2,1.3612385E-2,0E0,2.4651024E-2,0E0,0E0,1.2223788E-2,2.4006953E-2,0E0,0E0,0E0,2.2249037E-2,1.5829908E-2,9.478996E-3,2.061506E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,10,10,11,11,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,14,16,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,3.7902024E0,7.5656E4,7.5318073E-3,1.3105492E1,4.6E1,-8.622627E-3,5.9165E4,-4.2458405E-3,-2.1083932E-3,1E0,2.72258E5,3.8221332E-3,2.3723133E-3,-7.695643E-4,7.6077003E0,5.1737704E7,3.1347964E-2,2.6673E4,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"split_indices":[46,55,10,0,36,3,0,2,0,0,8,9,0,0,0,55,33,58,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.65E2,2.8E1,2E0,1.63E2,2.6E1,2E0,1.61E2,2E0,1.3E1,1.3E1,1.58E2,3E0,3E0,1E1,1.44E2,1.4E1,6.1E1,8.3E1,1.1E1,3E0,2.2E1,3.9E1,6.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-6.800024E-4,1.8960047E-3,-6.0736863E-3,-3.6225263E-2,6.760957E-3,-4.00987E-3,-1.3761937E-2,4.2979596E-3,5.311085E-3,1.561032E-3,-1.3046184E-3,-4.1869044E-2,6.551281E-3,-3.4434297E-3,1.4532126E-3,2.6219185E-3,3.280929E-2,-3.7609767E-3,2.3818783E-2,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,11,-1,-1,-1,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.290568E-2,3.6038343E-2,0E0,2.1682842E-2,4.2899136E-2,0E0,1.0427506E-2,1.7855216E-2,0E0,0E0,0E0,1.8689083E-2,1.662869E-2,0E0,0E0,1.9301808E-2,1.2423562E-2,2.420176E-2,2.810772E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,11,11,12,12,15,15,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,-1,10,12,-1,-1,-1,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.02E2,3.8834286E0,-6.0736863E-3,2.3443186E3,5.2086E4,-4.00987E-3,1.2633E4,1.8E2,5.311085E-3,1.561032E-3,-1.3046184E-3,1.3448806E8,4.6E1,-3.4434297E-3,1.4532126E-3,1.1482285E6,1.10597E5,8.351808E-1,3.1E1,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"split_indices":[8,54,0,4,2,0,2,11,0,0,0,7,3,0,0,29,30,28,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.92E2,3E0,2.1E1,1.71E2,6E0,1.5E1,1.68E2,3E0,3E0,1.2E1,7E0,1.61E2,5E0,2E0,1.41E2,2E1,1.09E2,3.2E1,3E0,1.7E1,8.7E1,2.2E1,2.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.1066506E-4,4.845112E-3,-3.1581253E-2,6.452437E-2,-6.428997E-4,-5.0642762E-2,5.648247E-4,1.1988164E-1,-7.721919E-4,-4.0026516E-2,3.5905282E-3,-2.9436452E-2,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,-2.7368726E-2,7.2603608E-3,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,1.2873769E-2,-1.3507972E-2,3.7806993E-3,1.0231802E-2,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1],"loss_changes":[3.406493E-2,5.8340125E-2,2.528421E-2,6.86559E-2,2.7487058E-2,4.734615E-2,0E0,2.8751194E-2,0E0,2.1383194E-2,1.7127382E-2,1.0340074E-2,0E0,0E0,0E0,0E0,0E0,2.1022052E-2,1.5795767E-2,0E0,0E0,0E0,0E0,1.7355684E-2,1.4121132E-2,0E0,1.356209E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,17,17,18,18,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1],"split_conditions":[1.8000048E10,6.677E3,3.244E3,5.1E1,1.142E3,3.08E3,5.648247E-4,5.8229775E3,-7.721919E-4,5.244755E-3,1.13002876E-4,7.520736E6,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,8.3949E4,7.384892E0,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,3.0404909E0,3.501498E7,3.7806993E-3,4.3E1,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"split_indices":[5,9,0,3,2,0,0,4,0,58,39,30,0,0,0,0,0,12,54,0,0,0,0,57,33,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.77E2,2.9E1,1.4E1,1.63E2,2E1,9E0,8E0,6E0,1.5E1,1.48E2,1.8E1,2E0,6E0,2E0,8E0,7E0,1.5E1,1.33E2,1.6E1,2E0,8E0,7E0,1.05E2,2.8E1,3E0,1.02E2,2.5E1,3E0,7.8E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-2.5543855E-3,-3.670569E-4,-9.478624E-2,-4.503718E-3,3.9570075E-2,-2.891256E-4,-7.399056E-3,7.92847E-3,-1.9427676E-2,1.2930362E-1,9.055213E-3,-2.102676E-2,1.859223E-2,-4.328215E-3,-3.512878E-2,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,6.881532E-3,-3.8464133E-2,3.4366657E-3,1.07829925E-2,-2.6562288E-3,8.551802E-5,-1.3136177E-2,-5.552462E-2,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,-1.9891918E-2,1.758985E-2,-5.6563243E-2,1.090067E-2,-8.1699066E-2,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,-1,-1,-1,-1,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3541346E-2,3.5201967E-2,2.452866E-2,3.5962764E-2,5.359327E-2,0E0,0E0,3.312072E-2,2.0710837E-2,1.6731478E-2,1.720418E-2,1.4377463E-2,3.0927734E-2,1.4162992E-2,1.8558595E-2,0E0,0E0,0E0,0E0,9.007291E-3,9.359658E-3,0E0,1.4757078E-2,0E0,0E0,2.358127E-2,2.1488503E-2,0E0,0E0,0E0,0E0,1.4124965E-2,1.671442E-2,8.771379E-3,1.1919808E-2,1.0015354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,25,25,26,26,31,31,32,32,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,-1,-1,-1,-1,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.25E2,5.2062725E3,2.133183E1,1.6278354E7,1.1823068E7,-2.891256E-4,-7.399056E-3,1.549E3,8.062193E-1,6.23E2,2.0090337E0,4.0431567E-3,1.26747945E5,1.775044E7,7.573E3,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,7.107292E6,8E-3,3.4366657E-3,3.104892E4,-2.6562288E-3,8.551802E-5,3.7614656E5,1.2067E4,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,4.9973932E-1,5.1821535E4,2.849513E3,9.8613055E4,9.377185E-1,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"split_indices":[8,53,55,46,1,0,0,2,28,0,35,43,48,46,9,0,0,0,0,46,58,0,34,0,0,29,9,0,0,0,0,28,34,4,34,28,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.11E2,4E0,1.92E2,1.9E1,2E0,2E0,1.05E2,8.7E1,4E0,1.5E1,2.8E1,7.7E1,4.5E1,4.2E1,2E0,2E0,1.3E1,2E0,1.1E1,1.7E1,9E0,6.8E1,4E0,4.1E1,2.1E1,2.1E1,5E0,6E0,4E0,1.3E1,1.2E1,5.6E1,7E0,1.4E1,1.2E1,9E0,9E0,3E0,1.2E1,4.4E1,4E0,3E0,1.1E1,3E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.7936366E-3,2.4298213E-3,-4.112654E-2,-4.0990007E-3,4.8309796E-2,-2.183913E-2,-8.530635E-3,2.3222237E-3,-3.93802E-2,1.3060242E-1,8.8548794E-4,-4.1647155E-2,3.9303087E-4,-8.698616E-3,2.1892874E-2,-5.8111317E-2,-5.4723965E-3,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,-4.9995827E-3,-3.3989844E-3,3.6737766E-2,-1.1948632E-2,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,-4.3598227E-2,7.2821265E-4,4.7861084E-2,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1747835E-2,5.5313785E-2,6.086246E-2,3.6792308E-2,5.4622788E-2,1.3965219E-2,0E0,2.9954968E-2,1.5560266E-2,4.6919495E-2,0E0,1.526756E-2,0E0,1.9780226E-2,2.5557641E-2,8.726589E-3,9.571268E-3,0E0,0E0,0E0,0E0,1.8823985E-2,0E0,1.4614824E-2,9.260894E-3,0E0,0E0,0E0,0E0,1.8716276E-2,2.0947833E-2,9.567626E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,21,21,23,23,24,24,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3849624E1,9.276886E1,1.019731E0,4.9386452E7,1.6064256E-2,1.7810251E1,-8.530635E-3,8.4182155E10,4.4E1,7.257E3,8.8548794E-4,2.6239506E5,3.9303087E-4,4.991491E8,1.7486866E7,8.3764136E2,9.39616E-1,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,6.974669E7,-3.3989844E-3,4.493248E6,2.6678162E9,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,1.0363E4,7.25E2,1.7747324E7,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"split_indices":[55,57,35,46,58,57,0,32,3,2,0,29,0,7,33,53,28,0,0,0,0,7,0,1,12,0,0,0,0,9,0,46,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.83E2,2.4E1,1.61E2,2.2E1,2.2E1,2E0,1.37E2,2.4E1,5E0,1.7E1,1.3E1,9E0,8.8E1,4.9E1,1.5E1,9E0,2E0,3E0,4E0,9E0,8.4E1,4E0,3.4E1,1.5E1,6E0,9E0,6E0,3E0,1E1,7.4E1,2.6E1,8E0,7E0,8E0,6E0,4E0,5E1,2.4E1,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.3284292E-3,-3.6234653E-3,6.495424E-2,-3.3305318E-4,-4.9157396E-2,9.216801E-3,3.1242464E-2,-1.132699E-2,1.368838E-2,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,-1.8146961E-3,-3.4096006E-2,5.2584518E-2,5.3565246E-3,-6.317367E-3,3.768521E-3,-8.287692E-2,-1.5646296E-2,4.7426983E-3,3.285665E-4,-1.2222636E-3,1.7330685E-3,-9.491318E-3,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,4.4616484E-3,-1.594493E-2,1.9072331E-2,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,27,29,-1,-1,31,-1,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5454505E-2,2.8899927E-2,5.700592E-2,2.8056335E-2,2.499102E-2,0E0,9.989295E-3,2.2029854E-2,2.564449E-2,0E0,0E0,0E0,0E0,2.65384E-2,2.5703099E-2,2.8132629E-2,1.3379129E-2,1.2622242E-2,0E0,1.1075061E-2,1.793515E-2,0E0,0E0,1.7036377E-2,0E0,1.3837478E-2,0E0,0E0,0E0,0E0,8.933199E-3,9.179806E-3,2.1313196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,28,30,-1,-1,32,-1,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.8032995E6,1.4164306E-3,4.7132755E6,5.7592998E0,9.216801E-3,6.5772717E8,2.1557302E6,6.505428E6,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,2.0041434E5,5.9083433E0,7.56E2,9.855582E5,1E0,3.768521E-3,1.1485E4,2.831E3,4.7426983E-3,3.285665E-4,3.6E1,1.7330685E-3,4.43E2,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,2.8037289E1,7.894435E-1,1.1156E4,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"split_indices":[4,48,58,33,54,0,33,33,33,0,0,0,0,34,54,0,29,106,0,9,2,0,0,3,0,0,0,0,0,0,59,28,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.92E2,1.4E1,1.8E2,1.2E1,2E0,1.2E1,1.01E2,7.9E1,1E1,2E0,1E1,2E0,7.2E1,2.9E1,1.3E1,6.6E1,6.9E1,3E0,7E0,2.2E1,6E0,7E0,5.5E1,1.1E1,6.6E1,3E0,3E0,4E0,7E0,1.5E1,3.2E1,2.3E1,3.1E1,3.5E1,3E0,1.2E1,2.1E1,1.1E1,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.613512E-4,9.168064E-3,-2.0914895E-2,2.484408E-3,9.1796964E-2,-1.50884595E-2,-5.1584444E-3,-3.610229E-2,6.524373E-3,2.7067963E-2,8.863301E-3,-3.4663703E-2,1.7349996E-2,-6.1991275E-4,-4.9615046E-3,2.1708814E-2,-6.561107E-3,2.5338226E-3,-1.149334E-3,-2.8247127E-2,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,3.0649362E-2,2.0936963E-3,-1.2641752E-2,-4.455137E-2,1.7206483E-3,3.736369E-3,4.129001E-2,-4.1820602E-3,-8.335901E-3,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,23,25,-1,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.960969E-2,7.377521E-2,3.0622425E-2,1.9721659E-2,5.3537726E-2,3.9970838E-2,0E0,1.7162973E-2,2.2793744E-2,1.0098371E-2,0E0,1.4024261E-2,1.5488645E-2,0E0,0E0,1.6034028E-2,1.923645E-2,0E0,0E0,1.8154921E-2,0E0,0E0,0E0,0E0,1.1873342E-2,0E0,1.6956246E-2,9.147577E-3,1.221883E-2,1.0724097E-2,1.0640074E-2,0E0,1.465915E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,19,19,24,24,26,26,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,24,26,-1,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7864855E11,1.3173E4,1.4146589E8,7.5999E4,5.4E9,1.0807868E7,-5.1584444E-3,5.6978E4,8.3333336E-2,7.438459E6,8.863301E-3,1.226189E0,3.093317E5,-6.1991275E-4,-4.9615046E-3,2.7E1,7.720203E4,2.5338226E-3,-1.149334E-3,8.305745E-1,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,5.2682075E5,2.0936963E-3,9.484755E4,7.4074075E-2,3.7791205E6,3.63E2,7.501344E0,-4.1820602E-3,3.6360288E8,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"split_indices":[32,2,46,1,5,1,0,1,58,1,0,40,34,0,0,3,29,0,0,58,0,0,0,0,33,0,29,58,48,0,54,0,5,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.33E2,6.4E1,1.24E2,9E0,6.1E1,3E0,1.1E1,1.13E2,6E0,3E0,3.8E1,2.3E1,9E0,2E0,5.2E1,6.1E1,4E0,2E0,3.6E1,2E0,1.9E1,4E0,1.1E1,4.1E1,6E0,5.5E1,2.3E1,1.3E1,1.2E1,2.9E1,2E0,5.3E1,1.9E1,4E0,6E0,7E0,9E0,3E0,2.3E1,6E0,1.6E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"3.6910433E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[1.5103106E-2,-2.2944748E-1,4.0485805E-1,-5.2396727E-1,-5.360323E-2,7.115805E-1,-2.6379624E-1,-3.949695E-1,-4.2585783E-2,-1.771069E-1,8.511205E-2,4.611889E-1,1.0766281E0,2.0126725E-2,-6.011713E-1,-4.39699E-1,2.4540755E-3,-2.5400817E-1,-1.0922299E-1,2.1650563E-1,-2.3319207E-1,-1.3541527E-2,5.946551E-1,5.5333883E-2,1.7301071E-2,-1.22435205E-1,1.9814072E-2,-4.192214E-2,-1.1715452E-2,-2.594052E-1,-5.554998E-1,-3.1053467E-4,-3.0310825E-1,8.845983E-2,-1.6904932E-1,2.8078866E-1,6.388342E-2,-1.6651804E-2,-3.3995867E-2,7.99645E-3,-1.5311554E-1,6.501042E-1,6.8630027E-3,-1.1348052E-2,5.7094876E-2,-4.0573996E-2,-3.4288028E-1,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,-3.68585E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,-1.9150509E-1,7.322452E-3,1.7035466E-2,-6.4032795E-3,1.3852504E-1,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,45,47,-1,49,51,53,55,57,-1,59,-1,61,63,-1,-1,65,67,69,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8298021E1,6.0830874E0,1.536447E1,1.6471167E0,1.2999624E0,4.1651173E0,2.3110242E0,6.9953203E-1,0E0,1.810708E-1,1.5561198E0,2.0355868E0,4.665451E-1,8.22276E-1,8.244722E-1,5.031233E-1,0E0,2.1691906E-1,2.8696674E-1,2.3714864E-1,2.271381E-1,2.2374773E-1,5.371609E-1,0E0,0E0,2.3201612E-1,0E0,0E0,0E0,2.418254E-1,3.252077E-2,0E0,1.0391843E-1,5.7198662E-2,4.2966306E-2,1.09799504E-1,1.495303E-1,0E0,2.0135637E-2,0E0,2.2688761E-2,2.5058937E-1,0E0,0E0,6.3281864E-2,1.3045962E-2,8.805108E-2,0E0,0E0,0E0,2.4855494E-2,0E0,0E0,0E0,2.7623773E-2,0E0,0E0,0E0,6.379056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,34,34,35,35,36,36,38,38,40,40,41,41,44,44,45,45,46,46,50,50,54,54,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,46,48,-1,50,52,54,56,58,-1,60,-1,62,64,-1,-1,66,68,70,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.039435E-3,6.627576E7,6.8652206E9,4.5570032E2,4.7162656E3,4E0,4.1211517E1,-4.2585783E-2,1.1383354E6,9.518453E6,1E0,7.317201E0,1.5312085E0,7.588906E-2,5.1147053E5,2.4540755E-3,4.7812922E5,1.630886E6,8.729897E0,1.2534044E3,8.78E2,2.709E4,5.5333883E-2,1.7301071E-2,6.7995167E0,1.9814072E-2,-4.192214E-2,-1.1715452E-2,2.77E2,1E0,-3.1053467E-4,2.8880866E0,4.35E2,2.6374866E1,5.716463E2,2.831E3,-1.6651804E-2,9.17E2,7.99645E-3,3.349E4,1.5145266E6,6.8630027E-3,-1.1348052E-2,6.695E3,9.513E3,1.4377E4,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,1.0491824E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,4.2658337E2,7.322452E-3,1.7035466E-2,-6.4032795E-3,5.586E3,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"split_indices":[53,39,46,5,53,53,8,59,0,52,46,102,54,58,39,29,0,52,46,55,4,0,9,0,0,55,0,0,0,0,105,0,57,0,59,53,2,0,0,0,10,49,0,0,2,9,9,0,0,0,40,0,0,0,53,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.17E2,7.3E1,4.3E1,7.4E1,5E1,2.3E1,3.3E1,1E1,3.9E1,3.5E1,3.1E1,1.9E1,1.3E1,1E1,3E1,3E0,1.7E1,2.2E1,2.5E1,1E1,7E0,2.4E1,1.7E1,2E0,1E1,3E0,5E0,5E0,1.3E1,1.7E1,3E0,1.4E1,5E0,1.7E1,1.7E1,8E0,6E0,4E0,3E0,4E0,2.1E1,3E0,6E0,4E0,4E0,9E0,1.4E1,3E0,5E0,9E0,2E0,3E0,3E0,1.4E1,7E0,1E1,2E0,6E0,2E0,2E0,2E0,2E0,5E0,1.6E1,2E0,2E0,2E0,2E0,5E0,4E0,2E0,7E0,1.2E1,2E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-3.3002432E-2,-1.7783551E-1,5.082772E-1,-2.7808794E-1,6.895006E-2,7.5295E-2,8.787938E-1,-1.8593931E-1,-9.442671E-1,3.6505783E-1,-2.5721067E-1,3.1757823E-1,-1.1269199E-1,4.4750738E-1,1.012557E0,-2.7686968E-1,-1.1995372E-3,-1.790164E-2,-4.942715E-2,1.0660491E-1,5.7554835E-1,3.059845E-2,-4.588169E-1,5.580259E-3,1.7479694E-2,-1.3313731E-2,4.1303214E-2,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,-2.2212821E-1,-4.4453266E-1,7.171647E-2,-2.8227565E-1,2.1667868E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,-7.6152295E-2,1.4846555E-2,-9.6043665E-3,-5.975643E-1,7.728357E-3,-3.7457045E-2,-2.4542458E-1,2.0091075E-3,-7.810127E-3,-2.595061E-2,-1.12920456E-1,1.1881216E-1,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,-1,-1,57,-1,59,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5220662E1,3.804525E0,6.511341E0,6.5156603E0,4.4399395E0,9.4671035E-1,8.160591E-1,1.6291182E0,1.6117287E-1,1.2442894E0,1.2905694E0,3.4778595E-2,3.0859464E-1,2.4943113E-2,1.920681E-1,4.978342E-1,6.9657713E-1,0E0,0E0,3.9868122E-1,7.959266E-1,3.1622946E-1,3.536806E-1,0E0,0E0,0E0,7.072824E-2,0E0,0E0,0E0,0E0,3.2309484E-1,2.9235983E-1,2.4819759E-1,3.8475174E-1,7.987356E-2,0E0,0E0,0E0,3.175994E-2,0E0,0E0,1.1517382E-1,0E0,1.07335895E-2,1.10358E-1,0E0,0E0,0E0,1.0522947E-2,4.5623663E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,26,26,31,31,32,32,33,33,34,34,35,35,39,39,42,42,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,-1,-1,58,-1,60,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5085194E0,1.2871602E3,1.020266E3,8.260109E9,6.971004E7,2.3238E4,1.9107767E0,4.684559E2,2.6E1,5.5326223E9,8.462096E11,5.380165E0,3.236715E0,5.8229775E3,2.8626094E5,1.03431056E6,5.5015685E9,-1.790164E-2,-4.942715E-2,6.4348924E-1,5.6759316E7,5.4827127E0,1.5214152E5,5.580259E-3,1.7479694E-2,-1.3313731E-2,5.884028E0,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,1.1556807E0,5.7625353E-1,7.1568984E7,2.533331E6,4.1808826E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,8.307621E7,1.4846555E-2,-9.6043665E-3,2.012E3,7.728357E-3,1.009E3,5.1146146E-2,2.0091075E-3,-7.810127E-3,-2.595061E-2,4.5E1,7.384995E4,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"split_indices":[43,53,53,5,46,9,43,53,3,5,32,54,57,4,29,29,5,0,0,28,46,58,34,0,0,0,54,0,0,0,0,43,28,7,1,28,0,0,0,46,0,0,0,0,0,28,0,0,0,3,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.92E2,1.52E2,4E1,1.08E2,4.4E1,1.9E1,2.1E1,9.6E1,1.2E1,2.3E1,2.1E1,8E0,1.1E1,6E0,1.5E1,6.4E1,3.2E1,2E0,1E1,1.1E1,1.2E1,9E0,1.2E1,2E0,6E0,5E0,6E0,4E0,2E0,2E0,1.3E1,5E1,1.4E1,2.6E1,6E0,8E0,3E0,9E0,3E0,7E0,2E0,5E0,7E0,2E0,4E0,4.6E1,4E0,4E0,1E1,5E0,2.1E1,4E0,2E0,5E0,3E0,4E0,3E0,2E0,5E0,2E0,2E0,6E0,4E1,3E0,2E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-7.2186072E-3,-1.7554772E-1,4.6836394E-1,-1.3343666E-1,-7.98081E-1,4.8964757E-2,7.305873E-1,-2.4706829E-1,1.5109251E-1,-1.1278819E-2,-9.1069216E-1,-2.2222637E-1,2.0841512E-1,5.310632E-1,1.0273924E0,-2.0253772E-1,-7.193482E-1,8.116926E-2,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,2.4997413E-1,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,-4.6462825E-1,-1.3208786E-1,-1.10341525E-2,-4.028151E-2,-1.6114761E-1,2.4269797E-1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,-5.287513E-1,-9.0207495E-2,-3.1034762E-1,1.1962035E-2,-2.5665852E-1,3.4205204E-1,-1.1185924E-2,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7454023E1,4.1142073E0,6.232873E0,4.9517193E0,4.2668343E-1,1.0408703E0,1.5270805E0,2.1466722E0,2.0214696E0,0E0,3.1103182E-1,1.4763036E-1,1.022948E-1,7.396126E-2,9.1228485E-3,1.8035164E0,2.927456E-1,1.6438966E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2034745E-2,0E0,0E0,0E0,0E0,2.2022247E-1,5.7671654E-1,0E0,0E0,2.841458E-1,6.34436E-1,0E0,0E0,0E0,1.3254166E-2,7.471074E-1,2.088238E-1,2.8850803E-1,9.1903985E-2,4.5164657E-1,1.8377072E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24,29,29,30,30,33,33,34,34,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,7.7567725E2,1.0523087E3,4.14E2,2.6263257E2,1.5605432E9,6.8652206E9,2.3274304E2,-1.1278819E-2,3.7284137E2,2.5546576E4,2.5086822E8,6.8734935E6,3.4456624E7,1.142E3,1.9E1,7.157224E0,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,1.630886E6,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,2.7373068E7,3.0143394E7,-1.10341525E-2,-4.028151E-2,2E0,1.0943299E1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,5.9215684E0,7.54579E5,1.5641167E5,6.802721E-3,1.0180757E0,2.4794E4,9.875E3,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"split_indices":[43,46,53,53,0,53,7,5,57,0,59,34,7,48,48,2,3,55,0,0,0,0,0,0,46,0,0,0,0,5,46,0,0,8,55,0,0,0,57,29,34,58,40,9,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.6E2,5.6E1,1.51E2,9E0,2.2E1,3.4E1,1.08E2,4.3E1,2E0,7E0,8E0,1.4E1,2.2E1,1.2E1,1E2,8E0,4E1,3E0,5E0,2E0,3E0,5E0,3E0,1.1E1,1.8E1,4E0,1E1,2E0,2E1,8E1,2E0,6E0,1.6E1,2.4E1,5E0,6E0,5E0,1.5E1,6.6E1,1.4E1,6E0,1E1,1.7E1,7E0,1.3E1,2E0,4.6E1,2E1,1.2E1,2E0,3E0,3E0,6E0,4E0,1.4E1,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-9.860434E-3,-1.3550225E-1,4.1085625E-1,-4.535459E-1,-6.382216E-2,2.3670215E-2,7.524566E-1,-5.6793636E-1,9.8671935E-2,-9.873285E-2,4.3589786E-1,2.2890316E-1,-1.3927497E-1,9.374098E-1,4.9539495E-1,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,-4.404742E-2,-6.1913097E-1,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,-2.5028974E-1,-2.472767E-2,1.5874594E-2,4.9255155E-2,5.498805E-1,6.3501373E-3,-1.7045158E-1,8.096087E-2,-4.8548445E-2,-3.0521184E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,-6.917094E-2,1.6786462E-2,3.2274876E-2,-2.1812399E-1,-5.0437325E-3,1.5704603E-1,-2.1648994E-2,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,39,-1,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1261909E1,3.7098274E0,6.463071E0,1.9510937E0,2.3981898E0,8.344817E-1,7.7176666E-1,1.549511E-1,3.8158774E-1,3.5869198E0,2.744478E-1,1.8382758E-1,1.7184111E-1,3.0896282E-1,1.867218E-1,0E0,0E0,0E0,0E0,1.8467818E0,1.121139E0,0E0,0E0,0E0,0E0,1.3394743E-2,3.954195E-2,0E0,0E0,2.6690483E-2,0E0,4.5625246E-1,4.6401808E-1,0E0,7.3043644E-2,0E0,0E0,0E0,1.0194156E-2,0E0,0E0,3.1877828E-1,1.343909E-1,8.3303446E-1,9.282285E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,25,25,26,26,29,29,31,31,32,32,34,34,38,38,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,40,-1,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.7575762E4,1.1382212E3,2.0199957E3,1.7730763E12,1.5593E4,8.812601E-1,4.6490747E2,3.20084E6,1.775894E10,4.6673E4,8.147317E0,3.2851852E2,4.558E3,1.4118051E7,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,5.470729E2,8.92851E6,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,6.314915E-1,4.1546512E2,1.5874594E-2,4.9255155E-2,3.76835E7,6.3501373E-3,3.9E1,3.291238E7,-4.8548445E-2,6.881048E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,8.27E2,1.6786462E-2,3.2274876E-2,4.7095413E2,6.9327216E0,1.4154823E-1,2.4654E4,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"split_indices":[43,52,4,4,32,9,58,53,7,5,10,54,4,2,1,0,0,0,0,53,1,0,0,0,0,39,4,0,0,49,0,3,46,0,35,0,0,0,0,0,0,4,54,39,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.63E2,4.8E1,2.9E1,1.34E2,2.3E1,2.5E1,2.4E1,5E0,1.26E2,8E0,1E1,1.3E1,1.3E1,1.2E1,1.4E1,1E1,3E0,2E0,1.15E2,1.1E1,4E0,4E0,8E0,2E0,6E0,7E0,2E0,1.1E1,1E1,2E0,5.7E1,5.8E1,4E0,7E0,4E0,2E0,2E0,5E0,5E0,5E0,4.4E1,1.3E1,3.3E1,2.5E1,5E0,2E0,3E0,2E0,1.2E1,3.2E1,7E0,6E0,1.9E1,1.4E1,2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.037411E-2,-1.7082255E-1,4.0125197E-1,-1.368692E-1,-4.7962084E-2,1.3270798E-1,7.9803574E-1,-2.2494297E-1,7.1677774E-2,2.673719E-1,-1.0539308E-1,1.9987479E-2,4.1916054E-2,-5.089811E-1,-1.4352345E-1,1.7057969E-1,-1.6686903E-2,1.1773016E-1,1.8603355E-2,-1.25300465E-2,-9.196732E-3,-3.748573E-1,-3.2896925E-2,-2.6696217E-1,-5.671464E-2,6.511845E-2,4.1393185E-1,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,-4.2942005E-1,-5.2706E-2,-3.20558E-1,5.7432473E-2,-1.8276703E-1,2.8034353E-1,-4.025757E-2,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,-1,27,-1,-1,29,31,-1,33,35,37,39,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3262604E1,4.125049E0,5.7545595E0,2.7528837E0,0E0,1.1183422E0,1.8529606E-1,2.355206E0,1.903528E0,3.3322942E-1,1.8780968E-1,0E0,0E0,2.6173544E-1,8.7425184E-1,9.226215E-1,0E0,4.9472928E-2,0E0,0E0,1.1085327E-1,8.237338E-2,0E0,3.7353468E-1,7.2992265E-1,6.2363416E-1,1.9650996E-1,0E0,0E0,0E0,0E0,0E0,7.688856E-2,1.9526544E-1,2.2509766E-1,3.0900213E-1,1.7987198E-1,2.4989247E-1,3.8682532E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,20,20,21,21,23,23,24,24,25,25,26,26,32,32,33,33,34,34,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,-1,28,-1,-1,30,32,-1,34,36,38,40,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,1.3453049E3,1.0648859E3,-4.7962084E-2,2.4608E4,2.7350403E5,1.7575762E4,9.820072E7,5.433479E2,2.5546576E4,1.9987479E-2,4.1916054E-2,1.3900659E6,2.802166E2,1.6E1,-1.6686903E-2,2.668E3,1.8603355E-2,-1.25300465E-2,1.9E1,2.9270105E2,-3.2896925E-2,2.124077E1,9.222491E0,3.10214E5,2.4986116E7,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,6.1E1,1.5497297E1,3.079376E8,6.735744E1,3.1E1,5.1E1,6.8125E0,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"split_indices":[43,46,53,53,0,9,29,52,46,53,34,0,0,29,53,8,0,2,0,0,3,34,0,59,55,30,49,0,0,0,0,0,3,59,5,59,3,3,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.53E2,5.4E1,1.48E2,5E0,3.3E1,2.1E1,1.04E2,4.4E1,2.1E1,1.2E1,4E0,1.7E1,2.2E1,8.2E1,3.6E1,8E0,1E1,1.1E1,4E0,8E0,1.4E1,8E0,3.3E1,4.9E1,2.6E1,1E1,4E0,6E0,3E0,5E0,4E0,1E1,7E0,2.6E1,2.6E1,2.3E1,8E0,1.8E1,8E0,2E0,8E0,2E0,3E0,4E0,1.3E1,1.3E1,2.1E1,5E0,9E0,1.4E1,6E0,2E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.8539885E-2,-1.315011E-1,4.7729447E-1,-3.645807E-1,-4.856526E-2,3.3575363E-2,7.048596E-1,-4.006475E-1,1.0404514E-2,-1.2890418E-1,1.0923791E-1,2.1161272E-1,-2.1542291E-1,4.2066466E-2,7.948288E-1,-5.199407E-1,-2.0412306E-1,-1.0585207E-1,-3.185486E-2,-2.5976712E-1,1.9807549E-1,1.8924989E-3,3.4906948E-1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,-5.954765E-1,-2.6221934E-1,-4.2201504E-1,1.2239022E-2,-1.8437941E-1,-8.044045E-3,-5.61047E-2,-2.8251013E-2,2.2316225E-2,1.2032818E-1,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,-1.095318E-1,-2.132905E-1,-1.3484026E-2,1.0698679E-1,-1.6088222E-1,-6.4098774E-3,3.0856705E-3,3.494343E-1,6.278915E-2,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.39668255E1,2.9398837E0,5.0250864E0,9.200578E-1,1.4593433E0,8.4354407E-1,1.9343872E0,7.847419E-1,0E0,8.516334E-1,1.3299325E0,2.4759096E-1,8.105555E-2,1.5741335E-1,1.7285347E-1,2.9159975E-1,7.59462E-1,5.668815E-1,0E0,4.9000573E-1,5.9287274E-1,0E0,1.3428587E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.4581165E-2,1.9908011E-2,1.5304077E-1,2.1783859E-1,1.9969559E-1,6.1496985E-1,6.256449E-2,0E0,0E0,3.2080543E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8989278E-2,1.5572238E-1,2.168184E-2,1.3415802E-1,2.2903234E-1,0E0,0E0,5.3615987E-2,2.547765E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,48,48,49,49,50,50,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.130317E8,6.794883E4,1E0,4.5342876E3,6.787619E2,9.820072E7,3.788E3,1.4862166E1,1.0404514E-2,4.462437E7,7.443E3,3.5073215E3,1.2895599E4,6.09E2,4.2414474E0,6.1E1,5.0548022E2,2.98022E2,-3.185486E-2,9.64E2,1.20171E5,1.8924989E-3,5.1E1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,5.0446276E7,1.0804272E3,1.0363E4,4.1749332E4,2.0890524E6,7.6420575E6,1.1484E4,-2.8251013E-2,2.2316225E-2,1.2117E4,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,1.7488E4,2.124077E1,4.35E2,1.725E3,6.9789816E7,-6.4098774E-3,3.0856705E-3,4E0,6.1108776E7,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"split_indices":[7,51,102,4,53,46,2,59,0,46,9,53,4,0,58,3,4,53,0,0,30,0,3,0,0,0,0,0,0,46,34,9,33,51,46,2,0,0,9,0,0,0,0,0,0,0,0,0,9,59,0,2,12,0,0,8,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.52E2,4.9E1,3.9E1,1.13E2,1.7E1,3.2E1,3.7E1,2E0,7.5E1,3.8E1,1E1,7E0,4E0,2.8E1,2.2E1,1.5E1,7.3E1,2E0,7E0,3.1E1,5E0,5E0,3E0,4E0,2E0,2E0,2.4E1,4E0,1.6E1,6E0,7E0,8E0,4E1,3.3E1,5E0,2E0,6E0,2.5E1,2E0,3E0,1.1E1,5E0,3E0,3E0,2E0,5E0,3E0,5E0,3.4E1,6E0,1.9E1,1.4E1,3E0,2E0,4E0,2.1E1,3E0,2E0,8E0,2.6E1,2E0,4E0,2E0,1.7E1,5E0,9E0,2E0,2E0,1.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-2.776328E-2,-1.2117824E-1,3.86292E-1,-5.909288E-2,-5.795699E-1,5.5683386E-1,-2.6300025E-1,-2.0977455E-1,9.621003E-2,-8.023927E-1,-1.1718366E-1,2.3014233E-1,6.94869E-1,-1.3781548E-3,-3.7562144E-1,-3.571755E-1,-1.0406611E-1,1.914385E-1,-8.06779E-2,-2.3914319E-2,-4.741177E-2,9.839836E-3,-3.492453E-1,3.794242E-1,-6.1298814E-3,3.9063934E-2,3.491601E-1,-2.482055E-2,-3.466254E-3,-4.0868425E-1,-5.727207E-2,-1.4600524E-1,2.0078115E-2,7.405951E-2,3.1055027E-1,-1.1359027E-1,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,-4.7098124E-1,-1.5451024E-1,2.9516693E-3,-6.394594E-3,4.4541497E-2,-1.712155E-1,7.7366033E-3,-5.9316065E-2,1.5788026E-1,-6.221433E-2,3.575082E-1,-1.2339829E-3,-7.365338E-2,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,61,63,-1,65,67,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.401466E0,5.003068E0,4.575707E0,3.694187E0,2.0696063E0,1.2831182E0,2.2723442E-1,1.2093525E0,1.3293368E0,2.6940823E-1,6.6352755E-1,6.3254225E-1,5.397568E-1,0E0,2.0912564E-1,4.8636627E-1,2.528435E-1,6.9004285E-1,3.1393635E-1,0E0,0E0,0E0,6.1558306E-2,2.0225549E-1,0E0,0E0,7.9099655E-2,0E0,0E0,3.7058783E-1,6.041512E-2,1.8056059E-1,1.5278538E-1,3.1823194E-1,4.0340137E-1,1.3923776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.6962032E-2,1.4837602E-1,0E0,0E0,2.0146374E-2,1.326319E-1,0E0,4.3555457E-2,1.6426831E-1,1.09528095E-1,6.0819864E-2,0E0,7.7050425E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,22,22,23,23,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,62,64,-1,66,68,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0105532E3,7.6599895E9,1.2055723E8,4.684559E2,1.5461028E12,8.601656E6,1.3611247E5,1.822764E4,2.2971932E7,3.6509445E1,2.5E1,2.4716E4,5.9283892E7,-1.3781548E-3,1.3382495E8,1E0,2.0806985E-1,7.7567725E2,7.4297624E0,-2.3914319E-2,-4.741177E-2,9.839836E-3,1.776E3,5.1E1,-6.1298814E-3,3.9063934E-2,1.046E4,-2.482055E-2,-3.466254E-3,1.01978E5,1.685E3,4.914413E-3,5.1E2,6.2742615E0,2.8314E4,8.539363E7,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,7.774228E-2,7.359468E8,2.9516693E-3,-6.394594E-3,3.76E2,5.2560944E1,7.7366033E-3,3.892E3,2.7883E4,4.4589956E5,7.3586698E0,-1.2339829E-3,1E0,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"split_indices":[53,5,46,53,32,1,34,34,46,57,3,2,46,0,33,105,39,53,54,0,0,0,0,3,0,0,9,0,0,30,2,39,0,54,9,46,0,0,0,0,0,0,0,58,5,0,0,0,59,0,11,9,29,54,0,84,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.76E2,3.9E1,1.56E2,2E1,3.1E1,8E0,7.9E1,7.7E1,1.3E1,7E0,1E1,2.1E1,3E0,5E0,3.2E1,4.7E1,5E1,2.7E1,6E0,7E0,3E0,4E0,7E0,3E0,1.5E1,6E0,3E0,2E0,2.7E1,5E0,3.5E1,1.2E1,2.6E1,2.4E1,2.5E1,2E0,2E0,2E0,5E0,2E0,3E0,3E0,2.1E1,6E0,2E0,3E0,4E0,3.1E1,4E0,8E0,1.6E1,1E1,2.1E1,3E0,2.1E1,4E0,1.9E1,2E0,2E0,4E0,2E0,2E0,2.3E1,8E0,3E0,5E0,1.4E1,2E0,4E0,6E0,1.8E1,3E0,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[7.874234E-3,-1.0717477E-1,4.6521157E-1,-1.982936E-1,9.943495E-2,1.02683544E-1,7.759888E-1,-1.3372196E-1,-6.0561585E-1,2.801086E-1,-2.3861834E-1,-1.4555064E-1,1.8935364E-1,4.1564103E-2,4.7552985E-1,-1.7659279E-2,-9.404589E-2,-7.024602E-3,-6.612021E-1,1.8482257E-1,3.9656345E-2,-7.397643E-2,-5.8301306E-1,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.4021983E-1,9.948502E-3,2.6430182E-2,-1.4144325E-1,2.806465E-2,-2.1147303E-2,-3.7808225E-2,2.9104125E-1,-3.4760606E-2,5.0945976E-3,-1.12476826E-1,-1.1272882E-2,-3.41051E-2,9.7476035E-2,1.4251037E-2,-8.572106E-2,-2.9373407E-1,9.719249E-2,-1.7345269E-1,5.5071485E-4,3.198904E-1,-1.1351033E-2,3.5261977E-2,-1.3395034E-1,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,35,37,-1,-1,-1,39,-1,-1,41,43,-1,-1,45,47,-1,49,-1,-1,51,-1,53,55,57,59,-1,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1257111E1,3.2389693E0,4.7649326E0,3.0488906E0,3.303937E0,4.7936463E-1,1.3339043E-1,9.1138387E-1,3.148079E-1,1.6327975E0,1.0342052E0,4.6653435E-2,1.7866707E-1,0E0,1.7190337E-2,0E0,5.248641E-1,0E0,2.9516697E-2,7.331557E-1,0E0,1.08073495E-1,8.598232E-2,0E0,0E0,0E0,6.9084704E-2,0E0,0E0,5.2640927E-1,3.790502E-1,0E0,0E0,1.658467E-1,1.6110414E-1,0E0,1.4867976E-2,0E0,0E0,1.5488528E-2,0E0,2.9356018E-1,1.905942E-2,1.5623116E-1,6.481768E-2,0E0,1.2719929E-1,0E0,4.8637956E-2,9.308532E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,18,18,19,19,21,21,22,22,26,26,29,29,30,30,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,46,46,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,36,38,-1,-1,-1,40,-1,-1,42,44,-1,-1,46,48,-1,50,-1,-1,52,-1,54,56,58,60,-1,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7987691E0,1.0648859E3,1.5262272E3,5.0446276E7,7.294459E7,1.0437421E8,2.98476E0,1.295E3,6.856226E-1,6.2150537E3,2.09E3,7.44E2,1.5675428E5,4.1564103E-2,3.145611E6,-1.7659279E-2,5.3287085E2,-7.024602E-3,9.180692E7,2.249985E5,3.9656345E-2,1.1341179E3,1.4233672E8,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.398624E8,9.948502E-3,2.6430182E-2,6.7841024E10,3.0143394E7,-2.1147303E-2,-3.7808225E-2,7.91144E5,2.3276744E5,5.0945976E-3,3.7387386E-1,-1.1272882E-2,-3.41051E-2,1.144E3,1.4251037E-2,6.0130353E0,7.273E3,9.4390506E5,2.9673992E9,5.5071485E-4,1E0,-1.1351033E-2,2.6387034E5,9.9764734E4,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"split_indices":[43,53,4,46,46,7,58,2,28,4,0,0,29,0,1,0,53,0,7,34,0,34,46,0,0,0,7,0,0,32,46,0,0,1,34,0,58,0,0,0,0,54,9,33,5,0,102,0,29,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.7E2,4.2E1,1.18E2,5.2E1,2E1,2.2E1,1.03E2,1.5E1,3.4E1,1.8E1,5E0,1.5E1,1.6E1,6E0,1.4E1,8.9E1,2E0,1.3E1,3E1,4E0,1.3E1,5E0,3E0,2E0,3E0,1.2E1,2E0,4E0,6.4E1,2.5E1,6E0,7E0,2E1,1E1,2E0,1.1E1,2E0,3E0,4E0,8E0,4.8E1,1.6E1,1.9E1,6E0,2E0,1.8E1,2E0,8E0,8E0,3E0,2E0,2E0,1.6E1,3.2E1,2E0,1.4E1,4E0,1.5E1,3E0,3E0,3E0,1.5E1,3E0,5E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.3249438E-2,-1.2597813E-1,3.6839122E-1,-3.741495E-1,-6.773875E-2,5.7275885E-1,-2.5949249E-2,-4.510603E-1,-5.7005443E-2,-1.6321269E-1,4.9919948E-2,2.9108667E-1,7.353367E-1,1.081298E-1,-2.378517E-2,-4.51294E-3,-4.978342E-1,1.5801355E-3,-4.8320345E-3,-1.921461E-2,-1.2809235E-1,1.7340758E-1,-1.6564597E-1,3.5022026E-1,-1.1079086E-3,4.0072575E-2,4.0726957E-1,-5.00003E-2,3.2730332E-1,-8.940148E-3,-5.212322E-1,-7.0953816E-3,-1.7170624E-1,2.1817629E-1,-2.174709E-2,-2.512538E-1,1.0451725E-1,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,4.3051414E-2,1.952225E-2,5.8277114E-3,-1.2334295E-2,-5.556018E-1,-1.2370801E-1,4.7271628E-2,-9.585648E-2,-2.4668616E-1,3.1131923E-2,2.7143374E-1,6.132134E-4,-4.048703E-3,-1.6482335E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,65,67,69,71,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.865889E0,2.272738E0,3.8026128E0,7.0604134E-1,1.4684818E0,1.2055569E0,1.1077032E0,3.5608482E-1,2.9706666E-2,5.3508985E-1,1.5992881E0,2.492423E-1,2.136507E-1,5.015763E-1,0E0,0E0,5.8598995E-2,0E0,0E0,0E0,3.3651924E-1,3.3674383E-1,5.350784E-1,1.20595574E-1,0E0,0E0,1.174857E-1,1.1404645E-1,4.39198E-2,0E0,3.405571E-2,1.1996572E-1,2.4365234E-1,3.003564E-1,1.7932303E-2,2.712661E-1,2.0998505E-1,0E0,0E0,0E0,0E0,0E0,1.23745585E-2,0E0,0E0,0E0,3.083849E-2,2.4845354E-2,9.429864E-2,2.3359656E-1,2.8410673E-2,1.0068264E-1,1.6567254E-1,0E0,0E0,6.312257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,42,42,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,66,68,70,72,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.4862166E1,6.627576E7,1.7497415E-3,5.3287085E2,3.5073215E3,5.7006E4,3.7453184E-3,9.096081E7,9.4418E4,3.1821228E7,5.2E1,1.0096877E8,1.0252E4,-2.378517E-2,-4.51294E-3,6.519E3,1.5801355E-3,-4.8320345E-3,-1.921461E-2,4.06E2,2.8314E4,4.4E1,1.075839E9,-1.1079086E-3,4.0072575E-2,1.5605432E9,3.9880952E-1,1E0,-8.940148E-3,2.0521739E0,1.369E3,9.258696E0,5.716463E2,1.3717402E0,7.6599895E9,1.2633E4,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,2E0,1.952225E-2,5.8277114E-3,-1.2334295E-2,1.0595825E3,1.36135E5,9.087618E6,1.1947E4,2.4734788E7,8.5093097E2,5.0691235E6,6.132134E-4,-4.048703E-3,1.8159722E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"split_indices":[53,59,46,40,53,53,10,59,7,1,46,3,52,9,0,0,9,0,0,0,0,9,3,7,0,0,7,58,8,0,57,2,55,53,43,5,2,0,0,0,0,0,8,0,0,0,34,1,33,9,12,4,46,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.58E2,4.6E1,2.9E1,1.29E2,3E1,1.6E1,2.3E1,6E0,7.1E1,5.8E1,1.2E1,1.8E1,1.3E1,3E0,3E0,2E1,2E0,4E0,8E0,6.3E1,3.7E1,2.1E1,1E1,2E0,1.3E1,5E0,8E0,5E0,2E0,1.8E1,1.7E1,4.6E1,3E1,7E0,1.6E1,5E0,8E0,2E0,2E0,3E0,3E0,5E0,3E0,2E0,3E0,1.5E1,5E0,1.2E1,2.4E1,2.2E1,7E0,2.3E1,5E0,2E0,1.3E1,3E0,3E0,2E0,3E0,2E0,9E0,6E0,3E0,2E0,9E0,3E0,3E0,2.1E1,7E0,1.5E1,5E0,2E0,7E0,1.6E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[6.0231384E-2,-6.6708066E-2,4.1548002E-1,-1.802126E-1,9.4408296E-2,6.121976E-1,4.159176E-2,-1.3381815E-1,-4.551602E-1,-2.3027116E-1,1.6258606E-1,2.7157372E-1,6.904492E-1,1.487131E-1,-1.9334765E-2,-3.371152E-1,-7.73753E-2,-1.0917176E-2,-2.8615614E-2,-1.011948E-1,-2.631605E-2,7.940243E-2,3.767569E-1,1.6384088E-2,2.4758964E-3,8.306936E-1,4.7551715E-1,3.2327557E-1,1.1648051E-2,-9.826521E-3,-2.175756E-2,-1.1204586E-1,1.6198908E-1,-1.3703959E-1,7.871614E-4,1.0334385E-1,-7.2968313E-3,4.7740215E-1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,-8.063122E-2,5.84258E-3,-3.1697118E-1,-6.857089E-2,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.4921291E-1,-1.0298919E-2,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,-1,35,37,-1,-1,39,41,43,45,-1,-1,47,49,51,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,57,-1,59,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.911444E0,2.6869793E0,3.7991571E0,1.0361109E0,1.3877299E0,7.1874046E-1,9.523439E-1,8.3451545E-1,2.3604941E-1,3.861003E-1,8.7867534E-1,9.7789764E-2,4.188528E-1,3.7905708E-1,0E0,1.4759302E-1,5.16084E-1,0E0,0E0,4.008031E-2,0E0,2.2414063E-1,3.420025E-1,0E0,0E0,4.281807E-2,4.2568207E-2,1.4053464E-1,1.0971589E-1,0E0,0E0,4.504485E-1,6.2172145E-2,1.6357675E-2,0E0,1.8398082E-1,0E0,1.7374277E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1939041E-2,0E0,2.7805746E-2,2.6995105E-1,0E0,0E0,0E0,0E0,2.1720147E-1,1.2581149E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,37,37,45,45,47,47,48,48,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,-1,36,38,-1,-1,40,42,44,46,-1,-1,48,50,52,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,58,-1,60,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0601392E3,2.1836595E8,6.627576E7,8.93182E5,1E0,1.2737E4,5.7006E4,1.042E3,4E0,7.6499896E9,1.068371E3,3.5E1,1.5213319E0,1E0,-1.9334765E-2,8.601996E-1,1E0,-1.0917176E-2,-2.8615614E-2,2.3276744E5,-2.631605E-2,4.759155E1,7.9762895E6,1.6384088E-2,2.4758964E-3,1.929E3,9.299267E9,3.6392304E-1,6.331E3,-9.826521E-3,-2.175756E-2,1.5497297E1,6.98247E5,1.1534339E9,7.871614E-4,7.183439E0,-7.2968313E-3,8.383825E1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,1.181E3,5.84258E-3,3.5E1,5.974511E0,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.8613E4,2.770862E8,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"split_indices":[53,7,46,30,102,2,10,2,8,5,53,3,58,8,0,28,89,0,0,34,0,57,46,0,0,0,12,28,9,0,0,59,29,7,0,54,0,59,0,0,0,0,0,0,0,0,0,10,54,0,0,0,0,9,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.45E2,5.1E1,8.5E1,6E1,3.3E1,1.8E1,7.4E1,1.1E1,1E1,5E1,7E0,2.6E1,1.5E1,3E0,1.5E1,5.9E1,5E0,6E0,8E0,2E0,3.7E1,1.3E1,5E0,2E0,1.4E1,1.2E1,6E0,9E0,8E0,7E0,5.2E1,7E0,6E0,2E0,3.4E1,3E0,9E0,4E0,8E0,6E0,1E1,2E0,2E0,4E0,5E0,4E0,8E0,4.4E1,3E0,4E0,4E0,2E0,2.4E1,1E1,5E0,4E0,2E0,3E0,2E0,6E0,1.7E1,2.7E1,1E1,1.4E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[6.2760315E-3,-8.942039E-2,3.3899674E-1,-3.464915E-1,-2.9866556E-2,-3.944495E-2,5.6220204E-1,-7.096159E-2,-4.272631E-1,-3.1877894E-2,-1.0096922E-2,-2.7154645E-1,1.3294205E-1,3.9848673E-1,3.567469E-2,-8.072219E-3,6.935243E-3,-2.918099E-1,-2.5652623E-2,-1.33155E-1,6.0559504E-2,-4.3664877E-3,-2.650756E-2,1.6947778E-2,8.673523E-3,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,5.782375E-2,-1.6802189E-1,2.2789724E-1,-4.5959535E-3,3.984736E-3,-5.162517E-2,6.689567E-3,-5.874931E-3,-2.1503107E-1,-5.4912195E-3,7.9473734E-2,3.573293E-1,-3.5994206E-2,2.3827569E-1,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,27,-1,29,31,-1,-1,-1,33,-1,-1,-1,-1,35,37,39,41,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5575504E0,2.4391828E0,3.907806E0,6.373303E-1,1.6124437E0,7.563031E-1,5.1900864E-1,1.857129E-1,1.5725279E-1,0E0,1.1210039E0,3.4842473E-1,2.9128677E-1,7.89125E-2,0E0,0E0,0E0,4.2960286E-2,0E0,3.2268065E-1,8.950907E-1,0E0,0E0,0E0,3.954031E-2,0E0,0E0,0E0,0E0,1.3033742E-1,3.0443656E-1,4.1344917E-1,4.6612126E-1,0E0,1.9462278E-2,0E0,0E0,1.0997474E-1,4.840184E-2,1.0358285E-1,2.309668E-2,5.0568026E-1,1.8324465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,17,17,19,19,20,20,24,24,29,29,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,28,-1,30,32,-1,-1,-1,34,-1,-1,-1,-1,36,38,40,42,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7108864E8,1.4133928E1,1E0,2.862988E5,3.673257E0,3.067586E5,1.5435694E9,5.34E2,3.203027E7,-3.1877894E-2,3.3969492E2,2.565E3,1E0,1.4008022E11,3.567469E-2,-8.072219E-3,6.935243E-3,1.3448806E8,-2.5652623E-2,2.0589474E1,1.10597E5,-4.3664877E-3,-2.650756E-2,1.6947778E-2,4.078932E5,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,7.269755E0,1.7869681E6,6.185678E2,3.1982856E3,3.984736E-3,1.743E3,6.689567E-3,-5.874931E-3,1.5373E4,7.343501E-1,4.0759E4,4.3411046E8,9.518453E6,1.4139E4,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"split_indices":[7,59,102,29,54,29,7,0,46,0,53,0,8,32,0,0,0,7,0,59,30,0,0,0,29,0,0,0,0,54,51,53,4,0,0,0,0,9,28,30,7,46,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.59E2,4.5E1,2.9E1,1.3E2,1.7E1,2.8E1,7E0,2.2E1,3E0,1.27E2,7E0,1E1,1.6E1,1.2E1,5E0,2E0,1.1E1,1.1E1,4.6E1,8.1E1,5E0,2E0,3E0,7E0,4E0,1.2E1,9E0,2E0,7E0,3.9E1,2.2E1,5.9E1,3E0,4E0,5E0,2E0,3E1,9E0,1.1E1,1.1E1,5.3E1,6E0,2E0,2E0,1.3E1,1.7E1,7E0,2E0,5E0,6E0,9E0,2E0,1.8E1,3.5E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.2222775E-2,-9.139186E-2,4.9998534E-1,-5.1995587E-2,-4.944162E-1,6.0979265E-1,-1.5150189E-2,-1.3674228E-1,1.0816623E-1,3.8813213E-3,-6.330518E-1,8.045889E-1,3.992678E-1,-3.3645314E-1,-1.01294234E-1,2.436709E-1,-8.044677E-2,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,-3.9906213E-1,-5.7864846E-3,-5.545319E-2,-2.0736766E-1,-8.008159E-2,3.159065E-1,-1.8992147E-1,-6.997158E-3,-2.360766E-2,-1.1299601E-2,-5.494948E-3,-2.1435164E-1,-2.4299191E-1,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.7862383E-1,4.077045E-1,-5.167938E-3,-1.228408E-2,-5.0322257E-2,4.2618223E-2,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,37,39,41,43,-1,-1,45,47,49,-1,-1,-1,51,53,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.480311E0,2.8481483E0,2.6842346E0,2.2679396E0,1.3240938E0,7.0399E-1,0E0,7.3756933E-1,1.5019631E0,0E0,4.9333286E-1,1.782751E-2,2.6048756E-1,1.7048275E-1,4.4557756E-1,8.2043767E-1,1.9969293E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.9644938E-2,0E0,5.306874E-1,1.2524438E-1,5.6175735E-2,2.8003812E-1,2.552101E-2,3.65381E-2,0E0,0E0,2.7235347E-1,1.0511035E-1,6.087303E-2,0E0,0E0,0E0,1.4527509E-1,1.0674596E-1,0E0,0E0,2.577142E-2,9.493223E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,23,23,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,39,39,40,40,43,43,44,44],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,38,40,42,44,-1,-1,46,48,50,-1,-1,-1,52,54,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,1.650269E10,1.2055723E8,6.978001E2,1.8159722E-1,8.812601E-1,-1.5150189E-2,1.142E3,3.291238E7,3.8813213E-3,5.535E3,1.929E3,1.7595541E0,1.3448806E8,1.06403265E1,1E0,2.1845242E3,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,1.22E2,-5.7864846E-3,1.9540394E7,9.744047E0,9.31E2,1.068371E3,5.5053414E9,7.951E3,-2.360766E-2,-1.1299601E-2,1.1888112E-1,4.22631E5,4.187504E7,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.617623E5,2.9E1,-5.167938E-3,-1.228408E-2,2.5821698E0,7.157903E7,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"split_indices":[53,5,46,53,28,58,0,2,46,0,9,0,58,7,55,102,4,0,0,0,0,0,0,10,0,46,54,0,53,5,9,0,0,58,29,33,0,0,0,34,8,0,0,54,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.8E2,2.7E1,1.65E2,1.5E1,2.4E1,3E0,1.08E2,5.7E1,3E0,1.2E1,1.1E1,1.3E1,1.5E1,9.3E1,3.3E1,2.4E1,5E0,7E0,8E0,3E0,3E0,1E1,1.1E1,4E0,6.6E1,2.7E1,6E0,2.7E1,9E0,1.5E1,6E0,5E0,5.1E1,1.5E1,2.1E1,6E0,3E0,3E0,1.2E1,1.5E1,5E0,4E0,8E0,7E0,1.9E1,3.2E1,6E0,9E0,1.8E1,3E0,1E1,2E0,1.2E1,3E0,2E0,6E0,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.469039E-3,-6.84523E-2,5.0491726E-1,-1.8420036E-1,8.1181556E-2,6.261244E-1,3.6942E-2,-2.2080618E-1,7.735341E-2,-3.1455643E-2,2.747695E-1,3.8490146E-2,4.343899E-1,5.4492303E-3,-3.6699688E-3,-1.701605E-1,-4.6023417E-1,-3.2597054E-2,1.7538356E-2,-2.7009997E-1,4.3159403E-2,1.4272796E-1,3.6220354E-1,7.502832E-3,2.3337092E-2,-2.6047137E-1,-8.6393684E-2,-3.3155903E-2,-2.9781944E-1,3.291199E-4,-5.0207926E-3,-1.3054396E-1,-2.0009596E-2,-3.049531E-3,2.1293099E-1,1.6921525E-1,-5.2545854E-4,2.1539481E-2,2.3263586E-1,-3.6815163E-1,-1.5577061E-1,-1.2372288E-2,-4.669773E-2,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,-5.099285E-2,7.831989E-2,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,-1,31,33,35,37,-1,-1,39,41,-1,43,-1,-1,45,-1,47,49,51,-1,-1,53,55,57,-1,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009293E0,3.3230407E0,1.3381958E0,1.0511143E0,1.8402805E0,2.5670147E-1,5.9022054E-2,1.0736556E0,4.4741017E-1,9.7320604E-1,3.0619597E-1,0E0,6.5217495E-2,0E0,0E0,5.8270144E-1,3.7153912E-1,3.1251714E-2,0E0,2.0223719E-1,3.3013782E-1,5.8757484E-2,8.452511E-2,0E0,0E0,3.710723E-1,2.7460602E-1,0E0,4.2746305E-2,0E0,0E0,6.497468E-2,0E0,1.3663186E-1,6.1325133E-2,2.643928E-2,0E0,0E0,1.6941309E-2,5.598998E-2,1.3381499E-1,0E0,1.9927172E-1,0E0,0E0,0E0,0E0,1.089835E-1,1.0501201E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28,31,31,33,33,34,34,35,35,38,38,39,39,40,40,42,42,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,-1,32,34,36,38,-1,-1,40,42,-1,44,-1,-1,46,-1,48,50,52,-1,-1,54,56,58,-1,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,2.2896422E6,9.298568E7,2.096851E3,1.537424E0,8.812601E-1,1.2055723E8,5.0446276E7,1.0625348E8,5.6E0,8.57703E2,3.8490146E-2,4.11964E5,5.4492303E-3,-3.6699688E-3,4.9658173E2,5.511724E0,5.9527163E3,1.7538356E-2,4.3927447E11,5.4732365E9,1.1216892E3,6.518218E6,7.502832E-3,2.3337092E-2,1.822764E4,1E0,-3.3155903E-2,2.0529972E1,3.291199E-4,-5.0207926E-3,6.4336755E8,-2.0009596E-2,7.9472524E-1,3.9952212E3,8.3154816E2,-5.2545854E-4,2.1539481E-2,7.8114974E-1,4.3117376E8,2.524E3,-1.2372288E-2,1.3474202E8,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,1.03235014E-1,9.445656E6,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"split_indices":[53,51,46,4,43,58,46,46,7,55,53,0,30,0,0,4,54,34,0,32,5,4,46,0,0,34,65,0,57,0,0,7,0,40,4,4,0,0,28,5,2,0,7,0,0,0,0,58,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.9E2,2.3E1,1.07E2,8.3E1,1.8E1,5E0,9.4E1,1.3E1,5.3E1,3E1,8E0,1E1,3E0,2E0,7.9E1,1.5E1,1E1,3E0,1.2E1,4.1E1,1.3E1,1.7E1,2E0,8E0,3.7E1,4.2E1,5E0,1E1,7E0,3E0,7E0,5E0,3.3E1,8E0,1.1E1,2E0,9E0,8E0,1.7E1,2E1,7E0,3.5E1,7E0,3E0,4E0,3E0,2.1E1,1.2E1,6E0,2E0,6E0,5E0,6E0,2E0,1.2E1,5E0,1.7E1,3E0,1.2E1,2.3E1,4E0,1.7E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-2.0219442E-2,-1.16520904E-1,2.8449756E-1,-3.0797613E-1,-5.9537902E-2,9.543412E-2,5.2078587E-1,-3.3963862E-1,7.4714734E-3,-2.8888127E-2,-3.689822E-1,2.1040678E-1,-8.497771E-2,3.1196967E-2,3.1238148E-1,-2.7664992E-1,-2.8590444E-2,-1.8818352E-1,6.0974874E-2,-4.8011044E-1,-2.6513548E-3,1.2447058E-2,4.99297E-3,2.9411163E-2,-1.22154895E-2,5.4155197E-3,1.8623266E-2,-3.116507E-1,-1.2212595E-1,-3.6836183E-1,-1.402538E-1,3.1627953E-2,3.3315614E-1,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,-3.2800922E-1,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,-1.5982786E-1,-2.9887237E-2,1.3616896E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.107501E0,1.705395E0,2.1839733E0,5.781839E-1,1.1559219E0,6.194103E-1,3.51089E-1,3.682499E-1,0E0,1.6290425E0,3.58294E-1,6.1777413E-2,2.3528156E-1,0E0,9.8772764E-2,1.18784666E-1,0E0,3.0240762E-1,5.7256365E-1,1.4015281E-1,0E0,0E0,0E0,2.9680826E-2,0E0,0E0,0E0,5.1579952E-2,1.1075765E-2,5.4989576E-2,1.3172704E-1,4.3493733E-1,5.3087175E-2,0E0,0E0,0E0,0E0,0E0,3.895402E-2,0E0,0E0,0E0,0E0,0E0,8.195257E-2,2.843659E-1,3.718426E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,23,23,27,27,28,28,29,29,30,30,31,31,32,32,38,38,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.5497297E1,1.3453049E3,4.5342876E3,9.996903E7,2.68E4,8.812601E-1,3.013037E6,7.4714734E-3,3.2384683E2,1.4539318E-1,1.5331E4,6.079755E0,3.1196967E-2,1.5545E4,1.3448806E8,-2.8590444E-2,8.2955E4,9.467578E6,2.1897722E10,-2.6513548E-3,1.2447058E-2,4.99297E-3,5.5566853E-1,-1.22154895E-2,5.4155197E-3,1.8623266E-2,3.7453184E-3,9.279E3,2.700379E2,2.0589474E1,6.5963456E5,6.2357716E-2,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,5.46562E5,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,1.2705392E1,1.617E3,7.6760286E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"split_indices":[43,59,53,4,46,9,58,29,0,53,39,9,54,0,9,7,0,1,1,5,0,0,0,28,0,0,0,59,9,4,59,29,39,0,0,0,0,0,12,0,0,0,0,0,57,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.57E2,4.9E1,3.5E1,1.22E2,2.8E1,2.1E1,3.3E1,2E0,1.12E2,1E1,1.7E1,1.1E1,1.2E1,9E0,2.8E1,5E0,4E1,7.2E1,7E0,3E0,1.1E1,6E0,7E0,4E0,3E0,6E0,2.2E1,6E0,7E0,3.3E1,6.6E1,6E0,3E0,4E0,4E0,3E0,2E0,2E1,4E0,2E0,2E0,5E0,3E0,3E1,4.2E1,2.4E1,2E0,4E0,1.7E1,3E0,2.1E1,9E0,3.5E1,7E0,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.1936052E-2,-1.5182267E-1,1.5760238E-1,-1.214642E-1,-3.187172E-2,1.07080944E-1,3.2834876E-2,-1.8811886E-1,6.19821E-2,-8.8205084E-2,1.7667614E-1,-3.115475E-1,-1.0234942E-1,1.5685846E-1,-1.4203048E-1,-1.3068446E-2,-2.3196185E-2,-3.587127E-2,2.4979593E-1,-2.4225932E-1,-2.5490196E-2,-2.0880654E-2,-1.4597873E-1,3.4969583E-2,3.1181702E-1,1.3681039E-2,-1.3338024E-2,-9.279402E-2,4.1761484E-2,2.3741236E-2,-1.5669461E-2,9.474784E-2,3.6113718E-1,5.085498E-3,-2.820379E-1,-8.3250865E-2,8.308102E-2,-1.855153E-1,-1.1149166E-2,-5.6521293E-2,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,-2.4819752E-2,-8.349065E-3,6.2872446E-3,1.8276206E-4,-3.816804E-2,4.864078E-3,1.2524661E-1,-6.326331E-3,3.7593523E-1,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,-1,-1,-1,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.107609E0,1.8600852E0,2.331173E0,1.4911933E0,0E0,1.1708164E0,0E0,9.103031E-1,6.6102564E-1,2.6339516E-1,9.880699E-1,4.0248227E-1,1.8889087E-1,4.235136E-1,2.2991076E-1,0E0,8.525165E-2,2.97587E-1,7.771559E-1,2.8676605E-1,0E0,1.3659878E-1,1.8450612E-1,2.3734151E-1,9.613538E-2,8.857786E-3,0E0,4.3926656E-2,3.2743193E-2,7.4582696E-2,0E0,1.6022247E-1,5.321765E-2,5.2886583E-2,9.587407E-2,5.3622194E-2,1.212705E-1,8.459413E-2,9.3104824E-2,2.9008571E-2,0E0,0E0,0E0,0E0,0E0,2.897961E-2,0E0,0E0,0E0,5.3322386E-2,0E0,4.675585E-2,0E0,3.5073757E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,45,45,49,49,51,51,53,53],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,-1,-1,-1,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.0728035E-1,1.3917024E8,1.1038146E0,1.0648859E3,-3.187172E-2,4.2447827E2,3.2834876E-2,6.794883E4,2.2882867E5,2.5546576E4,1E0,2.47565E6,8.623692E0,3.851071E7,4.1808826E-1,-1.3068446E-2,2.3739047E-1,1.019731E0,1.0861458E3,2.9377524E2,-2.5490196E-2,3.8E1,2.4079119E1,2.0601392E3,3.353193E6,4E1,-1.3338024E-2,6.741443E-1,4.35E2,8.836347E7,-1.5669461E-2,4.6E1,4.8420677E-1,1.4416069E9,4.2E1,1.4178E4,7.47E2,3.8718798E6,8.2092184E7,4.83E9,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,9.576E3,-8.349065E-3,6.2872446E-3,1.8276206E-4,9.31E2,4.864078E-3,6.98247E5,-6.326331E-3,1E0,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"split_indices":[40,46,35,53,0,53,0,51,34,34,102,29,55,12,28,0,39,35,53,34,0,3,57,53,1,3,0,28,0,49,0,3,39,5,3,9,0,46,7,5,0,0,0,0,0,9,0,0,0,0,0,29,0,84,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.26E2,9.1E1,1.2E2,6E0,8.4E1,7E0,8.8E1,3.2E1,2.2E1,6.2E1,3.5E1,5.3E1,2.2E1,1E1,5E0,1.7E1,1.6E1,4.6E1,2.8E1,7E0,1.9E1,3.4E1,1.3E1,9E0,5E0,5E0,8E0,9E0,1.4E1,2E0,2E1,2.6E1,4E0,2.4E1,1.2E1,7E0,2.6E1,8E0,9E0,4E0,4E0,5E0,3E0,2E0,5E0,3E0,2E0,7E0,8E0,6E0,1.8E1,2E0,2.4E1,2E0,2E0,2E0,1.7E1,7E0,7E0,5E0,5E0,2E0,4E0,2.2E1,4E0,4E0,3E0,6E0,3E0,2E0,2E0,6E0,1.2E1,6E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-1.6992955E-2,-8.875833E-2,2.2156855E-1,-6.0390815E-2,-5.453842E-1,6.565264E-2,3.5614845E-1,-8.633612E-2,3.0628803E-1,-3.5136674E-2,-2.5970185E-1,1.033548E-2,1.3761056E-3,4.3397513E-3,3.885008E-1,-1.4680506E-1,-1.1059224E-2,2.477328E-2,1.7252595E-3,-9.932688E-4,-2.007134E-2,-5.6842774E-2,1.1758452E-1,2.6560768E-1,4.3975347E-1,-1.07489556E-1,-2.2993198E-2,-1.1149532E-1,1.1053811E-1,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,-2.6751718E-1,-5.8981713E-2,-4.527185E-2,-3.2934055E-1,1.45027E-2,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,29,-1,-1,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4779978E0,1.9877847E0,9.62826E-1,1.4344559E0,2.6657557E-1,2.1661456E-1,1.8128777E-1,6.3109255E-1,6.166359E-1,0E0,1.7431176E-1,0E0,1.2169773E-1,0E0,2.1767378E-2,9.350836E-1,7.817284E-1,0E0,4.614174E-2,0E0,0E0,3.312366E-2,1.0790542E-2,2.8682768E-2,3.3157587E-2,5.27603E-1,0E0,4.9001646E-1,6.9180393E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0392995E-2,2.4817911E-1,1.5262431E-1,9.5196724E-2,2.9514036E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,39,39,40,40,41,41,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,30,-1,-1,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5064244E0,1.4233672E8,8.57703E2,4.9192E4,2.849513E3,1.1244E4,1E0,6.978001E2,7.424212E-1,-3.5136674E-2,2.09E3,1.033548E-2,1.4872659E1,4.3397513E-3,2.8626094E5,4.462437E7,1E0,2.477328E-2,1.3174513E4,-9.932688E-4,-2.007134E-2,5.9857685E6,3.261056E8,1.2519051E7,3.007E3,1.142E3,-2.2993198E-2,3.4521E4,3.212485E11,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,4.543028E2,1.06403265E1,2.7577372E11,5.4656273E-1,2.4394053E9,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"split_indices":[43,46,53,2,4,9,102,53,43,0,0,0,57,0,29,46,102,0,4,0,0,33,7,33,0,2,0,10,32,0,0,0,0,0,0,0,0,0,0,4,55,32,28,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.55E2,4.6E1,1.47E2,8E0,2.2E1,2.4E1,1.38E2,9E0,4E0,4E0,6E0,1.6E1,3E0,2.1E1,7.6E1,6.2E1,5E0,4E0,2E0,2E0,1.1E1,5E0,8E0,1.3E1,6.9E1,7E0,3.4E1,2.8E1,2E0,2E0,6E0,5E0,2E0,3E0,6E0,2E0,1.1E1,2E0,1.5E1,5.4E1,2.7E1,7E0,2.1E1,7E0,1.3E1,2E0,3.6E1,1.8E1,7E0,2E1,2E0,5E0,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.02255E-2,-6.558096E-2,3.4569636E-1,-1.2843427E-2,-4.0469694E-1,5.4956263E-1,4.3343384E-2,-1.403701E-1,1.0433215E-1,2.2149876E-2,-4.8705584E-1,6.838387E-1,3.7301254E-1,1.4347048E-1,-1.6691085E-2,-2.8438172E-1,-7.497648E-2,3.6531553E-2,2.410501E-1,-3.8605474E-3,5.6571406E-3,-5.657334E-1,-2.4888176E-1,9.962667E-3,7.5234216E-1,6.084204E-3,1.9794445E-2,7.620034E-2,1.50580425E-2,-3.147017E-1,-2.6331637E-3,-3.2688044E-2,-1.6758075E-1,3.0311698E-3,1.3236392E-1,-3.7357313E-4,2.712043E-1,-6.3281095E-1,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,2.171496E-3,8.731073E-3,-1.6129533E-2,-5.054191E-3,2.5078287E-2,-1.376335E-1,-2.738632E-3,-1.0184393E-2,6.306253E-2,-5.9866663E-2,2.3515062E-1,-2.9690387E-3,1.6760947E-2,1.7714213E-1,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,-1,41,-1,-1,43,-1,45,-1,47,49,51,53,-1,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,-1,-1,65,67,69,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7954755E0,3.3212526E0,2.565043E0,2.4354134E0,8.896537E-1,3.3758068E-1,7.454521E-1,7.132385E-1,7.791066E-1,5.6962486E-2,2.5271702E-1,2.7266836E-1,5.7875514E-2,1.438675E-1,0E0,1.5053928E-1,2.0990708E-1,1.8580317E-1,2.1208572E-1,0E0,0E0,2.0361853E-1,6.189382E-2,0E0,4.8636913E-2,0E0,0E0,9.129316E-2,0E0,3.965044E-2,0E0,2.3988187E-1,6.710768E-2,1.699124E-1,3.1248683E-1,0E0,1.11125946E-1,2.6826096E-1,0E0,0E0,0E0,0E0,0E0,3.272756E-2,0E0,0E0,0E0,2.1187598E-1,8.620715E-2,0E0,0E0,1.1212682E-1,1.685599E-1,7.086277E-2,0E0,0E0,4.61289E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,24,24,27,27,29,29,31,31,32,32,33,33,34,34,36,36,37,37,43,43,47,47,48,48,51,51,52,52,53,53,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,-1,42,-1,-1,44,-1,46,-1,48,50,52,54,-1,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,-1,-1,66,68,70,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6499896E9,6.627576E7,3.7763092E2,3.3478114E-1,8E0,5.7006E4,1.278E3,1.5022863E0,1.776E3,9.006135E0,2.6E1,1.625E3,8.129243E5,-1.6691085E-2,4.5715973E4,5.531948E8,1.8331E4,7.555544E2,-3.8605474E-3,5.6571406E-3,1.2728E4,1.2187347E3,9.962667E-3,2.2047469E4,6.084204E-3,1.9794445E-2,1.0252E4,1.50580425E-2,3.4322312E-1,-2.6331637E-3,7.269755E0,1.2421726E7,4.278E3,2.4794E4,-3.7357313E-4,1.5489E4,8.260109E9,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,4E0,8.731073E-3,-1.6129533E-2,-5.054191E-3,5.1E2,2.037744E10,-2.738632E-3,-1.0184393E-2,4.541179E1,5.382E3,5.586E3,-2.9690387E-3,1.6760947E-2,7.4353185E0,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"split_indices":[53,5,46,53,28,8,10,2,42,0,55,3,0,29,0,34,5,10,4,0,0,9,4,0,4,0,0,9,0,39,0,54,46,2,9,0,9,5,0,0,0,0,0,8,0,0,0,0,32,0,0,57,2,2,0,0,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.85E2,4.1E1,1.61E2,2.4E1,2.4E1,1.7E1,7.7E1,8.4E1,4E0,2E1,1.2E1,1.2E1,1.4E1,3E0,2.3E1,5.4E1,5.7E1,2.7E1,2E0,2E0,1.4E1,6E0,2E0,1E1,2E0,1E1,1.1E1,3E0,2E1,3E0,3.8E1,1.6E1,4.3E1,1.4E1,3E0,2.4E1,1.1E1,3E0,4E0,2E0,8E0,2E0,7E0,4E0,1.8E1,2E0,2.5E1,1.3E1,5E0,1.1E1,2.2E1,2.1E1,9E0,5E0,1.2E1,1.2E1,2E0,9E0,5E0,2E0,1.5E1,1E1,1E1,3E0,1.8E1,4E0,9E0,1.2E1,5E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-5.735387E-3,-6.410505E-2,2.797867E-1,-1.760073E-2,-3.442871E-1,5.04675E-1,2.686096E-2,-1.2676698E-1,1.1911592E-1,-4.2207205E-1,5.1267102E-2,6.152395E-1,1.3419471E-2,1.5410598E-1,-1.9511065E-1,-2.5507975E-1,-8.848773E-2,2.6512688E-1,4.352096E-2,-3.2537884E-1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,2.4225692E-1,2.5578894E-2,-4.040385E-2,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,-4.242069E-2,-1.8737647E-1,3.087647E-1,4.6277814E-2,-5.1333107E-2,1.4060691E-1,-3.4747878E-3,-3.5448268E-1,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.1831463E-3,-1.2673235E-1,-2.0977704E-1,8.250273E-4,2.3295416E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,5.527623E-2,-1.189838E-1,1.975704E-1,-1.7819712E-2,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,45,47,49,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,65,-1,-1,-1,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5344255E0,2.287163E0,2.0341249E0,2.2836764E0,7.9084253E-1,3.1127357E-1,5.384738E-1,3.9492452E-1,7.390245E-1,2.7931428E-1,7.600727E-2,2.1988153E-1,0E0,1.2976074E-1,1.931273E-1,6.892848E-2,2.9727066E-1,2.0536327E-1,4.308317E-1,1.00349784E-1,0E0,0E0,0E0,0E0,0E0,8.028874E-2,1.2697376E-2,1.2392008E-2,0E0,0E0,0E0,1.8929411E-1,9.967518E-2,7.952249E-2,1.2078894E-2,1.7966965E-1,2.110849E-1,0E0,1.2337124E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1306114E-1,7.93834E-2,1.05109274E-1,0E0,1.7682552E-2,0E0,0E0,0E0,2.9296512E-2,1.0169087E-1,1.4039552E-1,1.7305832E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,27,27,31,31,32,32,33,33,34,34,35,35,36,36,38,38,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,46,48,50,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,66,-1,-1,-1,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6599895E9,6.627576E7,5.242538E2,1.5593E4,8.812601E-1,1.3917024E8,1.056E3,1.26881E5,1.3917024E8,3.729811E6,8E0,1.3419471E-2,5.1E1,2.565E3,7.107292E6,5.4127317E10,6.3300834E0,1.5181E4,1.9E1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,1.739E3,7.7573473E9,1.3174513E4,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,1.0466548E1,1E0,4.361E3,3.098E3,5.7277904E0,3.2395E4,-3.4747878E-3,3.20084E6,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.998024E-1,2.124077E1,7.2899837E0,8.250273E-4,9.607843E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,2.7060036E11,5.36E3,1.7790995E0,4.0953446E8,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"split_indices":[53,5,46,53,9,58,46,2,30,46,1,8,0,3,0,46,32,54,10,3,0,0,0,0,0,0,5,4,0,0,0,55,105,2,2,54,9,0,7,0,0,0,0,0,0,58,59,54,0,58,0,0,0,32,2,58,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.75E2,3.5E1,1.51E2,2.4E1,1.8E1,1.7E1,8.4E1,6.7E1,2E1,4E0,1.1E1,7E0,1.1E1,6E0,1.8E1,6.6E1,2.2E1,4.5E1,1.6E1,4E0,2E0,2E0,5E0,6E0,6E0,5E0,4E0,2E0,1.1E1,7E0,4.6E1,2E1,1.8E1,4E0,2.3E1,2.2E1,2E0,1.4E1,4E0,2E0,2E0,3E0,2E0,2E0,3E1,1.6E1,1.8E1,2E0,1.3E1,5E0,2E0,2E0,9E0,1.4E1,1.6E1,6E0,2E0,1.2E1,2.2E1,8E0,7E0,9E0,1.1E1,7E0,1.1E1,2E0,5E0,4E0,9E0,5E0,7E0,9E0,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-2.2588685E-2,-1.4545202E-1,1.1374607E-1,-8.6952515E-2,-3.9759496E-1,-3.064367E-2,2.3289914E-1,-1.2433577E-1,1.7574582E-1,-5.7842267E-1,-9.222219E-2,-6.9834334E-3,-1.8266106E-2,6.878721E-2,3.451957E-1,-2.7263525E-1,-8.135537E-2,-5.932932E-4,1.1667584E-2,-6.281898E-1,-9.885883E-3,-1.32481875E-2,1.0622433E-1,-2.1569806E-1,3.714026E-2,-4.582277E-2,1.5392454E-1,2.7393732E-2,4.457966E-1,-1.735199E-1,-1.8242102E-2,-2.038975E-1,-3.7180442E-2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,-3.8278066E-2,8.7990396E-2,-4.001707E-3,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,-1.1330966E-1,2.5172912E-2,3.078277E-1,-2.0457634E-1,-1.095509E-3,-1.3675371E-1,-1.4804438E-2,3.1190168E-2,-1.298265E-1,-8.181965E-2,8.646828E-3,-2.6838033E-4,1.4474735E-1,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,61,-1,63,65,-1,67,-1,69,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6011605E0,1.6322045E0,1.7619938E0,9.3515986E-1,1.1062734E0,3.806838E-1,1.0082908E0,5.004853E-1,1.4725864E-1,1.1519289E-1,3.504039E-1,4.224278E-1,0E0,2.4123535E-1,1.0428362E0,1.1729789E-1,3.442828E-1,0E0,0E0,1.5132141E-1,0E0,0E0,1.4618695E-2,8.2455575E-2,1.4909932E-1,5.0066087E-2,1.4621168E-1,4.4345292E-1,8.724737E-2,4.6982825E-2,0E0,7.2891474E-2,3.1442845E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6498463E-1,1.1525272E-1,1.6504118E-2,0E0,0E0,0E0,0E0,9.535315E-2,0E0,1.2730312E-1,2.6230484E-2,0E0,1.0405913E-2,0E0,2.094729E-1,1.0832605E-1,9.527324E-2,0E0,3.1808536E-2,1.478824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,39,39,40,40,41,41,46,46,48,48,49,49,51,51,53,53,54,54,55,55,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,62,-1,64,66,-1,68,-1,70,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3905947E-1,1.4446067E10,4.4589956E5,1E0,1.5461028E12,1.4146589E8,1.2838955E3,6.639351E2,4.22631E5,8.92851E6,1.244125E3,2.853839E4,-1.8266106E-2,1.3012535E1,1E0,8.3949E4,4.7095413E2,-5.932932E-4,1.1667584E-2,6.1826086E0,-9.885883E-3,-1.32481875E-2,1.5011E4,1.338E3,1.0176675E0,1.1350955E0,6.695E3,1E0,3.1E1,1E0,-1.8242102E-2,7.1399055E6,7.11E2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,4.8008E4,1.2364354E3,4.1376923E2,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,2.5591E4,2.5172912E-2,5.86883E6,1.4008755E2,-1.095509E-3,6.1E1,-1.4804438E-2,6.38E2,9.388849E-1,6.8992513E-1,8.646828E-3,2.7953E4,1.1946838E7,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"split_indices":[40,5,29,89,32,46,53,34,29,1,34,34,0,57,102,12,4,0,0,54,0,0,9,2,40,58,2,8,3,105,0,46,0,0,0,0,0,0,0,2,4,53,0,0,0,0,2,0,1,34,0,3,0,0,28,28,0,9,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.12E2,1.01E2,9.2E1,2E1,4.6E1,5.5E1,8.1E1,1.1E1,1.2E1,8E0,4.4E1,2E0,2.3E1,3.2E1,1.7E1,6.4E1,3E0,8E0,1E1,2E0,4E0,4E0,7E0,3.7E1,1E1,1.3E1,8E0,2.4E1,1E1,7E0,1.6E1,4.8E1,7E0,3E0,2E0,2E0,2E0,5E0,1.5E1,2.2E1,8E0,2E0,1.1E1,2E0,2E0,6E0,1.4E1,1E1,8E0,2E0,1.1E1,5E0,2.8E1,2E1,1.3E1,2E0,9E0,1.3E1,6E0,2E0,4E0,2E0,2E0,8E0,2E0,6E0,9E0,2E0,2.4E1,4E0,1.8E1,2E0,1E1,3E0,7E0,2E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.9344994E-3,-8.175323E-2,2.3140544E-1,-4.4790298E-2,-4.8490012E-1,2.5693094E-2,3.847721E-1,-9.1722965E-2,4.259129E-2,-2.0991474E-1,-3.360973E-2,2.7427834E-2,-5.6275822E-2,3.184738E-2,3.1256717E-1,-1.0761265E-1,1.2113636E-1,-2.8683331E-3,2.0815976E-1,-1.3282049E-2,-1.7001278E-3,7.0364326E-2,-1.4656629E-1,4.4856225E-3,1.5982723E-2,-2.2537349E-1,-8.0327906E-2,8.497628E-3,1.8191043E-3,-6.4350456E-2,7.170837E-2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,-3.6205932E-2,-4.63827E-3,-1.3423674E-2,5.86282E-3,-1.1076284E-1,1.3306548E-2,-1.0665464E-1,-2.4310345E-2,1.2974514E-1,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,-1,-1,33,35,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6567056E0,2.3029141E0,1.5360355E0,5.94458E-1,5.9041786E-1,1.0017112E0,3.4057522E-1,3.3096892E-1,3.8445777E-1,6.7638606E-2,0E0,0E0,2.3900099E-1,0E0,5.3549767E-2,2.6464677E-1,2.4824701E-2,1.9260496E-1,3.0953288E-2,0E0,0E0,8.459512E-2,1.8438596E-1,0E0,0E0,7.871467E-2,1.9214371E-1,0E0,0E0,7.697916E-2,1.0903385E-1,0E0,0E0,0E0,0E0,0E0,4.2805668E-2,0E0,0E0,1.03736766E-1,1.2159997E-1,6.966427E-2,5.4593235E-2,3.1910963E-2,5.4574564E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26,29,29,30,30,36,36,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,-1,-1,34,36,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.8000048E10,1E0,2.4277832E6,6.0671224E7,1.7727281E2,1.2117E4,1E0,8.373168E5,5.441E3,-3.360973E-2,2.7427834E-2,3.1047E4,3.184738E-2,5.8E2,6.554877E-4,3.5E1,1.0415988E0,1.521636E5,-1.3282049E-2,-1.7001278E-3,3.5073215E3,5.785788E-1,4.4856225E-3,1.5982723E-2,3E2,3.7504044E5,8.497628E-3,1.8191043E-3,2E0,8.3154816E2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,6.804E3,-4.63827E-3,-1.3423674E-2,1.2599119E-3,5.948464E0,2.7954546E-1,2.1845242E3,8.412314E0,1.579932E5,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"split_indices":[4,5,102,49,46,34,9,89,29,2,0,0,10,0,0,40,3,40,34,0,0,53,43,0,0,0,49,0,0,8,4,0,0,0,0,0,9,0,0,40,54,58,4,55,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.55E2,4.8E1,1.43E2,1.2E1,2.1E1,2.7E1,9.3E1,5E1,6E0,6E0,2E0,1.9E1,4E0,2.3E1,8.7E1,6E0,4E1,1E1,4E0,2E0,8E0,1.1E1,2E0,2.1E1,1.5E1,7.2E1,3E0,3E0,2.2E1,1.8E1,7E0,3E0,5E0,3E0,4E0,7E0,5E0,1E1,1.9E1,5.3E1,8E0,1.4E1,7E0,1.1E1,5E0,2E0,7E0,1.2E1,1.2E1,4.1E1,5E0,3E0,1E1,4E0,5E0,2E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.2111462E-2,-1.5450773E-1,1.031491E-1,-1.09187275E-1,-4.5132187E-1,-2.8939374E-2,1.8243721E-1,-2.6891872E-1,-6.3274145E-2,-2.8736094E-2,-2.1376169E-1,-1.9380113E-1,1.3079939E-2,-5.3318623E-2,2.8723615E-1,-3.048581E-1,-3.2565857E-3,-2.1935193E-1,-2.3797065E-2,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,-1.15509786E-1,4.264821E-2,1.4384617E-1,-1.3136774E-1,1.7543119E-1,4.10858E-1,-3.4710836E-1,-7.3455116E-3,-2.187251E-3,-2.6718003E-1,-4.2986523E-2,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.1813E-2,-8.5979775E-2,1.9637176E-3,1.3061595E-2,-5.950475E-2,-2.1748714E-2,2.684922E-1,1.24782175E-1,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.991617E-2,-1.04395926E-1,8.574969E-3,4.3539573E-2,-5.5793906E-3,-9.2947256E-4,-9.633975E-2,3.2059713E-3,4.982372E-3,1.4664561E-2,1.5088551E-1,3.8726836E-2,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,-1,53,55,-1,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,63,65,-1,67,-1,-1,69,-1,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4133608E0,1.2017343E0,1.2096798E0,5.8435357E-1,2.704537E-1,3.0733958E-1,1.7942166E0,1.1027694E-1,3.9527723E-1,0E0,1.3330251E-2,9.3468994E-2,1.4180462E-1,3.728331E-1,6.189995E-1,4.515779E-2,0E0,9.6432924E-2,2.5955042E-1,0E0,0E0,0E0,0E0,1.570262E-2,1.1871672E-1,7.939862E-2,3.5887626E-1,9.847349E-2,1.3197088E-1,3.6898494E-2,0E0,0E0,1.9970298E-2,2.5871733E-1,0E0,0E0,0E0,6.611198E-2,9.313557E-3,0E0,0E0,7.5622745E-2,0E0,1.8241048E-2,3.9083302E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.800409E-1,1.35196E-1,0E0,3.6446035E-2,0E0,0E0,6.822489E-2,0E0,0E0,0E0,3.4981847E-2,1.2126244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,37,37,38,38,41,41,43,43,44,44,51,51,52,52,54,54,57,57,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,-1,54,56,-1,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,64,66,-1,68,-1,-1,70,-1,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8601307E-2,1.8000048E10,6.787619E2,1.142E3,1.527638E12,2.853839E4,1E0,1.3448806E8,4.059859E-5,-2.8736094E-2,2.906E3,3.6E1,1.7218965E2,2E0,1.3173E4,2.23695E5,-3.2565857E-3,1.7727281E2,3.3E1,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,3.0345E4,4.6E1,4.078932E5,8.476884E-1,9.686559E6,4.747723E6,3.7790768E2,-7.3455116E-3,-2.187251E-3,6.802721E-3,4.454691E5,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.751E3,1.251E3,1.9637176E-3,1.3061595E-2,1.257013E0,-2.1748714E-2,8.366892E4,8.442676E0,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.5E1,5.979E3,8.574969E-3,2.702E3,-5.5793906E-3,-9.2947256E-4,2.012E3,3.2059713E-3,4.982372E-3,1.4664561E-2,7.595543E-1,1.3235667E9,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"split_indices":[39,5,53,2,32,34,102,7,39,0,0,3,53,8,2,12,0,34,8,0,0,0,0,9,3,29,35,52,33,34,0,0,58,49,0,0,0,9,0,0,0,40,0,34,55,0,0,0,0,0,0,3,2,0,2,0,0,0,0,0,0,28,5,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,9.2E1,1.14E2,8.1E1,1.1E1,4.3E1,7.1E1,1.7E1,6.4E1,6E0,5E0,8E0,3.5E1,2.2E1,4.9E1,1.4E1,3E0,1.2E1,5.2E1,2E0,3E0,5E0,3E0,6E0,2.9E1,6E0,1.6E1,2.7E1,2.2E1,1E1,4E0,3E0,9E0,4.9E1,3E0,4E0,2E0,2.4E1,5E0,4E0,2E0,1.4E1,2E0,8E0,1.9E1,2E0,2E1,3E0,7E0,5E0,4E0,2.1E1,2.8E1,4E0,2E1,3E0,2E0,1.1E1,3E0,2E0,6E0,1.4E1,5E0,1E1,1.1E1,2E1,8E0,1.3E1,7E0,5E0,6E0,9E0,5E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.2747186E-3,-5.4162722E-2,1.8111008E-1,-3.108007E-2,-4.4845343E-1,7.1374275E-2,3.275183E-1,-9.3600295E-2,1.1067671E-1,-2.5658727E-2,-6.88157E-3,1.2687185E-1,-2.088327E-2,2.448392E-3,3.571717E-1,-2.3487736E-1,-5.843745E-2,1.925611E-1,-3.9538275E-2,9.78743E-3,3.3138932E-3,-7.214252E-2,6.806273E-3,1.9431083E-2,1.0387621E-2,-1.5502349E-1,-1.7014932E-2,5.5956807E-5,-1.2483689E-1,2.5137913E-1,-3.1633927E-3,-1.7023765E-1,8.103208E-2,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,2.859666E-2,-1.6140592E-1,6.8674185E-3,1.9025224E-1,2.8979616E-2,-1.5382758E-2,-9.495701E-4,-2.327769E-2,1.6913544E-1,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9040357E0,1.3148481E0,6.996037E-1,1.2422698E0,1.18970394E-1,1.4089137E-1,1.3821435E-1,4.6814162E-1,5.365249E-1,0E0,0E0,6.0488462E-2,1.007224E-1,0E0,1.7639875E-2,1.198957E-1,3.0683744E-1,4.4052112E-1,2.659302E-1,0E0,0E0,2.7664423E-2,0E0,0E0,0E0,2.1903068E-2,0E0,2.2701421E-1,1.7931145E-1,3.7047946E-1,0E0,1.6969696E-1,8.6647436E-2,0E0,0E0,0E0,0E0,0E0,1.13961965E-1,6.886476E-2,1.559542E-2,2.4640507E-1,0E0,0E0,0E0,1.2072772E-2,1.1905044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,25,25,27,27,28,28,29,29,31,31,32,32,38,38,39,39,40,40,41,41,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4436649E0,1.4233672E8,1.4008022E11,1.0648859E3,3.7284137E2,2.7883E4,1E0,6.126268E0,2.249985E5,-2.5658727E-2,-6.88157E-3,3.5037E4,5.53839E5,2.448392E-3,2.7E1,7.0712406E5,7.269755E0,9.040637E7,9.875E3,9.78743E-3,3.3138932E-3,1.009E3,6.806273E-3,1.9431083E-2,1.0387621E-2,2.44412E3,-1.7014932E-2,1.053E3,2.1102592E8,5.902126E3,-3.1633927E-3,4.6E1,3.858136E6,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,1E0,7.3376025E6,1.9519106E-1,2.8880222E6,2.8979616E-2,-1.5382758E-2,-9.495701E-4,4.895358E8,8.51E2,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"split_indices":[43,46,32,53,59,9,102,55,34,0,0,30,29,0,8,29,54,46,9,0,0,0,0,0,0,48,0,2,7,4,0,3,48,0,0,0,0,0,89,33,40,33,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.89E2,1.45E2,4.4E1,1.38E2,7E0,2.6E1,1.8E1,9.6E1,4.2E1,5E0,2E0,1.6E1,1E1,2E0,1.6E1,1.8E1,7.8E1,2.7E1,1.5E1,6E0,1E1,8E0,2E0,1.1E1,5E0,1.2E1,6E0,4.2E1,3.6E1,2.2E1,5E0,7E0,8E0,6E0,2E0,4E0,8E0,5E0,3.7E1,2.8E1,8E0,2E1,2E0,3E0,4E0,4E0,4E0,3.3E1,4E0,2.2E1,6E0,2E0,6E0,9E0,1.1E1,2E0,2E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.826838E-3,-6.721812E-2,1.9046006E-1,-7.924094E-2,2.6099376E-2,5.0246296E-3,2.6243427E-1,-2.5692692E-1,-4.875577E-2,-3.755318E-2,3.6572933E-3,2.1474557E-1,1.9588318E-2,-4.4567264E-3,-2.904459E-1,-3.542672E-2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,2.688706E-1,1.3754945E-1,-4.928484E-3,-3.0891788E-1,-1.2508754E-1,4.5123346E-2,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,-1.1071097E-1,1.8496023E-2,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,-1,21,23,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,-1,-1],"loss_changes":[2.477284E0,1.1091111E0,6.6600955E-1,7.729075E-1,0E0,4.7609035E-2,1.5403104E-1,8.979154E-2,6.413361E-1,3.7017375E-2,0E0,8.418608E-2,0E0,0E0,2.8120995E-2,8.8675094E-1,0E0,0E0,0E0,1.0490298E-2,3.518188E-2,0E0,1.5468597E-2,1.0648471E-1,4.4222242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2309903E-1,2.3909211E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,14,14,15,15,19,19,20,20,22,22,23,23,24,24,32,32,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,-1,22,24,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,7.8040985E2,5.5200674E3,2.6099376E-2,6.2958473E4,1.2790948E0,5.4E0,1.4146589E8,4.71E2,3.6572933E-3,9.6494156E-1,1.9588318E-2,-4.4567264E-3,4.146E3,5.3287085E2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,7.810231E4,1.5593E4,-4.928484E-3,1.2187347E3,6.35E2,1.2241036E12,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,4.2E1,2.122531E6,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"split_indices":[43,57,4,49,0,34,36,55,46,0,0,58,0,0,30,53,0,0,0,34,9,0,4,2,32,0,0,0,0,0,0,0,3,48,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.46E2,4.9E1,1.44E2,2E0,1.4E1,3.5E1,2E1,1.24E2,9E0,5E0,2.8E1,7E0,4E0,1.6E1,1.21E2,3E0,4E0,5E0,1.5E1,1.3E1,2E0,1.4E1,5.7E1,6.4E1,4E0,1.1E1,5E0,8E0,1.2E1,2E0,3E0,5.4E1,5.9E1,5E0,4.8E1,6E0,3.8E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.4262974E-3,-8.005432E-2,1.4594802E-1,-2.888352E-1,-4.6757724E-2,-1.8107321E-2,2.4755892E-1,-4.6657827E-3,-3.4083346E-1,-1.0410578E-1,1.8106598E-2,2.52526E-2,-7.499621E-2,7.3558204E-2,3.0511466E-1,-1.0108946E-2,-1.9372413E-2,-1.6241044E-1,-4.8085254E-2,6.1593957E-2,-1.17093466E-1,8.193563E-2,-1.12674244E-1,-1.9586612E-3,1.11724325E-1,3.5032213E-1,1.3559058E-1,-6.2673695E-2,-2.1433787E-1,7.613574E-2,-8.7488696E-2,8.202132E-2,-6.9475058E-3,-5.7769872E-2,-1.4384934E-2,7.815317E-5,7.870827E-3,-1.6748264E-1,-8.807537E-3,7.7555277E-3,2.0889724E-5,3.6501428E-1,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,-3.6418545E-3,-1.605696E-1,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,-1.1916163E-1,1.1715029E-1,1.5941553E-2,1.3056229E-3,-9.284757E-2,-2.2564225E-1,-2.0749527E-3,-2.448091E-3,2.7077156E-3,3.8617578E-1,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,53,-1,55,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,67,69,71,-1,73,75,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4140606E0,8.554564E-1,1.2713884E0,1.3789642E-1,4.0734783E-1,9.553926E-1,4.482491E-1,0E0,1.6353011E-2,1.8163341E-1,3.1294593E-1,0E0,1.7479275E-1,6.096188E-2,2.2073102E-1,0E0,0E0,1.3158071E-1,1.5839781E-1,1.786837E-1,1.20575935E-1,3.8199473E-2,1.2945855E-1,0E0,5.294875E-2,5.433297E-2,3.3570766E-2,4.5042265E-2,5.2512467E-2,7.294734E-2,5.8373034E-2,8.402106E-2,0E0,3.5133395E-2,0E0,0E0,0E0,9.929475E-2,2.6866937E-2,0E0,0E0,4.9077034E-2,0E0,0E0,0E0,0E0,2.0661239E-2,1.6366512E-2,0E0,0E0,0E0,0E0,5.8940127E-2,1.0018352E-1,5.5888433E-2,0E0,2.880384E-2,6.1261773E-2,0E0,0E0,0E0,1.6714334E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,37,37,38,38,41,41,46,46,47,47,52,52,53,53,54,54,56,56,57,57,61,61],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,54,-1,56,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,68,70,72,-1,74,76,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.6770158E-4,1E0,6.051637E-1,3.3030225E2,6.948189E7,6.201765E5,-4.6657827E-3,1.57525425E1,2.0562818E4,9.959017E0,2.52526E-2,9.31E2,1.0363E4,5.03433E0,-1.0108946E-2,-1.9372413E-2,1.0363E4,2.0589474E1,6.579211E9,3.2776688E7,8.319244E8,1.5574268E7,-1.9586612E-3,3.4E1,5.2E1,1.5E1,5.3943863E0,3.62043E1,9.075745E7,5.948464E0,4.9212202E-1,-6.9475058E-3,2E1,-1.4384934E-2,7.815317E-5,7.870827E-3,5.840393E-1,6.804E3,7.7555277E-3,2.0889724E-5,1.0528413E10,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,6.948189E7,1.2421726E7,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,2E0,1.4265206E1,2.1150263E-1,1.3056229E-3,1.5996247E1,2.6023E4,-2.0749527E-3,-2.448091E-3,2.7077156E-3,1.4456249E7,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"split_indices":[53,40,102,28,53,7,49,0,57,34,55,0,0,9,58,0,0,9,59,5,33,12,1,0,3,8,8,54,59,7,54,58,0,3,0,0,0,43,9,0,0,12,0,0,0,0,7,46,0,0,0,0,8,57,39,0,59,10,0,0,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.24E2,7.5E1,1.6E1,1.08E2,2.9E1,4.6E1,4E0,1.2E1,5.7E1,5.1E1,2E0,2.7E1,1.2E1,3.4E1,5E0,7E0,2.7E1,3E1,3.9E1,1.2E1,5E0,2.2E1,3E0,9E0,2.6E1,8E0,1E1,1.7E1,7E0,2.3E1,3.6E1,3E0,1E1,2E0,3E0,2E0,1.4E1,8E0,6E0,3E0,2.4E1,2E0,5E0,3E0,4E0,6E0,1.3E1,4E0,4E0,3E0,7E0,1.6E1,2.3E1,1.3E1,3E0,7E0,9E0,5E0,5E0,3E0,2.1E1,3E0,2E0,4E0,7E0,6E0,1.3E1,3E0,1.3E1,1E1,8E0,5E0,2E0,5E0,2E0,7E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.6450368E-2,-1.1724766E-1,1.2559639E-1,-7.511801E-2,-3.363559E-1,1.8126132E-1,-2.3499241E-1,-1.6364163E-1,-1.0940317E-2,-1.8125798E-1,-4.375745E-1,1.2667201E-1,3.6933804E-1,-4.659123E-2,-2.6914816E-2,-2.2536351E-1,-1.00311846E-1,7.6694086E-2,-4.540101E-2,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,-1.531599E-2,2.0203556E-1,4.1224214E-1,2.9349497E-3,1.9489948E-3,-3.9977366E-3,-2.5541717E-1,2.9264723E-3,1.9217292E-2,-1.2399441E-1,1.2778491E-1,-2.9145163E-2,-7.644804E-2,6.597971E-2,5.250523E-2,-1.4877042E-1,2.3447502E-1,7.5913966E-4,2.1494467E-2,8.123851E-3,-2.7855477E-1,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,-7.0292644E-2,7.10427E-2,8.864286E-3,-2.6523983E-3,1.1110839E-3,1.0315416E-2,-1.1217816E-1,-7.6833105E-4,1.1495934E-1,9.3826205E-2,-1.8227108E-2,-1.1465208E-2,-1.2050162E-3,2.9752174E-1,1.3666672E-1,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,-1,-1,65,-1,-1,-1,-1,67,69,-1,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.913867E0,1.1932194E0,1.6282953E0,6.345087E-1,2.5036097E-1,6.616366E-1,6.333728E-1,1.611085E-1,2.0279819E-1,5.5682838E-2,9.7411156E-2,5.916339E-1,1.740191E-1,2.7933039E-2,0E0,2.1270418E-1,7.315171E-2,1.06465176E-1,1.7102996E-1,0E0,0E0,0E0,0E0,1.8883207E-1,2.3487675E-1,2.0117044E-2,0E0,0E0,0E0,1.0595679E-1,0E0,2.1876162E-2,3.615102E-2,2.7157709E-2,1.046974E-2,1.1895977E-1,4.5869254E-2,4.262839E-2,6.952983E-2,1.5049982E-1,3.6906146E-2,0E0,0E0,4.3109298E-2,0E0,0E0,0E0,0E0,1.5443023E-2,1.32172E-2,0E0,0E0,0E0,9.080692E-2,1.4392808E-1,0E0,9.912826E-3,3.9224803E-2,1.2763529E-2,0E0,0E0,1.4601588E-2,1.00580364E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,48,48,49,49,53,53,54,54,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,-1,-1,66,-1,-1,-1,-1,68,70,-1,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.352552E9,1.2055723E8,5.162122E2,3.4E1,5.0561914E3,1.5119754E0,1.822764E4,5.415E4,9.285558E4,1.2E1,1E0,1.02E2,5.4438036E9,-2.6914816E-2,1E0,2.1768606E5,6.0511904E0,2.2583E4,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,3.20566E7,6.1108776E7,9.298568E7,2.9349497E-3,1.9489948E-3,-3.9977366E-3,6.1E1,2.9264723E-3,8.637E3,2.8774E4,2E0,4.411222E6,1.0092749E8,8.3154816E2,1.4364E4,5.840393E-1,8.383825E1,1.91E2,2.1494467E-2,8.123851E-3,3.5623631E0,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,5.6058385E6,6.314179E2,8.864286E-3,-2.6523983E-3,1.1110839E-3,9.075745E7,2.0053221E6,-7.6833105E-4,4.7E1,9.355911E-1,5.314E3,-1.1465208E-2,-1.2050162E-3,6.518218E6,2.6678162E9,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"split_indices":[53,5,46,56,3,53,58,34,30,34,8,102,3,5,0,105,29,54,10,0,0,0,0,51,46,46,0,0,0,3,0,9,9,8,33,7,4,10,43,59,10,0,0,57,0,0,0,0,33,4,0,0,0,7,49,0,8,28,9,0,0,46,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.31E2,7.8E1,1.11E2,2E1,6.8E1,1E1,4.6E1,6.5E1,9E0,1.1E1,5.4E1,1.4E1,7E0,3E0,2.2E1,2.4E1,1.8E1,4.7E1,7E0,2E0,7E0,4E0,1.9E1,3.5E1,1.2E1,2E0,2E0,5E0,2E1,2E0,4E0,2E1,1.2E1,6E0,3.7E1,1E1,1.3E1,6E0,3E1,5E0,1E1,2E0,1.8E1,2E0,2E0,2E0,1E1,1E1,7E0,5E0,4E0,2E0,1.1E1,2.6E1,4E0,6E0,8E0,5E0,3E0,3E0,1.7E1,1.3E1,2E0,3E0,1E1,8E0,7E0,3E0,3E0,4E0,7E0,4E0,1.2E1,1.4E1,4E0,2E0,6E0,2E0,2E0,3E0,7E0,1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.8897833E-2,-6.780625E-2,2.0897096E-1,-1.5521011E-2,-2.9606915E-1,3.5466662E-1,-4.421246E-3,-9.93627E-2,9.932131E-2,-1.4433515E-1,-4.58412E-1,4.7411785E-1,2.1561413E-1,9.5952526E-2,-1.4718074E-1,-2.0804802E-1,-6.7736566E-2,1.9759446E-1,2.8317139E-2,-1.7917114E-1,-4.480132E-2,-5.3120285E-1,-6.104539E-3,5.165622E-1,9.185563E-3,2.6545724E-1,2.4120836E-3,7.911076E-3,-3.163457E-3,-1.2183908E-2,-3.5092747E-4,-2.297472E-1,-1.1178532E-3,1.8829303E-2,-9.4082914E-2,1.3437358E-1,1.6593615E-2,-1.20509595E-1,6.209947E-2,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,-2.9204038E-1,-6.37282E-3,-4.2878713E-2,1.0429371E-1,-4.7475312E-2,-1.8176384E-1,2.5304276E-2,1.6177177E-1,6.3902856E-4,-8.5941E-3,1.6473201E-1,1.7290728E-2,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,-1,-1,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,67,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3322675E0,2.048187E0,1.1519101E0,1.3674978E0,7.220886E-1,2.631929E-1,2.4339132E-1,2.6776582E-1,4.1307282E-1,5.4385334E-2,2.9693174E-1,3.5738707E-2,8.616072E-2,6.653001E-2,9.928414E-2,6.7011595E-2,1.5001667E-1,1.8078548E-1,1.8848129E-1,1.2413353E-2,1.4049292E-2,2.2869587E-2,0E0,3.3781767E-2,0E0,5.2916467E-2,0E0,0E0,1.2511282E-2,0E0,0E0,6.0228944E-2,0E0,8.88534E-2,1.9570062E-1,5.2684218E-2,0E0,5.4782815E-2,1.3511673E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1208367E-2,0E0,1.15634445E-2,1.6972624E-2,1.08784124E-1,5.8658242E-2,1.0622809E-2,1.1181682E-2,0E0,0E0,4.1976526E-2,3.8918935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,28,28,31,31,33,33,34,34,35,35,37,37,38,38,51,51,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,-1,-1,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,68,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.2829284E7,6.627576E7,5.3287085E2,8.67241E9,8.812601E-1,3.511154E7,1.056E3,1.26881E5,2.1845242E3,5.770668E6,3.2707875E8,1.7414924E7,5.1E1,1.4539318E-1,6.1E1,2.5126263E1,1.467574E3,2.035483E6,1.0892118E3,7.6336784E1,6.1826086E0,-6.104539E-3,4.5E1,9.185563E-3,4.747723E6,2.4120836E-3,7.911076E-3,7.358531E3,-1.2183908E-2,-3.5092747E-4,2.23695E5,-1.1178532E-3,6.21E2,9.348848E0,5.9734796E2,1.6593615E-2,9.31E2,2.7E1,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,2.2290532E3,-6.37282E-3,5.093772E-1,3.396881E4,1.1556807E0,7.688406E0,3.7E1,1.617623E5,6.3902856E-4,-8.5941E-3,1.38635E5,1.8454842E-1,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"split_indices":[53,46,46,53,5,58,33,2,30,4,30,33,33,3,39,3,59,4,33,53,57,54,0,3,0,33,0,0,4,0,0,12,0,11,55,53,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,49,0,28,34,43,54,3,34,0,0,11,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.71E2,3.6E1,1.4E2,3.1E1,2.1E1,1.5E1,8.1E1,5.9E1,1.7E1,1.4E1,1E1,1.1E1,9E0,6E0,1.7E1,6.4E1,2.4E1,3.5E1,1.2E1,5E0,1.1E1,3E0,8E0,2E0,8E0,3E0,5E0,4E0,3E0,3E0,1.5E1,2E0,1.5E1,4.9E1,1.8E1,6E0,6E0,2.9E1,8E0,4E0,3E0,2E0,9E0,2E0,6E0,2E0,2E0,6E0,2E0,2E0,8E0,7E0,9E0,6E0,3.3E1,1.6E1,4E0,1.4E1,2E0,4E0,8E0,2.1E1,2E0,6E0,6E0,3E0,4E0,2E0,2E1,1.3E1,1E1,6E0,2E0,2E0,1.2E1,2E0,2E0,6E0,3E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[9.597382E-3,-7.689029E-2,1.3135731E-1,-5.166708E-2,-3.934461E-1,1.9286239E-1,-1.5746945E-1,-1.8152632E-1,-2.5963075E-2,-2.2153566E-2,-5.7512084E-3,1.2324249E-1,4.3318063E-1,-8.0194466E-2,-2.265891E-2,-2.6121805E-3,-2.3640567E-1,5.765824E-4,-9.947969E-2,-1.1233864E-2,1.7927498E-1,5.3216517E-1,2.2989117E-1,-1.9807076E-1,6.597765E-2,-4.3076635E-3,-1.2422029E-2,2.8069925E-2,-1.1729321E-1,6.657329E-2,-1.4218627E-1,9.798071E-2,-6.077645E-2,-1.3545295E-3,1.9635805E-1,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,7.5570874E-2,-6.374473E-2,-1.7920145E-1,-1.4495179E-3,2.901012E-4,4.642252E-3,-8.290076E-2,-1.9911025E-1,1.871269E-3,7.0417197E-3,-5.9855576E-3,-1.9459605E-2,2.7096123E-1,1.4672595E-1,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,63,65,-1,-1,-1,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2320848E0,9.697564E-1,1.5936292E0,3.824837E-1,9.9226475E-2,1.1679718E0,3.4362957E-1,1.2062222E-1,1.912657E-1,0E0,0E0,4.3904662E-1,1.9962025E-1,2.5429142E-1,0E0,0E0,1.3633668E-2,2.3985873E-1,1.9384623E-1,1.0334882E-1,1.5052485E-1,1.5721798E-2,2.7384043E-2,1.3692349E-2,4.27702E-2,0E0,0E0,2.664942E-1,7.050744E-2,9.9109635E-3,5.4363728E-2,1.1121236E-2,3.102351E-2,0E0,1.0094595E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2414542E-1,1.4501756E-1,3.8122535E-2,0E0,0E0,0E0,4.5270823E-2,2.6841223E-2,0E0,0E0,0E0,1.3282825E-2,1.0726178E-1,4.9301982E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,34,34,43,43,44,44,45,45,49,49,50,50,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,64,66,-1,-1,-1,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.2109607E2,8.260109E9,9.298568E7,1.295E3,9.4819355E0,5.08711E3,2.88001E10,5.7625353E-1,1.5989196E7,-2.2153566E-2,-5.7512084E-3,1E0,2.7E1,1.5574268E7,-2.265891E-2,-2.6121805E-3,2.9585883E2,1.193419E1,1.3729923E5,1E0,7.056125E7,9.886E3,1.917786E7,1.15E2,3.9E1,-4.3076635E-3,-1.2422029E-2,5.998024E-1,8.401064E0,2.68E2,1.09137096E8,1.9253E4,2.1845242E3,-1.3545295E-3,1.2951E4,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,4.1376923E2,8.05E2,2.6239506E5,-1.4495179E-3,2.901012E-4,4.642252E-3,3.1274893E2,1.0209414E1,1.871269E-3,7.0417197E-3,-5.9855576E-3,4E1,1.1865996E1,1.4456249E7,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"split_indices":[53,5,46,2,54,53,5,28,46,0,0,102,8,1,0,0,34,55,29,8,7,9,1,10,3,0,0,58,54,0,7,2,4,0,9,0,0,0,0,0,0,0,0,53,0,29,0,0,0,53,55,0,0,0,3,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.23E2,8.7E1,1.15E2,8E0,7.2E1,1.5E1,1.8E1,9.7E1,6E0,2E0,5.7E1,1.5E1,1.3E1,2E0,6E0,1.2E1,7.2E1,2.5E1,1.7E1,4E1,9E0,6E0,7E0,6E0,2E0,1E1,5.9E1,1.3E1,5E0,2E1,5E0,1.2E1,3E0,3.7E1,4E0,5E0,3E0,3E0,2E0,5E0,2E0,4E0,3.9E1,2E1,7E0,6E0,2E0,3E0,1.1E1,9E0,3E0,2E0,4E0,8E0,1.3E1,2.4E1,2.4E1,1.5E1,1.1E1,9E0,4E0,3E0,4E0,7E0,7E0,2E0,4E0,4E0,1.1E1,2E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-1.772178E-2,-7.057884E-2,8.537469E-2,-5.372641E-2,-3.503544E-1,-7.3327735E-2,1.9524743E-1,-1.9434269E-1,-2.6594644E-2,-2.087923E-2,-8.001967E-3,1.7575795E-2,-2.1872297E-1,1.2774226E-1,3.8615E-1,-2.1584399E-1,-2.1231158E-3,2.6761827E-3,-1.002963E-1,-2.3103694E-2,1.2448591E-2,-3.2898784E-1,7.034172E-5,-6.594587E-2,1.6341075E-1,2.4855476E-2,1.1777236E-2,-2.5548145E-1,-6.297792E-3,6.468007E-2,-6.0867704E-2,3.847441E-3,-1.2086692E-1,1.0591442E-1,-6.7531325E-2,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,1.8735889E-1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,1.1528385E-1,-3.7465915E-2,-1.02739446E-1,5.237328E-2,-1.373252E-1,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,-2.5229553E-2,2.3754518E-1,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,-1,-1,41,-1,43,45,-1,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,55,57,59,61,63,-1,-1,-1,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1885403E0,6.6012216E-1,1.3046603E0,5.1750946E-1,4.6478212E-2,4.1356945E-1,5.176964E-1,6.1877012E-2,2.5051618E-1,0E0,0E0,1.9972178E-1,2.9179746E-1,2.4566746E-1,7.394695E-2,3.203535E-2,0E0,3.3489424E-1,1.3130611E-1,1.10328E-1,0E0,9.977138E-2,0E0,1.4139006E-2,1.0379839E-1,0E0,0E0,6.927937E-2,0E0,2.2658505E-1,2.045279E-1,0E0,7.257894E-2,1.1413369E-2,8.147253E-2,0E0,0E0,0E0,0E0,8.3029985E-2,0E0,0E0,0E0,9.5561266E-2,5.3016253E-2,1.7131087E-1,4.7250487E-2,5.080968E-2,0E0,0E0,0E0,0E0,2.8906034E-2,3.853917E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,23,23,24,24,27,27,29,29,30,30,32,32,33,33,34,34,39,39,43,43,44,44,45,45,46,46,47,47,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,-1,-1,42,-1,44,46,-1,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,56,58,60,62,64,-1,-1,-1,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.142E3,6.645232E2,8.307621E7,1.9253E4,1.3448806E8,1.6278354E7,-2.087923E-2,-8.001967E-3,1.9142364E7,1.4539318E-1,7.5773336E7,1.2117E4,2.23695E5,-2.1231158E-3,1.643E4,5.577E3,8.87E2,1.2448591E-2,2.2982106E0,7.034172E-5,1.33511E5,7.4308796E0,2.4855476E-2,1.1777236E-2,3.7790768E2,-6.297792E-3,1.0466548E1,6.3004294E5,3.847441E-3,9.568003E2,1.4783E4,2.9E1,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,7.4009796E1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,5.3055557E1,6.693824E5,4.966E3,1.175E1,1.30468176E11,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,7.7573473E9,4.60675E5,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"split_indices":[53,5,102,2,34,46,2,7,46,0,0,1,39,7,9,12,0,9,9,0,0,54,0,12,54,0,0,34,0,55,29,0,53,9,3,0,0,0,0,59,0,0,0,59,49,2,57,32,0,0,0,0,5,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.43E2,7.3E1,1.36E2,7E0,3E1,4.3E1,2.1E1,1.15E2,4E0,3E0,1.9E1,1.1E1,3.3E1,1E1,1.8E1,3E0,8.3E1,3.2E1,1.7E1,2E0,7E0,4E0,5E0,2.8E1,4E0,6E0,1.1E1,7E0,4.2E1,4.1E1,3E0,2.9E1,4E0,1.3E1,2E0,5E0,3E0,2E0,2.4E1,4E0,4E0,7E0,2.8E1,1.4E1,3E1,1.1E1,2.6E1,3E0,2E0,2E0,2E0,1.1E1,1.4E1,1E1,1.7E1,1.1E1,4E0,1E1,2.5E1,5E0,6E0,5E0,1.3E1,1.3E1,7E0,4E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.1619442E-2,-6.1717395E-2,1.3433132E-1,-2.3387735E-1,-4.0964365E-2,1.77865E-3,2.3689537E-1,1.3427171E-2,-3.110039E-1,-1.2818594E-1,-3.7928703E-3,5.2376132E-2,-8.1358645E-3,2.3620052E-2,1.8211213E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,-1.5684637E-1,1.68092E-2,7.7523957E-3,-1.8851958E-2,-1.6274573E-2,1.226543E-1,2.2858551E-1,6.3767E-2,-1.8596564E-1,-4.8825707E-2,8.553022E-2,-6.1434526E-3,-7.6919543E-3,1.647666E-1,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,1.4144917E-1,1.4252614E-2,4.003073E-4,4.388098E-3,-1.9882518E-1,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,-2.6464459E-2,1.329478E-1,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,-1,-1,-1,-1,27,29,31,-1,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,53,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5730232E0,5.620738E-1,7.473112E-1,3.2842952E-1,4.6638432E-1,2.2247311E-1,3.3355105E-1,3.4519482E-2,7.46156E-2,1.8132824E-1,4.54711E-1,9.848905E-2,0E0,0E0,1.3392115E-1,0E0,0E0,0E0,0E0,1.0415751E-1,8.9438625E-2,2.4376561E-1,0E0,1.8215474E-2,2.6880935E-2,6.499237E-2,1.245489E-2,5.9958518E-2,5.9645988E-2,1.10448785E-2,0E0,2.4641316E-1,2.660959E-1,0E0,0E0,0E0,0E0,1.1662543E-2,0E0,0E0,0E0,2.202785E-2,0E0,0E0,0E0,0E0,0E0,1.3542011E-1,6.190428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,37,37,41,41,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,-1,-1,-1,-1,28,30,32,-1,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,54,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,6.794276E-5,6.48045E6,1.7727281E2,2.802166E2,1.0430015E1,6.677E3,1.94122E5,3.2445145E2,1.01978E5,1.0590684E1,2.806E3,-8.1358645E-3,2.3620052E-2,8.269529E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,2.1974122E5,4.87374E5,4.9192E4,-1.8851958E-2,1.5878E4,2.0837E4,1.980161E0,4.602015E0,6.1E1,3.7E1,3.52777E5,-6.1434526E-3,1E0,1.0670794E10,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,5.649635E0,1.4252614E-2,4.003073E-4,4.388098E-3,1.8E2,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,3.2441288E5,3.9360254E2,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"split_indices":[43,43,49,34,53,55,9,1,34,30,58,2,0,0,28,0,0,0,0,48,1,2,0,10,9,43,54,3,3,1,0,89,12,0,0,0,0,54,0,0,0,11,0,0,0,0,0,29,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.59E2,5.4E1,1.6E1,1.43E2,2.4E1,3E1,4E0,1.2E1,4.2E1,1.01E2,1.9E1,5E0,4E0,2.6E1,2E0,2E0,5E0,7E0,3.5E1,7E0,9.9E1,2E0,1E1,9E0,1.8E1,8E0,2.7E1,8E0,5E0,2E0,9.1E1,8E0,6E0,4E0,7E0,2E0,9E0,9E0,3E0,5E0,2.5E1,2E0,6E0,2E0,2E0,3E0,8.1E1,1E1,4E0,4E0,4E0,5E0,4E0,2.1E1,2.7E1,5.4E1,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.0766466E-3,-4.316265E-2,2.0560837E-1,-1.2818186E-2,-2.3354582E-1,2.703695E-1,-9.532744E-2,-5.3655125E-2,1.0286662E-1,-2.9023632E-1,-1.5101601E-2,2.0894068E-1,4.177894E-1,1.6216272E-2,-1.39497025E-2,-1.1859325E-1,-2.1754269E-2,2.239287E-1,4.9224846E-2,-7.534239E-3,-3.746514E-1,-5.9629036E-3,3.3697663E-3,7.089449E-2,2.384329E-1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,-1.6322732E-1,-5.6480993E-2,2.9111031E-2,-7.753057E-2,1.5182119E-2,4.083155E-3,8.881563E-2,-4.8623074E-2,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,2.5431147E-1,5.2253725E-3,6.213173E-4,-1.8242495E-1,-1.1897854E-2,-5.765586E-3,5.034584E-2,-1.3384877E-1,7.912595E-3,-1.0946924E-1,-9.919967E-3,1.289712E-1,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,37,-1,-1,39,41,-1,-1,-1,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,-1,55,-1,-1,57,59,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.055721E0,1.0079945E0,8.2482004E-1,7.235998E-1,2.9282618E-1,2.2060513E-1,1.7536506E-1,2.3150888E-1,2.487365E-1,1.5920115E-1,6.284737E-2,8.894992E-2,2.0777345E-2,1.1685234E-2,0E0,9.335673E-2,2.2071204E-1,1.2175274E-1,1.1666089E-1,0E0,1.730293E-1,0E0,0E0,3.1039445E-2,1.6396642E-2,0E0,0E0,0E0,0E0,7.328111E-2,4.5573883E-2,1.4861596E-1,1.0183704E-1,0E0,0E0,8.453986E-2,1.6937668E-2,0E0,0E0,0E0,0E0,1.6923666E-2,0E0,0E0,3.164649E-2,3.1010306E-2,0E0,1.0485908E-1,1.7051622E-2,8.5487984E-2,7.8597695E-2,2.3515034E-2,7.874721E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,20,20,23,23,24,24,29,29,30,30,31,31,32,32,35,35,36,36,41,41,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,38,-1,-1,40,42,-1,-1,-1,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,-1,56,-1,-1,58,60,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.0099896E9,1.2055723E8,8.297185E2,1.2728E4,1.4243088E6,2.374331E5,2.1015242E2,1.10597E5,5.033165E7,1.4792529E10,4.0492815E-1,1.4735E4,9.4908775E1,-1.39497025E-2,3.5E1,7.2929355E6,6.675932E1,8.7022424E-1,-7.534239E-3,8.260109E9,-5.9629036E-3,3.3697663E-3,8.601656E6,5.1E1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,7.36E2,3.5223506E5,1.996866E1,4.008916E10,1.5182119E-2,4.083155E-3,9.1014955E10,2.7577372E11,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,1.6503E4,5.2253725E-3,6.213173E-4,1.13002876E-4,6.693824E5,-5.765586E-3,2.579E3,6.09E2,1.438E3,1.8272479E3,5.0895057E10,1E0,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"split_indices":[53,5,46,53,9,29,34,53,30,7,5,28,2,57,0,3,46,59,28,0,5,0,0,1,3,0,0,0,0,30,29,57,32,0,0,32,32,0,0,0,0,2,0,0,39,49,0,2,0,2,4,32,102,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.74E2,4E1,1.51E2,2.3E1,3.3E1,7E0,1.12E2,3.9E1,1.8E1,5E0,2.5E1,8E0,5E0,2E0,3.6E1,7.6E1,1.1E1,2.8E1,8E0,1E1,2E0,3E0,5E0,2E1,2E0,6E0,2E0,3E0,2E1,1.6E1,4E1,3.6E1,6E0,5E0,2E1,8E0,3E0,7E0,3E0,2E0,1.7E1,3E0,2E0,1.8E1,1E1,6E0,3.6E1,4E0,1E1,2.6E1,6E0,1.4E1,2E0,6E0,3E0,1.4E1,2E0,1.6E1,4E0,6E0,1.8E1,1.8E1,2E0,2E0,3E0,7E0,2.4E1,2E0,2E0,4E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[5.1350137E-3,-3.3741128E-2,2.475987E-1,-2.1715815E-1,-6.9968994E-3,3.2988298E-1,2.8091062E-2,-6.1420463E-2,-2.4671237E-1,-9.5820725E-2,3.2276653E-2,2.3344913E-3,3.5703343E-1,-5.512472E-2,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,-3.0382487E-1,-1.4058068E-1,-2.9446453E-2,-2.1209379E-1,5.59181E-2,4.091285E-1,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,-6.627251E-2,-2.2693902E-1,5.2225657E-2,-9.5052674E-2,1.0859418E-2,-3.6253917E-1,2.6486173E-2,1.5046827E-1,2.1380894E-2,7.393621E-3,2.4758913E-3,-8.528379E-2,-3.1659063E-3,-2.504589E-1,-1.3437112E-3,5.355821E-3,-7.7356305E-3,-4.3233983E-2,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,1.0883494E-2,1.4663106E-1,1.9825523E-1,8.6160064E-2,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,59,-1,-1,-1,-1,61,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9783993E0,8.8634837E-1,5.145322E-1,9.070897E-2,5.577526E-1,1.4360905E-1,6.3942134E-2,9.758426E-3,6.0073018E-2,1.4143506E-1,6.5313697E-1,0E0,1.07219934E-1,1.2388382E-2,0E0,0E0,0E0,0E0,3.4586906E-2,1.7105693E-1,1.1748636E-1,3.393597E-1,2.7985966E-1,4.202509E-2,0E0,0E0,0E0,0E0,0E0,4.2199805E-2,3.3204734E-2,4.9204696E-2,3.2862037E-2,3.151442E-2,7.372153E-2,1.4649613E-1,5.6672513E-2,0E0,0E0,0E0,3.002061E-2,0E0,3.334278E-2,0E0,0E0,0E0,2.2537217E-2,0E0,0E0,0E0,0E0,1.6933751E-1,1.3314232E-2,2.8548539E-2,3.267061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,18,18,19,19,20,20,21,21,22,22,23,23,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,42,42,46,46,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,60,-1,-1,-1,-1,62,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7162656E3,2.750917E-4,9.298568E7,2.56917E-2,2.802166E2,4.6111366E1,5.547E3,6.054E3,2.799504E11,1.8716404E10,3.791641E0,2.3344913E-3,4.5E1,4.8008E4,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,6.645232E2,8.1446184E7,4.57E2,8.640031E9,1.5085194E0,3.2707875E8,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,1.5E1,2.2057E4,2.3168E5,3.1E1,8.601996E-1,8.92851E6,7.6599895E9,2.5661135E0,2.1380894E-2,7.393621E-3,2.4758913E-3,3.07E2,-3.1659063E-3,2.0562818E4,-1.3437112E-3,5.355821E-3,-7.7356305E-3,2.7898628E5,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,6.7835927E0,1.743E3,2.6198992E-1,1.51E3,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"split_indices":[53,40,46,59,53,57,9,2,32,32,54,0,3,2,0,0,0,0,34,5,0,5,43,33,0,0,0,0,0,3,12,12,3,28,1,5,43,0,0,0,11,0,34,0,0,0,29,0,0,0,0,54,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.8E2,2.8E1,2.2E1,1.58E2,2E1,8E0,4E0,1.8E1,4.8E1,1.1E2,2E0,1.8E1,4E0,4E0,2E0,2E0,8E0,1E1,2.8E1,2E1,9E0,1.01E2,1.3E1,5E0,2E0,2E0,5E0,5E0,1.6E1,1.2E1,9E0,1.1E1,4E0,5E0,7.8E1,2.3E1,1.1E1,2E0,2E0,1.4E1,2E0,1E1,4E0,5E0,4E0,7E0,2E0,2E0,3E0,2E0,7E1,8E0,1.2E1,1.1E1,5E0,9E0,5E0,5E0,2E0,5E0,4E1,3E1,2E0,6E0,6E0,6E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-6.245155E-3,-2.833662E-2,3.4589776E-1,-7.005383E-2,5.9592E-2,4.2491645E-1,-1.5961174E-3,-4.3598708E-2,-2.8613925E-1,-8.835135E-2,1.3214482E-1,5.290952E-1,1.0459326E-2,-9.431229E-2,1.7313663E-2,-4.463223E-3,-3.3082795E-1,2.9486376E-3,-2.370824E-1,-2.7359384E-3,1.4807577E-1,3.203436E-2,8.785884E-3,-1.4088191E-1,-5.170084E-2,4.7399618E-2,-1.0460413E-1,-2.1237668E-2,-1.0130584E-2,6.996463E-3,-4.852486E-2,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,1.24671705E-1,-2.0670156E-1,-8.551871E-2,5.622916E-3,-6.533785E-2,6.141754E-2,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,-8.802993E-2,6.8448015E-2,1.8364535E-1,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,51,53,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5812309E0,7.0478874E-1,3.730235E-1,7.306783E-1,6.77165E-1,1.050607E-1,0E0,3.6305583E-1,9.1302514E-2,2.8586107E-1,1.3435823E-1,1.5126264E-1,0E0,1.1971778E-1,2.0303094E-1,0E0,5.1597834E-2,1.0845146E-1,1.1068231E-1,0E0,6.660807E-2,0E0,0E0,9.164274E-2,8.737236E-2,1.02572635E-1,6.1553992E-2,0E0,0E0,0E0,2.9702732E-2,0E0,0E0,0E0,1.00164235E-1,4.0271103E-2,6.0178608E-2,0E0,5.5570483E-2,7.76602E-2,0E0,0E0,0E0,0E0,1.725753E-2,3.47196E-2,2.8324187E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,30,30,34,34,35,35,36,36,38,38,39,39,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,52,54,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,1.0648859E3,1.2055723E8,4.934166E7,1E0,3.0291426E-1,-1.5961174E-3,3.3969492E2,8.7793425E-2,5E0,7.340097E7,1.04E3,1.0459326E-2,2.0562818E4,1.2670352E1,-4.463223E-3,3E0,8.78E2,9.875E3,-2.7359384E-3,3.835264E-2,3.203436E-2,8.785884E-3,1.4357987E10,4.914413E-3,1.7328871E6,1.018E3,-2.1237668E-2,-1.0130584E-2,6.996463E-3,2E0,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,2.6673E4,7.3353534E2,5.571429E0,5.622916E-3,3.606519E-1,2.5356756E1,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,9.281264E-2,6.7835927E0,7.1847E4,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"split_indices":[4,53,46,46,102,39,0,53,59,8,7,0,0,34,55,0,8,0,9,0,28,0,0,32,39,48,0,0,0,0,8,0,0,0,10,4,57,0,39,57,0,0,0,0,39,54,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.9E2,1.1E1,1.29E2,6.1E1,9E0,2E0,1.16E2,1.3E1,2E1,4.1E1,5E0,4E0,6.3E1,5.3E1,3E0,1E1,1.3E1,7E0,3E0,3.8E1,3E0,2E0,2.9E1,3.4E1,4.3E1,1E1,4E0,6E0,3E0,1E1,4E0,3E0,5E0,3.3E1,1.2E1,1.7E1,2E0,3.2E1,4E1,3E0,8E0,2E0,4E0,6E0,1.8E1,1.5E1,1E1,2E0,6E0,1.1E1,2.6E1,6E0,3.1E1,9E0,3E0,3E0,1.3E1,5E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.3435227E-3,-3.277699E-2,1.774612E-1,-6.964357E-3,-1.5355606E-1,1.9180963E-2,2.988968E-1,-6.0709182E-2,7.332652E-2,-6.922103E-2,-2.4452102E-1,1.5958549E-1,-3.233358E-2,3.533102E-1,1.318238E-1,-1.3005522E-1,-4.060321E-2,1.0830404E-1,-3.1816754E-2,-1.1131207E-2,-1.2607405E-1,-2.9532254E-1,2.7801134E-3,1.0396815E-3,1.1904416E-2,1.8165827E-2,-1.0879546E-2,3.7512168E-1,6.5092915E-3,2.031597E-3,9.1953445E-3,-1.5207392E-1,-1.9281486E-4,-4.925895E-3,-9.33342E-2,5.5923946E-2,1.5398896E-1,-1.0388468E-1,2.1812472E-2,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,-3.3411142E-1,-2.0121949E-2,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,4.4080522E-2,-6.593762E-2,-4.5851987E-2,-1.4824677E-1,-2.6830027E-3,8.220542E-2,1.7252632E-3,1.8331738E-1,9.3954586E-4,-7.641705E-3,4.75524E-3,-9.375982E-3,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,67,69,71,73,-1,75,-1,77,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4195844E0,5.663384E-1,7.4388015E-1,6.560496E-1,2.2856313E-1,1.353517E-1,1.5342748E-1,1.21597886E-1,2.2778258E-1,5.8043107E-2,2.467323E-1,5.353722E-2,1.3925183E-1,2.644825E-2,2.8130919E-2,5.492565E-2,1.3499385E-1,1.01465404E-1,6.410514E-2,3.194621E-2,4.1086197E-2,4.452479E-2,0E0,0E0,0E0,3.9501768E-2,0E0,4.0338874E-2,0E0,0E0,0E0,3.306678E-2,0E0,1.3446508E-1,6.8816334E-2,7.1661495E-2,7.693821E-2,4.8853584E-2,2.4587668E-2,0E0,0E0,0E0,0E0,0E0,4.43269E-2,1.6359001E-2,0E0,0E0,0E0,0E0,0E0,1.0982905E-1,8.9442335E-2,3.789381E-2,2.6375294E-2,0E0,2.339761E-2,0E0,3.7862718E-2,0E0,0E0,0E0,1.0178003E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,27,27,31,31,33,33,34,34,35,35,36,36,37,37,38,38,44,44,45,45,51,51,52,52,53,53,54,54,56,56,58,58,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,68,70,72,74,-1,76,-1,78,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.075104E7,1E0,5.470729E2,7.6599895E9,1E0,3.3326385E0,1.142E3,2.7883E4,1.0363E4,7.2418816E12,3.5912186E-1,8.129243E5,4.4E1,3.6E1,1.3448806E8,5.549974E8,7.8761675E5,1.6595541E1,1.1484E4,1.0731037E-1,8.67241E9,2.7801134E-3,1.0396815E-3,1.1904416E-2,2.613E3,-1.0879546E-2,2.608036E6,6.5092915E-3,2.031597E-3,9.1953445E-3,1.8E2,-1.9281486E-4,4.61E2,1.2845426E1,3.3997994E5,3.88E2,8.45E2,1.3050505E0,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,6.1826086E0,1.970492E6,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,7.269755E0,3.6050353E5,9.5157854E-2,1E0,-2.6830027E-3,7E1,1.7252632E-3,7.3586698E0,9.3954586E-4,-7.641705E-3,4.75524E-3,1.82E9,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"split_indices":[53,46,102,53,5,8,58,2,9,9,32,28,29,3,3,7,5,29,57,2,28,5,0,0,0,0,0,1,0,0,0,11,0,0,57,48,0,0,58,0,0,0,0,0,54,30,0,0,0,0,0,54,29,39,65,0,8,0,54,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.81E2,3.8E1,1.5E2,3.1E1,1.7E1,2.1E1,9E1,6E1,1.7E1,1.4E1,4E0,1.3E1,1.5E1,6E0,1.9E1,7.1E1,4.5E1,1.5E1,9E0,8E0,1.2E1,2E0,2E0,2E0,1.1E1,2E0,1.3E1,2E0,3E0,3E0,1.6E1,3E0,4.3E1,2.8E1,2.2E1,2.3E1,6E0,9E0,7E0,2E0,2E0,6E0,3E0,9E0,8E0,3E0,2E0,1.1E1,5E0,1.1E1,2.4E1,1.9E1,1.6E1,1.2E1,4E0,1.8E1,5E0,1.8E1,2E0,4E0,2E0,7E0,5E0,4E0,4E0,4E0,1.7E1,7E0,1.3E1,6E0,1.2E1,4E0,2E0,1E1,1.6E1,2E0,1.6E1,2E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-1.0673481E-2,-2.922265E-2,3.2056907E-1,-9.558372E-2,6.192814E-3,7.489175E-3,1.9760614E-2,-6.519865E-2,-2.9253313E-1,-5.28204E-2,5.102089E-2,-1.4650565E-1,-1.8217016E-2,-8.463386E-3,-2.1037353E-2,2.8328342E-2,-8.429096E-2,-5.649434E-2,8.870596E-2,-1.687275E-1,-7.512345E-4,8.831442E-2,-5.69984E-2,7.728642E-2,-2.82216E-3,-1.0359726E-1,6.4482796E-4,4.802586E-3,-9.4108865E-2,1.5494351E-2,1.22702755E-1,-2.0045006E-1,-3.7551587E-3,5.5987155E-4,6.6966885E-3,-2.3729175E-3,-8.077933E-3,6.7342306E-3,6.569003E-4,-1.23539165E-1,-4.9365353E-2,-6.873523E-3,-9.76779E-4,4.8460853E-3,-3.942172E-2,7.549796E-2,1.7108953E-1,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,-1,39,-1,-1,41,43,45,47,-1,-1,-1,49,-1,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.158347E0,4.18929E-1,7.998216E-2,3.5195512E-1,3.1221244E-1,0E0,0E0,2.0742762E-1,5.744326E-2,1.3263142E-1,2.764765E-1,5.5647433E-2,1.537655E-1,0E0,0E0,6.787142E-2,7.104939E-2,1.143521E-1,1.2254682E-1,3.3913255E-2,0E0,3.627623E-2,1.603502E-1,3.7099797E-2,0E0,2.7427554E-2,0E0,0E0,4.950477E-2,8.2423344E-2,6.474209E-2,3.1638563E-2,0E0,0E0,0E0,9.902678E-2,0E0,0E0,0E0,5.4692864E-2,1.5442751E-2,0E0,0E0,0E0,3.445577E-2,8.0237605E-2,6.781274E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,28,28,29,29,30,30,31,31,35,35,39,39,40,40,44,44,45,45,46,46],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,-1,40,-1,-1,42,44,46,48,-1,-1,-1,50,-1,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3060927E0,1.04201E-2,5.9165E4,1.810504E10,5.470729E2,7.489175E-3,1.9760614E-2,5.162122E2,5.752E3,1.0275E4,3.8834286E0,1E0,9.8298E4,-8.463386E-3,-2.1037353E-2,2.8945708E6,4E1,1.4139E4,3.996671E5,3.5523141E-3,-7.512345E-4,2.7123368E10,5.347626E5,7.751E3,-2.82216E-3,7.2899837E0,6.4482796E-4,4.802586E-3,1.1334876E0,7.6420575E6,2.6673E4,3.0353E4,-3.7551587E-3,5.5987155E-4,6.6966885E-3,8.591E3,-8.077933E-3,6.7342306E-3,6.569003E-4,5.8910837E0,5.69434E-1,-6.873523E-3,-9.76779E-4,4.8460853E-3,5.754E3,7.6221466E8,7.6E0,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"split_indices":[36,39,2,5,53,0,0,56,2,9,54,105,30,0,0,46,3,10,29,40,0,32,29,9,0,54,0,0,40,46,10,12,0,0,0,2,0,0,0,54,28,0,0,0,2,7,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.77E2,9E0,6.1E1,1.16E2,4E0,5E0,5.4E1,7E0,5E1,6.6E1,1.9E1,3.5E1,5E0,2E0,1.4E1,3.6E1,1.7E1,4.9E1,1.6E1,3E0,9E0,2.6E1,9E0,5E0,3E1,6E0,3E0,1.4E1,1.6E1,3.3E1,1.1E1,5E0,4E0,5E0,1.8E1,8E0,4E0,5E0,2.1E1,9E0,8E0,6E0,6E0,1E1,1.8E1,1.5E1,3E0,8E0,1.5E1,3E0,1E1,1.1E1,5E0,4E0,3E0,7E0,1.1E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[3.5595038E-4,-4.4311643E-2,2.0678306E-1,-1.9426705E-2,-1.8886262E-1,3.1928593E-1,5.56452E-2,-7.896872E-2,2.8734265E-2,-9.082003E-2,-3.8585064E-1,2.895736E-2,2.4027427E-1,1.074728E-1,-3.3747476E-2,-1.00550115E-1,1.675159E-2,7.017641E-2,-3.8212094E-2,-1.407528E-1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.6446205E-1,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,-1.5302381E-1,-7.644423E-2,7.8863114E-2,-4.7753017E-2,1.0884225E-1,-1.3557769E-2,-5.902727E-2,4.1850857E-3,-9.2872605E-3,-4.909929E-2,1.4215367E-2,5.618002E-3,-3.536953E-3,-2.0744017E-1,-4.741403E-2,-1.308574E-1,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.7041807E-2,1.5840885E-1,-8.000341E-2,2.9958343E-2,-1.2175934E-1,-2.2903696E-2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9178276E0,6.114423E-1,6.1136055E-1,4.238065E-1,4.477874E-1,3.14538E-1,8.238807E-2,1.3802564E-1,2.3018244E-1,2.9552594E-1,3.0219913E-2,0E0,4.197371E-2,6.660308E-2,4.066568E-2,5.66082E-2,5.5850185E-2,1.6659597E-1,8.780883E-2,6.407347E-2,0E0,0E0,0E0,0E0,2.8056264E-2,0E0,0E0,0E0,0E0,5.4878265E-2,5.5400893E-2,1.0032747E-2,1.9345354E-2,9.9761456E-2,5.1551525E-2,6.0662225E-2,0E0,0E0,1.02921E-2,0E0,0E0,0E0,4.0799797E-2,7.5311735E-2,1.8520892E-2,0E0,0E0,0E0,0E0,3.455124E-2,3.763908E-2,3.8690723E-2,1.8420529E-2,3.3416137E-2,2.2927333E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,6.6932364E7,5.9337012E7,3.3969492E2,1.4792529E10,5.0655737E0,1.2055723E8,2E0,2.2971932E7,2E1,1.8942408E7,2.895736E-2,2.819228E-1,2.1516852E5,9.4908775E1,1.142E3,1.06403265E1,9.607843E-1,1.4792529E10,5.9862564E1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.5868E4,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,5.244755E-3,7.44E2,9.950143E7,1.2067E4,5.6426154E2,3.634E3,2.2502415E1,4.1850857E-3,-9.2872605E-3,1.2281632E3,1.4215367E-2,5.618002E-3,-3.536953E-3,5.322348E7,1.4300815E6,3.1001872E5,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.58881E5,2.94E2,5.4452E4,6.568221E0,2.88446E5,9.38E2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"split_indices":[53,46,46,53,5,55,46,8,46,8,33,0,58,34,57,2,55,58,5,57,0,0,0,0,9,0,0,0,0,58,0,12,9,53,2,57,0,0,53,0,0,0,5,33,29,0,0,0,0,1,0,30,54,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.7E2,3.6E1,1.46E2,2.4E1,2E1,1.6E1,6.5E1,8.1E1,1.7E1,7E0,3E0,1.7E1,1E1,6E0,5.3E1,1.2E1,5E1,3.1E1,1.5E1,2E0,5E0,2E0,3E0,1.4E1,3E0,7E0,3E0,3E0,1.5E1,3.8E1,6E0,6E0,3.4E1,1.6E1,2.7E1,4E0,9E0,6E0,1.1E1,3E0,7E0,8E0,2.6E1,1.2E1,4E0,2E0,3E0,3E0,1.6E1,1.8E1,6E0,1E1,9E0,1.8E1,3E0,3E0,3E0,5E0,1.4E1,1.2E1,9E0,3E0,9E0,7E0,3E0,1.5E1,2E0,4E0,6E0,4E0,3E0,6E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[2.6763344E-4,-3.562319E-2,1.7865828E-1,-1.8097268E-2,-2.4454215E-1,4.47172E-1,1.16943076E-1,-3.3486996E-2,1.414897E-1,-3.481418E-1,3.6788161E-3,2.790042E-2,6.3239834E-3,-1.0595338E-2,1.9547185E-1,-6.566554E-2,2.4707085E-2,1.8834342E-1,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,5.0848205E-2,2.2159144E-1,4.051122E-3,-5.639541E-2,-1.0586935E-2,-4.5325225E-3,4.210238E-2,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,-6.8116255E-2,7.0792675E-2,7.365901E-2,-2.802525E-2,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,37,-1,-1,-1,-1,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.299687E0,6.143675E-1,5.111331E-1,3.9033002E-1,4.5747256E-1,1.6556728E-1,2.925187E-1,2.7127695E-1,7.63599E-2,5.6399226E-2,0E0,0E0,0E0,1.5918176E-1,3.4319043E-2,1.1866668E-1,1.1051539E-1,3.6700696E-2,0E0,0E0,0E0,0E0,5.9108846E-2,4.338783E-2,0E0,1.3749906E-1,0E0,0E0,1.0386192E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3594824E-1,6.001374E-2,6.339806E-2,7.879637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,22,22,23,23,25,25,28,28,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,38,-1,-1,-1,-1,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,1.8000048E10,1.4164306E-3,7.918757E7,3.582157E12,8.50999E9,1E0,2.0053221E6,1.2117E4,6.1826086E0,3.6788161E-3,2.790042E-2,6.3239834E-3,5.9084116E3,7.1528664E0,4.187504E7,5.6865673E0,8.81832E5,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,7.644773E-1,2.819228E-1,4.051122E-3,1E0,-1.0586935E-2,-4.5325225E-3,6.773399E0,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,1.2936847E4,6.7111494E8,2.7883E4,3.4288502E8,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"split_indices":[53,5,58,33,32,5,102,49,9,54,0,0,0,4,54,33,55,1,0,0,0,0,35,58,0,89,0,0,54,0,0,0,0,0,0,52,5,9,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.68E2,3.3E1,1.56E2,1.2E1,5E0,2.8E1,1.43E2,1.3E1,9E0,3E0,3E0,2E0,1.1E1,1.7E1,9.2E1,5.1E1,9E0,4E0,6E0,3E0,2E0,9E0,1.3E1,4E0,8.8E1,4E0,6E0,4.5E1,4E0,5E0,7E0,2E0,3E0,1E1,8.1E1,7E0,3.1E1,1.4E1,2.4E1,5.7E1,4E0,3E0,2.4E1,7E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[7.671295E-3,-4.4080827E-2,1.6683623E-1,-2.5097556E-2,-3.2062513E-1,2.2227801E-1,-1.2322681E-2,-1.1087152E-1,-5.909076E-3,-4.2326204E-3,-1.9872338E-2,1.3815795E-1,3.3456573E-1,-9.755224E-2,3.63379E-3,-1.355743E-1,1.2135234E-3,-9.3863435E-2,9.225092E-3,9.646266E-2,1.2028726E-2,3.6554006E-1,3.8201683E-3,-7.6256203E-3,-6.825402E-4,-9.017518E-2,-2.0942573E-1,-7.649883E-3,-5.741548E-2,5.0508834E-2,-2.6426286E-2,1.13049895E-1,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,-1.8938167E-2,9.144381E-2,1.4660759E-2,-8.3531834E-2,1.4068319E-3,1.457002E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7124388E0,8.161584E-1,5.0931644E-1,2.4293393E-1,1.6942883E-1,3.2215738E-1,1.0366744E-1,9.3960494E-2,1.6326205E-1,0E0,0E0,8.524135E-2,9.821105E-2,3.2380693E-2,0E0,5.822915E-2,0E0,3.252116E-2,1.558761E-1,4.3112665E-2,0E0,9.481287E-2,0E0,0E0,0E0,2.1951303E-2,2.0213217E-2,0E0,1.926352E-2,1.4102952E-1,1.3496363E-1,4.2491734E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.587293E-2,1.4206815E-1,5.812499E-2,9.924857E-2,0E0,3.1057805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,19,19,21,21,25,25,26,26,28,28,29,29,30,30,31,31,41,41,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.8000048E10,9.298568E7,1.4862166E1,5.709406E7,5.8038013E3,9.4908775E1,3.068397E6,2.0293878E2,-4.2326204E-3,-1.9872338E-2,1.197355E6,5E1,6.97021E3,3.63379E-3,8.3949E4,1.2135234E-3,8.587809E4,7.9762895E6,1.127938E6,1.2028726E-2,6.677E3,3.8201683E-3,-7.6256203E-3,-6.825402E-4,1.3868161E5,1.0595825E3,-7.649883E-3,1.96897E5,2.495E3,4.497684E1,4.078932E5,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,2.7309215E6,5.511358E1,5.8454223E0,1.6594656E3,1.4068319E-3,2.819228E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"split_indices":[53,5,46,59,46,53,57,12,53,0,0,29,3,4,0,12,0,29,46,29,0,9,0,0,0,29,34,0,1,2,59,29,0,0,0,0,0,0,0,0,0,46,59,54,4,0,58,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.56E2,5E1,1.47E2,9E0,3.8E1,1.2E1,2.6E1,1.21E2,3E0,6E0,2.3E1,1.5E1,6E0,6E0,2.2E1,4E0,1.7E1,1.04E2,1.8E1,5E0,1.3E1,2E0,3E0,3E0,1.5E1,7E0,5E0,1.2E1,4.8E1,5.6E1,1.6E1,2E0,2E0,1.1E1,2E0,1.3E1,5E0,2E0,1E1,2E0,1.8E1,3E1,3.3E1,2.3E1,5E0,1.1E1,6E0,1.2E1,1.4E1,1.6E1,7E0,2.6E1,1.3E1,1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.0088654E-2,-4.6401702E-2,1.01957306E-1,-3.6083024E-2,-2.5793928E-1,3.6505505E-2,1.9830973E-1,-8.227722E-2,2.2556398E-2,-2.2164036E-3,-1.9751323E-2,1.0576863E-1,-6.8776435E-3,1.892742E-2,1.5552393E-1,-6.712239E-2,-1.4856316E-2,-5.168168E-2,6.346703E-2,6.68196E-3,1.6625825E-3,2.28483E-2,-3.8955654E-3,1.8000433E-1,1.2089454E-3,-7.764623E-2,3.5982527E-2,-1.81113E-2,-1.3062242E-1,-3.4493096E-2,1.2884454E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,-1.4065999E-1,-6.0010847E-2,-2.1526885E-4,3.771369E-3,-5.376633E-2,7.684118E-4,-8.152548E-3,-1.2452523E-3,-7.3128425E-2,5.791923E-3,1.6153444E-1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,-1,27,29,-1,-1,31,-1,33,-1,35,37,39,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.306833E-1,3.268422E-1,3.0808258E-1,4.0275842E-1,2.0142797E-1,9.3877874E-2,1.0262287E-1,2.599991E-1,2.0373048E-1,0E0,0E0,2.2404373E-2,4.5329027E-2,0E0,5.311328E-2,8.85137E-2,0E0,6.123595E-2,2.7983302E-1,0E0,0E0,9.2295585E-3,0E0,3.8849056E-2,0E0,7.1570426E-2,1.3761323E-2,2.2606054E-2,2.2841543E-2,1.1545663E-1,9.22758E-2,0E0,0E0,0E0,0E0,5.3088874E-2,9.419921E-2,0E0,0E0,2.1270107E-2,0E0,0E0,0E0,8.4302165E-2,0E0,8.3529234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,25,25,26,26,27,27,28,28,29,29,30,30,35,35,36,36,39,39,43,43,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,-1,28,30,-1,-1,32,-1,34,-1,36,38,40,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.537424E0,1.4146589E8,1.2236866E3,6.978001E2,1.5214152E5,1.1244E4,4.9695907E0,8.260109E9,3.41673E5,-2.2164036E-3,-1.9751323E-2,5.6352734E1,8.412314E0,1.892742E-2,4.8420677E-1,3.4741312E8,-1.4856316E-2,8.062193E-1,6.6217735E-2,6.68196E-3,1.6625825E-3,4.9852173E1,-3.8955654E-3,1.08E2,1.2089454E-3,8.8916E4,1.09524E5,4.737429E8,4.6E1,1E0,1.5107028E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,9.135018E0,5.580003E8,-2.1526885E-4,3.771369E-3,9.042926E3,7.684118E-4,-8.152548E-3,-1.2452523E-3,9.2312753E-1,5.791923E-3,1.234543E1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"split_indices":[43,46,53,53,34,9,55,5,29,0,0,59,55,0,39,7,0,28,58,0,0,59,0,3,0,1,30,7,3,89,39,0,0,0,0,59,5,0,0,34,0,0,0,28,0,55,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.53E2,4.9E1,1.47E2,6E0,3E1,1.9E1,8.2E1,6.5E1,3E0,3E0,1.1E1,1.9E1,2E0,1.7E1,7.8E1,4E0,2.3E1,4.2E1,7E0,4E0,1.4E1,5E0,1.4E1,3E0,7.1E1,7E0,1.7E1,6E0,1.7E1,2.5E1,4E0,1E1,1.2E1,2E0,1.4E1,5.7E1,4E0,3E0,8E0,9E0,4E0,2E0,1.4E1,3E0,1.9E1,6E0,6E0,8E0,2.4E1,3.3E1,3E0,5E0,8E0,6E0,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.7253997E-2,-2.9623577E-2,9.3434505E-2,-2.2292453E-1,-1.1781458E-2,1.2750259E-1,-1.9930653E-1,-3.3444623E-3,-1.2307007E-2,1.7842796E-2,-6.838117E-2,8.674666E-2,3.2339618E-1,-8.01149E-2,-1.8067626E-2,-4.725675E-3,3.552434E-2,-1.2093892E-1,-3.2683358E-2,2.175943E-3,1.3742693E-1,2.7282907E-2,2.0839193E-1,-5.710558E-3,-1.8373676E-4,4.8699785E-2,-4.864452E-3,-1.5385593E-1,3.9161125E-4,1.3277833E-3,-5.8980435E-2,6.1291102E-2,-4.3464635E-2,4.9143904E-3,1.6467385E-1,1.2900099E-2,5.3549507E-3,1.10007E-1,1.44394925E-2,-1.135703E-2,-1.014236E-1,-6.185148E-3,-1.7182616E-3,1.0451842E-1,7.325834E-4,-7.334734E-2,9.2549727E-4,-2.225592E-3,2.6441324E-3,2.1629092E-1,7.352542E-2,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,35,-1,-1,37,-1,39,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,-1,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.848589E-1,4.077926E-1,7.5345045E-1,2.8820753E-2,1.8565977E-1,4.9887896E-1,1.3421485E-1,0E0,0E0,1.5181659E-1,6.734279E-2,2.3972067E-1,1.832602E-1,1.6607814E-2,0E0,0E0,1.1910778E-1,6.492545E-2,3.9455958E-2,6.2021095E-2,1.2556028E-1,0E0,2.1007508E-2,0E0,0E0,1.2233533E-1,0E0,2.6357263E-2,0E0,0E0,2.2924516E-2,1.8414091E-2,2.566748E-2,2.000419E-2,1.20536804E-1,0E0,0E0,4.2832226E-2,5.8858946E-2,0E0,2.2556297E-2,0E0,0E0,1.198972E-2,0E0,1.4668021E-2,0E0,0E0,0E0,3.9754987E-2,2.4631858E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,22,22,25,25,27,27,30,30,31,31,32,32,33,33,34,34,37,37,38,38,40,40,43,43,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,36,-1,-1,38,-1,40,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,-1,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,1.6770158E-4,1.2055723E8,1.9E1,7.183439E0,8.258924E3,3.312524E6,-3.3444623E-3,-1.2307007E-2,1.056E3,7.8206635E0,1E0,1.6232848E7,1.5574268E7,-1.8067626E-2,-4.725675E-3,4.759155E1,2.2235178E1,2.7E1,9.32784E5,1.1114967E-3,2.7282907E-2,1.5307794E2,-5.710558E-3,-1.8373676E-4,1.2029E4,-4.864452E-3,2.9239766E-2,3.9161125E-4,1.3277833E-3,1.1079511E1,8.78E2,3.0558723E-1,2.69336E3,8.637931E1,1.2900099E-2,5.3549507E-3,4E1,4.711463E5,-1.135703E-2,3.4996462E-1,-6.185148E-3,-1.7182616E-3,5.88E2,7.325834E-4,9.042926E3,9.2549727E-4,-2.225592E-3,2.6441324E-3,1.5213319E0,4.6379595E6,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"split_indices":[53,40,46,3,54,53,30,0,0,2,54,102,1,1,0,0,57,57,3,30,39,0,57,0,0,9,0,58,0,0,55,0,40,4,59,0,0,10,52,0,58,0,0,0,0,34,0,0,0,58,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.18E2,7.2E1,9E0,1.09E2,6.5E1,7E0,2E0,7E0,7.2E1,3.7E1,5.5E1,1E1,5E0,2E0,9E0,6.3E1,1.4E1,2.3E1,2.1E1,3.4E1,2E0,8E0,3E0,2E0,5.8E1,5E0,1.1E1,3E0,7E0,1.6E1,9E0,1.2E1,6E0,2.8E1,4E0,4E0,2E1,3.8E1,3E0,8E0,3E0,1.3E1,4E0,5E0,8E0,4E0,3E0,3E0,1.7E1,1.1E1,3E0,1.7E1,1.1E1,2.7E1,2E0,6E0,2E0,2E0,2E0,6E0,1.2E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.821953E-2,-5.8290947E-2,5.3941235E-2,-1.5808642E-2,-4.884159E-2,-6.909937E-2,1.1836419E-1,-1.4827322E-2,-8.093995E-2,-2.6398048E-2,-1.7481435E-2,4.550153E-2,1.5699527E-1,-6.594871E-2,3.347421E-2,-9.2051856E-2,-6.509846E-3,6.4362705E-2,-7.304391E-2,8.8149734E-2,-1.3496414E-2,1.2912967E-2,1.3428916E-1,-1.2285006E-1,-2.9534832E-2,9.6844934E-2,3.1477788E-3,-6.093492E-2,-1.3656648E-1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,-1.4490403E-1,5.8896925E-2,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.758437E-2,1.9397938E-1,-1.2398881E-3,-9.721017E-3,-4.254263E-2,2.629008E-3,6.676271E-3,9.927296E-5,-2.7347464E-2,5.9932888E-2,-1.4137745E-1,-3.3839725E-2,-1.0662791E-3,-1.5445867E-1,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,1.0476206E-1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,63,65,67,69,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.496537E-1,2.9141405E-1,5.4719037E-1,0E0,1.2872437E-1,2.878313E-1,1.1952078E-1,1.4789282E-1,4.9851447E-2,9.7840816E-2,0E0,4.396437E-2,3.570825E-2,5.645959E-2,5.92712E-2,6.509176E-2,3.340755E-2,1.0004889E-2,1.5075365E-1,1.9013055E-2,1.2354755E-2,0E0,7.2952986E-2,7.7276245E-2,2.294885E-2,3.7963897E-2,3.9735876E-2,6.714282E-2,3.9696425E-2,0E0,0E0,0E0,0E0,4.4010997E-2,2.6443997E-2,0E0,0E0,0E0,0E0,4.359097E-2,9.496659E-3,0E0,0E0,1.2809919E-2,0E0,0E0,0E0,2.1849878E-2,1.7759107E-2,1.1468753E-2,4.3800417E-2,0E0,9.729177E-3,0E0,0E0,0E0,0E0,1.81911E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,39,39,40,40,43,43,47,47,48,48,49,49,50,50,52,52,57,57],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,64,66,68,70,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,5.243E3,1E0,-1.5808642E-2,7.6420575E6,6.3639E4,4.0679638E6,2.802166E2,5.754E3,1.4364E4,-1.7481435E-2,1.9071735E6,7.573E3,1.822764E4,4.585366E-2,3E0,5.178571E0,7.406885E8,1.0807868E7,9.1797715E-1,3.508923E6,1.2912967E-2,1.56E4,1.17E2,3.65102E5,1.7027842E8,2.2583E4,1.8716404E10,7E1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,2.6654E4,1.00152E5,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.8114974E-1,4.2414474E0,-1.2398881E-3,-9.721017E-3,1.0437421E8,2.629008E-3,6.676271E-3,9.927296E-5,1.8465776E8,8.3154816E2,1.1272E4,2.7048333E10,-1.0662791E-3,7.64764E5,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,3.6E1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"split_indices":[53,9,102,0,46,10,33,53,2,10,0,33,9,34,58,8,55,7,1,28,33,0,2,10,1,33,10,32,10,0,0,0,0,2,2,0,0,0,0,28,58,0,0,7,0,0,0,5,4,30,32,0,9,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,1.21E2,6.7E1,3E0,1.18E2,2.3E1,4.4E1,5.8E1,6E1,2.1E1,2E0,1.6E1,2.8E1,2.8E1,3E1,5.2E1,8E0,7E0,1.4E1,9E0,7E0,3E0,2.5E1,1E1,1.8E1,9E0,2.1E1,3.2E1,2E1,4E0,4E0,5E0,2E0,9E0,5E0,6E0,3E0,4E0,3E0,1.4E1,1.1E1,5E0,5E0,1.6E1,2E0,6E0,3E0,1.4E1,7E0,7E0,2.5E1,3E0,1.7E1,6E0,3E0,3E0,2E0,1.1E1,3E0,8E0,3E0,8E0,8E0,6E0,8E0,3E0,4E0,3E0,4E0,3E0,2.2E1,1.5E1,2E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.81344E-3,-2.7871855E-2,1.0370427E-1,-1.6869712E-1,-1.2145221E-2,1.3553725E-1,-1.1319494E-2,-6.7867875E-2,-2.5078225E-1,1.1689531E-2,-7.812123E-2,2.2571503E-1,4.7493696E-2,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,-2.2323536E-2,5.4143388E-2,-9.7720034E-2,1.1783024E-4,1.8657329E-2,1.5811518E-1,9.230261E-2,-1.6328286E-3,-4.379925E-2,5.0750427E-2,1.1329648E-1,1.5480721E-2,-1.1520765E-1,-1.8632218E-3,8.1961945E-2,9.722441E-3,1.8049882E-3,8.147564E-3,-2.9605734E-3,1.3710238E-3,-1.09546825E-1,-2.015405E-2,-1.4587996E-3,6.54756E-2,1.231898E-1,7.869888E-4,7.951029E-2,-2.6912533E-2,-1.2801106E-1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,37,39,41,43,45,-1,47,-1,49,-1,-1,-1,51,53,-1,55,57,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8306116E-1,3.744514E-1,4.811502E-1,1.2368178E-1,2.4308157E-1,2.989232E-1,0E0,2.157684E-2,3.9408505E-2,1.6582306E-1,6.4956516E-2,1.4187676E-1,4.637814E-2,0E0,0E0,0E0,0E0,1.0265903E-1,1.14883915E-1,2.9888332E-2,0E0,0E0,3.1216264E-2,7.44717E-2,2.112299E-2,7.5522915E-2,1.9793212E-2,1.7225742E-2,8.900131E-2,4.644516E-2,0E0,3.160474E-2,0E0,1.4609658E-2,0E0,0E0,0E0,6.457342E-2,9.696657E-2,0E0,1.6802516E-2,1.4416695E-2,0E0,1.5758365E-2,1.763527E-2,4.027137E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,37,37,38,38,40,40,41,41,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,38,40,42,44,46,-1,48,-1,50,-1,-1,-1,52,54,-1,56,58,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.178571E0,1.3578947E1,8.062193E-1,7.3808947E0,5.861021E7,-1.1319494E-2,3.662451E6,2.692487E6,5.716463E2,2.770862E8,6.661E3,2.1516852E5,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,3.9E1,1.26881E5,1.6259928E1,1.1783024E-4,1.8657329E-2,6.917027E3,4.078932E5,6.47008E1,1.4300815E6,2.1229417E-4,2.4025E4,3.6E1,4.2845196E1,-1.8632218E-3,5.59076E3,9.722441E-3,6.804E3,8.147564E-3,-2.9605734E-3,1.3710238E-3,2.07533E5,1.2670352E1,-1.4587996E-3,1.43E2,2.458E3,7.869888E-4,1E0,4.983287E-2,5.3E1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"split_indices":[53,55,55,28,54,46,0,1,1,53,7,9,34,0,0,0,0,3,30,55,0,0,4,29,57,33,39,9,3,57,0,4,0,9,0,0,0,1,55,0,10,2,0,84,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.69E2,4.1E1,1.6E1,1.53E2,3.8E1,3E0,8E0,8E0,1.13E2,4E1,1.8E1,2E1,6E0,2E0,4E0,4E0,6.3E1,5E1,3.2E1,8E0,4E0,1.4E1,1E1,1E1,4.9E1,1.4E1,1.9E1,3.1E1,2.4E1,8E0,6E0,8E0,5E0,5E0,3E0,7E0,1.2E1,3.7E1,2E0,1.2E1,1.7E1,2E0,1.2E1,1.9E1,2.2E1,2E0,4E0,2E0,2E0,3E0,7E0,5E0,3.5E1,2E0,4E0,8E0,4E0,1.3E1,1E1,2E0,3E0,1.6E1,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.1425839E-2,-5.037939E-2,8.030544E-2,-3.6732145E-2,-1.940263E-1,2.1132613E-2,5.40595E-2,-8.394264E-2,-1.2433366E-2,-2.411859E-3,-2.3667553E-1,-7.583577E-2,1.2632278E-1,-9.6114695E-2,3.007908E-3,3.00965E-2,-5.099469E-2,-1.4926101E-2,-5.8065807E-3,-2.905628E-2,-1.687885E-2,9.731201E-4,1.4231151E-1,-1.0521082E-1,6.9725106E-4,-3.51654E-2,6.765553E-2,-1.512059E-2,-1.1731359E-1,-7.190969E-2,1.2920251E-3,7.135396E-2,1.5594196E-1,-5.4826945E-2,-1.3231695E-1,1.7662038E-3,-8.1669055E-2,-1.3722117E-3,8.081185E-2,-6.1539885E-2,5.3886116E-2,-1.3972227E-1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,1.7186429E-1,7.422348E-2,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,33,-1,35,37,39,41,43,-1,45,47,49,51,-1,53,-1,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1869767E-1,2.692873E-1,5.338764E-1,1.4808676E-1,6.220767E-2,0E0,5.446193E-1,8.3999515E-2,1.4421546E-1,0E0,3.7372112E-2,2.5613624E-1,6.0055733E-2,4.269564E-2,0E0,1.05328426E-1,1.0766618E-1,0E0,0E0,4.71834E-2,0E0,0E0,1.9822717E-2,4.4642508E-2,0E0,5.613244E-2,3.6327116E-2,1.0261665E-1,3.450933E-2,6.169112E-2,0E0,1.8945247E-2,2.107352E-2,2.1910824E-2,4.9286067E-2,0E0,3.790599E-2,0E0,1.6026601E-2,3.391008E-2,3.6774833E-2,1.0349661E-2,0E0,0E0,0E0,0E0,0E0,1.815337E-2,3.8996745E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,19,19,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,38,38,39,39,40,40,41,41,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,34,-1,36,38,40,42,44,-1,46,48,50,52,-1,54,-1,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3453049E3,8.67241E9,1.4164306E-3,4.9054803E2,4.944487E7,2.1132613E-2,1E0,1.4864864E0,9.785595E6,-2.411859E-3,6.096518E0,4.4816768E8,1.6463401E3,9.741151E-1,3.007908E-3,2.4242616E5,8.78E2,-1.4926101E-2,-5.8065807E-3,3.0687964E3,-1.687885E-2,9.731201E-4,2.8626094E5,5.2646E4,6.9725106E-4,1.3585858E1,1.29E2,3.307175E-2,3.170081E6,7.6599895E9,1.2920251E-3,2.6295085E3,3.8014093E2,6.492337E0,2.0289855E-2,1.7662038E-3,1.1565725E0,-1.3722117E-3,1.2075471E7,1.2067E4,1.2117E4,4E1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,3.2014463E0,1.0807868E7,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"split_indices":[53,5,58,56,46,0,102,58,46,0,54,45,53,28,0,29,0,0,0,53,0,0,29,1,0,57,0,58,1,5,0,4,59,54,58,0,43,0,49,9,9,3,0,0,0,0,0,36,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.4E2,5.9E1,1.29E2,1.1E1,3E0,5.6E1,4.3E1,8.6E1,3E0,8E0,2E1,3.6E1,4E1,3E0,4.1E1,4.5E1,4E0,4E0,1.8E1,2E0,5E0,3.1E1,3.7E1,3E0,1.5E1,2.6E1,3E1,1.5E1,1E1,8E0,6E0,2.5E1,1.4E1,2.3E1,6E0,9E0,3E0,2.3E1,1.8E1,1.2E1,1.2E1,3E0,6E0,4E0,2E0,4E0,2E1,5E0,5E0,9E0,1.2E1,1.1E1,6E0,3E0,2E1,3E0,1.3E1,5E0,6E0,6E0,1E1,2E0,1.7E1,3E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[4.6584974E-3,-1.20946085E-2,1.9950889E-1,-1.0169338E-1,3.8820638E-3,3.8749582E-1,8.888968E-2,-5.8970988E-2,-1.9482663E-1,-9.452532E-2,1.551003E-2,2.2913612E-2,7.1646026E-3,1.3861427E-1,-4.1111445E-4,-8.537462E-2,2.566781E-2,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,-1.250599E-1,-3.5212412E-2,4.8972625E-2,2.1056903E-3,7.941639E-3,-4.738872E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,-8.0461E-2,-1.16186915E-2,8.0707185E-2,-5.17336E-2,5.499706E-2,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,-4.0313277E-2,4.2702768E-2,1.3941039E-1,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009837E-1,2.8382987E-1,3.1906998E-1,1.07877135E-1,1.9484903E-1,5.082476E-2,5.925162E-2,5.135636E-2,2.9909045E-2,1.1930962E-1,2.5971475E-1,0E0,0E0,1.16812885E-2,0E0,1.3007693E-2,1.56768E-2,0E0,0E0,0E0,6.1911702E-2,1.2099819E-1,1.173628E-1,0E0,0E0,1.1479644E-2,0E0,0E0,0E0,1.0476619E-2,0E0,6.582728E-2,7.438664E-2,8.7449014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.335335E-2,8.5745096E-2,9.818435E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,20,20,21,21,22,22,25,25,29,29,31,31,32,32,33,33,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1.4862166E1,2E0,1E0,4.060294E0,9.886E3,6.627576E7,1.3448806E8,8.3949E4,7.56E2,4.2658337E2,2.2913612E-2,7.1646026E-3,1.625E3,-4.1111445E-4,8.351808E-1,9.282E3,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,5.936149E1,2.124077E1,6.7948157E-1,2.1056903E-3,7.941639E-3,2.2970924E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,1E0,-1.16186915E-2,1.2623029E7,1.38E2,1.6974416E6,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,5.138372E8,3.810132E-2,1.2951E4,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"split_indices":[53,59,8,84,54,9,46,7,12,0,53,0,0,0,0,28,9,0,0,0,57,59,35,0,0,28,0,0,0,65,0,12,0,29,0,0,0,0,0,0,0,0,7,28,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.97E2,1.6E1,2.9E1,1.68E2,5E0,1.1E1,2.1E1,8E0,1.7E1,1.51E2,3E0,2E0,7E0,4E0,1.6E1,5E0,2E0,6E0,2E0,1.5E1,6E1,9.1E1,2E0,5E0,8E0,8E0,3E0,2E0,1.2E1,3E0,7E0,5.3E1,8.9E1,2E0,4E0,4E0,2E0,1E1,5E0,2E0,3E0,5E1,7.9E1,1E1,4.8E1,2E0,4E0,7.5E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-9.7492535E-4,2.4562754E-2,-1.2644652E-1,1.0812377E-2,3.050939E-1,-1.6741821E-1,3.276021E-2,-4.319679E-2,5.309041E-2,1.990497E-2,7.832562E-3,-1.8774702E-1,3.1424365E-3,3.0188295E-3,-1.7861315E-3,-2.2519317E-2,-9.010733E-2,8.511394E-2,-1.3702277E-2,-1.5522692E-1,-1.8752424E-2,-3.5709422E-2,8.45564E-2,-1.2079619E-1,5.7729695E-4,3.1312965E-2,1.5568885E-1,2.2016572E-2,-9.091352E-2,-1.9204946E-1,-1.8670933E-3,-5.38557E-2,2.1705188E-2,5.703837E-3,6.2500284E-4,-1.3545471E-1,-3.5225743E-4,8.462742E-2,-2.3649544E-2,1.15246095E-1,1.1550232E-2,-3.8249083E-2,4.199171E-2,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,27,29,-1,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.63182E-1,6.601755E-1,2.3378861E-1,3.7905455E-1,5.4006636E-2,1.4417273E-1,1.8542241E-2,6.878023E-2,2.0008749E-1,0E0,0E0,1.0724431E-1,0E0,0E0,0E0,7.6461144E-2,7.027824E-2,2.343359E-1,8.749639E-2,9.590763E-2,0E0,5.013643E-2,1.2541946E-2,2.7317584E-2,0E0,1.1042236E-1,6.1801553E-2,2.8090071E-2,2.3553073E-2,3.9512277E-2,0E0,1.5470095E-2,4.2363483E-2,0E0,0E0,2.9321939E-2,0E0,1.5620723E-2,5.383783E-2,5.4485172E-2,0E0,1.0460219E-2,2.2912873E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,37,37,38,38,39,39,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,28,30,-1,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,8.861832E3,1.4539318E-1,4.2658337E2,1.04E3,1E0,1.3507566E8,1.3839568E1,2.4934822E7,1.990497E-2,7.832562E-3,3.074E3,3.1424365E-3,3.0188295E-3,-1.7861315E-3,1.91126E5,7.407092E0,5.370622E5,1.06403265E1,8.66076E1,-1.8752424E-2,3.8E1,4.61E2,5.979E3,5.7729695E-4,9.6494156E-1,1.8907035E3,4.602015E0,1.27718E6,3.27E2,-1.8670933E-3,2.4397528E8,4.3E1,5.703837E-3,6.2500284E-4,1.7218965E2,-3.5225743E-4,8.047134E4,4.649E3,7.4245725E0,1.1550232E-2,8.062193E-1,3.706834E-4,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"split_indices":[46,4,39,53,0,89,46,57,46,0,0,0,0,0,0,30,54,29,55,57,0,3,0,2,0,58,4,54,29,0,0,32,10,0,0,53,0,34,2,55,0,28,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.71E2,3.4E1,1.64E2,7E0,2.7E1,7E0,7.2E1,9.2E1,3E0,4E0,2.5E1,2E0,5E0,2E0,5.1E1,2.1E1,6.2E1,3E1,2.3E1,2E0,4.6E1,5E0,1.6E1,5E0,3.6E1,2.6E1,2.1E1,9E0,1.7E1,6E0,3.5E1,1.1E1,3E0,2E0,1.4E1,2E0,1.8E1,1.8E1,1.9E1,7E0,5E0,1.6E1,7E0,2E0,4E0,1.3E1,2E0,3.3E1,4E0,7E0,4E0,1E1,8E0,1E1,7E0,1.1E1,8E0,1.1E1,2E0,3E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[6.475444E-3,-1.306737E-2,1.7832997E-1,2.339655E-3,-1.5614031E-1,2.8744474E-1,5.3003386E-2,-1.6489496E-2,6.7189105E-2,-2.4180317E-1,1.30160665E-2,3.55285E-1,3.888601E-3,9.1399415E-4,6.7161336E-3,-1.0044899E-2,-1.06382035E-2,1.3008821E-2,1.2666671E-1,-2.8520468E-1,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,-1.6523695E-2,1.5263823E-1,-4.6523552E-2,7.135857E-2,1.455868E-1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,-2.6452178E-2,7.834649E-2,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,-1,33,35,37,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9082713E-1,4.0816608E-1,2.7193904E-1,2.06246E-1,2.639749E-1,1.2648803E-1,2.9406311E-2,1.6842958E-1,1.2080774E-1,5.8472157E-2,2.9649243E-2,6.8154216E-2,0E0,0E0,0E0,1.3743296E-1,0E0,7.626754E-2,3.144613E-2,2.2196114E-2,0E0,0E0,0E0,0E0,0E0,1.1887014E-1,4.850761E-2,1.9805137E-2,2.4082582E-2,1.8560827E-2,0E0,0E0,0E0,7.060035E-2,5.3812273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,-1,34,36,38,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.993295E3,1.8000048E10,5.9337012E7,1.5085194E0,1.5461028E12,6.6737964E2,6.543368E6,1.3201905E0,7.7567725E2,4.577256E3,2.6023E4,1.04E3,3.888601E-3,9.1399415E-4,6.7161336E-3,2.3329E4,-1.06382035E-2,2.2906428E5,6.4487465E6,5.709406E7,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,1E0,9.39E2,3E1,5.1E2,3.5E1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,4.56567E7,3.2834557E-1,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"split_indices":[4,5,46,43,32,57,30,40,53,4,10,0,0,0,0,2,0,29,48,46,0,0,0,0,0,89,0,3,0,3,0,0,0,46,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.84E2,2E1,1.67E2,1.7E1,1E1,1E1,1.3E2,3.7E1,1.1E1,6E0,7E0,3E0,8E0,2E0,1.27E2,3E0,2E1,1.7E1,8E0,3E0,4E0,2E0,4E0,3E0,1.23E2,4E0,1E1,1E1,1.4E1,3E0,2E0,6E0,1.12E2,1.1E1,2E0,2E0,4E0,6E0,4E0,6E0,5E0,9E0,9.5E1,1.7E1,2E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0223907E-3,-2.7888758E-2,9.6585326E-2,-1.5584077E-2,-2.1336152E-1,1.575512E-1,-3.900375E-2,-3.563399E-2,2.183398E-2,-3.6633124E-3,-1.6371118E-2,1.8702927E-1,-5.3675026E-3,2.0629173E-2,-1.4183015E-1,-3.0176764E-2,-7.741039E-3,5.2294556E-2,-3.702995E-2,5.3999938E-2,2.235795E-1,4.8674736E-2,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,-4.079073E-2,3.0565633E-2,6.8358756E-3,7.473907E-2,2.9559305E-3,-5.9157714E-2,-3.6169306E-4,3.6848264E-3,1.9294271E-2,1.8000335E-1,4.2783604E-3,7.0099917E-4,-3.1242667E-2,-1.5811709E-1,-3.2913506E-2,1.07815154E-1,2.1910788E-3,-7.367315E-4,9.088779E-2,5.4238463E-4,-5.502338E-3,-4.669484E-3,8.802838E-2,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8288884E-1,3.6638862E-1,4.1119415E-1,1.1556303E-1,1.445714E-1,2.890743E-1,1.0103525E-1,6.2558055E-2,9.8760046E-2,0E0,0E0,1.3865662E-1,0E0,3.1503566E-2,8.29455E-2,6.378441E-2,0E0,3.621982E-2,4.469773E-2,1.12352E-2,1.0943866E-1,1.083583E-2,0E0,0E0,0E0,8.803986E-2,7.710456E-2,1.1590129E-2,2.3634195E-2,0E0,4.6466663E-2,0E0,0E0,0E0,4.115665E-2,0E0,0E0,7.36302E-2,2.4497524E-2,2.882525E-2,2.4574421E-2,0E0,0E0,1.8441036E-2,0E0,0E0,1.4278462E-2,2.0401236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,25,25,26,26,27,27,28,28,30,30,34,34,37,37,38,38,39,39,40,40,43,43,46,46,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.775894E10,9.040637E7,2.0053221E6,6.0671224E7,1.3578947E1,5.3868E4,5.138372E8,3.8279E5,-3.6633124E-3,-1.6371118E-2,1.2737E4,-5.3675026E-3,9.123462E-1,2.6022625E5,2.0997734E8,-7.741039E-3,5.716463E2,8.75E2,5.22E2,1.6064256E-2,3.9E1,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,5.1383884E7,4.016202E1,7.25E2,1.5562991E6,2.9559305E-3,2.7102E4,-3.6169306E-4,3.6848264E-3,1.9294271E-2,3.383436E5,4.2783604E-3,7.0099917E-4,2.47565E6,5.88E2,3.887674E6,1.631E4,2.1910788E-3,-7.367315E-4,7.3586698E0,5.4238463E-4,-5.502338E-3,1.617E3,9.851E3,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"split_indices":[4,5,46,49,46,55,10,7,30,0,0,2,0,28,34,7,0,53,0,0,58,3,0,0,0,33,59,0,48,0,10,0,0,0,29,0,0,29,0,46,9,0,0,54,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.61E2,4.8E1,1.52E2,9E0,3.3E1,1.5E1,9.9E1,5.3E1,5E0,4E0,3E1,3E0,1E1,5E0,9.6E1,3E0,3.5E1,1.8E1,7E0,2.3E1,8E0,2E0,3E0,2E0,8.2E1,1.4E1,1.2E1,2.3E1,3E0,1.5E1,2E0,5E0,3E0,2E1,3E0,5E0,7.7E1,5E0,8E0,6E0,4E0,8E0,1.8E1,5E0,7E0,8E0,6E0,1.4E1,7.3E1,4E0,2E0,3E0,4E0,4E0,4E0,2E0,1.5E1,3E0,2E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[8.825459E-3,-2.0716727E-2,9.291196E-2,-9.286735E-3,-2.2815323E-1,4.1376296E-2,1.8566836E-1,-4.1899815E-2,3.8700562E-2,-1.1017629E-3,-1.425366E-2,-6.1172745E-3,7.967702E-2,2.2176522E-1,7.0497654E-2,-1.486095E-1,-2.7861884E-2,-2.4466898E-2,7.862809E-2,2.2698525E-2,-3.884307E-3,1.145321E-1,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,-1.2128689E-2,-1.3520806E-1,1.7996103E-2,-7.628645E-2,5.7908777E-2,9.716253E-3,-1.2009995E-3,3.937936E-2,2.097745E-3,6.499496E-3,-2.237607E-2,1.0090606E-1,-8.440877E-3,5.7535426E-4,5.9123063E-3,-9.376124E-3,-4.9414644E-3,-3.266944E-4,8.224998E-3,4.116531E-2,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,-1,49,-1,-1,51,53,-1,-1,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1613337E-1,3.6301482E-1,2.5045186E-1,2.3167856E-1,9.979531E-2,6.5972775E-2,6.335485E-2,1.2768199E-1,1.5325102E-1,0E0,0E0,3.7913833E-2,3.5779223E-2,8.4248126E-2,1.1696836E-2,5.5736482E-2,1.3178307E-1,5.418279E-2,8.121562E-2,1.124549E-2,0E0,1.054576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.3339594E-2,5.8268234E-2,4.0797673E-2,1.8643163E-2,5.442866E-2,0E0,0E0,9.034543E-3,0E0,0E0,8.4523104E-2,1.790364E-2,0E0,0E0,0E0,1.3487471E-2,0E0,0E0,0E0,5.1540002E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,29,29,30,30,31,31,32,32,33,33,36,36,39,39,40,40,44,44,48,48],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,-1,50,-1,-1,52,54,-1,-1,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5585917E0,1.4233672E8,1.6059814E3,1.2534044E3,8.667E3,5.433479E2,2.7304773E0,8.35E2,7.157224E0,-1.1017629E-3,-1.425366E-2,1.1350955E0,1.2224206E7,8E0,2.8352614E9,4.1749332E4,1.448907E6,7.794012E7,5.2086E4,2.9539914E4,-3.884307E-3,7.810231E4,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,1E0,2.18E3,5.491E3,1.47245E7,6.489288E6,9.716253E-3,-1.2009995E-3,8.842206E7,2.097745E-3,6.499496E-3,1.5989196E7,2.94E2,-8.440877E-3,5.7535426E-4,5.9123063E-3,1.424506E6,-4.9414644E-3,-3.266944E-4,8.224998E-3,9.1487586E-1,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"split_indices":[42,46,53,4,2,53,43,2,55,0,0,58,51,8,5,33,1,46,2,34,0,34,0,0,0,0,0,0,0,89,0,9,1,46,0,0,5,0,0,46,0,0,0,0,30,0,0,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.53E2,5.3E1,1.46E2,7E0,3.5E1,1.8E1,8.7E1,5.9E1,2E0,5E0,1.6E1,1.9E1,1.3E1,5E0,9E0,7.8E1,2.3E1,3.6E1,1.2E1,4E0,1.1E1,8E0,9E0,4E0,2E0,3E0,6E0,3E0,6.9E1,9E0,1.3E1,1E1,3.2E1,4E0,3E0,9E0,3E0,8E0,6.4E1,5E0,7E0,2E0,2E0,1.1E1,7E0,3E0,3E0,2.9E1,4E0,5E0,4.5E1,1.9E1,2E0,3E0,9E0,2E0,2.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.4462977E-3,-2.893987E-2,6.9793515E-2,-3.6714904E-2,1.3322261E-2,2.450053E-2,1.3456057E-1,-3.0940516E-2,-1.5242134E-2,6.13726E-2,-2.383746E-2,1.9235724E-1,5.9276864E-2,-1.44404005E-2,-1.02318145E-1,6.015054E-3,1.4756152E-3,-5.1880803E-2,1.3078791E-3,2.2946529E-1,2.957695E-3,4.6630064E-3,-1.3928295E-3,-7.5536236E-2,1.9160501E-3,-1.6728845E-1,-1.4484057E-2,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,-4.1924756E-2,-1.2142303E-2,9.4770074E-2,-1.28825195E-2,-1.1275523E-1,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,27,-1,29,-1,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9038587E-1,3.7482685E-1,1.5511182E-1,2.329789E-1,0E0,6.027959E-2,8.43229E-2,1.7518334E-1,0E0,3.29639E-2,2.2726193E-2,4.4647485E-2,3.807029E-2,1.2305999E-1,1.5736201E-1,0E0,0E0,1.2245912E-2,0E0,1.3593733E-2,0E0,0E0,0E0,8.423267E-2,1.2907745E-1,5.515635E-2,1.3991938E-2,0E0,0E0,0E0,0E0,0E0,5.6269135E-2,7.5751625E-2,1.7076403E-2,0E0,2.499877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,23,23,24,24,25,25,26,26,32,32,33,33,34,34,36,36],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,28,-1,30,-1,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,1.3453049E3,2.8159826E0,1.3322261E-2,2.4608E4,8.812601E-1,7.361366E7,-1.5242134E-2,7.614E3,2.449092E5,2.5501368E9,8.2731545E-1,6.974669E7,8.66076E1,6.015054E-3,1.4756152E-3,3.239E3,1.3078791E-3,3.6E1,2.957695E-3,4.6630064E-3,-1.3928295E-3,8.35E2,5.506676E11,1E0,2.313233E6,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,3.54E2,8E0,3.3478114E-1,-1.28825195E-2,3.890129E5,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"split_indices":[43,57,53,36,0,9,58,46,0,10,29,7,28,7,57,0,0,2,0,3,0,0,0,2,32,65,1,0,0,0,0,0,0,8,28,0,29,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.54E2,5.3E1,1.51E2,3E0,3.2E1,2.1E1,1.49E2,2E0,1.8E1,1.4E1,1.1E1,1E1,1.22E2,2.7E1,5E0,1.3E1,9E0,5E0,8E0,3E0,7E0,3E0,2.5E1,9.7E1,1.5E1,1.2E1,5E0,4E0,6E0,2E0,5E0,2E1,8.5E1,1.2E1,4E0,1.1E1,3E0,9E0,1E1,1E1,6.1E1,2.4E1,3E0,9E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.4985325E-2,-3.9879307E-2,5.6348145E-2,-2.3429228E-2,-1.0598515E-1,-1.7923791E-2,9.930136E-2,-5.1214058E-2,3.0147022E-2,-3.333782E-2,-1.8125312E-1,3.6635974E-3,-3.763992E-2,1.5140602E-1,5.2652568E-2,-6.5377824E-2,1.8297803E-2,1.5232481E-2,8.540196E-3,-7.670071E-2,8.6073264E-2,-2.6167667E-1,-8.275727E-4,-9.86009E-3,-4.455716E-3,8.21394E-3,2.7782663E-3,8.663035E-2,1.0011745E-2,-1.3291118E-1,-5.3235017E-2,-3.8194293E-3,3.9224032E-2,7.2662863E-3,-3.2975117E-3,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,-1.8245183E-1,-3.2175884E-2,-1.0584831E-1,4.2874664E-3,-1.5806843E-3,9.24955E-3,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,-1,-1,-1,43,45,47,49,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.769355E-1,1.6824591E-1,1.7724186E-1,1.9059703E-1,1.6346598E-1,4.1423395E-2,7.709977E-2,8.389847E-2,9.2703044E-2,9.462805E-2,1.9369146E-1,0E0,2.6057394E-2,1.6584098E-2,2.810949E-2,5.03276E-2,3.3456847E-2,1.0235344E-1,0E0,3.1034954E-2,2.0551417E-2,4.1319788E-2,0E0,1.02055315E-2,0E0,0E0,0E0,1.1555165E-2,9.544139E-3,5.704157E-2,6.4181656E-2,0E0,4.842038E-2,0E0,6.882077E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.789392E-3,5.7711136E-2,5.315976E-2,0E0,0E0,3.5858765E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,27,27,28,28,29,29,30,30,32,32,34,34,48,48,49,49,50,50,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,-1,-1,-1,44,46,48,50,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,7.294459E7,3.239E3,1.2943141E3,1.3907746E10,7.751E3,6.4512783E-1,4.3E1,4.1983E4,1.753919E3,3.7284137E2,3.6635974E-3,8.724138E0,2.2151E4,2E0,8.35E2,3.99E2,2.73782E5,8.540196E-3,2.7864855E11,1.7837838E0,9.301866E7,-8.275727E-4,5.0900903E0,-4.455716E-3,8.21394E-3,2.7782663E-3,3.1E1,2.2621576E-1,4.569361E7,7E0,-3.8194293E-3,1.2623029E7,7.2662863E-3,1.2001309E0,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,4.1749332E4,1E0,3.677609E-1,4.2874664E-3,-1.5806843E-3,1E1,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"split_indices":[43,46,2,56,5,9,58,3,2,53,59,0,55,9,8,2,2,30,0,32,58,7,0,57,0,0,0,3,39,5,8,0,12,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,33,74,28,0,0,3,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.56E2,5.4E1,1.26E2,3E1,2E1,3.4E1,8.3E1,4.3E1,1.6E1,1.4E1,3E0,1.7E1,1.5E1,1.9E1,6.9E1,1.4E1,4E1,3E0,1.2E1,4E0,9E0,5E0,1.2E1,5E0,1.2E1,3E0,1E1,9E0,9E0,6E1,2E0,1.2E1,4E0,3.6E1,6E0,6E0,2E0,2E0,2E0,7E0,5E0,7E0,5E0,5E0,2E0,7E0,3E0,6E0,4.4E1,1.6E1,7E0,5E0,3.4E1,2E0,3E0,3E0,3.9E1,5E0,2E0,1.4E1,2E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.3111593E-4,8.332628E-3,-1.9431697E-1,-3.4451315E-3,1.5696207E-1,-1.8588053E-3,-1.294587E-2,-3.1280745E-2,2.9869221E-2,2.6512837E-1,5.53925E-2,-1.5708588E-1,-2.058258E-2,1.0180442E-2,9.531762E-2,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,2.4204077E-3,-7.460408E-2,-4.4569806E-3,1.9710528E-2,1.22437425E-1,-5.1242317E-4,-1.3157908E-2,7.7513896E-2,-1.2132418E-1,-5.7141267E-2,9.711912E-2,6.6852947E-3,1.4493192E-3,1.5084128E-1,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5398465E-1,3.5764718E-1,9.0578735E-2,1.7712615E-1,1.5010995E-1,0E0,0E0,1.367387E-1,1.1113774E-1,3.353101E-2,1.3905274E-2,3.2876134E-2,1.20718196E-1,6.775201E-2,5.870013E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.17382E-2,1.6727448E-2,0E0,6.287142E-2,3.7616774E-2,0E0,7.575147E-2,3.0201033E-2,3.716676E-2,8.603498E-3,2.5892913E-2,4.6929307E-2,0E0,2.140811E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,34,34],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4233672E8,4.9192E4,8.901E3,4.2350553E5,8.812601E-1,-1.8588053E-3,-1.294587E-2,5.160874E0,9.855582E5,1.8136424E7,6.804E3,5.3012047E-2,7.501344E0,2.884462E0,3.9247762E6,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,1E0,2.6239506E5,-4.4569806E-3,1.5324128E6,2.8244882E6,-5.1242317E-4,1.6278354E7,3.57E2,4.888E3,1.3593E4,1.7417817E-1,1.5489E4,1.4493192E-3,1.02952E5,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"split_indices":[46,2,2,48,58,0,0,55,29,1,9,58,54,54,48,0,0,0,0,0,0,74,29,0,33,33,0,46,0,2,10,39,9,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.03E2,8E0,1.89E2,1.4E1,3E0,5E0,1.03E2,8.6E1,6E0,8E0,7E0,9.6E1,6.7E1,1.9E1,3E0,3E0,2E0,6E0,5E0,2E0,6.8E1,2.8E1,5E0,6.2E1,1.5E1,4E0,5.7E1,1.1E1,6E0,2.2E1,8E0,5.4E1,4E0,1.1E1,3.5E1,2.2E1,6E0,5E0,4E0,2E0,2E1,2E0,2E0,6E0,2.7E1,2.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[7.821103E-3,-2.8981427E-3,1.7867728E-1,-5.675793E-2,1.8588716E-2,2.1517694E-1,-1.2397172E-4,-3.6930315E-2,-1.2052149E-2,-1.4890818E-2,4.163438E-2,2.7627808E-1,2.6501652E-3,-5.6514364E-2,6.0741283E-2,-4.3238487E-2,2.5888829E-2,-2.5325203E-2,7.3175E-2,1.5455141E-2,5.614201E-3,-7.2515686E-3,-4.0423278E-2,-8.0662005E-4,6.1275316E-3,1.2608571E-2,-6.090714E-2,-9.619795E-3,7.656288E-2,7.5304456E-2,-7.165898E-2,1.6621768E-1,5.4205015E-2,-1.64469E-2,-1.2827884E-1,2.7917859E-3,-9.4076915E-4,1.9582137E-3,-7.1012415E-2,3.3104515E-3,-3.469917E-2,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,-2.7355356E-2,-1.6309094E-1,1.9180217E-1,2.4875687E-3,5.2126553E-3,3.92498E-2,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,27,29,31,-1,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,-1,59,61,63,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7230206E-1,2.2325236E-1,7.992381E-2,1.9902351E-1,1.0709499E-1,8.315757E-2,0E0,1.0098582E-1,0E0,6.701185E-2,1.7503278E-1,9.8798275E-3,0E0,5.797288E-2,4.7433127E-2,3.4317195E-2,4.39213E-2,1.3128516E-1,8.98312E-2,0E0,0E0,0E0,7.7935606E-2,0E0,0E0,1.3974639E-2,2.9431812E-2,3.1568956E-2,1.9552685E-2,2.7546454E-2,7.250317E-2,1.6699761E-2,3.3773705E-2,4.245451E-2,5.6236163E-2,0E0,0E0,0E0,2.1543108E-2,0E0,1.2517829E-2,0E0,0E0,0E0,0E0,2.2041425E-2,2.2109985E-2,1.6221613E-2,0E0,0E0,3.1004522E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,40,40,45,45,46,46,47,47,50,50],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,28,30,32,-1,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,-1,60,62,64,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,4.039435E-3,4.6E1,2.0604518E8,5.470729E2,2.0867E4,-1.2397172E-4,1E0,-1.2052149E-2,3.5E1,1E0,9.299267E9,2.6501652E-3,9.342733E7,3.52777E5,6.027375E0,5.935E3,1.4364E4,7.573E3,1.5455141E-2,5.614201E-3,-7.2515686E-3,6.6932364E7,-8.0662005E-4,6.1275316E-3,1.625E3,6.558106E-3,6.693824E5,1.0275E4,9.123462E-1,5.4827127E0,1.81025E5,7.03482E5,1.3696512E7,9.64E2,2.7917859E-3,-9.4076915E-4,1.9582137E-3,4.053E3,3.3104515E-3,4E1,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,2.3541917E3,3.800374E6,2.6807916E0,2.4875687E-3,5.2126553E-3,2.1225555E-1,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"split_indices":[2,39,3,7,53,9,0,74,0,3,102,12,0,5,1,54,10,10,9,0,0,0,46,0,0,0,39,49,9,28,58,30,48,46,0,0,0,0,11,0,3,0,0,0,0,4,1,40,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.91E2,1.1E1,5.4E1,1.37E2,9E0,2E0,5E1,4E0,5.6E1,8.1E1,6E0,3E0,4.2E1,8E0,3.3E1,2.3E1,2.6E1,5.5E1,4E0,2E0,5E0,3.7E1,4E0,4E0,8E0,2.5E1,1.4E1,9E0,8E0,1.8E1,8E0,4.7E1,3E1,7E0,3E0,5E0,2E0,2.3E1,3E0,1.1E1,4E0,5E0,5E0,3E0,1.3E1,5E0,6E0,2E0,9E0,3.8E1,7E0,2.3E1,5E0,2E0,9E0,1.4E1,6E0,5E0,2E0,1.1E1,3E0,2E0,4E0,2E0,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[1.6695334E-3,-2.430477E-2,6.987881E-2,-1.628185E-1,-1.665332E-2,3.3794872E-2,1.4406632E-1,-2.0858636E-3,-2.262213E-1,-1.3216189E-3,-8.5313104E-2,7.368082E-2,5.320639E-3,2.1326382E-1,6.0462188E-2,-1.3826551E-2,-4.5224247E-3,-3.0901669E-2,5.3741764E-2,-4.580613E-2,-1.4649896E-2,6.6577974E-3,3.5978813E-2,1.8743427E-2,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,-1.9552542E-2,-8.157249E-3,-1.2044725E-2,9.7219266E-2,-6.857402E-2,-3.44251E-4,5.0435625E-2,-1.1778983E-3,-4.252783E-4,5.2595764E-2,-4.5224298E-2,2.5154488E-2,-2.6128737E-3,4.6722338E-2,-8.63488E-4,1.1892799E-1,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,-1,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,-1,49,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7906808E-1,1.6156235E-1,1.5307292E-1,5.1149473E-2,1.556826E-1,4.617256E-2,9.702435E-2,0E0,1.2178838E-2,2.0037158E-1,2.1631023E-1,3.5536163E-2,2.652885E-2,6.27754E-2,2.7581971E-2,0E0,0E0,1.2084925E-1,1.2416096E-1,2.0679474E-2,0E0,0E0,1.1661252E-2,2.092692E-2,0E0,0E0,0E0,0E0,0E0,8.721569E-2,0E0,4.656056E-2,6.699793E-2,2.1412484E-2,0E0,8.723641E-3,0E0,0E0,1.359814E-2,4.531187E-2,9.016034E-2,0E0,1.4617754E-2,0E0,5.3301275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,22,22,23,23,29,29,31,31,32,32,33,33,35,35,38,38,39,39,40,40,42,42,44,44],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,-1,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,-1,50,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.3125E0,1.3173E4,5.34E2,1.3845824E1,1.5489E4,8E0,-2.0858636E-3,4.233395E6,8.3764136E2,2.1024984E5,5.2560944E1,3.0794094E0,2.2047469E4,2.4711658E5,-1.3826551E-2,-4.5224247E-3,8.260109E9,7.157224E0,8.071663E5,-1.4649896E-2,6.6577974E-3,7.4809E4,1.5383295E6,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,3.7E1,-8.157249E-3,5E1,7.157903E7,1.17E2,-3.44251E-4,2.398624E8,-1.1778983E-3,-4.252783E-4,2.8314E4,1.4218562E5,7.47E2,-2.6128737E-3,5.5326223E9,-8.63488E-4,2.4039803E5,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"split_indices":[43,54,2,0,55,9,8,0,1,53,34,59,43,4,34,0,0,5,55,49,0,0,30,48,0,0,0,0,0,3,0,3,7,10,0,7,0,0,9,29,0,0,5,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.54E2,5.8E1,7E0,1.47E2,4E1,1.8E1,3E0,4E0,1.21E2,2.6E1,1.6E1,2.4E1,9E0,9E0,2E0,2E0,7.9E1,4.2E1,2.3E1,3E0,5E0,1.1E1,2.1E1,3E0,7E0,2E0,4E0,5E0,7.4E1,5E0,1.7E1,2.5E1,1.4E1,9E0,9E0,2E0,1.2E1,9E0,4.7E1,2.7E1,1E1,7E0,4E0,2.1E1,8E0,6E0,2E0,7E0,7E0,2E0,7E0,4E1,1.6E1,1.1E1,3E0,4E0,1.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[3.2811111E-3,-1.1759641E-2,9.609429E-2,-5.231051E-3,-1.926002E-1,4.223107E-2,1.7549825E-1,-2.3685513E-2,3.7570078E-2,-7.758012E-4,-1.34779E-2,7.0535643E-3,2.5339068E-3,1.425658E-2,4.615414E-3,-1.5669417E-2,-9.816772E-3,4.6593472E-2,-5.953628E-3,1.4607337E-3,-3.911037E-2,-1.08244635E-1,-6.703496E-3,1.3439576E-3,8.3413675E-2,-3.8560897E-3,3.702331E-5,-1.4499448E-1,1.2573712E-3,4.405412E-3,-4.881462E-2,2.0919163E-3,-1.956045E-2,9.7872205E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,23,-1,-1,25,27,29,31,33,-1,-1,35,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.80326E-1,2.0324215E-1,1.1234322E-1,1.3358834E-1,8.498618E-2,7.30211E-2,7.642406E-2,1.6523767E-1,7.928742E-2,0E0,0E0,0E0,1.7196054E-2,0E0,0E0,9.389689E-2,0E0,8.1526615E-2,0E0,0E0,9.659185E-3,5.3020775E-2,4.9334954E-2,2.086177E-2,4.4374347E-2,0E0,0E0,2.1213919E-2,0E0,4.3695264E-2,4.650379E-2,0E0,1.4095474E-2,3.8369343E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,17,17,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,32,32,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,24,-1,-1,26,28,30,32,34,-1,-1,36,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.918757E7,1.4623123E8,1.7227725E0,1.0892118E3,8.667E3,1.1944959E8,3.0291426E-1,6.6932364E7,1.8323593E1,-7.758012E-4,-1.34779E-2,7.0535643E-3,8.0756325E-1,1.425658E-2,4.615414E-3,8.35E2,-9.816772E-3,7.316901E0,-5.953628E-3,1.4607337E-3,1.467E3,6.1E1,1.1103951E1,4.8759037E1,2.3524794E7,-3.8560897E-3,3.702331E-5,1.0907971E6,1.2573712E-3,1.643E4,6.87E2,2.0919163E-3,6.879445E1,1.9652087E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"split_indices":[33,46,40,53,2,33,39,46,58,0,0,0,28,0,0,2,0,55,0,0,0,3,55,57,33,0,0,29,0,9,0,0,57,40,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.72E2,2.7E1,1.67E2,5E0,1.7E1,1E1,1.17E2,5E1,2E0,3E0,4E0,1.3E1,3E0,7E0,1.13E2,4E0,4.8E1,2E0,8E0,5E0,9E0,1.04E2,2.2E1,2.6E1,2E0,3E0,7E0,2E0,8.3E1,2.1E1,7E0,1.5E1,2.3E1,3E0,4E0,3E0,4.2E1,4.1E1,1.2E1,9E0,9E0,6E0,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.0387296E-3,-1.8574167E-2,8.5663736E-2,-8.023423E-3,-1.4978984E-1,1.4511184E-1,-3.3413577E-3,-2.7192928E-2,3.1242838E-2,-1.0438316E-2,-2.401442E-3,3.2174537E-1,9.6915334E-2,2.1752674E-2,-7.842911E-3,-2.143232E-2,-6.5535344E-3,9.6770816E-2,-2.4400586E-3,1.8892974E-2,6.1257724E-3,7.6597073E-4,1.1642241E-1,-6.7683046E-5,7.212856E-2,-7.260881E-2,-7.379258E-3,1.2818316E-1,1.6344215E-3,-7.859502E-2,1.6685827E-2,2.2511675E-3,1.3107267E-1,6.177627E-3,-1.2163549E-3,-2.2213712E-2,-1.08554E-1,2.886258E-2,-3.7549652E-2,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,4.6269923E-2,-4.119254E-2,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,-1,-1,-1,31,-1,33,35,37,39,-1,41,43,-1,45,-1,-1,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1510934E-1,2.230747E-1,2.6504415E-1,1.14509515E-1,6.739807E-2,2.2421151E-1,8.667014E-2,6.0260966E-2,1.10594735E-1,0E0,0E0,2.9250383E-2,3.775531E-2,2.2251856E-2,0E0,6.994447E-2,0E0,2.9437363E-2,5.0833475E-2,0E0,0E0,0E0,1.449576E-2,0E0,3.511203E-2,3.5629295E-2,8.637136E-2,1.2054026E-2,0E0,1.4433451E-2,4.9849793E-2,0E0,1.9013852E-2,0E0,0E0,1.4445642E-2,3.5027638E-2,4.9891964E-2,7.3443145E-2,0E0,0E0,0E0,0E0,2.8110322E-2,1.6329382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,22,22,24,24,25,25,26,26,27,27,29,29,30,30,32,32,35,35,36,36,37,37,38,38,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,-1,-1,-1,32,-1,34,36,38,40,-1,42,44,-1,46,-1,-1,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6503105E3,8.67241E9,5.9337012E7,6.90321E2,6.1826086E0,5.0655737E0,1.4623123E8,6.0073395E8,1.10597E5,-1.0438316E-2,-2.401442E-3,2E0,2.819228E-1,4.078932E5,-7.842911E-3,1.142E3,-6.5535344E-3,7.393122E1,1.6595541E1,1.8892974E-2,6.1257724E-3,7.6597073E-4,3.559158E6,-6.7683046E-5,2.1516852E5,5.12334E9,5.62E2,7.44905E0,1.6344215E-3,1E0,6.1093975E6,2.2511675E-3,6.797394E8,6.177627E-3,-1.2163549E-3,1.1086122E3,5.244755E-3,1.5989196E7,2.7816156E5,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,5.6E0,7.3376025E6,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"split_indices":[53,5,46,53,54,55,46,7,30,0,0,8,58,29,0,2,0,59,57,0,0,0,1,0,34,32,0,55,0,17,33,0,7,0,0,34,58,46,29,0,0,0,0,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.61E2,4.9E1,1.5E2,1.1E1,2.9E1,2E1,1.01E2,4.9E1,6E0,5E0,5E0,2.4E1,1.8E1,2E0,9.7E1,4E0,1.6E1,3.3E1,3E0,2E0,5E0,1.9E1,1.3E1,5E0,2E1,7.7E1,1E1,6E0,6E0,2.7E1,4E0,1.5E1,3E0,2E0,9E0,1.1E1,3.5E1,4.2E1,3E0,7E0,2E0,4E0,1.8E1,9E0,2E0,1.3E1,4E0,5E0,7E0,4E0,2.5E1,1E1,1.9E1,2.3E1,2E0,1.6E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.4796249E-3,-2.8328227E-2,4.2443145E-2,-1.2839314E-2,-1.4659679E-1,6.714284E-2,-6.827072E-2,-2.624263E-2,3.197302E-2,-5.994708E-2,-2.0733023E-1,1.3621603E-2,5.6769222E-2,-2.7255906E-2,-8.979017E-3,-2.0926751E-2,-6.972742E-3,6.93552E-2,-2.6764661E-2,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,3.61498E-2,1.4122447E-1,1.9069649E-3,-5.1106434E-2,-1.4165286E-2,-5.6748535E-3,8.369632E-2,-1.472121E-3,-3.9843167E-3,1.1822013E-3,8.772388E-2,2.698954E-3,1.7185566E-1,1.427893E-3,-4.393489E-3,-4.8403983E-4,-7.145611E-2,5.5866083E-4,1.9922133E-3,5.4703923E-3,1.7163315E-1,4.553083E-2,1.823748E-2,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,33,35,-1,37,39,-1,41,-1,-1,-1,43,45,47,-1,-1,-1,49,51,-1,-1,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6417843E-1,2.5274518E-1,2.370248E-1,7.531766E-2,7.177171E-2,1.3860276E-1,7.124599E-2,5.672677E-2,6.5532796E-2,1.3237113E-2,3.143558E-2,0E0,1.1331123E-1,2.2602845E-2,0E0,5.8513593E-2,0E0,2.8243646E-2,3.5750594E-2,0E0,0E0,0E0,0E0,9.637425E-2,3.9510548E-2,0E0,1.52250845E-2,7.427207E-2,0E0,1.5040562E-2,0E0,0E0,0E0,7.007395E-2,7.020194E-2,1.9569606E-2,0E0,0E0,0E0,2.5643677E-2,5.1912256E-2,0E0,0E0,1.0038182E-2,2.4570972E-2,2.6236098E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,18,18,23,23,24,24,26,26,27,27,29,29,33,33,34,34,35,35,39,39,40,40,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,34,36,-1,38,40,-1,42,-1,-1,-1,44,46,48,-1,-1,-1,50,52,-1,-1,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,4.56567E7,1.09495016E8,5.470729E2,7.683833E-1,1.7727281E2,1.4623123E8,5.138372E8,1.1424464E3,3.97E2,2.1E1,1.3621603E-2,4.2245E4,1E0,-8.979017E-3,9.85054E5,-6.972742E-3,2.4848485E0,1.2707859E3,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,2.1637352E7,3.832274E2,1.9069649E-3,8.427037E6,1.056E3,-5.6748535E-3,6.685237E0,-1.472121E-3,-3.9843167E-3,1.1822013E-3,7.573E3,1.28885E1,4.759155E1,1.427893E-3,-4.393489E-3,-4.8403983E-4,7.8206635E0,4.570007E7,1.9922133E-3,5.4703923E-3,5.142232E-1,2.3033286E5,2.249985E5,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"split_indices":[53,46,46,53,28,34,46,7,4,0,3,0,2,8,0,30,0,58,4,0,0,0,0,46,57,0,48,2,0,55,0,0,0,9,55,57,0,0,0,54,5,0,0,58,34,34,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.38E2,8.4E1,1.23E2,1.5E1,6.9E1,1.5E1,9.5E1,2.8E1,7E0,8E0,2E0,6.7E1,1.2E1,3E0,9.2E1,3E0,1.7E1,1.1E1,2E0,5E0,2E0,6E0,5.5E1,1.2E1,3E0,9E0,8.7E1,5E0,1.5E1,2E0,5E0,6E0,2.1E1,3.4E1,9E0,3E0,4E0,5E0,1.7E1,7E1,7E0,8E0,6E0,1.5E1,3.1E1,3E0,2E0,7E0,1.2E1,5E0,6E0,6.4E1,4E0,2E0,8E0,7E0,1.9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.5098556E-3,-3.430795E-2,4.6132654E-2,-7.989392E-2,-7.3299413E-3,6.539497E-2,-1.2817919E-1,-5.846227E-2,-1.0367309E-2,4.5335226E-2,-3.8782444E-2,4.477679E-2,1.7191277E-1,3.5083215E-4,-1.0261507E-2,1.7973488E-3,-6.9773324E-2,6.968256E-2,-1.4760661E-3,-5.74948E-2,8.712715E-3,1.1792199E-1,2.4033781E-2,2.3350367E-1,2.4967322E-3,2.0957861E-2,-8.140613E-2,-2.9711837E-3,8.504071E-2,-2.0592897E-3,-7.8736946E-2,-3.0431528E-2,3.5283656E-3,1.3522897E-3,1.7078307E-1,-1.0893706E-2,4.7292955E-2,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,-9.249218E-2,1.0750794E-3,7.799928E-4,1.05605684E-1,-2.1099192E-3,1.9261859E-3,-2.492124E-2,-1.10814184E-1,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,-3.874176E-2,3.5721097E-2,3.354856E-4,7.9686604E-2,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,29,31,33,35,37,-1,39,41,-1,43,45,47,49,-1,-1,51,53,55,-1,-1,-1,-1,57,-1,-1,59,-1,-1,61,63,-1,-1,-1,-1,65,67,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0736163E-1,1.5191185E-1,2.669213E-1,1.18796796E-1,1.3261785E-1,1.4603654E-1,9.106535E-2,4.6816275E-2,0E0,5.738108E-2,4.4938266E-2,8.8200994E-2,6.973371E-2,0E0,0E0,0E0,4.075806E-2,5.1372252E-2,0E0,4.223033E-2,3.9660476E-2,5.719474E-2,3.9509647E-2,2.0871669E-2,0E0,1.7594527E-2,3.99065E-2,0E0,2.8111845E-2,2.0630281E-2,4.2121127E-2,1.1639008E-2,0E0,0E0,1.40551925E-2,2.734653E-2,3.7402496E-2,0E0,0E0,0E0,0E0,5.2814364E-2,0E0,0E0,8.88367E-3,0E0,0E0,9.187069E-3,3.1335637E-2,0E0,0E0,0E0,0E0,2.286155E-2,2.1192856E-2,0E0,1.3821408E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,41,41,44,44,47,47,48,48,53,53,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,30,32,34,36,38,-1,40,42,-1,44,46,48,50,-1,-1,52,54,56,-1,-1,-1,-1,58,-1,-1,60,-1,-1,62,64,-1,-1,-1,-1,66,68,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,4.039435E-3,1.3917024E8,2.0604518E8,1.2903E4,5.4349E4,1.5214152E5,2.9270105E2,-1.0367309E-2,1.3845824E1,4.311809E5,1.20171E5,1.1406721E2,3.5083215E-4,-1.0261507E-2,1.7973488E-3,2.1013361E-1,9.33E2,-1.4760661E-3,5.948464E0,1.3012535E1,4.361E3,5.590909E0,1.671E3,2.4967322E-3,4E1,4.1211517E1,-2.9711837E-3,6.0775385E0,2.5848E4,1.2753862E1,5.8348556E0,3.5283656E-3,1.3522897E-3,1.57727E3,1.96E4,4.485647E5,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,6.258335E5,1.0750794E-3,7.799928E-4,3.0597075E8,-2.1099192E-3,1.9261859E-3,2E0,1.7261968E7,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,8.062193E-1,6.75741E5,3.354856E-4,1.184E3,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"split_indices":[53,39,46,7,9,2,34,34,0,55,29,30,57,0,0,0,28,2,0,54,57,2,54,0,0,3,59,0,55,9,57,55,0,0,4,2,29,0,0,0,0,33,0,0,7,0,0,8,46,0,0,0,0,28,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.23E2,7.6E1,4.5E1,7.8E1,6.9E1,7E0,4E1,5E0,2.9E1,4.9E1,5.9E1,1E1,3E0,4E0,4E0,3.6E1,2.2E1,7E0,3.5E1,1.4E1,1.2E1,4.7E1,6E0,4E0,4E0,3.2E1,2E0,2E1,1E1,2.5E1,9E0,5E0,5E0,7E0,1.9E1,2.8E1,4E0,2E0,2E0,2E0,2.9E1,3E0,5E0,1.5E1,5E0,5E0,1E1,1.5E1,3E0,6E0,3E0,4E0,1.2E1,7E0,1.3E1,1.5E1,1.5E1,1.4E1,1.1E1,4E0,6E0,4E0,9E0,6E0,5E0,7E0,3E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.6206973E-3,-1.9603977E-2,4.7420494E-2,-4.8373155E-2,9.0342695E-3,7.200922E-2,-6.0652774E-2,-2.3101505E-2,-7.6871574E-2,8.582439E-2,-5.1944912E-3,5.357398E-2,1.6620213E-1,1.5464444E-2,-1.2918839E-1,4.415707E-2,-5.9842244E-2,-5.267666E-4,-1.0224566E-1,5.1139877E-3,-2.7146392E-5,7.1116164E-2,-2.1878945E-2,1.2166778E-1,4.110889E-2,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,-3.735069E-2,-1.0302123E-2,5.6916097E-4,4.160516E-3,-7.803266E-2,1.388256E-3,-1.3427517E-1,-4.7179796E-2,5.5395655E-4,5.7911566E-3,-3.5666935E-2,2.1849289E-3,1.78514E-3,9.610851E-3,1.667288E-2,9.926257E-2,6.1937334E-4,-3.6489156E-3,-5.62302E-2,-5.528736E-3,-7.2530344E-2,-9.035339E-3,-3.0848335E-3,5.726469E-4,-9.4292425E-2,-1.1822653E-2,4.17147E-2,-1.5671453E-2,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,47,-1,49,51,-1,-1,53,-1,-1,-1,55,57,-1,-1,59,-1,61,-1,-1,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.23844E-1,1.1413506E-1,2.1164352E-1,4.791811E-2,7.69298E-2,1.0222992E-1,8.024894E-2,9.65454E-2,5.252196E-2,1.8357664E-2,7.7948034E-2,4.1682094E-2,4.1363686E-2,1.208929E-2,5.2706182E-2,1.7784836E-2,4.241229E-2,0E0,3.423077E-2,0E0,0E0,3.0106395E-2,4.7635924E-2,4.4303104E-2,6.655512E-2,0E0,0E0,0E0,0E0,1.0375681E-2,0E0,0E0,0E0,9.898692E-3,0E0,3.0951291E-2,1.0299623E-2,0E0,0E0,5.7596285E-2,0E0,0E0,0E0,2.9002367E-2,3.170462E-2,0E0,0E0,2.2735469E-2,0E0,9.1954805E-3,0E0,0E0,0E0,2.8296992E-2,2.1532139E-2,1.2805883E-2,1.26161575E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,29,29,33,33,35,35,36,36,39,39,43,43,44,44,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,48,-1,50,52,-1,-1,54,-1,-1,-1,56,58,-1,-1,60,-1,62,-1,-1,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,3.1749217E4,9.820072E7,1.4455165E7,8.637E3,1.6974416E6,1.3611247E5,1.1086122E3,6.9163686E-1,7.105731E0,3.9986582E4,6.518218E6,1.884E3,9.276886E1,2.1000721E0,1.9789175E2,9.946E3,-5.267666E-4,2.19E2,5.1139877E-3,-2.7146392E-5,3.067E3,1.3411796E3,5.62E2,1.6992E4,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,2.012E3,-1.0302123E-2,5.6916097E-4,4.160516E-3,1.6126542E8,1.388256E-3,6.7E1,7.0229135E6,5.5395655E-4,5.7911566E-3,2.4861076E-1,2.1849289E-3,1.78514E-3,9.610851E-3,6.9089854E-1,7.438459E6,6.1937334E-4,-3.6489156E-3,7.774228E-2,-5.528736E-3,7.856E3,-9.035339E-3,-3.0848335E-3,5.726469E-4,1.0092749E8,1.4519928E1,1.8156171E3,1.521636E5,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"split_indices":[53,34,46,46,9,29,34,34,28,54,34,46,0,57,58,53,10,0,10,0,0,2,4,0,2,0,0,0,0,0,0,0,0,5,0,10,33,0,0,40,0,0,0,28,1,0,0,58,0,9,0,0,0,7,57,53,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.37E2,7.7E1,6.8E1,6.9E1,6.3E1,1.4E1,3.7E1,3.1E1,1E1,5.9E1,5.4E1,9E0,7E0,7E0,1.3E1,2.4E1,9E0,2.2E1,8E0,2E0,1E1,4.9E1,7E0,4.7E1,4E0,5E0,3E0,4E0,4E0,3E0,8E0,5E0,2E1,4E0,1.3E1,9E0,5E0,5E0,4.1E1,8E0,4E0,3E0,3.4E1,1.3E1,2E0,2E0,1.4E1,6E0,7E0,6E0,7E0,2E0,1.1E1,3E1,1.9E1,1.5E1,3E0,1E1,8E0,6E0,4E0,3E0,3E0,8E0,2.1E1,9E0,1.2E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.1845648E-4,-1.3225078E-2,1.3611369E-1,-1.417116E-1,-6.7631784E-3,1.953729E-2,8.1282414E-2,-1.8065992E-1,-1.7495391E-4,-5.01936E-2,7.976252E-3,7.00604E-4,1.2666328E-1,-1.9851862E-3,-1.1117795E-2,-3.9532438E-2,-9.23816E-3,1.1314319E-2,-6.9974E-3,7.236729E-3,1.2495602E-3,-2.5365911E-2,-6.0845776E-3,-5.2135065E-4,6.29387E-2,-5.9177596E-3,-7.433232E-2,-5.97098E-3,4.050646E-3,-1.3454848E-3,8.696749E-2,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,-1,23,-1,-1,-1,25,-1,27,29,31,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8503218E-1,1.5596202E-1,2.437979E-1,4.776524E-2,1.1626564E-1,0E0,4.97777E-2,3.7470296E-2,0E0,6.0973354E-2,7.052197E-2,0E0,1.8436834E-2,0E0,0E0,5.011916E-2,0E0,8.190803E-2,0E0,0E0,0E0,3.6700055E-2,0E0,6.1938964E-2,5.6821078E-2,3.1497225E-2,1.2199353E-2,0E0,4.7399078E-2,0E0,2.0637825E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,17,17,21,21,23,23,24,24,25,25,26,26,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,-1,24,-1,-1,-1,26,-1,28,30,32,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,5.314E3,1.4164306E-3,5.3E1,3.5523141E-3,1.953729E-2,4.1151314E0,9.34E2,-1.7495391E-4,4.0515396E-3,3.167407E7,7.00604E-4,1.7652562E4,-1.9851862E-3,-1.1117795E-2,2.47565E6,-9.23816E-3,9.855582E5,-6.9974E-3,7.236729E-3,1.2495602E-3,2.2345362E1,-6.0845776E-3,1E1,2.55E2,1.4740072E1,6.2174755E-1,-5.97098E-3,5.6E0,-1.3454848E-3,1.548321E-1,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"split_indices":[53,9,58,3,40,0,54,0,0,42,44,0,4,0,0,29,0,29,0,0,0,59,0,3,0,59,28,0,55,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.88E2,1.8E1,8E0,1.8E2,2E0,1.6E1,6E0,2E0,4.5E1,1.35E2,7E0,9E0,2E0,4E0,4.3E1,2E0,1.33E2,2E0,7E0,2E0,3.8E1,5E0,1.09E2,2.4E1,2.8E1,1E1,3E0,1.06E2,5E0,1.9E1,1.6E1,1.2E1,3E0,7E0,8E0,9.8E1,9E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.666469E-3,-1.7545465E-2,1.19480304E-1,-8.49554E-3,-8.6954616E-2,1.4809049E-2,7.858761E-2,-1.9953433E-2,4.7460496E-2,-1.262078E-2,-4.3817658E-2,1.2613867E-1,2.3892207E-2,-3.333415E-2,2.116914E-2,-2.6011164E-3,9.102063E-2,7.2902715E-4,-9.959214E-2,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,-2.6154688E-2,-6.2321755E-3,6.586735E-2,-6.205673E-3,-4.2967666E-3,1.9037895E-2,1.7304541E-1,1.7110182E-2,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,-4.9978763E-2,-6.258132E-3,5.289967E-3,1.691339E-3,8.535292E-3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,37,39,-1,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9837632E-1,1.14212275E-1,1.5746966E-1,1.0545254E-1,1.447895E-1,0E0,5.1603585E-2,7.553959E-2,6.1296083E-2,0E0,4.4794835E-2,2.756174E-2,1.8413085E-2,6.669822E-2,4.2011324E-2,2.7640013E-2,8.797648E-2,1.5424975E-2,1.9613385E-2,0E0,0E0,0E0,0E0,4.566688E-2,0E0,1.3546545E-2,2.2841673E-2,0E0,3.328212E-2,1.718907E-2,1.4277115E-2,0E0,0E0,0E0,0E0,2.4923556E-2,6.463696E-2,0E0,0E0,1.3547961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,25,25,26,26,28,28,29,29,30,30,35,35,36,36,39,39],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,38,40,-1,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,7.810581E11,1.4164306E-3,4.5E1,8.0522484E11,1.4809049E-2,1.0096877E8,1.5064244E0,6.787619E2,-1.262078E-2,7.294459E7,3.2635366E8,1.1038146E0,1.4114665E0,1.5489E4,8.27725E3,5.1270317E-2,6.324075E-5,7.5213313E-1,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,3.52777E5,-6.2321755E-3,3.0597075E8,4.187504E7,-4.2967666E-3,6.58542E5,8.63332E5,2.0323718E7,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,2.1034482E-1,4.95715E5,5.289967E-3,1.691339E-3,1.46E3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"split_indices":[53,32,58,3,32,0,52,43,53,0,46,33,35,42,9,49,39,39,28,0,0,0,0,1,0,7,33,0,1,1,46,0,0,0,0,58,1,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.82E2,2.3E1,1.62E2,2E1,3E0,2E1,1.35E2,2.7E1,3E0,1.7E1,1E1,1E1,1.02E2,3.3E1,1.3E1,1.4E1,1E1,7E0,8E0,2E0,6E0,4E0,9.6E1,6E0,1.2E1,2.1E1,2E0,1.1E1,6E0,8E0,3E0,7E0,3E0,4E0,4.3E1,5.3E1,4E0,8E0,1.8E1,3E0,6E0,5E0,2E0,4E0,4E0,4E0,3E1,1.3E1,1E1,4.3E1,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-2.7402204E-3,-1.4488179E-2,8.47742E-2,-1.21823445E-1,-8.342185E-3,1.8740107E-1,-9.2469845E-3,-1.6784839E-3,-1.3934572E-1,1.2403458E-2,-4.2587325E-2,2.9215527E-1,4.7938665E-3,2.4776671E-2,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,4.4642862E-2,-1.8922342E-2,-3.3003904E-2,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,-3.79182E-2,5.552622E-2,-4.235295E-2,3.8699843E-2,1.6639564E-2,-5.707555E-2,-3.353905E-3,1.2455323E-4,7.502207E-2,-1.0837827E-2,-2.996247E-2,-1.0963698E-1,6.998143E-2,-2.54994E-4,7.42517E-2,-9.605715E-3,-8.033993E-2,-1.4847374E-3,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,-1,25,27,29,-1,-1,-1,-1,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1713255E-1,1.21684544E-1,2.4296036E-1,1.0496929E-2,1.2627566E-1,8.29688E-2,6.688262E-2,0E0,1.0810778E-2,1.1294642E-1,1.1641891E-1,3.8371354E-2,0E0,1.0111526E-2,0E0,0E0,0E0,5.1525243E-2,7.859604E-2,7.862471E-2,0E0,0E0,0E0,0E0,0E0,8.959427E-3,6.421043E-2,3.0895934E-2,2.3733113E-2,3.3978265E-2,5.6785285E-2,0E0,0E0,2.0940512E-2,1.2563176E-2,2.8267324E-2,2.118297E-2,8.720197E-3,0E0,1.4422674E-2,9.381125E-3,2.5498733E-2,8.510427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,-1,26,28,30,-1,-1,-1,-1,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,8.35E2,5.9337012E7,9.279E3,2.4934822E7,1.04E3,1.4233672E8,-1.6784839E-3,1.22E2,1.6672E4,6.3522E4,1.6232848E7,4.7938665E-3,9.5589536E-1,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,5.277751E0,1.02942106E11,6.4348924E-1,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,3.4E1,8.019512E0,4.305809E7,3.2395E4,2.7778377E11,3.9880952E-1,-3.353905E-3,1.2455323E-4,4.091697E7,6.2958473E4,2.237806E8,1.0519024E8,7.2023706E0,-2.54994E-4,7.951E3,4.83E9,4.944487E7,1.5107028E-1,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"split_indices":[53,2,46,9,46,0,46,0,10,9,10,1,0,28,0,0,0,55,32,28,0,0,0,0,0,3,54,33,9,32,58,0,0,33,34,7,7,55,0,9,5,46,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.85E2,2.4E1,9E0,1.76E2,1.1E1,1.3E1,2E0,7E0,1.1E2,6.6E1,4E0,7E0,1.1E1,2E0,5E0,2E0,5.4E1,5.6E1,6.4E1,2E0,2E0,2E0,9E0,2E0,6E0,4.8E1,4E1,1.6E1,2.1E1,4.3E1,3E0,3E0,3.7E1,1.1E1,3.5E1,5E0,9E0,7E0,6E0,1.5E1,3E1,1.3E1,2.7E1,1E1,4E0,7E0,2.1E1,1.4E1,3E0,2E0,6E0,3E0,2E0,4E0,4E0,1.1E1,1.2E1,1.8E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.9001366E-3,-1.343855E-2,1.1601315E-1,-1.0010185E-2,-1.111417E-2,2.2562835E-1,3.3268586E-2,-4.3225788E-2,2.3105354E-3,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,-5.3559467E-2,4.5828316E-3,4.5007914E-2,-1.1152626E-2,-4.4588957E-2,-8.577182E-3,-4.0048826E-3,5.9981905E-2,-3.185049E-2,2.3447553E-2,-2.0497924E-2,-8.2263514E-2,8.222998E-3,3.7578024E-2,-1.694235E-2,-1.1662346E-1,3.2336626E-2,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,23,-1,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.899919E-1,1.4066838E-1,1.6482484E-1,7.867399E-2,0E0,1.0884556E-1,1.5947495E-2,7.8872435E-2,8.1554644E-2,0E0,0E0,0E0,0E0,4.606816E-2,0E0,6.974669E-2,7.8090996E-2,4.0971868E-2,0E0,0E0,6.934259E-2,8.396302E-2,4.81951E-2,3.422363E-2,2.2953227E-2,0E0,6.309295E-2,6.78614E-2,3.393674E-2,2.4240296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,24,-1,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.5303E4,1.897887E6,3.5523141E-3,-1.111417E-2,3.0291426E-1,3.9637537E0,1E0,1.19855255E-1,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,4.0515396E-3,4.5828316E-3,5.178571E0,1.5064244E0,1.3431159E6,-8.577182E-3,-4.0048826E-3,3.849802E-1,1.4330416E0,5.46229E5,3.59E2,4.277001E3,8.222998E-3,6.448948E6,9.717001E5,4.3590187E1,1.3047E4,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"split_indices":[4,10,30,40,0,39,54,89,42,0,0,0,0,42,0,55,43,29,0,0,28,42,9,0,48,0,33,29,59,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.93E2,1.8E1,1.91E2,2E0,7E0,1.1E1,5.1E1,1.4E2,4E0,3E0,9E0,2E0,4.8E1,3E0,3.3E1,1.07E2,4.6E1,2E0,3E0,3E1,6.7E1,4E1,2.9E1,1.7E1,4E0,2.6E1,5.8E1,9E0,3.8E1,2E0,1.4E1,1.5E1,1.3E1,4E0,2E1,6E0,4.8E1,1E1,4E0,5E0,1.3E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.2056913E-3,-9.473315E-3,7.512071E-2,4.3374414E-4,-7.8392126E-2,1.5720385E-1,-1.8191284E-2,-2.5484065E-2,2.0686356E-2,-1.2111978E-2,-4.889973E-2,2.523382E-1,8.440286E-2,1.3624126E-2,-6.7835744E-3,1.453452E-2,-4.12981E-2,5.900995E-2,5.9917844E-3,-9.201298E-3,-1.0729721E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,2.635762E-2,-6.7390345E-2,-9.481888E-5,5.3193234E-3,3.617373E-2,-4.9073147E-3,6.533765E-2,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,1.4299699E-3,3.6974133E-3,-3.8031008E-2,-1.2230593E-1,7.3048964E-2,-2.9346012E-2,2.2003332E-3,-1.1724293E-3,-1.1925893E-2,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,-1,45,47,49,-1,-1,-1,-1,51,-1,53,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7272978E-1,1.1952097E-1,2.1732129E-1,8.136228E-2,9.84464E-2,8.259007E-2,5.5882495E-2,4.3634407E-2,4.864037E-2,0E0,4.537849E-2,8.354357E-2,1.8657587E-2,1.5502267E-2,0E0,1.9534279E-2,5.267284E-2,2.3609497E-2,4.1716885E-2,2.1403857E-2,2.0815924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2154378E-2,4.371713E-2,4.4934556E-2,0E0,1.089422E-2,2.8820923E-2,5.3309254E-2,0E0,0E0,0E0,0E0,1.3273055E-2,0E0,1.3264997E-2,2.0031199E-2,2.1552466E-2,2.3880532E-2,0E0,0E0,2.0580877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,28,28,29,29,30,30,32,32,33,33,34,34,39,39,41,41,42,42,43,43,44,44,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,-1,46,48,50,-1,-1,-1,-1,52,-1,54,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.8236975E11,5.9337012E7,3.9360254E2,8.0522484E11,1.04E3,3.6030095E10,1.9689655E1,1.15062E5,-1.2111978E-2,7.294459E7,7.65E2,5.8199444E0,3.6377484E7,-6.7835744E-3,1.7081589E2,4.2060562E1,1E0,8.657441E7,1.6491606E3,5.534958E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,8.351808E-1,3.2022728E1,4.497684E1,5.3193234E-3,2.6921875E1,1E0,6.695E3,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,5.093772E-1,3.6974133E-3,5.1184835E0,4.8887E5,3.7E1,2.0133628E6,2.2003332E-3,-1.1724293E-3,5.0123274E-1,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"split_indices":[53,32,46,53,32,0,5,59,30,0,46,0,54,1,0,4,59,65,46,4,28,0,0,0,0,0,0,0,28,59,59,0,57,89,2,0,0,0,0,28,0,54,1,3,46,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.74E2,2.7E1,1.53E2,2.1E1,1.4E1,1.3E1,6.7E1,8.6E1,2E0,1.9E1,5E0,9E0,1.1E1,2E0,1.9E1,4.8E1,2.3E1,6.3E1,1.2E1,7E0,2E0,3E0,4E0,5E0,8E0,3E0,2E0,1.7E1,2.9E1,1.9E1,6E0,1.7E1,5.4E1,9E0,6E0,6E0,3E0,4E0,1.2E1,5E0,2E1,9E0,5E0,1.4E1,1.5E1,2E0,5E1,4E0,4E0,5E0,7E0,5E0,4E0,1.6E1,7E0,2E0,3E0,2E0,7E0,7E0,2.4E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-3.738608E-4,-7.6801833E-3,1.01941355E-1,7.692978E-3,-4.625046E-2,1.483836E-1,7.440007E-4,-9.925026E-3,6.1333936E-2,-8.378208E-2,-1.7553208E-2,1.8119766E-1,1.0880003E-3,-9.9033825E-2,-3.6509018E-3,1.2566844E-1,3.761748E-2,-1.050452E-1,-2.485187E-3,-5.857001E-3,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,1.0020773E-4,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.9409139E-2,5.540788E-3,-1.2027661E-1,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,1.2911957E-2,-6.143175E-2,2.4736589E-2,-1.5001127E-2,2.155236E-3,-5.720792E-4,-9.470102E-4,-1.3429011E-1,3.1634532E-2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,39,-1,41,-1,-1,-1,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5702043E-1,1.1657343E-1,5.407299E-2,1.3386032E-1,5.8844402E-2,3.31683E-2,0E0,5.945997E-2,4.867518E-2,4.122594E-2,4.3614455E-2,1.10411495E-2,0E0,1.9426428E-2,4.644964E-2,1.4112622E-2,3.5269197E-2,1.4692083E-2,8.812408E-3,3.3128686E-2,0E0,0E0,0E0,0E0,0E0,3.7203256E-2,0E0,0E0,0E0,1.8351674E-2,0E0,1.8578902E-2,0E0,0E0,0E0,1.6290672E-2,2.7178083E-2,4.342707E-2,2.3945123E-2,0E0,0E0,0E0,1.3021171E-2,2.1517647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,29,29,31,31,35,35,36,36,37,37,38,38,42,42,43,43],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,40,-1,42,-1,-1,-1,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,3.0143394E7,7.7434E4,7.7567725E2,2.3443186E3,2.0867E4,7.440007E-4,1.0135136E0,5.63905E6,1.07431E1,1.3844339E1,1.04E3,1.0880003E-3,5.045749E-1,1.3237829E7,1E0,1.3526E4,4.5E1,1.476E3,1.09495016E8,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,6.0130353E0,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.7115049E6,5.540788E-3,2.95E2,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,2.249985E5,2.884462E0,2.7496E4,9.0636E0,2.155236E-3,-5.720792E-4,-9.470102E-4,2.1E1,5.57E2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"split_indices":[2,46,2,53,4,9,0,57,51,55,58,0,0,28,51,102,2,3,0,46,0,0,0,0,0,54,0,0,0,48,0,0,0,0,0,34,54,30,55,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.95E2,1.3E1,1.4E2,5.5E1,8E0,5E0,1.06E2,3.4E1,2.3E1,3.2E1,6E0,2E0,6E0,1E2,8E0,2.6E1,1.8E1,5E0,3E1,2E0,3E0,3E0,3E0,3E0,9.8E1,2E0,2E0,6E0,2.2E1,4E0,1.4E1,4E0,3E0,2E0,2.3E1,7E0,3.7E1,6.1E1,1.2E1,1E1,2E0,1.2E1,1.5E1,8E0,4E0,3E0,9E0,2.8E1,2.1E1,4E1,1E1,2E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.0470916E-4,-1.1145463E-2,9.843046E-2,-6.1118633E-2,4.996951E-3,1.2766055E-2,5.4078806E-2,-2.5317255E-2,-1.1859635E-1,-1.923276E-2,6.010194E-2,-2.131724E-4,3.8350024E-3,-5.8678132E-3,-2.225848E-3,-1.1466437E-2,-6.446723E-2,-1.1111286E-2,-6.9062617E-3,1.0727382E-1,4.0534303E-2,2.531128E-3,-8.6874526E-4,7.2951976E-4,-8.027679E-2,-5.5201775E-3,-5.2759657E-3,3.1679564E-3,8.534004E-3,2.4647668E-2,1.0526845E-1,-5.7665943E-3,-2.0202387E-3,-5.2320316E-2,9.702222E-3,3.7629638E-2,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.165946E-1,1.4873593E-1,1.3118732E-1,8.959766E-2,1.8808083E-1,0E0,2.4914969E-2,1.1248944E-2,9.248865E-2,9.6062645E-2,3.534253E-2,0E0,0E0,1.2025353E-2,0E0,0E0,1.753379E-2,4.9994655E-2,0E0,2.3044884E-2,3.0314423E-2,0E0,0E0,0E0,1.1376724E-2,6.391215E-2,0E0,0E0,0E0,1.0773087E-2,2.1219812E-2,0E0,0E0,2.3858346E-2,4.077713E-2,1.2596151E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,16,16,17,17,19,19,20,20,24,24,25,25,29,29,30,30,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1E0,1.4164306E-3,6.107924E11,8.3764136E2,1.2766055E-2,6.804E3,4.327473E5,8.0522484E11,4.462437E7,6.518218E6,-2.131724E-4,3.8350024E-3,8.6E1,-2.225848E-3,-1.1466437E-2,1E0,6.0073395E8,-6.9062617E-3,4.6008915E6,1.56E4,2.531128E-3,-8.6874526E-4,7.2951976E-4,3.7614656E5,1.292E3,-5.2759657E-3,3.1679564E-3,8.534004E-3,9.087618E6,9.56E2,-5.7665943E-3,-2.0202387E-3,2.8434807E3,7.269755E0,8.461558E10,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"split_indices":[53,102,58,32,53,0,9,29,32,46,46,0,0,10,0,0,8,7,0,46,2,0,0,0,29,2,0,0,0,33,0,0,0,48,54,32,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.83E2,1.9E1,4.4E1,1.39E2,3E0,1.6E1,2.8E1,1.6E1,9.7E1,4.2E1,5E0,1.1E1,1.5E1,1.3E1,4E0,1.2E1,9.2E1,5E0,1.1E1,3.1E1,2E0,1.3E1,2E0,1E1,8.8E1,4E0,8E0,3E0,2.6E1,5E0,4E0,6E0,2.1E1,6.7E1,1.8E1,8E0,2E0,3E0,6E0,1.5E1,4.2E1,2.5E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.050566E-3,4.2812275E-3,-1.5856923E-1,-1.1301997E-2,5.6136116E-2,-2.1941631E-4,-1.0672151E-2,-7.0011653E-3,-1.2052315E-1,8.916989E-3,1.02983706E-1,-5.7596188E-2,1.8009902E-3,-2.245222E-3,-8.737236E-3,-5.627105E-3,5.7158438E-3,1.7363462E-1,6.978703E-2,-1.5238659E-2,-8.90067E-2,1.7688595E-2,-2.4926206E-2,-3.5939007E-3,4.3866397E-3,1.0011509E-2,3.031237E-3,8.032361E-2,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,3.1169657E-2,-2.7009284E-2,-6.0450863E-2,-1.2030017E-3,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7956841E-1,1.679042E-1,6.4495996E-2,7.4314885E-2,1.0526259E-1,0E0,0E0,6.9132246E-2,1.8055126E-2,4.0123533E-2,4.4171542E-2,2.9244505E-2,5.6916848E-2,0E0,0E0,1.6036227E-2,0E0,1.2192935E-2,1.5846536E-2,1.4111153E-2,3.3006206E-2,5.1458158E-2,4.2065214E-2,0E0,8.906513E-3,0E0,0E0,9.820163E-3,0E0,0E0,0E0,0E0,0E0,2.6585214E-2,3.6565024E-2,2.0105965E-2,2.8099384E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,3.0455479E3,8.667E3,1.8600049E10,1E0,-2.1941631E-4,-1.0672151E-2,1.292E3,3.240192E6,8.129243E5,1.2117E4,8.137731E7,2.73782E5,-2.245222E-3,-8.737236E-3,2.3125E0,5.7158438E-3,1.884E3,7.317201E0,1.1086122E3,4.1749332E4,1.06403265E1,2.2009405E1,-3.5939007E-3,1.3174513E4,1.0011509E-2,3.031237E-3,3.2707875E8,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,2.0589474E1,5.3153326E2,6.704386E6,4.462437E7,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"split_indices":[46,4,2,5,102,0,0,2,7,29,9,5,30,0,0,54,0,0,54,34,33,55,57,0,4,0,0,33,0,0,0,0,0,59,4,46,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.06E2,6E0,1.59E2,4.7E1,2E0,4E0,1.54E2,5E0,2.4E1,2.3E1,2.2E1,1.32E2,3E0,2E0,2.2E1,2E0,6E0,1.7E1,1E1,1.2E1,8.3E1,4.9E1,2E0,2E1,4E0,2E0,1.5E1,2E0,3E0,7E0,6E0,6E0,6.4E1,1.9E1,1.9E1,3E1,1.5E1,5E0,1.2E1,3E0,8E0,5.6E1,1E1,9E0,3E0,1.6E1,2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.4155842E-3,-1.3810372E-3,1.2642601E-2,-1.6525343E-2,4.4375908E-2,-1.2493689E-2,-8.009961E-3,1.661861E-2,9.891294E-2,-8.0245316E-2,-4.4310438E-3,4.9835492E-2,-6.8227405E-4,3.6446485E-3,9.054623E-3,-2.6625823E-2,-1.3053659E-1,-7.4558124E-2,1.1256639E-3,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,5.806915E-3,-7.995392E-2,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,9,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[2.0164502E-1,1.420966E-1,0E0,8.987266E-2,7.556003E-2,8.2129285E-2,0E0,2.0147853E-2,2.1653563E-2,3.9669834E-2,5.307982E-2,1.9236067E-2,1.3649627E-2,0E0,0E0,9.523076E-3,1.3058096E-2,3.7478834E-2,4.8667632E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2858007E-2,2.4820045E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,29,29,30,30],"right_children":[2,4,-1,6,8,10,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.8375797E2,1.6572577E0,1.2642601E-2,7.5656E4,1.9846272E9,4.994744E-5,-8.009961E-3,1.5489E4,9.246969E-1,8.7793425E-2,1.9E1,7.614E3,5E0,3.6446485E-3,9.054623E-3,2.8800612E9,3.2445145E2,2.75108E5,1.4655151E0,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,1E0,3.14608E6,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"split_indices":[57,42,0,10,5,43,0,9,28,59,3,10,8,0,0,5,34,9,42,0,0,0,0,0,0,0,0,0,0,89,49,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,2.03E2,2E0,1.53E2,5E1,1.5E2,3E0,3.4E1,1.6E1,1.5E1,1.35E2,1.1E1,2.3E1,1.4E1,2E0,8E0,7E0,9E0,1.26E2,6E0,5E0,1.6E1,7E0,2E0,6E0,5E0,2E0,6E0,3E0,1.2E2,6E0,1.07E2,1.3E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.000657E-3,-1.0153504E-2,9.496438E-2,-3.1673333E-3,-7.777414E-2,1.5645174E-2,6.2693365E-2,5.9380285E-3,-5.026301E-2,-9.674218E-3,-2.5248412E-2,9.625793E-2,2.6888123E-2,-6.8429126E-3,4.4476364E-2,-2.1850644E-2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,8.983803E-3,-4.372294E-2,6.858251E-3,1.07610226E-1,-6.777975E-2,-4.1921283E-4,-1.2492558E-2,3.5249196E-2,-9.645313E-3,-8.684807E-2,-1.2344087E-3,1.4300514E-3,1.2848052E-1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,29,31,33,35,37,-1,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5137326E-1,8.573215E-2,1.699006E-1,7.148311E-2,1.03037156E-1,0E0,2.6254699E-2,6.926221E-2,8.881589E-2,0E0,1.071448E-2,2.5661737E-2,2.7722035E-2,6.226391E-2,8.238617E-2,1.3398929E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2746734E-2,4.618959E-2,1.7440343E-2,9.797484E-3,1.0284005E-2,0E0,3.8003672E-2,4.3830436E-2,3.493194E-2,1.7143585E-2,0E0,0E0,1.4258161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,30,32,34,36,38,-1,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.3983983E10,1.4164306E-3,1.2670352E1,5.535E3,1.5645174E-2,1.095857E10,6.695E3,2.0662694E-1,-9.674218E-3,2.1000721E0,1.7173885E8,3.6377484E7,1.6278354E7,4.485647E5,2.650027E2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,5.097052E8,7.149117E-1,4.602015E0,8.564948E6,2.3968115E2,-4.1921283E-4,1.5498047E1,6.034607E6,9.737113E0,5.8426323E0,-1.2344087E-3,1.4300514E-3,6.879445E1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"split_indices":[53,5,58,55,9,0,12,2,39,0,58,33,1,46,29,4,0,0,0,0,0,0,0,5,28,54,33,4,0,57,49,55,54,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.81E2,2.5E1,1.65E2,1.6E1,2E0,2.3E1,1.39E2,2.6E1,4E0,1.2E1,1.1E1,1.2E1,1.05E2,3.4E1,2.2E1,4E0,8E0,4E0,8E0,3E0,7E0,5E0,7.4E1,3.1E1,2.2E1,1.2E1,4E0,1.8E1,4.1E1,3.3E1,1.8E1,1.3E1,9E0,1.3E1,8E0,4E0,2E0,2E0,3.5E1,6E0,2.5E1,8E0,1.2E1,6E0,8E0,5E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.156695E-3,-7.7511407E-3,6.4903505E-2,1.10561E-3,-6.161546E-2,8.889031E-2,9.5223775E-3,-1.1973289E-2,2.9857075E-2,-4.3537878E-2,-9.037242E-3,8.965046E-4,1.1017409E-1,-6.7902607E-4,2.1986503E-3,7.537986E-3,-3.928328E-2,6.545862E-2,1.1178152E-2,5.3028204E-2,-6.55025E-2,1.2060009E-3,6.3895895E-3,3.2280452E-4,6.727481E-2,-7.918549E-2,-8.271147E-3,3.5601782E-4,7.7395335E-2,2.8093105E-2,-2.6743395E-2,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,6.4077914E-2,-8.103772E-3,-7.31776E-5,4.6275426E-3,2.5301395E-5,-9.328173E-2,-5.3631708E-2,1.3650758E-2,2.564571E-3,6.1303745E-3,2.7586694E-3,-5.2228123E-3,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,-1,-1,-1,-1,49,51,-1,-1,-1,53,55,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5891957E-1,8.784768E-2,4.6976537E-2,6.014745E-2,5.0235853E-2,3.546658E-2,1.0674542E-2,5.890991E-2,3.251822E-2,5.437839E-2,0E0,0E0,3.1012043E-2,0E0,0E0,2.7994063E-2,5.6267872E-2,1.12324655E-2,2.2560306E-2,1.894341E-2,1.4103778E-2,0E0,0E0,3.2216277E-2,1.35331005E-2,2.251944E-2,2.7580641E-2,0E0,1.0156989E-2,2.3157822E-2,1.0649132E-2,0E0,0E0,0E0,0E0,1.371366E-2,2.1627188E-2,0E0,0E0,0E0,1.5584886E-2,1.2574337E-2,9.651603E-3,0E0,0E0,0E0,1.44525645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,35,35,36,36,40,40,41,41,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,-1,-1,-1,-1,50,52,-1,-1,-1,54,56,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,5.075104E7,5.1E1,6.787619E2,6.3538E4,6.19999E9,9.532E3,1.7086E4,1E0,5.577E3,-9.037242E-3,8.965046E-4,1.5214152E5,-6.7902607E-4,2.1986503E-3,1E0,2.5259212E5,2.695E3,7.6760286E-1,2.083122E6,5.534958E-1,1.2060009E-3,6.3895895E-3,4.570007E7,2.77E2,5.8910837E0,2.368447E6,3.5601782E-4,1.4854577E5,8.729897E0,2.3117E4,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,1.297E3,2.0200117E-1,-7.31776E-5,4.6275426E-3,2.5301395E-5,1.2318777E3,2.187288E1,3.3E1,2.564571E-3,6.1303745E-3,2.7586694E-3,7.9206874E8,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"split_indices":[53,46,3,53,10,5,9,9,8,9,0,0,34,0,0,74,29,2,28,1,28,0,0,5,0,54,33,0,34,55,10,0,0,0,0,2,28,0,0,0,4,59,3,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.83E2,3.5E1,1.58E2,2.5E1,2.4E1,1.1E1,1.09E2,4.9E1,2.3E1,2E0,6E0,1.8E1,7E0,4E0,6.4E1,4.5E1,1.6E1,3.3E1,4E0,1.9E1,4E0,1.4E1,5.8E1,6E0,1.9E1,2.6E1,3E0,1.3E1,2.3E1,1E1,2E0,2E0,5E0,1.4E1,6E0,5.2E1,2E0,4E0,3E0,1.6E1,8E0,1.8E1,1E1,3E0,1.2E1,1.1E1,4E0,6E0,3E0,3E0,6E0,4.6E1,1.4E1,2E0,3E0,5E0,1.1E1,7E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-8.12835E-4,2.8342765E-3,-1.5601721E-1,-1.7404577E-2,2.4213504E-2,-2.3796102E-3,-1.0275062E-2,-1.0378984E-2,-1.0568353E-1,9.81486E-3,1.8756628E-2,-2.6070654E-2,1.303203E-2,-6.017882E-4,-7.747756E-3,5.4938868E-2,-4.513947E-3,-1.7527992E-2,-4.4134418E-3,-1.0665796E-2,3.9632928E-2,1.0806894E-1,3.8279194E-2,1.5463653E-3,-6.14344E-3,-6.364611E-2,-6.1187022E-3,2.6128873E-2,-4.657165E-2,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.8797043E-2,-8.169748E-4,-2.9431539E-2,2.5399424E-2,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,-1,-1,21,23,25,-1,27,29,31,33,35,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2327455E-1,9.2584625E-2,1.9196324E-2,6.712644E-2,9.692997E-2,0E0,0E0,3.8177617E-2,3.798466E-2,0E0,8.61301E-2,3.2634463E-2,2.6898822E-2,0E0,0E0,3.1268403E-2,4.6668507E-2,2.9076368E-2,0E0,3.1603176E-2,3.4378827E-2,1.0986567E-2,2.5717452E-2,4.5821525E-2,0E0,1.2384877E-2,2.5712674E-2,1.1580934E-2,1.3976639E-2,0E0,0E0,0E0,0E0,1.3670683E-2,1.1835324E-2,1.6557261E-2,5.3196885E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,-1,-1,22,24,26,-1,28,30,32,34,36,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,6.978001E2,2.9E1,4.462437E7,1.7727281E2,-2.3796102E-3,-1.0275062E-2,7.555544E2,9.214292E7,9.81486E-3,2.4611046E7,1.9284746E1,9.258696E0,-6.017882E-4,-7.747756E-3,7.573E3,1.3844339E1,2.547512E2,-4.4134418E-3,6.855402E6,1.2670352E1,1.884E3,6.5829754E-1,2.819228E-1,-6.14344E-3,3.7790768E2,1.3696512E7,6.735744E1,6.8539324E0,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.982359E-1,2.709E4,7.340202E1,1.3888E4,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"split_indices":[46,53,3,46,34,0,0,4,7,0,46,57,55,0,0,9,58,56,0,46,55,0,28,58,0,34,46,59,55,0,0,0,0,28,9,57,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,2.12E2,4E0,1.09E2,1.03E2,2E0,2E0,1.02E2,7E0,2E0,1.01E2,6.1E1,4.1E1,3E0,4E0,3.9E1,6.2E1,5.5E1,6E0,2.2E1,1.9E1,8E0,3.1E1,6E1,2E0,1E1,4.5E1,1.1E1,1.1E1,1E1,9E0,5E0,3E0,2E1,1.1E1,2.6E1,3.4E1,2E0,8E0,3.7E1,8E0,7E0,4E0,5E0,6E0,1.7E1,3E0,8E0,3E0,1.5E1,1.1E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.6912411E-3,-1.9064195E-2,2.4100082E-2,-6.059589E-3,-7.711967E-2,4.095765E-2,-4.2764593E-2,-5.51559E-2,1.8715139E-3,-1.06207624E-1,-1.2241886E-3,1.0488696E-2,3.2134976E-2,-7.6291934E-3,-1.7113544E-2,5.660094E-4,-7.0297986E-2,5.156652E-3,-5.481648E-3,-6.6033505E-2,-8.217553E-3,7.82574E-3,2.3170885E-2,-4.49811E-2,1.0298288E-3,-4.236804E-3,9.818355E-4,2.7860066E-2,-4.866169E-3,-4.3798788E-3,-7.048719E-4,9.212592E-4,4.4673063E-2,-4.4644894E-3,-3.072789E-4,4.526607E-2,-1.4959747E-2,-3.7038807E-2,1.067402E-2,-6.5524145E-3,3.5458216E-3,2.2589561E-4,8.1912346E-2,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,25,27,-1,29,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,43,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.82962E-2,1.1127101E-1,9.417995E-2,4.7608547E-2,3.814037E-2,9.47225E-2,4.7029093E-2,1.8009417E-2,4.045555E-2,3.1635314E-2,0E0,0E0,7.009145E-2,0E0,1.6951097E-2,0E0,2.2984944E-2,2.3801006E-2,0E0,1.3520818E-2,0E0,0E0,2.9142186E-2,1.5691848E-2,0E0,0E0,0E0,2.4527468E-2,3.6893085E-2,0E0,0E0,1.7144382E-2,4.5832694E-2,0E0,0E0,1.8606756E-2,1.2692557E-2,1.1324577E-2,3.5423E-2,2.0130176E-2,0E0,0E0,1.3577841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,16,16,17,17,19,19,22,22,23,23,27,27,28,28,31,31,32,32,35,35,36,36,37,37,38,38,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,26,28,-1,30,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,44,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,2.325945E9,1.09495016E8,1.142E3,7.888087E0,4.164E3,2.2970924E-2,4.569361E7,1.2224206E7,8.260109E9,-1.2241886E-3,1.0488696E-2,1.1307017E3,-7.6291934E-3,8.66076E1,5.660094E-4,5.9215684E0,2.6374866E1,-5.481648E-3,5.754E3,-8.217553E-3,7.82574E-3,7.6E0,2.884462E0,1.0298288E-3,-4.236804E-3,9.818355E-4,7.438972E0,4.2060562E1,-4.3798788E-3,-7.048719E-4,7.361366E7,2.6023E4,-4.4644894E-3,-3.072789E-4,6.8623977E0,7.444958E7,6.37E2,6.191606E0,8.062193E-1,3.5458216E-3,2.2589561E-4,7.3586698E0,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"split_indices":[53,5,46,2,54,12,28,5,51,5,0,0,53,0,57,0,57,59,0,2,0,0,55,54,0,0,0,54,59,0,0,46,10,0,0,54,7,0,54,28,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.47E2,8.1E1,1.21E2,2.6E1,6.5E1,1.6E1,1.6E1,1.05E2,1.6E1,1E1,2E0,6.3E1,2E0,1.4E1,3E0,1.3E1,1.03E2,2E0,1.1E1,5E0,3E0,6E1,8E0,6E0,1.1E1,2E0,3.1E1,7.2E1,7E0,4E0,3E1,3E1,3E0,5E0,2.2E1,9E0,2.3E1,4.9E1,2.8E1,2E0,1.5E1,1.5E1,1.5E1,7E0,4E0,5E0,1.1E1,1.2E1,2.7E1,2.2E1,1.4E1,1.4E1,1.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.5013376E-3,5.938352E-3,-1.24217264E-1,-2.7496833E-3,8.3399355E-2,-8.301269E-3,-2.700483E-4,-3.2061845E-2,9.8344935E-3,1.079024E-2,4.594788E-2,-1.9474141E-2,-7.3742555E-3,6.1866198E-2,-1.161049E-3,4.8218677E-3,7.569026E-4,-8.262385E-2,-3.5967962E-3,9.7572885E-2,-1.3912333E-3,3.422547E-2,-7.907227E-3,-1.2076516E-3,-6.5928353E-3,5.179689E-2,-2.5634378E-2,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,-5.0402116E-2,2.8263181E-3,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,29,31,-1,-1,33,35,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.14531964E-1,1.3348015E-1,3.7561677E-2,6.634294E-2,9.341748E-2,0E0,0E0,7.825357E-2,7.215855E-2,0E0,2.5913164E-2,4.983531E-2,0E0,7.3844664E-2,2.5336621E-2,0E0,0E0,2.7188063E-2,5.1378243E-2,3.849563E-2,0E0,1.8305652E-2,4.0737204E-2,0E0,0E0,2.3600813E-2,3.192762E-2,0E0,0E0,0E0,0E0,2.3702014E-2,4.1213304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,22,22,25,25,26,26,31,31,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,30,32,-1,-1,34,36,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,8.861832E3,6.45153E5,4.039435E-3,1.4164306E-3,-8.301269E-3,-2.700483E-4,2.0604518E8,7.573E3,1.079024E-2,3.0904205E2,7.5999E4,-7.3742555E-3,8.19162E5,1.2402962E-1,4.8218677E-3,7.569026E-4,4.1749332E4,5.4127317E10,1.946E3,-1.3912333E-3,7.317201E0,3.0738266E4,-1.2076516E-3,-6.5928353E-3,7.356E3,1.9990385E1,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,3.7E1,2.2759746E7,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"split_indices":[46,4,29,39,58,0,0,7,9,0,59,1,0,30,42,0,0,33,32,2,0,54,34,0,0,9,59,0,0,0,0,3,46,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.97E2,6E0,1.78E2,1.9E1,4E0,2E0,5.3E1,1.25E2,3E0,1.6E1,4.9E1,4E0,2.1E1,1.04E2,5E0,1.1E1,9E0,4E1,1.5E1,6E0,1.6E1,8.8E1,5E0,4E0,1.1E1,2.9E1,3E0,1.2E1,1E1,6E0,1.7E1,7.1E1,2E0,9E0,2.1E1,8E0,1.3E1,4E0,5.2E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.211318E-5,-7.6537346E-3,7.079598E-2,6.928152E-3,-3.6480606E-2,1.4365091E-2,3.16417E-2,-9.043619E-3,5.276066E-2,-6.891165E-2,-9.199198E-3,6.183768E-2,-4.7232076E-3,7.881159E-3,-3.276541E-2,7.939335E-3,8.772806E-2,-3.0282011E-2,-1.1089681E-1,-1.8124288E-2,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,1.8954586E-2,-4.7628164E-2,2.5371954E-2,2.6573949E-3,-1.2165993E-2,5.2440288E-3,4.916817E-4,-1.1942704E-2,-4.229069E-3,-3.970585E-5,-1.548476E-1,2.0617556E-3,-4.2224076E-2,6.205118E-2,4.711567E-4,-7.918416E-2,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,2.3531545E-2,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1673426E-1,8.264138E-2,1.751531E-1,9.6372604E-2,5.776053E-2,0E0,7.98365E-2,3.9603867E-2,5.2591182E-2,4.529798E-2,3.7174046E-2,1.3060965E-2,0E0,4.1106842E-2,3.6635753E-2,1.5675733E-2,2.8447852E-2,1.6613591E-2,6.760767E-2,1.7113835E-2,0E0,0E0,0E0,0E0,3.9799407E-2,3.9343596E-2,1.4240889E-2,0E0,1.0493089E-2,0E0,0E0,1.1072869E-2,0E0,0E0,1.0195777E-2,1.9533398E-2,1.2358395E-2,2.6103884E-2,2.7533289E-2,3.5623252E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1920147E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,25,25,26,26,28,28,31,31,34,34,35,35,36,36,37,37,38,38,39,39,50,50],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,2.2971932E7,1.4164306E-3,7.7567725E2,9.875E3,1.4365091E-2,1.402511E8,1.7086E4,4.5470334E5,6.107924E11,3.5632E4,8.0426145E-1,-4.7232076E-3,5.421359E0,1.5071542E0,3.4706132E-3,4.493248E6,7.951E3,2.358119E-1,7.4500206E5,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,4.573327E5,4.4589956E5,1.69228E5,2.6573949E-3,1.8590495E9,5.2440288E-3,4.916817E-4,9.1487586E-1,-4.229069E-3,-3.970585E-5,1.96995E10,1.1036333E3,1.1192292E1,1.4214917E3,4.570007E7,5.6111975E0,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,3.3478114E-1,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"split_indices":[53,46,58,53,9,0,46,9,29,32,2,28,0,55,58,28,1,9,28,49,0,0,0,0,49,29,30,0,5,0,0,28,0,0,5,34,55,49,5,54,0,0,0,0,0,0,0,0,0,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.95E2,2E1,1.3E2,6.5E1,2E0,1.8E1,9.7E1,3.3E1,2.9E1,3.6E1,1.5E1,3E0,5.7E1,4E1,1.5E1,1.8E1,1.6E1,1.3E1,3.4E1,2E0,8E0,7E0,8E0,4.9E1,3.2E1,8E0,4E0,1.1E1,1.4E1,4E0,1.3E1,3E0,4E0,9E0,1.9E1,1.5E1,1.4E1,3.5E1,1.7E1,1.5E1,2E0,6E0,5E0,6E0,9E0,4E0,5E0,4E0,6E0,1.3E1,1.2E1,3E0,2E0,1.2E1,6E0,2.9E1,3E0,1.4E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.1967006E-3,8.5786525E-3,-1.4402847E-1,8.315427E-4,8.315179E-2,-9.1837915E-3,-1.7736525E-3,-1.9747915E-2,1.3641278E-2,1.3899025E-2,4.2534754E-2,-1.29205575E-2,-7.1898274E-2,3.6052603E-2,-8.618837E-3,3.79517E-4,7.254509E-2,-1.8197084E-2,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,2.5952812E-2,-1.9911975E-2,1.2206658E-3,1.6528742E-4,4.4683916E-3,-2.9289007E-2,1.7770747E-2,5.043673E-2,-5.733453E-3,2.2968883E-3,-2.780274E-2,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.356853E-1,1.1663157E-1,2.196373E-2,4.877499E-2,1.4408445E-1,0E0,0E0,2.4350522E-2,5.7182897E-2,0E0,1.7155953E-2,2.955227E-2,1.8783536E-2,5.395387E-2,2.2573741E-2,0E0,1.1681624E-2,2.485658E-2,0E0,0E0,0E0,0E0,4.1406788E-2,2.4611853E-2,0E0,0E0,0E0,1.7023295E-2,2.698188E-2,2.68047E-2,2.1004716E-2,0E0,2.7273526E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,22,22,23,23,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,9.417025E3,1.6330187E8,8.1913336E2,1.4164306E-3,-9.1837915E-3,-1.7736525E-3,1.996866E1,4.91101E5,1.3899025E-2,8.476884E-1,1.6482397E1,2.25813E5,6.37E3,1.0807868E7,3.79517E-4,3.5766065E0,3.8E1,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,6.995449E-1,2E1,1.2206658E-3,1.6528742E-4,4.4683916E-3,1.5662762E10,6.2734294E0,7.269755E0,1E0,2.2968883E-3,1.01618E5,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"split_indices":[46,4,46,56,58,0,0,57,30,0,35,57,1,9,1,0,36,3,0,0,0,0,28,3,0,0,0,32,55,54,101,0,11,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,2.01E2,5E0,1.83E2,1.8E1,3E0,2E0,7E1,1.13E2,2E0,1.6E1,6.3E1,7E0,5.6E1,5.7E1,8E0,8E0,6E1,3E0,2E0,5E0,4E0,5.2E1,4.3E1,1.4E1,2E0,6E0,4.6E1,1.4E1,2.9E1,2.3E1,4E0,3.9E1,2.2E1,2.4E1,4E0,1E1,2.3E1,6E0,1.7E1,6E0,2.1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.9543333E-3,-2.559017E-2,1.9161552E-2,-2.0923022E-2,-5.902389E-3,2.6990097E-2,-7.9133324E-2,-8.834234E-3,-4.04624E-2,1.3379109E-2,7.079206E-2,3.3201702E-4,-8.117297E-3,8.160095E-3,-2.9381018E-2,-7.949769E-2,-1.22999055E-2,4.4339493E-2,-1.4131938E-2,1.0944012E-1,-7.471799E-4,7.50054E-2,-6.7173177E-3,-5.5795915E-2,-1.1298074E-3,-5.1226197E-3,1.4864717E-3,-1.065279E-3,-4.5908066E-3,-3.7182914E-3,6.852332E-2,-5.39476E-2,8.511279E-3,1.29912235E-2,5.2558865E-2,4.981872E-4,4.811432E-3,-3.245346E-3,1.7857874E-3,-2.4376418E-2,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,8.3817184E-2,-7.3740636E-3,-2.728049E-2,5.5922586E-2,-1.4394672E-2,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,-1,45,47,49,51,-1,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0851875E-1,4.7482766E-2,8.293395E-2,2.5152944E-2,0E0,5.747927E-2,6.141771E-2,2.3983913E-2,4.4094764E-2,6.53598E-2,7.825514E-2,0E0,0E0,3.8294274E-2,2.30708E-2,5.1408485E-2,2.2939531E-2,4.2094745E-2,3.7422903E-2,1.2931412E-1,0E0,9.805635E-3,1.6436392E-2,1.2211446E-2,2.4566848E-2,0E0,0E0,9.913108E-3,0E0,1.5312212E-2,2.8017767E-2,3.489702E-2,3.0127015E-2,0E0,2.5919788E-2,0E0,0E0,0E0,1.9405605E-2,2.0426273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.549913E-2,0E0,1.4559356E-2,1.8774323E-2,1.5052079E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,34,34,38,38,39,39,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,-1,46,48,50,52,-1,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,8.260109E9,1.4623123E8,1.7086E4,-5.902389E-3,9.276886E1,1.5214152E5,4.327473E5,2.5259212E5,2.4934822E7,5.2E1,3.3201702E-4,-8.117297E-3,4.570007E7,4.9054803E2,4.758E3,4.1969254E8,2.8626094E5,5.8816573E9,1.6064256E-2,-7.471799E-4,7.444958E7,8.8916E4,2.79E2,6.027375E0,-5.1226197E-3,1.4864717E-3,1.194E3,-4.5908066E-3,1E0,1E0,2.9E1,3.800374E6,1.29912235E-2,3.5935396E-1,4.981872E-4,4.811432E-3,-3.245346E-3,2.0558643E8,3.6E1,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,1.5262272E3,-7.3740636E-3,4E0,5.57E2,2.5848E4,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"split_indices":[53,5,46,9,0,57,34,29,29,46,3,0,0,5,56,2,7,29,5,58,0,7,1,0,54,0,0,10,0,8,102,3,1,0,58,0,0,0,7,10,0,0,0,0,0,0,0,0,4,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.11E2,1.04E2,1.07E2,4E0,9.7E1,7E0,6.7E1,4E1,7.5E1,2.2E1,4E0,3E0,3.7E1,3E1,1.6E1,2.4E1,3.5E1,4E1,1.5E1,7E0,6E0,3.1E1,1.5E1,1.5E1,1.3E1,3E0,2.2E1,2E0,1.2E1,2.3E1,1.4E1,2.6E1,3E0,1.2E1,2E0,4E0,3E0,2.8E1,8E0,7E0,7E0,8E0,1.2E1,1E1,5E0,7E0,4E0,1.9E1,2E0,1.2E1,8E0,1.8E1,4E0,8E0,1.6E1,1.2E1,4E0,4E0,5E0,1.4E1,7E0,5E0,2E0,6E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.1127363E-3,-5.290152E-3,6.0248755E-2,-5.542424E-2,-1.2838513E-3,6.840342E-3,8.6930476E-4,1.3661059E-4,-7.317434E-2,-5.0637187E-3,6.27141E-4,-3.2042078E-4,-4.331261E-3,-7.5224843E-3,3.989758E-2,6.3668867E-3,-2.6071595E-2,-3.1950418E-2,6.536836E-2,-2.7029186E-2,1.4747367E-2,-7.087237E-2,-1.15542095E-2,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,7.527232E-2,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,-1,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8252895E-2,3.5278834E-2,3.8336206E-2,1.3642367E-2,3.2272287E-2,0E0,0E0,0E0,9.951666E-3,0E0,5.21465E-2,0E0,0E0,3.4946784E-2,5.3329237E-2,2.2228872E-2,3.690375E-2,1.3936162E-2,1.901877E-2,1.1226752E-2,2.018778E-2,3.677965E-2,1.6500767E-2,0E0,0E0,0E0,1.2472153E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,-1,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8008E4,1.9388489E3,7.9684106E9,6.051637E-1,1E1,6.840342E-3,8.6930476E-4,1.3661059E-4,7.891632E1,-5.0637187E-3,9.198947E-1,-3.2042078E-4,-4.331261E-3,8.729897E0,7.5773336E7,4.817292E0,2.9E1,6.7995167E0,1.4740072E1,5.9145586E3,7.137324E0,1.6753285E-1,1.6144748E5,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,1.2633E4,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"split_indices":[2,49,12,28,3,0,0,0,52,0,28,0,0,55,7,54,3,55,59,4,54,40,29,0,0,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.75E2,1.1E1,1.2E1,1.63E2,3E0,8E0,3E0,9E0,2E0,1.61E2,2E0,7E0,1.34E2,2.7E1,7.7E1,5.7E1,7E0,2E1,1.5E1,6.2E1,1.3E1,4.4E1,4E0,3E0,2E0,1.8E1,9E0,6E0,5.5E1,7E0,5E0,8E0,1.5E1,2.9E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-9.433804E-4,-1.2022093E-2,5.917883E-2,-6.3716695E-2,-6.556876E-3,1.185642E-2,3.261322E-2,-7.952249E-2,-4.8074138E-4,-1.3831026E-2,3.4149677E-2,4.693409E-2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,-4.5358995E-4,-3.4879703E-2,-6.250601E-4,2.4170529E-3,7.417909E-2,8.9431816E-4,1.8055465E-2,-2.7939798E-2,-7.63545E-2,-1.986685E-2,4.411847E-3,-8.005692E-4,-7.7093607E-3,6.321066E-2,1.0480897E-2,-5.4052684E-2,-4.8673116E-3,-6.69903E-4,-7.4792397E-3,-7.8966774E-2,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,27,29,31,33,-1,-1,35,37,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4125362E-1,5.015369E-2,1.5375312E-1,1.3680756E-2,4.8737537E-2,0E0,6.7347795E-2,8.856602E-3,0E0,3.9123423E-2,1.886667E-2,2.0422306E-2,0E0,0E0,0E0,4.4256426E-2,3.210645E-2,0E0,0E0,2.239582E-2,0E0,6.0984924E-2,3.5681155E-2,1.9354679E-2,2.9425774E-2,0E0,0E0,1.626718E-2,2.6215322E-2,1.7650409E-2,2.1931183E-2,0E0,0E0,1.2872141E-2,1.4290784E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,15,15,16,16,19,19,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,28,30,32,34,-1,-1,36,38,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,4.060294E0,1.4164306E-3,4.668851E6,2.6493E4,1.185642E-2,1.18E2,7.7573473E9,-4.8074138E-4,1.7086E4,7.4353185E0,3.0904205E2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,1.5989196E7,2.2741E4,-6.250601E-4,2.4170529E-3,1.2003011E8,8.9431816E-4,4.984018E8,7.149117E-1,4.4589956E5,1.3000701E7,4.411847E-3,-8.005692E-4,2.80373E5,7.11E2,2.2809816E5,5.36E3,-4.8673116E-3,-6.69903E-4,1.2318777E3,1.1075563E3,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"split_indices":[53,54,58,30,10,0,3,5,0,9,55,59,0,0,0,46,9,0,0,33,0,5,28,29,51,0,0,1,0,29,2,0,0,4,53,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.78E2,3.2E1,1.6E1,1.62E2,3E0,2.9E1,1.2E1,4E0,1.38E2,2.4E1,2.7E1,2E0,8E0,4E0,8.5E1,5.3E1,6E0,1.8E1,1.3E1,1.4E1,5.1E1,3.4E1,1.3E1,4E1,1.1E1,2E0,3.3E1,1.8E1,1.4E1,2E1,9E0,4E0,3.4E1,6E0,1.9E1,1.4E1,1.3E1,5E0,4E0,1E1,1.5E1,5E0,2.8E1,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.7782936E-4,-1.382795E-2,2.3196248E-2,-7.2672334E-3,-1.0122567E-1,-7.0918067E-3,4.573724E-2,-4.658554E-3,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,2.595186E-2,-2.5460321E-2,5.915973E-3,3.5621542E-2,-5.366179E-2,-6.7920005E-4,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,1.8883888E-2,8.107459E-2,-4.6013836E-3,-9.936781E-5,2.8179128E-2,-8.45359E-3,7.830855E-4,5.3368773E-2,7.481408E-3,2.3286117E-3,-1.5134966E-3,4.5415547E-2,-1.37019865E-2,1.673101E-3,3.2098924E-3,-1.2978952E-2,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,-1,-1,31,33,35,37,-1,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.788673E-2,7.380529E-2,5.4942183E-2,3.1929776E-2,2.121576E-2,2.1904452E-2,3.0952565E-2,2.3363702E-2,0E0,0E0,0E0,9.656656E-3,1.7785855E-2,0E0,3.0421652E-2,1.8802509E-2,2.5366073E-2,0E0,0E0,0E0,0E0,1.993274E-2,1.9198395E-2,0E0,0E0,2.5934385E-2,2.0478453E-2,2.058842E-2,1.0298643E-2,0E0,0E0,0E0,1.0949407E-2,2.1243747E-2,0E0,0E0,1.2440793E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,15,15,16,16,21,21,22,22,25,25,26,26,27,27,28,28,32,32,33,33,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,-1,-1,32,34,36,38,-1,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.2224206E7,3E0,1.256E3,5.0655737E0,1.1475E2,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,1.7837838E0,2.3894766E7,5.915973E-3,4.4E1,7.8206635E0,3.06123E5,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,3.4170952E11,1.2117E4,-4.6013836E-3,-9.936781E-5,5.1184835E0,2.6493E4,2.95E2,3.5935396E-1,7.481408E-3,2.3286117E-3,-1.5134966E-3,7.856E3,2.325945E9,1.673101E-3,3.2098924E-3,3.5120192E-1,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"split_indices":[53,5,102,51,8,0,55,53,0,0,0,58,49,0,3,54,12,0,0,0,0,32,9,0,0,54,10,0,58,0,0,0,9,5,0,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.29E2,7.9E1,1.21E2,8E0,3.4E1,4.5E1,1.19E2,2E0,4E0,4E0,1.2E1,2.2E1,4E0,4.1E1,8E0,1.11E2,6E0,6E0,7E0,1.5E1,3.1E1,1E1,4E0,4E0,2.3E1,8.8E1,2.1E1,1E1,2E0,8E0,5E0,1.8E1,7.9E1,9E0,3E0,1.8E1,2E0,8E0,3E0,1.5E1,7E1,9E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.1277195E-4,4.217555E-3,-1.0916315E-1,-2.9917555E-3,5.3992495E-2,5.0713454E-4,-7.7418964E-3,3.2400577E-3,-3.849527E-2,1.5364976E-1,2.9980375E-2,-2.0002978E-2,1.4616362E-2,-1.2040921E-2,-9.3005456E-2,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,-2.5916894E-3,-7.337681E-2,2.9357852E-2,-1.9776661E-2,-3.767159E-2,6.012406E-4,-6.2919264E-3,-7.461606E-4,4.0725708E-2,-2.1920465E-2,-5.1689288E-3,-1.9261596E-3,3.854466E-2,-1.4079762E-2,-5.6224447E-3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7122716E-2,7.65951E-2,4.344309E-2,4.149871E-2,6.000404E-2,0E0,0E0,4.2599544E-2,3.9257012E-2,6.30681E-2,1.8668272E-2,4.906311E-2,5.533707E-2,1.2970728E-2,2.3380555E-2,0E0,0E0,0E0,0E0,3.5220888E-2,1.0222927E-2,3.087072E-2,1.9976046E-2,2.0381412E-2,0E0,0E0,0E0,1.7245183E-2,1.5136076E-2,0E0,0E0,2.189073E-2,2.3911959E-2,1.0170521E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,22,22,23,23,27,27,28,28,31,31,32,32,33,33],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,4.7162656E3,8.667E3,1E0,1.3277E4,5.0713454E-4,-7.7418964E-3,2.427E3,2.7577372E11,4.2E1,4.970186E7,1.3017557E7,8.40547E5,1.6391889E5,1.485E3,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,6.522946E5,6.5657372E0,2.7883E4,1.7657008E7,2.1692245E1,6.012406E-4,-6.2919264E-3,-7.461606E-4,1E0,2.85898E10,-5.1689288E-3,-1.9261596E-3,1.2670352E1,1.5324128E6,2.016E3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"split_indices":[46,53,2,84,10,0,0,2,32,3,52,46,30,29,0,0,0,0,0,49,55,9,33,57,0,0,0,69,32,0,0,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,2.12E2,6E0,1.86E2,2.6E1,2E0,4E0,1.59E2,2.7E1,4E0,2.2E1,5.2E1,1.07E2,1.9E1,8E0,2E0,2E0,6E0,1.6E1,4E1,1.2E1,7.5E1,3.2E1,9E0,1E1,5E0,3E0,1.2E1,2.8E1,5E0,7E0,6.2E1,1.3E1,2.5E1,7E0,5E0,4E0,1E1,2E0,2.3E1,5E0,5.5E1,7E0,3E0,1E1,1.4E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.3611307E-3,-9.2408955E-3,4.410103E-2,-6.9312286E-3,-7.04401E-3,1.3789503E-1,4.378376E-3,-2.0679065E-3,-5.6651037E-2,1.4739492E-3,1.1579285E-2,3.072128E-2,-8.0794595E-2,-4.1738063E-2,2.7708148E-3,-8.478865E-2,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,8.560338E-3,-6.760063E-2,-1.666495E-3,6.8934974E-3,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,-8.954448E-2,3.4993854E-3,4.3400084E-3,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,33,-1,35,-1,-1,-1,-1],"loss_changes":[6.6894464E-2,5.595534E-2,9.999141E-2,4.340747E-2,0E0,7.891193E-2,4.7356185E-2,3.178787E-2,3.738624E-2,0E0,0E0,2.251287E-2,2.4343904E-2,2.3993906E-2,2.2820415E-2,1.3125837E-2,0E0,0E0,0E0,0E0,0E0,1.6774707E-2,1.7332826E-2,0E0,2.1932926E-2,0E0,0E0,0E0,0E0,0E0,9.104654E-3,0E0,2.9089697E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,21,21,22,22,24,24,30,30,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,34,-1,36,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.4311E4,1.04E3,2.3361332E6,-7.04401E-3,7.65E2,3.093317E5,1.278E3,6.8125E0,1.4739492E-3,1.1579285E-2,5.8199444E0,9.532E3,2.8434807E3,1.0458333E1,1E0,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,1.7081589E2,8.137731E7,-1.666495E-3,2.2413793E0,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,2.1367E4,3.4993854E-3,5.3473053E0,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"split_indices":[53,10,0,29,0,0,34,2,54,0,0,54,9,48,59,84,0,0,0,0,0,4,5,0,57,0,0,0,0,0,9,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.81E2,2.6E1,1.79E2,2E0,7E0,1.9E1,1.64E2,1.5E1,4E0,3E0,1.5E1,4E0,1.7E1,1.47E2,1.1E1,4E0,1E1,5E0,2E0,2E0,6E0,1.1E1,1.4E1,1.33E2,4E0,7E0,2E0,4E0,3E0,8E0,4E0,1.29E2,6E0,2E0,7E0,1.22E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4610173E-3,9.7932376E-2,-4.4504614E-3,8.870057E-3,4.9648836E-4,-3.314924E-2,7.020033E-3,-2.3121499E-2,-1.00224335E-2,-5.438503E-3,5.8383252E-2,1.6109983E-2,-4.7826532E-2,7.8370846E-5,-3.111012E-3,7.136404E-3,4.1926984E-2,-1.6382972E-2,4.455269E-2,-6.4636454E-2,-4.9594564E-5,-5.14796E-2,5.3385887E-3,-4.3677186E-4,4.9276084E-2,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,-1.0062985E-1,-8.23799E-2,7.526664E-4,2.452092E-2,-9.486034E-3,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,-1,35,-1,-1,-1,-1,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.105056E-2,4.253041E-2,6.5741345E-2,0E0,0E0,9.4003044E-2,9.1824725E-2,5.4077312E-2,0E0,3.7433997E-2,3.4517996E-2,2.1062033E-2,2.668634E-2,2.929389E-2,0E0,0E0,9.794302E-3,1.1263341E-2,8.910602E-3,2.421417E-2,0E0,2.1960407E-2,2.8100949E-2,0E0,1.2502909E-2,0E0,0E0,0E0,0E0,0E0,1.0933511E-2,1.6032834E-2,0E0,1.9169066E-2,2.5302555E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,21,21,22,22,24,24,30,30,31,31,33,33,34,34],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,-1,36,-1,-1,-1,-1,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.813798E2,8.1356537E2,1E0,8.870057E-3,4.9648836E-4,4.4816768E8,2.7123457E3,2E0,-1.00224335E-2,4.462437E7,1.6064256E-2,6.822511E5,1.3088399E7,8.35E2,-3.111012E-3,7.136404E-3,5.4907064E0,8.901E3,5.1270317E-2,2.041E3,-4.9594564E-5,6.974669E7,1.6533424E6,-4.3677186E-4,7.317201E0,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,3.606E3,4.1749332E4,7.526664E-4,1.4653994E6,6.191606E0,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"split_indices":[34,52,102,0,0,45,4,8,0,46,58,33,1,2,0,0,55,2,39,0,0,7,51,0,54,0,0,0,0,0,10,33,0,51,54,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,5E0,1.98E2,2E0,3E0,5.6E1,1.42E2,5.4E1,2E0,1.15E2,2.7E1,2.1E1,3.3E1,1.06E2,9E0,3E0,2.4E1,1E1,1.1E1,2.4E1,9E0,9E0,9.7E1,3E0,2.1E1,8E0,2E0,4E0,7E0,1.4E1,1E1,6E0,3E0,4.2E1,5.5E1,1.8E1,3E0,3E0,7E0,3E0,3E0,3.7E1,5E0,3.7E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[4.282986E-4,4.1160197E-3,-9.1818266E-2,-1.1841053E-3,7.4616544E-2,-6.1210035E-3,3.2609148E-4,-1.09339515E-2,2.124421E-2,1.0531628E-1,7.8503106E-4,-7.0198895E-3,-9.130692E-2,-6.0027227E-4,4.235613E-2,6.323839E-3,6.2048313E-4,-3.122984E-2,2.2667213E-3,-1.7209452E-3,-6.5925727E-3,4.2787664E-2,-5.5588E-4,8.872538E-3,5.4689284E-2,3.1641412E-3,-7.729943E-2,-7.908227E-3,3.704432E-2,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,19,21,23,-1,-1,25,27,-1,-1,29,-1,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0795104E-2,7.4820675E-2,2.7605474E-2,4.112406E-2,2.3438096E-2,0E0,0E0,4.056964E-2,2.6312627E-2,1.8859334E-2,0E0,2.8399166E-2,1.00320205E-2,1.410548E-2,1.1490557E-2,0E0,0E0,5.5702664E-2,3.28481E-2,0E0,0E0,9.601074E-3,0E0,9.688978E-3,9.379521E-3,1.8371563E-2,2.2842146E-2,2.3135118E-2,2.1546472E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,20,22,24,-1,-1,26,28,-1,-1,30,-1,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,5.4349E4,6.45153E5,1.1565725E0,2.0216698E7,-6.1210035E-3,3.2609148E-4,9.964687E-1,7.6126127E0,2.3274304E2,7.8503106E-4,6.126268E0,9.31E2,7.614E3,6.314179E2,6.323839E-3,6.2048313E-4,8.351808E-1,1.0685499E0,-1.7209452E-3,-6.5925727E-3,1.3559702E8,-5.5588E-4,2.6164518E7,1.5107028E-1,1.424506E6,9.313E3,6.4745765E0,6.6352534E-1,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"split_indices":[46,2,29,43,48,0,0,40,55,57,0,55,0,10,4,0,0,28,58,0,0,7,0,33,39,30,2,55,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.99E2,7E0,1.86E2,1.3E1,5E0,2E0,1.3E2,5.6E1,8E0,5E0,1.25E2,5E0,2.8E1,2.8E1,6E0,2E0,3.4E1,9.1E1,3E0,2E0,5E0,2.3E1,8E0,2E1,2E1,1.4E1,7.1E1,2E1,3E0,2E0,6E0,2E0,2E0,1.8E1,1E1,1E1,8E0,6E0,3E0,6.8E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.1866337E-3,-1.4964924E-3,7.432874E-2,-7.1397156E-2,2.0264261E-3,1.4760831E-1,2.5388267E-2,2.4828428E-4,-5.377491E-3,-2.0973684E-2,1.2397231E-2,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,-8.560193E-2,-1.29591115E-2,2.8582712E-4,3.782403E-2,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,6.9721565E-2,-6.839452E-3,-3.537567E-3,4.555346E-2,5.6208028E-5,4.49603E-3,-2.1360328E-2,1.4701819E-2,6.0154725E-2,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.437666E-2,4.619424E-2,6.0806543E-2,2.7483527E-2,4.295682E-2,4.940191E-2,1.3439575E-2,0E0,0E0,2.7896874E-2,3.8175512E-2,0E0,0E0,0E0,0E0,1.4757402E-2,1.4061235E-2,4.253052E-2,3.747972E-2,0E0,0E0,0E0,0E0,1.2324721E-2,2.471229E-2,0E0,2.2664323E-2,0E0,0E0,2.4652388E-2,2.1451397E-2,1.7983623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,23,23,24,24,26,26,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,5.314E3,1.256E3,6.035227E9,1.05190264E8,5.3696685E0,9.532E3,2.4828428E-4,-5.377491E-3,1.2338027E5,8.1350375E5,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,1.4084E4,1.7669278E5,1.3448397E8,6.0599937E-6,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,2.8E1,1.1812092E0,-3.537567E-3,2.7991664E-1,5.6208028E-5,4.49603E-3,5.791575E-1,3.4E1,1.3839568E1,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"split_indices":[4,9,0,5,7,54,9,0,0,29,29,0,0,0,0,9,29,7,39,0,0,0,0,3,43,0,39,0,0,42,3,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.86E2,1.7E1,8E0,1.78E2,6E0,1.1E1,3E0,5E0,5.5E1,1.23E2,3E0,3E0,4E0,7E0,5E0,5E1,8.4E1,3.9E1,2E0,3E0,3E0,4.7E1,7E0,7.7E1,2E0,3.7E1,2E0,5E0,4.6E1,3.1E1,2.7E1,1E1,3E1,1.6E1,8E0,2.3E1,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4136843E-3,-1.7078152E-2,1.7051285E-2,-4.9061432E-2,-3.8076276E-3,2.0602716E-2,-4.516626E-3,-3.270243E-4,-7.544991E-2,6.5981923E-3,-3.537236E-2,-7.553134E-3,3.3188965E-2,-5.2146493E-3,-2.832102E-2,-1.7100578E-2,3.0886639E-2,1.3335078E-3,-4.885492E-2,4.1569836E-4,-4.885816E-2,2.7083127E-2,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,4.4731785E-2,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,4.3737285E-2,-3.549587E-3,6.7125736E-3,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6693766E-2,4.4803295E-2,3.7026227E-2,3.4130327E-2,2.5196122E-2,3.143581E-2,0E0,0E0,2.535192E-2,3.3913348E-2,1.7419232E-2,1.9031767E-2,2.084884E-2,0E0,1.2429329E-2,1.4870551E-2,2.5628617E-2,0E0,1.2944881E-2,0E0,1.3053814E-2,2.9234491E-2,0E0,0E0,0E0,0E0,0E0,2.8370071E-2,0E0,0E0,0E0,0E0,0E0,1.1751018E-2,1.2866924E-2,1.5719058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,27,27,33,33,34,34,35,35],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9640406E-1,5.511724E0,1.2083455E1,4.958582E-1,9.952575E5,3.225E3,-4.516626E-3,-3.270243E-4,1.0702152E-2,1.643614E6,5.916872E0,2.806E3,1.6974416E6,-5.2146493E-3,5.815166E0,1E0,5.2192265E1,1.3335078E-3,4.4589956E5,4.1569836E-4,1.8465776E8,8.062193E-1,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,1.01978E5,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,1.0252E4,1.0266666E1,7.1706E4,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"split_indices":[40,54,36,28,52,2,0,0,39,33,54,2,29,0,55,106,57,0,29,0,5,28,0,0,0,0,0,30,0,0,0,0,0,9,55,30,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.94E2,1.05E2,8.9E1,3E1,7.5E1,8.7E1,2E0,1.2E1,1.8E1,5.7E1,1.8E1,2.7E1,6E1,1E1,8E0,2.9E1,2.8E1,3E0,1.5E1,2E1,7E0,5.6E1,4E0,6E0,2E0,2.7E1,2E0,2.3E1,5E0,1.2E1,3E0,3E0,4E0,3.6E1,2E1,1.1E1,1.2E1,1E1,2.6E1,1.6E1,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.348882E-3,-1.4007735E-2,2.0349778E-2,-8.801855E-3,-7.60288E-2,3.1053293E-2,-1.0958469E-2,-5.7257973E-2,-5.0193043E-3,-4.4670764E-5,-4.9110665E-3,2.440679E-2,1.0895903E-1,-3.7627418E-2,2.2707816E-2,-4.087384E-3,3.10375E-4,3.1430066E-2,-1.0879586E-2,4.352828E-3,4.6011005E-2,6.7795296E-3,1.5659243E-3,-6.1382737E-2,-3.0169598E-4,4.8260357E-5,2.7864138E-3,6.0025465E-2,-8.0559475E-5,-1.663934E-2,1.3024335E-3,1.8343031E-2,-1.7778914E-2,-1.4250753E-4,6.0098227E-2,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,8.744232E-3,-2.3721121E-2,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,6.921383E-2,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,-1,41,43,-1,45,-1,-1,-1,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.254653E-2,3.32326E-2,3.619688E-2,1.76748E-2,1.4859837E-2,3.8764708E-2,2.5994083E-2,1.244238E-2,1.9993477E-2,0E0,0E0,3.2293484E-2,9.498283E-3,1.1564847E-2,9.696336E-3,0E0,0E0,1.2398537E-2,1.7790027E-2,1.2711223E-2,2.4995036E-2,0E0,0E0,1.1450622E-2,0E0,0E0,0E0,1.1509789E-2,0E0,1.27906855E-2,0E0,9.788271E-3,8.809424E-3,0E0,1.9199878E-2,0E0,0E0,0E0,0E0,8.899177E-3,1.8784469E-2,0E0,0E0,0E0,0E0,1.800523E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,27,27,29,29,31,31,32,32,34,34,39,39,40,40,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,-1,42,44,-1,46,-1,-1,-1,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,3.6509445E1,1.3983983E10,1.1475E2,2.1434378E5,5.2086E4,1.5461028E12,8.842206E7,4.1740056E7,-4.4670764E-5,-4.9110665E-3,4.9754506E5,1.8136424E7,1.3088399E7,6.1921E4,-4.087384E-3,3.10375E-4,4.327473E5,2.6493E4,5.0123274E-1,1E0,6.7795296E-3,1.5659243E-3,1.775894E10,-3.0169598E-4,4.8260357E-5,2.7864138E-3,1.0380544E8,-8.0559475E-5,1.43E2,1.3024335E-3,1.1201525E2,2.513265E5,-1.4250753E-4,1.3578947E1,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,6.1579504E2,2.109E3,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,2.1525126E0,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"split_indices":[53,57,5,53,29,2,32,5,5,0,0,29,1,1,10,0,0,29,10,28,17,0,0,5,0,0,0,7,0,10,0,57,34,0,55,0,0,0,0,4,2,0,0,0,0,40,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.04E2,1.06E2,9.7E1,7E0,7.9E1,2.7E1,6E0,9.1E1,2E0,5E0,7.4E1,5E0,1.5E1,1.2E1,4E0,2E0,1.2E1,7.9E1,3.9E1,3.5E1,3E0,2E0,8E0,7E0,8E0,4E0,6E0,6E0,6.9E1,1E1,2.4E1,1.5E1,8E0,2.7E1,2E0,6E0,3E0,3E0,1.5E1,5.4E1,1.9E1,5E0,1.2E1,3E0,2.4E1,3E0,8E0,7E0,1.8E1,3.6E1,1.6E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.8318873E-3,-1.0389051E-2,6.847433E-2,-9.090252E-2,-7.818896E-3,1.4557077E-1,1.5369026E-2,-1.1227432E-3,-5.793796E-3,-3.6059227E-3,-5.7291586E-2,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,-1.0661409E-2,3.3012316E-2,-1.4323252E-3,-7.734146E-3,-6.9573727E-3,-4.916107E-2,4.209554E-3,2.0994669E-2,-5.233113E-2,-3.3860938E-3,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.4740467E-2,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[8.666744E-2,4.0054876E-2,5.917617E-2,8.709792E-3,3.9811417E-2,4.4877455E-2,9.926351E-3,0E0,0E0,4.6339944E-2,3.7794035E-2,0E0,0E0,0E0,0E0,2.1048589E-2,1.682442E-2,0E0,0E0,2.2184646E-2,1.6582798E-2,0E0,2.2704527E-2,1.6044017E-2,2.14594E-2,0E0,0E0,0E0,9.11136E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,19,19,20,20,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,2.3125E0,2E0,5.34E2,7.8032995E6,6.677E3,6.881048E-1,-1.1227432E-3,-5.793796E-3,1.6503105E3,5.8199444E0,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,9.5096946E-1,6.0775385E0,-1.4323252E-3,-7.734146E-3,1.2425E2,6.128552E0,4.209554E-3,5.590909E0,7.005162E5,1.2804077E7,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.842571E7,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"split_indices":[4,54,8,0,48,9,35,0,0,53,54,0,0,0,0,28,55,0,0,53,54,0,54,29,46,0,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.96E2,1.4E1,5E0,1.91E2,5E0,9E0,2E0,3E0,1.77E2,1.4E1,2E0,3E0,7E0,2E0,1.49E2,2.8E1,1.2E1,2E0,1.37E2,1.2E1,4E0,2.4E1,9E0,1.28E2,9E0,3E0,4E0,2E1,5E0,4E0,7.1E1,5.7E1,1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6622468E-4,-4.8488043E-3,5.08311E-2,-7.834397E-4,-6.2016364E-2,7.5922403E-3,1.9395813E-2,2.0835144E-3,-6.411687E-2,-4.91167E-3,-1.6307954E-3,4.5530796E-2,-4.4596207E-4,-8.775512E-3,2.092946E-2,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,-4.312229E-3,-4.002928E-2,-6.387113E-4,3.425055E-2,-6.662406E-4,-2.56261E-3,-3.120549E-3,1.352972E-4,4.647988E-2,-1.3092252E-3,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1110663E-2,4.5503184E-2,5.6485604E-2,3.354321E-2,1.1037149E-2,0E0,1.1613244E-2,3.640647E-2,3.848807E-2,0E0,0E0,9.618994E-3,0E0,1.5590253E-2,2.9935649E-2,0E0,0E0,0E0,0E0,1.7591223E-2,1.4617277E-2,0E0,2.0499848E-2,1.4878065E-2,0E0,0E0,0E0,2.9250316E-2,9.96951E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,22,22,23,23,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.8600049E10,1.4164306E-3,7.8032995E6,2.884462E0,7.5922403E-3,5.861021E7,6.978001E2,7E0,-4.91167E-3,-1.6307954E-3,2.819228E-1,-4.4596207E-4,1E0,2.8626094E5,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,3.6509445E1,4.966E3,-6.387113E-4,5.2192265E1,4.454691E5,-2.56261E-3,-3.120549E-3,1.352972E-4,9.42191E-1,2.6673E4,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"split_indices":[53,5,58,48,54,0,46,53,8,0,0,58,0,84,29,0,0,0,0,57,2,0,57,49,0,0,0,28,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.95E2,1.7E1,1.83E2,1.2E1,3E0,1.4E1,1.76E2,7E0,4E0,8E0,7E0,7E0,1.12E2,6.4E1,5E0,2E0,2E0,5E0,9.9E1,1.3E1,1.8E1,4.6E1,9.3E1,6E0,8E0,5E0,3.4E1,1.2E1,2.7E1,6.6E1,2.9E1,5E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.3941145E-4,-2.0441902E-3,8.919721E-3,-1.1523694E-2,2.1440972E-2,-9.1706775E-3,-5.6796446E-3,5.424757E-3,1.2543296E-2,-1.505701E-2,4.4461366E-2,1.9426743E-2,-3.729582E-3,-7.865752E-3,-3.6841E-2,3.1923247E-4,3.1530529E-3,-4.811276E-3,5.0493825E-2,-1.1715324E-2,1.7492641E-3,-1.6764663E-2,-3.0829909E-3,-2.7253285E-2,1.2633276E-2,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0149213E-1,4.410473E-2,0E0,3.3826478E-2,4.4260666E-2,4.4551205E-2,0E0,0E0,3.4023255E-2,1.944866E-2,1.0361386E-2,3.789317E-2,0E0,1.6611358E-2,1.5435919E-2,0E0,0E0,1.1697531E-2,2.4396595E-2,1.604245E-2,0E0,1.6484369E-2,0E0,1.106496E-2,1.5908504E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,23,23,24,24],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9028345E2,8.1350375E5,8.919721E-3,2.61E2,5.0655737E0,3.2432E4,-5.6796446E-3,5.424757E-3,7.251039E0,1.8747724E7,1.0252E4,7.6188426E0,-3.729582E-3,2.513265E5,6.995449E-1,3.1923247E-4,3.1530529E-3,1.2153278E-3,9.161318E0,4.8E1,1.7492641E-3,4.588941E-1,-3.0829909E-3,6.809655E0,2.0350546E-3,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"split_indices":[57,29,0,3,55,2,0,0,58,33,9,55,0,34,28,0,0,40,55,10,0,39,0,55,43,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.96E2,2E0,1.4E2,5.6E1,1.38E2,2E0,4E0,5.2E1,1.25E2,1.3E1,4.9E1,3E0,9.5E1,3E1,5E0,8E0,2.8E1,2.1E1,8.8E1,7E0,1.8E1,1.2E1,1.2E1,1.6E1,7E0,1.4E1,7E0,8.1E1,1.5E1,3E0,1E1,2E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.881544E-3,-6.655184E-3,2.188296E-2,-2.8302656E-3,-2.666419E-3,-2.7059872E-2,3.2757495E-2,-9.625506E-3,2.9928401E-2,-5.4137424E-2,7.742816E-7,2.5661085E-2,7.91566E-2,1.4555868E-3,-2.7334698E-2,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,3.091641E-2,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,-5.2867554E-2,1.0829677E-2,-4.0220562E-2,6.7792195E-3,6.600478E-2,1.1288791E-3,-7.709266E-4,-4.6325806E-3,-1.4827026E-2,2.3715133E-2,-2.952948E-4,-5.428135E-2,2.0235393E-3,-9.2693866E-4,8.701906E-2,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,35,37,-1,-1,-1,39,41,-1,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3877617E-2,2.2490527E-2,5.198053E-2,2.5887422E-2,0E0,1.3197487E-2,2.3980856E-2,1.8878557E-2,3.1391144E-2,9.128876E-3,0E0,1.9829843E-2,2.8112642E-2,3.112634E-2,1.6537398E-2,0E0,0E0,0E0,0E0,1.5805755E-2,0E0,0E0,0E0,1.2595711E-2,1.7542738E-2,1.2636464E-2,1.0678305E-2,1.2879096E-2,0E0,0E0,0E0,9.937875E-3,1.8333944E-2,0E0,1.3910651E-2,0E0,0E0,1.5306905E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,19,19,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,37,37],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,36,38,-1,-1,-1,40,42,-1,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2936785E3,3.0978662E6,3.673257E0,1.0907971E6,-2.666419E-3,4.8601307E-2,5.4349E4,2.2906403E-1,1E0,3.26E2,7.742816E-7,1.1778515E7,7.65E2,9.7043E4,4.4589956E5,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,7.573E3,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,4.0289572E2,3.1E1,1.0485419E8,1.2857332E6,1.1865996E1,1.1288791E-3,-7.709266E-4,-4.6325806E-3,1.5403727E-1,1.9540394E7,-2.952948E-4,1.6680729E0,2.0235393E-3,-9.2693866E-4,1.884E3,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"split_indices":[4,29,54,29,0,39,2,58,17,0,0,48,0,1,29,0,0,0,0,9,0,0,0,4,3,7,49,55,0,0,0,58,46,0,40,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.22E2,9.5E1,1.14E2,8E0,1.7E1,7.8E1,9.5E1,1.9E1,8E0,9E0,6.9E1,9E0,5.9E1,3.6E1,1E1,9E0,2E0,6E0,6.3E1,6E0,2E0,7E0,8E0,5.1E1,2.6E1,1E1,1E1,5.3E1,5E0,3E0,1.7E1,3.4E1,8E0,1.8E1,4E0,6E0,7E0,3E0,1.5E1,2E0,2.8E1,6E0,1.3E1,5E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.403546E-3,-9.570334E-4,-4.4911288E-2,-7.611177E-2,1.5534413E-3,-2.1605039E-2,-8.161089E-3,-5.0543556E-3,-6.690376E-4,-5.41183E-3,1.55741805E-2,-2.0407424E-3,4.387262E-4,-1.5254784E-3,-6.8485916E-2,3.5775187E-3,7.775442E-3,5.1245918E-3,-2.5670715E-2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,1.9513896E-2,2.5408894E-2,-7.420587E-3,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4000924E-2,3.5022937E-2,7.077891E-2,8.954152E-3,1.7662771E-2,1.4694375E-2,0E0,0E0,0E0,2.955602E-2,2.6538467E-2,0E0,0E0,1.8596815E-2,1.0216929E-2,0E0,1.7291361E-2,2.3345338E-2,1.594018E-2,0E0,0E0,0E0,8.91481E-3,1.4992485E-2,2.6731981E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,13,13,14,14,16,16,17,17,18,18,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.361366E7,1.0135136E0,7.5656E4,1.38E2,1.500621E3,3.0687964E3,-8.161089E-3,-5.0543556E-3,-6.690376E-4,1.2473612E7,4.965E3,-2.0407424E-3,4.387262E-4,2.1067844E7,1E0,3.5775187E-3,5.1034484E0,1.2623029E7,8.103274E2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,9.43017E5,2.1768606E5,1.1565725E0,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"split_indices":[46,57,10,0,4,53,0,0,0,51,2,0,0,46,8,0,54,12,4,0,0,0,49,29,43,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.84E2,2.5E1,5E0,1.79E2,2.2E1,3E0,3E0,2E0,1.2E2,5.9E1,1.3E1,9E0,1.14E2,6E0,6E0,5.3E1,9E1,2.4E1,2E0,4E0,1.6E1,3.7E1,3.4E1,5.6E1,7E0,1.7E1,6E0,3.1E1,7E0,2.7E1,2.8E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.779959E-6,1.8453373E-3,-5.9890207E-3,-1.285875E-3,6.8091586E-2,-5.2687E-3,3.303071E-2,8.70132E-3,5.314312E-4,-5.49993E-2,-3.4504228E-3,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,-8.831604E-3,1.3855618E-2,-1.8325018E-2,8.489807E-3,-3.5646225E-3,4.3126993E-2,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.611087E-2,4.1366342E-2,0E0,2.6279328E-2,5.5009544E-2,1.53631605E-2,2.091176E-2,0E0,0E0,1.1431506E-2,1.5674567E-2,0E0,0E0,0E0,0E0,2.1203808E-2,2.0550307E-2,1.3397509E-2,1.3573993E-2,1.2377562E-2,1.1399828E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6085916E7,2.0494681E2,-5.9890207E-3,1E0,9.900047E6,2.3125E0,1.04E3,8.70132E-3,5.314312E-4,3.26E2,2.6673E4,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,3.9E1,5.8167644E0,5.540166E-3,7.31E3,3.7988422E5,2.3E1,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"split_indices":[30,57,0,74,1,54,0,0,0,0,10,0,0,0,0,3,54,58,9,29,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.98E2,2E0,1.9E2,8E0,1.71E2,1.9E1,2E0,6E0,5E0,1.66E2,1.3E1,6E0,2E0,3E0,1.27E2,3.9E1,8.2E1,4.5E1,2.5E1,1.4E1,2.1E1,6.1E1,1.2E1,3.3E1,1E1,1.5E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.3686826E-4,-3.519467E-3,5.055597E-2,-2.117268E-3,-4.640776E-3,7.0017064E-3,1.523785E-2,-4.871748E-2,3.465182E-4,-9.892045E-4,4.3544497E-2,6.1831385E-4,-3.1955007E-3,-5.2515082E-2,2.6555143E-3,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,-1.2239224E-3,2.5115369E-2,9.1643715E-3,-1.2135973E-2,5.6670893E-2,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,15,-1,-1,17,19,-1,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0816244E-2,2.5682297E-2,4.724356E-2,2.2864658E-2,0E0,0E0,1.2965487E-2,1.1241557E-2,2.333033E-2,0E0,8.747165E-3,0E0,0E0,2.0471422E-2,1.5975818E-2,0E0,0E0,0E0,0E0,1.790836E-2,3.341707E-2,2.1425955E-2,2.8729388E-2,1.4656175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,16,-1,-1,18,20,-1,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,6.7948157E-1,1.0038867E4,1.2425E2,-4.640776E-3,7.0017064E-3,6.881048E-1,3.7790768E2,2.3125E0,-9.892045E-4,2.7445236E-1,6.1831385E-4,-3.1955007E-3,8.591E3,4.4E1,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,3.3E1,8.87E2,1.127938E6,4.570007E7,1.7E1,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"split_indices":[4,35,4,53,0,0,35,34,54,0,39,0,0,2,3,0,0,0,0,3,0,29,5,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,2E2,1.4E1,1.98E2,2E0,3E0,1.1E1,9E0,1.89E2,5E0,6E0,2E0,7E0,7E0,1.82E2,3E0,3E0,3E0,4E0,1.56E2,2.6E1,8E1,7.6E1,1.4E1,1.2E1,5.3E1,2.7E1,7E0,6.9E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.8720286E-3,2.203888E-3,-3.8517896E-2,7.5318073E-3,-6.002819E-4,-2.1684619E-2,-8.622627E-3,1.0079312E-3,-4.2458405E-3,-2.1083932E-3,1.5030096E-3,-9.4883353E-4,3.8221332E-3,2.3723133E-3,-7.695643E-4,2.9969336E-3,-3.9027993E-2,-1.1395355E-2,1.3584136E-2,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,13,15,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0922936E-2,7.110839E-2,6.3847005E-2,0E0,2.2939872E-2,1.3612385E-2,0E0,2.4651024E-2,0E0,0E0,1.2223788E-2,2.4006953E-2,0E0,0E0,0E0,2.2249037E-2,1.5829908E-2,9.478996E-3,2.061506E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,10,10,11,11,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,14,16,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,3.7902024E0,7.5656E4,7.5318073E-3,1.3105492E1,4.6E1,-8.622627E-3,5.9165E4,-4.2458405E-3,-2.1083932E-3,1E0,2.72258E5,3.8221332E-3,2.3723133E-3,-7.695643E-4,7.6077003E0,5.1737704E7,3.1347964E-2,2.6673E4,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"split_indices":[46,55,10,0,36,3,0,2,0,0,8,9,0,0,0,55,33,58,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.65E2,2.8E1,2E0,1.63E2,2.6E1,2E0,1.61E2,2E0,1.3E1,1.3E1,1.58E2,3E0,3E0,1E1,1.44E2,1.4E1,6.1E1,8.3E1,1.1E1,3E0,2.2E1,3.9E1,6.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-6.800024E-4,1.8960047E-3,-6.0736863E-3,-3.6225263E-2,6.760957E-3,-4.00987E-3,-1.3761937E-2,4.2979596E-3,5.311085E-3,1.561032E-3,-1.3046184E-3,-4.1869044E-2,6.551281E-3,-3.4434297E-3,1.4532126E-3,2.6219185E-3,3.280929E-2,-3.7609767E-3,2.3818783E-2,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,11,-1,-1,-1,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.290568E-2,3.6038343E-2,0E0,2.1682842E-2,4.2899136E-2,0E0,1.0427506E-2,1.7855216E-2,0E0,0E0,0E0,1.8689083E-2,1.662869E-2,0E0,0E0,1.9301808E-2,1.2423562E-2,2.420176E-2,2.810772E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,11,11,12,12,15,15,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,-1,10,12,-1,-1,-1,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.02E2,3.8834286E0,-6.0736863E-3,2.3443186E3,5.2086E4,-4.00987E-3,1.2633E4,1.8E2,5.311085E-3,1.561032E-3,-1.3046184E-3,1.3448806E8,4.6E1,-3.4434297E-3,1.4532126E-3,1.1482285E6,1.10597E5,8.351808E-1,3.1E1,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"split_indices":[8,54,0,4,2,0,2,11,0,0,0,7,3,0,0,29,30,28,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.92E2,3E0,2.1E1,1.71E2,6E0,1.5E1,1.68E2,3E0,3E0,1.2E1,7E0,1.61E2,5E0,2E0,1.41E2,2E1,1.09E2,3.2E1,3E0,1.7E1,8.7E1,2.2E1,2.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.1066506E-4,4.845112E-3,-3.1581253E-2,6.452437E-2,-6.428997E-4,-5.0642762E-2,5.648247E-4,1.1988164E-1,-7.721919E-4,-4.0026516E-2,3.5905282E-3,-2.9436452E-2,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,-2.7368726E-2,7.2603608E-3,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,1.2873769E-2,-1.3507972E-2,3.7806993E-3,1.0231802E-2,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1],"loss_changes":[3.406493E-2,5.8340125E-2,2.528421E-2,6.86559E-2,2.7487058E-2,4.734615E-2,0E0,2.8751194E-2,0E0,2.1383194E-2,1.7127382E-2,1.0340074E-2,0E0,0E0,0E0,0E0,0E0,2.1022052E-2,1.5795767E-2,0E0,0E0,0E0,0E0,1.7355684E-2,1.4121132E-2,0E0,1.356209E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,17,17,18,18,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1],"split_conditions":[1.8000048E10,6.677E3,3.244E3,5.1E1,1.142E3,3.08E3,5.648247E-4,5.8229775E3,-7.721919E-4,5.244755E-3,1.13002876E-4,7.520736E6,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,8.3949E4,7.384892E0,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,3.0404909E0,3.501498E7,3.7806993E-3,4.3E1,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"split_indices":[5,9,0,3,2,0,0,4,0,58,39,30,0,0,0,0,0,12,54,0,0,0,0,57,33,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.77E2,2.9E1,1.4E1,1.63E2,2E1,9E0,8E0,6E0,1.5E1,1.48E2,1.8E1,2E0,6E0,2E0,8E0,7E0,1.5E1,1.33E2,1.6E1,2E0,8E0,7E0,1.05E2,2.8E1,3E0,1.02E2,2.5E1,3E0,7.8E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-2.5543855E-3,-3.670569E-4,-9.478624E-2,-4.503718E-3,3.9570075E-2,-2.891256E-4,-7.399056E-3,7.92847E-3,-1.9427676E-2,1.2930362E-1,9.055213E-3,-2.102676E-2,1.859223E-2,-4.328215E-3,-3.512878E-2,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,6.881532E-3,-3.8464133E-2,3.4366657E-3,1.07829925E-2,-2.6562288E-3,8.551802E-5,-1.3136177E-2,-5.552462E-2,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,-1.9891918E-2,1.758985E-2,-5.6563243E-2,1.090067E-2,-8.1699066E-2,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,-1,-1,-1,-1,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3541346E-2,3.5201967E-2,2.452866E-2,3.5962764E-2,5.359327E-2,0E0,0E0,3.312072E-2,2.0710837E-2,1.6731478E-2,1.720418E-2,1.4377463E-2,3.0927734E-2,1.4162992E-2,1.8558595E-2,0E0,0E0,0E0,0E0,9.007291E-3,9.359658E-3,0E0,1.4757078E-2,0E0,0E0,2.358127E-2,2.1488503E-2,0E0,0E0,0E0,0E0,1.4124965E-2,1.671442E-2,8.771379E-3,1.1919808E-2,1.0015354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,25,25,26,26,31,31,32,32,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,-1,-1,-1,-1,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.25E2,5.2062725E3,2.133183E1,1.6278354E7,1.1823068E7,-2.891256E-4,-7.399056E-3,1.549E3,8.062193E-1,6.23E2,2.0090337E0,4.0431567E-3,1.26747945E5,1.775044E7,7.573E3,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,7.107292E6,8E-3,3.4366657E-3,3.104892E4,-2.6562288E-3,8.551802E-5,3.7614656E5,1.2067E4,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,4.9973932E-1,5.1821535E4,2.849513E3,9.8613055E4,9.377185E-1,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"split_indices":[8,53,55,46,1,0,0,2,28,0,35,43,48,46,9,0,0,0,0,46,58,0,34,0,0,29,9,0,0,0,0,28,34,4,34,28,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.11E2,4E0,1.92E2,1.9E1,2E0,2E0,1.05E2,8.7E1,4E0,1.5E1,2.8E1,7.7E1,4.5E1,4.2E1,2E0,2E0,1.3E1,2E0,1.1E1,1.7E1,9E0,6.8E1,4E0,4.1E1,2.1E1,2.1E1,5E0,6E0,4E0,1.3E1,1.2E1,5.6E1,7E0,1.4E1,1.2E1,9E0,9E0,3E0,1.2E1,4.4E1,4E0,3E0,1.1E1,3E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.7936366E-3,2.4298213E-3,-4.112654E-2,-4.0990007E-3,4.8309796E-2,-2.183913E-2,-8.530635E-3,2.3222237E-3,-3.93802E-2,1.3060242E-1,8.8548794E-4,-4.1647155E-2,3.9303087E-4,-8.698616E-3,2.1892874E-2,-5.8111317E-2,-5.4723965E-3,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,-4.9995827E-3,-3.3989844E-3,3.6737766E-2,-1.1948632E-2,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,-4.3598227E-2,7.2821265E-4,4.7861084E-2,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1747835E-2,5.5313785E-2,6.086246E-2,3.6792308E-2,5.4622788E-2,1.3965219E-2,0E0,2.9954968E-2,1.5560266E-2,4.6919495E-2,0E0,1.526756E-2,0E0,1.9780226E-2,2.5557641E-2,8.726589E-3,9.571268E-3,0E0,0E0,0E0,0E0,1.8823985E-2,0E0,1.4614824E-2,9.260894E-3,0E0,0E0,0E0,0E0,1.8716276E-2,2.0947833E-2,9.567626E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,21,21,23,23,24,24,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3849624E1,9.276886E1,1.019731E0,4.9386452E7,1.6064256E-2,1.7810251E1,-8.530635E-3,8.4182155E10,4.4E1,7.257E3,8.8548794E-4,2.6239506E5,3.9303087E-4,4.991491E8,1.7486866E7,8.3764136E2,9.39616E-1,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,6.974669E7,-3.3989844E-3,4.493248E6,2.6678162E9,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,1.0363E4,7.25E2,1.7747324E7,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"split_indices":[55,57,35,46,58,57,0,32,3,2,0,29,0,7,33,53,28,0,0,0,0,7,0,1,12,0,0,0,0,9,0,46,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.83E2,2.4E1,1.61E2,2.2E1,2.2E1,2E0,1.37E2,2.4E1,5E0,1.7E1,1.3E1,9E0,8.8E1,4.9E1,1.5E1,9E0,2E0,3E0,4E0,9E0,8.4E1,4E0,3.4E1,1.5E1,6E0,9E0,6E0,3E0,1E1,7.4E1,2.6E1,8E0,7E0,8E0,6E0,4E0,5E1,2.4E1,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.3284292E-3,-3.6234653E-3,6.495424E-2,-3.3305318E-4,-4.9157396E-2,9.216801E-3,3.1242464E-2,-1.132699E-2,1.368838E-2,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,-1.8146961E-3,-3.4096006E-2,5.2584518E-2,5.3565246E-3,-6.317367E-3,3.768521E-3,-8.287692E-2,-1.5646296E-2,4.7426983E-3,3.285665E-4,-1.2222636E-3,1.7330685E-3,-9.491318E-3,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,4.4616484E-3,-1.594493E-2,1.9072331E-2,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,27,29,-1,-1,31,-1,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5454505E-2,2.8899927E-2,5.700592E-2,2.8056335E-2,2.499102E-2,0E0,9.989295E-3,2.2029854E-2,2.564449E-2,0E0,0E0,0E0,0E0,2.65384E-2,2.5703099E-2,2.8132629E-2,1.3379129E-2,1.2622242E-2,0E0,1.1075061E-2,1.793515E-2,0E0,0E0,1.7036377E-2,0E0,1.3837478E-2,0E0,0E0,0E0,0E0,8.933199E-3,9.179806E-3,2.1313196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,28,30,-1,-1,32,-1,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.8032995E6,1.4164306E-3,4.7132755E6,5.7592998E0,9.216801E-3,6.5772717E8,2.1557302E6,6.505428E6,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,2.0041434E5,5.9083433E0,7.56E2,9.855582E5,1E0,3.768521E-3,1.1485E4,2.831E3,4.7426983E-3,3.285665E-4,3.6E1,1.7330685E-3,4.43E2,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,2.8037289E1,7.894435E-1,1.1156E4,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"split_indices":[4,48,58,33,54,0,33,33,33,0,0,0,0,34,54,0,29,106,0,9,2,0,0,3,0,0,0,0,0,0,59,28,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.92E2,1.4E1,1.8E2,1.2E1,2E0,1.2E1,1.01E2,7.9E1,1E1,2E0,1E1,2E0,7.2E1,2.9E1,1.3E1,6.6E1,6.9E1,3E0,7E0,2.2E1,6E0,7E0,5.5E1,1.1E1,6.6E1,3E0,3E0,4E0,7E0,1.5E1,3.2E1,2.3E1,3.1E1,3.5E1,3E0,1.2E1,2.1E1,1.1E1,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.613512E-4,9.168064E-3,-2.0914895E-2,2.484408E-3,9.1796964E-2,-1.50884595E-2,-5.1584444E-3,-3.610229E-2,6.524373E-3,2.7067963E-2,8.863301E-3,-3.4663703E-2,1.7349996E-2,-6.1991275E-4,-4.9615046E-3,2.1708814E-2,-6.561107E-3,2.5338226E-3,-1.149334E-3,-2.8247127E-2,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,3.0649362E-2,2.0936963E-3,-1.2641752E-2,-4.455137E-2,1.7206483E-3,3.736369E-3,4.129001E-2,-4.1820602E-3,-8.335901E-3,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,23,25,-1,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.960969E-2,7.377521E-2,3.0622425E-2,1.9721659E-2,5.3537726E-2,3.9970838E-2,0E0,1.7162973E-2,2.2793744E-2,1.0098371E-2,0E0,1.4024261E-2,1.5488645E-2,0E0,0E0,1.6034028E-2,1.923645E-2,0E0,0E0,1.8154921E-2,0E0,0E0,0E0,0E0,1.1873342E-2,0E0,1.6956246E-2,9.147577E-3,1.221883E-2,1.0724097E-2,1.0640074E-2,0E0,1.465915E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,19,19,24,24,26,26,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,24,26,-1,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7864855E11,1.3173E4,1.4146589E8,7.5999E4,5.4E9,1.0807868E7,-5.1584444E-3,5.6978E4,8.3333336E-2,7.438459E6,8.863301E-3,1.226189E0,3.093317E5,-6.1991275E-4,-4.9615046E-3,2.7E1,7.720203E4,2.5338226E-3,-1.149334E-3,8.305745E-1,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,5.2682075E5,2.0936963E-3,9.484755E4,7.4074075E-2,3.7791205E6,3.63E2,7.501344E0,-4.1820602E-3,3.6360288E8,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"split_indices":[32,2,46,1,5,1,0,1,58,1,0,40,34,0,0,3,29,0,0,58,0,0,0,0,33,0,29,58,48,0,54,0,5,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.33E2,6.4E1,1.24E2,9E0,6.1E1,3E0,1.1E1,1.13E2,6E0,3E0,3.8E1,2.3E1,9E0,2E0,5.2E1,6.1E1,4E0,2E0,3.6E1,2E0,1.9E1,4E0,1.1E1,4.1E1,6E0,5.5E1,2.3E1,1.3E1,1.2E1,2.9E1,2E0,5.3E1,1.9E1,4E0,6E0,7E0,9E0,3E0,2.3E1,6E0,1.6E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"3.6910433E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg index 98f2c7164..707eaec55 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"96"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0431683473","gamma":"0.00687538367","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0431683473","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.00687538367","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.601921976"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.036102999","gamma":"0.00912368763","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.036102999","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00912368763","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.603316069"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json index 1e16cab6c..b569c8da7 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_ObjectHashAggregate","sqlOp_Project","sqlOp_RunningWindowFunction","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_Window","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-azure","platform_databricks-aws","platform_dataproc","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"96"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.6600735E-2,-4.183997E-1,4.016022E-1,-5.4472244E-1,2.4702572E-3,-3.0444106E-1,5.4333526E-1,-2.095471E-1,-6.9834214E-1,1.584308E-2,-2.0032455E-1,-8.038212E-4,-2.285217E-2,3.2672933E-1,8.139915E-1,-3.9752656E-1,4.633975E-3,-3.312472E-2,-1.4200953E-2,-1.0207548E-2,-4.1459585E-3,-7.775269E-3,4.0403152E-1,3.959308E-2,1.9630998E-2,-1.9778838E-2,-8.943015E-3,6.3850624E-3,4.5272997E-1,5.177103E-1,1.0208632E-2,1.0884345E-2,2.4699572E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,29,31,-1,-1,-1],"loss_changes":[1.7139414E1,2.7659492E0,5.1688957E0,1.9132452E0,1.0345602E0,5.687549E-1,2.1963263E0,8.766462E-1,4.3567657E-1,0E0,1.1204004E-2,0E0,0E0,1.0522766E0,2.5014305E-1,8.993506E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.0743561E-1,0E0,0E0,0E0,0E0,0E0,1.3050103E-1,4.3749094E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,22,22,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,30,32,-1,-1,-1],"split_conditions":[3.3433047E-1,2.561146E3,2.80369E5,4.828443E-1,2.5927516E11,6.313997E-1,1.5364555E0,1.3564575E5,1.1653038E-2,1.584308E-2,9.067368E5,-8.038212E-4,-2.285217E-2,2.4719655E5,2.326E4,3.6126646E3,4.633975E-3,-3.312472E-2,-1.4200953E-2,-1.0207548E-2,-4.1459585E-3,-7.775269E-3,1.1462246E3,3.959308E-2,1.9630998E-2,-1.9778838E-2,-8.943015E-3,6.3850624E-3,5E1,2.3E1,1.0208632E-2,1.0884345E-2,2.4699572E-2],"split_indices":[51,64,1,39,43,50,54,40,50,0,44,0,0,44,9,4,0,0,0,0,0,0,64,0,0,0,0,0,3,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.1E1,4.9E1,3.9E1,1.2E1,8E0,4.1E1,1.3E1,2.6E1,4E0,8E0,4E0,4E0,2.4E1,1.7E1,8E0,5E0,2.1E1,5E0,5E0,3E0,3E0,2.1E1,1.2E1,5E0,5E0,3E0,4E0,1.7E1,1.2E1,5E0,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.9184384E-2,-3.0499905E-1,5.997157E-1,-4.659277E-1,1.48396E-1,8.087583E-1,2.2526638E-1,-2.4764006E-1,-7.373196E-1,1.7082155E-2,2.4644613E-2,1.0247265E-2,8.790802E-1,3.1603104E-1,-4.4611995E-3,-3.9349648E-1,3.966547E-2,-1.5637465E-2,-3.765108E-2,8.804465E-3,-7.8063235E-2,2.0825353E-2,4.1997816E-2,2.713279E-3,1.698415E-2,-2.1313876E-2,-2.0647657E-1,9.323187E-3,-6.674804E-2,-6.708936E-3,1.3110015E-3,-1.1347641E-2,-5.18309E-3,4.4390626E-4,-5.4863608E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,-1,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,31,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9282547E1,4.5929246E0,2.7922964E0,2.5425186E0,9.334084E-1,7.5565434E-1,4.8005348E-1,1.1470525E0,6.958008E-1,2.6484475E-1,0E0,0E0,3.120346E-1,2.1044922E-1,0E0,2.3550534E-1,2.0203052E-1,0E0,0E0,0E0,8.859264E-2,0E0,0E0,0E0,0E0,0E0,7.4212253E-3,0E0,3.3845585E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,15,15,16,16,20,20,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,-1,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,32,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,2.7938022E3,1.4046429E1,4.307821E7,6.082581E3,1.2723797E5,6.2E1,2.8866995E1,2.9870138E-1,5.42913E5,2.4644613E-2,1.0247265E-2,3.8902016E8,5.816E3,-4.4611995E-3,2.3812747E-2,1.9233563E3,-1.5637465E-2,-3.765108E-2,8.804465E-3,1.476E3,2.0825353E-2,4.1997816E-2,2.713279E-3,1.698415E-2,-2.1313876E-2,5.677647E2,9.323187E-3,1.6671585E9,-6.708936E-3,1.3110015E-3,-1.1347641E-2,-5.18309E-3,4.4390626E-4,-5.4863608E-3],"split_indices":[51,64,66,57,64,40,3,68,39,41,0,0,7,2,0,50,4,0,0,0,0,0,0,0,0,0,64,0,5,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.1E1,3.7E1,4.5E1,1.6E1,2.3E1,1.4E1,2.6E1,1.9E1,1.3E1,3E0,3E0,2E1,1.1E1,3E0,1.7E1,9E0,6E0,1.3E1,4E0,9E0,5E0,1.5E1,3E0,8E0,1E1,7E0,3E0,6E0,5E0,4E0,3E0,4E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.4356768E-2,-3.7128082E-1,4.407543E-1,-4.5008594E-1,1.7260667E-2,-6.633097E-2,6.278911E-1,-2.9126567E-1,-7.5154513E-1,-2.5443378E-1,1.2500662E-2,4.1944593E-1,8.257863E-1,-2.9460223E-2,-1.7724106E-1,-1.50001105E-2,-3.7997894E-2,-1.5439722E-3,-3.319633E-1,2.3743086E-1,2.2758026E-2,2.5669403E-2,4.4991527E-2,-3.4965125E-1,-9.865322E-2,-1.8370384E-2,-8.232185E-3,4.676384E-3,1.196992E-2,-2.05841E-2,-3.0850465E-3,5.5712583E-3,-1.7321905E-1,-1.2469928E-1,-1.1427813E-2,-5.9849926E-4,-1.5656334E-1,-8.922326E-3,-2.8103695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,-1,23,-1,-1,-1,25,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,33,35,-1,-1,37,-1,-1],"loss_changes":[1.894429E1,3.765318E0,5.355731E0,2.3665028E0,0E0,1.1448425E0,1.3448906E0,1.529412E0,6.3005924E-1,1.7461199E-1,0E0,3.062811E-1,3.808956E-1,0E0,3.7198615E-1,0E0,0E0,0E0,2.4613142E-2,2.1413982E-2,0E0,0E0,0E0,2.843449E-1,3.9590538E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2467535E-2,4.374084E-2,0E0,0E0,3.238748E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,18,18,19,19,23,23,24,24,32,32,33,33,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,-1,24,-1,-1,-1,26,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,34,36,-1,-1,38,-1,-1],"split_conditions":[1.5363757E8,7.621429E1,5.9176934E-1,5.070136E7,1.7260667E-2,4.7308203E3,7.3845965E8,1.497E3,5.3323513E-1,5.4470426E-1,1.2500662E-2,1.218666E0,2.5172096E-1,-2.9460223E-2,1E0,-1.50001105E-2,-3.7997894E-2,-1.5439722E-3,3.271557E8,1.0558E4,2.2758026E-2,2.5669403E-2,4.4991527E-2,2.0765625E1,2.9000288E-2,-1.8370384E-2,-8.232185E-3,4.676384E-3,1.196992E-2,-2.05841E-2,-3.0850465E-3,5.5712583E-3,1.0128521E6,1.5662651E0,-1.1427813E-2,-5.9849926E-4,3E1,-8.922326E-3,-2.8103695E-3],"split_indices":[12,68,53,57,0,4,7,2,39,39,0,51,50,0,24,0,0,0,7,9,0,0,0,70,50,0,0,0,0,0,0,0,59,68,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.8E1,5.5E1,5.3E1,5E0,1.5E1,4E1,3.6E1,1.7E1,1E1,5E0,2.1E1,1.9E1,7E0,2.9E1,5E0,1.2E1,3E0,7E0,9E0,1.2E1,1.1E1,8E0,8E0,2.1E1,3E0,4E0,3E0,6E0,5E0,3E0,5E0,1.6E1,1.2E1,4E0,3E0,9E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.3021187E-2,-3.8461128E-1,4.6652454E-1,-5.1976633E-1,4.3478012E-2,1.0299254E-1,8.0977285E-1,-7.162183E-1,-2.0501499E-1,1.5731439E-2,-1.2061651E-1,-1.4734499E-1,2.7444845E-1,2.5281912E-2,4.71969E-2,-7.621867E-1,-9.297812E-3,-3.069822E-1,4.3628835E-3,-1.0588974E-2,3.481588E-3,2.9117886E-3,-2.4903682E-1,2.034996E-2,1.1264696E-1,-1.3295847E-2,-8.043691E-1,-4.586784E-3,-3.8856316E-1,-2.9522204E-3,3.2152357E-3,-2.6087824E-3,-1.6204605E-2,1.2846985E-3,6.4377086E-3,-2.0125577E-2,-3.827385E-2,-1.974413E-2,-9.260202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,25,-1,27,-1,-1,29,-1,31,-1,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9499622E1,3.8991117E0,5.5317783E0,3.0066757E0,9.392591E-1,1.0276412E0,8.2127666E-1,5.4953384E-1,6.86442E-1,0E0,1.8652219E-1,2.3522893E-1,3.93057E-1,0E0,0E0,2.886963E-1,0E0,2.2072434E-1,0E0,0E0,4.081299E-2,0E0,1.4411876E-1,0E0,2.2777721E-2,0E0,2.819786E-1,0E0,3.3630252E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,15,15,17,17,20,20,22,22,24,24,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,26,-1,28,-1,-1,30,-1,32,-1,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.2480776E0,2.667836E6,6.15786E5,8.184232E5,1.5605432E9,3.5904072E3,2.2781514E8,1.5731439E-2,1.309E4,1E0,1.7644464E5,2.5281912E-2,4.71969E-2,2.3452184E5,-9.297812E-3,1.1954737E1,4.3628835E-3,-1.0588974E-2,9.638E3,2.9117886E-3,8.44629E7,2.034996E-2,4.340646E-1,-1.3295847E-2,4.4E1,-4.586784E-3,1.4731E4,-2.9522204E-3,3.2152357E-3,-2.6087824E-3,-1.6204605E-2,1.2846985E-3,6.4377086E-3,-2.0125577E-2,-3.827385E-2,-1.974413E-2,-9.260202E-3],"split_indices":[62,2,51,44,41,44,7,4,7,0,9,13,40,0,0,40,0,66,0,0,2,0,57,0,51,0,10,0,9,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,6.6E1,4.4E1,5E1,1.6E1,2.2E1,2.2E1,3E1,2E1,5E0,1.1E1,9E0,1.3E1,1.4E1,8E0,2.7E1,3E0,1.5E1,5E0,5E0,6E0,3E0,6E0,5E0,8E0,3E0,2.4E1,5E0,1E1,3E0,3E0,3E0,3E0,3E0,5E0,6E0,1.8E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.247382E-2,-4.2130184E-1,4.5202878E-1,-2.586077E-1,-6.749045E-1,1.9564748E-1,7.5367093E-1,-3.9518073E-1,9.299449E-2,-1.5469744E-2,-3.5000037E-2,-1.0435869E-2,2.9279652E-1,2.4303524E-2,8.9743155E-1,-6.2369406E-1,-1.7053272E-1,1.938201E-2,-1.4112487E-1,4.2348337E-1,8.796934E-2,2.315018E-2,4.656951E-2,-3.2832623E-2,-1.8388182E-2,-2.2096992E-1,-6.637954E-4,-5.3356437E-4,-1.15173E-2,2.3217133E-1,2.4254566E-2,-2.8230796E-2,1.0060628E-2,-4.5429533E-3,-1.1469013E-2,5.789617E-3,1.2807884E-2,-3.6090114E-3,1.4763272E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,-1,27,29,31,-1,-1,-1,-1,33,-1,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.195361E1,2.4248238E0,3.9513226E0,1.9575248E0,7.4171066E-1,1.3455603E0,1.5263653E-1,1.3823566E0,1.063499E0,0E0,0E0,0E0,6.319518E-1,0E0,2.0744514E-1,5.493021E-2,1.21812195E-1,0E0,1.2616225E-1,2.66649E-1,1.920289E-1,0E0,0E0,0E0,0E0,3.413433E-2,0E0,0E0,0E0,1.0823369E-2,0E0,2.7057495E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,15,15,16,16,18,18,19,19,20,20,25,25,29,29,31,31],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,-1,28,30,32,-1,-1,-1,-1,34,-1,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,5.4875424E7,1.3536367E0,1.0664373E3,2.9385806E5,4.739085E-2,1.286E4,2.667836E6,1.7533583E1,-1.5469744E-2,-3.5000037E-2,-1.0435869E-2,9.48E2,2.4303524E-2,2.5172096E-1,2.3E1,2.3514317E8,1.938201E-2,8.62075E5,1.65326E6,6.9015497E-1,2.315018E-2,4.656951E-2,-3.2832623E-2,-1.8388182E-2,2.5445784E1,-6.637954E-4,-5.3356437E-4,-1.15173E-2,9.745363E-1,2.4254566E-2,2.1E1,1.0060628E-2,-4.5429533E-3,-1.1469013E-2,5.789617E-3,1.2807884E-2,-3.6090114E-3,1.4763272E-3],"split_indices":[12,57,51,64,40,50,2,44,70,0,0,0,0,0,50,3,7,0,1,1,51,0,0,0,0,70,0,0,0,54,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6.2E1,5.2E1,3.9E1,2.3E1,2.9E1,2.3E1,2.8E1,1.1E1,8E0,1.5E1,5E0,2.4E1,1.2E1,1.1E1,1.3E1,1.5E1,4E0,7E0,1.4E1,1E1,5E0,6E0,6E0,7E0,1.1E1,4E0,4E0,3E0,7E0,7E0,6E0,4E0,4E0,7E0,4E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.0993204E-2,-3.2869467E-1,4.7597894E-1,-5.119397E-1,-9.620262E-2,2.3545599E-1,7.062647E-1,-6.7831194E-1,-1.7820878E-1,1.6909474E-1,-2.6772293E-1,3.097452E-1,-9.21326E-3,3.369111E-2,1.835451E-2,-1.5816377E-2,-3.2143258E-2,-1.6527192E-3,-9.339784E-3,-1.09490864E-1,2.5081359E-2,-3.4002763E-1,-2.8682034E-3,3.686957E-4,3.8011438E-1,-1.042412E-2,7.768329E-4,-1.6418252E-2,-6.6592693E-3,4.4013268E-1,6.390814E-3,2.2084547E-2,1.1720172E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,-1,-1,-1,-1,25,-1,27,-1,-1,29,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[1.7215052E1,2.6347399E0,2.3816004E0,1.822444E0,1.3648832E0,8.6643827E-1,1.4781284E-1,2.0277119E-1,6.1108828E-2,1.4406686E0,2.3937142E-1,4.41859E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3895583E-1,0E0,3.8658977E-2,0E0,0E0,1.716249E-1,0E0,0E0,0E0,0E0,1.7757416E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,19,19,21,21,24,24,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,-1,-1,-1,-1,26,-1,28,-1,-1,30,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[1.8147125E5,3.3041668E1,1.3536367E0,2.9282052E6,6.15786E5,2.0195505E-1,2.1685E4,8.977316E0,3.6967509E0,3.09E2,2.7938022E3,4.1451712E5,-9.21326E-3,3.369111E-2,1.835451E-2,-1.5816377E-2,-3.2143258E-2,-1.6527192E-3,-9.339784E-3,6.336E3,2.5081359E-2,2.1915238E8,-2.8682034E-3,3.686957E-4,9.348536E-1,-1.042412E-2,7.768329E-4,-1.6418252E-2,-6.6592693E-3,1.5625807E1,6.390814E-3,2.2084547E-2,1.1720172E-2],"split_indices":[45,68,51,44,41,50,9,66,68,0,64,44,0,0,0,0,0,0,0,2,0,7,0,0,39,0,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.2E1,4.5E1,3.4E1,2.8E1,2.3E1,2.2E1,2.2E1,1.2E1,1.1E1,1.7E1,2E1,3E0,1.6E1,6E0,5E0,1.7E1,3E0,9E0,7E0,4E0,1.2E1,5E0,4E0,1.6E1,3E0,4E0,9E0,3E0,1.2E1,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.590726E-2,-2.6564786E-1,5.209033E-1,-3.4559515E-1,2.3260586E-1,1.5141532E-1,6.5583265E-1,-4.8569652E-1,-1.6156031E-1,2.0831615E-2,-6.243272E-4,8.819874E-3,1.5078055E-2,3.5418943E-1,7.959507E-1,-2.085015E-1,-5.971508E-1,1.5636977E-3,-1.8779255E-1,-2.497258E-3,2.6069894E-3,1.7661836E-2,6.7231995E-3,3.7542928E-2,1.3037699E-2,-1.2848796E-2,9.457621E-4,-3.6344275E-1,-3.0112753E-2,-1.1388204E-1,-2.3245135E-1,-5.5881105E-3,-1.9808426E-2,-7.608567E-3,-2.2906086E-3,-1.2086348E-2,-6.256113E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1],"loss_changes":[1.558047E1,2.7891011E0,1.8948879E0,1.44276E0,6.2455404E-1,3.3550313E-1,9.437866E-1,8.950138E-1,1.4688677E-1,0E0,0E0,3.0999336E-2,0E0,5.6227922E-2,4.293251E-1,2.3245355E-1,2.8623295E-1,0E0,5.2746058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4155316E-1,0E0,3.2375157E-2,1.7667234E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,18,18,27,27,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,5.188755E3,1.6492815E3,1.1816701E-2,6.15786E5,1.8589372E5,1.3698871E0,5.3323513E-1,2.6431797E5,2.0831615E-2,-6.243272E-4,3.1E2,1.5078055E-2,1.2948874E0,2.1685E4,2.4232728E5,5.070136E7,1.5636977E-3,5.990564E-1,-2.497258E-3,2.6069894E-3,1.7661836E-2,6.7231995E-3,3.7542928E-2,1.3037699E-2,-1.2848796E-2,9.457621E-4,7.690597E-4,-3.0112753E-2,2.5E1,1.3890547E1,-5.5881105E-3,-1.9808426E-2,-7.608567E-3,-2.2906086E-3,-1.2086348E-2,-6.256113E-3],"split_indices":[51,4,64,50,41,45,54,39,59,0,0,0,0,54,9,40,57,0,39,0,0,0,0,0,0,0,0,51,0,3,68,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,6.7E1,3.9E1,5.8E1,9E0,1.1E1,2.8E1,3.2E1,2.6E1,4E0,5E0,7E0,4E0,1E1,1.8E1,1E1,2.2E1,3E0,2.3E1,3E0,4E0,7E0,3E0,1.5E1,3E0,7E0,3E0,8E0,1.4E1,1E1,1.3E1,3E0,5E0,4E0,6E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-4.6974998E-2,-3.6099848E-1,3.8230646E-1,-4.7264844E-1,5.4613273E-3,1.3177113E-1,2.8077878E-2,-6.097007E-1,-2.2558379E-1,1.3707804E-2,-1.8233378E-1,3.2965675E-1,1.2224771E-2,-1.1010243E-2,-6.562421E-1,-1.4421758E-2,-7.996562E-2,-1.4261249E-2,1.3714043E-3,5.9419563E-3,1.9673372E-2,-1.3447106E-1,1.7572948E-1,-3.8833547E-1,-3.1895097E-2,4.527459E-4,-6.6663167E-3,-1.2813335E-2,2.3645272E-3,3.7049374E-4,1.1841127E-2,-2.1079637E-2,-7.1780533E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,23,-1,25,-1,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4022869E1,2.4574952E0,2.869938E0,1.4200563E0,9.370098E-1,5.637262E-1,0E0,3.1129074E-1,2.5767195E-1,0E0,3.2743078E-1,1.5514678E-1,4.073979E-1,0E0,2.6638317E-1,0E0,6.2236466E-2,0E0,0E0,0E0,0E0,2.927769E-1,1.294534E-1,9.64061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,14,14,16,16,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,24,-1,26,-1,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,2.667836E6,6.15786E5,1.1E1,2.8077878E-2,2.3452184E5,1.6757976E-1,1.3707804E-2,2.4502628E6,1.0262E4,2.3342845E5,-1.1010243E-2,4.4E1,-1.4421758E-2,1.675502E8,-1.4261249E-2,1.3714043E-3,5.9419563E-3,1.9673372E-2,1.4634906E-1,6.082581E3,7.158023E7,-3.1895097E-2,4.527459E-4,-6.6663167E-3,-1.2813335E-2,2.3645272E-3,3.7049374E-4,1.1841127E-2,-2.1079637E-2,-7.1780533E-3],"split_indices":[12,2,51,44,41,8,0,40,51,0,44,2,45,0,10,0,5,0,0,0,0,50,64,7,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.9E1,4.3E1,4.5E1,1.4E1,2.3E1,2E1,2.8E1,1.7E1,5E0,9E0,8E0,1.5E1,4E0,2.4E1,9E0,8E0,5E0,4E0,4E0,4E0,8E0,7E0,7E0,1.7E1,4E0,4E0,4E0,4E0,3E0,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.4338576E-2,-3.2437614E-1,3.0766904E-1,-3.9686412E-1,2.0715468E-2,1.684612E-1,5.4791147E-1,-5.330774E-1,-1.5907955E-1,-1.0510831E-2,2.2946987E-1,1.4976809E-2,2.7465144E-2,-6.1346686E-1,-1.5833418E-1,-2.4829881E-1,3.9059534E-3,2.754059E-1,-6.4515076E-3,-3.4713402E-1,-6.933193E-1,-9.961482E-3,-1.999812E-3,-1.5084888E-2,-1.4204215E-1,3.3344537E-1,8.862658E-2,-1.8399175E-2,-6.971434E-3,-3.5372216E-2,-4.6218315E-1,-2.1794029E-3,-7.744652E-3,3.766144E-1,5.1794113E-3,7.2250576E-3,3.4133458E-4,-8.917535E-3,-2.3982499E-2,6.8125227E-3,4.1222805E-1,2.0841524E-2,8.143524E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,21,23,-1,25,-1,27,29,-1,-1,-1,31,33,35,-1,-1,-1,37,-1,-1,39,-1,-1,-1,-1,-1,-1,41,-1,-1],"loss_changes":[1.0782088E1,3.3968697E0,1.6802106E0,1.5670595E0,0E0,9.355015E-1,1.1651993E-1,8.668814E-1,4.6778035E-1,0E0,5.8074796E-1,0E0,0E0,3.1227493E-1,4.6095118E-2,1.1157334E-1,0E0,2.7398396E-1,0E0,4.8618913E-2,1.7498875E-1,0E0,0E0,0E0,2.1730304E-2,1.4832854E-1,4.952523E-2,0E0,0E0,0E0,1.0003996E-1,0E0,0E0,6.741452E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2981224E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,17,17,19,19,20,20,24,24,25,25,26,26,30,30,33,33,40,40],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,22,24,-1,26,-1,28,30,-1,-1,-1,32,34,36,-1,-1,-1,38,-1,-1,40,-1,-1,-1,-1,-1,-1,42,-1,-1],"split_conditions":[2.0020565E8,1.0269E4,1.3536367E0,2.9282052E6,2.0715468E-2,3.0962205E-2,4.1372877E8,3.159013E3,2.1816934E8,-1.0510831E-2,2.641E3,1.4976809E-2,2.7465144E-2,4.4E1,1.2179555E-1,2.3315548E6,3.9059534E-3,1.8634315E1,-6.4515076E-3,1.16E2,1.2599278E1,-9.961482E-3,-1.999812E-3,-1.5084888E-2,2.5682794E5,9.014471E-1,2.0711127E5,-1.8399175E-2,-6.971434E-3,-3.5372216E-2,1.2875913E1,-2.1794029E-3,-7.744652E-3,2.8971782E6,5.1794113E-3,7.2250576E-3,3.4133458E-4,-8.917535E-3,-2.3982499E-2,6.8125227E-3,1.5625807E1,2.0841524E-2,8.143524E-3],"split_indices":[12,2,51,44,0,50,7,4,7,0,0,0,0,10,39,60,0,66,0,0,65,0,0,0,40,39,40,0,0,0,65,0,0,60,0,0,0,0,0,0,66,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,5.4E1,5.2E1,5E1,4E0,3.4E1,1.8E1,3.1E1,1.9E1,4E0,3E1,7E0,1.1E1,2.5E1,6E0,1.4E1,5E0,2.7E1,3E0,7E0,1.8E1,3E0,3E0,6E0,8E0,2E1,7E0,4E0,3E0,1E1,8E0,3E0,5E0,1.6E1,4E0,3E0,4E0,3E0,5E0,3E0,1.3E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-6.1048805E-3,-3.7457976E-1,3.7888232E-1,-5.770881E-1,-7.233911E-2,2.1148103E-1,6.995862E-1,-3.2538906E-1,-6.6149825E-1,-1.612411E-1,1.3968376E-2,4.088399E-1,4.6230737E-2,1.3538106E-2,7.6994544E-1,-7.293806E-3,-2.0663934E-2,-7.064796E-1,-1.31643135E-2,-2.1693973E-1,3.195079E-3,2.6285216E-1,3.0567346E-2,-1.16785236E-1,2.6459885E-1,3.989258E-2,1.9933203E-2,-1.52871255E-2,-3.2665E-2,-1.0590914E-2,-3.6522415E-3,1.8146776E-2,1.5123561E-1,-1.8116814E-1,1.7775374E-3,1.637872E-2,2.3711917E-3,8.071898E-3,2.9673108E-3,-1.3104974E-2,-2.0291868E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,-1,-1,27,-1,29,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3051011E1,2.8349476E0,2.267932E0,3.7964344E-1,7.561326E-1,9.921211E-1,1.6588879E-1,1.3493425E-1,1.0531521E-1,2.3881462E-1,0E0,4.2550373E-1,6.716561E-1,0E0,1.4345264E-1,0E0,0E0,3.278637E-2,0E0,3.1859696E-2,0E0,1.2982506E-1,0E0,1.19330585E-1,1.7174709E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0741755E-2,1.1711204E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,17,17,19,19,21,21,23,23,24,24,32,32,33,33],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,-1,-1,28,-1,30,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,4.852701E6,1.3536367E0,6.072427E-1,7.031107E6,8.69374E5,2.1E1,4.6268812E5,1.174E3,2.3514317E8,1.3968376E-2,4.916426E6,2.3342845E5,1.3538106E-2,4.4E1,-7.293806E-3,-2.0663934E-2,1.35E2,-1.31643135E-2,4.1399838E6,3.195079E-3,3.916934E1,3.0567346E-2,1.4634906E-1,2.3152558E9,3.989258E-2,1.9933203E-2,-1.52871255E-2,-3.2665E-2,-1.0590914E-2,-3.6522415E-3,1.8146776E-2,5.3433334E1,2.9870138E-1,1.7775374E-3,1.637872E-2,2.3711917E-3,8.071898E-3,2.9673108E-3,-1.3104974E-2,-2.0291868E-3],"split_indices":[2,44,51,39,60,41,3,44,10,7,0,1,45,0,3,0,0,0,0,60,0,70,0,50,7,0,0,0,0,0,0,0,68,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9E1,4.6E1,4.4E1,2.7E1,1.9E1,3E1,1.4E1,8E0,1.9E1,1.6E1,3E0,1.3E1,1.7E1,3E0,1.1E1,5E0,3E0,1.6E1,3E0,1.3E1,3E0,1E1,3E0,1E1,7E0,6E0,5E0,3E0,1.3E1,1E1,3E0,3E0,7E0,7E0,3E0,4E0,3E0,4E0,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-3.2901924E-2,-2.9736722E-1,4.4940415E-1,-4.154156E-1,7.32984E-2,-1.0883965E-2,5.36783E-1,-2.3850997E-1,-5.7116693E-1,-4.215241E-2,2.1062454E-2,3.8894626E-1,8.2397693E-1,-2.0389063E-2,-1.6390356E-1,-6.36192E-1,-1.2311911E-2,-1.0173181E-1,4.0471457E-2,2.2826691E-1,2.0075945E-2,1.9367278E-2,4.206151E-2,2.6643458E-3,-2.17471E-1,-1.2442433E-2,-2.9180221E-2,-1.879688E-3,-6.025181E-3,4.9434775E-3,-1.8860773E-3,4.400296E-3,1.1562552E-2,-4.3899044E-3,-1.0972471E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,-1,23,25,-1,27,29,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4036938E1,3.1371589E0,2.526002E0,1.3516483E0,8.821862E-1,0E0,1.1655407E0,3.935567E-1,3.3920193E-1,7.7957444E-2,0E0,1.9959617E-1,1.8373632E-1,0E0,2.7931583E-1,1.09622E-1,0E0,1.3740383E-2,4.862601E-2,2.2701442E-2,0E0,0E0,0E0,0E0,5.0650895E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,17,17,18,18,19,19,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,-1,24,26,-1,28,30,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,2.5323425E3,2.80369E5,7.332772E-1,6.1596104E3,-1.0883965E-2,1.62739E9,8.366883E0,2.4502628E6,4.7308203E3,2.1062454E-2,2.0681982E-1,2.0105577E-1,-2.0389063E-2,2.01E2,9.52E2,-1.2311911E-2,5.4470426E-1,7.3153823E3,2.435464E3,2.0075945E-2,1.9367278E-2,4.206151E-2,2.6643458E-3,1.19288025E1,-1.2442433E-2,-2.9180221E-2,-1.879688E-3,-6.025181E-3,4.9434775E-3,-1.8860773E-3,4.400296E-3,1.1562552E-2,-4.3899044E-3,-1.0972471E-2],"split_indices":[54,64,1,39,64,0,7,70,44,4,0,50,50,0,0,2,0,39,4,4,0,0,0,0,66,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7E1,3.8E1,5.3E1,1.7E1,4E0,3.4E1,2.6E1,2.7E1,1.4E1,3E0,2.4E1,1E1,5E0,2.1E1,2.1E1,6E0,8E0,6E0,9E0,1.5E1,4E0,6E0,4E0,1.7E1,3E0,1.8E1,4E0,4E0,3E0,3E0,3E0,6E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.520717E-3,-3.3133513E-1,3.068212E-1,-3.8692096E-1,6.477223E-3,9.265262E-2,5.9744245E-1,-4.9126738E-1,-1.1332434E-1,-2.2916572E-1,1.6979818E-1,4.1524345E-1,3.690761E-2,-3.7605798E-1,-6.395851E-1,3.7000265E-3,-8.700801E-3,-1.2574559E-2,-4.737676E-3,-7.875581E-3,2.1936417E-1,2.0538729E-2,1.014666E-2,-4.3827504E-1,-8.921963E-5,-1.249727E-2,-3.0893873E-2,-4.5019113E-3,4.7814287E-3,3.0008012E-1,1.0691523E-1,-9.169729E-3,-4.6928528E-1,4.23109E-1,5.033735E-3,1.4655487E-1,9.5439854E-4,-2.271128E-2,-1.0809992E-2,9.252922E-3,2.1821529E-2,2.316929E-3,8.268908E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,33,35,-1,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0961606E1,1.4227123E0,3.492435E0,1.2575736E0,0E0,8.830251E-1,7.710867E-1,3.233695E-1,1.4529788E-1,1.9962907E-2,5.288717E-1,6.270838E-2,0E0,4.877355E-1,1.3904142E-1,9.2480764E-2,0E0,0E0,0E0,0E0,1.9482744E-1,0E0,0E0,2.6385307E-2,0E0,0E0,0E0,0E0,0E0,2.6667726E-1,3.710024E-2,0E0,5.4811716E-2,2.9248357E-2,0E0,2.3153543E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,20,20,23,23,29,29,30,30,32,32,33,33,35,35],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,34,36,-1,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5363757E8,2.7938022E3,1.3536367E0,7.355655E6,6.477223E-3,4.739085E-2,1.0861106E4,2.56E2,1.0504551E1,5.786E3,2.4719655E5,1.2271712E1,3.690761E-2,2.834008E-1,4.6268812E5,9.115575E0,-8.700801E-3,-1.2574559E-2,-4.737676E-3,-7.875581E-3,1E0,2.0538729E-2,1.014666E-2,3.9022916E2,-8.921963E-5,-1.249727E-2,-3.0893873E-2,-4.5019113E-3,4.7814287E-3,1.4273524E-1,1.1867966E1,-9.169729E-3,4.8E1,1.62739E9,5.033735E-3,6.113901E8,9.5439854E-4,-2.271128E-2,-1.0809992E-2,9.252922E-3,2.1821529E-2,2.316929E-3,8.268908E-3],"split_indices":[12,64,51,44,0,50,4,10,65,9,44,66,0,69,44,65,0,0,0,0,26,0,0,4,0,0,0,0,0,50,65,0,3,7,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,5E1,5.6E1,4.5E1,5E0,3.3E1,2.3E1,3.2E1,1.3E1,6E0,2.7E1,1.5E1,8E0,2E1,1.2E1,6E0,7E0,3E0,3E0,3E0,2.4E1,1E1,5E0,1.7E1,3E0,3E0,9E0,3E0,3E0,1.3E1,1.1E1,3E0,1.4E1,7E0,6E0,7E0,4E0,1E1,4E0,3E0,4E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.1645104E-2,-2.6514617E-1,3.7495545E-1,-3.7249798E-1,1.4057569E-1,1.2649271E-2,4.7338292E-1,-2.5123048E-1,-5.3382564E-1,2.3523143E-2,5.3496337E-3,-5.869981E-2,5.7995743E-3,2.870593E-1,5.955656E-1,-3.0600238E-1,-1.0390969E-1,-5.70746E-1,-8.9158835E-3,-5.9524003E-2,7.17442E-3,4.2166625E-4,-4.8561203E-3,1.6997917E-2,9.171687E-3,1.6433729E-2,3.419012E-2,-3.2724297E-1,-4.8641507E-3,1.4706905E-4,-8.370687E-3,-1.1004423E-2,-2.6299093E-2,-4.9650613E-3,8.1896794E-4,-1.5366684E-2,-6.8367026E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0922768E1,3.2318764E0,1.4917688E0,1.0005875E0,8.7192345E-1,9.4717E-2,5.8029985E-1,2.5061965E-1,1.7212582E-1,0E0,1.4554389E-1,2.688033E-2,0E0,3.5731196E-2,5.005336E-1,6.578469E-2,1.0690201E-1,9.988308E-2,0E0,4.5740943E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.0352573E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,19,19,27,27],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,2.7938022E3,1.118632E3,8.0073607E-1,2.5927516E11,8.674699E-1,1.5941123E0,2.834008E-1,3.851071E7,2.3523143E-2,6.082581E3,2.9285714E-1,5.7995743E-3,1.6918114E7,1.286E4,2.1369668E3,9.069479E-1,1.2352941E0,-8.9158835E-3,2.7616888E6,7.17442E-3,4.2166625E-4,-4.8561203E-3,1.6997917E-2,9.171687E-3,1.6433729E-2,3.419012E-2,2.6326587E-2,-4.8641507E-3,1.4706905E-4,-8.370687E-3,-1.1004423E-2,-2.6299093E-2,-4.9650613E-3,8.1896794E-4,-1.5366684E-2,-6.8367026E-3],"split_indices":[54,64,64,39,43,69,51,69,12,0,64,69,0,62,2,64,69,68,0,59,0,0,0,0,0,0,0,50,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,7.2E1,4.1E1,5.7E1,1.5E1,9E0,3.2E1,3.4E1,2.3E1,3E0,1.2E1,6E0,3E0,1.4E1,1.8E1,2.4E1,1E1,2E1,3E0,9E0,3E0,3E0,3E0,4E0,1E1,1E1,8E0,2.1E1,3E0,5E0,5E0,3E0,1.7E1,5E0,4E0,1.7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.9617853E-2,-2.4183549E-1,4.0074706E-1,-3.1569645E-1,6.6529416E-2,1.7876478E-1,5.8006155E-1,-4.8227298E-1,-1.4976643E-1,-5.9424914E-2,1.5747074E-2,-7.737258E-3,2.4472438E-1,3.9061505E-1,3.7705556E-2,-2.9297882E-1,-5.889372E-1,-2.3732792E-1,2.2820866E-2,2.6211091E-3,-5.1624556E-3,4.1041452E-2,3.3557487E-1,1.0249229E-2,1.9537536E-2,-1.6120767E-2,-2.5388459E-3,-1.0899715E-2,-2.814607E-2,-1.5200834E-2,-8.3401196E-2,6.2257615E-3,-4.9810488E-2,4.934609E-3,-1.8341487E-3,3.783409E-1,6.249311E-3,-6.110078E-3,1.0644606E-3,-4.2503607E-3,4.8744713E-4,8.448114E-3,2.0180492E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,-1,25,27,29,31,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,37,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0977295E1,1.4558988E0,1.7273188E0,1.3374133E0,5.1003927E-1,5.633599E-1,1.0483694E0,3.3222723E-1,4.1344005E-1,7.95442E-2,0E0,0E0,3.378166E-1,5.3940058E-2,0E0,1.8528986E-1,1.5366507E-1,2.8870952E-1,9.53579E-2,0E0,0E0,4.769806E-2,5.1310778E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.0032926E-2,0E0,2.1920005E-2,0E0,0E0,7.13222E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,30,30,32,32,35,35],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,-1,26,28,30,32,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,38,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.095874E6,2.5323425E3,1.5364555E0,2.667836E6,5.2E1,2.4719655E5,1.5605432E9,1.08E2,2.0237808E8,3.24395E5,1.5747074E-2,-7.737258E-3,4.6961885E3,4.1372877E8,3.7705556E-2,3E0,2.4326778E5,2.6326587E-2,9.701793E4,2.6211091E-3,-5.1624556E-3,5.93E3,9.320347E-1,1.0249229E-2,1.9537536E-2,-1.6120767E-2,-2.5388459E-3,-1.0899715E-2,-2.814607E-2,-1.5200834E-2,2.8850128E6,6.2257615E-3,3.4707712E8,4.934609E-3,-1.8341487E-3,4.297157E9,6.249311E-3,-6.110078E-3,1.0644606E-3,-4.2503607E-3,4.8744713E-4,8.448114E-3,2.0180492E-2],"split_indices":[60,64,54,44,8,44,7,10,7,41,0,0,4,7,0,8,40,50,45,0,0,2,39,0,0,0,0,0,0,0,60,0,7,0,0,12,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.2E1,4.5E1,5E1,1.2E1,2.1E1,2.4E1,2.4E1,2.6E1,9E0,3E0,3E0,1.8E1,1.6E1,8E0,1E1,1.4E1,1.7E1,9E0,3E0,6E0,6E0,1.2E1,6E0,1E1,7E0,3E0,3E0,1.1E1,9E0,8E0,3E0,6E0,3E0,3E0,9E0,3E0,5E0,3E0,3E0,3E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-8.849997E-3,-2.5479588E-1,3.5491458E-1,-3.6063942E-1,3.715006E-2,1.5473656E-1,5.483649E-1,-5.1462966E-1,-1.4125961E-1,2.608611E-1,-3.0306023E-1,2.729508E-1,7.857002E-2,1.6496373E-2,3.5064273E-2,-5.58864E-1,-8.224945E-3,-2.0660158E-1,-1.0859799E-3,4.316575E-2,2.0347925E-2,-1.6339922E-2,-5.740316E-3,5.082417E-3,1.4285973E-2,-2.9868316E-2,9.594495E-3,-1.4752319E-2,-6.59638E-1,-1.2659241E-2,-9.412508E-2,-6.1832424E-3,7.635293E-3,3.7703575E-3,-5.094176E-4,-4.798476E-3,9.0003514E-4,-3.168077E-2,-1.5087695E-2,-1.4543182E-3,-6.30855E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,-1,-1,-1,-1,-1,35,-1,-1,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0200134E1,2.1146379E0,1.696475E0,1.6164207E0,1.5251238E0,1.9464952E-1,6.98544E-1,3.0927038E-1,2.0127845E-1,5.295543E-1,5.233836E-2,4.203981E-2,2.559384E-1,0E0,0E0,3.2158852E-1,0E0,1.1859107E-1,2.277082E-1,1.8027704E-2,0E0,0E0,0E0,0E0,0E0,4.3545492E-2,0E0,0E0,1.1193848E-1,0E0,2.0224303E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,25,25,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,-1,-1,-1,-1,-1,36,-1,-1,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.5364555E0,2.667836E6,6.1358623E9,1.7644464E5,1.5605432E9,3.0387878E3,3.167669E5,1.1653E4,2.711196E3,4.2163028E5,3.9219662E5,1.6496373E-2,3.5064273E-2,2.63E2,-8.224945E-3,3.301091E-2,1.0591631E8,3.24395E5,2.0347925E-2,-1.6339922E-2,-5.740316E-3,5.082417E-3,1.4285973E-2,2.1739589E-1,9.594495E-3,-1.4752319E-2,1.2599278E1,-1.2659241E-2,1.0128521E6,-6.1832424E-3,7.635293E-3,3.7703575E-3,-5.094176E-4,-4.798476E-3,9.0003514E-4,-3.168077E-2,-1.5087695E-2,-1.4543182E-3,-6.30855E-3],"split_indices":[62,2,54,44,5,40,7,4,40,2,64,44,40,0,0,11,0,50,7,41,0,0,0,0,0,51,0,0,65,0,59,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,6.7E1,4.5E1,4.9E1,1.8E1,2.3E1,2.2E1,2.8E1,2.1E1,1.1E1,7E0,8E0,1.5E1,1.5E1,7E0,2.4E1,4E0,1.4E1,7E0,6E0,5E0,4E0,3E0,3E0,5E0,9E0,6E0,9E0,1.5E1,7E0,7E0,4E0,3E0,3E0,3E0,3E0,6E0,1.1E1,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.2780731E-2,-2.103141E-1,2.635268E-1,-2.5380844E-1,1.0897097E-2,6.141693E-2,5.049536E-1,-3.825296E-1,-1.332459E-1,1.18093185E-1,-1.0856304E-2,2.3753097E-2,8.528177E-3,-4.3183994E-1,9.1916556E-4,-2.3779689E-1,-2.7006993E-2,-2.0716836E-4,2.4824429E-1,-5.1358575E-1,-2.2944905E-1,-2.7388647E-1,-2.635709E-3,5.1893227E-2,-1.2225639E-1,-4.8759147E-3,7.028372E-2,1.3858598E-2,4.8022736E-3,-1.2681425E-2,-2.619373E-2,-1.1537289E-2,-4.9801744E-3,-1.4314032E-2,-7.643388E-3,-1.2256168E-3,9.035693E-2,-1.8112168E-3,-6.995197E-3,4.628142E-4,4.4842E-3,5.4989634E-3,1.3270152E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,41,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6931057E0,1.3521886E0,2.0183668E0,8.646474E-1,0E0,4.5532116E-1,1.5581274E-1,5.6674814E-1,3.490141E-1,3.233868E-1,0E0,0E0,0E0,2.9577303E-1,0E0,8.533323E-2,1.3410266E-1,1.0330771E-1,6.401038E-2,1.7001104E-1,7.99185E-3,1.3918638E-2,0E0,3.344588E-2,1.8760808E-2,0E0,1.489355E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.15361735E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,42,-1,-1,-1,-1,-1,-1],"split_conditions":[9.379017E6,1.2424786E2,1.3589408E0,2.667836E6,1.0897097E-2,1.12782936E8,2.6448175E5,1E0,2.834008E-1,4.492037E3,-1.0856304E-2,2.3753097E-2,8.528177E-3,2.6631096E3,9.1916556E-4,1.3045E4,4.349136E6,7.331134E-2,2.581938E2,2.8825714E2,3.6623872E5,1.1301075E1,-2.635709E-3,3.257E3,2.1E1,-4.8759147E-3,1.2766116E7,1.3858598E-2,4.8022736E-3,-1.2681425E-2,-2.619373E-2,-1.1537289E-2,-4.9801744E-3,-1.4314032E-2,-7.643388E-3,-1.2256168E-3,2.5263105E6,-1.8112168E-3,-6.995197E-3,4.628142E-4,4.4842E-3,5.4989634E-3,1.3270152E-3],"split_indices":[63,68,54,44,0,57,45,83,69,64,0,0,0,67,0,9,44,50,68,64,44,68,0,2,3,0,60,0,0,0,0,0,0,0,0,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,6.3E1,4.1E1,5.8E1,5E0,2.3E1,1.8E1,2.7E1,3.1E1,2E1,3E0,1.5E1,3E0,2.4E1,3E0,1.5E1,1.6E1,1.1E1,9E0,1.6E1,8E0,1.2E1,3E0,9E0,7E0,4E0,7E0,5E0,4E0,6E0,1E1,5E0,3E0,6E0,6E0,3E0,6E0,3E0,4E0,3E0,4E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.9229734E-2,-1.8106729E-1,2.31691E-1,-2.532704E-1,2.4137679E-1,-3.7081845E-2,2.9602233E-1,-1.386291E-1,-4.0034923E-1,2.135235E-2,6.217838E-2,-6.855812E-3,2.1693984E-3,2.4885967E-2,3.6539522E-1,-3.0778393E-1,-8.963972E-2,-4.5370108E-1,-4.6134875E-3,-1.3428986E-4,4.8315316E-3,6.361907E-3,-5.8038114E-3,2.6357704E-1,2.0319479E-2,-6.9880816E-3,-1.6263334E-2,5.569616E-3,-1.451999E-1,-2.171691E-2,-8.847993E-3,1.2895742E-2,5.46041E-3,-1.6965102E-1,3.126648E-4,-8.192849E-3,-2.015482E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,35,-1,-1,-1],"loss_changes":[4.346442E0,2.0437489E0,7.3717856E-1,8.8873315E-1,4.2307E-1,1.0366717E-1,6.3043046E-1,2.4587435E-1,3.2729292E-1,0E0,2.308269E-2,0E0,0E0,1.759444E-1,1.654346E-1,9.442091E-3,3.4675187E-1,1.425085E-1,0E0,0E0,0E0,0E0,0E0,3.9319992E-2,0E0,0E0,0E0,0E0,8.323151E-2,0E0,0E0,0E0,0E0,3.8185477E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,23,23,28,28,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,36,-1,-1,-1],"split_conditions":[6.402099E-1,5.188755E3,9.489487E2,7.9888564E-1,2.5927516E11,2.862E3,1E0,1.0088235E1,3.271557E8,2.135235E-2,3.6734354E-1,-6.855812E-3,2.1693984E-3,1.5366939E7,5.3433334E1,6.948189E7,2.01E2,1.365445E1,-4.6134875E-3,-1.3428986E-4,4.8315316E-3,6.361907E-3,-5.8038114E-3,4.2298006E5,2.0319479E-2,-6.9880816E-3,-1.6263334E-2,5.569616E-3,3.566468E5,-2.171691E-2,-8.847993E-3,1.2895742E-2,5.46041E-3,2.9153493E-1,3.126648E-4,-8.192849E-3,-2.015482E-3],"split_indices":[54,4,64,39,43,2,27,70,7,0,39,0,0,59,68,7,0,66,0,0,0,0,0,40,0,0,0,0,40,0,0,0,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.4E1,4.1E1,5.5E1,9E0,8E0,3.3E1,3.2E1,2.3E1,3E0,6E0,3E0,5E0,7E0,2.6E1,6E0,2.6E1,1.9E1,4E0,3E0,3E0,4E0,3E0,1.5E1,1.1E1,3E0,3E0,5E0,2.1E1,1.5E1,4E0,1.1E1,4E0,1.8E1,3E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.8485053E-2,-2.4469754E-1,2.4042074E-1,-2.9261667E-1,1.3062776E-2,1.2849022E-1,6.154089E-1,-4.4995734E-1,-2.0333812E-1,-1.0900903E-1,2.0337364E-1,1.1400181E-2,3.0828413E-2,-5.02512E-1,-1.1271086E-2,-1.984166E-2,-1.6494675E-1,4.3643534E-2,-1.5061383E-2,1.01563744E-1,2.9841262E-1,-1.2221131E-2,-2.4544256E-2,-8.746716E-2,-3.0491078E-1,6.2602353E-3,-2.9632018E-3,8.140802E-3,-9.028029E-3,1.7035414E-2,5.7015675E-3,6.3874386E-3,-1.3242845E-1,-1.5794672E-2,-6.844947E-3,1.5426372E-3,-2.7077468E-3,-6.9686915E-3,-1.8426576E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,29,-1,-1,31,33,-1,-1,-1,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[6.149874E0,1.5572052E0,1.9775841E0,6.38958E-1,0E0,7.157751E-1,1.9297028E-1,4.7455788E-2,2.6856935E-1,3.81425E-1,2.5996935E-1,0E0,0E0,2.7454615E-2,0E0,0E0,3.203498E-1,8.963627E-2,0E0,1.556798E-1,1.8793714E-1,0E0,0E0,2.5247303E-1,4.014182E-2,0E0,0E0,0E0,2.1470888E-2,0E0,0E0,0E0,4.2561322E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,16,16,17,17,19,19,20,20,23,23,24,24,28,28,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,30,-1,-1,32,34,-1,-1,-1,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0020565E8,6.099784E3,1.8614753E0,1.3954372E1,1.3062776E-2,6.919127E5,5.3278846E-1,1.2599278E1,1.497E3,1.12782936E8,1.067E4,1.1400181E-2,3.0828413E-2,1.2179555E-1,-1.1271086E-2,-1.984166E-2,4.851587E7,1.2723797E5,-1.5061383E-2,1.1E1,1.5625807E1,-1.2221131E-2,-2.4544256E-2,1.82E2,3.0595828E3,6.2602353E-3,-2.9632018E-3,8.140802E-3,1.479E4,1.7035414E-2,5.7015675E-3,6.3874386E-3,1.0664373E3,-1.5794672E-2,-6.844947E-3,1.5426372E-3,-2.7077468E-3,-6.9686915E-3,-1.8426576E-3],"split_indices":[12,4,51,70,0,44,39,65,2,57,2,0,0,39,0,0,57,40,0,8,66,0,0,10,4,0,0,0,9,0,0,0,64,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,5.5E1,4.8E1,5.1E1,4E0,3.8E1,1E1,1.7E1,3.4E1,9E0,2.9E1,3E0,7E0,1.2E1,5E0,3E0,3.1E1,6E0,3E0,1.5E1,1.4E1,4E0,8E0,2.1E1,1E1,3E0,3E0,8E0,7E0,8E0,6E0,3E0,1.8E1,6E0,4E0,4E0,3E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.9082543E-2,-1.4096387E-1,3.7978494E-1,-3.1774506E-1,2.2337297E-2,2.12702E-1,6.4617896E-1,-2.0080552E-1,-4.36522E-1,1.8716106E-1,-1.513989E-1,2.7568874E-1,-6.708669E-3,1.2188055E-2,3.13876E-2,-2.5358658E-2,-7.527846E-2,-2.1284737E-2,-8.2409E-3,-4.3287493E-2,3.2193792E-1,-1.0191852E-1,-1.2490864E-2,1.9653396E-1,2.045516E-2,-1.6314532E-1,7.355209E-2,-5.868202E-3,1.1091604E-3,1.6044484E-2,8.3608385E-3,-3.6775964E-4,-1.3000146E-1,2.5251228E-1,1.08991615E-1,-2.9444713E-3,-8.963928E-3,5.3216093E-3,2.34854E-4,-3.6950705E-3,-7.2324597E-3,5.284635E-3,1.2827717E-2,2.428451E-3,6.3744118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,-1,25,-1,-1,27,29,31,-1,33,-1,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.576927E0,2.287715E0,1.420403E0,4.401536E-1,1.2310418E0,5.761217E-1,1.7745209E-1,9.749638E-1,1.558528E-1,6.9723684E-1,1.1385542E-1,2.2057533E-1,0E0,0E0,0E0,0E0,2.3431331E-1,0E0,0E0,6.1012916E-2,1.7318487E-2,4.348223E-2,0E0,5.0885797E-2,0E0,3.231305E-2,2.3036648E-2,0E0,0E0,0E0,0E0,0E0,7.3290765E-3,1.5892565E-2,8.008614E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,16,16,19,19,20,20,21,21,23,23,25,25,26,26,32,32,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,-1,26,-1,-1,28,30,32,-1,34,-1,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,6.363496E7,2.5172096E-1,5.070136E7,1.4163358E7,1.414835E8,2.463E4,5.587893E2,3.0595828E3,1.09484E3,8.44629E7,1.2768678E4,-6.708669E-3,1.2188055E-2,3.13876E-2,-2.5358658E-2,2.8866995E1,-2.1284737E-2,-8.2409E-3,2.38E3,1.4922581E1,2.7260774E7,-1.2490864E-2,5.7581736E7,2.045516E-2,2.4E1,1.77E2,-5.868202E-3,1.1091604E-3,1.6044484E-2,8.3608385E-3,-3.6775964E-4,9E0,7.21E2,9.71152E7,-2.9444713E-3,-8.963928E-3,5.3216093E-3,2.34854E-4,-3.6950705E-3,-7.2324597E-3,5.284635E-3,1.2827717E-2,2.428451E-3,6.3744118E-3],"split_indices":[64,12,50,57,57,57,2,4,4,64,57,64,0,0,0,0,68,0,0,2,66,57,0,57,0,3,0,0,0,0,0,0,29,0,57,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.8E1,3.4E1,3.7E1,4.1E1,2.2E1,1.2E1,2E1,1.7E1,2.1E1,2E1,1.9E1,3E0,3E0,9E0,4E0,1.6E1,1.3E1,4E0,8E0,1.3E1,1.6E1,4E0,1.5E1,4E0,1E1,6E0,3E0,5E0,8E0,5E0,4E0,1.2E1,8E0,7E0,4E0,6E0,3E0,3E0,7E0,5E0,3E0,5E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.2536993E-2,-1.4716563E-1,2.9429153E-1,-3.0872297E-1,4.5196623E-2,3.6962634E-1,5.033436E-2,-3.4127834E-1,1.5989435E-3,1.8164174E-1,-1.8597828E-1,2.6802035E-2,2.972784E-1,-2.5498024E-3,5.5164034E-3,-4.4382802E-1,-1.7040183E-1,4.7191728E-2,2.8735614E-1,-1.1843548E-2,-4.6403455E-3,1.41815385E-2,6.2208516E-3,-9.380218E-3,-5.048531E-1,-9.575259E-3,-1.4461846E-3,-2.3764484E-3,4.2682406E-3,1.5387667E-2,3.337561E-3,-1.01296585E-2,-2.3865541E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9262526E0,2.0650702E0,5.143707E-1,4.1787267E-1,1.0116047E0,2.1532488E-1,7.533609E-2,5.026407E-1,0E0,2.7070522E-1,5.3744316E-2,0E0,4.3571234E-2,0E0,0E0,1.6678405E-1,9.110242E-2,5.8284633E-2,1.3209522E-1,0E0,0E0,0E0,0E0,0E0,6.479406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,15,15,16,16,17,17,18,18,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,6.363496E7,3.143873E6,1E0,2.7070812E7,1.0822511E-2,6.185395E-2,2.9282052E6,1.5989435E-3,1.6511278E1,1.9452468E-2,2.6802035E-2,1.6135693E1,-2.5498024E-3,5.5164034E-3,2.07E2,4E1,1.4446624E3,8E0,-1.1843548E-2,-4.6403455E-3,1.41815385E-2,6.2208516E-3,-9.380218E-3,5.0947245E-2,-9.575259E-3,-1.4461846E-3,-2.3764484E-3,4.2682406E-3,1.5387667E-2,3.337561E-3,-1.01296585E-2,-2.3865541E-2],"split_indices":[64,12,41,83,57,69,50,44,0,68,50,0,66,0,0,11,3,4,8,0,0,0,0,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.3E1,6.5E1,2.8E1,3.5E1,3E1,2.1E1,7E0,3.2E1,3E0,1.9E1,1.1E1,3E0,1.8E1,3E0,4E0,1.9E1,1.3E1,9E0,1E1,4E0,7E0,1.4E1,4E0,5E0,1.4E1,9E0,4E0,3E0,6E0,7E0,3E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-7.2987857E-3,-1.16024114E-1,4.8776639E-1,-2.0400482E-1,1.380889E-1,2.4240726E-1,6.3113856E-1,-3.631536E-1,-1.2516913E-1,-1.3286858E-3,1.8414782E-1,6.0512605E-3,1.2784517E-2,3.1134924E-2,1.168205E-2,-4.0387216E-1,-3.4082292E-3,-6.569884E-2,-2.931492E-1,2.339586E-1,1.6072153E-3,-9.043663E-3,-4.5832756E-1,-9.188777E-3,-1.6050922E-2,-1.7343646E-2,-8.142373E-3,4.6309773E-3,2.7390346E-1,-8.914552E-3,-2.1939564E-2,8.457345E-2,-7.665187E-2,1.3565802E-2,5.8197463E-3,1.789516E-1,-2.783092E-3,-1.13226116E-1,6.6339504E-4,1.018419E-2,3.3346382E-3,-5.7979315E-3,-1.3904704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,-1,21,-1,23,25,27,-1,-1,29,-1,31,-1,-1,-1,33,-1,-1,35,37,-1,-1,39,-1,41,-1,-1,-1,-1,-1],"loss_changes":[5.9774666E0,2.0626616E0,5.5056095E-1,8.077135E-1,1.92337E-1,7.9366565E-3,1.9464111E-1,2.2270346E-1,4.5026207E-1,0E0,1.3033193E-1,0E0,0E0,0E0,0E0,1.0509205E-1,0E0,2.5960672E-1,6.0465932E-2,4.1460276E-2,0E0,0E0,7.098627E-2,0E0,1.7722483E-1,0E0,0E0,0E0,1.3755977E-2,0E0,0E0,1.662687E-1,6.2319785E-2,0E0,0E0,2.2332057E-2,0E0,1.7879233E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,15,15,17,17,18,18,19,19,22,22,24,24,28,28,31,31,32,32,35,35,37,37],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,-1,22,-1,24,26,28,-1,-1,30,-1,32,-1,-1,-1,34,-1,-1,36,38,-1,-1,40,-1,42,-1,-1,-1,-1,-1],"split_conditions":[6.0880933E3,6.402099E-1,1.3127055E4,1.6755434E1,8.873786E0,3.8E1,1.33556795E1,3.0387878E3,7.752181E7,-1.3286858E-3,4.5E1,6.0512605E-3,1.2784517E-2,3.1134924E-2,1.168205E-2,2.62E2,-3.4082292E-3,1.3101191E1,3.0595828E3,1.601247E1,1.6072153E-3,-9.043663E-3,1.09E2,-9.188777E-3,3.9709E5,-1.7343646E-2,-8.142373E-3,4.6309773E-3,7.6363635E-1,-8.914552E-3,-2.1939564E-2,1.6504E4,4.6562446E3,1.3565802E-2,5.8197463E-3,1.0803E4,-2.783092E-3,1.0819757E1,6.6339504E-4,1.018419E-2,3.3346382E-3,-5.7979315E-3,-1.3904704E-3],"split_indices":[64,54,4,70,68,3,66,4,57,0,3,0,0,0,0,11,0,68,4,68,0,0,0,0,41,0,0,0,69,0,0,9,67,0,0,9,0,65,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,6.7E1,2.3E1,8E0,1.1E1,2.1E1,4.6E1,5E0,1.8E1,4E0,4E0,8E0,3E0,1.8E1,3E0,3.5E1,1.1E1,1.3E1,5E0,5E0,1.3E1,8E0,2.7E1,4E0,7E0,4E0,9E0,3E0,1E1,1E1,1.7E1,6E0,3E0,6E0,4E0,1.2E1,5E0,3E0,3E0,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.1440982E-2,-6.2797554E-2,4.507278E-1,-1.9711125E-1,1.0496635E-1,5.129908E-1,3.8417152E-3,-2.6682916E-1,-2.106133E-2,2.3068475E-2,4.6171606E-2,1.4093773E-2,3.0483102E-2,-3.593251E-1,-1.8520272E-1,-5.4375096E-3,3.955617E-2,-1.0663052E-1,1.355443E-1,-7.612304E-3,-1.81764E-2,3.2093537E-3,-2.420686E-1,1.4924326E-4,3.618178E-3,3.683248E-3,-1.8009531E-1,8.529208E-3,1.8937011E-1,-6.282493E-3,-1.2529787E-2,-1.114512E-2,-2.8488305E-3,-1.9441842E-3,2.5885198E-3,3.2029503E-3,9.843238E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,23,25,27,-1,-1,-1,29,-1,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9960063E0,1.7106643E0,3.822775E-1,5.098814E-1,8.337417E-1,3.4584713E-1,0E0,1.5995574E-1,8.764601E-2,0E0,4.3779966E-1,0E0,0E0,7.260227E-2,2.8366607E-1,0E0,1.4089671E-2,1.8458804E-1,1.351186E-1,0E0,0E0,0E0,2.7530909E-2,0E0,0E0,0E0,6.314638E-2,2.1986613E-2,4.5401454E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,13,13,14,14,16,16,17,17,18,18,22,22,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,24,26,28,-1,-1,-1,30,-1,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3536367E0,2.561146E3,9.690435E0,2.6326587E-2,6.169213E0,1.5605432E9,3.8417152E-3,1.6755434E1,1.5938355E8,2.3068475E-2,9.250217E5,1.4093773E-2,3.0483102E-2,1.9988E4,6.948309E7,-5.4375096E-3,1.5959719E-1,1.5306258E5,1E0,-7.612304E-3,-1.81764E-2,3.2093537E-3,6.681133E-1,1.4924326E-4,3.618178E-3,3.683248E-3,2.1227852E5,6.70699E-2,1.8879941E6,-6.282493E-3,-1.2529787E-2,-1.114512E-2,-2.8488305E-3,-1.9441842E-3,2.5885198E-3,3.2029503E-3,9.843238E-3],"split_indices":[51,64,65,50,66,7,0,70,7,0,44,0,0,12,7,0,50,40,27,0,0,0,39,0,0,0,40,50,60,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.2E1,7.4E1,1.8E1,4.1E1,3.3E1,1.5E1,3E0,2.9E1,1.2E1,3E0,3E1,9E0,6E0,1.2E1,1.7E1,4E0,8E0,1.1E1,1.9E1,4E0,8E0,3E0,1.4E1,5E0,3E0,3E0,8E0,6E0,1.3E1,6E0,8E0,4E0,4E0,3E0,3E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.8891006E-4,-2.0461842E-1,2.3936723E-1,-2.6864213E-1,1.0395005E-1,4.0609453E-2,3.4401774E-1,-1.3072047E-1,-4.0689108E-1,1.6374014E-2,-6.824622E-2,-3.0380327E-3,1.2533718E-1,4.321348E-1,1.7583089E-1,-2.606203E-1,-3.5405897E-2,-1.9914381E-2,-1.0963779E-2,-4.3053646E-4,-4.7250977E-3,8.477849E-3,6.4777985E-4,2.1338698E-2,6.871182E-3,2.1178096E-3,9.17919E-3,-6.178429E-3,-1.508698E-2,7.1483594E-3,-1.2355014E-1,-2.7399268E-3,-7.0380745E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,31,-1,-1],"loss_changes":[4.896542E0,1.0947328E0,9.428344E-1,8.048494E-1,5.000389E-1,1.6868162E-1,3.686211E-1,2.8792483E-1,6.593704E-2,0E0,1.5718792E-2,0E0,7.54469E-2,2.4379659E-1,4.5559824E-2,5.4399848E-2,2.862119E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5748248E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,32,-1,-1],"split_conditions":[3.6225724E-1,2.7938022E3,1.6492815E3,8.0073607E-1,6.15786E5,2.0978074E8,1.4046429E1,1.0930736E1,1.0034782E-3,1.6374014E-2,3.3019773E3,-3.0380327E-3,1.9E1,2.57281E6,8.413108E-2,6.948189E7,2.01E2,-1.9914381E-2,-1.0963779E-2,-4.3053646E-4,-4.7250977E-3,8.477849E-3,6.4777985E-4,2.1338698E-2,6.871182E-3,2.1178096E-3,9.17919E-3,-6.178429E-3,-1.508698E-2,7.1483594E-3,2.8060134E7,-2.7399268E-3,-7.0380745E-3],"split_indices":[54,64,64,39,41,7,66,70,51,0,64,0,8,41,50,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,5.3E1,4.5E1,4.4E1,9E0,1.6E1,2.9E1,2.3E1,2.1E1,3E0,6E0,7E0,9E0,1.8E1,1.1E1,9E0,1.4E1,1.4E1,7E0,3E0,3E0,5E0,4E0,1.4E1,4E0,3E0,8E0,5E0,4E0,4E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.7145686E-2,-1.9018742E-1,2.53949E-1,-3.7682247E-1,-5.8942463E-2,1.3071105E-1,4.9037015E-1,-2.2524537E-1,-4.7982982E-1,-1.6981971E-1,8.1490755E-2,3.1089506E-1,6.432514E-3,1.2806316E-2,2.6506554E-2,-5.3719007E-3,-1.3179723E-2,-5.28697E-1,-9.733926E-3,-1.0721365E-2,-1.1280859E-1,1.6290917E-1,-4.0978785E-3,5.426799E-4,4.211951E-1,-8.977006E-2,1.0192037E-1,-1.170906E-2,-2.5527062E-2,-1.315957E-1,-1.4220468E-3,2.2514039E-3,2.1176746E-1,2.3396742E-2,7.1186614E-3,6.960829E-4,-9.7633805E-3,-7.3623896E-4,8.6557595E-3,-6.8847155E-3,-1.8526835E-3,1.1051859E-2,4.4684787E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,-1,-1,29,31,-1,-1,33,35,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6924567E0,1.4719789E0,1.0722091E0,2.8225923E-1,5.899601E-1,6.026113E-1,1.2924123E-1,5.88575E-2,5.479169E-2,6.601584E-2,2.596479E-1,3.5666144E-1,1.6531442E-1,0E0,0E0,0E0,0E0,1.7002821E-2,0E0,0E0,1.6671494E-2,5.3891152E-2,0E0,0E0,1.5828896E-1,1.3364337E-1,1.08989276E-1,0E0,0E0,2.0360395E-2,0E0,0E0,1.182121E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,17,17,20,20,21,21,24,24,25,25,26,26,29,29,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,-1,-1,30,32,-1,-1,34,36,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2789746E3,2.667836E6,1.6957201E0,7.1569096E7,2.3514317E8,6.19999E9,2.463E4,5.895365E5,2.873808E3,1.7301265E6,2.63382E5,3.09E2,8.184232E5,1.2806316E-2,2.6506554E-2,-5.3719007E-3,-1.3179723E-2,5.9E1,-9.733926E-3,-1.0721365E-2,7.307441E-1,9.489487E2,-4.0978785E-3,5.426799E-4,1E0,5.7194314E0,4.4432E4,-1.170906E-2,-2.5527062E-2,1.4198953E1,-1.4220468E-3,2.2514039E-3,3.8902016E8,2.3396742E-2,7.1186614E-3,6.960829E-4,-9.7633805E-3,-7.3623896E-4,8.6557595E-3,-6.8847155E-3,-1.8526835E-3,1.1051859E-2,4.4684787E-3],"split_indices":[64,44,51,7,7,5,2,40,4,60,41,0,44,0,0,0,0,10,0,0,51,64,0,0,20,65,2,0,0,68,0,0,7,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6E1,3.8E1,2.4E1,3.6E1,2.6E1,1.2E1,1.1E1,1.3E1,2E1,1.6E1,1E1,1.6E1,6E0,6E0,6E0,5E0,1E1,3E0,7E0,1.3E1,1.1E1,5E0,3E0,7E0,8E0,8E0,3E0,7E0,1E1,3E0,4E0,7E0,4E0,3E0,5E0,3E0,4E0,4E0,7E0,3E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.192404E-2,-1.0829817E-1,3.1503147E-1,-2.248941E-1,3.2058775E-2,3.0021418E-2,2.40816E-1,-2.61447E-1,-4.6001658E-2,-1.8774039E-1,8.586361E-2,1.8622797E-2,1.6836911E-1,-3.1294626E-1,-1.117041E-1,-5.690594E-3,1.3751705E-3,-1.1044114E-2,-3.1386595E-3,-2.307537E-2,1.6901006E-1,9.3525484E-2,1.0970366E-2,-2.5333232E-1,-2.1101238E-2,-1.3740658E-3,-6.1035077E-3,-6.5487735E-3,3.0683273E-2,2.6673624E-1,3.794186E-2,3.399705E-2,6.423635E-3,-2.84072E-1,-5.7958285E-3,2.332488E-3,-1.4989312E-4,1.3557983E-2,5.5707917E-3,2.753241E-3,-1.6577673E-4,-2.053139E-3,3.570154E-3,-7.380671E-3,-1.39659755E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,31,-1,33,-1,-1,-1,-1,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6291184E0,1.2018588E0,9.6184206E-1,2.4953413E-1,4.1821283E-1,0E0,4.1061664E-1,2.19527E-1,5.765434E-2,3.6233142E-2,2.5752085E-1,0E0,1.4953572E-1,1.2778616E-1,1.9209571E-2,0E0,0E0,0E0,0E0,9.454785E-2,1.9482026E-1,4.759167E-2,0E0,3.506434E-2,0E0,0E0,0E0,0E0,8.162632E-3,1.8128097E-2,8.881221E-3,3.968485E-2,0E0,1.1614084E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,19,19,20,20,21,21,23,23,28,28,29,29,30,30,31,31,33,33],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,32,-1,34,-1,-1,-1,-1,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.6777216E8,1.6451234E-2,6.939E3,1.937363E6,3.0021418E-2,2.8129198E7,3.0174E4,2.3770336E5,9.095E3,1.1462246E3,1.8622797E-2,8.640353E3,1E0,2.1E1,-5.690594E-3,1.3751705E-3,-1.1044114E-2,-3.1386595E-3,2.3514317E8,1.327538E7,1E0,1.0970366E-2,6.114E3,-2.1101238E-2,-1.3740658E-3,-6.1035077E-3,-6.5487735E-3,5E0,9.20242E0,7.401E3,6.70699E-2,6.423635E-3,1.23E3,-5.7958285E-3,2.332488E-3,-1.4989312E-4,1.3557983E-2,5.5707917E-3,2.753241E-3,-1.6577673E-4,-2.053139E-3,3.570154E-3,-7.380671E-3,-1.39659755E-2],"split_indices":[2,7,69,2,44,0,57,9,40,9,64,0,64,28,3,0,0,0,0,7,57,27,0,2,0,0,0,0,8,65,2,50,0,2,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,7.2E1,3.9E1,3.9E1,3.3E1,5E0,3.4E1,3.2E1,7E0,6E0,2.7E1,8E0,2.6E1,2.3E1,9E0,3E0,4E0,3E0,3E0,1.2E1,1.5E1,1.5E1,1.1E1,1.9E1,4E0,3E0,6E0,3E0,9E0,8E0,7E0,8E0,7E0,1.4E1,5E0,5E0,4E0,5E0,3E0,4E0,3E0,3E0,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[3.438576E-2,-1.06396236E-1,3.149673E-1,-2.5267357E-1,5.5917215E-2,4.5476103E-1,7.279377E-2,-3.00712E-1,-9.2898555E-2,-3.1709526E-2,1.8336323E-1,1.5574905E-1,5.8391666E-1,1.4583425E-1,-6.314062E-3,-3.4115317E-1,-4.1255495E-3,-6.9036176E-3,2.055638E-3,1.2486959E-1,-1.0266581E-1,2.3557204E-1,1.824854E-3,9.641667E-3,1.3947755E-3,1.1955766E-2,2.7523143E-2,8.071183E-3,2.550276E-3,-8.289272E-3,-4.2330736E-1,8.196883E-3,1.2363415E-3,-5.9210635E-3,-1.2113447E-3,4.4500385E-3,1.1984629E-2,-2.0162843E-2,-8.509359E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1415973E0,1.6761231E0,1.151798E0,2.5605202E-1,3.8551545E-1,7.586756E-1,2.4533387E-1,1.9969058E-1,1.0179678E-1,2.4613558E-1,9.316787E-2,5.954057E-2,7.049227E-2,2.8211087E-2,0E0,2.0050621E-1,0E0,0E0,0E0,3.8354456E-2,3.475502E-2,2.7095735E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6524525E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,21,21,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,3.851071E7,1.4046429E1,3.0595828E3,9.2954946E-1,1.7059127E5,2.145E3,3.0174E4,7.056912E7,3.9343938E-1,4.4E1,3.5925132E-1,2E0,4.075453E6,-6.314062E-3,2.63E2,-4.1255495E-3,-6.9036176E-3,2.055638E-3,4.24E2,1.2836887E1,2.2659668E-1,1.824854E-3,9.641667E-3,1.3947755E-3,1.1955766E-2,2.7523143E-2,8.071183E-3,2.550276E-3,-8.289272E-3,2.7667856E1,8.196883E-3,1.2363415E-3,-5.9210635E-3,-1.2113447E-3,4.4500385E-3,1.1984629E-2,-2.0162843E-2,-8.509359E-3],"split_indices":[2,12,66,4,51,40,0,9,7,39,3,39,8,44,0,11,0,0,0,0,66,50,0,0,0,0,0,0,0,0,70,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,3.6E1,3.3E1,2.1E1,1.3E1,2.7E1,9E0,2E1,1.3E1,7E0,1.4E1,1E1,3E0,2.2E1,5E0,6E0,3E0,6E0,1.4E1,9E0,4E0,4E0,3E0,3E0,1.1E1,6E0,4E0,9E0,1.3E1,3E0,3E0,9E0,5E0,3E0,6E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.966643E-3,-7.085544E-2,3.276036E-1,-1.5150239E-1,1.0737027E-1,1.8118754E-1,5.0269306E-1,-3.0233416E-1,-9.5745645E-2,3.5187614E-3,1.9714032E-1,2.1056935E-1,2.6952655E-3,2.668703E-2,8.795422E-3,-3.4865466E-1,-3.2898132E-3,-1.5185298E-1,-2.172009E-2,-4.94988E-3,4.617463E-2,1.2386015E-2,5.354189E-2,1.0946322E-2,5.0442745E-3,-9.308009E-3,-1.7476618E-2,-1.7975526E-1,2.0094123E-3,3.2748869E-3,-7.5930506E-2,2.1931513E-4,5.0428966E-3,-5.368571E-4,4.581658E-3,-1.1016613E-1,-2.5920448E-1,-6.399328E-3,-1.9286646E-2,-1.7694432E-3,-5.4092626E-3,-1.3955812E-2,-5.6318846E-3,-2.623268E-3,2.0616134E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,27,29,-1,31,-1,33,-1,-1,-1,-1,35,-1,-1,37,-1,-1,-1,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1],"loss_changes":[2.691765E0,1.2811226E0,4.472723E-1,4.84056E-1,2.603435E-1,3.2211542E-2,1.8483734E-1,1.410172E-1,1.877571E-1,7.5871564E-2,1.7802846E-1,1.0671765E-2,0E0,0E0,0E0,9.888887E-3,0E0,1.5229928E-1,1.1685091E-1,0E0,3.1340435E-2,0E0,2.5609786E-2,0E0,0E0,0E0,0E0,9.8606884E-2,0E0,0E0,5.448433E-2,0E0,0E0,0E0,0E0,9.526566E-3,4.0324807E-2,0E0,2.7932081E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,20,20,22,22,27,27,30,30,35,35,36,36,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,28,30,-1,32,-1,34,-1,-1,-1,-1,36,-1,-1,38,-1,-1,-1,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1],"split_conditions":[6.1596104E3,6.402099E-1,1.2768678E4,1.983671E4,2.435464E3,7.845511E-1,1.890618E6,2.8688198E3,7.388E3,2.862E3,1.4163358E7,1.4046429E1,2.6952655E-3,2.668703E-2,8.795422E-3,1.865798E1,-3.2898132E-3,3.365958E8,4.171707E9,-4.94988E-3,2.3145154E0,1.2386015E-2,1.4394E4,1.0946322E-2,5.0442745E-3,-9.308009E-3,-1.7476618E-2,6.681133E-1,2.0094123E-3,3.2748869E-3,2.074635E-1,2.1931513E-4,5.0428966E-3,-5.368571E-4,4.581658E-3,2.1E1,1.2163482E8,-6.399328E-3,5.4E1,-1.7694432E-3,-5.4092626E-3,-1.3955812E-2,-5.6318846E-3,-2.623268E-3,2.0616134E-3],"split_indices":[64,54,64,60,4,39,41,4,2,2,57,66,0,0,0,68,0,7,5,0,51,0,9,0,0,0,0,39,0,0,51,0,0,0,0,3,7,0,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,8.7E1,2E1,6E1,2.7E1,1.2E1,8E0,1.5E1,4.5E1,1.3E1,1.4E1,9E0,3E0,5E0,3E0,1.2E1,3E0,2.5E1,2E1,3E0,1E1,8E0,6E0,5E0,4E0,5E0,7E0,2.2E1,3E0,7E0,1.3E1,7E0,3E0,3E0,3E0,1.3E1,9E0,5E0,8E0,3E0,1E1,5E0,4E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.1407424E-2,-1.2011244E-1,2.75608E-1,-2.658962E-1,2.4977496E-2,1.5131083E-1,4.525968E-1,-9.385554E-2,-3.2845587E-1,-5.7428256E-2,1.8152876E-1,1.6480567E-2,8.923329E-2,2.4549235E-2,7.0940186E-3,-6.8400833E-3,2.538299E-3,-2.179937E-1,-4.2214927E-1,-9.75172E-2,7.6588416E-3,1.2400359E-2,7.954236E-2,1.410595E-1,-3.457052E-2,-1.2682401E-2,-2.934064E-3,-2.1995705E-2,-4.2370656E-3,-3.37424E-2,-1.47851E-1,2.4854776E-4,6.5567396E-3,7.753508E-3,1.2101998E-3,-3.975827E-3,1.3642104E-3,1.2855943E-3,-3.2072302E-3,-2.3993081E-3,-7.443619E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0262656E0,1.5295678E0,8.114855E-1,3.5663366E-1,4.8775166E-1,3.235982E-1,4.419732E-1,1.17786966E-1,1.7706299E-1,2.5266978E-1,1.1732638E-1,0E0,1.3961844E-1,0E0,0E0,0E0,0E0,1.392299E-1,3.1804585E-1,6.563176E-2,0E0,0E0,4.1829314E-2,6.0323924E-2,2.955894E-2,0E0,0E0,0E0,0E0,3.054962E-2,1.7633796E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,17,17,18,18,19,19,22,22,23,23,24,24,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,3.2316522E7,1.0825377E4,4.029937E-1,3.290356E5,1.5046106E7,1.200998E10,9.301E3,3.91028E5,2.4624854E8,7.6363635E-1,1.6480567E-2,5.7581736E7,2.4549235E-2,7.0940186E-3,-6.8400833E-3,2.538299E-3,1.0376E4,2.667836E6,5.3278846E-1,7.6588416E-3,1.2400359E-2,1.1329926E0,9.820493E5,1.662E3,-1.2682401E-2,-2.934064E-3,-2.1995705E-2,-4.2370656E-3,9.489487E2,3.568E3,2.4854776E-4,6.5567396E-3,7.753508E-3,1.2101998E-3,-3.975827E-3,1.3642104E-3,1.2855943E-3,-3.2072302E-3,-2.3993081E-3,-7.443619E-3],"split_indices":[64,12,64,39,40,57,5,2,12,44,69,0,57,0,0,0,0,9,44,39,0,0,51,40,0,0,0,0,0,64,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,7.1E1,3.9E1,3.5E1,3.6E1,2.4E1,1.5E1,1E1,2.5E1,2.4E1,1.2E1,4E0,2E1,1E1,5E0,7E0,3E0,1.3E1,1.2E1,2.1E1,3E0,5E0,7E0,1.4E1,6E0,8E0,5E0,9E0,3E0,1E1,1.1E1,4E0,3E0,1E1,4E0,3E0,3E0,4E0,6E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[9.089786E-3,-1.3958134E-1,1.823711E-1,-1.6602774E-1,1.2605594E-2,2.6837176E-2,2.6603702E-1,-1.961868E-1,9.034605E-2,8.4004225E-3,-5.8959585E-2,1.6100144E-1,3.4601426E-1,-2.4472453E-1,-8.5515514E-2,6.951016E-3,-1.2585899E-4,2.4648644E-3,-1.11092865E-1,5.2467916E-2,2.1543175E-1,2.6220462E-1,1.9522028E-2,-2.254253E-2,-2.0488253E-1,-2.7158562E-2,-5.589663E-3,-7.887794E-3,-2.3436823E-3,3.867521E-4,3.5769162E-3,1.1462305E-2,5.5873883E-3,1.3177972E-2,5.7088947E-3,-1.24723904E-1,-2.9810184E-1,-3.2299086E-3,2.2069851E-3,-7.651839E-3,1.1825152E-3,-1.4569329E-2,-4.5491266E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0138392E0,7.6756656E-1,6.955563E-1,4.8164392E-1,0E0,2.99338E-1,2.3830724E-1,2.6914597E-1,4.660836E-2,0E0,9.1050394E-2,8.91245E-2,5.055976E-2,3.020525E-1,4.2670235E-2,0E0,0E0,0E0,3.0765757E-2,8.513806E-3,1.3020754E-2,3.2384455E-2,0E0,0E0,2.1687901E-1,3.7406106E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6467547E-1,7.832742E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,18,18,19,19,20,20,21,21,24,24,25,25,35,35,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,5.412096E3,6.4813304E-1,1E0,1.2605594E-2,3.5925132E-1,2.6167098E5,1.977643E6,1.0712826E1,8.4004225E-3,9.069479E-1,2.0783027E5,1.5605432E9,2.7571955E6,3.752E3,6.951016E-3,-1.2585899E-4,2.4648644E-3,6.380585E8,1.0188E4,2.5375286E5,7.639958E-1,1.9522028E-2,-2.254253E-2,8.4395015E-1,2.862E3,-5.589663E-3,-7.887794E-3,-2.3436823E-3,3.867521E-4,3.5769162E-3,1.1462305E-2,5.5873883E-3,1.3177972E-2,5.7088947E-3,1.1245455E1,1.9121604E-4,-3.2299086E-3,2.2069851E-3,-7.651839E-3,1.1825152E-3,-1.4569329E-2,-4.5491266E-3],"split_indices":[7,64,51,22,0,39,40,60,66,0,69,45,7,57,2,0,0,0,7,9,45,39,0,0,39,2,0,0,0,0,0,0,0,0,0,65,50,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,6.2E1,5.3E1,5.9E1,3E0,1.9E1,3.4E1,5.3E1,6E0,6E0,1.3E1,1.6E1,1.8E1,3.6E1,1.7E1,3E0,3E0,4E0,9E0,6E0,1E1,1.2E1,6E0,3E0,3.3E1,8E0,9E0,3E0,6E0,3E0,3E0,5E0,5E0,8E0,4E0,1.9E1,1.4E1,5E0,3E0,1.4E1,5E0,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-6.7464183E-3,-1.5363954E-1,2.2110659E-1,-2.1393734E-1,1.1542939E-1,-1.1480358E-2,3.097251E-1,-1.2638564E-1,-3.6742738E-1,-3.2780755E-3,2.6296782E-1,3.3501063E-3,-4.9798533E-2,3.8724285E-1,1.23717055E-1,-3.2236975E-1,-6.363532E-2,-7.8177545E-3,-3.9261973E-1,4.83812E-3,1.502768E-2,7.5573096E-4,-3.7284011E-3,4.240873E-1,5.7777953E-3,3.0338094E-3,7.04075E-3,-7.081364E-3,-1.8980633E-2,-1.0656743E-1,4.5062695E-3,-8.285457E-3,-1.842411E-2,7.3884097E-3,2.0119132E-2,9.855747E-4,-1.36591E-1,-1.5233935E-1,-1.872943E-3,-3.4060667E-3,-7.4193855E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,25,27,29,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,-1],"loss_changes":[3.414445E0,1.030599E0,8.2801616E-1,6.389165E-1,3.587758E-1,4.4827987E-2,3.702402E-1,3.9762008E-1,1.3389349E-2,0E0,5.0923944E-2,0E0,2.3664368E-2,1.2996626E-1,9.5831305E-3,7.647318E-2,2.0589179E-1,0E0,2.096653E-2,0E0,0E0,0E0,0E0,1.0071826E-1,0E0,0E0,0E0,0E0,0E0,8.858858E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.9811034E-2,7.954299E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,18,18,23,23,29,29,36,36,37,37],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,26,28,30,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,-1],"split_conditions":[3.4707712E8,2.5323425E3,6.4813304E-1,2.5233118E11,1E0,3.347126E-1,1.0090909E1,9.7888E4,5.432393E-1,-3.2780755E-3,8.595346E-1,3.3501063E-3,6.113901E8,2.692756E6,7.21E2,9.513E3,1E1,-7.8177545E-3,2.3737179E1,4.83812E-3,1.502768E-2,7.5573096E-4,-3.7284011E-3,8.43433E5,5.7777953E-3,3.0338094E-3,7.04075E-3,-7.081364E-3,-1.8980633E-2,2.2E1,4.5062695E-3,-8.285457E-3,-1.842411E-2,7.3884097E-3,2.0119132E-2,9.855747E-4,7.292013E6,2.1821466E5,-1.872943E-3,-3.4060667E-3,-7.4193855E-3],"split_indices":[7,64,51,43,24,39,65,1,39,0,39,0,7,41,0,9,8,0,68,0,0,0,0,1,0,0,0,0,0,3,0,0,0,0,0,0,62,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,6.1E1,3.9E1,5E1,1.1E1,1.1E1,2.8E1,3.3E1,1.7E1,5E0,6E0,3E0,8E0,1.9E1,9E0,7E0,2.6E1,3E0,1.4E1,3E0,3E0,3E0,5E0,1.6E1,3E0,5E0,4E0,4E0,3E0,2.1E1,5E0,3E0,1.1E1,3E0,1.3E1,4E0,1.7E1,1.4E1,3E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.0708136E-2,-1.0068096E-1,2.156099E-1,-1.3142821E-1,1.29816355E-2,-1.5420415E-2,2.7208453E-1,-7.235448E-2,-2.433172E-1,-4.3015475E-3,1.8691875E-3,3.5029596E-1,1.4091931E-1,-1.0905419E-1,9.725106E-2,-9.605491E-2,-1.4096681E-2,6.2677674E-3,3.9249694E-1,6.9219116E-3,2.2561315E-3,-1.6781512E-1,-3.6965262E-2,8.701024E-3,-2.437507E-4,1.127498E-3,-6.971463E-3,1.9295165E-2,7.910626E-3,-1.8690637E-1,-3.3230288E-3,1.9675551E-3,-6.586944E-2,-3.2386596E-3,-9.0059955E-3,-3.8977624E-3,-3.7706218E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,-1,27,-1,-1,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,35,-1,-1,-1,-1],"loss_changes":[2.5295565E0,8.540961E-1,5.3913677E-1,3.9219618E-1,0E0,4.8826404E-2,2.8917122E-1,2.7203357E-1,2.2603536E-1,0E0,0E0,1.16122484E-1,1.5732884E-2,1.4205703E-1,8.7003425E-2,7.6173216E-2,0E0,0E0,1.0048032E-1,0E0,0E0,1.856184E-2,4.3561928E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.0797074E-2,0E0,0E0,1.7351337E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,18,18,21,21,22,22,29,29,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,-1,28,-1,-1,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,36,-1,-1,-1,-1],"split_conditions":[5.9176934E-1,6.082581E3,1.6511278E1,8.106617E-1,1.29816355E-2,2.80369E5,1.343964E1,4.899102E5,3.3012803E9,-4.3015475E-3,1.8691875E-3,2.1146134E5,1.3021952E1,2.834008E-1,2.0477528E1,4.347826E-2,-1.4096681E-2,6.2677674E-3,4.3E1,6.9219116E-3,2.2561315E-3,4.3E1,9.069479E-1,8.701024E-3,-2.437507E-4,1.127498E-3,-6.971463E-3,1.9295165E-2,7.910626E-3,6.948189E7,-3.3230288E-3,1.9675551E-3,5.691229E6,-3.2386596E-3,-9.0059955E-3,-3.8977624E-3,-3.7706218E-4],"split_indices":[53,64,68,39,0,1,66,40,5,0,0,40,65,69,70,69,0,0,3,0,0,3,69,0,0,0,0,0,0,7,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.5E1,4E1,6.1E1,4E0,8E0,3.2E1,4.1E1,2E1,3E0,5E0,1.9E1,1.3E1,3.4E1,7E0,8E0,1.2E1,4E0,1.5E1,1E1,3E0,1.8E1,1.6E1,3E0,4E0,3E0,5E0,1.1E1,4E0,1.4E1,4E0,4E0,1.2E1,3E0,1.1E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.1137647E-2,-1.4841504E-1,1.5058357E-1,-1.7133911E-1,9.296587E-3,-6.9744764E-3,2.443516E-1,-1.2089115E-1,-2.930563E-1,-8.016984E-2,1.12839796E-1,2.955783E-1,1.0209503E-1,-3.1862837E-1,-5.7307404E-2,-1.5062343E-2,-4.332797E-3,-1.5279446E-2,-7.810819E-3,2.445336E-3,6.685545E-3,5.385923E-3,3.48526E-1,1.1886753E-3,6.3548936E-3,-1.6327466E-1,-2.2052132E-2,-7.499482E-2,3.8711233E-3,-2.1845037E-3,1.7926816E-3,9.293106E-3,1.814251E-2,-3.4889022E-3,-8.845617E-3,-9.639144E-2,1.1507348E-3,-1.2245476E-1,-4.505097E-2,-6.0635726E-3,-1.4380678E-3,-3.6096722E-3,-2.3323331E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,-1,-1,-1,31,-1,-1,33,-1,35,-1,-1,-1,-1,-1,-1,-1,37,-1,39,41,-1,-1,-1,-1],"loss_changes":[2.4644222E0,5.372462E-1,7.5517356E-1,3.1457412E-1,0E0,1.8444332E-1,2.0301759E-1,5.098009E-1,1.3720882E-1,8.2241364E-2,1.01222545E-2,1.6453147E-1,2.9585175E-2,2.1520126E-1,9.251655E-2,0E0,0E0,2.016181E-2,0E0,0E0,0E0,0E0,7.205105E-2,0E0,0E0,7.4698627E-3,0E0,6.781961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7935863E-2,0E0,2.1005869E-2,1.48395095E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,22,22,25,25,27,27,35,35,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,-1,-1,-1,32,-1,-1,34,-1,36,-1,-1,-1,-1,-1,-1,-1,38,-1,40,42,-1,-1,-1,-1],"split_conditions":[1.5363757E8,5.188755E3,9.745363E-1,2.5233118E11,9.296587E-3,3.9219662E5,2.5588E4,9.7888E4,3.159013E3,2.2492808E5,9.638E3,2.0246488E5,3.3213286E8,1.1485E4,1.2726721E1,-1.5062343E-2,-4.332797E-3,1.10794815E2,-7.810819E-3,2.445336E-3,6.685545E-3,5.385923E-3,1.0666667E1,1.1886753E-3,6.3548936E-3,1.23E3,-2.2052132E-2,7.9918444E5,3.8711233E-3,-2.1845037E-3,1.7926816E-3,9.293106E-3,1.814251E-2,-3.4889022E-3,-8.845617E-3,2.3616585E6,1.1507348E-3,1.1286517E1,2.83E2,-6.0635726E-3,-1.4380678E-3,-3.6096722E-3,-2.3323331E-4],"split_indices":[12,4,54,43,0,40,9,1,4,45,2,40,7,9,65,0,0,68,0,0,0,0,66,0,0,2,0,40,0,0,0,0,0,0,0,60,0,65,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.9E1,5E1,5.6E1,3E0,1.9E1,3.1E1,4.1E1,1.5E1,1.2E1,7E0,2.2E1,9E0,9E0,3.2E1,1.1E1,4E0,8E0,4E0,4E0,3E0,6E0,1.6E1,4E0,5E0,6E0,3E0,2.9E1,3E0,5E0,3E0,7E0,9E0,3E0,3E0,2.4E1,5E0,1.5E1,9E0,1.2E1,3E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.2678417E-2,-1.466201E-1,1.862281E-1,-1.7620482E-1,9.240308E-3,9.705602E-2,3.3742642E-1,-7.090367E-2,-2.789737E-1,-4.018244E-2,1.3770209E-1,2.1060726E-1,2.0247113E-2,-1.4893571E-2,-2.6006185E-2,-3.2393265E-1,-1.4806783E-1,1.231255E-3,-3.422751E-3,2.1806447E-1,5.3517066E-2,1.1368617E-2,4.9197474E-3,3.7878655E-2,-6.5135113E-3,-6.5150172E-3,-1.6038842E-2,-8.6807115E-3,-1.3605808E-3,3.452168E-3,1.3452947E-2,8.193736E-2,-1.1955903E-4,-3.0691218E-2,9.580367E-2,4.393671E-3,1.153827E-3,8.160699E-4,-2.5313804E-3,7.6647317E-3,-9.263062E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,35,-1,37,39,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1789722E0,6.779419E-1,7.087977E-1,5.8623385E-1,0E0,2.063486E-1,2.3500848E-1,3.4792316E-1,1.1682677E-1,2.6442342E-2,1.7775846E-1,3.1157732E-2,0E0,0E0,2.1314193E-1,1.0866499E-1,4.9279302E-2,0E0,0E0,1.5598845E-1,2.4222191E-2,0E0,0E0,7.443473E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.2344115E-3,0E0,1.3583649E-2,9.967229E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,19,19,20,20,23,23,31,31,33,33,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,36,-1,38,40,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4707712E8,3.7249353E3,1.7618678E0,6.4987373E-1,9.240308E-3,3.5849205E-1,1.2750684E9,1.477E3,3.1371258E1,6.381E3,1.2320755E1,2.3559767E5,2.0247113E-2,-1.4893571E-2,3.925844E7,4.5E1,7.898E3,1.231255E-3,-3.422751E-3,1.2532358E6,2.5E1,1.1368617E-2,4.9197474E-3,1.5426178E3,-6.5135113E-3,-6.5150172E-3,-1.6038842E-2,-8.6807115E-3,-1.3605808E-3,3.452168E-3,1.3452947E-2,1.8854474E-1,-1.1955903E-4,1.1268256E10,1.0931163E9,4.393671E-3,1.153827E-3,8.160699E-4,-2.5313804E-3,7.6647317E-3,-9.263062E-4],"split_indices":[7,64,51,39,0,51,7,2,70,10,66,45,0,0,57,10,9,0,0,44,8,0,0,4,0,0,0,0,0,0,0,50,0,43,5,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.9E1,5.4E1,5.5E1,4E0,3.5E1,1.9E1,2.8E1,2.7E1,8E0,2.7E1,1.1E1,8E0,3E0,2.5E1,1.9E1,8E0,3E0,5E0,1.3E1,1.4E1,6E0,5E0,1.7E1,8E0,5E0,1.4E1,5E0,3E0,6E0,7E0,9E0,5E0,8E0,9E0,6E0,3E0,3E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.1551761E-3,-7.209351E-2,2.3750809E-1,-1.893742E-1,-1.2011297E-2,3.8561276E-1,8.1953146E-2,-2.3671758E-1,-2.4551033E-8,8.139956E-2,-1.0136746E-1,8.306119E-3,2.2212036E-2,-1.6384511E-3,1.2613972E-1,-1.8408073E-1,-1.7321073E-2,-1.7003229E-2,1.3781384E-1,9.786282E-4,-1.3971221E-1,2.3363598E-3,6.610291E-3,-2.6419435E-3,-8.894433E-3,-4.8848647E-3,4.037166E-2,5.8221202E-2,7.7593937E-3,-1.9836053E-1,-4.599087E-2,-1.8896362E-4,3.238825E-3,4.3928605E-3,5.437464E-6,-4.47034E-3,-1.084753E-2,3.4173686E-4,-3.2058493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6255085E0,5.1916516E-1,4.7524297E-1,2.2413957E-1,4.2559305E-1,1.7500174E-1,6.836755E-2,9.9342465E-2,0E0,1.4112632E-1,1.2682816E-1,0E0,0E0,0E0,9.205207E-3,3.3263385E-2,0E0,5.9737366E-2,4.2936414E-2,0E0,1.0080922E-1,0E0,0E0,0E0,0E0,0E0,1.1184216E-2,1.7693428E-2,0E0,3.418705E-2,1.4304984E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,15,15,17,17,18,18,20,20,26,26,27,27,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.492037E3,1.0485419E8,2.5927516E11,7.215E3,2.7260774E7,7.845511E-1,3.8737573E11,5.83E2,-2.4551033E-8,9.489487E2,3.9343938E-1,8.306119E-3,2.2212036E-2,-1.6384511E-3,9.904E3,3.640909E6,-1.7321073E-2,1.072435E5,1.5763713E1,9.786282E-4,2.561146E3,2.3363598E-3,6.610291E-3,-2.6419435E-3,-8.894433E-3,-4.8848647E-3,4.7937644E-1,4.1372877E8,7.7593937E-3,8.0073607E-1,2.3105123E5,-1.8896362E-4,3.238825E-3,4.3928605E-3,5.437464E-6,-4.47034E-3,-1.084753E-2,3.4173686E-4,-3.2058493E-3],"split_indices":[64,7,43,2,57,39,43,10,0,64,39,0,0,0,9,57,0,45,68,0,64,0,0,0,0,0,39,7,0,39,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,7.3E1,2.1E1,2.4E1,4.9E1,1E1,1.1E1,1.9E1,5E0,2.4E1,2.5E1,5E0,5E0,3E0,8E0,1.6E1,3E0,9E0,1.5E1,6E0,1.9E1,3E0,5E0,3E0,1.3E1,3E0,6E0,6E0,9E0,1.1E1,8E0,3E0,3E0,3E0,3E0,5E0,6E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.9130398E-2,-1.01273865E-1,1.706282E-1,-1.510783E-1,2.006148E-2,-1.645723E-2,2.2163033E-1,-6.57147E-2,-2.2240157E-1,1.6836631E-1,-1.06970415E-1,-3.6865987E-3,1.8125888E-3,2.8625572E-1,1.07481584E-1,-9.075783E-2,2.4872753E-3,-2.859306E-1,-1.3391115E-1,1.4168324E-2,3.0139238E-2,-2.6142051E-2,-9.974796E-3,6.3187378E-3,3.555471E-1,4.2929334E-4,6.1650816E-3,-6.204213E-3,-6.1913934E-2,-3.186166E-1,-4.6716114E-3,-7.52786E-3,-1.1303043E-3,-6.0597813E-4,2.882835E-3,-3.4041405E-3,2.5670652E-3,7.402762E-3,1.7696109E-2,-4.7247627E-4,-3.1441269E-3,-6.683228E-3,-1.5207107E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,-1,33,35,-1,-1,37,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7007227E0,4.6230155E-1,3.185253E-1,3.1415927E-1,4.5131737E-1,3.5821576E-2,1.6103244E-1,8.553387E-2,1.2474024E-1,2.331514E-1,1.2435664E-1,0E0,0E0,1.0303831E-1,3.6489055E-2,2.4710476E-2,0E0,5.8459044E-2,5.647424E-2,0E0,1.2268523E-2,4.5305207E-2,0E0,0E0,2.9804707E-2,0E0,0E0,0E0,8.108251E-3,1.7132163E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,20,20,21,21,24,24,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,-1,34,36,-1,-1,38,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,2.4722983E3,1.6511278E1,6.249741E-1,5.4875424E7,2.80369E5,2.8129198E7,3.4707712E8,2.3670887E1,1.0007143E1,5.4E1,-3.6865987E-3,1.8125888E-3,1.129854E6,8.300154E3,2.9870138E-1,2.4872753E-3,1.2599278E1,1.4342E4,1.4168324E-2,3.281129E-1,4.7E1,-9.974796E-3,6.3187378E-3,4.2298006E5,4.2929334E-4,6.1650816E-3,-6.204213E-3,9.109947E0,3.1E1,-4.6716114E-3,-7.52786E-3,-1.1303043E-3,-6.0597813E-4,2.882835E-3,-3.4041405E-3,2.5670652E-3,7.402762E-3,1.7696109E-2,-4.7247627E-4,-3.1441269E-3,-6.683228E-3,-1.5207107E-2],"split_indices":[51,64,68,39,57,1,57,7,70,65,3,0,0,1,64,39,0,65,9,0,39,8,0,0,40,0,0,0,65,10,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,7.5E1,3.2E1,5.3E1,2.2E1,7E0,2.5E1,2.5E1,2.8E1,1E1,1.2E1,3E0,4E0,1.5E1,1E1,2.1E1,4E0,1.5E1,1.3E1,4E0,6E0,8E0,4E0,6E0,9E0,3E0,7E0,6E0,1.5E1,1.2E1,3E0,9E0,4E0,3E0,3E0,5E0,3E0,3E0,6E0,3E0,1.2E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.2827643E-3,-9.53748E-2,1.9728713E-1,-2.1065272E-1,2.2883583E-2,2.050601E-2,1.1797428E-1,-1.5898795E-1,-3.2501957E-1,-8.292242E-2,9.96415E-2,1.6052288E-1,-7.2009508E-3,-9.00568E-3,-5.309207E-2,-1.9057255E-2,-6.1977473E-3,-8.6108E-3,-3.3003E-2,1.6671287E-1,-1.3420727E-2,2.168861E-1,8.6085826E-2,-6.421859E-3,2.2964445E-3,9.6911524E-4,-5.660396E-2,2.7195322E-3,8.3869435E-3,2.048237E-3,-3.7189974E-3,2.6665214E-3,2.572438E-1,1.8295267E-3,4.8024333E-3,-3.4685924E-3,-5.512584E-4,1.2874959E-2,5.230799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,-1,23,-1,-1,-1,25,27,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.854852E0,8.6367303E-1,6.7088723E-1,1.374054E-1,2.6814887E-1,0E0,3.5394168E-1,1.15058005E-1,1.267752E-1,7.486885E-2,1.4631891E-1,8.2021475E-2,0E0,0E0,9.943327E-2,0E0,0E0,0E0,1.5666839E-2,2.2075444E-2,3.950347E-2,6.54915E-2,8.482069E-3,0E0,0E0,0E0,7.301124E-3,0E0,0E0,0E0,0E0,0E0,1.9658208E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,14,14,18,18,19,19,20,20,21,21,22,22,26,26,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,-1,24,-1,-1,-1,26,28,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7249353E3,3.2316522E7,6.4417176E-2,4.01E2,3.8757455E-1,2.050601E-2,2.8799975E10,1.1127948E3,7.23E2,1.5698778E-2,9.1197605E0,3.7587825E11,-7.2009508E-3,-9.00568E-3,5.1E1,-1.9057255E-2,-6.1977473E-3,-8.6108E-3,1.0256757E1,2.4014617E5,9E0,4.798087E6,8.640353E3,-6.421859E-3,2.2964445E-3,9.6911524E-4,5.28E2,2.7195322E-3,8.3869435E-3,2.048237E-3,-3.7189974E-3,2.6665214E-3,5.8816573E9,1.8295267E-3,4.8024333E-3,-3.4685924E-3,-5.512584E-4,1.2874959E-2,5.230799E-3],"split_indices":[64,12,69,10,53,0,5,45,10,50,65,43,0,0,10,0,0,0,65,40,29,1,64,0,0,0,0,0,0,0,0,0,5,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6.2E1,3.2E1,3.1E1,3.1E1,6E0,2.6E1,2.3E1,8E0,1.3E1,1.8E1,2.3E1,3E0,1.5E1,8E0,4E0,4E0,3E0,1E1,1.1E1,7E0,1.2E1,1.1E1,4E0,4E0,3E0,7E0,3E0,8E0,4E0,3E0,3E0,9E0,5E0,6E0,4E0,3E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-8.650469E-3,-1.0900614E-1,9.558227E-2,-1.3269581E-1,1.002082E-2,-6.540227E-3,1.9064389E-1,-3.0541465E-1,-9.520324E-2,7.884212E-2,-8.010224E-2,1.5548244E-1,1.386956E-2,-6.395159E-3,-2.0071814E-2,-2.5630955E-2,-1.734939E-1,-7.618338E-4,1.114354E-2,-2.1134E-2,-1.0230096E-2,3.101782E-3,1.7370841E-1,-5.1496636E-2,5.311349E-3,-7.5631686E-2,-2.1702276E-1,1.436155E-3,-1.7891602E-3,-6.383569E-2,3.463323E-3,4.401475E-3,2.0706451E-1,2.8216012E-3,-7.338826E-2,-5.235087E-4,-4.8050247E-3,-1.1044809E-2,-5.292653E-3,-4.3971487E-3,-5.630774E-4,4.95257E-3,1.107831E-2,-5.7742145E-4,-3.617704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,-1,29,-1,-1,31,33,-1,35,37,-1,-1,39,-1,-1,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1505969E0,4.8336643E-1,5.2547514E-1,3.1413418E-1,0E0,1.7589964E-1,7.866871E-2,1.569516E-1,2.4066189E-1,1.8574482E-1,1.3375497E-1,2.4099946E-2,0E0,0E0,0E0,1.02471665E-1,7.304269E-2,1.5223938E-2,0E0,5.7061527E-2,0E0,0E0,2.435875E-2,6.1078176E-2,0E0,1.6775507E-2,1.9960463E-2,0E0,0E0,1.6053528E-2,0E0,0E0,2.5479138E-2,0E0,1.0756016E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,19,19,22,22,23,23,25,25,26,26,29,29,32,32,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,-1,30,-1,-1,32,34,-1,36,38,-1,-1,40,-1,-1,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,5.412096E3,1.0230335E0,9.2433E4,1.002082E-2,5.8816573E9,9.7156115E8,2.3812747E-2,5.070136E7,1.3137E4,1.0990962E8,2.0492467E5,1.386956E-2,-6.395159E-3,-2.0071814E-2,1E0,2.1129233E-1,5.466E3,1.114354E-2,4.340646E-1,-1.0230096E-2,3.101782E-3,3.9E1,1.8230173E5,5.311349E-3,1.6592433E9,1.0613095E1,1.436155E-3,-1.7891602E-3,7.1520762E0,3.463323E-3,4.401475E-3,1.5821141E0,2.8216012E-3,1.728E3,-5.235087E-4,-4.8050247E-3,-1.1044809E-2,-5.292653E-3,-4.3971487E-3,-5.630774E-4,4.95257E-3,1.107831E-2,-5.7742145E-4,-3.617704E-3],"split_indices":[7,64,51,1,0,5,44,50,57,2,57,45,0,0,0,22,39,10,0,51,0,0,3,40,0,5,65,0,0,66,0,0,51,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,5.5E1,5.3E1,5.2E1,3E0,2.6E1,2.7E1,8E0,4.4E1,1.2E1,1.4E1,2.3E1,4E0,5E0,3E0,2.4E1,2E1,9E0,3E0,1.1E1,3E0,5E0,1.8E1,2.1E1,3E0,7E0,1.3E1,5E0,4E0,8E0,3E0,7E0,1.1E1,3E0,1.8E1,3E0,4E0,8E0,5E0,4E0,4E0,5E0,6E0,3E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.2767181E-2,-7.550485E-2,2.3020868E-1,-1.8991242E-1,1.2425455E-2,3.5118714E-1,1.6441213E-2,-1.4472838E-1,-1.7234612E-2,-4.7490757E-2,9.4575204E-2,5.718184E-3,1.8365145E-2,-6.1712535E-3,5.178779E-3,-1.6428334E-1,1.4343623E-3,-8.239142E-2,2.705964E-2,1.2967508E-1,-3.756425E-3,-1.8885658E-3,-1.7650852E-1,-1.0625662E-1,-3.8836474E-4,4.5259483E-3,-1.824757E-3,7.448216E-2,8.8826E-3,-9.32075E-3,-1.2730783E-1,-1.8704714E-3,-1.2520008E-1,1.2367095E-4,4.232064E-3,-6.675141E-3,-2.810515E-3,-6.5508513E-3,-2.7191045E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,-1,21,-1,23,25,27,-1,-1,29,31,-1,-1,-1,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6332182E0,8.574506E-1,5.631739E-1,2.9217744E-1,2.4567443E-1,1.709913E-1,1.656675E-1,1.16815865E-1,0E0,7.7683374E-2,1.4513598E-1,0E0,0E0,0E0,0E0,3.5011113E-2,0E0,3.4075737E-2,5.8360104E-2,6.026879E-2,0E0,0E0,2.292335E-2,1.2455925E-2,0E0,0E0,0E0,1.996173E-2,0E0,0E0,1.3725907E-2,0E0,8.611158E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,15,15,17,17,18,18,19,19,22,22,23,23,27,27,30,30,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,-1,22,-1,24,26,28,-1,-1,30,32,-1,-1,-1,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[6.621782E3,3.851071E7,2.692756E6,1E0,1.0889989E0,1.1681136E4,4.2163028E5,2.834008E-1,-1.7234612E-2,3.9219662E5,4.15582E5,5.718184E-3,1.8365145E-2,-6.1712535E-3,5.178779E-3,2.07E2,1.4343623E-3,1E0,1.1024096E0,2.990621E5,-3.756425E-3,-1.8885658E-3,1.8601036E1,5.673198E5,-3.8836474E-4,4.5259483E-3,-1.824757E-3,8E0,8.8826E-3,-9.32075E-3,6.114E3,-1.8704714E-3,9.422078E0,1.2367095E-4,4.232064E-3,-6.675141E-3,-2.810515E-3,-6.5508513E-3,-2.7191045E-3],"split_indices":[64,12,41,28,54,4,44,69,0,40,9,0,0,0,0,11,0,26,69,40,0,0,70,44,0,0,0,8,0,0,2,0,65,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.4E1,2.1E1,3.6E1,4.8E1,1.3E1,8E0,3.1E1,5E0,2.8E1,2E1,4E0,9E0,3E0,5E0,2.8E1,3E0,1.9E1,9E0,1.7E1,3E0,3E0,2.5E1,1.4E1,5E0,4E0,5E0,1.1E1,6E0,1.2E1,1.3E1,4E0,1E1,3E0,8E0,8E0,5E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.6337137E-2,-4.741799E-2,2.0854513E-1,3.3891276E-2,-1.764001E-1,2.038422E-2,1.3394132E-1,-8.011908E-2,1.0597324E-1,-2.1075171E-1,-1.8665073E-3,-3.794602E-3,1.800641E-1,-6.9381413E-3,-2.6423613E-2,1.79193E-1,3.9858863E-2,-2.6873834E-3,-2.5076565E-1,2.4241447E-3,-2.7108064E-3,2.4701236E-1,3.6982333E-3,3.3450797E-3,-6.967112E-2,2.0675743E-3,9.229076E-3,9.755934E-3,5.5138357E-3,-1.1912616E-2,-4.3126848E-3,1.3826186E-2,4.734808E-3,-4.4871457E-3,-1.5199502E-3,2.4972642E-3,-7.7904493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,-1,23,25,27,-1,29,-1,-1,31,-1,-1,33,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3537662E0,7.610741E-1,5.0717914E-1,3.783046E-1,1.1896908E-1,0E0,1.5314957E-1,7.365817E-2,1.284022E-1,1.151129E-1,0E0,2.8286446E-2,9.2589796E-2,0E0,5.9326384E-2,4.8817217E-2,4.1076105E-2,0E0,3.7038207E-2,0E0,0E0,6.549597E-2,0E0,0E0,6.9702864E-3,0E0,0E0,1.8426606E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,21,21,24,24,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,-1,24,26,28,-1,30,-1,-1,32,-1,-1,34,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7249353E3,4.5843E7,6.4417176E-2,1.2239595E3,2.561146E3,2.038422E-2,7.94E2,1.00441E5,1.19461E5,8.471288E10,-1.8665073E-3,2.505442E9,8.260109E9,-6.9381413E-3,4.4493478E2,2.1E1,1.0269E4,-2.6873834E-3,1.5893851E1,2.4241447E-3,-2.7108064E-3,1.2320755E1,3.6982333E-3,3.3450797E-3,9.320373E4,2.0675743E-3,9.229076E-3,7.609895E8,5.5138357E-3,-1.1912616E-2,-4.3126848E-3,1.3826186E-2,4.734808E-3,-4.4871457E-3,-1.5199502E-3,2.4972642E-3,-7.7904493E-4],"split_indices":[64,57,69,4,64,0,0,1,41,43,0,12,5,0,4,3,2,0,66,0,0,66,0,0,45,0,0,5,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,7.1E1,2.8E1,4.4E1,2.7E1,5E0,2.3E1,1.7E1,2.7E1,2.1E1,6E0,6E0,1.7E1,6E0,1.1E1,1.2E1,1.5E1,5E0,1.6E1,3E0,3E0,9E0,8E0,3E0,8E0,3E0,9E0,1.2E1,3E0,1.3E1,3E0,5E0,4E0,3E0,5E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[8.00392E-3,-7.5311966E-2,2.2320004E-1,-1.6169657E-1,3.9531972E-2,2.037042E-2,1.5172923E-1,-2.0980756E-1,-3.453216E-2,-4.141178E-2,1.3942526E-1,-4.5428306E-4,1.8470596E-1,-3.0838817E-1,-1.6820538E-1,6.345308E-3,-1.0368773E-1,3.1139774E-4,-7.037769E-2,1.762802E-1,1.3148795E-3,2.3405357E-1,1.3189063E-3,-7.49871E-3,-1.7251827E-2,-1.2504387E-1,-1.1014954E-2,-8.347501E-3,-3.7627697E-2,-3.674535E-3,-8.465626E-4,2.6201599E-3,9.153292E-3,1.1427378E-2,3.606762E-3,-6.9958577E-3,-3.0226996E-3,-2.8252318E-3,-1.7337747E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,31,-1,33,-1,-1,-1,35,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.863523E0,7.504482E-1,4.438355E-1,2.5582922E-1,2.7260318E-1,0E0,1.3035733E-1,7.52635E-2,1.7843337E-1,2.7268749E-2,5.4869592E-2,0E0,1.4499098E-1,2.8897882E-2,5.8170617E-2,0E0,5.196862E-2,0E0,7.3124208E-3,3.2592297E-2,0E0,4.710555E-2,0E0,0E0,0E0,2.5313318E-2,0E0,0E0,7.2228936E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,18,18,19,19,21,21,25,25,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,32,-1,34,-1,-1,-1,36,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.944201E3,2.3616585E6,6.4417176E-2,7.33E3,1.0639322E0,2.037042E-2,4.867052E6,2.3E1,3.09E2,7.609895E8,7.5710654E-1,-4.5428306E-4,9.780346E9,1E0,5.5124454E9,6.345308E-3,1.32436E3,3.1139774E-4,1.4342E4,2.3314478E-2,1.3148795E-3,1.4046429E1,1.3189063E-3,-7.49871E-3,-1.7251827E-2,1.1947E4,-1.1014954E-2,-8.347501E-3,6.548438E5,-3.674535E-3,-8.465626E-4,2.6201599E-3,9.153292E-3,1.1427378E-2,3.606762E-3,-6.9958577E-3,-3.0226996E-3,-2.8252318E-3,-1.7337747E-5],"split_indices":[64,60,69,2,54,0,1,3,0,5,39,0,5,24,5,0,45,0,9,39,0,66,0,0,0,9,0,0,44,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.4E1,2.8E1,4.2E1,3.2E1,5E0,2.3E1,3E1,1.2E1,1.8E1,1.4E1,4E0,1.9E1,7E0,2.3E1,3E0,9E0,7E0,1.1E1,1E1,4E0,1.4E1,5E0,4E0,3E0,1.7E1,6E0,3E0,6E0,8E0,3E0,3E0,7E0,1.1E1,3E0,9E0,8E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.6296962E-2,-1.0957622E-1,8.764845E-2,-1.4331488E-1,7.344855E-2,1.5443563E-1,-5.450567E-2,-2.8713E-1,-1.01234846E-1,5.8184946E-3,-6.408877E-4,1.9159589E-2,8.410578E-2,-4.6187956E-3,-9.840442E-3,-3.5350266E-1,-3.565798E-3,-5.8636304E-2,-9.817635E-3,1.378036E-2,1.2779655E-1,-3.3760855E-3,3.0022373E-3,-1.8602198E-2,-7.267505E-3,-9.507158E-2,6.255345E-2,4.4702734E-3,-2.3507303E-2,6.6186287E-3,1.1078235E-3,-5.89974E-3,-2.9706039E-2,8.236857E-3,-2.2689598E-3,-2.7011842E-3,9.386042E-5,2.7714388E-3,-3.1489653E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,25,-1,27,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1052467E0,3.8102692E-1,5.210976E-1,2.795713E-1,5.6158952E-2,7.157453E-1,1.5469837E-1,1.2012559E-1,2.0706367E-1,0E0,0E0,0E0,9.348865E-2,8.730571E-2,0E0,4.2126894E-2,0E0,1.4724566E-1,0E0,4.595133E-2,4.5597196E-2,0E0,0E0,0E0,0E0,6.391476E-2,1.2814017E-1,0E0,1.0168814E-2,0E0,0E0,0E0,4.934913E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,17,17,19,19,20,20,25,25,26,26,28,28,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,26,-1,28,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9512938E3,3.4707712E8,7.096748E7,2.3E1,3E0,8.892E3,9.824742E0,1.4710438E1,5.5124454E9,5.8184946E-3,-6.408877E-4,1.9159589E-2,4.6947882E-1,1.9649E4,-9.840442E-3,3.57E2,-3.565798E-3,3.2354313E-1,-9.817635E-3,3.6948596E3,1.0488685E6,-3.3760855E-3,3.0022373E-3,-1.8602198E-2,-7.267505E-3,1.4600837E3,1.82E2,4.4702734E-3,3.3286803E11,6.6186287E-3,1.1078235E-3,-5.89974E-3,2.3435618E3,8.236857E-3,-2.2689598E-3,-2.7011842E-3,9.386042E-5,2.7714388E-3,-3.1489653E-3],"split_indices":[64,7,57,3,8,9,65,66,5,0,0,0,51,2,0,0,0,69,0,4,40,0,0,0,0,4,10,0,43,0,0,0,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5.9E1,5.3E1,5E1,9E0,3.6E1,1.7E1,1E1,4E1,5E0,4E0,6E0,3E1,1.4E1,3E0,7E0,3E0,3.1E1,9E0,1.2E1,1.8E1,7E0,7E0,4E0,3E0,2.4E1,7E0,3E0,9E0,1.4E1,4E0,1.4E1,1E1,3E0,4E0,3E0,6E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.722551E-2,-9.395458E-2,1.2420234E-1,-1.9171187E-1,-1.7700527E-2,7.193587E-2,2.3344992E-1,-1.7031208E-2,-1.439507E-1,-4.9728163E-2,9.450628E-3,1.3141455E-1,1.1863994E-2,1.8516548E-3,1.3099057E-2,-6.58328E-2,-8.506455E-3,5.512348E-2,-9.229657E-2,8.685897E-3,3.3824458E-3,-5.205936E-2,2.7690246E-3,-4.2396532E-3,-5.9620274E-4,8.679767E-3,-2.2171275E-3,-1.1598089E-1,2.887722E-4,-5.4623713E-5,-3.7000077E-3,-6.4911493E-3,-6.8706766E-2,-4.4000093E-3,-9.3871413E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[1.3318694E0,4.0659794E-1,3.0867577E-1,1.7569107E-1,2.5340047E-1,1.4136072E-1,2.2705501E-1,0E0,7.414064E-2,1.3452536E-1,0E0,5.8300734E-2,7.3520586E-2,0E0,0E0,1.5487831E-2,0E0,1.5019287E-1,5.000861E-2,0E0,0E0,1.698462E-2,0E0,0E0,0E0,0E0,0E0,1.7304435E-2,0E0,0E0,0E0,0E0,1.1824053E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,15,15,17,17,18,18,21,21,27,27,32,32],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[2.3514317E8,3.6E1,1.62739E9,1.73E2,8.303871E1,1.6505978E9,1E0,-1.7031208E-2,1.65E9,4.828443E-1,9.450628E-3,1.1247059E1,4.6961885E3,1.8516548E-3,1.3099057E-2,1.3101191E1,-8.506455E-3,9.3152986E8,4.7E1,8.685897E-3,3.3824458E-3,9.223E3,2.7690246E-3,-4.2396532E-3,-5.9620274E-4,8.679767E-3,-2.2171275E-3,1.4099028E-4,2.887722E-4,-5.4623713E-5,-3.7000077E-3,-6.4911493E-3,1.0666667E1,-4.4000093E-3,-9.3871413E-4],"split_indices":[7,3,7,0,68,5,27,0,5,39,0,66,4,0,0,68,0,5,3,0,0,10,0,0,0,0,0,50,0,0,0,0,68,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.4E1,5.6E1,2.3E1,3.1E1,3.9E1,1.7E1,3E0,2E1,2.8E1,3E0,1.9E1,2E1,5E0,1.2E1,9E0,1.1E1,8E0,2E1,7E0,1.2E1,9E0,1.1E1,5E0,4E0,3E0,5E0,1.6E1,4E0,4E0,5E0,8E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.907827E-3,-5.6617703E-2,1.1436547E-1,-3.379136E-2,-1.3040104E-2,6.10722E-2,2.3083279E-1,-5.5686966E-2,1.4029115E-2,-1.1292609E-3,3.6867806E-3,3.1635994E-3,1.397145E-2,-1.23878896E-1,4.354313E-3,-1.7694768E-1,-7.2888196E-2,1.5797482E-1,-4.7013797E-2,-4.1705673E-3,-1.0510715E-2,-4.437277E-3,-4.0424562E-5,8.359286E-3,2.8049736E-3,-6.693169E-2,2.3315763E-3,-4.708756E-2,-6.450188E-3,-3.0261981E-3,-1.6415893E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,-1,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,29,-1,-1,-1],"loss_changes":[6.6154355E-1,3.9320165E-1,1.9073102E-1,5.4442614E-1,0E0,5.3175725E-2,1.22513056E-1,2.6257658E-1,0E0,0E0,0E0,0E0,0E0,6.828302E-2,2.8361884E-1,5.1308125E-2,3.6479093E-2,1.7271608E-2,5.7944614E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.0612819E-2,0E0,1.9657135E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,13,13,14,14,15,15,16,16,17,17,18,18,25,25,27,27],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,-1,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,30,-1,-1,-1],"split_conditions":[1.0230335E0,1.5081978E8,1.5605432E9,8.974092E3,-1.3040104E-2,1.675502E8,2.5172096E-1,3.0870352E1,1.4029115E-2,-1.1292609E-3,3.6867806E-3,3.1635994E-3,1.397145E-2,2.1883162E1,3.24395E5,6.8917E4,1.3101191E1,3.6211365E1,3.3367968E7,-4.1705673E-3,-1.0510715E-2,-4.437277E-3,-4.0424562E-5,8.359286E-3,2.8049736E-3,9.064909E-1,2.3315763E-3,2.7616888E6,-6.450188E-3,-3.0261981E-3,-1.6415893E-4],"split_indices":[51,57,7,4,0,5,50,68,0,0,0,0,0,70,41,12,68,70,44,0,0,0,0,0,0,39,0,59,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.1E1,3.2E1,6.6E1,5E0,2.3E1,9E0,6.3E1,3E0,5E0,1.8E1,4E0,5E0,2.9E1,3.4E1,1.3E1,1.6E1,8E0,2.6E1,7E0,6E0,1.1E1,5E0,5E0,3E0,2.2E1,4E0,1.9E1,3E0,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5322901E-2,-5.5899013E-2,1.458031E-1,1.1183031E-2,-7.442089E-2,8.7336145E-2,2.603312E-1,-4.766866E-2,-2.8927243E-1,-5.629491E-3,1.2188294E-1,1.4381553E-2,1.355711E-3,-1.5175307E-1,-5.4104333E-3,-6.4348048E-3,-1.5418166E-2,-2.373209E-3,1.5097419E-3,1.3681918E-1,1.3670195E-3,-7.5224176E-2,-9.126674E-3,-4.6501562E-2,7.439311E-2,1.9537383E-3,6.668091E-3,-4.23978E-3,-9.4676315E-4,-6.1222162E-2,2.6249702E-3,1.2068098E-1,8.1182824E-4,-9.523721E-4,-8.431852E-2,2.087948E-3,6.664999E-3,-4.960419E-3,-1.6993107E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,-1,7,9,11,13,15,17,19,-1,-1,21,23,-1,-1,-1,-1,25,-1,27,-1,29,31,-1,-1,-1,-1,33,-1,35,-1,-1,37,-1,-1,-1,-1],"loss_changes":[9.746273E-1,4.1596153E-1,2.2501975E-1,0E0,3.59815E-1,8.4189385E-2,1.8957984E-1,2.586859E-1,1.33930445E-2,1.7951446E-2,2.126956E-2,0E0,0E0,6.1722875E-2,1.4446574E-1,0E0,0E0,0E0,0E0,1.8862754E-2,0E0,8.873064E-3,0E0,4.9532868E-2,3.632553E-2,0E0,0E0,0E0,0E0,2.165576E-2,0E0,1.2036376E-2,0E0,0E0,1.7478958E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,19,19,21,21,23,23,24,24,29,29,31,31,34,34],"right_children":[2,4,6,-1,8,10,12,14,16,18,20,-1,-1,22,24,-1,-1,-1,-1,26,-1,28,-1,30,32,-1,-1,-1,-1,34,-1,36,-1,-1,38,-1,-1,-1,-1],"split_conditions":[1.0639322E0,6.169213E0,1.5605432E9,1.1183031E-2,1.2511909E8,1.0984802E1,2.034139E6,1.9121604E-4,9.74026E0,3.568E3,1.5366939E7,1.4381553E-2,1.355711E-3,4.7641E4,3.9219662E5,-6.4348048E-3,-1.5418166E-2,-2.373209E-3,1.5097419E-3,2.1E1,1.3670195E-3,2.356399E3,-9.126674E-3,1.662E3,1E1,1.9537383E-3,6.668091E-3,-4.23978E-3,-9.4676315E-4,1.0364104E-1,2.6249702E-3,7.0957484E4,8.1182824E-4,-9.523721E-4,1.090934E6,2.087948E-3,6.664999E-3,-4.960419E-3,-1.6993107E-3],"split_indices":[54,66,7,0,57,68,41,50,65,2,59,0,0,12,40,0,0,0,0,3,0,4,0,0,8,0,0,0,0,51,0,45,0,0,59,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.7E1,3.6E1,3E0,6.4E1,2.5E1,1.1E1,5.8E1,6E0,7E0,1.8E1,8E0,3E0,1.6E1,4.2E1,3E0,3E0,3E0,4E0,1.5E1,3E0,8E0,8E0,2.8E1,1.4E1,3E0,1.2E1,5E0,3E0,2.5E1,3E0,7E0,7E0,1E1,1.5E1,3E0,4E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[9.514986E-3,-4.413402E-2,1.12077944E-1,1.8473206E-2,-1.2147427E-1,4.004377E-2,2.102562E-1,-2.2932958E-2,1.0793351E-2,-2.2200757E-1,-3.6533143E-2,6.661959E-2,-2.747144E-3,2.6705334E-1,1.8670871E-3,-8.130635E-2,2.1993736E-2,-5.175582E-3,-1.0829389E-2,5.6242696E-3,-8.007304E-3,-1.9168782E-3,9.178476E-2,1.3987968E-2,3.726753E-3,3.4942027E-4,-1.0433106E-1,-3.7648994E-2,1.16163954E-1,-2.3262224E-3,4.076874E-2,6.6216476E-3,5.5437732E-2,-5.412964E-3,-1.3322292E-3,-2.6625146E-3,1.3742411E-3,6.461073E-3,1.9528704E-3,5.2863813E-4,3.4746812E-3,9.7272336E-4,3.903129E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,-1,31,-1,-1,-1,33,35,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.771961E-1,3.363461E-1,2.461794E-1,3.7966E-1,2.5661114E-1,6.48589E-2,1.3073057E-1,9.0954036E-2,0E0,1.1176586E-2,1.1407681E-1,5.43667E-2,0E0,8.52986E-2,0E0,3.1720884E-2,1.1670484E-1,0E0,0E0,3.3569414E-2,0E0,0E0,2.508454E-2,0E0,0E0,0E0,1.46983415E-2,2.366803E-2,1.2241423E-2,0E0,1.0344313E-2,0E0,1.0623142E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,19,19,22,22,26,26,27,27,28,28,30,30,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,-1,32,-1,-1,-1,34,36,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.745363E-1,2.5927516E11,4.8E1,8.784E3,3.5904072E3,4.5E1,1.31E3,1.56967745E1,1.0793351E-2,7.382E3,1.2707124E1,1.2266658E5,-2.747144E-3,2.6448175E5,1.8670871E-3,5.59175E3,3.1228534E5,-5.175582E-3,-1.0829389E-2,3.4922136E11,-8.007304E-3,-1.9168782E-3,2.94E2,1.3987968E-2,3.726753E-3,3.4942027E-4,2.6326587E-2,3.1851864E7,1.6164E4,-2.3262224E-3,5.528107E3,6.6216476E-3,3.2736E4,-5.412964E-3,-1.3322292E-3,-2.6625146E-3,1.3742411E-3,6.461073E-3,1.9528704E-3,5.2863813E-4,3.4746812E-3,9.7272336E-4,3.903129E-3],"split_indices":[54,43,68,2,4,3,0,68,0,9,65,45,0,45,0,60,40,0,0,43,0,0,0,0,0,0,50,44,9,0,64,0,9,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.8E1,3.5E1,3.8E1,3E1,2.1E1,1.4E1,3.3E1,5E0,1.3E1,1.7E1,1.7E1,4E0,1E1,4E0,1.4E1,1.9E1,4E0,9E0,1.4E1,3E0,3E0,1.4E1,7E0,3E0,3E0,1.1E1,1.2E1,7E0,5E0,9E0,4E0,1E1,8E0,3E0,9E0,3E0,4E0,3E0,6E0,3E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-5.616614E-3,-3.6240645E-2,2.9506052E-1,-1.3659433E-1,4.805855E-2,1.5081817E-2,5.450009E-3,-1.6224517E-1,2.468237E-3,1.226239E-1,5.899979E-4,-1.3565136E-1,-1.5871992E-2,4.0753894E-2,9.810252E-3,-5.114403E-2,4.9417134E-2,-1.9186772E-1,-8.368329E-2,-2.3737843E-3,3.2365755E-3,2.6447626E-4,-2.9888488E-3,-1.5310099E-3,6.875817E-2,-2.139257E-1,-2.879235E-3,-1.14133545E-4,-1.0259586E-1,1.4898874E-3,4.075444E-3,-5.2859E-3,-1.0841341E-2,-5.7879556E-2,-5.8671227E-3,-4.5241355E-3,-7.3175685E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,35,-1,-1,-1],"loss_changes":[9.9594337E-1,8.3561856E-1,4.7575235E-2,2.3294461E-1,1.9106114E-1,0E0,0E0,1.6865242E-1,0E0,1.7062876E-1,8.841222E-2,9.203774E-2,0E0,4.671722E-2,0E0,1.8039577E-2,3.1989686E-2,3.266853E-2,3.1914666E-2,0E0,0E0,0E0,0E0,0E0,9.917311E-3,1.6909063E-2,0E0,0E0,1.7460078E-2,0E0,0E0,0E0,0E0,1.5507642E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,24,24,25,25,28,28,33,33],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,36,-1,-1,-1],"split_conditions":[1.0825377E4,6.363496E7,1.1517007E1,1E0,1.5046106E7,1.5081817E-2,5.450009E-3,8.674699E-1,2.468237E-3,4.524696E5,1.1855E4,4.1964554E-4,-1.5871992E-2,1.4792045E3,9.810252E-3,1.7761682E8,4.451217E6,2.022832E1,3.607307E7,-2.3737843E-3,3.2365755E-3,2.6447626E-4,-2.9888488E-3,-1.5310099E-3,1.4557152E2,3.08E2,-2.879235E-3,-1.14133545E-4,7.157903E7,1.4898874E-3,4.075444E-3,-5.2859E-3,-1.0841341E-2,3.07E2,-5.8671227E-3,-4.5241355E-3,-7.3175685E-4],"split_indices":[64,12,66,22,57,0,0,69,0,40,2,54,0,4,0,12,1,66,5,0,0,0,0,0,70,0,0,0,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,9.7E1,9E0,4.4E1,5.3E1,6E0,3E0,3.9E1,5E0,2E1,3.3E1,3.6E1,3E0,1.2E1,8E0,1.6E1,1.7E1,1.6E1,2E1,3E0,9E0,4E0,1.2E1,3E0,1.4E1,1.3E1,3E0,4E0,1.6E1,7E0,7E0,5E0,8E0,8E0,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.6528632E-2,-7.8308254E-2,1.4259002E-1,-1.6155429E-1,1.1379896E-2,1.6143376E-2,8.135684E-2,-1.3101895E-1,-1.506998E-2,-3.23907E-2,8.872882E-2,5.5014797E-2,7.5856303E-3,-4.4915564E-2,-2.2461006E-1,-5.902645E-2,3.1588573E-2,1.404996E-1,-1.4054965E-3,6.615632E-2,-3.8315114E-4,-7.72619E-2,2.9171375E-3,-4.154894E-3,-1.0586657E-2,-5.177119E-4,-7.1607985E-2,3.5703487E-3,-1.7356828E-3,7.903E-3,2.6466716E-3,3.2862735E-3,5.278393E-4,-4.9720565E-3,-1.2058234E-2,-3.7506795E-3,-1.1539966E-3,1.9815285E-3,-2.892463E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,21,23,25,27,29,-1,31,-1,33,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1],"loss_changes":[1.0531768E0,5.7673454E-1,3.946072E-1,1.8332994E-1,1.3152711E-1,0E0,5.2477002E-2,2.8001797E-1,0E0,4.4467956E-2,9.248242E-2,1.5535772E-2,0E0,7.799475E-2,2.1415949E-2,9.792805E-3,3.2686505E-2,2.248989E-2,0E0,8.747831E-3,0E0,3.8167752E-2,0E0,0E0,0E0,0E0,7.2752163E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5368577E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,19,19,21,21,26,26,34,34],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,22,24,26,28,30,-1,32,-1,34,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1],"split_conditions":[3.5324219E3,3.2936362E6,1.6451234E-2,8.79272E4,3.290356E5,1.6143376E-2,2.5172096E-1,4.352552E9,-1.506998E-2,1.2418952E7,9E0,9.348536E-1,7.5856303E-3,1.2804922E3,3.181762E5,3.717E3,6.58E2,7.6363635E-1,-1.4054965E-3,2.192772E10,-3.8315114E-4,1.4710438E1,2.9171375E-3,-4.154894E-3,-1.0586657E-2,-5.177119E-4,6.017926E1,3.5703487E-3,-1.7356828E-3,7.903E-3,2.6466716E-3,3.2862735E-3,5.278393E-4,-4.9720565E-3,1.2726721E1,-3.7506795E-3,-1.1539966E-3,1.9815285E-3,-2.892463E-3],"split_indices":[64,62,69,45,40,0,50,5,0,60,8,39,0,64,40,2,0,69,0,5,0,66,0,0,0,0,70,0,0,0,0,0,0,0,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.6E1,2.9E1,3.9E1,3.7E1,5E0,2.4E1,3.5E1,4E0,2.4E1,1.3E1,2E1,4E0,1.9E1,1.6E1,1.7E1,7E0,9E0,4E0,1.7E1,3E0,1.5E1,4E0,3E0,1.3E1,4E0,1.3E1,4E0,3E0,5E0,4E0,1.4E1,3E0,9E0,6E0,9E0,4E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4594325E-2,-5.8822367E-2,1.137161E-1,-1.185158E-1,3.909931E-3,2.989318E-1,4.6562478E-2,-8.9870565E-2,-1.5275348E-2,9.235659E-2,-1.8323919E-2,6.5881326E-3,1.5994554E-2,8.753229E-2,-6.451177E-3,-3.3385824E-2,-1.7048912E-1,6.9720345E-3,-7.4128015E-4,-4.0566307E-2,6.690836E-2,1.545394E-2,1.5789354E-1,3.8189027E-3,-6.3694775E-2,-8.970553E-3,-3.4484116E-3,-2.867881E-3,-4.850155E-3,4.9076765E-4,4.563798E-3,1.7631055E-3,-7.814845E-4,3.2213943E-3,9.047411E-3,-5.0141593E-3,-4.8664636E-3,1.8529476E-3,-1.4721521E-3,1.1150488E-3,-1.758213E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,23,25,-1,-1,27,29,31,33,-1,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,39,-1,-1,-1,-1],"loss_changes":[5.968806E-1,2.9255044E-1,3.219045E-1,2.3785299E-1,7.838622E-2,1.6771376E-2,1.817169E-1,1.6279563E-1,0E0,6.336578E-2,6.337884E-2,0E0,0E0,8.884731E-2,0E0,9.057793E-2,3.411439E-2,0E0,0E0,2.3699783E-2,1.3887554E-2,9.2590675E-3,2.3098454E-2,0E0,5.8069974E-2,0E0,0E0,0E0,1.8233843E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2060729E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,15,15,16,16,19,19,20,20,21,21,22,22,24,24,28,28,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,24,26,-1,-1,28,30,32,34,-1,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,40,-1,-1,-1,-1],"split_conditions":[4.3972437E3,3.851071E7,1.3453537E-1,5.093857E-1,2.5758067E-1,8.595346E-1,2.222E3,4.352552E9,-1.5275348E-2,3.7E1,1.2940105E0,6.5881326E-3,1.5994554E-2,8.640353E3,-6.451177E-3,4.347826E-2,3.159013E3,6.9720345E-3,-7.4128015E-4,4.1E1,1.5426178E3,2.505442E9,1.5821141E0,3.8189027E-3,7.48E2,-8.970553E-3,-3.4484116E-3,-2.867881E-3,5.4151624E-2,4.9076765E-4,4.563798E-3,1.7631055E-3,-7.814845E-4,3.2213943E-3,9.047411E-3,-5.0141593E-3,7.31693E6,1.8529476E-3,-1.4721521E-3,1.1150488E-3,-1.758213E-3],"split_indices":[64,12,69,51,39,39,0,5,0,3,51,0,0,64,0,69,4,0,0,3,4,12,51,0,11,0,0,0,69,0,0,0,0,0,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.7E1,2.6E1,3.9E1,3.8E1,6E0,2E1,3.6E1,3E0,7E0,3.1E1,3E0,3E0,1.7E1,3E0,2.2E1,1.4E1,4E0,3E0,2.5E1,6E0,9E0,8E0,4E0,1.8E1,9E0,5E0,1.4E1,1.1E1,3E0,3E0,5E0,4E0,4E0,4E0,9E0,9E0,4E0,7E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-5.19011E-3,-6.9614224E-2,1.23798385E-1,-1.536984E-1,-5.421018E-3,1.6950881E-1,-9.631899E-2,-1.2024319E-1,-1.544667E-2,3.243242E-2,-9.52699E-2,2.2457498E-1,4.897261E-2,-8.363126E-3,1.086746E-3,-1.5710287E-1,-5.493341E-2,5.881376E-2,-1.6658735E-3,-9.3669083E-4,-1.15171455E-1,2.8710824E-1,8.7069914E-2,-1.3542484E-3,5.2300375E-3,-5.3700106E-3,-1.0380883E-2,-4.925373E-3,-4.7871884E-4,-1.6946989E-3,8.152663E-2,-2.7651251E-3,-6.625372E-3,1.651281E-2,2.106909E-1,1.1578061E-3,5.087618E-3,6.644892E-4,9.8136075E-2,1.0911343E-2,3.6430939E-3,5.484079E-3,2.0269814E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,25,27,29,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,39,-1,-1,-1,41,-1,-1,-1,-1],"loss_changes":[9.2271477E-1,3.985916E-1,3.8860828E-1,1.7550409E-1,1.4933665E-1,1.9476342E-1,8.7723255E-2,6.1181396E-2,0E0,6.035342E-2,1.6535327E-2,1.5557289E-1,6.759389E-2,0E0,0E0,1.9167125E-2,2.9862132E-2,5.443281E-2,0E0,0E0,9.750009E-3,2.1483302E-2,1.1677392E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9359827E-2,0E0,0E0,0E0,3.1804472E-2,0E0,0E0,0E0,1.622422E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,17,17,20,20,21,21,22,22,30,30,34,34,38,38],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,26,28,30,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,40,-1,-1,-1,42,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.04856936E8,1.650269E10,1.0380537E8,5.070136E7,6.5566176E7,2.3342845E5,9.869215E-4,-1.544667E-2,4.5E1,8.51326E5,1.343964E1,8.300154E3,-8.363126E-3,1.086746E-3,1.2511909E8,1.44E2,1.2239595E3,-1.6658735E-3,-9.3669083E-4,6.919127E5,7.412844E0,4.1E1,-1.3542484E-3,5.2300375E-3,-5.3700106E-3,-1.0380883E-2,-4.925373E-3,-4.7871884E-4,-1.6946989E-3,1.9612434E5,-2.7651251E-3,-6.625372E-3,1.651281E-2,5.8816573E9,1.1578061E-3,5.087618E-3,6.644892E-4,1.3989743E1,1.0911343E-2,3.6430939E-3,5.484079E-3,2.0269814E-3],"split_indices":[2,7,5,7,57,57,45,51,0,3,41,66,64,0,0,57,0,4,0,0,44,65,3,0,0,0,0,0,0,0,40,0,0,0,5,0,0,0,66,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,7.3E1,3.6E1,3.1E1,4.2E1,3E1,6E0,2.8E1,3E0,3E1,1.2E1,2E1,1E1,3E0,3E0,1.7E1,1.1E1,2.2E1,8E0,3E0,9E0,1.3E1,7E0,5E0,5E0,1.4E1,3E0,4E0,7E0,4E0,1.8E1,5E0,4E0,4E0,9E0,3E0,4E0,4E0,1.4E1,6E0,3E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.1519549E-3,-2.8462943E-2,1.18478015E-1,4.3720705E-3,-9.794323E-2,1.4609349E-2,3.857946E-2,-4.5619905E-3,2.674226E-2,-1.4459822E-1,-2.7574643E-2,-6.293194E-3,8.5170716E-2,5.9438795E-3,1.100515E-2,-3.6097392E-3,-8.423176E-3,-3.701743E-3,1.2232612E-2,1.4714572E-1,1.2549363E-2,2.9163327E-2,-3.453845E-2,1.7094949E-3,-1.0807452E-3,2.288809E-3,9.8430645E-3,1.6306315E-3,-9.5481885E-4,-7.0271827E-3,7.6195925E-2,3.1492586E-4,-2.659017E-3,1.1259924E-2,-2.610954E-3,9.680759E-2,2.0067093E-5,2.018824E-3,-4.8218592E-4,1.6301913E-3,4.931062E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,-1,19,-1,21,-1,-1,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,33,35,-1,-1,37,-1,39,-1,-1,-1,-1,-1],"loss_changes":[3.898505E-1,1.9206466E-1,3.911372E-1,1.4561425E-1,8.465502E-2,0E0,1.6703027E-1,0E0,8.403908E-2,3.297913E-2,2.8839212E-2,0E0,6.5664046E-2,0E0,3.7394762E-2,0E0,0E0,0E0,9.151089E-3,4.880576E-2,7.83131E-3,5.5003725E-2,1.5164554E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.925455E-2,2.1808207E-2,0E0,0E0,1.24683995E-2,0E0,7.00102E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,12,12,14,14,18,18,19,19,20,20,21,21,22,22,29,29,30,30,33,33,35,35],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,-1,20,-1,22,-1,-1,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,34,36,-1,-1,38,-1,40,-1,-1,-1,-1,-1],"split_conditions":[4.3972437E3,5.070136E7,6.4417176E-2,1.21719E5,1.1309524E-1,1.4609349E-2,4.2163028E5,-4.5619905E-3,1.82E2,7.056912E7,1.2029E4,-6.293194E-3,3.077632E5,5.9438795E-3,2.727403E7,-3.6097392E-3,-8.423176E-3,-3.701743E-3,1.1864285E1,1.5270755E6,4.1E1,1.6987854E1,1.3657556E-1,1.7094949E-3,-1.0807452E-3,2.288809E-3,9.8430645E-3,1.6306315E-3,-9.5481885E-4,1.0521739E1,1.1E1,3.1492586E-4,-2.659017E-3,3E0,-2.610954E-3,2.4E1,2.0067093E-5,2.018824E-3,-4.8218592E-4,1.6301913E-3,4.931062E-3],"split_indices":[64,57,69,1,69,0,44,0,10,7,9,0,40,0,57,0,0,0,66,44,3,68,53,0,0,0,0,0,0,65,8,0,0,8,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,5.7E1,2.6E1,5E0,1.7E1,9E0,4.8E1,1.5E1,1.1E1,3E0,1.4E1,5E0,4.3E1,8E0,7E0,4E0,7E0,7E0,7E0,3.1E1,1.2E1,4E0,3E0,4E0,3E0,4E0,3E0,1.8E1,1.3E1,5E0,7E0,1.4E1,4E0,1E1,3E0,5E0,9E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[9.045424E-3,-5.7991352E-2,1.4286053E-1,-7.546159E-2,1.0754656E-2,1.8943007E-1,5.9475526E-2,-1.0677663E-1,-2.511424E-2,2.1046205E-1,1.8204363E-3,1.1770827E-3,4.834723E-3,-8.169177E-2,-9.3652485E-3,3.367179E-2,-5.163845E-2,9.584252E-2,1.1050702E-2,-5.55751E-2,-6.5169064E-3,-8.785663E-4,2.766044E-3,-8.527613E-2,-5.541489E-4,5.6156158E-3,1.6247693E-3,-8.219394E-2,-2.8800428E-2,-4.7539105E-3,-8.837312E-4,-1.6480784E-3,-5.4902784E-3,-3.4585765E-3,-9.2944205E-3,5.858133E-4,-1.4858984E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,-1,27,-1,-1,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1],"loss_changes":[9.4122016E-1,3.9438725E-1,1.2464559E-1,1.0295254E-1,0E0,5.9262216E-2,2.0628892E-2,9.558526E-2,4.383859E-2,7.461649E-2,0E0,0E0,0E0,5.4931566E-2,0E0,1.6086936E-2,2.3703989E-2,9.055868E-3,0E0,1.6875975E-2,0E0,0E0,0E0,1.384864E-2,0E0,0E0,0E0,2.0887509E-2,1.4270461E-2,0E0,0E0,0E0,0E0,0E0,7.3613008E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,19,19,23,23,27,27,28,28,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,-1,28,-1,-1,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1],"split_conditions":[7.90671E-1,6.0880933E3,1.33556795E1,5.1299828E1,1.0754656E-2,2.692756E6,5.72143E5,1.01949844E6,4.171707E9,3.840785E1,1.8204363E-3,1.1770827E-3,4.834723E-3,6.784095E7,-9.3652485E-3,1E0,1.9880192E-2,1.6984678E3,1.1050702E-2,1E0,-6.5169064E-3,-8.785663E-4,2.766044E-3,1.0613095E1,-5.541489E-4,5.6156158E-3,1.6247693E-3,2.28569E5,1E0,-4.7539105E-3,-8.837312E-4,-1.6480784E-3,-5.4902784E-3,-3.4585765E-3,5.5570245E-2,5.858133E-4,-1.4858984E-3],"split_indices":[51,64,66,68,0,41,41,40,5,68,0,0,0,57,0,24,50,4,0,24,0,0,0,65,0,0,0,12,8,0,0,0,0,0,50,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,6.6E1,3E0,2.1E1,1.3E1,4E1,2.6E1,1.8E1,3E0,9E0,4E0,3.4E1,6E0,8E0,1.8E1,6E0,1.2E1,2.6E1,8E0,3E0,5E0,9E0,9E0,3E0,3E0,1.2E1,1.4E1,6E0,3E0,7E0,5E0,3E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-4.6928497E-3,-5.3804614E-2,1.1823321E-1,-1.0577308E-1,1.265054E-2,5.9010614E-2,1.6101687E-1,-2.7333898E-3,-1.4315923E-1,4.252409E-2,-1.0229553E-2,-8.1868796E-4,8.0084935E-2,9.548521E-2,1.08073875E-2,-5.0462883E-2,5.062511E-3,-1.5714526E-1,-1.0680535E-4,4.9401456E-3,1.0878688E-2,5.308905E-3,1.376026E-3,6.852101E-3,4.402173E-2,-3.1441206E-3,-5.838538E-4,-1.2015258E-1,-9.070478E-3,-4.414422E-2,6.0683694E-2,3.683244E-3,9.395717E-5,-1.4257659E-1,2.2394129E-4,-2.7854303E-3,1.4301167E-3,5.514192E-3,1.3288945E-3,-4.6445415E-3,-9.901986E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,-1,25,-1,27,-1,29,-1,-1,-1,-1,31,-1,-1,33,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.7648137E-1,2.776239E-1,7.157862E-2,1.7296028E-1,2.785295E-1,2.616806E-2,8.147472E-2,8.0380365E-2,6.454289E-2,2.5856936E-1,0E0,0E0,2.089589E-2,3.206984E-2,0E0,7.278301E-3,0E0,3.3542573E-2,0E0,8.20268E-2,0E0,0E0,0E0,0E0,1.3640304E-2,0E0,0E0,5.6952298E-2,0E0,2.7335582E-2,2.4135944E-2,0E0,0E0,2.694726E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,15,15,17,17,19,19,24,24,27,27,29,29,30,30,33,33],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,-1,26,-1,28,-1,30,-1,-1,-1,-1,32,-1,-1,34,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,7.33E3,2.4221443E-1,5.1056176E-1,1.5081978E8,4.308E3,1.286E4,1.5151923E8,3.271557E8,8.0529043E3,-1.0229553E-2,-8.1868796E-4,7.2292525E-1,1.3047E4,1.08073875E-2,2.83E2,5.062511E-3,7.150216E7,-1.0680535E-4,1E0,1.0878688E-2,5.308905E-3,1.376026E-3,6.852101E-3,2.83E2,-3.1441206E-3,-5.838538E-4,1.2975548E3,-9.070478E-3,6.758564E6,3.09E2,3.683244E-3,9.395717E-5,1.0214403E6,2.2394129E-4,-2.7854303E-3,1.4301167E-3,5.514192E-3,1.3288945E-3,-4.6445415E-3,-9.901986E-3],"split_indices":[51,2,50,39,57,2,2,7,7,64,0,0,39,9,0,0,0,57,0,27,0,0,0,0,0,0,0,64,0,1,0,0,0,40,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,7.9E1,3.1E1,4.4E1,3.5E1,1.4E1,1.7E1,1.2E1,3.2E1,3.2E1,3E0,3E0,1.1E1,1.1E1,6E0,9E0,3E0,2.9E1,3E0,2.8E1,4E0,5E0,6E0,4E0,7E0,5E0,4E0,1.9E1,1E1,1.5E1,1.3E1,3E0,4E0,1.6E1,3E0,1.2E1,3E0,3E0,1E1,1.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[6.416159E-3,2.0315075E-2,-1.141829E-2,-1.9703843E-3,2.1439561E-1,-1.1751293E-1,2.3381E-2,1.2400889E-2,1.4362058E-3,-6.8428725E-2,-8.111986E-3,4.4363547E-2,-5.4142043E-2,-9.165299E-4,-3.987265E-3,1.4883289E-2,1.0409232E-1,-1.2365943E-1,-6.566495E-5,5.5762775E-2,-3.3234183E-2,1.3985713E-1,9.84309E-4,-7.4421433E-3,-1.8996599E-3,2.7235793E-3,-1.5603806E-3,2.68532E-2,3.7888405E-3,-4.4600507E-3,-7.6574264E-3,1.1171165E-3,8.100071E-3,2.1658905E-3,-3.762361E-4,2.1148104E-2,-2.3479438E-3,-3.88076E-5,2.350235E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,-1,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,-1,-1,35,-1,-1,-1,-1,37,-1,-1,-1],"loss_changes":[3.8569152E-1,4.1996434E-1,0E0,2.6051396E-1,1.22433424E-1,4.24314E-2,1.2114598E-1,0E0,0E0,1.046617E-2,0E0,1.0057831E-1,6.0139865E-2,0E0,0E0,8.053394E-2,5.205059E-2,1.958897E-2,2.506832E-2,1.7999388E-2,3.3266485E-2,6.5944105E-2,0E0,0E0,0E0,0E0,0E0,1.1220271E-2,0E0,0E0,2.1342777E-2,0E0,0E0,0E0,0E0,7.3896036E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,27,27,30,30,35,35],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,-1,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,-1,-1,36,-1,-1,-1,-1,38,-1,-1,-1],"split_conditions":[1.5081978E8,1.2768678E4,-1.141829E-2,6.7281544E-5,7.9684106E9,1E0,8.650247E-1,1.2400889E-2,1.4362058E-3,3.8E1,-8.111986E-3,3.167669E5,1.020707E1,-9.165299E-4,-3.987265E-3,2.1146134E5,2.8399336E7,2.6569445E1,9E0,1.453125E1,2.91E2,1.6987854E1,9.84309E-4,-7.4421433E-3,-1.8996599E-3,2.7235793E-3,-1.5603806E-3,9.824742E0,3.7888405E-3,-4.4600507E-3,3.077573E5,1.1171165E-3,8.100071E-3,2.1658905E-3,-3.762361E-4,2.4124284E5,-2.3479438E-3,-3.88076E-5,2.350235E-3],"split_indices":[57,64,0,50,12,20,39,0,0,3,0,40,66,0,0,40,57,70,8,66,0,68,0,0,0,0,0,65,0,0,40,0,0,0,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,9.6E1,4E0,8.7E1,9E0,1.5E1,7.2E1,6E0,3E0,1E1,5E0,5.7E1,1.5E1,4E0,6E0,3.9E1,1.8E1,6E0,9E0,2.1E1,1.8E1,1.2E1,6E0,3E0,3E0,3E0,6E0,1.2E1,9E0,4E0,1.4E1,4E0,8E0,7E0,5E0,9E0,5E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-8.2730055E-3,-7.338163E-2,6.2209774E-2,7.929268E-3,-9.2104666E-2,6.247671E-3,1.4584054E-1,-5.8499068E-2,-2.4243091E-1,3.0254044E-2,-7.051932E-3,9.6340045E-2,8.981123E-3,-1.4025567E-1,-1.8396016E-2,-1.4485877E-2,-4.3517393E-3,5.55158E-3,3.2823936E-3,5.777957E-3,1.945604E-3,-2.29994E-3,-7.3264777E-3,3.301846E-3,-4.043226E-2,-4.035622E-2,4.6647474E-2,-1.81596E-2,-9.511059E-2,5.7616085E-4,-2.5788865E-3,7.5935305E-4,3.1414768E-3,-4.15177E-2,8.2656613E-4,-1.4948387E-3,-5.6902533E-3,-4.2824852E-4,-2.7053594E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,-1,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,27,29,31,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1],"loss_changes":[4.7723585E-1,2.7682117E-1,2.3309654E-1,0E0,2.4346003E-1,1.3116303E-1,4.1543365E-2,1.3840349E-1,8.488339E-2,7.3851645E-2,0E0,1.8966258E-2,0E0,2.683431E-2,6.5818965E-2,0E0,0E0,0E0,4.5407526E-2,0E0,0E0,0E0,0E0,0E0,2.871862E-2,1.3289103E-2,7.829327E-3,1.7352002E-2,1.0975473E-2,0E0,0E0,0E0,0E0,7.39854E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,18,18,24,24,25,25,26,26,27,27,28,28,33,33],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,-1,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,28,30,32,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1],"split_conditions":[2.1915238E8,7.2047586E0,1.4494385E0,7.929268E-3,1E0,1.5939707E7,1.3127055E4,1.6755434E1,3.9E1,1.2192353E7,-7.051932E-3,8.568469E0,8.981123E-3,9.63E3,1.82E2,-1.4485877E-2,-4.3517393E-3,5.55158E-3,1.3137E4,5.777957E-3,1.945604E-3,-2.29994E-3,-7.3264777E-3,3.301846E-3,7.331148E7,1.1894739E5,8.517173E3,1.752E3,1.0979587E8,5.7616085E-4,-2.5788865E-3,7.5935305E-4,3.1414768E-3,3.4418715E6,8.2656613E-4,-1.4948387E-3,-5.6902533E-3,-4.2824852E-4,-2.7053594E-3],"split_indices":[7,65,51,0,28,59,4,70,3,57,0,65,0,12,10,0,0,0,2,0,0,0,0,0,57,45,4,10,7,0,0,0,0,62,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.3E1,4.9E1,3E0,5E1,3E1,1.9E1,4.2E1,8E0,2.7E1,3E0,1.2E1,7E0,1.3E1,2.9E1,4E0,4E0,5E0,2.2E1,6E0,6E0,4E0,9E0,5E0,2.4E1,1.1E1,1.1E1,1.8E1,6E0,3E0,8E0,6E0,5E0,1.1E1,7E0,3E0,3E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-5.914277E-3,-6.3014925E-2,5.446815E-2,-8.074947E-2,8.295022E-3,7.041603E-2,-7.0141805E-3,-5.532076E-2,-2.2004946E-1,5.082015E-2,8.043563E-3,-6.6297343E-3,-1.2672701E-1,-1.4056729E-2,-3.9532906E-3,2.9214947E-2,1.0779989E-1,-2.7783174E-2,3.4141405E-3,-2.0357228E-3,-1.4480165E-1,-5.473163E-3,6.0352884E-2,1.8873018E-3,8.14882E-3,-3.461433E-3,-1.4894113E-3,-2.8744617E-3,-7.2173094E-3,1.2548638E-3,-3.2559637E-2,-4.0737496E-4,7.4939966E-2,9.467737E-4,-1.6439334E-3,-2.3334473E-3,5.696684E-4,1.0534232E-1,1.0854903E-3,1.6203952E-3,5.7409173E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,17,19,-1,-1,21,23,25,-1,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,35,-1,37,-1,-1,-1,-1,39,-1,-1,-1],"loss_changes":[3.8269866E-1,2.7346003E-1,1.9827783E-1,1.7910653E-1,0E0,1.06374055E-1,0E0,1.6257149E-1,7.868773E-2,5.2248284E-2,0E0,5.4781463E-2,2.0496994E-2,0E0,0E0,3.7054278E-2,5.400899E-2,3.2948457E-2,0E0,0E0,1.51335E-2,1.6852144E-2,1.9031718E-2,0E0,0E0,0E0,1.5405114E-2,0E0,0E0,0E0,1.0548575E-2,0E0,2.0059146E-2,0E0,0E0,0E0,0E0,1.1879481E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,17,17,20,20,21,21,22,22,26,26,30,30,32,32,37,37],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,18,20,-1,-1,22,24,26,-1,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,36,-1,38,-1,-1,-1,-1,40,-1,-1,-1],"split_conditions":[3.4707712E8,5.412096E3,2.774E3,1E0,8.295022E-3,2.4241872E0,-7.0141805E-3,5.070136E7,1.21719E5,1.5791555E9,8.043563E-3,6.5549725E5,2.83E2,-1.4056729E-2,-3.9532906E-3,1.1693485E0,2.4190365E6,2.83E2,3.4141405E-3,-2.0357228E-3,6.948189E7,9.069479E-1,9.787E3,1.8873018E-3,8.14882E-3,-3.461433E-3,2.710413E5,-2.8744617E-3,-7.2173094E-3,1.2548638E-3,1.7302E4,-4.0737496E-4,2.3615636E5,9.467737E-4,-1.6439334E-3,-2.3334473E-3,5.696684E-4,6.976363E8,1.0854903E-3,1.6203952E-3,5.7409173E-3],"split_indices":[7,64,0,28,0,51,0,57,1,7,0,40,0,0,0,54,44,0,0,0,7,69,9,0,0,0,40,0,0,0,9,0,45,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,5.3E1,3E0,5E1,3E0,4.6E1,7E0,4.4E1,6E0,2.8E1,1.8E1,3E0,4E0,3.3E1,1.1E1,2.3E1,5E0,4E0,1.4E1,1.6E1,1.7E1,7E0,4E0,7E0,1.6E1,4E0,1E1,7E0,9E0,3E0,1.4E1,1E1,6E0,6E0,3E0,8E0,6E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.8314173E-3,-3.0353786E-2,2.0790528E-1,-2.3175654E-1,-1.3410574E-2,1.273021E-2,4.349704E-3,-3.1426456E-3,-1.4365312E-2,5.1338587E-2,-6.9422655E-2,-9.65634E-3,1.01682454E-1,-1.1646576E-1,-2.4433285E-3,1.8886267E-3,-3.7670948E-2,1.3250731E-1,7.2089356E-4,-1.4734446E-1,-6.558869E-2,-3.5374075E-2,4.1735813E-2,-2.4468997E-3,-2.608844E-4,1.4966428E-1,1.883017E-3,-2.7185094E-3,-1.6197023E-1,-3.6689057E-3,-2.3902867E-4,-8.961303E-4,-3.0089486E-3,2.687906E-3,2.1890086E-5,7.2421483E-3,2.8920968E-3,-3.2088284E-3,-7.806052E-3,-1.3068253E-3,1.2391275E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,-1,-1,35,-1,-1,37,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1142234E-1,3.074213E-1,6.349191E-2,8.817619E-2,3.118366E-1,0E0,0E0,0E0,0E0,1.2381069E-1,1.4465755E-1,3.007525E-2,5.5418044E-2,3.275335E-2,3.057341E-2,0E0,7.4742865E-3,1.7872095E-2,0E0,9.543568E-3,1.3510805E-2,1.4140488E-2,7.5862017E-3,0E0,0E0,8.213371E-3,0E0,0E0,8.043081E-3,0E0,0E0,6.9559426E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,19,19,20,20,21,21,22,22,25,25,28,28,31,31],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,-1,-1,36,-1,-1,38,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8937003E2,9.2433E4,1.5804847E7,2.6666668E-1,3.0327434E7,1.273021E-2,4.349704E-3,-3.1426456E-3,-1.4365312E-2,1.6511278E1,2.711196E3,5.1056176E-1,1.8831E5,1.4780533E-4,1.8027E4,1.8886267E-3,1.3101191E1,7.6363635E-1,7.2089356E-4,1.9846153E1,3.4E1,1.069938E6,3.226E4,-2.4468997E-3,-2.608844E-4,8.70707E0,1.883017E-3,-2.7185094E-3,4.382474E5,-3.6689057E-3,-2.3902867E-4,3.3004024E9,-3.0089486E-3,2.687906E-3,2.1890086E-5,7.2421483E-3,2.8920968E-3,-3.2088284E-3,-7.806052E-3,-1.3068253E-3,1.2391275E-3],"split_indices":[68,1,1,69,57,0,0,0,0,68,64,39,41,50,2,0,68,69,0,68,3,41,2,0,0,65,0,0,40,0,0,5,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9E1,9E0,6E0,8.4E1,4E0,5E0,3E0,3E0,3.9E1,4.5E1,1.8E1,2.1E1,2.6E1,1.9E1,6E0,1.2E1,1.5E1,6E0,1.5E1,1.1E1,1.1E1,8E0,7E0,5E0,1.2E1,3E0,3E0,1.2E1,8E0,3E0,6E0,5E0,5E0,3E0,9E0,3E0,3E0,9E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[9.909078E-3,-2.8303757E-2,8.916528E-2,-9.788609E-3,-1.4323531E-2,1.2504336E-2,4.878526E-2,6.307957E-2,-5.8484387E-2,8.34023E-2,-8.676164E-2,-7.1874973E-3,1.2884097E-1,-8.174694E-2,-2.299259E-2,3.8316663E-2,4.9173348E-3,-8.103612E-3,1.5492379E-3,2.75674E-3,-1.397343E-3,7.6417457E-3,3.0467126E-3,-1.3211191E-3,-9.4303966E-2,1.3865001E-3,-4.890578E-2,-8.19058E-4,6.933348E-2,-8.5698854E-4,-1.05014645E-1,-1.1759698E-4,-2.644196E-3,1.0146307E-3,4.717735E-3,-5.237304E-3,-1.5146607E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,-1,7,-1,9,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,31,-1,33,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4475595E-1,2.1037966E-1,2.8542662E-1,0E0,2.5321683E-1,0E0,1.5738755E-1,1.2568927E-1,3.6925018E-2,3.1398043E-2,9.341574E-2,2.7114982E-2,2.680692E-2,1.5095666E-2,3.0132828E-2,2.2639034E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5458658E-2,0E0,7.816222E-3,0E0,1.2079656E-2,0E0,1.7410427E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,24,24,26,26,28,28,30,30],"right_children":[2,4,6,-1,8,-1,10,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,32,-1,34,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.369606E6,1.6451234E-2,-9.788609E-3,1.5046106E7,1.2504336E-2,1.650269E10,1.118632E3,8.762869E-2,1E0,1E0,1.3959545E7,1.4117E4,2.5107703E-1,1.3657556E-1,4.4E1,4.9173348E-3,-8.103612E-3,1.5492379E-3,2.75674E-3,-1.397343E-3,7.6417457E-3,3.0467126E-3,-1.3211191E-3,8.8887E4,1.3865001E-3,4.423E3,-8.19058E-4,1.1751899E6,-8.5698854E-4,2.022832E1,-1.1759698E-4,-2.644196E-3,1.0146307E-3,4.717735E-3,-5.237304E-3,-1.5146607E-3],"split_indices":[2,57,69,0,57,0,5,64,53,27,27,12,9,39,53,3,0,0,0,0,0,0,0,0,41,0,2,0,44,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.6E1,3.6E1,4E0,7.2E1,5E0,3.1E1,2.6E1,4.6E1,2.5E1,6E0,1.3E1,1.3E1,2.7E1,1.9E1,1.1E1,1.4E1,3E0,3E0,3E0,1E1,6E0,7E0,6E0,2.1E1,6E0,1.3E1,4E0,7E0,3E0,1.8E1,3E0,1E1,4E0,3E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.5435862E-2,-3.2863114E-2,8.022434E-2,8.308631E-3,-4.86535E-2,3.945697E-2,1.9523335E-1,-1.7143324E-2,-1.4913687E-1,9.4008066E-2,7.908328E-3,1.0435308E-2,2.3061114E-3,1.6276492E-2,-8.753269E-2,-5.4580264E-2,-9.600132E-3,5.5656573E-3,2.1863047E-3,-4.1323137E-2,4.5170352E-2,1.04347706E-1,-1.0089132E-2,-1.4541814E-1,5.9936056E-4,3.399975E-4,-4.463242E-3,-2.653538E-3,1.8406264E-4,3.25684E-3,7.2600535E-4,7.075971E-4,7.175309E-3,-2.5404964E-2,3.1444372E-3,-3.1380153E-3,-8.161414E-3,-2.3538866E-3,-6.9485498E-3,7.411758E-4,-1.8116644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,-1,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,39,-1,-1],"loss_changes":[3.155582E-1,2.1573433E-1,1.9390708E-1,0E0,1.7158699E-1,5.604998E-2,5.9625894E-2,1.02577925E-1,7.790521E-2,1.1641234E-2,4.222479E-2,0E0,0E0,7.071435E-2,8.420695E-2,2.2154337E-2,0E0,0E0,0E0,9.446224E-3,9.889843E-3,3.536787E-2,3.2334086E-2,1.4821783E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0908779E-2,0E0,0E0,0E0,0E0,1.2544738E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,19,19,20,20,21,21,22,22,23,23,33,33,38,38],"right_children":[2,4,6,-1,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,40,-1,-1],"split_conditions":[8.8579214E-1,6.169213E0,1.5605432E9,8.308631E-3,7.752181E7,4.86911E-1,4.4E1,8.5154843E-1,3.5E1,8.462185E0,1.5705264E0,1.0435308E-2,2.3061114E-3,2.0477528E1,2.0765625E1,1.0625348E8,-9.600132E-3,5.5656573E-3,2.1863047E-3,4.0291533E8,9.641E3,1.84702E5,1.2424786E2,9.513E3,5.9936056E-4,3.399975E-4,-4.463242E-3,-2.653538E-3,1.8406264E-4,3.25684E-3,7.2600535E-4,7.075971E-4,7.175309E-3,1.0666667E1,3.1444372E-3,-3.1380153E-3,-8.161414E-3,-2.3538866E-3,8.69E2,7.411758E-4,-1.8116644E-3],"split_indices":[54,66,7,0,57,69,3,39,3,65,69,0,0,70,70,7,0,0,0,7,2,1,68,9,0,0,0,0,0,0,0,0,0,68,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,5.7E1,4.2E1,3E0,5.4E1,3.2E1,1E1,4.2E1,1.2E1,1.1E1,2.1E1,7E0,3E0,2.9E1,1.3E1,6E0,6E0,5E0,6E0,9E0,1.2E1,6E0,2.3E1,8E0,5E0,3E0,3E0,6E0,3E0,5E0,7E0,3E0,3E0,2E1,3E0,4E0,4E0,7E0,1.3E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.842198E-3,-2.1244808E-1,2.0210683E-2,-3.3984035E-3,-1.409406E-2,6.281207E-2,-6.308055E-2,4.3183185E-2,1.2676288E-2,-3.1623814E-2,-8.832591E-3,1.0114277E-2,9.8816805E-2,-7.799911E-2,6.428915E-2,-4.315913E-3,2.1742856E-2,1.347898E-1,6.267483E-4,-6.557982E-3,-2.1594963E-3,-3.581509E-4,4.16931E-3,1.02183506E-1,-3.9712377E-3,7.0163338E-3,4.715265E-4,5.4127566E-4,6.255789E-3,3.2379897E-3,-1.5969215E-2,-2.0773278E-3,-1.5909036E-3,8.3115784E-4,-1.1608438E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,-1,15,17,19,21,-1,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1,31,-1,33,-1,-1],"loss_changes":[4.0065587E-1,9.6298695E-2,3.5524607E-1,0E0,0E0,2.863684E-1,1.4489722E-1,1.13064624E-1,0E0,1.3400626E-1,0E0,5.3382795E-2,6.7789495E-2,3.3755824E-2,2.4241727E-2,0E0,7.693876E-2,5.3210467E-2,0E0,0E0,0E0,0E0,0E0,3.265973E-2,2.8678114E-2,0E0,0E0,0E0,0E0,0E0,1.1943147E-2,0E0,1.0537913E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,16,16,17,17,23,23,24,24,30,30,32,32],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,-1,16,18,20,22,-1,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1,32,-1,34,-1,-1],"split_conditions":[9.7888E4,1.44E2,5.9633E7,-3.3984035E-3,-1.409406E-2,1.1811385E4,1.5081978E8,5.235739E8,1.2676288E-2,1.9649E4,-8.832591E-3,1.9988E4,1.4046429E1,5.08473E5,9.03392E7,-4.315913E-3,2.083117E1,2.3559767E5,6.267483E-4,-6.557982E-3,-2.1594963E-3,-3.581509E-4,4.16931E-3,9E0,5.6449033E-2,7.0163338E-3,4.715265E-4,5.4127566E-4,6.255789E-3,3.2379897E-3,2E0,-2.0773278E-3,8.155485E-2,8.3115784E-4,-1.1608438E-3],"split_indices":[1,0,57,0,0,64,57,7,0,2,0,12,66,41,57,0,70,45,0,0,0,0,0,29,39,0,0,0,0,0,8,0,50,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7E0,9.8E1,4E0,3E0,6.5E1,3.3E1,6.1E1,4E0,2.8E1,5E0,3.9E1,2.2E1,1.9E1,9E0,3E0,3.6E1,1.5E1,7E0,4E0,1.5E1,3E0,6E0,8E0,2.8E1,1.2E1,3E0,3E0,5E0,3E0,2.5E1,7E0,1.8E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.181448E-2,-5.5569194E-2,4.0052373E-2,-1.78459E-1,-2.987985E-2,5.5202406E-2,-6.387775E-3,-1.34715745E-2,-3.307358E-3,2.856089E-2,-1.1896901E-1,1.0634608E-1,2.2676121E-2,1.4592548E-1,-5.443628E-3,-2.1639362E-3,-1.606858E-1,7.327786E-3,2.408209E-2,4.032706E-3,8.199733E-3,1.023817E-2,7.857126E-4,-4.470322E-2,2.3709647E-2,-7.912149E-3,-3.0072844E-3,-1.0384662E-3,2.251687E-3,-7.151866E-3,2.1968395E-3,2.2827693E-4,-2.767564E-3,5.3025775E-2,-1.736961E-2,-7.105293E-4,1.4760782E-3,4.0181875E-3,7.547609E-4,7.467596E-4,-2.0589398E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,19,21,23,-1,25,-1,27,-1,29,-1,-1,31,33,-1,-1,-1,-1,35,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4732298E-1,1.8092825E-1,1.5059844E-1,1.121656E-1,2.6371965E-1,7.577361E-2,0E0,0E0,0E0,1.2451307E-1,4.9381822E-2,8.9796156E-2,2.9956657E-2,7.726093E-2,2.967354E-2,0E0,1.1916727E-2,0E0,1.3419647E-2,0E0,1.740344E-2,0E0,0E0,1.1011418E-2,1.8985307E-2,0E0,0E0,0E0,0E0,8.259362E-3,0E0,0E0,0E0,1.118546E-2,8.184546E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,20,20,23,23,24,24,29,29,33,33,34,34],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,20,22,24,-1,26,-1,28,-1,30,-1,-1,32,34,-1,-1,-1,-1,36,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1],"split_conditions":[3.365958E8,1.73E2,1.8205798E7,4E1,2.5927516E11,9.275501E-1,-6.387775E-3,-1.34715745E-2,-3.307358E-3,8.977316E0,7.157903E7,3.8E1,2E0,1.1317E4,2E0,-2.1639362E-3,3.974E3,7.327786E-3,1.5493506E1,4.032706E-3,3.2736E4,1.023817E-2,7.857126E-4,1.728E3,1.61349E5,-7.912149E-3,-3.0072844E-3,-1.0384662E-3,2.251687E-3,5.2E1,2.1968395E-3,2.2827693E-4,-2.767564E-3,1.82E2,1.7148E4,-7.105293E-4,1.4760782E-3,4.0181875E-3,7.547609E-4,7.467596E-4,-2.0589398E-3],"split_indices":[7,0,59,3,43,69,0,0,0,66,7,3,8,9,8,0,10,0,68,0,9,0,0,2,41,0,0,0,0,8,0,0,0,10,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,5.8E1,4.9E1,9E0,4.9E1,4.6E1,3E0,3E0,6E0,3E1,1.9E1,1.7E1,2.9E1,6E0,2.4E1,8E0,1.1E1,9E0,8E0,4E0,2.5E1,3E0,3E0,1E1,1.4E1,8E0,3E0,3E0,5E0,1.9E1,6E0,3E0,7E0,8E0,6E0,1.6E1,3E0,3E0,5E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-3.3926292E-4,-3.800852E-2,1.1008233E-1,-2.1347788E-1,-2.1979481E-2,1.6478916E-1,5.58123E-3,-1.3077648E-2,-3.0494542E-3,-1.12992465E-1,4.798151E-3,1.1376639E-2,9.893888E-2,-3.0784213E-3,1.8704928E-3,-1.3592508E-1,5.4087828E-5,2.9376393E-2,-4.1224852E-2,1.51127E-3,5.329668E-3,-7.7004856E-3,-2.6729354E-3,8.022385E-2,-1.3905261E-2,-2.4410166E-2,-4.0367907E-3,1.2388512E-1,-5.319167E-4,-9.6464856E-4,1.3717066E-3,-2.7316262E-3,-4.5932536E-3,1.7628559E-3,6.7454027E-3,9.902515E-4,-1.3584917E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,21,-1,23,25,-1,-1,-1,-1,27,29,31,-1,33,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[4.742075E-1,2.3437926E-1,1.6487107E-1,6.805441E-2,1.9307405E-1,9.747964E-2,3.5019E-2,0E0,0E0,4.7329336E-2,7.1343236E-2,0E0,1.6059026E-2,0E0,0E0,3.608665E-2,0E0,9.1346785E-2,1.705984E-2,0E0,0E0,0E0,0E0,7.829916E-2,9.578651E-3,1.3573835E-2,0E0,2.8569683E-2,0E0,0E0,0E0,0E0,1.0341568E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,15,15,17,17,18,18,23,23,24,24,25,25,27,27,32,32],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,22,-1,24,26,-1,-1,-1,-1,28,30,32,-1,34,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[1.3137E4,5.279518E7,2.797345E6,9E0,1.1127948E3,6.4417176E-2,1.5438823E7,-1.3077648E-2,-3.0494542E-3,8.8132757E-1,7.5179994E-1,1.1376639E-2,1.2943E4,-3.0784213E-3,1.8704928E-3,1.3259058E-4,5.4087828E-5,9.20242E0,8.965631E7,1.51127E-3,5.329668E-3,-7.7004856E-3,-2.6729354E-3,7.408377E-1,3.095172E5,2.85836E5,-4.0367907E-3,2.7958122E8,-5.319167E-4,-9.6464856E-4,1.3717066E-3,-2.7316262E-3,6.983E3,1.7628559E-3,6.7454027E-3,9.902515E-4,-1.3584917E-3],"split_indices":[2,5,41,29,45,69,1,0,0,39,39,0,9,0,0,54,0,65,57,0,0,0,0,69,40,1,0,5,0,0,0,0,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,8.4E1,2.8E1,6E0,7.8E1,1.8E1,1E1,3E0,3E0,1.7E1,6.1E1,6E0,1.2E1,3E0,7E0,1.4E1,3E0,4E1,2.1E1,4E0,8E0,8E0,6E0,1.8E1,2.2E1,1.7E1,4E0,1.2E1,6E0,1.9E1,3E0,5E0,1.2E1,4E0,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-4.9484414E-3,-3.0573659E-2,7.747198E-2,-7.949267E-2,-4.0758713E-3,1.6831698E-1,-2.7399426E-2,-2.8160889E-2,-1.18680716E-1,2.5130546E-2,-4.3717537E-2,2.0984857E-1,9.673482E-4,2.0850638E-2,-5.573561E-3,1.302098E-3,-2.1011957E-3,-1.4724681E-1,-1.3856672E-3,8.8251315E-2,-6.123654E-3,-1.283933E-4,-3.6343622E-3,4.449858E-3,1.039839E-2,2.1339138E-3,-5.137562E-4,-3.0799925E-3,-7.7886363E-3,5.4329163E-3,1.0998275E-3,-1.7469241E-3,1.5411782E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1357271E-1,9.9486925E-2,2.332608E-1,5.0970823E-2,6.014599E-2,7.407498E-2,6.1583854E-2,1.7021855E-2,3.2372966E-2,5.9724096E-2,3.6027584E-2,8.3010495E-3,0E0,9.0132505E-3,0E0,0E0,0E0,1.48266405E-2,0E0,2.0398214E-2,3.1610172E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,3.572374E6,6.1358623E9,8.8824E4,2.7260774E7,1E0,3.2944346E9,2.3E1,2.1272728E1,2E0,5.4357555E6,3.5258648E-1,9.673482E-4,8.106617E-1,-5.573561E-3,1.302098E-3,-2.1011957E-3,5.432393E-1,-1.3856672E-3,1.4117E4,1.4960615E3,-1.283933E-4,-3.6343622E-3,4.449858E-3,1.039839E-2,2.1339138E-3,-5.137562E-4,-3.0799925E-3,-7.7886363E-3,5.4329163E-3,1.0998275E-3,-1.7469241E-3,1.5411782E-3],"split_indices":[2,12,5,12,57,20,7,3,70,8,62,39,0,39,0,0,0,39,0,9,4,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,7.6E1,2.3E1,2.6E1,5E1,1.2E1,1.1E1,1.2E1,1.4E1,2.9E1,2.1E1,9E0,3E0,8E0,3E0,3E0,9E0,1E1,4E0,9E0,2E1,1.1E1,1E1,3E0,6E0,4E0,4E0,4E0,6E0,5E0,4E0,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.1791591E-2,-6.353292E-2,4.639453E-2,-1.18505016E-1,-1.5377846E-2,9.4657876E-2,-4.4665743E-2,-7.255076E-2,-1.2821117E-2,-3.21256E-2,4.66324E-2,1.2993144E-2,6.3991964E-2,7.912091E-3,-1.2841836E-1,1.5380697E-3,-9.857025E-2,1.9340568E-3,-4.46523E-2,4.643149E-3,-1.1203228E-3,1.02168396E-1,1.9004308E-2,-1.6521363E-3,2.610596E-3,-8.557554E-4,-8.845558E-3,-5.0709166E-2,-6.564091E-3,-5.436632E-2,5.4618146E-4,5.8583565E-2,6.468213E-3,-9.822989E-3,3.3940463E-3,-3.1394912E-3,-4.205803E-4,-6.980763E-4,-6.685243E-2,3.909639E-3,3.663445E-4,-1.2240114E-3,1.2819067E-3,-1.5093788E-3,-4.046813E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,27,-1,29,-1,-1,31,33,-1,-1,-1,-1,35,-1,37,-1,39,-1,41,-1,-1,-1,-1,43,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1906343E-1,1.4642005E-1,2.2397164E-1,1.9172317E-1,3.3692595E-2,1.8953991E-1,8.027962E-2,6.543756E-2,0E0,2.6091976E-2,3.37479E-2,0E0,4.9582407E-2,3.144516E-2,5.4082863E-2,0E0,3.836891E-2,0E0,1.29343495E-2,0E0,0E0,2.4464026E-2,2.6552219E-2,0E0,0E0,0E0,0E0,9.213198E-3,0E0,7.980224E-3,0E0,1.525433E-2,0E0,8.897658E-3,0E0,0E0,0E0,0E0,8.727692E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,16,16,18,18,21,21,22,22,27,27,29,29,31,31,33,33,38,38],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,28,-1,30,-1,-1,32,34,-1,-1,-1,-1,36,-1,38,-1,40,-1,42,-1,-1,-1,-1,44,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9552084E3,2.667836E6,7.096748E7,1.697225E6,7.5748205E6,7.412844E0,1.0712826E1,3.842849E5,-1.2821117E-2,3.607307E7,1.1663035E9,1.2993144E-2,2.8399336E7,1.9649E4,7.539E3,1.5380697E-3,2.6078947E1,1.9340568E-3,6.542E3,4.643149E-3,-1.1203228E-3,1.5353E4,1.3400168E7,-1.6521363E-3,2.610596E-3,-8.557554E-4,-8.845558E-3,6.973054E7,-6.564091E-3,9.848E3,5.4618146E-4,1.5903609E0,6.468213E-3,1E0,3.3940463E-3,-3.1394912E-3,-4.205803E-4,-6.980763E-4,1.09141096E8,3.909639E-3,3.663445E-4,-1.2240114E-3,1.2819067E-3,-1.5093788E-3,-4.046813E-3],"split_indices":[64,44,57,12,62,65,66,40,0,5,5,0,57,2,9,0,68,0,2,0,0,2,1,0,0,0,0,7,0,9,0,51,0,22,0,0,0,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,5.5E1,4.9E1,2.5E1,3E1,3.2E1,1.7E1,2.1E1,4E0,2.4E1,6E0,3E0,2.9E1,1.1E1,6E0,4E0,1.7E1,3E0,2.1E1,3E0,3E0,1.5E1,1.4E1,6E0,5E0,3E0,3E0,1E1,7E0,1.8E1,3E0,9E0,6E0,1E1,4E0,6E0,4E0,5E0,1.3E1,5E0,4E0,7E0,3E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.0717857E-2,2.4003128E-3,1.7536616E-1,-2.9168421E-2,5.1821563E-2,2.2912546E-1,1.036301E-3,-1.1474531E-2,-4.4776993E-3,9.87852E-2,2.2266874E-2,1.18127195E-2,5.016036E-3,-8.501014E-2,2.1305136E-2,5.513498E-3,9.133066E-4,4.481407E-2,-1.2272446E-3,-1.12405285E-1,-1.0871179E-3,-4.265953E-2,3.9468553E-2,1.0629211E-3,3.8138726E-3,-1.9691468E-3,-5.806702E-3,3.779438E-4,-3.248821E-3,1.206595E-1,-7.830229E-3,6.3457745E-3,1.9165726E-3,3.4033207E-4,-1.7514403E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,-1,19,21,-1,-1,23,-1,25,-1,27,29,-1,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[2.9418787E-1,1.4816718E-1,8.400676E-2,7.532464E-2,4.9156204E-2,8.421898E-3,0E0,1.1751385E-1,0E0,2.8744996E-2,2.870737E-2,0E0,0E0,2.175378E-2,4.1185476E-2,0E0,0E0,1.276806E-2,0E0,8.629769E-3,0E0,1.4067763E-2,1.0463108E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.356177E-2,9.581069E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,13,13,14,14,17,17,19,19,21,21,22,22,29,29,30,30],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,-1,20,22,-1,-1,24,-1,26,-1,28,30,-1,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[1.3127055E4,3.4707712E8,9.299267E9,5.501383E9,9.275501E-1,7.412844E0,1.036301E-3,1E0,-4.4776993E-3,1.5E1,3.162046E6,1.18127195E-2,5.016036E-3,2.4593212E6,4.329583E5,5.513498E-3,9.133066E-4,4.7E1,-1.2272446E-3,2.3168E5,-1.0871179E-3,1.9003961E5,3.42E2,1.0629211E-3,3.8138726E-3,-1.9691468E-3,-5.806702E-3,3.779438E-4,-3.248821E-3,9.5032835E0,8.155485E-2,6.3457745E-3,1.9165726E-3,3.4033207E-4,-1.7514403E-3],"split_indices":[4,7,12,5,69,65,0,8,0,8,44,0,0,63,44,0,0,8,0,12,0,44,10,0,0,0,0,0,0,65,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.3E1,1E1,5.7E1,3.6E1,7E0,3E0,4.7E1,1E1,1.3E1,2.3E1,4E0,3E0,1.4E1,3.3E1,9E0,4E0,1.6E1,7E0,9E0,5E0,7E0,2.6E1,1.2E1,4E0,3E0,6E0,3E0,4E0,9E0,1.7E1,6E0,3E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.3093892E-3,1.2667894E-2,-8.684573E-3,-8.915071E-3,1.8417999E-1,-1.0061725E-1,1.2860652E-2,1.1041455E-2,3.5349126E-3,-2.2061167E-3,-5.998235E-3,-4.499706E-2,2.6371624E-2,-9.0649635E-2,2.3875334E-3,6.165036E-3,1.294524E-2,-6.3420385E-3,-1.9662564E-3,3.0318504E-2,-1.3041662E-3,2.0125767E-3,5.4484893E-2,2.206643E-3,-1.0389354E-2,5.2092987E-4,7.483168E-2,3.4727316E-4,-1.8326441E-3,1.4763542E-3,4.7734575E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,17,-1,-1,19,-1,-1,21,-1,23,25,-1,27,-1,29,-1,-1,-1,-1],"loss_changes":[2.1754794E-1,3.644125E-1,0E0,1.7701957E-1,5.9617758E-2,2.516386E-2,5.7470046E-2,0E0,0E0,0E0,0E0,6.912187E-2,9.039168E-2,1.8684141E-2,0E0,0E0,4.1403405E-2,0E0,0E0,2.6568372E-2,0E0,1.2101914E-2,1.7221227E-2,0E0,9.798616E-3,0E0,1.6552202E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,13,13,16,16,19,19,21,21,22,22,24,24,26,26],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,18,-1,-1,20,-1,-1,22,-1,24,26,-1,28,-1,30,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.1803382E4,-8.684573E-3,1.13892675E-4,3.813527E2,5.614759E5,1E0,1.1041455E-2,3.5349126E-3,-2.2061167E-3,-5.998235E-3,3.365958E8,1.8955729E1,2.8282486E1,2.3875334E-3,6.165036E-3,8.63558E-1,-6.3420385E-3,-1.9662564E-3,7.010949E6,-1.3041662E-3,2.1923357E1,2.2622842E6,2.206643E-3,4.6E1,5.2092987E-4,7.673114E8,3.4727316E-4,-1.8326441E-3,1.4763542E-3,4.7734575E-3],"split_indices":[57,4,0,50,68,40,8,0,0,0,0,7,70,70,0,0,39,0,0,60,0,70,44,0,3,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,9.7E1,4E0,8.7E1,1E1,1.6E1,7.1E1,5E0,5E0,8E0,8E0,1.3E1,5.8E1,9E0,4E0,5E0,5.3E1,3E0,6E0,3.8E1,1.5E1,1.8E1,2E1,3E0,1.5E1,7E0,1.3E1,1E1,5E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-2.1684999E-3,-2.421261E-2,9.837514E-2,-1.1853587E-2,-1.1228797E-2,6.847324E-2,8.025578E-3,1.5908306E-3,-8.688815E-3,3.1269476E-2,4.598498E-3,-1.10857515E-2,7.825564E-3,2.42836E-3,1.3729709E-6,-7.527546E-2,1.9369519E-3,-9.251636E-4,-4.296048E-3,7.997662E-2,-9.879166E-3,8.752537E-5,5.453913E-3,-4.8428416E-2,1.9040572E-2,-1.3609256E-3,-3.5639259E-3,-4.2054473E-4,4.0369183E-2,3.8619097E-3,1.0613626E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,-1,15,-1,-1,-1,17,19,-1,-1,21,23,-1,-1,25,27,-1,-1,-1,29,-1,-1],"loss_changes":[2.2180505E-1,2.6348642E-1,3.4386054E-2,0E0,1.9279207E-1,1.7903924E-2,0E0,1.7284809E-1,0E0,7.0221275E-3,0E0,5.949637E-2,0E0,0E0,0E0,1.3531245E-2,5.6117654E-2,0E0,0E0,2.865605E-2,6.000832E-2,0E0,0E0,1.0665614E-2,1.9423917E-2,0E0,0E0,0E0,1.1746945E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,11,11,15,15,16,16,19,19,20,20,23,23,24,24,28,28],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,-1,16,-1,-1,-1,18,20,-1,-1,22,24,-1,-1,26,28,-1,-1,-1,30,-1,-1],"split_conditions":[1.7123758E0,2.7571955E6,4.3201213E9,-1.1853587E-2,1.5081978E8,3.1346828E5,8.025578E-3,1.3127055E4,-8.688815E-3,8.645E3,4.598498E-3,6.7281544E-5,7.825564E-3,2.42836E-3,1.3729709E-6,1.6278E4,6.0881297E4,-9.251636E-4,-4.296048E-3,1.7367E5,2.518E3,8.752537E-5,5.453913E-3,1.66E3,1.3045E4,-1.3609256E-3,-3.5639259E-3,-4.2054473E-4,7.6363635E-1,3.8619097E-3,1.0613626E-3],"split_indices":[51,57,5,0,57,40,0,4,0,2,0,50,0,0,0,12,63,0,0,1,10,0,0,10,9,0,0,0,69,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,8.1E1,1.7E1,3E0,7.8E1,1.4E1,3E0,7.4E1,4E0,8E0,6E0,7E1,4E0,4E0,4E0,1.1E1,5.9E1,4E0,7E0,7E0,5.2E1,3E0,4E0,2.2E1,3E1,1.6E1,6E0,1.3E1,1.7E1,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.893769E-3,-4.022383E-2,7.829083E-2,-9.5232874E-2,8.262395E-3,4.0973503E-2,1.6699706E-1,-6.8775676E-2,-1.1078079E-2,4.6985713E-3,-1.1471341E-2,7.8225456E-2,-4.06418E-2,9.849924E-3,3.5381946E-3,-4.4428583E-2,-4.6689836E-3,-2.642711E-2,1.5035486E-3,1.0189108E-1,-2.5076445E-4,-4.125982E-3,-3.4936302E-4,-3.402508E-3,-2.249281E-2,-3.6794133E-2,8.774571E-4,1.17669545E-1,1.6116105E-3,9.256333E-4,-1.632524E-3,-2.0140194E-4,-5.3735282E-2,1.4937235E-3,5.9217727E-3,-2.2129259E-4,-2.927045E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,21,-1,-1,23,-1,25,-1,27,-1,-1,-1,-1,29,31,-1,33,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[3.757563E-1,1.7682077E-1,1.421201E-1,1.1471996E-1,7.2702475E-2,1.035101E-1,4.892683E-2,2.204208E-2,0E0,0E0,2.2387039E-2,4.6332598E-2,1.8895948E-2,0E0,0E0,1.2560569E-2,0E0,1.2380565E-2,0E0,1.3942286E-2,0E0,0E0,0E0,0E0,9.563784E-3,1.0635218E-2,0E0,1.7941192E-2,0E0,0E0,0E0,0E0,8.543532E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,12,12,15,15,17,17,19,19,24,24,25,25,27,27,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,22,-1,-1,24,-1,26,-1,28,-1,-1,-1,-1,30,32,-1,34,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[8.168E3,1.04856936E8,1.1803382E4,7.307441E-1,1.8955729E1,6.19999E9,1.0889231E1,2.4759493E1,-1.1078079E-2,4.6985713E-3,2.2539987E8,7.5305285E6,4.7308203E3,9.849924E-3,3.5381946E-3,3.9245284E0,-4.6689836E-3,4.664004E8,1.5035486E-3,1.26059235E2,-2.5076445E-4,-4.125982E-3,-3.4936302E-4,-3.402508E-3,5.817521E2,2.1821466E5,8.774571E-4,9.600312E4,1.6116105E-3,9.256333E-4,-1.632524E-3,-2.0140194E-4,2.7258823E1,1.4937235E-3,5.9217727E-3,-2.2129259E-4,-2.927045E-3],"split_indices":[2,7,4,51,70,5,66,68,0,0,44,59,4,0,0,68,0,12,0,70,0,0,0,0,4,40,0,40,0,0,0,0,70,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,6.5E1,4.4E1,3E1,3.5E1,3.2E1,1.2E1,2.7E1,3E0,5E0,3E1,2.2E1,1E1,6E0,6E0,1.8E1,9E0,2.3E1,7E0,1.7E1,5E0,3E0,7E0,6E0,1.2E1,1.9E1,4E0,1.3E1,4E0,3E0,9E0,7E0,1.2E1,3E0,1E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[9.353514E-3,-2.2271428E-2,8.1752606E-2,-8.745585E-3,-1.3203674E-2,9.433353E-3,5.355306E-2,2.416802E-2,-5.1164832E-2,1.1000288E-1,9.139872E-3,-5.3079035E-3,6.0128763E-2,-8.700619E-2,-2.4746116E-2,6.7905206E-3,5.4864712E-2,-2.136449E-3,1.451086E-3,2.5858392E-3,-2.1344805E-2,9.552204E-2,1.0189348E-2,-1.1202261E-1,-9.7009324E-4,-2.0442167E-4,-1.9870265E-3,3.9032293E-3,-1.4219873E-4,-3.3015995E-3,-7.1157813E-3,5.281222E-3,1.6780514E-3,1.5559085E-3,-7.6416536E-4,-1.7216621E-3,-5.6825913E-3,-1.1877334E-3,6.0751097E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,-1,7,-1,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,-1,29,31,33,35,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.788644E-1,1.3630915E-1,1.3100311E-1,0E0,1.1756772E-1,0E0,7.922232E-2,4.4838157E-2,3.6669135E-2,2.7882367E-2,2.8928172E-2,2.6376663E-2,3.260956E-2,2.4927482E-2,1.04295015E-2,0E0,1.684009E-2,0E0,0E0,0E0,1.5146728E-2,1.1956006E-2,7.127817E-3,1.1730552E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.7348524E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,20,20,21,21,22,22,23,23,30,30],"right_children":[2,4,6,-1,8,-1,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,-1,30,32,34,36,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2288E4,3.218612E7,1.6451234E-2,-8.745585E-3,2.2232678E7,9.433353E-3,1.5E1,1.6987854E1,1.4099028E-4,1.2320755E1,1.9649E4,4.4493478E2,1.4922581E1,4.6573498E1,4.0155898E6,6.7905206E-3,2.3278559E9,-2.136449E-3,1.451086E-3,2.5858392E-3,1.4111814E8,1.4117E4,8.1676E4,2.3737179E1,-9.7009324E-4,-2.0442167E-4,-1.9870265E-3,3.9032293E-3,-1.4219873E-4,-3.3015995E-3,8.771689E0,5.281222E-3,1.6780514E-3,1.5559085E-3,-7.6416536E-4,-1.7216621E-3,-5.6825913E-3,-1.1877334E-3,6.0751097E-4],"split_indices":[2,5,69,0,57,0,8,68,50,66,2,4,66,68,62,0,5,0,0,0,5,9,41,68,0,0,0,0,0,0,68,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.2E2,8.4E1,3.6E1,3E0,8.1E1,5E0,3.1E1,4.1E1,4E1,1.3E1,1.8E1,2.3E1,1.8E1,1.6E1,2.4E1,6E0,7E0,5E0,1.3E1,4E0,1.9E1,1E1,8E0,1.1E1,5E0,1.3E1,1.1E1,4E0,3E0,3E0,1.6E1,6E0,4E0,4E0,4E0,3E0,8E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.0878801E-2,3.6054023E-2,-5.7456926E-2,8.563855E-3,1.3904268E-1,-3.1578567E-2,-9.400841E-3,3.907737E-2,-2.4124376E-2,8.641356E-3,7.62301E-2,-7.91054E-2,6.040223E-3,1.1814831E-2,8.220802E-2,-5.9318338E-2,1.0418434E-2,6.8265747E-4,4.3109492E-3,-9.739495E-4,-4.5135557E-3,-6.899923E-4,1.5816952E-3,2.6639597E-3,-6.4905537E-3,1.032601E-1,1.2032175E-3,-1.3367342E-3,-3.6175235E-3,1.5073917E-3,-2.6418638E-4,-2.9143736E-2,9.772124E-4,1.7075811E-3,5.5479645E-3,-2.020549E-3,3.9074352E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[1.9131194E-1,2.2813445E-1,1.1758351E-1,6.577585E-2,5.002159E-2,4.875117E-2,0E0,3.9007667E-2,3.9520353E-2,0E0,1.2699287E-2,1.4910601E-2,1.1368564E-2,2.0494629E-2,1.1992201E-2,8.511186E-3,7.1020545E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.187697E-2,9.398185E-3,0E0,0E0,0E0,0E0,0E0,7.1699377E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,24,24,25,25,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[6.9754904E7,1.286E4,1.1221429E1,1.4163358E7,6.4417176E-2,2.7895107E4,-9.400841E-3,1.2733752E3,1.3045E4,8.641356E-3,1.1312172E-1,2.1129233E-1,1.5605432E9,4.4493478E2,2.7943066E3,9E0,1.6164E4,6.8265747E-4,4.3109492E-3,-9.739495E-4,-4.5135557E-3,-6.899923E-4,1.5816952E-3,2.6639597E-3,1.4243386E1,1.3081232E1,1.2032175E-3,-1.3367342E-3,-3.6175235E-3,1.5073917E-3,-2.6418638E-4,6.010474E-1,9.772124E-4,1.7075811E-3,5.5479645E-3,-2.020549E-3,3.9074352E-4],"split_indices":[57,2,65,57,69,45,0,64,9,0,50,39,7,4,4,29,9,0,0,0,0,0,0,0,68,68,0,0,0,0,0,39,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,8E1,2.9E1,6.4E1,1.6E1,2.6E1,3E0,3.3E1,3.1E1,7E0,9E0,1.1E1,1.5E1,2.1E1,1.2E1,1.5E1,1.6E1,3E0,6E0,4E0,7E0,9E0,6E0,5E0,1.6E1,8E0,4E0,8E0,7E0,6E0,1E1,9E0,7E0,3E0,5E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-6.2397677E-3,-4.1734338E-2,3.9318077E-2,-5.261427E-2,5.811247E-3,5.2167125E-2,-5.2444255E-3,-4.222381E-2,-6.9303615E-3,1.1731895E-2,8.1684634E-2,-2.9695258E-2,-9.813848E-2,-7.90505E-3,2.250585E-3,4.5514913E-4,9.359857E-2,1.6439302E-3,-3.846194E-2,-1.9852538E-3,-5.406415E-3,9.893352E-4,-1.105473E-3,1.0333794E-1,9.282079E-4,-4.4602446E-2,7.393188E-4,6.118695E-3,3.3134075E-3,5.493632E-4,-5.1517878E-2,-9.564591E-2,-3.7069947E-2,-5.5911522E-3,-1.268812E-3,-1.9009386E-3,6.791278E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,17,19,21,-1,-1,23,-1,25,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1,31,33,35,-1,-1,-1,-1],"loss_changes":[1.8436825E-1,1.2987992E-1,1.09647766E-1,6.1628252E-2,0E0,5.5138394E-2,0E0,3.7015967E-2,0E0,1.7073425E-2,2.1897525E-2,2.9985357E-2,8.373909E-3,8.77199E-3,0E0,0E0,1.3928056E-2,0E0,1.5441529E-2,0E0,0E0,0E0,0E0,9.53725E-3,0E0,1.6117238E-2,0E0,0E0,0E0,0E0,1.8768966E-2,1.4147252E-2,1.0991E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,11,11,12,12,13,13,16,16,18,18,23,23,25,25,30,30,31,31,32,32],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,18,20,22,-1,-1,24,-1,26,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1,32,34,36,-1,-1,-1,-1],"split_conditions":[3.4288502E8,5.412096E3,9.299267E9,1.0883114E0,5.811247E-3,9.8309404E-1,-5.2444255E-3,8.67241E9,-6.9303615E-3,1.9649E4,5.91633E5,8.977316E0,7.105352E5,4.739085E-2,2.250585E-3,4.5514913E-4,4.1649513E1,1.6439302E-3,9.319337E-1,-1.9852538E-3,-5.406415E-3,9.893352E-4,-1.105473E-3,1.0077E4,9.282079E-4,3.640909E6,7.393188E-4,6.118695E-3,3.3134075E-3,5.493632E-4,2.3E1,2.589813E-3,2.38477E6,-5.5911522E-3,-1.268812E-3,-1.9009386E-3,6.791278E-4],"split_indices":[7,64,12,50,0,51,0,5,0,2,1,66,40,50,0,0,69,0,39,0,0,0,0,9,0,57,0,0,0,0,3,50,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,6.3E1,4.9E1,6E1,3E0,4.6E1,3E0,5.6E1,4E0,2E1,2.6E1,4.7E1,9E0,1.4E1,6E0,4E0,2.2E1,5E0,4.2E1,4E0,5E0,5E0,9E0,1.9E1,3E0,3.8E1,4E0,6E0,1.3E1,4E0,3.4E1,7E0,2.7E1,4E0,3E0,2.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.2008813E-3,1.2648985E-2,-7.2876066E-3,-6.383567E-2,3.4337472E-2,-8.849885E-2,-7.933426E-4,2.0262532E-2,1.3843006E-1,-4.51584E-3,-1.6652977E-3,-4.9926154E-3,5.3260647E-2,9.907885E-3,2.3584387E-3,-7.166624E-2,8.881149E-3,2.9485138E-2,3.956248E-3,-1.0690381E-3,-4.34496E-3,3.5118724E-3,-7.5653964E-4,5.3212594E-2,-2.2181941E-2,-1.3253658E-2,1.5898636E-3,3.3967348E-3,7.1221695E-4,7.1797235E-4,-2.3936983E-3,3.0466839E-4,-1.2201911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,-1,15,17,-1,-1,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8617846E-1,1.6475794E-1,0E0,2.3038521E-2,1.10007524E-1,7.2246864E-3,0E0,5.77686E-2,5.6155875E-2,0E0,0E0,3.7637047E-2,2.467741E-2,0E0,0E0,7.023666E-3,2.3809208E-2,2.5698926E-2,0E0,0E0,0E0,0E0,1.5049311E-2,1.179491E-2,9.961163E-3,7.517529E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,11,11,12,12,15,15,16,16,17,17,22,22,23,23,24,24,25,25],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,-1,16,18,-1,-1,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1E0,-7.2876066E-3,5.127381E1,1.2768678E4,1.8009158E3,-7.933426E-4,9.8309404E-1,1.3309696E7,-4.51584E-3,-1.6652977E-3,3.4753364E-2,3.088763E7,9.907885E-3,2.3584387E-3,5.0717188E7,2.4E2,1.0875838E7,3.956248E-3,-1.0690381E-3,-4.34496E-3,3.5118724E-3,9.7752365E1,9.641E3,2.0195505E-1,1.1811135E6,1.5898636E-3,3.3967348E-3,7.1221695E-4,7.1797235E-4,-2.3936983E-3,3.0466839E-4,-1.2201911E-3],"split_indices":[57,24,0,68,64,64,0,51,59,0,0,69,44,0,0,57,10,44,0,0,0,0,68,2,50,59,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.7E1,5E0,2.1E1,7.6E1,1.3E1,8E0,6.8E1,8E0,9E0,4E0,3.9E1,2.9E1,3E0,5E0,6E0,3.3E1,1.9E1,1E1,3E0,3E0,3E0,3E1,1.3E1,6E0,2.3E1,7E0,7E0,6E0,3E0,3E0,1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[5.3067324E-3,-9.235121E-3,1.16758935E-1,-3.5940323E-4,-7.675413E-3,7.7252537E-3,1.6352852E-3,3.30251E-2,-4.9998935E-2,-4.2320974E-3,6.793956E-2,-8.8084176E-2,-5.2732034E-3,-2.4550872E-2,3.3562677E-3,1.0476574E-1,1.836921E-2,-1.4794896E-3,-4.85825E-3,2.051444E-3,-2.7523039E-2,-3.6084335E-3,-6.5984696E-2,6.2165307E-3,6.2896416E-2,2.694759E-3,-9.702193E-4,-2.6157036E-4,-2.379934E-3,-9.086927E-4,2.5184015E-3,-5.1036836E-3,-4.7457358E-4,3.8238254E-3,8.416785E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8259908E-1,1.495512E-1,5.699852E-2,1.6074157E-1,0E0,0E0,0E0,7.573479E-2,6.596773E-2,5.021531E-2,5.2840143E-2,2.3781255E-2,2.3626935E-2,2.0587303E-2,0E0,1.8873349E-2,2.6594762E-2,0E0,0E0,0E0,7.962059E-3,1.959586E-2,2.1683365E-2,0E0,9.419017E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0825377E4,6.313997E-1,4.0487366E2,4.3258228E7,-7.675413E-3,7.7252537E-3,1.6352852E-3,2.951724E1,1.8873918E3,2.3145154E0,9.690435E0,8.471288E10,5.9633E7,8.5154843E-1,3.3562677E-3,1.3047E4,1.0931163E9,-1.4794896E-3,-4.85825E-3,2.051444E-3,8.44629E7,1E0,4.3471072E2,6.2165307E-3,9E0,2.694759E-3,-9.702193E-4,-2.6157036E-4,-2.379934E-3,-9.086927E-4,2.5184015E-3,-5.1036836E-3,-4.7457358E-4,3.8238254E-3,8.416785E-4],"split_indices":[64,50,70,57,0,0,0,68,64,51,65,43,57,39,0,9,5,0,0,0,57,26,4,0,29,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,9.9E1,1.2E1,9.5E1,4E0,6E0,6E0,5.7E1,3.8E1,2.8E1,2.9E1,2E1,1.8E1,2.3E1,5E0,1.6E1,1.3E1,7E0,1.3E1,5E0,1.3E1,1.6E1,7E0,7E0,9E0,6E0,7E0,8E0,5E0,1.3E1,3E0,3E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.4676007E-4,9.654274E-3,-6.873463E-3,-2.3563176E-3,1.3669455E-1,2.1824956E-2,-4.7093492E-2,9.465969E-3,1.1556103E-3,9.852588E-4,7.003345E-2,-3.3537067E-2,-6.0600764E-3,-1.4879462E-2,5.137435E-2,9.839972E-2,-3.3786832E-4,-4.2776722E-2,1.70622E-3,-3.120522E-2,1.901201E-2,3.865826E-3,7.237246E-4,5.5461028E-3,1.5922701E-3,-5.2928276E-2,1.3539607E-3,-2.1497676E-2,-3.2103662E-3,-1.3044324E-3,1.6740372E-3,-1.7468516E-3,-4.6724807E-3,-1.3326688E-4,-1.9728816E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,-1,19,21,23,-1,25,-1,27,29,-1,-1,-1,-1,31,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6178642E-1,1.590083E-1,0E0,1.0488078E-1,7.583386E-2,6.322397E-2,3.829056E-2,0E0,0E0,3.6737822E-2,4.2733625E-2,2.2617798E-2,0E0,1.9958913E-2,1.3033038E-2,2.1163106E-2,0E0,2.2734288E-2,0E0,8.45748E-3,1.2849359E-2,0E0,0E0,0E0,0E0,1.2852289E-2,0E0,9.21683E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,13,13,14,14,15,15,17,17,19,19,20,20,25,25,27,27],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,-1,20,22,24,-1,26,-1,28,30,-1,-1,-1,-1,32,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.2768678E4,-6.873463E-3,7.959073E-1,8.50999E9,3.1982497E5,1.5088414E6,9.465969E-3,1.1556103E-3,1.3137E4,2.7987084E7,1E0,-6.0600764E-3,9.2954946E-1,2.3434343E0,9.275501E-1,-3.3786832E-4,2.9187E4,1.70622E-3,1E0,1.5426178E3,3.865826E-3,7.237246E-4,5.5461028E-3,1.5922701E-3,1.9041E4,1.3539607E-3,1.1697628E5,-3.2103662E-3,-1.3044324E-3,1.6740372E-3,-1.7468516E-3,-4.6724807E-3,-1.3326688E-4,-1.9728816E-3],"split_indices":[57,64,0,39,5,40,40,0,0,2,57,22,0,51,69,69,0,9,0,83,4,0,0,0,0,9,0,45,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,1.03E2,5E0,9.5E1,8E0,6.2E1,3.3E1,4E0,4E0,4.4E1,1.8E1,3E1,3E0,3.4E1,1E1,1.3E1,5E0,2.7E1,3E0,2.3E1,1.1E1,4E0,6E0,8E0,5E0,2.4E1,3E0,2E1,3E0,3E0,8E0,2.1E1,3E0,1.2E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.3740646E-3,-1.1502608E-2,1.4367871E-1,-4.3193027E-2,3.4797825E-2,8.299858E-3,2.4449155E-3,-8.141419E-3,-3.242046E-2,7.3124595E-2,-4.2750784E-5,-8.12278E-2,-6.5597617E-3,1.8093723E-3,4.95144E-3,-1.0575594E-3,1.2459842E-3,-8.0766185E-5,-1.0095008E-1,5.3410553E-3,-2.0853763E-2,-1.8876295E-3,-5.309072E-3,2.2844266E-2,-3.0354844E-2,-1.1981428E-3,2.9239038E-3,-5.676661E-2,-1.431063E-2,-4.7384106E-4,-3.1325454E-3,-1.1217392E-3,1.1185E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,23,-1,-1,25,27,-1,-1,29,31,-1,-1,-1,-1],"loss_changes":[2.1930572E-1,1.511656E-1,3.140299E-2,8.943436E-2,5.608685E-2,0E0,0E0,0E0,7.267429E-2,2.1144435E-2,1.6966285E-2,3.1113088E-2,7.521015E-2,0E0,0E0,0E0,0E0,0E0,1.4796913E-2,0E0,1.563986E-2,0E0,0E0,1.7779177E-2,1.1900313E-2,0E0,0E0,7.1613714E-3,1.0064513E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,10,10,11,11,12,12,18,18,20,20,23,23,24,24,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,24,-1,-1,26,28,-1,-1,30,32,-1,-1,-1,-1],"split_conditions":[1.0825377E4,3.4707712E8,2.9548872E9,3.218612E7,1.5046106E7,8.299858E-3,2.4449155E-3,-8.141419E-3,1.4616883E1,8.5731603E8,1.8489E4,2.07E2,1.7533583E1,1.8093723E-3,4.95144E-3,-1.0575594E-3,1.2459842E-3,-8.0766185E-5,3.3109E4,5.3410553E-3,4.5479352E2,-1.8876295E-3,-5.309072E-3,1E0,2.4E1,-1.1981428E-3,2.9239038E-3,8.1980075E5,1E0,-4.7384106E-4,-3.1325454E-3,-1.1217392E-3,1.1185E-3],"split_indices":[64,7,7,5,57,0,0,0,70,5,10,11,70,0,0,0,0,0,44,0,64,0,0,8,3,0,0,44,22,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.01E2,9E0,6E1,4.1E1,5E0,4E0,3E0,5.7E1,1.9E1,2.2E1,1.9E1,3.8E1,1.2E1,7E0,1.2E1,1E1,4E0,1.5E1,3E0,3.5E1,5E0,1E1,6E0,2.9E1,3E0,3E0,1E1,1.9E1,3E0,7E0,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-3.0301004E-3,6.665157E-3,-6.5961913E-3,-5.5094766E-3,1.2090724E-1,-1.5378403E-2,7.004531E-2,7.692578E-3,1.7022802E-3,-2.4768546E-2,2.1382675E-2,-3.17138E-4,4.5008506E-3,-3.695196E-2,1.9757095E-4,5.047163E-2,-4.825622E-5,1.6649732E-3,-4.3929145E-2,1.8264183E-3,-5.2664627E-4,3.4262734E-3,3.8658615E-4,-4.6227085E-3,-3.33494E-2,-4.152111E-3,-5.6181356E-2,-7.0527254E-4,1.5751767E-3,-2.8181307E-3,-2.4744563E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,23,-1,-1,-1,-1,-1,25,27,29,-1,-1,-1,-1],"loss_changes":[1.433216E-1,1.300157E-1,0E0,6.445697E-2,3.4983277E-2,2.67642E-2,2.7247477E-2,0E0,0E0,1.8561821E-2,1.0528695E-2,0E0,0E0,2.3298282E-2,1.147973E-2,7.6875035E-3,0E0,0E0,2.181957E-2,0E0,0E0,0E0,0E0,0E0,2.243219E-2,8.519971E-3,8.349407E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,18,18,24,24,25,25,26,26],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,24,-1,-1,-1,-1,-1,26,28,30,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825377E4,-6.5961913E-3,2.1729739E0,6.691E3,1.3137E4,3.077573E5,7.692578E-3,1.7022802E-3,1.9552084E3,1.8230173E5,-3.17138E-4,4.5008506E-3,1.8037184E2,1.19461E5,1.1335518E4,-4.825622E-5,1.6649732E-3,2.7227585E2,1.8264183E-3,-5.2664627E-4,3.4262734E-3,3.8658615E-4,-4.6227085E-3,7.609895E8,8.4395015E-1,8.3722024E7,-7.0527254E-4,1.5751767E-3,-2.8181307E-3,-2.4744563E-4],"split_indices":[57,64,0,51,9,2,40,0,0,64,40,0,0,64,41,4,0,0,64,0,0,0,0,0,5,39,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,9.2E1,5E0,8.4E1,8E0,7.5E1,9E0,4E0,4E0,6E1,1.5E1,3E0,6E0,4E1,2E1,6E0,9E0,3E0,3.7E1,4E0,1.6E1,3E0,3E0,4E0,3.3E1,1.5E1,1.8E1,1.2E1,3E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.294388E-3,-7.906155E-3,1.1624136E-3,8.734147E-3,-6.954644E-3,-2.7507804E-3,1.0921632E-1,-6.601795E-2,7.826495E-3,8.7707285E-3,2.0973317E-3,-1.1918991E-3,-4.337601E-3,-2.1518236E-2,2.2883115E-2,-7.354013E-3,-3.800356E-3,5.1263613E-3,1.3224819E-2,2.5627466E-3,-1.1656497E-3,6.2315505E-2,-3.0602752E-3,1.1794077E-3,4.255914E-3,-2.0771638E-2,1.5689667E-2,1.433832E-4,-2.055082E-3,1.7318494E-3,-2.3758012E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,15,17,19,-1,-1,21,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1],"loss_changes":[1.3112585E-1,0E0,1.3778692E-1,1.2378168E-1,0E0,6.542853E-2,5.2794024E-2,1.5756633E-2,3.7606332E-2,0E0,0E0,0E0,0E0,2.6675468E-2,5.0282024E-2,3.4376744E-2,0E0,0E0,4.176192E-2,0E0,0E0,1.38081275E-2,1.3609381E-2,0E0,0E0,1.3735233E-2,7.9565765E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15,18,18,21,21,22,22,25,25,26,26],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,16,18,20,-1,-1,22,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1],"split_conditions":[3.218612E7,-7.906155E-3,1.5081978E8,1.1803382E4,-6.954644E-3,4.1964554E-4,1.6451234E-2,7.9918444E5,2.4662777E1,8.7707285E-3,2.0973317E-3,-1.1918991E-3,-4.337601E-3,7.910084E5,1.53E2,2.8327732E6,-3.800356E-3,5.1263613E-3,1.5046106E7,2.5627466E-3,-1.1656497E-3,7.359468E8,1.1855E4,1.1794077E-3,4.255914E-3,4.349136E6,1.5360503E5,1.433832E-4,-2.055082E-3,1.7318494E-3,-2.3758012E-5],"split_indices":[5,0,57,4,0,54,69,40,68,0,0,0,0,40,10,44,0,0,57,0,0,5,2,0,0,44,40,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,3E0,1.1E2,1.06E2,4E0,9.6E1,1E1,1.3E1,8.3E1,3E0,7E0,7E0,6E0,2.8E1,5.5E1,2.4E1,4E0,4E0,5.1E1,5E0,1.9E1,1.2E1,3.9E1,7E0,5E0,2E1,1.9E1,1.1E1,9E0,7E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.1116747E-3,1.44678475E-2,-1.2588313E-1,2.2283437E-2,-7.2914646E-3,-9.902917E-3,-4.8746042E-2,1.19706625E-2,8.604814E-3,-1.2191389E-4,-3.5605866E-3,5.692198E-2,-8.85897E-3,1.5139296E-2,5.1853354E-3,-3.479786E-2,1.2763026E-2,-2.830238E-3,3.565057E-2,4.7945883E-5,-7.500824E-2,2.9715186E-3,5.116146E-3,5.1754218E-2,1.2386714E-4,2.4410721E-3,-7.4855174E-4,-1.5490388E-3,-4.9502426E-3,-1.1936995E-3,1.9778294E-2,1.480441E-4,3.1070376E-3,9.456945E-6,1.697276E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,-1,9,11,-1,-1,-1,13,15,17,-1,19,21,-1,23,25,27,-1,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1],"loss_changes":[1.7822239E-1,1.3991666E-1,6.8669215E-2,1.6559274E-1,0E0,0E0,1.0611406E-2,8.290039E-2,0E0,0E0,0E0,7.211654E-2,3.4656007E-2,3.21328E-2,0E0,3.9236173E-2,1.4226351E-2,0E0,7.769944E-3,1.6699461E-2,1.6059361E-2,0E0,1.5440996E-2,9.524763E-3,0E0,0E0,0E0,0E0,0E0,0E0,8.399208E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,22,22,23,23,30,30],"right_children":[2,4,6,8,-1,-1,10,12,-1,-1,-1,14,16,18,-1,20,22,-1,24,26,28,-1,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1],"split_conditions":[1E0,1.414835E8,1.73E2,1.2768678E4,-7.2914646E-3,-9.902917E-3,2.8688198E3,1.5046106E7,8.604814E-3,-1.2191389E-4,-3.5605866E-3,3.0765435E10,2.826087E-1,9.7E1,5.1853354E-3,5.070136E7,7.408377E-1,-2.830238E-3,5.4024815E1,2.8017812E3,1.09141096E8,2.9715186E-3,8.37E2,1.857E3,1.2386714E-4,2.4410721E-3,-7.4855174E-4,-1.5490388E-3,-4.9502426E-3,-1.1936995E-3,1.3557E4,1.480441E-4,3.1070376E-3,9.456945E-6,1.697276E-3],"split_indices":[28,57,0,64,0,0,4,57,0,0,0,43,69,10,0,57,69,0,70,59,7,0,0,2,0,0,0,0,0,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.4E1,9E0,9.1E1,3E0,3E0,6E0,8.7E1,4E0,3E0,3E0,2.7E1,6E1,1.7E1,1E1,2.7E1,3.3E1,3E0,1.4E1,1.5E1,1.2E1,3E0,3E1,9E0,5E0,3E0,1.2E1,7E0,5E0,9E0,2.1E1,3E0,6E0,1.1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-6.9426643E-3,-4.7145477E-5,-6.847996E-3,-2.1842895E-2,5.9529476E-2,-7.4822223E-3,-1.3896343E-2,7.289799E-3,3.0906795E-2,3.5376889E-3,-5.169536E-2,4.3982E-3,1.2535507E-2,-2.6034515E-2,3.51694E-2,-6.003417E-2,-2.733977E-4,-2.208749E-3,2.5994966E-2,-5.6588784E-2,-6.974963E-3,7.795734E-2,9.444469E-3,-6.916188E-4,-6.944767E-2,1.7397748E-3,-9.691736E-5,-4.4966792E-4,-3.134077E-3,1.8518975E-3,-1.9007627E-2,4.9702595E-3,1.466933E-3,1.1540188E-3,-6.69446E-4,-9.076275E-4,-3.7932694E-3,-3.045765E-3,-7.272191E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,-1,5,7,-1,9,-1,11,13,15,-1,17,19,21,23,-1,-1,25,27,29,31,33,-1,35,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2028195E-1,1.454357E-1,0E0,9.6300796E-2,8.866859E-2,0E0,5.2243035E-2,0E0,3.132177E-2,5.2363686E-2,9.076431E-3,0E0,1.9335184E-2,1.6493468E-2,2.898309E-2,7.5879395E-3,0E0,0E0,7.3630996E-3,7.376384E-3,1.221773E-2,1.2437306E-2,8.179548E-3,0E0,1.3318643E-2,0E0,0E0,0E0,0E0,0E0,1.4168579E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,9,9,10,10,12,12,13,13,14,14,15,15,18,18,19,19,20,20,21,21,22,22,24,24,30,30],"right_children":[2,4,-1,6,8,-1,10,-1,12,14,16,-1,18,20,22,24,-1,-1,26,28,30,32,34,-1,36,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.1855E4,-6.847996E-3,3.218612E7,1.6451234E-2,-7.4822223E-3,4.307821E7,7.289799E-3,2.70482E5,1.632896E3,1.6226676E0,4.3982E-3,1.8867925E-1,1.92E2,1.19461E5,2.3737179E1,-2.733977E-4,-2.208749E-3,2.2059325E5,7.6944447E0,1.2352941E0,2.7943066E3,4.2202312E1,-6.916188E-4,2.1129233E-1,1.7397748E-3,-9.691736E-5,-4.4966792E-4,-3.134077E-3,1.8518975E-3,4.46438E0,4.9702595E-3,1.466933E-3,1.1540188E-3,-6.69446E-4,-9.076275E-4,-3.7932694E-3,-3.045765E-3,-7.272191E-5],"split_indices":[5,2,0,5,69,0,57,0,41,4,69,0,69,0,41,68,0,0,40,65,68,4,68,0,39,0,0,0,0,0,68,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,1.1E2,4E0,8.1E1,2.9E1,3E0,7.8E1,5E0,2.4E1,5.4E1,2.4E1,4E0,2E1,2.8E1,2.6E1,2E1,4E0,3E0,1.7E1,1E1,1.8E1,9E0,1.7E1,4E0,1.6E1,1.1E1,6E0,3E0,7E0,3E0,1.5E1,4E0,5E0,1E1,7E0,5E0,1.1E1,3E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.14333E-3,3.1305335E-3,-6.6115023E-3,-6.4926157E-3,1.1821701E-1,-5.613723E-2,9.025205E-3,7.0631984E-3,1.3774673E-3,3.578613E-4,-7.4494205E-2,-1.2996169E-2,3.9990213E-2,-1.3751714E-3,-8.844305E-2,-2.2081165E-2,1.3229571E-3,6.83289E-2,6.983147E-4,-4.7437614E-3,-1.9471889E-3,-3.3023983E-2,-1.7448238E-3,2.0871742E-2,5.3978297E-3,1.4251655E-3,-2.7665313E-2,-6.0182484E-4,-2.3444209E-3,1.2396682E-3,-7.2528375E-4,-8.977537E-4,2.0571076E-3,-2.5210623E-3,1.0602444E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,15,17,-1,19,21,-1,23,25,-1,-1,27,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1645672E-1,1.1483945E-1,0E0,7.4462764E-2,2.612798E-2,2.7819358E-2,5.097973E-2,0E0,0E0,0E0,7.888123E-3,1.8122407E-2,3.4470186E-2,0E0,7.1256757E-3,8.176243E-3,0E0,4.5835614E-2,1.3745658E-2,0E0,0E0,8.990951E-3,6.9035594E-3,1.32663185E-2,0E0,0E0,7.5497776E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,22,22,23,23,26,26],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,16,18,-1,20,22,-1,24,26,-1,-1,28,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.1811385E4,-6.6115023E-3,1.9121604E-4,7.1154E4,1.56E2,3.290356E5,7.0631984E-3,1.3774673E-3,3.578613E-4,4.04E3,1.0098112E2,9.580646E0,-1.3751714E-3,3.159013E3,4.0291533E8,1.3229571E-3,1.087013E1,8.5154843E-1,-4.7437614E-3,-1.9471889E-3,1.072435E5,5.49E3,1.4960615E3,5.3978297E-3,1.4251655E-3,2.92E2,-6.0182484E-4,-2.3444209E-3,1.2396682E-3,-7.2528375E-4,-8.977537E-4,2.0571076E-3,-2.5210623E-3,1.0602444E-4],"split_indices":[5,64,0,50,2,0,40,0,0,0,2,68,65,0,4,7,0,66,39,0,0,45,10,4,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,1.02E2,4E0,9.5E1,7E0,2.2E1,7.3E1,4E0,3E0,5E0,1.7E1,4.3E1,3E1,5E0,1.2E1,3.6E1,7E0,1.7E1,1.3E1,7E0,5E0,2.3E1,1.3E1,1E1,7E0,6E0,7E0,1.3E1,1E1,4E0,9E0,4E0,6E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.4420389E-3,-7.042745E-3,1.2453013E-2,-1.7181087E-2,4.988612E-2,2.3596447E-3,-8.199733E-2,8.178715E-2,1.720313E-2,-3.0731563E-2,1.8826595E-2,-5.8230024E-4,-4.789384E-3,5.323021E-2,7.262413E-3,-3.6913253E-4,3.4370143E-2,1.4406352E-4,-2.0763841E-3,5.3907577E-2,-1.1171073E-2,8.20925E-4,4.041414E-3,2.1831803E-3,-2.1199432E-4,4.7142343E-3,9.008242E-4,-1.805115E-3,4.0343456E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,-1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7443226E-1,0E0,1.1067744E-1,7.112066E-2,4.446833E-2,2.4554664E-2,2.4285935E-2,4.5355722E-2,1.0462692E-2,9.036195E-3,3.217263E-2,0E0,0E0,2.3093231E-2,0E0,0E0,9.159217E-3,0E0,0E0,2.3299776E-2,1.107914E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,16,16,19,19,20,20],"right_children":[2,-1,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7571955E6,-7.042745E-3,2.108166E3,4.95E9,9.275501E-1,1.1301075E1,2.7782312E1,8.974092E3,5.12544E-1,1.093E4,9.3152986E8,-5.8230024E-4,-4.789384E-3,8.413108E-2,7.262413E-3,-3.6913253E-4,8.106617E-1,1.4406352E-4,-2.0763841E-3,5.83E2,2.3136793E1,8.20925E-4,4.041414E-3,2.1831803E-3,-2.1199432E-4,4.7142343E-3,9.008242E-4,-1.805115E-3,4.0343456E-4],"split_indices":[57,0,64,5,69,68,68,4,51,9,5,0,0,50,0,0,39,0,0,10,70,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,5E0,9.8E1,5.5E1,4.3E1,4.3E1,1.2E1,2.1E1,2.2E1,1.4E1,2.9E1,4E0,8E0,1.7E1,4E0,9E0,1.3E1,5E0,9E0,1.3E1,1.6E1,1E1,7E0,9E0,4E0,4E0,9E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.3970828E-3,-7.636826E-3,1.2250999E-1,-5.3837914E-2,4.013931E-3,7.6245815E-3,1.6303876E-3,-2.8783723E-4,-7.509584E-2,-5.3649355E-2,1.0641356E-2,-3.911505E-3,-7.569678E-4,-2.949914E-4,-3.4695335E-3,8.079652E-2,2.6774278E-3,1.1938084E-3,4.909933E-3,-1.4834898E-2,2.8169388E-2,6.387282E-4,-2.5945876E-2,4.563438E-3,4.6727867E-3,-4.5117043E-2,-1.5314241E-5,1.7757242E-3,-5.4626825E-4,-2.599258E-3,5.808946E-6],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,15,-1,-1,-1,-1,17,19,-1,-1,21,23,-1,25,-1,27,29,-1,-1,-1,-1,-1],"loss_changes":[1.1034788E-1,5.08142E-2,2.5429495E-2,1.8551324E-2,2.9614996E-2,0E0,0E0,0E0,1.0020547E-2,9.459179E-3,3.8334716E-2,0E0,0E0,0E0,0E0,9.109005E-3,2.8542573E-2,0E0,0E0,1.2894667E-2,4.687807E-2,0E0,1.3721118E-2,0E0,1.37880035E-2,1.0937329E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,10,10,15,15,16,16,19,19,20,20,22,22,24,24,25,25],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,16,-1,-1,-1,-1,18,20,-1,-1,22,24,-1,26,-1,28,30,-1,-1,-1,-1,-1],"split_conditions":[1.0825377E4,4.1964554E-4,1.767918E6,3.09E2,1.00441E5,7.6245815E-3,1.6303876E-3,-2.8783723E-4,2.022832E1,1.23E3,1.82E2,-3.911505E-3,-7.569678E-4,-2.949914E-4,-3.4695335E-3,2.84E2,6.5857696E8,1.1938084E-3,4.909933E-3,3.568E3,1.5046106E7,6.387282E-4,1.0754172E6,4.563438E-3,2.0086452E5,5.127381E1,-1.5314241E-5,1.7757242E-3,-5.4626825E-4,-2.599258E-3,5.808946E-6],"split_indices":[64,54,41,0,1,0,0,0,66,2,10,0,0,0,0,11,12,0,0,2,57,0,59,0,40,68,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9.3E1,6E0,1.8E1,7.5E1,3E0,3E0,6E0,1.2E1,7E0,6.8E1,9E0,3E0,3E0,4E0,6E0,6.2E1,3E0,3E0,3.7E1,2.5E1,1E1,2.7E1,5E0,2E1,1.5E1,1.2E1,6E0,1.4E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.96773E-3,7.5612008E-3,-1.23072654E-1,1.5706519E-2,-7.139215E-3,-1.7590164E-1,6.3122957E-6,2.92491E-3,1.21894926E-1,-3.1556776E-3,-1.0132743E-2,4.8421837E-2,-9.056659E-3,7.0949765E-3,1.197041E-3,-7.330372E-4,7.969407E-2,-4.50986E-2,1.0992825E-2,9.436842E-4,4.258517E-3,-2.7258256E-3,-7.394571E-4,-4.8836255E-3,4.4342656E-2,8.747722E-3,-1.3077775E-3,3.4263688E-3,5.407971E-4,9.94713E-4,-1.1251856E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,-1,15,17,-1,-1,-1,19,21,23,-1,-1,-1,-1,25,27,29,-1,-1,-1,-1,-1],"loss_changes":[1.5302381E-1,1.251516E-1,6.512101E-2,1.1365318E-1,0E0,2.517128E-2,0E0,4.186801E-2,3.142771E-2,0E0,0E0,3.407791E-2,4.457572E-2,0E0,0E0,0E0,9.902053E-3,1.002207E-2,2.1395631E-2,0E0,0E0,0E0,0E0,9.9639E-3,1.3493726E-2,1.0055887E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,11,11,12,12,16,16,17,17,18,18,23,23,24,24,25,25],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,-1,16,18,-1,-1,-1,20,22,24,-1,-1,-1,-1,26,28,30,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.414835E8,2.667836E6,1.1681136E4,-7.139215E-3,1.66E2,6.3122957E-6,7.7E2,6.5566176E7,-3.1556776E-3,-1.0132743E-2,8.031746E0,7.650722E4,7.0949765E-3,1.197041E-3,-7.330372E-4,4E1,2.4636926E3,7.3845965E8,9.436842E-4,4.258517E-3,-2.7258256E-3,-7.394571E-4,9E0,5.22136E5,4.7E1,-1.3077775E-3,3.4263688E-3,5.407971E-4,9.94713E-4,-1.1251856E-3],"split_indices":[28,57,44,4,0,0,0,11,57,0,0,65,45,0,0,0,10,4,7,0,0,0,0,29,41,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,8.6E1,9E0,8.3E1,3E0,6E0,3E0,7.5E1,8E0,3E0,3E0,1.5E1,6E1,5E0,3E0,5E0,1E1,2.1E1,3.9E1,3E0,7E0,1.2E1,9E0,2.7E1,1.2E1,1.8E1,9E0,5E0,7E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.309533E-3,-1.4890319E-1,6.0440083E-3,-8.777373E-4,-1.0371096E-2,1.3764441E-2,-6.969801E-3,-4.8170504E-3,6.4871006E-2,2.2111403E-2,-2.7760953E-2,1.145304E-1,4.2329563E-4,-1.3681561E-2,6.39949E-2,-2.693041E-3,-1.5191546E-2,7.769688E-3,3.1389226E-3,1.4331989E-3,-1.1266551E-3,9.805498E-4,4.40448E-3,-9.152746E-4,1.0650386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,15,17,-1,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5577815E-1,7.7324405E-2,1.173149E-1,0E0,0E0,8.198121E-2,0E0,4.0155835E-2,6.1770685E-2,4.5849588E-2,1.4284244E-2,1.975827E-2,0E0,1.0763156E-2,1.966403E-2,0E0,6.9926884E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,16,18,-1,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7888E4,1.21E2,1.5081978E8,-8.777373E-4,-1.0371096E-2,1.3137E4,-6.969801E-3,2.8399336E7,1.3343991E1,1.6511278E1,2.1750195E3,1.8484E4,4.2329563E-4,3.2785356E7,3.077573E5,-2.693041E-3,9.235709E-1,7.769688E-3,3.1389226E-3,1.4331989E-3,-1.1266551E-3,9.805498E-4,4.40448E-3,-9.152746E-4,1.0650386E-3],"split_indices":[1,0,57,0,0,2,0,57,69,68,4,2,0,12,40,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6E0,8.8E1,3E0,3E0,8.5E1,3E0,6.3E1,2.2E1,2.9E1,3.4E1,1.1E1,1.1E1,1.6E1,1.3E1,8E0,2.6E1,3E0,8E0,3E0,1.3E1,7E0,6E0,2.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.1595037E-2,1.7987208E-3,1.169777E-1,-1.3664011E-2,3.1306922E-2,6.9966456E-3,8.6693297E-4,-2.3341332E-2,2.1214613E-3,1.1902718E-2,5.817702E-2,-1.0640312E-2,-5.5222675E-2,2.3036774E-3,-3.5570053E-4,3.475511E-3,8.091471E-5,-8.1480615E-4,1.8093651E-3,-9.2862725E-2,-1.0080835E-3,-5.4043015E-3,-1.6109769E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,-1,-1,-1,-1,-1,-1,21,-1,-1,-1],"loss_changes":[1.0921994E-1,4.5142964E-2,3.607516E-2,4.0654894E-2,1.7034888E-2,0E0,0E0,2.2493174E-2,0E0,1.7977994E-2,1.7453264E-2,1.8967642E-2,1.7025031E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.897716E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,19,19],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,-1,-1,-1,-1,-1,-1,22,-1,-1,-1],"split_conditions":[1.0825377E4,1.1058695E0,8.50999E9,1E0,3.1346828E5,6.9966456E-3,8.6693297E-4,5.895365E5,2.1214613E-3,1.2606002E-1,2.5588E4,4.524696E5,8.670369E-1,2.3036774E-3,-3.5570053E-4,3.475511E-3,8.091471E-5,-8.1480615E-4,1.8093651E-3,6.7281544E-5,-1.0080835E-3,-5.4043015E-3,-1.6109769E-3],"split_indices":[64,53,5,22,40,0,0,40,0,39,9,40,39,0,0,0,0,0,0,50,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.7E1,8E0,6.4E1,3.3E1,5E0,3E0,5.6E1,8E0,2E1,1.3E1,4.1E1,1.5E1,6E0,1.4E1,9E0,4E0,3.6E1,5E0,6E0,9E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-5.576078E-3,3.6720806E-3,-1.0304895E-1,9.7370185E-3,-5.214178E-3,-8.68433E-3,-1.7714642E-4,-5.1603974E-3,6.1130766E-2,1.4246393E-2,-4.8425194E-2,6.836042E-3,3.8628835E-2,-1.32040195E-2,4.0258985E-2,-1.0665583E-3,-3.5195956E-3,5.7726443E-2,1.1943027E-4,2.7024143E-3,-2.4804782E-2,4.8427284E-3,2.7645424E-2,1.3015753E-3,3.6638023E-3,-2.3755315E-3,-9.86358E-4,4.776521E-2,8.284152E-6,1.1378502E-3,-7.465793E-4,7.788966E-4,3.0872507E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,17,19,21,-1,-1,23,-1,-1,25,-1,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1],"loss_changes":[1.0404453E-1,8.118883E-2,9.626418E-2,7.841013E-2,0E0,0E0,0E0,6.786035E-2,4.4202104E-2,4.051874E-2,1.7616399E-2,0E0,1.3537947E-2,2.6176158E-2,2.3207769E-2,0E0,0E0,7.172253E-3,0E0,0E0,1.794333E-2,0E0,1.4352167E-2,0E0,0E0,0E0,7.145042E-3,9.298656E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,20,20,22,22,26,26,27,27],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,18,20,22,-1,-1,24,-1,-1,26,-1,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.4974915E8,2.667836E6,6.627665E3,-5.214178E-3,-8.68433E-3,-1.7714642E-4,2.9929216E9,1.6451234E-2,1.1674918E3,7.705493E5,6.836042E-3,8.348932E9,2.8017812E3,2.0477528E1,-1.0665583E-3,-3.5195956E-3,5.5978696E9,1.1943027E-4,2.7024143E-3,2.8900872E6,4.8427284E-3,1.4163358E7,1.3015753E-3,3.6638023E-3,-2.3755315E-3,1.368163E3,7.359468E8,8.284152E-6,1.1378502E-3,-7.465793E-4,7.788966E-4,3.0872507E-3],"split_indices":[28,57,44,4,0,0,0,5,69,64,40,0,12,59,70,0,0,12,0,0,62,0,57,0,0,0,4,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,1.05E2,9E0,1.01E2,4E0,4E0,5E0,7.9E1,2.2E1,5.5E1,2.4E1,3E0,1.9E1,2.7E1,2.8E1,1.5E1,9E0,1.2E1,7E0,3E0,2.4E1,3E0,2.5E1,7E0,5E0,1E1,1.4E1,1.4E1,1.1E1,5E0,9E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-7.8038108E-3,-8.016338E-3,-5.3728244E-4,8.954204E-3,-1.0962222E-1,-3.0928101E-2,2.3485137E-2,-7.996317E-3,-1.1744822E-3,1.7285965E-2,-5.0638348E-2,7.380443E-2,1.13844955E-2,2.4151227E-3,-7.381676E-4,-1.3165803E-3,-4.183081E-3,7.3794923E-3,2.7567027E-2,-1.5233261E-2,2.6595283E-2,-6.835215E-4,2.5529729E-3,-3.2464527E-2,1.1348929E-2,3.2836013E-3,1.6162584E-2,-5.785765E-4,-3.1082386E-3,1.6648101E-3,-7.8296394E-4,-6.297484E-5,3.0760141E-2,3.385341E-4,3.6345504E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,17,19,-1,-1,-1,-1,-1,21,23,25,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,33,-1,-1],"loss_changes":[1.3175395E-1,0E0,1.0348402E-1,5.4042842E-2,4.4813856E-2,2.4633111E-2,4.0564682E-2,0E0,0E0,1.1591663E-2,1.3815638E-2,5.367846E-2,2.3078248E-2,0E0,0E0,0E0,0E0,0E0,1.4639331E-2,9.987378E-3,1.7350297E-2,0E0,0E0,8.832629E-3,7.922153E-3,0E0,8.018986E-3,0E0,0E0,0E0,0E0,0E0,2.0158157E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,18,18,19,19,20,20,23,23,24,24,26,26,32,32],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,18,20,-1,-1,-1,-1,-1,22,24,26,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.227133E6,-8.016338E-3,1.2511909E8,1E0,2.4719655E5,9.311144E8,7.573E3,-7.996317E-3,-1.1744822E-3,4E1,1.0979587E8,3.42E2,1.6492815E3,2.4151227E-3,-7.381676E-4,-1.3165803E-3,-4.183081E-3,7.3794923E-3,3.598611E6,1.5363757E8,2.7943066E3,-6.835215E-4,2.5529729E-3,1E0,4.6087875E8,3.2836013E-3,2.2622842E6,-5.785765E-4,-3.1082386E-3,1.6648101E-3,-7.8296394E-4,-6.297484E-5,4.901466E9,3.385341E-4,3.6345504E-3],"split_indices":[57,0,57,24,44,5,9,0,0,3,7,10,64,0,0,0,0,0,1,12,4,0,0,26,12,0,44,0,0,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,3E0,9.8E1,9.1E1,7E0,2.4E1,6.7E1,3E0,4E0,7E0,1.7E1,1.2E1,5.5E1,3E0,4E0,1.3E1,4E0,3E0,9E0,2E1,3.5E1,4E0,5E0,1.2E1,8E0,5E0,3E1,9E0,3E0,4E0,4E0,1.4E1,1.6E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.0557799E-2,-7.756798E-3,-3.3254274E-3,3.0223036E-3,-6.4101494E-3,-1.4390782E-2,4.7778245E-2,-6.2090706E-2,-1.7576391E-3,6.7133736E-3,2.6963908E-2,-1.0228007E-3,-3.7956678E-3,3.3249313E-3,-7.902005E-3,1.4387495E-2,3.5875053E-3,-5.380215E-2,3.2147442E-3,2.2065553E-3,8.680693E-5,-4.2648287E-4,-3.515514E-3,3.1426821E-3,-1.7287619E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,15,-1,-1,-1,17,19,-1,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.19276695E-1,0E0,8.798916E-2,7.2386764E-2,0E0,4.026507E-2,5.4111473E-2,1.1805423E-2,2.6747525E-2,0E0,1.5043559E-2,0E0,0E0,0E0,2.619624E-2,8.988477E-3,0E0,1.1333525E-2,2.1391133E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,10,10,14,14,15,15,17,17,18,18],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,16,-1,-1,-1,18,20,-1,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.227133E6,-7.756798E-3,1.4974915E8,8.8392685E1,-6.4101494E-3,6.178556E-4,1.0822511E-2,5.614759E5,7.690597E-4,6.7133736E-3,8.3899E4,-1.0228007E-3,-3.7956678E-3,3.3249313E-3,7.196866E2,7.84E3,3.5875053E-3,3.9E1,2.4E2,2.2065553E-3,8.680693E-5,-4.2648287E-4,-3.515514E-3,3.1426821E-3,-1.7287619E-4],"split_indices":[57,0,57,68,0,53,69,40,51,0,2,0,0,0,4,9,0,10,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,3E0,9.4E1,9.1E1,3E0,6.6E1,2.5E1,1.3E1,5.3E1,3E0,2.2E1,6E0,7E0,3E0,5E1,1.9E1,3E0,9E0,4.1E1,4E0,1.5E1,4E0,5E0,3E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[9.7624597E-4,-7.505662E-3,7.6916977E-3,-6.265249E-3,5.4475907E-2,3.628801E-3,-1.672432E-3,6.2040538E-3,3.4883387E-2,-2.1541739E-2,1.5837E-2,-1.741012E-3,4.9659673E-2,-1.3959821E-3,8.406696E-4,-9.418559E-3,3.4397744E-2,2.5700177E-3,9.200527E-6,-1.129456E-3,7.2854775E-4,5.0935548E-2,-3.0701442E-4,3.36255E-3,1.152551E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,-1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,-1,19,21,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[1.2697339E-1,0E0,6.8249635E-2,2.6168112E-2,3.694994E-2,1.9700235E-2,0E0,0E0,2.5341915E-2,9.929036E-3,2.0480745E-2,0E0,8.776523E-3,0E0,0E0,8.807809E-3,1.7524263E-2,0E0,0E0,0E0,0E0,9.681102E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,8,8,9,9,10,10,12,12,15,15,16,16,21,21],"right_children":[2,-1,4,6,8,10,-1,-1,12,14,16,-1,18,-1,-1,20,22,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.227133E6,-7.505662E-3,1.3137E4,5.9911316E7,1.0625348E8,1E0,-1.672432E-3,6.2040538E-3,4.798087E6,5E1,2.367141E1,-1.741012E-3,9.365851E-1,-1.3959821E-3,8.406696E-4,1.0033929E1,9.641E3,2.5700177E-3,9.200527E-6,-1.129456E-3,7.2854775E-4,1.3047E4,-3.0701442E-4,3.36255E-3,1.152551E-3],"split_indices":[57,0,2,57,7,24,0,0,1,3,68,0,39,0,0,65,2,0,0,0,0,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,3E0,1.03E2,8E1,2.3E1,6.2E1,1.8E1,3E0,2E1,2E1,4.2E1,3E0,1.7E1,1.6E1,4E0,1.8E1,2.4E1,1.4E1,3E0,1.1E1,7E0,1.7E1,7E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.9749636E-3,-7.0508695E-3,1.4091572E-3,2.3963526E-2,-2.9045787E-2,-1.2418207E-3,8.0044396E-2,-1.701378E-2,-6.1344123E-3,-2.9240232E-3,9.874929E-3,1.1565964E-1,9.094662E-4,-4.0250484E-2,-1.2750257E-4,1.3611903E-3,3.3517866E-3,6.6036787E-3,2.5498527E-3,3.4705468E-4,-5.8541372E-2,-8.6823956E-4,1.8242758E-2,-3.971727E-3,-6.613526E-4,2.4342758E-3,-3.759601E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,-1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,21,-1,-1,-1,-1,23,-1,25,-1,-1,-1,-1],"loss_changes":[1.043616E-1,0E0,6.9385245E-2,8.2084596E-2,5.607611E-2,3.0975752E-2,3.537161E-2,1.2949595E-2,0E0,0E0,2.066533E-2,1.4193401E-2,0E0,1.3719665E-2,0E0,1.2329938E-2,0E0,0E0,0E0,0E0,1.4500368E-2,0E0,2.0025948E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,13,13,15,15,20,20,22,22],"right_children":[2,-1,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,22,-1,-1,-1,-1,24,-1,26,-1,-1,-1,-1],"split_conditions":[3.3171368E7,-7.0508695E-3,1.0156463E1,4.3972437E3,9.384525E7,1.3954372E1,3.0376984E1,2.589813E-3,-6.1344123E-3,-2.9240232E-3,2.7686512E0,1.2236808E7,9.094662E-4,1.5151516E-2,-1.2750257E-4,1.118632E3,3.3517866E-3,6.6036787E-3,2.5498527E-3,3.4705468E-4,3.6E1,-8.6823956E-4,9.0533855E10,-3.971727E-3,-6.613526E-4,2.4342758E-3,-3.759601E-4],"split_indices":[5,0,65,64,57,70,69,50,0,0,51,44,0,69,0,64,0,0,0,0,3,0,43,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,3E0,9.9E1,5.7E1,4.2E1,4E1,1.7E1,3.9E1,3E0,5E0,3.5E1,1E1,7E0,1.4E1,2.5E1,3.2E1,3E0,5E0,5E0,4E0,1E1,1.4E1,1.8E1,5E0,5E0,7E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.8832774E-3,-6.4565483E-3,8.775673E-3,2.2068605E-2,-4.1169606E-2,1.3763825E-2,9.716734E-2,-1.6816061E-2,-4.45971E-3,-2.2430762E-3,2.0232098E-2,5.9791226E-3,9.1520377E-4,-2.6005965E-3,8.4641064E-4,3.432968E-2,-1.14458995E-2,-1.2507061E-3,5.054048E-2,-1.4659684E-3,6.110371E-4,1.427778E-3,-1.7142321E-3,2.1617712E-2,6.8259396E-2,2.0792454E-3,-5.098787E-4,3.774693E-3,8.6960057E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,-1,3,5,7,9,11,13,-1,-1,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[9.653189E-2,0E0,6.929223E-2,4.9804244E-2,3.1556044E-2,3.293581E-2,2.2187047E-2,2.8071048E-2,0E0,0E0,3.1207172E-2,0E0,0E0,0E0,0E0,2.7748965E-2,1.3160574E-2,2.2435686E-2,1.5436232E-2,0E0,0E0,0E0,0E0,1.2993695E-2,1.669722E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,10,10,15,15,16,16,17,17,18,18,23,23,24,24],"right_children":[2,-1,4,6,8,10,12,14,-1,-1,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[1.172E3,-6.4565483E-3,7.145873E7,1.1335518E4,1.1221429E1,6.205776E0,2.799219E7,1.105405E6,-4.45971E-3,-2.2430762E-3,4.3E1,5.9791226E-3,9.1520377E-4,-2.6005965E-3,8.4641064E-4,2.2E1,1.5363757E8,6.2186885E-1,3.566468E5,-1.4659684E-3,6.110371E-4,1.427778E-3,-1.7142321E-3,1.4664377E-1,1.4282178E0,2.0792454E-3,-5.098787E-4,3.774693E-3,8.6960057E-4],"split_indices":[2,0,57,4,65,70,1,41,0,0,3,0,0,0,0,3,12,39,40,0,0,0,0,50,69,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,3E0,1.02E2,8.1E1,2.1E1,7.4E1,7E0,1.6E1,5E0,6E0,6.8E1,4E0,3E0,7E0,9E0,4.7E1,2.1E1,1.5E1,3.2E1,1.1E1,1E1,8E0,7E0,1.3E1,1.9E1,7E0,6E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.179549E-3,8.89181E-3,-7.760346E-2,1.9167752E-3,8.075419E-2,-7.2815185E-3,-6.2486547E-4,-2.7838908E-2,8.847756E-3,5.5844067E-3,-5.330542E-4,-3.6355811E-3,-3.886897E-3,3.1965033E-3,5.2038874E-3,1.4153257E-3,-7.144083E-4,-1.6144066E-3,1.16779525E-2,3.8591542E-3,5.116034E-3,3.5906453E-2,-9.0358587E-4,1.6725119E-4,2.899343E-3,-9.614777E-3,1.4003583E-3,-1.9193036E-3,-1.5990401E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,-1,17,-1,-1,-1,19,-1,21,23,25,-1,-1,27,-1,-1,-1],"loss_changes":[6.813309E-2,5.097195E-2,5.5051792E-2,1.9632502E-2,4.2328194E-2,0E0,0E0,2.677136E-2,1.7908238E-2,0E0,0E0,7.1273847E-3,0E0,0E0,2.0846443E-2,0E0,0E0,0E0,3.2692064E-2,0E0,1.1338588E-2,9.752101E-3,1.5415701E-2,0E0,0E0,8.485663E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,14,14,18,18,20,20,21,21,22,22,25,25],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,-1,18,-1,-1,-1,20,-1,22,24,26,-1,-1,28,-1,-1,-1],"split_conditions":[1.1384717E8,1.3127055E4,2.4719655E5,5.981675E-4,4.9874042E2,-7.2815185E-3,-6.2486547E-4,1.358194E8,7.690597E-4,5.5844067E-3,-5.330542E-4,1.13474E5,-3.886897E-3,3.1965033E-3,7.449623E2,1.4153257E-3,-7.144083E-4,-1.6144066E-3,1.7533583E1,3.8591542E-3,1.6449378E-1,1.1058695E0,2.463E4,1.6725119E-4,2.899343E-3,7.84E3,1.4003583E-3,-1.9193036E-3,-1.5990401E-4],"split_indices":[57,4,44,54,68,0,0,7,51,0,0,41,0,0,4,0,0,0,70,0,39,53,2,0,0,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.01E2,9E0,9.3E1,8E0,3E0,6E0,1.7E1,7.6E1,5E0,3E0,1.3E1,4E0,3E0,7.3E1,3E0,1E1,9E0,6.4E1,4E0,6E1,9E0,5.1E1,5E0,4E0,4.1E1,1E1,5E0,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[8.5151853E-4,1.0325887E-2,-1.0661927E-1,1.5851332E-2,-5.569164E-3,-8.289388E-3,4.748277E-6,7.6359063E-3,9.715716E-2,-2.4693047E-3,1.2885643E-2,6.389211E-3,1.1601944E-3,7.2005587E-3,3.246479E-3,1.8201513E-3,2.6886053E-3,5.7200156E-3,-2.8872075E-3,-1.8318867E-2,1.9482037E-2,-2.1894544E-4,-2.311681E-3,3.3231785E-3,4.4765646E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,13,-1,-1,15,-1,17,-1,19,-1,21,23,-1,-1,-1,-1],"loss_changes":[1.131049E-1,8.057427E-2,8.205843E-2,6.538636E-2,0E0,0E0,0E0,3.1711716E-2,2.8186113E-2,0E0,2.957324E-2,0E0,0E0,2.3299068E-2,0E0,1.9941537E-2,0E0,2.351454E-2,0E0,1.1862874E-2,2.2778964E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,10,10,13,13,15,15,17,17,19,19,20,20],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,14,-1,-1,16,-1,18,-1,20,-1,22,24,-1,-1,-1,-1],"split_conditions":[1E0,4.0530096E10,2.667836E6,1.2768678E4,-5.569164E-3,-8.289388E-3,4.748277E-6,2.6151154E-5,9.9182825E0,-2.4693047E-3,1.3285092E6,6.389211E-3,1.1601944E-3,2.4241872E0,3.246479E-3,9.2134416E-1,2.6886053E-3,1.1674918E3,-2.8872075E-3,9.489487E2,7.96735E5,-2.1894544E-4,-2.311681E-3,3.3231785E-3,4.4765646E-4],"split_indices":[28,5,44,64,0,0,0,51,66,0,40,0,0,51,0,39,0,64,0,64,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,1.01E2,8E0,9.8E1,3E0,4E0,4E0,9E1,8E0,6E0,8.4E1,4E0,4E0,7.8E1,6E0,7.2E1,6E0,6.9E1,3E0,2.5E1,4.4E1,1.9E1,6E0,5E0,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.122662E-3,-5.6137014E-3,6.863309E-3,1.3041743E-2,-6.045542E-3,3.48551E-3,1.0290778E-1,-5.4410018E-2,1.5498668E-2,5.677489E-3,1.4790739E-3,-4.681723E-4,-2.9900908E-3,3.2605365E-2,9.981755E-4,8.928774E-3,7.329534E-2,2.1983278E-3,-7.589147E-3,1.6971489E-3,-4.853322E-4,1.3327006E-3,4.2817513E-3,-1.9836996E-3,3.6428936E-3,1.5279229E-3,-2.8601114E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,15,17,19,21,-1,23,-1,-1,-1,-1,-1,25,-1,-1],"loss_changes":[1.0590629E-1,0E0,8.987021E-2,8.03667E-2,0E0,6.065698E-2,1.3170317E-2,8.956626E-3,1.7827453E-2,0E0,0E0,0E0,0E0,3.1137753E-2,1.753582E-2,1.3926476E-2,1.0118894E-2,0E0,1.5248914E-2,0E0,0E0,0E0,0E0,0E0,9.363559E-3,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15,16,16,18,18,24,24],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,16,18,20,22,-1,24,-1,-1,-1,-1,-1,26,-1,-1],"split_conditions":[1.458E3,-5.6137014E-3,1.5081978E8,1.1811385E4,-6.045542E-3,5.981675E-4,9.071151E-1,4.352552E9,1.2320755E1,5.677489E-3,1.4790739E-3,-4.681723E-4,-2.9900908E-3,1.0912028E1,7.009612E5,2.8522336E-1,2.5332516E5,2.1983278E-3,2.91E2,1.6971489E-3,-4.853322E-4,1.3327006E-3,4.2817513E-3,-1.9836996E-3,2.322375E-1,1.5279229E-3,-2.8601114E-4],"split_indices":[2,0,57,64,0,54,39,5,66,0,0,0,0,66,60,69,40,0,0,0,0,0,0,0,39,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,5E0,9.6E1,9.3E1,3E0,8.5E1,8E0,1.4E1,7.1E1,5E0,3E0,4E0,1E1,3.2E1,3.9E1,2.1E1,1.1E1,5E0,3.4E1,8E0,1.3E1,5E0,6E0,7E0,2.7E1,6E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-9.636127E-4,4.270906E-3,-5.8498103E-3,-1.5678563E-3,7.380219E-2,5.767509E-3,-3.159894E-2,5.943625E-3,3.4256963E-4,4.4862044E-4,5.2251704E-2,-5.8579065E-2,1.5808755E-3,-5.6776335E-3,9.437667E-4,3.1315818E-4,3.358465E-3,-3.500866E-2,-4.058E-3,-1.0397295E-2,1.5275448E-3,-5.084212E-5,-2.4764081E-3,7.66075E-4,-8.0208585E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,-1,19,-1,-1,-1,21,-1,23,-1,-1,-1,-1,-1],"loss_changes":[7.5233884E-2,4.19313E-2,0E0,2.132999E-2,3.2569185E-2,1.926161E-2,3.711639E-2,0E0,0E0,9.4383815E-3,8.632192E-3,8.398786E-3,0E0,1.1221015E-2,0E0,0E0,0E0,7.4962256E-3,0E0,1.2075126E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,13,13,17,17,19,19],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,-1,20,-1,-1,-1,22,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825377E4,-5.8498103E-3,7.925698E5,7.7854166E0,1.7618678E0,4.8E1,5.943625E-3,3.4256963E-4,1.2491071E2,4E0,1.4394E4,1.5808755E-3,7.434519E5,9.437667E-4,3.1315818E-4,3.358465E-3,1.2028319E6,-4.058E-3,2.5E1,1.5275448E-3,-5.084212E-5,-2.4764081E-3,7.66075E-4,-8.0208585E-4],"split_indices":[57,64,0,40,66,51,68,0,0,68,8,9,0,40,0,0,0,40,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,1.02E2,3E0,9.5E1,7E0,7.7E1,1.8E1,3E0,4E0,7E1,7E0,1.3E1,5E0,5.5E1,1.5E1,3E0,4E0,9E0,4E0,5E1,5E0,4E0,5E0,1.1E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.5364487E-3,2.2687318E-3,-4.9000112E-3,7.8045633E-3,-4.33938E-3,-5.663903E-3,3.4894574E-2,1.0026503E-2,-3.561992E-2,4.8539997E-3,2.2087326E-2,3.71393E-3,2.0838734E-3,6.5132044E-4,-4.2889982E-2,4.5558423E-2,-1.158218E-4,-1.12850135E-2,2.0678725E-2,-3.02767E-3,-8.7911135E-4,6.113773E-2,-1.7523958E-4,1.2999981E-3,-8.0436765E-4,1.864978E-3,-1.3183822E-3,1.01564E-3,3.5269938E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,-1,-1,19,21,-1,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.42469E-2,6.679244E-2,0E0,4.06972E-2,0E0,3.56088E-2,3.4264393E-2,1.1892465E-2,1.0231998E-2,0E0,1.9308016E-2,1.1429337E-2,0E0,0E0,1.2687519E-2,1.3389386E-2,0E0,7.862301E-3,2.5546007E-2,0E0,0E0,8.1329085E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,14,14,15,15,17,17,18,18,21,21],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,-1,-1,20,22,-1,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0883114E0,2.8841052E10,-4.9000112E-3,4.818997E6,-4.33938E-3,4.307821E7,1.4440433E-2,2.4241872E0,2.3737179E1,4.8539997E-3,7.639958E-1,1.632896E3,2.0838734E-3,6.5132044E-4,2.873808E3,1.3649979E10,-1.158218E-4,7.286E3,8.983131E1,-3.02767E-3,-8.7911135E-4,4.297157E9,-1.7523958E-4,1.2999981E-3,-8.0436765E-4,1.864978E-3,-1.3183822E-3,1.01564E-3,3.5269938E-3],"split_indices":[50,5,0,1,0,57,69,51,68,0,39,4,0,0,4,5,0,9,70,0,0,12,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,1.15E2,4E0,1.1E2,5E0,7.4E1,3.6E1,4.9E1,2.5E1,4E0,3.2E1,4.3E1,6E0,3E0,2.2E1,1.6E1,1.6E1,2.3E1,2E1,9E0,1.3E1,1.2E1,4E0,3E0,2E1,1.4E1,6E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[4.926989E-3,1.0556868E-2,-6.523879E-3,3.1075887E-3,8.473891E-2,1.9022796E-2,-1.6335003E-2,5.1688305E-3,1.1134804E-3,3.7430145E-2,-3.656072E-3,-2.472838E-2,9.0302556E-4,-1.139413E-4,4.9227092E-2,1.282286E-3,-5.617718E-4,-7.4350233E-3,-1.9707822E-3,7.480428E-2,2.8893825E-2,-1.2574433E-3,7.29841E-4,4.237635E-3,1.3731922E-3,7.240361E-5,2.5796925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,-1,-1,19,-1,-1,21,-1,23,25,-1,-1,-1,-1,-1,-1],"loss_changes":[1.00872934E-1,6.052298E-2,0E0,3.156346E-2,1.7540902E-2,2.3514487E-2,1.4900764E-2,0E0,0E0,1.4783602E-2,8.50292E-3,1.3411395E-2,0E0,0E0,1.03202425E-2,0E0,0E0,1.2109972E-2,0E0,6.9212243E-3,1.2500395E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,14,14,17,17,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,-1,-1,20,-1,-1,22,-1,24,26,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0072324E0,1.1811385E4,-6.523879E-3,3.925844E7,3.0376984E1,1.2236808E7,3.8E1,5.1688305E-3,1.1134804E-3,8.150183E0,1.8018018E-2,9.067368E5,9.0302556E-4,-1.139413E-4,1.2320755E1,1.282286E-3,-5.617718E-4,9.301866E7,-1.9707822E-3,4.3E1,1.8585684E1,-1.2574433E-3,7.29841E-4,4.237635E-3,1.3731922E-3,7.240361E-5,2.5796925E-3],"split_indices":[50,64,0,57,69,44,8,0,0,65,69,44,0,0,66,0,0,7,0,3,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,1.09E2,3E0,1E2,9E0,5.5E1,4.5E1,5E0,4E0,3E1,2.5E1,3.7E1,8E0,7E0,2.3E1,5E0,2E1,2.1E1,1.6E1,9E0,1.4E1,1.1E1,1E1,5E0,4E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-azure","platform_databricks-aws","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.6600735E-2,-4.183997E-1,4.016022E-1,-5.4472244E-1,2.4702572E-3,-3.0444106E-1,5.4333526E-1,-2.095471E-1,-6.9834214E-1,1.3250048E-2,-7.2323172E-3,-6.7226007E-4,-1.9111963E-2,3.2672933E-1,8.139915E-1,-1.4351901E-2,3.8755343E-3,-2.7703207E-2,-1.1876688E-2,-2.0970795E-3,4.2583746E-1,3.311289E-2,1.6418E-2,2.9950497E-1,2.2299686E-2,5.3400216E-3,1.2468227E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,19,21,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,-1],"loss_changes":[1.7139414E1,2.7659492E0,5.1688957E0,1.9132452E0,1.0345602E0,5.687549E-1,2.1963263E0,8.766462E-1,4.3567657E-1,0E0,0E0,0E0,0E0,9.781935E-1,2.5014305E-1,0E0,0E0,0E0,0E0,0E0,2.9969501E-1,0E0,0E0,4.6219945E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,13,13,14,14,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,20,22,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,-1],"split_conditions":[3.3433047E-1,2.561146E3,2.80369E5,4.828443E-1,2.5927516E11,6.313997E-1,1.5364555E0,1.3564575E5,1.1653038E-2,1.3250048E-2,-7.2323172E-3,-6.7226007E-4,-1.9111963E-2,8.184232E5,2.326E4,-1.4351901E-2,3.8755343E-3,-2.7703207E-2,-1.1876688E-2,-2.0970795E-3,1.5791555E9,3.311289E-2,1.6418E-2,1.8589372E5,2.2299686E-2,5.3400216E-3,1.2468227E-2],"split_indices":[40,53,1,28,32,39,43,29,39,0,0,0,0,33,9,0,0,0,0,0,7,0,0,34,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.1E1,4.9E1,3.9E1,1.2E1,8E0,4.1E1,1.3E1,2.6E1,4E0,8E0,4E0,4E0,2.4E1,1.7E1,8E0,5E0,2.1E1,5E0,5E0,1.9E1,1.2E1,5E0,1.3E1,6E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[3.9494306E-2,-3.070603E-1,6.038862E-1,-4.283072E-1,2.940214E-1,8.1463903E-1,2.2642255E-1,-5.962956E-1,-1.721278E-1,1.8948603E-2,3.146083E-3,1.4225878E-2,9.0458536E-1,3.1656706E-1,-6.202453E-4,-6.8261397E-1,-4.4358503E-3,-1.1051375E-2,-5.5826362E-2,1.6590422E-2,3.583697E-2,4.103856E-3,1.5028562E-2,-7.388364E-3,-7.60215E-1,1.989614E-3,-7.0197335E-3,-1.7825114E-2,-3.1670492E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[1.9549099E1,4.644448E0,2.8385057E0,2.135189E0,4.7955102E-1,5.5152035E-1,3.3482993E-1,1.1829653E0,3.2571435E-1,0E0,0E0,0E0,2.883768E-1,1.7520332E-1,0E0,8.087826E-1,0E0,0E0,2.1061364E-1,0E0,0E0,0E0,0E0,0E0,2.5288773E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,18,18,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,3.5324219E3,1.4046429E1,1.1816701E-2,6.15786E5,2.1146134E5,1.2707124E1,2.2789746E3,1.2239595E3,1.8948603E-2,3.146083E-3,1.4225878E-2,3.8902016E8,7.765E3,-6.202453E-4,4.029937E-1,-4.4358503E-3,-1.1051375E-2,4.5843E7,1.6590422E-2,3.583697E-2,4.103856E-3,1.5028562E-2,-7.388364E-3,4.307821E7,1.989614E-3,-7.0197335E-3,-1.7825114E-2,-3.1670492E-2],"split_indices":[40,53,55,39,30,29,54,53,4,0,0,0,7,2,0,28,0,0,46,0,0,0,0,0,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.1E1,3.7E1,5.1E1,1E1,2.3E1,1.4E1,3E1,2.1E1,4E0,6E0,5E0,1.8E1,1E1,4E0,2.5E1,5E0,9E0,1.2E1,4E0,1.4E1,4E0,6E0,4E0,2.1E1,7E0,5E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.488806E-2,-3.7544534E-1,4.4622344E-1,-4.5515296E-1,1.4603574E-2,-3.0572563E-2,6.548415E-1,-2.9288656E-1,-7.6341426E-1,-1.2825903E-2,1.6137537E-1,4.3922216E-1,8.377188E-1,-2.4863131E-2,-1.7758082E-1,-1.2843414E-2,-3.2234408E-2,1.3643051E-2,2.1229606E-4,8.309624E-3,1.9282134E-2,2.190433E-2,3.800347E-2,-3.5284942E-1,-9.7808175E-2,-5.4097087E-3,-1.7520351E-2,4.9898024E-3,-1.7535536E-1,-4.5112697E-3,-9.795604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1],"loss_changes":[1.9396439E1,3.851984E0,5.5902433E0,2.4775124E0,0E0,1.1791408E0,1.169899E0,1.5662498E0,6.3215256E-1,0E0,4.0178597E-1,2.7371645E-1,3.542862E-1,0E0,3.8493752E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6925931E-1,4.2689222E-1,0E0,0E0,0E0,4.8322022E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,23,23,24,24,28,28],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1],"split_conditions":[1.5363757E8,7.621429E1,5.1707494E-1,5.070136E7,1.4603574E-2,3.0962205E-2,7.3845965E8,1.497E3,5.3323513E-1,-1.2825903E-2,2E0,1.218666E0,2.5172096E-1,-2.4863131E-2,1E0,-1.2843414E-2,-3.2234408E-2,1.3643051E-2,2.1229606E-4,8.309624E-3,1.9282134E-2,2.190433E-2,3.800347E-2,6.9533944E7,2.9000288E-2,-5.4097087E-3,-1.7520351E-2,4.9898024E-3,1.0128521E6,-4.5112697E-3,-9.795604E-3],"split_indices":[12,57,40,46,0,39,7,2,28,0,8,40,39,0,17,0,0,0,0,0,0,0,0,7,39,0,0,0,48,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.8E1,5.5E1,5.3E1,5E0,1.7E1,3.8E1,3.6E1,1.7E1,6E0,1.1E1,1.9E1,1.9E1,7E0,2.9E1,5E0,1.2E1,4E0,7E0,7E0,1.2E1,1.1E1,8E0,8E0,2.1E1,4E0,4E0,5E0,1.6E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-4.2027324E-2,-3.7183625E-1,5.0865483E-1,-1.9803534E-1,-6.767564E-1,1.4365476E-1,8.339433E-1,-5.00647E-1,7.6184355E-2,-7.7971935E-1,-1.0852783E-2,1.6833028E-2,1.3703063E-2,1.730201E-2,3.6189623E-2,-7.088462E-1,-1.7992336E-1,-1.0535246E-1,3.168087E-1,-3.1054046E-2,-1.6682882E-2,-6.622137E-3,1.5032132E-1,-1.8500404E-2,-3.3237103E-2,-2.6283695E-3,-9.837503E-3,6.741438E-2,-1.1279693E-2,1.9201145E-2,4.2584944E-3,1.3669189E-3,7.905985E-3,5.1801493E-3,-7.9919887E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0348944E1,3.575656E0,4.8668747E0,3.855317E0,7.338495E-1,8.739692E-1,8.3414936E-1,1.3414993E0,1.125431E0,8.793354E-2,0E0,0E0,4.5846978E-1,0E0,0E0,6.897545E-2,7.9315215E-2,5.577067E-1,4.0763426E-1,0E0,0E0,0E0,6.892693E-2,0E0,0E0,0E0,0E0,6.448403E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,22,22,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2310118E7,5.9633E7,1.2480776E0,1.3486356E3,3.5904072E3,1.5306258E5,4.867052E6,1.977643E6,9E0,3.1346153E1,-1.0852783E-2,1.6833028E-2,6.919127E5,1.730201E-2,3.6189623E-2,7.472597E7,1.1954737E1,3.863112E7,4.3650795E-2,-3.1054046E-2,-1.6682882E-2,-6.622137E-3,4.340646E-1,-1.8500404E-2,-3.3237103E-2,-2.6283695E-3,-9.837503E-3,5.526E3,-1.1279693E-2,1.9201145E-2,4.2584944E-3,1.3669189E-3,7.905985E-3,5.1801493E-3,-7.9919887E-4],"split_indices":[51,46,40,4,4,29,1,49,8,59,0,0,33,0,0,7,55,46,58,0,0,0,40,0,0,0,0,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,6.9E1,4.1E1,4.5E1,2.4E1,2E1,2.1E1,2.1E1,2.4E1,1.8E1,6E0,5E0,1.5E1,8E0,1.3E1,1.2E1,9E0,1.4E1,1E1,1.3E1,5E0,6E0,9E0,8E0,4E0,5E0,4E0,8E0,6E0,4E0,6E0,4E0,5E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.0581495E-2,-3.905329E-1,5.229766E-1,-2.1395065E-1,-6.927597E-1,2.7906787E-1,7.874607E-1,-3.7685356E-1,1.7368104E-1,-1.1198943E-2,-2.8782893E-2,4.6702987E-1,5.5180494E-2,1.8140873E-2,8.869554E-1,-6.437481E-1,-1.6036002E-1,-6.7222877E-3,3.9430216E-1,1.015999E-2,2.2110943E-2,-5.6470808E-2,8.849449E-3,2.1888694E-2,3.8152087E-2,-2.8269665E-2,-1.5936205E-2,-2.3753755E-1,2.1222164E-4,2.22954E-2,3.328482E-3,-4.3679755E-3,6.981977E-4,-3.8479902E-3,-1.024952E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.33296E1,3.5341682E0,2.812214E0,2.907037E0,7.5199986E-1,1.0683694E0,1.7812729E-1,1.745758E0,1.1849732E0,0E0,0E0,2.0551062E-1,2.8952807E-1,0E0,7.4178696E-2,4.890871E-2,2.4516359E-1,0E0,5.5007005E-1,0E0,0E0,4.6358094E-2,0E0,0E0,0E0,0E0,0E0,4.866296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,18,18,21,21,27,27],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,5.4875424E7,1.3536367E0,1.0664373E3,5.3323513E-1,9.48E2,3.1E1,2.667836E6,1E0,-1.1198943E-2,-2.8782893E-2,4.903324E1,6.9015497E-1,1.8140873E-2,1.9088202E0,2.3E1,2.3514317E8,-6.7222877E-3,9.070543E0,1.015999E-2,2.2110943E-2,2.1E1,8.849449E-3,2.1888694E-2,3.8152087E-2,-2.8269665E-2,-1.5936205E-2,2.5445784E1,2.1222164E-4,2.22954E-2,3.328482E-3,-4.3679755E-3,6.981977E-4,-3.8479902E-3,-1.024952E-2],"split_indices":[12,46,40,53,28,0,3,33,17,0,0,57,40,0,43,3,7,0,54,0,0,8,0,0,0,0,0,59,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6.8E1,4.6E1,4.4E1,2.4E1,2.5E1,2.1E1,3.1E1,1.3E1,6E0,1.8E1,1.3E1,1.2E1,7E0,1.4E1,1.3E1,1.8E1,5E0,8E0,7E0,6E0,8E0,4E0,7E0,7E0,6E0,7E0,1.2E1,6E0,4E0,4E0,4E0,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.979678E-3,-3.4272724E-1,4.9574625E-1,-5.276246E-1,-1.013813E-1,2.4871044E-1,7.319686E-1,-7.0078516E-1,-2.0545907E-1,1.7967814E-1,-2.8312197E-1,-2.1515675E-2,3.8516822E-1,2.9084628E-2,1.6195044E-2,-1.3636926E-2,-2.7782705E-2,-1.0596163E-2,-3.7093132E-3,-3.6961345E-3,2.1482423E-2,-3.5835654E-1,-2.632875E-3,-6.17006E-3,4.7718557E-3,1.934534E-2,2.4118295E-1,-6.990057E-3,-1.480449E-2,4.890936E-3,1.0436601E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.8815786E1,2.8024602E0,2.5022974E0,1.8643255E0,1.5321889E0,8.9327407E-1,1.1856079E-1,2.2020626E-1,9.644824E-2,1.466751E0,2.5851727E-1,2.2921947E-1,2.1787167E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.1342387E-2,0E0,0E0,0E0,0E0,1.1469007E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,21,21,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[1.8147125E5,3.3041668E1,1.3536367E0,2.9282052E6,6.15786E5,4.7308203E3,2.1685E4,8.977316E0,2.207872E-1,3.09E2,2.7938022E3,4.226681E-1,6.19999E9,2.9084628E-2,1.6195044E-2,-1.3636926E-2,-2.7782705E-2,-1.0596163E-2,-3.7093132E-3,-3.6961345E-3,2.1482423E-2,2.9673992E9,-2.632875E-3,-6.17006E-3,4.7718557E-3,1.934534E-2,5.5918517E0,-6.990057E-3,-1.480449E-2,4.890936E-3,1.0436601E-2],"split_indices":[34,57,40,33,30,4,9,55,40,0,53,40,5,0,0,0,0,0,0,0,0,5,0,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,6.3E1,4.5E1,3.5E1,2.8E1,2.3E1,2.2E1,2.2E1,1.3E1,1.1E1,1.7E1,8E0,1.5E1,1.6E1,6E0,5E0,1.7E1,6E0,7E0,7E0,4E0,1.2E1,5E0,4E0,4E0,6E0,9E0,4E0,8E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.1256974E-2,-2.8215232E-1,5.441066E-1,-4.1236064E-1,5.7476006E-2,1.9641325E-4,6.3900536E-1,-2.2134565E-1,-6.4540946E-1,1.152904E-2,-1.0809073E-1,3.4943733E-1,8.0162185E-1,-3.7772104E-1,-1.19162746E-1,-1.0983238E-2,-2.6278079E-2,-6.874574E-3,7.108995E-4,1.5086514E-2,5.151032E-3,2.1407075E-2,3.850762E-2,-1.6439706E-2,-8.56121E-3,2.1845386E-3,-1.7110898E-1,-8.0796555E-3,-7.5740006E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.7286366E1,3.075047E0,2.041273E0,2.0829325E0,9.016283E-1,0E0,1.3208942E0,4.1837752E-1,3.9748955E-1,0E0,1.4050426E-1,1.384809E-1,5.2877617E-1,1.2079835E-2,1.8768522E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1395046E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[5.1707494E-1,2.5323425E3,1.1462246E3,4.851587E7,6.15786E5,1.9641325E-4,1.5083523E0,7.8714124E2,5.3323513E-1,1.152904E-2,1.7802E4,5.5E1,1.5605432E9,1.44E2,1.77E2,-1.0983238E-2,-2.6278079E-2,-6.874574E-3,7.108995E-4,1.5086514E-2,5.151032E-3,2.1407075E-2,3.850762E-2,-1.6439706E-2,-8.56121E-3,2.1845386E-3,4.01E2,-8.0796555E-3,-7.5740006E-4],"split_indices":[40,53,53,46,30,0,43,4,28,0,10,3,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.8E1,3.9E1,4.9E1,1.9E1,6E0,3.3E1,2.8E1,2.1E1,7E0,1.2E1,1.3E1,2E1,1E1,1.8E1,5E0,1.6E1,7E0,5E0,9E0,4E0,1.3E1,7E0,5E0,5E0,4E0,1.4E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.717804E-2,-3.7835562E-1,4.055E-1,-4.9538177E-1,5.748244E-3,1.4678942E-1,2.4617178E-2,-6.4040095E-1,-2.3421851E-1,1.187477E-2,-1.8872583E-1,3.5689092E-1,1.943298E-2,-1.0111054E-2,-6.868529E-1,-1.2533823E-2,-8.269428E-2,-1.2363759E-2,1.2093743E-3,5.453824E-3,1.7738875E-2,-1.2941217E-1,6.659366E-3,-1.4572546E-2,-2.7964214E-2,6.718205E-4,-6.0457406E-3,-1.0731786E-2,2.3218838E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,23,-1,25,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5594854E1,2.6998606E0,3.0457854E0,1.5921955E0,1.0047839E0,6.352512E-1,0E0,2.9305458E-1,2.7935505E-1,0E0,3.531009E-1,1.7483854E-1,4.1687348E-1,0E0,3.0840778E-1,0E0,8.039741E-2,0E0,0E0,0E0,0E0,3.117548E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,14,14,16,16,21,21],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,24,-1,26,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,2.667836E6,6.15786E5,1.1E1,2.4617178E-2,2.3452184E5,1.6757976E-1,1.187477E-2,2.4502628E6,1.0262E4,2.3342845E5,-1.0111054E-2,4.4E1,-1.2533823E-2,1.675502E8,-1.2363759E-2,1.2093743E-3,5.453824E-3,1.7738875E-2,1.4634906E-1,6.659366E-3,-1.4572546E-2,-2.7964214E-2,6.718205E-4,-6.0457406E-3,-1.0731786E-2,2.3218838E-3],"split_indices":[12,2,40,33,30,8,0,29,40,0,33,2,34,0,10,0,5,0,0,0,0,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.9E1,4.3E1,4.5E1,1.4E1,2.3E1,2E1,2.8E1,1.7E1,5E0,9E0,8E0,1.5E1,4E0,2.4E1,9E0,8E0,5E0,4E0,4E0,4E0,8E0,7E0,7E0,1.7E1,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.0237521E-2,-3.393916E-1,3.3153063E-1,-4.1472912E-1,1.794064E-2,1.8513696E-1,5.837542E-1,-5.59573E-1,-1.6224243E-1,-9.584433E-3,2.518441E-1,1.3587898E-2,2.431059E-2,-6.446323E-1,-5.9100497E-3,-2.562468E-1,3.6034028E-3,3.3394757E-1,1.4041496E-2,-1.2825363E-2,-7.3255205E-1,-1.2800422E-2,-1.513147E-1,3.880833E-1,2.2182493E-3,-4.391941E-3,5.3044334E-3,-3.0928945E-2,-1.80312E-2,-7.1180547E-3,-2.7151913E-3,2.967635E-1,2.109953E-2,1.3289075E-2,7.5208065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1],"loss_changes":[1.2149417E1,3.6714392E0,1.8488941E0,1.7743425E0,0E0,1.1189239E0,9.96809E-2,9.720001E-1,5.1825535E-1,0E0,6.005678E-1,0E0,0E0,4.012823E-1,0E0,1.010828E-1,0E0,3.1545758E-1,1.8015428E-1,0E0,1.219511E-1,0E0,1.657471E-2,1.6723347E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7949343E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,15,15,17,17,18,18,20,20,22,22,23,23,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1],"split_conditions":[2.0020565E8,1.0269E4,1.3536367E0,2.9282052E6,1.794064E-2,3.0962205E-2,4.1372877E8,3.159013E3,2.1816934E8,-9.584433E-3,1.8634315E1,1.3587898E-2,2.431059E-2,4.4E1,-5.9100497E-3,2.1225102E-1,3.6034028E-3,2.3152558E9,1.5959719E-1,-1.2825363E-2,1.2599278E1,-1.2800422E-2,2.862E3,1.62739E9,2.2182493E-3,-4.391941E-3,5.3044334E-3,-3.0928945E-2,-1.80312E-2,-7.1180547E-3,-2.7151913E-3,2.0246488E5,2.109953E-2,1.3289075E-2,7.5208065E-3],"split_indices":[12,2,40,33,0,39,7,4,7,0,55,0,0,10,0,40,0,7,39,0,54,0,2,7,0,0,0,0,0,0,0,29,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,5.4E1,5.2E1,5E1,4E0,3.4E1,1.8E1,3.1E1,1.9E1,4E0,3E1,7E0,1.1E1,2.5E1,6E0,1.4E1,5E0,2.2E1,8E0,7E0,1.8E1,6E0,8E0,1.8E1,4E0,4E0,4E0,1E1,8E0,4E0,4E0,1.4E1,4E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-6.7869793E-3,-3.906748E-1,4.028442E-1,-5.969478E-1,-7.2045274E-2,2.2578067E-1,7.419192E-1,-3.3789346E-1,-2.5150344E-2,-2.10561E-1,7.7721947E-3,4.3038604E-1,5.4099772E-2,1.792945E-2,3.229338E-2,-5.669667E-3,-1.56068895E-2,-8.948621E-3,-3.3880319E-3,4.431246E-3,5.4117095E-1,-1.1790051E-1,1.0247393E-2,2.496625E-2,9.116291E-3,-6.723284E-3,4.8160528E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1,-1],"loss_changes":[1.4624603E1,3.1117768E0,2.5341916E0,5.134649E-1,8.413062E-1,1.0656481E0,1.1712742E-1,1.0282707E-1,0E0,3.76938E-2,0E0,4.1073418E-1,7.4473435E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5941038E-1,8.986153E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,4.852701E6,1.3536367E0,6.072427E-1,9.63264E2,8.69374E5,1.5605432E9,4.80378E5,-2.5150344E-2,3.670858E1,7.7721947E-3,1.2288E4,2.3342845E5,1.792945E-2,3.229338E-2,-5.669667E-3,-1.56068895E-2,-8.948621E-3,-3.3880319E-3,4.431246E-3,1.9045439E5,1.8154767E5,1.0247393E-2,2.496625E-2,9.116291E-3,-6.723284E-3,4.8160528E-5],"split_indices":[2,33,40,28,53,30,7,12,0,59,0,2,34,0,0,0,0,0,0,0,34,29,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.1E1,4.7E1,4.4E1,2.8E1,1.9E1,3E1,1.4E1,9E0,1.9E1,1.3E1,6E0,1.3E1,1.7E1,7E0,7E0,4E0,5E0,9E0,4E0,4E0,9E0,1E1,7E0,5E0,4E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.1054016E-2,-3.1460762E-1,4.859552E-1,-4.4477397E-1,6.504634E-2,-9.317041E-3,5.7835984E-1,-5.9096265E-1,-2.7023336E-1,1.4754604E-2,-8.687175E-2,4.267692E-1,8.703059E-1,-3.6451545E-1,-2.4515454E-2,-3.1843606E-1,-2.0392803E-3,-9.375171E-3,8.218775E-3,9.218639E-3,1.83128E-2,1.7255593E-2,3.704989E-2,-7.613364E-3,-1.5589032E-2,-9.2998715E-3,-1.5438079E-2,-2.0880434E-3,3.099097E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6132267E1,3.5376215E0,2.830555E0,1.1926422E0,1.0273844E0,0E0,1.1775494E0,3.1094837E-1,2.4989724E-1,0E0,2.3218612E-1,2.1534109E-1,1.8244553E-1,1.2307167E-2,0E0,2.9345512E-2,0E0,0E0,5.6237247E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,9.066E3,2.80369E5,3.572374E6,6.52678E5,-9.317041E-3,1.62739E9,4.4E1,3.365958E8,1.4754604E-2,5.8163033E1,2.0681982E-1,2.0105577E-1,5.189068E-1,-2.4515454E-2,7.959073E-1,-2.0392803E-3,-9.375171E-3,1.662E3,9.218639E-3,1.83128E-2,1.7255593E-2,3.704989E-2,-7.613364E-3,-1.5589032E-2,-9.2998715E-3,-1.5438079E-2,-2.0880434E-3,3.099097E-3],"split_indices":[43,2,1,12,30,0,7,10,7,0,57,39,39,28,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7E1,3.8E1,5.2E1,1.8E1,4E0,3.4E1,2.7E1,2.5E1,5E0,1.3E1,2.4E1,1E1,9E0,1.8E1,2E1,5E0,4E0,9E0,9E0,1.5E1,4E0,6E0,4E0,5E0,1.5E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.0797309E-2,-3.6156452E-1,3.3161274E-1,-4.2126018E-1,5.285972E-3,1.06708296E-1,6.242183E-1,-5.598041E-1,-2.0221046E-1,-8.68231E-3,1.8969616E-1,4.364018E-1,3.2840192E-2,-3.724325E-1,-2.7545957E-2,-1.3955579E-2,-1.2728357E-1,2.8850928E-1,1.075685E-2,1.87067E-2,8.866908E-3,-4.663343E-1,-4.9896906E-3,6.6139596E-4,-7.289815E-3,3.6883238E-1,5.126675E-3,3.4905286E-3,-4.03236E-3,-2.0380057E-2,-1.04860645E-2,1.781412E-2,4.355333E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,-1,-1,23,25,27,-1,-1,29,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2902058E1,1.5778832E0,3.7502704E0,1.2524624E0,0E0,1.025262E0,9.4314003E-1,7.437258E-1,2.13225E-1,0E0,4.9198008E-1,1.3789701E-1,0E0,2.8745627E-1,0E0,0E0,1.667025E-1,1.5944874E-1,1.2653352E-1,0E0,0E0,2.5963306E-2,0E0,0E0,0E0,2.8063214E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,21,21,25,25],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,-1,-1,24,26,28,-1,-1,30,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4350427E8,2.7938022E3,1.3536367E0,2.667836E6,5.285972E-3,4.739085E-2,1.0861106E4,4.6268812E5,1E0,-8.68231E-3,5E1,1.2271712E1,3.2840192E-2,1.9121604E-4,-2.7545957E-2,-1.3955579E-2,2.1923357E1,1E0,3.7E1,1.87067E-2,8.866908E-3,3.8E1,-4.9896906E-3,6.6139596E-4,-7.289815E-3,1.4273524E-1,5.126675E-3,3.4905286E-3,-4.03236E-3,-2.0380057E-2,-1.04860645E-2,1.781412E-2,4.355333E-3],"split_indices":[12,53,40,33,0,39,4,33,17,0,3,55,0,39,0,0,59,101,8,0,0,3,0,0,0,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,4.9E1,5.7E1,4.4E1,5E0,3.3E1,2.4E1,2.6E1,1.8E1,6E0,2.7E1,1.6E1,8E0,1.5E1,1.1E1,4E0,1.4E1,1.7E1,1E1,1E1,6E0,1E1,5E0,5E0,9E0,1E1,7E0,6E0,4E0,5E0,5E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-3.439152E-2,-2.8745982E-1,4.0628415E-1,-4.0453276E-1,1.5489583E-1,1.832985E-2,5.115344E-1,-2.7192265E-1,-5.81067E-1,-2.951975E-2,1.6866421E-2,4.405638E-3,-2.5684275E-3,3.12952E-1,6.413871E-1,-1.4641315E-1,-3.4862357E-1,-6.277753E-1,-1.0037368E-2,2.4273365E-3,-3.408564E-3,1.3045562E-2,6.50366E-3,1.4951235E-2,3.0611156E-2,-8.457483E-3,2.7486004E-3,-8.842935E-3,-1.5185564E-2,-1.171555E-2,-2.4669606E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.283019E1,3.843176E0,1.7055788E0,1.1997871E0,9.352175E-1,1.0146322E-1,6.5023136E-1,2.858913E-1,1.6518116E-1,7.541191E-2,0E0,0E0,0E0,3.1309128E-2,5.405083E-1,3.1108278E-1,5.3738117E-2,1.1511612E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,2.7938022E3,1.118632E3,8.0073607E-1,6.082581E3,9.5032835E0,1.5941123E0,2.8399336E7,2.7240606E5,1.8345214E-1,1.6866421E-2,4.405638E-3,-2.5684275E-3,1.5366939E7,1.286E4,1.2108435E3,2.1227852E5,2.07E2,-1.0037368E-2,2.4273365E-3,-3.408564E-3,1.3045562E-2,6.50366E-3,1.4951235E-2,3.0611156E-2,-8.457483E-3,2.7486004E-3,-8.842935E-3,-1.5185564E-2,-1.171555E-2,-2.4669606E-2],"split_indices":[43,53,53,28,53,54,40,46,48,28,0,0,0,48,2,4,29,11,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,7.2E1,4.1E1,5.7E1,1.5E1,9E0,3.2E1,3.4E1,2.3E1,1E1,5E0,4E0,5E0,1.4E1,1.8E1,1.4E1,2E1,1.9E1,4E0,4E0,6E0,9E0,5E0,1E1,8E0,1E1,4E0,1E1,1E1,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.472108E-2,-2.642644E-1,4.434464E-1,-3.4340566E-1,6.654095E-2,2.0786707E-1,6.330183E-1,-5.238764E-1,-1.6358441E-1,1.0051563E-2,-3.6348896E-3,-3.4342918E-3,3.025779E-1,4.346444E-1,3.3842564E-2,-3.2459053E-1,-2.2928808E-2,-2.596711E-1,2.5730023E-2,2.8112214E-3,3.8971868E-1,1.8355465E-2,1.0401099E-2,-6.572296E-3,-1.4911974E-2,-1.3982144E-2,-8.9025676E-2,3.8719538E-3,-2.7884827E-3,1.7398123E-2,9.207346E-3,-5.392798E-3,-3.92571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3315104E1,1.6722131E0,1.9227409E0,1.5693779E0,4.8861796E-1,6.601082E-1,1.1020508E0,3.479581E-1,4.9782735E-1,0E0,0E0,0E0,3.025396E-1,3.7325144E-2,0E0,6.349945E-2,0E0,3.5750234E-1,9.22194E-2,0E0,2.609384E-2,0E0,0E0,0E0,0E0,0E0,4.082188E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,17,17,18,18,20,20,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[9.095874E6,2.5323425E3,1.5364555E0,2.667836E6,6.15786E5,4.1451712E5,1.5605432E9,1.08E2,2.0237808E8,1.0051563E-2,-3.6348896E-3,-3.4342918E-3,3.486353E6,2.326E4,3.3842564E-2,8.317631E-1,-2.2928808E-2,2.6326587E-2,9.242E3,2.8112214E-3,7.118011E-1,1.8355465E-2,1.0401099E-2,-6.572296E-3,-1.4911974E-2,-1.3982144E-2,2.68E2,3.8719538E-3,-2.7884827E-3,1.7398123E-2,9.207346E-3,-5.392798E-3,-3.92571E-4],"split_indices":[49,53,43,33,30,33,7,10,7,0,0,0,1,9,0,28,0,39,9,0,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.2E1,4.5E1,5E1,1.2E1,2.1E1,2.4E1,2.4E1,2.6E1,5E0,7E0,5E0,1.6E1,1.6E1,8E0,1E1,1.4E1,1.7E1,9E0,5E0,1.1E1,9E0,7E0,5E0,5E0,9E0,8E0,5E0,4E0,5E0,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.6608386E-3,-2.7908042E-1,3.9974752E-1,-3.936006E-1,3.6976933E-2,1.7811656E-1,6.047223E-1,-5.568649E-1,-1.6049762E-1,2.739653E-1,-1.1665881E-2,3.156085E-1,8.964504E-2,4.4029915E-1,3.1717613E-2,-6.084411E-1,-6.773321E-3,-2.2856256E-1,-4.6261054E-4,1.5954038E-3,1.7920634E-2,1.399342E-2,6.5165246E-3,-3.280731E-2,9.089672E-3,1.8156307E-2,1.0073523E-2,-1.3630825E-2,-7.147181E-1,-1.0780131E-2,-2.6625989E-3,-3.5189209E-3,9.583637E-4,-2.8507112E-2,-1.4154024E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2806435E1,2.4758105E0,2.0274415E0,1.8134947E0,1.7099975E0,2.6364613E-1,6.9367695E-1,4.3814564E-1,2.1821755E-1,5.913157E-1,0E0,1.7577112E-2,3.2590234E-1,3.132105E-2,0E0,3.4360504E-1,0E0,1.4060438E-1,0E0,0E0,0E0,0E0,0E0,4.096568E-2,0E0,0E0,0E0,0E0,7.706261E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.5364555E0,2.667836E6,6.1358623E9,1.7644464E5,1.5605432E9,3.0387878E3,3.167669E5,1.1653E4,-1.1665881E-2,1.0550945E2,3.9219662E5,2.0837E4,3.1717613E-2,2.63E2,-6.773321E-3,2.207872E-1,-4.6261054E-4,1.5954038E-3,1.7920634E-2,1.399342E-2,6.5165246E-3,4.152984E-1,9.089672E-3,1.8156307E-2,1.0073523E-2,-1.3630825E-2,1.2599278E1,-1.0780131E-2,-2.6625989E-3,-3.5189209E-3,9.583637E-4,-2.8507112E-2,-1.4154024E-2],"split_indices":[51,2,43,33,5,29,7,4,29,2,0,58,29,9,0,11,0,40,0,0,0,0,0,40,0,0,0,0,54,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,6.7E1,4.6E1,4.9E1,1.8E1,2.3E1,2.3E1,2.8E1,2.1E1,1.1E1,7E0,8E0,1.5E1,1.6E1,7E0,2.4E1,4E0,1.4E1,7E0,6E0,5E0,4E0,4E0,9E0,6E0,1E1,6E0,9E0,1.5E1,9E0,5E0,4E0,5E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.1170119E-2,-2.286945E-1,2.9556155E-1,-2.757291E-1,9.817417E-3,7.455043E-2,5.5917764E-1,-4.1882515E-1,-1.4190349E-1,1.7954913E-1,-1.20397314E-1,2.1995092E-2,1.0729085E-2,-5.0299E-1,-2.1702932E-1,-2.541205E-1,-2.7940229E-2,4.6226624E-4,2.7831575E-1,-6.8809628E-3,-9.431044E-4,-8.674969E-3,-6.057556E-1,-2.6501452E-3,-1.0850561E-2,-1.1013974E-2,-4.2730784E-3,-1.1090766E-1,2.861581E-3,1.2885317E-2,4.6336856E-3,-2.5775805E-2,-1.3016385E-2,-1.4181143E-3,-6.306461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9691954E0,1.581959E0,2.4053206E0,1.0704117E0,0E0,5.128795E-1,6.811571E-2,3.6639166E-1,4.021356E-1,2.5993794E-1,5.457996E-2,0E0,0E0,3.6739302E-1,9.788659E-2,7.456696E-2,1.5999314E-1,0E0,7.2051585E-2,0E0,0E0,0E0,8.751106E-2,0E0,0E0,0E0,0E0,3.8817257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,22,22,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.379017E6,1.2424786E2,1.3589408E0,2.667836E6,9.817417E-3,5.6E1,2.1383E4,2.6631096E3,2.834008E-1,1.1015E4,1.0448E4,2.1995092E-2,1.0729085E-2,4.4E1,8.661788E7,2.942787E6,1.3448806E8,4.6226624E-4,2.581938E2,-6.8809628E-3,-9.431044E-4,-8.674969E-3,1.2599278E1,-2.6501452E-3,-1.0850561E-2,-1.1013974E-2,-4.2730784E-3,2.4E1,2.861581E-3,1.2885317E-2,4.6336856E-3,-2.5775805E-2,-1.3016385E-2,-1.4181143E-3,-6.306461E-3],"split_indices":[52,57,43,33,0,3,9,56,58,2,9,0,0,10,46,52,7,0,57,0,0,0,54,0,0,0,0,3,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,6.3E1,4.1E1,5.8E1,5E0,2.3E1,1.8E1,2.7E1,3.1E1,1.5E1,8E0,1.4E1,4E0,1.8E1,9E0,1.5E1,1.6E1,6E0,9E0,4E0,4E0,6E0,1.2E1,4E0,5E0,1E1,5E0,9E0,7E0,5E0,4E0,7E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6760997E-2,-1.9793408E-1,2.6402497E-1,-2.781832E-1,2.7125448E-1,-4.521447E-3,3.2001278E-1,-1.6120237E-1,-4.5164436E-1,1.738997E-2,1.8301925E-3,1.1050596E-3,3.8652018E-1,-2.1479665E-1,-2.4742452E-2,-5.228169E-1,-3.794221E-3,2.1083939E-1,4.408205E-1,-2.7492607E-1,-7.7352576E-2,3.2140955E-3,-4.8517697E-3,-1.2521984E-2,-2.1779887E-2,4.294831E-3,9.40665E-3,4.914743E-1,9.4613675E-3,-1.109944E-2,-4.8886845E-3,9.531097E-4,-5.979898E-3,1.0108562E-2,2.0295251E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,21,23,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4445705E0,2.5228367E0,9.554236E-1,1.0635252E0,4.3969083E-1,0E0,7.003269E-1,2.506569E-1,4.876814E-1,0E0,0E0,0E0,1.932435E-1,1.8534458E-1,1.4917842E-1,4.5605183E-2,0E0,1.0111302E-2,7.040262E-2,3.548169E-2,8.680768E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.1743956E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,22,24,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,5.188755E3,3.727E3,8.317631E-1,8.69374E5,-4.521447E-3,1E0,7.287E3,3.271557E8,1.738997E-2,1.8301925E-3,1.1050596E-3,1.2756285E3,2.834008E-1,4.5843E7,4.4815472E5,-3.794221E-3,1.2958418E5,1.0007143E1,3.7712722E6,3.39E2,3.2140955E-3,-4.8517697E-3,-1.2521984E-2,-2.1779887E-2,4.294831E-3,9.40665E-3,4.0383283E8,9.4613675E-3,-1.109944E-2,-4.8886845E-3,9.531097E-4,-5.979898E-3,1.0108562E-2,2.0295251E-2],"split_indices":[43,4,2,28,30,0,102,2,7,0,0,0,53,58,46,29,0,34,54,51,0,0,0,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.4E1,4.1E1,5.5E1,9E0,5E0,3.6E1,3.4E1,2.1E1,4E0,5E0,7E0,2.9E1,2.4E1,1E1,1.7E1,4E0,8E0,2.1E1,1.6E1,8E0,5E0,5E0,7E0,1E1,4E0,4E0,1.5E1,6E0,1.2E1,4E0,4E0,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7335935E-2,-2.5816494E-1,2.9193032E-1,-3.4373537E-1,1.05630405E-1,1.6434705E-1,6.829365E-1,-4.440817E-1,-1.3462842E-1,-3.7841632E-3,1.2042006E-2,-2.8410375E-3,2.1557806E-1,1.4404345E-2,2.845641E-2,-4.9313763E-1,-2.9081749E-3,-9.198718E-3,-2.5035532E-2,2.7649057E-1,-2.6771717E-3,-5.622826E-1,-1.1280322E-2,2.1476792E-3,-4.3849307E-3,3.290193E-3,3.435109E-1,-1.0248506E-2,-2.2297278E-2,1.6219167E-2,7.725943E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[7.8213167E0,1.8729858E0,2.18251E0,9.4740343E-1,6.105282E-1,4.652062E-1,1.4320374E-2,5.309415E-1,2.174935E-1,0E0,0E0,0E0,5.4995143E-1,0E0,0E0,1.7414093E-1,0E0,0E0,8.87228E-2,2.7926195E-1,0E0,1.17313385E-1,0E0,0E0,0E0,0E0,1.5035939E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,18,18,19,19,21,21,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[2.3724032E8,2.561146E3,1.8614753E0,3.2936362E6,1E0,2.2843675E-1,7.4684826E8,7.33E3,2.9544958E-1,-3.7841632E-3,1.2042006E-2,-2.8410375E-3,2.1972223E2,1.4404345E-2,2.845641E-2,2.9282052E6,-2.9081749E-3,-9.198718E-3,3.51387E5,8.784E3,-2.6771717E-3,2.63E2,-1.1280322E-2,2.1476792E-3,-4.3849307E-3,3.290193E-3,1.5625807E1,-1.0248506E-2,-2.2297278E-2,1.6219167E-2,7.725943E-3],"split_indices":[7,53,40,51,17,40,7,2,42,0,0,0,58,0,0,33,0,0,9,2,0,11,0,0,0,0,55,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,5.8E1,4.5E1,4.7E1,1.1E1,3.5E1,1E1,3.1E1,1.6E1,6E0,5E0,6E0,2.9E1,4E0,6E0,2.7E1,4E0,7E0,9E0,2.4E1,5E0,1.8E1,9E0,5E0,4E0,7E0,1.7E1,4E0,1.4E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5175937E-2,-1.7119141E-1,4.021599E-1,-2.3383586E-1,1.771783E-1,-9.246658E-3,4.8470944E-1,-9.42045E-2,-3.9910802E-1,1.671502E-2,-8.5188553E-4,1.0748329E-2,5.9519696E-1,-3.2642293E-1,7.6650004E-3,-1.8785229E-2,-2.3206866E-1,3.7581056E-1,2.8844167E-2,-1.50354775E-2,-6.5700733E-3,1.6668053E-1,-7.22686E-2,-1.0507577E-2,-4.74809E-3,1.5626846E-2,7.5599775E-3,9.283002E-3,1.5487989E-3,-1.13182046E-1,2.048584E-3,-4.8094895E-3,-1.8224122E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,21,-1,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[8.433935E0,1.6781878E0,2.2434268E0,1.4393415E0,6.995077E-1,0E0,5.459299E-1,8.5411704E-1,4.6643257E-1,0E0,0E0,0E0,6.3842154E-1,6.726909E-2,3.4252357E-1,0E0,4.47464E-2,2.3247719E-2,0E0,0E0,0E0,8.972788E-2,1.0143166E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0862216E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,14,14,16,16,17,17,21,21,22,22,29,29],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,22,-1,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[1.0230335E0,3.5324219E3,1.4960615E3,5.070136E7,5.4875424E7,-9.246658E-3,1.7908138E0,7.998101E2,3.0595828E3,1.671502E-2,-8.5188553E-4,1.0748329E-2,2.463E4,1.54E2,5.83E2,-1.8785229E-2,3.4707712E8,1.2257767E1,2.8844167E-2,-1.50354775E-2,-6.5700733E-3,9.20242E0,4.7992307E-1,-1.0507577E-2,-4.74809E-3,1.5626846E-2,7.5599775E-3,9.283002E-3,1.5487989E-3,4.01E2,2.048584E-3,-4.8094895E-3,-1.8224122E-3],"split_indices":[40,53,4,46,46,0,43,4,4,0,0,0,2,0,10,0,7,55,0,0,0,54,40,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.4E1,3.8E1,6.3E1,1.1E1,4E0,3.4E1,3.5E1,2.8E1,4E0,7E0,1.4E1,2E1,1E1,2.5E1,1.5E1,1.3E1,1.1E1,9E0,5E0,5E0,8E0,1.7E1,7E0,6E0,7E0,4E0,4E0,4E0,1.3E1,4E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.2068253E-2,-1.648847E-1,3.3613706E-1,-3.5189933E-1,5.7612408E-2,4.1618764E-1,2.6709302E-3,-4.3274614E-1,-1.9394387E-1,2.1607071E-1,-2.1128578E-1,2.1955945E-2,3.239555E-1,-2.0425077E-1,-1.9531492E-2,-9.119429E-3,-1.3666184E-3,-1.1566876E-3,2.8010044E-1,-1.1187661E-2,-4.449788E-3,6.5825083E-3,1.3277747E-2,-1.1341023E-2,-1.9322936E-3,1.3200576E-2,4.2562406E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.057305E0,2.7665477E0,5.777972E-1,3.7579823E-1,1.3665363E0,1.9249415E-1,0E0,4.58385E-1,1.1860639E-1,3.267014E-1,6.596291E-2,0E0,3.84475E-2,1.3224477E-1,0E0,0E0,0E0,0E0,1.7888987E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,6.363496E7,3.143873E6,2.9282052E6,2.7070812E7,1.9718067E5,2.6709302E-3,4.5E1,4E1,8.873786E0,1.9452468E-2,2.1955945E-2,3.9E1,1.7E2,-1.9531492E-2,-9.119429E-3,-1.3666184E-3,-1.1566876E-3,2.26182E5,-1.1187661E-2,-4.449788E-3,6.5825083E-3,1.3277747E-2,-1.1341023E-2,-1.9322936E-3,1.3200576E-2,4.2562406E-3],"split_indices":[53,12,30,33,46,34,0,10,3,57,39,0,3,0,0,0,0,0,30,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.3E1,6.5E1,2.8E1,3.5E1,3E1,2.1E1,7E0,2.2E1,1.3E1,1.9E1,1.1E1,5E0,1.6E1,8E0,1.4E1,9E0,4E0,4E0,1.5E1,4E0,7E0,5E0,1.1E1,4E0,4E0,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.045447E-3,-1.3045764E-1,5.493323E-1,-2.3233856E-1,1.6364071E-1,1.0528478E-2,6.968363E-1,-4.1000903E-1,-1.4423028E-1,-1.806817E-3,2.225081E-1,2.9008435E-2,1.39654195E-2,-4.578551E-1,-5.6234505E-3,-7.763605E-2,-3.319938E-1,2.806637E-1,1.795295E-3,-8.949189E-3,-5.195474E-1,-1.8612137E-1,-1.3987516E-4,-1.6001105E-2,-7.978266E-3,5.450663E-3,1.146059E-2,-1.0650814E-2,-2.1176642E-2,-8.391662E-3,-3.6097334E-3,6.374891E-3,-7.7443935E-2,-4.1714245E-3,1.0643514E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,19,-1,21,23,25,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,33,-1,-1],"loss_changes":[7.5769463E0,2.764654E0,5.5704784E-1,1.0053525E0,3.1303632E-1,0E0,8.5967064E-2,1.9630885E-1,5.6191164E-1,0E0,1.7695892E-1,0E0,0E0,1.0438442E-1,0E0,3.0263305E-1,5.020356E-2,1.8849254E-2,0E0,0E0,2.2569895E-2,3.660202E-2,3.1421196E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.858496E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,13,13,15,15,16,16,17,17,20,20,21,21,22,22,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,20,-1,22,24,26,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,34,-1,-1],"split_conditions":[6.0880933E3,6.402099E-1,1.3127055E4,1.6755434E1,8.873786E0,1.0528478E-2,1.890618E6,1.8839418E3,7.752181E7,-1.806817E-3,4.5E1,2.9008435E-2,1.39654195E-2,1.16E2,-5.6234505E-3,2.8866995E1,3.0595828E3,1.601247E1,1.795295E-3,-8.949189E-3,5.432393E-1,1.3101191E1,3.9709E5,-1.6001105E-2,-7.978266E-3,5.450663E-3,1.146059E-2,-1.0650814E-2,-2.1176642E-2,-8.391662E-3,-3.6097334E-3,6.374891E-3,4.6961885E3,-4.1714245E-3,1.0643514E-3],"split_indices":[53,43,4,59,57,0,30,53,46,0,3,0,0,0,0,57,4,57,0,0,28,57,30,0,0,0,0,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,6.7E1,2.3E1,8E0,1.1E1,2.1E1,4.6E1,5E0,1.8E1,7E0,4E0,1.7E1,4E0,3.5E1,1.1E1,1.3E1,5E0,5E0,1.2E1,1.4E1,2.1E1,4E0,7E0,4E0,9E0,4E0,8E0,8E0,6E0,6E0,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.197816E-2,-7.497011E-2,5.014067E-1,-2.3085347E-1,1.19796686E-1,5.8881146E-1,5.0151106E-3,-3.131732E-1,-2.3126926E-2,3.297368E-1,-9.704131E-3,1.1902153E-2,2.6172534E-2,-2.1767734E-1,-1.5210481E-2,-5.2955006E-3,4.8081987E-2,1.9302368E-2,4.8680156E-3,-1.1230028E-1,1.03101134E-1,-4.1702559E-4,-9.9552395E-3,3.2475363E-3,-1.2290914E-4,1.4748385E-3,-7.003339E-3,2.650139E-4,7.817953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.034423E0,2.3047233E0,5.2020025E-1,7.109568E-1,9.275719E-1,2.9015398E-1,0E0,2.1808338E-1,1.2142535E-1,4.4709122E-1,2.6619276E-1,0E0,0E0,2.1226358E-1,0E0,0E0,1.9707847E-2,0E0,0E0,1.580405E-1,1.17909305E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3536367E0,2.561146E3,2.326E4,2.6326587E-2,2.5927516E11,2.5172096E-1,5.0151106E-3,8.4395015E-1,1.5938355E8,8.012376E0,9.250217E5,1.1902153E-2,2.6172534E-2,8.1837E4,-1.5210481E-2,-5.2955006E-3,4.9457936E7,1.9302368E-2,4.8680156E-3,1.5457461E5,4.1822475E-1,-4.1702559E-4,-9.9552395E-3,3.2475363E-3,-1.2290914E-4,1.4748385E-3,-7.003339E-3,2.650139E-4,7.817953E-3],"split_indices":[40,53,9,39,32,39,0,28,7,54,33,0,0,12,0,0,33,0,0,29,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.2E1,7.4E1,1.8E1,4.1E1,3.3E1,1.4E1,4E0,2.9E1,1.2E1,1.2E1,2.1E1,6E0,8E0,1.7E1,1.2E1,4E0,8E0,5E0,7E0,1.1E1,1E1,4E0,1.3E1,4E0,4E0,4E0,7E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.737566E-4,-2.4148624E-1,2.8224903E-1,-3.1347293E-1,1.0660252E-1,5.9629556E-2,3.9899176E-1,-1.6329855E-1,-4.773247E-1,1.1325116E-2,-3.0231455E-3,-2.9234872E-3,1.6615129E-1,4.9787334E-1,2.0917995E-1,-3.1328753E-1,-5.934927E-2,-1.926247E-2,-1.0421838E-2,1.2407742E-3,8.963621E-3,2.0304255E-2,7.3911734E-3,3.2640689E-3,9.287992E-3,-6.30179E-3,-1.5059091E-2,4.346179E-3,-5.487278E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.8135777E0,1.3865364E0,1.1717119E0,1.0293384E0,4.2043418E-1,2.6807314E-1,4.5892143E-1,3.7118775E-1,6.87027E-2,0E0,0E0,0E0,9.9698305E-2,2.4425411E-1,4.5271814E-2,7.123941E-2,2.8470373E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6225724E-1,2.7938022E3,1.6492815E3,8.297822E-1,1.0803E4,2.0978074E8,1.4046429E1,1.0930736E1,1.0034782E-3,1.1325116E-2,-3.0231455E-3,-2.9234872E-3,1.8589372E5,2.57281E6,9.257902E-2,6.948189E7,2.083117E1,-1.926247E-2,-1.0421838E-2,1.2407742E-3,8.963621E-3,2.0304255E-2,7.3911734E-3,3.2640689E-3,9.287992E-3,-6.30179E-3,-1.5059091E-2,4.346179E-3,-5.487278E-3],"split_indices":[43,53,53,28,9,7,55,59,40,0,0,0,34,30,39,7,59,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,5.3E1,4.5E1,4.4E1,9E0,1.6E1,2.9E1,2.4E1,2E1,4E0,5E0,7E0,9E0,1.8E1,1.1E1,9E0,1.5E1,1.4E1,6E0,4E0,5E0,1.4E1,4E0,4E0,7E0,5E0,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.6093224E-2,-2.1192521E-1,2.9062074E-1,-4.335574E-1,-5.6446034E-2,1.5187746E-1,5.564244E-1,-4.9036043E-3,-4.83801E-1,-1.8762808E-1,1.0892272E-1,3.4369522E-1,1.8826129E-2,1.2425799E-2,2.488158E-2,-1.0767954E-2,-2.0046884E-2,-9.543987E-3,-1.3038243E-1,2.0427169E-1,-3.6077367E-3,3.526767E-3,1.9222017E-2,-9.245615E-2,1.2801662E-1,-2.9906249E-3,-5.9965374E-3,2.5470222E-3,9.470343E-3,1.3927397E-3,-7.4010394E-3,-6.437534E-4,8.962964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0079656E0,2.0775373E0,1.3537521E0,3.0827093E-1,8.230931E-1,6.8261296E-1,1.2911081E-1,0E0,1.1285877E-1,5.7770073E-2,3.5894716E-1,4.5870197E-1,2.1840233E-1,0E0,0E0,0E0,0E0,0E0,1.001364E-2,7.4634016E-2,0E0,0E0,0E0,1.4062831E-1,1.6226287E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2789746E3,2.667836E6,1.6957201E0,2.3452184E5,2.3514317E8,6.19999E9,2.463E4,-4.9036043E-3,2.9488824E2,1.7301265E6,2.63382E5,7.21E2,8.184232E5,1.2425799E-2,2.488158E-2,-1.0767954E-2,-2.0046884E-2,-9.543987E-3,9E0,9.489487E2,-3.6077367E-3,3.526767E-3,1.9222017E-2,5.6075206E0,4.4432E4,-2.9906249E-3,-5.9965374E-3,2.5470222E-3,9.470343E-3,1.3927397E-3,-7.4010394E-3,-6.437534E-4,8.962964E-3],"split_indices":[53,33,40,29,7,5,2,0,53,49,30,0,33,0,0,0,0,0,18,53,0,0,0,54,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6E1,3.8E1,2.4E1,3.6E1,2.6E1,1.2E1,4E0,2E1,2E1,1.6E1,1E1,1.6E1,6E0,6E0,7E0,1.3E1,7E0,1.3E1,1.1E1,5E0,5E0,5E0,8E0,8E0,7E0,6E0,4E0,7E0,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[5.2020777E-2,-1.21101394E-1,3.6666822E-1,-2.0498253E-1,1.4658998E-1,2.7944025E-2,2.8636226E-1,-1.4501555E-1,-3.592246E-1,2.809931E-1,1.0968173E-2,1.7762247E-2,2.072143E-1,-3.1797555E-1,-7.3385365E-2,-1.6689165E-2,-4.607437E-3,5.7943943E-3,1.24660535E-2,-2.0134482E-3,3.2081057E-3,1.1230931E-1,1.1426109E-2,-1.6463945E-2,-6.9298414E-3,-1.2628634E-1,6.995995E-2,4.787221E-2,6.1650337E-3,7.958925E-4,-1.5867427E-1,4.6231477E-3,-7.6772274E-5,-1.0806797E-3,4.1916743E-3,-6.5995953E-3,-1.969959E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[6.145319E0,1.6692088E0,1.0868273E0,4.6587515E-1,3.2502186E-1,0E0,4.6767354E-1,4.9700975E-1,2.9897773E-1,1.4311731E-2,5.693878E-2,0E0,2.4445832E-1,1.2124944E-1,2.4391133E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.208996E-2,0E0,0E0,0E0,1.1399335E-1,3.796269E-2,5.125404E-2,0E0,0E0,3.774631E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,21,21,25,25,26,26,27,27,30,30],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,2.8938947E8,1.6451234E-2,5.501383E9,5.46E2,2.7944025E-2,2.8129198E7,2.253E3,3.5904072E3,2.2659668E-1,3.9219662E5,1.7762247E-2,8.640353E3,4.0748124E7,2.3514317E8,-1.6689165E-2,-4.607437E-3,5.7943943E-3,1.24660535E-2,-2.0134482E-3,3.2081057E-3,1E0,1.1426109E-2,-1.6463945E-2,-6.9298414E-3,5.2287582E-2,7.0590184E7,8.18961E3,6.1650337E-3,7.958925E-4,3.5983E4,4.6231477E-3,-7.6772274E-5,-1.0806797E-3,4.1916743E-3,-6.5995953E-3,-1.969959E-3],"split_indices":[2,12,58,5,0,0,46,2,4,39,29,0,53,5,7,0,0,0,0,0,0,102,0,0,0,58,12,4,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,7.2E1,3.9E1,5.5E1,1.7E1,5E0,3.4E1,4.1E1,1.4E1,8E0,9E0,8E0,2.6E1,1.1E1,3E1,9E0,5E0,4E0,4E0,5E0,4E0,1.5E1,1.1E1,4E0,7E0,2.2E1,8E0,8E0,7E0,4E0,1.8E1,4E0,4E0,4E0,4E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.251468E-2,-1.2000842E-1,3.6634618E-1,-2.7589586E-1,8.500792E-2,5.2591234E-1,8.943176E-2,-3.3621106E-1,-8.631847E-2,-3.040177E-2,2.2731982E-1,7.803743E-3,6.5605694E-1,1.6560884E-1,-2.917438E-3,-3.7966388E-1,-3.578619E-3,-6.0078017E-3,1.5549397E-3,6.212109E-3,-1.00540236E-1,1.0412533E-2,2.1543296E-3,1.3267694E-2,2.6267806E-2,2.5978214E-3,7.8001083E-3,-2.2031178E-1,-1.7659152E-2,-4.9550044E-3,-2.5344206E-4,-9.553114E-3,-4.7625317E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[5.517495E0,2.2606163E0,1.4994822E0,4.2835498E-1,5.1606023E-1,7.4509144E-1,1.9494057E-1,2.7143145E-1,1.2115493E-1,2.7291396E-1,1.2617743E-1,0E0,4.2197704E-2,3.169495E-2,0E0,3.2834005E-1,0E0,0E0,0E0,0E0,4.709524E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2081087E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,20,20,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,5.6614784E7,1.4046429E1,3.0595828E3,9.2954946E-1,1.7059127E5,1.782E3,3.2936362E6,7.056912E7,2.7186182E-1,4.4E1,7.803743E-3,3E0,4.1E1,-2.917438E-3,2.9180167E2,-3.578619E-3,-6.0078017E-3,1.5549397E-3,6.212109E-3,1.2836887E1,1.0412533E-2,2.1543296E-3,1.3267694E-2,2.6267806E-2,2.5978214E-3,7.8001083E-3,9.7E1,-1.7659152E-2,-4.9550044E-3,-2.5344206E-4,-9.553114E-3,-4.7625317E-3],"split_indices":[2,12,55,4,40,29,0,51,7,28,3,0,8,3,0,53,0,0,0,0,55,0,0,0,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,3.9E1,3E1,2.1E1,1.3E1,2.9E1,1E1,1.7E1,1.3E1,7E0,1.4E1,9E0,4E0,2.4E1,5E0,6E0,4E0,4E0,1.3E1,9E0,4E0,4E0,1E1,4E0,5E0,1.1E1,1.3E1,9E0,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[9.404197E-3,-8.191902E-2,3.9164418E-1,-1.7726035E-1,1.2871456E-1,2.3037602E-1,2.0978406E-2,-3.552201E-1,-1.11507826E-1,-4.5729848E-3,1.7655551E-1,9.793322E-3,3.9969105E-3,-1.6186135E-2,-6.1901193E-3,-1.7534252E-1,-2.716451E-2,2.3359561E-1,5.5533648E-2,-2.0777021E-1,8.693186E-5,2.9419977E-3,-8.7311916E-2,2.4158028E-3,1.1802541E-2,-1.4702341E-4,3.7558996E-3,-4.640943E-3,-2.8998297E-1,-6.154108E-3,-2.2178702E-2,-1.2990197E-2,-5.3502736E-3,-2.6759414E-3,1.2346494E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,-1,-1,19,21,23,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[3.8020809E0,1.7900395E0,5.076542E-1,6.741699E-1,3.644562E-1,3.3572435E-2,0E0,1.968987E-1,2.4290222E-1,0E0,1.5270191E-1,0E0,0E0,0E0,0E0,1.5036523E-1,1.4435473E-1,2.2699332E-1,2.6441261E-2,1.06012404E-1,0E0,0E0,7.203901E-2,0E0,0E0,0E0,0E0,0E0,4.568374E-2,0E0,2.8889056E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,15,15,16,16,17,17,18,18,19,19,22,22,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,-1,-1,20,22,24,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[6.1596104E3,6.402099E-1,1.2768678E4,1.983671E4,3.568E3,7.118011E-1,2.0978406E-2,2.8374892E-2,7.388E3,-4.5729848E-3,1.4163358E7,9.793322E-3,3.9969105E-3,-1.6186135E-2,-6.1901193E-3,2.3514317E8,4.171707E9,1.09484E3,4.0389168E-1,6.681133E-1,8.693186E-5,2.9419977E-3,2.074635E-1,2.4158028E-3,1.1802541E-2,-1.4702341E-4,3.7558996E-3,-4.640943E-3,1.2163482E8,-6.154108E-3,3.3487454E11,-1.2990197E-2,-5.3502736E-3,-2.6759414E-3,1.2346494E-3],"split_indices":[53,43,53,49,2,28,0,58,2,0,46,0,0,0,0,7,5,53,28,28,0,0,40,0,0,0,0,0,7,0,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,8.7E1,2E1,6E1,2.7E1,1.2E1,8E0,1.5E1,4.5E1,4E0,2.3E1,8E0,4E0,9E0,6E0,2.5E1,2E1,1.5E1,8E0,2.1E1,4E0,7E0,1.3E1,6E0,9E0,4E0,4E0,1.2E1,9E0,5E0,8E0,5E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.6260551E-2,-1.8317385E-1,2.6741064E-1,-2.540629E-1,9.2660606E-2,1.4874665E-1,4.6568242E-1,-3.7536886E-1,-8.0223374E-2,-3.7698024E-3,9.9825505E-3,-2.4102514E-3,2.0709147E-1,9.956253E-3,2.3324372E-2,-1.4566985E-3,-4.2952442E-1,4.1348813E-3,-1.327848E-1,2.665169E-1,9.6425675E-2,-6.5715313E-3,-4.8507664E-1,-6.243079E-3,-7.78916E-2,1.7643574E-1,1.2484552E-2,4.8931944E-3,8.0829137E-4,-2.0885479E-2,-1.2388691E-2,-9.059528E-4,-4.155864E-3,2.2343034E-3,1.0058555E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.655051E0,1.1967599E0,1.1541727E0,9.8209953E-1,4.998774E-1,4.4133198E-1,4.7265387E-1,4.9067974E-1,2.301748E-1,0E0,0E0,0E0,1.5186274E-1,0E0,0E0,0E0,2.416892E-1,0E0,2.3988068E-2,6.040609E-2,2.8816007E-2,0E0,5.3410053E-2,0E0,1.4797598E-2,9.979406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,16,16,18,18,19,19,20,20,22,22,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.770862E8,2.561146E3,1.7618678E0,2.667836E6,2.9844797E-1,3.5849205E-1,9.563013E8,4.029937E-1,3.42E2,-3.7698024E-3,9.9825505E-3,-2.4102514E-3,1.6135693E1,9.956253E-3,2.3324372E-2,-1.4566985E-3,2.63E2,4.1348813E-3,9.320373E4,1.7302E4,1E0,-6.5715313E-3,1.4613986E3,-6.243079E-3,1.3560078E5,1.1803382E4,1.2484552E-2,4.8931944E-3,8.0829137E-4,-2.0885479E-2,-1.2388691E-2,-9.059528E-4,-4.155864E-3,2.2343034E-3,1.0058555E-2],"split_indices":[7,53,40,33,28,40,7,28,10,0,0,0,55,0,0,0,11,0,34,9,84,0,53,0,34,4,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.9E1,5.1E1,4.7E1,1.2E1,3.3E1,1.8E1,2.7E1,2E1,6E0,6E0,7E0,2.6E1,1E1,8E0,4E0,2.3E1,4E0,1.6E1,1.6E1,1E1,5E0,1.8E1,8E0,8E0,9E0,7E0,6E0,4E0,9E0,9E0,4E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0040615E-2,-1.6906129E-1,2.188069E-1,-2.1587087E-1,6.489098E-3,2.9824182E-2,3.124698E-1,-1.4519845E-1,-3.1849194E-1,8.985236E-3,-6.618626E-2,2.0420541E-1,3.985903E-1,-1.4569583E-2,-9.884405E-2,-3.6497048E-1,-3.1579603E-3,2.670431E-3,-1.2964426E-1,2.4638239E-3,2.6649892E-1,1.101048E-2,1.8611364E-2,-3.096264E-2,-6.4645973E-3,-2.5481278E-1,-1.6131384E-2,-6.710294E-3,-1.3087403E-3,1.1467256E-2,5.8643576E-3,-9.1229245E-2,5.1979795E-3,-1.1566119E-2,-4.5196684E-3,-4.2870683E-3,-6.480817E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,31,-1,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[4.3742824E0,1.0674121E0,9.46517E-1,3.5989428E-1,0E0,4.1606957E-1,2.5426173E-1,3.7927127E-1,2.0430112E-1,0E0,1.3410327E-1,1.3426387E-1,5.074501E-2,0E0,1.6325581E-1,4.8431396E-2,0E0,0E0,4.5769468E-2,0E0,1.2253225E-2,0E0,0E0,2.0290853E-1,0E0,4.486531E-2,0E0,0E0,0E0,0E0,0E0,2.609723E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,18,18,20,20,23,23,25,25,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,32,-1,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,3.7249353E3,6.2371206E-1,8.4947616E-1,6.489098E-3,3.5925132E-1,2.6167098E5,7.110474E7,2.4502628E6,8.985236E-3,9.069479E-1,2.0783027E5,1.5605432E9,-1.4569583E-2,3.925844E7,3.91028E5,-3.1579603E-3,2.670431E-3,1.939024E7,2.4638239E-3,2.548476E5,1.101048E-2,1.8611364E-2,8E0,-6.4645973E-3,7.434519E5,-1.6131384E-2,-6.710294E-3,-1.3087403E-3,1.1467256E-2,5.8643576E-3,1.4198953E1,5.1979795E-3,-1.1566119E-2,-4.5196684E-3,-4.2870683E-3,-6.480817E-4],"split_indices":[7,53,40,28,0,28,29,5,33,0,58,34,7,0,46,12,0,0,49,0,34,0,0,8,0,29,0,0,0,0,0,57,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,6.2E1,5.3E1,5.5E1,7E0,1.8E1,3.5E1,3.4E1,2.1E1,5E0,1.3E1,1.7E1,1.8E1,4E0,3E1,1.7E1,4E0,4E0,9E0,6E0,1.1E1,1.2E1,6E0,1.7E1,1.3E1,9E0,8E0,5E0,4E0,6E0,5E0,1.3E1,4E0,5E0,4E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-7.882657E-3,-1.962244E-1,2.4056515E-1,-9.932849E-2,-4.060304E-1,-6.703214E-2,3.3300647E-1,-1.7431891E-1,8.36856E-3,-6.773431E-3,-1.7170759E-2,-4.2780684E-3,6.651603E-4,4.1061303E-1,5.650482E-3,-2.618067E-1,-4.604724E-2,6.484681E-3,4.815757E-1,-3.1523302E-1,-3.0545453E-3,-4.3786177E-3,4.645628E-3,1.9725902E-2,7.826164E-3,-7.88878E-3,-1.6787257E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,-1,17,-1,19,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7733073E0,1.1387713E0,1.2734487E0,1.0584866E0,1.8430138E-1,5.0560266E-2,4.014423E-1,3.6949956E-1,0E0,0E0,0E0,0E0,0E0,2.9052138E-1,0E0,1.626724E-1,2.1278434E-1,0E0,1.7327881E-1,1.1566639E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,13,13,15,15,16,16,18,18,19,19],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,-1,18,-1,20,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,2.5927516E11,5.12544E-1,1E1,6.948189E7,5.3534385E3,1.4046429E1,1.386854E8,8.36856E-3,-6.773431E-3,-1.7170759E-2,-4.2780684E-3,6.651603E-4,5.403511E8,5.650482E-3,1.977643E6,3.167669E5,6.484681E-3,1.746E3,2E0,-3.0545453E-3,-4.3786177E-3,4.645628E-3,1.9725902E-2,7.826164E-3,-7.88878E-3,-1.6787257E-2],"split_indices":[12,32,40,8,7,4,55,7,0,0,0,0,0,7,0,49,29,0,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.7E1,4.3E1,4E1,1.7E1,1E1,3.3E1,3.3E1,7E0,5E0,1.2E1,6E0,4E0,2.2E1,1.1E1,1.9E1,1.4E1,6E0,1.6E1,1.4E1,5E0,1E1,4E0,1.2E1,4E0,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.234922E-2,-1.1959446E-1,2.692677E-1,-1.5458244E-1,1.2714416E-2,5.4225646E-4,3.244151E-1,-7.8147516E-2,-2.969515E-1,4.1584706E-1,5.732994E-3,-1.2855452E-1,1.2656279E-1,-1.4236198E-1,-1.4189349E-2,4.6046913E-1,6.855493E-3,-1.9554819E-1,-4.6117224E-2,8.02689E-3,1.9784305E-4,-7.7642547E-3,-9.622835E-4,8.53168E-3,1.8610887E-2,-8.071357E-3,-2.6134562E-3,1.9264045E-3,-8.0829665E-2,-3.943927E-3,-4.8823442E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[3.747346E0,1.1768546E0,5.4195094E-1,6.7324007E-1,0E0,0E0,4.1853905E-1,4.5998007E-1,2.707882E-1,1.1421871E-1,0E0,1.8427902E-1,1.0314682E-1,7.8383625E-2,0E0,7.552695E-2,0E0,4.937589E-2,6.3014776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3382068E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,28,28],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[5.1707494E-1,6.082581E3,1.6511278E1,8.106617E-1,1.2714416E-2,5.4225646E-4,1.343964E1,4.899102E5,3.3012803E9,4.698217E7,5.732994E-3,2.834008E-1,4.828443E-1,2.45E2,-1.4189349E-2,5.403511E8,6.855493E-3,4.3E1,9.069479E-1,8.02689E-3,1.9784305E-4,-7.7642547E-3,-9.622835E-4,8.53168E-3,1.8610887E-2,-8.071357E-3,-2.6134562E-3,1.9264045E-3,5.691229E6,-3.943927E-3,-4.8823442E-4],"split_indices":[40,53,57,28,0,0,55,29,5,46,0,58,28,11,0,7,0,3,58,0,0,0,0,0,0,0,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.7E1,3.8E1,6.3E1,4E0,7E0,3.1E1,4.2E1,2.1E1,1.9E1,1.2E1,3.4E1,8E0,9E0,1.2E1,1.5E1,4E0,1.8E1,1.6E1,4E0,4E0,5E0,4E0,4E0,1.1E1,1.4E1,4E0,4E0,1.2E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.3009777E-2,-1.8076913E-1,1.8460925E-1,-2.2224353E-1,4.2094267E-3,-4.635656E-3,2.9711828E-1,-1.4615564E-1,-3.4048074E-1,-9.690105E-2,5.266527E-3,2.193756E-1,4.255929E-1,-1.5337352E-2,-9.483938E-2,-3.8632032E-1,-4.5380723E-3,-2.348625E-2,-7.5696246E-3,2.442181E-1,4.058913E-3,1.9231215E-2,6.8796948E-3,-3.5304856E-2,-1.6997646E-1,-7.313136E-3,-1.554929E-2,-2.6376923E-3,1.1114292E-3,1.0115554E-2,5.5168583E-3,4.8919874E-3,-9.378512E-2,-7.996792E-3,-3.9744787E-3,-8.3479716E-4,-4.3228953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,-1,23,25,-1,27,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[3.6801403E0,7.6589215E-1,1.0872574E0,4.270475E-1,0E0,2.9187408E-1,2.2624993E-1,4.4591415E-1,1.4835119E-1,1.0269912E-1,0E0,3.0995369E-2,1.7534697E-1,0E0,1.2819651E-1,4.3216944E-2,0E0,2.6463097E-2,0E0,1.3077617E-2,0E0,0E0,0E0,1.9250572E-1,1.5729427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2905715E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,17,17,19,19,23,23,24,24,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,-1,24,26,-1,28,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[1.5363757E8,2.711196E3,9.745363E-1,8.297822E-1,4.2094267E-3,3.9219662E5,1.5435694E9,9.2433E4,3.1346153E1,2.2492808E5,5.266527E-3,1.5723504E1,4.4E1,-1.5337352E-2,3.925844E7,2.08E2,-4.5380723E-3,5.3990013E1,-7.5696246E-3,2.326E4,4.058913E-3,1.9231215E-2,6.8796948E-3,2.083117E1,1.7E2,-7.313136E-3,-1.554929E-2,-2.6376923E-3,1.1114292E-3,1.0115554E-2,5.5168583E-3,4.8919874E-3,3.2785356E7,-7.996792E-3,-3.9744787E-3,-8.3479716E-4,-4.3228953E-3],"split_indices":[12,53,43,28,0,29,7,1,59,34,0,55,3,0,46,11,0,57,0,9,0,0,0,59,10,0,0,0,0,0,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.9E1,5E1,5.2E1,7E0,1.9E1,3.1E1,3.3E1,1.9E1,1.2E1,7E0,2.1E1,1E1,4E0,2.9E1,1.5E1,4E0,8E0,4E0,1.6E1,5E0,6E0,4E0,1.7E1,1.2E1,4E0,1.1E1,4E0,4E0,1E1,6E0,4E0,1.3E1,5E0,7E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.9804446E-2,-1.6584219E-1,2.2534335E-1,-1.9925624E-1,8.962442E-3,1.1894395E-1,4.0559512E-1,-6.371361E-2,-3.373642E-1,-4.79808E-2,1.6835034E-1,2.5866732E-1,2.008889E-2,-1.2507846E-2,-6.890907E-3,-3.8854584E-1,-1.861423E-1,4.3842144E-4,-3.556473E-3,2.6246664E-1,6.928512E-2,1.119961E-2,5.6111203E-3,7.1935244E-2,-5.912861E-3,-6.931863E-3,-1.5930815E-2,-3.332643E-3,-8.763888E-3,1.2654458E-2,3.128588E-3,3.9051168E-3,-2.550271E-4,-1.4070388E-3,1.3063706E-1,8.102995E-3,-2.0714968E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1],"loss_changes":[4.4254956E0,8.93484E-1,1.0065844E0,1.0455177E0,0E0,3.049749E-1,2.9931355E-1,4.7958618E-1,1.4438891E-1,2.8538108E-2,2.428773E-1,1.5650094E-2,0E0,0E0,3.3331746E-1,1.2249017E-1,2.539453E-2,0E0,0E0,1.8632936E-1,4.5291804E-2,0E0,0E0,1.2228049E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.47826E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,19,19,20,20,23,23,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1],"split_conditions":[3.4707712E8,3.7249353E3,1.7618678E0,6.4987373E-1,8.962442E-3,3.5849205E-1,1.2750684E9,1.497E3,3.1371258E1,1.0824E4,1.2320755E1,2.3559767E5,2.008889E-2,-1.2507846E-2,3.925844E7,4.5E1,3.25E2,4.3842144E-4,-3.556473E-3,7.118011E-1,2.5E1,1.119961E-2,5.6111203E-3,3.22115E5,-5.912861E-3,-6.931863E-3,-1.5930815E-2,-3.332643E-3,-8.763888E-3,1.2654458E-2,3.128588E-3,3.9051168E-3,-2.550271E-4,-1.4070388E-3,1.0428423E9,8.102995E-3,-2.0714968E-5],"split_indices":[7,53,40,28,0,40,7,2,59,10,55,34,0,0,46,10,0,0,0,28,8,0,0,1,0,0,0,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6E1,5.4E1,5.6E1,4E0,3.5E1,1.9E1,2.9E1,2.7E1,8E0,2.7E1,1.1E1,8E0,4E0,2.5E1,1.9E1,8E0,4E0,4E0,1.3E1,1.4E1,6E0,5E0,1.7E1,8E0,5E0,1.4E1,4E0,4E0,8E0,5E0,9E0,5E0,6E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[8.9094596E-4,-8.418677E-2,2.9088813E-1,-1.6697854E-1,3.8710527E-2,4.689037E-1,1.03466496E-1,-2.052833E-1,6.5171765E-3,-4.9591303E-2,7.356622E-3,9.249497E-3,2.178669E-2,-5.2451534E-4,5.9309984E-3,-1.4593045E-1,-3.2189476E-1,8.833714E-4,-1.0717486E-1,-1.0503395E-2,-8.657066E-2,-6.019513E-3,-1.3442244E-2,-5.490938E-3,-5.008993E-4,-1.1700313E-1,1.4549642E-3,-6.5351226E-3,-2.1594516E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,19,21,-1,23,-1,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[2.3930295E0,7.6958007E-1,6.854913E-1,6.3603854E-1,4.619252E-1,1.602242E-1,8.849475E-2,2.367959E-1,0E0,9.2179015E-2,0E0,0E0,0E0,0E0,0E0,2.2441888E-1,3.9655447E-2,0E0,4.817827E-2,0E0,8.965723E-2,0E0,0E0,0E0,0E0,5.506572E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,15,15,16,16,18,18,20,20,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,20,22,-1,24,-1,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[4.492037E3,2.3514317E8,2.5927516E11,1E0,1.1329926E0,7.845511E-1,1.5438823E7,5.5309926E9,6.5171765E-3,9.513889E-1,7.356622E-3,9.249497E-3,2.178669E-2,-5.2451534E-4,5.9309984E-3,9.2433E4,5.145631E-1,8.833714E-4,1.447E3,-1.0503395E-2,6.939E3,-6.019513E-3,-1.3442244E-2,-5.490938E-3,-5.008993E-4,9.320373E4,1.4549642E-3,-6.5351226E-3,-2.1594516E-3],"split_indices":[53,7,32,89,40,28,1,5,0,58,0,0,0,0,0,1,59,0,0,0,2,0,0,0,0,34,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,7.4E1,2.1E1,4.4E1,3E1,1E1,1.1E1,4E1,4E0,2E1,1E1,5E0,5E0,4E0,7E0,2.8E1,1.2E1,9E0,1.1E1,7E0,2.1E1,4E0,8E0,7E0,4E0,1.7E1,4E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.5014396E-2,-1.1989209E-1,2.198597E-1,-1.8675263E-1,4.2297527E-2,-2.6039736E-4,2.7899745E-1,-3.1083202E-1,-1.0462264E-1,2.1596667E-1,-1.07906945E-1,3.4681892E-1,1.4881904E-1,-4.906404E-3,-3.5700446E-1,-1.3887258E-1,1.7085096E-3,1.4346364E-2,2.0050958E-3,-1.8250199E-1,1.7309767E-3,1.8190123E-2,9.37768E-3,1.8626243E-3,7.112547E-3,-1.4488746E-2,-6.4715906E-3,-9.498444E-2,-7.522882E-3,-9.739513E-3,-2.1204518E-3,-5.0669527E-3,-4.923456E-2,-2.7675268E-3,-3.599623E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[2.7113624E0,8.320497E-1,4.5858276E-1,5.177746E-1,6.2327963E-1,0E0,1.8676686E-1,1.21100664E-1,1.8351245E-1,2.9806107E-1,1.598858E-1,1.0155201E-1,4.1373044E-2,0E0,5.4087877E-2,6.56144E-2,0E0,0E0,0E0,8.136505E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2521486E-2,0E0,0E0,0E0,0E0,1.4473958E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,19,19,27,27,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.0230335E0,2.4722983E3,1.6511278E1,3.572374E6,5.4875424E7,-2.6039736E-4,2.8129198E7,3.842849E5,3.4707712E8,1.0007143E1,9.250217E5,8.892E3,3.8737573E11,-4.906404E-3,2.1883162E1,6.784095E7,1.7085096E-3,1.4346364E-2,2.0050958E-3,2.1227852E5,1.7309767E-3,1.8190123E-2,9.37768E-3,1.8626243E-3,7.112547E-3,-1.4488746E-2,-6.4715906E-3,1.3101191E1,-7.522882E-3,-9.739513E-3,-2.1204518E-3,-5.0669527E-3,3.1851864E7,-2.7675268E-3,-3.599623E-5],"split_indices":[40,53,57,12,46,0,46,29,7,54,33,9,32,0,59,46,0,0,0,29,0,0,0,0,0,0,0,57,0,0,0,0,33,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7.5E1,3.3E1,5.3E1,2.2E1,7E0,2.6E1,2E1,3.3E1,1E1,1.2E1,1.6E1,1E1,5E0,1.5E1,2.7E1,6E0,4E0,6E0,8E0,4E0,4E0,1.2E1,4E0,6E0,1.1E1,4E0,1.8E1,9E0,4E0,4E0,8E0,1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[8.900368E-3,-1.15445875E-1,2.4601895E-1,-2.604195E-1,3.313542E-2,2.0173583E-2,1.5582131E-1,-1.14240386E-1,-3.0438584E-1,-9.80291E-2,1.2803899E-1,2.1317573E-1,-5.024573E-3,-6.8414137E-3,-5.8254326E-4,-3.6944482E-3,-3.521724E-1,-6.209623E-3,-7.600827E-4,7.6640663E-3,-5.1744486E-4,2.620522E-1,3.2050724E-3,-4.1836056E-1,-7.5407824E-3,1.2138781E-2,4.283731E-3,-6.7753308E-3,-1.8421523E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,-1,-1,-1,-1,-1],"loss_changes":[2.8294582E0,1.3656723E0,8.438661E-1,1.7088962E-1,4.1088697E-1,0E0,4.8648798E-1,6.3389756E-2,1.9569445E-1,7.609184E-2,2.3092851E-1,1.1658132E-1,0E0,0E0,0E0,0E0,9.2832804E-2,0E0,0E0,0E0,0E0,1.3029003E-1,0E0,1.5862107E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,16,16,21,21,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,-1,-1,-1,-1,-1],"split_conditions":[3.7249353E3,3.2316522E7,6.4417176E-2,2.4232728E5,3.8757455E-1,2.0173583E-2,2.2950048E10,8.6E1,1.68E2,1.3045E4,9.1197605E0,1.5625807E1,-5.024573E-3,-6.8414137E-3,-5.8254326E-4,-3.6944482E-3,1.365445E1,-6.209623E-3,-7.600827E-4,7.6640663E-3,-5.1744486E-4,6.19999E9,3.2050724E-3,2E0,-7.5407824E-3,1.2138781E-2,4.283731E-3,-6.7753308E-3,-1.8421523E-2],"split_indices":[53,12,58,29,42,0,5,10,0,9,54,55,0,0,0,0,55,0,0,0,0,5,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6.2E1,3.2E1,3.1E1,3.1E1,6E0,2.6E1,8E0,2.3E1,1.3E1,1.8E1,2.2E1,4E0,4E0,4E0,5E0,1.8E1,6E0,7E0,1.1E1,7E0,1.5E1,7E0,1.1E1,7E0,9E0,6E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.832938E-3,-1.3853852E-1,1.2620504E-1,-1.905101E-1,9.6116275E-2,-2.5492298E-4,2.4364126E-1,-3.3862132E-1,-1.3584775E-1,9.738527E-3,-1.5030988E-3,-1.2742741E-1,7.0410624E-2,2.8838646E-1,4.847233E-3,-7.497833E-3,-1.7344054E-2,-5.534093E-2,-2.5840554E-1,-1.5488224E-3,-6.376834E-3,7.605512E-3,-1.7269943E-2,6.6758594E-3,1.47138005E-2,-9.5443234E-2,2.678462E-3,-5.5975732E-3,-1.0780397E-2,-2.1615345E-3,1.9620631E-3,-5.338603E-4,-4.5246924E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9434575E0,7.134478E-1,8.02015E-1,3.3451617E-1,2.743179E-1,2.516135E-1,9.831953E-2,1.2301552E-1,3.409021E-1,0E0,0E0,3.401135E-2,2.2498912E-1,1.5891886E-1,0E0,0E0,0E0,1.2655362E-1,1.1868715E-2,0E0,0E0,0E0,3.986518E-2,0E0,0E0,4.15328E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,17,17,18,18,22,22,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,2.7938022E3,1.0230335E0,1.73E2,9E0,1.937363E6,2.8129198E7,2.16268E5,7.332772E-1,9.738527E-3,-1.5030988E-3,8.44629E7,9E0,4.2864155E1,4.847233E-3,-7.497833E-3,-1.7344054E-2,1E1,8.07967E5,-1.5488224E-3,-6.376834E-3,7.605512E-3,5.759438E5,6.6758594E-3,1.47138005E-2,4.3786978E2,2.678462E-3,-5.5975732E-3,-1.0780397E-2,-2.1615345E-3,1.9620631E-3,-5.338603E-4,-4.5246924E-3],"split_indices":[7,53,40,0,18,33,46,12,28,0,0,46,8,57,0,0,0,8,1,0,0,0,29,0,0,53,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,4.6E1,1E1,2.6E1,2.7E1,1.1E1,3.5E1,4E0,6E0,9E0,1.7E1,1.8E1,9E0,7E0,4E0,2.2E1,1.3E1,4E0,5E0,6E0,1.1E1,1.1E1,7E0,1.7E1,5E0,5E0,8E0,7E0,4E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-7.356629E-3,-8.673777E-2,2.8287694E-1,-2.2338766E-1,2.0403562E-2,4.1299033E-1,3.459055E-2,-1.732699E-1,-1.6370945E-2,-5.2707877E-2,1.347688E-1,6.4738477E-3,1.8480975E-2,5.8937715E-3,-3.645891E-3,-1.9200978E-1,-1.2156821E-3,3.48537E-3,-8.6418964E-2,1.3644533E-3,1.8818837E-1,-1.4850397E-1,-9.223227E-3,-1.0390652E-1,1.5569644E-4,4.4298754E-3,9.2181945E-3,-2.1377632E-3,-1.683674E-1,-4.756301E-3,-4.4620893E-4,-8.739493E-3,-4.31646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,-1,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,31,-1,-1,-1,-1],"loss_changes":[2.4666777E0,1.2343894E0,7.2873807E-1,3.4805036E-1,4.0845093E-1,2.54884E-1,1.7347302E-1,7.724774E-2,0E0,1.5928172E-1,9.131506E-2,0E0,0E0,0E0,0E0,3.9366364E-2,0E0,0E0,4.011491E-2,0E0,2.1434158E-2,2.3729384E-2,0E0,5.188629E-2,0E0,0E0,0E0,0E0,2.5017768E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,15,15,18,18,20,20,21,21,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,-1,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,32,-1,-1,-1,-1],"split_conditions":[6.1596104E3,3.851071E7,2.692756E6,1E0,9.2954946E-1,1.1681136E4,1.063036E8,2.6666668E-1,-1.6370945E-2,1.283561E6,6E0,6.4738477E-3,1.8480975E-2,5.8937715E-3,-3.645891E-3,8.4395015E-1,-1.2156821E-3,3.48537E-3,1.1855E4,1.3644533E-3,5.506832E-1,1.728E3,-9.223227E-3,1E0,1.5569644E-4,4.4298754E-3,9.2181945E-3,-2.1377632E-3,5.127381E1,-4.756301E-3,-4.4620893E-4,-8.739493E-3,-4.31646E-3],"split_indices":[53,12,30,105,40,4,52,58,0,49,8,0,0,0,0,28,0,0,2,0,28,2,0,101,0,0,0,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,3.6E1,4.7E1,1.4E1,8E0,3.1E1,5E0,2.9E1,1.8E1,5E0,9E0,4E0,4E0,2.7E1,4E0,5E0,2.4E1,7E0,1.1E1,1.8E1,9E0,2E1,4E0,7E0,4E0,4E0,1.4E1,1.5E1,5E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.9744236E-2,-9.327785E-2,2.0905109E-1,-1.2495501E-1,1.01146875E-2,1.2056951E-1,3.5438117E-1,-4.6363864E-2,-2.357004E-1,2.1097495E-3,1.7033628E-1,1.5840996E-2,3.53248E-3,-1.7231452E-1,2.1365339E-2,-1.1647715E-2,-6.4637825E-2,1.5850387E-3,-1.4479358E-3,7.3449826E-3,2.335599E-3,-7.999368E-3,-3.1095452E-3,7.366086E-3,-5.5106208E-2,-3.941915E-3,-2.5859976E-4,-9.458616E-3,-3.8008306E-3,-1.6233418E-3,7.8364415E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[2.2253702E0,7.4477845E-1,4.844929E-1,4.764741E-1,0E0,1.5881532E-1,2.818097E-1,2.9371983E-1,3.2114422E-1,1.7639747E-2,5.329585E-2,0E0,0E0,3.185776E-2,3.325218E-1,0E0,2.2261191E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.7937537E-2,0E0,0E0,1.2041082E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,24,24,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[7.307441E-1,7.3153823E3,1.5605432E9,3.3012803E9,1.01146875E-2,1.5832817E5,2.9548872E9,1.1301075E1,4.0916511E3,2.4E1,7.639958E-1,1.5840996E-2,3.53248E-3,4.1100917E0,1.53E2,-1.1647715E-2,1.4E1,1.5850387E-3,-1.4479358E-3,7.3449826E-3,2.335599E-3,-7.999368E-3,-3.1095452E-3,7.366086E-3,3.6449524E7,-3.941915E-3,-2.5859976E-4,9.9987485E-2,-3.8008306E-3,-1.6233418E-3,7.8364415E-4],"split_indices":[40,4,7,5,0,34,7,57,4,3,28,0,0,57,10,0,8,0,0,0,0,0,0,0,46,0,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,5.9E1,4E1,5.5E1,4E0,2.6E1,1.4E1,3.3E1,2.2E1,8E0,1.8E1,1E1,4E0,1.1E1,2.2E1,1.4E1,8E0,4E0,4E0,1.3E1,5E0,6E0,5E0,6E0,1.6E1,4E0,4E0,9E0,7E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.354157E-3,-9.587389E-2,2.811731E-1,-2.1141864E-1,4.2217784E-2,2.0473529E-2,1.9797917E-1,-2.5688913E-1,-6.405464E-2,-5.279064E-2,1.8850118E-1,2.5317988E-1,2.01925E-3,-2.9090002E-1,-2.407845E-3,3.9753322E-3,-6.473553E-3,-2.2955444E-2,-3.6873005E-3,9.01752E-3,4.0192218E-3,2.1807535E-3,3.078492E-1,-3.9866236E-3,-3.1869465E-1,-2.6209368E-3,6.1207065E-5,1.2906996E-2,5.6645344E-3,-1.3646177E-2,-6.0544596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.973064E0,1.2056081E0,5.775304E-1,2.582742E-1,4.9638656E-1,0E0,1.7402846E-1,1.8112898E-1,2.4054828E-1,3.0042421E-2,3.8393497E-2,1.605668E-1,0E0,9.523201E-2,0E0,0E0,0E0,1.8478293E-2,0E0,0E0,0E0,0E0,4.134941E-2,0E0,1.3354897E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,22,22,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[3.944201E3,3.2936362E6,6.4417176E-2,7.33E3,9.2954946E-1,2.0473529E-2,1.0090909E1,3.0174E4,6.25379E5,1.5832817E5,7.6363635E-1,1.286E4,2.01925E-3,9.122283E0,-2.407845E-3,3.9753322E-3,-6.473553E-3,1E0,-3.6873005E-3,9.01752E-3,4.0192218E-3,2.1807535E-3,8.5E1,-3.9866236E-3,2.9282052E6,-2.6209368E-3,6.1207065E-5,1.2906996E-2,5.6645344E-3,-1.3646177E-2,-6.0544596E-3],"split_indices":[53,51,58,2,40,0,54,9,30,34,58,2,0,54,0,0,0,8,0,0,0,0,3,0,33,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.4E1,2.8E1,4E1,3.4E1,5E0,2.3E1,3E1,1E1,2.1E1,1.3E1,1.6E1,7E0,2.5E1,5E0,4E0,6E0,1.4E1,7E0,6E0,7E0,4E0,1.2E1,4E0,2.1E1,4E0,1E1,8E0,4E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.742977E-2,-1.3676295E-1,1.1548542E-1,-2.2288944E-1,-1.5563322E-2,1.9800499E-1,-6.0553983E-2,-1.2472117E-1,-3.194958E-1,-9.593062E-2,6.399417E-3,1.9342003E-2,1.1535384E-1,3.6078976E-3,-1.1354564E-1,-7.6137953E-3,7.265229E-4,-3.8447765E-1,-3.1281011E-3,-4.4159214E-3,-6.635637E-4,1.6669326E-2,1.7680356E-1,-1.704069E-1,8.2625286E-4,-1.0050992E-2,-1.7998286E-2,3.225355E-3,-1.4410706E-3,7.2806743E-3,2.4139055E-3,-1.8518103E-3,-1.0082227E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,-1,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8081089E0,6.22841E-1,7.964315E-1,2.9208076E-1,4.199064E-1,9.7104037E-1,1.6442794E-1,2.4138746E-1,2.23917E-1,3.6627695E-2,0E0,0E0,1.8504104E-1,0E0,1.1250734E-1,0E0,0E0,1.8489003E-2,0E0,0E0,0E0,5.702096E-2,3.844726E-2,1.1533919E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,17,17,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,-1,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9512938E3,3.2316522E7,7.096748E7,7.7E2,3.167669E5,8.892E3,1.6872E5,3.13E2,2.4636926E3,7.287E3,6.399417E-3,1.9342003E-2,4.6947882E-1,3.6078976E-3,2.3342845E5,-7.6137953E-3,7.265229E-4,1.4731E4,-3.1281011E-3,-4.4159214E-3,-6.635637E-4,4.171707E9,1.0488685E6,5.7194314E0,8.2625286E-4,-1.0050992E-2,-1.7998286E-2,3.225355E-3,-1.4410706E-3,7.2806743E-3,2.4139055E-3,-1.8518103E-3,-1.0082227E-2],"split_indices":[53,12,46,11,29,9,29,11,4,2,0,0,40,0,34,0,0,9,0,0,0,5,29,54,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5.9E1,5.3E1,3.4E1,2.5E1,3.6E1,1.7E1,1.8E1,1.6E1,1.8E1,7E0,6E0,3E1,4E0,1.3E1,1.1E1,7E0,1.2E1,4E0,1.3E1,5E0,1.2E1,1.8E1,9E0,4E0,8E0,4E0,5E0,7E0,1.4E1,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.02846E-2,-1.2583281E-1,1.609192E-1,-2.4537455E-1,-3.224422E-2,2.2265982E-2,2.2434208E-1,-3.0606943E-1,-1.10208705E-1,-8.0833375E-2,6.9238697E-3,6.414566E-3,-3.1021109E-2,1.4450054E-1,3.30083E-1,-7.0478697E-3,-1.5188813E-2,-1.463285E-3,-5.6986716E-3,2.9271317E-3,-1.4155595E-1,-2.425322E-3,1.2423156E-3,6.8439497E-3,9.876944E-2,2.1005973E-1,1.5302213E-2,-7.6886537E-3,-7.620329E-2,1.3894588E-3,4.297493E-3,3.7717826E-3,9.879033E-3,-4.072505E-3,-8.679343E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3012059E0,6.074141E-1,4.9625063E-1,1.631614E-1,3.6382857E-1,1.6285528E-1,2.8156102E-1,1.2120068E-1,2.347505E-2,2.76931E-1,0E0,0E0,3.779842E-2,3.0739427E-2,9.103167E-2,0E0,0E0,0E0,0E0,0E0,7.710606E-2,0E0,0E0,0E0,1.2521699E-2,3.1826735E-2,0E0,0E0,3.2140665E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,20,20,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,3.6E1,1.0539878E-1,1.977643E6,7.621429E1,5.28E2,1.5605432E9,1E0,1.453125E1,4.828443E-1,6.9238697E-3,6.414566E-3,7.331134E-2,1.8831E5,2.5172096E-1,-7.0478697E-3,-1.5188813E-2,-1.463285E-3,-5.6986716E-3,2.9271317E-3,9.5137325E-4,-2.425322E-3,1.2423156E-3,6.8439497E-3,2.0492467E5,2.4190365E6,1.5302213E-2,-7.6886537E-3,1.3890547E1,1.3894588E-3,4.297493E-3,3.7717826E-3,9.879033E-3,-4.072505E-3,-8.679343E-5],"split_indices":[7,3,39,49,57,0,7,17,55,28,0,0,39,30,39,0,0,0,0,0,40,0,0,0,34,33,0,0,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.4E1,5.6E1,2.3E1,3.1E1,1.8E1,3.8E1,1.5E1,8E0,2.6E1,5E0,4E0,1.4E1,2.3E1,1.5E1,9E0,6E0,4E0,4E0,7E0,1.9E1,9E0,5E0,1E1,1.3E1,8E0,7E0,8E0,1.1E1,4E0,9E0,4E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7413821E-3,-7.304192E-2,1.538762E-1,-4.7649376E-2,-1.2434536E-2,9.16067E-2,2.8793538E-1,-7.807313E-2,1.2463469E-2,-1.2068142E-3,4.2704353E-3,3.3709877E-3,1.4516395E-2,-1.6121142E-1,-2.4195444E-3,-2.2930211E-1,-9.5653705E-2,1.843168E-1,-6.696599E-2,-4.3924944E-3,-1.0643932E-2,-4.8598866E-3,-6.4819455E-5,8.599346E-3,3.378555E-3,-9.3958385E-2,7.120131E-4,-2.051382E-3,-5.5891774E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,-1,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.1651835E0,4.7973612E-1,2.4910057E-1,8.2777345E-1,0E0,8.400969E-2,1.8454671E-1,3.9586163E-1,0E0,0E0,0E0,0E0,0E0,1.11982524E-1,4.2215064E-1,6.286186E-2,6.1919376E-2,2.1703422E-2,6.269066E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.64033E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,13,13,14,14,15,15,16,16,17,17,18,18,25,25],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,-1,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[1.0230335E0,1.5081978E8,1.5605432E9,8.517173E3,-1.2434536E-2,3.22115E5,2.5172096E-1,3.0870352E1,1.2463469E-2,-1.2068142E-3,4.2704353E-3,3.3709877E-3,1.4516395E-2,2.1883162E1,3.24395E5,3.9024E4,1.3101191E1,1.1247059E1,3.5324219E3,-4.3924944E-3,-1.0643932E-2,-4.8598866E-3,-6.4819455E-5,8.599346E-3,3.378555E-3,5.4470426E-1,7.120131E-4,-2.051382E-3,-5.5891774E-3],"split_indices":[40,46,7,4,0,1,39,57,0,0,0,0,0,59,30,12,57,55,53,0,0,0,0,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.1E1,3.2E1,6.6E1,5E0,2.3E1,9E0,6.2E1,4E0,4E0,1.9E1,4E0,5E0,2.9E1,3.3E1,1.3E1,1.6E1,8E0,2.5E1,6E0,7E0,1.1E1,5E0,4E0,4E0,1.9E1,6E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.636178E-2,-7.384814E-2,1.9631414E-1,9.506933E-3,-9.827288E-2,1.5571484E-1,1.4757711E-2,-5.056059E-2,-2.8890294E-1,1.1577707E-1,1.2283535E-2,-1.6273664E-1,9.539735E-4,-4.91648E-3,-1.3261816E-2,7.256067E-5,1.4154191E-1,-1.8801776E-1,-2.3271416E-3,-6.0944006E-2,1.1141794E-1,6.11137E-3,2.0914278E-3,-8.3101755E-3,-2.690497E-3,-8.331574E-2,1.6320741E-3,6.5980474E-3,1.2661217E-3,-3.5396528E-3,-5.985618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,17,19,-1,-1,-1,21,23,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[1.7027802E0,6.023588E-1,2.3823917E-1,0E0,5.8568907E-1,1.8905854E-1,0E0,3.1220505E-1,1.0569215E-1,7.885781E-2,0E0,3.0118614E-2,2.6661515E-1,0E0,0E0,0E0,4.1209966E-2,4.5053452E-2,0E0,6.313539E-2,6.984249E-2,0E0,0E0,0E0,0E0,1.9014508E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,11,11,12,12,16,16,17,17,19,19,20,20,25,25],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,18,20,-1,-1,-1,22,24,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,6.371407E0,1.1276298E9,9.506933E-3,7.526629E7,6.2767E4,1.4757711E-2,2.782713E6,8.588E3,3.22115E5,1.2283535E-2,1.8873918E3,3.9219662E5,-4.91648E-3,-1.3261816E-2,7.256067E-5,1.9445117E2,1.9121604E-4,-2.3271416E-3,7.64755E5,8E0,6.11137E-3,2.0914278E-3,-8.3101755E-3,-2.690497E-3,3.8E1,1.6320741E-3,6.5980474E-3,1.2661217E-3,-3.5396528E-3,-5.985618E-4],"split_indices":[40,55,33,0,46,2,0,12,9,1,0,53,29,0,0,0,57,39,0,9,8,0,0,0,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,4E0,6.5E1,3E1,4E0,5.3E1,1.2E1,2.6E1,4E0,1.6E1,3.7E1,5E0,7E0,5E0,2.1E1,1.2E1,4E0,2.4E1,1.3E1,1.5E1,6E0,8E0,4E0,2E1,4E0,6E0,7E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.2470782E-2,-6.1596636E-2,1.5408692E-1,1.3200057E-2,-1.5370871E-1,6.761617E-2,2.7063823E-1,-4.1043464E-2,1.1494473E-2,-2.795465E-1,-4.729551E-2,1.0111448E-1,-2.4008842E-3,3.305493E-1,3.058147E-3,-1.1177263E-1,1.4055576E-2,-5.4431413E-3,-1.1407882E-2,1.5595459E-2,-6.3424553E-3,2.8277634E-4,5.310851E-3,6.1746673E-3,1.434267E-2,-6.1946614E-3,-5.2834257E-2,-5.1676974E-2,4.3003736E-3,-1.9323725E-3,5.226226E-2,-7.302295E-5,-3.3604323E-3,5.067166E-4,-2.9764003E-3,-2.6324703E-4,3.6595308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[1.100362E0,4.7741678E-1,3.4451842E-1,6.5867484E-1,4.0189344E-1,1.0556354E-1,1.3909137E-1,1.3407274E-1,0E0,1.8049717E-2,1.4807205E-1,7.6065645E-2,0E0,4.9132705E-2,0E0,4.3811426E-2,1.4427063E-1,0E0,0E0,3.574428E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.821586E-2,2.7438339E-2,0E0,0E0,2.705666E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,19,19,26,26,27,27,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[9.745363E-1,2.5927516E11,4.8E1,8.784E3,3.5904072E3,4.5E1,1.31E3,1.56967745E1,1.1494473E-2,3.290356E5,7.752181E7,9.1094507E2,-2.4008842E-3,1.5605432E9,3.058147E-3,9.7E1,3.1228534E5,-5.4431413E-3,-1.1407882E-2,4.6562446E3,-6.3424553E-3,2.8277634E-4,5.310851E-3,6.1746673E-3,1.434267E-2,-6.1946614E-3,1.7076684E7,1.4783E4,4.3003736E-3,-1.9323725E-3,1.8302105E6,-7.302295E-5,-3.3604323E-3,5.067166E-4,-2.9764003E-3,-2.6324703E-4,3.6595308E-3],"split_indices":[43,32,57,2,4,3,0,57,0,29,46,53,0,7,0,10,29,0,0,56,0,0,0,0,0,0,46,9,0,0,33,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.8E1,3.5E1,3.8E1,3E1,2.1E1,1.4E1,3.3E1,5E0,1.3E1,1.7E1,1.7E1,4E0,1E1,4E0,1.4E1,1.9E1,4E0,9E0,1.2E1,5E0,6E0,1.1E1,4E0,6E0,6E0,8E0,1.2E1,7E0,4E0,8E0,4E0,4E0,4E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-5.6217406E-3,-1.4812388E-1,1.00196324E-1,-2.0029053E-1,4.5021903E-3,4.8456933E-2,2.4591851E-1,-1.6120425E-1,-1.4022321E-2,-2.4330225E-2,1.2546659E-1,2.9834944E-1,2.5597236E-3,-2.0845999E-1,6.655365E-4,-6.092472E-2,4.7496065E-2,1.6492017E-1,6.743358E-2,6.5516746E-3,1.2849381E-2,-1.1372252E-1,-9.620884E-3,-3.717309E-3,-1.2116025E-2,4.403759E-3,-1.317208E-3,7.523454E-3,3.534184E-3,1.4049079E-5,4.449301E-3,-2.2817212E-3,-6.2029837E-3,1.0321692E-3,-1.819534E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,21,-1,23,25,27,29,-1,-1,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6283265E0,6.796731E-1,4.5553726E-1,2.2413254E-1,0E0,2.6650238E-1,1.2566847E-1,2.924673E-1,0E0,6.860485E-2,4.1539848E-2,1.6141653E-2,0E0,1.181159E-1,0E0,3.3634484E-2,6.0745515E-2,1.7477602E-2,4.1107986E-2,0E0,0E0,2.4359941E-2,0E0,0E0,1.5465637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,22,-1,24,26,28,30,-1,-1,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.363496E7,2.4722983E3,1.5791555E9,1E0,4.5021903E-3,1.1058695E0,2.797345E6,2.834008E-1,-1.4022321E-2,3.9219662E5,4.4E1,8.443332E-1,2.5597236E-3,3.842849E5,6.655365E-4,4.1E1,9.513889E-1,2.1987225E5,5.1E1,6.5516746E-3,1.2849381E-2,1.3578177E4,-9.620884E-3,-3.717309E-3,2.014695E6,4.403759E-3,-1.317208E-3,7.523454E-3,3.534184E-3,1.4049079E-5,4.449301E-3,-2.2817212E-3,-6.2029837E-3,1.0321692E-3,-1.819534E-3],"split_indices":[12,53,7,105,0,42,30,58,0,29,3,43,0,29,0,3,58,34,3,0,0,34,0,0,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,4.5E1,6.1E1,3.8E1,7E0,4.6E1,1.5E1,3.3E1,5E0,2.4E1,2.2E1,1.1E1,4E0,2.6E1,7E0,1.6E1,8E0,1.2E1,1E1,5E0,6E0,1.1E1,1.5E1,8E0,8E0,4E0,4E0,6E0,6E0,5E0,5E0,7E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7194271E-2,-9.72937E-2,1.8896739E-1,-2.0284453E-1,1.6372813E-2,1.5365627E-2,9.84426E-2,-2.4842085E-1,-6.0105067E-2,6.6005394E-2,-1.114531E-1,1.2767667E-1,4.831433E-4,-1.961372E-1,-1.6990492E-2,-5.9313225E-3,8.2670106E-4,-2.2887293E-2,1.4457154E-1,-1.8221717E-3,-5.0215493E-3,5.5879564E-3,1.1436479E-3,-1.0958742E-1,-1.0396929E-2,1.460551E-3,-6.257253E-2,7.2778035E-3,1.827394E-3,-6.322765E-3,-1.0955501E-3,-3.8567085E-3,-2.0959246E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7688091E0,9.2713606E-1,6.007501E-1,2.4529016E-1,2.4844103E-1,0E0,5.5484504E-2,2.56204E-1,9.888579E-2,1.988596E-1,1.1518404E-2,3.932637E-2,0E0,1.7511368E-1,0E0,0E0,0E0,3.7724E-2,7.014665E-2,0E0,0E0,0E0,0E0,7.259332E-2,0E0,0E0,2.198869E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,18,18,23,23,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,3.2936362E6,6.4417176E-2,5.127381E1,2.7260774E7,1.5365627E-2,8.442486E-1,1E0,6.444E3,1.1462246E3,5.661451E-1,2.1897722E10,4.831433E-4,4.178728E7,-1.6990492E-2,-5.9313225E-3,8.2670106E-4,2.4E1,1.4922581E1,-1.8221717E-3,-5.0215493E-3,5.5879564E-3,1.1436479E-3,9.7E1,-1.0396929E-2,1.460551E-3,2.83E2,7.2778035E-3,1.827394E-3,-6.322765E-3,-1.0955501E-3,-3.8567085E-3,-2.0959246E-4],"split_indices":[53,51,58,57,46,0,28,105,2,53,28,5,0,46,0,0,0,3,55,0,0,0,0,10,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.6E1,2.9E1,3.9E1,3.7E1,7E0,2.2E1,2.9E1,1E1,2.7E1,1E1,1.6E1,6E0,2.5E1,4E0,4E0,6E0,1.3E1,1.4E1,4E0,6E0,1.2E1,4E0,1.4E1,1.1E1,5E0,8E0,8E0,6E0,7E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.814685E-2,-7.426212E-2,1.5305519E-1,-1.5168595E-1,5.018264E-3,2.13539E-1,-5.187447E-3,-1.9152962E-1,1.1419349E-3,4.6743922E-2,-7.5811096E-2,1.53488E-2,1.2629546E-1,-1.550316E-1,-1.3174422E-2,-3.4991976E-2,1.2514096E-1,-3.5485455E-3,-1.0635336E-3,3.955607E-2,1.9900203E-1,-8.6807534E-2,-2.323222E-1,1.5362607E-3,-7.0264846E-2,6.1947E-3,2.609006E-4,2.6120916E-3,-4.1524476E-5,3.763078E-3,9.169147E-3,-5.7708244E-3,-1.9698124E-2,-1.0033611E-2,-3.7470725E-3,-3.7190807E-3,-6.106465E-4,9.7564876E-4,-2.5931012E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,21,-1,23,25,-1,-1,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0104992E0,4.8567823E-1,4.9732918E-1,2.9821712E-1,1.3845004E-1,3.5244358E-1,0E0,1.5225494E-1,0E0,1.7739096E-1,1.1691667E-2,0E0,9.933916E-2,1.32752E-1,0E0,4.128278E-2,7.5428426E-2,0E0,0E0,1.20978765E-2,2.0413637E-2,8.017501E-2,4.7340453E-2,0E0,1.5729092E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6295848E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,24,24,32,32],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,22,-1,24,26,-1,-1,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[4.492037E3,3.851071E7,9.302862E7,1.2599278E1,2.7260774E7,6.691E3,-5.187447E-3,8.8945635E-2,1.1419349E-3,1.1462246E3,4.1E1,1.53488E-2,3.3244818E7,4.8861527E9,-1.3174422E-2,2.4E1,1.5E1,-3.5485455E-3,-1.0635336E-3,2.2087644E9,1.5821141E0,1.66E2,3.159013E3,1.5362607E-3,3.670858E1,6.1947E-3,2.609006E-4,2.6120916E-3,-4.1524476E-5,3.763078E-3,9.169147E-3,-5.7708244E-3,1.13474E5,-1.0033611E-2,-3.7470725E-3,-3.7190807E-3,-6.106465E-4,9.7564876E-4,-2.5931012E-3],"split_indices":[53,12,46,54,46,9,0,39,0,53,3,0,52,5,0,3,8,0,0,12,40,0,4,0,59,0,0,0,0,0,0,0,30,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.8E1,2.5E1,3.9E1,3.9E1,2.1E1,4E0,3.2E1,7E0,2.6E1,1.3E1,5E0,1.6E1,2.8E1,4E0,1.3E1,1.3E1,8E0,5E0,8E0,8E0,1.6E1,1.2E1,4E0,9E0,9E0,4E0,4E0,4E0,4E0,4E0,7E0,9E0,8E0,4E0,5E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.737481E-3,-8.9211E-2,1.6433759E-1,-1.941062E-1,-9.074778E-3,2.2123896E-1,-4.01216E-3,-7.731144E-2,-2.4368058E-1,5.9842452E-2,-9.1321714E-2,1.3956001E-1,3.5705885E-1,-4.9284315E-3,-1.8872228E-4,-1.8668921E-1,-1.4445293E-2,1.2894379E-1,9.785402E-3,-5.6381626E-3,-4.4391062E-2,1.086708E-2,7.029671E-2,1.6602146E-2,5.1169656E-3,-7.6647694E-3,-2.8027901E-3,1.7613143E-3,6.291E-3,2.3403992E-3,-1.2315066E-3,4.2009E-4,-2.5483228E-3,6.3044066E-3,4.8197852E-3,2.5986042E-3,-2.18891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,25,-1,27,29,-1,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1],"loss_changes":[1.585721E0,6.202755E-1,6.0454416E-1,1.6644156E-1,2.4919885E-1,2.9407132E-1,0E0,4.6226688E-2,1.2144017E-1,8.175443E-2,5.3933844E-2,2.1366283E-1,1.7830575E-1,0E0,0E0,3.379804E-2,0E0,2.7816713E-2,3.845221E-2,0E0,1.9899528E-2,0E0,6.387175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3925844E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,20,20,22,22,33,33],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,26,-1,28,30,-1,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1],"split_conditions":[1.1855E4,1.04856936E8,1.650269E10,2.5107703E-1,2.8060134E7,1.1811385E4,-4.01216E-3,1.7593515E1,9.180692E7,8.66623E0,9.095E3,2.70482E5,3.813527E2,-4.9284315E-3,-1.8872228E-4,9.869215E-4,-1.4445293E-2,2.0195505E-1,1.2597537E7,-5.6381626E-3,1.1894881E6,1.086708E-2,1.5603712E2,1.6602146E-2,5.1169656E-3,-7.6647694E-3,-2.8027901E-3,1.7613143E-3,6.291E-3,2.3403992E-3,-1.2315066E-3,4.2009E-4,-2.5483228E-3,4.653145E9,4.8197852E-3,2.5986042E-3,-2.18891E-3],"split_indices":[2,7,5,28,46,53,0,59,7,54,9,30,57,0,0,40,0,39,33,0,49,0,59,0,0,0,0,0,0,0,0,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,7.3E1,3.6E1,3.1E1,4.2E1,3E1,6E0,1E1,2.1E1,2.3E1,1.9E1,2E1,1E1,5E0,5E0,1.7E1,4E0,9E0,1.4E1,7E0,1.2E1,5E0,1.5E1,6E0,4E0,1.3E1,4E0,4E0,5E0,6E0,8E0,4E0,8E0,8E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[5.060023E-3,-3.9900444E-2,1.6904347E-1,-1.4403671E-1,-2.902061E-3,1.5705138E-2,7.09969E-2,-1.0699426E-1,-8.122603E-3,5.0977856E-2,-7.2072595E-2,-4.707141E-3,1.3784634E-1,-1.8178518E-3,-5.1125013E-3,-9.907756E-3,1.2770411E-1,-1.289613E-1,3.4883092E-3,8.555458E-3,1.2231402E-3,-3.3133812E-3,1.47538865E-2,5.727273E-3,1.0081484E-3,-8.083487E-2,-7.0653395E-3,-1.5714758E-3,1.8053614E-3,3.6307182E-2,-1.1123942E-3,-4.3220604E-3,-3.695545E-4,8.1295395E-5,3.0158428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[7.8825873E-1,3.2322335E-1,5.688874E-1,4.1894823E-2,2.3846897E-1,0E0,2.6028833E-1,2.6200786E-2,0E0,1.6944015E-1,1.2080957E-1,0E0,1.3625526E-1,0E0,0E0,4.3753054E-2,4.4953912E-2,3.557062E-2,3.0608458E-2,0E0,0E0,0E0,1.7814184E-2,0E0,0E0,2.8968133E-2,0E0,0E0,0E0,1.9112017E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,16,16,17,17,18,18,22,22,25,25,29,29],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[4.3972437E3,7.5773336E7,6.4417176E-2,7.156922E7,2.727403E7,1.5705138E-2,9.040125E5,6.072427E-1,-8.122603E-3,1.6987854E1,5.127381E1,-4.707141E-3,3.077632E5,-1.8178518E-3,-5.1125013E-3,2.3616585E6,1.8634315E1,4.95E9,3.1676855E3,8.555458E-3,1.2231402E-3,-3.3133812E-3,1.499169E1,5.727273E-3,1.0081484E-3,2E0,-7.0653395E-3,-1.5714758E-3,1.8053614E-3,2.0195505E-1,-1.1123942E-3,-4.3220604E-3,-3.695545E-4,8.1295395E-5,3.0158428E-3],"split_indices":[53,7,58,7,46,0,33,28,0,57,57,0,29,0,0,49,55,5,53,0,0,0,55,0,0,8,0,0,0,39,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,2.1E1,6.2E1,5E0,1.7E1,1.6E1,5E0,3.5E1,2.7E1,4E0,1.3E1,7E0,9E0,2E1,1.5E1,1.5E1,1.2E1,6E0,7E0,4E0,1.6E1,1.1E1,4E0,1E1,5E0,6E0,6E0,1.1E1,5E0,6E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[9.323929E-3,-7.813383E-2,1.8397304E-1,-1.0277315E-1,9.485412E-3,2.4387056E-1,7.670746E-2,-1.3511458E-1,-2.7036062E-2,4.4523873E-3,2.813675E-1,1.5282334E-3,4.697767E-3,-1.0695489E-1,-1.1012138E-2,2.3927214E-3,-5.2890096E-2,1.204059E-2,3.6088598E-3,-7.6536015E-2,-6.6053187E-3,-3.0771093E-3,-7.8444736E-4,-1.20134674E-1,-2.815386E-2,-1.2017918E-3,-5.6531634E-3,6.4621645E-4,-1.6245084E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,-1,21,-1,-1,23,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[1.6029142E0,6.1491096E-1,2.0616627E-1,1.5801042E-1,0E0,6.474292E-2,2.0199351E-2,1.8746293E-1,5.416703E-2,0E0,1.1799741E-1,0E0,0E0,8.078575E-2,0E0,0E0,1.5281063E-2,0E0,0E0,6.2009066E-2,0E0,0E0,0E0,4.543622E-2,1.3215896E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,16,16,19,19,23,23,24,24],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,-1,22,-1,-1,24,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[7.90671E-1,6.082581E3,1.33556795E1,2.561146E3,9.485412E-3,3.840785E1,5.72143E5,1.01949844E6,3.09E2,4.4523873E-3,2.034139E6,1.5282334E-3,4.697767E-3,5.501383E9,-1.1012138E-2,2.3927214E-3,1.4E1,1.204059E-2,3.6088598E-3,2.91E2,-6.6053187E-3,-3.0771093E-3,-7.8444736E-4,9.122283E0,3.717E3,-1.2017918E-3,-5.6531634E-3,6.4621645E-4,-1.6245084E-3],"split_indices":[40,53,55,53,0,57,30,29,0,0,30,0,0,5,0,0,8,0,0,0,0,0,0,54,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,6.5E1,4E0,2.1E1,1.3E1,4.5E1,2E1,6E0,1.5E1,9E0,4E0,4E1,5E0,4E0,1.6E1,1.1E1,4E0,3E1,1E1,7E0,9E0,1.5E1,1.5E1,5E0,1E1,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.867907E-3,-6.967474E-2,1.558265E-1,-1.2791926E-1,3.3604927E-2,7.122294E-2,2.1767244E-1,1.9862214E-3,-1.8571019E-1,9.777716E-3,-1.3307461E-2,3.3243027E-2,3.966981E-3,1.3160989E-1,1.2062451E-2,4.2340755E-3,-5.5825025E-2,-1.08377844E-1,-2.2470771E-1,2.86648E-2,-7.051779E-3,2.6680941E-3,-5.077828E-4,6.3804924E-3,1.1948403E-3,-3.6459872E-3,-3.299267E-4,-1.6376357E-1,8.6896884E-4,-8.955007E-3,-4.9078837E-3,-3.211083E-2,8.198706E-2,-7.876448E-3,-3.2902192E-3,-2.5604675E-3,7.1524755E-6,4.625124E-3,1.5492659E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1663843E0,4.915521E-1,1.5193307E-1,3.9075166E-1,3.3746555E-1,1.8369898E-2,1.3640732E-1,1.1985446E-1,8.421445E-2,0E0,2.0422962E-1,1.8350948E-2,0E0,4.7491208E-2,0E0,0E0,2.4379428E-2,1.06641084E-1,1.3918042E-2,7.392795E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9629747E-2,0E0,0E0,0E0,1.3807188E-2,1.612898E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,19,19,27,27,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,2.561146E3,2.4221443E-1,5.1056176E-1,6.169213E0,1.0188E4,1.286E4,1.0526E4,4.178728E7,9.777716E-3,1.0990962E8,5E0,3.966981E-3,1.1180212E1,1.2062451E-2,4.2340755E-3,2.83E2,5.677647E2,1.1309524E-1,1E0,-7.051779E-3,2.6680941E-3,-5.077828E-4,6.3804924E-3,1.1948403E-3,-3.6459872E-3,-3.299267E-4,4.2E1,8.6896884E-4,-8.955007E-3,-4.9078837E-3,1.0837745E1,1.18631315E1,-7.876448E-3,-3.2902192E-3,-2.5604675E-3,7.1524755E-6,4.625124E-3,1.5492659E-3],"split_indices":[40,53,39,28,55,9,2,9,46,0,46,8,0,55,0,0,0,53,58,102,0,0,0,0,0,0,0,3,0,0,0,55,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,8E1,3.1E1,5.1E1,2.9E1,1.4E1,1.7E1,1.6E1,3.5E1,4E0,2.5E1,8E0,6E0,1.1E1,6E0,5E0,1.1E1,1.3E1,2.2E1,2.1E1,4E0,4E0,4E0,7E0,4E0,5E0,6E0,9E0,4E0,1.6E1,6E0,1E1,1.1E1,4E0,5E0,4E0,6E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.1503411E-2,-6.022531E-2,1.0210416E-1,1.1743105E-2,-1.8415894E-1,1.3324112E-1,-6.2689167E-3,-1.1934079E-1,5.8139388E-2,-2.0429802E-3,-2.1084705E-1,8.170475E-2,2.848334E-1,-1.7364284E-3,-6.5334076E-3,1.0161169E-1,-3.0271929E-2,-3.4372571E-3,-9.57698E-3,5.4357923E-3,4.3561555E-2,1.4179162E-2,3.5516846E-3,1.3515343E-1,2.3150533E-4,2.9081703E-4,-2.06386E-3,6.5976917E-3,3.8783143E-3,7.5871915E-2,8.184036E-3,1.1731047E-3,-1.8662412E-3,4.8724907E-3,5.049307E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,23,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1,-1],"loss_changes":[6.625155E-1,5.105631E-1,4.0949824E-1,2.319652E-1,5.956614E-2,2.9704094E-1,0E0,3.5200447E-2,1.1069242E-1,0E0,8.173025E-2,7.7488944E-2,1.6256648E-1,0E0,0E0,5.9803218E-2,1.0768166E-2,0E0,0E0,0E0,5.1224194E-2,0E0,0E0,5.876735E-2,0E0,0E0,0E0,2.432139E-2,0E0,3.4680277E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,15,15,16,16,20,20,23,23,27,27,29,29],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,24,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1,-1],"split_conditions":[1.9552084E3,4.307821E7,2.192772E10,1.92E2,5.3323513E-1,1.2768678E4,-6.2689167E-3,8.5959184E-1,2.70482E5,-2.0429802E-3,2.6078947E1,1.2187778E11,8.260109E9,-1.7364284E-3,-6.5334076E-3,4.3952936E7,2.1151997E5,-3.4372571E-3,-9.57698E-3,5.4357923E-3,2.3E1,1.4179162E-2,3.5516846E-3,1.5150703E3,2.3150533E-4,2.9081703E-4,-2.06386E-3,3.200296E9,3.8783143E-3,1.2410928E8,8.184036E-3,1.1731047E-3,-1.8662412E-3,4.8724907E-3,5.049307E-4],"split_indices":[53,46,5,0,28,53,0,28,30,0,57,32,5,0,0,33,29,0,0,0,8,0,0,4,0,0,0,12,0,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.6E1,4.4E1,3.6E1,2E1,4E1,4E0,9E0,2.7E1,4E0,1.6E1,3.1E1,9E0,5E0,4E0,1.8E1,9E0,6E0,1E1,1E1,2.1E1,5E0,4E0,1.3E1,5E0,4E0,5E0,1.4E1,7E0,9E0,4E0,1E1,4E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.697377E-3,-9.7170286E-2,8.702731E-2,-1.2952103E-1,5.0018826E-3,1.0664505E-2,1.9197288E-1,-1.9993305E-1,-4.4293467E-2,4.369449E-2,-5.8928872E-3,4.7823135E-3,9.264073E-3,-1.4409603E-1,-1.4644435E-2,-1.0872006E-1,4.8893443E-3,5.649267E-3,9.390396E-3,-8.207968E-2,-8.107118E-3,-5.162672E-3,-1.6550336E-3,2.0104758E-3,-1.4502563E-3,-3.0271173E-2,4.100922E-3,-4.444497E-3,5.9169147E-4,-4.6842344E-2,5.6154124E-4,-3.8674063E-4,-2.9311008E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,-1,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1],"loss_changes":[8.807678E-1,4.2972445E-1,3.9864957E-1,2.7919453E-1,0E0,1.7943832E-1,4.6776295E-2,2.4017239E-1,7.3411226E-2,9.912152E-2,0E0,0E0,0E0,9.134397E-2,0E0,1.499825E-2,3.4281798E-2,0E0,9.0225205E-2,5.8574945E-2,0E0,0E0,0E0,0E0,0E0,1.28785735E-2,0E0,0E0,0E0,1.402094E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,18,18,19,19,25,25,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,-1,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1],"split_conditions":[2.1915238E8,2.561146E3,1.3536367E0,2.667836E6,5.0018826E-3,8.637973E6,1.3127055E4,1.5755641E6,2.9544958E-1,1.2192353E7,-5.8928872E-3,4.7823135E-3,9.264073E-3,2.8225484E11,-1.4644435E-2,6.114E3,3.7E1,5.649267E-3,5.528107E3,3.845592E5,-8.107118E-3,-5.162672E-3,-1.6550336E-3,2.0104758E-3,-1.4502563E-3,4.599408E6,4.100922E-3,-4.444497E-3,5.9169147E-4,3.494164E7,5.6154124E-4,-3.8674063E-4,-2.9311008E-3],"split_indices":[7,53,40,33,0,48,4,33,42,46,0,0,0,32,0,2,3,0,53,33,0,0,0,0,0,1,0,0,0,46,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.3E1,4.9E1,4.7E1,6E0,2.9E1,2E1,2.5E1,2.2E1,2.5E1,4E0,1.2E1,8E0,2.1E1,4E0,9E0,1.3E1,5E0,2E1,1.3E1,8E0,5E0,4E0,6E0,7E0,1.5E1,5E0,9E0,4E0,1.1E1,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-4.237777E-3,-8.269796E-2,7.865978E-2,-1.0649972E-1,4.9593593E-3,2.8124793E-2,1.8699329E-1,-7.389247E-2,-9.9861985E-3,5.3282514E-2,-5.363919E-3,2.2256964E-1,2.0613405E-3,-1.5181887E-2,-1.7935653E-1,1.2531739E-2,1.1060278E-1,9.59934E-3,3.6133083E-3,-4.053344E-2,4.320899E-3,-3.6861477E-3,-7.78019E-3,5.7807863E-2,-3.3883635E-2,1.5514398E-1,9.154708E-4,-9.04947E-2,-3.6517316E-3,-5.268551E-4,3.655955E-3,-2.0437986E-3,1.7005613E-4,7.0945136E-3,2.9875794E-3,-4.632834E-3,-1.356904E-3,1.3300382E-3,-1.4166357E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.219622E-1,3.1319165E-1,2.90371E-1,2.6798433E-1,0E0,1.7683795E-1,6.5853715E-2,2.7591056E-1,0E0,7.8032695E-2,0E0,4.2365193E-2,0E0,1.0742183E-1,2.2673666E-2,4.609041E-2,4.9223155E-2,0E0,0E0,4.7578614E-2,0E0,0E0,0E0,3.608715E-2,9.914843E-3,1.0688394E-2,0E0,1.7193533E-2,2.4501283E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,19,19,23,23,24,24,25,25,27,27,28,28],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4707712E8,5.188755E3,1.7618678E0,1E0,4.9593593E-3,1.5939707E7,2.393487E5,4.352552E9,-9.9861985E-3,1.9649E4,-5.363919E-3,1.8613E4,2.0613405E-3,5.980516E5,7.056912E7,1.5E1,2.3152558E9,9.59934E-3,3.6133083E-3,2.83E2,4.320899E-3,-3.6861477E-3,-7.78019E-3,2.951724E1,4.1E1,1.2271712E1,9.154708E-4,1.85E2,1.0521739E1,-5.268551E-4,3.655955E-3,-2.0437986E-3,1.7005613E-4,7.0945136E-3,2.9875794E-3,-4.632834E-3,-1.356904E-3,1.3300382E-3,-1.4166357E-3],"split_indices":[7,4,40,105,0,48,34,5,0,2,0,9,0,29,7,8,7,0,0,0,0,0,0,57,3,55,0,10,54,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,5.1E1,5E0,3.7E1,1.6E1,4.4E1,7E0,3.3E1,4E0,1.2E1,4E0,2.9E1,1.5E1,2E1,1.3E1,8E0,4E0,2.5E1,4E0,6E0,9E0,1E1,1E1,8E0,5E0,1E1,1.5E1,4E0,6E0,6E0,4E0,4E0,4E0,5E0,5E0,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-5.985119E-3,-3.7334602E-2,2.798937E-1,-1.1924549E-1,3.852818E-2,1.2968256E-2,4.6480955E-3,-1.0174001E-2,-8.61623E-2,1.4390624E-1,-4.7238437E-3,-6.5295007E-3,-1.8414815E-1,6.496868E-3,1.5535147E-3,-8.2686484E-2,3.44543E-2,-6.916707E-2,9.163222E-2,-2.148222E-3,-7.867692E-3,-7.92647E-4,-3.9824406E-3,1.4682037E-2,3.2970735E-3,7.660684E-4,-3.8790284E-3,5.910601E-3,4.4155237E-5,-1.5095895E-3,5.1122166E-2,4.8930116E-4,2.7445937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1],"loss_changes":[9.06665E-1,5.700673E-1,7.3627055E-2,2.1235025E-1,2.1945485E-1,0E0,0E0,0E0,2.953073E-1,4.3165177E-2,1.0975401E-1,1.4160739E-1,5.8602095E-2,0E0,0E0,1.7707884E-2,2.5645847E-2,5.0714858E-2,5.845225E-2,0E0,0E0,0E0,0E0,4.125292E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0194946E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,23,23,30,30],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1],"split_conditions":[2.8937003E2,2.942787E6,2.95279E9,9.2433E4,9.366E4,1.2968256E-2,4.6480955E-3,-1.0174001E-2,2.4126838E11,7.6363635E-1,2.207872E-1,2.4557575E1,2.0161922E-1,6.496868E-3,1.5535147E-3,1.3919356E5,8.517173E3,5.9775025E-1,6.8E1,-2.148222E-3,-7.867692E-3,-7.92647E-4,-3.9824406E-3,1E0,3.2970735E-3,7.660684E-4,-3.8790284E-3,5.910601E-3,4.4155237E-5,-1.5095895E-3,5.3278846E-1,4.8930116E-4,2.7445937E-3],"split_indices":[57,52,7,1,30,0,0,0,32,58,40,57,28,0,0,34,4,28,10,0,0,0,0,102,0,0,0,0,0,0,28,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9E1,9E0,4.3E1,4.7E1,5E0,4E0,6E0,3.7E1,1.3E1,3.4E1,2.1E1,1.6E1,9E0,4E0,1.1E1,2.3E1,1.3E1,8E0,4E0,1.2E1,4E0,7E0,1.8E1,5E0,4E0,9E0,4E0,4E0,7E0,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.1812337E-2,-4.2948756E-2,1.2545536E-1,-1.0358797E-1,3.6788717E-2,1.3685177E-2,7.398414E-2,-9.595848E-3,-8.0722906E-2,1.05620585E-1,-5.7439685E-2,1.160214E-1,-3.3475934E-3,-4.8523424E-3,-4.3138888E-2,5.4669747E-4,1.5317461E-1,7.3793606E-4,-3.1632595E-3,5.3820442E-2,5.7059852E-3,1.9388814E-2,-7.952252E-2,6.6943145E-3,3.05605E-3,-1.1515613E-3,3.6343448E-3,-7.71638E-4,2.3259541E-3,-4.2981757E-3,-1.5848794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,-1,-1,21,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0861155E-1,3.761226E-1,4.5492786E-1,1.4173093E-1,2.26588E-1,0E0,2.3501164E-1,0E0,7.490218E-2,8.373228E-2,3.704451E-2,5.945933E-2,0E0,0E0,5.841647E-2,0E0,1.3032317E-2,0E0,0E0,5.1396437E-2,0E0,1.9734863E-2,1.730533E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,14,14,16,16,19,19,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,-1,-1,22,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,2.3514317E8,1.6451234E-2,4.570007E7,1.5046106E7,1.3685177E-2,1.650269E10,-9.595848E-3,1.9121604E-4,1.118632E3,3.9343938E-1,1E0,-3.3475934E-3,-4.8523424E-3,3.25E2,5.4669747E-4,2.326E4,7.3793606E-4,-3.1632595E-3,4.4E1,5.7059852E-3,1.33125E1,1.0613095E1,6.6943145E-3,3.05605E-3,-1.1515613E-3,3.6343448E-3,-7.71638E-4,2.3259541E-3,-4.2981757E-3,-1.5848794E-3],"split_indices":[2,7,58,5,46,0,5,0,39,53,28,102,0,0,0,0,9,0,0,3,0,55,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.6E1,3.6E1,4.3E1,3.3E1,5E0,3.1E1,4E0,3.9E1,1.9E1,1.4E1,2.5E1,6E0,1.5E1,2.4E1,7E0,1.2E1,4E0,1E1,1.1E1,1.4E1,9E0,1.5E1,7E0,5E0,4E0,7E0,5E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.9333752E-2,-1.7338527E-2,1.7851615E-1,3.232561E-2,-1.274609E-1,3.1809425E-1,-1.1914452E-2,-4.1906007E-2,1.1910658E-1,-1.8609008E-1,-5.4517087E-2,6.789022E-3,1.4265265E-2,3.2805654E-3,-4.054831E-3,-1.5610947E-2,-5.5327644E-3,1.4730425E-1,-1.2428112E-3,-2.8834061E-3,-7.964071E-3,-3.2274926E-3,3.8824248E-5,4.5374773E-2,-5.4154005E-2,1.8340005E-1,2.047027E-3,1.3910569E-4,3.409216E-3,-2.6057994E-3,1.2766357E-4,3.193395E-3,7.889118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,25,-1,-1,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9269094E-1,4.5805824E-1,5.088089E-1,3.78621E-1,1.01047456E-1,3.5895586E-2,1.0307721E-1,9.129667E-2,1.2195939E-1,3.3692688E-2,2.530377E-2,0E0,0E0,0E0,0E0,6.592278E-2,0E0,6.482148E-2,0E0,0E0,0E0,0E0,0E0,2.2041727E-2,1.7930765E-2,3.4020007E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,15,15,17,17,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,26,-1,-1,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.5081426E3,3.494164E7,8.50999E9,1.6987854E1,5.127381E1,2.2059325E5,7.096748E7,7.925698E5,2.8399336E7,2.6078947E1,9.74026E0,6.789022E-3,1.4265265E-2,3.2805654E-3,-4.054831E-3,3.1E1,-5.5327644E-3,1.4922581E1,-1.2428112E-3,-2.8834061E-3,-7.964071E-3,-3.2274926E-3,3.8824248E-5,1.5566606E1,3.95875E5,3.1346828E5,2.047027E-3,1.3910569E-4,3.409216E-3,-2.6057994E-3,1.2766357E-4,3.193395E-3,7.889118E-3],"split_indices":[53,46,5,57,57,29,46,29,46,57,54,0,0,0,0,3,0,55,0,0,0,0,0,55,1,29,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,8.2E1,1.8E1,5.7E1,2.5E1,1E1,8E0,3.1E1,2.6E1,1.3E1,1.2E1,5E0,5E0,4E0,4E0,2.6E1,5E0,2.2E1,4E0,4E0,9E0,7E0,5E0,1E1,1.6E1,1.5E1,7E0,6E0,4E0,1.2E1,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.561637E-3,-5.404985E-2,1.2403449E-1,-1.3783206E-1,1.10711185E-2,1.651183E-1,-5.31708E-3,-1.145633E-2,-1.00857444E-1,9.28129E-2,-4.8225813E-2,1.0642485E-1,1.1308259E-2,-1.2210609E-1,7.6926546E-4,5.9804716E-3,5.837162E-3,-4.2109964E-3,-2.0482969E-2,8.411137E-3,4.805522E-2,-1.4793777E-1,-1.3805073E-3,1.3471049E-3,-9.584615E-4,2.6229895E-3,-2.1830723E-3,-1.8132571E-3,9.3190506E-2,-2.3490298E-3,-1.7480707E-1,-7.038803E-4,1.5131987E-3,4.841804E-3,1.0801419E-3,-3.381597E-3,-7.7192737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,-1,21,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,31,-1,-1,33,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[7.356404E-1,3.998124E-1,4.032969E-1,1.8036765E-1,2.0805155E-1,2.2744715E-1,0E0,0E0,7.5285286E-2,1.06672496E-1,4.505991E-2,1.586996E-1,0E0,4.633802E-2,0E0,1.0163315E-2,0E0,0E0,1.7712517E-2,0E0,8.0090515E-2,2.8707564E-2,0E0,0E0,0E0,1.2115218E-2,0E0,0E0,2.705706E-2,0E0,2.1117866E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,15,15,18,18,20,20,21,21,25,25,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,-1,22,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,32,-1,-1,34,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,3.2334878E7,3.6181125E9,2.7571955E6,1.5046106E7,1.1811385E4,-5.31708E-3,-1.145633E-2,2.834008E-1,2.710413E5,7.898E3,2.70482E5,1.1308259E-2,3.159013E3,7.6926546E-4,3.8826916E-1,5.837162E-3,-4.2109964E-3,1.1787824E7,8.411137E-3,1.9649E4,6.059339E2,-1.3805073E-3,1.3471049E-3,-9.584615E-4,4.4178323E8,-2.1830723E-3,-1.8132571E-3,7.118011E-1,-2.3490298E-3,2.77E2,-7.038803E-4,1.5131987E-3,4.841804E-3,1.0801419E-3,-3.381597E-3,-7.7192737E-3],"split_indices":[2,12,7,46,46,53,0,0,58,29,9,30,0,4,0,28,0,0,51,0,2,34,0,0,0,12,0,0,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.2E1,3.3E1,3.1E1,4.1E1,2.9E1,4E0,4E0,2.7E1,1.7E1,2.4E1,2.2E1,7E0,2.3E1,4E0,8E0,9E0,6E0,1.8E1,6E0,1.6E1,1.7E1,6E0,4E0,4E0,1.2E1,6E0,5E0,1.1E1,5E0,1.2E1,8E0,4E0,6E0,5E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.1684482E-2,-8.92959E-2,4.733844E-2,-1.14849634E-1,5.3549916E-3,-5.167489E-2,8.901832E-2,-8.102203E-2,-1.0009397E-2,-7.7019674E-3,-6.255158E-3,5.3949416E-2,7.921108E-3,-1.3699213E-1,-2.659785E-2,-1.7146335E-3,2.9380014E-3,2.9502569E-2,4.477201E-3,-4.9719807E-2,-7.2869565E-3,3.1689203E-3,-6.0676824E-2,3.0852994E-3,1.0357463E-2,-3.169842E-3,-6.121934E-5,-3.8326145E-3,-2.2772465E-2,-1.8357523E-3,3.135E-2,-1.7745014E-3,4.850933E-4,1.9359319E-3,-6.650146E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,21,-1,-1,23,-1,25,-1,-1,27,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[4.9895924E-1,3.0242434E-1,2.604656E-1,2.1375608E-1,0E0,1.00247085E-1,1.8935111E-1,1.0906297E-1,0E0,5.7033602E-2,0E0,5.800236E-2,0E0,9.183115E-2,8.327968E-2,0E0,0E0,2.9110642E-2,0E0,1.6310014E-2,0E0,0E0,2.516535E-2,0E0,2.9860783E-2,0E0,0E0,0E0,1.0211804E-2,0E0,1.9832205E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,14,14,17,17,19,19,22,22,24,24,28,28,30,30],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,22,-1,-1,24,-1,26,-1,-1,28,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[2.870791E6,1.0098112E2,1E0,1E0,5.3549916E-3,5.9E1,1.5435694E9,1.4780533E-4,-1.0009397E-2,8.517173E3,-6.255158E-3,2.4624854E8,7.921108E-3,8.8824E4,5.739445E4,-1.7146335E-3,2.9380014E-3,3.162046E6,4.477201E-3,7.366E3,-7.2869565E-3,3.1689203E-3,1.94E2,3.0852994E-3,1.091E3,-3.169842E-3,-6.121934E-5,-3.8326145E-3,1.5285803E5,-1.8357523E-3,2.727403E7,-1.7745014E-3,4.850933E-4,1.9359319E-3,-6.650146E-4],"split_indices":[49,57,102,105,0,3,7,39,0,4,0,33,0,12,52,0,0,33,0,9,0,0,10,0,10,0,0,0,29,0,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,4.6E1,6.1E1,4.2E1,4E0,1.8E1,4.3E1,3.6E1,6E0,1.4E1,4E0,3.5E1,8E0,1.7E1,1.9E1,1E1,4E0,2.7E1,8E0,8E0,9E0,4E0,1.5E1,6E0,2.1E1,4E0,4E0,6E0,9E0,5E0,1.6E1,5E0,4E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[6.662455E-4,-4.9558103E-2,1.4785257E-1,-2.4057561E-1,-2.6587768E-2,2.1840347E-1,1.255075E-2,-3.0996343E-3,-1.2534267E-2,4.097304E-2,-9.346684E-2,1.2528654E-2,1.323453E-1,-2.1566772E-3,2.252529E-3,-3.8970392E-2,1.1886782E-1,-1.62801E-1,-5.253818E-2,1.8877957E-3,5.8528706E-3,-2.51236E-3,2.2326184E-4,1.6014557E-1,6.9791457E-4,-2.4208599E-3,-7.0182164E-3,-2.220887E-2,-4.260263E-3,2.6866517E-3,6.7511033E-3,-2.7233032E-3,-2.309643E-3,1.3026467E-3,-7.8897056E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1,33,-1,-1,-1],"loss_changes":[8.426613E-1,3.6656123E-1,2.7408314E-1,1.18638515E-1,3.5174674E-1,1.6438305E-1,4.335879E-2,0E0,0E0,2.474922E-1,1.0211915E-1,0E0,2.2506237E-2,0E0,0E0,2.808135E-2,7.9077154E-2,2.9314399E-2,4.900237E-2,0E0,0E0,0E0,0E0,1.831019E-2,0E0,0E0,0E0,1.5288202E-2,0E0,0E0,0E0,1.1181185E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,15,15,16,16,17,17,18,18,23,23,27,27,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1,34,-1,-1,-1],"split_conditions":[1.3137E4,5.7578644E2,2.797345E6,1.44E2,2.8399336E7,6.4417176E-2,1.7313588E7,-3.0996343E-3,-1.2534267E-2,1.6987854E1,1.0951729E3,1.2528654E-2,1.2943E4,-2.1566772E-3,2.252529E-3,1.0289962E3,1.8831E5,3.5E1,8.661788E7,1.8877957E-3,5.8528706E-3,-2.51236E-3,2.2326184E-4,1.7694729E3,6.9791457E-4,-2.4208599E-3,-7.0182164E-3,1.3163732E7,-4.260263E-3,2.6866517E-3,6.7511033E-3,9.985778E0,-2.309643E-3,1.3026467E-3,-7.8897056E-4],"split_indices":[2,4,30,0,46,58,1,0,0,57,34,0,9,0,0,4,30,3,46,0,0,0,0,4,0,0,0,51,0,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,8.4E1,2.8E1,8E0,7.6E1,1.8E1,1E1,4E0,4E0,3.8E1,3.8E1,6E0,1.2E1,4E0,6E0,1.9E1,1.9E1,1.3E1,2.5E1,4E0,8E0,1.1E1,8E0,1.3E1,6E0,4E0,9E0,1.8E1,7E0,4E0,9E0,1.3E1,5E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.530839E-3,-4.5932192E-2,1.1182062E-1,-1.1521912E-1,-8.350247E-3,2.2884932E-1,-2.427885E-2,-4.7911853E-2,-1.6587082E-1,3.1141045E-2,-6.1822448E-2,1.0794275E-2,2.0518857E-3,1.2534768E-3,-4.1092574E-3,-3.2648654E-3,5.245126E-5,-2.0179935E-1,-1.9370661E-3,-1.724491E-2,9.574152E-2,-5.9825747E-4,-3.811307E-3,-3.4863604E-3,-8.958482E-3,-5.2409623E-2,2.2318675E-3,2.1035306E-3,5.2006897E-3,-2.9059898E-3,-2.249932E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,-1,-1,23,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[4.5526692E-1,1.9954106E-1,3.8781723E-1,8.4101945E-2,1.0962121E-1,1.3984585E-1,6.734538E-2,2.7418276E-2,4.9649537E-2,9.542371E-2,4.18005E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.9676288E-2,0E0,5.3284883E-2,1.5143432E-2,0E0,0E0,0E0,0E0,1.6356282E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,17,17,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,-1,-1,24,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,3.572374E6,6.1358623E9,8.8824E4,2.7260774E7,1.343964E1,1.063036E8,2.69E2,2.1272728E1,1.118632E3,5.4357555E6,1.0794275E-2,2.0518857E-3,1.2534768E-3,-4.1092574E-3,-3.2648654E-3,5.245126E-5,5.432393E-1,-1.9370661E-3,1.8116838E1,3.6742297E5,-5.9825747E-4,-3.811307E-3,-3.4863604E-3,-8.958482E-3,2.1562665E10,2.2318675E-3,2.1035306E-3,5.2006897E-3,-2.9059898E-3,-2.249932E-4],"split_indices":[2,12,5,12,46,55,52,0,59,53,51,0,0,0,0,0,0,28,0,55,29,0,0,0,0,32,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,7.6E1,2.3E1,2.6E1,5E1,1.2E1,1.1E1,1.2E1,1.4E1,2.9E1,2.1E1,8E0,4E0,7E0,4E0,6E0,6E0,1E1,4E0,1.7E1,1.2E1,1.1E1,1E1,4E0,6E0,1.2E1,5E0,8E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.5239592E-2,-6.590716E-2,1.024054E-1,-1.0023185E-1,2.5494773E-2,1.6332898E-1,-3.6385134E-3,-7.927773E-2,-1.0460003E-2,9.016657E-2,-3.0522956E-2,7.439078E-2,2.6976025E-1,-2.9142814E-2,-1.471239E-1,1.1949413E-3,5.076637E-3,5.7299284E-4,-2.0110758E-3,1.0745387E-1,-4.775046E-4,1.2036726E-2,4.5747217E-3,-5.735839E-2,4.455354E-3,-3.456769E-3,-7.4006557E-3,5.036613E-3,1.4834374E-3,-9.322939E-2,-7.7397805E-3,-1.268817E-3,-4.0130746E-3,1.0774358E-3,-2.3945274E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[6.326317E-1,2.3471883E-1,4.1258442E-1,1.9144881E-1,7.8830265E-2,2.1657747E-1,0E0,1.6578552E-1,0E0,2.4136253E-2,1.4902991E-2,4.4874497E-2,5.7891726E-2,1.3620666E-1,4.2802185E-2,0E0,0E0,0E0,0E0,1.7666847E-2,0E0,0E0,0E0,4.5555018E-2,0E0,0E0,0E0,0E0,0E0,1.171273E-2,2.8401129E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,19,19,23,23,29,29,30,30],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.7249353E3,1.4112683E8,9.302862E7,8.674699E-1,1.8831E5,1.1681136E4,-3.6385134E-3,3.3012803E9,-1.0460003E-2,3.4E1,1.7108389E5,7.2266235E6,8.838004E0,6.444E3,7.444956E7,1.1949413E-3,5.076637E-3,5.7299284E-4,-2.0110758E-3,1.0007143E1,-4.775046E-4,1.2036726E-2,4.5747217E-3,1.2337662E-1,4.455354E-3,-3.456769E-3,-7.4006557E-3,5.036613E-3,1.4834374E-3,5.1612902E-2,3.727E3,-1.268817E-3,-4.0130746E-3,1.0774358E-3,-2.3945274E-3],"split_indices":[53,12,46,58,30,4,0,5,0,3,29,48,54,2,7,0,0,0,0,54,0,0,0,58,0,0,0,0,0,58,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,7.3E1,3.1E1,5.3E1,2E1,2.4E1,7E0,4.9E1,4E0,9E0,1.1E1,1.4E1,1E1,2.9E1,2E1,5E0,4E0,4E0,7E0,1E1,4E0,6E0,4E0,2.5E1,4E0,1.2E1,8E0,6E0,4E0,1.4E1,1.1E1,4E0,1E1,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8842183E-2,-4.7217514E-3,1.8946552E-1,-4.280184E-2,6.544878E-2,2.4351479E-1,1.4723232E-3,-6.7809536E-3,-1.3468155E-1,1.1246225E-1,3.279852E-4,4.6755387E-3,1.1651745E-2,-4.3688595E-2,1.0120317E-1,-7.6954566E-2,-6.5992386E-3,5.445598E-3,3.8187767E-4,1.6872303E-3,-1.9045807E-3,-9.187959E-2,-1.407047E-2,5.62055E-3,1.6945435E-4,-4.080795E-3,-9.201279E-4,-4.730798E-3,-1.1149936E-3,8.1509864E-4,-1.5682246E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6157434E-1,2.3527406E-1,1.3472217E-1,1.8773326E-1,9.487128E-2,7.561958E-2,0E0,1.7180988E-1,3.0367315E-2,6.87775E-2,3.6951836E-2,0E0,0E0,4.4381678E-2,5.7103172E-2,1.3830885E-2,0E0,0E0,0E0,0E0,0E0,2.461379E-2,2.35815E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,14,14,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.621782E3,6.402099E-1,9.299267E9,8.5154843E-1,2.5588E4,1.3772013E4,1.4723232E-3,4.3316594E5,9.067368E5,1.4922581E1,2.4E1,4.6755387E-3,1.1651745E-2,1E0,2.0477528E1,7.158023E7,-6.5992386E-3,5.445598E-3,3.8187767E-4,1.6872303E-3,-1.9045807E-3,3.0387878E3,3.3166495E6,5.62055E-3,1.6945435E-4,-4.080795E-3,-9.201279E-4,-4.730798E-3,-1.1149936E-3,8.1509864E-4,-1.5682246E-3],"split_indices":[53,43,12,28,9,4,0,29,33,55,3,0,0,17,59,7,0,0,0,0,0,4,33,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,8.6E1,1.7E1,5.6E1,3E1,1.2E1,5E0,4.1E1,1.5E1,1.7E1,1.3E1,6E0,6E0,3.1E1,1E1,8E0,7E0,1.2E1,5E0,7E0,6E0,1.1E1,2E1,6E0,4E0,4E0,4E0,6E0,5E0,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.5199473E-3,-2.00504E-2,1.7427467E-1,-1.5459003E-1,8.918726E-3,1.0275923E-2,-2.137881E-3,-2.296439E-3,-6.865453E-3,3.375763E-2,-5.6560256E-2,-3.2112934E-4,9.341152E-2,-1.1457382E-1,6.5951557E-3,-3.560842E-2,5.145497E-2,1.3627262E-1,-2.0416873E-4,-6.3188877E-3,-1.2646198E-3,1.8932971E-3,-1.4567714E-3,-3.0581884E-3,-7.04844E-3,8.38519E-4,3.728294E-3,2.2793598E-4,6.773822E-3,8.1962877E-4,-1.2967597E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2892936E-1,3.5215333E-1,3.5181886E-1,3.968811E-2,1.2389132E-1,0E0,0E0,0E0,0E0,1.1184118E-1,7.769693E-2,6.76056E-2,8.569278E-2,4.676324E-2,2.579118E-2,3.0252742E-2,1.9541252E-2,9.224725E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3699039E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3509795E3,4.1964554E-4,3.9858868E1,3.3109E4,8.63558E-1,1.0275923E-2,-2.137881E-3,-2.296439E-3,-6.865453E-3,3.1982497E5,1.020707E1,2.0699982E8,2.8399336E7,1.06993E-1,1.6E1,1E0,3.2736E4,1.688E3,-2.0416873E-4,-6.3188877E-3,-1.2646198E-3,1.8932971E-3,-1.4567714E-3,-3.0581884E-3,1.6872E5,8.38519E-4,3.728294E-3,2.2793598E-4,6.773822E-3,8.1962877E-4,-1.2967597E-3],"split_indices":[53,43,58,33,28,0,0,0,0,29,55,12,46,39,8,8,9,2,0,0,0,0,0,0,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,8.9E1,1.2E1,1.5E1,7.4E1,8E0,4E0,5E0,1E1,5.4E1,2E1,3.5E1,1.9E1,1E1,1E1,2.1E1,1.4E1,1.3E1,6E0,5E0,5E0,5E0,5E0,7E0,1.4E1,1E1,4E0,4E0,9E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.084436E-4,-3.0529765E-2,1.429765E-1,-1.0845123E-2,-1.2839371E-2,9.87389E-2,8.60141E-3,4.3266765E-3,-9.543071E-3,2.2865399E-3,4.8879883E-3,-1.3061572E-2,8.913715E-3,-1.0353696E-1,5.561586E-3,-1.5558228E-3,-4.6346034E-3,-1.9769596E-2,6.4358704E-2,2.68207E-3,-1.3542122E-3,4.7533456E-3,3.4518127E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,-1,13,-1,15,17,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[4.4434074E-1,3.876115E-1,5.233702E-2,0E0,3.3787614E-1,9.59219E-3,0E0,3.1534722E-1,0E0,0E0,0E0,1.1852147E-1,0E0,1.2480959E-2,8.914697E-2,0E0,0E0,7.286841E-2,6.512507E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,11,11,13,13,14,14,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,-1,14,-1,16,18,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.7123758E0,5.279518E7,9.683623E6,-1.0845123E-2,1.5081978E8,3.1346828E5,8.60141E-3,1.3127055E4,-9.543071E-3,2.2865399E-3,4.8879883E-3,6.7281544E-5,8.913715E-3,1.6278E4,1.1392E4,-1.5558228E-3,-4.6346034E-3,2.0477528E1,2.483304E0,2.68207E-3,-1.3542122E-3,4.7533456E-3,3.4518127E-4],"split_indices":[40,5,1,0,46,29,0,4,0,0,0,39,0,12,2,0,0,59,58,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,8.1E1,1.7E1,4E0,7.7E1,1.3E1,4E0,7.3E1,4E0,8E0,5E0,6.9E1,4E0,1.1E1,5.8E1,4E0,7E0,4.1E1,1.7E1,6E0,3.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.2409871E-2,-5.640977E-2,1.13069564E-1,-1.3070142E-1,9.130545E-3,5.9516463E-2,2.403144E-1,-9.858008E-3,-9.282424E-2,5.1835133E-3,-1.718563E-2,1.0855935E-1,-4.8438333E-2,1.14710005E-2,4.641702E-3,-1.2477392E-3,-1.167992E-1,-4.7894478E-2,5.2092396E-2,1.3644609E-1,2.4571328E-4,-3.8370572E-3,-7.3106407E-6,-4.8090126E-3,-1.7152494E-3,-6.056952E-2,2.640781E-4,3.30507E-3,-2.0470054E-4,5.9417128E-3,2.913899E-3,-6.979562E-4,-2.8729707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,19,21,-1,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.683883E-1,3.2255393E-1,2.9234338E-1,1.4180148E-1,1.2983838E-1,1.7997411E-1,7.1611345E-2,0E0,3.228198E-2,0E0,6.844584E-2,6.4335525E-2,3.0669386E-2,0E0,0E0,0E0,1.4129981E-2,1.5838135E-2,2.330812E-2,1.4938563E-2,0E0,0E0,0E0,0E0,0E0,1.2570299E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,11,11,12,12,16,16,17,17,18,18,19,19,25,25],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,20,22,-1,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,1.04856936E8,1.1803382E4,5.279518E7,1.8955729E1,6.19999E9,1.0889231E1,-9.858008E-3,3.566468E5,5.1835133E-3,4.664004E8,7.5305285E6,2.074635E-1,1.14710005E-2,4.641702E-3,-1.2477392E-3,8.0168776E-2,7.879123E7,8.63558E-1,1.343964E1,2.4571328E-4,-3.8370572E-3,-7.3106407E-6,-4.8090126E-3,-1.7152494E-3,2.1821466E5,2.640781E-4,3.30507E-3,-2.0470054E-4,5.9417128E-3,2.913899E-3,-6.979562E-4,-2.8729707E-3],"split_indices":[2,7,4,5,59,5,55,0,29,0,12,48,40,0,0,0,58,33,28,55,0,0,0,0,0,29,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,6.5E1,4.4E1,3E1,3.5E1,3.2E1,1.2E1,5E0,2.5E1,5E0,3E1,2.2E1,1E1,6E0,6E0,8E0,1.7E1,2.1E1,9E0,1.7E1,5E0,4E0,6E0,1.3E1,4E0,1.7E1,4E0,5E0,4E0,1E1,7E0,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.3187193E-2,-3.4000084E-2,1.1714888E-1,-5.2852556E-2,8.351912E-2,1.09519195E-2,7.9743795E-2,-7.7567585E-3,-4.0347572E-2,4.412856E-3,1.761286E-4,1.5121083E-1,1.9125398E-2,-9.946081E-3,-1.1433393E-1,2.8071261E-3,7.4288077E-3,-2.3928876E-3,5.4361384E-2,-2.4444059E-2,4.5282017E-3,-5.2205385E-3,-1.3526303E-3,2.845917E-3,3.1054555E-4,1.2113955E-2,-5.061873E-2,-8.5555745E-4,1.8158134E-3,-3.2026474E-3,-1.18605E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,-1,15,17,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1],"loss_changes":[5.9756905E-1,1.905185E-1,2.404775E-1,1.3921364E-1,3.5934314E-2,0E0,1.4007121E-1,0E0,1.5406421E-1,0E0,0E0,4.411444E-2,6.078022E-2,1.0119589E-1,4.111421E-2,0E0,0E0,0E0,1.499597E-2,4.463032E-2,0E0,0E0,0E0,0E0,0E0,2.853866E-2,1.535771E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,11,11,12,12,13,13,14,14,18,18,19,19,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,-1,16,18,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.8319664E0,1.6451234E-2,5.1114212E7,9.5032835E0,1.09519195E-2,1.5E1,-7.7567585E-3,5.501383E9,4.412856E-3,1.761286E-4,8.220836E8,1.9649E4,1E0,2.711196E3,2.8071261E-3,7.4288077E-3,-2.3928876E-3,8.106617E-1,1.4163358E7,4.5282017E-3,-5.2205385E-3,-1.3526303E-3,2.845917E-3,3.1054555E-4,1.4243386E1,7.48E2,-8.5555745E-4,1.8158134E-3,-3.2026474E-3,-1.18605E-3],"split_indices":[2,40,58,5,54,0,8,0,5,0,0,7,2,89,53,0,0,0,28,46,0,0,0,0,0,57,11,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.2E2,8.3E1,3.7E1,7.2E1,1.1E1,5E0,3.2E1,4E0,6.8E1,7E0,4E0,1.4E1,1.8E1,4.9E1,1.9E1,7E0,7E0,5E0,1.3E1,4.5E1,4E0,1.3E1,6E0,8E0,5E0,1.9E1,2.6E1,1E1,9E0,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.138847E-2,-1.6537586E-2,1.378826E-1,1.3055443E-3,-1.792534E-1,2.6409472E-3,7.540905E-3,-3.0678686E-2,1.1850852E-1,-1.8756153E-3,-9.773239E-3,7.5826794E-3,-6.916983E-2,8.814884E-3,5.13998E-2,-2.657893E-2,5.8283474E-2,-9.322858E-2,-4.540672E-4,5.1580014E-4,3.3329213E-3,-2.841676E-3,-4.7905644E-4,3.9274837E-3,2.4581605E-4,-4.93551E-3,-6.0926672E-2,1.3738719E-3,-8.502662E-4,-5.129459E-4,-3.2671015E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[3.9085332E-1,2.6443976E-1,7.662693E-2,3.1477338E-1,9.0713054E-2,0E0,0E0,9.7724244E-2,1.3923103E-1,0E0,0E0,6.0437817E-2,4.3758735E-2,0E0,1.842231E-2,2.8535325E-2,3.5653424E-2,2.3972481E-2,0E0,0E0,0E0,0E0,1.4231859E-2,0E0,0E0,0E0,1.9434478E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[2.0935743E0,1.1384717E8,4.4993465E1,1.1855E4,9.546808E0,2.6409472E-3,7.540905E-3,2.8399336E7,1.4994E4,-1.8756153E-3,-9.773239E-3,2.4662777E1,2.5E-1,8.814884E-3,4.3E1,1.92E2,1.4922581E1,2.1750195E3,-4.540672E-4,5.1580014E-4,3.3329213E-3,-2.841676E-3,3.1E1,3.9274837E-3,2.4581605E-4,-4.93551E-3,4.8861527E9,1.3738719E-3,-8.502662E-4,-5.129459E-4,-3.2671015E-3],"split_indices":[43,46,57,2,54,0,0,46,10,0,0,57,58,0,8,0,55,4,0,0,0,0,3,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,8.2E1,8E0,1.1E1,8E0,6.5E1,1.7E1,4E0,4E0,3.3E1,3.2E1,5E0,1.2E1,2E1,1.3E1,2.2E1,1E1,7E0,5E0,6E0,1.4E1,6E0,7E0,8E0,1.4E1,5E0,9E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-4.325153E-3,-6.465679E-2,5.7140812E-2,3.4588529E-3,-8.501403E-2,7.3060505E-2,-4.4419733E-3,-6.85497E-2,-7.2139436E-3,4.150803E-2,1.5799184E-1,-1.1530733E-1,-3.8210128E-2,-1.877802E-3,7.709088E-2,2.2758744E-3,7.3556066E-3,-1.5388815E-1,-1.1804346E-3,6.839154E-4,-5.6428324E-2,5.9329014E-4,-2.0374889E-3,1.4138842E-4,9.313434E-2,-2.7908136E-3,-7.1321437E-3,-9.8254375E-2,-1.6696407E-2,4.0959646E-3,1.2126976E-3,-4.4327853E-3,-1.5981115E-3,8.26462E-4,-1.5993807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,15,17,19,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2644554E-1,1.9760542E-1,1.7248571E-1,0E0,8.458957E-2,1.3547117E-1,0E0,6.22686E-2,0E0,6.189643E-2,4.797116E-2,5.2338034E-2,3.2305997E-2,1.9908579E-2,2.5462687E-2,0E0,0E0,2.4856001E-2,0E0,0E0,3.6581583E-2,0E0,0E0,0E0,1.7965809E-2,0E0,0E0,9.131268E-3,1.5220466E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,20,20,24,24,27,27,28,28],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,16,18,20,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3724032E8,8.977316E0,2.4692087E1,3.4588529E-3,6.2640357E-1,1.5791555E9,-4.4419733E-3,5.048584E-4,-7.2139436E-3,9.8309404E-1,2.4190365E6,8.670369E-1,1.0465239E-3,2.2087644E9,1.601247E1,2.2758744E-3,7.3556066E-3,3.08E2,-1.1804346E-3,6.839154E-4,2.8900872E6,5.9329014E-4,-2.0374889E-3,1.4138842E-4,2.3559767E5,-2.7908136E-3,-7.1321437E-3,4E0,5.661451E-1,4.0959646E-3,1.2126976E-3,-4.4327853E-3,-1.5981115E-3,8.26462E-4,-1.5993807E-3],"split_indices":[7,55,55,0,39,7,0,43,0,40,33,28,39,12,57,0,0,0,0,0,51,0,0,0,34,0,0,8,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.7E1,5.6E1,6E0,5.1E1,5.2E1,4E0,4.6E1,5E0,3.9E1,1.3E1,1.7E1,2.9E1,1.8E1,2.1E1,5E0,8E0,1.1E1,6E0,7E0,2.2E1,1.4E1,4E0,4E0,1.7E1,5E0,6E0,1E1,1.2E1,1.2E1,5E0,6E0,4E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8995876E-3,1.7307267E-2,-8.49541E-3,-1.9665277E-2,1.0299962E-1,-5.0792895E-2,5.0558295E-2,9.836733E-3,6.644898E-2,-6.829946E-2,2.3107193E-3,7.184726E-2,1.5602639E-4,-6.355511E-4,9.254501E-2,-4.8872404E-2,-5.313775E-3,6.652992E-4,3.8057142E-3,1.2834902E-1,1.5851419E-3,-3.2241517E-3,-1.1584569E-3,5.7846876E-3,2.095763E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[3.61006E-1,3.156837E-1,0E0,1.5558049E-1,1.6771504E-1,1.0284625E-1,2.134474E-2,0E0,5.865871E-2,5.8703825E-2,0E0,2.4953179E-2,0E0,0E0,2.9194072E-2,2.1565579E-2,0E0,0E0,0E0,1.5350193E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,14,14,15,15,19,19],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.4783066E1,-8.49541E-3,7.307441E-1,1.0822511E-2,1E0,4.3E1,9.836733E-3,1.9956842E5,1E0,2.3107193E-3,5.3278846E-1,1.5602639E-4,-6.355511E-4,5.8639234E11,1E0,-5.313775E-3,6.652992E-4,3.8057142E-3,1.2271712E1,1.5851419E-3,-3.2241517E-3,-1.1584569E-3,5.7846876E-3,2.095763E-3],"split_indices":[46,57,0,40,58,89,3,0,34,84,0,28,0,0,32,8,0,0,0,55,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.8E1,5E0,6.9E1,2.9E1,4.8E1,2.1E1,4E0,2.5E1,4.2E1,6E0,1.4E1,7E0,6E0,1.9E1,3.5E1,7E0,6E0,8E0,1E1,9E0,9E0,2.6E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.299195E-3,-1.4052154E-2,1.7097886E-1,-7.453978E-2,3.1034399E-2,9.727058E-3,2.7355868E-3,-3.8559325E-2,-1.0469606E-2,8.58175E-2,-1.615867E-2,7.1115796E-3,-1.2693503E-1,2.869411E-3,1.344417E-1,-1.118793E-3,2.5316318E-3,4.2062886E-3,-2.4483189E-2,-1.6516506E-3,-5.701923E-3,2.0075378E-3,-1.2711645E-3,5.727348E-3,2.787558E-3,-6.625224E-2,3.2847042E-3,-3.2566807E-3,1.6193952E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[3.9381814E-1,2.7503112E-1,1.0143152E-1,3.221581E-1,1.513389E-1,0E0,0E0,1.5427823E-1,0E0,1.0851292E-1,4.3119304E-2,9.2717715E-2,2.5492847E-2,2.404739E-2,1.1293352E-2,0E0,0E0,0E0,1.0730756E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.751474E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,25,25],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[1.0825377E4,3.2316522E7,1.3309696E7,2.915E4,1.5046106E7,9.727058E-3,2.7355868E-3,4.352552E9,-1.0469606E-2,9.489487E2,3.9887E4,5.2287582E-2,3.240192E6,1.0875838E7,7.6363635E-1,-1.118793E-3,2.5316318E-3,4.2062886E-3,2.6666668E-1,-1.6516506E-3,-5.701923E-3,2.0075378E-3,-1.2711645E-3,5.727348E-3,2.787558E-3,1.1674918E3,3.2847042E-3,-3.2566807E-3,1.6193952E-4],"split_indices":[53,12,48,9,46,0,0,5,0,53,10,58,7,33,58,0,0,0,58,0,0,0,0,0,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,9.9E1,1.2E1,4.2E1,5.7E1,5E0,7E0,3.7E1,5E0,2.6E1,3.1E1,2.5E1,1.2E1,1E1,1.6E1,2.7E1,4E0,5E0,2E1,4E0,8E0,4E0,6E0,1E1,6E0,1.5E1,5E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.852963E-4,1.2467966E-2,-8.251737E-3,-7.0391726E-3,1.7803189E-1,-4.2995278E-2,5.208442E-2,9.643615E-3,2.1401076E-3,-5.983899E-2,3.2737432E-3,-4.583001E-4,7.522155E-2,-1.2944703E-2,-9.8201685E-2,-1.5455259E-3,2.433131E-3,9.761275E-2,1.33563E-4,-2.6628922E-3,1.014157E-2,-7.3017746E-2,-7.3823784E-3,4.3414882E-3,3.6716516E-4,2.971794E-3,-7.7672035E-4,-3.8003572E-3,-1.4499791E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2955632E-1,3.371398E-1,0E0,2.0206968E-1,1.00533515E-1,1.3826767E-1,4.379922E-2,0E0,0E0,9.407537E-2,0E0,3.7368078E-2,3.9675504E-2,3.584695E-2,6.268868E-2,0E0,0E0,3.6391586E-2,0E0,0E0,4.5179658E-2,2.225922E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,17,17,20,20,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.1730811E4,-8.251737E-3,6.402099E-1,8.50999E9,1E0,1.6511278E1,9.643615E-3,2.1401076E-3,7.059086E-1,3.2737432E-3,1.7618678E0,2.2606992E5,1.2179555E-1,1.4342E4,-1.5455259E-3,2.433131E-3,1.3021952E1,1.33563E-4,-2.6628922E-3,7.0843E5,8.0168776E-2,-7.3823784E-3,4.3414882E-3,3.6716516E-4,2.971794E-3,-7.7672035E-4,-3.8003572E-3,-1.4499791E-3],"split_indices":[46,4,0,43,5,89,57,0,0,28,0,40,34,28,9,0,0,54,0,0,1,58,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,1.03E2,5E0,9.3E1,1E1,5.8E1,3.5E1,5E0,5E0,5.2E1,6E0,1.1E1,2.4E1,2.4E1,2.8E1,7E0,4E0,1.8E1,6E0,6E0,1.8E1,2.4E1,4E0,1.4E1,4E0,5E0,1.3E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[6.055449E-4,-1.9225232E-2,2.0281893E-1,-6.364392E-2,4.574536E-2,9.911284E-3,2.7512023E-3,-4.881323E-2,-7.942117E-3,9.781974E-2,-1.525634E-3,-1.08574174E-1,-1.0936733E-2,2.1899366E-3,5.270318E-3,-1.218257E-3,3.6689226E-2,-4.635602E-2,-6.0139713E-3,4.6248944E-3,-3.2319885E-2,2.4503567E-3,-2.1939763E-5,-2.736184E-3,3.612752E-4,1.9101058E-3,-5.0062187E-2,-3.055991E-3,-2.2767717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[4.4901463E-1,2.9727405E-1,6.987399E-2,1.3069914E-1,1.0353677E-1,0E0,0E0,1.2783408E-1,0E0,2.6939541E-2,2.9556045E-2,7.167393E-2,1.1117197E-1,0E0,0E0,0E0,1.283419E-2,2.0665012E-2,0E0,0E0,5.103659E-2,0E0,0E0,0E0,0E0,0E0,4.032433E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,16,16,17,17,20,20,26,26],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[1.0825377E4,3.4707712E8,2.9548872E9,6.2640357E-1,1.5046106E7,9.911284E-3,2.7512023E-3,1.4616883E1,-7.942117E-3,8.5731603E8,1.8489E4,3.9024E4,1.7593515E1,2.1899366E-3,5.270318E-3,-1.218257E-3,8.106617E-1,5.348407E0,-6.0139713E-3,4.6248944E-3,4.5479352E2,2.4503567E-3,-2.1939763E-5,-2.736184E-3,3.612752E-4,1.9101058E-3,3.7E1,-3.055991E-3,-2.2767717E-4],"split_indices":[53,7,7,39,46,0,0,59,0,5,10,12,59,0,0,0,28,59,0,0,53,0,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.01E2,9E0,6E1,4.1E1,5E0,4E0,5.6E1,4E0,1.9E1,2.2E1,2.1E1,3.5E1,1.2E1,7E0,1.2E1,1E1,1.1E1,1E1,4E0,3.1E1,5E0,5E0,7E0,4E0,5E0,2.6E1,1.4E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.9022836E-3,1.2229664E-2,-7.96542E-3,-5.760372E-3,1.8077672E-1,-1.9990418E-2,1.02964014E-1,9.545216E-3,2.2026303E-3,-4.9231723E-2,1.21620875E-2,1.1601846E-3,6.0423976E-3,-3.2194313E-2,-4.5111333E-3,7.429765E-2,-7.5407457E-3,-4.606831E-2,2.075508E-3,4.518162E-3,3.1010047E-4,-2.3640944E-2,1.6765204E-3,-2.268631E-3,-4.11228E-4,5.4941524E-4,-1.2737332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,-1,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1],"loss_changes":[3.0562162E-1,2.8303292E-1,0E0,1.3356629E-1,7.39952E-2,7.205254E-2,4.023634E-2,0E0,0E0,4.7580205E-2,4.585738E-2,0E0,0E0,4.4953216E-2,0E0,2.8995778E-2,2.6300408E-2,1.6730115E-2,0E0,0E0,0E0,1.09398505E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,15,15,16,16,17,17,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,-1,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825377E4,-7.96542E-3,2.1729739E0,6.691E3,7.33E3,3.6742297E5,9.545216E-3,2.2026303E-3,7.925698E5,1.216644E11,1.1601846E-3,6.0423976E-3,1E0,-4.5111333E-3,8.155485E-2,5.2E1,1.4731E4,2.075508E-3,4.518162E-3,3.1010047E-4,6.983E3,1.6765204E-3,-2.268631E-3,-4.11228E-4,5.4941524E-4,-1.2737332E-3],"split_indices":[46,53,0,40,9,2,29,0,0,29,32,0,0,101,0,39,8,9,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,9.2E1,5E0,8.4E1,8E0,7.5E1,9E0,4E0,4E0,3.9E1,3.6E1,5E0,4E0,3.3E1,6E0,8E0,2.8E1,2.9E1,4E0,4E0,4E0,2.2E1,6E0,1.9E1,1E1,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-5.688046E-3,4.129763E-3,-7.962236E-3,-1.5068855E-2,1.6217422E-1,-1.3067493E-1,8.668473E-4,8.916432E-3,3.2096936E-3,-5.2778964E-4,-1.0478153E-2,-1.00261845E-1,1.9732645E-2,-4.993728E-3,-1.965056E-3,5.618472E-3,5.6606433E-3,-3.927957E-2,2.614031E-2,-2.6951663E-3,-7.7076635E-4,2.9834446E-3,2.5671237E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,-1,-1,-1,17,19,21,-1,-1,-1,-1],"loss_changes":[2.4138209E-1,3.3620965E-1,0E0,1.8249744E-1,5.2601337E-2,2.179641E-1,1.6987129E-1,0E0,0E0,0E0,0E0,1.6891032E-2,1.425382E-1,0E0,0E0,0E0,6.4531654E-2,1.2359679E-2,5.10181E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,-1,-1,-1,18,20,22,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.669409E3,-7.962236E-3,3.0894287E2,7.7854166E0,1.1485E4,4.1964554E-4,8.916432E-3,3.2096936E-3,-5.2778964E-4,-1.0478153E-2,5.4432504E7,1.82E2,-4.993728E-3,-1.965056E-3,5.618472E-3,1.1674918E3,8.834262E4,1.5046106E7,-2.6951663E-3,-7.7076635E-4,2.9834446E-3,2.5671237E-4],"split_indices":[46,53,0,53,55,9,43,0,0,0,0,46,10,0,0,0,53,34,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,1.09E2,4E0,9.8E1,1.1E1,1.1E1,8.7E1,4E0,7E0,7E0,4E0,1.3E1,7.4E1,6E0,7E0,6E0,6.8E1,2.1E1,4.7E1,6E0,1.5E1,1.1E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.5438551E-3,1.9281605E-2,-1.6711915E-1,4.5982033E-2,-6.162308E-2,-9.372038E-3,-8.2055957E-4,2.5376797E-2,9.690412E-3,-3.176471E-2,-6.0917297E-3,-6.6602975E-2,4.268207E-2,-6.465919E-2,9.26382E-4,6.842585E-4,-5.092631E-3,8.374747E-2,1.2439377E-2,-3.4085168E-3,-9.2678075E-4,3.679606E-2,5.263811E-3,3.8492754E-3,-3.6719176E-3,-4.4132138E-4,2.5573298E-3,-1.368757E-3,1.0346461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,-1],"loss_changes":[3.1435934E-1,2.080516E-1,1.276218E-1,3.2317793E-1,7.139461E-2,0E0,0E0,1.0948901E-1,0E0,3.943786E-2,0E0,7.274461E-2,6.974814E-2,1.2656186E-2,0E0,0E0,0E0,6.4557835E-2,5.1981732E-2,0E0,0E0,2.5894187E-2,0E0,0E0,3.4296595E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,13,13,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,-1],"split_conditions":[1E0,6.784095E7,3E0,1.0825377E4,1.13808104E8,-9.372038E-3,-8.2055957E-4,2.0676967E4,9.690412E-3,1.1855E4,-6.0917297E-3,3.9024E4,1.5046106E7,8.3722024E7,9.26382E-4,6.842585E-4,-5.092631E-3,3.0765435E10,4.795186E10,-3.4085168E-3,-9.2678075E-4,2.028302E-1,5.263811E-3,3.8492754E-3,7.94E2,-4.4132138E-4,2.5573298E-3,-1.368757E-3,1.0346461E-3],"split_indices":[105,46,8,53,46,0,0,49,0,2,0,12,46,46,0,0,0,32,32,0,0,58,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.4E1,9E0,7.1E1,2.3E1,5E0,4E0,6.6E1,5E0,1.9E1,4E0,1E1,5.6E1,1.2E1,7E0,5E0,5E0,2.3E1,3.3E1,6E0,6E0,1.4E1,9E0,4E0,2.9E1,6E0,8E0,1.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.706806E-3,6.6160626E-4,-8.590499E-3,-2.2892008E-2,1.1591553E-1,-6.606376E-2,1.8957838E-2,8.783608E-3,1.9151013E-3,-4.0266957E-2,-9.731352E-3,-3.6894733E-3,7.422198E-2,1.895479E-2,-8.547305E-2,-2.3264242E-3,6.2277764E-3,4.1628326E-3,-8.226836E-4,-1.7624825E-2,4.5093615E-3,-1.0041023E-1,-1.2582879E-3,1.7482936E-3,-1.6172178E-2,-1.864533E-3,1.5167823E-3,-1.5103664E-3,-4.3222914E-3,-1.6289857E-3,5.5052835E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,23,-1,-1,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7230027E-1,3.039793E-1,0E0,1.6927843E-1,1.3925076E-1,2.3060688E-1,6.034989E-2,0E0,0E0,1.14061855E-1,0E0,2.1487534E-2,5.8422536E-2,7.583651E-2,1.4647439E-2,0E0,3.0084949E-2,0E0,0E0,3.260273E-2,0E0,1.5351996E-2,0E0,0E0,1.0827113E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,16,16,19,19,21,21,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,24,-1,-1,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.8027E4,-8.590499E-3,3.3209075E6,1.6451234E-2,7.650722E4,3.9219662E5,8.783608E-3,1.9151013E-3,3.925844E7,-9.731352E-3,4.475326E6,2.1212247E5,3E0,1.5225641E1,-2.3264242E-3,1.3710804E7,4.1628326E-3,-8.226836E-4,7.3434E4,4.5093615E-3,2.1129233E-1,-1.2582879E-3,1.7482936E-3,1.4342E4,-1.864533E-3,1.5167823E-3,-1.5103664E-3,-4.3222914E-3,-1.6289857E-3,5.5052835E-5],"split_indices":[5,2,0,51,58,34,29,0,0,46,0,46,34,8,55,0,46,0,0,30,0,28,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,1.1E2,4E0,9.2E1,1.8E1,4.5E1,4.7E1,5E0,1.3E1,4.1E1,4E0,3.4E1,1.3E1,1.8E1,2.3E1,4E0,3E1,9E0,4E0,1.4E1,4E0,1.7E1,6E0,1E1,2E1,9E0,5E0,5E0,1.2E1,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.736818E-3,4.113295E-3,-8.342385E-3,-1.0323528E-2,6.449642E-3,-7.903802E-2,1.0886561E-2,-2.5692797E-4,-9.862084E-2,-2.0650776E-2,5.631539E-2,-1.4796215E-3,-4.247032E-3,-4.948225E-2,1.4655777E-2,9.341702E-2,4.5907656E-3,-1.7335454E-4,-2.2867348E-3,-5.0949603E-3,2.342427E-3,6.286833E-4,1.2996759E-1,3.3477068E-2,-1.9531718E-3,-1.018831E-3,1.3773413E-3,6.3288775E-3,2.6190532E-3,2.1420955E-3,-1.5326959E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6382893E-1,2.6389223E-1,0E0,1.4223216E-1,0E0,3.169197E-2,1.0861577E-1,0E0,1.4907181E-2,4.6532497E-2,5.9062257E-2,0E0,0E0,1.5174277E-2,2.1162583E-2,4.774022E-2,2.5546132E-2,0E0,0E0,1.7077478E-2,0E0,0E0,1.8521264E-2,1.00053325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,19,19,22,22,23,23],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.1811385E4,-8.342385E-3,1.9121604E-4,6.449642E-3,1.56E2,3.290356E5,-2.5692797E-4,4.04E3,4.0291533E8,9.580646E0,-1.4796215E-3,-4.247032E-3,2.9777256E-3,2.3E1,1.6984678E3,1.6558E4,-1.7335454E-4,-2.2867348E-3,1.218666E0,2.342427E-3,6.286833E-4,1.4117E4,8.5154843E-1,-1.9531718E-3,-1.018831E-3,1.3773413E-3,6.3288775E-3,2.6190532E-3,2.1420955E-3,-1.5326959E-4],"split_indices":[5,53,0,39,0,0,29,0,2,7,54,0,0,39,8,4,9,0,0,40,0,0,9,28,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,1.03E2,4E0,9.6E1,7E0,2.2E1,7.4E1,5E0,1.7E1,4.4E1,3E1,5E0,1.2E1,2.4E1,2E1,1.7E1,1.3E1,6E0,1.8E1,1.5E1,5E0,6E0,1.1E1,9E0,4E0,1E1,5E0,5E0,6E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5940958E-3,-4.6554036E-2,6.830869E-2,-7.2021233E-3,-2.624325E-2,1.6035995E-1,3.6518317E-2,6.065297E-5,-1.11229755E-1,1.3220954E-3,8.153393E-3,-1.1043287E-2,7.03636E-2,-4.967899E-2,2.3270464E-2,-1.2433941E-3,-5.1097213E-3,1.3591817E-3,-1.4135528E-3,9.0586595E-2,2.9167102E-4,-3.290976E-4,-2.5705846E-3,7.671628E-2,-2.2112587E-2,4.0585003E-3,8.8264357E-4,4.894953E-3,1.4300863E-3,-2.1808026E-3,1.5399962E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,7,9,11,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3724707E-1,1.8424381E-1,1.22903034E-1,0E0,1.2295798E-1,8.0853134E-2,5.5508073E-2,5.0797306E-2,2.5375023E-2,0E0,0E0,2.2004453E-2,2.4459846E-2,1.15733E-2,7.446243E-2,0E0,0E0,0E0,0E0,1.8906705E-2,0E0,0E0,0E0,2.5208868E-2,1.7429084E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,19,19,23,23,24,24],"right_children":[2,4,6,-1,8,10,12,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.108166E3,9.687E3,1.6451234E-2,-7.2021233E-3,4.95E9,7.618855E1,1E0,1.1301075E1,2.7782312E1,1.3220954E-3,8.153393E-3,2.1486234E5,9.780346E9,1.433458E10,9.3152986E8,-1.2433941E-3,-5.1097213E-3,1.3591817E-3,-1.4135528E-3,1.0155914E1,2.9167102E-4,-3.290976E-4,-2.5705846E-3,5.83E2,2E0,4.0585003E-3,8.8264357E-4,4.894953E-3,1.4300863E-3,-2.1808026E-3,1.5399962E-4],"split_indices":[53,30,58,0,5,57,102,57,57,0,0,29,5,32,5,0,0,0,0,54,0,0,0,10,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6E1,4.3E1,6E0,5.4E1,1E1,3.3E1,4.2E1,1.2E1,4E0,6E0,1.4E1,1.9E1,1.3E1,2.9E1,4E0,8E0,5E0,9E0,1.4E1,5E0,5E0,8E0,1.3E1,1.6E1,1E1,4E0,4E0,9E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5897438E-4,-1.3053351E-2,6.461985E-3,-7.5818904E-2,2.809792E-3,-4.5627163E-4,-1.0400714E-1,-3.512621E-2,2.3239039E-2,-1.1999704E-3,-4.757195E-3,-4.7030836E-2,9.625173E-4,7.5231716E-2,-5.824817E-3,-8.361244E-4,-3.548405E-3,4.533259E-3,3.952262E-4,4.6399564E-2,-3.7876822E-2,-8.6584355E-4,2.9007045E-3,-1.1877371E-2,-3.6426443E-3,-2.1551596E-3,5.5181986E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,7,-1,9,11,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1],"loss_changes":[2.4026561E-1,9.3804985E-2,0E0,3.252372E-2,5.97166E-2,0E0,2.1159917E-2,2.1113586E-2,7.59936E-2,0E0,0E0,2.5864813E-2,0E0,5.6866713E-2,5.699604E-2,0E0,0E0,0E0,0E0,3.298611E-2,3.320931E-2,0E0,0E0,2.1693436E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,23,23],"right_children":[2,4,-1,6,8,-1,10,12,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1],"split_conditions":[1.0825377E4,4.1964554E-4,6.461985E-3,4.8861527E9,1.6511278E1,-4.5627163E-4,2.9916666E1,1.1789883E0,2.0752128E7,-1.1999704E-3,-4.757195E-3,2.6666668E-1,9.625173E-4,1.2320755E1,2.0086452E5,-8.361244E-4,-3.548405E-3,4.533259E-3,3.952262E-4,1.1653E4,7.331148E7,-8.6584355E-4,2.9007045E-3,1E0,-3.6426443E-3,-2.1551596E-3,5.5181986E-4],"split_indices":[53,43,0,5,57,0,57,58,46,0,0,58,0,55,29,0,0,0,0,2,46,0,0,102,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9.3E1,6E0,1.8E1,7.5E1,6E0,1.2E1,2.6E1,4.9E1,4E0,8E0,2.2E1,4E0,1.7E1,3.2E1,1.6E1,6E0,9E0,8E0,1.2E1,2E1,4E0,8E0,1.5E1,5E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-7.199455E-3,1.0465599E-2,-1.6016547E-1,3.932468E-2,-5.875473E-2,-8.316888E-3,-1.584643E-3,1.6268708E-2,7.876037E-3,-1.6845651E-2,-6.1411713E-3,3.0123955E-2,-4.315039E-2,-4.513322E-2,1.5212361E-3,3.5156552E-3,1.1132896E-2,-2.4072235E-4,-3.0902775E-3,-2.699677E-3,-4.8931065E-4,-2.5096811E-2,4.3182988E-2,-1.6545515E-3,8.6665107E-4,3.6377704E-3,8.1199093E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,19,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[2.610829E-1,1.7610475E-1,7.151261E-2,2.52083E-1,1.1846192E-1,0E0,0E0,4.7402777E-2,0E0,3.5270683E-2,0E0,5.7668686E-2,1.6463183E-2,1.2092693E-2,0E0,0E0,4.4046875E-2,0E0,0E0,0E0,0E0,1.942357E-2,2.1561876E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,13,13,16,16,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,20,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[1E0,6.9754904E7,3.9E1,8.974092E3,1.1245455E1,-8.316888E-3,-1.584643E-3,4.5843E7,7.876037E-3,1.1855E4,-6.1411713E-3,1.048576E8,6.25379E5,7.396E3,1.5212361E-3,3.5156552E-3,4.034396E8,-2.4072235E-4,-3.0902775E-3,-2.699677E-3,-4.8931065E-4,2.31308E5,1.3047E4,-1.6545515E-3,8.6665107E-4,3.6377704E-3,8.1199093E-4],"split_indices":[105,46,3,4,54,0,0,46,0,2,0,7,30,9,0,0,7,0,0,0,0,30,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,8.6E1,9E0,6.1E1,2.5E1,5E0,4E0,5.5E1,6E0,1.9E1,6E0,4.5E1,1E1,1.3E1,6E0,9E0,3.6E1,6E0,4E0,6E0,7E0,1.7E1,1.9E1,1.2E1,5E0,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.273087E-3,-7.470429E-3,7.443775E-3,3.6390252E-2,-5.0697304E-2,1.3581093E-2,1.6413313E-1,-2.5516236E-2,-6.191642E-3,5.562921E-2,-1.4437392E-2,8.375183E-3,2.2910745E-3,-5.694799E-2,3.046937E-2,-1.1643596E-2,1.1784481E-1,-2.5296456E-3,1.0377308E-2,-6.73674E-4,-2.9562493E-3,-6.682096E-4,2.390234E-3,9.113448E-4,-1.6820207E-3,1.8230348E-3,5.383556E-3,2.548729E-3,-1.5029071E-2,-1.8590839E-3,3.444982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,-1,3,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[2.9814023E-1,0E0,1.5163006E-1,1.7259336E-1,8.6881734E-2,6.206562E-2,4.6752453E-2,4.7488037E-2,0E0,8.926592E-2,4.490131E-2,0E0,0E0,1.4702745E-2,1.8728366E-2,1.5355412E-2,1.5637785E-2,0E0,3.624961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5723072E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,18,18,28,28],"right_children":[2,-1,4,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[9.7888E4,-7.470429E-3,1.0504551E1,1.0825377E4,2.192772E10,2.0752128E7,8E0,7.287E3,-6.191642E-3,1.6511278E1,6.604E3,8.375183E-3,2.2910745E-3,1.09141096E8,9.877E3,3.1E1,3.3052456E5,-2.5296456E-3,1.7994973E11,-6.73674E-4,-2.9562493E-3,-6.682096E-4,2.390234E-3,9.113448E-4,-1.6820207E-3,1.8230348E-3,5.383556E-3,2.548729E-3,1.6226676E0,-1.8590839E-3,3.444982E-4],"split_indices":[1,0,54,53,5,46,55,2,0,57,2,0,0,7,9,3,29,0,32,0,0,0,0,0,0,0,0,0,58,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6E0,8.8E1,5.9E1,2.9E1,5.1E1,8E0,2.5E1,4E0,2E1,3.1E1,4E0,4E0,1.6E1,9E0,1E1,1E1,9E0,2.2E1,7E0,9E0,4E0,5E0,5E0,5E0,4E0,6E0,6E0,1.6E1,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5544318E-2,8.2856574E-4,1.7569044E-1,2.3314485E-2,-5.16662E-2,8.75562E-3,2.6616931E-3,-1.423079E-2,5.7185426E-2,-9.980586E-2,-1.4028323E-2,3.732155E-2,-3.7265997E-2,8.990336E-2,1.340531E-2,-5.706309E-4,-4.8877364E-3,1.2460187E-3,-1.486541E-3,2.8164522E-3,-3.461824E-4,-3.1046993E-3,-5.865025E-4,1.3043243E-3,1.2722433E-1,3.3343535E-2,-1.48438E-3,5.527806E-3,2.1791419E-3,1.1838021E-4,2.4503965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1],"loss_changes":[2.5201818E-1,1.1806349E-1,4.3447137E-2,8.9832366E-2,5.2956127E-2,0E0,0E0,4.176647E-2,5.1793434E-2,3.6710635E-2,2.5140256E-2,2.1744628E-2,2.3186956E-2,3.7549675E-2,1.9850623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1530235E-2,1.327269E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,24,24,25,25],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1],"split_conditions":[1.0825377E4,4.3201213E9,1.055E3,1.1462246E3,3.0595828E3,8.75562E-3,2.6616931E-3,3.640909E6,2.0752128E7,2.4759493E1,5.9633E7,1.5662651E0,2.5975E4,3.6742297E5,1.4664377E-1,-5.706309E-4,-4.8877364E-3,1.2460187E-3,-1.486541E-3,2.8164522E-3,-3.461824E-4,-3.1046993E-3,-5.865025E-4,1.3043243E-3,7.408377E-1,2.7723257E3,-1.48438E-3,5.527806E-3,2.1791419E-3,1.1838021E-4,2.4503965E-3],"split_indices":[53,5,0,53,4,0,0,46,46,57,46,57,30,29,39,0,0,0,0,0,0,0,0,0,58,53,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,9.8E1,8E0,6.9E1,2.9E1,4E0,4E0,3.3E1,3.6E1,1.2E1,1.7E1,1E1,2.3E1,2E1,1.6E1,4E0,8E0,6E0,1.1E1,5E0,5E0,6E0,1.7E1,9E0,1.1E1,1.2E1,4E0,7E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.8123404E-3,-1.4526056E-1,5.97127E-3,1.5809156E-4,-9.597907E-3,1.5154353E-2,-6.6571943E-3,-6.059623E-4,1.2431084E-1,-9.94634E-2,1.5669389E-2,7.1533322E-3,1.181514E-3,-1.0294307E-3,-5.639434E-3,3.3276565E-2,-1.7589279E-2,-2.202198E-2,6.3199975E-2,-1.8199122E-3,1.361609E-3,-1.4930501E-3,1.4579241E-3,3.245211E-3,4.756501E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8983582E-1,1.6356577E-1,1.8984586E-1,0E0,0E0,1.7727041E-1,0E0,1.4797205E-1,8.141294E-2,4.7880545E-2,4.6847023E-2,0E0,0E0,0E0,0E0,8.743729E-2,5.2722573E-2,2.4593193E-2,4.4207484E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,10,10,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.477E3,2.16268E5,1.4974915E8,1.5809156E-4,-9.597907E-3,1.0825377E4,-6.6571943E-3,1.7145766E4,6.5566176E7,2.2367E4,3.925844E7,7.1533322E-3,1.181514E-3,-1.0294307E-3,-5.639434E-3,1.5493506E1,4.7308203E3,1.7618678E0,7.6363635E-1,-1.8199122E-3,1.361609E-3,-1.4930501E-3,1.4579241E-3,3.245211E-3,4.756501E-4],"split_indices":[2,12,46,0,0,53,0,52,46,33,46,0,0,0,0,57,4,40,58,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,8E0,1.06E2,4E0,4E0,1.02E2,4E0,9E1,1.2E1,1.2E1,7.8E1,6E0,6E0,6E0,6E0,5.1E1,2.7E1,1.8E1,3.3E1,1.7E1,1E1,1.4E1,4E0,2.1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.3294464E-2,-1.6678155E-4,-6.107358E-3,-6.838803E-3,1.0120793E-2,-2.4004694E-2,4.6296407E-2,-3.7772857E-2,3.6809193E-3,3.556834E-2,4.5131375E-3,-9.956454E-3,-9.088747E-2,5.878632E-2,1.9912338E-2,-3.7122454E-2,5.7555597E-2,-1.6427932E-3,-4.919216E-3,3.9683143E-3,6.1795994E-4,1.964694E-3,-1.5570889E-3,-2.056355E-3,-2.7524898E-4,3.552975E-5,3.7047437E-3,-1.0763321E-3,7.6549914E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,-1,-1,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1073212E-1,1.8882969E-1,0E0,0E0,1.146863E-1,8.7095074E-2,3.3552386E-2,6.410319E-2,0E0,1.3732959E-2,0E0,5.715758E-2,2.468478E-2,3.220789E-2,1.93466E-2,1.2438357E-2,2.2841193E-2,0E0,0E0,0E0,0E0,0E0,1.1564974E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,22,22],"right_children":[2,4,-1,-1,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2511909E8,2.7803582E6,-6.107358E-3,-6.838803E-3,3.4288502E8,3.7249353E3,1.2668315E6,2.8225484E11,3.6809193E-3,4.760274E-1,4.5131375E-3,4.899102E5,7.444956E7,7E0,2.2059325E5,4.757E3,1.68E2,-1.6427932E-3,-4.919216E-3,3.9683143E-3,6.1795994E-4,1.964694E-3,9.904E3,-2.056355E-3,-2.7524898E-4,3.552975E-5,3.7047437E-3,-1.0763321E-3,7.6549914E-4],"split_indices":[46,46,0,0,7,53,29,32,0,58,0,29,7,8,29,2,0,0,0,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.5E1,7E0,4E0,9.1E1,4.7E1,4.4E1,4.3E1,4E0,4E1,4E0,2.9E1,1.4E1,1.5E1,2.5E1,2.1E1,8E0,8E0,6E0,6E0,9E0,9E0,1.6E1,1.2E1,9E0,4E0,4E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.3286694E-2,-6.1373953E-3,-1.2186766E-3,1.1916811E-2,-1.10121354E-1,-1.8787432E-3,1.2555566E-1,-7.114069E-3,-6.9779414E-4,-7.865443E-2,1.7031323E-2,7.1104984E-3,1.0487871E-3,-1.0030704E-3,-3.9526024E-3,3.926016E-3,6.900742E-3,-3.933339E-3,3.5323286E-3,9.4712945E-4,-5.797085E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,-1,17,19,-1,-1,-1],"loss_changes":[1.8512852E-1,0E0,1.3291734E-1,1.303559E-1,7.511633E-2,1.1022709E-1,5.6288615E-2,0E0,0E0,2.0481043E-2,5.5925384E-2,0E0,0E0,0E0,0E0,0E0,5.555865E-2,1.907408E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,16,16,17,17],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,-1,18,20,-1,-1,-1],"split_conditions":[2.7571955E6,-6.1373953E-3,1.1384717E8,1.0861106E4,2.7140412E5,2.0676967E4,2.5669595E5,-7.114069E-3,-6.9779414E-4,5.895365E5,4.80378E5,7.1104984E-3,1.0487871E-3,-1.0030704E-3,-3.9526024E-3,3.926016E-3,2.4241872E0,2.8327732E6,3.5323286E-3,9.4712945E-4,-5.797085E-4],"split_indices":[46,0,46,4,33,49,34,0,0,29,12,0,0,0,0,0,40,33,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,6E0,9.1E1,8.2E1,9E0,7.4E1,8E0,4E0,5E0,1.4E1,6E1,4E0,4E0,6E0,8E0,5E0,5.5E1,5E1,5E0,1.4E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.8605918E-3,-2.1736171E-2,8.437174E-2,-6.7620915E-3,-1.111673E-2,6.9409097E-3,5.3182837E-2,1.4328168E-2,-4.2614054E-2,7.405703E-2,-8.7553845E-4,-1.8179422E-2,5.976905E-2,-6.996603E-2,-7.9285465E-3,4.372196E-3,1.5041559E-3,9.2035875E-4,-2.9263632E-2,8.864982E-2,3.2408815E-4,-3.7530188E-3,-1.5765788E-3,7.9274306E-4,-1.3334267E-3,-2.1656025E-3,-3.945525E-4,4.22959E-3,1.466527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,-1,7,-1,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1016249E-1,1.4560623E-1,7.0528194E-2,0E0,6.472628E-2,0E0,3.4123387E-2,6.7559384E-2,3.3599094E-2,1.933889E-2,0E0,1.3815888E-2,2.7075693E-2,1.333572E-2,1.5547715E-2,0E0,0E0,0E0,1.1736214E-2,1.1669204E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,18,18,19,19],"right_children":[2,4,6,-1,8,-1,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,1.369606E6,1.6451234E-2,-6.7620915E-3,2.8399336E7,6.9409097E-3,9.3393946E-1,1.6987854E1,1.2163482E8,9.275501E-1,-8.7553845E-4,6.910683E0,9.3977806E1,8.49491E5,5.4357555E6,4.372196E-3,1.5041559E-3,9.2035875E-4,2.5975E4,8.70707E0,3.2408815E-4,-3.7530188E-3,-1.5765788E-3,7.9274306E-4,-1.3334267E-3,-2.1656025E-3,-3.945525E-4,4.22959E-3,1.466527E-3],"split_indices":[2,46,58,0,46,0,28,57,7,58,0,59,59,1,51,0,0,0,30,54,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,8.3E1,2.3E1,4E0,7.9E1,4E0,1.9E1,4.4E1,3.5E1,1.5E1,4E0,2.6E1,1.8E1,1.9E1,1.6E1,5E0,1E1,5E0,2.1E1,1.1E1,7E0,7E0,1.2E1,8E0,8E0,7E0,1.4E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-8.72379E-3,-3.2157023E-2,6.567417E-2,-5.4606968E-3,-1.8477479E-2,1.06017515E-1,-4.220538E-3,1.6758474E-2,-5.2261718E-2,1.3683248E-1,2.2292566E-4,3.2988226E-3,4.576324E-3,-7.992488E-2,-1.09585095E-2,1.6422248E-1,1.5787827E-3,2.2053518E-3,-1.0813563E-2,-3.8940748E-3,-9.960631E-4,2.36622E-4,-1.833925E-3,3.5757283E-3,7.6861647E-3,-1.7838805E-3,2.6325219E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.816805E-1,1.259106E-1,1.9653776E-1,0E0,8.658606E-2,6.3764766E-2,0E0,3.23044E-2,4.1399516E-2,3.361982E-2,0E0,0E0,2.8489737E-2,2.918826E-2,1.149574E-2,1.6984403E-2,0E0,0E0,1.1619508E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8355938E3,9.7888E4,1.2707124E1,-5.4606968E-3,2.8399336E7,9.299267E9,-4.220538E-3,1.82E2,1.1309524E-1,1.343964E1,2.2292566E-4,3.2988226E-3,4.3650795E-2,9.08958E5,2.4835809E-1,1.1681136E4,1.5787827E-3,2.2053518E-3,6.4637297E9,-3.8940748E-3,-9.960631E-4,2.36622E-4,-1.833925E-3,3.5757283E-3,7.6861647E-3,-1.7838805E-3,2.6325219E-5],"split_indices":[53,1,54,0,46,12,0,10,58,55,0,0,58,30,42,4,0,0,32,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.8E1,2.4E1,7E0,7.1E1,2E1,4E0,3.5E1,3.6E1,1.5E1,5E0,4E0,3.1E1,2.1E1,1.5E1,1.1E1,4E0,6E0,2.5E1,1.3E1,8E0,1.1E1,4E0,6E0,5E0,5E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.9045455E-4,1.1739794E-2,-5.0104256E-3,2.1063041E-2,-6.0562724E-3,7.0360657E-3,1.2753847E-1,-2.1995079E-2,4.3521825E-2,7.092836E-3,1.348787E-3,2.283017E-2,-4.3062016E-2,1.8608687E-2,7.661435E-2,-1.0269417E-2,3.5273766E-3,-3.9992332E-3,-2.990283E-2,2.1972363E-3,3.4910275E-3,3.8514629E-3,3.1500537E-4,-1.3301228E-3,1.2383802E-3,-2.3676392E-4,-2.4372146E-3,1.796641E-3,-4.7961052E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6770124E-1,1.6920248E-1,0E0,1.4098781E-1,0E0,9.099125E-2,6.102948E-2,4.631099E-2,2.9902972E-2,0E0,0E0,4.065559E-2,2.609123E-2,1.4478652E-2,3.1727165E-2,1.5476296E-2,0E0,0E0,2.5015475E-2,0E0,1.4633759E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,18,18,20,20],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.5081978E8,-5.0104256E-3,1.1335518E4,-6.0562724E-3,5.8585525E-1,1.3477259E7,4.795186E10,3.1346828E5,7.092836E-3,1.348787E-3,9.144343E0,3.411305E2,1.6988876E-1,1.343964E1,1.0210356E1,3.5273766E-3,-3.9992332E-3,2.119303E6,2.1972363E-3,1.2606002E-1,3.8514629E-3,3.1500537E-4,-1.3301228E-3,1.2383802E-3,-2.3676392E-4,-2.4372146E-3,1.796641E-3,-4.7961052E-4],"split_indices":[105,46,0,4,0,43,48,32,29,0,0,57,34,39,55,54,0,0,33,0,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.8E1,7E0,9.4E1,4E0,8.4E1,1E1,4.7E1,3.7E1,5E0,5E0,1.5E1,3.2E1,2.2E1,1.5E1,1.1E1,4E0,4E0,2.8E1,5E0,1.7E1,1E1,5E0,7E0,4E0,1.8E1,1E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.4349855E-4,8.81773E-3,-5.9871273E-3,-2.0220012E-3,1.1308135E-1,-5.3416163E-2,1.1181911E-2,6.2670093E-3,6.447404E-4,7.6067676E-3,-4.159057E-3,7.13886E-2,2.2942477E-3,1.6861771E-3,-7.7285536E-4,-1.6962165E-4,4.4369227E-3,-2.8251627E-2,2.142086E-2,6.010141E-4,-3.92126E-2,5.6421056E-2,4.3734084E-3,-8.103488E-4,-2.5970933E-3,3.7789282E-3,6.493335E-4,5.017618E-4,-1.6618022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,-1,-1,-1,-1,19,21,-1,23,25,27,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7321569E-1,1.2002874E-1,0E0,6.642188E-2,5.4515243E-2,7.628062E-2,4.1573767E-2,0E0,0E0,1.3477924E-2,0E0,3.976794E-2,4.0917672E-2,0E0,0E0,0E0,0E0,1.3940422E-2,2.5409807E-2,0E0,1.0456149E-2,2.408043E-2,1.5041824E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,-1,-1,-1,-1,20,22,-1,24,26,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.3127055E4,-5.9871273E-3,1.1112019E3,8.50999E9,6.8917E4,1.53E2,6.2670093E-3,6.447404E-4,5.625738E2,-4.159057E-3,6.277045E0,2.2781514E8,1.6861771E-3,-7.7285536E-4,-1.6962165E-4,4.4369227E-3,8.1980075E5,4.760274E-1,6.010141E-4,8.63558E-1,7.84E3,9.014471E-1,-8.103488E-4,-2.5970933E-3,3.7789282E-3,6.493335E-4,5.017618E-4,-1.6618022E-3],"split_indices":[46,4,0,34,5,12,10,0,0,34,0,59,7,0,0,0,0,33,58,0,28,9,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.05E2,5E0,9.6E1,9E0,1.9E1,7.7E1,5E0,4E0,1E1,9E0,9E0,6.8E1,4E0,6E0,4E0,5E0,2.6E1,4.2E1,5E0,2.1E1,1.3E1,2.9E1,1.5E1,6E0,5E0,8E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.2598174E-3,1.4670459E-2,-1.386452E-1,2.5006693E-2,-4.663763E-3,-9.339003E-3,3.2908953E-4,1.2198944E-2,1.4745949E-1,-3.9475158E-2,2.7000148E-2,7.674119E-3,1.9085946E-3,-6.622676E-2,7.1005063E-4,5.454945E-3,6.2422793E-2,-7.4367505E-4,-3.2235263E-3,2.7724443E-2,-3.154511E-2,1.01069845E-1,-9.519478E-3,-7.5294264E-4,1.41495E-3,-1.7789672E-3,2.9685492E-5,4.9565737E-3,1.0727721E-3,-1.783428E-3,9.133871E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,-1,17,-1,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9439337E-1,1.5489066E-1,1.619811E-1,1.4876224E-1,0E0,0E0,0E0,6.837165E-2,4.4187605E-2,3.294565E-2,5.2319463E-2,0E0,0E0,1.2891479E-2,0E0,3.7129365E-2,7.325141E-2,0E0,0E0,1.6415996E-2,9.79618E-3,3.9857015E-2,1.5135152E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,-1,18,-1,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.8841052E10,2.667836E6,1.2768678E4,-4.663763E-3,-9.339003E-3,3.2908953E-4,1E0,9.9182825E0,4.7E1,3.290356E5,7.674119E-3,1.9085946E-3,1.5285803E5,7.1005063E-4,2.3365998E5,2.0752128E7,-7.4367505E-4,-3.2235263E-3,2.0511957E-1,7.22E2,9.275501E-1,3.598686E1,-7.5294264E-4,1.41495E-3,-1.7789672E-3,2.9685492E-5,4.9565737E-3,1.0727721E-3,-1.783428E-3,9.133871E-4],"split_indices":[105,5,33,53,0,0,0,17,55,3,29,0,0,29,0,29,46,0,0,40,0,58,57,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,1.01E2,8E0,9.5E1,6E0,4E0,4E0,8.7E1,8E0,1.9E1,6.8E1,4E0,4E0,1.3E1,6E0,4.3E1,2.5E1,5E0,8E0,2.7E1,1.6E1,1.6E1,9E0,5E0,2.2E1,1E1,6E0,1E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.5288215E-3,-6.140324E-3,8.652144E-3,1.9167883E-2,-4.613466E-3,4.772793E-3,1.3385132E-1,-6.674228E-2,1.9426296E-2,7.069505E-3,2.162803E-3,-3.366233E-3,-5.7304214E-4,7.623323E-2,1.0581485E-2,7.9137104E-4,4.1626757E-3,-2.1845449E-2,2.6141416E-2,-1.498703E-3,7.44017E-4,2.4439825E-3,1.5997738E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,15,17,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[2.0840193E-1,0E0,1.4018543E-1,1.4794022E-1,0E0,8.618021E-2,3.40883E-2,1.7390944E-2,3.3359714E-2,0E0,0E0,0E0,0E0,1.6569298E-2,3.0844722E-2,0E0,0E0,1.7553702E-2,3.6687523E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,17,17,18,18],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,16,18,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.477E3,-6.140324E-3,1.12782936E8,1.0825377E4,-4.613466E-3,5.981675E-4,4.2123712E5,2.2196926E-2,2.1923357E1,7.069505E-3,2.162803E-3,-3.366233E-3,-5.7304214E-4,3.2334878E7,1.1674918E3,7.9137104E-4,4.1626757E-3,5.407E3,9.275501E-1,-1.498703E-3,7.44017E-4,2.4439825E-3,1.5997738E-4],"split_indices":[2,0,46,53,0,43,29,58,59,0,0,0,0,12,53,0,0,2,58,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,6E0,9.5E1,8.9E1,6E0,8E1,9E0,1.3E1,6.7E1,4E0,5E0,8E0,5E0,8E0,5.9E1,4E0,4E0,1.9E1,4E1,1.3E1,6E0,1.3E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.739652E-3,6.9263284E-3,-4.405061E-3,-3.8693522E-3,4.7010025E-3,1.3269627E-3,-3.413561E-3,3.2038175E-2,-1.4311399E-2,2.8845035E-3,6.994032E-2,-2.5837254E-2,3.2984633E-2,-9.994091E-4,1.6085008E-3,4.070153E-3,3.83931E-2,-8.252682E-3,-1.6695434E-3,2.4891414E-3,-1.1246001E-3,-3.9452192E-4,2.8895128E-3,-9.2690036E-4,1.0629499E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,-1,5,-1,7,-1,9,11,13,15,17,19,-1,-1,-1,21,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2305258E-1,1.3226694E-1,0E0,4.3476637E-2,0E0,4.272254E-2,0E0,3.294786E-2,3.3014737E-2,2.4159394E-2,1.3223283E-2,1.6842786E-2,2.9823627E-2,0E0,0E0,0E0,1.9359013E-2,1.8486392E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,16,16,17,17],"right_children":[2,4,-1,6,-1,8,-1,10,12,14,16,18,20,-1,-1,-1,22,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8799975E10,1.0825377E4,-4.405061E-3,1.5088414E6,4.7010025E-3,1.5046106E7,-3.413561E-3,3.6742297E5,1.8027E4,2.0699982E8,1.82E2,1.7994973E11,2.4494735E9,-9.994091E-4,1.6085008E-3,4.070153E-3,3.97E2,5.3433334E1,-1.6695434E-3,2.4891414E-3,-1.1246001E-3,-3.9452192E-4,2.8895128E-3,-9.2690036E-4,1.0629499E-3],"split_indices":[5,53,0,29,0,46,0,29,2,12,10,32,7,0,0,0,0,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.8E1,7E0,9.1E1,7E0,8.7E1,4E0,2.9E1,5.8E1,1.7E1,1.2E1,4.7E1,1.1E1,1E1,7E0,4E0,8E0,2.6E1,2.1E1,7E0,4E0,4E0,4E0,1.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.6483202E-3,4.0761293E-3,-5.454882E-3,-1.9531498E-2,5.437144E-2,-9.483714E-3,-5.0616013E-3,6.2591457E-3,3.4693863E-2,1.9309424E-2,-5.747635E-2,6.914881E-2,-6.5053966E-5,-9.307056E-3,4.9574755E-2,-1.7867592E-4,-6.602468E-2,9.292712E-2,3.6346307E-4,1.7277653E-3,-2.987749E-2,4.009477E-3,2.6326647E-2,-1.4866267E-3,-3.7136124E-3,1.150766E-3,4.313193E-3,-1.6008106E-3,7.0349715E-4,1.5688151E-3,-5.4768653E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3728663E-1,1.3760753E-1,0E0,9.445317E-2,8.0623865E-2,1.03367366E-1,0E0,0E0,4.1620765E-2,4.1167498E-2,1.2245826E-2,2.2947013E-2,0E0,3.0826839E-2,2.9950999E-2,0E0,1.7731592E-2,1.5637465E-2,0E0,0E0,1.4428532E-2,0E0,1.3452238E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,20,20,22,22],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,4.818997E6,-5.454882E-3,6.2640357E-1,1.4440433E-2,4.178728E7,-5.0616013E-3,6.2591457E-3,7.639958E-1,1.6987854E1,2.4557575E1,9.780346E9,-6.5053966E-5,1.2410928E8,2.85E2,-1.7867592E-4,4.10362E5,7.4684826E8,3.6346307E-4,1.7277653E-3,9E0,4.009477E-3,4.7E1,-1.4866267E-3,-3.7136124E-3,1.150766E-3,4.313193E-3,-1.6008106E-3,7.0349715E-4,1.5688151E-3,-5.4768653E-4],"split_indices":[5,1,0,39,58,46,0,0,28,57,57,5,0,5,0,0,29,7,0,0,8,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,1.14E2,5E0,7.8E1,3.6E1,7.3E1,5E0,4E0,3.2E1,4.6E1,2.7E1,1.6E1,1.6E1,2.4E1,2.2E1,4E0,2.3E1,1.1E1,5E0,6E0,1.8E1,5E0,1.7E1,1.5E1,8E0,4E0,7E0,1.4E1,4E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.1603726E-3,1.5485994E-2,-6.3458327E-3,4.0161577E-3,1.2863575E-1,3.0635163E-2,-2.789822E-2,6.486663E-3,1.5042772E-3,7.0876986E-2,1.6491812E-2,-4.182692E-2,1.5052001E-2,1.00328155E-1,-7.9453544E-5,5.359031E-3,3.530967E-3,-2.1188335E-2,-7.1271345E-2,1.8600136E-3,-6.6271616E-4,1.1476396E-3,5.0805463E-3,-6.4179447E-4,2.1220494E-2,-2.0126458E-3,-4.8533773E-5,-7.362675E-4,-3.5117695E-3,1.4871164E-3,-1.5560671E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7482221E-1,1.4094457E-1,0E0,8.580757E-2,3.6898896E-2,3.0134596E-2,2.8148647E-2,0E0,0E0,3.0351937E-2,3.749491E-2,1.9759078E-2,1.5565556E-2,2.3213856E-2,0E0,1.4321999E-2,0E0,1.5012425E-2,1.6535476E-2,0E0,0E0,0E0,0E0,0E0,1.1904246E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[6.313997E-1,1.1811385E4,-6.3458327E-3,3.925844E7,3.0376984E1,2.083117E1,1.9649E4,6.486663E-3,1.5042772E-3,1.82E2,1.2028319E6,7.752181E7,1.6944988E9,1.16E2,-7.9453544E-5,2.3514317E8,3.530967E-3,9.371747E0,7.056912E7,1.8600136E-3,-6.6271616E-4,1.1476396E-3,5.0805463E-3,-6.4179447E-4,1.5046106E7,-2.0126458E-3,-4.8533773E-5,-7.362675E-4,-3.5117695E-3,1.4871164E-3,-1.5560671E-4],"split_indices":[39,53,0,46,58,59,2,0,0,10,29,46,7,0,0,7,0,54,7,0,0,0,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,1.08E2,4E0,9.9E1,9E0,5.4E1,4.5E1,5E0,4E0,1.3E1,4.1E1,3.4E1,1.1E1,9E0,4E0,3.7E1,4E0,2.1E1,1.3E1,5E0,6E0,4E0,5E0,1.5E1,2.2E1,7E0,1.4E1,5E0,8E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-8.785858E-3,-1.0762257E-3,-5.7649054E-3,1.0205673E-2,-1.1502917E-1,-5.704144E-4,8.119578E-2,-8.205172E-3,7.299546E-4,2.520764E-2,-2.697866E-2,5.2220975E-3,5.4919295E-4,-9.181918E-3,6.1775774E-2,-3.428799E-2,1.1207894E-3,1.398447E-3,-1.2320307E-3,3.1725105E-3,9.7081985E-4,-4.029546E-2,3.9776257E-4,-2.159745E-3,-6.148451E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[1.1965152E-1,1.2844892E-1,0E0,6.966092E-2,1.4121953E-1,5.514033E-2,4.803879E-2,0E0,0E0,5.2127264E-2,1.8028772E-2,0E0,0E0,2.761657E-2,1.5845947E-2,1.0242112E-2,0E0,0E0,0E0,0E0,0E0,1.35182515E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.4974915E8,1E0,-5.7649054E-3,1.0825377E4,1.04856936E8,2.727403E7,5.4875424E7,-8.205172E-3,7.299546E-4,1.1462246E3,3.8E1,5.2220975E-3,5.4919295E-4,2.8327732E6,9.366E4,2.022832E1,1.1207894E-3,1.398447E-3,-1.2320307E-3,3.1725105E-3,9.7081985E-4,9.08958E5,3.9776257E-4,-2.159745E-3,-6.148451E-4],"split_indices":[46,105,0,53,7,46,46,0,0,53,8,0,0,33,30,55,0,0,0,0,0,30,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.8E1,4E0,9E1,8E0,7.9E1,1.1E1,4E0,4E0,4E1,3.9E1,5E0,6E0,2.1E1,1.9E1,3.5E1,4E0,7E0,1.4E1,1E1,9E0,3.1E1,4E0,1.6E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.8353552E-3,4.223751E-3,-5.5867573E-3,1.25166355E-2,-5.0125904E-3,9.2093274E-4,7.112307E-2,2.7651932E-2,-2.9744381E-2,6.007968E-3,2.9002529E-2,1.5225844E-2,3.1166228E-3,-5.4415897E-2,4.8034373E-3,1.8822147E-3,-6.655829E-4,5.8158778E-2,-3.0645356E-3,-7.410822E-2,-2.082319E-4,-4.501191E-4,1.8846401E-3,4.3006588E-4,4.35119E-3,-4.699623E-4,1.7081059E-3,-6.844289E-4,-3.4481182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.20756365E-1,1.2987149E-1,0E0,6.993152E-2,0E0,7.213765E-2,6.3405104E-2,3.2952726E-2,3.520761E-2,0E0,1.5226593E-2,3.1588614E-2,0E0,2.256447E-2,1.5385991E-2,0E0,0E0,3.317311E-2,1.5156018E-2,1.8252775E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,6.313997E-1,-5.5867573E-3,1.3400168E7,-5.0125904E-3,3.925844E7,1.3453537E-1,2.4241872E0,3.4442917E3,6.007968E-3,2.5669595E5,8.01112E5,3.1166228E-3,1.1309524E-1,1E0,1.8822147E-3,-6.655829E-4,3.1259478E10,6.866558E5,2.4759493E1,-2.082319E-4,-4.501191E-4,1.8846401E-3,4.3006588E-4,4.35119E-3,-4.699623E-4,1.7081059E-3,-6.844289E-4,-3.4481182E-3],"split_indices":[46,39,0,1,0,46,58,40,4,0,34,33,0,58,89,0,0,32,29,57,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,1.07E2,4E0,1.02E2,5E0,8.6E1,1.6E1,4.6E1,4E1,4E0,1.2E1,3.9E1,7E0,2.3E1,1.7E1,8E0,4E0,1.1E1,2.8E1,1.6E1,7E0,1.3E1,4E0,7E0,4E0,2.4E1,4E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.20608555E-2,-8.557722E-3,5.3557336E-2,2.1239823E-2,-4.6016023E-2,1.1540856E-1,2.1647736E-2,3.8935628E-3,7.878925E-3,-3.1252787E-2,-3.6052517E-3,2.3051652E-3,5.939278E-3,-5.275031E-4,4.6000607E-2,-1.2518006E-3,2.4584552E-2,-7.370518E-4,-2.5459786E-3,3.3509156E-3,3.4946052E-4,2.684215E-3,8.426045E-3,7.491929E-4,-1.1805954E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[8.516258E-2,7.576045E-2,6.263205E-2,4.3121457E-2,2.072651E-2,1.7343491E-2,2.0981252E-2,0E0,2.5021566E-2,9.199277E-3,0E0,0E0,0E0,0E0,2.2906693E-2,0E0,1.947644E-2,0E0,0E0,0E0,0E0,0E0,1.0816756E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,14,14,16,16,22,22],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[3.062708E3,2.7260774E7,1.3453537E-1,4.4493478E2,7.705493E5,2.8129198E7,1.2532358E6,3.8935628E-3,4.7586722E2,1.2903092E7,-3.6052517E-3,2.3051652E-3,5.939278E-3,-5.275031E-4,1.2320755E1,-1.2518006E-3,3.854803E-2,-7.370518E-4,-2.5459786E-3,3.3509156E-3,3.4946052E-4,2.684215E-3,4.7E1,7.491929E-4,-1.1805954E-3],"split_indices":[53,46,58,4,29,46,33,0,53,33,0,0,0,0,55,0,39,0,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.6E1,3.2E1,3.7E1,2.9E1,1E1,2.2E1,4E0,3.3E1,2.4E1,5E0,6E0,4E0,9E0,1.3E1,9E0,2.4E1,2E1,4E0,5E0,8E0,5E0,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics index d5168e636..27abdd04d 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics @@ -1,109 +1,109 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,executorCPUTime_mean,0.12785757,3083.661435888062,5116.536518124452,79.01805054151625,529.5195832704212,1391.1708930540242,3098.8835876649596,46123.548214285714 -1,sr_remoteBytesReadRatio,0.092217326,0.821447639429395,1.190295307104319,4.361993847958805e-07,0.004681915012233263,0.2507003788881129,1.3747557349907953,9.945049508262114 -2,sw_recordsWritten_sum,0.08674005,1374410117.7790263,3387308853.036538,200.0,1055515.5,70762368.0,1083955333.0,32214909337.0 -3,duration_max,0.06980435,13014.565543071161,23588.431561357003,453.0,3392.5,6114.0,11157.5,238607.0 -4,input_bytesRead_mean,0.060718942,35669048.99371199,38136124.990772754,459755.9357142857,6486236.150910443,19828931.08045977,55778735.98521887,172540384.9351621 -5,sw_bytesWrittenRatio,0.05967704,0.8869646693077999,1.2280184306032964,5.108378528816216e-07,0.003869714259491267,0.35905628936826683,1.4713609319883258,11.3434662691877 -6,shuffle_read_bw,0.05236361,170230419.00411582,868425238.6185787,0.0,672086.8175217048,3162045.938095238,19115831.030837156,12002025920.666666 -7,peakExecutionMemory_max,0.051924855,625983011.6554307,1239878879.6041248,0.0,93695248.0,218169344.0,609353728.0,11609833472.0 -8,cache_hit_ratio,0.034012612,0.5818479081056788,0.3103653453688123,1.187602344157431e-05,0.3420876939265072,0.6498737129489489,0.8626540832583185,1.0 -9,duration_mean,0.03135956,4111.606057920529,5885.389553551322,165.25,1127.7932454448978,2285.394183483127,4375.702988070108,49986.05357142857 -10,jvmGCTime_mean,0.029379567,85.95293278190968,187.4390674747875,0.0,11.372529335653415,30.870352716873214,71.1075879639193,2096.9339285714286 -11,sr_totalBytesRead_mean,0.022102654,18229103.83876161,35775424.5056799,78.7411003236246,248444.1857191991,3494626.944915254,17084917.87147192,251271497.82890856 -12,sr_localBytesReadRatio,0.020691015,0.17759885752115132,0.29873276214277433,0.0,0.000931286144792052,0.06185394835231594,0.2359029041051135,2.2948561663133886 -13,input_recordsRead_sum,0.01932527,5493571789.872659,16697912537.18372,600000.0,279581211.0,1323566726.0,5501298942.5,230400506548.0 -14,scan_time,0.017291937,1429634.5767790263,6248258.313808985,157.0,54414.5,231308.0,883806.0,91377236.0 -15,duration_sum,0.016502932,4309284.955056179,8830994.729334807,661.0,276294.5,980699.0,3542482.0,74511844.0 -16,executorDeserializeTime_mean,0.016326765,15.251722354571783,17.94773710910694,3.6990291262135924,9.736833142773737,12.014332965821389,15.4860520606015,200.2536443148688 -17,shuffle_write_bw,0.015460291,118961.36635871192,98467.50188742377,142.17361451692005,4554.96867364436,115732.29374410023,210898.5361936413,364116.6899266902 -18,sr_remoteBytesRead_mean,0.01520188,14993982.41990812,30930608.85769257,67.23624595469256,179231.31632904644,2527588.171821306,13247257.400636984,219808445.37463126 -19,sr_fetchWaitTime_mean,0.013588414,24.41398828526225,163.0148145392446,0.0,0.04514407355772622,0.26666666666666666,1.5420825688073396,2439.190265486726 -20,data_size,0.0130872615,503941425933.2809,1845124535803.31,57436094.0,17447415178.0,77919859708.0,284879125360.5,22044104226233.0 -21,scan_bw,0.012340685,546795.340357651,924062.3721427768,10051.367316990334,158274.13560895674,299062.0855115105,677896.4525754678,12927459.114 -22,sw_writeTime_mean,0.012077854,80.2227292744631,132.46971341459806,0.026143790849673203,17.81384307846077,32.98275862068966,71.09646745948116,903.523598820059 -23,sr_totalBytesReadRatio,0.012053351,0.9990464969505464,1.365559384793236,5.108378528816216e-07,0.005347120858719093,0.3978628497915356,1.6787480002630113,11.365839535188458 -24,sr_localBlocksFetched_sum,0.011333223,7093.528089887641,13338.957317489889,0.0,179.0,1760.0,7417.0,94615.0 -25,executorDeserializeCPUTime_mean,0.009239109,10.04311471483636,5.154272519590641,3.2135922330097086,7.972676054468717,9.505882352941176,10.836784512438701,59.56851311953353 -26,duration_min,0.007569172,39.47191011235955,25.912558232588758,4.0,22.0,37.0,44.5,321.0 -27,sw_bytesWritten_mean,0.0071495394,16768197.572215984,33741236.27880933,65.79642190944548,81878.14984010742,2995362.978313253,15229625.137337428,251266600.89970502 -28,sqlOp_Window,0.006011271,0.07865168539325842,0.26969987919421534,0.0,0.0,0.0,0.0,1.0 -29,resultSize_max,0.005899458,103628.91760299625,322863.3680497398,5167.0,9168.5,13045.0,21886.5,2687556.0 -30,resultSerializationTime_sum,0.005048944,20.53932584269663,46.60529904197508,0.0,1.0,5.0,17.5,403.0 -31,numTasks_sum,0.0048986156,729.1985018726592,1003.3357980291394,4.0,238.5,404.0,913.5,11794.0 -32,sqlOp_Sort,0.0046998756,0.7265917602996255,0.44654563247974927,0.0,0.0,1.0,1.0,1.0 -33,sr_remoteBlocksFetched_sum,0.0042623663,49432.284644194755,93232.8358643141,3.0,1200.5,12270.0,51788.0,661803.0 -34,sqlOp_SubqueryBroadcast,0.003742783,0.7752808988764045,0.4181810501732485,0.0,1.0,1.0,1.0,1.0 -35,sqlOp_RunningWindowFunction,0.0026182493,0.06741573033707865,0.25121147708214225,0.0,0.0,0.0,0.0,1.0 -36,sr_localBytesRead_mean,0.0023243923,3235121.4188534864,8073494.560600611,0.0,37941.34757286536,512787.9633507853,2592251.0058443258,96841040.74163242 -37,sqlOp_Expand,0.0019313338,0.10861423220973783,0.3117389214311049,0.0,0.0,0.0,0.0,1.0 -38,executorRunTime_mean,0.0017111944,4084.432467240084,5886.28991028309,144.25,1108.6970838377952,2244.473941368078,4357.135471851895,49963.330357142855 -39,sqlOp_ObjectHashAggregate,0.0012390238,0.12734082397003746,0.33398039193838686,0.0,0.0,0.0,0.0,1.0 -40,numExecutors,0.00078163436,8.239700374531836,0.42770227167428976,8.0,8.0,8.0,8.0,9.0 -41,sqlOp_Subquery,0.00038553198,0.2247191011235955,0.4181810501732485,0.0,0.0,0.0,0.0,1.0 -42,sqlOp_AQEShuffleRead,0.00030097805,0.9176029962546817,0.2754849853147098,0.0,1.0,1.0,1.0,1.0 -43,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,maxOnHeapMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 -56,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_WindowGroupLimit,0.0,0.0149812734082397,0.1217058407722143,0.0,0.0,0.0,0.0,1.0 -59,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,executorCPUTime_mean,0.12423957,3083.661435888062,5116.536518124452,79.01805054151625,529.5195832704212,1391.1708930540242,3098.8835876649596,46123.548214285714 +1,sr_remoteBytesReadRatio,0.114727944,0.821447639429395,1.190295307104319,4.361993847958805e-07,0.004681915012233263,0.2507003788881129,1.3747557349907953,9.945049508262114 +2,sw_recordsWritten_sum,0.08057826,1374410117.7790263,3387308853.036538,200.0,1055515.5,70762368.0,1083955333.0,32214909337.0 +3,duration_max,0.070563,13014.565543071161,23588.431561357003,453.0,3392.5,6114.0,11157.5,238607.0 +4,input_bytesRead_mean,0.06338535,35669048.99371199,38136124.990772754,459755.9357142857,6486236.150910443,19828931.08045977,55778735.98521887,172540384.9351621 +5,peakExecutionMemory_max,0.056654345,625983011.6554307,1239878879.6041248,0.0,93695248.0,218169344.0,609353728.0,11609833472.0 +6,sw_bytesWrittenRatio,0.055294223,0.8869646693077999,1.2280184306032964,5.108378528816216e-07,0.003869714259491267,0.35905628936826683,1.4713609319883258,11.3434662691877 +7,shuffle_read_bw,0.042174872,170230419.00411582,868425238.6185787,0.0,672086.8175217048,3162045.938095238,19115831.030837156,12002025920.666666 +8,cache_hit_ratio,0.031942576,0.5818479081056788,0.3103653453688123,1.187602344157431e-05,0.3420876939265072,0.6498737129489489,0.8626540832583185,1.0 +9,jvmGCTime_mean,0.029369527,85.95293278190968,187.4390674747875,0.0,11.372529335653415,30.870352716873214,71.1075879639193,2096.9339285714286 +10,duration_mean,0.028666282,4111.606057920529,5885.389553551322,165.25,1127.7932454448978,2285.394183483127,4375.702988070108,49986.05357142857 +11,input_recordsRead_sum,0.023981718,5493571789.872659,16697912537.18372,600000.0,279581211.0,1323566726.0,5501298942.5,230400506548.0 +12,sr_totalBytesRead_mean,0.0236965,18229103.83876161,35775424.5056799,78.7411003236246,248444.1857191991,3494626.944915254,17084917.87147192,251271497.82890856 +13,scan_time,0.017897002,1429634.5767790263,6248258.313808985,157.0,54414.5,231308.0,883806.0,91377236.0 +14,executorDeserializeTime_mean,0.016180482,15.251722354571783,17.94773710910694,3.6990291262135924,9.736833142773737,12.014332965821389,15.4860520606015,200.2536443148688 +15,sr_localBytesReadRatio,0.0139266215,0.17759885752115132,0.29873276214277433,0.0,0.000931286144792052,0.06185394835231594,0.2359029041051135,2.2948561663133886 +16,data_size,0.013223694,503941425933.2809,1845124535803.31,57436094.0,17447415178.0,77919859708.0,284879125360.5,22044104226233.0 +17,sr_remoteBytesRead_mean,0.01320203,14993982.41990812,30930608.85769257,67.23624595469256,179231.31632904644,2527588.171821306,13247257.400636984,219808445.37463126 +18,shuffle_write_bw,0.011514859,118961.36635871192,98467.50188742377,142.17361451692005,4554.96867364436,115732.29374410023,210898.5361936413,364116.6899266902 +19,duration_sum,0.011256944,4309284.955056179,8830994.729334807,661.0,276294.5,980699.0,3542482.0,74511844.0 +20,sr_fetchWaitTime_mean,0.011137336,24.41398828526225,163.0148145392446,0.0,0.04514407355772622,0.26666666666666666,1.5420825688073396,2439.190265486726 +21,scan_bw,0.010432377,546795.340357651,924062.3721427768,10051.367316990334,158274.13560895674,299062.0855115105,677896.4525754678,12927459.114 +22,sw_bytesWritten_mean,0.009145087,16768197.572215984,33741236.27880933,65.79642190944548,81878.14984010742,2995362.978313253,15229625.137337428,251266600.89970502 +23,sr_localBlocksFetched_sum,0.008804374,7093.528089887641,13338.957317489889,0.0,179.0,1760.0,7417.0,94615.0 +24,resultSize_max,0.008223034,103628.91760299625,322863.3680497398,5167.0,9168.5,13045.0,21886.5,2687556.0 +25,executorDeserializeCPUTime_mean,0.008183868,10.04311471483636,5.154272519590641,3.2135922330097086,7.972676054468717,9.505882352941176,10.836784512438701,59.56851311953353 +26,sqlOp_Window,0.008081849,0.07865168539325842,0.26969987919421534,0.0,0.0,0.0,0.0,1.0 +27,sw_writeTime_mean,0.005321886,80.2227292744631,132.46971341459806,0.026143790849673203,17.81384307846077,32.98275862068966,71.09646745948116,903.523598820059 +28,numTasks_sum,0.00503938,729.1985018726592,1003.3357980291394,4.0,238.5,404.0,913.5,11794.0 +29,duration_min,0.004929467,39.47191011235955,25.912558232588758,4.0,22.0,37.0,44.5,321.0 +30,sqlOp_Sort,0.004546234,0.7265917602996255,0.44654563247974927,0.0,0.0,1.0,1.0,1.0 +31,resultSerializationTime_sum,0.004183445,20.53932584269663,46.60529904197508,0.0,1.0,5.0,17.5,403.0 +32,sqlOp_SubqueryBroadcast,0.0040676137,0.7752808988764045,0.4181810501732485,0.0,1.0,1.0,1.0,1.0 +33,sr_remoteBlocksFetched_sum,0.0038191557,49432.284644194755,93232.8358643141,3.0,1200.5,12270.0,51788.0,661803.0 +34,sr_totalBytesReadRatio,0.0029579804,0.9990464969505464,1.365559384793236,5.108378528816216e-07,0.005347120858719093,0.3978628497915356,1.6787480002630113,11.365839535188458 +35,sr_localBytesRead_mean,0.0024531633,3235121.4188534864,8073494.560600611,0.0,37941.34757286536,512787.9633507853,2592251.0058443258,96841040.74163242 +36,sqlOp_RunningWindowFunction,0.0021557377,0.06741573033707865,0.25121147708214225,0.0,0.0,0.0,0.0,1.0 +37,executorRunTime_mean,0.0016785505,4084.432467240084,5886.28991028309,144.25,1108.6970838377952,2244.473941368078,4357.135471851895,49963.330357142855 +38,numExecutors,0.000671226,8.239700374531836,0.42770227167428976,8.0,8.0,8.0,8.0,9.0 +39,sqlOp_ObjectHashAggregate,0.0005553033,0.12734082397003746,0.33398039193838686,0.0,0.0,0.0,0.0,1.0 +40,sqlOp_Subquery,0.00046715353,0.2247191011235955,0.4181810501732485,0.0,0.0,0.0,0.0,1.0 +41,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +42,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +43,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +44,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_HashAggregate,0.0,0.9887640449438202,0.10560054067615239,0.0,1.0,1.0,1.0,1.0 +48,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +52,sqlOp_Expand,0.0,0.10861423220973783,0.3117389214311049,0.0,0.0,0.0,0.0,1.0 +53,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +56,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_WindowGroupLimit,0.0,0.0149812734082397,0.1217058407722143,0.0,0.0,0.0,0.0,1.0 +58,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_SortMergeJoin,0.0,0.5617977528089888,0.4970981094018272,0.0,0.0,1.0,1.0,1.0 61,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 62,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 67,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 68,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 69,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,executorMemory,0.0,47016.0,0.0,47016.0,47016.0,47016.0,47016.0,47016.0 -79,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -82,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,maxMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 -84,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -85,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,sqlOp_SortMergeJoin,0.0,0.5617977528089888,0.4970981094018272,0.0,0.0,1.0,1.0,1.0 -87,diskBytesSpilledRatio,0.0,0.03193087515173895,0.2585787901798907,0.0,0.0,0.0,0.0,3.4683004089982026 -88,memoryBytesSpilledRatio,0.0,0.13140823253187295,0.9393631585625732,0.0,0.0,0.0,0.0,9.641198498202698 -89,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -90,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -92,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -94,diskBytesSpilled_mean,0.0,2880697.370667337,23636965.460199267,0.0,0.0,0.0,0.0,314481379.9144543 -95,memoryBytesSpilled_mean,0.0,11069065.0864877,79810143.77222423,0.0,0.0,0.0,0.0,874196883.2566372 -96,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_HashAggregate,0.0,0.9887640449438202,0.10560054067615239,0.0,1.0,1.0,1.0,1.0 -98,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -100,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -101,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -102,sqlOp_BroadcastNestedLoopJoin,0.0,0.04119850187265917,0.19912228909086188,0.0,0.0,0.0,0.0,1.0 -103,sqlOp_BroadcastHashJoin,0.0,0.947565543071161,0.22332010811379224,0.0,1.0,1.0,1.0,1.0 -104,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,platform_databricks-azure,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -106,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_BroadcastHashJoin,0.0,0.947565543071161,0.22332010811379224,0.0,1.0,1.0,1.0,1.0 +73,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,memoryBytesSpilled_mean,0.0,11069065.0864877,79810143.77222423,0.0,0.0,0.0,0.0,874196883.2566372 +76,diskBytesSpilled_mean,0.0,2880697.370667337,23636965.460199267,0.0,0.0,0.0,0.0,314481379.9144543 +77,maxMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 +78,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,maxOnHeapMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 +80,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +82,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,memoryBytesSpilledRatio,0.0,0.13140823253187295,0.9393631585625732,0.0,0.0,0.0,0.0,9.641198498202698 +84,diskBytesSpilledRatio,0.0,0.03193087515173895,0.2585787901798907,0.0,0.0,0.0,0.0,3.4683004089982026 +85,executorMemory,0.0,47016.0,0.0,47016.0,47016.0,47016.0,47016.0,47016.0 +86,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +90,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_BroadcastNestedLoopJoin,0.0,0.04119850187265917,0.19912228909086188,0.0,0.0,0.0,0.0,1.0 +94,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +95,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_AQEShuffleRead,0.0,0.9176029962546817,0.2754849853147098,0.0,1.0,1.0,1.0,1.0 +98,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +101,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,platform_databricks-azure,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +103,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +105,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +106,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg index ea2374289..24c179963 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.045395121","gamma":"0.0602301657","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.045395121","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"6","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.0602301657","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.665803671"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"100"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0427632518","gamma":"0.023735268","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0427632518","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.023735268","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.634272933"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json index ff33444cb..1aa1b696d 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_ObjectHashAggregate","sqlOp_Project","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-8.039039E-2,-5.380447E-1,2.1411616E-1,-1.9527984E-1,-7.6229006E-1,4.6708934E-2,4.2505676E-1,-2.9403624E-1,5.679849E-3,-9.005664E-1,-8.85219E-3,1.2529472E-1,-2.1817419E-1,2.5564346E-2,1.3887282E-2,-1.7234512E-2,-7.236476E-3,-2.60699E-2,-4.7858942E-2,1.8107329E-2,4.6703424E-2,-1.4608836E-2,-2.7232408E-3,1.5643975E-1,-3.073397E-3,2.3574838E-3,1.1374222E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,-1],"loss_changes":[1.103256E1,2.31077E0,1.7457948E0,4.797718E-1,1.3147917E0,6.310014E-1,1.8966842E-1,6.627071E-2,0E0,1.468172E-1,0E0,4.759083E-1,9.545508E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4912825E-1,0E0,0E0,8.535066E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,-1],"split_conditions":[1.1576994E-2,5.0415697E5,5.517496E-1,4.3646493E0,1.0735684E3,2.5042E4,3.0894554E0,8.25E2,5.679849E-3,9.29295E5,-8.85219E-3,2.9862975E5,2.9816154E1,2.5564346E-2,1.3887282E-2,-1.7234512E-2,-7.236476E-3,-2.60699E-2,-4.7858942E-2,1.8107329E-2,7.75024E0,-1.4608836E-2,-2.7232408E-3,2.3797054E0,-3.073397E-3,2.3574838E-3,1.1374222E-2],"split_indices":[50,40,54,66,64,9,66,0,0,40,0,44,68,0,0,0,0,0,0,0,69,0,0,69,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,3.1E1,4.9E1,1.3E1,1.8E1,2.8E1,2.1E1,1E1,3E0,1.4E1,4E0,2.2E1,6E0,8E0,1.3E1,5E0,5E0,6E0,8E0,4E0,1.8E1,3E0,3E0,9E0,9E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-6.5821044E-2,-4.8746797E-1,1.9907956E-1,-6.0051584E-1,2.9420324E-3,9.684171E-2,2.2594195E-2,-4.3532932E-1,-4.072492E-2,1.6123447E-1,-1.19265085E-2,-2.180903E-2,-7.144355E-3,-8.686349E-2,2.2182874E-1,-8.870369E-3,1.9698062E-3,1.7187353E-2,1.4413756E-1,6.648162E-4,1.075785E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,-1,15,17,-1,-1,-1,19,-1,-1],"loss_changes":[8.697524E0,1.911933E0,1.4173676E0,8.343363E-1,0E0,8.990438E-1,0E0,1.5001488E-1,0E0,5.0033486E-1,0E0,0E0,0E0,1.0744498E-1,2.621343E-1,0E0,0E0,0E0,2.2496104E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,13,13,14,14,18,18],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,-1,16,18,-1,-1,-1,20,-1,-1],"split_conditions":[1.0420895E-2,1.962943E3,9.306648E-1,1.2427474E8,2.9420324E-3,1.4181119E12,2.2594195E-2,1.137E3,-4.072492E-2,8.1729946E10,-1.19265085E-2,-2.180903E-2,-7.144355E-3,3.24971E5,2.325535E7,-8.870369E-3,1.9698062E-3,1.7187353E-2,1.381E3,6.648162E-4,1.075785E-2],"split_indices":[50,4,51,57,0,43,0,0,0,43,0,0,0,1,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.9E1,4.7E1,2.4E1,5E0,3.6E1,1.1E1,1.7E1,7E0,3.1E1,5E0,1.4E1,3E0,6E0,2.5E1,3E0,3E0,7E0,1.8E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.210696E-2,-3.3039266E-1,2.4782169E-1,6.978636E-2,-5.814457E-1,1.2769091E-1,4.9470505E-1,-9.370777E-2,1.3772721E-2,-7.0599675E-1,-1.3200934E-2,-7.4117025E-3,1.892729E-1,2.8984679E-2,1.1490692E-2,9.4135926E-4,-7.008387E-3,-1.4043725E-2,-3.784647E-2,2.6342523E-1,-2.1257795E-2,-1.6100534E-3,3.1003252E-1,-4.020618E-3,3.8596762E-3,1.7179081E-2,9.250938E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,19,-1,-1,-1,-1,-1,-1,21,23,-1,25,-1,-1,-1,-1],"loss_changes":[6.924413E0,3.456427E0,1.5087202E0,5.631451E-1,5.548978E-1,6.9386905E-1,4.2891312E-1,6.570062E-2,0E0,4.512596E-1,0E0,0E0,4.8955142E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.3139944E-1,7.1916506E-2,0E0,6.8228126E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,12,12,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,20,-1,-1,-1,-1,-1,-1,22,24,-1,26,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.81609E-1,6.2239337E-1,8.231683E2,1.500766E3,3.69E2,3.1198547E0,3.027892E5,1.3772721E-2,2.7288842E0,-1.3200934E-2,-7.4117025E-3,1.96E2,2.8984679E-2,1.1490692E-2,9.4135926E-4,-7.008387E-3,-1.4043725E-2,-3.784647E-2,2.0158982E0,3.327103E0,-1.6100534E-3,1.181508E6,-4.020618E-3,3.8596762E-3,1.7179081E-2,9.250938E-3],"split_indices":[54,39,51,64,4,0,66,40,0,66,0,0,8,0,0,0,0,0,0,65,65,0,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,3.3E1,5.2E1,1.3E1,2E1,3.6E1,1.6E1,8E0,5E0,1.3E1,7E0,6E0,3E1,9E0,7E0,3E0,5E0,4E0,9E0,2.2E1,8E0,3E0,1.9E1,5E0,3E0,1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.6712677E-2,-3.8655934E-1,2.6290038E-1,-3.5007633E-2,-6.2466246E-1,1.9734214E-1,4.317507E-2,-1.774812E-1,1.4250911E-2,-1.1876953E-2,-7.160449E-1,9.355879E-2,4.0559047E-1,-1.5833253E-2,-2.1893873E-3,-1.896389E-2,-3.6566727E-2,1.4194165E-1,-1.2738173E-2,2.2233933E-2,6.3208314E-3,-7.0550297E-3,1.9573508E-1,1.0539675E-2,3.125946E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,17,19,-1,-1,-1,-1,21,-1,-1,-1,-1,23,-1,-1],"loss_changes":[8.891812E0,3.0752368E0,2.0230331E0,8.5114795E-1,5.4249287E-1,9.5623815E-1,0E0,2.4887764E-1,0E0,0E0,1.1722469E-1,6.1913204E-1,2.6818752E-1,0E0,0E0,0E0,0E0,4.9429733E-1,0E0,0E0,0E0,0E0,9.9598706E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,11,11,12,12,17,17,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,18,20,-1,-1,-1,-1,22,-1,-1,-1,-1,24,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,7.6312915E3,8.231683E2,5.0415697E5,9.306648E-1,4.317507E-2,1.6361E4,1.4250911E-2,-1.1876953E-2,6.1358623E9,2.778E3,3.3934937E0,-1.5833253E-2,-2.1893873E-3,-1.896389E-2,-3.6566727E-2,4.9538263E2,-1.2738173E-2,2.2233933E-2,6.3208314E-3,-7.0550297E-3,1.3312784E7,1.0539675E-2,3.125946E-3],"split_indices":[54,39,64,64,40,51,0,9,0,0,5,0,66,0,0,0,0,4,0,0,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,3.6E1,4.8E1,1.5E1,2.1E1,4.5E1,3E0,1.1E1,4E0,5E0,1.6E1,3.1E1,1.4E1,4E0,7E0,5E0,1.1E1,2.8E1,3E0,1E1,4E0,4E0,2.4E1,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.457281E-3,-4.6151805E-1,1.768678E-1,-5.865223E-1,-4.3514595E-4,7.373939E-2,7.301515E-1,-1.8640812E-2,-3.650925E-2,1.2156692E-1,-1.729653E-2,4.2174112E-2,1.7487455E-2,1.0178648E-2,2.3251264E-1,-6.309016E-3,6.7619324E-2,2.9040247E-3,2.6462787E-1,1.3087609E-2,1.9347592E-4,1.500888E-2,6.0171164E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,-1,-1,-1,15,17,-1,19,-1,21,-1,-1,-1,-1],"loss_changes":[6.7117543E0,1.2937269E0,3.2555966E0,3.5908127E-1,0E0,1.1338248E0,2.5953436E-1,0E0,0E0,5.661037E-1,0E0,0E0,0E0,2.1502437E-1,1.07563734E-1,0E0,2.5044724E-1,0E0,1.2180042E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,13,13,14,14,16,16,18,18],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,-1,-1,-1,16,18,-1,20,-1,22,-1,-1,-1,-1],"split_conditions":[4.823969E-3,1.962943E3,4.0230347E3,9.1569895E-1,-4.3514595E-4,1.6597747E8,1.3946067E7,-1.8640812E-2,-3.650925E-2,3.839604E-1,-1.729653E-2,4.2174112E-2,1.7487455E-2,6.8177136E2,2.048724E0,-6.309016E-3,2.325535E7,2.9040247E-3,3.143662E0,1.3087609E-2,1.9347592E-4,1.500888E-2,6.0171164E-3],"split_indices":[51,4,64,39,0,57,59,0,0,54,0,0,0,4,65,0,57,0,66,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,2.2E1,5.7E1,1.7E1,5E0,4.9E1,8E0,1.1E1,6E0,4.5E1,4E0,4E0,4E0,2.3E1,2.2E1,6E0,1.7E1,4E0,1.8E1,3E0,1.4E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.3924711E-2,-4.2368004E-1,1.6192108E-1,-1.7530161E-1,-5.353477E-1,-9.608109E-3,3.1295002E-1,-1.27679445E-2,4.425406E-5,-2.9222198E-2,-1.2871777E-2,-1.0861849E-1,1.7128423E-1,2.1360101E-2,2.3424903E-1,4.6677E-3,-2.0658118E-1,1.5471225E-2,2.9556567E-3,1.3295392E-2,2.2704666E-3,-5.337116E-3,-1.5583547E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,-1,19,-1,21,-1,-1,-1,-1,-1,-1],"loss_changes":[5.379244E0,4.8860502E-1,1.609171E0,1.4970148E-1,1.9953632E-1,5.559112E-1,2.9905224E-1,0E0,0E0,0E0,0E0,4.3551058E-1,1.7580551E-1,0E0,2.4460089E-1,0E0,1.2999588E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,14,14,16,16],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,-1,20,-1,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9956966E-1,5.0415697E5,3.8151306E-1,1.3E1,1E0,1.1808436E3,4.511848E5,-1.27679445E-2,4.425406E-5,-2.9222198E-2,-1.2871777E-2,2.2519132E-1,5.77E2,2.1360101E-2,6.7584877E8,4.6677E-3,7.1974045E-1,1.5471225E-2,2.9556567E-3,1.3295392E-2,2.2704666E-3,-5.337116E-3,-1.5583547E-2],"split_indices":[69,40,54,3,23,64,44,0,0,0,0,39,0,0,7,0,39,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,2E1,6.1E1,7E0,1.3E1,2.9E1,3.2E1,4E0,3E0,8E0,5E0,1.9E1,1E1,9E0,2.3E1,6E0,1.3E1,3E0,7E0,1.7E1,6E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.9727616E-2,-3.923172E-1,1.7226477E-1,-4.9307355E-1,3.1331873E-3,1.02034435E-1,6.335652E-1,-3.483241E-1,-3.7619263E-2,1.5117201E-1,-2.56505E-1,3.699242E-2,1.6423292E-2,-1.9972168E-2,-8.204812E-3,-8.290564E-3,1.932549E-1,-1.7067494E-2,-3.309638E-3,3.8918972E-1,1.2418026E-1,8.737479E-3,2.1521866E-2,-5.379551E-3,7.285394E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,-1,-1,-1,-1,-1,19,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[5.3471622E0,1.1031339E0,1.9439859E0,6.344142E-1,0E0,1.0077534E0,9.9443674E-2,1.508354E-1,0E0,7.236171E-1,1.2613124E-1,0E0,0E0,0E0,0E0,0E0,5.4717183E-1,0E0,0E0,9.248185E-2,3.186087E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,16,16,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,-1,-1,-1,-1,-1,20,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[1.6405078E-2,1.962943E3,4.0230347E3,1.2427474E8,3.1331873E-3,1.4181119E12,5.0999485E9,1.648E3,-3.7619263E-2,1.543E3,2.8312179E12,3.699242E-2,1.6423292E-2,-1.9972168E-2,-8.204812E-3,-8.290564E-3,1.7329262E7,-1.7067494E-2,-3.309638E-3,6.498673E2,4.57563E5,8.737479E-3,2.1521866E-2,-5.379551E-3,7.285394E-3],"split_indices":[53,4,64,57,0,43,12,11,0,2,43,0,0,0,0,0,57,0,0,4,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.2E1,6.1E1,1.8E1,4E0,5.4E1,7E0,1.4E1,4E0,4.8E1,6E0,3E0,4E0,8E0,6E0,5E0,4.3E1,3E0,3E0,1E1,3.3E1,4E0,6E0,4E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[5.9039667E-2,-1.14112556E-1,3.784995E-1,-4.891154E-1,-1.5723949E-3,3.9820276E-2,2.835686E-1,-1.1356587E-2,-2.8328845E-2,7.1804956E-2,-1.4821477E-1,-1.8458208E-3,3.2963E-1,-1.7658219E-2,2.6263368E-1,-9.828386E-4,-9.810906E-3,1.7530927E-2,8.279392E-3,4.728852E-2,-1.3131324E-2,3.61504E-3,2.1402586E-2,-2.006978E-3,7.070392E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,17,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[4.4130907E0,2.1937807E0,1.0768156E0,2.3078227E-1,4.5181194E-1,0E0,3.86765E-1,0E0,0E0,4.826576E-1,1.1564621E-1,0E0,1.1277032E-1,3.6648148E-1,3.064106E-1,0E0,0E0,0E0,0E0,1.7560104E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,12,12,13,13,14,14,19,19],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,18,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[5.517496E-1,1.7363004E4,1E0,9.29295E5,5.81609E-1,3.9820276E-2,2.011628E0,-1.1356587E-2,-2.8328845E-2,1E0,1.5326E4,-1.8458208E-3,4.069474E0,9.71518E5,3.2311362E3,-9.828386E-4,-9.810906E-3,1.7530927E-2,8.279392E-3,1E0,-1.3131324E-2,3.61504E-3,2.1402586E-2,-2.006978E-3,7.070392E-3],"split_indices":[54,63,13,40,39,0,65,0,0,23,9,0,66,9,4,0,0,0,0,18,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,5.1E1,2.7E1,1.1E1,4E1,3E0,2.4E1,5E0,6E0,2.7E1,1.3E1,3E0,2.1E1,1.9E1,8E0,5E0,8E0,1.4E1,7E0,1.6E1,3E0,5E0,3E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.559397E-2,-2.9269102E-1,1.236124E-1,3.1023564E-2,-4.4124416E-1,1.6602427E-1,-8.800877E-3,-7.897527E-2,1.1043059E-2,-2.481859E-2,-2.2813775E-1,8.591894E-2,3.381116E-1,-8.233337E-3,8.5052213E-4,-2.9229359E-3,-1.3585888E-2,-6.4099794E-3,1.2629189E-1,1.9236414E-2,7.6067857E-3,1.1080994E-2,6.037112E-2,5.9624026E-3,-3.174122E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,-1,23,-1,-1],"loss_changes":[3.2973907E0,1.5525312E0,6.65049E-1,2.7602178E-1,3.697958E-1,5.499245E-1,0E0,8.3439484E-2,0E0,0E0,8.557606E-2,2.9292172E-1,1.1886895E-1,0E0,0E0,0E0,0E0,0E0,1.8718103E-1,0E0,0E0,0E0,1.7563635E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,11,11,12,12,18,18,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,-1,24,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,6.2012E4,8.231683E2,5.36E3,2.8364465E2,-8.800877E-3,1.3E1,1.1043059E-2,-2.481859E-2,2.6247186E8,2.074E3,1.1213404E6,-8.233337E-3,8.5052213E-4,-2.9229359E-3,-1.3585888E-2,-6.4099794E-3,3.4489678E6,1.9236414E-2,7.6067857E-3,1.1080994E-2,8.3E1,5.9624026E-3,-3.174122E-3],"split_indices":[54,39,10,64,2,70,0,3,0,0,7,2,44,0,0,0,0,0,60,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3.1E1,4.6E1,1E1,2.1E1,4.1E1,5E0,7E0,3E0,1.3E1,8E0,2.9E1,1.2E1,3E0,4E0,3E0,5E0,4E0,2.5E1,7E0,5E0,8E0,1.7E1,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.1990337E-3,-1.651639E-1,3.4353146E-1,1.7391844E-2,-3.8816187E-1,2.4685305E-1,3.5833992E-2,-1.2331864E-1,1.1023262E-1,-6.485262E-1,-1.7446148E-1,1.2870359E-2,4.6223737E-3,-1.6048856E-3,-1.0867539E-2,1.8390787E-1,-3.0763897E-3,-1.3923528E-2,-3.4100734E-2,-1.5766532E-4,-2.2077891E-1,1.2231507E-2,3.2728857E-3,-1.2189054E-2,-3.1831788E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,-1,-1,21,-1,-1,-1,-1,23,-1,-1,-1,-1],"loss_changes":[4.3412185E0,2.0829384E0,8.827369E-1,3.9244023E-1,1.1665766E0,8.811593E-2,0E0,1.14068165E-1,2.4852112E-1,1.2053442E-1,1.1010957E-1,0E0,0E0,0E0,0E0,1.0490519E-1,0E0,0E0,0E0,0E0,6.027341E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,15,15,20,20],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,-1,-1,22,-1,-1,-1,-1,24,-1,-1,-1,-1],"split_conditions":[3.3597556E-1,6.7376137E-1,7.6312915E3,4.146E3,1.0781229E3,8.0933E4,3.5833992E-2,6.433619E2,7.844101E0,2.202021E8,1.27597914E5,1.2870359E-2,4.6223737E-3,-1.6048856E-3,-1.0867539E-2,5.4600003E9,-3.0763897E-3,-1.3923528E-2,-3.4100734E-2,-1.5766532E-4,2.404913E-2,1.2231507E-2,3.2728857E-3,-1.2189054E-2,-3.1831788E-3],"split_indices":[51,39,64,2,4,9,0,4,69,7,45,0,0,0,0,5,0,0,0,0,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,5E1,2.4E1,2.8E1,2.2E1,2.1E1,3E0,1.1E1,1.7E1,9E0,1.3E1,1.6E1,5E0,7E0,4E0,1.2E1,5E0,3E0,6E0,3E0,1E1,6E0,6E0,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.6050934E-3,-3.4588704E-1,1.8281473E-1,-4.9191514E-1,4.9502403E-2,9.823562E-2,5.789709E-1,-3.3862367E-1,-3.0673996E-2,-1.8698768E-3,5.9147784E-3,1.424289E-1,-1.8727104E-1,3.8609654E-2,1.5483703E-2,-1.7859455E-2,-7.322303E-3,1.7206164E-1,-5.3199553E-3,-1.2776313E-2,-2.1007718E-3,7.431294E-2,2.4421686E-1,8.267169E-3,2.3765222E-4,3.5951093E-3,1.2551224E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[5.5269756E0,1.7564898E0,1.8865035E0,3.910246E-1,7.1313694E-2,6.4492905E-1,3.5587764E-1,7.257867E-2,0E0,0E0,0E0,3.5097545E-1,7.99226E-2,0E0,0E0,0E0,0E0,2.5692177E-1,0E0,0E0,0E0,1.330886E-1,9.526372E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,17,17,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,4.0230347E3,8.616169E-1,8.231683E2,1.1269586E12,1.3946067E7,1.5341808E3,-3.0673996E-2,-1.8698768E-3,5.9147784E-3,2.953641E5,2.8312179E12,3.8609654E-2,1.5483703E-2,-1.7859455E-2,-7.322303E-3,2.5222173E-1,-5.3199553E-3,-1.2776313E-2,-2.1007718E-3,1.9505986E6,3.0128152E0,8.267169E-3,2.3765222E-4,3.5951093E-3,1.2551224E-2],"split_indices":[54,4,64,39,64,43,59,4,0,0,0,45,43,0,0,0,0,51,0,0,0,60,69,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,2.9E1,5.7E1,2.1E1,8E0,4.8E1,9E0,1.3E1,8E0,4E0,4E0,4.2E1,6E0,3E0,6E0,9E0,4E0,3.8E1,4E0,3E0,3E0,1.7E1,2.1E1,6E0,1.1E1,4E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.749552E-2,-3.2715732E-1,1.8533774E-1,5.477187E-2,-4.2820457E-1,9.8413765E-2,2.1279946E-2,-1.5928174E-3,6.9637727E-3,-3.4048092E-1,-3.3868957E-2,-3.096505E-3,1.2572718E-1,-3.751525E-1,-3.7225285E-3,1.1022101E-2,7.855056E-2,-1.23166265E-2,-2.4024157E-2,3.1705864E-2,7.904757E-3,-6.627175E-4,4.8371227E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,15,17,-1,-1,19,-1,-1,21,-1,-1,-1],"loss_changes":[5.226652E0,1.3356116E0,1.1364522E0,7.6286726E-2,4.9889994E-1,1.8151897E-1,0E0,0E0,0E0,1.7535901E-1,0E0,0E0,1.6319561E-1,1.7636037E-1,0E0,0E0,1.06416434E-1,0E0,0E0,6.394208E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,12,12,13,13,16,16,19,19],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,16,18,-1,-1,20,-1,-1,22,-1,-1,-1],"split_conditions":[1.4865639E-2,3.459181E-1,1.0272479E0,8.231683E2,1.5135763E8,3.73402E5,2.1279946E-2,-1.5928174E-3,6.9637727E-3,2.277961E6,-3.3868957E-2,-3.096505E-3,2.325535E7,1.02E3,-3.7225285E-3,1.1022101E-2,2.0384211E5,-1.23166265E-2,-2.4024157E-2,1E0,7.904757E-3,-6.627175E-4,4.8371227E-3],"split_indices":[50,39,51,64,57,1,0,0,0,1,0,0,57,0,0,0,45,0,0,23,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,3.3E1,4.7E1,7E0,2.6E1,3.7E1,1E1,4E0,3E0,2.2E1,4E0,5E0,3.2E1,1.9E1,3E0,8E0,2.4E1,1.3E1,6E0,1.7E1,7E0,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.07699195E-2,-2.9531324E-1,1.5307122E-1,7.440804E-2,-4.8430765E-1,5.84366E-2,3.442456E-1,-5.83541E-2,1.308014E-2,-1.44599145E-2,-2.6559446E-2,1.0686309E-1,-1.4862727E-1,4.1020027E-1,3.519601E-3,-6.413726E-3,1.7779914E-3,6.7693226E-2,1.419395E-2,2.9000307E-3,-1.7118944E-2,2.2029571E-2,9.836773E-3,1.3448E-1,-3.812803E-2,-3.2083434E-3,8.181241E-3,-3.7226812E-3,3.183553E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[3.7039104E0,1.9226687E0,1.029203E0,3.0056894E-1,1.1430836E-1,4.1697776E-1,3.024242E-1,6.2147528E-2,0E0,0E0,0E0,2.4486849E-1,4.125325E-1,1.1278486E-1,0E0,0E0,0E0,2.1008381E-1,0E0,0E0,0E0,0E0,0E0,1.816572E-1,6.275297E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,13,13,17,17,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,9.306648E-1,8.231683E2,8.595346E-1,1.0743855E-1,3.3934937E0,1.3E1,1.308014E-2,-1.44599145E-2,-2.6559446E-2,2.8364465E2,1.6124614E5,1.3996E4,3.519601E-3,-6.413726E-3,1.7779914E-3,1.0191781E1,1.419395E-2,2.9000307E-3,-1.7118944E-2,2.2029571E-2,9.836773E-3,2.3660715E0,3.5101352E0,-3.2083434E-3,8.181241E-3,-3.7226812E-3,3.183553E-3],"split_indices":[54,39,51,64,39,50,66,3,0,0,0,70,45,9,0,0,0,68,0,0,0,0,0,66,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.6E1,5.7E1,9E0,1.7E1,3.9E1,1.8E1,6E0,3E0,8E0,9E0,3.2E1,7E0,1.4E1,4E0,3E0,3E0,2.8E1,4E0,4E0,3E0,9E0,5E0,1.7E1,1.1E1,3E0,1.4E1,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[4.1500002E-2,-1.1020562E-1,2.5445205E-1,-3.58531E-1,-3.5421815E-2,-1.24896895E-2,3.2412177E-1,-6.8192994E-3,-2.0704947E-2,8.161233E-2,-1.4483316E-1,-6.536249E-3,7.0363698E-3,2.5490408E-3,1.6718285E-2,1.30979465E-2,1.2294497E-2,-8.460815E-2,-1.2313737E-2,-3.8473636E-2,7.4692317E-3,-3.7698593E-4,-6.824509E-3,1.1083615E-3,-5.965015E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,-1,21,-1,23,-1,-1,-1,-1,-1],"loss_changes":[2.6448302E0,8.786946E-1,6.363716E-1,1.550659E-1,4.98405E-1,1.9851494E-1,2.9883957E-1,0E0,0E0,2.427751E-1,1.2932685E-1,0E0,0E0,0E0,0E0,1.234806E-1,0E0,7.398031E-2,0E0,7.333902E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,17,17,19,19],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,-1,22,-1,24,-1,-1,-1,-1,-1],"split_conditions":[3.839604E-1,1.346E3,2.045738E0,2.62518E5,5.81609E-1,1.6E1,3.2414E-1,-6.8192994E-3,-2.0704947E-2,7.347893E5,9.6910744E7,-6.536249E-3,7.0363698E-3,2.5490408E-3,1.6718285E-2,9.18853E0,1.2294497E-2,1.5326E4,-1.2313737E-2,2.1416E4,7.4692317E-3,-3.7698593E-4,-6.824509E-3,1.1083615E-3,-5.965015E-3],"split_indices":[54,11,68,41,39,8,39,0,0,40,57,0,0,0,0,66,0,9,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,4.7E1,3.3E1,1E1,3.7E1,7E0,2.6E1,4E0,6E0,1.8E1,1.9E1,4E0,3E0,4E0,2.2E1,1.4E1,4E0,1.4E1,5E0,1.1E1,3E0,7E0,7E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.783738E-2,-2.912773E-1,1.5091665E-1,-1.5920183E-1,-5.892889E-1,7.253032E-2,5.512725E-1,-2.5153354E-1,-5.604259E-4,-1.19953435E-2,-3.4818623E-2,6.7662774E-3,2.0002894E-1,3.4802377E-2,1.2706493E-2,-1.3977169E-2,-3.4462472E-3,-8.289583E-3,3.6144804E-2,1.5521637E-2,4.566219E-3,1.3001332E-1,-2.6676973E-2,1.1240531E-2,2.2803291E-3,-3.5798955E-3,3.6997786E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,-1,-1,-1,-1,21,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[3.3824525E0,8.5082483E-1,1.9057267E0,2.4096555E-1,2.0170045E-1,4.4304228E-1,3.698752E-1,8.551359E-2,0E0,0E0,0E0,2.0688936E-1,2.195872E-1,0E0,0E0,0E0,0E0,0E0,1.9217195E-1,0E0,0E0,1.0953201E-1,1.1933081E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,18,18,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,-1,-1,-1,-1,22,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[1.6405078E-2,7.916317E11,3.9509753E3,1.6E1,8.28645E-1,5.596155E-1,8.982776E9,1.0370839E6,-5.604259E-4,-1.19953435E-2,-3.4818623E-2,2.3428571E0,2.369681E0,3.4802377E-2,1.2706493E-2,-1.3977169E-2,-3.4462472E-3,-8.289583E-3,3.2449896E0,1.5521637E-2,4.566219E-3,1.5464425E5,3.6616542E0,1.1240531E-2,2.2803291E-3,-3.5798955E-3,3.6997786E-3],"split_indices":[53,43,64,8,39,51,5,40,0,0,0,68,65,0,0,0,0,0,66,0,0,44,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,2.3E1,6.1E1,1.7E1,6E0,5.2E1,9E0,1E1,7E0,3E0,3E0,3.5E1,1.7E1,4E0,5E0,7E0,3E0,4E0,3.1E1,6E0,1.1E1,1.2E1,1.9E1,4E0,8E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.3436422E-2,-2.3631646E-1,1.11033164E-1,6.628314E-2,-4.0438363E-1,2.6376931E-2,2.6559058E-1,-9.590969E-4,1.2176767E-2,-2.753368E-1,-6.415788E-1,6.491471E-2,-1.0238427E-2,1.7260915E-1,1.9895872E-2,-1.6248593E-2,-1.18002646E-1,-3.6040127E-2,-1.4927832E-2,1.9713245E-2,1.1689342E-2,2.1983718E-3,9.870673E-3,-9.525898E-3,1.5159528E-4,5.489354E-2,-4.023996E-3,-2.1634966E-3,5.5231806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,21,-1,-1,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[2.5166912E0,1.7559438E0,6.9954395E-1,2.351588E-1,4.9675417E-1,3.6414066E-1,2.2943354E-1,0E0,0E0,1.656959E-1,7.2425604E-2,2.7349132E-1,0E0,6.740874E-2,0E0,0E0,7.8710414E-2,0E0,0E0,1.0779085E-1,0E0,0E0,0E0,0E0,0E0,1.4960691E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,13,13,16,16,19,19,25,25],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,22,-1,-1,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,8.363552E-1,3.1672727E3,3.1457312E8,6.847574E-1,4.281481E5,-9.590969E-4,1.2176767E-2,4.593E3,9.7646296E7,5.6552525E6,-1.0238427E-2,3.956147E-1,1.9895872E-2,-1.6248593E-2,7.8987443E-1,-3.6040127E-2,-1.4927832E-2,1.2932927E1,1.1689342E-2,2.1983718E-3,9.870673E-3,-9.525898E-3,1.5159528E-4,2.6880343E0,-4.023996E-3,-2.1634966E-3,5.5231806E-3],"split_indices":[54,39,53,4,7,53,40,0,0,2,12,59,0,39,0,0,39,0,0,69,0,0,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,3.3E1,5.3E1,1.2E1,2.1E1,3.5E1,1.8E1,9E0,3E0,1.5E1,6E0,3.1E1,4E0,1.3E1,5E0,9E0,6E0,3E0,3E0,2.6E1,5E0,4E0,9E0,3E0,3E0,2E1,6E0,8E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.8239552E-2,-2.7828404E-1,1.142956E-1,-3.8252535E-1,2.5509887E-3,-5.121969E-3,2.5601724E-1,-4.6930257E-3,-4.543218E-1,2.653074E-2,-1.0950974E-2,1.590247E-1,3.826405E-1,-1.5706996E-2,-2.7840158E-2,-3.8930497E-3,7.126159E-2,1.1652136E-4,8.73257E-3,2.177474E-2,1.0070283E-2,9.812252E-3,3.7007716E-2,-1.662999E-3,4.2624013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,-1,9,11,-1,13,15,-1,17,19,-1,-1,-1,21,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[3.0647302E0,1.0674927E0,9.998305E-1,4.1398644E-1,0E0,2.5303146E-1,2.707516E-1,0E0,9.77273E-2,1.567961E-1,0E0,8.8192314E-2,6.522429E-2,0E0,0E0,0E0,1.0118808E-1,0E0,0E0,0E0,0E0,0E0,8.3036706E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,11,11,12,12,16,16,22,22],"right_children":[2,4,6,8,-1,10,12,-1,14,16,-1,18,20,-1,-1,-1,22,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,5.451147E-1,3.7582534E5,2.5509887E-3,3.6030095E10,1.1997641E0,-4.6930257E-3,1.2427474E8,7.095772E2,-1.0950974E-2,1.0966E4,1.3996E4,-1.5706996E-2,-2.7840158E-2,-3.8930497E-3,1.7175853E9,1.1652136E-4,8.73257E-3,2.177474E-2,1.0070283E-2,9.812252E-3,1.204781E3,-1.662999E-3,4.2624013E-3],"split_indices":[54,4,54,40,0,5,51,0,57,4,0,9,9,0,0,0,5,0,0,0,0,0,64,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,2.9E1,5.8E1,2.2E1,7E0,3.2E1,2.6E1,5E0,1.7E1,2.9E1,3E0,1.6E1,1E1,1.2E1,5E0,8E0,2.1E1,3E0,1.3E1,5E0,5E0,3E0,1.8E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.7810311E-2,-2.9339778E-1,1.433201E-1,8.202345E-2,-4.2244408E-1,9.144485E-2,2.2298355E-2,-8.1810955E-4,8.469566E-3,-5.0174516E-1,-6.3810605E-3,1.2394942E-1,-8.188362E-3,-1.4675606E-2,-2.9043509E-2,1.3441059E-2,8.367863E-2,1.1566455E-1,-1.3952386E-3,6.1567975E-3,-1.2726436E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,-1,-1,-1,-1,17,19,-1,-1,-1],"loss_changes":[3.3724637E0,1.3911586E0,8.213216E-1,8.704117E-2,3.988874E-1,4.0228632E-1,0E0,0E0,0E0,1.8733978E-1,0E0,2.4558145E-1,0E0,0E0,0E0,0E0,1.2432307E-1,7.8385055E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,16,16,17,17],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,-1,-1,-1,-1,18,20,-1,-1,-1],"split_conditions":[1.7489342E-2,5.81609E-1,6.7780703E3,2.4399805E3,1.650581E3,2.9461394E5,2.2298355E-2,-8.1810955E-4,8.469566E-3,1.0370839E6,-6.3810605E-3,8.403455E6,-8.188362E-3,-1.4675606E-2,-2.9043509E-2,1.3441059E-2,4.800481E5,2.3297022E7,-1.3952386E-3,6.1567975E-3,-1.2726436E-3],"split_indices":[50,39,4,4,4,45,0,0,0,40,0,57,0,0,0,0,40,44,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.7E1,4.7E1,7E0,2E1,4.2E1,5E0,4E0,3E0,1.5E1,5E0,3.8E1,4E0,8E0,7E0,6E0,3.2E1,2.5E1,7E0,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.5394238E-3,-2.7393386E-1,1.1003588E-1,-3.60773E-1,3.434535E-3,5.6935433E-2,4.4001254E-1,-2.6878086E-1,-2.7300242E-2,-1.3018504E-2,2.5833794E-1,2.9757988E-2,1.0122976E-2,-1.4993296E-2,-4.583872E-3,2.1140752E-2,-1.2878089E-2,1.757506E-2,5.87598E-3,-6.212103E-3,5.879974E-2,4.0496658E-3,-3.0366671E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,-1,-1,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[2.7499392E0,8.166115E-1,1.1313938E0,3.0319285E-1,0E0,8.205192E-1,2.7475917E-1,1.4209735E-1,0E0,4.1281688E-1,1.9895303E-1,0E0,0E0,0E0,0E0,2.4603006E-1,0E0,0E0,0E0,0E0,1.3210319E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,15,15,20,20],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,-1,-1,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[2.0113895E-2,1.962943E3,4.0230347E3,1.2427474E8,3.434535E-3,8.363552E-1,5.0999485E9,1E0,-2.7300242E-2,1.4899646E8,2.4696612E0,2.9757988E-2,1.0122976E-2,-1.4993296E-2,-4.583872E-3,6.2840106E2,-1.2878089E-2,1.757506E-2,5.87598E-3,-6.212103E-3,1.1269586E12,4.0496658E-3,-3.0366671E-3],"split_indices":[53,4,64,57,0,53,12,23,0,57,65,0,0,0,0,4,0,0,0,0,43,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9E1,2.5E1,6.5E1,2E1,5E0,5.7E1,8E0,1.6E1,4E0,4.3E1,1.4E1,3E0,5E0,1.1E1,5E0,3.9E1,4E0,6E0,8E0,7E0,3.2E1,2.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.0426013E-2,-1.9696409E-1,1.15880854E-1,7.172566E-2,-3.5923603E-1,8.615776E-2,1.7431319E-2,-9.752822E-4,1.090455E-2,-2.2716135E-1,-2.3399483E-2,1.0819109E-2,5.938154E-2,-1.3214653E-2,-1.9287409E-3,6.0612066E-3,1.1495231E-1,-3.2542814E-2,5.4865936E-3,-6.2958174E-4,6.2278933E-3,-2.9991926E-3,4.59919E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,15,-1,-1,17,19,21,-1,-1,-1,-1,-1],"loss_changes":[1.7505772E0,1.2029883E0,3.892765E-1,1.7791368E-1,2.3367739E-1,1.8550536E-1,0E0,0E0,0E0,1.17523134E-1,0E0,0E0,1.2671448E-1,0E0,0E0,1.05864935E-1,6.2069297E-2,8.747192E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,16,-1,-1,18,20,22,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,5.8871865E-1,6.7780703E3,8.231683E2,1.1686677E8,3.5175372E5,1.7431319E-2,-9.752822E-4,1.090455E-2,6.312E3,-2.3399483E-2,1.0819109E-2,5.517496E-1,-1.3214653E-2,-1.9287409E-3,3.327103E0,1.892952E5,1.1812E4,5.4865936E-3,-6.2958174E-4,6.2278933E-3,-2.9991926E-3,4.59919E-3],"split_indices":[50,39,4,64,57,44,0,0,0,2,0,0,53,0,0,65,40,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.6E1,5.2E1,1E1,1.6E1,4.8E1,4E0,7E0,3E0,1E1,6E0,6E0,4.2E1,7E0,3E0,2.2E1,2E1,1.7E1,5E0,3E0,1.7E1,1.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.9540882E-3,-2.5166765E-1,1.392207E-1,-3.9185032E-2,-4.1475496E-1,8.000073E-2,4.845404E-1,3.7000498E-3,-4.8882435E-3,-4.8174846E-1,-2.716472E-3,1.7776957E-2,1.6933635E-1,3.192463E-2,9.653529E-3,-1.1978023E-2,-2.7132142E-2,-2.9986171E-2,4.6758302E-3,1.26023535E-2,4.680315E-3,5.399366E-3,-2.8718812E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[3.1114268E0,1.0926509E0,1.1064266E0,1.3278343E-1,3.9915705E-1,2.6937085E-1,3.2618237E-1,0E0,0E0,2.2122025E-1,0E0,1.2520842E-1,1.1481297E-1,0E0,0E0,0E0,0E0,1.0664484E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,11,11,12,12,17,17],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,5.0415697E5,6.7780703E3,1.8144448E-4,1.650581E3,6.3104886E-1,1.1408508E9,3.7000498E-3,-4.8882435E-3,1.0370839E6,-2.716472E-3,2.6597537E3,1.181508E6,3.192463E-2,9.653529E-3,-1.1978023E-2,-2.7132142E-2,2.2519132E-1,4.6758302E-3,1.26023535E-2,4.680315E-3,5.399366E-3,-2.8718812E-3],"split_indices":[50,40,4,50,4,53,7,0,0,40,0,4,44,0,0,0,0,39,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,3.1E1,5.5E1,1.4E1,1.7E1,4.8E1,7E0,5E0,9E0,1.4E1,3E0,2.9E1,1.9E1,3E0,4E0,6E0,8E0,1.9E1,1E1,6E0,1.3E1,3E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.25183E-2,-2.5854746E-1,1.268138E-1,-1.5110075E-1,-5.2860266E-1,2.4083443E-1,4.4913583E-2,-6.360457E-5,-1.8843809E-1,-3.0226521E-2,-1.176645E-2,4.3794084E-3,3.3705422E-1,-3.122707E-3,8.60903E-2,-7.244103E-2,-1.2284329E-2,7.187997E-3,1.8909542E-2,6.9698785E-3,-9.4875006E-4,-7.5199325E-3,2.8229766E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.46665E0,6.745579E-1,4.1874576E-1,1.1152369E-1,8.624041E-2,2.3140132E-1,1.3701582E-1,0E0,1.329022E-1,0E0,0E0,0E0,9.0325E-2,0E0,1.5475002E-1,1.1069518E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,12,12,14,14,15,15],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,1.5135763E8,1.3996E4,3.027892E5,9.7646296E7,1.2096E4,4.57563E5,-6.360457E-5,1.5368E4,-3.0226521E-2,-1.176645E-2,4.3794084E-3,1.3E1,-3.122707E-3,4.8366976E7,1.885E3,-1.2284329E-2,7.187997E-3,1.8909542E-2,6.9698785E-3,-9.4875006E-4,-7.5199325E-3,2.8229766E-3],"split_indices":[50,57,9,40,12,9,1,0,9,0,0,0,3,0,57,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,2.5E1,4.5E1,1.9E1,6E0,1.8E1,2.7E1,4E0,1.5E1,3E0,3E0,8E0,1E1,7E0,2E1,7E0,8E0,4E0,6E0,1.2E1,8E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-5.6215986E-3,-2.3854735E-1,1.0663194E-1,-1.5780413E-1,-2.9197419E-2,5.548581E-2,2.2995051E-2,-2.0708188E-1,-1.5586142E-2,2.3890741E-1,2.328797E-3,-5.9584286E-3,-1.6469808E-2,3.7206237E-3,-4.95881E-3,3.0720867E-3,1.3376149E-2,-1.8005209E-1,4.7851242E-2,-2.206976E-3,-1.1311997E-2,5.633968E-3,-4.1092155E-3,-3.2174361E-3,3.8753855E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,-1,-1,-1,-1,-1,-1,19,21,-1,-1,-1,23,-1,-1],"loss_changes":[2.0910294E0,7.479663E-1,1.076433E0,1.579606E-1,0E0,4.7720075E-1,0E0,1.5311581E-1,7.29004E-2,8.507353E-2,3.3241022E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.0582012E-2,1.2730731E-1,0E0,0E0,0E0,1.2553605E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,17,17,18,18,22,22],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,-1,-1,-1,-1,-1,-1,20,22,-1,-1,-1,24,-1,-1],"split_conditions":[4.856813E0,1.5135763E8,3.4409692E3,5.36E3,-2.9197419E-2,1.7329262E7,2.2995051E-2,1.02E3,2.2858976E8,5.391345E6,4.492E3,-5.9584286E-3,-1.6469808E-2,3.7206237E-3,-4.95881E-3,3.0720867E-3,1.3376149E-2,4.841492E5,5.505491E7,-2.206976E-3,-1.1311997E-2,5.633968E-3,3.048E3,-3.2174361E-3,3.8753855E-3],"split_indices":[70,57,64,2,0,57,0,0,7,57,2,0,0,0,0,0,0,40,57,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.5E1,5.3E1,2.2E1,3E0,4.8E1,5E0,1.6E1,6E0,1E1,3.8E1,1.2E1,4E0,3E0,3E0,3E0,7E0,7E0,3.1E1,3E0,4E0,1.2E1,1.9E1,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-7.849184E-3,-2.2455516E-1,9.0429924E-2,-3.79671E-1,-8.663878E-2,1.1765177E-1,-1.23679275E-2,-8.0365455E-3,-2.3561342E-2,-1.5010403E-1,1.0440375E-3,4.249952E-2,1.8009628E-1,-1.0750746E-2,-1.5144366E-3,1.5942997E-1,-1.1770988E-3,-2.6721784E-3,9.642793E-3,3.2163842E-4,1.0641589E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,17,-1,-1,19,-1,-1,-1,-1,-1],"loss_changes":[1.6389296E0,4.8053825E-1,5.5560297E-1,2.1873927E-1,1.0086677E-1,2.2698826E-1,0E0,0E0,0E0,8.321676E-2,0E0,1.9617009E-1,2.210496E-1,0E0,0E0,1.0115999E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,12,12,15,15],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,18,-1,-1,20,-1,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,3.554517E0,6.2012E4,1.0370839E6,7.2828925E2,3.3597556E-1,-1.23679275E-2,-8.0365455E-3,-2.3561342E-2,9.4596675E-5,1.0440375E-3,1.9505986E6,1.4212261E0,-1.0750746E-2,-1.5144366E-3,7.5578E5,-1.1770988E-3,-2.6721784E-3,9.642793E-3,3.2163842E-4,1.0641589E-2],"split_indices":[53,66,10,40,64,51,0,0,0,50,0,60,68,0,0,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.3E1,5.2E1,1E1,1.3E1,4.9E1,3E0,5E0,5E0,8E0,5E0,2.3E1,2.6E1,4E0,4E0,8E0,1.5E1,3E0,2.3E1,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.182457E-2,-1.7961979E-1,6.0280535E-2,3.4418746E-3,-2.4955288E-1,8.1557594E-2,-1.0745508E-2,-2.905556E-1,-1.989761E-3,5.522869E-3,1.492485E-1,-1.6267997E-3,-3.3607563E-1,-3.7661197E-3,4.248944E-2,1.9486211E-1,1.4209464E-3,-1.707799E-2,-5.5213147E-3,4.4869604E-3,-4.361666E-3,1.0237033E-2,2.6246503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,15,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0991141E0,5.369692E-1,3.7317508E-1,0E0,1.8127179E-1,2.719592E-1,0E0,2.0825803E-1,0E0,8.857165E-2,1.445399E-1,0E0,9.194195E-2,0E0,1.5786646E-1,7.096261E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,10,10,12,12,14,14,15,15],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,16,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,4.4092423E-1,7.488E4,3.4418746E-3,1.0735684E3,3.3597556E-1,-1.0745508E-2,2.5346E3,-1.989761E-3,7.095772E2,5.7588155E6,-1.6267997E-3,8.565E3,-3.7661197E-3,1.2932927E1,1.6129E4,1.4209464E-3,-1.707799E-2,-5.5213147E-3,4.4869604E-3,-4.361666E-3,1.0237033E-2,2.6246503E-3],"split_indices":[50,39,10,0,64,51,0,44,0,4,44,0,2,0,69,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.8E1,5.5E1,6E0,2.2E1,5.2E1,3E0,1.8E1,4E0,2.5E1,2.7E1,3E0,1.5E1,7E0,1.8E1,1.9E1,8E0,1.2E1,3E0,1.3E1,5E0,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.0194096E-2,-1.9626883E-1,8.302531E-2,-8.017087E-2,-3.9722803E-1,1.1439448E-1,-1.1794524E-2,-9.948932E-3,4.083823E-2,-2.4882039E-2,-1.5369701E-3,1.8858615E-1,4.65193E-3,4.545634E-3,-3.993163E-3,1.200564E-1,3.2049298E-1,-3.3116487E-3,6.398915E-2,5.6456048E-2,9.335537E-3,1.9434514E-2,8.0526015E-3,-2.2037984E-3,4.657753E-3,-5.099421E-4,6.7448476E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,15,17,-1,-1,19,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4559946E0,6.214205E-1,6.319879E-1,3.2851723E-1,5.297437E-1,4.2250592E-1,0E0,0E0,9.282162E-2,0E0,0E0,2.4174857E-1,1.0597328E-1,0E0,0E0,1.0815394E-1,7.807326E-2,0E0,6.1474565E-2,8.8971406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,11,11,12,12,15,15,16,16,18,18,19,19],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,16,18,-1,-1,20,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.052686E6,1.6597747E8,5E0,1.650581E3,3.3934937E0,-1.1794524E-2,-9.948932E-3,5.0415697E5,-2.4882039E-2,-1.5369701E-3,2.9187737E0,4.0614333E0,4.545634E-3,-3.993163E-3,5.6296086E-1,7.33E2,-3.3116487E-3,7.0277144E7,9.105E3,9.335537E-3,1.9434514E-2,8.0526015E-3,-2.2037984E-3,4.657753E-3,-5.099421E-4,6.7448476E-3],"split_indices":[70,40,57,8,4,66,0,0,40,0,0,66,66,0,0,51,0,0,12,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.7E1,5.5E1,1.8E1,9E0,5.1E1,4E0,8E0,1E1,6E0,3E0,3E1,2.1E1,7E0,3E0,2.1E1,9E0,9E0,1.2E1,1.3E1,8E0,4E0,5E0,3E0,9E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.148071E-3,-1.5744287E-1,5.6998823E-2,-2.3415917E-1,5.402824E-2,7.96975E-2,-1.3257934E-2,-2.7629182E-1,1.5635673E-4,6.7162337E-3,-1.4487974E-3,9.649279E-3,4.4821005E-2,-9.046014E-3,-1.9124184E-2,-1.20576E-3,9.067582E-3,-2.9900339E-2,4.9996576E-3,-3.311612E-3,1.6033393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,-1,15,-1,-1,17,-1,19,-1,-1,-1],"loss_changes":[8.6487025E-1,4.5067978E-1,5.1450765E-1,2.011689E-1,6.9297984E-2,2.6386982E-1,0E0,1.0588992E-1,0E0,0E0,0E0,0E0,3.4251857E-1,0E0,0E0,1.247E-1,0E0,8.961692E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,12,12,15,15,17,17],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,-1,16,-1,-1,18,-1,20,-1,-1,-1],"split_conditions":[4.856813E0,4.2332153E0,1.7171936E8,5.36E3,4.2E1,1.7329262E7,-1.3257934E-2,7.916317E11,1.5635673E-4,6.7162337E-3,-1.4487974E-3,9.649279E-3,4.9891987E3,-9.046014E-3,-1.9124184E-2,5.2518907E0,9.067582E-3,1.204781E3,4.9996576E-3,-3.311612E-3,1.6033393E-3],"split_indices":[70,66,57,2,8,57,0,43,0,0,0,0,4,0,0,66,0,64,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,2.6E1,6.1E1,1.9E1,7E0,5.8E1,3E0,1.6E1,3E0,3E0,4E0,1.1E1,4.7E1,1.2E1,4E0,3.7E1,1E1,3E1,7E0,1.8E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.5839636E-2,3.150322E-2,-4.9376866E-1,-1.2215051E-1,8.9994974E-2,-1.0699115E-2,-2.8526591E-2,-1.5580004E-1,2.3332029E-3,1.1592733E-2,4.4538695E-2,-2.126912E-1,-2.0038544E-3,6.9915876E-3,8.236244E-3,-2.8313026E-3,-1.186017E-2,-1.2546367E-2,4.5513045E-3,1.743236E-3,-3.5501372E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,15,-1,17,-1,-1,-1,19,-1,-1,-1],"loss_changes":[1.7577977E0,6.4910376E-1,9.512305E-2,1.2480411E-1,3.795367E-1,0E0,0E0,9.8600805E-2,0E0,0E0,2.1461332E-1,6.4023584E-2,0E0,6.321506E-2,0E0,0E0,0E0,1.0001323E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10,11,11,13,13,17,17],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,16,-1,18,-1,-1,-1,20,-1,-1,-1],"split_conditions":[1.5414109E8,3.2424886E7,1.2096E4,7.4325244E2,1.6256282E7,-1.0699115E-2,-2.8526591E-2,7.97E2,2.3332029E-3,1.1592733E-2,1.6132394E2,2.68E2,-2.0038544E-3,5.2518907E0,8.236244E-3,-2.8313026E-3,-1.186017E-2,3.28464E0,4.5513045E-3,1.743236E-3,-3.5501372E-3],"split_indices":[57,12,9,64,57,0,0,10,0,0,69,11,0,66,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,1.9E1,5.1E1,3E0,3E0,1.6E1,3E0,1E1,4.1E1,1E1,6E0,3.3E1,8E0,3E0,7E0,2.8E1,5E0,1.6E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.5537694E-2,-1.8171269E-1,1.08694494E-1,-3.2136786E-1,-6.718472E-2,6.820693E-2,3.3954275E-1,-8.441578E-3,-2.0276563E-2,1.3846875E-3,-5.4055997E-3,-5.7516242E-3,8.633362E-2,2.5373014E-2,6.2050344E-3,8.462097E-3,5.3327255E-2,3.8628117E-3,-2.1473267E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,-1,15,-1,-1,-1,17,-1,-1],"loss_changes":[1.5018991E0,3.7826967E-1,5.56323E-1,1.0357487E-1,7.96736E-2,2.0172289E-1,3.241428E-1,0E0,0E0,0E0,0E0,0E0,1.5521684E-1,0E0,0E0,0E0,1.286928E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,12,12,16,16],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,-1,16,-1,-1,-1,18,-1,-1],"split_conditions":[2.0113895E-2,3.554517E0,3.9509753E3,9.29295E5,2.125951E-1,2.3688402E0,5.99999E9,-8.441578E-3,-2.0276563E-2,1.3846875E-3,-5.4055997E-3,-5.7516242E-3,1.7329262E7,2.5373014E-2,6.2050344E-3,8.462097E-3,5.6064875E5,3.8628117E-3,-2.1473267E-3],"split_indices":[53,66,64,40,39,66,5,0,0,0,0,0,57,0,0,0,40,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,2.4E1,6.1E1,1E1,1.4E1,5.3E1,8E0,6E0,4E0,5E0,9E0,4E0,4.9E1,3E0,5E0,1.1E1,3.8E1,2.9E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-8.738357E-3,-2.2589757E-1,3.230234E-2,-5.6503005E-3,-1.4684667E-2,7.671018E-2,-1.4763632E-1,2.342224E-2,1.7145735E-1,-3.789736E-2,-1.8725913E-2,1.6630657E-1,-4.3425232E-2,1.0119886E-2,3.803101E-3,3.1012665E-3,-3.9161216E-3,1.4498088E-3,1.340757E-2,-3.6220208E-3,3.0266824E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,-1,7,9,11,13,15,-1,17,19,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.291986E-1,8.8412285E-2,5.626826E-1,0E0,0E0,2.7872145E-1,3.9130357E-1,3.606258E-1,7.158369E-2,6.2407233E-2,0E0,1.9864476E-1,1.0941354E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9,11,11,12,12],"right_children":[2,4,6,-1,-1,8,10,12,14,16,-1,18,20,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.528E3,7.83225E-1,1.0232127E12,-5.6503005E-3,-1.4684667E-2,1.2544873E3,1.5135763E8,1.6939252E7,7.455661E-1,1.6627E4,-1.8725913E-2,2.372093E0,5.0175633E0,1.0119886E-2,3.803101E-3,3.1012665E-3,-3.9161216E-3,1.4498088E-3,1.340757E-2,-3.6220208E-3,3.0266824E-3],"split_indices":[11,39,43,0,0,64,57,57,39,9,0,69,66,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,1.2E1,6.8E1,7E0,5E0,5.5E1,1.3E1,3.6E1,1.9E1,1E1,3E0,1.1E1,2.5E1,1.1E1,8E0,3E0,7E0,6E0,5E0,1.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.9678643E-3,-1.7924425E-1,7.051481E-2,-2.9530045E-1,-2.5791967E-2,2.4483178E-2,2.0132555E-1,-7.506752E-3,-1.9035598E-2,5.2060843E-3,-4.2129327E-3,-8.491854E-3,8.457017E-3,1.5771853E-2,5.613992E-3,-6.1066123E-3,1.3355826E-2,-2.8867677E-3,4.2437468E-2,5.8197193E-3,-4.770231E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,-1,-1,-1,-1,17,-1,19,-1,-1],"loss_changes":[1.0465004E0,4.0199262E-1,3.683997E-1,1.4016604E-1,1.141954E-1,2.5236684E-1,1.3857341E-1,0E0,0E0,0E0,0E0,1.1193472E-1,0E0,0E0,0E0,0E0,8.102062E-2,0E0,1.1940887E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,16,16,18,18],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,-1,-1,-1,-1,18,-1,20,-1,-1],"split_conditions":[1.6405078E-2,3.554517E0,6.2239337E-1,1.0370839E6,2.805454E-1,3.6591887E3,1.3256E4,-7.506752E-3,-1.9035598E-2,5.2060843E-3,-4.2129327E-3,1.2104E4,8.457017E-3,1.5771853E-2,5.613992E-3,-6.1066123E-3,7.095772E2,-2.8867677E-3,4.8366976E7,5.8197193E-3,-4.770231E-4],"split_indices":[53,66,51,40,39,4,9,0,0,0,0,9,0,0,0,0,4,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.2E1,6.1E1,1.2E1,1E1,4.6E1,1.5E1,7E0,5E0,3E0,7E0,3.9E1,7E0,4E0,1.1E1,5E0,3.4E1,9E0,2.5E1,9E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.6597202E-2,-1.7761233E-1,4.6452716E-2,-2.5358334E-1,3.6271946E-3,1.2658683E-2,2.9568574E-1,-1.595643E-1,-1.5941024E-2,1.4717323E-1,-4.9487423E-2,1.7836573E-2,5.653132E-3,-2.387583E-3,-1.0892058E-2,4.90533E-4,9.219407E-3,-2.0000001E-2,-1.3062838E-2,-3.5058046E-3,2.6164937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,-1,-1,-1,-1,-1,-1,19,-1,-1,-1],"loss_changes":[8.305618E-1,4.702267E-1,4.9318975E-1,1.09102964E-1,0E0,4.5033962E-1,6.756139E-2,8.1953615E-2,0E0,1.2743995E-1,2.5420693E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5551779E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,17,17],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,-1,-1,-1,-1,-1,-1,20,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,4.0230347E3,4.1E1,3.6271946E-3,2.325535E7,9.5E2,1.5368E4,-1.5941024E-2,4.4939005E2,4.927835E0,1.7836573E-2,5.653132E-3,-2.387583E-3,-1.0892058E-2,4.90533E-4,9.219407E-3,1.204781E3,-1.3062838E-2,-3.5058046E-3,2.6164937E-3],"split_indices":[70,4,64,8,0,57,0,9,0,4,66,0,0,0,0,0,0,64,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.2E1,5.8E1,1.7E1,5E0,5.2E1,6E0,1E1,7E0,1.6E1,3.6E1,3E0,3E0,5E0,5E0,5E0,1.1E1,3.3E1,3E0,1.9E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.0134083E-3,-2.041003E-1,8.136195E-2,-9.391164E-2,-3.7101096E-1,5.105158E-2,2.3065235E-2,-7.090068E-3,-4.3301247E-3,-2.0301973E-2,-6.576766E-3,-7.4490914E-3,6.968211E-2,3.2094012E-3,-4.4048843E-3,1.4555176E-1,3.7592195E-2,-1.660366E-4,8.183199E-3,3.6768668E-4,6.277124E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,-1,15,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[1.4927157E0,4.433409E-1,7.760042E-1,9.4159946E-2,1.0755837E-1,2.4792331E-1,0E0,0E0,6.250455E-2,0E0,0E0,0E0,1.2843418E-1,0E0,0E0,8.353445E-2,1.1765846E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,12,12,15,15,16,16],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,-1,16,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.0789844E6,7.6312915E3,1.6E1,1.4235585E3,1.9956966E-1,2.3065235E-2,-7.090068E-3,9.502012E2,-2.0301973E-2,-6.576766E-3,-7.4490914E-3,2.6529046E7,3.2094012E-3,-4.4048843E-3,1.9222778E5,1.0193359E8,-1.660366E-4,8.183199E-3,3.6768668E-4,6.277124E-3],"split_indices":[70,40,64,8,4,69,0,0,4,0,0,0,57,0,0,40,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,2.5E1,6.1E1,1.6E1,9E0,5.8E1,3E0,9E0,7E0,6E0,3E0,4E0,5.4E1,4E0,3E0,1.5E1,3.9E1,3E0,1.2E1,3.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.4651191E-2,-2.0180845E-1,7.731667E-2,-1.0264668E-2,-3.3789223E-1,4.437135E-2,2.4061501E-1,3.393428E-3,-4.2476985E-3,-8.908961E-3,-1.9047312E-2,-1.2577588E-3,7.084128E-2,1.5625566E-2,4.9759434E-3,8.26836E-3,1.3976846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,-1,15,-1,-1,-1,-1],"loss_changes":[1.2548997E0,6.221157E-1,2.5293946E-1,8.4903166E-2,7.965469E-2,8.209488E-2,7.084042E-2,0E0,0E0,0E0,0E0,0E0,1.3258423E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,12,12],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,-1,16,-1,-1,-1,-1],"split_conditions":[7.940699E-3,3.7861178E5,1.1997641E0,4.4092423E-1,9.29295E5,2.6277744E8,9.07E2,3.393428E-3,-4.2476985E-3,-8.908961E-3,-1.9047312E-2,-1.2577588E-3,2.325535E7,1.5625566E-2,4.9759434E-3,8.26836E-3,1.3976846E-3],"split_indices":[50,40,51,39,40,12,0,0,0,0,0,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.3E1,4.8E1,1E1,1.3E1,4.1E1,7E0,5E0,5E0,6E0,7E0,1.1E1,3E1,3E0,4E0,7E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-7.994468E-4,-3.6206592E-2,4.09026E-1,-1.6905324E-1,4.9539395E-2,2.588857E-2,6.605055E-3,-2.0598345E-2,-1.2395436E-1,6.976397E-2,-6.6474155E-3,1.8188924E-3,-1.5180066E-1,1.7460525E-1,4.3097246E-2,-8.3302995E-3,-9.4487215E-4,1.0609024E-2,1.920473E-3,-2.9027248E-3,6.933044E-2,4.411607E-3,-4.3353217E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,-1,-1,15,17,19,-1,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.2772446E0,9.310512E-1,2.1450818E-1,3.546288E-1,2.0839038E-1,0E0,0E0,0E0,1.3853744E-1,1.2108058E-1,0E0,0E0,1.0000622E-1,6.0481668E-2,1.1113805E-1,0E0,0E0,0E0,0E0,0E0,6.903453E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,12,12,13,13,14,14,20,20],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,-1,-1,16,18,20,-1,-1,-1,-1,-1,22,-1,-1],"split_conditions":[6.7780703E3,7.2921924E-2,1.523967E6,7E0,2.953641E5,2.588857E-2,6.605055E-3,-2.0598345E-2,3.459181E-1,3.5175372E5,-6.6474155E-3,1.8188924E-3,2.277961E6,1.077E3,3.73402E5,-8.3302995E-3,-9.4487215E-4,1.0609024E-2,1.920473E-3,-2.9027248E-3,1.0284415E1,4.411607E-3,-4.3353217E-4],"split_indices":[4,54,41,3,45,0,0,0,39,44,0,0,1,0,1,0,0,0,0,0,69,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,8E1,6E0,3.1E1,4.9E1,3E0,3E0,3E0,2.8E1,4.5E1,4E0,4E0,2.4E1,8E0,3.7E1,1.9E1,5E0,5E0,3E0,7E0,3E1,2.2E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-9.264157E-3,-3.290431E-2,1.2706172E-2,4.2443708E-2,-1.03420734E-1,-2.8047685E-2,1.002392E-1,-1.8665272E-1,-1.9179318E-2,-3.4076695E-3,3.2078598E-3,1.0561938E-2,2.8682156E-3,-6.214615E-3,-1.3554022E-2,3.8804002E-2,-1.1839665E-2,5.6659393E-3,-3.3749393E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,-1,5,7,9,11,13,15,-1,-1,-1,-1,-1,-1,17,-1,-1,-1],"loss_changes":[5.4241204E-1,3.922585E-1,0E0,1.4947855E-1,2.628602E-1,8.420849E-2,8.3443716E-2,6.492722E-2,2.9033577E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.853649E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,-1,-1,-1,-1,-1,-1,18,-1,-1,-1],"split_conditions":[4.0230347E3,5.460315E9,1.2706172E-2,7.767353E2,2.667437E6,5.0175633E0,3.1841638E5,9.167896E-1,9.71518E5,-3.4076695E-3,3.2078598E-3,1.0561938E-2,2.8682156E-3,-6.214615E-3,-1.3554022E-2,2.8655863E0,-1.1839665E-2,5.6659393E-3,-3.3749393E-4],"split_indices":[64,5,0,4,1,66,44,39,9,0,0,0,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.2E1,5E0,3.5E1,3.7E1,1.6E1,1.9E1,1.8E1,1.9E1,1.1E1,5E0,3E0,1.6E1,1.4E1,4E0,1.6E1,3E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.7823437E-2,-6.995379E-2,1.6539052E-1,-2.7340285E-2,-3.115639E-1,1.1129056E-1,2.3436032E-2,-6.5054996E-3,2.1215392E-2,-1.900557E-2,-4.5299996E-3,7.279794E-3,3.8555324E-2,7.296017E-2,-2.6291567E-3,-3.3933597E-3,4.4133007E-3,-1.1126797E-3,5.6038704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,-1,15,17,-1,-1,-1,-1,-1],"loss_changes":[1.1125773E0,5.474578E-1,5.4998237E-1,2.6769298E-1,1.3967842E-1,9.733111E-2,0E0,0E0,1.4344534E-1,0E0,0E0,0E0,9.367941E-2,1.06366076E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,12,12,13,13],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,-1,16,18,-1,-1,-1,-1,-1],"split_conditions":[1.1808436E3,9.742515E7,7.6312915E3,2.2839906E8,1.650581E3,3.3027112E0,2.3436032E-2,-6.5054996E-3,6.2597164E7,-1.900557E-2,-4.5299996E-3,7.279794E-3,9.8084135E0,2.912E3,-2.6291567E-3,-3.3933597E-3,4.4133007E-3,-1.1126797E-3,5.6038704E-3],"split_indices":[64,57,64,7,4,66,0,0,57,0,0,0,68,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,5.3E1,3.1E1,4.6E1,7E0,2.8E1,3E0,1.3E1,3.3E1,4E0,3E0,1.6E1,1.2E1,2E1,1.3E1,4E0,8E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.6173733E-2,-4.451053E-2,1.5047204E-1,-1.8151723E-2,-2.2114779E-1,1.0435131E-2,-3.025378E-4,7.731114E-2,-6.545097E-2,-2.0189717E-2,-1.5987346E-3,9.029199E-4,6.8847295E-3,-1.2899105E-1,1.0892667E-2,-2.753095E-2,-7.792488E-3,5.3035673E-3,-1.4621284E-3,-7.4197287E-3,3.9361496E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,15,17,19,-1,-1,-1,-1,-1],"loss_changes":[3.8919637E-1,3.219021E-1,1.5125433E-1,2.8501004E-1,3.5851365E-1,0E0,0E0,8.663893E-2,2.0514207E-1,0E0,0E0,0E0,0E0,9.4843596E-2,9.508543E-2,1.3838893E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,13,13,14,14,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,16,18,20,-1,-1,-1,-1,-1],"split_conditions":[1.1997641E0,3.698723E10,1.8199778E0,3.6130127E-1,3.05E3,1.0435131E-2,-3.025378E-4,2.749683E-1,3.435678E8,-2.0189717E-2,-1.5987346E-3,9.029199E-4,6.8847295E-3,1.06502E5,1.76728E5,2.047268E-6,-7.792488E-3,5.3035673E-3,-1.4621284E-3,-7.4197287E-3,3.9361496E-3],"split_indices":[51,5,51,39,0,0,0,39,12,0,0,0,0,12,41,50,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.9E1,1.1E1,6.1E1,8E0,7E0,4E0,2E1,4.1E1,3E0,5E0,1.2E1,8E0,2.2E1,1.9E1,7E0,1.5E1,5E0,1.4E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.486244E-2,-8.568137E-3,-1.5841356E-2,-9.3234494E-2,5.2669656E-2,-6.0465336E-3,-8.464897E-3,6.17062E-3,2.975466E-2,4.510275E-3,-3.82547E-3,7.199694E-2,-1.1306953E-3,-2.3136789E-3,4.389466E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,-1,5,7,-1,9,-1,11,-1,-1,13,-1,-1,-1],"loss_changes":[6.4122254E-1,3.8339964E-1,0E0,1.0389054E-1,7.7996895E-2,0E0,9.828046E-2,0E0,8.261073E-2,0E0,0E0,6.566714E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,11,11],"right_children":[2,4,-1,6,8,-1,10,-1,12,-1,-1,14,-1,-1,-1],"split_conditions":[1.5414109E8,3.3580672E8,-1.5841356E-2,4.593E3,3.5010372E5,-6.0465336E-3,2.756E3,6.17062E-3,6.3315526E8,4.510275E-3,-3.82547E-3,7.767353E2,-1.1306953E-3,-2.3136789E-3,4.389466E-3],"split_indices":[57,7,0,2,44,0,10,0,7,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.2E1,5E0,3E1,4.2E1,2E1,1E1,8E0,3.4E1,4E0,6E0,1.9E1,1.5E1,3E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.1276435E-2,-5.21671E-2,1.3528326E-1,-2.2104222E-1,-2.426575E-3,9.068824E-2,1.7464217E-2,-4.5079584E-3,-1.4658926E-2,-5.468828E-3,2.1263339E-2,7.589699E-3,8.103197E-4,7.0716836E-3,-1.187136E-2,-2.8894562E-3,1.5143522E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,13,-1,-1,-1,15,-1,-1],"loss_changes":[6.839957E-1,3.6906084E-1,3.544314E-1,9.195417E-2,1.004384E-1,1.6975895E-1,0E0,0E0,0E0,0E0,1.355651E-1,0E0,0E0,0E0,6.0674407E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,10,10,14,14],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,14,-1,-1,-1,16,-1,-1],"split_conditions":[1.9377503E-1,3.2716873E0,7.6312915E3,8.903272E7,1.43E2,2.4696612E0,1.7464217E-2,-4.5079584E-3,-1.4658926E-2,-5.468828E-3,5.93348E5,7.589699E-3,8.103197E-4,7.0716836E-3,1.4865639E-2,-2.8894562E-3,1.5143522E-3],"split_indices":[51,68,64,57,10,65,0,0,0,0,12,0,0,0,50,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,4.3E1,3.4E1,9E0,3.4E1,3E1,4E0,5E0,4E0,5E0,2.9E1,1.4E1,1.6E1,5E0,2.4E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[3.3598235E-3,-3.1625196E-2,3.178517E-1,3.3148276E-4,-2.5855076E-1,2.5143519E-2,4.8810276E-3,-8.057785E-2,4.7814537E-2,3.9861184E-3,-1.9044844E-2,-1.021445E-1,3.130744E-3,1.3694708E-1,5.5963715E-4,-1.23283826E-1,1.5603699E-3,-9.846813E-4,8.475621E-3,9.173717E-4,-6.733092E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,-1,19,-1,-1,-1,-1,-1],"loss_changes":[9.8927486E-1,5.8748055E-1,3.8723922E-1,2.8046396E-1,5.944244E-1,0E0,0E0,9.41242E-2,1.438094E-1,0E0,0E0,7.3498964E-2,0E0,1.0667133E-1,0E0,7.844573E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,-1,20,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,1.523967E6,3.1457312E8,2.7307262E5,2.5143519E-2,4.8810276E-3,1.1808436E3,1.7329262E7,3.9861184E-3,-1.9044844E-2,4.3646493E0,3.130744E-3,1.9222778E5,5.5963715E-4,1.2015E4,1.5603699E-3,-9.846813E-4,8.475621E-3,9.173717E-4,-6.733092E-3],"split_indices":[64,57,41,7,40,0,0,64,57,0,0,66,0,40,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.8E1,8E1,8E0,7.1E1,9E0,3E0,5E0,2.6E1,4.5E1,3E0,6E0,2.3E1,3E0,1.2E1,3.3E1,2E1,3E0,3E0,9E0,3E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.2610001E-2,-2.2880238E-2,3.0696085E-1,-1.3691573E-1,4.042259E-2,2.2040514E-2,3.0416315E-3,-3.4794535E-3,-1.3534963E-2,5.6738425E-2,-7.1413415E-3,1.5000735E-1,1.1245539E-2,3.0404849E-3,9.727519E-3,1.6965141E-3,-3.259636E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,-1,13,15,-1,-1,-1,-1],"loss_changes":[8.740896E-1,5.469421E-1,3.5310042E-1,2.395255E-1,1.670124E-1,0E0,0E0,0E0,0E0,1.9349444E-1,0E0,6.570196E-2,7.211884E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,11,11,12,12],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,-1,14,16,-1,-1,-1,-1],"split_conditions":[4.9891987E3,9.254704E-3,8.982776E9,9.167896E-1,2.953641E5,2.2040514E-2,3.0416315E-3,-3.4794535E-3,-1.3534963E-2,1.7329262E7,-7.1413415E-3,4.189857E8,5.6064875E5,3.0404849E-3,9.727519E-3,1.6965141E-3,-3.259636E-3],"split_indices":[4,50,5,39,45,0,0,0,0,57,0,7,40,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,2.6E1,4.8E1,4E0,4E0,2E1,6E0,4.5E1,3E0,1.4E1,3.1E1,7E0,7E0,2.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[8.07689E-3,-1.09357E-2,1.7536564E-2,-4.4193663E-2,7.8753196E-2,-3.1009184E-2,-9.8837875E-3,7.3611983E-3,8.85675E-4,-6.987723E-2,3.6265536E-3,-4.595631E-3,7.864878E-3,2.485531E-3,-5.0534895E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,-1,-1,13,-1,-1],"loss_changes":[6.010925E-1,2.4211E-1,0E0,1.2823805E-1,1.0554172E-1,2.4696372E-1,0E0,0E0,0E0,1.0318862E-1,0E0,0E0,7.8745686E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9,12,12],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,-1,-1,14,-1,-1],"split_conditions":[7.6312915E3,7.370791E-1,1.7536564E-2,7.488E4,2.4696612E0,1.204781E3,-9.8837875E-3,7.3611983E-3,8.85675E-4,4.6911945E0,3.6265536E-3,-4.595631E-3,9.250027E7,2.485531E-3,-5.0534895E-3],"split_indices":[64,54,0,10,65,64,0,0,0,66,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,5.8E1,2.1E1,5.5E1,3E0,8E0,1.3E1,4.1E1,1.4E1,2.9E1,1.2E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-7.5713675E-3,2.3314428E-2,-2.435212E-1,3.3711407E-3,1.5731681E-2,3.7492139E-3,-1.9081488E-2,8.283984E-2,-3.0517051E-2,-1.2869912E-3,6.0156733E-3,-2.1319406E-3,5.312631E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[5.668149E-1,4.4363204E-1,5.536874E-1,1.8026936E-1,0E0,0E0,0E0,1.1423214E-1,1.1955239E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[1.5414109E8,2.1769184E7,2.7307262E5,2.5511668E7,1.5731681E-2,3.7492139E-3,-1.9081488E-2,1.938E3,1.6132394E2,-1.2869912E-3,6.0156733E-3,-2.1319406E-3,5.312631E-3],"split_indices":[57,1,40,57,0,0,0,2,69,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.8E1,8E0,6.5E1,3E0,3E0,5E0,1.9E1,4.6E1,6E0,1.3E1,4.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-9.670357E-3,2.7642703E-2,-1.7569067E-1,-5.719594E-2,5.997518E-2,-6.748999E-2,-1.7186323E-2,-5.451616E-3,3.077013E-3,9.255296E-3,4.6157647E-2,-7.4664084E-3,5.1873777E-4,8.2818896E-2,-1.2600952E-3,7.055182E-3,3.5118975E-2,-3.0096872E-3,4.304777E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,13,-1,-1,15,-1,-1,17,-1,-1],"loss_changes":[5.066293E-1,1.8721424E-1,3.0376294E-1,1.574944E-1,8.8023394E-2,8.607198E-2,0E0,0E0,0E0,0E0,1.269522E-1,0E0,0E0,1.01892844E-1,0E0,0E0,1.2740164E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,10,10,13,13,16,16],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,14,-1,-1,16,-1,-1,18,-1,-1],"split_conditions":[1.0129378E12,2.4786325E0,2.1446E4,1E0,7.446808E-2,2.277961E6,-1.7186323E-2,-5.451616E-3,3.077013E-3,9.255296E-3,6.585265E-1,-7.4664084E-3,5.1873777E-4,2.5719457E0,-1.2600952E-3,7.055182E-3,3.7050848E0,-3.0096872E-3,4.304777E-3],"split_indices":[43,68,9,25,69,1,0,0,0,0,39,0,0,65,0,0,66,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.6E1,1.4E1,1.8E1,4.8E1,1E1,4E0,1.2E1,6E0,3E0,4.5E1,4E0,6E0,3E1,1.5E1,1.1E1,1.9E1,7E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.0212095E-2,-4.35189E-2,3.1996912E-1,-1.5490663E-2,-2.7276715E-2,2.0518646E-2,3.4017672E-3,-1.2091547E-1,1.7533585E-3,1.190538E-3,-7.4824425E-3,1.6422926E-2,-9.263279E-3,3.747566E-3,-9.785032E-3,-3.0098208E-3,1.1517399E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,-1,7,-1,-1,9,11,-1,-1,13,-1,-1,15,-1,-1],"loss_changes":[9.596909E-1,3.7270862E-1,2.2494537E-1,0E0,2.0680583E-1,0E0,0E0,1.2063053E-1,1.8148267E-1,0E0,0E0,9.7865336E-2,0E0,0E0,8.31477E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,8,8,11,11,14,14],"right_children":[2,4,6,-1,8,-1,-1,10,12,-1,-1,14,-1,-1,16,-1,-1],"split_conditions":[4.0230347E3,7E0,8.982776E9,-1.5490663E-2,1.7363004E4,2.0518646E-2,3.4017672E-3,1.2015E4,2.5805172E7,1.190538E-3,-7.4824425E-3,1.2434905E5,-9.263279E-3,3.747566E-3,1.1451406E6,-3.0098208E-3,1.1517399E-3],"split_indices":[64,3,5,0,63,0,0,9,59,0,0,45,0,0,59,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,7.8E1,7E0,3E0,7.5E1,4E0,3E0,1.7E1,5.8E1,4E0,1.3E1,5.5E1,3E0,1.5E1,4E1,1.5E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.6884295E-2,9.294933E-3,-2.5365922E-1,-1.0374153E-1,4.54621E-2,-1.8104224E-2,-3.9950278E-4,-1.8567771E-3,-8.042989E-3,-4.076919E-3,6.048501E-2,1.5845771E-1,2.6276544E-2,1.021464E-3,8.942584E-3,-2.244895E-3,2.4556583E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,-1,-1,-1,-1,11,13,15,-1,-1,-1,-1],"loss_changes":[4.9824452E-1,3.0315554E-1,2.8083003E-1,7.581639E-2,1.1923392E-1,0E0,0E0,0E0,0E0,0E0,1.6676259E-1,6.3678145E-2,8.4835514E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,10,10,11,11,12,12],"right_children":[2,4,6,8,10,-1,-1,-1,-1,-1,12,14,16,-1,-1,-1,-1],"split_conditions":[1.5414109E8,1.528E3,1.6E1,2.62518E5,2E0,-1.8104224E-2,-3.9950278E-4,-1.8567771E-3,-8.042989E-3,-4.076919E-3,1.6939252E7,1.51E3,7.4325244E2,1.021464E-3,8.942584E-3,-2.244895E-3,2.4556583E-3],"split_indices":[57,11,3,41,65,0,0,0,0,0,57,2,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.2E1,7E0,1.7E1,5.5E1,4E0,3E0,1E1,7E0,5E0,5E1,1.2E1,3.8E1,3E0,9E0,1E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.1354432E-3,4.208336E-2,-1.4679512E-1,1.1537083E-2,2.5828347E-1,-6.279996E-2,-1.9296477E-2,4.5042545E-2,-5.9607945E-2,2.097079E-2,1.9830624E-3,-7.220609E-3,5.6147703E-5,-4.021973E-3,6.6665635E-2,-5.8822283E-3,2.396807E-3,5.8200946E-3,3.311587E-2,-4.12019E-3,2.5028542E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,-1,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[5.028884E-1,4.262872E-1,3.9404494E-1,1.4094222E-1,3.2949197E-1,9.2660844E-2,0E0,1.21496454E-1,1.562795E-1,0E0,0E0,0E0,0E0,0E0,6.802182E-2,0E0,0E0,0E0,7.0468634E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,14,14,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,-1,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[1.0232127E12,6.7780703E3,1.5135763E8,7.178207E7,9.192915E0,4.375793E6,-1.9296477E-2,1.543E3,1.272079E6,2.097079E-2,1.9830624E-3,-7.220609E-3,5.6147703E-5,-4.021973E-3,1.7329262E7,-5.8822283E-3,2.396807E-3,5.8200946E-3,4.57563E5,-4.12019E-3,2.5028542E-3],"split_indices":[43,4,57,57,69,1,0,2,41,0,0,0,0,0,57,0,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,6.4E1,1.7E1,5.7E1,7E0,1.4E1,3E0,3.9E1,1.8E1,3E0,4E0,5E0,9E0,5E0,3.4E1,1.1E1,7E0,1.1E1,2.3E1,3E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.0192863E-2,-2.550403E-2,2.146006E-1,3.5289207E-3,-1.7890768E-1,1.8153375E-2,1.1402272E-3,-5.259428E-3,2.1828413E-2,-1.303503E-2,3.8166344E-4,5.2756134E-2,-1.6452104E-2,5.3551286E-3,2.8831797E-4,-4.586093E-3,1.1037919E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,-1,13,15,-1,-1,-1,-1],"loss_changes":[5.8921087E-1,3.0794084E-1,4.1128135E-1,1.3142845E-1,2.2543594E-1,0E0,0E0,0E0,6.2432274E-2,0E0,0E0,8.700771E-2,8.5205E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,11,11,12,12],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,-1,14,16,-1,-1,-1,-1],"split_conditions":[2.6949062E3,1.23745944E8,1.3946067E7,2.1E2,2.009E3,1.8153375E-2,1.1402272E-3,-5.259428E-3,1.219233E6,-1.303503E-2,3.8166344E-4,8.903939E0,2.6349305E6,5.3551286E-3,2.8831797E-4,-4.586093E-3,1.1037919E-3],"split_indices":[64,57,59,10,0,0,0,0,44,0,0,68,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,6.8E1,1.1E1,5.8E1,1E1,5E0,6E0,7E0,5.1E1,6E0,4E0,2.8E1,2.3E1,1.1E1,1.7E1,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-9.3867835E-3,-7.210561E-2,3.2621324E-2,-1.0935204E-1,3.4745003E-3,2.2722995E-3,-3.4794041E-3,-5.061852E-4,-7.718577E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[2.1049342E-1,1.8553717E-1,9.527965E-2,1.5303174E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[6.015076E-2,1.0593817E3,1E0,3.7582534E5,3.4745003E-3,2.2722995E-3,-3.4794041E-3,-5.061852E-4,-7.718577E-3],"split_indices":[51,64,28,40,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,3.1E1,4.7E1,2.5E1,6E0,4.1E1,6E0,1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.1651403E-3,-1.0073047E-1,5.852907E-2,2.4681927E-3,-6.70188E-3,1.306708E-2,2.9839732E-2,4.44482E-2,-5.017347E-3,5.1826257E-3,7.4827974E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,-1,-1,-1,7,9,-1,-1,-1],"loss_changes":[4.5929107E-1,2.0444885E-1,3.5326254E-1,0E0,0E0,0E0,1.0743919E-1,8.7802075E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,6,6,7,7],"right_children":[2,4,6,-1,-1,-1,8,10,-1,-1,-1],"split_conditions":[6.015076E-2,3.027892E5,1E0,2.4681927E-3,-6.70188E-3,1.306708E-2,2.9461394E5,1.7329262E7,-5.017347E-3,5.1826257E-3,7.4827974E-4],"split_indices":[51,40,13,0,0,0,45,57,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.6E1,5.4E1,6E0,2E1,5E0,4.9E1,4.5E1,4E0,1.2E1,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[8.715846E-3,-1.3011844E-2,2.557872E-1,1.04121305E-2,-1.2418287E-1,2.0695774E-2,-3.7566587E-4,-3.0496349E-3,2.3610529E-2,3.0610838E-3,-1.080985E-2,1.0008543E-1,1.6263577E-4,-1.300933E-3,7.628168E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,-1,13,-1,-1,-1],"loss_changes":[4.6500117E-1,2.096184E-1,3.7367466E-1,7.0200115E-2,3.217277E-1,0E0,0E0,0E0,8.8475876E-2,0E0,0E0,1.0979917E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,11,11],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,-1,14,-1,-1,-1],"split_conditions":[4.0230347E3,1.15266744E8,1.1408508E9,1.692E3,2.7307262E5,2.0695774E-2,-3.7566587E-4,-3.0496349E-3,1.6939252E7,3.0610838E-3,-1.080985E-2,1.9222778E5,1.6263577E-4,-1.300933E-3,7.628168E-3],"split_indices":[64,57,7,2,40,0,0,0,57,0,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,7.9E1,6E0,6.6E1,1.3E1,3E0,3E0,9E0,5.7E1,5E0,8E0,1.1E1,4.6E1,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-2.5658377E-3,-1.9008258E-2,1.3534548E-2,1.7445772E-3,-4.792605E-2,-8.462399E-2,2.6130311E-2,-2.6886484E-3,-9.759848E-3,4.1133803E-3,-1.8728019E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,-1,5,7,9,-1,-1,-1,-1],"loss_changes":[3.814488E-1,1.2309592E-1,0E0,0E0,1.3537934E-1,9.652951E-2,7.7606894E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6],"right_children":[2,4,-1,-1,6,8,10,-1,-1,-1,-1],"split_conditions":[2.1769184E7,3.6130127E-1,1.3534548E-2,1.7445772E-3,3.7956493E0,2.1323267E5,5.1929092E0,-2.6886484E-3,-9.759848E-3,4.1133803E-3,-1.8728019E-3],"split_indices":[1,39,0,0,69,45,68,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,2.4E1,4.8E1,3.2E1,1.6E1,2.8E1,4E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-5.7976006E-4,-1.7966663E-2,1.197601E-2,-6.812497E-3,-9.85249E-3,-5.9205882E-2,2.2098739E-3,-4.661312E-3,6.992995E-3,-1.9702544E-3,4.4586607E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[3.7250337E-1,1.6727796E-1,0E0,2.1512862E-1,0E0,1.10087514E-1,0E0,0E0,7.782072E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[7.6312915E3,2.5805172E7,1.197601E-2,4.283039E1,-9.85249E-3,4.492E3,2.2098739E-3,-4.661312E-3,3.6391603E5,-1.9702544E-3,4.4586607E-3],"split_indices":[64,59,0,70,0,2,0,0,40,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.5E1,4E0,7.2E1,3E0,3.7E1,3.5E1,2.2E1,1.5E1,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[6.2044817E-3,-1.9131783E-2,2.1665064E-1,5.141827E-3,-1.0126108E-2,3.021888E-3,1.7595652E-2,3.4763984E-2,-1.999493E-3,-3.4479168E-3,5.84022E-2,-1.2001025E-3,3.6221764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,-1,-1,9,-1,-1,11,-1,-1],"loss_changes":[4.4669423E-1,3.7241343E-1,2.051202E-1,1.0061146E-1,0E0,0E0,0E0,1.16973855E-1,0E0,0E0,6.7438886E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,10,10],"right_children":[2,4,6,8,-1,-1,-1,10,-1,-1,12,-1,-1],"split_conditions":[4.9891987E3,1.15266744E8,1.3996E4,6.0487356E7,-1.0126108E-2,3.021888E-3,1.7595652E-2,2.1113522E8,-1.999493E-3,-3.4479168E-3,2.5630938E5,-1.2001025E-3,3.6221764E-3],"split_indices":[4,57,9,57,0,0,0,7,0,0,40,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,6.7E1,7E0,5E0,3E0,4.2E1,2.5E1,7E0,3.5E1,7E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[2.6789831E-4,-3.0439407E-2,2.356608E-1,-1.149268E-1,4.0483984E-4,1.5791671E-2,3.8265807E-4,-7.207675E-2,-1.3152456E-2,-4.7107995E-3,1.6990533E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,-1,-1,9,-1,-1,-1],"loss_changes":[5.637511E-1,2.3046486E-1,2.2654772E-1,1.4390677E-1,0E0,0E0,0E0,6.9831535E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7],"right_children":[2,4,6,8,-1,-1,-1,10,-1,-1,-1],"split_conditions":[3.3072104E3,8.872871E-3,8.982776E9,3.1457312E8,4.0483984E-4,1.5791671E-2,3.8265807E-4,1.962943E3,-1.3152456E-2,-4.7107995E-3,1.6990533E-3],"split_indices":[64,53,5,7,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.8E1,8E0,2.1E1,4.7E1,5E0,3E0,1.8E1,3E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.5484409E-2,-8.5234456E-4,1.5730172E-2,1.3718014E-2,-1.0349155E-2,4.2829835E-3,-1.2043758E-2,-4.8402636E-3,4.087845E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[4.5997488E-1,2.74492E-1,0E0,1.6315766E-1,0E0,0E0,1.2040965E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.5730172E-2,2.6529046E7,-1.0349155E-2,4.2829835E-3,2.23E2,-4.8402636E-3,4.087845E-4],"split_indices":[64,57,0,57,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8.1E1,3E0,7.7E1,4E0,1.8E1,5.9E1,1E1,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-2.1572858E-2,-2.590029E-3,-1.8972172E-1,-3.4217546E-3,2.6854577E-2,-2.2767416E-4,-1.6940288E-2,-2.1203437E-3,9.218067E-2,-9.4538985E-4,7.1454714E-3,6.9168685E-3,5.8762816E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,-1,7,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[2.51672E-1,1.5633376E-1,2.6920703E-1,0E0,9.8621994E-2,0E0,0E0,1.1368672E-1,7.433452E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,8,8],"right_children":[2,4,6,-1,8,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.0843103E5,9.465995E11,-3.4217546E-3,1.0379711E0,-2.2767416E-4,-1.6940288E-2,3.3795098E3,1.383E4,-9.4538985E-4,7.1454714E-3,6.9168685E-3,5.8762816E-4],"split_indices":[57,45,43,0,54,0,0,4,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.1E1,7E0,2E1,5.1E1,4E0,3E0,3.6E1,1.5E1,3.3E1,3E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-5.4015536E-3,-9.909809E-3,7.4888035E-3,-4.013894E-3,2.8230285E-2,9.309853E-2,6.8908436E-3,5.8700168E-3,-8.9289894E-4,-4.5935307E-3,8.0707384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,-1,3,-1,5,7,9,-1,-1,-1,-1],"loss_changes":[2.4039344E-1,0E0,1.6760945E-1,0E0,9.487187E-2,7.196176E-2,6.416873E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,6,6],"right_children":[2,-1,4,-1,6,8,10,-1,-1,-1,-1],"split_conditions":[7E0,-9.909809E-3,1.648E3,-4.013894E-3,3.5175372E5,1.55E2,4.279412E0,5.8700168E-3,-8.9289894E-4,-4.5935307E-3,8.0707384E-4],"split_indices":[3,0,11,0,44,8,70,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,4E0,8.2E1,1.4E1,6.8E1,1.6E1,5.2E1,1.2E1,4E0,4E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[2.1625163E-2,-1.984119E-4,1.962358E-1,1.3871906E-2,-8.821594E-3,1.6148694E-2,2.596427E-3,8.046712E-2,-9.44705E-3,-2.973778E-4,5.652692E-3,-2.7819995E-3,1.3895263E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[3.0869982E-1,2.020938E-1,1.7924526E-1,1.079124E-1,0E0,0E0,0E0,6.981999E-2,1.0984966E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[4.9891987E3,1.2427474E8,3.6274084E9,1.7329262E7,-8.821594E-3,1.6148694E-2,2.596427E-3,2.4460206E0,7.4325244E2,-2.973778E-4,5.652692E-3,-2.7819995E-3,1.3895263E-3],"split_indices":[4,57,12,57,0,0,0,68,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.2E1,8E0,6.8E1,4E0,3E0,5E0,1.7E1,5.1E1,6E0,1.1E1,2.2E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-9.763371E-3,2.0100173E-2,-1.3420676E-1,-1.8445465E-3,4.093017E-3,-2.5558094E-3,-1.1560569E-2,-2.142723E-2,2.657964E-3,-1.9209036E-3,3.771865E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,-1,-1,-1,9,-1,-1,-1],"loss_changes":[3.0743584E-1,1.0318126E-1,1.3581195E-1,6.275613E-2,0E0,0E0,0E0,9.0841785E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7],"right_children":[2,4,6,8,-1,-1,-1,10,-1,-1,-1],"split_conditions":[1.0232127E12,1.7830602E3,3.5715E4,5.0175633E0,4.093017E-3,-2.5558094E-3,-1.1560569E-2,9.3829626E-1,2.657964E-3,-1.9209036E-3,3.771865E-3],"split_indices":[43,64,9,66,0,0,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,6.6E1,1.5E1,5.1E1,1.5E1,1E1,5E0,3.9E1,1.2E1,3.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.946521E-3,7.325537E-3,-7.642583E-3,-2.1125557E-2,3.5148093E-3,-5.8077894E-2,9.581131E-4,-1.4777175E-3,-7.671093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,-1,5,-1,7,-1,-1,-1],"loss_changes":[1.1695583E-1,1.4523713E-1,0E0,8.2374796E-2,0E0,7.376614E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5],"right_children":[2,4,-1,6,-1,8,-1,-1,-1],"split_conditions":[1.04649E5,1.1808436E3,-7.642583E-3,1.9129465E0,3.5148093E-3,1.0789844E6,9.581131E-4,-1.4777175E-3,-7.671093E-3],"split_indices":[10,64,0,69,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7.1E1,3E0,5.1E1,2E1,2.7E1,2.4E1,2.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-4.7242325E-3,1.1173487E-2,-1.0996749E-2,-1.4070056E-3,1.066009E-2,1.0290867E-2,-4.370476E-3,-3.4634564E-3,1.0012541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,-1,7,-1,-1,-1],"loss_changes":[3.0101395E-1,2.1135761E-1,0E0,8.0894165E-2,0E0,6.790093E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5],"right_children":[2,4,-1,6,-1,8,-1,-1,-1],"split_conditions":[2.9851845E10,2.1747866E7,-1.0996749E-2,1E0,1.066009E-2,1.1E1,-4.370476E-3,-3.4634564E-3,1.0012541E-3],"split_indices":[5,1,0,28,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.4E1,4E0,7.1E1,3E0,6.4E1,7E0,7E0,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-3.0797683E-3,-1.6179575E-2,9.521765E-3,8.627425E-4,-1.24709114E-1,-4.4682864E-3,6.8246416E-4,3.5105532E-3,-2.1014167E-1,-1.4074667E-2,-1.4854787E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,7,-1,-1,-1,9,-1,-1],"loss_changes":[2.4037956E-1,1.4992478E-1,0E0,1.0138416E-1,2.0612189E-1,0E0,0E0,0E0,1.3165593E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,8,8],"right_children":[2,4,-1,6,8,-1,-1,-1,10,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,9.521765E-3,2.1199509E2,2.7307262E5,-4.4682864E-3,6.8246416E-4,3.5105532E-3,1.835052E6,-1.4074667E-2,-1.4854787E-3],"split_indices":[64,57,0,64,40,0,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8E1,4E0,7E1,1E1,8E0,6.2E1,3E0,7E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.38782915E-2,-9.374573E-3,-3.4195262E-3,7.8524565E-3,-9.326897E-3,-2.6572239E-2,4.5003485E-2,-4.534099E-2,4.828666E-3,3.172525E-3,-2.1659809E-3,-3.0889441E-3,2.3585141E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,-1,3,5,-1,7,9,11,-1,-1,-1,-1,-1],"loss_changes":[1.5644011E-1,0E0,1.724184E-1,9.3277566E-2,0E0,9.963878E-2,8.408406E-2,7.8644216E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7],"right_children":[2,-1,4,6,-1,8,10,12,-1,-1,-1,-1,-1],"split_conditions":[7E0,-9.374573E-3,2.292109E7,7.668801E2,-9.326897E-3,9.3829626E-1,6.349519E6,5.0175633E0,4.828666E-3,3.172525E-3,-2.1659809E-3,-3.0889441E-3,2.3585141E-3],"split_indices":[3,0,59,64,0,51,44,66,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3E0,7.4E1,7.1E1,3E0,3.7E1,3.4E1,3.3E1,4E0,2.7E1,7E0,2.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-7.1565825E-3,7.26496E-2,-3.5928532E-2,9.221539E-4,1.0716014E-2,-1.4468934E-2,-9.96424E-3,-1.252386E-3,3.4358194E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[1.817122E-1,1.7480168E-1,2.2530784E-1,0E0,0E0,6.477712E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[2.793975E5,3.3795098E3,1.5135763E8,9.221539E-4,1.0716014E-2,2.5536325E1,-9.96424E-3,-1.252386E-3,3.4358194E-3],"split_indices":[40,4,57,0,0,69,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2E1,5.7E1,1.6E1,4E0,5.2E1,5E0,4.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-5.4796E-3,1.9775677E-2,-9.2165574E-2,-2.266367E-3,4.3082923E-2,-1.2887701E-3,-1.3994516E-2,2.6401342E-3,-4.321061E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,-1,7,-1,-1,-1,-1],"loss_changes":[1.7477562E-1,1.0287234E-1,2.3934117E-1,0E0,1.0351117E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4],"right_children":[2,4,6,-1,8,-1,-1,-1,-1],"split_conditions":[1.0232127E12,3.69E2,1.5135763E8,-2.266367E-3,4.2E1,-1.2887701E-3,-1.3994516E-2,2.6401342E-3,-4.321061E-3],"split_indices":[43,0,57,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,6.1E1,1.7E1,1.5E1,4.6E1,1.4E1,3E0,4.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[5.3306096E-3,-5.2761934E-3,1.065405E-2,4.7322554E-3,-9.7206645E-3,-1.8386219E-2,2.2339686E-3,6.0350844E-4,-7.18206E-2,-4.88276E-3,6.821424E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[2.2019327E-1,1.8287367E-1,0E0,8.7281644E-2,0E0,9.575405E-2,0E0,0E0,6.6800065E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[2.3650324E7,5.139E4,1.065405E-2,1E0,-9.7206645E-3,6.5957415E-1,2.2339686E-3,6.0350844E-4,2.6739E4,-4.88276E-3,6.821424E-4],"split_indices":[1,2,0,23,0,39,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,8.6E1,3E0,8.3E1,3E0,5.5E1,2.8E1,3.5E1,2E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.7933674E-2,5.138807E-3,1.3388715E-2,1.4017696E-2,-6.5847565E-3,-1.7990405E-2,2.5041248E-3,1.2668338E-2,-5.039872E-3,-3.2994559E-3,1.7576927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,-1,7,-1,9,-1,-1,-1],"loss_changes":[3.2100883E-1,1.1938551E-1,0E0,1.1161859E-1,0E0,1.3831407E-1,0E0,8.508502E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7],"right_children":[2,4,-1,6,-1,8,-1,10,-1,-1,-1],"split_conditions":[7.6312915E3,2.5805172E7,1.3388715E-2,1.7807747E3,-6.5847565E-3,6.8989144E7,2.5041248E-3,2.1113522E8,-5.039872E-3,-3.2994559E-3,1.7576927E-3],"split_indices":[64,59,0,4,0,57,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9E1,8.7E1,3E0,8.3E1,4E0,4.7E1,3.6E1,3.6E1,1.1E1,8E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-7.5993245E-3,4.927036E-3,-1.4443937E-1,-1.1511139E-2,5.7920787E-2,-4.721822E-6,-1.3107784E-2,7.2711875E-4,-4.908481E-2,7.1502402E-3,7.6964404E-4,1.247663E-3,-3.6117658E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,-1,-1,-1],"loss_changes":[1.635519E-1,7.7325344E-2,1.6660151E-1,7.112228E-2,8.30067E-2,0E0,0E0,0E0,6.98636E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.1879E4,1.6867E4,5.050553E7,1.174E3,-4.721822E-6,-1.3107784E-2,7.2711875E-4,1.0671435E5,7.1502402E-3,7.6964404E-4,1.247663E-3,-3.6117658E-3],"split_indices":[57,2,9,57,0,0,0,0,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,8.7E1,7E0,6.7E1,2E1,4E0,3E0,3.9E1,2.8E1,5E0,1.5E1,8E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.3867403E-2,-6.384688E-2,6.2379736E-4,-3.8041612E-3,3.7141074E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":70,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.1264754E-1,9.1148265E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[2.289789E6,1.962943E3,6.2379736E-4,-3.8041612E-3,3.7141074E-3],"split_indices":[63,4,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[8E1,2.8E1,5.2E1,2.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[1.6984077E-2,3.514702E-2,-9.330004E-2,4.5772944E-2,-4.5391903E-3,-1.6594352E-3,-9.616353E-3,9.107207E-3,1.538382E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[1.8355677E-1,1.16583966E-1,7.9698436E-2,1.2753937E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[1.1269586E12,3.3E1,1.21144904E8,1E0,-4.5391903E-3,-1.6594352E-3,-9.616353E-3,9.107207E-3,1.538382E-3],"split_indices":[43,3,57,13,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,7.7E1,1.2E1,7.2E1,5E0,9E0,3E0,4E0,6.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[9.96357E-3,-2.306457E-4,1.0173036E-2,-3.384038E-2,1.5881503E-3,-5.5155467E-4,-6.661886E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[1.9215132E-1,1.0178905E-1,0E0,1.05978005E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[7.6312915E3,6.695231E8,1.0173036E-2,1.2427474E8,1.5881503E-3,-5.5155467E-4,-6.661886E-3],"split_indices":[64,12,0,57,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.4E1,3E0,4.3E1,4.1E1,3.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-5.2602747E-3,4.6011177E-3,-7.890769E-3,-1.9764243E-2,2.4138028E-3,1.9660324E-2,-4.0929504E-3,-3.895376E-3,3.8241062E-2,5.097977E-3,4.5619239E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[1.504045E-1,1.0282984E-1,0E0,1.6274767E-1,0E0,7.775104E-2,0E0,0E0,6.777253E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[1.3866357E8,1.2544873E3,-7.890769E-3,5.501383E9,2.4138028E-3,1.001E3,-4.0929504E-3,-3.895376E-3,3.5175372E5,5.097977E-3,4.5619239E-4],"split_indices":[60,64,0,5,0,2,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,8.5E1,4E0,5.7E1,2.8E1,3.7E1,2E1,5E0,3.2E1,8E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[6.617562E-3,1.732687E-2,-8.768477E-3,3.304512E-3,1.1888619E-2,1.1765056E-3,-1.7858057E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[1.6808592E-1,2.5290862E-1,0E0,6.94834E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[2.6293537E9,3.9509753E3,-8.768477E-3,6.237206E5,1.1888619E-2,1.1765056E-3,-1.7858057E-3],"split_indices":[7,64,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7.3E1,3E0,7E1,3E0,4.6E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-7.935781E-3,2.0136061E-4,-1.368447E-1,-3.9872536E-4,-1.1925757E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":75,"left_children":[1,-1,3,-1,-1],"loss_changes":[1.4573367E-1,0E0,1.2664032E-1,0E0,0E0],"parents":[2147483647,0,0,2,2],"right_children":[2,-1,4,-1,-1],"split_conditions":[1.5414109E8,2.0136061E-4,1.0232127E12,-3.9872536E-4,-1.1925757E-2],"split_indices":[57,0,43,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[9E1,8.3E1,7E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-6.4488803E-3,8.457503E-3,-3.7056662E-3,-6.003216E-3,7.8336825E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":76,"left_children":[1,3,-1,-1,-1],"loss_changes":[8.8104926E-2,8.3993495E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[1.0789844E6,4.8595375E6,-3.7056662E-3,-6.003216E-3,7.8336825E-4],"split_indices":[40,57,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.7E1,6.5E1,1.2E1,3E0,6.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-3.3654657E-3,-3.856154E-2,5.9879422E-2,-1.0812437E-3,-6.7589222E-3,1.8020524E-1,1.2556026E-3,1.2313276E-2,2.0024928E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[2.115504E-1,9.567571E-2,1.2683016E-1,0E0,0E0,7.476516E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.204781E3,9.8788344E5,5.050553E7,-1.0812437E-3,-6.7589222E-3,2.2659068E2,1.2556026E-3,1.2313276E-2,2.0024928E-3],"split_indices":[64,40,57,0,0,70,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[9.3E1,6E1,3.3E1,5.4E1,6E0,6E0,2.7E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.0059733E-2,6.930898E-5,9.299449E-3,1.0508416E-2,-9.001446E-3,-1.4342793E-2,4.9091697E-2,1.1239252E-5,-5.178902E-3,5.0699622E-3,5.256647E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,-1],"loss_changes":[1.5956551E-1,1.6566943E-1,0E0,7.357604E-2,0E0,6.8426244E-2,8.9880824E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,9.299449E-3,1.204781E3,-9.001446E-3,1.2159766E6,1.61E4,1.1239252E-5,-5.178902E-3,5.0699622E-3,5.256647E-5],"split_indices":[1,2,0,64,0,40,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,7.8E1,3E0,7.5E1,3E0,4.6E1,2.9E1,4.1E1,5E0,1.2E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.2379639E-2,2.1649037E-2,-8.309817E-3,3.811644E-4,1.2768994E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":79,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.5975747E-1,2.8322613E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[1.3866357E8,7.467988E3,-8.309817E-3,3.811644E-4,1.2768994E-2],"split_indices":[60,4,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[8.5E1,8.2E1,3E0,7.9E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[2.059407E-3,-1.094061E-2,1.1282726E-2,-1.7198315E-4,-9.293876E-3,-2.5111423E-3,2.0255448E-2,4.2873058E-3,5.6649E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[2.5586018E-1,1.5856746E-1,0E0,8.335354E-2,0E0,0E0,7.603001E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[9.55721E3,3.698723E10,1.1282726E-2,3.2716873E0,-9.293876E-3,-2.5111423E-3,1.6939252E7,4.2873058E-3,5.6649E-5],"split_indices":[4,5,0,68,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.5E1,3E0,7.2E1,3E0,1.9E1,5.3E1,1E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[7.501351E-3,-1.2510346E-4,8.636701E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":81,"left_children":[1,-1,-1],"loss_changes":[1.4111617E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[2.1698572E7,-1.2510346E-4,8.636701E-3],"split_indices":[1,0,0],"split_type":[0,0,0],"sum_hessian":[7.4E1,7.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-8.957935E-3,-6.524408E-4,-5.9807505E-3,-2.98053E-2,3.0212386E-3,-4.897708E-4,-6.955797E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":82,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[8.0481276E-2,1.4892872E-1,0E0,1.22739546E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[1.06102504E8,1.0768752E3,-5.9807505E-3,1.0789844E6,3.0212386E-3,-4.897708E-4,-6.955797E-3],"split_indices":[63,64,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.4E1,4E0,5.2E1,2.2E1,4.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-1.0040184E-2,-1.3869822E-3,-6.64963E-3,7.540792E-3,-5.1184357E-3,-1.9647408E-5,5.413721E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[9.8973125E-2,8.03903E-2,0E0,6.6860616E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[1.3866357E8,1.58206E6,-6.64963E-3,5.744167E3,-5.1184357E-3,-1.9647408E-5,5.413721E-3],"split_indices":[60,40,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.9E1,4E0,7.4E1,5E0,7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-5.656308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0],"id":84,"left_children":[-1],"loss_changes":[0E0],"parents":[2147483647],"right_children":[-1],"split_conditions":[-2.5676878E-4],"split_indices":[0],"split_type":[0],"sum_hessian":[8.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"1","size_leaf_vector":"1"}},{"base_weights":[-4.73791E-3,4.0201985E-4,-1.2988947E-1,-1.0904912E-3,-1.0429583E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":85,"left_children":[1,-1,3,-1,-1],"loss_changes":[1.3895567E-1,0E0,7.9057544E-2,0E0,0E0],"parents":[2147483647,0,0,2,2],"right_children":[2,-1,4,-1,-1],"split_conditions":[1.5092398E8,4.0201985E-4,1.2341693E10,-1.0904912E-3,-1.0429583E-2],"split_indices":[57,0,5,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[8E1,7.3E1,7E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[5.446052E-3,-3.93284E-3,8.7005375E-3,4.3700873E-3,-7.3894626E-3,-5.6732236E-3,1.1302621E-2,3.8888215E-3,-5.949689E-3,-2.6462413E-3,7.861229E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[1.4571352E-1,1.0622374E-1,0E0,7.045723E-2,0E0,0E0,9.544734E-2,0E0,7.530051E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[7.6312915E3,2.5805172E7,8.7005375E-3,4.8595375E6,-7.3894626E-3,-5.6732236E-3,1.7329262E7,3.8888215E-3,5.057E3,-2.6462413E-3,7.861229E-4],"split_indices":[64,59,0,57,0,0,57,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,7.6E1,3E0,3E0,7.3E1,1.3E1,6E1,1.8E1,4.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[7.626577E-3,-7.469057E-3,1.7043252E-1,-5.500498E-3,5.3053506E-4,1.3618894E-2,-7.9486264E-5,3.2980405E-3,-1.51045965E-2,-2.9228773E-4,-6.1200242E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,-1,7,-1,-1,-1,9,-1,-1],"loss_changes":[2.02801E-1,6.919055E-2,1.567005E-1,0E0,8.2286485E-2,0E0,0E0,0E0,6.1376408E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8],"right_children":[2,4,6,-1,8,-1,-1,-1,10,-1,-1],"split_conditions":[4.0230347E3,3.235021E2,8.982776E9,-5.500498E-3,1.7329262E7,1.3618894E-2,-7.9486264E-5,3.2980405E-3,1.06102504E8,-2.9228773E-4,-6.1200242E-3],"split_indices":[64,4,5,0,57,0,0,0,63,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,7.5E1,6E0,4E0,7.1E1,3E0,3E0,1.2E1,5.9E1,5.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.9901391E-2,5.3101947E-4,8.032599E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":88,"left_children":[1,-1,-1],"loss_changes":[1.03575826E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[2.1769184E7,5.3101947E-4,8.032599E-3],"split_indices":[1,0,0],"split_type":[0,0,0],"sum_hessian":[8.1E1,7.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[3.2032232E-3,1.2378052E-2,-6.630849E-3,-5.4740156E-3,1.9038027E-2,3.3821217E-3,3.52253E-3,-5.244447E-4,4.2482405E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[1.1822005E-1,7.4024245E-2,0E0,0E0,6.686306E-2,0E0,8.5259095E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[1.3866357E8,8.192308E1,-6.630849E-3,-5.4740156E-3,1.6939252E7,3.3821217E-3,4.9891987E3,-5.244447E-4,4.2482405E-3],"split_indices":[60,64,0,0,57,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8E1,4E0,3E0,7.7E1,1.6E1,6.1E1,5.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.118779E-3,9.8658865E-3,-8.101967E-3,-1.9745291E-3,1.0951465E-2,-2.5728557E-3,1.629251E-2,3.2005355E-2,-2.2346193E-3,3.7445899E-3,3.3170648E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,-1,-1,7,9,-1,-1,-1],"loss_changes":[1.3529074E-1,2.2512814E-1,0E0,7.986421E-2,0E0,0E0,6.334442E-2,6.049308E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7],"right_children":[2,4,-1,6,-1,-1,8,10,-1,-1,-1],"split_conditions":[1.278211E8,7.467988E3,-8.101967E-3,2.0492524E-1,1.0951465E-2,-2.5728557E-3,1.9E1,2.325535E7,-2.2346193E-3,3.7445899E-3,3.3170648E-4],"split_indices":[60,4,0,69,0,0,3,57,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8.1E1,3E0,7.8E1,3E0,1.9E1,5.9E1,4.8E1,1.1E1,1.5E1,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-9.864961E-3,-9.6404285E-5,-5.4101585E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":91,"left_children":[1,-1,-1],"loss_changes":[7.675419E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[1.2460206E9,-9.6404285E-5,-5.4101585E-3],"split_indices":[7,0,0],"split_type":[0,0,0],"sum_hessian":[9E1,8.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[3.5360595E-3,-4.614405E-3,7.7052116E-3,-5.801842E-3,7.633809E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":92,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.1592519E-1,6.381655E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[2.1769184E7,7E0,7.7052116E-3,-5.801842E-3,7.633809E-5],"split_indices":[1,3,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[8.4E1,8.1E1,3E0,3E0,7.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[7.144518E-3,-5.7439455E-3,1.3684972E-1,1.6079901E-4,-4.854297E-3,1.0662121E-2,2.0942058E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,-1,-1,-1,-1],"loss_changes":[1.29507E-1,6.709531E-2,8.9652866E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2],"right_children":[2,4,6,-1,-1,-1,-1],"split_conditions":[4.0230347E3,3.698723E10,1.105E3,1.6079901E-4,-4.854297E-3,1.0662121E-2,2.0942058E-4],"split_indices":[64,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,6.5E1,5E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[2.5227757E-3,-4.5463177E-3,7.225969E-3,-5.2445647E-3,9.2411145E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":94,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.02606356E-1,6.525031E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[2.1698572E7,7E0,7.225969E-3,-5.2445647E-3,9.2411145E-5],"split_indices":[1,3,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[9.1E1,8.8E1,3E0,4E0,8.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[4.2173546E-3,1.3653257E-2,-5.6231036E-3,1.2788597E-4,1.0056997E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":95,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.0563272E-1,1.818459E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[1.278211E8,7.467988E3,-5.6231036E-3,1.2788597E-4,1.0056997E-2],"split_indices":[60,4,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[8.5E1,8E1,5E0,7.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-1.2008457E-2,1.0264133E-3,-4.2220328E-2,-2.3896996E-2,-6.726535E-3,-1.7682847E-3,3.6431071E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":96,"left_children":[1,-1,3,5,-1,-1,-1],"loss_changes":[8.253547E-2,0E0,7.800152E-2,6.114606E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3],"right_children":[2,-1,4,6,-1,-1,-1],"split_conditions":[5.4734116E9,1.0264133E-3,1.5414109E8,1.9612895E6,-6.726535E-3,-1.7682847E-3,3.6431071E-3],"split_indices":[5,0,57,40,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3.6E1,4.1E1,3.6E1,5E0,3.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"100"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-9.1459654E-2,-5.54109E-1,2.0924011E-1,-1.2843856E-1,-7.506931E-1,4.6708934E-2,4.3444833E-1,-3.00433E-1,5.9030973E-3,-4.0931627E-2,-2.0354155E-2,1.2529472E-1,-2.1817419E-1,1.044215E-2,5.1811427E-1,-1.7521013E-2,-4.962099E-3,1.7057521E-2,4.6703424E-2,-1.376186E-2,-2.5653557E-3,1.03572775E-2,2.624943E-2,1.5643975E-1,-6.770325E-2,2.220804E-3,1.0714781E-2,-2.6944376E-4,-4.600815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[1.0967188E1,2.4976635E0,1.7366686E0,5.456359E-1,7.3559284E-1,6.310014E-1,1.9185519E-1,9.352398E-2,0E0,0E0,0E0,4.759083E-1,9.545508E-2,0E0,1.946559E-1,0E0,0E0,0E0,2.4912825E-1,0E0,0E0,0E0,0E0,8.535066E-2,2.3812339E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,18,18,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[1.1576994E-2,5.8871865E-1,5.517496E-1,4.2332153E0,1.4235585E3,2.5042E4,4.650193E-1,1.264233E6,5.9030973E-3,-4.0931627E-2,-2.0354155E-2,2.9862975E5,2.9816154E1,1.044215E-2,2.4786325E0,-1.7521013E-2,-4.962099E-3,1.7057521E-2,7.75024E0,-1.376186E-2,-2.5653557E-3,1.03572775E-2,2.624943E-2,2.3797054E0,6.5957415E-1,2.220804E-3,1.0714781E-2,-2.6944376E-4,-4.600815E-3],"split_indices":[38,27,42,54,4,9,27,1,0,0,0,32,56,0,56,0,0,0,57,0,0,0,0,57,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3E1,4.7E1,1E1,2E1,2.8E1,1.9E1,6E0,4E0,1E1,1E1,2.2E1,6E0,7E0,1.2E1,3E0,3E0,4E0,1.8E1,3E0,3E0,4E0,8E0,9E0,9E0,5E0,4E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.1174776E-2,-4.830118E-1,2.093845E-1,-5.955108E-1,2.832456E-3,2.718551E-1,-1.1238484E-2,-4.2921847E-1,-3.8282927E-2,1.8877497E-1,2.3755293E-2,-2.0353394E-2,-6.2712687E-3,-7.041739E-3,2.28291E-1,1.6740719E-2,1.4373985E-1,-1.1000476E-3,2.088966E-1,1.0797673E-2,2.6714997E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1,15,-1,17,-1,19,-1,-1],"loss_changes":[8.775161E0,1.8931375E0,1.4578552E0,8.6177444E-1,0E0,8.8489985E-1,0E0,1.6791415E-1,0E0,5.124587E-1,0E0,0E0,0E0,0E0,3.7144732E-1,0E0,2.4829155E-1,0E0,8.6377144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,14,14,16,16,18,18],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1,16,-1,18,-1,20,-1,-1],"split_conditions":[1.0420895E-2,1.962943E3,1.4696082E12,1.2427474E8,2.832456E-3,1.1997641E0,-1.1238484E-2,1.137E3,-3.8282927E-2,3.1183178E8,2.3755293E-2,-2.0353394E-2,-6.2712687E-3,-7.041739E-3,2.325535E7,1.6740719E-2,9.9583336E1,-1.1000476E-3,1.3312784E7,1.0797673E-2,2.6714997E-3],"split_indices":[38,4,31,45,0,39,0,0,0,7,0,0,0,0,45,0,58,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.9E1,4.6E1,2.4E1,5E0,4.1E1,5E0,1.7E1,7E0,3.3E1,8E0,1.4E1,3E0,3E0,3E1,9E0,2.1E1,6E0,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[4.9319595E-2,-1.9302101E-1,3.6433876E-1,7.7906296E-2,-4.79115E-1,9.435853E-2,4.2625263E-1,1.1187466E-2,1.46483015E-2,-6.209801E-1,-2.6790038E-1,-5.8295415E-4,8.798848E-3,2.8855914E-1,2.4525782E-2,-7.1154446E-3,6.2758505E-2,-3.1027814E-2,-8.73463E-3,-2.221211E-3,-1.4641795E-2,3.4274822E-1,1.1442407E-3,9.597006E-3,1.2086243E-2,1.618916E-2,6.7794104E-3,-3.4221742E-2,4.374038E-3,-3.5373755E-3,1.3179845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,-1,23,-1,-1,-1,-1,25,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[6.3303638E0,3.6803284E0,5.615382E-1,4.37576E-1,5.2281046E-1,9.9045046E-2,4.2257786E-1,2.0275913E-1,0E0,4.1841602E-1,1.5917337E-1,0E0,0E0,2.3200428E-1,0E0,0E0,1.3654894E-1,0E0,0E0,0E0,0E0,3.2384157E-2,0E0,0E0,6.1977267E-2,0E0,0E0,3.409381E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,16,16,21,21,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,-1,24,-1,-1,-1,-1,26,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[2.1091123E-1,5.81609E-1,2.1267605E0,7.347893E5,9.2989355E2,6.11E2,1.1328785E-1,1.58966E5,1.46483015E-2,7.940699E-3,1.5326E4,-5.8295415E-4,8.798848E-3,1.6129E4,2.4525782E-2,-7.1154446E-3,4.5E1,-3.1027814E-2,-8.73463E-3,-2.221211E-3,-1.4641795E-2,5.405855E6,1.1442407E-3,9.597006E-3,1.04776725E6,1.618916E-2,6.7794104E-3,3.7582534E5,4.374038E-3,-3.5373755E-3,1.3179845E-3],"split_indices":[39,27,56,28,4,0,38,1,0,38,9,0,0,9,0,0,8,0,0,0,0,32,0,0,32,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,4.6E1,3.5E1,2.4E1,2.2E1,7E0,2.8E1,2E1,4E0,1.2E1,1E1,4E0,3E0,1.6E1,1.2E1,4E0,1.6E1,9E0,3E0,3E0,7E0,1.3E1,3E0,3E0,1.3E1,1E1,3E0,9E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.1746E-5,-3.7669906E-1,2.8866062E-1,-3.884967E-2,-6.1617005E-1,2.2242902E-1,4.041311E-2,-1.8333656E-1,1.349988E-2,-1.0974476E-2,-7.124857E-1,1.21058464E-1,4.1034156E-1,-1.5099317E-2,-5.4323003E-2,-1.7728277E-2,-3.464742E-2,-6.61903E-3,1.6944882E-1,2.1158814E-2,6.093224E-3,2.6566937E-4,-4.978143E-3,1.9736634E-1,-1.4299648E-3,2.4857002E-1,1.0594456E-1,1.2756806E-2,5.6673386E-3,-2.7505087E-4,6.629363E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,17,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,29,-1,-1,-1,-1],"loss_changes":[9.024752E0,2.8886461E0,1.8330379E0,8.774959E-1,5.4436207E-1,7.884717E-1,0E0,2.4362645E-1,0E0,0E0,1.2995434E-1,4.2667007E-1,2.687955E-1,0E0,3.0791815E-2,0E0,0E0,0E0,1.5386707E-1,0E0,0E0,0E0,0E0,8.133137E-2,0E0,4.12758E-2,5.6151725E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,11,11,12,12,14,14,18,18,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,18,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,30,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,7.6312915E3,8.231683E2,5.0415697E5,9.306648E-1,4.041311E-2,1.6361E4,1.349988E-2,-1.0974476E-2,6.1358623E9,4.9538263E2,3.3934937E0,-1.5099317E-2,8.62E2,-1.7728277E-2,-3.464742E-2,-6.61903E-3,8.325696E8,2.1158814E-2,6.093224E-3,2.6566937E-4,-4.978143E-3,5.405855E6,-1.4299648E-3,3.1167011E0,1.1343E4,1.2756806E-2,5.6673386E-3,-2.7505087E-4,6.629363E-3],"split_indices":[42,27,52,52,28,39,0,9,0,0,5,4,54,0,0,0,0,0,7,0,0,0,0,32,0,54,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3.5E1,4.6E1,1.5E1,2E1,4.3E1,3E0,1.1E1,4E0,5E0,1.5E1,2.9E1,1.4E1,4E0,7E0,5E0,1E1,4E0,2.5E1,1E1,4E0,4E0,3E0,2.2E1,3E0,1.3E1,9E0,8E0,5E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.464384E-4,-4.6920645E-1,1.9333163E-1,-5.952402E-1,-5.518288E-4,8.82653E-2,7.322041E-1,-1.7733527E-2,-3.505385E-2,1.2868324E-1,-1.661365E-2,3.751995E-2,1.4170783E-2,1.1289165E-2,2.404824E-1,-1.4591739E-1,7.5357206E-2,3.0781708E-3,2.721677E-1,-2.24413E-3,-8.675699E-3,1.2053321E-2,1.097327E-2,1.6005205E-2,1.8141574E-1,2.3323535E-3,-3.3507994E-3,9.380849E-3,3.6096657E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,-1,-1,-1,15,17,19,21,-1,23,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[7.1566606E0,1.3150363E0,3.1059434E0,3.8960123E-1,0E0,9.7495353E-1,2.3299408E-1,0E0,0E0,5.878903E-1,0E0,0E0,0E0,2.4265E-1,1.0320592E-1,2.660957E-2,2.2293174E-1,0E0,1.0816252E-1,0E0,0E0,0E0,5.876059E-2,0E0,2.5660902E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,13,13,14,14,15,15,16,16,18,18,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,-1,-1,-1,16,18,20,22,-1,24,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[4.823969E-3,1.962943E3,4.0230347E3,9.1569895E-1,-5.518288E-4,1.6597747E8,8.795191E-1,-1.7733527E-2,-3.505385E-2,3.839604E-1,-1.661365E-2,3.751995E-2,1.4170783E-2,6.8177136E2,2.048724E0,4.191214E5,2.325535E7,3.0781708E-3,2.4696612E0,-2.24413E-3,-8.675699E-3,1.2053321E-2,2.2968803E1,1.6005205E-2,6.3315526E8,2.3323535E-3,-3.3507994E-3,9.380849E-3,3.6096657E-3],"split_indices":[39,4,52,27,0,45,27,0,0,42,0,0,0,4,53,28,45,0,53,0,0,0,57,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.2E1,5.5E1,1.7E1,5E0,4.7E1,8E0,1.1E1,6E0,4.4E1,3E0,5E0,3E0,2.2E1,2.2E1,6E0,1.6E1,4E0,1.8E1,3E0,3E0,3E0,1.3E1,7E0,1.1E1,9E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.242935E-2,-4.2519015E-1,1.5809536E-1,-1.7596853E-1,-5.4585034E-1,-1.1979334E-2,3.1758392E-1,-1.2096516E-2,7.0671114E-5,-2.8898947E-2,-1.2043131E-2,-1.1141433E-1,1.699006E-1,3.909627E-1,1.486259E-1,1.0207987E-1,-2.1020326E-1,1.4478304E-2,6.432891E-2,6.3625416E-3,4.2305976E-1,1.0282089E-2,2.6432208E-3,7.859005E-3,-2.1978715E-4,-1.2100349E-1,-1.4820124E-2,5.283804E-3,-1.1029282E-3,2.0101778E-2,1.07203815E-2,-2.3454928E-3,-7.534415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,19,21,23,25,-1,27,-1,29,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1031775E0,5.053699E-1,1.6313026E0,1.5237364E-1,2.5601578E-1,5.614863E-1,3.2622886E-1,0E0,0E0,0E0,0E0,4.432748E-1,1.7409271E-1,1.0029149E-1,7.281983E-2,6.226304E-2,1.283493E-1,0E0,4.5889825E-2,0E0,2.8016567E-2,0E0,0E0,0E0,0E0,2.6844382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,13,13,14,14,15,15,16,16,18,18,20,20,25,25],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,20,22,24,26,-1,28,-1,30,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9956966E-1,5.0415697E5,3.8151306E-1,1.3E1,1E0,1.1808436E3,3.3027112E0,-1.2096516E-2,7.0671114E-5,-2.8898947E-2,-1.2043131E-2,2.2519132E-1,5.77E2,1.9522085E0,5.643973E8,2.3526582E3,7.1974045E-1,1.4478304E-2,1.96E2,6.3625416E-3,1.5199E4,1.0282089E-2,2.6432208E-3,7.859005E-3,-2.1978715E-4,8.07E2,-1.4820124E-2,5.283804E-3,-1.1029282E-3,2.0101778E-2,1.07203815E-2,-2.3454928E-3,-7.534415E-3],"split_indices":[57,28,42,3,16,52,54,0,0,0,0,27,0,53,7,4,27,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,1.9E1,5.9E1,7E0,1.2E1,2.9E1,3E1,4E0,3E0,7E0,5E0,1.9E1,1E1,2E1,1E1,6E0,1.3E1,3E0,7E0,3E0,1.7E1,4E0,6E0,3E0,3E0,9E0,4E0,4E0,3E0,1.2E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.513954E-4,-3.7165117E-1,1.9000456E-1,-4.7413486E-1,2.825124E-3,3.6059546E-1,1.2376453E-2,-3.4155813E-1,-3.2959625E-2,8.805712E-2,4.5158103E-1,6.926323E-2,-1.0766846E-2,-1.8985951E-2,-2.011872E-1,6.119726E-4,6.7662527E-3,5.690954E-1,2.4471958E-1,-9.67671E-3,1.2490984E-1,-1.9245803E-3,-1.1622075E-2,2.7504956E-2,1.1122954E-2,3.9141355E-3,1.3088083E-2,2.4368338E-1,2.5420533E-2,4.2327684E-3,1.2809196E-2,-5.441798E-3,3.8679964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,-1,-1,29,31,-1,-1,-1,-1],"loss_changes":[5.6427393E0,1.2425084E0,1.6015294E0,6.018686E-1,0E0,6.2975526E-1,4.2316514E-1,1.550827E-1,0E0,3.913322E-2,3.4691477E-1,4.0653005E-1,0E0,0E0,8.6992264E-2,0E0,0E0,1.07441425E-1,5.655533E-2,0E0,2.301394E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.309106E-2,1.3065305E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,14,14,17,17,18,18,20,20,27,27,28,28],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,-1,-1,30,32,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.962943E3,1.6129E4,1.2427474E8,2.825124E-3,3.3379373E-1,1.4181119E12,1.648E3,-3.2959625E-2,1.3652755E6,7.8345644E-1,2.074E3,-1.0766846E-2,-1.8985951E-2,2.2858976E8,6.119726E-4,6.7662527E-3,2.5492783E0,4.932816E6,-9.67671E-3,1.2E1,-1.9245803E-3,-1.1622075E-2,2.7504956E-2,1.1122954E-2,3.9141355E-3,1.3088083E-2,2.5630938E5,6.8175425E6,4.2327684E-3,1.2809196E-2,-5.441798E-3,3.8679964E-3],"split_indices":[42,4,9,45,0,39,31,11,0,47,27,2,0,0,7,0,0,53,1,0,3,0,0,0,0,0,0,28,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.6E1,5.2E1,2.1E1,5E0,2.6E1,2.6E1,1.6E1,5E0,7E0,1.9E1,2.2E1,4E0,8E0,8E0,4E0,3E0,1.1E1,8E0,3E0,1.9E1,3E0,5E0,8E0,3E0,3E0,5E0,8E0,1.1E1,3E0,5E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[6.091429E-2,-4.1792938E-1,1.8350436E-1,-5.0144905E-1,-1.7964066E-3,8.22498E-2,6.454801E-1,-4.900659E-3,-2.5916405E-2,-2.5496872E-2,2.559193E-1,3.7555333E-2,1.6550634E-2,2.6243908E-2,-1.904673E-1,3.7283357E-5,3.196267E-1,1.13695696E-1,-1.2504852E-1,-1.4939752E-3,-1.1836821E-2,1.552213E-2,6.485372E-3,1.1388198E-2,2.7914882E-2,-1.0534041E-2,-1.619807E-3,-1.9044313E-3,5.9120446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[4.5967064E0,4.8128414E-1,2.8470929E0,4.565587E-1,0E0,9.795654E-1,3.2176065E-1,0E0,0E0,2.8667668E-1,3.246895E-1,0E0,0E0,3.582181E-1,9.774792E-2,0E0,6.1459064E-2,2.1511662E-1,9.639382E-2,0E0,0E0,0E0,0E0,0E0,1.0208167E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,13,13,14,14,16,16,17,17,18,18,24,24],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[1.6403629E4,4.593E3,4.0230347E3,8.0159146E-1,-1.7964066E-3,5.517496E-1,1.3946067E7,-4.900659E-3,-2.5916405E-2,1.2932927E1,7.97E2,3.7555333E-2,1.6550634E-2,5.484294E-1,5.303548E9,3.7283357E-5,1.984018E7,1.3E1,2.8506322E-2,-1.4939752E-3,-1.1836821E-2,1.552213E-2,6.485372E-3,1.1388198E-2,8.8E1,-1.0534041E-2,-1.619807E-3,-1.9044313E-3,5.9120446E-3],"split_indices":[47,2,52,27,0,42,47,0,0,57,10,0,0,27,12,0,48,3,39,0,0,0,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,1.5E1,6.1E1,1.2E1,3E0,5.1E1,1E1,3E0,9E0,3.2E1,1.9E1,4E0,6E0,2.5E1,7E0,4E0,1.5E1,1.6E1,9E0,3E0,4E0,1.1E1,4E0,5E0,1.1E1,3E0,6E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.6044117E-2,-2.813553E-1,1.1194357E-1,-3.8589445E-1,6.365203E-3,1.5455592E-1,-8.159058E-3,-2.0304742E-1,-2.2294467E-2,-7.5167357E-3,1.8609025E-1,-1.2465446E-2,1.0527307E-3,8.595018E-2,2.8701866E-1,1.0672706E-2,2.9445041E-2,1.3768066E-2,3.6020828E-3,-3.5456778E-3,1.2604819E-1,6.465945E-4,7.6542827E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,-1,-1,15,17,-1,19,-1,-1,-1,21,-1,-1],"loss_changes":[2.6090114E0,1.2717111E0,6.1000854E-1,4.3901563E-1,0E0,4.4938153E-1,0E0,2.2868678E-1,0E0,0E0,3.492887E-1,0E0,0E0,1.7756584E-1,1.00421906E-1,0E0,1.8411915E-1,0E0,0E0,0E0,5.0150126E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,13,13,14,14,16,16,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,-1,-1,16,18,-1,20,-1,-1,-1,22,-1,-1],"split_conditions":[9.254704E-3,1.962943E3,6.2012E4,5.0415697E5,6.365203E-3,2.074E3,-8.159058E-3,1.3E1,-2.2294467E-2,-7.5167357E-3,4.828561E-1,-1.2465446E-2,1.0527307E-3,3.4941578E6,8.0933E4,1.0672706E-2,1.077646E3,1.3768066E-2,3.6020828E-3,-3.5456778E-3,9.57E2,6.465945E-4,7.6542827E-3],"split_indices":[38,4,10,28,0,2,0,3,0,0,39,0,0,51,9,0,52,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,2.6E1,4.4E1,2.1E1,5E0,3.9E1,5E0,1E1,1.1E1,3E0,3.6E1,7E0,3E0,1.9E1,1.7E1,4E0,1.5E1,1.4E1,3E0,7E0,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.375215E-2,-1.7956609E-1,3.1396008E-1,-4.475621E-1,-7.0033245E-2,2.2233696E-1,3.357808E-2,-5.835884E-1,-1.26632E-3,1.17291234E-1,-1.402892E-1,1.2604502E-2,1.09961465E-1,-1.2833614E-2,-2.8878465E-2,8.959712E-3,1.5504903E-3,-8.761031E-2,-1.3359717E-2,3.247988E-4,6.303277E-3,-1.3335772E-1,-8.037925E-3,-1.5199379E-3,-8.473441E-3,2.137595E-3,-3.3594493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[4.002802E0,1.201539E0,9.912052E-1,6.6494465E-1,4.2443517E-1,1.6288972E-1,0E0,3.1345367E-2,0E0,5.966536E-2,1.8117502E-1,0E0,4.1036606E-2,0E0,0E0,0E0,0E0,6.8092436E-2,0E0,0E0,0E0,6.900616E-2,3.6662724E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,17,17,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[4.7985215E-2,5.382431E0,7.6312915E3,7.940699E-3,2.805454E-1,2.4832625E6,3.357808E-2,2.202021E8,-1.26632E-3,4.5E1,2.0536139E1,1.2604502E-2,5.316E3,-1.2833614E-2,-2.8878465E-2,8.959712E-3,1.5504903E-3,1.7058623E3,-1.3359717E-2,3.247988E-4,6.303277E-3,6.433619E2,1.0519099E6,-1.5199379E-3,-8.473441E-3,2.137595E-3,-3.3594493E-3],"split_indices":[38,56,52,38,27,32,0,7,0,8,56,0,2,0,0,0,0,4,0,0,0,4,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,4.1E1,2.6E1,1.1E1,3E1,2.3E1,3E0,8E0,3E0,8E0,2.2E1,1.3E1,1E1,3E0,5E0,3E0,5E0,1.8E1,4E0,3E0,7E0,1.1E1,7E0,5E0,6E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.3291592E-2,-3.5239986E-1,1.72509E-1,-5.0849515E-1,5.0978072E-2,1.1366999E-1,5.429267E-1,-3.4126726E-1,-2.9658329E-2,-1.7515961E-3,5.6755748E-3,1.6279958E-1,-1.9032758E-1,2.9985784E-2,1.0644514E-2,-1.7278347E-2,-6.842589E-3,-3.87651E-3,2.166911E-1,-1.2132395E-2,-2.1109018E-3,7.0085307E-3,-8.727592E-2,1.4054082E-2,1.5595174E-1,-3.872923E-4,-5.6616897E-3,1.5735772E-3,8.481439E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[5.16432E0,1.8519189E0,1.0934212E0,4.131775E-1,7.307397E-2,7.329432E-1,1.5120387E-1,8.3277225E-2,0E0,0E0,0E0,3.6911952E-1,7.814169E-2,0E0,0E0,0E0,0E0,1.68213E-1,1.5955377E-1,0E0,0E0,0E0,2.7035102E-2,0E0,1.0380441E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,17,17,18,18,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,4.0230347E3,8.616169E-1,8.231683E2,1.1269586E12,2.1050402E7,1.5341808E3,-2.9658329E-2,-1.7515961E-3,5.6755748E-3,3.6732382E2,2.8312179E12,2.9985784E-2,1.0644514E-2,-1.7278347E-2,-6.842589E-3,1.7970878E6,2.325535E7,-1.2132395E-2,-2.1109018E-3,7.0085307E-3,1.2E1,1.4054082E-2,9.9583336E1,-3.872923E-4,-5.6616897E-3,1.5735772E-3,8.481439E-3],"split_indices":[42,4,52,27,52,31,47,4,0,0,0,52,31,0,0,0,0,48,45,0,0,0,3,0,58,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.8E1,5.2E1,2E1,8E0,4.6E1,6E0,1.2E1,8E0,4E0,4E0,4E1,6E0,3E0,3E0,8E0,4E0,1E1,3E1,3E0,3E0,3E0,7E0,9E0,2.1E1,3E0,4E0,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.11815E-2,-3.3522725E-1,1.8761614E-1,6.0145766E-2,-4.4398713E-1,1.0632117E-1,4.696459E-1,-1.2920295E-3,6.759094E-3,-5.660932E-1,-2.2601584E-1,2.910224E-2,1.8621325E-1,2.3904392E-2,1.1481897E-2,-2.8437966E-1,-3.473485E-2,-3.3279383E-3,-1.1625643E-2,-6.4782416E-3,8.3718695E-2,9.543453E-3,3.2117711E-3,-1.5034129E-2,-4.811055E-3,5.5344934E-3,-1.8265166E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,-1,23,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[5.2518044E0,1.4457278E0,1.0047364E0,7.55541E-2,5.640602E-1,2.2284049E-1,2.96309E-2,0E0,0E0,8.7858295E-1,5.3575993E-2,2.099497E-1,5.7148635E-2,0E0,0E0,6.437707E-2,0E0,0E0,0E0,0E0,9.798034E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,15,15,20,20],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,-1,24,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4865639E-2,3.459181E-1,1.0272479E0,8.231683E2,1.500766E3,4.7565946E-1,1.383E4,-1.2920295E-3,6.759094E-3,1.0370839E6,5.0017652E4,4.57563E5,9.19322E0,2.3904392E-2,1.1481897E-2,2.23E2,-3.473485E-2,-3.3279383E-3,-1.1625643E-2,-6.4782416E-3,3.64E2,9.543453E-3,3.2117711E-3,-1.5034129E-2,-4.811055E-3,5.5344934E-3,-1.8265166E-3],"split_indices":[38,27,39,52,4,42,10,0,0,28,33,1,56,0,0,10,0,0,0,0,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3.2E1,4.5E1,7E0,2.5E1,3.6E1,9E0,4E0,3E0,1.5E1,1E1,1.9E1,1.7E1,5E0,4E0,8E0,7E0,3E0,7E0,4E0,1.5E1,1.2E1,5E0,5E0,3E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.4895428E-3,-3.0832598E-1,1.476936E-1,7.337417E-2,-5.032524E-1,5.1306404E-2,3.429725E-1,-6.620061E-2,1.2798463E-2,-5.4586494E-1,-9.307089E-3,8.52972E-2,-1.10731E-2,4.080719E-1,3.9384817E-3,-6.1457483E-3,1.1915804E-3,-1.4732508E-2,-2.903554E-2,1.234163E-1,-7.188532E-3,2.0559082E-2,9.879167E-3,1.8509729E-1,2.3773596E-2,1.8284154E-4,2.4937104E-1,-3.5463482E-2,7.0954026E-3,1.3132916E-2,6.3187866E-3,-3.965284E-3,7.7684724E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1,25,27,-1,29,31,-1,-1,-1,-1,-1],"loss_changes":[3.7659874E0,2.0458183E0,1.0176347E0,3.351296E-1,1.0026741E-1,4.2281622E-1,2.5638676E-1,5.504486E-2,0E0,1.6813612E-1,0E0,3.5882187E-1,0E0,6.912041E-2,0E0,0E0,0E0,0E0,0E0,1.8612745E-1,0E0,0E0,0E0,2.1975332E-1,1.1535082E-1,0E0,3.675276E-2,3.2394454E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,19,19,23,23,24,24,26,26,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1,26,28,-1,30,32,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,9.306648E-1,8.231683E2,1.3513911E1,1.1696494E7,3.3934937E0,1.3E1,1.2798463E-2,2.8225487E11,-9.307089E-3,1E0,-1.10731E-2,1.3996E4,3.9384817E-3,-6.1457483E-3,1.1915804E-3,-1.4732508E-2,-2.903554E-2,5.6039695E9,-7.188532E-3,2.0559082E-2,9.879167E-3,8.1729946E10,2.0901184E3,1.8284154E-4,1.181508E6,1.742E3,7.0954026E-3,1.3132916E-2,6.3187866E-3,-3.965284E-3,7.7684724E-4],"split_indices":[42,27,39,52,56,47,54,3,0,31,0,105,0,9,0,0,0,0,0,5,0,0,0,31,52,0,32,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.6E1,5.4E1,9E0,1.7E1,3.7E1,1.7E1,6E0,3E0,1.4E1,3E0,3.4E1,3E0,1.3E1,4E0,3E0,3E0,7E0,7E0,3E1,4E0,8E0,5E0,1.8E1,1.2E1,5E0,1.3E1,9E0,3E0,7E0,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.9342616E-2,-1.177099E-1,2.5990668E-1,-3.7666702E-1,-3.965099E-2,1.0666826E-2,3.361856E-1,-6.8167797E-3,-2.0442668E-2,7.8358404E-2,-1.4977737E-1,-3.909633E-3,6.890782E-3,1.4904882E-3,3.7397426E-1,1.1430879E-2,1.0151933E-2,-8.376353E-2,-1.1135905E-2,1.761733E-2,9.761365E-3,5.960206E-3,-3.3769023E-2,-7.8526065E-3,-6.7802267E-3,1.5333147E-3,-5.919079E-3,-2.4406018E-3,2.3791464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,19,-1,21,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1,-1],"loss_changes":[2.8364453E0,9.5533246E-1,6.428132E-1,1.660701E-1,5.0558233E-1,1.5298896E-1,2.830248E-1,0E0,0E0,2.4214801E-1,1.2425065E-1,0E0,0E0,0E0,3.3308268E-2,0E0,8.984181E-2,7.82373E-2,0E0,0E0,0E0,0E0,9.239468E-2,2.8174143E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,14,14,16,16,17,17,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,20,-1,22,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1,-1],"split_conditions":[3.839604E-1,1.346E3,2.1262457E0,2.62518E5,5.81609E-1,1.6E1,3.2414E-1,-6.8167797E-3,-2.0442668E-2,4.48E2,3.6979167E0,-3.909633E-3,6.890782E-3,1.4904882E-3,1.5199E4,1.1430879E-2,1.6439099E-2,1.5326E4,-1.1135905E-2,1.761733E-2,9.761365E-3,5.960206E-3,2.1446E4,1.27426E6,-6.7802267E-3,1.5333147E-3,-5.919079E-3,-2.4406018E-3,2.3791464E-3],"split_indices":[42,11,56,29,27,8,27,0,0,0,54,0,0,0,9,0,27,9,0,0,0,0,9,1,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,4.7E1,3.3E1,1E1,3.7E1,8E0,2.5E1,4E0,6E0,1.8E1,1.9E1,5E0,3E0,3E0,2.2E1,4E0,1.4E1,1.3E1,6E0,1.6E1,6E0,3E0,1.1E1,7E0,6E0,7E0,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.996961E-2,-2.8205118E-1,1.5501395E-1,-1.55176E-1,-2.9040772E-2,6.840957E-2,5.63874E-1,-2.587841E-1,1.1721904E-2,2.6414385E-1,1.1448369E-2,3.3335984E-2,1.2408488E-2,-1.2926956E-2,-3.6485724E-3,-2.4606911E-3,2.392361E-3,4.0180907E-3,1.3522474E-2,-1.0006361E-1,6.47153E-2,-7.220365E-4,-7.224762E-3,1.0255815E-1,-3.6275077E-3,2.6295839E-2,7.0096995E-3,4.39064E-3,-2.3290648E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1,21,23,-1,-1,25,-1,27,-1,-1,-1],"loss_changes":[3.428976E0,1.2044048E0,2.0151515E0,3.6616492E-1,0E0,5.4328966E-1,3.6411428E-1,7.2318256E-2,3.0786386E-2,6.776947E-2,2.3813179E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.1633935E-2,1.6149634E-1,0E0,0E0,9.8638654E-2,0E0,7.344261E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,19,19,20,20,23,23,25,25],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1,22,24,-1,-1,26,-1,28,-1,-1,-1],"split_conditions":[4.856813E0,7.916317E11,3.9509753E3,1.6E1,-2.9040772E-2,1.7329262E7,8.982776E9,1.0370839E6,1.938E3,1.0843103E5,1.4343789E6,3.3335984E-2,1.2408488E-2,-1.2926956E-2,-3.6485724E-3,-2.4606911E-3,2.392361E-3,4.0180907E-3,1.3522474E-2,1.8523E4,2.3297022E7,-7.220365E-4,-7.224762E-3,2.1202636E-1,-3.6275077E-3,1.3397689E-1,7.0096995E-3,4.39064E-3,-2.3290648E-3],"split_indices":[58,31,52,8,0,45,5,28,2,33,47,0,0,0,0,0,0,0,0,9,32,0,0,39,0,39,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.5E1,5.7E1,2E1,5E0,4.8E1,9E0,1.2E1,8E0,1E1,3.8E1,4E0,5E0,9E0,3E0,3E0,5E0,3E0,7E0,1.2E1,2.6E1,6E0,6E0,2.1E1,5E0,1E1,1.1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.7959079E-2,3.3515338E-2,-5.439698E-1,-9.869692E-2,1.8315943E-1,-3.189193E-2,-4.5014187E-3,3.8901392E-2,-2.3159537E-1,1.0944469E-1,1.3747038E-2,-1.9519506E-2,1.2489866E-2,-1.5762134E-2,-1.3676985E-1,-6.1160773E-3,1.64186E-1,2.8373315E-3,-4.879285E-2,1.6439244E-3,-9.010175E-3,-3.0944736E-3,4.0532374E-3,2.1961888E-3,8.336786E-3,3.047169E-4,-4.52133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,-1,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.682834E0,1.5217208E0,7.183106E-1,7.5876E-1,3.3185184E-1,0E0,0E0,3.1629825E-1,2.2239149E-1,1.5915287E-1,0E0,4.8483647E-2,0E0,0E0,1.8944594E-1,6.701711E-2,4.9000174E-2,0E0,4.532664E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,14,14,15,15,16,16,18,18],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,-1,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5414109E8,3.2711282E-2,7.493E3,5.81609E-1,2.4273067E1,-3.189193E-2,-4.5014187E-3,7.347893E5,3.507E3,2.6880343E0,1.3747038E-2,1.204507E-1,1.2489866E-2,-1.5762134E-2,5.7229916E7,1.6E1,1.2E1,2.8373315E-3,2.1416E4,1.6439244E-3,-9.010175E-3,-3.0944736E-3,4.0532374E-3,2.1961888E-3,8.336786E-3,3.047169E-4,-4.52133E-3],"split_indices":[45,38,2,27,57,0,0,28,2,54,0,27,0,0,45,3,3,0,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.5E1,8E0,4E1,3.5E1,5E0,3E0,2E1,2E1,2.4E1,1.1E1,1.7E1,3E0,7E0,1.3E1,8E0,1.6E1,4E0,1.3E1,4E0,9E0,5E0,3E0,4E0,1.2E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.9505762E-2,-3.0160806E-1,1.1446385E-1,-4.101308E-1,6.7047074E-2,-8.379094E-3,2.612527E-1,-2.6891392E-1,-5.6854063E-1,5.741581E-3,-7.2406675E-4,2.5123904E-2,-1.0655792E-2,1.7354633E-1,3.8422826E-1,-3.8618457E-3,-1.3404787E-2,-1.2552934E-2,-2.8100347E-2,-4.6093785E-3,7.231362E-2,2.4235931E-4,8.942462E-3,9.910835E-3,2.09687E-2,7.163702E-3,5.9442273E-3,-1.8904549E-3,4.4926545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[3.1713533E0,1.1076999E0,1.0300117E0,3.1688452E-1,4.1787274E-2,2.644232E-1,2.1374798E-1,7.513434E-2,2.6323795E-2,0E0,0E0,1.8968515E-1,0E0,1.0032731E-1,4.8149586E-2,0E0,0E0,0E0,0E0,0E0,1.3801764E-1,0E0,0E0,0E0,0E0,0E0,7.423505E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,20,20,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,5.451147E-1,9.29295E5,1.6867E4,3.6030095E10,1.1997641E0,3.3159972E5,8.28645E-1,5.741581E-3,-7.2406675E-4,3.7818575E0,-1.0655792E-2,1.0966E4,1.3E1,-3.8618457E-3,-1.3404787E-2,-1.2552934E-2,-2.8100347E-2,-4.6093785E-3,2.491358E0,2.4235931E-4,8.942462E-3,9.910835E-3,2.09687E-2,7.163702E-3,3.6616542E0,-1.8904549E-3,4.4926545E-3],"split_indices":[42,4,42,28,9,5,39,28,27,0,0,56,0,9,3,0,0,0,0,0,53,0,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.6E1,5.6E1,2E1,6E0,3.1E1,2.5E1,1.2E1,8E0,3E0,3E0,2.8E1,3E0,1.6E1,9E0,3E0,9E0,3E0,5E0,7E0,2.1E1,3E0,1.3E1,5E0,4E0,8E0,1.3E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.449704E-2,-3.1889904E-1,1.4883669E-1,-4.2908838E-1,5.913231E-3,9.372266E-2,2.1409126E-2,-1.1719601E-2,-2.513279E-2,1.2944044E-1,-8.096517E-3,1.7205084E-2,1.8951772E-1,6.2284645E-2,-4.7494997E-3,1.2188283E-2,1.3342847E-1,-1.9648627E-3,4.6447357E-3,8.554429E-2,8.850229E-3,5.8247205E-3,5.207967E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,13,15,17,-1,-1,19,-1,-1,21,-1,-1,-1],"loss_changes":[3.721609E0,1.4194877E0,8.449938E-1,3.7996912E-1,0E0,4.390232E-1,0E0,0E0,0E0,2.4622273E-1,0E0,8.7870665E-2,9.0527E-2,6.0149476E-2,0E0,0E0,4.215181E-2,0E0,0E0,4.2945392E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,11,11,12,12,13,13,16,16,19,19],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,14,16,18,-1,-1,20,-1,-1,22,-1,-1,-1],"split_conditions":[1.7489342E-2,1.9315491E3,6.7780703E3,8.616169E-1,5.913231E-3,2.9461394E5,2.1409126E-2,-1.1719601E-2,-2.513279E-2,2.847953E-1,-8.096517E-3,5.303548E9,9.693049E2,4.57563E5,-4.7494997E-3,1.2188283E-2,4.946992E6,-1.9648627E-3,4.6447357E-3,5.99187E0,8.850229E-3,5.8247205E-3,5.207967E-4],"split_indices":[38,4,4,27,0,33,0,0,0,42,0,12,4,1,0,0,1,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.6E1,4.5E1,2.1E1,5E0,4E1,5E0,1.2E1,9E0,3.6E1,4E0,1.3E1,2.3E1,1E1,3E0,7E0,1.6E1,3E0,7E0,1.1E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.07692E-3,-2.7644157E-1,1.2962286E-1,-8.736741E-2,-4.1278672E-1,8.3613865E-2,4.5945916E-1,5.1238113E-3,-1.6150486E-1,-5.141799E-1,-3.715523E-3,6.8261735E-3,2.6075876E-1,2.6412223E-2,7.971721E-3,-9.983092E-3,-1.8332362E-3,-9.530817E-3,-2.7025245E-2,5.4569438E-2,-1.7903225E-1,1.6170103E-2,6.5223556E-3,1.2187874E-1,5.6436267E-3,-1.6137054E-2,6.6004094E-4,6.8556686E-4,7.955213E-3,6.314655E-2,-2.7909344E-3,-2.401067E-3,4.913517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[3.130982E0,6.8575764E-1,8.569336E-1,2.1903388E-1,4.8403072E-1,7.1915495E-1,1.8719184E-1,0E0,7.534486E-2,2.7091742E-1,0E0,3.4696224E-1,1.4558935E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.01400204E-1,3.1436688E-1,0E0,0E0,8.528942E-2,8.159262E-2,0E0,0E0,0E0,0E0,7.43649E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,19,19,20,20,23,23,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[7.2921924E-2,5.0415697E5,4.4839956E3,8.205673E-4,1.0735684E3,8.363552E-1,5.902777E-1,5.1238113E-3,6.386E3,9.29295E5,-3.715523E-3,1.0743855E-1,2.4696612E0,2.6412223E-2,7.971721E-3,-9.983092E-3,-1.8332362E-3,-9.530817E-3,-2.7025245E-2,4.1508866E5,2.8808794E5,1.6170103E-2,6.5223556E-3,1.3397689E-1,6.0487356E7,-1.6137054E-2,6.6004094E-4,6.8556686E-4,7.955213E-3,1.1171711E9,-2.7909344E-3,-2.401067E-3,4.913517E-3],"split_indices":[42,28,52,42,52,41,27,0,10,28,0,38,53,0,0,0,0,0,0,28,28,0,0,39,45,0,0,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,2.7E1,5.8E1,1.2E1,1.5E1,5.2E1,6E0,3E0,9E0,1.1E1,4E0,3.7E1,1.5E1,3E0,3E0,5E0,4E0,4E0,7E0,3E1,7E0,6E0,9E0,1.2E1,1.8E1,3E0,4E0,5E0,7E0,1E1,8E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.484174E-2,-2.0651723E-1,1.23060025E-1,9.060144E-2,-3.7930906E-1,8.6913936E-2,1.6235925E-2,-8.894108E-4,1.12425E-2,-2.322102E-1,-2.28896E-2,9.985029E-3,6.3343115E-2,-1.4884285E-2,-4.1465354E-3,1.2130751E-3,1.28519E-1,-4.1467164E-2,5.214476E-3,7.1523585E-3,2.1171435E-3,-3.2622847E-3,3.881595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,15,-1,-1,17,19,21,-1,-1,-1,-1,-1],"loss_changes":[1.8220457E0,1.3178582E0,4.4004917E-1,1.9740945E-1,2.4275732E-1,1.4414191E-1,0E0,0E0,0E0,1.2293494E-1,0E0,0E0,1.6586871E-1,0E0,0E0,1.1841317E-1,5.2930146E-2,8.520052E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,16,-1,-1,18,20,22,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,5.8871865E-1,6.5317163E3,8.231683E2,1.1686677E8,3.5175372E5,1.6235925E-2,-8.894108E-4,1.12425E-2,1.93E2,-2.28896E-2,9.985029E-3,5.517496E-1,-1.4884285E-2,-4.1465354E-3,3.327103E0,6.7584877E8,1.1812E4,5.214476E-3,7.1523585E-3,2.1171435E-3,-3.2622847E-3,3.881595E-3],"split_indices":[38,27,4,52,45,32,0,0,0,10,0,0,41,0,0,53,7,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.4E1,5E1,9E0,1.5E1,4.5E1,5E0,6E0,3E0,9E0,6E0,5E0,4E1,4E0,5E0,2.1E1,1.9E1,1.6E1,5E0,1.2E1,7E0,1.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-3.6807183E-2,-2.6082924E-1,9.745226E-2,-3.38609E-2,-4.4575256E-1,5.3367496E-2,2.569166E-1,3.8533923E-3,-1.0485729E-1,-2.6165378E-1,-2.6519928E-2,-7.0941593E-3,1.42699E-1,1.3604495E-2,5.124206E-3,-2.7395165E-4,-6.2492257E-3,-2.6505715E-3,-1.4469911E-2,5.6039942E-3,-3.8167406E-2,7.6184836E-3,2.8479039E-3,-6.874908E-2,2.3452176E-3,3.404119E-4,-1.1131427E-1,-8.842438E-4,-6.103098E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,27,-1,-1],"loss_changes":[2.4917388E0,1.2860303E0,3.518464E-1,1.4147085E-1,3.8356328E-1,2.2785982E-1,5.4195166E-2,0E0,3.9702587E-2,1.3221234E-1,0E0,1.1660655E-1,3.400594E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6346675E-2,0E0,0E0,5.6393713E-2,0E0,0E0,2.8358817E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,20,20,23,23,26,26],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,28,-1,-1],"split_conditions":[1.7489342E-2,5.0415697E5,4.108022E1,1.8144448E-4,2.5036643E8,6.3104886E-1,4.668526E6,3.8533923E-3,4.4092423E-1,5.86E2,-2.6519928E-2,2.7252597E-1,2.6529046E7,1.3604495E-2,5.124206E-3,-2.7395165E-4,-6.2492257E-3,-2.6505715E-3,-1.4469911E-2,5.6039942E-3,2.6597537E3,7.6184836E-3,2.8479039E-3,3.9785632E8,2.3452176E-3,3.404119E-4,1.0192E4,-8.842438E-4,-6.103098E-3],"split_indices":[38,28,57,38,7,41,29,0,27,0,0,27,45,0,0,0,0,0,0,0,4,0,0,7,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3E1,5.1E1,1.4E1,1.6E1,4.1E1,1E1,5E0,9E0,9E0,7E0,2.5E1,1.6E1,6E0,4E0,3E0,6E0,3E0,6E0,4E0,2.1E1,1E1,6E0,1.6E1,5E0,6E0,1E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.750396E-3,-2.637187E-1,1.3615979E-1,-1.4564972E-1,-5.4651755E-1,5.4761996E-3,2.2307935E-1,-1.2620885E-2,-9.917557E-2,-3.0120319E-2,-1.07787E-2,9.194145E-2,-6.576594E-2,3.183191E-1,1.2566628E-1,7.8938337E-4,-1.3992551E-1,9.1104873E-4,6.1660395E-3,-4.332866E-4,-6.867432E-3,6.8999347E-3,1.6945135E-2,1.6493298E-3,7.0712506E-3,-4.079417E-4,-8.720541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5698216E0,7.5514436E-1,5.099317E-1,1.02725804E-1,1.4779592E-1,1.2308604E-1,2.1049309E-1,0E0,7.928079E-2,0E0,0E0,3.0144311E-2,5.6403577E-2,9.509659E-2,4.5478165E-2,0E0,9.819192E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,13,13,14,14,16,16],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,1.5135763E8,2.5222173E-1,5.4508613E3,9.7646296E7,4.4605106E5,1.5312917E5,-1.2620885E-2,5.7777777E-2,-3.0120319E-2,-1.07787E-2,2.9795604E9,2.1446E4,3.9976162E-1,1.7325155E5,7.8938337E-4,3.027892E5,9.1104873E-4,6.1660395E-3,-4.332866E-4,-6.867432E-3,6.8999347E-3,1.6945135E-2,1.6493298E-3,7.0712506E-3,-4.079417E-4,-8.720541E-3],"split_indices":[38,45,39,33,12,28,33,0,57,0,0,5,9,27,33,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,2.4E1,4.4E1,1.8E1,6E0,1.8E1,2.6E1,3E0,1.5E1,3E0,3E0,8E0,1E1,1.2E1,1.4E1,4E0,1.1E1,4E0,4E0,7E0,3E0,5E0,7E0,5E0,9E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.408408E-3,-2.5382924E-1,1.1687296E-1,-1.6837038E-1,-2.9154288E-2,5.9414465E-2,2.300275E-2,-2.20062E-1,-1.8780673E-2,2.3231715E-1,4.9326853E-3,-1.4188947E-1,-1.6220024E-2,2.9747307E-3,-4.3801954E-3,3.8770789E-3,1.28422305E-2,-1.8587545E-1,5.7399318E-2,-8.522451E-3,-2.7460514E-3,-2.3119294E-3,-1.0868278E-2,6.147688E-3,4.379439E-3,-5.93595E-2,8.179598E-2,1.3064143E-3,-4.1435147E-3,6.122379E-5,6.2349276E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,-1,25,27,29,-1,-1,-1,-1],"loss_changes":[2.3664117E0,8.3604467E-1,1.2018381E0,1.7371541E-1,0E0,4.3217617E-1,0E0,1.5779579E-1,5.8853693E-2,7.8718185E-2,3.710673E-1,4.5167238E-2,0E0,0E0,0E0,0E0,0E0,5.8254957E-2,1.3215809E-1,0E0,0E0,0E0,0E0,0E0,9.860988E-2,4.0082153E-2,4.6084777E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,17,17,18,18,24,24,25,25,26,26],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,-1,26,28,30,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.5135763E8,3.4409692E3,5.36E3,-2.9154288E-2,1.7329262E7,2.300275E-2,1.02E3,2.2858976E8,6.498673E2,4.492E3,3E0,-1.6220024E-2,2.9747307E-3,-4.3801954E-3,3.8770789E-3,1.28422305E-2,4.841492E5,5.090285E7,-8.522451E-3,-2.7460514E-3,-2.3119294E-3,-1.0868278E-2,6.147688E-3,1.204781E3,4.1485094E-2,1.7495675E5,1.3064143E-3,-4.1435147E-3,6.122379E-5,6.2349276E-3],"split_indices":[58,45,52,2,0,45,0,0,7,4,2,8,0,0,0,0,0,28,45,0,0,0,0,0,52,39,33,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.5E1,5E1,2.2E1,3E0,4.5E1,5E0,1.6E1,6E0,1E1,3.5E1,1.2E1,4E0,3E0,3E0,4E0,6E0,7E0,2.8E1,6E0,6E0,3E0,4E0,1E1,1.8E1,1E1,8E0,3E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.0632061E-2,-2.4092656E-1,9.551605E-2,-4.0311107E-1,-8.516071E-2,1.254183E-1,-1.1961503E-2,-7.877372E-3,-2.3726251E-2,-1.6082093E-1,1.2791782E-3,4.8489507E-2,1.8568814E-1,-1.0854732E-2,-2.319776E-3,1.9796067E-1,-1.993472E-2,-2.5485766E-3,2.2027308E-1,3.1508906E-3,1.1663632E-2,-2.6950627E-3,3.7402357E-3,2.500118E-1,5.6986962E-3,5.1142857E-3,2.8351766E-1,1.466244E-2,7.1420944E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,17,-1,-1,19,21,-1,23,-1,-1,-1,-1,25,-1,-1,27,-1,-1],"loss_changes":[1.7830759E0,5.4671514E-1,5.960934E-1,2.6311874E-1,1.1799519E-1,2.0891088E-1,0E0,0E0,0E0,6.5531775E-2,0E0,2.2895017E-1,2.3369044E-1,0E0,0E0,4.4965267E-2,7.9553865E-2,0E0,2.6198149E-2,0E0,0E0,0E0,0E0,3.600812E-2,0E0,0E0,2.5718153E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,12,12,15,15,16,16,18,18,23,23,26,26],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,18,-1,-1,20,22,-1,24,-1,-1,-1,-1,26,-1,-1,28,-1,-1],"split_conditions":[2.0113895E-2,3.554517E0,6.2012E4,1.0370839E6,1.7493458E3,3.3597556E-1,-1.1961503E-2,-7.877372E-3,-2.3726251E-2,9.4596675E-5,1.2791782E-3,2.5143658E6,1.4212261E0,-1.0854732E-2,-2.319776E-3,3.7700243E-2,3.6616542E0,-2.5485766E-3,3.2979845E6,3.1508906E-3,1.1663632E-2,-2.6950627E-3,3.7402357E-3,3.1823853E-1,5.6986962E-3,5.1142857E-3,6.585265E-1,1.466244E-2,7.1420944E-3],"split_indices":[41,54,10,28,4,39,0,0,0,38,0,51,56,0,0,27,53,0,32,0,0,0,0,27,0,0,27,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.2E1,4.9E1,1E1,1.2E1,4.6E1,3E0,5E0,5E0,7E0,5E0,2.1E1,2.5E1,3E0,4E0,6E0,1.5E1,3E0,2.2E1,3E0,3E0,1.1E1,4E0,1.5E1,7E0,4E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.2602936E-2,-1.7656966E-1,6.665869E-2,-2.5975782E-1,8.466752E-2,1.0225561E-1,-1.767866E-1,-1.4000464E-3,-2.921736E-1,8.9720427E-4,6.119811E-3,1.7116562E-2,1.6708885E-1,-1.2227611E-2,-1.0023373E-3,-3.4930915E-1,-4.303655E-3,-3.5181323E-3,6.2355846E-2,2.0918262E-1,4.6230074E-2,-1.6783658E-2,-5.6650047E-3,7.1607847E-3,-6.9373236E-3,9.782477E-3,3.3604016E-3,-1.9280773E-3,4.705579E-3,1.6120562E-3,-3.5627545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,-1,-1,21,-1,-1,23,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1249788E0,6.739482E-1,4.687044E-1,1.5969086E-1,2.6773125E-2,2.5105268E-1,1.1046371E-1,0E0,1.8371439E-1,0E0,0E0,9.954972E-2,1.2260193E-1,0E0,0E0,8.8416815E-2,0E0,0E0,1.1039816E-1,3.0600965E-2,5.1575392E-2,0E0,0E0,0E0,3.7230834E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,15,15,18,18,19,19,20,20,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,-1,-1,22,-1,-1,24,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.962943E3,1.4696082E12,2.5346E3,2.4399805E3,3.3597556E-1,3.05E3,-1.4000464E-3,2.2867646E0,8.9720427E-4,6.119811E-3,7.095772E2,5.7588155E6,-1.2227611E-2,-1.0023373E-3,1.650581E3,-4.303655E-3,-3.5181323E-3,4.8178736E7,1.6129E4,6.249143E-1,-1.6783658E-2,-5.6650047E-3,7.1607847E-3,7.844101E0,9.782477E-3,3.3604016E-3,-1.9280773E-3,4.705579E-3,1.6120562E-3,-3.5627545E-3],"split_indices":[42,4,31,32,4,39,0,0,57,0,0,4,32,0,0,4,0,0,45,9,42,0,0,0,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.9E1,5.1E1,2.2E1,7E0,4.5E1,6E0,3E0,1.9E1,4E0,3E0,2E1,2.5E1,3E0,3E0,1.4E1,5E0,6E0,1.4E1,1.8E1,7E0,1.1E1,3E0,5E0,9E0,1.5E1,3E0,3E0,4E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.6865763E-3,-1.8272516E-1,8.7893896E-2,-2.5947955E-1,3.8822438E-3,1.21314E-1,-1.1284622E-2,-1.7682245E-1,-2.2460883E-2,2.035009E-1,7.4624578E-3,-1.0745997E-2,-7.0943184E-2,2.0190727E-2,2.6090908E-1,-4.7269452E-2,3.9244615E-3,3.7158409E-3,-7.826691E-3,-1.5020958E-3,3.6044621E-3,1.7683446E-1,3.441152E-1,1.9708078E-3,-8.922047E-2,1.8667543E-3,9.08392E-3,1.899623E-2,8.695617E-3,-5.6091757E-3,-8.99704E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2168006E0,5.273904E-1,6.668652E-1,3.2449567E-1,0E0,4.6633655E-1,0E0,1.08326495E-1,0E0,2.9991376E-1,1.0585508E-1,0E0,1.5742677E-1,3.1326193E-2,9.305024E-2,5.894113E-2,0E0,0E0,0E0,0E0,0E0,5.2344352E-2,5.0588608E-2,0E0,2.584079E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,12,12,13,13,14,14,15,15,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,1.5414109E8,1.2427474E8,3.8822438E-3,3.3934937E0,-1.1284622E-2,5E0,-2.2460883E-2,2.0753424E0,5.2518907E0,-1.0745997E-2,5.0415697E5,1.6E1,2.9168832E0,7.843361E5,3.9244615E-3,3.7158409E-3,-7.826691E-3,-1.5020958E-3,3.6044621E-3,1.0737101E0,7.33E2,1.9708078E-3,9.21919E5,1.8667543E-3,9.08392E-3,1.899623E-2,8.695617E-3,-5.6091757E-3,-8.99704E-4],"split_indices":[58,4,45,45,0,54,0,8,0,53,54,0,28,3,54,32,0,0,0,0,0,57,0,0,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.3E1,5.3E1,1.8E1,5E0,4.9E1,4E0,1.5E1,3E0,2.8E1,2.1E1,8E0,7E0,7E0,2.1E1,1.3E1,8E0,3E0,4E0,4E0,3E0,1.2E1,9E0,4E0,9E0,3E0,9E0,4E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.621743E-2,3.558231E-2,-2.938916E-1,-1.8531272E-1,7.990702E-2,6.033404E-4,-3.8770264E-1,-9.306622E-3,-2.8338241E-3,3.3018474E-2,1.9380707E-1,-1.9141065E-2,-7.951701E-3,9.9072486E-2,-1.2703481E-1,2.4355294E-1,2.651927E-3,8.730972E-3,3.930368E-2,-7.339577E-3,-9.130585E-4,6.346931E-3,1.3425479E-2,-2.515763E-3,7.3647335E-2,4.856358E-3,1.2821703E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,23,-1,-1,-1,-1,-1,25,-1,-1],"loss_changes":[1.1899004E0,7.001853E-1,3.8109004E-1,3.1746447E-2,3.0869663E-1,0E0,3.7626266E-2,0E0,0E0,4.6718863E-1,9.635085E-2,0E0,0E0,1.8670347E-1,5.760783E-2,3.3767164E-2,0E0,0E0,7.510835E-2,0E0,0E0,0E0,0E0,0E0,2.448184E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,13,13,14,14,15,15,18,18,24,24],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,24,-1,-1,-1,-1,-1,26,-1,-1],"split_conditions":[1.0232127E12,1.885E3,2.7307262E5,1.4324325E1,3.029697E3,6.033404E-4,2.8312179E12,-9.306622E-3,-2.8338241E-3,6.98494E7,5.405855E6,-1.9141065E-2,-7.951701E-3,1.7329262E7,1.962943E3,3.3072104E3,2.651927E-3,8.730972E-3,4.57563E5,-7.339577E-3,-9.130585E-4,6.346931E-3,1.3425479E-2,-2.515763E-3,1.7231327E3,4.856358E-3,1.2821703E-3],"split_indices":[31,2,28,56,4,0,31,0,0,45,32,0,0,45,4,52,0,0,1,0,0,0,0,0,55,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,6.9E1,1.2E1,1.1E1,5.8E1,3E0,9E0,8E0,3E0,4.2E1,1.6E1,6E0,3E0,3E1,1.2E1,1.1E1,5E0,1E1,2E1,8E0,4E0,6E0,5E0,5E0,1.5E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.759476E-2,2.2812529E-2,-5.061224E-1,-1.1263222E-1,1.0611612E-1,-9.673372E-3,-2.8202647E-2,-1.7281608E-1,9.435549E-4,2.3669061E-1,5.7915013E-2,-2.286358E-1,-6.0411368E-2,-2.3361205E-3,2.8840434E-3,3.760488E-3,1.2037039E-2,8.164562E-3,1.3035301E-2,-3.5303698E-3,-2.7309614E-1,3.2271762E-4,-4.843644E-3,4.0815067E-3,-1.801196E-2,-5.5391397E-3,-1.4254254E-2,3.493249E-2,-2.957302E-3,-2.0820445E-3,3.3237783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,-1,-1,-1,23,-1,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[1.7723994E0,7.6791537E-1,1.513561E-1,1.7788422E-1,2.5063318E-1,0E0,0E0,9.285253E-2,4.063935E-2,4.8537076E-2,1.8853325E-1,4.889655E-2,2.5998276E-2,0E0,0E0,0E0,0E0,0E0,6.568345E-2,0E0,2.6003301E-2,0E0,0E0,0E0,5.3863045E-2,0E0,0E0,3.9567515E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,18,18,20,20,24,24,27,27],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,-1,-1,-1,24,-1,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.5414109E8,2.1647314E8,1.2096E4,3.5671377E0,2.325535E7,-9.673372E-3,-2.8202647E-2,3.6512393E2,9.18853E0,3.8198416E8,3.5010372E5,2.68E2,1.401E4,-2.3361205E-3,2.8840434E-3,3.760488E-3,1.2037039E-2,8.164562E-3,1.3E1,-3.5303698E-3,4.5859104E-1,3.2271762E-4,-4.843644E-3,4.0815067E-3,5.327983E8,-5.5391397E-3,-1.4254254E-2,7.84E3,-2.957302E-3,-2.0820445E-3,3.3237783E-3],"split_indices":[45,12,9,53,45,0,0,52,54,7,32,11,9,0,0,0,0,0,8,0,27,0,0,0,7,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.6E1,6E0,2.5E1,4.1E1,3E0,3E0,1.6E1,9E0,1E1,3.1E1,1E1,6E0,5E0,4E0,3E0,7E0,7E0,2.4E1,3E0,7E0,3E0,3E0,6E0,1.8E1,3E0,4E0,9E0,9E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.2696033E-2,-2.0930885E-1,1.13604575E-1,-2.9838702E-1,-2.7691113E-2,6.8088025E-2,3.6902377E-1,-3.4380555E-1,-3.2577056E-3,1.9661505E-3,-4.097642E-3,-5.626787E-3,8.707275E-2,2.508905E-2,6.9449507E-3,-8.780094E-3,-2.014807E-2,1.8381791E-1,5.423027E-2,3.3000312E-3,9.728457E-3,9.0375565E-2,-4.2705968E-2,-1.5911086E-4,1.0181595E-1,-4.5049815E-3,-2.5859752E-4,5.1225866E-3,1.6457686E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,-1,-1,-1,-1],"loss_changes":[1.7954413E0,3.8001347E-1,6.8404776E-1,1.352861E-1,4.9577374E-2,2.1236113E-1,3.0949605E-1,1.3253212E-1,0E0,0E0,0E0,0E0,1.4572173E-1,0E0,0E0,0E0,0E0,3.834212E-2,1.3700354E-1,0E0,0E0,3.0520767E-2,2.4622736E-2,0E0,2.3846567E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,17,17,18,18,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,4.0614333E0,3.9509753E3,1E0,9.256755E7,2.3688402E0,5.99999E9,8.2220456E5,-3.2577056E-3,1.9661505E-3,-4.097642E-3,-5.626787E-3,1.7329262E7,2.508905E-2,6.9449507E-3,-8.780094E-3,-2.014807E-2,6.498673E2,5.3920375E5,3.3000312E-3,9.728457E-3,3.786E3,1.0403953E3,-1.5911086E-4,7.53991E8,-4.5049815E-3,-2.5859752E-4,5.1225866E-3,1.6457686E-3],"split_indices":[41,54,52,16,45,54,5,28,0,0,0,0,45,0,0,0,0,4,28,0,0,2,52,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.3E1,6E1,1.5E1,8E0,5.2E1,8E0,1.2E1,3E0,4E0,4E0,4E0,4.8E1,3E0,5E0,7E0,5E0,1.1E1,3.7E1,4E0,7E0,2.7E1,1E1,3E0,2.4E1,3E0,7E0,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.4678428E-3,-2.3422952E-1,4.73607E-2,-4.2342767E-3,-1.3100968E-2,9.732043E-2,-1.4843525E-1,-3.4495117E-3,1.2342452E-1,-8.683877E-3,-1.9701239E-2,1.055637E-2,8.4631935E-2,-3.048812E-3,4.407044E-3,4.3589123E-2,1.1791568E-2,1.4780079E-2,4.264745E-3,-9.921147E-4,4.0220926E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,-1,7,9,-1,11,13,-1,-1,15,-1,-1,17,-1,19,-1,-1,-1],"loss_changes":[8.521338E-1,9.645444E-2,6.2092435E-1,0E0,0E0,2.4226391E-1,5.633207E-1,0E0,1.8979019E-1,7.7309705E-2,0E0,0E0,2.6250815E-1,0E0,0E0,4.704625E-2,0E0,6.5410495E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,9,9,12,12,15,15,17,17],"right_children":[2,4,6,-1,-1,8,10,-1,12,14,-1,-1,16,-1,-1,18,-1,20,-1,-1,-1],"split_conditions":[1.528E3,5.295928E-1,1.0232127E12,-4.2342767E-3,-1.3100968E-2,1.885E3,1.0202749E8,-3.4495117E-3,1.6256282E7,1.5E1,-1.9701239E-2,1.055637E-2,1.0011831E8,-3.048812E-3,4.407044E-3,1.2772205E3,1.1791568E-2,5.2518907E0,4.264745E-3,-9.921147E-4,4.0220926E-3],"split_indices":[11,27,31,0,0,2,45,0,45,3,0,0,45,0,0,52,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,1.2E1,6.1E1,5E0,7E0,4.9E1,1.2E1,6E0,4.3E1,9E0,3E0,9E0,3.4E1,6E0,3E0,2.9E1,5E0,2E1,9E0,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-6.2962943E-3,3.5496358E-2,-3.399379E-1,-1.1311451E-2,2.488019E-1,-2.0067945E-2,-2.6059947E-3,-6.1576482E-2,1.3076591E-1,1.859947E-2,6.391664E-3,-6.850429E-3,-1.2964153E-1,9.274452E-3,3.074991E-3,1.8079594E-2,-4.925148E-3,-7.928694E-3,-9.756059E-4,-3.9710565E-3,5.617174E-2,6.8172435E-3,5.1917305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.1275679E0,7.216886E-1,2.961781E-1,4.3654352E-1,1.7536354E-1,0E0,0E0,1.6673383E-1,6.5500766E-2,0E0,0E0,7.22945E-2,1.1492002E-1,0E0,0E0,9.818778E-2,0E0,0E0,0E0,0E0,7.534757E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,15,15,20,20],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.2159766E6,3.3795098E3,2.277961E6,5.517496E-1,7.33E2,-2.0067945E-2,-2.6059947E-3,5.6660336E-1,7.843361E5,1.859947E-2,6.391664E-3,1.6813238E-1,9.014711E-2,9.274452E-3,3.074991E-3,2.23E2,-4.925148E-3,-7.928694E-3,-9.756059E-4,-3.9710565E-3,2.325535E7,6.8172435E-3,5.1917305E-4],"split_indices":[28,4,1,41,0,0,0,27,32,0,0,42,39,0,0,10,0,0,0,0,45,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,5.9E1,1.2E1,5E0,3E0,4.4E1,1.5E1,3E0,9E0,2.5E1,1.9E1,5E0,1E1,2.1E1,4E0,1.2E1,7E0,5E0,1.6E1,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.8961756E-2,-1.8900508E-1,5.249203E-2,-2.7111888E-1,3.7989654E-3,1.823027E-2,1.40801435E-2,-1.6137661E-1,-1.659747E-2,1.5119879E-1,-4.879017E-2,-1.6829027E-3,-9.642337E-3,7.2035886E-4,2.0577563E-1,-1.814221E-2,-1.1722019E-2,4.2255507E-3,1.0558483E-2,-7.085719E-2,5.093006E-2,-1.0153317E-3,-4.9320213E-3,3.5038202E-3,-2.0127334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,-1,-1,17,19,-1,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[9.4548965E-1,5.48823E-1,5.1553446E-1,1.6849434E-1,0E0,4.5295742E-1,0E0,7.717165E-2,0E0,1.2118772E-1,2.2982162E-1,0E0,0E0,0E0,2.8395116E-2,1.16484195E-1,0E0,0E0,0E0,3.4979656E-2,4.6394322E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,14,14,15,15,19,19,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,-1,-1,18,20,-1,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,4.0230347E3,4.1E1,3.7989654E-3,2.325535E7,1.40801435E-2,8.918406E-2,-1.659747E-2,4.4939005E2,4.927835E0,-1.6829027E-3,-9.642337E-3,7.2035886E-4,1.7752522E-1,1.204781E3,-1.1722019E-2,4.2255507E-3,1.0558483E-2,5.2127117E-1,2.969772E0,-1.0153317E-3,-4.9320213E-3,3.5038202E-3,-2.0127334E-3],"split_indices":[58,4,52,8,0,45,0,57,0,4,54,0,0,0,27,52,0,0,0,27,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.2E1,5.4E1,1.7E1,5E0,4.9E1,5E0,1E1,7E0,1.6E1,3.3E1,4E0,6E0,5E0,1.1E1,3E1,3E0,4E0,7E0,1.7E1,1.3E1,9E0,8E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.6817848E-3,-2.2616802E-1,9.206123E-2,-1.0064887E-1,-4.0438184E-1,5.8719475E-2,2.2913849E-2,-7.919475E-3,-3.140276E-2,-2.117215E-2,-6.1804466E-3,-6.9123055E-3,7.877879E-2,1.6335608E-3,-4.646041E-3,1.4994971E-1,4.8611876E-2,5.5734406E-4,7.82964E-3,1.5576696E-2,1.7054671E-1,-1.7116611E-3,4.5251427E-3,1.0051072E-2,3.6281538E-3,-1.3453253E-3,1.6290755E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,-1,15,-1,-1,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1],"loss_changes":[1.7782433E0,5.185307E-1,8.415028E-1,8.7854594E-2,1.6418362E-1,2.6026985E-1,0E0,0E0,5.791353E-2,0E0,0E0,0E0,1.0435501E-1,0E0,0E0,6.568107E-2,1.5041274E-1,0E0,0E0,4.8545223E-2,2.4277136E-2,3.31714E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,12,12,15,15,16,16,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,-1,16,-1,-1,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.0789844E6,7.6312915E3,5E0,1.4235585E3,1.9956966E-1,2.2913849E-2,-7.919475E-3,9.502012E2,-2.117215E-2,-6.1804466E-3,-6.9123055E-3,2.6529046E7,1.6335608E-3,-4.646041E-3,1.0843103E5,9.599149E7,5.5734406E-4,7.82964E-3,5.2518907E0,3.882652E6,4.531609E6,4.5251427E-3,1.0051072E-2,3.6281538E-3,-1.3453253E-3,1.6290755E-3],"split_indices":[58,28,52,8,4,57,0,0,4,0,0,0,45,0,0,33,45,0,0,54,29,32,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.4E1,5.8E1,1.5E1,9E0,5.5E1,3E0,6E0,9E0,6E0,3E0,4E0,5.1E1,5E0,4E0,1.4E1,3.7E1,3E0,1.1E1,3E1,7E0,2.6E1,4E0,3E0,4E0,1.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.3843824E-2,-2.2078529E-1,8.560404E-2,-7.2355405E-4,-3.3810332E-1,4.831773E-2,2.659959E-1,-3.418276E-3,3.3625811E-3,-9.524407E-3,-2.0271981E-2,-2.8013967E-2,7.723982E-2,1.5543894E-2,5.7646437E-3,-2.8248949E-3,2.762108E-3,7.633473E-3,3.8658738E-2,1.4537007E-2,4.472216E-3,3.2292134E-3,-6.296606E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1,-1,19,21,-1,-1,-1],"loss_changes":[1.4594924E0,5.9355116E-1,3.1000277E-1,6.285858E-2,1.2970114E-1,9.267834E-2,5.332136E-2,0E0,0E0,0E0,0E0,4.654448E-2,1.1030732E-1,0E0,0E0,0E0,0E0,0E0,3.5053264E-2,3.32286E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,18,18,19,19],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1,-1,20,22,-1,-1,-1],"split_conditions":[1.0420895E-2,3.6588228E5,1.1997641E0,1.2E1,3.1457312E8,2.6277744E8,9.07E2,-3.418276E-3,3.3625811E-3,-9.524407E-3,-2.0271981E-2,4.57563E5,2.325535E7,1.5543894E-2,5.7646437E-3,-2.8248949E-3,2.762108E-3,7.633473E-3,3.3017554E2,4.4200982E8,4.472216E-3,3.2292134E-3,-6.296606E-4],"split_indices":[38,28,39,3,7,12,0,0,0,0,0,1,45,0,0,0,0,0,57,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.9E1,2.2E1,4.7E1,8E0,1.4E1,4E1,7E0,4E0,4E0,9E0,5E0,1.1E1,2.9E1,3E0,4E0,8E0,3E0,7E0,2.2E1,1.7E1,5E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-7.5184467E-4,-1.3201915E-1,1.3373545E-1,8.0386795E-2,-2.009081E-1,8.526345E-2,2.3981012E-2,-1.3384914E-3,1.4867929E-1,-2.0083936E-2,-1.5691182E-1,1.7738946E-1,4.602597E-2,8.84728E-3,2.2792364E-3,-1.8371418E-1,-1.0070925E-3,3.6515344E-3,1.0255674E-2,3.3308978E-3,5.2403803E-3,-1.181424E-2,-1.4481658E-1,1.7038058E-3,-2.2174134E-3,-2.304634E-3,-8.811093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,15,17,19,-1,-1,21,-1,-1,-1,23,-1,-1,25,-1,-1,-1,-1],"loss_changes":[1.4652659E0,6.3071203E-1,8.008842E-1,8.855475E-2,3.046682E-1,1.2919757E-1,0E0,0E0,2.7837962E-2,0E0,9.933102E-2,4.2705685E-2,9.1066234E-2,0E0,0E0,4.639697E-2,0E0,0E0,0E0,4.0348608E-2,0E0,0E0,9.466842E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,10,10,11,11,12,12,15,15,19,19,22,22],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,16,18,20,-1,-1,22,-1,-1,-1,24,-1,-1,26,-1,-1,-1,-1],"split_conditions":[4.283039E1,3.459181E-1,7.6312915E3,2.84E2,7E0,2.156825E7,2.3981012E-2,-1.3384914E-3,4.31E2,-2.0083936E-2,1.517E3,7.285458E-1,3.981664E7,8.84728E-3,2.2792364E-3,7.446808E-2,-1.0070925E-3,3.6515344E-3,1.0255674E-2,7.640048E7,5.2403803E-3,-1.181424E-2,1.5368E4,1.7038058E-3,-2.2174134E-3,-2.304634E-3,-8.811093E-3],"split_indices":[58,27,52,0,3,45,0,0,10,0,0,39,48,0,0,58,0,0,0,45,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,4.1E1,4E1,1E1,3.1E1,3.7E1,3E0,4E0,6E0,3E0,2.8E1,1E1,2.7E1,3E0,3E0,2.3E1,5E0,5E0,5E0,1.8E1,9E0,5E0,1.8E1,1.1E1,7E0,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-5.1782955E-3,-3.2325856E-2,1.3359509E-2,5.334816E-2,-1.058763E-1,-2.037616E-2,1.0874377E-1,-1.9263065E-1,-1.8165844E-2,-7.343333E-2,3.0553695E-3,9.943217E-3,7.100378E-2,-1.1755063E-2,-4.6106526E-3,3.9332747E-2,-1.1032636E-2,-5.766646E-3,-1.1908424E-3,5.5161244E-3,4.94738E-4,4.2416505E-3,-5.339266E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,-1,5,7,9,11,13,15,17,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5672225E-1,4.4674364E-1,0E0,1.369879E-1,2.856527E-1,7.832603E-2,7.2243735E-2,1.0291326E-1,2.8594252E-1,2.4242405E-2,0E0,0E0,5.3651884E-2,0E0,0E0,5.3966694E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,18,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,5.460315E9,1.3359509E-2,7.767353E2,2.667437E6,3.109375E0,3.1841638E5,1.2439503E3,9.71518E5,1.5039519E5,3.0553695E-3,9.943217E-3,1.0191781E1,-1.1755063E-2,-4.6106526E-3,3.309969E0,-1.1032636E-2,-5.766646E-3,-1.1908424E-3,5.5161244E-3,4.94738E-4,4.2416505E-3,-5.339266E-4],"split_indices":[52,5,0,4,1,53,32,4,9,33,0,0,56,0,0,54,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,3.2E1,3.7E1,1.4E1,1.8E1,1.8E1,1.9E1,9E0,5E0,3E0,1.5E1,8E0,1E1,1.6E1,3E0,3E0,6E0,7E0,8E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.7208477E-2,-3.595266E-2,3.2588106E-1,-1.5382302E-1,4.6290033E-2,2.2359533E-2,4.7245156E-3,-1.9070506E-2,-1.029633E-1,-5.3705927E-3,6.273173E-2,3.1597181E-3,-1.292435E-1,1.0151825E-1,1.1840872E-2,-1.8279533E-1,-2.5630025E-2,9.364669E-6,1.3129927E-1,-2.2221901E-3,2.9114087E-3,-1.1613733E-2,-4.7571445E-3,1.0529151E-3,-4.045425E-3,8.268006E-3,8.933545E-2,3.8189502E-4,4.9239676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,15,17,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.3408766E0,6.8569875E-1,4.5140564E-1,3.8495845E-1,1.2656963E-1,0E0,0E0,0E0,1.303899E-1,0E0,7.5779244E-2,0E0,1.2293485E-1,6.634076E-2,6.813111E-2,6.50807E-2,3.352238E-2,0E0,2.6989102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3874164E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,12,12,13,13,14,14,15,15,16,16,18,18,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,16,18,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[4.9891987E3,1.1576994E-2,8.982776E9,7E0,7.97E2,2.2359533E-2,4.7245156E-3,-1.9070506E-2,2.5346E3,-5.3705927E-3,5.090285E7,3.1597181E-3,1.7742582E3,2.727E3,2.6597537E3,5.9072212E-5,9.635135E7,9.364669E-6,1.6256282E7,-2.2221901E-3,2.9114087E-3,-1.1613733E-2,-4.7571445E-3,1.0529151E-3,-4.045425E-3,8.268006E-3,1.8E9,3.8189502E-4,4.9239676E-3],"split_indices":[4,38,5,3,10,0,0,0,32,0,45,0,4,2,4,38,45,0,45,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.9E1,1.1E1,2.8E1,4.1E1,5E0,6E0,3E0,2.5E1,3E0,3.8E1,3E0,2.2E1,2.1E1,1.7E1,1.4E1,8E0,5E0,1.6E1,8E0,9E0,5E0,9E0,5E0,3E0,5E0,1.1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.5003342E-2,-1.2385285E-1,3.199236E-2,-1.8356408E-1,3.4853884E-3,1.691761E-1,-7.5208065E-3,-9.232601E-2,-1.5646424E-2,9.289425E-3,1.3160805E-3,1.6775912E-2,-9.5922705E-3,-6.052781E-3,-3.775368E-4,-5.8194143E-3,3.479608E-2,9.331157E-2,1.233866E-2,4.0741113E-5,5.9582987E-3,4.076601E-2,-2.2735174E-3,-1.9654573E-3,2.943318E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,15,-1,-1,-1,-1,17,19,21,-1,-1,23,-1,-1,-1],"loss_changes":[4.3214613E-1,3.5830086E-1,2.6688004E-1,3.3218026E-1,0E0,6.647059E-2,2.0918739E-1,6.4500496E-2,0E0,0E0,0E0,1.0389972E-1,0E0,0E0,0E0,0E0,4.2214192E-2,3.8120978E-2,4.871987E-2,0E0,0E0,4.6971135E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,11,11,16,16,17,17,18,18,21,21],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,16,-1,-1,-1,-1,18,20,22,-1,-1,24,-1,-1,-1],"split_conditions":[7.2921924E-2,1.0735684E3,3.5175372E5,9.29295E5,3.4853884E-3,2.2980049E10,2.292109E7,1.348668E6,-1.5646424E-2,9.289425E-3,1.3160805E-3,2.074E3,-9.5922705E-3,-6.052781E-3,-3.775368E-4,-5.8194143E-3,1.7329262E7,4.5243898E-1,5.451104E5,4.0741113E-5,5.9582987E-3,2.721693E0,-2.2735174E-3,-1.9654573E-3,2.943318E-3],"split_indices":[42,52,32,28,0,5,47,1,0,0,0,2,0,0,0,0,45,27,28,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.7E1,4.8E1,2.1E1,6E0,1E1,3.8E1,1.5E1,6E0,7E0,3E0,3.5E1,3E0,9E0,6E0,3E0,3.2E1,8E0,2.4E1,3E0,5E0,1.7E1,7E0,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.6820382E-2,-7.763189E-3,-1.58089E-2,-1.0773235E-1,4.673664E-2,-1.5367822E-1,-8.975431E-3,1.9179458E-2,9.261775E-2,-2.6514006E-3,-1.9804698E-1,3.894548E-3,-3.1720933E-3,3.398798E-2,-2.6368145E-3,5.70798E-3,1.2813766E-3,-1.15429815E-2,-5.0636507E-3,2.8144785E-3,-2.9021094E-3,-2.6330808E-3,2.4026027E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,-1,5,7,9,11,13,15,-1,17,-1,-1,19,-1,-1,-1,-1,-1,-1,21,-1,-1],"loss_changes":[7.2253686E-1,3.8641617E-1,0E0,1.1205751E-1,5.638393E-2,5.6869358E-2,6.5465905E-2,3.800945E-2,3.9521575E-2,0E0,3.1004459E-2,0E0,0E0,3.0717986E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.852589E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,20,20],"right_children":[2,4,-1,6,8,10,12,14,16,-1,18,-1,-1,20,-1,-1,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.5414109E8,1.4007953E1,-1.58089E-2,4.593E3,1.0379711E0,1.762456E3,2.2858976E8,2.5922572E7,7.7591565E8,-2.6514006E-3,1.6633E4,3.894548E-3,-3.1720933E-3,9.343137E0,-2.6368145E-3,5.70798E-3,1.2813766E-3,-1.15429815E-2,-5.0636507E-3,2.8144785E-3,2.916545E3,-2.6330808E-3,2.4026027E-3],"split_indices":[45,58,0,2,42,47,7,32,7,0,9,0,0,57,0,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,2.4E1,4.5E1,1.6E1,8E0,2.9E1,1.6E1,6E0,1E1,3E0,5E0,2.5E1,4E0,9E0,7E0,4E0,6E0,1.3E1,1.2E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[4.8719868E-2,5.610019E-3,3.0250698E-1,-8.749326E-2,5.7985973E-2,2.1753406E-2,3.4324657E-3,-5.387793E-3,-2.0549142E-1,1.3562806E-1,2.2639867E-2,-5.5633806E-2,4.1130465E-3,-1.1024251E-2,-3.2354558E-3,9.438401E-4,9.133674E-3,-6.299642E-3,3.610687E-3,1.8283814E-4,-4.117503E-3,3.986649E-2,-5.6452423E-2,3.4658436E-3,-1.1015756E-3,-5.1110643E-3,-5.280906E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[7.5321287E-1,2.9765582E-1,4.1739923E-1,2.1203533E-1,1.03958875E-1,0E0,0E0,7.67995E-2,4.1613758E-2,1.0151906E-1,5.0604362E-2,2.4766162E-2,0E0,0E0,0E0,0E0,0E0,4.855776E-2,0E0,0E0,0E0,3.1815737E-2,2.633889E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,17,17,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[6.5317163E3,1.7489342E-2,1.3946067E7,9.635135E7,1.7329262E7,2.1753406E-2,3.4324657E-3,5.7630615E2,9.29342E5,4.4690585E2,5.0175633E0,5.7269626E0,4.1130465E-3,-1.1024251E-2,-3.2354558E-3,9.438401E-4,9.133674E-3,6.650418E-2,3.610687E-3,1.8283814E-4,-4.117503E-3,3.309969E0,4.828561E-1,3.4658436E-3,-1.1015756E-3,-5.1110643E-3,-5.280906E-4],"split_indices":[4,38,47,45,45,0,0,52,1,52,54,56,0,0,0,0,0,38,0,0,0,54,39,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,5.9E1,9E0,2.1E1,3.8E1,4E0,5E0,1.3E1,8E0,1.1E1,2.7E1,9E0,4E0,5E0,3E0,5E0,6E0,1.9E1,8E0,4E0,5E0,1E1,9E0,6E0,4E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[5.1178336E-3,-3.5064798E-2,3.4833393E-1,7.9573144E-4,-2.754099E-1,2.4909087E-2,5.7377797E-3,-9.032387E-2,5.5872798E-2,4.037428E-3,-1.9131992E-2,-1.1880068E-1,2.2681134E-3,1.655477E-1,1.7621385E-2,-7.609299E-2,-1.0638993E-2,2.5833421E-3,9.279461E-3,-3.336372E-3,3.7486546E-2,7.7561266E-3,-1.1866389E-1,3.4736774E-3,4.4034184E-3,3.5901912E-3,-3.009756E-3,-8.278113E-3,-3.0867772E-3,-2.1095695E-3,2.4652558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.184477E0,6.638746E-1,3.7315917E-1,3.4631148E-1,6.7827475E-1,0E0,0E0,1.1244637E-1,1.7747727E-1,0E0,0E0,1.0320121E-1,0E0,4.6391904E-2,6.562217E-2,6.5171964E-2,0E0,0E0,0E0,0E0,4.0163472E-2,4.7587115E-2,2.7812913E-2,0E0,5.1483214E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,14,14,15,15,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.4899646E8,1.523967E6,5.666351E7,2.7307262E5,2.4909087E-2,5.7377797E-3,1.7493458E3,1.7329262E7,4.037428E-3,-1.9131992E-2,1.0789844E6,2.2681134E-3,4.189857E8,4.57563E5,2.5722395E3,-1.0638993E-2,2.5833421E-3,9.279461E-3,-3.336372E-3,5.090285E7,5.0415697E5,2.6812036E2,3.4736774E-3,2.916545E3,3.5901912E-3,-3.009756E-3,-8.278113E-3,-3.0867772E-3,-2.1095695E-3,2.4652558E-3],"split_indices":[52,45,29,12,28,0,0,4,45,0,0,28,0,7,1,47,0,0,0,0,45,28,4,0,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,7.6E1,8E0,6.7E1,9E0,3E0,5E0,2.5E1,4.2E1,3E0,6E0,2.1E1,4E0,1E1,3.2E1,1.7E1,4E0,4E0,6E0,5E0,2.7E1,6E0,1.1E1,1.1E1,1.6E1,3E0,3E0,3E0,8E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5637407E-2,-2.5006428E-2,3.3905062E-1,-1.5941723E-1,4.133777E-2,2.185099E-2,4.2470414E-3,-7.091135E-2,-1.4007748E-2,5.673808E-2,-6.247722E-3,1.7179417E-3,-1.20544404E-1,1.5288186E-1,1.4230473E-2,-1.2896355E-3,-6.2999893E-3,2.6257324E-3,9.143504E-3,4.1992106E-2,-2.3015393E-3,6.445971E-2,-9.3140965E-4,-7.411087E-4,3.5054372E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,-1,15,17,19,-1,-1,-1,-1,21,-1,23,-1,-1,-1],"loss_changes":[1.0600588E0,6.486411E-1,3.2020152E-1,3.339417E-1,1.4973304E-1,0E0,0E0,9.857167E-2,0E0,1.858192E-1,0E0,0E0,2.4601787E-2,6.491217E-2,6.4603835E-2,0E0,0E0,0E0,0E0,3.5791658E-2,0E0,2.720429E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,12,12,13,13,14,14,19,19,21,21],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,-1,16,18,20,-1,-1,-1,-1,22,-1,24,-1,-1,-1],"split_conditions":[4.9891987E3,7.940699E-3,8.982776E9,1.0370839E6,2.953641E5,2.185099E-2,4.2470414E-3,4.4092423E-1,-1.4007748E-2,1.6256282E7,-6.247722E-3,1.7179417E-3,7.4338E4,4.189857E8,4.800481E5,-1.2896355E-3,-6.2999893E-3,2.6257324E-3,9.143504E-3,6.3315526E8,-2.3015393E-3,3.786E3,-9.3140965E-4,-7.411087E-4,3.5054372E-3],"split_indices":[4,38,5,28,33,0,0,27,0,45,0,0,12,7,28,0,0,0,0,7,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,2.3E1,4.8E1,4E0,4E0,1.6E1,7E0,4.5E1,3E0,5E0,1.1E1,1.3E1,3.2E1,3E0,8E0,6E0,7E0,2.3E1,9E0,1.7E1,6E0,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.2074906E-2,-1.9096171E-2,3.4432575E-1,-5.4967962E-2,8.952426E-2,2.3409376E-2,2.3584794E-3,3.024535E-2,-8.083202E-2,-4.1524414E-4,1.2565877E-1,-5.338709E-4,3.1969172E-3,-1.3285823E-2,-5.6436E-2,7.607575E-3,1.1269858E-3,-1.4143957E-1,-1.6384881E-2,-1.0491334E-3,-7.4432874E-3,3.075574E-3,-4.2548187E-2,-4.939354E-3,1.7371443E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,-1,-1,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1],"loss_changes":[8.4546125E-1,2.9715168E-1,3.8091153E-1,1.2807064E-1,6.935024E-2,0E0,0E0,2.756178E-2,2.2919422E-1,0E0,6.794058E-2,0E0,0E0,0E0,1.3726655E-1,0E0,0E0,4.53026E-2,7.0060916E-2,0E0,0E0,0E0,7.868144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,14,14,17,17,18,18,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,-1,-1,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1],"split_conditions":[4.0230347E3,7.114597E-1,1.904071E-1,2.7307262E5,1.9222778E5,2.3409376E-2,2.3584794E-3,3.015899E-2,2.8808794E5,-4.1524414E-4,2.156825E7,-5.338709E-4,3.1969172E-3,-1.3285823E-2,1.648E3,7.607575E-3,1.1269858E-3,1.1E1,1.774E3,-1.0491334E-3,-7.4432874E-3,3.075574E-3,4.8561827E8,-4.939354E-3,1.7371443E-4],"split_indices":[52,42,38,28,28,0,0,38,28,0,45,0,0,0,11,0,0,3,10,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.4E1,6E0,5.6E1,1.8E1,3E0,3E0,1.3E1,4.3E1,5E0,1.3E1,7E0,6E0,3E0,4E1,8E0,5E0,1.2E1,2.8E1,3E0,9E0,6E0,2.2E1,8E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-5.9960326E-3,2.7045945E-2,-2.5865164E-1,5.777442E-3,1.587435E-2,3.6735178E-3,-1.904019E-2,8.117006E-2,-2.6427446E-2,-1.9208613E-3,5.3964723E-3,-4.7913524E-3,-3.201398E-3,-1.8006846E-2,4.1957563E-3,3.6515899E-3,-3.8778543E-2,-2.1457253E-3,1.9245644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,13,15,-1,-1,17,-1,-1],"loss_changes":[6.4981E-1,5.029323E-1,6.1687857E-1,1.6239382E-1,0E0,0E0,0E0,1.192089E-1,9.3102075E-2,0E0,0E0,0E0,5.8768597E-2,7.633468E-2,0E0,0E0,2.9952977E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,12,12,13,13,16,16],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,14,16,-1,-1,18,-1,-1],"split_conditions":[1.5414109E8,2.1769184E7,2.7307262E5,2.5511668E7,1.587435E-2,3.6735178E-3,-1.904019E-2,1.51E3,4.593E3,-1.9208613E-3,5.3964723E-3,-4.7913524E-3,1.6132394E2,2.217E3,4.1957563E-3,3.6515899E-3,1.143417E6,-2.1457253E-3,1.9245644E-3],"split_indices":[45,1,28,45,0,0,0,2,2,0,0,0,57,10,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.8E1,8E0,6.5E1,3E0,3E0,5E0,1.9E1,4.6E1,5E0,1.4E1,9E0,3.7E1,3.3E1,4E0,5E0,2.8E1,2.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-5.711259E-3,-5.9429295E-2,9.795722E-2,-6.701965E-3,-1.8904324E-1,1.8801034E-4,1.24479495E-1,2.6698522E-2,-1.5843248E-1,-8.706835E-2,-2.6738772E-1,1.4957869E-1,6.092195E-4,-4.602374E-3,5.181901E-3,-1.15465885E-2,-3.0981613E-4,-3.0473905E-4,-5.7135304E-3,-1.41528435E-2,-5.1776837E-3,9.188629E-3,3.5230974E-3,2.6963288E-2,-3.8325759E-3,-2.293203E-3,5.869189E-3,-1.458305E-3,2.412053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,25,-1,27,-1,-1,-1],"loss_changes":[4.4019318E-1,3.541113E-1,6.645164E-2,1.9680908E-1,9.6556544E-2,0E0,5.5971086E-2,9.518956E-2,1.1612977E-1,2.8811987E-2,3.433603E-2,5.6880474E-2,0E0,6.950994E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1619308E-2,0E0,3.1797156E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,23,23,25,25],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,26,-1,28,-1,-1,-1],"split_conditions":[5.517496E-1,8.28645E-1,1.9222778E5,1.0129377E12,9.29295E5,1.8801034E-4,9.0161455E-1,2.8303965E3,4.077E3,1.2015E4,7.2767865E2,7.33E2,6.092195E-4,3.4941578E6,5.181901E-3,-1.15465885E-2,-3.0981613E-4,-3.0473905E-4,-5.7135304E-3,-1.41528435E-2,-5.1776837E-3,9.188629E-3,3.5230974E-3,3.241969E8,-3.8325759E-3,4.0614333E0,5.869189E-3,-1.458305E-3,2.412053E-3],"split_indices":[42,27,28,31,28,0,27,4,0,9,52,0,0,51,0,0,0,0,0,0,0,0,0,7,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,5.1E1,2.6E1,3.7E1,1.4E1,6E0,2E1,3.1E1,6E0,7E0,7E0,1.6E1,4E0,2.4E1,7E0,3E0,3E0,3E0,4E0,4E0,3E0,7E0,9E0,1.8E1,6E0,1.5E1,3E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.9165456E-4,-3.8781587E-2,3.4417918E-1,5.3044194E-3,-1.817248E-1,1.99703E-2,4.473567E-3,-8.6738095E-2,2.9731214E-2,-8.603092E-2,-1.6856587E-2,-2.9449616E-2,-8.294046E-3,7.4777864E-3,4.1837315E-3,-7.3961904E-3,5.6383194E-4,-3.3043532E-3,2.122967E-3,4.2452343E-2,-3.60083E-3,2.4153173E-3,-2.0879728E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,-1,-1,-1],"loss_changes":[1.0559442E0,4.546932E-1,1.8653047E-1,1.2848383E-1,3.1171566E-1,0E0,0E0,6.799406E-2,6.7467555E-2,1.14397846E-1,0E0,3.7877567E-2,0E0,1.15803E-1,0E0,0E0,0E0,0E0,0E0,3.605155E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,13,13,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,-1,-1,-1],"split_conditions":[4.0230347E3,1.0104842E12,8.982776E9,2.98E2,3.5715E4,1.99703E-2,4.473567E-3,1E0,1.204781E3,5.034889E6,-1.6856587E-2,1.58966E5,-8.294046E-3,6.98494E7,4.1837315E-3,-7.3961904E-3,5.6383194E-4,-3.3043532E-3,2.122967E-3,5.557455E8,-3.60083E-3,2.4153173E-3,-2.0879728E-3],"split_indices":[52,31,5,0,9,0,0,105,52,29,0,1,0,45,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.1E1,7E0,5.5E1,1.6E1,4E0,3E0,1.1E1,4.4E1,1.2E1,4E0,8E0,3E0,3.4E1,1E1,6E0,6E0,5E0,3E0,2.5E1,9E0,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.0148763E-2,5.6554414E-3,-2.7899873E-1,-9.1230854E-2,5.1017262E-2,-2.023174E-2,-6.473222E-4,-1.2074006E-1,3.3835531E-3,1.2188841E-1,1.9186279E-2,-6.4867088E-3,-9.940204E-4,8.8833953E-4,7.2855134E-3,-1.8867245E-2,7.7614225E-2,-1.7115562E-3,2.611574E-3,5.4111253E-3,1.0735798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,-1,-1,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[5.1589304E-1,3.1669497E-1,3.5136843E-1,1.251744E-1,1.0820043E-1,0E0,0E0,5.6821316E-2,0E0,6.99919E-2,7.928299E-2,0E0,0E0,0E0,0E0,3.9651208E-2,3.278754E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,15,15,16,16],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,-1,-1,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[1.5414109E8,4.279412E0,1.6E1,1.0735684E3,1.6256282E7,-2.023174E-2,-6.473222E-4,1.6825E4,3.3835531E-3,4.4939005E2,3.029697E3,-6.4867088E-3,-9.940204E-4,8.8833953E-4,7.2855134E-3,5.0175633E0,1.0129377E12,-1.7115562E-3,2.611574E-3,5.4111253E-3,1.0735798E-3],"split_indices":[45,58,3,52,45,0,0,9,0,4,4,0,0,0,0,54,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,2.2E1,4.8E1,3E0,3E0,1.9E1,3E0,1.4E1,3.4E1,1.4E1,5E0,5E0,9E0,2.1E1,1.3E1,1.7E1,4E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.2194844E-4,-2.5736807E-2,1.3198032E-2,1.7811785E-2,-1.6511537E-1,-6.985885E-2,5.2434742E-2,-6.9658205E-2,-1.958286E-2,-1.434458E-1,3.2796732E-3,2.7613118E-2,4.6206024E-3,-7.3789353E-3,2.8558375E-4,-7.75484E-3,-1.3593952E-3,3.210601E-3,-1.9622368E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,-1,5,7,9,11,13,-1,15,-1,17,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.18537E-1,4.3389258E-1,0E0,1.7061093E-1,4.4328308E-1,1.835511E-1,5.223558E-2,1.1111727E-1,0E0,4.0782303E-2,0E0,1.0461639E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,11,11],"right_children":[2,4,-1,6,8,10,12,14,-1,16,-1,18,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.660861E3,1.0232127E12,1.3198032E-2,5.02409E5,1.5135763E8,5.596155E-1,1.7830602E3,4.375793E6,-1.958286E-2,1.3513911E1,3.2796732E-3,5.7229916E7,4.6206024E-3,-7.3789353E-3,2.8558375E-4,-7.75484E-3,-1.3593952E-3,3.210601E-3,-1.9622368E-3],"split_indices":[52,31,0,1,45,39,52,1,0,56,0,45,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7E1,5E0,5.4E1,1.6E1,1.5E1,3.9E1,1.3E1,3E0,1E1,5E0,2.8E1,1.1E1,5E0,8E0,7E0,3E0,1.7E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.6400775E-2,-2.2976097E-2,2.3160684E-1,4.491022E-3,-2.2267409E-1,1.7653229E-2,1.8473951E-3,-1.3448575E-1,2.57857E-2,-1.4549338E-2,-8.578628E-4,-1.8978072E-3,-7.6834317E-3,3.6859646E-2,-3.6640125E-3,1.03758745E-1,1.8760912E-2,8.905121E-4,5.829795E-3,1.7850982E-3,2.6431878E-3,1.0591097E-3,-2.6358312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,15,-1,17,19,-1,-1,21,-1,-1,-1],"loss_changes":[6.578303E-1,3.6301848E-1,3.9185107E-1,1.7807628E-1,1.8372184E-1,0E0,0E0,2.4599954E-2,6.734595E-2,0E0,0E0,0E0,0E0,5.617126E-2,0E0,2.4171598E-2,2.8390463E-2,0E0,0E0,4.3796003E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,13,13,15,15,16,16,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,16,-1,18,20,-1,-1,22,-1,-1,-1],"split_conditions":[2.6949062E3,1.5135763E8,1.3946067E7,2.1E2,3.435678E8,1.7653229E-2,1.8473951E-3,1E0,1E0,-1.4549338E-2,-8.578628E-4,-1.8978072E-3,-7.6834317E-3,1.7329262E7,-3.6640125E-3,8.747313E6,3.1672727E3,8.905121E-4,5.829795E-3,7.76264E7,2.6431878E-3,1.0591097E-3,-2.6358312E-3],"split_indices":[52,45,47,10,12,0,0,104,105,0,0,0,0,45,0,45,4,0,0,45,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.5E1,1.1E1,5.8E1,7E0,5E0,6E0,7E0,5.1E1,4E0,3E0,3E0,4E0,4.7E1,4E0,9E0,3.8E1,3E0,6E0,2.8E1,1E1,2.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.6840015E-2,3.5289768E-2,-7.284617E-2,-4.283189E-3,5.815482E-2,-1.2934384E-1,2.505884E-4,-2.6282337E-3,7.1713045E-2,-1.9054475E-1,-2.6570966E-2,1.3743454E-1,4.287798E-2,-4.481184E-3,-1.0651398E-2,2.2508444E-3,-3.204954E-3,1.8109176E-3,7.608443E-3,-5.149703E-5,3.8947552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,-1,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.186203E-1,1.2661085E-1,1.6083947E-1,0E0,5.954753E-2,1.2702271E-1,0E0,0E0,5.2854672E-2,3.9152443E-2,3.842946E-2,2.711241E-2,4.897837E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,8,8,9,9,10,10,11,11,12,12],"right_children":[2,4,6,-1,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7376137E-1,3.7605792E2,1.6595918E-1,-4.283189E-3,2.23E2,1.4235585E3,2.505884E-4,-2.6282337E-3,6.1577463E0,9.29295E5,1.5368E4,2.1262457E0,1.0403953E3,-4.481184E-3,-1.0651398E-2,2.2508444E-3,-3.204954E-3,1.8109176E-3,7.608443E-3,-5.149703E-5,3.8947552E-3],"split_indices":[27,4,39,0,10,4,0,0,56,28,9,56,52,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3.8E1,3.5E1,5E0,3.3E1,2E1,1.5E1,3E0,3E1,1.2E1,8E0,8E0,2.2E1,6E0,6E0,3E0,5E0,3E0,5E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.2130767E-2,-8.4802E-3,2.1888897E-1,-7.729291E-2,4.2447068E-2,1.9493025E-2,5.2220885E-2,-2.2486696E-2,-5.272389E-3,4.797697E-3,7.241265E-4,-1.3039544E-3,5.2119405E-3,-5.7250157E-2,3.3669428E-3,-4.7865384E-3,-4.2821982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,15,-1,-1,-1],"loss_changes":[4.4755512E-1,2.310697E-1,3.7111074E-1,6.781736E-2,6.571936E-2,0E0,4.4047922E-2,5.3770743E-2,0E0,0E0,0E0,0E0,0E0,2.7291413E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,13,13],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,16,-1,-1,-1],"split_conditions":[6.7780703E3,6.0199605E6,9.192915E0,5.8871865E-1,1.7329262E7,1.9493025E-2,5.2E1,4.2332153E0,-5.272389E-3,4.797697E-3,7.241265E-4,-1.3039544E-3,5.2119405E-3,2.0435429E8,3.3669428E-3,-4.7865384E-3,-4.2821982E-4],"split_indices":[4,51,57,27,45,0,8,54,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.4E1,9E0,2.7E1,3.7E1,3E0,6E0,1.3E1,1.4E1,9E0,2.8E1,3E0,3E0,1E1,3E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.708013E-3,-1.3678592E-2,2.6891476E-1,4.5294645E-3,-1.7356527E-1,2.0133464E-2,-9.042819E-6,-5.253121E-2,4.2906906E-2,3.5952816E-3,-1.475177E-2,-7.14166E-2,2.2408804E-3,1.0652952E-1,2.2932407E-2,6.7398176E-3,-4.3697716E-3,-8.8201766E-4,7.0119365E-3,5.8309175E-3,2.9843901E-3,-1.8544922E-3,2.8945482E-3,1.4451584E-3,-1.5126783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[5.128834E-1,2.2786173E-1,3.8045102E-1,1.5773278E-1,3.822744E-1,0E0,0E0,6.1211832E-2,5.220271E-2,0E0,0E0,6.0807794E-2,0E0,7.64219E-2,2.6837666E-2,2.7366389E-2,0E0,0E0,0E0,3.115299E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,14,14,15,15,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,1.1408508E9,3.1457312E8,2.7307262E5,2.0133464E-2,-9.042819E-6,4.7985487E0,1.6256282E7,3.5952816E-3,-1.475177E-2,1.5368E4,2.2408804E-3,1.9222778E5,5.0175633E0,1.5023E4,-4.3697716E-3,-8.8201766E-4,7.0119365E-3,3.28464E0,2.9843901E-3,-1.8544922E-3,2.8945482E-3,1.4451584E-3,-1.5126783E-3],"split_indices":[52,45,7,7,28,0,0,54,45,0,0,9,0,28,54,9,0,0,0,54,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.7E1,6E0,7E1,7E0,3E0,3E0,2.8E1,4.2E1,3E0,4E0,2.4E1,4E0,9E0,3.3E1,7E0,1.7E1,3E0,6E0,2.5E1,8E0,4E0,3E0,1.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.4178472E-3,-1.9839479E-2,1.3543539E-2,2.2233801E-3,-5.4569315E-2,-9.610083E-2,1.768151E-2,-6.870575E-2,-8.305915E-3,2.7290715E-3,-1.0951506E-3,-4.373685E-3,-8.121649E-3,2.6205312E-3,-2.762623E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,-1,5,7,9,11,-1,-1,-1,-1,13,-1,-1],"loss_changes":[4.2823428E-1,1.7497307E-1,0E0,0E0,1.4273147E-1,6.730312E-2,3.758589E-2,5.0016E-2,0E0,0E0,0E0,0E0,4.315765E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,-1,-1,6,8,10,12,-1,-1,-1,-1,14,-1,-1],"split_conditions":[2.1769184E7,3.6130127E-1,1.3543539E-2,2.2233801E-3,4.283039E1,2.0333653E5,5.4734116E9,5.4111224E2,-8.305915E-3,2.7290715E-3,-1.0951506E-3,-4.373685E-3,1.5326E4,2.6205312E-3,-2.762623E-3],"split_indices":[1,27,0,0,58,33,5,52,0,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.8E1,3E0,2.2E1,4.6E1,2.9E1,1.7E1,2.4E1,5E0,8E0,9E0,1.5E1,9E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.3094041E-3,-2.1720128E-2,1.2349375E-2,-1.1930148E-1,6.5481183E-3,-8.695391E-4,-6.0114884E-3,2.242943E-2,-8.788545E-3,-4.956161E-3,3.6805633E-3,-1.9529995E-2,3.2511167E-3,2.3623288E-2,-2.6314133E-3,3.5430347E-3,-4.166359E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,7,-1,-1,9,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[4.5034984E-1,1.966314E-1,0E0,3.0829445E-2,1.9320983E-1,0E0,0E0,9.279443E-2,0E0,4.6333138E-2,0E0,6.127045E-2,0E0,2.484786E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,9,9,11,11,13,13],"right_children":[2,4,-1,6,8,-1,-1,10,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[7.6312915E3,3.731E3,1.2349375E-2,4.0246898E-1,2.5805172E7,-8.695391E-4,-6.0114884E-3,7.9276E-2,-8.788545E-3,7.0826065E1,3.6805633E-3,6.98494E7,3.2511167E-3,2.3209375E6,-2.6314133E-3,3.5430347E-3,-4.166359E-5],"split_indices":[52,11,0,27,47,0,0,38,0,57,0,45,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7E1,4E0,1.5E1,5.5E1,3E0,1.2E1,5.2E1,3E0,3.7E1,1.5E1,3.2E1,5E0,1.6E1,1.6E1,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[6.4163175E-3,-1.9764232E-2,2.2387463E-1,6.819803E-3,-2.43258E-1,3.4413047E-3,1.637866E-2,3.8325094E-2,-3.4364402E-2,-3.923652E-3,-1.3505083E-2,-2.6299434E-3,5.8338765E-2,-5.6182365E-3,-1.8718965E-2,-4.3122875E-4,7.7148154E-2,1.7720513E-3,-1.6198772E-3,9.911954E-4,4.2697773E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,15,-1,17,-1,19,-1,-1,-1,-1],"loss_changes":[4.769585E-1,4.472614E-1,1.7455706E-1,8.954841E-2,5.895999E-2,0E0,0E0,8.15047E-2,4.3076247E-2,0E0,0E0,0E0,4.324872E-2,0E0,3.2692313E-2,0E0,2.9536694E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,12,12,14,14,16,16],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,16,-1,18,-1,20,-1,-1,-1,-1],"split_conditions":[4.9891987E3,1.15266744E8,1.3996E4,5.090285E7,9.3570206E5,3.4413047E-3,1.637866E-2,2.0435429E8,7.801514E2,-3.923652E-3,-1.3505083E-2,-2.6299434E-3,2.5630938E5,-5.6182365E-3,1.120848E5,-4.3122875E-4,6.8177136E2,1.7720513E-3,-1.6198772E-3,9.911954E-4,4.2697773E-3],"split_indices":[4,45,9,45,28,0,0,7,4,0,0,0,28,0,32,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,6.7E1,7E0,5E0,3E0,3.8E1,2.9E1,3E0,4E0,6E0,3.2E1,3E0,2.6E1,7E0,2.5E1,6E0,2E1,8E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.0528256E-3,-2.9798124E-2,2.4247636E-1,3.5122198E-3,-1.5853176E-1,1.5186589E-2,5.5053935E-4,-5.8366675E-2,2.030846E-2,-1.118973E-2,9.6615084E-4,-7.1998354E-4,-4.8383037E-3,4.579047E-3,3.7899753E-3,2.2516385E-2,-2.525277E-3,2.4012732E-3,-2.783299E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,15,17,-1,-1,-1],"loss_changes":[5.892119E-1,2.9215178E-1,2.2822142E-1,5.834864E-2,2.6743665E-1,0E0,0E0,2.5392842E-2,6.2660165E-2,0E0,0E0,0E0,0E0,0E0,4.604858E-2,2.965433E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,14,14,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,16,18,-1,-1,-1],"split_conditions":[3.3072104E3,1.0789844E6,8.982776E9,2.566528E0,1.650581E3,1.5186589E-2,5.5053935E-4,1.9227438E5,1.3024019E7,-1.118973E-2,9.6615084E-4,-7.1998354E-4,-4.8383037E-3,4.579047E-3,5.451104E5,1E0,-2.525277E-3,2.4012732E-3,-2.783299E-4],"split_indices":[52,28,5,54,4,0,0,33,45,0,0,0,0,0,28,104,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,5.4E1,1.3E1,5E0,3E0,1.1E1,4.3E1,8E0,5E0,7E0,4E0,6E0,3.7E1,2.9E1,8E0,1.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.8821817E-2,6.513124E-4,1.5755745E-2,1.5206267E-2,-1.16494335E-2,-8.7076865E-2,3.607094E-2,3.1107303E-3,-6.085089E-3,9.0867594E-2,1.5483406E-2,1.9333474E-3,6.3988105E-3,2.9734147E-3,4.296543E-3,-4.5213415E-4,2.0260846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,13,-1,-1,15,-1,-1,-1],"loss_changes":[5.1433367E-1,3.1415313E-1,0E0,1.6319874E-1,0E0,1.2508008E-1,6.966513E-2,0E0,0E0,3.8847312E-2,4.9672425E-2,0E0,0E0,2.5765456E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9,10,10,13,13],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,14,-1,-1,16,-1,-1,-1],"split_conditions":[7.6312915E3,1.5277152E8,1.5755745E-2,1.7363004E4,-1.16494335E-2,1.2015E4,2.156825E7,3.1107303E-3,-6.085089E-3,4.1108544E8,3.4409692E3,1.9333474E-3,6.3988105E-3,4.9153895E0,4.296543E-3,-4.5213415E-4,2.0260846E-3],"split_indices":[52,45,0,51,0,9,45,0,0,7,52,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,7.4E1,3E0,1.2E1,6.2E1,3E0,9E0,1.6E1,4.6E1,1E1,6E0,4.1E1,5E0,3.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.4485215E-2,2.5037797E-3,-1.8230584E-1,-3.8953675E-3,2.7529225E-2,-2.2674387E-3,-1.13755455E-2,8.252429E-3,9.065614E-2,-3.3313275E-4,4.0266677E-3,7.7218297E-3,-5.221486E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,-1,7,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[2.1714033E-1,1.6646607E-1,6.1647728E-2,0E0,6.7397416E-2,0E0,0E0,6.137479E-2,1.2244543E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,8,8],"right_children":[2,4,6,-1,8,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.648E3,4.66E2,-3.8953675E-3,1.891862E3,-2.2674387E-3,-1.13755455E-2,1.0379711E0,1.3946067E7,-3.3313275E-4,4.0266677E-3,7.7218297E-3,-5.221486E-4],"split_indices":[45,11,0,0,52,0,0,42,47,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,1.4E1,5.5E1,3E0,3E0,4.3E1,1.2E1,3.7E1,6E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-5.363816E-3,-1.0751262E-1,2.6641496E-2,-2.35924E-2,-9.854531E-3,-6.254303E-2,3.694581E-2,-2.992021E-3,4.2011673E-3,8.7164465E-4,-5.552096E-3,5.8769984E-3,2.695744E-2,6.0710985E-2,-4.393295E-5,-6.2641635E-4,3.477225E-3,-1.0101526E-3,3.0231779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,-1,-1,-1,-1,-1,13,15,17,-1,-1,-1,-1],"loss_changes":[2.742166E-1,2.0089172E-1,6.1125763E-2,8.032447E-2,0E0,4.170724E-2,5.4508865E-2,0E0,0E0,0E0,0E0,0E0,4.921784E-2,3.8453184E-2,5.3374946E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,12,12,13,13,14,14],"right_children":[2,4,6,8,-1,10,12,-1,-1,-1,-1,-1,14,16,18,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,9.29295E5,2.074E3,1E0,-9.854531E-3,1.297959E7,2.2235682E6,-2.992021E-3,4.2011673E-3,8.7164465E-4,-5.552096E-3,5.8769984E-3,5.090285E7,2.5630938E5,4.9767124E1,-6.2641635E-4,3.477225E-3,-1.0101526E-3,3.0231779E-3],"split_indices":[41,28,2,16,0,12,51,0,0,0,0,0,45,28,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,1.9E1,6.3E1,1.2E1,7E0,6E0,5.7E1,9E0,3E0,3E0,3E0,4E0,5.3E1,2.3E1,3E1,5E0,1.8E1,2.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.693405E-2,4.13869E-3,1.9617298E-1,-5.844114E-2,3.7422627E-2,1.499067E-2,2.589713E-3,-1.5262753E-1,-1.8087281E-3,4.163394E-3,8.438932E-4,-1.6765323E-3,-1.0071796E-2,-1.434124E-3,2.4229684E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9238567E-1,1.4382444E-1,1.6719195E-1,1.2773997E-1,4.578729E-2,0E0,0E0,7.538779E-2,3.1581394E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9891987E3,2.8506322E-2,3.6274084E9,3.0810812E0,3.5175372E5,1.499067E-2,2.589713E-3,9.29295E5,9.402116E2,4.163394E-3,8.438932E-4,-1.6765323E-3,-1.0071796E-2,-1.434124E-3,2.4229684E-3],"split_indices":[4,39,12,53,32,0,0,28,52,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,2.3E1,4.4E1,3E0,5E0,8E0,1.5E1,9E0,3.5E1,4E0,4E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.064792E-2,2.0973708E-2,-1.3646895E-1,-4.197483E-2,4.283103E-2,-5.771324E-2,-1.2751524E-2,-4.0191924E-3,2.9526069E-3,2.0472784E-2,4.0910444E-3,-4.918086E-3,8.017551E-4,3.567889E-3,-8.048857E-3,-1.6376469E-3,1.1935147E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1,15,-1,-1],"loss_changes":[3.1717682E-1,8.985457E-2,1.8657237E-1,1.04654364E-1,5.474507E-2,5.472601E-2,0E0,0E0,0E0,6.352257E-2,0E0,0E0,0E0,0E0,2.8259704E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,14,14],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.0232127E12,2.4786325E0,1.3946397E5,1E0,1.7830602E3,7.575E3,-1.2751524E-2,-4.0191924E-3,2.9526069E-3,1.6939252E7,4.0910444E-3,-4.918086E-3,8.017551E-4,3.567889E-3,3.0546596E0,-1.6376469E-3,1.1935147E-3],"split_indices":[31,56,33,101,52,2,0,0,0,45,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,6.3E1,1.5E1,1.6E1,4.7E1,1.1E1,4E0,1.1E1,5E0,3.4E1,1.3E1,6E0,5E0,1E1,2.4E1,1.3E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-2.2003432E-3,1.3855694E-2,-1.1434812E-1,-1.1150522E-2,6.817658E-2,3.4854615E-3,-9.6584875E-3,-2.1333362E-3,2.6386695E-2,3.7836058E-3,2.591604E-4,4.293631E-3,1.0609669E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,-1,-1,-1],"loss_changes":[1.2943046E-1,8.633688E-2,2.1496859E-1,6.5295205E-2,2.4684705E-2,0E0,0E0,0E0,4.447379E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.1808436E3,2.7307262E5,1.9129465E0,4.815659E6,3.4854615E-3,-9.6584875E-3,-2.1333362E-3,9.9948E4,3.7836058E-3,2.591604E-4,4.293631E-3,1.0609669E-5],"split_indices":[45,52,28,57,29,0,0,0,29,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.2E1,8E0,4.3E1,1.9E1,3E0,5E0,2.1E1,2.2E1,1.4E1,5E0,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.0043612E-3,1.9909082E-2,-1.1606994E-2,7.1236794E-3,1.0069891E-2,-4.2050085E-3,1.6898869E-2,-5.4443326E-2,2.8062813E-2,1.2662858E-4,-4.317341E-3,7.0129715E-2,1.371399E-2,-1.040023E-3,4.195067E-3,-3.697128E-4,3.686076E-3,1.6813657E-3,-1.3889614E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,-1,-1,7,9,11,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[3.968184E-1,1.9676611E-1,0E0,7.27905E-2,0E0,0E0,5.250129E-2,2.4330903E-2,3.2840434E-2,0E0,0E0,3.9370775E-2,4.3927696E-2,0E0,0E0,4.842964E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,11,11,12,12,15,15],"right_children":[2,4,-1,6,-1,-1,8,10,12,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[2.9851845E10,2.1747866E7,-1.1606994E-2,1.09E2,1.0069891E-2,-4.2050085E-3,1.1E1,5.86E2,1.6939252E7,1.2662858E-4,-4.317341E-3,1.9222778E5,2.001E3,-1.040023E-3,4.195067E-3,4.0526348E-1,3.686076E-3,1.6813657E-3,-1.3889614E-3],"split_indices":[5,1,0,10,0,0,3,0,45,0,0,28,0,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.1E1,4E0,6.8E1,3E0,5E0,6.3E1,8E0,5.5E1,4E0,4E0,1.3E1,4.2E1,3E0,1E1,3.6E1,6E0,1.6E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.200589E-3,-2.1732036E-2,1.0016172E-2,-2.6431836E-3,-1.3602486E-1,-3.1311153E-3,1.9123519E-2,3.4403803E-3,-2.2726008E-1,2.9781288E-3,-7.403925E-3,-1.426959E-2,-1.5997718E-3,-1.9696243E-3,8.791814E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1],"loss_changes":[3.075554E-1,1.6763279E-1,0E0,1.0432693E-1,2.3553672E-1,0E0,7.020253E-2,0E0,1.4915979E-1,0E0,3.877893E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,10,10],"right_children":[2,4,-1,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.0016172E-2,6.942296E2,2.7307262E5,-3.1311153E-3,5.7229916E7,3.4403803E-3,1.835052E6,2.9781288E-3,1.077646E3,-1.426959E-2,-1.5997718E-3,-1.9696243E-3,8.791814E-4],"split_indices":[52,45,0,4,28,0,45,0,32,0,52,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.6E1,4E0,6.6E1,1E1,1.5E1,5.1E1,3E0,7E0,1.7E1,3.4E1,4E0,3E0,1.4E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.6793303E-2,-9.393666E-3,-5.1272856E-3,5.90596E-3,-8.185342E-3,-7.9633825E-2,2.5657946E-2,-1.709952E-3,-6.7926743E-3,4.0292032E-2,-1.619516E-3,1.4328537E-2,3.7665039E-3,2.9905946E-3,-1.9751016E-2,-2.4464256E-3,1.8638973E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,-1,3,5,-1,7,9,-1,-1,11,-1,13,-1,-1,15,-1,-1],"loss_changes":[1.740114E-1,0E0,1.4705747E-1,1.1693473E-1,0E0,3.447441E-2,5.358901E-2,0E0,0E0,5.5809796E-2,0E0,6.0126282E-2,0E0,0E0,4.9942862E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,9,9,11,11,14,14],"right_children":[2,-1,4,6,-1,8,10,-1,-1,12,-1,14,-1,-1,16,-1,-1],"split_conditions":[7E0,-9.393666E-3,2.292109E7,1.5278552E4,-8.185342E-3,9.1569895E-1,6.349519E6,-1.709952E-3,-6.7926743E-3,1.1808436E3,-1.619516E-3,1.1795429E9,3.7665039E-3,2.9905946E-3,3.5671377E0,-2.4464256E-3,1.8638973E-3],"split_indices":[3,0,47,51,0,27,32,0,0,52,0,5,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3E0,7E1,6.7E1,3E0,1.2E1,5.5E1,9E0,3E0,4.5E1,1E1,3E1,1.5E1,1.1E1,1.9E1,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.1337942E-3,-1.0757713E-2,8.935454E-3,2.2692056E-3,-1.2832415E-1,-5.536229E-3,1.3209011E-2,-9.376748E-3,-2.2647734E-4,-3.0485902E-3,2.391797E-2,1.5370602E-2,4.47338E-3,-2.4051809E-4,4.9921643E-2,3.2920255E-3,-3.1927234E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,-1,9,-1,-1,-1,11,13,-1,-1,15,-1,-1],"loss_changes":[1.8265042E-1,1.0749818E-1,0E0,9.394156E-2,7.716199E-2,0E0,5.599886E-2,0E0,0E0,0E0,3.563524E-2,3.655076E-2,0E0,0E0,3.0081086E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,10,10,11,11,14,14],"right_children":[2,4,-1,6,8,-1,10,-1,-1,-1,12,14,-1,-1,16,-1,-1],"split_conditions":[6.5915E4,3.698723E10,8.935454E-3,7.446808E-2,3.132E3,-5.536229E-3,1.296E3,-9.376748E-3,-2.2647734E-4,-3.0485902E-3,1.7823958E0,1.2544873E3,4.47338E-3,-2.4051809E-4,6.7584877E8,3.2920255E-3,-3.1927234E-4],"split_indices":[2,5,0,58,0,0,2,0,0,0,41,52,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.9E1,3E0,6.3E1,6E0,4E0,5.9E1,3E0,3E0,6E0,5.3E1,4.9E1,4E0,3.1E1,1.8E1,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-7.395596E-3,-8.771179E-3,3.8474181E-3,1.6381849E-2,-7.050952E-3,3.8326194E-3,9.11174E-3,-3.9967686E-2,2.9378345E-2,6.267657E-3,-4.503994E-3,1.7474805E-3,-1.9000736E-3,-3.121586E-3,1.3755604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,-1,3,5,-1,7,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[1.6528434E-1,0E0,1.5286219E-1,1.6456185E-1,0E0,7.278447E-2,0E0,7.318258E-2,3.657425E-2,3.3141453E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,9,9],"right_children":[2,-1,4,6,-1,8,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[5E0,-8.771179E-3,1.3866357E8,3.9509753E3,-7.050952E-3,2.881172E8,9.11174E-3,8.0159146E-1,2.5922572E7,1.001E3,-4.503994E-3,1.7474805E-3,-1.9000736E-3,-3.121586E-3,1.3755604E-3],"split_indices":[3,0,48,52,0,7,0,27,32,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3E0,7E1,6.6E1,4E0,6.3E1,3E0,2.3E1,4E1,1.4E1,9E0,3.5E1,5E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.0723137E-3,-8.5909795E-3,1.0697194E-2,9.614017E-4,-8.649597E-3,8.117288E-3,-6.012843E-3,-3.850339E-3,1.4552777E-2,6.908579E-3,3.7556298E-3,5.6384385E-2,-6.7104334E-3,3.755906E-4,4.1453713E-3,-3.2477104E-3,2.124768E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,-1,13,15,-1,-1,-1,-1],"loss_changes":[2.557413E-1,1.5745021E-1,0E0,8.4211655E-2,0E0,5.121307E-2,0E0,0E0,4.1400604E-2,4.6871368E-2,0E0,2.8104484E-2,4.4851087E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,11,11,12,12],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,-1,14,16,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,1.0697194E-2,2.2306405E6,-8.649597E-3,5.734767E-2,-6.012843E-3,-3.850339E-3,4.9891987E3,1.7329262E7,3.7556298E-3,2.9187737E0,2.35764E5,3.755906E-4,4.1453713E-3,-3.2477104E-3,2.124768E-4],"split_indices":[1,2,0,28,0,58,0,0,4,45,0,54,29,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.4E1,3E0,8.1E1,3E0,7.8E1,3E0,4E0,7.4E1,6.8E1,6E0,1.4E1,5.4E1,7E0,7E0,7E0,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.8342175E-2,4.210557E-3,1.2976714E-2,1.2470733E-2,-6.84129E-3,-1.9795425E-2,2.2920885E-3,2.4096826E-3,-8.463627E-3,-2.6712036E-2,4.958187E-2,-4.322486E-2,1.5195858E-3,4.8055174E-3,2.8589224E-5,-2.848112E-3,1.1254408E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,-1,7,-1,9,-1,11,13,15,-1,-1,-1,-1,-1],"loss_changes":[3.4119606E-1,1.1320739E-1,0E0,1.0589931E-1,0E0,1.7846255E-1,0E0,5.764757E-2,0E0,2.8260658E-2,4.906754E-2,3.589257E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,9,9,10,10,11,11],"right_children":[2,4,-1,6,-1,8,-1,10,-1,12,14,16,-1,-1,-1,-1,-1],"split_conditions":[9.55721E3,2.5805172E7,1.2976714E-2,1.7807747E3,-6.84129E-3,1.052686E6,2.2920885E-3,4.3652397E8,-8.463627E-3,2.1E1,2.325535E7,2.68219E5,1.5195858E-3,4.8055174E-3,2.8589224E-5,-2.848112E-3,1.1254408E-3],"split_indices":[4,47,0,4,0,28,0,7,0,3,45,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8.1E1,3E0,7.8E1,3E0,4.4E1,3.4E1,4E1,4E0,2.5E1,1.5E1,2E1,5E0,6E0,9E0,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.5442309E-2,5.7833386E-4,-1.7371845E-1,-2.5272978E-2,4.9490523E-2,-9.4285025E-4,-1.3678969E-2,-4.053253E-2,1.8985323E-3,1.0105279E-1,4.503017E-3,-5.2775275E-2,9.0473064E-4,2.198966E-3,8.546961E-3,-2.745345E-3,1.4006452E-3,-3.7324363E-3,-1.9081002E-2,-1.4987033E-3,2.28966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,-1,-1,-1,-1,-1,-1,19,-1,-1],"loss_changes":[2.207051E-1,1.02406316E-1,1.7029025E-1,5.8144998E-2,6.449538E-2,0E0,0E0,3.434737E-2,0E0,5.347757E-2,3.3156414E-2,4.1352957E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.506599E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,18,18],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,-1,-1,-1,-1,-1,-1,20,-1,-1],"split_conditions":[1.5135763E8,1.204781E3,1.6867E4,1.062872E6,1.4302E4,-9.4285025E-4,-1.3678969E-2,2.1E1,1.8985323E-3,9.698E3,1.2204E4,2.289789E6,9.0473064E-4,2.198966E-3,8.546961E-3,-2.745345E-3,1.4006452E-3,-3.7324363E-3,5.0175633E0,-1.4987033E-3,2.28966E-3],"split_indices":[45,52,9,9,10,0,0,3,0,2,2,51,0,0,0,0,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,5.2E1,2.7E1,4E0,3E0,4.3E1,9E0,1.2E1,1.5E1,3.6E1,7E0,9E0,3E0,4E0,1.1E1,1.7E1,1.9E1,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.1001062E-2,2.2680014E-2,-5.52554E-2,-3.5939754E-3,1.5027825E-3,-8.571232E-2,6.935895E-4,-6.4120725E-2,-7.5964686E-3,1.452593E-3,-8.235601E-2,-4.3654586E-3,-4.5523487E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,-1,7,-1,9,-1,-1,11,-1,-1],"loss_changes":[1.17514804E-1,6.280247E-2,7.540477E-2,0E0,0E0,3.6246404E-2,0E0,4.035978E-2,0E0,0E0,2.4378404E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,10,10],"right_children":[2,4,6,-1,-1,8,-1,10,-1,-1,12,-1,-1],"split_conditions":[7.8345644E-1,3.235021E2,1.6925E4,-3.5939754E-3,1.5027825E-3,1.58206E6,6.935895E-4,2.202021E8,-7.5964686E-3,1.452593E-3,7.0333635E6,-4.3654586E-3,-4.5523487E-4],"split_indices":[27,4,10,0,0,28,0,7,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,4.4E1,3.3E1,4E0,4E1,2.3E1,1E1,2E1,3E0,3E0,1.7E1,1.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.6991522E-2,5.2764895E-3,1.0940148E-2,1.7171655E-2,-7.856569E-3,6.267542E-2,1.7175402E-3,-4.366747E-4,4.047201E-3,-1.8609276E-3,1.6430588E-2,4.1084653E-3,8.903775E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[2.3927875E-1,1.8975322E-1,0E0,5.544676E-2,0E0,4.7565363E-2,4.064727E-2,0E0,0E0,0E0,5.236745E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,10,10],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.0940148E-2,1.7329262E7,-7.856569E-3,1.63254E5,5.2886304E2,-4.366747E-4,4.047201E-3,-1.8609276E-3,2.5036643E8,4.1084653E-3,8.903775E-5],"split_indices":[52,45,0,45,0,1,52,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8.2E1,3E0,7.8E1,4E0,1.9E1,5.9E1,6E0,1.3E1,1.4E1,4.5E1,6E0,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1138222E-2,-8.2536833E-4,1.058927E-2,-4.8880085E-2,1.2394956E-3,-1.3220548E-2,-1.345789E-1,-2.3657857E-3,1.2821914E-3,-8.202078E-3,-6.4570416E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,-1],"loss_changes":[2.3503102E-1,1.16019346E-1,0E0,9.295617E-2,0E0,4.3495543E-2,5.8636725E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,-1],"split_conditions":[7.6312915E3,1.03593215E-1,1.058927E-2,1.052622E6,1.2394956E-3,1.3E1,1.4235585E3,-2.3657857E-3,1.2821914E-3,-8.202078E-3,-6.4570416E-4],"split_indices":[52,42,0,28,0,3,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,3E1,4.9E1,2.2E1,8E0,1.1E1,1.1E1,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-4.331802E-3,5.3937766E-3,-7.135269E-3,-5.879663E-3,1.42568E-2,-1.999665E-3,3.19095E-3,4.591095E-2,-2.3248835E-2,-3.5294245E-3,3.557072E-3,-3.568722E-2,1.6566316E-3,-1.7328085E-2,-3.385792E-3,-3.899823E-3,-5.204307E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,-1,5,7,-1,9,11,-1,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.3999574E-1,1.08338825E-1,0E0,0E0,7.865125E-2,6.6223264E-2,0E0,1.02607176E-1,3.6079135E-2,0E0,0E0,2.8967366E-2,0E0,3.3211436E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,8,8,11,11,13,13],"right_children":[2,4,-1,-1,6,8,-1,10,12,-1,-1,14,-1,16,-1,-1,-1],"split_conditions":[1.3866357E8,7E0,-7.135269E-3,-5.879663E-3,3.029697E3,1.7329262E7,3.19095E-3,8.47E2,5.0175633E0,-3.5294245E-3,3.557072E-3,6.9E1,1.6566316E-3,2.045738E0,-3.385792E-3,-3.899823E-3,-5.204307E-5],"split_indices":[48,3,0,0,4,45,0,2,54,0,0,8,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.3E1,4E0,4E0,7.9E1,6.3E1,1.6E1,1.9E1,4.4E1,4E0,1.5E1,3.7E1,7E0,2.7E1,1E1,4E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.3860375E-3,-4.46792E-3,8.615266E-3,1.099775E-2,-1.1051276E-1,-3.062494E-2,2.1599693E-2,-8.0041E-3,-5.024926E-4,-3.2655252E-3,8.333703E-4,2.990377E-3,3.322378E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9781733E-1,1.1601921E-1,0E0,2.8020937E-2,6.594029E-2,3.128498E-2,3.2877292E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,8.615266E-3,3.7997437E2,9.1686124E-1,1.5039519E5,9.693049E2,-8.0041E-3,-5.024926E-4,-3.2655252E-3,8.333703E-4,2.990377E-3,3.322378E-4],"split_indices":[52,45,0,52,27,33,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.9E1,4E0,6.1E1,8E0,1.2E1,4.9E1,4E0,4E0,6E0,6E0,1E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-6.192797E-3,7.832392E-3,-1.456706E-1,-1.7304857E-3,2.0539446E-2,-1.2593521E-4,-1.2301277E-2,5.294579E-3,9.689084E-3,6.0695782E-2,-7.5284606E-3,6.346825E-4,4.7458704E-3,-2.6058521E-2,2.1714324E-2,-3.308736E-3,-3.6548026E-4,3.087236E-3,-4.0802493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,13,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[1.7133056E-1,4.993025E-2,1.6127712E-1,0E0,7.051483E-2,0E0,0E0,0E0,5.2271336E-2,3.293893E-2,2.4950016E-2,0E0,0E0,2.450012E-2,2.9088743E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9,10,10,13,13,14,14],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,14,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[1.5414109E8,1.528E3,1.0232127E12,-1.7304857E-3,7.446808E-2,-1.2593521E-4,-1.2301277E-2,5.294579E-3,1.7329262E7,4.189857E8,1E0,6.346825E-4,4.7458704E-3,1.1017415E1,5.81609E-1,-3.308736E-3,-3.6548026E-4,3.087236E-3,-4.0802493E-4],"split_indices":[45,11,31,0,57,0,0,0,45,7,16,0,0,56,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,1.6E1,6.3E1,4E0,3E0,5E0,5.8E1,1.4E1,4.4E1,8E0,6E0,2.7E1,1.7E1,6E0,2.1E1,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.83628E-3,-5.039373E-2,1.7211659E-2,-1.6240288E-2,-5.71947E-3,2.496904E-2,-3.4287497E-3,-1.5405546E-3,3.286436E-3,1.2113629E-2,4.829595E-3,3.5186373E-3,2.8071045E-3,-9.174293E-4,1.3813911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,13,-1,-1],"loss_changes":[8.256978E-2,7.753863E-2,4.0532652E-2,4.1446827E-2,0E0,5.2913994E-2,0E0,0E0,0E0,2.7848965E-2,0E0,0E0,2.8601397E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,12,12],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,14,-1,-1],"split_conditions":[6.6504064E0,1.0789844E6,1.8199778E0,6.817E3,-5.71947E-3,1.1997641E0,-3.4287497E-3,-1.5405546E-3,3.286436E-3,1.7329262E7,4.829595E-3,3.5186373E-3,1.381E3,-9.174293E-4,1.3813911E-3],"split_indices":[58,28,39,2,0,39,0,0,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.7E1,5E1,2E1,7E0,4.7E1,3E0,1.7E1,3E0,4.2E1,5E0,4E0,3.8E1,2.1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-9.158938E-3,-3.5582323E-2,3.7559703E-2,-2.010544E-2,-6.2207812E-3,2.1761567E-3,-1.3214346E-3,-4.500451E-3,-1.081828E-2,3.6963965E-3,-2.3378816E-3,2.4388537E-2,-1.2092583E-3,-3.7122457E-4,2.3701799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,-1,13,-1,-1,-1],"loss_changes":[1.1243827E-1,9.571718E-2,3.168499E-2,3.9976224E-2,0E0,0E0,0E0,0E0,3.0754011E-2,2.5169533E-2,0E0,2.408788E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9,11,11],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,-1,14,-1,-1,-1],"split_conditions":[1.204781E3,9.8788344E5,2.3297022E7,3.235021E2,-6.2207812E-3,2.1761567E-3,-1.3214346E-3,-4.500451E-3,6.237206E5,1.3793921E3,-2.3378816E-3,2.912E3,-1.2092583E-3,-3.7122457E-4,2.3701799E-3],"split_indices":[52,28,32,4,0,0,0,0,28,4,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,5.7E1,3.2E1,5.1E1,6E0,2.7E1,5E0,4E0,4.7E1,3.6E1,1.1E1,2.2E1,1.4E1,1.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.0962158E-2,9.0959285E-5,9.184452E-3,1.080748E-2,-8.360569E-3,-3.0612098E-3,9.848634E-2,-2.9778656E-2,2.138885E-2,8.029899E-3,1.4280527E-3,-3.3120395E-4,-6.106778E-3,2.3738118E-3,-2.2544239E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7501944E-1,1.6141982E-1,0E0,8.9068815E-2,0E0,4.244449E-2,5.184953E-2,7.609519E-2,3.122276E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,9.184452E-3,4.9891987E3,-8.360569E-3,7.2767865E2,7.33E2,1.0789844E6,1.9188015E9,8.029899E-3,1.4280527E-3,-3.3120395E-4,-6.106778E-3,2.3738118E-3,-2.2544239E-4],"split_indices":[1,2,0,4,0,52,0,28,12,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.5E1,3E0,7.2E1,3E0,6.3E1,9E0,3E1,3.3E1,3E0,6E0,2.6E1,4E0,1.4E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.4084175E-2,2.4047477E-2,-8.06955E-3,9.160869E-3,1.3025806E-2,-2.2201473E-2,1.2218094E-3,1.5059721E-2,-3.0826356E-3,4.5753196E-3,-8.0603943E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,-1,5,-1,7,-1,9,-1,-1,-1],"loss_changes":[1.722334E-1,3.3133096E-1,0E0,4.7509026E-2,0E0,5.684841E-2,0E0,5.8127902E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7],"right_children":[2,4,-1,6,-1,8,-1,10,-1,-1,-1],"split_conditions":[1.3866357E8,7.467988E3,-8.06955E-3,3.826E3,1.3025806E-2,1.4E1,1.2218094E-3,1.5414852E5,-3.0826356E-3,4.5753196E-3,-8.0603943E-4],"split_indices":[48,4,0,2,0,3,0,32,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,7.6E1,3E0,2.9E1,4.7E1,1.7E1,1.2E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.8934305E-3,-1.2732357E-2,1.1318758E-2,-5.003322E-4,-9.431507E-3,-4.851714E-2,1.7398095E-2,-4.1011097E-3,1.5908298E-4,-3.52695E-3,2.0294997E-3,1.7986002E-3,-1.4767022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,-1,-1,-1],"loss_changes":[2.9163423E-1,1.8291733E-1,0E0,6.014445E-2,0E0,4.738724E-2,3.2247663E-2,0E0,0E0,4.5268156E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,-1,-1,-1],"split_conditions":[9.55721E3,3.698723E10,1.1318758E-2,3.2716873E0,-9.431507E-3,4.283039E1,1.1773262E3,-4.1011097E-3,1.5908298E-4,1.0197425E1,2.0294997E-3,1.7986002E-3,-1.4767022E-3],"split_indices":[4,5,0,56,0,58,52,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7.1E1,3E0,6.8E1,3E0,1.8E1,5E1,9E0,9E0,3E1,2E1,1.2E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1080338E-2,-5.3662215E-3,1.4537941E-1,1.5275009E-2,-3.3924855E-2,1.3167339E-2,-5.868377E-4,-1.960574E-3,1.1878536E-3,-3.9606597E-3,-5.337057E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6201943E-1,3.9424237E-2,2.1110065E-1,3.1076372E-2,5.358601E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4],"right_children":[2,4,6,8,10,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7780703E3,6.401022E7,9.192915E0,1.018E3,7.553E3,1.3167339E-2,-5.868377E-4,-1.960574E-3,1.1878536E-3,-3.9606597E-3,-5.337057E-5],"split_indices":[4,45,57,11,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.5E1,7E0,3.8E1,2.7E1,3E0,4E0,6E0,3.2E1,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-6.9997166E-3,4.0016603E-3,-1.12870276E-1,-1.5674913E-2,5.9529636E-2,6.331049E-4,-9.07983E-3,5.378683E-3,-2.8464128E-3,6.647169E-3,8.990292E-4,-4.099305E-3,7.325761E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[8.6641096E-2,7.522994E-2,9.203134E-2,5.499731E-2,6.320955E-2,0E0,0E0,4.5664474E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[3.698723E10,1.1879E4,1.2845453E5,6.8989144E7,1.383E4,6.331049E-4,-9.07983E-3,4.8595375E6,-2.8464128E-3,6.647169E-3,8.990292E-4,-4.099305E-3,7.325761E-4],"split_indices":[5,2,33,45,10,0,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.7E1,6E0,5E1,1.7E1,3E0,3E0,3.6E1,1.4E1,4E0,1.3E1,3E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.0791813E-2,-1.8801097E-3,-6.238033E-3,1.1297966E-2,-1.02703124E-1,1.8365833E-3,5.646304E-3,-6.797323E-3,-1.1081329E-3,-1.8478564E-3,1.7427966E-2,3.1305603E-3,2.73192E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[9.72343E-2,1.0346665E-1,0E0,7.857981E-2,3.4755453E-2,4.6406426E-2,0E0,0E0,0E0,0E0,2.9705036E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,10,10],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[1.3866357E8,1.3237324E6,-6.238033E-3,5.639591E3,1.4235585E3,6E0,5.646304E-3,-6.797323E-3,-1.1081329E-3,-1.8478564E-3,1.7329262E7,3.1305603E-3,2.73192E-4],"split_indices":[48,28,0,4,4,8,0,0,0,0,45,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.6E1,4E0,6.8E1,8E0,6.4E1,4E0,4E0,4E0,1.6E1,4.8E1,7E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.8591696E-3,9.815839E-3,-3.6795665E-2,-3.6681313E-2,2.1475587E-2,9.056731E-4,-3.727448E-3,3.821858E-4,-3.2953229E-3,3.7346368E-3,4.598712E-4,-1.3272683E-3,3.4440418E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.421554E-2,3.4942888E-2,3.9957553E-2,2.4634743E-2,3.5072025E-2,3.556776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.533027E-1,7.095772E2,4.25174E0,1.2817016E7,4.07E2,3.1023192E0,-3.727448E-3,3.821858E-4,-3.2953229E-3,3.7346368E-3,4.598712E-4,-1.3272683E-3,3.4440418E-3],"split_indices":[27,4,56,12,0,56,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,6.2E1,2E1,1.2E1,5E1,1.2E1,8E0,6E0,6E0,6E0,4.4E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.418808E-3,1.1304898E-2,-1.4341435E-1,4.081586E-3,3.8006776E-3,-1.2294907E-3,-1.0728865E-2,2.30106E-3,-4.7141784E-3,-1.7767029E-2,1.9549048E-3,-3.3403442E-3,-1.2859463E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[1.7209224E-1,3.942059E-2,9.137453E-2,2.9121613E-2,0E0,0E0,0E0,0E0,3.8331848E-2,4.073698E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.5092398E8,3.4409692E3,1.2341693E10,5.7578983E10,3.8006776E-3,-1.2294907E-3,-1.0728865E-2,2.30106E-3,5.0175633E0,3.068E3,1.9549048E-3,-3.3403442E-3,-1.2859463E-4],"split_indices":[45,52,5,31,0,0,0,0,54,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7E1,7E0,6.5E1,5E0,4E0,3E0,9E0,5.6E1,4.5E1,1.1E1,8E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.9704896E-4,9.198559E-3,-6.9760242E-3,-4.2951675E-3,1.4474942E-1,-3.2552388E-2,1.972492E-2,2.8636443E-5,1.0803786E-2,-2.906538E-4,-9.1110535E-2,3.215161E-3,1.2774585E-4,-6.389038E-3,-1.1694393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1],"loss_changes":[1.1292363E-1,1.4146137E-1,0E0,4.88327E-2,1.086466E-1,4.9151298E-2,3.597805E-2,0E0,0E0,0E0,3.3084884E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2415501E9,2.6949062E3,-6.9760242E-3,8.6646006E5,1.4099E4,1.0112447E6,2.156825E7,2.8636443E-5,1.0803786E-2,-2.906538E-4,1.9598669E6,3.215161E-3,1.2774585E-4,-6.389038E-3,-1.1694393E-3],"split_indices":[7,52,0,47,9,32,45,0,0,0,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.6E1,3E0,7E1,6E0,3.2E1,3.8E1,3E0,3E0,2.3E1,9E0,8E0,3E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.299149E-3,-6.6466117E-3,1.030528E-2,1.0409964E-3,-5.9662475E-3,-2.8077487E-3,1.1958189E-2,-2.7244217E-3,1.8385906E-2,1.0324459E-3,-2.7340848E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[2.3250343E-1,7.4712105E-2,0E0,5.1756743E-2,0E0,0E0,3.0839918E-2,0E0,2.855911E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[8.791079E3,5.673536E2,1.030528E-2,2.1113522E8,-5.9662475E-3,-2.8077487E-3,7E0,-2.7244217E-3,4.2E1,1.0324459E-3,-2.7340848E-3],"split_indices":[4,58,0,7,0,0,3,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,6.9E1,3E0,9E0,6E1,4E0,5.6E1,5.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.937586E-2,9.897601E-3,8.018543E-3,-1.009328E-2,4.4329964E-2,-4.3930426E-2,1.2423929E-2,5.9737567E-2,-2.531505E-3,1.4012291E-4,-3.0678778E-3,4.245226E-3,-1.4152724E-3,4.926671E-3,1.4901694E-3,-1.4873481E-3,1.0849742E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,-1,-1,15,-1,-1,-1,-1],"loss_changes":[1.1925925E-1,5.0695766E-2,0E0,3.6355942E-2,4.6604592E-2,2.5179353E-2,3.499613E-2,2.9121995E-2,0E0,0E0,0E0,0E0,2.4120333E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,-1,-1,16,-1,-1,-1,-1],"split_conditions":[2.1769184E7,1.077646E3,8.018543E-3,2.8586518E8,2.2167318E-1,1.0793079E5,2.526E3,7.33E2,-2.531505E-3,1.4012291E-4,-3.0678778E-3,4.245226E-3,3.3392856E0,4.926671E-3,1.4901694E-3,-1.4873481E-3,1.0849742E-3],"split_indices":[1,52,0,7,38,32,2,0,0,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,4.6E1,2.6E1,1.8E1,2.8E1,2.3E1,3E0,7E0,1.1E1,3E0,2.5E1,6E0,1.7E1,1.1E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[5.880934E-4,9.789349E-3,-7.6539125E-3,-1.275904E-2,4.629207E-2,7.9654885E-4,-5.6536244E-3,8.215862E-2,9.067132E-4,-4.1097035E-3,1.0006172E-2,5.5095456E-3,1.3004519E-3,4.3614373E-2,-1.0981136E-3,8.3502494E-2,6.353207E-4,3.145271E-4,5.1465724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,13,-1,-1,15,-1,17,-1,-1,-1],"loss_changes":[1.3558705E-1,6.479069E-2,0E0,7.9445854E-2,2.5253631E-2,4.112026E-2,0E0,2.5069006E-2,0E0,0E0,5.149105E-2,0E0,0E0,2.3995396E-2,0E0,2.436763E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,10,10,13,13,15,15],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,14,-1,-1,16,-1,18,-1,-1,-1],"split_conditions":[5.139E4,1.077646E3,-7.6539125E-3,1.0789844E6,1.4E1,8.192308E1,-5.6536244E-3,7.33E2,9.067132E-4,-4.1097035E-3,6.1577463E0,5.5095456E-3,1.3004519E-3,3.5175372E5,-1.0981136E-3,2.37E2,6.353207E-4,3.145271E-4,5.1465724E-3],"split_indices":[2,52,0,28,3,52,0,0,0,0,56,0,0,32,0,10,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,4.8E1,2.9E1,4.4E1,4E0,1.1E1,1.8E1,3E0,4.1E1,5E0,6E0,2.1E1,2E1,8E0,1.3E1,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.808404E-3,-4.0421966E-3,8.400543E-3,2.2803247E-3,-5.1991097E-3,-1.4997546E-3,1.8446412E-2,2.4496089E-3,4.447638E-6],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[1.5290117E-1,5.8241256E-2,0E0,4.593393E-2,0E0,0E0,3.7748966E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[7.6312915E3,2.5805172E7,8.400543E-3,2.881172E8,-5.1991097E-3,-1.4997546E-3,4.5503766E5,2.4496089E-3,4.447638E-6],"split_indices":[52,47,0,7,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,7.4E1,3E0,2.2E1,5.2E1,1.6E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.4626985E-2,-6.5052616E-3,-5.2099507E-3,-3.3596627E-2,6.805525E-4,-4.6231814E-2,1.6844646E-3,-1.09878994E-4,-6.898021E-2,-3.9237575E-3,-4.4140316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[7.4087046E-2,4.973244E-2,0E0,3.5942703E-2,0E0,3.1606704E-2,0E0,0E0,2.7107805E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[1.278211E8,8.6646006E5,-5.2099507E-3,2.477407E6,6.805525E-4,1.5326E4,1.6844646E-3,-1.09878994E-4,1.9598669E6,-3.9237575E-3,-4.4140316E-4],"split_indices":[48,47,0,1,0,9,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8E1,5E0,3.6E1,4.4E1,3.1E1,5E0,1.1E1,2E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.0076585E-2,6.9905276E-4,7.84219E-3,1.0549464E-2,-7.270946E-2,-3.8446547E-3,1.6663069E-2,-6.822399E-4,-5.97255E-3,9.212863E-3,4.3931943E-3,-2.595745E-3,2.2281876E-3,-1.6221173E-2,2.1725013E-3,-1.4562735E-3,1.3798906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,-1,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.2663867E-1,5.4999854E-2,0E0,4.2645644E-2,3.197294E-2,0E0,4.009278E-2,0E0,0E0,3.0525176E-2,0E0,3.5747956E-2,0E0,3.4822833E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,9,9,11,11,13,13],"right_children":[2,4,-1,6,8,-1,10,-1,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[2.1769184E7,9.167896E-1,7.84219E-3,3.235021E2,6.33068E5,-3.8446547E-3,1.1997641E0,-6.822399E-4,-5.97255E-3,9.29295E5,4.3931943E-3,5.0175633E0,2.2281876E-3,3.029697E3,2.1725013E-3,-1.4562735E-3,1.3798906E-3],"split_indices":[1,27,0,4,29,0,39,0,0,28,0,54,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.4E1,3E0,6.6E1,8E0,3E0,6.3E1,5E0,3E0,5.9E1,4E0,4.7E1,1.2E1,3.8E1,9E0,2.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[7.1125855E-3,-5.903616E-3,1.3625851E-1,6.023849E-3,-5.7360013E-3,2.0349465E-4,9.993506E-3,-1.0403292E-3,2.3224136E-2,-2.6063612E-3,1.336854E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,-1,-1,-1,-1,9,-1,-1],"loss_changes":[1.2855962E-1,1.07870355E-1,8.857702E-2,3.4498528E-2,0E0,0E0,0E0,0E0,3.0320372E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8],"right_children":[2,4,6,8,-1,-1,-1,-1,10,-1,-1],"split_conditions":[4.0230347E3,3.698723E10,2.2620792E5,3.826E3,-5.7360013E-3,2.0349465E-4,9.993506E-3,-1.0403292E-3,5E0,-2.6063612E-3,1.336854E-3],"split_indices":[52,5,28,2,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,6.4E1,5E0,3E0,3E0,2.3E1,4.1E1,3E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[6.4143366E-3,-1.3931212E-3,7.4674455E-3,-5.4263496E-3,4.5996397E-3,4.8249832E-4,-5.2348804E-2,-4.3446505E-3,1.4801146E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[1.18438624E-1,6.601547E-2,0E0,0E0,3.25617E-2,0E0,4.206098E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[2.1698572E7,5E0,7.4674455E-3,-5.4263496E-3,1.8003757E-1,4.8249832E-4,1.4E1,-4.3446505E-3,1.4801146E-3],"split_indices":[1,3,0,0,38,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,8.6E1,3E0,3E0,8.3E1,7.5E1,8E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.14453E-3,1.5539227E-2,-6.0570575E-3,2.9807705E-3,1.0567889E-2,-3.317125E-3,8.611344E-3,1.3842303E-2,-2.7028853E-3,3.1798948E-2,-4.6114207E-4,-1.2050027E-2,2.02419E-3,-1.8756003E-3,2.3341302E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,-1,-1,7,9,-1,11,-1,13,-1,-1,-1],"loss_changes":[1.1605032E-1,2.2611591E-1,0E0,3.4683798E-2,0E0,0E0,2.7547667E-2,2.9970221E-2,0E0,2.71384E-2,0E0,2.5709478E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,9,9,11,11],"right_children":[2,4,-1,6,-1,-1,8,10,-1,12,-1,14,-1,-1,-1],"split_conditions":[1.278211E8,7.467988E3,-6.0570575E-3,1.09E2,1.0567889E-2,-3.317125E-3,3.698723E10,4.5090426E8,-2.7028853E-3,1.648E3,-4.6114207E-4,1.2293572E6,2.02419E-3,-1.8756003E-3,2.3341302E-3],"split_indices":[48,4,0,10,0,0,5,7,0,11,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,7.7E1,4E0,7.4E1,3E0,4E0,7E1,6.6E1,4E0,3.8E1,2.8E1,1E1,2.8E1,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.0882834E-2,-1.1136126E-4,-1.1858166E-1,7.435513E-4,-2.6054291E-2,7.699214E-4,-9.644061E-3,-2.0954113E-3,6.8199955E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,-1,7,-1,-1,-1,-1],"loss_changes":[8.931257E-2,3.2685246E-2,1.0630635E-1,0E0,2.873138E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4],"right_children":[2,4,6,-1,8,-1,-1,-1,-1],"split_conditions":[1.5414109E8,7.455661E-1,1.6867E4,7.435513E-4,1.419E3,7.699214E-4,-9.644061E-3,-2.0954113E-3,6.8199955E-4],"split_indices":[45,27,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,4.2E1,2.8E1,3E0,3E0,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.1758199E-2,-8.113677E-3,8.1760526E-2,6.226069E-5,-5.188308E-3,1.22552544E-1,-2.0872424E-3,2.5562535E-3,1.1312912E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[1.13340646E-1,6.951435E-2,1.0185341E-1,0E0,0E0,1.0897979E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.3305548E3,1.5092398E8,1.06102504E8,6.226069E-5,-5.188308E-3,3.9509753E3,-2.0872424E-3,2.5562535E-3,1.1312912E-2],"split_indices":[52,45,51,0,0,52,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.3E1,1.7E1,5.9E1,4E0,1.3E1,4E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[8.700694E-3,-1.2264316E-3,7.6019564E-3,8.359954E-3,-5.371242E-3,5.6372937E-2,-3.6212875E-4,8.92916E-4,4.451379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,-1,5,-1,7,-1,-1,-1],"loss_changes":[1.21059716E-1,8.3321005E-2,0E0,5.2995488E-2,0E0,2.661978E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5],"right_children":[2,4,-1,6,-1,8,-1,-1,-1],"split_conditions":[2.1747866E7,1.5277152E8,7.6019564E-3,1.7329262E7,-5.371242E-3,2.9719496E0,-3.6212875E-4,8.92916E-4,4.451379E-3],"split_indices":[1,45,0,45,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.8E1,3E0,6.4E1,4E0,1.6E1,4.8E1,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[5.437385E-3,-1.752151E-3,6.630797E-3,-1.9731363E-2,2.8080674E-2,-7.1678157E-3,-4.330542E-3,4.5535218E-2,-1.7431164E-3,1.08149936E-4,-3.1581158E-3,9.993475E-4,4.377305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":99,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,-1,-1,-1],"loss_changes":[9.3857996E-2,4.5609206E-2,0E0,5.3566657E-2,4.0307872E-2,3.0577855E-2,0E0,3.035758E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,-1,-1,-1],"split_conditions":[2.1769184E7,1.204781E3,6.630797E-3,1.3237324E6,1.06102504E8,1E0,-4.330542E-3,4.146482E3,-1.7431164E-3,1.08149936E-4,-3.1581158E-3,9.993475E-4,4.377305E-3],"split_indices":[1,52,0,28,51,105,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,8.3E1,3E0,5.2E1,3.1E1,4.6E1,6E0,2.5E1,6E0,4.1E1,5E0,1.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics index 81bf8eac2..02fd5b915 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics @@ -1,109 +1,109 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,sw_bytesWrittenRatio,0.0832446,0.5919399675594721,1.476025676540793,3.811915686539279e-07,0.004988403730711104,0.15715677052090143,0.6968701139378873,19.10476372587304 -1,input_bytesRead_mean,0.06974004,48660306.00109597,49428056.486782335,466150.2392857143,9457240.909980843,31206626.71954965,76555115.52221651,240185411.32144082 -2,executorCPUTime_mean,0.06779716,1084.78343193192,2189.7869800693916,8.678700361010831,171.72586021505379,491.5723086712559,1134.4773616891898,19830.198536139065 -3,sr_localBytesReadRatio,0.06722565,0.11475462581453616,0.25076456148616505,4.950703452775733e-08,0.0013396561493017212,0.03231300375934251,0.11114098070492967,2.3968573169615706 -4,duration_mean,0.050912667,1838.369521293885,2822.267657577495,48.0,445.2089864158829,956.2881858350818,2117.1486501006557,23602.61970074813 -5,cache_hit_ratio,0.048020836,0.5797031024761284,0.2958995453171238,2.3751764226164482e-05,0.34724318043450464,0.6077699084792183,0.8571339338875497,1.0 -6,sr_remoteBytesReadRatio,0.048011165,0.5431694364318369,1.3308894015763957,3.3168453412617064e-07,0.004778540082677,0.14424033071079345,0.597866155985421,16.78961797640585 -7,sr_totalBytesReadRatio,0.035243288,0.657924062246373,1.5241815199065611,3.811915686539279e-07,0.006417516514179832,0.19052090935635582,0.799098277696303,19.18647529336742 -8,scan_bw,0.031204652,682180.7475050243,709768.054847744,18541.21305321391,250700.44646224368,415709.30636541,843321.6773468265,5003420.826327434 -9,sw_writeTime_mean,0.024540022,103.69509381282296,238.56070095484813,0.0,0.657938732288993,13.45055733972235,96.60870167372909,1805.0101408450705 -10,duration_max,0.023943964,7443.127272727273,13363.593275228666,156.0,1387.75,3476.0,7819.75,121568.0 -11,data_size,0.015578491,668568812798.4546,1985841754832.3271,57436094.0,27908715879.25,115710188148.5,545671877091.0,22044104226233.0 -12,input_recordsRead_sum,0.015004172,8228875634.568182,19243603112.479984,600000.0,413535743.5,1810167936.0,7537489709.0,230400506548.0 -13,shuffle_read_bw,0.013040873,21097884.55916633,142877158.37645754,0.0,173985.0551887179,593988.9257987014,2812606.8795373663,1459791338.8333333 -14,duration_sum,0.01285409,2471074.709090909,4840603.443290839,240.0,134260.75,568925.0,2163656.75,31586217.0 -15,executorDeserializeTime_mean,0.0117216855,6.522922206172232,15.239475132670595,1.4,2.7173863626133223,3.2269249889527174,3.976825559667314,136.86302681992336 -16,sr_remoteBlocksFetched_sum,0.010922961,73710.92727272728,139978.87892030226,1.0,1302.75,17643.0,70517.25,940427.0 -17,resultSize_max,0.010806285,464635.34545454546,2610280.2584496085,9745.0,13582.0,16288.5,21372.5,33908356.0 -18,sr_fetchWaitTime_mean,0.010560745,42.158621909106756,151.8981045105995,0.0,0.33875324808973734,2.0408804539229113,9.230470613341367,1115.2329896907218 -19,sw_recordsWritten_sum,0.008042099,1993995676.168182,5072609516.6370945,200.0,1520171.75,126090791.5,1522106638.75,40948495624.0 -20,jvmGCTime_mean,0.007712294,12.723160383736214,14.422087022171816,0.0,3.3102295226222775,8.111694964818351,16.933459050062602,84.78 -21,peakExecutionMemory_max,0.007490599,480456154.1818182,474651559.48118466,159385856.0,220580735.75,346536456.5,531331736.0,4222092096.0 -22,sr_localBlocksFetched_sum,0.0070248935,10735.163636363637,20529.949658652342,1.0,210.0,2536.0,10083.0,135165.0 -23,sw_bytesWritten_mean,0.0061201327,16381353.803431287,34748537.32729478,49.30337459725284,90522.77177520556,2317902.8085535243,12524134.635000767,243100435.13633803 -24,duration_min,0.00611866,16.98181818181818,22.24581714032469,0.0,11.0,12.0,16.0,268.0 -25,shuffle_write_bw,0.0055911485,163094.76249962486,120588.59781638657,0.0,79856.67746661245,156255.07795132295,206153.42535075612,795840.9797297297 -26,executorDeserializeCPUTime_mean,0.005300696,3.2044441026558315,2.5376539911086757,1.4,2.2969823808927665,2.631830835117773,3.1553397661964473,23.9288256227758 -27,sqlOp_Sort,0.0047614016,0.2772727272727273,0.4486731510945088,0.0,0.0,0.0,1.0,1.0 -28,numTasks_sum,0.004689348,957.8454545454546,1285.2573516712155,5.0,276.75,561.5,1139.0,11794.0 -29,sr_remoteBytesRead_mean,0.0046631177,14571247.101848954,31581916.287754808,51.05526590198123,270542.5920445972,1917402.5879213854,11389068.207380045,212694915.64563382 -30,sr_localBytesRead_mean,0.0043532755,2916849.994645489,5960186.011236432,7.631067961165049,44445.41409518829,469171.9910395135,2269699.288765448,34055682.08306709 -31,resultSerializationTime_sum,0.003945646,75.90454545454546,130.42653936613254,0.0,3.0,25.5,99.0,1069.0 -32,sqlOp_AQEShuffleRead,0.0034873844,0.9090909090909091,0.28813538628606516,0.0,1.0,1.0,1.0,1.0 -33,scan_time,0.0031200983,1326606.9181818182,4390988.808916784,156.0,51219.75,227012.5,950299.75,58655184.0 -34,sqlOp_Window,0.001570162,0.11818181818181818,0.32355948919311184,0.0,0.0,0.0,0.0,1.0 -35,sqlOp_Filter,0.0008919083,0.4727272727272727,0.5003941991110044,0.0,0.0,0.0,1.0,1.0 -36,sqlOp_Subquery,0.00076304446,0.2818181818181818,0.45091116645433477,0.0,0.0,0.0,1.0,1.0 -37,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -38,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -39,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -40,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -41,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -42,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -73,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,diskBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_SortMergeJoin,0.0,0.5590909090909091,0.49762824776768727,0.0,0.0,1.0,1.0,1.0 -78,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,sqlOp_TakeOrderedAndProject,0.0,0.6181818181818182,0.4869402780332598,0.0,0.0,1.0,1.0,1.0 -81,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +0,sr_localBytesReadRatio,0.085029766,0.11475462581453616,0.25076456148616505,4.950703452775733e-08,0.0013396561493017212,0.03231300375934251,0.11114098070492967,2.3968573169615706 +1,sw_bytesWrittenRatio,0.07347684,0.5919399675594721,1.476025676540793,3.811915686539279e-07,0.004988403730711104,0.15715677052090143,0.6968701139378873,19.10476372587304 +2,input_bytesRead_mean,0.072008565,48660306.00109597,49428056.486782335,466150.2392857143,9457240.909980843,31206626.71954965,76555115.52221651,240185411.32144082 +3,executorCPUTime_mean,0.06485506,1084.78343193192,2189.7869800693916,8.678700361010831,171.72586021505379,491.5723086712559,1134.4773616891898,19830.198536139065 +4,duration_mean,0.06299606,1838.369521293885,2822.267657577495,48.0,445.2089864158829,956.2881858350818,2117.1486501006557,23602.61970074813 +5,cache_hit_ratio,0.048470605,0.5797031024761284,0.2958995453171238,2.3751764226164482e-05,0.34724318043450464,0.6077699084792183,0.8571339338875497,1.0 +6,scan_bw,0.03805825,682180.7475050243,709768.054847744,18541.21305321391,250700.44646224368,415709.30636541,843321.6773468265,5003420.826327434 +7,sr_remoteBytesReadRatio,0.035772987,0.5431694364318369,1.3308894015763957,3.3168453412617064e-07,0.004778540082677,0.14424033071079345,0.597866155985421,16.78961797640585 +8,sw_writeTime_mean,0.034270227,103.69509381282296,238.56070095484813,0.0,0.657938732288993,13.45055733972235,96.60870167372909,1805.0101408450705 +9,duration_max,0.028194813,7443.127272727273,13363.593275228666,156.0,1387.75,3476.0,7819.75,121568.0 +10,data_size,0.02125767,668568812798.4546,1985841754832.3271,57436094.0,27908715879.25,115710188148.5,545671877091.0,22044104226233.0 +11,sr_totalBytesReadRatio,0.016146142,0.657924062246373,1.5241815199065611,3.811915686539279e-07,0.006417516514179832,0.19052090935635582,0.799098277696303,19.18647529336742 +12,input_recordsRead_sum,0.014518568,8228875634.568182,19243603112.479984,600000.0,413535743.5,1810167936.0,7537489709.0,230400506548.0 +13,sr_remoteBlocksFetched_sum,0.014469172,73710.92727272728,139978.87892030226,1.0,1302.75,17643.0,70517.25,940427.0 +14,duration_sum,0.013677995,2471074.709090909,4840603.443290839,240.0,134260.75,568925.0,2163656.75,31586217.0 +15,resultSize_max,0.013444052,464635.34545454546,2610280.2584496085,9745.0,13582.0,16288.5,21372.5,33908356.0 +16,shuffle_read_bw,0.01247076,21097884.55916633,142877158.37645754,0.0,173985.0551887179,593988.9257987014,2812606.8795373663,1459791338.8333333 +17,peakExecutionMemory_max,0.009648207,480456154.1818182,474651559.48118466,159385856.0,220580735.75,346536456.5,531331736.0,4222092096.0 +18,jvmGCTime_mean,0.009505965,12.723160383736214,14.422087022171816,0.0,3.3102295226222775,8.111694964818351,16.933459050062602,84.78 +19,sr_localBytesRead_mean,0.009245939,2916849.994645489,5960186.011236432,7.631067961165049,44445.41409518829,469171.9910395135,2269699.288765448,34055682.08306709 +20,executorDeserializeTime_mean,0.009006469,6.522922206172232,15.239475132670595,1.4,2.7173863626133223,3.2269249889527174,3.976825559667314,136.86302681992336 +21,sr_localBlocksFetched_sum,0.008847067,10735.163636363637,20529.949658652342,1.0,210.0,2536.0,10083.0,135165.0 +22,sr_fetchWaitTime_mean,0.008237192,42.158621909106756,151.8981045105995,0.0,0.33875324808973734,2.0408804539229113,9.230470613341367,1115.2329896907218 +23,sw_recordsWritten_sum,0.008037027,1993995676.168182,5072609516.6370945,200.0,1520171.75,126090791.5,1522106638.75,40948495624.0 +24,duration_min,0.0057118977,16.98181818181818,22.24581714032469,0.0,11.0,12.0,16.0,268.0 +25,executorDeserializeCPUTime_mean,0.005369326,3.2044441026558315,2.5376539911086757,1.4,2.2969823808927665,2.631830835117773,3.1553397661964473,23.9288256227758 +26,numTasks_sum,0.005066579,957.8454545454546,1285.2573516712155,5.0,276.75,561.5,1139.0,11794.0 +27,resultSerializationTime_sum,0.004663864,75.90454545454546,130.42653936613254,0.0,3.0,25.5,99.0,1069.0 +28,sw_bytesWritten_mean,0.0044036005,16381353.803431287,34748537.32729478,49.30337459725284,90522.77177520556,2317902.8085535243,12524134.635000767,243100435.13633803 +29,sr_remoteBytesRead_mean,0.0037133617,14571247.101848954,31581916.287754808,51.05526590198123,270542.5920445972,1917402.5879213854,11389068.207380045,212694915.64563382 +30,shuffle_write_bw,0.0036576367,163094.76249962486,120588.59781638657,0.0,79856.67746661245,156255.07795132295,206153.42535075612,795840.9797297297 +31,sqlOp_Sort,0.0027435673,0.2772727272727273,0.4486731510945088,0.0,0.0,0.0,1.0,1.0 +32,sqlOp_Window,0.0023267556,0.11818181818181818,0.32355948919311184,0.0,0.0,0.0,0.0,1.0 +33,scan_time,0.0021827868,1326606.9181818182,4390988.808916784,156.0,51219.75,227012.5,950299.75,58655184.0 +34,sqlOp_TakeOrderedAndProject,0.00090243947,0.6181818181818182,0.4869402780332598,0.0,0.0,1.0,1.0,1.0 +35,sqlOp_Subquery,0.00058086135,0.2818181818181818,0.45091116645433477,0.0,0.0,0.0,1.0,1.0 +36,sr_totalBytesRead_mean,0.00057853124,17488097.096494444,36315663.17124838,58.75728155339806,345292.2404184723,2879392.1611024104,15671497.58876525,243101856.34140846 +37,executorRunTime_mean,0.00026640674,1820.9901176973162,2822.0779471701867,41.0,433.6194879832811,944.1120382732532,2108.383181217503,23591.60349127182 +38,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +39,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +40,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +41,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +42,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +43,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +44,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +46,sqlOp_HashAggregate,0.0,0.990909090909091,0.09512832433347063,0.0,1.0,1.0,1.0,1.0 +47,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_Filter,0.0,0.4727272727272727,0.5003941991110044,0.0,0.0,0.0,1.0,1.0 +51,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_SortMergeJoin,0.0,0.5590909090909091,0.49762824776768727,0.0,0.0,1.0,1.0,1.0 +60,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_BroadcastHashJoin,0.0,0.9045454545454545,0.2945118594681246,0.0,1.0,1.0,1.0,1.0 +72,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +74,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +75,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,diskBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +78,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +79,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +81,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 82,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -84,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -86,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -87,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -88,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 -89,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -90,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -91,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,sr_totalBytesRead_mean,0.0,17488097.096494444,36315663.17124838,58.75728155339806,345292.2404184723,2879392.1611024104,15671497.58876525,243101856.34140846 -93,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -94,executorRunTime_mean,0.0,1820.9901176973162,2822.0779471701867,41.0,433.6194879832811,944.1120382732532,2108.383181217503,23591.60349127182 -95,sqlOp_BroadcastHashJoin,0.0,0.9045454545454545,0.2945118594681246,0.0,1.0,1.0,1.0,1.0 -96,sqlOp_BroadcastNestedLoopJoin,0.0,0.03636363636363636,0.18762017042954973,0.0,0.0,0.0,0.0,1.0 -97,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -98,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -99,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -101,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,diskBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_HashAggregate,0.0,0.990909090909091,0.09512832433347063,0.0,1.0,1.0,1.0,1.0 -105,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +84,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,diskBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +88,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +91,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_BroadcastNestedLoopJoin,0.0,0.03636363636363636,0.18762017042954973,0.0,0.0,0.0,0.0,1.0 +95,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +96,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_AQEShuffleRead,0.0,0.9090909090909091,0.28813538628606516,0.0,1.0,1.0,1.0,1.0 +99,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +101,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +106,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg index 3aa9ab781..4eaa3245f 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0490389541","gamma":"0.0485602282","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0490389541","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.0485602282","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.800133407"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0499760844","gamma":"0.037331894","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0499760844","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"1","min_split_loss":"0.037331894","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.735622287"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json index 68c277cb3..4077c5635 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_dataproc","platform_databricks-aws","platform_databricks-azure","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.8890809E-3,-2.846971E-1,5.4441637E-1,-6.09458E-1,-1.8608256E-1,6.390984E-1,-5.82664E-1,-6.5375525E-1,-3.0437765E-1,-2.2892065E-1,2.4880955E-1,3.232467E-1,9.0748E-1,-1.0247247E0,-5.7365548E-2,-3.2571018E-2,-1.11420695E-2,-1.9031389E-2,-2.9847159E-3,-3.3582267E-1,-1.3285375E-1,3.4043756E-1,-2.5633355E-3,5.003245E-2,2.7680942E-1,7.7094984E-1,1.3052373E0,-5.80325E-2,-2.05325E-2,-1.0538936E-2,5.6159287E-3,-3.7601125E-1,-7.4982226E-2,-1.8952176E-1,1.2046365E-1,1.3471154E-3,3.8033518E-1,-6.7115836E-2,3.245809E-1,8.061074E-1,-9.277409E-3,5.3701156E-1,7.229797E-2,-2.518035E-1,-4.2059547E-1,4.2088074E-3,-1.2233638E-1,-2.7805755E-1,-6.397237E-2,2.4000727E-1,-4.698942E-2,1.17186E-2,2.4900168E-2,-2.374306E-1,1.0059619E-2,-1.7408561E-4,3.4850952E-1,5.508299E-1,8.7895924E-1,3.9425246E-2,4.4655623E-3,-5.2069398E-3,-1.523279E-2,-1.0222437E-2,-2.1723798E-2,-7.602005E-3,1.3430231E-3,-6.774862E-3,-1.5980454E-2,-6.096835E-3,5.4890197E-3,3.6091541E-3,1.3856955E-2,-7.2271517E-3,8.379269E-4,-3.2020707E-3,-1.5063469E-2,1.1561514E-2,1.9919658E-2,5.98026E-3,3.140561E-2,1.737663E-2,4.389572E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,35,-1,-1,37,39,41,-1,-1,-1,-1,43,45,47,49,-1,51,53,55,57,-1,59,-1,61,63,-1,65,67,69,71,73,-1,-1,75,-1,-1,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.2924286E1,9.846724E0,1.7553383E1,7.740364E-1,4.528146E0,1.2337204E1,2.9599957E0,1.2495804E-1,2.006079E-1,2.2044582E0,6.24719E-1,1.9832978E0,3.3381958E0,1.7762852E-1,2.1416774E-1,0E0,0E0,0E0,0E0,1.0515356E0,1.7023644E0,2.0182514E-1,0E0,0E0,1.1225824E0,2.245369E0,1.6759262E0,0E0,0E0,0E0,0E0,3.869486E-1,1.24723546E-1,1.0493109E0,4.516726E-1,0E0,1.4887953E-1,4.6602088E-1,4.6448803E-1,6.316948E-1,0E0,5.220138E-1,0E0,1.897341E-1,2.1171474E-1,0E0,6.296696E-2,3.2959557E-1,4.4919348E-1,7.1281254E-2,6.6842124E-2,0E0,0E0,5.1973492E-2,0E0,0E0,2.619834E-1,5.3668785E-1,6.4323425E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,24,24,25,25,26,26,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,53,53,56,56,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,36,-1,-1,38,40,42,-1,-1,-1,-1,44,46,48,50,-1,52,54,56,58,-1,60,-1,62,64,-1,66,68,70,72,74,-1,-1,76,-1,-1,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0429407E3,3.5002026E8,1.0823923E9,3.9837E4,1.700398E4,2.9778808E7,3.49399E5,7.377246E2,1.87E2,8.504733E7,8.4987135E0,3.1E1,1.6130411E5,-3.2571018E-2,-1.11420695E-2,-1.9031389E-2,-2.9847159E-3,7.164074E8,2.103E3,2.117203E3,-2.5633355E-3,5.003245E-2,3.0811954E-2,1.4656219E8,6.069097E4,-5.80325E-2,-2.05325E-2,-1.0538936E-2,5.6159287E-3,7.37365E6,4.45E3,5.0363316E5,1.6777562E7,1.3471154E-3,1E0,1.9128645E9,2.0772269E8,6.050605E9,-9.277409E-3,1.207E3,7.229797E-2,1.6459E4,5.1547173E1,4.2088074E-3,6.46E2,1.423795E7,3.23643E6,2.473E3,3.7730695E-7,1.17186E-2,2.4900168E-2,5.1E1,1.0059619E-2,-1.7408561E-4,7.3E1,1.4040858E-2,1.1041459E-1,3.9425246E-2,4.4655623E-3,-5.2069398E-3,-1.523279E-2,-1.0222437E-2,-2.1723798E-2,-7.602005E-3,1.3430231E-3,-6.774862E-3,-1.5980454E-2,-6.096835E-3,5.4890197E-3,3.6091541E-3,1.3856955E-2,-7.2271517E-3,8.379269E-4,-3.2020707E-3,-1.5063469E-2,1.1561514E-2,1.9919658E-2,5.98026E-3,3.140561E-2,1.737663E-2,4.389572E-2],"split_indices":[2,1,66,7,7,2,41,12,1,60,3,7,61,3,36,0,0,0,0,7,0,4,0,0,46,1,36,0,0,0,0,53,2,55,53,0,22,7,5,12,0,0,0,9,56,0,8,53,1,0,46,0,0,3,0,0,8,65,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.7E2,3.1E2,1.6E2,7.1E1,2.39E2,1.48E2,1.2E1,6.1E1,1E1,2.18E2,2.1E1,6.9E1,7.9E1,6E0,6E0,5.9E1,2E0,7E0,3E0,1.02E2,1.16E2,1.6E1,5E0,3E0,6.6E1,6.1E1,1.8E1,4E0,2E0,3E0,3E0,8.8E1,1.4E1,9.5E1,2.1E1,2E0,1.4E1,8E0,5.8E1,5.9E1,2E0,4E0,1.4E1,2.5E1,6.3E1,3E0,1.1E1,5.5E1,4E1,1.2E1,9E0,8E0,6E0,5E0,3E0,4E0,5.4E1,1.5E1,4.4E1,2E0,2E0,8E0,1.7E1,7E0,5.6E1,9E0,2E0,1.5E1,4E1,3E1,1E1,3E0,9E0,3E0,6E0,2E0,3E0,2E1,3.4E1,3E0,1.2E1,2E0,4.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[5.8160843E-3,-2.9693097E-1,5.342773E-1,-5.8964497E-1,-1.9762173E-1,4.0375826E-1,1.0967594E0,-6.12045E-1,-1.6395445E-3,-2.3075576E-1,3.2297143E-1,4.428639E-1,-5.1790174E-2,-1.3927489E-3,1.2141347E0,-3.1267602E-2,-1.5920445E-2,-2.7651832E-1,-5.274713E-2,-4.0823002E-3,3.9761153E-1,2.897549E-1,6.646191E-1,6.0210633E-1,6.774744E-2,-2.935697E-1,-3.3597837E-3,-7.581412E-2,1.4489154E-2,2.6231604E-2,1.0941689E-2,1.0935005E-1,3.6009434E-1,2.0855367E-2,7.137383E-1,3.5702776E-2,7.3322267E-3,-3.156745E-1,-1.4056353E-1,-5.077058E-3,9.701025E-3,-1.0289436E-1,7.3840185E-3,-2.5766399E-2,2.578734E-1,4.0023494E-1,1.1936791E-1,-1.4574049E-2,1.6278591E-2,2.610785E-2,3.8574565E-2,-1.73943E-2,-9.058018E-3,-8.71975E-3,-1.7331366E-3,-5.9431302E-3,4.291313E-3,-9.438474E-3,6.0474393E-3,1.6281392E-2,4.299792E-3,2.014778E-2,4.580334E-3,-4.2644884E-3,1.0799247E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,23,-1,-1,25,27,-1,29,31,33,35,-1,37,39,41,-1,-1,-1,43,45,47,49,-1,-1,51,53,-1,-1,55,-1,57,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.5993935E1,8.662373E0,1.2257908E1,9.2643166E-1,3.9901295E0,8.57432E0,4.2605934E0,3.2929993E-1,0E0,1.7286863E0,4.5758116E-1,4.5268497E0,0E0,0E0,2.138771E0,0E0,0E0,7.908287E-1,3.8384652E-1,0E0,1.6814148E-1,1.0265479E0,1.7558994E0,3.4714484E-1,0E0,4.9677372E-1,2.4216087E-1,2.7911162E-1,0E0,0E0,0E0,5.083431E-1,5.3199863E-1,5.933738E-1,2.6841164E-1,0E0,0E0,6.4496994E-1,7.994285E-2,0E0,0E0,1.4647827E-1,0E0,3.7167606E-1,1.22297525E-1,1.2777519E-1,2.2723298E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,14,14,17,17,18,18,20,20,21,21,22,22,23,23,25,25,26,26,27,27,31,31,32,32,33,33,34,34,37,37,38,38,41,41,43,43,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,24,-1,-1,26,28,-1,30,32,34,36,-1,38,40,42,-1,-1,-1,44,46,48,50,-1,-1,52,54,-1,-1,56,-1,58,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.6512886E3,1.2852292E7,1.4096699E9,4.673502E7,5.0314346E1,2.9778808E7,-1.6395445E-3,9.7467445E4,2.0532622E3,4.1331047E-1,-5.1790174E-2,-1.3927489E-3,6.069097E4,-3.1267602E-2,-1.5920445E-2,2.9063344E5,1.7670108E5,-4.0823002E-3,6.088569E6,1.2587E4,2.1853803E4,2.36316E5,6.774744E-2,7.164074E8,4.037E3,1.5166181E7,1.4489154E-2,2.6231604E-2,1.0941689E-2,2.58E2,2.5916522E7,3.11E3,1E0,3.5702776E-2,7.3322267E-3,1.1496869E3,1.7933134E5,-5.077058E-3,9.701025E-3,4.79E2,7.3840185E-3,8.931424E8,1.114253E6,6.2185767E10,2.1324906E4,-1.4574049E-2,1.6278591E-2,2.610785E-2,3.8574565E-2,-1.73943E-2,-9.058018E-3,-8.71975E-3,-1.7331366E-3,-5.9431302E-3,4.291313E-3,-9.438474E-3,6.0474393E-3,1.6281392E-2,4.299792E-3,2.014778E-2,4.580334E-3,-4.2644884E-3,1.0799247E-2],"split_indices":[2,1,60,9,7,56,64,12,0,41,4,43,0,0,36,0,0,36,36,0,53,2,41,2,0,7,10,58,0,0,0,8,53,0,102,0,0,60,55,0,0,8,0,12,37,5,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.73E2,3.01E2,1.72E2,7.5E1,2.26E2,1.41E2,3.1E1,7.2E1,3E0,2.13E2,1.3E1,1.38E2,3E0,3E0,2.8E1,6.5E1,7E0,1.69E2,4.4E1,2E0,1.1E1,8.3E1,5.5E1,7E0,2.1E1,1.59E2,1E1,4.2E1,2E0,5E0,6E0,2.4E1,5.9E1,4E0,5.1E1,5E0,2E0,1.38E2,2.1E1,7E0,3E0,3.8E1,4E0,1.3E1,1.1E1,5E1,9E0,2E0,2E0,1.7E1,3.4E1,1.05E2,3.3E1,1.5E1,6E0,3.5E1,3E0,6E0,7E0,7E0,4E0,4.8E1,2E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[6.444063E-3,-2.6806113E-1,4.928103E-1,-5.522722E-1,-1.6362543E-1,5.551124E-1,-5.6576526E-1,-5.718619E-1,-1.5792432E-3,-2.0002174E-1,2.8917804E-1,4.4039372E-1,1.1123952E0,8.0672964E-2,-4.9537655E-2,-2.9302206E-2,-1.6554179E-2,-2.538923E-1,-2.1135155E-2,-1.2426415E-3,3.5838473E-1,2.3019122E-1,6.240433E-1,-8.928575E-3,1.2232354E0,-9.57437E-3,1.61679E-2,-1.0405717E-1,-3.085376E-1,-5.473901E-2,1.4001925E-2,2.3079893E-2,8.772606E-3,2.9425126E-1,-2.4188314E-2,6.5021044E-1,1.1172827E-2,4.72893E-1,1.343639E0,-1.381235E-1,1.3296276E-2,-3.5333362E-1,-1.6834453E-1,-7.052085E-2,9.303363E-3,3.486201E-1,1.2177729E-1,-1.9778085E-1,8.712194E-3,5.207182E-1,7.3533034E-1,8.597366E-3,3.005293E-2,7.620855E-2,4.692011E-2,-9.0628965E-3,-1.1451325E-3,-1.8175509E-2,-8.976905E-3,-1.4943231E-2,3.0215185E-3,3.364355E-4,-5.5031613E-3,1.1495777E-2,2.0900846E-2,1.4478381E-3,1.1762515E-2,-1.3479172E-2,1.094436E-3,2.684302E-2,4.8199245E-3,1.7501399E-2,3.7150513E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,-1,31,33,35,-1,37,-1,-1,39,41,43,-1,-1,-1,45,47,49,-1,51,53,55,-1,57,59,61,-1,63,65,67,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.328236E1,8.903212E0,1.1591774E1,8.068161E-1,3.7329645E0,9.866894E0,2.9542928E0,2.6734734E-1,0E0,1.9894352E0,3.7911618E-1,5.1064396E0,4.0966415E0,4.079119E-1,0E0,0E0,0E0,1.2697515E0,5.205753E-1,0E0,1.979047E-1,1.066329E0,5.9832764E-1,0E0,1.6232033E0,0E0,0E0,5.99836E-1,6.6901016E-1,1.8698493E-1,0E0,0E0,0E0,4.4516087E-1,5.256852E-1,3.5559845E-1,0E0,1.0078025E-1,2.2236252E-1,2.1537548E-1,0E0,1.8844318E-1,9.596105E-1,1.3952039E-1,0E0,2.4564934E-1,1.4542799E-1,1.4186612E-1,0E0,2.841015E-1,4.947853E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,20,20,21,21,22,22,24,24,27,27,28,28,29,29,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,-1,32,34,36,-1,38,-1,-1,40,42,44,-1,-1,-1,46,48,50,-1,52,54,56,-1,58,60,62,-1,64,66,68,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.508E3,7.41718E5,1.0429407E3,1.2852292E7,1.4777102E9,7.7101436E3,2.909019E3,2.9778808E7,-1.5792432E-3,9.7467445E4,2.0532622E3,2.5096E4,5.0314346E1,2.5212732E4,-4.9537655E-2,-2.9302206E-2,-1.6554179E-2,7.556039E6,1.4998456E5,-1.2426415E-3,6.1614815E6,2.4722598E7,9.677921E7,-8.928575E-3,6.069097E4,-9.57437E-3,1.61679E-2,3.7069206E5,1.1496869E3,2.1695537E3,1.4001925E-2,2.3079893E-2,8.772606E-3,2.0136225E5,1.8722918E6,5.5835126E9,1.1172827E-2,2.043E3,3.3616E4,2.175722E6,1.3296276E-2,1.2677199E7,2.629627E6,1.4040858E-2,9.303363E-3,1.3428E4,6.340124E6,2.58E2,8.712194E-3,1.9703683E9,1.1041459E-1,8.597366E-3,3.005293E-2,7.620855E-2,4.692011E-2,-9.0628965E-3,-1.1451325E-3,-1.8175509E-2,-8.976905E-3,-1.4943231E-2,3.0215185E-3,3.364355E-4,-5.5031613E-3,1.1495777E-2,2.0900846E-2,1.4478381E-3,1.1762515E-2,-1.3479172E-2,1.094436E-3,2.684302E-2,4.8199245E-3,1.7501399E-2,3.7150513E-2],"split_indices":[2,1,66,9,7,60,40,12,0,41,4,2,64,4,0,0,0,53,36,0,53,53,1,0,36,0,0,36,60,60,0,0,0,41,55,12,0,0,10,1,0,9,1,65,0,2,1,8,0,40,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.72E2,3.02E2,1.7E2,8E1,2.22E2,1.61E2,9E0,7.7E1,3E0,2.06E2,1.6E1,1.35E2,2.6E1,4E0,5E0,6.8E1,9E0,1.58E2,4.8E1,3E0,1.3E1,6.4E1,7.1E1,2E0,2.4E1,2E0,2E0,4.3E1,1.15E2,4.4E1,4E0,7E0,6E0,5.1E1,1.3E1,6.6E1,5E0,4E0,2E1,4E1,3E0,8.6E1,2.9E1,4.2E1,2E0,3.8E1,1.3E1,7E0,6E0,2.9E1,3.7E1,2E0,2E0,1.1E1,9E0,2.8E1,1.2E1,7.7E1,9E0,1.8E1,1.1E1,1.5E1,2.7E1,1.7E1,2.1E1,8E0,5E0,5E0,2E0,2.7E1,2E0,3E0,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[9.31242E-3,-2.604165E-1,5.24193E-1,-5.3795576E-1,-1.6523193E-1,3.819555E-1,9.895143E-1,-5.5730367E-1,-1.5211606E-3,-1.9821197E-1,2.0086315E-1,4.4628996E-1,-1.7378865E-1,7.300265E-1,6.5649785E-2,-2.8410532E-2,-1.4455157E-2,-2.5269482E-1,-2.047418E-2,3.4379804E-1,-1.2034853E-2,2.9758614E-1,6.551042E-1,-3.365566E-2,2.8379725E-2,8.2309777E-1,-9.603719E-3,-2.644772E-1,1.0558134E-1,-6.752246E-2,1.2099462E-2,6.881782E-3,2.3003325E-2,8.076007E-3,-6.2692696E-3,3.4939465E-1,1.3138458E-1,5.026085E-1,3.6752194E-2,2.0510098E-2,-1.0472732E-1,1.5154765E-2,8.8800055E-1,-1.6339423E-1,-2.954513E-1,1.0040542E-2,2.3599163E-4,-1.1504222E-1,3.518655E-2,3.828578E-1,2.0594603E-1,-1.0528918E-2,2.1491621E-1,5.405029E-1,4.662348E-3,-1.1429119E-2,5.5883126E-3,4.556864E-2,1.7572934E-2,-9.527171E-3,-1.737532E-3,-1.7630909E-2,-1.1265992E-2,1.723069E-4,-7.671253E-3,-2.630114E-3,8.697303E-3,6.193252E-3,1.9605257E-2,-2.5251193E-3,1.2876789E-2,1.0168216E-3,1.39095355E-2,9.040107E-3,2.80287E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,-1,39,41,-1,43,45,47,-1,-1,-1,-1,-1,49,51,53,-1,-1,55,-1,57,59,61,-1,-1,63,65,67,69,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.665916E1,8.2326145E0,1.0542763E1,7.673664E-1,2.8892374E0,4.654007E0,2.4661217E0,2.7482033E-1,0E0,2.096181E0,6.127486E-1,3.3759708E0,1.4700804E0,2.229309E0,0E0,0E0,0E0,7.2866726E-1,6.7038167E-1,2.4006486E-1,2.0524423E-1,5.482297E-1,2.995186E-1,0E0,6.1462724E-1,4.595871E-1,0E0,4.5580864E-1,5.897119E-2,2.2367981E-1,0E0,0E0,0E0,0E0,0E0,1.7076445E-1,5.6651366E-1,2.7292013E-1,0E0,0E0,2.7914163E-1,0E0,8.195782E-2,1.5119326E-1,4.3113232E-1,0E0,0E0,1.5266606E-1,1.9892249E-1,1.2761688E-1,1.8848675E-1,0E0,1.9429922E-1,1.0479832E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,29,29,35,35,36,36,37,37,40,40,42,42,43,43,44,44,47,47,48,48,49,49,50,50,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,-1,40,42,-1,44,46,48,-1,-1,-1,-1,-1,50,52,54,-1,-1,56,-1,58,60,62,-1,-1,64,66,68,70,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.0495757E3,1.2730462E7,1.1576422E9,3.5301748E7,2.1678066E0,2.9778808E7,-1.5211606E-3,9.7467445E4,1.36E2,4.1331047E-1,2.2782404E0,4.52E2,6.5649785E-2,-2.8410532E-2,-1.4455157E-2,3.5921E4,1.4998456E5,2.6079019E3,1.9E2,1.7458724E7,1.2384782E5,-3.365566E-2,1.1E1,1.7895421E0,-9.603719E-3,7.556039E6,1.2230278E3,3.685938E6,1.2099462E-2,6.881782E-3,2.3003325E-2,8.076007E-3,-6.2692696E-3,1.955799E5,1.2584659E5,1.4485321E9,3.6752194E-2,2.0510098E-2,9.704588E6,1.5154765E-2,2.1030048E10,7.8936383E2,5.495E3,1.0040542E-2,2.3599163E-4,9.090909E-3,1.133457E6,2.1E1,1.2175E4,-1.0528918E-2,4.2220557E3,2E0,4.662348E-3,-1.1429119E-2,5.5883126E-3,4.556864E-2,1.7572934E-2,-9.527171E-3,-1.737532E-3,-1.7630909E-2,-1.1265992E-2,1.723069E-4,-7.671253E-3,-2.630114E-3,8.697303E-3,6.193252E-3,1.9605257E-2,-2.5251193E-3,1.2876789E-2,1.0168216E-3,1.39095355E-2,9.040107E-3,2.80287E-2],"split_indices":[2,1,60,9,7,53,50,12,0,41,3,43,62,3,0,0,0,10,36,4,10,53,36,0,8,61,0,53,4,1,0,0,0,0,0,41,41,40,0,0,9,0,5,60,2,0,0,65,9,8,2,0,4,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,3.14E2,1.64E2,7.9E1,2.35E2,1.27E2,3.7E1,7.6E1,3E0,2.16E2,1.9E1,1.14E2,1.3E1,2.3E1,1.4E1,6.9E1,7E0,1.65E2,5.1E1,1.1E1,8E0,6.8E1,4.6E1,3E0,1E1,2.1E1,2E0,1.6E2,5E0,4.4E1,7E0,5E0,6E0,3E0,5E0,5.1E1,1.7E1,2E1,2.6E1,2E0,8E0,3E0,1.8E1,3.9E1,1.21E2,2E0,3E0,3E1,1.4E1,4E1,1.1E1,3E0,1.4E1,1.8E1,2E0,5E0,3E0,1.6E1,2E0,3.1E1,8E0,5.9E1,6.2E1,8E0,2.2E1,9E0,5E0,3E0,3.7E1,2E0,9E0,4E0,1E1,2E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.0428554E-3,-2.3375034E-1,4.541337E-1,-5.006906E-1,-1.4976998E-1,3.2452E-1,8.727378E-1,-5.1503414E-1,5.12584E-4,-1.7805101E-1,2.8254488E-1,-5.51079E-1,3.902232E-1,2.1905756E-2,9.752052E-1,-2.651617E-2,-1.4796623E-2,-2.276966E-1,-5.180762E-3,-2.931465E-3,3.3980832E-1,-1.5467048E-1,-4.942744E-2,2.5589132E-1,5.7913125E-1,1.1343215E-2,-9.552822E-3,6.441708E-2,7.7096874E-1,-2.4262135E-1,4.1331388E-2,-7.648323E-2,1.1893303E-1,2.1373443E-2,7.6070717E-3,4.216619E-3,-1.4539782E-2,3.0148548E-1,3.6294505E-2,3.4672546E-1,6.7723626E-1,4.1612726E-2,-1.4315681E-3,-2.670048E-1,-8.824353E-2,1.2677863E-2,-4.3489956E-3,-1.0822589E-2,-9.341306E-3,-4.6018576E-3,1.819288E-1,1.3466248E-1,3.2214746E-1,-1.7874083E-1,1.2070688E-2,1.1681846E-3,2.0567393E-2,1.1905061E-2,3.4234516E-2,-1.4808739E-2,-7.826213E-3,-6.7395163E-3,4.41708E-4,-6.102523E-3,1.9401022E-3,1.3635325E-2,3.0926713E-3,-5.4910313E-3,1.1550458E-2,1.8762229E-2,1.2339731E-2,-3.2671655E-3,-1.5007006E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,33,35,-1,37,39,-1,-1,-1,41,43,45,47,49,-1,-1,-1,-1,51,53,55,57,-1,-1,59,61,-1,-1,63,-1,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9784035E1,6.896929E0,8.3436165E0,5.4792213E-1,2.9601703E0,7.291193E0,3.2043896E0,2.3127174E-1,0E0,1.9212427E0,3.143524E-1,1.4739721E0,2.77009E0,2.7195597E-1,1.2679081E0,0E0,0E0,7.086687E-1,4.60427E-1,0E0,1.6296828E-1,2.3027857E-1,0E0,6.7990875E-1,8.367939E-1,0E0,0E0,0E0,1.4528246E0,5.9754086E-1,3.053153E-1,2.4496293E-1,2.7174556E-1,0E0,0E0,0E0,0E0,1.5309143E-1,6.306244E-1,3.655274E-1,1.2070942E-1,0E0,0E0,4.852581E-1,1.1604449E-1,0E0,0E0,1.308675E-1,0E0,0E0,1.5384525E-1,2.2540624E-1,1.1686516E-1,7.950434E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,20,20,21,21,23,23,24,24,28,28,29,29,30,30,31,31,32,32,37,37,38,38,39,39,40,40,43,43,44,44,47,47,50,50,51,51,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,34,36,-1,38,40,-1,-1,-1,42,44,46,48,50,-1,-1,-1,-1,52,54,56,58,-1,-1,60,62,-1,-1,64,-1,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,5.932393E3,2.91E2,1.4777102E9,3.5226266E0,5.3140095E1,2.9778808E7,5.12584E-4,9.7467445E4,1E0,1.6244629E-1,2.7371603E-1,1.0014992E8,1.417988E6,-2.651617E-2,-1.4796623E-2,2.9063344E5,3.23643E6,-2.931465E-3,6.1614815E6,4.9014646E9,-4.942744E-2,2.4722598E7,1E0,1.1343215E-2,-9.552822E-3,6.441708E-2,2.0531621E4,7.164074E8,1.36E2,7.03764E5,4.3E1,2.1373443E-2,7.6070717E-3,4.216619E-3,-1.4539782E-2,4.0712055E1,3.8368778E8,1.5243992E0,8.825362E-2,4.1612726E-2,-1.4315681E-3,1.0864745E3,3.0811954E-2,1.2677863E-2,-4.3489956E-3,3.019898E8,-9.341306E-3,-4.6018576E-3,2.9194368E1,3.9380895E9,1.15E2,1E0,1.2070688E-2,1.1681846E-3,2.0567393E-2,1.1905061E-2,3.4234516E-2,-1.4808739E-2,-7.826213E-3,-6.7395163E-3,4.41708E-4,-6.102523E-3,1.9401022E-3,1.3635325E-2,3.0926713E-3,-5.4910313E-3,1.1550458E-2,1.8762229E-2,1.2339731E-2,-3.2671655E-3,-1.5007006E-2],"split_indices":[2,1,60,8,7,62,64,12,0,41,13,46,43,1,37,0,0,36,1,0,53,12,0,53,102,0,0,0,4,7,3,37,8,0,0,0,0,64,40,43,47,0,0,60,46,0,0,7,0,0,62,5,3,79,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,3.1E2,1.58E2,7.3E1,2.37E2,1.22E2,3.6E1,7.1E1,2E0,2.23E2,1.4E1,8E0,1.14E2,4E0,3.2E1,6.2E1,9E0,1.73E2,5E1,2E0,1.2E1,5E0,3E0,6.8E1,4.6E1,2E0,2E0,1E1,2.2E1,1.64E2,9E0,3.2E1,1.8E1,7E0,5E0,2E0,3E0,5.6E1,1.2E1,1.5E1,3.1E1,2E1,2E0,1.41E2,2.3E1,3E0,6E0,2.1E1,1.1E1,4E0,1.4E1,7E0,4.9E1,6E0,6E0,3E0,1.2E1,2E0,2.9E1,1.05E2,3.6E1,1.5E1,8E0,6E0,1.5E1,7E0,7E0,2E0,5E0,2.4E1,2.5E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[1.090001E-2,-2.0125449E-1,5.245391E-1,-4.5880142E-1,-1.2508579E-1,5.915077E-1,-4.0520313E-1,-4.8205665E-1,-1.3492837E-3,-1.6820766E-1,2.4283744E-1,4.6464685E-1,1.0216855E0,-7.2904617E-1,3.1605172E-3,-2.4885653E-2,-1.4353756E-2,-2.1574669E-1,7.182922E-6,2.7394533E-1,-5.2819857E-3,3.1228822E-1,6.0806996E-1,1.1267108E0,2.9222202E-1,-4.240126E-2,-1.4968307E-2,-2.500677E-1,-1.5598596E-2,-3.370713E-2,1.6202766E-2,3.1980896E-1,8.5113605E-4,3.6267152E-1,2.135414E-1,4.695611E-1,3.5255466E-2,6.8280116E-2,4.201578E-2,3.196325E-2,-8.079481E-3,-2.766193E-1,-1.1198159E-1,3.9400834E-1,-1.4521906E-1,-7.2935104E-2,1.0843815E-1,1.8508578E-2,8.424934E-3,3.9056635E-1,6.2940135E-3,7.248438E-3,1.5422687E-2,9.684674E-3,2.5742302E-2,-1.5569582E-2,-7.6223267E-3,-5.8714923E-4,-9.375765E-3,2.9577399E-2,9.3040215E-3,-1.7531946E-2,-4.554884E-3,-9.869089E-3,-2.13352E-3,8.3278236E-4,8.916322E-3,2.0376228E-2,9.206068E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,35,37,39,-1,-1,41,43,45,-1,47,-1,49,51,53,-1,-1,-1,-1,-1,55,57,59,61,63,65,-1,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.219363E1,6.592778E0,8.95657E0,7.5902367E-1,4.2134857E0,6.674305E0,1.5679222E0,2.027092E-1,0E0,1.8872395E0,3.3484066E-1,2.060835E0,1.8925018E0,8.0892086E-2,0E0,0E0,0E0,1.260047E0,6.015131E-1,3.0042005E-1,0E0,1.812644E-1,4.6086884E-1,5.997257E-1,9.2896813E-1,0E0,0E0,5.446491E-1,1.5438339E0,2.8113642E-1,0E0,1.22769356E-1,0E0,1.6557455E-1,8.623564E-2,2.8804398E-1,0E0,0E0,0E0,0E0,0E0,5.9955025E-1,2.1159208E-1,1.8462062E-1,2.1122456E-1,1.3851261E-1,7.07369E-2,0E0,0E0,6.872749E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,29,29,31,31,33,33,34,34,35,35,41,41,42,42,43,43,44,44,45,45,46,46,49,49],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,36,38,40,-1,-1,42,44,46,-1,48,-1,50,52,54,-1,-1,-1,-1,-1,56,58,60,62,64,66,-1,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.32424E5,1.0429407E3,1.2852292E7,1.4777102E9,7.7101436E3,1.31805E5,2.4434292E7,-1.3492837E-3,2.8884522E8,7.4433655E-2,4.1331047E-1,1.92E2,2.8E1,3.1605172E-3,-2.4885653E-2,-1.4353756E-2,2.2178302E5,2.4013862E5,1.436E3,-5.2819857E-3,1.955799E5,1.30817086E5,3.3616E4,2.235E5,-4.240126E-2,-1.4968307E-2,7.164074E8,1.36E2,2.0236879E3,1.6202766E-2,1.2155118E7,8.5113605E-4,6.03E2,8.714294E6,5.2060513E9,3.5255466E-2,6.8280116E-2,4.201578E-2,3.196325E-2,-8.079481E-3,1.1883916E3,8.9846605E8,6.738547E7,2.1066585E2,1.6776951E3,9.17E2,1.8508578E-2,8.424934E-3,2.5111986E-3,6.2940135E-3,7.248438E-3,1.5422687E-2,9.684674E-3,2.5742302E-2,-1.5569582E-2,-7.6223267E-3,-5.8714923E-4,-9.375765E-3,2.9577399E-2,9.3040215E-3,-1.7531946E-2,-4.554884E-3,-9.869089E-3,-2.13352E-3,8.3278236E-4,8.916322E-3,2.0376228E-2,9.206068E-3],"split_indices":[2,1,66,9,7,60,36,12,0,12,65,43,8,3,0,0,0,36,36,0,0,41,36,10,2,0,0,7,3,60,0,53,0,8,1,12,0,0,0,0,0,60,7,7,60,4,0,0,0,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.77E2,3.38E2,1.39E2,7.6E1,2.62E2,1.3E2,9E0,7.2E1,4E0,2.35E2,2.7E1,1.02E2,2.8E1,5E0,4E0,6.2E1,1E1,1.83E2,5.2E1,2.5E1,2E0,5.1E1,5.1E1,2.4E1,4E0,3E0,2E0,1.56E2,2.7E1,4.8E1,4E0,2.1E1,4E0,3.2E1,1.9E1,2.5E1,2.6E1,1E1,1.4E1,2E0,2E0,1.3E2,2.6E1,6E0,2.1E1,3.8E1,1E1,1.4E1,7E0,2.8E1,4E0,1.3E1,6E0,5E0,2E1,9.6E1,3.4E1,1.2E1,1.4E1,2E0,4E0,3E0,1.8E1,6E0,3.2E1,5E0,5E0,2.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[3.9354025E-4,-2.3259695E-1,3.9406484E-1,-4.652568E-1,-1.4789617E-1,2.7342576E-1,8.827721E-1,-4.8182985E-1,-1.1196867E-2,-1.7432804E-1,2.802906E-1,3.4866685E-1,-2.6009205E-1,3.45801E-3,9.5910287E-1,-2.394822E-2,-8.290545E-3,-2.1161978E-1,-3.7032522E-2,3.4730843E-1,-2.887003E-3,2.2603494E-1,5.664968E-1,-5.9868164E-2,-4.638637E-2,5.704899E-1,1.0694687E0,-2.2706585E-1,4.4322293E-2,-1.4362524E-1,6.2220503E-2,2.188584E-2,9.338836E-3,1.5561269E-1,3.3459944E-1,3.784149E-2,4.5556754E-1,4.065835E-2,-1.6991049E-2,3.4123566E-2,4.3070274E-3,3.0701652E-2,5.677997E-2,-2.4764997E-1,-1.0084831E-1,-2.764353E-3,1.0271415E-2,-1.0128967E-2,-1.4606183E-3,-4.4568717E-3,1.306098E-1,2.3245858E-3,2.0271866E-1,6.0021197E-3,3.707005E-1,4.7888964E-1,3.8380092E-3,-7.090009E-2,1.9564927E-2,-1.2656861E-2,-1.3333977E-3,1.4706246E-3,-6.809453E-3,2.384516E-3,1.3378378E-2,8.333407E-3,-1.086422E-2,1.5001749E-2,6.1531398E-3,4.655909E-3,1.9111674E-2,1.5534643E-2,2.6850637E-2,-7.931243E-3,5.187509E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,-1,-1,51,53,-1,55,57,-1,-1,-1,-1,-1,59,61,-1,-1,-1,-1,-1,63,65,67,-1,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.292568E1,5.7248926E0,1.0017178E1,1.9626617E-1,2.5048194E0,5.7507715E0,2.0403748E0,6.4291E-2,0E0,1.0417609E0,3.1593537E-1,3.184351E0,2.4150739E0,0E0,7.194557E-1,0E0,0E0,6.4654446E-1,4.8502076E-1,8.581865E-2,0E0,5.672059E-1,6.4984417E-1,4.4626966E-1,0E0,4.8342586E-1,1.7648697E-1,3.699336E-1,1.7808306E-1,1.5144077E-1,2.6316792E-1,0E0,0E0,3.677888E-1,1.921134E-1,0E0,2.3530436E-1,5.079551E-1,0E0,0E0,0E0,0E0,0E0,2.924223E-1,1.18545175E-1,0E0,0E0,0E0,0E0,0E0,1.8492681E-1,5.2543825E-1,2.7249646E-1,0E0,9.939337E-2,1.4927006E-1,0E0,1.7759642E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,43,43,44,44,50,50,51,51,52,52,54,54,55,55,57,57],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,-1,-1,52,54,-1,56,58,-1,-1,-1,-1,-1,60,62,-1,-1,-1,-1,-1,64,66,68,-1,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.204522E3,3.5002026E8,1.4096699E9,3.5301748E7,5.3140095E1,2.9778808E7,-1.1196867E-2,4.3159333E-1,1.5E2,4.1331047E-1,1.6244629E-1,3.45801E-3,6.274818E4,-2.394822E-2,-8.290545E-3,2.9063344E5,5.1457597E8,6.088569E6,-2.887003E-3,2.0663544E9,1.5990765E6,2.5503985E9,-4.638637E-2,5.37E2,9.430977E3,6.4065235E8,4.037E3,7.164074E8,1.656968E6,2.188584E-2,9.338836E-3,4.3290512E1,3.9997706E0,3.784149E-2,1.5631589E9,8.01E3,-1.6991049E-2,3.4123566E-2,4.3070274E-3,3.0701652E-2,5.677997E-2,3.0418E4,1.2504131E1,-2.764353E-3,1.0271415E-2,-1.0128967E-2,-1.4606183E-3,-4.4568717E-3,1.236E3,5.6E1,7.200105E8,6.0021197E-3,4.264618E6,8.4E1,3.8380092E-3,9.704588E6,1.9564927E-2,-1.2656861E-2,-1.3333977E-3,1.4706246E-3,-6.809453E-3,2.384516E-3,1.3378378E-2,8.333407E-3,-1.086422E-2,1.5001749E-2,6.1531398E-3,4.655909E-3,1.9111674E-2,1.5534643E-2,2.6850637E-2,-7.931243E-3,5.187509E-3],"split_indices":[2,1,60,7,7,53,64,12,0,47,3,43,46,0,36,0,0,36,5,53,0,12,55,7,0,3,4,7,10,7,9,0,0,64,61,0,40,0,0,0,0,0,0,10,66,0,0,0,0,0,0,3,5,0,1,3,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.66E2,2.93E2,1.73E2,7.7E1,2.16E2,1.4E2,3.3E1,7.1E1,6E0,2.04E2,1.2E1,1.23E2,1.7E1,3E0,3E1,6.9E1,2E0,1.6E2,4.4E1,1E1,2E0,8E1,4.3E1,1.4E1,3E0,8E0,2.2E1,1.51E2,9E0,2.1E1,2.3E1,5E0,5E0,5E1,3E1,1.3E1,3E1,1.1E1,3E0,6E0,2E0,5E0,1.7E1,1.29E2,2.2E1,6E0,3E0,1.3E1,8E0,7E0,1.6E1,1.2E1,3.8E1,5E0,2.5E1,2.8E1,2E0,9E0,2E0,1.23E2,6E0,5E0,1.7E1,1.1E1,5E0,7E0,5E0,1.5E1,2.3E1,2E0,2.3E1,1E1,1.8E1,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[9.059983E-3,-2.0842195E-1,4.2693752E-1,-4.3575704E-1,-1.2530746E-1,2.9884428E-1,8.101769E-1,-4.5101175E-1,-6.912644E-4,-1.5292355E-1,1.6339591E-1,3.7969777E-1,-2.807547E-1,-2.925789E-3,8.8041854E-1,-2.3273911E-2,-1.2534924E-2,-1.9522943E-1,-1.3053157E-2,-8.1437826E-2,2.4499632E-1,2.3961176E-1,5.520594E-1,-6.5199964E-2,-4.4680316E-2,5.768142E-2,7.1633565E-1,-2.3468864E-1,-8.478954E-2,-1.1094152E-2,2.0397974E-2,6.6168926E-4,-8.869504E-3,3.4602183E-1,6.56747E-2,8.066559E-2,2.876019E-1,3.3900094E-1,6.0481316E-1,4.8734993E-2,-1.6366132E-2,4.166219E-3,7.634476E-1,-2.5892565E-1,-8.16143E-2,-1.684155E-1,1.3401403E-1,-4.3062232E-2,1.2745374E-1,2.0173132E-2,7.118387E-3,-2.871371E-3,6.231689E-3,-1.0704313E-2,1.6644436E-1,3.4751168E-1,1.8635263E-1,1.853332E-2,4.719276E-3,1.2172822E-2,6.2828547E-1,-8.380266E-2,1.8925296E-2,3.9388083E-2,1.0013635E-2,-1.5498422E-2,-1.0208762E-2,-5.8014966E-3,1.8359823E-3,-9.459508E-3,-1.2796907E-3,8.916374E-3,-4.2151078E-3,-4.7696773E-3,4.5042652E-5,1.1894808E-2,2.0899738E-3,3.2650307E-3,1.3331517E-2,3.8220563E-3,1.7879277E-2,2.4716472E-3,1.181923E-2,1.5431431E-2,3.2297064E-2,-7.945448E-3,4.931976E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,31,33,35,37,39,-1,-1,41,43,45,-1,47,-1,-1,49,51,53,55,57,59,61,-1,-1,63,65,67,69,71,73,75,-1,-1,-1,-1,-1,77,79,81,-1,-1,-1,83,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4075294E1,5.972851E0,7.906887E0,5.4039E-1,1.8986101E0,6.0112906E0,2.5575848E0,2.566452E-1,0E0,1.2696786E0,4.3950105E-1,2.547865E0,2.1146429E0,0E0,1.0018921E0,0E0,0E0,6.991501E-1,3.6829844E-1,5.9073612E-2,2.6713204E-1,4.5735526E-1,3.4571075E-1,4.1401005E-1,0E0,0E0,7.382679E-1,4.277854E-1,8.4596455E-1,0E0,3.1120858E-1,0E0,0E0,7.1546435E-2,6.083507E-2,4.4662532E-1,2.263999E-1,7.7038884E-2,1.6209698E-1,4.792435E-1,0E0,0E0,3.9177513E-1,2.356677E-1,8.305358E-2,1.0994905E-1,1.523403E-1,6.921928E-2,1.55666E-1,0E0,0E0,0E0,0E0,0E0,1.187481E-1,1.0512185E-1,1.3648623E-1,0E0,0E0,0E0,1.2549686E-1,1.3167058E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,42,42,43,43,44,44,45,45,46,46,47,47,48,48,54,54,55,55,56,56,60,60,61,61],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,32,34,36,38,40,-1,-1,42,44,46,-1,48,-1,-1,50,52,54,56,58,60,62,-1,-1,64,66,68,70,72,74,76,-1,-1,-1,-1,-1,78,80,82,-1,-1,-1,84,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.0495757E3,1.2852292E7,1.1576422E9,3.5301748E7,5.0314346E1,2.9778808E7,-6.912644E-4,9.7467445E4,2.1530056E3,1.0693399E-1,1.6244629E-1,-2.925789E-3,1.417988E6,-2.3273911E-2,-1.2534924E-2,2.231E3,1.5571355E3,1.27E2,1.5E2,2.6300613E3,5.4730445E9,2.5503985E9,-4.4680316E-2,5.768142E-2,1.563E3,7.164074E8,7.330576E2,-1.1094152E-2,1.207E3,6.6168926E-4,-8.869504E-3,9.565983E6,2.9222983E3,2.8884522E8,1.955799E5,1.8163872E7,3.0811954E-2,8.01E3,-1.6366132E-2,4.166219E-3,1.1986E4,1.6415557E9,1.7933134E5,5.074412E0,1.0767E4,4.6976192E8,1.473681E6,2.0173132E-2,7.118387E-3,-2.871371E-3,6.231689E-3,-1.0704313E-2,1.5029658E-1,2.7E1,2.2969543E3,1.853332E-2,4.719276E-3,1.2172822E-2,7.796878E4,2.34E2,1.8925296E-2,3.9388083E-2,1.0013635E-2,-1.5498422E-2,-1.0208762E-2,-5.8014966E-3,1.8359823E-3,-9.459508E-3,-1.2796907E-3,8.916374E-3,-4.2151078E-3,-4.7696773E-3,4.5042652E-5,1.1894808E-2,2.0899738E-3,3.2650307E-3,1.3331517E-2,3.8220563E-3,1.7879277E-2,2.4716472E-3,1.181923E-2,1.5431431E-2,3.2297064E-2,-7.945448E-3,4.931976E-3],"split_indices":[2,1,60,9,7,53,64,12,0,41,4,43,46,0,37,0,0,0,63,3,3,63,5,7,0,0,0,7,60,0,0,0,0,53,4,12,41,1,46,0,0,0,0,5,55,61,0,7,37,0,0,0,0,0,46,3,60,0,0,0,36,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,3.18E2,1.65E2,8.4E1,2.34E2,1.25E2,4E1,8.1E1,3E0,2.14E2,2E1,1.1E2,1.5E1,3E0,3.7E1,7.1E1,1E1,1.64E2,5E1,5E0,1.5E1,6.2E1,4.8E1,1.2E1,3E0,1.1E1,2.6E1,1.2E2,4.4E1,6E0,4.4E1,3E0,2E0,9E0,6E0,1.5E1,4.7E1,1.1E1,3.7E1,9E0,3E0,2E0,2.4E1,1.03E2,1.7E1,3.2E1,1.2E1,2.8E1,1.6E1,6E0,3E0,2E0,4E0,3E0,1.2E1,2.8E1,1.9E1,9E0,2E0,3E0,3.4E1,7E0,2E0,2.2E1,2E0,4.6E1,5.7E1,1.3E1,4E0,2.7E1,5E0,1E1,2E0,1.2E1,1.6E1,6E0,1E1,7E0,5E0,2E0,2.6E1,6E0,1.3E1,4E0,3E1,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[1.3119875E-2,-1.7824717E-1,4.1675794E-1,-4.0649578E-1,-1.167514E-1,4.7617406E-1,-3.309651E-1,-4.2414996E-1,-6.6584133E-4,-1.5031372E-1,2.0340441E-1,3.768923E-1,8.4088165E-1,-7.051911E-1,3.7080754E-2,-2.179144E-2,-1.0887072E-2,-1.7581342E-1,1.3259873E-1,2.4578093E-1,-3.948496E-3,2.292831E-1,4.6850845E-1,6.415209E-1,5.3278998E-2,-4.1371707E-2,-1.4001387E-2,-5.9942263E-3,1.4233314E-2,-6.6182345E-2,-2.2052306E-1,-2.9217636E-2,2.6765007E-1,1.7800823E-2,1.3671118E-1,1.6688998E-1,1.6469222E-2,4.188739E-2,4.257015E-1,1.0774778E-2,3.3259206E-2,-1.3660364E-1,2.9238015E-2,-2.4384235E-1,-9.723872E-2,-1.038611E-1,8.805965E-3,3.2185426E-3,1.4969161E-2,8.200124E-3,-1.0157063E-3,-1.0274751E-2,2.0982051E-1,4.4874385E-1,-9.812133E-3,-2.5279573E-3,-9.427817E-3,4.664128E-3,-2.1296768E-3,-1.2487552E-2,-1.0162556E-3,1.3724109E-3,-1.013014E-2,-9.272174E-3,-7.314447E-4,-4.270574E-3,5.941943E-3,1.5987389E-2,6.6675274E-3,2.296995E-2,9.019397E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,39,-1,-1,-1,-1,-1,41,43,45,47,-1,49,51,-1,-1,53,-1,-1,55,57,59,61,63,-1,-1,-1,-1,-1,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.722818E1,4.5424185E0,7.043747E0,4.7286987E-1,2.8135731E0,4.897152E0,1.6789368E0,1.7110634E-1,0E0,1.7186451E0,3.2058036E-1,1.4477692E0,7.7178764E-1,1.077714E-1,3.1780747E-1,0E0,0E0,1.0441442E0,4.4489047E-1,2.3163033E-1,0E0,2.4977756E-1,7.919979E-1,1.4509773E-1,0E0,0E0,0E0,0E0,0E0,4.3405E-1,4.1614914E-1,1.7449737E-1,6.352097E-2,0E0,6.58928E-2,2.3793489E-1,0E0,0E0,1.0270681E0,0E0,0E0,1.6521269E-1,1.3815871E-1,3.0325365E-1,3.617578E-1,5.7816885E-2,0E0,0E0,0E0,0E0,0E0,8.122522E-2,1.7021537E-1,2.418232E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,23,23,29,29,30,30,31,31,32,32,34,34,35,35,38,38,41,41,42,42,43,43,44,44,45,45,51,51,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,40,-1,-1,-1,-1,-1,42,44,46,48,-1,50,52,-1,-1,54,-1,-1,56,58,60,62,64,-1,-1,-1,-1,-1,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.0429407E3,1.2677199E7,1.4777102E9,4.026531E9,3.5226266E0,7.154E3,-6.6584133E-4,8.931424E8,7.4074075E-2,9.096001E6,7.443074E0,3.1E1,3.0972284E1,-2.179144E-2,-1.0887072E-2,7.37365E6,1.656968E6,6.088569E6,-3.948496E-3,1.5271514E5,1.5116836E9,4.2464733E-2,5.3278998E-2,-4.1371707E-2,-1.4001387E-2,-5.9942263E-3,1.4233314E-2,1.701419E6,1.4849418E3,1.4998456E5,1.281E3,1.7800823E-2,1.323E3,4.0712055E1,1.6469222E-2,4.188739E-2,1.4609149E4,1.0774778E-2,3.3259206E-2,2.44753E5,1.52E2,6.33E2,2.3490406E1,9.5E1,8.805965E-3,3.2185426E-3,1.4969161E-2,8.200124E-3,-1.0157063E-3,1.3210104E9,5.4629724E10,2.7673964E7,-9.812133E-3,-2.5279573E-3,-9.427817E-3,4.664128E-3,-2.1296768E-3,-1.2487552E-2,-1.0162556E-3,1.3724109E-3,-1.013014E-2,-9.272174E-3,-7.314447E-4,-4.270574E-3,5.941943E-3,1.5987389E-2,6.6675274E-3,2.296995E-2,9.019397E-3],"split_indices":[2,1,66,9,7,7,62,2,0,12,65,1,61,3,64,0,0,53,9,53,0,36,12,46,0,0,0,0,0,1,60,36,0,0,0,64,0,0,4,0,0,37,3,8,61,3,0,0,0,0,0,7,39,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.8E2,3.26E2,1.54E2,6.8E1,2.58E2,1.43E2,1.1E1,6.5E1,3E0,2.34E2,2.4E1,1.14E2,2.9E1,5E0,6E0,5.8E1,7E0,2.15E2,1.9E1,2.1E1,3E0,4.5E1,6.9E1,1.8E1,1.1E1,3E0,2E0,4E0,2E0,6.3E1,1.52E2,9E0,1E1,9E0,1.2E1,3E1,1.5E1,5E0,6.4E1,2E0,1.6E1,3.6E1,2.7E1,1.27E2,2.5E1,7E0,2E0,2E0,8E0,1E1,2E0,6E0,2.4E1,6.2E1,2E0,1.5E1,2.1E1,1.4E1,1.3E1,1.21E2,6E0,1.2E1,1.3E1,3E0,4E0,4E0,2E0,8E0,1.6E1,5.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-2.4456151E-3,-1.9164747E-1,3.3113247E-1,-4.2175058E-1,-1.1742644E-1,3.8199288E-1,-3.8307709E-1,-2.182105E-2,-1.2979769E-2,-1.3854733E-1,2.2135237E-1,1.9199598E-1,5.791795E-1,-3.9656173E-2,3.6439355E-2,-2.3165552E-1,-8.2218215E-2,-3.8123748E-3,2.7767986E-1,2.3901893E-1,-2.279022E-2,5.238052E-1,4.6668813E-2,-5.7590674E-3,1.376799E-2,-1.3423592E-1,-2.7968356E-1,-3.250078E-3,-1.5047106E-1,1.9439625E-2,6.6811163E-3,2.7463195E-1,1.03886284E-1,-1.8672279E-1,1.0526452E-2,4.523199E-1,3.837513E-2,-1.6524993E-1,5.9792295E-4,-6.4490736E-3,-3.06396E-1,-4.5098208E-2,1.0116051E-2,-1.7365251E-1,3.0640776E-3,1.8361236E-1,1.7005024E-2,-2.594059E-3,7.0682946E-3,-1.1347614E-2,-2.6199565E-4,4.7682226E-1,-7.641765E-3,-1.1311721E-2,-3.2840287E-3,-2.007329E-2,-1.2098093E-2,-4.3420065E-3,9.413009E-4,-1.1088079E-2,-4.762737E-3,-5.341357E-3,1.0422175E-2,2.3945816E-2,5.455451E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,25,27,-1,29,31,33,35,-1,-1,-1,37,39,41,43,-1,-1,45,47,49,-1,51,-1,53,-1,-1,55,57,-1,59,-1,61,-1,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0168644E1,5.1594152E0,6.4305534E0,1.5214634E-1,1.6907003E0,5.958105E0,2.1719694E0,0E0,0E0,1.1295433E0,2.574485E-1,8.53837E-1,1.130991E0,0E0,2.961365E-1,3.4612417E-1,7.433035E-1,0E0,1.4751226E-1,3.0353117E-1,6.628808E-1,9.592552E-1,0E0,0E0,0E0,1.3371384E-1,1.7344284E-1,5.792683E-1,3.7615693E-1,0E0,0E0,2.9087067E-1,1.0859367E-1,7.9799354E-2,0E0,9.385681E-1,0E0,1.3885921E-1,0E0,0E0,1.7552233E-1,1.5532215E-1,0E0,2.4700737E-1,0E0,2.4308068E-1,0E0,0E0,0E0,0E0,0E0,1.8069077E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,20,20,21,21,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,37,37,40,40,41,41,43,43,45,45,51,51],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,26,28,-1,30,32,34,36,-1,-1,-1,38,40,42,44,-1,-1,46,48,50,-1,52,-1,54,-1,-1,56,58,-1,60,-1,62,-1,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,1.0429407E3,2.7299898E8,1.4096699E9,3.9837E4,3.5226266E0,-2.182105E-2,-1.2979769E-2,6.603817E2,2.0532622E3,2.4722598E7,4.2875492E1,-3.9656173E-2,3.0972284E1,7.2123305E6,1.3452116E7,-3.8123748E-3,6.088569E6,2.0136225E5,2.2989038E6,9.317209E0,4.6668813E-2,-5.7590674E-3,1.376799E-2,8.6419773E-1,5.1698097E1,2.7654392E9,7.1999524E9,1.9439625E-2,6.6811163E-3,1.2934E4,2.0236879E3,1.2538845E9,1.0526452E-2,4.52E2,3.837513E-2,1.964127E6,5.9792295E-4,-6.4490736E-3,4.94E4,5.447E3,1.0116051E-2,1.5930355E8,3.0640776E-3,4.026648E8,1.7005024E-2,-2.594059E-3,7.0682946E-3,-1.1347614E-2,-2.6199565E-4,6.029086E10,-7.641765E-3,-1.1311721E-2,-3.2840287E-3,-2.007329E-2,-1.2098093E-2,-4.3420065E-3,9.413009E-4,-1.1088079E-2,-4.762737E-3,-5.341357E-3,1.0422175E-2,2.3945816E-2,5.455451E-3],"split_indices":[2,1,66,7,7,2,62,0,0,60,4,53,62,0,64,53,53,0,53,41,55,43,0,0,0,49,56,5,5,0,0,2,60,7,0,3,0,1,0,0,9,10,0,7,0,7,0,0,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,3.04E2,1.72E2,7.3E1,2.31E2,1.61E2,1.1E1,6.2E1,1.1E1,2.18E2,1.3E1,8.3E1,7.8E1,5E0,6E0,8.1E1,1.37E2,2E0,1.1E1,6.8E1,1.5E1,7E1,8E0,4E0,2E0,2.8E1,5.3E1,6.4E1,7.3E1,5E0,6E0,5.3E1,1.5E1,9E0,6E0,5.7E1,1.3E1,2.3E1,5E0,9E0,4.4E1,5.4E1,1E1,6.6E1,7E0,2.5E1,2.8E1,3E0,1.2E1,7E0,2E0,5.5E1,2E0,1.3E1,1E1,1.4E1,3E1,3.2E1,2.2E1,3.8E1,2.8E1,2E0,2.3E1,5.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[4.436808E-3,-1.9417304E-1,3.2303852E-1,-3.7810808E-1,-1.3335645E-1,2.2332142E-1,6.839158E-1,-3.9365274E-1,-6.6853226E-3,-1.5638648E-1,1.801262E-1,2.966062E-1,-3.2243586E-1,4.522681E-4,7.5984365E-1,-1.959789E-2,-6.3131023E-3,-1.9048658E-1,-2.1856852E-2,-3.2859119E-3,2.4231108E-1,1.96506E-1,4.6560234E-1,-1.11662254E-1,-4.1590665E-2,9.278999E-3,-9.242034E-3,5.106996E-2,6.1149204E-1,-2.3822552E-1,-1.2893465E-1,-1.0431204E-2,8.066006E-3,1.6759347E-2,6.7398474E-3,1.35604E-1,2.70203E-1,2.9718912E-1,5.1348484E-1,9.354197E-3,-1.7796022E-1,3.1769841E-3,6.543516E-1,-1.3749225E-1,-2.767674E-1,8.41075E-3,-2.0478642E-1,-4.1223083E-2,1.2240171E-1,5.7117704E-2,2.6324126E-1,1.7765559E-3,1.4587416E-2,1.6470263E-2,2.7623558E-3,6.9590984E-3,5.311684E-1,-1.1794227E-2,4.4061593E-3,3.4240812E-2,1.2768048E-2,-2.5201265E-3,-8.609898E-3,-7.050651E-3,-1.4727534E-2,6.775236E-3,-4.1031833E-3,-2.5202518E-3,-1.1448801E-2,-9.602383E-4,-8.149268E-3,3.093796E-4,1.0024735E-2,4.3785684E-3,-8.942289E-3,1.3904397E-2,3.2976714E-3,1.3286436E-2,2.7306594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,33,35,37,39,-1,-1,-1,-1,41,43,45,-1,47,-1,-1,49,51,53,55,-1,57,-1,59,61,63,65,67,69,71,73,75,-1,-1,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.081556E1,3.3054686E0,6.576584E0,2.2445583E-1,1.6669998E0,6.0150023E0,2.0754108E0,5.7980537E-2,0E0,9.683809E-1,2.6212257E-1,2.1245737E0,1.8996844E0,2.139631E-1,8.6696625E-1,0E0,0E0,4.65878E-1,2.5299755E-1,0E0,8.8606715E-2,3.4037638E-1,2.3447323E-1,3.146367E-1,0E0,0E0,0E0,0E0,5.668297E-1,3.2689714E-1,7.934822E-1,0E0,2.2483037E-1,0E0,0E0,4.658175E-1,2.2524095E-1,1.0216868E-1,1.6117E-1,0E0,2.2261879E-1,0E0,2.3316956E-1,8.278352E-2,1.6671133E-1,3.4609523E-1,2.0354581E-1,7.246642E-2,1.1297661E-1,2.5509527E-1,5.253768E-2,0E0,0E0,0E0,0E0,0E0,7.622719E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,20,20,21,21,22,22,23,23,28,28,29,29,30,30,32,32,35,35,36,36,37,37,38,38,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,34,36,38,40,-1,-1,-1,-1,42,44,46,-1,48,-1,-1,50,52,54,56,-1,58,-1,60,62,64,66,68,70,72,74,76,-1,-1,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.0495757E3,3.5002026E8,1.2538845E9,3.5301748E7,5.6816803E1,2.9778808E7,-6.6853226E-3,9.7467445E4,2.0532622E3,4.1331047E-1,1.6244629E-1,4.452787E7,1.417988E6,-1.959789E-2,-6.3131023E-3,7.8936383E2,4.39E3,-3.2859119E-3,6.088569E6,7.228959E6,3.3406117E11,2.9E1,-4.1590665E-2,9.278999E-3,-9.242034E-3,5.106996E-2,8.713209E10,7.2123305E6,4.6033485E1,-1.0431204E-2,8.464347E8,1.6759347E-2,6.7398474E-3,3.44E2,3.7860696E0,5.441669E9,3.1095595E0,9.354197E-3,2.34E2,3.1769841E-3,2.3322949E2,5.428232E9,5.381665E1,2.58E2,8.8561325E0,7.094791E1,5.792806E4,2.47E2,1.114253E6,1.7765559E-3,1.4587416E-2,1.6470263E-2,2.7623558E-3,6.9590984E-3,7.796878E4,-1.1794227E-2,4.4061593E-3,3.4240812E-2,1.2768048E-2,-2.5201265E-3,-8.609898E-3,-7.050651E-3,-1.4727534E-2,6.775236E-3,-4.1031833E-3,-2.5202518E-3,-1.1448801E-2,-9.602383E-4,-8.149268E-3,3.093796E-4,1.0024735E-2,4.3785684E-3,-8.942289E-3,1.3904397E-2,3.2976714E-3,1.3286436E-2,2.7306594E-2],"split_indices":[2,1,60,7,7,53,64,12,0,41,4,43,46,1,37,0,0,60,2,0,53,1,39,3,0,0,0,0,39,53,64,0,12,0,0,8,61,5,62,0,8,0,65,39,56,3,66,64,36,8,37,0,0,0,0,0,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,2.99E2,1.86E2,7.3E1,2.26E2,1.47E2,3.9E1,6.8E1,5E0,2.11E2,1.5E1,1.3E2,1.7E1,4E0,3.5E1,6.6E1,2E0,1.68E2,4.3E1,3E0,1.2E1,8.3E1,4.7E1,1.3E1,4E0,2E0,2E0,1E1,2.5E1,9.3E1,7.5E1,5E0,3.8E1,5E0,7E0,4.7E1,3.6E1,1.2E1,3.5E1,2E0,1.1E1,2E0,2.3E1,2.7E1,6.6E1,2.7E1,4.8E1,2.7E1,1.1E1,3E1,1.7E1,4E0,3.2E1,1E1,2E0,2E0,3.3E1,9E0,2E0,2E1,3E0,9E0,1.8E1,1.1E1,5.5E1,1.1E1,1.6E1,8E0,4E1,2.4E1,3E0,5E0,6E0,2.7E1,3E0,1.5E1,2E0,4E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[6.439933E-3,-1.7402151E-1,3.3794793E-1,-3.6331376E-1,-1.0391866E-1,3.7325773E-1,-5.476766E-1,-3.9149588E-1,-2.1730989E-1,-1.24809474E-1,1.3684736E-1,2.979936E-1,7.185402E-1,-3.6861893E-2,-1.0138712E-2,-1.9828018E-2,-1.0700755E-2,-1.1821104E-2,-1.6703116E-3,-1.9401221E-1,-6.4672574E-2,2.5260627E-1,-1.9841002E-2,1.5851632E-1,4.0642995E-1,-8.9398865E-3,7.920211E-1,-2.5532547E-1,-1.20743826E-1,4.110032E-2,-1.426423E-1,1.727773E-3,1.4564411E-2,-4.6101985E-3,2.858831E-3,2.0501211E-1,3.347038E-2,4.3128365E-1,6.775034E-3,5.3306222E-2,5.8413285E-1,-2.6552093E-1,1.5232562E-4,-1.3661021E-1,1.954488E-3,-3.293915E-2,1.1645736E-1,-2.1769598E-1,-5.8717426E-2,1.2353599E-1,2.7757466E-1,-6.2949406E-3,1.2508582E-1,4.5238784E-1,3.2587184E-3,3.2532092E-2,-1.6329271E-3,-1.4937141E-2,-8.309669E-3,-2.931622E-3,-8.823295E-3,-3.8814654E-3,3.096504E-3,-3.0862878E-3,8.651458E-3,-1.2953775E-2,-4.527365E-3,2.1857019E-3,-5.5113626E-3,-1.1994615E-3,8.140695E-3,1.8123636E-2,1.050539E-2,3.7836484E-4,8.010608E-3,1.847171E-2,2.6563881E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,31,33,35,37,-1,39,41,43,45,47,-1,-1,-1,-1,49,51,53,-1,-1,55,57,-1,59,-1,61,63,65,67,69,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.91933E1,4.1555424E0,5.5830727E0,2.6467323E-1,1.1839931E0,4.099783E0,3.314588E-1,6.747341E-2,6.1410785E-2,8.780434E-1,3.4963536E-1,2.0227432E0,2.0640326E0,0E0,0E0,0E0,0E0,0E0,0E0,4.139111E-1,9.6252435E-1,9.567857E-2,5.764009E-2,3.5361314E-1,4.5381927E-1,0E0,1.0208511E0,1.4047098E-1,1.2520707E-1,2.8280577E-1,4.092418E-1,0E0,0E0,0E0,0E0,2.3175466E-1,2.829378E-1,5.089035E-1,0E0,0E0,9.0296793E-1,1.3984513E-1,0E0,1.3233751E-1,0E0,1.204414E-1,2.8002515E-1,1.7946982E-1,1.879439E-1,1.486192E-1,5.4876804E-2,0E0,5.2635267E-2,2.4220753E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,35,35,36,36,37,37,40,40,41,41,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,32,34,36,38,-1,40,42,44,46,48,-1,-1,-1,-1,50,52,54,-1,-1,56,58,-1,60,-1,62,64,66,68,70,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,2.3028242E3,2.1915232E8,1.1576422E9,7.3068325E3,3.1E1,2.2198055E6,1.494306E3,2.025313E6,1.24E2,2.5096E4,5.0314346E1,-3.6861893E-2,-1.0138712E-2,-1.9828018E-2,-1.0700755E-2,-1.1821104E-2,-1.6703116E-3,3.006846E6,1.4074289E7,1.8505337E3,2.8720352E3,1.7458724E7,1.773296E6,-8.9398865E-3,1.417988E6,2.9063344E5,7.8E2,2.972948E6,1E0,1.727773E-3,1.4564411E-2,-4.6101985E-3,2.858831E-3,5.5785873E1,3.0811954E-2,7.2125336E2,6.775034E-3,5.3306222E-2,2.0531621E4,4.898457E5,1.5232562E-4,1.082442E6,1.954488E-3,4.9910698E8,4.5E1,4.0288494E1,2.41E2,3.253389E8,3.3824077E3,-6.2949406E-3,7.7444346E8,1.646359E5,3.2587184E-3,3.2532092E-2,-1.6329271E-3,-1.4937141E-2,-8.309669E-3,-2.931622E-3,-8.823295E-3,-3.8814654E-3,3.096504E-3,-3.0862878E-3,8.651458E-3,-1.2953775E-2,-4.527365E-3,2.1857019E-3,-5.5113626E-3,-1.1994615E-3,8.140695E-3,1.8123636E-2,1.050539E-2,3.7836484E-4,8.010608E-3,1.847171E-2,2.6563881E-2],"split_indices":[2,1,66,7,7,60,3,58,4,1,3,2,64,0,0,0,0,0,0,9,53,4,4,53,11,0,37,36,8,1,98,0,0,0,0,64,46,66,0,0,4,59,0,1,0,7,8,64,3,7,63,0,7,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,3.15E2,1.71E2,8.4E1,2.31E2,1.65E2,6E0,6.9E1,1.5E1,2.13E2,1.8E1,1.37E2,2.8E1,3E0,3E0,6.3E1,6E0,1.3E1,2E0,9.8E1,1.15E2,1E1,8E0,6.1E1,7.6E1,2E0,2.6E1,5.2E1,4.6E1,4.9E1,6.6E1,2E0,8E0,4E0,4E0,4.4E1,1.7E1,6.9E1,7E0,9E0,1.7E1,5E1,2E0,4.2E1,4E0,2.5E1,2.4E1,3.4E1,3.2E1,2.2E1,2.2E1,6E0,1.1E1,6.5E1,4E0,1.5E1,2E0,3.4E1,1.6E1,1.6E1,2.6E1,1.7E1,8E0,6E0,1.8E1,2.4E1,1E1,1.1E1,2.1E1,5E0,1.7E1,7E0,1.5E1,3E0,8E0,3.8E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-3.2293135E-3,-1.5594651E-1,3.2082593E-1,-3.362208E-1,-9.522878E-2,3.7959564E-1,-3.0238086E-1,-3.485166E-1,-3.064866E-4,-1.2417388E-1,1.6908635E-1,3.0785912E-1,7.6015294E-1,-3.0656805E-2,-3.8512878E-2,-1.7868884E-2,-1.05863E-2,-1.6273548E-1,1.765418E-2,2.071113E-1,-4.037006E-3,2.2563083E-1,4.591123E-1,4.5159474E-1,9.7965187E-1,-1.8196996E-1,1.1857884E-2,-8.185059E-2,-1.9393542E-1,-8.95217E-3,4.9835358E-2,2.3712817E-1,-3.0422914E-3,2.5142166E-1,8.4244646E-2,5.496992E-1,3.2052293E-1,3.0056858E-2,-8.969552E-3,5.3696938E-2,2.3305206E-2,-1.9111938E-3,-1.3705134E-2,-1.2850624E-1,1.0824031E-2,-3.013967E-1,-1.5759279E-1,-5.79239E-2,1.2973899E-1,1.504553E-2,7.4650045E-3,2.0522271E-1,3.0647463E-1,-4.8685134E-3,8.218034E-3,2.8035006E-2,1.0063335E-2,4.392762E-3,1.6792106E-2,-1.877293E-3,-8.067141E-3,4.1765054E-3,-2.168878E-3,-1.6623635E-2,-8.426883E-3,-9.084759E-3,2.9339982E-4,-7.552923E-3,-6.180345E-4,1.6918088E-3,9.911064E-3,1.23193655E-2,5.4835198E-3,1.5777344E-2,3.2861142E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,35,37,39,41,-1,43,45,-1,47,49,-1,51,53,55,57,-1,-1,-1,-1,-1,-1,59,61,63,65,67,69,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4101591E1,3.5910578E0,5.82839E0,3.3457375E-1,1.926003E0,3.6698608E0,1.8132149E0,8.968544E-2,0E0,1.2334237E0,2.560451E-1,1.4137583E0,1.047615E0,4.5627248E-1,0E0,0E0,0E0,4.3080425E-1,3.2479855E-1,1.9244963E-1,0E0,2.741828E-1,3.5109234E-1,1.2380652E0,1.7772865E-1,8.6707965E-2,0E0,2.2190455E-1,4.484911E-1,0E0,3.7776026E-1,7.161474E-2,0E0,1.1005497E-1,2.3075727E-1,6.4484596E-2,6.5427065E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0413277E-1,7.743554E-2,9.272504E-2,4.4003606E-1,8.080613E-2,1.653327E-1,0E0,0E0,1.437453E-1,8.4387064E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,30,30,31,31,33,33,34,34,35,35,36,36,43,43,44,44,45,45,46,46,47,47,48,48,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,36,38,40,42,-1,44,46,-1,48,50,-1,52,54,56,58,-1,-1,-1,-1,-1,-1,60,62,64,66,68,70,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.0429407E3,1.2852292E7,1.4777102E9,8.10288E3,7.339209E6,1.7367111E6,-3.064866E-4,4.1619555E8,7.4074075E-2,6.8963E4,1.7476442E0,3.236383E1,-3.8512878E-2,-1.7868884E-2,-1.05863E-2,7.477264E6,1.5802073E3,3.931011E6,-4.037006E-3,2.4851996E7,1.9257854E7,3.14E2,8.639938E9,2.2201815E9,1.1857884E-2,1.701419E6,1.4587239E3,-8.95217E-3,1.07E3,6.088569E6,-3.0422914E-3,2.5096E4,2.2989038E6,6.520156E7,3.116E3,3.0056858E-2,-8.969552E-3,5.3696938E-2,2.3305206E-2,-1.9111938E-3,-1.3705134E-2,5.9460024E9,1.33E2,2.0964778E5,1.2128688E5,9.704588E6,1.133457E6,1.504553E-2,7.4650045E-3,1.8849752E5,9.652E3,-4.8685134E-3,8.218034E-3,2.8035006E-2,1.0063335E-2,4.392762E-3,1.6792106E-2,-1.877293E-3,-8.067141E-3,4.1765054E-3,-2.168878E-3,-1.6623635E-2,-8.426883E-3,-9.084759E-3,2.9339982E-4,-7.552923E-3,-6.180345E-4,1.6918088E-3,9.911064E-3,1.23193655E-2,5.4835198E-3,1.5777344E-2,3.2861142E-3],"split_indices":[2,1,66,9,7,60,55,59,0,12,65,2,50,64,0,0,0,53,4,1,0,53,51,3,12,7,0,1,4,0,0,53,0,2,55,1,0,0,0,0,0,0,0,39,3,36,36,9,9,0,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,3.3E2,1.55E2,8.2E1,2.48E2,1.42E2,1.3E1,7.9E1,3E0,2.24E2,2.4E1,1.21E2,2.1E1,9E0,4E0,6.9E1,1E1,1.76E2,4.8E1,2.1E1,3E0,8E1,4.1E1,1E1,1.1E1,6E0,3E0,5E1,1.26E2,6E0,4.2E1,1.9E1,2E0,6.7E1,1.3E1,2.3E1,1.8E1,8E0,2E0,8E0,3E0,3E0,3E0,3.3E1,1.7E1,3E1,9.6E1,1.8E1,2.4E1,9E0,1E1,3.9E1,2.8E1,4E0,9E0,2.1E1,2E0,2E0,1.6E1,1E1,2.3E1,7E0,1E1,2.2E1,8E0,8.2E1,1.4E1,5E0,1.3E1,1.1E1,1.3E1,2.5E1,1.4E1,2.6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.160108E-2,-1.2755595E-1,3.50646E-1,-3.368603E-1,-7.0714176E-2,4.1203713E-1,-3.7885758E-1,-3.4978926E-1,-1.7695631E-3,-1.0248896E-1,1.6731526E-1,3.2671326E-1,7.5749534E-1,-5.63887E-2,-3.7001975E-2,-2.2230716E-2,-1.5392648E-2,-1.1354348E-1,4.1767338E-1,5.6213632E-2,1.2529651E-2,2.1589646E-1,4.1748586E-1,8.1246835E-1,4.9951877E-3,-1.1840204E-2,1.3281439E-2,-1.4870664E-1,-1.3780293E-2,2.6154373E-2,7.982736E-3,1.9822319E-3,1.0045847E-2,4.472478E-4,2.2947887E-1,5.18971E-1,3.3410266E-1,4.8095267E-2,2.2578157E-2,-1.6012712E-1,4.6607632E-2,-7.09166E-2,1.4777684E-1,3.2386193E-3,-3.5839758E-3,2.4121352E-1,2.3977773E-3,5.6934864E-3,2.7060233E-2,3.5326144E-1,1.2142594E-3,-9.368537E-3,-4.8429426E-3,-4.3158866E-3,4.4754446E-3,-1.0829633E-2,-2.2879404E-3,-1.3302342E-4,8.960095E-3,1.38288755E-2,8.679948E-3,4.575131E-3,1.8045012E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,35,37,-1,-1,-1,39,41,-1,-1,43,-1,-1,45,47,49,-1,-1,51,53,55,57,-1,-1,59,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.188929E1,3.890367E0,6.2020855E0,2.5958872E-1,1.9836146E0,3.4369965E0,1.290066E0,1.0676861E-1,0E0,1.3699579E0,2.8937268E-1,9.358978E-1,7.822504E-1,4.8927426E-1,0E0,0E0,0E0,7.9075646E-1,6.0590863E-2,1.205089E-1,0E0,1.3271618E-1,2.9202938E-1,8.5187626E-1,0E0,0E0,0E0,3.7997055E-1,5.6432647E-1,0E0,0E0,6.2531315E-2,0E0,0E0,8.356428E-2,2.4022865E-1,1.86831E-1,0E0,0E0,2.8090572E-1,6.814578E-2,1.5337212E-1,8.461803E-2,0E0,0E0,6.0078144E-2,0E0,0E0,0E0,8.422899E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,21,21,22,22,23,23,27,27,28,28,31,31,34,34,35,35,36,36,39,39,40,40,41,41,42,42,45,45,49,49],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,36,38,-1,-1,-1,40,42,-1,-1,44,-1,-1,46,48,50,-1,-1,52,54,56,58,-1,-1,60,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0429407E3,1.2852292E7,1.4096699E9,7.7101436E3,7.339209E6,1.44E2,-1.7695631E-3,3.058642E3,1.0284004E2,4.1331047E-1,1.92E2,3.236383E1,-3.7001975E-2,-2.2230716E-2,-1.5392648E-2,2.6919386E-1,3.1E1,1.5029658E-1,1.2529651E-2,5.767668E0,9.170229E6,5.192243E6,4.9951877E-3,-1.1840204E-2,1.3281439E-2,2.889654E5,3.23643E6,2.6154373E-2,7.982736E-3,7.758838E2,1.0045847E-2,4.472478E-4,2.6704755E0,3.0811954E-2,1.7092716E9,4.8095267E-2,2.2578157E-2,2.1721223E3,6.52487E2,1.6036603E3,5.3040914E4,3.2386193E-3,-3.5839758E-3,1.15E2,2.3977773E-3,5.6934864E-3,2.7060233E-2,5.0491E4,1.2142594E-3,-9.368537E-3,-4.8429426E-3,-4.3158866E-3,4.4754446E-3,-1.0829633E-2,-2.2879404E-3,-1.3302342E-4,8.960095E-3,1.38288755E-2,8.679948E-3,4.575131E-3,1.8045012E-2],"split_indices":[2,1,66,9,7,60,55,0,0,60,62,43,8,64,0,0,0,47,3,46,0,62,56,37,0,0,0,36,1,0,0,60,0,0,49,46,40,0,0,63,4,63,36,0,0,3,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.62E2,3.28E2,1.34E2,6.9E1,2.59E2,1.24E2,1E1,6.6E1,3E0,2.29E2,3E1,1.01E2,2.3E1,6E0,4E0,1.4E1,5.2E1,2.25E2,4E0,1.4E1,1.6E1,4.7E1,5.4E1,2.1E1,2E0,4E0,2E0,1.66E2,5.9E1,2E0,2E0,1.1E1,3E0,3E0,4.4E1,2.2E1,3.2E1,1.3E1,8E0,1.57E2,9E0,4.4E1,1.5E1,6E0,5E0,4.1E1,3E0,2E0,2E1,3E1,2E0,1.03E2,5.4E1,2E0,7E0,5E0,3.9E1,3E0,1.2E1,2.3E1,1.8E1,2E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[7.6363346E-4,-1.5018706E-1,2.780872E-1,-3.1568587E-1,-9.1799445E-2,2.0873317E-1,6.7753965E-1,-3.7763423E-1,-2.5360242E-1,-1.09814405E-1,2.0042345E-1,2.5283432E-1,-3.635116E-1,7.686148E-1,3.550989E-4,-1.9332398E-2,-9.119753E-3,7.600268E-4,-1.3082895E-2,-1.9053496E-1,-6.8166725E-2,1.4798482E-2,5.4232106E-3,1.2760015E-1,3.533201E-1,-5.341756E-2,-3.5550345E-2,4.6261113E-2,4.113961E-1,-2.0155936E-1,2.7235253E-3,-9.7895384E-2,1.2699148E-1,1.8479685E-1,3.326928E-2,3.7022868E-1,4.626746E-3,-1.1375698E-2,1.2847234E-2,2.2752365E-2,7.434446E-3,-4.581842E-3,-2.2837459E-1,-5.056558E-2,-1.9137906E-1,1.0806247E-2,-5.6937395E-4,8.8502645E-2,2.551832E-1,-8.111852E-3,1.0471829E-1,3.0404726E-1,4.3330708E-1,-1.17767295E-2,-2.8381266E-3,3.7063452E-4,-4.803288E-3,-1.0301937E-2,-3.561357E-4,-3.2047562E-3,6.0626944E-3,1.5635742E-2,7.20415E-3,-4.0440817E-4,6.639485E-3,3.122654E-3,1.5831297E-2,6.851602E-3,2.1934647E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,29,31,-1,-1,33,35,37,-1,-1,39,41,-1,43,45,47,49,51,-1,-1,-1,-1,-1,-1,53,55,57,-1,-1,59,61,-1,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0428648E1,3.0245538E0,4.623601E0,2.1959972E-1,1.2591395E0,3.8269324E0,1.5206251E0,5.0703526E-2,1.6023111E-1,7.333367E-1,9.408778E-2,1.6915436E0,1.194123E0,9.851532E-1,0E0,0E0,0E0,0E0,0E0,2.1465492E-1,8.7109727E-1,0E0,0E0,3.33763E-1,2.9786205E-1,4.549834E-1,0E0,0E0,5.257547E-2,1.8742204E-1,0E0,5.610914E-1,2.6138124E-1,2.4174964E-1,3.722197E-1,1.6465187E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0054517E-1,2.4034335E-1,1.4650321E-1,0E0,0E0,1.0536048E-1,1.0480344E-1,0E0,6.695269E-2,1.5101576E-1,7.708597E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,19,19,20,20,23,23,24,24,25,25,28,28,29,29,31,31,32,32,33,33,34,34,35,35,42,42,43,43,44,44,47,47,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,30,32,-1,-1,34,36,38,-1,-1,40,42,-1,44,46,48,50,52,-1,-1,-1,-1,-1,-1,54,56,58,-1,-1,60,62,-1,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,7.7101436E3,4.90403E5,1.4777102E9,9.05929E2,1.92E2,9.7467445E4,1.3E1,6.505339E2,5.390361E6,2.5096E4,7.339209E6,5.192243E6,3.550989E-4,-1.9332398E-2,-9.119753E-3,7.600268E-4,-1.3082895E-2,3.01E2,2.251E3,1.4798482E-2,5.4232106E-3,1.3508742E7,1.773296E6,3.236383E1,-3.5550345E-2,4.6261113E-2,8.927E3,5.1698097E1,2.7235253E-3,2.3790412E7,1.6777562E7,5.5785873E1,3.672717E3,1.30817086E5,4.626746E-3,-1.1375698E-2,1.2847234E-2,2.2752365E-2,7.434446E-3,-4.581842E-3,3.4863858E6,7.581957E1,6.4065235E8,1.0806247E-2,-5.6937395E-4,2.0772269E8,1.14E2,-8.111852E-3,7.7444346E8,2E0,8.946923E2,-1.17767295E-2,-2.8381266E-3,3.7063452E-4,-4.803288E-3,-1.0301937E-2,-3.561357E-4,-3.2047562E-3,6.0626944E-3,1.5635742E-2,7.20415E-3,-4.0440817E-4,6.639485E-3,3.122654E-3,1.5831297E-2,6.851602E-3,2.1934647E-2],"split_indices":[2,1,60,9,7,66,8,41,10,60,53,2,55,37,0,0,0,0,0,8,0,0,0,53,11,64,0,0,0,56,0,53,53,64,4,36,0,0,0,0,0,0,58,62,7,0,0,5,3,0,7,8,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,3.15E2,1.71E2,8.1E1,2.34E2,1.47E2,2.4E1,3.8E1,4.3E1,2.21E2,1.3E1,1.37E2,1E1,2.1E1,3E0,3.4E1,4E0,2E0,4.1E1,7.4E1,1.47E2,5E0,8E0,6.2E1,7.5E1,6E0,4E0,1.3E1,8E0,7.1E1,3E0,1.28E2,1.9E1,3.8E1,2.4E1,7E1,5E0,4E0,2E0,6E0,2E0,1.5E1,5.6E1,8.6E1,4.2E1,1.1E1,8E0,1.7E1,2.1E1,6E0,1.8E1,3.7E1,3.3E1,5.2E1,4E0,3.9E1,4.7E1,3.8E1,4E0,3E0,1.4E1,1.2E1,9E0,4E0,1.4E1,3E0,3.4E1,2E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[9.183193E-3,-1.3674265E-1,2.6512635E-1,-2.9895145E-1,-8.5772514E-2,3.0931225E-1,-2.6510525E-1,-3.1218484E-1,3.6145851E-4,-1.0540455E-1,1.7217973E-1,2.3678607E-1,5.8564836E-1,-2.2897026E-2,-3.4155663E-2,-1.9312665E-2,-1.3552013E-2,-1.3845202E-1,-1.8622235E-3,-1.7455467E-3,2.192454E-1,1.2550806E-1,3.3037424E-1,-3.451065E-2,6.723776E-1,-1.5537785E-1,1.05271265E-2,-1.773304E-1,-9.1211684E-2,-2.5878107E-2,7.9974765E-3,1.46438675E-2,6.0018557E-3,1.5865685E-1,-3.9591674E-2,2.5176746E-1,4.0178767E-1,8.187974E-3,-1.1008585E-2,4.161031E-2,2.5382467E-2,-9.870362E-4,-1.23472065E-2,-2.3094527E-1,-1.07621275E-1,-1.6984096E-2,-1.4649637E-1,-1.4783126E-1,2.208303E-3,2.1727003E-1,7.785516E-2,-1.0348376E-2,4.3407143E-3,4.386052E-3,2.750641E-1,4.346665E-1,8.357654E-3,-1.3723863E-2,-7.0412983E-3,-3.8526058E-3,-1.0674303E-2,-4.6529477E-3,4.956708E-3,-6.1273004E-4,-9.629117E-3,-1.9155061E-4,-9.23868E-3,-3.5600537E-3,1.807586E-3,1.278109E-2,6.57721E-3,-3.3652813E-3,6.551146E-3,3.1415937E-3,1.409413E-2,1.4093986E-2,2.5346996E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,-1,31,33,35,37,39,41,-1,43,45,47,-1,-1,-1,49,51,53,55,-1,-1,-1,-1,-1,-1,57,59,61,63,65,67,69,71,-1,-1,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.800099E1,2.5125747E0,4.1820345E0,2.9822016E-1,1.2082181E0,3.1080656E0,1.4468775E0,9.593916E-2,0E0,7.4911904E-1,1.7404819E-1,1.3123951E0,1.7981386E0,3.4808373E-1,0E0,0E0,0E0,2.8870583E-1,2.181323E-1,0E0,7.1078956E-2,3.4012598E-1,2.9179382E-1,2.2886342E-1,3.5142326E-1,8.6204514E-2,0E0,3.1151533E-1,3.1349283E-1,1.6473731E-1,0E0,0E0,0E0,2.246226E-1,2.6025736E-1,1.1153889E-1,1.9215822E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7345905E-1,1.1401394E-1,3.2228127E-1,2.9152554E-1,5.1807046E-2,1.0636101E-1,6.9600224E-2,1.9694248E-1,0E0,0E0,0E0,6.23219E-2,2.1463394E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,33,33,34,34,35,35,36,36,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,-1,32,34,36,38,40,42,-1,44,46,48,-1,-1,-1,50,52,54,56,-1,-1,-1,-1,-1,-1,58,60,62,64,66,68,70,72,-1,-1,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,9.05929E2,1.2852292E7,1.4777102E9,6.6512886E3,7.339209E6,1.44E2,3.6145851E-4,9.7467445E4,2.0532622E3,2.5096E4,5.3140095E1,3.236383E1,-3.4155663E-2,-1.9312665E-2,-1.3552013E-2,2.09299E6,1.4998456E5,-1.7455467E-3,6.088569E6,2.4722598E7,1.30817086E5,1.0014992E8,1.681178E6,2.2201815E9,1.05271265E-2,3.006846E6,1.7377544E7,1.6359736E3,7.9974765E-3,1.46438675E-2,6.0018557E-3,5.1063637E1,6.426437E5,1.4040858E-2,5.543096E1,8.187974E-3,-1.1008585E-2,4.161031E-2,2.5382467E-2,-9.870362E-4,-1.23472065E-2,1.6415557E9,6.59701E5,5.16152E2,1.4E4,1.4590734E8,7.55E2,1.14E2,7.10711E8,-1.0348376E-2,4.3407143E-3,4.386052E-3,2E0,1.2511627E7,8.357654E-3,-1.3723863E-2,-7.0412983E-3,-3.8526058E-3,-1.0674303E-2,-4.6529477E-3,4.956708E-3,-6.1273004E-4,-9.629117E-3,-1.9155061E-4,-9.23868E-3,-3.5600537E-3,1.807586E-3,1.278109E-2,6.57721E-3,-3.3652813E-3,6.551146E-3,3.1415937E-3,1.409413E-2,1.4093986E-2,2.5346996E-2],"split_indices":[2,1,66,9,7,60,55,0,0,41,4,2,64,64,0,0,0,1,36,0,53,53,36,1,37,7,0,9,53,63,0,0,0,66,55,65,64,0,0,0,0,0,0,5,37,60,11,5,0,3,7,0,0,0,8,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.8E2,3.06E2,1.74E2,7.2E1,2.34E2,1.61E2,1.3E1,6.9E1,3E0,2.18E2,1.6E1,1.29E2,3.2E1,9E0,4E0,1.8E1,5.1E1,1.65E2,5.3E1,3E0,1.3E1,6E1,6.9E1,4E0,2.8E1,6E0,3E0,8.9E1,7.6E1,4.7E1,6E0,6E0,7E0,5E1,1E1,3.5E1,3.4E1,2E0,2E0,1.1E1,1.7E1,3E0,3E0,4.9E1,4E1,3.3E1,4.3E1,8E0,3.9E1,2.8E1,2.2E1,4E0,6E0,5E0,3E1,2.9E1,5E0,3E1,1.9E1,3.3E1,7E0,2E1,1.3E1,1.2E1,3.1E1,2E0,6E0,1.2E1,2.7E1,1.7E1,1.1E1,6E0,1.6E1,2E0,2.8E1,1.2E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-4.421483E-3,-1.2896904E-1,2.4164438E-1,-2.7579647E-1,-8.169758E-2,2.849048E-1,-4.4873086E-1,-2.8720698E-1,3.4816493E-4,-1.1848838E-1,3.896393E-2,2.0969656E-1,5.346216E-1,-9.329073E-3,-3.2815695E-2,-9.883062E-3,-1.5987303E-2,-1.6720478E-1,-4.449573E-2,1.7307281E-2,1.2998409E-2,1.3589135E-1,3.4859216E-1,8.562697E-1,3.7171158E-1,-8.902056E-2,-1.9834301E-1,1.36739025E-2,-1.7109968E-1,-3.617651E-2,9.674627E-2,4.714052E-2,1.7537554E-1,2.5100696E-1,4.1404983E-1,1.7227495E-2,4.557931E-2,4.6408433E-1,-1.1242987E-2,-1.4240308E-1,-1.2692115E-4,-5.480858E-4,-2.0804682E-1,-1.710902E-1,6.062114E-2,-2.6299763E-1,-4.8267934E-3,-9.2432894E-2,1.0140451E-2,8.473696E-3,4.0947225E-2,8.92066E-2,-1.4737834E-1,-2.1002663E-3,1.8878785E-1,6.681958E-3,1.5794149E-2,8.1651965E-3,2.2430481E-2,1.14952605E-2,2.462246E-2,-5.416921E-4,-8.467105E-3,-1.0893535E-2,-4.5515504E-3,-1.7348212E-3,-1.1945261E-2,1.2182035E-3,7.403981E-3,-1.532842E-2,-2.1146399E-3,2.473845E-3,-6.49063E-3,-2.342514E-3,3.2873403E-3,-6.059336E-4,5.594255E-3,-3.9538746E-3,7.05673E-3,-6.7795085E-4,-1.1367515E-2,-5.004864E-4,9.865629E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,-1,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,-1,59,-1,61,-1,-1,63,65,67,69,-1,71,73,-1,75,77,79,-1,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4864265E1,2.212389E0,4.9959927E0,2.538228E-1,1.0922289E0,2.7801294E0,4.425323E-1,1.8363667E-1,0E0,6.7023516E-1,2.7941215E-1,1.1827755E0,1.5157042E0,0E0,0E0,0E0,0E0,2.5744414E-1,5.6204194E-1,2.3292965E-1,0E0,2.7207303E-1,1.5796614E-1,7.997799E-2,1.4942484E0,1.5650672E-1,1.4297318E-1,4.701024E-1,1.3439536E-1,8.692293E-2,8.62806E-2,2.2589591E-1,1.6904986E-1,1.1105609E-1,1.7066455E-1,0E0,0E0,7.438946E-2,0E0,8.1558615E-2,0E0,0E0,1.02808475E-1,9.961003E-2,1.3527103E-1,9.553301E-2,0E0,9.224114E-2,6.5801784E-2,0E0,5.5983055E-2,2.1595755E-1,5.317332E-2,0E0,1.3027716E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,39,39,42,42,43,43,44,44,45,45,47,47,48,48,50,50,51,51,52,52,54,54],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,-1,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,-1,60,-1,62,-1,-1,64,66,68,70,-1,72,74,-1,76,78,80,-1,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0429407E3,1.2852292E7,7.10711E8,6.0495757E3,1.805424E-1,4.204762E1,3.4816493E-4,2.735322E6,3.4540886E3,4.1331047E-1,1.417988E6,-9.329073E-3,-3.2815695E-2,-9.883062E-3,-1.5987303E-2,7.477264E6,2.7394568E7,2.9124984E7,1.2998409E-2,1.3428E4,1.30817086E5,1.082556E7,2.0531621E4,5.336051E6,6.453125E0,2.4094029E2,5.5292703E9,1.7933134E5,6.404003E7,1E0,4.56E2,3.3361957E0,4.653159E9,1.7227495E-2,4.557931E-2,6.841449E1,-1.1242987E-2,8.840007E7,-1.2692115E-4,-5.480858E-4,1.7212875E5,5.24E0,1.656968E6,1.4171124E3,-4.8267934E-3,1.7608592E-4,1.79E2,8.473696E-3,1.10945766E5,7.10711E8,2.56E2,-2.1002663E-3,5.767668E0,6.681958E-3,1.5794149E-2,8.1651965E-3,2.2430481E-2,1.14952605E-2,2.462246E-2,-5.416921E-4,-8.467105E-3,-1.0893535E-2,-4.5515504E-3,-1.7348212E-3,-1.1945261E-2,1.2182035E-3,7.403981E-3,-1.532842E-2,-2.1146399E-3,2.473845E-3,-6.49063E-3,-2.342514E-3,3.2873403E-3,-6.059336E-4,5.594255E-3,-3.9538746E-3,7.05673E-3,-6.7795085E-4,-1.1367515E-2,-5.004864E-4,9.865629E-3],"split_indices":[2,1,66,9,7,60,46,61,0,1,4,43,37,0,0,0,0,53,53,9,0,2,36,1,4,53,55,60,5,55,9,79,10,61,5,0,0,64,0,5,0,0,36,64,9,60,0,46,8,0,36,7,0,0,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,3.21E2,1.62E2,7.7E1,2.44E2,1.53E2,9E0,7.4E1,3E0,1.87E2,5.7E1,1.19E2,3.4E1,5E0,4E0,2.5E1,4.9E1,1.12E2,7.5E1,5.3E1,4E0,7.9E1,4E1,1E1,2.4E1,3.3E1,7.9E1,5.2E1,2.3E1,3.2E1,2.1E1,2.5E1,5.4E1,1.8E1,2.2E1,2E0,8E0,2.1E1,3E0,2E1,1.3E1,4E0,7.5E1,1E1,4.2E1,9E0,1.4E1,1.4E1,1.8E1,8E0,1.3E1,2.1E1,4E0,3E0,5.1E1,8E0,1E1,4E0,1.8E1,4E0,1.7E1,4E0,1.6E1,6.6E1,9E0,4E0,6E0,3.1E1,1.1E1,7E0,2E0,3E0,1.1E1,9E0,9E0,8E0,5E0,5E0,1.6E1,2E0,2E0,3E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[5.3353526E-4,-1.3272116E-1,2.4356888E-1,-2.64707E-1,-8.707298E-2,1.6786568E-1,5.140735E-1,-2.7510542E-1,3.353602E-4,-2.2338811E-2,-1.454647E-1,2.0746039E-1,-3.1173283E-1,-1.0281123E-2,5.601042E-1,-1.7844781E-2,-1.2444739E-2,-5.430497E-2,1.1525927E-1,-1.8246764E-1,-5.497193E-2,1.0044947E-1,3.0498618E-1,-5.4325557E-1,7.474558E-2,3.8247578E-2,4.3217745E-1,-1.0753345E-1,3.8495418E-2,-2.063093E-3,9.3902005E-3,-1.5811374E-2,-1.6466415E-1,-1.0345065E-1,1.7420488E-3,1.441419E-1,-2.6256029E-2,3.2478464E-1,4.3589342E-3,-3.1528298E-2,-9.614433E-3,1.2806188E-2,-6.6971127E-3,1.0691915E-3,4.6691424E-1,-1.6125053E-1,-3.1086076E-2,1.1408478E-2,5.939806E-3,-1.9067657E-1,-2.655637E-2,2.8017E-5,-8.475732E-3,1.23961024E-1,1.4165837E-2,-7.3184106E-3,3.6658624E-3,2.4097557E-1,3.7263444E-1,2.4084378E-2,6.9529135E-3,-1.530091E-4,-9.194882E-3,-3.1816987E-3,2.6021411E-3,-2.788484E-3,3.7622645E-3,-1.0295177E-2,-2.0143122E-3,-2.7241975E-3,4.8194057E-3,7.5104507E-3,2.1976565E-3,1.2779177E-2,5.028915E-4,5.6389133E-3,1.8834474E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,31,33,35,37,39,41,-1,43,45,47,-1,-1,-1,49,51,-1,53,55,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,65,67,69,-1,-1,71,-1,-1,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6095585E1,1.9109483E0,3.5178442E0,2.334199E-1,9.03936E-1,2.7042217E0,1.3833418E0,5.72834E-2,0E0,5.120846E-1,4.1188025E-1,1.31565E0,1.02487E0,0E0,6.753626E-1,0E0,0E0,4.6846414E-1,2.718908E-1,1.6867161E-1,1.7038536E-1,3.5255706E-1,2.577858E-1,1.1618233E-1,2.3260455E-1,0E0,3.4619713E-1,2.4137914E-1,2.1983679E-1,0E0,0E0,0E0,2.859304E-1,1.8054065E-1,0E0,1.0146546E-1,2.2233498E-1,1.5482903E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1143303E-1,1.4470309E-1,7.738174E-2,0E0,1.4216256E-1,1.8732762E-1,5.613369E-2,0E0,0E0,9.248483E-2,0E0,0E0,0E0,1.10477686E-1,6.4866066E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,32,32,33,33,35,35,36,36,37,37,44,44,45,45,46,46,48,48,49,49,50,50,53,53,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,32,34,36,38,40,42,-1,44,46,48,-1,-1,-1,50,52,-1,54,56,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,66,68,70,-1,-1,72,-1,-1,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,6.204522E3,1.2852292E7,1.2885352E7,8.6206274E2,5.0314346E1,2.571E3,3.353602E-4,2.972948E6,7.194E3,2.5096E4,3.5226266E0,-1.0281123E-2,1.417988E6,-1.7844781E-2,-1.2444739E-2,7.164074E8,5.0713895E2,3.2471478E2,5.0363316E5,1.7458724E7,1.773296E6,3.1E1,8.494E3,3.8247578E-2,8.713209E10,8.037592E2,2.0673077E0,-2.063093E-3,9.3902005E-3,-1.5811374E-2,7.2008003E9,2.1528782E3,1.7420488E-3,6.668461E7,5.139744E-2,5.5835126E9,4.3589342E-3,-3.1528298E-2,-9.614433E-3,1.2806188E-2,-6.6971127E-3,1.0691915E-3,1.1986E4,8.840007E7,3.4015296E8,1.1408478E-2,1.5403295E6,6.4065235E8,2.1803274E3,2.8017E-5,-8.475732E-3,2.3961474E7,1.4165837E-2,-7.3184106E-3,3.6658624E-3,1.9703683E9,8.946923E2,2.4084378E-2,6.9529135E-3,-1.530091E-4,-9.194882E-3,-3.1816987E-3,2.6021411E-3,-2.788484E-3,3.7622645E-3,-1.0295177E-2,-2.0143122E-3,-2.7241975E-3,4.8194057E-3,7.5104507E-3,2.1976565E-3,1.2779177E-2,5.028915E-4,5.6389133E-3,1.8834474E-2],"split_indices":[2,1,60,9,53,66,64,2,0,1,2,2,62,0,37,0,0,7,60,60,55,53,11,3,0,0,39,60,66,0,0,0,5,55,0,9,46,12,0,0,0,0,0,0,0,5,40,0,58,7,4,0,0,9,0,0,0,40,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.95E2,3.2E2,1.75E2,8.1E1,2.39E2,1.38E2,3.7E1,7.8E1,3E0,1.14E2,1.25E2,1.28E2,1E1,2E0,3.5E1,1.2E1,6.6E1,9.3E1,2.1E1,8.8E1,3.7E1,6.2E1,6.6E1,6E0,4E0,1.1E1,2.4E1,5.9E1,3.4E1,7E0,1.4E1,8E0,8E1,2.4E1,1.3E1,4.6E1,1.6E1,6E1,6E0,4E0,2E0,2E0,2E0,2E0,2.2E1,3.4E1,2.5E1,4E0,3E1,6.7E1,1.3E1,1E1,1.4E1,4.2E1,4E0,7E0,9E0,2.4E1,3.6E1,2E1,2E0,5E0,2.9E1,1.8E1,7E0,1.6E1,1.4E1,5.9E1,8E0,1.1E1,2E0,3E1,1.2E1,2.2E1,2E0,2E0,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[4.925768E-3,-1.1608739E-1,2.4019873E-1,-1.3216697E-1,4.4530335E-1,1.8033585E-1,5.7689625E-1,-2.4413873E-1,-9.229895E-2,8.384773E-3,3.0922206E-2,2.0323656E-1,-1.730341E-1,2.0108458E-2,3.4796618E-2,-2.5661713E-1,-2.7655018E-4,-8.44498E-2,-3.2014377E-2,1.3364695E-1,3.169652E-1,6.8910643E-3,-1.3863129E-2,-2.9533446E-1,-2.001035E-1,-1.1509539E-1,2.2963438E-2,-1.1449612E-2,1.4559427E-1,2.2399422E-2,2.868353E-1,-1.5802182E-2,-9.2084E-3,-1.0341134E-2,-9.070624E-4,-1.3380735E-1,-2.3255354E-2,1.0893748E-1,-3.6017798E-2,1.0745542E-1,1.2232547E-2,5.6139E-3,3.015776E-1,-7.6244012E-3,-8.4239483E-4,-4.113551E-3,4.639237E-3,-1.237799E-3,6.9632228E-3,-4.8736446E-3,3.5598641E-3,2.547998E-3,7.3557543E-3,1.5267735E-2,3.955588E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,-1,25,-1,27,29,-1,-1,31,33,35,37,-1,39,-1,41,-1,-1,-1,-1,43,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3580139E1,2.9023352E0,3.1611853E0,1.3479428E0,3.495742E-1,1.1600137E0,2.4030828E-1,2.366395E-1,9.480089E-1,0E0,0E0,9.9441624E-1,3.4919244E-1,0E0,0E0,1.0719442E-1,0E0,7.465769E-1,0E0,3.9804685E-1,7.538223E-2,0E0,0E0,6.391597E-2,6.261504E-2,2.9934525E-1,2.6253733E-1,0E0,2.940178E-1,0E0,7.473612E-2,0E0,0E0,0E0,0E0,3.6630726E-1,2.2944675E-1,9.672825E-2,2.2056058E-1,1.3622642E-1,0E0,0E0,5.8987617E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,17,17,19,19,20,20,23,23,24,24,25,25,26,26,28,28,30,30,35,35,36,36,37,37,38,38,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,-1,26,-1,28,30,-1,-1,32,34,36,38,-1,40,-1,42,-1,-1,-1,-1,44,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1576422E9,3.8486113E3,7.765071E0,7.41718E5,7.0331914E3,1.4609149E4,1.6941456E1,1.2852292E7,3.764913E7,8.384773E-3,3.0922206E-2,4.1331047E-1,1.479E4,2.0108458E-2,3.4796618E-2,1.425966E6,-2.7655018E-4,4.2681945E6,-3.2014377E-2,3.1095595E0,2.0159504E1,6.8910643E-3,-1.3863129E-2,9.405346E4,8.332E3,4.9125927E11,4.7232704E1,-1.1449612E-2,1.2019813E0,2.2399422E-2,2E0,-1.5802182E-2,-9.2084E-3,-1.0341134E-2,-9.070624E-4,7.164074E8,8.185918E0,2.3188192E8,9.242727E1,2.9920125E3,1.2232547E-2,5.6139E-3,1.5953E4,-7.6244012E-3,-8.4239483E-4,-4.113551E-3,4.639237E-3,-1.237799E-3,6.9632228E-3,-4.8736446E-3,3.5598641E-3,2.547998E-3,7.3557543E-3,1.5267735E-2,3.955588E-3],"split_indices":[7,60,43,1,4,4,43,9,56,0,0,43,9,0,0,9,0,56,0,62,64,0,0,36,2,39,66,0,50,0,8,0,0,0,0,7,62,5,66,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.75E2,3.14E2,1.61E2,3.06E2,8E0,1.38E2,2.3E1,7.9E1,2.27E2,4E0,4E0,1.3E2,8E0,1.2E1,1.1E1,7.5E1,4E0,2.25E2,2E0,8.2E1,4.8E1,2E0,6E0,4.2E1,3.3E1,1.75E2,5E1,2E0,8E1,6E0,4.2E1,3.2E1,1E1,3.1E1,2E0,1.45E2,3E1,2E1,3E1,6E1,2E1,4E0,3.8E1,1.22E2,2.3E1,2E1,1E1,4E0,1.6E1,1.9E1,1.1E1,2.7E1,3.3E1,3.6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.19126635E-2,-9.279796E-2,2.954445E-1,-1.2310231E-1,1.1129323E-1,3.2751212E-1,-1.6138825E-1,-1.8866673E-1,-4.580346E-2,2.9806519E-2,1.879411E-1,2.8557274E-1,3.3232428E-2,-2.3216406E-2,7.353618E-2,-2.090279E-1,-3.3651043E-2,-6.285999E-2,2.4135472E-1,-4.157374E-3,3.6894276E-3,1.3086963E-2,5.232415E-3,3.0156294E-1,-1.00806225E-2,-6.3097463E-3,1.0141516E-2,-1.3878654E-2,-1.7856227E-1,6.1026216E-3,-3.969834E-3,3.5812212E-3,-1.3793203E-1,3.2828883E-3,1.9567955E-2,2.1877524E-1,4.0708488E-1,-2.872443E-3,-1.866173E-1,-1.699129E-2,5.75421E-3,-1.7621966E-1,-6.216129E-3,1.5774088E-1,3.0629733E-1,2.6508942E-2,3.223238E-1,-9.373386E-3,9.248504E-4,1.0306358E-3,-3.7496458E-3,-1.2746044E-2,-6.1911056E-3,4.608676E-3,-3.222454E-3,8.454991E-3,-4.944191E-4,1.6323885E-2,4.4226097E-3,1.254539E-2,2.0298626E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,-1,-1,-1,35,-1,-1,-1,-1,37,-1,-1,39,41,-1,-1,43,45,-1,47,49,-1,51,53,55,57,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3981149E1,2.1412892E0,1.9133501E0,1.511775E0,2.7545834E-1,1.554059E0,6.9456553E-1,5.042863E-1,6.959827E-1,1.2610033E-1,1.1900675E-1,9.1015434E-1,0E0,0E0,1.8829471E-1,2.717867E-1,1.596375E-1,6.5889376E-1,1.9328699E-1,0E0,0E0,0E0,0E0,8.282118E-1,0E0,0E0,0E0,0E0,8.9585304E-2,0E0,0E0,1.6815469E-1,3.1157613E-1,0E0,0E0,2.7753758E-1,3.3490324E-1,0E0,9.3780756E-2,1.3949037E-1,0E0,1.6794193E-1,9.559456E-2,9.53114E-2,1.078341E-1,0E0,7.6854944E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,23,23,28,28,31,31,32,32,35,35,36,36,38,38,39,39,41,41,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,-1,-1,-1,36,-1,-1,-1,-1,38,-1,-1,40,42,-1,-1,44,46,-1,48,50,-1,52,54,56,58,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7267E4,3.44E2,1.0429407E3,1.732734E6,1.4096699E9,1.6828056E1,2.8E1,7.164074E8,2.9034685E3,1.8374788E7,6.088569E6,5.37E2,3.3232428E-2,-2.3216406E-2,2.9648642E1,4.48149E5,4.45E3,1.8071064E7,1.571654E7,-4.157374E-3,3.6894276E-3,1.3086963E-2,5.232415E-3,9.3058E4,-1.00806225E-2,-6.3097463E-3,1.0141516E-2,-1.3878654E-2,1.4388227E4,6.1026216E-3,-3.969834E-3,3.0224E4,1.530655E3,3.2828883E-3,1.9567955E-2,1.4512005E5,9.6726194E-2,-2.872443E-3,2.9063344E5,2.9490616E-2,5.75421E-3,2.4851996E7,3.9E1,4.17E2,1.8105635E10,2.6508942E-2,1.30817086E5,-9.373386E-3,9.248504E-4,1.0306358E-3,-3.7496458E-3,-1.2746044E-2,-6.1911056E-3,4.608676E-3,-3.222454E-3,8.454991E-3,-4.944191E-4,1.6323885E-2,4.4226097E-3,1.254539E-2,2.0298626E-2],"split_indices":[2,8,66,1,7,43,3,7,60,12,53,3,0,0,64,1,2,53,1,0,0,0,0,2,0,0,0,0,36,0,0,10,60,0,0,36,65,0,36,65,0,53,8,3,5,0,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.69E2,3.43E2,1.26E2,2.99E2,4.4E1,1.18E2,8E0,1.61E2,1.38E2,2.2E1,2.2E1,1.07E2,1.1E1,3E0,5E0,1.42E2,1.9E1,1.31E2,7E0,6E0,1.6E1,1E1,1.2E1,1.04E2,3E0,2E0,3E0,3.9E1,1.03E2,4E0,1.5E1,7E1,6.1E1,4E0,3E0,6E1,4.4E1,7E0,9.6E1,6E1,1E1,4.7E1,1.4E1,3.7E1,2.3E1,1.5E1,2.9E1,9.4E1,2E0,3.7E1,2.3E1,1.6E1,3.1E1,5E0,9E0,3.4E1,3E0,2E1,3E0,1.9E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.628735E-3,-1.1869303E-1,2.271643E-1,-2.2609897E-1,-6.8276435E-2,1.702172E-1,4.962117E-1,-2.4724403E-1,-1.878502E-3,-8.470027E-2,1.351299E-1,1.9045474E-1,-2.4439485E-1,9.29886E-2,5.5700785E-1,-2.5788143E-1,-4.3562977E-3,-1.4692818E-1,-4.3376133E-2,-2.147673E-3,1.7978919E-1,2.937971E-1,1.2620868E-1,1.389006E-3,-2.2362525E-2,-2.3778258E-3,9.977932E-3,2.2081092E-2,4.0209208E-2,-5.994863E-3,-2.6710224E-1,-7.0480466E-2,-1.8680033E-1,-7.85297E-2,1.2273939E-1,1.2390647E-2,3.9831754E-3,1.8589844E-1,4.0743735E-1,1.8305786E-1,2.6331658E-2,-1.3728604E-2,-7.629324E-3,-1.1241103E-1,-1.006306E-3,-1.3884431E-1,-2.800291E-1,-4.059652E-2,-1.24161415E-1,8.106473E-3,-5.507231E-4,1.5587297E-3,2.357558E-1,4.4436067E-1,4.1870277E-5,1.9960625E-1,-9.33583E-4,-9.146845E-3,7.265723E-2,-7.0428224E-3,4.5652012E-4,-7.8088087E-3,6.4611337E-3,-1.55622475E-2,-6.152955E-4,-2.669795E-3,5.33299E-3,-2.8708018E-4,-6.9972435E-3,1.3419017E-2,4.6802196E-3,3.1820673E-2,1.7794346E-2,8.781245E-3,2.2641443E-2,4.988731E-3,-9.505447E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,41,43,45,47,49,-1,-1,51,53,55,57,-1,-1,59,-1,61,63,65,67,-1,-1,-1,69,71,-1,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3007729E1,1.6361504E0,2.5048647E0,3.7570477E-1,7.096547E-1,1.2296376E0,6.5912294E-1,1.23693466E-1,0E0,4.9250054E-1,1.3572463E-1,8.592315E-1,4.1690788E-1,8.8018544E-2,3.3238697E-1,6.86512E-2,0E0,2.2154069E-1,6.98706E-1,0E0,7.2861254E-2,5.496006E-1,4.778036E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.39348E-2,5.634898E-2,1.8466353E-1,1.6459322E-1,1.2161413E-1,0E0,0E0,2.0764899E-1,3.5991883E-1,1.843952E-1,3.2909036E-1,0E0,0E0,5.831285E-2,0E0,2.141177E-1,1.5571785E-1,1.1965098E-1,9.560239E-2,0E0,0E0,0E0,8.551562E-2,1.312542E-1,0E0,1.8649173E-1,0E0,0E0,2.2890523E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,20,20,21,21,22,22,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,43,43,45,45,46,46,47,47,48,48,52,52,53,53,55,55,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,42,44,46,48,50,-1,-1,52,54,56,58,-1,-1,60,-1,62,64,66,68,-1,-1,-1,70,72,-1,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,9.50639E5,4.026531E9,1.2852292E7,1.4096699E9,9.533205E6,2.6049583E0,2.4752695E6,-1.878502E-3,7.3123126E2,2.0532622E3,1.30569E5,1.618E3,6.001104E9,5.01E2,7.547656E1,-4.3562977E-3,1.0428788E7,2.194E3,-2.147673E-3,6.088569E6,1.6799316E7,1.8071064E7,1.389006E-3,-2.2362525E-2,-2.3778258E-3,9.977932E-3,2.2081092E-2,4.0209208E-2,-5.994863E-3,9.703059E6,1.523794E6,9E1,1.7458724E7,2.6957834E7,1.2390647E-2,3.9831754E-3,5.274031E6,9.705148E-1,5.37E2,2.3808822E-2,-1.3728604E-2,-7.629324E-3,4.458306E1,-1.006306E-3,4.75E2,5.853265E8,2.327793E9,7.3938284E0,8.106473E-3,-5.507231E-4,1.5587297E-3,2.3886503E5,3.7137297E3,4.1870277E-5,1.0705674E4,-9.33583E-4,-9.146845E-3,4.31E2,-7.0428224E-3,4.5652012E-4,-7.8088087E-3,6.4611337E-3,-1.55622475E-2,-6.152955E-4,-2.669795E-3,5.33299E-3,-2.8708018E-4,-6.9972435E-3,1.3419017E-2,4.6802196E-3,3.1820673E-2,1.7794346E-2,8.781245E-3,2.2641443E-2,4.988731E-3,-9.505447E-3],"split_indices":[2,1,7,9,7,55,61,58,0,60,4,9,0,7,3,62,0,53,0,0,53,1,53,0,0,0,0,0,0,0,9,1,8,53,53,0,0,1,42,3,46,0,0,64,0,3,7,5,55,0,0,0,41,40,0,4,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.72E2,3.04E2,1.68E2,9.6E1,2.08E2,1.4E2,2.8E1,8.6E1,1E1,1.93E2,1.5E1,1.34E2,6E0,4E0,2.4E1,8E1,6E0,7.6E1,1.17E2,3E0,1.2E1,5E1,8.4E1,3E0,3E0,2E0,2E0,1.9E1,5E0,6E0,7.4E1,2.7E1,4.9E1,9.7E1,2E1,6E0,6E0,2.7E1,2.3E1,5.3E1,3.1E1,6.5E1,9E0,1.4E1,1.3E1,3.4E1,1.5E1,5.4E1,4.3E1,1.5E1,5E0,7E0,2E1,2.1E1,2E0,4.9E1,4E0,5E0,2.6E1,1.1E1,3E0,3.2E1,2E0,1.3E1,2E0,5E1,4E0,6E0,3.7E1,1.5E1,5E0,4E0,1.7E1,4.7E1,2E0,2.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[2.5215542E-3,-9.2993066E-2,2.499372E-1,-1.8846478E-1,-3.203577E-2,1.7055792E-1,4.6204647E-1,-2.1533139E-1,-3.7503417E-4,-1.2799771E-1,8.452004E-3,1.9513835E-1,-2.1149827E-2,2.656742E-1,6.663606E-1,-2.4233313E-1,-1.4334747E-1,9.434103E-3,-2.7177902E-3,-1.4889005E-1,4.1608796E-3,6.371696E-2,-6.27371E-2,7.592274E-2,2.320488E-1,1.6916454E-2,-5.7809823E-3,1.60069E-2,3.8800057E-2,-2.6111147E-1,-3.1963775E-3,-2.6997093E-3,-1.7934307E-1,-9.021015E-2,-1.9478297E-1,1.12482905E-1,-3.5892196E-3,-7.142699E-2,6.6138627E-3,6.732355E-3,8.9054194E-4,1.7177152E-2,1.7302714E-1,-1.3302957E-2,-3.466515E-3,-9.751397E-3,-1.813096E-3,-9.625735E-3,-2.000695E-3,-1.2548707E-1,-2.8826958E-1,-4.548091E-3,1.3373068E-1,-7.326603E-2,7.449599E-2,-1.11166336E-1,-3.8730145E-2,1.9880904E-1,-6.38241E-3,-7.413687E-3,3.037117E-3,-1.669354E-2,-9.006211E-4,3.7138313E-3,9.360781E-3,-6.5320954E-3,-8.614586E-4,6.1415574E-3,-7.7894767E-4,-3.3865252E-3,-9.205111E-3,-9.333021E-5,-7.106121E-3,6.686884E-3,1.2229531E-2,3.6791982E-3,-4.226925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,33,-1,35,37,39,41,-1,-1,-1,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,59,61,-1,63,65,67,69,71,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0775717E1,1.9086428E0,2.059504E0,6.4674616E-1,7.859094E-1,1.4747577E0,1.1871004E0,1.8295574E-1,1.9103841E-1,2.7944452E-1,5.6659704E-1,3.8280964E-1,0E0,6.1846805E-1,4.208126E-1,2.4911165E-1,9.500861E-2,0E0,0E0,1.3054311E-1,0E0,2.66266E-1,1.1781563E-1,7.90306E-2,4.2087078E-1,0E0,0E0,0E0,0E0,1.201458E-1,0E0,0E0,5.652392E-2,1.282709E-1,1.5817308E-1,2.1735203E-1,1.9588353E-1,7.495493E-2,0E0,0E0,0E0,0E0,2.232939E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.00853324E-1,1.6263574E-1,0E0,1.2808186E-1,6.1089583E-2,7.995314E-2,7.406205E-2,1.3658106E-1,9.864831E-2,5.194897E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,21,21,22,22,23,23,24,24,29,29,32,32,33,33,34,34,35,35,36,36,37,37,42,42,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,34,-1,36,38,40,42,-1,-1,-1,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,60,62,-1,64,66,68,70,72,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,1.204977E6,6.0495757E3,7.164074E8,6.2651404E2,4.673502E7,1.9803026E1,3.006846E6,2.0673077E0,2.757849E1,1.423795E7,2.0663544E9,-2.1149827E-2,2.0531621E4,9.430977E3,1.7212875E5,1.24E2,9.434103E-3,-2.7177902E-3,1.0697867E3,4.1608796E-3,4.7232704E1,3.2991206E3,6.4731E5,9.362063E6,1.6916454E-2,-5.7809823E-3,1.60069E-2,3.8800057E-2,1.7626338E6,-3.1963775E-3,-2.6997093E-3,1.279709E5,1.964127E6,1.7377544E7,1.7102936E6,1.0336E4,6.844E3,6.6138627E-3,6.732355E-3,8.9054194E-4,1.7177152E-2,2.23871E5,-1.3302957E-2,-3.466515E-3,-9.751397E-3,-1.813096E-3,-9.625735E-3,-2.000695E-3,3.64E2,4.838871E8,-4.548091E-3,1.0643513E3,7.787589E7,1.14E2,1E0,9.778E3,7.757358E9,1E0,-7.413687E-3,3.037117E-3,-1.669354E-2,-9.006211E-4,3.7138313E-3,9.360781E-3,-6.5320954E-3,-8.614586E-4,6.1415574E-3,-7.7894767E-4,-3.3865252E-3,-9.205111E-3,-9.333021E-5,-7.106121E-3,6.686884E-3,1.2229531E-2,3.6791982E-3,-4.226925E-3],"split_indices":[2,1,60,7,60,56,62,9,66,61,53,12,0,4,4,36,10,0,0,41,0,66,60,37,53,0,0,0,0,58,0,0,41,1,53,53,10,2,0,0,0,0,10,0,0,0,0,0,0,3,7,0,60,40,3,79,2,5,13,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.54E2,3.28E2,1.26E2,1.27E2,2.01E2,9.3E1,3.3E1,1.11E2,1.6E1,5.9E1,1.42E2,9E1,3E0,1.8E1,1.5E1,7.9E1,3.2E1,3E0,1.3E1,5.4E1,5E0,8E1,6.2E1,2.2E1,6.8E1,1.5E1,3E0,5E0,1E1,7.1E1,8E0,1E1,2.2E1,2.5E1,2.9E1,4.6E1,3.4E1,6E1,2E0,1E1,1.2E1,2.1E1,4.7E1,6.7E1,4E0,1.9E1,3E0,7E0,1.8E1,1.8E1,1.1E1,4E0,4.2E1,1.8E1,1.6E1,2.6E1,3.4E1,4.1E1,6E0,1.6E1,2E0,9E0,2E0,2.2E1,2E1,8E0,1E1,1E1,6E0,1.8E1,8E0,2.6E1,8E0,2E1,2.1E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[8.956354E-3,-1.029906E-1,2.3425508E-1,-1.2910096E-1,3.7656035E-2,1.645635E-1,4.9263373E-1,-1.9878702E-1,-7.74813E-2,-2.715676E-3,1.387793E-1,1.8753625E-1,-4.105063E-1,3.5427976E-2,2.9737055E-1,-2.4934062E-1,-1.6633725E-1,-1.309313E-1,-3.66568E-2,-2.0498302E-2,5.4745255E-3,-3.7756818E-3,8.883777E-3,1.15957275E-1,2.6210397E-1,-2.9608194E-2,-3.9431388E-3,1.7851755E-2,-8.802584E-3,-2.678979E-1,-3.0477094E-3,-3.7450406E-3,-1.9835751E-1,-8.966339E-2,-1.893846E-1,-6.6955835E-2,8.320926E-2,4.73552E-3,-4.5753505E-2,1.4312956E-1,-3.9651502E-2,3.2528866E-2,2.7881667E-1,-1.4053945E-2,-2.06271E-3,-1.1084807E-2,-6.857414E-3,-1.2682968E-1,-9.446983E-4,-2.334991E-3,-2.295347E-1,-1.696495E-2,-1.0625718E-1,6.481679E-3,-6.187479E-4,-4.3343683E-3,1.1912953E-3,8.044676E-2,1.885048E-1,-5.4749865E-3,1.3217086E-3,-4.9193697E-3,7.578006E-3,1.749144E-1,3.2302287E-1,-8.524274E-3,-2.8204466E-3,-6.122603E-3,-1.489971E-2,-2.3631228E-3,4.056985E-3,-6.320596E-3,7.6582655E-4,1.1226605E-3,7.7840686E-3,1.2486174E-2,6.7099095E-3,4.3056896E-3,1.237114E-2,1.8044692E-2,9.191213E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,-1,-1,39,41,-1,-1,-1,-1,43,-1,-1,45,47,49,51,53,-1,55,57,59,61,63,-1,-1,-1,-1,65,-1,-1,67,69,71,-1,-1,-1,-1,73,75,-1,-1,-1,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1626019E1,1.1359406E0,2.673973E0,9.198618E-1,2.0042032E-1,1.6887193E0,1.221128E0,1.3897562E-1,3.2469296E-1,7.549356E-2,1.4196503E-1,5.994301E-1,2.704299E-1,0E0,6.693367E-1,1.257751E-1,1.8650866E-1,1.3981509E-1,3.2154018E-1,9.902611E-2,0E0,0E0,0E0,2.6782644E-1,2.0962834E-1,0E0,0E0,0E0,0E0,1.4411283E-1,0E0,0E0,5.10149E-2,1.0221577E-1,1.3706434E-1,1.3572693E-1,8.612621E-2,0E0,8.2776345E-2,1.3558042E-1,5.0959885E-2,9.653786E-2,1.8693447E-1,0E0,0E0,0E0,0E0,7.470012E-2,0E0,0E0,1.1762333E-1,1.03597656E-1,1.09585226E-1,0E0,0E0,0E0,0E0,1.04500785E-1,6.765568E-2,0E0,0E0,0E0,0E0,9.914899E-2,1.5042377E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,29,29,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,47,47,50,50,51,51,52,52,57,57,58,58,63,63,64,64],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,-1,-1,40,42,-1,-1,-1,-1,44,-1,-1,46,48,50,52,54,-1,56,58,60,62,64,-1,-1,-1,-1,66,-1,-1,68,70,72,-1,-1,-1,-1,74,76,-1,-1,-1,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.164074E8,7.7101436E3,1.204977E6,1.4096699E9,9.533205E6,3.3616E4,1.65602E5,7.3123126E2,1.2E3,2.0532622E3,4.1331047E-1,9.6083355E-1,3.5427976E-2,5.22E2,1.7212875E5,5.4966196E3,1.7377544E7,2.103E3,1.5124741E1,5.4745255E-3,-3.7756818E-3,8.883777E-3,2.5916522E7,3.1095595E0,-2.9608194E-2,-3.9431388E-3,1.7851755E-2,-8.802584E-3,1.6395413E7,-3.0477094E-3,-3.7450406E-3,1.1442429E7,3.345E3,6.465659E4,1.7045455E-2,1.6991062E3,4.73552E-3,2.8688732E7,5.589612E1,1.2538845E9,1.173433E5,1.6441814E9,-1.4053945E-2,-2.06271E-3,-1.1084807E-2,-6.857414E-3,2.735322E6,-9.446983E-4,-2.334991E-3,9.379015E0,1.2472753E5,1.7657828E3,6.481679E-3,-6.187479E-4,-4.3343683E-3,1.1912953E-3,6.424587E6,6.3349745E6,-5.4749865E-3,1.3217086E-3,-4.9193697E-3,7.578006E-3,1.3299019E0,9.83854E-1,-8.524274E-3,-2.8204466E-3,-6.122603E-3,-1.489971E-2,-2.3631228E-3,4.056985E-3,-6.320596E-3,7.6582655E-4,1.1226605E-3,7.7840686E-3,1.2486174E-2,6.7099095E-3,4.3056896E-3,1.237114E-2,1.8044692E-2,9.191213E-3],"split_indices":[2,7,60,1,7,55,10,9,60,0,4,43,47,0,3,36,55,53,0,66,0,0,0,53,62,0,0,0,0,12,0,0,12,10,36,65,41,0,9,64,7,36,7,0,0,0,0,1,0,0,62,36,60,0,0,0,0,1,53,0,0,0,0,47,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.59E2,3.07E2,1.52E2,2.59E2,4.8E1,1.21E2,3.1E1,1.09E2,1.5E2,3.5E1,1.3E1,1.17E2,4E0,1.3E1,1.8E1,4E1,6.9E1,6.4E1,8.6E1,3.1E1,4E0,2E0,1.1E1,6.1E1,5.6E1,2E0,2E0,1.6E1,2E0,3.6E1,4E0,1.9E1,5E1,3.9E1,2.5E1,6.9E1,1.7E1,5E0,2.6E1,5.2E1,9E0,4E0,5.2E1,3.3E1,3E0,3.2E1,1.8E1,2.5E1,1.4E1,6E0,1.9E1,3.1E1,3.8E1,1.1E1,6E0,1.6E1,1E1,2.3E1,2.9E1,4E0,5E0,2E0,2E0,1.7E1,3.5E1,1.4E1,1.1E1,9E0,1E1,2.4E1,7E0,3.2E1,6E0,1.4E1,9E0,1.1E1,1.8E1,9E0,8E0,2.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-3.272327E-3,-1.1593676E-1,1.7140466E-1,-1.3759407E-1,4.2494424E-2,7.765383E-2,3.5767373E-1,-1.2997684E-1,-2.8873824E-2,7.497038E-2,-8.525453E-3,1.8061865E-2,1.5533675E-1,6.4292204E-1,2.9172269E-1,-1.9040488E-1,-8.207464E-2,1.5932658E-1,5.08527E-4,-1.03327865E-2,7.351377E-3,2.279062E-1,1.03811316E-1,1.6244376E-2,3.7941225E-2,3.391622E-1,-1.7519891E-2,-2.4923554E-1,-1.4403139E-1,-1.14587225E-1,1.1633802E-2,4.638229E-3,1.1802641E-2,-4.011993E-3,4.507645E-3,-6.296848E-2,4.2636223E-2,3.1854764E-3,1.2270331E-2,-3.977616E-3,5.7818745E-3,4.3826294E-3,3.6082405E-1,-1.0017785E-2,6.639577E-3,-2.6138943E-1,-2.4615268E-3,-5.9579033E-2,-1.8585402E-1,-2.7657919E-2,-1.4968131E-1,-3.448635E-2,1.8889517E-1,-8.087151E-3,-3.190533E-2,-2.2600037E-4,6.3238107E-3,4.2062405E-1,1.3779668E-2,-1.3389269E-2,-3.618168E-3,3.8045302E-3,-3.743454E-3,-1.8338285E-3,-9.61426E-3,-4.163653E-3,1.7112507E-3,-8.998124E-3,-4.907239E-3,-5.965981E-3,5.510516E-5,1.3938279E-2,1.1035947E-3,-5.7748235E-3,1.1645707E-3,1.6879825E-2,2.7806023E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,41,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,-1,-1,-1,57,-1,-1,59,-1,61,63,65,67,69,71,-1,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.485896E0,1.0111873E0,3.2727094E0,8.101878E-1,2.659701E-1,5.851127E-1,9.976282E-1,7.225046E-1,0E0,2.009212E-1,0E0,2.7484286E-1,1.8199515E-1,1.9204617E-1,7.834692E-1,2.6965857E-1,4.4005847E-1,5.624214E-2,1.4297104E-1,1.7275642E-1,0E0,6.372404E-2,9.816745E-2,0E0,0E0,2.1665668E-1,2.5612688E-1,1.02997065E-1,2.1850824E-1,3.2228565E-1,3.171765E-1,0E0,0E0,0E0,0E0,9.290643E-2,1.270156E-1,0E0,0E0,0E0,0E0,0E0,8.230734E-2,0E0,0E0,7.815671E-2,0E0,5.878657E-2,5.777943E-2,1.17555335E-1,1.09114885E-1,9.6366495E-2,1.2050453E-1,0E0,1.2224862E-1,0E0,0E0,5.0922632E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,35,35,36,36,42,42,45,45,47,47,48,48,49,49,50,50,51,51,52,52,54,54,57,57],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,42,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,-1,-1,-1,58,-1,-1,60,-1,62,64,66,68,70,72,-1,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,1.4777102E9,6.0891E4,5.825042E6,1.505649E6,1.01685606E9,1.417988E6,1.129877E6,-2.8873824E-2,3.798313E3,-8.525453E-3,1.4998456E5,9.565983E6,1.08E2,1.5096262E7,4.90403E5,1.0975164E3,1E0,1.2028214E5,9.6E1,7.351377E-3,9.9800183E2,3.0811954E-2,1.6244376E-2,3.7941225E-2,1.3590209E7,3.677E3,1.7212875E5,5.5253574E4,8.459907E6,5.6024197E9,4.638229E-3,1.1802641E-2,-4.011993E-3,4.507645E-3,1.5802073E3,1.9180013E3,3.1854764E-3,1.2270331E-2,-3.977616E-3,5.7818745E-3,4.3826294E-3,7.911333E6,-1.0017785E-2,6.639577E-3,1.5650286E6,-2.4615268E-3,1.4543E5,2.586E4,2.143469E6,2.73E2,4.086266E4,1.8707484E7,-8.087151E-3,8.42E2,-2.2600037E-4,6.3238107E-3,3.701371E7,1.3779668E-2,-1.3389269E-2,-3.618168E-3,3.8045302E-3,-3.743454E-3,-1.8338285E-3,-9.61426E-3,-4.163653E-3,1.7112507E-3,-8.998124E-3,-4.907239E-3,-5.965981E-3,5.510516E-5,1.3938279E-2,1.1035947E-3,-5.7748235E-3,1.1645707E-3,1.6879825E-2,2.7806023E-2],"split_indices":[41,7,2,55,2,7,37,1,0,40,0,36,53,3,9,9,60,22,36,8,0,60,46,0,0,1,0,36,56,53,5,0,0,0,0,4,60,0,0,0,0,0,37,0,0,55,0,37,37,1,3,36,1,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.8E2,2.92E2,1.88E2,2.57E2,3.5E1,1.26E2,6.2E1,2.54E2,3E0,3.1E1,4E0,7.2E1,5.4E1,1E1,5.2E1,1.11E2,1.43E2,1.4E1,1.7E1,6E1,1.2E1,2.1E1,3.3E1,4E0,6E0,4.5E1,7E0,4.7E1,6.4E1,1.06E2,3.7E1,9E0,5E0,9E0,8E0,3E1,3E1,3E0,1.8E1,2E0,3.1E1,4E0,4.1E1,3E0,4E0,4.4E1,3E0,2.2E1,4.2E1,3.1E1,7.5E1,3E1,7E0,6E0,2.4E1,2E1,1E1,2.1E1,2E1,4.1E1,3E0,2E0,2E1,3E0,3.9E1,1.6E1,1.5E1,4.3E1,3.2E1,8E0,2.2E1,4E0,3E0,9E0,1.5E1,1.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[1.0189711E-2,-7.8954235E-2,2.2183582E-1,-1.914945E-1,-4.7165155E-2,1.649445E-1,4.3252468E-1,-2.0296462E-1,-1.2632678E-3,-6.316145E-2,1.18095584E-1,1.7911525E-1,-2.0782799E-2,3.5365153E-2,2.9478455E-1,-1.1832499E-2,-1.6226324E-1,-8.166842E-2,3.9802026E-2,-1.9548738E-3,1.4255947E-1,1.310862E-1,2.4877945E-1,3.4930623E-1,-8.187295E-3,-9.052988E-4,-8.552822E-3,-1.0763756E-1,-1.7322518E-2,-3.0132467E-2,1.9383779E-1,1.00640915E-2,3.847449E-3,1.4146262E-1,-7.07874E-3,2.6990637E-1,-6.716674E-3,1.143213E-2,2.0751905E-2,-7.874799E-2,-1.6804391E-1,-1.3072363E-1,-2.0156635E-4,-4.020751E-3,2.3878345E-2,1.8543096E-2,5.843795E-3,1.1711761E-1,1.104653E-2,3.7679815E-1,1.061076E-2,-8.027117E-3,-2.0146486E-3,-9.63793E-3,-2.7917605E-3,5.396953E-4,-8.817247E-3,-1.7761858E-3,2.4480077E-3,4.13615E-3,-1.5699351E-3,3.902947E-3,9.51927E-3,3.715178E-3,2.0824155E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,23,-1,25,27,29,-1,31,33,35,37,-1,-1,-1,39,41,43,45,-1,-1,47,-1,49,-1,-1,-1,51,53,55,57,-1,59,-1,-1,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.1679945E0,1.2144907E0,1.6546946E0,1.3734603E-1,7.159331E-1,1.0353515E0,9.801302E-1,7.003403E-2,0E0,4.7010845E-1,9.8426074E-2,3.454759E-1,0E0,0E0,6.121644E-1,0E0,6.140262E-2,3.457049E-1,4.151925E-1,0E0,6.82579E-2,2.176243E-1,4.037006E-1,7.298708E-2,0E0,0E0,0E0,2.4504364E-1,1.1840728E-1,7.758263E-2,1.2007585E-1,0E0,0E0,1.1030185E-1,0E0,1.6452336E-1,0E0,0E0,0E0,3.1960583E-1,1.3760889E-1,5.762419E-2,9.929335E-2,0E0,5.0014302E-2,0E0,0E0,1.4059347E-1,0E0,1.5507841E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,14,14,16,16,17,17,18,18,20,20,21,21,22,22,23,23,27,27,28,28,29,29,30,30,33,33,35,35,39,39,40,40,41,41,42,42,44,44,47,47,49,49],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,24,-1,26,28,30,-1,32,34,36,38,-1,-1,-1,40,42,44,46,-1,-1,48,-1,50,-1,-1,-1,52,54,56,58,-1,60,-1,-1,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.41718E5,7.7101436E3,5.609174E8,1.5522031E9,1.0067637E7,3.0418E4,4.26944E5,-1.2632678E-3,2.0964778E5,2.0532622E3,6.6685635E9,-2.0782799E-2,3.5365153E-2,1.852084E7,-1.1832499E-2,1.4388227E4,2.6919386E-1,4.735985E6,-1.9548738E-3,6.3349745E6,8.6206274E2,2.8756912E8,4.966055E9,-8.187295E-3,-9.052988E-4,-8.552822E-3,2.7075084E5,1.6016256E3,5.9827844E2,3.1E1,1.00640915E-2,3.847449E-3,4.0292856E3,-7.07874E-3,4.5E1,-6.716674E-3,1.143213E-2,2.0751905E-2,9.5751E4,2.8884522E8,1.4590734E8,9.53E2,-4.020751E-3,2.8529238E1,1.8543096E-2,5.843795E-3,1.6768953E5,1.104653E-2,5.016E3,1.061076E-2,-8.027117E-3,-2.0146486E-3,-9.63793E-3,-2.7917605E-3,5.396953E-4,-8.817247E-3,-1.7761858E-3,2.4480077E-3,4.13615E-3,-1.5699351E-3,3.902947E-3,9.51927E-3,3.715178E-3,2.0824155E-2],"split_indices":[2,1,60,7,7,55,10,1,0,36,4,12,0,0,9,0,36,47,1,0,53,66,1,7,0,0,0,56,63,60,3,0,0,60,0,3,0,0,0,9,12,5,0,0,64,0,0,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.84E2,3.41E2,1.43E2,7.4E1,2.67E2,1.14E2,2.9E1,6.9E1,5E0,2.44E2,2.3E1,1.12E2,2E0,8E0,2.1E1,3.3E1,3.6E1,2.07E2,3.7E1,3E0,2E1,6.8E1,4.4E1,1.9E1,2E0,3E0,3.3E1,1.47E2,6E1,2.6E1,1.1E1,9E0,1.1E1,6.6E1,2E0,4.2E1,2E0,9E0,1E1,1.01E2,4.6E1,7E0,5.3E1,1.3E1,1.3E1,2E0,9E0,5.3E1,1.3E1,1.2E1,3E1,3E1,7.1E1,3.6E1,1E1,2E0,5E0,3.1E1,2.2E1,6E0,7E0,3.7E1,1.6E1,2E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[3.2963627E-3,-7.5435355E-2,1.980902E-1,-1.0852179E-1,3.30385E-2,2.3360048E-1,-2.299004E-1,-1.6498575E-1,-4.1272283E-2,4.4674505E-2,-9.769254E-3,1.8078376E-1,3.9834842E-1,-3.9970037E-1,8.198068E-3,-1.0685373E-2,-1.3533448E-1,-8.928022E-2,3.2599628E-2,1.7733887E-2,1.3871586E-1,9.365281E-2,2.3440582E-1,-7.919631E-3,4.4410512E-1,-2.7047494E-2,-9.723425E-3,4.2347293E-4,-1.4536269E-1,-1.4481057E-2,-1.3825054E-1,-4.939182E-2,1.317437E-1,-1.9674307E-2,3.7379751E-3,-2.0093333E-3,8.834251E-3,9.1238655E-3,3.492295E-3,2.4477321E-1,1.376468E-3,1.6680017E-1,5.121519E-1,1.3383328E-3,-1.4974621E-1,-5.4799538E-2,6.1702016E-3,-1.9603407E-1,-8.1929535E-2,2.613449E-3,-7.515178E-2,2.32083E-1,2.8359026E-3,7.689415E-2,-5.229535E-2,2.9570675E-3,2.5390503E-1,-1.1360404E-3,1.2133212E-2,3.3915184E-2,1.7664429E-2,-7.954959E-3,-3.3443668E-3,-8.38861E-4,-6.2793563E-3,-1.4532603E-2,-6.1156135E-3,3.96212E-4,-5.858554E-3,-3.0595096E-4,-6.001424E-3,1.633646E-2,6.1806883E-3,6.091167E-3,-1.2325216E-3,-4.552527E-3,2.3247622E-4,1.5708812E-2,9.720754E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,39,-1,41,-1,-1,-1,43,45,47,49,51,53,-1,-1,-1,-1,-1,55,-1,57,59,-1,61,63,-1,65,67,-1,69,71,-1,73,75,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.299758E0,1.2202609E0,2.13584E0,9.8045564E-1,2.2509922E-1,1.0330687E0,8.0847627E-1,1.9171834E-1,4.28483E-1,1.9262202E-1,0E0,4.3591714E-1,8.402214E-1,1.3532579E-1,0E0,0E0,1.3552523E-1,2.653926E-1,3.966974E-1,1.3496965E-1,1.3394427E-1,6.7608714E-2,1.2148023E-1,0E0,4.4292498E-1,0E0,0E0,0E0,6.993091E-2,1.7659976E-1,1.2713385E-1,7.5424545E-2,1.4974153E-1,1.2372743E-1,0E0,0E0,0E0,0E0,0E0,8.0709934E-2,0E0,1.12936944E-1,3.5070372E-1,0E0,7.5380564E-2,6.407374E-2,0E0,1.1329508E-1,8.205509E-2,0E0,7.08395E-2,5.445671E-2,0E0,5.139792E-2,6.749429E-2,0E0,1.3623285E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,28,28,29,29,30,30,31,31,32,32,33,33,39,39,41,41,42,42,44,44,45,45,47,47,48,48,50,50,51,51,53,53,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,40,-1,42,-1,-1,-1,44,46,48,50,52,54,-1,-1,-1,-1,-1,56,-1,58,60,-1,62,64,-1,66,68,-1,70,72,-1,74,76,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.164074E8,1.0429407E3,1.732734E6,1E0,7.3068325E3,3.236383E1,5.87114E5,1.156185E3,2.9920125E3,-9.769254E-3,3.0561674E9,5.0314346E1,1.2799757E9,8.198068E-3,-1.0685373E-2,5.195307E9,1.7377544E7,8.26042E4,6.136946E5,9.2849056E8,1.5963512E-2,1.773296E6,-7.919631E-3,6.274818E4,-2.7047494E-2,-9.723425E-3,4.2347293E-4,6.453125E0,1.2833E4,7.121916E2,4.816E3,1.03E2,1.136E1,3.7379751E-3,-2.0093333E-3,8.834251E-3,9.1238655E-3,3.492295E-3,8.825362E-2,1.376468E-3,2.1152368E-1,3.0418E4,1.3383328E-3,2.0964778E5,6.920599E7,6.1702016E-3,2.515304E7,2.5457E4,2.613449E-3,1.8E2,1.3482595E8,2.8359026E-3,1.24E2,1.5189052E6,2.9570675E-3,6.0837906E1,-1.1360404E-3,1.2133212E-2,3.3915184E-2,1.7664429E-2,-7.954959E-3,-3.3443668E-3,-8.38861E-4,-6.2793563E-3,-1.4532603E-2,-6.1156135E-3,3.96212E-4,-5.858554E-3,-3.0595096E-4,-6.001424E-3,1.633646E-2,6.1806883E-3,6.091167E-3,-1.2325216E-3,-4.552527E-3,2.3247622E-4,1.5708812E-2,9.720754E-3],"split_indices":[2,7,66,1,79,60,64,1,60,63,0,12,64,7,0,0,39,53,36,55,7,65,11,0,36,0,0,0,55,10,60,10,3,66,0,0,0,0,0,47,0,46,10,0,36,7,0,53,9,0,3,7,0,3,59,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.74E2,3.38E2,1.36E2,2.59E2,7.9E1,1.26E2,1E1,1.4E2,1.19E2,7.6E1,3E0,9.7E1,2.9E1,7E0,3E0,4.8E1,9.2E1,7.2E1,4.7E1,6E1,1.6E1,3.8E1,5.9E1,2E0,2.7E1,3E0,4E0,6E0,8.6E1,2.9E1,4.3E1,2.6E1,2.1E1,3.7E1,2.3E1,3E0,1.3E1,6E0,3.2E1,5.6E1,3E0,6E0,2.1E1,2E0,8.4E1,2.3E1,6E0,2E1,2.3E1,5E0,2.1E1,8E0,1.3E1,9E0,2.8E1,3E0,5.3E1,2E0,4E0,8E0,1.3E1,7.2E1,1.2E1,1.6E1,7E0,7E0,1.3E1,7E0,1.6E1,9E0,1.2E1,3E0,5E0,6E0,3E0,1.6E1,1.2E1,2.2E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[3.586574E-3,-7.978201E-2,1.7134984E-1,-1.574065E-1,-4.167852E-2,2.0124516E-1,-1.7987801E-1,-1.8457961E-1,-5.8006383E-2,4.215094E-3,-8.512938E-2,1.5456542E-1,3.641703E-1,-2.1335188E-2,-2.6052708E-2,-4.4388594E-3,-1.963867E-1,4.2625167E-3,-4.0628477E-3,-2.5544064E-2,6.278968E-2,-1.05294295E-1,1.2329735E-3,7.511142E-2,2.136198E-1,2.8276239E-2,2.2543605E-1,-1.2249375E-1,7.896551E-3,-2.0237538E-1,-2.156976E-3,-4.7955606E-2,1.0106031E-1,8.455447E-2,-2.9485496E-3,-1.1453644E-1,3.6646577E-3,-5.54244E-3,5.11421E-2,1.5122259E-1,3.9197806E-2,2.7210832E-1,1.6615061E-1,3.12726E-1,-1.1643696E-1,1.553687E-3,-9.341964E-3,-1.4838132E-1,-1.109E-2,-9.974502E-2,-6.633612E-3,8.797992E-3,1.5035506E-3,9.590543E-5,6.070733E-3,-8.79212E-2,-1.7962909E-1,-9.15507E-4,7.040336E-3,3.6345774E-3,1.1758141E-2,-3.1977475E-3,6.225264E-2,7.395392E-3,1.4974586E-2,1.6366048E-2,1.4807208E-1,9.220438E-3,1.8694434E-2,-1.01383785E-2,6.21801E-4,-8.941084E-3,-1.9565201E-3,-7.782226E-3,-2.4175537E-3,2.467222E-3,-1.7080148E-3,-9.7912615E-3,-3.4517574E-3,-1.0840878E-2,-2.3251371E-3,7.5590922E-3,1.0705591E-3,5.251485E-3,1.0567767E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,-1,31,33,35,37,39,41,-1,43,45,-1,47,-1,49,51,53,-1,55,-1,-1,57,59,61,63,65,67,69,-1,-1,71,-1,73,75,-1,-1,-1,-1,77,79,-1,-1,-1,-1,-1,81,-1,-1,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.6990676E0,9.39924E-1,1.7057109E0,2.7289915E-1,4.310881E-1,1.0616913E0,7.1289504E-1,7.329321E-2,9.3611754E-2,1.8637574E-1,1.9343519E-1,5.2823925E-1,8.1312704E-1,2.0419873E-1,0E0,0E0,5.6812048E-2,0E0,0E0,2.0630175E-1,1.0139294E-1,1.5830189E-1,1.312313E-1,1.3347718E-1,1.3803172E-1,0E0,6.630963E-1,7.943126E-2,0E0,5.5006504E-2,0E0,1.2988088E-1,5.51719E-2,1.00233376E-1,0E0,1.3319802E-1,0E0,0E0,1.05915844E-1,9.147212E-2,9.071376E-2,6.689072E-2,6.874108E-2,7.3518515E-2,6.0920432E-2,0E0,0E0,8.110505E-2,0E0,7.2469205E-2,5.794944E-2,0E0,0E0,0E0,0E0,1.0937458E-1,1.2100637E-1,0E0,0E0,0E0,0E0,0E0,1.0806824E-1,0E0,0E0,0E0,7.916725E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,31,31,32,32,33,33,35,35,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,49,49,50,50,55,55,56,56,62,62,66,66],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,-1,32,34,36,38,40,42,-1,44,46,-1,48,-1,50,52,54,-1,56,-1,-1,58,60,62,64,66,68,70,-1,-1,72,-1,74,76,-1,-1,-1,-1,78,80,-1,-1,-1,-1,-1,82,-1,-1,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,9.50639E5,1.0429407E3,9.703059E6,1.3508742E7,6.204522E3,7.339209E6,7.581957E1,2.409091E0,2.306533E6,1.530655E3,2.5096E4,1.417988E6,3.236383E1,-2.6052708E-2,-4.4388594E-3,1.6120845E5,4.2625167E-3,-4.0628477E-3,4.479355E1,1E0,6.46E2,1.1172539E9,4.19719E5,1.1532659E7,2.8276239E-2,1.46E2,1.0823923E9,7.896551E-3,4.421264E2,-2.156976E-3,1E0,1.5E2,6.5610016E4,-2.9485496E-3,3.069012E6,3.6646577E-3,-5.54244E-3,9.4332875E4,2.6979439E1,8.879629E4,5.3515047E-1,1.2662238E1,6.274818E4,3.087E3,1.553687E-3,-9.341964E-3,1.3350947E3,-1.109E-2,4.5593824E4,1E0,8.797992E-3,1.5035506E-3,9.590543E-5,6.070733E-3,3.2471478E2,6.4065235E8,-9.15507E-4,7.040336E-3,3.6345774E-3,1.1758141E-2,-3.1977475E-3,1.5191719E5,7.395392E-3,1.4974586E-2,1.6366048E-2,3.638653E6,9.220438E-3,1.8694434E-2,-1.01383785E-2,6.21801E-4,-8.941084E-3,-1.9565201E-3,-7.782226E-3,-2.4175537E-3,2.467222E-3,-1.7080148E-3,-9.7912615E-3,-3.4517574E-3,-1.0840878E-2,-2.3251371E-3,7.5590922E-3,1.0705591E-3,5.251485E-3,1.0567767E-2],"split_indices":[2,1,66,9,53,60,55,62,66,1,60,2,37,64,0,0,36,0,0,61,73,8,5,37,53,0,8,7,0,60,0,21,3,36,0,9,0,0,36,66,41,43,64,36,0,0,0,4,0,41,22,0,0,0,0,60,7,0,0,0,0,0,41,0,0,0,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.77E2,3.19E2,1.58E2,1.04E2,2.15E2,1.46E2,1.2E1,8.1E1,2.3E1,1.05E2,1.1E2,1.15E2,3.1E1,9E0,3E0,1E1,7.1E1,3E0,2E1,7E1,3.5E1,8.9E1,2.1E1,5E1,6.5E1,1.1E1,2E1,6E0,3E0,6.8E1,3E0,6E1,1E1,3E1,5E0,8.5E1,4E0,6E0,1.5E1,1.5E1,3.5E1,2.7E1,3.8E1,1.6E1,4E0,2E0,4E0,2.3E1,4.5E1,2.6E1,3.4E1,4E0,6E0,1E1,2E1,6.2E1,2.3E1,9E0,6E0,9E0,6E0,6E0,2.9E1,7E0,2E1,2E0,3.6E1,7E0,9E0,2E0,2E0,1.7E1,6E0,1.1E1,1.5E1,1.1E1,2.3E1,7E0,5.5E1,1.7E1,6E0,8E0,2.1E1,2.4E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[3.8112686E-3,-7.65908E-2,1.6922955E-1,-1.01616934E-1,2.3537008E-2,1.1302314E-1,3.603139E-1,-1.4252031E-1,-4.9870852E-2,-5.214634E-3,1.1191483E-1,-2.88786E-1,1.3999227E-1,3.1540874E-2,2.6662666E-1,-1.8318996E-1,-1.1003203E-1,7.251665E-3,-1.24275155E-1,-2.5172774E-2,5.932805E-3,-2.3265167E-3,6.7707514E-3,4.0248604E-3,-2.089478E-2,8.921271E-2,2.2946727E-1,1.4282947E-2,-1.7513562E-3,-1.1200906E-2,-1.392334E-1,-5.679392E-3,1.5529484E-3,-8.729969E-3,2.2076055E-2,-2.0628655E-1,-7.1697295E-2,1.3499813E-3,-5.93748E-2,1.24853946E-1,4.2561397E-2,1.7614087E-2,1.8713598E-1,-1.598383E-1,-1.8106793E-3,-7.3207915E-3,8.899107E-2,-1.0492332E-1,-1.3263408E-2,-5.6322146E-4,-1.1789087E-1,-3.6420447E-3,4.1452684E-3,1.6019277E-1,5.7416324E-2,2.9586127E-3,4.8512015E-3,1.1286022E-3,1.02481535E-2,-9.017189E-3,-3.0134742E-3,-1.987013E-3,2.9209475E-3,-9.537745E-4,6.3313707E-3,-7.1623824E-3,1.2762347E-3,-9.954446E-3,-3.410441E-3,9.274057E-3,1.3457088E-3,-4.8391023E-3,4.2548208E-3,-1.7518009E-3,4.189121E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,51,53,55,-1,57,59,-1,61,63,65,-1,-1,67,-1,-1,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.343748E0,8.067572E-1,1.6347413E0,5.3685594E-1,1.6574907E-1,1.362474E0,7.5604486E-1,1.6878724E-1,4.8966974E-1,1.2897742E-1,7.323553E-2,4.4232243E-1,5.020311E-1,0E0,2.1891212E-1,9.1431856E-2,7.084906E-2,1.8520269E-1,2.00809E-1,8.094447E-2,0E0,0E0,0E0,0E0,0E0,1.19389296E-1,1.579206E-1,0E0,0E0,0E0,6.577331E-2,0E0,0E0,0E0,1.2261108E-1,9.5226526E-2,8.765122E-2,0E0,6.4146E-2,9.0895295E-2,7.560125E-2,0E0,1.1349702E-1,5.4926693E-2,0E0,1.0011345E-1,8.5168794E-2,5.2276097E-2,0E0,0E0,5.9938014E-2,0E0,0E0,9.847981E-2,8.185847E-2,7.0039034E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,30,30,34,34,35,35,36,36,38,38,39,39,40,40,42,42,43,43,45,45,46,46,47,47,50,50,53,53,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,52,54,56,-1,58,60,-1,62,64,66,-1,-1,68,-1,-1,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,6.204522E3,1.732734E6,1.5522031E9,2.4950776E0,1E0,2.53383E5,1.8457164E7,1E0,2.0532622E3,2.203418E7,6.7304E4,3.1540874E-2,1.4656219E8,4E1,3.01E2,2.9E1,7.121916E2,8.9846605E8,5.932805E-3,-2.3265167E-3,6.7707514E-3,4.0248604E-3,-2.089478E-2,1.16E2,2.1207E4,1.4282947E-2,-1.7513562E-3,-1.1200906E-2,2.1592189E5,-5.679392E-3,1.5529484E-3,-8.729969E-3,1.272E4,3.8E1,5.56844E5,1.3499813E-3,7.02E2,6.3973755E-2,2.6196917E3,1.7614087E-2,4.0235848E4,1.5953781E6,-1.8106793E-3,1.656968E6,9.4332875E4,1.5E1,-1.3263408E-2,-5.6322146E-4,4.5709072E1,-3.6420447E-3,4.1452684E-3,5.78E2,1.3989E4,2.58E2,4.8512015E-3,1.1286022E-3,1.02481535E-2,-9.017189E-3,-3.0134742E-3,-1.987013E-3,2.9209475E-3,-9.537745E-4,6.3313707E-3,-7.1623824E-3,1.2762347E-3,-9.954446E-3,-3.410441E-3,9.274057E-3,1.3457088E-3,-4.8391023E-3,4.2548208E-3,-1.7518009E-3,4.189121E-3],"split_indices":[2,7,60,1,7,62,23,9,53,24,4,1,2,0,1,8,8,3,60,7,0,0,0,0,0,3,9,0,0,0,36,0,0,0,10,8,9,0,8,65,60,0,41,58,0,9,36,8,0,0,64,0,0,8,2,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.75E2,3.2E2,1.55E2,2.56E2,6.4E1,1.21E2,3.4E1,1.42E2,1.14E2,4.9E1,1.5E1,7E0,1.14E2,7E0,2.7E1,6.1E1,8.1E1,6.5E1,4.9E1,4.3E1,6E0,2E0,1.3E1,2E0,5E0,7.4E1,4E1,2.5E1,2E0,2.8E1,3.3E1,7.8E1,3E0,4E0,6.1E1,1.8E1,3.1E1,1.7E1,2.6E1,4.1E1,3.3E1,8E0,3.2E1,2.7E1,6E0,4.3E1,1.8E1,8E0,1E1,1.4E1,1.7E1,2.4E1,2E0,2.6E1,1.5E1,2E1,1.3E1,4E0,2.8E1,2.1E1,6E0,2.9E1,1.4E1,5E0,1.3E1,6E0,2E0,5E0,1.2E1,2.1E1,5E0,2E0,1.3E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[4.828785E-3,-7.417202E-2,1.5843876E-1,-1.4406271E-1,-3.8378287E-2,1.3095048E-1,3.0121865E-2,-1.62958E-1,-2.039179E-3,-5.1588994E-2,1.1541385E-1,1.5519609E-1,-1.4582421E-1,-1.718138E-1,-1.638321E-3,-1.025648E-1,-1.9566068E-2,-2.2504576E-3,6.864503E-3,9.543497E-2,2.317318E-1,2.6290582E-3,-1.882484E-2,-3.044355E-3,-1.8254785E-1,-1.1780427E-1,1.9513249E-3,2.2641735E-2,-6.28547E-2,1.8361306E-2,8.377998E-2,3.0626266E-3,2.4232943E-1,-8.168897E-3,1.1499676E-1,-1.1034423E-2,-7.3539456E-3,-1.3510238E-1,-1.2662457E-3,4.1592468E-2,-1.10474765E-1,1.1197194E-3,-8.2556754E-2,-7.271442E-2,9.9782385E-2,1.6212646E-2,2.0384622E-1,1.04586E-2,6.150317E-4,-5.170562E-3,-9.743994E-3,4.8129796E-3,7.784656E-4,1.9509381E-3,-8.198892E-3,-4.968712E-3,3.7212947E-5,5.0887223E-3,-7.298813E-3,3.6686168E-3,7.0977244E-3,7.4624894E-3,1.310115E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,-1,25,27,-1,-1,29,31,33,-1,-1,35,37,-1,39,41,-1,43,-1,45,-1,47,-1,-1,49,-1,51,53,-1,55,57,59,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.824566E0,7.874937E-1,1.9618387E0,2.004168E-1,4.3464342E-1,1.0627694E0,0E0,9.7513914E-2,0E0,3.1631005E-1,7.379052E-2,6.319463E-1,4.604222E-1,8.760953E-2,0E0,1.6739511E-1,2.2250444E-1,0E0,0E0,2.3526198E-1,9.609103E-2,1.9027857E-1,0E0,0E0,7.142568E-2,1.0470855E-1,0E0,1.6099954E-1,1.03178784E-1,0E0,2.075985E-1,0E0,1.2964606E-1,0E0,5.773752E-2,0E0,0E0,8.658385E-2,0E0,7.8436576E-2,7.482857E-2,0E0,7.668421E-2,1.2291971E-1,7.112193E-2,0E0,1.0001612E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,24,24,25,25,27,27,28,28,30,30,32,32,34,34,37,37,39,39,40,40,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,-1,26,28,-1,-1,30,32,34,-1,-1,36,38,-1,40,42,-1,44,-1,46,-1,48,-1,-1,50,-1,52,54,-1,56,58,60,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,9.50639E5,2.3767492E4,1.1482625E7,1.5522031E9,9.05929E2,3.0121865E-2,2.812953E5,-2.039179E-3,7.283496E2,2.0532622E3,6.0891E4,6.6053805E6,6.738547E7,-1.638321E-3,3.01E2,2.1692945E1,-2.2504576E-3,6.864503E-3,8.504733E7,1.1763399E-1,1.4414646E-1,-1.882484E-2,-3.044355E-3,2.0011E5,1.0038E4,1.9513249E-3,3.931053E7,4.3290512E1,1.8361306E-2,3.0811954E-2,3.0626266E-3,1.0894711E7,-8.168897E-3,2.9E1,-1.1034423E-2,-7.3539456E-3,5.861E3,-1.2662457E-3,1.6991062E3,1.7249636E-7,1.1197194E-3,1.6505348E9,3.61E2,1.5271514E5,1.6212646E-2,6.992308E1,1.04586E-2,6.150317E-4,-5.170562E-3,-9.743994E-3,4.8129796E-3,7.784656E-4,1.9509381E-3,-8.198892E-3,-4.968712E-3,3.7212947E-5,5.0887223E-3,-7.298813E-3,3.6686168E-3,7.0977244E-3,7.4624894E-3,1.310115E-2],"split_indices":[2,1,60,9,7,66,0,55,0,60,4,2,55,7,0,8,61,0,0,7,47,46,0,0,9,0,0,53,64,0,46,0,51,0,3,0,0,10,0,41,46,0,5,0,36,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,3.16E2,1.62E2,1.06E2,2.1E2,1.54E2,8E0,8.9E1,1.7E1,1.94E2,1.6E1,1.42E2,1.2E1,8.3E1,6E0,7.4E1,1.2E2,2E0,1.4E1,8.1E1,6.1E1,8E0,4E0,8E0,7.5E1,6.7E1,7E0,6.1E1,5.9E1,2E0,7.9E1,4E0,5.7E1,3E0,5E0,3E1,4.5E1,5.6E1,1.1E1,5.4E1,7E0,1.1E1,4.8E1,7E0,7.2E1,1.5E1,4.2E1,2E0,3E0,4E1,1.6E1,1.6E1,3.8E1,2E0,5E0,3.9E1,9E0,2E0,5E0,4.8E1,2.4E1,2.5E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.7017047E-3,-6.7955755E-2,1.6166192E-1,-9.663242E-2,3.198042E-2,1.940943E-1,-1.8258178E-1,-1.2279251E-1,-2.5633996E-2,4.3776702E-2,-9.236377E-3,1.5886685E-1,4.1033268E-1,-1.5150468E-2,-2.1942077E-2,-1.3430971E-1,-3.8015507E-2,3.9540257E-2,-1.110053E-1,1.978081E-2,1.4177532E-1,1.6852632E-1,-8.927854E-2,1.3607733E-2,4.8885942E-1,-1.20827116E-1,7.2161863E-3,-1.6378862E-1,-9.89516E-2,-8.238448E-3,7.1818655E-4,6.0221672E-2,-4.4693258E-3,-1.4923295E-3,-6.5338523E-3,3.7946086E-3,4.7927606E-3,-9.933644E-4,8.359607E-3,1.5112133E-1,2.5047563E-2,-1.0996536E-2,3.699659E-3,2.6462233E-2,9.34456E-3,2.1280925E-4,-9.04746E-3,-1.8917693E-1,-1.01722054E-1,-8.657341E-2,-1.1429068E-2,5.6760767E-3,1.35527E-3,5.1273033E-3,-9.72453E-3,1.7587629E-1,3.6343555E-3,-9.959267E-3,-4.7335215E-3,-1.2303985E-3,-6.4300005E-3,-3.0622663E-4,-5.726198E-3,-2.7642306E-3,8.804236E-4,6.43047E-3,1.0713205E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,31,33,35,37,39,41,-1,43,45,-1,47,49,-1,-1,51,-1,-1,-1,53,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3259215E0,9.574139E-1,1.6543112E0,4.7789955E-1,2.0317417E-1,9.479432E-1,5.6971717E-1,1.7943239E-1,3.9943397E-1,1.665082E-1,0E0,2.9309225E-1,5.2120447E-2,1.7214432E-1,0E0,1.5681767E-1,1.6673978E-1,1.1629579E-1,5.147791E-2,7.315324E-2,6.854254E-2,5.8996487E-1,1.2807363E-1,0E0,4.8585415E-2,4.8615508E-2,0E0,1.20999575E-1,1.0930556E-1,0E0,0E0,6.1935022E-2,0E0,0E0,0E0,6.912693E-2,0E0,0E0,0E0,1.9285393E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.031251E-2,5.9789658E-2,1.7633426E-1,0E0,0E0,0E0,0E0,5.9107166E-2,1.2570739E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,31,31,35,35,39,39,47,47,48,48,49,49,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,32,34,36,38,40,42,-1,44,46,-1,48,50,-1,-1,52,-1,-1,-1,54,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.164074E8,1.0429407E3,1.0975164E3,1E0,8.625562E0,7.339209E6,2.1592189E5,2.474199E1,3.4540886E3,-9.236377E-3,3.7826266E4,4.966055E9,3.236383E1,-2.1942077E-2,4.822E3,6.52487E2,4.7374536E7,5.888557E1,9.652488E5,9.2849056E8,1.4609149E4,1E0,1.3607733E-2,8.3E1,3.7137984E7,7.2161863E-3,2.613714E6,3.703296E7,-8.238448E-3,7.1818655E-4,5.050505E-3,-4.4693258E-3,-1.4923295E-3,-6.5338523E-3,2.829932E0,4.7927606E-3,-9.933644E-4,8.359607E-3,1.5096262E7,2.5047563E-2,-1.0996536E-2,3.699659E-3,2.6462233E-2,9.34456E-3,2.1280925E-4,-9.04746E-3,6.623914E4,2.37E2,9.842559E3,-1.1429068E-2,5.6760767E-3,1.35527E-3,5.1273033E-3,1.8756017E5,2.203418E7,3.6343555E-3,-9.959267E-3,-4.7335215E-3,-1.2303985E-3,-6.4300005E-3,-3.0622663E-4,-5.726198E-3,-2.7642306E-3,8.804236E-4,6.43047E-3,1.0713205E-2],"split_indices":[2,7,66,60,79,43,55,36,61,4,0,60,7,64,0,2,4,53,64,55,7,4,21,0,8,1,0,9,53,0,0,65,0,0,0,66,0,0,0,9,0,0,0,0,0,0,0,41,10,56,0,0,0,0,55,1,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,3.32E2,1.44E2,2.58E2,7.4E1,1.32E2,1.2E1,1.88E2,7E1,7.1E1,3E0,1.15E2,1.7E1,8E0,4E0,1.65E2,2.3E1,4E1,3E1,5.8E1,1.3E1,1.11E2,4E0,8E0,9E0,5E0,3E0,8.8E1,7.7E1,6E0,1.7E1,3.5E1,5E0,7E0,2.3E1,4.9E1,9E0,2E0,1.1E1,1.07E2,4E0,2E0,2E0,7E0,2E0,2E0,3E0,6.1E1,2.7E1,7.2E1,5E0,1.2E1,2.3E1,5E0,4.4E1,8E1,2.7E1,5.2E1,9E0,8E0,1.9E1,2E1,5.2E1,1.6E1,2.8E1,4.1E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[8.791292E-3,-7.252759E-2,1.3072285E-1,-1.274002E-1,-4.4413615E-2,6.3988306E-2,2.4488838E-1,-1.3503097E-1,7.435309E-3,-3.6279738E-2,-1.3980742E-2,3.851482E-2,7.1749566E-3,4.4527638E-1,1.9920658E-1,-1.5636764E-1,-3.044542E-2,-6.97704E-2,1.8780483E-2,-3.220118E-2,7.439219E-2,1.3588976E-2,3.0663772E-2,2.2416604E-1,-1.4664642E-2,-8.775373E-3,-4.857249E-3,4.070829E-3,-2.9760366E-3,-8.4969245E-2,4.486202E-2,5.0645612E-2,-3.731275E-3,-4.294261E-3,-1.497601E-4,4.703031E-2,1.3866399E-1,1.1921623E-1,2.596945E-1,-8.913126E-3,4.3410817E-3,-9.3349524E-2,2.0057708E-3,-5.285618E-3,3.8880554E-3,1.8437387E-1,1.5258252E-2,5.362997E-3,5.992109E-4,9.013303E-3,2.0341766E-3,-9.391378E-4,6.962385E-3,2.7262816E-1,4.0016347E-3,-4.913049E-3,2.4399853E-3,4.714577E-3,1.3290897E-2,-6.6279666E-3,1.2697219E-3,4.8581217E-3,1.4015516E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,-1,-1,-1,-1,41,43,45,-1,-1,-1,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.659774E0,4.2937934E-1,1.416661E0,2.2274196E-1,3.576988E-1,2.4601158E-1,5.4292965E-1,2.0276952E-1,0E0,3.3876288E-1,0E0,2.383462E-1,0E0,1.9001198E-1,3.1289768E-1,8.193064E-2,6.336746E-2,2.0243192E-1,2.1546324E-1,5.1374264E-2,1.02511644E-1,0E0,0E0,1.6293263E-1,1.3678212E-1,0E0,0E0,0E0,0E0,1.1035621E-1,8.210921E-2,2.4799815E-1,0E0,0E0,0E0,9.630657E-2,7.13301E-2,4.995829E-2,6.528258E-2,0E0,0E0,9.7935855E-2,0E0,0E0,0E0,5.8047622E-2,7.227783E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.172348E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,29,29,30,30,31,31,35,35,36,36,37,37,38,38,41,41,45,45,46,46,53,53],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,-1,-1,-1,-1,42,44,46,-1,-1,-1,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,9.54947E5,5.6503E4,8.588E3,4.166257E7,1.5271514E5,1.417988E6,1.1482625E7,7.435309E-3,2.1721223E3,-1.3980742E-2,2.4339888E3,7.1749566E-3,1.92144E5,1.5096262E7,8.1672195E4,2.409091E0,2.889654E5,3.4867884E7,3.1547388E1,1.3210104E9,1.3588976E-2,3.0663772E-2,3.706875E5,3.677E3,-8.775373E-3,-4.857249E-3,4.070829E-3,-2.9760366E-3,1.1340564E3,5.8955963E1,8.8561325E0,-3.731275E-3,-4.294261E-3,-1.497601E-4,9.6E1,7.201163E1,4.410076E1,2.8492312E5,-8.913126E-3,4.3410817E-3,6.46E2,2.0057708E-3,-5.285618E-3,3.8880554E-3,3.2498508E3,5.2151023E-5,5.362997E-3,5.992109E-4,9.013303E-3,2.0341766E-3,-9.391378E-4,6.962385E-3,1.1041459E-1,4.0016347E-3,-4.913049E-3,2.4399853E-3,4.714577E-3,1.3290897E-2,-6.6279666E-3,1.2697219E-3,4.8581217E-3,1.4015516E-2],"split_indices":[41,1,2,2,59,36,37,9,0,63,0,4,0,2,9,36,66,36,53,64,7,0,0,40,0,0,0,0,0,60,58,66,0,0,0,3,64,66,41,0,0,8,0,0,0,4,46,0,0,0,0,0,0,47,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,2.81E2,1.87E2,9.4E1,1.87E2,1.19E2,6.8E1,9.2E1,2E0,1.82E2,5E0,9.2E1,2.7E1,1.1E1,5.7E1,7.6E1,1.6E1,1.13E2,6.9E1,3.1E1,6.1E1,7E0,4E0,5.1E1,6E0,5.3E1,2.3E1,3E0,1.3E1,1E2,1.3E1,5.2E1,1.7E1,1E1,2.1E1,4.4E1,1.7E1,1.4E1,3.7E1,2E0,4E0,9.4E1,6E0,2E0,1.1E1,1E1,4.2E1,1.5E1,2.9E1,1.1E1,6E0,2E0,1.2E1,3.4E1,3E0,9E1,4E0,6E0,4E0,2E0,4E1,3E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[3.6833659E-3,-4.000748E-2,2.2099927E-1,-6.7634754E-2,9.999084E-2,3.821724E-1,1.68043E-1,-9.248511E-2,9.559461E-3,1.3365372E-1,-2.3373207E-2,9.3757035E-3,2.6232835E-2,2.1819304E-1,4.3061722E-2,-8.629025E-2,-2.231244E-2,-1.5845917E-2,7.141166E-3,1.5693499E-1,2.1072808E-3,-7.4855355E-3,1.8395191E-3,2.4842207E-1,1.1240785E-3,5.0378987E-3,-1.0121012E-2,-1.1046044E-1,-3.749807E-2,-4.769891E-2,3.9893724E-2,9.909611E-4,1.7117018E-1,6.1963284E-3,2.819478E-1,-1.330724E-1,-4.473925E-2,-1.0108109E-1,1.4707308E-2,-9.381321E-4,-1.0356069E-1,4.6273298E-3,-3.1086328E-4,1.01455925E-2,4.7917156E-3,1.8193351E-2,1.0951641E-2,-7.043421E-3,-6.144018E-4,1.4293737E-3,-3.4450602E-3,-1.9003103E-3,-7.9134945E-3,-1.3142934E-3,3.7901243E-3,1.4732747E-3,-6.199803E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,-1,31,-1,-1,-1,33,-1,-1,-1,35,37,39,41,-1,43,-1,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.537821E0,1.5461359E0,6.1858916E-1,6.396793E-1,2.7720803E-1,4.5206618E-1,3.7920868E-1,5.266535E-1,2.857592E-1,1.0303831E-1,1.2378552E-1,0E0,0E0,2.5403094E-1,3.0401453E-1,2.8950775E-1,0E0,1.261861E-1,0E0,7.634103E-2,0E0,0E0,0E0,1.1988497E-1,0E0,0E0,0E0,2.4073672E-1,2.8031418E-1,6.983394E-2,6.606989E-2,0E0,7.686436E-2,0E0,6.853676E-2,1.5472293E-1,8.498868E-2,1.365473E-1,1.2415956E-1,0E0,4.9621716E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,19,19,23,23,27,27,28,28,29,29,30,30,32,32,34,34,35,35,36,36,37,37,38,38,40,40],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,-1,32,-1,-1,-1,34,-1,-1,-1,36,38,40,42,-1,44,-1,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.5522031E9,1.417988E6,9.7467445E4,1.662639E7,9.256843E3,1.5096262E7,5.825042E6,1.5271514E5,1.59E2,2.7157537E4,9.3757035E-3,2.6232835E-2,2.2576077E10,1.92144E5,2.025313E6,-2.231244E-2,1.8697775E3,7.141166E-3,1.504E4,2.1072808E-3,-7.4855355E-3,1.8395191E-3,2.1360708E7,1.1240785E-3,5.0378987E-3,-1.0121012E-2,1.0865863E7,7.121916E2,1.1629627E7,1.1106066E7,9.909611E-4,1.0595947E7,6.1963284E-3,7.8E1,8.163E3,2.1037032E-4,1.7197124E7,1.039103E5,-9.381321E-4,2.1862803E8,4.6273298E-3,-3.1086328E-4,1.01455925E-2,4.7917156E-3,1.8193351E-2,1.0951641E-2,-7.043421E-3,-6.144018E-4,1.4293737E-3,-3.4450602E-3,-1.9003103E-3,-7.9134945E-3,-1.3142934E-3,3.7901243E-3,1.4732747E-3,-6.199803E-3],"split_indices":[60,7,37,41,53,4,9,55,36,3,41,0,0,12,2,1,0,60,0,9,0,0,0,1,0,0,0,9,60,53,56,0,53,0,3,2,46,53,36,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,3.97E2,7.9E1,3.32E2,6.5E1,1.8E1,6.1E1,2.51E2,8.1E1,5.1E1,1.4E1,9E0,9E0,4.3E1,1.8E1,2.48E2,3E0,6.9E1,1.2E1,4E1,1.1E1,4E0,1E1,3.7E1,6E0,1.5E1,3E0,1.65E2,8.3E1,4.4E1,2.5E1,4E0,3.6E1,9E0,2.8E1,1.22E2,4.3E1,3.7E1,4.6E1,3E1,1.4E1,1.1E1,1.4E1,2.3E1,1.3E1,9E0,1.9E1,1.12E2,1E1,1.1E1,3.2E1,1.9E1,1.8E1,2.8E1,1.8E1,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[6.680436E-3,-5.6123756E-2,1.4115986E-1,-7.820028E-2,2.327379E-2,1.635148E-1,-1.6635151E-1,-3.2174565E-2,-1.2190774E-1,3.4348004E-2,-8.647695E-3,1.1146579E-1,2.4665925E-1,-2.8232417E-1,1.4358849E-3,-6.264816E-2,4.164838E-2,-9.582013E-3,-1.0794571E-1,-3.292813E-3,5.042343E-2,7.4503005E-2,1.6586718E-1,2.2795364E-2,1.9386606E-1,-1.6912805E-2,-4.1167145E-3,-2.7420701E-2,-1.0309399E-1,9.452002E-3,1.2914237E-2,-2.1696656E-3,-1.18653856E-1,3.4972057E-2,7.278226E-3,5.217648E-2,7.0782197E-3,1.8905343E-1,2.9954796E-3,2.1337157E-1,-7.6911123E-3,-6.477099E-2,1.924142E-2,-1.14192E-2,-7.048769E-2,6.9467817E-3,-2.2960684E-4,-1.2307708E-1,7.3340087E-4,-9.133832E-5,7.7774726E-2,1.1059722E-1,7.2789966E-4,5.4219277E-3,2.3087695E-1,1.894197E-2,8.890847E-3,-8.8811765E-4,-5.401049E-3,-2.1479663E-3,5.1773847E-3,-5.5978107E-3,-2.2856462E-4,-5.76042E-4,-6.2718163E-3,6.133077E-3,1.2015213E-3,6.741229E-3,-2.9725686E-3,1.2391562E-2,1.8434516E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,-1,31,-1,33,35,37,-1,39,-1,-1,41,43,-1,45,-1,47,49,-1,51,-1,53,-1,55,-1,57,59,-1,61,-1,-1,63,-1,-1,65,67,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.112616E0,5.837569E-1,1.092732E0,5.191114E-1,1.6743106E-1,6.000061E-1,2.5783417E-1,2.9056448E-1,1.15371585E-1,1.1759591E-1,0E0,1.7062771E-1,5.434036E-1,5.7921946E-2,0E0,1.2683022E-1,1.6249263E-1,0E0,7.251322E-2,0E0,8.717199E-2,7.978344E-2,7.591528E-2,0E0,3.4813392E-1,0E0,0E0,8.8388465E-2,1.6129592E-1,0E0,7.542247E-2,0E0,5.9166074E-2,8.528234E-2,0E0,9.432379E-2,0E0,6.967223E-2,0E0,1.739335E-1,0E0,5.7277016E-2,1.298093E-1,0E0,1.0006389E-1,0E0,0E0,4.891312E-2,0E0,0E0,5.991909E-2,8.654027E-2,0E0,0E0,6.638509E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,18,18,20,20,21,21,22,22,24,24,27,27,28,28,30,30,32,32,33,33,35,35,37,37,39,39,41,41,42,42,44,44,47,47,50,50,51,51,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,-1,32,-1,34,36,38,-1,40,-1,-1,42,44,-1,46,-1,48,50,-1,52,-1,54,-1,56,-1,58,60,-1,62,-1,-1,64,-1,-1,66,68,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.164074E8,1.0429407E3,2.6505424E1,1E0,9.3058E4,1.31805E5,1.0517595E3,1.44E2,1.4338295E6,-8.647695E-3,1.5271514E5,2.24784E5,2.9648642E1,1.4358849E-3,1.7239808E7,2.7E1,-9.582013E-3,1.5755866E4,-3.292813E-3,1E0,4.0292856E3,9.4416E4,2.2795364E-2,2.0531621E4,-1.6912805E-2,-4.1167145E-3,4.858648E2,2.1678448E7,9.452002E-3,2.3199144E1,-2.1696656E-3,1.6120845E5,8.287E3,7.278226E-3,1.5191719E5,7.0782197E-3,7.15016E6,2.9954796E-3,2.1207E4,-7.6911123E-3,1.213018E-6,2.325945E9,-1.14192E-2,2.231E3,6.9467817E-3,-2.2960684E-4,1.869E4,7.3340087E-4,-9.133832E-5,7.200105E8,4.4605376E3,7.2789966E-4,5.4219277E-3,2.3023677E0,1.894197E-2,8.890847E-3,-8.8811765E-4,-5.401049E-3,-2.1479663E-3,5.1773847E-3,-5.5978107E-3,-2.2856462E-4,-5.76042E-4,-6.2718163E-3,6.133077E-3,1.2015213E-3,6.741229E-3,-2.9725686E-3,1.2391562E-2,1.8434516E-3],"split_indices":[2,7,66,61,79,2,36,60,0,53,0,36,11,64,0,53,8,0,36,0,24,60,10,0,4,0,0,60,53,0,64,0,36,2,0,41,0,1,0,9,0,46,5,0,0,0,0,9,0,0,5,4,0,0,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,3.31E2,1.54E2,2.59E2,7.2E1,1.44E2,1E1,1.27E2,1.32E2,6.9E1,3E0,9E1,5.4E1,6E0,4E0,9E1,3.7E1,1.9E1,1.13E2,9E0,6E1,5.5E1,3.5E1,9E0,4.5E1,4E0,2E0,4.9E1,4.1E1,5E0,3.2E1,1.7E1,9.6E1,5.3E1,7E0,4.3E1,1.2E1,2.8E1,7E0,4.3E1,2E0,2.7E1,2.2E1,7E0,3.4E1,3E0,2.9E1,9.3E1,3E0,2.9E1,2.4E1,1.6E1,2.7E1,1.1E1,1.7E1,5E0,3.8E1,1.4E1,1.3E1,1.3E1,9E0,2E1,1.4E1,4E0,8.9E1,1.2E1,1.2E1,1.4E1,2E0,1.5E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[8.62062E-3,-5.045262E-2,1.362883E-1,-1.14835545E-1,-1.9801728E-2,1.5735787E-1,-2.0660387E-1,-1.4705087E-1,-4.8044313E-2,-4.609879E-2,3.0952655E-2,1.2576295E-1,3.4063923E-1,7.85918E-3,-1.639461E-2,-1.5210655E-1,7.295073E-4,2.0799034E-3,-3.5393117E-3,-6.729048E-2,5.266696E-2,-2.2018632E-2,8.31094E-2,9.428616E-2,1.8488142E-1,2.3239154E-2,1.0411621E-2,-7.7998233E-3,-1.6747803E-3,-5.5992354E-2,-8.0113E-3,6.714215E-3,1.44217545E-2,-2.04375E-3,2.7906199E-3,1.3874413E-3,1.1390698E-1,1.7610547E-1,6.2714316E-2,2.1046638E-1,-5.1194727E-3,-4.7936156E-2,-1.0414049E-2,-2.6693398E-3,2.4709774E-3,6.7761363E-3,8.857531E-4,1.3727515E-3,2.1802129E-1,6.7977593E-3,4.284538E-2,8.964234E-3,2.31625E-2,-3.748115E-3,-4.3533248E-4,1.1895268E-2,1.1090314E-3,1.2038022E-3,5.0913864E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,35,37,39,-1,-1,-1,-1,41,-1,-1,43,-1,-1,-1,45,47,49,51,-1,53,-1,-1,-1,-1,-1,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.52887E0,6.2921214E-1,1.1017506E0,2.1456039E-1,2.9238957E-1,7.678666E-1,4.752675E-1,5.8749557E-2,7.945442E-2,3.0493784E-1,2.090407E-1,2.0773232E-1,2.413814E-1,0E0,0E0,4.9470544E-2,0E0,0E0,0E0,1.20024145E-1,8.180055E-2,6.1327033E-2,6.0929745E-2,1.9819385E-1,3.2541907E-1,0E0,0E0,0E0,0E0,1.258733E-1,0E0,0E0,4.979537E-2,0E0,0E0,0E0,5.3335667E-2,1.3047844E-1,8.45502E-2,1.8897831E-1,0E0,1.1507259E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.605505E-2,0E0,5.2122824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,19,19,20,20,21,21,22,22,23,23,24,24,29,29,32,32,36,36,37,37,38,38,39,39,41,41,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,36,38,40,-1,-1,-1,-1,42,-1,-1,44,-1,-1,-1,46,48,50,52,-1,54,-1,-1,-1,-1,-1,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.227E4,9.50639E5,2.3028242E3,3.006846E6,1.0975164E3,4.966055E9,2.909019E3,9.65325E8,5.9073865E2,3.94E2,8.459549E4,4.23168E3,4.2588E5,7.85918E-3,-1.639461E-2,1.7367111E6,7.295073E-4,2.0799034E-3,-3.5393117E-3,1E0,3.08031E5,1.3596371E9,2.0534246E3,2.412E4,1.4656219E8,2.3239154E-2,1.0411621E-2,-7.7998233E-3,-1.6747803E-3,4.086885E7,-8.0113E-3,6.714215E-3,1.5292238E5,-2.04375E-3,2.7906199E-3,1.3874413E-3,1.567512E8,3.059068E-1,1.6340721E3,3.305542E4,-5.1194727E-3,2.735322E6,-1.0414049E-2,-2.6693398E-3,2.4709774E-3,6.7761363E-3,8.857531E-4,1.3727515E-3,2.7129436E1,6.7977593E-3,3.7379084E0,8.964234E-3,2.31625E-2,-3.748115E-3,-4.3533248E-4,1.1895268E-2,1.1090314E-3,1.2038022E-3,5.0913864E-3],"split_indices":[2,1,66,9,60,7,40,5,55,8,36,60,11,0,0,59,0,0,0,79,37,7,63,9,1,0,0,0,0,53,0,0,55,0,0,0,40,49,60,4,0,1,0,0,0,0,0,0,62,0,43,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.66E2,3.19E2,1.47E2,1.02E2,2.17E2,1.39E2,8E0,6.8E1,3.4E1,1.43E2,7.4E1,1.2E2,1.9E1,2E0,6E0,6.6E1,2E0,7E0,2.7E1,1.18E2,2.5E1,3.7E1,3.7E1,8E1,4E1,8E0,1.1E1,6.2E1,4E0,1.07E2,1.1E1,7E0,1.8E1,3E1,7E0,1.4E1,2.3E1,2.1E1,5.9E1,3.7E1,3E0,1.03E2,4E0,6E0,1.2E1,1.8E1,5E0,5E0,1.6E1,1.1E1,4.8E1,3.5E1,2E0,5.9E1,4.4E1,1.4E1,2E0,3.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.1052844E-2,-5.1901545E-2,1.2659301E-1,-8.161439E-2,-4.6262057E-3,8.635613E-2,3.3838415E-1,-1.0438285E-1,6.881679E-3,3.3052877E-2,-4.2865507E-2,-2.1576354E-1,1.0630099E-1,2.5628373E-2,2.1010955E-1,-9.906702E-3,-9.5069036E-2,4.782374E-3,-2.7736409E-2,-5.4680496E-3,4.46559E-3,-9.280344E-3,-2.8778968E-2,-2.3600375E-3,-2.0266784E-2,6.881275E-2,1.5917636E-1,1.255287E-2,5.296974E-3,-1.03852265E-1,-1.5316134E-3,-3.3741407E-3,2.2421733E-3,2.5446746E-2,-5.1233363E-3,-4.6989985E-2,2.8378644E-3,1.043971E-1,4.3656245E-2,8.211335E-2,1.9236441E-1,-1.1154555E-1,-8.5711654E-4,-3.1787765E-3,5.1813297E-2,-2.5046706E-2,-6.591731E-3,1.2983464E-1,5.813648E-4,1.1248177E-2,4.7589755E-3,1.13701224E-1,-2.788822E-3,1.0516785E-2,3.2876534E-3,-2.4153452E-4,-5.716755E-3,4.6118554E-3,-1.37647E-4,-5.7023917E-5,-4.995925E-3,2.8982891E-3,8.136936E-3,-3.2111262E-3,1.583098E-3,-2.1973643E-3,6.9406866E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,-1,35,-1,-1,37,39,-1,-1,41,-1,-1,-1,43,-1,45,-1,47,49,51,53,55,-1,-1,57,59,-1,61,-1,63,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4034166E0,4.25242E-1,1.376878E0,3.7549317E-1,1.714338E-1,8.665385E-1,5.046215E-1,1.15377784E-1,1.2507881E-1,1.3457209E-1,1.16210006E-1,2.7811053E-1,2.506975E-1,0E0,5.195874E-2,0E0,7.357097E-2,0E0,9.063513E-2,1.159072E-1,0E0,0E0,8.9507E-2,0E0,0E0,6.616151E-2,1.2231326E-1,0E0,0E0,7.779038E-2,0E0,0E0,0E0,7.238039E-2,0E0,8.488255E-2,0E0,7.3790014E-2,8.195897E-2,8.548927E-2,8.806801E-2,5.8421493E-2,0E0,0E0,5.3317823E-2,6.961143E-2,0E0,5.401808E-2,0E0,5.214622E-2,0E0,7.251878E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,18,18,19,19,22,22,25,25,26,26,29,29,33,33,35,35,37,37,38,38,39,39,40,40,41,41,44,44,45,45,47,47,49,49,51,51],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,-1,36,-1,-1,38,40,-1,-1,42,-1,-1,-1,44,-1,46,-1,48,50,52,54,56,-1,-1,58,60,-1,62,-1,64,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,2.025313E6,8.10288E3,7.164074E8,1.423795E7,2.4950776E0,3.0418E4,2.398E3,1.2504131E1,1.656968E6,8.74756E2,1.6244629E-1,4.1331047E-1,2.5628373E-2,6.755346E1,-9.906702E-3,8.067265E5,4.782374E-3,2.8716172E7,1.7359158E-1,4.46559E-3,-9.280344E-3,1.2028214E5,-2.3600375E-3,-2.0266784E-2,8.16997E6,1E0,1.255287E-2,5.296974E-3,2.0964778E5,-1.5316134E-3,-3.3741407E-3,2.2421733E-3,5.8955963E1,-5.1233363E-3,1.0736829E5,2.8378644E-3,7.4433655E-2,3.723672E3,8.934E3,1.0850484E0,5.45E2,-8.5711654E-4,-3.1787765E-3,1.10375E1,4.086885E7,-6.591731E-3,4.98884E1,5.813648E-4,1.0336E4,4.7589755E-3,4.152439E3,-2.788822E-3,1.0516785E-2,3.2876534E-3,-2.4153452E-4,-5.716755E-3,4.6118554E-3,-1.37647E-4,-5.7023917E-5,-4.995925E-3,2.8982891E-3,8.136936E-3,-3.2111262E-3,1.583098E-3,-2.1973643E-3,6.9406866E-3],"split_indices":[2,1,60,7,53,62,10,2,66,9,4,46,43,0,65,0,55,0,9,46,0,0,36,0,0,53,102,0,0,36,0,0,0,58,0,36,0,65,60,0,65,12,0,0,61,53,0,64,0,10,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.66E2,3.02E2,1.64E2,1.85E2,1.17E2,1.39E2,2.5E1,1.47E2,3.8E1,5.9E1,5.8E1,8E0,1.31E2,9E0,1.6E1,1.1E1,1.36E2,1E1,2.8E1,3.6E1,2.3E1,4E0,5.4E1,5E0,3E0,7.8E1,5.3E1,1E1,6E0,1.19E2,1.7E1,1.8E1,1E1,2.8E1,8E0,4.5E1,9E0,3.1E1,4.7E1,1.7E1,3.6E1,1.09E2,1E1,6E0,2.2E1,3.7E1,8E0,2.4E1,7E0,3E1,1.7E1,1.4E1,3E0,3E1,6E0,5E0,1.04E2,1.2E1,1E1,2.9E1,8E0,9E0,1.5E1,6E0,2.4E1,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[3.302771E-3,-5.047337E-2,1.278923E-1,-9.739954E-2,-2.1127462E-2,8.44082E-2,2.5253755E-1,-1.153945E-1,-1.3863494E-2,-2.6540192E-2,1.2826475E-2,1.020902E-1,-1.515884E-1,1.3858455E-1,3.7844786E-1,-1.2373812E-1,3.8198535E-3,-4.3108296E-3,7.6996547E-4,-4.34656E-2,4.2221576E-2,1.3875148E-1,6.417068E-2,-9.788452E-3,-2.4639393E-4,8.687016E-3,-7.445472E-3,1.0470143E-2,2.4585156E-2,-1.4209585E-1,-7.4943036E-2,-5.933918E-2,9.339795E-3,1.8852288E-2,7.5047435E-3,4.721371E-3,1.03605E-2,1.5234102E-4,8.884246E-2,-1.4981611E-1,-3.0916879E-2,-4.4047087E-3,2.168322E-3,-5.0181456E-2,-7.4220276E-3,-1.3527198E-2,4.886256E-3,3.2337601E-3,-2.427345E-2,1.0544928E-1,1.2780304E-4,-7.578194E-3,1.9223361E-4,-5.3605502E-3,4.1151377E-3,-3.261995E-3,-3.2270947E-4,1.565311E-4,-6.4417273E-3,-4.298134E-3,9.807105E-4,3.7184514E-3,9.620503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,-1,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,-1,-1,-1,49,51,53,-1,-1,55,-1,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2224283E0,4.6064478E-1,7.6459646E-1,1.9231653E-1,3.2520136E-1,4.7032094E-1,4.784236E-1,1.8058777E-1,5.3917266E-2,2.404185E-1,0E0,1.328944E-1,5.5297866E-2,2.620637E-1,2.3754835E-1,8.0718994E-2,0E0,0E0,0E0,1.3885897E-1,1.03291094E-1,1.3674688E-1,7.80652E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.8026433E-2,5.7155594E-2,9.941611E-2,8.187899E-2,7.267894E-2,0E0,0E0,0E0,0E0,5.2713305E-2,4.9928665E-2,6.3186936E-2,0E0,0E0,8.2902014E-2,0E0,6.346112E-2,0E0,0E0,5.506097E-2,6.8443656E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,22,22,29,29,30,30,31,31,32,32,33,33,38,38,39,39,40,40,43,43,45,45,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,-1,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,-1,-1,-1,50,52,54,-1,-1,56,-1,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,4.844E3,6.204522E3,2.0964778E5,3.058642E3,9.05929E2,2.2876814E1,7.164074E8,6.52487E2,3.44E2,1.2826475E-2,1.1291153E7,1.31805E5,5.01E2,2.235E5,3.006846E6,3.8198535E-3,-4.3108296E-3,7.6996547E-4,1.2472753E5,1.3738772E5,5.6503E4,2.0663544E9,-9.788452E-3,-2.4639393E-4,8.687016E-3,-7.445472E-3,1.0470143E-2,2.4585156E-2,1.3421E4,8.89E2,9.409E3,1.4318394E3,1.3404826E-3,7.5047435E-3,4.721371E-3,1.03605E-2,1.5234102E-4,8.80085E5,3.5111064E7,1.5964883E5,-4.4047087E-3,2.168322E-3,1.0640507E8,-7.4220276E-3,7.929922E-2,4.886256E-3,3.2337601E-3,1.570662E5,8.9181E6,1.2780304E-4,-7.578194E-3,1.9223361E-4,-5.3605502E-3,4.1151377E-3,-3.261995E-3,-3.2270947E-4,1.565311E-4,-6.4417273E-3,-4.298134E-3,9.807105E-4,3.7184514E-3,9.620503E-3],"split_indices":[2,2,60,36,60,66,62,7,4,8,0,53,36,3,2,9,0,0,0,36,41,2,12,0,0,0,0,0,0,10,0,2,60,65,0,0,0,0,11,53,36,0,0,12,0,65,0,0,55,37,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.79E2,3.35E2,1.44E2,1.28E2,2.07E2,1.08E2,3.6E1,1.05E2,2.3E1,2.04E2,3E0,1.01E2,7E0,2E1,1.6E1,1.01E2,4E0,6E0,1.7E1,1.64E2,4E1,5E1,5.1E1,5E0,2E0,1.8E1,2E0,8E0,8E0,7.2E1,2.9E1,1.26E2,3.8E1,3.4E1,6E0,3.3E1,1.7E1,1.5E1,3.6E1,6.7E1,5E0,2.6E1,3E0,1.16E2,1E1,3.1E1,7E0,1.6E1,1.8E1,3E1,6E0,6.5E1,2E0,3E0,2E0,8.4E1,3.2E1,2.8E1,3E0,7E0,1.1E1,2.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-7.483993E-3,-6.627997E-2,8.106253E-2,-6.1756123E-2,-1.7966868E-2,3.0444058E-2,1.4755306E-1,-1.05768345E-1,-3.709878E-2,1.3025602E-2,5.728372E-3,1.2901223E-1,1.4741676E-2,-1.2068383E-1,-1.9919353E-2,4.052764E-2,-5.2789643E-2,-6.2930603E-3,5.8713234E-3,1.0714949E-1,2.7500433E-1,-1.5543383E-1,-8.140563E-2,-2.968114E-3,2.836678E-3,8.60748E-2,-2.4573598E-3,-6.55386E-2,2.5283177E-3,-3.4173682E-2,2.6569702E-3,1.1969185E-1,-9.618372E-3,5.5321367E-3,1.8742528E-2,-1.2701504E-3,-8.225775E-3,3.9348667E-3,-9.58973E-2,4.027228E-4,9.267332E-3,-4.521273E-2,-1.1441221E-1,-8.750686E-3,-4.888698E-3,9.628929E-2,1.7005111E-1,-6.342441E-3,-1.9491785E-3,-1.1199857E-3,-5.020295E-3,-6.3838516E-3,-1.1640654E-3,7.128121E-4,-3.1530445E-3,2.800678E-3,6.5214057E-3,6.251858E-4,9.209875E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,21,23,25,27,29,-1,31,33,35,37,-1,-1,39,-1,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,51,53,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4522772E0,3.6157608E-1,6.2824976E-1,2.9993916E-1,0E0,1.609545E-1,1.9108164E-1,1.2564385E-1,2.2260207E-1,1.8833563E-1,0E0,2.067256E-1,0E0,1.03364825E-1,5.4024942E-2,1.3229068E-1,2.042565E-1,1.333634E-1,0E0,2.7453578E-1,1.1335689E-1,6.579161E-2,1.06080204E-1,0E0,0E0,1.6670795E-1,0E0,1.2893587E-1,0E0,8.907131E-2,0E0,5.905217E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.8081915E-2,0E0,0E0,1.21717125E-1,5.2666783E-2,5.26731E-2,0E0,5.853623E-2,5.0674498E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,27,27,29,29,31,31,38,38,41,41,42,42,43,43,45,45,46,46],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,22,24,26,28,30,-1,32,34,36,38,-1,-1,40,-1,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,52,54,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,2.5096E4,4.407E3,-1.7966868E-2,1.5271514E5,1.6828056E1,2.0964778E5,1E0,1.3210104E9,5.728372E-3,4.2875492E1,1.4741676E-2,1.2555067E3,4.174811E2,8.213E3,1.577058E9,2.143564E3,5.8713234E-3,5.22E2,5.9161E4,5.8955963E1,1.422708E3,-2.968114E-3,2.836678E-3,3.377402E6,-2.4573598E-3,4.6263345E-2,2.5283177E-3,1.3508742E7,2.6569702E-3,4.026531E9,-9.618372E-3,5.5321367E-3,1.8742528E-2,-1.2701504E-3,-8.225775E-3,3.9348667E-3,7.6E1,4.027228E-4,9.267332E-3,2.3790412E7,1.6120845E5,2.02E2,-4.888698E-3,1.7895791E-1,1.5200746E-1,-6.342441E-3,-1.9491785E-3,-1.1199857E-3,-5.020295E-3,-6.3838516E-3,-1.1640654E-3,7.128121E-4,-3.1530445E-3,2.800678E-3,6.5214057E-3,6.251858E-4,9.209875E-3],"split_indices":[41,55,2,2,0,36,43,36,102,7,0,62,0,63,41,0,7,60,0,3,2,58,4,0,0,37,0,65,0,53,0,7,0,0,0,0,0,0,8,0,0,53,36,3,0,46,49,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.69E2,2.82E2,1.87E2,2.79E2,3E0,1.07E2,8E1,9.9E1,1.8E2,9E1,1.7E1,7.3E1,7E0,8.4E1,1.5E1,3E1,1.5E2,7.7E1,1.3E1,6.5E1,8E0,4.3E1,4.1E1,1E1,5E0,2E1,1E1,1.34E2,1.6E1,5.3E1,2.4E1,6.3E1,2E0,4E0,4E0,4E0,3.9E1,3E0,3.8E1,1.2E1,8E0,9.6E1,3.8E1,3.9E1,1.4E1,4.5E1,1.8E1,2.3E1,1.5E1,7E1,2.6E1,3.2E1,6E0,2.8E1,1.1E1,2.3E1,2.2E1,2E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[2.2375495E-3,-4.0655874E-2,1.12655416E-1,-6.027597E-2,2.958203E-2,7.742791E-2,2.3753335E-1,-1.0313382E-1,-2.8772328E-2,1.5791528E-2,6.3990955E-3,8.928193E-2,-1.2120961E-2,1.5515102E-1,4.3439305E-1,-1.2426116E-1,-5.931813E-2,1.858612E-3,-6.994611E-2,-7.745004E-3,6.883734E-2,9.772946E-2,-1.1027827E-2,9.780576E-3,5.270583E-2,7.568802E-3,2.4618506E-2,-1.3274015E-1,-1.7456484E-3,-1.0619721E-4,-4.8539876E-3,-1.7329036E-3,4.066636E-2,-9.4031915E-3,-5.0180413E-2,1.7321428E-3,-1.9502046E-3,4.651631E-3,-1.637559E-3,1.6762016E-2,8.934591E-2,5.2135848E-3,-3.5348E-3,-7.0959404E-3,-2.339482E-3,-6.924365E-4,6.853028E-2,-8.331712E-2,-4.917781E-3,1.0219621E-1,-1.728203E-3,-1.1263692E-4,5.1029297E-3,-7.3177367E-3,-1.7365884E-3,2.5743558E-3,-2.1617592E-3,3.0718525E-3,8.431383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,-1,43,-1,-1,-1,-1,45,-1,47,-1,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2684836E0,4.7721875E-1,5.7109284E-1,3.615806E-1,1.0369886E-1,4.2993355E-1,4.0291846E-1,9.815991E-2,1.9813709E-1,8.543212E-2,0E0,2.9532743E-1,0E0,8.931774E-2,7.40273E-2,5.2410364E-2,8.8195026E-2,1.3279453E-1,1.5168223E-1,6.760379E-2,5.9025027E-2,1.7613333E-1,0E0,0E0,6.1181474E-2,0E0,0E0,6.326938E-2,0E0,0E0,0E0,0E0,6.966315E-2,0E0,8.808169E-2,0E0,0E0,0E0,0E0,0E0,1.6209906E-1,0E0,0E0,0E0,0E0,0E0,7.5730324E-2,1.0206038E-1,6.0777742E-2,2.3748809E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,24,24,27,27,32,32,34,34,40,40,46,46,47,47,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,-1,44,-1,-1,-1,-1,46,-1,48,-1,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5704E4,7.164074E8,2.1678066E0,1.204977E6,1E0,9.533205E6,1.2123712E1,3.006846E6,1.8071064E7,1.577058E9,6.3990955E-3,2.3767492E4,-1.2120961E-2,4.08326E5,6.7304E4,1.7212875E5,1E0,2.735322E6,1.9196308E7,8.9846605E8,1.436E3,8.169258E3,-1.1027827E-2,9.780576E-3,2.4585695E0,7.568802E-3,2.4618506E-2,1.1282727E5,-1.7456484E-3,-1.0619721E-4,-4.8539876E-3,-1.7329036E-3,9.286926E4,-9.4031915E-3,1E0,1.7321428E-3,-1.9502046E-3,4.651631E-3,-1.637559E-3,1.6762016E-2,7.2125336E2,5.2135848E-3,-3.5348E-3,-7.0959404E-3,-2.339482E-3,-6.924365E-4,5.555794E0,3.8499493E1,7.7815704E2,9.3058E4,-1.728203E-3,-1.1263692E-4,5.1029297E-3,-7.3177367E-3,-1.7365884E-3,2.5743558E-3,-2.1617592E-3,3.0718525E-3,8.431383E-3],"split_indices":[2,7,50,1,24,55,61,9,53,7,0,60,0,9,2,36,22,1,53,7,0,41,0,0,50,0,0,55,0,0,0,0,36,0,13,0,0,0,0,0,66,0,0,0,0,0,66,64,41,2,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.77E2,3.44E2,1.33E2,2.69E2,7.5E1,1.05E2,2.8E1,1.13E2,1.56E2,6.7E1,8E0,1.02E2,3E0,2.1E1,7E0,7.5E1,3.8E1,9E1,6.6E1,4.7E1,2E1,1E2,2E0,1.4E1,7E0,2E0,5E0,6.8E1,7E0,1.6E1,2.2E1,4.6E1,4.4E1,8E0,5.8E1,2E1,2.7E1,1.6E1,4E0,2E0,9.8E1,5E0,2E0,5.9E1,9E0,1.5E1,2.9E1,3.3E1,2.5E1,8.9E1,9E0,1E1,1.9E1,1.3E1,2E1,1E1,1.5E1,5.8E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[3.3435605E-3,-5.0766263E-2,8.492005E-2,-4.608403E-2,-1.805477E-2,3.6092125E-2,1.7127426E-1,-6.266745E-2,1.631908E-2,2.228319E-2,5.1688794E-3,3.5512733E-1,1.2553562E-1,-9.228392E-2,-2.6629444E-2,7.548219E-3,8.849048E-3,-3.1642225E-3,3.2311182E-2,1.03442045E-2,2.3940384E-2,1.3771829E-1,-6.6495645E-3,-1.1337767E-1,-5.2313697E-2,-6.5010116E-2,1.6211078E-2,4.982286E-3,-2.5782516E-4,4.0836823E-3,1.8100742E-2,-4.5279556E-4,1.4654355E-1,-6.1958E-3,-2.4337904E-3,-3.528186E-4,-1.06096804E-1,-3.346441E-2,-1.5275039E-1,-1.3748578E-2,1.3442859E-1,-2.1050917E-5,3.0364166E-3,1.934412E-1,5.6444407E-3,-6.1277174E-3,2.080461E-3,-2.5693625E-3,4.2022676E-3,-1.01315975E-2,-7.108186E-4,1.2081781E-3,-2.8183449E-3,1.22238705E-2,3.656346E-3,1.57288E-3,1.0871587E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,-1,-1,29,-1,-1,31,-1,33,35,37,39,-1,-1,-1,41,-1,43,-1,-1,-1,45,47,49,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0921752E0,4.0659076E-1,7.907363E-1,2.9285234E-1,0E0,1.14419535E-1,5.2724373E-1,2.3555458E-1,8.5011765E-2,9.1747254E-2,0E0,1.4644384E-1,1.9682711E-1,9.692967E-2,1.6876718E-1,7.038863E-2,0E0,0E0,6.5844E-2,0E0,0E0,7.1385026E-2,0E0,5.9031367E-2,1.0600892E-1,1.4435102E-1,1.7539425E-1,0E0,0E0,0E0,6.0110345E-2,0E0,5.2937984E-2,0E0,0E0,0E0,6.1321467E-2,9.697056E-2,1.012412E-1,6.8549395E-2,5.016759E-2,0E0,0E0,7.9508424E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,18,18,21,21,23,23,24,24,25,25,26,26,30,30,32,32,36,36,37,37,38,38,39,39,40,40,43,43],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,-1,-1,30,-1,-1,32,-1,34,36,38,40,-1,-1,-1,42,-1,44,-1,-1,-1,46,48,50,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,5.6503E4,7.10711E8,-1.805477E-2,1.7670108E5,1.799389E6,1.6415557E9,1.0987323E7,2.4522E1,5.1688794E-3,1.92144E5,5.22E2,3.006846E6,7.121916E2,2.829932E0,8.849048E-3,-3.1642225E-3,7.507176E-3,1.03442045E-2,2.3940384E-2,4.2464733E-2,-6.6495645E-3,1.2472753E5,4.7749156E5,8.7E1,2.473E3,4.982286E-3,-2.5782516E-4,4.0836823E-3,7.415715E6,-4.5279556E-4,4.917967E7,-6.1958E-3,-2.4337904E-3,-3.528186E-4,4.91E2,3.8E2,2.040228E5,3.271774E7,3.7577084E2,-2.1050917E-5,3.0364166E-3,6.8963E4,5.6444407E-3,-6.1277174E-3,2.080461E-3,-2.5693625E-3,4.2022676E-3,-1.01315975E-2,-7.108186E-4,1.2081781E-3,-2.8183449E-3,1.22238705E-2,3.656346E-3,1.57288E-3,1.0871587E-2],"split_indices":[41,55,2,7,0,36,37,5,37,64,0,2,3,9,60,66,0,0,65,0,0,46,0,36,59,8,0,0,0,0,1,0,52,0,0,0,8,3,36,53,41,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.72E2,2.84E2,1.88E2,2.81E2,3E0,1.21E2,6.7E1,2.22E2,5.9E1,1.02E2,1.9E1,1.2E1,5.5E1,1.21E2,1.01E2,5.7E1,2E0,1E1,9.2E1,7E0,5E0,5.3E1,2E0,7.8E1,4.3E1,5.3E1,4.8E1,6E0,5.1E1,1.9E1,7.3E1,3E0,5E1,6.4E1,1.4E1,2.4E1,1.9E1,4E1,1.3E1,3.9E1,9E0,5.2E1,2.1E1,1.8E1,3.2E1,1.7E1,2E0,3.5E1,5E0,9E0,4E0,2.1E1,1.8E1,2E0,7E0,3E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[2.770151E-3,-5.3348947E-2,8.318271E-2,-4.8599977E-2,-1.739073E-2,3.576612E-2,1.6563576E-1,-6.937995E-2,3.036566E-3,2.1461675E-2,4.5159813E-3,3.3350098E-1,1.2370927E-1,-7.6933555E-2,1.9028254E-3,1.2589431E-2,-1.4380819E-2,5.3814426E-3,7.0148155E-2,9.890043E-3,2.2248112E-2,1.4613874E-1,-1.0023988E-2,-1.1200731E-1,-5.7050254E-2,1.0128594E-3,-4.380105E-2,3.8439367E-2,-1.0161249E-3,2.916072E-4,5.309583E-3,4.870233E-3,2.028971E-1,-7.91561E-3,-7.046799E-2,-2.4765251E-2,-9.568852E-2,-6.669302E-2,4.3517433E-4,-3.593084E-3,2.5526069E-3,1.6854946E-2,8.071352E-3,-8.2265865E-4,-5.2972296E-3,-1.6704749E-3,6.04335E-3,-5.1583163E-3,7.181985E-4,-1.5710278E-3,-6.1726044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,-1,-1,25,27,29,-1,-1,31,-1,33,35,-1,37,39,-1,-1,-1,-1,41,-1,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0531945E0,3.6621368E-1,7.2517884E-1,2.8455716E-1,0E0,9.470485E-2,4.373269E-1,1.5742981E-1,3.4392068E-1,7.5562835E-2,0E0,1.1445832E-1,4.4200486E-1,1.17046595E-1,0E0,0E0,7.614854E-2,6.464998E-2,5.8100127E-2,0E0,0E0,1.2070882E-1,0E0,1.1792356E-1,1.4108545E-1,0E0,4.8825324E-2,5.3997967E-2,0E0,0E0,0E0,0E0,6.748825E-2,0E0,7.0774764E-2,9.2278555E-2,5.6043327E-2,5.339355E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,16,16,17,17,18,18,21,21,23,23,24,24,26,26,27,27,32,32,34,34,35,35,36,36,37,37],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,-1,-1,26,28,30,-1,-1,32,-1,34,36,-1,38,40,-1,-1,-1,-1,42,-1,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,5.9161E4,6.979E3,-1.739073E-2,1.7670108E5,1.417988E6,2.889654E5,1.3812E4,1.29186E9,4.5159813E-3,1.92144E5,5.22E2,1.65602E5,1.9028254E-3,1.2589431E-2,2.4999522E5,1.06E2,9.992596E-2,9.890043E-3,2.2248112E-2,1.2384782E5,-1.0023988E-2,2.55E2,2.7075084E5,1.0128594E-3,1.173433E5,1.293844E0,-1.0161249E-3,2.916072E-4,5.309583E-3,4.870233E-3,9.957E3,-7.91561E-3,6.738547E7,4.17E2,2.91E2,1E0,4.3517433E-4,-3.593084E-3,2.5526069E-3,1.6854946E-2,8.071352E-3,-8.2265865E-4,-5.2972296E-3,-1.6704749E-3,6.04335E-3,-5.1583163E-3,7.181985E-4,-1.5710278E-3,-6.1726044E-3],"split_indices":[41,55,2,2,0,36,37,36,9,7,0,2,3,9,0,0,56,3,46,0,0,36,0,3,56,0,36,61,0,0,0,0,9,0,7,3,8,21,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.53E2,2.67E2,1.86E2,2.64E2,3E0,1.19E2,6.7E1,1.88E2,7.6E1,9.6E1,2.3E1,1.2E1,5.5E1,1.76E2,1.2E1,4E0,7.2E1,7.3E1,2.3E1,7E0,5E0,5.2E1,3E0,6.2E1,1.14E2,3.3E1,3.9E1,3.2E1,4.1E1,9E0,1.4E1,3E1,2.2E1,2.7E1,3.5E1,6.3E1,5.1E1,2.7E1,1.2E1,3E0,2.9E1,3E0,1.9E1,1.5E1,2E1,6E1,3E0,4.7E1,4E0,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-3.3531704E-3,-4.0538464E-2,8.34358E-2,-5.371104E-2,2.6283557E-2,1.00905985E-1,-1.0037189E-1,-5.676057E-2,1.2151784E-2,9.692133E-3,4.5741796E-3,8.799721E-2,1.752279E-2,-2.6426488E-1,3.5094347E-2,-1.11816704E-1,-4.3954957E-2,3.982454E-3,-2.5631275E-4,1.2514776E-1,3.7088357E-2,-3.8644264E-3,-1.654059E-2,7.086873E-3,-2.9159142E-3,-7.909958E-3,-4.185171E-3,-2.4614394E-2,-7.8090414E-2,1.9816592E-1,9.944286E-2,5.4373506E-2,-9.655314E-3,-5.215099E-2,1.292886E-2,-1.6850583E-2,-1.0465437E-1,6.908289E-4,2.3404248E-1,5.1582355E-2,6.8263477E-3,3.4538861E-3,-2.096381E-3,-3.8227965E-3,-1.101648E-4,-4.6763162E-4,3.2098184E-3,3.6251734E-3,-2.0588452E-3,-7.5617996E-3,-2.9976638E-3,2.2689395E-3,1.271932E-2,4.8326133E-3,3.649156E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,23,25,27,-1,-1,29,31,-1,-1,-1,-1,-1,-1,33,35,37,39,41,-1,43,45,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5492463E0,2.9716504E-1,4.725343E-1,2.7243394E-1,6.134037E-2,4.0548158E-1,2.978998E-1,1.9178772E-1,0E0,4.9504835E-2,0E0,2.3555249E-1,0E0,5.468571E-2,9.1363795E-2,5.435157E-2,1.4860329E-1,0E0,0E0,1.1730933E-1,2.331456E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5264481E-1,1.3163185E-1,1.1557686E-1,9.9383116E-2,8.410987E-2,0E0,1.0927947E-1,7.496267E-2,6.2421933E-2,1.13495946E-1,0E0,5.9342265E-2,5.5238806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,19,19,20,20,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,24,26,28,-1,-1,30,32,-1,-1,-1,-1,-1,-1,34,36,38,40,42,-1,44,46,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,2.76E2,1.0429407E3,3.2991206E3,1.3738772E5,1.0871896E10,1.0095158E0,5.63321E5,1.2151784E-2,1.4634644E-4,4.5741796E-3,2.024909E6,1.752279E-2,3.11E3,1.505649E6,3.133E3,1.8071064E7,3.982454E-3,-2.5631275E-4,9.276807E6,2.5212732E4,-3.8644264E-3,-1.654059E-2,7.086873E-3,-2.9159142E-3,-7.909958E-3,-4.185171E-3,1.964127E6,6.465659E4,1.7387E4,3.941285E3,9.0648186E1,-9.655314E-3,1.21506645E2,8.4672E4,8.270692E0,2.73E2,6.908289E-4,1.139952E0,4.5709072E1,6.8263477E-3,3.4538861E-3,-2.096381E-3,-3.8227965E-3,-1.101648E-4,-4.6763162E-4,3.2098184E-3,3.6251734E-3,-2.0588452E-3,-7.5617996E-3,-2.9976638E-3,2.2689395E-3,1.271932E-2,4.8326133E-3,3.649156E-4],"split_indices":[2,8,66,60,41,7,47,1,0,46,0,9,0,0,2,2,53,0,0,59,4,0,0,0,0,0,0,1,36,2,60,64,0,62,11,61,3,0,61,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,3.35E2,1.43E2,2.8E2,5.5E1,1.31E2,1.2E1,2.78E2,2E0,4.5E1,1E1,1.26E2,5E0,5E0,7E0,5.1E1,2.27E2,7E0,3.8E1,7.2E1,5.4E1,2E0,3E0,3E0,4E0,1.6E1,3.5E1,1.46E2,8.1E1,1.7E1,5.5E1,5.1E1,3E0,8.4E1,6.2E1,2.5E1,5.6E1,3E0,1.4E1,2.6E1,2.9E1,4.4E1,7E0,5.5E1,2.9E1,4.4E1,1.8E1,5E0,2E1,2.5E1,3.1E1,2E0,1.2E1,1.2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.6064148E-3,-3.6120635E-2,8.3337195E-2,-3.8673323E-2,1.1754509E-2,-1.8196073E-1,9.6190445E-2,-5.149891E-2,8.611998E-3,-1.5699603E-2,-3.0726676E-3,7.915969E-2,1.0842634E-2,-1.9448826E-2,-7.556948E-2,1.6918814E-2,-8.544532E-3,8.8223934E-2,-1.12296954E-1,-6.37185E-2,1.2996076E-2,-7.092777E-2,-9.110552E-3,3.5238885E-3,4.2820445E-4,7.800127E-2,2.2268204E-2,-9.852423E-3,2.122714E-3,-3.2224763E-2,-8.372131E-3,2.6573408E-2,-6.991193E-2,-7.796141E-2,3.7582527E-4,-2.4599014E-3,1.1230377E-3,1.3400582E-2,7.217856E-2,-8.08472E-3,-2.703661E-4,1.7295522E-3,-4.459506E-3,-6.40079E-3,1.8872673E-3,-2.1744266E-3,-4.671547E-3,1.9797063E-3,5.0114943E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,19,21,23,-1,25,27,29,31,33,-1,-1,35,37,-1,-1,-1,39,-1,41,43,45,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.405985E0,2.2894204E-1,5.085343E-1,1.884467E-1,0E0,9.534152E-2,2.760712E-1,1.8676746E-1,1.0471524E-1,0E0,0E0,2.2336257E-1,0E0,1.5307856E-1,5.793178E-2,5.8858998E-2,0E0,4.0591758E-1,9.141702E-2,1.4614873E-1,7.164931E-2,7.5223505E-2,0E0,0E0,5.913087E-2,1.12163424E-1,0E0,0E0,0E0,1.2663794E-1,0E0,5.838584E-2,6.415534E-2,6.790125E-2,0E0,0E0,0E0,0E0,1.04716E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,24,24,25,25,29,29,31,31,32,32,33,33,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,20,22,24,-1,26,28,30,32,34,-1,-1,36,38,-1,-1,-1,40,-1,42,44,46,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,3.2991206E3,2.4950776E0,7.164074E8,1.1754509E-2,1.2799757E9,8.625562E0,2.225E1,1E0,-1.5699603E-2,-3.0726676E-3,3.768034E4,1.0842634E-2,6.210482E2,7.806329E10,1.2504131E1,-8.544532E-3,3.305542E4,9.820362E4,1.68804E6,4.086885E7,1.643477E9,-9.110552E-3,3.5238885E-3,1.8756017E5,8.504733E7,2.2268204E-2,-9.852423E-3,2.122714E-3,1.198E5,-8.372131E-3,1.9985462E5,5.9485336E1,1.26E2,3.7582527E-4,-2.4599014E-3,1.1230377E-3,1.3400582E-2,1.3590209E7,-8.08472E-3,-2.703661E-4,1.7295522E-3,-4.459506E-3,-6.40079E-3,1.8872673E-3,-2.1744266E-3,-4.671547E-3,1.9797063E-3,5.0114943E-3],"split_indices":[2,60,62,7,0,7,43,61,79,0,0,4,0,60,39,66,0,4,36,12,53,5,0,0,55,7,0,0,0,7,0,41,64,3,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.54E2,3.11E2,1.43E2,3.09E2,2E0,6E0,1.37E2,2.43E2,6.6E1,2E0,4E0,1.22E2,1.5E1,1.05E2,1.38E2,6.4E1,2E0,1.17E2,5E0,4.4E1,6.1E1,1.34E2,4E0,1.4E1,5E1,1.15E2,2E0,3E0,2E0,3.5E1,9E0,5.3E1,8E0,1.23E2,1.1E1,1.5E1,3.5E1,2E0,1.13E2,5E0,3E1,5E1,3E0,5E0,3E0,4.3E1,8E1,5.6E1,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.1023666E-3,-4.4042777E-2,8.00722E-2,-5.8423035E-2,1.8124469E-2,4.8342943E-2,1.9154449E-1,-7.00003E-2,-3.612062E-3,4.0801545E-3,5.254375E-3,6.1848182E-2,-6.796745E-3,3.4834862E-1,1.08616725E-1,-7.844813E-2,-1.046303E-2,4.139222E-2,-2.7960336E-3,4.7140596E-3,-1.0015958E-2,3.8398627E-2,1.020122E-1,7.4878554E-3,1.9448932E-2,1.5711059E-1,-9.084646E-3,7.4977684E-4,-8.284578E-2,-8.377057E-3,8.189429E-4,5.7310034E-2,-4.6313256E-3,-1.7774695E-3,1.6084831E-3,9.146783E-2,2.0953406E-2,1.356386E-1,2.4827484E-3,5.5649597E-3,1.3866115E-2,-1.1932741E-1,-7.1422726E-2,6.8261633E-3,1.3157097E-3,6.8788333E-3,1.6435911E-3,-2.7320227E-3,3.381934E-2,8.584644E-4,1.5632157E-1,-6.8410193E-3,-2.1042959E-4,-3.0364718E-3,-6.7787706E-3,-3.3058874E-5,3.4866333E-3,2.0501958E-4,8.401678E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,35,37,-1,-1,39,-1,-1,41,-1,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6522483E0,2.7819294E-1,5.707971E-1,1.5965658E-1,7.332815E-2,3.3425763E-1,4.3028653E-1,1.0379493E-1,1.1051449E-1,6.843434E-2,0E0,1.1004412E-1,0E0,5.2734494E-2,3.8537806E-1,7.629156E-2,1.2046149E-1,5.9466973E-2,0E0,0E0,5.2850112E-2,7.029615E-2,6.915882E-2,0E0,0E0,8.8516235E-2,0E0,0E0,6.398857E-2,0E0,0E0,5.3672895E-2,0E0,0E0,0E0,4.903704E-2,6.106589E-2,6.07903E-2,0E0,0E0,0E0,9.2232585E-2,7.715243E-2,0E0,0E0,0E0,0E0,0E0,6.691547E-2,0E0,4.950708E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,25,25,28,28,31,31,35,35,36,36,37,37,41,41,42,42,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,36,38,-1,-1,40,-1,-1,42,-1,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.10711E8,6.204522E3,1.347372E3,1E0,9.05929E2,1.417988E6,2.0964778E5,6.7211235E1,2.829932E0,5.254375E-3,4.1331047E-1,-6.796745E-3,9.256843E3,1.5096262E7,1E0,2.1066585E2,4.850336E6,-2.7960336E-3,4.7140596E-3,2.8716172E7,4.41341E5,5.3699964E7,7.4878554E-3,1.9448932E-2,1.6828056E1,-9.084646E-3,7.4977684E-4,3.606E3,-8.377057E-3,8.189429E-4,1E0,-4.6313256E-3,-1.7774695E-3,1.6084831E-3,1.14E2,1.1883916E3,1.3183452E-1,2.4827484E-3,5.5649597E-3,1.3866115E-2,6.1859414E4,1E0,6.8261633E-3,1.3157097E-3,6.8788333E-3,1.6435911E-3,-2.7320227E-3,2.0663544E9,8.584644E-4,1.2170949E0,-6.8410193E-3,-2.1042959E-4,-3.0364718E-3,-6.7787706E-3,-3.3058874E-5,3.4866333E-3,2.0501958E-4,8.401678E-3],"split_indices":[2,7,60,60,24,66,37,36,64,66,0,43,0,4,9,19,60,37,0,0,9,37,52,0,0,43,0,0,2,0,0,22,0,0,0,3,60,47,0,0,0,41,79,0,0,0,0,0,12,0,61,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.72E2,3.09E2,1.63E2,2.51E2,5.8E1,1.28E2,3.5E1,2.07E2,4.4E1,5.1E1,7E0,1.2E2,8E0,1.1E1,2.4E1,1.81E2,2.6E1,2.4E1,2E1,6E0,4.5E1,7.7E1,4.3E1,3E0,8E0,2.1E1,3E0,8E0,1.73E2,3E0,2.3E1,2.2E1,2E0,2.8E1,1.7E1,1.8E1,5.9E1,2.5E1,1.8E1,1.7E1,4E0,3.9E1,1.34E2,5E0,1.7E1,9E0,9E0,8E0,5.1E1,4E0,2.1E1,3.3E1,6E0,1.19E2,1.5E1,2.7E1,2.4E1,2E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[2.436813E-3,-2.1232488E-2,1.3112552E-1,-3.8423408E-2,6.4012684E-2,3.0480462E-1,9.871924E-2,-3.5317745E-2,-1.3901648E-2,7.4122734E-2,-5.5782436E-3,1.1271127E-3,1.7893218E-2,1.3052759E-1,1.1946867E-2,-4.0888265E-2,8.36463E-2,9.763977E-2,1.6443036E-3,-2.3572443E-3,1.4410567E-1,2.4754952E-3,-9.68747E-3,-4.7048364E-2,1.2913843E-3,6.586363E-3,-7.224547E-4,1.4954304E-3,5.8358787E-3,1.2843569E-2,1.1392501E-1,-5.4322906E-2,5.382388E-3,1.100015E-3,6.8294047E-3,-3.1491865E-3,-6.8022666E-4,-6.314563E-3,1.2835904E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,19,21,23,25,27,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4313449E0,5.842404E-1,3.8077533E-1,2.4190158E-1,1.2884414E-1,1.7784154E-1,1.7232603E-1,2.2078294E-1,0E0,5.7820976E-2,0E0,0E0,0E0,1.1847681E-1,1.5527664E-1,1.3126475E-1,7.6740235E-2,5.3753614E-2,0E0,0E0,1.23039365E-1,0E0,0E0,1.1013836E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.033979E-2,1.0037917E-1,1.0378487E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,13,13,14,14,15,15,16,16,17,17,20,20,23,23,30,30,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,20,22,24,26,28,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.5522031E9,1.064816E6,3.1822095E0,2.4070468E7,8.511305E6,1.5096262E7,2.2860639E9,-1.3901648E-2,1.1532659E7,-5.5782436E-3,1.1271127E-3,1.7893218E-2,1.1804314E0,1.92144E5,3.44E2,1.8607925E6,9.278185E2,1.6443036E-3,-2.3572443E-3,2.1207E4,2.4754952E-3,-9.68747E-3,2.1592189E5,1.2913843E-3,6.586363E-3,-7.224547E-4,1.4954304E-3,5.8358787E-3,1.2843569E-2,2E1,1.4720299E3,5.8955963E1,1.100015E-3,6.8294047E-3,-3.1491865E-3,-6.8022666E-4,-6.314563E-3,1.2835904E-3],"split_indices":[60,7,37,65,56,1,9,12,0,53,0,0,0,61,2,8,55,60,0,0,9,0,0,36,0,0,0,0,0,0,8,60,58,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,3.96E2,7.2E1,3.3E2,6.6E1,1E1,6.2E1,3.27E2,3E0,6.3E1,3E0,2E0,8E0,4.5E1,1.7E1,3.13E2,1.4E1,3.9E1,2.4E1,3E0,4.2E1,1.5E1,2E0,2.87E2,2.6E1,9E0,5E0,1E1,2.9E1,7E0,3.5E1,2.52E2,3.5E1,8E0,2.7E1,2.02E2,5E1,4E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[4.574827E-4,-2.9296046E-2,7.406653E-2,-6.442463E-2,-4.841948E-3,8.7421E-2,-8.4422156E-2,-7.690429E-2,-2.461857E-3,5.1041137E-2,-3.191328E-2,7.2295494E-2,9.1185495E-3,-7.328183E-3,1.1515123E-3,-4.5621875E-3,-3.7843898E-2,-7.143473E-3,8.424079E-4,3.4745205E-2,7.983861E-3,2.4260003E-3,-5.11076E-2,7.814607E-2,-7.965088E-3,-2.8982023E-3,2.0244732E-3,4.646155E-2,-4.8421235E-3,3.5172645E-2,-5.1130418E-2,-4.217791E-3,-2.6633443E-2,7.0702836E-2,1.54517675E-2,1.5324875E-3,5.4073064E-3,4.246603E-3,-9.137494E-5,-6.032876E-4,-7.7181896E-3,-6.6708424E-4,-7.179184E-3,5.647579E-2,1.6016088E-1,-3.2617457E-3,3.006797E-3,1.6117774E-2,5.1337E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,-1,25,-1,-1,27,-1,29,31,33,-1,-1,-1,35,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,-1,-1,-1,-1],"loss_changes":[1.0271502E0,2.8660667E-1,2.9311216E-1,1.0574931E-1,3.0116916E-1,1.7283523E-1,8.067653E-2,6.829363E-2,7.0005335E-2,1.1272912E-1,8.844593E-2,1.6374934E-1,0E0,0E0,0E0,0E0,6.1307333E-2,0E0,0E0,9.529746E-2,0E0,8.7734245E-2,7.116899E-2,1.6819549E-1,0E0,0E0,0E0,4.896286E-2,0E0,5.920187E-2,7.64545E-2,0E0,7.9157755E-2,1.2586349E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.942297E-2,9.646666E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,16,16,19,19,21,21,22,22,23,23,27,27,29,29,30,30,32,32,33,33,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,-1,26,-1,-1,28,-1,30,32,34,-1,-1,-1,36,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,-1,-1,-1,-1],"split_conditions":[1.2587E4,1.6858337E3,1.0429407E3,2.1592189E5,1.24E2,8.625562E0,1.19573875E5,5.175E3,3.516E3,1.7670108E5,2.6505424E1,3.768034E4,9.1185495E-3,-7.328183E-3,1.1515123E-3,-4.5621875E-3,2.033552E6,-7.143473E-3,8.424079E-4,1.1153E4,7.983861E-3,3.271774E7,7.4E1,1.4609149E4,-7.965088E-3,-2.8982023E-3,2.0244732E-3,3.8781055E1,-4.8421235E-3,4.1286E4,1E0,-4.217791E-3,9.06E2,9E0,1.54517675E-2,1.5324875E-3,5.4073064E-3,4.246603E-3,-9.137494E-5,-6.032876E-4,-7.7181896E-3,-6.6708424E-4,-7.179184E-3,4.56E2,3.7137297E3,-3.2617457E-3,3.006797E-3,1.6117774E-2,5.1337E-3],"split_indices":[2,4,66,36,3,43,36,2,12,36,61,4,0,0,0,0,9,0,0,2,0,53,8,4,0,0,0,61,0,11,21,0,0,25,0,0,0,0,0,0,0,0,0,10,40,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.67E2,3.33E2,1.34E2,1.36E2,1.97E2,1.24E2,1E1,1.13E2,2.3E1,6.4E1,1.33E2,1.09E2,1.5E1,6E0,4E0,7.9E1,3.4E1,2E0,2.1E1,5.7E1,7E0,4.8E1,8.5E1,1.07E2,2E0,2.7E1,7E0,5.3E1,4E0,3E1,1.8E1,3.4E1,5.1E1,1.05E2,2E0,4.4E1,9E0,1.2E1,1.8E1,1.4E1,4E0,4.7E1,4E0,9.2E1,1.3E1,3E0,8.9E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.5934088E-3,-4.1360766E-2,6.2841244E-2,-4.47931E-2,1.5199084E-2,2.220483E-2,1.3720167E-1,-4.108324E-2,-1.4928173E-2,-2.888602E-2,4.057673E-2,2.7417248E-1,9.884521E-2,-8.7294094E-2,-2.8502475E-2,-8.462025E-3,-6.5417173E-3,2.875106E-3,-7.749053E-3,7.7884113E-3,1.8746572E-2,1.1791116E-1,-1.7221817E-3,1.8512653E-3,-9.4292164E-2,-4.5554306E-2,1.1367509E-2,-4.965925E-3,5.924116E-4,-4.0801805E-3,1.0800629E-3,3.6044056E-3,7.7962377E-3,-6.079185E-3,-5.701112E-2,-1.9499399E-2,-8.4136255E-2,-6.522582E-4,2.6297374E-3,-3.6782552E-3,1.9993377E-3,-1.9327662E-3,2.3121599E-3,-8.598214E-3,-3.5254154E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,27,-1,-1,29,-1,-1,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.199935E0,3.719992E-1,5.287799E-1,2.716257E-1,0E0,1.0911523E-1,2.9862964E-1,1.6727129E-1,0E0,6.504255E-2,7.460758E-2,1.0139972E-1,1.318483E-1,5.781421E-2,1.5756463E-1,5.2696094E-2,0E0,0E0,5.5749338E-2,0E0,0E0,6.85665E-2,0E0,0E0,5.820352E-2,1.6120973E-1,7.3683605E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.2328117E-2,1.319249E-1,5.9988916E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,21,21,24,24,25,25,26,26,34,34,35,35,36,36],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,28,-1,-1,30,-1,-1,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,5.6503E4,1.0067637E7,1.5199084E-2,1.0643513E3,2.35312E5,3.836E3,-1.4928173E-2,1E0,7.309061E1,1.99814E5,5.30113E5,5.5347366E0,1.2028214E5,1.7895421E0,-6.5417173E-3,2.875106E-3,3.685938E6,7.7884113E-3,1.8746572E-2,1.2384782E5,-1.7221817E-3,1.8512653E-3,2.39E2,3.4129692E-3,2.735322E6,-4.965925E-3,5.924116E-4,-4.0801805E-3,1.0800629E-3,3.6044056E-3,7.7962377E-3,-6.079185E-3,1.222E3,7.198E3,2.9648642E1,-6.522582E-4,2.6297374E-3,-3.6782552E-3,1.9993377E-3,-1.9327662E-3,2.3121599E-3,-8.598214E-3,-3.5254154E-3],"split_indices":[41,60,2,55,0,60,11,2,0,79,64,2,2,55,36,61,0,0,1,0,0,36,0,0,3,65,1,0,0,0,0,0,0,0,10,2,64,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.71E2,2.96E2,1.75E2,2.94E2,2E0,1.14E2,6.1E1,2.91E2,3E0,3E1,8.4E1,1.2E1,4.9E1,6.1E1,2.3E2,2.6E1,4E0,6.1E1,2.3E1,7E0,5E0,4.3E1,6E0,3E0,5.8E1,1.61E2,6.9E1,4E0,2.2E1,6E0,1.7E1,2.2E1,2.1E1,3.1E1,2.7E1,9.7E1,6.4E1,4.4E1,2.5E1,2.3E1,4E0,7.5E1,2.2E1,6E0,5.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-8.073527E-4,-3.1038564E-2,9.697474E-2,-4.79223E-2,1.7069004E-2,8.1164435E-2,1.4240268E-2,-5.708778E-2,-3.8083372E-4,2.390454E-2,-6.998184E-3,1.3382283E-1,4.0317897E-2,-6.13981E-2,1.5479414E-3,1.4569487E-3,-2.4480172E-3,7.10973E-4,4.4623874E-3,1.6352605E-2,1.1162337E-1,8.23161E-2,-4.7794323E-2,-5.7894856E-2,-7.058268E-3,-1.0945666E-4,6.2313583E-3,1.7998703E-1,5.318415E-2,4.4209473E-3,-8.791532E-2,-7.539034E-2,-3.7062634E-2,3.452557E-3,1.468051E-2,6.6801146E-2,-5.161491E-3,-1.2211825E-3,-8.109797E-3,-5.4878165E-4,-4.07313E-3,1.5687605E-3,-2.4639678E-3,1.1724947E-3,5.174481E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,23,-1,-1,-1,-1,-1,-1,25,27,29,31,-1,-1,-1,33,35,-1,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3894074E0,2.9241046E-1,3.1585824E-1,1.158852E-1,1.06073596E-1,2.1829957E-1,0E0,8.7151825E-2,6.707267E-2,5.6110576E-2,0E0,1.6221207E-1,2.2596756E-1,5.3252995E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.414621E-2,1.0708669E-1,1.186164E-1,7.2286725E-2,0E0,0E0,0E0,9.665853E-2,7.822643E-2,0E0,7.3976874E-2,5.3719044E-2,8.899483E-2,0E0,0E0,4.8956394E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,13,13,20,20,21,21,22,22,23,23,27,27,28,28,30,30,31,31,32,32,35,35],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,24,-1,-1,-1,-1,-1,-1,26,28,30,32,-1,-1,-1,34,36,-1,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1722E4,7.164074E8,1.0871896E10,1.46705E3,1E0,1.423795E7,1.4240268E-2,2.9063344E5,7.094791E1,1E0,-6.998184E-3,1.01267644E11,6.27427E5,4.086885E7,1.5479414E-3,1.4569487E-3,-2.4480172E-3,7.10973E-4,4.4623874E-3,1.6352605E-2,1.1E1,1.3812E4,5.3930223E4,4.844E3,-7.058268E-3,-1.0945666E-4,6.2313583E-3,2.9632012E7,1.4656219E8,4.4209473E-3,7.6559805E3,1.4297844E2,1.3038821E-5,3.452557E-3,1.468051E-2,3.7288462E6,-5.161491E-3,-1.2211825E-3,-8.109797E-3,-5.4878165E-4,-4.07313E-3,1.5687605E-3,-2.4639678E-3,1.1724947E-3,5.174481E-3],"split_indices":[2,7,7,60,79,53,0,36,64,24,0,39,9,53,0,0,0,0,0,0,8,9,36,2,0,0,0,1,1,0,4,41,46,0,0,55,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,3.58E2,1.1E2,2.65E2,9.3E1,1.03E2,7E0,2.22E2,4.3E1,9E1,3E0,4.4E1,5.9E1,2.12E2,1E1,2.7E1,1.6E1,8E1,1E1,3E0,4.1E1,4E1,1.9E1,2.05E2,7E0,5E0,3.6E1,8E0,3.2E1,4E0,1.5E1,1.1E2,9.5E1,5E0,3E0,3E1,2E0,9E0,6E0,1.2E1,9.8E1,1.5E1,8E1,1.5E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.622732E-3,-1.7010402E-2,1.4166452E-1,-3.1118583E-2,5.8479752E-2,1.0734597E-1,2.7853963E-1,-2.8227655E-2,-1.4080854E-2,6.77913E-2,-3.6419432E-3,6.2665604E-3,2.3820587E-3,7.168884E-4,1.6455727E-2,-4.0141027E-2,1.4715506E-2,4.623988E-2,1.21495575E-1,-5.3788754E-3,-3.4781344E-2,7.240145E-2,-4.8212547E-4,1.7863048E-2,5.584512E-3,8.131079E-3,1.3816189E-3,-2.8919188E-2,-9.8749936E-2,5.054445E-3,-8.922157E-5,-7.799929E-5,5.4999106E-3,-1.2263838E-3,-7.790986E-3,-2.6698476E-3,-9.117071E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,19,21,23,25,-1,27,29,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2888526E0,4.6617234E-1,2.4254656E-1,2.64262E-1,9.032357E-2,5.145836E-2,1.6064477E-1,1.8732399E-1,0E0,6.961328E-2,0E0,0E0,0E0,0E0,0E0,1.02788E-1,1.1399309E-1,9.009633E-2,6.976506E-2,0E0,9.72676E-2,5.4817423E-2,0E0,6.437879E-2,0E0,0E0,0E0,1.2105107E-1,7.484165E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,15,15,16,16,17,17,18,18,20,20,21,21,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,20,22,24,26,-1,28,30,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,1.5522031E9,3.305542E4,3.764913E7,7.2125336E2,1.7427103E2,1.5435187E2,1.4720299E3,-1.4080854E-2,9.3058E4,-3.6419432E-3,6.2665604E-3,2.3820587E-3,7.168884E-4,1.6455727E-2,1.44E2,1.03E2,1.1593098E2,8.80085E5,-5.3788754E-3,1E0,1.1868755E7,-4.8212547E-4,3.9596844E9,5.584512E-3,8.131079E-3,1.3816189E-3,4.086885E7,1.3486675E8,5.054445E-3,-8.922157E-5,-7.799929E-5,5.4999106E-3,-1.2263838E-3,-7.790986E-3,-2.6698476E-3,-9.117071E-3],"split_indices":[60,7,4,56,66,66,66,60,0,2,0,0,0,0,0,0,3,62,11,0,79,56,0,7,0,0,0,53,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,4.35E2,5.7E1,3.67E2,6.8E1,4.7E1,1E1,3.64E2,3E0,6.4E1,4E0,3.4E1,1.3E1,2E0,8E0,2.85E2,7.9E1,4.7E1,1.7E1,1.9E1,2.66E2,2.3E1,5.6E1,3.4E1,1.3E1,1.1E1,6E0,2.45E2,2.1E1,1.6E1,7E0,2.9E1,5E0,2.39E2,6E0,1.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.5496023E-4,-1.9110058E-2,9.600828E-2,-3.1401142E-2,5.132286E-2,2.3088834E-1,6.958758E-2,-2.8467525E-2,-1.3562972E-2,1.1745863E-2,4.088358E-2,1.9177442E-3,1.3952182E-2,8.395152E-2,-9.7619735E-2,-3.749861E-2,1.1770503E-2,2.4364276E-3,-4.5378893E-3,2.6958932E-3,6.282662E-3,-8.755098E-3,2.099125E-3,-4.1728135E-2,1.5146577E-3,4.517454E-3,-6.164035E-4,-6.9708023E-3,-3.8864024E-2,-2.2446974E-3,4.9302215E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,-1,-1,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,29,-1,-1],"loss_changes":[9.0482986E-1,3.5481596E-1,2.7615035E-1,2.424249E-1,1.1005138E-1,1.2256688E-1,1.7631829E-1,1.2581202E-1,0E0,0E0,7.3868684E-2,0E0,0E0,7.614043E-2,7.5816095E-2,8.285397E-2,1.263236E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.0980996E-2,0E0,0E0,0E0,0E0,8.834243E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,13,13,14,14,15,15,16,16,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,-1,-1,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,30,-1,-1],"split_conditions":[3.2991206E3,1.613448E9,1E0,3.372042E7,1.4E1,5.993877E3,3.768034E4,1.4720299E3,-1.3562972E-2,1.1745863E-2,2.4070468E7,1.9177442E-3,1.3952182E-2,1.7552137E0,9.820362E4,4.8384636E7,7.9E1,2.4364276E-3,-4.5378893E-3,2.6958932E-3,6.282662E-3,-8.755098E-3,2.099125E-3,2.398E3,1.5146577E-3,4.517454E-3,-6.164035E-4,-6.9708023E-3,2.1592189E5,-2.2446974E-3,4.9302215E-4],"split_indices":[60,7,23,56,3,4,4,60,0,0,56,0,0,50,36,9,3,0,0,0,0,0,0,2,0,0,0,0,36,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,4.07E2,8.1E1,3.47E2,6E1,1.2E1,6.9E1,3.44E2,3E0,2E0,5.8E1,3E0,9E0,6.4E1,5E0,2.81E2,6.3E1,5.5E1,3E0,4E1,2.4E1,3E0,2E0,2.65E2,1.6E1,1.4E1,4.9E1,6E0,2.59E2,2.27E2,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5132208E-3,-8.829693E-3,1.5745689E-1,-3.306252E-2,3.3374254E-2,2.802149E-1,9.190057E-2,-2.9659614E-2,-1.3064137E-2,6.0940348E-3,7.633063E-2,5.4180464E-3,1.6185466E-2,1.3761143E-1,-7.828033E-3,-5.8121467E-3,-2.3912214E-2,-9.867155E-3,3.7435628E-3,8.909192E-2,-1.9767657E-3,2.6387249E-3,8.758201E-3,1.6624911E-3,-2.8459849E-2,-3.5225782E-3,7.15003E-5,7.305861E-2,8.8352375E-3,-7.6753786E-3,-2.4108985E-2,-6.82563E-4,4.16659E-3,-1.8679391E-3,2.036362E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,33,-1,-1,-1,-1],"loss_changes":[8.5026896E-1,4.623506E-1,2.3981762E-1,2.199871E-1,1.9289163E-1,5.658412E-2,2.8641215E-1,1.4221096E-1,0E0,1.1511422E-1,9.885439E-2,0E0,0E0,5.908364E-2,0E0,0E0,7.142614E-2,5.9053093E-2,0E0,7.1531475E-2,0E0,0E0,0E0,0E0,1.3495803E-1,0E0,0E0,5.39971E-2,0E0,0E0,9.578471E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,16,16,17,17,19,19,24,24,27,27,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,34,-1,-1,-1,-1],"split_conditions":[7.3068325E3,9.7467445E4,1.417988E6,7.339209E6,1.4931864E9,1.1254387E7,1.5096262E7,1.44E2,-1.3064137E-2,1.5271514E5,5.878961E7,5.4180464E-3,1.6185466E-2,6.274818E4,-7.828033E-3,-5.8121467E-3,2E1,2.8529238E1,3.7435628E-3,1.8600049E10,-1.9767657E-3,2.6387249E-3,8.758201E-3,1.6624911E-3,2.1066585E2,-3.5225782E-3,7.15003E-5,3.3361957E0,8.8352375E-3,-7.6753786E-3,1.2028214E5,-6.82563E-4,4.16659E-3,-1.8679391E-3,2.036362E-4],"split_indices":[60,41,37,55,7,53,9,0,0,36,1,0,0,36,0,0,10,64,0,5,0,0,0,0,60,0,0,61,0,0,36,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,4.5E2,3.2E1,2.86E2,1.64E2,1E1,2.2E1,2.83E2,3E0,1.01E2,6.3E1,3E0,7E0,1.9E1,3E0,1.6E1,2.67E2,8.3E1,1.8E1,5.7E1,6E0,7E0,1.2E1,1.9E1,2.48E2,1.2E1,7.1E1,5E1,7E0,7E0,2.41E2,6E0,4.4E1,1.61E2,8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.0551132E-3,-2.0824187E-2,7.629015E-2,-1.8561646E-2,-1.3810179E-2,1.8686402E-1,6.3364126E-2,-3.2392506E-2,3.625121E-2,3.5568813E-3,1.3243082E-2,5.274432E-2,1.9692095E-1,-1.8531777E-2,-5.8922656E-2,3.9038603E-4,3.7069784E-3,6.171075E-2,-3.3106867E-3,1.5994892E-2,-4.125625E-3,-3.808349E-2,1.2790554E-2,-9.240063E-3,-5.278517E-2,6.7755684E-2,-1.5788574E-3,-4.3254606E-2,5.7631275E-3,1.2379847E-3,-2.5805724E-3,1.2561872E-3,-6.0376175E-2,-1.5550246E-3,3.6552765E-3,-3.1682982E-3,-3.428368E-4,-7.956214E-3,-2.4866746E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,-1,27,29,-1,31,33,-1,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9306436E-1,2.0542155E-1,1.4538038E-1,2.7030292E-1,0E0,8.503476E-2,1.314483E-1,1.0334262E-1,7.953311E-2,0E0,0E0,1.0552296E-1,2.817131E-1,1.1561928E-1,6.7150235E-2,0E0,0E0,5.2135557E-2,0E0,0E0,0E0,9.8422855E-2,6.0800362E-2,0E0,5.749017E-2,5.8357924E-2,0E0,8.704883E-2,0E0,0E0,0E0,0E0,7.646999E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,21,21,22,22,24,24,25,25,27,27,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,-1,28,30,-1,32,34,-1,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[1.615586E9,3.372042E7,1.3812E4,1.7985837E3,-1.3810179E-2,2.822E3,2.3767492E4,1.5881974E7,9.799349E4,3.5568813E-3,1.3243082E-2,9.05929E2,2.043E3,1.523794E6,1.6013113E7,3.9038603E-4,3.7069784E-3,6.029086E10,-3.3106867E-3,1.5994892E-2,-4.125625E-3,8.32E3,6.55629E1,-9.240063E-3,5.125401E1,1E1,-1.5788574E-3,1.7685201E3,5.7631275E-3,1.2379847E-3,-2.5805724E-3,1.2561872E-3,7.0151276E1,-1.5550246E-3,3.6552765E-3,-3.1682982E-3,-3.428368E-4,-7.956214E-3,-2.4866746E-3],"split_indices":[7,56,9,60,0,0,60,53,36,0,0,66,0,1,53,0,0,5,0,0,0,2,66,0,56,8,0,4,0,0,0,0,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.65E2,3.56E2,1.09E2,3.54E2,2E0,1E1,9.9E1,2.83E2,7.1E1,5E0,5E0,9.3E1,6E0,1.87E2,9.6E1,4.2E1,2.9E1,8.7E1,6E0,4E0,2E0,1.15E2,7.2E1,3E0,9.3E1,8.2E1,5E0,1.12E2,3E0,6.1E1,1.1E1,8E0,8.5E1,5E0,7.7E1,7E1,4.2E1,6E0,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.0685377E-3,-2.1490717E-2,6.7997895E-2,-2.9680919E-2,5.587719E-2,5.6843363E-2,1.1922865E-2,-2.6853655E-2,-1.235664E-2,1.7295696E-3,8.457781E-3,6.386376E-2,-7.6350584E-3,-3.1554804E-3,-1.8809266E-2,5.5258345E-2,1.3953876E-2,-5.774051E-3,-4.4670705E-2,2.2440045E-3,7.333344E-2,-1.8273234E-2,4.302747E-2,1.2856757E-2,-6.34716E-2,5.213602E-3,-6.793516E-4,8.263053E-2,-5.520562E-4,-5.5108783E-3,-6.555238E-4,-1.4506625E-4,3.6541764E-3,-3.091552E-3,2.125997E-3,-8.754133E-3,-2.525884E-3,-1.374645E-3,4.3818536E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,-1,-1,17,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.307317E-1,2.3104298E-1,2.2450942E-1,1.9937736E-1,7.801316E-2,1.8302622E-1,0E0,9.681171E-2,0E0,0E0,0E0,1.988712E-1,0E0,0E0,9.0424486E-2,1.05241776E-1,0E0,1.105825E-1,9.815715E-2,5.0055698E-2,6.541914E-2,6.497673E-2,5.3796828E-2,5.5970475E-2,8.43972E-2,0E0,0E0,5.7567716E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,-1,-1,18,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5522031E9,2.9034685E3,1.0871896E10,4.4533052E0,8.10288E3,3.7826266E4,1.1922865E-2,5.63321E5,-1.235664E-2,1.7295696E-3,8.457781E-3,3.305542E4,-7.6350584E-3,-3.1554804E-3,1.8071064E7,9.043313E4,1.3953876E-2,1.2096E4,2.6E1,1.12608824E8,1.5953E4,3.3E1,1.8E9,1E0,1.7683E4,5.213602E-3,-6.793516E-4,8.946923E2,-5.520562E-4,-5.5108783E-3,-6.555238E-4,-1.4506625E-4,3.6541764E-3,-3.091552E-3,2.125997E-3,-8.754133E-3,-2.525884E-3,-1.374645E-3,4.3818536E-3],"split_indices":[7,60,7,65,60,60,0,1,0,0,0,4,0,0,53,41,0,10,8,5,0,3,5,13,9,0,0,60,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,3.61E2,1.21E2,3.27E2,3.4E1,1.15E2,6E0,3.24E2,3E0,3E1,4E0,1.12E2,3E0,5.6E1,2.68E2,1.09E2,3E0,1.79E2,8.9E1,2.8E1,8.1E1,1.43E2,3.6E1,2.2E1,6.7E1,3E0,2.5E1,7.3E1,8E0,6E0,1.37E2,1.5E1,2.1E1,6E0,1.6E1,5E0,6.2E1,4E0,6.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.0286809E-3,-2.3620002E-2,5.117074E-2,-3.2595966E-2,3.072786E-2,3.8399205E-2,8.824233E-3,-3.9964598E-2,1.2531661E-3,5.6364425E-3,-1.8802326E-4,8.055721E-2,1.19635165E-2,-2.9572712E-2,-6.82602E-2,6.294628E-2,1.2678576E-2,-4.4842246E-3,2.5087615E-2,-4.12947E-2,2.318389E-2,-1.22635856E-1,-4.1355424E-2,2.1955776E-3,6.4078365E-3,3.2411758E-2,-7.3209684E-3,-3.2111001E-3,-1.32299885E-2,6.5316595E-2,-5.728469E-4,-4.2231463E-2,-1.7505187E-1,1.10021836E-4,-8.694756E-2,2.7276613E-4,3.843205E-3,1.2779175E-3,-1.6185556E-3,4.228191E-4,6.717247E-3,-4.0886053E-3,3.9996575E-3,-9.927117E-3,-7.994632E-4,-2.6455345E-3,-8.40803E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,-1,35,-1,-1,37,39,-1,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.410722E-1,1.644228E-1,2.1675918E-1,1.24426216E-1,1.4014347E-1,1.3834576E-1,0E0,7.325676E-2,0E0,0E0,0E0,1.3417983E-1,1.0814829E-1,1.1783728E-1,9.440997E-2,4.953532E-2,0E0,0E0,9.423977E-2,1.0434312E-1,5.2177005E-2,8.532581E-2,9.362031E-2,0E0,0E0,8.347399E-2,0E0,0E0,5.7578526E-2,6.3823655E-2,0E0,5.773219E-2,5.3207994E-2,0E0,5.4892257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,12,12,13,13,14,14,15,15,18,18,19,19,20,20,21,21,22,22,25,25,28,28,29,29,31,31,32,32,34,34],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,-1,36,-1,-1,38,40,-1,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2587E4,1.4720299E3,1.5820756E1,3.94E2,1.08E2,1.3307285E7,8.824233E-3,1.8071064E7,1.2531661E-3,5.6364425E-3,-1.8802326E-4,5.462141E3,3.5226266E0,1.175502E6,2.4851996E7,9.243444E-1,1.2678576E-2,-4.4842246E-3,3.768034E4,7.6E1,1.6176678E3,8.3896356E2,2.0353708E3,2.1955776E-3,6.4078365E-3,4.0601485E6,-7.3209684E-3,-3.2111001E-3,5.1E2,4.350778E2,-5.728469E-4,3.312442E6,2.3808822E-2,1.10021836E-4,3.069012E6,2.7276613E-4,3.843205E-3,1.2779175E-3,-1.6185556E-3,4.228191E-4,6.717247E-3,-4.0886053E-3,3.9996575E-3,-9.927117E-3,-7.994632E-4,-2.6455345E-3,-8.40803E-3],"split_indices":[2,60,43,8,3,56,0,53,0,0,0,4,62,37,53,43,0,0,4,8,41,41,63,0,0,55,0,0,11,60,0,9,46,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.69E2,3.34E2,1.35E2,2.87E2,4.7E1,1.24E2,1.1E1,2.55E2,3.2E1,1.3E1,3.4E1,4.7E1,7.7E1,1.88E2,6.7E1,4.4E1,3E0,8E0,6.9E1,1.54E2,3.4E1,2.1E1,4.6E1,3.6E1,8E0,6.7E1,2E0,8.2E1,7.2E1,1.5E1,1.9E1,9E0,1.2E1,2.4E1,2.2E1,4.3E1,2.4E1,2.4E1,4.8E1,9E0,6E0,7E0,2E0,1E1,2E0,1.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[5.493215E-6,-1.2167718E-2,1.0455179E-1,-3.2944556E-2,2.5601596E-2,2.477014E-1,6.7979224E-2,-2.9909844E-2,-1.1629019E-2,1.1286709E-2,3.666531E-3,4.646858E-3,1.4697535E-2,9.480333E-2,-5.787369E-2,-5.916308E-3,-2.468075E-2,-2.8898296E-4,4.7999598E-2,1.0330992E-2,6.8470076E-2,3.3281562E-3,-5.448165E-3,-3.0985363E-2,1.1824174E-3,5.4666894E-3,1.0919508E-3,-3.0035763E-3,4.175753E-3,-4.5754846E-2,-5.164583E-3,-2.650869E-4,-3.2304602E-3,-8.43551E-3,1.3937999E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,19,21,-1,23,-1,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[6.223323E-1,3.4455383E-1,2.501642E-1,1.6827378E-1,1.0895009E-1,5.1145434E-2,1.4727414E-1,1.2987196E-1,0E0,7.722998E-2,0E0,0E0,0E0,9.236476E-2,6.108067E-2,0E0,8.257353E-2,0E0,6.12213E-2,0E0,7.01333E-2,0E0,0E0,8.976455E-2,0E0,0E0,0E0,0E0,0E0,1.214211E-1,1.16716765E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,13,13,14,14,16,16,18,18,20,20,23,23,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,20,22,-1,24,-1,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[4.23168E3,8.393928E4,1.064816E6,5.38168E6,1.7670108E5,9.256843E3,1.5096262E7,1.44E2,-1.1629019E-2,1.484798E9,3.666531E-3,4.646858E-3,1.4697535E-2,1.3812E4,2.9632012E7,-5.916308E-3,3.94E2,-2.8898296E-4,6.3349745E6,1.0330992E-2,3.5366E4,3.3281562E-3,-5.448165E-3,1.173433E5,1.1824174E-3,5.4666894E-3,1.0919508E-3,-3.0035763E-3,4.175753E-3,1.2755817E4,6.105804E2,-2.650869E-4,-3.2304602E-3,-8.43551E-3,1.3937999E-4],"split_indices":[60,41,37,55,36,4,9,0,0,7,0,0,0,9,1,0,8,0,53,0,9,0,0,36,0,0,0,0,0,58,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.87E2,4.37E2,5E1,2.82E2,1.55E2,9E0,4.1E1,2.79E2,3E0,1.21E2,3.4E1,3E0,6E0,3.4E1,7E0,1.4E1,2.65E2,8.3E1,3.8E1,5E0,2.9E1,2E0,5E0,2.35E2,3E1,1E1,2.8E1,3E0,2.6E1,1.49E2,8.6E1,5E1,9.9E1,3E0,8.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.311898E-3,-1.11486185E-2,1.01450056E-1,-3.1890213E-2,1.9297296E-2,2.43681E-1,6.601378E-2,-2.8565027E-2,-1.1201313E-2,9.204946E-3,1.040102E-1,1.8610496E-2,7.958391E-3,1.1956229E-2,4.1526627E-2,-4.7440495E-2,-5.2930554E-4,1.8778643E-4,2.5146229E-3,8.704888E-3,2.5729765E-3,3.1710602E-3,-5.6214117E-2,-6.794703E-2,-1.9411272E-3,-6.720828E-4,2.3346802E-3,-6.4021572E-3,5.4531638E-3,-2.8033156E-2,-4.3215076E-3,2.9119058E-3,-1.2731506E-3,-3.0660175E-3,2.1453567E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,23,-1,25,-1,-1,-1,-1,27,29,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[5.863366E-1,2.6458108E-1,2.4600065E-1,1.5619355E-1,1.4438447E-1,4.896325E-2,1.7708635E-1,8.2007304E-2,0E0,5.8035523E-2,5.8876216E-2,0E0,0E0,0E0,9.183252E-2,1.1089918E-1,0E0,8.432165E-2,0E0,0E0,0E0,0E0,1.1408024E-1,6.356749E-2,5.7514668E-2,0E0,0E0,0E0,0E0,7.4529216E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,15,15,17,17,22,22,23,23,24,24,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,24,-1,26,-1,-1,-1,-1,28,30,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,7.222818E4,1.37893E6,5.38168E6,9.3058E4,4.542969E1,1.3812E4,7.683712E4,-1.1201313E-2,3.73E2,2.1207E4,1.8610496E-2,7.958391E-3,1.1956229E-2,2.0531621E4,7.177E3,-5.2930554E-4,1.646359E5,2.5146229E-3,8.704888E-3,2.5729765E-3,3.1710602E-3,2.785032E9,1.1347253E1,1.572445E3,-6.720828E-4,2.3346802E-3,-6.4021572E-3,5.4531638E-3,1.303394E3,-4.3215076E-3,2.9119058E-3,-1.2731506E-3,-3.0660175E-3,2.1453567E-3],"split_indices":[60,41,37,55,2,62,9,36,0,8,9,0,0,0,4,2,0,36,0,0,0,0,7,66,55,0,0,0,0,56,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,4.17E2,5.1E1,2.48E2,1.69E2,9E0,4.2E1,2.45E2,3E0,1.52E2,1.7E1,2E0,7E0,4E0,3.8E1,1.18E2,1.27E2,1.26E2,2.6E1,6E0,1.1E1,3.1E1,7E0,8.1E1,3.7E1,9.8E1,2.8E1,5E0,2E0,2.8E1,5.3E1,1E1,2.7E1,1.9E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.3762696E-3,-2.1869948E-2,4.1498102E-2,-2.4818946E-2,1.2107609E-2,3.1093284E-2,6.6019814E-3,-2.1804022E-2,-1.06338775E-2,1.4409247E-2,6.1159547E-2,-3.27843E-3,-1.3898522E-2,-7.9291186E-4,3.192286E-2,8.7735616E-2,1.2984702E-3,2.445407E-3,-1.9684218E-2,4.864945E-3,1.0496385E-3,7.396775E-2,1.0316046E-2,1.9509841E-3,-3.9896537E-3,-6.2585645E-3,-1.5268192E-2,4.4620056E-3,-4.7794243E-4,-2.1277515E-3,-5.3640997E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,17,-1,19,21,23,-1,25,-1,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[4.6248958E-1,2.1854328E-1,1.915504E-1,1.4927146E-1,0E0,9.091641E-2,0E0,9.2735484E-2,0E0,6.424494E-2,1.0329276E-1,0E0,8.398989E-2,0E0,5.1663823E-2,6.041491E-2,7.003908E-2,0E0,9.672136E-2,0E0,0E0,6.0733244E-2,0E0,0E0,0E0,0E0,7.9671375E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,12,12,14,14,15,15,16,16,18,18,21,21,26,26],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,18,-1,20,22,24,-1,26,-1,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[8.393928E4,8.861901E3,8.625562E0,7.339209E6,1.2107609E-2,1.3590209E7,6.6019814E-3,4.26944E5,-1.06338775E-2,5E1,5.145648E7,-3.27843E-3,1.12608824E8,-7.9291186E-4,9.457892E6,2.47815E5,1.2852292E7,2.445407E-3,2.2744698E2,4.864945E-3,1.0496385E-3,4.307733E1,1.0316046E-2,1.9509841E-3,-3.9896537E-3,-6.2585645E-3,7.683712E4,4.4620056E-3,-4.7794243E-4,-2.1277515E-3,-5.3640997E-5],"split_indices":[41,60,43,55,0,1,0,1,0,8,1,0,5,0,40,2,9,0,60,0,0,65,0,0,0,0,36,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,2.67E2,2.01E2,2.65E2,2E0,1.82E2,1.9E1,2.62E2,3E0,1.18E2,6.4E1,3.8E1,2.24E2,4.3E1,7.5E1,4.4E1,2E1,1.8E1,2.06E2,9E0,6.6E1,4.1E1,3E0,1.4E1,6E0,7E0,1.99E2,3.4E1,7E0,6.6E1,1.33E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.5004417E-3,-1.3788984E-2,6.8667494E-2,-5.0163176E-2,-3.294341E-4,1.0068443E-1,-1.3882418E-2,-5.331028E-3,-4.0592387E-2,3.5259826E-3,-9.340699E-3,9.906038E-3,7.478037E-2,6.5073685E-4,-6.502407E-3,-5.0900716E-2,9.3925215E-4,-1.0690948E-2,2.349207E-3,4.6171667E-3,7.038097E-5,-1.9310714E-3,-6.0190526E-3,6.2812897E-3,-3.819586E-2,9.934033E-3,-5.90407E-3,-4.5918733E-2,2.1009268E-3,-4.969121E-4,1.5227007E-3,-1.2622967E-3,-4.4668373E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,19,-1,-1,21,-1,23,-1,-1,-1,-1,-1,25,27,29,-1,31,-1,-1,-1,-1,-1],"loss_changes":[4.1655695E-1,2.0120096E-1,1.916934E-1,5.7784945E-2,2.2133192E-1,1.21371984E-1,6.9450036E-2,0E0,6.071499E-2,1.8726808E-1,0E0,0E0,6.096326E-2,0E0,0E0,6.387107E-2,0E0,1.0527437E-1,0E0,0E0,0E0,0E0,0E0,6.597652E-2,5.5789948E-2,5.836863E-2,0E0,6.965734E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,12,12,15,15,17,17,23,23,24,24,25,25,27,27],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,20,-1,-1,22,-1,24,-1,-1,-1,-1,-1,26,28,30,-1,32,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,4.407E3,1.852084E7,6.506511E0,2.4070468E7,1E0,1.92144E5,-5.331028E-3,2.6076303E11,2.332E3,-9.340699E-3,9.906038E-3,1.9658574E2,6.5073685E-4,-6.502407E-3,5.28587E5,9.3925215E-4,1.423795E7,2.349207E-3,4.6171667E-3,7.038097E-5,-1.9310714E-3,-6.0190526E-3,1E0,6.33E2,7.104811E4,-5.90407E-3,6.54925E6,2.1009268E-3,-4.969121E-4,1.5227007E-3,-1.2622967E-3,-4.4668373E-3],"split_indices":[60,2,9,64,56,23,2,0,39,0,0,0,66,0,0,37,0,53,0,0,0,0,0,73,8,36,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.81E2,4.1E2,7.1E1,1.1E2,3E2,5.1E1,2E1,1.4E1,9.6E1,2.95E2,5E0,9E0,4.2E1,1.7E1,3E0,8.2E1,1.4E1,2.24E2,7.1E1,3.3E1,9E0,7.2E1,1E1,1.39E2,8.5E1,1.36E2,3E0,7.8E1,7E0,7E1,6.6E1,5.5E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.0245475E-3,-1.0171894E-2,1.2163909E-1,-2.931948E-2,1.9459417E-2,2.2481546E-1,4.84925E-2,-2.6121844E-2,-9.210208E-3,5.202745E-3,3.722732E-3,2.5118089E-3,1.3327406E-2,5.4494664E-3,-1.2373064E-4,-3.756408E-2,1.0471707E-2,-3.1696454E-2,9.221001E-4,-5.395006E-3,-3.3077408E-2,6.4292036E-2,-1.3549954E-3,6.791474E-4,-3.1080781E-3,-1.2942269E-2,-4.9532402E-2,1.0920385E-2,2.0505008E-3,-3.2173307E-4,-6.8913237E-3,-6.5633454E-3,-1.7513809E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,19,21,23,-1,-1,25,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1737887E-1,2.4397147E-1,2.3060206E-1,1.2737094E-1,1.356249E-1,9.204763E-2,6.4185336E-2,1.0800235E-1,0E0,6.8848155E-2,0E0,0E0,0E0,0E0,0E0,6.0073197E-2,1.2891987E-1,5.427008E-2,0E0,0E0,6.0985148E-2,8.326093E-2,0E0,0E0,0E0,6.828384E-2,1.1404818E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,15,15,16,16,17,17,20,20,21,21,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,20,22,24,-1,-1,26,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1],"split_conditions":[6.204522E3,8.393928E4,3.0418E4,7.339209E6,1.7670108E5,5.5580516E4,4.621003E6,7.198E3,-9.210208E-3,8.689165E-2,3.722732E-3,2.5118089E-3,1.3327406E-2,5.4494664E-3,-1.2373064E-4,2.398E3,2.909019E3,3.1061419E1,9.221001E-4,-5.395006E-3,1.2755817E4,4.8E1,-1.3549954E-3,6.791474E-4,-3.1080781E-3,3.9034148E7,2.600076E-4,1.0920385E-2,2.0505008E-3,-3.2173307E-4,-6.8913237E-3,-6.5633454E-3,-1.7513809E-3],"split_indices":[60,41,10,55,36,36,55,2,0,46,0,0,0,0,0,2,40,66,0,0,58,3,0,0,0,53,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.59E2,4.28E2,3.1E1,2.6E2,1.68E2,1.2E1,1.9E1,2.56E2,4E0,1.35E2,3.3E1,3E0,9E0,8E0,1.1E1,1.95E2,6.1E1,3.6E1,9.9E1,1E1,1.85E2,2.5E1,3.6E1,1.5E1,2.1E1,8.4E1,1.01E2,2E0,2.3E1,8.1E1,3E0,1.3E1,8.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.6851446E-3,-8.876933E-3,8.3412245E-2,-2.3140624E-2,1.9611971E-2,1.8821445E-1,5.5315956E-2,-2.0854574E-2,-1.1107603E-2,6.7381333E-3,7.324181E-2,1.6036078E-2,4.6729906E-3,-6.205937E-3,6.698942E-2,-4.1793175E-3,-1.6843991E-2,-4.6251267E-3,3.5233323E-3,8.1381004E-4,5.7597607E-3,9.44127E-3,4.870858E-2,2.1514774E-3,-2.1502906E-2,1.5342527E-3,-9.892043E-4,-6.224048E-4,3.8022592E-3,-4.1051548E-2,-8.100041E-3,-3.5529237E-3,-1.0369078E-3,7.0262357E-4,-2.1773663E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,-1,23,25,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[4.073114E-1,1.7031102E-1,1.4859459E-1,1.24984175E-1,9.616554E-2,1.1070627E-1,1.0189043E-1,7.007498E-2,0E0,8.46759E-2,6.56697E-2,0E0,0E0,0E0,8.700867E-2,0E0,7.505652E-2,5.5345003E-2,0E0,0E0,0E0,0E0,6.824372E-2,0E0,6.359738E-2,0E0,0E0,0E0,0E0,6.034988E-2,1.1950191E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,16,16,17,17,22,22,24,24,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,-1,24,26,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[4.23168E3,9.7467445E4,1.417988E6,5.825042E6,1.6846506E5,4.542969E1,1.379E3,1.44E2,-1.1107603E-2,3.53E2,3.5151117E-2,1.6036078E-2,4.6729906E-3,-6.205937E-3,1.3812E4,-4.1793175E-3,1.27E2,1.4669927E-2,3.5233323E-3,8.1381004E-4,5.7597607E-3,9.44127E-3,5.3474556E5,2.1514774E-3,7.683712E4,1.5342527E-3,-9.892043E-4,-6.224048E-4,3.8022592E-3,1.6845247E3,1.8071064E7,-3.5529237E-3,-1.0369078E-3,7.0262357E-4,-2.1773663E-3],"split_indices":[60,41,37,55,36,62,0,0,0,8,65,0,0,0,9,0,11,65,0,0,0,0,40,0,36,0,0,0,0,4,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.7E2,4.17E2,5.3E1,2.78E2,1.39E2,1E1,4.3E1,2.76E2,2E0,1.13E2,2.6E1,3E0,7E0,2E0,4.1E1,1.5E1,2.61E2,9.7E1,1.6E1,1.2E1,1.4E1,4E0,3.7E1,1.8E1,2.43E2,2.9E1,6.8E1,1.2E1,2.5E1,9.8E1,1.45E2,3.7E1,6.1E1,9E1,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.9114564E-3,-7.668522E-3,7.640325E-2,-2.6566839E-2,1.0652851E-2,1.5743595E-1,4.6557017E-2,-2.1793493E-3,-3.3522744E-3,1.4850393E-2,-8.309822E-3,4.4605555E-3,1.2410301E-2,3.0364883E-3,-6.3865124E-3,-8.5726875E-4,3.1803516E-3,-4.7689545E-4,3.1053858E-2,6.589666E-3,2.5761276E-2,3.868925E-2,-2.1637784E-2,2.4310749E-3,-5.8054423E-4,-7.840601E-3,-2.0510984E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,-1,-1,-1,-1,19,-1,21,23,25,-1,-1,-1,-1],"loss_changes":[3.7691602E-1,1.4709009E-1,1.4020002E-1,8.6574554E-2,1.6581328E-1,7.869986E-2,1.3300073E-1,0E0,8.979303E-2,8.472119E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.653405E-2,0E0,7.620833E-2,5.3936824E-2,6.44859E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,18,18,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,-1,-1,-1,-1,20,-1,22,24,26,-1,-1,-1,-1],"split_conditions":[4.0292856E3,6.24E3,1.417988E6,6.505339E2,4.2064876E7,1.92144E5,2.3767492E4,-2.1793493E-3,5.502256E9,7.104811E4,-8.309822E-3,4.4605555E-3,1.2410301E-2,3.0364883E-3,-6.3865124E-3,-8.5726875E-4,3.1803516E-3,-4.7689545E-4,1.2662238E1,6.589666E-3,1.9051096E7,1.1868755E7,6.79499E7,2.4310749E-3,-5.8054423E-4,-7.840601E-3,-2.0510984E-4],"split_indices":[60,2,37,60,58,2,60,0,5,36,0,0,0,0,0,0,0,0,64,0,53,56,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,4.23E2,6E1,2.08E2,2.15E2,1.5E1,4.5E1,1.17E2,9.1E1,2.11E2,4E0,1E1,5E0,4.2E1,3E0,7.6E1,1.5E1,8.4E1,1.27E2,5E0,1.22E2,9.6E1,2.6E1,7.9E1,1.7E1,2E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[8.07694E-4,-2.1228425E-2,3.1984765E-2,-3.1981857E-3,-1.3930141E-2,2.3746384E-2,7.800815E-3,-3.124979E-3,-4.4270832E-2,2.6445713E-2,-7.474867E-3,5.783151E-3,-7.2340886E-3,-8.020723E-2,-5.9843843E-4,2.2171091E-2,1.3791325E-2,-3.7211586E-2,1.33908065E-2,-5.2022026E-3,2.0653817E-4,1.1788731E-2,5.517391E-2,-4.1295844E-3,-9.206634E-4,-1.24222515E-2,2.2317392E-3,-3.2187311E-3,9.2149596E-4,6.8205833E-3,1.7428338E-3,5.502454E-4,-3.5059901E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,-1,17,19,-1,21,-1,23,25,-1,-1,27,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.318234E-1,8.994667E-2,2.0562418E-1,0E0,7.9853654E-2,9.3909994E-2,0E0,9.088741E-2,7.27742E-2,1.9742924E-1,0E0,0E0,1.09316066E-1,6.597912E-2,0E0,6.27393E-2,0E0,6.080968E-2,8.761898E-2,0E0,0E0,7.821982E-2,6.754787E-2,0E0,0E0,8.320294E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,12,12,13,13,15,15,17,17,18,18,21,21,22,22,25,25],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,-1,18,20,-1,22,-1,24,26,-1,-1,28,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.043313E4,4.26944E5,1.6941456E1,-3.1981857E-3,5.3403894E5,3.7826266E4,7.800815E-3,4E0,4.5593824E4,3.305542E4,-7.474867E-3,5.783151E-3,2.53383E5,1.4690361E5,-5.9843843E-4,9.3058E4,1.3791325E-2,2.55E2,7.683712E4,-5.2022026E-3,2.0653817E-4,5.767668E0,2.1207E4,-4.1295844E-3,-9.206634E-4,2.3099316E1,2.2317392E-3,-3.2187311E-3,9.2149596E-4,6.8205833E-3,1.7428338E-3,5.502454E-4,-3.5059901E-3],"split_indices":[41,1,43,0,56,60,0,8,41,4,0,0,9,36,0,2,0,3,36,0,0,62,9,0,0,66,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.81E2,2.82E2,1.99E2,3.9E1,2.43E2,1.88E2,1.1E1,1.8E2,6.3E1,1.86E2,2E0,5E0,1.75E2,2.9E1,3.4E1,1.84E2,2E0,7.1E1,1.04E2,2.2E1,7E0,1.41E2,4.3E1,1.9E1,5.2E1,5.8E1,4.6E1,1.1E1,1.3E2,7E0,3.6E1,4.2E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.0944726E-4,-2.2059606E-2,2.9954989E-2,-1.9678326E-2,-8.810453E-3,6.327473E-3,6.3674256E-2,-2.839149E-3,-1.1713857E-2,-4.876149E-3,2.2929693E-3,5.1218074E-2,1.7871976E-1,1.773121E-2,-1.118982E-3,-9.5490756E-4,2.4309005E-3,6.364446E-2,-8.221603E-4,3.2645955E-3,1.3447716E-2,2.038004E-3,-4.9188305E-2,8.80483E-3,5.5406064E-2,-8.79365E-4,-7.1738297E-3,-1.9755222E-4,3.5370965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,-1,13,15,-1,17,19,21,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[3.22285E-1,1.0042156E-1,1.6610795E-1,8.272244E-2,0E0,5.6396242E-2,1.1408666E-1,0E0,7.4953504E-2,7.8848116E-2,0E0,6.7805395E-2,6.7430586E-2,1.02499984E-1,0E0,0E0,0E0,5.402735E-2,0E0,0E0,0E0,0E0,4.8971638E-2,0E0,6.38988E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,11,11,12,12,13,13,17,17,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,-1,14,16,-1,18,20,22,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[8.393928E4,7.339209E6,1.3590209E7,3.606E3,-8.810453E-3,1.04280655E2,4.2875492E1,-2.839149E-3,1.031356E1,1.7670108E5,2.2929693E-3,1.73E2,8.926081E-2,3.0840238E7,-1.118982E-3,-9.5490756E-4,2.4309005E-3,5.473125E3,-8.221603E-4,3.2645955E-3,1.3447716E-2,2.038004E-3,5.309569E6,8.80483E-3,5.3474556E5,-8.79365E-4,-7.1738297E-3,-1.9755222E-4,3.5370965E-3],"split_indices":[41,55,1,2,0,62,62,0,66,36,0,8,65,53,0,0,0,40,0,0,0,0,9,0,40,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.84E2,2.76E2,2.08E2,2.73E2,3E0,1.23E2,8.5E1,4.6E1,2.27E2,9.7E1,2.6E1,7.8E1,7E0,6.2E1,1.65E2,7.7E1,2E1,6.6E1,1.2E1,4E0,3E0,4.6E1,1.6E1,3E0,6.3E1,1.3E1,3E0,1.4E1,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[7.059385E-4,-7.0748627E-3,9.967541E-2,-1.6592193E-2,3.2024805E-2,1.8234636E-1,2.488343E-3,-1.501325E-2,-1.0014686E-2,2.0813928E-3,-3.2591738E-3,1.4390714E-2,4.9214894E-3,-2.0410122E-2,3.5174955E-2,-2.3892373E-2,2.6860714E-3,3.20694E-3,-5.6364853E-4,-3.721587E-2,-5.788753E-3,-3.8271795E-3,-1.3256861E-3,4.985129E-4,-2.1740666E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,-1,-1,15,17,19,-1,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[3.8034564E-1,1.7096436E-1,1.3638023E-1,1.06251106E-1,9.517066E-2,8.897278E-2,0E0,1.0012332E-1,0E0,0E0,0E0,0E0,0E0,8.822985E-2,5.1524986E-2,7.648727E-2,0E0,0E0,0E0,7.396895E-2,8.408495E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,13,13,14,14,15,15,19,19,20,20],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,-1,-1,16,18,20,-1,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[6.204522E3,1.5522031E9,1.681178E6,3.764913E7,9.677921E7,4.542969E1,2.488343E-3,1.7267E4,-1.0014686E-2,2.0813928E-3,-3.2591738E-3,1.4390714E-2,4.9214894E-3,4.79E2,1.2394286E9,2.025313E6,2.6860714E-3,3.20694E-3,-5.6364853E-4,4.94E4,4.0266477E8,-3.8271795E-3,-1.3256861E-3,4.985129E-4,-2.1740666E-3],"split_indices":[60,7,37,56,1,62,0,2,0,0,0,0,0,8,7,1,0,0,0,9,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,4.57E2,3.5E1,3.68E2,8.9E1,1.2E1,2.3E1,3.66E2,2E0,8.1E1,8E0,4E0,8E0,3.31E2,3.5E1,3.17E2,1.4E1,2.1E1,1.4E1,1.82E2,1.35E2,3.5E1,1.47E2,9.6E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-5.263831E-4,-2.2929257E-2,1.876707E-2,-2.975486E-2,1.1870485E-3,2.43629E-2,-7.127125E-2,-5.092893E-2,-7.58848E-3,6.896942E-3,1.9706918E-2,-1.2257652E-2,-1.204081E-3,-6.272478E-2,2.121637E-4,2.471451E-2,-1.4687018E-3,9.150761E-3,5.3942714E-2,-2.3084928E-3,-5.2629267E-3,-4.583377E-3,1.8222358E-3,1.2164137E-2,-6.09156E-3,3.912053E-3,1.0589323E-4,-3.919205E-3,1.6502526E-2,6.7361334E-4,8.00044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,21,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1,-1,29,-1,-1],"loss_changes":[2.0271339E-1,7.053527E-2,1.2870383E-1,8.8790506E-2,0E0,1.2608014E-1,1.19079754E-1,6.3494384E-2,6.8639606E-2,0E0,8.262803E-2,0E0,0E0,5.0783157E-2,0E0,6.082894E-2,0E0,7.264601E-2,7.206088E-2,0E0,0E0,0E0,0E0,7.0831515E-2,0E0,0E0,0E0,0E0,6.5501936E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,15,15,17,17,18,18,23,23,28,28],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,22,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1,-1,30,-1,-1],"split_conditions":[6.979E3,2.1592189E5,4.7374536E7,1.656968E6,1.1870485E-3,5.0957993E2,2.9580229E1,2.56445E7,1.33E2,6.896942E-3,1.646359E5,-1.2257652E-2,-1.204081E-3,1.5881974E7,2.121637E-4,3.4E1,-1.4687018E-3,3.7826266E4,2.3961474E7,-2.3084928E-3,-5.2629267E-3,-4.583377E-3,1.8222358E-3,2.4950776E0,-6.09156E-3,3.912053E-3,1.0589323E-4,-3.919205E-3,2.5212732E4,6.7361334E-4,8.00044E-3],"split_indices":[2,36,53,9,0,55,64,53,3,0,36,0,0,53,0,3,0,60,9,0,0,0,0,62,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.67E2,2.16E2,2.51E2,1.89E2,2.7E1,2.37E2,1.4E1,9.6E1,9.3E1,8E0,2.29E2,2E0,1.2E1,7.9E1,1.7E1,3.8E1,5.5E1,1.76E2,5.3E1,6E1,1.9E1,3E0,3.5E1,1.73E2,3E0,3.5E1,1.8E1,7E0,1.66E2,1.64E2,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.6341973E-4,-8.809986E-3,7.391689E-2,-1.6815946E-2,2.5925914E-2,1.1310422E-2,5.9900578E-2,-1.4446749E-2,-6.350001E-3,5.297993E-3,1.2652465E-2,1.5345948E-2,1.004377E-1,-2.2536954E-3,-7.530366E-3,-2.903554E-3,1.188E-3,2.148186E-3,-5.2596354E-3,1.8199097E-3,1.7122562E-1,1.7810713E-2,-1.6224276E-2,2.7090383E-3,1.2072354E-2,2.1999322E-3,-9.5196784E-4,-4.7461493E-3,-1.2536441E-2,-1.7035796E-3,4.68451E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,15,17,19,-1,21,-1,-1,-1,-1,-1,23,25,27,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[3.2250017E-1,1.1862601E-1,1.02609664E-1,9.049207E-2,8.5824236E-2,0E0,8.876449E-2,7.375309E-2,0E0,0E0,5.965104E-2,9.192763E-2,1.1019656E-1,0E0,6.197663E-2,0E0,0E0,0E0,0E0,0E0,9.072009E-2,7.33673E-2,6.087813E-2,0E0,0E0,0E0,0E0,0E0,6.0243648E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,10,10,11,11,12,12,14,14,20,20,21,21,22,22,28,28],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,16,18,20,-1,22,-1,-1,-1,-1,-1,24,26,28,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[4.23168E3,1.646359E5,8.169258E3,8.6206274E2,2.8E1,1.1310422E-2,1.7552137E0,1.3202006E3,-6.350001E-3,5.297993E-3,3.8850647E2,1.4609149E4,2.2876814E1,-2.2536954E-3,7.477264E6,-2.903554E-3,1.188E-3,2.148186E-3,-5.2596354E-3,1.8199097E-3,3.2006908E7,3.311258E-2,1.463286E3,2.7090383E-3,1.2072354E-2,2.1999322E-3,-9.5196784E-4,-4.7461493E-3,1.1662405E9,-1.7035796E-3,4.68451E-5],"split_indices":[60,36,41,66,3,0,50,4,0,0,60,4,62,0,53,0,0,0,0,0,1,65,4,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,4.24E2,5.2E1,3.45E2,7.9E1,3E0,4.9E1,3.39E2,6E0,1E1,6.9E1,2.4E1,2.5E1,6E1,2.79E2,9E0,6E1,2E1,4E0,1.4E1,1.1E1,7.1E1,2.08E2,5E0,6E0,4.1E1,3E1,8E0,2E2,7.5E1,1.25E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.3527481E-4,-8.826922E-3,6.726782E-2,-2.165392E-2,1.6798086E-2,1.4928684E-1,1.6074241E-3,-1.9602206E-2,-7.638158E-3,2.5971225E-2,-2.2459356E-3,3.6730287E-3,1.1486234E-2,-2.9398903E-2,1.6947777E-4,3.5737876E-2,-1.979018E-3,-4.4016015E-2,-2.916244E-3,8.727387E-3,3.1291436E-2,-1.6287959E-3,-1.01392925E-1,-5.022855E-3,3.1671047E-4,-2.111803E-4,2.1804234E-3,-6.1161513E-3,8.466411E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,-1,-1,17,-1,19,-1,21,23,-1,25,-1,27,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8096503E-1,1.3578482E-1,1.5068945E-1,7.26106E-2,8.0430694E-2,8.35506E-2,0E0,6.1544806E-2,0E0,7.982547E-2,0E0,0E0,0E0,7.380773E-2,0E0,6.1467588E-2,0E0,7.286371E-2,6.5028936E-2,0E0,4.8955195E-2,0E0,5.4744616E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,13,13,15,15,17,17,18,18,20,20,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,-1,-1,18,-1,20,-1,22,24,-1,26,-1,28,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,9.7467445E4,3.0418E4,5.825042E6,2.4722598E7,3.2006908E7,1.6074241E-3,6.979E3,-7.638158E-3,7.201163E1,-2.2459356E-3,3.6730287E-3,1.1486234E-2,1.16985195E5,1.6947777E-4,8.534E3,-1.979018E-3,5.861E3,5.349E3,8.727387E-3,1.0643513E3,-1.6287959E-3,7.164074E8,-5.022855E-3,3.1671047E-4,-2.111803E-4,2.1804234E-3,-6.1161513E-3,8.466411E-4],"split_indices":[60,41,10,55,53,1,0,2,0,64,0,0,0,36,0,9,0,10,9,0,60,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.65E2,4.11E2,5.4E1,2.74E2,1.37E2,1.5E1,3.9E1,2.71E2,3E0,1.2E2,1.7E1,9E0,6E0,1.9E2,8.1E1,1.05E2,1.5E1,1.22E2,6.8E1,2E0,1.03E2,1.04E2,1.8E1,5E0,6.3E1,2.8E1,7.5E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.9209189E-3,-8.936479E-3,5.216045E-2,-7.5569E-3,-9.628288E-3,1.0250129E-2,4.210777E-2,-3.357158E-2,1.6194588E-3,-9.806047E-4,6.3321896E-2,-1.6253604E-2,-6.260579E-2,1.1270105E-2,-1.3171048E-3,1.6885335E-3,1.254537E-1,-2.6674527E-3,3.959094E-4,-1.9221171E-3,-7.42188E-3,1.3272086E-4,2.1012216E-3,1.7834685E-3,1.0223522E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,15,17,19,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7840993E-1,1.06067866E-1,7.6349095E-2,9.8884456E-2,0E0,0E0,6.977554E-2,5.3286314E-2,8.467523E-2,0E0,6.568338E-2,6.446108E-2,7.716502E-2,6.2081255E-2,0E0,0E0,8.117248E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,10,10,11,11,12,12,13,13,16,16],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,16,18,20,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,1.1770536E8,8.169258E3,4.39E3,-9.628288E-3,1.0250129E-2,1.4015416E6,7.9879355E6,2.1678448E7,-9.806047E-4,2.0141928E0,2.53383E5,1.099906E6,4.95E9,-1.3171048E-3,1.6885335E-3,2.47815E5,-2.6674527E-3,3.959094E-4,-1.9221171E-3,-7.42188E-3,1.3272086E-4,2.1012216E-3,1.7834685E-3,1.0223522E-2],"split_indices":[60,1,41,2,0,0,40,53,53,0,47,9,9,5,0,0,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,4.15E2,5.3E1,4.13E2,2E0,2E0,5.1E1,1.07E2,3.06E2,1.3E1,3.8E1,6.8E1,3.9E1,2.29E2,7.7E1,2.7E1,1.1E1,2.6E1,4.2E1,3.2E1,7E0,1.81E2,4.8E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-9.7509805E-4,-9.0707615E-3,5.5036157E-2,8.009481E-3,-2.2978773E-2,1.0311888E-2,4.317634E-2,1.156075E-2,-7.7163395E-3,-1.2810403E-3,2.7844552E-3,5.1978705E-3,8.518213E-2,-2.2317076E-3,1.8408215E-2,1.472015E-3,-5.0617587E-3,2.452327E-3,9.805726E-3,1.904119E-3,-2.3863776E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,15,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[2.154337E-1,9.876822E-2,1.0139038E-1,1.1163302E-1,5.891055E-2,0E0,9.049022E-2,7.274167E-2,0E0,0E0,0E0,8.651452E-2,9.9004075E-2,0E0,7.860963E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,11,11,12,12,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,16,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,1.24E2,8.169258E3,7.78623E6,6.46E2,1.0311888E-2,1.7552137E0,5.42218E5,-7.7163395E-3,-1.2810403E-3,2.7844552E-3,3.63796E5,3.305542E4,-2.2317076E-3,9.296044E6,1.472015E-3,-5.0617587E-3,2.452327E-3,9.805726E-3,1.904119E-3,-2.3863776E-4],"split_indices":[60,3,41,55,8,0,50,1,0,0,0,2,4,0,53,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.73E2,4.14E2,5.9E1,1.86E2,2.28E2,3E0,5.6E1,1.83E2,3E0,2.2E2,8E0,3E1,2.6E1,1.9E1,1.64E2,2.5E1,5E0,2.1E1,5E0,8.7E1,7.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.238848E-3,-1.021846E-2,3.6686562E-2,-1.1871729E-2,5.3554117E-3,2.2721767E-2,8.927154E-2,-1.549472E-2,2.0154545E-3,2.9772988E-2,-5.263563E-3,1.6914135E-3,8.66713E-3,-1.8742552E-3,-5.992411E-3,3.8445666E-2,-2.529881E-3,-1.7879377E-4,-6.9230963E-3,1.1293393E-3,8.43137E-2,1.1170659E-2,2.7322853E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,-1,17,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.9288306E-1,7.217847E-2,8.315483E-2,6.876582E-2,0E0,8.84817E-2,1.0890083E-1,7.1679E-2,0E0,6.478428E-2,0E0,0E0,0E0,0E0,7.434478E-2,5.5334404E-2,0E0,0E0,0E0,0E0,6.936914E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,14,14,15,15,20,20],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,-1,18,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.6053742E9,8.10288E3,2.029183E0,2.5653894E5,5.3554117E-3,3.768034E4,2.2876814E1,1.5031561E3,2.0154545E-3,1.5953E4,-5.263563E-3,1.6914135E-3,8.66713E-3,-1.8742552E-3,3.764913E7,9.462212E9,-2.529881E-3,-1.7879377E-4,-6.9230963E-3,1.1293393E-3,2.9E1,1.1170659E-2,2.7322853E-3],"split_indices":[7,60,50,36,0,4,62,4,0,0,0,0,0,0,56,12,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.73E2,3.58E2,1.15E2,3.54E2,4E0,9.2E1,2.3E1,3.32E2,2.2E1,8.8E1,4E0,1.5E1,8E0,9.7E1,2.35E2,8E1,8E0,2.32E2,3E0,6.1E1,1.9E1,2E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-9.455477E-4,-1.5472108E-2,2.7585065E-2,-2.0897157E-2,1.0519235E-3,8.022141E-3,1.9613069E-2,-1.6052965E-2,-6.623272E-2,-6.211502E-3,2.4265196E-2,-1.3191324E-2,-5.7066623E-3,-4.544871E-3,2.0486016E-3,1.3869654E-2,6.642459E-2,-2.2889932E-3,-7.18362E-3,1.4279506E-3,-7.1193144E-4,1.5262994E-3,1.596101E-1,7.6510175E-3,-3.1636186E-2,3.8080828E-3,1.3256133E-2,-3.8313825E-4,2.4502003E-3,-3.2460548E-3,-4.3580166E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,15,17,-1,-1,-1,19,21,-1,23,-1,-1,-1,25,27,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0267919E-1,6.53482E-2,1.7568584E-1,6.108758E-2,0E0,0E0,1.09912455E-1,7.224228E-2,8.099388E-2,0E0,6.613398E-2,5.0158933E-2,0E0,0E0,0E0,5.4099806E-2,9.371674E-2,0E0,7.786946E-2,0E0,0E0,0E0,5.1594213E-2,8.807081E-2,6.30902E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,10,10,11,11,15,15,16,16,18,18,22,22,23,23,24,24],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,16,18,-1,-1,-1,20,22,-1,24,-1,-1,-1,26,28,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5135906E3,1.5522031E9,9.957E3,1E0,1.0519235E-3,8.022141E-3,2.2782404E0,4.086885E7,1.4998456E5,-6.211502E-3,2.1678066E0,9.252759E2,-5.7066623E-3,-4.544871E-3,2.0486016E-3,7.309061E1,9.920844E0,-2.2889932E-3,8.227176E-3,1.4279506E-3,-7.1193144E-4,1.5262994E-3,1.323E3,2.025313E6,6.505339E2,3.8080828E-3,1.3256133E-2,-3.8313825E-4,2.4502003E-3,-3.2460548E-3,-4.3580166E-4],"split_indices":[60,7,9,79,0,0,62,53,36,0,50,63,0,0,0,64,61,0,65,0,0,0,0,1,60,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.87E2,3.23E2,1.64E2,2.82E2,4.1E1,8E0,1.56E2,2.56E2,2.6E1,4E0,1.52E2,2.5E2,6E0,2.1E1,5E0,1.23E2,2.9E1,3.7E1,2.13E2,8E1,4.3E1,2.2E1,7E0,1.33E2,8E1,5E0,2E0,9.8E1,3.5E1,3.1E1,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.716793E-4,-1.1486375E-2,3.89027E-2,-1.6805504E-2,1.2398638E-3,9.2565954E-2,2.4384908E-2,-2.1540977E-2,1.9934852E-2,1.8018703E-1,2.3757587E-3,-5.0734044E-3,3.2070193E-2,-4.0823743E-2,-1.0889232E-2,-5.9815557E-3,3.7050694E-2,2.6831185E-3,1.27802035E-2,1.9964816E-3,-1.9923965E-3,-1.5113535E-3,-4.9802503E-3,2.4401058E-2,-2.0272214E-2,3.0923414E-3,-1.2982175E-3,2.3463173E-3,-4.065936E-3,-2.2486069E-4,-4.8671108E-2,-3.1746982E-3,1.0035711E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1,31,-1,-1],"loss_changes":[2.0439416E-1,6.790081E-2,7.954532E-2,5.2972272E-2,0E0,7.631834E-2,8.584775E-2,5.460134E-2,8.953138E-2,5.638008E-2,0E0,0E0,5.191508E-2,5.4591477E-2,5.811096E-2,0E0,5.4537874E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.705741E-2,6.1254393E-2,0E0,0E0,0E0,0E0,0E0,5.575502E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,16,16,23,23,24,24,30,30],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1,32,-1,-1],"split_conditions":[2.1959E4,3.94E2,1.417988E6,2.1592189E5,1.2398638E-3,1.01267644E11,1.9813E4,2.83593E5,2.1066585E2,1E0,2.3757587E-3,-5.0734044E-3,1.773296E6,1.4690361E5,8.593763E7,-5.9815557E-3,8.586521E-2,2.6831185E-3,1.27802035E-2,1.9964816E-3,-1.9923965E-3,-1.5113535E-3,-4.9802503E-3,4.4329906E1,8.94E2,3.0923414E-3,-1.2982175E-3,2.3463173E-3,-4.065936E-3,-2.2486069E-4,3.11E3,-3.1746982E-3,1.0035711E-3],"split_indices":[2,8,37,36,0,39,10,9,60,21,0,0,11,36,7,0,46,0,0,0,0,0,0,66,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.48E2,3.44E2,1.04E2,3.01E2,4.3E1,2.1E1,8.3E1,2.67E2,3.4E1,6E0,1.5E1,4E0,7.9E1,9.4E1,1.73E2,3E0,3.1E1,3E0,3E0,7.1E1,8E0,8.2E1,1.2E1,3.6E1,1.37E2,2.2E1,9E0,3E1,6E0,8.9E1,4.8E1,3.9E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.1542191E-3,-8.63144E-3,3.672599E-2,-6.8346993E-3,-8.952264E-3,2.675007E-2,5.917607E-3,-8.365402E-3,5.5568377E-3,7.6841556E-2,4.0643604E-4,-1.6888262E-3,-1.9038919E-3,6.229793E-4,6.536657E-3,-7.236475E-3,2.9089665E-2,-4.6526235E-2,-1.8783872E-3,4.6094344E-3,5.2772666E-4,-8.003662E-4,-7.4573727E-3,1.0429038E-3,-7.6184206E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3400565E-1,1.2218741E-1,6.2091917E-2,7.304187E-2,0E0,6.5023325E-2,0E0,6.5301314E-2,0E0,6.7102075E-2,0E0,0E0,5.1776532E-2,0E0,0E0,5.6120906E-2,5.3710565E-2,9.9533506E-2,7.492901E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,12,12,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5257432E6,1.92144E5,1.0871896E10,6.942396E3,-8.952264E-3,2.1207E4,5.917607E-3,1.2595157E3,5.5568377E-3,5.5835126E9,4.0643604E-4,-1.6888262E-3,1.5271514E5,6.229793E-4,6.536657E-3,2.9648642E1,2.823537E1,6.4E1,1.33E2,4.6094344E-3,5.2772666E-4,-8.003662E-4,-7.4573727E-3,1.0429038E-3,-7.6184206E-4],"split_indices":[52,2,7,60,0,9,0,4,0,12,0,0,36,0,0,64,64,8,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.71E2,3.94E2,7.7E1,3.91E2,3E0,7E1,7E0,3.87E2,4E0,1.8E1,5.2E1,7.6E1,3.11E2,9E0,9E0,2.66E2,4.5E1,3.1E1,2.35E2,9E0,3.6E1,2.5E1,6E0,8.7E1,1.48E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[3.4977135E-3,-3.7092662E-3,5.206071E-2,-2.452504E-3,-8.811139E-3,9.232446E-3,4.211257E-2,-6.412796E-4,8.521602E-4,-6.0694217E-4,2.8915983E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,-1,-1,9,-1,-1,-1,-1],"loss_changes":[1.7063946E-1,9.354754E-2,7.7436864E-2,8.934472E-2,0E0,0E0,5.5827513E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6],"right_children":[2,4,6,8,-1,-1,10,-1,-1,-1,-1],"split_conditions":[4.23168E3,1.1770536E8,2.909019E3,7.10711E8,-8.811139E-3,9.232446E-3,6.219584E5,-6.412796E-4,8.521602E-4,-6.0694217E-4,2.8915983E-3],"split_indices":[60,1,40,7,0,0,40,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,4.24E2,6.2E1,4.22E2,2E0,3E0,5.9E1,2.75E2,1.47E2,1.4E1,4.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[2.1459642E-3,-6.641324E-3,3.377643E-2,-1.1643688E-2,1.224717E-3,2.5212996E-2,7.2389315E-3,-1.6885804E-2,1.2047876E-3,-7.156605E-3,3.0352337E-2,-4.3851815E-2,-1.0129244E-2,3.6513515E-2,-5.832831E-3,-1.4027426E-3,-5.449614E-3,9.058885E-3,-1.3771508E-3,8.9964725E-2,1.0933789E-3,-1.167908E-3,2.3457387E-2,1.0797704E-2,2.5395579E-3,2.0613144E-3,-7.158948E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,15,17,19,-1,-1,-1,21,-1,23,-1,-1,25,-1,-1,-1,-1],"loss_changes":[1.3169278E-1,5.8965646E-2,9.668846E-2,6.1348896E-2,0E0,8.975063E-2,0E0,5.075331E-2,0E0,0E0,9.172722E-2,5.426459E-2,7.811027E-2,6.790829E-2,0E0,0E0,0E0,5.262694E-2,0E0,8.305812E-2,0E0,0E0,5.5049386E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,11,11,12,12,13,13,17,17,19,19,22,22],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,16,18,20,-1,-1,-1,22,-1,24,-1,-1,26,-1,-1,-1,-1],"split_conditions":[1.3590209E7,3.53E2,4.2875492E1,2.1592189E5,1.224717E-3,8.713209E10,7.2389315E-3,4.9256845E2,1.2047876E-3,-7.156605E-3,4.5859156E7,1.1991839E4,1.4040858E-2,2.1207E4,-5.832831E-3,-1.4027426E-3,-5.449614E-3,1.3063165E3,-1.3771508E-3,2.9E1,1.0933789E-3,-1.167908E-3,3.7755436E1,1.0797704E-2,2.5395579E-3,2.0613144E-3,-7.158948E-4],"split_indices":[1,8,62,36,0,39,0,41,0,0,53,58,65,9,0,0,0,4,0,3,0,0,66,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.72E2,3.7E2,1.02E2,3.2E2,5E1,9.6E1,6E0,2.8E2,4E1,2E0,9.4E1,5.5E1,2.25E2,9.1E1,3E0,4.6E1,9E0,1.09E2,1.16E2,1.8E1,7.3E1,3.3E1,7.6E1,3E0,1.5E1,5.1E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.7774926E-3,-5.175342E-3,2.0344527E-3,-3.421477E-3,-8.296571E-3,-1.2096599E-2,1.3350049E-2,-7.348735E-3,-3.3927094E-3,7.1233446E-3,1.0128973E-2,-1.497591E-3,4.4446043E-3,2.6227256E-2,-7.861981E-4,2.1726392E-2,-1.2841584E-3,3.2089269E-3,4.8773855E-4,1.3618025E-3,-6.3136313E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,-1,7,9,11,-1,-1,13,-1,15,17,-1,19,-1,-1,-1,-1,-1],"loss_changes":[1.4090034E-1,1.08727284E-1,0E0,5.5013113E-2,0E0,6.698131E-2,5.333928E-2,6.331154E-2,0E0,0E0,5.390635E-2,0E0,8.2090415E-2,4.9780753E-2,0E0,9.360346E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,10,10,12,12,13,13,15,15],"right_children":[2,4,-1,6,-1,8,10,12,-1,-1,14,-1,16,18,-1,20,-1,-1,-1,-1,-1],"split_conditions":[4.2729566E4,1.69595E5,2.0344527E-3,1.2347272E3,-8.296571E-3,1E0,5.738E3,2.0011E5,-3.3927094E-3,7.1233446E-3,7.201163E1,-1.497591E-3,1.8856065E-2,1.5963512E-2,-7.861981E-4,3.9034148E7,-1.2841584E-3,3.2089269E-3,4.8773855E-4,1.3618025E-3,-6.3136313E-3],"split_indices":[51,2,0,60,0,79,9,9,0,0,64,0,65,65,0,53,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.56E2,3.79E2,7.7E1,3.76E2,3E0,2.48E2,1.28E2,2.3E2,1.8E1,2E0,1.26E2,7.7E1,1.53E2,7.8E1,4.8E1,9.8E1,5.5E1,2.2E1,5.6E1,9.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.294253E-3,-6.0062725E-3,2.9053257E-3,-1.4417405E-2,8.182557E-3,-3.3363586E-3,-3.3244416E-2,6.8904674E-3,2.8084664E-4,1.6380523E-3,-5.546975E-4,-2.7475676E-3,-7.5172884E-3,2.475096E-3,-2.7157918E-2,-4.6741227E-3,-2.8470464E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,-1,13,-1,15,-1,-1],"loss_changes":[1.0840602E-1,5.34648E-2,0E0,5.8512144E-2,5.345967E-2,5.253135E-2,6.0539775E-2,0E0,0E0,0E0,0E0,0E0,5.8318056E-2,0E0,5.4747846E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,12,12,14,14],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,-1,14,-1,16,-1,-1],"split_conditions":[4.026531E9,1.347372E3,2.9053257E-3,1.3026638E7,6.738547E7,2.1037032E-4,2.92E2,6.8904674E-3,2.8084664E-4,1.6380523E-3,-5.546975E-4,-2.7475676E-3,2.7434757E1,2.475096E-3,3.7075245E1,-4.6741227E-3,-2.8470464E-4],"split_indices":[7,60,0,53,7,46,3,0,0,0,0,0,62,0,64,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.72E2,4.46E2,2.6E1,2.8E2,1.66E2,1.77E2,1.03E2,2E0,1.64E2,3.1E1,1.46E2,5.4E1,4.9E1,1.2E1,3.7E1,8E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[2.7848224E-3,-8.2314145E-3,2.3301566E-2,-1.2974604E-2,1.9122078E-3,8.656673E-2,1.2742213E-2,-9.907986E-4,1.3051745E-2,8.772235E-3,2.5921047E-3,-3.8194756E-3,9.4470824E-4,-2.0605398E-3,1.7660671E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,-1,-1,-1,-1,-1],"loss_changes":[1.09358154E-1,7.106512E-2,1.11795574E-1,5.4266486E-2,0E0,6.522916E-2,8.78013E-2,0E0,8.117742E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5135906E3,4.2524424E7,2.1207E4,5.5292703E9,1.9122078E-3,3.7137297E3,5.767668E0,-9.907986E-4,1.7895421E0,8.772235E-3,2.5921047E-3,-3.8194756E-3,9.4470824E-4,-2.0605398E-3,1.7660671E-3],"split_indices":[60,9,9,5,0,40,62,0,61,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,3.14E2,1.68E2,2.86E2,2.8E1,2.3E1,1.45E2,2.24E2,6.2E1,5E0,1.8E1,9E0,1.36E2,1.8E1,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.1356957E-3,-7.667369E-3,3.461073E-2,-6.4514764E-3,-6.021012E-3,8.951973E-3,2.9826498E-2,8.803559E-3,-1.4818899E-2,2.1656698E-2,4.6076425E-3,3.563573E-3,-4.1658688E-5,-3.3146452E-2,-4.911974E-5,1.2911217E-3,-6.0122507E-3,-2.9679528E-3,-1.230068E-2,-1.8290022E-3,2.618564E-2,3.8046418E-3,-9.680567E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,-1,-1,17,-1,-1,-1,-1,19,-1,21,-1,-1],"loss_changes":[1.443653E-1,5.3514235E-2,6.674749E-2,4.927391E-2,0E0,0E0,5.0359994E-2,8.452077E-2,6.302266E-2,6.388295E-2,0E0,0E0,0E0,6.0317412E-2,0E0,0E0,0E0,0E0,6.0621478E-2,0E0,6.054493E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,13,13,18,18,20,20],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,-1,-1,18,-1,-1,-1,-1,20,-1,22,-1,-1],"split_conditions":[1.6219076E9,4.242889E3,1.6441814E9,1.26E2,-6.021012E-3,8.951973E-3,1.6941456E1,1.3436554E3,2.72E2,3.7826266E4,4.6076425E-3,3.563573E-3,-4.1658688E-5,7.885718E2,-4.911974E-5,1.2911217E-3,-6.0122507E-3,-2.9679528E-3,1.6670956E9,-1.8290022E-3,8.557E3,3.8046418E-3,-9.680567E-4],"split_indices":[7,66,7,3,0,0,43,55,3,60,0,0,0,60,0,0,0,0,5,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,3.87E2,1.01E2,3.84E2,3E0,2E0,9.9E1,1.36E2,2.48E2,8.9E1,1E1,1.7E1,1.19E2,1.06E2,1.42E2,8.7E1,2E0,4.5E1,6.1E1,3.7E1,2.4E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.7629686E-3,-5.676313E-3,2.8261766E-2,-4.6083124E-4,1.67305E-3,-1.842994E-3,3.9289333E-2,6.448256E-3,2.2783587E-2,-7.977237E-4,1.9796465E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,-1,-1,-1,7,-1,9,-1,-1],"loss_changes":[9.0841144E-2,5.362489E-2,7.4812606E-2,0E0,0E0,0E0,1.2940793E-1,0E0,5.2160494E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,6,6,8,8],"right_children":[2,4,6,-1,-1,-1,8,-1,10,-1,-1],"split_conditions":[1.4542315E7,4.7619216E7,2.4950776E0,-4.6083124E-4,1.67305E-3,-1.842994E-3,1.6459E4,6.448256E-3,6.219584E5,-7.977237E-4,1.9796465E-3],"split_indices":[1,9,62,0,0,0,9,0,40,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.59E2,3.59E2,1E2,3.29E2,3E1,1.4E1,8.6E1,1.2E1,7.4E1,2.3E1,5.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.4191044E-4,-3.8864152E-4,3.0795354E-2,2.5477504E-2,9.525655E-3,-6.693414E-3,3.0273654E-2,-1.6516544E-3,3.894631E-2,5.0716856E-3,1.1032126E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,-1,3,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[1.1869283E-1,0E0,8.197206E-2,8.0261335E-2,0E0,0E0,5.4558314E-2,0E0,8.782397E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,8,8],"right_children":[2,-1,4,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[1.4689211E7,-3.8864152E-4,4.2875492E1,8.713209E10,9.525655E-3,-6.693414E-3,2.4950776E0,-1.6516544E-3,2.1207E4,5.0716856E-3,1.1032126E-3],"split_indices":[1,0,62,39,0,0,62,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,3.79E2,9.9E1,9.7E1,2E0,2E0,9.5E1,1.1E1,8.4E1,1.6E1,6.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[5.1979204E-3,1.092874E-3,7.9537645E-2,2.5159237E-3,-5.177573E-3,8.075512E-3,1.4606743E-3,-1.859762E-4,1.3095737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[1.4578336E-1,6.918372E-2,1.0098861E-1,6.861483E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[8.10288E3,1.2622761E8,3.0418E4,1.6053742E9,-5.177573E-3,8.075512E-3,1.4606743E-3,-1.859762E-4,1.3095737E-3],"split_indices":[60,1,10,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.77E2,4.53E2,2.4E1,4.48E2,5E0,8E0,1.6E1,3.56E2,9.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-7.624187E-4,-2.5829643E-3,4.198111E-3,1.0276825E-2,-6.7024265E-4,3.3254633E-3,3.2152128E-3,6.444116E-3,-4.9355533E-3,-2.1345353E-2,1.833556E-2,-3.0486658E-4,-3.8652488E-3,3.1438754E-3,3.8548894E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,-1,5,-1,-1,7,9,-1,11,13,-1,-1,-1,-1],"loss_changes":[7.616839E-2,6.7707025E-2,0E0,8.916707E-2,0E0,0E0,6.671378E-2,6.3839115E-2,0E0,4.9941353E-2,6.4224556E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,9,9,10,10],"right_children":[2,4,-1,6,-1,-1,8,10,-1,12,14,-1,-1,-1,-1],"split_conditions":[7.986019E9,1.33E2,4.198111E-3,9.1417163E2,-6.7024265E-4,3.3254633E-3,1.0067637E7,3.059068E-1,-4.9355533E-3,1.33E2,2.1207E4,-3.0486658E-4,-3.8652488E-3,3.1438754E-3,3.8548894E-4],"split_indices":[7,3,0,55,0,0,55,49,0,8,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,4.73E2,9E0,2.19E2,2.54E2,2.3E1,1.96E2,1.91E2,5E0,5.7E1,1.34E2,4.6E1,1.1E1,2.4E1,1.1E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[2.237517E-3,-4.710012E-4,1.905406E-2,-2.9238043E-3,1.0979495E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":82,"left_children":[1,-1,3,-1,-1],"loss_changes":[9.159013E-2,0E0,5.119212E-2,0E0,0E0],"parents":[2147483647,0,0,2,2],"right_children":[2,-1,4,-1,-1],"split_conditions":[1.00493375E5,-4.710012E-4,1.7895421E0,-2.9238043E-3,1.0979495E-3],"split_indices":[41,0,61,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[4.58E2,2.69E2,1.89E2,7E0,1.82E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[3.8087282E-3,-6.0081406E-4,3.607855E-2,7.962185E-4,-4.740257E-3,-2.6701149E-3,2.247465E-3,-2.0424975E-3,4.110296E-3,-1.3709611E-4,2.957851E-2,2.371314E-3,-1.2724548E-2,-2.6877911E-3,1.6465471E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,-1,-1,-1,-1,9,-1,11,-1,13,-1,-1],"loss_changes":[6.7469575E-2,5.6173455E-2,5.2814536E-2,5.826863E-2,0E0,0E0,0E0,0E0,6.762149E-2,0E0,6.575066E-2,0E0,5.250286E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,10,10,12,12],"right_children":[2,4,6,8,-1,-1,-1,-1,10,-1,12,-1,14,-1,-1],"split_conditions":[4.23168E3,9.163994E6,5.0314346E1,9.687575E2,-4.740257E-3,-2.6701149E-3,2.247465E-3,-2.0424975E-3,5.5292703E9,-1.3709611E-4,1.1138894E6,2.371314E-3,1.4414646E-1,-2.6877911E-3,1.6465471E-3],"split_indices":[60,55,64,4,0,0,0,0,5,0,55,0,46,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.73E2,4.17E2,5.6E1,4.12E2,5E0,5E0,5.1E1,2.9E1,3.83E2,3.02E2,8.1E1,5.6E1,2.5E1,1.3E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.1840937E-3,-9.372075E-3,1.8840503E-2,-1.713462E-2,2.0506365E-2,7.5069964E-2,7.554507E-4,-1.1419435E-2,-3.7363886E-3,-4.6286486E-2,2.4071955E-3,5.123198E-3,-1.5645222E-3,-9.3981146E-4,1.7102092E-3,2.466074E-3,-4.4108145E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.468149E-2,5.0448842E-2,5.0033607E-2,5.7184286E-2,8.8177525E-2,5.0511688E-2,0E0,5.731985E-2,0E0,6.355154E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.783E3,1.903E3,8.547769E7,1.121092E6,2.4094029E2,6.6E1,7.554507E-4,7.164074E8,-3.7363886E-3,1.30817086E5,2.4071955E-3,5.123198E-3,-1.5645222E-3,-9.3981146E-4,1.7102092E-3,2.466074E-3,-4.4108145E-3],"split_indices":[2,0,7,37,60,8,0,7,0,36,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.79E2,2.15E2,2.64E2,1.71E2,4.4E1,1.4E1,2.5E2,1.57E2,1.4E1,1.3E1,3.1E1,1.1E1,3E0,1.35E2,2.2E1,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[5.969839E-4,-4.2450763E-3,1.7351966E-3,-1.4965239E-4,-5.0823633E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":85,"left_children":[1,3,-1,-1,-1],"loss_changes":[8.1338786E-2,4.997619E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[4.0292856E3,1.3674345E4,1.7351966E-3,-1.4965239E-4,-5.0823633E-3],"split_indices":[60,4,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[4.81E2,4.23E2,5.8E1,4.19E2,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[2.857574E-3,4.955318E-5,3.857612E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":86,"left_children":[1,-1,-1],"loss_changes":[6.47505E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[1.6941456E1,4.955318E-5,3.857612E-3],"split_indices":[43,0,0],"split_type":[0,0,0],"sum_hessian":[4.62E2,4.52E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-1.7028693E-3,-1.1161088E-2,1.9494893E-2,-3.708913E-3,-1.6955636E-3,6.0186395E-3,7.7608344E-4,-9.393261E-4,8.826113E-3,2.6467836E-3,-3.9903975E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,-1,-1,-1,-1,9,-1,-1],"loss_changes":[9.745672E-2,5.8500707E-2,5.4874685E-2,4.9734063E-2,0E0,0E0,0E0,0E0,6.184364E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8],"right_children":[2,4,6,8,-1,-1,-1,-1,10,-1,-1],"split_conditions":[3.1867346E3,1.8071064E7,8.534E3,7.959718E4,-1.6955636E-3,6.0186395E-3,7.7608344E-4,-9.393261E-4,1.0545219E5,2.6467836E-3,-3.9903975E-5],"split_indices":[63,53,9,36,0,0,0,0,36,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.84E2,3.35E2,1.49E2,2.55E2,8E1,4E0,1.45E2,1.14E2,1.41E2,2.4E1,1.17E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[8.170274E-4,-5.6041805E-3,2.528582E-2,-2.3012497E-3,-2.1061806E-3,8.197344E-3,1.016636E-3,-8.236484E-3,1.2487981E-3,-3.0205667E-4,-5.2085053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,7,-1,-1,9,-1,-1,-1],"loss_changes":[7.6344505E-2,5.55518E-2,6.1799057E-2,0E0,6.03817E-2,0E0,0E0,5.8764946E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7],"right_children":[2,4,6,-1,8,-1,-1,10,-1,-1,-1],"split_conditions":[1.6219076E9,9.687575E2,1.6441814E9,-2.3012497E-3,1.5271514E5,8.197344E-3,1.016636E-3,4.4533052E0,1.2487981E-3,-3.0205667E-4,-5.2085053E-3],"split_indices":[7,4,7,0,36,0,0,65,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.84E2,3.84E2,1E2,2.9E1,3.55E2,2E0,9.8E1,2.91E2,6.4E1,2.86E2,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-4.3933634E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0],"id":89,"left_children":[-1],"loss_changes":[0E0],"parents":[2147483647],"right_children":[-1],"split_conditions":[-2.1544595E-4],"split_indices":[0],"split_type":[0],"sum_hessian":[4.79E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"1","size_leaf_vector":"1"}},{"base_weights":[8.8435976E-4,-2.25255E-4,2.071267E-2,1.3719745E-1,7.1696826E-4,1.318601E-4,1.1081506E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":90,"left_children":[1,-1,3,5,-1,-1,-1],"loss_changes":[5.1687036E-2,0E0,7.1088985E-2,5.9098102E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3],"right_children":[2,-1,4,6,-1,-1,-1],"split_conditions":[1.4689211E7,-2.25255E-4,1.906E3,8.8E2,7.1696826E-4,1.318601E-4,1.1081506E-2],"split_indices":[1,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.74E2,3.72E2,1.02E2,4E0,9.8E1,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-1.280401E-3,-6.0841837E-4,8.539587E-3,5.2842665E-3,8.061475E-3,3.9461648E-4,-5.135576E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":91,"left_children":[1,-1,3,5,-1,-1,-1],"loss_changes":[5.0584756E-2,0E0,1.2393709E-1,7.439249E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3],"right_children":[2,-1,4,6,-1,-1,-1],"split_conditions":[8.361085E4,-6.0841837E-4,3.305542E4,1.5291998E4,8.061475E-3,3.9461648E-4,-5.135576E-3],"split_indices":[36,0,4,4,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.61E2,2.16E2,2.45E2,2.41E2,4E0,2.36E2,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[1.5944857E-3,-3.0502975E-3,1.7262097E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":92,"left_children":[1,-1,-1],"loss_changes":[5.869324E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[6.744371E2,-3.0502975E-3,1.7262097E-4],"split_indices":[63,0,0],"split_type":[0,0,0],"sum_hessian":[4.75E2,1.3E1,4.62E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-2.7383037E-3,-1.4535351E-2,9.370271E-3,-1.1799012E-2,-8.552281E-3,-1.7572236E-3,8.5424876E-4,-9.211681E-4,8.4036915E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[6.827346E-2,1.03802145E-1,8.649678E-2,4.8630506E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[4.9762596E1,4.352718E7,7.154521E7,2.0964778E5,-8.552281E-3,-1.7572236E-3,8.5424876E-4,-9.211681E-4,8.4036915E-4],"split_indices":[64,53,7,36,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,2.41E2,2.35E2,2.38E2,3E0,3.5E1,2E2,1.92E2,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.048444E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0],"id":94,"left_children":[-1],"loss_changes":[0E0],"parents":[2147483647],"right_children":[-1],"split_conditions":[-5.1414594E-5],"split_indices":[0],"split_type":[0],"sum_hessian":[4.89E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"1","size_leaf_vector":"1"}},{"base_weights":[6.927139E-4,1.3116467E-2,-4.3621365E-4,-3.299449E-2,1.9378321E-2,-3.054494E-3,2.512144E-3,3.5062013E-3,6.131051E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,7,-1,-1,-1,-1],"loss_changes":[5.634059E-2,6.0119826E-2,0E0,6.486787E-2,6.432987E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4],"right_children":[2,4,-1,6,8,-1,-1,-1,-1],"split_conditions":[1.26E2,4.241E3,-4.3621365E-4,1.5568828E3,1.3474197E8,-3.054494E-3,2.512144E-3,3.5062013E-3,6.131051E-4],"split_indices":[3,2,0,4,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.71E2,2.05E2,2.66E2,2.4E1,1.81E2,1.8E1,6E0,2E1,1.61E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-5.720851E-4,-2.0742682E-3,1.9913102E-3,-4.339295E-3,1.2192891E-3,6.144285E-3,-1.8751515E-2,4.391336E-4,-3.7189496E-3,-3.5483576E-3,-1.23069985E-2,-2.623621E-3,-3.1499178E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,-1,3,5,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[5.1746383E-2,0E0,6.6134855E-2,5.4206032E-2,0E0,4.8603944E-2,5.127509E-2,0E0,0E0,0E0,6.53152E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,10,10],"right_children":[2,-1,4,6,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[9.687575E2,-2.0742682E-3,1.5271514E5,2.6548672E-2,1.2192891E-3,1.4512005E5,2.9648642E1,4.391336E-4,-3.7189496E-3,-3.5483576E-3,2.3808822E-2,-2.623621E-3,-3.1499178E-5],"split_indices":[4,0,36,65,0,36,64,0,0,0,46,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,2.7E1,4.55E2,3.57E2,9.8E1,2.07E2,1.5E2,2.01E2,6E0,1.5E1,1.35E2,2.9E1,1.06E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan csv ","sqlOp_Scan orc ","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortAggregate","sqlOp_SortMergeJoin","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_dataproc","platform_databricks-aws","platform_databricks-azure","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan json ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[1.4857235E-3,-2.965608E-1,5.363805E-1,-6.081042E-1,-1.9329263E-1,6.194652E-1,-5.773755E-1,-6.254226E-1,-1.5078232E-3,-2.2354639E-1,3.1191745E-1,3.4459683E-1,9.4478184E-1,-9.608562E-1,-9.766551E-2,-3.2782823E-2,-1.8114489E-2,-3.2691827E-1,-1.3094649E-1,1.8283451E-2,1.4088118E-3,2.9292017E-1,5.199973E-2,7.9174286E-1,1.3544148E0,-5.6336094E-2,-2.0924874E-2,-1.0740333E-2,4.5585646E-3,-3.7175104E-1,-7.4982226E-2,-2.3960607E-2,-2.4165808E-1,3.1552166E-2,3.5883486E-1,8.375097E-1,-9.4547E-3,5.445212E-1,7.4935496E-2,-2.2782652E-1,-4.2883322E-1,4.2892373E-3,-1.2233638E-1,8.3871454E-2,-1.7294605E-1,-1.07227445E-1,-3.0571964E-1,1.5722747E-1,-2.76379E-1,4.087299E-1,1.69263E-1,8.51175E-1,5.2201133E-3,4.017866E-2,-5.8419164E-3,-2.6189747E-1,-2.2977677E-3,-1.0398948E-2,-4.520926E-1,-7.7472785E-3,1.3686883E-3,-7.307728E-2,1.8169455E-1,-1.2570597E-3,-2.5009203E-1,-8.530849E-3,-2.478975E-2,-3.241502E-1,-3.6085164E-4,2.5900844E-1,-5.5516716E-3,-1.648653E-2,-1.5577908E-3,4.6767187E-1,2.689842E-1,-1.61118E-3,1.2648332E-2,1.4754654E-2,4.3465957E-2,-1.4071626E-2,-2.8276553E-3,-4.6481928E-1,-9.79002E-3,9.4813167E-4,-1.5468666E-1,2.328049E-1,-9.094357E-4,-8.83379E-4,-2.8448862E-1,-5.1299143E-3,1.5616007E-3,-3.3635673E-1,1.355652E-3,1.464775E-2,-3.4198043E-4,3.648782E-1,2.7636437E-2,-1.1774825E-3,1.561549E-2,-2.371319E-2,-8.558666E-3,-1.0140763E-2,3.3650717E-3,1.4689181E-2,6.7440243E-3,-1.5088612E-2,-1.8829007E-3,-1.7929971E-2,-7.3988643E-3,1.9438792E-2,6.1387527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,-1,35,37,-1,-1,-1,-1,39,41,43,45,47,49,51,-1,53,-1,55,57,-1,59,61,63,65,67,69,71,73,75,77,-1,-1,-1,79,-1,-1,81,-1,-1,83,85,-1,87,-1,89,91,-1,93,-1,-1,-1,95,97,-1,-1,-1,-1,-1,-1,99,-1,-1,101,103,-1,-1,105,-1,-1,107,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.7276375E1,8.601212E0,1.4330605E1,6.5172577E-1,3.2030501E0,1.2282547E1,1.9297128E0,2.843647E-1,0E0,1.8142033E0,1.7329252E-1,2.5002213E0,2.9397964E0,6.932497E-2,1.5247346E-1,0E0,0E0,9.9997044E-1,1.2260457E0,0E0,0E0,1.2635498E0,0E0,2.3582478E0,1.062643E0,0E0,0E0,0E0,0E0,5.3489494E-1,1.24723546E-1,8.820373E-1,4.0642214E-1,6.626438E-1,4.8887968E-1,3.8191223E-1,0E0,7.803595E-1,0E0,1.3665736E-1,1.8319893E-1,0E0,6.296696E-2,5.045805E-1,2.5594664E-1,8.995195E-2,1.847105E-1,3.5648343E-1,5.5321872E-2,2.5310087E-1,2.4463114E-1,2.178688E-1,0E0,0E0,0E0,6.483948E-2,0E0,0E0,6.545162E-2,0E0,0E0,1.0059145E-1,2.0856738E-1,0E0,1.14886224E-1,0E0,4.2473387E-2,1.463356E-1,0E0,8.3563745E-2,0E0,0E0,0E0,7.011032E-2,2.1085656E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6504974E-2,0E0,0E0,8.861235E-2,5.6034446E-2,0E0,0E0,4.4543862E-2,0E0,0E0,7.772994E-2,0E0,0E0,0E0,5.132079E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,55,55,58,58,61,61,62,62,64,64,66,66,67,67,69,69,73,73,74,74,81,81,84,84,85,85,88,88,91,91,95,95],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,-1,36,38,-1,-1,-1,-1,40,42,44,46,48,50,52,-1,54,-1,56,58,-1,60,62,64,66,68,70,72,74,76,78,-1,-1,-1,80,-1,-1,82,-1,-1,84,86,-1,88,-1,90,92,-1,94,-1,-1,-1,96,98,-1,-1,-1,-1,-1,-1,100,-1,-1,102,104,-1,-1,106,-1,-1,108,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,1.0429407E3,1.2730462E7,1.2678202E9,6.0891E4,1.700398E4,2.9778808E7,-1.5078232E-3,7.377246E2,1.5E2,7.1853364E3,8.4987135E0,3.1E1,1.6130411E5,-3.2782823E-2,-1.8114489E-2,7.164074E8,2.1692945E1,1.8283451E-2,1.4088118E-3,2.3890688E3,5.199973E-2,1.4656219E8,6.069097E4,-5.6336094E-2,-2.0924874E-2,-1.0740333E-2,4.5585646E-3,7.37365E6,4.45E3,1.6777562E7,8.237054E6,1.3508742E7,2.47E2,7.2125336E2,-9.4547E-3,1.207E3,7.4935496E-2,6.1614815E6,5.1547173E1,4.2892373E-3,6.46E2,2.753337E6,2.58E2,1.849533E6,5.9E2,2.9743198E1,1.2587E4,6.0796064E8,1.3428E4,3.0811954E-2,5.2201133E-3,4.017866E-2,-5.8419164E-3,3.01E2,-2.2977677E-3,-1.0398948E-2,2.779416E8,-7.7472785E-3,1.3686883E-3,6.59701E5,7.753E3,-1.2570597E-3,2.4401142E9,-8.530849E-3,3.4458816E8,3.15584E6,-3.6085164E-4,1.14E2,-5.5516716E-3,-1.648653E-2,-1.5577908E-3,2.0663544E9,6.168645E8,-1.61118E-3,1.2648332E-2,1.4754654E-2,4.3465957E-2,-1.4071626E-2,-2.8276553E-3,7.177E3,-9.79002E-3,9.4813167E-4,1.852E3,9.663487E0,-9.094357E-4,-8.83379E-4,9E1,-5.1299143E-3,1.5616007E-3,9.072834E8,1.355652E-3,1.464775E-2,-3.4198043E-4,1.9412996E9,2.7636437E-2,-1.1774825E-3,1.561549E-2,-2.371319E-2,-8.558666E-3,-1.0140763E-2,3.3650717E-3,1.4689181E-2,6.7440243E-3,-1.5088612E-2,-1.8829007E-3,-1.7929971E-2,-7.3988643E-3,1.9438792E-2,6.1387527E-3],"split_indices":[2,1,70,9,7,2,45,12,0,64,3,4,65,3,40,0,0,7,65,0,0,4,0,1,40,0,0,0,0,57,2,57,57,57,3,70,0,0,0,57,60,0,8,1,3,1,8,65,2,44,2,50,0,0,0,8,0,0,12,0,0,41,2,0,5,0,7,1,0,3,0,0,0,12,44,0,0,0,0,0,0,2,0,0,0,65,0,0,8,0,0,7,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,2.7E2,1.5E2,6.6E1,2.04E2,1.4E2,1E1,6.4E1,2E0,1.93E2,1.1E1,7.7E1,6.3E1,5E0,5E0,5.6E1,8E0,9E1,1.03E2,9E0,2E0,7.3E1,4E0,4.8E1,1.5E1,3E0,2E0,3E0,2E0,7.6E1,1.4E1,5.3E1,5E1,1.5E1,5.8E1,4.6E1,2E0,3E0,1.2E1,2.3E1,5.3E1,3E0,1.1E1,3.1E1,2.2E1,1.7E1,3.3E1,1.1E1,4E0,4.5E1,1.3E1,4.5E1,1E0,2E0,1E0,1.9E1,4E0,6E0,4.7E1,9E0,2E0,1.2E1,1.9E1,8E0,1.4E1,9E0,8E0,3.1E1,2E0,8E0,3E0,3E0,1E0,3E1,1.5E1,4E0,9E0,2E0,4.3E1,1.7E1,2E0,4.4E1,3E0,6E0,6E0,1.5E1,4E0,2E0,1.2E1,3E0,5E0,3E1,1E0,7E0,1E0,1.6E1,1.4E1,2E0,1.3E1,4.2E1,2E0,5E0,1E0,8E0,7E0,1.1E1,1E0,2.6E1,4E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[1.6865663E-2,-2.8437036E-1,5.739436E-1,-5.9718937E-1,-1.8265463E-1,4.3719468E-1,1.0980139E0,-6.337835E-1,-3.1790537E-1,-2.1990728E-1,3.2870778E-1,4.833704E-1,-5.284346E-2,-2.8204769E-2,1.2196875E0,-3.2210924E-2,-1.0735854E-2,-1.8561725E-2,-3.4609516E-3,-2.7191135E-1,3.081771E-3,3.9041665E-1,-2.4121741E-3,3.2801256E-1,6.610112E-1,-9.139693E-3,1.0890412E-2,6.019764E-1,1.3969256E0,-3.0097374E-1,-9.944806E-2,-1.1908863E-1,1.2509808E-1,2.7008452E-2,1.1449842E-2,3.710176E-1,5.0082535E-2,2.0846318E-2,7.108115E-1,3.6009606E-2,-5.6959386E-3,7.134287E-2,1.9606447E-2,-3.162087E-1,8.943549E-2,-1.7669444E-1,-1.039947E-3,-1.5393186E-1,2.9369553E-3,-1.32860895E-2,2.3952933E-1,-2.3339844E-4,3.847639E-1,-9.213355E-2,2.1072534E-1,-1.4842666E-2,1.6579028E-2,2.6544219E-2,3.923254E-2,-3.6207968E-1,-2.43738E-1,-6.1127082E-3,7.8086457E-3,7.921228E-4,-9.631631E-3,-5.9219208E-3,2.5626682E-3,1.4226036E-3,-8.351544E-3,-3.3814607E-3,2.729779E-3,4.7222944E-3,1.5438092E-2,2.4136802E-2,3.3209166E-1,7.6020934E-4,-7.476868E-3,-2.1661585E-4,1.3272341E-2,-2.533375E-1,-4.097456E-1,-2.9037826E-2,-3.16342E-1,3.4128216E-1,-4.2847602E-4,-4.3180785E-3,-1.4360616E-2,-2.2047773E-2,-1.3829033E-2,5.3282483E-3,-6.562854E-3,-4.335553E-3,-1.693279E-2,1.0201701E-2,1.8831342E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,33,-1,35,37,-1,-1,39,41,43,45,47,49,-1,-1,51,53,55,57,-1,-1,-1,-1,59,61,63,65,67,-1,69,71,-1,73,75,77,-1,-1,-1,-1,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,83,-1,-1,-1,-1,85,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3161415E1,8.895588E0,1.0484856E1,5.03706E-1,4.167951E0,8.99968E0,4.282364E0,1.3160515E-1,1.07325315E-1,2.3317013E0,3.6778414E-1,3.0949268E0,0E0,1.9212641E-1,2.2245827E0,0E0,0E0,0E0,0E0,7.870569E-1,5.962622E-1,1.9076216E-1,0E0,7.70277E-1,1.7386017E0,0E0,0E0,7.6118803E-1,8.5819244E-2,8.550329E-1,1.89858E-1,1.3298753E-1,3.1989187E-1,0E0,0E0,2.961626E-1,2.4787478E-1,5.9259874E-1,2.6309776E-1,0E0,0E0,0E0,0E0,3.5720348E-1,1.03995174E-1,4.6266437E-2,7.981616E-2,4.5621693E-2,0E0,4.0620986E-2,8.1501424E-2,0E0,1.2410259E-1,3.929394E-2,6.0129657E-2,0E0,0E0,0E0,0E0,3.1085968E-1,8.481598E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.25031E-1,0E0,0E0,0E0,0E0,1.2847352E-1,1.0364342E-1,2.2212505E-1,1.8217516E-1,1.0822773E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,19,19,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,43,43,44,44,45,45,46,46,47,47,49,49,50,50,52,52,53,53,54,54,59,59,60,60,74,74,79,79,80,80,81,81,82,82,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,34,-1,36,38,-1,-1,40,42,44,46,48,50,-1,-1,52,54,56,58,-1,-1,-1,-1,60,62,64,66,68,-1,70,72,-1,74,76,78,-1,-1,-1,-1,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,84,-1,-1,-1,-1,86,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.6512886E3,3.5002026E8,1.4777102E9,4.673502E7,5.0314346E1,2.9778808E7,4.48149E5,4.1619555E8,1.87E2,4.1331047E-1,-5.284346E-2,3.677E3,6.069097E4,-3.2210924E-2,-1.0735854E-2,-1.8561725E-2,-3.4609516E-3,7.10711E8,1.347372E3,6.088569E6,-2.4121741E-3,2.7246006E7,2.1853803E4,-9.139693E-3,1.0890412E-2,3.14E2,2.3322949E2,3.3616E4,2.6452028E7,1E0,7.959718E4,2.7008452E-2,1.1449842E-2,2.56E2,8.5E1,3.11E3,1E0,3.6009606E-2,-5.6959386E-3,7.134287E-2,1.9606447E-2,7.8936383E2,2.094E3,4.45E3,1E2,6.7370984E7,2.9369553E-3,1.6602323E5,8.307654E8,-2.3339844E-4,1.4040298E5,6.27427E5,1.1459E4,-1.4842666E-2,1.6579028E-2,2.6544219E-2,3.923254E-2,7.517043E6,4.0757E1,-6.1127082E-3,7.8086457E-3,7.921228E-4,-9.631631E-3,-5.9219208E-3,2.5626682E-3,1.4226036E-3,-8.351544E-3,-3.3814607E-3,2.729779E-3,4.7222944E-3,1.5438092E-2,2.4136802E-2,3.3400793E0,7.6020934E-4,-7.476868E-3,-2.1661585E-4,1.3272341E-2,7.140485E-2,6.057E3,2.58E2,6.9140625E-1,1.2592834E-1,-4.2847602E-4,-4.3180785E-3,-1.4360616E-2,-2.2047773E-2,-1.3829033E-2,5.3282483E-3,-6.562854E-3,-4.335553E-3,-1.693279E-2,1.0201701E-2,1.8831342E-2],"split_indices":[2,1,64,7,7,60,68,12,1,12,3,47,0,0,40,0,0,0,0,7,64,57,0,57,45,0,0,3,69,10,12,101,40,0,0,0,8,0,102,0,0,0,0,64,0,2,8,7,0,45,12,0,45,9,2,0,0,0,0,57,68,0,0,0,0,0,0,0,0,0,0,0,0,0,53,0,0,0,0,70,2,3,70,50,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,2.82E2,1.52E2,6.8E1,2.14E2,1.22E2,3E1,5.9E1,9E0,2E2,1.4E1,1.19E2,3E0,3E0,2.7E1,5.7E1,2E0,7E0,2E0,1.62E2,3.8E1,1.2E1,2E0,6.5E1,5.4E1,2E0,1E0,7E0,2E1,1.38E2,2.4E1,1.9E1,1.9E1,5E0,7E0,5.6E1,9E0,4E0,5E1,6E0,1E0,1.9E1,1E0,1.33E2,5E0,1.3E1,1.1E1,1.6E1,3E0,9E0,1E1,2E0,5.4E1,5E0,4E0,2E0,2E0,1.7E1,3.3E1,7.9E1,5.4E1,1E0,4E0,1E0,1.2E1,3E0,8E0,1E0,1.5E1,5E0,4E0,4E0,6E0,1.6E1,3.8E1,2E0,3E0,1E0,3E0,2.6E1,5.3E1,1.4E1,4E1,3.7E1,1E0,5E0,2.1E1,4.1E1,1.2E1,6E0,8E0,4E0,3.6E1,9E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-2.027892E-3,-2.295055E-1,5.5601895E-1,-5.4557E-1,-1.2273176E-1,6.451821E-1,-5.60624E-1,-5.726306E-1,-1.526629E-3,-1.7274493E-1,2.7080524E-1,5.3345466E-1,1.1411738E0,9.074902E-2,-1.0099975E0,-2.9976021E-2,-1.6792016E-2,-2.1824294E-1,7.790328E-2,3.311019E-1,2.2905087E-4,4.099725E-1,6.668152E-1,7.477354E-2,4.4949643E-2,-9.704379E-3,1.7263182E-2,-5.5229854E-2,-1.3352525E-2,-2.2830525E-1,3.9050918E-2,1.8723865E-1,-4.9967963E-2,3.5946295E-1,-6.6088433E-3,4.5154583E-1,9.0904115E-3,7.0917875E-1,-8.835182E-3,-9.611593E-2,-2.8868943E-1,-1.8732274E-3,2.54122E-1,-1.6863608E-1,3.0518796E-3,2.36246E-2,1.1750798E-2,3.0921918E-1,5.269149E-1,9.723287E-1,5.0747955E-1,-1.2200937E-1,1.3867086E-2,-3.3915642E-1,-1.7801484E-1,-5.604069E-4,1.4956795E-2,8.461375E-4,-9.873492E-3,1.9507347E-2,7.947918E-3,2.8286302E-2,1.4457171E-2,-5.5536083E-3,5.2104972E-2,5.3694E-1,4.638572E-3,-1.7135441E-1,-2.209195E-2,-3.5352802E-1,-2.3552878E-3,4.9724903E-2,-2.6611355E-1,1.0436168E-2,2.7806325E-2,-1.0598646E-2,-1.2809286E-3,-7.6905536E-3,1.7975719E-3,-1.8079907E-2,-5.526689E-3,6.23212E-3,-2.3006948E-3,-1.4206145E-2,-7.047072E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,49,-1,51,53,-1,55,57,-1,-1,-1,59,61,63,65,67,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,73,-1,75,77,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6109547E1,1.052972E1,1.3105816E1,1.0835152E0,4.69174E0,5.9857254E0,3.0187538E0,2.8077698E-1,0E0,2.411841E0,4.3189478E-1,1.3569202E0,4.5661354E-1,4.299045E-1,1.2871313E-1,0E0,0E0,1.968832E0,4.7072396E-1,3.3663464E-1,0E0,4.298916E-1,1.7693233E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3850918E0,0E0,2.8006697E-1,2.2111863E-1,1.8456244E-1,0E0,3.197832E-1,0E0,1.8419552E0,0E0,5.852461E-1,6.154604E-1,0E0,1.7110074E-1,4.6289936E-2,0E0,0E0,0E0,1.5720212E-1,9.630823E-2,1.4862652E0,2.800007E-1,2.6245767E-1,0E0,3.2748508E-1,8.157753E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3380737E-2,0E0,2.0818305E-1,1.5092234E-1,1.1619663E-1,0E0,9.189924E-2,1.2860465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,29,29,31,31,32,32,33,33,35,35,37,37,39,39,40,40,42,42,43,43,47,47,48,48,49,49,50,50,51,51,53,53,54,54,65,65,67,67,68,68,69,69,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,50,-1,52,54,-1,56,58,-1,-1,-1,60,62,64,66,68,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,74,-1,76,78,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3664E4,7.41718E5,1.0429407E3,1.2852292E7,1.4777102E9,4.026531E9,2.909019E3,2.9778808E7,-1.526629E-3,5.9170656E8,1.436E3,2.1360708E7,2.231E3,2.5212732E4,3.1E1,-2.9976021E-2,-1.6792016E-2,5.2191963E3,1.4674221E6,2.47E2,2.2905087E-4,2.237815E7,1.4656219E8,7.477354E-2,4.4949643E-2,-9.704379E-3,1.7263182E-2,-5.5229854E-2,-1.3352525E-2,7.556039E6,3.9050918E-2,5.0363316E5,1.685028E6,6.3349745E6,-6.6088433E-3,2.1959E4,9.0904115E-3,6.363344E6,-8.835182E-3,3.7069206E5,1.0864745E3,-1.8732274E-3,4.4626824E4,6.691311E8,3.0518796E-3,2.36246E-2,1.1750798E-2,1.8053012E5,1.1448067E-1,8.713209E10,6.2185767E10,7.164074E8,1.3867086E-2,1.6459569E2,4.98884E1,-5.604069E-4,1.4956795E-2,8.461375E-4,-9.873492E-3,1.9507347E-2,7.947918E-3,2.8286302E-2,1.4457171E-2,-5.5536083E-3,5.2104972E-2,6.1707415E-2,4.638572E-3,2.612106E6,1.7597401E3,2.3176976E7,-2.3552878E-3,7.062E3,2.34E2,1.0436168E-2,2.7806325E-2,-1.0598646E-2,-1.2809286E-3,-7.6905536E-3,1.7975719E-3,-1.8079907E-2,-5.526689E-3,6.23212E-3,-2.3006948E-3,-1.4206145E-2,-7.047072E-4],"split_indices":[2,1,70,9,7,7,44,12,0,12,0,1,0,4,3,0,0,4,59,3,0,57,1,0,0,0,0,0,0,57,0,59,9,57,0,2,0,41,0,40,64,0,40,5,0,0,0,45,69,43,5,7,0,66,68,0,0,0,0,0,0,0,0,0,0,50,0,1,45,9,0,2,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.13E2,1.27E2,7.8E1,2.35E2,1.18E2,9E0,7.4E1,4E0,2.09E2,2.6E1,9.8E1,2E1,4E0,5E0,6.5E1,9E0,1.77E2,3.2E1,2.1E1,5E0,5.3E1,4.5E1,6E0,1.4E1,2E0,2E0,4E0,1E0,1.76E2,1E0,1.7E1,1.5E1,2E1,1E0,4.4E1,9E0,4.3E1,2E0,5.6E1,1.2E2,4E0,1.3E1,7E0,8E0,9E0,1.1E1,1.7E1,2.7E1,1.7E1,2.6E1,5.3E1,3E0,8.1E1,3.9E1,2E0,1.1E1,1E0,6E0,1E1,7E0,2.2E1,5E0,1E0,1.6E1,2.4E1,2E0,3.5E1,1.8E1,7.7E1,4E0,1.1E1,2.8E1,2E0,2.2E1,2.7E1,8E0,5E0,1.3E1,7.4E1,3E0,6E0,5E0,2.6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[9.293402E-3,-2.641622E-1,5.168235E-1,-5.402759E-1,-1.6958784E-1,3.7590942E-1,9.6435225E-1,-5.751695E-1,-2.664181E-1,-1.9889978E-1,2.9666507E-1,4.459382E-1,-2.2401074E-1,8.2886785E-2,1.0419737E0,-2.9203515E-2,-9.834308E-3,-1.7775215E-2,-2.1797032E-3,-2.5721452E-1,-1.2755776E-2,2.2628702E-2,7.120982E-3,2.866922E-1,6.4741796E-1,-3.5113607E-2,-1.0170695E-2,1.0945568E-2,-8.133638E-3,5.770292E-1,1.1676749E0,-1.4419201E-1,-2.9330724E-1,-5.1734265E-2,1.1419823E-2,3.4406483E-1,1.1795918E-1,5.0118923E-1,7.5020254E-1,2.0626307E-2,-1.674847E-1,3.3796076E-2,-5.4148347E-3,1.2286386E0,1.5545595E-2,-1.9770765E-1,-7.306183E-3,-3.328452E-1,-1.7018813E-1,-2.1674609E-1,-1.3627906E-2,6.7440574E-3,3.6117536E-1,-1.1420029E-2,2.0852469E-1,5.3781307E-1,4.496206E-3,9.695658E-3,3.8246386E-2,-1.2232458E-2,2.144305E-3,6.874553E-2,4.234175E-2,-5.278183E-3,-2.2926433E-1,-2.7646052E-3,7.7548134E-3,-3.5174114E-1,-1.5406208E-1,4.200838E-3,-2.5122172E-1,-2.0730558E-3,-1.34063E-2,-5.1586296E-2,6.658773E-2,4.2835382E-1,2.7679804E-1,1.611665E-3,1.3783914E-2,9.56936E-3,2.826932E-2,-1.22741135E-2,-2.7402095E-3,-9.011662E-3,-3.6630732E-1,-8.935276E-3,1.7116477E-3,-1.0756968E-3,-2.7207273E-1,4.720956E-3,-8.390376E-2,-1.8053015E-3,8.761774E-3,2.2050852E-2,4.269671E-3,-4.7856346E-3,1.500351E-2,-2.0779597E-2,-1.5393791E-2,-6.7824116E-3,-1.5302828E-2,1.5795637E-3,-5.243059E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,35,37,-1,39,-1,-1,41,43,45,47,49,-1,51,53,55,57,-1,59,-1,-1,61,-1,63,65,67,69,71,73,-1,75,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,81,-1,-1,83,85,-1,87,-1,-1,89,91,93,95,-1,-1,-1,-1,-1,-1,-1,97,-1,-1,-1,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0647636E1,7.324545E0,9.286148E0,5.3479004E-1,2.9700303E0,5.057274E0,2.3767815E0,1.0385513E-1,1.833533E-1,2.1785617E0,2.4840605E-1,3.2153835E0,1.3233168E0,1.6939883E-1,1.2548676E0,0E0,0E0,0E0,0E0,5.8183956E-1,4.7261807E-1,0E0,0E0,5.499344E-1,3.1538963E-1,0E0,7.343967E-1,0E0,0E0,6.8527246E-1,9.233818E-1,2.8425115E-1,5.031595E-1,2.6743016E-1,0E0,1.1227703E-1,5.8107847E-1,2.829461E-1,6.137085E-2,0E0,1.4057702E-1,0E0,0E0,1.2345505E-1,0E0,5.7159066E-2,1.021945E-1,2.3634243E-1,6.3918877E-1,6.10933E-2,1.1304985E-1,0E0,1.1383438E-1,0E0,1.5715694E-1,8.455467E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3522835E-2,0E0,0E0,1.2621403E-1,5.2690968E-2,0E0,1.0430062E-1,0E0,0E0,1.2592553E-1,1.4034833E-1,5.5015802E-2,2.1194541E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.381439E-2,0E0,0E0,0E0,5.6177855E-2,0E0,5.4274276E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,23,23,24,24,26,26,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,40,40,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,54,54,55,55,64,64,67,67,68,68,70,70,73,73,74,74,75,75,76,76,84,84,88,88,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,36,38,-1,40,-1,-1,42,44,46,48,50,-1,52,54,56,58,-1,60,-1,-1,62,-1,64,66,68,70,72,74,-1,76,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,82,-1,-1,84,86,-1,88,-1,-1,90,92,94,96,-1,-1,-1,-1,-1,-1,-1,98,-1,-1,-1,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.0495757E3,3.5002026E8,1.4096699E9,3.5301748E7,5.3140095E1,2.9778808E7,1.0321884E3,9.7467445E4,6.088569E6,2.7371603E-1,2.2782404E0,1.0014992E8,6.069097E4,-2.9203515E-2,-9.834308E-3,-1.7775215E-2,-2.1797032E-3,7.556039E6,1.4998456E5,2.2628702E-2,7.120982E-3,1.7458724E7,1.2384782E5,-3.5113607E-2,1.1E1,1.0945568E-2,-8.133638E-3,2.96E2,2.3322949E2,7.192302E2,1.1883916E3,1.6359736E3,1.1419823E-2,2.1E1,1.2584659E5,1.4485321E9,2.4768E4,2.0626307E-2,9.704588E6,3.3796076E-2,-5.4148347E-3,6.0976E4,1.5545595E-2,1.6459E4,4.52737E5,6.4065235E8,4.98884E1,1.4590734E8,1.207E3,6.7440574E-3,1.15E2,-1.1420029E-2,4.2220557E3,2E0,4.496206E-3,9.695658E-3,3.8246386E-2,-1.2232458E-2,2.144305E-3,6.874553E-2,4.234175E-2,-5.278183E-3,2.8246716E7,-2.7646052E-3,7.7548134E-3,5.125401E1,7.792E3,4.200838E-3,4.041298E-1,-2.0730558E-3,-1.34063E-2,3.44094E1,1.133457E6,1.8956E4,2.4903E4,1.611665E-3,1.3783914E-2,9.56936E-3,2.826932E-2,-1.22741135E-2,-2.7402095E-3,-9.011662E-3,2.72E2,-8.935276E-3,1.7116477E-3,-1.0756968E-3,6.053216E1,4.720956E-3,4.1286648E1,-1.8053015E-3,8.761774E-3,2.2050852E-2,4.269671E-3,-4.7856346E-3,1.500351E-2,-2.0779597E-2,-1.5393791E-2,-6.7824116E-3,-1.5302828E-2,1.5795637E-3,-5.243059E-3],"split_indices":[2,1,64,7,7,57,68,12,4,45,57,47,66,1,40,0,0,0,0,57,40,0,0,57,40,0,8,0,0,3,69,64,64,67,0,8,45,44,10,0,9,0,0,10,0,9,41,7,68,5,0,0,3,0,4,8,0,0,0,0,0,0,0,0,44,0,0,60,2,0,70,0,0,70,9,0,9,0,0,0,0,0,0,0,3,0,0,0,68,0,68,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,2.83E2,1.52E2,7.1E1,2.12E2,1.17E2,3.5E1,6.2E1,9E0,2E2,1.2E1,1.05E2,1.2E1,3E0,3.2E1,6E1,2E0,6E0,3E0,1.52E2,4.8E1,5E0,7E0,6E1,4.5E1,3E0,9E0,2E0,1E0,8E0,2.4E1,3.8E1,1.14E2,4.2E1,6E0,4.4E1,1.6E1,2.1E1,2.4E1,2E0,7E0,7E0,1E0,2.2E1,2E0,2.7E1,1.1E1,8.5E1,2.9E1,7E0,3.5E1,4E0,4E1,3E0,1.3E1,1.9E1,2E0,1E0,2.3E1,5E0,2E0,1.4E1,8E0,8E0,1.9E1,9E0,2E0,7.6E1,9E0,7E0,2.2E1,2E0,5E0,2.4E1,1.1E1,2E1,2E1,4E0,9E0,2E0,1.7E1,1.7E1,2E0,7E0,6.9E1,8E0,1E0,2E0,2E1,4E0,2E1,6E0,5E0,1.9E1,1E0,1E0,1.9E1,3.4E1,3.5E1,5E0,1.5E1,3E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.7218053E-2,-2.2466917E-1,4.7799495E-1,-4.9538025E-1,-1.4453138E-1,3.4568125E-1,8.8003534E-1,-5.154521E-1,-1.554426E-1,-1.746414E-1,2.766076E-1,4.1629118E-1,-2.8330997E-1,2.774607E-2,9.857045E-1,-2.6679318E-2,-1.4990479E-2,2.295397E-3,-1.08582135E-2,-2.2530545E-1,-4.2338395E-3,-3.3439083E-3,3.3460343E-1,2.861624E-1,6.0291785E-1,-5.9500265E-1,2.432807E-2,1.0580891E-2,-8.269824E-3,6.563999E-2,7.770379E-1,-2.3631316E-1,1.587932E-1,-7.746222E-2,1.2211691E-1,2.1618191E-2,7.4071134E-3,-1.7245399E-3,3.030525E-1,3.5442153E-1,3.5264783E-2,-4.1049374E-3,-3.404111E-2,-7.2548673E-3,1.4928363E-2,4.2733055E-2,-2.1891594E-3,-2.5941142E-1,-9.347482E-2,1.3056653E-2,-6.27365E-3,-1.202256E-2,-1.8007313E-1,-4.2224615E-3,1.8718232E-1,6.770307E-2,3.1893787E-1,3.9174935E-1,4.3464038E-3,-5.2795284E-3,5.11542E-3,-2.929884E-1,-1.5798415E-1,-1.4242786E-1,-1.5003218E-3,-1.01551205E-1,2.8049478E-3,-9.976153E-3,8.727457E-4,1.3414694E-2,6.759633E-2,-3.9295065E-3,9.568731E-3,2.674296E-1,3.840444E-1,3.458414E-3,2.0729125E-2,-1.796125E-1,-3.197564E-1,-2.7858168E-1,2.6367754E-3,3.8412923E-3,-9.099513E-3,3.027823E-3,-3.1627875E-3,-7.4748807E-3,-2.0672473E-4,8.963537E-3,-6.4864237E-4,1.8115351E-2,2.062105E-1,2.0484848E-2,4.1832784E-3,-1.1119628E-2,-3.1012138E-3,-1.3562522E-2,-1.9813031E-2,-1.51024815E-2,-5.426561E-3,4.810513E-3,-4.0417435E-3,-1.9018073E-3,1.094692E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,-1,65,67,-1,69,71,73,75,-1,-1,-1,77,79,81,83,85,-1,-1,-1,-1,87,-1,-1,89,91,-1,-1,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7922035E1,6.0461645E0,7.568714E0,3.565588E-1,2.8253374E0,5.136613E0,3.2148342E0,7.764816E-2,7.222898E-2,1.7689204E0,3.2122743E-1,2.3045864E0,1.1651378E0,2.1277224E-1,1.167736E0,0E0,0E0,0E0,0E0,7.055578E-1,4.5357752E-1,0E0,1.7326915E-1,3.4625864E-1,7.964096E-1,2.0913696E-1,3.689072E-1,0E0,0E0,0E0,1.3395767E0,4.838457E-1,1.784637E-1,2.0599455E-1,2.5778794E-1,0E0,0E0,0E0,1.989851E-1,1.05697274E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.3259766E-2,4.032545E-1,1.0334225E-1,0E0,0E0,1.2772425E-1,4.9817175E-2,0E0,1.1786789E-1,1.0560635E-1,8.173895E-2,6.044185E-2,0E0,0E0,0E0,2.399826E-1,6.7870164E-1,1.444163E-1,3.835832E-2,4.1480213E-2,0E0,0E0,0E0,0E0,6.536373E-2,0E0,0E0,1.23096466E-1,1.3648319E-1,0E0,0E0,9.2068195E-2,1.8180275E-1,4.7468662E-2,1.3287576E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.0430806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,23,23,24,24,25,25,26,26,30,30,31,31,32,32,33,33,34,34,38,38,39,39,46,46,47,47,48,48,51,51,52,52,54,54,55,55,56,56,57,57,61,61,62,62,63,63,64,64,65,65,70,70,73,73,74,74,77,77,78,78,79,79,80,80,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,-1,66,68,-1,70,72,74,76,-1,-1,-1,78,80,82,84,86,-1,-1,-1,-1,88,-1,-1,90,92,-1,-1,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,5.932393E3,6.932264E8,1.4777102E9,3.3503532E7,5.3140095E1,2.9778808E7,2.3E2,9.7467445E4,1E0,4.1331047E-1,3.5226266E0,1.0014992E8,1.417988E6,-2.6679318E-2,-1.4990479E-2,2.295397E-3,-1.08582135E-2,3.7069206E5,3.23643E6,-3.3439083E-3,6.1614815E6,1E0,1E0,1.6799316E7,4.579283E6,1.0580891E-2,-8.269824E-3,6.563999E-2,2.0531621E4,7.164074E8,1.0767E4,7.03764E5,4.3E1,2.1618191E-2,7.4071134E-3,-1.7245399E-3,2.0772269E8,1.5631589E9,3.5264783E-2,-4.1049374E-3,-3.404111E-2,-7.2548673E-3,1.4928363E-2,4.2733055E-2,1.379E3,1.0864745E3,3.0811954E-2,1.3056653E-2,-6.27365E-3,1.2103396E5,1.7997152E7,-4.2224615E-3,2.9194368E1,3.2604893E3,1.74384E5,1.872503E4,4.3464038E-3,-5.2795284E-3,5.11542E-3,6.1453125E1,7.203055E10,4.45E3,9.481747E8,6.2581446E8,2.8049478E-3,-9.976153E-3,8.727457E-4,1.3414694E-2,7.9E1,-3.9295065E-3,9.568731E-3,8.57253E8,7.378667E12,3.458414E-3,2.0729125E-2,1.0038E4,9E1,1.6505348E9,7.062E3,3.8412923E-3,-9.099513E-3,3.027823E-3,-3.1627875E-3,-7.4748807E-3,-2.0672473E-4,8.963537E-3,-6.4864237E-4,1.8115351E-2,8.15E2,2.0484848E-2,4.1832784E-3,-1.1119628E-2,-3.1012138E-3,-1.3562522E-2,-1.9813031E-2,-1.51024815E-2,-5.426561E-3,4.810513E-3,-4.0417435E-3,-1.9018073E-3,1.094692E-2],"split_indices":[2,1,64,7,7,57,68,12,0,45,13,47,66,1,41,0,0,0,0,40,1,0,57,8,102,1,41,0,0,0,4,7,0,41,8,0,0,0,5,44,0,0,0,0,0,0,0,64,50,0,0,45,9,0,66,4,11,45,0,0,0,60,43,2,7,7,0,0,0,0,3,0,0,5,43,0,0,0,8,5,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.81E2,1.47E2,6.3E1,2.18E2,1.12E2,3.5E1,5.9E1,4E0,2.04E2,1.4E1,1.01E2,1.1E1,4E0,3.1E1,5.3E1,6E0,1E0,3E0,1.57E2,4.7E1,2E0,1.2E1,6.1E1,4E1,5E0,6E0,2E0,2E0,1E1,2.1E1,1.53E2,4E0,3E1,1.7E1,7E0,5E0,3E0,5.8E1,1.3E1,2.7E1,1E0,4E0,4E0,2E0,1.9E1,2E0,1.31E2,2.2E1,3E0,1E0,1.9E1,1.1E1,4E0,1.3E1,4E0,5.4E1,1.1E1,2E0,1E0,1E0,9.7E1,3.4E1,1.4E1,8E0,8E0,1.1E1,1E1,1E0,7E0,6E0,2E0,2E0,3.3E1,2.1E1,1E0,1E1,2E1,7.7E1,1.9E1,1.5E1,2E0,1.2E1,4E0,4E0,5E0,3E0,2E0,4E0,1.1E1,2.2E1,1.9E1,2E0,1.4E1,6E0,5E1,2.7E1,1.6E1,3E0,7E0,8E0,1E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.3297128E-2,-2.032274E-1,5.2392906E-1,-4.5092753E-1,-1.2879252E-1,5.9602433E-1,-4.0802637E-1,-4.7635454E-1,-1.0005311E-3,-1.707354E-1,2.3706791E-1,4.570334E-1,1.0259942E0,-7.347379E-1,3.2800676E-3,-2.5173776E-2,-1.4556293E-2,-2.8419396E-1,-9.4364844E-2,3.1133983E-1,4.032263E-2,3.0342624E-1,5.9791887E-1,1.1265488E0,3.1802255E-1,-4.3545235E-2,-1.53783355E-2,-1.7457744E-1,-3.2878485E-1,-1.4484467E-1,2.4969776E-1,1.7776366E-2,8.163177E-3,6.0333996E-3,-3.511415E-3,3.9762172E-1,2.3124896E-1,4.5573443E-1,3.501827E-2,6.958406E-2,4.2805854E-2,3.2766066E-2,-6.276865E-3,-1.4203016E-3,-1.9876055E-1,-3.5112506E-1,-6.9808033E-3,-2.0976418E-1,-1.4620871E-2,-6.740831E-3,4.2987886E-1,2.2030275E-2,6.268879E-3,1.4800412E-2,1.5093143E-1,4.8712742E-1,4.201592E-3,-3.595816E-3,-2.4974293E-1,-2.1475814E-2,-2.9456446E-1,-7.715504E-2,-2.4666588E-1,-4.6119086E-2,1.3595942E-1,-3.2261293E-3,3.358901E-3,1.2899818E-2,3.1134473E-2,8.287592E-3,-1.4409218E-3,8.559894E-3,2.5555762E-2,-4.5632706E-3,-1.4740203E-2,-2.343262E-1,-3.854741E-1,-1.03635825E-2,2.456147E-2,-2.8377247E-1,-7.676894E-3,-9.007862E-2,6.2178336E-3,-1.7946715E-3,8.525397E-3,-1.2430013E-2,2.0570315E-3,-1.0125012E-2,-2.2568597E-2,3.976036E-3,-7.4273245E-3,-1.5749516E-2,-5.7464684E-3,-7.602002E-3,-1.8378153E-3,4.099022E-3,-3.0911432E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,57,59,-1,61,63,65,67,-1,-1,-1,69,71,-1,-1,73,-1,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,87,-1,89,91,-1,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8644222E1,5.639121E0,9.0450325E0,7.6838684E-1,3.7079883E0,6.8220406E0,1.5957217E0,1.9259834E-1,0E0,1.8361335E0,3.527621E-1,1.8528328E0,1.7062359E0,8.182907E-2,0E0,0E0,0E0,3.6296082E-1,2.2940357E0,6.006694E-2,7.961315E-2,2.2759247E-1,3.821373E-1,6.016998E-1,8.31204E-1,0E0,0E0,8.9784205E-2,2.0442867E-1,9.6069646E-1,7.883892E-1,0E0,0E0,0E0,0E0,1.6808629E-1,1.0646939E-1,2.2740078E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3072723E-1,1.0902929E-1,0E0,3.5287476E-1,1.9124773E-1,3.8541064E-2,1.7087603E-1,0E0,0E0,0E0,4.4957638E-2,7.3143005E-2,0E0,0E0,8.7653756E-2,0E0,9.586549E-2,2.4350512E-1,1.6013265E-1,7.6483406E-2,4.7789097E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0143435E-1,5.3597927E-2,0E0,1.1659201E-1,1.808443E-1,0E0,5.3927094E-2,8.7630756E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,35,35,36,36,37,37,44,44,45,45,47,47,48,48,49,49,50,50,54,54,55,55,58,58,60,60,61,61,62,62,63,63,64,64,75,75,76,76,78,78,79,79,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,58,60,-1,62,64,66,68,-1,-1,-1,70,72,-1,-1,74,-1,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,88,-1,90,92,-1,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.32424E5,1.0429407E3,1.2852292E7,1.4777102E9,7.7101436E3,1.31805E5,2.4434292E7,-1.0005311E-3,7.377246E2,5.4629724E10,1.8618166E-1,1.92E2,2.8E1,3.2800676E-3,-2.5173776E-2,-1.4556293E-2,7.37365E6,2.3171705E5,1.2155118E7,2.341E4,1.8887723E-2,1.30817086E5,3.3616E4,2.235E5,-4.3545235E-2,-1.53783355E-2,8.87682E5,3.312442E6,2.8884522E8,3.724162E6,1.7776366E-2,8.163177E-3,6.0333996E-3,-3.511415E-3,6.2185767E10,1.8053012E5,1.7092716E9,3.501827E-2,6.958406E-2,4.2805854E-2,3.2766066E-2,-6.276865E-3,-1.4203016E-3,1.6459E4,1.934371E6,-6.9808033E-3,4.2393894E1,2.1145713E3,1.214E3,1.901216E3,2.2030275E-2,6.268879E-3,1.4800412E-2,3.007E3,2E0,4.201592E-3,-3.595816E-3,6.1E1,-2.1475814E-2,6.9E1,8.8543115E2,1.4849418E3,1.07E3,8.42E2,-3.2261293E-3,3.358901E-3,1.2899818E-2,3.1134473E-2,8.287592E-3,-1.4409218E-3,8.559894E-3,2.5555762E-2,-4.5632706E-3,-1.4740203E-2,4.61E2,2.362677E6,-1.03635825E-2,3E2,2.3961474E7,-7.676894E-3,9.704588E6,1.4040858E-2,-1.7946715E-3,8.525397E-3,-1.2430013E-2,2.0570315E-3,-1.0125012E-2,-2.2568597E-2,3.976036E-3,-7.4273245E-3,-1.5749516E-2,-5.7464684E-3,-7.602002E-3,-1.8378153E-3,4.099022E-3,-3.0911432E-3],"split_indices":[2,1,70,9,7,64,40,12,0,64,43,47,8,3,0,0,0,57,40,57,11,69,40,10,2,0,0,1,9,12,1,0,0,0,0,5,45,44,0,0,0,0,0,0,9,1,0,68,64,0,4,0,0,0,0,8,0,0,3,0,8,64,64,0,0,0,0,0,0,0,0,0,0,0,0,3,1,0,3,9,0,9,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,3.08E2,1.3E2,7E1,2.38E2,1.21E2,9E0,6.6E1,4E0,2.14E2,2.4E1,9.3E1,2.8E1,5E0,4E0,5.6E1,1E1,8.5E1,1.29E2,1.7E1,7E0,4.6E1,4.7E1,2.4E1,4E0,3E0,2E0,2.6E1,5.9E1,1.13E2,1.6E1,1.2E1,5E0,4E0,3E0,1.8E1,2.8E1,2.2E1,2.5E1,1E1,1.4E1,2E0,2E0,4E0,2.2E1,5.2E1,7E0,7.5E1,3.8E1,7E0,9E0,1.5E1,3E0,1.4E1,1.4E1,2E1,2E0,7E0,1.5E1,1.9E1,3.3E1,1.7E1,5.8E1,3.2E1,6E0,4E0,3E0,6E0,3E0,1.3E1,1E0,2E0,1.8E1,4E0,1.1E1,2.2E1,1.1E1,7E0,1E1,4E1,1.8E1,1.7E1,1.5E1,1E0,5E0,2.1E1,1E0,4E0,7E0,8E0,2E0,3.3E1,7E0,7E0,1E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-2.354039E-3,-2.3067957E-1,3.907276E-1,-4.6439278E-1,-1.4581376E-1,2.7093747E-1,8.92801E-1,-4.8461908E-1,-1.2094458E-2,-1.7238756E-1,2.8708923E-1,3.5039607E-1,-2.7316603E-1,5.706212E-2,9.8354745E-1,-2.4448477E-2,-4.778079E-3,-2.0829849E-1,-2.8006803E-2,3.665053E-1,-3.6646174E-3,2.234144E-1,5.5973613E-1,-5.977377E-2,-4.756461E-2,9.664576E-3,-8.793381E-3,6.1766434E-2,7.623919E-1,-2.2451666E-1,4.4451267E-2,-1.0253909E-1,1.289454E-1,2.2316683E-2,9.852981E-3,1.0819259E-1,2.8618777E-1,3.7053436E-2,4.3133864E-1,5.180612E-2,-1.757534E-2,4.1106544E-2,-4.990757E-3,-2.3061898E-1,2.2585997E-3,-3.0591192E-3,1.09072095E-2,-1.2478681E-1,5.8491156E-3,1.5086938E-3,2.391306E-1,1.4920136E-1,-1.3474122E-2,2.900256E-3,3.189602E-1,4.5609322E-1,4.061606E-3,-7.066511E-2,2.0087942E-2,-2.537372E-1,-1.446657E-1,-1.5621501E-1,3.8563634E-5,1.0988088E-3,1.3901449E-2,4.854119E-2,1.2704159E-2,3.7386715E-1,1.0656553E-2,1.4259201E-2,2.6703238E-2,-8.866142E-3,5.353189E-3,-7.450575E-3,-2.688514E-1,4.618591E-2,-2.1020733E-1,-1.0092341E-2,-3.117481E-3,-4.3383013E-3,8.041336E-3,2.0440955E-2,7.570073E-3,-1.4229931E-2,-7.883766E-3,4.1355332E-3,-6.155274E-3,-6.494236E-4,-1.1371812E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,39,-1,-1,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,57,-1,-1,-1,59,-1,-1,-1,61,-1,-1,63,65,-1,-1,67,69,-1,71,-1,73,75,77,-1,-1,-1,79,-1,81,-1,-1,-1,-1,-1,-1,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8593132E1,5.3272047E0,9.260775E0,2.0658112E-1,2.3617702E0,5.672863E0,2.2190876E0,6.914997E-2,0E0,9.7928715E-1,3.7034935E-1,2.8787518E0,2.4047785E0,1.6108583E-1,5.647106E-1,0E0,0E0,6.330495E-1,4.694442E-1,4.7513485E-2,0E0,4.8980236E-1,6.5997887E-1,4.7420326E-1,0E0,0E0,0E0,0E0,9.635792E-1,2.4577951E-1,1.9699869E-1,1.4650244E-1,1.3424145E-1,0E0,0E0,4.585448E-1,3.2544065E-1,0E0,1.9690847E-1,5.0011355E-1,0E0,0E0,0E0,2.4148607E-1,0E0,0E0,0E0,9.877244E-2,0E0,0E0,4.4736743E-2,2.5190222E-1,0E0,0E0,1.5248537E-1,1.8152428E-1,0E0,1.897934E-1,0E0,1.3233137E-1,4.099878E-1,7.322192E-2,0E0,0E0,0E0,2.2688137E-1,0E0,1.3345933E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.20960236E-1,6.592151E-2,7.910454E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,23,23,28,28,29,29,30,30,31,31,32,32,35,35,36,36,38,38,39,39,43,43,47,47,50,50,51,51,54,54,55,55,57,57,59,59,60,60,61,61,65,65,67,67,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,40,-1,-1,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,58,-1,-1,-1,60,-1,-1,-1,62,-1,-1,64,66,-1,-1,68,70,-1,72,-1,74,76,78,-1,-1,-1,80,-1,82,-1,-1,-1,-1,-1,-1,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.203E3,7.41718E5,6.6512886E3,2.7299898E8,1.4096699E9,3.5301748E7,5.3140095E1,3.6851662E6,-1.2094458E-2,4.3159333E-1,1.5E2,4.1331047E-1,1.6244629E-1,1.0014992E8,1.37893E6,-2.4448477E-2,-4.778079E-3,2.9063344E5,4.819212E10,6.088569E6,-3.6646174E-3,1.3428E4,7.7688866E1,2.5503985E9,-4.756461E-2,9.664576E-3,-8.793381E-3,6.1766434E-2,5.01E2,7.8E2,4.037E3,5.4210108E7,5.792806E4,2.2316683E-2,9.852981E-3,1.9051096E7,3.8952081E0,3.7053436E-2,1.5631589E9,8.01E3,-1.757534E-2,4.1106544E-2,-4.990757E-3,1.1883916E3,2.2585997E-3,-3.0591192E-3,1.09072095E-2,1.8931546E8,5.8491156E-3,1.5086938E-3,1.094E3,3.44E2,-1.3474122E-2,2.900256E-3,1.955799E5,8.4E1,4.061606E-3,9.704588E6,2.0087942E-2,5.1698097E1,4.0757E1,7.164074E8,3.8563634E-5,1.0988088E-3,1.3901449E-2,9.65325E8,1.2704159E-2,2.5111986E-3,1.0656553E-2,1.4259201E-2,2.6703238E-2,-8.866142E-3,5.353189E-3,-7.450575E-3,6.4065235E8,8.8E1,4.041298E-1,-1.0092341E-2,-3.117481E-3,-4.3383013E-3,8.041336E-3,2.0440955E-2,7.570073E-3,-1.4229931E-2,-7.883766E-3,4.1355332E-3,-6.155274E-3,-6.494236E-4,-1.1371812E-2],"split_indices":[2,1,64,7,7,57,68,59,0,51,3,47,50,1,41,0,0,40,43,57,0,2,70,7,0,0,0,0,3,8,10,9,40,0,0,57,65,0,44,0,0,0,0,64,0,0,0,44,0,0,0,8,0,0,45,3,0,9,0,60,68,7,0,0,0,5,0,46,0,0,0,0,0,0,7,8,70,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.71E2,1.57E2,7.1E1,2E2,1.28E2,2.9E1,6.4E1,7E0,1.89E2,1.1E1,1.12E2,1.6E1,3E0,2.6E1,6.3E1,1E0,1.51E2,3.8E1,9E0,2E0,7.1E1,4.1E1,1.3E1,3E0,2E0,1E0,1E1,1.6E1,1.42E2,9E0,2.6E1,1.2E1,5E0,4E0,2.6E1,4.5E1,1.5E1,2.6E1,1E1,3E0,1.5E1,1E0,1.39E2,3E0,6E0,3E0,2.4E1,2E0,7E0,5E0,2.4E1,2E0,6E0,3.9E1,2.4E1,2E0,8E0,2E0,1.08E2,3.1E1,1.9E1,5E0,1E0,4E0,1.3E1,1.1E1,2.4E1,1.5E1,9E0,1.5E1,5E0,3E0,1.5E1,9.3E1,8E0,2.3E1,1.2E1,7E0,6E0,7E0,2E1,4E0,8E1,1.3E1,7E0,1E0,2E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.4103465E-2,-1.7913064E-1,5.0167763E-1,-4.2415485E-1,-1.00428194E-1,5.528132E-1,-2.2450832E-1,-4.4663623E-1,-5.5450725E-4,-1.3513464E-1,2.359129E-1,3.3365214E-1,7.0905435E-1,-5.05738E-1,2.5346333E-1,-4.7088405E-1,-1.2333427E-2,-1.4963098E-1,4.0684015E-1,-2.4871837E-2,2.92124E-1,2.905069E-1,3.360827E-2,5.975827E-1,1.0083172E0,-3.0478288E-3,-2.9110637E-2,1.8522026E-2,4.5414562E-3,-2.3746148E-2,-5.0504724E-3,-1.9482227E-1,-1.4623186E-2,3.1687373E-3,2.8121863E-2,-5.4569016E-3,3.38524E-3,1.8675435E-2,7.995031E-3,-6.147868E-3,3.116194E-1,6.3146263E-1,-8.309849E-3,6.152765E-1,6.1066683E-2,-2.2247672E-1,-4.2675026E-2,-4.5777075E-2,1.02722375E-2,2.2516921E-1,1.7794747E-2,8.878683E-3,6.466699E-1,-4.8660478E-3,3.653214E-2,-2.4695952E-1,-7.664429E-2,1.7438428E-1,-1.1651203E-1,-1.0656836E-2,-1.745129E-2,1.495144E-2,6.7992397E-3,6.5721726E-1,3.771792E-3,-2.918693E-1,-1.6392966E-1,4.7109234E-3,-9.959582E-2,3.4508284E-4,1.1994012E-2,-7.044373E-2,-1.2025656E-2,1.228225E-2,-1.7212126E-1,6.7097795E-1,1.1581204E-2,-5.0471444E-3,-1.5204595E-2,-4.990555E-3,-1.2362542E-2,-6.07733E-3,2.9146126E-3,-8.564569E-3,-1.697673E-3,-2.041634E-3,3.5302208E-3,1.2756065E-3,-1.2808088E-2,3.398795E-2,6.754233E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,49,51,-1,53,-1,55,57,59,-1,61,-1,-1,63,-1,-1,65,67,69,71,-1,73,-1,-1,75,-1,77,79,-1,81,-1,-1,83,-1,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1827534E1,6.089716E0,4.802986E0,7.1007824E-1,2.856795E0,3.8039742E0,1.3379647E0,2.3754406E-1,0E0,1.7893395E0,3.4442258E-1,5.502124E-1,1.6658821E0,1.6929364E-1,4.2513937E-2,5.295849E-2,0E0,1.3089042E0,2.855013E-1,4.6439487E-2,1.4500725E-1,4.48668E-1,0E0,1.4507294E0,6.753998E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.6792107E-1,3.8665605E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.2731E-2,2.301712E-1,0E0,7.308891E-1,0E0,4.666438E-1,4.3702012E-1,2.2870937E-1,0E0,7.800895E-2,0E0,0E0,2.3958015E-1,0E0,0E0,3.8470125E-1,9.17628E-2,7.5261936E-2,9.74057E-2,0E0,1.9986612E-1,0E0,0E0,1.1969185E-1,0E0,1.4397383E-1,2.0517743E-1,0E0,7.312772E-2,0E0,0E0,5.307944E-2,0E0,1.1761209E-1,1.22982755E-1,1.2766266E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,31,31,32,32,40,40,41,41,43,43,45,45,46,46,47,47,49,49,52,52,55,55,56,56,57,57,58,58,60,60,63,63,65,65,66,66,68,68,71,71,73,73,74,74,75,75],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,50,52,-1,54,-1,56,58,60,-1,62,-1,-1,64,-1,-1,66,68,70,72,-1,74,-1,-1,76,-1,78,80,-1,82,-1,-1,84,-1,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0429407E3,1.2852292E7,1.5522031E9,5.4276E4,3.236383E1,2.9778808E7,-5.5450725E-4,3.058642E3,2.0532622E3,7.1853364E3,8.4987135E0,3.087E3,2.9E1,3.84142E5,-1.2333427E-2,9.7467445E4,1.563E3,2.255E3,6.3349745E6,3.0811954E-2,3.360827E-2,2.0531621E4,1.7552137E0,-3.0478288E-3,-2.9110637E-2,1.8522026E-2,4.5414562E-3,-2.3746148E-2,-5.0504724E-3,2.1898598E5,1.4998456E5,3.1687373E-3,2.8121863E-2,-5.4569016E-3,3.38524E-3,1.8675435E-2,7.995031E-3,-6.147868E-3,1.9412996E9,3.0811954E-2,-8.309849E-3,1.379E3,6.1066683E-2,7.164074E8,1.36E2,1.6826648E3,1.02722375E-2,5.05991E5,1.7794747E-2,8.878683E-3,7.2125336E2,-4.8660478E-3,3.653214E-2,2.735322E6,2.0673077E0,2.251E3,1.0088743E7,-1.0656836E-2,1E0,1.495144E-2,6.7992397E-3,7.6807027E0,3.771792E-3,6.453125E0,2.7394568E7,4.7109234E-3,7.87768E8,3.4508284E-4,1.1994012E-2,6.7777224E0,-1.2025656E-2,1.3E2,6.691311E8,6.029086E10,1.1581204E-2,-5.0471444E-3,-1.5204595E-2,-4.990555E-3,-1.2362542E-2,-6.07733E-3,2.9146126E-3,-8.564569E-3,-1.697673E-3,-2.041634E-3,3.5302208E-3,1.2756065E-3,-1.2808088E-2,3.398795E-2,6.754233E-3],"split_indices":[2,1,70,9,7,2,68,12,0,64,4,4,65,0,3,41,0,45,0,0,57,50,0,4,54,0,0,0,0,0,0,40,40,0,0,0,0,0,0,0,12,50,0,0,0,7,3,4,0,41,0,0,70,0,0,1,70,0,57,0,19,0,0,65,0,59,57,0,44,0,0,59,0,8,5,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,3.17E2,1.25E2,7.6E1,2.41E2,1.17E2,8E0,7.2E1,4E0,2.19E2,2.2E1,5E1,6.7E1,5E0,3E0,6.3E1,9E0,2.14E2,5E0,4E0,1.8E1,4.6E1,4E0,5.1E1,1.6E1,1E0,4E0,1E0,2E0,6.2E1,1E0,1.6E2,5.4E1,2E0,3E0,2E0,2E0,1E1,8E0,2E0,4.4E1,4.9E1,2E0,7E0,9E0,1.35E2,2.5E1,4.8E1,6E0,1.7E1,2.7E1,2E0,4.7E1,1E0,6E0,1.15E2,2E1,6E0,1.9E1,6E0,4.2E1,8E0,9E0,4.6E1,1E0,7.3E1,4.2E1,2E0,1.8E1,2E0,4E0,1.5E1,4E0,3.6E1,6E0,4.4E1,2E0,5E0,6.8E1,2.5E1,1.7E1,1.6E1,2E0,3E0,1.2E1,1.9E1,1.7E1,2E0,4E0,4.3E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.3329867E-2,-1.6674933E-1,4.3204945E-1,-2.3522067E-1,4.8191648E-2,5.0440174E-1,-3.3350694E-1,-3.6315867E-1,-1.3965711E-1,-3.516428E-2,2.1631008E-1,3.927016E-1,8.613807E-1,-7.248316E-1,4.955804E-2,-3.8404727E-1,-1.1160061E-1,-2.3415881E-1,-6.970452E-2,-5.857936E-2,1.2724296E-2,2.4812916E-1,-4.98459E-3,2.1842732E-1,4.7097975E-1,-8.3660055E-3,9.056837E-1,-4.3623816E-2,-1.4283402E-2,-6.263455E-2,1.8059198E-2,-4.010596E-1,-7.866838E-3,-8.145454E-3,-3.3088212E-4,-1.3212469E-1,-2.5843182E-1,-1.286527E-1,1.2321885E-1,-3.561487E-2,-1.5107022E-2,1.680368E-2,9.996888E-2,3.3530596E-1,1.02450415E-1,4.9058023E-1,9.982469E-3,5.121193E-1,5.2197088E-2,-7.329156E-3,3.5117492E-3,-4.6025065E-1,-3.1343177E-1,-1.0500927E-2,-2.854301E-3,-1.785379E-1,-3.0994898E-1,-1.702927E-1,-1.0861361E-2,1.1460323E-2,-1.0068486E-3,-8.894478E-3,-9.149907E-3,6.4834408E-3,-4.1952175E-3,1.7659241E-2,2.0649028E-3,-2.2416927E-3,6.8577123E-3,1.1034693E-2,5.1147103E-1,2.9979045E-2,-4.7444543E-3,-2.3456901E-2,-8.351913E-3,-9.650872E-3,-1.725224E-2,-9.606703E-3,2.0374204E-3,-2.1968503E-2,-2.5815198E-1,-1.3319312E-1,-1.3636259E-2,-1.4113423E-1,9.444307E-2,6.1928765E-3,-2.7898012E-2,1.3559164E-2,2.6509313E-2,-1.6290975E-1,-1.7323082E-2,-1.5785855E-1,-5.9872353E-4,-9.142012E-3,-5.248717E-4,7.806926E-4,8.502957E-3,-5.643887E-3,1.7076257E-3,-9.33372E-3,2.485914E-3,-8.573968E-3,-1.4655588E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,-1,47,-1,-1,49,-1,51,-1,-1,-1,53,55,57,59,61,-1,-1,63,65,67,69,-1,71,-1,-1,-1,73,75,-1,-1,77,79,81,83,-1,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,89,91,-1,93,95,-1,97,-1,-1,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.309842E1,4.529546E0,7.4796677E0,2.797989E0,1.0586315E0,4.486561E0,1.8347542E0,4.773569E-1,8.7611437E-1,3.6095148E-1,2.7615857E-1,1.1794615E0,1.4277973E0,1.405425E-1,2.6750398E-1,2.877264E-1,4.7471732E-2,1.07025385E-1,9.142786E-1,2.578698E-1,0E0,2.6662207E-1,0E0,3.858596E-1,2.4225521E-1,0E0,9.3959427E-1,0E0,0E0,7.730324E-2,0E0,2.9143238E-1,0E0,0E0,0E0,6.405541E-2,1.298778E-1,2.9734504E-1,2.936231E-1,1.6791116E-1,0E0,0E0,6.5626465E-2,6.498814E-2,9.322466E-2,2.1958828E-1,0E0,5.3636074E-1,0E0,0E0,0E0,4.9768448E-2,7.481003E-2,0E0,0E0,6.787425E-2,6.1099052E-2,1.3488448E-1,2.4654041E-1,0E0,0E0,0E0,1.0075223E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.02083206E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.204896E-1,1.0275072E-1,0E0,4.175529E-2,5.700825E-2,0E0,1.8858099E-1,0E0,0E0,7.0161134E-2,0E0,4.604596E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,26,26,29,29,31,31,35,35,36,36,37,37,38,38,39,39,42,42,43,43,44,44,45,45,47,47,51,51,52,52,55,55,56,56,57,57,58,58,62,62,70,70,80,80,81,81,83,83,84,84,86,86,89,89,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,-1,48,-1,-1,50,-1,52,-1,-1,-1,54,56,58,60,62,-1,-1,64,66,68,70,-1,72,-1,-1,-1,74,76,-1,-1,78,80,82,84,-1,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,90,92,-1,94,96,-1,98,-1,-1,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.164074E8,1.0429407E3,1.204977E6,1.5522031E9,7.7101436E3,3.5226266E0,1.73E2,7.3123126E2,2.478066E5,7.4433655E-2,2.5096E4,5.0314346E1,3.1E1,2.5212732E4,1.7212875E5,1.9299634E3,7.37365E6,2.103E3,1E0,1.2724296E-2,7.200105E8,-4.98459E-3,1.6087094E5,1.773296E6,-8.3660055E-3,6.274818E4,-4.3623816E-2,-1.4283402E-2,1.6130411E5,1.8059198E-2,1.425966E6,-7.866838E-3,-8.145454E-3,-3.3088212E-4,2.652E3,1E0,5.3008755E8,1.4074289E7,1.384081E3,-1.5107022E-2,1.680368E-2,4.264618E6,1.6197167E4,2.3133965E3,3E0,9.982469E-3,5.37E2,5.2197088E-2,-7.329156E-3,3.5117492E-3,1.5650286E6,2.68269E5,-1.0500927E-2,-2.854301E-3,4.75E2,9.3E4,6.54925E6,1.656968E6,1.1460323E-2,-1.0068486E-3,-8.894478E-3,2.409091E0,6.4834408E-3,-4.1952175E-3,1.7659241E-2,2.0649028E-3,-2.2416927E-3,6.8577123E-3,1.1034693E-2,2.160242E-1,2.9979045E-2,-4.7444543E-3,-2.3456901E-2,-8.351913E-3,-9.650872E-3,-1.725224E-2,-9.606703E-3,2.0374204E-3,-2.1968503E-2,1.902843E-2,1.2472753E5,-1.3636259E-2,2.029E3,2.972948E6,6.1928765E-3,1.8756017E5,1.3559164E-2,2.6509313E-2,4.61E2,-1.7323082E-2,3.061705E6,-5.9872353E-4,-9.142012E-3,-5.248717E-4,7.806926E-4,8.502957E-3,-5.643887E-3,1.7076257E-3,-9.33372E-3,2.485914E-3,-8.573968E-3,-1.4655588E-3],"split_indices":[2,7,70,1,7,64,66,8,64,40,69,2,68,3,4,40,4,57,0,19,0,5,0,45,11,0,40,0,0,40,0,9,0,0,0,10,25,12,57,4,0,0,1,55,4,8,0,3,0,0,0,59,12,0,0,3,12,9,9,0,0,0,70,0,0,0,0,0,0,0,54,0,0,0,0,0,0,0,0,0,69,40,0,0,1,0,59,0,0,3,0,9,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.06E2,1.31E2,2.32E2,7.4E1,1.2E2,1.1E1,9.8E1,1.34E2,5E1,2.4E1,9.3E1,2.7E1,5E0,6E0,9E1,8E0,5.6E1,7.8E1,4.7E1,3E0,2.2E1,2E0,3E1,6.3E1,1E0,2.6E1,3E0,2E0,5E0,1E0,8.3E1,7E0,5E0,3E0,1.2E1,4.4E1,6E1,1.8E1,4.4E1,3E0,1.3E1,9E0,1.4E1,1.6E1,5.8E1,5E0,8E0,1.8E1,3E0,2E0,4.7E1,3.6E1,5E0,7E0,1.9E1,2.5E1,4.4E1,1.6E1,1E1,8E0,6E0,3.8E1,8E0,1E0,1.3E1,1E0,3E0,1.3E1,5E0,5.3E1,7E0,1E0,4.5E1,2E0,9E0,2.7E1,1.8E1,1E0,5E0,2E1,3.4E1,1E1,7E0,9E0,4E0,3.4E1,5E0,4.8E1,1.1E1,9E0,2.8E1,6E0,5E0,2E0,5E0,4E0,1.4E1,2E1,1E1,1E0,2.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-6.6460073E-3,-1.5758432E-1,4.0623355E-1,-2.5586525E-1,9.642101E-2,1.2914006E-1,5.046144E-1,-4.0369043E-1,-1.9301175E-1,-3.891419E-2,3.3136103E-1,1.791976E-1,-2.0714182E-2,4.4297755E-1,4.236941E-2,-4.1853684E-1,1.4768438E-2,-1.7371638E-1,-4.60478E-2,-7.4190535E-2,9.537275E-3,2.5830218E-1,3.9804053E-2,-9.411167E-3,2.1217386E-1,4.5955458E-1,-5.4002153E-3,-4.4349834E-1,-1.33521985E-2,-2.3844753E-1,-1.18670434E-1,-3.2388043E-2,-1.823551E-1,1.15950875E-1,1.7329775E-2,-3.1637207E-3,2.4776347E-1,3.0998316E-1,5.4573274E-1,-2.2388462E-2,-4.9180016E-3,-1.4762458E-1,-2.799195E-1,-1.5843473E-1,5.9062764E-2,-5.225001E-2,5.454937E-3,-5.486195E-4,-2.2809644E-1,7.4985437E-3,8.664679E-5,6.4550093E-3,2.7841857E-1,5.818711E-3,3.362945E-1,5.6420326E-1,1.0762791E-2,-2.0894608E-1,-7.494735E-2,-3.347672E-1,-1.9084792E-1,-4.9186133E-2,-1.9873257E-1,6.2205414E-3,-4.0778597E-3,-9.443754E-2,7.4833673E-3,-1.5939359E-2,-6.5280357E-3,3.353264E-1,1.779292E-1,1.7393623E-2,1.3600695E-3,5.1582444E-1,4.472091E-2,-1.2316352E-2,-3.1466186E-3,-7.008333E-3,-8.3030964E-4,-3.5992387E-1,-7.098125E-3,-9.257362E-2,-1.6940491E-2,4.869057E-2,-4.1388073E-3,-3.0724942E-3,-2.1174897E-1,-9.030734E-3,-1.703577E-3,2.230602E-3,-3.7853566E-3,6.95425E-3,1.956554E-2,1.0056348E-2,-2.1047056E-4,1.1845644E-2,2.6610944E-2,-8.612565E-3,-1.9363182E-2,-6.1325957E-3,2.9402035E-3,-3.9653103E-3,4.506161E-3,-1.123102E-2,-4.1701035E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,-1,29,-1,31,-1,33,-1,-1,35,37,-1,39,-1,41,43,45,47,49,-1,-1,51,53,55,-1,-1,57,59,61,63,65,-1,-1,67,-1,-1,-1,69,-1,71,73,-1,75,77,79,81,83,85,-1,-1,87,89,-1,-1,91,93,-1,-1,95,-1,-1,-1,-1,-1,97,-1,99,-1,101,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7235247E1,8.013113E0,3.1243382E0,2.0683813E0,2.8745036E0,9.4507E-1,1.4435616E0,8.2956505E-1,2.144867E0,4.8423514E-1,9.156034E-1,4.0352947E-1,0E0,7.34745E-1,0E0,1.3465595E-1,0E0,5.453768E-1,0E0,2.2469285E-1,0E0,3.3756113E-1,0E0,0E0,2.9020154E-1,7.7178764E-1,0E0,4.259205E-2,0E0,2.3363924E-1,6.383594E-1,1.12530366E-1,1.07242525E-1,5.0360084E-2,0E0,0E0,5.4932356E-2,1.0853195E-1,1.5309143E-1,0E0,0E0,9.575701E-2,1.7549372E-1,3.116064E-1,1.6655913E-1,8.638684E-2,0E0,0E0,5.5741787E-2,0E0,0E0,0E0,4.8989654E-2,0E0,8.127713E-2,2.1025372E-1,0E0,5.564797E-2,4.7178887E-2,8.94897E-2,2.7431262E-1,7.3155046E-2,8.1668615E-2,0E0,0E0,9.7959876E-2,4.975881E-2,0E0,0E0,6.673658E-2,3.903249E-2,0E0,0E0,5.4818153E-2,0E0,0E0,0E0,0E0,0E0,6.2315226E-2,0E0,7.1100146E-2,0E0,3.9016206E-2,0E0,0E0,6.1182022E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,17,17,19,19,21,21,24,24,25,25,27,27,29,29,30,30,31,31,32,32,33,33,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,48,48,52,52,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,69,69,70,70,73,73,79,79,81,81,83,83,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,-1,30,-1,32,-1,34,-1,-1,36,38,-1,40,-1,42,44,46,48,50,-1,-1,52,54,56,-1,-1,58,60,62,64,66,-1,-1,68,-1,-1,-1,70,-1,72,74,-1,76,78,80,82,84,86,-1,-1,88,90,-1,-1,92,94,-1,-1,96,-1,-1,-1,-1,-1,98,-1,100,-1,102,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,9.7467445E4,1.00493375E5,7.41718E5,2.081964E3,8.494E3,1.1275989E1,8.588E3,5.825042E6,1.5271514E5,8.596084E3,3.0972284E1,-2.0714182E-2,5.22E2,4.236941E-2,2.7299898E8,1.4768438E-2,6.505339E2,-4.60478E-2,1.716E3,9.537275E-3,2.0663544E9,3.9804053E-2,-9.411167E-3,1.484798E9,5.6503E4,-5.4002153E-3,8.95654E8,-1.33521985E-2,7.2123305E6,2.103E3,1.207E3,1.0843525E5,1.556838E0,1.7329775E-2,-3.1637207E-3,1.7532333E9,1.6772E4,3.0113987E10,-2.2388462E-2,-4.9180016E-3,1.523794E6,2.72E2,9.174246E6,2.7394568E7,7.10711E8,5.454937E-3,-5.486195E-4,1.1762006E9,7.4985437E-3,8.664679E-5,6.4550093E-3,1.59E2,5.818711E-3,1.1367206E8,3.63796E5,1.0762791E-2,2.96E2,5.8955963E1,3.312442E6,9.3E1,5.6E1,8.824219E0,6.2205414E-3,-4.0778597E-3,8.7998315E2,5.192606E6,-1.5939359E-2,-6.5280357E-3,2.4339888E3,1.31E3,1.7393623E-2,1.3600695E-3,1.5200746E-1,4.472091E-2,-1.2316352E-2,-3.1466186E-3,-7.008333E-3,-8.3030964E-4,5.381665E1,-7.098125E-3,4.75E2,-1.6940491E-2,3.84E2,-4.1388073E-3,-3.0724942E-3,2.3176976E7,-9.030734E-3,-1.703577E-3,2.230602E-3,-3.7853566E-3,6.95425E-3,1.956554E-2,1.0056348E-2,-2.1047056E-4,1.1845644E-2,2.6610944E-2,-8.612565E-3,-1.9363182E-2,-6.1325957E-3,2.9402035E-3,-3.9653103E-3,4.506161E-3,-1.123102E-2,-4.1701035E-3],"split_indices":[7,45,45,1,64,0,47,2,59,40,4,68,0,3,0,7,0,64,0,0,0,12,0,0,7,2,0,5,0,57,0,0,45,53,0,0,7,9,12,0,0,1,3,57,57,7,0,0,12,0,0,0,3,0,1,2,0,8,62,9,8,3,59,0,0,64,62,0,0,4,0,0,0,53,0,0,0,0,0,60,0,3,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,3.19E2,1.16E2,2.3E2,8.9E1,3.1E1,8.5E1,6.7E1,1.63E2,5.7E1,3.2E1,2.9E1,2E0,7.4E1,1.1E1,6.6E1,1E0,1.6E2,3E0,5E1,7E0,2.9E1,3E0,2E0,2.7E1,7.2E1,2E0,5.5E1,1.1E1,7.2E1,8.8E1,3.7E1,1.3E1,1.2E1,1.7E1,3E0,2.4E1,2.8E1,4.4E1,5.4E1,1E0,2.4E1,4.8E1,7.2E1,1.6E1,3.3E1,4E0,3E0,1E1,9E0,3E0,6E0,1.8E1,4E0,2.4E1,4.1E1,3E0,1.2E1,1.2E1,2.8E1,2E1,2E1,5.2E1,1.1E1,5E0,1.9E1,1.4E1,4E0,6E0,1E1,8E0,2.3E1,1E0,3.8E1,3E0,9E0,3E0,5E0,7E0,2.4E1,4E0,1.3E1,7E0,5E0,1.5E1,5E0,4.7E1,7E0,1.2E1,1E1,4E0,3E0,7E0,7E0,1E0,3E0,3.5E1,4E0,2E1,1.1E1,2E0,1E0,4E0,4.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.2736447E-3,-1.6982651E-1,3.8341367E-1,-2.4999872E-1,5.2451666E-2,7.9158686E-2,4.6317568E-1,-2.3756562E-1,-4.432183E-2,-4.259399E-2,2.5116095E-1,2.6578328E-1,-1.0030152E-1,3.7390417E-1,7.377287E-1,-3.777308E-1,-1.7950164E-1,-8.4150046E-2,9.41491E-3,1.5766539E-1,1.863215E-2,6.3292584E-3,1.5431302E-2,-1.9954007E-2,7.7134767E-3,3.9701095E-1,-7.9262275E-3,5.6576836E-1,4.8050314E-2,-1.9852467E-2,-1.1968977E-1,-1.8862449E-1,3.198856E-2,-2.0385182E-1,-3.526186E-2,9.027301E-3,1.892313E-3,-2.9471847E-3,7.7876337E-3,2.8026792E-1,5.124896E-1,2.936646E-2,5.95064E-3,2.7580117E-3,-8.2811555E-3,-2.402078E-1,-1.2543026E-1,-2.7877465E-1,-5.6379447E-3,-6.458512E-2,5.9884038E-2,1.678117E-1,3.2819825E-1,4.841647E-3,5.3347075E-1,-1.3233073E-1,-2.7690846E-1,-1.6364545E-1,6.647529E-2,-1.6228428E-2,-7.694407E-5,-1.0366019E-1,5.2272785E-3,-6.1576103E-4,7.6573733E-3,1.0369907E-2,2.742588E-3,6.8039894E-3,3.465322E-1,7.0908684E-1,2.225975E-2,-9.676703E-2,-1.2400035E-2,-8.1062075E-3,-2.976551E-1,-6.869177E-2,-2.0092897E-1,-3.6976798E-3,5.797899E-3,-6.796912E-3,3.8144336E-4,-1.3955958E-3,5.144663E-3,2.7737576E-3,1.7830098E-2,9.661771E-3,3.722781E-2,-7.429623E-3,-2.487845E-3,-2.073147E-2,-1.2454204E-2,3.0667293E-3,-5.9733097E-3,-1.0729715E-2,-4.7849392E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,47,49,-1,-1,-1,-1,51,53,-1,-1,-1,-1,55,57,59,-1,61,63,65,67,-1,69,71,73,75,77,-1,-1,79,81,-1,-1,-1,-1,-1,83,85,-1,87,-1,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7881329E1,5.5374775E0,3.0939064E0,1.594656E0,1.5782737E0,9.6442586E-1,2.2241573E0,1.7633686E0,0E0,5.629796E-1,2.4833608E-1,4.4637322E-2,4.8747778E-1,1.0350189E0,3.7940884E-1,2.79665E-1,1.3245153E0,2.7955994E-1,0E0,3.993553E-2,0E0,0E0,0E0,0E0,1.2771367E-1,8.809452E-1,0E0,8.614159E-2,0E0,0E0,5.7423674E-2,4.9143314E-1,0E0,6.4043224E-2,1.04307175E-1,0E0,0E0,0E0,0E0,1.6052699E-1,2.709589E-1,0E0,0E0,0E0,0E0,3.0780363E-1,5.5357265E-1,8.867121E-2,0E0,7.692374E-2,6.254238E-2,5.135885E-2,4.78909E-2,0E0,2.2544956E-1,7.481918E-2,1.0402584E-1,2.0569074E-1,9.904221E-2,0E0,0E0,6.582996E-2,3.8509298E-2,0E0,0E0,0E0,0E0,0E0,5.172515E-2,4.0761948E-2,0E0,4.1367948E-2,0E0,0E0,1.9751692E-1,1.3294229E-1,1.1386573E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,24,24,25,25,27,27,30,30,31,31,33,33,34,34,39,39,40,40,45,45,46,46,47,47,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,61,61,62,62,68,68,69,69,71,71,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,48,50,-1,-1,-1,-1,52,54,-1,-1,-1,-1,56,58,60,-1,62,64,66,68,-1,70,72,74,76,78,-1,-1,80,82,-1,-1,-1,-1,-1,84,86,-1,88,-1,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.7467445E4,6.028226E4,5.825042E6,2.1695537E3,8.169258E3,4.026531E9,7.41718E5,-4.432183E-2,1.5271514E5,9.825568E6,3.149493E3,2.412E4,3.768034E4,7.443074E0,3.5002026E8,9.256843E3,8.689165E-2,9.41491E-3,1.2678202E9,1.863215E-2,6.3292584E-3,1.5431302E-2,-1.9954007E-2,3.087E3,6.4945E4,-7.9262275E-3,6.029086E10,4.8050314E-2,-1.9852467E-2,2.3E2,7.8936383E2,3.198856E-2,6.426437E5,3.53E2,9.027301E-3,1.892313E-3,-2.9471847E-3,7.7876337E-3,7.996302E-1,2.4126951E-1,2.936646E-2,5.95064E-3,2.7580117E-3,-8.2811555E-3,7.2123305E6,2.103E3,3.672717E3,-5.6379447E-3,2.681357E6,2.1530056E3,1.6255458E5,5E1,4.841647E-3,5.339615E-1,1.376072E8,5.381665E1,1.423795E7,3.7730695E-7,-1.6228428E-2,-7.694407E-5,6.2581446E8,6.212394E4,-6.1576103E-4,7.6573733E-3,1.0369907E-2,2.742588E-3,6.8039894E-3,1.01319025E3,9.904738E6,2.225975E-2,5.92E2,-1.2400035E-2,-8.1062075E-3,4.94E4,2.1692945E1,6.4065235E8,-3.6976798E-3,5.797899E-3,-6.796912E-3,3.8144336E-4,-1.3955958E-3,5.144663E-3,2.7737576E-3,1.7830098E-2,9.661771E-3,3.722781E-2,-7.429623E-3,-2.487845E-3,-2.073147E-2,-1.2454204E-2,3.0667293E-3,-5.9733097E-3,-1.0729715E-2,-4.7849392E-4],"split_indices":[7,45,45,59,64,45,7,1,0,40,1,4,9,4,65,7,4,50,0,7,0,0,0,0,0,2,0,5,0,0,0,64,0,59,8,0,0,0,0,54,53,0,0,0,0,57,0,4,0,1,4,45,3,0,46,12,60,57,50,0,0,7,40,0,0,0,0,0,4,1,0,10,0,0,9,65,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.09E2,1.28E2,2.27E2,8.2E1,2.7E1,1.01E2,2.24E2,3E0,5.6E1,2.6E1,1.3E1,1.4E1,7.8E1,2.3E1,6.4E1,1.6E2,4.8E1,8E0,1.6E1,1E1,4E0,9E0,3E0,1.1E1,7.5E1,3E0,1.5E1,8E0,5.9E1,5E0,1.59E2,1E0,1.3E1,3.5E1,1.3E1,3E0,8E0,3E0,3.9E1,3.6E1,1.4E1,1E0,1E0,4E0,8.6E1,7.3E1,6E0,7E0,2.7E1,8E0,1.3E1,2.6E1,2E0,3.4E1,2.3E1,6.3E1,6.1E1,1.2E1,5E0,1E0,1.7E1,1E1,5E0,3E0,9E0,4E0,3E0,2.3E1,9E0,2.5E1,1.9E1,4E0,1.1E1,5.2E1,1.8E1,4.3E1,3E0,9E0,1.3E1,4E0,8E0,2E0,1E0,2.2E1,1E0,8E0,8E0,1.1E1,1.3E1,3.9E1,5E0,1.3E1,4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[7.4815685E-3,-1.7263705E-1,3.3803168E-1,-3.5948348E-1,-1.0460054E-1,3.77988E-1,-5.7788205E-1,-3.8422588E-1,-1.7609951E-1,-1.2210213E-1,1.1961301E-1,2.9822886E-1,7.530429E-1,-3.9222434E-2,-1.1318062E-2,-3.9243245E-1,-7.839435E-3,-1.1320916E-2,-1.5602678E-3,-1.443582E-1,4.6196647E-2,2.5564477E-1,-4.8374355E-2,1.947771E-1,4.344733E-1,-7.3102503E-3,8.340808E-1,-1.9808382E-2,-3.6275962E-3,-1.5324715E-1,9.510105E-3,-1.07010804E-1,9.2146546E-2,6.691333E-4,1.6139673E-2,2.5584572E-4,-6.95852E-3,2.2025315E-1,-4.2931974E-2,2.8291345E-1,5.1530516E-1,5.475735E-2,6.040819E-1,-1.7119913E-1,-4.5855258E-2,-8.251245E-3,3.0569904E-4,9.584191E-3,3.141196E-2,1.6033211E-1,2.8078142E-1,-7.7728657E-3,5.424938E-3,-6.874888E-5,3.1291103E-1,2.6609106E-2,8.036138E-3,3.4810822E-2,1.0133701E-4,-2.2007543E-1,-1.21343546E-1,-1.5836855E-3,-6.4620757E-3,-6.479751E-4,6.7219394E-3,1.061145E-1,1.2990009E-2,3.4018382E-3,3.004389E-1,3.4222263E-1,2.1243975E-3,-1.9926394E-3,-2.3273934E-1,-1.66619E-1,-4.3781817E-2,4.8799803E-3,-2.7939439E-2,1.7360361E-1,4.0687077E-2,1.4515755E-3,1.5447318E-2,1.7816847E-2,2.8402624E-3,-1.1971159E-2,-2.7959378E-4,-6.3724034E-3,-1.2071872E-2,-5.748621E-3,4.160958E-3,-3.833612E-3,1.2155423E-3,1.0632084E-2,-7.3616445E-4,-6.4967843E-3,3.9964737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,35,37,39,-1,41,-1,-1,43,-1,45,47,-1,-1,-1,-1,49,51,53,55,-1,57,59,61,-1,-1,-1,63,65,67,-1,-1,-1,69,-1,-1,-1,-1,71,73,75,-1,-1,-1,77,-1,-1,79,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6671944E1,3.6489878E0,6.0006905E0,2.8171444E-1,8.5436106E-1,4.322357E0,3.3131528E-1,6.345558E-2,7.329211E-2,7.520573E-1,3.779925E-1,1.7054043E0,2.0168562E0,0E0,0E0,5.156994E-2,0E0,0E0,0E0,5.527084E-1,1.7881748E-1,1.4241725E-1,3.959745E-2,4.575913E-1,5.157604E-1,0E0,7.8209686E-1,0E0,0E0,3.237481E-1,0E0,4.044155E-2,1.0894355E-1,0E0,0E0,0E0,0E0,1.9798541E-1,1.5333833E-1,1.7951798E-1,1.20874405E-1,0E0,8.336463E-1,3.3149862E-1,9.584908E-2,0E0,0E0,0E0,6.121821E-2,1.6680604E-1,1.1335349E-1,0E0,0E0,0E0,1.3604558E-1,0E0,0E0,0E0,0E0,1.5731215E-1,2.5607836E-1,4.9803246E-2,0E0,0E0,0E0,1.0332754E-1,0E0,0E0,5.9143066E-2,4.9043655E-2,0E0,0E0,1.0366702E-1,1.06940985E-1,2.7205902E-1,0E0,4.0097557E-2,9.159008E-2,1.04260236E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,31,31,32,32,37,37,38,38,39,39,40,40,42,42,43,43,44,44,48,48,49,49,50,50,54,54,59,59,60,60,61,61,65,65,68,68,69,69,72,72,73,73,74,74,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,36,38,40,-1,42,-1,-1,44,-1,46,48,-1,-1,-1,-1,50,52,54,56,-1,58,60,62,-1,-1,-1,64,66,68,-1,-1,-1,70,-1,-1,-1,-1,72,74,76,-1,-1,-1,78,-1,-1,80,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,2.3028242E3,3.5002026E8,1.1576422E9,7.3068325E3,3.1E1,2.9778808E7,2.033552E6,5.9170656E8,1.24E2,4.1331047E-1,5.0314346E1,-3.9222434E-2,-1.1318062E-2,3.84142E5,-7.839435E-3,-1.1320916E-2,-1.5602678E-3,3.346087E5,8.7998315E2,1.8505337E3,2.08E2,3.271774E7,1E0,-7.3102503E-3,1.417988E6,-1.9808382E-2,-3.6275962E-3,7.164074E8,9.510105E-3,1.6776951E3,2.299087E3,6.691333E-4,1.6139673E-2,2.5584572E-4,-6.95852E-3,2.0663544E9,1.2538845E9,2.1853803E4,5.848343E7,5.475735E-2,4.42384E5,1.947899E6,8.9846605E8,-8.251245E-3,3.0569904E-4,9.584191E-3,3.274075E-1,2.1597655E9,3.8952081E0,-7.7728657E-3,5.424938E-3,-6.874888E-5,1.5631589E9,2.6609106E-2,8.036138E-3,3.4810822E-2,1.0133701E-4,1.8460208E8,1.0623282E3,1.2504131E1,-6.4620757E-3,-6.479751E-4,6.7219394E-3,1.2254369E7,1.2990009E-2,3.4018382E-3,4.264618E6,9.677921E7,2.1243975E-3,-1.9926394E-3,2.9063344E5,8.593763E7,8.26042E4,4.8799803E-3,1.4555919E-1,2.0569E4,1.2009271E3,1.4515755E-3,1.5447318E-2,1.7816847E-2,2.8402624E-3,-1.1971159E-2,-2.7959378E-4,-6.3724034E-3,-1.2071872E-2,-5.748621E-3,4.160958E-3,-3.833612E-3,1.2155423E-3,1.0632084E-2,-7.3616445E-4,-6.4967843E-3,3.9964737E-3],"split_indices":[2,1,70,7,7,64,3,12,9,12,3,47,68,0,0,41,0,0,0,40,64,4,3,57,102,0,41,0,0,7,0,4,4,0,0,0,0,12,7,45,1,0,2,1,7,0,0,0,50,7,65,0,0,0,44,0,0,0,0,5,64,70,0,0,0,9,0,0,1,1,0,0,40,7,40,0,50,10,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,2.89E2,1.57E2,7.6E1,2.13E2,1.51E2,6E0,6.6E1,1E1,1.98E2,1.5E1,1.26E2,2.5E1,3E0,3E0,6.3E1,3E0,7E0,3E0,1.75E2,2.3E1,8E0,7E0,7.3E1,5.3E1,2E0,2.3E1,6.2E1,1E0,1.71E2,4E0,5E0,1.8E1,2E0,6E0,5E0,2E0,6.6E1,7E0,2E1,3.3E1,9E0,1.4E1,1.46E2,2.5E1,3E0,2E0,6E0,1.2E1,3.5E1,3.1E1,4E0,3E0,2E0,1.8E1,3.1E1,2E0,1.2E1,2E0,7.2E1,7.4E1,1.7E1,8E0,9E0,3E0,2.4E1,1.1E1,3E0,2.8E1,1.6E1,2E0,5E0,6.7E1,4.6E1,2.8E1,3E0,1.4E1,1.1E1,1.3E1,1E0,2.7E1,1.5E1,1E0,6.5E1,2E0,3.2E1,1.4E1,1.8E1,1E1,7E0,7E0,9E0,2E0,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-1.754247E-2,-1.6004801E-1,2.8222972E-1,-3.3112457E-1,-1.02329634E-1,-4.8885766E-1,3.304729E-1,-3.444614E-1,-8.054545E-5,-1.4688173E-1,3.0040538E-2,1.7713524E-2,-6.385749E-1,2.2134902E-1,4.6459076E-1,-1.806396E-2,-1.0709468E-2,-2.3818153E-1,-8.4017605E-2,-7.640849E-3,2.1544342E-1,-4.0565256E-2,-1.3120374E-2,1.1967729E-4,2.3007527E-1,-4.569553E-2,5.10968E-1,-1.2915762E-1,-2.7920794E-1,-3.0383179E-2,-1.6907312E-1,9.016252E-3,-1.357388E-2,-3.3601238E-3,1.2710049E-2,2.4681231E-1,9.660319E-2,-6.633865E-3,4.2777867E-3,4.466933E-1,3.8079903E-2,-2.3045359E-4,-1.5819138E-1,-7.373073E-3,-3.084386E-1,3.5484686E-2,-1.0841185E-1,-1.8635686E-1,-2.0026613E-3,1.0175195E-2,-8.480554E-3,4.438103E-3,2.5292692E-1,8.693154E-3,-7.7608577E-4,5.065033E-1,1.7474592E-1,-2.5452941E-3,-1.05983E-2,-3.3202383E-1,-7.4873776E-3,-4.040507E-2,7.877535E-3,-1.345567E-1,9.292383E-4,-4.4534383E-3,-2.0748568E-1,-8.699064E-2,5.780714E-2,3.1351006E-1,2.2172675E-1,5.911461E-1,1.8971637E-2,-7.957809E-3,1.3900575E-2,-1.9531962E-3,-3.397165E-1,-9.066579E-3,-5.8458456E-5,-3.6012467E-3,-2.0754786E-1,-7.1972203E-3,-1.2638661E-2,-7.6336204E-3,-2.4945727E-2,-1.9616764E-3,7.816381E-2,3.3376533E-1,4.7090854E-3,3.976867E-3,2.336543E-1,3.0415181E-2,6.0514547E-3,-1.7951231E-2,-7.935686E-3,-1.1895051E-2,-2.347086E-3,-2.733295E-3,4.1242247E-3,5.297672E-3,-4.5500082E-4,1.7379833E-2,2.743761E-3,8.867864E-3,1.39303375E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,-1,27,29,31,33,-1,-1,-1,35,37,39,41,43,45,47,49,-1,-1,-1,51,53,-1,-1,55,-1,-1,57,-1,59,61,63,65,-1,-1,67,-1,69,-1,-1,71,73,-1,-1,75,-1,77,-1,79,-1,-1,81,83,85,87,89,91,-1,-1,-1,-1,93,-1,-1,-1,95,-1,-1,-1,97,-1,99,101,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9354248E1,2.9975142E0,5.5927105E0,3.3783436E-1,1.3667042E0,1.3626423E0,1.9168558E0,8.6206436E-2,0E0,9.729192E-1,4.138342E-1,0E0,3.0770206E-1,1.484797E-1,1.4669561E0,0E0,0E0,2.770381E-1,4.69301E-1,2.2229485E-1,1.2700221E-1,0E0,0E0,0E0,1.4370179E-1,7.993237E-2,5.630617E-1,7.520756E-2,1.4694381E-1,3.379198E-1,8.1716776E-2,1.6514826E-1,0E0,0E0,0E0,4.1305065E-2,8.9427836E-2,0E0,0E0,6.7595387E-1,0E0,0E0,8.743903E-2,0E0,9.989953E-2,3.4006307E-1,1.02118045E-1,5.3424716E-2,0E0,0E0,2.3922004E-1,0E0,4.956913E-2,0E0,0E0,1.8372154E-1,3.896185E-1,0E0,0E0,6.3342094E-2,0E0,1.2704E-1,0E0,9.908804E-2,0E0,0E0,4.4281125E-2,8.1864476E-2,5.24633E-2,5.294144E-2,5.7705164E-2,9.8546505E-2,0E0,0E0,0E0,0E0,5.9269667E-2,0E0,0E0,0E0,4.2639375E-2,0E0,0E0,0E0,4.2875357E-2,0E0,5.1986173E-2,4.7265768E-2,0E0,0E0,5.4448843E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,35,35,36,36,39,39,42,42,44,44,45,45,46,46,47,47,50,50,52,52,55,55,56,56,59,59,61,61,63,63,66,66,67,67,68,68,69,69,70,70,71,71,76,76,80,80,84,84,86,86,87,87,90,90],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,-1,28,30,32,34,-1,-1,-1,36,38,40,42,44,46,48,50,-1,-1,-1,52,54,-1,-1,56,-1,-1,58,-1,60,62,64,66,-1,-1,68,-1,70,-1,-1,72,74,-1,-1,76,-1,78,-1,80,-1,-1,82,84,86,88,90,92,-1,-1,-1,-1,94,-1,-1,-1,96,-1,-1,-1,98,-1,100,102,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.700398E4,1.2852292E7,7.10711E8,1.3812E4,6.8963E4,1.7367111E6,-8.054545E-5,7.283496E2,3.4491902E3,1.7713524E-2,3.1E1,1.4E1,2.943086E4,-1.806396E-2,-1.0709468E-2,7.37365E6,2.474199E1,1E0,8.287E3,-4.0565256E-2,-1.3120374E-2,1.1967729E-4,2.5916522E7,1.6130411E5,1.3861362E8,8.840007E7,5.381665E1,1.6777562E7,1.4205709E0,2.829932E0,-1.357388E-2,-3.3601238E-3,1.2710049E-2,4.841781E-2,5.5580516E4,-6.633865E-3,4.2777867E-3,4.42384E5,3.8079903E-2,-2.3045359E-4,1.5112E4,-7.373073E-3,2.0964778E5,2.327793E9,6.0070217E1,3.26021E5,-2.0026613E-3,1.0175195E-2,1.9252769E5,4.438103E-3,1.7045455E-2,8.693154E-3,-7.7608577E-4,5.3699964E7,3.087E3,-2.5452941E-3,-1.05983E-2,5.39E2,-7.4873776E-3,1.6845247E3,7.877535E-3,1.656968E6,9.292383E-4,-4.4534383E-3,2.9480488E1,2.8688732E7,5.898222E2,1.5953E4,8.879629E4,1.5953E4,1.8971637E-2,-7.957809E-3,1.3900575E-2,-1.9531962E-3,7.177E3,-9.066579E-3,-5.8458456E-5,-3.6012467E-3,1.16985195E5,-7.1972203E-3,-1.2638661E-2,-7.6336204E-3,1.5522031E9,-1.9616764E-3,6.843157E-2,6.7304E4,4.7090854E-3,3.976867E-3,1.3210104E9,3.0415181E-2,6.0514547E-3,-1.7951231E-2,-7.935686E-3,-1.1895051E-2,-2.347086E-3,-2.733295E-3,4.1242247E-3,5.297672E-3,-4.5500082E-4,1.7379833E-2,2.743761E-3,8.867864E-3,1.39303375E-2],"split_indices":[2,1,45,9,7,9,2,63,0,64,4,0,3,3,45,0,0,57,65,19,2,0,0,0,57,40,56,5,60,57,51,70,0,0,0,51,40,0,0,2,0,0,9,0,40,5,68,41,0,0,59,0,69,0,0,56,0,0,0,0,0,4,0,9,0,0,65,9,64,0,45,0,0,0,0,0,2,0,0,0,40,0,0,0,7,0,69,2,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.51E2,3.06E2,1.45E2,7.6E1,2.3E2,8E0,1.37E2,7.3E1,3E0,1.72E2,5.8E1,1E0,7E0,7.7E1,6E1,6.3E1,1E1,6.9E1,1.03E2,4.9E1,9E0,4E0,3E0,3E0,7.4E1,5E0,5.5E1,2E1,4.9E1,6.4E1,3.9E1,4.7E1,2E0,1E0,8E0,6.5E1,9E0,3E0,2E0,4.6E1,9E0,4E0,1.6E1,1E1,3.9E1,3.5E1,2.9E1,3.4E1,5E0,3E0,4.4E1,3E0,6.2E1,5E0,4E0,3.7E1,9E0,6E0,1E1,3.3E1,6E0,2.2E1,1.3E1,2.4E1,5E0,7E0,2.7E1,2E1,2.4E1,1.8E1,4.4E1,2E1,1.7E1,2E0,7E0,1E0,3.2E1,4E0,1.8E1,1.4E1,1E1,1.3E1,1.4E1,9E0,1.1E1,4E0,2E1,1.6E1,2E0,4E0,4E1,1.9E1,1E0,2.8E1,4E0,8E0,2E0,9E0,2E0,1.5E1,5E0,1.5E1,1E0,2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[6.4767827E-3,-1.29907E-1,3.4626526E-1,-3.3372983E-1,-7.41637E-2,4.1594756E-1,-3.9470613E-1,-3.4834534E-1,-1.187278E-3,-1.03079624E-1,1.5776859E-1,3.2167333E-1,7.869933E-1,-6.677378E-1,6.647938E-2,-2.24362E-2,-1.5617291E-2,-1.152558E-1,4.289358E-1,1.442738E-2,7.6271966E-2,2.0965776E-1,4.0933296E-1,4.892368E-2,2.292708E-2,-3.8943425E-2,-1.2959774E-2,2.607892E-1,-7.495924E-3,-1.4931358E-1,2.7999345E-3,4.933318E-3,2.4329005E-2,1.3995814E-1,-4.380032E-2,2.4859093E-1,1.308544E-1,5.0667703E-1,3.2754767E-1,2.278936E-3,1.7270897E-2,-1.6080798E-1,-2.5181806E-2,-8.115722E-2,6.0604874E-2,8.441808E-3,-1.6112167E-4,3.1464421E-3,-4.952421E-3,8.9546805E-4,2.6632807E-1,4.0654964E-3,1.1169499E-2,5.924501E-3,5.4129356E-1,3.4719107E-1,1.7505038E-3,-1.7070049E-1,1.2042357E-2,-4.946436E-3,2.5867706E-3,-1.2088463E-1,3.1086083E-3,-1.718546E-2,1.451792E-1,1.4649181E-2,6.627871E-3,2.794906E-2,5.4499237E-3,5.897614E-3,1.8005468E-2,-1.9438305E-1,-1.1756515E-1,-2.8376507E-3,3.9209444E-3,-1.7186281E-1,-2.4200217E-3,-3.271167E-3,1.9623388E-3,9.656817E-3,-1.1620384E-3,-1.015751E-2,3.7971328E-4,1.7207018E-3,-7.0578856E-3,-9.858656E-3,7.839862E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,33,35,37,-1,-1,-1,-1,39,-1,41,43,-1,-1,45,47,49,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,-1,-1,65,-1,-1,-1,67,69,-1,71,73,-1,-1,75,-1,77,79,-1,-1,-1,-1,-1,-1,81,83,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9740177E1,3.429224E0,6.463355E0,2.86201E-1,1.6258318E0,3.66461E0,1.4294915E0,8.329773E-2,0E0,1.4357109E0,2.6605248E-1,8.077316E-1,9.3698025E-1,1.167078E-1,2.4942662E-1,0E0,0E0,8.4476876E-1,4.750651E-2,0E0,1.437752E-1,9.632945E-2,2.3210526E-1,0E0,0E0,0E0,0E0,3.8981393E-2,0E0,2.2852206E-1,2.3786917E-1,0E0,0E0,5.030121E-2,4.7562126E-2,1.0569787E-1,5.519989E-2,2.1798134E-1,1.6022205E-1,0E0,0E0,2.568345E-1,9.029114E-2,1.21425E-1,1.9328783E-1,0E0,0E0,0E0,0E0,0E0,5.069542E-2,0E0,0E0,0E0,8.647251E-2,4.1487455E-2,0E0,1.5457869E-1,4.5591813E-2,0E0,0E0,5.0780684E-2,0E0,4.6167627E-2,1.1779547E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7342615E-1,1.7028922E-1,0E0,0E0,4.59764E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,20,20,21,21,22,22,27,27,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,50,50,54,54,55,55,57,57,58,58,61,61,63,63,64,64,71,71,72,72,75,75],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,34,36,38,-1,-1,-1,-1,40,-1,42,44,-1,-1,46,48,50,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,-1,-1,66,-1,-1,-1,68,70,-1,72,74,-1,-1,76,-1,78,80,-1,-1,-1,-1,-1,-1,82,84,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0429407E3,1.2852292E7,1.4096699E9,7.7101436E3,3.5226266E0,1.44E2,-1.187278E-3,3.058642E3,3.907527E8,4.1331047E-1,5.192243E6,3.1E1,1.872503E4,-2.24362E-2,-1.5617291E-2,2.8884522E8,1.563E3,1.442738E-2,5.78E2,1.921711E5,9.170229E6,4.892368E-2,2.292708E-2,-3.8943425E-2,-1.2959774E-2,3.498E3,-7.495924E-3,3.94E2,1.656968E6,4.933318E-3,2.4329005E-2,1.24E2,2.1933604E3,3.7860696E0,1.55592E5,3.0811954E-2,1.7092716E9,2.278936E-3,1.7270897E-2,2.889654E5,2.6452028E7,1.4998456E5,3.15584E6,8.441808E-3,-1.6112167E-4,3.1464421E-3,-4.952421E-3,8.9546805E-4,2.5111986E-3,4.0654964E-3,1.1169499E-2,5.924501E-3,5.848343E7,5.0491E4,1.7505038E-3,2.1721223E3,6.471207E6,-4.946436E-3,2.5867706E-3,8.9E1,3.1086083E-3,1.2022638E0,1.6130304E7,1.4649181E-2,6.627871E-3,2.794906E-2,5.4499237E-3,5.897614E-3,1.8005468E-2,1.1340564E3,1.64E2,-2.8376507E-3,3.9209444E-3,8.01E3,-2.4200217E-3,-3.271167E-3,1.9623388E-3,9.656817E-3,-1.1620384E-3,-1.015751E-2,3.7971328E-4,1.7207018E-3,-7.0578856E-3,-9.858656E-3,7.839862E-4],"split_indices":[2,1,70,9,7,64,66,0,0,64,5,47,41,3,45,0,0,12,0,0,8,45,60,0,0,0,0,0,0,8,9,0,0,3,44,65,11,50,44,0,0,40,12,40,1,0,0,0,0,0,46,0,0,0,1,2,0,67,57,0,0,8,0,53,57,0,0,0,0,0,0,64,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,3.03E2,1.21E2,6.4E1,2.39E2,1.11E2,1E1,6.1E1,3E0,2.13E2,2.6E1,9E1,2.1E1,6E0,4E0,1.3E1,4.8E1,2.09E2,4E0,9E0,1.7E1,4.1E1,4.9E1,1.2E1,9E0,4E0,2E0,2E0,2E0,1.62E2,4.7E1,1E0,3E0,1.1E1,6E0,2.6E1,1.5E1,2E1,2.9E1,1E0,1E0,1.48E2,1.4E1,1.9E1,2.8E1,9E0,2E0,2E0,4E0,2E0,2.4E1,1.1E1,4E0,2E0,1.8E1,2.7E1,2E0,1.4E2,8E0,7E0,7E0,1.5E1,4E0,1.5E1,1.3E1,1.9E1,5E0,1.7E1,1E0,2E0,2.5E1,9.5E1,4.5E1,4E0,4E0,8E0,7E0,8E0,7E0,1E1,3E0,9.1E1,4E0,6E0,3.9E1,7E0,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-3.3745257E-4,-1.5864608E-1,2.55752E-1,-3.185605E-1,-9.708536E-2,1.8780926E-1,6.818292E-1,-3.2428056E-1,2.7597616E-3,-1.1393065E-1,2.3436834E-1,2.3159711E-1,-3.7860292E-1,7.793327E-1,2.3854628E-2,-3.7428176E-1,-1.3420012E-2,-1.8471307E-1,-7.594994E-2,1.6746635E-2,5.5658254E-3,1.1959833E-1,3.418626E-1,-6.417816E-1,6.5567784E-2,4.7681917E-2,1.988688E-2,-6.5651285E-3,1.2232014E-2,-3.81463E-1,-2.892548E-3,-1.9490217E-1,1.0966551E-3,-9.427757E-2,1.2422776E-1,-2.7191767E-2,1.5043752E-1,2.6100993E-1,4.174165E-1,-3.7386432E-2,-1.2527987E-2,1.2707546E-2,-7.246178E-3,-1.9439433E-2,-2.962436E-3,-5.2711805E-3,-2.1830052E-1,-5.6959458E-2,-1.6649641E-1,-1.1975896E-4,1.1501857E-2,-1.0810249E-1,6.191332E-3,2.2629377E-1,9.412359E-2,2.7282703E-1,2.0030153E-3,2.5350976E-1,4.5809898E-1,-2.3912685E-1,-6.2278295E-3,-8.255702E-2,1.6450921E-2,-1.9840728E-1,-2.7044136E-3,-7.547312E-3,1.4027756E-3,2.4949484E-1,-9.634742E-4,1.125155E-1,-3.825301E-3,2.813431E-1,-2.0499254E-4,1.494005E-2,3.1481597E-3,4.9241576E-1,1.30757615E-2,-2.9563266E-1,-1.5677936E-1,-3.358885E-2,-1.2129401E-1,6.104189E-3,-3.3968404E-2,-1.1637037E-2,-3.7270912E-3,1.5680652E-2,7.831384E-3,4.9984753E-2,1.4753471E-1,4.4852975E-3,1.486843E-2,1.49526475E-2,2.7474714E-2,-1.5658757E-2,-3.955088E-3,-1.1362235E-2,-6.695873E-4,-3.594884E-3,3.5750872E-4,-3.7430048E-3,-1.0148317E-2,1.0090659E-3,-4.192081E-3,-2.727012E-3,4.33826E-3,9.037825E-3,1.9827937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,-1,35,37,39,41,-1,-1,-1,-1,43,-1,45,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,59,61,63,-1,-1,65,-1,67,69,71,-1,73,75,77,-1,79,81,83,-1,-1,-1,85,-1,87,-1,89,-1,-1,-1,91,-1,93,95,97,99,-1,101,-1,-1,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8040854E1,2.6762958E0,4.793252E0,1.8039942E-1,1.1433887E0,3.7583761E0,1.4569693E0,1.08569145E-1,0E0,4.9944925E-1,8.657056E-2,1.6633582E0,1.3279387E0,9.5344067E-1,1.6930676E-1,6.1991215E-2,0E0,1.4324951E-1,4.7397858E-1,0E0,0E0,3.2097268E-1,3.125534E-1,1.0350251E-1,2.3553678E-1,0E0,0E0,0E0,0E0,6.4068794E-2,0E0,1.0880041E-1,0E0,3.0314457E-1,1.48083E-1,1.7230181E-1,2.2646272E-1,8.3052635E-2,1.1058378E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.580496E-2,1.4793006E-1,1.2912583E-1,0E0,0E0,5.683428E-2,0E0,1.4155102E-1,1.1847299E-1,8.133769E-2,0E0,5.9072256E-2,4.645157E-2,1.351223E-1,0E0,1.0594487E-1,1.16055526E-1,1.1059332E-1,0E0,0E0,0E0,8.203149E-2,0E0,6.2698394E-2,0E0,7.552171E-2,0E0,0E0,0E0,6.8184376E-2,0E0,5.9469342E-2,1.7909583E-1,4.269389E-2,1.06197625E-1,0E0,4.2242784E-2,0E0,0E0,0E0,0E0,5.478381E-2,6.4677656E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,21,21,22,22,23,23,24,24,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,46,46,47,47,48,48,51,51,53,53,54,54,55,55,57,57,58,58,59,59,61,61,62,62,63,63,67,67,69,69,71,71,75,75,77,77,78,78,79,79,80,80,82,82,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,-1,36,38,40,42,-1,-1,-1,-1,44,-1,46,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,60,62,64,-1,-1,66,-1,68,70,72,-1,74,76,78,-1,80,82,84,-1,-1,-1,86,-1,88,-1,90,-1,-1,-1,92,-1,94,96,98,100,-1,102,-1,-1,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,7.7101436E3,2.91E2,1.4096699E9,9.05929E2,1.92E2,4.90403E5,2.7597616E-3,6.505339E2,3.907527E8,2.5096E4,3.5226266E0,5.192243E6,3.677E3,1.6395413E7,-1.3420012E-2,3.01E2,2.4082131E5,1.6746635E-2,5.5658254E-3,4.0712055E1,1.30817086E5,3.1E1,1.872503E4,4.7681917E-2,1.988688E-2,-6.5651285E-3,1.2232014E-2,8.95654E8,-2.892548E-3,5.1698097E1,1.0966551E-3,2.515304E7,2.251E3,1.5271514E5,1.12E2,1.0014992E8,5.3515047E-1,-3.7386432E-2,-1.2527987E-2,1.2707546E-2,-7.246178E-3,-1.9439433E-2,-2.962436E-3,-5.2711805E-3,9.457892E6,2.753337E6,5.502256E9,-1.1975896E-4,1.1501857E-2,1.153215E9,6.191332E-3,6.88E2,4.2663252E7,5.0772004E9,2.0030153E-3,2.724043E6,1.7176836E7,2.72E2,-6.2278295E-3,7.477264E6,1.3474197E8,1.530655E3,-2.7044136E-3,-7.547312E-3,1.4027756E-3,9.1747506E5,-9.634742E-4,1.79E2,-3.825301E-3,2E0,-2.0499254E-4,1.494005E-2,3.1481597E-3,5.6503E4,1.30757615E-2,3.8147795E-1,2.662171E6,6.2581446E8,1.7045455E-2,6.104189E-3,1.0586429E11,-1.1637037E-2,-3.7270912E-3,1.5680652E-2,7.831384E-3,8.42E2,3.27E2,4.4852975E-3,1.486843E-2,1.49526475E-2,2.7474714E-2,-1.5658757E-2,-3.955088E-3,-1.1362235E-2,-6.695873E-4,-3.594884E-3,3.5750872E-4,-3.7430048E-3,-1.0148317E-2,1.0090659E-3,-4.192081E-3,-2.727012E-3,4.33826E-3,9.037825E-3,1.9827937E-3],"split_indices":[2,1,64,8,7,70,8,9,0,64,5,2,66,41,0,12,0,8,40,0,0,68,40,3,45,0,0,0,0,5,0,60,0,57,0,40,3,1,47,0,0,0,0,0,0,0,44,1,5,0,0,7,0,8,57,44,0,41,55,3,0,57,7,64,0,0,0,59,0,3,0,8,0,0,0,2,0,50,1,7,69,0,43,0,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,2.74E2,1.69E2,7.5E1,1.99E2,1.47E2,2.2E1,7.4E1,1E0,1.9E2,9E0,1.37E2,1E1,1.9E1,3E0,3.6E1,3.8E1,6.5E1,1.25E2,4E0,5E0,6.9E1,6.8E1,6E0,4E0,1.2E1,7E0,2E0,1E0,3.5E1,1E0,6.2E1,3E0,1.15E2,1E1,1.2E1,5.7E1,3.5E1,3.3E1,4E0,2E0,2E0,2E0,3.4E1,1E0,1.4E1,4.8E1,7.7E1,3.8E1,5E0,5E0,8E0,4E0,2.3E1,3.4E1,3.3E1,2E0,8E0,2.5E1,3.8E1,1E1,5.7E1,2E1,2.9E1,9E0,6E0,2E0,2.1E1,2E0,3.1E1,3E0,3.2E1,1E0,6E0,2E0,2E1,5E0,2.1E1,1.7E1,2.6E1,3.1E1,6E0,1.4E1,2.2E1,7E0,1.1E1,1E1,1.2E1,1.9E1,3E0,2.9E1,6E0,1.4E1,1.9E1,2E0,1.1E1,6E0,1.3E1,1.3E1,2.1E1,1E1,7E0,7E0,3E0,9E0,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[6.914658E-3,-1.375686E-1,2.5927174E-1,-3.0013898E-1,-8.7718986E-2,3.0763754E-1,-2.7399808E-1,-3.1545487E-1,8.05666E-4,-1.08775854E-1,1.676461E-1,2.3644063E-1,6.353598E-1,-2.4508651E-2,-3.589169E-2,-1.764556E-2,-2.4414134E-1,-1.4596921E-1,-1.4592769E-3,-2.9842185E-3,2.206311E-1,1.1860321E-1,3.3492017E-1,7.305997E-1,-7.1109384E-2,-1.6415203E-1,1.1294316E-2,-1.2647287E-2,-7.4791734E-4,-1.7872418E-1,-1.03842266E-1,-8.606982E-3,1.9916467E-2,1.4995705E-2,6.1747446E-3,1.5337913E-1,-2.74861E-2,2.673003E-1,3.9268947E-1,4.745792E-2,2.7511101E-2,-9.480651E-3,7.113438E-3,-9.918533E-4,-1.3364579E-2,-2.2289921E-1,-9.7433835E-2,-6.406284E-2,-2.107816E-1,-3.143761E-2,1.07253134E-1,2.1023472E-1,5.8526974E-2,-1.1436814E-2,5.0875377E-3,8.097888E-3,3.0440927E-1,2.3801678E-1,4.3885097E-1,-2.4345683E-1,-6.1107804E-3,-1.165747E-2,-1.2676226E-1,-1.1431789E-1,1.9548368E-2,-2.142738E-3,-2.556923E-1,-6.991408E-2,2.0277679E-2,9.76633E-3,4.6884656E-2,2.512935E-1,1.3112697E-1,-4.8032343E-2,5.5674855E-3,7.729273E-3,1.6775828E-2,3.0432711E-3,1.3727707E-2,7.474615E-3,4.575036E-1,-1.5563093E-2,-1.8704803E-1,4.182032E-3,-2.5413497E-3,-1.4250593E-1,1.2842668E-3,-1.3079995E-1,4.428318E-3,7.899326E-2,-4.779585E-2,-2.711135E-1,-1.3797609E-3,1.2413735E-3,-5.410832E-3,-2.8242762E-3,2.8308968E-3,-3.0703156E-4,6.114718E-3,1.5607449E-2,6.7967544E-3,-3.347285E-4,7.964274E-3,-5.4446757E-3,3.4733661E-3,2.3871105E-2,8.194765E-3,-1.1245429E-2,-4.5142686E-3,-8.430691E-3,-1.2475826E-3,-3.5433506E-3,-1.0178906E-2,6.9158785E-3,3.2170716E-4,3.415501E-3,-4.266625E-3,-5.418153E-3,-1.5446687E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,27,29,31,-1,33,35,37,39,41,43,-1,-1,-1,45,47,-1,49,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1,59,61,63,65,67,69,71,73,-1,-1,-1,75,77,79,81,-1,83,85,87,89,-1,91,93,95,-1,97,99,101,103,-1,-1,-1,-1,-1,-1,105,-1,107,-1,-1,109,-1,111,-1,113,115,117,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6224516E1,2.2671237E0,4.2626E0,3.2477713E-1,1.1904632E0,3.3263636E0,1.5338507E0,8.283901E-2,0E0,8.060622E-1,2.1796706E-1,1.3992448E0,1.8013382E0,3.8690832E-1,0E0,0E0,4.735005E-2,1.8842101E-1,1.9688964E-1,0E0,7.087362E-2,2.988786E-1,1.5675974E-1,3.845415E-1,1.282559E-1,9.900704E-2,0E0,0E0,0E0,2.7633548E-1,2.7576762E-1,0E0,2.1865629E-1,0E0,0E0,2.4116051E-1,3.3569106E-1,8.3488226E-2,1.3830328E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.986784E-2,7.650846E-2,2.1653089E-1,1.24771476E-1,6.310217E-2,8.664283E-2,6.124258E-2,1.12405166E-1,0E0,0E0,0E0,6.0461044E-2,4.8221648E-2,6.6473484E-2,1.0657883E-1,0E0,3.788529E-2,5.8868527E-2,1.18618816E-1,8.3840944E-2,0E0,4.1757524E-2,6.810502E-2,4.229836E-2,0E0,4.877504E-2,8.412957E-2,4.7764674E-2,5.7686985E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.769348E-2,0E0,7.515341E-2,0E0,0E0,6.0432345E-2,0E0,1.16514504E-1,0E0,4.650972E-2,4.947658E-2,4.67965E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,29,29,30,30,32,32,35,35,36,36,37,37,38,38,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,66,66,67,67,68,68,70,70,71,71,72,72,73,73,80,80,82,82,85,85,87,87,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,28,30,32,-1,34,36,38,40,42,44,-1,-1,-1,46,48,-1,50,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1,60,62,64,66,68,70,72,74,-1,-1,-1,76,78,80,82,-1,84,86,88,90,-1,92,94,96,-1,98,100,102,104,-1,-1,-1,-1,-1,-1,106,-1,108,-1,-1,110,-1,112,-1,114,116,118,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,9.05929E2,1.2852292E7,1.4777102E9,7.7101436E3,7.339209E6,1.425966E6,8.05666E-4,9.7467445E4,2.0532622E3,2.5096E4,1.3936486E8,3.236383E1,-3.589169E-2,-1.764556E-2,8.32E3,2.09299E6,1.5802073E3,-2.9842185E-3,6.088569E6,2.4722598E7,1.3047821E5,2.3222E4,3.677E3,2.2201815E9,1.1294316E-2,-1.2647287E-2,-7.4791734E-4,6.82045E6,2.8248588E-2,-8.606982E-3,1.2E3,1.4995705E-2,6.1747446E-3,5.1063637E1,6.426437E5,3.3810287E0,5.3515047E-1,4.745792E-2,2.7511101E-2,-9.480651E-3,7.113438E-3,-9.918533E-4,-1.3364579E-2,5.7192594E5,2.50089E5,7.330576E2,1.339596E7,6.2581446E8,1.473681E6,1.14E2,7.10711E8,-1.1436814E-2,5.0875377E-3,8.097888E-3,1.1274355E7,6.45254E0,4.1322E4,2.55E2,-6.1107804E-3,4.138E3,7.8E2,4.61E2,2.58E2,-2.142738E-3,1.0707373E-1,1.4301278E1,1.0368186E-1,9.76633E-3,2.024909E6,2.488E3,1.66E2,1.0336E4,5.5674855E-3,7.729273E-3,1.6775828E-2,3.0432711E-3,1.3727707E-2,7.474615E-3,5.825042E6,-1.5563093E-2,1E0,4.182032E-3,-2.5413497E-3,1.0643513E3,1.2842668E-3,1.7377544E7,4.428318E-3,6.1E1,9.06E2,3.7382904E7,-1.3797609E-3,1.2413735E-3,-5.410832E-3,-2.8242762E-3,2.8308968E-3,-3.0703156E-4,6.114718E-3,1.5607449E-2,6.7967544E-3,-3.347285E-4,7.964274E-3,-5.4446757E-3,3.4733661E-3,2.3871105E-2,8.194765E-3,-1.1245429E-2,-4.5142686E-3,-8.430691E-3,-1.2475826E-3,-3.5433506E-3,-1.0178906E-2,6.9158785E-3,3.2170716E-4,3.415501E-3,-4.266625E-3,-5.418153E-3,-1.5446687E-2],"split_indices":[2,1,70,9,7,64,59,9,0,45,4,2,1,68,0,0,2,1,4,0,57,57,40,10,0,7,0,0,0,9,69,0,0,0,0,70,59,65,47,0,0,0,0,0,0,63,41,64,57,7,41,3,7,0,0,0,1,66,2,3,0,2,8,3,3,0,51,65,50,0,9,0,3,10,0,0,0,0,0,0,59,0,26,0,0,64,0,57,0,8,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,2.82E2,1.61E2,6.5E1,2.17E2,1.48E2,1.3E1,6.2E1,3E0,2.01E2,1.6E1,1.23E2,2.5E1,9E0,4E0,3.8E1,2.4E1,1.49E2,5.2E1,3E0,1.3E1,5.7E1,6.6E1,2.2E1,3E0,6E0,3E0,2.3E1,1E0,8.2E1,6.7E1,5E0,4.7E1,6E0,7E0,4.6E1,1.1E1,3.3E1,3.3E1,8E0,1.4E1,2E0,1E0,3E0,3E0,5.2E1,3E1,5E1,1.7E1,3E1,1.7E1,2.8E1,1.8E1,4E0,7E0,1E1,2.3E1,9E0,2.4E1,4.2E1,1E1,8E0,2.2E1,3.1E1,1.9E1,4E0,1.3E1,1.7E1,1.3E1,6E0,1.1E1,1.7E1,1.1E1,6E0,1.2E1,5E0,1.8E1,2E0,7E0,2E0,2.2E1,1.7E1,2.5E1,2E0,6E0,2E1,2E0,2.9E1,2E0,1E1,9E0,1.2E1,1E0,5E0,1.2E1,4E0,9E0,7E0,4E0,1E1,7E0,2E0,9E0,4E0,2E0,2E1,2E0,1.7E1,8E0,1.6E1,4E0,1.7E1,1.2E1,5E0,5E0,2E0,7E0,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-4.5158667E-3,-1.1468655E-1,2.8693977E-1,-1.7676862E-1,4.648886E-2,2.1168965E-1,6.262045E-1,-1.6690476E-1,-3.5463657E-2,-3.8676865E-2,2.3283775E-1,2.4176304E-1,-1.5732015E-2,1.8295826E-2,4.0603448E-2,-3.2807168E-1,-1.3398427E-1,-6.376014E-2,8.729884E-3,1.8902464E-1,2.7464014E-2,1.00318655E-1,3.026897E-1,-1.7497705E-2,-5.402815E-3,-1.4107442E-1,7.760317E-2,-8.407632E-3,-3.849008E-2,4.8027337E-3,1.2841712E-2,1.3443221E-1,-1.0395823E-1,2.2458532E-1,3.7418863E-1,-1.5926063E-1,-4.630308E-2,2.3180418E-4,8.663032E-3,-1.6562516E-2,-1.6455689E-1,1.00598615E-2,4.6943597E-2,-1.0924818E-2,2.2657758E-3,1.275023E-1,2.76157E-1,4.8426003E-3,3.884912E-1,-1.635734E-1,1.3047633E-2,-9.789412E-2,-3.6889873E-5,-2.1150315E-3,5.1360115E-2,-3.608747E-3,-1.3174543E-2,-1.6283733E-3,7.254288E-3,1.0035674E-2,3.2216536E-3,4.0086736E-3,2.9174092E-1,2.0434916E-2,7.1690106E-3,-1.8277587E-1,-7.1852885E-2,-1.0371063E-2,-3.9164472E-2,7.680068E-3,9.006588E-6,6.9954344E-3,1.6039012E-2,-9.884866E-3,-5.4080896E-3,7.098563E-4,-7.129677E-3,2.2595797E-3,-3.8210596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,35,37,-1,39,-1,-1,41,43,45,47,49,51,-1,-1,53,55,-1,57,-1,-1,59,61,-1,63,65,-1,67,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,71,-1,-1,73,75,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4514458E1,3.2902198E0,3.033124E0,1.1272106E0,1.469095E0,1.7069097E0,7.683964E-1,1.197576E0,0E0,3.5364774E-1,2.9852247E-1,8.1417274E-1,0E0,0E0,0E0,1.5130568E-1,3.0498433E-1,1.4664695E-1,0E0,1.4574534E-1,0E0,2.300022E-1,2.9570103E-1,0E0,0E0,3.2009673E-1,4.8095778E-2,0E0,1.2996274E-1,0E0,0E0,1.4848989E-1,9.54889E-2,1.4002502E-1,9.719801E-2,3.3094358E-1,7.515147E-2,0E0,0E0,7.565577E-2,4.4999927E-2,0E0,1.06265314E-1,0E0,0E0,5.175139E-2,4.3777943E-2,0E0,1.013298E-1,2.6514578E-1,0E0,8.844736E-2,0E0,0E0,6.2809795E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.368639E-2,0E0,0E0,1.2414837E-1,1.7874038E-1,0E0,3.8070843E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,17,17,19,19,21,21,22,22,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,42,42,45,45,46,46,48,48,49,49,51,51,54,54,62,62,65,65,66,66,68,68],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,36,38,-1,40,-1,-1,42,44,46,48,50,52,-1,-1,54,56,-1,58,-1,-1,60,62,-1,64,66,-1,68,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,72,-1,-1,74,76,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.7467445E4,1.9900316E0,2.2233E4,2.1695537E3,2.0531621E4,6.0495757E3,4.26944E5,-3.5463657E-2,1.5271514E5,3.3056E4,9.396226E4,-1.5732015E-2,1.8295826E-2,4.0603448E-2,7.154E3,2.9063344E5,1.6826648E3,8.729884E-3,1.757E3,2.7464014E-2,1.1863593E-1,6.7304E4,-1.7497705E-2,-5.402815E-3,6.4065235E8,2.8182E4,-8.407632E-3,1E0,4.8027337E-3,1.2841712E-2,7.801943E3,2.4469028E7,7.996302E-1,4.2464733E-2,1.3664E4,3.738719E4,2.3180418E-4,8.663032E-3,3.15584E6,2.865605E8,1.00598615E-2,8.255324E4,-1.0924818E-2,2.2657758E-3,1.5191719E5,1.0721749E3,4.8426003E-3,3.0113987E10,2.822E3,1.3047633E-2,3.923606E1,-3.6889873E-5,-2.1150315E-3,7.25E3,-3.608747E-3,-1.3174543E-2,-1.6283733E-3,7.254288E-3,1.0035674E-2,3.2216536E-3,4.0086736E-3,1.375746E9,2.0434916E-2,7.1690106E-3,1.347372E3,1.4399977E10,-1.0371063E-2,4.422E3,7.680068E-3,9.006588E-6,6.9954344E-3,1.6039012E-2,-9.884866E-3,-5.4080896E-3,7.098563E-4,-7.129677E-3,2.2595797E-3,-3.8210596E-3],"split_indices":[7,45,51,2,64,4,64,1,0,40,2,45,0,0,0,2,40,4,0,0,0,50,2,0,0,7,9,0,19,0,0,44,9,54,50,2,40,0,0,1,7,0,45,0,0,45,4,0,12,0,0,65,0,0,2,0,0,0,0,0,0,0,7,0,0,64,5,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.5E2,3.27E2,1.23E2,2.36E2,9.1E1,1.02E2,2.1E1,2.33E2,3E0,6.3E1,2.8E1,9.7E1,5E0,1E1,1.1E1,3.8E1,1.95E2,5.7E1,6E0,2.6E1,2E0,3E1,6.7E1,3.4E1,4E0,1.89E2,6E0,1E1,4.7E1,1.2E1,1.4E1,2.6E1,4E0,3.4E1,3.3E1,1.58E2,3.1E1,4E0,2E0,4.1E1,6E0,1.4E1,1.2E1,2E0,2E0,1.3E1,2.1E1,2E0,3.1E1,1.57E2,1E0,1.4E1,1.7E1,3E1,1.1E1,4E0,2E0,7E0,5E0,5E0,8E0,2E0,1.9E1,2.8E1,3E0,1.29E2,2.8E1,4E0,1E1,3E0,8E0,4E0,1.5E1,1.06E2,2.3E1,1.3E1,1.5E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.29316E-3,-1.2366014E-1,2.6226074E-1,-2.6201954E-1,-7.987835E-2,3.1039494E-1,-3.2606393E-1,-2.737112E-1,8.425078E-4,-1.10770546E-1,6.2356103E-2,2.3377381E-1,5.665446E-1,8.241814E-2,-5.712563E-1,-1.7760327E-2,-1.2409545E-2,-1.2958758E-1,5.5793077E-2,1.2357809E-2,1.686126E-1,1.6638435E-1,3.3238313E-1,-9.579106E-3,6.2980795E-1,-6.1339363E-3,1.2998829E-2,-3.3236064E-2,-1.1221248E-2,-1.4182223E-1,3.8490176E-2,-7.1630003E-3,9.6942715E-2,-9.020776E-2,6.5494165E-2,1.3357296E-2,6.1207533E-2,-4.7589215E-3,1.8130043E-1,2.7086937E-1,4.1684806E-1,2.1011192E-2,8.229569E-1,-3.7125804E-2,-1.5571955E-1,-1.7791794E-3,8.203853E-3,1.2700339E-1,-2.0000655E-3,-9.409242E-3,-3.634753E-4,-1.1811532E-4,1.074345E-1,5.9928386E-3,-1.3732224E-3,2.0163079E-1,8.487063E-2,5.1353686E-3,2.967646E-1,2.1663504E-2,4.5987293E-3,9.541472E-3,4.2965267E-2,-3.8816715E-3,6.7940517E-3,-1.8447061E-1,-9.001048E-2,8.107463E-4,7.6636653E-3,6.5866155E-3,-8.845845E-4,1.3973573E-2,1.7089231E-1,-8.119495E-4,6.869721E-3,1.6269138E-2,6.688087E-3,4.384359E-3,-1.7253119E-3,-2.1550064E-1,-1.3866964E-1,-1.7942494E-2,-1.6443175E-1,6.8859295E-3,1.1825405E-2,-8.143927E-3,-1.358049E-2,-2.5548001E-3,-9.105189E-3,2.407192E-3,-2.9055818E-3,-9.103464E-3,7.222216E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,35,37,39,-1,41,-1,-1,-1,-1,43,45,-1,47,49,51,-1,53,-1,55,57,59,-1,61,63,65,-1,-1,67,-1,-1,-1,-1,69,-1,-1,71,73,-1,75,-1,-1,-1,-1,-1,77,79,81,-1,-1,-1,-1,-1,83,-1,-1,-1,-1,-1,-1,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4405259E1,1.9053941E0,4.037485E0,2.5241566E-1,1.074633E0,2.3993711E0,1.1488053E0,6.318998E-2,0E0,6.3407683E-1,2.3167266E-1,6.1586475E-1,1.5117731E0,2.1418726E-1,7.829523E-2,0E0,0E0,3.756063E-1,1.859641E-1,1.7485687E-1,1.3199472E-1,2.5919867E-1,9.491873E-2,0E0,7.4594593E-1,0E0,0E0,0E0,0E0,2.3865843E-1,1.2688246E-1,0E0,7.91924E-2,8.809562E-2,6.0019284E-2,0E0,4.4946156E-2,0E0,9.855425E-2,8.269191E-2,4.1137695E-2,0E0,1.3789654E-1,3.7922837E-2,2.6299787E-1,0E0,0E0,4.128723E-2,0E0,0E0,0E0,0E0,4.196185E-2,0E0,0E0,6.7264795E-2,6.604587E-2,0E0,5.9578896E-2,0E0,0E0,0E0,0E0,0E0,3.9812423E-2,1.1507964E-1,2.4912912E-1,0E0,0E0,0E0,0E0,0E0,4.7914505E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3006568E-1,1.5833747E-1,6.923527E-2,7.555705E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,29,29,30,30,32,32,33,33,34,34,36,36,38,38,39,39,40,40,42,42,43,43,44,44,47,47,52,52,55,55,56,56,58,58,64,64,65,65,66,66,72,72,79,79,80,80,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,36,38,40,-1,42,-1,-1,-1,-1,44,46,-1,48,50,52,-1,54,-1,56,58,60,-1,62,64,66,-1,-1,68,-1,-1,-1,-1,70,-1,-1,72,74,-1,76,-1,-1,-1,-1,-1,78,80,82,-1,-1,-1,-1,-1,84,-1,-1,-1,-1,-1,-1,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.41718E5,1.0429407E3,1.2852292E7,3.44E2,6.6512886E3,2.909019E3,1.44E2,8.425078E-4,8.464347E8,2.9222983E3,6.0891E4,5.0314346E1,2.5212732E4,3.1E1,-1.7760327E-2,-1.2409545E-2,2.889654E5,4.6288285E0,1.570662E5,1.5E2,4.56E2,1.619807E5,-9.579106E-3,2.2876814E1,-6.1339363E-3,1.2998829E-2,-3.3236064E-2,-1.1221248E-2,2.8101E5,4.392E3,-7.1630003E-3,1.6804138E1,2.4469028E7,3.6828573E8,1.3357296E-2,3.61E2,-4.7589215E-3,2.237815E7,3E0,9.2E1,2.1011192E-2,4.9E1,6.2581446E8,1.0864745E3,-1.7791794E-3,8.203853E-3,1.9937408E-1,-2.0000655E-3,-9.409242E-3,-3.634753E-4,-1.1811532E-4,1.03816045E9,5.9928386E-3,-1.3732224E-3,3.4E1,3.7860696E0,5.1353686E-3,1.0038E4,2.1663504E-2,4.5987293E-3,9.541472E-3,4.2965267E-2,-3.8816715E-3,4.45E3,2.73E2,2.695258E1,8.107463E-4,7.6636653E-3,6.5866155E-3,-8.845845E-4,1.3973573E-2,1.6283882E0,-8.119495E-4,6.869721E-3,1.6269138E-2,6.688087E-3,4.384359E-3,-1.7253119E-3,1.36E2,2.7434757E1,1.4040858E-2,1.8449864E6,6.8859295E-3,1.1825405E-2,-8.143927E-3,-1.358049E-2,-2.5548001E-3,-9.105189E-3,2.407192E-3,-2.9055818E-3,-9.103464E-3,7.222216E-4],"split_indices":[2,1,70,9,8,64,44,0,0,12,4,2,68,4,3,0,0,40,66,59,3,10,40,0,66,0,0,0,0,41,10,0,65,9,5,0,0,0,57,8,8,0,3,7,64,0,0,53,0,0,0,0,44,0,0,8,65,0,0,0,0,0,0,0,2,3,65,0,0,0,0,0,51,0,0,0,0,0,0,3,66,69,59,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.55E2,3.17E2,1.38E2,7.5E1,2.42E2,1.28E2,1E1,7.2E1,3E0,1.99E2,4.3E1,1E2,2.8E1,4E0,6E0,1.4E1,5.8E1,1.79E2,2E1,3E1,1.3E1,6.1E1,3.9E1,2E0,2.6E1,2E0,2E0,4E0,2E0,1.67E2,1.2E1,3E0,1.7E1,1E1,2E1,6E0,7E0,3E0,5.8E1,2.5E1,1.4E1,1.4E1,1.2E1,2E1,1.47E2,8E0,4E0,1.4E1,3E0,4E0,6E0,8E0,1.2E1,4E0,3E0,4.7E1,1.1E1,4E0,2.1E1,1.3E1,1E0,1E0,1.1E1,1E1,1E1,1.01E2,4.6E1,3E0,1.1E1,1E1,2E0,1.1E1,3.6E1,4E0,7E0,1.7E1,4E0,3E0,7E0,5.8E1,4.3E1,2.4E1,2.2E1,2.6E1,1E1,3.2E1,2.6E1,1.5E1,2.8E1,9E0,1.5E1,2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.0118642E-2,-1.1064205E-1,2.322225E-1,-1.482631E-1,4.0452495E-2,1.8081708E-1,6.41901E-1,-2.2425067E-1,-8.486963E-2,-1.196619E-2,1.6757792E-1,2.0487635E-1,-3.055771E-1,1.5708769E-2,3.44657E-2,-2.8851342E-1,-1.8747228E-1,-4.037244E-2,-1.4056477E-1,-3.4082413E-2,7.921726E-3,1.308292E-2,3.770976E-3,1.5031286E-1,3.1306732E-1,2.013823E-1,-2.7418746E-2,-1.6460095E-2,-8.2830265E-3,-5.230244E-3,-1.1324873E-2,-1.012855E-1,3.0685853E-2,-1.6689155E-1,-4.06461E-2,4.707947E-3,-6.107999E-2,7.198736E-2,2.287792E-1,2.9271256E-2,2.456573E-1,1.6307868E-2,-1.2114196E-3,-1.0273271E-2,-7.205077E-2,-6.565904E-2,1.1020335E-1,-1.9082592E-1,-5.261052E-2,-7.1322955E-3,-4.554945E-4,-4.349437E-3,7.450287E-4,-2.6296644E-2,1.0643607E-1,2.65676E-1,-1.0786434E-2,4.7467453E-3,2.6104298E-1,-1.8523576E-2,-6.2180366E-3,7.382024E-4,-5.1755747E-3,-6.4423727E-4,7.1909525E-3,-1.325472E-1,-2.4816407E-1,-8.191436E-3,1.5170552E-3,-4.2720055E-3,4.272681E-3,1.0401469E-2,6.859637E-2,3.1505397E-1,1.3129017E-1,6.9745984E-3,2.9433557E-1,-3.5113832E-3,1.2353559E-3,-1.5905786E-1,6.6267577E-4,-1.5944216E-2,-7.482292E-3,1.0424624E-3,6.803541E-3,6.2172273E-3,3.3959997E-1,-7.021423E-3,8.278387E-3,1.660147E-2,5.95235E-3,-2.9454117E-3,-1.0410603E-2,2.3336867E-3,1.7464057E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,-1,-1,-1,37,39,41,-1,-1,-1,-1,-1,43,45,47,49,-1,51,53,55,-1,57,-1,-1,-1,59,61,63,65,67,-1,-1,-1,-1,69,71,73,-1,-1,75,77,-1,-1,-1,-1,-1,79,81,-1,-1,-1,-1,-1,83,85,87,-1,89,-1,-1,91,-1,-1,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1880759E1,1.637402E0,3.1692448E0,1.0889049E0,3.8850266E-1,1.7009263E0,4.9096584E-2,1.9171381E-1,3.090663E-1,1.6375487E-1,1.2778053E-1,7.4398136E-1,9.730376E-1,0E0,0E0,1.2594533E-1,1.9990945E-1,3.1475127E-1,1.4052653E-1,1.3736403E-1,0E0,0E0,0E0,5.4280925E-1,6.6477203E-1,9.247157E-2,0E0,0E0,0E0,0E0,0E0,1.04082316E-1,2.677134E-1,1.1031127E-1,4.05384E-2,0E0,6.439856E-2,1.6192947E-1,7.955539E-1,0E0,6.0982227E-2,0E0,0E0,0E0,8.739738E-2,5.0305113E-2,8.063592E-2,8.740401E-2,8.808014E-2,0E0,0E0,0E0,0E0,9.331989E-2,1.20404035E-1,2.3733592E-1,0E0,0E0,9.2392206E-2,3.9770342E-2,0E0,0E0,0E0,0E0,0E0,7.9417884E-2,7.0846796E-2,0E0,0E0,0E0,0E0,0E0,8.421224E-2,9.81431E-2,1.4466262E-1,0E0,1.262598E-1,0E0,0E0,7.155603E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.8681965E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,23,23,24,24,25,25,31,31,32,32,33,33,34,34,36,36,37,37,38,38,40,40,44,44,45,45,46,46,47,47,48,48,53,53,54,54,55,55,58,58,59,59,65,65,66,66,72,72,73,73,74,74,76,76,79,79,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,-1,-1,-1,38,40,42,-1,-1,-1,-1,-1,44,46,48,50,-1,52,54,56,-1,58,-1,-1,-1,60,62,64,66,68,-1,-1,-1,-1,70,72,74,-1,-1,76,78,-1,-1,-1,-1,-1,80,82,-1,-1,-1,-1,-1,84,86,88,-1,90,-1,-1,92,-1,-1,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,8.625562E0,1.252737E6,1.5522031E9,2.3028242E3,4.6033485E1,1.86005E5,1.8071064E7,1E0,6.1614815E6,1.5271514E5,2.909019E3,1.5708769E-2,3.44657E-2,1.10388414E5,1.2921541E-2,2.753337E6,1.530655E3,1.4193897E-3,7.921726E-3,1.308292E-2,3.770976E-3,2.5096E4,5.695737E0,8.494E3,-2.7418746E-2,-1.6460095E-2,-8.2830265E-3,-5.230244E-3,-1.1324873E-2,4.5765094E2,5.16152E2,6.842986E6,6.844E3,4.707947E-3,6.426437E5,7.10711E8,7.79557E5,2.9271256E-2,1E0,1.6307868E-2,-1.2114196E-3,-1.0273271E-2,1.8518519E-2,7.012954E2,6.5610016E4,5.989E3,1.7895421E0,-7.1322955E-3,-4.554945E-4,-4.349437E-3,7.450287E-4,2.7385738E5,5.4629724E10,2.024909E6,-1.0786434E-2,4.7467453E-3,1.8805E4,1.533115E6,-6.2180366E-3,7.382024E-4,-5.1755747E-3,-6.4423727E-4,7.1909525E-3,1E0,9.3E1,-8.191436E-3,1.5170552E-3,-4.2720055E-3,4.272681E-3,1.0401469E-2,7.228959E6,5.0150156E0,1.379E3,6.9745984E-3,1.18138E5,-3.5113832E-3,1.2353559E-3,7.94203E5,6.6267577E-4,-1.5944216E-2,-7.482292E-3,1.0424624E-3,6.803541E-3,6.2172273E-3,1.1274355E7,-7.021423E-3,8.278387E-3,1.660147E-2,5.95235E-3,-2.9454117E-3,-1.0410603E-2,2.3336867E-3,1.7464057E-2],"split_indices":[2,7,47,1,7,70,68,9,57,101,57,40,44,0,0,40,51,1,64,51,0,0,0,2,66,0,0,0,0,0,0,64,64,41,2,0,59,7,2,0,102,0,0,0,69,67,40,2,65,0,0,0,0,45,43,9,0,0,2,1,0,0,0,0,0,24,8,0,0,0,0,0,1,66,0,0,10,0,0,41,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,2.86E2,1.55E2,2.29E2,5.7E1,1.39E2,1.6E1,1.03E2,1.26E2,4.1E1,1.6E1,1.33E2,6E0,3E0,1.3E1,3.5E1,6.8E1,7.1E1,5.5E1,3.7E1,4E0,7E0,9E0,9E1,4.3E1,2E0,4E0,2.5E1,1E1,2.3E1,4.5E1,3.8E1,3.3E1,4.3E1,1.2E1,6E0,3.1E1,4.6E1,4.4E1,7E0,3.6E1,1E0,1E0,7E0,3.1E1,1.5E1,1.8E1,3.5E1,8E0,2E0,1E1,2.3E1,8E0,1.2E1,3.4E1,4.1E1,3E0,4E0,3.2E1,1.6E1,1.5E1,5E0,1E1,4E0,1.4E1,1.9E1,1.6E1,3E0,5E0,8E0,4E0,8E0,2.6E1,2.9E1,1.2E1,8E0,2.4E1,7E0,9E0,1.6E1,3E0,8E0,8E0,1.6E1,1E1,4E0,2.5E1,1E0,1.1E1,1.9E1,5E0,6E0,1E1,1E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[1.647935E-2,-1.0800233E-1,2.5379223E-1,-2.305319E-1,-6.834447E-2,1.7691189E-1,4.700406E-1,-2.3805451E-1,8.408377E-4,-9.220398E-2,5.97228E-2,7.9566844E-2,2.8957498E-1,7.856261E-1,3.3412743E-1,-1.675159E-2,-2.134281E-1,-1.43772E-1,-4.6933446E-2,7.622511E-3,2.0938781E-1,1.0626647E-1,-2.4035897E-2,3.0569378E-1,-1.33056985E-2,1.8956881E-2,4.450752E-2,4.0694165E-1,-9.6769236E-2,-1.3231741E-3,-1.0981993E-2,-1.5539342E-1,4.789232E-4,5.154256E-3,-9.707956E-2,-3.586917E-2,4.927209E-3,1.5041862E-3,1.2809556E-2,4.5523564E-3,1.3712397E-1,2.086619E-1,3.5229716E-1,6.351443E-3,4.5419493E-1,-1.009842E-2,8.106471E-3,-3.5865575E-3,-1.705486E-1,-2.0502027E-2,8.810765E-2,-7.420468E-4,-1.1677388E-1,-4.1350573E-3,7.29116E-4,-1.1129401E-2,5.6643765E-2,1.5328002E-1,-3.5400363E-3,2.393879E-1,1.678819E-3,6.7270654E-3,3.7768197E-1,2.3491438E-2,3.4239031E-3,-1.9676779E-1,-1.1183504E-1,-3.1494997E-2,6.0926825E-3,8.684901E-3,1.1768502E-3,-3.648602E-3,-7.559087E-3,6.416758E-3,-2.4222042E-3,5.6535593E-4,1.6364478E-1,1.3668489E-2,5.4793707E-3,4.1697806E-1,1.1115056E-2,-2.1669492E-1,-2.8803984E-3,-2.3783368E-1,-4.184603E-2,-9.468447E-2,7.942482E-4,2.3164696E-1,1.2622789E-1,4.683328E-3,2.1385228E-2,-1.2016297E-2,-5.5386415E-3,-1.3395162E-2,-6.609183E-4,-3.8383205E-3,1.4601192E-3,-1.0110219E-3,-6.863419E-3,3.8866235E-3,1.3495201E-2,3.6749647E-3,8.345667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,41,-1,-1,-1,43,45,-1,-1,47,-1,49,51,53,-1,-1,-1,55,57,59,61,-1,63,-1,-1,-1,65,67,69,-1,71,-1,-1,-1,73,75,-1,77,-1,-1,79,-1,-1,81,83,85,-1,-1,-1,-1,-1,-1,-1,-1,87,-1,-1,89,-1,91,-1,93,95,97,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2966414E1,1.3837118E0,2.4272566E0,1.3425922E-1,6.746881E-1,1.2162526E0,1.4102974E0,9.112668E-2,0E0,4.2513692E-1,2.713196E-1,9.7897196E-1,5.472603E-1,1.312499E-1,9.492667E-1,0E0,6.1496258E-2,1.5475857E-1,2.6169991E-1,1.1035886E-1,6.800538E-2,1.8769431E-1,0E0,1.5711212E-1,0E0,0E0,0E0,2.7287483E-1,1.6912194E-1,0E0,0E0,8.9152455E-2,0E0,1.08220175E-1,8.0860406E-2,4.6143394E-2,0E0,0E0,0E0,1.9017844E-1,1.6540784E-1,9.416282E-2,1.315217E-1,0E0,9.6241E-2,0E0,0E0,0E0,8.113575E-2,6.489619E-2,6.228009E-2,0E0,5.0830126E-2,0E0,0E0,0E0,1.04269184E-1,6.1421514E-2,0E0,4.0830314E-2,0E0,0E0,8.420944E-2,0E0,0E0,1.12273335E-1,1.9287357E-1,1.1229738E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.2536945E-2,0E0,0E0,3.7641525E-2,0E0,7.13979E-2,0E0,5.0716847E-2,4.1990094E-2,4.8028886E-2,0E0,5.5787623E-2,4.7866046E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,23,23,27,27,28,28,31,31,33,33,34,34,35,35,39,39,40,40,41,41,42,42,44,44,48,48,49,49,50,50,52,52,56,56,57,57,59,59,62,62,65,65,66,66,67,67,76,76,79,79,81,81,83,83,84,84,85,85,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,42,-1,-1,-1,44,46,-1,-1,48,-1,50,52,54,-1,-1,-1,56,58,60,62,-1,64,-1,-1,-1,66,68,70,-1,72,-1,-1,-1,74,76,-1,78,-1,-1,80,-1,-1,82,84,86,-1,-1,-1,-1,-1,-1,-1,-1,88,-1,-1,90,-1,92,-1,94,96,98,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.32424E5,5.932393E3,1.2852292E7,2.76E2,1.0693399E-1,1.417988E6,1.44E2,8.408377E-4,7.621583E2,1.2103396E5,3.764913E7,2.8756912E8,9.256843E3,1.5096262E7,-1.675159E-2,2.586E4,1.348197E2,2.1692945E1,1.4096699E9,3.636015E8,4.7051765E1,-2.4035897E-2,5.2060513E9,-1.33056985E-2,1.8956881E-2,4.450752E-2,2.3193505E0,3.677E3,-1.3231741E-3,-1.0981993E-2,5.1698097E1,4.789232E-4,2.473E3,1.32E2,1.8374788E7,4.927209E-3,1.5041862E-3,1.2809556E-2,2.8884522E8,4.2663252E7,2.237815E7,7.815434E4,6.351443E-3,2.1030048E10,-1.009842E-2,8.106471E-3,-3.5865575E-3,1.646359E5,2.2860639E9,7.494142E6,-7.420468E-4,1.70386E7,-4.1350573E-3,7.29116E-4,-1.1129401E-2,4.3603E6,2.8593372E3,-3.5400363E-3,5.31972E7,1.678819E-3,6.7270654E-3,1.5508488E0,2.3491438E-2,3.4239031E-3,1.958583E3,1.763213E-5,1.1016051E3,6.0926825E-3,8.684901E-3,1.1768502E-3,-3.648602E-3,-7.559087E-3,6.416758E-3,-2.4222042E-3,5.6535593E-4,7.2123305E6,1.3668489E-2,5.4793707E-3,2.640416E-2,1.1115056E-2,6.623914E4,-2.8803984E-3,2.1933604E3,2.135E3,7.1303144E7,7.942482E-4,3.44E2,4.729687E3,4.683328E-3,2.1385228E-2,-1.2016297E-2,-5.5386415E-3,-1.3395162E-2,-6.609183E-4,-3.8383205E-3,1.4601192E-3,-1.0110219E-3,-6.863419E-3,3.8866235E-3,1.3495201E-2,3.6749647E-3,8.345667E-3],"split_indices":[2,1,64,9,8,47,41,0,0,64,45,60,1,4,9,0,41,66,65,7,5,68,0,12,0,0,0,65,0,0,0,60,0,0,3,12,0,0,0,12,57,57,40,0,5,0,0,0,40,12,1,0,57,0,0,0,60,67,0,56,0,0,46,0,0,4,50,64,0,0,0,0,0,0,0,0,57,0,0,50,0,45,0,44,0,7,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.87E2,1.5E2,6.9E1,2.18E2,1.12E2,3.8E1,6.7E1,2E0,1.84E2,3.4E1,6.1E1,5.1E1,1E1,2.8E1,1.1E1,5.6E1,8.5E1,9.9E1,2.6E1,8E0,5.9E1,2E0,5E1,1E0,3E0,7E0,2.4E1,4E0,2E0,5.4E1,7.9E1,6E0,4.9E1,5E1,1.8E1,8E0,2E0,6E0,1.4E1,4.5E1,1.8E1,3.2E1,4E0,2E1,3E0,1E0,1.3E1,6.6E1,3.8E1,1.1E1,1E1,4E1,9E0,9E0,2E0,1.2E1,4.2E1,3E0,1.5E1,3E0,4E0,2.8E1,1.9E1,1E0,4.4E1,2.2E1,3.6E1,2E0,4E0,7E0,1.9E1,2.1E1,7E0,5E0,3E0,3.9E1,1.1E1,4E0,2.1E1,7E0,3.8E1,6E0,7E0,1.5E1,1.5E1,2.1E1,1.2E1,2.7E1,1E0,2E1,3E1,8E0,6E0,1E0,1E1,5E0,6E0,9E0,3E0,9E0,1.3E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[8.508928E-3,-1.1184809E-1,2.2897916E-1,-2.291963E-1,-6.3077055E-2,1.6880473E-1,4.8460573E-1,-2.4334742E-1,-2.038602E-3,-6.2371246E-3,-1.1985523E-1,1.8931116E-1,-2.7629325E-1,3.8017595E-1,4.1058216E-2,-2.5463197E-1,-4.7040437E-3,-3.608007E-2,1.2602837E-1,-2.3302774E-1,-1.0343384E-1,2.854015E-1,1.2944797E-1,4.267786E-3,-2.3912922E-2,4.0902588E-1,-7.080921E-3,-4.823212E-3,-1.31253E-2,-7.333689E-2,6.592007E-2,-4.903948E-3,1.7419016E-1,-2.8294215E-1,-3.1216764E-3,-1.1915602E-1,-2.3271507E-2,1.4631501E-2,3.256512E-1,1.944744E-1,3.1927224E-2,6.954831E-3,2.1394776E-2,-1.19564235E-1,-2.1494145E-3,-4.018017E-3,1.22291364E-1,2.1949092E-1,1.5307837E-3,-1.5577815E-2,-1.3202757E-3,-1.5962383E-1,-5.9789155E-2,-3.3478504E-3,2.124315E-3,-3.935328E-3,3.3849124E-3,3.3129446E-2,2.8038985E-1,1.4749454E-1,2.9132158E-1,-1.124878E-2,7.6681696E-2,-1.3365248E-1,1.1796481E-3,1.0378074E-2,2.8410386E-3,1.3134291E-2,3.3557345E-3,-1.3175184E-1,-1.4399387E-2,-9.465883E-2,-7.717996E-5,2.959239E-1,-6.967557E-3,9.332888E-2,1.1177012E-2,1.6353047E-2,-4.3810206E-3,1.05758764E-1,-1.0039201E-2,-1.2337245E-3,-8.208533E-3,2.2882062E-3,-1.4288688E-1,-2.4354638E-4,-6.636338E-3,9.600813E-3,1.6983235E-2,6.6534174E-3,1.2726691E-3,-2.7975924E-3,1.2928881E-1,-3.0124153E-3,-8.612954E-3,4.538085E-3,1.0209081E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,-1,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,43,45,-1,47,49,-1,51,53,55,57,59,61,-1,-1,63,-1,-1,65,67,-1,-1,-1,69,71,-1,-1,-1,-1,-1,73,75,77,-1,79,81,-1,-1,-1,-1,-1,83,-1,85,-1,87,-1,89,-1,-1,-1,91,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,95,-1,-1,-1,-1],"loss_changes":[1.1648003E1,1.61093E0,2.302435E0,2.1136284E-1,6.511606E-1,1.2114811E0,7.081089E-1,1.0569286E-1,0E0,4.0716067E-1,1.6360056E-1,6.648326E-1,4.7965032E-1,4.192896E-1,0E0,6.446266E-2,0E0,3.2405126E-1,2.2221032E-1,8.448988E-2,1.1065638E-1,4.9656224E-1,4.820807E-1,0E0,0E0,5.8816195E-2,0E0,0E0,0E0,8.17807E-2,1.9954029E-1,0E0,9.733081E-2,5.817139E-2,0E0,1.7063987E-1,4.885801E-2,4.0040597E-2,4.2471218E-1,1.6632533E-1,3.8533562E-1,0E0,0E0,5.1562577E-2,0E0,0E0,8.3171636E-2,6.154865E-2,0E0,0E0,0E0,1.19629264E-1,6.484511E-2,0E0,0E0,0E0,0E0,0E0,2.825458E-1,1.1524761E-1,2.1914017E-1,0E0,2.4722399E-1,6.929353E-2,0E0,0E0,0E0,0E0,0E0,8.1664145E-2,0E0,6.782594E-2,0E0,9.879923E-2,0E0,5.333437E-2,0E0,0E0,0E0,1.0618526E-1,0E0,0E0,0E0,0E0,7.720834E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.782632E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,43,43,46,46,47,47,51,51,52,52,58,58,59,59,60,60,62,62,63,63,69,69,71,71,73,73,75,75,79,79,84,84,92,92],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,-1,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,44,46,-1,48,50,-1,52,54,56,58,60,62,-1,-1,64,-1,-1,66,68,-1,-1,-1,70,72,-1,-1,-1,-1,-1,74,76,78,-1,80,82,-1,-1,-1,-1,-1,84,-1,86,-1,88,-1,90,-1,-1,-1,92,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,96,-1,-1,-1,-1],"split_conditions":[1.0262E4,9.04782E5,4.026531E9,1.2852292E7,1.4074289E7,9.533205E6,5.01E2,2.4752695E6,-2.038602E-3,9.481747E8,1.2808544E3,1.30569E5,1.618E3,2.617838E8,4.1058216E-2,7.547656E1,-4.7040437E-3,2.972948E6,1.5560601E6,4.241E3,4.723148E11,5.274031E6,1.6285607E7,4.267786E-3,-2.3912922E-2,3.9541063E0,-7.080921E-3,-4.823212E-3,-1.31253E-2,4.827E3,3E1,-4.903948E-3,1.5E2,2.4927816E1,-3.1216764E-3,7.194E3,5.767668E0,1.153215E9,4.7431055E-2,5.6503E4,2.3808822E-2,6.954831E-3,2.1394776E-2,3.53E2,-2.1494145E-3,-4.018017E-3,1.03E2,5.88824E7,1.5307837E-3,-1.5577815E-2,-1.3202757E-3,6.746E3,1.0821084E-2,-3.3478504E-3,2.124315E-3,-3.935328E-3,3.3849124E-3,3.3129446E-2,1.505649E6,5.841568E1,1.2622761E8,-1.124878E-2,4.31E2,5.428232E9,1.1796481E-3,1.0378074E-2,2.8410386E-3,1.3134291E-2,3.3557345E-3,3.904E3,-1.4399387E-2,1.83918E5,-7.717996E-5,5.0491E4,-6.967557E-3,7.0217915E-2,1.1177012E-2,1.6353047E-2,-4.3810206E-3,1.5165479E0,-1.0039201E-2,-1.2337245E-3,-8.208533E-3,2.2882062E-3,7.058968E0,-2.4354638E-4,-6.636338E-3,9.600813E-3,1.6983235E-2,6.6534174E-3,1.2726691E-3,-2.7975924E-3,3.5940964E7,-3.0124153E-3,-8.612954E-3,4.538085E-3,1.0209081E-2],"split_indices":[2,1,7,9,57,59,3,62,0,7,4,9,0,1,0,66,0,1,57,2,43,1,57,0,0,66,0,0,0,2,8,0,3,70,0,2,66,7,50,2,50,0,0,3,0,0,3,9,0,0,0,2,50,0,0,0,0,0,2,68,1,0,3,43,0,0,0,0,0,2,0,12,0,2,0,69,0,0,0,65,0,0,0,0,70,0,0,0,0,0,0,0,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.83E2,1.54E2,8.2E1,2.01E2,1.26E2,2.8E1,7.6E1,6E0,1.01E2,1E2,1.21E2,5E0,2.3E1,5E0,7E1,6E0,8.3E1,1.8E1,1.1E1,8.9E1,4.5E1,7.6E1,2E0,3E0,2.2E1,1E0,4E0,6.6E1,6.1E1,2.2E1,3E0,1.5E1,8E0,3E0,7.4E1,1.5E1,6E0,3.9E1,4.5E1,3.1E1,2E0,2E1,2.3E1,3.8E1,6E0,1.6E1,1.1E1,4E0,7E0,1E0,4.3E1,3.1E1,9E0,6E0,2E0,4E0,3E0,3.6E1,3.2E1,1.3E1,4E0,2.7E1,2.1E1,2E0,6E0,1E1,8E0,3E0,3.7E1,6E0,1.9E1,1.2E1,3.5E1,1E0,2E1,1.2E1,1.2E1,1E0,2.5E1,2E0,5E0,1.6E1,2E0,3.5E1,6E0,1.3E1,1.2E1,2.3E1,1.2E1,8E0,3E0,2.2E1,1E1,2.5E1,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[1.2828724E-3,-9.232781E-2,2.5336355E-1,-1.9655019E-1,-3.796487E-2,1.7874552E-1,5.5727774E-1,-2.3793441E-1,-9.008674E-2,-6.471515E-2,1.24768384E-1,1.9975714E-1,-2.5897043E-2,2.9595125E-1,7.916996E-1,-2.5218174E-1,-3.1737727E-3,-1.3411176E-1,-9.4018725E-4,-9.093448E-2,5.529416E-2,-4.184684E-3,1.5204684E-1,2.1040517E-1,-9.131623E-3,1.6911719E-2,-4.271548E-3,1.642885E-2,4.2882215E-2,-2.665827E-1,-5.4499703E-3,-7.0190924E-4,-7.7835997E-3,-1.3793243E-1,-5.1611952E-2,-3.6615107E-2,9.873542E-2,1.0650064E-2,7.4597925E-2,1.4914721E-1,2.8171748E-1,-1.348688E-2,-1.6557266E-3,-2.364135E-1,-1.1788345E-1,8.2582384E-2,-7.196232E-2,2.727557E-3,-3.879863E-3,7.5293533E-3,1.0675424E-3,4.7237566E-3,-4.109812E-3,9.992521E-2,2.0725133E-1,1.7489443E-2,2.1606466E-1,-1.3125584E-2,-9.973045E-6,-1.3885228E-1,-9.734801E-4,-4.5183612E-4,6.8082525E-3,-4.1797828E-2,-1.1078838E-1,-1.6454906E-3,1.1652052E-1,3.5269302E-3,1.125051E-2,4.897515E-3,2.7250084E-1,-4.904379E-4,-1.5090805E-1,-8.380902E-2,6.5043746E-3,-8.0856174E-2,-1.9758618E-1,3.1698558E-3,7.871149E-3,8.35037E-4,1.4471986E-2,-5.5293133E-3,-1.0795636E-2,1.1080916E-3,-5.0830627E-3,2.8360258E-3,-3.0590051E-3,-4.7098952E-3,2.2122338E-3,-1.24630565E-2,-2.2689353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,-1,33,35,-1,37,39,-1,-1,-1,-1,-1,41,-1,-1,-1,43,45,47,49,-1,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,-1,-1,-1,65,67,-1,69,-1,-1,71,-1,-1,-1,73,75,-1,77,-1,-1,-1,79,-1,81,83,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.934319E0,1.7319491E0,2.485621E0,4.4118547E-1,8.9176595E-1,1.4653776E0,1.1134295E0,1.7037058E-1,9.475002E-2,5.556767E-1,1.7767233E-1,4.090538E-1,0E0,2.2319734E-1,5.585003E-2,1.13152504E-1,0E0,4.716721E-2,0E0,2.5900435E-1,1.3138016E-1,0E0,1.0705048E-1,3.467741E-1,0E0,0E0,0E0,0E0,0E0,3.8591385E-2,0E0,0E0,0E0,1.0046971E-1,2.2441514E-1,4.5384172E-2,8.516815E-2,0E0,4.8391476E-2,1.2229049E-1,1.03589535E-1,0E0,0E0,6.1891854E-2,1.1284727E-1,5.5301048E-2,7.735106E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.7772776E-2,3.7836492E-2,0E0,1.2655127E-1,0E0,0E0,7.0079505E-2,0E0,0E0,0E0,8.374367E-2,6.1965466E-2,0E0,4.6576828E-2,0E0,0E0,0E0,6.0685396E-2,0E0,8.621687E-2,4.3992713E-2,7.1516715E-2,4.4371396E-2,4.3854415E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,17,17,19,19,20,20,22,22,23,23,29,29,33,33,34,34,35,35,36,36,38,38,39,39,40,40,43,43,44,44,45,45,46,46,53,53,54,54,56,56,59,59,63,63,64,64,66,66,70,70,72,72,73,73,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,-1,34,36,-1,38,40,-1,-1,-1,-1,-1,42,-1,-1,-1,44,46,48,50,-1,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,-1,-1,-1,66,68,-1,70,-1,-1,72,-1,-1,-1,74,76,-1,78,-1,-1,-1,80,-1,82,84,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,1.074952E6,8.10288E3,3.006846E6,1.2678202E9,5.0328052E7,7.4065337E0,1.7212875E5,7.66951E5,5.3008755E8,1.7102936E6,1.505649E6,-2.5897043E-2,1.354316E6,1E1,1.7626338E6,-3.1737727E-3,2E1,-9.4018725E-4,7.283496E2,1.0643513E3,-4.184684E-3,6.3349745E6,6.0891E4,-9.131623E-3,1.6911719E-2,-4.271548E-3,1.642885E-2,4.2882215E-2,8.332E3,-5.4499703E-3,-7.0190924E-4,-7.7835997E-3,6.506511E0,2.823537E1,1.4010049E6,1.03E2,1.0650064E-2,4.264618E6,1.5271514E5,1.423795E7,-1.348688E-2,-1.6557266E-3,1.154375E6,2.1592189E5,5.0563E4,2.6402641E-2,2.727557E-3,-3.879863E-3,7.5293533E-3,1.0675424E-3,4.7237566E-3,-4.109812E-3,3.7860696E0,8.447218E4,1.7489443E-2,7.6559805E3,-1.3125584E-2,-9.973045E-6,1.4590734E8,-9.734801E-4,-4.5183612E-4,6.8082525E-3,6.746E3,1E0,-1.6454906E-3,2.0663544E9,3.5269302E-3,1.125051E-2,4.897515E-3,1.4817759E9,-4.904379E-4,9E1,4.3712845E0,2.9991518E7,3.53E2,2.55E2,3.1698558E-3,7.871149E-3,8.35037E-4,1.4471986E-2,-5.5293133E-3,-1.0795636E-2,1.1080916E-3,-5.0830627E-3,2.8360258E-3,-3.0590051E-3,-4.7098952E-3,2.2122338E-3,-1.24630565E-2,-2.2689353E-3],"split_indices":[2,1,64,9,7,57,65,40,1,12,57,2,0,2,8,62,0,10,0,64,64,0,57,2,0,0,0,0,0,2,0,0,0,68,68,59,3,0,1,40,57,0,0,9,40,12,69,0,0,0,0,0,0,65,45,0,4,0,0,5,0,0,0,2,19,0,12,0,0,0,7,0,8,70,57,8,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,3.06E2,1.13E2,1.04E2,2.02E2,9.2E1,2.1E1,7.4E1,3E1,1.74E2,2.8E1,9E1,2E0,1.1E1,1E1,6.8E1,6E0,1.8E1,1.2E1,1.43E2,3.1E1,3E0,2.5E1,8.8E1,2E0,1E1,1E0,2E0,8E0,6.1E1,7E0,3E0,1.5E1,6.4E1,7.9E1,1E1,2.1E1,1.3E1,1.2E1,4.9E1,3.9E1,6E1,1E0,9E0,5.5E1,1E1,6.9E1,3E0,7E0,1.2E1,9E0,1.1E1,1E0,2.8E1,2.1E1,1.7E1,2.2E1,8E0,1E0,4.5E1,1E1,4E0,6E0,4E1,2.9E1,3E0,2.5E1,3E0,1.8E1,8E0,1.4E1,4E0,4.1E1,2.1E1,1.9E1,2.3E1,6E0,1.2E1,1.3E1,1E0,1.3E1,2.7E1,1.4E1,3E0,1.8E1,1.1E1,8E0,2.1E1,2E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.2343949E-2,-1.0123858E-1,2.2944875E-1,-1.2859523E-1,4.2503256E-2,1.5653016E-1,4.9960825E-1,-1.9947056E-1,-8.256784E-2,-2.3097374E-2,9.853168E-2,1.8036947E-1,-4.1831186E-1,3.912851E-2,3.274031E-1,-3.314055E-3,-2.1294802E-1,-1.3085209E-1,-5.0173968E-2,1.7185119E-3,-5.494903E-3,1.735962E-1,1.3720532E-4,1.1557563E-1,2.477742E-1,3.7989735E-3,-2.803147E-2,3.901973E-1,-8.967108E-3,-2.298753E-1,-5.9850146E-3,-9.936132E-2,-1.8160433E-1,-7.034077E-2,8.5091375E-2,6.073988E-4,1.0820875E-2,1.4323488E-1,-3.704415E-2,-1.433982E-2,2.6317194E-1,1.3379053E-2,2.4815617E-2,-2.4041462E-1,-2.2379875E-3,-1.4083368E-1,-1.3184419E-3,-2.5154254E-1,-9.515663E-2,-3.1342376E-2,-1.0660568E-1,6.3773454E-3,-1.5914246E-3,1.7366499E-1,8.738497E-2,-5.3142677E-3,1.3430204E-3,5.240277E-3,-9.293712E-3,1.6014807E-1,3.0863643E-1,-1.3157342E-2,-1.6483335E-1,-1.5932426E-1,-1.9343072E-3,-1.3826277E-2,-9.724315E-6,-7.7204104E-3,3.463515E-4,-5.388501E-4,-4.9315607E-3,-1.333763E-1,-3.302618E-2,2.0928809E-1,5.272545E-3,1.1697024E-3,7.447644E-3,-1.957076E-6,9.004388E-3,1.8096011E-2,8.795927E-3,-9.159422E-3,-5.761027E-4,-9.882032E-3,-3.8477043E-3,-2.6160264E-3,-1.5677947E-1,-6.510743E-3,-2.7400258E-5,1.1700692E-2,4.395383E-3,-3.464542E-3,-9.643906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,31,33,-1,-1,35,-1,37,39,-1,-1,41,-1,43,-1,45,47,49,51,-1,-1,53,55,57,59,-1,-1,61,-1,63,-1,65,67,69,71,-1,-1,73,75,-1,-1,-1,-1,77,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0626734E1,1.1183054E0,2.8373446E0,7.599356E-1,1.7134881E-1,1.6925025E0,1.2562141E0,1.5277934E-1,2.2179526E-1,1.13619775E-1,1.7927718E-1,4.6924853E-1,3.9505786E-1,0E0,7.3835945E-1,0E0,1.0497713E-1,7.465273E-2,2.4876572E-1,0E0,0E0,9.4386935E-2,0E0,2.58587E-1,2.2575164E-1,0E0,0E0,1.1458111E-1,0E0,1.2578773E-1,0E0,1.1129719E-1,1.03030026E-1,1.0697201E-1,6.3723646E-2,0E0,0E0,7.1499705E-2,4.7147293E-2,1.01326235E-1,1.9413662E-1,0E0,0E0,7.0628405E-2,0E0,3.9155602E-2,0E0,6.938571E-2,6.769054E-2,5.2929338E-2,7.438594E-2,0E0,0E0,5.562228E-2,7.538524E-2,0E0,0E0,0E0,0E0,5.7749033E-2,1.844275E-1,0E0,4.8780113E-2,4.8026264E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6788156E-2,3.783068E-2,3.9684117E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.6254685E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,17,17,18,18,21,21,23,23,24,24,27,27,29,29,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,43,43,45,45,47,47,48,48,49,49,50,50,53,53,54,54,59,59,60,60,62,62,63,63,71,71,72,72,73,73,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,32,34,-1,-1,36,-1,38,40,-1,-1,42,-1,44,-1,46,48,50,52,-1,-1,54,56,58,60,-1,-1,62,-1,64,-1,66,68,70,72,-1,-1,74,76,-1,-1,-1,-1,78,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.164074E8,7.7101436E3,1.129877E6,2.1530056E3,9.533205E6,3.0418E4,3.5454545E0,7.283496E2,9.481747E8,1.5E2,4.1331047E-1,1.563E3,3.912851E-2,1.852084E7,-3.314055E-3,1.6395413E7,1.7377544E7,7.1999524E9,1.7185119E-3,-5.494903E-3,1.01685606E9,1.3720532E-4,2.5916522E7,2.2782404E0,3.7989735E-3,-2.803147E-2,1.2307312E5,-8.967108E-3,1.7212875E5,-5.9850146E-3,3.345E3,2.56445E7,1.70386E7,1.6991062E3,6.073988E-4,1.0820875E-2,1.921711E5,1.2538845E9,2.203418E7,1.6441814E9,1.3379053E-2,2.4815617E-2,3.006846E6,-2.2379875E-3,5.79E3,-1.3184419E-3,1.5592E4,1E0,9.9378884E-2,1.530655E3,6.3773454E-3,-1.5914246E-3,1.14E2,4.3018994E3,-5.3142677E-3,1.3430204E-3,5.240277E-3,-9.293712E-3,2.8372517E3,9.83854E-1,-1.3157342E-2,2.19546E5,1.926778E6,-1.9343072E-3,-1.3826277E-2,-9.724315E-6,-7.7204104E-3,3.463515E-4,-5.388501E-4,-4.9315607E-3,1.89285E3,6.844E3,4.316273E7,5.272545E-3,1.1697024E-3,7.447644E-3,-1.957076E-6,9.004388E-3,1.8096011E-2,8.795927E-3,-9.159422E-3,-5.761027E-4,-9.882032E-3,-3.8477043E-3,-2.6160264E-3,4.90403E5,-6.510743E-3,-2.7400258E-5,1.1700692E-2,4.395383E-3,-3.464542E-3,-9.643906E-3],"split_indices":[2,7,64,1,4,59,10,70,64,7,3,47,0,0,9,0,12,57,5,0,0,7,0,57,66,0,0,40,0,40,0,10,57,57,45,0,0,45,7,1,7,0,0,9,0,2,0,0,13,69,64,0,0,3,4,0,0,0,0,64,46,0,41,41,0,0,0,0,0,0,0,4,2,1,0,0,0,0,0,0,0,0,0,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,2.82E2,1.47E2,2.37E2,4.5E1,1.17E2,3E1,9.2E1,1.45E2,2.1E1,2.4E1,1.13E2,4E0,1E1,2E1,9E0,8.3E1,5.7E1,8.8E1,1.3E1,8E0,1.3E1,1.1E1,5.9E1,5.4E1,1E0,3E0,1.8E1,2E0,6.9E1,1.4E1,3.7E1,2E1,7.7E1,1.1E1,3E0,1E1,5E1,9E0,3E0,5.1E1,1E1,8E0,6.5E1,4E0,2.3E1,1.4E1,1E1,1E1,3.8E1,3.9E1,8E0,3E0,3.1E1,1.9E1,4E0,5E0,2E0,1E0,1.7E1,3.4E1,4.8E1,1.7E1,1.9E1,4E0,9E0,1E0,6E0,4E0,3E1,8E0,2.8E1,1.1E1,1.9E1,1.2E1,1E1,9E0,2E0,1.5E1,2.3E1,1.1E1,1.5E1,2E0,1.2E1,7E0,7E0,2.1E1,2E0,9E0,1.5E1,4E0,7E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[6.8671815E-4,-1.1392014E-1,1.752277E-1,-1.0517078E-1,-2.7379524E-2,8.527372E-2,3.4911638E-1,-1.38489E-1,1.3830686E-2,4.7940973E-2,9.530251E-3,3.7010703E-2,2.8217924E-1,-1.4519805E-1,3.5941693E-1,-7.2818124E-3,3.163048E-2,9.156724E-3,1.4936809E-1,3.0387342E-1,-8.668348E-3,-2.0726171E-1,-1.0365096E-1,2.316739E-2,3.7759875E-3,-3.2205682E-2,9.4295144E-2,-2.772771E-2,5.2453587E-3,7.396025E-2,1.0369311E-2,2.5295886E-1,3.8033488E-1,-2.2931817E-1,-1.2437948E-1,-1.16077416E-1,1.6704737E-3,-5.790598E-2,3.3682056E-3,1.0766968E-1,-6.3330387E-3,-6.5123383E-3,-1.2413044E-2,6.0656457E-3,3.1590098E-4,2.1158643E-3,2.6518163E-1,3.9501175E-1,3.2691362E-3,-2.4025877E-1,-2.8258248E-3,-2.2461326E-3,-7.875466E-3,-1.4267032E-1,-3.4130048E-2,2.4808235E-3,-5.064492E-3,1.2568059E-2,7.797764E-2,2.381576E-4,-5.8111264E-3,5.4056407E-4,2.732999E-1,3.967589E-3,2.0479014E-2,-1.5862577E-2,-1.0293394E-2,-1.02868654E-1,-1.828753E-1,2.0434083E-2,-5.140428E-3,7.79772E-3,1.2967946E-3,1.4398391E-2,5.2493038E-3,-5.712038E-3,5.6995533E-4,-9.605883E-3,-2.8544913E-5,-3.4411408E-3,3.7680592E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,29,31,-1,33,35,-1,-1,37,39,41,-1,43,-1,45,47,49,51,53,-1,55,-1,57,-1,-1,59,-1,-1,-1,61,63,-1,65,-1,-1,-1,67,69,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,-1,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.781535E0,9.484887E-1,2.6919336E0,1.0367582E0,0E0,4.4740057E-1,1.3369851E0,7.336445E-1,1.6931248E-1,3.4087068E-1,0E0,0E0,5.6670046E-1,4.9938726E-1,5.3669184E-2,0E0,2.1505125E-1,2.2905762E-1,8.983445E-2,9.84416E-2,0E0,1.1772442E-1,2.13395E-1,0E0,0E0,7.301731E-2,9.3455866E-2,7.221945E-2,0E0,3.7714683E-2,0E0,7.372713E-2,5.735755E-2,1.0361099E-1,4.3033153E-2,2.4138224E-1,0E0,1.0779208E-1,0E0,9.141177E-2,0E0,0E0,7.549237E-2,0E0,0E0,0E0,6.105876E-2,4.668808E-2,0E0,8.8063E-2,0E0,0E0,0E0,1.1720693E-1,1.1085376E-1,0E0,0E0,0E0,8.935343E-2,0E0,0E0,0E0,4.447651E-2,0E0,0E0,0E0,0E0,6.0063362E-2,6.9658995E-2,8.862161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,39,39,42,42,46,46,47,47,49,49,53,53,54,54,58,58,62,62,67,67,68,68,69,69],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,30,32,-1,34,36,-1,-1,38,40,42,-1,44,-1,46,48,50,52,54,-1,56,-1,58,-1,-1,60,-1,-1,-1,62,64,-1,66,-1,-1,-1,68,70,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,-1,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,6.0891E4,7.164074E8,-2.7379524E-2,1.5271514E5,3.0418E4,5.2191963E3,2.9648642E1,1.29186E9,9.530251E-3,3.7010703E-2,5.22E2,9.50639E5,1.906E3,-7.2818124E-3,1.577058E9,2.143564E3,1.0339844E-1,1.6846506E5,-8.668348E-3,3.006846E6,2.889654E5,2.316739E-2,3.7759875E-3,3.0251338E5,6.6053805E6,1.7895421E0,5.2453587E-3,1.6255458E5,1.0369311E-2,1.7648785E-1,5.88824E7,1.6120845E5,1.7062635E3,1.0623282E3,1.6704737E-3,8.110162E0,3.3682056E-3,2.412E4,-6.3330387E-3,-6.5123383E-3,6.855442E1,6.0656457E-3,3.1590098E-4,2.1158643E-3,1.926778E6,1.1100566E3,3.2691362E-3,3.586E3,-2.8258248E-3,-2.2461326E-3,-7.875466E-3,1.7377544E7,6.5E1,2.4808235E-3,-5.064492E-3,1.2568059E-2,3.5546432E6,2.381576E-4,-5.8111264E-3,5.4056407E-4,4.242988E2,3.967589E-3,2.0479014E-2,-1.5862577E-2,-1.0293394E-2,3.8E2,2.5252392E6,6.12E2,-5.140428E-3,7.79772E-3,1.2967946E-3,1.4398391E-2,5.2493038E-3,-5.712038E-3,5.6995533E-4,-9.605883E-3,-2.8544913E-5,-3.4411408E-3,3.7680592E-3],"split_indices":[45,59,2,7,0,40,10,4,68,7,0,0,3,1,0,0,7,64,50,40,0,9,40,0,0,59,59,65,0,45,0,53,9,40,59,64,0,70,0,9,0,0,68,0,0,0,41,4,0,2,0,0,0,57,8,0,0,0,57,0,0,0,69,0,0,0,0,3,60,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.64E2,1.73E2,2.6E2,4E0,1.15E2,5.8E1,2.03E2,5.7E1,8.6E1,2.9E1,7E0,5.1E1,2.01E2,2E0,5E0,5.2E1,6.3E1,2.3E1,4.9E1,2E0,7.9E1,1.22E2,1E0,1E0,2.6E1,2.6E1,4.6E1,1.7E1,1.1E1,1.2E1,3.2E1,1.7E1,6.1E1,1.8E1,1.12E2,1E1,2.1E1,5E0,2.5E1,1E0,5E0,4.1E1,6E0,5E0,2E0,3E1,1.6E1,1E0,5.7E1,4E0,6E0,1.2E1,8.4E1,2.8E1,6E0,1.5E1,3E0,2.2E1,3.6E1,5E0,1E0,2.9E1,1E0,1.5E1,1.5E1,4.2E1,4.4E1,4E1,1.6E1,1.2E1,8E0,1.4E1,2.6E1,3E0,4E1,4E0,3.8E1,2E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[6.4035733E-3,-8.937738E-2,1.9856532E-1,-1.9758534E-1,-5.7276003E-2,1.3598657E-1,4.528227E-1,-1.0094103E-2,2.418048E-3,-9.565375E-3,-1.0250067E-1,1.559042E-1,-2.1065827E-1,3.6591727E-2,2.905005E-1,-3.331173E-2,1.0370807E-1,-1.303187E-1,-4.6400063E-2,1.0148708E-1,2.4140713E-1,1.7749779E-1,-2.0061411E-2,3.507235E-1,-8.37954E-3,-6.774013E-2,2.248793E-2,-7.109631E-4,1.3408074E-1,-1.06043324E-1,-1.8153128E-1,-8.55397E-2,8.363892E-3,1.3143325E-1,1.7055387E-2,2.5595507E-1,1.5961862E-3,1.3813749E-2,-5.077823E-4,3.2630486E-3,1.910694E-2,-4.1275444E-3,6.6394237E-4,4.782669E-2,-4.6447343E-3,-5.407716E-4,7.7739717E-3,-1.2945457E-1,-9.414223E-3,-2.2072856E-1,-4.5816624E-3,7.847245E-4,-5.647731E-3,-3.024223E-3,1.974557E-3,5.9647363E-2,1.5820011E-1,-9.319103E-3,6.394075E-2,1.7609752E-3,2.6637048E-1,1.8049946E-2,6.555792E-3,-1.1079848E-1,-1.2821042E-2,-4.312158E-3,2.8895882E-3,-1.1715736E-2,-3.811696E-4,-1.7549363E-3,5.293153E-3,2.0451404E-3,1.7079389E-1,-2.3967717E-3,4.1009304E-3,1.5632087E-2,9.737114E-3,1.4893975E-3,-6.303284E-3,-1.2120445E-2,-9.000306E-2,1.1207155E-2,1.2216185E-1,-3.0160283E-3,-9.493145E-3,3.0382609E-3,9.516032E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,39,-1,41,43,-1,45,47,49,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,63,65,67,-1,-1,-1,-1,-1,69,71,-1,73,-1,75,77,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,81,-1,-1,-1,-1,-1,-1,-1,83,-1,85,-1,-1,-1,-1],"loss_changes":[8.244977E0,1.027297E0,2.309177E0,8.325529E-2,4.9960518E-1,8.68268E-1,1.1023536E0,0E0,0E0,3.100955E-1,1.7966568E-1,5.105748E-1,5.895691E-1,0E0,6.1065555E-1,1.8402717E-1,7.3546946E-2,7.896292E-2,8.859652E-2,1.7980355E-1,1.2590742E-1,5.8491938E-2,0E0,1.3738108E-1,0E0,7.208881E-2,1.126618E-1,0E0,5.1466852E-2,1.2546498E-1,6.4293206E-2,6.8518594E-2,4.100561E-2,9.24024E-2,2.0277151E-1,8.4867954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.4572615E-2,0E0,0E0,0E0,7.826954E-2,6.700808E-2,4.491806E-2,0E0,0E0,0E0,0E0,0E0,7.386856E-2,4.9135685E-2,0E0,3.839927E-2,0E0,6.856489E-2,4.4424273E-2,0E0,9.0483546E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.0994864E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.820071E-2,0E0,7.219058E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,43,43,47,47,48,48,49,49,55,55,56,56,58,58,60,60,61,61,63,63,72,72,80,80,82,82],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,40,-1,42,44,-1,46,48,50,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,64,66,68,-1,-1,-1,-1,-1,70,72,-1,74,-1,76,78,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,82,-1,-1,-1,-1,-1,-1,-1,84,-1,86,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.41718E5,7.7101436E3,2.91E2,1.3026638E7,2.3028242E3,1.681178E6,-1.0094103E-2,2.418048E-3,3.23643E6,7.198E3,6.050605E9,2.909019E3,3.6591727E-2,1.5096262E7,7.164074E8,3.3E1,2.7075084E5,1.4253456E-2,1.9196308E7,9.677921E7,8.494E3,-2.0061411E-2,1.7895421E0,-8.37954E-3,3.4015296E8,2.39E2,-7.109631E-4,5.66936E6,4.5474697E3,2.946708E6,1.572445E3,1.1172539E9,1.5704E4,1.2587E4,8.946923E2,1.5961862E-3,1.3813749E-2,-5.077823E-4,3.2630486E-3,1.910694E-2,-4.1275444E-3,6.6394237E-4,1.5522031E9,-4.6447343E-3,-5.407716E-4,7.7739717E-3,3.9034148E7,8.7182234E4,6.4065235E8,-4.5816624E-3,7.847245E-4,-5.647731E-3,-3.024223E-3,1.974557E-3,3.253389E8,1.1E1,-9.319103E-3,2.1853803E4,1.7609752E-3,8.080874E-1,1.007452E6,6.555792E-3,9.687575E2,-1.2821042E-2,-4.312158E-3,2.8895882E-3,-1.1715736E-2,-3.811696E-4,-1.7549363E-3,5.293153E-3,2.0451404E-3,7.8E1,-2.3967717E-3,4.1009304E-3,1.5632087E-2,9.737114E-3,1.4893975E-3,-6.303284E-3,-1.2120445E-2,1.5362013E3,1.1207155E-2,2.5096E4,-3.0160283E-3,-9.493145E-3,3.0382609E-3,9.516032E-3],"split_indices":[2,1,64,8,57,70,41,0,0,1,2,12,44,0,9,7,8,60,50,57,1,0,0,65,0,44,3,0,57,59,1,59,5,2,2,64,0,0,0,0,0,0,0,7,0,0,0,57,40,7,0,0,0,0,0,7,8,0,45,0,46,41,0,4,0,0,0,0,0,0,0,0,3,0,0,0,0,0,0,0,59,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,2.98E2,1.48E2,6.7E1,2.31E2,1.2E2,2.8E1,6.6E1,1E0,1.13E2,1.18E2,1.14E2,6E0,9E0,1.9E1,9.4E1,1.9E1,7.8E1,4E1,7.1E1,4.3E1,2E0,4E0,1.7E1,2E0,5.8E1,3.6E1,4E0,1.5E1,5.5E1,2.3E1,2.3E1,1.7E1,5.2E1,1.9E1,4E1,3E0,1E0,1E0,2E0,1.5E1,4.9E1,9E0,3E1,6E0,2E0,1.3E1,4.4E1,1.1E1,1.5E1,8E0,5E0,1.8E1,5E0,1.2E1,1.5E1,3.7E1,3E0,1.6E1,2E0,3.8E1,2.3E1,7E0,4E1,4E0,5E0,6E0,1.4E1,1E0,5E0,1E1,4E0,3.3E1,2E0,1.4E1,2.1E1,1.7E1,2.2E1,1E0,4E0,3.6E1,1.4E1,1.9E1,2.9E1,7E0,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[1.4494569E-3,-5.165316E-2,2.869512E-1,-1.11480385E-1,6.841784E-2,5.767569E-1,2.2805607E-1,-1.03099346E-1,-2.4897022E-2,5.59849E-3,1.825076E-1,8.92731E-3,3.516941E-2,2.8314754E-1,3.0751565E-2,-1.2607013E-1,-6.675537E-3,-2.1473894E-2,8.9871725E-3,5.681992E-3,2.5939894E-1,-3.2622525E-3,3.0046853E-1,6.0253954E-3,-1.1190888E-2,-1.063556E-2,-1.04680754E-1,4.7284206E-3,-3.247596E-2,-5.905905E-2,3.4996297E-2,3.007514E-1,6.9785137E-3,7.0875264E-3,3.4295923E-1,-1.0830692E-1,1.1791333E-2,-7.7164926E-2,1.8626368E-2,-3.819994E-2,-1.0492394E-2,5.9445333E-3,1.2550305E-2,3.9323866E-3,1.6552342E-2,3.6996064E-1,2.3967717E-3,-1.2640475E-1,-2.6272746E-2,-6.0842456E-3,-2.0991419E-4,-2.9134548E-3,3.4161853E-3,-5.1620933E-3,-6.093524E-4,2.153887E-3,-2.1565587E-3,3.4100902E-3,1.887815E-2,7.6157926E-4,-1.3458966E-1,-1.1289586E-2,4.274695E-3,-9.945303E-3,-5.384774E-3,-1.2942656E-3,7.107238E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,25,27,29,-1,-1,31,-1,33,-1,-1,-1,35,-1,37,39,41,43,-1,-1,45,47,-1,49,51,53,-1,-1,55,-1,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,-1,-1,-1,-1],"loss_changes":[6.670453E0,2.6727111E0,1.0461688E0,7.523892E-1,8.8769186E-1,4.3033338E-1,6.326108E-1,5.396011E-1,0E0,3.859263E-1,2.033186E-1,0E0,0E0,2.9722548E-1,3.4722677E-1,3.4323192E-1,1.2851128E-1,1.5276015E-1,0E0,0E0,5.705273E-2,0E0,2.4797297E-1,0E0,0E0,0E0,2.2317696E-1,0E0,9.0501994E-2,1.2731887E-1,5.315389E-2,6.861782E-2,0E0,0E0,2.5308228E-1,2.3273659E-1,0E0,6.779395E-2,7.666758E-2,6.476112E-2,0E0,0E0,4.2697962E-2,0E0,0E0,4.7023773E-2,0E0,1.5276551E-1,1.8390852E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9122243E-1,0E0,1.16054885E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,17,17,20,20,22,22,26,26,28,28,29,29,30,30,31,31,34,34,35,35,37,37,38,38,39,39,42,42,45,45,47,47,48,48,60,60,62,62],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,26,28,30,-1,-1,32,-1,34,-1,-1,-1,36,-1,38,40,42,44,-1,-1,46,48,-1,50,52,54,-1,-1,56,-1,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,-1,-1,-1,-1],"split_conditions":[3.2991206E3,8.393928E4,1.064816E6,2.7148398E7,1.484798E9,9.256843E3,1.5096262E7,7.164074E8,-2.4897022E-2,1.7670108E5,4.9014646E9,8.92731E-3,3.516941E-2,2.4950776E0,1.92144E5,5.46223E5,2.0718214E-4,1.7657828E3,8.9871725E-3,5.681992E-3,1.5029658E-1,-3.2622525E-3,2.1360708E7,6.0253954E-3,-1.1190888E-2,-1.063556E-2,1.3428E4,4.7284206E-3,2.943086E4,1E0,7.9E1,1.5200746E-1,6.9785137E-3,7.0875264E-3,3.020675E5,2.0964778E5,1.1791333E-2,2.579534E7,1.9252769E5,4.39E3,-1.0492394E-2,5.9445333E-3,5.792948E8,3.9323866E-3,1.6552342E-2,3.1095595E0,2.3967717E-3,1.0640107E8,2.4094029E2,-6.0842456E-3,-2.0991419E-4,-2.9134548E-3,3.4161853E-3,-5.1620933E-3,-6.093524E-4,2.153887E-3,-2.1565587E-3,3.4100902E-3,1.887815E-2,7.6157926E-4,4.822E3,-1.1289586E-2,4.735985E6,-9.945303E-3,-5.384774E-3,-1.2942656E-3,7.107238E-3],"split_indices":[64,45,41,60,7,4,9,7,0,40,12,0,0,66,2,1,50,64,0,0,50,0,1,0,0,0,2,0,45,19,3,53,0,0,45,40,0,9,59,2,0,0,44,0,0,66,0,5,64,0,0,0,0,0,0,0,0,0,0,0,2,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,3.7E2,6.8E1,2.47E2,1.23E2,1E1,5.8E1,2.43E2,4E0,8E1,4.3E1,3E0,7E0,4.5E1,1.3E1,1.96E2,4.7E1,7E1,1E1,2.4E1,1.9E1,2E0,4.3E1,1E1,3E0,3.7E1,1.59E2,9E0,3.8E1,4.2E1,2.8E1,1.3E1,6E0,1E1,3.3E1,1.58E2,1E0,2E1,1.8E1,3.8E1,4E0,5E0,2.3E1,2E0,1.1E1,3E1,3E0,1.29E2,2.9E1,1.2E1,8E0,7E0,1.1E1,1E1,2.8E1,1.5E1,8E0,1E0,2.9E1,7E0,1.22E2,3E0,2.6E1,3.4E1,8.8E1,2.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[3.603184E-3,-7.994202E-2,1.6792168E-1,-1.54378E-1,-4.2881798E-2,1.9374241E-1,-1.806661E-1,-1.7485082E-1,-2.8073438E-2,3.1336157E-3,-8.669388E-2,1.6879725E-1,2.529483E-2,1.1698272E-2,-2.5998974E-2,-1.1904041E-2,-1.5551198E-1,3.546691E-3,-2.935981E-3,-1.1669727E-2,6.7992634E-3,-1.3551107E-1,-5.8441192E-2,1.0471378E-1,2.3036137E-1,-8.466878E-3,7.7089164E-3,-9.7586575E-4,-1.6370595E-1,-4.056159E-2,5.0501492E-2,-7.6681904E-2,-1.7213634E-1,-8.078895E-2,3.132184E-3,6.619255E-2,1.7097296E-1,3.0366208E-2,1.967641E-1,-8.513913E-3,-2.5547482E-3,-6.208802E-2,7.748312E-3,-5.214544E-4,1.00663766E-1,-9.193193E-2,3.7998955E-3,-7.33133E-4,-1.8714951E-1,9.985294E-4,-8.877114E-2,1.4587475E-1,3.5211366E-2,7.131661E-5,9.311612E-3,2.2148854E-1,-1.343371E-1,-1.08217E-1,-1.0525064E-3,3.125872E-3,-2.1376465E-3,7.421694E-3,1.7121563E-3,-5.639626E-3,1.4663892E-3,-2.1417004E-1,-3.1611836E-3,-3.2876777E-3,-1.37815E-1,1.1822629E-2,2.3638883E-3,-3.8596846E-3,6.305887E-2,2.1970807E-2,2.019521E-1,4.758754E-3,-1.077143E-2,-7.684168E-3,-1.0338827E-3,-1.2652668E-2,-7.772546E-2,-1.6754599E-1,2.3422774E-3,-1.552597E-3,8.0779284E-2,4.228538E-3,2.1434079E-1,1.9372143E-3,-8.411238E-3,-1.0653637E-2,-3.557226E-3,6.441184E-3,2.0314215E-3,6.4416993E-3,1.2038158E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,-1,27,-1,-1,29,-1,31,33,35,37,-1,-1,-1,39,41,43,45,47,49,-1,51,53,-1,55,-1,-1,57,59,-1,61,63,-1,-1,65,-1,67,69,71,-1,-1,73,75,77,-1,-1,-1,-1,-1,-1,-1,79,-1,-1,81,-1,-1,-1,83,-1,85,-1,-1,-1,-1,-1,87,89,-1,-1,91,-1,93,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.191034E0,8.182055E-1,1.4031229E0,2.510867E-1,4.054939E-1,1.0211396E0,7.246021E-1,6.708264E-2,4.9739458E-2,1.9638124E-1,1.3413996E-1,5.005951E-1,0E0,2.3268442E-1,0E0,0E0,7.2573066E-2,0E0,0E0,1.6371024E-1,0E0,6.6521704E-2,1.8640852E-1,1.596927E-1,7.296479E-1,0E0,0E0,0E0,3.9235353E-2,6.495734E-2,8.969367E-2,4.4252798E-2,4.9264193E-2,4.7622174E-2,0E0,1.0348414E-1,6.2188327E-2,0E0,5.454905E-1,0E0,0E0,7.91015E-2,5.8046184E-2,0E0,4.574199E-2,4.135695E-2,0E0,0E0,5.340737E-2,0E0,5.0682217E-2,9.608799E-2,1.0820064E-1,0E0,0E0,1.6265297E-1,1.13717295E-1,7.654081E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.547296E-2,0E0,0E0,9.570548E-2,0E0,0E0,0E0,4.7544077E-2,0E0,6.327152E-2,0E0,0E0,0E0,0E0,0E0,5.9281416E-2,4.6386242E-2,0E0,0E0,3.916405E-2,0E0,8.232713E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,16,16,19,19,21,21,22,22,23,23,24,24,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,38,38,41,41,42,42,44,44,45,45,48,48,50,50,51,51,52,52,55,55,56,56,57,57,65,65,68,68,72,72,74,74,80,80,81,81,84,84,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,-1,28,-1,-1,30,-1,32,34,36,38,-1,-1,-1,40,42,44,46,48,50,-1,52,54,-1,56,-1,-1,58,60,-1,62,64,-1,-1,66,-1,68,70,72,-1,-1,74,76,78,-1,-1,-1,-1,-1,-1,-1,80,-1,-1,82,-1,-1,-1,84,-1,86,-1,-1,-1,-1,-1,88,90,-1,-1,92,-1,94,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,9.50639E5,1.0429407E3,1.2254369E7,1.3508742E7,1.6828056E1,6.6053805E6,1.44E2,2.409091E0,1.577058E9,7.121916E2,1.2511627E7,2.529483E-2,1.4414646E-1,-2.5998974E-2,-1.1904041E-2,2E1,3.546691E-3,-2.935981E-3,2.455358E6,6.7992634E-3,1.7377544E7,7.1999524E9,1.5271514E5,1.064816E6,-8.466878E-3,7.7089164E-3,-9.7586575E-4,1.6120845E5,2.2582366E3,1E0,4.94E2,2E1,7.0934258E0,3.132184E-3,5.65E2,1.9937408E-1,3.0366208E-2,5.22E2,-8.513913E-3,-2.5547482E-3,1E0,1.52E2,-5.214544E-4,1E0,6.24E3,3.7998955E-3,-7.33133E-4,3.7075245E1,9.985294E-4,7.313879E6,1.93E2,7.10711E8,7.131661E-5,9.311612E-3,3.7137297E3,1.3590209E7,6.927258E4,-1.0525064E-3,3.125872E-3,-2.1376465E-3,7.421694E-3,1.7121563E-3,-5.639626E-3,1.4663892E-3,6.842986E6,-3.1611836E-3,-3.2876777E-3,6.33E2,1.1822629E-2,2.3638883E-3,-3.8596846E-3,2.8529238E1,2.1970807E-2,2.4547362E0,4.758754E-3,-1.077143E-2,-7.684168E-3,-1.0338827E-3,-1.2652668E-2,2.2967976E7,7.567335E7,2.3422774E-3,-1.552597E-3,2.5871767E-2,4.228538E-3,6.7304E4,1.9372143E-3,-8.411238E-3,-1.0653637E-2,-3.557226E-3,6.441184E-3,2.0314215E-3,6.4416993E-3,1.2038158E-2],"split_indices":[2,1,70,9,57,47,59,0,70,7,64,1,0,50,0,0,10,0,0,1,0,57,5,40,41,0,0,0,40,4,13,3,3,59,0,0,53,0,3,0,0,25,3,0,26,2,0,0,68,0,9,3,7,0,0,44,1,45,0,0,0,0,0,0,0,41,0,0,8,0,0,0,68,0,65,0,0,0,0,0,41,44,0,0,69,0,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.49E2,2.98E2,1.51E2,9.8E1,2E2,1.41E2,1E1,8.4E1,1.4E1,9.8E1,1.02E2,1.32E2,9E0,7E0,3E0,1.7E1,6.7E1,3E0,1.1E1,8.9E1,9E0,3.6E1,6.6E1,6.6E1,6.6E1,3E0,4E0,4E0,6.3E1,6.1E1,2.8E1,1.5E1,2.1E1,5.6E1,1E1,4.3E1,2.3E1,4E0,6.2E1,5.9E1,4E0,4.2E1,1.9E1,1.3E1,1.5E1,1.4E1,1E0,2E0,1.9E1,4E0,5.2E1,1.1E1,3.2E1,2E0,2.1E1,5.8E1,4E0,1.9E1,2.3E1,9E0,1E1,8E0,7E0,1.2E1,2E0,1.5E1,4E0,3.7E1,1.5E1,5E0,6E0,6E0,2.6E1,3E0,5.5E1,1E0,3E0,1.2E1,7E0,1.1E1,4E0,1.3E1,2E0,4E0,2.2E1,6E0,4.9E1,2E0,2E0,8E0,5E0,9E0,1.3E1,1.3E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[4.52243E-3,-7.515865E-2,1.6385403E-1,-9.711443E-2,1.4682604E-2,-2.3352416E-1,1.8822938E-1,-1.4995912E-1,-6.338092E-2,-8.357022E-3,6.4175525E-3,5.008996E-3,-4.171048E-1,1.4523996E-1,4.005999E-1,-1.8921623E-1,-5.498939E-3,-2.3654286E-2,-1.1900119E-1,-2.5664924E-2,1.22845076E-1,-2.502448E-2,-8.324558E-3,9.591124E-2,2.1771716E-1,3.0360255E-2,2.4975577E-1,-1.08294785E-2,-5.6421114E-3,-4.4695266E-2,5.400266E-2,-2.0094751E-1,-7.714576E-2,2.068826E-3,-4.128347E-2,9.1205565E-3,1.17889256E-4,7.062934E-2,7.901855E-3,1.8485922E-2,1.9495882E-1,1.4528658E-2,-8.086992E-3,-6.92315E-2,4.194826E-3,-1.9041717E-3,5.5426136E-3,-1.11552946E-1,-1.2556382E-2,-6.7922415E-4,-1.0516718E-1,-3.6953972E-3,-1.3499704E-4,3.5845558E-3,9.460479E-2,2.4139805E-1,7.2006206E-3,-8.885009E-2,-3.1191792E-4,3.4691186E-3,-1.3365628E-3,-7.902519E-3,1.7142725E-3,-3.662599E-3,-9.739957E-3,-3.2665918E-3,2.2286682E-3,1.593681E-1,4.748738E-2,1.7795531E-3,1.3053055E-2,-4.7815223E-3,3.5792359E-3,2.0177453E-3,1.0484275E-2,1.4844714E-4,4.9890312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,-1,41,-1,-1,43,45,47,49,-1,51,-1,-1,53,-1,-1,55,-1,-1,57,59,-1,-1,61,-1,-1,63,-1,-1,65,67,69,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.636516E0,5.8529615E-1,1.4775188E0,4.1476274E-1,1.5567048E-1,5.9323925E-1,1.2199726E0,1.2338543E-1,3.2199717E-1,1.1669446E-1,0E0,0E0,4.2297482E-2,4.0099287E-1,5.660815E-1,6.9903135E-2,0E0,1.4456978E-1,1.9372165E-1,4.8841946E-2,4.2693913E-2,0E0,0E0,1.02057815E-1,9.0661764E-2,0E0,2.9989076E-1,0E0,0E0,8.306135E-2,1.03804305E-1,6.1904013E-2,7.3368266E-2,0E0,4.6423584E-2,0E0,0E0,8.48546E-2,0E0,0E0,6.6815734E-2,0E0,0E0,5.6291252E-2,5.0285008E-2,0E0,0E0,6.6560164E-2,0E0,0E0,6.8670034E-2,0E0,0E0,4.5328137E-2,1.11439705E-1,7.621539E-2,0E0,4.5190543E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.752051E-2,5.5625033E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,23,23,24,24,26,26,29,29,30,30,31,31,32,32,34,34,37,37,40,40,43,43,44,44,47,47,50,50,53,53,54,54,55,55,57,57,67,67,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,-1,42,-1,-1,44,46,48,50,-1,52,-1,-1,54,-1,-1,56,-1,-1,58,60,-1,-1,62,-1,-1,64,-1,-1,66,68,70,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,2.4950776E0,1.129877E6,1.6219076E9,2.9632012E7,7.7101436E3,4.90403E5,1.7377544E7,1E0,6.4175525E-3,5.008996E-3,3.1E1,6.0891E4,3.3616E4,1.10388414E5,-5.498939E-3,1.272E4,7.121916E2,1.5124741E1,1.21E2,-2.502448E-2,-8.324558E-3,1.5271514E5,1.3812E4,3.0360255E-2,7.79557E5,-1.08294785E-2,-5.6421114E-3,1.0220021E3,1.8E9,3.8E1,2.0839167E3,2.068826E-3,3.0251338E5,9.1205565E-3,1.17889256E-4,5.2798065E1,7.901855E-3,1.8485922E-2,4.917967E7,1.4528658E-2,-8.086992E-3,2.1592189E5,3.7755436E1,-1.9041717E-3,5.5426136E-3,1.5E1,-1.2556382E-2,-6.7922415E-4,6.54925E6,-3.6953972E-3,-1.3499704E-4,5.795E3,1.16E2,8.825362E-2,7.2006206E-3,2.1603052E9,-3.1191792E-4,3.4691186E-3,-1.3365628E-3,-7.902519E-3,1.7142725E-3,-3.662599E-3,-9.739957E-3,-3.2665918E-3,2.2286682E-3,4.1E1,2.6196917E3,1.7795531E-3,1.3053055E-2,-4.7815223E-3,3.5792359E-3,2.0177453E-3,1.0484275E-2,1.4844714E-4,4.9890312E-3],"split_indices":[2,7,66,1,7,1,64,9,57,101,0,0,3,2,10,40,0,10,64,70,3,0,0,40,9,0,2,0,0,64,5,8,4,0,59,0,0,68,0,0,56,0,0,40,70,0,0,8,0,0,9,0,0,10,3,51,0,12,0,0,0,0,0,0,0,0,0,3,64,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,2.95E2,1.47E2,2.37E2,5.8E1,8E0,1.39E2,9.1E1,1.46E2,4.9E1,9E0,3E0,5E0,1.17E2,2.2E1,4.4E1,4.7E1,8.6E1,6E1,4.4E1,5E0,3E0,2E0,7.1E1,4.6E1,8E0,1.4E1,3.1E1,1.3E1,6.8E1,1.8E1,1.9E1,4.1E1,8E0,3.6E1,3E0,2E0,5.2E1,1.9E1,4E0,4.2E1,1.3E1,1E0,4.5E1,2.3E1,7E0,1.1E1,8E0,1.1E1,1.3E1,2.8E1,1.9E1,1.7E1,1.4E1,3.8E1,2E1,2.2E1,3.4E1,1.1E1,7E0,1.6E1,6E0,2E0,2.2E1,6E0,5E0,9E0,1.5E1,2.3E1,2E0,1.8E1,3.3E1,1E0,5E0,1E1,1.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[1.7336373E-4,-6.1993726E-2,1.7657764E-1,-9.623901E-2,2.3195093E-2,1.431533E-1,3.4052044E-2,-9.99158E-2,1.9277878E-2,-2.4719685E-2,1.07409984E-1,2.079739E-2,1.8223439E-1,-9.505005E-2,-2.229628E-2,-1.03549E-1,-1.6925395E-3,7.5254175E-3,6.750316E-2,5.845359E-2,-1.6411185E-1,1.2184009E-1,2.1892405E-1,-1.7511977E-1,-7.558591E-2,-3.3454841E-3,-1.076794E-2,-9.022027E-4,4.2184717E-3,4.648295E-3,-3.7559227E-4,6.9758315E-3,-2.093189E-2,-1.0690592E-2,8.770143E-4,3.5932206E-3,9.010977E-3,2.7140152E-1,1.6975486E-1,-1.1493872E-2,-6.098388E-3,-8.864319E-2,-2.99231E-3,-8.7407485E-2,3.5578418E-3,1.4016579E-2,1.7988716E-3,6.4493595E-3,1.1703963E-2,-9.193334E-2,1.0939286E-2,7.260179E-2,-5.057846E-2,-7.7335215E-3,9.303342E-4,-1.0526128E-1,-3.0670233E-2,5.2850293E-3,-1.0062566E-3,-7.892783E-3,-1.4322475E-3,-4.3181903E-4,-5.6560463E-3,-4.7149705E-3,2.9406662E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,-1,25,27,-1,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,-1,-1,45,47,-1,-1,49,51,53,-1,-1,-1,-1,-1,55,-1,57,59,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.923957E0,9.7078085E-1,1.8765128E0,4.5853996E-1,3.9002836E-1,5.269902E-1,0E0,3.5516834E-1,0E0,1.1236802E-1,5.013314E-2,2.0455636E-1,1.594572E-1,3.5144973E-1,0E0,4.2932212E-2,7.054754E-2,0E0,3.8969353E-2,1.5749443E-1,4.8989475E-2,8.631122E-2,8.5522175E-2,1.0302615E-1,1.7983091E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.583353E-2,0E0,0E0,0E0,0E0,3.8981557E-2,4.5843363E-2,0E0,0E0,1.8302822E-1,1.1158905E-1,5.999483E-2,0E0,0E0,0E0,0E0,0E0,1.2741339E-1,0E0,3.9019175E-2,4.018377E-2,0E0,0E0,9.91894E-2,7.038238E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,32,32,37,37,38,38,41,41,42,42,43,43,49,49,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,-1,26,28,-1,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,-1,-1,46,48,-1,-1,50,52,54,-1,-1,-1,-1,-1,56,-1,58,60,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.484798E9,9.7467445E4,1.5324995E4,3.723672E3,2.1695537E3,8.447218E4,3.4052044E-2,5.38168E6,1.9277878E-2,3.0192542E-1,1.3058374E7,1.505649E6,5.3515047E-1,4.66085E5,-2.229628E-2,1.153215E9,1.5271514E5,7.5254175E-3,1.2753316E9,3.798313E3,2.412E4,7.3036104E-1,1.7176836E7,3.606E3,4.723148E11,-3.3454841E-3,-1.076794E-2,-9.022027E-4,4.2184717E-3,4.648295E-3,-3.7559227E-4,6.9758315E-3,1.173433E5,-1.0690592E-2,8.770143E-4,3.5932206E-3,9.010977E-3,6.520156E7,7.309061E1,-1.1493872E-2,-6.098388E-3,1.3664E4,1.4399977E10,1.6167939E7,3.5578418E-3,1.4016579E-2,1.7988716E-3,6.4493595E-3,1.1703963E-2,6.4065235E8,1.0939286E-2,2.58E2,4.9256845E2,-7.7335215E-3,9.303342E-4,1.0640107E8,3.738719E4,5.2850293E-3,-1.0062566E-3,-7.892783E-3,-1.4322475E-3,-4.3181903E-4,-5.6560463E-3,-4.7149705E-3,2.9406662E-4],"split_indices":[7,45,64,64,64,45,0,59,0,54,57,2,47,1,0,7,40,0,7,44,9,51,55,2,43,0,0,0,0,0,0,0,40,0,0,0,0,1,68,0,0,2,5,44,0,0,0,0,0,7,0,3,45,0,0,5,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.31E2,1.16E2,2.36E2,9.5E1,1.1E2,6E0,2.35E2,1E0,6.1E1,3.4E1,2.7E1,8.3E1,2.33E2,2E0,1.3E1,4.8E1,1.5E1,1.9E1,2.3E1,4E0,3.3E1,5E1,4.4E1,1.89E2,1.1E1,2E0,4.1E1,7E0,1.4E1,5E0,1.1E1,1.2E1,3E0,1E0,1.9E1,1.4E1,2.2E1,2.8E1,2E1,2.4E1,1.6E2,2.9E1,7E0,5E0,2.1E1,1E0,1.9E1,9E0,1.59E2,1E0,1.1E1,1.8E1,4E0,3E0,1.3E2,2.9E1,8E0,3E0,2E0,1.6E1,1E1,1.2E2,1E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[2.2962114E-3,-6.9581464E-2,1.4898972E-1,-9.317597E-2,1.3157424E-2,1.7996335E-1,-1.8173766E-1,-1.293594E-1,-4.57916E-2,2.6108235E-2,-9.862527E-3,1.3907112E-1,3.9338005E-1,-1.3150192E-2,-2.2431644E-2,-1.061326E-2,-1.16794944E-1,-5.2521453E-4,-1.0311735E-1,6.413465E-3,1.3320901E-2,1.4907606E-1,-1.08851746E-1,2.834243E-2,1.3293716E-2,1.21679725E-2,-7.566294E-2,-1.2260261E-1,-1.1527503E-3,-5.417974E-2,2.4973813E-2,-1.2551725E-1,1.8197875E-3,2.739252E-4,5.414325E-3,1.2665074E-1,3.866658E-1,1.0042598E-2,-1.1821278E-2,-8.105767E-3,2.5695337E-3,-1.4652708E-1,-9.0317E-2,-6.1172387E-4,-6.213389E-3,-1.3162934E-3,3.2007399E-3,-9.241534E-2,-1.9229527E-1,-3.6289403E-3,1.3225609E-2,9.441477E-2,1.7845827E-1,2.6187172E-2,8.596104E-3,-1.1555315E-3,-1.5478146E-1,-5.310119E-2,-5.930876E-3,-1.6134007E-1,-3.832639E-2,-2.2248366E-1,-1.3777835E-3,-1.178233E-3,1.8196992E-3,-3.312013E-3,1.0325883E-1,1.1742057E-1,2.2752118E-1,-8.546352E-3,-4.3331007E-3,-1.5702773E-3,-7.4079037E-3,-1.1643921E-2,-7.2127517E-4,8.322883E-4,-3.922016E-3,4.372218E-4,-1.2451451E-2,-1.5942693E-3,1.0829583E-1,-4.895034E-4,1.3296324E-1,1.9619276E-3,1.2307032E-2,9.80109E-3,4.7430457E-3,3.1993613E-3,8.73698E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,-1,33,35,37,-1,-1,-1,39,41,-1,43,45,47,-1,49,-1,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,59,61,-1,63,65,67,-1,-1,-1,69,71,-1,73,75,77,-1,-1,-1,-1,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,-1,87,-1,-1,-1,-1,-1,-1],"loss_changes":[4.681354E0,5.8297634E-1,1.5283046E0,3.9184713E-1,1.8780613E-1,1.1147218E0,5.7950556E-1,1.1182237E-1,2.6462865E-1,8.194828E-2,0E0,2.9899883E-1,3.1389022E-1,1.6280374E-1,0E0,0E0,6.0316324E-2,8.070509E-2,1.4521667E-1,0E0,7.187234E-2,5.371115E-1,2.4531896E-1,0E0,0E0,0E0,9.6307784E-2,7.095945E-2,0E0,5.4375138E-2,8.188102E-2,6.8437636E-2,0E0,5.0049167E-2,0E0,1.534977E-1,1.7518234E-1,0E0,0E0,0E0,0E0,5.8557868E-2,4.540986E-2,0E0,0E0,0E0,0E0,9.820227E-2,5.354029E-2,0E0,3.9255716E-2,9.921622E-2,8.7082386E-2,0E0,0E0,0E0,4.8142433E-2,4.2778425E-2,0E0,1.2294823E-1,3.8834997E-2,6.383672E-2,0E0,0E0,0E0,0E0,4.566413E-2,3.8870394E-2,6.0883522E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.193925E-2,0E0,3.7871063E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,16,16,17,17,18,18,20,20,21,21,22,22,26,26,27,27,29,29,30,30,31,31,33,33,35,35,36,36,41,41,42,42,47,47,48,48,50,50,51,51,52,52,56,56,57,57,59,59,60,60,61,61,66,66,67,67,68,68,80,80,82,82],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,-1,34,36,38,-1,-1,-1,40,42,-1,44,46,48,-1,50,-1,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,60,62,-1,64,66,68,-1,-1,-1,70,72,-1,74,76,78,-1,-1,-1,-1,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,-1,88,-1,-1,-1,-1,-1,-1],"split_conditions":[1.227E4,7.164074E8,1.0429407E3,1.732734E6,1E0,4.966055E9,7.339209E6,1.44E2,1.8071064E7,2.829932E0,-9.862527E-3,3.768034E4,6.0976E4,1.3812E4,-2.2431644E-2,-1.061326E-2,2.1592189E5,9.286926E4,1.7076351E3,6.413465E-3,1E0,8.10288E3,1.92144E5,2.834243E-2,1.3293716E-2,1.21679725E-2,1.173433E5,4.822E3,-1.1527503E-3,1E0,5.878966E2,3.069012E6,1.8197875E-3,1.4338295E6,5.414325E-3,5.9161E4,1.6441814E9,1.0042598E-2,-1.1821278E-2,-8.105767E-3,2.5695337E-3,2E1,1E0,-6.1172387E-4,-6.213389E-3,-1.3162934E-3,3.2007399E-3,2.47771E7,4.058817E6,-3.6289403E-3,3.24E2,4.56E2,1.9333144E7,2.6187172E-2,8.596104E-3,-1.1555315E-3,6.1859414E4,6.17296E5,-5.930876E-3,2.206001E3,4.084E4,1.24E2,-1.3777835E-3,-1.178233E-3,1.8196992E-3,-3.312013E-3,2.0772269E8,7.335845E1,1E1,-8.546352E-3,-4.3331007E-3,-1.5702773E-3,-7.4079037E-3,-1.1643921E-2,-7.2127517E-4,8.322883E-4,-3.922016E-3,4.372218E-4,-1.2451451E-2,-1.5942693E-3,3.457E3,-4.895034E-4,7.769761E0,1.9619276E-3,1.2307032E-2,9.80109E-3,4.7430457E-3,3.1993613E-3,8.73698E-3],"split_indices":[2,7,70,1,19,7,59,0,57,70,0,4,10,9,0,0,40,40,64,0,101,64,2,0,0,0,40,2,0,25,64,9,0,57,0,2,7,0,0,0,0,10,24,0,0,0,0,57,41,0,8,10,44,0,0,0,45,41,0,4,11,3,0,0,0,0,5,70,8,0,0,0,0,0,0,0,0,0,0,0,10,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,2.97E2,1.45E2,2.31E2,6.6E1,1.33E2,1.2E1,1.3E2,1.01E2,6.3E1,3E0,1.13E2,2E1,8E0,4E0,1.5E1,1.15E2,5.7E1,4.4E1,6E0,5.7E1,1.09E2,4E0,7E0,1.3E1,1E0,7E0,1.08E2,7E0,1.8E1,3.9E1,3.8E1,6E0,5.1E1,6E0,1.01E2,8E0,1E0,3E0,4E0,3E0,6E1,4.8E1,1.2E1,6E0,1.7E1,2.2E1,2.7E1,1.1E1,7E0,4.4E1,6.4E1,3.7E1,4E0,4E0,4E0,5.6E1,2.2E1,2.6E1,1.1E1,1.6E1,9E0,2E0,1.7E1,2.7E1,3E0,6.1E1,1.8E1,1.9E1,4.4E1,1.2E1,1.9E1,3E0,7E0,4E0,7E0,9E0,1E0,8E0,2E0,5.9E1,2E0,1.6E1,2E0,1.7E1,6E0,5.3E1,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.069079E-2,-6.958858E-2,1.2871398E-1,-7.284245E-2,1.8750688E-2,9.591172E-2,3.6001065E-1,-8.402459E-2,3.205247E-2,-1.9426567E-2,1.24432825E-1,1.3232165E-2,4.8168057E-1,-1.6705692E-1,-7.251025E-2,4.102122E-3,-1.400537E-1,-4.3530106E-3,1.497196E-5,1.3128097E-1,-9.543978E-3,2.7499074E-2,9.195203E-3,-9.530848E-3,-1.2418949E-3,-6.845348E-2,-1.5931956E-2,-9.225682E-4,-9.807078E-3,1.1844186E-1,2.368891E-2,-7.2468616E-2,1.026306E-1,6.788819E-2,1.6373886E-1,-9.440486E-2,-3.80727E-2,9.0794E-3,-5.309415E-4,4.1678756E-2,8.131126E-3,1.7375877E-1,2.1454657E-3,-1.2868752E-1,-2.6756634E-3,-8.0464475E-2,2.1458392E-3,-1.6143502E-4,4.1470123E-3,1.9491435E-3,1.8065067E-1,-7.896157E-3,-4.6172184E-3,-6.6227913E-3,-7.628198E-4,2.3792102E-3,-2.6486826E-3,2.797376E-3,9.479764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,33,-1,35,37,39,41,43,45,-1,-1,47,-1,49,-1,51,-1,53,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0262837E0,3.9348733E-1,1.2704439E0,2.9827678E-1,0E0,5.024328E-1,1.1581206E-1,2.0586884E-1,2.2675432E-1,4.1410267E-2,2.8858256E-1,0E0,6.2045455E-2,8.607215E-2,1.7999732E-1,0E0,3.736546E-2,0E0,0E0,4.718907E-1,0E0,0E0,0E0,0E0,0E0,1.4482278E-1,0E0,0E0,0E0,2.568059E-1,0E0,1.4429104E-1,4.6690717E-2,1.3486227E-1,6.638098E-2,1.5982854E-1,1.3315761E-1,0E0,0E0,8.5588604E-2,0E0,4.496324E-2,0E0,5.349648E-2,0E0,1.2694961E-1,1.05307944E-1,0E0,0E0,0E0,4.908669E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,19,19,25,25,29,29,31,31,32,32,33,33,34,34,35,35,36,36,39,39,41,41,43,43,45,45,46,46,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,34,-1,36,38,40,42,44,46,-1,-1,48,-1,50,-1,52,-1,54,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,7.765071E0,1.577058E9,1.8750688E-2,1.1016051E3,1.8114515E1,4.26944E5,3.3503532E7,4E3,3.768034E4,1.3232165E-2,3.964E3,7.154E3,5.825042E6,4.102122E-3,3.087E3,-4.3530106E-3,1.497196E-5,1.4609149E4,-9.543978E-3,2.7499074E-2,9.195203E-3,-9.530848E-3,-1.2418949E-3,1.3664E4,-1.5931956E-2,-9.225682E-4,-9.807078E-3,1.3210104E9,2.368891E-2,2.09299E6,4.6976192E8,1.7670108E5,1.773296E6,2.551424E6,6.210482E2,9.0794E-3,-5.309415E-4,2.1695537E3,8.131126E-3,2E0,2.1454657E-3,2.88E2,-2.6756634E-3,2.72E2,2.6957834E7,-1.6143502E-4,4.1470123E-3,1.9491435E-3,3.534225E4,-7.896157E-3,-4.6172184E-3,-6.6227913E-3,-7.628198E-4,2.3792102E-3,-2.6486826E-3,2.797376E-3,9.479764E-3],"split_indices":[45,64,47,7,0,64,47,1,57,2,4,0,0,2,59,0,0,0,0,4,0,0,0,0,0,2,0,0,0,7,0,1,7,40,11,9,64,0,0,64,0,8,0,3,0,3,57,0,0,0,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.23E2,2.52E2,1.71E2,2.51E2,1E0,1.51E2,2E1,2.27E2,2.4E1,3E1,1.21E2,1.3E1,7E0,2.6E1,2.01E2,1.9E1,5E0,6E0,2.4E1,1.19E2,2E0,5E0,2E0,2.2E1,4E0,1.99E2,2E0,2E0,3E0,1.16E2,3E0,1.95E2,4E0,5.6E1,6E1,1.18E2,7.7E1,2E0,2E0,4.5E1,1.1E1,5.5E1,5E0,6.3E1,5.5E1,3.7E1,4E1,2.2E1,2.3E1,3E0,5.2E1,3.3E1,3E1,2E1,1.7E1,2.2E1,1.8E1,4E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.7791603E-3,-3.876945E-2,2.1839918E-1,-6.4392515E-2,9.912304E-2,4.197145E-1,1.509777E-1,-8.868081E-2,9.594918E-3,1.2764938E-1,6.935815E-3,9.723304E-3,2.7880026E-2,2.0848687E-1,2.437677E-2,-8.23837E-2,-2.1812852E-2,-1.0962109E-2,7.623083E-3,1.5458927E-1,3.0327065E-2,-6.2850025E-3,2.5282819E-3,2.4070145E-1,9.635124E-5,4.4006975E-3,-1.1020465E-2,-1.5392084E-1,-6.90951E-2,-7.135608E-2,1.8205049E-2,4.7456933E-4,8.390736E-3,3.1535435E-3,-3.9033282E-3,9.669534E-3,3.2523322E-1,-8.683164E-3,-1.2911523E-3,-7.2703786E-2,6.2923315E-3,-1.1397471E-3,-1.2224671E-1,4.4683544E-3,-5.193903E-4,2.0063888E-2,8.4311E-3,-7.515616E-2,1.019415E-2,9.348538E-4,-7.778677E-3,6.872145E-4,-3.3843438E-3,-8.581904E-2,-1.3294231E-2,-5.45577E-3,-2.634318E-3,4.594236E-3,-1.4188427E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,35,-1,-1,-1,37,39,41,43,-1,-1,-1,-1,-1,45,-1,-1,47,-1,-1,49,-1,51,-1,-1,53,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1],"loss_changes":[3.89962E0,1.3278099E0,8.867409E-1,5.68712E-1,1.5427071E-1,4.2026138E-1,3.880763E-1,4.852717E-1,2.3380691E-1,1.1329508E-1,9.3252905E-2,0E0,0E0,2.457416E-1,3.0011904E-1,2.1276271E-1,0E0,1.2460548E-1,0E0,6.5975785E-2,4.4019315E-2,0E0,0E0,6.479895E-2,0E0,0E0,0E0,8.692825E-2,1.4989442E-1,5.403936E-2,6.40434E-2,0E0,0E0,0E0,0E0,0E0,5.160451E-2,0E0,0E0,1.4900148E-1,0E0,0E0,5.469957E-2,0E0,3.8330622E-2,0E0,0E0,1.2814581E-1,0E0,0E0,0E0,0E0,0E0,1.2333858E-1,5.0263893E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,19,19,20,20,23,23,27,27,28,28,29,29,30,30,36,36,39,39,42,42,44,44,47,47,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,36,-1,-1,-1,38,40,42,44,-1,-1,-1,-1,-1,46,-1,-1,48,-1,-1,50,-1,52,-1,-1,54,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.5522031E9,1.417988E6,9.7467445E4,1.3185959E7,9.256843E3,1.5096262E7,2.2233E4,1.7670108E5,1.59E2,2.1853803E4,9.723304E-3,2.7880026E-2,2.2576077E10,1.92144E5,4.15408E5,-2.1812852E-2,2.0926301E3,7.623083E-3,1.504E4,7.4433655E-2,-6.2850025E-3,2.5282819E-3,4.321724E7,9.635124E-5,4.4006975E-3,-1.1020465E-2,7.154E3,3.346087E5,6.3349745E6,7.9E1,4.7456933E-4,8.390736E-3,3.1535435E-3,-3.9033282E-3,9.669534E-3,8.3E1,-8.683164E-3,-1.2911523E-3,1.3428E4,6.2923315E-3,-1.1397471E-3,7.5091325E6,4.4683544E-3,7.1366974E1,2.0063888E-2,8.4311E-3,7.164074E8,1.019415E-2,9.348538E-4,-7.778677E-3,6.872145E-4,-3.3843438E-3,1.732734E6,2.409091E0,-5.45577E-3,-2.634318E-3,4.594236E-3,-1.4188427E-3],"split_indices":[64,7,41,45,57,4,9,2,40,3,45,0,0,12,2,1,0,4,0,9,69,0,0,1,0,0,0,2,40,57,3,0,0,0,0,0,8,0,0,2,0,0,44,0,68,0,0,7,0,0,0,0,0,1,70,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,3.73E2,6.9E1,3.15E2,5.8E1,1.6E1,5.3E1,2.37E2,7.8E1,4.4E1,1.4E1,7E0,9E0,3.6E1,1.7E1,2.34E2,3E0,6.9E1,9E0,3.4E1,1E1,3E0,1.1E1,3.1E1,5E0,1.4E1,3E0,3.5E1,1.99E2,2.2E1,4.7E1,3E0,3.1E1,8E0,2E0,2.2E1,9E0,3E1,5E0,1.96E2,3E0,1.2E1,1E1,9E0,3.8E1,5E0,4E0,1.95E2,1E0,2E0,8E0,3.2E1,6E0,1.66E2,2.9E1,9.6E1,7E1,3E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.3981937E-3,-5.366389E-2,1.298958E-1,-7.399241E-2,2.1687092E-2,1.422995E-1,-1.6672356E-2,-1.1124854E-1,-2.7512455E-2,3.4535494E-2,-9.3004005E-3,1.18673764E-1,1.4719158E-2,-1.3351567E-1,-7.239193E-2,-3.4283455E-2,1.7585479E-2,-6.245318E-5,7.394082E-2,1.2818965E-1,-6.478702E-3,-9.742109E-3,-5.8070845E-3,2.5505287E-5,-8.802599E-2,9.407555E-3,-8.515318E-2,9.161102E-3,1.2293936E-1,1.0740006E-1,2.3086174E-1,-6.2922677E-3,-2.5617743E-3,-7.8645666E-5,6.989779E-3,-6.3888365E-3,-1.2553515E-1,3.1099168E-3,-3.5965915E-3,2.1766622E-3,8.703518E-3,7.817381E-2,1.8352386E-1,2.5169834E-1,1.7908249E-3,-3.1298962E-3,1.3231375E-2,-3.3689619E-3,1.9276812E-3,-1.145071E-2,-1.0334624E-1,1.2354696E-1,4.9479727E-2,1.4677053E-2,7.2534727E-3,1.6497146E-2,8.18641E-3,2.2560188E-3,-1.2535286E-3,2.3066152E-3,-1.16129786E-1,1.5912277E-1,2.178869E-3,-2.7824363E-3,5.791474E-2,-3.8113971E-3,-9.364853E-3,3.011504E-4,8.834757E-3,-1.2868326E-3,3.4633968E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,25,-1,-1,27,29,-1,-1,-1,-1,31,33,35,37,39,41,43,-1,-1,45,-1,47,49,-1,-1,-1,-1,51,53,55,-1,-1,57,-1,-1,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,65,67,-1,-1,69,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2713912E0,4.7185922E-1,8.6385083E-1,4.1600168E-1,1.8262675E-1,4.649496E-1,0E0,1.0627484E-1,2.9206848E-1,8.893033E-2,0E0,3.0489063E-1,0E0,6.669903E-2,5.8172643E-2,2.4083647E-1,0E0,0E0,9.409709E-2,2.270776E-1,0E0,0E0,0E0,0E0,5.203864E-2,7.302477E-2,1.5823093E-1,6.475091E-2,6.1531126E-2,2.0813453E-1,6.819165E-2,0E0,0E0,4.7450423E-2,0E0,5.1985607E-2,5.2172065E-2,0E0,0E0,0E0,0E0,9.0230584E-2,7.154226E-2,6.307316E-2,0E0,0E0,5.859545E-2,0E0,0E0,0E0,6.0783178E-2,7.2701246E-2,4.4912703E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.2407494E-2,5.029595E-2,0E0,0E0,4.3699875E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,18,18,19,19,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,36,36,41,41,42,42,43,43,46,46,50,50,51,51,52,52,60,60,61,61,64,64],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,26,-1,-1,28,30,-1,-1,-1,-1,32,34,36,38,40,42,44,-1,-1,46,-1,48,50,-1,-1,-1,-1,52,54,56,-1,-1,58,-1,-1,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,66,68,-1,-1,70,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.164074E8,2.3028242E3,1.732734E6,1E0,6.001104E9,-1.6672356E-2,7.6E1,4.6454727E3,1.1576422E9,-9.3004005E-3,1.2622761E8,1.4719158E-2,3.133E3,8.9E1,1.7239808E7,1.7585479E-2,-6.245318E-5,2.9222983E3,6.204522E3,-6.478702E-3,-9.742109E-3,-5.8070845E-3,2.5505287E-5,3.006846E6,5.0543875E-1,1.4E4,5.78E2,1E0,7.2473733E9,3.5301748E7,-6.2922677E-3,-2.5617743E-3,2.2737327E0,6.989779E-3,1.534E3,6.7440504E7,3.1099168E-3,-3.5965915E-3,2.1766622E-3,8.703518E-3,8.16997E6,9.151661E6,3.69814E6,1.7908249E-3,-3.1298962E-3,2.1813516E-2,-3.3689619E-3,1.9276812E-3,-1.145071E-2,9.97639E-6,1.52E2,2.1853803E4,1.4677053E-2,7.2534727E-3,1.6497146E-2,8.18641E-3,2.2560188E-3,-1.2535286E-3,2.3066152E-3,3.069012E6,6.287E3,2.178869E-3,-2.7824363E-3,5.767668E0,-3.8113971E-3,-9.364853E-3,3.011504E-4,8.834757E-3,-1.2868326E-3,3.4633968E-3],"split_indices":[2,7,70,1,19,7,0,8,4,7,0,1,0,2,8,57,0,0,4,64,0,0,0,0,9,69,11,8,26,12,57,0,0,65,0,11,7,0,0,0,0,57,63,59,0,0,69,0,0,0,50,3,45,0,0,0,0,0,0,0,9,11,0,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.06E2,1.41E2,2.41E2,6.5E1,1.38E2,3E0,1.33E2,1.08E2,6.2E1,3E0,1.21E2,1.7E1,8.3E1,5E1,1.07E2,1E0,3.3E1,2.9E1,1.17E2,4E0,1.6E1,6.7E1,9E0,4.1E1,5.8E1,4.9E1,1.3E1,1.6E1,9.9E1,1.8E1,1.9E1,2.2E1,5.5E1,3E0,1.7E1,3.2E1,8E0,5E0,7E0,9E0,7.3E1,2.6E1,1.6E1,2E0,9E0,4.6E1,7E0,1E1,4E0,2.8E1,2.7E1,4.6E1,5E0,2.1E1,7E0,9E0,2.5E1,2.1E1,2E0,2.6E1,1.8E1,9E0,3E0,4.3E1,1.8E1,8E0,2E0,1.6E1,5E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[7.979123E-3,-6.958848E-2,1.0561463E-1,-6.238112E-2,-1.963937E-2,5.514976E-2,2.5690955E-1,-1.09892555E-1,-3.4030605E-2,-3.2504003E-2,8.385704E-2,2.6674887E-2,1.8643518E-1,-1.1569285E-1,9.718655E-3,-3.8098093E-2,1.4090513E-2,-1.3982342E-2,-8.854848E-3,4.6036143E-2,1.1815436E-1,2.1883409E-1,-2.8231194E-2,-1.4651564E-1,-4.291283E-2,-5.804345E-2,2.9771794E-3,-4.869472E-3,6.1138254E-4,7.2105194E-4,5.603442E-3,1.2988494E-1,8.738465E-4,1.1519298E-2,1.0741537E-3,-6.9388556E-3,3.0878147E-3,-2.00759E-3,-1.5545295E-1,2.4806203E-3,-3.0568568E-3,-7.613656E-2,2.7605334E-2,3.2494925E-2,-7.313825E-2,9.2955925E-2,1.6309573E-1,-1.0220464E-2,-6.3000624E-3,-4.8556764E-2,-1.15616955E-1,5.0903847E-3,-1.8008241E-2,3.170746E-3,-2.9129567E-4,-1.3175979E-3,-7.599262E-3,5.2687745E-3,-1.8885626E-3,9.3981E-3,2.6350247E-3,-9.8031096E-2,-2.7811804E-3,-2.5810536E-3,-1.4525501E-1,-4.437872E-3,1.4236979E-3,-7.470877E-4,-6.9285785E-3,-1.59672E-3,2.9538753E-3,-9.2616575E-3,-2.5974023E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,-1,25,-1,27,-1,29,31,33,35,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,51,53,55,57,59,-1,-1,61,63,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,67,69,-1,71,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2335978E0,5.302081E-1,1.4288611E0,3.1145215E-1,0E0,3.625567E-1,8.1747746E-1,1.7608535E-1,2.0095526E-1,9.399077E-2,1.3465345E-1,0E0,2.7742112E-1,1.866349E-1,0E0,1.2060416E-1,0E0,7.338969E-2,0E0,1.09092414E-1,6.38597E-2,7.3293924E-2,6.832053E-2,5.078149E-2,4.8648488E-2,1.555174E-1,1.1246768E-1,0E0,0E0,0E0,0E0,4.6161354E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.3516865E-2,0E0,0E0,8.367261E-2,6.2797695E-2,4.3070458E-2,4.7670186E-2,4.63742E-2,5.89453E-2,0E0,0E0,1.1296316E-1,5.5065572E-2,0E0,4.20279E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.889636E-2,5.062745E-2,0E0,7.259363E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,38,38,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,52,52,61,61,62,62,64,64],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,-1,26,-1,28,-1,30,32,34,36,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,52,54,56,58,60,-1,-1,62,64,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,68,70,-1,72,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.393928E4,7.339209E6,9.3058E4,9.50639E5,-1.963937E-2,9.9800183E2,1.681178E6,1.0043E4,8.861901E3,1E0,2.325945E9,2.6674887E-2,1.5096262E7,3.006846E6,9.718655E-3,1.0975164E3,1.4090513E-2,3.957E3,-8.854848E-3,1.29186E9,2.192772E10,3.0113987E10,3.677E3,6.738547E7,2.409091E0,3.94E2,3.653602E7,-4.869472E-3,6.1138254E-4,7.2105194E-4,5.603442E-3,1.334086E5,8.738465E-4,1.1519298E-2,1.0741537E-3,-6.9388556E-3,3.0878147E-3,-2.00759E-3,3.606E3,2.4806203E-3,-3.0568568E-3,8.593763E7,4.16606E5,7.2E1,7.494142E6,4.17E2,2.5519034E7,-1.0220464E-2,-6.3000624E-3,2.09299E6,1.1004841E9,5.0903847E-3,2.6490103E1,3.170746E-3,-2.9129567E-4,-1.3175979E-3,-7.599262E-3,5.2687745E-3,-1.8885626E-3,9.3981E-3,2.6350247E-3,5.91294E5,2.53383E5,-2.5810536E-3,1.16985195E5,-4.437872E-3,1.4236979E-3,-7.470877E-4,-6.9285785E-3,-1.59672E-3,2.9538753E-3,-9.2616575E-3,-2.5974023E-3],"split_indices":[45,59,2,1,0,64,41,2,64,19,5,0,9,9,0,64,0,2,0,7,5,12,0,7,70,8,57,0,0,0,0,40,0,0,0,0,0,0,2,0,0,7,41,8,1,3,9,0,0,1,5,0,70,0,0,0,0,0,0,0,0,41,9,0,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,2.37E2,1.88E2,2.33E2,4E0,1.42E2,4.6E1,8.6E1,1.47E2,3.5E1,1.07E2,8E0,3.8E1,8.5E1,1E0,1.46E2,1E0,3.2E1,3E0,5.2E1,5.5E1,3.3E1,5E0,5.9E1,2.6E1,9.8E1,4.8E1,7E0,2.5E1,3.6E1,1.6E1,4.9E1,6E0,3.1E1,2E0,2E0,3E0,5E0,5.4E1,4E0,2.2E1,8.1E1,1.7E1,3.5E1,1.3E1,2.5E1,2.4E1,1.8E1,3.6E1,4.9E1,3.2E1,6E0,1.1E1,1.9E1,1.6E1,9E0,4E0,2.3E1,2E0,1.9E1,5E0,2.3E1,2.6E1,1.1E1,2.1E1,4E0,7E0,8E0,1.5E1,1.8E1,8E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[9.305403E-3,-5.4016035E-2,1.21552825E-1,-1.1848685E-1,-3.3374026E-2,8.2017794E-2,3.4051996E-1,-1.318432E-1,-1.8636568E-3,-3.9393086E-2,4.6955533E-3,9.62635E-2,-1.5516191E-2,2.7030058E-2,2.0654626E-1,-7.6234695E-3,-3.3970748E-3,-6.5418184E-2,-1.3239449E-2,5.9112474E-2,1.5052421E-1,3.4680734E-3,1.2318133E-2,-1.0822117E-1,-3.166306E-2,1.3496502E-2,-5.4650705E-2,-7.814791E-3,6.9538705E-2,-1.3629463E-2,1.6064602E-1,-1.5090683E-1,-6.0830254E-2,-4.2783473E-2,1.7427024E-3,3.2116822E-3,5.61637E-3,-8.648985E-3,-3.7172068E-2,1.0156484E-1,4.5747947E-2,4.8482283E-3,-4.1403486E-3,1.9462213E-1,4.4694473E-3,-3.0355821E-3,-9.323449E-3,-1.91301E-3,-9.91604E-3,-4.170164E-6,-3.2050181E-3,3.1411946E-3,-1.335345E-2,-7.001184E-3,-2.045191E-2,1.22105524E-1,1.2333898E-3,-4.300899E-3,5.6945164E-2,1.0117019E-2,-1.3858333E-3,-4.274658E-2,2.8464606E-2,3.0407312E-3,-4.890026E-2,3.9898884E-4,1.3730472E-1,2.0047787E-3,5.8006984E-3,-4.17755E-3,9.334796E-4,8.714481E-5,5.411335E-3,-4.036655E-3,3.49039E-4,3.4255232E-3,8.580945E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,-1,21,-1,-1,23,25,27,29,-1,-1,31,33,35,37,-1,39,41,43,45,47,49,-1,51,-1,-1,53,55,57,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,63,65,-1,-1,67,-1,-1,69,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1339397E0,3.7133318E-1,1.3485141E0,6.8828166E-2,1.6847876E-1,7.903035E-1,5.32454E-1,6.630921E-2,0E0,1.39348E-1,0E0,2.571931E-1,0E0,0E0,7.052845E-2,0E0,0E0,1.4538988E-1,1.1588321E-1,1.9060796E-1,9.025073E-2,0E0,0E0,8.18513E-2,4.5144998E-2,6.472775E-2,7.842364E-2,0E0,5.4021925E-2,3.866988E-2,1.0357392E-1,6.260294E-2,6.3769154E-2,4.6483256E-2,0E0,5.9130266E-2,0E0,0E0,6.0805045E-2,4.752329E-2,7.27908E-2,0E0,0E0,6.2957525E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.8987454E-2,0E0,7.929526E-2,4.226783E-2,0E0,0E0,3.8132593E-2,0E0,0E0,7.1809754E-2,4.2465128E-2,0E0,4.5092065E-2,0E0,4.026419E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,35,35,38,38,39,39,40,40,43,43,52,52,54,54,55,55,58,58,61,61,62,62,64,64,66,66],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,-1,22,-1,-1,24,26,28,30,-1,-1,32,34,36,38,-1,40,42,44,46,48,50,-1,52,-1,-1,54,56,58,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,64,66,-1,-1,68,-1,-1,70,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,8.10288E3,2.7299898E8,1.577058E9,4.673502E7,3.0418E4,8.1672195E4,-1.8636568E-3,2.025313E6,4.6955533E-3,4.1331047E-1,-1.5516191E-2,2.7030058E-2,2.9169512E5,-7.6234695E-3,-3.3970748E-3,2.551424E6,1.8071064E7,1.293844E0,2.2782404E0,3.4680734E-3,1.2318133E-2,1E0,6.03E2,5.0543875E-1,1.9120918E7,-7.814791E-3,8.16997E6,2.722E3,9.705148E-1,4.52737E5,1.09014104E8,1.039769E6,1.7427024E-3,1.6176678E3,5.61637E-3,-8.648985E-3,7.0151276E1,7.4433655E-2,2.6248154E3,4.8482283E-3,-4.1403486E-3,3.020675E5,4.4694473E-3,-3.0355821E-3,-9.323449E-3,-1.91301E-3,-9.91604E-3,-4.170164E-6,-3.2050181E-3,3.1411946E-3,1.656968E6,-7.001184E-3,1.53042E5,3.44E2,1.2333898E-3,-4.300899E-3,4.23168E3,1.0117019E-2,-1.3858333E-3,1.152E4,1.1762006E9,3.0407312E-3,2.058478E7,3.9898884E-4,4.884611E1,2.0047787E-3,5.8006984E-3,-4.17755E-3,9.334796E-4,8.714481E-5,5.411335E-3,-4.036655E-3,3.49039E-4,3.4255232E-3,8.580945E-3],"split_indices":[2,1,64,7,7,60,10,40,0,1,0,47,0,0,44,0,0,9,57,65,66,0,0,26,8,69,57,0,57,0,46,41,44,1,0,45,0,0,60,69,45,0,0,45,0,0,0,0,0,0,0,0,9,0,12,0,0,0,64,0,0,10,12,0,44,0,68,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,2.81E2,1.58E2,6.7E1,2.14E2,1.35E2,2.3E1,5.7E1,1E1,2.05E2,9E0,1.31E2,4E0,8E0,1.5E1,4.2E1,1.5E1,1.02E2,1.03E2,7.9E1,5.2E1,4E0,1.1E1,4.4E1,5.8E1,6.3E1,4E1,3E0,7.6E1,3E0,4.9E1,2.2E1,2.2E1,5E1,8E0,5.8E1,5E0,4E0,3.6E1,3.1E1,4.5E1,1E0,2E0,3.2E1,1.7E1,7E0,1.5E1,2E1,2E0,1.7E1,3.3E1,1.2E1,4.6E1,4E0,3.2E1,2.4E1,7E0,3E0,4.2E1,3.1E1,1E0,2.7E1,1.9E1,8E0,2.4E1,3E0,2.1E1,3.4E1,8E0,1.6E1,1.1E1,1.5E1,4E0,1.5E1,9E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[3.5049938E-4,-4.9051628E-2,1.1725362E-1,-9.582418E-2,-1.980691E-2,1.0032905E-1,2.0126745E-2,-1.1352916E-1,-1.6989084E-2,-2.5552236E-2,1.2824435E-2,1.13377996E-1,-1.3583568E-1,-1.4107162E-1,-5.0339695E-2,-6.161619E-5,-6.0780537E-3,-4.1804936E-2,4.2345475E-2,8.793043E-2,1.9294219E-1,-9.290382E-3,-3.5594657E-4,-1.4938375E-1,-4.0602504E-4,3.4829488E-4,-4.8949197E-3,-5.220551E-2,2.9969169E-2,1.7358916E-2,7.344029E-3,1.2719208E-1,4.9290765E-2,-6.5178975E-2,2.3867817E-1,-7.7102073E-3,-3.699678E-6,-6.1885342E-2,-2.7991377E-3,-2.6278704E-4,5.64944E-3,7.1505643E-3,-6.1028833E-5,9.154261E-3,8.666187E-2,1.2838417E-3,5.2361996E-3,1.6890946E-3,-7.1180784E-3,1.9115858E-2,9.030614E-3,-4.8920758E-2,-1.424616E-1,2.589322E-3,-2.857444E-3,3.0204253E-3,7.831318E-3,-4.0444978E-2,-1.1946312E-1,-8.47109E-3,-1.2855014E-3,-4.467584E-3,-1.200106E-3,-8.759733E-3,4.037847E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,37,39,41,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5468671E0,4.2213553E-1,5.925242E-1,1.6416824E-1,3.1147474E-1,4.006132E-1,0E0,1.617167E-1,3.7767176E-2,2.103357E-1,0E0,2.2397554E-1,4.378049E-2,7.2826385E-2,8.525749E-2,0E0,0E0,1.1577785E-1,9.415706E-2,1.3329977E-1,3.4610868E-1,0E0,0E0,4.6025753E-2,0E0,0E0,0E0,6.391168E-2,5.912386E-2,7.25877E-2,0E0,8.622694E-2,6.0164362E-2,4.304408E-2,1.3101208E-1,0E0,0E0,1.10030025E-1,7.126208E-2,0E0,0E0,0E0,0E0,0E0,4.1958302E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.3762645E-2,4.2991698E-2,0E0,0E0,0E0,0E0,6.943664E-2,7.26046E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,27,27,28,28,29,29,31,31,32,32,33,33,34,34,37,37,38,38,44,44,51,51,52,52,57,57,58,58],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,38,40,42,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,4.844E3,1.0871896E10,2.0964778E5,3.058642E3,9.05929E2,2.0126745E-2,3.006846E6,9.30419E5,3.53E2,1.2824435E-2,6.204522E3,1.31805E5,1.3421E4,1.19E2,-6.161619E-5,-6.0780537E-3,1.4998456E5,1.3738772E5,1.1291153E7,5.3140095E1,-9.290382E-3,-3.5594657E-4,3.5111064E7,-4.0602504E-4,3.4829488E-4,-4.8949197E-3,1.7985837E3,1.3412494E6,1.12608824E8,7.344029E-3,1.50301E6,1.2393866E2,1.0014992E8,2.1172E4,-7.7102073E-3,-3.699678E-6,1E0,6.7211235E1,-2.6278704E-4,5.64944E-3,7.1505643E-3,-6.1028833E-5,9.154261E-3,2.5501363E9,1.2838417E-3,5.2361996E-3,1.6890946E-3,-7.1180784E-3,1.9115858E-2,9.030614E-3,3.9034148E7,3.77E2,2.589322E-3,-2.857444E-3,3.0204253E-3,7.831318E-3,3.34E2,1.4171124E3,-8.47109E-3,-1.2855014E-3,-4.467584E-3,-1.200106E-3,-8.759733E-3,4.037847E-4],"split_indices":[2,2,7,40,64,70,0,9,9,8,0,64,40,10,3,0,0,40,45,57,68,0,0,57,0,0,0,64,60,5,0,9,70,1,10,0,0,19,68,0,0,0,0,0,7,0,0,0,0,0,0,57,3,0,0,0,0,0,64,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.09E2,1.3E2,1.18E2,1.91E2,1.24E2,6E0,9.6E1,2.2E1,1.88E2,3E0,1.18E2,6E0,6.6E1,3E1,2E1,2E0,1.52E2,3.6E1,9.1E1,2.7E1,4E0,2E0,6.2E1,4E0,1.4E1,1.6E1,1.33E2,1.9E1,3E1,6E0,4.4E1,4.7E1,4E0,2.3E1,6E1,2E0,1.11E2,2.2E1,1.4E1,5E0,3E0,2.7E1,1.7E1,2.7E1,3.4E1,1.3E1,2E0,2E0,5E0,1.8E1,9.7E1,1.4E1,1.1E1,1.1E1,2.1E1,6E0,8.8E1,9E0,1.1E1,3E0,2.1E1,6.7E1,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-5.343586E-3,-6.0238432E-2,7.3978655E-2,-5.5175144E-2,-1.8515898E-2,2.5159422E-2,1.4045288E-1,-1.0111012E-1,-3.0238695E-2,-3.4809798E-2,4.961124E-2,9.180722E-2,2.1612665E-1,-1.1652162E-1,-1.628455E-2,-3.3435658E-2,1.2585321E-2,-1.9319358E-4,-7.014219E-3,1.112606E-2,8.883586E-2,1.08493365E-1,-9.727819E-3,1.6305342E-1,2.5341421E-2,-7.6062037E-3,-8.060897E-2,-2.6466285E-3,2.3764523E-3,-4.3950394E-2,3.4306634E-2,-2.309749E-2,3.086823E-3,7.2743637E-3,1.9335019E-3,8.555502E-3,4.314024E-3,-1.5590776E-3,8.599449E-3,7.415862E-4,-1.0493463E-1,-2.7480425E-2,-8.9603186E-2,2.9606454E-3,-4.9447864E-3,-2.6438022E-3,1.6219004E-3,-6.5997257E-3,-1.6021406E-3,-1.0580803E-2,-8.618824E-2,-5.627741E-3,-1.4804305E-4,1.3065879E-2,-4.6435118E-2,-2.4611312E-3,-9.43582E-3,-2.557373E-3,1.576671E-3,-3.992351E-3,-6.1288744E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,31,33,35,-1,37,-1,-1,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,47,49,51,-1,-1,-1,-1,-1,-1,53,55,-1,-1,57,59,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9160302E0,3.8799703E-1,5.806061E-1,2.9132456E-1,0E0,1.556944E-1,2.599293E-1,1.1575854E-1,1.5991443E-1,1.00700065E-1,1.1222908E-1,2.5055143E-1,3.6511958E-1,8.51022E-2,3.763463E-2,1.2046063E-1,0E0,0E0,0E0,6.901671E-2,9.96117E-2,4.916829E-2,0E0,5.091268E-2,0E0,0E0,9.443033E-2,0E0,0E0,1.0656318E-1,8.2279675E-2,4.145207E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.90187E-2,1.0623918E-1,7.54087E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.267274E-2,8.16845E-2,0E0,0E0,6.335315E-2,3.78837E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,23,23,26,26,29,29,30,30,31,31,40,40,41,41,42,42,49,49,50,50,53,53,54,54],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,32,34,36,-1,38,-1,-1,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,48,50,52,-1,-1,-1,-1,-1,-1,54,56,-1,-1,58,60,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,2.5096E4,4.407E3,-1.8515898E-2,2.0231652E3,1.5452943E0,2.0964778E5,8.861901E3,1.1291153E7,9.2849056E8,4.52E2,3.305542E4,2.53383E5,4.174811E2,1.577058E9,1.2585321E-2,-1.9319358E-4,-7.014219E-3,1.07E3,7.625084E6,1.0548451E7,-9.727819E-3,1.4797951E-2,2.5341421E-2,-7.6062037E-3,2.1016268E-2,-2.6466285E-3,2.3764523E-3,4.6263345E-2,1.4241472E3,9.6E1,3.086823E-3,7.2743637E-3,1.9335019E-3,8.555502E-3,4.314024E-3,-1.5590776E-3,8.599449E-3,7.415862E-4,1.6776951E3,3.0840238E7,1.6120845E5,2.9606454E-3,-4.9447864E-3,-2.6438022E-3,1.6219004E-3,-6.5997257E-3,-1.6021406E-3,1.0062742E2,6.54925E6,-5.627741E-3,-1.4804305E-4,4.5765094E2,3.069055E6,-2.4611312E-3,-9.43582E-3,-2.557373E-3,1.576671E-3,-3.992351E-3,-6.1288744E-4],"split_indices":[45,59,2,2,0,67,51,40,64,57,7,3,4,9,45,7,0,0,0,0,60,57,0,69,0,0,51,0,0,69,70,8,0,0,0,0,0,0,0,0,4,57,40,0,0,0,0,0,0,66,9,0,0,64,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,2.59E2,1.79E2,2.56E2,3E0,1.04E2,7.5E1,8.9E1,1.67E2,3E1,7.4E1,4.7E1,2.8E1,7.5E1,1.4E1,1.66E2,1E0,2.4E1,6E0,3.8E1,3.6E1,4.5E1,2E0,2.5E1,3E0,3.6E1,3.9E1,9E0,5E0,1.44E2,2.2E1,2.3E1,1.5E1,1.6E1,2E1,1E1,3.5E1,1E0,2.4E1,8E0,3.1E1,1.07E2,3.7E1,1.9E1,3E0,1.5E1,8E0,2.2E1,9E0,8.4E1,2.3E1,2.9E1,8E0,5.1E1,3.3E1,1.8E1,5E0,1.1E1,4E1,1.6E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[2.81754E-3,-3.865204E-2,1.1897957E-1,-9.380661E-2,-1.3721979E-2,8.6210854E-2,2.477309E-1,-1.2186428E-1,-4.198079E-2,-2.6082521E-2,4.7791287E-2,1.01563305E-1,-2.0737547E-1,1.5459806E-1,2.4280427E-2,-1.3208808E-1,-9.137162E-4,9.5844455E-4,-3.428025E-3,-6.613803E-2,-3.3564316E-3,5.962589E-3,1.0194378E-2,2.2037597E-1,8.04591E-2,-1.3263213E-2,6.16893E-4,8.561089E-3,-3.2334009E-3,-6.9519817E-3,-1.4228976E-3,-2.2631954E-2,-8.787153E-2,-1.3570071E-2,5.5659353E-3,-3.5176654E-3,2.4947412E-3,1.9352641E-2,1.03188604E-1,2.1141209E-3,1.2159726E-1,-4.4853375E-3,5.139614E-4,-2.2214537E-3,-1.1517269E-1,2.8771344E-3,-6.074232E-2,-1.3534238E-3,7.2104186E-3,-4.5213387E-3,1.4296997E-1,-1.1080875E-2,-4.348017E-3,-1.5337454E-3,1.778965E-3,-2.0411878E-3,-7.4009066E-3,1.7334595E-1,4.026411E-3,1.9516768E-1,3.004489E-3,4.1581163E-5,1.0322573E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,41,43,45,-1,-1,-1,-1,47,-1,49,-1,-1,-1,51,53,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,59,-1,61,-1,-1,-1],"loss_changes":[2.1241624E0,4.4559002E-1,4.680488E-1,1.4174938E-1,1.7232242E-1,4.4474512E-1,4.3493474E-1,6.641966E-2,6.1536357E-2,1.7091537E-1,1.0095959E-1,2.0793903E-1,6.70118E-2,7.702616E-2,0E0,3.894341E-2,0E0,0E0,0E0,6.2307954E-2,1.4329846E-1,0E0,8.674066E-2,2.1424508E-1,1.1729169E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.3945087E-2,4.822573E-2,8.7070145E-2,0E0,0E0,0E0,0E0,5.208098E-2,0E0,1.8063521E-1,0E0,0E0,0E0,6.1753273E-2,9.3347624E-2,4.4356823E-2,0E0,0E0,0E0,5.0483048E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.262483E-2,0E0,3.9644778E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,22,22,23,23,24,24,31,31,32,32,33,33,38,38,40,40,44,44,45,45,46,46,50,50,57,57,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,42,44,46,-1,-1,-1,-1,48,-1,50,-1,-1,-1,52,54,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,60,-1,62,-1,-1,-1],"split_conditions":[1.7267E4,9.26787E5,2.1809912E0,3.006846E6,3.53E2,3.768034E4,1.2123712E1,1.7626338E6,1.19E2,7.283496E2,4.16606E5,7.801943E3,9.652E3,1.3936486E8,2.4280427E-2,1.4921534E5,-9.137162E-4,9.5844455E-4,-3.428025E-3,4.1E1,2.5653894E5,5.962589E-3,1.5292238E5,4.8E1,6.0891E4,-1.3263213E-2,6.16893E-4,8.561089E-3,-3.2334009E-3,-6.9519817E-3,-1.4228976E-3,1E0,6.17296E5,2.7582266E7,5.5659353E-3,-3.5176654E-3,2.4947412E-3,1.9352641E-2,3E0,2.1141209E-3,1.5165479E0,-4.4853375E-3,5.139614E-4,-2.2214537E-3,1.874E3,1.1639729E9,6.54925E6,-1.3534238E-3,7.2104186E-3,-4.5213387E-3,4.128514E-1,-1.1080875E-2,-4.348017E-3,-1.5337454E-3,1.778965E-3,-2.0411878E-3,-7.4009066E-3,1.3590306E10,4.026411E-3,1E0,3.004489E-3,4.1581163E-5,1.0322573E-2],"split_indices":[2,1,54,9,8,4,65,62,3,64,41,44,0,1,0,40,0,0,0,8,40,0,59,3,2,0,0,0,0,0,0,14,41,57,0,0,0,0,8,0,65,0,0,0,11,5,9,0,0,0,69,0,0,0,0,0,0,12,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.24E2,1.15E2,1E2,2.24E2,9.3E1,2.2E1,6.4E1,3.6E1,1.87E2,3.7E1,8.9E1,4E0,1.7E1,5E0,5.8E1,6E0,1.1E1,2.5E1,6.7E1,1.2E2,1.2E1,2.5E1,1.2E1,7.7E1,3E0,1E0,1.6E1,1E0,5.4E1,4E0,2.3E1,4.4E1,1.11E2,9E0,8E0,1.7E1,4E0,8E0,4.1E1,3.6E1,7E0,1.6E1,1.8E1,2.6E1,8.3E1,2.8E1,2E0,6E0,3E0,3.3E1,4E0,2.2E1,4.1E1,4.2E1,2.4E1,4E0,2.1E1,1.2E1,1.7E1,4E0,1E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[5.544589E-3,-3.512823E-2,1.0414052E-1,-9.243788E-2,-1.8505335E-2,-1.5029754E-1,1.24820516E-1,-1.0577108E-1,-1.3426739E-3,-3.0529955E-2,3.7294213E-2,1.4248106E-3,-1.6732337E-2,9.522637E-2,2.3110168E-1,-5.9993328E-3,-1.757704E-3,-5.89731E-2,-1.3808691E-2,5.7297708E-3,2.1651333E-2,5.371469E-2,1.2917319E-1,4.169747E-1,1.19104095E-1,-6.796784E-3,-4.311335E-2,-4.7851186E-2,9.217308E-3,-2.7165072E-2,2.4600944E-3,4.726046E-3,1.4654844E-3,1.0412661E-2,9.660243E-2,2.2237627E-2,4.8239273E-3,1.6246264E-1,-6.8662115E-3,-1.2421554E-3,-4.930933E-3,-7.008409E-5,-7.777294E-2,4.2907475E-3,-3.6954654E-3,5.427002E-4,-4.8866915E-3,1.1576109E-3,5.7679447E-3,9.287843E-3,3.489136E-4,-6.5392083E-3,-2.1009946E-3,1.0022652E-2,-5.5364378E-2,4.7005154E-2,-6.010395E-4,-3.894747E-3,2.6668124E-3,5.3358534E-3,6.879782E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,-1,-1,25,27,-1,29,31,33,35,37,-1,39,41,43,45,-1,-1,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1],"loss_changes":[1.7440737E0,2.9170623E-1,6.870061E-1,5.7483554E-2,1.62171E-1,3.3946165E-1,3.4913838E-1,5.3215504E-2,0E0,9.335552E-2,4.9550723E-2,0E0,0E0,1.255253E-1,4.7077E-1,0E0,0E0,8.331618E-2,9.920773E-2,0E0,5.1141746E-2,4.1068897E-2,1.1139864E-1,3.777623E-2,2.1138123E-1,0E0,6.043408E-2,7.077228E-2,7.552826E-2,3.8653273E-2,0E0,0E0,0E0,0E0,4.982406E-2,0E0,0E0,5.323693E-2,0E0,0E0,0E0,0E0,5.329019E-2,0E0,4.7335714E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3783333E-2,3.8510602E-2,3.8259458E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,17,17,18,18,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,34,34,37,37,42,42,44,44,53,53,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,-1,-1,26,28,-1,30,32,34,36,38,-1,40,42,44,46,-1,-1,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.41718E5,2.4950776E0,9.703059E6,3.53E2,6.27427E5,9.571874E3,3.91E2,-1.3426739E-3,1.732734E6,2.25161E5,1.4248106E-3,-1.6732337E-2,4.1331047E-1,3.3616E4,-5.9993328E-3,-1.757704E-3,2.8E1,1E0,5.7297708E-3,1.570662E5,1.2394286E9,2.1207E4,2.1592189E5,5.22E2,-6.796784E-3,6.03257E5,1.7377544E7,9.639932E0,3.4129692E-3,2.4600944E-3,4.726046E-3,1.4654844E-3,1.0412661E-2,2.9651852E0,2.2237627E-2,4.8239273E-3,1.0038E4,-6.8662115E-3,-1.2421554E-3,-4.930933E-3,-7.008409E-5,7.121916E2,4.2907475E-3,2.3177492E6,5.427002E-4,-4.8866915E-3,1.1576109E-3,5.7679447E-3,9.287843E-3,3.489136E-4,-6.5392083E-3,-2.1009946E-3,1.0270569E7,4.735985E6,1.3482595E8,-6.010395E-4,-3.894747E-3,2.6668124E-3,5.3358534E-3,6.879782E-4],"split_indices":[2,1,66,9,8,9,4,3,0,1,41,0,0,47,10,0,0,8,13,0,59,7,9,40,3,0,41,57,70,69,0,0,0,0,65,0,0,0,0,0,0,0,64,0,59,0,0,0,0,0,0,0,0,57,1,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.07E2,1.26E2,6.8E1,2.39E2,9E0,1.17E2,5.6E1,1.2E1,1.97E2,4.2E1,5E0,4E0,9.3E1,2.4E1,4.6E1,1E1,7.2E1,1.25E2,6E0,3.6E1,4.3E1,5E1,8E0,1.6E1,1.1E1,6.1E1,5E1,7.5E1,1.3E1,2.3E1,1.5E1,2.8E1,1.3E1,3.7E1,7E0,1E0,1.4E1,2E0,4.7E1,1.4E1,2E1,3E1,1E1,6.5E1,9E0,4E0,8E0,2.9E1,1.2E1,2E0,1.1E1,1.9E1,5.2E1,1.3E1,1.9E1,3.3E1,1.1E1,2E0,6E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[1.9057722E-4,-4.3841608E-2,9.4081335E-2,-1.073289E-1,-3.112049E-2,1.1287331E-1,-1.5398487E-1,-7.144994E-3,-3.2850541E-3,-6.172148E-2,-1.30347125E-2,9.499429E-2,1.46319885E-2,-2.5451863E-1,8.267714E-3,-4.3367893E-2,-1.19960986E-1,-2.7721755E-2,1.5484613E-3,1.03856444E-1,-1.2428917E-1,-1.5503317E-2,-2.9129505E-3,-5.136403E-2,1.7958932E-3,1.8471581E-3,-7.949092E-3,-4.6263598E-2,1.3117957E-2,9.361801E-2,2.2392258E-2,6.915568E-3,-1.1222142E-2,-7.501377E-2,-5.7321985E-4,-1.6360085E-2,-7.888432E-2,-1.2085952E-3,5.2499683E-3,2.5931022E-3,1.20249115E-1,-7.0243017E-3,-4.8482165E-2,-3.6388896E-2,1.9197523E-3,-2.2173068E-3,-5.735838E-3,1.4044355E-3,-3.1177495E-3,1.2897697E-2,9.986786E-2,-6.7056715E-4,-4.377773E-3,-8.7089435E-4,-5.078734E-3,5.425433E-3,1.7690935E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,21,-1,23,25,27,-1,29,31,-1,-1,33,-1,-1,-1,35,37,39,-1,-1,-1,41,-1,43,45,47,-1,-1,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7611661E0,2.308724E-1,6.5136576E-1,6.1760783E-2,1.3368206E-1,3.8068974E-1,3.6322904E-1,0E0,0E0,9.099886E-2,1.0042076E-1,2.4027705E-1,0E0,6.935328E-2,0E0,4.623398E-2,1.3471735E-1,8.8447124E-2,0E0,3.5627663E-1,1.6274893E-1,0E0,0E0,5.952272E-2,0E0,0E0,0E0,7.67474E-2,4.8858315E-2,1.1712009E-1,0E0,0E0,0E0,6.2740415E-2,0E0,4.8570897E-2,4.1979298E-2,6.0926147E-2,0E0,0E0,1.6243702E-1,0E0,3.9971158E-2,3.781368E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.98569E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,13,13,15,15,16,16,17,17,19,19,20,20,23,23,27,27,28,28,29,29,33,33,35,35,36,36,37,37,40,40,42,42,43,43,50,50],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,22,-1,24,26,28,-1,30,32,-1,-1,34,-1,-1,-1,36,38,40,-1,-1,-1,42,-1,44,46,48,-1,-1,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,5.53564E5,1.0429407E3,3.606E3,6.505339E2,1.6941456E1,3.236383E1,-7.144994E-3,-3.2850541E-3,1.7377544E7,1.2472753E5,3.7826266E4,1.46319885E-2,3.2991206E3,8.267714E-3,1.6285607E7,6.465659E4,7.10711E8,1.5484613E-3,3.305542E4,1.92144E5,-1.5503317E-2,-2.9129505E-3,3.006846E6,1.7958932E-3,1.8471581E-3,-7.949092E-3,2.6366013E1,9.78696E4,1.2511627E7,2.2392258E-2,6.915568E-3,-1.1222142E-2,4.9256845E2,-5.7321985E-4,6.0070217E1,1E0,2.1035208E7,5.2499683E-3,2.5931022E-3,1.6459E4,-7.0243017E-3,4.8E1,4.086885E7,1.9197523E-3,-2.2173068E-3,-5.735838E-3,1.4044355E-3,-3.1177495E-3,1.2897697E-2,6.029086E10,-6.7056715E-4,-4.377773E-3,-8.7089435E-4,-5.078734E-3,5.425433E-3,1.7690935E-4],"split_indices":[2,1,70,2,64,47,68,0,0,57,40,64,0,64,0,57,40,7,0,4,2,0,0,9,0,0,0,65,40,1,0,0,0,45,0,68,26,9,0,0,9,0,8,57,0,0,0,0,0,0,5,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,2.89E2,1.35E2,4.7E1,2.42E2,1.26E2,9E0,2.4E1,2.3E1,8.9E1,1.53E2,1.16E2,1E1,7E0,2E0,6.9E1,2E1,1.15E2,3.8E1,1.12E2,4E0,5E0,2E0,6.3E1,6E0,4E0,1.6E1,7.9E1,3.6E1,1.1E2,2E0,1E0,3E0,3.9E1,2.4E1,4.2E1,3.7E1,3.2E1,4E0,4.4E1,6.6E1,1E1,2.9E1,3.1E1,1.1E1,2E1,1.7E1,2.2E1,1E1,7E0,5.9E1,1.6E1,1.3E1,2.5E1,6E0,5.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-5.0757453E-3,-4.1375384E-2,7.95706E-2,-4.409644E-2,1.2582895E-2,9.641743E-2,-1.1604129E-1,-5.9519663E-2,7.2841127E-3,7.9281606E-2,1.40994545E-2,1.0094637E-2,-1.6957028E-2,-8.08756E-2,-3.0951012E-2,1.7707981E-2,-8.713262E-3,8.757089E-2,-1.18939005E-1,1.1394761E-2,-5.3607352E-2,-1.22296095E-1,-6.735318E-2,-1.5308324E-3,-6.597227E-2,5.1729707E-3,4.758531E-3,6.392052E-2,1.307E-1,6.7427615E-3,-1.0801513E-2,-5.2492185E-3,2.497498E-3,-9.160124E-4,-1.3649392E-1,-4.2416914E-3,-1.97244E-3,-1.0990009E-3,2.0263288E-3,2.0269063E-4,-1.022466E-1,-3.4250531E-4,4.1252333E-3,9.712118E-2,2.709981E-2,1.6429149E-2,1.1056611E-1,-7.962517E-3,-2.2606745E-3,-1.6814854E-3,-1.287812E-1,5.3988313E-3,1.7634178E-4,-9.655979E-4,2.5545456E-3,6.2915757E-3,6.474933E-4,-9.946406E-3,-3.7195652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,21,23,25,-1,27,29,-1,31,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,49,-1,-1,51,53,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3613955E0,2.5838578E-1,4.494819E-1,2.4450928E-1,0E0,3.6904764E-1,3.1319946E-1,1.42093E-1,1.394061E-1,1.9772983E-1,0E0,1.23272926E-1,0E0,6.737715E-2,1.0602317E-1,7.617162E-2,0E0,1.038478E-1,1.5252924E-1,0E0,4.253727E-2,4.4733644E-2,4.7463417E-2,4.9961727E-2,1.1980499E-1,0E0,5.566335E-2,8.660847E-2,1.1517072E-1,0E0,0E0,0E0,0E0,0E0,5.1135838E-2,0E0,0E0,0E0,0E0,0E0,5.2096248E-2,0E0,0E0,3.8410246E-2,4.1121352E-2,0E0,5.222258E-2,0E0,0E0,0E0,6.9182396E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,17,17,18,18,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,34,34,40,40,43,43,44,44,46,46,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,22,24,26,-1,28,30,-1,32,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,50,-1,-1,52,54,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,3.2991206E3,1.0429407E3,7.164074E8,1.2582895E-2,1.146155E1,6.6053805E6,1.732734E6,1E0,3.305542E4,1.40994545E-2,1.3812E4,-1.6957028E-2,4.94E4,1.8071064E7,1.7608592E-4,-8.713262E-3,5.8145845E3,1.92144E5,1.1394761E-2,1.6130411E5,1.4297844E2,7.37642E4,1.1669E4,6.465659E4,5.1729707E-3,1E0,1.1532659E7,3.7137297E3,6.7427615E-3,-1.0801513E-2,-5.2492185E-3,2.497498E-3,-9.160124E-4,2.0454705E-3,-4.2416914E-3,-1.97244E-3,-1.0990009E-3,2.0263288E-3,2.0269063E-4,2.7E1,-3.4250531E-4,4.1252333E-3,2.4585695E0,2.326757E9,1.6429149E-2,3.0038656E5,-7.962517E-3,-2.2606745E-3,-1.6814854E-3,2.73E2,5.3988313E-3,1.7634178E-4,-9.655979E-4,2.5545456E-3,6.2915757E-3,6.474933E-4,-9.946406E-3,-3.7195652E-3],"split_indices":[2,64,70,7,0,47,59,1,19,4,0,9,0,9,57,50,0,4,2,0,40,45,40,10,40,0,101,57,44,0,0,0,0,0,51,0,0,0,0,0,8,0,0,54,12,0,45,0,0,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,3.09E2,1.32E2,3.07E2,2E0,1.22E2,1E1,2.36E2,7.1E1,1.13E2,9E0,7E0,3E0,1.34E2,1.02E2,6.8E1,3E0,1.09E2,4E0,1E0,6E0,3.1E1,1.03E2,5.6E1,4.6E1,8E0,6E1,7.2E1,3.7E1,1E0,3E0,4E0,2E0,4E0,2.7E1,6.2E1,4.1E1,3.8E1,1.8E1,1.6E1,3E1,5.3E1,7E0,3.7E1,3.5E1,2E0,3.5E1,2.1E1,6E0,9E0,2.1E1,3.3E1,4E0,1.2E1,2.3E1,3E1,5E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.5044258E-3,-3.7345733E-2,7.6977715E-2,-4.0150125E-2,1.2163665E-2,-1.7551139E-1,9.039395E-2,-7.624012E-2,-2.4069777E-2,-1.5107096E-3,-1.3839191E-2,8.135891E-2,1.2870588E-2,-9.3174994E-2,5.7902485E-3,3.3054322E-2,-3.3561226E-2,9.200262E-2,-1.379793E-1,-1.0300372E-1,-1.0471808E-3,-1.4885436E-3,3.5006606E-3,-1.6533234E-4,5.121249E-3,-1.5047007E-2,-6.8765625E-2,8.211675E-2,2.183945E-2,6.574274E-3,-1.0904508E-2,-4.6008118E-4,-1.10796645E-1,-3.1999074E-2,2.2571333E-2,-7.723607E-2,1.9072073E-3,1.3323898E-2,7.812477E-2,-1.1685783E-1,8.791054E-5,-8.757582E-4,-6.648194E-2,1.6622155E-3,-3.5610355E-3,-3.2211542E-3,-9.603749E-3,-2.853789E-4,8.6319536E-2,-4.731293E-3,-8.105512E-3,-6.27397E-3,-1.3301288E-3,6.0308803E-2,5.5308673E-3,8.134608E-3,2.0366893E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,37,-1,-1,-1,-1,39,41,43,45,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,55,-1,-1,-1],"loss_changes":[1.1842937E0,2.3729962E-1,4.6300906E-1,1.6472238E-1,0E0,9.475586E-2,1.6932309E-1,1.2278688E-1,1.0954743E-1,0E0,0E0,2.951777E-1,0E0,4.9660265E-2,3.866151E-2,7.354301E-2,1.11071765E-1,3.5974216E-1,1.584245E-1,4.5761347E-2,0E0,0E0,0E0,0E0,0E0,7.342104E-2,5.6385487E-2,6.312978E-2,0E0,0E0,0E0,0E0,4.0456533E-2,3.826846E-2,3.8474225E-2,6.84126E-2,0E0,0E0,7.812494E-2,4.0965974E-2,0E0,0E0,5.0810494E-2,0E0,0E0,0E0,0E0,0E0,5.856377E-2,0E0,0E0,0E0,0E0,9.755808E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,27,27,32,32,33,33,34,34,35,35,38,38,39,39,42,42,48,48,53,53],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,38,-1,-1,-1,-1,40,42,44,46,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,56,-1,-1,-1],"split_conditions":[1.2934E4,3.2991206E3,2.4950776E0,1.5031561E3,1.2163665E-2,6.27427E5,1.6828056E1,2.1592189E5,4.3712845E0,-1.5107096E-3,-1.3839191E-2,3.768034E4,1.2870588E-2,6.209E3,4.18E3,8.6665695E1,1.8457164E7,3.305542E4,1.92144E5,5.895129E1,-1.0471808E-3,-1.4885436E-3,3.5006606E-3,-1.6533234E-4,5.121249E-3,9.69E2,9.407814E6,1.2662238E1,2.183945E-2,6.574274E-3,-1.0904508E-2,-4.6008118E-4,1.3421E4,5.559E3,2.0136225E5,2.231E3,1.9072073E-3,1.3323898E-2,6E0,4.02959E5,8.791054E-5,-8.757582E-4,5.787671E8,1.6622155E-3,-3.5610355E-3,-3.2211542E-3,-9.603749E-3,-2.853789E-4,1.648428E-1,-4.731293E-3,-8.105512E-3,-6.27397E-3,-1.3301288E-3,1.8008743E3,5.5308673E-3,8.134608E-3,2.0366893E-3],"split_indices":[2,64,66,4,0,9,47,40,70,0,0,4,0,2,2,59,57,4,2,62,0,0,0,0,0,0,1,68,0,0,0,0,10,10,45,0,0,0,8,41,0,0,5,0,0,0,0,0,50,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,2.88E2,1.31E2,2.86E2,2E0,6E0,1.25E2,8.7E1,1.99E2,3E0,3E0,1.2E2,5E0,7.2E1,1.5E1,2.8E1,1.71E2,1.15E2,5E0,6.3E1,9E0,1E1,5E0,1.9E1,9E0,1.13E2,5.8E1,1.13E2,2E0,1E0,4E0,5E0,5.8E1,7.8E1,3.5E1,5.4E1,4E0,1E0,1.12E2,5.5E1,3E0,5.6E1,2.2E1,3.2E1,3E0,5E1,4E0,1E1,1.02E2,3.9E1,1.6E1,8E0,1.4E1,5.1E1,5.1E1,7E0,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-3.5769674E-3,-4.46986E-2,8.625172E-2,-5.7014383E-2,7.03394E-3,4.8121482E-2,1.949098E-1,-6.6504985E-2,-3.1469143E-3,5.966327E-3,-3.4990686E-3,5.664029E-2,-4.8786397E-3,3.587558E-1,1.0486555E-1,-7.352405E-2,-1.3307479E-2,-4.930942E-3,1.142104E-2,-7.999162E-4,3.5191367E-3,-1.6422821E-4,6.595374E-2,7.388784E-3,2.062171E-2,1.5791379E-1,-9.987926E-3,-9.800828E-2,-5.667645E-2,-1.16811685E-1,9.200175E-4,-5.3346355E-4,3.1638825E-3,7.9098895E-2,4.7444645E-4,5.802352E-3,1.3418038E-2,-1.2714735E-1,-5.7611093E-2,-2.8463665E-2,-4.5299237E-3,-3.1593223E-5,-8.732991E-3,5.3880274E-2,5.208167E-3,-1.4484613E-3,-7.153948E-3,-4.0786555E-3,1.9873406E-3,-5.9464197E-3,-5.179148E-3,5.455523E-3,1.420283E-3,-1.1146478E-3,2.5538218E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,-1,31,-1,-1,-1,33,-1,-1,35,-1,37,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1],"loss_changes":[1.599621E0,1.8962765E-1,5.5409014E-1,1.2248701E-1,6.90538E-2,1.3216622E-1,4.787476E-1,7.51946E-2,5.261303E-2,0E0,5.1047616E-2,5.435452E-2,0E0,7.5348735E-2,4.19516E-1,6.869531E-2,8.4220536E-2,0E0,3.8509037E-2,0E0,0E0,0E0,6.159213E-2,0E0,0E0,6.5916836E-2,0E0,7.742876E-2,1.0187012E-1,4.02724E-2,0E0,0E0,0E0,3.745264E-2,0E0,0E0,0E0,6.0247183E-2,7.803415E-2,1.0263209E-1,0E0,0E0,0E0,4.7569335E-2,0E0,0E0,0E0,0E0,0E0,4.593019E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,18,18,22,22,25,25,27,27,28,28,29,29,33,33,37,37,38,38,39,39,43,43,49,49],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,-1,32,-1,-1,-1,34,-1,-1,36,-1,38,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1],"split_conditions":[1.1153E4,7.10711E8,6.204522E3,3.4863858E6,2.829932E0,9.05929E2,1.417988E6,2.1592189E5,7.6807027E0,5.966327E-3,1E0,1E1,-4.8786397E-3,9.256843E3,1.5096262E7,3.88672E5,6.52487E2,-4.930942E-3,6.8480647E10,-7.999162E-4,3.5191367E-3,-1.6422821E-4,7.789796E1,7.388784E-3,2.062171E-2,1.6828056E1,-9.987926E-3,7.121916E2,3.5389444E5,1.0029164E-6,9.200175E-4,-5.3346355E-4,3.1638825E-3,1.93536E5,4.7444645E-4,5.802352E-3,1.3418038E-2,5.8955963E1,3.685938E6,3.271774E7,-4.5299237E-3,-3.1593223E-5,-8.732991E-3,4.41341E5,5.208167E-3,-1.4484613E-3,-7.153948E-3,-4.0786555E-3,1.9873406E-3,2.143469E6,-5.179148E-3,5.455523E-3,1.420283E-3,-1.1146478E-3,2.5538218E-3],"split_indices":[2,7,64,62,70,70,41,40,65,0,101,8,0,4,9,9,4,0,43,0,0,0,68,0,0,47,0,64,60,50,0,0,0,11,0,0,0,62,1,57,0,0,0,41,0,0,0,0,0,1,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.96E2,1.35E2,2.39E2,5.7E1,1.01E2,3.4E1,2.03E2,3.6E1,4E0,5.3E1,9.6E1,5E0,1.1E1,2.3E1,1.79E2,2.4E1,4E0,3.2E1,4.6E1,7E0,1.3E1,8.3E1,3E0,8E0,2E1,3E0,7.1E1,1.08E2,5E0,1.9E1,2.3E1,9E0,6.7E1,1.6E1,1.6E1,4E0,4E1,3.1E1,6E1,4.8E1,2E0,3E0,3.5E1,3.2E1,6E0,3.4E1,2.5E1,6E0,4.7E1,1.3E1,1E1,2.5E1,3.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.1722329E-3,-2.2786593E-2,1.247586E-1,-4.923096E-2,2.8184226E-2,3.0928555E-1,8.410662E-2,-5.501502E-2,3.0558933E-2,3.7491437E-2,-1.3246593E-2,3.7163594E-3,1.9395031E-2,1.4031406E-2,7.088948E-2,-6.759944E-2,-1.2662115E-2,-3.952504E-3,2.5213454E-3,1.6902057E-2,3.9876844E-3,1.4128439E-2,1.17010035E-1,-8.941428E-2,-2.311165E-3,-2.9152814E-2,1.5521493E-3,7.671637E-3,5.86423E-3,4.380074E-2,-7.25569E-3,1.4798077E-3,6.7329067E-3,6.1095477E-4,-9.728847E-2,-6.7934254E-4,-4.914642E-3,6.539371E-4,-6.598049E-3,4.6469066E-3,-4.1690386E-5,-1.05115004E-1,-1.2362063E-3,-9.370906E-2,-8.659715E-3,-5.4381443E-3,-2.125715E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,-1,27,-1,29,31,33,-1,35,-1,37,-1,39,-1,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,43,-1,45,-1,-1,-1],"loss_changes":[1.1705441E0,4.9606353E-1,4.4657457E-1,1.13366485E-1,3.538259E-1,1.747737E-1,1.1785588E-1,1.19598925E-1,4.106889E-2,1.05776906E-1,0E0,0E0,0E0,0E0,1.3185722E-1,7.692027E-2,3.911697E-2,0E0,0E0,7.697545E-2,0E0,1.19810596E-1,3.9356798E-2,6.9217026E-2,0E0,4.0647257E-2,0E0,6.071301E-2,0E0,4.61774E-2,0E0,0E0,0E0,0E0,4.22588E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.896159E-2,0E0,4.4072747E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,19,19,21,21,22,22,23,23,25,25,27,27,29,29,34,34,41,41,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,-1,28,-1,30,32,34,-1,36,-1,38,-1,40,-1,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,44,-1,46,-1,-1,-1],"split_conditions":[3.2991206E3,7.10711E8,1.37893E6,2.889654E5,1.190724E4,1.082556E7,8.169258E3,3.447954E5,5.012955E6,1.9795151E9,-1.3246593E-2,3.7163594E-3,1.9395031E-2,1.4031406E-2,1.9078612E-1,2.73E2,1.7657828E3,-3.952504E-3,2.5213454E-3,1E0,3.9876844E-3,2.5212732E4,3.1063303E5,5.8601086E1,-2.311165E-3,4.0266477E8,1.5521493E-3,1E0,5.86423E-3,3.7057E5,-7.25569E-3,1.4798077E-3,6.7329067E-3,6.1095477E-4,1.3421E4,-6.7934254E-4,-4.914642E-3,6.539371E-4,-6.598049E-3,4.6469066E-3,-4.1690386E-5,4.816E3,-1.2362063E-3,1.6395413E7,-8.659715E-3,-5.4381443E-3,-2.125715E-3],"split_indices":[64,7,41,40,4,1,45,59,57,7,0,0,0,0,50,3,64,0,0,101,0,4,44,62,0,7,0,19,0,9,0,0,0,0,10,0,0,0,0,0,0,10,0,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,3.66E2,6.2E1,2.41E2,1.25E2,1E1,5.2E1,2.25E2,1.6E1,1.22E2,3E0,3E0,7E0,2E0,5E1,1.73E2,5.2E1,2E0,1.4E1,8.3E1,3.9E1,2.3E1,2.7E1,8.4E1,8.9E1,3.8E1,1.4E1,7.7E1,6E0,2E1,3E0,5E0,2.2E1,6E0,7.8E1,3.2E1,6E0,7.5E1,2E0,9E0,1.1E1,7E1,8E0,6.2E1,8E0,4.7E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.9097693E-3,-2.9472075E-2,7.402421E-2,-6.247058E-2,-4.235935E-3,-7.856033E-3,8.60728E-2,-7.396903E-2,-2.7621519E-3,4.452734E-2,-2.8582273E-2,1.4437816E-2,7.145314E-2,-9.398713E-2,-4.2333867E-2,-6.7318873E-3,8.1050984E-4,1.3003242E-1,2.2678599E-2,-8.1427254E-2,-1.4522827E-2,4.0605575E-2,1.2538108E-1,-1.2326426E-3,-1.0397442E-1,-5.4636825E-4,-3.957893E-3,3.7241462E-3,1.1116206E-2,3.3514205E-2,-7.7282395E-3,-5.827089E-3,-1.3026411E-3,-6.6318964E-3,-4.639639E-4,5.3676233E-2,-8.929544E-3,9.8143436E-2,3.255558E-1,-1.4418462E-1,-3.4475864E-3,-1.2340529E-3,2.5980375E-3,1.3109803E-1,4.027494E-2,6.953521E-3,2.1654468E-3,3.2724515E-3,1.9511703E-2,-7.735769E-3,-3.87298E-4,4.3679236E-5,8.4049795E-3,1.2731963E-3,5.287566E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,-1,41,-1,-1,-1,-1,-1,43,-1,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.958566E-1,2.6122037E-1,3.3112007E-1,9.3167126E-2,2.1378279E-1,0E0,3.0256265E-1,6.8214595E-2,5.978201E-2,1.0914175E-1,8.7974E-2,0E0,1.6625404E-1,4.5207202E-2,5.2379638E-2,0E0,0E0,4.4977278E-2,9.932829E-2,4.5638263E-2,5.820647E-2,2.0162332E-1,1.6978347E-1,0E0,7.447004E-2,0E0,0E0,0E0,0E0,5.1816512E-2,0E0,0E0,0E0,0E0,0E0,6.1123386E-2,0E0,7.03696E-2,4.1914314E-2,3.7865818E-2,0E0,0E0,0E0,4.3313727E-2,5.1609047E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,29,29,35,35,37,37,38,38,39,39,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,-1,42,-1,-1,-1,-1,-1,44,-1,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5704E4,1.6861708E3,1.4144695E0,2.1592189E5,1.24E2,-7.856033E-3,9.957E3,1.656968E6,3.516E3,3.34031E5,9.04782E5,1.4437816E-2,2.1383806E-1,5.8981678E1,1.33E2,-6.7318873E-3,8.1050984E-4,7.09E3,1E0,6.2E1,1.32E2,1.4609149E4,9.920844E0,-1.2326426E-3,2.7276272E-2,-5.4636825E-4,-3.957893E-3,3.7241462E-3,1.1116206E-2,1.1327554E6,-7.7282395E-3,-5.827089E-3,-1.3026411E-3,-6.6318964E-3,-4.639639E-4,1.8297544E3,-8.929544E-3,2.27901E5,1.281E3,4.61E2,-3.4475864E-3,-1.2340529E-3,2.5980375E-3,1.504E4,8.0200186E3,6.953521E-3,2.1654468E-3,3.2724515E-3,1.9511703E-2,-7.735769E-3,-3.87298E-4,4.3679236E-5,8.4049795E-3,1.2731963E-3,5.287566E-3],"split_indices":[2,67,65,40,3,0,9,9,12,12,1,0,50,62,3,0,0,2,19,8,3,4,65,0,54,0,0,0,0,60,0,0,0,0,0,4,0,9,0,3,0,0,0,9,4,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,3.13E2,1.13E2,1.35E2,1.78E2,5E0,1.08E2,1.13E2,2.2E1,5.9E1,1.19E2,6E0,1.02E2,6.8E1,4.5E1,2E0,2E1,1.1E1,4.8E1,2.4E1,9.5E1,6.6E1,3.6E1,9E0,5.9E1,2.5E1,2E1,8E0,3E0,4.6E1,2E0,1.4E1,1E1,3E0,9.2E1,6.3E1,3E0,3.3E1,3E0,2.6E1,3.3E1,1.1E1,3.5E1,8E0,5.5E1,1.8E1,1.5E1,1E0,2E0,2.4E1,2E0,2E0,6E0,4.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.6762093E-3,-4.329192E-2,6.305308E-2,-4.6974506E-2,1.5428075E-2,2.5035758E-2,1.2997788E-1,-4.245986E-2,-1.41069135E-2,-3.7218716E-2,4.466814E-2,9.261448E-2,2.9632443E-1,-8.666192E-2,-3.0281452E-2,-8.339047E-3,-9.34087E-4,6.776784E-4,7.241056E-2,5.717102E-3,-1.529093E-3,7.1113985E-3,1.8755725E-2,-5.008331E-3,-2.6358923E-2,-4.7315866E-2,6.988253E-3,4.225571E-3,-6.24984E-4,-6.507599E-3,3.715007E-4,-2.5326205E-2,-8.062196E-2,-7.988225E-4,2.3558754E-3,-4.8004776E-2,4.282462E-2,-9.167577E-3,-3.5822098E-3,-6.590091E-2,2.682642E-4,-1.2775036E-3,7.4341536E-2,-4.3635294E-3,-1.2616031E-3,8.261517E-3,2.0306706E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,-1,27,-1,-1,-1,-1,-1,29,31,33,-1,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,43,-1,-1,45,-1,-1,-1,-1],"loss_changes":[1.1378195E0,3.7356943E-1,3.9874554E-1,2.801445E-1,0E0,1.2632728E-1,3.2682663E-1,1.4325795E-1,0E0,5.693157E-2,6.6333175E-2,1.3232324E-1,6.8232715E-2,4.475984E-2,1.3619228E-1,0E0,0E0,0E0,4.332812E-2,0E0,0E0,0E0,0E0,0E0,4.3082323E-2,1.0561943E-1,6.3475415E-2,0E0,0E0,0E0,0E0,1.4115648E-1,4.0184945E-2,0E0,0E0,6.530486E-2,5.1473275E-2,0E0,0E0,4.2192414E-2,0E0,0E0,4.2345524E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,24,24,25,25,26,26,31,31,32,32,35,35,36,36,39,39,42,42],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,-1,28,-1,-1,-1,-1,-1,30,32,34,-1,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,44,-1,-1,46,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,5.6503E4,7.339209E6,1.5428075E-2,1.0643513E3,1.2123712E1,3.836E3,-1.41069135E-2,1E0,7.959718E4,4.42384E5,1.92144E5,1.7212875E5,1.2028214E5,-8.339047E-3,-9.34087E-4,6.776784E-4,2.237815E7,5.717102E-3,-1.529093E-3,7.1113985E-3,1.8755725E-2,-5.008331E-3,3.516E3,3.4129692E-3,2.735322E6,4.225571E-3,-6.24984E-4,-6.507599E-3,3.715007E-4,7.198E3,7.07812E7,-7.988225E-4,2.3558754E-3,1.54E2,5.625619E1,-9.167577E-3,-3.5822098E-3,1.0599062E3,2.682642E-4,-1.2775036E-3,4.9832974E2,-4.3635294E-3,-1.2616031E-3,8.261517E-3,2.0306706E-3],"split_indices":[45,64,2,59,0,64,65,2,0,13,40,2,2,40,40,0,0,0,57,0,0,0,0,0,12,69,1,0,0,0,0,2,7,0,0,8,68,0,0,45,0,0,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.76E2,1.57E2,2.74E2,2E0,1.01E2,5.6E1,2.7E2,4E0,2.4E1,7.7E1,4.7E1,9E0,5.7E1,2.13E2,2E0,2.2E1,3.7E1,4E1,4E1,7E0,4E0,5E0,4.6E1,1.1E1,1.46E2,6.7E1,3.5E1,5E0,2E0,9E0,8.9E1,5.7E1,4.3E1,2.4E1,6.7E1,2.2E1,3E0,5.4E1,5E1,1.7E1,7E0,1.5E1,3.2E1,1.8E1,3E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.83888E-3,-3.1505685E-2,9.495612E-2,-4.551759E-2,1.0406145E-2,2.4448071E-1,7.442078E-2,-6.255245E-2,-2.5327334E-2,1.8394146E-2,-7.7937418E-3,1.703688E-2,4.56002E-3,-1.2077044E-1,8.944308E-2,-3.6171337E-3,-8.0137077E-4,-5.765748E-2,-9.046284E-3,-8.121633E-4,3.5697255E-2,-2.9794502E-4,-1.358657E-2,7.414078E-2,1.0601171E-2,-2.0557244E-3,-8.085591E-3,-2.095319E-2,2.38548E-3,6.705194E-2,5.530207E-4,1.8580599E-1,6.0514547E-2,6.6048084E-3,-2.1355385E-3,6.4281086E-3,1.7058217E-3,3.2917245E-3,1.4457056E-2,7.4992E-2,-4.1262205E-3,1.7764501E-3,-2.3296846E-3,-3.9353026E-3,3.9293445E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,-1,-1,25,27,-1,29,-1,-1,31,-1,-1,-1,33,-1,35,-1,37,39,41,-1,-1,-1,-1,-1,43,-1,-1,-1,-1,-1],"loss_changes":[1.2607517E0,1.9779453E-1,2.980711E-1,8.508819E-2,1.1582068E-1,1.4619565E-1,2.8056633E-1,6.0994923E-2,6.1063565E-2,4.973661E-2,0E0,0E0,0E0,1.1980515E-1,1.4572191E-1,0E0,0E0,6.0448393E-2,5.4408792E-2,0E0,4.1735314E-2,0E0,0E0,1.0743901E-1,0E0,0E0,0E0,3.98927E-2,0E0,4.4255123E-2,0E0,8.023122E-2,1.5326184E-1,4.7865223E-2,0E0,0E0,0E0,0E0,0E0,4.2490035E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,17,17,18,18,20,20,23,23,27,27,29,29,31,31,32,32,33,33,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,-1,-1,26,28,-1,30,-1,-1,32,-1,-1,-1,34,-1,36,-1,38,40,42,-1,-1,-1,-1,-1,44,-1,-1,-1,-1,-1],"split_conditions":[2.1722E4,7.164074E8,9.73307E5,1.732734E6,1E0,2.3879834E9,1.9813E4,3.0946184E5,7.121916E2,2.0069241E3,-7.7937418E-3,1.703688E-2,4.56002E-3,7.600717E6,1.6941456E1,-3.6171337E-3,-8.0137077E-4,6.97E3,7.1999524E9,-8.121633E-4,1.5E2,-2.9794502E-4,-1.358657E-2,1.3812E4,1.0601171E-2,-2.0557244E-3,-8.085591E-3,1.4040858E-2,2.38548E-3,6.72E2,5.530207E-4,2.9632012E7,1.4656219E8,2.515304E7,-2.1355385E-3,6.4281086E-3,1.7058217E-3,3.2917245E-3,1.4457056E-2,1.700398E4,-4.1262205E-3,1.7764501E-3,-2.3296846E-3,-3.9353026E-3,3.9293445E-3],"split_indices":[2,7,41,1,19,5,10,59,64,67,0,0,0,1,47,0,0,2,5,0,3,0,0,9,0,0,0,69,0,0,0,1,1,57,0,0,0,0,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.35E2,1.02E2,2.51E2,8.4E1,1.1E1,9.1E1,1.35E2,1.16E2,8.1E1,3E0,6E0,5E0,6E0,8.5E1,1.11E2,2.4E1,3.8E1,7.8E1,2.7E1,5.4E1,4E0,2E0,7.7E1,8E0,3.4E1,4E0,6.5E1,1.3E1,2.3E1,3.1E1,7E0,7E1,2.9E1,3.6E1,7E0,1.6E1,4E0,3E0,6.4E1,6E0,1.9E1,1E1,1E0,6.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.6657364E-3,-1.6230375E-2,1.4032856E-1,-3.0203806E-2,6.1110664E-2,8.405176E-2,2.2616886E-1,-2.7168322E-2,-1.3988816E-2,6.905196E-2,-5.901448E-3,1.297636E-2,6.4346895E-2,6.0318024E-3,1.707545E-2,-3.9163496E-2,1.7857725E-2,1.10801635E-2,5.996534E-2,3.6801326E-3,-5.356998E-3,-5.2230526E-3,-3.404043E-2,4.5894124E-2,-1.1456036E-3,1.0188965E-1,3.341668E-2,-3.984543E-2,4.926508E-4,1.2162745E-3,5.727184E-3,1.1536736E-3,7.260139E-3,5.3720153E-4,3.985338E-3,-4.1311376E-2,7.0631974E-3,-4.5591306E-2,1.2454672E-3,2.9451582E-3,-2.3959118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,19,-1,-1,21,23,-1,25,-1,-1,-1,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,37,-1,39,-1,-1,-1],"loss_changes":[1.12889E0,4.373759E-1,2.3219264E-1,2.5162446E-1,9.638305E-2,9.7477704E-2,2.0241714E-1,1.837917E-1,0E0,6.993353E-2,0E0,0E0,5.7299808E-2,0E0,0E0,8.61634E-2,8.337292E-2,0E0,6.1151147E-2,0E0,0E0,0E0,6.409994E-2,6.0896553E-2,0E0,7.185948E-2,3.799819E-2,6.453535E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.349981E-2,0E0,5.3667307E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,12,12,15,15,16,16,18,18,22,22,23,23,25,25,26,26,27,27,35,35,37,37],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,20,-1,-1,22,24,-1,26,-1,-1,-1,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,38,-1,40,-1,-1,-1],"split_conditions":[4.23168E3,1.5522031E9,2.2876814E1,3.764913E7,1.190724E4,8.169258E3,2.235E5,1.5135906E3,-1.3988816E-2,1.3E1,-5.901448E-3,1.297636E-2,5.22E2,6.0318024E-3,1.707545E-2,1.44E2,6.878971E1,1.10801635E-2,6.3504875E6,3.6801326E-3,-5.356998E-3,-5.2230526E-3,2.0964778E5,1.6130411E5,-1.1456036E-3,2.867292E3,9.9179E4,1.9485E4,4.926508E-4,1.2162745E-3,5.727184E-3,1.1536736E-3,7.260139E-3,5.3720153E-4,3.985338E-3,4.79E2,7.0631974E-3,5.949655E0,1.2454672E-3,2.9451582E-3,-2.3959118E-3],"split_indices":[64,7,66,60,4,45,2,64,0,3,0,0,3,0,0,0,68,0,57,0,0,0,40,40,0,4,2,2,0,0,0,0,0,0,0,8,0,68,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.53E2,4.02E2,5.1E1,3.41E2,6.1E1,3.2E1,1.9E1,3.38E2,3E0,5.9E1,2E0,2E0,3E1,1.1E1,8E0,2.67E2,7.1E1,2E0,5.7E1,2.9E1,1E0,1.8E1,2.49E2,4.2E1,2.9E1,2.1E1,3.6E1,2.2E2,2.9E1,3.3E1,9E0,8E0,1.3E1,2.5E1,1.1E1,2.19E2,1E0,2.06E2,1.3E1,4E0,2.02E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-3.8273662E-4,-1.9588554E-2,9.514597E-2,-3.0770771E-2,4.2636964E-2,2.3074453E-1,6.6115856E-2,-2.767215E-2,-1.3464486E-2,8.73124E-3,2.6881015E-2,2.0919892E-3,1.4154424E-2,8.468875E-2,-1.2753686E-1,-3.6554012E-2,1.363874E-2,1.8004556E-3,-4.7055413E-3,6.1234064E-2,1.3903362E-1,5.206573E-3,-9.731181E-3,-3.7750006E-2,6.8867034E-3,3.7153366E-3,-1.6284496E-2,8.0076605E-2,5.792999E-4,9.16635E-3,3.3073185E-3,-3.4658718E-3,-3.0303154E-2,-1.645871E-3,2.1424915E-3,2.4728107E-3,6.9657536E-3,-1.7256556E-2,-5.5613063E-2,-1.2454471E-2,-7.313641E-3,-1.4665913E-3,-3.9152782E-3,-1.9834144E-3,3.5979378E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,-1,-1,19,21,23,25,-1,-1,27,29,-1,-1,31,-1,-1,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1],"loss_changes":[8.3298606E-1,2.6433164E-1,2.8391373E-1,2.2991759E-1,1.1599646E-1,1.1700761E-1,2.4098915E-1,1.17178395E-1,0E0,0E0,6.205896E-2,0E0,0E0,6.6690266E-2,1.1368664E-1,5.9835732E-2,1.0474809E-1,0E0,0E0,3.9293602E-2,4.282996E-2,0E0,0E0,5.9318304E-2,0E0,0E0,3.9820995E-2,4.935424E-2,0E0,0E0,0E0,0E0,6.9374666E-2,0E0,0E0,0E0,0E0,8.62008E-2,4.0688753E-2,7.414715E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,13,13,14,14,15,15,16,16,19,19,20,20,23,23,26,26,27,27,32,32,37,37,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,-1,-1,20,22,24,26,-1,-1,28,30,-1,-1,32,-1,-1,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.6053742E9,1E0,3.372042E7,1.5E1,5.993877E3,3.768034E4,1.4720299E3,-1.3464486E-2,8.73124E-3,2.4070468E7,2.0919892E-3,1.4154424E-2,6.477944E0,1.92144E5,1.7267E4,1.03E2,1.8004556E-3,-4.7055413E-3,1.0392101E0,4.1923336E7,5.206573E-3,-9.731181E-3,5.87114E5,6.8867034E-3,3.7153366E-3,1.2472753E5,8.51E4,5.792999E-4,9.16635E-3,3.3073185E-3,-3.4658718E-3,1.902843E-2,-1.645871E-3,2.1424915E-3,2.4728107E-3,6.9657536E-3,4.086885E7,7.9879355E6,1.59551E5,-7.313641E-3,-1.4665913E-3,-3.9152782E-3,-1.9834144E-3,3.5979378E-4],"split_indices":[64,7,28,60,3,4,4,64,0,0,60,0,0,47,2,2,3,0,0,46,55,0,0,1,0,0,40,2,0,0,0,0,69,0,0,0,0,57,57,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.52E2,3.77E2,7.5E1,3.2E2,5.7E1,1.2E1,6.3E1,3.17E2,3E0,5E0,5.2E1,3E0,9E0,5.8E1,5E0,2.61E2,5.6E1,4.9E1,3E0,4.2E1,1.6E1,1E0,4E0,2.6E2,1E0,1.8E1,3.8E1,3E1,1.2E1,9E0,7E0,4.8E1,2.12E2,3E1,8E0,2.1E1,9E0,1.41E2,7.1E1,1.37E2,4E0,3.4E1,3.7E1,5.7E1,8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.704983E-3,-9.6624885E-3,1.4588639E-1,-3.3298146E-2,3.2194354E-2,2.9020366E-1,7.9218075E-2,-2.974744E-2,-1.295981E-2,5.000202E-3,7.571256E-2,4.355774E-3,1.627426E-2,1.2057051E-1,-1.1857627E-1,-5.752256E-3,-2.414346E-2,-1.0916167E-2,3.6387136E-3,1.05796E-2,6.4446464E-2,4.164135E-3,1.0295639E-2,-1.1102328E-2,1.2256985E-3,-5.749331E-3,-2.218985E-2,-3.7259907E-3,-1.228004E-4,8.3263926E-2,1.6945582E-4,-3.1893063E-2,-1.0061341E-3,-6.431286E-4,9.372181E-2,7.042771E-3,-4.219229E-2,-4.8525423E-2,6.5131055E-4,6.127938E-3,2.539507E-3,1.8619348E-3,-1.8861094E-3,-8.623703E-3,-1.9433085E-3,4.0130914E-4,-4.260224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,-1,33,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.018076E-1,4.1361597E-1,3.233807E-1,2.0656356E-1,1.7832097E-1,5.076772E-2,2.2695808E-1,1.23496264E-1,0E0,1.0224632E-1,7.5211436E-2,0E0,0E0,5.28495E-2,7.955851E-2,0E0,4.166457E-2,4.1267738E-2,0E0,0E0,6.290795E-2,0E0,0E0,0E0,0E0,0E0,5.0327696E-2,0E0,0E0,4.3430537E-2,0E0,6.766665E-2,5.2666456E-2,0E0,4.0882975E-2,5.011632E-2,5.0637454E-2,3.8065236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,16,16,17,17,20,20,26,26,29,29,31,31,32,32,34,34,35,35,36,36,37,37],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,-1,34,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.204522E3,9.7467445E4,1.417988E6,7.339209E6,1.484798E9,9.256843E3,1.5096262E7,1.44E2,-1.295981E-2,1.5271514E5,1.4E1,4.355774E-3,1.627426E-2,1.2472753E5,3.087E3,-5.752256E-3,2.398E3,3.6340196E-2,3.6387136E-3,1.05796E-2,3.2006908E7,4.164135E-3,1.0295639E-2,-1.1102328E-2,1.2256985E-3,-5.749331E-3,1.2028214E5,-3.7259907E-3,-1.228004E-4,8.946923E2,1.6945582E-4,3.2081733E3,4.5765094E2,-6.431286E-4,1.006467E7,3.0840238E7,1.5988636E1,3.251E3,6.5131055E-4,6.127938E-3,2.539507E-3,1.8619348E-3,-1.8861094E-3,-8.623703E-3,-1.9433085E-3,4.0130914E-4,-4.260224E-3],"split_indices":[64,45,41,59,7,4,9,0,0,40,3,0,0,40,0,0,2,50,0,0,1,0,0,0,0,0,40,0,0,64,0,60,64,0,60,57,68,2,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.51E2,4.16E2,3.5E1,2.66E2,1.5E2,1E1,2.5E1,2.63E2,3E0,9.3E1,5.7E1,2E0,8E0,2.1E1,4E0,1.5E1,2.48E2,7.6E1,1.7E1,3E0,5.4E1,1.6E1,5E0,2E0,2E0,4E0,2.44E2,8E0,6.8E1,4.1E1,1.3E1,1.67E2,7.7E1,4E0,3.7E1,3.5E1,1.32E2,1.7E1,6E1,2.1E1,1.6E1,2.1E1,1.4E1,2E0,1.3E2,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.2027072E-3,-2.1871498E-2,7.529484E-2,-1.9424658E-2,-1.4261402E-2,2.046406E-1,6.0886916E-2,-3.410473E-2,2.724388E-2,4.6764086E-3,1.3148219E-2,7.102984E-2,-3.8367254E-3,-2.2410464E-2,-6.263292E-2,-2.9200143E-3,3.0315376E-3,6.2274534E-2,1.0278284E-2,-2.4408007E-3,-2.81583E-3,-2.1828445E-2,-7.761898E-2,-1.4639634E-3,1.9559993E-3,8.757298E-2,3.0189885E-2,1.5609784E-2,-3.195849E-2,1.2093167E-3,-3.7420704E-3,1.4959046E-3,-8.467071E-2,1.431511E-1,3.104124E-3,-1.6171422E-2,2.6983346E-3,6.146431E-6,3.1682611E-3,-4.1960245E-3,-6.234612E-4,-6.8117365E-2,-8.1491545E-3,4.979199E-3,1.771413E-2,-3.4728453E-3,1.624417E-3,-4.4827728E-4,5.2710976E-3,-5.688526E-3,-3.385374E-2,-3.8308282E-3,4.151397E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,19,21,23,-1,25,-1,-1,27,29,31,-1,-1,33,35,37,39,-1,-1,-1,41,43,-1,45,-1,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,51,-1,-1],"loss_changes":[7.4874943E-1,2.0979507E-1,1.8135482E-1,2.2838153E-1,0E0,4.030052E-2,1.3809735E-1,8.3001286E-2,8.081339E-2,0E0,0E0,9.210235E-2,0E0,9.359599E-2,4.2945087E-2,4.884321E-2,0E0,6.634998E-2,0E0,0E0,5.688087E-2,5.3086355E-2,4.2730838E-2,0E0,0E0,5.761653E-2,4.391046E-2,4.846566E-2,4.04927E-2,0E0,0E0,0E0,5.377382E-2,9.3425244E-2,0E0,3.859297E-2,0E0,4.827923E-2,0E0,0E0,0E0,6.348924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.7552392E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,20,20,21,21,22,22,25,25,26,26,27,27,28,28,32,32,33,33,35,35,37,37,41,41,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,20,22,24,-1,26,-1,-1,28,30,32,-1,-1,34,36,38,40,-1,-1,-1,42,44,-1,46,-1,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,52,-1,-1],"split_conditions":[1.615586E9,3.2875448E7,1.3812E4,1.4720299E3,-1.4261402E-2,2.094E3,1.0429407E3,1.8071064E7,9.5430945E4,4.6764086E-3,1.3148219E-2,1.0871896E10,-3.8367254E-3,4.39E3,6.465659E4,8.5716705E1,3.0315376E-3,1E0,1.0278284E-2,-2.4408007E-3,2.1813516E-2,1.3404826E-3,7.236662E0,-1.4639634E-3,1.9559993E-3,1.06904055E5,5.2610065E1,2.2331E4,1.19279E6,1.2093167E-3,-3.7420704E-3,1.4959046E-3,6.588828E6,4.52E2,3.104124E-3,1.1796855E7,2.6983346E-3,8.32E3,3.1682611E-3,-4.1960245E-3,-6.234612E-4,6.603817E2,-8.1491545E-3,4.979199E-3,1.771413E-2,-3.4728453E-3,1.624417E-3,-4.4827728E-4,5.2710976E-3,-5.688526E-3,1.6507361E9,-3.8308282E-3,4.151397E-4],"split_indices":[7,60,9,64,0,0,70,57,40,0,0,7,0,2,40,70,0,24,0,0,69,69,59,0,0,40,68,11,1,0,0,0,9,3,0,44,0,2,0,0,0,64,0,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.36E2,3.33E2,1.03E2,3.31E2,2E0,9E0,9.4E1,2.52E2,7.9E1,4E0,5E0,8.8E1,6E0,1.8E2,7.2E1,4.2E1,3.7E1,8.4E1,4E0,7.6E1,1.04E2,2E1,5.2E1,2.6E1,1.6E1,4.6E1,3.8E1,6.4E1,4E1,1.1E1,9E0,3E0,4.9E1,1.3E1,3.3E1,1.3E1,2.5E1,4.9E1,1.5E1,1E1,3E1,4.2E1,7E0,1.2E1,1E0,6E0,7E0,4.6E1,3E0,1.7E1,2.5E1,1.2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-5.8905076E-4,-2.2802964E-2,6.124458E-2,-3.0460376E-2,6.1969485E-2,3.2799777E-2,1.3222851E-1,-2.749425E-2,-1.2407606E-2,1.718641E-3,9.0936795E-3,4.1141883E-2,-1.0719525E-2,6.3347906E-2,2.532306E-1,-3.424972E-2,1.3202839E-2,8.490057E-3,3.4360126E-2,5.821894E-3,1.0425934E-3,4.3374733E-3,1.49462E-2,-2.9194035E-2,-8.080763E-2,-3.1669595E-5,6.0598273E-3,3.0099688E-2,9.481855E-3,-2.646238E-2,-6.4945067E-3,-1.8529937E-3,-6.6514835E-3,3.6367524E-2,-4.9800174E-3,-4.6864793E-2,-1.1972643E-2,1.6971994E-2,3.4260624E-3,-2.5660405E-3,3.4495862E-3,3.9442282E-2,-1.9097282E-2,2.9542698E-3,-2.2979111E-4,3.478441E-4,5.797586E-3,-2.5193265E-3,-3.3893393E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,31,-1,-1,33,-1,35,-1,-1,-1,37,-1,39,41,43,-1,-1,-1,45,47,-1,-1,-1,-1,-1,-1],"loss_changes":[6.2362236E-1,2.1869954E-1,2.3795778E-1,1.915124E-1,8.64052E-2,1.8830067E-1,2.673381E-1,8.40303E-2,0E0,0E0,0E0,6.837517E-2,0E0,4.950262E-2,6.559169E-2,5.9075624E-2,6.585952E-2,0E0,4.856789E-2,0E0,0E0,0E0,0E0,6.1792225E-2,5.2226588E-2,0E0,0E0,6.949617E-2,0E0,6.8024054E-2,0E0,0E0,0E0,4.752992E-2,0E0,5.339223E-2,5.093808E-2,4.5721635E-2,0E0,0E0,0E0,4.1471347E-2,4.631998E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,13,13,14,14,15,15,16,16,18,18,23,23,24,24,27,27,29,29,33,33,35,35,36,36,37,37,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,32,-1,-1,34,-1,36,-1,-1,-1,38,-1,40,42,44,-1,-1,-1,46,48,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4817759E9,2.9034685E3,1.486066E0,4.836269E3,6.0495757E3,3.768034E4,2.2876814E1,1.5135906E3,-1.2407606E-2,1.718641E-3,9.0936795E-3,1.5988636E1,-1.0719525E-2,1.0647493E2,1.8636872E0,1E0,1.7670108E5,8.490057E-3,3.305542E4,5.821894E-3,1.0425934E-3,4.3374733E-3,1.49462E-2,4.086885E7,1.3485402E8,-3.1669595E-5,6.0598273E-3,1.4609149E4,9.481855E-3,4.407E3,-6.4945067E-3,-1.8529937E-3,-6.6514835E-3,2.5065362E9,-4.9800174E-3,7.10711E8,5.7844446E2,5.473125E3,3.4260624E-3,-2.5660405E-3,3.4495862E-3,1.303394E3,7.1032936E7,2.9542698E-3,-2.2979111E-4,3.478441E-4,5.797586E-3,-2.5193265E-3,-3.3893393E-4],"split_indices":[7,64,51,4,64,4,66,64,0,0,0,68,0,70,51,19,40,0,4,0,0,0,0,57,7,0,0,4,0,2,0,0,0,7,0,7,59,44,0,0,0,60,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.52E2,3.33E2,1.19E2,3.06E2,2.7E1,8.6E1,3.3E1,3.03E2,3E0,2.3E1,4E0,8.4E1,2E0,2.2E1,1.1E1,2.6E2,4.3E1,3E0,8.1E1,9E0,1.3E1,3E0,8E0,2.36E2,2.4E1,3.9E1,4E0,8E1,1E0,2.31E2,5E0,1.4E1,1E1,7.7E1,3E0,9.5E1,1.36E2,4.9E1,2.8E1,9.2E1,3E0,1.6E1,1.2E2,1.6E1,3.3E1,1.2E1,4E0,3.3E1,8.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-7.499905E-4,-2.3788698E-2,4.984043E-2,-5.3966142E-2,-1.2720708E-2,3.0126855E-2,1.3774073E-1,-8.1066124E-2,-3.0484688E-2,-1.6807934E-2,4.9496304E-2,7.1392097E-3,7.973511E-2,3.9393133E-3,1.1680582E-2,-5.866339E-3,-4.43277E-2,-7.157993E-5,-3.0381735E-3,-2.2309765E-2,2.1548534E-3,4.408565E-3,-1.8909608E-3,-3.943479E-2,4.5082953E-2,9.267941E-3,3.0348385E-3,-3.791648E-3,1.2537962E-3,-2.7007198E-2,1.967339E-2,-6.917565E-3,-9.4978325E-4,5.9343453E-2,-1.1235789E-1,1.69204E-3,-3.0532258E-2,2.9136832E-3,-1.700024E-3,8.866031E-3,2.2666254E-3,4.3668235E-3,-1.0398159E-2,-9.822416E-3,-2.1126342E-3,-2.003821E-3,5.4463476E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,45,-1,-1,-1],"loss_changes":[5.1402235E-1,1.0051662E-1,2.3498073E-1,4.9072847E-2,5.753971E-2,1.2943372E-1,1.22951776E-1,4.4162884E-2,3.9536156E-2,7.019909E-2,5.067612E-2,1.41386E-1,6.1081216E-2,0E0,0E0,0E0,4.569292E-2,0E0,0E0,3.8593434E-2,0E0,0E0,0E0,7.0169345E-2,1.0545585E-1,0E0,0E0,0E0,0E0,3.836541E-2,4.3462254E-2,0E0,0E0,5.966948E-2,9.464298E-2,0E0,3.9704025E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.870043E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,19,19,23,23,24,24,29,29,30,30,33,33,34,34,36,36,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,46,-1,-1,-1],"split_conditions":[1.0863E4,4.069E3,9.908038E1,1.65602E5,2.4082131E5,1.646359E5,1E0,2.39E2,3.5389444E5,1.2678202E9,2.823537E1,4.98884E1,5.210789E-2,3.9393133E-3,1.1680582E-2,-5.866339E-3,3.5254697E2,-7.157993E-5,-3.0381735E-3,1.7985837E3,2.1548534E-3,4.408565E-3,-1.8909608E-3,1.2799757E9,3.768034E4,9.267941E-3,3.0348385E-3,-3.791648E-3,1.2537962E-3,5.0442883E1,6.7211235E1,-6.917565E-3,-9.4978325E-4,2.909019E3,1.92144E5,1.69204E-3,1.18E2,2.9136832E-3,-1.700024E-3,8.866031E-3,2.2666254E-3,4.3668235E-3,-1.0398159E-2,6.606756E1,-2.1126342E-3,-2.003821E-3,5.4463476E-4],"split_indices":[2,2,68,9,40,40,13,3,60,7,68,68,50,0,0,0,45,0,0,64,0,0,0,7,4,0,0,0,0,60,68,0,0,44,2,0,3,0,0,0,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.02E2,1.37E2,8E1,2.22E2,1.13E2,2.4E1,3.6E1,4.4E1,2.09E2,1.3E1,7.8E1,3.5E1,1.6E1,8E0,1.7E1,1.9E1,2.3E1,2.1E1,1.92E2,1.7E1,9E0,4E0,3.5E1,4.3E1,4E0,3.1E1,1.3E1,6E0,1.73E2,1.9E1,5E0,3E1,4E1,3E0,9E0,1.64E2,1.1E1,8E0,3E0,3.7E1,1E0,2E0,6E1,1.04E2,2.4E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.3846618E-3,-2.8292919E-2,4.0907655E-2,-3.1308882E-2,1.2773425E-2,2.8989794E-2,1.3182159E-1,-2.7768802E-2,-1.077002E-2,3.3040933E-2,-9.996281E-2,4.056247E-3,9.231509E-3,-5.6596645E-3,-2.2985715E-2,2.6974764E-2,1.625901E-2,4.2577037E-3,-8.373539E-3,-2.4491964E-2,9.023375E-3,-1.5710019E-2,3.690004E-2,1.6839002E-3,-2.905333E-2,1.1015086E-4,-6.4081387E-3,3.694021E-3,2.1758558E-2,-7.132073E-3,-2.4973746E-2,7.930922E-3,2.9136955E-3,-4.1895144E-2,3.0376622E-4,-2.8510723E-4,3.4299132E-3,9.094786E-4,-2.339734E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,-1,-1,-1,19,21,-1,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,31,-1,33,35,-1,37,-1,-1,-1,-1,-1],"loss_changes":[5.121251E-1,2.4345255E-1,1.8579093E-1,1.7202917E-1,0E0,8.479802E-2,3.8600415E-2,1.0737808E-1,0E0,2.6073694E-1,7.6846756E-2,0E0,0E0,0E0,8.29069E-2,6.412846E-2,0E0,0E0,0E0,6.8657964E-2,0E0,5.873466E-2,6.6769406E-2,0E0,1.06325954E-1,0E0,0E0,0E0,4.4004593E-2,0E0,1.2110652E-1,5.4191716E-2,0E0,4.492253E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,14,14,15,15,19,19,21,21,22,22,24,24,28,28,30,30,31,31,33,33],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,-1,-1,-1,20,22,-1,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,32,-1,34,36,-1,38,-1,-1,-1,-1,-1],"split_conditions":[1.0136646E5,6.0495757E3,7.765071E0,7.339209E6,1.2773425E-2,3.7826266E4,2.833843E8,1.44E2,-1.077002E-2,1.4609149E4,1.92144E5,4.056247E-3,9.231509E-3,-5.6596645E-3,3.305542E4,9.9800183E2,1.625901E-2,4.2577037E-3,-8.373539E-3,1.27E2,9.023375E-3,1E0,6.527467E6,1.6839002E-3,2.2744698E2,1.1015086E-4,-6.4081387E-3,3.694021E-3,1.8822492E0,-7.132073E-3,1.173433E5,2.6691768E7,2.9136955E-3,6.281509E1,3.0376622E-4,-2.8510723E-4,3.4299132E-3,9.094786E-4,-2.339734E-3],"split_indices":[45,64,47,59,0,64,44,0,0,4,2,0,0,0,4,64,0,0,0,11,0,19,62,0,64,0,0,0,53,0,40,60,0,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.48E2,2.74E2,1.74E2,2.72E2,2E0,1.55E2,1.9E1,2.68E2,4E0,1.51E2,4E0,1.1E1,8E0,1.3E1,2.55E2,1.49E2,2E0,1E0,3E0,2.54E2,1E0,2.8E1,1.21E2,1.8E1,2.36E2,2.5E1,3E0,3.4E1,8.7E1,7E0,2.29E2,6.4E1,2.3E1,1.48E2,8.1E1,5.3E1,1.1E1,1.1E1,1.37E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.9832794E-4,-1.3666869E-2,7.710252E-2,-3.2414395E-2,1.7542977E-2,1.9919232E-1,5.0046735E-2,-2.859398E-2,-1.0078713E-2,1.4275513E-3,7.239154E-2,3.1677037E-3,1.4350286E-2,1.02148E-2,3.1275023E-2,-4.2730268E-2,-8.055775E-3,-2.6946248E-2,1.6776865E-2,9.223771E-3,2.7198612E-3,2.701284E-3,-5.8533017E-2,-5.1055796E-2,4.0533545E-4,4.526924E-2,-8.526822E-4,-9.3447656E-4,-8.504971E-3,1.7791375E-3,-8.942937E-4,-1.12102956E-1,3.1604932E-3,-3.392925E-3,-3.0157525E-2,1.837462E-4,5.9671695E-3,-9.23903E-3,-1.5727225E-3,1.9886768E-3,-1.9855648E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,31,33,-1,35,-1,-1,-1,-1,-1,37,-1,-1,39,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7406402E-1,2.1939884E-1,2.1465111E-1,1.4554481E-1,1.24527685E-1,1.2513253E-1,1.5875697E-1,6.6432595E-2,0E0,4.8359092E-2,5.4452732E-2,0E0,0E0,0E0,1.0953051E-1,5.7979763E-2,4.639581E-2,4.289117E-2,4.7625307E-2,0E0,0E0,0E0,7.8846574E-2,3.9116204E-2,0E0,4.272733E-2,0E0,0E0,0E0,0E0,0E0,4.1121297E-2,0E0,0E0,3.7740257E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,15,15,16,16,17,17,18,18,22,22,23,23,25,25,31,31,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,32,34,-1,36,-1,-1,-1,-1,-1,38,-1,-1,40,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,7.222818E4,1.064816E6,5.0350455E6,1.646359E5,1.082556E7,1.3812E4,5.855E3,-1.0078713E-2,2.0231652E3,1.4E1,3.1677037E-3,1.4350286E-2,1.02148E-2,4.42384E5,2.1592189E5,4.9832974E2,2.0192318E3,1.1106066E7,9.223771E-3,2.7198612E-3,2.701284E-3,2.785032E9,1.6776951E3,4.0533545E-4,6.651711E1,-8.526822E-4,-9.3447656E-4,-8.504971E-3,1.7791375E-3,-8.942937E-4,9.820362E4,3.1604932E-3,-3.392925E-3,4.2969458E2,1.837462E-4,5.9671695E-3,-9.23903E-3,-1.5727225E-3,1.9886768E-3,-1.9855648E-3],"split_indices":[64,45,41,59,40,1,9,2,0,67,3,0,0,0,2,40,64,67,60,0,0,0,7,4,0,66,0,0,0,0,0,40,0,0,64,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.73E2,6.7E1,2.33E2,1.4E2,1.1E1,5.6E1,2.29E2,4E0,1.09E2,3.1E1,5E0,6E0,5E0,5.1E1,1.35E2,9.4E1,3.8E1,7.1E1,3E0,2.8E1,4.1E1,1E1,1.16E2,1.9E1,1.3E1,8.1E1,3.7E1,1E0,4.6E1,2.5E1,7E0,3E0,6.3E1,5.3E1,9E0,4E0,3E0,4E0,6E0,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[6.5604253E-3,-1.9894673E-2,4.1327238E-2,-2.2994876E-2,1.1938625E-2,1.635464E-2,8.0101505E-2,-1.9575879E-2,-1.11567015E-2,8.897198E-3,3.4742989E-3,9.3693696E-2,1.5992568E-4,-2.8514485E-3,-1.293088E-2,-2.5995204E-4,2.0667922E-3,7.978188E-2,1.5805694E-1,2.215752E-4,-3.689661E-2,4.7045415E-3,4.3210274E-4,1.07525345E-2,9.3073264E-4,-4.0708864E-3,3.9321505E-3,-6.328488E-3,-1.1946433E-3,-3.0273305E-2,1.0499983E-2,1.0152361E-3,-2.650745E-3,-3.8711175E-2,2.4723245E-2,5.5133854E-4,-4.7580833E-3,2.063624E-3,-5.976653E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,-1,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,31,33,-1,-1,35,37,-1,-1,-1,-1],"loss_changes":[3.9815828E-1,2.0285283E-1,1.7976102E-1,1.6268216E-1,0E0,4.4975728E-2,7.600415E-2,5.928497E-2,0E0,4.6892606E-2,0E0,4.290527E-2,0E0,0E0,6.494105E-2,0E0,0E0,5.330333E-2,7.5604886E-2,4.5449167E-2,8.207991E-2,0E0,0E0,0E0,0E0,4.9187247E-2,0E0,0E0,0E0,5.4057036E-2,5.9052348E-2,0E0,0E0,5.4445926E-2,4.000004E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,14,14,17,17,18,18,19,19,20,20,25,25,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,-1,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,32,34,-1,-1,36,38,-1,-1,-1,-1],"split_conditions":[8.393928E4,8.861901E3,1.3590209E7,7.339209E6,1.1938625E-2,2.4251762E5,1.773296E6,4.26944E5,-1.11567015E-2,2.34E2,3.4742989E-3,3.63796E5,1.5992568E-4,-2.8514485E-3,1.8071064E7,-2.5995204E-4,2.0667922E-3,4.307733E1,7.9E1,6.46E2,1.9281628E7,4.7045415E-3,4.3210274E-4,1.07525345E-2,9.3073264E-4,7.959718E4,3.9321505E-3,-6.328488E-3,-1.1946433E-3,1.1347253E1,3.9560037E2,1.0152361E-3,-2.650745E-3,1.0547217E-5,6.427212E4,5.5133854E-4,-4.7580833E-3,2.063624E-3,-5.976653E-4],"split_indices":[45,64,1,59,0,40,11,1,0,8,0,2,0,0,57,0,0,69,8,8,57,0,0,0,0,40,0,0,0,70,64,0,0,51,59,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.45E2,1.86E2,2.43E2,2E0,1.14E2,7.2E1,2.4E2,3E0,1.01E2,1.3E1,6.1E1,1.1E1,3.5E1,2.05E2,7.1E1,3E1,5.2E1,9E0,1.33E2,7.2E1,4.3E1,9E0,6E0,3E0,1.27E2,6E0,8E0,6.4E1,4.5E1,8.2E1,1.4E1,3.1E1,1.8E1,6.4E1,1E1,8E0,4.4E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.5774983E-3,-1.5281041E-2,6.605846E-2,-1.2914552E-2,-2.3331916E-1,4.5070697E-2,8.722418E-3,-1.5305939E-2,8.3514685E-3,-1.5380709E-2,-2.4969102E-4,8.035125E-2,-1.1198101E-2,-4.5491386E-2,-3.6910432E-3,2.945361E-3,8.181789E-3,-2.710587E-3,1.4689042E-3,-4.8944904E-3,-3.74993E-2,1.8921254E-2,-1.5357561E-2,-2.2661048E-3,1.5280386E-3,-4.9741205E-2,1.3020115E-3,-2.2505603E-3,1.13149064E-4,1.7212956E-3,-6.195863E-3,6.982532E-3,-2.794035E-3,9.487465E-4,-2.7128158E-2,1.973539E-3,-3.0366655E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,17,19,21,-1,-1,-1,-1,-1,23,25,27,-1,-1,29,-1,-1,31,-1,-1,33,-1,-1,35,-1,-1],"loss_changes":[3.907576E-1,1.9202907E-1,1.5149659E-1,1.639978E-1,6.644985E-2,1.2001145E-1,0E0,1.29905E-1,0E0,0E0,0E0,5.6406975E-2,4.3521855E-2,3.949958E-2,7.125461E-2,0E0,0E0,0E0,0E0,0E0,4.99814E-2,4.63448E-2,8.172422E-2,0E0,0E0,6.0514644E-2,0E0,0E0,4.5799945E-2,0E0,0E0,4.390914E-2,0E0,0E0,6.513734E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,12,12,13,13,14,14,20,20,21,21,22,22,25,25,28,28,31,31,34,34],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,18,20,22,-1,-1,-1,-1,-1,24,26,28,-1,-1,30,-1,-1,32,-1,-1,34,-1,-1,36,-1,-1],"split_conditions":[3.5257432E6,1.69595E5,1.6828056E1,6.6512886E3,8.75E3,2.3619232E7,8.722418E-3,4.407E3,8.3514685E-3,-1.5380709E-2,-2.4969102E-4,2.36316E5,5.0314346E1,6.506511E0,7.37365E6,2.945361E-3,8.181789E-3,-2.710587E-3,1.4689042E-3,-4.8944904E-3,2.681357E6,1.366276E6,1.1767653E9,-2.2661048E-3,1.5280386E-3,1.2E3,1.3020115E-3,-2.2505603E-3,4.2663252E7,1.7212956E-3,-6.195863E-3,1.73E2,-2.794035E-3,9.487465E-4,1.0717E4,1.973539E-3,-3.0366655E-3],"split_indices":[56,2,47,64,0,55,0,2,0,0,0,2,68,68,57,0,0,0,0,0,1,57,5,0,0,11,0,0,57,0,0,8,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,3.77E2,6.9E1,3.74E2,3E0,5.9E1,1E1,3.7E2,4E0,2E0,1E0,3.6E1,2.3E1,1.02E2,2.68E2,3E1,6E0,1.1E1,1.2E1,1.2E1,9E1,9.1E1,1.77E2,8.1E1,9E0,8E0,8.3E1,6E1,1.17E2,4E0,4E0,1.05E2,1.2E1,7.8E1,2.7E1,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.6588347E-3,-9.5964605E-3,1.378394E-1,-2.8568476E-2,2.2045098E-2,1.2562832E-2,4.6895545E-2,-2.5289115E-2,-9.303901E-3,8.972755E-3,3.5376062E-3,8.821252E-3,5.2895217E-4,-3.574927E-2,6.160709E-3,-1.3929022E-3,1.0439812E-3,-5.4965876E-2,-2.149367E-2,2.4303705E-3,-9.7329856E-4,-2.3981698E-2,-8.7765075E-2,-4.824239E-3,-1.2086107E-2,-2.8381802E-3,1.1453906E-3,-5.9536034E-3,-2.5053585E-3,-2.3366518E-2,4.7186404E-2,-8.572188E-3,-1.946792E-2,-7.9979276E-4,4.332962E-3,8.783337E-5,-2.2654224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,-1,17,19,-1,-1,21,23,-1,-1,25,27,-1,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,-1,-1],"loss_changes":[4.730261E-1,2.4315268E-1,2.2569886E-1,1.260482E-1,9.621281E-2,0E0,6.4022355E-2,8.213383E-2,0E0,5.370925E-2,0E0,0E0,0E0,5.004616E-2,6.953896E-2,0E0,0E0,7.817949E-2,7.5778365E-2,0E0,0E0,6.592953E-2,3.8881212E-2,0E0,6.6627175E-2,0E0,0E0,0E0,0E0,4.4602677E-2,4.1338008E-2,0E0,4.5092463E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,13,13,14,14,17,17,18,18,21,21,22,22,24,24,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,-1,18,20,-1,-1,22,24,-1,-1,26,28,-1,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,-1,-1],"split_conditions":[8.10288E3,9.7467445E4,3.0418E4,7.339209E6,1.7670108E5,1.2562832E-2,9.812318E3,7.198E3,-9.303901E-3,3.6308475E1,3.5376062E-3,8.821252E-3,5.2895217E-4,7.683712E4,2.1933604E3,-1.3929022E-3,1.0439812E-3,3.5546432E6,9.284744E2,2.4303705E-3,-9.7329856E-4,1.3202006E3,2.55E2,-4.824239E-3,7.2008003E9,-2.8381802E-3,1.1453906E-3,-5.9536034E-3,-2.5053585E-3,2.90694E5,5.0563E4,-8.572188E-3,1.7197124E7,-7.9979276E-4,4.332962E-3,8.783337E-5,-2.2654224E-3],"split_indices":[64,45,10,59,40,0,4,2,0,68,0,0,0,40,44,0,0,57,4,0,0,4,3,0,5,0,0,0,0,1,12,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,4.03E2,2.2E1,2.52E2,1.51E2,9E0,1.3E1,2.48E2,4E0,1.2E2,3.1E1,2E0,1.1E1,1.86E2,6.2E1,2.9E1,9.1E1,7.8E1,1.08E2,2.3E1,3.9E1,4.1E1,3.7E1,1.1E1,9.7E1,2.4E1,1.7E1,1.9E1,1.8E1,8.2E1,1.5E1,1E0,8.1E1,6E0,9E0,4.5E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.5892266E-4,-1.09341815E-2,9.025434E-2,-9.21702E-3,-1.1463298E-2,1.9932939E-1,5.441878E-2,-1.7021887E-2,2.9119173E-2,1.6513603E-2,5.44054E-3,-1.0448571E-2,6.792054E-2,-2.365136E-2,2.7895581E-2,4.4368234E-4,5.5452664E-3,-8.3438033E-4,8.6003035E-2,-4.3994483E-2,-1.34084765E-2,4.4338023E-3,1.0974945E-2,1.00451045E-1,-1.4697844E-3,-6.001143E-2,-1.2568874E-2,-8.357987E-3,-1.1700653E-2,-1.7247086E-3,1.8668042E-3,3.1580643E-3,8.186527E-3,-4.6693064E-2,-6.2269014E-3,-1.5862783E-3,3.947845E-3,1.2214053E-3,-2.878561E-2,-2.699951E-3,4.6733543E-3,-4.8144758E-4,2.0868229E-3,-1.0428665E-3,-5.4729963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,17,19,21,-1,-1,-1,23,25,27,-1,29,31,-1,33,35,-1,37,-1,-1,-1,-1,39,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2130667E-1,1.4417459E-1,1.6895592E-1,1.16588704E-1,0E0,9.4490886E-2,1.4227319E-1,9.6842445E-2,1.0834597E-1,0E0,0E0,0E0,5.498837E-2,5.817108E-2,4.2251132E-2,0E0,0E0,0E0,5.131127E-2,4.65599E-2,4.7696646E-2,0E0,4.208454E-2,5.1026434E-2,0E0,4.7513485E-2,6.0437035E-2,0E0,4.1540448E-2,0E0,0E0,0E0,0E0,5.953349E-2,0E0,0E0,0E0,4.792294E-2,4.9491443E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,12,12,13,13,14,14,18,18,19,19,20,20,22,22,23,23,25,25,26,26,28,28,33,33,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,18,20,22,-1,-1,-1,24,26,28,-1,30,32,-1,34,36,-1,38,-1,-1,-1,-1,40,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,3.372042E7,1.417988E6,1.646359E5,-1.1463298E-2,4.542969E1,1.461379E6,3.73E2,1.1895911E-1,1.6513603E-2,5.44054E-3,-1.0448571E-2,3.6308475E1,1.6858337E3,2.96147E5,4.4368234E-4,5.5452664E-3,-8.3438033E-4,2.3423826E10,2.033552E6,2.90694E5,4.4338023E-3,1.570662E5,2.20229E5,-1.4697844E-3,6.8E1,1.875E3,-8.357987E-3,1.8071064E7,-1.7247086E-3,1.8668042E-3,3.1580643E-3,8.186527E-3,5.4630665E1,-6.2269014E-3,-1.5862783E-3,3.947845E-3,1.963372E6,2.11E2,-2.699951E-3,4.6733543E-3,-4.8144758E-4,2.0868229E-3,-1.0428665E-3,-5.4729963E-3],"split_indices":[64,60,41,40,0,66,41,8,69,0,0,0,68,4,41,0,0,0,12,9,1,0,59,2,0,8,0,0,57,0,0,0,0,68,0,0,0,41,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,3.89E2,4.5E1,3.87E2,2E0,1E1,3.5E1,3.22E2,6.5E1,3E0,7E0,1E0,3.4E1,2.81E2,4.1E1,5.3E1,1.2E1,6E0,2.8E1,9.3E1,1.88E2,8E0,3.3E1,2.5E1,3E0,6.1E1,3.2E1,1E0,1.87E2,1.2E1,2.1E1,1.7E1,8E0,5.2E1,9E0,2.7E1,5E0,1.07E2,8E1,5E1,2E0,8.5E1,2.2E1,7.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[4.1947532E-3,-6.4652883E-3,7.40463E-2,-4.352768E-3,-8.345571E-3,1.5560552E-1,4.270553E-2,-2.5049772E-2,1.4910038E-2,2.3859008E-3,1.0772358E-2,3.0541616E-3,-7.828351E-3,-3.276617E-2,8.7902957E-4,7.6503777E-3,7.0129596E-2,-1.8673334E-3,-2.063619E-3,-3.9048255E-3,1.008547E-2,6.597789E-3,1.0451834E-3,-1.6532236E-3,2.0967615E-3,8.6663617E-4,-1.0038671E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,23,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3183482E-1,1.3051042E-1,1.4416638E-1,1.5306884E-1,0E0,9.3164116E-2,1.7102532E-1,6.1444618E-2,7.923764E-2,0E0,0E0,0E0,0E0,4.1283324E-2,0E0,3.7762E-2,6.729565E-2,4.9262088E-2,0E0,0E0,3.8221046E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,24,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,4.2064876E7,1.417988E6,6.24E3,-8.345571E-3,1.2273383E7,2.3767492E4,2.1592189E5,7.049895E9,2.3859008E-3,1.0772358E-2,3.0541616E-3,-7.828351E-3,4.3712845E0,8.7902957E-4,6.7440504E7,1.3590306E10,5.24228E5,-2.063619E-3,-3.9048255E-3,2.4722598E7,6.597789E-3,1.0451834E-3,-1.6532236E-3,2.0967615E-3,8.6663617E-4,-1.0038671E-3],"split_indices":[64,62,41,2,0,57,64,40,12,0,0,0,0,70,0,7,12,9,0,0,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.44E2,3.86E2,5.8E1,3.82E2,4E0,1.5E1,4.3E1,1.84E2,1.98E2,6E0,9E0,4E1,3E0,1.56E2,2.8E1,1.76E2,2.2E1,3.4E1,1.22E2,4E0,1.72E2,9E0,1.3E1,2E1,1.4E1,1.39E2,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[5.0503097E-4,-1.5768487E-2,4.367275E-2,-4.909478E-2,-8.844461E-3,1.4040245E-1,2.8660033E-2,-2.8972556E-3,1.6527841E-3,-7.8774395E-4,2.387284E-2,2.1866704E-3,2.1071161E-1,3.8257696E-2,-7.903229E-2,-3.8938315E-3,1.6284013E-3,5.468569E-3,1.5490864E-2,1.6072996E-2,6.870782E-2,-6.49673E-3,3.3098636E-3,1.0861254E-3,-4.744543E-3,1.4262411E-1,2.0121906E-3,-1.181502E-3,8.076875E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[3.1048703E-1,7.3637225E-2,1.7168932E-1,4.203847E-2,6.080825E-2,9.950438E-2,1.125846E-1,0E0,0E0,0E0,4.4210855E-2,0E0,5.656144E-2,6.509729E-2,7.523803E-2,0E0,0E0,0E0,0E0,3.8504772E-2,8.046125E-2,0E0,0E0,0E0,0E0,3.8553223E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,14,14,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[9.407814E6,3.807E3,1.417988E6,4.049296E-2,1.5271514E5,9.256843E3,1.5291998E4,-2.8972556E-3,1.6527841E-3,-7.8774395E-4,4.4593253E2,2.1866704E-3,1.281E3,7.2038723E9,1.2408761E0,-3.8938315E-3,1.6284013E-3,5.468569E-3,1.5490864E-2,8.6206274E2,2.1207E4,-6.49673E-3,3.3098636E-3,1.0861254E-3,-4.744543E-3,1.2511627E7,2.0121906E-3,-1.181502E-3,8.076875E-3],"split_indices":[1,2,41,69,40,4,4,0,0,0,64,0,0,12,46,0,0,0,0,70,9,0,0,0,0,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.2E2,1.2E2,5.4E1,2.66E2,1.5E1,1.05E2,4.9E1,5E0,2.2E2,4.6E1,7E0,8E0,9.7E1,8E0,3E0,4.3E1,5E0,3E0,5.7E1,4E1,6E0,2E0,5.5E1,2E0,1E1,3E1,1E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.267155E-3,-1.8551797E-2,3.064989E-2,-1.6366217E-2,-8.723522E-3,2.5247024E-2,7.6668016E-3,-4.7053327E-3,-1.32957725E-2,3.0088557E-2,-8.950915E-2,-2.1004928E-2,1.5835155E-2,1.1270756E-2,2.7374548E-3,-6.3549865E-5,-8.8619E-3,-3.553209E-3,-1.2933667E-2,1.8717118E-3,-2.0568362E-2,1.6429644E-3,-7.720068E-4,-2.0668117E-2,1.4318541E-3,-2.4964176E-3,2.0375475E-3,-2.9045392E-2,1.2611569E-3,-3.100027E-2,6.0641654E-3,4.9184648E-5,-1.9896654E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,-1,-1,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,25,-1,-1,27,-1,-1,-1,29,-1,31,-1,-1,-1],"loss_changes":[2.3803532E-1,9.8406844E-2,9.362508E-2,6.8965964E-2,0E0,8.15878E-2,0E0,0E0,6.411187E-2,6.302603E-2,4.6265297E-2,9.005148E-2,4.7986403E-2,4.61089E-2,0E0,0E0,0E0,0E0,6.331056E-2,0E0,4.3491147E-2,0E0,0E0,6.418285E-2,0E0,0E0,0E0,4.4919915E-2,0E0,3.921625E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,9,9,10,10,11,11,12,12,13,13,18,18,20,20,23,23,27,27,29,29],"right_children":[2,4,6,8,-1,10,-1,-1,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,26,-1,-1,28,-1,-1,-1,30,-1,32,-1,-1,-1],"split_conditions":[1.279709E5,7.339209E6,1.0871896E10,2.398E3,-8.723522E-3,5.37E2,7.6668016E-3,-4.7053327E-3,2.332E3,1.5333999E7,1.4912397E7,4.516183E1,2.2980049E10,2.0421524E6,2.7374548E-3,-6.3549865E-5,-8.8619E-3,-3.553209E-3,1.643477E9,1.8717118E-3,3.7137297E3,1.6429644E-3,-7.720068E-4,3.94E2,1.4318541E-3,-2.4964176E-3,2.0375475E-3,1.9381E4,1.2611569E-3,1.1347253E1,6.0641654E-3,4.9184648E-5,-1.9896654E-3],"split_indices":[45,59,7,2,0,3,0,0,0,1,57,66,5,59,0,0,0,0,5,0,44,0,0,8,0,0,0,2,0,70,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,2.96E2,1.46E2,2.93E2,3E0,1.41E2,5E0,1E1,2.83E2,1.36E2,5E0,2.24E2,5.9E1,7.8E1,5.8E1,3E0,2E0,3E1,1.94E2,3.7E1,2.2E1,4.3E1,3.5E1,1.64E2,3E1,1.5E1,7E0,1.39E2,2.5E1,1.38E2,1E0,3E1,1.08E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.3851766E-3,-8.555901E-3,8.765691E-2,-1.754065E-2,2.780584E-2,7.671462E-3,2.473388E-3,-1.5805518E-2,-1.032541E-2,3.6449753E-2,-2.941969E-3,-2.2944536E-3,-8.304134E-3,9.346795E-3,1.5996163E-3,-1.5579082E-2,2.1913178E-2,8.616356E-4,-2.9919378E-2,3.382069E-3,1.033959E-4,7.201604E-4,-2.7057605E-2,-4.991514E-2,-6.471156E-3,-4.8421015E-4,-6.289924E-3,-3.8345456E-2,-5.7508796E-3,1.3765211E-4,-4.1116285E-3,-4.3159006E-3,-9.0862904E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,-1,15,-1,-1,17,19,21,23,-1,-1,-1,25,27,29,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.9126576E-1,1.3835563E-1,7.784611E-2,1.0794522E-1,6.5078266E-2,0E0,0E0,7.572448E-2,0E0,4.5517102E-2,0E0,0E0,5.991514E-2,0E0,0E0,5.165826E-2,4.7988504E-2,3.9350975E-2,5.4534204E-2,0E0,0E0,0E0,5.7126373E-2,4.3080136E-2,3.8688175E-2,0E0,0E0,5.1678687E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,12,12,15,15,16,16,17,17,18,18,22,22,23,23,24,24,27,27],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,-1,16,-1,-1,18,20,22,24,-1,-1,-1,26,28,30,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[6.204522E3,1.5522031E9,1.681178E6,3.764913E7,2.785563E7,7.671462E-3,2.473388E-3,1.0780007E3,-1.032541E-2,1.2662238E1,-2.941969E-3,-2.2944536E-3,1.5271514E5,9.346795E-3,1.5996163E-3,2.7977833E1,2.5159248E1,2.9991518E7,9.606877E-2,3.382069E-3,1.033959E-4,7.201604E-4,6.54925E6,1E0,1.3958886E-1,-4.8421015E-4,-6.289924E-3,7.1032936E7,-5.7508796E-3,1.3765211E-4,-4.1116285E-3,-4.3159006E-3,-9.0862904E-4],"split_indices":[64,7,41,60,55,0,0,67,0,68,0,0,40,0,0,70,68,57,50,0,0,0,9,19,69,0,0,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.54E2,4.21E2,3.3E1,3.38E2,8.3E1,1.1E1,2.2E1,3.36E2,2E0,7.6E1,7E0,6.6E1,2.7E2,1E0,7.5E1,2.18E2,5.2E1,1.02E2,1.16E2,1.5E1,3.7E1,6.9E1,3.3E1,6.2E1,5.4E1,2.9E1,4E0,5.4E1,8E0,4.9E1,5E0,1.5E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.3344603E-3,-2.2538431E-2,2.0072022E-2,-1.1229228E-2,-2.1540523E-3,1.0385378E-1,1.2850213E-2,-2.3204312E-3,3.520242E-3,1.2997606E-2,1.7881122E-3,1.7302338E-2,-6.6960733E-3,5.0529605E-4,-3.2470908E-3,6.968949E-3,2.7376635E-3,1.0510577E-2,-8.735646E-3,-1.4848099E-3,8.410847E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,-1,15,-1,-1,-1,17,-1,19,-1,-1,-1],"loss_changes":[1.9699216E-1,5.021271E-2,1.2935753E-1,7.389055E-2,0E0,1.7148578E-1,1.3306448E-1,0E0,4.624884E-2,0E0,0E0,7.5135075E-2,0E0,0E0,0E0,1.00974046E-1,0E0,3.913282E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,11,11,15,15,17,17],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,-1,16,-1,-1,-1,18,-1,20,-1,-1,-1],"split_conditions":[1.0975164E3,1.3058374E7,1.3812E4,5.42218E5,-2.1540523E-3,3.7137297E3,4.242889E3,-2.3204312E-3,1E0,1.2997606E-2,1.7881122E-3,1.8770729E0,-6.6960733E-3,5.0529605E-4,-3.2470908E-3,2.5212732E4,2.7376635E-3,4.59E2,-8.735646E-3,-1.4848099E-3,8.410847E-4],"split_indices":[64,57,9,1,0,44,70,0,19,0,0,53,0,0,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,2.17E2,2.15E2,1.41E2,7.6E1,1.6E1,1.99E2,4.1E1,1E2,4E0,1.2E1,1.94E2,5E0,9.2E1,8E0,1.53E2,4.1E1,1.51E2,2E0,2E1,1.31E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.2136965E-3,-1.4582835E-2,3.895704E-2,-1.2849405E-2,-7.7585457E-3,4.847287E-2,-2.1642826E-3,-8.7156694E-4,1.3359499E-3,2.6864478E-2,8.2278736E-2,3.8729828E-2,-5.2115633E-3,2.9061933E-3,1.4551783E-1,1.1539502E-3,1.1221862E-2,9.146057E-3,4.7643282E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,15,-1,-1,17,-1,-1,-1,-1],"loss_changes":[2.225996E-1,7.993151E-2,8.116244E-2,6.0968477E-2,0E0,6.426427E-2,0E0,0E0,0E0,9.273586E-2,4.4939965E-2,1.488387E-1,0E0,0E0,4.4138953E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,11,11,14,14],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,16,-1,-1,18,-1,-1,-1,-1],"split_conditions":[1.6053742E9,3.764913E7,2.00036E5,2.2860639E9,-7.7585457E-3,7.2038723E9,-2.1642826E-3,-8.7156694E-4,1.3359499E-3,5.358657E7,3.63796E5,4.5446856E7,-5.2115633E-3,2.9061933E-3,2.2576077E10,1.1539502E-3,1.1221862E-2,9.146057E-3,4.7643282E-4],"split_indices":[7,60,10,12,0,12,0,0,0,1,2,1,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,3.35E2,1E2,3.32E2,3E0,9E1,1E1,2.98E2,3.4E1,5.6E1,3.4E1,5.2E1,4E0,2.6E1,8E0,4.9E1,3E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.0018868E-4,-8.872676E-3,6.43786E-2,-2.2091264E-2,1.2409362E-2,1.3756613E-1,3.2702204E-2,-1.0198755E-3,-1.0440747E-2,3.2443993E-2,-8.910657E-3,9.02812E-4,1.7589448E-1,2.2559494E-3,-3.0438865E-3,-1.6471515E-4,2.4489632E-3,7.7277753E-3,-7.8923354E-4,1.2278668E-2,4.5448863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5386387E-1,1.077378E-1,1.20690405E-1,6.998452E-2,6.307849E-2,7.010704E-2,4.815881E-2,0E0,0E0,4.5138776E-2,8.347163E-2,0E0,4.8809767E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,8.393928E4,3.0418E4,1.1770536E8,1.0270569E7,1.3590209E7,3.768034E4,-1.0198755E-3,-1.0440747E-2,1.0220021E3,1.2662238E1,9.02812E-4,3.5432243E8,2.2559494E-3,-3.0438865E-3,-1.6471515E-4,2.4489632E-3,7.7277753E-3,-7.8923354E-4,1.2278668E-2,4.5448863E-3],"split_indices":[64,45,10,1,57,1,4,0,0,64,68,0,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,3.81E2,5.3E1,2.35E2,1.46E2,1.5E1,3.8E1,2.34E2,1E0,7.5E1,7.1E1,4E0,1.1E1,3.4E1,4E0,2.4E1,5.1E1,2E0,6.9E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.1175605E-3,-2.2947702E-3,1.6213124E-1,-1.8514453E-2,1.3484989E-2,1.968001E-1,-2.879115E-3,-2.6227355E-2,3.6798354E-4,4.3082857E-3,9.906706E-3,1.1368203E-2,-4.4751805E-4,-1.8123848E-3,6.76526E-4,-2.0128433E-2,9.302147E-4,1.3849856E-3,-1.7244085E-3,-3.286672E-4,-4.82912E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,-1,-1,-1,17,19,-1,-1,-1,-1,-1],"loss_changes":[2.3827605E-1,1.09025314E-1,7.99012E-2,4.210738E-2,5.5353425E-2,5.252549E-2,0E0,4.3773085E-2,0E0,0E0,5.450529E-2,0E0,0E0,0E0,4.375908E-2,4.8090644E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,14,14,15,15],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,-1,-1,-1,18,20,-1,-1,-1,-1,-1],"split_conditions":[3.305542E4,6.97E3,2.6279484E5,9.704588E6,8.547769E7,8.494E3,-2.879115E-3,2.103E3,3.6798354E-4,4.3082857E-3,1.1016051E3,1.1368203E-2,-4.4751805E-4,-1.8123848E-3,1.4399977E10,1E0,9.302147E-4,1.3849856E-3,-1.7244085E-3,-3.286672E-4,-4.82912E-3],"split_indices":[4,2,45,9,7,0,0,0,0,0,64,0,0,0,5,19,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,4.24E2,8E0,2.09E2,2.15E2,7E0,1E0,1.61E2,4.8E1,9E0,2.06E2,6E0,1E0,1.17E2,4.4E1,4.6E1,1.6E2,2.5E1,1.9E1,4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.0222289E-3,-6.2944572E-3,4.8310764E-2,-4.540565E-3,-8.387627E-3,1.052603E-2,3.562798E-2,7.7409536E-4,-1.5621538E-2,1.0020816E-1,1.5275183E-2,-1.0354397E-3,1.1849942E-3,1.2609476E-3,9.169429E-3,-9.848899E-3,1.250565E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,-1,-1,9,-1,11,13,15,-1,-1,-1,-1,-1,-1],"loss_changes":[1.497741E-1,1.0548328E-1,1.1189191E-1,8.2807586E-2,0E0,0E0,7.07606E-2,0E0,4.8607793E-2,7.653242E-2,9.394058E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,9,9,10,10],"right_children":[2,4,6,8,-1,-1,10,-1,12,14,16,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,7.78623E6,8.169258E3,1.04E2,-8.387627E-3,1.052603E-2,1.417988E6,7.7409536E-4,1.484798E9,1.752E5,1.461379E6,-1.0354397E-3,1.1849942E-3,1.2609476E-3,9.169429E-3,-9.848899E-3,1.250565E-3],"split_indices":[64,59,45,3,0,0,41,0,7,2,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,3.74E2,5.7E1,3.71E2,3E0,3E0,5.4E1,1.32E2,2.39E2,1.2E1,4.2E1,2.12E2,2.7E1,7E0,5E0,1E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-2.3966386E-4,-9.587779E-3,3.3721242E-2,-1.1249492E-2,5.4631056E-3,1.6092595E-2,8.192009E-2,-9.733436E-3,-7.0322864E-3,2.5347946E-2,-7.491376E-3,1.3625825E-2,3.167741E-3,-1.8143073E-2,7.150901E-4,7.7934284E-3,1.921363E-2,-7.2814217E-3,-1.9394836E-3,1.3743006E-3,-2.8445057E-3,-1.6929297E-2,1.840906E-3,-2.6223448E-4,-2.3662644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1],"loss_changes":[1.4413743E-1,7.156868E-2,8.195037E-2,6.771883E-2,0E0,1.15950465E-1,7.4631155E-2,7.082224E-2,0E0,5.3081635E-2,0E0,0E0,0E0,5.7851993E-2,0E0,0E0,4.4458613E-2,7.34075E-2,0E0,0E0,0E0,4.9856395E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13,16,16,17,17,21,21],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.6219076E9,8.10288E3,2.388522E-1,4.242889E3,5.4631056E-3,3.768034E4,1.6441814E9,1.4720299E3,-7.0322864E-3,1.5988636E1,-7.491376E-3,1.3625825E-2,3.167741E-3,8.227176E-3,7.150901E-4,7.7934284E-3,1.5953E4,6.2581446E8,-1.9394836E-3,1.3743006E-3,-2.8445057E-3,4.2726562E1,1.840906E-3,-2.6223448E-4,-2.3662644E-3],"split_indices":[7,64,50,70,0,4,7,64,0,68,0,0,0,69,0,0,0,7,0,0,0,65,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.52E2,3.55E2,9.7E1,3.51E2,4E0,7.2E1,2.5E1,3.48E2,3E0,6.9E1,3E0,1E0,2.4E1,2.58E2,9E1,2E0,6.7E1,1.7E2,8.8E1,6.1E1,6E0,1.4E2,3E1,1.02E2,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.4567467E-3,-1.439525E-2,2.21131E-2,-1.6210277E-2,8.379526E-3,1.0068247E-2,5.741454E-2,-1.4920245E-2,-9.817784E-3,-1.277596E-3,2.412581E-2,3.7584007E-3,-2.625312E-4,-2.8482422E-2,-3.3966477E-3,5.66768E-4,7.6855E-2,-5.102506E-2,-1.4886961E-2,-4.1062804E-3,1.8772137E-3,2.420301E-3,1.3285341E-2,-1.1120378E-3,-3.9435457E-3,3.108493E-3,-1.0759338E-3,-6.651892E-3,3.1172158E-2,5.345242E-4,-3.4645557E-2,2.294754E-3,-3.1750596E-3,-4.0618945E-3,-1.5225825E-2,1.3052122E-3,-2.352041E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,15,-1,-1,17,19,-1,21,23,25,-1,27,-1,-1,-1,-1,-1,-1,29,31,-1,33,-1,-1,-1,35,-1,-1],"loss_changes":[1.3937941E-1,1.0020207E-1,6.830444E-2,6.519563E-2,0E0,6.172713E-2,4.6102807E-2,4.5538493E-2,0E0,0E0,5.8453083E-2,0E0,0E0,4.030838E-2,6.620489E-2,0E0,7.844783E-2,3.8359836E-2,4.4859946E-2,0E0,3.7686307E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.7188086E-2,5.0394636E-2,0E0,3.9512943E-2,0E0,0E0,0E0,4.4665188E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,10,10,13,13,14,14,16,16,17,17,18,18,20,20,27,27,28,28,30,30,34,34],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,16,-1,-1,18,20,-1,22,24,26,-1,28,-1,-1,-1,-1,-1,-1,30,32,-1,34,-1,-1,-1,36,-1,-1],"split_conditions":[1.05381016E5,8.861901E3,1.6846506E5,1.3180642E7,8.379526E-3,4.359929E1,2.84E2,7.805608E4,-9.817784E-3,-1.277596E-3,2.0901188E7,3.7584007E-3,-2.625312E-4,1.6776951E3,2.2744698E2,5.66768E-4,2.9632012E7,1.6225006E4,4.2969458E2,-4.1062804E-3,4.723148E11,2.420301E-3,1.3285341E-2,-1.1120378E-3,-3.9435457E-3,3.108493E-3,-1.0759338E-3,1.7239808E7,1.929271E6,5.345242E-4,6.603817E2,2.294754E-3,-3.1750596E-3,-4.0618945E-3,2.0353708E3,1.3052122E-3,-2.352041E-3],"split_indices":[45,64,40,59,0,68,8,40,0,0,9,0,0,4,64,0,1,40,64,0,43,0,0,0,0,0,0,57,2,0,64,0,0,0,67,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.55E2,2.94E2,1.61E2,2.92E2,2E0,1.21E2,4E1,2.91E2,1E0,3.4E1,8.7E1,3.1E1,9E0,1.33E2,1.58E2,7.1E1,1.6E1,4.9E1,8.4E1,9E0,1.49E2,1.5E1,1E0,2.5E1,2.4E1,6E0,7.8E1,1.16E2,3.3E1,7.2E1,4.4E1,2.9E1,4E0,1.2E1,3.2E1,1.4E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.3349347E-3,-1.1454055E-3,4.8283143E-3,-8.797994E-3,2.1234712E-2,-7.5815814E-3,-9.42976E-3,1.6176079E-3,-2.0999634E-3,-1.8322743E-3,-1.5505636E-3,9.0180174E-4,-6.366056E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[1.3824834E-1,6.9892004E-2,0E0,6.508992E-2,7.459718E-2,5.3086534E-2,0E0,0E0,0E0,0E0,5.5474404E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,10,10],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[6.0538214E9,5.4779566E9,4.8283143E-3,2.47815E5,2.2967976E7,1.3202006E3,-9.42976E-3,1.6176079E-3,-2.0999634E-3,-1.8322743E-3,7.477264E6,9.0180174E-4,-6.366056E-4],"split_indices":[7,5,0,2,41,4,0,0,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,4.06E2,1.4E1,3.03E2,1.03E2,3.02E2,1E0,8.8E1,1.5E1,5.1E1,2.51E2,9.1E1,1.6E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.6614007E-4,-2.7712164E-3,6.070747E-3,-8.236409E-3,2.979659E-2,-5.7226825E-3,-9.349261E-3,5.120406E-2,-5.237731E-4,-2.4833063E-2,4.2195097E-3,-2.297449E-3,6.327826E-2,-2.6930284E-2,5.7729334E-3,5.1385295E-3,1.6513156E-3,9.543727E-3,2.5063292E-3,-3.9867926E-2,-1.1164478E-4,9.439829E-3,-2.6175365E-2,-5.48291E-2,1.1051044E-2,2.5118568E-3,2.1855347E-4,-3.8490375E-4,-6.999857E-2,-1.9224414E-3,-5.4063764E-3,2.4131944E-3,-2.3081077E-3,-8.077932E-3,-1.0985597E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,7,9,-1,11,-1,13,15,-1,17,19,-1,-1,21,-1,-1,23,-1,25,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.21200666E-1,7.55349E-2,0E0,1.6211629E-1,5.3132676E-2,6.8314455E-2,0E0,4.9841367E-2,0E0,3.9314464E-2,5.9844453E-2,0E0,5.076769E-2,3.8891412E-2,0E0,0E0,5.0528094E-2,0E0,0E0,6.1550498E-2,0E0,3.7422933E-2,4.0749084E-2,4.995896E-2,4.2723082E-2,0E0,0E0,0E0,6.244906E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,16,16,19,19,21,21,22,22,23,23,24,24,28,28],"right_children":[2,4,-1,6,8,10,-1,12,-1,14,16,-1,18,20,-1,-1,22,-1,-1,24,-1,26,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0871896E10,6.0784E4,6.070747E-3,2.235E5,1.40368E5,1.7066003E3,-9.349261E-3,1.1804314E0,-5.237731E-4,1.3989E4,5.738E3,-2.297449E-3,9.957E3,2.55E2,5.7729334E-3,5.1385295E-3,2.4722598E7,9.543727E-3,2.5063292E-3,2.613714E6,-1.1164478E-4,1.0106809E1,6.54925E6,1.1160287E7,1E0,2.5118568E-3,2.1855347E-4,-3.8490375E-4,4.3290512E1,-1.9224414E-3,-5.4063764E-3,2.4131944E-3,-2.3081077E-3,-8.077932E-3,-1.0985597E-3],"split_indices":[7,10,0,2,10,4,0,65,0,2,9,0,9,3,0,0,57,0,0,9,0,70,9,57,26,0,0,0,68,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,4.22E2,7E0,3.62E2,6E1,3.58E2,4E0,3.9E1,2.1E1,1.22E2,2.36E2,4E0,3.5E1,1.21E2,1E0,5E0,2.31E2,2E0,3.3E1,7.9E1,4.2E1,1.81E2,5E1,6.1E1,1.8E1,1.9E1,1.62E2,3.6E1,1.4E1,4.8E1,1.3E1,1.1E1,7E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.5935398E-3,-3.1889186E-3,4.9055573E-2,-1.6836694E-3,-7.5105275E-3,9.519404E-3,3.8077895E-2,-4.8786888E-4,1.3653868E-2,4.9860578E-2,-5.152383E-2,4.125789E-2,3.3038415E-3,1.0143504E-3,1.0129096E-1,-8.960509E-3,-1.7283037E-5,8.447715E-4,5.2636988E-3,6.54983E-4,-1.9577527E-3,9.060968E-4,8.296339E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,-1,-1,9,-1,11,13,15,17,19,-1,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4014478E-1,8.742629E-2,8.434464E-2,4.882694E-2,0E0,0E0,6.1690934E-2,0E0,3.8719222E-2,7.396816E-2,4.571184E-2,5.59507E-2,4.215036E-2,0E0,9.386155E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,9,9,10,10,11,11,12,12,14,14],"right_children":[2,4,6,8,-1,-1,10,-1,12,14,16,18,20,-1,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,1.3674345E4,2.909019E3,7.10711E8,-7.5105275E-3,9.519404E-3,1.93E2,-4.8786888E-4,5.473125E3,2.0928232E1,1.379E3,2.5573445E6,2.237815E7,1.0143504E-3,1.69595E5,-8.960509E-3,-1.7283037E-5,8.447715E-4,5.2636988E-3,6.54983E-4,-1.9577527E-3,9.060968E-4,8.296339E-3],"split_indices":[64,4,44,7,0,0,8,0,44,66,0,62,57,0,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.53E2,3.95E2,5.8E1,3.92E2,3E0,3E0,5.5E1,2.57E2,1.35E2,4.9E1,6E0,3.6E1,9.9E1,3.2E1,1.7E1,1E0,5E0,2.7E1,9E0,8.1E1,1.8E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.770981E-3,-6.8052746E-3,2.4614258E-2,-6.1822747E-4,1.1728033E-3,9.7862E-2,1.2499398E-2,2.2103896E-3,8.218987E-3,1.9971963E-2,-7.842133E-2,6.36417E-3,8.0613495E-4,-1.1952125E-2,-8.342836E-3,6.9880723E-3,-2.4730975E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,-1,-1,7,9,-1,-1,11,13,-1,-1,-1,15,-1,-1],"loss_changes":[9.760367E-2,4.982125E-2,1.3099732E-1,0E0,0E0,6.773332E-2,8.921026E-2,0E0,0E0,4.7443036E-2,1.1064582E-1,0E0,0E0,0E0,5.5688884E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,9,9,10,10,14,14],"right_children":[2,4,6,-1,-1,8,10,-1,-1,12,14,-1,-1,-1,16,-1,-1],"split_conditions":[5.837049E6,3.53E2,1.417988E6,-6.1822747E-4,1.1728033E-3,9.256843E3,1.4609149E4,2.2103896E-3,8.218987E-3,1.4E1,1.3936486E8,6.36417E-3,8.0613495E-4,-1.1952125E-2,1.3812E4,6.9880723E-3,-2.4730975E-3],"split_indices":[1,8,41,0,0,4,4,0,0,3,1,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,2.93E2,1.48E2,2.48E2,4.5E1,2E1,1.28E2,1.2E1,8E0,1.19E2,9E0,3E0,1.16E2,2E0,7E0,1E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[4.4245797E-4,-5.1608505E-3,4.6877827E-2,-3.5756961E-3,-6.052757E-3,1.139685E-3,4.452132E-3,-1.1354662E-2,2.466775E-2,-1.0225511E-2,-8.437944E-3,6.399685E-3,9.483774E-4,-8.891559E-3,-6.7183687E-3,-2.942466E-2,-7.439917E-5,-7.2903914E-4,-3.819273E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,-1,-1,15,-1,17,-1,-1,-1],"loss_changes":[1.0874187E-1,6.812514E-2,4.3691523E-2,8.135116E-2,0E0,0E0,0E0,4.9842574E-2,4.4641092E-2,4.668765E-2,0E0,0E0,0E0,4.358127E-2,0E0,5.18796E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,13,13,15,15],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,-1,-1,16,-1,18,-1,-1,-1],"split_conditions":[2.6422684E9,1.0067637E7,2.2876814E1,5.5292703E9,-6.052757E-3,1.139685E-3,4.452132E-3,3.2006908E7,3.0969632E2,5.5051494E9,-8.437944E-3,6.399685E-3,9.483774E-4,1.5031561E3,-6.7183687E-3,4.70436E5,-7.439917E-5,-7.2903914E-4,-3.819273E-3],"split_indices":[7,59,66,5,0,0,0,1,45,5,0,0,0,4,0,41,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.16E2,3.72E2,4.4E1,3.68E2,4E0,2.9E1,1.5E1,2.89E2,7.9E1,2.88E2,1E0,3E0,7.6E1,2.86E2,2E0,7.5E1,2.11E2,5.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.4875304E-3,-3.4001325E-3,5.80789E-3,8.639358E-3,-1.4305864E-2,7.00403E-2,4.117209E-3,-1.7585488E-2,2.0381366E-3,1.642436E-3,7.734491E-3,-1.5564246E-2,7.638626E-4,-7.506525E-4,-5.198058E-3,-2.3548726E-3,5.34564E-3,4.3585626E-3,-3.8588815E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,-1,15,-1,-1,-1,-1,17,-1,-1],"loss_changes":[9.85453E-2,5.6582358E-2,0E0,5.6632794E-2,4.1548207E-2,3.900916E-2,4.2437535E-2,4.5882203E-2,0E0,0E0,0E0,4.643978E-2,0E0,0E0,0E0,0E0,4.6673454E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,11,11,16,16],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,-1,16,-1,-1,-1,-1,18,-1,-1],"split_conditions":[1.0871896E10,1.33E2,5.80789E-3,3.7577084E2,9.2242E4,4.230839E6,6.927258E4,1.99814E5,2.0381366E-3,1.642436E-3,7.734491E-3,5.9376766E1,7.638626E-4,-7.506525E-4,-5.198058E-3,-2.3548726E-3,5.9073865E2,4.3585626E-3,-3.8588815E-4],"split_indices":[7,3,0,45,10,1,45,2,0,0,0,66,0,0,0,0,59,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,4.29E2,6E0,2.04E2,2.25E2,1.3E1,1.91E2,2.13E2,1.2E1,1E1,3E0,6.9E1,1.22E2,2.08E2,5E0,2.7E1,4.2E1,5E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.514629E-3,-7.583127E-3,2.5424462E-2,-1.6222168E-2,6.31014E-4,2.9532779E-2,-8.11846E-3,-1.4181763E-3,2.2991034E-4,3.723783E-2,-6.2971143E-4,1.4901076E-3,4.5423433E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,-1],"loss_changes":[1.0059344E-1,5.2931074E-2,1.074516E-1,5.3787116E-2,0E0,4.2996667E-2,0E0,0E0,0E0,4.148771E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,-1],"split_conditions":[5.4779566E9,1.4171124E3,5.374925E7,9.704588E6,6.31014E-4,1.5953E4,-8.11846E-3,-1.4181763E-3,2.2991034E-4,1.00973885E2,-6.2971143E-4,1.4901076E-3,4.5423433E-3],"split_indices":[5,64,57,9,0,0,0,0,0,68,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.01E2,1.32E2,2.11E2,9E1,1.3E2,2E0,1.33E2,7.8E1,1.1E2,2E1,9.8E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.608586E-4,-5.1743905E-3,4.697952E-2,7.6955543E-3,-1.6441539E-2,5.7546433E-2,-4.4442015E-3,2.6359228E-2,-6.467867E-4,-1.4422469E-2,-5.888559E-3,3.5687543E-2,8.8635655E-3,1.9271031E-3,-1.0102665E-3,-4.2104147E-2,-3.9102763E-4,4.6697054E-3,8.2585687E-4,4.999236E-4,-5.9806727E-2,2.0934251E-4,-4.1614207E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,17,-1,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.20788395E-1,5.7694536E-2,7.692279E-2,7.198298E-2,4.169818E-2,1.16759226E-1,0E0,5.6469478E-2,0E0,3.7844118E-2,0E0,4.382893E-2,0E0,0E0,0E0,3.7495673E-2,0E0,0E0,0E0,0E0,4.5391828E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,11,11,15,15,20,20],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,18,-1,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[4.23168E3,9.362063E6,1.93E2,1.33E2,3.764913E7,3.305542E4,-4.4442015E-3,2.236341E0,-6.467867E-4,1.6858337E3,-5.888559E-3,9.957E3,8.8635655E-3,1.9271031E-3,-1.0102665E-3,3.543E3,-3.9102763E-4,4.6697054E-3,8.2585687E-4,4.999236E-4,8.553271E-5,2.0934251E-4,-4.1614207E-3],"split_indices":[64,57,8,3,60,4,0,53,0,4,0,9,0,0,0,2,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.45E2,3.96E2,4.9E1,1.85E2,2.11E2,4.6E1,3E0,9.7E1,8.8E1,2.08E2,3E0,4E1,6E0,7.7E1,2E1,3.9E1,1.69E2,9E0,3.1E1,1E1,2.9E1,8E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.8960129E-3,-1.7645143E-4,4.2399704E-2,6.449769E-3,3.0174833E-2,4.335445E-5,3.1038662E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":78,"left_children":[1,-1,3,-1,5,-1,-1],"loss_changes":[9.4884634E-2,0E0,5.013389E-2,0E0,4.2995106E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4],"right_children":[2,-1,4,-1,6,-1,-1],"split_conditions":[7.2038723E9,-1.7645143E-4,1.3812E4,6.449769E-3,4.09591E0,4.335445E-5,3.1038662E-3],"split_indices":[12,0,9,0,65,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.3E2,3.8E2,5E1,5E0,4.5E1,2.4E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[9.320372E-5,-8.848887E-3,2.8536681E-2,-1.2093044E-2,1.2837705E-3,3.5204917E-2,-5.590775E-3,-5.0592783E-4,-3.796509E-3,2.8014757E-2,1.3031024E-2,4.9126926E-3,1.079404E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,-1],"loss_changes":[1.1165526E-1,3.77327E-2,1.02245435E-1,3.772823E-2,0E0,1.5648402E-1,0E0,0E0,0E0,4.2523876E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,-1],"split_conditions":[2.1959E4,4.61E2,3.768034E4,4.14E2,1.2837705E-3,3.305542E4,-5.590775E-3,-5.0592783E-4,-3.796509E-3,9.957E3,1.3031024E-2,4.9126926E-3,1.079404E-3],"split_indices":[2,3,4,3,0,4,0,0,0,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.33E2,1.04E2,3.05E2,2.8E1,1E2,4E0,2.97E2,8E0,9.8E1,2E0,7E0,9.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[5.997633E-3,1.3719666E-3,8.272882E-2,3.110966E-3,-4.3831966E-3,8.617526E-3,1.517872E-3,-1.3718063E-4,1.5506065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[1.5605696E-1,6.471389E-2,1.1217858E-1,6.6945665E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[8.10288E3,5.30113E5,1.681178E6,1.6219076E9,-4.3831966E-3,8.617526E-3,1.517872E-3,-1.3718063E-4,1.5506065E-3],"split_indices":[64,2,41,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,4.15E2,2.4E1,4.08E2,7E0,8E0,1.6E1,3.38E2,7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.5147931E-3,-9.452077E-3,1.9095305E-2,-5.821288E-3,-4.4933233E-2,3.4971032E-3,5.0503353E-4,-8.86664E-3,1.8711337E-3,-4.824376E-3,-8.439133E-4,-3.8430656E-4,-8.073147E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[7.117777E-2,4.018935E-2,5.463865E-2,3.801179E-2,4.0339645E-2,0E0,0E0,4.69096E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[5.5015055E9,2.1730006E11,2.064164E6,2.3528348E9,1.0517595E3,3.4971032E-3,5.0503353E-4,2.2860639E9,1.8711337E-3,-4.824376E-3,-8.439133E-4,-3.8430656E-4,-8.073147E-3],"split_indices":[5,43,41,12,64,0,0,12,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.13E2,1.2E2,2.85E2,2.8E1,1.7E1,1.03E2,2.67E2,1.8E1,9E0,1.9E1,2.66E2,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[7.317351E-4,-4.651968E-3,2.5586313E-2,1.4756955E-4,-2.4314823E-2,9.582981E-2,1.0918317E-2,-1.6035828E-4,-4.910184E-2,3.058532E-3,1.2778599E-2,-5.316873E-3,8.205246E-4,-1.661268E-2,-7.683766E-2,-8.272678E-5,-8.263973E-3,-7.1197334E-3,-2.070778E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,-1,7,9,11,-1,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[5.7121944E-2,5.2557357E-2,7.7258885E-2,0E0,5.1111914E-2,5.6320697E-2,4.3038875E-2,0E0,3.9273627E-2,0E0,0E0,0E0,0E0,4.867285E-2,4.985571E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,13,13,14,14],"right_children":[2,4,6,-1,8,10,12,-1,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[3.2784192E3,1.7239808E7,1.064816E6,1.4756955E-4,1.4935554E11,4.916536E7,1.379E3,-1.6035828E-4,2.4E1,3.058532E-3,1.2778599E-2,-5.316873E-3,8.205246E-4,1.1367206E8,7.121916E2,-8.272678E-5,-8.263973E-3,-7.1197334E-3,-2.070778E-3],"split_indices":[64,57,41,0,43,1,0,0,8,0,0,0,0,1,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,3.5E2,7.5E1,2.53E2,9.7E1,1.2E1,6.3E1,5.3E1,4.4E1,1.1E1,1E0,2E0,6.1E1,2.1E1,2.3E1,2E1,1E0,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[4.5351475E-3,-2.2029572E-3,7.411274E-3,6.1844926E-2,4.542152E-3,4.6532806E-3,-2.9977407E-5,6.77689E-3,-4.965478E-2,4.9521375E-5,2.7254438E-2,-4.780898E-5,-7.826223E-3,4.0257527E-3,7.801361E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,-1,3,5,7,-1,-1,9,11,-1,13,-1,-1,-1,-1],"loss_changes":[6.254835E-2,0E0,6.546952E-2,4.1055188E-2,4.8903994E-2,0E0,0E0,4.5707382E-2,8.317824E-2,0E0,5.149597E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,7,7,8,8,10,10],"right_children":[2,-1,4,6,8,-1,-1,10,12,-1,14,-1,-1,-1,-1],"split_conditions":[9.63428E2,-2.2029572E-3,1.0097E4,2.8184534E7,4.5859156E7,4.6532806E-3,-2.9977407E-5,2.940448E6,2.003E3,4.9521375E-5,8.169258E3,-4.780898E-5,-7.826223E-3,4.0257527E-3,7.801361E-4],"split_indices":[4,0,9,57,57,0,0,41,0,0,45,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.44E2,2.4E1,4.2E2,2E1,4E2,1.3E1,7E0,3.85E2,1.5E1,3.01E2,8.4E1,1.1E1,4E0,1.4E1,7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.6320193E-3,-4.3623816E-4,1.968047E-2,3.3609013E-3,1.5325875E-2,6.047856E-4,9.064974E-2,-3.8844817E-3,9.385133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,-1,3,-1,5,-1,7,-1,-1],"loss_changes":[8.9767E-2,0E0,4.9098887E-2,0E0,5.306354E-2,0E0,1.6180554E-1,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,6,6],"right_children":[2,-1,4,-1,6,-1,8,-1,-1],"split_conditions":[6.97E3,-4.3623816E-4,6.798766E-4,3.3609013E-3,3.305542E4,6.047856E-4,1.5435187E2,-3.8844817E-3,9.385133E-3],"split_indices":[2,0,51,0,4,0,70,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,2.05E2,2.41E2,1.9E1,2.22E2,2.14E2,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.5456805E-3,-4.2547244E-3,2.5850266E-2,-5.9326465E-4,5.8211706E-4,8.0401026E-2,3.7743204E-4,6.585361E-4,6.293344E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[6.439524E-2,4.4830967E-2,8.72318E-2,0E0,0E0,6.5670446E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.8163872E7,6.211238E5,2.1207E4,-5.9326465E-4,5.8211706E-4,2.565595E0,3.7743204E-4,6.585361E-4,6.293344E-3],"split_indices":[1,59,9,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.55E2,3.68E2,8.7E1,2.49E2,1.19E2,2.1E1,6.6E1,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.4880665E-3,-3.502576E-4,3.5587521E-3,-2.2103162E-3,1.7347862E-3,5.5235E-4,-8.830971E-3,-1.7932688E-3,1.457947E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[5.488164E-2,3.8317848E-2,0E0,0E0,3.947618E-2,0E0,5.9852753E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[1.6941456E1,2.849E3,3.5587521E-3,-2.2103162E-3,1.8887723E-2,5.5235E-4,1.156185E3,-1.7932688E-3,1.457947E-4],"split_indices":[47,2,0,0,69,0,64,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.27E2,4.17E2,1E1,1.8E1,3.99E2,2.12E2,1.87E2,5.6E1,1.31E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-5.086955E-4,-9.392726E-3,2.560235E-2,-1.2143699E-4,-1.4153268E-3,1.8314151E-2,9.9568196E-2,6.176781E-3,6.4573664E-4,1.1929295E-2,3.4947716E-2,4.51772E-3,-2.1540504E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,-1,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[1.0416363E-1,4.4080127E-2,5.9631355E-2,0E0,0E0,5.802239E-2,8.1565656E-2,0E0,0E0,0E0,3.8518928E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,10,10],"right_children":[2,4,6,-1,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[2.143564E3,1.8071064E7,3.305542E4,-1.2143699E-4,-1.4153268E-3,8.534E3,3.768034E4,6.176781E-3,6.4573664E-4,1.1929295E-2,2.1358348E7,4.51772E-3,-2.1540504E-3],"split_indices":[64,57,4,0,0,9,4,0,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.34E2,1.13E2,2.45E2,8.9E1,1.04E2,9E0,4E0,1E2,2E0,7E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.640311E-5,-2.8585454E-3,2.3645193E-3,-3.8125531E-3,-1.4292703E-3,-3.9724482E-4,1.1720464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[6.1631534E-2,4.397007E-2,0E0,0E0,6.7180224E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[7.0484486E0,2.1066585E2,2.3645193E-3,-3.8125531E-3,1.5271514E5,-3.9724482E-4,1.1720464E-3],"split_indices":[47,64,0,0,40,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.44E2,4.19E2,2.5E1,7E0,4.12E2,3.27E2,8.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-2.8323252E-3,-1.5177465E-3,-4.2039575E-3,-3.9357827E-3,3.8295478E-2,2.5520467E-3,-1.8138263E-2,5.1070424E-3,6.3669233E-4,-7.589453E-4,8.236626E-4,-6.1916307E-2,-4.2772095E-4,-6.468739E-3,-1.5228546E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,-1,-1,-1],"loss_changes":[4.6996817E-2,4.202809E-2,0E0,3.7917174E-2,3.9519414E-2,7.017312E-2,5.357046E-2,0E0,0E0,0E0,0E0,4.4888385E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,-1,-1,-1],"split_conditions":[4.8537783E3,9.430977E3,-4.2039575E-3,1.7239808E7,3.2946018E6,7.796878E4,1.960369E7,5.1070424E-3,6.3669233E-4,-7.589453E-4,8.236626E-4,1.2309012E1,-4.2772095E-4,-6.468739E-3,-1.5228546E-3],"split_indices":[69,4,0,57,59,40,57,0,0,0,0,70,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,4.34E2,6E0,4.1E2,2.4E1,2.82E2,1.28E2,6E0,1.8E1,1.24E2,1.58E2,2.2E1,1.06E2,6E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.2353722E-3,-3.2569517E-4,1.4113727E-2,-3.2355892E-4,2.7614553E-2,2.4603676E-2,1.4085202E-1,3.5940714E-2,-2.3008192E-4,1.1906386E-2,-1.3475377E-3,2.058871E-3,-2.9763326E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,-1,3,-1,5,7,9,11,-1,-1,-1,-1,-1],"loss_changes":[4.6028942E-2,0E0,5.7159565E-2,0E0,3.8811587E-2,4.0562935E-2,5.5454392E-2,4.662647E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,6,6,7,7],"right_children":[2,-1,4,-1,6,8,10,12,-1,-1,-1,-1,-1],"split_conditions":[1.0975164E3,-3.2569517E-4,7.959718E4,-3.2355892E-4,1.5324995E4,1.2730462E7,1.492E3,1.929271E6,-2.3008192E-4,1.1906386E-2,-1.3475377E-3,2.058871E-3,-2.9763326E-3],"split_indices":[64,0,40,0,64,9,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,2.28E2,2.04E2,8.1E1,1.23E2,1.21E2,2E0,8.7E1,3.4E1,1E0,1E0,8.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.0338613E-3,-1.44663155E-2,6.743359E-3,-6.092748E-4,-5.0124833E-3,3.3679686E-3,9.95236E-3,5.6080264E-3,-6.2268856E-3,-7.425854E-3,6.9477805E-3,3.0603059E-2,-6.04302E-6,9.2832965E-4,4.719382E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,-1,7,-1,9,-1,-1,11,13,-1,-1,-1],"loss_changes":[4.827371E-2,3.7635047E-2,1.5064463E-1,0E0,0E0,6.662816E-2,0E0,4.7927126E-2,0E0,0E0,3.7793618E-2,3.816717E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,10,10,11,11],"right_children":[2,4,6,-1,-1,8,-1,10,-1,-1,12,14,-1,-1,-1],"split_conditions":[8.361085E4,4.42384E5,3.305542E4,-6.092748E-4,-5.0124833E-3,4.352718E7,9.95236E-3,1.44E2,-6.2268856E-3,-7.425854E-3,1.2452798E3,6.840166E6,-6.04302E-6,9.2832965E-4,4.719382E-3],"split_indices":[40,2,4,0,0,57,0,0,0,0,45,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,1.98E2,2.32E2,1.94E2,4E0,2.29E2,3E0,2.26E2,3E0,1E0,2.25E2,5.1E1,1.74E2,4.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.6258632E-3,-2.8895526E-3,2.8265058E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":92,"left_children":[1,-1,-1],"loss_changes":[5.4581713E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[6.744371E2,-2.8895526E-3,2.8265058E-4],"split_indices":[67,0,0],"split_type":[0,0,0],"sum_hessian":[4.34E2,1.3E1,4.21E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-2.3945232E-3,-1.2231922E-2,9.752088E-3,-5.041748E-4,-9.0135485E-3,-3.2860443E-2,1.6713258E-2,-2.6360992E-3,1.7159383E-3,4.002852E-3,1.2672672E-2,4.163746E-3,2.2764516E-3,2.1918885E-2,-1.42113E-2,7.079407E-4,8.996505E-3,-4.786922E-4,-7.6484005E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,-1,-1,7,9,-1,-1,-1,11,13,-1,15,17,-1,-1,-1,-1],"loss_changes":[5.257586E-2,8.575614E-2,5.898592E-2,0E0,0E0,3.92276E-2,4.2522103E-2,0E0,0E0,0E0,4.4851333E-2,4.2397942E-2,0E0,7.835111E-2,3.969773E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,10,10,11,11,13,13,14,14],"right_children":[2,4,6,-1,-1,8,10,-1,-1,-1,12,14,-1,16,18,-1,-1,-1,-1],"split_conditions":[5.3257984E1,4.352718E7,6.944307E7,-5.041748E-4,-9.0135485E-3,9.309699E8,4.9832974E2,-2.6360992E-3,1.7159383E-3,4.002852E-3,2.3781708E5,7.555878E1,2.2764516E-3,6.204522E3,2.3767492E4,7.079407E-4,8.996505E-3,-4.786922E-4,-7.6484005E-3],"split_indices":[68,57,7,0,0,5,64,0,0,0,45,68,0,64,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,2.42E2,1.96E2,2.4E2,2E0,2.7E1,1.69E2,2.1E1,6E0,9E0,1.6E2,1.28E2,3.2E1,6.5E1,6.3E1,6.3E1,2E0,6.2E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-2.8932016E-4,-4.2714477E-3,2.2612168E-2,-1.6022487E-4,-7.0464606E-3,4.7503058E-2,-5.247355E-4,6.8740016E-3,1.5100779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[4.186465E-2,5.650465E-2,5.654332E-2,0E0,0E0,5.6550294E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[3.33069E3,1.1770536E8,1E0,-1.6022487E-4,-7.0464606E-3,2.3528348E9,-5.247355E-4,6.8740016E-3,1.5100779E-3],"split_indices":[64,1,24,0,0,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.57E2,3.9E2,6.7E1,3.88E2,2E0,3.8E1,2.9E1,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.7021253E-4,-3.6585496E-3,2.655077E-2,-1.2234553E-2,5.130406E-4,-3.9796607E-4,2.488876E-3,5.8548115E-3,-2.2336284E-2,1.3355107E-2,-3.3986196E-3,-7.438595E-3,-9.814933E-4,-2.9382936E-4,1.9871562E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[4.383225E-2,4.538373E-2,4.453231E-2,4.3073244E-2,0E0,0E0,0E0,4.799565E-2,4.8596814E-2,3.9852034E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[2.3624256E5,1.0368186E-1,4.95E9,1.0878975E4,5.130406E-4,-3.9796607E-4,2.488876E-3,3.9034148E7,1.3436805E4,2.613714E6,-3.3986196E-3,-7.438595E-3,-9.814933E-4,-2.9382936E-4,1.9871562E-3],"split_indices":[40,50,5,60,0,0,0,57,62,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,3.78E2,5.4E1,2.34E2,1.44E2,2.2E1,3.2E1,8.4E1,1.5E2,7.7E1,7E0,2E0,1.48E2,4.5E1,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[4.3620815E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0],"id":96,"left_children":[-1],"loss_changes":[0E0],"parents":[2147483647],"right_children":[-1],"split_conditions":[2.1799975E-5],"split_indices":[0],"split_type":[0],"sum_hessian":[4.39E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"1","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics index d443a3fa7..5f5bdac0d 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics @@ -1,109 +1,109 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,duration_max,0.24739769,82414.25463743677,311681.84687962633,1504.0,4702.0,7190.0,20850.0,3183955.0 -1,duration_sum,0.07175633,16930121.96627319,45073353.38047576,3197.0,1082442.0,2678990.0,10368109.0,515693115.0 -2,executorCPUTime_mean,0.06844651,2722.177687148267,7012.88983905512,23.339985149433822,585.2242380261248,1063.8777813608513,2163.7598204264873,68323.01515151515 -3,peakExecutionMemory_max,0.066340074,1233353538.7048905,2601007219.6365647,0.0,85937632.0,471303176.0,1493186488.0,25501360736.0 -4,shuffle_write_bw,0.040212527,88801.55237700409,93971.75983640963,0.0,1351.9543033361367,52744.59381627414,166023.2400627615,326131.0503035477 -5,input_bytesRead_mean,0.03796609,15367291.833589979,13710603.535320966,21863.21726572529,5113159.273237375,11711469.0760095,21549125.812765956,92298335.07638542 -6,scan_bw,0.030263554,110505.86766836004,89612.1377495093,0.0,45187.54901663616,92722.52201161183,143609.24321367222,465063.5526726399 -7,memoryBytesSpilledRatio,0.02068907,1.1100054272857585,3.9835905644793064,0.0,0.0,0.0,0.0,43.971493342592005 -8,sw_writeTime_mean,0.018493572,131.22221226581013,570.380294897338,0.0,16.074844074844076,33.17843866171004,70.21488764044943,6475.538414708389 -9,resultSize_max,0.015288353,10957325.65767285,22245624.20798865,2645.0,40208.0,1477382.0,12756524.0,263794922.0 -10,scan_time,0.012532342,5138075.225969646,16908006.48547024,0.0,322677.0,911898.0,2940448.0,217849366.0 -11,jvmGCTime_mean,0.011914344,63.70179049174891,111.16738988131927,0.0,31.140514631685168,49.11107416029265,70.23493975903614,1390.436668309512 -12,duration_min,0.011451286,217.46880269814503,213.6000053492964,5.0,77.0,148.0,295.0,1779.0 -13,sw_recordsWritten_sum,0.0113346195,2556345618.8684654,5824955615.583718,0.0,319167.0,53631567.0,2066354478.0,44204763473.0 -14,duration_mean,0.009216025,4389.021479495678,8051.0668064364545,245.4699965313909,1580.207299270073,2302.596153846154,3826.246829791034,72233.70872350912 -15,resultSerializationTime_sum,0.008987576,142.54300168634063,201.73330481784959,0.0,27.0,68.0,152.0,1135.0 -16,executorDeserializeTime_mean,0.008662206,77.619961219705,78.66729885547291,1.2399579021224347,16.74,57.15866922584773,111.59245283018868,438.3862068965517 -17,sr_localBytesRead_mean,0.008305384,1550809.257874584,3239093.401866476,0.0,2585.3312883435583,284627.3147440794,1918199.7854019508,29629639.667323805 -18,numTasks_sum,0.00764963,3448.059021922428,7298.431350220087,4.0,527.0,1156.0,3101.0,86122.0 -19,sr_remoteBytesRead_mean,0.006606229,8621230.876874011,20763598.714249212,0.0,14815.464675201518,1045794.9007633588,7696222.868940513,207400140.1710202 -20,sr_fetchWaitTime_mean,0.0065765264,282.57444030933357,2063.740776556858,0.0,0.0,0.014450867052023121,0.0744336569579288,26216.59561667383 -21,input_recordsRead_sum,0.006167026,7537500441.369308,19524836240.42767,30000.0,410551846.0,1658123725.0,5999989709.0,230400923204.0 -22,sr_remoteBytesReadRatio,0.0056557287,0.6856243468838018,1.1473671005608985,0.0,0.0015316802906526895,0.13301082808430215,1.0519672879672015,11.86796722526039 -23,executorDeserializeCPUTime_mean,0.0056107068,26.95913143088412,29.063846586016425,1.0653743823641202,6.2544901065449015,19.212783171521036,36.59782608695652,245.76923076923077 -24,executorRunTime_mean,0.004142396,4297.896908257712,8062.7718056424565,226.90115532734274,1490.5288200108755,2236.8281325012003,3741.984555984556,72194.51503203549 -25,sr_localBlocksFetched_sum,0.0039907056,34560.099494097805,81564.66670257768,0.0,329.0,5582.0,30418.0,870491.0 -26,sr_localBytesReadRatio,0.0037682652,0.13775142996534326,0.21834409881656594,0.0,0.0003435677540369934,0.052422868005860185,0.18397368378307108,1.6954497523177325 -27,sw_bytesWrittenRatio,0.0033000703,0.8014652434613514,1.2780756184305504,0.0,0.0023701803469738034,0.22863716255153424,1.206131247096677,13.563416977578123 -28,sqlOp_Expand,0.0026863571,0.11129848229342328,0.3147669133570336,0.0,0.0,0.0,0.0,1.0 -29,sr_remoteBlocksFetched_sum,0.0024882252,224234.64755480608,509217.62116016017,0.0,2164.0,37041.0,193536.0,3942472.0 -30,data_size,0.0022313634,650411028780.6053,2082773012482.3335,0.0,21721183788.0,89736852644.0,333792838797.0,21374406089219.0 -31,shuffle_read_bw,0.0021603908,276816955.4836904,847565126.6737152,0.0,0.0,4930489.4375,137218859.4920635,10530883927.666666 -32,sr_totalBytesRead_mean,0.0019514404,10172040.134748595,23817088.532298744,0.0,19575.48201438849,1695974.662708024,9072792.773504274,237029779.838344 -33,sqlOp_SortMergeJoin,0.0011491332,0.41652613827993257,0.49339906934430416,0.0,0.0,0.0,1.0,1.0 -34,sqlOp_SubqueryBroadcast,0.0011491068,0.7976391231028668,0.40209900029018336,0.0,1.0,1.0,1.0,1.0 -35,sw_bytesWritten_mean,0.0010734663,9961926.690653043,23574771.301399454,0.0,19575.48201438849,1687125.7618110236,9963399.731213873,237029779.838344 -36,diskBytesSpilled_mean,0.0008522308,4546045.185000375,15069027.640347425,0.0,0.0,0.0,0.0,111631484.38187319 -37,sqlOp_Subquery,0.00083412044,0.09106239460370995,0.28794070420987344,0.0,0.0,0.0,0.0,1.0 -38,sr_totalBytesReadRatio,0.00073427934,0.823375776849145,1.319042835980761,0.0,0.002400770847119191,0.22866221128988315,1.2670990125430908,13.563416977578123 -39,memoryBytesSpilled_mean,0.00065191,18194132.33421661,71075266.1814507,0.0,0.0,0.0,0.0,724082083.4348422 -40,sqlOp_AQEShuffleRead,0.0005745393,0.7065767284991569,0.45571511734167724,0.0,0.0,1.0,1.0,1.0 -41,diskBytesSpilledRatio,0.0005704398,0.24474497918684088,0.8105072165034726,0.0,0.0,0.0,0.0,6.729163387759685 -42,sqlOp_Sort,0.000531721,0.6053962900505903,0.4891780403788745,0.0,0.0,1.0,1.0,1.0 -43,sqlOp_Scan orc ,0.00048783096,0.45868465430016864,0.498710794080452,0.0,0.0,0.0,1.0,1.0 -44,sqlOp_Scan parquet ,0.00035646255,0.5311973018549747,0.49944707588925713,0.0,0.0,1.0,1.0,1.0 -45,sqlOp_BroadcastNestedLoopJoin,0.00023598468,0.04047217537942664,0.1972302630779745,0.0,0.0,0.0,0.0,1.0 -46,numExecutors,0.00018737219,8.062394603709949,0.3504133651237026,8.0,8.0,8.0,8.0,14.0 -47,sqlOp_HashAggregate,0.00012896206,0.9797639123102867,0.14092579360637603,0.0,1.0,1.0,1.0,1.0 -48,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_Window,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,sqlOp_BroadcastExchange,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 -81,sqlOp_BroadcastHashJoin,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 -82,sqlOp_ColumnarToRow,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -83,sqlOp_Exchange,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -84,sqlOp_Filter,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -85,sqlOp_Project,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -86,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -87,maxMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 -88,maxOnHeapMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 -89,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,executorMemory,0.0,40960.0,0.0,40960.0,40960.0,40960.0,40960.0,40960.0 -91,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -93,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -95,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -98,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,platform_dataproc,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -102,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,duration_max,0.21813749,82414.25463743677,311681.84687962633,1504.0,4702.0,7190.0,20850.0,3183955.0 +1,peakExecutionMemory_max,0.082618274,1233353538.7048905,2601007219.6365647,0.0,85937632.0,471303176.0,1493186488.0,25501360736.0 +2,executorCPUTime_mean,0.071007006,2722.177687148267,7012.88983905512,23.339985149433822,585.2242380261248,1063.8777813608513,2163.7598204264873,68323.01515151515 +3,duration_sum,0.06950031,16930121.96627319,45073353.38047576,3197.0,1082442.0,2678990.0,10368109.0,515693115.0 +4,shuffle_write_bw,0.053588066,88801.55237700409,93971.75983640963,0.0,1351.9543033361367,52744.59381627414,166023.2400627615,326131.0503035477 +5,input_bytesRead_mean,0.034031473,15367291.833589979,13710603.535320966,21863.21726572529,5113159.273237375,11711469.0760095,21549125.812765956,92298335.07638542 +6,scan_bw,0.028711345,110505.86766836004,89612.1377495093,0.0,45187.54901663616,92722.52201161183,143609.24321367222,465063.5526726399 +7,memoryBytesSpilledRatio,0.020162081,1.1100054272857585,3.9835905644793064,0.0,0.0,0.0,0.0,43.971493342592005 +8,sw_writeTime_mean,0.015406664,131.22221226581013,570.380294897338,0.0,16.074844074844076,33.17843866171004,70.21488764044943,6475.538414708389 +9,sw_recordsWritten_sum,0.014074505,2556345618.8684654,5824955615.583718,0.0,319167.0,53631567.0,2066354478.0,44204763473.0 +10,resultSize_max,0.013840832,10957325.65767285,22245624.20798865,2645.0,40208.0,1477382.0,12756524.0,263794922.0 +11,scan_time,0.01295209,5138075.225969646,16908006.48547024,0.0,322677.0,911898.0,2940448.0,217849366.0 +12,duration_mean,0.011668744,4389.021479495678,8051.0668064364545,245.4699965313909,1580.207299270073,2302.596153846154,3826.246829791034,72233.70872350912 +13,duration_min,0.010455646,217.46880269814503,213.6000053492964,5.0,77.0,148.0,295.0,1779.0 +14,jvmGCTime_mean,0.0098547805,63.70179049174891,111.16738988131927,0.0,31.140514631685168,49.11107416029265,70.23493975903614,1390.436668309512 +15,sr_localBytesRead_mean,0.008638238,1550809.257874584,3239093.401866476,0.0,2585.3312883435583,284627.3147440794,1918199.7854019508,29629639.667323805 +16,resultSerializationTime_sum,0.008087278,142.54300168634063,201.73330481784959,0.0,27.0,68.0,152.0,1135.0 +17,executorDeserializeCPUTime_mean,0.006968015,26.95913143088412,29.063846586016425,1.0653743823641202,6.2544901065449015,19.212783171521036,36.59782608695652,245.76923076923077 +18,numTasks_sum,0.006788302,3448.059021922428,7298.431350220087,4.0,527.0,1156.0,3101.0,86122.0 +19,sr_remoteBytesRead_mean,0.006599492,8621230.876874011,20763598.714249212,0.0,14815.464675201518,1045794.9007633588,7696222.868940513,207400140.1710202 +20,sr_fetchWaitTime_mean,0.005970912,282.57444030933357,2063.740776556858,0.0,0.0,0.014450867052023121,0.0744336569579288,26216.59561667383 +21,input_recordsRead_sum,0.005626504,7537500441.369308,19524836240.42767,30000.0,410551846.0,1658123725.0,5999989709.0,230400923204.0 +22,sr_localBytesReadRatio,0.0047795097,0.13775142996534326,0.21834409881656594,0.0,0.0003435677540369934,0.052422868005860185,0.18397368378307108,1.6954497523177325 +23,executorDeserializeTime_mean,0.0046688416,77.619961219705,78.66729885547291,1.2399579021224347,16.74,57.15866922584773,111.59245283018868,438.3862068965517 +24,sr_localBlocksFetched_sum,0.00451675,34560.099494097805,81564.66670257768,0.0,329.0,5582.0,30418.0,870491.0 +25,sr_remoteBytesReadRatio,0.0034871036,0.6856243468838018,1.1473671005608985,0.0,0.0015316802906526895,0.13301082808430215,1.0519672879672015,11.86796722526039 +26,shuffle_read_bw,0.0034601516,276816955.4836904,847565126.6737152,0.0,0.0,4930489.4375,137218859.4920635,10530883927.666666 +27,sqlOp_Expand,0.0031522522,0.11129848229342328,0.3147669133570336,0.0,0.0,0.0,0.0,1.0 +28,executorRunTime_mean,0.0027702448,4297.896908257712,8062.7718056424565,226.90115532734274,1490.5288200108755,2236.8281325012003,3741.984555984556,72194.51503203549 +29,sw_bytesWrittenRatio,0.0026040608,0.8014652434613514,1.2780756184305504,0.0,0.0023701803469738034,0.22863716255153424,1.206131247096677,13.563416977578123 +30,data_size,0.0019577835,650411028780.6053,2082773012482.3335,0.0,21721183788.0,89736852644.0,333792838797.0,21374406089219.0 +31,sr_totalBytesRead_mean,0.0018648852,10172040.134748595,23817088.532298744,0.0,19575.48201438849,1695974.662708024,9072792.773504274,237029779.838344 +32,sqlOp_SubqueryBroadcast,0.0018241289,0.7976391231028668,0.40209900029018336,0.0,1.0,1.0,1.0,1.0 +33,diskBytesSpilledRatio,0.0013953688,0.24474497918684088,0.8105072165034726,0.0,0.0,0.0,0.0,6.729163387759685 +34,sr_remoteBlocksFetched_sum,0.0013747218,224234.64755480608,509217.62116016017,0.0,2164.0,37041.0,193536.0,3942472.0 +35,memoryBytesSpilled_mean,0.0013567632,18194132.33421661,71075266.1814507,0.0,0.0,0.0,0.0,724082083.4348422 +36,sr_totalBytesReadRatio,0.001344917,0.823375776849145,1.319042835980761,0.0,0.002400770847119191,0.22866221128988315,1.2670990125430908,13.563416977578123 +37,sqlOp_AQEShuffleRead,0.0013331274,0.7065767284991569,0.45571511734167724,0.0,0.0,1.0,1.0,1.0 +38,sqlOp_Subquery,0.0009835393,0.09106239460370995,0.28794070420987344,0.0,0.0,0.0,0.0,1.0 +39,diskBytesSpilled_mean,0.0009471796,4546045.185000375,15069027.640347425,0.0,0.0,0.0,0.0,111631484.38187319 +40,sqlOp_Scan orc ,0.0006244791,0.45868465430016864,0.498710794080452,0.0,0.0,0.0,1.0,1.0 +41,sw_bytesWritten_mean,0.00060904864,9961926.690653043,23574771.301399454,0.0,19575.48201438849,1687125.7618110236,9963399.731213873,237029779.838344 +42,sqlOp_SortMergeJoin,0.00042181378,0.41652613827993257,0.49339906934430416,0.0,0.0,0.0,1.0,1.0 +43,sqlOp_Sort,0.00041253344,0.6053962900505903,0.4891780403788745,0.0,0.0,1.0,1.0,1.0 +44,sqlOp_Scan parquet ,0.00033403642,0.5311973018549747,0.49944707588925713,0.0,0.0,1.0,1.0,1.0 +45,sqlOp_BroadcastExchange,9.627697e-05,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 +46,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Window,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_BroadcastNestedLoopJoin,0.0,0.04047217537942664,0.1972302630779745,0.0,0.0,0.0,0.0,1.0 +75,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +76,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,maxOnHeapMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 +79,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,executorMemory,0.0,40960.0,0.0,40960.0,40960.0,40960.0,40960.0,40960.0 +81,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +84,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,numExecutors,0.0,8.062394603709949,0.3504133651237026,8.0,8.0,8.0,8.0,14.0 +87,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +90,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_Project,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +94,sqlOp_HashAggregate,0.0,0.9797639123102867,0.14092579360637603,0.0,1.0,1.0,1.0,1.0 +95,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_Filter,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +97,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Exchange,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +100,sqlOp_ColumnarToRow,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +101,sqlOp_BroadcastHashJoin,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 +102,maxMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 +103,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,platform_dataproc,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +105,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json index f99655120..0f46af9d6 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan csv ","sqlOp_Scan orc ","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Window","sqlOp_WindowGroupLimit","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_emr","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan json ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"61"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.78048E-2,-1.8552697E-1,4.901243E-1,-3.3088633E-1,3.0965138E-2,2.4628823E-1,8.852223E-1,-4.659008E-1,-1.3334489E-1,-1.5243791E-1,1.4812984E-1,2.971024E-1,-1.2563897E-2,5.130134E-2,6.705256E-1,-1.8754591E-1,-5.3297985E-1,9.115728E-2,-2.2618434E-1,-2.66628E-1,1.2518686E-2,2.2718608E-1,4.980866E-2,4.346211E-1,1.5858072E-1,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,-6.1508644E-1,-3.1681323E-1,7.0068543E-3,-7.234637E-4,-2.9873994E-1,1.068607E-2,-7.976921E-3,-2.4829587E-2,-7.415798E-2,4.8681386E-3,1.2035196E-2,4.350847E-2,-3.566741E-2,9.323246E-3,2.2088567E-2,1.0012548E-2,2.3240472E-1,2.6123306E-2,-1.783583E-2,-6.754464E-1,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4057398E1,5.265595E0,7.200094E0,2.60497E0,1.4837892E0,6.482475E-1,1.0196114E0,1.0016766E0,8.960098E-1,5.119209E-1,3.1514597E-1,7.120998E-1,2.5112772E-1,0E0,1.8040752E-1,5.0723886E-1,6.393223E-1,9.9871E-2,5.187807E-1,3.0155098E-1,1.1128237E-1,1.9333434E-1,2.8770238E-1,1.774602E-1,2.0905662E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3500404E-1,1.0710728E-1,0E0,0E0,1.3412046E-1,3.246751E-1,0E0,0E0,1.1355559E-1,0E0,0E0,9.497507E-2,1.2899351E-1,0E0,0E0,0E0,9.763676E-2,1.0520158E-1,0E0,3.8818073E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,35,35,36,36,39,39,42,42,43,43,47,47,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,2.7558826E-2,2.0224316E-1,1.9415865E2,2.4118989E8,2.05E0,7.1239825E5,1.433458E10,8.938651E4,7.919613E5,3.2744168E6,7.3E1,6.035389E6,5.130134E-2,2.5E1,2.7287993E0,1.021924E6,1.1936242E7,4.145324E6,2.7575057E0,6.406965E-1,1.2183861E7,2.1469998E5,6.75602E5,2.8994352E7,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,3.12986E10,1.858E3,7.0068543E-3,-7.234637E-4,3.1100525E5,4.7E3,-7.976921E-3,-2.4829587E-2,4.565566E-1,4.8681386E-3,1.2035196E-2,6.7E2,1.526402E6,9.323246E-3,2.2088567E-2,1.0012548E-2,1E0,1.479938E6,-1.783583E-2,7.359468E8,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"split_indices":[67,53,53,67,7,68,48,46,48,47,65,8,1,0,3,69,44,60,1,73,54,47,43,9,1,0,0,0,0,0,0,46,0,0,0,48,0,0,0,54,0,0,0,9,0,0,0,104,47,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,1.66E2,7.6E1,9.9E1,6.7E1,4.8E1,2.8E1,5.8E1,4.1E1,2.6E1,4.1E1,4E1,8E0,1E1,1.8E1,1.2E1,4.6E1,1.2E1,2.9E1,1.5E1,1.1E1,2.2E1,1.9E1,1.9E1,2.1E1,4E0,4E0,1.6E1,2E0,2E0,1E1,3.2E1,1.4E1,7E0,5E0,2.2E1,7E0,1.3E1,2E0,6E0,5E0,1.7E1,5E0,1.3E1,6E0,1.3E1,6E0,1.3E1,8E0,9E0,2.3E1,2E0,1.2E1,6E0,1.6E1,3E0,4E0,3E0,3E0,2E0,3E0,7E0,6E0,7E0,6E0,4E0,4E0,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.0040125E-2,-2.1713006E-1,4.0957847E-1,-3.5627815E-1,9.978797E-3,2.0629005E-1,8.082211E-1,-4.7081342E-1,-1.7093469E-1,-5.8718614E-2,1.0682804E-2,6.47079E-2,2.835063E-1,2.307691E-2,4.1448183E-2,9.978426E-4,-4.971977E-1,1.4917599E-3,-3.1879982E-1,-1.7213058E-1,8.313783E-2,1.541623E-1,-7.4060783E-3,3.6715305E-1,1.0359522E-2,-1.8699963E-1,-5.3688544E-1,-9.420938E-2,1.2580958E-2,-1.8095352E-2,-1.4491154E-1,-2.1190276E-2,-1.1271654E-1,-1.211096E-2,2.5133708E-1,9.38944E-3,-7.0502064E-3,1.9026043E-5,3.9769462E-1,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,-3.106009E-1,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,-1.4537086E-1,7.170774E-3,-6.025724E-2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,4.859038E-1,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,5.357659E-1,3.981885E-3,7.6941703E-3,2.5353398E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,-1,-1,55,-1,-1,-1,-1,-1,57,-1,-1,-1,-1,59,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[2.0510149E1,4.996633E0,6.1543484E0,1.9999018E0,9.984025E-1,5.610745E-1,5.2623177E-1,7.9289055E-1,9.9481404E-1,7.8654516E-1,0E0,4.4556248E-1,7.730229E-1,0E0,0E0,0E0,6.0133266E-1,5.4866415E-1,2.8996253E-1,4.4809937E-1,3.5550103E-1,3.2364437E-1,0E0,2.9102945E-1,1.4653622E-1,9.1507256E-2,6.157198E-1,1.915977E-1,0E0,0E0,1.275117E-1,0E0,2.4100256E-1,1.6567487E-1,2.9098767E-1,0E0,0E0,0E0,3.2995868E-1,0E0,0E0,0E0,0E0,0E0,1.7156088E-1,0E0,0E0,0E0,0E0,1.24156505E-1,0E0,9.488517E-2,0E0,0E0,0E0,0E0,2.6596856E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2510395E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,30,30,32,32,33,33,34,34,38,38,44,44,49,49,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,-1,-1,56,-1,-1,-1,-1,-1,58,-1,-1,-1,-1,60,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.5665247E2,1.7613442E-1,2.0224316E-1,1.9415865E2,9.749826E6,4.633287E2,3.856E3,3.8757736E-7,1.5248811E1,1.5225257E-1,1.0682804E-2,3.6217573E0,2.1030048E10,2.307691E-2,4.1448183E-2,9.978426E-4,1.1266387E10,3.165913E6,3.03E9,1.386452E0,1.7573398E2,1.9647126E7,-7.4060783E-3,2.34E3,2.51838E6,2.85E2,1.021924E6,5.63651E5,1.2580958E-2,-1.8095352E-2,2.3407683E0,-2.1190276E-2,2.7538432E8,2.424E3,1.2730925E0,9.38944E-3,-7.0502064E-3,1.9026043E-5,5.792986E4,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,1.892E3,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,2.5187686E-1,7.170774E-3,3.56E2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,1.1107071E7,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,3.302E3,3.981885E-3,7.6941703E-3,2.5353398E-2],"split_indices":[67,56,53,67,9,67,2,53,71,53,0,69,5,0,0,0,46,1,5,68,67,60,0,2,47,0,44,9,0,0,69,0,7,2,57,0,0,0,43,0,0,0,0,0,2,0,0,0,0,57,0,10,0,0,0,0,9,0,0,0,0,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,1.57E2,7.7E1,9.7E1,6E1,5.2E1,2.5E1,5.9E1,3.8E1,4.7E1,1.3E1,1.9E1,3.3E1,1E1,1.5E1,3E0,5.6E1,1.8E1,2E1,2.6E1,2.1E1,1.4E1,5E0,2.5E1,8E0,7E0,4.9E1,1.4E1,4E0,1.2E1,8E0,3E0,2.3E1,1.4E1,7E0,1.2E1,2E0,2E0,2.3E1,5E0,3E0,5E0,2E0,3.6E1,1.3E1,7E0,7E0,6E0,2E0,2.1E1,2E0,1.2E1,2E0,2E0,5E0,8E0,1.5E1,6E0,7E0,8E0,1.3E1,7E0,5E0,1.3E1,2E0,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.5844442E-2,-2.1939752E-1,3.5634845E-1,-3.067927E-1,7.70621E-2,1.8756232E-1,7.0530635E-1,-3.6990502E-1,-1.6308445E-2,3.355599E-2,1.0062139E-2,-4.2225484E-2,2.3436439E-1,1.2260787E-2,7.853399E-1,-4.3145156E-1,-2.022222E-1,1.7129338E-1,-1.4914486E-1,-7.0826644E-3,8.609564E-3,7.4184006E-3,-1.4439643E-2,2.7769628E-1,6.108423E-3,1.9752327E-2,3.9388932E-2,-3.5376546E-1,-6.491721E-1,-1.04918115E-1,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,4.5147832E-2,-5.3988285E-3,2.0416094E-1,4.445212E-1,-5.368166E-3,3.978625E-3,-5.482469E-1,-2.6226673E-1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,-3.322308E-2,-8.5412245E-4,7.761204E-3,2.3548925E-1,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,-2.9240888E-1,-3.515024E-3,6.6058734E-3,2.612413E-1,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,49,51,-1,-1,53,55,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,-1,-1,61,-1,-1,63,-1,-1,-1,-1,-1],"loss_changes":[1.863963E1,4.1431956E0,4.991946E0,2.2489424E0,2.3938012E-1,6.55169E-1,7.3028564E-1,9.5510197E-1,5.9871566E-1,2.0101649E-1,0E0,7.301139E-1,4.9284196E-1,0E0,4.888401E-1,1.0050564E0,4.5257246E-1,2.7720916E-1,2.4958798E-1,1.5788087E-1,0E0,0E0,0E0,4.2448258E-1,1.1006676E-1,0E0,0E0,8.527217E-1,3.161502E-1,1.413299E-1,0E0,0E0,0E0,0E0,0E0,1.5807515E-1,0E0,3.994676E-1,2.7063847E-1,0E0,0E0,1.5897751E-1,3.2701302E-1,0E0,0E0,0E0,1.2576254E-1,0E0,0E0,1.6656256E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.2975593E-1,0E0,0E0,1.3126945E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,27,27,28,28,29,29,35,35,37,37,38,38,41,41,42,42,46,46,49,49,56,56,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,50,52,-1,-1,54,56,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,-1,-1,62,-1,-1,64,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,6.005889E-1,2.0224316E-1,2.727992E6,9.750871E6,7.12E2,2.744E3,1.1670399E-1,1.6E1,1E0,1.0062139E-2,3.9378E5,2.8812434E7,1.2260787E-2,2.2324745E5,3.98231E-2,3.4047124E-1,6.0578914E4,2.6017979E11,4.33E2,8.609564E-3,7.4184006E-3,-1.4439643E-2,7.2259143E2,1.04E2,1.9752327E-2,3.9388932E-2,3.3617156E-4,1.868632E7,1.86204E5,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,7.948E4,-5.3988285E-3,5.130435E0,9.521E3,-5.368166E-3,3.978625E-3,5.8013948E7,9.3E1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,2.3544362E6,-8.5412245E-4,7.761204E-3,2.1061536E3,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,1.65E9,-3.515024E-3,6.6058734E-3,7.83495E5,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"split_indices":[67,54,53,1,9,0,2,54,3,103,0,1,60,0,43,57,57,47,46,0,0,0,0,67,8,0,0,54,60,9,0,0,0,0,0,1,0,69,2,0,0,7,11,0,0,0,66,0,0,4,0,0,0,0,0,0,5,0,0,48,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.58E2,8.6E1,1.22E2,3.6E1,5.9E1,2.7E1,1E2,2.2E1,2.9E1,7E0,1E1,4.9E1,5E0,2.2E1,7.2E1,2.8E1,9E0,1.3E1,2.4E1,5E0,6E0,4E0,4.1E1,8E0,7E0,1.5E1,5.5E1,1.7E1,1.9E1,9E0,2E0,7E0,2E0,1.1E1,1.7E1,7E0,3E1,1.1E1,3E0,5E0,1.6E1,3.9E1,1.5E1,2E0,7E0,1.2E1,1.2E1,5E0,2.8E1,2E0,6E0,5E0,1.3E1,3E0,4E0,3.5E1,1E1,2E0,2.5E1,3E0,2.5E1,1E1,1.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[1.0417009E-2,-1.92214E-1,4.0404752E-1,-3.015024E-1,-7.670976E-5,1.7257118E-1,7.666108E-1,-3.8266033E-1,-1.3505321E-2,-5.040442E-2,1.1856716E-2,2.0422818E-1,-5.236619E-3,4.0671965E-1,9.444278E-1,-4.4783047E-1,-2.4137214E-1,-8.257532E-2,9.555998E-3,-1.7077446E-2,-2.2676414E-2,-1.2199888E-3,2.4267583E-1,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,-1.7631327E-1,-5.2404433E-1,-4.6340473E-2,-1.6422002E-2,8.984605E-3,-1.4559102E-1,-8.014499E-2,7.040712E-2,3.7856903E-2,1.1994875E-2,8.994386E-3,-1.1225067E-2,-5.8429706E-1,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,-1.4150277E-1,4.9856484E-2,1.1765836E-2,1.939237E-2,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,-1,35,-1,-1,-1,-1,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0656195E1,3.5906997E0,7.253089E0,2.5425148E0,8.754045E-1,5.339109E-1,1.7917042E0,6.7456627E-1,4.1952434E-1,5.01606E-1,0E0,4.531319E-1,0E0,9.731603E-2,3.5835075E-1,1.0871277E0,7.457906E-1,3.9352587E-1,0E0,0E0,2.7846104E-1,0E0,2.731049E-1,0E0,0E0,0E0,0E0,4.8621753E-1,5.456524E-1,1.2857142E-1,0E0,0E0,3.2480013E-1,2.623061E-1,1.9802536E-1,1.5393375E-1,0E0,0E0,0E0,2.466135E-1,0E0,0E0,0E0,0E0,0E0,1.9748119E-1,1.3966252E-1,0E0,1.250949E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,20,20,22,22,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,45,45,46,46,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,-1,36,-1,-1,-1,-1,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0845802E6,1.7613442E-1,1.5972575E0,2.727992E6,2.03693E11,1.25440235E1,7.2610445E8,1.9415865E2,1.22E2,2.9927E4,1.1856716E-2,4.0123675E-2,-5.236619E-3,1.3589149E6,2.2324745E5,2.431768E2,1.2247422E1,2.6017979E11,9.555998E-3,-1.7077446E-2,1.1343225E6,-1.2199888E-3,3.4021906E2,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,1.0395554E5,6.6037946E2,6.0515753E2,-1.6422002E-2,8.984605E-3,2.3407683E0,9.74979E6,1.2355705E6,1.19881555E-1,1.1994875E-2,8.994386E-3,-1.1225067E-2,1.51E3,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,4.4E1,5.3164E4,1.1765836E-2,6.0039372E0,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"split_indices":[66,56,56,1,46,72,12,67,8,9,0,53,0,62,43,70,71,46,0,0,48,0,67,0,0,0,0,43,4,4,0,0,69,9,48,53,0,0,0,2,0,0,0,0,0,10,44,0,69,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.7E2,8.7E1,1.08E2,6.2E1,5.4E1,3.3E1,8.4E1,2.4E1,5.3E1,9E0,4.9E1,5E0,1.2E1,2.1E1,5.6E1,2.8E1,1.9E1,5E0,3E0,5E1,7E0,4.2E1,1E1,2E0,4E0,1.7E1,1.3E1,4.3E1,1.2E1,1.6E1,3E0,1.6E1,3.1E1,1.9E1,6E0,3.6E1,2E0,1.1E1,3.4E1,9E0,3E0,9E0,1.4E1,2E0,2.1E1,1E1,3E0,1.6E1,2E0,4E0,2.2E1,1.2E1,3E0,1.8E1,4E0,6E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-2.2585599E-2,-1.9692676E-1,3.6539125E-1,-2.9716375E-1,-2.7832244E-2,1.6051705E-1,7.1087146E-1,-3.7421748E-1,1.4051265E-2,-6.813547E-2,1.1468622E-2,-8.938615E-2,2.114588E-1,4.520597E-1,9.747266E-1,-4.3956405E-1,-2.1756326E-1,-5.356781E-2,1.1113507E-2,-1.736248E-1,1.3845394E-2,4.104848E-3,-6.774424E-3,2.3279007E-1,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,-2.5017148E-1,-4.929268E-1,1.6659833E-3,-1.6076026E-2,9.1863185E-2,-6.740727E-3,-7.370915E-2,-3.767319E-1,-5.412376E-3,5.573497E-2,9.279637E-2,2.8638762E-1,-2.9118592E-1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,3.14243E-1,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,-1,47,-1,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1,-1],"loss_changes":[1.6576199E1,2.8561993E0,5.265172E0,2.5471687E0,7.675839E-1,6.4269626E-1,1.4714527E0,7.744961E-1,3.715908E-1,4.9534598E-1,0E0,1.2332737E-1,2.3372102E-1,1.488092E-1,2.703581E-1,4.729681E-1,8.93371E-1,2.783168E-1,0E0,4.7957075E-1,1.9750053E-1,0E0,0E0,2.585504E-1,0E0,0E0,0E0,0E0,0E0,1.6383308E-1,2.5936127E-1,1.265059E-1,0E0,2.4251655E-1,0E0,1.5377875E-1,1.1529529E-1,0E0,2.033091E-1,1.5228397E-1,2.5932693E-1,1.0870254E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3727503E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,29,29,30,30,31,31,33,33,35,35,36,36,38,38,39,39,40,40,41,41,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,-1,48,-1,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1,-1],"split_conditions":[4.083633E6,1.7613442E-1,1.5972575E0,2.246309E6,2.03693E11,3.359399E-1,1.6525185E9,1.9415865E2,1.22E2,7.251646E-2,1.1468622E-2,2.2183135E-1,1.25440235E1,1.6790323E8,4.6E1,2.4747229E2,1.2288136E1,4.7E3,1.1113507E-2,3.4047124E-1,2.0358758E8,4.104848E-3,-6.774424E-3,1.0705044E3,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,1.499E3,1.021924E6,2.797224E2,-1.6076026E-2,1.445113E6,-6.740727E-3,9.593298E6,6.9268906E5,-5.412376E-3,2.7352352E6,1.8E1,1.1107071E7,2.923913E1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,6.464779E6,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"split_indices":[66,56,56,1,46,57,5,67,8,53,0,54,72,7,8,70,71,0,0,57,7,0,0,70,0,0,0,0,0,2,44,67,0,9,0,60,48,0,63,3,9,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,44,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.68E2,7.5E1,1.05E2,6.3E1,4.8E1,2.7E1,8.4E1,2.1E1,5.6E1,7E0,8E0,4E1,1.5E1,1.2E1,5.8E1,2.6E1,1.7E1,4E0,2.4E1,3.2E1,2E0,6E0,3.7E1,3E0,2E0,1.3E1,2E0,1E1,1.4E1,4.4E1,1.1E1,1.5E1,7E0,1E1,1.7E1,7E0,7E0,2.5E1,1.1E1,2.6E1,1.2E1,2E0,3.8E1,6E0,7E0,4E0,4E0,3E0,1.3E1,4E0,4E0,3E0,1.4E1,1.1E1,8E0,3E0,2.4E1,2E0,3E0,9E0,1.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-8.34782E-3,-1.667577E-1,4.3978554E-1,-3.0709088E-1,2.6771413E-2,1.9930072E-1,7.418625E-1,-4.0431347E-1,-1.2503225E-1,-6.46251E-2,1.2227376E-1,2.7885714E-1,3.0976776E-2,3.739352E-1,8.703366E-1,-4.299911E-1,4.975969E-3,1.0842858E-1,-2.3695372E-1,-1.4191392E-1,1.8143274E-1,1.5553638E-1,-3.5369173E-3,2.103493E-2,1.11278795E-1,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,-4.528867E-1,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,-2.1131526E-1,5.121328E-2,8.2729507E-4,1.0838011E-2,7.531465E-2,2.1820368E-1,6.924229E-3,-5.724572E-3,-4.900431E-1,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,-4.1408205E-1,-6.208327E-1,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,45,-1,-1,-1,-1,47,49,-1,-1,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.7394594E1,4.9239187E0,4.501422E0,1.807971E0,6.800027E-1,4.868381E-1,9.980869E-1,9.540901E-1,1.0186052E0,7.864486E-1,2.696029E-1,7.975136E-1,3.8219005E-1,1.8597853E-1,5.8754253E-1,3.8879967E-1,0E0,1.0245718E-1,1.4958644E-1,4.2632413E-1,1.0402632E-1,1.4946616E-1,0E0,0E0,1.9317481E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9329586E-1,0E0,0E0,0E0,0E0,2.1796727E-1,1.8252599E-1,0E0,0E0,1.0288309E-1,1.249153E-1,0E0,0E0,2.2973442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3063469E-1,2.7027082E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,24,24,32,32,37,37,38,38,41,41,42,42,45,45,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,46,-1,-1,-1,-1,48,50,-1,-1,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.4210773E-2,2.0224316E-1,1.773747E2,6.75553E5,4.3147E5,2.174459E5,4.108148E6,9.165388E4,1.5225257E-1,2.4219654E0,3.5630266E8,1.4E1,5.76E3,1.6507233E3,6E0,4.975969E-3,1.1936242E7,2.1055435E1,2.5055168E9,6.765595E7,2.151E3,-3.5369173E-3,2.103493E-2,8.96347E7,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,1.437883E6,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,2.5335E4,4.933E4,8.2729507E-4,1.0838011E-2,5.546848E1,9.173913E5,6.924229E-3,-5.724572E-3,1.499E3,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,1.53091E5,2.373E3,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"split_indices":[67,53,53,67,9,9,43,9,48,53,68,7,3,2,4,10,0,60,71,5,5,2,0,0,59,0,0,0,0,0,0,0,1,0,0,0,0,9,9,0,0,67,62,0,0,2,0,0,0,0,0,0,0,0,0,9,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.8E2,6.3E1,1.04E2,7.6E1,3.6E1,2.7E1,6.7E1,3.7E1,3.9E1,3.7E1,2.4E1,1.2E1,8E0,1.9E1,6.4E1,3E0,1.2E1,2.5E1,3E1,9E0,3.2E1,5E0,1E1,1.4E1,4E0,8E0,3E0,5E0,1E1,9E0,5E0,5.9E1,7E0,5E0,1.4E1,1.1E1,2.2E1,8E0,3E0,6E0,1.5E1,1.7E1,1.2E1,2E0,4.8E1,1.1E1,2E0,2E1,4E0,4E0,2E0,1.3E1,1.5E1,2E0,3.3E1,1.5E1,2.4E1,9E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.9497597E-3,-1.8956819E-1,3.4107372E-1,-3.159441E-1,-1.5801242E-2,1.5268305E-1,6.5795535E-1,-3.6932546E-1,7.846604E-3,-5.214277E-2,1.0556764E-1,-1.2807088E-2,2.3161104E-1,4.560374E-1,8.566107E-1,-2.9471084E-1,-5.894394E-1,1.2438455E-1,-7.3430347E-3,-1.3571183E-1,1.1542113E-2,7.1960697E-3,-4.9598026E-3,1.3622053E-1,-1.1057054E-2,6.492877E-2,3.2902324E-1,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,-2.0076081E-1,-6.4882565E-1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,9.8576896E-2,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,2.6239768E-1,2.3697935E-2,-2.679108E-1,-6.2030084E-2,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,4.4976994E-3,-3.273256E-1,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6410194E1,3.4866414E0,5.391714E0,1.5930605E0,3.054343E-1,7.736126E-1,9.7316456E-1,1.1237602E0,3.082666E-1,2.8363693E-1,2.2685544E-1,7.515435E-1,6.2811255E-1,2.3523116E-1,7.9534054E-1,9.5042276E-1,3.015952E-1,2.1883255E-1,0E0,9.567392E-2,1.899715E-1,0E0,0E0,2.035301E-1,0E0,1.436969E-1,2.1399069E-1,0E0,0E0,0E0,0E0,0E0,3.7464368E-1,1.0184717E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1995159E-1,0E0,0E0,0E0,0E0,1.2598455E-1,0E0,4.546547E-1,1.02785856E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.053444E-1,1.2376809E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,26,26,32,32,33,33,40,40,45,45,47,47,48,48,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.1306552E-1,1.8829846E-1,3.402546E6,2.424E3,3.165913E6,1.4968063E3,5.1653096E-3,3.771E3,5.670282E-1,3.2410275E2,1.1936242E7,6.2143734E4,2.0270285E8,2.65826E6,3.3617156E-4,1.8018788E7,3.995065E6,-7.3430347E-3,2.8507206E-1,2.6801108E2,7.1960697E-3,-4.9598026E-3,1.9816086E0,-1.1057054E-2,2.51838E6,8.0267444E2,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,8.456733E10,7.5E1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,7E0,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,8.0774775E6,2.3697935E-2,2.3498028E6,3.948533E5,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,7.1E1,6E0,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"split_indices":[67,54,53,1,2,1,4,53,2,57,67,60,43,7,62,54,60,44,0,57,4,0,0,68,0,47,67,0,0,0,0,0,46,8,0,0,0,0,0,0,8,0,0,0,0,47,0,60,43,0,0,0,0,0,0,0,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.58E2,9.1E1,9.1E1,6.7E1,5.8E1,3.3E1,7.8E1,1.3E1,5.2E1,1.5E1,1.9E1,3.9E1,1.8E1,1.5E1,6E1,1.8E1,8E0,5E0,2.2E1,3E1,1.2E1,3E0,1.2E1,7E0,1.5E1,2.4E1,3E0,1.5E1,9E0,6E0,1.9E1,4.1E1,1.5E1,3E0,3E0,5E0,1.4E1,8E0,1.7E1,1.3E1,1E1,2E0,6E0,9E0,2E1,4E0,2.7E1,1.4E1,1.2E1,3E0,9E0,4E0,1.5E1,5E0,5E0,2.2E1,1E1,4E0,3E0,2E0,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.0813884E-3,-2.0281385E-1,3.2203957E-1,-2.7021798E-1,6.471507E-2,1.3456307E-1,6.046468E-1,-3.230015E-1,-2.4385922E-2,-4.1049863E-3,1.0497509E-1,-1.7012404E-1,1.8569289E-1,7.2859275E-1,3.886841E-1,-2.9304287E-1,-5.693168E-1,7.866887E-3,-7.695989E-2,1.368179E-1,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.1308988E-1,1.2391388E-1,4.675196E-2,5.738674E-1,9.75509E-3,2.0310232E-2,-3.7767187E-1,-1.9021347E-1,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,9.295686E-2,2.2762196E-2,-4.9231676E-3,1.4885345E-1,8.6237835E-3,2.6556352E-2,-4.5262882E-1,-2.0554197E-1,-2.254751E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.7958388E-2,1.8340372E-1,-3.690774E-1,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,-2.566865E-1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,-2.7930382E-1,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,-1,-1,51,53,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[1.6330715E1,2.812533E0,5.0049305E0,1.6011333E0,2.1443705E-1,9.507282E-1,7.339792E-1,5.858507E-1,2.613468E-1,0E0,1.8759432E-1,2.9529205E-1,3.6259735E-1,6.0491276E-1,9.870744E-2,7.3748016E-1,1.4939356E-1,0E0,3.703289E-1,1.539154E-1,0E0,0E0,0E0,6.859244E-1,2.2933495E-1,0E0,1.1461353E-1,0E0,0E0,5.5208206E-1,3.4554946E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2862067E-1,0E0,0E0,1.4594752E-1,0E0,0E0,4.9916983E-1,1.4001942E-1,4.1745436E-1,0E0,0E0,0E0,1.3290837E-1,1.4290059E-1,2.839706E-1,0E0,0E0,0E0,0E0,1.334548E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1397505E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,23,23,24,24,26,26,29,29,30,30,37,37,40,40,43,43,44,44,45,45,49,49,50,50,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,-1,-1,52,54,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.1259552E2,4.9615598E-1,1.2711891E0,2.727992E6,1.8681856E8,7.5E2,9.8110194E5,1E0,2.6017979E11,-4.1049863E-3,1E0,3.9378E5,6.5E1,2.7479675E-1,2.0787016E5,1.31654E5,3.7274265E2,7.866887E-3,3.2829787E12,3.1277522E6,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.302E3,3.0327642E-1,4.675196E-2,3.301E3,9.75509E-3,2.0310232E-2,1.3738942E0,2.0766662E6,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,1.4958695E5,2.2762196E-2,-4.9231676E-3,1.7377981E9,8.6237835E-3,2.6556352E-2,4.969844E3,7.07816E10,4.5595855E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.5E1,4.089454E4,1.9415865E2,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,2E1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,2.5605E4,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"split_indices":[67,54,54,1,7,0,48,85,46,0,85,1,8,53,43,9,4,0,46,66,0,0,0,2,57,0,2,0,0,68,63,0,0,0,0,0,0,43,0,0,12,0,0,62,46,71,0,0,0,3,43,67,0,0,0,0,3,0,0,0,0,0,0,44,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.54E2,9.5E1,1.23E2,3.1E1,5.8E1,3.7E1,1.01E2,2.2E1,6E0,2.5E1,8E0,5E1,2.2E1,1.5E1,9.2E1,9E0,4E0,1.8E1,2.2E1,3E0,3E0,5E0,1.5E1,3.5E1,5E0,1.7E1,6E0,9E0,4.9E1,4.3E1,2E0,7E0,1.3E1,5E0,1.3E1,9E0,8E0,7E0,3E0,3.2E1,2E0,1.5E1,3.3E1,1.6E1,3.7E1,6E0,2E0,6E0,7E0,2.5E1,2.7E1,6E0,1E1,6E0,3E0,3.4E1,4E0,3E0,4E0,2.1E1,2.5E1,2E0,3E1,4E0,2E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-6.4136046E-3,-1.6499257E-1,2.6240352E-1,-2.5779408E-1,-1.8667922E-4,1.3376746E-1,5.1147556E-1,-3.5323516E-1,-8.524396E-2,-9.4428E-2,1.0056048E-1,-9.662582E-2,1.73225E-1,3.220377E-2,3.9076596E-1,-1.4722008E-1,-4.0586957E-1,-1.8771532E-1,1.03829466E-1,-1.9139503E-1,7.505271E-2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,2.0356877E-1,-2.7897414E-3,4.3953964E-1,5.554044E-3,-9.59708E-3,4.6992786E-3,-4.2843238E-1,-4.7170566E-3,-1.2011155E-3,-2.7610272E-1,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,9.2738315E-2,2.768057E-1,7.707731E-3,2.085054E-2,-3.687053E-1,-5.6087404E-1,-1.6907474E-2,-7.4428786E-3,9.665818E-3,-4.444956E-2,3.618798E-1,4.56879E-3,-3.8534006E-1,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,4.9356914E-1,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,41,-1,43,-1,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,-1,57,59,-1,61,-1,-1,-1,-1,-1,-1,63,-1,-1,-1,-1],"loss_changes":[1.0913624E1,2.4623237E0,2.975348E0,1.6661301E0,5.696739E-1,5.985464E-1,5.864744E-1,6.559181E-1,7.557257E-1,5.238199E-1,2.0130792E-1,5.320686E-1,4.088E-1,0E0,2.3964357E-1,3.0165756E-1,3.0306053E-1,3.463729E-1,1.8510923E-1,2.6532936E-1,1.640884E-1,0E0,0E0,0E0,0E0,3.7205648E-1,0E0,1.0671902E-1,0E0,0E0,0E0,1.7289925E-1,0E0,0E0,1.2194073E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.8078758E-1,3.8673472E-1,0E0,0E0,1.6442966E-1,1.01364136E-1,0E0,0E0,0E0,2.1470177E-1,1.7472506E-1,0E0,8.7862015E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.314799E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,27,27,31,31,34,34,41,41,42,42,45,45,46,46,50,50,51,51,53,53,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,42,-1,44,-1,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,-1,58,60,-1,62,-1,-1,-1,-1,-1,-1,64,-1,-1,-1,-1],"split_conditions":[3.3103763E2,2.5106274E-2,1.8829846E-1,1.9415865E2,9.50379E5,7.5E2,7.5921344E5,2.431768E2,5.99999E9,1.2185057E0,1.3E1,3.13E2,1.25440235E1,3.220377E-2,2.56701E5,3.160848E6,7.4E1,3.0364912E5,4.654E3,2.587E3,1.7468556E2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,4.871647E2,-2.7897414E-3,1.5789307E8,5.554044E-3,-9.59708E-3,4.6992786E-3,1.912E3,-4.7170566E-3,-1.2011155E-3,5.316779E5,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,1.1572553E7,1.8751362E7,7.707731E-3,2.085054E-2,7.872337E5,1.2967741E0,-1.6907474E-2,-7.4428786E-3,9.665818E-3,1.1E1,5.5053414E9,4.56879E-3,1.0169992E0,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,1.6851284E7,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"split_indices":[67,53,53,67,9,0,48,70,5,56,3,0,72,0,44,9,3,48,2,2,67,0,0,0,0,67,0,5,0,0,0,2,0,0,48,0,0,0,0,0,0,60,59,0,0,66,68,0,0,0,3,5,0,68,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.54E2,1.6E2,9.4E1,1.02E2,5.8E1,6.3E1,3.1E1,6.5E1,3.7E1,3E1,2.8E1,9E0,5.4E1,9E0,2.2E1,1.4E1,5.1E1,2.4E1,1.3E1,1.9E1,1.1E1,1.9E1,9E0,4E0,5E0,4.8E1,6E0,1.8E1,4E0,1.1E1,3E0,4.7E1,4E0,9E0,1.5E1,1E1,3E0,1.7E1,2E0,7E0,4E0,2E1,2.8E1,3E0,1.5E1,3.5E1,1.2E1,6E0,9E0,1E1,1E1,1.8E1,1E1,3.3E1,2E0,4E0,8E0,4E0,6E0,1.1E1,7E0,2E0,3.1E1,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.2190764E-2,-1.8730694E-1,2.753607E-1,-2.6419362E-1,5.672748E-2,1.16373606E-1,5.309576E-1,-3.1258324E-1,-1.8713716E-2,-1.0737888E-1,1.2148461E-1,-1.23983465E-1,1.630511E-1,4.4448983E-2,4.534144E-1,-2.807187E-1,-2.4374343E-2,1.0378545E-1,-1.09028615E-1,-7.691218E-3,4.587331E-4,1.4246969E-1,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,2.1820955E-1,-6.480758E-3,5.3746756E-3,4.961946E-1,-2.0117857E-2,-2.3034851E-1,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1.7839447E-1,6.969961E-2,3.162982E-1,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,-2.1047248E-1,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.4450694E-1,9.220269E-3,-2.461917E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,-7.4021526E-2,-2.9362077E-1,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,-1,-1,37,39,-1,41,-1,43,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.2237465E1,2.848199E0,3.6767783E0,1.360198E0,4.0624598E-1,6.7093295E-1,8.615341E-1,5.956049E-1,2.3258556E-1,9.1160655E-2,1.3540426E-1,3.5389587E-1,4.596188E-1,0E0,3.909912E-1,6.838703E-1,0E0,1.7328064E-1,1.0814984E-1,0E0,0E0,1.2905765E-1,0E0,0E0,0E0,5.169381E-1,1.1691285E-1,0E0,1.8245935E-1,0E0,3.9284968E-1,0E0,0E0,0E0,0E0,0E0,1.09398365E-1,1.9227347E-1,2.0228004E-1,0E0,0E0,0E0,0E0,0E0,3.2970786E-1,0E0,0E0,0E0,0E0,1.4234447E-1,0E0,4.1922283E-1,0E0,0E0,0E0,2.4974391E-1,2.2993183E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,25,25,26,26,28,28,30,30,36,36,37,37,38,38,44,44,49,49,51,51,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,-1,-1,38,40,-1,42,-1,44,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[3.3103763E2,4.4269168E-1,1.7842576E-1,2.727992E6,2.1123085E8,7.5E2,1.431E4,1E0,1.8000048E10,1.3402277E0,1.1771224E8,3.9378E5,4.183787E0,4.4448983E-2,1E0,2.647867E-4,-2.4374343E-2,2.686459E6,1.6868966E8,-7.691218E-3,4.587331E-4,2.25E2,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,4.5971466E2,4.9863962E2,5.3746756E-3,2.9743274E8,-2.0117857E-2,1E0,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1E0,1.1572553E7,8.312613E5,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,2.933E3,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.488E3,9.220269E-3,6.7897725E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,2.7287993E0,6.767991E5,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"split_indices":[67,57,53,1,7,0,9,85,5,56,47,1,72,0,27,54,0,44,7,0,0,0,0,0,0,67,67,0,5,0,15,0,0,0,0,0,85,60,62,0,0,0,0,0,10,0,0,0,0,2,0,71,0,0,0,69,43,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.5E2,9.1E1,1.14E2,3.6E1,5.7E1,3.4E1,9.5E1,1.9E1,1E1,2.6E1,9E0,4.8E1,3E0,3.1E1,8.6E1,9E0,8E0,1.1E1,6E0,4E0,2.4E1,2E0,4E0,5E0,3.6E1,1.2E1,4E0,2.7E1,1.7E1,6.9E1,4E0,4E0,8E0,3E0,5E0,1.9E1,1.5E1,2.1E1,5E0,7E0,5E0,2.2E1,2E0,6.7E1,1.6E1,3E0,9E0,6E0,8E0,1.3E1,5.3E1,1.4E1,3E0,5E0,1.2E1,4.1E1,2E0,1E1,3.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.1875391E-3,-1.9125406E-1,2.4149728E-1,-3.2296014E-1,-4.532346E-2,9.999164E-2,4.7495925E-1,-3.5961127E-1,-3.3613036E-3,4.6751868E-2,-1.47186E-1,-3.1877293E-3,1.2955628E-1,1.2192243E-1,6.471233E-1,-2.8752145E-1,-5.190118E-1,-3.20057E-2,7.1740914E-3,-1.8803763E-1,5.201352E-3,1.4700729E-1,-4.1586305E-3,6.593719E-4,9.316681E-3,9.0452635E-1,5.182496E-1,4.057908E-2,-3.2754722E-1,-5.1887655E-3,-5.5908924E-1,3.8056083E-3,-3.9105513E-3,-2.4499905E-1,-2.4137127E-3,4.0515624E-2,1.8407537E-1,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,-2.7528796E-1,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,1.9600414E-1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,1.5410087E-1,4.4805845E-3,9.989807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,-1,33,-1,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,47,-1,49,51,-1,-1,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,57,-1,-1],"loss_changes":[1.1762257E1,2.7338152E0,3.6525607E0,6.483345E-1,6.540341E-1,3.7268925E-1,2.4738722E0,5.976982E-1,0E0,3.480822E-1,3.8151592E-1,0E0,2.5336754E-1,1.4349061E-1,4.6006012E-1,6.3051796E-1,2.3819494E-1,1.6576359E-1,0E0,2.0569742E-1,0E0,2.1754622E-1,0E0,0E0,0E0,2.0789146E-1,1.3972569E-1,1.1563162E-1,1.779995E-1,0E0,1.1710739E-1,0E0,0E0,1.0057318E-1,0E0,2.0077212E-1,1.1754835E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.099534E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.906554E-2,0E0,0E0,0E0,0E0,8.8078976E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,21,21,25,25,26,26,27,27,28,28,30,30,33,33,35,35,36,36,44,44,51,51,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,-1,34,-1,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,48,-1,50,52,-1,-1,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,58,-1,-1],"split_conditions":[3.5300907E-1,1.9415865E2,1.5972575E0,1.848519E-2,1.560615E1,1.8681856E8,2.744E3,5.039375E-3,-3.3613036E-3,3.165913E6,5.4878284E2,-3.1877293E-3,9.723711E0,6.765595E7,2.7479675E-1,1.0395554E5,1.47887E5,9.165388E4,7.1740914E-3,3.1692016E0,5.201352E-3,2.2599022E8,-4.1586305E-3,6.593719E-4,9.316681E-3,4.063E3,2.0467837E0,4.56E2,1.7565872E-3,-5.1887655E-3,6.6037946E2,3.8056083E-3,-3.9105513E-3,3.2829E4,-2.4137127E-3,1.19532734E5,1E0,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,4.5E2,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,5.7E1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,2.63E2,4.4805845E-3,9.989807E-3],"split_indices":[57,67,56,53,71,7,2,53,0,1,67,0,72,5,53,43,1,48,0,73,0,12,0,0,0,2,68,0,73,0,4,0,0,9,0,43,105,0,0,0,0,0,0,0,10,0,0,0,0,0,0,8,0,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,7.4E1,6.8E1,7E1,4.1E1,6.4E1,1E1,3.6E1,3.2E1,1E1,6E1,1.4E1,2.7E1,4.6E1,1.8E1,2.2E1,1.4E1,2.8E1,4E0,5.6E1,4E0,7E0,7E0,7E0,2E1,5E0,4.1E1,2E0,1.6E1,7E0,1.5E1,1.9E1,9E0,1.5E1,4.1E1,2E0,5E0,1.7E1,3E0,2E0,3E0,9E0,3.2E1,1.4E1,2E0,8E0,1.1E1,3E0,1.2E1,3.9E1,2E0,2.3E1,9E0,1.1E1,2.8E1,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.9416563E-3,-1.3463952E-1,3.7984976E-1,-2.3853315E-1,1.360329E-2,1.961173E-1,5.876671E-1,-3.1009814E-1,-1.1053116E-1,-3.428409E-2,1.00340895E-1,2.3631513E-2,1.5987208E-1,1.1244957E-2,6.9594884E-1,-6.4688744E-3,-3.5531205E-1,1.6310263E-3,-2.0668645E-1,-7.293062E-2,1.3145976E-1,9.800815E-3,-3.9949227E-2,-4.0140087E-3,1.8721226E-1,4.3395724E-2,2.3986792E-2,3.5975182E-3,-3.7373397E-1,-1.2723958E-2,-3.946153E-3,-9.6303485E-2,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,2.38724E-1,1.5173156E-3,-3.9501062E-1,-4.1339686E-3,8.0993416E-4,-1.22839086E-1,1.2143473E-2,4.7381404E-3,-4.0761718E-1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,-4.2140347E-1,-2.2882612E-2,-1.5165369E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,-1,29,31,33,-1,35,-1,37,-1,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,43,-1,45,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,-1],"loss_changes":[1.2767532E1,2.8833227E0,2.3666296E0,9.7337484E-1,3.2742375E-1,3.641218E-1,8.926077E-1,4.596181E-1,5.9082735E-1,3.3626413E-1,4.9703383E-1,0E0,2.5133562E-1,0E0,4.1397285E-1,0E0,4.6627617E-1,0E0,2.203567E-1,2.0018259E-1,1.2457131E-1,0E0,2.0385656E-1,0E0,2.3512948E-1,0E0,0E0,0E0,2.7126503E-1,0E0,0E0,1.1810273E-1,0E0,0E0,0E0,0E0,0E0,8.816862E-2,0E0,1.6809559E-1,0E0,0E0,9.8291665E-2,0E0,0E0,9.2234135E-2,0E0,0E0,0E0,0E0,1.4487553E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,18,18,19,19,20,20,22,22,24,24,28,28,31,31,37,37,39,39,42,42,45,45,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,-1,30,32,34,-1,36,-1,38,-1,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,44,-1,46,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.427E3,8.9397516E4,3.856E3,2.4747229E2,1.3048244E1,1.4191151E6,9E0,2.3631513E-2,7.5E2,1.1244957E-2,2.7479675E-1,-6.4688744E-3,3.8757736E-7,1.6310263E-3,3.03E9,4.222002E8,1.31903E5,9.800815E-3,1.3116745E1,-4.0140087E-3,4.183787E0,4.3395724E-2,2.3986792E-2,3.5975182E-3,7.4E1,-1.2723958E-2,-3.946153E-3,2.6639402E-1,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,4.3147E5,1.5173156E-3,9.4983295E2,-4.1339686E-3,8.0993416E-4,4.9615598E-1,1.2143473E-2,4.7381404E-3,4.699202E1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,9.6713684E1,-2.2882612E-2,-1.5165369E-2],"split_indices":[67,56,53,67,2,48,2,70,71,48,3,0,0,0,53,0,53,0,5,7,1,0,71,0,72,0,0,0,3,0,0,57,0,0,0,0,0,9,0,4,0,0,54,0,0,67,0,0,0,0,67,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.86E2,6.4E1,1.09E2,7.7E1,3.5E1,2.9E1,6.9E1,4E1,5E1,2.7E1,2E0,3.3E1,8E0,2.1E1,1.6E1,5.3E1,1.6E1,2.4E1,4.1E1,9E0,1.4E1,1.3E1,3E0,3E1,5E0,1.6E1,2E0,5.1E1,1.3E1,1.1E1,3.7E1,4E0,6E0,3E0,4E0,9E0,2.2E1,8E0,4.7E1,4E0,7E0,3E1,1.6E1,6E0,4.5E1,2E0,1.1E1,1.9E1,3E0,4.2E1,1.5E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[4.442649E-3,-1.2406673E-1,3.1314126E-1,-2.231523E-1,2.7577164E-2,1.5845458E-1,5.6371045E-1,-2.9410875E-1,-5.8644965E-2,-2.6454885E-2,1.4853756E-1,2.294401E-2,1.3163756E-1,2.7344096E-3,6.19114E-1,-9.231322E-2,-3.2586154E-1,-1.3393481E-1,2.8952213E-2,-4.4858553E-3,1.6983777E-2,1.7551805E-1,-3.9204443E-3,2.2740224E-1,3.826856E-2,1.9222386E-2,8.226514E-1,-7.2305724E-3,7.988117E-3,-3.752356E-1,-1.8137792E-1,3.932153E-3,-7.2262976E-3,-2.8384456E-3,1.1973143E-1,4.0585145E-2,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,-1.2312204E-1,5.3003617E-3,1.6423428E-2,4.1150242E-2,-3.9276025E-1,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,9.395361E-2,3.7756322E-3,-8.526156E-3,-3.2505807E-1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,-3.9313173E-1,-1.9234711E-2,-7.488049E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,-1,39,41,-1,43,-1,-1,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,53,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,61,-1,-1],"loss_changes":[1.0392895E1,2.7859118E0,2.895176E0,1.2901125E0,4.8757413E-1,4.1345346E-1,7.6624584E-1,4.6970892E-1,2.3470895E-1,1.741834E-1,1.6369265E-1,0E0,4.1154587E-1,0E0,5.829191E-1,2.727965E-1,4.0990114E-1,1.4968318E-1,1.5284702E-1,0E0,1.21554464E-1,9.423089E-2,0E0,2.5063312E-1,3.5061046E-1,0E0,2.3580599E-1,0E0,0E0,1.4392567E-1,3.3857572E-1,0E0,0E0,0E0,1.4192495E-1,1.1094484E-1,0E0,0E0,0E0,0E0,0E0,1.5323834E-1,0E0,0E0,0E0,2.5782776E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.29731E-2,0E0,0E0,1.570108E-1,0E0,0E0,0E0,0E0,1.24889135E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,26,26,29,29,30,30,34,34,35,35,41,41,45,45,52,52,55,55,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,-1,40,42,-1,44,-1,-1,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,54,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,62,-1,-1],"split_conditions":[3.705974E2,2.1619815E-1,2.0224316E-1,4.5537444E9,2.427E3,8.9397516E4,1E0,1.1563724E8,2.0256741E2,1.503E3,5.4281917E0,2.294401E-2,1.3844932E7,2.7344096E-3,1.6507233E3,3.160848E6,2.2348747E2,1.903E3,4.231592E6,-4.4858553E-3,2.302E3,3.5734247E2,-3.9204443E-3,5.264E4,1.1218195E6,1.9222386E-2,1.61628E5,-7.2305724E-3,7.988117E-3,9.0814667E2,4.1081388E5,3.932153E-3,-7.2262976E-3,-2.8384456E-3,4.3560153E5,6.4796996E-1,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,1.4153711E-3,5.3003617E-3,1.6423428E-2,4.1150242E-2,1.549E3,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,1.5130641E0,3.7756322E-3,-8.526156E-3,1.5E1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,1E0,-1.9234711E-2,-7.488049E-3],"split_indices":[67,56,53,5,2,48,27,5,67,2,69,0,60,0,4,9,67,0,1,0,2,67,0,9,62,0,43,0,0,70,43,0,0,0,47,54,0,0,0,0,0,53,0,0,0,2,0,0,0,0,0,0,71,0,0,3,0,0,0,0,30,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,1.84E2,7.6E1,1.11E2,7.3E1,4.8E1,2.8E1,7.7E1,3.4E1,5.1E1,2.2E1,2E0,4.6E1,3E0,2.5E1,1.1E1,6.6E1,1.8E1,1.6E1,1.8E1,3.3E1,2E1,2E0,2.2E1,2.4E1,1.5E1,1E1,9E0,2E0,4.8E1,1.8E1,2E0,1.6E1,8E0,8E0,2.9E1,4E0,1.8E1,2E0,1E1,1.2E1,8E0,1.6E1,3E0,7E0,4.4E1,4E0,1.4E1,4E0,3E0,5E0,1.3E1,1.6E1,2E0,6E0,3.2E1,1.2E1,8E0,8E0,1.4E1,1.8E1,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.2187884E-2,-1.0560222E-1,2.8632665E-1,-1.6603857E-1,6.5966286E-2,1.4586735E-1,4.827894E-1,-2.163441E-1,3.5765186E-2,-2.1459074E-3,9.6117355E-2,2.2276511E-2,1.16351E-1,3.9712414E-3,5.3956425E-1,-1.9085151E-1,-4.2924163E-1,1.310548E-1,-6.960556E-2,8.844785E-3,5.8431923E-2,-3.4577996E-2,1.8520117E-1,1.25307795E-2,2.6944714E-2,-3.2403958E-1,-1.5634167E-1,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,5.7000898E-2,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,4.627612E-2,-1.0834768E-2,2.3908277E-1,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,-1.3881442E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,-1.12805806E-1,-3.083408E-1,-7.1671814E-2,-2.4239872E-1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,-1,47,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1],"loss_changes":[7.844618E0,1.7736785E0,1.9440131E0,1.2868705E0,1.6094884E-1,4.141271E-1,6.1820793E-1,4.6623707E-1,2.7018133E-1,0E0,1.327295E-1,0E0,4.4284636E-1,0E0,3.3383512E-1,3.7219834E-1,2.928431E-1,1.1195864E-1,1.3693142E-1,0E0,1.6056333E-1,2.5339377E-1,1.8323988E-1,0E0,0E0,1.318934E-1,3.0327547E-1,0E0,0E0,0E0,1.019789E-1,0E0,0E0,0E0,0E0,1.6095355E-1,0E0,1.4495695E-1,0E0,0E0,0E0,0E0,2.8267586E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2379E-1,1.0256362E-1,2.1890551E-1,9.2020154E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,30,30,35,35,37,37,42,42,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,-1,48,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,4.9615598E-1,1.9530877E-1,2.727992E6,1.8681856E8,8.9397516E4,1.2354571E0,1E0,1.6E1,-2.1459074E-3,2.0383387E0,2.2276511E-2,1.479938E6,3.9712414E-3,7.2610445E8,3.3617156E-4,2.22105E5,1.2051221E0,1.7108529E3,8.844785E-3,2.2599022E8,2.629593E0,1.8990228E0,1.25307795E-2,2.6944714E-2,1.7565872E-3,3.4955364E7,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,4.654E3,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,6.2143734E4,-1.0834768E-2,1.6851284E7,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,3.219065E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,8.3170245E2,2.07298E5,6.214038E6,1.2271346E1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"split_indices":[67,54,53,1,7,48,68,85,3,0,73,0,47,0,12,54,1,68,4,0,12,69,68,0,0,73,7,0,0,0,2,0,0,0,0,43,0,1,0,0,0,0,57,0,0,0,0,0,0,70,1,9,71,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.69E2,7.2E1,1.25E2,4.4E1,4.3E1,2.9E1,1E2,2.5E1,9E0,3.5E1,2E0,4.1E1,4E0,2.5E1,9.1E1,9E0,1.3E1,1.2E1,8E0,2.7E1,1.3E1,2.8E1,7E0,1.8E1,1.7E1,7.4E1,2E0,7E0,4E0,9E0,1E1,2E0,1.7E1,1E1,1E1,3E0,1.9E1,9E0,6E0,1.1E1,3E0,7.1E1,6E0,3E0,5E0,5E0,1.3E1,6E0,6.3E1,8E0,4.9E1,1.4E1,4E0,4E0,4.4E1,5E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[8.535432E-5,-1.3134183E-1,2.4541517E-1,-2.1222556E-1,-3.1241742E-3,1.1487452E-1,4.674134E-1,-2.7328297E-1,-3.8918845E-2,-8.365333E-2,9.0494335E-2,2.394816E-3,1.6543052E-1,2.984512E-1,6.282826E-1,-2.368147E-1,-2.039022E-2,-7.2169475E-2,5.6061996E-3,-1.5359263E-1,5.1900256E-2,2.500614E-4,6.980609E-3,5.4644295E-3,-7.968769E-2,1.8786167E-1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,7.535615E-1,1.2825073E-2,-2.2865131E-2,-2.539767E-1,4.7256513E-3,-1.1367583E-1,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.703592E-1,1.374485E-1,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,-2.2705331E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,-2.3829083E-1,-2.8504506E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,-1,-1,-1,39,41,-1,-1,-1,43,-1,45,47,-1,49,-1,-1,-1,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1],"loss_changes":[8.318686E0,1.7416182E0,2.5544882E0,1.0792751E0,5.0514543E-1,3.2942206E-1,7.09455E-1,4.2694187E-1,1.6511148E-1,3.4927028E-1,1.8162525E-1,2.0153141E-1,2.3894668E-1,1.7641377E-1,4.4881153E-1,2.3653197E-1,0E0,1.9207498E-1,0E0,2.4937719E-1,1.370348E-1,0E0,0E0,0E0,2.0430191E-1,1.1672115E-1,0E0,0E0,0E0,1.8685484E-1,0E0,1.2997259E-1,1.903727E-1,0E0,2.3296595E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.380126E-2,9.920034E-2,0E0,0E0,0E0,0E0,0E0,1.8071914E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.499064E-1,1.1059618E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,24,24,25,25,29,29,31,31,32,32,34,34,41,41,42,42,48,48,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,-1,-1,-1,40,42,-1,-1,-1,44,-1,46,48,-1,50,-1,-1,-1,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1],"split_conditions":[3.4021906E2,2.5106274E-2,1.8829846E-1,1.781973E6,9.50379E5,3.359399E-1,5.6E1,2.418166E0,1.22E2,1.5225257E-1,5.3164E4,1.1936242E7,5.615313E3,2.598887E7,3.2307692E0,1.1623532E6,-2.039022E-2,2.207E3,5.6061996E-3,2.587E3,8.180814E7,2.500614E-4,6.980609E-3,5.4644295E-3,1.0520055E5,5.7E1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,1.4475617E0,1.2825073E-2,2.1426034E5,1.4614057E-3,4.7256513E-3,2.4616451E0,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.7307575E5,1.0899524E3,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,4.2446044E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,1.3496346E0,1.1306552E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"split_indices":[67,53,53,1,9,57,8,68,8,53,44,60,4,9,69,60,0,0,0,2,47,0,0,0,48,8,0,0,0,68,0,43,73,0,69,0,0,0,0,0,0,43,4,0,0,0,0,0,71,0,0,0,0,0,0,0,68,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.56E2,1.67E2,8.9E1,1.02E2,6.5E1,5.7E1,3.2E1,7.5E1,2.7E1,3.5E1,3E1,1.8E1,3.9E1,1.7E1,1.5E1,6.5E1,1E1,2.3E1,4E0,2.3E1,1.2E1,1.4E1,1.6E1,7E0,1.1E1,3.6E1,3E0,1.5E1,2E0,1E1,5E0,5E0,6E1,4E0,1.9E1,2E1,3E0,9E0,3E0,2E0,9E0,1.2E1,2.4E1,4E0,6E0,2E0,3E0,6E0,5.4E1,1.7E1,2E0,7E0,5E0,5E0,1.9E1,2E0,5.2E1,3.1E1,2.1E1,2.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-7.534086E-3,-8.694848E-2,3.8782644E-1,-1.7279062E-1,7.5343594E-2,1.2632059E-1,5.605976E-1,-2.4380171E-1,-4.4355273E-2,1.0239507E-1,-7.070103E-2,-8.078534E-4,9.607972E-3,6.888959E-1,4.1058496E-1,-2.8629518E-1,-1.11263916E-1,-1.6469195E-2,-1.506756E-2,4.8591096E-2,1.8556008E-1,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,-6.6435128E-3,-3.0423003E-1,-1.0063401E-2,2.9212602E-3,-4.0811297E-2,7.316056E-3,-2.3700453E-2,1.1168387E-1,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,-3.2206443E-1,-3.669968E-3,-6.68522E-2,7.2118444E-3,4.6604417E-2,-4.6691555E-3,8.108817E-3,-7.8055467E-3,-3.4489202E-1,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,-3.753321E-1,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,47,-1,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,-1],"loss_changes":[7.88305E0,2.924702E0,1.8267727E0,1.2386718E0,2.9564223E-1,2.4874002E-1,1.9830894E-1,4.6376085E-1,4.8201522E-1,2.6840913E-1,1.2740509E-1,0E0,0E0,3.1174803E-1,9.529376E-2,3.2900333E-1,5.045887E-1,0E0,2.2792198E-1,1.8052885E-1,1.6800141E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.805466E-2,2.1632099E-1,0E0,0E0,2.3643395E-1,0E0,1.16251014E-1,1.8746912E-1,0E0,0E0,0E0,0E0,1.5240574E-1,0E0,1.1973961E-1,0E0,9.648324E-2,0E0,0E0,1.9439551E-1,1.6225243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4404154E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,27,27,28,28,31,31,33,33,34,34,39,39,41,41,43,43,46,46,47,47,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,48,-1,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,-1],"split_conditions":[1.311002E0,3.051E3,2.787E3,1.5157113E-2,1.50321455E1,2.024E3,2.9898608E-1,2.2348747E2,2.718E4,4.871647E2,1.471E4,-8.078534E-4,9.607972E-3,1.792184E0,3.4048372E6,1E1,2.8041378E5,-1.6469195E-2,1.3431462E7,6.477987E-1,1.8751362E7,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,7.1E1,6.8E1,-1.0063401E-2,2.9212602E-3,1.00849255E3,7.316056E-3,4.303E3,4.99E3,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,1E0,-3.669968E-3,2.1799392E6,7.2118444E-3,2.5063604E7,-4.6691555E-3,8.108817E-3,6.074E3,4.239E3,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,3.1872916E2,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"split_indices":[54,2,2,53,72,2,53,67,9,67,0,0,0,57,60,10,43,0,9,72,59,0,0,0,0,0,0,0,3,0,0,4,0,2,2,0,0,0,0,30,0,63,0,60,0,0,2,10,0,0,0,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,2.08E2,4.1E1,1.36E2,7.2E1,1.7E1,2.4E1,8.7E1,4.9E1,6.1E1,1.1E1,7E0,1E1,1.1E1,1.3E1,6.5E1,2.2E1,3E0,4.6E1,3.8E1,2.3E1,8E0,3E0,5E0,6E0,8E0,5E0,4E0,6.1E1,1.3E1,9E0,4.1E1,5E0,1.8E1,2E1,1.4E1,9E0,2E0,2E0,5.6E1,5E0,3.7E1,4E0,1E1,8E0,1.2E1,8E0,4.8E1,8E0,2E1,1.7E1,7E0,3E0,6E0,2E0,3.9E1,9E0,1.4E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.6526852E-2,-1.3732727E-1,1.8663803E-1,-2.051646E-1,-2.9185692E-2,7.73927E-2,3.695984E-1,-2.6628572E-1,-9.389659E-2,-8.110631E-2,7.3622145E-2,4.680393E-3,1.8481968E-1,1.6930054E-3,3.9849573E-1,-2.2754145E-1,-2.0849528E-2,6.2429474E-4,-1.7701784E-1,-3.0688336E-1,-5.0080888E-2,4.623225E-3,-4.2899745E-3,1.03257135E-1,-1.1058583E-1,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,-2.4562204E-1,7.5825364E-3,-1.0521853E-2,-6.476905E-2,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,1.3644478E-1,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,-2.6290497E-1,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,-2.9959735E-1,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,31,33,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,-1,43,-1,-1,-1,-1,45,-1,-1,-1,-1,47,-1,-1,-1,-1,49,-1,-1,-1],"loss_changes":[6.3821125E0,1.1897857E0,1.9101899E0,6.5507936E-1,3.4759343E-1,4.8343337E-1,3.2867527E-1,4.1569996E-1,3.3530772E-1,2.8583866E-1,1.3266498E-1,4.4319636E-1,5.4417163E-1,0E0,2.268424E-1,4.489975E-1,0E0,0E0,1.3546717E-1,1.6951728E-1,1.4221326E-1,0E0,0E0,1.3188584E-1,1.2820828E-1,0E0,0E0,0E0,0E0,1.5322685E-1,0E0,0E0,9.677847E-2,0E0,0E0,0E0,0E0,1.2525946E-1,0E0,0E0,0E0,0E0,2.1090531E-1,0E0,0E0,0E0,0E0,1.8816972E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,18,18,19,19,20,20,23,23,24,24,29,29,32,32,37,37,42,42,47,47],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,32,34,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,-1,44,-1,-1,-1,-1,46,-1,-1,-1,-1,48,-1,-1,-1,-1,50,-1,-1,-1],"split_conditions":[3.3103763E2,1.1670399E-1,1.7842576E-1,1.9415865E2,2.7538432E8,6.483E3,1E0,4.1938826E-2,3.0364912E5,3.3789E4,1.1771224E8,1.1936242E7,6.2143734E4,1.6930054E-3,7.2610445E8,1.1661899E8,-2.0849528E-2,6.2429474E-4,1.3469651E3,2.151E3,1.2473118E0,4.623225E-3,-4.2899745E-3,1.9816086E0,1.4813267E3,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,6E0,7.5825364E-3,-1.0521853E-2,2.8659306E0,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,3.8692224E8,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,4.426498E0,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,1.1078758E0,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"split_indices":[67,54,53,67,7,2,27,57,48,9,47,60,43,0,12,7,0,0,4,2,68,0,0,68,70,0,0,0,0,10,0,0,73,0,0,0,0,7,0,0,0,0,71,0,0,0,0,69,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.62E2,9.6E1,9.9E1,6.3E1,6.1E1,3.5E1,6.3E1,3.6E1,4.2E1,2.1E1,3.7E1,2.4E1,3E0,3.2E1,5.5E1,8E0,1.6E1,2E1,4E0,3.8E1,1.8E1,3E0,2E1,1.7E1,1.4E1,1E1,1.2E1,2E1,5.3E1,2E0,1.2E1,8E0,2E0,2E0,9E0,2.9E1,1.7E1,3E0,1.2E1,5E0,5E0,4.8E1,6E0,2E0,1.5E1,2E0,3.7E1,1.1E1,5E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.752704E-3,-1.0890242E-1,2.1311827E-1,-1.7942376E-1,-4.630112E-3,8.463161E-2,4.3359125E-1,-2.6526305E-1,-4.0561736E-2,-8.0085285E-2,6.466574E-2,1.7120663E-2,6.155428E-2,2.8981546E-1,6.182841E-1,-1.9980457E-1,-3.8043132E-1,2.1313105E-2,-7.922915E-3,-1.4972365E-1,5.4258812E-2,-3.958611E-6,5.6440667E-3,-6.662196E-2,1.1769473E-1,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,-1.3539992E-1,-2.9197437E-1,-4.6381897E-1,-6.432207E-3,-5.7250984E-2,5.7095457E-3,-7.0690963E-4,-2.4091765E-1,-1.8624915E-3,6.908497E-3,7.913083E-2,-2.8256294E-3,-1.419902E-1,4.703381E-3,1.525669E-1,-8.453891E-4,2.933514E-3,-7.012109E-3,-3.371898E-1,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,7.1813464E-2,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,43,-1,-1,-1,-1,45,47,49,-1,51,-1,-1,53,-1,-1,55,-1,57,-1,59,-1,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,65,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1809554E0,1.2198377E0,2.597793E0,1.1693025E0,3.607325E-1,4.0189618E-1,7.2366E-1,3.8385105E-1,3.4410596E-1,3.1685543E-1,1.5055917E-1,0E0,4.1800576E-1,1.5071511E-1,4.769044E-1,2.0384932E-1,3.3971214E-1,2.4841867E-1,0E0,2.639835E-1,1.2779592E-1,1.0247372E-1,0E0,2.5190163E-1,1.941486E-1,0E0,0E0,0E0,0E0,9.141275E-2,1.3482976E-1,1.2367654E-1,0E0,1.1534542E-1,0E0,0E0,1.0435748E-1,0E0,0E0,1.1921386E-1,0E0,1.5446314E-1,0E0,1.8064415E-1,0E0,1.0156908E-1,0E0,1.0845828E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.946659E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,29,29,30,30,31,31,33,33,36,36,39,39,41,41,43,43,45,45,47,47,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,44,-1,-1,-1,-1,46,48,50,-1,52,-1,-1,54,-1,-1,56,-1,58,-1,60,-1,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,66,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.7613442E-1,1.8829846E-1,1.9415865E2,9.50379E5,8.9397516E4,1.623319E3,1.607E3,2.142943E1,1.2185057E0,6.6727E4,1.7120663E-2,3.359399E-1,2.0270285E8,2.65826E6,4.3399894E5,6.608892E2,6.135367E9,-7.922915E-3,2.8507206E-1,1.7468556E2,1.2495134E6,5.6440667E-3,5.4878284E2,3.2383578E7,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,1.1623532E6,3.342E3,2.1710082E7,-6.432207E-3,2.2E1,5.7095457E-3,-7.0690963E-4,6.2684685E-2,-1.8624915E-3,6.908497E-3,1.6537195E0,-2.8256294E-3,1.409674E7,4.703381E-3,1.7111111E0,-8.453891E-4,4.14E2,-7.012109E-3,1E0,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,1.0685258E6,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"split_indices":[67,56,53,67,9,48,4,2,71,56,44,0,57,7,62,43,4,5,0,57,67,60,0,67,1,0,0,0,0,60,0,47,0,3,0,0,53,0,0,68,0,60,0,68,0,2,0,30,0,0,0,0,0,0,0,0,0,0,0,0,48,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.65E2,9.2E1,9.8E1,6.7E1,5.9E1,3.3E1,6E1,3.8E1,3.2E1,3.5E1,3E0,5.6E1,2E1,1.3E1,4E1,2E1,2.7E1,1.1E1,2.1E1,1.1E1,1.8E1,1.7E1,1.7E1,3.9E1,3E0,1.7E1,9E0,4E0,2.5E1,1.5E1,1.4E1,6E0,1.6E1,1.1E1,9E0,1.2E1,6E0,5E0,8E0,1E1,1.2E1,5E0,3.1E1,8E0,4E0,2.1E1,1.2E1,3E0,1.2E1,2E0,1.1E1,5E0,5E0,7E0,5E0,3E0,3E0,9E0,1.5E1,1.6E1,2E0,2E0,8E0,4E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-4.591145E-3,-1.0742565E-1,2.1129104E-1,-1.6721527E-1,-3.2239808E-3,9.313745E-2,3.8930318E-1,-2.2656383E-1,-3.6957923E-2,-4.179782E-2,9.553272E-2,1.1645269E-1,-6.1090174E-3,9.100737E-3,4.8732993E-1,-1.9007076E-1,-3.574318E-1,8.0027975E-2,-3.7945511E-3,3.2766287E-3,-6.838066E-2,5.598663E-3,-2.234741E-3,-1.6062098E-3,1.4490457E-1,3.515733E-2,1.7769938E-2,2.4345317E-3,-2.1554193E-1,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,-8.616666E-2,7.324565E-3,2.3872483E-1,1.1299209E-1,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,-1.5753022E-1,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.572733E0,1.0585915E0,1.6673958E0,8.2653E-1,2.4398685E-1,2.8807345E-1,4.6537733E-1,2.8546977E-1,2.1166551E-1,1.4805996E-1,9.310773E-2,2.0591748E-1,0E0,0E0,1.8203974E-1,2.947247E-1,2.1712995E-1,1.3969424E-1,0E0,0E0,1.7411983E-1,0E0,0E0,0E0,8.9793384E-2,0E0,0E0,1.5928355E-1,2.97374E-1,0E0,0E0,0E0,0E0,1.2090984E-1,0E0,9.095979E-2,1.5865728E-1,0E0,0E0,0E0,1.3265473E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,20,20,24,24,27,27,28,28,33,33,35,35,36,36,40,40],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0471938E6,1.529448E-1,1.2711891E0,2.8704E4,9.74979E6,1.25440235E1,3.301E3,4.1295107E-2,1.0520055E5,4.46E2,1E0,3.359399E-1,-6.1090174E-3,9.100737E-3,1.431E4,1.1695156E5,1.1E1,2.00902E2,-3.7945511E-3,3.2766287E-3,2.65852E6,5.598663E-3,-2.234741E-3,-1.6062098E-3,8.312613E5,3.515733E-2,1.7769938E-2,1E0,2.647867E-4,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,1.1343225E6,7.324565E-3,9.616974E2,3.7724147E8,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,5.0090684E5,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"split_indices":[63,54,54,11,9,72,2,57,48,11,85,57,0,0,9,43,3,67,0,0,1,0,0,0,62,0,0,8,54,0,0,0,0,48,0,4,7,0,0,0,43,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.69E2,8E1,1.07E2,6.2E1,4.9E1,3.1E1,7.3E1,3.4E1,4.5E1,1.7E1,4.5E1,4E0,1.2E1,1.9E1,5.9E1,1.4E1,1E1,2.4E1,8E0,3.7E1,1.4E1,3E0,7E0,3.8E1,2E0,1.7E1,7E0,5.2E1,6E0,8E0,2E0,8E0,3.5E1,2E0,8E0,3E1,3E0,4E0,1.6E1,3.6E1,2.4E1,1.1E1,3E0,5E0,1.7E1,1.3E1,3.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.739959E-3,-1.0096392E-1,2.598716E-1,-1.7680193E-1,-3.243053E-2,1.19685955E-1,4.226019E-1,-8.241986E-2,-2.739108E-1,-8.830396E-2,4.9990978E-2,1.8303644E-2,8.3055995E-2,2.0394288E-3,4.746282E-1,-1.0853977E-2,-3.597446E-2,-3.5187337E-1,-1.5231963E-1,-2.7551723E-1,-2.8254518E-2,9.033917E-3,5.8722845E-3,-8.941312E-3,1.0602045E-1,6.4259046E-1,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.3898046E-2,-4.325442E-3,3.7839024E-3,-3.718894E-2,1.2833083E-1,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,29,31,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4447184E0,8.7263286E-1,1.5462923E0,7.1065116E-1,4.1887993E-1,3.9337415E-1,6.0357904E-1,3.1546003E-1,3.161056E-1,5.992862E-1,1.2694931E-1,0E0,2.597763E-1,0E0,4.780264E-1,0E0,2.3357649E-1,1.2557936E-1,1.1205655E-1,3.2829583E-1,1.5759072E-1,9.657455E-2,0E0,0E0,1.6324815E-1,1.374054E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4665566E-1,0E0,0E0,1.1835869E-1,1.3544708E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,17,17,18,18,19,19,20,20,21,21,24,24,25,25,35,35,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,30,32,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,1.6375645E2,2.0224316E-1,1.69607E5,3.2764925E5,1.2E1,1.2354571E0,4.0561585E-3,5.3597E2,3.6477748E7,7.128343E5,1.8303644E-2,1.3E1,2.0394288E-3,2.9898608E-1,-1.0853977E-2,1.2245615E0,3.9647983E2,9.714086E0,6.779745E6,3.642097E6,9.598357E0,5.8722845E-3,-8.941312E-3,9.723711E0,2.4214827E5,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.142943E1,-4.325442E-3,3.7839024E-3,8.1262054E2,1.7021878E7,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"split_indices":[67,67,53,1,43,3,68,56,4,12,43,0,3,0,53,0,68,4,62,12,47,71,0,0,72,43,0,0,0,0,0,0,0,0,0,71,0,0,4,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,1.68E2,6.9E1,7.9E1,8.9E1,3.8E1,3.1E1,4.1E1,3.8E1,5.3E1,3.6E1,3E0,3.5E1,4E0,2.7E1,8E0,3.3E1,2.2E1,1.6E1,1.2E1,4.1E1,2.5E1,1.1E1,2E0,3.3E1,1.1E1,1.6E1,7E0,2.6E1,1.2E1,1E1,2E0,1.4E1,8E0,4E0,2.4E1,1.7E1,9E0,1.6E1,3E1,3E0,4E0,7E0,1.5E1,9E0,1.1E1,5E0,2.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.7042104E-3,-7.20367E-2,2.6177105E-1,-1.2977166E-1,4.7325946E-2,1.1269311E-1,4.0182242E-1,-1.1040865E-1,-3.0936033E-1,1.5922122E-2,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,-1.6456133E-1,-5.346679E-2,-1.7062483E-2,-2.109847E-3,-2.8860811E-2,1.2831074E-1,-2.0478494E-1,5.846947E-4,8.376397E-3,-6.544534E-2,-5.855391E-3,-6.571167E-3,1.2199961E-2,1.0790331E-3,-1.0410925E-1,-2.5988343E-1,-7.7399775E-2,7.660132E-3,3.2776052E-3,-5.569639E-2,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,-5.8419913E-2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,2.7633142E-3,1.8774311E-3,-5.525428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,25,27,29,-1,-1,31,33,-1,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,-1,-1,45,-1,-1],"loss_changes":[4.8915606E0,1.3223984E0,1.1447959E0,4.1442585E-1,2.5857258E-1,1.2807718E-1,6.305504E-1,3.5506094E-1,2.3678052E-1,2.6542392E-1,0E0,0E0,0E0,0E0,0E0,4.3223488E-1,1.8600516E-1,0E0,0E0,1.0552636E-1,2.2810276E-1,2.4475026E-1,0E0,0E0,1.7779067E-1,1.3734703E-1,0E0,0E0,0E0,1.1772323E-1,1.4893365E-1,1.1551082E-1,0E0,0E0,1.1556485E-1,0E0,0E0,0E0,0E0,1.1671467E-1,0E0,0E0,0E0,0E0,1.0962085E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,15,15,16,16,19,19,20,20,21,21,24,24,25,25,29,29,30,30,31,31,34,34,39,39,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,26,28,30,-1,-1,32,34,-1,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,-1,-1,46,-1,-1],"split_conditions":[1.3394321E0,7.629E3,2.0787016E5,3.3135505E0,5.2328894E5,7.323253E8,1.623319E3,1.66863E5,1.5157113E-2,8.834E3,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,2.951829E9,1.55E2,-1.7062483E-2,-2.109847E-3,2.1445148E0,5.2638875E6,6.8618E4,5.846947E-4,8.376397E-3,6.098826E6,3.1791415E11,-6.571167E-3,1.2199961E-2,1.0790331E-3,1.3182504E0,1.7218688E8,2.2733334E1,7.660132E-3,3.2776052E-3,4.1E1,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,2.0256741E2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,4.4326835E0,1.8774311E-3,-5.525428E-3],"split_indices":[57,10,43,68,43,46,4,9,53,2,0,0,0,0,0,5,11,0,0,68,58,44,0,0,63,46,0,0,0,71,7,71,0,0,8,0,0,0,0,67,0,0,0,0,69,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.9E2,5.6E1,1.28E2,6.2E1,2.8E1,2.8E1,1.17E2,1.1E1,5.1E1,1.1E1,6E0,2.2E1,2E1,8E0,5.9E1,5.8E1,8E0,3E0,3.7E1,1.4E1,4.8E1,1.1E1,2E0,5.6E1,3.2E1,5E0,5E0,9E0,1.8E1,3E1,5.4E1,2E0,1.2E1,2E1,1.2E1,6E0,2.1E1,9E0,4.8E1,6E0,5E0,1.5E1,2.9E1,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.6167102E-2,-9.1617934E-2,2.0921354E-1,-1.5892246E-1,1.5889113E-4,1.0256023E-1,3.3971918E-1,-2.3020288E-1,-4.6520796E-2,-1.315317E-1,3.3601597E-2,1.9421874E-2,6.8316124E-2,2.9019982E-2,2.54235E-1,-7.909159E-2,-2.7484733E-1,5.3665806E-3,-8.432639E-3,-3.2230811E-3,-1.557175E-2,-2.764732E-3,5.4405066E-3,-5.779646E-2,1.18016526E-1,5.7629147E-3,3.5522467E-1,7.019195E-3,-5.448651E-3,-3.0026987E-1,3.8947738E-4,-5.0486766E-2,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,1.5366653E-1,-2.1028195E-3,1.7452618E-2,3.831069E-3,-2.4670707E-1,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,-1,-1,33,-1,35,37,-1,39,-1,-1,41,-1,43,-1,-1,-1,-1,-1,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.134416E0,1.1242383E0,8.142085E-1,8.305547E-1,3.4793866E-1,3.800948E-1,5.817108E-1,4.100237E-1,3.2077062E-1,1.8154746E-1,2.089919E-1,0E0,2.1365988E-1,0E0,2.3687482E-1,1.9577982E-1,3.561766E-1,2.3537263E-1,0E0,0E0,0E0,1.8856645E-1,0E0,8.858822E-2,1.4960828E-1,0E0,1.08366966E-1,0E0,0E0,1.7012787E-1,0E0,1.1466624E-1,0E0,0E0,0E0,0E0,0E0,9.963548E-2,0E0,0E0,0E0,1.5310645E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,21,21,23,23,24,24,26,26,29,29,31,31,37,37,41,41],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,-1,-1,34,-1,36,38,-1,40,-1,-1,42,-1,44,-1,-1,-1,-1,-1,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,1.9530877E-1,1.80025E2,1.8681856E8,8.9397516E4,1.8681E4,1.433458E10,2.142943E1,2.151E3,2.704375E6,1.9421874E-2,1.479938E6,2.9019982E-2,6.7E1,2.7287993E0,3.402546E6,6.135367E9,-8.432639E-3,-3.2230811E-3,-1.557175E-2,9.143112E5,5.4405066E-3,9.71709E5,2.30273E6,5.7629147E-3,6.3898524E7,7.019195E-3,-5.448651E-3,4.1938826E-2,3.8947738E-4,9.165388E4,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,5.7E1,-2.1028195E-3,1.7452618E-2,3.831069E-3,1.0142518E0,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"split_indices":[67,56,53,67,7,48,9,46,71,2,9,0,47,0,8,69,1,5,0,0,0,48,0,44,9,0,47,0,0,57,0,48,0,0,0,0,0,8,0,0,0,73,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.81E2,6E1,1.04E2,7.7E1,3.4E1,2.6E1,6.3E1,4.1E1,1.5E1,6.2E1,2E0,3.2E1,4E0,2.2E1,1.5E1,4.8E1,3.1E1,1E1,1.3E1,2E0,4.5E1,1.7E1,9E0,2.3E1,1.1E1,1.1E1,2E0,1.3E1,4.4E1,4E0,2.2E1,9E0,2.2E1,2.3E1,2E0,7E0,1.9E1,4E0,9E0,2E0,3.2E1,1.2E1,5E0,1.7E1,7E0,1.2E1,2.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[6.0046404E-3,-9.218389E-2,2.1822302E-1,-1.7119181E-1,2.2936372E-2,1.080785E-1,3.675177E-1,-2.2839919E-1,-7.109087E-2,4.932593E-2,-3.4302478E-3,1.885684E-2,8.236004E-2,1.287323E-1,4.4497648E-1,-2.0051514E-1,-2.1602163E-2,4.6547647E-3,-1.2568788E-1,-1.1251967E-2,4.7163595E-3,1.5830024E-1,1.2744217E-2,9.789093E-3,3.2283113E-4,3.5435382E-1,3.0562386E-2,-1.4268523E-2,-1.7262784E-1,-6.920345E-3,9.3187834E-4,-4.0626727E-2,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,5.3271845E-2,1.711268E-2,5.001436E-3,-1.4809307E-1,-1.3950878E-2,-6.9640996E-3,7.891728E-3,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,29,31,-1,33,35,-1,-1,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8752337E0,1.4591926E0,1.1771345E0,5.277815E-1,1.8073346E-1,3.3109027E-1,5.1609373E-1,3.4681034E-1,3.6278534E-1,1.8984374E-1,0E0,0E0,2.2002146E-1,1.0272367E-1,2.8410387E-1,1.4103603E-1,0E0,0E0,1.3881755E-1,1.4883459E-1,0E0,1.2233335E-1,2.0214176E-1,0E0,0E0,1.2719083E-1,0E0,0E0,1.2584579E-1,0E0,0E0,1.397791E-1,0E0,0E0,0E0,0E0,9.9094376E-2,0E0,0E0,9.3803644E-2,0E0,0E0,9.208519E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,19,19,21,21,22,22,25,25,28,28,31,31,36,36,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,30,32,-1,34,36,-1,-1,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6984555E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.260197E0,8.9397516E4,2.2324745E5,5.6835636E-2,8.938651E4,1.7573398E2,-3.4302478E-3,1.885684E-2,1.3844932E7,2.81002E5,1.2461E4,1.7565872E-3,-2.1602163E-2,4.6547647E-3,1.7725672E8,2.424E3,4.7163595E-3,1.8990228E0,1.312252E6,9.789093E-3,3.2283113E-4,4.7201157E-1,3.0562386E-2,-1.4268523E-2,1.269689E1,-6.920345E-3,9.3187834E-4,8E0,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,2.0215728E0,1.711268E-2,5.001436E-3,8.938651E4,-1.3950878E-2,-6.9640996E-3,2.176974E8,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"split_indices":[67,56,53,67,68,48,43,72,48,67,0,0,60,9,2,73,0,0,12,2,0,68,44,0,0,53,0,0,71,0,0,3,0,0,0,0,50,0,0,48,0,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.59E2,7.3E1,9.4E1,6.5E1,4.3E1,3E1,5.9E1,3.5E1,5.2E1,1.3E1,2E0,4.1E1,8E0,2.2E1,5.5E1,4E0,8E0,2.7E1,2.6E1,2.6E1,1.9E1,2.2E1,4E0,4E0,1.8E1,4E0,8E0,4.7E1,2.2E1,5E0,2.3E1,3E0,1.3E1,6E0,3E0,1.9E1,1.5E1,3E0,4.2E1,5E0,6E0,1.7E1,1.6E1,3E0,1.7E1,2.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.3128728E-2,-7.6841384E-2,2.5555712E-1,-1.3273934E-1,6.3761264E-2,8.655129E-2,4.008202E-1,-1.6426608E-1,-1.455439E-2,1.4814147E-2,3.6547862E-2,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,-1.4727283E-1,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,5.5398457E-2,-4.1562365E-3,-1.9884297E-1,-8.877025E-2,-5.379106E-4,9.418675E-2,-2.5566745E-1,-8.6616755E-2,-7.036469E-4,-1.4556472E-1,5.3083412E-3,-5.8485573E-4,-2.0745803E-1,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,-7.540202E-2,-9.3988655E-3,-2.2761422E-1,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,-1,25,27,-1,29,31,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,-1,43,-1,-1,-1,-1,-1],"loss_changes":[4.1965976E0,1.5646251E0,1.1191728E0,5.266993E-1,4.1526425E-1,1.3371919E-1,5.893326E-1,3.3139777E-1,9.107072E-2,0E0,1.3694459E-1,0E0,0E0,0E0,0E0,2.9913235E-1,0E0,0E0,0E0,1.246268E-1,0E0,3.2860088E-1,2.5498408E-1,0E0,9.024802E-2,4.5705009E-1,9.878099E-2,1.712665E-1,1.3560718E-1,0E0,0E0,1.3404357E-1,0E0,0E0,0E0,0E0,0E0,1.554379E-1,0E0,9.5205545E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,31,31,37,37,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,-1,26,28,-1,30,32,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,-1,44,-1,-1,-1,-1,-1],"split_conditions":[1.4272063E0,3.165913E6,2.174459E5,7.087292E-2,2.207E3,7.002603E1,1.6507233E3,1E0,1.1721691E5,1.4814147E-2,3.605082E7,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,1.73507E5,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,1.462136E6,-4.1562365E-3,1.3738942E0,1.833E5,-5.379106E-4,1.9879582E7,1.0665042E8,1.2288136E1,1.1942446E0,1.2271346E1,5.3083412E-3,-5.8485573E-4,8.672049E9,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,9.3E4,-9.3988655E-3,6.7001674E-3,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"split_indices":[57,1,43,53,0,71,4,85,43,0,60,0,0,0,0,9,0,0,0,47,0,68,44,0,47,7,71,68,71,0,0,5,0,0,0,0,0,12,0,73,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.97E2,4.6E1,1.41E2,5.6E1,2.2E1,2.4E1,1.11E2,3E1,4E0,5.2E1,1.9E1,3E0,1.6E1,8E0,1.04E2,7E0,5E0,2.5E1,4.6E1,6E0,5.4E1,5E1,1.7E1,2.9E1,3.5E1,1.9E1,2E1,3E1,2.3E1,6E0,3.2E1,3E0,1E1,9E0,3E0,1.7E1,1.6E1,1.4E1,2.9E1,3E0,2E0,1.4E1,8E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.4699904E-3,-8.2270786E-2,1.7413537E-1,-1.3975397E-1,3.89982E-3,7.456945E-2,3.0463025E-1,-1.7296548E-1,6.0730726E-3,-9.173903E-2,3.4571238E-2,1.801554E-2,4.9235556E-2,1.7605738E-3,3.2971406E-1,-2.1693262E-1,-1.0940522E-1,7.686381E-2,-3.4685563E-3,-2.7480236E-1,-8.64333E-4,1.01417145E-5,4.4262665E-3,-7.801436E-2,7.197917E-2,8.4248185E-3,3.7063202E-1,-1.9041932E-1,-2.0984167E-2,-4.680374E-2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,2.399795E-2,3.9364552E-1,4.087419E-3,-1.0795465E-3,-2.0910926E-1,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,5.9234392E-2,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5455477E0,8.9215004E-1,9.777329E-1,5.2410936E-1,2.1730821E-1,3.6530188E-1,2.0546484E-1,2.22188E-1,1.3329768E-1,2.3158492E-1,1.2854296E-1,0E0,1.3006273E-1,0E0,9.619784E-2,2.65311E-1,1.5102664E-1,2.2796245E-1,0E0,1.5092689E-1,0E0,0E0,0E0,1.5338422E-1,1.3331974E-1,0E0,1.03357315E-1,1.4346242E-1,0E0,1.5426993E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3046479E-1,9.462261E-2,0E0,0E0,1.1954689E-1,0E0,0E0,0E0,9.6233115E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,19,19,23,23,24,24,26,26,27,27,29,29,38,38,39,39,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.8509178E-1,1.8829846E-1,2.219883E6,1.8681856E8,8.9397516E4,1E0,5.283708E0,1.8000048E10,8E0,2.069532E6,1.801554E-2,7.5E2,1.7605738E-3,2.2324745E5,5.144033E-2,1.2288136E1,3.131673E6,-3.4685563E-3,3.56E2,-8.64333E-4,1.01417145E-5,4.4262665E-3,3.13E2,8.312613E5,8.4248185E-3,1.8373182E8,6E0,-2.0984167E-2,6.0515753E2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,6.925995E6,1.7598395E3,4.087419E-3,-1.0795465E-3,7.5646E4,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,2.8812434E7,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"split_indices":[67,54,53,1,7,48,27,71,5,3,9,0,0,0,43,72,71,44,0,0,0,0,0,0,62,0,47,10,0,4,0,0,0,0,0,0,0,0,66,4,0,0,9,0,0,0,60,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.79E2,7.6E1,1.07E2,7.2E1,4.4E1,3.2E1,8.7E1,2E1,1.7E1,5.5E1,2E0,4.2E1,3E0,2.9E1,5E1,3.7E1,1.1E1,9E0,4E0,1.3E1,3.7E1,1.8E1,6E0,3.6E1,8E0,2.1E1,4.7E1,3E0,2E1,1.7E1,7E0,4E0,2E0,2E0,2E0,4E0,1.3E1,2.3E1,1.9E1,2E0,5E0,4.2E1,9E0,1.1E1,4E0,1.9E1,1.4E1,5E0,2.4E1,1.8E1,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.0756515E-4,-9.093915E-2,1.3531232E-1,-1.3121241E-1,3.0161556E-2,6.471361E-2,2.7240935E-1,-1.5608981E-1,-4.138077E-2,5.6453147E-3,-1.665546E-3,1.331529E-2,4.4281885E-2,1.4165652E-4,2.9921746E-1,-1.3490759E-1,-3.431053E-1,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,1.077188E-1,-1.1128485E-2,1.87167E-2,1.0148881E-2,-1.18122905E-1,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,5.707247E-2,-4.600261E-3,3.8549118E-2,-1.6461147E-1,-4.9828332E-2,6.6000526E-3,6.4204754E-3,4.2778063E-3,-2.1197742E-3,-2.1401511E-1,-2.0505309E-3,7.0574425E-2,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,35,-1,37,39,41,-1,43,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9934402E0,7.1114016E-1,9.309007E-1,2.3713171E-1,1.1703685E-1,3.1068093E-1,2.37149E-1,2.9043174E-1,1.1233303E-1,0E0,1.0321621E-1,0E0,2.1913269E-1,0E0,1.5956187E-1,2.0051861E-1,2.3705983E-1,0E0,0E0,0E0,0E0,2.0704418E-1,1.6331983E-1,0E0,0E0,2.2356677E-1,0E0,0E0,0E0,0E0,1.08736545E-1,0E0,1.2394792E-1,2.3981667E-1,1.24148294E-1,0E0,9.681947E-2,0E0,0E0,2.0636463E-1,0E0,8.820005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,21,21,22,22,25,25,30,30,32,32,33,33,34,34,36,36,39,39,41,41],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,36,-1,38,40,42,-1,44,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.5343175E0,2.2348747E2,1.0632315E6,8.9397516E4,1E0,2.8659306E0,2.2616908E2,5.6453147E-3,7.432E3,1.331529E-2,1.1572553E7,1.4165652E-4,3.2307692E0,3.0962145E0,1.020435E4,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,4.0012E4,1.4813267E3,1.87167E-2,1.0148881E-2,1.73507E5,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,7E0,-4.600261E-3,8.303E3,2.951829E9,4.4E1,6.6000526E-3,1.9816086E0,4.2778063E-3,-2.1197742E-3,1.42E2,-2.0505309E-3,7.510128E7,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"split_indices":[67,53,56,67,63,48,27,73,67,0,10,0,60,0,69,68,47,0,0,0,0,9,70,0,0,9,0,0,0,0,3,0,0,5,10,0,68,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.44E2,9.7E1,1.08E2,3.6E1,6.5E1,3.2E1,8.4E1,2.4E1,8E0,2.8E1,4E0,6.1E1,3E0,2.9E1,7.7E1,7E0,2E0,2.2E1,2.4E1,4E0,2.8E1,3.3E1,8E0,2.1E1,7.2E1,5E0,3E0,4E0,6E0,2.2E1,1.1E1,2.2E1,4.2E1,3E1,7E0,1.5E1,1.3E1,9E0,2.9E1,1.3E1,6E0,2.4E1,1.1E1,4E0,6E0,2.3E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.0671469E-2,-4.057193E-2,2.5710663E-1,-9.11908E-2,7.657581E-2,1.0005701E-1,3.5594457E-1,-1.3330418E-1,3.5337575E-2,6.0731232E-2,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,-1.425013E-1,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,-2.7416292E-1,-1.2234801E-1,-3.3937553E-3,-2.139736E-2,-1.513492E-1,-3.387177E-2,-1.1739622E-1,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,-8.5462265E-2,-4.7841365E-3,7.8635005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,27,29,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[3.1548471E0,1.2344693E0,6.318512E-1,7.7734864E-1,2.7344996E-1,1.175154E-1,3.9797258E-1,1.8897259E-1,1.1001776E-1,1.5682302E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.4241185E-1,0E0,0E0,0E0,0E0,5.137621E-1,2.3093772E-1,0E0,0E0,1.8179822E-1,9.140951E-2,1.8776286E-1,0E0,0E0,0E0,0E0,1.0951534E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,16,16,21,21,22,22,25,25,26,26,27,27,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,28,30,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.5648934E0,3.165913E6,2.174459E5,6.408734E-2,1.4122614E2,1E0,1.7598395E3,1E1,1.599097E0,1.25440235E1,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,1.0187571E5,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,8.3E1,1.1670399E-1,-3.3937553E-3,-2.139736E-2,3.671916E0,1E0,1E0,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,1.8E1,-4.7841365E-3,7.8635005E-4],"split_indices":[56,1,43,53,71,8,4,10,68,72,0,0,0,0,0,0,43,0,0,0,0,10,54,0,0,69,29,104,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,2.06E2,4.2E1,1.44E2,6.2E1,1.7E1,2.5E1,1.08E2,3.6E1,6E1,2E0,7E0,1E1,1.9E1,6E0,5E0,1.03E2,2.2E1,1.4E1,5E1,1E1,1.2E1,9.1E1,7E0,5E0,6.8E1,2.3E1,5.1E1,1.7E1,2E0,2.1E1,9E0,4.2E1,3.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.4315777E-2,-1.0690338E-1,1.1009746E-1,-1.4777888E-1,2.0821875E-2,2.117857E-2,1.7040308E-1,-1.806778E-1,-5.4927632E-2,2.9963165E-4,6.637665E-3,3.901988E-2,-4.5500887E-3,7.291719E-2,2.6567936E-1,-1.4094864E-1,-2.987772E-1,-2.4235064E-4,-2.5972742E-1,8.2612615E-3,-1.8001318E-2,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,9.182606E-2,1.6551235E-1,2.0925388E-2,-2.1879025E-1,-9.1945045E-2,-3.4581E-1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,-1,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9952228E0,7.809812E-1,5.9160745E-1,3.3525777E-1,9.964019E-2,1.07021526E-1,5.936415E-1,3.473227E-1,3.1202346E-1,1.1755685E-1,0E0,9.129217E-2,0E0,1.14367574E-1,6.274619E-1,2.2401261E-1,1.6717279E-1,0E0,9.88501E-2,0E0,1.3668847E-1,0E0,0E0,0E0,1.5995112E-1,2.5624245E-1,0E0,1.0733318E-1,1.4268285E-1,2.0528078E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,18,18,20,20,24,24,25,25,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,-1,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.1677386E-1,2.796589E6,3.4021906E2,1.1670399E-1,5.5446387E2,1.9E1,2.174459E5,5.740651E-3,3.4047124E-1,2.207E3,6.637665E-3,2.0358758E8,-4.5500887E-3,1.9241573E-1,7.2259143E2,4.949154E-4,5.906883E5,-2.4235064E-4,2.07298E5,8.2612615E-3,1E1,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,8.303E3,1.9816086E0,2.0925388E-2,1.7565872E-3,4.8831225E5,7.5E1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"split_indices":[57,1,67,54,67,3,43,53,57,0,0,7,0,72,67,54,48,0,1,0,3,0,0,0,0,68,0,73,43,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.48E2,1.1E2,1.12E2,3.6E1,4.5E1,6.5E1,8.2E1,3E1,3.2E1,4E0,4E1,5E0,3.3E1,3.2E1,6.3E1,1.9E1,2.5E1,5E0,2E0,3E1,1E1,3E1,3E0,3E1,2.3E1,9E0,2.3E1,4E1,1.5E1,4E0,3E0,2E0,1E1,2E1,2E1,1E1,1.8E1,5E0,8E0,1.5E1,3.1E1,9E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.5204665E-2,-6.9504075E-2,1.5126899E-1,-1.1839543E-1,2.2287078E-2,7.562342E-2,2.5493225E-1,-1.4351347E-1,-1.2403568E-2,-5.980414E-3,3.6651608E-2,5.731069E-2,9.66742E-3,6.0085384E-3,3.3954874E-1,-1.3105348E-2,-1.2924905E-1,3.58532E-2,-7.821619E-3,1.6858906E-2,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,-9.744661E-2,-1.9808635E-1,4.93121E-3,-2.631804E-3,3.7851118E-2,-3.5518215E-3,-1.1887726E-1,1.7840436E-3,-5.2316664E-3,-2.911272E-1,6.4006194E-2,-1.814889E-3,-1.4108726E-1,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.0247637E-1,-1.5689014E-1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1],"loss_changes":[2.3878818E0,8.9570975E-1,4.904294E-1,3.4431982E-1,1.6523217E-1,9.151134E-2,2.2111058E-1,1.9603395E-1,2.1574554E-1,0E0,1.2080229E-1,1.1728562E-1,0E0,0E0,9.910309E-2,0E0,1.9523811E-1,1.6260572E-1,0E0,1.1651751E-1,0E0,0E0,0E0,0E0,0E0,2.094084E-1,1.8337464E-1,0E0,0E0,9.8238766E-2,0E0,1.4792949E-1,0E0,0E0,2.5384462E-1,1.126506E-1,0E0,1.1237788E-1,0E0,0E0,0E0,0E0,8.794698E-2,1.0356009E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,16,16,17,17,19,19,25,25,26,26,29,29,31,31,34,34,35,35,37,37,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1],"split_conditions":[1.2185057E0,1.781973E6,2.0787016E5,6.808926E6,9.315286E1,4.328683E0,3.778E3,3.4903356E7,1.409674E7,-5.980414E-3,4.43001E5,1.1832633E0,9.66742E-3,6.0085384E-3,4.7201157E-1,-1.3105348E-2,4.352025E0,4.8593006E5,-7.821619E-3,1.939974E7,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,3.9376082E0,8.287425E6,4.93121E-3,-2.631804E-3,1.25440235E1,-3.5518215E-3,2.2E1,1.7840436E-3,-5.2316664E-3,4.752366E0,5.6024827E9,-1.814889E-3,2.427E3,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.1770233E0,4.2E1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"split_indices":[56,1,43,9,67,50,2,7,60,0,43,54,0,0,53,0,69,48,0,47,0,0,0,0,0,69,60,0,0,72,0,3,0,0,71,5,0,2,0,0,0,0,69,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.62E2,1.98E2,6.4E1,1.29E2,6.9E1,3.8E1,2.6E1,1.04E2,2.5E1,5E0,6.4E1,3.5E1,3E0,1.2E1,1.4E1,7E0,9.7E1,2E1,5E0,5.4E1,1E1,8E0,2.7E1,1.2E1,2E0,6.8E1,2.9E1,1.1E1,9E0,4.5E1,9E0,5.9E1,9E0,1.7E1,1.2E1,3.4E1,1.1E1,4.9E1,1E1,4E0,8E0,1.1E1,2.3E1,4.4E1,5E0,3E0,2E1,3.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.1777132E-3,-6.0567748E-2,1.9037953E-1,-1.3071354E-1,5.439912E-4,8.848939E-2,2.9270196E-1,-1.7778724E-1,-1.4740403E-3,-7.950044E-2,3.19054E-2,3.1027812E-4,1.2356255E-1,2.2399522E-2,2.3053867E-1,-1.3397929E-1,-2.7813753E-1,1.4052213E-4,-1.3808084E-1,7.668798E-2,-2.3035506E-2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.975542E0,8.102633E-1,6.309147E-1,3.9554834E-1,2.5858772E-1,9.331095E-2,3.067577E-1,2.2210789E-1,0E0,1.4097291E-1,1.838358E-1,0E0,1.22035086E-1,0E0,9.3972206E-2,2.5386703E-1,9.3033075E-2,0E0,1.3659179E-1,1.5234712E-1,1.16067775E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,1.9415865E2,2.174459E5,1.848519E-2,8.461558E10,5.958114E4,1.8681E4,1.607E3,-1.4740403E-3,1.2271346E1,2.305287E1,3.1027812E-4,1.718354E8,2.2399522E-2,3.778E3,6.581686E6,6.608892E2,1.4052213E-4,4.6955062E5,1.2119408E3,6.657047E2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"split_indices":[57,67,43,53,46,43,9,2,0,71,71,0,7,0,2,9,4,0,48,4,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.88E2,6.2E1,8.7E1,1.01E2,3.2E1,3E1,5.8E1,2.9E1,2.8E1,7.3E1,1E1,2.2E1,5E0,2.5E1,4.2E1,1.6E1,1.2E1,1.6E1,4E1,3.3E1,6E0,1.6E1,1.1E1,1.4E1,4E1,2E0,1.2E1,4E0,4E0,1.2E1,3E1,1E1,2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.9849198E-3,-6.908789E-2,1.1069185E-1,-1.1135524E-1,4.8647843E-2,1.8852478E-2,1.622326E-1,-9.491666E-2,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,5.3762157E-2,2.2892316E-1,6.1087083E-2,-1.4048992E-1,-5.2776527E-2,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,-1.5545653E-1,-1.0782661E-3,-8.035333E-3,-2.0769975E-1,-9.872578E-2,-2.4276859E-1,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1],"loss_changes":[1.8784683E0,7.269919E-1,4.5608807E-1,2.1379781E-1,3.8304985E-1,1.0936768E-1,4.042337E-1,1.7887038E-1,0E0,0E0,0E0,0E0,9.703164E-2,2.1851015E-1,1.605283E-1,2.029301E-1,1.871668E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.09901786E-1,0E0,0E0,1.7119968E-1,1.0259637E-1,1.2492943E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,14,14,15,15,16,16,24,24,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1],"split_conditions":[4.259334E-1,3.343327E6,3.1259552E2,3.7979183E0,2.207E3,2.1123085E8,1.0568718E7,1.3466058E0,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,1.6070912E0,1.2461E4,1.25440235E1,4.6376264E9,2.2733334E1,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,6.7329E4,-1.0782661E-3,-8.035333E-3,8.640031E9,1.1710652E0,4.19891E0,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"split_indices":[57,1,67,68,0,7,60,68,0,0,0,0,73,2,72,46,71,0,0,0,0,0,0,0,9,0,0,5,68,71,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.44E2,9.6E1,1.06E2,3.8E1,3.5E1,6.1E1,9.6E1,1E1,3E0,3.5E1,1E1,2.5E1,3.6E1,2.5E1,4.5E1,5.1E1,3E0,2.2E1,3.2E1,4E0,2E1,5E0,2E0,4.3E1,4.3E1,8E0,2.1E1,2.2E1,1.8E1,3E0,8E0,1.4E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0547296E-2,-4.2344254E-2,1.742515E-1,-1.0237627E-1,3.5556585E-2,2.740988E-1,7.519926E-2,-7.69858E-2,-2.2364104E-1,7.678524E-3,2.2331232E-2,1.4626542E-1,3.7681144E-1,4.2196088E-2,8.162208E-3,-4.9231295E-2,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,4.149654E-2,-7.011419E-4,9.022605E-3,4.7000867E-1,6.24682E-3,-3.9323145E-3,3.069897E-3,-7.009031E-2,6.090464E-2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.163362E0,8.8750696E-1,5.826844E-1,3.122326E-1,1.5018767E-1,3.3878827E-1,1.1022253E-1,2.4382734E-1,2.6022464E-1,0E0,1.0540541E-1,1.5061551E-1,2.8114796E-1,1.0317968E-1,0E0,1.6967641E-1,0E0,0E0,0E0,0E0,1.09696545E-1,0E0,0E0,9.1567755E-2,0E0,0E0,0E0,1.7041293E-1,1.1754744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,20,20,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.5971466E2,2.7558826E-2,9.8110194E5,1.2460274E-2,2.3755E4,7.116346E2,2.0787016E5,3.774648E0,5.906883E5,7.678524E-3,1.496E3,2.4894228E6,2.660842E6,1.5E1,8.162208E-3,3.901529E6,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,1.6815423E5,-7.011419E-4,9.022605E-3,5.505334E9,6.24682E-3,-3.9323145E-3,3.069897E-3,2.797224E2,2.1374529E2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"split_indices":[67,53,48,53,9,67,43,69,48,0,2,47,44,3,0,9,0,0,0,0,43,0,0,5,0,0,0,67,67,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,1.88E2,6E1,1.06E2,8.2E1,2.9E1,3.1E1,8.9E1,1.7E1,6E0,7.6E1,1.4E1,1.5E1,2.5E1,6E0,7.1E1,1.8E1,8E0,9E0,1.6E1,6E1,4E0,1E1,1E1,5E0,4E0,2.1E1,6E1,1.1E1,2.2E1,3.8E1,6E0,4E0,5.4E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.9339443E-3,-1.0605051E-1,7.4423306E-2,-1.4161547E-1,-2.8023109E-2,5.737777E-2,3.6554796E-1,-1.08674295E-1,-2.4172173E-1,-5.5312406E-2,5.1337723E-3,2.0626027E-2,1.1801036E-1,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,-2.894412E-1,4.924759E-4,-5.1691607E-3,-2.6259364E-3,4.9695414E-2,4.6624467E-4,1.5626377E-1,-1.5225716E-2,-4.396114E-3,1.359324E-1,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.0659106E0,3.0406094E-1,7.134572E-1,2.243501E-1,1.496107E-1,3.117638E-1,1.7670429E-1,1.5272987E-1,2.0493579E-1,1.3169232E-1,0E0,2.1203475E-1,2.1592683E-1,0E0,0E0,0E0,0E0,0E0,1.3315952E-1,0E0,0E0,0E0,1.3145865E-1,0E0,1.2719214E-1,0E0,0E0,8.859834E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,18,18,22,22,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.7613442E-1,2.8704E4,4.328683E0,5.1653096E-3,1.22E2,1.2446029E0,2.174459E5,1.0665042E8,3.7274265E2,2.8488008E11,5.1337723E-3,1.805E3,1.7468556E2,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,2.1710082E7,4.924759E-4,-5.1691607E-3,-2.6259364E-3,1.325E0,4.6624467E-4,1.1560919E3,-1.5225716E-2,-4.396114E-3,8.8356856E5,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"split_indices":[56,11,50,53,8,57,43,7,4,46,0,2,67,0,0,0,0,0,47,0,0,0,68,0,67,0,0,62,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.1E2,1.47E2,7.5E1,3.5E1,1.4E2,7E0,5.8E1,1.7E1,3E1,5E0,8.8E1,5.2E1,4E0,3E0,5.3E1,5E0,3E0,1.4E1,1.5E1,1.5E1,2.3E1,6.5E1,1.4E1,3.8E1,1E1,4E0,1.3E1,5.2E1,3.5E1,3E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.345121E-3,-6.562803E-2,1.1551625E-1,-8.816371E-2,4.981665E-4,3.869827E-2,2.208219E-1,-1.4935678E-1,-4.5134716E-2,1.6114242E-2,6.035789E-3,2.2871085E-2,1.6961017E-1,-2.867197E-1,-1.2161767E-1,-6.569096E-2,3.5917396E-3,-3.0824807E-3,5.086014E-2,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,-1.0346314E-1,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.6731131E0,2.8574848E-1,5.9006786E-1,3.276595E-1,0E0,9.675721E-2,3.9257836E-1,1.6326678E-1,2.0450398E-1,1.1511861E-1,0E0,0E0,1.3534534E-1,3.4041297E-1,1.645894E-1,9.34082E-2,0E0,0E0,1.10176116E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0852566E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[3.7136658E2,9.1318935E-2,1.8829846E-1,4.426498E0,4.981665E-4,1E0,1.431E4,9.975593E4,6.767991E5,1.312252E6,6.035789E-3,2.2871085E-2,1.1207349E0,2.83E2,5.22241E5,1.7012288E8,3.5917396E-3,-3.0824807E-3,8.303E3,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,2.22105E5,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"split_indices":[67,53,53,71,0,30,9,43,43,44,0,0,68,0,12,7,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.63E2,7.3E1,1.26E2,3.7E1,4.3E1,3E1,5.1E1,7.5E1,3.6E1,7E0,3E0,2.7E1,7E0,4.4E1,6.5E1,1E1,1E1,2.6E1,3E0,2.4E1,4E0,3E0,2.6E1,1.8E1,3.3E1,3.2E1,1.7E1,9E0,2E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.5972516E-3,-6.415109E-2,1.004628E-1,-9.126678E-2,7.985584E-5,7.822119E-2,3.8027394E-1,-1.3547492E-1,-1.7177572E-2,1.2492152E-1,-1.8076582E-2,2.4130752E-2,6.7940415E-3,-2.0095895E-1,-7.504207E-2,3.5000306E-3,-4.5314096E-2,2.2494642E-1,8.771981E-2,-6.7835734E-3,8.8194094E-4,-1.3448668E-1,-3.6327535E-1,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,-8.5903674E-2,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[1.4975804E0,2.8284806E-1,4.9924803E-1,3.645115E-1,0E0,3.6202925E-1,1.4661348E-1,2.6193678E-1,1.21453926E-1,1.8120694E-1,1.4514734E-1,0E0,0E0,3.07199E-1,2.7256477E-1,0E0,1.0324016E-1,1.1432767E-1,8.934039E-2,0E0,0E0,1.8839267E-1,1.3768709E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.900878E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,17,17,18,18,21,21,22,22,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[3.4021906E2,5.530166E-2,2.069929E3,1.6408995E11,7.985584E-5,1.8018788E7,5.1145615E3,1.386452E0,2.6017979E11,8E0,1.628E3,2.4130752E-2,6.7940415E-3,1.609E3,3.9069E5,3.5000306E-3,1E0,4.0123675E-2,4.0012E4,-6.7835734E-3,8.8194094E-4,1.393E3,4.7201145E10,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,1.1710652E0,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"split_indices":[67,53,67,46,0,60,4,68,46,3,0,0,0,2,44,0,103,53,9,0,0,12,46,0,0,0,0,0,0,0,0,0,68,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.57E2,8.4E1,1.11E2,4.6E1,7.9E1,5E0,6.9E1,4.2E1,5.3E1,2.6E1,2E0,3E0,3.2E1,3.7E1,9E0,3.3E1,1.3E1,4E1,5E0,2.1E1,2.4E1,8E0,3.1E1,6E0,2.5E1,8E0,2E0,1.1E1,9E0,3.1E1,4E0,2E1,4E0,4E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.0949485E-4,-7.023364E-2,1.07531965E-1,-9.5697366E-2,5.999637E-3,4.481289E-2,2.2272988E-1,-1.2741327E-1,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,3.131387E-2,6.709275E-3,1.5677929E-2,-1.9639476E-1,-7.4869595E-2,-1.0507656E-2,5.4045306E-3,-2.6656508E-1,-3.303782E-3,-3.9352458E-2,-1.0309419E-2,-5.7465717E-2,3.690726E-2,-8.487278E-3,-2.1258341E-2,-5.933776E-2,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,19,21,23,-1,25,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9060686E0,2.956339E-1,7.1252954E-1,3.6945975E-1,9.6911065E-2,2.3330902E-1,2.7442348E-1,3.0063093E-1,0E0,0E0,0E0,0E0,2.4897242E-1,0E0,0E0,2.9316032E-1,2.830122E-1,1.0231397E-1,0E0,2.7091873E-1,0E0,9.987865E-2,0E0,1.0716895E-1,1.2748241E-1,0E0,0E0,1.180381E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,15,15,16,16,17,17,19,19,21,21,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,20,22,24,-1,26,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.2711891E0,1.769817E6,9.750871E6,3.4903356E7,1.6507233E3,7.5646E4,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,2.8041378E5,6.709275E-3,1.5677929E-2,1.3738942E0,3.0962145E0,1.0685258E6,5.4045306E-3,6.4146477E-3,-3.303782E-3,2.0766662E6,-1.0309419E-2,2.0250492E0,2.51838E6,-8.487278E-3,-2.1258341E-2,4.6376264E9,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"split_indices":[67,53,54,1,9,7,4,9,0,0,0,0,43,0,0,68,68,48,0,53,0,63,0,72,47,0,0,46,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.51E2,9.9E1,1.13E2,3.8E1,6.5E1,3.4E1,8.6E1,2.7E1,2.8E1,1E1,2E0,6.3E1,2.4E1,1E1,3.6E1,5E1,4.4E1,1.9E1,2.2E1,1.4E1,4.2E1,8E0,2.2E1,2.2E1,1.8E1,4E0,3.6E1,6E0,1.6E1,6E0,9E0,1.3E1,4E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.23015465E-2,-3.6813665E-2,1.6186102E-1,-8.2913704E-2,3.2519907E-2,3.2985918E-3,2.6083997E-1,-9.4713494E-2,1.0172514E-3,2.072838E-2,1.34672765E-2,7.684792E-3,4.2016974E-1,-1.21286646E-1,-3.079713E-2,8.215429E-2,-2.6565844E-2,8.788486E-3,2.4215858E-2,-3.9600227E-3,-1.8518592E-1,1.9243775E-3,-4.5282864E-3,6.782496E-3,1.653277E-2,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8129876E0,5.9720445E-1,5.011734E-1,1.4259148E-1,2.3776487E-1,0E0,3.040551E-1,1.6780907E-1,0E0,2.1425411E-1,0E0,0E0,1.535585E-1,1.2175703E-1,1.739935E-1,1.5128689E-1,8.827151E-2,0E0,0E0,0E0,9.1700196E-2,0E0,0E0,0E0,9.928738E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,12,12,13,13,14,14,15,15,16,16,20,20,24,24],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,2.427E3,2.2324745E5,6.4796996E-1,1.4122614E2,3.2985918E-3,1.7598395E3,2.0256741E2,1.0172514E-3,1.3844932E7,1.34672765E-2,7.684792E-3,2.4394053E9,1.607E3,7.336684E2,1.6254681E0,6.925995E6,8.788486E-3,2.4215858E-2,-3.9600227E-3,1.7842995E8,1.9243775E-3,-4.5282864E-3,6.782496E-3,4.488E3,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"split_indices":[57,2,43,54,71,0,4,67,0,60,0,0,5,2,4,68,66,0,0,0,7,0,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.85E2,6E1,1.11E2,7.4E1,3.3E1,2.7E1,1E2,1.1E1,7.2E1,2E0,1.9E1,8E0,7E1,3E1,3.1E1,4.1E1,4E0,4E0,4.9E1,2.1E1,1.5E1,1.5E1,1.4E1,1.7E1,2.5E1,1.6E1,1.3E1,8E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.4086864E-3,-5.6603998E-2,1.0975099E-1,-1.0331851E-1,7.5110937E-3,8.584382E-2,1.7369809E-2,-1.5483831E-1,-2.4599966E-2,-6.041252E-3,1.6805973E-2,4.8704322E-2,1.6501543E-1,-1.18140824E-1,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.352543E-2,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.7069752E-2,3.0004673E-3,-3.8697182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.617809E0,4.5603204E-1,6.2057626E-1,3.530805E-1,9.050379E-2,2.5365812E-1,0E0,3.3182883E-1,1.1731397E-1,0E0,1.0263331E-1,2.5914997E-1,2.4869812E-1,9.6805155E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2964478E-1,0E0,0E0,0E0,0E0,0E0,8.78565E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,20,20,26,26],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[3.4021906E2,1.1670399E-1,1.3821641E0,1.6796057E2,8.89E2,2.968068E5,1.7369809E-2,7.33452E-3,2.939566E5,-6.041252E-3,1.10018056E8,8.9397516E4,2.3842E4,6E0,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.0327642E-1,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.615313E3,3.0004673E-3,-3.8697182E-3],"split_indices":[67,54,49,67,2,43,0,53,48,0,47,48,9,10,0,0,0,0,0,0,57,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,8.7E1,6.4E1,8.8E1,6E0,5.2E1,3.5E1,3E0,6.1E1,6.1E1,2.7E1,4.5E1,7E0,1.7E1,1.8E1,5.5E1,6E0,2E0,5.9E1,5E0,2.2E1,5E0,4E1,1.2E1,4.7E1,4.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.5817066E-3,-3.926012E-2,1.1888153E-1,-7.5602576E-2,5.771031E-2,6.1283264E-2,8.008875E-3,-9.3968615E-2,8.966747E-3,1.21591035E-2,3.8098045E-2,8.102908E-3,1.4816463E-3,-1.2472329E-1,-4.609013E-2,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,-1.0609691E-1,-2.3297147E-1,-7.477302E-3,-1.8048729E-1,-1.9920397E-1,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0151035E0,6.5632963E-1,1.823703E-1,2.1052504E-1,2.1035415E-1,9.229251E-2,0E0,1.576975E-1,1.7606108E-1,0E0,1.1426413E-1,0E0,0E0,1.1037278E-1,2.3217589E-1,0E0,0E0,0E0,0E0,1.10173464E-1,9.862223E-2,1.0037966E-1,1.288043E-1,1.144903E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1427817E0,3.165913E6,2.174459E5,5.8536225E5,2.207E3,1.1832633E0,8.008875E-3,1.1670399E-1,3.2744165E2,1.21591035E-2,1E1,8.102908E-3,1.4816463E-3,1.4031231E-2,2.587E3,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,1.1408451E0,5.1607388E5,2.8507206E-1,7.12E2,7.5608E4,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"split_indices":[56,1,43,43,0,54,0,54,4,0,3,0,0,53,2,0,0,0,0,68,48,57,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,1.84E2,5.1E1,1.34E2,5E1,2.8E1,2.3E1,1.1E2,2.4E1,3E0,4.7E1,4E0,2.4E1,6.6E1,4.4E1,9E0,1.5E1,1.4E1,3.3E1,5.8E1,8E0,3.5E1,9E0,1E1,4.8E1,5E0,3E0,2E1,1.5E1,2E0,7E0,8E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-6.5358323E-3,-5.01191E-2,1.1463253E-1,-6.566863E-2,3.5350563E-3,6.801129E-2,2.2740725E-1,-9.271781E-2,1.6208226E-4,3.614062E-3,4.4614673E-3,2.6938426E-1,5.7348673E-4,-7.9707764E-2,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,-1.20486856E-1,-4.1228443E-2,-2.9383153E-1,-9.8548844E-2,3.4898527E-3,-6.404513E-2,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2573216E0,3.6213574E-1,3.1576872E-1,2.9499376E-1,0E0,1.0101345E-1,1.5835595E-1,1.3562757E-1,0E0,1.0294547E-1,0E0,1.21626854E-1,0E0,1.5508604E-1,0E0,0E0,0E0,0E0,0E0,1.5738219E-1,1.544519E-1,1.2428334E-1,2.1362385E-1,0E0,1.0190545E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,5.8536225E5,5.76E3,3.165913E6,3.5350563E-3,1.7468556E2,3.41212E7,1E0,1.6208226E-4,1.2495134E6,4.4614673E-3,2.9375613E-1,5.7348673E-4,1.73507E5,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,9.675635E4,1.2473118E0,9.48E2,4.43001E5,3.4898527E-3,1.8979991E6,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"split_indices":[57,43,2,1,0,67,66,92,0,60,0,53,0,9,0,0,0,0,0,43,68,0,43,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.74E2,6.2E1,1.56E2,1.8E1,4.5E1,1.7E1,1.12E2,4.4E1,1.6E1,2.9E1,1.4E1,3E0,1.01E2,1.1E1,9E0,7E0,6E0,8E0,4.8E1,5.3E1,4E0,4.4E1,8E0,4.5E1,2E0,2E0,3.8E1,6E0,3.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.814724E-3,-4.223491E-2,1.1547112E-1,-8.021347E-2,1.0189096E-3,1.424386E-1,-2.2660806E-3,-5.76489E-2,-2.370273E-1,-5.7016924E-2,1.174304E-3,-1.1189182E-3,1.5711717E-1,-3.7425077E-3,2.5416018E-2,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.0735268E-1,2.2521736E-1,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.3238825E-1,3.9426633E-3,2.2386394E-2,9.90202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.2799627E0,2.77701E-1,3.414572E-1,3.0479503E-1,1.2174119E-1,1.6192663E-1,0E0,1.8898878E-1,1.3618523E-1,2.1964492E-1,0E0,0E0,1.7577171E-1,0E0,9.3134895E-2,0E0,0E0,0E0,0E0,1.2113613E-1,3.1686854E-1,0E0,0E0,0E0,0E0,1.3124907E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,14,14,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[3.799535E2,1.1306552E-1,4.183787E0,1.4031231E-2,1.83437E5,1.2055675E0,-2.2660806E-3,4.239E3,1.8018788E7,1.5086387E8,1.174304E-3,-1.1189182E-3,7.3667456E2,-3.7425077E-3,1.7E1,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.57257E0,3.21879E6,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.0256798E0,3.9426633E-3,2.2386394E-2,9.90202E-3],"split_indices":[67,54,72,53,9,68,0,10,60,7,0,0,67,0,3,0,0,0,0,68,44,0,0,0,0,69,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.68E2,7.3E1,8.9E1,7.9E1,6.3E1,1E1,7.9E1,1E1,2.4E1,5.5E1,5E0,5.8E1,5.9E1,2E1,5E0,5E0,2E0,2.2E1,3.5E1,2.3E1,1.1E1,9E0,1.2E1,2.3E1,1.2E1,1.1E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.0411531E-3,-5.0822284E-2,8.184148E-2,-8.0308676E-2,-6.2485766E-3,1.5205893E-1,6.5722945E-3,-1.08080305E-1,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,2.7616316E-1,-1.5828831E-2,1.3877054E-2,-8.242552E-2,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,-1.7038673E-1,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[9.903048E-1,2.005888E-1,4.6938437E-1,2.0104021E-1,1.332165E-1,2.687695E-1,3.1198943E-1,1.609565E-1,0E0,0E0,0E0,0E0,1.3902038E-1,9.690636E-2,0E0,1.4167267E-1,0E0,0E0,0E0,0E0,0E0,1.059061E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,15,15,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[3.2769293E2,1.529448E-1,1.0568718E7,5.5015685E9,9.74979E6,7.115615E2,1.4122614E2,1.3244295E5,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,1.8373182E8,1.128757E6,1.3877054E-2,6.7001674E-3,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,2.647867E-4,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"split_indices":[67,54,60,5,9,67,71,62,0,0,0,0,47,44,0,73,0,0,0,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.52E2,8.8E1,9.1E1,6.1E1,4.5E1,4.3E1,6.7E1,2.4E1,4.7E1,1.4E1,3.3E1,1.2E1,4.1E1,2E0,5.5E1,1.2E1,1E1,2E0,1E1,3.1E1,1.3E1,4.2E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.606068E-4,-4.0152058E-2,1.3688791E-1,-7.5740196E-2,5.956878E-3,1.0230381E-1,3.1381202E-1,-9.340738E-2,3.2788247E-2,6.146593E-3,-6.0711894E-3,9.786554E-3,6.620768E-2,1.6971182E-2,1.4838147E-3,-6.620601E-2,-1.9074214E-1,5.796871E-3,-1.0807036E-3,-2.5402438E-2,2.655493E-3,-2.4867726E-3,8.962364E-2,-1.7166455E-1,-3.69803E-2,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,25,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.397829E0,3.1715095E-1,3.2278395E-1,2.120583E-1,1.3876921E-1,2.0526332E-1,1.7587048E-1,2.362498E-1,9.541947E-2,0E0,1.03748016E-1,0E0,1.20494455E-1,0E0,0E0,2.2480932E-1,2.1755302E-1,0E0,0E0,1.3877995E-1,0E0,0E0,1.2990308E-1,1.283262E-1,1.8078491E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,15,15,16,16,19,19,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,26,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.633287E2,2.0358758E8,1.9292948E3,3.402546E6,2.3755E4,2.02983E5,3.41212E7,1.2869189E5,1.8000048E10,6.146593E-3,1.9425332E7,9.786554E-3,1.9241573E-1,1.6971182E-2,1.4838147E-3,1E0,9.975593E4,5.796871E-3,-1.0807036E-3,4.327E3,2.655493E-3,-2.4867726E-3,3.3996282E0,1.9E1,1.081E3,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"split_indices":[67,7,67,1,9,44,66,62,5,0,9,0,72,0,0,104,43,0,0,11,0,0,72,3,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.92E2,5.7E1,1.08E2,8.4E1,4.9E1,8E0,9.3E1,1.5E1,6E0,7.8E1,1E1,3.9E1,6E0,2E0,7.4E1,1.9E1,5E0,1E1,6.1E1,1.7E1,6E0,3.3E1,1.5E1,5.9E1,3E0,1.6E1,1.6E1,4.5E1,2.1E1,1.2E1,6E0,9E0,2.2E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.4908163E-3,-4.9458664E-2,8.817324E-2,6.931918E-2,-6.1066084E-2,6.9811605E-2,3.090321E-1,-5.98609E-4,7.956278E-3,-8.879114E-3,-5.08166E-2,1.4328934E-1,3.887047E-2,1.7160837E-2,2.8655254E-3,-1.0146142E-1,-3.0556865E-2,1.2192971E-3,2.755345E-1,8.296684E-2,-6.053145E-4,-6.862668E-2,-1.2999088E-2,1.7750489E-3,-5.1321223E-2,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,-1.3914266E-1,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,17,19,-1,-1,21,23,-1,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[1.1073979E0,2.1379605E-1,3.6368233E-1,1.3386652E-1,1.9253713E-1,1.9677043E-1,1.176427E-1,0E0,0E0,0E0,1.3131228E-1,3.889135E-1,1.4999606E-1,0E0,0E0,2.181817E-1,1.4260218E-1,0E0,1.17243946E-1,9.568061E-2,0E0,1.2064642E-1,0E0,0E0,8.933057E-2,0E0,0E0,0E0,0E0,1.0965279E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,11,11,12,12,15,15,16,16,18,18,19,19,21,21,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,18,20,-1,-1,22,24,-1,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[3.2769293E2,1.1623532E6,1.3758811E0,1E0,1.7565872E-3,2.3842E4,6.901E3,-5.98609E-4,7.956278E-3,-8.879114E-3,4.0835E4,7.6411234E4,9.380036E6,1.7160837E-2,2.8655254E-3,2.9138212E0,1.217759E0,1.2192971E-3,8.7292106E5,1.9643908E5,-6.053145E-4,5.3091907E2,-1.2999088E-2,1.7750489E-3,3.0962145E0,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,1.0288912E5,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"split_indices":[67,60,49,8,73,9,0,0,0,0,9,43,60,0,0,73,68,0,48,43,0,4,0,0,68,0,0,0,0,48,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,1.3E1,1.38E2,8.8E1,6E0,8E0,5E0,8E0,1.3E2,2.5E1,6.3E1,4E0,2E0,3.6E1,9.4E1,1.4E1,1.1E1,3.4E1,2.9E1,3.2E1,4E0,2.1E1,7.3E1,3E0,8E0,6E0,2.8E1,1.3E1,1.9E1,6.4E1,9E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.576465E-3,-3.1257384E-2,1.1510225E-1,-6.0538724E-2,4.1183308E-2,1.9610284E-2,9.548955E-2,-1.616713E-1,-4.8437126E-2,1.07467435E-2,2.4615793E-2,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,-3.3256467E-2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,-4.011636E-2,4.8860135E-3,-9.835392E-2,-1.4123669E-2,7.2520383E-4,-1.8231265E-1,7.876769E-5,-6.9828774E-3,-2.5798842E-1,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[9.6316123E-1,4.0339252E-1,3.4389424E-1,1.5739092E-1,1.8081136E-1,0E0,1.22983694E-1,1.2277371E-1,1.5455526E-1,0E0,1.18286856E-1,0E0,0E0,0E0,0E0,1.0909271E-1,0E0,0E0,0E0,1.5149972E-1,0E0,3.022702E-1,1.6528234E-1,0E0,1.5847087E-1,0E0,0E0,1.461187E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.2185057E0,3.165913E6,1E0,6.7001674E-3,2.207E3,1.9610284E-2,2.0787016E5,2.647867E-4,2.0017123E0,1.07467435E-2,1E1,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,4.5971466E2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,4.0054E4,4.8860135E-3,1.8259804E-1,2.2912737E1,7.2520383E-4,1.3738942E0,7.876769E-5,-6.9828774E-3,1.622E3,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"split_indices":[57,1,13,73,0,0,43,54,68,0,3,0,0,0,0,67,0,0,0,9,0,73,71,0,68,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.88E2,5.8E1,1.34E2,5.4E1,2E0,5.6E1,1.3E1,1.21E2,3E0,5.1E1,3.2E1,2.4E1,9E0,4E0,1.04E2,1.7E1,1.5E1,3.6E1,1E2,4E0,3E1,7E1,1.3E1,1.7E1,6.4E1,6E0,1E1,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.720544E-3,-4.5713637E-2,8.041132E-2,4.5472946E-2,-5.675482E-2,4.4581052E-2,1.6859385E-1,-1.5774339E-3,5.743392E-3,-6.856326E-2,2.9137306E-4,8.072134E-3,3.0883143E-2,1.6059604E-2,5.5405004E-3,-6.1498176E-2,-1.2888393E-2,7.4021585E-2,-2.9413155E-3,-9.75898E-3,-5.4604717E-2,7.0612812E-3,1.7218302E-3,-2.8861985E-3,3.3276238E-2,-7.160593E-3,-4.491236E-2,-1.0041681E-3,3.9801714E-3,4.2815937E-4,-6.328989E-2,-7.997919E-3,-2.0749776E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,17,-1,-1,19,-1,21,23,-1,25,-1,-1,-1,27,-1,29,-1,-1,-1,31,-1,-1],"loss_changes":[9.9902195E-1,1.4655435E-1,3.463596E-1,1.1724831E-1,9.633365E-2,1.5005943E-1,2.1894133E-1,0E0,0E0,1.5487564E-1,0E0,0E0,1.096528E-1,0E0,0E0,1.0625979E-1,0E0,9.375355E-2,1.00620806E-1,0E0,1.0192835E-1,0E0,0E0,0E0,9.3399785E-2,0E0,9.716697E-2,0E0,0E0,0E0,1.2727156E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,12,12,15,15,17,17,18,18,20,20,24,24,26,26,30,30],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,18,-1,-1,20,-1,22,24,-1,26,-1,-1,-1,28,-1,30,-1,-1,-1,32,-1,-1],"split_conditions":[2.797224E2,1.1696849E6,1.5972575E0,2.25E2,4.343271E5,8.9397516E4,1.8426E4,-1.5774339E-3,5.743392E-3,2.9892595E6,2.9137306E-4,8.072134E-3,1.1572553E7,1.6059604E-2,5.5405004E-3,1.3525729E6,-1.2888393E-2,4.0012E4,1.4813267E3,-9.75898E-3,1.7565872E-3,7.0612812E-3,1.7218302E-3,-2.8861985E-3,5.415429E4,-7.160593E-3,1.1710652E0,-1.0041681E-3,3.9801714E-3,4.2815937E-4,1E0,-7.997919E-3,-2.0749776E-3],"split_indices":[67,60,56,0,62,48,9,0,0,66,0,0,60,0,0,60,0,9,70,0,73,0,0,0,43,0,68,0,0,0,15,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,1.5E1,1.27E2,8E1,3.1E1,8E0,7E0,1.07E2,2E1,6E0,7.4E1,4E0,2.7E1,1.05E2,2E0,3.2E1,4.2E1,3E0,1.02E2,8E0,2.4E1,1.5E1,2.7E1,7E0,9.5E1,1.4E1,1.3E1,2.4E1,7.1E1,7E0,6.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.432755E-3,-3.2541275E-2,1.2062466E-1,-6.6113114E-2,7.3056045E-4,7.963434E-2,2.6087046E-1,-2.4810357E-2,-1.07705206E-1,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,-1.1733579E-1,-3.5330087E-3,-1.7337382E-1,-5.0983664E-2,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,3.909637E-2,-2.709494E-1,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0041343E0,3.2042485E-1,2.954054E-1,1.9488263E-1,0E0,1.0474539E-1,2.4700874E-1,1.15738496E-1,2.0347583E-1,0E0,0E0,0E0,0E0,1.0116187E-1,1.1524878E-1,1.1706191E-1,1.1447847E-1,0E0,0E0,0E0,1.1847381E-1,1.464771E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15,16,16,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.3341238E9,2.2973125E3,1.19331E5,7.3056045E-4,2.0787016E5,2.174459E5,3.559166E7,6.6037946E2,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,4.3399894E5,1.09E3,7.359468E8,1.48068905E1,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,1.2765957E0,1.721649E8,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"split_indices":[57,5,4,44,0,43,43,7,4,0,0,0,0,43,2,5,71,0,0,0,68,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.92E2,5.4E1,1.14E2,7.8E1,4.3E1,1.1E1,5.8E1,5.6E1,2.1E1,2.2E1,6E0,5E0,1E1,4.8E1,2.5E1,3.1E1,5E0,5E0,2.1E1,2.7E1,8E0,1.7E1,1.5E1,1.6E1,7E0,2E1,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.4172934E-3,-4.873043E-2,5.479038E-2,2.9505575E-3,-7.196424E-2,1.9374692E-1,3.8230103E-2,-2.232737E-3,2.4846569E-3,-2.2767627E-1,-5.168247E-2,1.6559279E-3,3.4322336E-1,6.4282104E-2,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,9.1448665E-2,1.5415065E-2,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.809334E-1,1.707184E-1,2.5722635E-1,1.343776E-1,2.9772854E-1,2.6643854E-1,1.8172754E-1,0E0,0E0,1.076473E-1,2.4328896E-1,0E0,1.7182684E-1,9.815651E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.945521E-2,1.7858982E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.47887E5,1.431E4,1.081E3,2.07684E5,9.675635E4,1.8018788E7,-2.232737E-3,2.4846569E-3,1.3738942E0,9.476661E1,1.6559279E-3,9.63E2,2.1949912E7,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,7.3667456E2,1.25836E5,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"split_indices":[67,1,9,2,1,43,60,0,0,68,67,0,0,47,0,0,0,0,0,0,0,67,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.41E2,1.14E2,4.4E1,9.7E1,1.1E1,1.03E2,2.2E1,2.2E1,1E1,8.7E1,6E0,5E0,7.4E1,2.9E1,4E0,6E0,1.7E1,7E1,2E0,3E0,4.7E1,2.7E1,3.9E1,8E0,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.1405309E-3,-4.1295454E-2,7.539111E-2,-4.773554E-2,2.649901E-3,5.8098752E-2,2.4969846E-1,-3.5499748E-2,-1.1151391E-1,7.256275E-2,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,-2.639264E-2,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,-3.14371E-2,-4.9293704E-2,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1],"loss_changes":[7.971391E-1,1.0867721E-1,2.6279783E-1,1.1465886E-1,0E0,1.4647299E-1,1.3158771E-1,9.851079E-2,1.7702582E-1,1.16677344E-1,0E0,0E0,0E0,0E0,9.4192E-2,0E0,0E0,0E0,0E0,0E0,9.489479E-2,9.564981E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,14,14,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1],"split_conditions":[5.9570425E5,2.407794E7,1.3821641E0,2.9138212E0,2.649901E-3,1E0,3.41212E7,1.4781966E-3,3.67E2,2.3581433E5,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,1E1,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,3.3341238E9,5.77032E5,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"split_indices":[62,9,49,73,0,92,66,73,0,43,0,0,0,0,10,0,0,0,0,0,5,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.6E2,9.1E1,1.51E2,9E0,8.4E1,7E0,1.28E2,2.3E1,7.5E1,9E0,5E0,2E0,1.1E1,1.17E2,2E0,2.1E1,3.8E1,3.7E1,3E0,1.14E2,8.2E1,3.2E1,6.9E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.8670357E-3,-4.5970436E-2,5.2696485E-2,6.0541928E-2,-5.781559E-2,2.443929E-2,1.2535055E-1,-9.904153E-4,6.934784E-3,-7.859939E-3,-4.945917E-2,3.5898713E-3,3.4437096E-3,2.2750926E-1,1.946528E-3,-6.620565E-2,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,-1.1629289E-1,-4.547119E-2,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.025647E-1,1.8644026E-1,2.186605E-1,1.2642913E-1,1.2496975E-1,9.5626205E-2,2.3731667E-1,0E0,0E0,0E0,9.588456E-2,0E0,9.368942E-2,1.0582209E-1,0E0,8.921841E-2,0E0,0E0,0E0,0E0,0E0,1.2605882E-1,9.986128E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.1623532E6,6.657047E2,1E0,1.7565872E-3,1E1,2.45226E6,-9.904153E-4,6.934784E-3,-7.859939E-3,2.9073395E9,3.5898713E-3,5.1332E4,3.0256798E0,1.946528E-3,4.0054E4,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,2.8516157E0,2.704375E6,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"split_indices":[67,60,67,8,73,3,44,0,0,0,5,0,9,69,0,9,0,0,0,0,0,73,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.44E2,1.07E2,1.4E1,1.3E2,7.8E1,2.9E1,8E0,6E0,7E0,1.23E2,2E1,5.8E1,1.2E1,1.7E1,9E1,3.3E1,2.2E1,3.6E1,4E0,8E0,2.5E1,6.5E1,2.2E1,3E0,4.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.004087E-3,-2.8343314E-2,7.549351E-2,-5.5867407E-2,2.2657372E-2,1.4241742E-2,5.9626795E-2,-6.4280997E-3,-4.644786E-2,4.461195E-2,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,-5.3805247E-2,4.6029454E-3,4.787177E-5,-8.447469E-3,-4.588249E-2,-1.3219344E-3,-5.0460016E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,-1,-1,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[5.418513E-1,2.4271044E-1,2.6314437E-1,8.962074E-2,1.0539442E-1,0E0,1.0044208E-1,0E0,1.2327367E-1,1.2701635E-1,0E0,0E0,0E0,0E0,1.0074824E-1,0E0,0E0,0E0,9.979747E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,14,14,18,18],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,-1,-1,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[9.821357E-1,7.422E3,1.215951E5,9.87574E2,2.1949912E7,1.4241742E-2,2.216135E5,-6.4280997E-3,1.8E1,1.1572553E7,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,1.3525729E6,4.6029454E-3,4.787177E-5,-8.447469E-3,2.0134454E0,-1.3219344E-3,-5.0460016E-3],"split_indices":[56,10,48,48,47,0,43,0,10,60,0,0,0,0,60,0,0,0,68,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.71E2,7E1,1.11E2,6E1,3E0,6.7E1,9E0,1.02E2,4.7E1,1.3E1,3.8E1,2.9E1,4E0,9.8E1,1.9E1,2.8E1,4E0,9.4E1,7.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.1594397E-2,-1.194265E-2,9.3121335E-2,-3.9583016E-2,2.608233E-2,1.6468607E-2,7.527401E-2,-5.18759E-2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,4.0722504E-4,-1.7415832E-3,3.3587667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,-1,15,-1,-1],"loss_changes":[4.712434E-1,2.0177807E-1,2.5062546E-1,1.2383151E-1,1.3076647E-1,0E0,1.04620725E-1,1.0193059E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.9496724E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.3394321E0,2.427E3,1.431E4,9.74979E6,2.637815E1,1.6468607E-2,2.174459E5,2.0256741E2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,3.4234106E5,-1.7415832E-3,3.3587667E-3],"split_indices":[57,2,9,9,71,0,43,67,0,0,0,0,0,0,43,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.9E2,5.4E1,1.1E2,8E1,2E0,5.2E1,9.7E1,1.3E1,5.1E1,2.9E1,2.8E1,2.4E1,7E1,2.7E1,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-8.11151E-3,-2.3770498E-2,1.3511473E-1,-4.903885E-2,3.7162323E-4,2.8097564E-1,1.2597005E-3,-4.1468557E-2,-9.574549E-3,1.8381007E-2,7.012952E-3,-1.3489653E-3,-6.7639E-2,-9.2418713E-4,6.152402E-3,-8.961814E-2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[5.4596394E-1,1.7908293E-1,3.6382854E-1,1.495364E-1,0E0,1.009686E-1,0E0,1.2485254E-1,0E0,0E0,0E0,1.3845807E-1,1.4358482E-1,0E0,0E0,9.946346E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,11,11,12,12,15,15],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[7.2259143E2,2.2348747E2,2.45226E6,2.1405182E2,3.7162323E-4,3.0256798E0,1.2597005E-3,4.37E2,-9.574549E-3,1.8381007E-2,7.012952E-3,4.02E2,5.8536225E5,-9.2418713E-4,6.152402E-3,4.73E2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"split_indices":[67,67,44,67,0,69,0,0,0,0,0,0,43,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.18E2,2.3E1,1.22E2,9.6E1,9E0,1.4E1,1.18E2,4E0,3E0,6E0,4.7E1,7.1E1,4.2E1,5E0,5.7E1,1.4E1,2E0,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.816721E-3,-3.212387E-2,7.8340545E-2,-5.9203006E-2,-2.047665E-3,1.05734095E-1,-1.2828258E-3,-4.2488243E-2,-1.6570118E-1,5.200363E-3,-1.1345699E-2,6.766135E-2,1.9234525E-1,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,1.8420164E-2,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.675061E-1,1.4077646E-1,2.3921198E-1,1.54948E-1,9.489868E-2,1.9802713E-1,0E0,1.2677193E-1,1.3189653E-1,0E0,9.6501775E-2,1.6102695E-1,1.9053322E-1,0E0,0E0,0E0,0E0,0E0,8.944257E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.5442211E8,3.6207435E0,1.2869189E5,2.3755E4,5.5303E9,-1.2828258E-3,7.92582E5,3.396945E7,5.200363E-3,3.7230852E0,9.90141E6,6.464779E6,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,4.33121E0,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"split_indices":[67,7,72,62,9,5,0,12,12,0,69,60,44,0,0,0,0,0,69,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.72E2,7.9E1,9E1,8.2E1,6.3E1,1.6E1,7.9E1,1.1E1,5E0,7.7E1,4.5E1,1.8E1,5.4E1,2.5E1,6E0,5E0,3.2E1,4.5E1,3.4E1,1.1E1,1E1,8E0,1.7E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.911687E-4,-2.2826223E-2,8.1994176E-2,-2.6359616E-2,9.769677E-3,1.5213658E-2,6.624614E-2,-5.9948202E-2,-4.920752E-3,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9672684E-1,1.8459845E-1,2.2044265E-1,1.4588554E-1,0E0,0E0,1.2346178E-1,1.278441E-1,1.16657905E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.4122614E2,1E0,5.283708E0,9.769677E-3,1.5213658E-2,1.1598054E8,3.901529E6,5.8536225E5,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"split_indices":[57,71,13,71,0,0,47,9,43,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,2.04E2,5.7E1,2.02E2,2E0,2E0,5.5E1,7.8E1,1.24E2,4.5E1,1E1,6.5E1,1.3E1,1.05E2,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-4.2027067E-3,-2.6648754E-2,8.658342E-2,-3.084103E-3,-2.5529137E-3,1.4771055E-2,2.9963553E-3,4.294585E-3,-1.1864337E-2,-5.120143E-2,4.8290283E-4,1.1069501E-3,-3.521963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[5.0360364E-1,2.0985942E-1,1.97959E-1,0E0,1.22443795E-1,0E0,0E0,0E0,1.0790554E-1,1.0145053E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.4125891E0,5.283708E0,1E0,-3.084103E-3,9.165388E4,1.4771055E-2,2.9963553E-3,4.294585E-3,1.1670399E-1,3.712E4,4.8290283E-4,1.1069501E-3,-3.521963E-3],"split_indices":[57,71,13,0,48,0,0,0,54,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.97E2,4.8E1,6.9E1,1.28E2,2E0,4.6E1,1E1,1.18E2,4.3E1,7.5E1,1.2E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.173793E-3,-2.6264051E-2,8.658401E-2,-6.669964E-2,-8.2213007E-4,1.4341326E-2,7.03292E-2,1.6294725E-2,-8.671885E-2,2.6084114E-2,-4.4527646E-2,8.693059E-4,1.125322E-1,-1.6205347E-3,6.1647515E-3,-2.2572479E-1,-6.812656E-2,4.0135182E-2,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.266243E-1,2.0775451E-1,1.8148205E-1,1.3100863E-1,1.4815538E-1,0E0,1.0412836E-1,1.1208219E-1,1.4744174E-1,1.12065986E-1,9.200167E-2,0E0,9.0320885E-2,0E0,0E0,9.239629E-2,1.0407472E-1,1.1784806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5098532E0,5.283708E0,1E0,3.6E1,1.7303246E7,1.4341326E-2,2.174459E5,7.014272E7,9.975593E4,5.8414636E0,2.094E3,8.693059E-4,4.7201157E-1,-1.6205347E-3,6.1647515E-3,7.725141E4,1.9415865E2,2.4256398E8,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"split_indices":[56,71,13,10,60,0,43,7,43,69,2,0,53,0,0,43,67,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.01E2,5.1E1,7.7E1,1.24E2,2E0,4.9E1,1.5E1,6.2E1,7.7E1,4.7E1,2.3E1,2.6E1,1.1E1,4E0,6E0,5.6E1,6.8E1,9E0,7E0,4E1,2.1E1,5E0,3E0,3E0,4.8E1,8E0,2E0,6.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[5.15778E-3,-1.8465955E-2,8.479578E-2,-4.582774E-2,2.8703757E-2,1.1748621E-2,2.8553656E-3,-6.113014E-2,7.824005E-4,7.2038495E-3,8.444443E-3,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,-1,-1,-1,-1],"loss_changes":[4.6061453E-1,2.4524778E-1,1.8237612E-1,1.1817169E-1,1.9200628E-1,0E0,0E0,1.1865306E-1,0E0,0E0,1.02744974E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2446029E0,1.781973E6,1.8426E4,5.4279834E-1,2.207E3,1.1748621E-2,2.8553656E-3,4.1150245E-1,7.824005E-4,7.2038495E-3,1E1,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"split_indices":[57,1,9,56,0,0,0,57,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.88E2,5.5E1,1.19E2,6.9E1,4E0,5.1E1,9.6E1,2.3E1,8E0,6.1E1,9.4E1,2E0,1.8E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[2.6694385E-3,-1.2630397E-2,9.763298E-2,-2.7587097E-3,-3.7950324E-3,2.2221443E-1,6.921976E-4,4.2745727E-3,-9.810083E-3,4.104886E-3,1.3904189E-2,-3.826968E-2,8.1882236E-4,-4.5390716E-3,-2.056427E-2,-1.03351854E-1,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,-1,-1,11,-1,-1,13,-1,-1,15,17,-1,-1,-1],"loss_changes":[3.4557673E-1,1.5018955E-1,3.3265132E-1,1.3020961E-1,0E0,1.3063407E-1,0E0,0E0,1.3977444E-1,0E0,0E0,9.7968146E-2,0E0,0E0,9.9550575E-2,1.2142539E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,11,11,14,14,15,15],"right_children":[2,4,6,8,-1,10,-1,-1,12,-1,-1,14,-1,-1,16,18,-1,-1,-1],"split_conditions":[6.56755E2,2.4219654E0,2.660842E6,1.1623532E6,-3.7950324E-3,2.1061536E3,6.921976E-4,4.2745727E-3,1.7179383E5,4.104886E-3,1.3904189E-2,1E0,8.1882236E-4,-4.5390716E-3,1.27318E5,1.3738942E0,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"split_indices":[67,68,44,60,0,4,0,0,47,0,0,104,0,0,43,68,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2.04E2,3.2E1,1.81E2,2.3E1,1.2E1,2E1,1.1E1,1.7E2,6E0,6E0,8.5E1,8.5E1,1.7E1,6.8E1,1.1E1,5.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.2459247E-4,-2.1839963E-2,5.5326115E-2,6.403799E-2,-2.9754521E-2,6.706761E-2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,3.54164E-2,1.4349867E-1,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,13,15,-1,-1,-1,-1],"loss_changes":[3.0025756E-1,1.2017081E-1,1.2092781E-1,9.031686E-2,1.1387043E-1,1.5458077E-1,0E0,0E0,0E0,0E0,0E0,1.247524E-1,1.0393754E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,11,11,12,12],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,14,16,-1,-1,-1,-1],"split_conditions":[3.799535E2,1.1623532E6,6.236639E3,2.0535674E0,1.4781966E-3,9.386909E2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,9E0,7.529323E4,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"split_indices":[67,60,4,68,73,67,0,0,0,0,0,3,43,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.73E2,7E1,1.4E1,1.59E2,6.5E1,5E0,8E0,6E0,6E0,1.53E2,4.7E1,1.8E1,8E0,3.9E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.029568E-4,-6.5150703E-3,1.8107072E-1,-3.652972E-2,2.4049532E-2,1.0449934E-2,-1.5273974E-3,-2.0122712E-2,-9.696651E-2,3.5459854E-2,-2.3149115E-3,4.5559215E-3,-2.7940093E-2,-1.3360148E-3,-1.7182508E-1,6.808551E-3,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,4.025893E-2,3.4104274E-3,-9.366535E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[3.37975E-1,2.2380325E-1,1.3455206E-1,1.2001395E-1,1.08324245E-1,0E0,0E0,9.831457E-2,1.2388055E-1,1.0830644E-1,0E0,0E0,1.4380896E-1,0E0,1.03295654E-1,1.3083223E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.5229566E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,12,12,14,14,15,15,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[1.9292948E3,2.0718746E8,6.236639E3,4.352025E0,2.2979798E0,1.0449934E-2,-1.5273974E-3,1E1,5.694004E6,3.642194E0,-2.3149115E-3,4.5559215E-3,9.476661E1,-1.3360148E-3,4.752366E0,4.854831E8,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,1.9647126E7,3.4104274E-3,-9.366535E-4],"split_indices":[67,7,4,69,68,0,0,10,60,69,0,0,67,0,71,12,0,0,0,0,0,0,60,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.42E2,9E0,1.22E2,1.2E2,7E0,2E0,9.7E1,2.5E1,1.05E2,1.5E1,5E0,9.2E1,1.4E1,1.1E1,5.9E1,4.6E1,2.5E1,6.7E1,4E0,7E0,2E1,3.9E1,2.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_emr","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"61"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.78048E-2,-1.8552697E-1,4.901243E-1,-3.3088633E-1,3.0965138E-2,2.4628823E-1,8.852223E-1,-4.659008E-1,-1.3334489E-1,-1.5243791E-1,1.4812984E-1,2.971024E-1,-1.2563897E-2,5.130134E-2,6.705256E-1,-1.8754591E-1,-5.3297985E-1,9.115728E-2,-2.2618434E-1,-2.66628E-1,1.2518686E-2,2.2718608E-1,4.980866E-2,4.346211E-1,1.5858072E-1,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,-6.1508644E-1,-3.1681323E-1,7.0068543E-3,-7.234637E-4,-2.9873994E-1,1.068607E-2,-7.976921E-3,-2.4829587E-2,-7.415798E-2,4.8681386E-3,1.2035196E-2,4.350847E-2,-3.566741E-2,9.323246E-3,2.2088567E-2,1.0012548E-2,2.3240472E-1,2.6123306E-2,-1.783583E-2,-6.754464E-1,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4057398E1,5.265595E0,7.200094E0,2.60497E0,1.4837892E0,6.482475E-1,1.0196114E0,1.0016766E0,8.960098E-1,5.119209E-1,3.1514597E-1,7.120998E-1,2.5112772E-1,0E0,1.8040752E-1,5.0723886E-1,6.393223E-1,9.9871E-2,5.187807E-1,3.0155098E-1,1.1128237E-1,1.9333434E-1,2.8770238E-1,1.774602E-1,2.0905662E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3500404E-1,1.0710728E-1,0E0,0E0,1.3412046E-1,3.246751E-1,0E0,0E0,1.1355559E-1,0E0,0E0,9.497507E-2,1.2899351E-1,0E0,0E0,0E0,9.763676E-2,1.0520158E-1,0E0,3.8818073E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,35,35,36,36,39,39,42,42,43,43,47,47,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,2.7558826E-2,2.0224316E-1,1.9415865E2,2.4118989E8,2.05E0,7.1239825E5,1.433458E10,8.938651E4,7.919613E5,3.2744168E6,7.3E1,6.035389E6,5.130134E-2,2.5E1,2.7287993E0,1.021924E6,1.1936242E7,4.145324E6,2.7575057E0,6.406965E-1,1.2183861E7,2.1469998E5,6.75602E5,2.8994352E7,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,3.12986E10,1.858E3,7.0068543E-3,-7.234637E-4,3.1100525E5,4.7E3,-7.976921E-3,-2.4829587E-2,4.565566E-1,4.8681386E-3,1.2035196E-2,6.7E2,1.526402E6,9.323246E-3,2.2088567E-2,1.0012548E-2,1E0,1.479938E6,-1.783583E-2,7.359468E8,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"split_indices":[54,40,40,54,7,55,35,33,35,34,52,8,1,0,3,56,31,47,1,60,41,34,30,9,1,0,0,0,0,0,0,33,0,0,0,35,0,0,0,41,0,0,0,9,0,0,0,102,34,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,1.66E2,7.6E1,9.9E1,6.7E1,4.8E1,2.8E1,5.8E1,4.1E1,2.6E1,4.1E1,4E1,8E0,1E1,1.8E1,1.2E1,4.6E1,1.2E1,2.9E1,1.5E1,1.1E1,2.2E1,1.9E1,1.9E1,2.1E1,4E0,4E0,1.6E1,2E0,2E0,1E1,3.2E1,1.4E1,7E0,5E0,2.2E1,7E0,1.3E1,2E0,6E0,5E0,1.7E1,5E0,1.3E1,6E0,1.3E1,6E0,1.3E1,8E0,9E0,2.3E1,2E0,1.2E1,6E0,1.6E1,3E0,4E0,3E0,3E0,2E0,3E0,7E0,6E0,7E0,6E0,4E0,4E0,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.0040125E-2,-2.1713006E-1,4.0957847E-1,-3.5627815E-1,9.978797E-3,2.0629005E-1,8.082211E-1,-4.7081342E-1,-1.7093469E-1,-5.8718614E-2,1.0682804E-2,6.47079E-2,2.835063E-1,2.307691E-2,4.1448183E-2,9.978426E-4,-4.971977E-1,1.4917599E-3,-3.1879982E-1,-1.7213058E-1,8.313783E-2,1.541623E-1,-7.4060783E-3,3.6715305E-1,1.0359522E-2,-1.8699963E-1,-5.3688544E-1,-9.420938E-2,1.2580958E-2,-1.8095352E-2,-1.4491154E-1,-2.1190276E-2,-1.1271654E-1,-1.211096E-2,2.5133708E-1,9.38944E-3,-7.0502064E-3,1.9026043E-5,3.9769462E-1,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,-3.106009E-1,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,-1.4537086E-1,7.170774E-3,-6.025724E-2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,4.859038E-1,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,5.357659E-1,3.981885E-3,7.6941703E-3,2.5353398E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,-1,-1,55,-1,-1,-1,-1,-1,57,-1,-1,-1,-1,59,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[2.0510149E1,4.996633E0,6.1543484E0,1.9999018E0,9.984025E-1,5.610745E-1,5.2623177E-1,7.9289055E-1,9.9481404E-1,7.8654516E-1,0E0,4.4556248E-1,7.730229E-1,0E0,0E0,0E0,6.0133266E-1,5.4866415E-1,2.8996253E-1,4.4809937E-1,3.5550103E-1,3.2364437E-1,0E0,2.9102945E-1,1.4653622E-1,9.1507256E-2,6.157198E-1,1.915977E-1,0E0,0E0,1.275117E-1,0E0,2.4100256E-1,1.6567487E-1,2.9098767E-1,0E0,0E0,0E0,3.2995868E-1,0E0,0E0,0E0,0E0,0E0,1.7156088E-1,0E0,0E0,0E0,0E0,1.24156505E-1,0E0,9.488517E-2,0E0,0E0,0E0,0E0,2.6596856E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2510395E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,30,30,32,32,33,33,34,34,38,38,44,44,49,49,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,-1,-1,56,-1,-1,-1,-1,-1,58,-1,-1,-1,-1,60,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.5665247E2,1.7613442E-1,2.0224316E-1,1.9415865E2,9.749826E6,4.633287E2,3.856E3,3.8757736E-7,1.5248811E1,1.5225257E-1,1.0682804E-2,3.6217573E0,2.1030048E10,2.307691E-2,4.1448183E-2,9.978426E-4,1.1266387E10,3.165913E6,3.03E9,1.386452E0,1.7573398E2,1.9647126E7,-7.4060783E-3,2.34E3,2.51838E6,2.85E2,1.021924E6,5.63651E5,1.2580958E-2,-1.8095352E-2,2.3407683E0,-2.1190276E-2,2.7538432E8,2.424E3,1.2730925E0,9.38944E-3,-7.0502064E-3,1.9026043E-5,5.792986E4,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,1.892E3,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,2.5187686E-1,7.170774E-3,3.56E2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,1.1107071E7,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,3.302E3,3.981885E-3,7.6941703E-3,2.5353398E-2],"split_indices":[54,43,40,54,9,54,2,40,58,40,0,56,5,0,0,0,33,1,5,55,54,47,0,2,34,0,31,9,0,0,56,0,7,2,44,0,0,0,30,0,0,0,0,0,2,0,0,0,0,44,0,10,0,0,0,0,9,0,0,0,0,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,1.57E2,7.7E1,9.7E1,6E1,5.2E1,2.5E1,5.9E1,3.8E1,4.7E1,1.3E1,1.9E1,3.3E1,1E1,1.5E1,3E0,5.6E1,1.8E1,2E1,2.6E1,2.1E1,1.4E1,5E0,2.5E1,8E0,7E0,4.9E1,1.4E1,4E0,1.2E1,8E0,3E0,2.3E1,1.4E1,7E0,1.2E1,2E0,2E0,2.3E1,5E0,3E0,5E0,2E0,3.6E1,1.3E1,7E0,7E0,6E0,2E0,2.1E1,2E0,1.2E1,2E0,2E0,5E0,8E0,1.5E1,6E0,7E0,8E0,1.3E1,7E0,5E0,1.3E1,2E0,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.5844442E-2,-2.1939752E-1,3.5634845E-1,-3.067927E-1,7.70621E-2,1.8756232E-1,7.0530635E-1,-3.6990502E-1,-1.6308445E-2,3.355599E-2,1.0062139E-2,-4.2225484E-2,2.3436439E-1,1.2260787E-2,7.853399E-1,-4.3145156E-1,-2.022222E-1,1.7129338E-1,-1.4914486E-1,-7.0826644E-3,8.609564E-3,7.4184006E-3,-1.4439643E-2,2.7769628E-1,6.108423E-3,1.9752327E-2,3.9388932E-2,-3.5376546E-1,-6.491721E-1,-1.04918115E-1,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,4.5147832E-2,-5.3988285E-3,2.0416094E-1,4.445212E-1,-5.368166E-3,3.978625E-3,-5.482469E-1,-2.6226673E-1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,-3.322308E-2,-8.5412245E-4,7.761204E-3,2.3548925E-1,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,-2.9240888E-1,-3.515024E-3,6.6058734E-3,2.612413E-1,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,49,51,-1,-1,53,55,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,-1,-1,61,-1,-1,63,-1,-1,-1,-1,-1],"loss_changes":[1.863963E1,4.1431956E0,4.991946E0,2.2489424E0,2.3938012E-1,6.55169E-1,7.3028564E-1,9.5510197E-1,5.9871566E-1,2.0101649E-1,0E0,7.301139E-1,4.9284196E-1,0E0,4.888401E-1,1.0050564E0,4.5257246E-1,2.7720916E-1,2.4958798E-1,1.5788087E-1,0E0,0E0,0E0,4.2448258E-1,1.1006676E-1,0E0,0E0,8.527217E-1,3.161502E-1,1.413299E-1,0E0,0E0,0E0,0E0,0E0,1.5807515E-1,0E0,3.994676E-1,2.7063847E-1,0E0,0E0,1.5897751E-1,3.2701302E-1,0E0,0E0,0E0,1.2576254E-1,0E0,0E0,1.6656256E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.2975593E-1,0E0,0E0,1.3126945E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,27,27,28,28,29,29,35,35,37,37,38,38,41,41,42,42,46,46,49,49,56,56,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,50,52,-1,-1,54,56,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,-1,-1,62,-1,-1,64,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,6.005889E-1,2.0224316E-1,2.727992E6,9.750871E6,7.12E2,2.744E3,1.1670399E-1,1.6E1,1E0,1.0062139E-2,3.9378E5,2.8812434E7,1.2260787E-2,2.2324745E5,3.98231E-2,3.4047124E-1,6.0578914E4,2.6017979E11,4.33E2,8.609564E-3,7.4184006E-3,-1.4439643E-2,7.2259143E2,1.04E2,1.9752327E-2,3.9388932E-2,3.3617156E-4,1.868632E7,1.86204E5,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,7.948E4,-5.3988285E-3,5.130435E0,9.521E3,-5.368166E-3,3.978625E-3,5.8013948E7,9.3E1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,2.3544362E6,-8.5412245E-4,7.761204E-3,2.1061536E3,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,1.65E9,-3.515024E-3,6.6058734E-3,7.83495E5,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"split_indices":[54,41,40,1,9,0,2,41,3,101,0,1,47,0,30,44,44,34,33,0,0,0,0,54,8,0,0,41,47,9,0,0,0,0,0,1,0,56,2,0,0,7,11,0,0,0,53,0,0,4,0,0,0,0,0,0,5,0,0,35,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.58E2,8.6E1,1.22E2,3.6E1,5.9E1,2.7E1,1E2,2.2E1,2.9E1,7E0,1E1,4.9E1,5E0,2.2E1,7.2E1,2.8E1,9E0,1.3E1,2.4E1,5E0,6E0,4E0,4.1E1,8E0,7E0,1.5E1,5.5E1,1.7E1,1.9E1,9E0,2E0,7E0,2E0,1.1E1,1.7E1,7E0,3E1,1.1E1,3E0,5E0,1.6E1,3.9E1,1.5E1,2E0,7E0,1.2E1,1.2E1,5E0,2.8E1,2E0,6E0,5E0,1.3E1,3E0,4E0,3.5E1,1E1,2E0,2.5E1,3E0,2.5E1,1E1,1.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[1.0417009E-2,-1.92214E-1,4.0404752E-1,-3.015024E-1,-7.670976E-5,1.7257118E-1,7.666108E-1,-3.8266033E-1,-1.3505321E-2,-5.040442E-2,1.1856716E-2,2.0422818E-1,-5.236619E-3,4.0671965E-1,9.444278E-1,-4.4783047E-1,-2.4137214E-1,-8.257532E-2,9.555998E-3,-1.7077446E-2,-2.2676414E-2,-1.2199888E-3,2.4267583E-1,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,-1.7631327E-1,-5.2404433E-1,-4.6340473E-2,-1.6422002E-2,8.984605E-3,-1.4559102E-1,-8.014499E-2,7.040712E-2,3.7856903E-2,1.1994875E-2,8.994386E-3,-1.1225067E-2,-5.8429706E-1,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,-1.4150277E-1,4.9856484E-2,1.1765836E-2,1.939237E-2,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,-1,35,-1,-1,-1,-1,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0656195E1,3.5906997E0,7.253089E0,2.5425148E0,8.754045E-1,5.339109E-1,1.7917042E0,6.7456627E-1,4.1952434E-1,5.01606E-1,0E0,4.531319E-1,0E0,9.731603E-2,3.5835075E-1,1.0871277E0,7.457906E-1,3.9352587E-1,0E0,0E0,2.7846104E-1,0E0,2.731049E-1,0E0,0E0,0E0,0E0,4.8621753E-1,5.456524E-1,1.2857142E-1,0E0,0E0,3.2480013E-1,2.623061E-1,1.9802536E-1,1.5393375E-1,0E0,0E0,0E0,2.466135E-1,0E0,0E0,0E0,0E0,0E0,1.9748119E-1,1.3966252E-1,0E0,1.250949E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,20,20,22,22,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,45,45,46,46,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,-1,36,-1,-1,-1,-1,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0845802E6,1.7613442E-1,1.5972575E0,2.727992E6,2.03693E11,1.25440235E1,7.2610445E8,1.9415865E2,1.22E2,2.9927E4,1.1856716E-2,4.0123675E-2,-5.236619E-3,1.3589149E6,2.2324745E5,2.431768E2,1.2247422E1,2.6017979E11,9.555998E-3,-1.7077446E-2,1.1343225E6,-1.2199888E-3,3.4021906E2,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,1.0395554E5,6.6037946E2,6.0515753E2,-1.6422002E-2,8.984605E-3,2.3407683E0,9.74979E6,1.2355705E6,1.19881555E-1,1.1994875E-2,8.994386E-3,-1.1225067E-2,1.51E3,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,4.4E1,5.3164E4,1.1765836E-2,6.0039372E0,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"split_indices":[53,43,43,1,33,59,12,54,8,9,0,40,0,49,30,57,58,33,0,0,35,0,54,0,0,0,0,30,4,4,0,0,56,9,35,40,0,0,0,2,0,0,0,0,0,10,31,0,56,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.7E2,8.7E1,1.08E2,6.2E1,5.4E1,3.3E1,8.4E1,2.4E1,5.3E1,9E0,4.9E1,5E0,1.2E1,2.1E1,5.6E1,2.8E1,1.9E1,5E0,3E0,5E1,7E0,4.2E1,1E1,2E0,4E0,1.7E1,1.3E1,4.3E1,1.2E1,1.6E1,3E0,1.6E1,3.1E1,1.9E1,6E0,3.6E1,2E0,1.1E1,3.4E1,9E0,3E0,9E0,1.4E1,2E0,2.1E1,1E1,3E0,1.6E1,2E0,4E0,2.2E1,1.2E1,3E0,1.8E1,4E0,6E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-2.2585599E-2,-1.9692676E-1,3.6539125E-1,-2.9716375E-1,-2.7832244E-2,1.6051705E-1,7.1087146E-1,-3.7421748E-1,1.4051265E-2,-6.813547E-2,1.1468622E-2,-8.938615E-2,2.114588E-1,4.520597E-1,9.747266E-1,-4.3956405E-1,-2.1756326E-1,-5.356781E-2,1.1113507E-2,-1.736248E-1,1.3845394E-2,4.104848E-3,-6.774424E-3,2.3279007E-1,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,-2.5017148E-1,-4.929268E-1,1.6659833E-3,-1.6076026E-2,9.1863185E-2,-6.740727E-3,-7.370915E-2,-3.767319E-1,-5.412376E-3,5.573497E-2,9.279637E-2,2.8638762E-1,-2.9118592E-1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,3.14243E-1,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,-1,47,-1,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1,-1],"loss_changes":[1.6576199E1,2.8561993E0,5.265172E0,2.5471687E0,7.675839E-1,6.4269626E-1,1.4714527E0,7.744961E-1,3.715908E-1,4.9534598E-1,0E0,1.2332737E-1,2.3372102E-1,1.488092E-1,2.703581E-1,4.729681E-1,8.93371E-1,2.783168E-1,0E0,4.7957075E-1,1.9750053E-1,0E0,0E0,2.585504E-1,0E0,0E0,0E0,0E0,0E0,1.6383308E-1,2.5936127E-1,1.265059E-1,0E0,2.4251655E-1,0E0,1.5377875E-1,1.1529529E-1,0E0,2.033091E-1,1.5228397E-1,2.5932693E-1,1.0870254E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3727503E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,29,29,30,30,31,31,33,33,35,35,36,36,38,38,39,39,40,40,41,41,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,-1,48,-1,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1,-1],"split_conditions":[4.083633E6,1.7613442E-1,1.5972575E0,2.246309E6,2.03693E11,3.359399E-1,1.6525185E9,1.9415865E2,1.22E2,7.251646E-2,1.1468622E-2,2.2183135E-1,1.25440235E1,1.6790323E8,4.6E1,2.4747229E2,1.2288136E1,4.7E3,1.1113507E-2,3.4047124E-1,2.0358758E8,4.104848E-3,-6.774424E-3,1.0705044E3,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,1.499E3,1.021924E6,2.797224E2,-1.6076026E-2,1.445113E6,-6.740727E-3,9.593298E6,6.9268906E5,-5.412376E-3,2.7352352E6,1.8E1,1.1107071E7,2.923913E1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,6.464779E6,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"split_indices":[53,43,43,1,33,44,5,54,8,40,0,41,59,7,8,57,58,0,0,44,7,0,0,57,0,0,0,0,0,2,31,54,0,9,0,47,35,0,50,3,9,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,31,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.68E2,7.5E1,1.05E2,6.3E1,4.8E1,2.7E1,8.4E1,2.1E1,5.6E1,7E0,8E0,4E1,1.5E1,1.2E1,5.8E1,2.6E1,1.7E1,4E0,2.4E1,3.2E1,2E0,6E0,3.7E1,3E0,2E0,1.3E1,2E0,1E1,1.4E1,4.4E1,1.1E1,1.5E1,7E0,1E1,1.7E1,7E0,7E0,2.5E1,1.1E1,2.6E1,1.2E1,2E0,3.8E1,6E0,7E0,4E0,4E0,3E0,1.3E1,4E0,4E0,3E0,1.4E1,1.1E1,8E0,3E0,2.4E1,2E0,3E0,9E0,1.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-8.34782E-3,-1.667577E-1,4.3978554E-1,-3.0709088E-1,2.6771413E-2,1.9930072E-1,7.418625E-1,-4.0431347E-1,-1.2503225E-1,-6.46251E-2,1.2227376E-1,2.7885714E-1,3.0976776E-2,3.739352E-1,8.703366E-1,-4.299911E-1,4.975969E-3,1.0842858E-1,-2.3695372E-1,-1.4191392E-1,1.8143274E-1,1.5553638E-1,-3.5369173E-3,2.103493E-2,1.11278795E-1,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,-4.528867E-1,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,-2.1131526E-1,5.121328E-2,8.2729507E-4,1.0838011E-2,7.531465E-2,2.1820368E-1,6.924229E-3,-5.724572E-3,-4.900431E-1,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,-4.1408205E-1,-6.208327E-1,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,45,-1,-1,-1,-1,47,49,-1,-1,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.7394594E1,4.9239187E0,4.501422E0,1.807971E0,6.800027E-1,4.868381E-1,9.980869E-1,9.540901E-1,1.0186052E0,7.864486E-1,2.696029E-1,7.975136E-1,3.8219005E-1,1.8597853E-1,5.8754253E-1,3.8879967E-1,0E0,1.0245718E-1,1.4958644E-1,4.2632413E-1,1.0402632E-1,1.4946616E-1,0E0,0E0,1.9317481E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9329586E-1,0E0,0E0,0E0,0E0,2.1796727E-1,1.8252599E-1,0E0,0E0,1.0288309E-1,1.249153E-1,0E0,0E0,2.2973442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3063469E-1,2.7027082E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,24,24,32,32,37,37,38,38,41,41,42,42,45,45,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,46,-1,-1,-1,-1,48,50,-1,-1,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.4210773E-2,2.0224316E-1,1.773747E2,6.75553E5,4.3147E5,2.174459E5,4.108148E6,9.165388E4,1.5225257E-1,2.4219654E0,3.5630266E8,1.4E1,5.76E3,1.6507233E3,6E0,4.975969E-3,1.1936242E7,2.1055435E1,2.5055168E9,6.765595E7,2.151E3,-3.5369173E-3,2.103493E-2,8.96347E7,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,1.437883E6,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,2.5335E4,4.933E4,8.2729507E-4,1.0838011E-2,5.546848E1,9.173913E5,6.924229E-3,-5.724572E-3,1.499E3,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,1.53091E5,2.373E3,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"split_indices":[54,40,40,54,9,9,30,9,35,40,55,7,3,2,4,10,0,47,58,5,5,2,0,0,46,0,0,0,0,0,0,0,1,0,0,0,0,9,9,0,0,54,49,0,0,2,0,0,0,0,0,0,0,0,0,9,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.8E2,6.3E1,1.04E2,7.6E1,3.6E1,2.7E1,6.7E1,3.7E1,3.9E1,3.7E1,2.4E1,1.2E1,8E0,1.9E1,6.4E1,3E0,1.2E1,2.5E1,3E1,9E0,3.2E1,5E0,1E1,1.4E1,4E0,8E0,3E0,5E0,1E1,9E0,5E0,5.9E1,7E0,5E0,1.4E1,1.1E1,2.2E1,8E0,3E0,6E0,1.5E1,1.7E1,1.2E1,2E0,4.8E1,1.1E1,2E0,2E1,4E0,4E0,2E0,1.3E1,1.5E1,2E0,3.3E1,1.5E1,2.4E1,9E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.9497597E-3,-1.8956819E-1,3.4107372E-1,-3.159441E-1,-1.5801242E-2,1.5268305E-1,6.5795535E-1,-3.6932546E-1,7.846604E-3,-5.214277E-2,1.0556764E-1,-1.2807088E-2,2.3161104E-1,4.560374E-1,8.566107E-1,-2.9471084E-1,-5.894394E-1,1.2438455E-1,-7.3430347E-3,-1.3571183E-1,1.1542113E-2,7.1960697E-3,-4.9598026E-3,1.3622053E-1,-1.1057054E-2,6.492877E-2,3.2902324E-1,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,-2.0076081E-1,-6.4882565E-1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,9.8576896E-2,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,2.6239768E-1,2.3697935E-2,-2.679108E-1,-6.2030084E-2,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,4.4976994E-3,-3.273256E-1,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6410194E1,3.4866414E0,5.391714E0,1.5930605E0,3.054343E-1,7.736126E-1,9.7316456E-1,1.1237602E0,3.082666E-1,2.8363693E-1,2.2685544E-1,7.515435E-1,6.2811255E-1,2.3523116E-1,7.9534054E-1,9.5042276E-1,3.015952E-1,2.1883255E-1,0E0,9.567392E-2,1.899715E-1,0E0,0E0,2.035301E-1,0E0,1.436969E-1,2.1399069E-1,0E0,0E0,0E0,0E0,0E0,3.7464368E-1,1.0184717E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1995159E-1,0E0,0E0,0E0,0E0,1.2598455E-1,0E0,4.546547E-1,1.02785856E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.053444E-1,1.2376809E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,26,26,32,32,33,33,40,40,45,45,47,47,48,48,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.1306552E-1,1.8829846E-1,3.402546E6,2.424E3,3.165913E6,1.4968063E3,5.1653096E-3,3.771E3,5.670282E-1,3.2410275E2,1.1936242E7,6.2143734E4,2.0270285E8,2.65826E6,3.3617156E-4,1.8018788E7,3.995065E6,-7.3430347E-3,2.8507206E-1,2.6801108E2,7.1960697E-3,-4.9598026E-3,1.9816086E0,-1.1057054E-2,2.51838E6,8.0267444E2,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,8.456733E10,7.5E1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,7E0,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,8.0774775E6,2.3697935E-2,2.3498028E6,3.948533E5,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,7.1E1,6E0,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"split_indices":[54,41,40,1,2,1,4,40,2,44,54,47,30,7,49,41,47,31,0,44,4,0,0,55,0,34,54,0,0,0,0,0,33,8,0,0,0,0,0,0,8,0,0,0,0,34,0,47,30,0,0,0,0,0,0,0,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.58E2,9.1E1,9.1E1,6.7E1,5.8E1,3.3E1,7.8E1,1.3E1,5.2E1,1.5E1,1.9E1,3.9E1,1.8E1,1.5E1,6E1,1.8E1,8E0,5E0,2.2E1,3E1,1.2E1,3E0,1.2E1,7E0,1.5E1,2.4E1,3E0,1.5E1,9E0,6E0,1.9E1,4.1E1,1.5E1,3E0,3E0,5E0,1.4E1,8E0,1.7E1,1.3E1,1E1,2E0,6E0,9E0,2E1,4E0,2.7E1,1.4E1,1.2E1,3E0,9E0,4E0,1.5E1,5E0,5E0,2.2E1,1E1,4E0,3E0,2E0,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.0813884E-3,-2.0281385E-1,3.2203957E-1,-2.7021798E-1,6.471507E-2,1.3456307E-1,6.046468E-1,-3.230015E-1,-2.4385922E-2,-4.1049863E-3,1.0497509E-1,-1.7012404E-1,1.8569289E-1,7.2859275E-1,3.886841E-1,-2.9304287E-1,-5.693168E-1,7.866887E-3,-7.695989E-2,1.368179E-1,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.1308988E-1,1.2391388E-1,4.675196E-2,5.738674E-1,9.75509E-3,2.0310232E-2,-3.7767187E-1,-1.9021347E-1,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,9.295686E-2,2.2762196E-2,-4.9231676E-3,1.4885345E-1,8.6237835E-3,2.6556352E-2,-4.5262882E-1,-2.0554197E-1,-2.254751E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.7958388E-2,1.8340372E-1,-3.690774E-1,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,-2.566865E-1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,-2.7930382E-1,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,-1,-1,51,53,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[1.6330715E1,2.812533E0,5.0049305E0,1.6011333E0,2.1443705E-1,9.507282E-1,7.339792E-1,5.858507E-1,2.613468E-1,0E0,1.8759432E-1,2.9529205E-1,3.6259735E-1,6.0491276E-1,9.870744E-2,7.3748016E-1,1.4939356E-1,0E0,3.703289E-1,1.539154E-1,0E0,0E0,0E0,6.859244E-1,2.2933495E-1,0E0,1.1461353E-1,0E0,0E0,5.5208206E-1,3.4554946E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2862067E-1,0E0,0E0,1.4594752E-1,0E0,0E0,4.9916983E-1,1.4001942E-1,4.1745436E-1,0E0,0E0,0E0,1.3290837E-1,1.4290059E-1,2.839706E-1,0E0,0E0,0E0,0E0,1.334548E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1397505E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,23,23,24,24,26,26,29,29,30,30,37,37,40,40,43,43,44,44,45,45,49,49,50,50,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,-1,-1,52,54,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.1259552E2,4.9615598E-1,1.2711891E0,2.727992E6,1.8681856E8,7.5E2,9.8110194E5,1E0,2.6017979E11,-4.1049863E-3,1E0,3.9378E5,6.5E1,2.7479675E-1,2.0787016E5,1.31654E5,3.7274265E2,7.866887E-3,3.2829787E12,3.1277522E6,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.302E3,3.0327642E-1,4.675196E-2,3.301E3,9.75509E-3,2.0310232E-2,1.3738942E0,2.0766662E6,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,1.4958695E5,2.2762196E-2,-4.9231676E-3,1.7377981E9,8.6237835E-3,2.6556352E-2,4.969844E3,7.07816E10,4.5595855E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.5E1,4.089454E4,1.9415865E2,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,2E1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,2.5605E4,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"split_indices":[54,41,41,1,7,0,35,77,33,0,77,1,8,40,30,9,4,0,33,53,0,0,0,2,44,0,2,0,0,55,50,0,0,0,0,0,0,30,0,0,12,0,0,49,33,58,0,0,0,3,30,54,0,0,0,0,3,0,0,0,0,0,0,31,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.54E2,9.5E1,1.23E2,3.1E1,5.8E1,3.7E1,1.01E2,2.2E1,6E0,2.5E1,8E0,5E1,2.2E1,1.5E1,9.2E1,9E0,4E0,1.8E1,2.2E1,3E0,3E0,5E0,1.5E1,3.5E1,5E0,1.7E1,6E0,9E0,4.9E1,4.3E1,2E0,7E0,1.3E1,5E0,1.3E1,9E0,8E0,7E0,3E0,3.2E1,2E0,1.5E1,3.3E1,1.6E1,3.7E1,6E0,2E0,6E0,7E0,2.5E1,2.7E1,6E0,1E1,6E0,3E0,3.4E1,4E0,3E0,4E0,2.1E1,2.5E1,2E0,3E1,4E0,2E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-6.4136046E-3,-1.6499257E-1,2.6240352E-1,-2.5779408E-1,-1.8667922E-4,1.3376746E-1,5.1147556E-1,-3.5323516E-1,-8.524396E-2,-9.4428E-2,1.0056048E-1,-9.662582E-2,1.73225E-1,3.220377E-2,3.9076596E-1,-1.4722008E-1,-4.0586957E-1,-1.8771532E-1,1.03829466E-1,-1.9139503E-1,7.505271E-2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,2.0356877E-1,-2.7897414E-3,4.3953964E-1,5.554044E-3,-9.59708E-3,4.6992786E-3,-4.2843238E-1,-4.7170566E-3,-1.2011155E-3,-2.7610272E-1,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,9.2738315E-2,2.768057E-1,7.707731E-3,2.085054E-2,-3.687053E-1,-5.6087404E-1,-1.6907474E-2,-7.4428786E-3,9.665818E-3,-4.444956E-2,3.618798E-1,4.56879E-3,-3.8534006E-1,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,4.9356914E-1,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,41,-1,43,-1,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,-1,57,59,-1,61,-1,-1,-1,-1,-1,-1,63,-1,-1,-1,-1],"loss_changes":[1.0913624E1,2.4623237E0,2.975348E0,1.6661301E0,5.696739E-1,5.985464E-1,5.864744E-1,6.559181E-1,7.557257E-1,5.238199E-1,2.0130792E-1,5.320686E-1,4.088E-1,0E0,2.3964357E-1,3.0165756E-1,3.0306053E-1,3.463729E-1,1.8510923E-1,2.6532936E-1,1.640884E-1,0E0,0E0,0E0,0E0,3.7205648E-1,0E0,1.0671902E-1,0E0,0E0,0E0,1.7289925E-1,0E0,0E0,1.2194073E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.8078758E-1,3.8673472E-1,0E0,0E0,1.6442966E-1,1.01364136E-1,0E0,0E0,0E0,2.1470177E-1,1.7472506E-1,0E0,8.7862015E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.314799E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,27,27,31,31,34,34,41,41,42,42,45,45,46,46,50,50,51,51,53,53,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,42,-1,44,-1,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,-1,58,60,-1,62,-1,-1,-1,-1,-1,-1,64,-1,-1,-1,-1],"split_conditions":[3.3103763E2,2.5106274E-2,1.8829846E-1,1.9415865E2,9.50379E5,7.5E2,7.5921344E5,2.431768E2,5.99999E9,1.2185057E0,1.3E1,3.13E2,1.25440235E1,3.220377E-2,2.56701E5,3.160848E6,7.4E1,3.0364912E5,4.654E3,2.587E3,1.7468556E2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,4.871647E2,-2.7897414E-3,1.5789307E8,5.554044E-3,-9.59708E-3,4.6992786E-3,1.912E3,-4.7170566E-3,-1.2011155E-3,5.316779E5,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,1.1572553E7,1.8751362E7,7.707731E-3,2.085054E-2,7.872337E5,1.2967741E0,-1.6907474E-2,-7.4428786E-3,9.665818E-3,1.1E1,5.5053414E9,4.56879E-3,1.0169992E0,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,1.6851284E7,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"split_indices":[54,40,40,54,9,0,35,57,5,43,3,0,59,0,31,9,3,35,2,2,54,0,0,0,0,54,0,5,0,0,0,2,0,0,35,0,0,0,0,0,0,47,46,0,0,53,55,0,0,0,3,5,0,55,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.54E2,1.6E2,9.4E1,1.02E2,5.8E1,6.3E1,3.1E1,6.5E1,3.7E1,3E1,2.8E1,9E0,5.4E1,9E0,2.2E1,1.4E1,5.1E1,2.4E1,1.3E1,1.9E1,1.1E1,1.9E1,9E0,4E0,5E0,4.8E1,6E0,1.8E1,4E0,1.1E1,3E0,4.7E1,4E0,9E0,1.5E1,1E1,3E0,1.7E1,2E0,7E0,4E0,2E1,2.8E1,3E0,1.5E1,3.5E1,1.2E1,6E0,9E0,1E1,1E1,1.8E1,1E1,3.3E1,2E0,4E0,8E0,4E0,6E0,1.1E1,7E0,2E0,3.1E1,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.2190764E-2,-1.8730694E-1,2.753607E-1,-2.6419362E-1,5.672748E-2,1.16373606E-1,5.309576E-1,-3.1258324E-1,-1.8713716E-2,-1.0737888E-1,1.2148461E-1,-1.23983465E-1,1.630511E-1,4.4448983E-2,4.534144E-1,-2.807187E-1,-2.4374343E-2,1.0378545E-1,-1.09028615E-1,-7.691218E-3,4.587331E-4,1.4246969E-1,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,2.1820955E-1,-6.480758E-3,5.3746756E-3,4.961946E-1,-2.0117857E-2,-2.3034851E-1,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1.7839447E-1,6.969961E-2,3.162982E-1,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,-2.1047248E-1,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.4450694E-1,9.220269E-3,-2.461917E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,-7.4021526E-2,-2.9362077E-1,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,-1,-1,37,39,-1,41,-1,43,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.2237465E1,2.848199E0,3.6767783E0,1.360198E0,4.0624598E-1,6.7093295E-1,8.615341E-1,5.956049E-1,2.3258556E-1,9.1160655E-2,1.3540426E-1,3.5389587E-1,4.596188E-1,0E0,3.909912E-1,6.838703E-1,0E0,1.7328064E-1,1.0814984E-1,0E0,0E0,1.2905765E-1,0E0,0E0,0E0,5.169381E-1,1.1691285E-1,0E0,1.8245935E-1,0E0,3.9284968E-1,0E0,0E0,0E0,0E0,0E0,1.09398365E-1,1.9227347E-1,2.0228004E-1,0E0,0E0,0E0,0E0,0E0,3.2970786E-1,0E0,0E0,0E0,0E0,1.4234447E-1,0E0,4.1922283E-1,0E0,0E0,0E0,2.4974391E-1,2.2993183E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,25,25,26,26,28,28,30,30,36,36,37,37,38,38,44,44,49,49,51,51,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,-1,-1,38,40,-1,42,-1,44,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[3.3103763E2,4.4269168E-1,1.7842576E-1,2.727992E6,2.1123085E8,7.5E2,1.431E4,1E0,1.8000048E10,1.3402277E0,1.1771224E8,3.9378E5,4.183787E0,4.4448983E-2,1E0,2.647867E-4,-2.4374343E-2,2.686459E6,1.6868966E8,-7.691218E-3,4.587331E-4,2.25E2,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,4.5971466E2,4.9863962E2,5.3746756E-3,2.9743274E8,-2.0117857E-2,1E0,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1E0,1.1572553E7,8.312613E5,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,2.933E3,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.488E3,9.220269E-3,6.7897725E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,2.7287993E0,6.767991E5,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"split_indices":[54,44,40,1,7,0,9,77,5,43,34,1,59,0,18,41,0,31,7,0,0,0,0,0,0,54,54,0,5,0,68,0,0,0,0,0,77,47,49,0,0,0,0,0,10,0,0,0,0,2,0,58,0,0,0,56,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.5E2,9.1E1,1.14E2,3.6E1,5.7E1,3.4E1,9.5E1,1.9E1,1E1,2.6E1,9E0,4.8E1,3E0,3.1E1,8.6E1,9E0,8E0,1.1E1,6E0,4E0,2.4E1,2E0,4E0,5E0,3.6E1,1.2E1,4E0,2.7E1,1.7E1,6.9E1,4E0,4E0,8E0,3E0,5E0,1.9E1,1.5E1,2.1E1,5E0,7E0,5E0,2.2E1,2E0,6.7E1,1.6E1,3E0,9E0,6E0,8E0,1.3E1,5.3E1,1.4E1,3E0,5E0,1.2E1,4.1E1,2E0,1E1,3.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.1875391E-3,-1.9125406E-1,2.4149728E-1,-3.2296014E-1,-4.532346E-2,9.999164E-2,4.7495925E-1,-3.5961127E-1,-3.3613036E-3,4.6751868E-2,-1.47186E-1,-3.1877293E-3,1.2955628E-1,1.2192243E-1,6.471233E-1,-2.8752145E-1,-5.190118E-1,-3.20057E-2,7.1740914E-3,-1.8803763E-1,5.201352E-3,1.4700729E-1,-4.1586305E-3,6.593719E-4,9.316681E-3,9.0452635E-1,5.182496E-1,4.057908E-2,-3.2754722E-1,-5.1887655E-3,-5.5908924E-1,3.8056083E-3,-3.9105513E-3,-2.4499905E-1,-2.4137127E-3,4.0515624E-2,1.8407537E-1,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,-2.7528796E-1,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,1.9600414E-1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,1.5410087E-1,4.4805845E-3,9.989807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,-1,33,-1,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,47,-1,49,51,-1,-1,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,57,-1,-1],"loss_changes":[1.1762257E1,2.7338152E0,3.6525607E0,6.483345E-1,6.540341E-1,3.7268925E-1,2.4738722E0,5.976982E-1,0E0,3.480822E-1,3.8151592E-1,0E0,2.5336754E-1,1.4349061E-1,4.6006012E-1,6.3051796E-1,2.3819494E-1,1.6576359E-1,0E0,2.0569742E-1,0E0,2.1754622E-1,0E0,0E0,0E0,2.0789146E-1,1.3972569E-1,1.1563162E-1,1.779995E-1,0E0,1.1710739E-1,0E0,0E0,1.0057318E-1,0E0,2.0077212E-1,1.1754835E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.099534E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.906554E-2,0E0,0E0,0E0,0E0,8.8078976E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,21,21,25,25,26,26,27,27,28,28,30,30,33,33,35,35,36,36,44,44,51,51,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,-1,34,-1,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,48,-1,50,52,-1,-1,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,58,-1,-1],"split_conditions":[3.5300907E-1,1.9415865E2,1.5972575E0,1.848519E-2,1.560615E1,1.8681856E8,2.744E3,5.039375E-3,-3.3613036E-3,3.165913E6,5.4878284E2,-3.1877293E-3,9.723711E0,6.765595E7,2.7479675E-1,1.0395554E5,1.47887E5,9.165388E4,7.1740914E-3,3.1692016E0,5.201352E-3,2.2599022E8,-4.1586305E-3,6.593719E-4,9.316681E-3,4.063E3,2.0467837E0,4.56E2,1.7565872E-3,-5.1887655E-3,6.6037946E2,3.8056083E-3,-3.9105513E-3,3.2829E4,-2.4137127E-3,1.19532734E5,1E0,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,4.5E2,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,5.7E1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,2.63E2,4.4805845E-3,9.989807E-3],"split_indices":[44,54,43,40,58,7,2,40,0,1,54,0,59,5,40,30,1,35,0,60,0,12,0,0,0,2,55,0,60,0,4,0,0,9,0,30,103,0,0,0,0,0,0,0,10,0,0,0,0,0,0,8,0,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,7.4E1,6.8E1,7E1,4.1E1,6.4E1,1E1,3.6E1,3.2E1,1E1,6E1,1.4E1,2.7E1,4.6E1,1.8E1,2.2E1,1.4E1,2.8E1,4E0,5.6E1,4E0,7E0,7E0,7E0,2E1,5E0,4.1E1,2E0,1.6E1,7E0,1.5E1,1.9E1,9E0,1.5E1,4.1E1,2E0,5E0,1.7E1,3E0,2E0,3E0,9E0,3.2E1,1.4E1,2E0,8E0,1.1E1,3E0,1.2E1,3.9E1,2E0,2.3E1,9E0,1.1E1,2.8E1,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.9416563E-3,-1.3463952E-1,3.7984976E-1,-2.3853315E-1,1.360329E-2,1.961173E-1,5.876671E-1,-3.1009814E-1,-1.1053116E-1,-3.428409E-2,1.00340895E-1,2.3631513E-2,1.5987208E-1,1.1244957E-2,6.9594884E-1,-6.4688744E-3,-3.5531205E-1,1.6310263E-3,-2.0668645E-1,-7.293062E-2,1.3145976E-1,9.800815E-3,-3.9949227E-2,-4.0140087E-3,1.8721226E-1,4.3395724E-2,2.3986792E-2,3.5975182E-3,-3.7373397E-1,-1.2723958E-2,-3.946153E-3,-9.6303485E-2,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,2.38724E-1,1.5173156E-3,-3.9501062E-1,-4.1339686E-3,8.0993416E-4,-1.22839086E-1,1.2143473E-2,4.7381404E-3,-4.0761718E-1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,-4.2140347E-1,-2.2882612E-2,-1.5165369E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,-1,29,31,33,-1,35,-1,37,-1,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,43,-1,45,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,-1],"loss_changes":[1.2767532E1,2.8833227E0,2.3666296E0,9.7337484E-1,3.2742375E-1,3.641218E-1,8.926077E-1,4.596181E-1,5.9082735E-1,3.3626413E-1,4.9703383E-1,0E0,2.5133562E-1,0E0,4.1397285E-1,0E0,4.6627617E-1,0E0,2.203567E-1,2.0018259E-1,1.2457131E-1,0E0,2.0385656E-1,0E0,2.3512948E-1,0E0,0E0,0E0,2.7126503E-1,0E0,0E0,1.1810273E-1,0E0,0E0,0E0,0E0,0E0,8.816862E-2,0E0,1.6809559E-1,0E0,0E0,9.8291665E-2,0E0,0E0,9.2234135E-2,0E0,0E0,0E0,0E0,1.4487553E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,18,18,19,19,20,20,22,22,24,24,28,28,31,31,37,37,39,39,42,42,45,45,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,-1,30,32,34,-1,36,-1,38,-1,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,44,-1,46,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.427E3,8.9397516E4,3.856E3,2.4747229E2,1.3048244E1,1.4191151E6,9E0,2.3631513E-2,7.5E2,1.1244957E-2,2.7479675E-1,-6.4688744E-3,3.8757736E-7,1.6310263E-3,3.03E9,4.222002E8,1.31903E5,9.800815E-3,1.3116745E1,-4.0140087E-3,4.183787E0,4.3395724E-2,2.3986792E-2,3.5975182E-3,7.4E1,-1.2723958E-2,-3.946153E-3,2.6639402E-1,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,4.3147E5,1.5173156E-3,9.4983295E2,-4.1339686E-3,8.0993416E-4,4.9615598E-1,1.2143473E-2,4.7381404E-3,4.699202E1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,9.6713684E1,-2.2882612E-2,-1.5165369E-2],"split_indices":[54,43,40,54,2,35,2,57,58,35,3,0,0,0,40,0,40,0,5,7,1,0,58,0,59,0,0,0,3,0,0,44,0,0,0,0,0,9,0,4,0,0,41,0,0,54,0,0,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.86E2,6.4E1,1.09E2,7.7E1,3.5E1,2.9E1,6.9E1,4E1,5E1,2.7E1,2E0,3.3E1,8E0,2.1E1,1.6E1,5.3E1,1.6E1,2.4E1,4.1E1,9E0,1.4E1,1.3E1,3E0,3E1,5E0,1.6E1,2E0,5.1E1,1.3E1,1.1E1,3.7E1,4E0,6E0,3E0,4E0,9E0,2.2E1,8E0,4.7E1,4E0,7E0,3E1,1.6E1,6E0,4.5E1,2E0,1.1E1,1.9E1,3E0,4.2E1,1.5E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[4.442649E-3,-1.2406673E-1,3.1314126E-1,-2.231523E-1,2.7577164E-2,1.5845458E-1,5.6371045E-1,-2.9410875E-1,-5.8644965E-2,-2.6454885E-2,1.4853756E-1,2.294401E-2,1.3163756E-1,2.7344096E-3,6.19114E-1,-9.231322E-2,-3.2586154E-1,-1.3393481E-1,2.8952213E-2,-4.4858553E-3,1.6983777E-2,1.7551805E-1,-3.9204443E-3,2.2740224E-1,3.826856E-2,1.9222386E-2,8.226514E-1,-7.2305724E-3,7.988117E-3,-3.752356E-1,-1.8137792E-1,3.932153E-3,-7.2262976E-3,-2.8384456E-3,1.1973143E-1,4.0585145E-2,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,-1.2312204E-1,5.3003617E-3,1.6423428E-2,4.1150242E-2,-3.9276025E-1,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,9.395361E-2,3.7756322E-3,-8.526156E-3,-3.2505807E-1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,-3.9313173E-1,-1.9234711E-2,-7.488049E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,-1,39,41,-1,43,-1,-1,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,53,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,61,-1,-1],"loss_changes":[1.0392895E1,2.7859118E0,2.895176E0,1.2901125E0,4.8757413E-1,4.1345346E-1,7.6624584E-1,4.6970892E-1,2.3470895E-1,1.741834E-1,1.6369265E-1,0E0,4.1154587E-1,0E0,5.829191E-1,2.727965E-1,4.0990114E-1,1.4968318E-1,1.5284702E-1,0E0,1.21554464E-1,9.423089E-2,0E0,2.5063312E-1,3.5061046E-1,0E0,2.3580599E-1,0E0,0E0,1.4392567E-1,3.3857572E-1,0E0,0E0,0E0,1.4192495E-1,1.1094484E-1,0E0,0E0,0E0,0E0,0E0,1.5323834E-1,0E0,0E0,0E0,2.5782776E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.29731E-2,0E0,0E0,1.570108E-1,0E0,0E0,0E0,0E0,1.24889135E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,26,26,29,29,30,30,34,34,35,35,41,41,45,45,52,52,55,55,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,-1,40,42,-1,44,-1,-1,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,54,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,62,-1,-1],"split_conditions":[3.705974E2,2.1619815E-1,2.0224316E-1,4.5537444E9,2.427E3,8.9397516E4,1E0,1.1563724E8,2.0256741E2,1.503E3,5.4281917E0,2.294401E-2,1.3844932E7,2.7344096E-3,1.6507233E3,3.160848E6,2.2348747E2,1.903E3,4.231592E6,-4.4858553E-3,2.302E3,3.5734247E2,-3.9204443E-3,5.264E4,1.1218195E6,1.9222386E-2,1.61628E5,-7.2305724E-3,7.988117E-3,9.0814667E2,4.1081388E5,3.932153E-3,-7.2262976E-3,-2.8384456E-3,4.3560153E5,6.4796996E-1,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,1.4153711E-3,5.3003617E-3,1.6423428E-2,4.1150242E-2,1.549E3,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,1.5130641E0,3.7756322E-3,-8.526156E-3,1.5E1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,1E0,-1.9234711E-2,-7.488049E-3],"split_indices":[54,43,40,5,2,35,18,5,54,2,56,0,47,0,4,9,54,0,1,0,2,54,0,9,49,0,30,0,0,57,30,0,0,0,34,41,0,0,0,0,0,40,0,0,0,2,0,0,0,0,0,0,58,0,0,3,0,0,0,0,105,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,1.84E2,7.6E1,1.11E2,7.3E1,4.8E1,2.8E1,7.7E1,3.4E1,5.1E1,2.2E1,2E0,4.6E1,3E0,2.5E1,1.1E1,6.6E1,1.8E1,1.6E1,1.8E1,3.3E1,2E1,2E0,2.2E1,2.4E1,1.5E1,1E1,9E0,2E0,4.8E1,1.8E1,2E0,1.6E1,8E0,8E0,2.9E1,4E0,1.8E1,2E0,1E1,1.2E1,8E0,1.6E1,3E0,7E0,4.4E1,4E0,1.4E1,4E0,3E0,5E0,1.3E1,1.6E1,2E0,6E0,3.2E1,1.2E1,8E0,8E0,1.4E1,1.8E1,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.2187884E-2,-1.0560222E-1,2.8632665E-1,-1.6603857E-1,6.5966286E-2,1.4586735E-1,4.827894E-1,-2.163441E-1,3.5765186E-2,-2.1459074E-3,9.6117355E-2,2.2276511E-2,1.16351E-1,3.9712414E-3,5.3956425E-1,-1.9085151E-1,-4.2924163E-1,1.310548E-1,-6.960556E-2,8.844785E-3,5.8431923E-2,-3.4577996E-2,1.8520117E-1,1.25307795E-2,2.6944714E-2,-3.2403958E-1,-1.5634167E-1,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,5.7000898E-2,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,4.627612E-2,-1.0834768E-2,2.3908277E-1,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,-1.3881442E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,-1.12805806E-1,-3.083408E-1,-7.1671814E-2,-2.4239872E-1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,-1,47,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1],"loss_changes":[7.844618E0,1.7736785E0,1.9440131E0,1.2868705E0,1.6094884E-1,4.141271E-1,6.1820793E-1,4.6623707E-1,2.7018133E-1,0E0,1.327295E-1,0E0,4.4284636E-1,0E0,3.3383512E-1,3.7219834E-1,2.928431E-1,1.1195864E-1,1.3693142E-1,0E0,1.6056333E-1,2.5339377E-1,1.8323988E-1,0E0,0E0,1.318934E-1,3.0327547E-1,0E0,0E0,0E0,1.019789E-1,0E0,0E0,0E0,0E0,1.6095355E-1,0E0,1.4495695E-1,0E0,0E0,0E0,0E0,2.8267586E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2379E-1,1.0256362E-1,2.1890551E-1,9.2020154E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,30,30,35,35,37,37,42,42,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,-1,48,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,4.9615598E-1,1.9530877E-1,2.727992E6,1.8681856E8,8.9397516E4,1.2354571E0,1E0,1.6E1,-2.1459074E-3,2.0383387E0,2.2276511E-2,1.479938E6,3.9712414E-3,7.2610445E8,3.3617156E-4,2.22105E5,1.2051221E0,1.7108529E3,8.844785E-3,2.2599022E8,2.629593E0,1.8990228E0,1.25307795E-2,2.6944714E-2,1.7565872E-3,3.4955364E7,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,4.654E3,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,6.2143734E4,-1.0834768E-2,1.6851284E7,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,3.219065E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,8.3170245E2,2.07298E5,6.214038E6,1.2271346E1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"split_indices":[54,41,40,1,7,35,55,77,3,0,60,0,34,0,12,41,1,55,4,0,12,56,55,0,0,60,7,0,0,0,2,0,0,0,0,30,0,1,0,0,0,0,44,0,0,0,0,0,0,57,1,9,58,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.69E2,7.2E1,1.25E2,4.4E1,4.3E1,2.9E1,1E2,2.5E1,9E0,3.5E1,2E0,4.1E1,4E0,2.5E1,9.1E1,9E0,1.3E1,1.2E1,8E0,2.7E1,1.3E1,2.8E1,7E0,1.8E1,1.7E1,7.4E1,2E0,7E0,4E0,9E0,1E1,2E0,1.7E1,1E1,1E1,3E0,1.9E1,9E0,6E0,1.1E1,3E0,7.1E1,6E0,3E0,5E0,5E0,1.3E1,6E0,6.3E1,8E0,4.9E1,1.4E1,4E0,4E0,4.4E1,5E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[8.535432E-5,-1.3134183E-1,2.4541517E-1,-2.1222556E-1,-3.1241742E-3,1.1487452E-1,4.674134E-1,-2.7328297E-1,-3.8918845E-2,-8.365333E-2,9.0494335E-2,2.394816E-3,1.6543052E-1,2.984512E-1,6.282826E-1,-2.368147E-1,-2.039022E-2,-7.2169475E-2,5.6061996E-3,-1.5359263E-1,5.1900256E-2,2.500614E-4,6.980609E-3,5.4644295E-3,-7.968769E-2,1.8786167E-1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,7.535615E-1,1.2825073E-2,-2.2865131E-2,-2.539767E-1,4.7256513E-3,-1.1367583E-1,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.703592E-1,1.374485E-1,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,-2.2705331E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,-2.3829083E-1,-2.8504506E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,-1,-1,-1,39,41,-1,-1,-1,43,-1,45,47,-1,49,-1,-1,-1,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1],"loss_changes":[8.318686E0,1.7416182E0,2.5544882E0,1.0792751E0,5.0514543E-1,3.2942206E-1,7.09455E-1,4.2694187E-1,1.6511148E-1,3.4927028E-1,1.8162525E-1,2.0153141E-1,2.3894668E-1,1.7641377E-1,4.4881153E-1,2.3653197E-1,0E0,1.9207498E-1,0E0,2.4937719E-1,1.370348E-1,0E0,0E0,0E0,2.0430191E-1,1.1672115E-1,0E0,0E0,0E0,1.8685484E-1,0E0,1.2997259E-1,1.903727E-1,0E0,2.3296595E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.380126E-2,9.920034E-2,0E0,0E0,0E0,0E0,0E0,1.8071914E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.499064E-1,1.1059618E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,24,24,25,25,29,29,31,31,32,32,34,34,41,41,42,42,48,48,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,-1,-1,-1,40,42,-1,-1,-1,44,-1,46,48,-1,50,-1,-1,-1,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1],"split_conditions":[3.4021906E2,2.5106274E-2,1.8829846E-1,1.781973E6,9.50379E5,3.359399E-1,5.6E1,2.418166E0,1.22E2,1.5225257E-1,5.3164E4,1.1936242E7,5.615313E3,2.598887E7,3.2307692E0,1.1623532E6,-2.039022E-2,2.207E3,5.6061996E-3,2.587E3,8.180814E7,2.500614E-4,6.980609E-3,5.4644295E-3,1.0520055E5,5.7E1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,1.4475617E0,1.2825073E-2,2.1426034E5,1.4614057E-3,4.7256513E-3,2.4616451E0,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.7307575E5,1.0899524E3,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,4.2446044E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,1.3496346E0,1.1306552E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"split_indices":[54,40,40,1,9,44,8,55,8,40,31,47,4,9,56,47,0,0,0,2,34,0,0,0,35,8,0,0,0,55,0,30,60,0,56,0,0,0,0,0,0,30,4,0,0,0,0,0,58,0,0,0,0,0,0,0,55,41,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.56E2,1.67E2,8.9E1,1.02E2,6.5E1,5.7E1,3.2E1,7.5E1,2.7E1,3.5E1,3E1,1.8E1,3.9E1,1.7E1,1.5E1,6.5E1,1E1,2.3E1,4E0,2.3E1,1.2E1,1.4E1,1.6E1,7E0,1.1E1,3.6E1,3E0,1.5E1,2E0,1E1,5E0,5E0,6E1,4E0,1.9E1,2E1,3E0,9E0,3E0,2E0,9E0,1.2E1,2.4E1,4E0,6E0,2E0,3E0,6E0,5.4E1,1.7E1,2E0,7E0,5E0,5E0,1.9E1,2E0,5.2E1,3.1E1,2.1E1,2.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-7.534086E-3,-8.694848E-2,3.8782644E-1,-1.7279062E-1,7.5343594E-2,1.2632059E-1,5.605976E-1,-2.4380171E-1,-4.4355273E-2,1.0239507E-1,-7.070103E-2,-8.078534E-4,9.607972E-3,6.888959E-1,4.1058496E-1,-2.8629518E-1,-1.11263916E-1,-1.6469195E-2,-1.506756E-2,4.8591096E-2,1.8556008E-1,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,-6.6435128E-3,-3.0423003E-1,-1.0063401E-2,2.9212602E-3,-4.0811297E-2,7.316056E-3,-2.3700453E-2,1.1168387E-1,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,-3.2206443E-1,-3.669968E-3,-6.68522E-2,7.2118444E-3,4.6604417E-2,-4.6691555E-3,8.108817E-3,-7.8055467E-3,-3.4489202E-1,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,-3.753321E-1,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,47,-1,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,-1],"loss_changes":[7.88305E0,2.924702E0,1.8267727E0,1.2386718E0,2.9564223E-1,2.4874002E-1,1.9830894E-1,4.6376085E-1,4.8201522E-1,2.6840913E-1,1.2740509E-1,0E0,0E0,3.1174803E-1,9.529376E-2,3.2900333E-1,5.045887E-1,0E0,2.2792198E-1,1.8052885E-1,1.6800141E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.805466E-2,2.1632099E-1,0E0,0E0,2.3643395E-1,0E0,1.16251014E-1,1.8746912E-1,0E0,0E0,0E0,0E0,1.5240574E-1,0E0,1.1973961E-1,0E0,9.648324E-2,0E0,0E0,1.9439551E-1,1.6225243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4404154E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,27,27,28,28,31,31,33,33,34,34,39,39,41,41,43,43,46,46,47,47,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,48,-1,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,-1],"split_conditions":[1.311002E0,3.051E3,2.787E3,1.5157113E-2,1.50321455E1,2.024E3,2.9898608E-1,2.2348747E2,2.718E4,4.871647E2,1.471E4,-8.078534E-4,9.607972E-3,1.792184E0,3.4048372E6,1E1,2.8041378E5,-1.6469195E-2,1.3431462E7,6.477987E-1,1.8751362E7,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,7.1E1,6.8E1,-1.0063401E-2,2.9212602E-3,1.00849255E3,7.316056E-3,4.303E3,4.99E3,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,1E0,-3.669968E-3,2.1799392E6,7.2118444E-3,2.5063604E7,-4.6691555E-3,8.108817E-3,6.074E3,4.239E3,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,3.1872916E2,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"split_indices":[41,2,2,40,59,2,40,54,9,54,0,0,0,44,47,10,30,0,9,59,46,0,0,0,0,0,0,0,3,0,0,4,0,2,2,0,0,0,0,105,0,50,0,47,0,0,2,10,0,0,0,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,2.08E2,4.1E1,1.36E2,7.2E1,1.7E1,2.4E1,8.7E1,4.9E1,6.1E1,1.1E1,7E0,1E1,1.1E1,1.3E1,6.5E1,2.2E1,3E0,4.6E1,3.8E1,2.3E1,8E0,3E0,5E0,6E0,8E0,5E0,4E0,6.1E1,1.3E1,9E0,4.1E1,5E0,1.8E1,2E1,1.4E1,9E0,2E0,2E0,5.6E1,5E0,3.7E1,4E0,1E1,8E0,1.2E1,8E0,4.8E1,8E0,2E1,1.7E1,7E0,3E0,6E0,2E0,3.9E1,9E0,1.4E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.6526852E-2,-1.3732727E-1,1.8663803E-1,-2.051646E-1,-2.9185692E-2,7.73927E-2,3.695984E-1,-2.6628572E-1,-9.389659E-2,-8.110631E-2,7.3622145E-2,4.680393E-3,1.8481968E-1,1.6930054E-3,3.9849573E-1,-2.2754145E-1,-2.0849528E-2,6.2429474E-4,-1.7701784E-1,-3.0688336E-1,-5.0080888E-2,4.623225E-3,-4.2899745E-3,1.03257135E-1,-1.1058583E-1,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,-2.4562204E-1,7.5825364E-3,-1.0521853E-2,-6.476905E-2,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,1.3644478E-1,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,-2.6290497E-1,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,-2.9959735E-1,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,31,33,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,-1,43,-1,-1,-1,-1,45,-1,-1,-1,-1,47,-1,-1,-1,-1,49,-1,-1,-1],"loss_changes":[6.3821125E0,1.1897857E0,1.9101899E0,6.5507936E-1,3.4759343E-1,4.8343337E-1,3.2867527E-1,4.1569996E-1,3.3530772E-1,2.8583866E-1,1.3266498E-1,4.4319636E-1,5.4417163E-1,0E0,2.268424E-1,4.489975E-1,0E0,0E0,1.3546717E-1,1.6951728E-1,1.4221326E-1,0E0,0E0,1.3188584E-1,1.2820828E-1,0E0,0E0,0E0,0E0,1.5322685E-1,0E0,0E0,9.677847E-2,0E0,0E0,0E0,0E0,1.2525946E-1,0E0,0E0,0E0,0E0,2.1090531E-1,0E0,0E0,0E0,0E0,1.8816972E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,18,18,19,19,20,20,23,23,24,24,29,29,32,32,37,37,42,42,47,47],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,32,34,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,-1,44,-1,-1,-1,-1,46,-1,-1,-1,-1,48,-1,-1,-1,-1,50,-1,-1,-1],"split_conditions":[3.3103763E2,1.1670399E-1,1.7842576E-1,1.9415865E2,2.7538432E8,6.483E3,1E0,4.1938826E-2,3.0364912E5,3.3789E4,1.1771224E8,1.1936242E7,6.2143734E4,1.6930054E-3,7.2610445E8,1.1661899E8,-2.0849528E-2,6.2429474E-4,1.3469651E3,2.151E3,1.2473118E0,4.623225E-3,-4.2899745E-3,1.9816086E0,1.4813267E3,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,6E0,7.5825364E-3,-1.0521853E-2,2.8659306E0,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,3.8692224E8,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,4.426498E0,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,1.1078758E0,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"split_indices":[54,41,40,54,7,2,18,44,35,9,34,47,30,0,12,7,0,0,4,2,55,0,0,55,57,0,0,0,0,10,0,0,60,0,0,0,0,7,0,0,0,0,58,0,0,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.62E2,9.6E1,9.9E1,6.3E1,6.1E1,3.5E1,6.3E1,3.6E1,4.2E1,2.1E1,3.7E1,2.4E1,3E0,3.2E1,5.5E1,8E0,1.6E1,2E1,4E0,3.8E1,1.8E1,3E0,2E1,1.7E1,1.4E1,1E1,1.2E1,2E1,5.3E1,2E0,1.2E1,8E0,2E0,2E0,9E0,2.9E1,1.7E1,3E0,1.2E1,5E0,5E0,4.8E1,6E0,2E0,1.5E1,2E0,3.7E1,1.1E1,5E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.752704E-3,-1.0890242E-1,2.1311827E-1,-1.7942376E-1,-4.630112E-3,8.463161E-2,4.3359125E-1,-2.6526305E-1,-4.0561736E-2,-8.0085285E-2,6.466574E-2,1.7120663E-2,6.155428E-2,2.8981546E-1,6.182841E-1,-1.9980457E-1,-3.8043132E-1,2.1313105E-2,-7.922915E-3,-1.4972365E-1,5.4258812E-2,-3.958611E-6,5.6440667E-3,-6.662196E-2,1.1769473E-1,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,-1.3539992E-1,-2.9197437E-1,-4.6381897E-1,-6.432207E-3,-5.7250984E-2,5.7095457E-3,-7.0690963E-4,-2.4091765E-1,-1.8624915E-3,6.908497E-3,7.913083E-2,-2.8256294E-3,-1.419902E-1,4.703381E-3,1.525669E-1,-8.453891E-4,2.933514E-3,-7.012109E-3,-3.371898E-1,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,7.1813464E-2,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,43,-1,-1,-1,-1,45,47,49,-1,51,-1,-1,53,-1,-1,55,-1,57,-1,59,-1,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,65,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1809554E0,1.2198377E0,2.597793E0,1.1693025E0,3.607325E-1,4.0189618E-1,7.2366E-1,3.8385105E-1,3.4410596E-1,3.1685543E-1,1.5055917E-1,0E0,4.1800576E-1,1.5071511E-1,4.769044E-1,2.0384932E-1,3.3971214E-1,2.4841867E-1,0E0,2.639835E-1,1.2779592E-1,1.0247372E-1,0E0,2.5190163E-1,1.941486E-1,0E0,0E0,0E0,0E0,9.141275E-2,1.3482976E-1,1.2367654E-1,0E0,1.1534542E-1,0E0,0E0,1.0435748E-1,0E0,0E0,1.1921386E-1,0E0,1.5446314E-1,0E0,1.8064415E-1,0E0,1.0156908E-1,0E0,1.0845828E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.946659E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,29,29,30,30,31,31,33,33,36,36,39,39,41,41,43,43,45,45,47,47,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,44,-1,-1,-1,-1,46,48,50,-1,52,-1,-1,54,-1,-1,56,-1,58,-1,60,-1,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,66,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.7613442E-1,1.8829846E-1,1.9415865E2,9.50379E5,8.9397516E4,1.623319E3,1.607E3,2.142943E1,1.2185057E0,6.6727E4,1.7120663E-2,3.359399E-1,2.0270285E8,2.65826E6,4.3399894E5,6.608892E2,6.135367E9,-7.922915E-3,2.8507206E-1,1.7468556E2,1.2495134E6,5.6440667E-3,5.4878284E2,3.2383578E7,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,1.1623532E6,3.342E3,2.1710082E7,-6.432207E-3,2.2E1,5.7095457E-3,-7.0690963E-4,6.2684685E-2,-1.8624915E-3,6.908497E-3,1.6537195E0,-2.8256294E-3,1.409674E7,4.703381E-3,1.7111111E0,-8.453891E-4,4.14E2,-7.012109E-3,1E0,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,1.0685258E6,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"split_indices":[54,43,40,54,9,35,4,2,58,43,31,0,44,7,49,30,4,5,0,44,54,47,0,54,1,0,0,0,0,47,0,34,0,3,0,0,40,0,0,55,0,47,0,55,0,2,0,105,0,0,0,0,0,0,0,0,0,0,0,0,35,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.65E2,9.2E1,9.8E1,6.7E1,5.9E1,3.3E1,6E1,3.8E1,3.2E1,3.5E1,3E0,5.6E1,2E1,1.3E1,4E1,2E1,2.7E1,1.1E1,2.1E1,1.1E1,1.8E1,1.7E1,1.7E1,3.9E1,3E0,1.7E1,9E0,4E0,2.5E1,1.5E1,1.4E1,6E0,1.6E1,1.1E1,9E0,1.2E1,6E0,5E0,8E0,1E1,1.2E1,5E0,3.1E1,8E0,4E0,2.1E1,1.2E1,3E0,1.2E1,2E0,1.1E1,5E0,5E0,7E0,5E0,3E0,3E0,9E0,1.5E1,1.6E1,2E0,2E0,8E0,4E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-4.591145E-3,-1.0742565E-1,2.1129104E-1,-1.6721527E-1,-3.2239808E-3,9.313745E-2,3.8930318E-1,-2.2656383E-1,-3.6957923E-2,-4.179782E-2,9.553272E-2,1.1645269E-1,-6.1090174E-3,9.100737E-3,4.8732993E-1,-1.9007076E-1,-3.574318E-1,8.0027975E-2,-3.7945511E-3,3.2766287E-3,-6.838066E-2,5.598663E-3,-2.234741E-3,-1.6062098E-3,1.4490457E-1,3.515733E-2,1.7769938E-2,2.4345317E-3,-2.1554193E-1,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,-8.616666E-2,7.324565E-3,2.3872483E-1,1.1299209E-1,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,-1.5753022E-1,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.572733E0,1.0585915E0,1.6673958E0,8.2653E-1,2.4398685E-1,2.8807345E-1,4.6537733E-1,2.8546977E-1,2.1166551E-1,1.4805996E-1,9.310773E-2,2.0591748E-1,0E0,0E0,1.8203974E-1,2.947247E-1,2.1712995E-1,1.3969424E-1,0E0,0E0,1.7411983E-1,0E0,0E0,0E0,8.9793384E-2,0E0,0E0,1.5928355E-1,2.97374E-1,0E0,0E0,0E0,0E0,1.2090984E-1,0E0,9.095979E-2,1.5865728E-1,0E0,0E0,0E0,1.3265473E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,20,20,24,24,27,27,28,28,33,33,35,35,36,36,40,40],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0471938E6,1.529448E-1,1.2711891E0,2.8704E4,9.74979E6,1.25440235E1,3.301E3,4.1295107E-2,1.0520055E5,4.46E2,1E0,3.359399E-1,-6.1090174E-3,9.100737E-3,1.431E4,1.1695156E5,1.1E1,2.00902E2,-3.7945511E-3,3.2766287E-3,2.65852E6,5.598663E-3,-2.234741E-3,-1.6062098E-3,8.312613E5,3.515733E-2,1.7769938E-2,1E0,2.647867E-4,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,1.1343225E6,7.324565E-3,9.616974E2,3.7724147E8,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,5.0090684E5,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"split_indices":[50,41,41,11,9,59,2,44,35,11,77,44,0,0,9,30,3,54,0,0,1,0,0,0,49,0,0,8,41,0,0,0,0,35,0,4,7,0,0,0,30,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.69E2,8E1,1.07E2,6.2E1,4.9E1,3.1E1,7.3E1,3.4E1,4.5E1,1.7E1,4.5E1,4E0,1.2E1,1.9E1,5.9E1,1.4E1,1E1,2.4E1,8E0,3.7E1,1.4E1,3E0,7E0,3.8E1,2E0,1.7E1,7E0,5.2E1,6E0,8E0,2E0,8E0,3.5E1,2E0,8E0,3E1,3E0,4E0,1.6E1,3.6E1,2.4E1,1.1E1,3E0,5E0,1.7E1,1.3E1,3.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.739959E-3,-1.0096392E-1,2.598716E-1,-1.7680193E-1,-3.243053E-2,1.19685955E-1,4.226019E-1,-8.241986E-2,-2.739108E-1,-8.830396E-2,4.9990978E-2,1.8303644E-2,8.3055995E-2,2.0394288E-3,4.746282E-1,-1.0853977E-2,-3.597446E-2,-3.5187337E-1,-1.5231963E-1,-2.7551723E-1,-2.8254518E-2,9.033917E-3,5.8722845E-3,-8.941312E-3,1.0602045E-1,6.4259046E-1,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.3898046E-2,-4.325442E-3,3.7839024E-3,-3.718894E-2,1.2833083E-1,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,29,31,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4447184E0,8.7263286E-1,1.5462923E0,7.1065116E-1,4.1887993E-1,3.9337415E-1,6.0357904E-1,3.1546003E-1,3.161056E-1,5.992862E-1,1.2694931E-1,0E0,2.597763E-1,0E0,4.780264E-1,0E0,2.3357649E-1,1.2557936E-1,1.1205655E-1,3.2829583E-1,1.5759072E-1,9.657455E-2,0E0,0E0,1.6324815E-1,1.374054E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4665566E-1,0E0,0E0,1.1835869E-1,1.3544708E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,17,17,18,18,19,19,20,20,21,21,24,24,25,25,35,35,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,30,32,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,1.6375645E2,2.0224316E-1,1.69607E5,3.2764925E5,1.2E1,1.2354571E0,4.0561585E-3,5.3597E2,3.6477748E7,7.128343E5,1.8303644E-2,1.3E1,2.0394288E-3,2.9898608E-1,-1.0853977E-2,1.2245615E0,3.9647983E2,9.714086E0,6.779745E6,3.642097E6,9.598357E0,5.8722845E-3,-8.941312E-3,9.723711E0,2.4214827E5,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.142943E1,-4.325442E-3,3.7839024E-3,8.1262054E2,1.7021878E7,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"split_indices":[54,54,40,1,30,3,55,43,4,12,30,0,3,0,40,0,55,4,49,12,34,58,0,0,59,30,0,0,0,0,0,0,0,0,0,58,0,0,4,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,1.68E2,6.9E1,7.9E1,8.9E1,3.8E1,3.1E1,4.1E1,3.8E1,5.3E1,3.6E1,3E0,3.5E1,4E0,2.7E1,8E0,3.3E1,2.2E1,1.6E1,1.2E1,4.1E1,2.5E1,1.1E1,2E0,3.3E1,1.1E1,1.6E1,7E0,2.6E1,1.2E1,1E1,2E0,1.4E1,8E0,4E0,2.4E1,1.7E1,9E0,1.6E1,3E1,3E0,4E0,7E0,1.5E1,9E0,1.1E1,5E0,2.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.7042104E-3,-7.20367E-2,2.6177105E-1,-1.2977166E-1,4.7325946E-2,1.1269311E-1,4.0182242E-1,-1.1040865E-1,-3.0936033E-1,1.5922122E-2,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,-1.6456133E-1,-5.346679E-2,-1.7062483E-2,-2.109847E-3,-2.8860811E-2,1.2831074E-1,-2.0478494E-1,5.846947E-4,8.376397E-3,-6.544534E-2,-5.855391E-3,-6.571167E-3,1.2199961E-2,1.0790331E-3,-1.0410925E-1,-2.5988343E-1,-7.7399775E-2,7.660132E-3,3.2776052E-3,-5.569639E-2,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,-5.8419913E-2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,2.7633142E-3,1.8774311E-3,-5.525428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,25,27,29,-1,-1,31,33,-1,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,-1,-1,45,-1,-1],"loss_changes":[4.8915606E0,1.3223984E0,1.1447959E0,4.1442585E-1,2.5857258E-1,1.2807718E-1,6.305504E-1,3.5506094E-1,2.3678052E-1,2.6542392E-1,0E0,0E0,0E0,0E0,0E0,4.3223488E-1,1.8600516E-1,0E0,0E0,1.0552636E-1,2.2810276E-1,2.4475026E-1,0E0,0E0,1.7779067E-1,1.3734703E-1,0E0,0E0,0E0,1.1772323E-1,1.4893365E-1,1.1551082E-1,0E0,0E0,1.1556485E-1,0E0,0E0,0E0,0E0,1.1671467E-1,0E0,0E0,0E0,0E0,1.0962085E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,15,15,16,16,19,19,20,20,21,21,24,24,25,25,29,29,30,30,31,31,34,34,39,39,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,26,28,30,-1,-1,32,34,-1,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,-1,-1,46,-1,-1],"split_conditions":[1.3394321E0,7.629E3,2.0787016E5,3.3135505E0,5.2328894E5,7.323253E8,1.623319E3,1.66863E5,1.5157113E-2,8.834E3,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,2.951829E9,1.55E2,-1.7062483E-2,-2.109847E-3,2.1445148E0,5.2638875E6,6.8618E4,5.846947E-4,8.376397E-3,6.098826E6,3.1791415E11,-6.571167E-3,1.2199961E-2,1.0790331E-3,1.3182504E0,1.7218688E8,2.2733334E1,7.660132E-3,3.2776052E-3,4.1E1,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,2.0256741E2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,4.4326835E0,1.8774311E-3,-5.525428E-3],"split_indices":[44,10,30,55,30,33,4,9,40,2,0,0,0,0,0,5,11,0,0,55,45,31,0,0,50,33,0,0,0,58,7,58,0,0,8,0,0,0,0,54,0,0,0,0,56,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.9E2,5.6E1,1.28E2,6.2E1,2.8E1,2.8E1,1.17E2,1.1E1,5.1E1,1.1E1,6E0,2.2E1,2E1,8E0,5.9E1,5.8E1,8E0,3E0,3.7E1,1.4E1,4.8E1,1.1E1,2E0,5.6E1,3.2E1,5E0,5E0,9E0,1.8E1,3E1,5.4E1,2E0,1.2E1,2E1,1.2E1,6E0,2.1E1,9E0,4.8E1,6E0,5E0,1.5E1,2.9E1,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.6167102E-2,-9.1617934E-2,2.0921354E-1,-1.5892246E-1,1.5889113E-4,1.0256023E-1,3.3971918E-1,-2.3020288E-1,-4.6520796E-2,-1.315317E-1,3.3601597E-2,1.9421874E-2,6.8316124E-2,2.9019982E-2,2.54235E-1,-7.909159E-2,-2.7484733E-1,5.3665806E-3,-8.432639E-3,-3.2230811E-3,-1.557175E-2,-2.764732E-3,5.4405066E-3,-5.779646E-2,1.18016526E-1,5.7629147E-3,3.5522467E-1,7.019195E-3,-5.448651E-3,-3.0026987E-1,3.8947738E-4,-5.0486766E-2,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,1.5366653E-1,-2.1028195E-3,1.7452618E-2,3.831069E-3,-2.4670707E-1,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,-1,-1,33,-1,35,37,-1,39,-1,-1,41,-1,43,-1,-1,-1,-1,-1,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.134416E0,1.1242383E0,8.142085E-1,8.305547E-1,3.4793866E-1,3.800948E-1,5.817108E-1,4.100237E-1,3.2077062E-1,1.8154746E-1,2.089919E-1,0E0,2.1365988E-1,0E0,2.3687482E-1,1.9577982E-1,3.561766E-1,2.3537263E-1,0E0,0E0,0E0,1.8856645E-1,0E0,8.858822E-2,1.4960828E-1,0E0,1.08366966E-1,0E0,0E0,1.7012787E-1,0E0,1.1466624E-1,0E0,0E0,0E0,0E0,0E0,9.963548E-2,0E0,0E0,0E0,1.5310645E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,21,21,23,23,24,24,26,26,29,29,31,31,37,37,41,41],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,-1,-1,34,-1,36,38,-1,40,-1,-1,42,-1,44,-1,-1,-1,-1,-1,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,1.9530877E-1,1.80025E2,1.8681856E8,8.9397516E4,1.8681E4,1.433458E10,2.142943E1,2.151E3,2.704375E6,1.9421874E-2,1.479938E6,2.9019982E-2,6.7E1,2.7287993E0,3.402546E6,6.135367E9,-8.432639E-3,-3.2230811E-3,-1.557175E-2,9.143112E5,5.4405066E-3,9.71709E5,2.30273E6,5.7629147E-3,6.3898524E7,7.019195E-3,-5.448651E-3,4.1938826E-2,3.8947738E-4,9.165388E4,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,5.7E1,-2.1028195E-3,1.7452618E-2,3.831069E-3,1.0142518E0,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"split_indices":[54,43,40,54,7,35,9,33,58,2,9,0,34,0,8,56,1,5,0,0,0,35,0,31,9,0,34,0,0,44,0,35,0,0,0,0,0,8,0,0,0,60,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.81E2,6E1,1.04E2,7.7E1,3.4E1,2.6E1,6.3E1,4.1E1,1.5E1,6.2E1,2E0,3.2E1,4E0,2.2E1,1.5E1,4.8E1,3.1E1,1E1,1.3E1,2E0,4.5E1,1.7E1,9E0,2.3E1,1.1E1,1.1E1,2E0,1.3E1,4.4E1,4E0,2.2E1,9E0,2.2E1,2.3E1,2E0,7E0,1.9E1,4E0,9E0,2E0,3.2E1,1.2E1,5E0,1.7E1,7E0,1.2E1,2.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[6.0046404E-3,-9.218389E-2,2.1822302E-1,-1.7119181E-1,2.2936372E-2,1.080785E-1,3.675177E-1,-2.2839919E-1,-7.109087E-2,4.932593E-2,-3.4302478E-3,1.885684E-2,8.236004E-2,1.287323E-1,4.4497648E-1,-2.0051514E-1,-2.1602163E-2,4.6547647E-3,-1.2568788E-1,-1.1251967E-2,4.7163595E-3,1.5830024E-1,1.2744217E-2,9.789093E-3,3.2283113E-4,3.5435382E-1,3.0562386E-2,-1.4268523E-2,-1.7262784E-1,-6.920345E-3,9.3187834E-4,-4.0626727E-2,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,5.3271845E-2,1.711268E-2,5.001436E-3,-1.4809307E-1,-1.3950878E-2,-6.9640996E-3,7.891728E-3,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,29,31,-1,33,35,-1,-1,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8752337E0,1.4591926E0,1.1771345E0,5.277815E-1,1.8073346E-1,3.3109027E-1,5.1609373E-1,3.4681034E-1,3.6278534E-1,1.8984374E-1,0E0,0E0,2.2002146E-1,1.0272367E-1,2.8410387E-1,1.4103603E-1,0E0,0E0,1.3881755E-1,1.4883459E-1,0E0,1.2233335E-1,2.0214176E-1,0E0,0E0,1.2719083E-1,0E0,0E0,1.2584579E-1,0E0,0E0,1.397791E-1,0E0,0E0,0E0,0E0,9.9094376E-2,0E0,0E0,9.3803644E-2,0E0,0E0,9.208519E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,19,19,21,21,22,22,25,25,28,28,31,31,36,36,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,30,32,-1,34,36,-1,-1,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6984555E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.260197E0,8.9397516E4,2.2324745E5,5.6835636E-2,8.938651E4,1.7573398E2,-3.4302478E-3,1.885684E-2,1.3844932E7,2.81002E5,1.2461E4,1.7565872E-3,-2.1602163E-2,4.6547647E-3,1.7725672E8,2.424E3,4.7163595E-3,1.8990228E0,1.312252E6,9.789093E-3,3.2283113E-4,4.7201157E-1,3.0562386E-2,-1.4268523E-2,1.269689E1,-6.920345E-3,9.3187834E-4,8E0,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,2.0215728E0,1.711268E-2,5.001436E-3,8.938651E4,-1.3950878E-2,-6.9640996E-3,2.176974E8,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"split_indices":[54,43,40,54,55,35,30,59,35,54,0,0,47,9,2,60,0,0,12,2,0,55,31,0,0,40,0,0,58,0,0,3,0,0,0,0,37,0,0,35,0,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.59E2,7.3E1,9.4E1,6.5E1,4.3E1,3E1,5.9E1,3.5E1,5.2E1,1.3E1,2E0,4.1E1,8E0,2.2E1,5.5E1,4E0,8E0,2.7E1,2.6E1,2.6E1,1.9E1,2.2E1,4E0,4E0,1.8E1,4E0,8E0,4.7E1,2.2E1,5E0,2.3E1,3E0,1.3E1,6E0,3E0,1.9E1,1.5E1,3E0,4.2E1,5E0,6E0,1.7E1,1.6E1,3E0,1.7E1,2.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.3128728E-2,-7.6841384E-2,2.5555712E-1,-1.3273934E-1,6.3761264E-2,8.655129E-2,4.008202E-1,-1.6426608E-1,-1.455439E-2,1.4814147E-2,3.6547862E-2,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,-1.4727283E-1,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,5.5398457E-2,-4.1562365E-3,-1.9884297E-1,-8.877025E-2,-5.379106E-4,9.418675E-2,-2.5566745E-1,-8.6616755E-2,-7.036469E-4,-1.4556472E-1,5.3083412E-3,-5.8485573E-4,-2.0745803E-1,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,-7.540202E-2,-9.3988655E-3,-2.2761422E-1,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,-1,25,27,-1,29,31,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,-1,43,-1,-1,-1,-1,-1],"loss_changes":[4.1965976E0,1.5646251E0,1.1191728E0,5.266993E-1,4.1526425E-1,1.3371919E-1,5.893326E-1,3.3139777E-1,9.107072E-2,0E0,1.3694459E-1,0E0,0E0,0E0,0E0,2.9913235E-1,0E0,0E0,0E0,1.246268E-1,0E0,3.2860088E-1,2.5498408E-1,0E0,9.024802E-2,4.5705009E-1,9.878099E-2,1.712665E-1,1.3560718E-1,0E0,0E0,1.3404357E-1,0E0,0E0,0E0,0E0,0E0,1.554379E-1,0E0,9.5205545E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,31,31,37,37,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,-1,26,28,-1,30,32,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,-1,44,-1,-1,-1,-1,-1],"split_conditions":[1.4272063E0,3.165913E6,2.174459E5,7.087292E-2,2.207E3,7.002603E1,1.6507233E3,1E0,1.1721691E5,1.4814147E-2,3.605082E7,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,1.73507E5,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,1.462136E6,-4.1562365E-3,1.3738942E0,1.833E5,-5.379106E-4,1.9879582E7,1.0665042E8,1.2288136E1,1.1942446E0,1.2271346E1,5.3083412E-3,-5.8485573E-4,8.672049E9,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,9.3E4,-9.3988655E-3,6.7001674E-3,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"split_indices":[44,1,30,40,0,58,4,77,30,0,47,0,0,0,0,9,0,0,0,34,0,55,31,0,34,7,58,55,58,0,0,5,0,0,0,0,0,12,0,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.97E2,4.6E1,1.41E2,5.6E1,2.2E1,2.4E1,1.11E2,3E1,4E0,5.2E1,1.9E1,3E0,1.6E1,8E0,1.04E2,7E0,5E0,2.5E1,4.6E1,6E0,5.4E1,5E1,1.7E1,2.9E1,3.5E1,1.9E1,2E1,3E1,2.3E1,6E0,3.2E1,3E0,1E1,9E0,3E0,1.7E1,1.6E1,1.4E1,2.9E1,3E0,2E0,1.4E1,8E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.4699904E-3,-8.2270786E-2,1.7413537E-1,-1.3975397E-1,3.89982E-3,7.456945E-2,3.0463025E-1,-1.7296548E-1,6.0730726E-3,-9.173903E-2,3.4571238E-2,1.801554E-2,4.9235556E-2,1.7605738E-3,3.2971406E-1,-2.1693262E-1,-1.0940522E-1,7.686381E-2,-3.4685563E-3,-2.7480236E-1,-8.64333E-4,1.01417145E-5,4.4262665E-3,-7.801436E-2,7.197917E-2,8.4248185E-3,3.7063202E-1,-1.9041932E-1,-2.0984167E-2,-4.680374E-2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,2.399795E-2,3.9364552E-1,4.087419E-3,-1.0795465E-3,-2.0910926E-1,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,5.9234392E-2,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5455477E0,8.9215004E-1,9.777329E-1,5.2410936E-1,2.1730821E-1,3.6530188E-1,2.0546484E-1,2.22188E-1,1.3329768E-1,2.3158492E-1,1.2854296E-1,0E0,1.3006273E-1,0E0,9.619784E-2,2.65311E-1,1.5102664E-1,2.2796245E-1,0E0,1.5092689E-1,0E0,0E0,0E0,1.5338422E-1,1.3331974E-1,0E0,1.03357315E-1,1.4346242E-1,0E0,1.5426993E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3046479E-1,9.462261E-2,0E0,0E0,1.1954689E-1,0E0,0E0,0E0,9.6233115E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,19,19,23,23,24,24,26,26,27,27,29,29,38,38,39,39,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.8509178E-1,1.8829846E-1,2.219883E6,1.8681856E8,8.9397516E4,1E0,5.283708E0,1.8000048E10,8E0,2.069532E6,1.801554E-2,7.5E2,1.7605738E-3,2.2324745E5,5.144033E-2,1.2288136E1,3.131673E6,-3.4685563E-3,3.56E2,-8.64333E-4,1.01417145E-5,4.4262665E-3,3.13E2,8.312613E5,8.4248185E-3,1.8373182E8,6E0,-2.0984167E-2,6.0515753E2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,6.925995E6,1.7598395E3,4.087419E-3,-1.0795465E-3,7.5646E4,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,2.8812434E7,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"split_indices":[54,41,40,1,7,35,18,58,5,3,9,0,0,0,30,59,58,31,0,0,0,0,0,0,49,0,34,10,0,4,0,0,0,0,0,0,0,0,53,4,0,0,9,0,0,0,47,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.79E2,7.6E1,1.07E2,7.2E1,4.4E1,3.2E1,8.7E1,2E1,1.7E1,5.5E1,2E0,4.2E1,3E0,2.9E1,5E1,3.7E1,1.1E1,9E0,4E0,1.3E1,3.7E1,1.8E1,6E0,3.6E1,8E0,2.1E1,4.7E1,3E0,2E1,1.7E1,7E0,4E0,2E0,2E0,2E0,4E0,1.3E1,2.3E1,1.9E1,2E0,5E0,4.2E1,9E0,1.1E1,4E0,1.9E1,1.4E1,5E0,2.4E1,1.8E1,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.0756515E-4,-9.093915E-2,1.3531232E-1,-1.3121241E-1,3.0161556E-2,6.471361E-2,2.7240935E-1,-1.5608981E-1,-4.138077E-2,5.6453147E-3,-1.665546E-3,1.331529E-2,4.4281885E-2,1.4165652E-4,2.9921746E-1,-1.3490759E-1,-3.431053E-1,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,1.077188E-1,-1.1128485E-2,1.87167E-2,1.0148881E-2,-1.18122905E-1,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,5.707247E-2,-4.600261E-3,3.8549118E-2,-1.6461147E-1,-4.9828332E-2,6.6000526E-3,6.4204754E-3,4.2778063E-3,-2.1197742E-3,-2.1401511E-1,-2.0505309E-3,7.0574425E-2,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,35,-1,37,39,41,-1,43,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9934402E0,7.1114016E-1,9.309007E-1,2.3713171E-1,1.1703685E-1,3.1068093E-1,2.37149E-1,2.9043174E-1,1.1233303E-1,0E0,1.0321621E-1,0E0,2.1913269E-1,0E0,1.5956187E-1,2.0051861E-1,2.3705983E-1,0E0,0E0,0E0,0E0,2.0704418E-1,1.6331983E-1,0E0,0E0,2.2356677E-1,0E0,0E0,0E0,0E0,1.08736545E-1,0E0,1.2394792E-1,2.3981667E-1,1.24148294E-1,0E0,9.681947E-2,0E0,0E0,2.0636463E-1,0E0,8.820005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,21,21,22,22,25,25,30,30,32,32,33,33,34,34,36,36,39,39,41,41],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,36,-1,38,40,42,-1,44,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.5343175E0,2.2348747E2,1.0632315E6,8.9397516E4,1E0,2.8659306E0,2.2616908E2,5.6453147E-3,7.432E3,1.331529E-2,1.1572553E7,1.4165652E-4,3.2307692E0,3.0962145E0,1.020435E4,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,4.0012E4,1.4813267E3,1.87167E-2,1.0148881E-2,1.73507E5,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,7E0,-4.600261E-3,8.303E3,2.951829E9,4.4E1,6.6000526E-3,1.9816086E0,4.2778063E-3,-2.1197742E-3,1.42E2,-2.0505309E-3,7.510128E7,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"split_indices":[54,40,43,54,50,35,18,60,54,0,10,0,47,0,56,55,34,0,0,0,0,9,57,0,0,9,0,0,0,0,3,0,0,5,10,0,55,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.44E2,9.7E1,1.08E2,3.6E1,6.5E1,3.2E1,8.4E1,2.4E1,8E0,2.8E1,4E0,6.1E1,3E0,2.9E1,7.7E1,7E0,2E0,2.2E1,2.4E1,4E0,2.8E1,3.3E1,8E0,2.1E1,7.2E1,5E0,3E0,4E0,6E0,2.2E1,1.1E1,2.2E1,4.2E1,3E1,7E0,1.5E1,1.3E1,9E0,2.9E1,1.3E1,6E0,2.4E1,1.1E1,4E0,6E0,2.3E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.0671469E-2,-4.057193E-2,2.5710663E-1,-9.11908E-2,7.657581E-2,1.0005701E-1,3.5594457E-1,-1.3330418E-1,3.5337575E-2,6.0731232E-2,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,-1.425013E-1,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,-2.7416292E-1,-1.2234801E-1,-3.3937553E-3,-2.139736E-2,-1.513492E-1,-3.387177E-2,-1.1739622E-1,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,-8.5462265E-2,-4.7841365E-3,7.8635005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,27,29,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[3.1548471E0,1.2344693E0,6.318512E-1,7.7734864E-1,2.7344996E-1,1.175154E-1,3.9797258E-1,1.8897259E-1,1.1001776E-1,1.5682302E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.4241185E-1,0E0,0E0,0E0,0E0,5.137621E-1,2.3093772E-1,0E0,0E0,1.8179822E-1,9.140951E-2,1.8776286E-1,0E0,0E0,0E0,0E0,1.0951534E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,16,16,21,21,22,22,25,25,26,26,27,27,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,28,30,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.5648934E0,3.165913E6,2.174459E5,6.408734E-2,1.4122614E2,1E0,1.7598395E3,1E1,1.599097E0,1.25440235E1,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,1.0187571E5,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,8.3E1,1.1670399E-1,-3.3937553E-3,-2.139736E-2,3.671916E0,1E0,1E0,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,1.8E1,-4.7841365E-3,7.8635005E-4],"split_indices":[43,1,30,40,58,8,4,10,55,59,0,0,0,0,0,0,30,0,0,0,0,10,41,0,0,56,100,102,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,2.06E2,4.2E1,1.44E2,6.2E1,1.7E1,2.5E1,1.08E2,3.6E1,6E1,2E0,7E0,1E1,1.9E1,6E0,5E0,1.03E2,2.2E1,1.4E1,5E1,1E1,1.2E1,9.1E1,7E0,5E0,6.8E1,2.3E1,5.1E1,1.7E1,2E0,2.1E1,9E0,4.2E1,3.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.4315777E-2,-1.0690338E-1,1.1009746E-1,-1.4777888E-1,2.0821875E-2,2.117857E-2,1.7040308E-1,-1.806778E-1,-5.4927632E-2,2.9963165E-4,6.637665E-3,3.901988E-2,-4.5500887E-3,7.291719E-2,2.6567936E-1,-1.4094864E-1,-2.987772E-1,-2.4235064E-4,-2.5972742E-1,8.2612615E-3,-1.8001318E-2,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,9.182606E-2,1.6551235E-1,2.0925388E-2,-2.1879025E-1,-9.1945045E-2,-3.4581E-1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,-1,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9952228E0,7.809812E-1,5.9160745E-1,3.3525777E-1,9.964019E-2,1.07021526E-1,5.936415E-1,3.473227E-1,3.1202346E-1,1.1755685E-1,0E0,9.129217E-2,0E0,1.14367574E-1,6.274619E-1,2.2401261E-1,1.6717279E-1,0E0,9.88501E-2,0E0,1.3668847E-1,0E0,0E0,0E0,1.5995112E-1,2.5624245E-1,0E0,1.0733318E-1,1.4268285E-1,2.0528078E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,18,18,20,20,24,24,25,25,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,-1,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.1677386E-1,2.796589E6,3.4021906E2,1.1670399E-1,5.5446387E2,1.9E1,2.174459E5,5.740651E-3,3.4047124E-1,2.207E3,6.637665E-3,2.0358758E8,-4.5500887E-3,1.9241573E-1,7.2259143E2,4.949154E-4,5.906883E5,-2.4235064E-4,2.07298E5,8.2612615E-3,1E1,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,8.303E3,1.9816086E0,2.0925388E-2,1.7565872E-3,4.8831225E5,7.5E1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"split_indices":[44,1,54,41,54,3,30,40,44,0,0,7,0,59,54,41,35,0,1,0,3,0,0,0,0,55,0,60,30,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.48E2,1.1E2,1.12E2,3.6E1,4.5E1,6.5E1,8.2E1,3E1,3.2E1,4E0,4E1,5E0,3.3E1,3.2E1,6.3E1,1.9E1,2.5E1,5E0,2E0,3E1,1E1,3E1,3E0,3E1,2.3E1,9E0,2.3E1,4E1,1.5E1,4E0,3E0,2E0,1E1,2E1,2E1,1E1,1.8E1,5E0,8E0,1.5E1,3.1E1,9E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.5204665E-2,-6.9504075E-2,1.5126899E-1,-1.1839543E-1,2.2287078E-2,7.562342E-2,2.5493225E-1,-1.4351347E-1,-1.2403568E-2,-5.980414E-3,3.6651608E-2,5.731069E-2,9.66742E-3,6.0085384E-3,3.3954874E-1,-1.3105348E-2,-1.2924905E-1,3.58532E-2,-7.821619E-3,1.6858906E-2,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,-9.744661E-2,-1.9808635E-1,4.93121E-3,-2.631804E-3,3.7851118E-2,-3.5518215E-3,-1.1887726E-1,1.7840436E-3,-5.2316664E-3,-2.911272E-1,6.4006194E-2,-1.814889E-3,-1.4108726E-1,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.0247637E-1,-1.5689014E-1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1],"loss_changes":[2.3878818E0,8.9570975E-1,4.904294E-1,3.4431982E-1,1.6523217E-1,9.151134E-2,2.2111058E-1,1.9603395E-1,2.1574554E-1,0E0,1.2080229E-1,1.1728562E-1,0E0,0E0,9.910309E-2,0E0,1.9523811E-1,1.6260572E-1,0E0,1.1651751E-1,0E0,0E0,0E0,0E0,0E0,2.094084E-1,1.8337464E-1,0E0,0E0,9.8238766E-2,0E0,1.4792949E-1,0E0,0E0,2.5384462E-1,1.126506E-1,0E0,1.1237788E-1,0E0,0E0,0E0,0E0,8.794698E-2,1.0356009E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,16,16,17,17,19,19,25,25,26,26,29,29,31,31,34,34,35,35,37,37,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1],"split_conditions":[1.2185057E0,1.781973E6,2.0787016E5,6.808926E6,9.315286E1,4.328683E0,3.778E3,3.4903356E7,1.409674E7,-5.980414E-3,4.43001E5,1.1832633E0,9.66742E-3,6.0085384E-3,4.7201157E-1,-1.3105348E-2,4.352025E0,4.8593006E5,-7.821619E-3,1.939974E7,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,3.9376082E0,8.287425E6,4.93121E-3,-2.631804E-3,1.25440235E1,-3.5518215E-3,2.2E1,1.7840436E-3,-5.2316664E-3,4.752366E0,5.6024827E9,-1.814889E-3,2.427E3,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.1770233E0,4.2E1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"split_indices":[43,1,30,9,54,37,2,7,47,0,30,41,0,0,40,0,56,35,0,34,0,0,0,0,0,56,47,0,0,59,0,3,0,0,58,5,0,2,0,0,0,0,56,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.62E2,1.98E2,6.4E1,1.29E2,6.9E1,3.8E1,2.6E1,1.04E2,2.5E1,5E0,6.4E1,3.5E1,3E0,1.2E1,1.4E1,7E0,9.7E1,2E1,5E0,5.4E1,1E1,8E0,2.7E1,1.2E1,2E0,6.8E1,2.9E1,1.1E1,9E0,4.5E1,9E0,5.9E1,9E0,1.7E1,1.2E1,3.4E1,1.1E1,4.9E1,1E1,4E0,8E0,1.1E1,2.3E1,4.4E1,5E0,3E0,2E1,3.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.1777132E-3,-6.0567748E-2,1.9037953E-1,-1.3071354E-1,5.439912E-4,8.848939E-2,2.9270196E-1,-1.7778724E-1,-1.4740403E-3,-7.950044E-2,3.19054E-2,3.1027812E-4,1.2356255E-1,2.2399522E-2,2.3053867E-1,-1.3397929E-1,-2.7813753E-1,1.4052213E-4,-1.3808084E-1,7.668798E-2,-2.3035506E-2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.975542E0,8.102633E-1,6.309147E-1,3.9554834E-1,2.5858772E-1,9.331095E-2,3.067577E-1,2.2210789E-1,0E0,1.4097291E-1,1.838358E-1,0E0,1.22035086E-1,0E0,9.3972206E-2,2.5386703E-1,9.3033075E-2,0E0,1.3659179E-1,1.5234712E-1,1.16067775E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,1.9415865E2,2.174459E5,1.848519E-2,8.461558E10,5.958114E4,1.8681E4,1.607E3,-1.4740403E-3,1.2271346E1,2.305287E1,3.1027812E-4,1.718354E8,2.2399522E-2,3.778E3,6.581686E6,6.608892E2,1.4052213E-4,4.6955062E5,1.2119408E3,6.657047E2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"split_indices":[44,54,30,40,33,30,9,2,0,58,58,0,7,0,2,9,4,0,35,4,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.88E2,6.2E1,8.7E1,1.01E2,3.2E1,3E1,5.8E1,2.9E1,2.8E1,7.3E1,1E1,2.2E1,5E0,2.5E1,4.2E1,1.6E1,1.2E1,1.6E1,4E1,3.3E1,6E0,1.6E1,1.1E1,1.4E1,4E1,2E0,1.2E1,4E0,4E0,1.2E1,3E1,1E1,2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.9849198E-3,-6.908789E-2,1.1069185E-1,-1.1135524E-1,4.8647843E-2,1.8852478E-2,1.622326E-1,-9.491666E-2,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,5.3762157E-2,2.2892316E-1,6.1087083E-2,-1.4048992E-1,-5.2776527E-2,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,-1.5545653E-1,-1.0782661E-3,-8.035333E-3,-2.0769975E-1,-9.872578E-2,-2.4276859E-1,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1],"loss_changes":[1.8784683E0,7.269919E-1,4.5608807E-1,2.1379781E-1,3.8304985E-1,1.0936768E-1,4.042337E-1,1.7887038E-1,0E0,0E0,0E0,0E0,9.703164E-2,2.1851015E-1,1.605283E-1,2.029301E-1,1.871668E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.09901786E-1,0E0,0E0,1.7119968E-1,1.0259637E-1,1.2492943E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,14,14,15,15,16,16,24,24,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1],"split_conditions":[4.259334E-1,3.343327E6,3.1259552E2,3.7979183E0,2.207E3,2.1123085E8,1.0568718E7,1.3466058E0,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,1.6070912E0,1.2461E4,1.25440235E1,4.6376264E9,2.2733334E1,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,6.7329E4,-1.0782661E-3,-8.035333E-3,8.640031E9,1.1710652E0,4.19891E0,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"split_indices":[44,1,54,55,0,7,47,55,0,0,0,0,60,2,59,33,58,0,0,0,0,0,0,0,9,0,0,5,55,58,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.44E2,9.6E1,1.06E2,3.8E1,3.5E1,6.1E1,9.6E1,1E1,3E0,3.5E1,1E1,2.5E1,3.6E1,2.5E1,4.5E1,5.1E1,3E0,2.2E1,3.2E1,4E0,2E1,5E0,2E0,4.3E1,4.3E1,8E0,2.1E1,2.2E1,1.8E1,3E0,8E0,1.4E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0547296E-2,-4.2344254E-2,1.742515E-1,-1.0237627E-1,3.5556585E-2,2.740988E-1,7.519926E-2,-7.69858E-2,-2.2364104E-1,7.678524E-3,2.2331232E-2,1.4626542E-1,3.7681144E-1,4.2196088E-2,8.162208E-3,-4.9231295E-2,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,4.149654E-2,-7.011419E-4,9.022605E-3,4.7000867E-1,6.24682E-3,-3.9323145E-3,3.069897E-3,-7.009031E-2,6.090464E-2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.163362E0,8.8750696E-1,5.826844E-1,3.122326E-1,1.5018767E-1,3.3878827E-1,1.1022253E-1,2.4382734E-1,2.6022464E-1,0E0,1.0540541E-1,1.5061551E-1,2.8114796E-1,1.0317968E-1,0E0,1.6967641E-1,0E0,0E0,0E0,0E0,1.09696545E-1,0E0,0E0,9.1567755E-2,0E0,0E0,0E0,1.7041293E-1,1.1754744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,20,20,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.5971466E2,2.7558826E-2,9.8110194E5,1.2460274E-2,2.3755E4,7.116346E2,2.0787016E5,3.774648E0,5.906883E5,7.678524E-3,1.496E3,2.4894228E6,2.660842E6,1.5E1,8.162208E-3,3.901529E6,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,1.6815423E5,-7.011419E-4,9.022605E-3,5.505334E9,6.24682E-3,-3.9323145E-3,3.069897E-3,2.797224E2,2.1374529E2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"split_indices":[54,40,35,40,9,54,30,56,35,0,2,34,31,3,0,9,0,0,0,0,30,0,0,5,0,0,0,54,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,1.88E2,6E1,1.06E2,8.2E1,2.9E1,3.1E1,8.9E1,1.7E1,6E0,7.6E1,1.4E1,1.5E1,2.5E1,6E0,7.1E1,1.8E1,8E0,9E0,1.6E1,6E1,4E0,1E1,1E1,5E0,4E0,2.1E1,6E1,1.1E1,2.2E1,3.8E1,6E0,4E0,5.4E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.9339443E-3,-1.0605051E-1,7.4423306E-2,-1.4161547E-1,-2.8023109E-2,5.737777E-2,3.6554796E-1,-1.08674295E-1,-2.4172173E-1,-5.5312406E-2,5.1337723E-3,2.0626027E-2,1.1801036E-1,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,-2.894412E-1,4.924759E-4,-5.1691607E-3,-2.6259364E-3,4.9695414E-2,4.6624467E-4,1.5626377E-1,-1.5225716E-2,-4.396114E-3,1.359324E-1,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.0659106E0,3.0406094E-1,7.134572E-1,2.243501E-1,1.496107E-1,3.117638E-1,1.7670429E-1,1.5272987E-1,2.0493579E-1,1.3169232E-1,0E0,2.1203475E-1,2.1592683E-1,0E0,0E0,0E0,0E0,0E0,1.3315952E-1,0E0,0E0,0E0,1.3145865E-1,0E0,1.2719214E-1,0E0,0E0,8.859834E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,18,18,22,22,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.7613442E-1,2.8704E4,4.328683E0,5.1653096E-3,1.22E2,1.2446029E0,2.174459E5,1.0665042E8,3.7274265E2,2.8488008E11,5.1337723E-3,1.805E3,1.7468556E2,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,2.1710082E7,4.924759E-4,-5.1691607E-3,-2.6259364E-3,1.325E0,4.6624467E-4,1.1560919E3,-1.5225716E-2,-4.396114E-3,8.8356856E5,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"split_indices":[43,11,37,40,8,44,30,7,4,33,0,2,54,0,0,0,0,0,34,0,0,0,55,0,54,0,0,49,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.1E2,1.47E2,7.5E1,3.5E1,1.4E2,7E0,5.8E1,1.7E1,3E1,5E0,8.8E1,5.2E1,4E0,3E0,5.3E1,5E0,3E0,1.4E1,1.5E1,1.5E1,2.3E1,6.5E1,1.4E1,3.8E1,1E1,4E0,1.3E1,5.2E1,3.5E1,3E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.345121E-3,-6.562803E-2,1.1551625E-1,-8.816371E-2,4.981665E-4,3.869827E-2,2.208219E-1,-1.4935678E-1,-4.5134716E-2,1.6114242E-2,6.035789E-3,2.2871085E-2,1.6961017E-1,-2.867197E-1,-1.2161767E-1,-6.569096E-2,3.5917396E-3,-3.0824807E-3,5.086014E-2,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,-1.0346314E-1,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.6731131E0,2.8574848E-1,5.9006786E-1,3.276595E-1,0E0,9.675721E-2,3.9257836E-1,1.6326678E-1,2.0450398E-1,1.1511861E-1,0E0,0E0,1.3534534E-1,3.4041297E-1,1.645894E-1,9.34082E-2,0E0,0E0,1.10176116E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0852566E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[3.7136658E2,9.1318935E-2,1.8829846E-1,4.426498E0,4.981665E-4,1E0,1.431E4,9.975593E4,6.767991E5,1.312252E6,6.035789E-3,2.2871085E-2,1.1207349E0,2.83E2,5.22241E5,1.7012288E8,3.5917396E-3,-3.0824807E-3,8.303E3,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,2.22105E5,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"split_indices":[54,40,40,58,0,105,9,30,30,31,0,0,55,0,12,7,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.63E2,7.3E1,1.26E2,3.7E1,4.3E1,3E1,5.1E1,7.5E1,3.6E1,7E0,3E0,2.7E1,7E0,4.4E1,6.5E1,1E1,1E1,2.6E1,3E0,2.4E1,4E0,3E0,2.6E1,1.8E1,3.3E1,3.2E1,1.7E1,9E0,2E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.5972516E-3,-6.415109E-2,1.004628E-1,-9.126678E-2,7.985584E-5,7.822119E-2,3.8027394E-1,-1.3547492E-1,-1.7177572E-2,1.2492152E-1,-1.8076582E-2,2.4130752E-2,6.7940415E-3,-2.0095895E-1,-7.504207E-2,3.5000306E-3,-4.5314096E-2,2.2494642E-1,8.771981E-2,-6.7835734E-3,8.8194094E-4,-1.3448668E-1,-3.6327535E-1,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,-8.5903674E-2,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[1.4975804E0,2.8284806E-1,4.9924803E-1,3.645115E-1,0E0,3.6202925E-1,1.4661348E-1,2.6193678E-1,1.21453926E-1,1.8120694E-1,1.4514734E-1,0E0,0E0,3.07199E-1,2.7256477E-1,0E0,1.0324016E-1,1.1432767E-1,8.934039E-2,0E0,0E0,1.8839267E-1,1.3768709E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.900878E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,17,17,18,18,21,21,22,22,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[3.4021906E2,5.530166E-2,2.069929E3,1.6408995E11,7.985584E-5,1.8018788E7,5.1145615E3,1.386452E0,2.6017979E11,8E0,1.628E3,2.4130752E-2,6.7940415E-3,1.609E3,3.9069E5,3.5000306E-3,1E0,4.0123675E-2,4.0012E4,-6.7835734E-3,8.8194094E-4,1.393E3,4.7201145E10,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,1.1710652E0,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"split_indices":[54,40,54,33,0,47,4,55,33,3,0,0,0,2,31,0,101,40,9,0,0,12,33,0,0,0,0,0,0,0,0,0,55,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.57E2,8.4E1,1.11E2,4.6E1,7.9E1,5E0,6.9E1,4.2E1,5.3E1,2.6E1,2E0,3E0,3.2E1,3.7E1,9E0,3.3E1,1.3E1,4E1,5E0,2.1E1,2.4E1,8E0,3.1E1,6E0,2.5E1,8E0,2E0,1.1E1,9E0,3.1E1,4E0,2E1,4E0,4E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.0949485E-4,-7.023364E-2,1.07531965E-1,-9.5697366E-2,5.999637E-3,4.481289E-2,2.2272988E-1,-1.2741327E-1,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,3.131387E-2,6.709275E-3,1.5677929E-2,-1.9639476E-1,-7.4869595E-2,-1.0507656E-2,5.4045306E-3,-2.6656508E-1,-3.303782E-3,-3.9352458E-2,-1.0309419E-2,-5.7465717E-2,3.690726E-2,-8.487278E-3,-2.1258341E-2,-5.933776E-2,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,19,21,23,-1,25,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9060686E0,2.956339E-1,7.1252954E-1,3.6945975E-1,9.6911065E-2,2.3330902E-1,2.7442348E-1,3.0063093E-1,0E0,0E0,0E0,0E0,2.4897242E-1,0E0,0E0,2.9316032E-1,2.830122E-1,1.0231397E-1,0E0,2.7091873E-1,0E0,9.987865E-2,0E0,1.0716895E-1,1.2748241E-1,0E0,0E0,1.180381E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,15,15,16,16,17,17,19,19,21,21,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,20,22,24,-1,26,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.2711891E0,1.769817E6,9.750871E6,3.4903356E7,1.6507233E3,7.5646E4,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,2.8041378E5,6.709275E-3,1.5677929E-2,1.3738942E0,3.0962145E0,1.0685258E6,5.4045306E-3,6.4146477E-3,-3.303782E-3,2.0766662E6,-1.0309419E-2,2.0250492E0,2.51838E6,-8.487278E-3,-2.1258341E-2,4.6376264E9,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"split_indices":[54,40,41,1,9,7,4,9,0,0,0,0,30,0,0,55,55,35,0,40,0,50,0,59,34,0,0,33,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.51E2,9.9E1,1.13E2,3.8E1,6.5E1,3.4E1,8.6E1,2.7E1,2.8E1,1E1,2E0,6.3E1,2.4E1,1E1,3.6E1,5E1,4.4E1,1.9E1,2.2E1,1.4E1,4.2E1,8E0,2.2E1,2.2E1,1.8E1,4E0,3.6E1,6E0,1.6E1,6E0,9E0,1.3E1,4E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.23015465E-2,-3.6813665E-2,1.6186102E-1,-8.2913704E-2,3.2519907E-2,3.2985918E-3,2.6083997E-1,-9.4713494E-2,1.0172514E-3,2.072838E-2,1.34672765E-2,7.684792E-3,4.2016974E-1,-1.21286646E-1,-3.079713E-2,8.215429E-2,-2.6565844E-2,8.788486E-3,2.4215858E-2,-3.9600227E-3,-1.8518592E-1,1.9243775E-3,-4.5282864E-3,6.782496E-3,1.653277E-2,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8129876E0,5.9720445E-1,5.011734E-1,1.4259148E-1,2.3776487E-1,0E0,3.040551E-1,1.6780907E-1,0E0,2.1425411E-1,0E0,0E0,1.535585E-1,1.2175703E-1,1.739935E-1,1.5128689E-1,8.827151E-2,0E0,0E0,0E0,9.1700196E-2,0E0,0E0,0E0,9.928738E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,12,12,13,13,14,14,15,15,16,16,20,20,24,24],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,2.427E3,2.2324745E5,6.4796996E-1,1.4122614E2,3.2985918E-3,1.7598395E3,2.0256741E2,1.0172514E-3,1.3844932E7,1.34672765E-2,7.684792E-3,2.4394053E9,1.607E3,7.336684E2,1.6254681E0,6.925995E6,8.788486E-3,2.4215858E-2,-3.9600227E-3,1.7842995E8,1.9243775E-3,-4.5282864E-3,6.782496E-3,4.488E3,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"split_indices":[44,2,30,41,58,0,4,54,0,47,0,0,5,2,4,55,53,0,0,0,7,0,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.85E2,6E1,1.11E2,7.4E1,3.3E1,2.7E1,1E2,1.1E1,7.2E1,2E0,1.9E1,8E0,7E1,3E1,3.1E1,4.1E1,4E0,4E0,4.9E1,2.1E1,1.5E1,1.5E1,1.4E1,1.7E1,2.5E1,1.6E1,1.3E1,8E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.4086864E-3,-5.6603998E-2,1.0975099E-1,-1.0331851E-1,7.5110937E-3,8.584382E-2,1.7369809E-2,-1.5483831E-1,-2.4599966E-2,-6.041252E-3,1.6805973E-2,4.8704322E-2,1.6501543E-1,-1.18140824E-1,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.352543E-2,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.7069752E-2,3.0004673E-3,-3.8697182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.617809E0,4.5603204E-1,6.2057626E-1,3.530805E-1,9.050379E-2,2.5365812E-1,0E0,3.3182883E-1,1.1731397E-1,0E0,1.0263331E-1,2.5914997E-1,2.4869812E-1,9.6805155E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2964478E-1,0E0,0E0,0E0,0E0,0E0,8.78565E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,20,20,26,26],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[3.4021906E2,1.1670399E-1,1.3821641E0,1.6796057E2,8.89E2,2.968068E5,1.7369809E-2,7.33452E-3,2.939566E5,-6.041252E-3,1.10018056E8,8.9397516E4,2.3842E4,6E0,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.0327642E-1,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.615313E3,3.0004673E-3,-3.8697182E-3],"split_indices":[54,41,36,54,2,30,0,40,35,0,34,35,9,10,0,0,0,0,0,0,44,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,8.7E1,6.4E1,8.8E1,6E0,5.2E1,3.5E1,3E0,6.1E1,6.1E1,2.7E1,4.5E1,7E0,1.7E1,1.8E1,5.5E1,6E0,2E0,5.9E1,5E0,2.2E1,5E0,4E1,1.2E1,4.7E1,4.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.5817066E-3,-3.926012E-2,1.1888153E-1,-7.5602576E-2,5.771031E-2,6.1283264E-2,8.008875E-3,-9.3968615E-2,8.966747E-3,1.21591035E-2,3.8098045E-2,8.102908E-3,1.4816463E-3,-1.2472329E-1,-4.609013E-2,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,-1.0609691E-1,-2.3297147E-1,-7.477302E-3,-1.8048729E-1,-1.9920397E-1,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0151035E0,6.5632963E-1,1.823703E-1,2.1052504E-1,2.1035415E-1,9.229251E-2,0E0,1.576975E-1,1.7606108E-1,0E0,1.1426413E-1,0E0,0E0,1.1037278E-1,2.3217589E-1,0E0,0E0,0E0,0E0,1.10173464E-1,9.862223E-2,1.0037966E-1,1.288043E-1,1.144903E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1427817E0,3.165913E6,2.174459E5,5.8536225E5,2.207E3,1.1832633E0,8.008875E-3,1.1670399E-1,3.2744165E2,1.21591035E-2,1E1,8.102908E-3,1.4816463E-3,1.4031231E-2,2.587E3,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,1.1408451E0,5.1607388E5,2.8507206E-1,7.12E2,7.5608E4,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"split_indices":[43,1,30,30,0,41,0,41,4,0,3,0,0,40,2,0,0,0,0,55,35,44,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,1.84E2,5.1E1,1.34E2,5E1,2.8E1,2.3E1,1.1E2,2.4E1,3E0,4.7E1,4E0,2.4E1,6.6E1,4.4E1,9E0,1.5E1,1.4E1,3.3E1,5.8E1,8E0,3.5E1,9E0,1E1,4.8E1,5E0,3E0,2E1,1.5E1,2E0,7E0,8E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-6.5358323E-3,-5.01191E-2,1.1463253E-1,-6.566863E-2,3.5350563E-3,6.801129E-2,2.2740725E-1,-9.271781E-2,1.6208226E-4,3.614062E-3,4.4614673E-3,2.6938426E-1,5.7348673E-4,-7.9707764E-2,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,-1.20486856E-1,-4.1228443E-2,-2.9383153E-1,-9.8548844E-2,3.4898527E-3,-6.404513E-2,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2573216E0,3.6213574E-1,3.1576872E-1,2.9499376E-1,0E0,1.0101345E-1,1.5835595E-1,1.3562757E-1,0E0,1.0294547E-1,0E0,1.21626854E-1,0E0,1.5508604E-1,0E0,0E0,0E0,0E0,0E0,1.5738219E-1,1.544519E-1,1.2428334E-1,2.1362385E-1,0E0,1.0190545E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,5.8536225E5,5.76E3,3.165913E6,3.5350563E-3,1.7468556E2,3.41212E7,1E0,1.6208226E-4,1.2495134E6,4.4614673E-3,2.9375613E-1,5.7348673E-4,1.73507E5,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,9.675635E4,1.2473118E0,9.48E2,4.43001E5,3.4898527E-3,1.8979991E6,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"split_indices":[44,30,2,1,0,54,53,85,0,47,0,40,0,9,0,0,0,0,0,30,55,0,30,0,50,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.74E2,6.2E1,1.56E2,1.8E1,4.5E1,1.7E1,1.12E2,4.4E1,1.6E1,2.9E1,1.4E1,3E0,1.01E2,1.1E1,9E0,7E0,6E0,8E0,4.8E1,5.3E1,4E0,4.4E1,8E0,4.5E1,2E0,2E0,3.8E1,6E0,3.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.814724E-3,-4.223491E-2,1.1547112E-1,-8.021347E-2,1.0189096E-3,1.424386E-1,-2.2660806E-3,-5.76489E-2,-2.370273E-1,-5.7016924E-2,1.174304E-3,-1.1189182E-3,1.5711717E-1,-3.7425077E-3,2.5416018E-2,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.0735268E-1,2.2521736E-1,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.3238825E-1,3.9426633E-3,2.2386394E-2,9.90202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.2799627E0,2.77701E-1,3.414572E-1,3.0479503E-1,1.2174119E-1,1.6192663E-1,0E0,1.8898878E-1,1.3618523E-1,2.1964492E-1,0E0,0E0,1.7577171E-1,0E0,9.3134895E-2,0E0,0E0,0E0,0E0,1.2113613E-1,3.1686854E-1,0E0,0E0,0E0,0E0,1.3124907E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,14,14,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[3.799535E2,1.1306552E-1,4.183787E0,1.4031231E-2,1.83437E5,1.2055675E0,-2.2660806E-3,4.239E3,1.8018788E7,1.5086387E8,1.174304E-3,-1.1189182E-3,7.3667456E2,-3.7425077E-3,1.7E1,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.57257E0,3.21879E6,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.0256798E0,3.9426633E-3,2.2386394E-2,9.90202E-3],"split_indices":[54,41,59,40,9,55,0,10,47,7,0,0,54,0,3,0,0,0,0,55,31,0,0,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.68E2,7.3E1,8.9E1,7.9E1,6.3E1,1E1,7.9E1,1E1,2.4E1,5.5E1,5E0,5.8E1,5.9E1,2E1,5E0,5E0,2E0,2.2E1,3.5E1,2.3E1,1.1E1,9E0,1.2E1,2.3E1,1.2E1,1.1E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.0411531E-3,-5.0822284E-2,8.184148E-2,-8.0308676E-2,-6.2485766E-3,1.5205893E-1,6.5722945E-3,-1.08080305E-1,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,2.7616316E-1,-1.5828831E-2,1.3877054E-2,-8.242552E-2,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,-1.7038673E-1,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[9.903048E-1,2.005888E-1,4.6938437E-1,2.0104021E-1,1.332165E-1,2.687695E-1,3.1198943E-1,1.609565E-1,0E0,0E0,0E0,0E0,1.3902038E-1,9.690636E-2,0E0,1.4167267E-1,0E0,0E0,0E0,0E0,0E0,1.059061E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,15,15,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[3.2769293E2,1.529448E-1,1.0568718E7,5.5015685E9,9.74979E6,7.115615E2,1.4122614E2,1.3244295E5,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,1.8373182E8,1.128757E6,1.3877054E-2,6.7001674E-3,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,2.647867E-4,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"split_indices":[54,41,47,5,9,54,58,49,0,0,0,0,34,31,0,60,0,0,0,0,0,41,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.52E2,8.8E1,9.1E1,6.1E1,4.5E1,4.3E1,6.7E1,2.4E1,4.7E1,1.4E1,3.3E1,1.2E1,4.1E1,2E0,5.5E1,1.2E1,1E1,2E0,1E1,3.1E1,1.3E1,4.2E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.606068E-4,-4.0152058E-2,1.3688791E-1,-7.5740196E-2,5.956878E-3,1.0230381E-1,3.1381202E-1,-9.340738E-2,3.2788247E-2,6.146593E-3,-6.0711894E-3,9.786554E-3,6.620768E-2,1.6971182E-2,1.4838147E-3,-6.620601E-2,-1.9074214E-1,5.796871E-3,-1.0807036E-3,-2.5402438E-2,2.655493E-3,-2.4867726E-3,8.962364E-2,-1.7166455E-1,-3.69803E-2,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,25,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.397829E0,3.1715095E-1,3.2278395E-1,2.120583E-1,1.3876921E-1,2.0526332E-1,1.7587048E-1,2.362498E-1,9.541947E-2,0E0,1.03748016E-1,0E0,1.20494455E-1,0E0,0E0,2.2480932E-1,2.1755302E-1,0E0,0E0,1.3877995E-1,0E0,0E0,1.2990308E-1,1.283262E-1,1.8078491E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,15,15,16,16,19,19,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,26,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.633287E2,2.0358758E8,1.9292948E3,3.402546E6,2.3755E4,2.02983E5,3.41212E7,1.2869189E5,1.8000048E10,6.146593E-3,1.9425332E7,9.786554E-3,1.9241573E-1,1.6971182E-2,1.4838147E-3,1E0,9.975593E4,5.796871E-3,-1.0807036E-3,4.327E3,2.655493E-3,-2.4867726E-3,3.3996282E0,1.9E1,1.081E3,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"split_indices":[54,7,54,1,9,31,53,49,5,0,9,0,59,0,0,102,30,0,0,11,0,0,59,3,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.92E2,5.7E1,1.08E2,8.4E1,4.9E1,8E0,9.3E1,1.5E1,6E0,7.8E1,1E1,3.9E1,6E0,2E0,7.4E1,1.9E1,5E0,1E1,6.1E1,1.7E1,6E0,3.3E1,1.5E1,5.9E1,3E0,1.6E1,1.6E1,4.5E1,2.1E1,1.2E1,6E0,9E0,2.2E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.4908163E-3,-4.9458664E-2,8.817324E-2,6.931918E-2,-6.1066084E-2,6.9811605E-2,3.090321E-1,-5.98609E-4,7.956278E-3,-8.879114E-3,-5.08166E-2,1.4328934E-1,3.887047E-2,1.7160837E-2,2.8655254E-3,-1.0146142E-1,-3.0556865E-2,1.2192971E-3,2.755345E-1,8.296684E-2,-6.053145E-4,-6.862668E-2,-1.2999088E-2,1.7750489E-3,-5.1321223E-2,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,-1.3914266E-1,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,17,19,-1,-1,21,23,-1,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[1.1073979E0,2.1379605E-1,3.6368233E-1,1.3386652E-1,1.9253713E-1,1.9677043E-1,1.176427E-1,0E0,0E0,0E0,1.3131228E-1,3.889135E-1,1.4999606E-1,0E0,0E0,2.181817E-1,1.4260218E-1,0E0,1.17243946E-1,9.568061E-2,0E0,1.2064642E-1,0E0,0E0,8.933057E-2,0E0,0E0,0E0,0E0,1.0965279E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,11,11,12,12,15,15,16,16,18,18,19,19,21,21,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,18,20,-1,-1,22,24,-1,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[3.2769293E2,1.1623532E6,1.3758811E0,1E0,1.7565872E-3,2.3842E4,6.901E3,-5.98609E-4,7.956278E-3,-8.879114E-3,4.0835E4,7.6411234E4,9.380036E6,1.7160837E-2,2.8655254E-3,2.9138212E0,1.217759E0,1.2192971E-3,8.7292106E5,1.9643908E5,-6.053145E-4,5.3091907E2,-1.2999088E-2,1.7750489E-3,3.0962145E0,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,1.0288912E5,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"split_indices":[54,47,36,8,60,9,0,0,0,0,9,30,47,0,0,60,55,0,35,30,0,4,0,0,55,0,0,0,0,35,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,1.3E1,1.38E2,8.8E1,6E0,8E0,5E0,8E0,1.3E2,2.5E1,6.3E1,4E0,2E0,3.6E1,9.4E1,1.4E1,1.1E1,3.4E1,2.9E1,3.2E1,4E0,2.1E1,7.3E1,3E0,8E0,6E0,2.8E1,1.3E1,1.9E1,6.4E1,9E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.576465E-3,-3.1257384E-2,1.1510225E-1,-6.0538724E-2,4.1183308E-2,1.9610284E-2,9.548955E-2,-1.616713E-1,-4.8437126E-2,1.07467435E-2,2.4615793E-2,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,-3.3256467E-2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,-4.011636E-2,4.8860135E-3,-9.835392E-2,-1.4123669E-2,7.2520383E-4,-1.8231265E-1,7.876769E-5,-6.9828774E-3,-2.5798842E-1,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[9.6316123E-1,4.0339252E-1,3.4389424E-1,1.5739092E-1,1.8081136E-1,0E0,1.22983694E-1,1.2277371E-1,1.5455526E-1,0E0,1.18286856E-1,0E0,0E0,0E0,0E0,1.0909271E-1,0E0,0E0,0E0,1.5149972E-1,0E0,3.022702E-1,1.6528234E-1,0E0,1.5847087E-1,0E0,0E0,1.461187E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.2185057E0,3.165913E6,1E0,6.7001674E-3,2.207E3,1.9610284E-2,2.0787016E5,2.647867E-4,2.0017123E0,1.07467435E-2,1E1,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,4.5971466E2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,4.0054E4,4.8860135E-3,1.8259804E-1,2.2912737E1,7.2520383E-4,1.3738942E0,7.876769E-5,-6.9828774E-3,1.622E3,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"split_indices":[44,1,13,60,0,0,30,41,55,0,3,0,0,0,0,54,0,0,0,9,0,60,58,0,55,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.88E2,5.8E1,1.34E2,5.4E1,2E0,5.6E1,1.3E1,1.21E2,3E0,5.1E1,3.2E1,2.4E1,9E0,4E0,1.04E2,1.7E1,1.5E1,3.6E1,1E2,4E0,3E1,7E1,1.3E1,1.7E1,6.4E1,6E0,1E1,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.720544E-3,-4.5713637E-2,8.041132E-2,4.5472946E-2,-5.675482E-2,4.4581052E-2,1.6859385E-1,-1.5774339E-3,5.743392E-3,-6.856326E-2,2.9137306E-4,8.072134E-3,3.0883143E-2,1.6059604E-2,5.5405004E-3,-6.1498176E-2,-1.2888393E-2,7.4021585E-2,-2.9413155E-3,-9.75898E-3,-5.4604717E-2,7.0612812E-3,1.7218302E-3,-2.8861985E-3,3.3276238E-2,-7.160593E-3,-4.491236E-2,-1.0041681E-3,3.9801714E-3,4.2815937E-4,-6.328989E-2,-7.997919E-3,-2.0749776E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,17,-1,-1,19,-1,21,23,-1,25,-1,-1,-1,27,-1,29,-1,-1,-1,31,-1,-1],"loss_changes":[9.9902195E-1,1.4655435E-1,3.463596E-1,1.1724831E-1,9.633365E-2,1.5005943E-1,2.1894133E-1,0E0,0E0,1.5487564E-1,0E0,0E0,1.096528E-1,0E0,0E0,1.0625979E-1,0E0,9.375355E-2,1.00620806E-1,0E0,1.0192835E-1,0E0,0E0,0E0,9.3399785E-2,0E0,9.716697E-2,0E0,0E0,0E0,1.2727156E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,12,12,15,15,17,17,18,18,20,20,24,24,26,26,30,30],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,18,-1,-1,20,-1,22,24,-1,26,-1,-1,-1,28,-1,30,-1,-1,-1,32,-1,-1],"split_conditions":[2.797224E2,1.1696849E6,1.5972575E0,2.25E2,4.343271E5,8.9397516E4,1.8426E4,-1.5774339E-3,5.743392E-3,2.9892595E6,2.9137306E-4,8.072134E-3,1.1572553E7,1.6059604E-2,5.5405004E-3,1.3525729E6,-1.2888393E-2,4.0012E4,1.4813267E3,-9.75898E-3,1.7565872E-3,7.0612812E-3,1.7218302E-3,-2.8861985E-3,5.415429E4,-7.160593E-3,1.1710652E0,-1.0041681E-3,3.9801714E-3,4.2815937E-4,1E0,-7.997919E-3,-2.0749776E-3],"split_indices":[54,47,43,0,49,35,9,0,0,53,0,0,47,0,0,47,0,9,57,0,60,0,0,0,30,0,55,0,0,0,68,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,1.5E1,1.27E2,8E1,3.1E1,8E0,7E0,1.07E2,2E1,6E0,7.4E1,4E0,2.7E1,1.05E2,2E0,3.2E1,4.2E1,3E0,1.02E2,8E0,2.4E1,1.5E1,2.7E1,7E0,9.5E1,1.4E1,1.3E1,2.4E1,7.1E1,7E0,6.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.432755E-3,-3.2541275E-2,1.2062466E-1,-6.6113114E-2,7.3056045E-4,7.963434E-2,2.6087046E-1,-2.4810357E-2,-1.07705206E-1,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,-1.1733579E-1,-3.5330087E-3,-1.7337382E-1,-5.0983664E-2,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,3.909637E-2,-2.709494E-1,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0041343E0,3.2042485E-1,2.954054E-1,1.9488263E-1,0E0,1.0474539E-1,2.4700874E-1,1.15738496E-1,2.0347583E-1,0E0,0E0,0E0,0E0,1.0116187E-1,1.1524878E-1,1.1706191E-1,1.1447847E-1,0E0,0E0,0E0,1.1847381E-1,1.464771E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15,16,16,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.3341238E9,2.2973125E3,1.19331E5,7.3056045E-4,2.0787016E5,2.174459E5,3.559166E7,6.6037946E2,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,4.3399894E5,1.09E3,7.359468E8,1.48068905E1,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,1.2765957E0,1.721649E8,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"split_indices":[44,5,4,31,0,30,30,7,4,0,0,0,0,30,2,5,58,0,0,0,55,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.92E2,5.4E1,1.14E2,7.8E1,4.3E1,1.1E1,5.8E1,5.6E1,2.1E1,2.2E1,6E0,5E0,1E1,4.8E1,2.5E1,3.1E1,5E0,5E0,2.1E1,2.7E1,8E0,1.7E1,1.5E1,1.6E1,7E0,2E1,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.4172934E-3,-4.873043E-2,5.479038E-2,2.9505575E-3,-7.196424E-2,1.9374692E-1,3.8230103E-2,-2.232737E-3,2.4846569E-3,-2.2767627E-1,-5.168247E-2,1.6559279E-3,3.4322336E-1,6.4282104E-2,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,9.1448665E-2,1.5415065E-2,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.809334E-1,1.707184E-1,2.5722635E-1,1.343776E-1,2.9772854E-1,2.6643854E-1,1.8172754E-1,0E0,0E0,1.076473E-1,2.4328896E-1,0E0,1.7182684E-1,9.815651E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.945521E-2,1.7858982E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.47887E5,1.431E4,1.081E3,2.07684E5,9.675635E4,1.8018788E7,-2.232737E-3,2.4846569E-3,1.3738942E0,9.476661E1,1.6559279E-3,9.63E2,2.1949912E7,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,7.3667456E2,1.25836E5,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"split_indices":[54,1,9,2,1,30,47,0,0,55,54,0,0,34,0,0,0,0,0,0,0,54,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.41E2,1.14E2,4.4E1,9.7E1,1.1E1,1.03E2,2.2E1,2.2E1,1E1,8.7E1,6E0,5E0,7.4E1,2.9E1,4E0,6E0,1.7E1,7E1,2E0,3E0,4.7E1,2.7E1,3.9E1,8E0,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.1405309E-3,-4.1295454E-2,7.539111E-2,-4.773554E-2,2.649901E-3,5.8098752E-2,2.4969846E-1,-3.5499748E-2,-1.1151391E-1,7.256275E-2,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,-2.639264E-2,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,-3.14371E-2,-4.9293704E-2,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1],"loss_changes":[7.971391E-1,1.0867721E-1,2.6279783E-1,1.1465886E-1,0E0,1.4647299E-1,1.3158771E-1,9.851079E-2,1.7702582E-1,1.16677344E-1,0E0,0E0,0E0,0E0,9.4192E-2,0E0,0E0,0E0,0E0,0E0,9.489479E-2,9.564981E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,14,14,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1],"split_conditions":[5.9570425E5,2.407794E7,1.3821641E0,2.9138212E0,2.649901E-3,1E0,3.41212E7,1.4781966E-3,3.67E2,2.3581433E5,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,1E1,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,3.3341238E9,5.77032E5,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"split_indices":[49,9,36,60,0,85,53,60,0,30,0,0,0,0,10,0,0,0,0,0,5,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.6E2,9.1E1,1.51E2,9E0,8.4E1,7E0,1.28E2,2.3E1,7.5E1,9E0,5E0,2E0,1.1E1,1.17E2,2E0,2.1E1,3.8E1,3.7E1,3E0,1.14E2,8.2E1,3.2E1,6.9E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.8670357E-3,-4.5970436E-2,5.2696485E-2,6.0541928E-2,-5.781559E-2,2.443929E-2,1.2535055E-1,-9.904153E-4,6.934784E-3,-7.859939E-3,-4.945917E-2,3.5898713E-3,3.4437096E-3,2.2750926E-1,1.946528E-3,-6.620565E-2,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,-1.1629289E-1,-4.547119E-2,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.025647E-1,1.8644026E-1,2.186605E-1,1.2642913E-1,1.2496975E-1,9.5626205E-2,2.3731667E-1,0E0,0E0,0E0,9.588456E-2,0E0,9.368942E-2,1.0582209E-1,0E0,8.921841E-2,0E0,0E0,0E0,0E0,0E0,1.2605882E-1,9.986128E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.1623532E6,6.657047E2,1E0,1.7565872E-3,1E1,2.45226E6,-9.904153E-4,6.934784E-3,-7.859939E-3,2.9073395E9,3.5898713E-3,5.1332E4,3.0256798E0,1.946528E-3,4.0054E4,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,2.8516157E0,2.704375E6,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"split_indices":[54,47,54,8,60,3,31,0,0,0,5,0,9,56,0,9,0,0,0,0,0,60,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.44E2,1.07E2,1.4E1,1.3E2,7.8E1,2.9E1,8E0,6E0,7E0,1.23E2,2E1,5.8E1,1.2E1,1.7E1,9E1,3.3E1,2.2E1,3.6E1,4E0,8E0,2.5E1,6.5E1,2.2E1,3E0,4.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.004087E-3,-2.8343314E-2,7.549351E-2,-5.5867407E-2,2.2657372E-2,1.4241742E-2,5.9626795E-2,-6.4280997E-3,-4.644786E-2,4.461195E-2,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,-5.3805247E-2,4.6029454E-3,4.787177E-5,-8.447469E-3,-4.588249E-2,-1.3219344E-3,-5.0460016E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,-1,-1,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[5.418513E-1,2.4271044E-1,2.6314437E-1,8.962074E-2,1.0539442E-1,0E0,1.0044208E-1,0E0,1.2327367E-1,1.2701635E-1,0E0,0E0,0E0,0E0,1.0074824E-1,0E0,0E0,0E0,9.979747E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,14,14,18,18],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,-1,-1,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[9.821357E-1,7.422E3,1.215951E5,9.87574E2,2.1949912E7,1.4241742E-2,2.216135E5,-6.4280997E-3,1.8E1,1.1572553E7,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,1.3525729E6,4.6029454E-3,4.787177E-5,-8.447469E-3,2.0134454E0,-1.3219344E-3,-5.0460016E-3],"split_indices":[43,10,35,35,34,0,30,0,10,47,0,0,0,0,47,0,0,0,55,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.71E2,7E1,1.11E2,6E1,3E0,6.7E1,9E0,1.02E2,4.7E1,1.3E1,3.8E1,2.9E1,4E0,9.8E1,1.9E1,2.8E1,4E0,9.4E1,7.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.1594397E-2,-1.194265E-2,9.3121335E-2,-3.9583016E-2,2.608233E-2,1.6468607E-2,7.527401E-2,-5.18759E-2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,4.0722504E-4,-1.7415832E-3,3.3587667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,-1,15,-1,-1],"loss_changes":[4.712434E-1,2.0177807E-1,2.5062546E-1,1.2383151E-1,1.3076647E-1,0E0,1.04620725E-1,1.0193059E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.9496724E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.3394321E0,2.427E3,1.431E4,9.74979E6,2.637815E1,1.6468607E-2,2.174459E5,2.0256741E2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,3.4234106E5,-1.7415832E-3,3.3587667E-3],"split_indices":[44,2,9,9,58,0,30,54,0,0,0,0,0,0,30,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.9E2,5.4E1,1.1E2,8E1,2E0,5.2E1,9.7E1,1.3E1,5.1E1,2.9E1,2.8E1,2.4E1,7E1,2.7E1,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-8.11151E-3,-2.3770498E-2,1.3511473E-1,-4.903885E-2,3.7162323E-4,2.8097564E-1,1.2597005E-3,-4.1468557E-2,-9.574549E-3,1.8381007E-2,7.012952E-3,-1.3489653E-3,-6.7639E-2,-9.2418713E-4,6.152402E-3,-8.961814E-2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[5.4596394E-1,1.7908293E-1,3.6382854E-1,1.495364E-1,0E0,1.009686E-1,0E0,1.2485254E-1,0E0,0E0,0E0,1.3845807E-1,1.4358482E-1,0E0,0E0,9.946346E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,11,11,12,12,15,15],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[7.2259143E2,2.2348747E2,2.45226E6,2.1405182E2,3.7162323E-4,3.0256798E0,1.2597005E-3,4.37E2,-9.574549E-3,1.8381007E-2,7.012952E-3,4.02E2,5.8536225E5,-9.2418713E-4,6.152402E-3,4.73E2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"split_indices":[54,54,31,54,0,56,0,0,0,0,0,0,30,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.18E2,2.3E1,1.22E2,9.6E1,9E0,1.4E1,1.18E2,4E0,3E0,6E0,4.7E1,7.1E1,4.2E1,5E0,5.7E1,1.4E1,2E0,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.816721E-3,-3.212387E-2,7.8340545E-2,-5.9203006E-2,-2.047665E-3,1.05734095E-1,-1.2828258E-3,-4.2488243E-2,-1.6570118E-1,5.200363E-3,-1.1345699E-2,6.766135E-2,1.9234525E-1,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,1.8420164E-2,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.675061E-1,1.4077646E-1,2.3921198E-1,1.54948E-1,9.489868E-2,1.9802713E-1,0E0,1.2677193E-1,1.3189653E-1,0E0,9.6501775E-2,1.6102695E-1,1.9053322E-1,0E0,0E0,0E0,0E0,0E0,8.944257E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.5442211E8,3.6207435E0,1.2869189E5,2.3755E4,5.5303E9,-1.2828258E-3,7.92582E5,3.396945E7,5.200363E-3,3.7230852E0,9.90141E6,6.464779E6,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,4.33121E0,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"split_indices":[54,7,59,49,9,5,0,12,12,0,56,47,31,0,0,0,0,0,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.72E2,7.9E1,9E1,8.2E1,6.3E1,1.6E1,7.9E1,1.1E1,5E0,7.7E1,4.5E1,1.8E1,5.4E1,2.5E1,6E0,5E0,3.2E1,4.5E1,3.4E1,1.1E1,1E1,8E0,1.7E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.911687E-4,-2.2826223E-2,8.1994176E-2,-2.6359616E-2,9.769677E-3,1.5213658E-2,6.624614E-2,-5.9948202E-2,-4.920752E-3,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9672684E-1,1.8459845E-1,2.2044265E-1,1.4588554E-1,0E0,0E0,1.2346178E-1,1.278441E-1,1.16657905E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.4122614E2,1E0,5.283708E0,9.769677E-3,1.5213658E-2,1.1598054E8,3.901529E6,5.8536225E5,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"split_indices":[44,58,13,58,0,0,34,9,30,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,2.04E2,5.7E1,2.02E2,2E0,2E0,5.5E1,7.8E1,1.24E2,4.5E1,1E1,6.5E1,1.3E1,1.05E2,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-4.2027067E-3,-2.6648754E-2,8.658342E-2,-3.084103E-3,-2.5529137E-3,1.4771055E-2,2.9963553E-3,4.294585E-3,-1.1864337E-2,-5.120143E-2,4.8290283E-4,1.1069501E-3,-3.521963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[5.0360364E-1,2.0985942E-1,1.97959E-1,0E0,1.22443795E-1,0E0,0E0,0E0,1.0790554E-1,1.0145053E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.4125891E0,5.283708E0,1E0,-3.084103E-3,9.165388E4,1.4771055E-2,2.9963553E-3,4.294585E-3,1.1670399E-1,3.712E4,4.8290283E-4,1.1069501E-3,-3.521963E-3],"split_indices":[44,58,13,0,35,0,0,0,41,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.97E2,4.8E1,6.9E1,1.28E2,2E0,4.6E1,1E1,1.18E2,4.3E1,7.5E1,1.2E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.173793E-3,-2.6264051E-2,8.658401E-2,-6.669964E-2,-8.2213007E-4,1.4341326E-2,7.03292E-2,1.6294725E-2,-8.671885E-2,2.6084114E-2,-4.4527646E-2,8.693059E-4,1.125322E-1,-1.6205347E-3,6.1647515E-3,-2.2572479E-1,-6.812656E-2,4.0135182E-2,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.266243E-1,2.0775451E-1,1.8148205E-1,1.3100863E-1,1.4815538E-1,0E0,1.0412836E-1,1.1208219E-1,1.4744174E-1,1.12065986E-1,9.200167E-2,0E0,9.0320885E-2,0E0,0E0,9.239629E-2,1.0407472E-1,1.1784806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5098532E0,5.283708E0,1E0,3.6E1,1.7303246E7,1.4341326E-2,2.174459E5,7.014272E7,9.975593E4,5.8414636E0,2.094E3,8.693059E-4,4.7201157E-1,-1.6205347E-3,6.1647515E-3,7.725141E4,1.9415865E2,2.4256398E8,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"split_indices":[43,58,13,10,47,0,30,7,30,56,2,0,40,0,0,30,54,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.01E2,5.1E1,7.7E1,1.24E2,2E0,4.9E1,1.5E1,6.2E1,7.7E1,4.7E1,2.3E1,2.6E1,1.1E1,4E0,6E0,5.6E1,6.8E1,9E0,7E0,4E1,2.1E1,5E0,3E0,3E0,4.8E1,8E0,2E0,6.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[5.15778E-3,-1.8465955E-2,8.479578E-2,-4.582774E-2,2.8703757E-2,1.1748621E-2,2.8553656E-3,-6.113014E-2,7.824005E-4,7.2038495E-3,8.444443E-3,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,-1,-1,-1,-1],"loss_changes":[4.6061453E-1,2.4524778E-1,1.8237612E-1,1.1817169E-1,1.9200628E-1,0E0,0E0,1.1865306E-1,0E0,0E0,1.02744974E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2446029E0,1.781973E6,1.8426E4,5.4279834E-1,2.207E3,1.1748621E-2,2.8553656E-3,4.1150245E-1,7.824005E-4,7.2038495E-3,1E1,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"split_indices":[44,1,9,43,0,0,0,44,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.88E2,5.5E1,1.19E2,6.9E1,4E0,5.1E1,9.6E1,2.3E1,8E0,6.1E1,9.4E1,2E0,1.8E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[2.6694385E-3,-1.2630397E-2,9.763298E-2,-2.7587097E-3,-3.7950324E-3,2.2221443E-1,6.921976E-4,4.2745727E-3,-9.810083E-3,4.104886E-3,1.3904189E-2,-3.826968E-2,8.1882236E-4,-4.5390716E-3,-2.056427E-2,-1.03351854E-1,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,-1,-1,11,-1,-1,13,-1,-1,15,17,-1,-1,-1],"loss_changes":[3.4557673E-1,1.5018955E-1,3.3265132E-1,1.3020961E-1,0E0,1.3063407E-1,0E0,0E0,1.3977444E-1,0E0,0E0,9.7968146E-2,0E0,0E0,9.9550575E-2,1.2142539E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,11,11,14,14,15,15],"right_children":[2,4,6,8,-1,10,-1,-1,12,-1,-1,14,-1,-1,16,18,-1,-1,-1],"split_conditions":[6.56755E2,2.4219654E0,2.660842E6,1.1623532E6,-3.7950324E-3,2.1061536E3,6.921976E-4,4.2745727E-3,1.7179383E5,4.104886E-3,1.3904189E-2,1E0,8.1882236E-4,-4.5390716E-3,1.27318E5,1.3738942E0,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"split_indices":[54,55,31,47,0,4,0,0,34,0,0,102,0,0,30,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2.04E2,3.2E1,1.81E2,2.3E1,1.2E1,2E1,1.1E1,1.7E2,6E0,6E0,8.5E1,8.5E1,1.7E1,6.8E1,1.1E1,5.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.2459247E-4,-2.1839963E-2,5.5326115E-2,6.403799E-2,-2.9754521E-2,6.706761E-2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,3.54164E-2,1.4349867E-1,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,13,15,-1,-1,-1,-1],"loss_changes":[3.0025756E-1,1.2017081E-1,1.2092781E-1,9.031686E-2,1.1387043E-1,1.5458077E-1,0E0,0E0,0E0,0E0,0E0,1.247524E-1,1.0393754E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,11,11,12,12],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,14,16,-1,-1,-1,-1],"split_conditions":[3.799535E2,1.1623532E6,6.236639E3,2.0535674E0,1.4781966E-3,9.386909E2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,9E0,7.529323E4,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"split_indices":[54,47,4,55,60,54,0,0,0,0,0,3,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.73E2,7E1,1.4E1,1.59E2,6.5E1,5E0,8E0,6E0,6E0,1.53E2,4.7E1,1.8E1,8E0,3.9E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.029568E-4,-6.5150703E-3,1.8107072E-1,-3.652972E-2,2.4049532E-2,1.0449934E-2,-1.5273974E-3,-2.0122712E-2,-9.696651E-2,3.5459854E-2,-2.3149115E-3,4.5559215E-3,-2.7940093E-2,-1.3360148E-3,-1.7182508E-1,6.808551E-3,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,4.025893E-2,3.4104274E-3,-9.366535E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[3.37975E-1,2.2380325E-1,1.3455206E-1,1.2001395E-1,1.08324245E-1,0E0,0E0,9.831457E-2,1.2388055E-1,1.0830644E-1,0E0,0E0,1.4380896E-1,0E0,1.03295654E-1,1.3083223E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.5229566E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,12,12,14,14,15,15,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[1.9292948E3,2.0718746E8,6.236639E3,4.352025E0,2.2979798E0,1.0449934E-2,-1.5273974E-3,1E1,5.694004E6,3.642194E0,-2.3149115E-3,4.5559215E-3,9.476661E1,-1.3360148E-3,4.752366E0,4.854831E8,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,1.9647126E7,3.4104274E-3,-9.366535E-4],"split_indices":[54,7,4,56,55,0,0,10,47,56,0,0,54,0,58,12,0,0,0,0,0,0,47,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.42E2,9E0,1.22E2,1.2E2,7E0,2E0,9.7E1,2.5E1,1.05E2,1.5E1,5E0,9.2E1,1.4E1,1.1E1,5.9E1,4.6E1,2.5E1,6.7E1,4E0,7E0,2E1,3.9E1,2.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics index 99dfa2910..70e919724 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics @@ -46,64 +46,64 @@ 44,diskBytesSpilled_mean,0.00030759146,1407298.6859824378,5836681.87707674,0.0,0.0,0.0,0.0,56913253.10538942 45,sqlOp_SortMergeJoin,0.00025351142,0.629746835443038,0.4836382609453798,0.0,0.0,1.0,1.0,1.0 46,sqlOp_SubqueryOutputBroadcast,0.00011199158,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 -47,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_BroadcastNestedLoopJoin,0.0,0.028481012658227847,0.16660637848720578,0.0,0.0,0.0,0.0,1.0 -53,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_ReusedSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 -63,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_LocalTableScan,0.0,0.0031645569620253164,0.056254395046301184,0.0,0.0,0.0,0.0,1.0 -66,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.10443037974683544,0.30630308399462325,0.0,0.0,0.0,0.0,1.0 -69,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,platform_emr,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -76,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_Filter,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -78,sqlOp_HashAggregate,0.0,0.990506329113924,0.09712565886833821,0.0,1.0,1.0,1.0,1.0 -79,sqlOp_Project,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -80,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,sqlOp_Scan parquet ,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -83,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,sqlOp_WindowGroupLimit,0.0,0.0189873417721519,0.1366965815037675,0.0,0.0,0.0,0.0,1.0 -85,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -86,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -87,maxMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 -88,maxOnHeapMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 -89,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,executorMemory,0.0,16384.0,0.0,16384.0,16384.0,16384.0,16384.0,16384.0 -91,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -94,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -95,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -97,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -100,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_ColumnarToRow,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -102,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_LocalTableScan,0.0,0.0031645569620253164,0.056254395046301184,0.0,0.0,0.0,0.0,1.0 +51,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.10443037974683544,0.30630308399462325,0.0,0.0,0.0,0.0,1.0 +53,sqlOp_HashAggregate,0.0,0.990506329113924,0.09712565886833821,0.0,1.0,1.0,1.0,1.0 +54,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +57,sqlOp_Project,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +58,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_WindowGroupLimit,0.0,0.0189873417721519,0.1366965815037675,0.0,0.0,0.0,0.0,1.0 +60,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_ReusedSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +67,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Filter,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +74,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +77,maxMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 +78,maxOnHeapMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 +79,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,executorMemory,0.0,16384.0,0.0,16384.0,16384.0,16384.0,16384.0,16384.0 +81,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +84,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_Scan parquet ,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +88,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +89,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_ColumnarToRow,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +94,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,platform_emr,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +98,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +100,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 103,sqlOp_BroadcastExchange,0.0,0.9240506329113924,0.2653372528417936,0.0,1.0,1.0,1.0,1.0 -104,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_BroadcastNestedLoopJoin,0.0,0.028481012658227847,0.16660637848720578,0.0,0.0,0.0,0.0,1.0 +105,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 107,sqlOp_WindowSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg index ead208016..f1909ca15 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0459315479","gamma":"0.0072274562","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0459315479","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"6","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.0072274562","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.656415164"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"83"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0461990498","gamma":"0.0150604062","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0461990498","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"10","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.0150604062","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.671188235"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json index 0906f9a53..691af8005 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_Project","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_Window","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_onprem","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.643018E-2,-2.509553E-1,6.920586E-1,-7.251147E-1,-4.272643E-2,-4.5599207E-2,7.6895946E-1,-9.915738E-1,-5.506106E-1,-2.3798254E-1,1.6535082E-1,9.4180655E-1,4.6457025E-1,-1.0161444E0,-1.0458543E-2,-7.0368546E-1,-1.7502832E-1,-4.1097206E-1,-6.83809E-2,3.136601E-1,8.320871E-3,5.654987E-1,1.2173053E0,5.123619E-1,-1.1852422E-2,-9.3713164E-1,-1.2371283E0,-6.3041854E-1,-4.6454594E-2,7.0085746E-1,-3.4520468E-1,-5.987527E-1,-2.4146362E-1,-1.3473535E-1,1.3074206E-1,3.6425155E-1,-6.0084127E-2,4.4946205E-2,-5.9456974E-1,8.158183E-1,3.4814405E-1,1.4593371E0,9.641096E-1,2.8917283E-1,6.934715E-1,-4.4574197E-2,-2.031198E-2,-6.495018E-2,-3.7529863E-2,-1.5991386E-2,-3.166033E-2,1.4663583E-2,4.1671485E-2,-3.613732E-2,-1.1270952E-2,-2.3985183E-2,-4.780001E-2,-6.0631777E-3,-2.7680065E-2,-2.1264644E-2,-4.109702E-3,1.0041936E-2,-3.3481976E-3,1.0534429E-2,2.5075726E-2,-7.1340897E-3,5.7198117E-3,8.814541E-3,-2.0931101E-4,-4.7956286E-3,-3.5355933E-2,9.378983E-3,3.9205376E-2,2.124107E-2,-5.0402978E-3,5.2029464E-2,8.16598E-2,4.5951128E-2,6.1688717E-3,2.5536882E-2,9.163E-3,3.7161652E-2,1.9109875E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6167967E2,7.214333E1,3.2117958E1,9.823135E0,2.0719185E1,0E0,1.1768631E1,1.4015121E0,7.7532654E0,7.688134E0,5.749772E0,1.4551453E1,3.1022224E0,3.073349E-1,0E0,1.549202E0,6.3531966E0,3.9933758E0,1.7881169E0,2.423236E0,2.6985335E0,3.1980991E0,3.7406693E0,3.0736217E0,0E0,6.591072E-1,2.566986E-1,1.1060982E0,0E0,2.616949E-1,1.3495989E0,1.5203953E0,2.687808E0,1.4590225E0,6.2443244E-1,2.6198568E0,3.006507E-1,8.374978E-1,5.2070713E-1,4.9515152E-1,1.9349341E0,2.2509995E0,1.191616E0,7.9831576E-1,1.0743923E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7471867E5,1E0,9.9E1,9.31E2,-4.5599207E-2,8E0,1.3745962E0,8.75E2,8.079573E5,3.925926E0,1.087521E3,4.023131E2,1.4504054E8,-1.0458543E-2,1.9E1,2.74614E5,4.97E2,4.1973075E6,1E0,1.2294118E1,7E0,1.4127854E7,7.958E3,-1.1852422E-2,5.159652E-7,9.558022E7,2E0,-4.6454594E-2,1.357E3,1E0,7.668863E6,6.33E3,5.862757E1,8E0,4.2210345E2,2.9149944E5,1.0439024E1,1.4787234E1,6.978874E5,1.606722E8,1.6975454E3,1.9500381E2,1.1840488E8,2.3876712E7,-4.4574197E-2,-2.031198E-2,-6.495018E-2,-3.7529863E-2,-1.5991386E-2,-3.166033E-2,1.4663583E-2,4.1671485E-2,-3.613732E-2,-1.1270952E-2,-2.3985183E-2,-4.780001E-2,-6.0631777E-3,-2.7680065E-2,-2.1264644E-2,-4.109702E-3,1.0041936E-2,-3.3481976E-3,1.0534429E-2,2.5075726E-2,-7.1340897E-3,5.7198117E-3,8.814541E-3,-2.0931101E-4,-4.7956286E-3,-3.5355933E-2,9.378983E-3,3.9205376E-2,2.124107E-2,-5.0402978E-3,5.2029464E-2,8.16598E-2,4.5951128E-2,6.1688717E-3,2.5536882E-2,9.163E-3,3.7161652E-2,1.9109875E-2],"split_indices":[2,40,17,41,2,0,29,50,2,40,66,64,70,43,0,3,5,2,40,6,65,3,63,2,0,49,7,29,0,2,27,5,41,59,8,64,45,70,68,40,12,64,70,7,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.71E2,7.3E2,2.41E2,2.22E2,5.08E2,1E1,2.31E2,8.6E1,1.36E2,2.62E2,2.46E2,1.46E2,8.5E1,8.3E1,3E0,9.6E1,4E1,1.29E2,1.33E2,1.26E2,1.2E2,6.3E1,8.3E1,8E1,5E0,6.5E1,1.8E1,8E1,1.6E1,6E0,3.4E1,6E1,6.9E1,1E2,3.3E1,1.11E2,1.5E1,1.14E2,6E0,2.8E1,3.5E1,4E1,4.3E1,3.7E1,4.3E1,6E1,5E0,1.1E1,7E0,1.5E1,6.5E1,3E0,3E0,5E0,2.9E1,5.3E1,7E0,5.4E1,1.5E1,1.1E1,8.9E1,2.3E1,1E1,6.5E1,4.6E1,1E1,5E0,2.8E1,8.6E1,2E0,4E0,2E0,2.6E1,2.8E1,7E0,2.2E1,1.8E1,4.1E1,2E0,8E0,2.9E1,2.9E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[1.2891826E-2,-2.1696894E-1,6.872192E-1,-6.5165985E-1,-2.7308898E-2,-7.5670797E-1,7.487414E-1,3.5275993E-1,-6.8961513E-1,-1.993661E-1,1.634397E-1,-4.9645487E-2,-4.1707104E-1,3.7010625E-1,9.6343595E-1,-1.6712336E-2,3.5445772E-2,-9.307362E-1,-5.668537E-1,-5.3387505E-1,-1.2151727E-1,1.2559313E-1,6.169643E-1,-5.9868013E-3,-2.3227327E-2,1.3189232E-1,5.628038E-1,1.1470181E0,5.815575E-1,-9.644234E-1,-4.9847346E-3,-6.4138234E-1,-8.204187E-2,-5.547154E-1,-6.6652247E-3,-1.0736645E-1,-5.2252006E-2,-4.91621E-2,1.7265415E-1,3.1534988E-1,3.3617906E-2,-9.233331E-3,1.7222622E-1,6.7509687E-1,1.0330149E-1,9.8636776E-1,1.53402E0,6.6581017E-1,-8.698978E-3,-4.5047402E-2,-1.2532977E-2,-3.1882267E-2,-1.2749191E-2,1.683586E-2,-1.1259355E-2,-1.8174091E-2,-2.8791696E-2,-7.863931E-3,3.0332298E-3,9.317452E-3,-4.6848007E-3,2.652042E-3,1.2284683E-2,1.6686909E-2,5.9856707E-3,6.0416693E-3,2.2286257E-2,3.506797E-2,1.7117327E-2,1.5204436E-2,-3.4042017E-3,5.3487927E-2,3.7213102E-2,7.46604E-2,4.1914653E-2,2.4497312E-2,4.8820313E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,-1,29,31,33,35,37,39,-1,-1,41,43,45,47,49,-1,51,53,55,-1,57,-1,59,61,63,-1,-1,65,67,69,71,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5670053E2,6.2144596E1,2.339679E1,8.973442E0,1.729573E1,7.6023245E-1,1.972229E1,2.9828053E0,6.014099E0,7.1485157E0,4.2136016E0,0E0,1.1196315E-1,4.1029663E0,1.03396E1,0E0,0E0,1.9172859E0,5.3311005E0,3.40806E-1,3.1273978E0,1.9164937E0,4.2792606E-1,0E0,0E0,5.985892E-1,2.5102072E0,5.0149384E0,3.5686817E0,5.22789E-1,0E0,2.2826767E0,1.6262176E0,3.1402397E-1,0E0,2.4911482E0,0E0,6.864339E-1,1.9781966E0,1.4761567E-2,0E0,0E0,4.2011154E-1,7.8469086E-1,4.6895158E-1,1.473259E0,1.6487885E-1,1.9579144E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,35,35,37,37,38,38,39,39,42,42,43,43,44,44,45,45,46,46,47,47],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,-1,30,32,34,36,38,40,-1,-1,42,44,46,48,50,-1,52,54,56,-1,58,-1,60,62,64,-1,-1,66,68,70,72,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,6E0,9.48E2,5E0,9.616803E2,1.684E0,8E1,3.89E2,4.9321495E6,-4.9645487E-2,1.91E2,6.7148806E5,5.489183E1,-1.6712336E-2,3.5445772E-2,1.1408248E0,3.2874417E2,1.1362371E6,5.1212654E0,4.7563504E7,3.7788504E7,-5.9868013E-3,-2.3227327E-2,2.9585715E2,1E0,1.3388E4,7.706545E7,5.8E1,-4.9847346E-3,7.997723E6,8E0,3.1E1,-6.6652247E-3,2.331083E6,-5.2252006E-2,8.138348E1,3.850926E2,5.92E2,3.3617906E-2,-9.233331E-3,5.159139E8,5.6E1,7.1913286E8,1.2909952E1,2.395631E7,4.435876E2,-8.698978E-3,-4.5047402E-2,-1.2532977E-2,-3.1882267E-2,-1.2749191E-2,1.683586E-2,-1.1259355E-2,-1.8174091E-2,-2.8791696E-2,-7.863931E-3,3.0332298E-3,9.317452E-3,-4.6848007E-3,2.652042E-3,1.2284683E-2,1.6686909E-2,5.9856707E-3,6.0416693E-3,2.2286257E-2,3.506797E-2,1.7117327E-2,1.5204436E-2,-3.4042017E-3,5.3487927E-2,3.7213102E-2,7.46604E-2,4.1914653E-2,2.4497312E-2,4.8820313E-2],"split_indices":[2,40,17,3,2,6,64,66,41,2,40,0,0,40,68,0,0,50,64,40,65,7,7,0,0,64,6,2,56,3,0,9,29,10,0,40,0,59,64,10,0,0,7,8,5,68,59,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.009E3,7.53E2,2.56E2,2.28E2,5.25E2,1E1,2.46E2,8E0,2.2E2,2.76E2,2.49E2,4E0,6E0,9E1,1.56E2,3E0,5E0,7.2E1,1.48E2,5.1E1,2.25E2,2.31E2,1.8E1,2E0,4E0,4.1E1,4.9E1,1.04E2,5.2E1,6.9E1,3E0,1.28E2,2E1,4.8E1,3E0,2.23E2,2E0,4.9E1,1.82E2,6E0,1.2E1,4E0,3.7E1,3.9E1,1E1,7.6E1,2.8E1,4.7E1,5E0,6.7E1,2E0,1.11E2,1.7E1,5E0,1.5E1,1.7E1,3.1E1,1.63E2,6E1,8E0,4.1E1,8.3E1,9.9E1,4E0,2E0,3.4E1,3E0,2.9E1,1E1,4E0,6E0,3.5E1,4.1E1,2.3E1,5E0,3.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[1.8347733E-2,-2.1890882E-1,6.1889106E-1,-6.230002E-1,-3.5454784E-2,7.080461E-1,-6.022729E-1,-8.328948E-1,-4.7267786E-1,-2.0936815E-1,1.8785624E-1,4.0000445E-1,9.9742866E-1,-7.5713056E-1,1.372602E-2,-8.6421245E-1,-2.347539E-1,-1.4880635E-1,-5.9013265E-1,-4.380052E-1,-1.1131322E-1,3.21445E-1,5.4982085E-2,1.7655966E-1,6.2417054E-1,1.6802098E0,8.5219914E-1,-4.511751E-2,-4.7287467E-1,-8.264342E-3,9.315105E-3,-9.004549E-1,-6.598789E-1,-1.4789872E-2,-1.8453912E-3,5.01582E-1,-2.6610398E-1,-7.2257775E-1,-3.8002273E-1,-3.0180463E-1,-6.731061E-1,-9.319394E-2,-4.4406604E-2,2.03595E-1,4.6159613E-1,1.07230134E-1,-2.4022657E-1,2.5962995E-2,4.1304046E-1,6.721704E-1,-2.7186114E-2,2.423186E-2,8.1062905E-2,5.3803694E-1,1.0255797E0,-5.8034784E-3,-2.5438337E-2,-4.2023968E-2,-1.9553728E-2,-4.0827528E-2,-1.8863719E-2,-1.5161987E-3,3.5694808E-2,-2.6065527E-2,-5.672473E-3,-3.0742172E-2,-5.9149057E-2,-2.0004265E-2,1.0489132E-2,-1.633231E-2,2.6901793E-3,-2.5478413E-2,-4.514603E-2,-1.8447401E-3,-1.3109896E-2,1.4044499E-2,-3.6214744E-3,2.345136E-2,1.1715922E-3,-9.4388435E-3,5.828127E-3,-5.516775E-3,-3.4942E-2,9.031879E-3,-5.0633266E-3,3.0272402E-2,7.1228594E-3,3.566211E-2,7.5517707E-3,-1.2797235E-3,2.601201E-2,5.1119428E-2,3.088024E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,55,-1,-1,57,59,-1,-1,61,63,65,67,69,71,73,-1,75,77,79,81,83,85,87,-1,-1,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4390067E2,5.3649635E1,3.1564087E1,6.7370224E0,1.941888E1,2.3401901E1,1.9182358E0,1.5392303E0,5.009577E0,6.2509823E0,3.8696837E0,6.4514236E0,1.215303E1,5.59494E-1,2.1956801E-1,5.7495117E-2,8.8917345E-2,2.956166E0,2.4525528E0,2.4866047E0,2.9790576E0,1.7063541E0,1.7378321E0,2.375362E0,4.191923E0,1.3131638E0,5.7122116E0,0E0,1.8250442E-1,0E0,0E0,1.051445E-1,6.0502434E-1,0E0,0E0,9.0948904E-1,1.3052382E0,9.145622E-1,1.4500947E0,1.0866718E0,5.551834E-1,1.9763852E0,0E0,1.7844691E0,1.0335093E0,6.1007154E-1,9.7152084E-1,9.9800026E-1,1.5479784E0,3.2071114E0,0E0,0E0,0E0,7.040386E-1,1.5048676E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,56,-1,-1,58,60,-1,-1,62,64,66,68,70,72,74,-1,76,78,80,82,84,86,88,-1,-1,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.62E3,1.972052E5,3.9661028E7,1.16E2,9.48E2,1.1459359E3,3.82E3,1.3745962E0,1.5522388E0,4.97E2,3.7105262E0,1.8062708E6,1.059448E7,1.3795834E3,8E0,7.7E1,6.5E1,1.3050649E4,1.318849E6,4.2619047E0,5.3E1,4.2010452E2,9.932432E0,7.075161E2,3.3386944E5,3.116E3,1.5484655E6,-4.511751E-2,1.91E2,-8.264342E-3,9.315105E-3,1E1,3.6E1,-1.4789872E-2,-1.8453912E-3,1.6E1,8.03211E1,3.9401392E6,5.130175E2,4.1973075E6,7.668863E6,2E0,-4.4406604E-2,5.9192824E0,1E0,2.4177586E2,4.9E1,1.3E1,1.6942337E1,2E0,-2.7186114E-2,2.423186E-2,8.1062905E-2,4.25681E-1,2.5842668E7,-5.8034784E-3,-2.5438337E-2,-4.2023968E-2,-1.9553728E-2,-4.0827528E-2,-1.8863719E-2,-1.5161987E-3,3.5694808E-2,-2.6065527E-2,-5.672473E-3,-3.0742172E-2,-5.9149057E-2,-2.0004265E-2,1.0489132E-2,-1.633231E-2,2.6901793E-3,-2.5478413E-2,-4.514603E-2,-1.8447401E-3,-1.3109896E-2,1.4044499E-2,-3.6214744E-3,2.345136E-2,1.1715922E-3,-9.4388435E-3,5.828127E-3,-5.516775E-3,-3.4942E-2,9.031879E-3,-5.0633266E-3,3.0272402E-2,7.1228594E-3,3.566211E-2,7.5517707E-3,-1.2797235E-3,2.601201E-2,5.1119428E-2,3.088024E-2],"split_indices":[2,40,57,41,2,64,0,50,65,2,66,40,12,64,3,41,0,40,9,66,3,64,66,64,45,2,40,0,0,0,0,8,2,0,0,0,64,57,64,40,5,29,0,68,6,4,3,3,68,6,0,0,0,50,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,7.23E2,2.85E2,2.25E2,4.98E2,2.66E2,1.9E1,9.2E1,1.33E2,2.8E2,2.18E2,1.3E2,1.36E2,1.5E1,4E0,8.7E1,5E0,3.6E1,9.7E1,8.3E1,1.97E2,1.08E2,1.1E2,6.6E1,6.4E1,2.2E1,1.14E2,7E0,8E0,2E0,2E0,7.1E1,1.6E1,3E0,2E0,5E0,3.1E1,5.8E1,3.9E1,5.4E1,2.9E1,1.94E2,3E0,6E1,4.8E1,9.4E1,1.6E1,4.1E1,2.5E1,6.2E1,2E0,2E0,2E1,4.2E1,7.2E1,2E0,6E0,6.8E1,3E0,7E0,9E0,2E0,3E0,9E0,2.2E1,5.5E1,3E0,3.6E1,3E0,4.7E1,7E0,2.3E1,6E0,1.53E2,4.1E1,4.4E1,1.6E1,4.3E1,5E0,5E0,8.9E1,1.4E1,2E0,1.8E1,2.3E1,1.2E1,1.3E1,5.1E1,1.1E1,2E0,4E1,5.6E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.8060206E-3,-2.0393403E-1,5.7864773E-1,-6.1152923E-1,-4.1125532E-2,6.7910844E-1,-3.709379E-1,-8.559121E-1,-4.686689E-1,-1.9980156E-1,1.4138977E-1,1.4972037E0,6.019998E-1,-1.0167042E-2,-3.9591793E-2,-8.771785E-1,-1.7888825E-3,-6.09683E-1,-2.3633212E-1,-4.701662E-1,-1.286504E-1,-1.0663123E-1,2.0934743E-1,7.473643E-2,3.591895E-2,4.171171E-1,8.98569E-1,1.8947752E-1,-2.939211E-1,-8.911842E-1,-1.1206419E-2,-6.8770534E-1,-4.091829E-1,6.4473796E-1,-3.8442403E-1,-3.658954E-1,-5.412312E-1,-6.409656E-2,-3.4466624E-1,-3.3220837E-3,-4.0156293E-1,1.4021069E-1,3.6256826E-1,1.2748384E-1,5.425037E-1,9.2589647E-1,-6.7934594E-3,-8.063437E-3,3.7099835E-1,-1.5471789E-2,-5.7142717E-3,-3.5512023E-2,-4.7520474E-2,-2.7264096E-2,-5.0045E-2,-2.2903318E-2,-9.929809E-3,1.5533171E-2,3.495556E-2,-2.9141895E-2,-1.0560619E-2,-1.8498816E-2,-7.79865E-3,-2.5968473E-2,-1.0328047E-2,-3.925174E-3,1.3735049E-2,-8.416567E-3,-2.5559146E-2,1.3856556E-3,-1.1505077E-2,-2.4981325E-2,-5.5649593E-3,1.351931E-3,1.2143153E-2,2.9071443E-2,1.4183776E-2,-1.2856917E-2,9.237465E-3,1.8146897E-2,3.0536747E-2,4.4399433E-2,1.1421319E-2,2.1222882E-2,4.389765E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.196282E2,4.9280308E1,2.5531868E1,7.0003357E0,1.5435398E1,1.4238419E1,4.597174E0,1.3036385E0,4.3116455E0,5.4219894E0,4.202345E0,5.859642E-1,1.1742737E1,9.6209073E-1,0E0,4.476967E-1,0E0,1.0326557E0,7.1632586E0,2.3915768E-1,3.1433437E0,1.6439997E0,2.0148363E0,0E0,0E0,4.942463E0,2.5391312E0,7.277372E-1,9.02617E-3,4.3577576E-1,0E0,1.5848713E0,3.4108782E-1,2.7848244E-2,1.6311922E0,1.2674594E-1,1.2492275E-1,1.3907197E0,1.6758628E0,3.469301E-1,4.9280143E-1,1.8628387E0,6.7672205E-1,1.3461535E0,1.4685364E0,2.0224533E0,0E0,0E0,1.3139564E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,3.5056704E7,7.7E1,9.31E2,3.6E1,9.7172376E7,1.2434699E8,6.34E2,4.03E2,4.7567694E5,6.7321223E-1,2.8530578E6,5.396127E9,-3.9591793E-2,1.7971E4,-1.7888825E-3,1.318849E6,2.74614E5,3.1E1,2.21875E0,4.6330696E7,2.331083E6,7.473643E-2,3.591895E-2,7.075161E2,1.0534078E4,1.926089E6,9.4578236E-1,8.75477E5,-1.1206419E-2,1.5E1,4.1277572E5,1.9E1,6.953366E7,5E0,1.7907799E2,6.9307615E6,9E0,4.2609915E6,1.1840488E8,2.437372E6,7E0,1E0,5.97E3,2E0,-6.7934594E-3,-8.063437E-3,1.168E3,-1.5471789E-2,-5.7142717E-3,-3.5512023E-2,-4.7520474E-2,-2.7264096E-2,-5.0045E-2,-2.2903318E-2,-9.929809E-3,1.5533171E-2,3.495556E-2,-2.9141895E-2,-1.0560619E-2,-1.8498816E-2,-7.79865E-3,-2.5968473E-2,-1.0328047E-2,-3.925174E-3,1.3735049E-2,-8.416567E-3,-2.5559146E-2,1.3856556E-3,-1.1505077E-2,-2.4981325E-2,-5.5649593E-3,1.351931E-3,1.2143153E-2,2.9071443E-2,1.4183776E-2,-1.2856917E-2,9.237465E-3,1.8146897E-2,3.0536747E-2,4.4399433E-2,1.1421319E-2,2.1222882E-2,4.389765E-3],"split_indices":[2,40,57,41,2,0,57,12,2,2,40,50,40,12,0,6,0,9,5,10,65,5,40,0,0,64,4,1,46,43,0,8,57,0,7,8,64,40,8,57,7,9,3,24,2,6,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.006E3,7.42E2,2.64E2,2.11E2,5.31E2,2.39E2,2.5E1,7.6E1,1.35E2,2.84E2,2.47E2,1.9E1,2.2E2,1.5E1,1E1,7.4E1,2E0,8.3E1,5.2E1,5.8E1,2.26E2,5.3E1,1.94E2,1.5E1,4E0,1.37E2,8.3E1,9E0,6E0,7.2E1,2E0,5.8E1,2.5E1,7E0,4.5E1,2.6E1,3.2E1,1.75E2,5.1E1,4E1,1.3E1,1.35E2,5.9E1,4.2E1,9.5E1,8.1E1,2E0,3E0,6E0,4E0,2E0,4.3E1,2.9E1,4.9E1,9E0,1.6E1,9E0,3E0,4E0,1.6E1,2.9E1,2.1E1,5E0,2.9E1,3E0,1.66E2,9E0,3E1,2.1E1,3.6E1,4E0,8E0,5E0,7.2E1,6.3E1,8E0,5.1E1,6E0,3.6E1,4.5E1,5E1,7.6E1,5E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.7720073E-3,-1.9673449E-1,5.748723E-1,-5.9583646E-1,-1.7345853E-2,-3.2686234E-2,6.207208E-1,-7.710779E-1,-4.936326E-1,-1.6410981E-1,1.4705373E-1,3.442735E-1,8.3318526E-1,-3.9697338E-2,-6.3941276E-1,-5.402071E-1,1.7982872E-1,-2.4421054E-1,5.908375E-2,2.7706066E-1,5.390903E-3,1.8424462E-1,6.9269365E-1,7.115126E-2,7.4420494E-1,-3.7256777E-1,-3.694815E-2,-4.7858983E-1,-9.361267E-1,5.526534E-1,-1.6082998E-1,-4.1173542E-1,-1.476706E-1,2.1695E-2,1.8027361E-2,1.639771E-1,4.6691528E-1,4.4924986E-2,-5.1910955E-1,5.04404E-1,8.7228164E-2,4.7611699E-1,8.096626E-1,8.815994E-1,4.0474194E-1,4.598761E-3,-2.0542722E-2,-2.6349733E-2,-1.2460028E-2,-4.7518305E-2,-2.2644753E-2,1.114617E-2,3.1160872E-2,8.494165E-4,-1.1717818E-2,-1.678514E-2,-4.3199193E-2,-4.260743E-3,-1.6599415E-2,1.9511377E-3,-1.906517E-2,4.180781E-3,2.56135E-2,2.4162624E-2,-2.4638811E-2,-1.3924558E-3,6.431396E-3,-2.6973328E-3,-3.0442676E-2,7.4827466E-3,2.9080039E-2,1.0448711E-2,-3.268986E-3,2.3274727E-2,8.955442E-3,1.3541063E-2,3.906433E-2,3.457676E-2,5.151388E-2,2.1661222E-2,-5.394926E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,-1,43,45,-1,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0966795E2,5.1251225E1,1.5069809E1,3.5954742E0,1.196746E1,0E0,1.3809601E1,4.2900085E-1,4.5824966E0,4.6984553E0,4.3081193E0,5.782646E0,7.4466476E0,0E0,1.3294373E0,2.8209763E0,1.3589429E0,3.0605574E0,8.640624E-1,2.5427084E0,2.3664489E0,2.2544956E0,4.504881E-1,0E0,5.3332977E0,6.0957694E-1,0E0,2.1704483E0,2.3134804E-1,3.0296326E-2,1.0616249E-1,1.3699665E0,1.4216275E0,5.98624E-1,0E0,2.189409E0,2.952467E0,7.6268876E-1,4.9024034E-1,6.440859E-1,1.3047441E0,2.1680355E-2,1.6967106E-1,1.8136673E0,1.3471365E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,-1,44,46,-1,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,9.9E1,9.31E2,-3.2686234E-2,1.1043767E3,3E1,4.9182608E2,2.331083E6,3.8396227E0,5.53648E8,1.059448E7,-3.9697338E-2,8.910034E0,9.932432E0,1.04E2,4.97E2,6.789622E7,4.569524E2,1.2294118E1,7E0,6.714162E8,7.115126E-2,9E1,3.0791914E0,-3.694815E-2,6.15E2,7.86127E6,2.239E3,1.2972876E7,1E0,2.4285715E0,3E1,1.8027361E-2,5.405423E6,2.83E3,9.860918E5,1.4787234E1,4.9963706E5,1.1736916E0,1E0,5.943433E4,4.391553E6,4.1325716E7,4.598761E-3,-2.0542722E-2,-2.6349733E-2,-1.2460028E-2,-4.7518305E-2,-2.2644753E-2,1.114617E-2,3.1160872E-2,8.494165E-4,-1.1717818E-2,-1.678514E-2,-4.3199193E-2,-4.260743E-3,-1.6599415E-2,1.9511377E-3,-1.906517E-2,4.180781E-3,2.56135E-2,2.4162624E-2,-2.4638811E-2,-1.3924558E-3,6.431396E-3,-2.6973328E-3,-3.0442676E-2,7.4827466E-3,2.9080039E-2,1.0448711E-2,-3.268986E-3,2.3274727E-2,8.955442E-3,1.3541063E-2,3.906433E-2,3.457676E-2,5.151388E-2,2.1661222E-2,-5.394926E-3],"split_indices":[2,40,17,41,2,0,64,67,64,40,66,7,12,0,68,66,10,2,7,64,65,3,7,0,8,66,0,2,5,2,5,105,65,3,0,40,2,40,68,40,50,82,45,40,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.62E2,7.15E2,2.47E2,2.21E2,4.94E2,8E0,2.39E2,7.9E1,1.42E2,2.61E2,2.33E2,1.05E2,1.34E2,4.3E1,3.6E1,1.33E2,9E0,1.92E2,6.9E1,1.21E2,1.12E2,7.3E1,3.2E1,1.3E1,1.21E2,1.5E1,2.1E1,1.17E2,1.6E1,4E0,5E0,6.9E1,1.23E2,6.3E1,6E0,7.7E1,4.4E1,1.05E2,7E0,1.6E1,5.7E1,1.3E1,1.9E1,8.5E1,3.6E1,2E0,1.3E1,7.9E1,3.8E1,1.2E1,4E0,2E0,2E0,2E0,3E0,6.5E1,4E0,9.9E1,2.4E1,6.1E1,2E0,6.6E1,1.1E1,4.2E1,2E0,5.9E1,4.6E1,2E0,5E0,5E0,1.1E1,3E1,2.7E1,1.1E1,2E0,2E0,1.7E1,5.8E1,2.7E1,3.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-4.8373E-4,-1.7292647E-1,5.838658E-1,-5.372178E-1,-2.5977219E-2,5.3297795E-2,7.169414E-1,-7.926597E-1,-4.50907E-1,1.2408231E-2,-9.127199E-1,-5.380596E-1,2.4893643E-1,8.2895696E-2,6.7624545E-1,-3.7214354E-2,-1.6824603E-2,-1.2336244E-1,-5.042705E-1,-1.502545E-1,1.1085041E-1,-4.8203918E-1,-4.7602065E-2,-7.950126E-3,-6.404855E-1,4.84497E-1,9.472842E-2,5.868462E-1,1.1342113E0,-2.974424E-1,3.466056E-2,-4.745349E-1,-9.5417535E-1,-5.318515E-2,-4.203126E-1,1.411521E-1,-4.9643013E-1,-2.6798977E-2,-6.996918E-3,-3.3879157E-2,-9.204105E-3,1.2431131E-2,3.0300386E-2,3.9956357E-2,2.051104E-2,6.362508E-1,5.794027E-2,3.2729007E-2,5.8206454E-2,-1.6936898E-2,-2.8779486E-3,6.919467E-3,-6.6645024E-3,-2.5519E-2,-1.6378727E-2,-2.2383662E-2,-5.9673924E-2,3.0770688E-3,-8.231821E-3,-2.2695648E-2,6.214019E-3,3.9158367E-2,5.510653E-3,-8.30345E-3,-3.0670471E-2,9.796246E-3,-6.1416626E-4,1.4801394E-2,3.1686146E-2,-5.524283E-3,1.163604E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,-1,-1,-1,-1,-1,65,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0127088E2,4.1527134E1,1.6130486E1,4.5591583E0,1.8868387E1,5.571484E0,6.880966E0,1.14852905E-1,2.8917313E0,8.535504E0,7.252598E-1,3.3773303E-1,1.261816E0,0E0,6.62204E0,0E0,0E0,6.9802547E-1,1.4870453E0,5.2467337E0,6.179414E0,1.4518106E-1,0E0,0E0,2.3685718E-1,3.4596276E-1,4.2537558E-1,3.9088287E0,6.676216E-1,1.7777705E-1,3.1375054E-1,1.1211319E0,6.5543556E-1,2.268833E0,2.2402134E0,4.6732717E0,7.444887E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.9695295E-1,0E0,2.100521E0,5.1797205E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,43,43,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,-1,-1,-1,-1,-1,66,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,5.02E2,4.7567694E5,3.89E2,1E0,8.147158E4,1.4E1,4.3827028E5,1.2051282E0,9.14E2,3.2856784E7,2.6463525E8,7.6937294E-1,8.2895696E-2,6.663214E6,-3.7214354E-2,-1.6824603E-2,3.6E2,4.5E1,6.516E3,1.6254545E1,6.6E1,-4.7602065E-2,-7.950126E-3,1.6E1,8.625455E2,3.0561172E5,2.3876712E7,8.901939E2,1.0865825E0,1.2894E4,1.972052E5,5.5E1,1E0,1.2673605E7,2.04115E5,8.620714E2,-2.6798977E-2,-6.996918E-3,-3.3879157E-2,-9.204105E-3,1.2431131E-2,3.0300386E-2,2.4E0,2.051104E-2,7.760169E2,1.8235294E0,3.2729007E-2,5.8206454E-2,-1.6936898E-2,-2.8779486E-3,6.919467E-3,-6.6645024E-3,-2.5519E-2,-1.6378727E-2,-2.2383662E-2,-5.9673924E-2,3.0770688E-3,-8.231821E-3,-2.2695648E-2,6.214019E-3,3.9158367E-2,5.510653E-3,-8.30345E-3,-3.0670471E-2,9.796246E-3,-6.1416626E-4,1.4801394E-2,3.1686146E-2,-5.524283E-3,1.163604E-2],"split_indices":[2,2,40,1,80,40,0,40,65,2,57,7,50,0,40,0,0,2,6,41,66,0,0,0,8,64,45,57,64,68,12,40,0,24,57,5,4,0,0,0,0,0,0,66,0,64,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.003E3,7.75E2,2.28E2,2.22E2,5.53E2,4.6E1,1.82E2,5.4E1,1.68E2,5.31E2,2.2E1,1.1E1,3.5E1,5E0,1.77E2,5.1E1,3E0,2.4E1,1.44E2,2E2,3.31E2,6E0,1.6E1,3E0,8E0,1.3E1,2.2E1,1.5E2,2.7E1,1.1E1,1.3E1,1.37E2,7E0,1.48E2,5.2E1,3.16E2,1.5E1,4E0,2E0,6E0,2E0,7E0,6E0,2E1,2E0,1.37E2,1.3E1,8E0,1.9E1,8E0,3E0,8E0,5E0,7.9E1,5.8E1,4E0,3E0,7.6E1,7.2E1,4.6E1,6E0,8E0,3.08E2,6E0,9E0,4E0,1.6E1,2.1E1,1.16E2,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[3.8438076E-3,-1.8884808E-1,5.6937104E-1,-5.7640517E-1,-2.0070598E-2,1.3565744E0,5.012735E-1,-7.883865E-1,-4.6501455E-1,-1.8982136E-1,1.03823185E-1,6.83484E-2,3.1731136E-2,2.328788E-1,7.1669716E-1,-1.2354031E0,-6.634926E-1,1.2273106E-1,-5.045214E-1,-2.982253E-1,1.3618238E-2,2.2934932E-1,-3.038997E-2,-4.6619737E-1,3.0477118E-1,7.9639703E-1,1.9425365E-1,-3.275436E-2,-6.546334E-2,-7.124321E-1,9.175652E-3,3.5362113E-2,-2.646055E-1,-6.4763534E-1,-3.811322E-1,-4.3924442E-1,-1.977878E-1,-9.738599E-3,1.573439E-2,1.8162782E-1,5.3284365E-1,1.3867569E-2,-2.8605807E-1,-7.310716E-3,-3.4053475E-2,8.433225E-1,2.517461E-1,6.144503E-1,9.101079E-1,-1.2453984E-1,4.0844405E-1,-3.3295587E-2,-1.1319994E-2,-2.1247761E-2,-2.128314E-5,-3.1181082E-2,-1.1558753E-3,-2.1211676E-2,-9.563839E-3,-1.6823828E-2,-2.7071062E-2,-1.3882897E-2,-1.0037335E-3,1.8888135E-3,-9.294753E-3,1.0084156E-2,-8.376593E-3,5.093196E-3,2.9001294E-2,-5.131203E-3,3.4091221E-3,-6.7668967E-3,-2.4916776E-2,1.3830256E-2,5.0911967E-2,3.93268E-3,1.5716838E-2,3.0668678E-2,2.5207172E-3,4.4079755E-2,1.6638251E-2,-1.254304E-2,4.2381845E-3,6.467256E-3,2.2562012E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,-1,53,55,57,59,61,63,-1,65,67,69,71,-1,-1,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0548393E2,4.721511E1,1.250721E1,4.804573E0,1.0619364E1,6.3503647E-1,1.3015728E1,3.3117142E0,3.4520416E0,4.701491E0,4.9263687E0,0E0,0E0,5.318683E0,5.091774E0,4.7087288E-1,2.6771717E0,2.7103891E0,2.172409E0,1.8818636E0,5.7956696E-1,2.0901847E0,1.6107309E0,7.269435E-1,2.4720297E0,1.6959229E0,1.2799525E0,0E0,0E0,1.5239334E-1,0E0,0E0,3.658695E-1,1.1879673E0,9.640684E-1,4.0852165E-1,1.5188427E0,7.039056E-1,0E0,1.861999E0,7.6173735E-1,9.329768E-1,6.671233E-1,0E0,0E0,8.1621647E-1,1.2777314E0,1.3043194E0,1.4178276E0,2.828415E-1,1.7450333E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,-1,54,56,58,60,62,64,-1,66,68,70,72,-1,-1,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.6E1,3.434405E7,8.1E2,6.7321223E-1,1.4283589E6,2E0,6E0,1.5859042E6,3.7105262E0,6.83484E-2,3.1731136E-2,8.147158E4,1E0,6.0512455E1,4.2210345E2,1.1634076E4,2.661509E4,4.48E2,7.4711205E6,5.5426865E6,9.932432E0,3.019898E8,1.2083E4,5.97E3,5.071E3,-3.275436E-2,-6.546334E-2,7.280084E0,9.175652E-3,3.5362113E-2,4.54E2,2.908E3,8.01E2,6.2233735E-6,1.559733E6,7.37E2,1.573439E-2,2.19724E5,3.3852024E7,1.221E3,3.5E1,-7.310716E-3,-3.4053475E-2,1.7754222E3,7.075161E2,1.853776E7,2.2040408E7,2.8E1,3.714144E8,-3.3295587E-2,-1.1319994E-2,-2.1247761E-2,-2.128314E-5,-3.1181082E-2,-1.1558753E-3,-2.1211676E-2,-9.563839E-3,-1.6823828E-2,-2.7071062E-2,-1.3882897E-2,-1.0037335E-3,1.8888135E-3,-9.294753E-3,1.0084156E-2,-8.376593E-3,5.093196E-3,2.9001294E-2,-5.131203E-3,3.4091221E-3,-6.7668967E-3,-2.4916776E-2,1.3830256E-2,5.0911967E-2,3.93268E-3,1.5716838E-2,3.0668678E-2,2.5207172E-3,4.4079755E-2,1.6638251E-2,-1.254304E-2,4.2381845E-3,6.467256E-3,2.2562012E-2],"split_indices":[2,40,0,7,2,50,40,29,3,40,66,0,0,40,6,45,64,40,40,2,40,40,66,7,9,2,2,0,0,65,0,0,1,2,2,49,9,2,0,1,7,2,3,0,0,4,64,59,63,3,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.66E2,7.21E2,2.45E2,2.18E2,5.03E2,1.8E1,2.27E2,7.3E1,1.45E2,2.12E2,2.91E2,1.4E1,4E0,1.02E2,1.25E2,1.4E1,5.9E1,9E0,1.36E2,1.38E2,7.4E1,1.5E2,1.41E2,9E0,9.3E1,1.08E2,1.7E1,5E0,9E0,5.6E1,3E0,3E0,6E0,6.1E1,7.5E1,5.6E1,8.2E1,7E1,4E0,1.31E2,1.9E1,1.21E2,2E1,5E0,4E0,7E0,8.6E1,4.4E1,6.4E1,7E0,1E1,5.4E1,2E0,3E0,3E0,5.8E1,3E0,5E1,2.5E1,4E1,1.6E1,5.1E1,3.1E1,5.6E1,1.4E1,1.19E2,1.2E1,4E0,1.5E1,3.9E1,8.2E1,1.4E1,6E0,3E0,4E0,3.1E1,5.5E1,4E1,4E0,5.8E1,6E0,4E0,3E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[2.1086197E-2,-1.6189569E-1,5.2352494E-1,-4.9145257E-1,-1.6754683E-2,-6.6713876E-1,5.8194333E-1,-7.167674E-1,-3.468926E-1,-1.5452714E-1,1.2270433E-1,-3.48818E-2,-1.2384325E-2,2.617757E-1,7.0822996E-1,-6.474436E-1,-4.3476023E-2,-5.434946E-1,-7.897505E-2,-2.2192584E-1,8.170851E-2,1.8243384E-1,-1.2583534E-1,1.2810698E-1,6.41969E-1,8.278331E-1,3.824369E-1,-3.4865282E-2,-4.911657E-1,-4.6798068E-1,-9.7462904E-1,6.580705E-1,-2.3954557E-1,-2.0268469E-1,-5.139196E-2,1.6659714E-1,-1.5193495E-2,2.7563202E-1,1.0628891E-1,-6.677126E-2,-5.478434E-1,6.743377E-2,2.2217872E-2,4.224423E-1,3.5055924E-2,8.8775814E-1,3.3414665E-1,4.6463382E-1,-2.0060574E-1,-2.8379455E-2,-1.4072654E-2,-1.1517125E-2,-2.3783902E-2,-6.956491E-2,-2.6771098E-2,1.0168221E-2,3.3553723E-2,-2.8618088E-2,-7.4695405E-3,-1.727232E-2,-6.203663E-3,-1.0128468E-3,1.02215335E-2,-5.1542805E-3,4.1267215E-3,1.475068E-2,-5.956762E-3,3.000178E-3,1.7227579E-2,-1.4870767E-2,-1.5526096E-3,-3.2162674E-2,-1.3622938E-2,8.268244E-3,-2.1974624E-3,2.1823462E-2,7.288874E-3,4.7770698E-2,3.2716688E-2,2.218647E-2,5.7359682E-3,1.692563E-2,4.2894464E-2,-1.2746193E-2,-3.378537E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,-1,75,-1,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.229776E1,3.519628E1,1.9029419E1,7.0799446E0,9.856329E0,2.7216864E-1,1.0165421E1,7.729645E-1,7.2786427E0,4.1260037E0,3.8084705E0,0E0,0E0,3.6784363E0,6.814163E0,8.4830856E-1,0E0,2.1484222E0,7.2585044E0,3.2421703E0,4.7963223E-1,1.4325962E0,1.2096957E0,1.1756372E0,1.8332624E-1,3.6380463E0,2.5373473E0,0E0,5.5986977E-1,6.279192E-1,1.4440308E0,1.8628025E-1,1.373714E0,2.270337E0,0E0,3.3205724E-1,2.954346E-1,1.7387967E0,1.2418941E0,3.703199E-1,2.2042036E-2,6.445141E-1,0E0,4.9670458E-2,0E0,2.412201E0,4.381888E-1,1.6858969E0,4.7977388E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,-1,76,-1,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.8988005E5,1E0,9.9E1,9.31E2,1.665909E3,7.760169E2,7.046396E8,6.76E2,2.331083E6,3.1316226E0,-3.48818E-2,-1.2384325E-2,3.5510652E6,8.7E1,4.622222E1,-4.3476023E-2,4.4895835E0,2.74614E5,6.0692043E0,3.5364E4,2.9375E0,4.1E1,5.53648E8,6.46E2,2E0,4.1325716E7,-3.4865282E-2,2.231884E0,2E0,2E0,2.123E3,1E0,1.7312857E2,-5.139196E-2,1.45064E5,2.2907576E2,2.43025E5,2.41E3,7.547528E6,6.185E3,1.124641E0,2.2217872E-2,6.16E2,3.5055924E-2,2.1298597E0,3.97371E5,4.907764E3,1.620407E6,-2.8379455E-2,-1.4072654E-2,-1.1517125E-2,-2.3783902E-2,-6.956491E-2,-2.6771098E-2,1.0168221E-2,3.3553723E-2,-2.8618088E-2,-7.4695405E-3,-1.727232E-2,-6.203663E-3,-1.0128468E-3,1.02215335E-2,-5.1542805E-3,4.1267215E-3,1.475068E-2,-5.956762E-3,3.000178E-3,1.7227579E-2,-1.4870767E-2,-1.5526096E-3,-3.2162674E-2,-1.3622938E-2,8.268244E-3,-2.1974624E-3,2.1823462E-2,7.288874E-3,4.7770698E-2,3.2716688E-2,2.218647E-2,5.7359682E-3,1.692563E-2,4.2894464E-2,-1.2746193E-2,-3.378537E-3],"split_indices":[2,40,17,41,2,4,64,43,2,40,65,0,0,40,8,4,0,65,5,65,9,66,3,7,0,6,63,0,65,29,29,1,27,67,0,7,4,1,2,5,41,50,0,0,0,50,1,4,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.002E3,7.35E2,2.67E2,2.24E2,5.11E2,1.2E1,2.55E2,8.6E1,1.38E2,2.57E2,2.54E2,9E0,3E0,7.3E1,1.82E2,6.9E1,1.7E1,7.9E1,5.9E1,2E2,5.7E1,2.05E2,4.9E1,5.5E1,1.8E1,1.32E2,5E1,3.8E1,3.1E1,6.9E1,1E1,1E1,4.9E1,1.97E2,3E0,3E1,2.7E1,9.1E1,1.14E2,4.4E1,5E0,4.8E1,7E0,8E0,1E1,1.17E2,1.5E1,4.4E1,6E0,1.7E1,1.4E1,1.4E1,5.5E1,3E0,7E0,2E0,8E0,7E0,4.2E1,5.4E1,1.43E2,7E0,2.3E1,1.4E1,1.3E1,8.2E1,9E0,1E2,1.4E1,4E0,4E1,2E0,3E0,2.4E1,2.4E1,6E0,2E0,6E1,5.7E1,8E0,7E0,3.8E1,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.7460564E-2,-3.6376417E-1,2.1413898E-1,-6.6233605E-1,-2.2881177E-1,2.7219212E-2,5.802781E-1,-6.836673E-1,-9.897847E-2,-3.082942E-1,3.6980104E-1,-1.0641033E-1,1.6011429E-1,6.309057E-1,3.739382E-2,-6.4023143E-1,-4.3748148E-2,-7.7742566E-3,1.9719987E-4,-4.4049332E-1,-1.6925149E-1,7.8828394E-1,1.3008368E-1,-4.3955315E-2,-2.9070508E-1,9.801484E-2,4.0487307E-1,5.0159025E-1,8.5922146E-1,2.6168826E-1,-2.1306944E-1,-6.7494047E-1,-4.036006E-1,-5.3447765E-1,-3.0121502E-1,4.811823E-1,-2.2908993E-1,4.0206596E-2,1.3472747E-2,2.791943E-1,-3.7416979E-3,-9.394852E-2,1.1337232E-1,-5.106239E-1,-1.3985457E-1,1.1356896E-1,-2.7522389E-2,7.8156844E-2,4.9866736E-1,4.137323E-1,7.9937094E-1,1.1580023E0,6.4276993E-1,1.575612E-2,4.384873E-3,-5.961764E-4,-1.3640198E-2,-2.5809214E-2,-3.344111E-2,-1.9954612E-2,-6.692216E-3,-2.2330629E-2,-3.3853866E-2,8.442503E-3,-1.5067785E-2,2.8239153E-2,7.369527E-3,-1.472539E-2,6.0474576E-5,1.300314E-3,1.59305E-2,4.343787E-3,-4.658867E-3,-1.72612E-2,-2.1175789E-3,1.2621989E-2,7.294166E-4,-1.8106826E-2,-4.2988174E-2,-2.0531907E-3,-1.1212276E-2,1.1589663E-2,1.4608948E-3,-5.867146E-4,4.583184E-3,1.1654913E-2,2.648824E-2,2.1312485E-2,3.8817616E-3,3.8161185E-2,1.1473991E-2,3.6138665E-2,6.1600685E-2,3.6338E-2,2.0329446E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.468168E1,1.3507774E1,4.4948387E1,1.19413E0,1.1295121E1,7.759834E0,6.078079E0,5.6941986E-1,3.701587E-2,3.7487602E0,2.7063506E0,2.4964306E0,3.288982E0,5.562546E0,1.1793082E0,4.6678543E-1,0E0,0E0,0E0,1.2265453E0,4.119092E0,1.742258E-1,3.8018557E-1,1.3005624E0,1.7532043E0,1.9767064E0,1.3093657E0,3.1227112E0,3.9120216E0,1.1599034E-1,1.640178E-1,5.3268433E-2,4.8059702E-2,2.483635E-1,6.4596844E-1,2.870884E-1,2.003326E0,0E0,0E0,1.4290297E-1,1.1523698E-1,1.6683593E0,6.251509E-1,7.2024155E-1,3.263719E-1,1.954675E0,0E0,2.2905774E-2,5.0338554E-1,1.6820812E0,2.93787E-1,9.162941E-1,9.5653343E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.257507E5,8.8E1,3.167E3,1.1408248E0,5.862504E2,9.31E2,2.856934E7,1.1558494E8,6.5E1,6.76E2,1E0,4.066351E0,2.8971518E6,1.6466942E3,2.4077083E3,4.2E1,-4.3748148E-2,-7.7742566E-3,1.9719987E-4,7.90913E5,7.784E3,3.4836065E-2,2.9789968E1,3.3486558E6,1E0,2.4994128E8,6.1723955E9,4.391553E6,1.4595818E7,5.369854E5,3.0070068E7,1E0,6.364486E0,2.4E1,1E0,1.924525E4,3.6161574E4,4.0206596E-2,1.3472747E-2,6.1319604E2,2.8E1,1.4110284E2,5.894636E6,8.2E1,8.6875E0,1.4915254E0,-2.7522389E-2,2.7513E4,1E0,2E0,3.2462872E5,2.2365898E3,1.2792593E2,1.575612E-2,4.384873E-3,-5.961764E-4,-1.3640198E-2,-2.5809214E-2,-3.344111E-2,-1.9954612E-2,-6.692216E-3,-2.2330629E-2,-3.3853866E-2,8.442503E-3,-1.5067785E-2,2.8239153E-2,7.369527E-3,-1.472539E-2,6.0474576E-5,1.300314E-3,1.59305E-2,4.343787E-3,-4.658867E-3,-1.72612E-2,-2.1175789E-3,1.2621989E-2,7.294166E-4,-1.8106826E-2,-4.2988174E-2,-2.0531907E-3,-1.1212276E-2,1.1589663E-2,1.4608948E-3,-5.867146E-4,4.583184E-3,1.1654913E-2,2.648824E-2,2.1312485E-2,3.8817616E-3,3.8161185E-2,1.1473991E-2,3.6138665E-2,6.1600685E-2,3.6338E-2,2.0329446E-2],"split_indices":[40,41,2,50,64,2,57,57,0,2,27,66,40,64,64,6,0,0,0,9,1,69,70,40,27,12,43,40,59,40,55,22,66,8,20,40,45,0,0,64,8,4,57,0,70,65,0,9,27,6,45,64,70,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.94E2,3.38E2,6.56E2,1.04E2,2.34E2,4.35E2,2.21E2,1E2,4E0,2.07E2,2.7E1,2.17E2,2.18E2,2.02E2,1.9E1,8.9E1,1.1E1,2E0,2E0,1.05E2,1.02E2,9E0,1.8E1,1.63E2,5.4E1,1.75E2,4.3E1,1.31E2,7.1E1,1E1,9E0,7.6E1,1.3E1,6.1E1,4.4E1,8E0,9.4E1,7E0,2E0,8E0,1E1,1.24E2,3.9E1,2.1E1,3.3E1,1.72E2,3E0,1E1,3.3E1,1.03E2,2.8E1,2.8E1,4.3E1,6E0,4E0,3E0,6E0,2.8E1,4.8E1,1.1E1,2E0,5.2E1,9E0,2E0,4.2E1,5E0,3E0,6.7E1,2.7E1,2E0,6E0,5E0,5E0,1.7E1,1.07E2,1.4E1,2.5E1,1.8E1,3E0,1.8E1,1.5E1,6.3E1,1.09E2,2E0,8E0,9E0,2.4E1,8.9E1,1.4E1,2.6E1,2E0,1.1E1,1.7E1,2.3E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[8.253235E-3,-1.67044E-1,4.7900128E-1,-4.9911642E-1,-1.2645365E-2,-5.2040017E-1,5.3283644E-1,-4.2846775E-1,-7.881766E-1,-1.413968E-1,1.3553171E-1,-3.1299524E-2,-2.620135E-1,2.608999E-1,7.266202E-1,-5.9809786E-1,-2.938632E-1,-4.8551345E-1,-9.329778E-1,-3.9595646E-1,-8.48281E-2,9.4111465E-2,5.353907E-1,-1.3003204E-3,-1.6068367E-2,1.0835735E-1,4.9076575E-1,1.2846204E0,6.2785333E-1,-7.026659E-1,-5.2130365E-1,3.4992257E-1,-3.737725E-1,-8.004339E-3,-5.175296E-1,-1.1472396E0,-5.975034E-1,-2.4136892E-1,-4.8358428E-1,-7.1575634E-2,-2.8682696E-2,-3.4769464E-2,1.6362038E-1,1.4301603E-1,6.490693E-1,-2.2842476E-2,2.5299078E-1,5.332044E-1,-1.0962534E-2,8.0113786E-1,6.983994E-2,3.4207684E-1,8.176204E-1,-1.2652374E-2,-3.31915E-2,-2.4646549E-2,-9.706622E-3,5.779958E-4,2.374748E-2,-1.4991109E-2,-2.8380757E-2,-1.1423969E-2,-2.609975E-2,-6.0473096E-2,-2.9651292E-2,-3.151056E-2,-1.04174325E-2,-1.6384758E-2,-8.023854E-3,-2.3635328E-2,-1.9958643E-3,-2.3234605E-3,-1.4832307E-2,6.2670704E-4,-1.1013182E-2,-1.3796062E-3,9.77275E-3,9.8997E-3,-6.1704864E-5,1.2168014E-2,3.146429E-2,-3.990398E-3,9.376184E-3,1.830509E-2,5.038333E-3,2.610446E-2,-4.1111186E-3,4.067566E-2,1.5482638E-2,2.0457586E-2,4.7386494E-3,1.3383219E-2,3.9845984E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,-1,73,75,77,79,81,83,85,-1,87,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.029027E1,3.63452E1,1.4481251E1,4.2571754E0,9.271999E0,4.03996E-1,1.3037346E1,4.002678E0,1.4315434E0,3.7104397E0,3.6926842E0,0E0,1.3376218E-1,3.6636405E0,7.239792E0,2.7611542E-1,5.481017E0,6.925082E-2,1.3271427E0,5.19948E-1,1.4857321E0,1.8511143E0,8.438692E-1,0E0,0E0,1.2392111E0,1.4274244E0,1.1769981E0,6.5534096E0,6.532955E-2,1.2356949E-1,6.699108E-1,8.743057E-1,0E0,4.9480915E-2,5.094528E-1,2.7103853E-1,8.3820224E-2,3.732276E-1,1.0915926E0,0E0,7.277999E-1,1.2817624E0,6.309938E-2,3.9529324E-2,5.2643627E-1,6.028056E-1,9.258909E-1,0E0,5.4128647E-2,0E0,1.2375746E0,1.7162285E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,-1,74,76,78,80,82,84,86,-1,88,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.8988005E5,1E0,2.2E1,9.31E2,1.5900173E3,1.087521E3,2.71E2,1.19111115E2,3.89E2,5.1167645E6,-3.1299524E-2,1.91E2,2.2318378E6,1.059448E7,4.2E1,2.7E1,6.123E3,6.711294E7,3.1E1,4.6E1,8.079573E5,3.7788504E7,-1.3003204E-3,-1.6068367E-2,4.1032645E6,3.2952412E5,4.626E3,1.7587205E6,4E0,2.020055E6,1.31306E5,2.1609572E7,-8.004339E-3,6.5E1,7.9E1,9.837568E-7,9.793E3,2.0936034E6,2E0,-2.8682696E-2,7.127857E6,4.7563504E7,2.82E2,5.294E3,5.486433E5,9.113598E6,2E0,-1.0962534E-2,1.6E1,6.983994E-2,1E0,1E0,-1.2652374E-2,-3.31915E-2,-2.4646549E-2,-9.706622E-3,5.779958E-4,2.374748E-2,-1.4991109E-2,-2.8380757E-2,-1.1423969E-2,-2.609975E-2,-6.0473096E-2,-2.9651292E-2,-3.151056E-2,-1.04174325E-2,-1.6384758E-2,-8.023854E-3,-2.3635328E-2,-1.9958643E-3,-2.3234605E-3,-1.4832307E-2,6.2670704E-4,-1.1013182E-2,-1.3796062E-3,9.77275E-3,9.8997E-3,-6.1704864E-5,1.2168014E-2,3.146429E-2,-3.990398E-3,9.376184E-3,1.830509E-2,5.038333E-3,2.610446E-2,-4.1111186E-3,4.067566E-2,1.5482638E-2,2.0457586E-2,4.7386494E-3,1.3383219E-2,3.9845984E-2],"split_indices":[2,40,17,3,2,67,64,2,64,2,40,0,0,40,12,2,0,40,7,10,3,40,7,0,0,57,45,2,40,0,5,9,57,0,41,0,49,9,57,6,0,57,7,0,12,40,57,6,0,0,0,89,25,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.71E2,7.08E2,2.63E2,2.24E2,4.84E2,1.3E1,2.5E2,1.82E2,4.2E1,2.59E2,2.25E2,7E0,6E0,1.05E2,1.45E2,7.9E1,1.03E2,1.5E1,2.7E1,4.6E1,2.13E2,2.05E2,2E1,2E0,4E0,6.4E1,4.1E1,2E1,1.25E2,3E1,4.9E1,1.1E1,9.2E1,2E0,1.3E1,1.5E1,1.2E1,1.8E1,2.8E1,2.09E2,4E0,7.2E1,1.33E2,5E0,1.5E1,3.4E1,3E1,3.9E1,2E0,8E0,1.2E1,5.1E1,7.4E1,2E0,2.8E1,4.6E1,3E0,4E0,7E0,7.9E1,1.3E1,3E0,1E1,1E1,5E0,9E0,3E0,5E0,1.3E1,2.6E1,2E0,1.94E2,1.5E1,5.9E1,1.3E1,2.7E1,1.06E2,3E0,2E0,2E0,1.3E1,2.7E1,7E0,1.4E1,1.6E1,3.7E1,2E0,6E0,2E0,3.5E1,1.6E1,7E0,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-8.587656E-3,-1.6297148E-1,4.673583E-1,-4.6127307E-1,-1.8824136E-2,-6.057559E-1,5.162242E-1,-7.006527E-1,-3.5638428E-1,-1.9507995E-1,7.552191E-2,-3.374843E-2,-1.726097E-2,7.533587E-2,4.7396713E-1,-9.383492E-1,-5.996419E-1,-5.27421E-1,-2.5918046E-1,-2.5570014E-1,1.3447313E-1,1.0920795E-1,-2.1996605E-1,1.695181E-1,5.8115375E-1,-2.9026357E-2,-6.0581747E-2,-2.9600184E-2,-3.6593154E-1,-5.440859E-1,-1.0654142E-2,4.397991E-1,-3.327408E-1,-2.1172833E-1,-5.1363635E-1,5.150479E-2,2.2624393E-1,4.8974734E-2,2.4435997E-1,-4.1390294E-1,5.3280886E-2,4.0364265E-1,-3.426995E-2,6.1389273E-1,-1.1491544E-2,-8.590366E-3,-2.2223938E-2,-2.9159512E-2,-2.1949204E-2,2.6909305E-2,6.768412E-3,-2.3374941E-2,-9.126964E-3,-1.5204253E-2,-5.9475023E-3,-2.678058E-2,-3.3459882E-3,-2.8102368E-3,5.7522776E-3,1.8046519E-3,1.17889615E-2,-3.1961654E-3,5.732408E-3,2.4210243E-3,1.5957203E-2,-9.400121E-3,-2.7682273E-2,-4.236308E-3,1.3178698E-2,2.4156932E-2,7.6630292E-3,-1.2101207E-2,7.379596E-4,2.4840575E-2,3.9905127E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.25993E1,3.206886E1,1.3002884E1,5.8516006E0,8.395768E0,5.018854E-2,1.0241447E1,1.1900482E0,2.7062511E0,3.550436E0,3.2988286E0,0E0,0E0,0E0,7.2438736E0,1.4988594E0,3.2956314E-1,1.7415237E-1,5.853646E0,1.5811949E0,2.015413E-1,2.3976626E0,1.8238035E0,2.8765402E0,4.671669E0,0E0,0E0,0E0,1.415658E-1,3.919792E-2,0E0,3.8351345E-1,2.2837105E0,1.2227287E0,6.000967E-1,1.406E-1,6.384796E-2,1.8606744E0,1.7789478E0,6.667042E-1,5.3642243E-1,7.2311497E-1,3.849442E-1,2.5104866E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,2.0976269E5,1E0,3.434405E7,7.06E2,5.97E3,3.3817584E7,2E0,3.39E2,2.9652428E6,2.3E1,-3.374843E-2,-1.726097E-2,7.533587E-2,7.075161E2,1.8987958E0,1.7E1,2.511352E6,2.3636363E0,2E0,4.82425E6,2.0766992E6,9.5E2,6.5199727E-1,7.134432E7,-2.9026357E-2,-6.0581747E-2,-2.9600184E-2,2.3E1,9.9E1,-1.0654142E-2,2.04115E5,5.9668895E4,1.5422421E2,1.3E1,5.08E2,1.4979E4,1.273801E6,1.124E3,1E1,4.1E1,2.0130434E1,1E0,4.391553E6,-1.1491544E-2,-8.590366E-3,-2.2223938E-2,-2.9159512E-2,-2.1949204E-2,2.6909305E-2,6.768412E-3,-2.3374941E-2,-9.126964E-3,-1.5204253E-2,-5.9475023E-3,-2.678058E-2,-3.3459882E-3,-2.8102368E-3,5.7522776E-3,1.8046519E-3,1.17889615E-2,-3.1961654E-3,5.732408E-3,2.4210243E-3,1.5957203E-2,-9.400121E-3,-2.7682273E-2,-4.236308E-3,1.3178698E-2,2.4156932E-2,7.6630292E-3,-1.2101207E-2,7.379596E-4,2.4840575E-2,3.9905127E-2],"split_indices":[2,40,17,7,2,2,7,29,2,40,3,0,0,0,64,65,3,5,70,6,40,40,10,54,56,0,0,0,3,41,0,5,40,64,8,2,1,9,2,8,3,68,24,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.86E2,7.45E2,2.41E2,2.42E2,5.03E2,1E1,2.31E2,7.2E1,1.7E2,1.75E2,3.28E2,5E0,5E0,7E0,2.24E2,1.9E1,5.3E1,6E1,1.1E2,1.48E2,2.7E1,2.95E2,3.3E1,5.9E1,1.65E2,1.2E1,7E0,4.3E1,1E1,5.6E1,4E0,1E1,1E2,1.28E2,2E1,1.5E1,1.2E1,2.05E2,9E1,1.9E1,1.4E1,2.7E1,3.2E1,1.59E2,6E0,5E0,5E0,2E1,3.6E1,6E0,4E0,4.2E1,5.8E1,5.1E1,7.7E1,1.7E1,3E0,6E0,9E0,2E0,1E1,8E1,1.25E2,3.2E1,5.8E1,1E1,9E0,9E0,5E0,1.7E1,1E1,5E0,2.7E1,1.26E2,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.6045049E-2,-1.2717846E-1,4.6644136E-1,-3.980341E-1,-8.226667E-3,5.527263E-1,-1.332549E-1,-5.6803155E-1,-2.8112647E-1,-1.2704259E-1,1.1782138E-1,1.1095173E0,4.936933E-1,-5.3730875E-1,7.482906E-2,-5.85815E-1,-5.7686237E-4,4.531727E-1,-3.222534E-1,-6.465785E-2,-2.9067174E-1,2.392855E-1,2.9507494E-2,5.4127127E-2,2.4768949E-2,2.124679E-1,6.264243E-1,-2.9133482E-2,-9.601448E-3,-1.4350837E-1,2.6873595E-1,-6.056648E-1,-1.3742847E-2,3.0977014E-2,2.9085812E-3,-4.0777212E-1,-1.0681419E-1,-1.0874428E-1,8.913346E-2,-4.7554186E-1,-1.9264683E-1,2.5967115E-1,-1.7038241E-2,6.753892E-2,-2.4427046E-1,6.0647994E-1,1.3831453E-1,5.4709363E-1,9.205462E-1,-4.935187E-2,-1.7368637E-2,-3.432618E-3,3.4490314E-1,-2.6277343E-2,-3.8090184E-2,-2.2032335E-2,-1.2521302E-2,1.3049163E-3,-2.3669804E-2,-3.6838464E-3,-2.6434587E-2,1.6621794E-3,1.5437591E-2,-1.5657725E-2,-4.468635E-2,6.902107E-4,-1.0347716E-2,7.4636484E-3,1.6955426E-2,-2.03105E-4,7.1693556E-3,-2.5926353E-3,-1.6036134E-2,1.1956168E-2,4.136572E-2,-1.7249452E-4,1.1252512E-2,-6.2827654E-3,2.5780799E-2,1.88544E-2,4.5536738E-2,-4.553999E-3,3.8255737E-3,2.1362083E-2,6.049371E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,-1,-1,45,47,-1,-1,49,51,53,-1,-1,-1,55,57,59,61,63,65,67,-1,69,71,73,75,77,79,81,-1,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.630836E1,2.5127357E1,1.2906361E1,4.5703278E0,8.147216E0,6.5848236E0,2.730674E0,9.409504E-1,4.4645643E0,2.8394346E0,2.824859E0,1.5048599E-1,7.257717E0,2.1757364E-1,9.699828E-1,3.478756E-1,0E0,6.473069E-1,2.46212E0,1.3928349E0,1.3037724E0,1.4771667E0,1.6199713E0,0E0,0E0,1.8152409E0,2.622982E0,0E0,0E0,2.2435287E-1,3.3970875E-1,1.710968E-1,0E0,0E0,0E0,8.11985E-1,2.224278E0,2.072108E0,5.893197E-1,1.4095063E0,3.5585237E-1,1.0889525E0,0E0,8.6728936E-1,3.5450637E-1,7.837267E-1,8.4957683E-1,1.1251144E0,5.688629E-1,6.7702115E-2,0E0,0E0,1.9497359E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,29,29,30,30,31,31,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,-1,-1,46,48,-1,-1,50,52,54,-1,-1,-1,56,58,60,62,64,66,68,-1,70,72,74,76,78,80,82,-1,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,1.2E2,9.48E2,3.6E1,1E0,4.455844E1,6E0,4.325E0,3.1226995E0,6.7321223E-1,8.995735E5,1.665909E3,1.0089981E3,1.0795462E-6,-5.7686237E-4,2.00087E5,8.75E2,3.2911258E6,1E0,1.4676277E7,9.639872E0,5.4127127E-2,2.4768949E-2,1.4141E4,6.9307615E6,-2.9133482E-2,-9.601448E-3,2.5205562E5,1.8661781E5,3.583621E1,-1.3742847E-2,3.0977014E-2,2.9085812E-3,1.318849E6,2E1,2.829581E10,7.4711205E6,8.2E1,2.1111112E0,4.2210345E2,-1.7038241E-2,5.163891E2,5.194E3,3.4915986E3,5.202E3,7.549744E7,3.53562E5,2.2539758E5,-1.7368637E-2,-3.432618E-3,7.792289E9,-2.6277343E-2,-3.8090184E-2,-2.2032335E-2,-1.2521302E-2,1.3049163E-3,-2.3669804E-2,-3.6838464E-3,-2.6434587E-2,1.6621794E-3,1.5437591E-2,-1.5657725E-2,-4.468635E-2,6.902107E-4,-1.0347716E-2,7.4636484E-3,1.6955426E-2,-2.03105E-4,7.1693556E-3,-2.5926353E-3,-1.6036134E-2,1.1956168E-2,4.136572E-2,-1.7249452E-4,1.1252512E-2,-6.2827654E-3,2.5780799E-2,1.88544E-2,4.5536738E-2,-4.553999E-3,3.8255737E-3,2.1362083E-2,6.049371E-3],"split_indices":[2,40,6,41,2,0,17,70,3,66,66,50,40,4,4,49,0,5,2,40,27,59,66,0,0,9,40,0,0,45,45,68,0,0,0,9,3,43,40,0,65,64,0,67,41,4,2,7,1,45,0,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.026E3,7.79E2,2.47E2,2.37E2,5.42E2,2.16E2,3.1E1,9.5E1,1.42E2,2.79E2,2.63E2,1.9E1,1.97E2,1E1,2.1E1,9.2E1,3E0,7E0,1.35E2,2.03E2,7.6E1,1.1E2,1.53E2,1.6E1,3E0,6.4E1,1.33E2,7E0,3E0,1E1,1.1E1,8.5E1,7E0,4E0,3E0,9.6E1,3.9E1,1.58E2,4.5E1,2.5E1,5.1E1,1.07E2,3E0,1.35E2,1.8E1,9E0,5.5E1,1.07E2,2.6E1,8E0,2E0,2E0,9E0,7.7E1,8E0,6.1E1,3.5E1,3E1,9E0,1.5E2,8E0,3.8E1,7E0,2.1E1,4E0,7E0,4.4E1,5.8E1,4.9E1,7.5E1,6E1,7E0,1.1E1,5E0,4E0,2.4E1,3.1E1,2E0,1.05E2,4E0,2.2E1,6E0,2E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[9.4811525E-3,-1.3793513E-1,4.307139E-1,-4.1410962E-1,-1.1947248E-2,-4.079361E-1,4.7177953E-1,-5.98967E-1,-3.3399728E-1,-1.2100382E-1,1.11286476E-1,-2.460376E-2,-2.2193412E-1,1.9252406E-1,6.1274666E-1,-5.3517383E-1,-3.986945E-2,1.916287E-1,-3.6130556E-1,-1.9968821E-1,4.1101277E-2,7.858097E-2,4.2639315E-1,1.13530165E-4,-1.4339407E-2,4.808583E-2,4.1626346E-1,7.214113E-1,3.5437799E-1,-2.7566297E-2,-3.633523E-1,2.9203078E-2,-1.3642073E-1,-3.421213E-1,-4.725243E-2,-2.490755E-1,-1.654168E-2,7.486178E-3,3.1599647E-1,1.3513996E-2,1.7972484E-1,1.383077E-1,5.1993465E-1,1.7259505E-1,-5.727472E-2,4.9302426E-1,1.482586E-1,5.8496994E-1,9.932622E-1,1.6798587E-1,5.081432E-1,-4.9095335E-3,-1.7786397E-2,-1.4668153E-2,1.6020923E-3,-2.4219884E-2,-1.3447253E-2,-1.2704912E-2,-2.1433631E-4,-4.310356E-3,4.506336E-3,2.4652868E-3,-5.650367E-3,1.9852784E-2,7.6463586E-3,6.8026795E-3,-1.9719454E-3,1.4317881E-2,5.1870835E-3,1.2446971E-2,-1.3297686E-3,2.642107E-2,9.022217E-3,1.6419005E-2,2.9657597E-3,-7.478094E-3,5.2635246E-3,1.1389861E-2,2.7015043E-2,5.126516E-4,9.872853E-3,2.9539652E-2,1.659771E-2,5.335528E-2,2.8554047E-2,1.3207179E-2,-5.6041167E-3,1.3368347E-2,2.9005548E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,51,-1,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.395677E1,2.6543362E1,9.387852E0,3.3393822E0,7.0691757E0,1.8996191E-1,9.920319E0,7.540779E-1,2.4950542E0,3.5668917E0,2.512008E0,0E0,1.4255053E-1,2.791521E0,4.489113E0,4.5189667E-1,0E0,1.3981141E0,1.7095776E0,1.6963606E0,8.4777224E-1,1.4783245E0,5.478916E-1,0E0,0E0,7.182768E-1,6.2636185E-1,3.7957954E0,1.4049911E0,0E0,7.4994564E-2,0E0,1.9915244E-1,1.2896652E0,0E0,9.9453354E-1,3.7257826E-1,5.068886E-1,1.0183585E-1,1.0547643E0,7.406616E-1,1.6319092E-1,1.9109726E-1,4.761923E-1,5.628043E-1,4.6028137E-1,7.988629E-2,8.209133E-1,1.6552696E0,9.0184176E-1,5.660143E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,52,-1,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,1.1411E4,9.61E2,1.5900173E3,8.625455E2,5.356E3,7E0,1.5859042E6,4.5866325E6,-2.460376E-2,1.91E2,2.2318378E6,1.7493458E7,3.62E2,-3.986945E-2,1.1634076E4,2.5013582E-5,4.39776E6,6.663214E6,2.742234E6,3.574764E7,1.13530165E-4,-1.4339407E-2,3.39498E5,1.835821E1,1.767635E3,1.5416006E5,-2.7566297E-2,1.957E3,2.9203078E-2,3.23E2,2.71E2,-4.725243E-2,3.5897333E2,3.056087E2,3.8795E4,3.52E2,1E0,8.214286E0,2.2657063E3,1.4793489E9,2.276E4,3.5E1,1.2003246E6,1.606722E8,8E0,8.992806E-4,6.747343E8,1.7026364E3,-4.9095335E-3,-1.7786397E-2,-1.4668153E-2,1.6020923E-3,-2.4219884E-2,-1.3447253E-2,-1.2704912E-2,-2.1433631E-4,-4.310356E-3,4.506336E-3,2.4652868E-3,-5.650367E-3,1.9852784E-2,7.6463586E-3,6.8026795E-3,-1.9719454E-3,1.4317881E-2,5.1870835E-3,1.2446971E-2,-1.3297686E-3,2.642107E-2,9.022217E-3,1.6419005E-2,2.9657597E-3,-7.478094E-3,5.2635246E-3,1.1389861E-2,2.7015043E-2,5.126516E-4,9.872853E-3,2.9539652E-2,1.659771E-2,5.335528E-2,2.8554047E-2,1.3207179E-2,-5.6041167E-3,1.3368347E-2,2.9005548E-2],"split_indices":[2,40,17,9,2,67,64,1,3,40,40,0,0,40,63,1,0,40,49,9,40,9,7,0,0,9,68,64,45,0,9,0,1,2,0,64,4,1,0,8,70,45,5,10,8,63,12,29,69,44,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,7.62E2,2.66E2,2.38E2,5.24E2,1.2E1,2.54E2,7E1,1.68E2,2.78E2,2.46E2,6E0,6E0,8.6E1,1.68E2,5.9E1,1.1E1,8E0,1.6E2,1.87E2,9.1E1,2.24E2,2.2E1,2E0,4E0,5.3E1,3.3E1,1.17E2,5.1E1,4.1E1,1.8E1,3E0,5E0,1.57E2,3E0,1.47E2,4E1,8.2E1,9E0,1.37E2,8.7E1,6E0,1.6E1,2.4E1,2.9E1,2.5E1,8E0,8E1,3.7E1,2.4E1,2.7E1,2E0,1.6E1,2E0,3E0,3.1E1,1.26E2,1.32E2,1.5E1,2.4E1,1.6E1,6.1E1,2.1E1,4E0,5E0,4E1,9.7E1,2.8E1,5.9E1,3E0,3E0,1.3E1,3E0,8E0,1.6E1,1.8E1,1.1E1,8E0,1.7E1,3E0,5E0,6.2E1,1.8E1,2.4E1,1.3E1,1.7E1,7E0,1.1E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-6.545979E-3,-1.4479561E-1,3.9999977E-1,-3.2904798E-1,2.9224405E-2,4.4538638E-1,-6.3486946E-1,-5.7315904E-1,-2.606038E-1,-9.112754E-2,1.0915755E-1,1.0542918E0,4.005059E-1,-3.3603955E-2,-1.298357E-2,-5.01763E-1,-7.832702E-1,-4.0268758E-1,-1.8550895E-1,-2.4369973E-1,2.741803E-2,8.397062E-2,4.6457207E-1,5.282198E-2,2.20296E-2,1.7153293E-1,5.169018E-1,-5.9043384E-1,-3.404608E-1,-3.9860528E-2,-3.271384E-3,-4.438932E-1,-2.508133E-1,-6.259852E-2,-3.1728354E-1,-1.7869881E-1,-6.427391E-1,-4.13097E-2,1.5937771E-1,-1.2755185E-1,1.0983885E-1,5.8557494E-3,2.3270037E-2,1.2004886E-1,3.5336692E-2,5.574214E-1,1.6710594E-1,-9.140476E-3,-2.7864475E-2,-1.7576506E-2,-1.550688E-3,-1.6550418E-2,-2.53027E-2,-1.2640657E-2,1.6272815E-4,1.7208666E-2,-4.372118E-3,-1.18254E-2,-2.6543384E-2,-5.683662E-3,-2.0008376E-2,-1.3492045E-2,-3.5288304E-2,-3.2799537E-3,8.643903E-3,4.217607E-3,1.9028962E-2,1.8389025E-3,-1.1488259E-2,6.3088546E-3,-6.2021245E-3,6.881915E-3,-1.0907675E-2,3.6818843E-2,2.1574175E-2,-3.6257226E-3,1.4169314E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5813084E1,2.3768742E1,1.2119003E1,5.865509E0,3.6848285E0,6.191128E0,1.679821E-1,7.770691E-1,2.9448414E0,2.7739553E0,2.019903E0,3.2858658E-1,5.9612236E0,0E0,0E0,6.7145157E-1,1.0219126E0,4.9569988E-1,3.0088782E0,1.6229842E0,7.95612E-1,1.1959E0,1.4803743E-1,0E0,0E0,2.3082662E0,2.0328903E0,1.2051296E-1,2.9775548E-1,0E0,0E0,4.720211E-1,1.4367926E-1,1.4321492E0,1.2544231E0,7.7912784E-1,1.6864467E-1,4.140997E-1,4.7856754E-1,5.0298285E-1,1.3188748E0,0E0,0E0,8.1284535E-1,0E0,2.49366E0,6.1571395E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,4.539777E5,9.7172376E7,8.8E1,2.5367088E2,3.6E1,1.665909E3,1.4504054E8,4.21E2,1.4127266E6,4.9321495E6,1.059448E7,1.1251919E6,-3.3603955E-2,-1.298357E-2,3.9125E1,1E0,2.378914E6,4.519E3,1.8E1,4.5866325E6,4.5650104E2,2.709091E0,5.282198E-2,2.20296E-2,5.428175E3,1E0,2.3E1,5.62E2,-3.9860528E-2,-3.271384E-3,1E0,3.95E2,1.1E1,4.9735293E0,2E0,1.4E1,9.575663E6,2.0833333E0,8.4E1,4.29E2,5.8557494E-3,2.3270037E-2,3.0070068E7,3.5336692E-2,1.4875445E0,5.071E3,-9.140476E-3,-2.7864475E-2,-1.7576506E-2,-1.550688E-3,-1.6550418E-2,-2.53027E-2,-1.2640657E-2,1.6272815E-4,1.7208666E-2,-4.372118E-3,-1.18254E-2,-2.6543384E-2,-5.683662E-3,-2.0008376E-2,-1.3492045E-2,-3.5288304E-2,-3.2799537E-3,8.643903E-3,4.217607E-3,1.9028962E-2,1.8389025E-3,-1.1488259E-2,6.3088546E-3,-6.2021245E-3,6.881915E-3,-1.0907675E-2,3.6818843E-2,2.1574175E-2,-3.6257226E-3,1.4169314E-2],"split_indices":[2,40,57,41,64,0,4,43,2,40,40,12,40,0,0,4,16,9,41,3,40,45,68,0,0,64,6,41,10,0,0,8,2,10,65,6,0,9,65,10,0,0,0,55,0,66,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.91E2,7.4E2,2.51E2,3.59E2,3.81E2,2.41E2,1E1,7.7E1,2.82E2,1.52E2,2.29E2,1.5E1,2.26E2,7E0,3E0,6E1,1.7E1,9.6E1,1.86E2,6.6E1,8.6E1,2.15E2,1.4E1,1.2E1,3E0,7.7E1,1.49E2,3.7E1,2.3E1,1.5E1,2E0,7.4E1,2.2E1,9.7E1,8.9E1,5.8E1,8E0,5.7E1,2.9E1,2.3E1,1.92E2,2E0,1.2E1,7.2E1,5E0,1.33E2,1.6E1,2E0,3.5E1,2E1,3E0,4.4E1,3E1,2E1,2E0,6E0,9.1E1,7.4E1,1.5E1,4.9E1,9E0,3E0,5E0,5.1E1,6E0,2.4E1,5E0,1E1,1.3E1,1.73E2,1.9E1,6.7E1,5E0,3.3E1,1E2,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[4.5220493E-3,-2.0986335E-1,2.1975935E-1,-4.1195747E-1,-1.0263118E-1,-5.210235E-1,2.5836655E-1,-3.8409877E-1,-5.1596843E-2,-3.385639E-1,-6.3919045E-2,-6.492547E-1,-2.6375133E-1,1.376055E-1,5.1328397E-1,-5.452826E-1,-3.18855E-1,-3.4418333E-4,-3.5344225E-1,-8.870398E-3,-1.8991572E-1,-3.2566965E-2,-7.0676133E-3,-1.5289243E-2,3.8962418E-4,1.8327343E-1,-1.8097079E-1,6.247532E-1,2.509645E-1,-2.5656637E-2,-7.532662E-3,1.18096165E-1,-3.5892293E-1,-3.6534658E-1,-3.1677731E-3,1.8779049E-2,-1.6433327E-1,-4.88198E-1,-1.1476264E-1,3.4331557E-1,1.290656E-1,-1.0581684E-1,-4.364274E-1,5.623635E-1,1.0666087E0,3.8561526E-1,-2.247667E-1,1.6891453E-2,-3.5413618E-3,-1.487408E-2,-3.143889E-2,-1.0008714E-2,-1.832461E-2,4.104437E-3,-1.2030939E-3,6.1671897E-3,-1.281578E-2,-9.507369E-3,-3.951738E-2,-3.2986295E-3,-1.4530555E-2,1.0233331E-2,2.390477E-2,2.6717314E-3,1.0613714E-2,-9.727037E-3,9.1848336E-4,-2.234503E-2,-7.998955E-3,1.8499162E-2,3.4640875E-2,1.8450145E-2,5.3254142E-2,-1.5254456E-2,2.003154E-2,6.2781414E-3,-1.4710792E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,-1,33,35,37,-1,-1,-1,-1,39,41,43,45,-1,-1,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6051422E1,1.0793217E1,1.4377192E1,2.9970837E0,2.974134E0,6.535163E-1,1.452103E1,1.5928631E0,0E0,2.2393703E-1,1.9611881E0,3.6470556E-1,1.9098872E-1,4.732403E0,4.2877693E0,1.467638E-1,2.1783295E0,0E0,1.2380886E-1,8.541816E-1,1.8718207E0,0E0,0E0,0E0,0E0,2.4108667E0,7.409601E-1,2.3223038E0,3.0973797E0,0E0,0E0,5.6440854E-1,1.0578365E0,1.2118578E-1,0E0,5.3877664E-1,1.0648277E0,1.6010199E0,5.8331007E-1,1.3997211E0,1.527056E0,4.441466E-1,3.344083E-2,2.6047993E0,2.748518E-1,1.4859676E0,4.235204E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,-1,34,36,38,-1,-1,-1,-1,40,42,44,46,-1,-1,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.7302156E5,1E0,6.958929E1,3.56E2,9.3326636E-2,1.0236667E3,3.89E2,-5.1596843E-2,3.9333332E1,4.046875E0,2.3751075E5,5.9395876E-7,7.627907E0,1.9493858E7,1.4473684E-1,8.650138E-1,-3.4418333E-4,8.328548E6,8.931E3,1E0,-3.2566965E-2,-7.0676133E-3,-1.5289243E-2,3.8962418E-4,7E0,4E1,4.0277927E3,3.85374E8,-2.5656637E-2,-7.532662E-3,3.806E3,1.2951E4,2.1E2,-3.1677731E-3,1.3364486E0,5.132743E-1,7.1E1,2.9E1,5.235714E2,1.5115256E6,2.008E3,1.4504672E3,3.2911258E6,1E1,2.984E3,1.7291568E2,1.6891453E-2,-3.5413618E-3,-1.487408E-2,-3.143889E-2,-1.0008714E-2,-1.832461E-2,4.104437E-3,-1.2030939E-3,6.1671897E-3,-1.281578E-2,-9.507369E-3,-3.951738E-2,-3.2986295E-3,-1.4530555E-2,1.0233331E-2,2.390477E-2,2.6717314E-3,1.0613714E-2,-9.727037E-3,9.1848336E-4,-2.234503E-2,-7.998955E-3,1.8499162E-2,3.4640875E-2,1.8450145E-2,5.3254142E-2,-1.5254456E-2,2.003154E-2,6.2781414E-3,-1.4710792E-2],"split_indices":[2,40,17,58,2,50,64,1,0,59,66,45,49,66,63,68,65,0,9,41,27,0,0,0,0,3,3,4,44,0,0,1,41,41,0,65,66,0,8,4,40,2,4,40,3,2,70,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.96E2,4.99E2,4.97E2,1.72E2,3.27E2,2.4E1,4.73E2,1.67E2,5E0,4.5E1,2.82E2,1.5E1,9E0,3.22E2,1.51E2,4.6E1,1.21E2,2E0,4.3E1,1.97E2,8.5E1,1.3E1,2E0,7E0,2E0,2.82E2,4E1,1.05E2,4.6E1,4.4E1,2E0,1E1,1.11E2,4.1E1,2E0,1.68E2,2.9E1,1.6E1,6.9E1,7E1,2.12E2,3.2E1,8E0,9.4E1,1.1E1,3.6E1,1E1,4E0,6E0,1.02E2,9E0,9E0,3.2E1,6.5E1,1.03E2,8E0,2.1E1,1E1,6E0,5.8E1,1.1E1,4.3E1,2.7E1,1.26E2,8.6E1,1.7E1,1.5E1,6E0,2E0,5.3E1,4.1E1,2E0,9E0,2E0,3.4E1,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-2.626108E-3,-2.3917316E-1,1.8024117E-1,-4.873125E-1,-1.6673414E-1,4.1855216E-2,4.5242268E-1,-5.081851E-1,2.5545E-3,-2.062967E-1,5.104403E-1,-1.3090314E-1,1.07756004E-1,9.236719E-1,4.0675178E-1,-5.2900153E-1,-3.1852204E-2,1.0897661E-2,-1.0702107E-2,-3.127548E-1,-8.8820994E-2,7.412431E-1,8.886402E-3,-2.4707069E-1,9.772975E-3,1.6898045E-2,2.2270192E-1,1.3076616E-2,9.9461836E-1,2.5344738E-1,5.3919816E-1,-4.6755737E-1,-6.622657E-1,-4.736713E-3,2.2983446E-3,-3.758228E-1,-1.519198E-1,-3.424847E-2,-6.633483E-2,8.515366E-1,4.122485E-3,-1.0275385E-2,1.4666747E-1,-1.8401776E-1,-5.629065E-1,5.83606E-2,-1.8069606E-1,-9.5461905E-3,2.0258868E-2,1.7223462E-1,4.0758827E-1,1.780006E-2,4.8848405E-2,3.1810945E-1,-1.6305923E-2,5.7969224E-1,9.684291E-2,-2.3019584E-2,-5.3943815E-3,-3.5445306E-2,-1.8832091E-2,-1.4171292E-2,-2.3089504E-2,1.8779552E-3,-1.02280285E-2,8.285077E-3,-4.7536697E-3,4.299269E-2,1.4434032E-2,2.3924232E-4,1.0988531E-2,-4.88566E-4,-1.1951229E-2,-3.4191262E-2,2.2141112E-3,-1.05729345E-2,3.6827545E-3,-1.4167982E-2,-3.7606526E-3,-3.0311844E-3,5.782178E-3,1.5729953E-2,5.373884E-3,2.1733554E-2,-1.0614254E-2,8.2571234E-4,1.673616E-2,4.010003E-3,-1.0142049E-2,3.176836E-2,1.943007E-2,-2.9054563E-3,1.0912117E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,-1,-1,61,63,-1,65,67,-1,-1,69,71,73,75,77,79,-1,81,83,-1,-1,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.325657E1,7.7558365E0,2.1224745E1,1.0033283E0,9.187366E0,4.2834473E0,3.7135468E0,9.152107E-1,3.317112E-1,3.9867249E0,2.192854E0,1.7019824E0,2.836937E0,4.4215965E-1,3.4166508E0,4.273014E-1,3.4343332E-2,0E0,0E0,1.6470184E0,2.2250214E0,8.576889E-1,2.571436E-1,1.0315344E0,4.5475855E-1,1.7203826E0,1.0494847E0,0E0,1.733303E-1,1.45294E0,1.6097507E0,7.061119E-1,4.0135098E-1,0E0,0E0,8.693943E-1,6.776087E-1,0E0,1.3938438E0,2.0514488E-1,0E0,0E0,6.422783E-2,6.444212E-1,1.0340757E0,2.6399046E-1,1.01001024E-1,1.1168085E0,0E0,8.626218E-1,1.1565266E0,0E0,0E0,9.2373276E-1,3.7948045E-1,1.1982975E0,2.1780556E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,35,35,36,36,38,38,39,39,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,-1,-1,62,64,-1,66,68,-1,-1,70,72,74,76,78,80,-1,82,84,-1,-1,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9236734E5,7.7E1,3.116E3,1.5900173E3,9.616803E2,1E0,7.9E1,1.1408248E0,1.81E2,7.06E2,2.0750147E8,6.325E4,6.946813E7,2.0170736E7,2.36433E6,4.217427E7,6.5E1,1.0897661E-2,-1.0702107E-2,1.585814E6,1.6716E5,1E0,1E1,1.318849E6,2.1851852E0,6.663214E6,5.4275E2,1.3076616E-2,1.0838E4,2.3876712E7,1.0424884E-7,8.7151864E7,1.5416006E5,-4.736713E-3,2.2983446E-3,1.046E3,1.4392524E0,-3.424847E-2,1.4979E4,7.122041E7,4.122485E-3,-1.0275385E-2,1.027E3,2.4E0,6.99E2,7.96E2,1.2218E4,1.1394967E7,2.0258868E-2,4.5683857E-2,2.8366232E0,1.780006E-2,4.8848405E-2,6.726722E2,2.2365898E3,1.8791208E0,3.72381E5,-2.3019584E-2,-5.3943815E-3,-3.5445306E-2,-1.8832091E-2,-1.4171292E-2,-2.3089504E-2,1.8779552E-3,-1.02280285E-2,8.285077E-3,-4.7536697E-3,4.299269E-2,1.4434032E-2,2.3924232E-4,1.0988531E-2,-4.88566E-4,-1.1951229E-2,-3.4191262E-2,2.2141112E-3,-1.05729345E-2,3.6827545E-3,-1.4167982E-2,-3.7606526E-3,-3.0311844E-3,5.782178E-3,1.5729953E-2,5.373884E-3,2.1733554E-2,-1.0614254E-2,8.2571234E-4,1.673616E-2,4.010003E-3,-1.0142049E-2,3.176836E-2,1.943007E-2,-2.9054563E-3,1.0912117E-2],"split_indices":[40,41,2,67,64,27,0,50,0,2,7,1,7,5,40,57,0,0,0,9,7,6,3,9,65,40,64,0,9,57,49,7,45,0,0,41,65,0,1,7,0,0,0,66,2,12,41,57,0,50,50,0,0,4,64,66,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,4.35E2,5.63E2,9.7E1,3.38E2,3.74E2,1.89E2,9.3E1,4E0,3.2E2,1.8E1,1.03E2,2.71E2,1.5E1,1.74E2,8.9E1,4E0,2E0,2E0,1.67E2,1.53E2,1.2E1,6E0,5.6E1,4.7E1,1.52E2,1.19E2,2E0,1.3E1,8.2E1,9.2E1,6.4E1,2.5E1,2E0,2E0,1.19E2,4.8E1,4E0,1.49E2,1E1,2E0,2E0,4E0,4.8E1,8E0,3.8E1,9E0,1.44E2,8E0,9.5E1,2.4E1,2E0,1.1E1,6.6E1,1.6E1,8.4E1,8E0,5.8E1,6E0,1.6E1,9E0,8E1,3.9E1,1.3E1,3.5E1,1.9E1,1.3E2,8E0,2E0,2E0,2E0,1.5E1,3.3E1,6E0,2E0,2E0,3.6E1,3E0,6E0,1.02E2,4.2E1,2.2E1,7.3E1,2.2E1,2E0,9E0,5.7E1,1.1E1,5E0,4.7E1,3.7E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[9.666441E-3,-1.0173676E-1,3.7711823E-1,-2.5279585E-1,4.2166814E-2,2.3187083E-1,6.13989E-1,-4.7719914E-1,-1.8826951E-1,-1.11847684E-1,1.3894819E-1,-4.0115923E-1,2.8447434E-1,8.292909E-1,3.4305835E-1,-4.075766E-1,-7.358097E-1,-4.1631854E-1,-1.4155501E-1,-8.086615E-2,-4.2473873E-1,4.2795558E-3,1.7326729E-1,3.0272309E-4,-2.2201855E-2,1.1600377E-1,3.827069E-1,6.569042E-1,5.4554414E-2,4.3385723E-1,4.7728986E-2,-5.142968E-1,-2.9849374E-1,-5.6468594E-1,-4.7473446E-2,-4.6460778E-1,-2.2066121E-1,5.4752034E-1,-1.7813769E-1,-1.3057755E-1,4.8415873E-2,-2.2514405E-2,-6.366564E-3,9.54133E-2,-9.052393E-2,1.3157745E-1,3.3673722E-1,2.8378755E-2,3.1686294E-1,5.5676055E-1,2.5726092E-1,7.2374845E-1,1.5271368E-2,9.020802E-2,4.7972828E-1,1.9707498E-1,-1.2075186E-2,-7.939403E-3,-2.4354532E-2,-7.785298E-3,-2.2888085E-2,-1.3246853E-2,-2.8419226E-2,-2.7399931E-2,-1.8096793E-2,-1.1667742E-2,-4.153511E-3,1.14339E-2,2.9973384E-2,-1.9505523E-2,-5.831688E-3,-3.969457E-3,-2.00455E-2,5.036511E-4,7.505962E-3,7.824405E-3,7.3915224E-5,-5.527673E-3,8.186216E-3,2.4491465E-3,9.563354E-3,1.2213998E-2,2.4004053E-2,-3.7765803E-3,5.5957474E-3,6.6064093E-3,1.7565966E-2,3.9992165E-3,2.8144605E-2,5.008462E-4,1.38169145E-2,1.3529216E-2,3.4997184E-2,5.6693074E-4,6.338726E-3,2.3012945E-2,6.211317E-3,2.9001841E-3,1.8338189E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,51,-1,53,55,57,59,61,-1,63,65,67,69,71,73,-1,-1,75,77,79,81,83,85,87,89,91,-1,93,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2406406E1,1.7292324E1,8.149208E0,5.528366E0,6.09708E0,5.142359E0,5.0237083E0,1.2485409E0,3.1689863E0,1.4972807E0,1.1593075E0,4.0543473E-1,2.2725058E0,2.309307E0,1.1055474E0,6.6138077E-1,2.825718E-1,3.7933445E-1,6.475214E0,9.3820655E-1,1.9417381E-1,4.579066E-1,1.3186994E0,0E0,0E0,9.23823E-1,1.7781887E0,5.400562E-1,0E0,4.611578E-1,5.621054E-1,9.337902E-2,8.6922383E-1,1.5663624E-2,0E0,1.3760662E-1,2.9169142E-2,2.4530387E-1,2.986424E0,1.3803041E0,1.7478839E-1,0E0,0E0,1.8951687E-1,2.2983006E-1,9.5439076E-1,3.8755512E-1,4.0276775E-1,1.2653768E-1,8.933859E-1,5.654061E-1,2.5095558E-1,0E0,1.6904712E-2,1.3769293E-1,1.9141528E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,52,-1,54,56,58,60,62,-1,64,66,68,70,72,74,-1,-1,76,78,80,82,84,86,88,90,92,-1,94,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,4.9236734E5,9.154E3,3.3817584E7,7.69E2,1E0,9.133387E1,8.773336E6,2.71E2,6.4153387E-6,3.927456E7,1.91E2,7.445455E2,4.907764E3,4.24038E8,4.41875E1,2.1785715E0,1.7909248E5,1.2E1,2.9652428E6,4.48E2,2.75E0,2.8530578E6,3.0272309E-4,-2.2201855E-2,5.53648E8,1.1057851E1,3.925926E0,5.4554414E-2,2.9697892E-1,4.5698816E7,4E0,3E0,3.7253174E2,-4.7473446E-2,3.89E2,2E0,2.638E3,2.661509E4,9.540121E9,2.680201E2,-2.2514405E-2,-6.366564E-3,1.103E3,6.6933334E1,4.5432812E2,2.0342858E0,1.0039E4,9.1233586E4,1.342376E0,9.49372E4,1.4523518E8,1.5271368E-2,4.6750406E4,1.4164478E1,6.7947706E8,-1.2075186E-2,-7.939403E-3,-2.4354532E-2,-7.785298E-3,-2.2888085E-2,-1.3246853E-2,-2.8419226E-2,-2.7399931E-2,-1.8096793E-2,-1.1667742E-2,-4.153511E-3,1.14339E-2,2.9973384E-2,-1.9505523E-2,-5.831688E-3,-3.969457E-3,-2.00455E-2,5.036511E-4,7.505962E-3,7.824405E-3,7.3915224E-5,-5.527673E-3,8.186216E-3,2.4491465E-3,9.563354E-3,1.2213998E-2,2.4004053E-2,-3.7765803E-3,5.5957474E-3,6.6064093E-3,1.7565966E-2,3.9992165E-3,2.8144605E-2,5.008462E-4,1.38169145E-2,1.3529216E-2,3.4997184E-2,5.6693074E-4,6.338726E-3,2.3012945E-2,6.211317E-3,2.9001841E-3,1.8338189E-2],"split_indices":[2,40,2,7,2,17,70,57,2,49,7,0,64,4,44,4,65,40,30,40,2,66,40,0,0,7,68,66,0,50,59,0,8,4,0,1,29,1,40,43,64,0,0,10,68,4,65,41,45,66,45,12,0,45,47,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.034E3,7.94E2,2.4E2,3.87E2,4.07E2,1.5E2,9E1,8.5E1,3.02E2,1.57E2,2.5E2,1.1E1,1.39E2,4.9E1,4.1E1,6.9E1,1.6E1,5E1,2.52E2,1.44E2,1.3E1,5.1E1,1.99E2,2E0,9E0,5.2E1,8.7E1,3.5E1,1.4E1,3.1E1,1E1,3.3E1,3.6E1,1.2E1,4E0,3.9E1,1.1E1,1.2E1,2.4E2,1.04E2,4E1,1E1,3E0,2.6E1,2.5E1,1.6E2,3.9E1,3.7E1,1.5E1,3.5E1,5.2E1,2.8E1,7E0,4E0,2.7E1,7E0,3E0,2E0,3.1E1,2.3E1,1.3E1,3E0,9E0,1.1E1,2.8E1,8E0,3E0,4E0,8E0,4E1,2E2,9.2E1,1.2E1,3.1E1,9E0,1.4E1,1.2E1,2.3E1,2E0,8E1,8E1,3E1,9E0,1.7E1,2E1,5E0,1E1,4E0,3.1E1,8E0,4.4E1,3E0,2.5E1,2E0,2E0,2.5E1,2E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[4.4383467E-3,-1.9609079E-1,1.8935812E-1,-3.7009954E-1,-8.8846706E-2,1.0580017E-1,5.36479E-1,-3.2876745E-1,-7.592186E-1,-6.0186192E-2,-3.907742E-1,1.5007943E-1,-4.8368868E-1,5.4614905E-2,4.5262292E-1,-4.370826E-1,-2.2165339E-1,-2.7871957E-1,-5.810065E-2,-5.0174452E-2,-3.357338E-2,-5.700894E-1,-2.4307163E-1,7.510726E-2,3.3784035E-1,-3.3444345E-2,-3.617023E-1,6.7494863E-1,2.7628988E-1,-4.5020136E-1,-6.5401746E-3,1.748728E-2,-2.709743E-1,-1.5556456E-3,-1.7399661E-2,-1.6474439E-1,-6.9598504E-3,-3.6657885E-1,-3.4034643E-2,-2.893759E-1,-1.4613037E-3,3.1314895E-2,5.285449E-2,3.055123E-1,3.965908E-2,-3.952625E-1,-7.992051E-4,7.011605E-1,5.4175695E-3,3.6759633E-1,2.0473743E-2,-2.1339957E-2,-1.0825776E-2,-9.891295E-3,-2.5049878E-2,-2.74346E-3,-1.0223031E-2,1.7690728E-3,-6.792539E-3,-1.9521322E-2,-6.752043E-3,-1.4784738E-2,-3.3857196E-3,5.8993986E-3,-2.0682986E-3,1.5355854E-2,-2.2287795E-3,-2.3633195E-2,-1.2533469E-2,3.3096958E-2,1.0771377E-2,7.613579E-3,1.9411689E-2,6.701213E-3,-1.1921496E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,-1,37,39,41,43,-1,45,47,49,51,-1,-1,53,-1,-1,55,57,59,-1,61,-1,-1,63,65,-1,67,-1,69,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.756391E1,9.036562E0,1.5228725E1,2.7257671E0,2.5898063E0,1.1192099E1,5.043455E0,1.8638725E0,3.7784128E0,1.8235052E0,5.500455E-1,5.5631323E0,6.312041E-1,0E0,3.420227E0,2.56773E-1,2.687923E0,2.3225981E-1,0E0,1.3554533E0,0E0,1.1094284E-1,1.473071E-1,3.8114653E0,1.6562586E0,0E0,2.4732971E-1,5.013008E-1,1.2304301E0,1.3455582E-1,0E0,0E0,1.1189122E0,0E0,0E0,4.421563E-1,1.2874081E0,2.7331948E-2,0E0,6.7420244E-2,0E0,0E0,2.047782E0,1.1177282E0,0E0,1.515429E-1,0E0,1.7522812E-1,0E0,3.6309004E-1,5.646823E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,32,32,35,35,36,36,37,37,39,39,42,42,43,43,45,45,47,47,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,-1,38,40,42,44,-1,46,48,50,52,-1,-1,54,-1,-1,56,58,60,-1,62,-1,-1,64,66,-1,68,-1,70,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.037E3,1.972052E5,8.439E3,4.6463413E0,6.4153387E-6,3.9661028E7,1.059448E7,2.71E2,3.89E2,5.6E1,3.9595376E7,2.36433E6,1.5889913E-2,5.4614905E-2,4.420569E1,4.629112E6,8E0,6.48334E5,-5.810065E-2,4.97E2,-3.357338E-2,5.5E1,1.3E1,7.336111E4,1.9851073E3,-3.3444345E-2,1E0,1E0,2.3876712E7,2.1E1,-6.5401746E-3,1.748728E-2,1.2951E4,-1.5556456E-3,-1.7399661E-2,1.3165267E0,2E0,5E0,-3.4034643E-2,8.743577E4,-1.4613037E-3,3.1314895E-2,4.6603775E0,1.9269184E7,3.965908E-2,7.295292E-2,-7.992051E-4,9.901493E9,5.4175695E-3,2.234478E3,3.0070068E7,-2.1339957E-2,-1.0825776E-2,-9.891295E-3,-2.5049878E-2,-2.74346E-3,-1.0223031E-2,1.7690728E-3,-6.792539E-3,-1.9521322E-2,-6.752043E-3,-1.4784738E-2,-3.3857196E-3,5.8993986E-3,-2.0682986E-3,1.5355854E-2,-2.2287795E-3,-2.3633195E-2,-1.2533469E-2,3.3096958E-2,1.0771377E-2,7.613579E-3,1.9411689E-2,6.701213E-3,-1.1921496E-2],"split_indices":[2,40,2,65,49,57,12,2,2,3,7,40,50,0,68,9,10,43,0,2,0,0,8,57,64,0,16,6,57,3,0,0,41,0,0,65,29,8,0,45,0,0,66,59,0,50,0,44,0,4,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.011E3,4.85E2,5.26E2,1.84E2,3.01E2,4.25E2,1.01E2,1.68E2,1.6E1,2.76E2,2.5E1,3.96E2,2.9E1,1E1,9.1E1,8.2E1,8.6E1,9E0,7E0,2.73E2,3E0,1E1,1.5E1,2.84E2,1.12E2,8E0,2.1E1,3.9E1,5.2E1,7.8E1,4E0,6E0,8E1,3E0,6E0,7.4E1,1.99E2,6E0,4E0,1.2E1,3E0,9E0,2.75E2,1.07E2,5E0,1.9E1,2E0,3.7E1,2E0,3.8E1,1.4E1,7.2E1,6E0,6.8E1,1.2E1,2.7E1,4.7E1,1.51E2,4.8E1,4E0,2E0,1E1,2E0,1.55E2,1.2E2,9.9E1,8E0,8E0,1.1E1,3.5E1,2E0,9E0,2.9E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.3272476E-2,-1.22585826E-1,3.2122698E-1,-3.514675E-1,-2.1580866E-2,5.7822596E-2,2.8929842E-1,3.659989E-1,-3.7499702E-1,-1.360552E-1,6.622868E-2,3.5211486E-1,-9.7660676E-2,2.6337767E-2,6.99582E-4,-4.0413702E-1,-8.2446456E-2,-1.7975473E-1,7.92889E-2,1.4231277E-1,-2.3914913E-2,1.9359675E-1,5.133441E-1,-3.673963E-1,5.90447E-2,-3.895855E-1,-8.2910705E-1,5.4623093E-2,-2.2448175E-1,-1.5836307E-1,-5.5752504E-1,1.250063E-1,-1.6692413E-2,1.1631887E-1,4.6404174E-1,-5.1958496E-3,-4.4110802E-1,2.8540848E-2,1.5316048E-1,3.5063824E-1,6.2681776E-1,-4.1437772E-1,-7.261487E-3,-4.7145367E-2,3.084999E-1,-1.6710222E-2,-2.7892739E-2,-5.2505363E-2,-6.1571896E-3,-1.5789294E-3,4.7502154E-3,-5.351207E-3,-1.5192056E-2,-1.2095025E-2,-4.3684407E-3,-5.0243717E-3,-3.8493004E-2,1.1339725E-2,2.2369984E-3,5.8238166E-3,-1.4154897E-2,6.2667686E-3,2.3696953E-2,-7.629545E-3,1.1897525E-3,-2.7599648E-2,-9.691425E-3,9.119562E-4,1.1101659E-2,1.7525364E-2,1.2402339E-3,3.1840906E-2,1.990464E-2,-2.2434717E-2,-1.1144358E-2,6.59124E-3,-7.5410446E-3,4.3510254E-3,1.7227214E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,59,61,63,65,-1,67,69,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.740907E1,1.7816534E1,7.180214E0,4.121317E0,5.3968506E0,0E0,5.9896984E0,5.7330143E-1,1.9188881E0,2.206884E0,2.088385E0,5.2933044E0,1.5011085E0,0E0,0E0,9.63253E-1,4.405736E-1,1.4794493E0,8.550086E-1,1.3259168E0,1.0910331E0,1.7729998E0,1.7003498E0,6.2322855E-2,6.238067E-1,9.1820335E-1,1.1563096E0,5.5668924E-2,8.7691545E-2,1.2032728E0,1.1654694E0,3.3206213E-1,0E0,7.2381306E-1,1.3355207E-1,6.7985654E-1,9.381533E-2,0E0,1.1754856E0,4.3994284E-1,4.7043037E-1,8.687615E-3,0E0,4.0286893E-1,6.407553E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,60,62,64,66,-1,68,70,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.7302156E5,3.3817584E7,6E0,8.1E2,5.7822596E-2,2E0,1.89991E5,7.997723E6,3.3486558E6,3.875E0,1.7587205E6,1.332E3,2.6337767E-2,6.99582E-4,6.9934395E-5,2.590909E0,2.4240502E8,7.77E2,2.1111112E0,1.2294118E1,3.5E1,5.97E3,1.7026364E3,5.854E3,4.217427E7,1.75E2,4.711E3,1.3422592E8,4.48E2,1.5881818E2,2.1111E4,-1.6692413E-2,3.068E3,1.017E3,2.2084616E2,6.185E3,2.8540848E-2,3.3580637E8,4.1E1,4.7E1,7.295292E-2,-7.261487E-3,1.5209424E1,1.41565E3,-1.6710222E-2,-2.7892739E-2,-5.2505363E-2,-6.1571896E-3,-1.5789294E-3,4.7502154E-3,-5.351207E-3,-1.5192056E-2,-1.2095025E-2,-4.3684407E-3,-5.0243717E-3,-3.8493004E-2,1.1339725E-2,2.2369984E-3,5.8238166E-3,-1.4154897E-2,6.2667686E-3,2.3696953E-2,-7.629545E-3,1.1897525E-3,-2.7599648E-2,-9.691425E-3,9.119562E-4,1.1101659E-2,1.7525364E-2,1.2402339E-3,3.1840906E-2,1.990464E-2,-2.2434717E-2,-1.1144358E-2,6.59124E-3,-7.5410446E-3,4.3510254E-3,1.7227214E-2],"split_indices":[2,40,7,3,2,0,6,9,9,40,66,40,41,0,0,49,65,5,2,65,65,0,2,4,2,57,6,41,7,2,45,9,0,2,2,64,41,0,7,8,8,50,0,68,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.021E3,7.7E2,2.51E2,2.35E2,5.35E2,7E0,2.44E2,7E0,2.28E2,2.32E2,3.03E2,2.1E2,3.4E1,4E0,3E0,2.07E2,2.1E1,1.93E2,3.9E1,1.64E2,1.39E2,1.07E2,1.03E2,1.2E1,2.2E1,2.02E2,5E0,1.1E1,1E1,1.84E2,9E0,3.6E1,3E0,1.53E2,1.1E1,1.34E2,5E0,8E0,9.9E1,4.4E1,5.9E1,9E0,3E0,1.6E1,6E0,1.83E2,1.9E1,3E0,2E0,4E0,7E0,6E0,4E0,6.8E1,1.16E2,4E0,5E0,1.3E1,2.3E1,1.5E2,3E0,2E0,9E0,2.1E1,1.13E2,2E0,3E0,4E1,5.9E1,4E1,4E0,4.2E1,1.7E1,5E0,4E0,6E0,1E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.8155187E-3,-1.7835276E-1,1.5806039E-1,-4.090034E-1,-1.05686635E-1,4.4865157E-2,3.7519175E-1,-4.2423043E-1,-5.4060146E-2,-2.6181132E-1,-1.3671893E-3,-1.1849511E-2,1.7675953E-1,2.8481624E-1,5.588737E-1,-4.6025628E-1,-2.7145723E-1,8.626645E-3,-1.0194584E-2,-2.3928829E-1,-6.909585E-1,-1.7860165E-2,4.2375587E-2,-6.120887E-2,1.0280136E-1,-2.8234478E-2,2.5858226E-1,2.1673715E-1,7.1378416E-1,6.0676914E-1,5.0634068E-2,-4.9197224E-1,-2.9512474E-1,-1.595387E-2,-7.627386E-2,-1.4649399E-1,-3.0306667E-1,-9.55364E-3,-8.425435E-1,4.588703E-1,-4.700939E-2,-2.3553091E-1,-1.150563E-2,7.365525E-2,1.888396E-2,1.7817618E-2,-7.181374E-2,2.9357514E-1,-2.2875236E-2,2.5497335E-1,-1.00841306E-1,4.063811E-2,1.6883183E-2,6.306125E-1,3.2721483E-4,9.127868E-3,-7.5190887E-3,-1.16765695E-2,-2.3743825E-2,-1.8235248E-2,-1.936857E-3,-9.0847E-3,1.6623574E-3,-2.24718E-3,-1.0719844E-2,-1.2728337E-2,-2.7493399E-2,-4.7625747E-2,-1.6873132E-2,2.5624694E-2,8.674769E-3,-1.584804E-2,-5.711859E-4,-7.3350947E-3,-2.469845E-2,3.3666677E-4,-2.2397287E-2,-2.8438289E-3,4.2756293E-3,-5.826862E-3,5.1189316E-3,5.4500033E-3,1.5785666E-2,-1.2120445E-2,1.0229194E-2,-7.5370073E-3,1.2428706E-2,1.6554528E-4,-1.4267868E-2,2.2652037E-2,3.2418586E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,-1,67,69,71,73,75,77,-1,-1,79,81,83,85,87,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8508524E1,8.236454E0,1.2616381E1,6.148262E-1,6.1397815E0,2.5379317E0,2.7624855E0,4.969864E-1,2.8533885E-1,1.2941828E0,3.4756534E0,1.3531566E0,1.7181644E0,3.3497448E0,1.3567905E0,3.2616615E-1,3.2897305E-1,0E0,0E0,8.137512E-1,3.372209E-1,3.1343474E0,0E0,1.4442397E0,6.167573E-1,5.7192457E-1,7.2569275E-1,1.3007264E0,6.218548E-1,7.3897934E-1,2.2298291E-1,3.078518E-1,4.2138827E-1,0E0,1.1648775E-1,5.057483E-1,4.8597383E-1,0E0,8.084226E-2,2.421999E-1,2.1801393E0,7.620437E-1,1.1783631E0,1.8226528E-1,0E0,0E0,2.9659545E-1,5.288825E-1,5.914434E-1,6.596508E-1,2.6406148E-1,0E0,0E0,1.6863441E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,46,46,47,47,48,48,49,49,50,50,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,-1,68,70,72,74,76,78,-1,-1,80,82,84,86,88,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.176636E5,1.16E2,3.116E3,1.7026364E3,6.9E2,4.265829E2,3.8181802E6,7.6588124E-2,1.91E2,9.639872E0,1.0534078E4,2.9652428E6,1E0,4.907764E3,4.9036694E0,2.135961E4,2.0081382E-7,8.626645E-3,-1.0194584E-2,2E0,3.89E2,4.427818E7,4.2375587E-2,1.9875325E2,2.7777777E0,7.8571427E-1,1.2434699E8,1.6677892E2,3.2295492E0,1E0,1.6327E4,7E0,3.07E2,-1.595387E-2,6.5E1,1E0,1.0039E4,-9.55364E-3,6.8E1,2.04115E5,3.3832976E7,2.0045958E6,4.5158855E6,5.8899284E1,1.888396E-2,1.7817618E-2,2.1931148E6,5.1604336E7,2.847E4,1E0,1E0,4.063811E-2,1.6883183E-2,1.2841364E3,3.2721483E-4,9.127868E-3,-7.5190887E-3,-1.16765695E-2,-2.3743825E-2,-1.8235248E-2,-1.936857E-3,-9.0847E-3,1.6623574E-3,-2.24718E-3,-1.0719844E-2,-1.2728337E-2,-2.7493399E-2,-4.7625747E-2,-1.6873132E-2,2.5624694E-2,8.674769E-3,-1.584804E-2,-5.711859E-4,-7.3350947E-3,-2.469845E-2,3.3666677E-4,-2.2397287E-2,-2.8438289E-3,4.2756293E-3,-5.826862E-3,5.1189316E-3,5.4500033E-3,1.5785666E-2,-1.2120445E-2,1.0229194E-2,-7.5370073E-3,1.2428706E-2,1.6554528E-4,-1.4267868E-2,2.2652037E-2,3.2418586E-2],"split_indices":[40,41,2,4,2,64,40,54,0,66,4,40,27,4,50,40,49,0,0,29,2,43,0,67,65,65,12,70,50,6,2,0,1,0,0,24,41,0,0,5,7,40,59,59,0,0,40,7,10,24,26,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.006E3,4.93E2,5.13E2,1.17E2,3.76E2,3.38E2,1.75E2,1.12E2,5E0,1.5E2,2.26E2,2.37E2,1.01E2,1.19E2,5.6E1,8.9E1,2.3E1,2E0,3E0,1.44E2,6E0,2.23E2,3E0,1.66E2,7.1E1,2.9E1,7.2E1,1.04E2,1.5E1,5.1E1,5E0,7.3E1,1.6E1,1.6E1,7E0,6E1,8.4E1,2E0,4E0,1.2E1,2.11E2,3.6E1,1.3E2,6.6E1,5E0,2E0,2.7E1,6.4E1,8E0,9.3E1,1.1E1,9E0,6E0,4.9E1,2E0,3E0,2E0,8E0,6.5E1,1.1E1,5E0,3E0,4E0,2.9E1,3.1E1,7.9E1,5E0,2E0,2E0,8E0,4E0,2.1E1,1.9E2,3E1,6E0,1.26E2,4E0,8E0,5.8E1,2.1E1,6E0,1.5E1,4.9E1,4E0,4E0,3E0,9E1,8E0,3E0,2E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-1.1357255E-3,-9.095336E-2,2.9749355E-1,-1.8876508E-1,6.2176656E-2,3.6789942E-1,-1.4249068E-1,-1.4602427E-1,-4.8764902E-1,-2.0062072E-2,1.3157815E-1,9.754213E-1,3.2780433E-1,8.269699E-2,-3.8692114E-1,-2.609569E-1,-3.0392943E-2,-3.5271883E-1,-9.867324E-1,-1.20388865E-1,2.7175093E-2,1.04150474E-1,3.804889E-1,1.635172E-2,4.8857614E-2,3.5249084E-1,-1.5427063E-1,-6.8222485E-2,1.6384193E-1,-4.5616687E-3,-4.4565234E-1,-3.6857063E-1,-1.8119493E-1,5.4726005E-1,-6.524871E-2,-4.731442E-1,-2.1257912E-1,-4.933257E-2,-1.6846698E-2,8.122315E-2,-1.8800521E-1,1.0912661E-2,3.209688E-1,7.548398E-2,2.867085E-1,1.0503391E-1,2.129177E-2,2.8076786E-1,5.414376E-1,-2.7971262E-1,1.0640727E-2,1.5529803E-3,-5.3187786E-3,1.3754076E-2,5.8608353E-2,-1.0876527E-2,-2.3524543E-2,-4.234955E-3,-1.76427E-2,-4.338301E-3,-1.2503415E-2,2.726205E-2,9.672502E-3,-5.844309E-4,-1.2817743E-2,-2.2440325E-2,-9.059557E-3,-1.2320502E-2,-3.111198E-3,-1.141936E-3,5.4818005E-3,-1.2171988E-2,-1.3407801E-4,1.8134656E-3,-1.04404E-2,1.9569477E-2,5.0015133E-3,1.5789894E-2,2.644116E-3,2.5675306E-3,2.0022629E-2,1.3517587E-3,6.688858E-3,7.389203E-3,1.8985E-2,3.3895347E-2,1.9014627E-2,-1.6886711E-2,-7.0468457E-3,6.32402E-3,-5.469448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,61,63,65,67,-1,-1,69,71,73,75,77,79,81,-1,83,85,87,-1,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7144276E1,1.1664552E1,7.3011513E0,5.9966984E0,1.7377166E0,4.6007023E0,1.8484073E0,5.5339212E0,3.625104E0,6.6675305E-1,1.086834E0,2.7753353E-1,2.327423E0,2.3161206E-1,2.2600412E-1,1.7271652E0,4.2438264E0,7.0948505E-1,2.5562382E-1,6.2872225E-1,4.551662E-1,7.633532E-1,3.174038E-1,0E0,0E0,2.314415E0,5.489248E-1,3.789516E-2,1.5369272E-1,0E0,5.9309006E-2,3.4247208E-1,9.375639E-1,6.1971188E-2,2.2200851E0,9.9692345E-3,1.7530835E-1,0E0,0E0,5.1499747E-2,4.8376667E-1,6.350949E-1,6.504136E-2,6.099306E-1,6.644356E-1,1.1059113E-2,0E0,2.0632544E0,9.2185307E-1,3.244078E-2,0E0,0E0,0E0,0E0,1.2879999E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,62,64,66,68,-1,-1,70,72,74,76,78,80,82,-1,84,86,88,-1,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,7.9016544E5,1E0,1E0,9.24E2,2.2968E4,2.2092846E7,6.76E2,3.7568388E0,1E0,6.0052995E6,1.2328386E3,3.206931E2,1.0089981E3,3.53562E5,4.7721977E4,2.74614E5,5E0,1.753E3,6.728972E-1,7.6606E4,1E0,1E0,1.635172E-2,4.8857614E-2,1.9851073E3,2.433735E0,9.5E1,2E0,-4.5616687E-3,5.111912E9,1.9186046E0,2.1652174E0,1.7E1,2.0734E4,2.0666666E0,5.3E1,-4.933257E-2,-1.6846698E-2,4.57E2,3.2182397E2,4.2212E4,2.87E2,1.1160929E0,4.4091418E4,2.483E3,2.129177E-2,1.8062708E6,1.4497429E7,3.82E3,1.0640727E-2,1.5529803E-3,-5.3187786E-3,1.3754076E-2,1.4622712E3,-1.0876527E-2,-2.3524543E-2,-4.234955E-3,-1.76427E-2,-4.338301E-3,-1.2503415E-2,2.726205E-2,9.672502E-3,-5.844309E-4,-1.2817743E-2,-2.2440325E-2,-9.059557E-3,-1.2320502E-2,-3.111198E-3,-1.141936E-3,5.4818005E-3,-1.2171988E-2,-1.3407801E-4,1.8134656E-3,-1.04404E-2,1.9569477E-2,5.0015133E-3,1.5789894E-2,2.644116E-3,2.5675306E-3,2.0022629E-2,1.3517587E-3,6.688858E-3,7.389203E-3,1.8985E-2,3.3895347E-2,1.9014627E-2,-1.6886711E-2,-7.0468457E-3,6.32402E-3,-5.469448E-3],"split_indices":[2,40,6,6,2,12,57,2,65,27,40,64,70,4,1,40,5,6,41,66,1,26,27,0,0,64,65,0,8,0,43,66,65,3,41,65,0,0,0,0,64,1,0,66,45,2,0,40,59,0,0,0,0,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,7.77E2,2.33E2,4.74E2,3.03E2,2.01E2,3.2E1,4.16E2,5.8E1,1.39E2,1.64E2,1.1E1,1.9E2,1.7E1,1.5E1,2.08E2,2.08E2,4.7E1,1.1E1,4.4E1,9.5E1,1.49E2,1.5E1,2E0,9E0,1.81E2,9E0,6E0,1.1E1,3E0,1.2E1,8.7E1,1.21E2,1.1E1,1.97E2,2.4E1,2.3E1,9E0,2E0,1.1E1,3.3E1,9.1E1,4E0,1.3E2,1.9E1,4E0,1.1E1,1.33E2,4.8E1,7E0,2E0,2E0,4E0,4E0,7E0,4E0,8E0,5E0,8.2E1,6.3E1,5.8E1,9E0,2E0,1.59E2,3.8E1,2.2E1,2E0,1.6E1,7E0,3E0,8E0,2.3E1,1E1,8.2E1,9E0,2E0,2E0,7E0,1.23E2,8E0,1.1E1,2E0,2E0,7.1E1,6.2E1,1.7E1,3.1E1,3E0,4E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-6.670009E-4,-1.880574E-1,1.4720227E-1,-3.99263E-1,-1.339532E-1,6.743952E-2,4.4181618E-1,-2.999972E-1,-4.8745036E-1,-2.115003E-1,4.927224E-2,-5.1236015E-2,1.3624187E-1,5.719775E-1,2.2084747E-1,-3.197865E-1,-8.168375E-2,-6.445275E-1,-2.7724186E-1,-2.673008E-1,-3.5923723E-2,-7.004264E-3,4.1155007E-1,-3.2172126E-1,-2.9108103E-2,1.6144456E-1,-7.496156E-2,3.6859176E-1,6.581631E-1,1.4463758E-1,4.7311834E-1,-3.3490694E-1,-9.951128E-4,5.856888E-4,-6.8387906E-3,-5.409132E-1,-4.36718E-2,3.993022E-4,-3.253807E-1,-2.4297276E-1,-8.5121804E-1,5.4739054E-2,-1.4149329E-1,2.1790931E-2,-2.3270749E-2,-9.309346E-3,5.308114E-1,-1.0630412E-1,-2.7652834E-2,-6.5120295E-2,1.817405E-1,1.0024513E-2,2.0499471E-1,-9.505071E-3,-3.3520332E-1,-4.1380964E-2,4.5265535E-1,4.0074512E-2,5.505806E-1,6.510144E-2,2.6497287E-1,5.3548464E-3,2.5155753E-2,-1.58178E-2,-5.03511E-3,-2.612841E-2,-8.434944E-3,-1.7806532E-2,-3.9445753E-3,-2.3128816E-3,-1.2855161E-2,-7.5714104E-3,-5.0194025E-2,-2.128773E-3,7.5805094E-3,-1.40785305E-2,-1.5060176E-3,9.051737E-3,-6.671032E-4,3.357321E-2,1.3018946E-2,-7.219476E-3,1.4183851E-3,-3.8307447E-3,7.300089E-3,1.3822086E-2,2.633729E-3,-3.6086186E-3,4.6583805E-3,1.09425085E-2,1.4875614E-3,-7.020113E-3,2.5136853E-3,-2.004575E-3,-2.159118E-2,7.839182E-3,-1.1007003E-2,1.1772439E-2,2.4741825E-2,2.6743261E-2,6.9692843E-3,-2.5512122E-3,6.7347824E-3,6.6703544E-3,1.5462016E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,-1,-1,65,-1,-1,67,69,71,73,75,77,-1,-1,79,81,-1,83,85,87,89,91,93,95,97,-1,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7265905E1,4.905881E0,1.2894772E1,6.2411785E-1,4.940806E0,3.550658E0,3.2387848E0,1.6397E-1,1.3071632E0,2.38093E0,2.1232064E0,9.4827926E-1,1.4758983E0,1.0370159E0,7.9671836E-1,1.7305326E-1,3.3632137E-2,2.790575E-1,2.9188526E-1,2.4212074E0,5.82019E-1,1.3202665E0,1.1331367E0,6.6063726E-1,1.1432099E0,1.6201591E0,5.078492E-1,8.4586215E-1,6.7248344E-1,3.2730615E-1,2.019856E-1,4.3326378E-2,0E0,0E0,0E0,1.0517645E-1,0E0,0E0,2.352606E-1,1.258296E0,9.8057795E-1,3.7559152E-1,4.9222142E-1,5.566371E-1,0E0,0E0,3.8688827E-1,6.0687758E-2,0E0,5.375318E-1,3.0893737E-1,4.6138337E-1,1.0881605E0,2.3852876E-1,2.154209E-1,2.5110477E-1,1.9970894E-1,0E0,3.6626244E-1,2.343632E-1,7.1248114E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,35,35,38,38,39,39,40,40,41,41,42,42,43,43,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,-1,-1,66,-1,-1,68,70,72,74,76,78,-1,-1,80,82,-1,84,86,88,90,92,94,96,98,-1,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.095006E5,7.7E1,5.97E3,8.75477E5,3.2874417E2,9.24E2,8.166121E1,9.932432E0,1.4723502E1,4.39776E6,1.1459359E3,1.45064E5,1E0,1.3733966E3,3.8537518E2,1.1408248E0,2.3E1,3.7798166E0,9.61E2,6.27957E0,4.87E2,4E1,7E0,6.99E2,4.5866325E6,5.4007E7,2.2092846E7,1.4523518E8,1.3587301E1,1.6326721E0,1.5E1,1.056178E6,-9.951128E-4,5.856888E-4,-6.8387906E-3,1.82E3,-4.36718E-2,3.993022E-4,2.3751075E5,9.2058825E-1,1.3E1,1.264E3,1.24272164E5,1.703125E0,-2.3270749E-2,-9.309346E-3,4.1221875E-1,2.01E2,-2.7652834E-2,9.489462E6,2.1525E4,2.0467092E6,4.6E1,7.368848E2,1.0041E5,2.5959E4,2.8172503E10,4.0074512E-2,8.7E1,7.073537E5,1.7307587E2,5.3548464E-3,2.5155753E-2,-1.58178E-2,-5.03511E-3,-2.612841E-2,-8.434944E-3,-1.7806532E-2,-3.9445753E-3,-2.3128816E-3,-1.2855161E-2,-7.5714104E-3,-5.0194025E-2,-2.128773E-3,7.5805094E-3,-1.40785305E-2,-1.5060176E-3,9.051737E-3,-6.671032E-4,3.357321E-2,1.3018946E-2,-7.219476E-3,1.4183851E-3,-3.8307447E-3,7.300089E-3,1.3822086E-2,2.633729E-3,-3.6086186E-3,4.6583805E-3,1.09425085E-2,1.4875614E-3,-7.020113E-3,2.5136853E-3,-2.004575E-3,-2.159118E-2,7.839182E-3,-1.1007003E-2,1.1772439E-2,2.4741825E-2,2.6743261E-2,6.9692843E-3,-2.5512122E-3,6.7347824E-3,6.6703544E-3,1.5462016E-2],"split_indices":[40,41,2,43,64,2,68,66,70,9,64,7,6,4,68,50,3,65,2,65,10,3,3,2,40,7,57,12,68,50,3,9,0,0,0,2,0,0,45,65,0,41,40,65,0,0,50,0,0,12,9,40,8,67,1,9,43,0,8,40,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.82E2,4.33E2,5.49E2,8.7E1,3.46E2,4.33E2,1.16E2,4.3E1,4.4E1,2.43E2,1.03E2,1.59E2,2.74E2,7.2E1,4.4E1,3.9E1,4E0,2.4E1,2E1,1.84E2,5.9E1,9E1,1.3E1,1.1E1,1.48E2,2.45E2,2.9E1,2.3E1,4.9E1,3.5E1,9E0,3.7E1,2E0,2E0,2E0,2E1,4E0,3E0,1.7E1,1.78E2,6E0,3.2E1,2.7E1,8.6E1,4E0,2E0,1.1E1,7E0,4E0,1.27E2,2.1E1,5.5E1,1.9E2,2.4E1,5E0,4E0,1.9E1,1.4E1,3.5E1,2.2E1,1.3E1,2E0,7E0,3.5E1,2E0,1.8E1,2E0,1.3E1,4E0,2.9E1,1.49E2,2E0,4E0,1.7E1,1.5E1,1E1,1.7E1,1.4E1,7.2E1,5E0,6E0,5E0,2E0,1.18E2,9E0,1E1,1.1E1,2.8E1,2.7E1,1.59E2,3.1E1,7E0,1.7E1,2E0,3E0,2E0,2E0,7E0,1.2E1,3.2E1,3E0,9E0,1.3E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.809268E-4,-8.959013E-2,3.0198127E-1,-2.1527271E-1,2.94869E-2,4.5945078E-2,2.777281E-1,-4.0959895E-1,-1.6189522E-1,-7.219105E-2,9.876495E-2,1.3273177E-1,4.110785E-1,-2.8136003E-1,-5.022981E-1,-2.583163E-1,-7.512944E-2,-1.1569461E-1,9.478277E-2,1.10449895E-1,-3.5588136E-1,-2.6961333E-1,1.736867E-1,4.283885E-1,-1.6300071E-2,-3.298631E-1,-4.0224455E-2,-5.324634E-1,-3.1017358E-3,-2.3122478E-1,-5.1899356E-1,2.1526728E-2,-1.0836436E-1,-9.093822E-2,-3.4123915E-1,1.484454E-2,1.4068926E-2,4.4675447E-2,1.7641273E-1,-2.9413996E-2,-2.4587743E-3,-3.9314153E-3,-1.5444559E-2,1.9764742E-1,-1.5576918E-1,2.4953221E-1,5.027665E-1,-1.616122E-2,-7.070415E-3,-5.433052E-3,2.1998018E-3,-2.1966722E-2,-4.199903E-2,-2.3419857E-2,-9.5451735E-3,-2.8385462E-2,-1.1794885E-2,-3.0402977E-3,-1.2620687E-2,1.0005545E-3,-6.778842E-3,-7.856417E-3,-2.2181464E-2,2.5272267E-3,-1.3693466E-2,2.8442629E-3,-2.0445943E-2,1.2992709E-2,2.7990695E-3,6.5084593E-3,1.9579234E-2,-6.508825E-4,-9.626078E-3,1.431125E-2,4.104838E-4,2.7289633E-2,1.6469453E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,-1,53,55,-1,57,59,61,63,-1,65,67,-1,-1,-1,-1,69,71,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6081593E1,1.1230781E1,3.4481754E0,3.7197037E0,2.7261336E0,0E0,4.0831947E0,7.8590965E-1,2.3900156E0,1.151407E0,1.2611861E0,1.760542E0,1.6237411E0,3.960848E-1,5.410986E-1,8.404474E-1,2.815975E0,6.669271E-1,5.534327E-1,9.683192E-1,4.8184222E-1,9.3847334E-2,7.807176E-1,1.3245659E0,0E0,5.8036566E-2,5.381509E-2,4.11623E-1,0E0,7.095332E-1,1.528027E-1,0E0,9.993936E-1,7.311522E-1,2.0676363E-1,3.3076096E-1,0E0,9.907022E-1,1.3595817E0,0E0,0E0,0E0,0E0,1.0780008E0,5.036135E-2,5.0476336E-1,7.8832245E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,32,32,33,33,34,34,35,35,37,37,38,38,43,43,44,44,45,45,46,46],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,-1,54,56,-1,58,60,62,64,-1,66,68,-1,-1,-1,-1,70,72,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.56E3,4.539777E5,2.2968E4,7.7E1,2.485E2,4.5945078E-2,1.7587205E6,6.48334E5,6.07E2,4.5866325E6,1.4676277E7,8.147158E4,1.0534078E4,2.0376764E-7,1.75E2,3.1E1,1.9722162E7,2E0,1.9696634E0,3.9892856E2,2.78E2,4.3919165E8,2.5130852E2,1.0555E3,-1.6300071E-2,2.7546012E0,1.7864E4,7.280084E0,-3.1017358E-3,3.6E1,4.8E2,2.1526728E-2,2.0734E4,1.3619632E0,5E1,1.3508157E7,1.4068926E-2,2.511E3,3.5834332E0,-2.9413996E-2,-2.4587743E-3,-3.9314153E-3,-1.5444559E-2,3.4915986E3,3.0612E4,1.835821E1,1.7103828E7,-1.616122E-2,-7.070415E-3,-5.433052E-3,2.1998018E-3,-2.1966722E-2,-4.199903E-2,-2.3419857E-2,-9.5451735E-3,-2.8385462E-2,-1.1794885E-2,-3.0402977E-3,-1.2620687E-2,1.0005545E-3,-6.778842E-3,-7.856417E-3,-2.2181464E-2,2.5272267E-3,-1.3693466E-2,2.8442629E-3,-2.0445943E-2,1.2992709E-2,2.7990695E-3,6.5084593E-3,1.9579234E-2,-6.508825E-4,-9.626078E-3,1.431125E-2,4.104838E-4,2.7289633E-2,1.6469453E-2],"split_indices":[2,40,12,41,64,0,40,43,2,40,59,40,4,49,6,8,43,6,65,64,0,43,70,64,0,65,1,65,0,2,10,0,41,65,0,57,0,2,66,0,0,0,0,4,2,68,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.68E2,7.49E2,2.19E2,3.64E2,3.85E2,6E0,2.13E2,7.7E1,2.87E2,1.56E2,2.29E2,1.03E2,1.1E2,3.4E1,4.3E1,1.35E2,1.52E2,1.24E2,3.2E1,2.24E2,5E0,9E0,9.4E1,1.08E2,2E0,2.8E1,6E0,4E1,3E0,1.24E2,1.1E1,8E0,1.44E2,1.13E2,1.1E1,2.4E1,8E0,1.13E2,1.11E2,2E0,3E0,3E0,6E0,8.8E1,6E0,3.3E1,7.5E1,2.4E1,4E0,3E0,3E0,3.7E1,3E0,8E0,1.16E2,7E0,4E0,1.16E2,2.8E1,3.8E1,7.5E1,6E0,5E0,2.2E1,2E0,1.1E2,3E0,5.7E1,5.4E1,7.2E1,1.6E1,2E0,4E0,2.6E1,7E0,4.4E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[8.49203E-5,-8.69809E-2,3.0905265E-1,-1.902063E-1,2.4204485E-2,-3.5905713E-1,3.289059E-1,-1.5319097E-1,-3.877323E-1,-1.0101091E-1,7.1798235E-2,-4.3697474E-3,-2.0467022E-2,2.8612792E-1,5.902954E-1,-3.2812786E-1,-8.05042E-2,-1.698533E-1,-4.7403336E-1,1.0991316E-2,-2.2420926E-1,5.429689E-2,2.1109924E-2,7.984793E-1,2.5936678E-1,8.487771E-3,6.285072E-1,-3.4379065E-1,-8.09052E-4,5.069724E-1,-1.2554315E-1,-1.9900829E-1,2.8337177E-3,-6.658989E-1,-3.495084E-1,1.6704199E-1,-4.8718255E-2,-4.5588478E-1,-8.402612E-2,6.1651103E-2,-2.8513312E-2,1.303777E-2,4.1566458E-2,6.421131E-2,3.083842E-1,3.044699E-2,1.2179146E-2,-1.810503E-2,-1.3395492E-2,7.369115E-3,2.5978135E-2,-8.606934E-3,-2.0976772E-3,-4.4873916E-3,-1.1694047E-2,-1.6335694E-2,-3.516199E-2,-1.7358991E-3,-1.701415E-2,6.5789744E-4,1.0801143E-2,-1.8229572E-2,9.339835E-4,-1.1213866E-2,-2.8571172E-2,1.665785E-3,-5.148975E-3,5.9448916E-4,9.030915E-3,6.816539E-3,-5.9976843E-3,1.6587941E-2,8.356808E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,-1,41,43,-1,45,47,-1,49,51,53,-1,55,57,59,61,63,65,67,-1,-1,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7653728E1,9.209283E0,3.1157932E0,2.9821835E0,2.3135674E0,1.1748779E-1,2.2910767E0,4.4494743E0,1.1416903E0,1.4789027E0,1.882184E0,0E0,0E0,2.411724E0,3.4870815E-1,4.9488544E-1,6.6784782E0,1.4729303E-1,8.6629677E-1,5.39248E-1,1.6180212E0,1.3787262E0,0E0,2.3632908E-1,1.7307787E0,0E0,1.6141891E-1,1.4253998E-1,0E0,2.748556E-1,1.1423726E0,7.649845E-2,0E0,2.5321102E-1,1.8159175E-1,1.6307148E-1,1.0178273E0,5.165837E-1,1.1551361E-1,1.759269E0,0E0,0E0,0E0,6.4259326E-1,9.0829563E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,-1,42,44,-1,46,48,-1,50,52,54,-1,56,58,60,62,64,66,68,-1,-1,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.659E3,4.9236734E5,1E0,2.2E1,1E0,1.2E1,6.9307615E6,2.71E2,8.908E3,2.4E0,7.168071E6,-4.3697474E-3,-2.0467022E-2,2.4481E4,1.59E2,4.4761734E5,1.2E1,3.4380876E-3,3.3817584E7,5E-1,2.906237E2,3.174172E5,2.1109924E-2,1.5E1,7.445455E2,8.487771E-3,1.0593E5,3.89E2,-8.09052E-4,4.456432E6,1.339646E6,1E0,2.8337177E-3,1.2131076E2,3.7990784E7,9.66E2,7.13E2,5E1,1E0,4.371613E2,-2.8513312E-2,1.303777E-2,4.1566458E-2,9.6203804E-1,8.7E1,3.044699E-2,1.2179146E-2,-1.810503E-2,-1.3395492E-2,7.369115E-3,2.5978135E-2,-8.606934E-3,-2.0976772E-3,-4.4873916E-3,-1.1694047E-2,-1.6335694E-2,-3.516199E-2,-1.7358991E-3,-1.701415E-2,6.5789744E-4,1.0801143E-2,-1.8229572E-2,9.339835E-4,-1.1213866E-2,-2.8571172E-2,1.665785E-3,-5.148975E-3,5.9448916E-4,9.030915E-3,6.816539E-3,-5.9976843E-3,1.6587941E-2,8.356808E-3],"split_indices":[2,40,17,3,27,3,40,2,1,66,40,0,0,12,0,40,30,49,7,65,4,45,0,3,64,0,41,1,0,7,9,19,0,45,7,12,2,0,15,64,0,0,0,50,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.026E3,8.01E2,2.25E2,4.15E2,3.86E2,6E0,2.19E2,3.51E2,6.4E1,1.06E2,2.8E2,2E0,4E0,1.9E2,2.9E1,1.02E2,2.49E2,1.9E1,4.5E1,5.6E1,5E1,2.69E2,1.1E1,8E0,1.82E2,3E0,2.6E1,9.7E1,5E0,1.7E1,2.32E2,1.7E1,2E0,1.6E1,2.9E1,1.5E1,4.1E1,1.8E1,3.2E1,2.67E2,2E0,2E0,6E0,3.7E1,1.45E2,2.3E1,3E0,4.6E1,5.1E1,3E0,1.4E1,1.3E2,1.02E2,7E0,1E1,5E0,1.1E1,2E0,2.7E1,5E0,1E1,6E0,3.5E1,9E0,9E0,6E0,2.6E1,1.97E2,7E1,2.6E1,1.1E1,1.01E2,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[1.8789526E-3,-1.1016901E-1,2.0307167E-1,-2.5340596E-1,-3.3297695E-2,2.5248772E-1,-1.1527587E-1,-2.956751E-1,-1.0666415E-1,-6.523511E-2,1.438304E-1,7.727589E-2,3.36067E-1,9.7952366E-2,-2.7306017E-1,-2.6324698E-1,-4.532111E-1,-1.9514383E-1,3.7454605E-2,-1.1680621E-1,6.0205545E-2,5.7106033E-2,3.4044117E-1,-7.0118405E-2,1.7825457E-1,4.2345765E-1,1.5285979E-1,-7.072935E-2,1.656286E-1,-3.0799836E-1,6.0814945E-3,-3.7800208E-1,-2.0721306E-1,-6.364392E-1,-2.86436E-1,-7.0093647E-3,-2.4552584E-1,9.942096E-2,-1.1517686E-2,-9.894194E-2,-4.7626728E-1,8.19752E-2,-1.591576E-1,-6.5196864E-2,1.1096778E-1,1.0940386E-1,3.9136273E-1,-2.3909032E-2,-1.612617E-2,-1.09182345E-2,2.1927065E-1,5.2534777E-1,3.0035636E-1,2.1924303E-1,-7.0901304E-2,-5.5972203E-3,1.7483805E-3,-1.0874013E-3,1.9436291E-1,-3.6277097E-1,-9.968074E-2,-9.522097E-3,-1.9174831E-2,4.0059495E-3,-1.1344966E-2,-2.1405453E-2,-4.1492447E-2,-7.702522E-3,-2.1871561E-2,-3.6386438E-3,2.3957933E-3,-3.9614798E-3,-1.3764215E-2,5.7127294E-3,-3.0685381E-3,-9.667524E-3,-2.710114E-3,-2.921437E-2,-5.7586753E-3,-4.614218E-3,5.1010977E-3,-1.0324743E-2,3.164798E-3,-8.844582E-3,4.9546207E-3,3.2242793E-3,1.159184E-2,4.6020118E-4,7.914946E-3,7.5175804E-3,1.8932972E-2,8.299546E-3,-1.8508176E-3,7.66168E-3,-4.0738466E-3,2.487279E-3,1.2094902E-2,1.9204324E-2,3.224008E-2,2.1210872E-2,8.075034E-3,7.3013254E-3,2.8013503E-2,4.2114975E-3,-1.0362865E-2,1.9340281E-3,9.93178E-3,-5.302127E-3,-1.9023802E-2,6.5306793E-3,-1.0328295E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,-1,75,77,79,81,83,85,87,89,91,-1,93,95,97,99,101,103,-1,-1,-1,105,107,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3129187E1,7.2476873E0,5.802559E0,1.395071E0,2.4418988E0,4.630846E0,1.7089505E0,7.900238E-1,6.8314296E-1,2.3682585E0,1.105921E0,1.5604022E0,3.3775501E0,2.6286113E-1,4.5159864E-1,8.8619995E-1,7.0387983E-1,3.1107235E-1,3.9997926E-1,1.6160481E0,5.2402115E-1,3.1684476E-1,1.924789E-1,5.4989994E-1,4.834596E-1,1.6609135E0,1.0701779E0,4.3577667E-2,9.163326E-2,2.7191162E-1,0E0,2.3678493E-1,1.2280431E0,1.7031384E-1,3.0315614E-1,3.831282E-2,1.983701E-1,8.2973585E-2,0E0,1.0923088E0,5.930178E-1,5.311264E-1,1.6515914E-1,3.51406E-1,1.7253217E-1,2.953818E-2,8.446932E-3,1.3468146E-1,0E0,1.8066654E-1,3.567207E-1,1.1834717E0,1.2860923E0,1.198267E0,4.4832668E-1,0E0,0E0,0E0,3.7506938E-2,2.1919584E-1,2.4391045E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,58,58,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,-1,76,78,80,82,84,86,88,90,92,-1,94,96,98,100,102,104,-1,-1,-1,106,108,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,1E0,1.380863E6,3.086474E6,5.862504E2,2.2092846E7,1E0,1.5422421E2,3.422351E6,5.4007E7,3.1720988E6,6.1E1,1.342376E0,8.874315E-6,3.89E2,1E0,1.375E0,3.78E2,1.0948052E1,4.3E1,1.1967312E3,7.3904707E9,6.70019E1,5.201557E7,2.115238E5,3.85374E8,1.0089981E3,6.875854E2,1.6322028E3,6.0814945E-3,7E0,9.0909094E-1,2E1,7.042942E7,1.5077367E3,9.185E3,1E0,-1.1517686E-2,1.5422421E2,1.639E3,5.3E2,3.1146461E1,2.2E1,2.9994638E2,3.2075E4,2.6732E4,1.0512184E4,-1.612617E-2,3.412336E7,6.374563E5,1.4194314E3,2.1356704E0,3.5675005E3,1.6677892E2,-5.5972203E-3,1.7483805E-3,-1.0874013E-3,3.1561086E0,1.7234043E1,2.6506329E1,-9.522097E-3,-1.9174831E-2,4.0059495E-3,-1.1344966E-2,-2.1405453E-2,-4.1492447E-2,-7.702522E-3,-2.1871561E-2,-3.6386438E-3,2.3957933E-3,-3.9614798E-3,-1.3764215E-2,5.7127294E-3,-3.0685381E-3,-9.667524E-3,-2.710114E-3,-2.921437E-2,-5.7586753E-3,-4.614218E-3,5.1010977E-3,-1.0324743E-2,3.164798E-3,-8.844582E-3,4.9546207E-3,3.2242793E-3,1.159184E-2,4.6020118E-4,7.914946E-3,7.5175804E-3,1.8932972E-2,8.299546E-3,-1.8508176E-3,7.66168E-3,-4.0738466E-3,2.487279E-3,1.2094902E-2,1.9204324E-2,3.224008E-2,2.1210872E-2,8.075034E-3,7.3013254E-3,2.8013503E-2,4.2114975E-3,-1.0362865E-2,1.9340281E-3,9.93178E-3,-5.302127E-3,-1.9023802E-2,6.5306793E-3,-1.0328295E-2],"split_indices":[2,40,6,9,40,64,57,6,64,9,7,57,8,66,49,1,24,65,0,66,8,45,43,70,7,45,44,4,64,4,0,0,65,6,57,45,12,28,0,64,2,2,68,8,64,1,1,45,0,7,40,64,50,64,70,0,0,0,58,68,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.024E3,6.58E2,3.66E2,2.29E2,4.29E2,3.17E2,4.9E1,1.77E2,5.2E1,3.64E2,6.5E1,1.03E2,2.14E2,2.1E1,2.8E1,1.49E2,2.8E1,3.2E1,2E1,2.58E2,1.06E2,4.6E1,1.9E1,4.2E1,6.1E1,1.44E2,7E1,6E0,1.5E1,2.6E1,2E0,4.7E1,1.02E2,1.2E1,1.6E1,7E0,2.5E1,1.7E1,3E0,2.47E2,1.1E1,9.7E1,9E0,1.4E1,3.2E1,4E0,1.5E1,3.7E1,5E0,1.1E1,5E1,7.7E1,6.7E1,5.4E1,1.6E1,4E0,2E0,2E0,1.3E1,2E1,6E0,1E1,3.7E1,1.2E1,9E1,9E0,3E0,1.1E1,5E0,3E0,4E0,7E0,1.8E1,1.5E1,2E0,6.4E1,1.83E2,7E0,4E0,1.3E1,8.4E1,7E0,2E0,8E0,6E0,2.6E1,6E0,2E0,2E0,2E0,1.3E1,2E0,3.5E1,3E0,8E0,1.1E1,3.9E1,5E1,2.7E1,2.8E1,3.9E1,4.8E1,6E0,8E0,8E0,2E0,1.1E1,4E0,1.6E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-8.7014E-3,-1.6137205E-1,1.00930125E-1,-3.408949E-1,-1.12017594E-1,3.9468266E-2,3.6502317E-1,-3.7271026E-1,2.7824223E-2,-1.4939044E-1,1.5232448E-1,-6.1801128E-2,1.0030233E-1,4.5397004E-1,1.8688636E-1,-3.0780628E-1,-5.357298E-1,1.9051906E-2,-1.702956E-1,-1.1759468E-1,-4.0816846E-1,4.6030018E-1,3.81291E-2,-3.5128843E-2,-3.698335E-1,4.088556E-2,1.8491568E-1,2.4165693E-1,5.550921E-1,-3.7485544E-2,2.9188436E-1,-3.2233346E-1,3.6973339E-3,-4.1632843E-1,-3.9428968E-2,-1.0312402E-2,-1.8940115E-3,-2.416814E-1,-6.317048E-2,-8.0345577E-1,-3.0124953E-1,2.7886374E-4,2.5747634E-2,1.5113804E-2,-1.7157447E-2,-2.5300285E-1,-1.3107252E-2,-1.16462104E-1,-4.5953587E-1,-2.960105E-2,1.3653776E-1,2.1949798E-1,-1.808617E-1,3.135056E-1,-1.0843879E-1,4.8895305E-1,4.0081132E-2,4.2859063E-2,-1.3836409E-1,3.7489432E-1,1.30152535E-2,-1.5355457E-2,5.8328145E-4,-2.008593E-2,-7.610685E-3,-1.8499516E-2,-9.990954E-3,1.3008415E-2,-4.1153724E-3,-1.0825408E-2,-4.7236893E-2,-1.971359E-3,-1.6568102E-2,-5.2227913E-3,4.3895408E-3,1.6316818E-3,-2.6999617E-2,-3.4768963E-3,2.8231891E-3,-1.2661224E-3,-7.6493523E-3,-2.5101272E-2,-8.840759E-3,3.6496553E-5,-2.0112714E-2,4.355579E-3,1.4170896E-2,1.2964873E-2,5.0368523E-3,-2.788747E-2,-8.5749134E-4,7.1635093E-3,1.8301858E-2,7.0035364E-3,-1.5304806E-2,2.4311405E-2,1.1544048E-2,4.4553294E-3,-1.6319959E-3,-8.815105E-3,-9.570806E-4,3.8830389E-3,1.848558E-2,4.5065503E-3,-3.4603816E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,-1,-1,-1,65,67,69,71,-1,-1,-1,73,75,77,79,81,83,85,87,89,91,93,95,-1,97,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6619902E1,3.6388178E0,9.367485E0,1.0544233E0,3.2532597E0,2.9016943E0,1.6422338E0,7.0270824E-1,6.839601E-1,2.313449E0,1.4243948E0,1.441227E0,1.4705105E0,1.411665E0,9.0616846E-1,3.7004375E-1,4.902873E-1,0E0,3.2728642E-2,1.7203369E0,1.0681343E0,4.9757314E-1,5.0425106E-1,7.835447E-1,2.6467383E-1,1.1771842E0,1.5702786E0,6.628438E-1,5.81398E-1,1.11295775E-1,5.969925E-1,2.4101973E-1,0E0,2.2157907E-2,0E0,0E0,0E0,2.1598434E-1,1.6731029E0,5.2398205E-1,3.83909E-1,0E0,0E0,0E0,3.047246E-1,1.4699517E0,7.0477676E-1,1.7667256E-2,1.2703967E-1,1.2537615E0,5.015819E-1,7.343049E-1,7.4922E-1,2.1892118E-1,3.4403822E-1,2.6960278E-1,0E0,3.7398983E-2,3.376537E-2,1.2604856E-1,6.002317E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,37,37,38,38,39,39,40,40,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,-1,-1,-1,66,68,70,72,-1,-1,-1,74,76,78,80,82,84,86,88,90,92,94,96,-1,98,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7567694E5,7.7E1,5.97E3,1.5900173E3,5.928425E2,9.24E2,6.6933334E1,1.4504054E8,1.3E1,4.586207E0,1.2193182E1,2.727619E1,6.880842E-1,3.97371E5,1.3388E4,1.1408248E0,3.5E0,1.9051906E-2,1.75E2,3.72E2,1E0,1.32E2,1.1411E4,5.862757E1,3.6828358E0,2.2318378E6,1.853776E7,2.1356704E0,3.0612E4,8.611817E6,2.3876712E7,1.0428572E1,3.6973339E-3,6.233195E-2,-3.9428968E-2,-1.0312402E-2,-1.8940115E-3,3.9E1,1.4E1,1.654E3,3.13E2,2.7886374E-4,2.5747634E-2,1.5113804E-2,1.559733E6,2.28E0,2.732381E2,3.4E1,8.599521E5,2.115238E5,2.7302158E0,3.464E0,1E0,1.8863014E1,1.65915E5,8.992806E-4,4.0081132E-2,4.362075E8,1.94E3,4.110903E10,4.4910244E3,-1.5355457E-2,5.8328145E-4,-2.008593E-2,-7.610685E-3,-1.8499516E-2,-9.990954E-3,1.3008415E-2,-4.1153724E-3,-1.0825408E-2,-4.7236893E-2,-1.971359E-3,-1.6568102E-2,-5.2227913E-3,4.3895408E-3,1.6316818E-3,-2.6999617E-2,-3.4768963E-3,2.8231891E-3,-1.2661224E-3,-7.6493523E-3,-2.5101272E-2,-8.840759E-3,3.6496553E-5,-2.0112714E-2,4.355579E-3,1.4170896E-2,1.2964873E-2,5.0368523E-3,-2.788747E-2,-8.5749134E-4,7.1635093E-3,1.8301858E-2,7.0035364E-3,-1.5304806E-2,2.4311405E-2,1.1544048E-2,4.4553294E-3,-1.6319959E-3,-8.815105E-3,-9.570806E-4,3.8830389E-3,1.848558E-2,4.5065503E-3,-3.4603816E-3],"split_indices":[40,41,2,67,64,2,68,43,0,65,70,58,54,1,2,50,65,0,6,2,13,12,9,59,66,40,59,50,2,57,57,65,0,50,0,0,0,2,10,1,41,0,0,0,9,68,4,3,40,45,65,66,8,70,1,69,0,7,0,43,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.91E2,4.14E2,5.77E2,8.8E1,3.26E2,4.69E2,1.08E2,8.1E1,7E0,2.86E2,4E1,1.76E2,2.93E2,7.1E1,3.7E1,6E1,2.1E1,2E0,5E0,2.56E2,3E1,1E1,3E1,1.63E2,1.3E1,1.73E2,1.2E2,2.4E1,4.7E1,1.2E1,2.5E1,5.8E1,2E0,1.7E1,4E0,3E0,2E0,7.7E1,1.79E2,5E0,2.5E1,2E0,8E0,4E0,2.6E1,1.4E1,1.49E2,4E0,9E0,1E2,7.3E1,1.1E2,1E1,2E1,4E0,4.1E1,6E0,7E0,5E0,1.9E1,6E0,5.6E1,2E0,1.5E1,2E0,8E0,6.9E1,1.2E1,1.67E2,2E0,3E0,5E0,2E1,1.4E1,1.2E1,8E0,6E0,8.1E1,6.8E1,2E0,2E0,6E0,3E0,9.4E1,6E0,6E1,1.3E1,6.9E1,4.1E1,2E0,8E0,8E0,1.2E1,2E0,2E0,3.4E1,7E0,4E0,3E0,3E0,2E0,2E0,1.7E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.099742E-4,-6.69437E-2,2.6098165E-1,-2.0924953E-1,-4.78734E-3,3.19267E-1,-3.9835416E-2,-1.6859972E-1,-4.9495322E-1,-2.3822293E-2,2.5108707E-1,5.8845792E-2,3.8111523E-1,-2.8049248E-1,5.8020826E-2,-2.529156E-1,-5.647788E-2,-7.733696E-1,-3.1707653E-1,-1.2218365E-1,2.7565727E-2,1.932515E-1,1.8348968E-2,-4.592153E-3,1.9190623E-1,4.3080923E-1,1.13914505E-1,-3.2799202E-1,-2.7710304E-3,1.5131697E-1,-1.591481E-1,-2.3545948E-1,-3.27662E-2,2.2250187E-2,-1.1944854E-1,-1.4992161E-2,-4.5111697E-2,-1.1417726E-1,-3.9587784E-1,-9.145763E-2,-3.634189E-1,-8.516926E-2,5.4040525E-2,1.13214314E-1,3.4200448E-1,-1.1718103E-1,5.90261E-2,2.3776194E-1,2.3984655E-3,3.4046817E-1,6.301565E-1,1.840908E-1,-1.2192327E-2,-6.237599E-3,-1.6968109E-2,6.6943906E-2,1.6411256E-2,-9.133079E-3,-3.2034877E-3,-1.48474695E-2,-8.838534E-3,-1.1709346E-2,3.052094E-4,-7.926751E-3,9.74463E-4,-1.9358397E-2,-7.421728E-3,-3.43953E-3,-2.3293432E-2,-1.1126936E-2,-2.782397E-2,-5.181643E-3,5.94098E-3,2.8607878E-3,-2.4357997E-2,8.244561E-3,1.6829192E-3,4.651035E-3,1.7891856E-2,-8.046522E-3,-4.040621E-5,5.388589E-3,-2.6493203E-3,4.2367415E-3,1.2442661E-2,1.8362304E-2,5.639341E-3,3.5226934E-2,1.5601754E-2,-6.3216215E-4,1.4566747E-2,-2.957353E-3,4.3266485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,-1,55,57,59,-1,-1,61,-1,-1,63,65,67,69,71,73,75,77,79,81,83,-1,85,87,89,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7506277E1,7.1282387E0,3.5894985E0,2.7619333E0,2.7432346E0,2.7318E0,8.169674E-1,2.0277042E0,1.2404962E0,2.6519759E0,2.5752354E-1,2.8787798E-1,1.7833252E0,8.478874E-2,5.306074E-1,8.20673E-1,3.2595725E0,7.05925E-1,2.7457452E-1,1.3009582E0,1.034302E0,3.184533E-1,0E0,1.7882174E-1,5.804494E-2,1.8745251E0,6.6117406E-1,1.3530672E-2,0E0,2.889078E-1,1.4521316E-2,3.9113426E-1,0E0,0E0,1.4537804E0,0E0,0E0,5.9010133E-2,1.5818357E-2,1.0712065E0,4.4055915E-1,4.0595856E-1,1.3817677E0,1.0000977E-1,7.497811E-2,6.05594E-2,1.1561464E-1,1.3585925E-2,0E0,1.0095911E0,1.1301947E0,5.308524E-1,0E0,0E0,0E0,5.617448E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,34,34,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,-1,56,58,60,-1,-1,62,-1,-1,64,66,68,70,72,74,76,78,80,82,84,-1,86,88,90,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.812E3,1.972052E5,2.856934E7,2.6E1,5.405423E6,7.445455E2,1.465E4,1.5292561E2,3.355864E0,2.6801266E2,2.5E0,6.103178E8,2.2040408E7,2.060658E5,7.5491136E-1,4.5E1,1.176582E6,1.08712E5,1.0142858E1,2E0,1E0,3.1316666E2,1.8348968E-2,1.0039E4,1.03984375E1,1.9180963E3,4.7429975E6,5.111912E9,-2.7710304E-3,4.435876E2,1.620407E6,5.4E1,-3.27662E-2,2.2250187E-2,1.4912975E-1,-1.4992161E-2,-4.5111697E-2,3.3817584E7,1.138871E0,3.4E1,2.0303884E6,1.4433751E0,3.174172E5,1.20105E5,1.8018E4,2.2539758E5,5.376E3,1E0,2.3984655E-3,4.1681065E1,1.7103828E7,3.714144E8,-1.2192327E-2,-6.237599E-3,-1.6968109E-2,5.352619E2,1.6411256E-2,-9.133079E-3,-3.2034877E-3,-1.48474695E-2,-8.838534E-3,-1.1709346E-2,3.052094E-4,-7.926751E-3,9.74463E-4,-1.9358397E-2,-7.421728E-3,-3.43953E-3,-2.3293432E-2,-1.1126936E-2,-2.782397E-2,-5.181643E-3,5.94098E-3,2.8607878E-3,-2.4357997E-2,8.244561E-3,1.6829192E-3,4.651035E-3,1.7891856E-2,-8.046522E-3,-4.040621E-5,5.388589E-3,-2.6493203E-3,4.2367415E-3,1.2442661E-2,1.8362304E-2,5.639341E-3,3.5226934E-2,1.5601754E-2,-6.3216215E-4,1.4566747E-2,-2.957353E-3,4.3266485E-3],"split_indices":[2,40,57,3,40,64,41,64,70,67,66,7,63,45,50,6,5,7,70,6,27,4,0,41,68,64,40,43,0,68,10,2,0,0,50,0,0,7,50,3,57,50,45,12,9,45,2,8,0,68,59,7,0,0,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,8.05E2,2.03E2,2.44E2,5.61E2,1.7E2,3.3E1,2.15E2,2.9E1,5.23E2,3.8E1,3.3E1,1.37E2,9E0,2.4E1,1.22E2,9.3E1,1E1,1.9E1,1.79E2,3.44E2,2.9E1,9E0,2.3E1,1E1,1.15E2,2.2E1,7E0,2E0,1.7E1,7E0,1.19E2,3E0,9E0,8.4E1,4E0,6E0,6E0,1.3E1,1.6E2,1.9E1,6.5E1,2.79E2,2E1,9E0,8E0,1.5E1,7E0,3E0,8.1E1,3.4E1,1.9E1,3E0,2E0,5E0,1.3E1,4E0,4E0,3E0,3.7E1,8.2E1,4E1,4.4E1,4E0,2E0,1.1E1,2E0,1.55E2,5E0,1.4E1,5E0,5.8E1,7E0,2.76E2,3E0,1E1,1E1,2E0,7E0,5E0,3E0,1E1,5E0,2E0,5E0,6.3E1,1.8E1,2.2E1,1.2E1,8E0,1.1E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-6.7399116E-3,-1.2694593E-1,1.2694953E-1,-2.4144505E-1,-5.8762327E-2,1.6268817E-1,-3.2494223E-1,-2.0906088E-1,-4.6482083E-1,-1.6214064E-1,-1.8289754E-2,1.0879417E-1,3.9253724E-1,-2.5445553E-2,-2.1116415E-1,-3.1249744E-1,-1.6325955E-1,-4.5360293E-2,-3.1089428E-1,1.4846442E-2,-2.3476939E-1,-1.5143521E-1,1.4229036E-2,1.7846456E-1,-1.1314188E-2,6.446146E-1,2.1327388E-1,1.450032E-2,-2.7070373E-1,-3.2058105E-1,-3.3807787E-3,2.820982E-1,-1.8872717E-1,-3.8848866E-2,-3.7647757E-1,5.8374837E-2,-1.24078E-1,-3.2490367E-1,-1.2252413E-1,-2.5063342E-1,-6.867235E-2,3.606957E-2,-1.7649503E-1,2.1204583E-1,-3.9612714E-2,-2.4316168E-1,1.17724575E-2,6.9237095E-1,1.0525369E-2,-1.3251074E-2,2.4019131E-1,-2.9968274E-1,-2.6701314E-3,-8.136786E-3,-1.560802E-2,5.587562E-4,1.7804835E-2,-7.4661095E-3,-2.2134421E-2,-5.4618698E-3,2.4878888E-3,-1.3692446E-2,-2.8326977E-2,-3.917103E-3,3.9292546E-3,-1.1314999E-2,1.3415839E-3,-7.2865607E-3,-2.4013877E-2,-1.0979775E-2,7.6063225E-5,-1.8968568E-3,-1.5231572E-2,-4.613683E-3,4.1901134E-3,7.805728E-4,9.702393E-3,-3.3737528E-3,-1.2502252E-2,3.2883247E-3,1.4067473E-2,7.118424E-4,-1.2485262E-2,-1.2590495E-2,-5.1777638E-3,3.7527585E-3,-1.5500598E-3,1.4332672E-2,3.3937953E-2,1.6952852E-2,5.370678E-3,-1.5322736E-2,-8.682789E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,33,35,37,39,41,43,45,47,49,-1,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.605425E1,4.0922747E0,7.691813E0,1.3206577E0,1.381934E0,5.395402E0,7.951336E-1,7.6857424E-1,1.6241059E0,1.19894E0,1.03696E0,2.9893937E0,3.613144E0,0E0,8.696828E-1,7.6821804E-2,1.4369888E0,0E0,3.4220684E-1,1.7656346E-1,6.2792325E-1,3.6865497E-1,8.103104E-1,1.6634684E0,7.094033E-1,5.159321E-1,7.4461174E-1,0E0,1.24260664E-1,7.7706814E-2,0E0,1.9470614E-1,8.1194973E-1,4.3676503E-2,1.1740661E-1,9.316779E-2,1.3838816E-1,1.1016824E0,4.4895855E-1,3.4230304E-1,1.4634156E-1,5.778053E-1,1.7722785E-1,2.5719795E0,4.0092784E-1,1.7546952E-2,3.8792753E-1,3.0010605E-1,0E0,0E0,7.1948814E-1,1.1991501E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,34,36,38,40,42,44,46,48,50,-1,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.194E3,1.972052E5,3.592428E7,4.4895835E0,1E0,1.5142261E3,2.5807297E-2,4.9E2,2E0,1.5714285E0,4.92E2,8E0,1.4497429E7,-2.5445553E-2,1.2083E4,7.122052E7,2.6333334E1,-4.5360293E-2,6.48334E5,2.709091E0,5.0200376E7,1.561132E2,1.394E3,2E0,8.147158E4,1.094E5,5.854E3,1.450032E-2,1.75E2,1.863E3,-3.3807787E-3,3.25E2,2.8025E4,4.9E2,3.9682608E2,5.44E2,8.414097E-1,5.4E1,5E0,3.1E1,6.789622E7,8.257819E9,4.911E3,4.88E2,3.3E0,2.4737226E9,3.9E2,1E1,1.0525369E-2,-1.3251074E-2,1.2792593E2,4.2E1,-2.6701314E-3,-8.136786E-3,-1.560802E-2,5.587562E-4,1.7804835E-2,-7.4661095E-3,-2.2134421E-2,-5.4618698E-3,2.4878888E-3,-1.3692446E-2,-2.8326977E-2,-3.917103E-3,3.9292546E-3,-1.1314999E-2,1.3415839E-3,-7.2865607E-3,-2.4013877E-2,-1.0979775E-2,7.6063225E-5,-1.8968568E-3,-1.5231572E-2,-4.613683E-3,4.1901134E-3,7.805728E-4,9.702393E-3,-3.3737528E-3,-1.2502252E-2,3.2883247E-3,1.4067473E-2,7.118424E-4,-1.2485262E-2,-1.2590495E-2,-5.1777638E-3,3.7527585E-3,-1.5500598E-3,1.4332672E-2,3.3937953E-2,1.6952852E-2,5.370678E-3,-1.5322736E-2,-8.682789E-3],"split_indices":[2,40,57,65,27,64,50,1,29,66,2,29,59,0,9,7,59,0,43,68,7,4,10,6,40,10,2,0,6,9,0,2,41,1,4,12,65,0,8,10,7,43,41,4,65,43,0,3,0,0,70,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,5.25E2,4.72E2,1.95E2,3.3E2,4.38E2,3.4E1,1.72E2,2.3E1,9.2E1,2.38E2,3.56E2,8.2E1,1E1,2.4E1,5.1E1,1.21E2,4E0,1.9E1,2.7E1,6.5E1,4.6E1,1.92E2,2.25E2,1.31E2,3.3E1,4.9E1,2E0,2.2E1,4.9E1,2E0,6E0,1.15E2,4E0,1.5E1,2.1E1,6E0,3.5E1,3E1,2E1,2.6E1,1.73E2,1.9E1,1.95E2,3E1,1.1E1,1.2E2,2.9E1,4E0,2E0,4.7E1,1.9E1,3E0,7E0,4.2E1,2E0,4E0,1.07E2,8E0,2E0,2E0,1.3E1,2E0,3E0,1.8E1,3E0,3E0,2E1,1.5E1,1.5E1,1.5E1,6E0,1.4E1,2.2E1,4E0,1.57E2,1.6E1,1E1,9E0,7.9E1,1.16E2,2.5E1,5E0,8E0,3E0,4.7E1,7.3E1,4E0,2.5E1,2.2E1,2.5E1,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[5.9692697E-3,-1.34653E-1,1.0448756E-1,-2.7088928E-1,-8.157908E-2,4.5547903E-2,2.975885E-1,-2.899197E-1,7.5014435E-2,-1.16459645E-1,1.4841047E-1,2.147418E-1,7.337894E-3,3.5142758E-1,4.3866877E-2,-2.455034E-1,-4.1010815E-1,1.674798E-2,-5.225047E-3,1.07114315E-2,-1.6060796E-1,1.932328E-2,5.8683E-2,1.4665894E-1,3.3080304E-1,-8.928573E-2,5.1933777E-2,2.5011134E-1,4.5368835E-1,3.1040955E-1,-7.06704E-2,-3.3056825E-1,-1.3159241E-1,-3.3173972E-1,-3.306744E-2,-1.892569E-1,8.35708E-2,-1.2967385E-1,-3.3054766E-1,1.2049228E-1,-1.4180636E-1,1.0027826E-1,1.7349593E-2,1.9160846E-2,1.9614269E-1,-7.079681E-2,-2.8127907E-2,3.608535E-2,1.7412752E-2,3.5184342E-1,1.4031114E-1,5.785226E-1,2.9355684E-1,1.6553931E-2,4.912383E-3,-1.8330877E-1,2.0393084E-1,-1.2536181E-2,-1.8564377E-2,8.564615E-5,-1.004572E-2,-5.5211447E-3,-1.6825108E-2,-9.755663E-3,2.2729031E-4,1.6403345E-2,2.4497325E-3,-9.197256E-3,-2.3656697E-3,-2.6022096E-3,-1.7594548E-2,-3.7960932E-3,8.680662E-3,-2.7857223E-3,-9.544619E-3,7.144421E-3,-2.7780195E-3,1.085959E-2,1.6681079E-3,-4.896239E-3,2.2087314E-3,1.9926776E-3,-2.671842E-2,-4.2528115E-4,1.8633649E-2,-7.1816356E-3,1.00022955E-2,2.788733E-2,5.004356E-3,5.902879E-3,1.6364692E-2,6.3950247E-3,-1.1421688E-2,3.679514E-3,1.3827699E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,-1,-1,77,79,-1,81,-1,83,85,87,89,-1,-1,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4325113E1,3.0581336E0,6.9062777E0,8.05089E-1,2.488111E0,3.0175326E0,1.9243231E0,5.025482E-1,4.2417657E-1,1.5063767E0,9.7700834E-1,6.310377E-1,1.6506809E0,1.0987282E0,8.156927E-1,7.801347E-1,4.652953E-1,0E0,0E0,1.035686E0,1.0032663E0,0E0,4.1363385E-1,5.622797E-1,2.7867556E-1,1.1269072E0,1.3448175E0,6.32015E-1,9.69594E-1,4.283011E-2,6.250643E-1,9.3324184E-2,4.422167E-1,1.3878083E-1,0E0,8.643079E-2,4.1106132E-1,9.2740583E-1,4.0978384E-1,3.6349007E-1,3.024435E-2,4.3587205E-1,0E0,0E0,8.155835E-2,5.073659E-1,0E0,1.1569402E0,0E0,6.06446E-1,7.463676E-1,3.5056686E-1,2.3373461E-1,0E0,0E0,3.2975194E-1,4.803644E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,44,44,45,45,47,47,49,49,50,50,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,-1,-1,78,80,-1,82,-1,84,86,88,90,-1,-1,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.539777E5,1.16E2,1.1941613E3,1.665909E3,5.928425E2,7E0,1.6677892E2,1.26942376E8,1.3E1,1.4695653E0,1.3617094E6,1E0,1E0,1.6975454E3,9.706078E4,3.175E1,3.5964913E0,1.674798E-2,-5.225047E-3,1.561132E2,2.2E1,1.932328E-2,4.586207E0,1.7233427E8,4.362075E8,9.369565E0,6.663214E6,5.49E4,1.4676277E7,5.76E2,5.332332E-2,9.55188E5,4.1665E3,4.217427E7,-3.306744E-2,1.295313E6,2.00087E5,2.796E2,3.39E2,2.3625128E6,2.5393645E5,1.9067797E1,1.7349593E-2,1.9160846E-2,2.923409E7,2.6224005E6,-2.8127907E-2,3.1027884E5,1.7412752E-2,5.735198E-1,5.658231E9,1.0284041E2,4.990618E6,1.6553931E-2,4.912383E-3,1.6533886E8,4.907764E3,-1.2536181E-2,-1.8564377E-2,8.564615E-5,-1.004572E-2,-5.5211447E-3,-1.6825108E-2,-9.755663E-3,2.2729031E-4,1.6403345E-2,2.4497325E-3,-9.197256E-3,-2.3656697E-3,-2.6022096E-3,-1.7594548E-2,-3.7960932E-3,8.680662E-3,-2.7857223E-3,-9.544619E-3,7.144421E-3,-2.7780195E-3,1.085959E-2,1.6681079E-3,-4.896239E-3,2.2087314E-3,1.9926776E-3,-2.671842E-2,-4.2528115E-4,1.8633649E-2,-7.1816356E-3,1.00022955E-2,2.788733E-2,5.004356E-3,5.902879E-3,1.6364692E-2,6.3950247E-3,-1.1421688E-2,3.679514E-3,1.3827699E-2],"split_indices":[40,41,4,4,64,3,70,43,0,65,57,89,27,64,45,67,65,0,0,4,3,0,65,12,7,66,40,9,59,8,69,43,59,57,0,9,5,67,41,57,40,70,0,0,9,40,0,45,0,50,43,70,57,0,0,5,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.032E3,4.25E2,6.07E2,1.18E2,3.07E2,4.66E2,1.41E2,1.12E2,6E0,2.67E2,4E1,8.5E1,3.81E2,1.16E2,2.5E1,8.4E1,2.8E1,2E0,4E0,6.9E1,1.98E2,9E0,3.1E1,5.5E1,3E1,1.2E2,2.61E2,6E1,5.6E1,7E0,1.8E1,4.7E1,3.7E1,2.4E1,4E0,1.8E1,5.1E1,1.69E2,2.9E1,2.4E1,7E0,4.7E1,8E0,1.7E1,1.3E1,1.17E2,3E0,2.5E2,1.1E1,3E1,3E1,3E1,2.6E1,5E0,2E0,1.3E1,5E0,2.9E1,1.8E1,1.5E1,2.2E1,4E0,2E1,1.6E1,2E0,4E0,4.7E1,8.8E1,8.1E1,5E0,2.4E1,6E0,1.8E1,4E0,3E0,3.5E1,1.2E1,1E1,3E0,9E1,2.7E1,2.48E2,2E0,4E0,2.6E1,6E0,2.4E1,2.8E1,2E0,8E0,1.8E1,2E0,1.1E1,3E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-9.87001E-4,-5.6432746E-2,2.093932E-1,-2.926968E-1,-3.13457E-2,3.6844987E-2,1.8180913E-1,-2.2981533E-1,-4.3659613E-1,-7.777934E-2,8.365608E-2,2.2919758E-1,-9.594494E-2,-2.9835427E-1,-9.49545E-2,-3.0152157E-1,-3.42898E-2,-1.0538704E-1,5.461894E-2,1.1700663E-1,-1.5301529E-1,3.7165135E-1,1.572229E-1,-2.4678826E-1,3.3922788E-2,-2.049461E-3,-3.1196746E-1,-1.9939214E-1,2.9422361E-2,-3.6780494E-1,-1.8097717E-1,-1.3607788E-1,6.0175776E-2,2.0173455E-2,2.970348E-1,3.4011588E-1,9.0445146E-2,-9.286011E-2,-2.115854E-2,1.1377922E-1,4.3788883E-1,9.746148E-2,2.4288411E-1,-4.200828E-3,-2.9977435E-1,1.6963547E-1,-1.055591E-1,-6.2651173E-3,-1.550346E-2,-1.1013027E-2,-5.416809E-4,3.452113E-3,-6.2440324E-4,-1.8562235E-2,-6.813542E-3,-1.301166E-3,-1.0432832E-2,-4.0262686E-3,-9.565886E-3,5.3998125E-3,-4.179214E-3,-1.0832291E-3,7.330432E-3,1.55623015E-2,3.3384126E-3,4.4630533E-3,1.667234E-2,1.2825155E-3,8.189689E-3,-5.840466E-3,1.1714328E-3,8.4674675E-3,-1.6536969E-3,3.700339E-4,2.102258E-2,1.8974826E-3,8.513693E-3,1.6988803E-2,4.8584505E-3,-7.504335E-3,-1.6694887E-2,1.4821604E-2,3.6656808E-3,8.7339315E-4,-7.855001E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1688169E1,4.6910696E0,3.2715693E0,5.8735704E-1,3.8405914E0,0E0,2.6676197E0,4.80047E-1,6.9594765E-1,1.8772249E0,1.6517797E0,1.7050943E0,5.9606206E-1,1.1042905E-1,2.656591E-1,6.926727E-2,0E0,2.1626434E0,7.371696E-1,1.04779E0,4.3838102E-1,9.2378235E-1,5.701597E-1,8.7807536E-2,3.3970743E-1,0E0,1.0657334E-1,8.049649E-2,2.069552E-2,1.9767404E-2,4.9937546E-2,1.2330403E0,5.9190416E-1,4.857157E-1,7.848275E-2,7.028198E-2,8.9588356E-1,9.66088E-2,0E0,1.4405036E-1,3.7938786E-1,3.37815E-1,7.791984E-1,0E0,2.7496636E-2,9.798339E-2,7.6639354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,45,45,46,46],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.812E3,7.7E1,2.4481E4,4.9558692E7,3.7625E2,3.6844987E-2,2.7555334E7,3.8714287E1,3.5E0,2.5247778E6,3.7568388E0,1.684E0,1.3655363E2,4E0,1.8987958E0,8.1643506E-8,-3.42898E-2,9.750085E6,2.4285715E0,3.45E2,4.9E1,1E0,1.6466942E3,3.714144E8,2.342E4,-2.049461E-3,2E0,5E0,5.62201E5,1.3293153E3,1.4E1,3.074398E0,2.5417458E1,2.7840833E2,7E0,3.8192307E2,1.8062708E6,1.672018E8,-2.115854E-2,1.5209424E1,1.84E2,6.2868685E6,8.710612E1,-4.200828E-3,5.111912E9,3.6440647E5,3.0161016E7,-6.2651173E-3,-1.550346E-2,-1.1013027E-2,-5.416809E-4,3.452113E-3,-6.2440324E-4,-1.8562235E-2,-6.813542E-3,-1.301166E-3,-1.0432832E-2,-4.0262686E-3,-9.565886E-3,5.3998125E-3,-4.179214E-3,-1.0832291E-3,7.330432E-3,1.55623015E-2,3.3384126E-3,4.4630533E-3,1.667234E-2,1.2825155E-3,8.189689E-3,-5.840466E-3,1.1714328E-3,8.4674675E-3,-1.6536969E-3,3.700339E-4,2.102258E-2,1.8974826E-3,8.513693E-3,1.6988803E-2,4.8584505E-3,-7.504335E-3,-1.6694887E-2,1.4821604E-2,3.6656808E-3,8.7339315E-4,-7.855001E-3],"split_indices":[2,41,12,43,64,0,57,67,65,40,65,66,68,0,65,49,0,9,65,41,3,27,64,7,9,0,29,6,43,4,3,68,70,64,8,64,40,5,0,68,0,57,70,0,43,40,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.92E2,2.08E2,7.5E1,7.17E2,8E0,2E2,5.4E1,2.1E1,5.11E2,2.06E2,1.71E2,2.9E1,3.5E1,1.9E1,1.6E1,5E0,4.23E2,8.8E1,1.81E2,2.5E1,5.6E1,1.15E2,1.3E1,1.6E1,2E0,3.3E1,1E1,9E0,9E0,7E0,3.57E2,6.6E1,7.8E1,1E1,1.8E1,1.63E2,2.2E1,3E0,1.2E1,4.4E1,6.9E1,4.6E1,4E0,9E0,8E0,8E0,5E0,2.8E1,8E0,2E0,4E0,5E0,7E0,2E0,2E0,5E0,2.15E2,1.42E2,4.8E1,1.8E1,6E1,1.8E1,8E0,2E0,2E0,1.6E1,9.6E1,6.7E1,1.7E1,5E0,8E0,4E0,2E0,4.2E1,4.3E1,2.6E1,2.3E1,2.3E1,4E0,5E0,2E0,6E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.6467074E-3,-1.1917745E-1,1.2072408E-1,-9.879404E-2,-4.5104274E-1,-3.218918E-1,1.4848639E-1,-2.1044339E-1,-4.5076624E-2,-7.1958214E-1,-2.942209E-1,-2.6487198E-2,-2.4478593E-1,8.314489E-2,3.0097893E-1,-2.9530573E-1,-1.6598873E-1,-1.0396842E-2,-1.804855E-1,-1.7239358E-3,-4.0667925E-2,-7.104565E-2,-3.4557435E-1,3.7192337E-3,-2.7808103E-1,1.5283053E-1,1.2509989E-2,3.9076995E-2,2.7370846E-1,-2.304162E-1,-3.7331802E-1,-9.270048E-2,-2.2479308E-1,5.6126118E-2,-7.891742E-2,-2.709232E-1,1.450642E-1,-7.3800134E-3,1.941285E-3,-1.5085988E-1,-3.9790753E-1,-2.9917774E-1,-4.9767448E-3,1.7531992E-1,-7.925361E-2,2.8704232E-2,-2.6418215E-1,3.4375855E-1,1.5932836E-1,-1.3319999E-2,-5.769581E-3,-3.3877841E-3,-1.8353615E-2,-4.3801212E-4,-5.8178795E-3,-1.1798375E-2,-2.2208306E-3,3.9437483E-3,-6.8931854E-3,-5.81139E-3,-5.2074203E-4,-3.549888E-3,-1.6298218E-2,7.228041E-4,8.087844E-3,-2.2433961E-3,-9.305317E-3,-1.9904712E-2,-6.7569916E-3,-1.4421325E-2,-5.3099063E-3,2.6494265E-3,1.1312877E-2,-2.615105E-2,1.8345801E-3,4.55073E-3,-4.401966E-4,-1.7547742E-2,-9.805536E-4,-3.5969822E-3,1.6581107E-2,5.4022064E-4,1.0890651E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,37,39,-1,41,43,45,-1,47,49,51,53,55,57,59,61,63,-1,-1,65,67,69,-1,71,73,75,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4267277E1,3.3783007E0,6.019784E0,2.857326E0,1.0095568E0,4.2853498E-1,4.5385284E0,5.3572464E-1,1.5196118E0,1.0977478E0,2.0466483E-1,0E0,2.828256E-1,1.583049E0,1.8211727E0,1.7721462E-1,4.2736673E-1,1.185001E0,1.9824705E0,0E0,0E0,5.7570063E-2,1.03012204E-1,0E0,4.6360254E-2,8.594427E-1,7.2899514E-1,0E0,1.0028582E0,1.5714467E-1,1.4364433E-1,1.3437113E-1,3.1017113E-1,8.222384E-1,4.1036427E-1,8.175144E-1,5.7406217E-2,0E0,0E0,1.6492464E-2,4.293382E-2,1.5311599E-2,0E0,1.2215605E0,8.98995E-1,4.131937E-1,2.4942565E-1,6.175804E-1,5.940372E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,38,40,-1,42,44,46,-1,48,50,52,54,56,58,60,62,64,-1,-1,66,68,70,-1,72,74,76,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.177E3,4.4895835E0,4.8465605E3,3.95E2,2E0,1.0479E4,5.526E3,3.89E2,6.33E3,1.040054E6,1.9E1,-2.6487198E-2,5.1E1,3.875E0,3.3817584E7,9.55188E5,9.263158E0,1E0,3.2911258E6,-1.7239358E-3,-4.0667925E-2,1.3E1,3.3817584E7,3.7192337E-3,2.7664675E5,1.9269184E7,1.0428572E1,3.9076995E-2,6.6933334E1,1.1634076E4,1.6666666E0,1.31E2,2.0399521E2,1.1203416E1,1E0,9.5596474E-1,1.288E3,-7.3800134E-3,1.941285E-3,8.2E1,7.69E2,1.665909E3,-4.9767448E-3,4.2010452E2,1E0,3.39498E5,2.402121E4,1.3306123E1,1.8664102E2,-1.3319999E-2,-5.769581E-3,-3.3877841E-3,-1.8353615E-2,-4.3801212E-4,-5.8178795E-3,-1.1798375E-2,-2.2208306E-3,3.9437483E-3,-6.8931854E-3,-5.81139E-3,-5.2074203E-4,-3.549888E-3,-1.6298218E-2,7.228041E-4,8.087844E-3,-2.2433961E-3,-9.305317E-3,-1.9904712E-2,-6.7569916E-3,-1.4421325E-2,-5.3099063E-3,2.6494265E-3,1.1312877E-2,-2.615105E-2,1.8345801E-3,4.55073E-3,-4.401966E-4,-1.7547742E-2,-9.805536E-4,-3.5969822E-3,1.6581107E-2,5.4022064E-4,1.0890651E-2],"split_indices":[2,65,40,2,29,9,2,1,41,5,3,0,0,66,7,43,70,24,40,0,0,3,7,0,45,59,65,0,68,40,66,41,4,68,25,65,12,0,0,0,2,4,0,64,8,9,45,70,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E2,5.05E2,4.85E2,4.77E2,2.8E1,2.8E1,4.57E2,1.54E2,3.23E2,9E0,1.9E1,5E0,2.3E1,3.21E2,1.36E2,5.1E1,1.03E2,2.58E2,6.5E1,2E0,7E0,4E0,1.5E1,2E0,2.1E1,1.61E2,1.6E2,5E0,1.31E2,3E1,2.1E1,4.7E1,5.6E1,1.31E2,1.27E2,5.1E1,1.4E1,2E0,2E0,4E0,1.1E1,1.8E1,3E0,1.47E2,1.4E1,1.52E2,8E0,8E1,5.1E1,1.8E1,1.2E1,2E0,1.9E1,1.4E1,3.3E1,4.7E1,9E0,1.15E2,1.6E1,7.4E1,5.3E1,1.6E1,3.5E1,3E0,1.1E1,2E0,2E0,9E0,2E0,1.6E1,2E0,5.6E1,9.1E1,2E0,1.2E1,5.3E1,9.9E1,5E0,3E0,3E0,7.7E1,1.8E1,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[-1.0853218E-2,-8.1350096E-2,2.0764473E-1,-2.0789105E-1,-2.101585E-2,5.3601277E-1,1.7416842E-1,-1.7752245E-1,-4.656043E-1,-6.4137E-2,6.360313E-2,1.8552104E-2,3.1323142E-2,7.96028E-2,3.020918E-1,-1.9208246E-1,2.3050818E-1,-7.2556853E-1,-2.7402636E-1,-1.1139452E-1,3.797302E-2,2.8404826E-2,2.4486561E-1,-2.8037423E-1,1.02555364E-1,3.32212E-1,-6.3226916E-2,8.479962E-2,-2.0477575E-1,1.5946064E-2,1.2426128E-3,-8.0683637E-1,-1.2263637E-2,-1.69129E-1,-2.5098663E-2,-1.0408015E-1,-2.9167973E-2,6.048918E-2,-2.5104013E-1,1.0175382E-2,1.661356E-1,3.102309E-1,7.240207E-2,-1.8281579E-2,2.2217676E-3,8.730039E-2,2.4378264E-2,3.558715E-1,-1.2697783E-1,3.581789E-3,-7.512008E-3,1.7044723E-2,-4.1755876E-3,-1.0788751E-2,-4.309756E-3,-1.535838E-2,-4.1733135E-2,-9.855492E-3,2.473925E-3,-2.4099303E-3,-8.006171E-3,4.0869326E-3,-4.996574E-3,-1.8812425E-3,-1.6944066E-2,1.0847375E-3,-1.2472324E-2,2.447021E-3,1.3156811E-2,3.7142853E-3,1.665102E-2,5.8680965E-4,5.3991606E-3,-2.03692E-3,6.3537965E-3,6.9071855E-3,1.9039871E-2,1.1793171E-3,-1.0899797E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,-1,57,-1,59,-1,61,63,65,67,69,71,-1,-1,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5343299E1,5.744601E0,2.5778332E0,1.8191576E0,1.8686504E0,1.4870071E-1,2.661521E0,1.3453398E0,1.0462604E0,1.6400632E0,1.0968075E0,0E0,0E0,1.0946031E0,1.0553312E0,7.624483E-1,1.8049121E-1,1.5724659E-1,3.6478305E-1,8.224828E-1,7.1799165E-1,3.648798E-1,2.931863E-1,3.2865137E-1,7.5058484E-1,9.903736E-1,1.2608258E-1,5.367714E-1,6.561394E-1,0E0,0E0,8.080101E-2,0E0,1.43282E-1,0E0,8.245003E-1,0E0,4.967515E-1,1.8297023E-1,4.986657E-1,2.1556592E-1,2.1054208E-1,2.2725206E-2,0E0,0E0,8.055948E-1,0E0,9.380808E-1,9.030219E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,45,45,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,-1,58,-1,60,-1,62,64,66,68,70,72,-1,-1,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.972052E5,5.6E1,4.6463413E0,1.7831801E6,1.84146E5,2.6224005E6,2.62E3,2E0,4.39776E6,4.6547827E2,1.8552104E-2,3.1323142E-2,7.209302E-1,1.0424884E-7,7E0,4.88451E5,1.2E1,3.8304348E1,1.7587205E6,1E0,7.4711205E6,1E0,5.53648E8,1.0534078E4,6.292039E0,2.748204E-1,1.3050649E4,1.339646E6,1.5946064E-2,1.2426128E-3,1.16408E5,-1.2263637E-2,2.347364E7,-2.5098663E-2,2.1851852E0,-2.9167973E-2,2.7586578E1,2.9690266E4,6.5590655E6,1E0,7.176E3,2.9E1,-1.8281579E-2,2.2217676E-3,4.0449125E5,2.4378264E-2,7.8653125E2,3.01E2,3.581789E-3,-7.512008E-3,1.7044723E-2,-4.1755876E-3,-1.0788751E-2,-4.309756E-3,-1.535838E-2,-4.1733135E-2,-9.855492E-3,2.473925E-3,-2.4099303E-3,-8.006171E-3,4.0869326E-3,-4.996574E-3,-1.8812425E-3,-1.6944066E-2,1.0847375E-3,-1.2472324E-2,2.447021E-3,1.3156811E-2,3.7142853E-3,1.665102E-2,5.8680965E-4,5.3991606E-3,-2.03692E-3,6.3537965E-3,6.9071855E-3,1.9039871E-2,1.1793171E-3,-1.0899797E-2],"split_indices":[2,40,0,65,40,1,40,2,29,9,64,0,0,65,49,3,9,8,65,40,82,40,6,7,4,50,50,40,9,0,0,7,0,5,0,65,0,70,45,40,27,12,3,0,0,40,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.94E2,7.52E2,2.42E2,2.42E2,5.1E2,2.1E1,2.21E2,2.18E2,2.4E1,3.38E2,1.72E2,1.3E1,8E0,1.28E2,9.3E1,2.11E2,7E0,9E0,1.5E1,2.31E2,1.07E2,1.45E2,2.7E1,7E0,1.21E2,8.6E1,7E0,9E0,2.02E2,4E0,3E0,7E0,2E0,1.2E1,3E0,2.29E2,2E0,1E2,7E0,1.29E2,1.6E1,1.9E1,8E0,5E0,2E0,1.18E2,3E0,8.2E1,4E0,3E0,4E0,3E0,6E0,1.58E2,4.4E1,2E0,5E0,1E1,2E0,1.33E2,9.6E1,8.6E1,1.4E1,3E0,4E0,1.24E2,5E0,9E0,7E0,4E0,1.5E1,4E0,4E0,3.3E1,8.5E1,1.9E1,6.3E1,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-4.8688782E-4,-1.0547561E-1,9.343118E-2,-1.9421673E-1,-5.8021553E-2,-2.788391E-1,1.1696687E-1,-1.38956765E-2,-1.7534012E-1,3.47279E-2,-1.18605696E-1,-1.9358253E-2,-1.7727274E-1,5.9212487E-2,2.582629E-1,-1.0371243E-1,-2.3492725E-1,3.1545025E-1,1.588074E-2,-7.662181E-2,-2.9581738E-1,-2.1928121E-1,1.010631E-3,1.649698E-2,1.385553E-1,5.184513E-1,2.1188764E-1,-1.7636906E-1,-5.567891E-2,-2.627806E-1,-7.116146E-2,4.9222433E-3,1.6857702E-2,-1.7516652E-1,4.049908E-2,-5.9666768E-2,-1.7876094E-2,-4.7227725E-1,-6.0436487E-2,-2.0283826E-3,-2.396865E-1,4.07141E-3,-3.9940435E-3,9.799249E-2,-3.1016724E-2,1.5375133E-1,-1.721323E-2,2.6102057E-2,1.0810978E-2,1.6844583E-1,4.704208E-1,-2.1712347E-3,-9.691703E-3,-1.5036745E-3,-1.026514E-2,-9.784053E-3,-1.4986014E-2,2.3003132E-3,-5.999199E-3,-6.065801E-4,-1.5398104E-2,2.5475055E-3,-7.0497543E-3,1.3013745E-3,-4.295712E-3,-1.0083735E-2,-2.9558633E-2,6.637664E-3,-1.0029231E-2,-1.3644288E-2,-6.99791E-3,5.362925E-3,-6.362648E-3,-8.7194087E-4,-1.4820602E-2,-7.5675705E-3,7.603725E-3,2.2363044E-2,5.9226234E-3,4.7576185E-3,2.3415998E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,-1,59,61,63,-1,65,67,-1,69,-1,-1,71,73,75,-1,-1,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.02744665E1,2.0606055E0,4.851517E0,2.9657936E-1,1.813484E0,4.0330362E-1,4.2118278E0,0E0,6.073122E-1,6.7221534E-1,1.4281409E0,0E0,1.5749931E-1,1.2483174E0,1.7000914E0,2.3098671E-1,3.462143E-1,4.6594977E-2,5.761893E-1,8.194156E-1,1.5082631E0,5.0162613E-2,4.6250742E-2,9.362283E-1,1.0638618E0,1.7657423E-1,1.3695779E0,1.1074793E-1,1.5822534E-1,1.4535236E-1,1.0024415E-1,0E0,0E0,3.5853112E-1,3.2201123E-1,4.555254E-1,0E0,7.676811E-1,5.8175087E-1,0E0,2.9895484E-2,0E0,0E0,4.11211E-1,5.30828E-1,5.005684E-1,0E0,0E0,0E0,1.3460071E0,2.072339E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,37,37,38,38,40,40,43,43,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,-1,60,62,64,-1,66,68,-1,70,-1,-1,72,74,76,-1,-1,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.99E2,3.89E2,3.964269E3,3.2E1,1.4615384E0,2.5330253E-2,1.1424088E3,-1.38956765E-2,4.0869565E0,4.9069305E1,2.1E1,-1.9358253E-2,2.0376764E-7,2.0766992E6,1.307E3,3.89E2,1.901875E2,3.2226672E7,5.764631E1,2.2715E4,1E0,5.1917776E-2,1.06314E5,1.75E0,1.4676277E7,3.4583332E1,6.0052995E6,1.93E2,7.130314E7,1E0,1.3597637E8,4.9222433E-3,1.6857702E-2,1E0,1E0,6.8808866E0,-1.7876094E-2,6.6E1,2E0,-2.0283826E-3,7.0296685E6,4.07141E-3,-3.9940435E-3,1.4569893E0,2.673585E1,1.4806E4,-1.721323E-2,2.6102057E-2,1.0810978E-2,1.2083E4,4E0,-2.1712347E-3,-9.691703E-3,-1.5036745E-3,-1.026514E-2,-9.784053E-3,-1.4986014E-2,2.3003132E-3,-5.999199E-3,-6.065801E-4,-1.5398104E-2,2.5475055E-3,-7.0497543E-3,1.3013745E-3,-4.295712E-3,-1.0083735E-2,-2.9558633E-2,6.637664E-3,-1.0029231E-2,-1.3644288E-2,-6.99791E-3,5.362925E-3,-6.362648E-3,-8.7194087E-4,-1.4820602E-2,-7.5675705E-3,7.603725E-3,2.2363044E-2,5.9226234E-3,4.7576185E-3,2.3415998E-2],"split_indices":[2,2,40,2,65,50,67,0,70,59,3,0,49,40,41,1,4,7,59,41,27,50,1,65,59,70,40,1,7,24,43,0,0,8,82,70,0,0,29,0,59,0,0,58,65,1,0,0,0,9,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,4.91E2,5.49E2,1.7E2,3.21E2,3.2E1,5.17E2,2.3E1,1.47E2,1.27E2,1.94E2,1.2E1,2E1,3.68E2,1.49E2,6.8E1,7.9E1,7E0,1.2E2,1.58E2,3.6E1,1.6E1,4E0,2.4E2,1.28E2,2.1E1,1.28E2,2.6E1,4.2E1,6.7E1,1.2E1,2E0,5E0,1.3E1,1.07E2,1.51E2,7E0,2E1,1.6E1,2E0,1.4E1,2E0,2E0,8.8E1,1.52E2,1.25E2,3E0,1.7E1,4E0,1.11E2,1.7E1,6E0,2E1,3.8E1,4E0,4E1,2.7E1,4E0,8E0,7E0,6E0,1E2,7E0,4.2E1,1.09E2,9E0,1.1E1,7E0,9E0,7E0,7E0,8.2E1,6E0,1.47E2,5E0,4E0,1.21E2,1.1E1,1E2,2E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[5.3126547E-3,-1.3955046E-1,5.998988E-2,-2.2197002E-1,-9.294573E-2,1.2323078E-3,1.7749976E-1,-1.3768007E-1,-2.982896E-1,-1.1734083E-1,2.400678E-1,1.4032327E-1,-2.2704123E-2,1.3294521E-1,2.9895842E-1,-1.5331903E-1,6.5589226E-3,-3.8759074E-1,-1.4942339E-1,8.992636E-2,-1.4930153E-1,2.0111548E-2,3.6236983E-2,9.093065E-2,1.3139075E-2,-7.6604284E-2,1.852979E-2,1.0651918E-1,5.5303776E-1,3.3683503E-1,-2.0020172E-1,3.5879088E-3,-1.6589998E-1,-2.4004443E-1,-4.4936165E-1,-1.9375901E-1,3.507279E-3,-1.11888036E-1,1.7259976E-1,-1.7506815E-1,-1.8541995E-2,8.020063E-3,-5.107327E-3,1.6372412E-1,-2.7789123E-2,-6.7121185E-2,-2.1259677E-2,6.696888E-2,-5.2505117E-2,1.1682618E-1,-1.1235254E-2,2.781165E-2,1.0508528E-2,2.4157214E-1,4.1201326E-1,9.6184615E-4,-1.6287806E-2,-8.028792E-3,5.062074E-4,-1.2928525E-2,-1.641669E-3,-2.6262848E-2,-1.4702659E-2,3.2142166E-3,-1.0729115E-2,7.1702636E-4,-7.625083E-3,2.6251008E-3,1.6328048E-2,-6.744163E-3,-1.700891E-2,6.6681327E-3,-3.7287131E-3,8.735689E-3,-7.547663E-3,-5.2906005E-3,5.9080077E-3,1.7608702E-6,-4.6253656E-3,4.1588657E-3,-4.2539528E-3,-6.026158E-3,1.4124557E-3,6.4581027E-3,-1.091622E-3,5.686422E-3,1.609897E-2,2.1374695E-2,7.469889E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,-1,41,43,-1,45,47,49,51,53,55,-1,57,59,61,63,-1,65,67,69,71,-1,-1,73,75,77,-1,79,81,83,-1,-1,-1,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.659781E0,9.9727106E-1,4.8468075E0,5.7006407E-1,1.4173566E0,1.5646471E0,1.2265544E0,2.2805351E-1,5.9348726E-1,1.0733309E0,4.6792495E-1,4.7066486E-1,8.926478E-1,1.8616297E0,1.2396374E0,1.4398348E-1,0E0,1.6555548E-1,2.1499908E-1,3.8637483E-1,4.6507692E-1,0E0,1.6221812E-1,4.6257702E-1,0E0,6.062702E-1,7.8803384E-1,6.2210476E-1,3.1585455E-2,3.158474E-1,1.7815754E-1,0E0,6.9627285E-2,8.3045065E-2,1.3346052E-1,1.9492638E-1,0E0,5.089467E-2,3.1750366E-1,5.8514977E-1,2.5790286E-1,0E0,0E0,3.1775397E-1,3.0188543E-1,3.8564712E-1,0E0,5.2006507E-1,6.1334693E-1,5.381911E-1,0E0,0E0,0E0,3.1580114E-1,3.367319E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,49,49,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,-1,42,44,-1,46,48,50,52,54,56,-1,58,60,62,64,-1,66,68,70,72,-1,-1,74,76,78,-1,80,82,84,-1,-1,-1,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.16577E5,7.7E1,5.862504E2,1.039795E6,6.68411E2,7E0,3.1993368E6,8.75477E5,5E0,8.650138E-1,1E0,2.4285715E0,2.778995E2,7.6255713E3,3.0427273E3,5E0,6.5589226E-3,5.797E3,1E0,4.54E2,3.541824E2,2.0111548E-2,5.884866E7,4.2782607E0,1.3139075E-2,6.0692043E0,2.1340206E0,2.921332E9,1E0,1.1E1,3.01E2,3.5879088E-3,1.1408248E0,4.2619047E0,3.343E3,1.3439851E7,3.507279E-3,1.05E2,1.8029197E0,1.597E4,5.281628E2,8.020063E-3,-5.107327E-3,1E0,2.6463525E8,4.631579E0,-2.1259677E-2,6.1114804E-8,6.946813E7,1E0,-1.1235254E-2,2.781165E-2,1.0508528E-2,6.149315E6,2.0295714E7,9.6184615E-4,-1.6287806E-2,-8.028792E-3,5.062074E-4,-1.2928525E-2,-1.641669E-3,-2.6262848E-2,-1.4702659E-2,3.2142166E-3,-1.0729115E-2,7.1702636E-4,-7.625083E-3,2.6251008E-3,1.6328048E-2,-6.744163E-3,-1.700891E-2,6.6681327E-3,-3.7287131E-3,8.735689E-3,-7.547663E-3,-5.2906005E-3,5.9080077E-3,1.7608702E-6,-4.6253656E-3,4.1588657E-3,-4.2539528E-3,-6.026158E-3,1.4124557E-3,6.4581027E-3,-1.091622E-3,5.686422E-3,1.609897E-2,2.1374695E-2,7.469889E-3],"split_indices":[40,41,64,43,64,3,40,43,6,65,27,65,67,64,4,0,0,9,16,1,64,0,5,68,0,65,65,7,28,3,0,0,50,66,12,57,0,41,66,41,4,0,0,82,7,70,0,49,7,6,0,0,0,57,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,2.64E2,7.01E2,9.4E1,1.7E2,4.68E2,2.33E2,4.6E1,4.8E1,1.59E2,1.1E1,6.8E1,4E2,1.72E2,6.1E1,4.4E1,2E0,2.9E1,1.9E1,2.1E1,1.38E2,5E0,6E0,5.2E1,1.6E1,1.73E2,2.27E2,1.63E2,9E0,5.7E1,4E0,2E0,4.2E1,1E1,1.9E1,1.6E1,3E0,6E0,1.5E1,1.15E2,2.3E1,3E0,3E0,3.2E1,2E1,1.7E2,3E0,1.35E2,9.2E1,1.59E2,4E0,7E0,2E0,2.7E1,3E1,2E0,2E0,4E1,2E0,8E0,2E0,8E0,1.1E1,2E0,1.4E1,2E0,4E0,1E1,5E0,1.02E2,1.3E1,6E0,1.7E1,3E1,2E0,1.3E1,7E0,5.7E1,1.13E2,1.18E2,1.7E1,4.7E1,4.5E1,1.36E2,2.3E1,1.4E1,1.3E1,2.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-4.766309E-3,-9.36316E-2,8.3933435E-2,-7.0610605E-2,-2.9370743E-1,1.2523177E-1,-1.4885637E-1,-1.9042219E-1,-2.6276968E-2,-5.3576696E-1,-1.12448804E-1,3.5987087E-2,2.0794117E-1,-3.0728334E-1,-4.673835E-2,-1.6899321E-2,-1.7978293E-1,1.52188735E-2,-3.4945603E-2,-2.5828856E-1,-4.169989E-2,-1.9363923E-1,5.277371E-2,1.450209E-1,-7.866356E-3,3.0044425E-1,7.916918E-2,1.6326712E-3,-3.4669197E-1,2.737144E-2,-1.612939E-1,-1.2877597E-1,-2.1939203E-1,-5.959168E-2,4.7235407E-2,-3.3381203E-1,6.4180995E-4,-2.2019602E-1,1.795934E-3,1.654174E-1,-5.304276E-3,2.874069E-1,1.07191265E-1,1.7615424E-2,-2.0826001E-1,3.8250038E-1,1.700043E-1,3.5120535E-1,5.011751E-2,-2.4082638E-2,-2.7097487E-1,-1.9279674E-2,1.22601636E-1,-2.071011E-1,6.983662E-3,-7.852504E-3,-1.9322806E-3,-1.0699948E-2,-4.6184222E-4,-4.5466474E-3,8.085043E-4,3.2623832E-3,-7.5269016E-3,-1.0532663E-2,-2.462119E-2,-4.2217583E-3,-1.1916743E-2,2.5317664E-3,9.117968E-3,2.233579E-3,1.5171178E-2,1.1369486E-2,3.5784212E-3,1.9690506E-3,-8.52441E-3,-2.625758E-2,-2.4099294E-3,1.9573864E-2,1.1099041E-2,-1.985517E-3,1.5644742E-2,7.367719E-3,2.5244083E-2,4.0464127E-3,-3.7442131E-3,-1.3756652E-2,1.3939626E-6,-4.8095547E-3,2.4582578E-3,9.0243295E-4,7.529014E-3,-1.1494395E-2,3.5679513E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,-1,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,-1,65,-1,67,-1,69,71,73,75,77,79,81,83,-1,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.00054E0,2.3144746E0,4.902468E0,2.4171238E0,2.221274E0,3.181271E0,1.2313634E0,1.661377E-1,1.0469905E0,2.0370212E0,4.2606816E-1,1.000476E0,2.6444588E0,4.1763616E-1,4.119199E-1,0E0,2.1014786E-1,0E0,6.651335E-1,2.9253453E-1,0E0,1.384067E-1,2.2758538E-1,2.923771E-1,7.696267E-1,1.3264151E0,7.3319334E-1,0E0,2.2105098E-1,1.3564157E-1,3.0419916E-1,1.9032633E-1,1.8219686E-1,7.6810485E-1,3.9335698E-1,1.0954118E-1,0E0,7.181871E-2,0E0,1.4609665E-2,0E0,1.0684073E-1,1.8208832E-1,6.9560367E-1,9.3247885E-1,3.827772E-1,1.9064107E0,2.5252545E-1,4.4212624E-1,0E0,1.6236758E-1,1.3621157E-1,3.931655E-2,2.2833759E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,-1,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,-1,66,-1,68,-1,70,72,74,76,78,80,82,84,-1,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,3.5E0,3.0923106E-10,3.51E2,1E0,5.862504E2,7.295292E-2,6E0,1.2E1,6.98E2,6.7723384E7,2.3316922E0,8E0,8.620714E2,1.2E1,-1.6899321E-2,1.4452E4,1.52188735E-2,4.39776E6,8.4683485E-2,-4.169989E-2,8.75E2,5.05E2,1.0622318E0,1.3997573E7,2.1298597E0,1.2083E4,1.6326712E-3,4.5E0,1.4127854E7,6.2601447E2,4.54E2,1.901875E2,1.5859042E6,1.2E1,2.3E1,6.4180995E-4,6.0147805E0,1.795934E-3,1.30175E3,-5.304276E-3,2.3525E2,2.5084135E6,4.7220547E1,5.1942E4,1.2820834E7,2.9251662E6,2.5134058E3,4.1831533E8,-2.4082638E-2,1.6322028E3,9.015703E-7,8.7390656E2,3.2462872E5,6.983662E-3,-7.852504E-3,-1.9322806E-3,-1.0699948E-2,-4.6184222E-4,-4.5466474E-3,8.085043E-4,3.2623832E-3,-7.5269016E-3,-1.0532663E-2,-2.462119E-2,-4.2217583E-3,-1.1916743E-2,2.5317664E-3,9.117968E-3,2.233579E-3,1.5171178E-2,1.1369486E-2,3.5784212E-3,1.9690506E-3,-8.52441E-3,-2.625758E-2,-2.4099294E-3,1.9573864E-2,1.1099041E-2,-1.985517E-3,1.5644742E-2,7.367719E-3,2.5244083E-2,4.0464127E-3,-3.7442131E-3,-1.3756652E-2,1.3939626E-6,-4.8095547E-3,2.4582578E-3,9.0243295E-4,7.529014E-3,-1.1494395E-2,3.5679513E-3],"split_indices":[2,65,49,2,27,64,50,64,10,41,7,66,29,4,8,0,9,0,9,54,0,2,10,66,57,50,9,0,70,59,58,1,4,40,68,0,0,66,0,59,0,64,57,70,1,9,40,4,44,0,4,49,4,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.013E3,5.06E2,5.07E2,4.55E2,5.1E1,4.31E2,7.6E1,1.22E2,3.33E2,2.1E1,3E1,2.08E2,2.23E2,2.9E1,4.7E1,5E0,1.17E2,7E0,3.26E2,1.3E1,8E0,2E1,1E1,5.9E1,1.49E2,1.29E2,9.4E1,3E0,2.6E1,2.9E1,1.8E1,5.3E1,6.4E1,2.51E2,7.5E1,1E1,3E0,1.8E1,2E0,6E0,4E0,1.1E1,4.8E1,1.33E2,1.6E1,7.8E1,5.1E1,8E0,8.6E1,6E0,2E1,2E1,9E0,1.6E1,2E0,3.5E1,1.8E1,6E1,4E0,1.66E2,8.5E1,6.8E1,7E0,8E0,2E0,5E0,1.3E1,2E0,4E0,2E0,9E0,7E0,4.1E1,1.19E2,1.4E1,4E0,1.2E1,5.8E1,2E1,2.3E1,2.8E1,5E0,3E0,6.7E1,1.9E1,1.8E1,2E0,9E0,1.1E1,3E0,6E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[9.376501E-3,-4.505828E-2,1.5736116E-1,-1.3953927E-1,-3.2101441E-3,1.897763E-1,-8.821433E-2,-1.1239754E-1,-2.8746906E-1,-7.23948E-2,4.5806803E-2,1.4406335E-1,3.6469984E-1,-2.189352E-1,-5.371089E-3,-1.7268616E-1,-2.1671746E-2,-2.9206114E-2,-2.3143439E-1,-4.7959335E-2,-2.830487E-1,5.071325E-3,1.3116537E-1,2.367041E-1,7.668001E-2,5.285793E-1,1.8562455E-1,2.8904763E-3,-2.8160128E-1,5.995489E-2,-7.706894E-3,-4.2586487E-2,-1.9576177E-1,1.7063707E-2,-6.1244953E-2,-2.5015202E-1,9.373912E-4,-4.328656E-4,-1.3094686E-1,-9.150255E-2,-4.4886312E-1,-1.0575393E-2,2.2614247E-1,2.2221084E-1,5.1829673E-2,3.1449175E-1,8.9343816E-2,-2.244283E-2,1.3531761E-1,3.1223682E-1,2.835059E-2,2.5215524E-1,-5.599338E-2,-1.5858907E-2,-6.838075E-3,1.7939163E-2,1.0198552E-2,-6.7439596E-3,8.743723E-3,-9.58088E-3,1.247345E-3,-5.5608856E-3,1.1687621E-3,-1.4119033E-2,-5.345474E-3,-3.6132992E-3,3.5139755E-3,-7.1329754E-3,1.338319E-2,5.627031E-3,-7.012498E-3,-2.9429294E-2,-8.368498E-3,-1.3020573E-3,7.6099234E-3,2.7973012E-3,1.1512732E-2,1.3941607E-2,3.032122E-3,1.8592432E-7,6.7447303E-3,1.8898424E-2,1.1349518E-2,9.980365E-3,2.8442132E-4,6.2158303E-3,-3.5689783E-3,-1.7367965E-2,7.193144E-3,5.491792E-3,1.7851086E-2,4.7033485E-3,1.31026E-2,3.212042E-3,-9.426448E-3,-3.3486462E-3,1.8870541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,-1,53,55,-1,57,59,-1,61,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,-1,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.788716E0,2.7949262E0,2.0864348E0,8.389602E-1,1.6684055E0,1.786602E0,3.3453366E-1,1.0081582E0,4.970677E-1,1.0365002E0,1.0007459E0,1.1304591E0,1.2811804E0,2.2968185E-1,2.2226447E-1,3.2489944E-1,1.1896915E0,0E0,1.4652252E-1,7.256583E-1,6.2591124E-1,6.8036306E-1,6.6023636E-1,8.545389E-1,6.3076794E-1,2.6759958E-1,3.999192E-1,0E0,3.310591E-2,9.816746E-2,0E0,4.6503982E-1,2.744801E-1,0E0,3.6092752E-1,1.7862964E-1,0E0,7.1627706E-1,7.342111E-1,1.627064E-1,4.4604945E-1,5.8107156E-1,3.737998E-2,5.261409E-1,2.5113004E-1,2.2505188E-1,2.9653913E-1,3.6796603E-1,8.024758E-1,8.602804E-2,0E0,6.499481E-2,1.2710567E-1,0E0,0E0,3.0328793E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,31,31,32,32,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,-1,54,56,-1,58,60,-1,62,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,-1,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,1.8988005E5,2.856934E7,2.670362E7,5.036081E7,1.4176E4,1.3655363E2,1.3823239E2,6.27907E0,3.5531914E0,4.518317E2,3.66025E5,5.4123135E1,2.459E3,3.0070068E7,2.2007043E0,2.04115E5,-2.9206114E-2,4.1942386E-8,1E0,3E0,4.5866325E6,3.5419354E0,1.465E4,8.484036E5,1.4194314E3,3.7481603E8,2.8904763E-3,1.8661781E5,4.435876E2,-7.706894E-3,1.2E1,1.317484E1,1.7063707E-2,1.212945E6,1.921E3,9.373912E-4,1E0,3.2E1,2.3636363E0,2E0,1.5282027E5,1.81E2,1.8729467E8,5.603014E6,2.124E3,4.0663E4,1.138871E0,3.72424E5,3.8807898E6,2.835059E-2,4.5271382E8,8E0,-1.5858907E-2,-6.838075E-3,2.5393645E5,1.0198552E-2,-6.7439596E-3,8.743723E-3,-9.58088E-3,1.247345E-3,-5.5608856E-3,1.1687621E-3,-1.4119033E-2,-5.345474E-3,-3.6132992E-3,3.5139755E-3,-7.1329754E-3,1.338319E-2,5.627031E-3,-7.012498E-3,-2.9429294E-2,-8.368498E-3,-1.3020573E-3,7.6099234E-3,2.7973012E-3,1.1512732E-2,1.3941607E-2,3.032122E-3,1.8592432E-7,6.7447303E-3,1.8898424E-2,1.1349518E-2,9.980365E-3,2.8442132E-4,6.2158303E-3,-3.5689783E-3,-1.7367965E-2,7.193144E-3,5.491792E-3,1.7851086E-2,4.7033485E-3,1.31026E-2,3.212042E-3,-9.426448E-3,-3.3486462E-3,1.8870541E-3],"split_indices":[2,40,57,5,7,2,68,64,68,65,67,9,70,10,55,66,5,0,49,24,8,40,66,41,40,64,44,0,45,68,0,3,66,0,9,2,0,27,3,70,29,45,0,7,9,41,9,50,9,40,0,5,3,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,7.06E2,2.59E2,2.16E2,4.9E2,2.29E2,3E1,1.84E2,3.2E1,2.03E2,2.87E2,1.83E2,4.6E1,1.1E1,1.9E1,1.1E2,7.4E1,3E0,2.9E1,1.83E2,2E1,1.95E2,9.2E1,7.6E1,1.07E2,2.3E1,2.3E1,2E0,9E0,1.4E1,5E0,1.7E1,9.3E1,6E0,6.8E1,2.7E1,2E0,1.17E2,6.6E1,1E1,1E1,1.83E2,1.2E1,4.2E1,5E1,4.9E1,2.7E1,4E1,6.7E1,8E0,1.5E1,1.8E1,5E0,5E0,4E0,1.2E1,2E0,1.2E1,5E0,8.8E1,5E0,4E1,2.8E1,1.8E1,9E0,5.8E1,5.9E1,6.3E1,3E0,2E0,8E0,5E0,5E0,1.67E2,1.6E1,2E0,1E1,2.7E1,1.5E1,3.3E1,1.7E1,1.8E1,3.1E1,1E1,1.7E1,1E1,3E1,2E0,6.5E1,3E0,5E0,4E0,1.4E1,3E0,2E0,2E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[3.8320425E-3,-8.353085E-2,9.420578E-2,-1.7517012E-1,-2.7006958E-2,1.3105617E-1,-1.5427896E-1,-1.6270888E-1,-4.4455552E-1,4.1608732E-2,-7.7754825E-2,1.9177784E-1,4.0093623E-2,-2.83924E-1,-4.5557145E-2,-2.4233685E-1,-1.3651331E-1,-3.2776635E-2,1.3254651E-4,-5.0948128E-2,7.205309E-2,-1.1017207E-1,5.226336E-2,9.871396E-2,2.664513E-1,2.501764E-1,1.856566E-2,-5.6375477E-2,-3.1807676E-1,-1.01017095E-1,7.3780835E-2,-2.8116424E-3,-2.5381675E-1,-1.1415182E-1,-2.3749346E-1,5.619181E-2,-1.1194227E-1,1.0282579E-1,-5.1538065E-2,-1.3001792E-1,4.807103E-2,-2.768543E-2,1.4794932E-1,1.1366447E-2,2.1791634E-1,3.7817523E-1,1.4940096E-1,1.9967576E-2,9.597497E-2,3.8968406E-3,1.7891545E-2,-4.9610483E-3,6.453598E-4,-3.4205803E-1,-4.8990897E-3,-4.5222253E-2,-2.701637E-1,6.4718286E-3,-1.682352E-2,-1.4632024E-2,-9.820548E-3,-6.556576E-3,-2.2709982E-3,-9.4766E-4,-1.2223654E-2,-2.5986207E-3,4.394718E-3,1.0262978E-3,-6.56936E-3,9.380855E-3,3.5401199E-3,-8.667829E-3,1.4345562E-4,-7.379104E-3,-5.822307E-4,-2.7252715E-3,6.895894E-3,2.0423066E-3,-1.2284707E-2,1.4087687E-2,1.5373081E-3,5.446273E-3,-7.1571153E-3,1.1572511E-2,5.50883E-4,1.5471532E-2,2.7594596E-2,1.3073209E-2,2.9831282E-3,-5.0160736E-3,9.163869E-3,7.29405E-4,-9.888496E-3,-2.028582E-2,-1.1648229E-2,-4.6737054E-3,4.760504E-3,-1.4967885E-2,-4.860894E-3,2.038487E-3,-4.2634434E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,-1,-1,-1,93,-1,95,97,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.903181E0,2.6317916E0,4.525345E0,5.789013E-1,1.1035788E0,2.3618479E0,8.89161E-1,3.5698128E-1,9.651029E-1,3.8407862E-1,7.728572E-1,1.7596941E0,7.7777445E-1,2.0743918E-1,2.457183E-1,7.803035E-2,2.9579306E-1,0E0,0E0,2.284771E-1,3.9322925E-1,4.6475875E-1,2.8714514E-1,1.2053186E0,1.8004808E0,4.13599E-1,8.582476E-1,1.9699493E-2,9.0266466E-2,2.2372031E-1,7.534765E-2,0E0,4.0682793E-2,2.1313238E-1,1.4975059E-1,6.3301705E-2,9.502351E-2,1.9840193E-1,1.5804952E-1,4.6530533E-1,1.9526239E-1,3.7517974E-1,2.9870856E-1,1.2372906E0,3.4039783E-1,4.60145E-1,7.923391E-1,0E0,2.3422688E-1,4.052856E-1,0E0,0E0,0E0,8.181739E-2,0E0,1.7885801E-1,2.0443588E-2,0E0,3.202809E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,53,53,55,55,56,56,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,-1,-1,-1,94,-1,96,98,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.177E3,1.972052E5,3.0923106E-10,6.958929E1,1.5357143E0,8E0,1.24141544E-1,3.23E2,1.75E2,5.02E2,1.9791039E6,5.862504E2,1.2083E4,6.68411E2,1.4433751E0,1.798E3,6.8780864E8,-3.2776635E-2,1.3254651E-4,3.6401E4,7.0858893E0,9.750085E6,4.385E4,6.325E4,2.3308511E5,5.501002E9,3.6966505E0,1.6E1,5.524E3,4.11566E5,2.9157894E0,-2.8116424E-3,6.6E0,5.88E2,9.5596474E-1,3.46E2,9.624326E7,2.1111E4,8.025477E0,3.5897333E2,1.0674361E8,4.5865917E8,1.68574E5,1.574E3,2.5369E4,2.5045E4,2.1356704E0,1.9967576E-2,1.4342732E2,3.0070068E7,1.7891545E-2,-4.9610483E-3,6.453598E-4,1.0479E4,-4.8990897E-3,2.83475E5,7.25E2,6.4718286E-3,2.62E3,-1.4632024E-2,-9.820548E-3,-6.556576E-3,-2.2709982E-3,-9.4766E-4,-1.2223654E-2,-2.5986207E-3,4.394718E-3,1.0262978E-3,-6.56936E-3,9.380855E-3,3.5401199E-3,-8.667829E-3,1.4345562E-4,-7.379104E-3,-5.822307E-4,-2.7252715E-3,6.895894E-3,2.0423066E-3,-1.2284707E-2,1.4087687E-2,1.5373081E-3,5.446273E-3,-7.1571153E-3,1.1572511E-2,5.50883E-4,1.5471532E-2,2.7594596E-2,1.3073209E-2,2.9831282E-3,-5.0160736E-3,9.163869E-3,7.29405E-4,-9.888496E-3,-2.028582E-2,-1.1648229E-2,-4.6737054E-3,4.760504E-3,-1.4967885E-2,-4.860894E-3,2.038487E-3,-4.2634434E-3],"split_indices":[2,40,49,58,65,29,50,1,6,2,40,64,9,64,50,9,43,0,0,9,68,9,12,1,45,5,47,0,41,1,66,0,64,2,65,12,43,9,68,64,7,5,9,2,41,2,50,0,68,55,0,0,0,9,0,1,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.99E2,5.08E2,4.91E2,1.93E2,3.15E2,4.28E2,6.3E1,1.86E2,7E0,1.34E2,1.81E2,2.56E2,1.72E2,2.8E1,3.5E1,4.4E1,1.42E2,4E0,3E0,3.3E1,1.01E2,1.45E2,3.6E1,1.15E2,1.41E2,1.5E1,1.57E2,4E0,2.4E1,2.4E1,1.1E1,3E0,4.1E1,1.18E2,2.4E1,1.2E1,2.1E1,8.1E1,2E1,1.29E2,1.6E1,2E1,1.6E1,6.7E1,4.8E1,7.1E1,7E1,6E0,9E0,1.52E2,5E0,2E0,2E0,2.1E1,3E0,1.9E1,5E0,6E0,5E0,1.3E1,2.8E1,8.1E1,3.7E1,3E0,2.1E1,3E0,9E0,4E0,1.7E1,1.5E1,6.6E1,5E0,1.5E1,1.02E2,2.7E1,8E0,8E0,1.6E1,4E0,6E0,1E1,4.1E1,2.6E1,4.1E1,7E0,6.2E1,9E0,2.6E1,4.4E1,3E0,6E0,1.45E2,7E0,8E0,1.3E1,1.4E1,5E0,3E0,2E0,3E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-1.420773E-4,-6.55594E-2,1.1893255E-1,-1.1896166E-1,1.937947E-2,1.5012269E-1,-9.996199E-2,-1.4905229E-1,4.430263E-3,-7.400764E-2,5.241264E-2,2.7814502E-2,1.392135E-1,-3.2931012E-1,-2.5025098E-2,-1.2295819E-1,-3.514902E-1,3.4299817E-2,-1.6519973E-1,-3.996357E-2,-4.0778774E-1,3.7167087E-2,1.3392569E-2,1.5016833E-1,-2.0006649E-1,-4.491507E-2,-2.2295408E-2,-1.4500843E-1,6.186895E-2,-1.05249204E-1,-3.2011062E-1,-5.187395E-1,-2.2147974E-1,5.982319E-2,-1.0996573E-2,-1.1653159E-2,-8.923267E-2,9.4216004E-2,-8.193655E-2,-2.3922442E-2,-5.564055E-3,-9.740317E-4,1.0392927E-1,1.0828714E-1,2.5684652E-1,-2.545613E-4,-2.826423E-1,1.2578025E-3,-4.696167E-3,1.5408935E-3,-8.040738E-3,9.499922E-2,-7.7430955E-3,-8.476282E-3,-1.9470359E-3,-3.6895616E-4,-1.605077E-2,-5.284847E-3,-3.0088492E-2,-2.1910928E-2,-7.903559E-3,-2.9418704E-3,4.747264E-3,-1.2605949E-3,-5.549276E-3,2.438977E-3,8.911146E-3,-1.4000453E-2,-1.9239035E-3,1.3410267E-3,-4.7333012E-3,6.201843E-3,-1.3319623E-2,8.657191E-3,2.2524935E-3,1.6174547E-2,6.188198E-3,-1.6132524E-2,-6.586324E-3,6.0918885E-3,-3.6411132E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,-1,49,51,53,55,57,59,61,-1,-1,63,65,67,-1,-1,69,71,73,75,-1,77,-1,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.82071E0,2.9415863E0,2.4455433E0,1.4784112E0,7.781819E-1,1.4658694E0,7.651575E-1,1.6471605E0,4.0594134E-1,7.336767E-1,6.673539E-1,0E0,1.1673713E0,4.665171E-1,3.738758E-1,9.61349E-1,6.6789913E-1,4.8937404E-1,5.804473E-2,3.5126665E-1,1.313259E-1,4.482649E-1,0E0,1.2991266E0,1.5906355E-1,2.3523606E-2,0E0,8.6358964E-2,1.7634572E-1,1.3030384E0,2.0778346E-1,7.5015163E-1,1.9569266E-1,3.463413E-1,0E0,0E0,1.2295954E-2,5.124417E-2,4.067632E-1,0E0,0E0,3.510139E-1,8.2306635E-1,1.012264E0,9.1849375E-1,0E0,1.6490221E-2,0E0,0E0,0E0,0E0,1.351147E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,36,36,37,37,38,38,41,41,42,42,43,43,44,44,46,46,51,51],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,-1,50,52,54,56,58,60,62,-1,-1,64,66,68,-1,-1,70,72,74,76,-1,78,-1,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,7.785302E5,2E0,4.39776E6,1E0,3.3817584E7,5E0,9.932432E0,4.07E2,3.6E1,6.663214E6,2.7814502E-2,4.1325716E7,8.620714E2,3.808001E5,1.7479E4,2E0,1E0,1.266E3,5E-1,1.8370886E2,6.946813E7,1.3392569E-2,8.439E3,3.206931E2,2.06E2,-2.2295408E-2,1.4359008E8,1.9617874E3,3.72E2,2E0,8.317E3,3.964269E3,1.241995E2,-1.0996573E-2,-1.1653159E-2,1.5321098E8,1.17566875E4,1.342376E0,-2.3922442E-2,-5.564055E-3,4.066351E0,1E0,3.72381E5,8.992806E-4,-2.545613E-4,9.730783E8,1.2578025E-3,-4.696167E-3,1.5408935E-3,-8.040738E-3,3.5510652E6,-7.7430955E-3,-8.476282E-3,-1.9470359E-3,-3.6895616E-4,-1.605077E-2,-5.284847E-3,-3.0088492E-2,-2.1910928E-2,-7.903559E-3,-2.9418704E-3,4.747264E-3,-1.2605949E-3,-5.549276E-3,2.438977E-3,8.911146E-3,-1.4000453E-2,-1.9239035E-3,1.3410267E-3,-4.7333012E-3,6.201843E-3,-1.3319623E-2,8.657191E-3,2.2524935E-3,1.6174547E-2,6.188198E-3,-1.6132524E-2,-6.586324E-3,6.0918885E-3,-3.6411132E-3],"split_indices":[2,40,6,9,27,7,6,66,0,3,40,0,63,4,40,41,29,82,2,65,4,7,0,2,70,0,0,7,4,2,29,9,40,64,0,0,7,45,66,0,0,66,82,9,69,0,7,0,0,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.002E3,6.47E2,3.55E2,3.97E2,2.5E2,3.11E2,4.4E1,3.19E2,7.8E1,6.5E1,1.85E2,6E0,3.05E2,1E1,3.4E1,2.84E2,3.5E1,6.7E1,1.1E1,6E1,5E0,1.75E2,1E1,2.96E2,9E0,4E0,6E0,1.4E1,2E1,2.62E2,2.2E1,1.4E1,2.1E1,6.2E1,5E0,4E0,7E0,1.4E1,4.6E1,3E0,2E0,1.12E2,6.3E1,2.14E2,8.2E1,3E0,6E0,2E0,2E0,2E0,1.2E1,1.8E1,2E0,1.15E2,1.47E2,2E0,2E1,4E0,1E1,2E0,1.9E1,1.6E1,4.6E1,3E0,4E0,1.1E1,3E0,6E0,4E1,8.7E1,2.5E1,5.9E1,4E0,9E1,1.24E2,4.5E1,3.7E1,3E0,3E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[9.253357E-3,-4.7993522E-2,1.4785032E-1,-1.4394008E-1,-1.3673422E-2,1.807617E-1,-7.9409294E-2,-2.87073E-1,-1.2945157E-1,1.1363263E-2,-1.381156E-1,2.3566316E-1,9.116903E-2,7.5243026E-2,-1.882742E-1,-4.591257E-3,-1.566792E-2,-6.564618E-2,-1.8271938E-1,2.6180907E-3,2.9033467E-1,-1.7414494E-1,9.857733E-2,3.153112E-1,1.5504847E-1,2.2814969E-2,2.4306594E-1,1.0379227E-1,-3.8154325E-3,-2.4674058E-1,6.2701E-3,-1.0384232E-1,8.8854015E-2,-1.5921687E-1,-4.4200426E-1,1.6213113E-2,-3.370156E-3,3.873287E-3,1.470381E-2,-3.1213945E-1,-1.2175145E-1,-5.2412916E-3,1.4125094E-2,1.2292955E-1,3.8525912E-1,2.306585E-1,6.298686E-2,4.900866E-2,-1.45615535E-2,4.6186113E-1,9.208656E-2,3.552618E-2,1.669832E-1,-3.0629647E-1,2.4420915E-3,-5.543865E-3,2.216153E-3,1.1226426E-2,7.6392555E-4,-2.4155423E-3,-8.59278E-3,-2.5781829E-2,-9.746567E-3,8.526147E-4,-4.0166937E-3,-2.0729288E-2,-7.222552E-3,-1.020671E-2,-1.8124151E-3,-3.5491586E-3,2.0049948E-3,-4.7043436E-3,7.528261E-3,1.8604398E-2,7.021072E-3,-2.2609683E-3,1.234551E-2,-4.699665E-3,6.72559E-3,-4.057577E-4,1.0246807E-2,2.4796994E-2,1.1238398E-2,1.3535519E-3,1.4676764E-2,1.2496878E-4,3.1073338E-3,1.0224106E-2,3.1980383E-3,-1.9550225E-2,-7.632992E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,37,39,41,43,45,47,49,51,-1,53,-1,55,57,59,61,-1,63,-1,-1,65,67,69,-1,71,73,75,77,79,-1,81,83,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.7905893E0,2.2865942E0,2.1609488E0,3.4233975E-1,1.5998824E0,1.2131262E0,6.3710606E-1,1.2769389E-1,5.5899906E-1,1.0434115E0,7.5056505E-1,9.463358E-1,1.0015831E0,8.093626E-2,4.5143908E-1,0E0,0E0,4.695291E-1,4.8416543E-1,8.739904E-1,5.2784562E-2,5.042474E-1,2.619236E-1,9.888058E-1,5.353731E-1,6.2085414E-1,9.484757E-1,5.4460913E-2,0E0,3.556503E-1,0E0,1.6733456E-1,1.7569587E-1,2.4267459E-1,7.281184E-2,0E0,7.577058E-1,0E0,0E0,3.601606E-1,4.542082E-1,3.5068624E-2,0E0,2.1991506E-1,1.9560051E-1,4.720297E-1,5.253244E-1,6.464024E-1,0E0,7.120991E-2,2.6118785E-1,8.246989E-3,2.2400588E-2,1.9681096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,36,36,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,38,40,42,44,46,48,50,52,-1,54,-1,56,58,60,62,-1,64,-1,-1,66,68,70,-1,72,74,76,78,80,-1,82,84,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.62E3,4.58E2,2E0,7.785714E0,1.9E1,8E0,1.889865E1,5.3E1,4.0869565E0,7.4711205E6,1E0,2.115238E5,1.9616238E2,3.5510652E6,3.2462872E5,-4.591257E-3,-1.566792E-2,3.06E2,4.4895835E0,2.04115E5,5.51E2,2.277451E2,1.338756E1,7.233077E2,2.1356704E0,1.7291568E2,1.1246073E2,1.6E1,-3.8154325E-3,1.0795462E-6,6.2701E-3,5E0,1.040054E6,1.3165267E0,2.2E1,1.6213113E-2,1.9E1,3.873287E-3,1.470381E-2,1E0,4.4761734E5,1.27E2,1.4125094E-2,1.705E3,2.6E1,4.9963706E5,1.1293673E3,1.68991E6,-1.45615535E-2,2.1298597E0,5.428175E3,3.115515E10,1.5209424E1,4.888E3,2.4420915E-3,-5.543865E-3,2.216153E-3,1.1226426E-2,7.6392555E-4,-2.4155423E-3,-8.59278E-3,-2.5781829E-2,-9.746567E-3,8.526147E-4,-4.0166937E-3,-2.0729288E-2,-7.222552E-3,-1.020671E-2,-1.8124151E-3,-3.5491586E-3,2.0049948E-3,-4.7043436E-3,7.528261E-3,1.8604398E-2,7.021072E-3,-2.2609683E-3,1.234551E-2,-4.699665E-3,6.72559E-3,-4.057577E-4,1.0246807E-2,2.4796994E-2,1.1238398E-2,1.3535519E-3,1.4676764E-2,1.2496878E-4,3.1073338E-3,1.0224106E-2,3.1980383E-3,-1.9550225E-2,-7.632992E-3],"split_indices":[2,2,6,64,3,29,68,41,70,40,28,45,68,40,45,0,0,2,65,5,2,64,68,67,50,70,70,3,0,49,0,8,5,65,3,0,8,0,0,27,40,0,0,41,8,40,4,40,0,50,64,43,68,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E2,6.94E2,2.86E2,1.82E2,5.12E2,2.5E2,3.6E1,1.5E1,1.67E2,4.27E2,8.5E1,1.54E2,9.6E1,1.5E1,2.1E1,4E0,1.1E1,7.7E1,9E1,4.15E2,1.2E1,7.4E1,1.1E1,7.6E1,7.8E1,6.7E1,2.9E1,1.3E1,2E0,1.8E1,3E0,6.2E1,1.5E1,8.4E1,6E0,6E0,4.09E2,2E0,1E1,1.9E1,5.5E1,8E0,3E0,2.1E1,5.5E1,4.2E1,3.6E1,6.3E1,4E0,1.1E1,1.8E1,7E0,6E0,1.5E1,3E0,5.6E1,6E0,4E0,1.1E1,1.8E1,6.6E1,3E0,3E0,3.25E2,8.4E1,9E0,1E1,2.4E1,3.1E1,3E0,5E0,3E0,1.8E1,5E1,5E0,5E0,3.7E1,1.2E1,2.4E1,4.8E1,1.5E1,7E0,4E0,1.5E1,3E0,4E0,3E0,3E0,3E0,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-6.504201E-3,-1.04098246E-1,5.6704264E-2,-1.3814317E-1,-1.111054E-2,8.568826E-2,-1.1994506E-1,-1.7487964E-1,-3.1073796E-2,4.3668516E-4,-1.3774886E-2,1.4617826E-1,-3.5612443E-3,-2.6620266E-1,-5.345958E-2,-1.5311252E-1,-3.4425235E-1,7.258876E-4,-1.5408096E-1,-1.1994684E-1,2.4508253E-2,8.3154604E-2,2.3824432E-1,-2.7062416E-2,2.9905793E-1,-1.1187519E-1,-3.5566515E-1,-9.02866E-2,1.15448005E-1,-1.5815076E-1,9.582107E-3,-2.7376268E-2,-2.5987834E-1,-9.2234366E-2,4.0548716E-2,5.9098545E-3,-2.07478E-1,4.2706937E-2,-3.082707E-1,1.0879402E-1,-2.1082483E-2,1.0622772E-1,-1.2037935E-1,3.5957795E-1,1.5579642E-1,2.6025617E-2,-8.4426165E-2,2.4830164E-2,1.24232486E-1,-2.653804E-2,-9.598414E-3,-1.7396804E-2,-5.941589E-3,-1.2605762E-1,6.538255E-3,1.4138197E-3,6.5843104E-3,2.1557831E-3,-7.5919037E-3,-1.4384009E-2,-4.2365924E-3,-6.577567E-3,3.820568E-4,-1.5923934E-4,3.8917526E-3,-1.1089208E-2,-6.3531025E-4,3.6212273E-3,-1.4331584E-3,-7.191249E-3,-1.932964E-2,-6.3378783E-4,6.148602E-3,-1.8518704E-3,1.52577795E-2,8.060812E-3,3.4130523E-3,-3.302509E-2,-3.5174852E-4,1.9012218E-2,9.590332E-3,1.1476306E-2,6.5367884E-4,5.8038686E-3,-3.1638483E-4,-9.822575E-3,-1.297054E-3,9.299622E-3,-5.7783565E-3,-6.984299E-3,2.4840727E-3,-6.66168E-3,2.1669117E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,-1,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,85,-1,87,89,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0266223E0,1.2141166E0,3.0464401E0,1.0998373E0,3.469419E-1,2.753859E0,8.025855E-1,7.17823E-1,2.8565782E-1,2.9561955E-1,0E0,1.7440996E0,1.4802575E0,3.1917942E-1,3.7742484E-1,3.737979E-1,3.336935E-1,2.2215447E-1,2.5316378E-1,5.339023E-1,3.289189E-1,8.67489E-1,1.1744342E0,5.899457E-1,5.662321E-1,8.560003E-2,3.4611702E-2,4.2570108E-1,2.157414E-2,2.7852488E-1,0E0,0E0,1.4083266E-1,9.344269E-2,8.19686E-2,0E0,8.212888E-2,2.916528E-2,7.072669E-2,9.4050646E-2,3.9393005E-1,3.4957552E-1,1.2765759E0,3.1510735E-1,9.951166E-1,3.3440834E-1,6.70268E-1,0E0,2.2108561E-1,7.906017E-2,0E0,0E0,0E0,1.4820606E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,-1,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,86,-1,88,90,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.01E2,8.995735E5,3.0923106E-10,1.345672E6,2E0,8E0,5.4225586E-2,1.5E1,1.6533333E1,1.3511679E-5,-1.3774886E-2,6.4610266E2,4.435876E2,4.217427E7,1.4573088E0,6.5E2,1E0,1.545E2,9.8E1,1.0280637E0,6.642857E0,4.5158855E6,1.9674084E5,1.3254E4,4.1885E4,2.1391304E1,5.4051904E8,3.9E1,1.01E2,7E0,9.582107E-3,-2.7376268E-2,2.0262831E2,6.593836E6,6.946813E7,5.9098545E-3,6.54E2,3.5714287E-1,6.99E2,2.0411983E2,3.2E1,1.0164831E-2,3.819455E-1,1.2E1,2.342612E0,2.01184E5,1.862E3,2.4830164E-2,5.3142116E7,4.8614563E2,-9.598414E-3,-1.7396804E-2,-5.941589E-3,6.165E3,6.538255E-3,1.4138197E-3,6.5843104E-3,2.1557831E-3,-7.5919037E-3,-1.4384009E-2,-4.2365924E-3,-6.577567E-3,3.820568E-4,-1.5923934E-4,3.8917526E-3,-1.1089208E-2,-6.3531025E-4,3.6212273E-3,-1.4331584E-3,-7.191249E-3,-1.932964E-2,-6.3378783E-4,6.148602E-3,-1.8518704E-3,1.52577795E-2,8.060812E-3,3.4130523E-3,-3.302509E-2,-3.5174852E-4,1.9012218E-2,9.590332E-3,1.1476306E-2,6.5367884E-4,5.8038686E-3,-3.1638483E-4,-9.822575E-3,-1.297054E-3,9.299622E-3,-5.7783565E-3,-6.984299E-3,2.4840727E-3,-6.66168E-3,2.1669117E-3],"split_indices":[2,40,49,9,6,29,50,8,70,54,0,67,68,57,50,10,27,67,10,66,70,59,45,41,2,68,7,3,0,3,0,0,64,9,7,0,2,65,2,45,3,50,54,8,54,9,2,0,1,4,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.75E2,3.83E2,5.92E2,2.8E2,1.03E2,5.09E2,8.3E1,2.08E2,7.2E1,1E2,3E0,3.03E2,2.06E2,2.5E1,5.8E1,1.86E2,2.2E1,5.8E1,1.4E1,1.6E1,8.4E1,1.81E2,1.22E2,1.92E2,1.4E1,1E1,1.5E1,4.8E1,1E1,1.84E2,2E0,4E0,1.8E1,1.7E1,4.1E1,2E0,1.2E1,9E0,7E0,2.9E1,5.5E1,1.63E2,1.8E1,4.8E1,7.4E1,1E2,9.2E1,5E0,9E0,6E0,4E0,1.3E1,2E0,4.2E1,6E0,3E0,7E0,6E0,1.78E2,1.3E1,5E0,1.1E1,6E0,2.1E1,2E1,1E1,2E0,6E0,3E0,4E0,3E0,5E0,2.4E1,5.3E1,2E0,5E1,1.13E2,2E0,1.6E1,3.4E1,1.4E1,4.4E1,3E1,2.4E1,7.6E1,2.7E1,6.5E1,7E0,2E0,2E0,4E0,3.8E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[7.5790175E-4,-7.0860036E-2,8.639305E-2,-1.0402498E-1,3.179185E-2,1.2459245E-1,-7.407029E-2,-8.7206826E-2,-3.0784768E-1,-1.6577229E-2,4.3192014E-2,7.339697E-2,2.0434889E-1,-2.3885497E-1,-3.0166468E-2,-1.5334551E-1,-4.9740814E-2,-6.340864E-1,-1.5262008E-1,5.018158E-3,1.3372947E-1,8.124357E-2,-2.5503773E-2,2.7222437E-1,5.7270348E-2,-1.2099361E-1,-3.135729E-1,1.09331556E-1,-4.9017478E-2,-2.731464E-1,-1.4018585E-1,2.9462287E-1,-5.929873E-2,-3.2316096E-2,-1.0905633E-2,-8.970185E-3,-3.0851358E-1,-7.089355E-3,2.2679277E-1,2.6052368E-1,7.6651536E-2,1.2270165E-1,-8.372935E-3,2.2656043E-1,5.00654E-1,1.1110982E-1,-1.3392697E-1,-8.798745E-3,3.858137E-3,-1.5609517E-2,-5.9820455E-3,1.9551596E-2,9.951925E-3,-6.577429E-2,9.922101E-2,-1.4526791E-2,-1.1012565E-3,-4.9901237E-3,-9.3557825E-3,1.550912E-2,5.7272646E-3,-1.9489989E-3,-9.2335325E-3,-4.4781063E-3,9.8562185E-3,-1.8458216E-2,-5.3336117E-3,6.758105E-4,-4.480183E-3,1.4790571E-2,2.5710003E-3,1.3816758E-2,1.8091628E-3,4.6795527E-3,-4.8918445E-3,3.6883764E-3,1.0926869E-2,-1.4462237E-3,5.221086E-3,1.4662752E-2,5.376895E-3,2.8282806E-2,1.4173469E-2,9.978423E-3,1.4902693E-3,-9.352104E-3,5.5009117E-3,-2.2600375E-3,4.809453E-3,-3.726593E-3,5.343364E-3,6.7960327E-3,2.302705E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,59,61,-1,-1,63,65,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,-1,87,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.329258E0,1.9166045E0,2.8901424E0,1.4300199E0,6.333983E-1,1.5366058E0,6.511502E-1,9.686856E-1,1.5237463E0,0E0,4.6591106E-1,1.1944493E0,1.4618974E0,1.2938058E-1,1.9732572E-1,1.8557763E-1,8.501947E-1,1.0856247E-1,5.122965E-1,2.5938115E-1,2.695166E-1,8.592683E-1,0E0,9.401016E-1,5.089909E-1,1.4628412E-1,8.717895E-3,8.249422E-2,1.6798651E-1,1.3210511E-1,2.3827577E-1,9.087384E-3,5.819023E-1,0E0,0E0,2.7819395E-1,1.5089357E-1,1.8286698E-1,6.330246E-2,9.506142E-2,1.4485647E-1,7.5259876E-1,2.1232355E-1,8.352823E-1,1.6373539E-1,3.102171E-1,2.188408E-1,0E0,0E0,0E0,0E0,4.0282745E-2,0E0,1.7446443E-1,9.3296245E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,51,51,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,60,62,-1,-1,64,66,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,-1,88,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5897333E2,1.5859042E6,1E0,9.932432E0,1.65E2,1.1941613E3,2.5807297E-2,3.39E2,2E0,-1.6577229E-2,6.789622E7,1.6776881E7,1.7493458E7,7.760169E2,7.1864405E0,7.785714E0,3.7974394E-6,3.6255838E2,4.2782607E0,1.3177083E1,1.8992E4,8E0,-2.5503773E-2,4.0277927E3,2.5130852E2,3.6E1,1.2376862E10,1.515E3,1.8484E4,5.0363636E1,3.53969E5,7.9E1,1E0,-3.2316096E-2,-1.0905633E-2,9.277255E6,8.237624E0,1.8E1,8.0274E4,3.8E1,4.1388846E2,2.43E2,1.6566709E6,8E0,1.159499E6,1.8571428E0,2.433735E0,-8.798745E-3,3.858137E-3,-1.5609517E-2,-5.9820455E-3,2.5E1,9.951925E-3,6.2601447E2,2E0,-1.4526791E-2,-1.1012565E-3,-4.9901237E-3,-9.3557825E-3,1.550912E-2,5.7272646E-3,-1.9489989E-3,-9.2335325E-3,-4.4781063E-3,9.8562185E-3,-1.8458216E-2,-5.3336117E-3,6.758105E-4,-4.480183E-3,1.4790571E-2,2.5710003E-3,1.3816758E-2,1.8091628E-3,4.6795527E-3,-4.8918445E-3,3.6883764E-3,1.0926869E-2,-1.4462237E-3,5.221086E-3,1.4662752E-2,5.376895E-3,2.8282806E-2,1.4173469E-2,9.978423E-3,1.4902693E-3,-9.352104E-3,5.5009117E-3,-2.2600375E-3,4.809453E-3,-3.726593E-3,5.343364E-3,6.7960327E-3,2.302705E-3],"split_indices":[64,40,6,66,12,4,50,2,29,0,7,59,63,64,68,64,50,4,68,68,9,29,0,4,70,0,43,2,41,59,9,45,82,0,0,9,68,3,1,8,4,0,40,29,9,65,65,0,0,0,0,8,0,58,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,5.61E2,4.69E2,4.24E2,1.37E2,3.79E2,9E1,3.93E2,3.1E1,3E0,1.34E2,2.32E2,1.47E2,1.8E1,7.2E1,1.41E2,2.52E2,9E0,2.2E1,9.5E1,3.9E1,2.3E2,2E0,1E2,4.7E1,8E0,1E1,8E0,6.4E1,1.2E1,1.29E2,6E0,2.46E2,7E0,2E0,1.2E1,1E1,9.1E1,4E0,1.1E1,2.8E1,1.57E2,7.3E1,8.5E1,1.5E1,3.7E1,1E1,6E0,2E0,8E0,2E0,5E0,3E0,5.8E1,6E0,1E1,2E0,8.8E1,4.1E1,4E0,2E0,2.21E2,2.5E1,9E0,3E0,6E0,4E0,7.4E1,1.7E1,2E0,2E0,9E0,2E0,2.5E1,3E0,1.16E2,4.1E1,6.2E1,1.1E1,4.5E1,4E1,8E0,7E0,1.5E1,2.2E1,8E0,2E0,3E0,2E0,5.4E1,4E0,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[7.374137E-4,-4.5013603E-2,1.444419E-1,-1.5913586E-1,-1.2689689E-2,4.7899124E-1,1.24819264E-1,-1.02500744E-1,-2.3426926E-1,-2.8383318E-2,9.73372E-2,1.2180519E-2,2.6615925E-2,3.921345E-2,2.0501445E-1,-1.4950655E-1,8.594694E-2,-2.0627342E-1,-2.9930428E-2,3.099653E-2,-6.4921714E-2,-1.4485243E-1,1.1698648E-1,-2.125511E-1,5.5469472E-2,2.3394604E-1,5.953945E-3,-2.0166647E-1,-7.607409E-2,2.088008E-1,-1.2443394E-1,3.394368E-3,-2.1588326E-1,3.9551396E-2,-2.4722822E-2,-2.303232E-2,-5.7831634E-2,-5.066039E-5,-1.3239045E-2,8.9605704E-2,2.695725E-1,2.0813078E-3,-1.916621E-2,1.2534364E-1,-2.5296463E-2,-2.6265264E-3,2.5662455E-1,1.1523984E-1,-7.114157E-2,-5.1525813E-3,-1.3325749E-2,-1.797113E-3,-2.0431736E-2,2.4985378E-3,1.065286E-2,-8.1354175E-3,1.029653E-3,-1.036932E-2,-2.94755E-4,3.586927E-4,7.4937497E-3,-5.9051244E-3,-1.4017175E-3,8.018372E-3,1.9639481E-3,4.110145E-3,1.3936048E-2,-7.7581475E-4,7.1637426E-3,-2.4507057E-3,7.1051093E-3,2.7465478E-3,-7.726262E-3,1.5837647E-2,1.0005541E-2,7.965382E-3,1.2976193E-3,-5.4596285E-3,3.6668673E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,-1,-1,61,-1,-1,63,65,-1,-1,67,69,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5614038E0,2.7904613E0,1.522315E0,6.86604E-1,1.023255E0,1.2560368E-1,1.5640855E0,8.720887E-1,6.9517803E-1,1.1258882E0,3.6910093E-1,0E0,0E0,4.7017145E-1,6.78185E-1,2.8341663E-1,5.429039E-1,1.9903874E-1,0E0,9.8552084E-1,9.645003E-1,1.2334846E-1,2.6412147E-1,3.8845006E-1,6.0137665E-1,5.5328035E-1,1.4300558E-1,3.2365632E-1,4.458412E-1,3.3807874E-2,6.586712E-2,0E0,1.3503909E-1,7.6769114E-1,0E0,0E0,6.085141E-1,0E0,0E0,2.3136398E-1,3.3785462E-2,0E0,0E0,2.5089765E-1,2.6076552E-1,1.134224E-1,2.4642324E-1,3.0526958E-2,8.153891E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,36,36,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,-1,-1,62,-1,-1,64,66,-1,-1,68,70,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,8.147158E4,2.4481E4,4.427818E7,3.2911258E6,3.921E3,1.8062708E6,6.76E2,6.9934395E-5,1.5132743E0,8.612209E-6,1.2180519E-2,2.6615925E-2,1.55E2,2E0,3.89E2,1.0657745E7,1E0,-2.9930428E-2,2.8366232E0,5.862757E1,6.83E2,5.130175E2,1.3E1,5.59E2,1E0,2.6750835E6,6.48334E5,5.02E2,2.123E3,6.2478176E7,3.394368E-3,1.363176E7,4.371613E2,-2.4722822E-2,-2.303232E-2,5.56E2,-5.066039E-5,-1.3239045E-2,2.1525E4,1.68574E5,2.0813078E-3,-1.916621E-2,2.16577E5,4.435876E2,2.2313573E2,8.536586E0,4.7220547E1,6.165E3,-5.1525813E-3,-1.3325749E-2,-1.797113E-3,-2.0431736E-2,2.4985378E-3,1.065286E-2,-8.1354175E-3,1.029653E-3,-1.036932E-2,-2.94755E-4,3.586927E-4,7.4937497E-3,-5.9051244E-3,-1.4017175E-3,8.018372E-3,1.9639481E-3,4.110145E-3,1.3936048E-2,-7.7581475E-4,7.1637426E-3,-2.4507057E-3,7.1051093E-3,2.7465478E-3,-7.726262E-3,1.5837647E-2,1.0005541E-2,7.965382E-3,1.2976193E-3,-5.4596285E-3,3.6668673E-3],"split_indices":[2,40,12,43,40,2,40,2,49,65,50,0,0,10,6,1,57,20,0,50,59,2,64,3,0,25,40,43,2,1,7,0,9,64,0,0,2,0,0,9,9,0,0,40,68,70,68,70,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.96E2,7.56E2,2.4E2,1.66E2,5.9E2,1.2E1,2.28E2,9.6E1,7E1,5.17E2,7.3E1,5E0,7E0,1.11E2,1.17E2,7.7E1,1.9E1,6.7E1,3E0,1.97E2,3.2E2,5E0,6.8E1,6E0,1.05E2,1.02E2,1.5E1,4.4E1,3.3E1,1.2E1,7E0,2E0,6.5E1,1.95E2,2E0,4E0,3.16E2,3E0,2E0,5.9E1,9E0,3E0,3E0,5.6E1,4.9E1,9E0,9.3E1,6E0,9E0,2.3E1,2.1E1,3.1E1,2E0,2E0,1E1,5E0,2E0,6.2E1,3E0,1.56E2,3.9E1,8.7E1,2.29E2,2E1,3.9E1,2E0,7E0,1E1,4.6E1,4.3E1,6E0,7E0,2E0,2.6E1,6.7E1,3E0,3E0,7E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-1.4998834E-3,-5.0740156E-2,8.3907366E-2,-1.1081732E-1,-1.6996881E-2,1.168198E-1,-9.4585486E-2,-1.02968544E-1,-2.4276242E-2,-6.65566E-3,-2.718798E-1,3.3487314E-1,1.0073781E-1,-2.0243758E-2,-6.0782783E-2,6.690225E-2,-1.1791867E-1,-8.853084E-2,2.1676285E-2,-3.0332676E-1,-4.6715676E-3,1.7092463E-2,3.8354401E-3,3.3505805E-2,1.7450033E-1,-1.3059106E-1,3.8984526E-2,-5.1576048E-2,1.1289993E-1,-7.483086E-2,-1.698364E-1,-5.152833E-3,-1.18478216E-1,4.0467028E-2,-8.938731E-2,-6.1387816E-3,-1.5656434E-2,-1.07668385E-1,4.951502E-2,3.153794E-1,1.4083931E-1,-1.4971867E-1,1.8708755E-3,-3.5679042E-2,9.916258E-2,-6.274236E-3,1.1522252E-3,-1.3052962E-3,6.3762707E-3,-4.4787014E-3,6.9126417E-3,-7.015124E-3,-1.8155633E-2,-3.2525333E-3,2.2284973E-3,-8.38429E-3,-3.1782794E-3,4.009353E-3,9.420151E-5,2.3426802E-3,-8.013724E-3,-1.6037472E-2,8.844591E-5,1.1432836E-3,7.6461257E-3,1.6355095E-2,3.3640768E-3,3.280993E-3,9.950402E-3,-5.2839657E-3,-1.5511097E-2,-4.4236844E-3,1.4192292E-3,8.530709E-3,-1.5057817E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,29,31,33,35,-1,-1,-1,37,39,41,43,45,47,49,51,53,55,57,59,-1,-1,61,63,65,67,69,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2433777E0,1.2955245E0,2.1787891E0,6.9964194E-1,1.0815043E0,1.0603657E0,6.518615E-1,5.8437586E-1,0E0,9.205135E-1,5.477214E-2,1.65591E-1,1.4422405E0,0E0,3.8117868E-1,1.0936774E-1,4.548285E-1,2.5407863E-1,6.1927265E-1,3.7243605E-2,0E0,0E0,0E0,3.533848E-1,6.1473846E-1,1.1096221E-1,1.0687951E-1,4.2535074E-2,5.5229038E-2,6.074199E-1,3.0213737E-1,1.02107726E-1,2.2414756E-1,4.5490402E-1,5.2253336E-1,0E0,0E0,4.241293E-1,3.96155E-1,2.3012638E-1,5.842552E-1,1.5020049E-1,0E0,4.73796E-2,1.5457147E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,30,32,34,36,-1,-1,-1,38,40,42,44,46,48,50,52,54,56,58,60,-1,-1,62,64,66,68,70,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1E0,1E0,7.9E1,2.8025E4,5.6E1,4.5E0,3.151751E-1,-2.4276242E-2,5.14E2,2.402121E4,1.2769117E7,1.7587205E6,-2.0243758E-2,1.8062708E6,6.845E3,2.0303884E6,3.2E1,6.97E0,3.43E2,-4.6715676E-3,1.7092463E-2,3.8354401E-3,3.64E2,2.5959E4,1.1246073E2,1.5714285E0,1.4378479E-2,3.6E2,6.9E2,8.13362E0,2.6E1,1.463E2,1E0,1.5853742E1,-6.1387816E-3,-1.5656434E-2,1.1388888E0,1.5416006E5,8.433083E6,6.941074E6,5.2224144E5,1.8708755E-3,5.071E3,2.64E0,-6.274236E-3,1.1522252E-3,-1.3052962E-3,6.3762707E-3,-4.4787014E-3,6.9126417E-3,-7.015124E-3,-1.8155633E-2,-3.2525333E-3,2.2284973E-3,-8.38429E-3,-3.1782794E-3,4.009353E-3,9.420151E-5,2.3426802E-3,-8.013724E-3,-1.6037472E-2,8.844591E-5,1.1432836E-3,7.6461257E-3,1.6355095E-2,3.3640768E-3,3.280993E-3,9.950402E-3,-5.2839657E-3,-1.5511097E-2,-4.4236844E-3,1.4192292E-3,8.530709E-3,-1.5057817E-3],"split_indices":[2,27,6,3,41,0,70,65,0,2,45,59,40,0,40,41,57,10,66,0,0,0,0,10,9,70,66,70,2,2,66,0,67,24,70,0,0,65,45,57,57,40,0,2,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.007E3,6.39E2,3.68E2,2.29E2,4.1E2,3.11E2,5.7E1,2.26E2,3E0,3.95E2,1.5E1,2E1,2.91E2,4E0,5.3E1,1.8E1,2.08E2,1.01E2,2.94E2,1.2E1,3E0,1.7E1,3E0,1.53E2,1.38E2,3.1E1,2.2E1,5E0,1.3E1,1.15E2,9.3E1,2.7E1,7.4E1,2.52E2,4.2E1,3E0,9E0,1.5E1,1.38E2,2.5E1,1.13E2,2.8E1,3E0,1E1,1.2E1,2E0,3E0,2E0,1.1E1,1.05E2,1E1,8.8E1,5E0,1.2E1,1.5E1,3.1E1,4.3E1,1.13E2,1.39E2,1.6E1,2.6E1,4E0,1.1E1,1.15E2,2.3E1,2.1E1,4E0,6E1,5.3E1,2.5E1,3E0,5E0,5E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-6.9715925E-3,-2.9886194E-2,1.8786955E-1,-8.923327E-2,1.5243968E-2,2.2074902E-1,-2.342778E-1,-7.891504E-2,-2.7204758E-1,2.9498E-2,-1.6507702E-1,2.5140805E-2,1.9319977E-1,-1.2979915E-2,-2.7354814E-3,-1.4217429E-1,-4.4270553E-2,-1.1337001E-2,-5.3162444E-1,3.4215108E-2,-2.853713E-1,2.347781E-2,-2.0232885E-1,2.5000563E-1,1.1782662E-1,-3.1283203E-1,-1.2506065E-1,2.4538133E-1,-5.683552E-2,5.2929852E-2,-6.2942966E-3,-3.526232E-2,-6.521881E-3,6.1643403E-2,-9.739115E-3,-1.593632E-2,-4.0237373E-3,3.9410777E-3,-2.0539265E-3,-2.5574645E-1,-6.457719E-2,2.6600653E-1,-4.3486883E-3,-2.6587795E-2,1.5927404E-1,-3.338503E-3,-1.6449105E-2,-4.4069947E-3,-9.200329E-3,1.3566902E-2,1.3907413E-3,-1.2447333E-3,-6.478532E-3,4.764556E-3,7.8195146E-5,3.2813128E-3,-5.456226E-3,1.0070625E-2,-1.0015343E-3,-1.3150351E-2,-7.6468773E-3,3.395726E-3,-4.98106E-3,9.970747E-3,1.6647357E-2,3.700615E-4,-5.0575593E-3,8.148806E-3,-5.532234E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,59,61,63,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4122806E0,2.3690155E0,1.495235E0,6.991062E-1,1.2977307E0,7.6702976E-1,5.0703406E-2,7.8925085E-1,1.3474617E0,7.0338637E-1,2.6463878E-1,0E0,3.6017203E-1,0E0,0E0,3.3471084E-1,8.7286067E-1,9.611471E-2,8.1087685E-1,5.56377E-1,5.4864883E-2,3.358884E-2,2.1159399E-1,3.0646205E-1,2.4964452E-1,9.360957E-2,2.3927724E-1,9.858829E-2,5.6461054E-1,2.0642973E-2,0E0,0E0,0E0,5.1045394E-1,4.965905E-1,0E0,0E0,0E0,0E0,1.2341499E-2,6.877792E-2,1.5251565E-1,0E0,2.9823354E-2,1.0157251E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,33,33,34,34,39,39,40,40,41,41,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,60,62,64,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.439E3,8.01E2,7.06191E7,9.639872E0,3.2856784E7,3.3817584E7,3.3748108E7,3.06E2,3.89E2,2.673585E1,1E0,2.5140805E-2,8.710612E1,-1.2979915E-2,-2.7354814E-3,8.0833334E-1,1.2E1,3.6E1,2.6235E4,8E0,4.29E2,4.179E3,2.3308511E5,1E0,1.6975454E3,1.6666666E0,6.81531E-2,1.2895454E3,3.7509E4,1.0005E5,-6.2942966E-3,-3.526232E-2,-6.521881E-3,1.9269184E7,1.6470588E0,-1.593632E-2,-4.0237373E-3,3.9410777E-3,-2.0539265E-3,5.680271E-2,1E0,2.1511E4,-4.3486883E-3,6.0147805E0,3.9152692E6,-3.338503E-3,-1.6449105E-2,-4.4069947E-3,-9.200329E-3,1.3566902E-2,1.3907413E-3,-1.2447333E-3,-6.478532E-3,4.764556E-3,7.8195146E-5,3.2813128E-3,-5.456226E-3,1.0070625E-2,-1.0015343E-3,-1.3150351E-2,-7.6468773E-3,3.395726E-3,-4.98106E-3,9.970747E-3,1.6647357E-2,3.700615E-4,-5.0575593E-3,8.148806E-3,-5.532234E-4],"split_indices":[2,2,56,66,57,7,55,2,2,65,6,0,70,0,0,65,10,3,1,29,0,2,45,6,64,66,54,45,1,5,0,0,0,59,65,0,0,0,0,50,15,2,0,66,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.86E2,8.83E2,1.03E2,3.81E2,5.02E2,9.6E1,7E0,3.62E2,1.9E1,4.66E2,3.6E1,6E0,9E1,5E0,2E0,1.27E2,2.35E2,1E1,9E0,4.6E2,6E0,6E0,3E1,5E1,4E1,1E1,1.17E2,9E0,2.26E2,7E0,3E0,5E0,4E0,2.83E2,1.77E2,4E0,2E0,3E0,3E0,2.1E1,9E0,4.8E1,2E0,9E0,3.1E1,2E0,8E0,8.6E1,3.1E1,7E0,2E0,1.68E2,5.8E1,3E0,4E0,2.69E2,1.4E1,8E0,1.69E2,1.4E1,7E0,2E0,7E0,3.4E1,1.4E1,7E0,2E0,2.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[1.732836E-3,-5.8314465E-2,6.0930915E-2,-7.712636E-2,5.197025E-2,8.304576E-2,-1.04651235E-1,-6.684619E-2,-2.6877686E-1,-5.9581816E-2,9.025764E-2,3.9049837E-1,7.274674E-2,-1.3378719E-2,-1.6685586E-1,-9.425914E-2,-9.397408E-3,-2.1230681E-1,-2.4657354E-2,3.759554E-2,-1.1940157E-1,5.776182E-4,1.3892457E-1,1.0109112E-2,2.1161338E-2,4.2499058E-2,1.5615404E-1,4.0464915E-2,-1.9062772E-1,-1.850699E-1,4.0932903E-3,-8.115869E-2,-2.9715642E-1,2.7527312E-2,-5.8682345E-2,-1.0085619E-3,-2.4662486E-1,5.830918E-3,-1.9018204E-3,-7.936518E-2,-9.785976E-3,9.459043E-2,-8.494109E-2,1.9373566E-1,6.9493085E-2,-4.5499034E-2,5.9736326E-2,1.699559E-1,-1.20540755E-2,1.7418106E-3,5.995377E-3,-1.1557307E-2,-2.1019096E-3,-2.4330513E-1,-1.3505174E-1,-5.546216E-3,-2.33213E-3,-1.9073907E-3,-2.5151895E-2,-1.2796267E-4,8.584793E-3,-3.8952709E-3,4.0035807E-3,-5.680937E-3,-1.50887E-2,-4.701008E-3,3.8480305E-4,5.289499E-3,3.7695217E-4,-1.1157799E-2,2.1437036E-4,5.3892825E-3,1.2198987E-2,4.104213E-3,-7.99063E-4,-8.27701E-3,-6.023635E-4,9.132208E-3,1.8192717E-3,1.4359514E-2,6.1905254E-3,2.6506784E-3,-1.9282987E-3,-4.410317E-3,-1.1935366E-2,-7.314581E-3,7.129101E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,-1,-1,45,47,49,51,53,-1,55,57,59,61,-1,63,-1,-1,65,-1,67,69,71,73,75,77,79,-1,81,-1,-1,-1,83,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5084724E0,1.0205939E0,1.828617E0,8.075471E-1,3.1289238E-1,1.3616688E0,3.3274448E-1,6.2747574E-1,2.0390213E-1,1.14938356E-1,2.3560014E-1,6.611633E-2,1.0681095E0,2.4630699E-1,1.796735E-1,6.935556E-1,2.3819359E-1,1.18700325E-1,0E0,6.1727874E-2,2.8085664E-2,1.6883169E-1,1.19733274E-1,0E0,0E0,4.7900784E-1,6.974392E-1,6.95235E-2,4.149984E-2,6.326437E-2,0E0,3.0062282E-1,1.0015733E0,3.628628E-1,2.2075437E-1,0E0,1.204322E-1,0E0,0E0,2.1023355E-2,0E0,1.6824044E-2,1.5685442E-1,7.3145926E-2,3.0893065E-2,2.2447887E-1,7.283986E-1,5.09959E-1,0E0,3.912945E-2,0E0,0E0,0E0,9.167135E-3,6.635758E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,-1,-1,46,48,50,52,54,-1,56,58,60,62,-1,64,-1,-1,66,-1,68,70,72,74,76,78,80,-1,82,-1,-1,-1,84,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,2.331083E6,2.856934E7,3.84E2,1.185E3,8E0,3.0923106E-10,1.559733E6,1.2600376E7,3.3875784E7,1E0,9.21641E3,5.709E3,3.0070068E7,1E0,4.6463413E0,2.02E2,2.1818182E0,-2.4657354E-2,1.2139681E3,1.5723522E4,6.728972E-1,1.0979382E1,1.0109112E-2,2.1161338E-2,5.1586456E7,3.206931E2,2.8582963E-1,1.1997242E0,1.36E2,4.0932903E-3,3.39E2,1.3E1,1.15E2,3.4734247E2,-1.0085619E-3,1.4195632E6,5.830918E-3,-1.9018204E-3,1.7E1,-9.785976E-3,7.4711205E6,1.6666667E-1,1.6100003E8,2.9157894E0,3.0359898E6,8.533334E0,3.3382E4,-1.20540755E-2,1.8E1,5.995377E-3,-1.1557307E-2,-2.1019096E-3,7.4654E4,1.7308458E1,-5.546216E-3,-2.33213E-3,-1.9073907E-3,-2.5151895E-2,-1.2796267E-4,8.584793E-3,-3.8952709E-3,4.0035807E-3,-5.680937E-3,-1.50887E-2,-4.701008E-3,3.8480305E-4,5.289499E-3,3.7695217E-4,-1.1157799E-2,2.1437036E-4,5.3892825E-3,1.2198987E-2,4.104213E-3,-7.99063E-4,-8.27701E-3,-6.023635E-4,9.132208E-3,1.8192717E-3,1.4359514E-2,6.1905254E-3,2.6506784E-3,-1.9282987E-3,-4.410317E-3,-1.1935366E-2,-7.314581E-3,7.129101E-5],"split_indices":[2,40,57,0,41,10,49,9,57,7,27,40,2,55,16,65,10,68,0,45,45,66,70,0,0,7,70,46,50,0,0,2,0,0,64,0,57,0,0,3,0,40,70,5,66,57,70,9,0,3,0,0,0,1,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.85E2,4.89E2,4.96E2,4.18E2,7.1E1,4.38E2,5.8E1,3.98E2,2E1,1.8E1,5.3E1,1.3E1,4.25E2,2.4E1,3.4E1,2.69E2,1.29E2,1.8E1,2E0,7E0,1.1E1,1.9E1,3.4E1,5E0,8E0,3.13E2,1.12E2,1.9E1,5E0,3.2E1,2E0,2.54E2,1.5E1,7.4E1,5.5E1,3E0,1.5E1,3E0,4E0,9E0,2E0,9E0,1E1,1.8E1,1.6E1,5.1E1,2.62E2,1.09E2,3E0,1.4E1,5E0,3E0,2E0,1.3E1,1.9E1,1.09E2,1.45E2,8E0,7E0,6.3E1,1.1E1,4.7E1,8E0,7E0,8E0,7E0,2E0,7E0,2E0,3E0,7E0,1E1,8E0,1.3E1,3E0,9E0,4.2E1,3.2E1,2.3E2,2E1,8.9E1,6E0,8E0,2E0,1.1E1,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[6.6727307E-3,-6.039415E-2,7.427457E-2,-2.0608965E-2,-1.1572298E-1,-1.6245325E-1,8.737942E-2,-1.4970109E-1,1.0505917E-2,-1.8245874E-1,-6.2372304E-2,-1.1394433E-2,-5.7749644E-2,2.7758043E-2,1.2973796E-1,-2.1459655E-1,-9.432334E-2,1.7912337E-1,2.9314507E-3,-1.1298829E-1,-5.564757E-1,2.151174E-2,-1.2190092E-1,3.4291245E-2,-1.19561605E-1,9.43034E-2,-1.9767672E-2,2.216813E-1,9.7234145E-2,-1.7654945E-1,-1.9969149E-2,-3.5199214E-2,-1.7802788E-1,9.8194685E-3,1.2394614E-3,-6.366652E-3,1.5433656E-1,-7.3329605E-2,-2.1060452E-1,-3.379031E-2,-1.5163112E-2,-6.27708E-2,9.251124E-2,-3.861422E-2,-1.9063805E-1,-1.1388025E-3,3.2166769E-3,-7.381643E-3,-2.8813253E-3,7.58835E-2,2.528995E-1,1.8230841E-3,-2.718624E-1,1.848935E-1,4.0218914E-1,2.2798559E-2,8.8722914E-2,-9.282413E-3,-5.252795E-3,-6.3675223E-3,3.913364E-4,-1.0336186E-2,2.4652635E-3,-8.125192E-4,5.0333436E-3,1.2872367E-3,1.0554062E-2,-6.2207542E-3,-1.1050076E-3,-1.5866663E-2,-5.72542E-3,-1.1093918E-3,-7.7239377E-3,8.243325E-3,-2.3098683E-3,-5.126135E-3,2.3676588E-3,-1.2424982E-2,-5.690155E-3,8.898672E-3,1.838718E-3,4.5047547E-3,1.323571E-2,-2.7598483E-3,2.6719356E-3,-2.7232133E-2,-5.7583617E-4,9.947419E-3,-2.2083186E-3,6.9688405E-3,2.0255372E-2,4.809811E-3,-1.4819051E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,59,61,-1,-1,63,65,67,69,-1,-1,71,73,75,77,-1,-1,-1,-1,79,81,83,85,87,89,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4521875E0,1.0828398E0,1.5312107E0,1.1583784E0,7.220619E-1,2.1875924E-1,1.1682189E0,1.8100941E-1,2.9705957E-1,2.3015897E0,5.818684E-1,0E0,7.8059874E-2,6.154116E-1,7.883315E-1,1.3265705E-1,1.5210027E-1,4.6964437E-2,3.1632537E-1,2.8652036E-1,3.2498074E-1,2.990239E-1,3.8193107E-1,1.4406722E-2,8.626334E-3,2.1743631E-1,6.209881E-1,3.904717E-1,6.421654E-1,1.3423026E-2,0E0,9.161997E-2,1.5366727E-1,0E0,0E0,2.8027752E-1,1.1743906E-1,1.7116553E-1,2.1191365E-1,0E0,0E0,8.954736E-2,3.4430423E-1,2.1634072E-1,1.6376793E-1,0E0,0E0,0E0,0E0,3.0491546E-1,1.541394E-2,3.732789E-1,7.4181175E-1,4.5320368E-1,3.9994597E-2,0E0,3.8845456E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,60,62,-1,-1,64,66,68,70,-1,-1,72,74,76,78,-1,-1,-1,-1,80,82,84,86,88,90,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,4.084788E0,1E0,2.71E2,1E0,5E0,5.862504E2,9.2E1,1.2E1,7.77E2,8.6875E0,-1.1394433E-2,1.81E2,4.9390244E0,1.0717949E1,1.43E2,3.1E1,1.4339622E0,7.4711205E6,2.3781754E8,6.845E3,6.34E2,1E0,7E0,9.45635E6,6.627033E7,4.5158855E6,1.7265024E3,7.9E1,1.3E1,-1.9969149E-2,5.4E1,2.831341E6,9.8194685E-3,1.2394614E-3,1.559287E5,3.3852024E7,4.36875E1,3.3817584E7,-3.379031E-2,-1.5163112E-2,1.27E2,1.81E2,3.3330768E2,2.1481E4,-1.1388025E-3,3.2166769E-3,-7.381643E-3,-2.8813253E-3,6.7459077E3,3E0,3.4187552E6,5.7774055E6,2E1,4.48375E5,2.2798559E-2,2.1199985E-7,-9.282413E-3,-5.252795E-3,-6.3675223E-3,3.913364E-4,-1.0336186E-2,2.4652635E-3,-8.125192E-4,5.0333436E-3,1.2872367E-3,1.0554062E-2,-6.2207542E-3,-1.1050076E-3,-1.5866663E-2,-5.72542E-3,-1.1093918E-3,-7.7239377E-3,8.243325E-3,-2.3098683E-3,-5.126135E-3,2.3676588E-3,-1.2424982E-2,-5.690155E-3,8.898672E-3,1.838718E-3,4.5047547E-3,1.323571E-2,-2.7598483E-3,2.6719356E-3,-2.7232133E-2,-5.7583617E-4,9.947419E-3,-2.2083186E-3,6.9688405E-3,2.0255372E-2,4.809811E-3,-1.4819051E-3],"split_indices":[2,66,17,2,27,6,64,41,10,41,70,0,0,68,68,2,10,65,40,43,41,2,24,6,59,12,59,4,45,3,0,2,5,0,0,45,7,67,7,0,0,0,10,4,1,0,0,0,0,45,68,57,59,3,1,0,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E2,4.92E2,4.88E2,2.87E2,2.05E2,2.5E1,4.63E2,5.5E1,2.32E2,9E1,1.15E2,1.3E1,1.2E1,1.93E2,2.7E2,2.4E1,3.1E1,9E0,2.23E2,7.7E1,1.3E1,4.8E1,6.7E1,5E0,7E0,8E1,1.13E2,6.9E1,2.01E2,2.2E1,2E0,1.9E1,1.2E1,7E0,2E0,2.11E2,1.2E1,5.6E1,2.1E1,6E0,7E0,2.2E1,2.6E1,3.1E1,3.6E1,2E0,3E0,3E0,4E0,7.3E1,7E0,1.05E2,8E0,5.9E1,1E1,3E0,1.98E2,1.4E1,8E0,5E0,1.4E1,1E1,2E0,1.93E2,1.8E1,5E0,7E0,2.4E1,3.2E1,7E0,1.4E1,1.7E1,5E0,1.6E1,1E1,1.7E1,1.4E1,1.5E1,2.1E1,1.6E1,5.7E1,2E0,5E0,5E1,5.5E1,3E0,5E0,5.2E1,7E0,2E0,8E0,1.75E2,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.953119E-3,-4.1091524E-2,8.00481E-2,-6.958575E-2,2.5026882E-2,-1.7370917E-1,9.782954E-2,-5.8237117E-2,-2.559077E-1,3.7842393E-2,-1.2311469E-1,-9.537862E-2,-1.8042112E-2,6.474791E-2,1.8428387E-1,-4.4590045E-2,-1.376788E-1,-1.520612E-1,-5.160538E-1,4.4611525E-2,-1.2687631E-2,-1.933468E-1,3.218208E-3,7.961395E-2,-1.473509E-1,7.4163914E-2,-1.6488613E-1,3.1211987E-1,8.1914805E-2,-1.1751223E-1,-2.7240992E-2,-1.5436399E-1,1.1367116E-2,-1.2363565E-2,-5.1555037E-2,-2.8669696E-2,-7.524521E-3,-4.616362E-2,7.257326E-2,-2.4645826E-1,-1.1716826E-3,5.7752933E-3,3.1935936E-4,-8.832487E-3,-3.8576166E-3,8.388431E-2,-9.97787E-2,-1.4631639E-3,-9.983157E-3,3.5160825E-1,7.008946E-2,1.2846628E-1,-1.0750593E-1,-4.187187E-3,-1.0475371E-2,-3.988848E-5,-3.3845778E-3,-8.578745E-3,3.843932E-3,-4.1677877E-3,2.2735149E-3,7.51788E-4,-6.1512357E-3,3.8297083E-3,-5.21029E-3,-1.52846575E-2,-5.09172E-3,1.020345E-3,5.2909004E-3,-2.1775415E-2,2.752279E-3,1.9216329E-2,8.671249E-3,-4.2266077E-3,7.043009E-3,-8.068312E-3,6.778856E-3,9.21721E-4,-1.2153821E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,-1,41,43,45,47,49,51,53,55,57,-1,-1,59,-1,-1,61,63,65,-1,-1,-1,-1,-1,67,69,-1,-1,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4152837E0,1.2357566E0,1.6458871E0,9.53537E-1,3.805639E-1,3.7441927E-1,9.5476484E-1,4.632963E-1,6.239252E-1,4.0137938E-1,2.3062594E-1,1.8453169E-1,0E0,5.441383E-1,1.191473E0,4.6616238E-1,4.430673E-1,2.2153327E-1,1.6435349E-1,4.6023172E-1,0E0,1.0068205E-1,0E0,1.588248E-2,2.6569426E-2,4.0908754E-1,6.286752E-2,3.6720896E-1,4.81155E-1,1.8604279E-1,3.6863437E-1,4.862392E-1,0E0,0E0,5.200738E-2,0E0,0E0,2.3815984E-1,2.8682363E-1,6.844914E-2,0E0,0E0,0E0,0E0,0E0,4.3184352E-1,8.0550325E-1,0E0,0E0,3.0147028E-1,1.0857646E-1,2.759621E-1,2.2577105E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,34,34,37,37,38,38,39,39,45,45,46,46,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,-1,42,44,46,48,50,52,54,56,58,-1,-1,60,-1,-1,62,64,66,-1,-1,-1,-1,-1,68,70,-1,-1,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,2.026453E6,4.8465605E3,5.27439E0,1.6438356E1,3.9E1,1.5142261E3,1.9E1,7.90913E5,1E0,1.7022566E7,1.8729467E8,-1.8042112E-2,1.6254545E1,1.5449402E7,7.7E1,4E1,6.875E0,2.402121E4,5.88E2,-1.2687631E-2,3.1146461E1,3.218208E-3,3.8795E4,1.3141646E3,3.3386944E5,1.9E1,8.913649E-3,6.747343E8,3.2856784E7,1E0,1.4831979E0,1.1367116E-2,-1.2363565E-2,7.547528E6,-2.8669696E-2,-7.524521E-3,5.801167E-1,4.7220547E1,1.266E3,-1.1716826E-3,5.7752933E-3,3.1935936E-4,-8.832487E-3,-3.8576166E-3,4.032683E6,6.27907E0,-1.4631639E-3,-9.983157E-3,1E0,1.0851E4,5.467E3,1.8421053E0,-4.187187E-3,-1.0475371E-2,-3.988848E-5,-3.3845778E-3,-8.578745E-3,3.843932E-3,-4.1677877E-3,2.2735149E-3,7.51788E-4,-6.1512357E-3,3.8297083E-3,-5.21029E-3,-1.52846575E-2,-5.09172E-3,1.020345E-3,5.2909004E-3,-2.1775415E-2,2.752279E-3,1.9216329E-2,8.671249E-3,-4.2266077E-3,7.043009E-3,-8.068312E-3,6.778856E-3,9.21721E-4,-1.2153821E-2],"split_indices":[2,9,40,65,68,3,64,8,9,76,9,7,0,66,59,41,3,65,45,2,0,68,0,1,64,45,3,69,44,57,24,50,0,0,5,0,0,54,70,2,0,0,0,0,0,57,68,0,0,82,2,2,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.014E3,6.54E2,3.6E2,4.57E2,1.97E2,2.3E1,3.37E2,4.32E2,2.5E1,1.82E2,1.5E1,1.8E1,5E0,2.45E2,9.2E1,3.7E2,6.2E1,1.9E1,6E0,1.79E2,3E0,1.1E1,4E0,4E0,1.4E1,2.36E2,9E0,4E1,5.2E1,7E1,3E2,6E1,2E0,8E0,1.1E1,4E0,2E0,4.2E1,1.37E2,8E0,3E0,2E0,2E0,7E0,7E0,2.24E2,1.2E1,3E0,6E0,3.4E1,6E0,4.2E1,1E1,5.8E1,1.2E1,1.92E2,1.08E2,5.3E1,7E0,8E0,3E0,2.5E1,1.7E1,1.3E2,7E0,4E0,4E0,7.6E1,1.48E2,3E0,9E0,2.3E1,1.1E1,2E0,4E0,2E0,4E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-1.5964098E-3,-3.961975E-2,9.004516E-2,-7.702076E-2,1.7466864E-2,1.18104465E-1,-9.52929E-2,-6.0338505E-2,-1.8499029E-1,-4.249085E-3,9.634328E-2,2.3355318E-2,1.0849367E-1,8.8533826E-2,-1.4401627E-1,-1.0456351E-1,-1.3649178E-2,-1.4188835E-1,-4.0389103E-1,-1.3431516E-1,1.4822456E-2,6.770941E-2,2.3846847E-1,4.3213565E-2,1.5215796E-1,9.28567E-3,-4.2802334E-2,-6.213902E-2,-2.589031E-1,-9.821632E-2,-2.1957358E-2,4.1423414E-2,-9.04346E-2,-1.5975082E-1,4.7524297E-3,-2.3457282E-3,-2.2846412E-2,-1.1533201E-2,-3.0729514E-1,2.1406135E-2,-1.4231314E-2,-3.4506865E-2,8.401421E-2,3.9774342E-3,1.2199995E-2,2.0761167E-1,1.7446676E-2,-1.2476795E-1,1.6430981E-1,2.8372137E-4,-3.5603503E-3,-1.2887295E-1,8.020585E-2,-3.37312E-1,8.4742096E-5,-3.916778E-3,-1.1831915E-2,7.36804E-3,6.669681E-4,-6.2966687E-3,3.3389044E-3,-9.354903E-3,-5.0145676E-3,3.4297744E-3,-4.040325E-3,-7.3193363E-3,-2.2163963E-2,1.5289262E-3,-3.3775389E-3,4.3793456E-3,-6.0394024E-3,6.4148162E-3,1.9867145E-3,1.0538645E-2,2.3464055E-3,1.9676553E-3,-5.2390904E-3,-7.2279724E-4,-1.216717E-2,9.215261E-3,3.1569526E-3,-4.543601E-4,-7.3684757E-3,-8.8977645E-4,5.6914366E-3,-1.7578699E-2,-4.767679E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,57,59,61,-1,-1,-1,63,65,67,-1,69,71,-1,-1,73,75,77,79,-1,-1,81,83,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.564746E0,1.54502E0,1.5713916E0,7.7595115E-1,4.9200484E-1,9.240601E-1,3.7101656E-1,7.832105E-1,4.8992503E-1,5.625975E-1,2.3616827E-1,0E0,7.231295E-1,1.4296588E-1,2.8492373E-1,4.1530776E-1,7.902978E-1,2.356621E-1,2.7152908E-1,6.123819E-1,4.298488E-1,9.122969E-2,1.822406E-2,4.3692476E-1,5.353708E-1,0E0,8.979655E-3,2.0032105E-1,2.6640695E-1,3.7614226E-1,0E0,3.4659386E-1,6.035823E-1,8.2164764E-2,0E0,0E0,0E0,1.251649E-1,2.3422408E-1,2.2240356E-1,0E0,1.1328111E-1,9.829447E-2,0E0,0E0,3.611827E-2,3.0876434E-1,1.02781534E-1,4.9651313E-1,0E0,0E0,5.0966978E-2,3.286496E-2,6.629622E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,31,31,32,32,33,33,37,37,38,38,39,39,41,41,42,42,45,45,46,46,47,47,48,48,51,51,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,58,60,62,-1,-1,-1,64,66,68,-1,70,72,-1,-1,74,76,78,80,-1,-1,82,84,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.56E3,7.785302E5,1E0,3.0923106E-10,3.7951445E6,3.3817584E7,1.1621469E1,6.07E2,8.13362E0,1.8154167E2,2.742234E6,2.3355318E-2,1.1251919E6,4.5154482E-1,1.1E1,4.784047E6,5.524E3,6.364486E0,1.6E1,2.2307692E0,2.511E3,1.994E3,1.0641107E8,3.6E1,7.549744E7,9.28567E-3,2.62E3,2.3751075E5,1.038946E6,3.1E1,-2.1957358E-2,2.857143E0,3.1146461E1,5E0,4.7524297E-3,-2.3457282E-3,-2.2846412E-2,5.004138E-3,1.2707816E6,1E0,-1.4231314E-2,7.1428573E-1,2.1111E4,3.9774342E-3,1.2199995E-2,1.2769117E7,4.4895835E0,3.307E3,1.853776E7,2.8372137E-4,-3.5603503E-3,8.0274E4,1.1388888E0,5.854E3,8.4742096E-5,-3.916778E-3,-1.1831915E-2,7.36804E-3,6.669681E-4,-6.2966687E-3,3.3389044E-3,-9.354903E-3,-5.0145676E-3,3.4297744E-3,-4.040325E-3,-7.3193363E-3,-2.2163963E-2,1.5289262E-3,-3.3775389E-3,4.3793456E-3,-6.0394024E-3,6.4148162E-3,1.9867145E-3,1.0538645E-2,2.3464055E-3,1.9676553E-3,-5.2390904E-3,-7.2279724E-4,-1.216717E-2,9.215261E-3,3.1569526E-3,-4.543601E-4,-7.3684757E-3,-8.8977645E-4,5.6914366E-3,-1.7578699E-2,-4.767679E-3],"split_indices":[2,40,6,49,40,7,68,2,66,64,9,0,40,50,8,57,41,66,0,68,2,12,5,0,7,0,2,45,9,8,0,70,68,6,0,0,0,50,40,82,0,68,9,0,0,59,65,2,59,0,0,1,65,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.021E3,7.22E2,2.99E2,4.36E2,2.86E2,2.6E2,3.9E1,3.79E2,5.7E1,2.25E2,6.1E1,5E0,2.55E2,8E0,3.1E1,1.94E2,1.85E2,4.9E1,8E0,2.8E1,1.97E2,5.2E1,9E0,1.03E2,1.52E2,4E0,4E0,1.9E1,1.2E1,1.92E2,2E0,1.08E2,7.7E1,4.6E1,3E0,2E0,6E0,1.7E1,1.1E1,1.94E2,3E0,7E0,4.5E1,2E0,7E0,1.3E1,9E1,6E0,1.46E2,2E0,2E0,1.3E1,6E0,9E0,3E0,1.79E2,1.3E1,1.9E1,8.9E1,6E1,1.7E1,2.3E1,2.3E1,8E0,9E0,7E0,4E0,1.73E2,2.1E1,3E0,4E0,1.8E1,2.7E1,1.1E1,2E0,7.6E1,1.4E1,4E0,2E0,1.05E2,4.1E1,3E0,1E1,2E0,4E0,7E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[2.1713583E-3,-5.628201E-2,4.659621E-2,-1.497882E-1,-3.341256E-2,6.4392954E-2,-9.539353E-2,-1.0198355E-1,-3.2206196E-1,3.96001E-2,-6.65992E-2,2.206992E-2,1.3054483E-1,-7.767994E-2,-1.5693773E-2,-6.1675068E-2,-2.1928105E-1,-3.789478E-1,-9.836916E-2,-1.6857517E-1,9.355954E-2,-1.4783436E-1,-4.6156485E-2,3.3454984E-2,-8.184211E-2,1.6957138E-1,-9.901988E-3,-1.09391645E-1,4.7824178E-2,-9.426189E-2,3.3507876E-2,-2.4445036E-1,-9.3428505E-4,-9.033626E-3,-2.4487173E-2,-8.499972E-3,9.695586E-4,-1.5294736E-2,-1.04405E-1,1.6951498E-1,3.7715107E-2,-1.7477582E-1,1.6045574E-3,-1.7557157E-2,-1.12914264E-1,-2.764153E-2,5.424305E-2,2.0573227E-2,-1.391992E-1,1.4334542E-1,3.0021262E-1,-2.3689369E-2,2.6466042E-2,-7.4570835E-2,-2.000823E-1,-2.2810638E-2,6.1563537E-3,-2.4390088E-3,-7.5314264E-3,3.3214577E-3,-3.747269E-3,-1.3352299E-2,-4.308891E-3,-6.2830173E-3,3.2870873E-4,1.0806113E-3,1.1373191E-2,-2.927458E-4,4.987045E-3,3.876903E-3,-8.719878E-3,1.6529262E-3,-2.8021312E-3,-9.531903E-3,-2.4623002E-3,2.5655623E-3,-3.9157993E-3,2.9878102E-3,-5.9471754E-3,-7.5313593E-3,4.6072174E-3,-4.0901736E-3,-1.1086682E-2,8.635078E-3,3.3020733E-3,1.7837526E-2,7.9171145E-3,4.960621E-3,-3.5788731E-3,-4.765975E-3,4.296752E-3,-5.2739144E-3,-1.6683005E-2,-3.0540973E-3,7.669131E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,-1,-1,-1,-1,-1,63,65,67,69,-1,71,73,75,77,79,81,83,85,-1,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6097918E0,9.2307615E-1,1.4490343E0,6.6717136E-1,8.50703E-1,1.4193909E0,2.5266314E-1,3.0798554E-1,1.9177234E-1,1.2514042E0,3.9339733E-1,3.7066504E-1,1.0869474E0,2.4800852E-1,0E0,1.6444743E-1,8.014989E-2,2.885735E-1,5.5692967E-2,2.07896E-1,3.6687535E-1,2.4245954E-1,3.684077E-1,3.5839143E-1,1.8496539E-1,4.8802614E-1,8.231106E-1,1.398179E-1,8.221908E-2,1.0319486E-1,6.842597E-2,7.723421E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.601329E-2,4.1601992E-1,1.6314739E-1,1.8006253E-1,0E0,3.2091996E-1,3.143102E-1,3.5111237E-1,4.2534143E-1,1.9301647E-1,8.098689E-2,4.044633E-1,2.1475935E-1,0E0,3.6587033E-1,1.9128956E-1,1.3911015E-1,1.49162635E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,-1,-1,-1,-1,-1,64,66,68,70,-1,72,74,76,78,80,82,84,86,-1,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.095006E5,3.3820656E7,1E0,1.0273995E-8,9.384615E0,5.817547E2,1.318849E6,2E0,1.753E3,3.9125E1,1.2597849E2,5.2E1,1.7493458E7,1.038946E6,-1.5693773E-2,1.02E2,6.875E0,2.231884E0,4.5E1,1.1010101E0,2.9565217E0,7.997723E6,3.82E2,1.5933E4,5.8606E4,1.9851073E3,3.167E3,1.3E1,1.06E2,1.139605E6,1.2586111E3,1.2269693E4,-9.3428505E-4,-9.033626E-3,-2.4487173E-2,-8.499972E-3,9.695586E-4,-1.5294736E-2,7.2E1,3.72E2,2.7664474E2,6.2085107E3,1.6045574E-3,4.2619047E0,4.488263E0,1.4007937E0,4.5620965E6,1.8235294E0,1.9931606E6,2E0,8.959487E8,-2.3689369E-2,2.0421052E0,2.2539758E5,1.7951853E1,5.525965E2,6.1563537E-3,-2.4390088E-3,-7.5314264E-3,3.3214577E-3,-3.747269E-3,-1.3352299E-2,-4.308891E-3,-6.2830173E-3,3.2870873E-4,1.0806113E-3,1.1373191E-2,-2.927458E-4,4.987045E-3,3.876903E-3,-8.719878E-3,1.6529262E-3,-2.8021312E-3,-9.531903E-3,-2.4623002E-3,2.5655623E-3,-3.9157993E-3,2.9878102E-3,-5.9471754E-3,-7.5313593E-3,4.6072174E-3,-4.0901736E-3,-1.1086682E-2,8.635078E-3,3.3020733E-3,1.7837526E-2,7.9171145E-3,4.960621E-3,-3.5788731E-3,-4.765975E-3,4.296752E-3,-5.2739144E-3,-1.6683005E-2,-3.0540973E-3,7.669131E-4],"split_indices":[40,7,6,49,70,64,9,8,41,4,64,8,63,9,0,2,65,65,3,65,70,9,0,1,9,64,2,8,0,43,45,59,0,0,0,0,0,0,2,2,4,59,0,66,66,65,59,65,59,29,12,0,65,45,68,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.003E3,4.33E2,5.7E2,8.4E1,3.49E2,5.07E2,6.3E1,6.7E1,1.7E1,1.09E2,2.4E2,3.1E2,1.97E2,6E1,3E0,5.1E1,1.6E1,1.3E1,4E0,2.2E1,8.7E1,4.7E1,1.93E2,2.8E2,3E1,1.54E2,4.3E1,4.8E1,1.2E1,3.8E1,1.3E1,1.4E1,2E0,7E0,6E0,2E0,2E0,5E0,1.7E1,3.6E1,5.1E1,4.1E1,6E0,1.36E2,5.7E1,7.1E1,2.09E2,1.1E1,1.9E1,1.3E2,2.4E1,2E0,4.1E1,3.6E1,1.2E1,7E0,5E0,2.5E1,1.3E1,1E1,3E0,1E1,4E0,1.3E1,4E0,1.3E1,2.3E1,3.2E1,1.9E1,2E0,3.9E1,6.1E1,7.5E1,2.1E1,3.6E1,2.9E1,4.2E1,1.98E2,1.1E1,3E0,8E0,1.4E1,5E0,7.9E1,5.1E1,1.3E1,1.1E1,2.3E1,1.8E1,3.1E1,5E0,9E0,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[4.2408686E-3,-1.6000811E-2,1.6834393E-1,-5.725712E-2,2.0666527E-2,2.3667073E-1,8.161835E-2,-3.8817458E-2,-1.5736885E-1,-2.1235984E-2,5.9748195E-2,1.9299074E-1,3.9894813E-1,3.647484E-2,2.0729798E-1,-1.096558E-1,-1.4793979E-2,-1.8350038E-1,8.536383E-2,-5.4896425E-2,5.1447924E-2,2.3128419E-1,4.9139347E-2,1.2449706E-4,2.0507862E-1,7.914801E-3,2.0914717E-2,1.5445356E-1,-4.7183182E-2,5.877645E-2,2.6677683E-1,-9.134319E-2,-2.4702239E-1,1.2673053E-1,-3.598953E-2,-6.746153E-2,-2.2886227E-1,-5.628444E-4,5.8269566E-3,9.700646E-2,-7.04464E-2,-2.026769E-3,9.5615216E-2,2.418062E-4,2.9244733E-1,-1.0663174E-3,9.441549E-2,1.2925115E-1,2.3658365E-1,1.98204E-1,1.8561807E-2,-8.430037E-3,-1.7553702E-2,7.7195675E-3,-3.2200785E-3,1.4249244E-2,6.1964495E-3,-2.6364431E-3,-7.7885226E-3,-1.740485E-2,-4.4061593E-3,8.105114E-3,-1.6783942E-3,-2.4815237E-3,1.4287711E-3,-6.687679E-4,-6.8214005E-3,-1.370175E-2,-6.9894176E-3,-2.3106346E-3,7.126184E-3,-1.4083206E-2,-2.6578424E-3,-4.3808916E-3,5.630179E-3,8.305361E-3,2.4178752E-3,1.4606308E-2,5.433789E-3,-6.4976225E-3,1.2690141E-3,5.376565E-3,-4.7962214E-3,7.408645E-3,1.216708E-3,1.3441059E-2,7.052923E-3,1.0558849E-2,1.2191018E-3,2.8304935E-3,-1.4095393E-3,4.361228E-4,-2.3732053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,-1,-1,49,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,-1,77,79,81,83,85,87,89,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.327737E0,1.3505642E0,6.325078E-1,7.692226E-1,7.7586305E-1,3.5539174E-1,2.7611983E-1,6.042127E-1,4.2795765E-1,5.630894E-1,4.3448555E-1,1.1444843E-1,7.491219E-2,3.8234255E-1,9.916037E-2,2.0383501E-1,8.054783E-1,2.951213E-1,2.9911108E-2,3.7768137E-1,1.7260754E-1,1.9199741E-1,5.274643E-1,0E0,8.1701756E-2,0E0,0E0,9.144613E-2,8.96989E-2,8.221054E-2,9.717584E-3,2.05342E-1,1.6295385E-1,2.906317E-1,2.836604E-1,7.50163E-2,1.7829561E-1,0E0,0E0,1.3627936E-1,4.071446E-1,4.0718132E-1,1.3694704E-1,0E0,1.1331737E-2,4.51088E-1,5.614389E-1,4.841748E-2,1.03986025E-1,5.9154153E-2,1.2495139E-2,0E0,1.894575E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,-1,-1,50,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,-1,78,80,82,84,86,88,90,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.098E3,5.095006E5,8.710612E1,2.2E1,7.582512E7,4.907764E3,4.435876E2,2.71E2,1.75E2,3.2911258E6,1.0164831E-2,1E0,1.8333334E0,7.153514E1,8E0,6.999242E7,3.6E1,6.511E3,7E0,2.132956E8,1.594941E3,1.6494E4,7.5491136E-1,1.2449706E-4,1.2697166E3,7.914801E-3,2.0914717E-2,2.1E1,1.926089E6,7.61E2,1E0,2.772925E3,1.76421E6,9.566785E5,2.041E3,9.182E2,2.022131E6,-5.628444E-4,5.8269566E-3,1.3781496E8,1.3351785E2,2.6818182E0,1.125584E-3,2.418062E-4,1.5E1,1.027972E0,1.766486E7,1.2E1,2.1089442E0,9.522029E6,1.069E3,-8.430037E-3,6.44833E-1,7.7195675E-3,-3.2200785E-3,1.4249244E-2,6.1964495E-3,-2.6364431E-3,-7.7885226E-3,-1.740485E-2,-4.4061593E-3,8.105114E-3,-1.6783942E-3,-2.4815237E-3,1.4287711E-3,-6.687679E-4,-6.8214005E-3,-1.370175E-2,-6.9894176E-3,-2.3106346E-3,7.126184E-3,-1.4083206E-2,-2.6578424E-3,-4.3808916E-3,5.630179E-3,8.305361E-3,2.4178752E-3,1.4606308E-2,5.433789E-3,-6.4976225E-3,1.2690141E-3,5.376565E-3,-4.7962214E-3,7.408645E-3,1.216708E-3,1.3441059E-2,7.052923E-3,1.0558849E-2,1.2191018E-3,2.8304935E-3,-1.4095393E-3,4.361228E-4,-2.3732053E-3],"split_indices":[2,40,70,3,7,4,68,2,6,40,50,14,65,68,3,7,10,1,8,43,45,1,50,0,4,0,0,8,1,8,26,59,43,57,2,45,12,0,0,43,67,66,50,0,3,65,63,3,50,9,0,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,8.91E2,1.09E2,4.19E2,4.72E2,6E1,4.9E1,3.55E2,6.4E1,2.28E2,2.44E2,4.9E1,1.1E1,3.7E1,1.2E1,8.9E1,2.66E2,5.8E1,6E0,1.56E2,7.2E1,1.3E1,2.31E2,3E0,4.6E1,3E0,8E0,1.5E1,2.2E1,4E0,8E0,8E1,9E0,3.4E1,2.32E2,1.7E1,4.1E1,2E0,4E0,1.4E1,1.42E2,3.3E1,3.9E1,3E0,1E1,1.1E2,1.21E2,1.5E1,3.1E1,1.1E1,4E0,3E0,1.9E1,2E0,2E0,5E0,3E0,5.7E1,2.3E1,4E0,5E0,2.6E1,8E0,1.83E2,4.9E1,1.1E1,6E0,2E1,2.1E1,4E0,1E1,6E0,1.36E2,1.9E1,1.4E1,1.2E1,2.7E1,8E0,2E0,1.8E1,9.2E1,1.09E2,1.2E1,1.1E1,4E0,1.7E1,1.4E1,9E0,2E0,2E0,2E0,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-2.6324631E-3,-4.9774747E-2,4.6127632E-2,-4.0884264E-2,-1.9705063E-1,-1.6665176E-1,5.9186213E-2,-8.297844E-2,-4.5786346E-3,-4.609738E-1,-3.3881936E-2,-2.166477E-1,-7.777019E-2,3.765473E-2,1.3826083E-1,-7.511736E-2,-1.6415415E-2,2.4544837E-2,-6.3442454E-2,-6.5978873E-3,-2.3679107E-2,-7.1865916E-2,6.0394336E-2,-7.1735354E-3,-1.4310232E-2,-4.5782016E-4,-5.07202E-3,4.3468464E-2,-2.7003866E-1,1.8847123E-1,-3.8029224E-2,-6.1711423E-2,-1.8202785E-1,9.604707E-2,-7.0459405E-3,-7.928084E-2,1.635892E-1,-1.3566925E-1,3.5816744E-2,4.2185606E-3,-7.673707E-5,1.430409E-1,2.6414596E-2,-7.0374524E-3,-1.7211933E-2,1.5913369E-1,3.5269675E-1,1.1709165E-2,-9.471889E-2,-4.1728523E-3,-7.8262435E-4,-1.1443148E-2,7.595687E-4,2.3423436E-3,7.314935E-3,-1.8495038E-3,2.558214E-3,6.5362924E-3,-4.2472277E-3,1.4816112E-2,1.5877349E-4,-1.9169724E-4,-7.929771E-3,-1.6009284E-3,5.4248082E-3,-7.804397E-5,9.196407E-3,9.6035665E-3,6.0569606E-4,3.5027813E-3,9.176949E-3,6.119041E-3,1.7760206E-2,5.3943456E-3,-6.387593E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,-1,-1,37,39,-1,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,-1,65,67,-1,-1,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3239448E0,6.649518E-1,1.3908261E0,7.423524E-1,1.233237E0,1.1201912E-1,7.933103E-1,4.5851564E-1,4.5070872E-1,1.1638141E-1,7.237905E-2,5.4643154E-2,2.5468878E-2,6.7361176E-1,8.923267E-1,3.0407858E-1,0E0,3.9822754E-1,3.2580534E-1,0E0,0E0,1.0104641E-1,1.1865182E-2,0E0,0E0,0E0,0E0,6.143284E-1,2.8197527E-2,3.1967568E-1,4.0612015E-1,2.5488E-1,3.2392198E-1,1.456523E-1,2.5861543E-1,2.5278175E-1,1.5163232E-1,4.303676E-2,3.900969E-2,0E0,0E0,4.3897247E-1,7.511267E-1,0E0,0E0,2.1391082E-1,3.0499697E-2,0E0,2.045151E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,21,21,22,22,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,45,45,46,46,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,-1,-1,38,40,-1,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,-1,66,68,-1,-1,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.194E3,9.932432E0,1E0,1.5422421E2,2E0,1.6074808E5,8.439E3,3.2E1,2.192E3,7.460487E7,1.6254545E1,2.9922134E-8,3.6451373E8,9.185E1,3.1879792E8,2.1E1,-1.6415415E-2,1.3695652E0,3.2E1,-6.5978873E-3,-2.3679107E-2,1.54E2,9E0,-7.1735354E-3,-1.4310232E-2,-4.5782016E-4,-5.07202E-3,8.856631E0,3.907812E7,9.62385E3,6.5199727E-1,3.06E2,1.547E3,2.6947792E0,1.339646E6,1.6379E2,2.8266037E2,4.6463413E0,8.23E2,4.2185606E-3,-7.673707E-5,3.7788504E7,1.3358E4,-7.0374524E-3,-1.7211933E-2,1E0,1.5E1,1.1709165E-2,4.831007E6,-4.1728523E-3,-7.8262435E-4,-1.1443148E-2,7.595687E-4,2.3423436E-3,7.314935E-3,-1.8495038E-3,2.558214E-3,6.5362924E-3,-4.2472277E-3,1.4816112E-2,1.5877349E-4,-1.9169724E-4,-7.929771E-3,-1.6009284E-3,5.4248082E-3,-7.804397E-5,9.196407E-3,9.6035665E-3,6.0569606E-4,3.5027813E-3,9.176949E-3,6.119041E-3,1.7760206E-2,5.3943456E-3,-6.387593E-3],"split_indices":[2,66,17,64,29,45,2,3,41,43,66,49,7,66,44,8,0,65,3,0,0,0,0,0,0,0,0,70,5,64,54,2,10,66,9,64,4,65,2,0,0,7,9,0,0,27,3,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.009E3,5.13E2,4.96E2,4.85E2,2.8E1,2.8E1,4.68E2,2.24E2,2.61E2,1E1,1.8E1,1.7E1,1.1E1,3.69E2,9.9E1,2.19E2,5E0,1.75E2,8.6E1,2E0,8E0,1.3E1,5E0,1.2E1,5E0,4E0,7E0,3.63E2,6E0,7.7E1,2.2E1,1.96E2,2.3E1,5.3E1,1.22E2,8.1E1,5E0,8E0,5E0,3E0,2E0,5.2E1,3.11E2,4E0,2E0,6.7E1,1E1,3E0,1.9E1,1.18E2,7.8E1,1.7E1,6E0,3.2E1,2.1E1,8E1,4.2E1,4E0,7.7E1,2E0,3E0,2E0,6E0,3E0,2E0,1.5E1,3.7E1,2E1,2.91E2,2.3E1,4.4E1,2E0,8E0,3E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[2.7873316E-3,-2.5221283E-2,7.465983E-2,-6.7042015E-2,1.1838654E-2,2.4998458E-1,5.9102524E-2,1.19025655E-1,-7.704865E-2,-4.8866263E-3,9.110964E-2,1.23360105E-1,2.783047E-1,-1.8622186E-2,9.062221E-2,-5.5078365E-2,2.1066752E-1,-6.6339925E-2,-2.1617134E-1,2.3464104E-3,-2.1025516E-1,1.1245126E-2,6.35122E-2,7.3734927E-4,7.946169E-3,1.7401804E-1,3.288491E-1,-3.7091464E-1,5.762533E-3,1.7528059E-1,5.1503234E-2,-6.156013E-3,1.7288029E-3,2.6640266E-1,1.497211E-3,-1.4783363E-1,-4.984571E-2,2.2790874E-3,-2.4384753E-1,9.1605205E-3,-1.4383784E-1,-2.6601529E-2,4.7711283E-4,1.1718547E-2,4.1045103E-2,9.299938E-3,2.7145725E-3,1.6384482E-2,6.229899E-3,-2.2843841E-2,-5.550632E-3,6.4374425E-2,-4.1553445E-2,9.252879E-2,2.921049E-1,2.2348957E-2,1.8616231E-1,4.8859026E-3,1.363841E-2,-5.966536E-3,-1.7980447E-2,-1.4702086E-3,-7.781299E-3,-1.4191556E-2,-7.1908263E-3,5.508144E-4,-1.2638538E-2,-2.1761453E-3,-8.826317E-3,-3.4450593E-3,4.3501533E-3,4.7413004E-3,-1.6173876E-3,-1.284119E-3,5.441358E-3,3.6449893E-3,-3.3653963E-3,-4.5240074E-3,5.184718E-3,9.10641E-3,1.5002891E-2,1.573093E-3,-1.0547454E-2,1.2169503E-2,2.4432424E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,57,-1,59,61,-1,63,65,67,-1,69,-1,71,-1,-1,-1,-1,-1,-1,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.05318E0,1.138401E0,7.653134E-1,6.5153015E-1,5.175187E-1,4.8146725E-2,6.4845014E-1,2.9879722E-1,4.743222E-1,4.8028988E-1,2.7291423E-1,2.9183157E-2,3.765416E-2,6.636099E-1,6.135813E-1,5.6282975E-2,1.1041391E-1,4.0394855E-1,1.812892E-1,3.131018E-1,8.5540855E-1,0E0,2.4352965E-1,0E0,0E0,1.41943395E-2,1.0842681E-2,9.79799E-2,2.0512435E-1,5.4334986E-1,5.0620484E-1,0E0,0E0,1.238054E-2,0E0,1.7196727E-1,5.4054457E-1,0E0,7.101071E-2,2.4482031E-1,5.6155056E-2,0E0,6.400589E-2,0E0,2.5968927E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6812807E-1,1.6304296E-1,1.5106475E-1,1.301384E-2,3.317047E-1,3.2629377E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,36,36,38,38,39,39,40,40,42,42,44,44,51,51,52,52,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,58,-1,60,62,-1,64,66,68,-1,70,-1,72,-1,-1,-1,-1,-1,-1,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.652E3,4.4761734E5,1.059448E7,7E0,5.281628E2,4.7185095E2,3.4183568E8,3.72E2,2.0734E4,4.5620965E6,8E0,1.25789E5,3E0,1.2825651E0,1.0717949E1,1.63757E4,1.07E2,3.9125E1,1.5254237E-1,2.727619E1,2.1512408E5,1.1245126E-2,1.0918E4,7.3734927E-4,7.946169E-3,1.8E1,1.05039425E1,1E0,4.88451E5,1.025173E3,3.6966505E0,-6.156013E-3,1.7288029E-3,3.3527696E-1,1.497211E-3,1.3E2,1.3439851E7,2.2790874E-3,1.03860024E8,5.6E1,1.5E1,-2.6601529E-2,2.466E3,1.1718547E-2,2E0,9.299938E-3,2.7145725E-3,1.6384482E-2,6.229899E-3,-2.2843841E-2,-5.550632E-3,5.7621355E4,2.2830357E1,1.0950326E6,4.053E3,3.206931E2,1.034073E6,4.8859026E-3,1.363841E-2,-5.966536E-3,-1.7980447E-2,-1.4702086E-3,-7.781299E-3,-1.4191556E-2,-7.1908263E-3,5.508144E-4,-1.2638538E-2,-2.1761453E-3,-8.826317E-3,-3.4450593E-3,4.3501533E-3,4.7413004E-3,-1.6173876E-3,-1.284119E-3,5.441358E-3,3.6449893E-3,-3.3653963E-3,-4.5240074E-3,5.184718E-3,9.10641E-3,1.5002891E-2,1.573093E-3,-1.0547454E-2,1.2169503E-2,2.4432424E-4],"split_indices":[2,40,12,3,4,4,7,2,41,59,3,1,8,66,68,40,0,4,65,58,45,0,9,0,0,3,66,8,9,4,47,0,0,68,0,2,57,0,7,3,3,0,41,0,29,0,0,0,0,0,0,45,70,59,10,70,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.33E2,2.85E2,3.44E2,3.89E2,2.2E1,2.63E2,1.7E1,3.27E2,3.22E2,6.7E1,5E0,1.7E1,7.6E1,1.87E2,6E0,1.1E1,3.05E2,2.2E1,3.12E2,1E1,9E0,5.8E1,2E0,3E0,7E0,1E1,4E0,7.2E1,5.8E1,1.29E2,3E0,3E0,8E0,3E0,5E1,2.55E2,2E0,2E1,2.99E2,1.3E1,3E0,7E0,5E0,5.3E1,5E0,2E0,8E0,2E0,2E0,2E0,3.2E1,4E1,3.5E1,2.3E1,1.07E2,2.2E1,2E0,6E0,4.8E1,2E0,2.23E2,3.2E1,1E1,1E1,2.97E2,2E0,5E0,8E0,4E0,3E0,2.9E1,2.4E1,1.2E1,2E1,8E0,3.2E1,3E0,3.2E1,8E0,1.5E1,1.03E2,4E0,1.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[3.525591E-3,-4.4778958E-2,5.9083927E-2,-6.86015E-2,3.767498E-2,7.876612E-2,-8.9991145E-2,-4.247075E-2,-1.1408911E-1,7.506059E-2,-2.1193631E-2,5.5407755E-2,2.0505995E-1,-7.375416E-2,-1.568264E-2,-1.251099E-1,-1.9909091E-2,-2.5017798E-2,-1.01510786E-1,1.0695952E-1,-3.6309972E-2,-2.4879278E-1,1.1825142E-2,4.0707607E-2,1.727114E-1,2.9756024E-1,1.0615154E-1,-9.321731E-2,6.468284E-2,-3.1759426E-2,-1.7608906E-1,2.831181E-2,-5.8732E-2,-7.668947E-2,-2.3792687E-1,9.013017E-2,1.4120087E-2,-7.538461E-2,3.0280164E-2,-2.7811192E-2,3.7172385E-3,1.386767E-1,-2.6973061E-2,8.946334E-3,2.6953535E-2,2.621291E-1,1.1572904E-1,1.5878081E-1,3.449617E-1,1.2949133E-1,-4.115456E-3,3.814389E-2,-1.1387509E-1,-3.8588792E-3,6.4747036E-3,5.818617E-4,-4.847502E-3,-7.025881E-3,-1.7429868E-2,2.1881564E-3,-8.40547E-3,1.254743E-3,-4.3035513E-3,-5.2580326E-3,-5.302904E-4,-1.3265525E-2,8.295605E-3,4.8783235E-3,-2.5812786E-3,-4.4296663E-3,-6.626294E-4,3.1663538E-3,-2.0320218E-3,8.066983E-3,-2.7983743E-4,-5.2493108E-3,1.0048887E-3,1.8197285E-3,-3.728403E-3,1.3376194E-2,5.3554475E-3,1.3378691E-3,7.469479E-3,8.67491E-3,1.1770685E-3,3.8117794E-3,1.6809097E-2,6.6549545E-3,-2.399804E-3,-3.5543414E-3,4.585416E-3,-6.327531E-3,-1.3064928E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,-1,-1,73,75,-1,77,79,81,83,85,87,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6487942E0,1.0397149E0,1.3538511E0,4.8249102E-1,2.6349065E-1,1.1863453E0,1.9528297E-1,4.860019E-1,7.517023E-1,2.6322365E-1,3.5614914E-1,5.861565E-1,5.4480386E-1,1.4613941E-1,0E0,2.6090753E-1,3.8918248E-1,0E0,4.7611332E-1,1.6458488E-1,4.651641E-2,7.5467384E-1,2.1044827E-1,6.4530116E-1,1.6328275E-1,1.5573382E-1,1.5780377E-1,1.2917063E-1,9.124234E-2,7.0170164E-2,1.1127198E-1,3.885458E-1,3.485422E-1,3.0637032E-1,5.2068865E-1,1.3301605E-1,0E0,1.2727588E-2,2.321448E-2,0E0,0E0,5.4567814E-2,1.4325532E-1,0E0,3.907546E-1,2.5316715E-2,9.678829E-2,3.521979E-2,9.6150875E-2,8.8719904E-2,0E0,5.7611547E-2,7.770175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,-1,-1,74,76,-1,78,80,82,84,86,88,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.541824E2,1.5115256E6,2.856934E7,3.6245735E0,6.93729E5,1.4176E4,1.0428572E1,4.54E2,1.1E1,2.810313E4,2.8024E4,4.284608E6,5.4123135E1,6.1454544E0,-1.568264E-2,5.3E1,7.2647057E0,-2.5017798E-2,3.0084E4,1.3177083E1,1.0079E5,5E0,1.2051282E0,3.63E2,1.6166983E0,1.41565E3,4.49239E6,1.3705882E1,2.5393645E5,5.853E3,1.02E2,1E0,1.4392524E0,1.0590052E8,6.23694E5,2.9E1,1.4120087E-2,3E0,2.466E3,-2.7811192E-2,3.7172385E-3,4.16E2,4.3460325E6,8.946334E-3,4.142857E0,9.026191E8,5E0,7.4391775E6,2.77843E5,7.61E2,-4.115456E-3,5.1E1,2.9925186E-2,-3.8588792E-3,6.4747036E-3,5.818617E-4,-4.847502E-3,-7.025881E-3,-1.7429868E-2,2.1881564E-3,-8.40547E-3,1.254743E-3,-4.3035513E-3,-5.2580326E-3,-5.302904E-4,-1.3265525E-2,8.295605E-3,4.8783235E-3,-2.5812786E-3,-4.4296663E-3,-6.626294E-4,3.1663538E-3,-2.0320218E-3,8.066983E-3,-2.7983743E-4,-5.2493108E-3,1.0048887E-3,1.8197285E-3,-3.728403E-3,1.3376194E-2,5.3554475E-3,1.3378691E-3,7.469479E-3,8.67491E-3,1.1770685E-3,3.8117794E-3,1.6809097E-2,6.6549545E-3,-2.399804E-3,-3.5543414E-3,4.585416E-3,-6.327531E-3,-1.3064928E-3],"split_indices":[64,40,57,68,9,2,65,1,12,63,12,40,70,66,0,41,70,0,41,68,9,8,65,41,66,4,40,68,40,9,2,82,65,7,1,8,0,8,41,0,0,10,57,0,65,5,3,57,1,8,0,0,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.85E2,5.27E2,4.58E2,4.09E2,1.18E2,4.05E2,5.3E1,2.61E2,1.48E2,7.2E1,4.6E1,3.43E2,6.2E1,5.1E1,2E0,5.5E1,2.06E2,3E0,1.45E2,5.6E1,1.6E1,5E0,4.1E1,3.06E2,3.7E1,3.1E1,3.1E1,4.5E1,6E0,2E1,3.5E1,9.2E1,1.14E2,1.24E2,2.1E1,5.3E1,3E0,1E1,6E0,2E0,3E0,9E0,3.2E1,2.4E1,2.82E2,1.3E1,2.4E1,9E0,2.2E1,2.8E1,3E0,6E0,3.9E1,2E0,4E0,1.3E1,7E0,3.3E1,2E0,8.5E1,7E0,3.3E1,8.1E1,7.8E1,4.6E1,1.9E1,2E0,4.8E1,5E0,7E0,3E0,4E0,2E0,7E0,2E0,1.1E1,2.1E1,2.53E2,2.9E1,1E1,3E0,9E0,1.5E1,7E0,2E0,2E0,2E1,2.6E1,2E0,2E0,4E0,3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-4.9132667E-3,-2.771539E-2,8.778702E-2,-1.226133E-2,-1.3072416E-1,1.6779233E-2,7.445975E-2,-1.0161934E-1,7.9570245E-3,-2.97658E-1,-9.5884904E-2,8.9624495E-4,1.260754E-1,-1.400336E-1,-7.322955E-2,9.294784E-2,-1.3081124E-2,-1.10593244E-1,-2.6448203E-2,-1.5272872E-1,-1.7634552E-2,1.6538596E-2,-9.0664085E-3,-2.4357085E-1,1.4526625E-1,-1.24495916E-1,-1.6896147E-2,9.6586505E-3,-1.12509996E-1,2.1309292E-1,6.0943216E-2,-7.361754E-2,8.179003E-3,-7.3263296E-3,1.2450856E-3,-1.7305629E-1,7.296158E-2,4.680799E-2,-9.541796E-2,-8.827071E-2,3.756957E-2,-2.3489005E-3,-1.5019705E-2,-9.781293E-2,1.5789326E-1,-8.650969E-3,-4.3329964E-3,-6.3427873E-3,2.7967033E-3,-5.558083E-3,3.1903468E-3,-3.233081E-5,1.0681886E-2,1.2690678E-3,6.306023E-3,2.923047E-3,-5.0363513E-3,6.579379E-4,-9.23063E-3,-3.1432295E-3,-9.292457E-3,1.3717985E-5,5.571679E-3,-5.163684E-3,4.06209E-3,-9.3968585E-3,3.0596172E-3,-6.425069E-3,5.9894747E-3,4.3353834E-3,5.139081E-5,-8.184106E-3,-6.009695E-4,3.8234794E-3,8.527106E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,-1,41,43,45,-1,47,49,51,53,55,57,-1,-1,59,61,63,65,67,69,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.162699E0,1.3053502E0,7.199197E-1,1.2925789E0,5.9375525E-1,0E0,7.365104E-1,1.3370109E-1,1.0454367E0,8.729286E-1,3.976311E-1,2.544987E-1,8.4407866E-1,1.488936E-1,2.5504583E-1,4.3305635E-1,6.0421073E-1,8.5146606E-2,0E0,2.511884E-1,1.999562E-1,1.7083867E-1,0E0,7.9606056E-2,3.5000467E-1,8.4293365E-2,0E0,2.0520882E-1,9.036231E-2,1.0005677E-1,2.3274755E-1,6.103375E-1,4.5014718E-1,0E0,0E0,1.3307023E-1,1.7527267E-2,1.5577374E-1,3.3201587E-1,1.649617E-1,1.3244286E-1,0E0,0E0,3.8525414E-2,2.0002437E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,-1,42,44,46,-1,48,50,52,54,56,58,-1,-1,60,62,64,66,68,70,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.812E3,2.2E1,7.122041E7,3.89E2,5.862757E1,1.6779233E-2,1.1251919E6,3.89E2,5.0442476E0,3.292496E-2,5.095006E5,3.0070068E7,2.0750147E8,1.43E2,3.83E2,1.783854E8,2.61E2,4.23E2,-2.6448203E-2,2.3186667E2,7.3066E4,2.89196E6,-9.0664085E-3,2.192E3,2.0778275E-1,1.7857143E1,-1.6896147E-2,6.123E3,5.1984615E1,5.445545E-1,6.763312E7,1.1481482E0,3.174172E5,-7.3263296E-3,1.2450856E-3,1E0,3.4380876E-3,8.3E2,2.3601625E1,1.926089E6,3.62349E5,-2.3489005E-3,-1.5019705E-2,4E0,5.766206E6,-8.650969E-3,-4.3329964E-3,-6.3427873E-3,2.7967033E-3,-5.558083E-3,3.1903468E-3,-3.233081E-5,1.0681886E-2,1.2690678E-3,6.306023E-3,2.923047E-3,-5.0363513E-3,6.579379E-4,-9.23063E-3,-3.1432295E-3,-9.292457E-3,1.3717985E-5,5.571679E-3,-5.163684E-3,4.06209E-3,-9.3968585E-3,3.0596172E-3,-6.425069E-3,5.9894747E-3,4.3353834E-3,5.139081E-5,-8.184106E-3,-6.009695E-4,3.8234794E-3,8.527106E-3],"split_indices":[2,3,7,2,59,0,40,1,70,70,40,55,7,2,12,43,67,41,0,58,1,59,0,41,50,4,0,40,58,65,7,65,45,0,0,8,49,41,68,1,1,0,0,3,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.021E3,8.2E2,2.01E2,7.14E2,1.06E2,8E0,1.93E2,1.31E2,5.83E2,1.7E1,8.9E1,8E1,1.13E2,5.4E1,7.7E1,1.15E2,4.68E2,1.1E1,6E0,5.1E1,3.8E1,7.5E1,5E0,5E0,1.08E2,5.2E1,2E0,2.5E1,5.2E1,2.3E1,9.2E1,1.21E2,3.47E2,8E0,3E0,4.7E1,4E0,2.1E1,1.7E1,1.2E1,6.3E1,2E0,3E0,5E0,1.03E2,1.5E1,3.7E1,6E0,1.9E1,5E1,2E0,2E0,2.1E1,6.5E1,2.7E1,2.5E1,9.6E1,3.38E2,9E0,1.1E1,3.6E1,2E0,2E0,4E0,1.7E1,1E1,7E0,1E1,2E0,2.4E1,3.9E1,2E0,3E0,2.9E1,7.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.6790269E-3,-3.9343525E-2,4.3897018E-2,-4.663743E-2,2.111083E-1,1.1244351E-2,9.185552E-2,-3.4634832E-2,-1.3304056E-1,1.5214396E-2,4.957086E-2,2.2769235E-3,1.2002611E-1,1.1115373E-1,-5.793355E-2,9.037344E-2,-4.2302083E-2,-2.831564E-1,-7.5180866E-2,-2.191658E-2,5.8120606E-3,1.6818528E-1,-4.4492325E-3,1.6801052E-2,1.8225683E-1,1.21717595E-1,-4.8765942E-2,-5.4497225E-4,-9.685995E-3,-7.596583E-2,1.494373E-1,-2.877097E-2,-1.0392272E-1,-7.4675314E-2,-4.833402E-1,-1.6877262E-2,-4.8132334E-2,-2.5595645E-3,8.8179385E-4,9.523992E-3,3.5272184E-4,-4.2583812E-2,2.3123305E-2,3.4583022E-3,-2.499344E-2,2.048655E-1,3.6823764E-3,9.1466084E-2,2.1012685E-1,7.2117085E-3,-1.8878213E-1,4.6261847E-2,-1.0766236E-2,2.455468E-4,-5.779202E-3,9.634033E-3,-1.6671547E-3,-4.086262E-3,-3.6930578E-5,-8.10451E-3,-1.1562613E-3,5.181874E-4,-1.2815009E-2,-2.5577167E-2,-6.9216355E-3,1.8075E-3,-5.1054247E-3,7.030968E-4,-3.917619E-3,1.5823984E-3,-5.3727766E-3,-2.6944336E-3,1.2966034E-3,3.019138E-3,1.063006E-2,5.350591E-3,-2.4817018E-3,1.121858E-2,8.8477996E-4,-1.584826E-3,-1.3589521E-2,-1.7581999E-3,4.242237E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,-1,65,-1,-1,-1,-1,67,69,-1,71,73,-1,75,77,-1,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7526579E0,1.0310113E0,7.2244656E-1,5.59554E-1,2.9407406E-1,2.6896262E-1,5.4699326E-1,4.6292996E-1,5.519655E-1,0E0,4.6790417E-2,2.8642523E-1,1.3183707E-1,2.86371E-1,1.9298603E-1,2.8644195E-1,3.7388575E-1,7.1512794E-1,3.6967295E-1,8.020046E-3,0E0,6.1272383E-2,2.6069707E-1,2.3883412E-2,1.3580114E-2,3.9800477E-1,3.465721E-1,1.9692376E-1,0E0,3.3103824E-2,2.4284813E-1,6.261605E-1,4.581709E-1,1.7878109E-1,1.3617182E-1,0E0,2.5798652E-1,0E0,0E0,0E0,0E0,2.5844195E-1,2.325589E-1,0E0,1.2407496E-2,2.1751642E-2,0E0,4.3592894E-1,2.4926472E-1,0E0,1.0543482E-1,6.199285E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,36,36,41,41,42,42,44,44,45,45,47,47,48,48,50,50,51,51],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,-1,66,-1,-1,-1,-1,68,70,-1,72,74,-1,76,78,-1,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.785302E5,3.4915986E3,2.2741232E6,9.369565E0,2.2040408E7,7.82261E6,2.2E1,6E0,2E0,1.5214396E-2,1.5E1,1.6E1,1E0,4.9036694E0,2E1,1.4473684E-1,3.78E2,7.90913E5,4.8465605E3,8E0,5.8120606E-3,4.273336E-3,3.1540035E9,5.51E2,6.4190677E-3,1.6466942E3,2.4180895E6,9.899632E8,-9.685995E-3,6.737099E7,3.222E3,3.39E2,2.7578741E1,2.9E1,4.727453E7,-1.6877262E-2,1.2193182E1,-2.5595645E-3,8.8179385E-4,9.523992E-3,3.5272184E-4,1.6E0,6.5590655E6,3.4583022E-3,1.3E1,9.830357E1,3.6823764E-3,3.0561172E5,1.2792593E2,7.2117085E-3,2.01E0,9.557399E6,-1.0766236E-2,2.455468E-4,-5.779202E-3,9.634033E-3,-1.6671547E-3,-4.086262E-3,-3.6930578E-5,-8.10451E-3,-1.1562613E-3,5.181874E-4,-1.2815009E-2,-2.5577167E-2,-6.9216355E-3,1.8075E-3,-5.1054247E-3,7.030968E-4,-3.917619E-3,1.5823984E-3,-5.3727766E-3,-2.6944336E-3,1.2966034E-3,3.019138E-3,1.063006E-2,5.350591E-3,-2.4817018E-3,1.121858E-2,8.8477996E-4,-1.584826E-3,-1.3589521E-2,-1.7581999E-3,4.242237E-3],"split_indices":[40,4,63,66,59,40,3,3,29,0,3,10,27,50,6,68,0,9,40,3,0,50,43,2,46,64,40,44,0,7,2,2,68,0,7,0,70,0,0,0,0,65,40,0,3,4,0,45,70,0,65,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.019E3,5.58E2,4.61E2,5.43E2,1.5E1,2.75E2,1.86E2,4.78E2,6.5E1,8E0,7E0,2.55E2,2E1,1.65E2,2.1E1,2.7E1,4.51E2,1.7E1,4.8E1,4E0,3E0,9E0,2.46E2,8E0,1.2E1,1.55E2,1E1,1.6E1,5E0,7E0,2E1,3.71E2,8E1,9E0,8E0,3E0,4.5E1,2E0,2E0,7E0,2E0,1.03E2,1.43E2,3E0,5E0,9E0,3E0,1.17E2,3.8E1,4E0,6E0,1.4E1,2E0,3E0,4E0,1.5E1,5E0,1.17E2,2.54E2,4.1E1,3.9E1,7E0,2E0,6E0,2E0,1.9E1,2.6E1,4.4E1,5.9E1,1.33E2,1E1,3E0,2E0,2E0,7E0,1E2,1.7E1,3.2E1,6E0,3E0,3E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[-1.6611331E-3,-2.8918091E-2,6.407643E-2,-3.589428E-2,2.9623985E-1,2.3271568E-2,9.810506E-2,-1.047401E-1,-2.234734E-2,1.8776065E-2,1.7579027E-1,-3.7111256E-2,6.3984945E-2,1.6602834E-1,7.172275E-2,-8.915662E-2,-1.9595714E-2,-1.9516042E-2,-1.9593993E-2,9.150769E-3,2.8711653E-3,3.4700442E-2,-1.24673314E-1,8.625882E-2,-1.02785446E-1,2.0952633E-1,-2.6411222E-2,2.3414072E-2,1.239266E-1,-5.7170074E-2,-2.2199626E-1,3.4228172E-2,-3.5934277E-2,1.22960955E-1,-1.3183661E-2,-1.4017199E-2,-6.4288884E-2,-1.523162E-1,1.063363E-1,-2.9277414E-2,-1.2150964E-2,1.1420218E-1,2.5242996E-1,1.0749208E-3,-3.2585156E-3,4.0315304E-2,-1.4105E-1,1.442875E-1,-1.6274698E-2,-5.062512E-3,-1.3232338E-3,-4.6155895E-3,-1.2121343E-2,2.1921003E-3,-1.5108885E-2,-3.2417846E-3,1.7054525E-4,1.6762211E-3,6.3441102E-3,-1.0020219E-2,1.520245E-3,-1.4964245E-2,-7.8694563E-4,1.3316898E-3,-1.5767824E-2,3.7039772E-3,1.207846E-2,-3.47651E-3,3.3670017E-3,7.2438284E-3,5.993649E-4,1.2149597E-2,3.6589573E-3,4.341458E-3,-8.8193966E-4,-1.5288881E-2,1.7486945E-3,2.4451956E-3,8.0696475E-3,5.695603E-3,-8.614548E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,-1,-1,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,-1,69,71,-1,-1,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7972425E0,1.6189027E0,4.0594578E-1,6.4585114E-1,1.314776E-1,3.342571E-1,2.748077E-1,5.338037E-1,6.5779436E-1,0E0,1.1440009E-2,3.501651E-1,3.0975056E-1,3.7384605E-1,2.921605E-1,4.547072E-1,0E0,5.12131E-1,0E0,0E0,0E0,1.3263561E-1,2.528692E-1,3.6107576E-1,1.111611E-1,1.184032E-1,2.16249E-2,1.7802449E-1,1.6220725E-1,1.3223189E-1,8.040583E-2,6.8234384E-1,6.101201E-1,9.378701E-3,2.0643592E-1,0E0,2.4104866E-1,2.1770431E-1,2.4844372E-1,4.3636326E-2,0E0,5.515334E-2,2.9039502E-2,0E0,0E0,1.8570165E-1,2.1881917E-1,1.3074422E-1,2.1546634E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,-1,-1,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,-1,70,72,-1,-1,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8062708E6,5.428175E3,1E0,3.3820656E7,1.18033766E5,2.5912744E2,3.9914E4,1E1,3.8304348E1,1.8776065E-2,1.7E1,1.9523809E0,3.921E3,1.0254098E7,6.941074E6,2.8689062E9,-1.9595714E-2,1.3902439E0,-1.9593993E-2,9.150769E-3,2.8711653E-3,3.5714287E-1,2.5247778E6,2.3257812E2,2.0277777E0,6.64E2,2.0319266E-7,1.9269184E7,2E0,1.3E1,8.8324326E-1,1.7587205E6,3.4734247E2,3.57E2,1.994E3,-1.4017199E-2,2.1557376E0,1.0130841E1,6.441311E-1,1.1497237E1,-1.2150964E-2,5.118881E0,1.4595818E7,1.0749208E-3,-3.2585156E-3,4.900919E8,3.558194E6,4.7185095E2,3.29E2,-5.062512E-3,-1.3232338E-3,-4.6155895E-3,-1.2121343E-2,2.1921003E-3,-1.5108885E-2,-3.2417846E-3,1.7054525E-4,1.6762211E-3,6.3441102E-3,-1.0020219E-2,1.520245E-3,-1.4964245E-2,-7.8694563E-4,1.3316898E-3,-1.5767824E-2,3.7039772E-3,1.207846E-2,-3.47651E-3,3.3670017E-3,7.2438284E-3,5.993649E-4,1.2149597E-2,3.6589573E-3,4.341458E-3,-8.8193966E-4,-1.5288881E-2,1.7486945E-3,2.4451956E-3,8.0696475E-3,5.695603E-3,-8.614548E-3],"split_indices":[40,64,25,7,45,4,9,6,65,0,3,66,2,57,57,43,0,65,0,0,0,65,40,45,65,10,49,59,6,3,65,40,64,0,12,0,66,68,54,50,0,68,59,0,0,7,1,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.001E3,7.08E2,2.93E2,6.94E2,1.4E1,1.34E2,1.59E2,1.13E2,5.81E2,6E0,8E0,5.4E1,8E1,4.3E1,1.16E2,1.09E2,4E0,5.78E2,3E0,6E0,2E0,3E1,2.4E1,7.1E1,9E0,3.5E1,8E0,6.1E1,5.5E1,8.9E1,2E1,1.35E2,4.43E2,1E1,2E1,5E0,1.9E1,5E0,6.6E1,7E0,2E0,1.2E1,2.3E1,4E0,4E0,5.6E1,5E0,4.8E1,7E0,3E1,5.9E1,6E0,1.4E1,1.31E2,4E0,2.36E2,2.07E2,2E0,8E0,3E0,1.7E1,2E0,1.7E1,3E0,2E0,5.8E1,8E0,5E0,2E0,8E0,4E0,2.1E1,2E0,2.9E1,2.7E1,2E0,3E0,1.3E1,3.5E1,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-1.9474622E-3,-4.518151E-2,4.2813316E-2,-5.3638093E-2,5.164498E-2,6.0411245E-2,-7.67547E-2,-6.322195E-2,4.190154E-2,1.0754211E-1,-1.8013822E-2,1.4525937E-1,3.9148968E-2,-1.4998174E-1,-2.2213371E-2,7.413579E-2,-6.921388E-2,-8.7248E-2,7.661248E-2,-6.005898E-3,1.3773926E-1,-5.0874177E-2,8.504361E-3,1.6612406E-1,-1.5600476E-2,1.8164493E-1,2.5562007E-2,8.699374E-4,-1.875273E-1,6.864586E-3,-4.6102405E-2,1.04464255E-2,4.2722113E-2,-5.4114383E-2,-1.3895671E-1,-8.837119E-3,5.6381106E-2,-4.6803404E-2,9.350207E-2,3.2867712E-3,-2.878869E-3,2.514748E-3,1.718305E-1,1.1803432E-3,-7.458747E-2,-2.2974696E-2,1.8581335E-1,9.433391E-3,7.092992E-2,-1.0941685E-2,6.53905E-2,-2.1081719E-1,-6.1520234E-2,-1.0136934E-1,1.8378433E-2,4.7057313E-3,-4.2868746E-4,-2.0797832E-3,-8.1818225E-3,-1.5019794E-2,-4.9771112E-3,4.5732926E-5,4.2703864E-3,-3.9251526E-3,3.422313E-4,2.9413337E-3,9.669038E-3,8.529574E-3,2.9254747E-3,-1.2620076E-3,-4.8911E-3,-7.641978E-3,3.511757E-3,8.895652E-3,-3.6694118E-4,-1.05298706E-4,7.7773132E-3,1.2395128E-4,-7.5821457E-3,1.3159094E-2,2.1482075E-3,-6.671899E-3,-1.2287682E-2,8.575296E-4,-5.567062E-3,-6.8432074E-3,7.09623E-5,-2.710716E-3,6.309775E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,49,-1,51,-1,53,-1,55,57,59,-1,61,63,65,-1,-1,-1,67,-1,69,71,73,-1,75,77,79,81,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0125961E0,4.359268E-1,1.0798572E0,4.4856453E-1,1.6936752E-1,7.98926E-1,2.6026574E-1,3.6973763E-1,2.0841485E-1,8.3119094E-2,1.4294215E-1,9.568722E-1,6.884761E-1,1.8113494E-1,1.6682646E-1,8.178145E-2,4.400401E-1,1.61873E-1,7.94162E-2,3.085905E-2,4.4347286E-2,3.4600694E-2,0E0,3.2446575E-1,0E0,6.689286E-2,4.7759035E-1,0E0,5.4528356E-2,0E0,1.2659103E-1,0E0,5.381265E-2,3.7649012E-1,3.9609778E-1,0E0,1.00406725E-2,1.1122233E-2,9.6842706E-2,0E0,0E0,0E0,7.6726377E-3,0E0,1.8113755E-2,1.4104882E-1,1.2034273E-1,0E0,5.0820753E-2,3.6206216E-1,6.329859E-1,3.1773508E-2,2.619281E-2,9.334059E-2,1.6467056E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,26,26,28,28,30,30,32,32,33,33,34,34,36,36,37,37,38,38,42,42,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,50,-1,52,-1,54,-1,56,58,60,-1,62,64,66,-1,-1,-1,68,-1,70,72,74,-1,76,78,80,82,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,9.750085E6,3.0923106E-10,3.3486558E6,1E0,1.9047619E0,1.8661781E5,1.16E0,1.994E3,6.07E2,1E0,3.2462872E5,3.6E1,5.7420593E2,8E1,3.493E3,1.65E2,8.657E2,5.6873284E7,1.834412E6,2.64E0,2.0399521E2,8.504361E-3,3.0805944E7,-1.5600476E-2,1.9719212E1,5.862504E2,8.699374E-4,5.904348E1,6.864586E-3,1.4497429E7,1.04464255E-2,1.3390654E2,4.6463413E0,9.66E2,-8.837119E-3,8.33E2,4.97E2,3.083671E2,3.2867712E-3,-2.878869E-3,2.514748E-3,1.017E3,1.1803432E-3,1.3E1,1.2E2,4.9036694E0,9.433391E-3,3.4E1,4.5620965E6,1.2083E4,7.671875E0,2.01E2,1.75E2,1.0424884E-7,4.7057313E-3,-4.2868746E-4,-2.0797832E-3,-8.1818225E-3,-1.5019794E-2,-4.9771112E-3,4.5732926E-5,4.2703864E-3,-3.9251526E-3,3.422313E-4,2.9413337E-3,9.669038E-3,8.529574E-3,2.9254747E-3,-1.2620076E-3,-4.8911E-3,-7.641978E-3,3.511757E-3,8.895652E-3,-3.6694118E-4,-1.05298706E-4,7.7773132E-3,1.2395128E-4,-7.5821457E-3,1.3159094E-2,2.1482075E-3,-6.671899E-3,-1.2287682E-2,8.575296E-4,-5.567062E-3,-6.8432074E-3,7.09623E-5,-2.710716E-3,6.309775E-3],"split_indices":[2,9,49,40,24,66,45,66,12,2,26,45,0,64,0,41,0,45,5,5,66,4,0,5,0,68,64,0,68,0,59,0,64,65,12,0,0,2,4,0,0,0,2,0,3,0,50,0,0,59,9,58,0,6,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.038E3,5.28E2,5.1E2,4.86E2,4.2E1,4.45E2,6.5E1,4.42E2,4.4E1,2.3E1,1.9E1,8.8E1,3.57E2,2.7E1,3.8E1,1.8E1,4.24E2,9E0,3.5E1,5E0,1.8E1,1.7E1,2E0,8.5E1,3E0,3E1,3.27E2,5E0,2.2E1,4E0,3.4E1,2E0,1.6E1,3.5E2,7.4E1,5E0,4E0,4E0,3.1E1,2E0,3E0,6E0,1.2E1,4E0,1.3E1,8E0,7.7E1,2.4E1,6E0,1.71E2,1.56E2,1.8E1,4E0,1.8E1,1.6E1,7E0,9E0,3.28E2,2.2E1,9E0,6.5E1,2E0,2E0,2E0,2E0,2.6E1,5E0,1E1,2E0,6E0,7E0,3E0,5E0,7.4E1,3E0,4E0,2E0,1.58E2,1.3E1,1.1E1,1.45E2,1E1,8E0,2E0,2E0,1.2E1,6E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[5.763167E-3,-3.6614597E-2,3.9394405E-2,-2.0722374E-2,-1.1472188E-1,-1.139621E-1,5.1252123E-2,-1.06632724E-1,-7.7304635E-3,-2.6165462E-1,-7.2931275E-2,-2.036222E-1,-5.2021146E-2,3.2846592E-2,1.155448E-1,8.525177E-3,-1.4964923E-1,5.1426902E-2,-3.2301534E-2,-9.004322E-2,-4.6943453E-1,-6.78108E-3,-1.3246109E-1,-7.2409194E-3,-1.3877744E-2,4.7534998E-3,-8.657064E-2,2.3645915E-2,2.5275996E-1,1.2740313E-1,-1.684473E-1,6.8735726E-2,-6.1504766E-2,-1.2654239E-1,-1.6257083E-2,-5.8999557E-2,7.6573E-2,-3.4818164E-1,-2.5041616E-2,-5.7482966E-3,1.5427097E-3,-2.6869636E-2,-5.528264E-3,-5.988691E-2,5.4395664E-2,1.8894534E-3,-1.5252997E-1,-7.6406784E-3,-4.356925E-2,3.5061646E-2,-8.83143E-2,1.6492615E-2,1.5306735E-1,-3.8474132E-3,1.4978458E-1,-1.6688895E-3,-1.12261865E-2,-1.9331154E-3,6.597914E-3,-5.1741526E-3,-8.505211E-4,-8.127041E-3,-4.2854995E-3,2.3455596E-4,-1.3074319E-2,2.7367836E-3,8.471667E-3,-3.0668035E-2,4.0138313E-3,2.694953E-3,-2.198298E-3,6.399534E-4,-5.458978E-3,-1.2802879E-3,7.7502117E-3,-7.6277126E-3,7.1809476E-4,-4.264259E-3,-1.8692637E-5,2.6123284E-4,3.5481006E-3,-2.0922646E-2,-1.8890991E-3,-3.028187E-3,9.9238455E-3,-2.4805306E-3,4.4603613E-3,7.542204E-3,-2.429897E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,49,51,53,55,57,59,61,-1,63,65,67,69,-1,-1,-1,-1,71,73,-1,75,-1,77,79,81,-1,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4138322E0,5.425527E-1,1.011448E0,4.0768278E-1,4.3530738E-1,2.1155262E-1,6.044419E-1,2.3905295E-1,4.6534672E-1,5.28249E-1,2.3143843E-1,2.4840415E-2,1.432809E-1,8.0538726E-1,4.0537512E-1,6.3259155E-2,1.2043077E-1,2.6553646E-1,5.095587E-1,4.760506E-2,2.1196187E-1,9.7474E-2,1.14275575E-1,0E0,0E0,0E0,6.4794704E-2,4.96446E-1,1.1461043E-1,3.237096E-1,4.1298613E-2,7.246024E-2,1.3304105E-2,3.8772285E-2,0E0,2.6183182E-1,1.3011858E-1,7.54187E-1,4.2688668E-1,0E0,0E0,0E0,0E0,7.1298435E-2,1.3639498E-1,0E0,6.633639E-2,0E0,3.1861417E-2,4.345123E-1,6.05029E-1,0E0,1.5218951E-1,9.132757E-2,2.8343964E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,43,43,44,44,46,46,48,48,49,49,50,50,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,50,52,54,56,58,60,62,-1,64,66,68,70,-1,-1,-1,-1,72,74,-1,76,-1,78,80,82,-1,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8470706E2,3E0,1E0,3.89E2,3.3817584E7,1E0,2.9652428E6,2.416E3,4.577342E0,3.06E2,1.0142858E1,3.5964913E0,3.8795E4,5.428175E3,5.2964807E0,9.55188E5,1.3E2,1.45064E5,7.5456814E-4,8.5714287E-1,3.087586E2,1.4387174E4,3.9E1,-7.2409194E-3,-1.3877744E-2,4.7534998E-3,4.4522205E6,1.7493458E7,1.18033766E5,1.185E3,2.01E0,1.1E1,1.957E3,1.00011E5,-1.6257083E-2,1.1015E4,2.712766E0,5.1942E4,1.2608696E0,-5.7482966E-3,1.5427097E-3,-2.6869636E-2,-5.528264E-3,3.1488764E0,1.5152774E2,1.8894534E-3,3.1095755E0,-7.6406784E-3,9.234473E-9,5.862504E2,3.167E3,1.6492615E-2,6.176636E5,3.6451373E8,2E0,-1.6688895E-3,-1.12261865E-2,-1.9331154E-3,6.597914E-3,-5.1741526E-3,-8.505211E-4,-8.127041E-3,-4.2854995E-3,2.3455596E-4,-1.3074319E-2,2.7367836E-3,8.471667E-3,-3.0668035E-2,4.0138313E-3,2.694953E-3,-2.198298E-3,6.399534E-4,-5.458978E-3,-1.2802879E-3,7.7502117E-3,-7.6277126E-3,7.1809476E-4,-4.264259E-3,-1.8692637E-5,2.6123284E-4,3.5481006E-3,-2.0922646E-2,-1.8890991E-3,-3.028187E-3,9.9238455E-3,-2.4805306E-3,4.4603613E-3,7.542204E-3,-2.429897E-3],"split_indices":[64,65,17,1,7,25,40,9,70,2,70,65,1,64,50,43,2,7,50,68,67,45,0,0,0,0,59,63,45,41,65,5,9,5,0,9,65,1,65,0,0,0,0,70,64,0,50,0,49,64,2,0,40,7,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E2,4.38E2,5.52E2,3.65E2,7.3E1,3.9E1,5.13E2,4.7E1,3.18E2,1.5E1,5.8E1,1.5E1,2.4E1,4E2,1.13E2,1.3E1,3.4E1,9.3E1,2.25E2,9E0,6E0,2.8E1,3E1,1.2E1,3E0,4E0,2E1,3.85E2,1.5E1,1.09E2,4E0,7E0,6E0,3.2E1,2E0,1.7E1,7.6E1,4E0,2.21E2,7E0,2E0,4E0,2E0,1.5E1,1.3E1,3E0,2.7E1,6E0,1.4E1,3.5E2,3.5E1,6E0,9E0,1.6E1,9.3E1,2E0,2E0,3E0,4E0,2E0,4E0,1.1E1,2.1E1,1.4E1,3E0,6.7E1,9E0,2E0,2E0,4.7E1,1.74E2,7E0,8E0,8E0,5E0,2.5E1,2E0,6E0,8E0,2.07E2,1.43E2,3E0,3.2E1,2E0,7E0,1.1E1,5E0,8.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-3.1352986E-3,-6.492026E-2,2.035752E-2,-5.07281E-2,-1.6788898E-1,8.934491E-3,1.1248679E-1,-3.4351077E-2,-1.6214602E-1,-4.0198103E-1,-1.0098035E-1,-1.6079446E-2,4.240466E-2,1.7397293E-1,6.561411E-2,-8.840557E-2,1.1218942E-2,-3.5631556E-2,-1.7956921E-1,-9.153555E-3,-2.3157766E-2,-1.9431548E-1,-1.33578805E-2,-4.1638006E-2,4.4162564E-2,4.8454866E-2,-1.9788264E-1,8.8862576E-2,2.3648836E-1,1.0060557E-1,-9.330599E-2,-3.3983304E-3,-1.10158734E-1,1.8525709E-1,-1.2317144E-2,-3.3986215E-3,6.7093404E-4,-2.3711305E-3,-1.934115E-1,-2.4825099E-1,-5.0055387E-4,-5.553299E-2,5.351343E-3,-3.0706903E-1,-3.4188375E-2,5.847223E-2,-1.245106E-1,4.052643E-2,2.747575E-1,-1.6891999E-3,-1.1711158E-2,5.6271167E-3,1.2363493E-2,1.1954117E-2,3.7793468E-3,-1.787993E-2,1.226902E-1,-1.297629E-1,1.0500924E-3,3.9488734E-3,-2.0613344E-3,-7.7351965E-3,-3.4916836E-3,9.195758E-3,2.9263103E-3,-3.277666E-3,1.7279024E-3,-9.438366E-3,-1.8548354E-3,-6.103061E-3,-1.3918369E-2,2.2442536E-3,-4.398401E-3,1.8636108E-3,-2.6545886E-2,-1.9670704E-3,3.4612026E-3,-3.771785E-4,4.0714936E-3,-2.2728546E-3,-9.458388E-3,1.0265259E-2,1.5232543E-3,1.8037394E-2,4.6786736E-3,-1.17165E-3,2.6979488E-3,-2.8889016E-3,1.4517092E-3,1.857029E-3,6.7701326E-3,-2.3807639E-3,-8.0496045E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,-1,-1,67,69,-1,71,-1,73,75,77,79,81,83,-1,-1,-1,85,-1,-1,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4542761E0,3.9481568E-1,7.6303387E-1,4.3965542E-1,4.7627962E-1,5.423636E-1,2.1914661E-1,5.281304E-1,6.193453E-2,4.452932E-2,2.182172E-1,5.7306707E-1,4.1228765E-1,1.5988922E-1,2.7074683E-1,1.808449E-1,4.8153454E-1,1.0717053E-2,3.7834346E-2,0E0,0E0,1.2590003E-1,8.8625915E-2,5.055935E-1,2.7519634E-1,4.7510028E-1,5.500281E-2,3.9670423E-2,4.8232436E-2,1.04245394E-1,4.1082814E-2,8.170805E-2,1.4230591E-1,1.2682229E-2,3.0931467E-1,0E0,0E0,0E0,3.6053896E-2,2.293253E-2,0E0,5.5072207E-2,0E0,6.8262446E-1,2.4451202E-1,2.079519E-1,4.478377E-2,3.4688535E-1,1.4352685E-1,0E0,0E0,0E0,1.2036244E-2,0E0,0E0,1.7581424E-2,6.110677E-2,1.5731059E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,48,48,52,52,55,55,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,-1,-1,68,70,-1,72,-1,74,76,78,80,82,84,-1,-1,-1,86,-1,-1,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,4.4895835E0,5.1167645E6,3.38E2,3.964269E3,4.8865112E2,1.7047619E0,1.545E2,4.273927E8,8.612209E-6,1.317484E1,4.39776E6,6.77311E7,1.42E1,2.1512408E5,3.959E3,1.1E1,1.3E1,5.753789E5,-9.153555E-3,-2.3157766E-2,1.33E2,6.593462E4,4E0,2.8138569E1,4.435876E2,2.2734E4,8.785115E6,3.3765998E0,4.990618E6,6.663214E6,1E1,3.89E2,2.63E2,8.579633E-2,-3.3986215E-3,6.7093404E-4,-2.3711305E-3,9.18E2,4.7721977E4,-5.0055387E-4,5.8E1,5.351343E-3,2.41E3,1.3567028E5,1.6494E4,3E0,3.79E2,1.9407515E9,-1.6891999E-3,-1.1711158E-2,5.6271167E-3,9.39E2,1.1954117E-2,3.7793468E-3,1.3586957E1,1E0,2.577232E0,1.0500924E-3,3.9488734E-3,-2.0613344E-3,-7.7351965E-3,-3.4916836E-3,9.195758E-3,2.9263103E-3,-3.277666E-3,1.7279024E-3,-9.438366E-3,-1.8548354E-3,-6.103061E-3,-1.3918369E-2,2.2442536E-3,-4.398401E-3,1.8636108E-3,-2.6545886E-2,-1.9670704E-3,3.4612026E-3,-3.771785E-4,4.0714936E-3,-2.2728546E-3,-9.458388E-3,1.0265259E-2,1.5232543E-3,1.8037394E-2,4.6786736E-3,-1.17165E-3,2.6979488E-3,-2.8889016E-3,1.4517092E-3,1.857029E-3,6.7701326E-3,-2.3807639E-3,-8.0496045E-3],"split_indices":[40,65,40,0,40,67,66,67,43,50,66,9,56,70,45,9,10,3,57,0,0,0,40,3,70,68,9,40,50,57,40,3,1,41,50,0,0,0,0,40,0,64,0,2,45,1,65,41,12,0,0,0,2,0,0,70,27,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,2.75E2,7.25E2,2.43E2,3.2E1,6.46E2,7.9E1,2.13E2,3E1,6E0,2.6E1,3.7E2,2.76E2,3.3E1,4.6E1,9.7E1,1.16E2,4E0,2.6E1,3E0,3E0,1.2E1,1.4E1,2.6E2,1.1E2,2.7E2,6E0,1.5E1,1.8E1,3.8E1,8E0,2E1,7.7E1,1.3E1,1.03E2,2E0,2E0,3E0,2.3E1,9E0,3E0,1.1E1,3E0,6E0,2.54E2,1.02E2,8E0,2.62E2,8E0,2E0,4E0,1E1,5E0,1.5E1,3E0,6E0,3.2E1,6E0,2E0,6E0,1.4E1,2.7E1,5E1,1.1E1,2E0,4.7E1,5.6E1,2.1E1,2E0,4E0,5E0,3E0,8E0,3E0,3E0,2.36E2,1.8E1,3.2E1,7E1,5E0,3E0,9E0,2.53E2,4E0,4E0,3E0,2E0,3E0,3E0,8E0,2.4E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[4.956395E-4,-1.0411534E-2,1.274334E-1,-3.5842977E-2,2.4904223E-2,1.5240328E-1,-1.2570965E-1,-2.6890375E-2,-1.311502E-1,2.3108402E-3,8.398949E-2,1.07252546E-1,2.734627E-1,-2.4393386E-1,2.4572467E-3,-1.55407945E-2,-8.933185E-2,-2.5727165E-1,-5.368774E-2,2.6600916E-2,-6.7713335E-2,1.3591267E-1,1.1927087E-2,1.4773503E-1,-2.7639022E-2,1.6317012E-2,1.7425379E-1,-4.6937154E-3,-1.3980051E-2,-5.271263E-2,1.5190722E-2,-1.8759912E-1,-4.036368E-2,-1.0462029E-1,-2.2148496E-2,1.6771583E-2,-1.2996246E-1,-3.416876E-2,6.4666264E-2,-2.195094E-1,1.5610805E-2,1.5434511E-1,-3.8862605E-2,-7.755883E-2,5.105537E-2,2.4900077E-1,9.217766E-2,4.599089E-2,-1.3347739E-1,9.8023E-3,8.458288E-2,-8.122887E-3,-1.976604E-3,7.913963E-3,2.8322148E-4,1.7214896E-3,-1.0040027E-2,-5.161565E-3,-3.2546278E-4,1.0212624E-3,-9.113149E-3,-7.290172E-3,3.8112863E-3,-8.967101E-4,-6.680809E-3,-8.865139E-4,-1.04133235E-2,2.5940628E-4,6.0641784E-3,-1.692589E-2,-3.4792274E-3,-7.020221E-3,2.1398417E-3,4.7729616E-3,9.320947E-3,2.1423786E-3,-3.7844547E-3,-8.179087E-4,-7.542812E-3,3.937152E-3,-3.3214167E-3,2.2655511E-3,1.267371E-2,8.583685E-3,1.6297671E-3,-2.702165E-4,4.072596E-3,-9.121581E-5,-9.127823E-3,8.210007E-4,5.654037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,-1,-1,51,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4523165E0,8.684868E-1,5.437976E-1,4.759984E-1,5.4045504E-1,3.8608038E-1,1.8254343E-1,3.6358422E-1,4.5513207E-1,5.0186366E-1,4.1745043E-1,3.1534576E-1,1.0037017E-1,1.1727333E-2,0E0,5.000064E-1,3.7257326E-1,5.6761074E-1,1.6878192E-1,5.088045E-1,9.6469575E-1,2.1539795E-1,1.7202845E-1,2.245943E-1,1.15238935E-1,0E0,2.057302E-2,0E0,0E0,2.3028558E-1,3.392176E-1,1.8953335E-1,1.2860867E-1,1.471792E-1,0E0,2.1068396E-1,2.281934E-2,2.3895904E-1,5.3594714E-1,5.5041146E-1,2.6802546E-1,1.2347555E-1,2.9897884E-2,7.474673E-2,1.5159015E-1,6.703502E-2,1.5444383E-1,2.0445446E-2,5.1083677E-2,0E0,1.0645799E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,-1,-1,52,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6466942E3,8.484036E5,4.1325716E7,4.9735293E0,2.593592E6,4.907764E3,6.9879E4,3.9E2,2E0,6.93729E5,2.8367348E0,1.0907809E9,1.29443414E5,1.3E1,2.4572467E-3,1.5292561E2,1.3092E4,7.90913E5,2.3364584E1,1E0,1.084437E6,1.49660635E1,6.485856E7,1.6386554E0,2.397273E8,1.6317012E-2,2.0666666E0,-4.6937154E-3,-1.3980051E-2,1.7857143E1,1.2E1,9.052632E-1,4.5454545E0,2.1095E4,-2.2148496E-2,1.2788898E2,4.3E1,4.9E1,4.3E2,2.6463525E8,6.99E2,1.6121496E1,1.8791208E0,2.9251662E6,5.49E2,3.0984934E8,3.6444E4,1.9428571E0,1.7638037E0,9.8023E-3,6.5026445E3,-8.122887E-3,-1.976604E-3,7.913963E-3,2.8322148E-4,1.7214896E-3,-1.0040027E-2,-5.161565E-3,-3.2546278E-4,1.0212624E-3,-9.113149E-3,-7.290172E-3,3.8112863E-3,-8.967101E-4,-6.680809E-3,-8.865139E-4,-1.04133235E-2,2.5940628E-4,6.0641784E-3,-1.692589E-2,-3.4792274E-3,-7.020221E-3,2.1398417E-3,4.7729616E-3,9.320947E-3,2.1423786E-3,-3.7844547E-3,-8.179087E-4,-7.542812E-3,3.937152E-3,-3.3214167E-3,2.2655511E-3,1.267371E-2,8.583685E-3,1.6297671E-3,-2.702165E-4,4.072596E-3,-9.121581E-5,-9.127823E-3,8.210007E-4,5.654037E-3],"split_indices":[64,40,63,65,9,4,2,0,29,9,66,7,45,3,0,64,10,9,70,27,9,68,7,66,44,0,65,0,0,4,30,70,66,1,0,4,8,3,10,7,2,70,66,40,0,7,9,65,65,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,9.65E2,8.2E1,5.61E2,4.04E2,7.5E1,7E0,5.14E2,4.7E1,2.93E2,1.11E2,5.6E1,1.9E1,4E0,3E0,4.36E2,7.8E1,1.7E1,3E1,2.18E2,7.5E1,6.4E1,4.7E1,4.3E1,1.3E1,9E0,1E1,2E0,2E0,1.97E2,2.39E2,2.5E1,5.3E1,1.1E1,6E0,1.6E1,1.4E1,8.4E1,1.34E2,2.6E1,4.9E1,5.8E1,6E0,1.4E1,3.3E1,1.4E1,2.9E1,8E0,5E0,6E0,4E0,1.3E1,1.84E2,1.2E1,2.27E2,3E0,2.2E1,1.6E1,3.7E1,5E0,6E0,4E0,1.2E1,2E0,1.2E1,7.9E1,5E0,7.2E1,6.2E1,1.2E1,1.4E1,7E0,4.2E1,3E1,2.8E1,2E0,4E0,9E0,5E0,2.6E1,7E0,2E0,1.2E1,1E1,1.9E1,4E0,4E0,2E0,3E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-6.406205E-3,-1.6956834E-2,1.0196297E-1,-8.370115E-3,-1.3819332E-1,1.532258E-1,-2.4200808E-2,-5.1239382E-2,1.1540088E-2,6.316963E-2,-1.7570204E-1,1.3359398E-1,1.8578792E-2,-6.0243085E-2,8.68531E-3,2.5991185E-3,-9.128116E-2,3.1764659E-3,1.0923094E-1,-1.9605296E-2,8.829574E-3,-2.8072625E-1,-8.996981E-2,1.4902687E-1,-6.288674E-2,-9.340476E-2,5.540542E-3,-7.0075266E-2,5.101104E-2,-1.0409671E-1,3.2707285E-2,8.346085E-3,-1.354821E-1,-3.2835472E-2,1.7444918E-1,-4.6673254E-3,3.0914475E-3,-4.7464174E-1,-1.957722E-1,5.5269208E-2,-1.3026126E-1,1.7821285E-1,1.6081782E-2,-7.319791E-3,2.5056486E-3,-2.3582837E-2,-1.7612235E-1,-1.6927174E-3,-6.5856385E-3,3.3071279E-3,-6.6687064E-3,-1.4847563E-2,-4.291257E-3,5.0264047E-3,-6.551202E-3,1.5498311E-3,-5.686366E-4,-7.954022E-3,-2.118982E-3,1.6723251E-3,-3.8143373E-3,-1.1904843E-3,9.041296E-3,-1.1687003E-2,-2.801939E-2,-1.1153652E-2,-1.5737118E-3,-2.5233852E-3,5.0680717E-3,-8.023035E-3,-2.5288835E-3,9.467811E-3,4.301067E-3,3.664069E-3,-8.163755E-4,-8.161124E-3,1.1485016E-3,-4.675871E-3,-1.3358351E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,-1,63,65,67,69,71,73,-1,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1210814E0,9.284772E-1,5.663818E-1,7.1329373E-1,4.567182E-1,2.5179303E-1,2.112685E-1,5.7152134E-1,4.6618822E-1,1.1060117E-1,4.2494106E-1,1.8827558E-1,0E0,1.492193E-1,0E0,4.046616E-1,2.4492967E-1,3.7889072E-1,4.2266762E-1,5.673184E-2,0E0,2.695042E-1,1.7690428E-1,2.1054125E-1,6.534371E-2,1.1135656E-1,0E0,1.0674104E-1,2.9461318E-1,2.9248405E-1,2.1739984E-1,2.6833704E-1,5.599296E-2,5.51735E-2,1.4419788E-1,0E0,0E0,2.3649573E-2,1.20891035E-1,4.8545953E-2,6.720239E-2,8.693862E-2,2.513692E-2,0E0,0E0,9.42892E-2,4.7121435E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,-1,64,66,68,70,72,74,-1,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5142261E3,9.639872E0,2.5842668E7,1.8154167E2,8.1632656E-1,5.857143E0,2.1362119E0,4E0,4.9321495E6,7.329299E-2,8E0,3.7798166E0,1.8578792E-2,9.226311E2,8.68531E-3,4.9E2,2.3E1,4.217427E7,1.7068776E10,1.00011E5,8.829574E-3,6.34E2,4.911E3,1.8776652E7,7.25E2,6.915691E7,5.540542E-3,6.0921145E4,2.2E1,7.5456814E-4,4.923077E0,2.192E3,7.295292E-2,6.229955E4,7.9146667E0,-4.6673254E-3,3.0914475E-3,4.97E2,6.0078123E-6,1.43E2,3.7E1,2.4159293E0,7.739144E8,-7.319791E-3,2.5056486E-3,7.958E3,1E1,-1.6927174E-3,-6.5856385E-3,3.3071279E-3,-6.6687064E-3,-1.4847563E-2,-4.291257E-3,5.0264047E-3,-6.551202E-3,1.5498311E-3,-5.686366E-4,-7.954022E-3,-2.118982E-3,1.6723251E-3,-3.8143373E-3,-1.1904843E-3,9.041296E-3,-1.1687003E-2,-2.801939E-2,-1.1153652E-2,-1.5737118E-3,-2.5233852E-3,5.0680717E-3,-8.023035E-3,-2.5288835E-3,9.467811E-3,4.301067E-3,3.664069E-3,-8.163755E-4,-8.161124E-3,1.1485016E-3,-4.675871E-3,-1.3358351E-2],"split_indices":[64,66,59,64,68,65,46,70,40,54,29,65,0,68,0,1,3,57,43,5,0,2,41,57,0,56,0,57,3,50,66,41,50,45,70,0,0,2,49,0,3,65,7,0,0,2,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.78E2,8.92E2,8.6E1,8.34E2,5.8E1,6.1E1,2.5E1,2.64E2,5.7E2,9E0,4.9E1,5.8E1,3E0,2.2E1,3E0,1.13E2,1.51E2,5.26E2,4.4E1,6E0,3E0,2.1E1,2.8E1,5.4E1,4E0,1.9E1,3E0,4.5E1,6.8E1,1.37E2,1.4E1,5.08E2,1.8E1,1.4E1,3E1,3E0,3E0,5E0,1.6E1,6E0,2.2E1,4.4E1,1E1,2E0,2E0,1.1E1,8E0,3.2E1,1.3E1,6.2E1,6E0,5E0,1.32E2,1E1,4E0,2.28E2,2.8E2,1.2E1,6E0,6E0,8E0,3E0,2.7E1,3E0,2E0,1.2E1,4E0,2E0,4E0,1.3E1,9E0,3.2E1,1.2E1,3E0,7E0,2E0,9E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[1.4178382E-3,-4.6084564E-2,2.9812988E-2,-4.323936E-2,-1.7964555E-2,1.3029246E-2,9.5427565E-2,-3.286E-2,-1.17888205E-1,2.1056077E-2,-1.0363302E-1,1.1532343E-1,-1.3029344E-1,-3.7492733E-2,1.2145314E-1,-1.3966003E-1,1.1057006E-2,2.6500849E-2,-1.2815219E-1,-1.4865015E-1,1.4771411E-2,2.2501232E-1,8.699501E-2,-1.8419188E-1,3.4371954E-3,-3.1762872E-2,-3.9899725E-1,1.7238879E-3,7.860978E-3,-1.822536E-1,-4.516222E-2,8.291745E-2,1.0687098E-2,-8.397089E-3,2.5370547E-3,-1.6468684E-1,8.498409E-4,5.3906437E-2,-2.6368261E-3,1.5184788E-1,1.4342481E-2,5.5874903E-2,1.6168833E-1,-5.1495545E-3,-1.3365028E-2,-2.580729E-3,-2.7817223E-4,-1.2999434E-3,-2.9244326E-2,-9.359238E-3,3.6409849E-3,-3.9191064E-3,1.3459171E-3,4.7591752E-3,-6.4380085E-3,1.0302186E-2,2.4004818E-4,-2.2021108E-3,3.1692097E-3,-8.68485E-3,-3.1287698E-3,3.9310916E-3,-7.7447196E-4,8.069927E-3,-9.6640164E-5,1.5428247E-3,7.916361E-3,8.238331E-3,4.9659616E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,-1,45,47,-1,-1,49,51,53,55,-1,57,59,-1,61,-1,63,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3771886E0,3.58092E-1,7.0246065E-1,2.9055303E-1,0E0,4.7976056E-1,5.9884477E-1,2.440423E-1,3.9277554E-1,3.9172575E-1,1.7809328E-1,3.533026E-1,1.3539957E-1,6.625936E-1,3.52792E-2,1.6886902E-1,0E0,4.1129088E-1,1.21914595E-1,6.7380846E-2,3.1341974E-2,1.0578251E-1,2.1643859E-1,3.6649466E-2,0E0,2.0182654E-1,4.2254555E-1,0E0,0E0,1.8492383E-1,4.7360938E-2,4.8085606E-1,4.2109495E-1,0E0,2.3438022E-2,3.8948834E-2,0E0,1.7136304E-2,0E0,5.543846E-2,0E0,1.7591016E-1,7.272011E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,37,37,39,39,41,41,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,-1,46,48,-1,-1,50,52,54,56,-1,58,60,-1,62,-1,64,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.01E2,5.5E1,7.098E3,2.1E1,-1.7964555E-2,5.5952E4,3.206931E2,4.07E2,4E1,7.042942E7,7.153514E1,8.959817E7,2.433735E0,2.8302418E10,1.1626786E2,5.0710382E0,1.1057006E-2,8.960784E0,1.7540792E5,2.0467092E6,7.362415E1,2.5192308E0,4.7429975E6,1.3E1,3.4371954E-3,4.21E2,3.8795E4,1.7238879E-3,7.860978E-3,1.407E3,7.90913E5,2.4E1,1.0918E4,-8.397089E-3,1.4595416E3,3.8767453E4,8.498409E-4,2.707256E7,-2.6368261E-3,2.0666666E0,1.4342481E-2,4.435876E2,1.7655972E0,-5.1495545E-3,-1.3365028E-2,-2.580729E-3,-2.7817223E-4,-1.2999434E-3,-2.9244326E-2,-9.359238E-3,3.6409849E-3,-3.9191064E-3,1.3459171E-3,4.7591752E-3,-6.4380085E-3,1.0302186E-2,2.4004818E-4,-2.2021108E-3,3.1692097E-3,-8.68485E-3,-3.1287698E-3,3.9310916E-3,-7.7447196E-4,8.069927E-3,-9.6640164E-5,1.5428247E-3,7.916361E-3,8.238331E-3,4.9659616E-4],"split_indices":[2,3,2,8,0,41,70,0,3,57,68,12,65,43,4,66,0,70,45,40,70,65,40,3,0,2,1,0,0,0,9,3,9,0,4,45,0,57,0,65,0,68,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.019E3,3.81E2,6.38E2,3.79E2,2E0,5.09E2,1.29E2,3.34E2,4.5E1,4.77E2,3.2E1,1.19E2,1E1,3.25E2,9E0,4.3E1,2E0,4.61E2,1.6E1,2.3E1,9E0,2.3E1,9.6E1,8E0,2E0,3.21E2,4E0,4E0,5E0,2.9E1,1.4E1,1E2,3.61E2,1.1E1,5E0,2.1E1,2E0,6E0,3E0,1.4E1,9E0,6.9E1,2.7E1,6E0,2E0,1.64E2,1.57E2,2E0,2E0,2.7E1,2E0,9E0,5E0,9.2E1,8E0,8E0,3.53E2,3E0,2E0,1.6E1,5E0,4E0,2E0,1.2E1,2E0,5.9E1,1E1,2.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-2.8712496E-3,-1.707376E-2,6.386421E-2,-2.2034116E-2,2.466327E-1,7.28158E-2,-1.9593239E-1,-4.169378E-2,1.9206872E-2,4.681527E-3,1.2599767E-2,-7.543376E-2,8.5820206E-2,-1.1538579E-3,-1.2633824E-2,-1.2926568E-2,-9.346737E-2,3.5144545E-2,-7.960898E-2,4.2754196E-2,-1.9972946E-1,-3.935072E-2,1.0163959E-1,-4.3011744E-2,2.7341058E-2,-2.6777254E-2,-8.5246265E-2,4.1512456E-2,-2.0101891E-1,-1.2183121E-1,1.4988095E-2,-2.1309168E-3,8.020225E-2,-1.24104405E-2,-3.643855E-3,2.1601461E-3,-6.73901E-2,1.11353576E-1,-6.8917535E-2,-1.0263299E-3,-5.6087105E-3,-7.750013E-4,4.4554635E-3,-3.4567474E-3,-1.7407559E-2,4.442189E-3,9.75711E-4,-1.2622156E-2,-1.6366774E-3,2.5071816E-3,-6.814824E-3,2.6978757E-3,-4.004941E-3,7.925182E-4,4.9313316E-3,-1.4201073E-3,-5.8267135E-3,3.7173894E-3,7.7611487E-3,7.366632E-4,-5.6541134E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,-1,53,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.2899746E-1,1.0623615E0,4.1426873E-1,6.4456326E-1,3.2131255E-2,3.2843262E-1,7.418303E-2,8.0007917E-1,4.0754578E-1,0E0,0E0,2.1420284E-1,3.0893922E-1,0E0,0E0,4.2155212E-1,7.2297823E-1,3.4220335E-1,1.4561361E-1,3.0428056E-2,3.79512E-2,4.676629E-2,2.3465419E-1,3.2258183E-1,4.6055692E-1,0E0,5.2889645E-1,2.4074793E-1,6.342566E-2,1.2514186E-1,5.876542E-2,0E0,8.405663E-3,0E0,0E0,0E0,2.6442021E-2,2.1418524E-1,3.8798317E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,32,32,36,36,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,-1,54,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9652428E6,5.428175E3,5.2964807E0,2.026453E6,1.5E1,1.994E3,3.01E2,3.72381E5,1.8E1,4.681527E-3,1.2599767E-2,5.69E2,8.91E2,-1.1538579E-3,-1.2633824E-2,1E0,4E0,1E0,3.9742767E2,3.57E2,1.4110284E2,3.3875784E7,2E0,1.5E1,5.7621355E4,-2.6777254E-2,3.3386944E5,9E0,1.1E1,1.1967312E3,1.465E4,-2.1309168E-3,4.3151297E-2,-1.24104405E-2,-3.643855E-3,2.1601461E-3,3.7951445E6,1.3400493E0,3.8807898E6,-1.0263299E-3,-5.6087105E-3,-7.750013E-4,4.4554635E-3,-3.4567474E-3,-1.7407559E-2,4.442189E-3,9.75711E-4,-1.2622156E-2,-1.6366774E-3,2.5071816E-3,-6.814824E-3,2.6978757E-3,-4.004941E-3,7.925182E-4,4.9313316E-3,-1.4201073E-3,-5.8267135E-3,3.7173894E-3,7.7611487E-3,7.366632E-4,-5.6541134E-3],"split_indices":[40,64,50,9,3,12,0,9,3,0,0,2,41,0,0,27,3,76,64,0,4,7,6,8,45,0,45,3,3,45,41,0,65,0,0,0,40,54,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.78E2,8.07E2,1.71E2,7.93E2,1.4E1,1.66E2,5E0,5.37E2,2.56E2,3E0,1.1E1,1.3E1,1.53E2,2E0,3E0,3.46E2,1.91E2,2.21E2,3.5E1,7E0,6E0,1.7E1,1.36E2,1.98E2,1.48E2,2E0,1.89E2,2.16E2,5E0,2.4E1,1.1E1,2E0,5E0,3E0,3E0,4E0,1.3E1,1.29E2,7E0,1.58E2,4E1,9.1E1,5.7E1,1.84E2,5E0,5.7E1,1.59E2,3E0,2E0,3E0,2.1E1,8E0,3E0,2E0,3E0,9E0,4E0,8.6E1,4.3E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-2.2949683E-3,-5.6825187E-2,1.7537316E-2,-1.3240497E-1,-2.9436354E-2,-1.2739021E-2,4.4361748E-2,-7.089745E-2,-2.4990311E-1,-8.019383E-2,5.655105E-3,-3.37886E-3,-1.4444831E-1,3.956117E-2,2.6924407E-1,3.361081E-2,-9.263937E-2,-1.5854454E-1,-3.6195064E-1,-5.8427423E-2,-2.0048174E-1,1.9828233E-1,-2.6504874E-2,7.1007437E-3,-8.041135E-2,-1.7858827E-1,2.864455E-3,-2.0328939E-2,5.904393E-2,1.5750172E-2,5.0458782E-3,-1.2358462E-3,4.0146797E-3,-1.18872434E-1,1.0926992E-2,-2.1881858E-1,-3.6034419E-3,-2.2281341E-2,-1.9900204E-1,-9.8187506E-2,3.90217E-2,-1.1639469E-2,-7.570172E-2,2.8946947E-3,2.1211559E-1,6.2370454E-3,-9.515425E-2,-1.3204201E-2,6.467211E-2,7.2382125E-3,-9.753524E-2,-7.6367795E-2,-2.4037127E-1,5.3265803E-2,-6.6137016E-2,8.2173094E-2,-1.6495188E-3,3.1056541E-3,-6.1345603E-3,-2.1434668E-3,2.1818182E-3,-1.1403565E-2,-3.994672E-3,-1.5615971E-3,-1.253951E-2,-1.0140443E-2,-3.1959815E-3,8.347556E-3,-2.4111252E-4,-5.276382E-3,-5.1877997E-4,1.1111302E-2,5.8381567E-3,1.2228714E-2,-2.4459913E-4,-9.3496084E-5,-5.518509E-3,8.092999E-4,-3.6373534E-3,4.9623866E-3,-1.6794582E-5,-1.1203764E-2,-3.3254279E-3,-4.2259623E-3,-6.6249457E-4,-1.4507451E-2,-5.1558884E-3,7.148782E-3,4.4025533E-4,-6.1354944E-3,-1.2918813E-3,1.7967902E-3,6.549528E-3,5.0571025E-3,-3.0355775E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,-1,-1,-1,-1,57,59,61,-1,-1,63,65,67,-1,69,-1,71,73,75,77,79,-1,81,83,85,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0867811E0,5.508738E-1,5.991163E-1,4.954008E-1,3.5312206E-1,4.2730057E-1,4.1145796E-1,1.12179816E-1,1.8829083E-1,2.0036352E-1,7.362521E-1,2.6293373E-1,1.7353001E-1,4.4926095E-1,5.625516E-2,3.165135E-2,1.099734E-1,5.524528E-2,1.0413277E-1,2.7505243E-1,6.0067683E-2,1.8440604E-2,2.2986013E-1,3.3635217E-1,1.7431301E-1,1.0795313E-1,0E0,3.2379004E-1,4.072343E-1,0E0,0E0,0E0,0E0,9.6672535E-2,2.1174839E-2,9.475857E-3,0E0,0E0,6.39822E-2,1.599729E-1,1.3363732E-1,0E0,1.13176815E-2,0E0,7.2442293E-3,2.1185304E-1,7.655418E-2,4.3462932E-1,2.1157461E-1,0E0,1.18483186E-1,7.3910356E-3,8.058828E-2,1.6396402E-1,1.4749098E-1,5.407039E-1,5.905699E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,33,33,34,34,35,35,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,-1,-1,-1,-1,58,60,62,-1,-1,64,66,68,-1,70,-1,72,74,76,78,80,-1,82,84,86,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,3.3832976E7,3.4734247E2,1E0,1.3390654E2,1E0,4.7263342E2,1E0,3.3502305E0,3.477E4,3E1,1.138871E0,1.9505818E8,1E0,1.94E3,1.1E1,1.16E2,3.8575E4,5.75E2,2.71E2,4.7839867E1,4.58E2,1.6942337E1,3.422351E6,2.0317461E0,1.3123183E-2,2.864455E-3,1.3619632E0,2.1298597E0,1.5750172E-2,5.0458782E-3,-1.2358462E-3,4.0146797E-3,4E0,1.4473684E-1,1.6E1,-3.6034419E-3,-2.2281341E-2,2.2227E4,8.0833334E-1,4.5650104E2,-1.1639469E-2,4.76E2,2.8946947E-3,1.8543E4,1.2083E4,1.5310831E7,3.53969E5,1E1,7.2382125E-3,2E2,6.7723384E7,2.49E2,1.4283589E6,2.652E3,9.706078E4,3.9914E4,3.1056541E-3,-6.1345603E-3,-2.1434668E-3,2.1818182E-3,-1.1403565E-2,-3.994672E-3,-1.5615971E-3,-1.253951E-2,-1.0140443E-2,-3.1959815E-3,8.347556E-3,-2.4111252E-4,-5.276382E-3,-5.1877997E-4,1.1111302E-2,5.8381567E-3,1.2228714E-2,-2.4459913E-4,-9.3496084E-5,-5.518509E-3,8.092999E-4,-3.6373534E-3,4.9623866E-3,-1.6794582E-5,-1.1203764E-2,-3.3254279E-3,-4.2259623E-3,-6.6249457E-4,-1.4507451E-2,-5.1558884E-3,7.148782E-3,4.4025533E-4,-6.1354944E-3,-1.2918813E-3,1.7967902E-3,6.549528E-3,5.0571025E-3,-3.0355775E-3],"split_indices":[40,7,64,8,64,82,68,19,65,1,0,50,5,27,0,3,41,9,2,2,70,2,68,9,66,50,0,65,50,0,0,0,0,0,68,3,0,0,1,65,45,0,2,0,1,9,12,9,8,0,41,7,10,40,2,45,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.003E3,2.67E2,7.36E2,7E1,1.97E2,3.46E2,3.9E2,4.7E1,2.3E1,8E1,1.17E2,3.24E2,2.2E1,3.83E2,7E0,8E0,3.9E1,1.4E1,9E0,6.9E1,1.1E1,1.6E1,1.01E2,2.86E2,3.8E1,1.9E1,3E0,9.4E1,2.89E2,4E0,3E0,4E0,4E0,3.1E1,8E0,7E0,7E0,4E0,5E0,4.9E1,2E1,7E0,4E0,2E0,1.4E1,6.9E1,3.2E1,2.12E2,7.4E1,2E0,3.6E1,8E0,1.1E1,3.6E1,5.8E1,2.09E2,8E1,2E0,2.9E1,3E0,5E0,5E0,2E0,2E0,3E0,8E0,4.1E1,4E0,1.6E1,2E0,2E0,9E0,5E0,2E0,6.7E1,7E0,2.5E1,1.45E2,6.7E1,4.4E1,3E1,4E0,3.2E1,6E0,2E0,6E0,5E0,1E1,2.6E1,2E1,3.8E1,1.23E2,8.6E1,2.9E1,5.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[1.9311918E-3,-7.966601E-3,1.0612641E-1,-2.3582013E-2,3.2192394E-2,1.3004476E-1,-1.4034332E-1,-2.110563E-2,-2.5101267E-2,1.5406759E-2,1.1081102E-1,6.907168E-2,1.9574273E-1,-2.1316516E-1,2.3921824E-3,2.9459799E-2,-4.097966E-2,-6.486098E-2,3.6718115E-2,2.3217263E-2,1.9734944E-1,9.8042805E-3,4.37915E-2,2.474464E-1,8.125544E-2,-1.1849504E-2,-3.7826744E-3,-9.986864E-3,7.663096E-2,-8.7707624E-4,-5.7881743E-2,2.6656574E-2,-1.4223629E-1,5.717841E-2,-3.8186237E-2,-4.617763E-3,4.6807006E-2,2.286497E-1,2.076043E-3,-5.937531E-2,8.420052E-2,2.7695474E-1,6.0630986E-3,4.8536216E-3,-1.6237972E-3,7.1889797E-4,-4.9230023E-3,1.6120102E-3,5.848031E-3,7.7294186E-4,-7.919235E-3,-3.6802352E-3,-3.889379E-4,3.4193338E-3,-1.5320074E-3,-1.494656E-2,-2.189571E-3,2.1012356E-3,9.451928E-3,-4.851383E-3,2.4210464E-3,2.7975189E-3,-2.4523248E-3,3.8099505E-3,1.1100649E-2,-5.7029994E-3,-2.1202887E-4,-3.460324E-3,4.5920955E-3,1.3560183E-2,6.1838217E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,41,43,-1,-1,45,47,49,51,53,55,57,59,-1,61,63,-1,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0104909E0,5.6201303E-1,5.193386E-1,8.236946E-1,3.2952833E-1,2.989328E-1,1.2320377E-1,6.47383E-1,0E0,3.581893E-1,3.2894796E-1,1.4395311E-1,1.989119E-1,1.392895E-2,0E0,3.3942086E-1,3.1310076E-1,3.153643E-1,2.5505668E-1,7.1849726E-2,9.443724E-2,0E0,1.592476E-1,4.8604727E-2,4.074701E-2,0E0,0E0,2.5086823E-1,1.7027932E-1,4.2202026E-1,3.5652757E-1,6.2706076E-2,4.0021443E-1,2.1142256E-1,2.262348E-1,0E0,3.150615E-2,1.4121711E-2,0E0,3.8451727E-2,7.548888E-2,9.661317E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,39,39,40,40,41,41],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,42,44,-1,-1,46,48,50,52,54,56,58,60,-1,62,64,-1,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5142261E3,1.7831801E6,7.134432E7,1.7587205E6,1E0,1.5E1,1.121807E0,1.4915254E0,-2.5101267E-2,2.5437157E2,7.271717E6,1.0236775E1,8.959487E8,3.162807E7,2.3921824E-3,1.5454154E-1,8.045774E0,1.1481482E0,6.888377E6,1.1269E4,6.319223E8,9.8042805E-3,7.958E3,2.1298597E0,4.01057E5,-1.1849504E-2,-3.7826744E-3,3.2E2,6.374563E5,9.557399E6,1.688E3,3.5714287E-1,1.8370886E2,1.5057336E0,1.172766E3,-4.617763E-3,2.9234132E7,1.9791039E6,2.076043E-3,1.2804051E7,1.3935602E1,5.8E1,6.0630986E-3,4.8536216E-3,-1.6237972E-3,7.1889797E-4,-4.9230023E-3,1.6120102E-3,5.848031E-3,7.7294186E-4,-7.919235E-3,-3.6802352E-3,-3.889379E-4,3.4193338E-3,-1.5320074E-3,-1.494656E-2,-2.189571E-3,2.1012356E-3,9.451928E-3,-4.851383E-3,2.4210464E-3,2.7975189E-3,-2.4523248E-3,3.8099505E-3,1.1100649E-2,-5.7029994E-3,-2.1202887E-4,-3.460324E-3,4.5920955E-3,1.3560183E-2,6.1838217E-3],"split_indices":[64,40,56,40,89,3,46,65,0,4,57,68,12,55,0,50,70,65,63,1,12,0,2,50,9,0,0,10,40,57,2,65,4,54,4,0,9,40,0,63,70,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.78E2,8.94E2,8.4E1,6.44E2,2.5E2,7.7E1,7E0,6.42E2,2E0,2.07E2,4.3E1,4.1E1,3.6E1,5E0,2E0,1.81E2,4.61E2,4.3E1,1.64E2,2.2E1,2.1E1,5E0,3.6E1,2.4E1,1.2E1,3E0,2E0,9.9E1,8.2E1,1.37E2,3.24E2,2E1,2.3E1,1.29E2,3.5E1,3E0,1.9E1,1.7E1,4E0,1E1,2.6E1,1.8E1,6E0,1E1,2E0,7.9E1,2E1,4.6E1,3.6E1,1.25E2,1.2E1,2.23E2,1.01E2,1.1E1,9E0,7E0,1.6E1,1.21E2,8E0,2E1,1.5E1,1.7E1,2E0,2E0,1.5E1,4E0,6E0,2E0,2.4E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-1.8917252E-3,-1.0512708E-2,1.0442081E-1,-1.7715368E-2,7.758897E-2,1.5154395E-1,1.8226022E-2,-1.1593509E-2,-1.0966866E-1,-5.248559E-3,8.773372E-2,1.7331351E-1,3.553323E-2,-4.2172074E-2,1.3115981E-1,-4.3059573E-2,1.0995229E-3,4.636167E-2,-1.5535259E-1,6.5914495E-4,1.14513226E-1,1.9493887E-1,1.4675744E-2,8.8025176E-5,4.6908967E-3,2.9482334E-3,-8.474248E-2,-3.2349196E-4,8.791099E-3,-3.1053076E-2,-1.6336103E-1,5.87543E-3,-1.1502359E-1,-4.62984E-3,4.1673733E-3,-3.5278106E-1,-6.4621836E-2,5.7525896E-2,-1.3039706E-2,2.3495331E-2,1.2814945E-1,5.1594097E-3,2.1831545E-1,-1.5248241E-3,3.381258E-3,1.994678E-3,-1.4035307E-1,3.3207616E-4,-3.2210178E-3,-1.6213069E-2,-1.9905264E-3,1.2655145E-3,-5.515283E-4,-6.19595E-3,-7.581632E-4,-3.2337774E-3,2.8616302E-3,-4.433887E-3,-2.094219E-2,3.0641628E-3,-5.3010406E-3,-4.9507227E-3,4.1912287E-3,3.214156E-3,-1.8593414E-3,8.038321E-3,4.004162E-3,1.0824176E-2,2.7410176E-3,-8.594326E-3,-1.0765138E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,-1,47,49,51,53,55,-1,57,59,61,-1,63,65,-1,67,-1,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.2861456E-1,5.9596324E-1,3.0593234E-1,4.8678452E-1,1.4821646E-1,1.1759126E-1,1.9651888E-1,3.2515877E-1,3.921188E-1,0E0,1.5848786E-1,1.3779366E-1,1.9952592E-2,9.146708E-2,8.4595606E-2,3.329165E-1,3.2289445E-1,2.9831424E-2,7.295418E-1,2.91419E-1,6.1528325E-2,4.4998765E-2,1.9373672E-2,0E0,0E0,0E0,1.0588141E-1,0E0,0E0,3.2061732E-1,4.6264744E-1,2.1702518E-1,4.2800605E-2,3.520319E-2,0E0,2.9964447E-1,2.0781162E-1,9.3458265E-2,0E0,2.6622472E-2,7.113737E-2,0E0,6.0222983E-2,0E0,0E0,0E0,5.02827E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,39,39,40,40,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,-1,48,50,52,54,56,-1,58,60,62,-1,64,66,-1,68,-1,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6466942E3,4.82425E6,1.2792593E2,9.932432E0,6.1723955E9,4.6553187E0,5.428175E3,2.0671213E2,1E0,-5.248559E-3,8.01E2,1.8421166E7,2.7663412E7,1.7291568E2,1.4E1,1.3711089E6,4.217427E7,1.00011E5,9E1,1.8E1,1.7068776E10,4.36853E5,2.5134058E3,8.8025176E-5,4.6908967E-3,2.9482334E-3,2.5427E4,-3.2349196E-4,8.791099E-3,4.201202E-2,2.5247778E6,2.165E3,1.4872362E-1,6.293005E-1,4.1673733E-3,4.23E2,1.7E1,1.994E3,-1.3039706E-2,8.847733E4,1.4285715E0,5.1594097E-3,1E0,-1.5248241E-3,3.381258E-3,1.994678E-3,3.8771296E3,3.3207616E-4,-3.2210178E-3,-1.6213069E-2,-1.9905264E-3,1.2655145E-3,-5.515283E-4,-6.19595E-3,-7.581632E-4,-3.2337774E-3,2.8616302E-3,-4.433887E-3,-2.094219E-2,3.0641628E-3,-5.3010406E-3,-4.9507227E-3,4.1912287E-3,3.214156E-3,-1.8593414E-3,8.038321E-3,4.004162E-3,1.0824176E-2,2.7410176E-3,-8.594326E-3,-1.0765138E-3],"split_indices":[64,40,70,66,43,50,64,4,14,0,2,57,63,70,3,40,57,5,0,3,43,1,4,0,0,0,9,0,0,54,40,41,54,70,0,41,8,12,0,45,65,0,82,0,0,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.011E3,9.36E2,7.5E1,8.66E2,7E1,4.8E1,2.7E1,8.13E2,5.3E1,3E0,6.7E1,4E1,8E0,1.8E1,9E0,2.33E2,5.8E2,1.2E1,4.1E1,1.6E1,5.1E1,3.5E1,5E0,6E0,2E0,5E0,1.3E1,3E0,6E0,2.13E2,2E1,5.58E2,2.2E1,6E0,6E0,1.2E1,2.9E1,1.4E1,2E0,7E0,4.4E1,9E0,2.6E1,3E0,2E0,4E0,9E0,1.08E2,1.05E2,7E0,1.3E1,2.52E2,3.06E2,1.8E1,4E0,3E0,3E0,4E0,8E0,8E0,2.1E1,2E0,1.2E1,4E0,3E0,1.9E1,2.5E1,2.3E1,3E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-3.5421632E-3,-4.224384E-2,2.3771103E-2,-1.2345397E-1,-2.9822899E-2,-1.7321173E-2,3.8676195E-2,-8.9943424E-2,-1.7551953E-2,-5.2196704E-2,3.762624E-2,6.2106863E-2,-4.7560412E-2,4.892136E-2,-5.562106E-2,-1.0388773E-1,4.4272533E-3,5.3654355E-3,-7.402764E-2,1.1615404E-1,-6.9338186E-3,-1.2211711E-2,1.5189974E-1,-1.478804E-2,-1.1104534E-1,4.0226415E-2,1.5249948E-1,-2.7785602E-1,-1.9067617E-2,-1.1793451E-1,3.7692085E-2,-8.7599725E-2,5.8378883E-2,-4.830574E-2,-1.5125741E-1,5.972003E-2,1.0269971E-2,1.6402999E-1,-3.31026E-2,-5.1295396E-2,9.561711E-2,1.9128889E-1,-2.4563768E-3,-4.288184E-2,4.1968327E-2,-1.7052943E-1,7.3569184E-3,5.633209E-2,-9.253374E-3,1.8001008E-1,5.720987E-4,-1.9852E-2,9.446166E-4,6.1287116E-3,-5.7807125E-2,-6.646947E-3,-2.9258004E-3,-2.722109E-3,5.607535E-3,-9.516862E-4,-7.494889E-3,8.083007E-3,7.111108E-4,9.948654E-4,-3.1692577E-3,-2.6353013E-3,-8.772413E-3,-1.9693232E-4,6.584682E-3,1.0155014E-2,-2.1895664E-4,-1.3053509E-2,-7.9806126E-4,-5.664924E-3,1.1883778E-3,5.6886533E-3,7.665417E-4,6.343255E-3,1.2207708E-2,-8.493353E-3,-4.0667894E-4,3.646081E-3,-3.8098253E-4,-4.962986E-3,-1.6571166E-2,-1.5254385E-3,5.377657E-3,-4.1724783E-3,2.8860185E-3,-2.987287E-3,2.4450652E-3,3.6836949E-3,9.521983E-3,-1.2401337E-3,-8.344469E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,79,81,83,85,87,89,91,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0654728E0,4.1694874E-1,3.6264798E-1,4.4239545E-1,5.497651E-1,3.8244507E-1,4.2200464E-1,1.3992676E-1,0E0,3.4345865E-1,3.1923288E-1,2.954792E-1,2.376213E-1,3.469031E-1,3.4401047E-1,9.791595E-2,0E0,3.7967262E-1,3.860197E-1,1.8673295E-1,2.693908E-1,1.1026341E-1,1.7202389E-1,1.24576315E-1,2.7593353E-1,2.897979E-1,1.1333877E-1,2.852613E-1,2.3220731E-1,5.3235292E-2,4.814733E-2,1.3814487E-1,2.444658E-1,2.1867275E-1,1.7181706E-1,1.2374512E-1,0E0,7.810664E-2,2.0041406E-1,1.0881377E-1,1.353208E-2,3.0648232E-2,0E0,2.4967977E-1,4.955032E-2,2.7006316E-1,6.6205755E-2,2.678616E-1,3.1718013E-1,5.148965E-2,0E0,0E0,0E0,0E0,1.1338982E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,54,54],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,80,82,84,86,88,90,92,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.539777E5,3.964269E3,2.5367088E2,5.857143E0,3.35E2,9.917037E2,1.766486E7,5.8E1,-1.7551953E-2,1.4695653E0,1.5853742E1,3.3820656E7,3.364486E0,1.7379E4,1.172766E3,1E0,4.4272533E-3,3.73125E1,9.144571E5,5.7420593E2,2.5454E4,3.08564E5,2.8266037E2,1.4E1,6.0875E4,2E0,7.3274844E9,5.59E2,3E0,9.9347424E2,7.784E3,1.00033E5,9E0,7.6811076E-3,5E0,1.666618E-2,1.0269971E-2,9.18E2,5.7E1,1.3020051E-5,4E-1,2.1790265E2,-2.4563768E-3,1.594941E3,1E0,1E1,6.3157897E0,6.6903913E-1,2.9541414E1,9.9347424E2,5.720987E-4,-1.9852E-2,9.446166E-4,6.1287116E-3,3.206931E2,-6.646947E-3,-2.9258004E-3,-2.722109E-3,5.607535E-3,-9.516862E-4,-7.494889E-3,8.083007E-3,7.111108E-4,9.948654E-4,-3.1692577E-3,-2.6353013E-3,-8.772413E-3,-1.9693232E-4,6.584682E-3,1.0155014E-2,-2.1895664E-4,-1.3053509E-2,-7.9806126E-4,-5.664924E-3,1.1883778E-3,5.6886533E-3,7.665417E-4,6.343255E-3,1.2207708E-2,-8.493353E-3,-4.0667894E-4,3.646081E-3,-3.8098253E-4,-4.962986E-3,-1.6571166E-2,-1.5254385E-3,5.377657E-3,-4.1724783E-3,2.8860185E-3,-2.987287E-3,2.4450652E-3,3.6836949E-3,9.521983E-3,-1.2401337E-3,-8.344469E-3],"split_indices":[40,40,64,65,64,45,63,3,0,65,70,7,68,2,4,16,0,64,57,64,9,1,4,3,1,29,5,0,8,64,1,5,3,50,8,50,0,0,0,50,70,4,0,45,24,8,68,65,68,64,0,0,0,0,70,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.006E3,4.16E2,5.9E2,5.4E1,3.62E2,1.57E2,4.33E2,4.9E1,5E0,2.72E2,9E1,4.3E1,1.14E2,3.91E2,4.2E1,4.6E1,3E0,7.5E1,1.97E2,3.2E1,5.8E1,2.4E1,1.9E1,7.6E1,3.8E1,3.62E2,2.9E1,5E0,3.7E1,4.2E1,4E0,2.7E1,4.8E1,1.49E2,4.8E1,2.2E1,1E1,7E0,5.1E1,1.8E1,6E0,1.6E1,3E0,5.1E1,2.5E1,2.5E1,1.3E1,2.73E2,8.9E1,2.4E1,5E0,3E0,2E0,7E0,3E1,2.7E1,1.5E1,2E0,2E0,1.5E1,1.2E1,1.2E1,3.6E1,3.4E1,1.15E2,1.5E1,3.3E1,1.3E1,9E0,5E0,2E0,2E0,4.9E1,9E0,9E0,4E0,2E0,1.1E1,5E0,9E0,4.2E1,1.4E1,1.1E1,2E1,5E0,1E1,3E0,1.1E1,2.62E2,4.7E1,4.2E1,6E0,1.8E1,2.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-2.7955426E-3,-3.20523E-2,3.5786074E-2,-4.156337E-2,2.8518397E-2,5.982994E-2,-5.6198835E-2,-1.7671496E-2,-6.9536105E-2,6.171004E-2,-7.605136E-2,8.3527274E-2,9.737068E-3,-1.534143E-1,-3.1114433E-2,-4.257412E-2,2.781808E-2,-2.2244659E-1,-5.67043E-2,-2.5651855E-2,1.0716876E-1,-2.948455E-2,-9.211183E-3,5.715681E-2,1.4298609E-1,-1.3508686E-2,2.4354547E-1,-7.2962895E-2,-1.2374631E-2,1.2202484E-1,-4.4364005E-2,-4.7258466E-2,7.392468E-2,2.0844026E-1,8.592292E-3,-4.6688538E-2,-3.4080836E-1,-4.307289E-2,-2.2922361E-1,1.9141734E-2,-8.75035E-2,8.825581E-3,7.124039E-2,2.14975E-3,-6.148045E-2,-2.616924E-1,7.334925E-2,1.5488972E-1,-4.2324197E-3,-2.611382E-2,1.594908E-1,3.1445855E-3,1.4183695E-2,-5.7852E-3,2.6413277E-2,1.8975196E-3,6.9840453E-3,-1.6294431E-2,-1.0103838E-1,-8.994113E-4,-3.5101078E-3,5.3829285E-3,5.591533E-4,1.2410337E-2,2.925959E-3,1.5403923E-3,-5.1311813E-3,1.7395862E-3,-6.4634364E-3,-7.0351437E-3,-2.022714E-2,-4.4890493E-3,-3.2333622E-4,-5.447276E-3,-2.9038114E-2,1.9723396E-3,-1.2642667E-3,1.7765358E-4,-6.147193E-3,4.571885E-3,-1.4657266E-3,-4.1835024E-3,-3.5566007E-4,-2.2123843E-2,3.6149295E-3,8.210936E-3,2.7481667E-3,-3.0800974E-4,7.985397E-3,-2.4388328E-3,1.5671797E-3,2.7225483E-3,8.622393E-3,-7.94387E-4,2.8163914E-3,-2.0175634E-3,3.0155848E-3,-7.0628044E-3,4.8042185E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,-1,81,83,85,87,-1,89,91,-1,-1,-1,93,-1,-1,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1175036E0,3.2552522E-1,9.4774675E-1,3.242435E-1,2.7194798E-1,4.016025E-1,2.1226129E-1,3.0024612E-1,4.269004E-1,2.370593E-1,1.0423223E-1,3.5245752E-1,6.011458E-1,1.4832571E-1,1.5150364E-1,9.681186E-2,3.2463378E-1,3.3773977E-1,4.7927654E-1,5.9856668E-2,1.0538259E-1,3.949099E-2,0E0,8.5042965E-1,2.1019733E-1,2.2441372E-1,9.310961E-2,6.651784E-2,0E0,8.260511E-3,1.04938984E-1,1.3167718E-1,1.7277043E-2,6.322825E-2,2.6182905E-1,6.8940654E-2,1.1938012E-1,3.814153E-1,5.937379E-1,1.5620105E-2,3.8616948E-2,0E0,8.782239E-2,0E0,1.6792022E-2,6.369425E-1,2.0874637E-1,2.0649362E-1,0E0,1.5621556E-1,8.678094E-3,0E0,0E0,0E0,8.688424E-3,0E0,0E0,1.0704236E-1,1.3095173E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,49,49,50,50,54,54,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,-1,82,84,86,88,-1,90,92,-1,-1,-1,94,-1,-1,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.885813E2,9.750085E6,1E0,2.3265307E0,1.9460785E1,3.7E1,7.937392E-3,5.51E2,5.8182236E1,1.088E3,9.12225E3,1.3457517E10,4.7263342E2,2.511E3,5.268293E0,1.3E1,2.020055E6,1.1015E4,2.4718345E6,9.7E2,7E0,1.9545455E0,-9.211183E-3,1.2825651E0,6.292039E0,1E0,4.907764E3,1E0,-1.2374631E-2,3.5609E4,1.2E1,3.5364E4,3.26E2,1.4865319E5,1.7E1,2.9849844E7,7.86127E6,2.064332E2,8.16E0,2.7563172E6,1.1149426E0,8.825581E-3,6.9508715E0,2.14975E-3,6.648E3,3.0984934E8,2.82E2,2.022131E6,-4.2324197E-3,1.1251919E6,3.3412E5,3.1445855E-3,1.4183695E-2,-5.7852E-3,1.6E1,1.8975196E-3,6.9840453E-3,2.6007314E5,3.6828618E8,-8.994113E-4,-3.5101078E-3,5.3829285E-3,5.591533E-4,1.2410337E-2,2.925959E-3,1.5403923E-3,-5.1311813E-3,1.7395862E-3,-6.4634364E-3,-7.0351437E-3,-2.022714E-2,-4.4890493E-3,-3.2333622E-4,-5.447276E-3,-2.9038114E-2,1.9723396E-3,-1.2642667E-3,1.7765358E-4,-6.147193E-3,4.571885E-3,-1.4657266E-3,-4.1835024E-3,-3.5566007E-4,-2.2123843E-2,3.6149295E-3,8.210936E-3,2.7481667E-3,-3.0800974E-4,7.985397E-3,-2.4388328E-3,1.5671797E-3,2.7225483E-3,8.622393E-3,-7.94387E-4,2.8163914E-3,-2.0175634E-3,3.0155848E-3,-7.0628044E-3,4.8042185E-4],"split_indices":[64,9,6,68,70,8,54,2,59,41,45,43,68,2,68,8,5,9,59,2,3,65,0,66,50,28,4,17,0,1,8,9,12,40,3,5,5,64,68,57,65,0,68,0,41,7,41,12,0,40,1,0,0,0,0,0,0,45,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.88E2,5.62E2,4.26E2,4.86E2,7.6E1,3.38E2,8.8E1,2.63E2,2.23E2,5.8E1,1.8E1,2.29E2,1.09E2,1.7E1,7.1E1,1.7E2,9.3E1,1.6E1,2.07E2,2E1,3.8E1,1.4E1,4E0,1.6E2,6.9E1,1E2,9E0,1.1E1,6E0,5E0,6.6E1,1.64E2,6E0,8E0,8.5E1,7E0,9E0,1.93E2,1.4E1,1.2E1,8E0,1E1,2.8E1,4E0,1E1,7E0,1.53E2,6.6E1,3E0,9.4E1,6E0,3E0,6E0,7E0,4E0,2E0,3E0,4.5E1,2.1E1,8.5E1,7.9E1,3E0,3E0,5E0,3E0,7.1E1,1.4E1,4E0,3E0,4E0,5E0,7.6E1,1.17E2,1.2E1,2E0,8E0,4E0,3E0,5E0,2.2E1,6E0,6E0,4E0,4E0,3E0,1.6E1,1.37E2,7E0,5.9E1,6.5E1,2.9E1,2E0,4E0,2E0,2E0,3.4E1,1.1E1,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[-3.253364E-3,-3.669352E-2,1.8647894E-2,-2.6028449E-2,-1.19129755E-1,3.0873418E-2,-5.422794E-2,-1.716571E-2,-1.1605815E-1,9.650392E-2,-1.670659E-1,9.733114E-2,1.6843775E-2,-1.2937048E-1,-2.2853972E-2,-7.2537825E-2,-5.498884E-3,-6.338569E-2,-1.5329249E-2,-1.6698628E-3,6.4146803E-3,-1.9526164E-1,2.667629E-3,1.1420129E-1,-1.2383529E-1,1.2742622E-1,8.107947E-3,5.1026186E-4,-1.4154E-1,-4.6988405E-2,5.911773E-2,-5.8003716E-2,-1.4460044E-2,1.6308254E-2,-4.3170244E-2,-1.8416764E-1,-2.837094E-3,-2.4753423E-1,-4.5054015E-2,2.0503162E-1,9.0511054E-2,-1.8392192E-2,6.688626E-2,1.5110731E-1,-3.4962283E-3,1.6796197E-3,1.759896E-1,-1.0138584E-1,-1.2578293E-2,-8.120288E-2,3.82589E-2,9.536899E-2,-3.2458585E-2,5.7397305E-4,-4.1279616E-3,1.1773241E-3,-9.575024E-3,4.758506E-3,-2.9562293E-3,-1.018373E-2,-3.7573902E-3,5.1615904E-3,-2.3889623E-3,-2.03609E-3,-1.2549836E-2,-8.955935E-3,7.7172485E-4,1.0827644E-2,2.1272793E-3,2.0637116E-3,7.415133E-3,4.8564803E-3,2.638354E-4,2.5930465E-3,7.532478E-3,-3.6513265E-3,5.3369236E-4,1.3726094E-2,2.796607E-3,8.657914E-4,-5.682314E-3,-1.3954111E-3,-7.298386E-3,-1.051107E-3,6.145071E-3,6.9839796E-3,-1.4059171E-4,-4.10005E-3,1.6152614E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,-1,37,-1,39,41,43,45,-1,47,49,51,53,-1,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.4771345E-1,3.5209125E-1,5.513874E-1,2.844574E-1,4.9160832E-1,4.915886E-1,2.068626E-1,2.1149462E-1,3.4575447E-1,5.6677386E-2,2.5256872E-1,3.574046E-1,4.2208758E-1,4.6021223E-2,1.2939163E-1,1.8245444E-1,2.2415324E-1,1.9693331E-1,0E0,0E0,0E0,2.558068E-1,0E0,1.664964E-1,3.960458E-1,1.6574699E-1,4.389381E-1,0E0,9.974146E-2,1.4894132E-1,5.2891895E-2,1.2469247E-1,0E0,3.679067E-1,3.176943E-1,1.6454428E-2,1.1348942E-1,1.18421555E-1,9.601646E-2,7.192421E-2,2.1905297E-1,0E0,1.1268528E-2,2.6113689E-2,0E0,3.1801546E-1,1.9390854E-1,5.6021452E-2,0E0,1.3732308E-1,9.0675436E-2,6.183765E-2,2.2346668E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,45,45,46,46,47,47,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,-1,38,-1,40,42,44,46,-1,48,50,52,54,-1,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,8.172E3,1E0,6.4153387E-6,3.5714287E-1,1.684E0,5.4225586E-2,3.175E1,9.639872E0,4.57E2,1.3683688E7,3.2952412E5,3.5E1,1.2131076E2,1.75E2,1.43E2,1E0,5E0,-1.5329249E-2,-1.6698628E-3,6.4146803E-3,4.8E0,2.667629E-3,8.051603E7,3.167E3,1E0,4.435876E2,5.1026186E-4,4.9E1,1.4595818E7,5.526E3,3.984E3,-1.4460044E-2,1.4992306E8,1.8738E4,7.635E3,1.70698E6,9.836066E-1,8.634796E0,1.7540792E5,4.439357E8,-1.8392192E-2,6.676E3,4.814389E4,-3.4962283E-3,3.927456E7,4.8647133E-1,7.9146667E0,-1.2578293E-2,1.2434699E8,7E0,1.131645E6,6.165E3,5.7397305E-4,-4.1279616E-3,1.1773241E-3,-9.575024E-3,4.758506E-3,-2.9562293E-3,-1.018373E-2,-3.7573902E-3,5.1615904E-3,-2.3889623E-3,-2.03609E-3,-1.2549836E-2,-8.955935E-3,7.7172485E-4,1.0827644E-2,2.1272793E-3,2.0637116E-3,7.415133E-3,4.8564803E-3,2.638354E-4,2.5930465E-3,7.532478E-3,-3.6513265E-3,5.3369236E-4,1.3726094E-2,2.796607E-3,8.657914E-4,-5.682314E-3,-1.3954111E-3,-7.298386E-3,-1.051107E-3,6.145071E-3,6.9839796E-3,-1.4059171E-4,-4.10005E-3,1.6152614E-3],"split_indices":[2,41,6,49,65,66,54,67,66,0,57,45,0,45,6,2,24,6,0,0,0,66,0,7,2,28,68,0,3,59,2,9,0,5,9,1,5,68,68,45,12,0,10,57,0,7,54,70,0,12,6,9,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.019E3,4.03E2,6.16E2,3.58E2,4.5E1,5.28E2,8.8E1,3.27E2,3.1E1,8E0,3.7E1,9.1E1,4.37E2,2.5E1,6.3E1,5.6E1,2.71E2,2.6E1,5E0,2E0,6E0,3.3E1,4E0,8.5E1,6E0,3.1E1,4.06E2,2E0,2.3E1,4.9E1,1.4E1,5.4E1,2E0,1.72E2,9.9E1,8E0,1.8E1,2.4E1,9E0,1.6E1,6.9E1,2E0,4E0,2.8E1,3E0,3.92E2,1.4E1,1.9E1,4E0,3.5E1,1.4E1,1E1,4E0,1.7E1,3.7E1,1.66E2,6E0,1.2E1,8.7E1,5E0,3E0,5E0,1.3E1,3E0,2.1E1,2E0,7E0,1.3E1,3E0,4.3E1,2.6E1,2E0,2E0,4E0,2.4E1,4.2E1,3.5E2,6E0,8E0,3E0,1.6E1,2.2E1,1.3E1,9E0,5E0,6E0,4E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[4.232328E-3,-2.3729976E-2,4.286836E-2,-3.32392E-2,2.7691716E-2,6.3896574E-2,-4.1748807E-2,-3.0171506E-2,-2.4173034E-2,9.316352E-3,1.294033E-1,9.647217E-2,1.7794885E-2,-7.006137E-2,1.5339646E-2,-2.3317581E-2,-1.4760157E-1,-7.2661266E-3,9.406325E-2,-2.1256311E-4,1.5212746E-1,1.0147181E-1,-1.2125719E-2,1.68468E-1,1.3048201E-3,-1.0653662E-1,-1.1509416E-2,3.8431685E-2,-6.365815E-3,-1.9375263E-2,-2.2178495E-1,-1.7362024E-1,2.0857882E-3,-1.17980346E-1,1.6588101E-2,1.16353944E-1,6.807381E-4,1.729981E-1,1.4628961E-3,1.1430199E-1,3.424604E-2,1.2916215E-2,4.8763953E-2,-1.2992195E-2,1.19672604E-1,-8.767714E-2,-1.2790956E-2,-4.79582E-2,7.865326E-2,5.884159E-2,-6.32285E-3,-4.243464E-3,-4.879581E-4,5.5846297E-3,-1.9003475E-2,-2.786017E-3,-8.583211E-3,2.1206606E-4,-1.6734207E-2,1.1691211E-3,-4.824014E-3,2.2692485E-3,6.338025E-3,2.3551371E-3,9.049412E-3,5.8723926E-3,1.8517969E-3,-8.080685E-3,4.482831E-3,-1.8870148E-4,6.3502137E-3,-1.4197684E-3,2.16874E-3,2.0904657E-3,7.1568526E-3,-4.5331027E-3,7.743082E-4,3.535059E-3,-3.2035122E-3,6.5637525E-4,4.663907E-3,4.686045E-4,4.8977155E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,43,45,47,49,-1,51,53,55,-1,57,59,61,-1,63,-1,65,67,-1,69,71,73,75,-1,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0910931E0,2.8746295E-1,7.5711095E-1,7.3281395E-1,1.7074123E-1,5.088918E-1,1.3846196E-1,3.9335278E-1,0E0,1.1170326E-1,4.3344453E-2,3.8542807E-1,3.5259473E-1,1.2050855E-1,1.0841068E-1,3.6109367E-1,1.4347619E-1,1.7890422E-1,2.123838E-2,0E0,2.461347E-2,1.6600156E-1,0E0,1.7521963E-1,2.1986228E-1,8.907956E-2,7.9357415E-2,1.015434E-1,0E0,2.923066E-1,6.434939E-1,2.458626E-2,0E0,3.6410284E-1,6.3050546E-2,7.6667666E-3,0E0,1.9137502E-2,0E0,1.6024613E-1,4.5656565E-1,0E0,3.8420264E-2,1.2660547E-1,3.0433536E-2,3.9663196E-2,0E0,5.1636618E-2,8.86054E-3,5.4698557E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,37,37,39,39,40,40,42,42,43,43,44,44,45,45,47,47,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,44,46,48,50,-1,52,54,56,-1,58,60,62,-1,64,-1,66,68,-1,70,72,74,76,-1,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.885813E2,2.5247778E6,1E0,4.5158855E6,2.0833333E0,8E0,4.6E1,2.8025E4,-2.4173034E-2,3.3343E4,2.7391305E0,5.619497E7,1.2083E4,1.3026532E6,3.5510652E6,2.295413E6,5.501002E9,1.5933333E2,7.4711205E6,-2.1256311E-4,4.70127E5,2.8366232E0,-1.2125719E-2,1.562E5,3.4396572E0,7.03E3,2.9E1,2.614094E0,-6.365815E-3,3.89E2,7.44488E5,5.44E2,2.0857882E-3,6E0,1E0,2.68334E5,6.807381E-4,1.5942905E-4,1.4628961E-3,1E0,3.921E3,1.2916215E-2,2.2E1,1.3388E4,2.276E4,6.0078123E-6,-1.2790956E-2,1.4283589E6,1.0041E5,2.688E3,-6.32285E-3,-4.243464E-3,-4.879581E-4,5.5846297E-3,-1.9003475E-2,-2.786017E-3,-8.583211E-3,2.1206606E-4,-1.6734207E-2,1.1691211E-3,-4.824014E-3,2.2692485E-3,6.338025E-3,2.3551371E-3,9.049412E-3,5.8723926E-3,1.8517969E-3,-8.080685E-3,4.482831E-3,-1.8870148E-4,6.3502137E-3,-1.4197684E-3,2.16874E-3,2.0904657E-3,7.1568526E-3,-4.5331027E-3,7.743082E-4,3.535059E-3,-3.2035122E-3,6.5637525E-4,4.663907E-3,4.686045E-4,4.8977155E-3],"split_indices":[64,40,6,59,65,29,6,41,0,41,70,59,9,40,40,40,5,4,40,0,12,50,0,10,54,41,3,65,0,1,9,12,0,8,16,1,0,50,0,82,2,0,3,2,10,49,0,40,1,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,5.85E2,4.23E2,4.94E2,9.1E1,3.39E2,8.4E1,4.92E2,2E0,7.8E1,1.3E1,1.98E2,1.41E2,5.6E1,2.8E1,4.66E2,2.6E1,6.6E1,1.2E1,2E0,1.1E1,1.96E2,2E0,1.3E1,1.28E2,3.4E1,2.2E1,2.5E1,3E0,4.58E2,8E0,2.3E1,3E0,1.1E1,5.5E1,9E0,3E0,9E0,2E0,1.64E2,3.2E1,6E0,7E0,1.15E2,1.3E1,3.2E1,2E0,1.6E1,6E0,2.3E1,2E0,4.8E1,4.1E2,3E0,5E0,3E0,2E1,8E0,3E0,5.2E1,3E0,3E0,6E0,2E0,7E0,1.38E2,2.6E1,7E0,2.5E1,5E0,2E0,8.9E1,2.6E1,5E0,8E0,2.9E1,3E0,2E0,1.4E1,2E0,4E0,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[1.4211327E-3,1.1759696E-2,-6.598768E-2,-1.1377993E-2,4.781651E-2,-4.8193652E-2,-2.3842897E-1,-2.6170516E-3,-1.0629509E-1,1.2831847E-1,3.9156843E-2,-5.757539E-2,6.4145843E-3,-2.4113972E-3,-2.7036482E-1,9.368328E-2,-1.3088113E-2,-6.664528E-2,-2.8751343E-2,1.4991505E-1,5.6469996E-2,-2.6814751E-2,4.900422E-2,-6.7380294E-2,5.211409E-2,-1.4173672E-2,-3.59779E-3,1.1115433E-1,-9.168409E-3,1.8296827E-2,-3.0539831E-2,1.0581657E-2,-8.7255955E-2,2.3845495E-3,7.1866494E-3,4.986478E-3,-1.6437694E-3,-1.2819872E-4,-1.2261963E-2,5.398723E-2,-9.817969E-2,-4.811667E-2,-1.3480482E-1,-4.168556E-2,5.5850367E-3,5.683079E-3,-4.8751053E-3,-2.3810936E-3,2.3078115E-3,-7.858672E-3,-1.0046519E-3,4.8173796E-3,-5.1221927E-3,-2.182579E-3,5.3374325E-3,1.7789975E-3,5.5578314E-3,-7.8067346E-4,-7.336508E-3,-3.4271693E-3,4.8628406E-4,-2.3542645E-3,-9.51334E-3,7.806958E-4,-3.971833E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,-1,33,35,37,39,41,43,-1,-1,45,-1,47,49,-1,51,-1,-1,-1,-1,53,-1,55,57,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.865357E-1,7.130118E-1,3.9047283E-1,4.329662E-1,2.2781366E-1,2.1952274E-1,5.7059646E-2,4.832254E-1,8.64887E-1,4.118812E-2,1.9815704E-1,1.2719333E-1,0E0,0E0,4.9220383E-2,2.6302242E-1,2.371743E-1,2.7722037E-1,0E0,7.2810054E-3,4.7139063E-2,2.563135E-1,1.9892311E-1,1.3119072E-1,7.0241384E-2,0E0,0E0,1.3609284E-1,0E0,3.4991562E-1,3.349166E-1,0E0,1.9816199E-1,0E0,0E0,0E0,0E0,2.0950176E-1,0E0,2.570393E-1,4.2254694E-2,1.3145499E-1,1.2834635E-1,1.461093E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,32,32,37,37,39,39,40,40,41,41,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,-1,34,36,38,40,42,44,-1,-1,46,-1,48,50,-1,52,-1,-1,-1,-1,54,-1,56,58,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.885813E2,8.719101E0,4.1388846E2,4.1E1,1.03992E3,2.3636363E0,7E0,2.848973E6,6.441311E-1,1.98E2,1.4172E4,6.4145843E-3,-2.4113972E-3,1.3E1,1.6527965E7,1.480315E0,1.0918E4,-2.8751343E-2,2.5435904E7,3.1E1,1.2655613E3,4.1325716E7,2.124E3,1.7E1,-1.4173672E-2,-3.59779E-3,1E0,-9.168409E-3,1.6262903E2,1.4268688E-5,1.0581657E-2,9.4E1,2.3845495E-3,7.1866494E-3,4.986478E-3,-1.6437694E-3,4.39776E6,-1.2261963E-2,1.4176E4,3.433001E-2,7.785302E5,2.67637E5,7.758433E8,5.5850367E-3,5.683079E-3,-4.8751053E-3,-2.3810936E-3,2.3078115E-3,-7.858672E-3,-1.0046519E-3,4.8173796E-3,-5.1221927E-3,-2.182579E-3,5.3374325E-3,1.7789975E-3,5.5578314E-3,-7.8067346E-4,-7.336508E-3,-3.4271693E-3,4.8628406E-4,-2.3542645E-3,-9.51334E-3,7.806958E-4,-3.971833E-3],"split_indices":[6,64,66,4,0,58,70,3,63,50,10,41,0,0,8,57,65,9,0,7,0,64,63,41,3,0,0,82,0,4,50,0,10,0,0,0,0,9,0,2,69,40,9,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.83E2,8.53E2,1.3E2,5.2E2,3.33E2,1.19E2,1.1E1,4.77E2,4.3E1,3.1E1,3.02E2,1.14E2,5E0,2E0,9E0,4.6E1,4.31E2,4.1E1,2E0,2.3E1,8E0,3.9E1,2.63E2,1.05E2,9E0,7E0,2E0,4.4E1,2E0,1.54E2,2.77E2,2E0,3.9E1,2E0,2.1E1,5E0,3E0,3.6E1,3E0,2.55E2,8E0,8.3E1,2.2E1,4E0,5E0,4.2E1,2E0,4.8E1,1.06E2,1.5E1,2.62E2,4E0,3.5E1,2.6E1,1E1,2.09E2,4.6E1,4E0,4E0,5.7E1,2.6E1,1.1E1,1.1E1,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[1.7902923E-3,-3.428713E-2,2.2794565E-2,-8.440451E-2,-2.1385457E-2,3.4987193E-2,-5.007575E-2,-4.498257E-2,-2.3909344E-1,2.6718166E-3,-5.21771E-2,5.528346E-2,5.482286E-3,-6.732814E-2,4.9621522E-2,-7.452518E-2,5.9519403E-2,-1.4924598E-1,-2.0616949E-2,-1.1012213E-1,9.839127E-3,2.810082E-2,-7.678759E-2,9.432513E-2,3.150572E-2,-3.1096467E-3,1.5610024E-1,1.2183232E-2,-8.8043705E-2,1.498333E-2,4.285807E-3,-2.7850883E-2,-1.20039016E-1,6.479648E-3,-1.4621599E-3,4.7991588E-4,-2.3039928E-1,-1.1109512E-3,-8.783018E-3,3.3880502E-4,1.06853925E-1,-6.9691725E-3,4.5734968E-2,-9.4042465E-2,1.8599696E-2,-1.1605207E-2,1.0829364E-1,4.8465766E-2,-8.983165E-2,8.875011E-3,-8.812964E-3,1.0581929E-3,9.089593E-3,-5.4800906E-3,4.888248E-2,-1.05479494E-1,-6.604621E-3,2.598372E-3,-6.999367E-4,-4.5101247E-3,-2.4600525E-4,-6.927334E-3,-1.1775449E-4,3.1535707E-3,-2.2031195E-3,-1.2809611E-2,-2.601034E-3,1.8491567E-3,-1.1383356E-3,6.559131E-3,1.613224E-3,-1.0468857E-3,4.3840692E-3,-3.6353006E-3,-1.2608431E-2,-7.371065E-3,2.5558714E-3,-4.5376923E-3,1.9230476E-3,3.8632373E-3,7.4569397E-3,8.5377606E-4,5.2546654E-3,-1.1543272E-2,-1.6946517E-3,-9.853718E-4,3.6524963E-3,3.5981988E-3,-2.0366788E-3,-3.5121078E-3,-8.470343E-3,-6.6777677E-3,2.563416E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,57,-1,59,61,-1,63,-1,65,-1,-1,67,69,-1,71,73,75,77,79,81,83,-1,85,-1,-1,-1,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.578038E-1,2.3614305E-1,5.6327176E-1,4.466052E-1,2.1792373E-1,3.2426792E-1,1.5986235E-1,1.9275859E-1,2.1571743E-1,1.3528274E-1,2.570204E-1,2.9501987E-1,2.877755E-1,1.2954828E-1,1.9787285E-2,9.862447E-2,6.983101E-2,1.5792412E-1,0E0,6.506621E-2,1.4466627E-1,1.05245166E-1,1.6469276E-1,1.8029916E-1,4.1798913E-1,2.3796901E-1,6.2175184E-2,8.7869085E-2,8.7340504E-2,1.2173704E-2,0E0,3.9032303E-2,8.6393684E-2,0E0,3.264048E-2,0E0,5.1610142E-2,0E0,0E0,1.4541228E-1,3.1085327E-2,0E0,1.0096949E-1,2.0408636E-1,1.1507516E-1,7.406856E-2,1.262294E-1,3.4656882E-1,2.0443633E-1,0E0,2.317135E-1,0E0,0E0,0E0,4.191752E-2,1.0270339E-1,1.1205626E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,34,34,36,36,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,58,-1,60,62,-1,64,-1,66,-1,-1,68,70,-1,72,74,76,78,80,82,84,-1,86,-1,-1,-1,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.82E2,5.8182236E1,1E0,2.4332604E0,4.084788E0,8E0,1.4433751E0,3.5443038E-1,2.5435904E7,1.02E2,3.5318289E3,3.72381E5,1.5017116E5,1.2909952E1,1.5787431E0,1.08712E5,7E0,2.7726634E5,-2.0616949E-2,1.1E1,3.91E0,1.6157333E-4,2.4E1,3.693316E7,2.4822025E0,1.0366492E0,1.6493898E6,1.9471E4,3.5E1,2E0,4.285807E-3,1.1E1,5.66946E5,6.479648E-3,1.5254237E-1,4.7991588E-4,3.720198E2,-1.1109512E-3,-8.783018E-3,3.883E4,2.2460432E0,-6.9691725E-3,3.9125E1,4.586207E0,6.44E2,2.1605E4,2.8367348E0,5.817547E2,3.9157894E0,8.875011E-3,2.4693803E2,1.0581929E-3,9.089593E-3,-5.4800906E-3,2.83475E5,2.9697892E-1,7.87E3,2.598372E-3,-6.999367E-4,-4.5101247E-3,-2.4600525E-4,-6.927334E-3,-1.1775449E-4,3.1535707E-3,-2.2031195E-3,-1.2809611E-2,-2.601034E-3,1.8491567E-3,-1.1383356E-3,6.559131E-3,1.613224E-3,-1.0468857E-3,4.3840692E-3,-3.6353006E-3,-1.2608431E-2,-7.371065E-3,2.5558714E-3,-4.5376923E-3,1.9230476E-3,3.8632373E-3,7.4569397E-3,8.5377606E-4,5.2546654E-3,-1.1543272E-2,-1.6946517E-3,-9.853718E-4,3.6524963E-3,3.5981988E-3,-2.0366788E-3,-3.5121078E-3,-8.470343E-3,-6.6777677E-3,2.563416E-3],"split_indices":[2,59,6,68,66,29,50,68,7,2,59,9,45,68,50,7,10,40,0,3,66,54,3,7,54,65,57,1,3,8,0,3,9,0,65,0,67,0,0,9,65,0,4,65,1,9,66,64,68,0,68,0,0,0,1,50,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,3.67E2,6.31E2,7.4E1,2.93E2,5.41E2,9E1,6E1,1.4E1,1.65E2,1.28E2,3.2E2,2.21E2,7.7E1,1.3E1,4.7E1,1.3E1,1.1E1,3E0,9E0,1.56E2,3E1,9.8E1,1.2E2,2E2,2.1E2,1.1E1,1.6E1,6.1E1,8E0,5E0,2.4E1,2.3E1,5E0,8E0,4E0,7E0,5E0,4E0,1.43E2,1.3E1,2E0,2.8E1,8.3E1,1.5E1,1.4E1,1.06E2,1.76E2,2.4E1,5E0,2.05E2,3E0,8E0,3E0,1.3E1,5E1,1.1E1,3E0,5E0,5E0,1.9E1,1.8E1,5E0,3E0,5E0,5E0,2E0,5.5E1,8.8E1,8E0,5E0,1.2E1,1.6E1,7.8E1,5E0,2E0,1.3E1,5E0,9E0,7.5E1,3.1E1,1.22E2,5.4E1,5E0,1.9E1,1.8E2,2.5E1,1E1,3E0,3.8E1,1.2E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-3.4017644E-3,-1.2908241E-2,6.294422E-2,-6.3309046E-3,-7.508266E-2,7.348068E-2,-1.2667277E-1,-2.4515636E-2,1.9289687E-2,-5.021362E-2,-2.3555778E-1,3.5061948E-2,1.18247144E-1,-8.096701E-3,-8.147678E-5,1.0860174E-2,-4.4869516E-2,2.5922269E-2,-1.4494744E-1,-8.68691E-2,2.9771503E-2,6.645227E-4,-1.3445382E-2,6.453403E-2,-6.107169E-2,1.6591364E-1,7.5991996E-2,2.0317262E-2,-1.8436766E-1,-3.6692895E-2,-1.50197E-1,1.9202014E-2,1.4417607E-1,-2.5360692E-1,2.4089774E-2,-5.5912144E-2,-1.8281984E-1,7.9720706E-2,-3.6506712E-2,-5.474997E-2,8.165523E-2,-3.0918955E-3,-1.3517544E-1,1.8948969E-1,-2.403612E-4,8.520286E-2,-1.7627153E-3,-2.1603175E-3,2.085497E-3,-1.0389893E-2,-3.949217E-3,-2.5283676E-3,1.909331E-3,-8.789621E-3,3.3347E-3,1.1869626E-3,-4.318756E-3,2.959045E-3,8.890555E-3,-1.6766023E-2,-5.2248747E-3,-1.5585753E-3,4.291062E-3,4.4685023E-4,-4.2087696E-3,3.2641654E-4,-1.1341339E-2,1.0418143E-3,6.9812345E-3,-5.8052805E-3,5.9702655E-4,-4.128686E-3,1.0133922E-3,4.6149786E-3,-1.6013391E-3,1.490802E-3,-4.448857E-3,-8.646786E-3,-7.5930126E-5,9.916904E-3,5.849061E-3,4.935699E-3,1.4777492E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,-1,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.3405186E-1,3.589847E-1,2.6104227E-1,3.713322E-1,3.2340422E-1,2.022385E-1,4.3056406E-2,3.3602354E-1,3.643209E-1,2.1954627E-1,1.6146511E-1,1.909363E-1,9.817046E-2,0E0,0E0,3.1946087E-1,2.4941075E-1,2.5073946E-1,2.4488679E-1,1.4126566E-1,8.2363404E-2,0E0,0E0,1.08627155E-1,6.832637E-2,1.0186565E-1,3.5926387E-2,2.7923638E-1,1.3480067E-2,3.987299E-1,2.0650178E-1,2.3023452E-1,5.4486454E-2,8.313161E-2,2.7307328E-2,9.467688E-2,1.4784259E-1,5.4264404E-2,5.0589032E-2,2.0876525E-2,1.0218227E-1,3.647674E-2,4.9298763E-2,8.623183E-3,0E0,3.0767828E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,-1,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,2E0,7.134432E7,3.0291306E2,9.639872E0,1.8697667E3,1.468542E0,1.5132743E0,1.853776E7,5.7E1,1.2131076E2,8.710612E1,1.7103828E7,-8.096701E-3,-8.147678E-5,3.440238E2,2.8025E4,5.6530495E6,3.0984934E8,1.2E1,2.2166292E1,6.645227E-4,-1.3445382E-2,1.3935602E1,5.5933E5,2.0663544E9,7.466243E0,3.56E2,3.6255838E2,6.512E4,4.83929E9,5.159139E8,1.3E1,4.626E3,6.46E2,1E0,7.3333335E-1,1.1E1,6.726722E2,7.81E2,1.2748043E7,3.313E3,1.3997573E7,1.9347133E5,-2.403612E-4,2.6595745E0,-1.7627153E-3,-2.1603175E-3,2.085497E-3,-1.0389893E-2,-3.949217E-3,-2.5283676E-3,1.909331E-3,-8.789621E-3,3.3347E-3,1.1869626E-3,-4.318756E-3,2.959045E-3,8.890555E-3,-1.6766023E-2,-5.2248747E-3,-1.5585753E-3,4.291062E-3,4.4685023E-4,-4.2087696E-3,3.2641654E-4,-1.1341339E-2,1.0418143E-3,6.9812345E-3,-5.8052805E-3,5.9702655E-4,-4.128686E-3,1.0133922E-3,4.6149786E-3,-1.6013391E-3,1.490802E-3,-4.448857E-3,-8.646786E-3,-7.5930126E-5,9.916904E-3,5.849061E-3,4.935699E-3,1.4777492E-3],"split_indices":[2,6,56,64,66,64,50,65,59,6,45,70,59,0,0,4,41,40,7,8,70,0,0,70,9,12,47,2,4,1,5,7,3,2,0,15,65,8,4,0,63,0,57,45,0,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.003E3,8.78E2,1.25E2,7.95E2,8.3E1,1.19E2,6E0,4.65E2,3.3E2,7.3E1,1E1,6.5E1,5.4E1,4E0,2E0,1.7E2,2.95E2,3.18E2,1.2E1,5E1,2.3E1,2E0,8E0,5E1,1.5E1,2.4E1,3E1,1.63E2,7E0,2.75E2,2E1,3.02E2,1.6E1,7E0,5E0,3.9E1,1.1E1,1.3E1,1E1,6E0,4.4E1,9E0,6E0,2.1E1,3E0,2.8E1,2E0,4.4E1,1.19E2,4E0,3E0,2.23E2,5.2E1,1.7E1,3E0,2.86E2,1.6E1,7E0,9E0,3E0,4E0,3E0,2E0,1.4E1,2.5E1,3E0,8E0,8E0,5E0,3E0,7E0,4E0,2E0,3.8E1,6E0,7E0,2E0,4E0,2E0,1.3E1,8E0,1.9E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[-2.308615E-3,-1.849885E-2,1.9455727E-2,-3.356989E-2,1.4938177E-2,8.475418E-4,5.1898453E-2,-1.5481752E-2,-7.52085E-2,3.2759003E-2,-8.5519254E-2,2.565052E-2,-4.4054326E-2,1.5939476E-1,2.3779104E-2,-4.4607222E-2,8.547976E-3,-3.855021E-2,-1.4758484E-1,3.74536E-2,-1.0137773E-2,-1.0964059E-1,9.73706E-4,1.01193655E-2,1.0670625E-1,-8.396544E-2,3.5889767E-2,-1.0960922E-2,2.0568132E-1,1.2884754E-1,-1.714824E-2,-5.2393176E-2,9.728399E-3,1.5457016E-2,-1.2206504E-1,-5.945404E-2,7.858224E-2,-2.1053702E-1,-2.4431923E-2,5.7841834E-2,-1.3669679E-2,-5.87187E-2,-8.374612E-3,2.6638214E-2,-6.1431512E-2,1.4203997E-1,-2.4633994E-2,-1.6893272E-1,-2.5963442E-2,4.7896396E-2,-4.783505E-3,7.2673303E-3,-4.3049343E-3,2.2224146E-1,2.1311613E-4,2.3269968E-2,1.491874E-1,-1.468234E-1,1.5931446E-2,-1.2334675E-3,-7.096211E-3,-2.5025043E-3,1.0854427E-3,-6.5241015E-4,-7.14931E-3,1.0043701E-2,-3.2466527E-3,7.633148E-3,1.4042076E-4,-4.543855E-3,-1.1981072E-2,1.9250825E-3,-4.879812E-3,4.7393E-4,4.4382047E-3,2.629563E-3,-5.660105E-3,-5.006188E-3,-1.0418483E-3,8.5955433E-4,1.2019044E-2,7.648656E-3,-4.3239356E-3,2.8702724E-3,8.667995E-3,4.32064E-3,-4.1764528E-3,-4.2730565E-3,-2.6737213E-2,3.8368716E-3,-1.9976504E-3,-2.733419E-3,2.6199613E-3,1.0725145E-2,3.0120874E-3,-7.8064884E-4,3.795008E-3,7.240624E-3,9.449196E-4,-3.2029094E-4,-1.3845501E-2,1.2865778E-3,-9.4379755E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,-1,-1,-1,93,-1,95,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.62596E-1,2.9782063E-1,2.6498044E-1,3.0537382E-1,3.3220467E-1,3.1296602E-1,4.7984597E-1,1.9984628E-1,3.206532E-1,1.9508989E-1,7.440154E-2,2.2677913E-1,3.2207927E-1,2.6246542E-1,5.526315E-1,2.6863143E-1,1.4356524E-1,2.0789894E-1,3.127187E-1,1.6234922E-1,0E0,7.442582E-2,0E0,1.8247297E-1,1.3808045E-1,3.2580817E-1,6.2153563E-2,1.2684667E-1,8.552992E-2,7.383603E-2,4.012732E-1,3.244033E-1,0E0,8.704909E-2,2.6770413E-2,2.372213E-1,8.550289E-2,1.258012E-1,8.586575E-2,2.0359698E-1,3.5669938E-1,2.4702746E-2,0E0,2.2791684E-1,2.3189282E-1,7.36067E-2,6.363721E-2,7.925894E-1,8.232567E-2,3.482327E-2,0E0,0E0,0E0,2.7029276E-2,0E0,1.8133609E-2,2.9368818E-2,4.087319E-1,2.0634231E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,53,53,55,55,56,56,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,-1,-1,-1,94,-1,96,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.860918E5,1.585814E6,6.880842E-1,4.01057E5,4.2438595E1,5.680271E-2,1.1736916E0,1.3743396E-2,3.015873E0,1E0,1.6876252E0,7.156528E7,3.7878788E0,1.48E1,4.0663E4,1.237036E6,2.5652106E7,5.338167E5,2.008E3,1.6E1,-1.0137773E-2,8.504982E-1,9.73706E-4,4.066351E0,1.6108533E8,5.1942E4,6.459696E6,1.1090909E1,2.712766E0,3.6981132E0,1E0,7E0,9.728399E-3,4.9E2,2.2734E4,3.7289145E4,1E0,3.95E2,4.4522205E6,1.157602E6,1.4081731E1,4.22201E6,-8.374612E-3,1.9E1,5.69E2,2.906237E2,3.2E1,8.237624E0,4.1388846E2,1.4127266E6,-4.783505E-3,7.2673303E-3,-4.3049343E-3,2E1,2.1311613E-4,3.1993368E6,3.3571344E7,1.886819E0,3.206931E2,-1.2334675E-3,-7.096211E-3,-2.5025043E-3,1.0854427E-3,-6.5241015E-4,-7.14931E-3,1.0043701E-2,-3.2466527E-3,7.633148E-3,1.4042076E-4,-4.543855E-3,-1.1981072E-2,1.9250825E-3,-4.879812E-3,4.7393E-4,4.4382047E-3,2.629563E-3,-5.660105E-3,-5.006188E-3,-1.0418483E-3,8.5955433E-4,1.2019044E-2,7.648656E-3,-4.3239356E-3,2.8702724E-3,8.667995E-3,4.32064E-3,-4.1764528E-3,-4.2730565E-3,-2.6737213E-2,3.8368716E-3,-1.9976504E-3,-2.733419E-3,2.6199613E-3,1.0725145E-2,3.0120874E-3,-7.8064884E-4,3.795008E-3,7.240624E-3,9.449196E-4,-3.2029094E-4,-1.3845501E-2,1.2865778E-3,-9.4379755E-3],"split_indices":[40,9,54,9,70,50,50,50,65,76,50,7,66,70,9,1,55,40,2,3,0,50,0,66,7,1,9,70,65,68,8,8,0,1,9,57,24,2,59,12,70,5,0,70,2,4,0,68,4,40,0,0,0,3,0,40,63,50,70,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.027E3,5.89E2,4.38E2,4.06E2,1.83E2,2.79E2,1.59E2,2.84E2,1.22E2,1.56E2,2.7E1,1.8E2,9.9E1,3.2E1,1.27E2,1.28E2,1.56E2,8.2E1,4E1,1.54E2,2E0,2.2E1,5E0,1.52E2,2.8E1,6.6E1,3.3E1,7E0,2.5E1,3.5E1,9.2E1,1.25E2,3E0,1.49E2,7E0,7E1,1.2E1,2.6E1,1.4E1,1.1E2,4.4E1,1.4E1,8E0,1.24E2,2.8E1,2.2E1,6E0,2.6E1,4E1,3.1E1,2E0,2E0,5E0,2.3E1,2E0,6E0,2.9E1,1.8E1,7.4E1,1.01E2,2.4E1,1.5E1,1.34E2,2E0,5E0,2E0,6.8E1,5E0,7E0,9E0,1.7E1,8E0,6E0,5E1,6E1,2.7E1,1.7E1,5E0,9E0,1.21E2,3E0,3E0,2.5E1,9E0,1.3E1,2E0,4E0,2.3E1,3E0,5E0,3.5E1,2E0,2.9E1,2.1E1,2E0,4E0,2E0,2.7E1,2E0,1E1,8E0,7.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-6.2097446E-3,-3.315852E-2,1.9783206E-2,-8.150755E-2,-2.44903E-2,2.3381745E-2,-2.3751111E-1,-5.6483172E-2,-2.5318277E-1,-4.729571E-3,-5.192103E-2,9.097217E-4,5.5432964E-2,-1.4761156E-2,-4.3300367E-3,-9.749241E-2,8.780393E-3,3.3085446E-3,-3.563916E-1,4.4021357E-4,-1.5470836E-1,-6.656178E-2,9.4986446E-2,6.025645E-3,-2.2687096E-2,6.263029E-2,-1.5606737E-1,-6.522624E-2,-1.2376853E-2,1.2723601E-1,-8.656946E-2,-1.9979743E-2,-8.667089E-3,1.1800104E-2,-7.128285E-2,-1.7372444E-3,-8.606037E-3,-8.2635224E-2,1.0442106E-2,4.677878E-2,1.3241763E-2,2.6320936E-2,-4.9473364E-2,1.9002506E-1,5.1149704E-2,-5.0879683E-4,-1.2035931E-2,-5.5109356E-3,-8.341505E-4,8.930686E-3,-2.631459E-4,-7.0660905E-4,-1.0374268E-2,-3.9652488E-4,3.187965E-3,-1.0160414E-2,-5.55354E-4,5.881537E-3,-4.070468E-3,2.133784E-3,-3.2495845E-3,8.352807E-3,-3.0277486E-4,3.8688716E-3,3.4770017E-4,-6.3361106E-3,-2.3598866E-4,9.668108E-3,1.432269E-3,2.6595949E-3,-2.7429515E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,31,33,35,37,39,41,-1,43,45,47,-1,49,51,-1,-1,53,55,-1,-1,57,59,61,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.89226E-1,2.0042914E-1,4.6942118E-1,2.993122E-1,2.2253749E-1,3.564762E-1,5.3790033E-2,1.748226E-1,3.2775706E-1,1.8615402E-1,3.7626633E-1,7.4219656E-1,3.2031065E-1,0E0,0E0,2.0437723E-1,3.0467764E-1,0E0,1.0183096E-2,1.9068165E-1,2.3451626E-2,1.9518453E-1,1.3561605E-1,3.2794905E-1,0E0,2.7986836E-1,1.0465276E-1,8.801897E-2,0E0,1.08334884E-1,1.4526123E-1,0E0,0E0,2.3836978E-1,2.8130445E-1,0E0,0E0,1.7527384E-1,8.515771E-2,1.0212501E-1,0E0,2.3065367E-1,3.052476E-1,4.5446217E-2,1.4048353E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,33,33,34,34,37,37,38,38,39,39,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,32,34,36,38,40,42,-1,44,46,48,-1,50,52,-1,-1,54,56,-1,-1,58,60,62,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.071E3,5.8182236E1,9.185E1,6E0,1E0,1.7831801E6,5.4E1,2.35E2,4.327E3,2E0,1.5209424E1,1.7587205E6,1.8545505E2,-1.4761156E-2,-4.3300367E-3,1.3E2,1.9741463E2,3.3085446E-3,1.3E1,8.172E3,1.0675E4,8.328548E6,3E1,5.4E2,-2.2687096E-2,1.1533101E0,1.2816234E7,1.1E1,-1.2376853E-2,3.000176E5,9.9E1,-1.9979743E-2,-8.667089E-3,3.056087E2,1.2706E4,-1.7372444E-3,-8.606037E-3,5E0,8.747289E5,4.9948E4,1.3241763E-2,6.904904E3,4.5418E4,2.8146256E5,5.2409735E-7,-5.0879683E-4,-1.2035931E-2,-5.5109356E-3,-8.341505E-4,8.930686E-3,-2.631459E-4,-7.0660905E-4,-1.0374268E-2,-3.9652488E-4,3.187965E-3,-1.0160414E-2,-5.55354E-4,5.881537E-3,-4.070468E-3,2.133784E-3,-3.2495845E-3,8.352807E-3,-3.0277486E-4,3.8688716E-3,3.4770017E-4,-6.3361106E-3,-2.3598866E-4,9.668108E-3,1.432269E-3,2.6595949E-3,-2.7429515E-3],"split_indices":[2,59,66,8,24,40,0,2,1,6,68,40,70,0,0,2,4,0,8,41,1,9,3,0,0,66,9,3,0,40,0,0,0,4,41,0,0,3,40,9,0,45,10,45,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.82E2,4.82E2,5E2,7.2E1,4.1E2,4.94E2,6E0,6.4E1,8E0,2.39E2,1.71E2,2.91E2,2.03E2,3E0,3E0,3.9E1,2.5E1,2E0,6E0,2.32E2,7E0,1.56E2,1.5E1,2.89E2,2E0,1.97E2,6E0,3.4E1,5E0,1.1E1,1.4E1,3E0,3E0,2.01E2,3.1E1,2E0,5E0,1.29E2,2.7E1,1.3E1,2E0,2.12E2,7.7E1,1.5E1,1.82E2,3E0,3E0,1.5E1,1.9E1,7E0,4E0,1E1,4E0,1.49E2,5.2E1,8E0,2.3E1,3E0,1.26E2,1.9E1,8E0,3E0,1E1,5.1E1,1.61E2,2.5E1,5.2E1,1.3E1,2E0,1.72E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-3.2872448E-3,-1.48153305E-2,2.9580442E-2,-2.223885E-2,5.3419605E-2,1.5978802E-2,9.5988974E-2,-2.4851765E-2,1.22889206E-1,8.758388E-2,-3.5348292E-2,5.415567E-2,-1.4192984E-2,1.23524964E-1,-9.8993145E-2,-2.254485E-2,-1.8870106E-1,1.5946716E-1,4.531832E-4,1.1512229E-1,-2.8028445E-2,-7.8185335E-2,3.0455021E-2,-1.9637179E-2,1.0721551E-1,-2.1272019E-2,1.262137E-1,1.621218E-1,2.3171926E-2,-7.093404E-3,-1.5003094E-3,6.1695788E-2,-2.6720518E-2,-2.7462506E-1,-5.805655E-4,8.302053E-3,2.6022631E-3,9.342527E-2,1.3931585E-2,-1.1090268E-1,4.299247E-3,-9.91169E-2,1.1310882E-3,-2.8777715E-2,3.8397282E-3,-1.1205166E-1,1.7983187E-2,4.3673214E-2,1.337936E-1,-3.0678771E-2,9.501775E-2,7.202131E-3,1.9915397E-3,6.911865E-2,9.643342E-3,-1.0456462E-2,4.273601E-3,4.5157317E-3,-1.7793953E-3,-2.9978158E-3,-7.6514145E-4,-3.0595753E-3,-1.662625E-2,7.515207E-3,2.4605107E-3,-3.6402734E-4,-6.9130883E-3,-1.4235167E-3,-6.922903E-3,-3.427061E-3,1.2240526E-3,1.9428263E-3,-1.0488205E-2,-1.4993879E-3,2.4595505E-3,4.976926E-6,4.006427E-3,7.2651277E-3,2.634532E-3,-5.008851E-4,-4.7833673E-3,-1.648048E-3,6.317565E-3,6.8172775E-3,-9.969457E-4,1.7357304E-3,-1.3613893E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,-1,-1,57,59,61,-1,-1,-1,63,-1,65,-1,67,-1,69,-1,71,73,75,77,79,81,-1,-1,83,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7933975E-1,3.7653664E-1,2.333151E-1,2.5674367E-1,2.2448234E-1,2.5072315E-1,2.4846527E-1,2.4446088E-1,4.8036322E-2,1.71967E-1,6.1576344E-2,3.7798735E-1,1.23945974E-1,1.4730245E-1,1.7019711E-2,2.3029318E-1,1.3267979E-1,9.45133E-3,0E0,1.4723861E-1,1.212602E-1,3.0416697E-2,3.0735295E-2,1.4455837E-1,8.873141E-2,1.3090236E-1,8.407302E-3,1.1003816E-1,2.9168647E-2,0E0,0E0,1.1782123E-1,2.3938814E-1,8.491257E-2,0E0,0E0,0E0,1.0313904E-1,0E0,2.7356647E-2,0E0,3.4000367E-2,0E0,1.4690813E-2,0E0,2.2505563E-1,5.5764653E-2,3.4143254E-2,6.360227E-2,1.572007E-1,5.5033512E-2,0E0,0E0,8.245091E-2,0E0,9.449611E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,37,37,39,39,41,41,43,43,45,45,46,46,47,47,48,48,49,49,50,50,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,-1,-1,58,60,62,-1,-1,-1,64,-1,66,-1,68,-1,70,-1,72,74,76,78,80,82,-1,-1,84,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.201807E6,1.3795834E3,1.7068776E10,3.1909645E0,2.856934E7,2.56E2,3.3765998E0,3.8304348E1,1.4839433E5,1.2792593E2,1.8E1,4.9E1,1E0,4.7E1,1.625E3,5.445545E-1,4.5E0,2.5503825E6,4.531832E-4,4.907764E3,1.94E3,1.32E3,2.8582963E-1,1.1E1,2.2007043E0,3.0561172E5,2.0766992E6,1.3540794E8,5.98E2,-7.093404E-3,-1.5003094E-3,3.4698796E0,1.2051282E0,8.612209E-6,-5.805655E-4,8.302053E-3,2.6022631E-3,1.684E0,1.3931585E-2,3.8434E4,4.299247E-3,3.917053E5,1.1310882E-3,1.20086E5,3.8397282E-3,6.055E3,4E0,2.583E3,3.8343322E6,3.222E3,9.5E2,7.202131E-3,1.9915397E-3,2.641726E-1,9.643342E-3,2.427869E8,4.273601E-3,4.5157317E-3,-1.7793953E-3,-2.9978158E-3,-7.6514145E-4,-3.0595753E-3,-1.662625E-2,7.515207E-3,2.4605107E-3,-3.6402734E-4,-6.9130883E-3,-1.4235167E-3,-6.922903E-3,-3.427061E-3,1.2240526E-3,1.9428263E-3,-1.0488205E-2,-1.4993879E-3,2.4595505E-3,4.976926E-6,4.006427E-3,7.2651277E-3,2.634532E-3,-5.008851E-4,-4.7833673E-3,-1.648048E-3,6.317565E-3,6.8172775E-3,-9.969457E-4,1.7357304E-3,-1.3613893E-3],"split_indices":[9,64,43,54,57,10,50,65,45,70,3,0,16,8,0,65,70,44,0,4,0,8,46,3,66,45,40,12,8,0,0,68,65,50,0,0,0,66,0,9,0,40,0,9,0,12,8,41,57,2,10,0,0,50,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.99E2,7.4E2,2.59E2,6.68E2,7.2E1,2.16E2,4.3E1,6.57E2,1.1E1,5.2E1,2E1,9.5E1,1.21E2,3.8E1,5E0,6.49E2,8E0,8E0,3E0,4.2E1,1E1,1.2E1,8E0,4E1,5.5E1,1.16E2,5E0,2.7E1,1.1E1,2E0,3E0,3E1,6.19E2,5E0,3E0,6E0,2E0,3.9E1,3E0,6E0,4E0,1E1,2E0,4E0,4E0,1.1E1,2.9E1,1.7E1,3.8E1,1.08E2,8E0,3E0,2E0,1E1,1.7E1,8E0,3E0,2.2E1,8E0,1.27E2,4.92E2,2E0,3E0,1.3E1,2.6E1,2E0,4E0,5E0,5E0,2E0,2E0,5E0,6E0,1.2E1,1.7E1,9E0,8E0,2.8E1,1E1,8.6E1,2.2E1,2E0,6E0,5E0,5E0,2E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-2.8632345E-4,-1.774027E-2,2.238076E-2,-1.5221341E-2,-1.3841477E-2,3.321602E-2,-5.1387485E-2,1.1558125E-2,-2.9727966E-2,1.3240299E-1,2.6274333E-2,-9.410699E-3,-3.268798E-2,-6.260398E-2,2.867155E-2,9.714019E-2,-3.611295E-2,1.6215679E-1,-2.9554064E-3,-5.7362236E-2,3.3730257E-2,-1.6988851E-2,-6.899676E-3,1.4055534E-2,-9.048436E-2,1.2997037E-1,1.1225854E-2,2.2503527E-2,9.701275E-3,-2.8901523E-2,-1.7694455E-1,1.9391862E-1,-1.3435455E-3,-8.7648675E-2,2.3044871E-2,1.2319817E-1,2.6694901E-2,-6.0970753E-2,6.4212784E-2,-1.6790428E-3,3.770963E-3,-5.2220854E-3,-1.3254134E-3,-1.197997E-3,7.0507503E-3,-4.1483503E-4,4.4595767E-3,1.8311325E-3,-1.0191826E-3,-1.0645166E-3,-7.799753E-3,2.3571026E-4,-1.2710428E-2,5.933394E-3,1.3156933E-2,7.725167E-4,-5.1350584E-3,2.1950868E-3,-9.1103563E-4,-1.0389887E-3,6.664968E-3,1.6926748E-3,-2.4878688E-3,-3.374411E-3,8.588376E-4,6.3952454E-4,7.0775636E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,-1,49,51,53,-1,55,57,59,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0196157E-1,4.0547463E-1,3.5499662E-1,2.2177696E-1,0E0,2.6229173E-1,1.5691169E-1,2.5689763E-1,3.0322692E-1,1.5678287E-1,2.276737E-1,0E0,9.339293E-2,8.248906E-2,2.8836647E-1,1.4849736E-1,3.533916E-1,1.3942045E-1,0E0,7.507719E-2,2.063142E-1,1.7177102E-1,0E0,4.088266E-2,3.676738E-2,9.214938E-2,2.4613582E-1,1.0196608E-2,0E0,2.6794508E-1,3.1027156E-1,7.6811194E-2,0E0,5.738254E-2,1.0497589E-2,7.995394E-2,2.5660172E-1,3.2233894E-2,7.469176E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,-1,50,52,54,-1,56,58,60,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.518317E2,5.9E1,3.2856784E7,5.004138E-3,-1.3841477E-2,1.0972E4,1.3586957E1,5.67E2,1.16E0,1.669031E2,2.2734E4,-9.410699E-3,3.0392378E7,1.863E3,2.63E2,5.1766735E-1,2.954124E6,1.4791E4,-2.9554064E-3,2.77843E5,3.3382E4,1.465E4,-6.899676E-3,1.3E1,1.870878E6,5.714286E-1,4.5981E4,8.93004E5,9.701275E-3,3.0977E4,2.5E0,2.5176924E1,-1.3435455E-3,3.8795E4,3.812E3,1E0,2.9366477E0,5.97E3,1.11436836E5,-1.6790428E-3,3.770963E-3,-5.2220854E-3,-1.3254134E-3,-1.197997E-3,7.0507503E-3,-4.1483503E-4,4.4595767E-3,1.8311325E-3,-1.0191826E-3,-1.0645166E-3,-7.799753E-3,2.3571026E-4,-1.2710428E-2,5.933394E-3,1.3156933E-2,7.725167E-4,-5.1350584E-3,2.1950868E-3,-9.1103563E-4,-1.0389887E-3,6.664968E-3,1.6926748E-3,-2.4878688E-3,-3.374411E-3,8.588376E-4,6.3952454E-4,7.0775636E-3],"split_indices":[67,3,57,50,0,9,70,1,66,66,9,0,55,9,41,50,59,2,0,1,9,41,0,5,43,65,12,9,0,41,66,68,0,1,2,27,50,2,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.014E3,5.73E2,4.41E2,5.69E2,4E0,3.85E2,5.6E1,2E2,3.69E2,2.4E1,3.61E2,5E0,5.1E1,3.7E1,1.63E2,1.7E1,3.52E2,2.1E1,3E0,2.9E1,3.32E2,4.6E1,5E0,1E1,2.7E1,2.3E1,1.4E2,1.1E1,6E0,3.36E2,1.6E1,1.8E1,3E0,2.1E1,8E0,2.3E1,3.09E2,3E1,1.6E1,6E0,4E0,1.9E1,8E0,3E0,2E1,1.14E2,2.6E1,8E0,3E0,3.24E2,1.2E1,6E0,1E1,1.2E1,6E0,4E0,1.7E1,5E0,3E0,3E0,2E1,2.75E2,3.4E1,2.6E1,4E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-9.1852073E-4,-2.2387886E-2,2.2948084E-2,1.1474349E-3,-4.2071506E-2,3.340278E-2,-5.4415755E-2,1.3962858E-2,-6.723091E-2,-9.680554E-3,-6.61143E-2,7.0049785E-2,1.2048599E-2,-1.837068E-1,-3.306783E-2,-3.5578888E-2,3.289967E-2,8.9614384E-2,-1.24405645E-1,1.2475434E-2,-2.0322688E-1,-7.561503E-2,1.10133715E-1,5.3777214E-2,1.5381788E-1,-5.724029E-2,2.9956605E-2,-1.7292722E-3,-1.0385947E-2,-1.4210627E-2,-1.3264897E-1,-1.0387358E-2,-9.0982996E-2,4.4371422E-2,-3.63834E-2,4.1697178E-2,5.6310175E-3,-1.9545387E-2,-8.633944E-2,-2.6432782E-2,8.511645E-2,-4.4039216E-2,-2.0628743E-2,-6.670276E-2,-2.026288E-1,2.2188729E-2,1.2797789E-2,7.3225475E-3,8.543835E-2,2.0803323E-1,4.031429E-3,-5.5176844E-3,-2.278983E-1,1.12373745E-2,9.5239796E-2,2.607904E-2,-4.3106515E-2,-1.1954479E-2,-2.1464566E-3,-5.9033576E-3,8.1368996E-4,-7.7058678E-3,-1.2340792E-3,7.435479E-3,1.6231716E-3,1.1739059E-3,-4.1398476E-3,-4.0998566E-4,3.1803132E-3,-7.2251232E-3,2.2978205E-3,-2.4814962E-4,-5.4051518E-3,-3.680361E-3,4.6243966E-3,2.1528983E-3,-4.469449E-3,-4.2224526E-3,-1.2360382E-3,-1.2575778E-2,-1.2600254E-3,-2.2034647E-3,3.9869994E-3,7.4119093E-3,-1.0257171E-3,2.260707E-3,6.0596582E-3,5.0560483E-3,1.2689487E-2,1.891568E-3,-3.4568617E-3,-2.223648E-2,-4.713446E-3,1.914467E-3,-1.4888193E-3,-4.4500143E-3,5.0617023E-3,-1.6446186E-4,5.0452184E-3,-2.8274897E-3,7.2444614E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,57,59,61,63,65,67,-1,-1,69,71,73,75,-1,77,79,81,-1,83,85,87,-1,89,91,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.236751E-1,2.4928236E-1,3.9328423E-1,2.1650791E-1,2.2753924E-1,3.33312E-1,1.5401073E-1,1.9630839E-1,3.608845E-1,5.4284984E-1,2.8964138E-1,2.057383E-1,3.3797848E-1,4.1042775E-2,9.388389E-2,8.000635E-2,1.21628925E-1,1.2272276E-2,2.9568022E-1,3.244518E-1,4.890806E-1,1.6760105E-1,1.271812E-1,1.9499174E-1,7.126057E-2,4.9304765E-1,2.6289275E-1,0E0,0E0,5.234923E-2,7.555428E-2,1.3839613E-1,8.410859E-2,1.3226274E-1,7.55454E-2,8.983959E-3,0E0,0E0,2.6914597E-1,1.4328268E-1,1.1093938E-1,4.8144437E-2,0E0,1.4910305E-1,1.1716756E-1,3.5898272E-2,0E0,2.5484788E-1,1.2652874E-1,5.6482255E-2,0E0,1.4640835E-1,3.676142E-1,2.2593457E-1,1.486477E-1,4.7596548E-2,2.921854E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,39,39,40,40,41,41,43,43,44,44,45,45,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,58,60,62,64,66,68,-1,-1,70,72,74,76,-1,78,80,82,-1,84,86,88,-1,90,92,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.255E3,1.6923077E0,2.856934E7,1.0292E4,8.6875E0,3.39498E5,2.012328E6,3.89E2,3.5714287E-1,2.9E1,3.4E1,1.6975454E3,5.9754E4,5.04E2,2.9622424E3,3.09627E5,1.3558896E7,4.48E2,1.2706E4,5.0857143E0,5.98E2,4.586207E0,1.4047974E8,1E0,8.255237E1,3.8496505E6,1.1046409E7,-1.7292722E-3,-1.0385947E-2,5.1987316E7,4.4910244E3,6E0,1.2562814E0,6.6671006E5,9.7E2,5.04E2,5.6310175E-3,-1.9545387E-2,7.271717E6,6.233195E-2,1.6875E1,8E0,-2.0628743E-2,6.953366E7,2.062E1,4.8E1,1.2797789E-2,2.2E0,2.142857E0,6E0,4.031429E-3,4.888889E0,1.6108533E8,4.06478E1,4.30453E2,1.0828667E5,1.2376862E10,-1.1954479E-2,-2.1464566E-3,-5.9033576E-3,8.1368996E-4,-7.7058678E-3,-1.2340792E-3,7.435479E-3,1.6231716E-3,1.1739059E-3,-4.1398476E-3,-4.0998566E-4,3.1803132E-3,-7.2251232E-3,2.2978205E-3,-2.4814962E-4,-5.4051518E-3,-3.680361E-3,4.6243966E-3,2.1528983E-3,-4.469449E-3,-4.2224526E-3,-1.2360382E-3,-1.2575778E-2,-1.2600254E-3,-2.2034647E-3,3.9869994E-3,7.4119093E-3,-1.0257171E-3,2.260707E-3,6.0596582E-3,5.0560483E-3,1.2689487E-2,1.891568E-3,-3.4568617E-3,-2.223648E-2,-4.713446E-3,1.914467E-3,-1.4888193E-3,-4.4500143E-3,5.0617023E-3,-1.6446186E-4,5.0452184E-3,-2.8274897E-3,7.2444614E-4],"split_indices":[2,65,57,41,70,9,12,2,65,3,3,64,1,7,64,9,9,2,41,66,41,65,5,27,70,59,57,0,0,57,4,64,65,57,2,7,0,0,57,50,64,0,0,7,70,3,0,70,65,8,0,68,7,70,45,45,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,5.37E2,4.83E2,2.45E2,2.92E2,4.26E2,5.7E1,2.07E2,3.8E1,1.25E2,1.67E2,1.56E2,2.7E2,7E0,5E1,5.7E1,1.5E2,1E1,2.8E1,1.13E2,1.2E1,1.59E2,8E0,1.32E2,2.4E1,5.5E1,2.15E2,2E0,5E0,4.3E1,7E0,4E1,1.7E1,1.29E2,2.1E1,5E0,5E0,2E0,2.6E1,7.4E1,3.9E1,8E0,4E0,1.5E2,9E0,6E0,2E0,5.4E1,7.8E1,1.2E1,1.2E1,4.3E1,1.2E1,1.68E2,4.7E1,1.8E1,2.5E1,2E0,5E0,7E0,3.3E1,7E0,1E1,8E0,1.21E2,1E1,1.1E1,2E0,3E0,1.7E1,9E0,6.1E1,1.3E1,3E0,3.6E1,3E0,5E0,9.1E1,5.9E1,6E0,3E0,3E0,3E0,8E0,4.6E1,4.5E1,3.3E1,6E0,6E0,2.6E1,1.7E1,3E0,9E0,9.9E1,6.9E1,3E0,4.4E1,1.4E1,4E0,1.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[5.442678E-3,-1.9686988E-2,3.362935E-2,-1.599868E-2,-1.4598995E-1,4.0203944E-2,-4.528623E-2,-3.4474276E-2,6.475974E-3,-2.0123072E-2,-1.1648611E-2,-1.3675251E-2,4.7796547E-2,-9.3123764E-2,5.0192576E-2,-3.9909493E-2,1.870828E-1,5.5690303E-2,-1.348266E-2,-3.3792253E-3,4.293694E-3,5.6709666E-2,-7.4219674E-2,6.171206E-2,8.056769E-3,-1.9893765E-1,-4.9238443E-2,2.4528254E-2,4.6760724E-3,-5.360455E-3,-7.4173465E-2,-1.382081E-3,1.28594525E-2,-2.5445128E-2,9.002684E-2,-3.6477804E-4,-1.1053675E-1,1.5127544E-1,2.0776158E-2,-4.666542E-2,-1.1099632E-2,5.1059794E-2,1.6160926E-1,1.4677512E-2,-7.1729296E-3,-2.6453042E-3,-1.3345347E-2,-1.662218E-2,-7.6886597E-3,3.642841E-3,9.189844E-4,-2.7078646E-3,2.805452E-3,-2.8982433E-3,-9.108606E-3,1.8815862E-3,-1.997138E-3,7.01494E-3,-2.474901E-4,6.5852357E-3,-4.8820587E-4,2.4368458E-3,-6.5464624E-3,9.714047E-3,2.2656572E-5,-2.834602E-3,2.4062155E-3,-1.0034422E-3,-1.0928715E-2,1.3779054E-3,4.115991E-3,2.566963E-4,9.0399925E-3,-2.1301198E-3,1.5625419E-3,-2.3652946E-3,4.633277E-3,1.8209642E-3,-2.1917843E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,-1,51,53,-1,-1,55,57,59,61,63,65,67,-1,69,71,73,-1,-1,-1,75,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1394193E-1,2.4595928E-1,2.4826878E-1,2.1572898E-1,1.9808003E-1,1.8024117E-1,1.7367053E-1,3.5066643E-1,2.3157981E-1,5.5452246E-2,0E0,2.3858674E-1,2.1240741E-1,1.0629654E-1,1.472275E-2,3.2987905E-1,1.4963223E-1,1.9173262E-1,2.1384788E-1,0E0,0E0,8.5207164E-2,1.2713046E-1,2.9502583E-1,1.12106845E-1,7.390806E-2,7.043953E-2,1.2860737E-2,0E0,5.021056E-1,1.7994535E-1,0E0,0E0,2.6933894E-2,2.8798017E-1,2.2134814E-1,1.12226024E-1,6.345029E-2,5.5384398E-2,1.229738E-1,0E0,2.0759994E-1,1.4720392E-1,1.1730199E-1,0E0,0E0,0E0,7.0755124E-2,0E0,0E0,1.696016E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,47,47,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,-1,52,54,-1,-1,56,58,60,62,64,66,68,-1,70,72,74,-1,-1,-1,76,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.785302E5,2.5013582E-5,2E0,1.5454154E-1,5E1,1.5933E4,1.4433751E0,1.9616238E2,2.01184E5,1.048E3,-1.1648611E-2,1.480315E0,1.5153E4,1.3881669E6,1.7049885E0,1.1074469E6,1.4595416E3,1.08367164E5,4.9469455E6,-3.3792253E-3,4.293694E-3,6.201215E6,2.4E1,2.2125713E1,7.06191E7,8.025477E0,1.1736916E0,9.5596474E-1,4.6760724E-3,2.25E2,1.2674157E1,-1.382081E-3,1.28594525E-2,1E0,2.276E4,6E0,1E0,6.642857E0,1.2972876E7,6.769866E7,-1.1099632E-2,4.139313E-1,1.1806228E6,2.8172503E10,-7.1729296E-3,-2.6453042E-3,-1.3345347E-2,1.2218E4,-7.6886597E-3,3.642841E-3,1.4E1,-2.7078646E-3,2.805452E-3,-2.8982433E-3,-9.108606E-3,1.8815862E-3,-1.997138E-3,7.01494E-3,-2.474901E-4,6.5852357E-3,-4.8820587E-4,2.4368458E-3,-6.5464624E-3,9.714047E-3,2.2656572E-5,-2.834602E-3,2.4062155E-3,-1.0034422E-3,-1.0928715E-2,1.3779054E-3,4.115991E-3,2.566963E-4,9.0399925E-3,-2.1301198E-3,1.5625419E-3,-2.3652946E-3,4.633277E-3,1.8209642E-3,-2.1917843E-3],"split_indices":[40,49,6,50,0,1,50,68,9,6,0,65,10,40,54,57,4,40,59,0,0,5,8,68,56,68,50,65,0,2,66,0,0,19,10,3,8,70,5,7,0,54,40,43,0,0,0,41,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.006E3,5.32E2,4.74E2,5.18E2,1.4E1,4.38E2,3.6E1,2.84E2,2.34E2,7E0,7E0,5.4E1,3.84E2,2.4E1,1.2E1,2.78E2,6E0,6.7E1,1.67E2,5E0,2E0,2.5E1,2.9E1,2.84E2,1E2,6E0,1.8E1,9E0,3E0,1.39E2,1.39E2,2E0,4E0,2E1,4.7E1,1.48E2,1.9E1,6E0,1.9E1,2.6E1,3E0,2.58E2,2.6E1,9.7E1,3E0,3E0,3E0,1.5E1,3E0,2E0,7E0,7.7E1,6.2E1,1.29E2,1E1,4E0,1.6E1,2.8E1,1.9E1,9E0,1.39E2,3E0,1.6E1,4E0,2E0,5E0,1.4E1,2.4E1,2E0,1.68E2,9E1,5E0,2.1E1,2.3E1,7.4E1,1.2E1,3E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[3.3294875E-4,-1.5049665E-2,2.2931276E-2,-1.3537452E-2,-1.4190427E-2,6.1113127E-2,7.05489E-4,-7.582083E-3,-7.949946E-2,-4.251354E-2,7.303039E-2,-4.4803403E-2,2.8575275E-2,-8.532484E-4,-9.251229E-2,-4.6880823E-2,-2.5906544E-2,-2.7198508E-1,5.6642402E-2,-7.098754E-2,8.273778E-2,-3.1435072E-2,-1.319004E-2,1.0288232E-2,2.3414535E-2,7.754899E-2,-9.429307E-3,-1.3728424E-2,-5.4490253E-2,-1.3157338E-1,-1.0628571E-2,-4.5413007E-3,-1.627986E-2,-1.6452108E-2,6.0849627E-3,3.1859954E-3,-5.557585E-3,7.0838794E-2,1.8987489E-1,9.411715E-2,-4.1064963E-2,-1.8333277E-2,4.9426924E-2,4.319514E-3,-6.2299785E-3,-1.0183378E-3,2.816959E-3,-6.141473E-3,-3.373745E-4,-1.2514423E-2,-1.0578476E-3,-4.8927558E-3,4.3993746E-4,-2.4316176E-3,2.2894645E-3,3.8410716E-3,-2.6484486E-3,-2.0354872E-3,1.1076143E-2,5.980371E-3,1.1958968E-4,-2.7314173E-3,9.406083E-4,-2.1846155E-3,3.8210654E-3,1.4896847E-3,7.6751555E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,-1,-1,41,43,45,-1,47,49,51,-1,-1,53,-1,-1,-1,55,57,59,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4761882E-1,2.60257E-1,3.4364867E-1,2.3234428E-1,0E0,1.8711174E-1,3.272393E-1,3.113784E-1,7.4798167E-1,3.7946135E-1,1.9321048E-1,3.08493E-1,1.5611336E-1,3.4095657E-1,2.9758763E-1,1.4290547E-1,0E0,3.632468E-2,6.869836E-2,7.156314E-2,1.4717722E-1,1.1751602E-1,0E0,0E0,1.7140195E-1,1.8856335E-1,4.1337752E-1,0E0,1.2973544E-1,2.0781256E-1,6.6902205E-2,0E0,0E0,1.9572146E-2,0E0,0E0,0E0,1.9378418E-1,1.54769E-1,2.2776645E-2,1.0088067E-1,1.8497121E-1,1.8855709E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,24,24,25,25,26,26,28,28,29,29,30,30,33,33,37,37,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,-1,-1,42,44,46,-1,48,50,52,-1,-1,54,-1,-1,-1,56,58,60,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.716184E2,7.9E1,1.2183674E1,1.3508157E7,-1.4190427E-2,1.342376E0,6.648E3,1E0,1.6074808E5,2.8483894E8,9.99E2,5.524E3,1.2131076E2,7E0,6E0,1.3071136E3,-2.5906544E-2,1.56132E5,3.201807E6,3.4828237E8,6.2850784E1,9.63381E5,-1.319004E-2,1.0288232E-2,1E0,2.3670635E1,3.0291306E2,-1.3728424E-2,1.27E2,4.251E3,7.046396E8,-4.5413007E-3,-1.627986E-2,7.53052E5,6.0849627E-3,3.1859954E-3,-5.557585E-3,1.3922E4,2.62E3,1.4839433E5,3.887E3,3.8537518E2,1.3439851E7,4.319514E-3,-6.2299785E-3,-1.0183378E-3,2.816959E-3,-6.141473E-3,-3.373745E-4,-1.2514423E-2,-1.0578476E-3,-4.8927558E-3,4.3993746E-4,-2.4316176E-3,2.2894645E-3,3.8410716E-3,-2.6484486E-3,-2.0354872E-3,1.1076143E-2,5.980371E-3,1.1958968E-4,-2.7314173E-3,9.406083E-4,-2.1846155E-3,3.8210654E-3,1.4896847E-3,7.6751555E-3],"split_indices":[67,3,68,57,0,66,41,82,45,5,2,41,45,3,3,45,0,1,9,43,70,57,0,0,27,70,64,0,0,41,43,0,0,1,0,0,0,41,2,45,41,68,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,5.94E2,4.04E2,5.92E2,2E0,1.48E2,2.56E2,5.44E2,4.8E1,1.5E1,1.33E2,9.7E1,1.59E2,5.05E2,3.9E1,4.6E1,2E0,4E0,1.1E1,8E0,1.25E2,9.3E1,4E0,3E0,1.56E2,4.9E1,4.56E2,5E0,3.4E1,1.3E1,3.3E1,2E0,2E0,6E0,5E0,2E0,6E0,1.14E2,1.1E1,6E0,8.7E1,6E1,9.6E1,4.6E1,3E0,3.87E2,6.9E1,1.2E1,2.2E1,5E0,8E0,5E0,2.8E1,4E0,2E0,1.04E2,1E1,2E0,9E0,4E0,2E0,6.7E1,2E1,4.7E1,1.3E1,8.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-4.734579E-3,-1.6663298E-2,2.7648129E-2,-1.3938321E-2,-2.4159054E-1,-2.119703E-2,4.739089E-2,-1.1444154E-2,-1.9711275E-1,-4.6680802E-3,-1.796529E-2,-8.749283E-2,3.3783898E-2,5.5881143E-2,-1.6229902E-1,-2.0672789E-2,2.6567644E-2,-2.380819E-4,-2.786277E-1,-1.319999E-2,-1.1841673E-1,1.4842078E-1,-1.2399842E-2,-3.848298E-2,7.359353E-2,-9.661045E-3,-5.5696454E-4,-1.4695312E-1,-1.6711272E-2,5.8789175E-2,-3.2189887E-2,-1.9519873E-2,-6.204999E-3,2.730098E-3,-3.367151E-2,-5.7605907E-2,-1.9610368E-1,2.9769232E-2,1.0052433E-2,-7.412545E-2,2.1832297E-2,-1.660985E-2,-8.358355E-3,1.6186057E-1,5.474115E-2,-2.7308178E-3,-2.5934344E-2,5.814999E-4,-1.4755939E-3,-9.756356E-3,3.0771215E-3,4.883756E-3,-2.4145634E-3,-2.387239E-3,3.112064E-4,-4.068956E-3,1.2985233E-3,-9.840594E-3,-3.5051515E-3,2.8027159E-3,-1.0022606E-3,-2.2171144E-4,-6.5507293E-3,2.6754602E-3,-2.2588118E-3,1.5345495E-4,-6.299506E-3,4.7453847E-3,1.017231E-2,-1.02640784E-4,3.328387E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,-1,-1,45,47,49,51,-1,-1,-1,53,55,57,59,-1,61,63,65,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0026486E-1,4.6061438E-1,2.7005497E-1,3.3993793E-1,1.4861608E-1,2.9820123E-1,3.633531E-1,2.60139E-1,1.5500686E-1,0E0,0E0,8.3441556E-2,2.4008659E-1,3.2374358E-1,5.5159986E-2,2.9570025E-1,2.7641475E-1,0E0,8.9633405E-2,1.9845601E-2,1.1153269E-1,1.0213035E-1,7.134731E-2,9.4274074E-2,2.6071453E-1,0E0,0E0,6.2426496E-1,2.6254916E-1,2.2338483E-1,1.5237567E-1,0E0,0E0,0E0,7.75488E-3,4.5073863E-2,7.555008E-3,1.1004662E-2,0E0,5.626986E-2,5.9755288E-2,6.779456E-2,0E0,7.4831545E-2,1.3648188E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,34,34,35,35,36,36,37,37,39,39,40,40,41,41,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,-1,-1,46,48,50,52,-1,-1,-1,54,56,58,60,-1,62,64,66,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3992583E10,1.3359968E10,1.1643481E6,3.8304348E1,1.3881669E6,7.493625E1,1.8545505E2,7.080933E2,1E1,-4.6680802E-3,-1.796529E-2,1.1E1,5.58E4,3.3852024E7,6.95E2,4E0,8.959817E7,-2.380819E-4,1.639E3,3.38E2,5.723202E5,5.36763E5,2.234478E3,4.9E1,1.394E3,-9.661045E-3,-5.5696454E-4,3.18E3,1.5357143E0,9.6682465E-1,1.6869704E6,-1.9519873E-2,-6.204999E-3,2.730098E-3,4.1877E4,1.24141544E-1,1.0784811E3,4.24587E5,1.0052433E-2,1.2792593E2,2.8389828E7,5.2117184E4,-8.358355E-3,8.657E2,4.4149057E2,-2.7308178E-3,-2.5934344E-2,5.814999E-4,-1.4755939E-3,-9.756356E-3,3.0771215E-3,4.883756E-3,-2.4145634E-3,-2.387239E-3,3.112064E-4,-4.068956E-3,1.2985233E-3,-9.840594E-3,-3.5051515E-3,2.8027159E-3,-1.0022606E-3,-2.2171144E-4,-6.5507293E-3,2.6754602E-3,-2.2588118E-3,1.5345495E-4,-6.299506E-3,4.7453847E-3,1.017231E-2,-1.02640784E-4,3.328387E-3],"split_indices":[43,43,40,65,40,68,70,4,8,0,0,3,10,7,0,3,12,0,2,0,40,1,4,3,10,0,0,41,65,66,57,0,0,0,10,50,4,1,0,70,59,59,0,45,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.034E3,7.56E2,2.78E2,7.48E2,8E0,8E1,1.98E2,7.39E2,9E0,5E0,3E0,3.6E1,4.4E1,1.91E2,7E0,5.95E2,1.44E2,3E0,6E0,1.1E1,2.5E1,1.2E1,3.2E1,3E1,1.61E2,5E0,2E0,1.7E1,5.78E2,9.3E1,5.1E1,2E0,4E0,2E0,9E0,1.5E1,1E1,5E0,7E0,1.1E1,2.1E1,2.7E1,3E0,2.7E1,1.34E2,1.5E1,2E0,1.99E2,3.79E2,2E0,9.1E1,6E0,4.5E1,6E0,3E0,1.1E1,4E0,8E0,2E0,3E0,2E0,6E0,5E0,1.4E1,7E0,2.4E1,3E0,1.5E1,1.2E1,3.2E1,1.02E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[6.2126917E-4,-2.4431797E-2,1.9908814E-2,7.131589E-2,-3.0692814E-2,1.2051919E-1,1.5803413E-2,9.7493954E-2,-7.836224E-3,-4.520113E-2,1.3997911E-2,8.18255E-3,8.174995E-2,-5.960597E-2,2.2394652E-2,1.2580433E-1,-4.0908433E-2,-2.88124E-2,-9.097667E-2,3.6228854E-2,-4.8273157E-2,1.0146041E-3,4.355426E-3,-1.0202788E-1,-8.285413E-3,4.654102E-2,6.3295225E-3,1.2331628E-3,1.4750892E-1,3.4872978E-4,-3.4803757E-3,-3.4782168E-2,1.2097996E-1,-7.51968E-2,-2.4705E-1,7.314253E-2,-1.6382104E-3,1.1793385E-1,-8.230628E-2,-4.059222E-2,-1.5088941E-1,4.7860793E-3,-3.2821927E-2,-3.074425E-3,7.508434E-2,-5.5337664E-2,2.2167847E-2,3.1565924E-3,8.992759E-3,-1.2006932E-3,-9.852237E-3,8.843833E-3,-3.96522E-3,-6.3237143E-3,-2.0629573E-3,-1.7148675E-2,-2.7092558E-3,3.8710705E-3,-3.898548E-3,5.140686E-3,-8.400812E-4,1.0313782E-3,7.996762E-3,-9.915309E-3,-2.3644613E-3,-3.3721179E-3,3.7825382E-3,-8.989684E-3,-3.881401E-3,-2.3778025E-3,6.2921963E-4,8.288899E-3,-8.7221985E-4,9.397868E-4,5.4143965E-3,-3.8444492E-4,-1.5203126E-2,2.696728E-3,-1.7102649E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,-1,47,-1,-1,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8321533E-1,2.6296175E-1,2.3146625E-1,1.8762577E-1,2.6582313E-1,3.4588277E-2,2.717252E-1,1.0310206E-1,0E0,2.2919148E-1,1.415677E-1,0E0,9.605691E-3,9.4947755E-2,1.9405061E-1,4.1143715E-2,9.030122E-3,2.0898567E-1,1.8091077E-1,1.0495576E-1,1.6243337E-1,0E0,0E0,6.591922E-2,6.138019E-2,2.8346872E-1,2.9619157E-1,0E0,5.120471E-2,0E0,0E0,3.3759922E-1,1.5014438E-1,1.3572311E-1,1.4425024E-1,7.402587E-2,7.390273E-2,2.2904918E-2,8.359201E-2,5.4471955E-2,2.2149473E-2,0E0,1.6574532E-2,2.1958141E-1,2.944687E-1,7.9994416E-1,2.2824676E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,-1,48,-1,-1,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8470706E2,7E0,4.060171E6,1.45752E8,2.118835E6,8.01E2,8.147158E4,7.056E3,-7.836224E-3,4.28649E5,6.4889906E5,8.18255E-3,3.3817584E7,1.11436836E5,3.72381E5,6.2886596E-1,4.1E1,1.3177083E1,4.2964826E0,1.1090909E1,2E0,1.0146041E-3,4.355426E-3,9E1,2.1609572E7,1E0,1.048796E6,1.2331628E-3,1.0332258E2,3.4872978E-4,-3.4803757E-3,3.4E1,1E0,1.169375E2,6.8E1,2.6043478E2,4.0753E4,7.24E2,6.459696E6,7.9E1,1.6E1,4.7860793E-3,1.5958088E10,7.324E3,6.7553375E4,8.93004E5,1.2194473E8,3.1565924E-3,8.992759E-3,-1.2006932E-3,-9.852237E-3,8.843833E-3,-3.96522E-3,-6.3237143E-3,-2.0629573E-3,-1.7148675E-2,-2.7092558E-3,3.8710705E-3,-3.898548E-3,5.140686E-3,-8.400812E-4,1.0313782E-3,7.996762E-3,-9.915309E-3,-2.3644613E-3,-3.3721179E-3,3.7825382E-3,-8.989684E-3,-3.881401E-3,-2.3778025E-3,6.2921963E-4,8.288899E-3,-8.7221985E-4,9.397868E-4,5.4143965E-3,-3.8444492E-4,-1.5203126E-2,2.696728E-3,-1.7102649E-4],"split_indices":[64,3,5,7,9,2,40,10,0,9,63,0,7,45,9,65,8,68,65,70,8,0,0,0,57,27,9,0,64,0,0,3,26,64,0,64,12,2,9,0,3,0,43,9,45,9,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,4.34E2,5.64E2,2.6E1,4.08E2,2.1E1,5.43E2,2.4E1,2E0,3.08E2,1E2,7E0,1.4E1,4.3E1,5E2,2E1,4E0,2.28E2,8E1,7.4E1,2.6E1,3E0,1.1E1,2.3E1,2E1,1.99E2,3.01E2,4E0,1.6E1,2E0,2E0,2.2E2,8E0,7.4E1,6E0,3.7E1,3.7E1,4E0,2.2E1,1.1E1,1.2E1,3E0,1.7E1,7.3E1,1.26E2,6.1E1,2.4E2,7E0,9E0,2.11E2,9E0,6E0,2E0,2.3E1,5.1E1,3E0,3E0,3.5E1,2E0,4E0,3.3E1,2E0,2E0,3E0,1.9E1,9E0,2E0,6E0,6E0,1.2E1,5E0,5E0,6.8E1,5.6E1,7E1,5.3E1,8E0,9.9E1,1.41E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.4256355E-3,-3.918855E-2,7.5359964E-3,-1.2464064E-2,-7.129474E-2,3.0209547E-2,-6.8613514E-3,-5.069389E-2,4.2404335E-2,-8.4079646E-2,2.6271645E-2,1.017719E-1,1.7850174E-2,-1.0736817E-2,1.24010466E-1,-2.7252298E-2,-2.2895966E-1,-3.567836E-2,7.46568E-2,-6.839874E-2,-3.2443064E-1,6.05761E-2,-4.447724E-3,1.5985556E-2,1.1988826E-1,2.780624E-3,6.878914E-2,-8.414246E-3,-1.8933365E-1,1.1537083E-2,2.5451878E-2,-4.561386E-2,8.7985285E-2,-1.2161181E-2,-3.7215624E-3,6.994027E-4,-3.2752506E-3,1.1296237E-1,1.6051868E-2,5.628449E-3,-7.720071E-2,-2.595126E-2,1.1152587E-3,-2.1947434E-4,4.1002054E-3,2.0574196E-3,-1.4340865E-3,1.5712324E-1,4.7493428E-2,-3.8938537E-2,3.540622E-2,2.8678963E-3,1.0421166E-1,9.581214E-2,-1.2332989E-2,-6.7236414E-4,-1.649629E-2,3.3008333E-3,-2.3209006E-3,-3.1893826E-3,2.571898E-3,5.744635E-3,-1.2802482E-3,5.6873495E-3,2.9987856E-4,1.9341666E-3,-3.0376734E-3,-4.129435E-3,1.4511066E-3,2.0800296E-3,7.5893556E-3,-7.5748435E-4,3.898351E-3,-1.5908276E-4,-9.3146525E-3,7.528046E-5,4.767061E-3,2.777272E-3,-2.3043333E-3,5.303637E-3,1.0589736E-3,1.1167488E-2,2.754076E-3,4.9795974E-3,-6.9094996E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,55,-1,57,59,61,-1,-1,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,69,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6761257E-1,1.8272793E-1,2.5809407E-1,2.4966753E-1,1.232045E-1,2.6971444E-1,2.4681103E-1,2.8627864E-1,1.2606323E-1,3.0193657E-1,5.6542575E-2,6.801963E-2,2.0169468E-1,1.9377945E-1,1.6907798E-1,1.3731769E-1,2.0925492E-2,2.8291328E-2,7.675809E-2,1.4727244E-1,4.331628E-1,1.9177794E-2,0E0,1.3638042E-2,9.227562E-2,2.790456E-1,1.3980693E-1,1.9151302E-1,1.7273872E-1,0E0,3.536953E-2,1.370264E-1,4.2154662E-2,0E0,0E0,0E0,0E0,2.346465E-2,3.49091E-2,0E0,1.1202207E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.4281929E-2,3.4884002E-2,5.224488E-1,2.653654E-1,7.0238814E-2,3.2966077E-2,7.5210735E-2,1.4888377E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,37,37,38,38,40,40,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,56,-1,58,60,62,-1,-1,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,70,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7667647E2,4.125E0,1.5723623E0,2.5E-1,1.4929079E2,1.6869704E6,4.435876E2,2.28E0,4.9E2,2.2318378E6,1.42E1,4.795239E7,3.8807898E6,7.9E1,7.969064E8,4.57E2,3.074398E0,1E0,1.8992E4,1.2316317E-3,4.07E2,3.56E2,-4.447724E-3,1.3628496E3,1.8454931E8,1.0909091E0,1.1E1,2.855464E5,1E0,1.1537083E-2,2.5130852E2,4.591837E0,7.3911E4,-1.2161181E-2,-3.7215624E-3,6.994027E-4,-3.2752506E-3,9.059046E6,1E0,5.628449E-3,2.083138E6,-2.595126E-2,1.1152587E-3,-2.1947434E-4,4.1002054E-3,2.0574196E-3,-1.4340865E-3,1.9741463E2,4.0481758E3,1E0,3.289E3,1.0366492E0,1.200096E9,1.8497453E0,1.4E1,-6.7236414E-4,-1.649629E-2,3.3008333E-3,-2.3209006E-3,-3.1893826E-3,2.571898E-3,5.744635E-3,-1.2802482E-3,5.6873495E-3,2.9987856E-4,1.9341666E-3,-3.0376734E-3,-4.129435E-3,1.4511066E-3,2.0800296E-3,7.5893556E-3,-7.5748435E-4,3.898351E-3,-1.5908276E-4,-9.3146525E-3,7.528046E-5,4.767061E-3,2.777272E-3,-2.3043333E-3,5.303637E-3,1.0589736E-3,1.1167488E-2,2.754076E-3,4.9795974E-3,-6.9094996E-4],"split_indices":[67,70,65,70,64,57,68,68,1,40,70,7,40,3,12,0,68,19,9,50,0,2,0,45,7,65,3,57,14,0,70,66,41,0,0,0,0,40,25,0,12,0,0,0,0,0,0,4,45,65,41,65,12,65,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.002E3,2.13E2,7.89E2,1.17E2,9.6E1,3.06E2,4.83E2,6.9E1,4.8E1,8.5E1,1.1E1,4.4E1,2.62E2,4.7E2,1.3E1,6.2E1,7E0,1.4E1,3.4E1,8.1E1,4E0,9E0,2E0,8E0,3.6E1,2.03E2,5.9E1,4.65E2,5E0,5E0,8E0,5.4E1,8E0,5E0,2E0,6E0,8E0,2E1,1.4E1,3E0,7.8E1,2E0,2E0,3E0,6E0,5E0,3E0,2.3E1,1.3E1,8.9E1,1.14E2,2.1E1,3.8E1,1.6E1,4.49E2,3E0,2E0,5E0,3E0,4.4E1,1E1,6E0,2E0,1.8E1,2E0,1.1E1,3E0,7E1,8E0,2E0,2.1E1,5E0,8E0,7.4E1,1.5E1,7.7E1,3.7E1,1E1,1.1E1,3.3E1,5E0,2E0,1.4E1,9E0,4.4E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[2.6627076E-3,-1.7318545E-1,4.0654275E-3,-1.011678E-2,-3.2633778E-3,9.713244E-3,-3.4878705E-2,-3.330519E-3,2.947587E-2,-8.469532E-2,7.900829E-4,2.6104334E-3,-8.4596045E-2,5.49919E-2,4.2272927E-3,3.91873E-2,-9.841616E-2,1.0347196E-2,-5.0911102E-3,5.498833E-3,-1.7318615E-1,-5.5886738E-2,-2.1882814E-1,8.315768E-2,1.1333942E-2,1.2249078E-1,-7.5518563E-3,-2.002246E-3,4.2015845E-3,-1.0661795E-1,2.7627856E-3,1.7536078E-2,-7.1676117E-3,7.10265E-5,5.5809687E-3,1.23623E-3,-2.3685016E-2,-9.184242E-4,-6.494682E-3,-1.3349802E-2,-2.3024937E-3,-5.051738E-4,4.1814484E-3,3.9592753E-3,-1.5231991E-3,1.0567216E-2,-6.357366E-4,6.1392034E-3,-6.8155484E-4,-4.203396E-3,-1.3439484E-2,1.2015065E-3,-5.8316737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,-1,-1,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,-1,-1,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4934311E-1,2.281347E-2,2.2050807E-1,0E0,0E0,2.2565478E-1,2.2573297E-1,2.551364E-1,2.2396424E-1,9.394467E-2,8.1162386E-2,2.5128332E-1,1.2650403E-1,2.1178877E-1,2.4610063E-1,2.9957552E-2,6.883836E-2,8.6471125E-2,0E0,2.2265117E-1,5.621114E-1,9.231619E-2,5.812365E-2,7.8894496E-2,2.3271419E-1,2.3783562E-1,1.6764553E-1,0E0,0E0,1.0243565E-1,0E0,8.804675E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,31,31],"right_children":[2,4,6,-1,-1,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,-1,-1,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,1.1634076E4,1E0,-1.011678E-2,-3.2633778E-3,4.6134964E2,5E0,1E0,8E0,1.4473684E-1,3.5531914E0,4.75356E6,4.39776E6,2.3308511E5,1.0972E4,2.0549193E2,1.1246073E2,2.4077083E3,-5.0911102E-3,2.7311892E6,2.239E3,9E0,9.1E1,1.0274E3,1.6429862E0,1.094E5,2.059E3,-2.002246E-3,4.2015845E-3,1.2951E4,2.7627856E-3,3.8807898E6,-7.1676117E-3,7.10265E-5,5.5809687E-3,1.23623E-3,-2.3685016E-2,-9.184242E-4,-6.494682E-3,-1.3349802E-2,-2.3024937E-3,-5.051738E-4,4.1814484E-3,3.9592753E-3,-1.5231991E-3,1.0567216E-2,-6.357366E-4,6.1392034E-3,-6.8155484E-4,-4.203396E-3,-1.3439484E-2,1.2015065E-3,-5.8316737E-3],"split_indices":[64,40,6,0,0,4,6,82,29,68,65,59,9,45,9,4,70,64,0,63,2,8,0,45,54,10,41,0,0,41,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.007E3,7E0,1E3,4E0,3E0,8.74E2,1.26E2,5.27E2,3.47E2,5.2E1,7.4E1,4.92E2,3.5E1,1.72E2,1.75E2,5E0,4.7E1,6.9E1,5E0,4.85E2,7E0,3E1,5E0,1.04E2,6.8E1,1.5E1,1.6E2,2E0,3E0,4.5E1,2E0,6.7E1,2E0,4.7E2,1.5E1,5E0,2E0,2.2E1,8E0,3E0,2E0,8E0,9.6E1,2.5E1,4.3E1,8E0,7E0,7E0,1.53E2,4.3E1,2E0,6.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.354652E-3,-8.11005E-2,1.1934921E-3,-4.803151E-2,-2.284147E-1,-5.5839546E-4,1.6732827E-1,5.5839594E-2,-6.7604944E-2,-1.5077683E-2,-1.3872717E-1,3.5591237E-2,-7.0691262E-3,1.966095E-3,9.856012E-3,-2.6098022E-4,4.3124626E-3,1.309327E-3,-7.341357E-2,-1.8538137E-3,-7.8084473E-3,4.7026552E-2,-1.4243425E-1,2.415679E-2,-1.6755437E-2,-1.0122245E-1,-4.027721E-2,2.6012033E-2,1.1804356E-1,-1.348267E-2,3.7156653E-2,-3.703998E-2,5.2468594E-2,7.166607E-3,-1.8427478E-2,-5.2028135E-3,-1.1345602E-3,3.924089E-3,-2.7601272E-3,1.6477655E-3,-5.511944E-3,2.2154786E-3,6.8433024E-3,3.1931484E-3,-3.487106E-4,-9.6719037E-4,-1.1920619E-2,3.9904364E-3,8.694203E-6,-2.7257416E-3,-3.6614705E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,-1,-1,-1,25,-1,-1,27,29,31,33,35,37,39,41,-1,43,45,47,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.582929E-1,2.6183465E-1,2.7891883E-1,9.9331394E-2,4.401338E-2,2.2360875E-1,4.96549E-2,1.9260285E-2,2.4423674E-2,0E0,1.4673397E-2,3.0188593E-1,2.4388824E-1,0E0,0E0,0E0,0E0,0E0,3.093946E-2,0E0,0E0,2.0138666E-1,2.551391E-1,3.328683E-1,1.8054931E-1,1.5644848E-2,5.2461993E-2,1.5839088E-1,5.9782535E-2,0E0,7.7688037E-3,2.114515E-1,2.3563874E-1,0E0,2.604937E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,34,34],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,-1,-1,-1,26,-1,-1,28,30,32,34,36,38,40,42,-1,44,46,48,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.964269E3,6.1839433E0,4.7263342E2,1.0657745E7,6.34E2,1.6954315E0,1.3E1,1.631108E6,8.910034E0,-1.5077683E-2,1.70698E6,3.174172E5,3.5E1,1.966095E-3,9.856012E-3,-2.6098022E-4,4.3124626E-3,1.309327E-3,1.688E3,-1.8538137E-3,-7.8084473E-3,1.5350968E0,3.596E3,4.9E2,1E0,1.0784811E3,9E1,2.4919647E5,4.455844E1,-1.348267E-2,5.526E3,3.09627E5,1.783854E8,7.166607E-3,1E0,-5.2028135E-3,-1.1345602E-3,3.924089E-3,-2.7601272E-3,1.6477655E-3,-5.511944E-3,2.2154786E-3,6.8433024E-3,3.1931484E-3,-3.487106E-4,-9.6719037E-4,-1.1920619E-2,3.9904364E-3,8.694203E-6,-2.7257416E-3,-3.6614705E-4],"split_indices":[40,66,68,57,2,66,3,5,68,0,5,45,0,0,0,0,0,0,2,0,0,54,2,1,20,4,0,45,70,0,2,9,43,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.012E3,5.5E1,9.57E2,4.6E1,9E0,9.48E2,9E0,7E0,3.9E1,3E0,6E0,1.44E2,8.04E2,3E0,6E0,3E0,4E0,2E0,3.7E1,2E0,4E0,1.36E2,8E0,1.9E2,6.14E2,1.9E1,1.8E1,1.06E2,3E1,4E0,4E0,6E1,1.3E2,5E0,6.09E2,1.6E1,3E0,2E0,1.6E1,1E2,6E0,1E1,2E1,2E0,2E0,5.7E1,3E0,7.8E1,5.2E1,1.23E2,4.86E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[4.775846E-4,-2.2645298E-2,1.3514977E-2,-1.9404316E-2,-2.5017053E-1,1.0729035E-2,1.0949823E-1,7.092952E-2,-2.4959365E-2,2.168014E-3,-2.1319214E-2,1.8306384E-2,-3.2269668E-2,9.951373E-3,2.1610498E-2,1.2235779E-1,2.1226436E-3,-1.0781617E-2,-8.5489385E-2,8.914946E-3,5.5965226E-2,2.8337749E-2,-5.230218E-2,-3.2426456E-3,2.986153E-3,4.746243E-2,7.7658365E-3,-4.324609E-2,1.8177937E-3,-2.2612581E-2,4.0389873E-2,-2.5589266E-1,-5.6576546E-2,2.1343585E-2,-2.5810154E-2,1.0179842E-1,5.7154405E-4,-2.2862589E-2,7.341977E-2,-5.9764985E-2,6.2823505E-3,3.5280364E-3,-3.4400262E-4,-4.0761367E-3,7.6553697E-4,-1.6531821E-3,1.6637862E-3,4.458624E-3,-5.2287654E-5,-1.5560344E-2,-7.2567565E-3,-4.5726965E-3,2.2394468E-4,4.159111E-3,4.3514522E-4,9.5119164E-4,-2.561814E-3,1.0045358E-3,6.302696E-3,5.2280445E-4,-7.7618216E-3,2.6237878E-3,-2.56628E-3,1.07796615E-4,4.8111896E-3,2.5372372E-3,-3.1091995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,43,-1,45,47,49,51,53,55,57,59,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0267256E-1,2.6208627E-1,1.7049842E-1,1.8141793E-1,3.4006816E-1,2.0422584E-1,1.6484025E-1,7.405127E-2,2.891429E-1,0E0,0E0,1.8765388E-1,1.156093E-1,0E0,4.861243E-2,3.3961385E-2,1.8703694E-2,1.6740054E-1,3.0084035E-1,1.84838E-1,2.69062E-1,5.7075724E-2,1.08359426E-1,0E0,0E0,1.0251744E-2,0E0,1.51085835E-2,0E0,1.776379E-1,1.2251179E-1,1.950401E-2,1.4838666E-1,2.5790498E-1,1.5866774E-1,1.6120434E-1,9.174266E-2,3.4875132E-2,2.8698832E-2,6.6642806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,44,-1,46,48,50,52,54,56,58,60,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.01E2,1.8285715E1,4.435876E2,5.445545E-1,1.3E1,2E1,1.8545505E2,1.2877E4,2.988E4,2.168014E-3,-2.1319214E-2,1.6E1,4E0,9.951373E-3,1E1,1.08712E5,7.2896E5,1.102E3,5.8182236E1,1.4E1,3.14403E5,2.483E3,1.54E2,-3.2426456E-3,2.986153E-3,2.3257812E2,7.7658365E-3,2.78E2,1.8177937E-3,5E0,1.5016073E4,5.8036E4,2.0262831E2,1.0164831E-2,1.6939986E9,1E0,2.4262331E5,5.5177975E6,1.5043378E0,2.1959582E8,6.2823505E-3,3.5280364E-3,-3.4400262E-4,-4.0761367E-3,7.6553697E-4,-1.6531821E-3,1.6637862E-3,4.458624E-3,-5.2287654E-5,-1.5560344E-2,-7.2567565E-3,-4.5726965E-3,2.2394468E-4,4.159111E-3,4.3514522E-4,9.5119164E-4,-2.561814E-3,1.0045358E-3,6.302696E-3,5.2280445E-4,-7.7618216E-3,2.6237878E-3,-2.56628E-3,1.07796615E-4,4.8111896E-3,2.5372372E-3,-3.1091995E-3],"split_indices":[2,66,68,65,0,3,70,9,1,0,0,3,8,0,3,7,9,41,59,3,9,2,3,0,0,45,0,0,0,8,59,1,64,54,43,27,45,57,66,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.002E3,3.61E2,6.41E2,3.57E2,4E0,6.24E2,1.7E1,2E1,3.37E2,2E0,2E0,5.31E2,9.3E1,7E0,1E1,1.1E1,9E0,2.74E2,6.3E1,4.26E2,1.05E2,2.3E1,7E1,3E0,7E0,5E0,6E0,4E0,5E0,2.23E2,5.1E1,8E0,5.5E1,3.14E2,1.12E2,5.7E1,4.8E1,1.1E1,1.2E1,6.8E1,2E0,3E0,2E0,2E0,2E0,1.82E2,4.1E1,2.1E1,3E1,3E0,5E0,3.2E1,2.3E1,4.5E1,2.69E2,4.4E1,6.8E1,1.8E1,3.9E1,4.6E1,2E0,3E0,8E0,4E0,8E0,4E0,6.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.0335568E-3,-2.0092472E-2,1.9977175E-2,-1.5800243E-2,-9.915197E-2,3.0000778E-2,-4.3200836E-2,-1.8874831E-2,9.271935E-2,4.8408553E-2,-2.0362535E-1,3.909588E-2,-9.679341E-3,-7.699165E-2,2.788041E-2,-1.1034981E-2,-6.895395E-2,3.367623E-2,1.368693E-2,5.3698584E-3,-4.5593223E-4,-3.6083925E-1,-7.32131E-2,5.5995215E-2,2.2468788E-3,6.873212E-2,-6.0118422E-2,-9.813E-2,3.3039473E-2,5.8618367E-2,-5.7458063E-3,-3.402521E-2,3.1682581E-3,-1.3903534E-1,-1.36562055E-2,-1.0028918E-2,7.7226846E-3,-2.3957712E-3,1.8831101E-3,-1.9005936E-2,-6.9958866E-3,-5.403711E-3,3.2005934E-3,7.462194E-2,4.0187887E-3,-3.0488865E-2,4.981144E-2,-2.9337924E-2,1.1018522E-1,-1.4606185E-2,-4.1993205E-2,-1.1013639E-1,3.6128236E-3,3.2000558E-3,-9.6496125E-4,6.243504E-3,2.1297513E-2,-6.55352E-4,-4.0589836E-3,1.2412185E-3,-1.0556794E-3,5.6292793E-5,-7.7306214E-3,6.4454897E-4,-6.3515846E-3,1.3304583E-3,-3.4799112E-3,4.1935295E-3,4.596866E-4,2.7671896E-3,-1.2056987E-3,5.60302E-3,-1.8366842E-3,5.3960998E-3,-2.9014665E-4,-4.447204E-3,1.4601095E-3,5.9836507E-3,2.2290142E-3,-9.793033E-3,-1.1524013E-3,4.357719E-5,-6.2918304E-3,-4.2577423E-3,2.205701E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,-1,-1,-1,-1,-1,-1,-1,67,69,71,73,75,77,-1,79,81,-1,-1,-1,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0324584E-1,1.7760564E-1,3.0463302E-1,1.6924393E-1,4.260932E-1,1.497888E-1,1.6067152E-1,1.912268E-1,1.5963861E-1,4.0220767E-2,3.016237E-1,2.0973206E-1,3.1262222E-1,1.0790661E-1,1.1078018E-1,1.3855894E-1,2.5387394E-1,7.2204515E-2,0E0,0E0,1.9285088E-2,1.4264941E-2,7.169172E-2,2.2334296E-1,1.6812427E-1,1.2925716E-1,2.11005E-1,8.932063E-2,1.7302223E-2,5.192685E-2,0E0,1.7267951E-1,1.6469504E-1,1.1927247E-1,1.3307142E-1,2.7827581E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8227899E-1,1.0710065E-1,9.756872E-2,1.67863E-1,4.432895E-2,2.0926058E-2,0E0,1.2778294E-1,1.0749036E-1,0E0,0E0,0E0,0E0,4.7101088E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,-1,-1,-1,-1,-1,-1,-1,68,70,72,74,76,78,-1,80,82,-1,-1,-1,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2874417E2,9.932432E0,2.3E1,2.2503355E1,3.89E2,2.3385885E0,5.467E3,2.4E1,1.3881669E6,8E1,6.34E2,8E0,2.4595E5,3.149243E8,5.712652E8,1.463E2,1.9637654E2,2.194E3,1.368693E-2,5.3698584E-3,1.3E2,1.2916E4,8.875E0,3.0923106E-10,9.8897E2,3.13173E0,1.45E2,1.5038776E2,4.15E2,1.3145597E-1,-5.7458063E-3,3.14403E5,1E0,1.5522388E0,4.2438595E1,3.78E2,7.7226846E-3,-2.3957712E-3,1.8831101E-3,-1.9005936E-2,-6.9958866E-3,-5.403711E-3,3.2005934E-3,4.900919E8,1.2909952E1,5.2571965E8,4.5576923E1,1.1724138E0,2.115238E5,-1.4606185E-2,3.72424E5,1.3809524E0,3.6128236E-3,3.2000558E-3,-9.6496125E-4,6.243504E-3,1.332E3,-6.55352E-4,-4.0589836E-3,1.2412185E-3,-1.0556794E-3,5.6292793E-5,-7.7306214E-3,6.4454897E-4,-6.3515846E-3,1.3304583E-3,-3.4799112E-3,4.1935295E-3,4.596866E-4,2.7671896E-3,-1.2056987E-3,5.60302E-3,-1.8366842E-3,5.3960998E-3,-2.9014665E-4,-4.447204E-3,1.4601095E-3,5.9836507E-3,2.2290142E-3,-9.793033E-3,-1.1524013E-3,4.357719E-5,-6.2918304E-3,-4.2577423E-3,2.205701E-3],"split_indices":[64,66,3,68,2,50,2,8,40,41,2,29,9,7,44,67,64,2,0,0,2,1,65,49,4,50,10,58,0,50,0,9,24,65,70,0,0,0,0,0,0,0,0,7,68,43,70,65,45,0,9,66,0,0,0,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.005E3,5.27E2,4.78E2,5.01E2,2.6E1,4.13E2,6.5E1,4.88E2,1.3E1,1.1E1,1.5E1,3.36E2,7.7E1,4.4E1,2.1E1,4.23E2,6.5E1,1.1E1,2E0,4E0,7E0,6E0,9E0,2.3E2,1.06E2,3E1,4.7E1,3.7E1,7E0,1.8E1,3E0,1.61E2,2.62E2,2.8E1,3.7E1,9E0,2E0,3E0,4E0,4E0,2E0,7E0,2E0,1.69E2,6.1E1,6.3E1,4.3E1,9E0,2.1E1,2E0,4.5E1,3.5E1,2E0,4E0,3E0,5E0,1.3E1,1.19E2,4.2E1,1.37E2,1.25E2,5E0,2.3E1,3.1E1,6E0,6E0,3E0,1.34E2,3.5E1,2.1E1,4E1,3E0,6E1,1.9E1,2.4E1,4E0,5E0,1.5E1,6E0,3E0,4.2E1,7E0,2.8E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[3.080543E-4,-2.5304684E-2,1.6470935E-2,7.538098E-2,-3.2407485E-2,9.123548E-2,1.276815E-2,-8.302389E-3,1.20172314E-1,-2.3605386E-2,-9.447564E-2,1.0286128E-1,-2.0121923E-3,-4.7116075E-2,1.8279135E-2,-2.6601583E-3,2.429507E-3,8.738744E-3,7.5116195E-2,-6.0317427E-2,-1.137163E-2,-1.1171715E-1,8.45804E-3,6.552605E-5,1.1091899E-1,-1.01736434E-1,-1.3212856E-2,1.5083826E-2,1.1028264E-1,1.07564E-1,-7.6569756E-4,-4.9116623E-2,-8.739606E-3,1.28324395E-2,-3.8660694E-2,-1.4104569E-1,3.422591E-2,4.8235033E-2,1.2817523E-1,1.9285941E-4,-1.144464E-1,4.568055E-2,-5.0832927E-2,2.0102313E-2,-4.0120162E-2,1.3596784E-1,7.4903073E-4,5.7737892E-3,1.3495676E-3,-2.999712E-3,2.3437357E-3,3.6218273E-5,4.494094E-3,-1.5217765E-3,-1.0665337E-2,-4.0792944E-3,-9.222431E-3,3.966538E-3,-3.740945E-3,-1.6372796E-3,4.0840814E-3,6.7961877E-3,2.7853046E-3,-8.645437E-3,-3.1242738E-3,3.4944678E-3,-1.1323757E-4,2.4414985E-3,-3.0011775E-3,4.362932E-3,7.3314784E-4,-1.0115229E-2,-2.704151E-4,7.1317353E-3,6.1738817E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,49,-1,51,53,55,57,59,61,-1,63,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1729027E-1,2.8135148E-1,1.6992036E-1,9.845325E-2,1.9718042E-1,5.0036013E-2,1.9557366E-1,3.3425E-2,3.938946E-2,1.4348358E-1,2.367444E-1,2.1909714E-2,0E0,9.124607E-2,1.5737954E-1,0E0,0E0,0E0,3.753241E-2,1.0710433E-1,1.6033866E-1,1.8888015E-1,0E0,0E0,2.0859718E-2,2.6062787E-2,7.322045E-2,1.4597572E-1,4.1230544E-2,9.070672E-3,0E0,1.2494421E-1,0E0,1.320012E-1,1.1430271E-1,9.450382E-2,5.5274896E-2,2.7056495E-2,1.6418874E-2,0E0,4.5425832E-2,1.9214423E-2,3.3644885E-2,1.4762633E-1,2.699837E-1,3.1021178E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,18,18,19,19,20,20,21,21,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,50,-1,52,54,56,58,60,62,-1,64,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.778995E2,7E0,4.1719616E5,6E-1,2.4E1,8.303E3,8.147158E4,4.5801528E-2,9.0909094E-1,1.1E1,2.7E1,7.402E3,-2.0121923E-3,1.2469E5,8.2471845E6,-2.6601583E-3,2.429507E-3,8.738744E-3,1.5899301E2,5.1485147E0,1.5E1,2.284E3,8.45804E-3,6.552605E-5,1.3E1,2.01E4,5.12009E5,2.9366477E0,1E0,2.871795E0,-7.6569756E-4,8E0,-8.739606E-3,8.83425E7,6.2521395E6,8.201515E5,1.357E3,1.2460085E7,3.167E3,1.9285941E-4,1.6223962E7,2.93E2,1.6580646E2,2.3781754E8,3.091744E0,2.115238E5,7.4903073E-4,5.7737892E-3,1.3495676E-3,-2.999712E-3,2.3437357E-3,3.6218273E-5,4.494094E-3,-1.5217765E-3,-1.0665337E-2,-4.0792944E-3,-9.222431E-3,3.966538E-3,-3.740945E-3,-1.6372796E-3,4.0840814E-3,6.7961877E-3,2.7853046E-3,-8.645437E-3,-3.1242738E-3,3.4944678E-3,-1.1323757E-4,2.4414985E-3,-3.0011775E-3,4.362932E-3,7.3314784E-4,-1.0115229E-2,-2.704151E-4,7.1317353E-3,6.1738817E-4],"split_indices":[67,3,57,65,8,41,40,70,65,3,3,1,0,9,40,0,0,0,64,66,3,0,0,0,3,1,9,54,82,65,0,8,0,7,40,57,2,12,2,0,5,0,64,43,54,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.006E3,3.89E2,6.17E2,2.5E1,3.64E2,2.8E1,5.89E2,9E0,1.6E1,3.2E2,4.4E1,2.6E1,2E0,4.9E1,5.4E2,5E0,4E0,5E0,1.1E1,7.9E1,2.41E2,4.2E1,2E0,2E0,2.4E1,1.8E1,3.1E1,5.23E2,1.7E1,8E0,3E0,7.4E1,5E0,1.28E2,1.13E2,3.5E1,7E0,6E0,1.8E1,2E0,1.6E1,1.2E1,1.9E1,4.8E2,4.3E1,1.3E1,4E0,6E0,2E0,6.4E1,1E1,1.13E2,1.5E1,1.11E2,2E0,2E1,1.5E1,5E0,2E0,2E0,4E0,1.3E1,5E0,5E0,1.1E1,7E0,5E0,2E0,1.7E1,2.4E1,4.56E2,6E0,3.7E1,1.1E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[4.9743433E-3,8.706346E-3,-5.9891395E-2,-1.7202914E-2,1.6991122E-2,-7.387483E-2,6.4470437E-3,-1.2362657E-2,-1.7378236E-1,1.4344245E-2,1.3536945E-1,-1.03110485E-2,-5.6998566E-2,1.6385067E-2,-3.8394783E-2,-1.0381114E-2,-1.3230274E-3,1.1046446E-3,3.2858424E-2,2.0241418E-1,-1.0107477E-2,-3.7261736E-2,-1.9223554E-1,-1.6392866E-2,7.415628E-2,-1.7656839E-2,-2.6339857E-2,1.6565597E-2,-4.8845276E-2,4.6038803E-2,-3.9351266E-2,3.622285E-3,1.2022534E-2,-2.198576E-3,2.0029305E-3,-1.2608421E-3,-1.1431645E-1,-4.186536E-3,-1.2077304E-2,2.2829762E-3,-1.7509816E-3,5.5212928E-5,4.8402403E-3,1.0501925E-3,-1.9580424E-3,-3.1194807E-4,1.4851325E-3,-3.4558305E-3,2.7784177E-3,1.2810801E-3,5.2674017E-3,-4.2919605E-3,9.716605E-4,1.6118073E-3,-6.012924E-3,-2.6901911E-3,-8.21642E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.491367E-1,2.0893443E-1,1.6718212E-1,1.767124E-1,2.2898445E-1,1.2191868E-1,0E0,1.7275292E-1,4.6495616E-2,1.7694059E-1,1.5810132E-1,0E0,1.2362528E-1,2.0951393E-1,4.945975E-1,0E0,0E0,3.2670847E-1,2.8828877E-1,5.9997648E-2,1.4256461E-2,1.2191355E-1,1.8918782E-2,1.03864625E-1,9.098773E-2,0E0,9.555425E-2,1.1917927E-1,2.931256E-1,3.1444746E-1,1.5580003E-1,0E0,0E0,0E0,0E0,1.5069906E-1,3.9914995E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,35,35,36,36],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.05039425E1,1.5422421E2,3.558194E6,2.7E1,4.6463413E0,8.612209E-6,6.4470437E-3,2.772925E3,5.7676364E-2,1.273801E6,1.3520833E1,-1.03110485E-2,4.9E1,1.65E2,7.5456814E-4,-1.0381114E-2,-1.3230274E-3,3.72381E5,4.900919E8,2.9211267E2,2.7863942E8,3.2927834E1,2.9849844E7,2.416E3,1.3909517E-5,-1.7656839E-2,3.168978E-2,1E0,4.797448E8,1.5222016E8,1.016E4,3.622285E-3,1.2022534E-2,-2.198576E-3,2.0029305E-3,6.323775E-7,1.3342042E0,-4.186536E-3,-1.2077304E-2,2.2829762E-3,-1.7509816E-3,5.5212928E-5,4.8402403E-3,1.0501925E-3,-1.9580424E-3,-3.1194807E-4,1.4851325E-3,-3.4558305E-3,2.7784177E-3,1.2810801E-3,5.2674017E-3,-4.2919605E-3,9.716605E-4,1.6118073E-3,-6.012924E-3,-2.6901911E-3,-8.21642E-3],"split_indices":[66,64,1,3,65,50,0,59,50,9,70,0,3,12,50,0,0,9,7,4,7,70,5,9,50,0,50,27,7,5,2,0,0,0,0,49,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.026E3,9.71E2,5.5E1,2.35E2,7.36E2,5.2E1,3E0,2.29E2,6E0,7.21E2,1.5E1,4E0,4.8E1,1.09E2,1.2E2,4E0,2E0,4.21E2,3E2,1E1,5E0,4.3E1,5E0,7E1,3.9E1,3E0,1.17E2,3.22E2,9.9E1,2.54E2,4.6E1,4E0,6E0,3E0,2E0,3E1,1.3E1,3E0,2E0,1.7E1,5.3E1,1.2E1,2.7E1,2.9E1,8.8E1,1.3E2,1.92E2,8E1,1.9E1,2.02E2,5.2E1,2.4E1,2.2E1,2.4E1,6E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[4.152351E-3,-1.1634417E-2,1.8075839E-2,-1.4559461E-2,8.505137E-3,5.1871806E-2,7.991346E-3,2.7774338E-2,-2.6915325E-2,-9.062824E-3,9.81375E-2,1.3424793E-2,-9.971882E-2,-8.363865E-3,8.7611474E-2,-2.1804923E-2,-1.3606496E-1,-2.2275524E-2,5.7357824E-3,1.8594988E-1,8.040293E-2,3.954473E-2,-5.0971666E-3,-1.6003113E-2,-1.576969E-2,3.2826804E-3,-1.1846561E-2,2.6297059E-2,1.0717304E-1,-1.2069256E-2,-8.524768E-2,3.6817114E-3,-2.2028112E-1,6.3722946E-2,-4.5368217E-2,1.04617085E-2,4.143733E-3,1.429716E-1,6.1782196E-2,2.3224906E-3,6.334374E-2,-1.0564828E-1,1.0936329E-2,5.81484E-3,-7.37754E-2,7.8114E-4,-6.1779646E-3,-4.604869E-4,3.301984E-3,7.4599385E-3,3.280969E-3,-4.812954E-3,-3.7684248E-4,-8.4308935E-3,-1.6766329E-3,2.316804E-3,-3.4667575E-3,-1.2919388E-2,7.2552334E-4,-1.5771422E-3,4.8130495E-3,4.3450603E-3,-2.9188502E-3,2.27414E-3,8.965619E-3,-3.5485486E-3,3.4989733E-3,7.002808E-3,-4.6032996E-4,3.536187E-3,-1.4199516E-4,-2.2611944E-3,-2.9149005E-2,-6.998752E-4,2.1180506E-3,9.5513684E-4,-6.6277483E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,-1,43,45,-1,47,49,51,53,55,57,59,61,-1,-1,63,65,67,69,71,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2177562E-1,2.7498358E-1,1.8226242E-1,2.4502157E-1,0E0,3.4764895E-1,2.4362703E-1,2.2994785E-1,1.9895616E-1,9.834568E-2,9.406096E-2,1.9129959E-1,4.1205755E-1,1.9557819E-1,4.515791E-2,2.1329828E-1,1.8911389E-1,1.02187805E-1,0E0,2.3489028E-2,6.1071366E-2,1.4513266E-1,3.7518254E-1,0E0,1.4147E-1,1.2520057E-1,0E0,1.8936995E-2,4.763034E-2,1.07356265E-1,2.1372697E-1,2.9716419E-2,1.4843497E-1,4.7892578E-2,1.077932E-1,0E0,0E0,5.3786337E-2,1.0196142E-1,1.2199943E-1,9.088522E-2,9.237584E-1,1.85765E-1,0E0,8.303062E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,44,44],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,-1,44,46,-1,48,50,52,54,56,58,60,62,-1,-1,64,66,68,70,72,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.953366E7,2.628361E6,4.0663E4,2.4E0,8.505137E-3,1E0,3.174172E5,3.72381E5,4.9E1,2.2539758E5,1.0918E4,5.9192824E0,6.27907E0,1.7288135E0,6.2886596E-1,2.4E1,1E1,1.1428572E1,5.7357824E-3,4.284608E6,3.4698796E0,1.013E3,5.9754E4,-1.6003113E-2,6.627033E7,6.763314E7,-1.1846561E-2,2.617E3,9.4658756E-1,2.5384614E0,2.5570058E2,2.6E1,1.3254E4,8.2434414E4,3.8795E4,1.04617085E-2,4.143733E-3,1.8224286E2,1.512E4,1.3050649E4,3.9E2,7.755551E7,6.523292E6,5.81484E-3,4E0,7.8114E-4,-6.1779646E-3,-4.604869E-4,3.301984E-3,7.4599385E-3,3.280969E-3,-4.812954E-3,-3.7684248E-4,-8.4308935E-3,-1.6766329E-3,2.316804E-3,-3.4667575E-3,-1.2919388E-2,7.2552334E-4,-1.5771422E-3,4.8130495E-3,4.3450603E-3,-2.9188502E-3,2.27414E-3,8.965619E-3,-3.5485486E-3,3.4989733E-3,7.002808E-3,-4.6032996E-4,3.536187E-3,-1.4199516E-4,-2.2611944E-3,-2.9149005E-2,-6.998752E-4,2.1180506E-3,9.5513684E-4,-6.6277483E-3],"split_indices":[7,1,9,66,0,27,45,9,3,45,9,68,68,65,65,8,8,68,0,40,68,41,1,0,12,7,0,41,65,66,4,0,41,45,1,0,0,4,9,40,0,5,57,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.007E3,4.72E2,5.35E2,4.66E2,6E0,1.22E2,4.13E2,1.05E2,3.61E2,5.3E1,6.9E1,3.94E2,1.9E1,6.6E1,3.9E1,3.46E2,1.5E1,4.9E1,4E0,1E1,5.9E1,1.63E2,2.31E2,4E0,1.5E1,6.4E1,2E0,1E1,2.9E1,3.01E2,4.5E1,6E0,9E0,1E1,3.9E1,6E0,4E0,1.2E1,4.7E1,6.4E1,9.9E1,3.1E1,2E2,4E0,1.1E1,5.9E1,5E0,6E0,4E0,1E1,1.9E1,1.1E1,2.9E2,1.4E1,3.1E1,4E0,2E0,7E0,2E0,3E0,7E0,4E0,3.5E1,5E0,7E0,4E0,4.3E1,4E0,6E1,8.2E1,1.7E1,2.9E1,2E0,1.15E2,8.5E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[2.1474946E-3,-3.0962624E-2,9.234008E-3,-3.0468155E-3,-6.6551924E-2,1.33669805E-2,-4.071058E-2,-3.3687454E-2,5.832679E-2,-1.1445356E-1,-1.8172065E-2,1.1709892E-2,1.8687202E-1,-2.7679688E-1,-1.5145171E-2,1.7687617E-2,-5.977698E-2,-1.6200395E-2,1.1409421E-1,-9.875359E-2,-1.0252608E-2,4.7185685E-2,-7.0820585E-2,3.9060528E-3,4.46014E-2,1.0330111E-2,2.8108987E-3,-1.5146548E-2,-5.23202E-3,-6.858248E-2,2.0403495E-2,-5.504753E-2,5.2949715E-2,-9.0413846E-2,-5.115997E-4,-6.1349444E-2,6.1748087E-2,1.7540744E-1,5.3556636E-2,-1.3195257E-1,-3.952994E-2,1.0318349E-1,2.1704035E-3,-9.966824E-2,5.234149E-3,-9.878564E-3,2.2722414E-2,6.831604E-2,-1.1586169E-2,3.9003026E-2,-1.4053053E-1,-6.534902E-2,6.79852E-2,-3.560274E-3,3.780995E-4,4.7727013E-3,-5.0516415E-4,-3.2834325E-3,-1.0593305E-2,2.2307562E-3,-1.5991271E-3,-1.2414559E-4,-3.4757892E-3,5.046947E-4,4.999444E-3,1.076209E-2,3.7400343E-3,-8.1782945E-4,3.7913313E-3,-4.012377E-3,-9.143953E-3,1.8051737E-3,-4.0278267E-3,5.7727494E-3,1.0928414E-3,-1.8589721E-3,2.8218788E-3,-3.3597238E-3,-7.3925653E-3,2.610734E-3,-1.2298628E-3,-2.9527053E-4,-1.852622E-2,9.545952E-3,9.040232E-4,9.13387E-4,5.4344903E-3,-1.6894399E-3,3.4557618E-3,3.509488E-3,-6.2844675E-4,-6.086858E-4,-7.99384E-3,-4.332546E-3,1.0762257E-3,1.475993E-3,6.6756327E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,-1,-1,-1,-1,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3022851E-1,1.7167342E-1,1.6733122E-1,1.8625647E-1,1.7404822E-1,2.1258405E-1,3.700177E-1,8.95205E-2,1.3900259E-1,4.284364E-2,1.375403E-1,1.9002572E-1,1.969257E-2,1.4202654E-2,1.0967796E-1,6.190501E-2,7.984479E-2,5.6577712E-2,6.1129108E-2,6.3599706E-2,0E0,4.5149673E-2,4.878971E-2,1.5585026E-1,1.9000193E-1,0E0,0E0,0E0,0E0,1.8351415E-1,1.4725155E-1,1.2010673E-2,5.3282645E-2,6.048286E-2,2.862822E-2,8.195888E-3,1.3148041E-2,3.074038E-2,2.4223559E-2,4.846832E-2,5.6599863E-2,1.1298776E-2,3.0286727E-2,1.423116E-2,1.3085239E-2,4.6845436E-1,1.3886327E-1,2.3980358E-1,9.683244E-2,2.075176E-2,5.7400793E-2,3.565387E-2,5.9112467E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,-1,-1,-1,-1,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.147158E4,4.427818E7,1.6007428E7,2.44E2,2.022131E6,4.435876E2,3.596E3,3.984E3,3.13276E5,5.899652E-5,2.558E3,9.750042E6,2.4704508E7,4.44E2,1.1891E4,7.785714E0,7.338571E2,1.8388145E-7,7.31E2,5.912179E3,-1.0252608E-2,2E1,1.4595416E3,4.16E2,4.29E2,1.0330111E-2,2.8108987E-3,-1.5146548E-2,-5.23202E-3,2.224E3,1.1E1,5.04E2,6.0921145E4,1.3E2,5.7018555E4,9.443621E-10,6.5E1,1E0,7.402E3,7.836E3,5.25E2,1.34214E5,7.6588124E-2,1.380863E6,1.5364104E3,3.8940645E6,3.051837E2,1.23E2,6.149315E6,1E0,1.2562814E0,1.2544625E5,4.4910244E3,-3.560274E-3,3.780995E-4,4.7727013E-3,-5.0516415E-4,-3.2834325E-3,-1.0593305E-2,2.2307562E-3,-1.5991271E-3,-1.2414559E-4,-3.4757892E-3,5.046947E-4,4.999444E-3,1.076209E-2,3.7400343E-3,-8.1782945E-4,3.7913313E-3,-4.012377E-3,-9.143953E-3,1.8051737E-3,-4.0278267E-3,5.7727494E-3,1.0928414E-3,-1.8589721E-3,2.8218788E-3,-3.3597238E-3,-7.3925653E-3,2.610734E-3,-1.2298628E-3,-2.9527053E-4,-1.852622E-2,9.545952E-3,9.040232E-4,9.13387E-4,5.4344903E-3,-1.6894399E-3,3.4557618E-3,3.509488E-3,-6.2844675E-4,-6.086858E-4,-7.99384E-3,-4.332546E-3,1.0762257E-3,1.475993E-3,6.6756327E-3],"split_indices":[40,43,63,2,12,68,2,9,43,49,10,9,1,0,41,64,45,49,2,45,0,3,4,67,0,0,0,0,0,41,3,7,57,2,57,49,0,13,1,41,2,1,54,9,4,63,64,0,57,8,65,45,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.79E2,1.72E2,8.07E2,9.7E1,7.5E1,7.46E2,6.1E1,6.5E1,3.2E1,3.7E1,3.8E1,7.4E2,6E0,5E0,5.6E1,2.2E1,4.3E1,1.4E1,1.8E1,3.4E1,3E0,1.7E1,2.1E1,5.99E2,1.41E2,4E0,2E0,3E0,2E0,2.2E1,3.4E1,7E0,1.5E1,2.8E1,1.5E1,9E0,5E0,8E0,1E1,2.1E1,1.3E1,7E0,1E1,1.5E1,6E0,3.46E2,2.53E2,9.9E1,4.2E1,9E0,1.3E1,1.2E1,2.2E1,5E0,2E0,8E0,7E0,2.6E1,2E0,6E0,9E0,2E0,7E0,3E0,2E0,4E0,4E0,3E0,7E0,1.4E1,7E0,5E0,8E0,5E0,2E0,6E0,4E0,1.2E1,3E0,2E0,4E0,3.44E2,2E0,3E0,2.5E2,5.1E1,4.8E1,3.3E1,9E0,5E0,4E0,3E0,1E1,9E0,3E0,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-2.605089E-3,-4.3416047E-3,4.8180413E-3,-9.827388E-3,2.6897391E-2,-1.15914345E-2,1.0355611E-1,4.148565E-2,-2.9499756E-2,-8.68785E-3,-9.44481E-2,1.2803197E-1,7.568341E-4,-6.3755945E-3,6.257565E-2,5.381751E-3,-5.5568043E-2,3.9180283E-2,-1.2467088E-2,-1.1779285E-2,-4.8079498E-2,3.111655E-3,8.027427E-3,1.3338272E-2,-1.075926E-2,7.3881656E-2,-5.12401E-2,-1.0707053E-1,1.7732605E-2,-1.1129143E-3,4.4661793E-3,-1.2699032E-3,2.2646599E-4,-5.68334E-3,9.018715E-4,-1.4680794E-3,1.9088988E-3,3.6857498E-3,-4.8491065E-3,7.628921E-4,-4.3759733E-3,-6.8889502E-3,-2.085761E-3,3.8251164E-3,-1.7015179E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,-1,-1,35,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.878797E-1,1.6974965E-1,0E0,1.696119E-1,1.22988746E-1,1.988619E-1,2.5597602E-2,1.1950742E-1,1.250363E-1,1.4575522E-1,1.9800422E-1,1.633647E-2,0E0,1.6933574E-1,1.09303415E-1,0E0,1.0382811E-1,2.1955222E-1,1.9689588E-1,0E0,1.20258175E-1,0E0,0E0,4.6073925E-2,0E0,9.4108194E-2,2.5482507E-2,3.5525173E-2,1.7169882E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,20,20,23,23,25,25,26,26,27,27,28,28],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,-1,-1,36,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5847838E5,3.086474E6,4.8180413E-3,3.8771296E3,1.647E4,1.7493458E7,1.213274E9,4.8581E4,3.5714287E-1,1E0,4.043435E8,3.19E2,7.568341E-4,6.5590655E6,2E0,5.381751E-3,7.271717E6,4.622222E1,7.969174E7,-1.1779285E-2,1.5017116E5,3.111655E-3,8.027427E-3,1.185E3,-1.075926E-2,4.9036694E0,3.8807898E6,2.2171207E11,1.1967312E3,-1.1129143E-3,4.4661793E-3,-1.2699032E-3,2.2646599E-4,-5.68334E-3,9.018715E-4,-1.4680794E-3,1.9088988E-3,3.6857498E-3,-4.8491065E-3,7.628921E-4,-4.3759733E-3,-6.8889502E-3,-2.085761E-3,3.8251164E-3,-1.7015179E-4],"split_indices":[45,40,0,64,41,63,7,1,65,20,12,0,0,40,6,0,57,4,7,0,45,0,0,41,0,50,40,43,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.003E3,9.88E2,1.5E1,8.41E2,1.47E2,8.29E2,1.2E1,1.17E2,3E1,8.02E2,2.7E1,9E0,3E0,3.6E1,8.1E1,4E0,2.6E1,5.8E1,7.44E2,5E0,2.2E1,5E0,4E0,3.4E1,2E0,7.4E1,7E0,1.5E1,1.1E1,2.8E1,3E1,3.97E2,3.47E2,1E1,1.2E1,1.3E1,2.1E1,7.2E1,2E0,3E0,4E0,8E0,7E0,2E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.9627488E-4,-4.476878E-3,5.089908E-2,-1.0499842E-2,3.271462E-2,7.940047E-2,-2.5259033E-2,-5.5255152E-2,-6.6754045E-3,6.735259E-2,-4.0581734E-3,5.1564384E-2,1.2831903E-1,7.089962E-2,-9.728422E-2,-4.3241978E-2,-1.0180222E-2,6.470813E-2,-1.1720372E-2,7.89332E-3,9.825418E-2,-6.384341E-2,2.0723948E-2,7.6393455E-2,-3.692856E-2,1.6125649E-1,5.875574E-2,4.417398E-3,4.1087632E-4,-1.4514579E-1,2.1445784E-3,-8.343018E-2,-2.727675E-4,9.591075E-2,5.6419047E-3,-3.6612544E-2,-2.96343E-3,9.293661E-2,-1.9377893E-2,1.19433194E-1,-5.57851E-2,6.5149465E-3,-1.15605384E-1,4.2136386E-2,-4.3456E-2,9.49294E-3,1.0028942E-1,-3.801959E-3,-9.63956E-6,1.9219164E-3,1.9289622E-1,-1.2946414E-3,4.2456435E-3,-3.246508E-3,-9.4377315E-3,-4.357539E-3,-2.2816779E-4,5.953822E-4,-5.682187E-3,5.7444377E-3,3.448915E-4,-3.4727554E-3,2.94969E-3,-2.0363864E-3,2.081439E-3,9.186784E-4,-1.103813E-3,1.0987811E-3,7.072403E-3,-3.0146865E-3,3.1828354E-3,4.0885787E-3,7.965591E-3,-4.4832453E-3,8.5306866E-4,-4.2632194E-3,3.2910088E-3,-6.0957405E-3,-1.1825366E-3,3.7986098E-3,-3.5352583E-5,-1.8767109E-4,-7.3584346E-3,2.8293943E-3,-1.2322247E-3,2.034384E-3,5.4438235E-3,1.0521687E-2,3.7371037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,-1,-1,53,-1,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2069655E-1,2.0889038E-1,1.7131336E-1,1.369046E-1,1.6585886E-1,7.100028E-2,1.5926601E-1,1.16340175E-1,2.6725283E-1,1.22329324E-1,9.6145794E-2,8.63146E-2,3.635001E-2,1.479793E-2,9.635824E-2,1.0358856E-1,0E0,8.976692E-2,1.507332E-1,5.746253E-2,1.502082E-1,6.994904E-2,6.504655E-2,4.7007516E-2,1.5133242E-2,4.1045368E-2,2.6029943E-2,0E0,0E0,3.039971E-2,0E0,2.735497E-2,5.0614838E-2,8.15348E-2,9.0523876E-2,1.15897E-1,2.4868104E-1,2.1592569E-2,8.248047E-2,4.8612356E-2,2.0471549E-2,6.4880416E-2,1.349394E-2,6.0980845E-2,5.450218E-2,1.8685574E-2,1.721017E-2,0E0,0E0,0E0,2.1711588E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,-1,-1,54,-1,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6975454E3,9.750085E6,2.5842668E7,3.175E1,1.058E3,3.4915986E3,3.1879792E8,2.4636364E1,1.2E1,1.118E3,1.2E1,3E0,1.159499E6,2.4180895E6,3.3748108E7,1.63757E4,-1.0180222E-2,3.5318289E3,1.610753E3,2.5818555E2,1.6485592E0,1.2544625E5,4.4744192E8,1E0,1.767635E3,2.596275E3,2.1511E4,4.417398E-3,4.1087632E-4,1.3784861E5,2.1445784E-3,8E0,4.5E0,1.8002975E5,1.3E2,2.22E3,3.72381E5,6.2E1,3.8580435E2,7.4654E4,1.5E1,6.5062125E5,1.711E4,1.1203416E1,2.5503825E6,2.880929E6,3.7827366E8,-3.801959E-3,-9.63956E-6,1.9219164E-3,5.8E1,-1.2946414E-3,4.2456435E-3,-3.246508E-3,-9.4377315E-3,-4.357539E-3,-2.2816779E-4,5.953822E-4,-5.682187E-3,5.7444377E-3,3.448915E-4,-3.4727554E-3,2.94969E-3,-2.0363864E-3,2.081439E-3,9.186784E-4,-1.103813E-3,1.0987811E-3,7.072403E-3,-3.0146865E-3,3.1828354E-3,4.0885787E-3,7.965591E-3,-4.4832453E-3,8.5306866E-4,-4.2632194E-3,3.2910088E-3,-6.0957405E-3,-1.1825366E-3,3.7986098E-3,-3.5352583E-5,-1.8767109E-4,-7.3584346E-3,2.8293943E-3,-1.2322247E-3,2.034384E-3,5.4438235E-3,1.0521687E-2,3.7371037E-3],"split_indices":[64,9,59,67,10,4,44,64,10,41,8,65,9,40,55,40,0,59,45,4,50,45,7,27,64,64,2,0,0,45,0,10,70,57,2,10,9,0,4,1,3,40,41,68,44,1,7,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.007E3,9.3E2,7.7E1,8.01E2,1.29E2,5.6E1,2.1E1,6.2E1,7.39E2,6.6E1,6.3E1,3.7E1,1.9E1,9E0,1.2E1,5.9E1,3E0,4.8E1,6.91E2,2.3E1,4.3E1,1.8E1,4.5E1,2.9E1,8E0,1.2E1,7E0,6E0,3E0,9E0,3E0,3E1,2.9E1,3.1E1,1.7E1,1.79E2,5.12E2,5E0,1.8E1,3.8E1,5E0,8E0,1E1,3.4E1,1.1E1,8E0,2.1E1,3E0,5E0,3E0,9E0,2E0,5E0,5E0,4E0,2.6E1,4E0,2.7E1,2E0,2.3E1,8E0,7E0,1E1,1.64E2,1.5E1,2.45E2,2.67E2,3E0,2E0,1.2E1,6E0,2.6E1,1.2E1,3E0,2E0,3E0,5E0,8E0,2E0,1.7E1,1.7E1,9E0,2E0,3E0,5E0,6E0,1.5E1,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-2.561578E-3,-1.6282525E-2,1.7686557E-2,-1.1383709E-2,-8.791822E-2,2.8763115E-2,-3.6507796E-2,-5.464635E-3,-7.2878875E-2,-1.5152259E-1,-4.3502275E-4,1.3488384E-1,2.0534169E-2,-1.3908862E-1,-2.0774433E-2,-2.6647314E-2,1.3099926E-2,-8.089682E-3,-2.0600097E-1,2.7868964E-4,-1.7672458E-1,6.325782E-2,-1.277482E-1,1.5852498E-1,6.1284495E-4,4.4778466E-2,-8.230169E-3,-2.8385343E-3,-8.660865E-3,6.1068997E-2,-4.206403E-2,-2.0872448E-2,-2.1506108E-1,1.9344475E-2,-1.2433106E-1,-9.006912E-2,5.3799383E-2,-1.364574E-1,-2.1214435E-2,-5.4812245E-3,-1.1056593E-2,4.8732297E-3,8.0385426E-4,-5.17E-4,-8.41376E-3,4.907521E-3,9.637299E-3,7.921892E-2,3.1407725E-3,-1.3739551E-2,8.63665E-3,7.955023E-2,-1.2425344E-3,-6.416277E-2,5.0537325E-2,-8.371856E-3,-6.9204933E-4,-2.382347E-2,4.6475878E-4,-1.3472232E-3,1.3708896E-3,-1.559258E-4,-6.806084E-3,-3.133087E-4,-1.1721764E-2,5.065085E-3,-1.2621378E-3,-7.1195206E-3,-7.7118125E-4,1.3460672E-3,5.129351E-3,-1.5658945E-2,7.661731E-4,-3.840621E-3,-1.6008824E-4,2.007297E-4,4.398925E-3,-3.531575E-3,3.1961172E-3,3.3934475E-3,-1.3538914E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,69,71,73,-1,75,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7783376E-1,2.0815356E-1,2.4356605E-1,2.030746E-1,2.1137798E-1,2.9064626E-1,1.0847247E-1,2.0129836E-1,4.2095226E-1,8.844715E-2,1.4593305E-1,6.6845804E-2,2.186457E-1,2.275981E-2,1.0885603E-1,2.555585E-1,2.366922E-1,1.7734914E-1,2.2167993E-1,0E0,4.105389E-2,2.1665681E-2,3.6682688E-2,3.0600071E-2,0E0,2.4353471E-1,1.5828912E-1,0E0,0E0,2.3323558E-2,1.0351491E-1,2.155698E-1,4.8381925E-1,1.34806E-1,3.4105748E-2,2.0446216E-1,9.4834164E-2,2.8467596E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4737988E-1,3.69055E-1,1.0073902E-1,0E0,1.2995996E-2,0E0,7.342757E-2,1.2702877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,47,47,48,48,49,49,51,51,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,70,72,74,-1,76,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,1E0,1E0,2.2386675E6,3.653E3,1.2083E4,5E1,1.5292561E2,6E0,4.1943024E7,3.0977E4,1.562E5,8E0,4.9182608E2,8E1,1.2600376E7,3.0084E4,2.7311892E6,4.5158855E6,2.7868964E-4,4.831007E6,7.82E2,1.0366492E0,2.5176924E1,6.1284495E-4,2.115238E5,4.91027E0,-2.8385343E-3,-8.660865E-3,1.0564016E3,1.4433751E0,3.964269E3,4.07E2,3.3852024E7,1.027972E0,1.453785E6,1.2783505E0,2.7863942E8,-2.1214435E-2,-5.4812245E-3,-1.1056593E-2,4.8732297E-3,8.0385426E-4,-5.17E-4,-8.41376E-3,4.907521E-3,9.637299E-3,7.0093E4,1.26E2,4E0,8.63665E-3,5.3E1,-1.2425344E-3,6.355E3,9.69239E5,-8.371856E-3,-6.9204933E-4,-2.382347E-2,4.6475878E-4,-1.3472232E-3,1.3708896E-3,-1.559258E-4,-6.806084E-3,-3.133087E-4,-1.1721764E-2,5.065085E-3,-1.2621378E-3,-7.1195206E-3,-7.7118125E-4,1.3460672E-3,5.129351E-3,-1.5658945E-2,7.661731E-4,-3.840621E-3,-1.6008824E-4,2.007297E-4,4.398925E-3,-3.531575E-3,3.1961172E-3,3.3934475E-3,-1.3538914E-4],"split_indices":[4,82,6,59,41,9,0,64,68,7,41,10,29,64,0,57,41,63,59,0,57,2,65,68,0,45,47,0,0,4,50,40,0,7,65,9,65,7,0,0,0,0,0,0,0,0,0,1,10,8,0,0,0,2,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,5.95E2,4.03E2,5.58E2,3.7E1,3.35E2,6.8E1,5.1E2,4.8E1,2.1E1,1.6E1,2.3E1,3.12E2,8E0,6E1,2.38E2,2.72E2,3.3E1,1.5E1,3E0,1.8E1,1.1E1,5E0,1.9E1,4E0,1.69E2,1.43E2,4E0,4E0,1.2E1,4.8E1,2.32E2,6E0,2.61E2,1.1E1,1.4E1,1.9E1,1.3E1,2E0,1.1E1,7E0,5E0,6E0,2E0,3E0,1.1E1,8E0,9.2E1,7.7E1,1.4E2,3E0,1E1,2E0,3.9E1,9E0,7E0,2.25E2,2E0,4E0,4.6E1,2.15E2,2E0,9E0,1E1,4E0,1.1E1,8E0,1.1E1,2E0,3.7E1,5.5E1,2E0,7.5E1,1.7E1,1.23E2,2E0,8E0,3.6E1,3E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-1.9304756E-3,-4.6898262E-4,-1.6312976E-1,-2.4434002E-3,7.615718E-2,-1.4077784E-2,-4.036473E-2,-3.2146752E-4,-8.6069524E-2,3.5720903E-2,1.4407492E-1,8.422834E-4,-4.831074E-3,-9.105333E-3,1.8108478E-2,-4.8168298E-2,-1.5585093E-1,8.426155E-2,-6.6837072E-3,1.4658957E-3,7.8800805E-3,-6.3316557E-3,-1.2462295E-1,-1.5341899E-1,2.2050424E-2,-2.1815859E-2,-4.917017E-3,-9.620508E-3,1.5174983E-4,1.0810971E-1,3.8940975E-4,-1.6752004E-3,2.1641182E-3,1.3871266E-3,-5.999531E-4,-2.145029E-3,-1.9325498E-2,-7.5780926E-4,-1.5179452E-2,2.23561E-3,-3.2741483E-4,3.4881037E-4,-3.233153E-3,6.5591355E-4,6.9565345E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,23,25,27,29,31,-1,-1,33,35,37,39,41,-1,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3598632E-1,1.5057038E-1,1.4603041E-1,1.7209788E-1,6.351203E-2,0E0,2.4757635E-2,1.5347107E-1,5.596815E-2,3.555508E-2,2.2270665E-2,0E0,0E0,2.0491326E-1,2.1028547E-1,2.404353E-2,6.89404E-2,1.3541821E-2,1.7744292E-2,0E0,0E0,1.5479827E-1,3.264698E-1,1.6425017E-1,2.3338787E-1,1.910616E-2,0E0,0E0,0E0,2.2239245E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,29,29],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,24,26,28,30,32,-1,-1,34,36,38,40,42,-1,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8304348E1,2.9905329E0,6.042857E1,4.24038E8,1.3E1,-1.4077784E-2,1.6E1,5.372237E2,6.619098E8,1.4839433E5,4.73909E0,8.422834E-4,-4.831074E-3,4.5158855E6,6.6903913E-1,2.1356704E0,2.0120485E0,4.1885E4,3.774648E0,1.4658957E-3,7.8800805E-3,1.0366492E0,3.18E3,1.921E3,9.122396E7,1.2516639E2,-4.917017E-3,-9.620508E-3,1.5174983E-4,3.97371E5,3.8940975E-4,-1.6752004E-3,2.1641182E-3,1.3871266E-3,-5.999531E-4,-2.145029E-3,-1.9325498E-2,-7.5780926E-4,-1.5179452E-2,2.23561E-3,-3.2741483E-4,3.4881037E-4,-3.233153E-3,6.5591355E-4,6.9565345E-3],"split_indices":[65,47,65,44,3,0,0,4,7,45,47,0,0,59,65,50,47,2,66,0,0,65,41,2,12,70,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.001E3,9.93E2,8E0,9.69E2,2.4E1,3E0,5E0,9.46E2,2.3E1,1.6E1,8E0,3E0,2E0,6.41E2,3.05E2,1.6E1,7E0,7E0,9E0,2E0,6E0,6.27E2,1.4E1,6E0,2.99E2,1.2E1,4E0,5E0,2E0,5E0,2E0,6E0,3E0,9.7E1,5.3E2,1.2E1,2E0,4E0,2E0,1.56E2,1.43E2,8E0,4E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.1246781E-3,6.4419764E-3,-3.140376E-2,1.1328572E-3,4.5930494E-2,-1.3779782E-1,-1.7308567E-2,4.947498E-3,-9.321991E-2,7.496331E-2,-2.4311518E-2,-1.0951096E-2,-9.774728E-3,2.2097724E-2,-3.9536092E-2,9.780108E-3,-5.012957E-2,-1.4427812E-1,5.8842666E-2,8.488489E-2,-2.8170082E-3,-9.14936E-2,1.38542E-2,-5.3965817E-3,5.5163566E-2,3.4722794E-2,-5.327598E-3,-1.395429E-1,-2.9404808E-2,2.8355506E-3,5.8023166E-2,-1.3531876E-1,5.6893476E-3,1.1835308E-4,-1.7705569E-1,-2.8482277E-3,5.0277603E-3,7.113903E-2,1.3737653E-1,-2.5037026E-3,1.4750507E-3,9.362063E-5,-1.4523156E-1,6.237724E-2,-2.3692599E-2,-1.1622371E-4,4.339137E-3,4.6028454E-2,-5.8605713E-3,-9.887614E-3,-3.040621E-3,-5.3644422E-2,2.231959E-2,4.3473447E-5,6.3070115E-3,3.391485E-4,4.9275844E-3,-2.5435726E-3,-1.0734418E-2,3.9232336E-3,-1.104622E-3,-1.7595896E-2,-4.171748E-3,5.0173416E-3,2.1935839E-3,7.7349558E-3,3.595719E-3,-2.1500215E-3,-9.523721E-3,4.109181E-3,-9.928391E-4,-2.7591838E-3,1.0425723E-3,4.4284123E-3,1.0836336E-3,-3.1938644E-3,2.4759232E-3,2.13074E-3,-7.42404E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,-1,45,47,-1,49,51,53,55,57,59,-1,61,-1,-1,63,65,-1,-1,-1,67,69,71,-1,-1,73,-1,-1,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6987942E-1,1.7689194E-1,2.0456591E-1,2.6860303E-1,2.0575482E-1,1.4991176E-1,1.08609885E-1,1.9133127E-1,2.3364699E-1,5.5030048E-2,7.818914E-2,5.5788603E-2,0E0,8.247899E-2,7.593936E-2,2.2095308E-1,2.7656966E-1,1.0635421E-1,5.5727754E-2,3.5684347E-2,1.8001588E-2,5.558434E-2,3.7915573E-2,0E0,1.1577654E-2,8.294687E-2,0E0,2.4627537E-2,9.272322E-2,1.467037E-1,2.0624211E-1,1.6795531E-1,8.740757E-2,0E0,2.8500038E-1,0E0,0E0,4.1101635E-2,8.817032E-3,0E0,0E0,0E0,3.30884E-2,2.2409014E-2,2.1615451E-2,0E0,0E0,4.2942956E-2,0E0,0E0,0E0,8.9200765E-2,1.1376307E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,34,34,37,37,38,38,42,42,43,43,44,44,47,47,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,-1,46,48,-1,50,52,54,56,58,60,-1,62,-1,-1,64,66,-1,-1,-1,68,70,72,-1,-1,74,-1,-1,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0923106E-10,1.4743204E3,1.6379E2,3.1E1,1.3475722E2,9.624326E7,1.0975045E7,4.7E3,1.1046409E7,3.913718E5,1.3655363E2,1.88E2,-9.774728E-3,3.5510652E6,1.28636E7,6.698333E2,4.31E3,4.327E3,3.7E1,4.0277927E3,3.4E1,2.1391304E1,2.101E3,-5.3965817E-3,2E0,1.779855E6,-5.327598E-3,8.487615E-7,5.159652E-7,3.7725E1,3.515484E1,1.65915E5,8.818731E0,1.1835308E-4,6.845E3,-2.8482277E-3,5.0277603E-3,1.9001543E1,4.7621E4,-2.5037026E-3,1.4750507E-3,9.362063E-5,1.6677892E2,1.9E1,1.01E3,-1.1622371E-4,4.339137E-3,3.6828358E0,-5.8605713E-3,-9.887614E-3,-3.040621E-3,1.465E4,1.669031E2,4.3473447E-5,6.3070115E-3,3.391485E-4,4.9275844E-3,-2.5435726E-3,-1.0734418E-2,3.9232336E-3,-1.104622E-3,-1.7595896E-2,-4.171748E-3,5.0173416E-3,2.1935839E-3,7.7349558E-3,3.595719E-3,-2.1500215E-3,-9.523721E-3,4.109181E-3,-9.928391E-4,-2.7591838E-3,1.0425723E-3,4.4284123E-3,1.0836336E-3,-3.1938644E-3,2.4759232E-3,2.13074E-3,-7.42404E-3],"split_indices":[49,67,64,3,70,43,57,2,57,45,68,2,0,40,57,67,10,1,3,4,0,68,0,0,8,9,0,49,49,68,70,1,68,0,41,0,0,68,2,0,0,0,70,3,8,0,0,66,0,0,0,41,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E2,8.43E2,1.37E2,7.44E2,9.9E1,1.5E1,1.22E2,7.16E2,2.8E1,7E1,2.9E1,6E0,9E0,4.4E1,7.8E1,6.59E2,5.7E1,2.1E1,7E0,6.2E1,8E0,1E1,1.9E1,2E0,4E0,4.1E1,3E0,6E0,7.2E1,5.77E2,8.2E1,2.2E1,3.5E1,4E0,1.7E1,2E0,5E0,5.1E1,1.1E1,3E0,5E0,4E0,6E0,8E0,1.1E1,2E0,2E0,3.9E1,2E0,2E0,4E0,4.9E1,2.3E1,5.7E2,7E0,4.1E1,4.1E1,1.3E1,9E0,9E0,2.6E1,4E0,1.3E1,1.8E1,3.3E1,6E0,5E0,3E0,3E0,6E0,2E0,6E0,5E0,1.1E1,2.8E1,4.3E1,6E0,2.1E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-1.4829445E-3,-1.513688E-1,-1.0492857E-4,-8.7957765E-4,-8.562099E-3,-1.3716954E-3,7.0724804E-3,-2.4630965E-4,-1.5663974E-1,1.6376855E-2,-1.2460022E-2,-1.5280554E-3,-1.1062681E-2,-5.9776106E-3,3.9424907E-2,-4.268845E-2,2.8914742E-3,1.2142773E-3,-1.5501606E-3,2.3741806E-3,-3.157607E-3,-2.3208936E-3,4.045334E-3,7.3343934E-4,-1.992987E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0405065E-1,3.812833E-2,1.914955E-1,0E0,0E0,1.6998E-1,0E0,1.965346E-1,6.471267E-2,2.1147871E-1,2.5906423E-1,0E0,0E0,1.8903431E-1,2.7154186E-1,1.9720566E-1,2.2522585E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2E1,1.3888889E1,6.7473444E2,-8.7957765E-4,-8.562099E-3,7.9E1,7.0724804E-3,2.01184E5,1.45064E5,1E0,1.212945E6,-1.5280554E-3,-1.1062681E-2,2.5493E4,1E0,3.8248518E-1,3.9E2,1.2142773E-3,-1.5501606E-3,2.3741806E-3,-3.157607E-3,-2.3208936E-3,4.045334E-3,7.3343934E-4,-1.992987E-3],"split_indices":[2,4,68,0,0,3,0,9,7,27,9,0,0,1,82,46,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.87E2,8E0,9.79E2,2E0,6E0,9.72E2,7E0,9.66E2,6E0,4.09E2,5.57E2,3E0,3E0,2.08E2,2.01E2,1.87E2,3.7E2,9.6E1,1.12E2,1.81E2,2E1,1.77E2,1E1,2.89E2,8.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_ObjectHashAggregate","sqlOp_Project","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_onprem","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"83"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-4.5668036E-3,-2.3331916E-1,6.575819E-1,-7.3830986E-1,-2.9545367E-2,7.585778E-1,-9.047681E-1,-9.8671955E-1,-5.689058E-1,-2.1748337E-1,1.7790031E-1,4.7353148E-1,9.98321E-1,-5.0334964E-2,-1.9184373E-2,-1.009283E0,-1.9581432E-2,-7.572694E-1,-2.6597184E-1,-3.4746066E-1,2.4795938E-2,3.172561E-1,1.9483695E-2,3.2106972E-1,8.287864E-1,8.6895436E-2,8.755892E-1,-4.7518518E-2,-2.757553E-2,-6.920537E-1,-1.1876076E0,3.4099087E-2,-4.1624886E-1,-5.5970305E-1,-2.3217718E-1,-1.1214657E-2,3.8214678E-1,2.0068753E-1,4.5767924E-1,7.318743E-2,-3.5774136E-1,-6.0188014E-2,4.028545E-1,8.6816084E-1,2.0080749E-2,5.641912E-1,1.1029484E0,-3.3124804E-2,-1.3965118E-2,-6.606181E-2,-3.2697614E-2,-7.3586625E-1,-2.912001E-1,-5.0898236E-1,-3.7304517E-2,-1.3420476E-1,-4.1529045E-1,-2.2086361E-1,2.3870613E-2,2.2889538E-2,8.889135E-3,3.0159533E-1,-9.188867E-2,5.2135813E-1,-9.34718E-3,1.5662609E-2,2.2340204E-1,-2.1273324E-1,-2.7960172E-2,-1.0369004E-2,4.438314E-2,2.2642371E-1,5.3501856E-1,2.715325E-2,4.549749E-2,6.6693574E-1,1.7469372E-1,1.2068436E0,6.308485E-1,-3.998909E-2,-2.1338705E-2,-4.135335E-1,-1.0867557E-1,-5.4981214E-1,-1.2486093E-2,-1.0617143E-2,-2.5443727E-1,-8.711125E-2,-5.048686E-1,8.628427E-3,-1.9532233E-2,-1.7093034E-1,5.5031493E-2,6.1803424E-1,2.2785096E-1,1.0172568E-1,-1.824982E-2,4.6515972E-3,5.708142E-1,-8.626919E-2,7.5669944E-2,1.1983855E-1,3.8197517E-1,-1.3491115E-2,-2.7342E-3,-1.4498445E-3,5.8407313E-3,3.234492E-1,-9.9598905E-3,6.387942E-1,2.3165841E-1,7.3940027E-1,1.0149398E-2,-7.78564E-3,1.6972052E-2,1.2719901E0,2.2384929E-2,3.457816E-2,1.2448564E-2,-1.0324918E-2,-4.790183E-1,-8.777231E-3,-1.1875937E-2,-5.716248E-1,-1.1855732E-2,-4.4726193E-2,1.0953399E-2,-3.516006E-1,-8.671726E-2,3.5364935E-4,-8.473294E-3,-5.602632E-1,-4.3281703E-3,-1.1312302E-2,-1.5357796E-3,1.956537E-1,2.4461351E-2,3.5201266E-2,1.8253269E-2,7.594841E-2,2.8369758E-1,-2.4591943E-2,1.4264061E-2,4.69715E-1,3.320946E-2,8.841761E-3,-1.5208796E-1,-9.6291214E-2,1.2223156E-1,1.9617124E-1,1.3227642E-2,1.0226706E-2,2.4505768E-2,2.6079942E-2,2.1761277E-1,6.965718E-1,1.1330703E-2,4.0160073E-3,1.3949478E-2,3.624573E-2,1.4810829E-2,6.3704684E-2,4.4903275E-2,-2.6027502E-2,-9.909527E-3,-3.1908199E-3,1.9831837E-3,-1.4045271E-2,-5.9957147E-1,4.5774165E-3,-8.285467E-2,-5.9024906E-3,-1.7932279E-2,-9.371145E-3,3.2423376E-3,-6.361793E-1,-1.116317E-2,5.031539E-3,1.2207902E-2,-1.4691992E-1,6.4051405E-2,-1.1300913E-3,5.477288E-3,2.2185285E-1,1.9904312E-2,3.4397193E-3,-5.4847435E-3,2.4269028E-2,1.0960829E-2,-6.01094E-2,-1.3799966E-2,-9.214199E-3,3.1130412E-3,1.6172273E-1,-5.6327586E-3,1.2809288E-3,1.3326791E-2,-3.5169765E-3,4.6169646E-3,1.0040688E-1,1.6919017E-2,1.1915705E-2,3.5820182E-2,-2.899007E-2,-1.3642825E-2,-8.5165E-3,-4.8935594E-4,-3.4403258E-3,3.116593E-3,-2.0048281E-2,-3.329749E-2,-4.7226588E-4,-1.1971596E-2,-9.0992794E-4,5.181902E-3,4.1945907E-3,1.27073815E-2,-5.0962237E-3,3.344545E-3,1.1613483E-2,3.99133E-3,8.124161E-3,-1.8657715E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,35,37,39,41,43,-1,45,-1,-1,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,-1,-1,-1,-1,79,81,83,-1,85,87,89,91,-1,-1,93,95,97,-1,99,101,103,-1,-1,105,107,109,-1,-1,111,113,115,117,-1,-1,119,121,123,-1,125,127,129,131,-1,-1,133,135,137,139,141,-1,-1,143,145,147,149,151,-1,-1,-1,-1,153,-1,155,157,159,-1,-1,-1,161,-1,-1,-1,-1,163,165,-1,167,-1,169,-1,171,173,-1,-1,175,-1,-1,-1,177,179,-1,-1,181,183,185,-1,187,-1,-1,189,191,193,195,197,-1,-1,-1,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,203,-1,205,-1,-1,-1,207,209,-1,-1,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,217,-1,-1,-1,219,-1,-1,-1,-1,-1,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5101564E2,7.622995E1,4.108082E1,8.402031E0,2.0662775E1,1.5996719E1,9.946165E-1,6.51741E-1,7.1595535E0,8.763061E0,5.5570397E0,5.8001347E0,1.2434647E1,0E0,0E0,3.0021667E-2,0E0,1.3952293E0,8.002494E0,4.298765E0,1.2653905E0,2.1015854E0,2.4369311E0,2.518735E0,1.3476181E-1,0E0,7.590439E0,0E0,0E0,4.7367477E-1,3.4502983E-2,0E0,1.5842652E0,3.807373E-1,2.0791326E0,6.6727525E-1,9.815848E-2,2.2419546E0,2.627181E0,9.038132E-1,4.0952802E-1,2.6760477E-1,1.4178333E0,3.6981392E-1,0E0,1.9244442E0,2.475891E0,0E0,0E0,0E0,0E0,2.6675224E-2,7.1423984E-1,3.8747025E-1,0E0,1.1711794E0,1.1613255E0,1.1489779E0,4.8287967E-1,0E0,0E0,1.1659727E0,1.2140188E0,1.0867891E0,0E0,4.775105E-1,4.304738E-1,1.1663926E-1,0E0,0E0,6.6127405E-2,1.356319E0,1.0168295E0,0E0,0E0,1.1339455E0,8.554605E-1,1.8776474E0,3.9338207E-1,0E0,0E0,1.4184737E-1,2.2009441E-1,1.4748955E-1,0E0,3.4856838E-1,6.2135935E-1,9.266107E-2,6.763859E-1,0E0,0E0,1.0383153E-1,2.8753528E-1,1.9785881E-2,3.7716627E-1,3.4755725E-1,0E0,0E0,3.9562225E-1,5.5907613E-1,4.0410244E-1,1.520252E-1,1.4487875E-1,0E0,0E0,0E0,0E0,5.14987E-1,0E0,4.4307423E-1,8.564776E-2,3.914795E-1,0E0,0E0,0E0,1.9971466E-1,0E0,0E0,0E0,0E0,1.9156551E-1,3.413704E-2,0E0,1.3182831E-1,0E0,2.0560172E-1,0E0,1.5495229E-1,1.6780338E-1,0E0,0E0,5.235777E-1,0E0,0E0,0E0,4.2620093E-2,3.9615512E-1,0E0,0E0,6.27986E-2,2.1381283E-1,9.2746355E-2,0E0,2.889619E-1,0E0,0E0,3.0570364E-1,1.9916046E-1,3.95755E-1,1.6301173E-1,7.733794E-2,0E0,0E0,0E0,2.9416418E-1,5.9475803E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2915039E-1,0E0,2.1072836E-1,0E0,0E0,0E0,5.492708E-2,6.3435555E-2,0E0,0E0,0E0,1.6607939E-1,1.915888E-1,0E0,0E0,1.56322E-1,0E0,0E0,0E0,0E0,0E0,1.1441429E-1,0E0,0E0,0E0,2.0790786E-1,0E0,0E0,0E0,0E0,0E0,1.3456574E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,51,51,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,63,63,65,65,66,66,67,67,70,70,71,71,72,72,75,75,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,95,95,98,98,99,99,100,100,101,101,102,102,107,107,109,109,110,110,111,111,115,115,120,120,121,121,123,123,125,125,127,127,128,128,131,131,135,135,136,136,139,139,140,140,141,141,143,143,146,146,147,147,148,148,149,149,150,150,154,154,155,155,168,168,170,170,174,174,175,175,179,179,180,180,183,183,189,189,193,193,199,199],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,36,38,40,42,44,-1,46,-1,-1,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,-1,-1,-1,-1,80,82,84,-1,86,88,90,92,-1,-1,94,96,98,-1,100,102,104,-1,-1,106,108,110,-1,-1,112,114,116,118,-1,-1,120,122,124,-1,126,128,130,132,-1,-1,134,136,138,140,142,-1,-1,144,146,148,150,152,-1,-1,-1,-1,154,-1,156,158,160,-1,-1,-1,162,-1,-1,-1,-1,164,166,-1,168,-1,170,-1,172,174,-1,-1,176,-1,-1,-1,178,180,-1,-1,182,184,186,-1,188,-1,-1,190,192,194,196,198,-1,-1,-1,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,204,-1,206,-1,-1,-1,208,210,-1,-1,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,218,-1,-1,-1,220,-1,-1,-1,-1,-1,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.9661028E7,9.9E1,9.31E2,1.1598715E3,1.665909E3,1.4912975E-1,6.76E2,1.5859042E6,3.875E0,2.9251662E6,1.059448E7,-5.0334964E-2,-1.9184373E-2,8.875E0,-1.9581432E-2,4.6463413E0,2.74614E5,4.48E2,7.4711205E6,4.265829E2,9.932432E0,1.4828649E9,2.6662E4,8.6895436E-2,1.7587205E6,-4.7518518E-2,-2.757553E-2,7.997723E6,6.8E1,3.4099087E-2,9.876542E-2,2E0,2.835E3,5.8899284E1,3.52E2,5.9192824E0,4.455844E1,1.4603988E6,8.4797815E2,2.6463525E8,4.087E3,1.2003246E6,2.0080749E-2,2.3876712E7,2.1257812E7,-3.3124804E-2,-1.3965118E-2,-6.606181E-2,-3.2697614E-2,4.9094446E2,2.026453E6,2.0145264E2,-3.7304517E-2,1E0,1.5714285E0,3.5714287E-1,3.4187552E6,2.2889538E-2,8.889135E-3,1.407037E2,2.1347518E0,1.089E3,-9.34718E-3,5E0,1.2585366E1,1.079E3,-2.7960172E-2,-1.0369004E-2,7.7E1,1.0494994E7,5.5454545E0,2.715325E-2,4.549749E-2,4.233978E0,2.9925186E-2,1E0,1.8545505E2,-3.998909E-2,-2.1338705E-2,6.782E4,3.401282E1,6.920598E7,-1.2486093E-2,1.3366154E6,2.118835E6,2.33E2,1.559733E6,8.628427E-3,-1.9532233E-2,1.9751773E0,3.355864E0,1.8992E4,1E0,3.9892856E2,-1.824982E-2,4.6515972E-3,6.965855E9,9.1E1,6.228074E5,7.7E1,2.239E3,-1.3491115E-2,-2.7342E-3,-1.4498445E-3,5.8407313E-3,1.3902439E0,-9.9598905E-3,6.797565E1,2.427869E8,1.4603988E6,1.0149398E-2,-7.78564E-3,1.6972052E-2,1.6063418E7,2.2384929E-2,3.457816E-2,1.2448564E-2,-1.0324918E-2,6.9740294E5,2.9711267E2,-1.1875937E-2,1.9E1,-1.1855732E-2,3.53E2,1.0953399E-2,1.3529412E0,1E0,3.5364935E-4,-8.473294E-3,1.2739792E1,-4.3281703E-3,-1.1312302E-2,-1.5357796E-3,9.66E2,7.3809524E0,3.5201266E-2,1.8253269E-2,1.8181819E0,8.684966E-1,1E0,1.4264061E-2,1.1394967E7,3.320946E-2,8.841761E-3,2.5E0,3.1095755E0,1.03339244E6,5.80418E7,7.6825094E0,1.0226706E-2,2.4505768E-2,2.6079942E-2,1.9505818E8,3.9149782E6,1.1330703E-2,4.0160073E-3,1.3949478E-2,3.624573E-2,1.4810829E-2,6.3704684E-2,4.4903275E-2,-2.6027502E-2,-9.909527E-3,-3.1908199E-3,1.9831837E-3,-1.4045271E-2,4.03E2,4.5774165E-3,4.0481758E3,-5.9024906E-3,-1.7932279E-2,-9.371145E-3,2.0671213E2,8E0,-1.116317E-2,5.031539E-3,1.2207902E-2,5.4E0,2.9547384E9,-1.1300913E-3,5.477288E-3,1.5484655E6,1.9904312E-2,3.4397193E-3,-5.4847435E-3,2.4269028E-2,1.0960829E-2,9.557399E6,-1.3799966E-2,-9.214199E-3,3.1130412E-3,1.48E1,-5.6327586E-3,1.2809288E-3,1.3326791E-2,-3.5169765E-3,4.6169646E-3,1.4172E4,1.6919017E-2,1.1915705E-2,3.5820182E-2,-2.899007E-2,-1.3642825E-2,-8.5165E-3,-4.8935594E-4,-3.4403258E-3,3.116593E-3,-2.0048281E-2,-3.329749E-2,-4.7226588E-4,-1.1971596E-2,-9.0992794E-4,5.181902E-3,4.1945907E-3,1.27073815E-2,-5.0962237E-3,3.344545E-3,1.1613483E-2,3.99133E-3,8.124161E-3,-1.8657715E-3],"split_indices":[2,43,60,44,2,67,4,53,2,43,69,43,12,0,0,68,0,68,5,2,43,67,69,46,10,0,43,0,0,9,0,0,53,6,44,62,0,71,73,43,4,7,2,66,0,60,66,0,0,0,0,4,9,70,0,26,69,68,60,0,0,67,69,2,0,69,71,10,0,0,0,62,69,0,0,68,72,6,73,0,0,9,73,7,0,62,9,0,9,0,0,71,73,9,8,67,0,0,46,10,60,0,2,0,0,0,0,68,0,73,7,43,0,0,0,62,0,0,0,0,62,4,0,67,0,44,0,68,27,0,0,71,0,0,0,12,73,0,0,71,57,6,0,60,0,0,68,53,43,7,71,0,0,0,5,60,0,0,0,0,0,0,0,0,0,0,0,0,2,0,48,0,0,0,4,8,0,0,0,73,46,0,0,43,0,0,0,0,0,60,0,0,0,73,0,0,0,0,0,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.95E2,7.4E2,2.55E2,2.12E2,5.28E2,2.4E2,1.5E1,8.4E1,1.28E2,2.77E2,2.51E2,1.11E2,1.29E2,1E1,5E0,8E1,4E0,7.8E1,5E1,1.8E2,9.7E1,1.33E2,1.18E2,7.9E1,3.2E1,1.4E1,1.15E2,7.5E1,5E0,7E1,8E0,6E0,4.4E1,6.2E1,1.18E2,8.9E1,8E0,7.4E1,5.9E1,1.04E2,1.4E1,1.4E1,6.5E1,2.8E1,4E0,5E1,6.5E1,6.5E1,5E0,4E0,4E0,1.1E1,3.3E1,5.4E1,8E0,7.8E1,4E1,1.2E1,7.7E1,4E0,4E0,5.5E1,1.9E1,5.4E1,5E0,7.6E1,2.8E1,1E1,4E0,5E0,9E0,2.9E1,3.6E1,1E1,1.8E1,3.9E1,1.1E1,5.2E1,1.3E1,6E0,5E0,1.9E1,1.4E1,4.5E1,9E0,3.9E1,3.9E1,9E0,3.1E1,4E0,8E0,1E1,6.7E1,9E0,4.6E1,1.2E1,7E0,6E0,4.8E1,2.8E1,4.8E1,1.8E1,1E1,6E0,4E0,5E0,4E0,2.4E1,5E0,2.6E1,1E1,3.3E1,6E0,4E0,7E0,4.7E1,5E0,9E0,4E0,6E0,1.3E1,9E0,5E0,4.1E1,4E0,3.5E1,4E0,2.4E1,1.5E1,5E0,4E0,2.7E1,4E0,6E0,4E0,1.1E1,5.6E1,4E0,5E0,1.3E1,3.3E1,8E0,4E0,3.1E1,1.7E1,5E0,2.3E1,1E1,3.8E1,1E1,8E0,6E0,4E0,6E0,1.8E1,2.2E1,4E0,4E0,6E0,2.9E1,4E0,3.2E1,1.5E1,9E0,4E0,4E0,5E0,5E0,3.6E1,7E0,2.8E1,4E0,2E1,6E0,9E0,2.1E1,6E0,6E0,5E0,1E1,4.6E1,4E0,9E0,2.5E1,8E0,4E0,4E0,2.4E1,7E0,1.5E1,8E0,6E0,4E0,3.3E1,5E0,4E0,6E0,4E0,4E0,1.1E1,7E0,4E0,1.8E1,3.2E1,4E0,1.1E1,1.7E1,4E0,5E0,8E0,1.3E1,5E0,5E0,1.7E1,2.9E1,8E0,1.7E1,1.1E1,4E0,1.4E1,1.9E1,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[1.3471586E-2,-2.2610085E-1,7.3868704E-1,-6.5098697E-1,-3.5431497E-2,8.4848815E-1,-1.5004842E-1,-8.9800495E-1,-4.8295447E-1,-2.2586998E-1,1.7531002E-1,4.6864507E-1,1.1211873E0,-8.2168055E-1,2.4668853E-1,-4.416024E-2,-6.1319447E-1,1.7007243E-2,-5.28793E-1,-2.930738E-1,5.057889E-2,3.0750713E-1,3.9650675E-2,1.7390183E-1,6.044502E-1,1.7208714E0,9.987359E-1,-4.5871697E-2,-1.9293519E-2,3.8010448E-1,-9.373652E-3,-3.344311E-2,-1.8930763E-2,-5.636617E-1,9.437213E-3,-2.7136397E-1,-5.556443E-2,-1.9204797E-2,3.378035E-1,1.8561703E-1,4.7336155E-1,7.310219E-2,-4.070584E-1,1.6812421E-2,4.7227135E-1,7.144502E-1,4.9912896E-2,5.6426976E-2,9.535076E-2,6.8532443E-1,1.126565E0,7.007599E-3,4.493776E-1,-6.2934786E-1,-2.4185112E-1,-5.183244E-1,-1.9883876E-1,6.202525E-2,-3.1783548E-1,9.521465E-3,2.1003593E-2,1.1989366E-1,2.4307655E-2,5.542791E-1,-6.889406E-3,-3.158213E-2,1.3950029E-1,-4.08395E-3,-2.976633E-2,-8.820557E-3,1.066468E-1,2.6161164E-2,1.1375045E-2,3.7389975E-2,3.7879828E-1,-5.8667203E-3,1.4920981E-2,4.2009376E-2,5.289257E-1,2.4025898E-2,1.1790373E0,1.3040129E-2,2.7417619E-2,-6.601182E-1,-3.2561103E-1,-4.49822E-3,-1.6475413E-2,-4.1487828E-1,-2.7016576E-2,-1.3602722E-1,-3.4765822E-1,-1.6290968E-2,1.7797509E-1,-2.3467105E-2,-4.9169087E-3,-2.3018967E-2,2.1575898E-1,4.4199905E-1,3.565774E-2,6.239491E-2,-2.3622614E-1,-1.15486704E-1,1.9155489E-1,1.4931753E-1,-7.8964967E-4,7.838179E-3,2.1996956E-2,9.713744E-3,6.057634E-1,1.2241651E0,2.7329534E-2,-3.5721904E-1,-6.9430983E-1,-2.3551518E-3,-2.376916E-2,-2.0790437E-2,-1.1102127E-2,-9.744354E-2,-3.903768E-1,-5.040833E-1,-2.9203922E-1,-8.791621E-2,1.0489454E-1,1.1815381E-2,3.6013946E-3,9.236729E-2,-1.690374E-2,2.5943175E-1,-9.6785446E-4,8.035426E-3,4.7194386E-1,1.3761136E-1,-8.0701485E-2,-1.3299356E-1,-1.7043263E-2,-1.3238124E-2,-7.622262E-3,2.6666063E-1,6.465602E-2,2.151845E-3,1.1286171E-2,1.5127124E-2,3.3154383E-2,1.3348699E0,4.6511665E-2,-2.0508487E-2,-6.794053E-3,-7.2029847E-1,-1.606041E-2,-4.970796E-2,-1.6955613E-1,-5.098529E-1,-4.9097845E-3,-2.9359356E-2,-1.4757413E-2,-1.8703546E-2,-1.9996542E-1,2.4301666E-3,-1.4379577E-1,8.326625E-3,3.9622452E-4,1.8074419E-1,-5.091463E-3,2.1009976E-2,2.0830798E-1,1.1900312E-2,4.9501446E-1,2.2191374E-1,-1.7397186E-3,1.122717E-3,-8.7665E-3,-1.00850025E-2,-1.8634964E-4,1.6903182E-3,-2.3241725E-3,3.962152E-1,1.8344365E-1,1.5655515E-1,-4.0289376E-2,3.6649484E-2,6.500713E-2,-3.3923816E-2,-1.964705E-2,-6.456711E-4,-1.4390323E-2,-3.4553998E-3,-1.2255195E-2,-1.00895045E-2,-2.9807769E-2,-9.6507213E-4,-1.0844586E-2,-8.552617E-3,-2.641438E-3,4.1510696E-3,1.0688406E-2,4.239972E-3,1.0899175E-2,1.2333621E-2,2.4224138E-2,1.2011358E-2,4.3316246E-3,1.0150402E-2,2.1707145E-2,1.0621646E-2,-2.236304E-3,1.9091156E-3,9.386613E-3,2.0133564E-3,-7.6885647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,-1,81,83,85,87,89,91,93,-1,-1,95,-1,97,-1,99,101,-1,-1,-1,103,-1,-1,-1,105,-1,-1,-1,107,-1,109,-1,-1,111,113,-1,-1,115,-1,117,119,121,123,-1,-1,125,127,129,-1,131,133,135,137,139,-1,-1,-1,-1,141,143,-1,145,147,-1,-1,-1,-1,149,151,153,155,157,159,-1,-1,161,-1,163,-1,-1,165,167,169,171,-1,-1,173,175,177,-1,-1,-1,-1,179,-1,-1,-1,181,-1,183,185,187,-1,-1,-1,-1,189,-1,191,-1,-1,193,-1,-1,195,-1,197,199,-1,-1,-1,-1,-1,-1,-1,201,203,205,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.791209E2,6.2761826E1,2.510884E1,9.528E0,2.1551044E1,2.3091461E1,7.930076E0,9.959717E-1,5.851471E0,5.2539377E0,4.560919E0,3.7775688E0,7.97435E0,3.4642363E-1,1.216778E0,0E0,1.3123655E-1,0E0,3.6425667E0,4.156475E0,1.1289285E0,2.5200014E0,1.9275302E0,1.491922E0,3.9902802E0,1.0773582E0,3.7149963E0,0E0,0E0,1.6918921E-1,0E0,0E0,0E0,2.6380768E0,0E0,3.891058E0,0E0,1.135572E0,7.556093E-2,1.6378746E0,2.802637E0,8.315556E-1,6.234549E-1,4.3092486E-1,9.4311E-2,1.4448681E0,6.206651E-1,0E0,0E0,7.45142E-1,1.8692017E0,0E0,9.7367525E-2,8.037758E-1,3.63248E-1,1.01145744E-1,1.5791879E0,3.3888727E-1,3.4786725E-1,0E0,0E0,8.9549017E-1,0E0,8.3132267E-1,0E0,9.1863567E-1,9.9106896E-1,0E0,0E0,0E0,8.703281E-2,0E0,0E0,0E0,2.0421124E-1,0E0,0E0,0E0,4.3580246E-1,0E0,1.1581421E0,0E0,0E0,7.669296E-1,5.9625506E-1,0E0,0E0,3.294611E-2,0E0,1.1788642E0,2.9649878E-1,2.0886058E-1,9.674963E-2,0E0,0E0,1.1019844E0,4.086063E-1,1.9084072E-1,0E0,3.662911E-1,1.5639597E-1,2.3768076E-1,5.641396E-1,1.0571629E-1,0E0,0E0,0E0,0E0,3.255248E-1,2.2045898E-1,0E0,1.5335834E-1,5.334358E-1,0E0,0E0,0E0,0E0,3.6678624E-1,4.8891044E-1,8.81927E-2,3.345101E-1,1.2534547E-1,6.376294E-2,0E0,0E0,3.958801E-1,0E0,2.3482108E-1,0E0,0E0,2.0850182E-2,3.3341855E-1,1.42023E-1,1.0912365E-1,0E0,0E0,1.8824793E-2,3.3700037E-1,2.3777288E-1,0E0,0E0,0E0,0E0,4.4866943E-1,0E0,0E0,0E0,1.8115997E-2,0E0,6.22355E-1,3.7017024E-1,2.930348E-1,0E0,0E0,0E0,0E0,1.4453489E-1,0E0,2.8795391E-2,0E0,0E0,4.821509E-2,0E0,0E0,6.843603E-2,0E0,3.391266E-2,4.8820138E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.8751326E-2,2.804619E-1,6.144482E-2,1.3419731E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,63,63,65,65,66,66,70,70,74,74,78,78,80,80,83,83,84,84,87,87,89,89,90,90,91,91,92,92,95,95,96,96,97,97,99,99,100,100,101,101,102,102,103,103,108,108,109,109,111,111,112,112,117,117,118,118,119,119,120,120,121,121,122,122,125,125,127,127,130,130,131,131,132,132,133,133,136,136,137,137,138,138,143,143,147,147,149,149,150,150,151,151,156,156,158,158,161,161,164,164,166,166,167,167,175,175,176,176,177,177,178,178],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,-1,82,84,86,88,90,92,94,-1,-1,96,-1,98,-1,100,102,-1,-1,-1,104,-1,-1,-1,106,-1,-1,-1,108,-1,110,-1,-1,112,114,-1,-1,116,-1,118,120,122,124,-1,-1,126,128,130,-1,132,134,136,138,140,-1,-1,-1,-1,142,144,-1,146,148,-1,-1,-1,-1,150,152,154,156,158,160,-1,-1,162,-1,164,-1,-1,166,168,170,172,-1,-1,174,176,178,-1,-1,-1,-1,180,-1,-1,-1,182,-1,184,186,188,-1,-1,-1,-1,190,-1,192,-1,-1,194,-1,-1,196,-1,198,200,-1,-1,-1,-1,-1,-1,-1,202,204,206,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.856934E7,1.16E2,9.31E2,1.1459359E3,1.3655363E2,7.6588124E-2,6E0,2.331083E6,3.6666667E0,5.486433E5,1.059448E7,1.665909E3,3.0070068E7,-4.416024E-2,2.0081382E-7,1.7007243E-2,8.909296E4,1.1290322E1,6.9307615E6,4.265829E2,1.2294118E1,4.032683E6,1E0,2.35184E5,1.7587205E6,-4.5871697E-2,-1.9293519E-2,1.4595416E3,-9.373652E-3,-3.344311E-2,-1.8930763E-2,7.997723E6,9.437213E-3,3.89E2,-5.556443E-2,8E0,1.1311654E2,1.9090909E0,2.48334E5,1.47E3,1.4787234E1,1.5204346E8,1.8333334E0,2E0,1.038946E6,5.6426976E-2,9.535076E-2,2.5817454E8,1E0,7.007599E-3,7.802E-2,3.3238492E2,1.8E0,3.1E1,2.2970297E0,1.821E3,1.5933333E2,9.521465E-3,2.1003593E-2,1.212945E6,2.4307655E-2,1.2774637E10,-6.889406E-3,4.911E3,3.2892792E2,-4.08395E-3,-2.976633E-2,-8.820557E-3,2.8E1,2.6161164E-2,1.1375045E-2,3.7389975E-2,9.0505896E2,-5.8667203E-3,1.4920981E-2,4.2009376E-2,6.319223E8,2.4025898E-2,1E0,1.3040129E-2,2.7417619E-2,9.935484E-1,1.3209776E6,-4.49822E-3,-1.6475413E-2,5.4E0,-2.7016576E-2,8.931E3,1E0,5.95E2,7.597862E6,-2.3467105E-2,-4.9169087E-3,4.68418E5,3.2857144E0,1.124E3,3.565774E-2,5.44E2,1.5E1,2.686063E6,3.093458E0,1.409E3,-7.8964967E-4,7.838179E-3,2.1996956E-2,9.713744E-3,4.3948618E8,1.7103828E7,2.7329534E-2,6.848509E7,5.0953402E4,-2.3551518E-3,-2.376916E-2,-2.0790437E-2,-1.1102127E-2,1E0,1.6E1,2.57E1,6.47E2,3.3817584E7,1.4285715E0,1.1815381E-2,3.6013946E-3,1.8E1,-1.690374E-2,1.2825651E0,-9.6785446E-4,8.035426E-3,4.797342E2,2.6647997E9,1.0674361E8,1E0,-1.7043263E-2,-1.3238124E-2,2.8655008E2,2.11429E5,1.5222016E8,2.151845E-3,1.1286171E-2,1.5127124E-2,3.3154383E-2,4.454E3,4.6511665E-2,-2.0508487E-2,-6.794053E-3,1.3091388E0,-1.606041E-2,1.1203416E1,1.6153846E0,9.2789966E-1,-4.9097845E-3,-2.9359356E-2,-1.4757413E-2,-1.8703546E-2,3.449123E0,2.4301666E-3,2.3112903E2,8.326625E-3,3.9622452E-4,4.745397E6,-5.091463E-3,2.1009976E-2,6.479833E5,1.1900312E-2,5.1604336E7,1.4839433E5,-1.7397186E-3,1.122717E-3,-8.7665E-3,-1.00850025E-2,-1.8634964E-4,1.6903182E-3,-2.3241725E-3,5.185192E2,1.26E2,6E0,4.9E1,3.6649484E-2,6.500713E-2,-3.3923816E-2,-1.964705E-2,-6.456711E-4,-1.4390323E-2,-3.4553998E-3,-1.2255195E-2,-1.00895045E-2,-2.9807769E-2,-9.6507213E-4,-1.0844586E-2,-8.552617E-3,-2.641438E-3,4.1510696E-3,1.0688406E-2,4.239972E-3,1.0899175E-2,1.2333621E-2,2.4224138E-2,1.2011358E-2,4.3316246E-3,1.0150402E-2,2.1707145E-2,1.0621646E-2,-2.236304E-3,1.9091156E-3,9.386613E-3,2.0133564E-3,-7.6885647E-3],"split_indices":[2,43,60,44,2,67,71,57,3,43,69,43,12,4,58,0,52,0,48,69,43,67,68,60,6,1,43,0,0,4,0,0,0,9,0,2,0,8,4,68,1,2,71,7,68,32,9,0,0,12,27,0,72,67,68,10,68,44,4,0,0,9,0,46,0,44,4,0,0,0,71,0,0,0,4,0,0,0,12,0,6,0,0,68,60,0,0,73,0,44,29,2,60,0,0,9,69,2,0,10,3,12,68,10,0,0,0,0,7,62,0,7,48,0,0,0,0,26,3,61,2,7,68,0,0,3,0,69,0,0,4,46,7,26,0,0,4,9,5,0,0,0,0,2,0,0,0,53,0,71,68,68,0,0,0,0,73,0,67,0,0,60,0,0,43,0,7,48,0,0,0,0,0,0,0,67,8,8,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.74E2,2.55E2,2.39E2,5.35E2,2.27E2,2.8E1,9.5E1,1.44E2,2.81E2,2.54E2,9.6E1,1.31E2,1E1,1.8E1,7.7E1,1.8E1,7E0,1.37E2,2.26E2,5.5E1,1.28E2,1.26E2,3.1E1,6.5E1,2E1,1.11E2,6E0,4E0,1.4E1,4E0,1E1,8E0,1.31E2,6E0,2.22E2,4E0,4.5E1,1E1,7.5E1,5.3E1,1.18E2,8E0,2.1E1,1E1,5.4E1,1.1E1,1E1,1E1,3.4E1,7.7E1,4E0,1E1,1.08E2,2.3E1,4.9E1,1.73E2,3.6E1,9E0,6E0,4E0,6.4E1,1.1E1,4.7E1,6E0,4.6E1,7.2E1,4E0,4E0,6E0,1.5E1,6E0,4E0,4.1E1,1.3E1,7E0,4E0,1.2E1,2.2E1,7E0,7E1,6E0,4E0,9.7E1,1.1E1,1.1E1,1.2E1,2.2E1,2.7E1,1.23E2,5E1,2.2E1,1.4E1,4E0,5E0,2.6E1,3.8E1,3.3E1,1.4E1,3.2E1,1.4E1,1.2E1,6E1,1.1E1,4E0,5E0,8E0,5E0,1.7E1,6.4E1,6E0,1.1E1,8.6E1,5E0,6E0,1.7E1,5E0,1.08E2,1.5E1,1.1E1,3.9E1,1.4E1,8E0,7E0,7E0,2E1,6E0,3.2E1,6E0,4E0,2.9E1,2.1E1,1.1E1,9E0,5E0,4E0,8E0,3.7E1,2.3E1,6E0,5E0,6E0,1.1E1,3.9E1,2.5E1,7E0,4E0,7.9E1,7E0,6.6E1,4.2E1,1E1,5E0,5E0,6E0,1.6E1,2.3E1,4E0,1E1,4E0,4E0,1.4E1,6E0,5E0,2.7E1,4E0,2.5E1,1.4E1,7E0,6E0,5E0,5E0,4E0,4E0,4E0,1.3E1,2.4E1,1.2E1,1.1E1,6E0,3.3E1,7.4E1,5E0,5.9E1,7E0,2.2E1,2E1,4E0,6E0,4E0,1.9E1,6E0,4E0,6E0,8E0,6E0,2.1E1,4E0,2.1E1,1E1,4E0,5E0,8E0,2E1,4E0,4E0,8E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"209","size_leaf_vector":"1"}},{"base_weights":[-5.7131415E-3,-2.1280274E-1,6.4551926E-1,-6.5236235E-1,-1.1418982E-2,7.558895E-1,-1.8537945E-1,-1.0013372E0,-5.050563E-1,-1.9263324E-1,1.7599282E-1,5.0836426E-1,1.0645207E0,-6.724832E-1,1.7671658E-1,-1.3869389E0,-8.3693385E-1,-8.493203E-1,-4.0978426E-1,-2.9323986E-1,1.7531294E-5,1.0551816E-1,4.7692347E-1,3.4867108E-1,7.809663E-1,1.2617676E0,6.7906153E-1,-4.0480398E-2,-1.3340322E-2,3.5177413E-1,-1.071945E-2,-3.424277E-2,-7.1736574E-2,-9.073491E-1,-5.2073604E-1,-9.080936E-1,-1.3469453E-2,-5.8451164E-1,-1.629588E-1,-2.6605478E-1,-4.7920655E-2,-4.2612037E-1,5.5164583E-2,1.2563364E-1,-2.8618587E-2,1.8970884E-1,5.530684E-1,4.8565168E-2,5.006389E-1,8.5673195E-1,4.301134E-1,8.1547685E-2,1.1213405E0,5.0695276E-1,1.0084851E0,8.504519E-3,2.003051E-2,-4.3692246E-2,-2.3995148E-2,-9.402905E-3,-3.3181816E-2,-4.461401E-2,-2.2543114E-2,-3.4037873E-1,-6.393693E-1,3.5912238E-2,-3.044859E-1,-4.7926924E-1,-1.7449664E-1,4.4251833E-4,-3.6534168E-2,-1.4875851E-2,1.5091062E-1,1.6019753E-1,-1.6385362E-1,4.930058E-3,1.1777003E-2,9.275461E-3,6.2385917E-1,1.4891837E-2,-7.70621E-2,6.118049E-1,2.0568298E-2,4.150985E-2,1.9314384E-2,2.6112651E-2,8.406481E-3,6.12446E-1,5.606765E-2,3.3181302E-2,3.4322318E-1,2.5779746E-2,5.4800406E-2,-2.5190907E-2,1.3960036E-3,-7.3375976E-1,-4.7934717E-1,-1.3277434E-1,-4.0765643E-1,-5.0291383E-1,-7.226789E-3,-2.4905093E-1,-1.7495241E-2,2.2120388E-2,-1.6507284E-1,2.0853803E-1,-5.4340563E-3,2.6271075E-1,6.8277985E-2,-3.2757118E-1,-1.2719233E-2,6.793986E-1,1.247167E-2,-1.2805148E-2,-9.710938E-3,6.8976617E-1,2.85556E-1,-8.303377E-3,1.0045475E-2,1.5552431E-2,3.4197014E-2,5.7263775E-3,4.0598503E-1,-1.6412728E-2,-3.5428498E-2,-8.322831E-3,-2.4187442E-2,-2.1534975E-1,5.061374E-3,-4.7666577E-1,-4.3011047E-3,-1.08891465E-2,-5.285037E-1,-3.1408364E-1,6.8513486E-3,2.5285916E-2,-1.3646984E-1,1.017859E-1,-5.130827E-2,-1.283611E-2,1.5091422E-4,3.032222E-1,1.1829586E-1,2.9116493E-1,-1.2637876E-2,-1.8512274E-1,1.1118296E-1,-7.440535E-3,-2.0304175E-2,7.4566975E-3,-6.5473993E-3,3.322434E-2,1.666092E-2,-5.1218826E-2,4.243391E-3,3.4646105E-2,4.7037703E-1,7.2624353E-3,1.7669909E-2,2.1585908E-2,1.0477241E-2,-1.236151E-1,-1.606056E-2,-5.556968E-1,-8.443694E-3,-2.5964834E-2,-1.5681634E-2,-2.0712969E-1,-4.3908915E-1,1.4772722E-1,-8.671995E-3,1.3502525E-1,-2.5744956E-2,-1.6614421E-3,-8.720762E-3,-3.0788453E-3,6.840154E-3,-1.1333294E-1,5.087547E-2,1.740212E-2,4.419779E-3,3.644565E-2,1.2117101E-2,1.8387146E-1,4.721319E-1,-1.3897703E-4,-2.5107273E-1,2.8319046E-1,7.43159E-2,1.503767E-3,-4.4892463E-3,1.1996412E-2,2.6221275E-2,-2.2326733E-3,-8.237311E-3,-2.762764E-2,-1.4282202E-2,1.1469987E-5,-1.1021158E-2,-2.2399798E-2,-5.234065E-3,8.955E-3,2.4776862E-3,2.861803E-3,8.011894E-3,-3.1723324E-3,4.627169E-3,2.5558582E-4,-8.315084E-3,6.157977E-3,-1.9272601E-3,-2.8932558E-3,4.3339166E-3,-2.726627E-3,1.3586501E-2,1.6983192E-2,3.0254316E-2,-6.7836526E-3,-1.448177E-2,8.466768E-3,1.7149435E-2,2.0640255E-3,1.604899E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,63,65,67,-1,69,71,73,-1,75,77,79,81,83,85,-1,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,93,95,-1,97,99,101,-1,-1,103,105,107,109,-1,-1,-1,111,-1,113,115,117,-1,-1,-1,-1,119,-1,-1,121,-1,-1,-1,-1,123,125,127,129,131,-1,133,135,137,139,141,-1,143,145,147,149,151,-1,-1,153,155,157,-1,-1,-1,-1,-1,159,-1,-1,-1,-1,161,-1,163,-1,-1,165,167,169,171,173,175,177,-1,-1,179,181,183,-1,185,187,-1,-1,-1,-1,-1,-1,189,-1,-1,191,-1,-1,-1,-1,193,-1,195,-1,-1,-1,197,199,201,-1,203,205,-1,-1,-1,-1,207,209,-1,-1,211,-1,213,215,-1,217,219,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.391816E2,6.930384E1,2.2975037E1,1.214016E1,1.830529E1,1.6265518E1,5.410189E0,3.4036102E0,5.3917465E0,5.3106565E0,5.569791E0,5.1257553E0,6.5181503E0,7.63299E-1,1.3695884E0,9.9077606E-1,7.075691E-1,9.481716E-1,5.8994236E0,3.3647804E0,2.2560303E0,3.2278268E0,9.88575E-1,3.6242008E0,8.2772446E-1,2.4739532E0,1.4731293E0,0E0,0E0,9.849846E-2,0E0,0E0,0E0,1.3433838E-1,6.0559464E-1,3.273468E-1,0E0,8.671684E-1,8.088253E0,3.332635E0,0E0,1.7553664E0,5.722629E-1,2.1374733E0,0E0,3.7742227E-2,8.4738255E-1,9.8786324E-1,2.815895E0,2.8396225E-1,2.3242116E-1,0E0,1.6984177E0,6.1827564E-1,2.18606E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3073114E0,6.3006973E-1,0E0,8.6708355E-1,3.1826973E-1,1.447079E0,0E0,0E0,2.8148764E-1,5.8162445E-1,1.7630029E0,5.649306E-1,0E0,0E0,0E0,4.4557285E-1,0E0,2.66863E-1,8.967304E-1,4.7284296E-1,0E0,0E0,0E0,0E0,1.0319781E-1,0E0,0E0,1.698699E-1,0E0,0E0,0E0,0E0,2.1626472E-1,2.62753E-1,4.4381347E-1,6.5025806E-1,2.0365429E-1,0E0,1.4000812E0,2.121355E-1,2.4506144E-1,1.9074523E-1,2.204653E-1,0E0,1.4376955E0,1.1186697E0,1.3422465E-1,3.1488374E-1,8.2829475E-2,0E0,0E0,7.215386E-2,9.60331E-2,6.42761E-2,0E0,0E0,0E0,0E0,0E0,2.5754333E-2,0E0,0E0,0E0,0E0,1.5119374E-1,0E0,5.023408E-1,0E0,0E0,8.1721306E-2,8.048301E-1,5.210902E-1,1.7707913E-1,5.1027894E-2,1.6573937E-1,1.4519995E-1,0E0,0E0,1.7740571E-1,1.9078872E-1,1.5635972E0,0E0,1.7940146E-1,5.306573E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.935592E-2,0E0,0E0,5.7418108E-2,0E0,0E0,0E0,0E0,3.6671773E-2,0E0,2.379036E-2,0E0,0E0,0E0,2.4772656E-1,3.923688E-1,3.7329063E-2,0E0,1.731725E-2,1.2576392E-1,0E0,0E0,0E0,0E0,1.119104E-1,7.424067E-2,0E0,0E0,7.407257E-2,0E0,1.4897128E0,3.513689E-1,0E0,2.5511563E-2,6.389427E-2,5.753132E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,63,63,64,64,66,66,67,67,68,68,71,71,72,72,73,73,74,74,78,78,80,80,81,81,82,82,87,87,90,90,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,107,107,108,108,109,109,110,110,111,111,114,114,115,115,116,116,122,122,127,127,129,129,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,145,145,146,146,153,153,156,156,161,161,163,163,167,167,168,168,169,169,171,171,172,172,177,177,178,178,181,181,183,183,184,184,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,64,66,68,-1,70,72,74,-1,76,78,80,82,84,86,-1,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,94,96,-1,98,100,102,-1,-1,104,106,108,110,-1,-1,-1,112,-1,114,116,118,-1,-1,-1,-1,120,-1,-1,122,-1,-1,-1,-1,124,126,128,130,132,-1,134,136,138,140,142,-1,144,146,148,150,152,-1,-1,154,156,158,-1,-1,-1,-1,-1,160,-1,-1,-1,-1,162,-1,164,-1,-1,166,168,170,172,174,176,178,-1,-1,180,182,184,-1,186,188,-1,-1,-1,-1,-1,-1,190,-1,-1,192,-1,-1,-1,-1,194,-1,196,-1,-1,-1,198,200,202,-1,204,206,-1,-1,-1,-1,208,210,-1,-1,212,-1,214,216,-1,218,220,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8988005E5,3.2856784E7,3.3817584E7,9.31E2,7.459E3,1.3655363E2,2E0,8E1,1.5859042E6,2.8530578E6,1.087521E3,7.153514E1,1.4916515E3,4.4910244E3,4.869229E7,1.7E1,9.643839E6,6.15E2,1.05039425E1,1.45064E5,1.2294118E1,4.8581E4,5.486433E5,2.6506329E1,7.9E1,3.8537518E2,-4.0480398E-2,-1.3340322E-2,1.4595416E3,-1.071945E-2,-3.424277E-2,-7.1736574E-2,2.568306E0,2.3E1,5.159652E-7,-1.3469453E-2,2.7317073E0,2.7E1,4.33E2,-4.7920655E-2,1.1015E4,8.216578E9,2.1129E4,-2.8618587E-2,6.5718125E6,4.745397E6,7.6937294E-1,2E0,1E0,6.439711E1,8.1547685E-2,1.8598528E6,8.710612E1,2.9622424E3,8.504519E-3,2.003051E-2,-4.3692246E-2,-2.3995148E-2,-9.402905E-3,-3.3181816E-2,-4.461401E-2,-2.2543114E-2,1E1,1.852364E6,3.5912238E-2,3.3E1,2.3043478E2,2.240836E6,4.4251833E-4,-3.6534168E-2,9.1E1,7.77E2,3.7105262E0,2.2503355E1,4.930058E-3,1.1777003E-2,9.275461E-3,2E0,1.4891837E-2,8.295515E7,2E0,8.399457E2,4.150985E-2,1.9314384E-2,2.6112651E-2,8.406481E-3,3.3412E5,5.606765E-2,3.3181302E-2,7.25E2,2.5779746E-2,5.4800406E-2,-2.5190907E-2,1.3960036E-3,2E0,2.2E1,2.152E3,1.363176E7,1.9E1,-7.226789E-3,3.541824E2,2.1578948E0,1.4067796E0,1.3902439E0,2.1111E4,-5.4340563E-3,2.8366232E0,2.7015875E2,2.956111E2,2.884188E2,2E1,1.247167E-2,-1.2805148E-2,7.792289E9,7.234179E6,1.2E1,-8.303377E-3,1.0045475E-2,1.5552431E-2,3.4197014E-2,5.7263775E-3,1.39442E5,-1.6412728E-2,-3.5428498E-2,-8.322831E-3,-2.4187442E-2,3.9157894E0,5.061374E-3,3.2874417E2,-4.3011047E-3,-1.08891465E-2,1.585814E6,3.6981132E0,3.9649122E0,3.4846212E5,1.3E1,7.117E3,1.0772152E1,-1.283611E-2,1.5091422E-4,1.1E1,2.3534782E2,7.6937294E-1,-1.2637876E-2,4.878788E0,1.26E2,-7.440535E-3,-2.0304175E-2,7.4566975E-3,-6.5473993E-3,3.322434E-2,1.666092E-2,1.1E1,4.243391E-3,3.4646105E-2,5.1792985E6,7.2624353E-3,1.7669909E-2,2.1585908E-2,1.0477241E-2,1E0,-1.606056E-2,1.266544E0,-8.443694E-3,-2.5964834E-2,-1.5681634E-2,1.2877E4,2.7E1,1.8578552E7,-8.671995E-3,1.4473684E-1,1.746E3,-1.6614421E-3,-8.720762E-3,-3.0788453E-3,6.840154E-3,1.2E1,1E0,1.740212E-2,4.419779E-3,1.3E1,1.2117101E-2,1.196966E6,1.935E3,-1.3897703E-4,1.0142858E1,6.7148806E5,3.3412E5,1.503767E-3,-4.4892463E-3,1.1996412E-2,2.6221275E-2,-2.2326733E-3,-8.237311E-3,-2.762764E-2,-1.4282202E-2,1.1469987E-5,-1.1021158E-2,-2.2399798E-2,-5.234065E-3,8.955E-3,2.4776862E-3,2.861803E-3,8.011894E-3,-3.1723324E-3,4.627169E-3,2.5558582E-4,-8.315084E-3,6.157977E-3,-1.9272601E-3,-2.8932558E-3,4.3339166E-3,-2.726627E-3,1.3586501E-2,1.6983192E-2,3.0254316E-2,-6.7836526E-3,-1.448177E-2,8.466768E-3,1.7149435E-2,2.0640255E-3,1.604899E-2],"split_indices":[2,43,60,7,2,2,71,32,44,43,43,67,71,4,4,46,3,62,2,69,7,68,1,43,71,0,71,0,0,4,0,0,0,68,3,52,0,69,0,2,0,9,46,44,0,60,60,53,6,6,73,0,43,73,67,0,0,0,0,0,0,0,0,3,9,0,8,4,9,0,0,0,2,69,71,0,0,0,32,0,7,32,67,0,0,0,0,1,0,0,0,0,0,0,0,32,10,2,9,67,0,67,68,68,68,9,0,53,4,4,67,3,0,0,46,62,3,0,0,0,0,0,44,0,0,0,0,71,0,67,0,0,9,71,69,43,3,1,73,0,0,8,4,53,0,73,10,0,0,0,0,0,0,3,0,0,60,0,0,0,0,26,0,53,0,0,0,9,3,5,0,71,44,0,0,0,0,3,26,0,0,3,0,9,2,0,73,43,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,7.82E2,2.48E2,2.45E2,5.37E2,2.19E2,2.9E1,7.1E1,1.74E2,2.73E2,2.64E2,1.23E2,9.6E1,1.2E1,1.7E1,1.9E1,5.2E1,3.6E1,1.38E2,1.79E2,9.4E1,2.15E2,4.9E1,7.9E1,4.4E1,6.2E1,3.4E1,7E0,5E0,1.2E1,5E0,5E0,1.4E1,4.1E1,1.1E1,3.2E1,4E0,8E1,5.8E1,1.74E2,5E0,1E1,8.4E1,2.1E2,5E0,1.1E1,3.8E1,2.7E1,5.2E1,3.5E1,9E0,1.1E1,5.1E1,2.4E1,1E1,5E0,7E0,3.6E1,5E0,5E0,6E0,2.7E1,5E0,1.6E1,6.4E1,7E0,5.1E1,5.1E1,1.23E2,5E0,5E0,4.9E1,3.5E1,1.88E2,2.2E1,6E0,5E0,7E0,3.1E1,8E0,1.9E1,4.2E1,1E1,3.1E1,4E0,5E0,4E0,9E0,4.2E1,9E0,1.5E1,4E0,6E0,1E1,6E0,3.8E1,2.6E1,2E1,3.1E1,4.7E1,4E0,8.3E1,4E1,4E1,9E0,2.9E1,6E0,8.8E1,1E2,1E1,1.2E1,2.6E1,5E0,4E0,1.5E1,3.3E1,9E0,5E0,5E0,4E0,5E0,4E0,1.1E1,4E0,3.4E1,4E0,2.2E1,1.5E1,5E0,2.5E1,6E0,5E0,4.2E1,6.6E1,1.7E1,3E1,1E1,1.9E1,2.1E1,5E0,4E0,1.3E1,1.6E1,8.4E1,4E0,1.4E1,8.6E1,5E0,5E0,5E0,7E0,2.2E1,4E0,1.1E1,4E0,2.4E1,9E0,5E0,4E0,7E0,4E0,1E1,5E0,1.9E1,6E0,3.4E1,8E0,3.7E1,2.9E1,1E1,7E0,9E0,2.1E1,4E0,6E0,4E0,1.5E1,1.3E1,8E0,9E0,4E0,1.1E1,5E0,5.4E1,3E1,4E0,1E1,1.4E1,7.2E1,4E0,7E0,4E0,5E0,5E0,5E0,1.5E1,4E0,5E0,3.2E1,2.5E1,4E0,6E0,4E0,4E0,5E0,1.6E1,5E0,5E0,8E0,4E0,4E0,4E0,7E0,1.7E1,3.7E1,2.1E1,9E0,5E0,5E0,8E0,6E0,6.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[-2.7321313E-2,-2.1315159E-1,6.306059E-1,-6.0968655E-1,-3.1799924E-2,7.2708744E-1,-5.9831965E-1,-8.8215536E-1,-4.7226286E-1,-2.0722754E-1,1.3934027E-1,8.293986E-2,6.703266E-1,-7.6598215E-1,-1.9742332E-3,-9.230321E-1,-6.1476046E-1,-5.123159E-1,3.2396268E-2,-4.908164E-1,-1.3956308E-1,8.3611384E-2,4.0579364E-1,4.7080368E-1,8.793944E-1,-4.0456094E-2,-1.9186908E-2,-3.615302E-2,-4.6607025E-2,-2.1061836E-2,-3.7400108E-2,1.5865411E-1,-5.477417E-1,-5.198807E-1,-5.2498714E-3,-1.2169319E-1,-4.0987976E-2,2.1637887E-1,-5.33264E-4,1.7481834E-1,5.268182E-1,1.8728648E-1,6.160114E-1,9.235906E-1,8.279611E-3,-1.2028875E-2,2.5222281E-2,-6.198929E-1,-2.2404358E-1,-1.082487E-2,-5.405165E-1,-1.6959687E-1,4.6009153E-2,2.5578412E-1,-1.0140608E-2,2.3873476E-2,-3.665751E-1,4.7975876E-3,1.383455E-2,5.8828557E-1,4.1362466E-3,-7.1966544E-2,2.8597155E-1,7.824462E-1,4.319763E-1,9.632503E-1,1.4324811E-2,-6.794125E-1,-2.871075E-1,-7.158672E-2,-2.9669267E-1,-2.6181515E-2,-1.6252914E-2,-1.1395624E-1,-3.5214362E-1,3.2989262E-3,1.6241241E-2,1.1122572E-1,4.8269832E-1,5.7618495E-2,-1.851813E-1,-2.8752903E-3,-2.760847E-2,1.4357027E-2,6.527789E-1,1.8022266E-3,-6.5119686E-3,2.2731097E-2,1.8925148E-1,8.800229E-3,8.526836E-1,3.4963164E-1,2.9152198E-2,7.874639E-1,5.2728053E-2,-2.3668635E-1,-7.176559E-1,5.290712E-3,-3.8834456E-1,-5.3105573E-3,-7.527128E-4,-1.6047608E-2,-5.570708E-3,-2.509221E-2,-1.9591716E-1,-5.0313705E-1,-1.4803258E-1,9.098035E-3,-5.211884E-2,2.0809026E-1,-6.796483E-2,5.4342055E-1,2.1304288E-3,3.8789086E-2,1.6075788E-2,-3.3229384E-1,1.5198614E-2,1.7619217E-2,3.2974865E-2,-3.0607564E-4,2.7367184E-1,9.658273E-1,2.3954164E-2,3.9227602E-1,3.8595155E-3,8.813821E-1,6.2130105E-1,-1.5695505E-2,-3.034763E-3,-6.6468513E-1,-1.0128735E0,-2.6655287E-2,-2.6167172E-1,-1.051066E-2,1.6605336E-2,-2.5957415E-1,-8.310299E-2,-6.68079E-3,-6.0008013E-1,-1.804626E-2,-1.87105E-2,-8.626734E-4,-1.6607787E-2,1.8373882E-2,2.756532E-1,-1.4515831E-2,1.01857E-1,1.8210046E-2,2.7924657E-2,-1.0048564E-1,6.1597485E-2,-1.9242357E-2,-8.902351E-3,6.1707953E-3,-4.9069044E-3,1.476883E-2,5.8638025E-3,3.70601E-2,5.860416E-2,1.9564262E-2,9.020591E-3,4.287401E-2,2.2662248E-2,1.7723396E-2,3.214333E-2,-3.265898E-2,-2.2636048E-2,-5.324868E-2,-2.5816234E-2,-6.2112897E-3,-1.5400013E-2,2.2749745E-3,-8.187207E-3,-6.239129E-3,-1.5513657E-2,-9.91701E-3,-1.9199622E-3,-3.824728E-2,-1.6531495E-2,-3.667687E-3,3.083856E-3,-2.4959133E-3,3.1207814E-3,4.0119765E-3,-3.1166603E-3,4.8071994E-3,1.4476486E-2,-1.2227817E-4,7.1349684E-3,6.553441E-3,-8.920388E-3,8.233219E-3,1.6653351E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,45,47,49,-1,51,-1,53,55,57,59,61,63,65,-1,-1,-1,67,69,-1,71,73,75,77,-1,79,81,-1,-1,83,-1,85,87,89,91,93,-1,95,97,99,101,-1,-1,103,105,107,-1,109,111,113,115,-1,-1,-1,117,-1,-1,-1,119,-1,121,123,-1,125,-1,127,129,-1,131,-1,-1,-1,-1,133,135,137,139,-1,141,143,145,147,-1,149,-1,151,153,-1,-1,-1,155,157,-1,159,-1,161,163,-1,-1,165,167,-1,169,-1,171,173,175,-1,177,179,-1,181,-1,183,185,-1,187,-1,-1,189,191,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2545379E2,5.7510433E1,2.7231544E1,8.9817505E0,1.654137E1,1.1529106E1,1.5508342E0,3.3976746E-1,8.302612E0,5.150878E0,4.108984E0,0E0,7.970459E0,1.3645267E-1,0E0,5.4271698E-2,6.415844E-2,3.985115E0,0E0,5.1097107E-1,2.8446512E0,2.5919433E0,1.2496796E0,4.207363E0,2.8361664E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6027203E0,3.5045319E0,1.5621662E-1,0E0,1.7510853E0,0E0,1.6019707E0,1.2862974E0,1.2812662E-1,7.8057575E-1,9.6720326E-1,1.7754593E0,1.9155502E0,0E0,0E0,0E0,2.3204575E0,3.1103635E-1,0E0,1.813507E-2,1.6764069E0,6.3827497E-1,2.6689067E0,0E0,9.6303517E-1,5.955895E-1,0E0,0E0,2.8039265E-1,0E0,8.971529E-2,4.4498086E-1,1.2928009E0,3.53014E-1,1.7619934E0,0E0,1.6417923E0,8.983244E-1,2.4501812E-2,1.3657093E-1,0E0,0E0,9.492065E-1,1.1274762E0,4.824133E-1,0E0,9.167177E-1,8.232727E-1,6.261544E-1,5.6514E-1,0E0,0E0,0E0,7.532787E-2,0E0,0E0,0E0,3.3253253E-1,0E0,6.713581E-1,2.7360535E-1,0E0,1.7026138E-1,0E0,1.5389633E-1,8.59272E-1,0E0,2.5520897E-1,0E0,0E0,0E0,0E0,5.439538E-1,4.6676445E-1,6.778736E-1,5.9392595E-1,0E0,5.510912E-1,4.3074036E-1,8.26516E-1,1.6768456E-2,0E0,3.58796E-1,0E0,4.906094E-2,1.4353059E-1,0E0,0E0,0E0,5.3943157E-2,7.2143555E-2,0E0,5.8703184E-2,0E0,3.7015915E-2,8.700609E-2,0E0,0E0,2.8822327E-1,1.806736E-1,0E0,6.4600706E-2,0E0,3.5032257E-1,3.6281037E-1,1.3317958E-1,0E0,6.418886E-1,7.2921544E-2,0E0,1.1639744E-1,0E0,6.462759E-2,1.2911427E-1,0E0,6.63499E-2,0E0,0E0,3.8643885E-1,2.796137E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,47,47,48,48,50,50,51,51,52,52,53,53,55,55,56,56,59,59,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,73,73,74,74,75,75,77,77,78,78,79,79,80,80,84,84,88,88,90,90,91,91,93,93,95,95,96,96,98,98,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,113,113,115,115,116,116,120,120,121,121,123,123,125,125,126,126,129,129,130,130,132,132,134,134,135,135,136,136,138,138,139,139,141,141,143,143,144,144,146,146,149,149,150,150],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,46,48,50,-1,52,-1,54,56,58,60,62,64,66,-1,-1,-1,68,70,-1,72,74,76,78,-1,80,82,-1,-1,84,-1,86,88,90,92,94,-1,96,98,100,102,-1,-1,104,106,108,-1,110,112,114,116,-1,-1,-1,118,-1,-1,-1,120,-1,122,124,-1,126,-1,128,130,-1,132,-1,-1,-1,-1,134,136,138,140,-1,142,144,146,148,-1,150,-1,152,154,-1,-1,-1,156,158,-1,160,-1,162,164,-1,-1,166,168,-1,170,-1,172,174,176,-1,178,180,-1,182,-1,184,186,-1,188,-1,-1,190,192,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,4.217427E7,8.8E1,9.31E2,2.2968E4,2.629E3,1.0795462E-6,6.68411E2,3.95E2,2.8530578E6,8.293986E-2,2.36433E6,1.665909E3,-1.9742332E-3,6.48334E5,4.9558692E7,6E0,3.2396268E-2,7.9016544E5,9.639872E0,1.7142857E0,6.009E4,8.5023944E2,1.0424884E-7,-4.0456094E-2,-1.9186908E-2,-3.615302E-2,-4.6607025E-2,-2.1061836E-2,-3.7400108E-2,1.0332258E2,7.997723E6,1.9E1,-5.2498714E-3,2.9652428E6,-4.0987976E-2,1.0767881E7,1.2294118E1,6.965855E9,8.0605554E2,6.516E3,9E1,7.153514E1,8.279611E-3,-1.2028875E-2,2.5222281E-2,3.2479605E2,1.8E0,-1.082487E-2,1.585814E6,1.23E2,3.1847827E2,4.797342E2,-1.0140608E-2,2.2968E4,1.4787234E1,4.7975876E-3,1.383455E-2,3.7788504E7,4.1362466E-3,1.3E1,3.3412E5,1.2825651E0,1.000501E6,7.459E3,1.4324811E-2,1.9186046E0,4.6847186E2,2.019537E2,4.6404468E2,-2.6181515E-2,-1.6252914E-2,1E0,7.6E2,6.728972E-1,1.6241241E-2,3.4E3,1E0,4.86392E5,3.1797794E1,-2.8752903E-3,-2.760847E-2,1.4357027E-2,3.474468E2,1.8022266E-3,-6.5119686E-3,2.2731097E-2,8.7390656E2,8.800229E-3,2.1298597E0,5.712652E8,2.9152198E-2,5.0741018E8,5.2728053E-2,1.1E1,4.9735293E0,5.290712E-3,5.100098E4,-5.3105573E-3,-7.527128E-4,-1.6047608E-2,-5.570708E-3,4.82E2,2.118835E6,1.0865825E0,9E0,9.098035E-3,5.6530495E6,5.5E1,5E0,1.798E3,2.1304288E-3,2.8788235E2,1.6075788E-2,7.3131656E5,4.383864E7,1.7619217E-2,3.2974865E-2,-3.0607564E-4,7.683389E7,2.64538E3,2.3954164E-2,4.4895835E0,3.8595155E-3,2.395631E7,3.422592E6,-1.5695505E-2,-3.034763E-3,1.318849E6,8.237624E0,-2.6655287E-2,1.9032816E7,-1.051066E-2,1.6527965E7,1.22896E5,5.3E2,-6.68079E-3,1E0,2.9417648E2,-1.87105E-2,4.1973075E6,-1.6607787E-2,1.3529412E0,5.6151875E5,-1.4515831E-2,7.9016544E5,1.8210046E-2,2.7924657E-2,1.089E3,1E0,-1.9242357E-2,-8.902351E-3,6.1707953E-3,-4.9069044E-3,1.476883E-2,5.8638025E-3,3.70601E-2,5.860416E-2,1.9564262E-2,9.020591E-3,4.287401E-2,2.2662248E-2,1.7723396E-2,3.214333E-2,-3.265898E-2,-2.2636048E-2,-5.324868E-2,-2.5816234E-2,-6.2112897E-3,-1.5400013E-2,2.2749745E-3,-8.187207E-3,-6.239129E-3,-1.5513657E-2,-9.91701E-3,-1.9199622E-3,-3.824728E-2,-1.6531495E-2,-3.667687E-3,3.083856E-3,-2.4959133E-3,3.1207814E-3,4.0119765E-3,-3.1166603E-3,4.8071994E-3,1.4476486E-2,-1.2227817E-4,7.1349684E-3,6.553441E-3,-8.920388E-3,8.233219E-3,1.6653351E-3],"split_indices":[2,43,60,44,2,12,0,52,67,2,43,0,43,4,0,46,46,3,0,43,69,68,1,67,52,0,0,0,0,0,0,67,9,67,0,43,0,62,68,46,4,44,8,71,0,0,0,67,68,0,9,0,4,4,0,44,71,0,0,7,0,3,1,69,43,2,0,69,4,4,4,0,0,26,2,69,0,44,6,1,71,0,0,0,4,0,0,0,4,0,53,47,0,7,0,3,68,0,43,0,0,0,0,2,9,71,8,0,43,0,8,2,0,70,0,43,12,0,0,0,47,4,0,68,0,62,43,0,0,9,71,0,5,0,60,9,2,0,29,4,0,43,0,68,66,0,43,0,0,2,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.024E3,7.99E2,2.25E2,2.5E2,5.49E2,2.09E2,1.6E1,8.2E1,1.68E2,2.71E2,2.78E2,9E0,2E2,1.2E1,4E0,6.9E1,1.3E1,1.63E2,5E0,5.1E1,2.2E2,2.31E2,4.7E1,1.04E2,9.6E1,8E0,4E0,3E1,3.9E1,9E0,4E0,8E0,1.55E2,4.7E1,4E0,2.16E2,4E0,8.9E1,1.42E2,1.7E1,3E1,3.6E1,6.8E1,9E1,6E0,4E0,4E0,1.26E2,2.9E1,4E0,4.3E1,1.68E2,4.8E1,8.2E1,7E0,1.34E2,8E0,1.2E1,5E0,2.6E1,4E0,1E1,2.6E1,3.4E1,3.4E1,8.4E1,6E0,1.06E2,2E1,1E1,1.9E1,3.6E1,7E0,1.3E2,3.8E1,4.3E1,5E0,5.1E1,3.1E1,1.16E2,1.8E1,4E0,4E0,6E0,2E1,4E0,6E0,7E0,1.9E1,4E0,3E1,2.6E1,8E0,4.5E1,3.9E1,9E0,9.7E1,4E0,1.6E1,5E0,5E0,1.4E1,5E0,6.3E1,6.7E1,2.1E1,1.7E1,9E0,3.4E1,3.3E1,1.8E1,2.7E1,4E0,1.1E2,6E0,1E1,8E0,5E0,1.5E1,6E0,1.3E1,2.1E1,9E0,2.2E1,4E0,2.6E1,1.9E1,5E0,4E0,8.5E1,1.2E1,5E0,1.1E1,1E1,5.3E1,4.2E1,2.5E1,5E0,1.6E1,1.2E1,5E0,3E1,4E0,9E0,2.4E1,7E0,1.1E1,1E1,1.7E1,1.5E1,9.5E1,5E0,5E0,4E0,4E0,9E0,4E0,1.6E1,5E0,1.8E1,4E0,2.2E1,4E0,6E0,1.3E1,6.6E1,1.9E1,8E0,4E0,5E0,6E0,4.6E1,7E0,1.7E1,2.5E1,5E0,2E1,7E0,9E0,7E0,5E0,1.7E1,1.3E1,5E0,4E0,5E0,1.9E1,4E0,7E0,4E0,1.1E1,1.6E1,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"193","size_leaf_vector":"1"}},{"base_weights":[8.976589E-3,-1.8565607E-1,5.751452E-1,-5.6272644E-1,-2.3799082E-2,6.2997967E-1,-2.76464E-1,-7.7734095E-1,-4.4256908E-1,-2.1286839E-1,1.2544522E-1,2.6550955E-1,7.993267E-1,4.3805264E-2,-3.2085083E-2,-8.1119436E-1,-5.5827997E-3,-4.7796077E-1,2.0840403E-2,-4.5805752E-1,-1.431906E-1,2.819183E-2,3.1765613E-1,1.5147437E-1,6.3087624E-1,1.403616E0,7.234894E-1,-1.037821E-2,1.6501375E-2,-3.9388802E-2,-2.1298224E-2,-7.748547E-2,-5.2942514E-1,-4.893122E-1,-9.062538E-3,-4.8304686E-1,-1.1806283E-1,1.6423035E-1,-5.593352E-2,-1.1546289E-2,3.5989767E-1,9.9220954E-2,2.211636E-2,2.016987E-2,3.5880797E-2,3.6084726E-2,7.382241E-2,5.482194E-1,9.394883E-1,-1.85779E-2,5.3337198E-2,-5.7060766E-1,-1.9487064E-1,-1.4499942E-2,-5.382861E-1,-1.003566E-2,-3.750208E-2,-1.6307665E-1,7.102636E-2,-1.2485949E-1,2.2788653E-1,-1.2284219E-2,-2.6112384E-1,4.6732873E-3,-8.757485E-3,3.8353354E-1,-4.6064984E-3,-5.8206096E-2,2.1282099E-1,1.7325893E-3,5.804768E-1,9.9883366E-1,9.209491E-3,1.3763387E-2,-8.8465676E-2,-6.1320615E-1,-2.8467864E-1,-1.3864814E-2,-1.4656074E-3,-1.4489231E-2,-2.620424E-2,-1.2501457E-1,-3.0542275E-1,-1.4960766E-3,1.5735818E-2,-1.7184617E-2,5.3753514E-2,2.5714293E-1,-3.8416544E-3,-4.5830585E-2,1.8086722E-1,-2.8967168E-2,-4.0176123E-1,2.6755875E-1,5.107791E-1,-9.77747E-3,-3.8284925E-3,3.148756E-1,-5.5413647E-3,4.393887E-1,7.174278E-1,1.0461825E0,1.285394E-2,-7.5843628E-3,2.2770879E-4,-5.5861133E-1,-8.444426E-1,1.9463387E-3,-4.3705767E-1,-2.5320935E-1,-4.7029063E-2,-2.8062437E-2,-1.954484E-1,8.442496E-3,-6.283745E-2,5.73461E-3,-1.2645593E-3,1.9284983E-1,2.015421E-2,-1.522078E-1,5.430307E-3,1.5427817E-2,4.1712478E-2,-5.9377323E-3,3.528872E-3,-2.2562139E-2,-7.6467153E-3,1.6048105E-1,4.5160702E-1,1.0057352E-2,5.5892915E-1,-4.9862205E-3,3.362375E-2,5.703676E-3,3.718224E-1,4.9299067E-1,2.4838407E-1,1.233643E-2,3.5229653E-2,2.9737024E-2,5.0002776E-2,-5.8186895E-1,-8.254169E-3,-4.769092E-2,-2.6963526E-2,-2.3705885E-2,-1.1233391E-2,-6.110349E-2,-2.9792884E-1,2.6852906E-2,-1.15500115E-1,-2.787465E-1,3.3788867E-3,-1.0285062E-2,-1.0577184E-2,2.4601297E-1,5.1241584E-2,9.319882E-4,-2.1062881E-1,1.6773035E-1,-4.8790377E-2,5.2741645E-3,-1.8054264E-3,1.9213195E-1,8.572016E-5,2.77106E-2,8.10215E-3,2.7518088E-2,1.1159483E-2,3.4063791E-3,1.4545783E-4,2.1889698E-2,1.0033796E-2,2.5096098E-2,8.924695E-3,4.0383646E-3,1.5147768E-2,-2.8510474E-2,-2.0666638E-2,-4.3014597E-4,-4.6511157E-3,-1.6400533E-2,-4.911072E-3,-3.5677375E-3,3.1142822E-3,4.9126344E-3,-7.147473E-3,-1.4926865E-2,-4.9747466E-3,-3.3059209E-3,3.122686E-3,1.32718235E-2,5.2311765E-3,7.835372E-3,-3.4389351E-3,-1.5331047E-2,-4.2531174E-3,1.9013898E-3,1.2627951E-2,-8.465574E-3,4.5229162E-6,1.6677508E-2,6.5962044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,-1,33,35,37,39,41,43,45,47,-1,-1,-1,-1,49,51,53,-1,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,71,-1,73,75,77,-1,79,-1,-1,81,83,85,87,89,91,-1,-1,93,-1,95,97,-1,99,101,-1,-1,103,105,107,-1,-1,-1,-1,109,111,113,-1,-1,115,117,-1,119,121,123,125,127,129,-1,131,133,-1,135,137,139,-1,-1,-1,141,143,-1,145,147,149,-1,151,-1,153,-1,-1,155,-1,157,159,-1,161,-1,-1,-1,-1,163,165,-1,167,-1,169,-1,171,173,175,-1,-1,-1,-1,177,-1,-1,-1,-1,-1,179,181,183,185,187,-1,189,-1,191,193,-1,195,197,199,-1,-1,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0897934E2,4.4905777E1,1.198864E1,5.355011E0,1.4586942E1,1.4477592E1,2.1725507E0,1.6355019E0,4.770111E0,3.8296175E0,5.393369E0,3.112617E0,6.408577E0,9.2148066E-1,0E0,6.796303E-1,0E0,2.8237972E0,0E0,3.1328678E-1,1.4715998E0,2.2179337E0,1.3530397E0,9.7847974E-1,1.7074823E-1,8.082886E-1,5.012245E0,0E0,0E0,0E0,0E0,7.4344707E-1,1.5834732E0,2.091322E-1,0E0,8.7217784E-1,1.4415362E0,1.3902282E0,1.0715619E0,2.5992435E-1,9.7247887E-1,9.602246E-1,0E0,0E0,0E0,0E0,0E0,1.3456078E0,2.6120338E0,0E0,4.77219E-1,1.1530685E0,2.4802244E-1,0E0,3.64933E-2,0E0,0E0,6.986644E-1,6.4568603E-1,6.379125E-1,5.764792E-1,6.5624213E-1,6.740104E-1,0E0,0E0,1.0734329E0,0E0,1.9108456E-1,1.0905383E0,0E0,1.1918983E0,1.7219849E0,0E0,0E0,6.444004E-2,7.1496964E-1,8.151921E-1,0E0,0E0,0E0,0E0,1.0778015E0,8.1122875E-1,3.1662092E-1,0E0,0E0,5.478035E-2,5.607197E-1,0E0,4.6978372E-1,3.0558726E-1,1.04214124E-1,1.8515182E-1,8.143618E-1,4.1457844E-1,0E0,7.380738E-2,2.1570921E-1,0E0,2.9731607E-1,5.68861E-1,1.03645325E-1,0E0,0E0,0E0,5.941353E-1,1.813879E-1,0E0,3.748107E-2,3.3402133E-1,3.5215706E-1,0E0,5.1258624E-1,0E0,1.80965E-1,0E0,0E0,3.1111252E-1,0E0,2.8622818E-1,5.273847E-1,0E0,5.714111E-2,0E0,0E0,0E0,0E0,1.5026277E-1,5.498185E-1,0E0,2.289772E-1,0E0,1.6880456E-2,0E0,1.8097973E-1,3.6187077E-1,1.1210269E-1,0E0,0E0,0E0,0E0,7.489395E-2,0E0,0E0,0E0,0E0,0E0,1.750857E-2,3.2314444E-1,1.4872387E-1,3.2958877E-1,9.419072E-2,0E0,8.595689E-2,0E0,1.3731492E-1,2.0600218E-1,0E0,2.7127624E-1,1.8925777E-1,2.9580626E-1,0E0,0E0,1.56807E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,47,47,48,48,50,50,51,51,52,52,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,74,74,75,75,76,76,81,81,82,82,83,83,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,99,99,100,100,101,101,105,105,106,106,108,108,109,109,110,110,112,112,114,114,117,117,119,119,120,120,122,122,127,127,128,128,130,130,132,132,134,134,135,135,136,136,141,141,147,147,148,148,149,149,150,150,151,151,153,153,155,155,156,156,158,158,159,159,160,160,163,163],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,-1,34,36,38,40,42,44,46,48,-1,-1,-1,-1,50,52,54,-1,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,72,-1,74,76,78,-1,80,-1,-1,82,84,86,88,90,92,-1,-1,94,-1,96,98,-1,100,102,-1,-1,104,106,108,-1,-1,-1,-1,110,112,114,-1,-1,116,118,-1,120,122,124,126,128,130,-1,132,134,-1,136,138,140,-1,-1,-1,142,144,-1,146,148,150,-1,152,-1,154,-1,-1,156,-1,158,160,-1,162,-1,-1,-1,-1,164,166,-1,168,-1,170,-1,172,174,176,-1,-1,-1,-1,178,-1,-1,-1,-1,-1,180,182,184,186,188,-1,190,-1,192,194,-1,196,198,200,-1,-1,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,3.9661028E7,8.8E1,8.1E2,8.7115955E2,9.7172376E7,1.1408248E0,6.68411E2,3.89E2,2.0766992E6,3.1993368E6,1.059448E7,1E1,-3.2085083E-2,1.0795462E-6,-5.5827997E-3,1.9186046E0,2.0840403E-2,8.599521E5,1.45064E5,1.75E0,2.9077E4,5.53648E8,1.2003246E6,1.2655613E3,2.8045622E6,-1.037821E-2,1.6501375E-2,-3.9388802E-2,-2.1298224E-2,3.6E2,7.997723E6,1.68E2,-9.062538E-3,1.0153885E-5,3.3486558E6,1E0,9.639872E0,2.426015E-1,9.29554E2,2.4361508E6,2.211636E-2,2.016987E-2,3.5880797E-2,3.6084726E-2,7.382241E-2,1.2825651E0,6.292039E0,-1.85779E-2,6.2E1,3.2479605E2,1.85552E6,-1.4499942E-2,2.2116136E5,-1.003566E-2,-3.750208E-2,3.6507E4,2.0833333E0,3.1157124E-1,2.2968E4,3.1797794E1,5.194E3,4.6732873E-3,-8.757485E-3,4.2210345E2,-4.6064984E-3,1.5137369E6,6.1095314E-8,1.7325893E-3,1.1251919E6,1.0424884E-7,9.209491E-3,1.3763387E-2,1.037E3,7.382199E0,4.046E3,-1.3864814E-2,-1.4656074E-3,-1.4489231E-2,-2.620424E-2,5.47E2,5.4E0,9.4658756E-1,1.5735818E-2,-1.7184617E-2,9.783615E4,4.7185095E2,-3.8416544E-3,4.7567694E5,8.931E3,1.43E2,1.212945E6,5.405423E6,1.9501E4,-9.77747E-3,3.8121E4,4.1032645E6,-5.5413647E-3,2.5345264E7,2.0750147E8,2.707947E8,1.285394E-2,-7.5843628E-3,2.2770879E-4,6.077143E5,3.6981132E0,1.9463387E-3,7.582512E7,2.82E2,1.6E0,-2.8062437E-2,1.0937695E6,8.442496E-3,5.504831E8,5.73461E-3,-1.2645593E-3,1.5132743E0,2.015421E-2,1E0,1.99693E9,1.5427817E-2,4.699871E2,-5.9377323E-3,3.528872E-3,-2.2562139E-2,-7.6467153E-3,4.0911578E2,1.162E3,1.0057352E-2,4.1104166E8,-4.9862205E-3,2.8608696E1,5.703676E-3,1.6942337E1,4.2964826E0,6.2193125E-2,1.233643E-2,3.5229653E-2,2.9737024E-2,5.0002776E-2,2.240836E6,-8.254169E-3,-4.769092E-2,-2.6963526E-2,-2.3705885E-2,-1.1233391E-2,1.9E1,1.4576986E7,1E0,7.9634375E2,7.6E2,3.3788867E-3,9.356961E6,-1.0577184E-2,2.3072304E8,2.749236E6,9.319882E-4,9.44363E5,7.026624E7,2.1340206E0,5.2741645E-3,-1.8054264E-3,7E0,8.572016E-5,2.77106E-2,8.10215E-3,2.7518088E-2,1.1159483E-2,3.4063791E-3,1.4545783E-4,2.1889698E-2,1.0033796E-2,2.5096098E-2,8.924695E-3,4.0383646E-3,1.5147768E-2,-2.8510474E-2,-2.0666638E-2,-4.3014597E-4,-4.6511157E-3,-1.6400533E-2,-4.911072E-3,-3.5677375E-3,3.1142822E-3,4.9126344E-3,-7.147473E-3,-1.4926865E-2,-4.9747466E-3,-3.3059209E-3,3.122686E-3,1.32718235E-2,5.2311765E-3,7.835372E-3,-3.4389351E-3,-1.5331047E-2,-4.2531174E-3,1.9013898E-3,1.2627951E-2,-8.465574E-3,4.5229162E-6,1.6677508E-2,6.5962044E-3],"split_indices":[2,43,60,44,2,67,60,53,67,2,43,43,12,3,0,52,0,69,0,43,7,68,1,7,66,67,43,0,0,0,0,2,9,44,0,53,43,8,69,53,67,60,0,0,0,0,0,69,53,0,0,67,12,0,43,0,0,1,68,53,44,71,44,0,0,67,0,62,52,0,43,52,0,0,2,69,44,0,0,0,0,2,73,68,0,0,48,4,0,43,44,0,9,43,9,0,1,60,0,60,7,7,0,0,0,62,71,0,7,44,68,0,43,0,5,0,0,68,0,26,46,0,67,0,0,0,0,4,10,0,7,0,73,0,71,68,72,0,0,0,0,9,0,0,0,0,0,0,5,29,48,2,0,60,0,7,12,0,9,7,68,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.87E2,7.35E2,2.52E2,2.2E2,5.15E2,2.37E2,1.5E1,7.7E1,1.43E2,2.27E2,2.88E2,7.6E1,1.61E2,9E0,6E0,7.3E1,4E0,1.38E2,5E0,4.9E1,1.78E2,1.92E2,9.6E1,5.9E1,1.7E1,1.6E1,1.45E2,5E0,4E0,6.4E1,9E0,1.6E1,1.22E2,4.3E1,6E0,1.1E1,1.67E2,7.3E1,1.19E2,1.1E1,8.5E1,5.2E1,7E0,9E0,8E0,5E0,1.1E1,8.2E1,6.3E1,4E0,1.2E1,1.08E2,1.4E1,1.1E1,3.2E1,7E0,4E0,1.35E2,3.2E1,1.3E1,6E1,9.9E1,2E1,7E0,4E0,8.1E1,4E0,2.2E1,3E1,5E0,7.7E1,5.8E1,5E0,4E0,8E0,9.3E1,1.5E1,8E0,6E0,5E0,2.7E1,1.08E2,2.7E1,2.6E1,6E0,5E0,8E0,5.5E1,5E0,8.5E1,1.4E1,8E0,1.2E1,4.4E1,3.7E1,5E0,1.7E1,2.3E1,7E0,4E1,3.7E1,5.4E1,4E0,4E0,4E0,7.8E1,1.5E1,5E0,1E1,4E1,6.8E1,6E0,2.1E1,6E0,2E1,4E0,4E0,4.2E1,1.3E1,2.7E1,5.8E1,6E0,8E0,4E0,4E0,8E0,4E0,2.9E1,1.5E1,6E0,3.1E1,4E0,1.3E1,6E0,1.7E1,3E1,1E1,4E0,3.3E1,6E0,4.8E1,7.3E1,5E0,7E0,8E0,6E0,4E0,8E0,3.2E1,3.3E1,3.5E1,1.6E1,5E0,1.6E1,4E0,3E1,1.2E1,7E0,2E1,1.4E1,4.4E1,4E0,4E0,2.4E1,5E0,9E0,6E0,2.7E1,4E0,5E0,8E0,9E0,8E0,2.5E1,5E0,4E0,6E0,5.5E1,1.8E1,4E0,4E0,2.4E1,8E0,9E0,2.4E1,5E0,3E1,1.2E1,4E0,9E0,7E0,2.2E1,8E0,6E0,6E0,9E0,1.1E1,7E0,7E0,1.1E1,3.3E1,4E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"203","size_leaf_vector":"1"}},{"base_weights":[2.5899991E-2,-1.9419216E-1,5.6307477E-1,-5.480851E-1,-1.6844464E-2,-6.132231E-1,6.104611E-1,-7.590241E-1,-4.331985E-1,-1.6650082E-1,1.4112706E-1,-3.4462553E-2,-1.2852693E-2,3.1988806E-1,7.7225965E-1,-6.60953E-1,-4.053686E-2,-3.6462346E-1,-9.5011306E-1,-4.4129208E-1,-1.069993E-1,2.9474112E-1,1.5751341E-2,1.5035282E-1,6.103874E-1,9.355166E-1,4.9535182E-1,-3.4822058E-2,-4.5412448E-1,-5.099062E-1,-1.3240522E-1,-1.2051009E0,-2.5928061E-2,-3.4435704E-1,-6.0574263E-1,-1.6271408E-1,7.9785064E-2,2.10817E-1,4.4555056E-1,7.495849E-2,-2.979508E-1,-1.1562876E-1,2.3606525E-1,6.521973E-1,9.4247125E-3,8.18089E-1,6.682793E-2,1.0034611E0,4.039347E-1,-2.3135023E-2,-1.0750364E-2,-5.448496E-1,-2.1894737E-1,2.403337E-2,-2.5201276E-1,-3.447157E-2,-6.998315E-2,-1.7853767E-2,-8.329051E-3,-1.5855588E-2,-3.0953916E-2,-3.6070856E-1,-9.5359966E-2,1.8069474E-1,-3.775566E-2,1.6005524E-1,4.9938202E-1,5.095464E-1,4.5501604E-3,1.4347373E-1,-3.2632295E-2,-1.7336248E-1,-3.0581964E-2,-1.3022984E-2,6.846515E-4,2.784489E-1,-9.681204E-3,2.2284646E-2,3.5152428E-2,8.643828E-1,4.8850462E-1,2.0195067E-2,6.0435697E-2,5.3143036E-1,4.268525E-2,-4.9129814E-1,-6.93085E-1,-3.5155436E-3,-1.3384149E-2,-1.8813774E-1,-4.759241E-1,-1.7540015E-1,-5.3724694E-1,-1.3317022E-1,6.989068E-2,8.843524E-2,3.194664E-1,-1.0942415E-1,8.5886896E-2,2.0008877E-1,-3.15232E-3,2.8689932E-2,1.4543351E-2,1.710361E-2,3.013517E-2,-4.9161958E-3,1.693727E-1,-1.8062603E-1,2.8509917E-2,-6.3914515E-4,-2.3867954E-1,-3.1393045E-3,3.197293E-3,4.2300195E-1,1.6525213E-1,9.7641134E-1,6.860471E-1,1.2540972E-2,2.8521437E-2,3.4125382E-1,6.6992146E-1,2.1136494E-1,-1.0555592E-2,-2.7661588E-2,-3.5579148E-1,-3.688066E-2,-2.1333795E-2,-2.251704E-1,6.15445E-4,-2.8207693E-2,-1.01252515E-2,-2.2520797E-1,1.340996E-3,-2.25969E-1,-7.7138627E-1,-9.460886E-3,-1.8760873E-1,1.7805059E-2,1.1727069E-2,1.493668E-1,-4.050449E-3,6.199278E-3,1.9432344E-2,-1.6709466E-1,1.8172765E-3,7.148093E-3,-5.885711E-6,1.937808E-2,1.7126915E-1,-8.527477E-3,9.799177E-2,5.077055E-2,2.2464107E-1,-2.4591428E-1,-1.2465415E-3,9.393035E-2,-1.24808066E-1,-1.4112135E-2,-5.589379E-3,1.0473481E-2,4.909368E-1,-3.731076E-5,2.1057786E-1,2.4195883E-2,1.0056611E0,1.1438845E-2,7.508684E-1,4.195695E-1,3.48971E-3,1.5414384E-2,3.317483E-2,1.0831631E-3,1.4571091E-2,-1.8158314E-3,-4.0485722E-1,-1.2284252E-3,-2.683146E-1,-1.5851019E-2,-6.814049E-3,-1.6526833E-2,-1.0469066E-3,-4.748191E-2,-1.7853165E-2,-9.79328E-2,7.960233E-2,-1.12324744E-1,-2.5538152E-1,7.7016495E-2,-4.375171E-3,8.708977E-3,2.2654105E-3,-1.2472493E-2,-1.6801341E-3,5.627967E-4,1.908505E-1,1.1876486E-3,6.9611794E-3,-5.1907017E-3,1.2351117E-1,2.7262053E-1,9.663919E-2,-1.5007516E-2,-5.4422948E-3,1.5850979E-1,9.070066E-3,-3.568707E-4,-1.1103784E-2,2.631044E-2,1.0885399E-2,2.6020753E-1,1.0997864E-1,1.0956743E0,3.504616E-2,2.376837E-2,3.8961507E-2,2.3762712E-2,1.0358896E-2,-2.0228496E-2,-8.560411E-3,-1.428076E-2,-3.418675E-3,1.5080068E-3,-8.294648E-3,2.0374921E-4,5.2380306E-3,-7.723287E-3,8.1116444E-4,-1.4845412E-2,-7.698857E-3,5.2661817E-3,-2.280954E-4,2.2682855E-3,9.481751E-3,1.0832815E-2,2.8570371E-3,8.336831E-3,1.5360508E-2,1.07226E-2,4.919231E-4,1.0295539E-3,1.0005695E-2,3.7886964E-3,-3.0204777E-3,7.2013107E-3,1.4736199E-2,1.6959865E-3,7.449649E-3,5.29985E-2,2.8122451E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,-1,79,-1,81,83,-1,-1,85,87,-1,89,-1,-1,-1,-1,-1,-1,91,93,95,97,99,101,103,-1,105,107,109,-1,-1,111,113,-1,-1,-1,115,117,-1,-1,119,121,123,125,-1,-1,127,129,131,133,135,137,139,141,143,145,147,149,-1,-1,-1,-1,-1,151,153,155,-1,157,-1,-1,159,161,163,165,-1,-1,167,169,171,-1,-1,173,-1,-1,175,-1,-1,-1,177,-1,179,181,183,185,187,-1,189,-1,-1,-1,191,-1,-1,-1,-1,193,-1,195,197,199,201,-1,203,205,-1,-1,-1,207,-1,209,-1,211,-1,213,215,-1,-1,-1,-1,-1,-1,217,-1,219,-1,-1,-1,-1,-1,-1,221,223,225,227,229,-1,-1,-1,-1,-1,-1,231,-1,-1,-1,233,235,237,-1,-1,239,241,-1,-1,-1,-1,243,245,247,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.20820076E2,4.5493286E1,1.6929268E1,5.5358353E0,1.1465957E1,3.2609987E-1,1.3199051E1,3.9554977E-1,5.289707E0,4.024057E0,4.5405936E0,0E0,0E0,5.0302134E0,7.809494E0,6.1725426E-1,0E0,4.7231846E0,1.1085796E0,4.928131E-1,2.1567519E0,1.2476654E0,2.4554505E0,1.5547571E0,5.1260567E-1,5.6573105E0,2.846161E0,0E0,6.9381475E-2,7.652416E-1,4.5023556E0,2.911539E-1,0E0,1.3752985E-1,2.3151398E-2,2.0859861E0,5.7615006E-1,9.4543624E-1,7.715993E-1,8.2293034E-1,8.376204E-1,3.2894325E-1,1.0215862E0,2.6061726E-1,0E0,1.0272217E0,0E0,1.1537552E0,2.7868376E0,0E0,0E0,2.7073288E-1,8.914286E-2,0E0,5.969732E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2143288E0,7.590878E-1,3.04626E-1,2.1320744E-1,3.9919102E-1,2.9009104E-2,3.7505722E-1,0E0,4.58107E-1,4.0329087E-1,1.7383623E-1,0E0,0E0,5.6437027E-2,6.9915605E-1,0E0,0E0,0E0,9.683914E-1,2.239654E-1,0E0,0E0,9.5658684E-1,8.258761E-1,6.605797E-1,6.941986E-2,0E0,0E0,2.93962E-1,2.1189547E-1,2.2035992E-1,1.2833385E0,6.5821123E-1,2.158435E-1,1.9551578E-1,1.3097978E-1,1.3525873E-1,5.3308338E-2,2.3793173E-1,2.566434E-1,0E0,0E0,0E0,0E0,0E0,3.9500177E-1,1.237697E-1,3.3268738E-1,0E0,5.7365417E-2,0E0,0E0,1.5541339E-1,2.1091986E-1,1.08860016E-1,8.088131E-1,0E0,0E0,4.0531397E-1,1.6530609E-1,2.0765361E-1,0E0,0E0,4.3441868E-1,0E0,0E0,2.7559638E-1,0E0,0E0,0E0,1.05151296E-1,0E0,2.5977653E-1,6.884546E-1,2.5206307E-1,3.1826448E-1,1.3386676E-1,0E0,4.181066E-2,0E0,0E0,0E0,1.3812086E-1,0E0,0E0,0E0,0E0,1.3785636E-1,0E0,3.0402087E-2,2.630603E-1,2.3159552E-1,5.274129E-2,0E0,1.2460758E-1,1.3341984E-1,0E0,0E0,0E0,1.4664006E-1,0E0,8.1222296E-2,0E0,1.4737701E-1,0E0,2.2190475E-1,1.8092537E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.999657E-2,0E0,1.9117963E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7635658E-1,4.0136524E-2,2.4515104E-1,1.5128946E-1,3.9953195E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.149565E-2,0E0,0E0,0E0,8.815101E-2,1.0552621E-1,1.4895368E-1,0E0,0E0,9.800756E-2,6.509405E-2,0E0,0E0,0E0,0E0,3.6113918E-2,2.789019E-2,5.406952E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,51,51,52,52,54,54,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,74,74,75,75,79,79,80,80,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,106,106,107,107,108,108,110,110,113,113,114,114,115,115,116,116,119,119,120,120,121,121,124,124,127,127,131,131,133,133,134,134,135,135,136,136,137,137,139,139,143,143,148,148,150,150,151,151,152,152,153,153,155,155,156,156,160,160,162,162,164,164,166,166,167,167,174,174,176,176,183,183,184,184,185,185,186,186,187,187,194,194,198,198,199,199,200,200,203,203,204,204,209,209,210,210,211,211],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,-1,80,-1,82,84,-1,-1,86,88,-1,90,-1,-1,-1,-1,-1,-1,92,94,96,98,100,102,104,-1,106,108,110,-1,-1,112,114,-1,-1,-1,116,118,-1,-1,120,122,124,126,-1,-1,128,130,132,134,136,138,140,142,144,146,148,150,-1,-1,-1,-1,-1,152,154,156,-1,158,-1,-1,160,162,164,166,-1,-1,168,170,172,-1,-1,174,-1,-1,176,-1,-1,-1,178,-1,180,182,184,186,188,-1,190,-1,-1,-1,192,-1,-1,-1,-1,194,-1,196,198,200,202,-1,204,206,-1,-1,-1,208,-1,210,-1,212,-1,214,216,-1,-1,-1,-1,-1,-1,218,-1,220,-1,-1,-1,-1,-1,-1,222,224,226,228,230,-1,-1,-1,-1,-1,-1,232,-1,-1,-1,234,236,238,-1,-1,240,242,-1,-1,-1,-1,244,246,248,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,1.972052E5,1E0,9.9E1,9.31E2,2.3308511E5,8.625455E2,9.55188E5,8.462282E0,3.79E2,3.3358974E0,-3.4462553E-2,-1.2852693E-2,2.308821E6,5.489183E1,3.9125E1,-4.053686E-2,6.83E2,7.3E1,1E0,3.3486558E6,4.371613E2,2E1,1E0,1E0,1.7379E4,1.2083E4,-3.4822058E-2,5.2755904E0,7.997723E6,2.7E1,8.317E3,-2.5928061E-2,3E0,3.7043128E5,1E0,3.5364E4,6.0052995E6,1E0,1.9E1,4.1E1,1E0,8.920169E6,1.0410659E5,9.4247125E-3,1E0,6.682793E-2,2.8016653E3,2.707256E7,-2.3135023E-2,-1.0750364E-2,1.2E1,1.2E1,2.403337E-2,2.694192E9,-3.447157E-2,-6.998315E-2,-1.7853767E-2,-8.329051E-3,-1.5855588E-2,-3.0953916E-2,7.4142776E7,7.814751E6,3.3832976E7,1.594941E3,1.0292E4,1.9913E4,5.460753E9,4.5501604E-3,1.4473684E-1,4.591837E0,1.4081731E1,-3.0581964E-2,-1.3022984E-2,3.414E3,8E0,-9.681204E-3,2.2284646E-2,3.5152428E-2,2.1089442E0,4.797448E8,2.0195067E-2,6.0435697E-2,1.3388E4,3.0070068E7,2.95E2,2.2E0,-3.5155436E-3,-1.3384149E-2,3.1158695E6,8.57E2,8.2E1,1.027972E0,1.5132743E0,8E0,6.34E2,2.6732E4,1.3683688E7,1.5E1,1.1160929E0,1.007E3,2.8689932E-2,1.4543351E-2,1.710361E-2,3.013517E-2,-4.9161958E-3,7.709291E7,8.847733E4,3.4444444E0,-6.3914515E-4,9.936263E6,-3.1393045E-3,3.197293E-3,4.2010452E2,4.3333566E2,9.2789966E-1,9.5E2,1.2540972E-2,2.8521437E-2,3.6440072E0,2.7317073E0,6.619098E8,-1.0555592E-2,-2.7661588E-2,3E1,-3.688066E-2,-2.1333795E-2,1.4285715E0,6.15445E-4,-2.8207693E-2,-1.01252515E-2,2.1651703E2,1.340996E-3,3.8E2,5.4E0,2.732381E2,7.619497E0,1.7288135E0,1.1727069E-2,1E0,-4.050449E-3,6.199278E-3,1.9432344E-2,2.1557376E0,1.8172765E-3,7.148093E-3,-5.885711E-6,1.937808E-2,4.0449125E5,-8.527477E-3,3.1096E4,4.135135E0,2.0158867E8,3.0291306E2,-1.2465415E-3,1.390602E8,2.7012987E1,-1.4112135E-2,-5.589379E-3,1.0473481E-2,5.7819215E6,-3.731076E-5,3.1530054E0,2.4195883E-2,1.2820834E7,1.1438845E-2,2.5247778E6,6.439711E1,3.48971E-3,1.5414384E-2,3.317483E-2,1.0831631E-3,1.4571091E-2,-1.8158314E-3,1.432632E6,-1.2284252E-3,2.1340163E8,-1.5851019E-2,-6.814049E-3,-1.6526833E-2,-1.0469066E-3,-4.748191E-2,-1.7853165E-2,1.01417E0,4.0481758E3,2.884188E2,1.339646E6,1.4E1,-4.375171E-3,8.708977E-3,2.2654105E-3,-1.2472493E-2,-1.6801341E-3,5.627967E-4,2.2734E4,1.1876486E-3,6.9611794E-3,-5.1907017E-3,1.0439024E1,8.995735E5,5E0,-1.5007516E-2,-5.4422948E-3,5.617715E6,9E1,-3.568707E-4,-1.1103784E-2,2.631044E-2,1.0885399E-2,8.4797815E2,2.427869E8,3.4332926E0,3.504616E-2,2.376837E-2,3.8961507E-2,2.3762712E-2,1.0358896E-2,-2.0228496E-2,-8.560411E-3,-1.428076E-2,-3.418675E-3,1.5080068E-3,-8.294648E-3,2.0374921E-4,5.2380306E-3,-7.723287E-3,8.1116444E-4,-1.4845412E-2,-7.698857E-3,5.2661817E-3,-2.280954E-4,2.2682855E-3,9.481751E-3,1.0832815E-2,2.8570371E-3,8.336831E-3,1.5360508E-2,1.07226E-2,4.919231E-4,1.0295539E-3,1.0005695E-2,3.7886964E-3,-3.0204777E-3,7.2013107E-3,1.4736199E-2,1.6959865E-3,7.449649E-3,5.29985E-2,2.8122451E-2],"split_indices":[2,43,17,44,2,48,67,46,69,2,69,0,0,43,71,4,0,2,0,26,43,67,3,27,6,2,9,0,69,9,0,9,0,8,43,29,9,43,6,8,3,26,62,48,0,6,0,4,60,0,0,8,3,0,46,0,0,0,0,0,0,5,60,7,48,44,9,46,0,71,69,73,0,0,2,32,0,0,0,53,7,0,0,2,58,2,68,0,0,60,2,0,68,68,8,2,1,60,3,69,2,0,0,0,0,0,7,48,68,0,60,0,0,67,67,68,10,0,0,68,69,7,0,0,0,0,0,71,0,0,0,67,0,12,73,4,73,68,0,28,0,0,0,69,0,0,0,0,43,0,9,69,7,67,0,7,73,0,0,0,66,0,68,0,9,0,43,73,0,0,0,0,0,0,12,0,7,0,0,0,0,0,0,68,48,67,9,3,0,0,0,0,0,0,9,0,0,0,73,43,3,0,0,12,8,0,0,0,0,4,7,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,7.24E2,2.96E2,2.41E2,4.83E2,1.1E1,2.85E2,8.3E1,1.58E2,2.48E2,2.35E2,7E0,4E0,1.03E2,1.82E2,4.9E1,3.4E1,1.41E2,1.7E1,4.3E1,2.05E2,1.05E2,1.3E2,6.6E1,3.7E1,1.13E2,6.9E1,3.2E1,1.7E1,8.6E1,5.5E1,9E0,8E0,2.9E1,1.4E1,1.58E2,4.7E1,6.9E1,3.6E1,1.1E2,2E1,1.6E1,5E1,3.3E1,4E0,9.4E1,1.9E1,9E0,6E1,1.3E1,4E0,7.6E1,1E1,8E0,4.7E1,5E0,4E0,2.2E1,7E0,4E0,1E1,3.9E1,1.19E2,2.5E1,2.2E1,6E1,9E0,3E1,6E0,6.7E1,4.3E1,1.6E1,4E0,6E0,1E1,4.6E1,4E0,1.5E1,1.8E1,8.1E1,1.3E1,4E0,5E0,4.4E1,1.6E1,5.9E1,1.7E1,4E0,6E0,3.8E1,9E0,2E1,1.9E1,9.7E1,2.2E1,1.6E1,9E0,1.4E1,8E0,4.8E1,1.2E1,4E0,5E0,1.7E1,1.3E1,6E0,6.1E1,1.2E1,3.1E1,5E0,1.1E1,5E0,5E0,1.9E1,2.7E1,4.7E1,3.4E1,6E0,7E0,2E1,2.4E1,1E1,6E0,3.1E1,2.8E1,1E1,7E0,3.2E1,6E0,5E0,4E0,1.6E1,4E0,9E0,1E1,3E1,6.7E1,1.8E1,4E0,1.2E1,4E0,4E0,5E0,1E1,4E0,4E0,4E0,4E0,4.4E1,4E0,8E0,2E1,4.1E1,8E0,4E0,2.2E1,9E0,6E0,5E0,6E0,1.3E1,6E0,2.1E1,4E0,4.3E1,5E0,2.9E1,1.5E1,5E0,4E0,2E1,4E0,6E0,4E0,2.4E1,6E0,2.6E1,5E0,1.1E1,5E0,4E0,5E0,5E0,1.5E1,1.5E1,3.3E1,3.4E1,1.2E1,6E0,8E0,4E0,5E0,5E0,5E0,3.9E1,4E0,4E0,6E0,1.4E1,2.9E1,1.2E1,4E0,4E0,1.2E1,1E1,5E0,4E0,9E0,4E0,1.3E1,8E0,2.9E1,1.4E1,1E1,1.9E1,9E0,6E0,2E1,4E0,2.1E1,5E0,6E0,9E0,5E0,1E1,2.3E1,1E1,1.8E1,1.6E1,8E0,4E0,4E0,3.5E1,4E0,1E1,1.3E1,1.6E1,4E0,8E0,4E0,8E0,5E0,5E0,6E0,7E0,4E0,4E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"249","size_leaf_vector":"1"}},{"base_weights":[6.798278E-3,-2.9250383E-1,3.2333016E-1,-5.4509145E-1,-1.3893844E-1,1.4397086E-1,6.3014203E-1,-7.511041E-1,-4.2006698E-1,-3.5289416E-1,-5.0119236E-2,-4.0301904E-2,1.9171517E-1,7.1512866E-1,2.2618681E-2,-9.9959886E-1,-6.9973826E-1,-6.033013E-1,-2.9881284E-1,-4.2399275E-1,-1.2785693E-1,-1.0241604E-1,8.260068E-2,4.0933564E-2,1.591436E-1,6.866623E-2,6.468455E-1,-2.665817E-1,3.099342E-1,-2.5027623E-2,-5.611118E-2,-3.5596162E-2,-5.5265373E-1,-3.6661573E-2,-5.016208E-1,2.8036669E-2,-3.575571E-1,-3.2117665E-1,-5.392631E-1,-1.8218723E-1,5.3145397E-3,-1.884132E-1,-2.8034728E-3,-1.7533407E-1,1.3838987E-1,-9.642109E-2,2.6033372E-1,2.0197384E-1,6.962432E-1,-1.9054212E-2,9.287665E-4,-6.836584E-3,2.5750862E-2,-1.0586381E-2,-2.8907977E-2,-1.20543E-2,-5.250168E-1,-2.9863966E-1,-5.8031213E-1,-3.5235187E-1,-3.9635E-3,-4.5235357E-1,-3.8829327E-2,-9.630789E-2,-2.584358E-1,-2.783935E-1,-3.0009903E-2,-9.539431E-2,5.8326732E-2,-1.5262059E-2,4.978653E-3,9.87916E-2,1.6342035E-2,-1.3543558E-2,-5.288925E-1,2.2214657E-1,2.958916E-2,-1.2322253E-3,3.418286E-1,5.4083705E-2,6.473597E-1,-2.6804253E-2,-1.5329408E-2,-1.705044E-1,-3.7021998E-1,-3.2409858E-2,-1.8558647E-2,-3.7659407E-1,-6.078864E-3,-2.2322569E-2,-1.2192952E-2,-6.845971E-3,-1.3111394E-3,-1.463086E-2,-6.321943E-3,-1.8398893E-1,-3.9660004E-1,9.742137E-2,-1.8942219E-1,2.3122365E-2,-1.529319E-1,-1.7521594E-2,1.932074E-1,1.3263568E-1,-5.8842912E-2,-1.347676E-1,8.889626E-2,-1.2324528E-2,-3.0462394E-2,2.4081929E-1,-1.3385908E-2,2.0845033E-2,6.5702763E-3,4.8914114E-1,7.5950795E-1,-9.660245E-2,-1.6806886E-2,-2.7759965E-3,-4.059909E-1,-2.3605634E-2,-1.4212878E-2,-1.0439892E-1,-2.8454685E-1,-2.4071284E-2,-2.705018E-1,-2.6948992E-3,8.922826E-3,-1.0214229E-2,-3.782071E-3,7.119939E-3,-3.4070218E-3,-3.6927234E-4,-1.9772571E-1,-6.125142E-2,5.6821585E-2,-1.4373868E-4,2.538525E-1,8.0264695E-2,1.068138E-2,-8.271929E-3,3.3786534E-3,8.9172255E-3,-2.3354761E-1,1.17935985E-1,-3.956408E-3,3.507249E-1,1.3423313E-1,5.385398E-1,1.8779878E-1,8.2583904E-1,2.8557053E-1,-1.4705496E-1,1.7037791E-4,-4.6494633E-1,-2.2389881E-1,-7.8066083E-4,-1.5021509E-1,-6.403517E-3,-1.5202354E-2,-1.599685E-2,-2.9976934E-3,-2.386771E-1,-2.7671948E-3,2.7382474E-3,-9.508635E-2,-9.336852E-4,4.521208E-3,4.5469464E-3,1.4413992E-2,3.927989E-2,6.297303E-3,5.585332E-3,-6.672683E-2,-3.1092018E-1,-4.079159E-3,1.4590019E-1,-3.1294962E-4,3.9253423E-1,-1.4641273E-2,8.984514E-2,3.7829068E-1,8.155576E-3,5.6955206E-1,1.1485854E-2,4.1311714E-3,2.4195999E-2,4.1874755E-2,-4.9958057E-3,2.615165E-2,-1.2944418E-3,-9.381672E-3,-1.5504474E-2,-2.4640905E-2,-1.536285E-2,-4.437479E-3,-3.7726064E-3,-1.1252776E-2,-5.464779E-3,-1.3124496E-2,-7.428129E-4,-5.984501E-3,-1.7113466E-4,5.483901E-3,-3.7516936E-4,-5.1737204E-3,-8.49017E-3,-1.7347082E-2,2.1123588E-3,9.338601E-3,1.2624343E-2,2.470133E-2,-6.8272697E-3,4.5620366E-3,-6.8897502E-3,5.6473007E-3,2.1985939E-2,4.9627996E-3,2.7737483E-2,1.5430196E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,-1,-1,53,-1,55,-1,57,59,61,63,-1,65,67,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,81,83,85,87,-1,89,-1,91,93,95,97,99,101,-1,-1,103,-1,105,107,109,-1,-1,111,-1,113,-1,-1,115,117,-1,-1,119,-1,-1,-1,-1,-1,-1,-1,121,123,125,127,129,131,133,135,137,139,141,143,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,153,-1,-1,155,157,-1,159,-1,-1,-1,-1,-1,-1,-1,161,163,165,-1,167,169,-1,-1,-1,171,173,175,-1,177,179,181,183,185,187,189,-1,191,193,-1,195,-1,-1,-1,-1,197,-1,-1,199,-1,-1,-1,-1,201,-1,-1,203,205,-1,207,-1,209,211,213,215,-1,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.4738495E1,1.9861668E1,2.665364E1,4.6812706E0,6.0823965E0,1.5112545E1,9.22583E0,2.1956253E-1,2.5525303E0,1.4497738E0,1.5909684E0,0E0,6.530362E0,7.3060303E0,1.9937334E0,3.26231E-1,2.2984123E-1,5.3050995E-1,4.2604527E0,6.6826916E-1,3.3767235E-1,1.4043611E0,9.596126E-1,0E0,7.3436575E0,0E0,3.0678177E0,5.100673E-1,1.4422456E0,0E0,0E0,0E0,4.2396593E-1,0E0,5.44796E-2,0E0,7.716341E-1,2.6329756E-1,4.5751858E-1,1.06074035E-1,0E0,1.2459252E0,4.4139153E-1,5.622345E-1,4.255196E-1,2.8960526E0,2.8045444E0,5.207529E-1,2.280243E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2728119E-1,4.7802162E-1,2.1276474E-2,1.3776827E-1,0E0,2.9323578E-2,0E0,3.455674E-2,2.7499557E-2,5.519471E-1,6.8881947E-1,2.1493018E-1,4.8421752E-1,0E0,0E0,2.5854942E-1,0E0,8.688341E-1,2.687223E-1,1.8476362E0,0E0,0E0,1.5415096E-1,0E0,1.7755051E0,0E0,0E0,2.933979E-1,3.8607264E-1,0E0,0E0,1.299963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.34406E-1,2.3564172E-1,2.9452467E-1,3.2994866E-2,1.5094508E-1,1.3475919E-1,1.04261845E-1,2.0318782E-1,1.8276405E-1,1.5587352E-1,4.5626467E-1,2.027004E-1,0E0,0E0,2.0615168E0,0E0,0E0,0E0,6.8781567E-1,1.9333076E0,9.162052E-2,0E0,0E0,2.501645E-1,0E0,0E0,7.7641055E-2,4.5341134E-2,0E0,1.9560897E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.6012194E-2,8.719295E-2,3.9916288E-2,0E0,8.678162E-2,5.6662843E-2,0E0,0E0,0E0,1.2665567E-1,1.860795E-1,1.1607301E-1,0E0,1.3500109E0,9.7350204E-1,4.0465736E-1,3.161651E-2,9.8553467E-1,1.1655376E0,7.4324995E-2,0E0,3.338909E-2,1.0694945E-1,0E0,6.331104E-2,0E0,0E0,0E0,0E0,3.1996906E-2,0E0,0E0,4.1467562E-2,0E0,0E0,0E0,0E0,5.8475614E-2,0E0,0E0,2.2963595E-2,2.9507041E-2,0E0,1.4286971E-1,0E0,1.1932125E0,1.6555646E-1,6.3753015E-1,3.1901288E-1,0E0,9.731102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,26,26,27,27,28,28,32,32,34,34,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,56,56,57,57,58,58,59,59,61,61,63,63,64,64,65,65,66,66,67,67,68,68,71,71,73,73,74,74,75,75,78,78,80,80,83,83,84,84,87,87,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,109,109,113,113,114,114,115,115,118,118,121,121,122,122,124,124,132,132,133,133,134,134,136,136,137,137,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,150,150,151,151,153,153,154,154,156,156,161,161,164,164,169,169,172,172,173,173,175,175,177,177,178,178,179,179,180,180,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,-1,-1,54,-1,56,-1,58,60,62,64,-1,66,68,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,82,84,86,88,-1,90,-1,92,94,96,98,100,102,-1,-1,104,-1,106,108,110,-1,-1,112,-1,114,-1,-1,116,118,-1,-1,120,-1,-1,-1,-1,-1,-1,-1,122,124,126,128,130,132,134,136,138,140,142,144,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,154,-1,-1,156,158,-1,160,-1,-1,-1,-1,-1,-1,-1,162,164,166,-1,168,170,-1,-1,-1,172,174,176,-1,178,180,182,184,186,188,190,-1,192,194,-1,196,-1,-1,-1,-1,198,-1,-1,200,-1,-1,-1,-1,202,-1,-1,204,206,-1,208,-1,210,212,214,216,-1,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,4.27E3,3.557841E7,5.02E2,1E0,2.856934E7,2E0,3.06E2,9.860918E5,2.0766992E6,-4.0301904E-2,1.2E1,3.6E1,1.3E1,1.5E1,3.62E2,3.89E2,2E0,4.325E0,4.0705118E6,1.559733E6,5.8899284E1,4.0933564E-2,4.8709216E5,6.866623E-2,3.257507E5,4.9236734E5,2.0618556E-2,-2.5027623E-2,-5.611118E-2,-3.5596162E-2,1.0005E5,-3.6661573E-2,6.695E3,2.8036669E-2,1.9E1,4.67E2,7.668863E6,3.429012E7,5.3145397E-3,3.4734247E2,4.8709216E5,1.7010834E3,7.168071E6,2.3E1,5.405423E6,1.8452264E6,1.0918E4,-1.9054212E-2,9.287665E-4,-6.836584E-3,2.5750862E-2,-1.0586381E-2,-2.8907977E-2,-1.20543E-2,3.201807E6,3.464E0,6.81531E-2,2.241E2,-3.9635E-3,1.9637654E2,-3.8829327E-2,1.0865825E0,6.7723384E7,3.6981132E0,1.798E3,2.812736E5,9.24E2,-1.5262059E-2,4.978653E-3,2.8302418E10,1.6342035E-2,3.6161574E4,4E1,1.6776881E7,2.958916E-2,-1.2322253E-3,1.8996E4,5.4083705E-2,7.098E3,-2.6804253E-2,-1.5329408E-2,1.1769393E5,2.3636363E0,-3.2409858E-2,-1.8558647E-2,1.1967312E3,-6.078864E-3,-2.2322569E-2,-1.2192952E-2,-6.845971E-3,-1.3111394E-3,-1.463086E-2,-6.321943E-3,1E0,6.176636E5,8.23E2,1.835821E1,2.0939393E0,1.5599597E8,2.9427118E2,5.1E1,5.1586456E7,1E1,3.82E2,4.2904656E8,-1.2324528E-2,-3.0462394E-2,3.464E0,-1.3385908E-2,2.0845033E-2,6.5702763E-3,4.3657026E0,1.9500381E2,1.3009709E0,-1.6806886E-2,-2.7759965E-3,3.53274E2,-2.3605634E-2,-1.4212878E-2,2.5454545E0,2.4645097E2,-2.4071284E-2,1.0563E4,-2.6948992E-3,8.922826E-3,-1.0214229E-2,-3.782071E-3,7.119939E-3,-3.4070218E-3,-3.6927234E-4,3.990487E5,1.2051282E0,7.24E2,-1.4373868E-4,1.977157E7,3.235955E0,1.068138E-2,-8.271929E-3,3.3786534E-3,5.513889E0,3.3238492E2,3.687715E6,-3.956408E-3,6.929741E8,8.5023944E2,1E0,4.75356E6,1.3543621E6,9.132011E6,1E0,1.7037791E-4,8.91E2,2.0939393E0,-7.8066083E-4,1.7755102E0,-6.403517E-3,-1.5202354E-2,-1.599685E-2,-2.9976934E-3,6.789622E7,-2.7671948E-3,2.7382474E-3,1.98E2,-9.336852E-4,4.521208E-3,4.5469464E-3,1.4413992E-2,1.0404834E7,6.297303E-3,5.585332E-3,1.4E1,2.35184E5,-4.079159E-3,3.53274E2,-3.1294962E-4,5.372237E2,5E0,2.1924414E2,1.2979348E3,8.155576E-3,1.766486E7,1.1485854E-2,4.1311714E-3,2.4195999E-2,4.1874755E-2,-4.9958057E-3,2.615165E-2,-1.2944418E-3,-9.381672E-3,-1.5504474E-2,-2.4640905E-2,-1.536285E-2,-4.437479E-3,-3.7726064E-3,-1.1252776E-2,-5.464779E-3,-1.3124496E-2,-7.428129E-4,-5.984501E-3,-1.7113466E-4,5.483901E-3,-3.7516936E-4,-5.1737204E-3,-8.49017E-3,-1.7347082E-2,2.1123588E-3,9.338601E-3,1.2624343E-2,2.470133E-2,-6.8272697E-3,4.5620366E-3,-6.8897502E-3,5.6473007E-3,2.1985939E-2,4.9627996E-3,2.7737483E-2,1.5430196E-2],"split_indices":[2,43,2,7,2,17,60,32,2,43,43,0,33,0,3,3,1,1,10,69,43,9,62,0,43,0,43,43,72,0,0,0,5,0,9,0,3,2,5,7,0,67,43,48,43,3,43,60,9,0,0,0,0,0,0,0,9,69,53,4,0,67,0,71,7,71,44,43,2,0,0,46,0,48,3,62,0,0,44,0,2,0,0,43,73,0,0,48,0,0,0,0,0,0,0,26,43,2,71,68,46,67,10,7,8,0,7,0,0,69,0,0,0,69,73,68,0,0,4,0,0,73,4,0,44,0,0,0,0,0,0,0,43,68,2,0,5,71,0,0,0,69,67,62,0,5,67,15,62,43,1,29,0,44,68,0,68,0,0,0,0,7,0,0,10,0,0,0,0,60,0,0,3,1,0,4,0,4,6,67,4,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,5.13E2,4.85E2,1.93E2,3.2E2,3.07E2,1.78E2,7.1E1,1.22E2,9.3E1,2.27E2,1.3E1,2.94E2,1.56E2,2.2E1,9E0,6.2E1,4.7E1,7.5E1,7E1,2.3E1,1.63E2,6.4E1,1.2E1,2.82E2,1.1E1,1.45E2,1.1E1,1.1E1,4E0,5E0,3.9E1,2.3E1,1.4E1,3.3E1,4E0,7.1E1,3.9E1,3.1E1,1.9E1,4E0,8.7E1,7.6E1,1.1E1,5.3E1,8E1,2.02E2,1.5E1,1.3E2,7E0,4E0,4E0,7E0,5E0,1.8E1,4E0,2.9E1,5.8E1,1.3E1,3.4E1,5E0,2.6E1,5E0,1E1,9E0,5.5E1,3.2E1,3E1,4.6E1,7E0,4E0,4.6E1,7E0,6.8E1,1.2E1,1.85E2,1.7E1,6E0,9E0,1E1,1.2E2,2.1E1,8E0,2.2E1,3.6E1,6E0,7E0,3E1,4E0,2.1E1,5E0,5E0,5E0,5E0,4E0,3.2E1,2.3E1,1.8E1,1.4E1,1E1,2E1,3E1,1.6E1,3.8E1,8E0,3.1E1,3.7E1,5E0,7E0,1.79E2,6E0,5E0,4E0,5.2E1,6.8E1,1.7E1,5E0,4E0,3.2E1,8E0,2.2E1,1.9E1,1.3E1,1E1,1.3E1,7E0,1.1E1,1E1,4E0,4E0,6E0,5E0,1.5E1,1.9E1,1.1E1,4E0,1.2E1,2.6E1,1.2E1,4E0,4E0,1.3E1,1.8E1,3.2E1,5E0,8.7E1,9.2E1,4.4E1,8E0,5.9E1,9E0,1.1E1,6E0,2.3E1,9E0,7E0,1.2E1,4E0,9E0,9E0,4E0,1.1E1,4E0,4E0,1.5E1,4E0,7E0,4E0,8E0,1.6E1,1E1,5E0,8E0,1.1E1,7E0,2.6E1,6E0,7.8E1,9E0,7.9E1,1.3E1,4E0,4E1,4E0,4E0,1.4E1,4.5E1,4E0,5E0,4E0,7E0,1E1,1.3E1,4E0,5E0,8E0,4E0,4E0,7E0,5E0,1E1,1.1E1,5E0,4E0,4E0,5E0,6E0,1E1,1.6E1,4.4E1,3.4E1,4E0,5E0,9E0,7E1,9E0,4E0,3.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-9.604084E-3,-1.7769752E-1,5.3377515E-1,-5.027555E-1,-1.1044416E-2,5.953061E-1,-5.064273E-1,-6.957941E-1,-3.919236E-1,-1.5918878E-1,1.365706E-1,4.4329765E-1,8.657004E-1,-3.2116305E-2,-1.0088568E-2,-6.123844E-1,-8.735108E-1,-4.9474064E-1,-9.532652E-2,-2.2376442E-1,8.322249E-2,2.804698E-1,2.5025155E-2,1.8223284E-1,6.0022587E-1,1.1787167E0,6.684661E-1,-3.4130882E-2,-3.930898E-1,-5.0244346E-2,-6.9417316E-1,-4.3455964E-1,-9.8442465E-1,3.1324722E-2,-2.0894806E-1,-2.0233789E-1,-4.786559E-2,3.7036147E-2,3.1517094E-1,3.03033E-1,-7.965279E-3,4.1885342E-2,-2.1352336E-2,3.462123E-2,2.964725E-1,4.8907918E-1,8.248178E-1,1.2883264E0,3.3090036E-2,7.35882E-1,2.6460323E-1,-5.182883E-1,1.2078336E-2,-1.9782528E-2,-3.65654E-2,-2.3568341E-1,-4.9656954E-1,-1.9125406E-2,-1.1919587E0,-1.1826896E-1,-1.704303E-2,-3.6924347E-1,-1.2643513E-1,-6.714747E-3,5.7381816E-2,1.921139E-2,6.9976863E-3,2.2981884E-1,4.318533E-1,-1.08133584E-1,8.7874055E-2,-8.603366E-3,8.323961E-2,1.94815E-1,1.9868504E-2,5.381012E-1,2.0396389E-1,4.410762E-2,2.7326044E-2,3.589514E-2,6.4087585E-2,8.632517E-1,5.22856E-1,1.7521976E-2,4.4819764E-3,-2.7962103E-2,-3.1575984E-1,-3.2417864E-1,1.1731296E-2,-5.764673E-1,-3.855013E-1,-6.996596E-2,-2.9155284E-2,-3.753598E-2,-2.3228638E-1,-1.3757024E-3,-4.1190678E-1,-1.0416744E-1,-2.2085348E-2,1.3092345E-1,2.4690751E-2,1.690638E-1,2.521731E-2,4.6240836E-1,6.361989E-3,-1.82799E-1,8.934329E-2,-1.0108118E-1,1.2350115E-1,1.4519924E-1,-3.2465826E-3,2.4155864E-1,3.3954915E-3,6.183139E-1,2.883272E-1,1.7264051E-3,1.3574328E-2,2.0680983E-2,9.18468E-1,3.2642597E-1,2.93844E-2,-1.8470675E-2,-7.7873715E-3,-4.2645237E-1,-2.4735157E-3,-4.7742063E-1,-3.3033323E-2,-2.4655715E-1,-4.4549042E-1,2.4755226E-2,-9.565652E-2,-1.5569443E-2,-5.742614E-3,-2.6252744E-1,-4.7748247E-1,-6.2056348E-2,-2.1061331E-1,3.3671854E-3,8.615587E-3,-2.0534683E-2,9.800891E-2,-3.6841254E-3,1.8792307E-1,9.079898E-3,5.0147074E-1,2.8283142E-3,-2.468648E-1,-2.4922558E-3,8.956172E-3,-1.089296E-2,6.0731443E-3,8.775925E-2,3.5744572E-1,-3.2624108E-4,2.1505071E-1,4.6179355E-3,1.4281034E-2,8.376324E-3,6.618336E-1,7.028194E-3,1.7947633E-2,4.588095E-2,2.1530917E-2,2.0102886E-2,7.0421374E-3,-2.2482986E-2,-7.973739E-3,-2.3983054E-2,-1.0031399E-2,-1.3448424E-2,-4.9971286E-3,-9.1125015E-3,-4.816604E-1,-1.1962886E-3,3.2548907E-3,-6.571131E-3,-1.3835018E-3,-3.20505E-1,-5.4215016E-3,-1.1348064E-2,-5.432325E-1,-1.04204014E-1,4.9513765E-2,-1.1366056E-1,-1.2545401E-2,-4.2906925E-3,2.5561439E-2,6.4209225E-3,5.9352885E-4,1.505344E-1,3.1558603E-1,2.4542887E-2,1.1656909E-2,-1.3030397E-2,-3.9224033E-3,5.7654423E-3,-3.6772718E-3,1.1004653E-1,-6.1053853E-3,2.3943515E-2,9.805783E-3,4.3220334E-3,1.2525193E-2,2.3565728E-2,3.500608E-2,-2.5173325E-2,-1.2396623E-2,-1.926584E-2,-9.612037E-3,-3.0312736E-2,-1.771572E-2,-2.2826523E-3,-9.571008E-3,9.623021E-3,-7.610982E-4,-9.670485E-3,-7.391419E-4,3.0173867E-3,-1.4645348E-3,7.753292E-3,3.3735766E-4,2.1841483E-2,6.2726284E-3,1.3855398E-2,3.977356E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,53,55,57,-1,59,61,-1,63,65,67,-1,69,-1,71,73,75,77,79,-1,81,83,85,-1,-1,-1,87,89,-1,91,93,-1,95,97,-1,99,-1,-1,101,103,105,107,-1,109,111,-1,113,115,-1,-1,-1,-1,117,119,-1,-1,-1,121,123,-1,125,127,-1,-1,129,131,-1,133,135,-1,137,139,141,-1,143,-1,145,147,149,151,153,-1,155,-1,157,159,-1,-1,-1,161,163,-1,-1,-1,165,-1,167,-1,169,171,173,175,-1,-1,177,179,181,183,-1,-1,185,187,-1,189,-1,191,-1,193,-1,-1,-1,195,197,199,-1,201,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,205,-1,-1,-1,-1,207,-1,-1,209,211,213,215,-1,-1,217,-1,-1,219,221,-1,-1,-1,-1,-1,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.316876E1,4.2194374E1,1.5726601E1,5.3755417E0,1.13058E1,8.987541E0,6.0936093E-1,8.73291E-1,5.1370296E0,4.0573187E0,4.1502686E0,5.953951E0,4.2128067E0,0E0,0E0,1.5943947E0,3.198948E-1,3.2853851E0,4.0958652E0,3.3409758E0,5.7753414E-1,1.2069321E0,1.2268742E0,9.509324E-1,1.8931618E0,5.979614E-1,1.221077E0,0E0,2.3480525E0,0E0,1.2691975E-1,1.2888489E0,1.0455866E0,0E0,5.365578E-1,2.487033E0,0E0,1.8274027E-1,8.533102E-2,9.193506E-1,0E0,9.960845E-1,0E0,2.9466662E-1,3.5079265E-1,7.8477E-1,3.1436348E-1,8.51326E-2,0E0,7.424984E-1,1.3615894E-1,2.1397972E-1,0E0,0E0,0E0,1.3388417E0,5.4094315E-1,0E0,6.721506E-1,2.3981497E-1,0E0,8.8566494E-1,1.0385175E0,0E0,1.00146025E-1,0E0,0E0,1.2709115E0,2.7785206E-1,5.1765144E-1,7.5299567E-1,0E0,2.1945965E-1,9.566301E-2,0E0,9.036169E-1,1.5369049E-1,0E0,0E0,0E0,0E0,1.8549728E-1,2.1479893E-1,0E0,0E0,0E0,4.395187E-2,6.69147E-1,0E0,3.1735516E-1,2.1523142E-1,0E0,0E0,6.3914776E-2,8.2500994E-2,0E0,4.1340733E-1,5.786269E-1,0E0,2.8332159E-2,1.04803786E-1,3.0121863E-1,0E0,2.5454903E-1,0E0,4.0594524E-1,1.6022004E-1,2.6124227E-1,7.4626195E-1,1.71689E-1,0E0,1.0303539E-1,0E0,6.407356E-1,1.4552057E-1,0E0,0E0,0E0,2.8013992E-1,1.039083E-1,0E0,0E0,0E0,1.9109583E-1,0E0,2.2378206E-1,0E0,5.480963E-2,1.3845062E-1,2.2655735E-2,2.3286693E-2,0E0,0E0,1.22310996E-1,4.3059254E-1,4.5659184E-1,1.8773448E-1,0E0,0E0,6.906527E-2,4.0088408E-2,0E0,2.2456312E-1,0E0,8.106756E-2,0E0,9.0023994E-2,0E0,0E0,0E0,1.2181306E-1,4.1644692E-1,1.7020512E-1,0E0,4.9563766E-2,0E0,0E0,0E0,1.3730907E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.505351E-1,0E0,0E0,0E0,0E0,5.4321647E-2,0E0,0E0,2.5345707E-1,3.8470095E-1,2.8632823E-1,1.4613031E-1,0E0,0E0,3.1661548E-2,0E0,0E0,1.1268771E-1,2.7497137E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.1528503E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,34,34,35,35,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,55,55,56,56,58,58,59,59,61,61,62,62,64,64,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,81,81,82,82,86,86,87,87,89,89,90,90,93,93,94,94,96,96,97,97,99,99,100,100,101,101,103,103,105,105,106,106,107,107,108,108,109,109,111,111,113,113,114,114,118,118,119,119,123,123,125,125,127,127,128,128,129,129,130,130,133,133,134,134,135,135,136,136,139,139,140,140,142,142,144,144,146,146,150,150,151,151,152,152,154,154,158,158,172,172,177,177,180,180,181,181,182,182,183,183,186,186,189,189,190,190,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,54,56,58,-1,60,62,-1,64,66,68,-1,70,-1,72,74,76,78,80,-1,82,84,86,-1,-1,-1,88,90,-1,92,94,-1,96,98,-1,100,-1,-1,102,104,106,108,-1,110,112,-1,114,116,-1,-1,-1,-1,118,120,-1,-1,-1,122,124,-1,126,128,-1,-1,130,132,-1,134,136,-1,138,140,142,-1,144,-1,146,148,150,152,154,-1,156,-1,158,160,-1,-1,-1,162,164,-1,-1,-1,166,-1,168,-1,170,172,174,176,-1,-1,178,180,182,184,-1,-1,186,188,-1,190,-1,192,-1,194,-1,-1,-1,196,198,200,-1,202,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,206,-1,-1,-1,-1,208,-1,-1,210,212,214,216,-1,-1,218,-1,-1,220,222,-1,-1,-1,-1,-1,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.2116136E5,4.217427E7,9.9E1,9.31E2,1.5142261E3,1.0201538E-1,1.4504054E8,8.57E2,2.331083E6,3.21875E0,9.155886E5,2.3601625E1,-3.2116305E-2,-1.0088568E-2,3.89E2,3.3817584E7,9.932432E0,2.04115E5,1.1290322E1,7.168071E6,1.4676277E7,2.673585E1,4.032683E6,4.391553E6,1.4595818E7,2.856934E7,-3.4130882E-2,1.1634076E4,-5.0244346E-2,4.217427E7,1.3208092E0,2.49E2,3.1324722E-2,1.6942337E1,1.7934177E2,-4.786559E-2,3.72E2,3.52E2,5.144203E2,-7.965279E-3,1.221E3,-2.1352336E-2,8.295515E7,1.29443414E5,2E0,1.4598765E0,1E1,3.3090036E-2,2.171312E0,2.2365898E3,1.9363927E-6,1.2078336E-2,-1.9782528E-2,-3.65654E-2,6.47E2,1.339646E6,-1.9125406E-2,6.8E1,4.3E1,-1.704303E-2,9E0,1.7479E4,-6.714747E-3,1.2E1,1.921139E-2,6.9976863E-3,6.0052995E6,2E0,1.5115256E6,2.4080281E2,-8.603366E-3,5.8485405E1,6.364486E0,1.9868504E-2,2E0,8E0,4.410762E-2,2.7326044E-2,3.589514E-2,6.4087585E-2,1.8697667E3,7.958E3,1.7521976E-2,4.4819764E-3,-2.7962103E-2,5.5E1,1.302273E6,1.1731296E-2,1.88E2,9.263158E0,-6.996596E-2,-2.9155284E-2,4.046E3,5.2767E4,-1.3757024E-3,3.2E1,2.1851852E0,-2.2085348E-2,1.1E1,2.7840833E2,3.7760816E5,2.521731E-2,2.9816E4,6.361989E-3,1.9095E4,4.7563504E7,2.0227273E0,2.295413E6,1.8471494E7,-3.2465826E-3,1.0784811E3,3.3954915E-3,2.0750147E8,8.5023944E2,1.7264051E-3,1.3574328E-2,2.0680983E-2,2.5842668E7,2E0,2.93844E-2,-1.8470675E-2,-7.7873715E-3,6.666667E-1,-2.4735157E-3,2.7976523E2,-3.3033323E-2,4.5087484E5,3.9961785E-2,2.7777777E0,1.1770261E8,-1.5569443E-2,-5.742614E-3,1E0,1.1010101E0,5E0,9.538462E0,3.3671854E-3,8.615587E-3,1.3071136E3,1E0,-3.6841254E-3,7.6569915E-1,9.079898E-3,1.8809018E0,2.8283142E-3,4.88E2,-2.4922558E-3,8.956172E-3,-1.089296E-2,1.1631579E1,9.639872E0,1.4E1,-3.2624108E-4,2.848973E6,4.6179355E-3,1.4281034E-2,8.376324E-3,6.255748E6,7.028194E-3,1.7947633E-2,4.588095E-2,2.1530917E-2,2.0102886E-2,7.0421374E-3,-2.2482986E-2,-7.973739E-3,-2.3983054E-2,-1.0031399E-2,-1.3448424E-2,-4.9971286E-3,-9.1125015E-3,1.5292561E2,-1.1962886E-3,3.2548907E-3,-6.571131E-3,-1.3835018E-3,8.3E1,-5.4215016E-3,-1.1348064E-2,4.795239E7,1E0,2.3786941E4,5.0857143E0,-1.2545401E-2,-4.2906925E-3,1.880118E-3,6.4209225E-3,5.9352885E-4,1.8E1,1.124641E0,2.4542887E-2,1.1656909E-2,-1.3030397E-2,-3.9224033E-3,5.7654423E-3,-3.6772718E-3,6.4722223E0,-6.1053853E-3,2.3943515E-2,9.805783E-3,4.3220334E-3,1.2525193E-2,2.3565728E-2,3.500608E-2,-2.5173325E-2,-1.2396623E-2,-1.926584E-2,-9.612037E-3,-3.0312736E-2,-1.771572E-2,-2.2826523E-3,-9.571008E-3,9.623021E-3,-7.610982E-4,-9.670485E-3,-7.391419E-4,3.0173867E-3,-1.4645348E-3,7.753292E-3,3.3735766E-4,2.1841483E-2,6.2726284E-3,1.3855398E-2,3.977356E-3],"split_indices":[2,43,60,44,2,67,53,46,2,43,69,43,71,0,0,1,7,69,5,69,43,62,68,60,43,62,60,0,43,0,60,68,2,0,71,70,0,44,0,4,0,2,0,7,48,6,69,3,0,57,67,52,0,0,0,2,9,0,0,8,0,3,44,0,3,0,0,43,6,43,67,0,73,69,0,32,8,0,0,0,0,67,2,0,0,0,0,9,0,10,73,0,0,44,1,0,10,68,0,3,67,43,0,1,0,12,7,68,43,5,0,4,0,7,67,0,0,0,62,32,0,0,0,71,0,4,0,60,53,71,7,0,0,8,68,8,73,0,0,48,26,0,57,0,57,0,4,0,0,0,73,69,3,0,66,0,0,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,67,0,0,0,0,12,0,0,7,26,62,69,0,0,53,0,0,3,53,0,0,0,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.78E2,2.4E2,2.63E2,5.15E2,2.27E2,1.3E1,9.4E1,1.69E2,2.57E2,2.58E2,1.47E2,8E1,7E0,6E0,6.7E1,2.7E1,1.25E2,4.4E1,2.03E2,5.4E1,1.12E2,1.46E2,5.6E1,9.1E1,2.9E1,5.1E1,4.1E1,2.6E1,1E1,1.7E1,1.13E2,1.2E1,5E0,3.9E1,1.99E2,4E0,4.6E1,8E0,1.07E2,5E0,1.42E2,4E0,2.5E1,3.1E1,6.3E1,2.8E1,2.2E1,7E0,4.3E1,8E0,2.2E1,4E0,6E0,1.1E1,2.8E1,8.5E1,4E0,8E0,2.6E1,1.3E1,6.1E1,1.38E2,4E0,4.2E1,4E0,4E0,7E1,3.7E1,3.3E1,1.09E2,4E0,2.1E1,1.9E1,1.2E1,5.3E1,1E1,1.6E1,1.2E1,5E0,1.7E1,2.5E1,1.8E1,4E0,4E0,1.4E1,8E0,2.4E1,4E0,4.7E1,3.8E1,4E0,4E0,1.6E1,1E1,7E0,5.4E1,1.31E2,7E0,1.2E1,3E1,6E1,1E1,3.3E1,4E0,2.4E1,9E0,1.7E1,9.2E1,1.5E1,6E0,1.3E1,6E0,3.9E1,1.4E1,4E0,6E0,4E0,2.1E1,8E0,1E1,4E0,4E0,1.7E1,7E0,3E1,1.7E1,1.3E1,2.5E1,8E0,8E0,4E0,6E0,1.8E1,3.6E1,9.5E1,3.6E1,7E0,5E0,1.9E1,1.1E1,4E0,5.6E1,5E0,2.8E1,5E0,1.9E1,4E0,5E0,7E0,1E1,8.1E1,1.1E1,5E0,1E1,5E0,8E0,4E0,3.5E1,7E0,7E0,1.7E1,4E0,4E0,4E0,1.3E1,4E0,2.5E1,5E0,9E0,4E0,4E0,2.1E1,4E0,4E0,4E0,4E0,1.2E1,6E0,9E0,2.7E1,6.9E1,2.6E1,1.5E1,2.1E1,7E0,1.2E1,7E0,4E0,4.5E1,1.1E1,2.4E1,4E0,1.5E1,4E0,4E0,6E0,7.4E1,7E0,4E0,7E0,4E0,6E0,1.6E1,1.9E1,1.5E1,6E0,5E0,7E0,1.4E1,1.3E1,4.6E1,2.3E1,7E0,1.9E1,7E0,8E0,7E0,5E0,4E1,5E0,5E0,6E0,7E0,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[6.180792E-3,-1.6503148E-1,4.7275862E-1,-5.0639033E-1,-1.129815E-2,-6.803958E-1,5.4719776E-1,1.8274754E-2,-5.280954E-1,-1.6625014E-1,1.1087337E-1,-3.9352454E-2,-1.7605912E-2,1.3548586E0,4.8434323E-1,-6.065767E-1,-3.0085424E-1,-1.2424855E-1,-5.163365E-1,4.6761047E-2,2.686597E-1,4.5754105E-2,7.792276E-2,3.3147123E-1,7.643023E-1,-5.4452336E-1,-1.0120652E0,-3.3467343E-1,6.379688E-3,-1.7157865E-1,6.810231E-2,-5.913036E-1,-5.229772E-3,8.2113095E-2,-2.3808844E-1,2.206505E-1,2.2615165E-2,1.6233137E-1,4.803148E-1,7.994446E-1,2.2713481E-3,-6.016546E-1,-3.5165614E-1,-2.0412194E-2,-1.2107033E0,-4.5247874E-1,-2.5665715E-1,-3.8528496E-1,-1.127225E-1,1.734763E-1,1.8517341E-2,-6.655063E-1,-1.4194483E-2,1.07354075E-1,-1.8962681E-1,-2.392149E-2,-8.3356254E-2,1.07245244E-1,3.375724E-1,3.3497408E-1,5.235329E-2,5.8765614E-1,2.4847795E-1,5.636321E-1,9.4491065E-1,-6.7435145E-1,-4.8269066E-1,-4.9582702E-1,-8.968568E-2,-7.161074E-2,-3.326427E-2,-2.2630611E-2,-1.0853086E-2,-3.0224049E-3,-3.0056164E-1,-3.0194435E-1,-5.634421E-1,-7.476078E-2,-2.0266244E-1,1.4698161E-3,1.1921103E-2,-6.736173E-3,5.0641637E-2,-2.244549E-2,-4.2857196E-2,6.4812E-3,1.5728682E-1,-2.97582E-1,1.8431344E-3,-1.4241126E-2,1.729825E-2,-2.1520426E-2,1.6255073E-1,3.9851704E-1,1.5145347E-1,2.513066E-2,2.1373E-1,-3.0838272E-1,1.3425767E-1,4.5744762E-1,8.594612E-1,3.4686705E-1,-7.858146E-3,1.4932823E-2,6.2763226E-1,4.9509685E-2,3.5136476E-2,-3.2075506E-2,-1.3317501E-2,-8.192813E-4,-5.5199087E-1,-1.1377268E-2,-2.473519E-2,-8.901832E-3,1.886736E-3,-3.4619373E-1,-6.3525126E-3,-2.2607056E-2,-2.3402435E-1,-3.4160864E-2,-1.4916765E-2,-1.6875029E-1,-2.025404E-2,-8.9234844E-2,-3.0028555E-1,1.2991291E-1,-1.0866737E-2,4.3241512E-2,-1.776235E-2,-7.973841E-3,2.1443602E-1,-7.090227E-3,-1.8114451E-2,6.0518007E-3,-5.462109E-3,4.420927E-3,-9.4596185E-3,2.274301E-1,5.3403392E-2,2.3255488E-1,4.7733608E-1,1.9099436E-3,1.1702081E-2,2.999682E-1,1.1590164E-1,-3.993294E-3,-2.1651287E-2,3.341527E-1,2.1458121E-2,7.0477296E-3,5.174998E-1,4.65909E-2,2.6137244E-2,2.1844682E-1,2.5887595E-2,3.205296E-2,1.3973525E-2,-5.954478E-1,-5.945938E-3,-2.4844922E-1,-1.8800398E-2,-5.5004717E-3,-1.2569496E-2,-7.12295E-4,-1.8868068E-1,-8.481989E-2,2.828785E-2,-1.4959584E-1,5.0678095E-3,-3.65702E-1,-7.2277533E-3,8.353372E-3,1.5093553E-3,-4.2204354E-3,1.7792116E-3,-4.433357E-2,1.2921506E-1,-8.109424E-2,7.055284E-2,2.3125322E-1,-4.505623E-3,1.7135364E-1,1.4623275E-2,4.7639236E-3,-2.4074948E-4,4.0995292E-3,1.6568195E-2,1.2153308E-2,2.5447248E-2,1.6693758E-2,7.683984E-3,-3.5270027E-6,9.820194E-3,2.3110531E-1,2.1871414E-2,-9.265563E-3,7.094084E-2,5.201964E-3,5.694655E-1,3.5538576E-3,2.592104E-1,-2.043926E-2,-3.1079069E-2,-5.291665E-3,-1.4257282E-2,-1.2106815E-2,-4.519799E-3,1.220963E-3,-7.2071347E-3,5.0935005E-3,-8.90217E-4,-9.215974E-3,-1.9180061E-3,-1.2051613E-2,-2.2234317E-2,-4.1192146E-3,4.457565E-3,9.330464E-3,2.3287753E-4,5.383695E-4,-7.138954E-3,-2.3208114E-3,5.4996517E-3,3.7311022E-3,1.22307185E-2,4.3693925E-3,9.825735E-3,1.3281742E-2,5.4156007E-3,-3.6526588E-3,5.0203376E-3,2.8532118E-2,1.5245063E-2,7.564245E-3,1.6637865E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,-1,47,49,51,-1,53,55,57,-1,59,61,63,-1,65,67,-1,69,71,73,75,77,79,81,83,-1,85,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,115,117,119,121,123,-1,-1,-1,125,-1,-1,127,129,131,-1,-1,133,135,137,139,141,-1,143,145,147,149,151,153,-1,-1,155,-1,-1,-1,-1,-1,157,-1,-1,-1,-1,159,-1,-1,161,-1,-1,163,165,167,169,171,173,175,-1,177,179,-1,-1,-1,-1,-1,-1,181,183,185,187,-1,-1,189,191,-1,-1,193,195,-1,197,-1,-1,199,-1,-1,-1,201,-1,203,-1,-1,-1,-1,205,207,209,211,-1,213,-1,-1,-1,-1,-1,215,217,219,221,223,-1,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,227,-1,-1,229,-1,231,-1,233,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.1240005E1,3.9038086E1,2.3806591E1,4.731186E0,9.748823E0,5.475435E-1,1.2390434E1,0E0,3.84803E0,3.273695E0,2.9008594E0,0E0,0E0,5.0655365E-1,1.0018154E1,3.65094E0,9.3690205E-1,1.8688982E0,6.5838957E-1,2.093973E0,7.720475E-1,0E0,0E0,3.8745651E0,2.0717506E0,1.408184E0,1.8437462E0,3.9854956E-1,0E0,2.019351E0,2.1101129E-1,2.170825E-1,0E0,1.2824098E0,9.5225835E-1,8.9196634E-1,0E0,1.4064816E0,1.9194889E0,2.2116356E0,0E0,6.4310455E-1,1.315198E0,0E0,1.3815918E0,5.5664062E-2,2.825966E-1,3.4954214E-1,4.287045E-1,1.4751896E-1,1.604698E-1,2.5580263E-1,0E0,8.502102E-1,4.0832222E-1,0E0,3.6752677E-1,2.6699176E-1,3.2530355E-1,6.1856604E-1,1.4300581E0,1.5887585E0,1.2121193E0,3.048668E-1,3.0662918E-1,3.156147E-1,1.4724045E0,9.531975E-2,1.9608359E-1,0E0,0E0,0E0,0E0,0E0,1.681776E-1,2.1640873E-1,1.8460774E-1,4.7302693E-1,3.9398372E-1,0E0,0E0,0E0,1.23307906E-1,0E0,0E0,8.358092E-1,1.0689397E0,8.964944E-2,0E0,0E0,2.0039521E-1,2.773286E-1,1.7197114E-1,2.1601772E-1,1.0167009E-1,0E0,1.3396406E-1,2.7963167E-1,8.722045E-1,6.5344334E-1,1.7351437E-1,5.0814795E-1,0E0,0E0,3.2805347E-1,0E0,0E0,0E0,0E0,0E0,6.591568E-1,0E0,0E0,0E0,0E0,2.9530048E-2,0E0,0E0,5.946493E-2,0E0,0E0,1.0099387E-1,1.9044825E-1,2.445545E-1,1.3097334E-1,4.8538357E-2,6.313289E-2,4.04773E-1,0E0,1.7797852E-1,4.6324515E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.5702178E-2,3.2590926E-2,1.4949113E-1,1.0702038E-1,0E0,0E0,2.1925747E-2,1.2333231E-1,0E0,0E0,9.150219E-2,2.9986197E-1,0E0,6.3022995E-1,0E0,0E0,7.236326E-2,0E0,0E0,0E0,1.2808132E-1,0E0,5.3262115E-2,0E0,0E0,0E0,0E0,1.7434561E-1,2.100132E-1,1.3855205E-1,7.259676E-2,0E0,3.1965733E-2,0E0,0E0,0E0,0E0,0E0,1.790477E-1,2.4288192E-1,1.1077051E-1,9.3834616E-2,3.8177347E-1,0E0,1.796195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.051114E-2,0E0,0E0,1.3309339E-1,0E0,1.3042927E-1,0E0,5.6667805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,74,74,75,75,76,76,77,77,78,78,82,82,85,85,86,86,87,87,90,90,91,91,92,92,93,93,94,94,96,96,97,97,98,98,99,99,100,100,101,101,104,104,110,110,115,115,118,118,121,121,122,122,123,123,124,124,125,125,126,126,127,127,129,129,130,130,137,137,138,138,139,139,140,140,143,143,144,144,147,147,148,148,150,150,153,153,157,157,159,159,164,164,165,165,166,166,167,167,169,169,175,175,176,176,177,177,178,178,179,179,181,181,193,193,196,196,198,198,200,200],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,-1,48,50,52,-1,54,56,58,-1,60,62,64,-1,66,68,-1,70,72,74,76,78,80,82,84,-1,86,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,116,118,120,122,124,-1,-1,-1,126,-1,-1,128,130,132,-1,-1,134,136,138,140,142,-1,144,146,148,150,152,154,-1,-1,156,-1,-1,-1,-1,-1,158,-1,-1,-1,-1,160,-1,-1,162,-1,-1,164,166,168,170,172,174,176,-1,178,180,-1,-1,-1,-1,-1,-1,182,184,186,188,-1,-1,190,192,-1,-1,194,196,-1,198,-1,-1,200,-1,-1,-1,202,-1,204,-1,-1,-1,-1,206,208,210,212,-1,214,-1,-1,-1,-1,-1,216,218,220,222,224,-1,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,228,-1,-1,230,-1,232,-1,234,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.972052E5,1E0,6E0,8.1E2,1.5900173E3,6.743651E7,1.8274754E-2,1.559733E6,6.2233735E-6,2.0766992E6,-3.9352454E-2,-1.7605912E-2,2.35184E5,2.8530578E6,4.6463413E0,5.185192E2,3.2911258E6,1.3E1,2.3E1,6.0052995E6,4.5754105E-2,7.792276E-2,5.709E3,6.292039E0,1.4912975E-1,9E0,1.901875E2,6.379688E-3,1.3920766E2,5.894636E6,7.714286E0,-5.229772E-3,2.1129E4,5.095006E5,4.4935583E2,2.2615165E-2,3.53969E5,4.05E2,1.2400667E3,2.2713481E-3,8.7601E4,8.01E2,-2.0412194E-2,1.753E3,8.390338E7,1.5522388E0,1.1244019E2,2.165E3,4.49239E6,2.234E3,5.4E1,-1.4194483E-2,1.295313E6,3.3704692E10,-2.392149E-2,4.236E3,2.41E2,7.7630096E2,4E0,7.70947E5,3.4915986E3,4.1325716E7,1.0950326E6,1.6063418E7,3.1530054E0,1.5E1,5.4009717E3,4.6750406E4,-7.161074E-2,-3.326427E-2,-2.2630611E-2,-1.0853086E-2,-3.0224049E-3,1.3999298E5,1.02E2,2.8024E4,5.47E2,1.9637654E2,1.4698161E-3,1.1921103E-2,-6.736173E-3,3.5364E4,-2.244549E-2,-4.2857196E-2,1.28636E7,4.8709216E5,1.97525E1,1.8431344E-3,-1.4241126E-2,5.857143E0,2.8055556E0,1.28636E7,5.281628E2,1.8645384E1,2.513066E-2,3.3382E4,1.4283589E6,3.1226995E0,9.49372E4,2.1298597E0,4.435876E2,-7.858146E-3,1.4932823E-2,1.8201E4,4.9509685E-2,3.5136476E-2,-3.2075506E-2,-1.3317501E-2,-8.192813E-4,2E1,-1.1377268E-2,-2.473519E-2,-8.901832E-3,1.886736E-3,1.6E1,-6.3525126E-3,-2.2607056E-2,2.0487332E-3,-3.4160864E-2,-1.4916765E-2,2.16E2,3.0283334E2,1.4390917E8,7.54E2,1.422162E3,1.0079E5,8.6363555E4,-1.776235E-2,1.387E3,1E0,-7.090227E-3,-1.8114451E-2,6.0518007E-3,-5.462109E-3,4.420927E-3,-9.4596185E-3,8.051603E7,1.1716E4,6.965855E9,2.36433E6,1.9099436E-3,1.1702081E-2,6.941074E6,4.8709216E5,-3.993294E-3,-2.1651287E-2,2.379747E0,2.9585715E2,7.0477296E-3,1.36E2,4.65909E-2,2.6137244E-2,4.6750406E4,2.5887595E-2,3.205296E-2,1.3973525E-2,2.6210527E0,-5.945938E-3,2.6699028E0,-1.8800398E-2,-5.5004717E-3,-1.2569496E-2,-7.12295E-4,4.6E1,5.5555553E0,1.8949389E6,1.5422421E2,5.0678095E-3,5.494667E2,-7.2277533E-3,8.353372E-3,1.5093553E-3,-4.2204354E-3,1.7792116E-3,1.47176E5,1.1362371E6,5E0,2.8846003E5,5.1604336E7,-4.505623E-3,6.009E4,1.4623275E-2,4.7639236E-3,-2.4074948E-4,4.0995292E-3,1.6568195E-2,1.2153308E-2,2.5447248E-2,1.6693758E-2,7.683984E-3,-3.5270027E-6,9.820194E-3,6.968415E6,2.1871414E-2,-9.265563E-3,1.345672E6,5.201964E-3,1.6007428E7,3.5538576E-3,1E0,-2.043926E-2,-3.1079069E-2,-5.291665E-3,-1.4257282E-2,-1.2106815E-2,-4.519799E-3,1.220963E-3,-7.2071347E-3,5.0935005E-3,-8.90217E-4,-9.215974E-3,-1.9180061E-3,-1.2051613E-2,-2.2234317E-2,-4.1192146E-3,4.457565E-3,9.330464E-3,2.3287753E-4,5.383695E-4,-7.138954E-3,-2.3208114E-3,5.4996517E-3,3.7311022E-3,1.22307185E-2,4.3693925E-3,9.825735E-3,1.3281742E-2,5.4156007E-3,-3.6526588E-3,5.0203376E-3,2.8532118E-2,1.5245063E-2,7.564245E-3,1.6637865E-2],"split_indices":[2,43,17,3,2,70,7,0,9,52,43,0,0,1,43,68,67,43,8,3,43,0,0,2,53,53,0,4,0,70,60,71,0,44,43,70,0,9,8,4,0,9,2,0,44,7,68,67,44,43,12,0,0,9,46,0,12,10,4,8,9,4,66,62,62,68,0,43,48,0,0,0,0,0,43,2,12,2,67,0,0,0,9,0,0,60,43,73,0,0,68,71,60,4,71,0,9,43,69,48,53,71,0,0,10,0,0,0,0,0,6,0,0,0,0,3,0,0,53,0,0,44,4,5,2,48,9,48,0,2,79,0,0,0,0,0,0,7,44,46,43,0,0,60,43,0,0,69,67,0,0,0,0,48,0,0,0,71,0,68,0,0,0,0,0,73,60,67,0,48,0,0,0,0,0,1,43,8,43,7,0,1,0,0,0,0,0,0,0,0,0,0,0,9,0,0,9,0,66,0,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,7.43E2,2.72E2,2.3E2,5.13E2,1.6E1,2.56E2,5E0,2.25E2,2.26E2,2.87E2,9E0,7E0,1.7E1,2.39E2,1.66E2,5.9E1,2.03E2,2.3E1,2.05E2,8.2E1,1E1,7E0,1.56E2,8.3E1,1.46E2,2E1,5.5E1,4E0,1.63E2,4E1,1.9E1,4E0,1.83E2,2.2E1,6.9E1,1.3E1,7.4E1,8.2E1,7.9E1,4E0,1.11E2,3.5E1,6E0,1.4E1,2E1,3.5E1,3.4E1,1.29E2,1.2E1,2.8E1,1.4E1,5E0,1.68E2,1.5E1,7E0,1.5E1,3.6E1,3.3E1,2.8E1,4.6E1,5.5E1,2.7E1,3.2E1,4.7E1,6.6E1,4.5E1,2.2E1,1.3E1,7E0,7E0,1.6E1,4E0,7E0,2.8E1,2.5E1,9E0,9.2E1,3.7E1,5E0,7E0,4E0,2.4E1,1E1,4E0,5.6E1,1.12E2,1E1,5E0,4E0,1.1E1,1.1E1,2.5E1,2.4E1,9E0,9E0,1.9E1,8E0,3.8E1,3.9E1,1.6E1,2.2E1,5E0,8E0,2.4E1,2.5E1,2.2E1,6.2E1,4E0,6E0,3.9E1,4E0,1.8E1,7E0,6E0,2.1E1,7E0,5E0,2E1,4E0,5E0,3.3E1,5.9E1,1.8E1,1.9E1,1E1,1.4E1,5.2E1,4E0,2.9E1,8.3E1,5E0,5E0,6E0,5E0,7E0,4E0,1.5E1,1E1,9E0,1.5E1,5E0,4E0,9E0,1E1,4E0,4E0,1.3E1,2.5E1,7E0,3.2E1,9E0,7E0,1.5E1,7E0,1.9E1,5E0,3.5E1,4E0,1E1,1.1E1,6E0,1.4E1,4E0,2.9E1,2.5E1,3.4E1,1.4E1,4E0,1.2E1,7E0,6E0,4E0,5E0,9E0,2.6E1,2.6E1,1.5E1,1.4E1,7.9E1,4E0,1.1E1,4E0,5E0,5E0,5E0,4E0,5E0,1E1,5E0,4E0,5E0,5E0,9E0,4E0,4E0,2.1E1,4E0,2.8E1,4E0,1.1E1,1.4E1,2.1E1,4E0,6E0,1.5E1,1.4E1,1E1,1.5E1,1.2E1,2.2E1,9E0,5E0,8E0,4E0,2E1,6E0,1.6E1,1E1,7E0,8E0,4E0,1E1,1.5E1,6.4E1,5E0,6E0,5E0,4E0,4E0,1.7E1,2.2E1,6E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"235","size_leaf_vector":"1"}},{"base_weights":[1.4377204E-2,-1.5660624E-1,4.9529484E-1,-4.8663077E-1,-2.1084787E-3,-5.829388E-1,5.4487056E-1,2.9609978E-1,-5.1911163E-1,-1.6130061E-1,1.134766E-1,-3.1639814E-2,-1.4011223E-2,1.3125441E0,4.939846E-1,3.288987E-2,-4.6090093E-3,-6.983173E-1,-4.0818495E-1,-2.4788819E-1,6.9673453E-3,7.799139E-2,4.5886874E-1,3.4380168E-2,6.706123E-2,3.5069853E-1,7.50795E-1,-6.149155E-1,-8.846754E-1,-4.7176003E-1,3.2110855E-3,-3.9059E-1,-1.7390017E-1,-2.7103266E-1,5.5396438E-2,1.4779969E-1,-1.6382823E-2,2.3503827E-1,2.5766423E-2,9.7157516E-2,4.5448008E-1,5.896566E-1,8.7109065E-1,-6.482583E-1,-9.039893E-3,-2.1030493E-2,-9.478067E-1,-4.1875008E-1,-8.000923E-1,1.7956136E-2,-1.5118612E-1,-2.8482133E-1,-4.650639E-1,-2.5084972E-1,-5.3899504E-2,-5.529362E-4,-2.2403058E-2,-4.924974E-3,1.7627378E-1,9.2411615E-2,2.7075124E-1,2.0955662E-1,-4.5018412E-2,2.8565787E-3,1.8289194E-2,2.7418974E-1,-6.251715E-2,4.8526943E-1,-7.1584913E-3,1.265036E-2,2.9511178E-2,9.343002E-1,1.7807515E-2,-3.1096052E-2,-1.731282E-2,-4.7187775E-2,-2.4150196E-2,-5.386295E-1,-2.7420583E-1,-1.7646331E-2,-1.0127735E0,-1.3990566E-2,-4.50798E-2,-3.202222E-1,-7.5404365E-3,-1.1275629E-2,-2.33795E-2,-1.4055581E-1,-3.3882025E-1,-1.10522784E-1,5.1063456E-2,5.1522065E-2,-7.241517E-2,2.7589345E-1,9.3746796E-2,1.4835468E-1,-1.3073278E-2,3.2628033E-1,-1.2055591E-2,1.520202E-2,-2.1922132E-3,-1.2874891E-1,4.624318E-3,1.860571E-2,1.3351272E-1,3.8614534E-3,-1.13053136E-1,4.25396E-1,8.7166077E-1,2.323293E-2,4.4732958E-2,-3.0387348E-2,-4.7319612E-1,5.3308133E-2,-3.6560488E-1,-6.0503263E-2,-2.7562568E-2,2.1564204E-3,-6.7529925E-3,-8.010528E-3,-1.6387561E-2,3.213173E-2,-2.4402682E-1,-4.5196897E-1,-2.0449527E-1,-5.7164866E-2,-1.247199E-2,-3.1442575E-3,1.055183E-1,-3.3468108E-3,1.0682799E-1,3.0929053E-3,-1.1144002E-1,1.7358132E-2,5.5846944E-3,9.972215E-3,1.0431361E-2,2.3153735E-2,2.0525607E-1,1.2611057E-1,-1.9242454E-1,1.7359488E-2,8.3915464E-4,3.5793418E-3,-2.3604223E-1,4.1813046E-2,-1.4734697E-1,4.511797E-4,8.970255E-3,-7.2414996E-3,-2.0755947E-1,-3.7772248E-3,4.4937763E-1,4.681434E-2,2.4832327E-2,-2.2883609E-2,-1.3626187E-2,1.165859E-2,-3.5924413E-3,-2.906079E-1,-2.3053942E-2,5.409874E-3,-1.53148E-3,-1.3219251E-2,-3.960894E-3,-7.624876E-3,-5.33857E-1,-1.5479107E-3,-2.5869983E-1,-4.8279176E-3,3.266586E-3,1.8617671E-3,6.5732696E-3,5.6622062E-2,8.751726E-3,-6.577063E-3,-2.3720202E-3,-2.4738437E-3,3.303116E-3,9.3819596E-2,-5.593421E-3,4.828695E-2,2.562378E-1,-3.7309334E-3,1.9170468E-1,-1.9731954E-2,-2.5587803E-2,9.149446E-2,-1.1240177E-2,-2.6808238E-1,-3.6911797E-3,8.559756E-3,1.2190492E-2,-1.0530989E-2,1.2568238E-3,1.8117186E-3,-2.0673494E-3,-2.913659E-3,-1.35537E-2,3.4046194E-1,5.7156485E-1,-1.7561594E-2,-8.670689E-3,-2.8586531E-2,-1.3454394E-2,-1.5883882E-2,-8.000104E-3,6.2799673E-3,-1.1510374E-6,2.0641368E-3,5.8707697E-3,7.5796368E-3,-5.2727587E-3,8.692511E-3,1.3919663E-2,1.2261971E-2,4.467113E-3,-5.3287987E-3,3.1615566E-3,6.265921E-4,8.923013E-3,-1.4147436E-2,-6.7963875E-3,-5.1409528E-3,2.3398283E-3,6.366647E-4,1.7056411E-2,1.5334917E-2,2.7908405E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,-1,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,-1,75,77,79,-1,81,83,85,87,89,-1,-1,91,93,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,109,-1,-1,-1,-1,-1,111,113,-1,115,-1,117,119,-1,-1,-1,121,123,125,127,129,131,133,135,137,139,141,-1,-1,-1,143,145,-1,147,-1,149,151,153,-1,-1,-1,155,157,159,-1,-1,-1,-1,-1,-1,161,163,165,167,169,-1,-1,171,-1,173,-1,175,-1,-1,177,-1,179,181,183,185,-1,-1,187,189,191,193,-1,-1,195,197,-1,199,-1,-1,-1,-1,-1,-1,201,-1,-1,-1,-1,-1,-1,203,-1,205,-1,-1,-1,-1,207,-1,-1,-1,-1,-1,209,-1,211,213,-1,215,-1,217,219,-1,221,-1,-1,223,-1,-1,-1,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.214206E1,3.757717E1,1.4322632E1,6.128292E0,9.273622E0,1.3433027E-1,9.156639E0,1.7170925E0,4.2075424E0,3.09055E0,3.5430548E0,0E0,0E0,1.0506058E-1,8.457668E0,0E0,0E0,7.771225E-1,3.715458E0,1.4102278E0,9.978342E-1,1.7548567E0,4.6635056E-1,0E0,0E0,4.008028E0,1.1146317E0,7.3983574E-1,2.192421E-1,1.8057613E0,1.2490338E0,2.3775864E-1,8.5447526E-1,6.037253E-1,4.6023214E-1,1.016917E0,7.470035E-1,2.5410682E-1,0E0,1.3174495E0,2.1204605E0,4.3020344E-1,1.0519333E0,1.3487625E-1,0E0,0E0,9.171867E-2,1.7237167E0,1.0360098E0,0E0,2.2770771E-1,4.2442918E-2,1.1205006E-1,5.0542593E-1,2.3149088E-1,0E0,0E0,1.6754472E-1,1.4678335E-1,6.3336235E-1,1.4728086E0,4.148702E-1,4.2456967E-1,0E0,0E0,3.440467E-1,1.9403213E-1,2.0351944E0,0E0,0E0,0E0,9.919739E-2,0E0,0E0,0E0,0E0,0E0,1.4082909E-1,1.5461471E0,0E0,4.5406246E-1,0E0,9.9581204E-2,1.9532084E-2,0E0,0E0,0E0,4.9028122E-1,3.7873387E-1,1.9725138E-1,9.799653E-2,1.7228341E-1,1.1623266E-1,9.385514E-2,1.4155445E-1,4.9393535E-1,9.728532E-1,6.4865494E-1,0E0,0E0,0E0,5.403665E-1,3.7351626E-1,0E0,8.817053E-2,0E0,1.884945E-1,1.1556816E0,7.0693016E-2,0E0,0E0,0E0,1.5640259E-2,3.3268875E-1,2.551918E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.489652E-2,8.87841E-2,2.8782153E-1,1.3962722E-1,1.2845194E-1,0E0,0E0,1.8242985E-2,0E0,5.9779093E-2,0E0,2.2424966E-2,0E0,0E0,3.9001126E-2,0E0,2.4577151E-1,3.7174964E-1,3.1400722E-1,6.546844E-1,0E0,0E0,4.1293845E-1,8.3642006E-2,2.2298366E-1,1.8909898E-1,0E0,0E0,1.9179584E-2,1.0549575E-1,0E0,9.765167E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.8665218E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.702423E-2,0E0,2.7824879E-2,0E0,0E0,0E0,0E0,5.3915963E-2,0E0,0E0,0E0,0E0,0E0,2.0469725E-2,0E0,2.631826E-1,5.37529E-2,0E0,9.3395054E-2,0E0,1.0072256E-1,1.0828302E-1,0E0,3.3397198E-2,0E0,0E0,2.1930218E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5483685E-1,1.11252785E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,71,71,77,77,78,78,80,80,82,82,83,83,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,101,101,102,102,104,104,106,106,107,107,108,108,112,112,113,113,114,114,121,121,122,122,123,123,124,124,125,125,128,128,130,130,132,132,135,135,137,137,138,138,139,139,140,140,143,143,144,144,145,145,146,146,149,149,150,150,152,152,159,159,166,166,168,168,173,173,179,179,181,181,182,182,184,184,186,186,187,187,189,189,192,192,199,199,200,200],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,-1,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,-1,76,78,80,-1,82,84,86,88,90,-1,-1,92,94,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,110,-1,-1,-1,-1,-1,112,114,-1,116,-1,118,120,-1,-1,-1,122,124,126,128,130,132,134,136,138,140,142,-1,-1,-1,144,146,-1,148,-1,150,152,154,-1,-1,-1,156,158,160,-1,-1,-1,-1,-1,-1,162,164,166,168,170,-1,-1,172,-1,174,-1,176,-1,-1,178,-1,180,182,184,186,-1,-1,188,190,192,194,-1,-1,196,198,-1,200,-1,-1,-1,-1,-1,-1,202,-1,-1,-1,-1,-1,-1,204,-1,206,-1,-1,-1,-1,208,-1,-1,-1,-1,-1,210,-1,212,214,-1,216,-1,218,220,-1,222,-1,-1,224,-1,-1,-1,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,6E0,8.1E2,5.4051904E8,2.4481E4,1.89991E5,7.7E1,1.5859042E6,5.1167645E6,-3.1639814E-2,-1.4011223E-2,1.2328386E3,2.8045622E6,3.288987E-2,-4.6090093E-3,1.4504054E8,3.513913E2,3.79E2,1.0203835E-5,2.0342858E0,3.7788504E7,3.4380168E-2,6.706123E-2,7.075161E2,7.459E3,4.56E2,1.4E1,4.9735293E0,2.2E0,3.1E1,1.559733E6,6.83E2,1.65281E8,5.372237E2,9E0,1.2E1,2.5766423E-2,1.124641E0,4.1325716E7,3.532346E-1,4.9036694E0,5.159652E-7,-9.039893E-3,-2.1030493E-2,1E0,3.95E2,3.89E2,1.7956136E-2,2.87834E5,1E0,1E1,4.9E1,3.0283334E2,-5.529362E-4,-2.2403058E-2,2.4915715E-3,2.1111E4,5.0149255E0,1.4676277E7,8.4683485E-2,1.244E3,2.8565787E-3,1.8289194E-2,2.276E4,9.47E4,4.907764E3,-7.1584913E-3,1.265036E-2,2.9511178E-2,1E0,1.7807515E-2,-3.1096052E-2,-1.731282E-2,-4.7187775E-2,-2.4150196E-2,1.5827801E4,2.3961E4,-1.7646331E-2,7.3E1,-1.3990566E-2,3.01E2,1.9E1,-7.5404365E-3,-1.1275629E-2,-2.33795E-2,1.664557E0,1E0,2.4067588E0,1.944578E6,5.08E2,1.2368386E0,3.52E2,2.6138635E2,5.201557E7,2.2842104E0,5.2409735E-7,-1.2055591E-2,1.520202E-2,-2.1922132E-3,1E0,9.932432E0,1.860571E-2,3.0714026E8,3.8614534E-3,1.3E1,1.2825651E0,2.3385885E0,2.323293E-2,4.4732958E-2,-3.0387348E-2,1.901875E2,1.610025E6,1.0885427E9,-6.0503263E-2,-2.7562568E-2,2.1564204E-3,-6.7529925E-3,-8.010528E-3,-1.6387561E-2,1.2916666E0,1.1320151E7,1.8181819E0,1.8738E4,1.802361E6,-1.247199E-2,-3.1442575E-3,3.875E0,-3.3468108E-3,6.23E2,3.0929053E-3,2.8266037E2,1.7358132E-2,5.5846944E-3,1.4E1,1.0431361E-2,4.1221875E-1,2.560356E5,9.7E2,2.5751933E-1,1.7359488E-2,8.3915464E-4,3.792E3,5.821732E2,1.0439024E1,2.402121E4,4.511797E-4,8.970255E-3,2.6845297E5,4.6847186E2,-3.7772248E-3,1.4839433E5,4.681434E-2,2.4832327E-2,-2.2883609E-2,-1.3626187E-2,1.165859E-2,-3.5924413E-3,6.76E2,-2.3053942E-2,5.409874E-3,-1.53148E-3,-1.3219251E-2,-3.960894E-3,-7.624876E-3,5E0,-1.5479107E-3,4.0663E4,-4.8279176E-3,3.266586E-3,1.8617671E-3,6.5732696E-3,5.51E2,8.751726E-3,-6.577063E-3,-2.3720202E-3,-2.4738437E-3,3.303116E-3,1.75E0,-5.593421E-3,2.992648E-2,7.6937294E-1,-3.7309334E-3,8.051603E7,-1.9731954E-2,6.6414773E-1,3E0,-1.1240177E-2,1E0,-3.6911797E-3,8.559756E-3,2.9427118E2,-1.0530989E-2,1.2568238E-3,1.8117186E-3,-2.0673494E-3,-2.913659E-3,-1.35537E-2,2.7726634E5,1.0476191E0,-1.7561594E-2,-8.670689E-3,-2.8586531E-2,-1.3454394E-2,-1.5883882E-2,-8.000104E-3,6.2799673E-3,-1.1510374E-6,2.0641368E-3,5.8707697E-3,7.5796368E-3,-5.2727587E-3,8.692511E-3,1.3919663E-2,1.2261971E-2,4.467113E-3,-5.3287987E-3,3.1615566E-3,6.265921E-4,8.923013E-3,-1.4147436E-2,-6.7963875E-3,-5.1409528E-3,2.3398283E-3,6.366647E-4,1.7056411E-2,1.5334917E-2,2.7908405E-2],"split_indices":[2,43,17,3,2,7,12,9,44,43,43,0,0,67,43,0,0,46,67,2,53,68,7,0,0,67,2,10,3,68,73,10,9,2,5,4,3,3,0,53,66,53,53,52,0,0,106,2,2,0,12,29,3,0,4,0,0,53,9,71,62,57,2,0,0,10,9,4,0,0,0,29,0,0,0,0,0,43,1,0,0,0,0,67,0,0,0,68,29,68,9,2,68,0,4,7,69,52,0,0,0,26,69,0,5,0,3,69,53,0,0,0,4,5,46,0,0,0,0,0,0,68,5,71,9,12,0,0,69,0,2,0,4,0,0,3,0,53,62,2,57,0,0,44,4,73,48,0,0,43,4,0,48,0,0,0,0,0,0,2,0,0,0,0,0,0,6,0,9,0,0,0,0,2,0,0,0,0,0,68,0,53,53,0,7,0,53,8,0,30,0,0,67,0,0,0,0,0,0,43,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,7.36E2,2.61E2,2.34E2,5.02E2,1.1E1,2.5E2,9E0,2.25E2,2.11E2,2.91E2,7E0,4E0,1.4E1,2.36E2,4E0,5E0,8.4E1,1.41E2,1.39E2,7.2E1,2.65E2,2.6E1,4E0,1E1,1.53E2,8.3E1,6.1E1,2.3E1,1.22E2,1.9E1,4.6E1,9.3E1,1E1,6.2E1,1.52E2,1.13E2,9E0,1.7E1,4.5E1,1.08E2,3.8E1,4.5E1,5.6E1,5E0,4E0,1.9E1,1.07E2,1.5E1,5E0,1.4E1,2.1E1,2.5E1,5.6E1,3.7E1,5E0,5E0,4.2E1,2E1,1.06E2,4.6E1,1.2E1,1.01E2,5E0,4E0,2.1E1,2.4E1,1.03E2,5E0,6E0,3.2E1,3.9E1,6E0,5E1,6E0,1.5E1,4E0,5.7E1,5E1,6E0,9E0,5E0,9E0,1.5E1,6E0,5E0,2E1,2.6E1,3E1,2.4E1,1.3E1,2.3E1,1.9E1,8E0,1.2E1,6.9E1,3.7E1,4.2E1,4E0,8E0,4E0,3.7E1,6.4E1,1E1,1.1E1,6E0,1.8E1,9.1E1,1.2E1,4E0,3.5E1,1.7E1,4E1,1.1E1,3.9E1,4E0,5E0,5E0,4E0,4E0,1.1E1,1E1,1.6E1,1.5E1,1.5E1,1.9E1,5E0,4E0,9E0,7E0,1.6E1,4E0,1.5E1,4E0,4E0,8E0,4E0,2.2E1,4.7E1,2.1E1,1.6E1,3.6E1,6E0,1.7E1,2E1,5.2E1,1.2E1,4E0,7E0,9E0,9E0,4E0,8.7E1,7E0,5E0,3.4E1,6E0,4E0,7E0,2.7E1,1.2E1,4E0,6E0,1.2E1,4E0,4E0,1.1E1,4E0,1.1E1,1.4E1,5E0,4E0,5E0,1.1E1,5E0,9E0,6E0,4E0,4E0,1.5E1,7E0,1.2E1,3.5E1,5E0,1.6E1,6E0,1E1,1.3E1,4E0,1.6E1,4E0,8E0,4.4E1,8E0,4E0,4E0,5E0,4E0,5E0,4.8E1,3.9E1,1.3E1,1.4E1,7E0,4E0,4E0,7E0,4E0,7E0,7E0,8E0,7E0,5E0,1.6E1,1.9E1,8E0,8E0,5E0,5E0,8E0,5E0,1.1E1,5E0,1E1,3.4E1,4E0,4.4E1,6E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[4.282539E-3,-1.5980986E-1,4.428412E-1,-4.5826387E-1,-9.64763E-3,-2.8706862E-2,4.8953718E-1,-6.6484565E-1,-3.5655844E-1,-1.7110102E-1,1.2145714E-1,2.0425431E-1,5.975026E-1,-6.287699E-1,-4.9905527E-2,-4.7406977E-1,-1.5504657E-1,-3.5407957E-1,-9.038243E-2,-1.948989E-2,1.9048887E-1,4.245525E-1,6.0930364E-2,1.1161122E0,5.404194E-1,-4.0212203E-2,-5.7448566E-1,-4.203092E-1,-8.556885E-1,5.480729E-1,-2.8238264E-1,-2.4379371E-1,-4.819723E-1,-6.863111E-2,-2.8434414E-2,1.8420845E-2,-2.9343253E-1,1.5160462E-1,4.2861778E-1,2.7033815E-1,2.7421888E-2,1.9839756E-1,-5.2444313E-2,3.0397184E-2,5.9613295E-2,4.2763776E-1,7.372566E-1,-2.7440354E-2,-1.3120598E-2,-2.3259267E-1,-4.647628E-1,-1.4245516E-2,-5.5558696E-2,1.17730005E-2,3.238991E-2,-2.2586961E-1,-2.9085081E-2,-3.053415E-1,-6.4963534E-2,-1.2324744E-2,-5.143408E-1,-1.0789787E-1,1.8881796E-2,1.4778185E-1,-3.637012E-2,-2.1648295E-2,4.8374466E-4,8.579984E-2,2.879833E-1,5.6159967E-3,4.9930394E-1,1.02200955E-1,1.8869437E-2,2.6119836E-3,2.6528913E-1,4.206135E-2,-2.078595E-1,4.638437E-1,-4.2280406E-2,7.809948E-1,9.187501E-3,-1.7436896E-2,-5.001263E-2,-5.210282E-1,-3.6299708E-1,-1.5876178E-1,-4.3716303E-1,-1.6737146E-2,-1.0280319E-2,3.6121877E-3,-9.114485E-3,-1.5027487E-2,-5.5692536E-1,-1.6829827E-1,2.5324164E-2,5.002059E-2,-1.1673954E-2,-1.5445804E-3,2.1004426E-1,-1.4001645E-1,6.1578788E-2,1.0641734E-1,-1.6641088E-2,-5.280974E-3,3.2465306E-1,2.5540506E-2,1.0539849E-2,1.3102842E-3,7.188572E-3,2.316542E-3,1.6000278E-2,9.190085E-3,-2.150581E-2,-1.4457218E-2,-1.857162E-3,1.0171475E-1,4.9673375E-1,-1.2682702E-2,9.166736E-3,8.893733E-3,8.217765E-1,-7.426931E-3,2.338216E-3,-2.4966089E-2,-1.1738144E-2,-1.8443297E-1,-4.3074203E-1,-3.051808E-1,-8.7806605E-2,-2.4272174E-2,-1.04512945E-2,-3.1522036E-2,-1.8650971E-2,-1.01339296E-1,-3.750304E-1,-1.3413919E-2,6.840111E-3,-5.4356554E-3,7.517972E-2,1.0871677E-2,4.493147E-3,1.9456498E-2,-2.0477846E-1,9.0122856E-2,-4.58138E-3,-1.2808391E-2,1.5958446E-1,3.6497775E-1,1.9443567E-3,-3.8519052E-3,2.0067447E-3,1.3277823E-2,-6.5351385E-3,2.1206862E-1,5.427489E-1,1.8452225E-2,3.918162E-2,-2.7686933E-3,-1.1893801E-2,-2.5489086E-2,-1.1202827E-2,-6.0975584E-3,-1.7337618E-2,-1.3765106E-2,-3.830511E-2,-3.1301405E-2,-1.841327E-1,-2.5823444E-1,-2.7890079E-2,6.411128E-2,-6.587412E-2,1.1746271E-1,-1.5169332E-2,-3.6964463E-3,5.314416E-3,-5.311406E-3,-1.0859397E-2,-2.8596302E-3,1.2057656E-1,4.6179496E-2,-1.349823E-1,2.1995944E-1,8.5637145E-2,2.7764356E-1,4.8408473E-1,3.9302735E-3,1.2337889E-2,5.0258005E-1,3.4941804E-2,-1.0424447E-4,-7.744306E-3,3.7727735E-3,-2.8495346E-3,-2.9315157E-3,-1.20281475E-2,-5.2722883E-3,-1.5432335E-2,-1.3650125E-3,5.629392E-3,-8.359306E-4,-7.7323965E-3,6.391512E-3,9.6006435E-5,-3.9546434E-3,3.7544388E-3,-2.256193E-4,6.871089E-3,-9.2264666E-4,3.528156E-3,-1.1525731E-2,-1.5668486E-3,1.1844793E-2,7.766568E-4,5.55371E-3,-2.998259E-3,8.021801E-3,1.7764298E-2,1.1768102E-2,2.491563E-2,2.4774328E-2,1.3376156E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,-1,73,75,-1,-1,77,79,-1,-1,81,83,-1,-1,-1,-1,85,-1,87,89,-1,91,93,95,97,99,-1,-1,101,103,-1,105,107,-1,-1,109,111,113,115,117,119,-1,-1,121,123,125,127,129,-1,-1,-1,-1,-1,131,133,135,137,-1,-1,139,141,143,145,-1,-1,147,-1,-1,-1,-1,-1,-1,-1,149,-1,-1,151,153,-1,-1,-1,155,-1,-1,-1,-1,157,159,161,163,-1,-1,-1,-1,165,167,169,-1,-1,171,-1,-1,173,175,177,-1,179,181,183,-1,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,189,191,193,195,-1,197,199,201,203,-1,-1,-1,-1,-1,205,207,209,211,213,215,217,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.268259E1,3.2935722E1,1.39701805E1,4.931526E0,1.0392763E1,0E0,8.009335E0,5.196037E-1,3.8810444E0,3.2173467E0,2.640493E0,2.3106086E0,5.051178E0,4.9687958E-1,0E0,1.7945328E0,5.795312E0,8.275175E-1,1.7166666E0,9.400883E-1,1.6202416E0,5.953741E-1,7.2732866E-1,5.8794785E-1,3.547451E0,0E0,1.9900513E-1,6.7629814E-1,1.9076719E0,2.7006102E-1,9.172845E-1,3.984151E-1,7.1755886E-2,5.1405317E-1,0E0,5.7240003E-1,5.9043854E-1,1.3932416E0,4.819212E-1,3.5299194E-1,0E0,1.8427515E-1,3.906209E-1,0E0,0E0,1.9421749E0,1.3046684E0,0E0,0E0,5.1001084E-1,2.4031067E-1,0E0,0E0,0E0,0E0,6.112082E-1,0E0,5.1428556E-2,2.237905E-1,0E0,2.0124912E-2,8.330685E-1,4.1008863E-1,2.7781165E-1,5.872886E-1,0E0,0E0,1.0314445E0,7.8946114E-1,0E0,1.6419601E-1,3.1073406E-2,0E0,0E0,2.267437E-1,1.7379001E-1,1.6358727E-1,1.2000694E0,5.5757487E-1,1.209774E0,0E0,0E0,1.19574584E-1,1.0060692E-1,2.834618E-1,3.6799222E-1,8.584547E-2,0E0,0E0,0E0,0E0,0E0,8.955145E-2,9.448173E-1,1.5888289E-1,1.902465E-1,0E0,0E0,2.8432906E-2,2.931623E-1,1.4658654E-1,6.612884E-1,0E0,0E0,5.201764E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.58561E-2,0E0,0E0,4.9219903E-1,1.1423969E0,0E0,0E0,0E0,2.5062943E-1,0E0,0E0,0E0,0E0,7.547602E-2,3.4448504E-1,9.6161604E-2,2.694549E-1,0E0,0E0,0E0,0E0,3.1316352E-1,2.9812407E-1,1.09964944E-1,0E0,0E0,1.4695047E-1,0E0,0E0,9.476501E-2,2.7362287E-2,1.2783307E-1,0E0,2.4405715E-1,3.026122E-1,2.9208374E-1,0E0,0E0,0E0,0E0,0E0,8.203459E-2,3.2339096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0684687E-1,1.1141836E-1,2.1343923E-1,1.0389882E-1,0E0,6.308546E-2,7.4564174E-2,6.2366217E-2,9.525315E-2,0E0,0E0,0E0,0E0,0E0,7.842755E-2,4.7456782E-2,1.1883013E-1,2.842903E-1,1.847666E-1,2.2224092E-1,6.528759E-2,0E0,0E0,3.612442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,41,41,42,42,45,45,46,46,49,49,50,50,55,55,57,57,58,58,60,60,61,61,62,62,63,63,64,64,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,82,82,83,83,84,84,85,85,86,86,92,92,93,93,94,94,95,95,98,98,99,99,100,100,101,101,104,104,112,112,115,115,116,116,120,120,125,125,126,126,127,127,128,128,133,133,134,134,135,135,138,138,141,141,142,142,143,143,145,145,146,146,147,147,153,153,154,154,164,164,165,165,166,166,167,167,169,169,170,170,171,171,172,172,178,178,179,179,180,180,181,181,182,182,183,183,184,184,187,187],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,-1,74,76,-1,-1,78,80,-1,-1,82,84,-1,-1,-1,-1,86,-1,88,90,-1,92,94,96,98,100,-1,-1,102,104,-1,106,108,-1,-1,110,112,114,116,118,120,-1,-1,122,124,126,128,130,-1,-1,-1,-1,-1,132,134,136,138,-1,-1,140,142,144,146,-1,-1,148,-1,-1,-1,-1,-1,-1,-1,150,-1,-1,152,154,-1,-1,-1,156,-1,-1,-1,-1,158,160,162,164,-1,-1,-1,-1,166,168,170,-1,-1,172,-1,-1,174,176,178,-1,180,182,184,-1,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,190,192,194,196,-1,198,200,202,204,-1,-1,-1,-1,-1,206,208,210,212,214,216,218,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,2.16577E5,1E0,7.7E1,8.1E2,-2.8706862E-2,7.445455E2,1.05039425E1,6.76E2,4.58E2,7.502167E5,9E0,1.059448E7,8.8324326E-1,-4.9905527E-2,8.719101E0,2.74614E5,3.875E0,4.6E1,2.3E1,5.405423E6,5.766206E6,7.44488E5,1.767635E3,2.9251662E6,-4.0212203E-2,6.1454544E0,2E0,3.89E2,1.9E1,2.5E1,1.000501E6,3.808001E5,2.331083E6,-2.8434414E-2,1.88E2,2.22E3,5.163891E2,3.3852024E7,5.404E2,2.7421888E-2,2.3625128E6,1.6E1,3.0397184E-2,5.9613295E-2,3.0070068E7,2E0,-2.7440354E-2,-1.3120598E-2,3.6E2,4.60349E5,-1.4245516E-2,-5.5558696E-2,1.17730005E-2,3.238991E-2,1.7479E4,-2.9085081E-2,4.3827028E5,6.642857E0,-1.2324744E-2,1.9913E4,7.06E2,7.89E2,1.4473684E-1,1.445E3,-2.1648295E-2,4.8374466E-4,2.41E3,1.071E3,5.6159967E-3,1.639E3,6.185E3,1.8869437E-2,2.6119836E-3,1E0,1.124641E0,1.453785E6,1.0476191E0,9.4578236E-1,1E0,9.187501E-3,-1.7436896E-2,1.2E1,3.0588236E0,1.4338E4,1.6580646E2,5.25E0,-1.6737146E-2,-1.0280319E-2,3.6121877E-3,-9.114485E-3,-1.5027487E-2,4.795239E7,2.5493E4,1.6802971E6,5.02E2,-1.1673954E-2,-1.5445804E-3,4.957196E2,3.6086957E0,1.6533886E8,5.4007E7,-1.6641088E-2,-5.280974E-3,5.929839E8,2.5540506E-2,1.0539849E-2,1.3102842E-3,7.188572E-3,2.316542E-3,1.6000278E-2,9.190085E-3,2.8003533E0,-1.4457218E-2,-1.857162E-3,2.6402525E5,9.143923E4,-1.2682702E-2,9.166736E-3,8.893733E-3,1.06314E5,-7.426931E-3,2.338216E-3,-2.4966089E-2,-1.1738144E-2,9.9E1,1.8224286E2,2.9157894E0,3.741267E7,-2.4272174E-2,-1.04512945E-2,-3.1522036E-2,-1.8650971E-2,1E0,4.1361522E6,5.539245E6,6.840111E-3,-5.4356554E-3,1.6E1,1.0871677E-2,4.493147E-3,1.3E1,9.31E2,3.9960475E0,-4.58138E-3,3.774648E0,3.0416667E0,1.8062708E6,1.9443567E-3,-3.8519052E-3,2.0067447E-3,1.3277823E-2,-6.5351385E-3,3.1E2,4.907764E3,1.8452225E-2,3.918162E-2,-2.7686933E-3,-1.1893801E-2,-2.5489086E-2,-1.1202827E-2,-6.0975584E-3,-1.7337618E-2,-1.3765106E-2,4.7220547E1,7.784E3,1.6808511E0,2.28E0,-2.7890079E-2,2.0671213E2,1.3E1,1E0,3.5510652E6,-3.6964463E-3,5.314416E-3,-5.311406E-3,-1.0859397E-2,-2.8596302E-3,3.2892792E2,2.0467092E6,5.092769E7,9.51E3,1.7500242E7,1.724907E8,2.02E2,3.9302735E-3,1.2337889E-2,2.2040408E7,3.4941804E-2,-1.0424447E-4,-7.744306E-3,3.7727735E-3,-2.8495346E-3,-2.9315157E-3,-1.20281475E-2,-5.2722883E-3,-1.5432335E-2,-1.3650125E-3,5.629392E-3,-8.359306E-4,-7.7323965E-3,6.391512E-3,9.6006435E-5,-3.9546434E-3,3.7544388E-3,-2.256193E-4,6.871089E-3,-9.2264666E-4,3.528156E-3,-1.1525731E-2,-1.5668486E-3,1.1844793E-2,7.766568E-4,5.55371E-3,-2.998259E-3,8.021801E-3,1.7764298E-2,1.1768102E-2,2.491563E-2,2.4774328E-2,1.3376156E-2],"split_indices":[2,43,17,44,2,0,67,69,2,2,43,3,12,68,0,69,5,69,3,3,43,60,9,67,43,0,69,32,2,0,3,43,43,43,0,10,10,70,7,4,0,60,3,0,0,58,6,0,0,2,9,0,0,0,0,44,0,43,73,0,9,2,2,71,2,0,0,2,2,0,2,44,0,0,68,53,9,68,49,27,0,0,3,68,12,67,69,0,0,0,0,0,7,1,43,2,0,0,4,69,5,7,0,0,5,0,0,0,0,0,0,0,68,0,0,48,48,0,0,0,1,0,0,0,0,0,4,69,7,0,0,0,0,26,60,5,0,0,3,0,0,3,2,69,0,69,69,43,0,0,0,0,0,8,4,0,0,0,0,0,0,0,0,0,73,1,68,71,0,4,3,30,43,0,0,0,0,0,4,43,5,44,12,7,10,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,7.34E2,2.74E2,2.45E2,4.89E2,1.1E1,2.63E2,7.9E1,1.66E2,2.19E2,2.7E2,7.3E1,1.9E2,7.5E1,4E0,1.04E2,6.2E1,6.6E1,1.53E2,8.9E1,1.81E2,2.8E1,4.5E1,1.7E1,1.73E2,1.1E1,6.4E1,9.3E1,1.1E1,9E0,5.3E1,3.7E1,2.9E1,1.48E2,5E0,7.9E1,1E1,1.57E2,2.4E1,1.6E1,1.2E1,2E1,2.5E1,6E0,1.1E1,1.12E2,6.1E1,5.9E1,5E0,1.9E1,7.4E1,5E0,6E0,4E0,5E0,4.7E1,6E0,2.7E1,1E1,5E0,2.4E1,1.02E2,4.6E1,2.3E1,5.6E1,6E0,4E0,1.07E2,5E1,5E0,1.9E1,8E0,8E0,7E0,1.3E1,1.6E1,9E0,1.04E2,8E0,5.6E1,5E0,1E1,9E0,4.5E1,2.9E1,3.7E1,1E1,1.4E1,1.3E1,5E0,5E0,6E0,1.8E1,7E1,3.2E1,4.2E1,4E0,6E0,1.7E1,2.7E1,2.9E1,1.03E2,4E0,4E0,4.6E1,1.5E1,4E0,4E0,4E0,4E0,9E0,4E0,1.2E1,5E0,4E0,9E0,9.5E1,4E0,4E0,4E0,5.2E1,4E0,5E0,4.1E1,4E0,9E0,2E1,1.1E1,2.6E1,6E0,4E0,8E0,1E1,5.4E1,1.6E1,2.5E1,7E0,5E0,3.7E1,1.3E1,4E0,8E0,1.9E1,2.5E1,4E0,3.2E1,7.1E1,4E1,6E0,6E0,6E0,5E0,4E0,1.4E1,8.1E1,4E0,4.8E1,4E0,5E0,1.1E1,9E0,4E0,7E0,4E0,2.2E1,3E1,2.4E1,1.2E1,4E0,1E1,1.5E1,2.5E1,1.2E1,4E0,4E0,6E0,1.3E1,4E0,2.1E1,2.2E1,1E1,3.8E1,3.3E1,2.5E1,1.5E1,5E0,9E0,7.1E1,1E1,1.8E1,4E0,6E0,2.4E1,1E1,1.4E1,5E0,7E0,4E0,6E0,1.1E1,4E0,2.1E1,4E0,7E0,5E0,4E0,1.7E1,7E0,1.5E1,4E0,6E0,3.2E1,6E0,2.7E1,6E0,1.4E1,1.1E1,4E0,1.1E1,6E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[1.0388326E-2,-1.4030552E-1,4.7239754E-1,-4.348304E-1,-1.2833244E-2,-5.790869E-1,5.2626246E-1,-5.837202E-1,-3.2643813E-1,-1.360659E-1,1.2067847E-1,-3.274073E-2,-1.4311097E-2,3.134193E-1,6.3941395E-1,-5.323711E-1,-3.7736334E-2,-7.323976E-2,-4.166818E-1,-1.8608682E-1,4.474006E-2,2.010919E-1,1.8798243E-2,-1.12498924E-1,3.7654564E-1,6.8760234E-1,-7.1448244E-2,-3.0266145E-2,-3.587346E-1,-2.4892873E-1,2.1274523E-1,-7.369641E-1,-3.4930846E-1,-3.253191E-1,-1.24986306E-1,-9.495562E-3,1.6182142E-1,1.0867217E-1,3.2406023E-1,3.726113E-2,-1.8752895E-2,-1.0087916E-3,-8.045053E-3,4.3161398E-1,1.1904129E-2,1.1011767E0,6.295695E-1,-1.0426399E-2,7.3351082E-3,-2.4215284E-1,-2.5655964E-2,-3.673265E-1,-7.561075E-2,2.6332855E-2,-2.287433E-2,-1.8815547E-2,-8.797719E-1,-4.0034622E-1,-9.780679E-2,-3.643531E-1,-1.8678468E-1,-4.7936368E-1,-8.720403E-2,8.236923E-2,-7.038707E-2,2.3411028E-1,2.1924276E-3,7.45788E-2,2.2304257E-2,3.87757E-1,2.1035975E-2,6.268567E-2,-1.4882028E-1,5.064681E-1,2.4709468E-1,-6.804852E-3,7.813111E-3,3.4257486E-2,5.9078656E-2,4.0932286E-1,7.4131685E-1,-3.1504092E-1,1.901843E-3,-8.280606E-3,-2.0877583E-2,3.6218555E-3,-8.076267E-3,-3.437439E-3,2.0113823E-3,-5.5872116E-2,-2.396147E-2,-4.272993E-1,-2.299726E-1,-1.4218594E-2,-1.2882934E-2,-5.5001173E-3,-3.8231453E-1,-4.089209E-3,-9.711663E-3,-2.734341E-1,-3.8647167E-2,-4.0184204E-2,-2.0043829E-1,1.1519904E-2,8.872906E-3,-1.0236356E-2,4.9308194E-3,4.8747207E-3,1.436282E-2,1.385777E-1,-3.1050999E-2,3.0358925E-1,2.7466597E-2,-5.1160636E-3,7.912424E-3,2.2566853E-1,3.604424E-2,-1.4750997E-2,8.752503E-4,5.362695E-1,7.1535977E-3,2.5675008E-3,2.8888908E-1,9.555297E-3,4.518391E-1,8.5291106E-1,5.8210063E-1,-1.6858865E-2,-6.3681426E-3,-1.17612695E-2,-4.522976E-1,-5.4745353E-3,-1.2515189E-2,-5.9187347E-3,2.7138807E-3,-2.2638229E-1,-4.078414E-1,-5.9702927E-3,-1.7189099E-2,8.165699E-2,-8.0282755E-2,-2.1667428E-2,-1.5982631E-1,-1.5348726E-3,2.8551018E-3,7.133243E-2,-9.020835E-3,-3.4746742E-3,1.6944005E-1,-2.7185538E-1,1.14984185E-1,5.0500883E-3,3.2894447E-1,1.8673599E-3,1.366226E-2,-7.78244E-2,6.852726E-2,5.714867E-1,1.5875896E-2,5.0973957E-3,3.353413E-1,4.889097E-1,7.976E-3,1.4878836E-2,9.059156E-1,9.089502E-3,6.2051976E-1,-4.75416E-1,-1.33524025E-2,-1.3847336E-2,-4.9782274E-3,-1.9682078E-2,-1.0800449E-2,1.2242195E-1,-2.147313E-3,6.240829E-3,-9.374977E-2,-2.0394827E-1,5.0579747E-3,4.847221E-3,1.6002495E-4,2.4208382E-1,1.147353E-1,6.509094E-4,-2.2088427E-2,-5.4375856E-4,1.7014807E-1,2.3618416E-1,1.8400008E-2,1.3297886E-3,-1.37443E-1,-5.8076844E-2,1.0477056E-1,3.8360858E-1,2.9782811E-2,9.697665E-3,1.9074026E-2,5.2090096E-1,1.1925462E-2,1.968422E-2,4.3577157E-2,1.5370795E-2,2.9887285E-2,-1.8692313E-2,-2.606006E-2,1.0476403E-3,7.5558624E-3,-6.3167918E-3,-1.3731056E-3,-3.2650512E-3,-1.159511E-2,3.3349416E-3,1.3188432E-2,3.732447E-3,1.016734E-2,9.53655E-3,4.0367595E-3,1.3284303E-2,5.6239846E-3,-1.1354617E-2,-1.802359E-3,-5.509594E-3,2.946002E-3,1.3176004E-3,8.583301E-3,8.835795E-3,2.2807637E-2,2.6123086E-2,1.4159386E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,-1,73,75,77,79,-1,-1,81,-1,83,85,-1,87,-1,89,91,93,95,97,99,101,103,105,107,-1,109,-1,111,113,115,117,119,121,-1,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,129,131,133,-1,-1,135,-1,-1,137,-1,139,141,143,-1,-1,145,-1,-1,147,149,151,-1,-1,-1,153,155,-1,-1,157,-1,-1,159,-1,161,163,165,-1,-1,-1,167,-1,-1,-1,-1,169,171,-1,-1,173,175,-1,177,-1,-1,179,-1,-1,181,183,185,-1,187,-1,-1,189,191,193,-1,-1,195,197,-1,-1,199,-1,201,203,-1,-1,-1,-1,-1,205,-1,-1,207,209,-1,-1,-1,211,213,-1,-1,-1,215,217,-1,-1,219,221,223,225,-1,-1,-1,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.2751595E1,2.9578901E1,1.4860748E1,3.6504135E0,9.082026E0,2.3421955E-1,5.700058E0,7.501259E-1,3.1684208E0,2.6039767E0,2.1672423E0,0E0,0E0,2.3814993E0,5.49572E0,1.5554047E0,0E0,1.9622456E0,1.9651566E0,1.8734517E0,4.0158534E-1,1.6467595E0,9.404141E-1,6.3259274E-2,1.520958E0,2.882866E0,4.264229E-1,0E0,7.135663E-1,4.647174E-1,1.2756351E0,4.4209957E-1,1.0775833E0,2.9965782E-1,2.0736883E0,2.519243E-1,1.5423673E-1,1.0609074E0,1.2078476E0,5.4866683E-1,0E0,0E0,0E0,7.864485E-1,3.0022097E-1,1.3768768E-1,2.969963E0,0E0,0E0,5.292337E-1,0E0,1.4171755E-1,1.8176433E-1,0E0,3.7461735E-2,0E0,6.820288E-1,2.3236275E-1,2.3817374E-1,1.8359327E-1,2.0534039E-2,8.745477E-1,7.5863826E-1,1.376725E-1,3.08537E-1,8.56778E-2,0E0,5.451183E-1,0E0,7.2448397E-1,2.5652924E-1,4.304177E-1,4.0643933E-1,3.9899254E-1,1.520387E-1,0E0,0E0,0E0,0E0,3.1118298E-1,1.0493011E0,1.2646317E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4823723E-1,2.2643149E-2,1.0724579E-1,0E0,0E0,9.87339E-2,0E0,0E0,1.0837275E-1,0E0,5.0977814E-1,3.8313055E-1,2.9049342E-2,0E0,0E0,2.6649553E-1,0E0,0E0,3.4264457E-1,1.1214024E0,1.5615511E-1,0E0,0E0,0E0,1.6973954E-1,3.3082727E-1,0E0,0E0,8.8534355E-2,0E0,0E0,1.0400152E-1,0E0,3.088894E-1,1.0869789E0,4.553089E-1,0E0,0E0,0E0,5.8882713E-2,0E0,0E0,0E0,0E0,4.601291E-2,2.702427E-2,0E0,0E0,1.4150064E-1,2.3889303E-1,0E0,4.7386074E-1,0E0,0E0,3.3830076E-2,0E0,0E0,1.4707863E-1,7.1447974E-1,1.4184761E-1,0E0,1.2459707E-1,0E0,0E0,1.3107294E-1,3.2269472E-1,2.201395E-1,0E0,0E0,3.974712E-2,1.0917282E-1,0E0,0E0,3.8630295E-1,0E0,5.074787E-2,4.80299E-2,0E0,0E0,0E0,0E0,0E0,7.8338176E-2,0E0,0E0,1.9958001E-1,1.8911779E-1,0E0,0E0,0E0,1.120739E-1,7.876623E-2,0E0,0E0,0E0,2.399829E-2,6.4926445E-2,0E0,0E0,1.2903346E-1,1.2687759E-1,3.2829916E-1,1.5965712E-1,0E0,0E0,0E0,9.413195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,43,43,44,44,45,45,46,46,49,49,51,51,52,52,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,69,69,70,70,71,71,72,72,73,73,74,74,79,79,80,80,81,81,91,91,92,92,93,93,96,96,99,99,101,101,102,102,103,103,106,106,109,109,110,110,111,111,115,115,116,116,119,119,122,122,124,124,125,125,126,126,130,130,135,135,136,136,139,139,140,140,142,142,145,145,148,148,149,149,150,150,152,152,155,155,156,156,157,157,160,160,161,161,164,164,166,166,167,167,173,173,176,176,177,177,181,181,182,182,186,186,187,187,190,190,191,191,192,192,193,193,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,-1,74,76,78,80,-1,-1,82,-1,84,86,-1,88,-1,90,92,94,96,98,100,102,104,106,108,-1,110,-1,112,114,116,118,120,122,-1,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,130,132,134,-1,-1,136,-1,-1,138,-1,140,142,144,-1,-1,146,-1,-1,148,150,152,-1,-1,-1,154,156,-1,-1,158,-1,-1,160,-1,162,164,166,-1,-1,-1,168,-1,-1,-1,-1,170,172,-1,-1,174,176,-1,178,-1,-1,180,-1,-1,182,184,186,-1,188,-1,-1,190,192,194,-1,-1,196,198,-1,-1,200,-1,202,204,-1,-1,-1,-1,-1,206,-1,-1,208,210,-1,-1,-1,212,214,-1,-1,-1,216,218,-1,-1,220,222,224,226,-1,-1,-1,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,1.2E2,9.48E2,9.3326636E-2,9.9347424E2,5.17259E8,1.5522388E0,2.331083E6,3.887931E0,-3.274073E-2,-1.4311097E-2,2.8204132E7,3.206931E2,3E1,-3.7736334E-2,1.5292561E2,1E0,4.67E2,2.9254214E2,4.371613E2,2.5401869E1,1.1E1,2.2E1,1.059448E7,4.7263342E2,-3.0266145E-2,1.388E1,1.2E1,2.3E1,1.654E3,3.2874417E2,1.8154167E2,2.86152E5,1.4285715E0,9.244374E6,2.1578948E0,1E0,1E0,-1.8752895E-2,-1.0087916E-3,-8.045053E-3,1.835821E1,1.9719212E1,4.626E3,1.3711089E6,-1.0426399E-2,7.3351082E-3,4.7837E4,-2.5655964E-2,1.16E0,1.5077367E3,2.6332855E-2,9E0,-1.8815547E-2,6.27907E0,7.997723E6,3.478E4,1.9E1,1.2707424E0,1.2916666E0,2.712E3,5.405423E6,8.611817E6,4.6E1,2.1924276E-3,2.5E0,2.2304257E-2,1.28738E5,2.9149944E5,9E0,4.265829E2,2E0,3.202146E5,-6.804852E-3,7.813111E-3,3.4257486E-2,5.9078656E-2,9.49372E4,1.7103828E7,3.015873E0,1.901843E-3,-8.280606E-3,-2.0877583E-2,3.6218555E-3,-8.076267E-3,-3.437439E-3,2.0113823E-3,-5.5872116E-2,-2.396147E-2,7.090909E0,2.29E2,4.7280664E3,-1.2882934E-2,-5.5001173E-3,2.6003375E5,-4.089209E-3,-9.711663E-3,1E0,-3.8647167E-2,1.3364486E0,1.19111115E2,1.4E1,8.872906E-3,-1.0236356E-2,1.7544615E0,4.8747207E-3,1.436282E-2,1.007E3,2.026453E6,3.655004E7,2.7466597E-2,-5.1160636E-3,7.912424E-3,1.0542694E1,3.82E2,-1.4750997E-2,8.752503E-4,2.2539758E5,7.1535977E-3,2.5675008E-3,1E0,9.555297E-3,4.4895835E0,1E0,1E0,-1.6858865E-2,-6.3681426E-3,-1.17612695E-2,6.926E3,-5.4745353E-3,-1.2515189E-2,-5.9187347E-3,2.7138807E-3,3.8E1,9.860918E5,-5.9702927E-3,-1.7189099E-2,1E0,1.4E1,-2.1667428E-2,2.0200005E8,-1.5348726E-3,2.8551018E-3,1.6E1,-9.020835E-3,-3.4746742E-3,5.92E2,3.1888113E0,6.642857E0,5.0500883E-3,5.5885565E2,1.8673599E-3,1.366226E-2,1.7161617E2,1E0,4.6847186E2,1.5875896E-2,5.0973957E-3,5.489183E1,1E0,7.976E-3,1.4878836E-2,1.32476E5,9.089502E-3,1.5822886E0,1.5E1,-1.33524025E-2,-1.3847336E-2,-4.9782274E-3,-1.9682078E-2,-1.0800449E-2,8.754E3,-2.147313E-3,6.240829E-3,7.09E2,1.08E2,5.0579747E-3,4.847221E-3,1.6002495E-4,1.124E3,1.25789E5,6.509094E-4,-2.2088427E-2,-5.4375856E-4,2.23099E7,1.3881669E6,1.8400008E-2,1.3297886E-3,1.6869704E6,4.888889E0,4.162742E5,5.941442E6,2.9782811E-2,9.697665E-3,1.9074026E-2,2.9E1,1.1925462E-2,1.968422E-2,4.3577157E-2,1.5370795E-2,2.9887285E-2,-1.8692313E-2,-2.606006E-2,1.0476403E-3,7.5558624E-3,-6.3167918E-3,-1.3731056E-3,-3.2650512E-3,-1.159511E-2,3.3349416E-3,1.3188432E-2,3.732447E-3,1.016734E-2,9.53655E-3,4.0367595E-3,1.3284303E-2,5.6239846E-3,-1.1354617E-2,-1.802359E-3,-5.509594E-3,2.946002E-3,1.3176004E-3,8.583301E-3,8.835795E-3,2.2807637E-2,2.6123086E-2,1.4159386E-2],"split_indices":[2,43,17,44,2,53,67,46,68,43,69,0,0,5,73,70,0,67,29,2,70,67,69,3,3,12,71,0,71,3,0,1,67,67,7,71,60,68,6,19,0,0,0,71,71,2,43,0,0,9,0,69,48,0,8,0,71,9,44,67,68,68,44,43,60,0,0,69,0,1,48,3,67,32,43,0,0,0,0,48,62,68,0,0,0,0,0,0,0,0,0,73,10,48,0,0,43,0,0,8,0,68,67,3,0,0,68,0,0,2,9,7,0,0,0,71,4,0,0,48,0,0,29,0,68,15,26,0,0,0,10,0,0,0,0,0,43,0,0,26,10,0,5,0,0,3,0,0,10,71,73,0,4,0,0,67,29,4,0,0,71,23,0,0,1,0,53,8,0,0,0,0,0,1,0,0,2,10,0,0,0,2,1,0,0,0,9,43,0,0,60,69,62,60,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.043E3,7.87E2,2.56E2,2.37E2,5.5E2,1.2E1,2.44E2,9.8E1,1.39E2,2.86E2,2.64E2,7E0,5E0,8.6E1,1.58E2,8.3E1,1.5E1,3.7E1,1.02E2,2.24E2,6.2E1,1.47E2,1.17E2,1.1E1,7.5E1,1.48E2,1E1,4.7E1,3.6E1,2.3E1,1.4E1,1.6E1,8.6E1,6.7E1,1.57E2,4.3E1,1.9E1,8.5E1,6.2E1,1.13E2,4E0,5E0,6E0,6.5E1,1E1,1.6E1,1.32E2,6E0,4E0,2.4E1,1.2E1,1.3E1,1E1,5E0,9E0,6E0,1E1,7.1E1,1.5E1,5.1E1,1.6E1,1.4E1,1.43E2,1.7E1,2.6E1,1.1E1,8E0,7.9E1,6E0,5.1E1,1.1E1,1E2,1.3E1,4.5E1,2E1,5E0,5E0,7E0,9E0,4.6E1,8.6E1,1.9E1,5E0,5E0,8E0,4E0,6E0,5E0,4E0,4E0,6E0,6E1,1.1E1,1.1E1,4E0,4E0,4.7E1,4E0,1.2E1,1E1,4E0,1.02E2,4.1E1,1.1E1,6E0,8E0,1.8E1,5E0,6E0,4.9E1,3E1,3.8E1,1.3E1,6E0,5E0,1.3E1,8.7E1,6E0,7E0,4.1E1,4E0,4E0,1.6E1,9E0,3.7E1,4.8E1,3.8E1,1.4E1,5E0,9E0,5.1E1,4E0,7E0,4E0,7E0,8E0,3.9E1,5E0,5E0,2.5E1,7.7E1,4E0,3.7E1,6E0,5E0,1.4E1,4E0,6E0,4.3E1,1.1E1,1.9E1,5E0,3.3E1,4E0,9E0,1.9E1,6.8E1,3.3E1,8E0,4E0,1.2E1,3.2E1,5E0,5E0,4.3E1,4E0,3.4E1,4.3E1,8E0,4E0,4E0,3.4E1,5E0,1.9E1,6E0,4E0,7.3E1,3.2E1,5E0,9E0,5E0,1.7E1,2.6E1,5E0,6E0,6E0,1.3E1,1.6E1,1.7E1,7E0,1.2E1,1.5E1,5.3E1,1.1E1,2.2E1,6E0,6E0,2.7E1,5E0,4E0,3.9E1,4E0,3E1,2.7E1,1.6E1,6E0,1.3E1,4.3E1,3E1,9E0,2.3E1,4E0,1.3E1,2.1E1,5E0,8E0,5E0,1E1,6E0,5E0,7E0,1E1,5E0,2.8E1,2.5E1,5E0,6E0,2.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[-6.069969E-3,-1.39584E-1,4.168373E-1,-4.165128E-1,-1.692058E-2,4.6412694E-1,-5.636922E-1,-3.6771145E-1,-6.8826175E-1,-1.2340638E-1,9.475499E-2,1.1592902E0,4.1920882E-1,-3.1316992E-2,-1.239372E-2,-2.764794E-2,-2.95202E-1,-5.5115956E-1,-5.0783135E-2,-1.905333E-1,1.5783371E-2,1.8670271E-1,9.308463E-3,3.617117E-2,6.5777734E-2,2.5238645E-1,5.974901E-1,9.19941E-2,-3.5077587E-1,-9.668635E-3,-6.3124573E-1,-2.6704106E-1,-6.037407E-2,-1.902264E-2,1.8447621E-1,1.4883727E-1,4.2964143E-1,2.7907675E-2,-1.4016416E-2,2.1528159E-1,3.7666537E-2,4.4670904E-1,7.004935E-1,1.843852E-2,-9.007192E-2,-3.835936E-1,-1.4497967E-1,-3.1929143E-2,-1.8278925E-2,-1.8729578E-1,-4.479107E-1,8.411741E-3,-1.7532864E-1,-6.9576964E-2,5.9868664E-2,1.3444636E-2,9.1451146E-2,1.2956489E-2,2.4056609E-1,9.602993E-3,2.3317957E-2,-1.4078775E-3,2.4419786E-1,5.267276E-2,2.875718E-1,4.47477E-3,4.7700074E-1,7.6333183E-1,3.698196E-1,-7.621937E-3,3.8831128E-4,-3.9534763E-1,-4.2695575E-3,-4.9766037E-3,-2.3075259E-1,-2.910212E-1,-9.30281E-2,-3.7576923E-1,-3.2020617E-2,7.1031146E-2,-1.0258791E-1,5.824676E-4,-2.4173677E-1,-2.1200903E-1,1.5893976E-3,1.5678743E-2,1.6614638E-1,5.87003E-3,1.405876E-3,1.10222556E-1,-2.0189263E-1,-1.3502848E-3,2.624422E-1,1.11344926E-1,-3.5708643E-2,8.19227E-4,3.783038E-1,1.8270288E-1,-2.8766137E-2,3.2911843E-1,-8.354947E-3,3.5599896E-1,2.6182918E-2,7.9507124E-1,1.4135402E-2,3.64063E-3,2.335258E-2,-3.3707044E-1,-4.7670263E-1,5.5863406E-3,-6.0001863E-3,-1.6127607E-2,-1.5943821E-3,-2.9655686E-3,-3.1444147E-1,2.7288396E-2,-1.230318E-1,-1.0543343E-2,-4.281391E-1,3.4443801E-3,7.352528E-3,1.4278679E-3,-1.4604068E-1,-1.630179E-2,-9.5327705E-2,-7.322823E-2,-1.9916672E-2,3.375031E-2,-5.668018E-3,6.965579E-2,-4.3535763E-3,1.0984553E-2,2.831896E-3,2.071923E-1,-5.757317E-3,-1.9149872E-2,-2.6035389E-2,2.8901154E-1,1.10567905E-1,-4.011358E-2,1.9771707E-1,-4.7556735E-3,-2.2344165E-1,2.1848662E-2,8.736158E-3,1.1240756E-3,1.1254942E-2,2.849864E-2,-9.044208E-3,9.710715E-2,3.6936462E-1,2.4912652E-2,2.4675043E-1,3.7781075E-2,2.0149276E-2,-2.4418607E-1,-3.8531062E-1,-3.404038E-2,-4.1112208E-1,-9.160771E-3,-1.5732905E-2,-1.6468738E-3,4.4976445E-3,-6.2192682E-2,-1.7739557E-1,-2.175964E-2,-1.1202959E-2,-5.877514E-2,4.0916502E-3,-9.827798E-3,-1.5146399E-3,-7.0481887E-3,-3.5027225E-4,-7.862704E-3,2.9995697E-4,8.753806E-2,-3.038841E-3,-7.617977E-4,5.0962986E-3,1.0533791E-1,1.6271347E-2,1.11285366E-1,-6.3159033E-3,5.706902E-3,-9.936016E-3,1.4836109E-1,3.144091E-1,1.5741846E-3,6.902647E-3,-7.4920487E-3,2.439277E-3,3.5132426E-3,1.0489423E-2,-2.4694036E-1,2.5583463E-2,-1.4868317E-2,-4.3025813E-3,4.087016E-3,-2.8321285E-2,5.378837E-4,7.2296844E-3,3.958921E-1,2.7818275E-3,4.9096565E-3,1.3313295E-2,-3.2483886E-3,-1.3930216E-2,-1.887233E-2,-9.289061E-3,-2.327068E-2,-1.4219126E-2,-7.5650658E-3,-1.258577E-4,-1.0275359E-2,-1.9765997E-3,1.2900188E-3,-5.1884297E-3,5.4901233E-4,5.962981E-3,-1.8559955E-3,2.0216622E-3,7.915611E-3,1.5577816E-3,1.4711095E-3,7.7831917E-3,8.856118E-3,3.0809408E-3,1.1301039E-2,1.7829586E-2,-5.616756E-3,-1.4918382E-2,1.1033503E-2,4.401701E-4,-4.265691E-3,1.3740485E-3,2.468877E-2,1.4867516E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,29,-1,31,33,35,37,-1,-1,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,-1,65,67,-1,69,71,73,-1,-1,75,77,79,81,83,85,-1,87,89,91,-1,-1,93,95,97,99,-1,101,103,105,-1,-1,107,-1,109,111,113,115,117,-1,119,121,-1,123,125,127,129,131,-1,-1,133,135,-1,137,139,141,-1,143,145,147,149,-1,151,-1,153,-1,-1,-1,155,157,-1,-1,-1,-1,-1,159,161,163,-1,165,167,-1,-1,169,-1,171,173,-1,175,-1,177,-1,-1,-1,179,181,183,-1,185,187,189,191,193,195,-1,-1,-1,-1,197,-1,199,201,-1,203,-1,-1,205,207,-1,209,-1,-1,-1,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,217,219,-1,-1,221,-1,223,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,229,231,-1,-1,-1,233,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.906312E1,2.699833E1,1.1900219E1,2.9827728E0,6.576001E0,7.008114E0,2.2292304E-1,3.320097E0,1.422535E0,2.6471834E0,2.119548E0,2.7886963E-1,6.600483E0,0E0,0E0,0E0,3.496665E0,6.619015E-1,0E0,1.8858919E0,5.49561E-1,1.1319208E0,8.285201E-1,0E0,0E0,2.3100548E0,1.3799553E0,1.2101727E0,8.966675E-1,0E0,5.0992966E-2,1.6469278E0,5.707963E-1,3.152526E-1,1.3195938E-1,1.4178E0,1.7818308E-1,8.4999305E-1,0E0,1.3131709E0,0E0,4.5188427E-1,1.0267811E0,0E0,1.04661435E-1,3.7758636E-1,2.5102952E-1,0E0,0E0,8.040533E-1,3.7633705E-1,3.2702065E-1,3.4002143E-1,4.8688287E-1,1.4298357E-1,0E0,1.786182E-2,1.0044371E0,4.0900922E-1,0E0,0E0,4.6420935E-1,4.7921723E-1,3.86478E-1,1.5774946E0,0E0,2.651682E-1,5.1159286E-1,4.3391085E-1,0E0,0E0,3.8928795E-1,0E0,1.5722403E-1,2.8984672E-1,1.92312E-1,1.6941464E-1,1.3411283E-1,0E0,1.7811963E-1,1.03125855E-1,0E0,2.9175115E-1,4.5493335E-1,1.3603643E-1,1.430191E-1,5.3008273E-2,0E0,0E0,3.723287E-1,9.8055714E-1,0E0,2.2226238E-1,3.7422144E-1,5.3353965E-1,0E0,8.959854E-2,1.291348E-1,2.2553302E-1,6.3643646E-1,0E0,2.873497E-1,0E0,3.472519E-2,0E0,0E0,0E0,2.240467E-1,4.8177433E-1,0E0,0E0,0E0,0E0,0E0,4.1629553E-2,4.756647E-2,1.1134267E-1,0E0,5.2872658E-2,1.0053718E-1,0E0,0E0,9.120992E-2,0E0,4.9063466E-2,8.613539E-2,0E0,5.3693768E-2,0E0,5.7856083E-2,0E0,0E0,0E0,2.2875565E-1,2.6044804E-1,3.3405617E-1,0E0,1.4267969E-1,2.7592897E-2,1.33308E-1,4.6972096E-2,5.941316E-1,1.3670337E-1,0E0,0E0,0E0,0E0,6.383695E-2,0E0,5.907937E-2,4.764223E-1,0E0,5.6659102E-2,0E0,0E0,2.4798572E-1,1.1013079E-1,0E0,2.1811104E-1,0E0,0E0,0E0,0E0,1.1430286E-1,1.05626225E-1,0E0,0E0,5.4187227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.303025E-2,3.1301804E-2,0E0,0E0,5.094424E-2,0E0,3.5522006E-2,0E0,0E0,0E0,2.2607997E-2,1.252265E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6460927E-2,2.4423552E-1,0E0,0E0,0E0,3.991334E-2,0E0,0E0,4.0260315E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,61,61,62,62,63,63,64,64,66,66,67,67,68,68,71,71,73,73,74,74,75,75,76,76,77,77,79,79,80,80,82,82,83,83,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,96,96,97,97,98,98,99,99,101,101,103,103,107,107,108,108,114,114,115,115,116,116,118,118,119,119,122,122,124,124,125,125,127,127,129,129,133,133,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,147,147,149,149,150,150,152,152,155,155,156,156,158,158,163,163,164,164,167,167,175,175,176,176,179,179,181,181,185,185,186,186,193,193,194,194,198,198,201,201],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,30,-1,32,34,36,38,-1,-1,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,-1,66,68,-1,70,72,74,-1,-1,76,78,80,82,84,86,-1,88,90,92,-1,-1,94,96,98,100,-1,102,104,106,-1,-1,108,-1,110,112,114,116,118,-1,120,122,-1,124,126,128,130,132,-1,-1,134,136,-1,138,140,142,-1,144,146,148,150,-1,152,-1,154,-1,-1,-1,156,158,-1,-1,-1,-1,-1,160,162,164,-1,166,168,-1,-1,170,-1,172,174,-1,176,-1,178,-1,-1,-1,180,182,184,-1,186,188,190,192,194,196,-1,-1,-1,-1,198,-1,200,202,-1,204,-1,-1,206,208,-1,210,-1,-1,-1,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,218,220,-1,-1,222,-1,224,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,230,232,-1,-1,-1,234,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.7909248E5,9.7172376E7,1E0,9.31E2,2.4481E4,1.665909E3,3.89E2,3.9935064E0,1.5859042E6,3.7326388E0,6.23694E5,1.8062708E6,-3.1316992E-2,-1.239372E-2,-2.764794E-2,9.83871E-1,1.0013907E1,-5.0783135E-2,2.956111E2,5.5426865E6,5.405423E6,1.2294118E1,3.617117E-2,6.5777734E-2,7.6255713E3,6.355E3,2E0,3.3238492E2,-9.668635E-3,1.6074808E5,3.4698796E0,1.934E3,2.732381E2,5.894636E6,1.084437E6,1E0,2.445E3,-1.4016416E-2,7.075161E2,3.7666537E-2,1E0,5.6926E-3,1.843852E-2,6.923077E-1,1.803948E7,1.1544592E1,-3.1929143E-2,-1.8278925E-2,4.33E2,5.4051723E0,1E0,1.2E1,2.331083E6,1.2397727E1,1.3444636E-2,1.4E1,3.72381E5,3.7760816E5,9.602993E-3,2.3317957E-2,9.777778E0,9.68E2,2.0421052E0,3.0070068E7,4.47477E-3,1.0089981E3,1E0,1.3655363E2,-7.621937E-3,3.8831128E-4,1.2E1,-4.2695575E-3,1E0,1.1939979E8,1.0201707E-3,9E0,7.106795E6,-3.2020617E-2,3.82E2,1.1652361E6,5.824676E-4,3.90225E2,3E0,2.0487332E-3,1E0,1.5539158E3,5.87003E-3,1.405876E-3,1.2562814E0,6.9508715E0,-1.3502848E-3,2.23099E7,1.5026077E7,2.0734E4,8.19227E-4,3.4779343E1,3.9896116E7,1.7E1,6.996357E4,-8.354947E-3,2.5959E4,2.6182918E-2,2.4080592E7,1.4135402E-2,3.64063E-3,2.335258E-2,1.4E1,1.63757E4,5.5863406E-3,-6.0001863E-3,-1.6127607E-2,-1.5943821E-3,-2.9655686E-3,1E0,6.407767E-1,1.6808511E0,-1.0543343E-2,3.1540035E9,2.8E1,7.352528E-3,1.4278679E-3,5.9506125E8,-1.630179E-2,2.335E3,3.39E2,-1.9916672E-2,3.5364E4,-5.668018E-3,5.44E2,-4.3535763E-3,1.0984553E-2,2.831896E-3,2.9080848E6,1.57E2,4.5154482E-1,-2.6035389E-2,5.1604336E7,4.8E2,1.1E1,3.7788504E7,2.1924414E2,2.2503355E1,2.1848662E-2,8.736158E-3,1.1240756E-3,1.1254942E-2,1.1E1,-9.044208E-3,2.829581E10,4.2964826E0,2.4912652E-2,4E0,3.7781075E-2,2.0149276E-2,2E0,4.6463413E0,-3.404038E-2,1.547E3,-9.160771E-3,-1.5732905E-2,-1.6468738E-3,4.4976445E-3,1.407037E2,6.9E2,-2.175964E-2,-1.1202959E-2,1.4473684E-1,4.0916502E-3,-9.827798E-3,-1.5146399E-3,-7.0481887E-3,-3.5027225E-4,-7.862704E-3,2.9995697E-4,1.2E1,6.47E2,-7.617977E-4,5.0962986E-3,7.7983914E4,1.6271347E-2,1.7638037E0,-6.3159033E-3,5.706902E-3,-9.936016E-3,4.6E1,4.797342E2,1.5741846E-3,6.902647E-3,-7.4920487E-3,2.439277E-3,3.5132426E-3,1.0489423E-2,1.196966E6,7E0,-1.4868317E-2,-4.3025813E-3,4.087016E-3,2.8003533E0,5.378837E-4,7.2296844E-3,9.643839E6,2.7818275E-3,4.9096565E-3,1.3313295E-2,-3.2483886E-3,-1.3930216E-2,-1.887233E-2,-9.289061E-3,-2.327068E-2,-1.4219126E-2,-7.5650658E-3,-1.258577E-4,-1.0275359E-2,-1.9765997E-3,1.2900188E-3,-5.1884297E-3,5.4901233E-4,5.962981E-3,-1.8559955E-3,2.0216622E-3,7.915611E-3,1.5577816E-3,1.4711095E-3,7.7831917E-3,8.856118E-3,3.0809408E-3,1.1301039E-2,1.7829586E-2,-5.616756E-3,-1.4918382E-2,1.1033503E-2,4.401701E-4,-4.265691E-3,1.3740485E-3,2.468877E-2,1.4867516E-2],"split_indices":[2,43,60,6,2,12,4,1,68,43,69,1,43,0,0,0,68,71,0,4,43,43,68,0,0,67,2,10,67,0,48,71,44,4,60,9,29,2,0,67,0,15,72,0,68,9,71,0,0,2,69,26,3,43,73,0,3,9,43,0,0,73,10,68,58,0,4,6,71,0,0,8,0,13,7,53,3,5,0,4,60,0,67,8,53,26,48,0,0,68,71,0,9,5,44,0,71,5,3,48,0,9,0,9,0,0,0,3,43,0,0,0,0,0,73,71,68,0,46,0,0,0,46,0,44,44,0,9,0,12,0,0,0,66,0,53,0,7,10,3,7,67,71,0,0,0,0,3,0,46,68,0,8,0,0,32,68,0,10,0,0,0,0,67,2,0,0,71,0,0,0,0,0,0,0,3,2,0,0,48,0,68,0,0,0,0,4,0,0,0,0,0,0,9,3,0,0,0,68,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.044E3,7.94E2,2.5E2,2.43E2,5.51E2,2.39E2,1.1E1,2.08E2,3.5E1,2.82E2,2.69E2,1.3E1,2.26E2,7E0,4E0,4.8E1,1.6E2,2.8E1,7E0,1.9E2,9.2E1,1.29E2,1.4E2,7E0,6E0,1.18E2,1.08E2,2E1,1.4E2,6E0,2.2E1,1.19E2,7.1E1,7.7E1,1.5E1,1.13E2,1.6E1,1.33E2,7E0,1.12E2,6E0,4.6E1,6.2E1,7E0,1.3E1,1.2E2,2E1,1.6E1,6E0,8.4E1,3.5E1,4.5E1,2.6E1,4.7E1,3E1,6E0,9E0,4.6E1,6.7E1,5E0,1.1E1,1.18E2,1.5E1,3.5E1,7.7E1,4E0,4.2E1,5.1E1,1.1E1,7E0,6E0,1.15E2,5E0,8E0,1.2E1,3.9E1,4.5E1,2.9E1,6E0,2.9E1,1.6E1,7E0,1.9E1,1.5E1,3.2E1,2.2E1,8E0,5E0,4E0,3.2E1,1.4E1,5E0,6.2E1,2.7E1,9.1E1,6E0,9E0,1.3E1,2.2E1,7.1E1,6E0,2E1,2.2E1,4.7E1,4E0,4E0,7E0,7E1,4.5E1,4E0,4E0,7E0,5E0,4E0,3.5E1,9E0,3.6E1,9E0,2E1,1.7E1,1.2E1,4E0,1.2E1,1E1,9E0,1E1,5E0,2.6E1,6E0,1.5E1,7E0,4E0,4E0,1.7E1,1.5E1,1E1,4E0,5.2E1,1E1,1E1,1.7E1,7.9E1,1.2E1,5E0,4E0,4E0,9E0,1.7E1,5E0,1.1E1,6E1,6E0,1.4E1,4.3E1,4E0,2.6E1,4.4E1,7E0,3.8E1,8E0,2.7E1,5E0,4E0,1.8E1,1.8E1,1.5E1,5E0,1E1,7E0,7E0,5E0,5E0,4E0,4E0,6E0,1E1,1.6E1,5E0,1E1,1.1E1,6E0,8E0,7E0,6E0,4E0,9E0,4.3E1,4E0,6E0,4E0,6E0,4E0,1.3E1,8E0,7.1E1,6E0,6E0,8E0,9E0,5E0,6E0,5.5E1,5E0,4E0,1E1,7E0,1.9E1,3.8E1,6E0,1.8E1,2E1,6E0,1.2E1,1.3E1,5E0,4E0,6E0,4E0,6E0,9E0,7E0,5E0,6E0,4E0,4E0,5E0,4E0,2.4E1,1.9E1,4E0,4E0,4E0,6.7E1,4E0,5E0,1.7E1,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[8.597817E-3,-1.2879427E-1,4.256468E-1,-3.8591677E-1,-3.969808E-3,4.8857808E-1,-3.875296E-2,-5.740737E-1,-3.0354354E-1,-1.351161E-1,9.396026E-2,6.364543E-2,4.5750412E-1,-4.631766E-1,1.5592837E-1,-4.4338825E-1,-7.1076024E-1,-3.5070378E-1,2.542251E-2,-2.0304939E-1,-1.0437911E-2,5.080184E-2,2.8214893E-1,3.692196E-1,6.972939E-1,-2.5734607E-2,-1.1915106E-2,2.7503145E-1,-6.402389E-3,-4.653785E-1,-1.0974521E-2,-6.29406E-1,-4.9571794E-2,1.263199E-2,-3.7785378E-1,1.9859787E-2,-1.10807136E-1,-1.7515352E-1,-4.2941716E-1,1.7590694E-2,-1.39172375E-2,6.307764E-2,-2.044582E-2,8.2467906E-2,3.35956E-1,1.3044524E-1,4.5310163E-1,7.579595E-1,2.5552644E-3,2.8878017E-3,3.6595058E-1,-2.2936359E-2,-1.2272982E-2,-3.387607E-2,-4.0385985E-1,-3.350037E-1,-6.8834525E-1,-1.1423461E-2,-3.9302424E-2,-2.7000123E-1,-1.2789492E-1,-1.2143544E-2,-2.9357424E-2,-1.623632E-2,8.819292E-3,1.4592235E-1,1.4928196E-2,9.02584E-5,5.4531014E-3,3.8234964E-1,3.935705E-3,7.739058E-2,1.6302979E-2,6.589423E-1,3.963303E-1,4.2970426E-2,2.8703295E-2,2.2502005E-2,8.49337E-3,-2.2833815E-2,-1.067643E-2,-4.057083E-1,-2.4064533E-1,-1.3344708E-2,-8.307296E-1,1.5412492E-3,-7.4951584E-3,-3.0325803E-1,-4.192686E-3,-8.940315E-2,-2.3930645E-1,-9.753107E-3,8.43118E-4,8.251604E-2,2.4233465E-1,-2.1061499E-2,1.0581072E-1,1.8188989E-1,4.331278E-1,1.7360204E-1,3.889039E-2,3.307639E-2,1.3818471E-2,5.214149E-4,4.1759622E-1,-2.628421E-3,-4.3016022E-1,-3.1865135E-2,-3.243929E-1,-5.8177676E-2,-2.1207329E-2,-5.5476143E-3,-1.4923227E-2,-1.2677565E-1,-1.2395965E-2,-4.294914E-3,-2.88374E-1,-4.682604E-2,9.375582E-2,1.2210748E-1,-1.5241143E-2,2.8454947E-1,-2.74521E-3,-6.2461957E-2,9.004178E-2,-7.915614E-2,1.718667E-1,2.2793636E-3,1.2846289E-2,1.1451453E-2,2.2317665E-2,2.1162769E-3,1.1603518E-2,-4.8818752E-2,1.1231229E-1,2.7839504E-3,4.4240287E-1,-4.7453263E-1,-3.1379494E-1,-1.2640013E-2,5.6744467E-2,-2.9149225E-1,-2.0662732E-2,-1.7108446E-1,-5.713807E-2,-6.658633E-2,6.471014E-2,-1.6361587E-2,-8.2384255E-3,1.0217967E-2,-9.620748E-2,5.180747E-3,1.4862325E-3,7.362799E-2,3.0866265E-1,1.6457209E-1,3.666162E-1,-8.473663E-2,1.3333295E-2,1.8417571E-1,-6.650711E-2,-5.2396295E-5,-8.692818E-3,1.1907593E-1,1.209602E-2,3.9888298E-4,-5.4185004E-3,8.2136495E-3,5.5828143E-4,4.9335462E-1,1.9502313E-1,-1.8970046E-2,-2.715469E-2,-7.6681697E-3,-1.55885415E-2,-1.7287824E-3,1.0797878E-2,-1.7378371E-2,-1.0187471E-2,7.745435E-5,-9.295376E-3,2.5393679E-3,-4.1901073E-3,-2.0631547E-4,-4.2035575E-3,5.1161516E-3,-1.6028775E-4,-1.9313481E-3,4.868096E-3,-5.922737E-3,1.4695912E-3,-1.0086013E-3,5.485132E-3,6.548525E-3,1.8309431E-2,1.1172498E-2,2.4375792E-3,1.9502638E-2,8.983798E-3,6.6987835E-3,-5.088571E-3,1.7708457E-3,1.0187459E-2,-7.0623914E-3,2.0943263E-3,7.5206594E-3,9.939047E-5,1.6885199E-2,2.5971701E-2,2.066766E-3,1.5694655E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,-1,51,-1,53,-1,-1,55,-1,57,59,61,63,-1,65,-1,67,69,71,73,75,-1,-1,77,-1,-1,-1,79,81,83,-1,85,87,89,-1,-1,91,-1,93,95,-1,-1,97,-1,99,-1,101,103,-1,-1,-1,-1,-1,-1,105,107,-1,109,-1,-1,111,-1,113,115,-1,117,119,121,123,125,127,129,131,133,-1,-1,-1,135,-1,137,139,141,-1,-1,-1,-1,143,145,-1,147,149,151,153,-1,155,-1,157,159,161,163,-1,-1,-1,-1,-1,-1,165,167,-1,169,171,173,-1,175,177,-1,179,181,183,185,-1,-1,187,189,-1,-1,191,193,195,197,199,-1,201,203,-1,-1,205,-1,-1,-1,-1,-1,207,209,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.827084E1,2.4551277E1,7.3837013E0,3.7005157E0,6.6397686E0,5.5028152E0,2.6336706E0,1.0245762E0,2.7394009E0,1.8697062E0,2.3897562E0,0E0,4.3152046E0,4.9002647E-2,8.098125E-1,7.503605E-2,3.5609818E-1,2.712862E0,1.3148947E0,8.277917E-1,6.492217E-1,1.4935653E0,5.6322813E-1,3.1455154E0,2.1780167E0,0E0,0E0,2.8628445E-1,0E0,9.164715E-2,0E0,4.3002796E-1,0E0,0E0,1.7384796E0,0E0,1.6591617E-1,5.436938E-1,3.4774804E-1,4.3105894E-1,0E0,9.4319725E-1,0E0,3.699687E-2,4.6768665E-1,4.8015428E-1,1.1177711E0,6.4554214E-1,0E0,0E0,1.5307367E-1,0E0,0E0,0E0,7.31715E-2,7.830162E-1,7.2702694E-1,0E0,1.2153871E-1,2.2285032E-1,3.559867E-1,0E0,0E0,2.0653522E-1,0E0,5.118475E-1,4.961245E-1,0E0,0E0,2.8766203E-1,0E0,1.2659828E-1,0E0,2.7820778E-1,7.7317715E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.066866E-1,1.0212688E0,0E0,1.3333883E0,0E0,0E0,8.791113E-2,0E0,1.9125122E-1,1.3116431E-1,0E0,2.612763E-1,9.2200714E-1,4.5001972E-1,5.0699437E-1,5.389855E-1,1.0101345E-1,1.3934946E-1,8.761391E-2,1.7935678E-1,0E0,0E0,0E0,7.7112865E-1,0E0,2.0736885E-1,4.0108418E-1,3.9402008E-2,0E0,0E0,0E0,0E0,1.290747E-1,1.004122E-1,0E0,5.2670717E-2,1.10842735E-1,1.903607E-2,4.294787E-1,0E0,2.3504996E-1,0E0,6.5004814E-1,4.5434594E-1,1.018431E-1,1.20788395E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.214985E-2,9.6289545E-2,0E0,9.6255875E-1,1.0608387E-1,2.082491E-2,0E0,2.4206087E-1,1.334076E-1,0E0,1.4082551E-1,7.646449E-2,2.0815466E-2,3.1768225E-2,0E0,0E0,9.845888E-2,9.008677E-2,0E0,0E0,1.791074E-1,9.0132356E-2,1.0818079E-1,8.0421925E-2,4.595884E-1,0E0,9.223825E-2,1.2283397E-1,0E0,0E0,1.12092614E-1,0E0,0E0,0E0,0E0,0E0,4.0167427E-1,3.3273482E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,31,31,34,34,36,36,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,50,50,54,54,55,55,56,56,58,58,59,59,60,60,63,63,65,65,66,66,69,69,71,71,73,73,74,74,81,81,82,82,84,84,87,87,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,104,104,106,106,107,107,108,108,113,113,114,114,116,116,117,117,118,118,119,119,121,121,123,123,124,124,125,125,126,126,133,133,134,134,136,136,137,137,138,138,140,140,141,141,143,143,144,144,145,145,146,146,149,149,150,150,153,153,154,154,155,155,156,156,157,157,159,159,160,160,163,163,169,169,170,170],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,-1,52,-1,54,-1,-1,56,-1,58,60,62,64,-1,66,-1,68,70,72,74,76,-1,-1,78,-1,-1,-1,80,82,84,-1,86,88,90,-1,-1,92,-1,94,96,-1,-1,98,-1,100,-1,102,104,-1,-1,-1,-1,-1,-1,106,108,-1,110,-1,-1,112,-1,114,116,-1,118,120,122,124,126,128,130,132,134,-1,-1,-1,136,-1,138,140,142,-1,-1,-1,-1,144,146,-1,148,150,152,154,-1,156,-1,158,160,162,164,-1,-1,-1,-1,-1,-1,166,168,-1,170,172,174,-1,176,178,-1,180,182,184,186,-1,-1,188,190,-1,-1,192,194,196,198,200,-1,202,204,-1,-1,206,-1,-1,-1,-1,-1,208,210,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,2.16577E5,2.856934E7,7.7E1,8.1E2,3.3817584E7,1.465E4,9.55188E5,3.541824E2,1.5859042E6,2.8530578E6,6.364543E-2,4.1973075E6,1.7026364E3,7.5491136E-1,2.0376764E-7,7.45E0,6E0,1.8729E4,2E0,7.89E2,1.2294118E1,4.6712E4,7.445455E2,6.292039E0,-2.5734607E-2,-1.1915106E-2,1.5896863E3,-6.402389E-3,2.7546012E0,-1.0974521E-2,1.8987958E0,-4.9571794E-2,1.263199E-2,8.719101E0,1.9859787E-2,7.70947E5,4.39E2,1.7913036E6,5.5426865E6,-1.39172375E-2,1.4392524E0,-2.044582E-2,2.524828E2,1.0039E4,6.103178E8,2.6793549E1,3.9914E4,2.5552644E-3,2.8878017E-3,5.2224144E5,-2.2936359E-2,-1.2272982E-2,-3.387607E-2,7E0,8.81059E5,1.7915686E2,-1.1423461E-2,8.215373E6,1.7566667E2,1.36E2,-1.2143544E-2,-2.9357424E-2,3.0502679E-6,8.819292E-3,7.5491136E-1,1.4603988E6,9.02584E-5,5.4531014E-3,3.2911258E6,3.935705E-3,3.56E3,1.6302979E-2,4.8149962E8,1E0,4.2970426E-2,2.8703295E-2,2.2502005E-2,8.49337E-3,-2.2833815E-2,-1.067643E-2,1.4953704E0,1.8E0,-1.3344708E-2,6.8E1,1.5412492E-3,-7.4951584E-3,1.9E1,-4.192686E-3,2.118835E6,9.836066E-1,-9.753107E-3,2.485E2,4.5158855E6,2.0110958E0,9.751103E6,4.7563504E7,1.935E3,6.4258E4,4.3566666E2,1.0039E4,3.307639E-2,1.3818471E-2,5.214149E-4,3.808001E5,-2.628421E-3,2.2470966E-1,3.6E2,1.04618106E6,-5.8177676E-2,-2.1207329E-2,-5.5476143E-3,-1.4923227E-2,5.9565744E5,3.056087E2,-4.294914E-3,4.5172415E0,5.56E2,1.6E1,5.372237E2,-1.5241143E-2,1.387E3,-2.74521E-3,1.0564016E3,2.5417458E1,1.72959E5,6.328E2,2.2793636E-3,1.2846289E-2,1.1451453E-2,2.2317665E-2,2.1162769E-3,1.1603518E-2,2.73E2,1.99E2,2.7839504E-3,1E0,6E0,5.912179E3,-1.2640013E-2,1.4E1,1.7915686E2,-2.0662732E-2,9E0,2.9276064E3,5.13657E6,5.539245E6,-1.6361587E-2,-8.2384255E-3,5.08E2,1.118178E6,5.180747E-3,1.4862325E-3,1.071E3,1.47E3,3.6245735E0,2.277593E5,3.5E1,1.3333295E-2,9.72E2,6.648E3,-5.2396295E-5,-8.692818E-3,4.7185095E2,1.209602E-2,3.9888298E-4,-5.4185004E-3,8.2136495E-3,5.5828143E-4,1.1251919E6,4.797448E8,-1.8970046E-2,-2.715469E-2,-7.6681697E-3,-1.55885415E-2,-1.7287824E-3,1.0797878E-2,-1.7378371E-2,-1.0187471E-2,7.745435E-5,-9.295376E-3,2.5393679E-3,-4.1901073E-3,-2.0631547E-4,-4.2035575E-3,5.1161516E-3,-1.6028775E-4,-1.9313481E-3,4.868096E-3,-5.922737E-3,1.4695912E-3,-1.0086013E-3,5.485132E-3,6.548525E-3,1.8309431E-2,1.1172498E-2,2.4375792E-3,1.9502638E-2,8.983798E-3,6.6987835E-3,-5.088571E-3,1.7708457E-3,1.0187459E-2,-7.0623914E-3,2.0943263E-3,7.5206594E-3,9.939047E-5,1.6885199E-2,2.5971701E-2,2.066766E-3,1.5694655E-2],"split_indices":[2,43,60,44,2,7,44,46,67,43,43,0,43,4,53,52,69,3,1,6,2,68,1,67,53,0,0,4,0,68,0,68,0,0,69,0,9,2,60,43,0,68,0,4,44,7,73,9,0,0,43,0,0,0,6,9,4,0,12,67,0,0,0,53,0,53,43,0,0,43,0,2,0,5,26,0,0,0,0,0,0,69,68,0,0,0,0,67,0,9,71,0,67,62,53,9,7,2,1,67,44,0,0,0,43,0,53,2,60,0,0,0,0,43,4,0,69,2,3,4,0,2,0,4,73,12,4,0,0,0,0,0,0,0,8,0,6,8,48,0,3,4,0,3,62,9,5,0,0,2,9,0,0,2,2,71,48,0,0,44,44,0,0,4,0,0,0,0,0,43,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,7.64E2,2.51E2,2.49E2,5.15E2,2.21E2,3E1,7.4E1,1.75E2,2.2E2,2.95E2,6E0,2.15E2,9E0,2.1E1,4E1,3.4E1,1.53E2,2.2E1,1.42E2,7.8E1,2.41E2,5.4E1,1.59E2,5.6E1,5E0,4E0,1.5E1,6E0,3.5E1,5E0,3E1,4E0,6E0,1.47E2,5E0,1.7E1,1.28E2,1.4E1,7.2E1,6E0,2.36E2,5E0,1.2E1,4.2E1,4.2E1,1.17E2,5.1E1,5E0,5E0,1E1,2.9E1,6E0,1.9E1,1.1E1,1.31E2,1.6E1,5E0,1.2E1,4.1E1,8.7E1,9E0,5E0,6.1E1,1.1E1,8.6E1,1.5E2,4E0,8E0,3.5E1,7E0,3.5E1,7E0,2.3E1,9.4E1,2E1,3.1E1,5E0,5E0,6E0,5E0,7.3E1,5.8E1,5E0,1.1E1,8E0,4E0,3.4E1,7E0,6.6E1,2.1E1,4E0,5.7E1,5.3E1,3.3E1,1.08E2,4.2E1,8E0,2.7E1,9E0,2.6E1,1.9E1,4E0,5E0,8.9E1,5E0,6.8E1,1.7E1,4.1E1,4E0,7E0,4E0,3E1,4.4E1,2.2E1,6E0,1.5E1,3.8E1,1.9E1,4.9E1,4E0,2.9E1,4E0,7.9E1,2.9E1,1.1E1,3.1E1,4E0,4E0,7E0,2E1,4E0,5E0,1.2E1,1.4E1,6E0,8.3E1,4.7E1,2.1E1,4E0,1.3E1,3.5E1,6E0,2.6E1,1.8E1,1.3E1,9E0,8E0,7E0,1.8E1,2E1,1.4E1,5E0,4E1,9E0,1.3E1,1.6E1,7.5E1,4E0,1.8E1,1.1E1,7E0,4E0,2.1E1,1E1,7E0,5E0,8E0,6E0,6.8E1,1.5E1,3.3E1,1.4E1,4E0,1.7E1,9E0,4E0,1.4E1,2.1E1,4E0,2.2E1,4E0,1.4E1,4E0,9E0,5E0,4E0,1.2E1,6E0,1.6E1,4E0,1.3E1,2.7E1,4E0,5E0,7E0,6E0,1.1E1,5E0,7E0,6.8E1,4E0,1.4E1,6E0,5E0,1.5E1,6E0,2.6E1,4.2E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"211","size_leaf_vector":"1"}},{"base_weights":[3.220995E-3,-1.2848191E-1,3.9728412E-1,-2.554451E-1,4.9032506E-2,-3.9518216E-1,4.3124267E-1,-3.9445123E-1,-1.5593712E-1,-6.254721E-2,1.4237507E-1,-2.3489894E-2,-7.279638E-3,1.9914469E-1,5.425681E-1,-3.6553645E-1,-8.639063E-1,-9.621763E-2,-5.5182266E-1,-4.52914E-2,-2.1142263E-2,1.6072036E-1,-7.876543E-2,1.0174252E-1,4.7259173E-1,5.721472E-1,-9.554424E-3,-5.348445E-1,-3.0329442E-1,-1.1962927E-2,-5.6550313E-2,4.4917592E-1,-1.1809183E-1,-7.22109E-1,-2.87986E-1,-9.610479E-2,2.4908427E-2,1.0262005E-1,2.3670018E-1,-7.750425E-3,1.5798523E-3,-8.152097E-2,1.5307474E-1,1.6141608E-2,2.6479976E-2,5.93647E-2,5.4065937E-1,-1.2771418E-2,-2.6180707E-2,-9.120253E-2,-3.320098E-1,1.0742796E-2,2.6609905E-2,-2.6936036E-1,-3.7991833E-2,-1.819198E-2,-3.6015943E-2,-2.7202473E-3,-1.6906269E-2,-6.44165E-2,-2.9478228E-1,-1.7224561E-2,1.4185068E-1,3.2275656E-1,7.220895E-2,-9.732608E-3,2.674715E-1,-8.080454E-3,-1.0206928E-2,3.3646703E-1,6.766417E-2,4.661095E-1,7.7206975E-1,9.574252E-3,-1.9897221E-1,-2.5316775E-1,-3.908813E-1,-3.996865E-1,-1.543419E-1,-1.5883903E-1,1.4510805E-2,-2.4133502E-2,-1.9332422E-1,-1.9171244E-2,-3.1213132E-3,-5.4154865E-2,1.1744698E-1,7.7865735E-2,1.23371985E-2,2.119341E-2,8.293495E-3,9.585977E-2,-1.4595016E-2,3.1908882E-1,1.5934035E-1,-3.2006756E-3,2.351885E-3,8.581722E-3,1.9232757E-2,1.1739641E-1,-6.5108836E-3,7.5008446E-1,4.0807354E-1,8.432998E-1,2.321695E-2,-1.2538147E-2,-1.3313857E-3,-2.7091724E-1,-1.4848002E-3,-2.1639124E-2,-2.8742635E-1,-2.3689473E-1,-2.4222445E-2,-5.7015553E-2,-2.2496103E-1,-7.381873E-2,-2.8952903E-1,4.1781794E-2,-1.5989126E-1,1.8479452E-2,-1.0176079E-1,-2.4506535E-1,-2.662048E-3,-6.7606554E-3,-1.4837989E-1,1.978625E-3,8.477528E-3,5.658173E-3,5.9344375E-4,1.120397E-1,-5.528606E-3,2.5520054E-1,4.1095024E-1,1.9344163E-1,-9.759751E-5,-1.0161004E-3,1.7425226E-1,3.885535E-2,1.9072609E-2,2.0911404E-1,4.7076926E-1,2.3250774E-2,4.1472994E-2,-1.5806496E-2,-2.315277E-1,-2.3873536E-1,-2.232034E-2,-3.4274857E-3,-1.586334E-2,-2.580456E-3,-9.651801E-3,-1.2431198E-2,-1.6153818E-4,-8.438632E-3,-2.1150062E-2,-1.576765E-2,-6.363268E-3,1.2111861E-1,-2.8508376E-2,-1.1337749E-2,1.9923884E-3,-1.4758602E-2,6.5053906E-3,-1.554788E-1,9.3379774E-4,-4.6505737E-3,-1.4076071E-2,-5.6894593E-2,4.938022E-3,-5.731622E-4,-1.5534965E-2,1.4330468E-1,-2.0489521E-2,1.6641114E-2,1.9518864E-1,2.1520257E-2,9.105019E-3,1.6813251E-3,1.0759529E-2,2.3198147E-1,1.0593004E-1,2.789836E-1,7.8698323E-4,5.118178E-1,2.3247096E-1,-4.2572315E-3,-1.1924129E-2,-4.202423E-3,-1.24546895E-2,-4.7498825E-3,3.495662E-3,1.7887024E-3,-3.4026618E-3,6.856046E-3,-6.184847E-4,3.8336935E-3,-2.7734078E-3,-6.407575E-3,4.454952E-4,-3.4182416E-3,-1.1632907E-2,-7.1993507E-3,-8.0827257E-4,2.2610591E-3,8.585639E-3,2.2618072E-3,-6.13617E-3,4.183312E-3,1.0350049E-2,5.4446855E-3,1.3324968E-2,1.1464513E-3,6.871469E-3,1.7427403E-2,4.469738E-3,2.5338389E-2,1.3185349E-2,-2.4528322E-3,1.6875928E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,71,-1,-1,73,75,-1,-1,77,79,-1,-1,-1,-1,81,83,85,87,89,91,-1,93,-1,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,121,-1,-1,123,125,127,-1,-1,-1,129,-1,131,133,-1,-1,-1,-1,135,-1,137,139,141,-1,-1,-1,143,-1,-1,145,147,-1,149,151,153,155,157,159,161,163,165,-1,167,169,-1,-1,-1,-1,171,-1,173,175,177,-1,-1,179,-1,-1,181,183,-1,-1,-1,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,-1,193,195,-1,-1,197,-1,199,-1,-1,-1,201,-1,-1,-1,203,205,-1,207,-1,-1,-1,-1,209,211,213,-1,215,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.283259E1,1.7208862E1,7.0327263E0,6.0938263E0,3.3313334E0,2.1593273E-1,6.2223587E0,2.1954384E0,6.1174884E0,9.7463465E-1,7.185576E-1,0E0,0E0,2.1092985E0,3.775711E0,1.6924877E0,1.8618231E0,2.773095E0,1.3195868E0,5.053537E-1,0E0,6.8643665E-1,1.4648151E-1,5.863254E-1,5.9955597E-2,2.8740883E0,0E0,2.2191334E-1,7.6754856E-1,0E0,0E0,1.1331332E-1,2.6554205E0,7.042885E-2,2.4636161E-1,4.9726284E-1,2.980709E-1,5.9929144E-1,1.0062056E0,0E0,0E0,9.144924E-2,7.2908795E-1,0E0,0E0,0E0,2.2968788E0,0E0,0E0,5.880052E-1,4.3481922E-1,0E0,0E0,1.0775442E0,9.2230046E-1,0E0,0E0,0E0,0E0,3.692152E-1,2.7236444E-1,2.3016608E-1,1.0737461E-1,1.3191128E-1,7.9218E-1,0E0,3.1194973E-1,0E0,3.6018983E-2,1.03150606E-1,3.6925775E-1,1.6551208E0,2.6735115E-1,0E0,1.5237588E-1,1.9529629E-1,3.7856007E-1,5.8584166E-1,2.7584398E-1,4.6752465E-1,4.9180454E-1,1.8712448E-1,1.05243266E-1,0E0,0E0,1.5976277E-1,4.142897E-2,3.3231974E-2,0E0,0E0,0E0,2.8889984E-1,0E0,1.4748669E-1,1.2704086E-1,0E0,0E0,0E0,0E0,2.2573695E-1,0E0,2.356205E-1,1.1614208E0,4.7662735E-2,0E0,0E0,0E0,5.544424E-2,0E0,0E0,1.7053485E-1,2.690333E-1,0E0,1.565683E-1,2.3355317E-1,1.566447E-1,8.6580396E-2,4.9977022E-1,2.5365123E-1,1.52524E-1,1.3418582E-1,7.263613E-2,0E0,1.485198E-1,3.023915E-1,0E0,0E0,0E0,0E0,3.0955702E-1,0E0,1.0613406E-1,9.594011E-2,1.0925019E-1,0E0,0E0,5.4309845E-2,0E0,0E0,3.441807E-1,6.244354E-1,0E0,0E0,0E0,1.0215378E-1,9.401369E-2,0E0,0E0,0E0,1.2542138E-1,0E0,0E0,0E0,0E0,6.57149E-2,0E0,0E0,1.5611923E-1,1.7368001E-1,0E0,0E0,9.206462E-2,0E0,9.1257304E-2,0E0,0E0,0E0,6.743945E-2,0E0,0E0,0E0,2.317313E-1,1.2351875E-1,0E0,4.0075958E-2,0E0,0E0,0E0,0E0,3.04268E-2,3.450325E-2,3.0365694E-1,0E0,3.7445068E-1,5.124518E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,46,46,49,49,50,50,53,53,54,54,59,59,60,60,61,61,62,62,63,63,64,64,66,66,68,68,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,86,86,87,87,91,91,93,93,94,94,99,99,101,101,102,102,103,103,107,107,110,110,111,111,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,124,124,129,129,131,131,132,132,133,133,136,136,139,139,140,140,144,144,145,145,149,149,154,154,157,157,158,158,161,161,163,163,167,167,171,171,172,172,174,174,179,179,180,180,181,181,183,183,184,184],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,72,-1,-1,74,76,-1,-1,78,80,-1,-1,-1,-1,82,84,86,88,90,92,-1,94,-1,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,122,-1,-1,124,126,128,-1,-1,-1,130,-1,132,134,-1,-1,-1,-1,136,-1,138,140,142,-1,-1,-1,144,-1,-1,146,148,-1,150,152,154,156,158,160,162,164,166,-1,168,170,-1,-1,-1,-1,172,-1,174,176,178,-1,-1,180,-1,-1,182,184,-1,-1,-1,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,-1,194,196,-1,-1,198,-1,200,-1,-1,-1,202,-1,-1,-1,204,206,-1,208,-1,-1,-1,-1,210,212,214,-1,216,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,7.785302E5,1E0,4.97E2,9.48E2,2.3308511E5,8.5023944E2,4.5E1,2.3E1,3.4E1,8.695652E0,-2.3489894E-2,-7.279638E-3,3.1993368E6,3.206931E2,3E1,5.5E1,2.74614E5,6.5139695E-3,2.331083E6,-2.1142263E-2,4.2210345E2,2.0394794E8,1E0,5.941442E6,3.3817584E7,-9.554424E-3,1.957E3,1.2051282E0,-1.1962927E-2,-5.6550313E-2,1.9E1,1.7302156E5,3.9401392E6,2.12791E5,1.4992306E8,2.9254214E2,1.8992E4,1.5043378E0,-7.750425E-3,1.5798523E-3,2.0939393E0,1.1057851E1,1.6141608E-2,2.6479976E-2,5.93647E-2,4.391553E6,-1.2771418E-2,-2.6180707E-2,1.9E1,8.6875E0,1.0742796E-2,2.6609905E-2,1.4912975E-1,7.69E2,-1.819198E-2,-3.6015943E-2,-2.7202473E-3,-1.6906269E-2,4.135135E0,1.01573E5,1E0,2.9816E4,9.8933E4,2.511E3,-9.732608E-3,3.875E0,-8.080454E-3,1.7909248E5,4.3566666E2,1.2544625E5,1.307E3,1.6808511E0,9.574252E-3,1E0,4.58E2,1.463E2,1.01E2,2.3412812E0,2E0,7.955228E9,1E0,4.3460325E6,-1.9171244E-2,-3.1213132E-3,6.83E2,2.0833333E0,1.4067796E0,1.23371985E-2,2.119341E-2,8.293495E-3,1E0,-1.4595016E-2,3.18E3,3.1206896E0,-3.2006756E-3,2.351885E-3,8.581722E-3,1.9232757E-2,5.595208E2,-6.5108836E-3,3.256391E1,3.3580637E8,4.029152E8,2.321695E-2,-1.2538147E-2,-1.3313857E-3,7.2864324E-1,-1.4848002E-3,-2.1639124E-2,4.046E3,1.0130841E1,-2.4222445E-2,1.32476E5,6.5162756E2,1E0,1E0,2.1340206E0,1.82E3,8.051603E7,1.559733E6,4.5650104E2,-2.662048E-3,4.49239E6,6.0052995E6,1.978625E-3,8.477528E-3,5.658173E-3,5.9344375E-4,3.472E3,-5.528606E-3,1.9197379E9,1.9E1,7.755551E7,-9.759751E-5,-1.0161004E-3,3.0588236E0,3.885535E-2,1.9072609E-2,1.260841E2,1E0,2.3250774E-2,4.1472994E-2,-1.5806496E-2,4.4908694E2,2.855464E5,-2.232034E-2,-3.4274857E-3,-1.586334E-2,2.5055911E2,-9.651801E-3,-1.2431198E-2,-1.6153818E-4,-8.438632E-3,1.2707424E0,-1.576765E-2,-6.363268E-3,4.41537E6,6.8808866E0,-1.1337749E-2,1.9923884E-3,4.48E2,6.5053906E-3,8.7601E4,9.3379774E-4,-4.6505737E-3,-1.4076071E-2,5.342944E6,4.938022E-3,-5.731622E-4,-1.5534965E-2,5.4007E7,2.5E0,1.6641114E-2,3.221296E6,2.1520257E-2,9.105019E-3,1.6813251E-3,1.0759529E-2,1.5E1,5.657353E1,2.1298597E0,7.8698323E-4,7.61E2,1.4595818E7,-4.2572315E-3,-1.1924129E-2,-4.202423E-3,-1.24546895E-2,-4.7498825E-3,3.495662E-3,1.7887024E-3,-3.4026618E-3,6.856046E-3,-6.184847E-4,3.8336935E-3,-2.7734078E-3,-6.407575E-3,4.454952E-4,-3.4182416E-3,-1.1632907E-2,-7.1993507E-3,-8.0827257E-4,2.2610591E-3,8.585639E-3,2.2618072E-3,-6.13617E-3,4.183312E-3,1.0350049E-2,5.4446855E-3,1.3324968E-2,1.1464513E-3,6.871469E-3,1.7427403E-2,4.469738E-3,2.5338389E-2,1.3185349E-2,-2.4528322E-3,1.6875928E-2],"split_indices":[2,43,17,2,2,48,67,6,3,3,61,0,0,43,73,70,0,5,53,43,0,67,7,27,60,7,0,9,68,0,0,0,43,60,12,5,70,9,69,0,0,68,71,0,0,0,43,0,0,0,73,0,0,53,2,0,0,0,0,69,1,28,1,1,2,0,69,0,43,67,48,44,68,0,8,2,70,0,68,32,46,26,60,0,0,2,68,68,0,0,0,79,0,44,68,0,0,0,0,67,0,73,7,7,0,0,0,73,0,0,44,71,0,1,4,8,30,68,2,7,9,48,0,43,43,0,0,0,0,10,0,46,3,5,0,0,68,0,0,71,6,0,0,0,66,60,0,0,0,4,0,0,0,0,68,0,0,60,73,0,0,2,0,9,0,0,0,60,0,0,0,7,68,0,60,0,0,0,0,3,71,53,0,8,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,7.62E2,2.54E2,4.44E2,3.18E2,1E1,2.44E2,1.84E2,2.6E2,1.45E2,1.73E2,6E0,4E0,8E1,1.64E2,1.75E2,9E0,2.27E2,3.3E1,1.4E2,5E0,1.6E2,1.3E1,6E1,2E1,1.58E2,6E0,4.5E1,1.3E2,4E0,5E0,8E0,2.19E2,1.9E1,1.4E1,8.1E1,5.9E1,9.2E1,6.8E1,7E0,6E0,1.3E1,4.7E1,1.1E1,9E0,5E0,1.53E2,6E0,3.9E1,1.6E1,1.14E2,4E0,4E0,7.5E1,1.44E2,4E0,1.5E1,4E0,1E1,7.1E1,1E1,4.4E1,1.5E1,1E1,8.2E1,4E0,6.4E1,5E0,8E0,1.4E1,3.3E1,1.18E2,3.5E1,4E0,1.2E1,5.1E1,6.3E1,3.4E1,4.1E1,4.3E1,1.01E2,5.5E1,1.6E1,6E0,4E0,3.5E1,9E0,1.1E1,4E0,4E0,6E0,7.8E1,4E0,4.2E1,2.2E1,4E0,4E0,6E0,8E0,2.7E1,6E0,1.8E1,1E2,2.6E1,9E0,8E0,4E0,4.7E1,4E0,3.4E1,2.9E1,1.6E1,1.8E1,1.8E1,2.3E1,2.7E1,1.6E1,8.8E1,1.3E1,3.6E1,1.9E1,1.1E1,5E0,2.4E1,1.1E1,5E0,4E0,6E0,5E0,7.3E1,5E0,2.7E1,1.5E1,1.8E1,4E0,8E0,1.9E1,1.3E1,5E0,2.5E1,7.5E1,5E0,2.1E1,1.4E1,3.3E1,2.5E1,4E0,7E0,9E0,1.4E1,4E0,1.9E1,4E0,8E0,1.9E1,1.1E1,5E0,4.1E1,4.7E1,9E0,4E0,2.9E1,7E0,1.3E1,6E0,4E0,7E0,1.7E1,7E0,7E0,4E0,5.9E1,1.4E1,8E0,1.9E1,1.1E1,4E0,4E0,1.4E1,9E0,1E1,1.8E1,7E0,6.3E1,1.2E1,6E0,2.7E1,5E0,2E1,6E0,8E0,9E0,1E1,3.4E1,7E0,1E1,3.7E1,4E0,2.5E1,8E0,5E0,4E0,1.3E1,1.9E1,4E1,9E0,5E0,5E0,1.4E1,4E0,5E0,4E0,6E0,1.1E1,7E0,5.3E1,1E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-4.4613914E-3,-1.9807957E-1,2.0014012E-1,-3.745944E-1,-9.339277E-2,2.4397902E-1,-5.112069E-1,-4.272029E-1,-2.056768E-1,-2.4976265E-1,-3.2554865E-2,1.2366045E-1,5.148828E-1,-3.2893173E-2,-3.8525602E-1,-3.9716187E-1,-6.6508645E-1,-2.5036967E-1,-7.9253405E-2,-1.6846828E-1,-4.205584E-1,-6.1155513E-2,7.092352E-2,3.402724E-1,6.586562E-2,6.2717986E-1,2.866561E-1,-8.778473E-3,-1.9504804E-2,-4.8264036E-1,-2.6317227E-1,-7.724392E-3,-8.973108E-1,-2.898917E-1,-5.6628706E-3,1.678695E-3,-1.2912284E-1,-2.0375587E-1,4.246028E-2,-3.2448512E-1,-3.3945143E-2,-4.4609047E-2,-3.218436E-1,1.6104831E-1,2.254725E-2,2.3030083E-1,4.8914957E-1,9.570672E-2,-2.0294215E-1,5.04491E-1,8.890067E-1,3.494736E-1,-3.648233E-3,-2.8487349E-2,-2.0430839E-2,1.521193E-2,-3.3034527E-1,-5.3707164E-2,-1.846122E-2,-1.6118526E-2,-1.0319226E-2,-2.2626396E-3,-8.056722E-3,-2.507658E-1,-8.392392E-2,-2.1121674E-4,4.1767093E-3,-9.395055E-3,-1.6742427E-2,6.4748465E-3,-9.873822E-2,-2.2507835E-2,-7.288339E-3,4.5214454E-3,1.0523641E-2,-6.8514915E-3,6.300965E-2,2.8972468E-1,1.711443E-2,2.4119057E-2,1.30020855E-2,1.7322531E-1,2.6793553E-2,-9.353181E-2,-3.9921367E-1,6.3372624E-1,2.465392E-1,1.0068929E0,1.730918E-2,1.009027E-3,3.93158E-1,-2.7900547E-1,-2.2283504E-2,-2.7296433E-1,-3.8428288E-3,-6.0890834E-3,1.192751E-3,5.172601E-2,-1.398573E-1,-1.1622585E-1,9.600464E-3,2.4009965E-2,7.501921E-3,3.495059E-1,1.7546707E-1,-1.2547317E-3,3.0870188E-3,2.0642166E-1,1.5180469E-2,-1.3186507E-1,6.2306833E-2,-1.759049E-1,1.451697E-3,-2.3611257E-2,-9.586669E-3,4.4171327E-1,7.382635E-1,2.8708488E-1,1.6192159E-3,5.0096635E-2,2.3613272E-2,3.2663527E-1,2.6048169E-2,-3.1818485E-1,-4.0383595E-3,-1.7529752E-2,-2.1945564E-1,7.809879E-2,-1.3863076E-1,-2.4217283E-2,-2.1805292E-1,-1.7754853E-1,-4.014939E-3,8.396162E-2,-1.6245382E-2,6.5664626E-3,4.091542E-1,1.8692205E-3,1.3185545E-2,2.1174159E-2,1.8297236E-1,-4.3492056E-2,7.360973E-3,9.450751E-3,-2.2338599E-1,7.9136506E-2,-1.0171062E-2,-1.1788716E-2,-3.110122E-3,5.527112E-1,1.9669945E-3,3.607859E-2,1.8216664E-2,4.285529E-3,3.4567493E-1,7.658027E-3,1.8598875E-2,-1.1427673E-2,-3.8196558E-1,-6.8128845E-3,-1.3173806E-2,4.3001838E-2,2.0977922E-1,-1.3725284E-3,-1.0155769E-2,1.4527359E-3,-3.980914E-3,-2.9408297E-1,-1.736542E-3,-2.4079856E-1,-1.0260431E-1,6.268127E-2,-1.24915786E-1,5.71508E-3,1.2670245E-3,-3.16025E-3,2.1106717E-3,2.3716556E-2,1.1725698E-2,1.7361093E-2,2.2473401E-1,7.4811797E-3,-6.7882463E-3,-6.691366E-3,6.303831E-3,-1.4504919E-2,-4.063392E-3,9.252638E-3,1.5028353E-1,1.269861E-2,2.9399319E-2,1.801399E-2,8.278784E-3,-1.9761374E-2,-9.464315E-3,-6.204706E-4,5.1875E-3,4.5663933E-3,1.1462093E-2,-6.2363097E-3,-1.8219126E-2,-9.189131E-3,-1.8190295E-2,-5.0416205E-4,-6.594905E-3,6.896549E-4,6.8944483E-3,-2.5300044E-3,-8.050146E-3,5.3206435E-3,-3.3448602E-3,1.1482408E-2,2.964767E-3,-4.5070266E-3,3.7624273E-3,-4.783242E-3,3.7754192E-3,-7.2395295E-4,8.979616E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,-1,-1,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,87,89,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,93,95,-1,-1,-1,-1,97,99,-1,-1,-1,-1,-1,101,103,105,-1,-1,107,109,111,113,115,117,119,-1,-1,121,123,-1,125,-1,-1,-1,127,129,131,-1,133,-1,135,137,-1,-1,139,141,143,145,147,-1,-1,-1,149,151,153,-1,-1,-1,155,-1,157,-1,-1,159,161,163,165,167,169,171,173,175,-1,177,-1,-1,-1,179,181,-1,183,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,-1,-1,193,-1,-1,195,197,-1,-1,-1,-1,199,-1,201,203,205,207,-1,-1,-1,-1,-1,-1,209,211,213,-1,-1,-1,-1,-1,215,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0248436E1,9.610933E0,1.5529947E1,1.636116E0,3.1217172E0,1.5125486E1,4.5907068E-1,7.8635406E-1,2.5136828E-1,1.2065444E0,7.0870996E-1,4.03376E0,3.4851875E0,0E0,6.400156E-2,1.3962612E0,1.5842981E0,1.4124036E-1,8.539321E-2,4.8501372E-1,6.369829E-1,7.92261E-1,2.2308365E-1,1.0006618E0,2.0809536E0,2.6032143E0,1.153008E0,0E0,0E0,1.6039467E-1,2.2577612E0,0E0,8.554096E-1,2.412486E-2,0E0,0E0,2.7740464E-2,2.8878593E-1,2.2963624E-2,3.6514282E-2,0E0,4.9006712E-1,2.2158325E-1,5.360627E-2,2.5130102E-1,5.144441E-1,4.452467E-2,1.2347145E0,5.209882E-1,2.0773048E0,1.3811302E0,5.925355E-1,0E0,0E0,0E0,0E0,2.477827E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2238097E-1,9.2722684E-2,0E0,0E0,0E0,0E0,6.164307E-1,4.851762E-1,0E0,0E0,0E0,0E0,0E0,1.08491085E-1,1.5724778E-1,2.3821253E-2,0E0,0E0,5.687363E-1,7.068926E-1,1.9079934E-1,8.695078E-2,5.5708885E-1,1.9573855E-1,3.2915497E-1,0E0,0E0,2.3590374E-1,2.6494098E-1,0E0,1.10440016E-1,0E0,0E0,0E0,3.6726367E-1,1.9365522E-1,5.635351E-1,0E0,5.4458227E-2,0E0,1.8923283E-1,1.814073E-1,0E0,0E0,4.477644E-1,1.7608398E-1,2.995784E-1,5.0230813E-1,7.7501416E-2,0E0,0E0,0E0,7.739277E-1,8.8378906E-2,1.95642E-1,0E0,0E0,0E0,2.8758144E-1,0E0,5.007553E-2,0E0,0E0,7.618415E-2,2.8213874E-1,7.306486E-2,3.7193272E-2,1.6731554E-1,2.2652495E-1,2.4973874E-1,1.6830124E-2,4.835005E-2,0E0,1.1251593E-1,0E0,0E0,0E0,5.7705927E-1,8.019138E-2,0E0,2.1570726E-1,1.4412886E-1,4.808964E-1,0E0,0E0,0E0,1.5047073E-1,0E0,0E0,0E0,0E0,4.0616393E-2,0E0,0E0,0E0,3.398776E-2,0E0,0E0,2.0091042E-1,3.0746281E-2,0E0,0E0,0E0,0E0,9.035039E-2,0E0,1.0109317E-1,9.414825E-2,8.0435514E-2,2.852711E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.6636644E-1,2.4016619E-1,9.339764E-2,0E0,0E0,0E0,0E0,0E0,4.1713798E-1,3.5421288E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,56,56,63,63,64,64,69,69,70,70,76,76,77,77,78,78,81,81,82,82,83,83,84,84,85,85,86,86,87,87,90,90,91,91,93,93,97,97,98,98,99,99,101,101,103,103,104,104,107,107,108,108,109,109,110,110,111,111,115,115,116,116,117,117,121,121,123,123,126,126,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,136,136,140,140,141,141,143,143,144,144,145,145,149,149,154,154,158,158,161,161,162,162,167,167,169,169,170,170,171,171,172,172,179,179,180,180,181,181,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,-1,-1,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,88,90,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,94,96,-1,-1,-1,-1,98,100,-1,-1,-1,-1,-1,102,104,106,-1,-1,108,110,112,114,116,118,120,-1,-1,122,124,-1,126,-1,-1,-1,128,130,132,-1,134,-1,136,138,-1,-1,140,142,144,146,148,-1,-1,-1,150,152,154,-1,-1,-1,156,-1,158,-1,-1,160,162,164,166,168,170,172,174,176,-1,178,-1,-1,-1,180,182,-1,184,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,-1,-1,194,-1,-1,196,198,-1,-1,-1,-1,200,-1,202,204,206,208,-1,-1,-1,-1,-1,-1,210,212,214,-1,-1,-1,-1,-1,216,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,5.1987316E7,1.339646E6,4.97E2,1.0236667E3,4.4522205E6,4.6463413E0,8.0509944E5,4.325E0,2.331083E6,7E0,1.6007428E7,-3.2893173E-2,1E0,2.55E2,1.3E1,6.54E2,1.0526316E0,2.9652428E6,7.668863E6,2.829581E10,2.1111E4,1.0254199E10,9.932432E0,1.9180963E3,4.24038E8,-8.778473E-3,-1.9504804E-2,8.0833334E-1,1.1E1,-7.724392E-3,7.3E1,8.147158E4,-5.6628706E-3,1.678695E-3,4.03E2,2.0399521E2,1.3E1,1.1967312E3,-3.3945143E-2,1E0,2.2084616E2,5.405423E6,9.693913E3,1.5723623E0,9.244374E6,1E1,4E1,8E0,1.4497429E7,1.2841364E3,-3.648233E-3,-2.8487349E-2,-2.0430839E-2,1.521193E-2,1.3439851E7,-5.3707164E-2,-1.846122E-2,-1.6118526E-2,-1.0319226E-2,-2.2626396E-3,-8.056722E-3,2.5882354E0,6.920598E7,-2.1121674E-4,4.1767093E-3,-9.395055E-3,-1.6742427E-2,3.493E3,3.2E1,-2.2507835E-2,-7.288339E-3,4.5214454E-3,1.0523641E-2,-6.8514915E-3,1.273801E6,1.02E2,8.328548E6,2.4119057E-2,1.30020855E-2,8E0,2.2783158E5,1.8285715E1,6.001692E-7,1.7942307E8,4.9735293E0,1.2816234E7,1.730918E-2,1.009027E-3,3.8181802E6,3.1158695E6,-2.2283504E-2,1.1967312E3,-3.8428288E-3,-6.0890834E-3,1.192751E-3,1E0,3.074398E0,4.39776E6,9.600464E-3,2.0487332E-3,7.501921E-3,1.357E3,1.9539816E7,-1.2547317E-3,3.0870188E-3,9.63381E5,1.94261E5,3.63E2,2.277593E5,2.0312773E8,1.451697E-3,-2.3611257E-2,-9.586669E-3,5.526E3,1.7103828E7,2.9697892E-1,1.6192159E-3,5.0096635E-2,2.3613272E-2,2.337765E3,2.6048169E-2,8E0,-4.0383595E-3,-1.7529752E-2,3.2E1,8.15736E7,1E0,7.579E3,9.61E2,3.4919708E2,1.3E1,1.411E3,2.5384614E0,6.5664626E-3,5.55E2,1.8692205E-3,1.3185545E-2,2.1174159E-2,1.977157E7,6.4356956E0,7.360973E-3,9.0927E4,6.808571E2,4.5960168E2,-1.0171062E-2,-1.1788716E-2,-3.110122E-3,1.25E0,1.9669945E-3,3.607859E-2,1.8216664E-2,4.285529E-3,8.255237E1,7.658027E-3,1.8598875E-2,-1.1427673E-2,2.8998098E1,-6.8128845E-3,-1.3173806E-2,3.388854E2,8.62E2,-1.3725284E-3,-1.0155769E-2,1.4527359E-3,-3.980914E-3,2.866353E2,-1.736542E-3,9.455644E5,1E0,9.31E2,1.5E1,5.71508E-3,1.2670245E-3,-3.16025E-3,2.1106717E-3,2.3716556E-2,1.1725698E-2,2.3600838E6,5.2409735E-7,2.4796897E3,-6.7882463E-3,-6.691366E-3,6.303831E-3,-1.4504919E-2,-4.063392E-3,7.026624E7,4.8709216E5,1.269861E-2,2.9399319E-2,1.801399E-2,8.278784E-3,-1.9761374E-2,-9.464315E-3,-6.204706E-4,5.1875E-3,4.5663933E-3,1.1462093E-2,-6.2363097E-3,-1.8219126E-2,-9.189131E-3,-1.8190295E-2,-5.0416205E-4,-6.594905E-3,6.896549E-4,6.8944483E-3,-2.5300044E-3,-8.050146E-3,5.3206435E-3,-3.3448602E-3,1.1482408E-2,2.964767E-3,-4.5070266E-3,3.7624273E-3,-4.783242E-3,3.7754192E-3,-7.2395295E-4,8.979616E-3],"split_indices":[2,43,60,9,2,67,62,68,60,69,43,3,66,0,15,2,0,2,71,43,5,46,9,46,69,67,47,0,0,68,10,0,0,43,0,0,2,4,3,48,0,26,67,43,62,68,60,8,3,32,62,4,0,0,0,0,60,0,0,0,0,0,0,71,7,0,0,0,0,44,3,0,0,0,0,0,9,0,9,0,0,32,43,69,52,12,68,9,0,0,43,60,0,48,0,0,0,19,71,9,0,53,0,2,12,0,0,60,1,0,48,7,0,0,0,2,62,53,0,0,0,67,0,8,0,0,10,7,8,44,2,70,8,44,69,0,44,0,0,0,5,69,0,1,4,67,0,0,0,68,0,0,0,0,73,0,0,0,73,0,0,4,44,0,0,0,0,67,0,62,8,2,3,0,0,0,0,0,0,62,52,48,0,0,0,0,0,7,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.014E3,5.21E2,4.93E2,1.93E2,3.28E2,4.65E2,2.8E1,1.46E2,4.7E1,9.1E1,2.37E2,3.23E2,1.42E2,9E0,1.9E1,1.32E2,1.4E1,3.4E1,1.3E1,6.3E1,2.8E1,1.86E2,5.1E1,6.7E1,2.56E2,9.4E1,4.8E1,4E0,1.5E1,7.9E1,5.3E1,5E0,9E0,2.5E1,9E0,4E0,9E0,5.4E1,9E0,2.3E1,5E0,1.76E2,1E1,1.7E1,3.4E1,4E1,2.7E1,2.31E2,2.5E1,6.6E1,2.8E1,4.1E1,7E0,1.5E1,6.4E1,5E0,4.8E1,5E0,4E0,1.1E1,1.4E1,4E0,5E0,3.8E1,1.6E1,5E0,4E0,7E0,1.6E1,9.1E1,8.5E1,4E0,6E0,1E1,7E0,6E0,2.8E1,3.1E1,9E0,2.2E1,5E0,1.08E2,1.23E2,1.7E1,8E0,4.3E1,2.3E1,2.2E1,6E0,5E0,3.6E1,3.8E1,1E1,3.3E1,5E0,1.1E1,5E0,7E1,2.1E1,8.1E1,4E0,2.1E1,7E0,1.9E1,1.2E1,5E0,4E0,8.9E1,1.9E1,2.2E1,1.01E2,1E1,7E0,4E0,4E0,1.7E1,2.6E1,1.9E1,4E0,1.8E1,4E0,2.8E1,8E0,3.1E1,7E0,9E0,2.4E1,6.2E1,8E0,9E0,1.2E1,5.2E1,2.9E1,8E0,1.3E1,5E0,1.4E1,6E0,6E0,6E0,8.3E1,1.4E1,5E0,9E0,1.3E1,9.6E1,5E0,5E0,5E0,1.3E1,4E0,2.2E1,4E0,5E0,1.4E1,1E1,1.8E1,1.7E1,1.4E1,1.3E1,1.1E1,5E1,1.2E1,4E0,4E0,5E0,4E0,8E0,4E0,2.7E1,2.5E1,1.9E1,1E1,4E0,4E0,7E0,6E0,7E0,7E0,1.7E1,6.6E1,1E1,4E0,4E0,5E0,7E0,6E0,4.9E1,4.7E1,4E0,9E0,1E1,4E0,1E1,4E0,2.8E1,2.2E1,4E0,8E0,4E0,4E0,2.3E1,4E0,8E0,1.7E1,1.3E1,6E0,5E0,5E0,8E0,9E0,5.7E1,9E0,4E0,6E0,1.9E1,3E1,1E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[3.2505463E-3,-1.1654815E-1,3.6172676E-1,-3.590055E-1,-1.0095838E-2,-4.5593828E-1,4.0589243E-1,-5.265351E-1,-2.8014255E-1,-1.14962645E-1,9.217608E-2,-1.11706415E-2,-2.3904162E-2,1.0189188E0,3.708496E-1,-7.593408E-1,-4.1117904E-1,-4.1346416E-1,-2.1395218E-1,-1.6278714E-1,4.9649157E-2,1.6716285E-1,2.4673175E-2,3.0435188E-2,5.6986243E-2,2.9117757E-1,6.2600684E-1,-1.5495122E-2,-8.820214E-1,-2.0719653E-2,1.6941458E-3,-6.6070645E-3,-4.3224E-1,1.6305804E-2,-2.5471857E-1,-2.3922963E-1,-5.7356335E-2,7.908246E-2,-6.7375004E-2,9.513606E-2,2.5741884E-1,9.834508E-3,6.5021114E-3,9.256604E-2,3.666409E-1,7.5896996E-1,4.5508462E-1,-5.306592E-2,-3.192911E-2,-4.5905703E-1,-9.328848E-3,-3.917255E-1,-1.816627E-1,-2.0540728E-1,-4.8547304E-1,-2.5786358E-1,-2.0155588E-2,5.466478E-2,9.260237E-3,1.2802752E-3,-1.0397415E-2,1.234197E-1,-9.112795E-3,2.9675186E-1,-1.1783756E-3,-1.4289995E-1,3.198535E-2,1.701426E-1,-5.1127076E-2,4.4959348E-1,2.244506E-1,4.4486746E-2,2.9615002E-2,5.400847E-1,2.5657594E-1,-5.59759E-1,-3.9877042E-1,-2.676839E-1,-2.6219243E-2,-2.08072E-1,4.1356627E-3,-3.022815E-1,-1.5142633E-1,-1.0892647E-2,-3.599395E-2,-2.4272904E-3,-1.691322E-2,1.18870055E-2,-3.8883276E-2,-3.799378E-3,7.484267E-2,-6.754106E-2,1.4957097E-1,2.1982755E-1,1.9160535E-2,-1.9103704E-1,1.8349694E-3,4.8519544E-2,-9.21015E-3,1.568868E-2,1.13634735E-1,-1.312766E-1,3.487478E-3,4.7322676E-1,-6.343543E-4,3.0808827E-1,-1.85981E-2,2.7130585E-2,1.3859407E-2,2.9573445E-3,1.7291486E-2,-3.0396149E-2,-1.474656E-2,-1.4725634E-2,-2.3049546E-2,-3.7483564E-1,-1.1671515E-1,-2.5105888E-1,-9.0784915E-2,-1.601454E-1,-3.6172333E-1,-1.152518E-1,-1.4908189E-2,8.4625065E-2,-8.084283E-2,-4.8527718E-3,1.16478406E-1,-6.4006546E-3,7.840552E-4,1.5481961E-2,1.2702042E-1,2.6843554E-1,6.038011E-2,-2.4007928E-1,-1.6228916E-3,-3.0662296E-3,1.591481E-1,1.8448913E-1,6.110338E-2,-9.1805327E-4,-9.60999E-3,4.2353433E-1,6.0785896E-1,2.3549022E-1,3.0307941E-2,5.5388524E-3,-8.0485465E-3,-9.137015E-3,-1.9937396E-2,-1.2191463E-2,2.348425E-4,-4.0918523E-3,-2.6411435E-1,-3.68976E-2,-1.082397E-2,-9.446531E-3,-2.2325153E-3,-4.3337488E-1,-2.4201532E-1,1.13993045E-2,-1.373914E-1,-2.925816E-4,1.09302774E-1,2.5645468E-2,-1.1783019E-1,3.9501796E-3,-2.2889606E-3,7.961968E-3,2.7847271E-3,7.358406E-2,1.816897E-1,1.9170721E-1,1.7719213E-2,5.412028E-3,-3.9092152E-4,-4.9004625E-3,-1.3574363E-2,3.6472507E-2,-1.4243434E-1,8.738414E-2,1.523735E-2,1.0475402E-2,4.085527E-3,-8.1262493E-4,1.0562591E-1,2.566952E-2,3.455461E-1,3.1130448E-2,1.3322173E-2,1.3223937E-3,2.8040037E-1,-1.3813071E-2,-7.877615E-3,-4.636489E-3,1.4402999E-3,-2.222312E-2,-1.117366E-2,-3.419595E-3,-1.4634069E-2,-3.15442E-3,3.506412E-3,-7.409692E-3,-1.1445563E-3,2.3210004E-3,7.836609E-3,-4.517476E-3,3.6690293E-3,1.2596359E-3,-6.8607777E-3,-6.2639493E-4,6.9613336E-3,1.0762418E-2,5.0152917E-3,4.3124678E-3,1.0390745E-2,-2.8561084E-3,2.8797733E-3,-3.5921042E-4,-1.0899195E-2,1.3185804E-3,1.1227438E-2,7.2503136E-3,5.8515766E-4,1.7867856E-2,4.1635795E-3,1.42089855E-2,8.1980685E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,-1,75,-1,77,79,81,83,85,87,89,-1,-1,-1,91,-1,93,-1,95,97,99,101,103,105,-1,-1,107,109,111,113,115,-1,117,-1,119,121,-1,-1,-1,-1,-1,123,-1,125,127,129,131,-1,133,-1,135,-1,-1,137,139,-1,141,-1,143,145,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,153,155,157,159,-1,161,163,165,167,-1,-1,-1,169,171,173,175,-1,177,179,181,183,-1,-1,185,187,189,-1,-1,-1,-1,-1,-1,-1,-1,191,193,-1,-1,-1,195,197,199,201,-1,203,205,207,-1,-1,-1,-1,209,211,213,-1,-1,-1,-1,-1,215,217,219,-1,-1,-1,-1,221,-1,223,-1,-1,-1,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5263226E1,2.038639E1,9.7188225E0,3.03656E0,5.909308E0,5.9205532E-2,5.0141563E0,1.728796E0,1.3764238E0,2.1493962E0,1.4081249E0,0E0,0E0,1.9198227E-1,4.6763306E0,9.5239735E-1,9.0197945E-1,2.123928E-1,2.7013812E0,1.6868105E0,2.188814E-1,8.336222E-1,5.0474364E-1,0E0,0E0,2.7238321E0,9.272804E-1,0E0,1.223011E-1,0E0,0E0,0E0,2.1611881E-1,0E0,9.8981E-1,9.1636133E-1,6.661143E-1,1.3720903E-1,2.011526E-1,6.445767E-1,6.5301085E-1,0E0,5.2235645E-1,5.883543E-1,1.4678326E0,1.04400635E-1,3.1700563E-1,0E0,0E0,4.390812E-2,0E0,6.3312626E-1,5.27107E-1,5.2422667E-1,8.427727E-1,2.9187888E-1,4.101042E-1,1.24939755E-1,0E0,0E0,0E0,3.54679E-1,0E0,3.717084E-1,0E0,1.8340248E-1,4.5720655E-1,2.869624E-1,2.0272456E-1,9.153366E-1,1.0424592E0,0E0,0E0,2.767706E-2,2.0269978E-1,1.20646E-1,8.942413E-2,3.283118E-1,0E0,3.0492496E-1,0E0,2.6312733E-1,4.088887E-1,0E0,0E0,0E0,0E0,0E0,3.8515317E-1,0E0,1.2665604E-1,5.6357764E-2,2.1266389E-1,2.3955548E-1,0E0,1.139043E-1,0E0,6.282425E-1,0E0,0E0,8.922315E-2,9.845179E-2,0E0,2.710762E-1,0E0,8.053863E-1,3.226034E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5203686E-2,1.985236E-1,7.631159E-2,1.3828415E-1,5.455941E-2,1.1812568E-1,1.7261052E-1,0E0,4.3340355E-2,2.1965146E-1,6.077222E-2,6.810048E-2,0E0,0E0,0E0,1.5022486E-1,1.4726806E-1,3.6161963E-2,5.5258095E-2,0E0,4.2398834E-1,3.9986098E-1,2.4596423E-2,6.141933E-2,0E0,0E0,4.3641567E-1,2.061801E-1,2.9855704E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.9386225E-2,6.79294E-2,0E0,0E0,0E0,2.0558596E-2,1.2723851E-1,5.6573566E-2,1.306861E-1,0E0,4.4922665E-2,1.0732127E-1,1.865294E-1,0E0,0E0,0E0,0E0,2.0359685E-1,7.5941324E-2,3.4529686E-2,0E0,0E0,0E0,0E0,0E0,1.5637608E-1,2.121692E-1,2.3117812E-1,0E0,0E0,0E0,0E0,5.0480276E-2,0E0,3.967309E-1,0E0,0E0,0E0,2.0917177E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,61,61,63,63,65,65,66,66,67,67,68,68,69,69,70,70,73,73,74,74,75,75,76,76,77,77,79,79,81,81,82,82,88,88,90,90,91,91,92,92,93,93,95,95,97,97,100,100,101,101,103,103,105,105,106,106,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,130,130,131,131,132,132,133,133,135,135,136,136,137,137,138,138,141,141,142,142,143,143,152,152,153,153,157,157,158,158,159,159,160,160,162,162,163,163,164,164,169,169,170,170,171,171,177,177,178,178,179,179,184,184,186,186,190,190],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,-1,76,-1,78,80,82,84,86,88,90,-1,-1,-1,92,-1,94,-1,96,98,100,102,104,106,-1,-1,108,110,112,114,116,-1,118,-1,120,122,-1,-1,-1,-1,-1,124,-1,126,128,130,132,-1,134,-1,136,-1,-1,138,140,-1,142,-1,144,146,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,154,156,158,160,-1,162,164,166,168,-1,-1,-1,170,172,174,176,-1,178,180,182,184,-1,-1,186,188,190,-1,-1,-1,-1,-1,-1,-1,-1,192,194,-1,-1,-1,196,198,200,202,-1,204,206,208,-1,-1,-1,-1,210,212,214,-1,-1,-1,-1,-1,216,218,220,-1,-1,-1,-1,222,-1,224,-1,-1,-1,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8988005E5,1E0,3.434405E7,9.24E2,3.714144E8,2.4481E4,2E0,2.71E2,2.331083E6,3.7105262E0,-1.11706415E-2,-2.3904162E-2,6.23694E5,4.391553E6,1.5E1,3.7066666E2,2.416E3,1.2E1,2.61E2,1.8E1,4.265829E2,4.0869565E0,3.0435188E-2,5.6986243E-2,7.601393E2,1.6386554E0,-1.5495122E-2,3.6255838E2,-2.0719653E-2,1.6941458E-3,-6.6070645E-3,3.328629E6,1.6305804E-2,2.4360857E4,2.1E1,1E0,6.9307615E6,1.6923077E0,4.5158855E6,7.978062E6,9.834508E-3,2.1924414E2,1.124641E0,6.9E1,2.5959E4,8.81059E5,-5.306592E-2,-3.192911E-2,3.89E2,-9.328848E-3,5.17259E8,8.909296E4,1.5933333E2,2.85E2,3.1275E4,2.3112903E2,1.3020051E-5,9.260237E-3,1.2802752E-3,-1.0397415E-2,4.0449125E5,-9.112795E-3,1.0029973E8,-1.1783756E-3,9.48E0,2.5243637E-6,6E0,3.149243E8,3.2E1,5.712652E8,4.4486746E-2,2.9615002E-2,1.647E4,1.3E1,2.7546012E0,9.183673E-1,3.183206E-8,-2.6219243E-2,7.997723E6,4.1356627E-3,3.1E1,2E0,-1.0892647E-2,-3.599395E-2,-2.4272904E-3,-1.691322E-2,1.18870055E-2,1.3364486E0,-3.799378E-3,9.22E2,1.4327235E7,1.3600995E2,5.4983668E7,1.9160535E-2,2.008E3,1.8349694E-3,2.008E3,-9.21015E-3,1.568868E-2,1E1,1.2E1,3.487478E-3,1.711E4,-6.343543E-4,4.907764E3,6.619098E8,2.7130585E-2,1.3859407E-2,2.9573445E-3,1.7291486E-2,-3.0396149E-2,-1.474656E-2,-1.4725634E-2,-2.3049546E-2,2.56E2,4.9E1,7.090909E0,5.4444447E0,1E0,1.329762E2,6.055E3,-1.4908189E-2,2.2007043E0,3.373913E0,3.863E3,9.166947E6,-6.4006546E-3,7.840552E-4,1.5481961E-2,2.378914E6,1.763E3,1.1090909E1,2.083138E6,-1.6228916E-3,1.3254E4,6.875854E2,3.744E3,1.5E1,-9.1805327E-4,-9.60999E-3,1.798E3,9.2732E4,1.7068776E10,3.0307941E-2,5.5388524E-3,-8.0485465E-3,-9.137015E-3,-1.9937396E-2,-1.2191463E-2,2.348425E-4,-4.0918523E-3,8.01E2,2.132956E8,-1.082397E-2,-9.446531E-3,-2.2325153E-3,4.7563504E7,1.5277778E0,4.0906172E5,1.03860024E8,-2.925816E-4,1.3717398E4,6.07E2,2.57E2,3.9501796E-3,-2.2889606E-3,7.961968E-3,2.7847271E-3,2.2461708E6,9E0,1.2151898E0,1.7719213E-2,5.412028E-3,-3.9092152E-4,-4.9004625E-3,-1.3574363E-2,1.14E3,1.9E1,1.836095E6,1.523735E-2,1.0475402E-2,4.085527E-3,-8.1262493E-4,7.362415E1,2.566952E-2,2.7578741E1,3.1130448E-2,1.3322173E-2,1.3223937E-3,2.5345264E7,-1.3813071E-2,-7.877615E-3,-4.636489E-3,1.4402999E-3,-2.222312E-2,-1.117366E-2,-3.419595E-3,-1.4634069E-2,-3.15442E-3,3.506412E-3,-7.409692E-3,-1.1445563E-3,2.3210004E-3,7.836609E-3,-4.517476E-3,3.6690293E-3,1.2596359E-3,-6.8607777E-3,-6.2639493E-4,6.9613336E-3,1.0762418E-2,5.0152917E-3,4.3124678E-3,1.0390745E-2,-2.8561084E-3,2.8797733E-3,-3.5921042E-4,-1.0899195E-2,1.3185804E-3,1.1227438E-2,7.2503136E-3,5.8515766E-4,1.7867856E-2,4.1635795E-3,1.42089855E-2,8.1980685E-3],"split_indices":[2,43,17,7,2,7,12,32,2,43,69,0,0,1,43,3,4,9,33,70,3,67,73,0,0,67,69,0,4,0,0,0,9,0,43,3,13,43,68,62,62,0,67,53,8,9,9,0,0,1,0,46,48,4,2,1,67,53,0,0,0,43,0,5,0,71,52,8,7,3,47,0,0,44,8,68,71,52,0,9,0,10,6,0,0,0,0,0,68,0,44,12,67,5,0,2,0,2,0,0,3,3,0,44,0,4,7,0,0,0,0,0,0,0,0,10,0,73,69,8,4,12,0,69,69,12,60,0,0,0,9,2,73,12,0,44,67,10,3,0,0,44,44,46,0,0,0,0,0,0,0,0,2,46,0,0,0,7,68,43,7,0,48,2,44,0,0,0,0,43,8,68,0,0,0,0,0,2,8,43,0,0,0,0,73,0,71,0,0,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.052E3,7.89E2,2.63E2,2.4E2,5.49E2,1.3E1,2.5E2,7.5E1,1.65E2,2.71E2,2.78E2,4E0,9E0,1.2E1,2.38E2,2.3E1,5.2E1,5.3E1,1.12E2,2.1E2,6.1E1,1.31E2,1.47E2,6E0,6E0,1.83E2,5.5E1,6E0,1.7E1,4.8E1,4E0,4E0,4.9E1,7E0,1.05E2,1.21E2,8.9E1,4.9E1,1.2E1,7.4E1,5.7E1,1.2E1,1.35E2,5.1E1,1.32E2,2.9E1,2.6E1,5E0,1.2E1,4.3E1,6E0,3.5E1,7E1,1.08E2,1.3E1,1.3E1,7.6E1,4.2E1,7E0,8E0,4E0,6.8E1,6E0,5E1,7E0,1.9E1,1.16E2,3.3E1,1.8E1,8.2E1,5E1,8E0,2.1E1,1.7E1,9E0,1.3E1,3E1,2.2E1,1.3E1,6.4E1,6E0,3.7E1,7.1E1,8E0,5E0,5E0,8E0,4E0,7.2E1,5E0,3.7E1,8E0,6E1,3.2E1,1.8E1,1.5E1,4E0,1.09E2,7E0,7E0,2.6E1,1.1E1,7E0,7.8E1,4E0,3.7E1,1.3E1,1.3E1,4E0,4E0,5E0,8E0,5E0,1.9E1,1.1E1,1.2E1,1E1,4.6E1,1.8E1,1.2E1,2.5E1,6E1,1.1E1,1.8E1,5.4E1,1.3E1,2.4E1,4E0,4E0,5E0,5.5E1,2.4E1,8E0,1.1E1,4E0,7.5E1,3.4E1,1E1,1.6E1,5E0,6E0,6E1,1.8E1,3.2E1,5E0,7E0,6E0,4E0,8E0,4E0,6E0,4E0,4.2E1,1.4E1,4E0,8E0,4E0,1.4E1,1.1E1,9E0,5.1E1,4E0,1.4E1,1.4E1,4E1,4E0,9E0,1.1E1,1.3E1,2.9E1,2.6E1,1.6E1,8E0,4E0,4E0,4E0,7E0,5.9E1,1.6E1,2.5E1,9E0,6E0,4E0,6E0,1E1,2E1,4E1,1.4E1,4E0,6E0,2.6E1,2.9E1,1.3E1,7E0,7E0,1E1,4E0,4E0,7E0,4E0,5E0,4.2E1,9E0,8E0,6E0,4E0,1E1,7E0,3.3E1,1.4E1,1.5E1,1.4E1,1.2E1,5E0,1.1E1,1.2E1,4.7E1,7E0,9E0,1.9E1,6E0,6E0,4E0,3.4E1,6E0,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[9.038484E-3,-1.2883458E-1,2.5897896E-1,-3.4229952E-1,-2.2767497E-2,-4.104476E-1,2.9649198E-1,-2.956752E-1,-5.1426053E-1,-9.994397E-2,8.2556054E-2,-2.1128622E-1,-2.4282899E-2,1.7741084E-1,4.522518E-1,-4.2905805E-1,-1.8625176E-1,-1.9732893E-1,-5.968157E-1,-8.501878E-2,-3.3037852E-2,5.9052687E-2,3.5282513E-1,-2.869962E-3,-1.4700239E-2,3.4714615E-1,1.0899194E-1,7.490896E-1,3.6288962E-1,-4.9013755E-1,-3.4248704E-1,1.7432038E-2,-2.2666878E-1,-4.1610957E-3,-1.1353574E-2,-6.827386E-1,-1.2158992E-2,-1.2904565E-1,5.340388E-2,1.0425414E-1,-3.7144285E-2,6.8349433E-3,1.9120734E-2,2.4442317E-1,4.448084E-1,1.2789638E-1,-1.6036898E-2,4.6931082E-1,9.2146915E-1,5.40026E-1,2.4629833E-1,-1.579187E-2,-5.350738E-1,-5.937914E-3,-3.6521524E-1,-1.9714724E-1,-2.5926823E-2,-2.3892788E-2,-8.3399636E-1,-2.2614989E-1,-8.668468E-2,7.3612556E-2,-8.8562425E-3,4.4637952E-2,1.4767136E-1,-1.4357184E-1,8.642603E-2,1.3238703E-1,1.7590212E-2,4.8416483E-1,7.712279E-3,6.384352E-2,2.3062661E-1,3.0198673E-2,7.6339548E-3,2.4037002E-2,4.5702185E-2,6.253453E-1,3.0034307E-1,2.928491E-1,2.9202359E-2,-2.5938027E-2,-1.2954414E-2,-1.4005636E-2,-2.1072648E-2,-1.6577858E-1,-1.676818E-2,-4.501945E-2,-2.3455385E-2,-1.8234089E-1,-3.7888932E-1,-2.1259548E-2,-1.5575287E-1,-4.1580193E-2,1.1476305E-1,-6.599024E-3,6.762143E-2,1.6652918E-1,-5.867957E-3,-9.1943525E-2,-1.8186273E-2,1.4028837E-1,-5.79676E-3,1.6521081E-1,6.4692704E-4,1.2622806E-2,2.426811E-2,2.0737039E-1,2.8578728E-2,2.7084735E-1,-9.686181E-3,1.3115358E-2,3.0613964E-2,5.8611194E-3,1.9153757E-2,2.5180168E-2,2.3151086E-1,1.2744788E-2,-9.7050145E-2,-1.9552541E-1,2.5068957E-2,-2.1722208E-1,-4.8436772E-2,-8.439384E-3,-2.5162708E-2,-4.962803E-2,5.6526614E-3,1.7392533E-3,-1.7572029E-1,-9.547308E-2,1.9103154E-3,8.470641E-2,1.0887848E-2,-3.8268674E-2,9.727466E-2,1.2087663E-2,1.400573E-1,-1.6351989E-2,-1.6096766E-1,3.9882407E-2,1.1863329E-2,3.417695E-3,1.0366536E-2,2.832335E-1,-1.4482354E-3,1.0993777E-2,1.0036747E-2,3.687134E-1,1.5354046E-1,1.7514554E-1,2.0302527E-2,-9.418493E-3,3.7603674E-4,-2.6285863E-1,-1.3030097E-1,-9.0386387E-4,3.813366E-3,-2.3222806E-1,-4.6654847E-3,-4.7475668E-3,6.450564E-4,-9.573933E-2,4.0566247E-2,-2.4257724E-1,-1.21034555E-1,-1.2344385E-3,-6.3225767E-3,7.3939515E-3,5.7465035E-2,1.8067614E-3,-4.0687923E-3,4.295742E-2,1.4647219E-1,1.6614866E-1,-4.251073E-3,-4.428699E-3,2.5530807E-3,-4.714015E-3,-1.0280934E-2,8.606419E-2,-2.3663742E-3,1.6952554E-2,8.5489005E-3,-5.757856E-2,3.9939586E-2,2.2588288E-2,2.307433E-1,2.1240956E-1,4.2739182E-4,2.3145424E-1,-8.865486E-3,-5.257754E-3,-1.3586165E-2,-7.855505E-3,-6.4492127E-4,-7.4248053E-3,-1.2885934E-2,-7.843881E-4,-6.574897E-3,-3.31523E-3,5.322587E-3,-1.2294298E-2,-4.1806474E-3,-1.6670658E-3,-7.3210024E-3,4.6119912E-4,4.2712283E-3,-2.9135854E-3,3.9420547E-3,5.457052E-4,8.389474E-3,1.0541899E-2,4.8453845E-3,6.2551233E-3,9.018273E-4,-8.8055115E-3,-2.4561705E-5,2.3880196E-4,7.7648554E-3,5.467038E-3,1.5072199E-2,1.2010223E-2,2.1380277E-3,6.979607E-3,1.672586E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,57,-1,59,61,63,65,-1,-1,67,69,71,-1,73,75,77,79,-1,81,-1,83,85,-1,-1,87,89,91,93,-1,95,97,99,101,103,-1,105,-1,107,109,-1,-1,-1,-1,111,113,115,117,-1,-1,-1,-1,119,-1,-1,-1,121,123,125,127,129,131,-1,133,135,-1,137,-1,139,-1,141,-1,-1,-1,143,145,147,-1,-1,-1,-1,-1,-1,149,-1,151,153,155,157,159,-1,-1,161,-1,-1,163,165,-1,167,-1,169,171,-1,173,175,177,179,-1,-1,-1,181,-1,183,-1,185,187,189,-1,-1,-1,191,193,-1,-1,195,-1,-1,-1,197,199,201,203,-1,-1,-1,205,-1,-1,207,209,211,-1,-1,-1,-1,-1,213,-1,-1,-1,215,217,-1,219,221,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5768444E1,1.513944E1,9.3881E0,1.6278458E0,3.6498191E0,3.4768152E-1,6.411585E0,2.503439E0,1.0857916E0,2.3172398E0,1.1860889E0,1.2375435E-1,0E0,2.2847843E0,3.7336025E0,2.4986362E-1,2.3405576E0,3.49994E-2,8.522997E-1,1.5548478E0,0E0,7.6916337E-1,1.2639368E-1,0E0,0E0,4.631114E-1,1.2860588E0,1.2073002E0,2.303996E0,1.1034775E-1,1.2812328E-1,0E0,8.0781317E-1,0E0,0E0,2.698984E-1,0E0,7.742882E-1,3.2120812E-1,3.0401254E-1,7.6203173E-1,0E0,0E0,4.0779436E-1,2.2538614E-1,8.999188E-1,0E0,7.328346E-1,2.910614E-2,7.52862E-1,7.291846E-1,0E0,5.7444572E-2,0E0,4.22554E-2,3.9233136E-1,0E0,0E0,1.0111141E-1,3.1901193E-1,6.112449E-1,2.797057E-1,0E0,2.3372027E-1,3.7857032E-1,3.6405146E-1,3.257645E-1,6.7631155E-2,0E0,6.469536E-2,0E0,4.344194E-1,9.9534893E-1,0E0,0E0,0E0,0E0,2.322731E-1,2.2487509E-1,8.019414E-1,4.6276113E-1,0E0,0E0,0E0,0E0,4.3068504E-1,0E0,0E0,0E0,2.0900309E-1,2.9082143E-1,2.9533422E-1,2.6148736E-1,7.545705E-2,1.3630474E-1,0E0,1.5130182E-1,1.2701488E-1,0E0,1.3824415E-1,0E0,2.470968E-1,0E0,5.898586E-2,0E0,0E0,0E0,3.1674695E-1,2.3317374E-1,5.0592566E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.090804E-1,0E0,1.1402033E-1,2.512145E-1,2.9832404E-2,4.1084528E-2,3.872416E-2,0E0,0E0,2.5342226E-1,0E0,0E0,1.9000721E-1,2.4794668E-2,0E0,6.468722E-2,0E0,4.5833245E-2,9.258211E-2,0E0,3.3620977E-1,8.501391E-2,2.808845E-2,5.910361E-2,0E0,0E0,0E0,3.8953424E-2,0E0,1.3330185E-1,0E0,3.1843972E-1,2.0189536E-1,8.917923E-1,0E0,0E0,0E0,1.10788345E-1,1.5942556E-1,0E0,0E0,6.203127E-2,0E0,0E0,0E0,1.4432275E-1,1.8434021E-1,6.899512E-2,1.05576575E-1,0E0,0E0,0E0,4.5593224E-2,0E0,0E0,9.07336E-2,8.7715864E-2,1.585064E-1,0E0,0E0,0E0,0E0,0E0,2.6900962E-2,0E0,0E0,0E0,1.5166724E-1,2.0853072E-1,0E0,1.1168337E-1,1.2228173E-1,0E0,3.2316256E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,32,32,35,35,37,37,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,54,54,55,55,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,71,71,72,72,77,77,78,78,79,79,80,80,85,85,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,99,99,101,101,103,103,107,107,108,108,109,109,116,116,118,118,119,119,120,120,121,121,122,122,125,125,128,128,129,129,131,131,133,133,134,134,136,136,137,137,138,138,139,139,143,143,145,145,147,147,148,148,149,149,153,153,154,154,157,157,161,161,162,162,163,163,164,164,168,168,171,171,172,172,173,173,179,179,183,183,184,184,186,186,187,187,189,189],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,58,-1,60,62,64,66,-1,-1,68,70,72,-1,74,76,78,80,-1,82,-1,84,86,-1,-1,88,90,92,94,-1,96,98,100,102,104,-1,106,-1,108,110,-1,-1,-1,-1,112,114,116,118,-1,-1,-1,-1,120,-1,-1,-1,122,124,126,128,130,132,-1,134,136,-1,138,-1,140,-1,142,-1,-1,-1,144,146,148,-1,-1,-1,-1,-1,-1,150,-1,152,154,156,158,160,-1,-1,162,-1,-1,164,166,-1,168,-1,170,172,-1,174,176,178,180,-1,-1,-1,182,-1,184,-1,186,188,190,-1,-1,-1,192,194,-1,-1,196,-1,-1,-1,198,200,202,204,-1,-1,-1,206,-1,-1,208,210,212,-1,-1,-1,-1,-1,214,-1,-1,-1,216,218,-1,220,222,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,1E0,2.1E1,9.31E2,1.91E2,1.1459359E3,2.71E2,1E0,1.1290322E1,5.197839E6,1E0,-2.4282899E-2,8E0,3.4583332E1,3.89E2,2E0,4.0869565E0,6.03E2,2.331083E6,-3.3037852E-2,5.194E3,5.45901E6,-2.869962E-3,-1.4700239E-2,5.691698E2,1.0428572E1,1.562752E3,2.2842104E0,2.416E3,4.456432E6,1.7432038E-2,3.2856784E7,-4.1610957E-3,-1.1353574E-2,4.6463413E0,-1.2158992E-2,1.8149019E2,2.7E1,2.378914E6,7.9016544E5,6.8349433E-3,1.9120734E-2,1E0,1.4553613E1,6.9664386E2,-1.6036898E-2,1.934E3,9E0,2.395631E7,2.3876712E7,-1.579187E-2,7.338571E2,-5.937914E-3,1.5272727E0,3.78E2,-2.5926823E-2,-2.3892788E-2,1.746E3,2.4E0,1.7755102E0,3.0479442E9,-8.8562425E-3,5.4603375E5,1E0,1.28636E7,1.2906634E7,3.222E3,1.7590212E-2,1.000501E6,7.712279E-3,2.992648E-2,1.6063418E7,3.0198673E-2,7.6339548E-3,2.4037002E-2,4.5702185E-2,1.4992306E8,5.854E3,3.3382E4,1.2176E4,-2.5938027E-2,-1.2954414E-2,-1.4005636E-2,-2.1072648E-2,1.3022917E8,-1.676818E-2,-4.501945E-2,-2.3455385E-2,5.41E2,4.9963706E5,3.961275E2,1.6580646E2,2.8080197E2,6.7723384E7,-6.599024E-3,1E0,1.880118E-3,-5.867957E-3,3.78E2,-1.8186273E-2,5.4007E7,-5.79676E-3,4.23025E2,6.4692704E-4,1.2622806E-2,2.426811E-2,1.302273E6,6.0450394E8,3.8396227E0,-9.686181E-3,1.3115358E-2,3.0613964E-2,5.8611194E-3,1.9153757E-2,2.5180168E-2,4.435876E2,1.2744788E-2,2.9925186E-2,8.147158E4,7.8918E4,1.3337367E7,8.8790035E-1,-8.439384E-3,-2.5162708E-2,4.39776E6,5.6526614E-3,1.7392533E-3,2.3238889E2,5.2846E4,1.9103154E-3,8.5E1,1.0887848E-2,1.821E3,6.077143E5,1.2087663E-2,1E0,1.445E3,1.1860938E8,1.3062992E1,1.1863329E-2,3.417695E-3,1.0366536E-2,1E0,-1.4482354E-3,3.0805944E7,1.0036747E-2,1.2629019E8,4.771E3,1.8545505E2,2.0302527E-2,-9.418493E-3,3.7603674E-4,1E0,1.605076E6,-9.0386387E-4,3.813366E-3,1.6667E4,-4.6654847E-3,-4.7475668E-3,6.450564E-4,1.4473684E-1,8.856631E0,1.339646E6,1E0,-1.2344385E-3,-6.3225767E-3,7.3939515E-3,1.5539158E3,1.8067614E-3,-4.0687923E-3,2.9432205E6,2.906237E2,8E0,-4.251073E-3,-4.428699E-3,2.5530807E-3,-4.714015E-3,-1.0280934E-2,3.2283451E4,-2.3663742E-3,1.6952554E-2,8.5489005E-3,1.5204346E8,1.3261502E0,2.2588288E-2,1.1342433E3,7.0296685E6,4.2739182E-4,2.5134058E3,-8.865486E-3,-5.257754E-3,-1.3586165E-2,-7.855505E-3,-6.4492127E-4,-7.4248053E-3,-1.2885934E-2,-7.843881E-4,-6.574897E-3,-3.31523E-3,5.322587E-3,-1.2294298E-2,-4.1806474E-3,-1.6670658E-3,-7.3210024E-3,4.6119912E-4,4.2712283E-3,-2.9135854E-3,3.9420547E-3,5.457052E-4,8.389474E-3,1.0541899E-2,4.8453845E-3,6.2551233E-3,9.018273E-4,-8.8055115E-3,-2.4561705E-5,2.3880196E-4,7.7648554E-3,5.467038E-3,1.5072199E-2,1.2010223E-2,2.1380277E-3,6.979607E-3,1.672586E-2],"split_indices":[2,43,17,3,2,0,67,2,20,69,43,15,0,3,73,1,10,73,10,43,0,44,60,0,0,67,68,67,69,9,7,0,60,0,0,68,0,70,3,9,43,0,0,23,71,67,0,44,3,62,60,0,48,0,71,0,0,0,44,71,68,46,0,60,79,60,12,2,0,43,0,53,62,0,0,0,0,5,2,9,9,0,0,0,0,7,0,0,0,2,43,70,67,4,7,0,29,53,0,0,0,7,0,4,0,0,0,9,7,69,0,0,0,0,0,0,71,0,72,43,1,5,71,0,0,9,0,0,67,9,0,10,0,44,62,0,19,2,5,71,0,0,0,23,0,5,0,5,2,73,0,0,0,15,12,0,0,9,0,0,0,71,73,9,26,0,0,0,48,0,0,60,4,8,0,0,0,0,0,48,0,0,0,7,57,0,4,62,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.036E3,6.68E2,3.68E2,2.21E2,4.47E2,1.9E1,3.49E2,1.76E2,4.5E1,2.58E2,1.89E2,8E0,1.1E1,1.99E2,1.5E2,7.8E1,9.8E1,1E1,3.5E1,2.53E2,5E0,1.75E2,1.4E1,4E0,4E0,5.6E1,1.43E2,3.3E1,1.17E2,4.3E1,3.5E1,6E0,9.2E1,4E0,6E0,2.7E1,8E0,1.92E2,6.1E1,1.19E2,5.6E1,4E0,1E1,2.9E1,2.7E1,1.38E2,5E0,1.4E1,1.9E1,4.5E1,7.2E1,1.2E1,3.1E1,4E0,3.1E1,8.6E1,6E0,1.5E1,1.2E1,5.7E1,1.35E2,5.7E1,4E0,5.1E1,6.8E1,3E1,2.6E1,1.7E1,1.2E1,2.3E1,4E0,8.6E1,5.2E1,8E0,6E0,4E0,1.5E1,3.2E1,1.3E1,5.9E1,1.3E1,2.7E1,4E0,2.1E1,1E1,7.4E1,1.2E1,7E0,5E0,4.6E1,1.1E1,7E1,6.5E1,1.5E1,4.2E1,5E0,4.6E1,6.4E1,4E0,2.6E1,4E0,2.1E1,5E0,1.3E1,4E0,5E0,1.8E1,1.6E1,7E1,4.8E1,4E0,4E0,2.8E1,6E0,7E0,1E1,4.9E1,4E0,9E0,6.4E1,1E1,3.6E1,1E1,6E0,5E0,5.9E1,1.1E1,6E0,5.9E1,9E0,6E0,3.5E1,7E0,1E1,3.6E1,1.2E1,5.2E1,1.3E1,1.3E1,1.2E1,9E0,6E0,7E0,1.2E1,4E0,6.5E1,5E0,2.5E1,2.3E1,4E1,9E0,4E0,5E0,3E1,3.4E1,6E0,4E0,3.1E1,5E0,5E0,5E0,3.9E1,2E1,2.5E1,3.4E1,4E0,5E0,8E0,2.7E1,4E0,6E0,1.8E1,1.8E1,4.7E1,5E0,6E0,7E0,8E0,5E0,8E0,4E0,5E0,7E0,1.9E1,4.6E1,1.2E1,1.3E1,1.6E1,7E0,3.5E1,5E0,6E0,2.4E1,2.5E1,9E0,1.4E1,1.7E1,1.5E1,2.4E1,8E0,1.2E1,2.1E1,4E0,1.1E1,2.3E1,1.2E1,1.5E1,5E0,1.3E1,4E0,1.4E1,2.2E1,2.5E1,4E0,4E0,5E0,1.4E1,3.7E1,9E0,7E0,6E0,1.2E1,4E0,2.3E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[3.913325E-3,-9.931895E-2,3.1083712E-1,-3.1763157E-1,-2.1071972E-3,-4.4784325E-1,3.5547167E-1,-4.5457786E-1,-2.5262824E-1,-9.130332E-2,9.1489136E-2,-2.5141533E-2,-9.822276E-3,4.3573078E-2,3.1894782E-1,-4.1667438E-1,-4.2016767E-2,-3.936701E-1,-1.9808681E-1,-6.4099506E-2,-3.05111E-1,1.7501368E-1,3.434601E-2,1.5826014E-1,4.5669937E-1,-3.713674E-1,-2.9297657E-2,-2.4053128E-2,-3.258181E-1,3.4862563E-1,-2.451318E-1,-2.639635E-1,-3.4193274E-2,-2.065337E-1,-2.5996389E-2,2.1157478E-1,2.4137817E-2,1.2612889E-2,1.9797083E-2,1.2478939E-1,3.3153426E-2,3.062011E-1,5.5495554E-1,-2.0340558E-2,-2.4390937E-1,-1.8274166E-1,-3.7385923E-1,8.407582E-3,2.2123247E-2,-3.3229895E-2,-2.209109E-1,-1.82136E-1,-3.5616052E-1,-1.097188E-1,-1.6712171E-3,-2.4444067E-1,-2.4057087E-3,1.7775781E-1,1.7915823E-2,9.569802E-2,-8.437751E-3,3.8218956E-2,-1.368748E-1,-4.61842E-2,1.6069409E-1,3.5291427E-1,-2.0565116E-3,5.8687085E-1,-4.2657557E-4,-1.3189177E-1,-1.6830482E-2,-4.1841697E-3,-1.0584011E-2,-1.9104682E-2,-6.853139E-3,9.133671E-3,-2.3891242E-1,-9.636414E-3,-3.065895E-3,-8.395773E-3,-1.9998362E-2,2.4143733E-2,-1.5036044E-1,-2.6829498E-2,1.2720603E-1,-5.235127E-3,-1.308249E-2,1.508029E-2,1.5215199E-1,1.3449143E-1,-1.1228424E-3,-9.066222E-2,6.833433E-2,-7.916652E-5,-1.1171382E-2,-8.8848695E-3,5.5086453E-2,1.8045875E-1,-9.218476E-3,4.0806955E-1,1.8441735E-3,3.2386098E-2,4.6930048E-1,-8.346902E-3,-3.3596912E-3,-2.7991658E-1,-7.335519E-2,-3.6704876E-3,4.3540094E-3,-1.982777E-1,-1.2981921E-2,-7.813894E-2,3.0807743E-2,1.8298261E-1,-6.121228E-3,1.6421536E-1,-1.1758386E-3,2.7201432E-3,8.056629E-3,-1.0870488E-2,-4.50208E-2,-5.41519E-2,9.450346E-2,-1.7841109E-3,5.891995E-3,3.3544743E-1,1.3843986E-1,4.3896627E-1,5.0901463E-3,4.9769557E-1,1.0852653E-2,-1.2632258E-3,-2.9235902E-1,-3.0671412E-3,-1.3167296E-1,-3.5097697E-1,-1.1094296E-1,1.8554485E-3,-3.3200176E-3,-1.1270994E-1,4.6013794E-3,8.948036E-2,-5.161618E-2,8.999942E-2,1.4914457E-2,2.9274397E-2,1.8403146E-1,-9.048598E-3,1.4358122E-2,3.214653E-3,-7.4449317E-3,2.0508094E-2,1.3625069E-1,1.0811578E-2,1.9417403E-2,9.204352E-2,2.596275E-1,9.556691E-3,2.1355174E-2,2.5890335E-2,1.5765306E-2,-1.0036199E-2,-1.617885E-2,-2.627989E-3,2.3682076E-3,-8.100321E-3,-1.639086E-3,-2.5239557E-2,-9.399405E-3,2.6148575E-4,-8.734189E-3,1.9759787E-3,-5.905007E-3,2.4998833E-3,-3.8167748E-3,3.16947E-3,7.113256E-3,-7.1091275E-4,-5.467072E-3,-3.112851E-4,5.9766853E-3,-1.2287435E-3,3.6631525E-3,3.609185E-3,9.420872E-3,4.633323E-3,-1.0461089E-3,2.9267166E-3,-1.7201228E-3,6.909068E-3,-1.1526215E-3,4.6080537E-3,1.392952E-2,9.913788E-4,6.5795113E-3,6.3021085E-3,1.3699164E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,-1,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,-1,57,59,-1,61,63,-1,65,67,-1,69,71,73,-1,-1,-1,75,77,79,81,83,85,-1,87,-1,89,-1,91,93,95,97,99,-1,101,-1,103,-1,-1,-1,-1,-1,-1,105,-1,-1,-1,-1,107,109,111,113,-1,-1,-1,115,117,-1,119,121,-1,-1,-1,123,125,-1,127,-1,-1,129,-1,-1,131,133,-1,-1,135,137,139,141,143,-1,145,-1,-1,-1,-1,147,149,151,-1,-1,153,155,157,-1,159,-1,-1,161,163,165,167,169,-1,-1,171,173,175,177,179,-1,181,183,-1,185,187,-1,189,191,-1,-1,193,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2982693E1,1.65317E1,9.031397E0,2.0216312E0,4.5165052E0,2.2429752E-1,5.0224094E0,9.314451E-1,1.1913033E0,1.5915806E0,1.2540224E0,0E0,0E0,0E0,5.129442E0,4.7395802E-1,0E0,2.1805143E-1,3.197588E0,1.4673373E0,6.709602E-1,5.8512497E-1,5.432236E-1,1.9698946E0,1.6932831E0,4.3719292E-1,0E0,0E0,1.5163422E-1,1.2988806E-1,1.1273355E0,1.7654634E-1,5.303656E-1,1.2779498E-1,0E0,4.4356394E-1,3.4301212E-1,0E0,4.3815967E-1,6.519908E-1,0E0,8.6762667E-1,1.3558903E0,0E0,2.75622E-1,2.1977395E-2,1.4442635E-1,0E0,0E0,0E0,8.6108685E-1,4.4336677E-2,1.0866916E-1,3.572557E-1,4.9624702E-1,6.441629E-2,0E0,2.3202395E-1,0E0,8.2408905E-2,0E0,5.301715E-1,2.2651586E-1,2.9177466E-1,6.5543747E-1,7.7782726E-1,0E0,6.238079E-1,0E0,2.7269974E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.854744E-1,0E0,0E0,0E0,0E0,1.2367914E-1,3.2656598E-1,3.809461E-1,3.863781E-1,0E0,0E0,0E0,1.4344203E-1,2.8950542E-2,0E0,1.6104124E-1,3.577069E-1,0E0,0E0,0E0,8.6390056E-2,4.9409437E-1,0E0,3.105855E-1,0E0,0E0,1.3599586E-1,0E0,0E0,2.5284433E-1,8.977506E-2,0E0,0E0,4.5449173E-1,4.6694864E-2,1.9502687E-1,2.996163E-1,2.440188E-1,0E0,1.5772676E-1,0E0,0E0,0E0,0E0,1.9090365E-1,1.2327917E-1,2.7831393E-1,0E0,0E0,4.7944784E-2,3.5282135E-1,9.347296E-2,0E0,1.3253593E-1,0E0,0E0,2.6717567E-1,3.5077408E-2,4.4179976E-2,2.6347935E-1,2.287741E-1,0E0,0E0,1.1820942E-1,9.515066E-2,3.7902355E-2,6.1025783E-2,5.4189637E-2,0E0,2.7259042E-2,9.570241E-2,0E0,5.3145107E-2,3.641135E-2,0E0,2.0317085E-1,3.198514E-1,0E0,0E0,1.7472222E-1,4.13841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,38,38,39,39,41,41,42,42,44,44,45,45,46,46,50,50,51,51,52,52,53,53,54,54,55,55,57,57,59,59,61,61,62,62,63,63,64,64,65,65,67,67,69,69,76,76,81,81,82,82,83,83,84,84,88,88,89,89,91,91,92,92,96,96,97,97,99,99,102,102,105,105,106,106,109,109,110,110,111,111,112,112,113,113,115,115,120,120,121,121,122,122,125,125,126,126,127,127,129,129,132,132,133,133,134,134,135,135,136,136,139,139,140,140,141,141,142,142,143,143,145,145,146,146,148,148,149,149,151,151,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,-1,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,-1,58,60,-1,62,64,-1,66,68,-1,70,72,74,-1,-1,-1,76,78,80,82,84,86,-1,88,-1,90,-1,92,94,96,98,100,-1,102,-1,104,-1,-1,-1,-1,-1,-1,106,-1,-1,-1,-1,108,110,112,114,-1,-1,-1,116,118,-1,120,122,-1,-1,-1,124,126,-1,128,-1,-1,130,-1,-1,132,134,-1,-1,136,138,140,142,144,-1,146,-1,-1,-1,-1,148,150,152,-1,-1,154,156,158,-1,160,-1,-1,162,164,166,168,170,-1,-1,172,174,176,178,180,-1,182,184,-1,186,188,-1,190,192,-1,-1,194,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.8988005E5,1E0,3.3832976E7,9.31E2,9.3326636E-2,2.4481E4,6.9934395E-5,2.71E2,2.9802957E0,3.1706784E0,-2.5141533E-2,-9.822276E-3,4.3573078E-2,1.7587205E6,4.1361522E6,-4.2016767E-2,3.89E2,2.5E-1,3.89E2,2.0303884E6,1.0292E4,1.3208092E0,5.428175E3,5.97E3,4.622222E1,-2.9297657E-2,-2.4053128E-2,1.16E2,1.9E1,1.0479E4,3.8275862E0,1E0,1.339646E6,-2.5996389E-2,1.28738E5,4.68418E5,1.2612889E-2,1E0,1E0,3.3153426E-2,1.2673605E7,2.2313573E2,-2.0340558E-2,1E0,1.2E2,3.328629E6,8.407582E-3,2.2123247E-2,-3.3229895E-2,2.2E1,3E0,4.4761734E5,5E-1,5.5593027E9,3.1488764E0,-2.4057087E-3,1.9913E4,1.7915823E-2,1.9851096E7,-8.437751E-3,1E0,6.763312E7,2.0939393E0,3.0070068E7,1.9269184E7,-2.0565116E-3,1.7288135E0,-4.2657557E-4,1.16408E5,-1.6830482E-2,-4.1841697E-3,-1.0584011E-2,-1.9104682E-2,-6.853139E-3,9.133671E-3,7.997723E6,-9.636414E-3,-3.065895E-3,-8.395773E-3,-1.9998362E-2,5.44E2,1.2673605E7,2.956111E2,1.3508157E7,-5.235127E-3,-1.308249E-2,1.508029E-2,3.472E3,1.42946E5,-1.1228424E-3,4.8709216E5,3.7990784E7,-7.916652E-5,-1.1171382E-2,-8.8848695E-3,1.090477E6,1.4007937E0,-9.218476E-3,5.526E3,1.8441735E-3,3.2386098E-2,1.9995576E7,-8.346902E-3,-3.3596912E-3,1E0,1.8E0,-3.6704876E-3,4.3540094E-3,5.4E0,6.346204E-7,3.011152E6,1E0,1.65E2,-6.121228E-3,4.0906172E5,-1.1758386E-3,2.7201432E-3,8.056629E-3,-1.0870488E-2,4.88E2,5.2663404E-1,5.095006E5,-1.7841109E-3,5.891995E-3,1.1772152E0,1.9616238E2,2.0045958E6,5.0901463E-3,1.407E3,1.0852653E-2,-1.2632258E-3,1.6213593E1,1.3823239E2,5.7E3,2.8055556E0,1.5277778E0,1.8554485E-3,-3.3200176E-3,1.631108E6,1E0,6.5E1,4.017544E0,6.078218E2,1.4914457E-2,1.2E1,1.071E3,-9.048598E-3,1.5724638E1,1.15E2,-7.4449317E-3,1.26E2,2.954124E6,1.0811578E-2,1.9417403E-2,5.615327E5,1.7755102E0,9.556691E-3,2.1355174E-2,2.5890335E-2,1.5765306E-2,-1.0036199E-2,-1.617885E-2,-2.627989E-3,2.3682076E-3,-8.100321E-3,-1.639086E-3,-2.5239557E-2,-9.399405E-3,2.6148575E-4,-8.734189E-3,1.9759787E-3,-5.905007E-3,2.4998833E-3,-3.8167748E-3,3.16947E-3,7.113256E-3,-7.1091275E-4,-5.467072E-3,-3.112851E-4,5.9766853E-3,-1.2287435E-3,3.6631525E-3,3.609185E-3,9.420872E-3,4.633323E-3,-1.0461089E-3,2.9267166E-3,-1.7201228E-3,6.909068E-3,-1.1526215E-3,4.6080537E-3,1.392952E-2,9.913788E-4,6.5795113E-3,6.3021085E-3,1.3699164E-2],"split_indices":[2,43,17,7,2,53,12,52,2,68,69,0,0,0,43,60,0,1,73,2,60,44,68,67,2,4,0,0,2,0,9,69,29,9,0,1,9,0,19,27,0,60,73,0,8,44,9,0,0,0,10,8,43,68,46,73,0,9,0,60,0,29,7,68,58,62,0,69,0,7,0,0,0,0,0,0,9,0,0,0,0,12,60,4,60,0,0,0,10,1,0,43,7,0,0,0,60,68,0,2,0,0,9,0,0,13,68,0,0,73,52,9,26,0,0,43,0,0,0,0,4,57,43,0,0,68,71,43,0,0,0,0,73,67,10,71,68,0,0,5,30,0,69,48,0,3,2,0,73,0,0,10,62,0,0,43,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,7.78E2,2.61E2,2.39E2,5.39E2,1.4E1,2.47E2,7.5E1,1.64E2,2.76E2,2.63E2,9E0,5E0,1.3E1,2.34E2,7.1E1,4E0,4.4E1,1.2E2,2.46E2,3E1,1.06E2,1.57E2,1.09E2,1.25E2,6.1E1,1E1,1.3E1,3.1E1,9E0,1.11E2,3.1E1,2.15E2,2.3E1,7E0,8.5E1,2.1E1,8E0,1.49E2,1.04E2,5E0,5.1E1,7.4E1,3.8E1,2.3E1,9E0,2.2E1,5E0,4E0,4E0,1.07E2,1.8E1,1.3E1,6.4E1,1.51E2,1.8E1,5E0,7.3E1,1.2E1,1.6E1,5E0,1.34E2,1.5E1,1.8E1,8.6E1,4.5E1,6E0,7E1,4E0,1.3E1,1E1,4E0,5E0,1.8E1,4E0,4E0,1.03E2,1.4E1,4E0,5E0,8E0,1.5E1,4.9E1,1.27E2,2.4E1,5E0,1.3E1,9E0,6.4E1,1.2E1,4E0,2.5E1,1.09E2,7E0,8E0,7E0,1.1E1,8.2E1,4E0,3.8E1,7E0,3.3E1,3.7E1,6E0,7E0,8.2E1,2.1E1,6E0,9E0,3.6E1,1.3E1,6.7E1,6E1,2E1,4E0,6E1,4E0,5E0,7E0,5E0,2E1,1.9E1,9E1,5E0,6E0,1.6E1,6.6E1,3.4E1,4E0,3.2E1,5E0,4E0,7.8E1,1E1,1.1E1,1.2E1,2.4E1,7E0,6E0,4.7E1,2E1,3.5E1,2.5E1,1.3E1,7E0,8E0,5.2E1,5E0,1.5E1,1.3E1,6E0,3.3E1,5.7E1,9E0,7E0,4.9E1,1.7E1,4E0,3E1,2.1E1,1.1E1,3.6E1,4.2E1,5E0,5E0,7E0,4E0,4E0,8E0,1E1,1.4E1,4E0,4.3E1,1.3E1,7E0,2.8E1,7E0,1.7E1,8E0,4E0,9E0,4E0,4E0,9E0,4.3E1,4E0,1.1E1,5E0,8E0,8E0,2.5E1,4.8E1,9E0,2.1E1,2.8E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-2.1113707E-3,-1.0200969E-1,2.9720458E-1,-3.0300242E-1,-7.6127104E-3,-4.0487608E-1,3.3611038E-1,-4.32657E-1,-2.3474638E-1,-8.9797586E-2,8.5915886E-2,-2.2989823E-2,-6.3940496E-3,1.9984965E-1,4.3687844E-1,-4.8388547E-1,-2.7575174E-1,-3.1926566E-1,-9.4179496E-2,-2.3252317E-1,-6.087035E-2,1.5551583E-1,1.09599745E-2,3.3258677E-2,3.0441844E-1,3.7213E-2,3.862008E-1,-4.2371097E-1,-3.2307148E-2,-1.2989971E-1,-1.8416071E-2,-2.7564287E-1,-5.7377714E-1,3.7168902E-1,-1.6870017E-1,-1.5832473E-1,-2.6235604E-1,-4.6340823E-2,-3.5253155E-1,1.7503917E-1,-7.3555475E-3,3.770238E-2,-1.6522206E-1,-7.3001226E-3,6.3377075E-2,3.429147E-1,-5.881937E-4,4.063643E-1,-8.269427E-3,-4.4448215E-1,-1.0609695E-2,-1.1470621E-2,1.3799048E-3,-3.544665E-1,-2.0092966E-1,-1.1726351E-2,-3.6128372E-2,2.182578E-2,9.083242E-3,-8.356933E-2,-3.2560232E-1,-1.1683327E-2,-4.266479E-3,-3.1641376E-1,-1.7025228E-1,-7.5457096E-2,5.7075273E-2,-4.0173042E-4,-2.9457076E-2,1.3323134E-1,3.30252E-1,9.147734E-3,1.752083E-1,-1.2054948E-4,-2.914087E-1,3.6195736E-2,1.0630331E-2,3.89102E-1,1.699616E-1,3.0364278E-1,5.201724E-1,-1.6423136E-2,-2.2907563E-2,-3.745789E-1,-6.0390932E-3,4.752744E-3,-2.4677978E-1,-1.4514278E-1,1.6414579E-2,-1.7861769E-2,-7.6578357E-3,-8.806472E-3,-1.727421E-2,-1.05719855E-2,-3.6487186E-3,-5.0267972E-2,-2.2558868E-1,8.590074E-2,-6.110608E-3,1.4697424E-1,-4.605972E-3,1.7539717E-2,1.0079776E-2,2.7939972E-2,-9.807906E-3,1.1873181E-2,8.407445E-2,-1.7347636E-2,-6.885414E-3,7.664771E-3,1.2261668E-3,4.2744943E-1,2.1214108E-1,3.9057012E-3,1.0323624E-2,1.3112285E-3,3.2573983E-1,5.5834115E-1,8.002984E-3,-1.814009E-2,-8.166824E-3,-2.0136353E-1,-1.7184913E-2,7.246878E-4,-2.1648586E-1,-3.1306863E-2,4.3064235E-3,-9.680902E-2,2.788872E-3,-6.1936937E-2,-3.1377172E-1,3.7719805E-2,1.6785988E-1,1.073693E-1,2.5471452E-1,2.0379356E-1,6.174806E-3,5.137581E-4,6.045468E-3,3.122814E-2,-6.33057E-3,2.4400657E-1,2.176655E-2,1.4226803E-2,2.5292698E-3,1.7177637E-1,3.6794648E-1,4.9142686E-1,3.2846943E-2,-2.3944591E-1,4.5805506E-4,-1.2775961E-2,-5.629924E-3,-3.0526544E-3,7.704906E-4,1.2867505E-3,-1.3116398E-1,4.4401463E-2,-6.0443643E-2,-6.868443E-3,9.546565E-4,-4.096277E-1,-6.098741E-3,6.5622004E-3,-1.930035E-3,3.8364648E-3,1.1193912E-2,2.1227369E-1,8.599105E-2,1.2040161E-1,3.1434587E-1,1.3900035E-2,3.0470886E-3,-7.5299838E-3,2.3498027E-2,4.6168263E-3,1.1008066E-3,6.1910963E-3,1.4100073E-2,9.821588E-3,2.20022E-3,3.9764947E-1,8.01268E-3,2.4961524E-2,1.704951E-2,-1.1986963E-2,-6.5753767E-3,1.9012865E-3,-6.5830667E-3,-1.111884E-2,-5.067491E-3,6.215874E-4,6.2839515E-3,-4.8681768E-3,1.6382809E-3,-2.540426E-2,-1.0370468E-2,2.1250919E-3,-6.3480353E-3,1.525366E-2,4.5152805E-3,-1.952035E-3,4.439086E-3,8.087964E-3,1.9244282E-3,1.7417217E-2,6.795048E-3,2.045517E-3,-4.0745554E-3,-2.1880982E-3,1.471022E-3,2.2327043E-2,1.1705415E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,-1,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,-1,79,-1,81,-1,-1,-1,83,85,-1,-1,-1,-1,87,89,-1,-1,91,93,95,97,-1,-1,99,101,103,105,-1,107,109,-1,111,113,115,117,-1,-1,119,-1,-1,121,123,125,-1,-1,-1,-1,-1,-1,127,129,131,-1,133,-1,-1,-1,135,-1,-1,137,-1,-1,-1,139,141,143,-1,-1,-1,145,147,-1,-1,-1,149,-1,-1,151,153,-1,155,157,159,161,163,165,167,169,171,173,-1,-1,175,-1,177,-1,-1,-1,179,181,183,-1,185,-1,-1,-1,-1,-1,187,189,191,193,-1,-1,195,-1,-1,197,-1,-1,199,201,203,205,-1,-1,-1,207,-1,209,-1,-1,-1,-1,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0828514E1,1.4663904E1,7.183359E0,2.0843143E0,4.058572E0,2.7714157E-1,3.27672E0,5.4176044E-1,1.9183521E0,1.1459987E0,1.2868767E0,0E0,0E0,1.8364182E0,2.3041878E0,5.073824E-1,3.604071E-1,9.7429657E-1,2.2498646E0,6.2499523E-2,9.7952265E-1,8.143754E-1,5.716131E-1,2.5196993E-1,7.9795694E-1,0E0,1.4972763E0,1.0349655E-1,0E0,2.3439042E-1,0E0,4.5291328E-1,7.342825E-1,6.585169E-2,7.074363E-1,7.646945E-2,1.054554E-1,6.8180233E-1,1.0726861E0,7.4118257E-1,0E0,4.1067266E-1,3.2755244E-1,0E0,1.5934475E-1,3.971858E-1,0E0,1.256609E0,0E0,2.9431343E-2,0E0,0E0,0E0,1.4003944E-1,6.9400346E-1,0E0,0E0,0E0,0E0,2.301676E-1,1.2124705E-1,0E0,0E0,6.614208E-2,5.6786835E-2,6.542363E-1,2.842557E-1,0E0,0E0,3.251629E-1,5.5583E-2,3.7213796E-1,1.1256564E-1,0E0,5.1782787E-2,1.4948344E-1,0E0,2.1469021E-1,4.5086473E-2,3.9219618E-1,6.4485455E-1,0E0,0E0,6.642771E-2,0E0,0E0,1.6390634E-1,2.6729858E-1,5.7893336E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.7560678E-1,3.4134865E-1,1.6741857E-1,0E0,3.6521983E-1,0E0,0E0,0E0,3.1259516E-1,0E0,0E0,3.2674477E-2,0E0,0E0,0E0,1.16272494E-1,1.7361403E-1,1.3393152E-1,0E0,0E0,0E0,3.3319092E-1,1.6131306E-1,0E0,0E0,0E0,2.678914E-1,0E0,0E0,5.722308E-2,1.7785849E-2,0E0,2.7314502E-1,1.9213514E-1,7.471498E-2,2.2469199E-1,1.2022841E-1,8.089721E-2,1.3649082E-1,1.5437448E-1,1.00586504E-1,2.2068545E-1,0E0,0E0,4.749781E-2,0E0,1.9682765E-2,0E0,0E0,0E0,6.5886766E-2,2.0257378E-1,2.7907372E-2,0E0,1.7505407E-2,0E0,0E0,0E0,0E0,0E0,1.319717E-1,1.1470795E-1,1.22922346E-1,1.286989E-1,0E0,0E0,1.3626087E-1,0E0,0E0,1.4917064E-1,0E0,0E0,1.1627385E-1,8.0442876E-2,3.145042E-2,1.1223614E-1,0E0,0E0,0E0,1.6171852E-1,0E0,2.6834093E-2,0E0,0E0,0E0,0E0,3.7173986E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,47,47,49,49,53,53,54,54,59,59,60,60,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,77,77,78,78,79,79,80,80,83,83,86,86,87,87,88,88,95,95,96,96,97,97,99,99,103,103,106,106,110,110,111,111,112,112,116,116,117,117,121,121,124,124,125,125,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,141,141,145,145,146,146,147,147,149,149,155,155,156,156,157,157,158,158,161,161,164,164,167,167,168,168,169,169,170,170,174,174,176,176,181,181],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,-1,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,-1,80,-1,82,-1,-1,-1,84,86,-1,-1,-1,-1,88,90,-1,-1,92,94,96,98,-1,-1,100,102,104,106,-1,108,110,-1,112,114,116,118,-1,-1,120,-1,-1,122,124,126,-1,-1,-1,-1,-1,-1,128,130,132,-1,134,-1,-1,-1,136,-1,-1,138,-1,-1,-1,140,142,144,-1,-1,-1,146,148,-1,-1,-1,150,-1,-1,152,154,-1,156,158,160,162,164,166,168,170,172,174,-1,-1,176,-1,178,-1,-1,-1,180,182,184,-1,186,-1,-1,-1,-1,-1,188,190,192,194,-1,-1,196,-1,-1,198,-1,-1,200,202,204,206,-1,-1,-1,208,-1,210,-1,-1,-1,-1,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,7.7E1,9.48E2,1.5900173E3,9.9347424E2,1.4956522E1,7.24E2,3.79E2,3.774648E0,-2.2989823E-2,-6.3940496E-3,6.7148806E5,1.059448E7,1.0195397E8,3.2856784E7,7.382199E0,2.31E2,1.4452E4,3.6E1,2.83475E5,9.639872E0,8.147158E4,2E0,3.7213E-2,7.06191E7,5.707547E0,-3.2307148E-2,6.8E1,-1.8416071E-2,2.55E2,2.49E2,1.0101351E4,2.0734E4,4.9E2,4.795239E7,2.331083E6,5.69E2,6.009825E2,-7.3555475E-3,3.1797794E1,5.144203E2,-7.3001226E-3,7.098E3,2E0,-5.881937E-4,2.8530578E6,-8.269427E-3,1E0,-1.0609695E-2,-1.1470621E-2,1.3799048E-3,1.8E1,8.414097E-1,-1.1726351E-2,-3.6128372E-2,2.182578E-2,9.083242E-3,1.3022917E8,2.9994638E2,-1.1683327E-2,-4.266479E-3,3.390625E0,3E0,2.87E2,1.8E1,-4.0173042E-4,-2.9457076E-2,1E0,7.403968E2,5.1034E4,3.093458E0,-1.2054948E-4,1.2218E4,3.3382E4,1.0630331E-2,7.7191065E6,1.8062708E6,1E0,5.489183E1,-1.6423136E-2,-2.2907563E-2,3.328629E6,-6.0390932E-3,4.752744E-3,2.1E1,3.716E3,4.080551E-1,-1.7861769E-2,-7.6578357E-3,-8.806472E-3,-1.727421E-2,-1.05719855E-2,-3.6487186E-3,6.99E2,9.935484E-1,1.4285715E0,-6.110608E-3,2.8802464E8,-4.605972E-3,1.7539717E-2,1.0079776E-2,7.9555137E3,-9.807906E-3,1.1873181E-2,1.909E3,-1.7347636E-2,-6.885414E-3,7.664771E-3,4.539777E5,4.6134964E2,1.8667632E0,3.9057012E-3,1.0323624E-2,1.3112285E-3,1.0168186E5,6.663214E6,8.002984E-3,-1.814009E-2,-8.166824E-3,1.157602E6,-1.7184913E-2,7.246878E-4,7.0093E4,1.1358E4,4.3064235E-3,1.2916666E0,9.6E1,3.57E2,8.49E2,3.5714287E-1,4.5866325E6,7.8571427E-1,5.4007E7,1.3E1,3.257507E5,5.137581E-4,6.045468E-3,1.5998265E6,-6.33057E-3,5.941442E6,2.176655E-2,1.4226803E-2,2.5292698E-3,5.5933E5,2E1,2.9366477E0,3.2846943E-2,6.763314E7,4.5805506E-4,-1.2775961E-2,-5.629924E-3,-3.0526544E-3,7.704906E-4,2.8361E4,2.590909E0,7.1E1,1.432632E6,-6.868443E-3,9.546565E-4,7.6116E4,-6.098741E-3,6.5622004E-3,5.405423E6,3.8364648E-3,1.1193912E-2,3.074398E0,3.6E1,5.5426865E6,8.051603E7,1.3900035E-2,3.0470886E-3,-7.5299838E-3,4.2E2,4.6168263E-3,2.6463525E8,6.1910963E-3,1.4100073E-2,9.821588E-3,2.20022E-3,1.9347133E5,8.01268E-3,2.4961524E-2,1.704951E-2,-1.1986963E-2,-6.5753767E-3,1.9012865E-3,-6.5830667E-3,-1.111884E-2,-5.067491E-3,6.215874E-4,6.2839515E-3,-4.8681768E-3,1.6382809E-3,-2.540426E-2,-1.0370468E-2,2.1250919E-3,-6.3480353E-3,1.525366E-2,4.5152805E-3,-1.952035E-3,4.439086E-3,8.087964E-3,1.9244282E-3,1.7417217E-2,6.795048E-3,2.045517E-3,-4.0745554E-3,-2.1880982E-3,1.471022E-3,2.2327043E-2,1.1705415E-2],"split_indices":[2,43,17,44,2,70,67,73,2,2,69,0,0,43,12,46,60,69,44,9,3,1,69,43,6,0,59,69,0,0,0,2,2,43,44,1,7,43,2,70,0,71,4,0,2,32,0,43,0,20,0,0,0,3,68,0,0,0,0,7,67,0,0,69,8,0,3,0,0,6,4,44,68,0,44,9,0,62,43,26,71,0,0,9,0,0,8,44,53,0,0,0,0,0,0,2,68,68,0,5,0,0,0,62,0,0,2,0,0,0,43,4,57,0,0,0,48,43,0,0,0,12,0,0,1,44,0,68,0,0,2,68,43,68,7,3,43,0,0,62,0,60,0,0,0,9,3,53,0,7,0,0,0,0,0,1,69,0,12,0,0,12,0,0,43,0,0,71,0,43,7,0,0,0,0,0,7,0,0,0,0,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.72E2,2.57E2,2.46E2,5.26E2,1.3E1,2.44E2,8.3E1,1.63E2,2.8E2,2.46E2,9E0,4E0,1.05E2,1.39E2,6.1E1,2.2E1,1.01E2,6.2E1,4.6E1,2.34E2,1.27E2,1.19E2,4.1E1,6.4E1,1.5E1,1.24E2,5E1,1.1E1,1.1E1,1.1E1,8.8E1,1.3E1,8E0,5.4E1,1.5E1,3.1E1,2.24E2,1E1,1.2E2,7E0,1.04E2,1.5E1,5E0,3.6E1,5.7E1,7E0,1.2E2,4E0,4.4E1,6E0,6E0,5E0,4.1E1,4.7E1,6E0,7E0,4E0,4E0,3.6E1,1.8E1,5E0,1E1,1.8E1,1.3E1,1.75E2,4.9E1,5E0,5E0,9.6E1,2.4E1,8.7E1,1.7E1,7E0,8E0,3.2E1,4E0,4.4E1,1.3E1,6.5E1,5.5E1,1.9E1,2.5E1,3.7E1,4E0,6E0,4.1E1,2.2E1,1.4E1,1.2E1,6E0,7E0,1.1E1,7E0,6E0,1.51E2,2.4E1,4.3E1,6E0,9.1E1,5E0,1.5E1,9E0,8.1E1,6E0,8E0,9E0,4E0,4E0,6E0,2.6E1,3.5E1,9E0,6E0,7E0,5E0,6E1,4.9E1,6E0,3.3E1,4E0,3.2E1,9E0,7E0,1.5E1,9E0,5E0,8E1,7.1E1,9E0,1.5E1,2.8E1,1.5E1,6.8E1,2.3E1,8E0,7.3E1,4E0,5E0,2.2E1,4E0,8E0,2.7E1,5E0,4E0,1.4E1,4.6E1,3.7E1,1.2E1,2.7E1,5E0,8E0,7E0,5E0,4E0,2.1E1,5.9E1,4.3E1,2.8E1,4E0,5E0,9E0,6E0,7E0,2.1E1,8E0,7E0,1E1,5.8E1,8E0,1.5E1,4E0,4E0,6E0,6.7E1,6E0,1.6E1,4E0,4E0,1E1,4E0,3.9E1,7E0,2.4E1,1.3E1,2.1E1,6E0,1.7E1,4E0,8E0,5.1E1,3.3E1,1E1,1.9E1,9E0,4E0,5E0,1.6E1,5E0,4E0,6E0,4E0,5.4E1,4E0,4E0,1E1,5E0,5.7E1,1E1,6E0,1E1,2.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[1.4223594E-2,-8.634216E-2,3.0193153E-1,-2.7222386E-1,-6.765204E-3,7.940106E-1,2.5887245E-1,-4.040506E-1,-1.9873267E-1,-1.9419628E-1,3.405146E-2,4.0034797E-2,1.7948333E-2,-3.1415752E-1,2.9465175E-1,-3.5044417E-1,-5.3188396E-1,-2.8547212E-1,-4.5296635E-2,-1.10069335E-1,-2.6626772E-1,-2.4604792E-2,8.945693E-2,-2.5621648E-3,-1.8626895E-2,2.5246102E-1,5.9453166E-1,-3.798945E-1,-3.3814332E-3,-3.2094732E-2,-3.3830726E-1,-2.3854767E-1,-4.3946558E-1,2.438176E-2,-1.398615E-1,-3.6346477E-2,-1.7054267E-1,-1.8699847E-1,-3.5463905E-1,-1.1870257E-1,3.8861327E-3,1.0711176E-1,-1.8679948E-1,7.600477E-2,3.0271718E-1,1.3814948E-2,6.457696E-1,-1.8644042E-2,-2.1673195E-1,-7.077512E-3,-1.9020766E-2,-3.0667937E-1,-1.5621948E-1,-5.764379E-1,-2.6254186E-1,1.785968E-4,-1.918815E-1,-6.8915084E-2,2.8638325E-3,-2.1919231E-1,-1.8370199E-3,-1.3457746E-2,-1.2549819E-1,-2.8971314E-1,-2.4505343E-2,5.7134286E-2,-1.7573352E-1,-1.1868533E-2,2.2696266E-1,5.4524697E-2,1.6540694E-1,-8.068745E-4,-2.527867E-1,1.9602962E-2,1.4412609E-2,3.270215E-1,-1.413914E-1,3.3096164E-2,1.5074941E-2,-4.93375E-3,-1.3089308E-2,-2.1446358E-2,-2.6522526E-1,6.500218E-3,-1.9773133E-1,-1.43390065E-2,-3.1606384E-2,-1.4578584E-2,-6.2741884E-3,-4.3213803E-2,3.0139075E-3,-2.7602658E-1,-1.0649476E-1,-2.5482537E-4,-4.758274E-3,-4.357136E-3,-1.2010289E-2,-3.8744456E-3,-1.0793365E-2,-1.4583387E-2,-7.110835E-3,1.5007585E-4,4.751945E-3,-2.5428542E-1,-5.1831815E-2,3.7173957E-2,-7.588196E-2,5.351729E-3,1.3871613E-2,3.405184E-2,2.109637E-1,1.7860527E-1,-3.3879743E-3,-1.4970619E-2,-5.3922664E-3,7.656629E-2,-1.002134E-1,6.5897727E-1,2.9645064E-1,-1.0526263E-2,-1.2316033E-3,-1.7388396E-1,-1.418976E-2,-2.4303529E-1,-8.914783E-2,2.676173E-4,-3.8612033E-3,-1.4714607E-2,-6.7019206E-3,1.4255417E-3,-1.7226124E-1,-2.7240536E-3,-3.360536E-1,2.2319464E-2,-8.275968E-3,5.644833E-2,-1.0463053E-1,-2.444161E-2,-1.7546345E-1,1.2201927E-1,-1.5779171E-2,1.6101167E-2,5.1329285E-3,5.2408434E-2,2.0906413E-1,-1.5491253E-2,1.2345568E-1,-9.0388E-3,-7.442658E-4,1.6963217E-2,3.7650026E-2,3.2575783E-1,1.3192868E-1,-3.6361762E-3,-9.58352E-3,-8.411567E-3,-1.510531E-2,-9.4763655E-4,-6.603029E-3,-9.883966E-3,-4.064374E-3,-2.4326963E-2,-2.0158608E-1,3.7598812E-3,-1.8694615E-3,2.863704E-2,1.4276624E-1,-8.576235E-4,-9.433755E-3,-7.633371E-2,8.7890975E-2,-1.5007787E-3,-2.0385443E-1,6.7740284E-2,1.0025926E-2,-9.127981E-3,5.693843E-3,1.0643449E-1,-1.175741E-3,2.3600961E-1,2.5935834E-2,2.5258532E-3,-4.4623865E-3,1.1203726E-2,7.2118066E-2,-2.6310363E-3,3.3956462E-1,-1.8976894E-3,1.8226767E-1,-1.234629E-2,-3.2039825E-3,1.9455624E-3,-6.038761E-3,4.492031E-3,8.926657E-3,-6.647862E-3,-3.8139103E-4,9.4821565E-3,1.2269466E-3,-1.0759202E-2,-4.718485E-3,5.160075E-3,1.1655367E-3,5.1182546E-3,-1.3258873E-3,7.995434E-3,5.0633756E-4,1.1686313E-2,-1.1417135E-3,4.007029E-3,-9.792607E-4,1.2326427E-4,5.1006023E-3,9.331165E-3,1.7439645E-2,1.1267652E-2,5.0161807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,-1,77,-1,79,-1,-1,81,83,85,87,89,91,93,-1,95,-1,-1,97,99,-1,101,103,105,107,109,111,-1,113,115,-1,117,119,-1,-1,-1,-1,-1,121,-1,123,-1,-1,-1,-1,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,131,133,135,137,-1,-1,139,141,143,-1,-1,-1,145,147,149,151,-1,-1,153,-1,155,157,-1,-1,-1,-1,-1,159,-1,161,163,-1,165,167,169,171,173,175,-1,-1,177,179,181,183,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,189,-1,-1,191,193,-1,-1,195,197,-1,199,201,-1,-1,203,205,-1,207,209,-1,-1,-1,211,-1,213,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9827642E1,1.1299171E1,5.4517746E0,2.138277E0,4.107179E0,2.812481E-1,5.1567135E0,3.6867237E-1,1.9701042E0,5.527544E-1,1.435397E0,0E0,0E0,3.2311928E-1,2.7978554E0,4.621029E-1,4.5838213E-1,5.816426E-1,3.0542905E0,1.9888163E-1,2.9094434E-1,5.7686174E-1,1.1271461E0,0E0,0E0,1.8038015E0,2.3083878E-1,1.2103939E-1,0E0,0E0,9.998667E-2,3.7277794E-1,3.575673E-1,0E0,3.4970886E-1,7.473025E-2,1.5034372E-1,1.4946353E-1,1.0883498E-1,5.114477E-1,5.861001E-1,6.4701366E-1,1.5201959E-1,6.193949E-1,1.773242E0,0E0,2.8507996E-1,0E0,3.5632372E-2,0E0,0E0,1.3810015E-1,4.7154558E-1,1.0285854E-1,3.1056046E-2,4.2341985E-2,2.220751E-1,3.6162883E-2,0E0,6.7139804E-2,0E0,0E0,7.044408E-2,1.8370748E-2,0E0,3.1696096E-2,3.56663E-1,4.9256375E-1,5.4809332E-2,3.5793018E-1,3.3134794E-1,0E0,5.913955E-2,2.7385336E-1,0E0,1.347168E0,8.319841E-2,0E0,0E0,0E0,0E0,0E0,7.677555E-2,0E0,1.357503E-1,0E0,0E0,0E0,0E0,1.8286915E-2,0E0,4.8365593E-2,1.7694445E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.4752095E-1,1.5503596E-1,2.4916035E-1,3.4404194E-1,0E0,0E0,4.4904298E-1,1.3984573E-1,3.5804892E-1,0E0,0E0,0E0,1.184586E-1,1.0119216E-1,2.469325E-1,6.368799E-1,0E0,0E0,2.5191784E-2,0E0,5.3030014E-2,3.7670232E-2,0E0,0E0,0E0,0E0,0E0,2.6849806E-2,0E0,3.037399E-1,4.408544E-2,0E0,1.8555996E-1,9.0473436E-2,2.7485105E-1,8.674151E-2,1.7608303E-1,2.5877574E-1,0E0,0E0,8.68358E-2,3.738339E-1,6.2183775E-2,9.25259E-2,0E0,0E0,0E0,0E0,6.4812946E-1,2.077915E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.696691E-2,0E0,0E0,1.364909E-1,2.4864078E-2,0E0,0E0,1.4599709E-1,1.0251513E-1,0E0,3.2327116E-2,4.389397E-2,0E0,0E0,2.2004953E-1,7.444134E-2,0E0,3.0227232E-1,3.3359654E-2,0E0,0E0,0E0,3.693171E-2,0E0,5.2381897E-1,0E0,5.526066E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,82,82,84,84,89,89,91,91,92,92,103,103,104,104,105,105,106,106,109,109,110,110,111,111,115,115,116,116,117,117,118,118,121,121,123,123,124,124,130,130,132,132,133,133,135,135,136,136,137,137,138,138,139,139,140,140,143,143,144,144,145,145,146,146,151,151,152,152,162,162,165,165,166,166,169,169,170,170,172,172,173,173,176,176,177,177,179,179,180,180,184,184,186,186,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,-1,78,-1,80,-1,-1,82,84,86,88,90,92,94,-1,96,-1,-1,98,100,-1,102,104,106,108,110,112,-1,114,116,-1,118,120,-1,-1,-1,-1,-1,122,-1,124,-1,-1,-1,-1,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,132,134,136,138,-1,-1,140,142,144,-1,-1,-1,146,148,150,152,-1,-1,154,-1,156,158,-1,-1,-1,-1,-1,160,-1,162,164,-1,166,168,170,172,174,176,-1,-1,178,180,182,184,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,190,-1,-1,192,194,-1,-1,196,198,-1,200,202,-1,-1,204,206,-1,208,210,-1,-1,-1,212,-1,214,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.6E1,7.7E1,4.97E2,6.7321223E-1,6.829817E4,1.4504054E8,7.24E2,1.6923077E0,1.037E3,4.0034797E-2,1.7948333E-2,1.0322473E8,6.663214E6,4.56E2,1.4956522E1,1.7E1,2.04115E5,8.510249E-3,4.0753E4,1E0,9.639872E0,-2.5621648E-3,-1.8626895E-2,7.075161E2,3.53562E5,3.1530054E0,-3.3814332E-3,-3.2094732E-2,7.042942E7,2.55E2,6.593462E4,2.438176E-2,1.5522388E0,4.49239E6,6.769866E7,1.43E2,7.668863E6,5E-1,2.0200005E8,1.5115256E6,5.737478E7,6.103178E8,7.134432E7,1.3814948E-2,1.4379113E9,-1.8644042E-2,4.233978E0,-7.077512E-3,-1.9020766E-2,3.9E1,1E0,1.6E1,5E0,1.1E3,9.458613E3,2.0409561E6,2.8638325E-3,1.188E0,-1.8370199E-3,-1.3457746E-2,1.31625E2,4.48E2,-2.4505343E-2,7.52E2,8.1E2,1E0,1.6923077E0,3.299862E1,3.1027884E5,-8.068745E-4,3.38E2,1.9347133E5,1.4412609E-2,1.2083E4,9.4578236E-1,3.3096164E-2,1.5074941E-2,-4.93375E-3,-1.3089308E-2,-2.1446358E-2,1.4E1,6.500218E-3,8E0,-1.43390065E-2,-3.1606384E-2,-1.4578584E-2,-6.2741884E-3,2.02E2,3.0139075E-3,1E0,1.2038125E7,-2.5482537E-4,-4.758274E-3,-4.357136E-3,-1.2010289E-2,-3.8744456E-3,-1.0793365E-2,-1.4583387E-2,-7.110835E-3,1.5007585E-4,4.751945E-3,6.07E2,1.5E1,1E0,2E0,5.351729E-3,1.3871613E-2,1.4915254E0,3.13E2,3.7788504E7,-3.3879743E-3,-1.4970619E-2,-5.3922664E-3,3.0990322E5,1.4521204E0,2.8016653E3,1.7172844E7,-1.0526263E-2,-1.2316033E-3,1.00019E5,-1.418976E-2,5.80563E5,2.3266666E0,2.676173E-4,-3.8612033E-3,-1.4714607E-2,-6.7019206E-3,1.4255417E-3,2.58466E5,-2.7240536E-3,2.019537E2,1E1,-8.275968E-3,7.919006E7,6.763314E7,8.7E1,1.1356604E2,8.684966E-1,4.186E4,1.6101167E-2,5.1329285E-3,1.103E3,2.23099E7,3.921E3,5.7777777E0,-9.0388E-3,-7.442658E-4,1.6963217E-2,3.7650026E-2,3.167E3,1.465E4,-3.6361762E-3,-9.58352E-3,-8.411567E-3,-1.510531E-2,-9.4763655E-4,-6.603029E-3,-9.883966E-3,-4.064374E-3,-2.4326963E-2,1.4386049E6,3.7598812E-3,-1.8694615E-3,9.07826E0,2.28E0,-8.576235E-4,-9.433755E-3,2.118835E6,2E0,-1.5007787E-3,5.1042255E2,9.6E1,1.0025926E-2,-9.127981E-3,1.0439024E1,3.412336E7,-1.175741E-3,5.2409735E-7,2.7777777E0,2.5258532E-3,-4.4623865E-3,1.1203726E-2,8.11E2,-2.6310363E-3,8.747289E5,-1.8976894E-3,6.072304E5,-1.234629E-2,-3.2039825E-3,1.9455624E-3,-6.038761E-3,4.492031E-3,8.926657E-3,-6.647862E-3,-3.8139103E-4,9.4821565E-3,1.2269466E-3,-1.0759202E-2,-4.718485E-3,5.160075E-3,1.1655367E-3,5.1182546E-3,-1.3258873E-3,7.995434E-3,5.0633756E-4,1.1686313E-2,-1.1417135E-3,4.007029E-3,-9.792607E-4,1.2326427E-4,5.1006023E-3,9.331165E-3,1.7439645E-2,1.1267652E-2,5.0161807E-3],"split_indices":[2,43,0,44,2,53,43,46,2,68,2,0,0,5,43,10,73,8,5,53,12,29,69,0,0,67,1,68,0,0,60,2,43,0,68,43,7,2,5,68,5,43,5,7,59,0,12,0,68,0,0,2,21,3,69,2,48,60,0,68,0,0,67,2,0,12,2,26,68,71,48,0,0,48,0,9,49,0,0,0,0,0,3,0,8,0,0,0,0,10,0,27,12,0,0,0,0,0,0,0,0,0,0,2,3,19,32,0,0,68,0,7,0,0,0,43,53,4,60,0,0,5,0,12,68,0,0,0,0,0,1,0,4,3,0,7,7,0,67,57,9,0,0,10,9,2,71,0,0,0,0,2,44,0,0,0,0,0,0,0,0,0,43,0,0,71,71,0,0,9,8,0,4,0,0,0,73,7,0,52,69,0,0,0,0,0,43,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.63E2,2.66E2,2.28E2,5.35E2,2E1,2.46E2,8E1,1.48E2,9.5E1,4.4E2,1.6E1,4E0,1.4E1,2.32E2,5.9E1,2.1E1,9.4E1,5.4E1,4.5E1,5E1,2.14E2,2.26E2,4E0,1E1,2.05E2,2.7E1,5.3E1,6E0,1E1,1.1E1,7.4E1,2E1,7E0,4.7E1,2.1E1,2.4E1,2.8E1,2.2E1,4.9E1,1.65E2,2.13E2,1.3E1,4.6E1,1.59E2,5E0,2.2E1,4.5E1,8E0,4E0,7E0,3.9E1,3.5E1,1E1,1E1,1.3E1,3.4E1,1.6E1,5E0,1.7E1,7E0,9E0,1.9E1,1.8E1,4E0,1.2E1,3.7E1,1.55E2,1E1,1.13E2,1E2,4E0,9E0,3.8E1,8E0,1.51E2,8E0,1.7E1,5E0,4E0,4E0,6E0,3.3E1,4E0,3.1E1,4E0,6E0,6E0,4E0,8E0,5E0,1.6E1,1.8E1,6E0,1E1,5E0,1.2E1,1.5E1,4E0,1.4E1,4E0,6E0,6E0,2.2E1,1.5E1,8.8E1,6.7E1,5E0,5E0,1.01E2,1.2E1,9.5E1,5E0,5E0,4E0,2.6E1,1.2E1,1.1E1,1.4E2,4E0,4E0,1.2E1,2.1E1,2.1E1,1E1,4E0,4E0,1.1E1,5E0,6E0,1.2E1,7E0,1.5E1,1E1,5E0,7.8E1,1E1,4.5E1,2.2E1,3.6E1,6.5E1,4E0,8E0,1.9E1,7.6E1,9E0,1.7E1,5E0,7E0,5E0,6E0,1.18E2,2.2E1,4E0,8E0,1.4E1,7E0,5E0,5E0,7E0,5E0,5E0,1E1,5E0,5E0,6E1,1.8E1,6E0,4E0,3.1E1,1.4E1,4E0,1.8E1,2.4E1,1.2E1,6E0,5.9E1,1.1E1,8E0,6.6E1,1E1,5E0,4E0,4E0,1.3E1,4E0,1.14E2,5E0,1.7E1,6E0,4E0,5.6E1,4E0,1.1E1,7E0,1.5E1,1.6E1,4E0,1E1,1.3E1,5E0,1.1E1,1.3E1,1.4E1,4.5E1,6E0,5E0,6.2E1,4E0,4E0,6E0,5E0,8E0,2.6E1,8.8E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[1.1543609E-2,-8.338619E-2,3.051977E-1,-2.646278E-1,-3.4403836E-3,-3.7307516E-1,3.4792015E-1,-4.4438487E-1,-2.0215484E-1,-9.747073E-2,6.0152967E-2,-2.2339892E-2,-9.977077E-3,2.5551268E-1,5.1589066E-1,-3.5952973E-1,-6.9145286E-1,-2.3411669E-1,1.2810075E-1,-5.7790004E-2,-2.326693E-1,3.556915E-2,2.69155E-1,1.2523043E-1,3.551997E-1,6.626197E-1,2.2994138E-1,-3.8405544E-1,-7.7624316E-3,-1.7001208E-2,-3.622165E-2,-4.2762083E-1,-1.9992173E-1,2.8149905E-2,-1.4602683E-1,-8.362085E-2,9.852671E-2,-3.1050396E-1,-2.1463407E-2,1.14119865E-1,-7.978343E-3,1.4414949E-3,3.1053227E-1,2.2622343E-2,9.431877E-2,2.8278008E-1,4.7222808E-1,7.161346E-1,9.639213E-3,3.1366363E-1,-3.6032856E-3,-9.934622E-3,-4.0205657E-1,-8.603879E-3,-2.3185492E-2,-4.4929963E-2,-2.2759324E-1,3.441804E-4,-1.27123995E-2,-5.3285617E-2,-1.8312359E-1,6.300127E-2,1.02064395E-2,-1.6993105E-1,-4.6681356E-1,5.875403E-3,-9.363669E-2,1.3160466E-1,-1.1453167E-2,6.3196644E-3,-2.851087E-1,3.6019534E-1,5.0858995E-3,-2.9385116E-2,1.7119771E-1,2.0045193E-1,3.356893E-1,1.2639584E-2,2.4458908E-2,7.5985986E-1,1.8848816E-2,2.0518926E-1,2.1233212E-2,-2.591244E-1,-2.0399967E-2,-2.0485388E-1,2.0883279E-1,-2.8055418E-1,-1.5207374E-1,-1.4177686E-1,-2.4549743E-2,-1.0495798E-1,-2.6992512E-1,1.15091E-1,-1.3637824E-3,-1.19288955E-2,-2.4217737E-4,-3.022225E-1,-3.5502557E-2,-7.2634085E-3,6.42378E-5,6.72919E-2,2.542933E-1,-3.192825E-2,8.0160305E-2,-2.6347525E-3,-2.1074397E-2,3.8972566E-1,8.401162E-3,8.1129834E-2,-1.5587047E-1,1.6458493E-2,1.1194983E-1,2.489145E-1,-1.547148E-3,3.5823494E-1,6.5739336E-3,3.7956085E-2,2.4137238E-2,2.5682065E-1,2.335932E-3,-1.4580121E-2,-6.446454E-3,-1.4555292E-2,-3.8262142E-3,7.306444E-5,1.7293112E-2,-3.2006103E-1,-1.8519649E-1,-1.9031139E-1,4.326568E-2,-1.7694527E-1,2.2338717E-3,1.4073353E-2,-5.9194125E-2,-1.6171026E-1,7.129953E-4,-1.5398336E-3,-1.6305007E-2,7.289084E-3,2.585521E-3,-1.6314238E-2,-7.40696E-3,9.140162E-2,-1.1101867E-2,2.8632256E-1,8.160982E-4,-6.1204627E-2,5.801139E-2,-4.7625955E-3,1.6524196E-1,2.0019598E-2,8.762965E-3,7.8591466E-4,7.566744E-3,-1.0268122E-2,-2.4226597E-3,6.4916926E-4,1.39918E-1,6.4655896E-3,1.5030578E-2,1.7991068E-2,8.268109E-3,1.4572807E-2,5.700786E-3,-2.4285667E-1,-1.8167546E-2,-9.113013E-2,-1.5634745E-2,-2.2198202E-1,-1.1111426E-1,-1.84157E-3,5.43947E-3,-1.2372878E-2,-9.939814E-2,-2.0517794E-2,6.940438E-2,-9.404668E-2,2.4697002E-2,-3.5157346E-3,-8.8380575E-3,1.3244663E-1,6.370217E-4,3.975059E-3,3.1492576E-1,2.7158529E-2,-1.0737796E-1,1.1556205E-1,-4.6668056E-2,2.1825334E-2,-6.8090423E-3,1.936915E-1,-7.2736363E-4,1.5811813E-1,2.7241095E-3,-3.5474019E-3,-1.2960464E-2,-9.305543E-4,-8.974389E-3,-6.4456435E-3,-1.2947898E-2,-2.2215578E-3,-6.504209E-3,-5.7108663E-3,-1.4361461E-3,3.832691E-3,-2.9108948E-3,9.4427145E-4,5.979993E-3,-6.3294666E-3,-2.2214286E-3,2.6789808E-3,-2.1632232E-3,1.1095829E-2,4.425329E-3,-2.0961347E-3,1.9590857E-3,8.213827E-3,1.655673E-2,4.260382E-3,-3.8593335E-3,-7.925799E-3,-2.474342E-3,6.598231E-4,6.542611E-3,-5.9609422E-3,3.6020759E-3,-3.8036013E-3,2.5642654E-3,1.0251868E-2,1.4232753E-3,3.6959965E-3,8.329804E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,-1,73,75,77,79,-1,81,-1,-1,83,-1,-1,85,87,-1,-1,89,91,93,-1,95,97,-1,99,101,-1,103,105,107,-1,109,111,113,115,-1,-1,117,-1,119,-1,121,-1,123,125,127,129,131,133,135,137,139,-1,-1,-1,141,-1,-1,-1,143,145,147,149,-1,-1,151,-1,153,155,-1,157,159,-1,161,-1,-1,-1,163,-1,-1,-1,-1,-1,-1,-1,165,167,169,171,173,-1,175,177,179,-1,-1,-1,-1,-1,-1,-1,181,-1,183,-1,185,187,189,191,-1,-1,-1,-1,-1,-1,-1,193,-1,-1,-1,-1,-1,-1,195,-1,197,-1,199,201,-1,-1,-1,203,205,207,209,211,-1,-1,213,215,-1,217,219,221,223,225,227,-1,229,-1,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7957724E1,1.0982136E1,7.229288E0,2.5155144E0,3.1571677E0,1.5595078E-1,3.4537048E0,9.8833656E-1,1.8666615E0,1.1292517E0,1.6089178E0,0E0,0E0,1.9166231E0,3.2461567E0,1.5152693E-1,1.3071775E-1,9.6659184E-1,2.1996148E0,6.785125E-1,7.8241396E-1,9.689647E-1,3.1522393E-1,7.08601E-1,5.868511E-1,1.1381836E0,7.721162E-1,3.5241127E-2,0E0,0E0,0E0,3.1870747E-1,5.7734776E-1,0E0,2.2006813E-1,4.2187893E-1,9.043807E-2,6.897249E-1,1.6209728E-1,6.719564E-1,7.269231E-1,0E0,2.3907995E-1,0E0,6.0417473E-1,1.7228603E-1,2.0735884E-1,3.23246E-1,0E0,2.6979947E-1,0E0,0E0,8.323383E-2,0E0,0E0,9.356003E-1,4.2128706E-1,0E0,0E0,2.7877834E-1,1.955707E-1,9.9785306E-2,0E0,2.8940648E-1,5.621488E-1,0E0,6.0639888E-2,7.482611E-1,0E0,4.9680316E-1,3.2511312E-1,4.499674E-2,0E0,3.6210883E-1,3.8771605E-1,2.8043252E-1,8.855057E-2,0E0,0E0,1.568203E-1,0E0,1.0676861E-1,0E0,2.3493469E-2,0E0,1.621843E-1,3.080803E-1,1.9313383E-1,3.8169992E-1,1.8909329E-1,1.1456929E-1,1.3231261E-1,2.828126E-1,2.3979247E-2,0E0,0E0,0E0,3.006506E-2,0E0,0E0,0E0,5.0400937E-1,2.4505019E-1,3.0899096E-1,4.343098E-1,0E0,0E0,7.4831486E-2,0E0,7.141037E-2,7.074347E-2,0E0,8.291304E-2,1.2102866E-1,0E0,8.686447E-2,0E0,0E0,0E0,4.7103286E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6440296E-1,2.8625238E-1,8.068001E-2,6.0410865E-2,1.3545632E-1,0E0,7.989654E-2,1.3389459E-1,1.8377662E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2716707E-1,0E0,1.3903475E-1,0E0,3.619581E-1,1.8009982E-1,1.2076971E-1,1.5751868E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6037365E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3489258E-1,0E0,1.0589628E-1,0E0,1.0027349E-1,1.9412532E-2,0E0,0E0,0E0,1.9316971E-2,1.1954331E-1,4.438863E-2,5.659473E-2,3.6049075E-2,0E0,0E0,1.4285707E-1,4.035822E-2,0E0,8.525276E-2,2.3095919E-1,1.9115108E-1,4.811704E-2,1.2297499E-1,9.9283375E-2,0E0,1.12653494E-1,0E0,1.8501848E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,49,49,52,52,55,55,56,56,59,59,60,60,61,61,63,63,64,64,66,66,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,97,97,101,101,102,102,103,103,104,104,107,107,109,109,110,110,112,112,113,113,115,115,119,119,127,127,128,128,129,129,130,130,131,131,133,133,134,134,135,135,143,143,145,145,147,147,148,148,149,149,150,150,158,158,165,165,167,167,169,169,170,170,174,174,175,175,176,176,177,177,178,178,181,181,182,182,184,184,185,185,186,186,187,187,188,188,189,189,191,191,193,193],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,-1,74,76,78,80,-1,82,-1,-1,84,-1,-1,86,88,-1,-1,90,92,94,-1,96,98,-1,100,102,-1,104,106,108,-1,110,112,114,116,-1,-1,118,-1,120,-1,122,-1,124,126,128,130,132,134,136,138,140,-1,-1,-1,142,-1,-1,-1,144,146,148,150,-1,-1,152,-1,154,156,-1,158,160,-1,162,-1,-1,-1,164,-1,-1,-1,-1,-1,-1,-1,166,168,170,172,174,-1,176,178,180,-1,-1,-1,-1,-1,-1,-1,182,-1,184,-1,186,188,190,192,-1,-1,-1,-1,-1,-1,-1,194,-1,-1,-1,-1,-1,-1,196,-1,198,-1,200,202,-1,-1,-1,204,206,208,210,212,-1,-1,214,216,-1,218,220,222,224,226,228,-1,230,-1,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,1E0,3.3817584E7,8.1E2,9.3326636E-2,1.6975454E3,2.2E1,2.41E3,4.5709968E0,3.7951445E6,-2.2339892E-2,-9.977077E-3,1.6802971E6,1.2792593E2,2.46875E0,4.869229E7,1.0479E4,2.3E1,4.1973075E6,2.6801266E2,1.5132743E0,3.3875784E7,5E1,4.391553E6,4.262058E0,6.747343E8,2E0,-7.7624316E-3,-1.7001208E-2,-3.622165E-2,4E0,9.935484E-1,2.8149905E-2,2E1,3.6245735E0,1.6341463E0,2.6742857E0,3.1316666E2,1.4676277E7,1.2294118E1,1.4414949E-3,8.931E3,2.2622343E-2,8.190129E9,1.0089981E3,1.7068776E10,8.913649E-3,9.639213E-3,3.8771296E3,-3.6032856E-3,-9.934622E-3,1.863E3,-8.603879E-3,-2.3185492E-2,1.8236995E0,1.559733E6,3.441804E-4,-1.27123995E-2,1.3920766E2,9.6E1,6.34E2,1.02064395E-2,8.39777E5,1.1742609E7,5.875403E-3,7.09E2,4.797342E2,-1.1453167E-2,1.4603988E6,4E1,1.8E1,5.0858995E-3,1.2E1,1.3864E4,9.7944025E2,1E0,1.2639584E-2,2.4458908E-2,1.557718E7,1.8848816E-2,5.501002E9,2.1233212E-2,8.209663E3,-2.0399967E-2,8.579633E-2,4.13E2,3.9682608E2,6.4522544E5,2.331083E6,1.4007937E0,3.0948E2,1.45064E5,7.318515E6,-1.3637824E-3,-1.19288955E-2,-2.4217737E-4,1.7E1,-3.5502557E-2,-7.2634085E-3,6.42378E-5,2.511E3,2.0319266E-7,9.751103E6,6.763312E7,-2.6347525E-3,-2.1074397E-2,1E0,8.401162E-3,3.1720988E6,2.0312773E8,1.6458493E-2,8.620714E2,1.2544625E5,-1.547148E-3,1.8039524E7,6.5739336E-3,3.7956085E-2,2.4137238E-2,5.6516E4,2.335932E-3,-1.4580121E-2,-6.446454E-3,-1.4555292E-2,-3.8262142E-3,7.306444E-5,1.7293112E-2,3.962617E0,8.9535345E2,1.5152774E2,9.593451E4,2.6818182E0,2.2338717E-3,2.732381E2,2.347364E7,2.227328E4,7.129953E-4,-1.5398336E-3,-1.6305007E-2,7.289084E-3,2.585521E-3,-1.6314238E-2,-7.40696E-3,5.2767E4,-1.1101867E-2,3.292496E-2,8.160982E-4,2.67637E5,4.3E1,2.1651245E6,2.03353E-6,2.0019598E-2,8.762965E-3,7.8591466E-4,7.566744E-3,-1.0268122E-2,-2.4226597E-3,6.4916926E-4,1.2792593E2,6.4655896E-3,1.5030578E-2,1.7991068E-2,8.268109E-3,1.4572807E-2,5.700786E-3,2.90099E0,-1.8167546E-2,6.185E3,-1.5634745E-2,1.1967312E3,1.3E1,-1.84157E-3,5.43947E-3,-1.2372878E-2,2.707673E6,1E1,2.6818182E0,1.084437E6,1E0,-3.5157346E-3,-8.8380575E-3,7.8571427E-1,1.047E3,3.975059E-3,5.281628E2,7.755551E7,1.244E3,3.0536E4,1.7022566E7,1.0738284E4,-6.8090423E-3,1.7022608E7,-7.2736363E-4,1.4E1,2.7241095E-3,-3.5474019E-3,-1.2960464E-2,-9.305543E-4,-8.974389E-3,-6.4456435E-3,-1.2947898E-2,-2.2215578E-3,-6.504209E-3,-5.7108663E-3,-1.4361461E-3,3.832691E-3,-2.9108948E-3,9.4427145E-4,5.979993E-3,-6.3294666E-3,-2.2214286E-3,2.6789808E-3,-2.1632232E-3,1.1095829E-2,4.425329E-3,-2.0961347E-3,1.9590857E-3,8.213827E-3,1.655673E-2,4.260382E-3,-3.8593335E-3,-7.925799E-3,-2.474342E-3,6.598231E-4,6.542611E-3,-5.9609422E-3,3.6020759E-3,-3.8036013E-3,2.5642654E-3,1.0251868E-2,1.4232753E-3,3.6959965E-3,8.329804E-3],"split_indices":[2,43,17,7,2,53,67,3,2,69,43,0,0,43,73,68,46,9,0,43,70,68,7,0,43,53,47,32,0,0,0,10,68,0,3,71,68,71,4,62,68,0,44,0,46,4,46,72,0,67,0,0,9,0,0,69,9,0,0,70,0,2,0,9,5,0,2,4,0,43,3,3,0,3,10,70,6,0,0,66,0,5,0,43,0,53,2,4,62,43,68,4,7,60,0,0,0,3,0,0,0,2,52,9,7,0,0,31,0,60,7,0,4,48,0,9,0,0,0,9,0,0,0,0,0,0,0,71,4,67,43,69,0,4,5,66,0,0,0,0,0,0,0,1,0,73,0,9,8,62,52,0,0,0,0,0,0,0,73,0,0,0,0,0,0,69,0,44,0,48,3,0,0,0,5,3,69,9,30,0,0,68,2,0,4,5,2,1,9,62,0,9,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.001E3,7.57E2,2.44E2,2.31E2,5.26E2,1.4E1,2.3E2,5.8E1,1.73E2,2.12E2,3.14E2,7E0,7E0,1.5E2,8E1,4.5E1,1.3E1,1.58E2,1.5E1,1.65E2,4.7E1,2.82E2,3.2E1,6.6E1,8.4E1,5.2E1,2.8E1,3.9E1,6E0,4E0,9E0,2.2E1,1.36E2,5E0,1E1,1.42E2,2.3E1,3.4E1,1.3E1,1E2,1.82E2,5E0,2.7E1,4E0,6.2E1,5.4E1,3E1,4.6E1,6E0,2.2E1,6E0,5E0,3.4E1,6E0,1.6E1,2.1E1,1.15E2,5E0,5E0,1.1E2,3.2E1,1.9E1,4E0,1.9E1,1.5E1,4E0,9E0,9.6E1,4E0,1.74E2,8E0,2.1E1,6E0,2.4E1,3.8E1,2.3E1,3.1E1,8E0,2.2E1,3.9E1,7E0,1.4E1,8E0,9E0,2.5E1,1.3E1,8E0,6.6E1,4.9E1,2.6E1,8.4E1,1.8E1,1.4E1,1.2E1,7E0,1.2E1,7E0,1.1E1,4E0,5E0,4E0,6.4E1,3.2E1,1.15E2,5.9E1,4E0,4E0,1.7E1,4E0,1.3E1,1.1E1,8E0,3E1,1.9E1,4E0,2.7E1,4E0,2.9E1,1E1,1E1,4E0,5E0,4E0,6E0,7E0,4E0,4E0,4.5E1,2.1E1,4.1E1,8E0,2.2E1,4E0,4E1,4.4E1,1.2E1,6E0,4E0,1E1,6E0,6E0,7E0,4E0,6E1,4E0,2.8E1,4E0,8.7E1,2.8E1,3E1,2.9E1,1.3E1,4E0,8E0,5E0,6E0,5E0,7E0,2.3E1,9E0,1E1,2.2E1,5E0,6E0,4E0,2.4E1,2.1E1,1.4E1,7E0,2.8E1,1.3E1,4E0,4E0,9E0,1.3E1,2.5E1,1.5E1,3.1E1,1.3E1,4E0,8E0,4.1E1,1.9E1,4E0,2.4E1,3E1,5.7E1,1.8E1,1E1,2.6E1,4E0,2.5E1,4E0,1.8E1,5E0,5E0,1.9E1,9E0,5E0,1.3E1,1.5E1,5E0,8E0,9E0,4E0,7E0,1.8E1,9E0,6E0,1.5E1,1.6E1,9E0,4E0,9E0,3.2E1,9E0,1E1,7E0,1.7E1,1.9E1,1.1E1,2.5E1,3.2E1,4E0,1.4E1,6E0,4E0,6E0,2E1,2.1E1,4E0,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[-3.9656907E-3,-9.501214E-2,2.7065217E-1,-2.784495E-1,-9.365859E-3,-3.6793208E-1,3.017125E-1,-4.4126916E-1,-2.2017272E-1,-1.7643312E-1,2.1908537E-2,-2.1849465E-2,-8.505184E-3,1.5604289E-1,4.175075E-1,-3.507889E-1,-7.0138806E-1,-2.3586765E-1,9.097859E-3,-1.4819905E-1,-2.6608398E-2,3.0838398E-2,-2.1082714E-2,2.8791332E-1,1.0092565E-1,5.0469875E-1,2.5951615E-1,-2.0058792E-2,-2.6352626E-1,-1.7305136E-2,-4.054622E-2,2.1098584E-3,-2.636969E-1,-1.8473676E-1,-5.438283E-2,-5.381478E-2,6.6572115E-2,1.5014201E-1,1.7512841E-2,1.2969783E-1,-7.551079E-3,5.437801E-1,1.9590639E-1,2.812694E-2,1.86803E-1,-1.603637E-1,-2.1972109E-2,-2.7934167E-1,2.3219657E-1,-3.302879E-1,-1.7906928E-1,-1.1146918E-1,-2.4207743E-1,1.619824E-2,-1.09921984E-1,-1.9888683E-1,-3.408119E-2,2.1942863E-2,1.4924604E-1,1.7885772E-3,1.0573727E-2,2.3934389E-2,1.9963692E-1,4.638127E-1,6.910533E-1,1.28993485E-2,1.8524277E-3,1.1050021E-1,3.6835158E-1,-2.061758E-1,4.2250715E-4,-6.0287667E-3,-1.7200807E-2,1.6376795E-2,6.724597E-4,-4.2167246E-1,-2.4743077E-1,-2.2192113E-1,-6.0843613E-2,-9.97553E-3,-6.9336176E-2,-2.721887E-1,-1.7132155E-3,-3.5586585E-3,3.717867E-3,-6.844297E-3,-1.8772351E-3,-1.376032E-2,-3.4678967E-3,7.466982E-3,-4.6565972E-2,3.2273047E-2,-1.7515205E-2,2.2839847E-1,7.858436E-2,-3.4470063E-2,1.3834985E-1,1.7988099E-2,1.5056755E-1,1.0686118E-2,2.2362962E-2,3.714944E-2,2.0898255E-2,1.2367412E-2,4.418352E-2,1.1051776E-2,2.1141248E-2,-1.143979E-2,-5.7026832E-3,-3.279505E-1,-5.030666E-1,-2.6878613E-1,-6.6011306E-4,-1.2509137E-1,-2.6261213E-1,-9.1879135E-3,1.2518891E-2,1.7837885E-4,-4.059825E-3,-3.0420086E-1,-6.5529323E-3,-9.235642E-2,3.6423183E-3,1.7449673E-2,2.3503724E-1,2.5325418E-1,-1.8933563E-3,-2.1648366E-2,1.376548E-1,-4.706644E-3,9.66299E-3,1.0267954E-3,9.797057E-3,2.8059423E-2,1.9189501E-1,9.446828E-2,-3.8922029E-3,-1.7883098E-2,-5.340247E-3,-2.667475E-2,-1.5185724E-2,-2.9505622E-1,-2.6615907E-3,-7.375062E-3,-4.7378655E-4,-2.841051E-1,-3.5496268E-3,6.878219E-2,-4.735933E-3,-7.836087E-3,-1.5603953E-2,-3.6249865E-2,-1.4716834E-1,7.715605E-2,-4.8646215E-2,-1.1873774E-2,7.4849084E-2,1.8090488E-3,1.6288273E-2,1.6469273E-1,3.308046E-1,3.084686E-2,-1.0090223E-1,1.0215925E-2,5.1329482E-2,4.0874323E-3,-3.519478E-2,-4.97195E-3,8.558977E-3,1.2989512E-3,2.2955103E-1,-6.5646076E-4,1.4093477E-1,-1.5428957E-2,-8.987475E-3,-1.4099805E-2,-6.715339E-3,4.6868604E-3,7.3111086E-4,1.4541056E-3,-5.130876E-3,-7.5847246E-3,-1.6679307E-3,4.994398E-3,-2.1293836E-4,-6.8203514E-3,1.4341802E-3,-4.3626237E-3,7.148556E-4,4.195675E-3,-5.6436253E-3,1.6728593E-3,8.770242E-3,9.296893E-3,1.732066E-2,3.204331E-3,-2.4183947E-3,-6.6490406E-3,-1.7418164E-3,5.538608E-3,-2.6759E-3,6.589656E-4,-3.5857034E-3,1.4160664E-2,4.2908005E-3,3.2838914E-3,8.34704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,-1,35,-1,37,39,41,43,-1,45,-1,-1,47,49,51,53,55,57,59,-1,61,-1,63,65,-1,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,-1,-1,109,111,113,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,121,123,-1,125,127,129,131,-1,133,-1,-1,-1,-1,-1,135,-1,-1,-1,-1,137,139,141,-1,143,145,-1,147,-1,-1,149,-1,151,153,155,157,159,-1,161,163,-1,165,-1,-1,167,169,171,-1,-1,-1,-1,-1,173,-1,-1,-1,175,-1,177,-1,-1,-1,179,181,183,185,187,189,-1,-1,191,193,195,197,-1,199,-1,201,-1,-1,-1,203,-1,205,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5003695E1,1.1796147E1,5.049837E0,2.1706238E0,2.6842532E0,1.4458358E-1,3.9493904E0,1.21317E0,1.2116995E0,8.2832646E-1,1.8633032E0,0E0,0E0,7.486539E-1,1.6873856E0,2.3488188E-1,5.352783E-1,1.1396675E0,0E0,2.5389636E-1,0E0,1.2922211E0,0E0,3.3242273E-1,6.069044E-1,9.071808E-1,1.1308231E0,0E0,4.9255216E-1,0E0,0E0,1.295269E0,8.087139E-1,2.0408261E-1,9.194082E-2,3.5679632E-1,1.1036274E0,1.1395872E-1,0E0,5.130117E-1,0E0,5.509834E-1,1.315833E-1,0E0,5.4110885E-1,1.4943129E-1,0E0,7.596862E-2,2.8760093E-1,5.3927326E-1,3.4113073E-1,9.9455565E-2,1.7508519E-1,7.211452E-2,2.8412238E-2,1.614477E-1,2.836265E-1,8.2428366E-1,5.6932116E-1,0E0,0E0,1.9855772E-1,3.3237934E-1,1.2808514E-1,2.839775E-1,0E0,0E0,3.1134298E-1,2.8845072E-2,2.0869553E-2,0E0,0E0,0E0,0E0,0E0,1.3252687E-1,2.30165E-1,1.680944E-1,2.0501871E-1,0E0,2.748178E-2,6.534195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4633583E-1,5.7298344E-1,0E0,3.3846617E-1,3.4046504E-1,6.0482375E-2,8.088446E-2,0E0,1.7181867E-1,0E0,0E0,0E0,0E0,0E0,1.5649688E-1,0E0,0E0,0E0,0E0,2.1898055E-1,5.6430817E-2,2.2154713E-1,0E0,6.5802634E-2,1.1792421E-1,0E0,9.765879E-2,0E0,0E0,3.302896E-2,0E0,1.6688243E-1,2.0363745E-1,3.0385548E-1,2.8547662E-1,2.5457764E-1,0E0,9.464241E-2,2.4806172E-1,0E0,4.907278E-2,0E0,0E0,2.3323132E-1,1.5389436E-1,8.785039E-2,0E0,0E0,0E0,0E0,0E0,8.36575E-2,0E0,0E0,0E0,4.6357393E-2,0E0,1.5694037E-2,0E0,0E0,0E0,1.5042421E-1,4.6960175E-2,5.617225E-2,2.4910985E-1,2.7238593E-1,2.029925E-1,0E0,0E0,6.742704E-2,5.6003332E-2,4.8486833E-2,1.904314E-2,0E0,1.4925034E-1,0E0,1.998904E-2,0E0,0E0,0E0,1.924293E-1,0E0,2.046971E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,21,21,23,23,24,24,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,69,69,75,75,76,76,77,77,78,78,80,80,81,81,90,90,91,91,93,93,94,94,95,95,96,96,98,98,104,104,109,109,110,110,111,111,113,113,114,114,116,116,119,119,121,121,122,122,123,123,124,124,125,125,127,127,128,128,130,130,133,133,134,134,135,135,141,141,145,145,147,147,151,151,152,152,153,153,154,154,155,155,156,156,159,159,160,160,161,161,162,162,164,164,166,166,170,170,172,172],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,-1,36,-1,38,40,42,44,-1,46,-1,-1,48,50,52,54,56,58,60,-1,62,-1,64,66,-1,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,-1,-1,110,112,114,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,122,124,-1,126,128,130,132,-1,134,-1,-1,-1,-1,-1,136,-1,-1,-1,-1,138,140,142,-1,144,146,-1,148,-1,-1,150,-1,152,154,156,158,160,-1,162,164,-1,166,-1,-1,168,170,172,-1,-1,-1,-1,-1,174,-1,-1,-1,176,-1,178,-1,-1,-1,180,182,184,186,188,190,-1,-1,192,194,196,198,-1,200,-1,202,-1,-1,-1,204,-1,206,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.972052E5,1E0,3.3817584E7,4.97E2,9.3326636E-2,1.087521E3,2.4545455E0,2.8366232E0,1E0,4.6E1,-2.1849465E-2,-8.505184E-3,7E0,8E0,3.6E1,2.8914285E0,5.597832E-4,9.097859E-3,8.599521E5,-2.6608398E-2,6.374563E5,-2.1082714E-2,1.2774637E10,1.2118524E7,4.9036694E0,1.2083E4,-2.0058792E-2,3.7509E4,-1.7305136E-2,-4.054622E-2,4.1142857E1,5.100098E4,3.2E1,3.5364E4,1E0,3.9742767E2,6.4348956E2,1.7512841E-2,3.3193566E5,-7.551079E-3,1.9617874E3,2.0003653E0,2.812694E-2,9.0577506E-2,2.6412E5,-2.1972109E-2,6.737099E7,2E0,8.3925985E-2,7.9E2,8.3E1,7.026624E7,8.754E3,1E0,5.0200376E7,1.5E1,2.483E3,2.9375E0,1.7885772E-3,1.0573727E-2,2.3625128E6,2.08E3,1.9675E4,1.7493458E7,1.28993485E-2,1.8524277E-3,4.229649E6,4.907764E3,5.6E1,4.2250715E-4,-6.0287667E-3,-1.7200807E-2,1.6376795E-2,6.724597E-4,3.6E2,1.3091103E0,9.117022E0,1.7356666E3,-9.97553E-3,1.2E1,2.241E2,-1.7132155E-3,-3.5586585E-3,3.717867E-3,-6.844297E-3,-1.8772351E-3,-1.376032E-2,-3.4678967E-3,7.466982E-3,4.220534E5,7.4711205E6,-1.7515205E-2,3.0561172E5,1.574E3,1.5204346E8,5.130175E2,1.7988099E-2,4.032683E6,1.0686118E-2,2.2362962E-2,3.714944E-2,2.0898255E-2,1.2367412E-2,2.2807484E7,1.1051776E-2,2.1141248E-2,-1.143979E-2,-5.7026832E-3,4.1142857E1,1.9885094E-8,5.13657E6,-6.6011306E-4,1.4276666E2,7.997723E6,-9.1879135E-3,3.3594444E1,1.7837885E-4,-4.059825E-3,1.2608696E0,-6.5529323E-3,1E0,6.933837E-1,2.378914E6,1E0,1.0641107E8,-1.8933563E-3,5.5586E4,4.857143E0,-4.706644E-3,1.4195632E6,1.0267954E-3,9.797057E-3,2.0094643E0,2.7662E4,2.6776E4,-3.8922029E-3,-1.7883098E-2,-5.340247E-3,-2.667475E-2,-1.5185724E-2,1.0433326E8,-2.6615907E-3,-7.375062E-3,-4.7378655E-4,1.547E3,-3.5496268E-3,1.364E3,-4.735933E-3,-7.836087E-3,-1.5603953E-2,4.2609915E6,4.032683E6,1.5998265E6,9.558022E7,3.2883008E6,8.561815E7,1.8090488E-3,1.6288273E-2,4.371613E2,4.7185095E2,2.141096E1,4.7563504E7,1.0215925E-2,1.3736341E8,4.0874323E-3,2.9355192E1,-4.97195E-3,8.558977E-3,1.2989512E-3,4.205298E0,-6.5646076E-4,1.562752E3,-1.5428957E-2,-8.987475E-3,-1.4099805E-2,-6.715339E-3,4.6868604E-3,7.3111086E-4,1.4541056E-3,-5.130876E-3,-7.5847246E-3,-1.6679307E-3,4.994398E-3,-2.1293836E-4,-6.8203514E-3,1.4341802E-3,-4.3626237E-3,7.148556E-4,4.195675E-3,-5.6436253E-3,1.6728593E-3,8.770242E-3,9.296893E-3,1.732066E-2,3.204331E-3,-2.4183947E-3,-6.6490406E-3,-1.7418164E-3,5.538608E-3,-2.6759E-3,6.589656E-4,-3.5857034E-3,1.4160664E-2,4.2908005E-3,3.2838914E-3,8.34704E-3],"split_indices":[2,43,17,7,2,53,67,73,53,106,3,0,0,3,32,2,68,57,0,43,0,43,0,46,62,53,9,0,1,0,0,67,43,10,9,29,67,4,0,43,0,4,68,0,72,7,0,7,10,53,2,12,7,1,8,7,10,2,69,0,0,60,44,9,66,0,0,60,4,44,0,0,0,0,0,2,57,73,48,0,3,4,0,0,0,0,0,0,0,0,66,43,0,48,2,7,67,0,60,0,0,0,0,0,62,0,0,0,0,67,52,9,0,67,9,0,73,0,0,68,0,26,57,9,29,5,0,1,69,0,60,0,0,57,9,44,0,0,0,0,0,7,0,0,0,10,0,2,0,0,0,60,60,62,7,60,12,0,0,67,4,61,7,0,7,0,73,0,0,0,69,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,7.5E2,2.48E2,2.38E2,5.12E2,1.1E1,2.37E2,6.1E1,1.77E2,8E1,4.32E2,6E0,5E0,1.06E2,1.31E2,4.7E1,1.4E1,1.71E2,6E0,7.6E1,4E0,4.25E2,7E0,3E1,7.6E1,8.3E1,4.8E1,2.2E1,2.5E1,6E0,8E0,1.8E1,1.53E2,5.4E1,2.2E1,1.26E2,2.99E2,1.3E1,1.7E1,6.9E1,7E0,7.3E1,1E1,7E0,4.1E1,1.8E1,7E0,8E0,1E1,8.4E1,6.9E1,2.5E1,2.9E1,1E1,1.2E1,1.4E1,1.12E2,1.95E2,1.04E2,6E0,7E0,2.8E1,4.1E1,5E1,2.3E1,6E0,4E0,3E1,1.1E1,1.4E1,4E0,4E0,4E0,6E0,4E0,3.8E1,4.6E1,5E1,1.9E1,6E0,1.9E1,2.5E1,4E0,4E0,6E0,7E0,5E0,7E0,7E0,6E0,1.06E2,1.91E2,4E0,4.8E1,5.6E1,1.9E1,9E0,7E0,3.4E1,5E0,4.5E1,1.4E1,9E0,8E0,2.2E1,6E0,5E0,8E0,6E0,2E1,1.8E1,4.2E1,4E0,1.6E1,3.4E1,6E0,1.3E1,4E0,1.5E1,1.9E1,6E0,5.5E1,5.1E1,1.79E2,1.2E1,4.4E1,4E0,2.1E1,3.5E1,7E0,1.2E1,4E0,5E0,9E0,2.5E1,1.6E1,6E0,1.5E1,5E0,1.1E1,7E0,3.7E1,5E0,1.2E1,4E0,3E1,4E0,9E0,4E0,5E0,1.4E1,2.8E1,2.7E1,2.1E1,3E1,1.19E2,6E1,5E0,7E0,2.2E1,2.2E1,1.3E1,8E0,1.7E1,1.8E1,4E0,8E0,5E0,4E0,5E0,2E1,5E0,1.1E1,2.5E1,1.2E1,2.5E1,5E0,5E0,4E0,1.5E1,1.3E1,2.3E1,4E0,1.5E1,6E0,1.3E1,1.7E1,2.9E1,9E1,5.6E1,4E0,4E0,1.8E1,7E0,1.5E1,9E0,4E0,4E0,4E0,1.1E1,7E0,4E0,4E0,1.2E1,8E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"207","size_leaf_vector":"1"}},{"base_weights":[3.6584307E-3,-8.717372E-2,2.6933727E-1,-1.9764858E-1,1.747211E-2,3.1734458E-1,-1.3559663E-1,-3.3104765E-1,-1.5139028E-1,-4.751485E-2,6.8291664E-2,3.481294E-2,2.7624777E-1,-4.2068493E-1,1.372734E-2,-2.1810003E-1,-3.7600914E-1,-1.3114409E-1,-4.616602E-1,-8.735668E-2,7.129579E-2,4.550107E-2,2.2154155E-1,1.398515E-1,3.7319E-1,-2.5613237E-2,-8.134605E-3,1.4498642E-1,-1.6281003E-1,-1.39393695E-2,-1.4558814E-1,-4.174836E-1,-2.3760873E-1,-2.134523E-1,-5.4366227E-2,-3.4546994E-2,-3.1609958E-1,-1.8754998E-1,-2.8570468E-2,1.05273254E-1,-7.7200104E-3,5.653491E-2,-1.9249128E-1,1.4698611E-1,1.6100287E-2,1.1064811E-1,2.9183999E-2,3.960429E-1,6.3921496E-2,1.2552225E-2,-4.0077963E-3,-1.1661417E-2,-1.8775862E-3,-1.7426285E-1,-1.6715435E-3,-2.9512233E-1,-5.237878E-1,-1.0580909E-1,-1.5359684E-2,-2.5049242E-1,-1.1815903E-1,2.0022953E-2,-7.944116E-2,-5.7165525E-3,-1.7918082E-2,-1.4690425E-1,-3.51875E-1,-5.6904454E-2,7.284781E-2,8.705399E-2,9.951149E-3,-3.8742732E-2,8.637103E-2,-1.6535815E-2,5.2856794E-4,6.1813336E-2,1.1282641E-2,-8.473943E-2,1.3368845E-1,2.6736724E-1,4.7632608E-1,1.1844745E-2,-4.948767E-3,-3.588991E-3,-1.1999852E-2,-1.8307015E-1,-1.8160485E-2,-5.6335413E-1,-1.131592E-2,-4.64755E-4,-8.334148E-3,-1.8143399E-1,-3.07392E-1,-9.175192E-3,-1.6222003E-1,-3.5944086E-2,-2.6765063E-1,-1.9269049E-1,-3.3465356E-2,-2.6381325E-2,-2.879998E-3,-8.2045004E-2,2.8358858E-2,-5.3661563E-3,8.364929E-3,1.1922198E-1,4.408848E-2,7.3120794E-3,-6.1274584E-2,1.4336166E-1,4.445573E-2,-2.9693093E-4,5.1500276E-3,-6.913837E-3,4.6684203E-4,5.759979E-2,1.7818967E-1,3.1480128E-1,5.5637907E-2,5.3607506E-1,3.2572708E-1,-1.31378705E-2,-1.9006245E-3,-2.814211E-2,-1.3204679E-2,-2.0752932E-1,-6.4006355E-4,-3.2659554E-1,-5.3283176E-3,-3.0754057E-3,2.740203E-3,-5.1902775E-3,-9.2632305E-3,-7.218088E-2,3.7500963E-2,-3.2925078E-1,-2.8742468E-3,-1.2656488E-1,-2.4597009E-1,1.3971746E-2,-5.1816422E-3,-1.8467259E-2,-1.2120399E-1,-1.27165755E-2,4.570811E-3,1.8031854E-3,-2.599539E-3,2.1877536E-3,1.3422905E-1,5.5799777E-5,4.85894E-3,-3.0685423E-2,-1.10736955E-2,1.6237536E-1,-1.2248257E-3,-5.7272278E-2,1.0098505E-1,2.0620754E-2,9.620597E-3,2.0337267E-1,-4.1178144E-3,3.6593717E-1,6.0018757E-3,-9.654969E-3,1.2329838E-2,5.579276E-1,1.0524922E-2,7.307919E-3,3.6347908E-1,-1.02076165E-1,-2.4671112E-1,-3.789711E-1,-2.112526E-1,1.8101072E-2,-1.036983E-1,1.4309485E-1,-6.9076917E-3,-1.6559511E-2,-8.123538E-3,-7.3194234E-3,-1.4387994E-3,-1.3026931E-2,-5.4315035E-3,-1.9422622E-3,3.1041286E-3,-5.6895424E-2,3.0730215E-3,-1.4101261E-1,-2.5369686E-3,1.8590864E-3,-2.9361583E-3,4.1070725E-3,8.30705E-3,-5.9224144E-2,4.2153742E-2,1.2688813E-1,2.28272E-1,2.3988217E-2,-1.8472953E-2,1.5314375E-2,7.130514E-2,5.5312112E-2,-3.171402E-2,1.4755993E-1,2.9879907E-1,9.34993E-3,1.8799089E-2,2.1985715E-2,3.28682E-2,8.785764E-3,1.919603E-2,-9.236898E-3,-1.669248E-4,-1.3185396E-2,-7.040399E-3,-9.734367E-3,-1.9539097E-2,-1.2630345E-2,-2.8701501E-3,-4.281962E-3,2.3834284E-3,3.5186842E-4,-6.084418E-3,2.3185445E-3,8.732369E-3,4.232548E-3,-2.45839E-3,2.5006486E-3,-4.720996E-3,-1.9990192E-3,-7.3178234E-3,-9.091291E-4,-6.9615114E-3,4.0126187E-3,-1.3332478E-3,7.975032E-3,2.7829132E-3,1.187323E-2,4.720372E-3,-1.2999049E-3,5.8013718E-3,2.0875838E-3,8.559218E-3,4.879857E-3,6.726875E-4,-3.108075E-3,4.2195295E-4,8.844434E-3,2.327079E-3,1.599773E-2,7.4821156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,51,-1,53,55,57,59,61,-1,63,65,67,69,-1,71,73,75,-1,77,-1,79,81,-1,-1,-1,-1,83,-1,85,87,89,-1,91,93,-1,95,-1,-1,97,99,101,103,105,-1,107,109,-1,-1,111,-1,113,115,117,119,-1,-1,-1,-1,121,-1,123,-1,-1,-1,125,127,129,131,133,135,137,139,-1,-1,141,143,145,-1,147,149,-1,151,153,155,-1,-1,-1,-1,157,159,161,163,165,167,-1,-1,-1,-1,169,-1,171,-1,-1,-1,-1,-1,173,175,177,-1,179,181,183,-1,185,187,189,-1,-1,-1,-1,191,-1,-1,193,-1,195,-1,197,199,201,-1,203,-1,205,-1,-1,-1,207,-1,-1,209,211,213,215,217,219,221,223,225,-1,-1,-1,-1,-1,-1,-1,-1,227,-1,229,-1,-1,-1,-1,-1,231,233,235,237,239,-1,-1,241,243,245,247,249,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5289871E1,9.032387E0,5.2334538E0,2.2947922E0,1.3308935E0,4.076231E0,1.24032E0,4.1058636E-1,1.7234836E0,8.440813E-1,7.792599E-1,0E0,2.8567734E0,2.2948349E-1,4.8704785E-1,1.3798487E-1,2.911272E-1,1.6820073E0,4.7165155E-1,7.7800965E-1,3.821002E-1,5.308244E-1,2.2326982E-1,1.2659254E0,8.7934685E-1,0E0,0E0,3.7593743E-1,8.826588E-2,0E0,5.016467E-2,5.088024E-1,1.8986428E-1,4.2573023E-1,1.7467504E0,0E0,1.3142407E-1,2.7558446E-1,2.484E-1,6.1503947E-2,0E0,5.434621E-1,3.0773142E-1,1.5053433E-1,0E0,4.119885E-1,0E0,1.1019726E0,3.5665295E-1,0E0,0E0,0E0,0E0,9.539741E-2,0E0,2.1345782E-1,1.4887285E-1,6.2460452E-2,0E0,3.0573225E-1,1.8098527E-1,0E0,1.0873787E0,0E0,0E0,2.0650327E-1,5.3549767E-1,1.4594564E-1,1.6178617E-1,4.4499785E-2,0E0,2.1389848E-1,3.399999E-1,0E0,0E0,4.138354E-2,0E0,6.307938E-2,2.6061928E-1,4.6455765E-1,4.7325325E-1,0E0,0E0,0E0,0E0,1.895966E-1,0E0,1.0550308E-1,0E0,0E0,0E0,1.8953872E-1,1.4305449E-1,5.11176E-2,2.9031038E-2,2.9546228E-1,2.9588842E-1,7.1040034E-2,5.009608E-2,0E0,0E0,1.2722299E-1,4.7642812E-2,2.9314958E-2,0E0,1.5213877E-2,4.440259E-2,0E0,2.2186998E-1,2.0133007E-1,4.9458593E-1,0E0,0E0,0E0,0E0,1.6839492E-1,3.5472524E-1,3.21182E-1,6.147865E-1,2.0970535E-1,8.7980986E-2,0E0,0E0,0E0,0E0,1.3182676E-1,0E0,1.8575239E-1,0E0,0E0,0E0,0E0,0E0,2.1244073E-1,1.7449191E-1,2.205038E-2,0E0,4.067844E-2,3.6194026E-2,2.9653106E-2,0E0,7.236607E-2,3.4276187E-2,3.2172166E-2,0E0,0E0,0E0,0E0,1.7073601E-2,0E0,0E0,7.940978E-2,0E0,9.965205E-2,0E0,8.7258524E-1,3.5263324E-1,4.895884E-2,0E0,2.0090568E-1,0E0,1.18709564E-1,0E0,0E0,0E0,2.4076462E-1,0E0,0E0,8.3284855E-2,1.1490561E-1,5.504179E-2,1.2540674E-1,1.3127655E-1,7.898476E-2,1.7243844E-1,3.744553E-2,1.2805308E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.095936E-2,0E0,3.3928603E-2,0E0,0E0,0E0,0E0,0E0,9.469057E-2,3.7424363E-2,1.05436265E-1,3.8950086E-2,1.4120728E-1,0E0,0E0,1.4045513E-1,3.1268362E-2,1.6593087E-2,1.22567415E-1,4.7875762E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,47,47,48,48,53,53,55,55,56,56,57,57,59,59,60,60,62,62,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,77,77,78,78,79,79,80,80,85,85,87,87,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,105,105,106,106,108,108,109,109,110,110,115,115,116,116,117,117,118,118,119,119,120,120,125,125,127,127,133,133,134,134,135,135,137,137,138,138,139,139,141,141,142,142,143,143,148,148,151,151,153,153,155,155,156,156,157,157,159,159,161,161,165,165,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,185,185,187,187,193,193,194,194,195,195,196,196,197,197,200,200,201,201,202,202,203,203,204,204],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,52,-1,54,56,58,60,62,-1,64,66,68,70,-1,72,74,76,-1,78,-1,80,82,-1,-1,-1,-1,84,-1,86,88,90,-1,92,94,-1,96,-1,-1,98,100,102,104,106,-1,108,110,-1,-1,112,-1,114,116,118,120,-1,-1,-1,-1,122,-1,124,-1,-1,-1,126,128,130,132,134,136,138,140,-1,-1,142,144,146,-1,148,150,-1,152,154,156,-1,-1,-1,-1,158,160,162,164,166,168,-1,-1,-1,-1,170,-1,172,-1,-1,-1,-1,-1,174,176,178,-1,180,182,184,-1,186,188,190,-1,-1,-1,-1,192,-1,-1,194,-1,196,-1,198,200,202,-1,204,-1,206,-1,-1,-1,208,-1,-1,210,212,214,216,218,220,222,224,226,-1,-1,-1,-1,-1,-1,-1,-1,228,-1,230,-1,-1,-1,-1,-1,232,234,236,238,240,-1,-1,242,244,246,248,250,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,4.4761734E5,1E0,1.16E2,8.39E2,3.6E1,1.0201538E-1,1E0,3E1,3.3486558E6,3.7951445E6,3.481294E-2,1.3026532E6,7E0,1.1E1,4.2E1,9.558022E7,6.07E2,1E0,1E0,7.89E2,1.7500046E8,3.962617E0,1.0534078E4,1.8545505E2,-2.5613237E-2,-8.134605E-3,4.433E3,2.8E1,-1.39393695E-2,1.5832983E-8,6.0921145E4,7.080933E2,1.852364E6,2.00087E5,-3.4546994E-2,2.1095E4,2.43E2,5.751617E6,6.7723384E7,-7.7200104E-3,4.7563504E7,1.7694E4,3.7788504E7,1.6100287E-2,6.4258E4,2.9183999E-2,1.1293673E3,2.4344263E0,1.2552225E-2,-4.0077963E-3,-1.1661417E-2,-1.8775862E-3,6.101E3,-1.6715435E-3,5E0,1.318849E6,1.7E1,-1.5359684E-2,1E0,1.3809524E0,2.0022953E-2,2.0734E4,-5.7165525E-3,-1.7918082E-2,1.1362371E6,3.304531E3,6.887749E7,4.9E1,1.9668E4,9.951149E-3,3.7974394E-6,4.9390244E0,-1.6535815E-2,5.2856794E-4,3E0,1.1282641E-2,2.0939393E0,8.072743E4,7.7191065E6,1.853776E7,1.1844745E-2,-4.948767E-3,-3.588991E-3,-1.1999852E-2,3.6E1,-1.8160485E-2,7.7E1,-1.131592E-2,-4.64755E-4,-8.334148E-3,3.0416667E0,5.57E2,7.1428573E-1,3.478747E4,1.158E3,1.609E3,9.836066E-1,2.141096E1,-2.6381325E-2,-2.879998E-3,3.6E1,4.1226246E4,1.3529412E0,8.364929E-3,4.33E2,1.3935602E1,7.3120794E-3,1.764442E7,1E0,4.326193E2,-2.9693093E-4,5.1500276E-3,-6.913837E-3,4.6684203E-4,2.4693803E2,4.1325716E7,2E0,1E0,2.53E2,7.15244E5,-1.31378705E-2,-1.9006245E-3,-2.814211E-2,-1.3204679E-2,3.8575E4,-6.4006355E-4,1.7909248E5,-5.3283176E-3,-3.0754057E-3,2.740203E-3,-5.1902775E-3,-9.2632305E-3,1.3867E4,1.3520833E1,1.1939979E8,-2.8742468E-3,1.7171982E6,1.605076E6,1.512E4,-5.1816422E-3,2.6530768E2,8.1407714E-1,1E0,4.570811E-3,1.8031854E-3,-2.599539E-3,2.1877536E-3,2.0518206E3,5.5799777E-5,4.85894E-3,4.699871E2,-1.10736955E-2,1.7642576E8,-1.2248257E-3,1.18033766E5,1.684E0,4.5179688E1,9.620597E-3,1.767635E3,-4.1178144E-3,4.6134964E2,6.0018757E-3,-9.654969E-3,1.2329838E-2,1.6975454E3,1.0524922E-2,7.307919E-3,8E0,3.103E3,1.86619E5,4.0869565E0,4.58E2,1.5152774E2,2.978142E0,1.04E2,2.3266666E0,-1.6559511E-2,-8.123538E-3,-7.3194234E-3,-1.4387994E-3,-1.3026931E-2,-5.4315035E-3,-1.9422622E-3,3.1041286E-3,1.7676923E2,3.0730215E-3,1E1,-2.5369686E-3,1.8590864E-3,-2.9361583E-3,4.1070725E-3,8.30705E-3,1.2E1,2.75E0,4.05E2,1.57E2,1E0,-1.8472953E-2,1.5314375E-2,2.445E3,5.117647E0,2.3266666E0,3.776793E8,5.501002E9,9.34993E-3,1.8799089E-2,2.1985715E-2,3.28682E-2,8.785764E-3,1.919603E-2,-9.236898E-3,-1.669248E-4,-1.3185396E-2,-7.040399E-3,-9.734367E-3,-1.9539097E-2,-1.2630345E-2,-2.8701501E-3,-4.281962E-3,2.3834284E-3,3.5186842E-4,-6.084418E-3,2.3185445E-3,8.732369E-3,4.232548E-3,-2.45839E-3,2.5006486E-3,-4.720996E-3,-1.9990192E-3,-7.3178234E-3,-9.091291E-4,-6.9615114E-3,4.0126187E-3,-1.3332478E-3,7.975032E-3,2.7829132E-3,1.187323E-2,4.720372E-3,-1.2999049E-3,5.8013718E-3,2.0875838E-3,8.559218E-3,4.879857E-3,6.726875E-4,-3.108075E-3,4.2195295E-4,8.844434E-3,2.327079E-3,1.599773E-2,7.4821156E-3],"split_indices":[2,43,6,44,2,0,53,24,3,43,43,0,43,6,8,2,7,2,13,29,2,12,71,4,73,0,0,10,3,0,52,60,4,9,5,0,1,0,60,7,0,7,10,7,0,1,0,4,69,0,0,0,0,9,0,12,9,3,0,8,68,0,44,0,0,43,48,7,0,12,0,53,71,0,0,8,0,68,48,62,62,0,0,0,0,2,0,44,0,0,0,71,10,71,62,2,2,71,61,0,0,10,48,68,0,2,73,0,60,19,70,0,0,0,0,71,66,32,8,8,1,0,0,0,0,9,0,43,0,0,0,0,0,1,73,7,0,60,12,9,0,67,53,26,0,0,0,0,48,0,0,67,0,7,0,48,69,71,0,67,0,4,0,0,0,67,0,0,3,1,12,73,2,67,69,10,68,0,0,0,0,0,0,0,0,67,0,3,0,0,0,0,0,8,68,10,0,23,0,0,2,69,68,12,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.046E3,7.8E2,2.66E2,3.79E2,4.01E2,2.38E2,2.8E1,9.6E1,2.83E2,1.76E2,2.25E2,1.9E1,2.19E2,9E0,1.9E1,2.9E1,6.7E1,2.67E2,1.6E1,1.32E2,4.4E1,1.97E2,2.8E1,9.2E1,1.27E2,5E0,4E0,1.1E1,8E0,1.2E1,1.7E1,5E1,1.7E1,1.28E2,1.39E2,4E0,1.2E1,4.8E1,8.4E1,3.9E1,5E0,1.89E2,8E0,1.9E1,9E0,8.8E1,4E0,1.18E2,9E0,7E0,4E0,4E0,4E0,1.3E1,4E0,2.5E1,2.5E1,8E0,9E0,9.1E1,3.7E1,6E0,1.33E2,4E0,8E0,4E1,8E0,6.6E1,1.8E1,3.5E1,4E0,4.5E1,1.44E2,4E0,4E0,1.1E1,8E0,9E0,7.9E1,4.7E1,7.1E1,4E0,5E0,7E0,6E0,1.3E1,1.2E1,2.1E1,4E0,4E0,4E0,4.3E1,4.8E1,1.1E1,2.6E1,1.09E2,2.4E1,2.8E1,1.2E1,4E0,4E0,5.1E1,1.5E1,1.1E1,7E0,1.9E1,1.6E1,4E0,4.1E1,6E1,8.4E1,5E0,6E0,5E0,4E0,3E1,4.9E1,3.8E1,9E0,4.9E1,2.2E1,7E0,6E0,1.7E1,4E0,3.7E1,6E0,4.3E1,5E0,6E0,5E0,1.3E1,1.3E1,7.3E1,3.6E1,1.8E1,6E0,1.4E1,1.4E1,8E0,4E0,2E1,3.1E1,1E1,5E0,6E0,5E0,4E0,1.5E1,1E1,6E0,3.6E1,5E0,5.4E1,6E0,3E1,5.4E1,2.5E1,5E0,4.5E1,4E0,2.9E1,9E0,4E0,5E0,4.5E1,4E0,5E0,1.7E1,1.1E1,2.6E1,2.8E1,1.5E1,1.9E1,5.4E1,1E1,2.6E1,1.4E1,4E0,1E1,4E0,1E1,4E0,4E0,4E0,1.4E1,6E0,2.3E1,8E0,5E0,5E0,9E0,6E0,2.6E1,1E1,3.7E1,1.7E1,2.5E1,5E0,5E0,4.9E1,1.5E1,1E1,3E1,1.5E1,7E0,2.2E1,3.2E1,1.3E1,5E0,1.2E1,5E0,6E0,1.7E1,9E0,7E0,2.1E1,1E1,5E0,4E0,1.5E1,1.1E1,4.3E1,4E0,6E0,8E0,1.8E1,4E0,1E1,4E0,1.9E1,1.9E1,7E0,6E0,4E0,2.1E1,1.6E1,1.3E1,4E0,1.7E1,8E0,4.1E1,8E0,6E0,9E0,5E0,5E0,2E1,1E1,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"251","size_leaf_vector":"1"}},{"base_weights":[-2.99391E-3,-8.831429E-2,2.4912943E-1,-2.6406372E-1,-1.5137249E-2,6.127956E-1,2.1382841E-1,-3.5943136E-1,-2.1115749E-1,-9.3182884E-2,4.57518E-2,3.1338148E-2,1.6913528E-2,-2.1640071E-1,2.4496906E-1,-2.95355E-1,-4.3520728E-1,-4.5271154E-2,-2.502184E-1,-1.7284678E-1,-4.950818E-2,-7.196651E-2,8.058637E-2,-1.0896794E-3,-1.2654556E-2,1.4805791E-1,3.2795694E-1,-4.214891E-3,-3.1222135E-1,-5.516985E-1,-2.7941078E-1,-2.5595522E-1,7.2868645E-2,-2.76405E-1,1.2859181E-2,1.576933E-2,-2.2300647E-1,-1.21104665E-1,3.9490927E-3,-3.0619962E-2,-2.7614754E-1,6.265724E-2,2.4083194E-1,9.1958314E-2,2.658024E-1,4.0736428E-1,2.054631E-1,-1.0115258E-2,-1.6603213E-2,-1.9671071E-2,-3.2649584E-2,-3.5516988E-3,-1.9103112E-2,-1.4849484E-2,-5.225477E-3,1.8012885E-2,-3.0757098E-2,-2.4766478E-1,-4.3157107E-1,1.0021293E-2,-5.3721853E-3,-6.6432714E-2,3.7694715E-3,-1.8739763E-1,-2.1558696E-2,-1.5851676E-1,4.5616136E-4,-1.1833065E-2,9.114149E-3,8.394522E-3,-1.2428759E-1,-6.4939214E-3,-1.507793E-2,1.2589181E-2,1.1591291E-1,3.2262672E-3,3.0713496E-1,-1.8852325E-2,1.2636074E-1,3.2381943E-1,-6.2581357E-3,3.4137793E-2,3.666432E-1,2.4937008E-1,-4.614155E-3,-8.232715E-2,2.1855973E-3,-2.0621872E-1,-3.7024492E-1,-8.3336495E-3,-5.002674E-1,-5.276657E-3,-2.4777374E-4,-1.8875582E-2,-1.6470881E-1,-1.4460988E-2,-1.3912079E-1,-4.388345E-2,5.3428244E-3,1.0040181E-2,-1.6188158E-1,2.9081505E-2,-5.888991E-3,-1.8223038E-1,1.0149796E-3,2.704253E-2,-1.4224796E-2,1.3864143E-1,-2.0600121E-2,9.547849E-3,1.7254243E-2,3.7968703E-2,2.1366796E-1,3.6912245E-1,5.6471373E-3,1.1089947E-1,3.9944065E-1,1.7773433E-1,2.3478601E-2,-1.4606287E-4,-7.4315965E-3,-2.5133523E-1,-8.815539E-2,-8.5149985E-3,-4.116537E-1,-3.2499995E-2,-1.4355317E-2,-8.8783694E-5,-1.8382254E-1,-4.267908E-4,-1.574555E-1,-3.9355992E-3,1.8129069E-3,-5.0451376E-2,7.8654446E-2,-1.37755E-2,2.8323957E-3,-1.898887E-2,8.068132E-2,-3.8668283E-3,-1.0012353E-2,4.0165115E-2,-9.568277E-3,2.094816E-1,5.4692373E-2,4.9777194E-3,-8.8733844E-2,9.831434E-2,-9.091543E-2,2.5220698E-1,-3.1418859E-3,2.0643573E-2,2.5416845E-1,-2.5801815E-3,1.1802392E-2,3.1811407E-1,4.6263516E-1,7.929093E-2,1.1666393E-2,-6.084328E-3,-2.6380596E-1,1.1817312E-3,-1.1485079E-1,-2.1868676E-2,-1.1414294E-2,-2.099745E-1,-6.963592E-2,-1.1265667E-1,-1.863606E-1,-1.1348648E-1,2.8509617E-2,1.24131896E-1,6.7105573E-3,4.246519E-2,-4.8807436E-3,1.2765835E-1,6.033308E-6,3.4689752E-3,1.201243E-1,1.6168655E-1,1.4664508E-2,7.527092E-2,-5.394145E-3,-6.124588E-4,-6.9031436E-3,5.178071E-2,1.7528737E-1,-7.75603E-4,-8.839531E-3,4.5169266E-3,2.8536078E-1,1.4764176E-2,7.474719E-3,8.617172E-3,1.5907837E-2,1.2407149E-2,2.2620369E-2,-9.172108E-3,7.343055E-3,-1.341973E-2,-8.324688E-3,-6.6140885E-3,-5.8214075E-4,-5.578171E-3,-1.2115863E-2,-6.068335E-4,-5.2912077E-3,-6.5090233E-3,-2.5052743E-4,-5.3579533E-3,-9.900675E-3,-2.3328223E-4,-6.28536E-3,-6.032966E-4,2.768627E-3,2.5625895E-3,8.164715E-3,-1.3723923E-3,3.949327E-3,-2.6384208E-3,5.1668244E-3,3.1676488E-3,8.099799E-3,9.294547E-4,-1.048663E-2,4.782991E-4,7.733759E-3,1.066699E-2,4.6780817E-3,5.370828E-3,-7.2246086E-4,3.9856415E-3,-4.5347452E-4,1.1775024E-2,2.8015731E-3,9.146807E-3,1.6794136E-2,-2.6498754E-3,1.8871386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,87,89,-1,-1,91,-1,93,-1,95,97,99,-1,101,103,-1,-1,105,107,-1,109,-1,111,113,-1,-1,115,117,-1,119,-1,121,123,-1,125,-1,-1,-1,127,-1,129,131,-1,133,135,137,-1,139,-1,141,-1,143,145,-1,-1,147,149,151,-1,153,155,157,-1,-1,-1,159,161,-1,163,-1,-1,-1,165,-1,167,-1,-1,169,171,-1,-1,173,175,-1,-1,177,-1,179,181,-1,183,185,187,189,-1,-1,191,-1,-1,193,195,197,-1,-1,199,-1,201,-1,-1,203,205,207,209,211,213,215,217,219,-1,221,-1,223,225,227,-1,229,-1,-1,-1,231,233,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,237,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2630335E1,1.0104609E1,3.2838192E0,1.0896244E0,2.6464446E0,1.2349701E-1,3.322029E0,2.6475143E-1,9.67103E-1,8.4039116E-1,1.2893082E0,0E0,0E0,1.8205786E-1,1.7851229E0,1.3332558E-1,4.8106194E-1,7.653563E-1,8.440256E-1,8.171563E-1,6.090274E-1,5.993813E-1,6.8019164E-1,0E0,0E0,6.823368E-1,1.1029844E0,0E0,1.1046934E-1,6.993055E-2,4.298731E-1,6.6516876E-2,6.690387E-1,3.8812065E-1,3.4145224E-1,1.0822474E-1,5.025053E-1,3.0932593E-1,2.830429E-1,2.2396141E-1,3.583449E-2,5.824842E-1,2.5065565E-1,1.3245192E0,8.3538723E-1,7.146969E-1,6.9523096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.8308115E-2,4.1137314E-1,1.858468E-1,0E0,0E0,2.5650207E-2,0E0,2.3881245E-1,0E0,1.1227834E-1,9.1903515E-2,2.8388146E-1,0E0,1.2737457E-1,1.5654552E-1,0E0,0E0,5.3639454E-1,3.3174634E-1,0E0,3.6121607E-2,0E0,5.3209555E-1,2.288518E-1,0E0,0E0,5.176811E-1,7.2551584E-1,0E0,6.1662704E-2,0E0,3.7201214E-1,1.0121131E-1,0E0,2.4360752E-1,0E0,0E0,0E0,1.770004E-1,0E0,1.12850785E-1,4.79773E-2,0E0,3.194126E-1,3.5289887E-1,9.826852E-2,0E0,2.604106E-2,0E0,3.4631383E-1,0E0,5.297166E-1,1.494748E-1,0E0,0E0,2.8893167E-1,3.970946E-1,1.0084152E-1,0E0,2.3122764E-1,1.5605354E-1,2.4503076E-1,0E0,0E0,0E0,4.4331312E-2,6.973393E-2,0E0,6.584978E-2,0E0,0E0,0E0,1.3767469E-1,0E0,3.3391714E-2,0E0,0E0,2.0730536E-1,1.1695267E-1,0E0,0E0,1.16320334E-1,7.1879975E-2,0E0,0E0,3.0767363E-1,0E0,2.0146489E-1,1.6050184E-1,0E0,4.8405014E-2,8.0469385E-2,8.611426E-2,1.3082421E-1,0E0,0E0,2.0800054E-2,0E0,0E0,3.5827875E-2,3.0908108E-2,1.2124563E-1,0E0,0E0,5.2469492E-2,0E0,5.0308466E-2,0E0,0E0,1.5959918E-1,2.6398048E-2,5.6761548E-2,2.235037E-2,5.5586487E-2,2.5596896E-2,6.96502E-2,4.5569938E-2,9.6188895E-2,0E0,2.1779373E-2,0E0,2.8466728E-1,1.6952386E-1,1.271683E-1,0E0,1.471216E-1,0E0,0E0,0E0,3.6962986E-2,6.666455E-2,0E0,0E0,0E0,9.8748684E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.4035225E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,56,56,57,57,58,58,61,61,63,63,65,65,66,66,67,67,69,69,70,70,73,73,74,74,76,76,78,78,79,79,82,82,83,83,85,85,87,87,88,88,90,90,94,94,96,96,97,97,99,99,100,100,101,101,103,103,105,105,107,107,108,108,111,111,112,112,113,113,115,115,116,116,117,117,121,121,122,122,124,124,128,128,130,130,133,133,134,134,137,137,138,138,141,141,143,143,144,144,146,146,147,147,148,148,149,149,152,152,155,155,156,156,157,157,160,160,162,162,165,165,166,166,167,167,168,168,169,169,170,170,171,171,172,172,173,173,175,175,177,177,178,178,179,179,181,181,185,185,186,186,190,190,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,88,90,-1,-1,92,-1,94,-1,96,98,100,-1,102,104,-1,-1,106,108,-1,110,-1,112,114,-1,-1,116,118,-1,120,-1,122,124,-1,126,-1,-1,-1,128,-1,130,132,-1,134,136,138,-1,140,-1,142,-1,144,146,-1,-1,148,150,152,-1,154,156,158,-1,-1,-1,160,162,-1,164,-1,-1,-1,166,-1,168,-1,-1,170,172,-1,-1,174,176,-1,-1,178,-1,180,182,-1,184,186,188,190,-1,-1,192,-1,-1,194,196,198,-1,-1,200,-1,202,-1,-1,204,206,208,210,212,214,216,218,220,-1,222,-1,224,226,228,-1,230,-1,-1,-1,232,234,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,238,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,5.6E1,7.7E1,8.1E2,6.7321223E-1,9.183432E4,1.139605E6,9.935484E-1,1E0,4.539777E5,3.1338148E-2,1.6913528E-2,3.63E2,5.97E3,5E0,1E0,3.06E2,4.2210345E2,5.714286E-1,6.072304E5,1.9E1,5.405423E6,-1.0896794E-3,-1.2654556E-2,2.8530578E6,8.992806E-4,-4.214891E-3,1E0,2.3060504E7,7.042942E7,5.4E1,2.00087E5,4.6463413E0,9.566785E5,5.44E2,3.7568388E0,4.39776E6,1.9E1,2E1,2.9E1,4.5432812E2,1E0,1.4598765E0,1E0,1.0918E4,3.3571344E7,-1.0115258E-2,-1.6603213E-2,-1.9671071E-2,-3.2649584E-2,-3.5516988E-3,-1.9103112E-2,-1.4849484E-2,-5.225477E-3,1.8012885E-2,9.14E2,8.499432E5,2.49E2,1.0021293E-2,-5.3721853E-3,1.3026532E6,3.7694715E-3,1E1,-2.1558696E-2,8.2765434E1,1.185E3,3.6507E4,9.114149E-3,2.3719013E0,1.445E3,-6.4939214E-3,-1.507793E-2,2.41E3,1E0,3.2262672E-3,2.47E2,-1.8852325E-2,5.723202E5,1.6864318E11,-6.2581357E-3,3.4137793E-2,1.0322473E8,2.9622424E3,-4.614155E-3,6.2478176E7,2.1855973E-3,1.19111115E2,2E0,-8.3336495E-3,7.3E1,-5.276657E-3,-2.4777374E-4,-1.8875582E-2,1.6666666E0,-1.4460988E-2,4.1719616E5,6.07E2,5.3428244E-3,1.2463E4,3.6977E4,2.6369516E5,-5.888991E-3,3.074398E0,1.0149796E-3,1.81E2,-1.4224796E-2,1.9545455E0,1.3358E4,9.547849E-3,1.7254243E-2,4.4522205E6,9.4163445E1,4.026E3,5.6471373E-3,1.034073E6,3.7951445E6,2.234478E3,2.3478601E-2,-1.4606287E-4,-7.4315965E-3,6.48334E5,1.9166666E0,-8.5149985E-3,3.3025316E5,-3.2499995E-2,-1.4355317E-2,-8.8783694E-5,2.866353E2,-4.267908E-4,9.966666E0,-3.9355992E-3,1.8129069E-3,2.3112903E2,4.9E4,-1.37755E-2,2.8323957E-3,1.281479E5,2.3625128E6,-3.8668283E-3,-1.0012353E-2,9.750085E6,-9.568277E-3,1.11791E5,1.481649E8,4.9777194E-3,2E0,8.620714E2,1.1E1,2.8608696E1,-3.1418859E-3,2.0643573E-2,9.6732026E-1,-2.5801815E-3,1.1802392E-2,1.1074919E0,1.3586957E1,1.5E1,1.1666393E-2,-6.084328E-3,2.026453E6,1.1817312E-3,3.221198E0,-2.1868676E-2,-1.1414294E-2,1.3556826E6,1.9184E4,7.7E1,1.625E0,5.4E0,2.4E1,5.75E2,3.5834332E0,1.1497368E-2,-4.8807436E-3,3.9123364E2,6.033308E-6,9.575663E6,1.088E3,1.9197379E9,1.4664508E-2,1E0,-5.394145E-3,-6.124588E-4,-6.9031436E-3,3.0594551E1,1.7172844E7,-7.75603E-4,-8.839531E-3,4.5169266E-3,1.3748E4,1.4764176E-2,7.474719E-3,8.617172E-3,1.5907837E-2,1.2407149E-2,2.2620369E-2,5.32E2,7.343055E-3,-1.341973E-2,-8.324688E-3,-6.6140885E-3,-5.8214075E-4,-5.578171E-3,-1.2115863E-2,-6.068335E-4,-5.2912077E-3,-6.5090233E-3,-2.5052743E-4,-5.3579533E-3,-9.900675E-3,-2.3328223E-4,-6.28536E-3,-6.032966E-4,2.768627E-3,2.5625895E-3,8.164715E-3,-1.3723923E-3,3.949327E-3,-2.6384208E-3,5.1668244E-3,3.1676488E-3,8.099799E-3,9.294547E-4,-1.048663E-2,4.782991E-4,7.733759E-3,1.066699E-2,4.6780817E-3,5.370828E-3,-7.2246086E-4,3.9856415E-3,-4.5347452E-4,1.1775024E-2,2.8015731E-3,9.146807E-3,1.6794136E-2,-2.6498754E-3,1.8871386E-3],"split_indices":[2,43,0,44,2,53,43,46,68,29,43,0,0,0,2,0,26,2,67,68,43,3,43,0,0,43,72,0,24,7,60,2,5,68,60,12,68,9,3,8,0,4,29,69,6,9,66,0,0,0,0,0,0,0,0,0,2,60,2,0,0,43,0,3,0,4,44,1,0,53,2,0,0,2,6,0,0,0,43,46,0,0,5,67,0,7,0,67,8,0,0,0,0,0,69,0,60,2,0,1,12,66,0,71,0,8,0,68,9,0,0,62,73,2,0,9,43,4,0,0,0,46,68,0,62,0,0,0,67,0,73,0,0,67,12,0,0,62,60,0,0,9,0,1,12,0,8,4,3,73,0,0,68,0,0,68,73,3,0,0,9,0,68,0,0,60,1,0,68,73,0,2,69,53,0,4,0,9,44,46,0,30,0,0,0,73,60,0,0,0,44,0,0,0,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E3,7.85E2,2.65E2,2.3E2,5.55E2,2.2E1,2.43E2,8E1,1.5E2,2.43E2,3.12E2,1.6E1,6E0,1.6E1,2.27E2,4.6E1,3.4E1,2.9E1,1.21E2,8.5E1,1.58E2,7.1E1,2.41E2,4E0,1.2E1,1.06E2,1.21E2,4E0,4.2E1,1.8E1,1.6E1,1E1,1.9E1,1.1E2,1.1E1,1.8E1,6.7E1,6.7E1,9.1E1,6E1,1.1E1,2.18E2,2.3E1,7.3E1,3.3E1,7.2E1,4.9E1,1.6E1,2.6E1,1.2E1,6E0,7E0,9E0,6E0,4E0,4E0,1.5E1,9.5E1,1.5E1,4E0,7E0,8E0,1E1,6E1,7E0,5.1E1,1.6E1,8.5E1,6E0,4.3E1,1.7E1,4E0,7E0,1.13E2,1.05E2,7E0,1.6E1,4E0,6.9E1,2.9E1,4E0,6E0,6.6E1,4.3E1,6E0,9E0,6E0,7.3E1,2.2E1,4E0,1.1E1,4E0,4E0,4E0,5.6E1,4E0,4.7E1,1.2E1,4E0,7.5E1,1E1,3.8E1,5E0,1.2E1,5E0,1.09E2,4E0,9E1,1.5E1,8E0,8E0,3.5E1,3.4E1,2.3E1,6E0,8E0,5.8E1,3.5E1,8E0,5E0,4E0,5.2E1,2.1E1,5E0,1.7E1,4E0,7E0,6E0,5E1,6E0,4.1E1,8E0,4E0,4E1,3.5E1,6E0,4E0,2E1,1.8E1,4E0,8E0,1.04E2,5E0,4.8E1,4.2E1,5E0,1E1,2.4E1,1.1E1,3E1,4E0,1.2E1,1.1E1,4E0,4E0,2.8E1,3E1,1.6E1,1.9E1,6E0,4.6E1,4E0,1.7E1,1.1E1,6E0,4E1,1E1,1.8E1,2.3E1,2.2E1,1.8E1,2.1E1,1.4E1,1.2E1,8E0,1.1E1,7E0,7.2E1,3.2E1,3.5E1,1.3E1,3.8E1,4E0,5E0,5E0,1.6E1,8E0,7E0,4E0,6E0,2.4E1,5E0,6E0,6E0,2.2E1,5E0,2.5E1,8E0,8E0,3.3E1,1.3E1,1.3E1,4E0,1.6E1,2.4E1,5E0,5E0,1.4E1,4E0,8E0,1.5E1,4E0,1.8E1,8E0,1E1,1E1,1.1E1,1E1,4E0,5E0,7E0,6E0,5E0,6.8E1,4E0,1E1,2.2E1,1.5E1,2E1,2.6E1,1.2E1,1E1,6E0,4E0,4E0,1.3E1,1.1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"239","size_leaf_vector":"1"}},{"base_weights":[8.385376E-3,-8.538872E-2,2.0839147E-1,-2.301312E-1,-1.4239848E-2,-3.2906845E-1,2.3747429E-1,-3.666555E-1,-1.7427713E-1,-7.8797385E-2,6.739256E-2,-3.815999E-1,-5.852169E-3,1.3782695E-1,3.538121E-1,-2.868886E-1,-4.448873E-1,-1.8602254E-2,-1.5831576E-1,-6.749313E-2,-2.9546818E-2,4.5568712E-2,2.4685593E-1,-2.1202961E-2,-1.15376385E-2,2.5929764E-1,8.518831E-2,7.612947E-1,3.0647954E-1,-3.0748403E-1,-4.703944E-3,-2.2583077E-2,-1.0368227E-2,-3.2264754E-2,-2.0078294E-1,-1.0796738E-1,3.296368E-2,5.540572E-4,9.257787E-2,3.8242887E-3,2.7871877E-1,1.3354308E-3,2.7870327E-1,6.6096395E-2,1.9891214E-2,2.4160905E-2,4.2729717E-2,3.4492496E-1,3.5163887E-2,-1.5100135E-2,-6.533194E-3,-1.4553766E-1,2.7664334E-2,-2.3006064E-1,-9.171461E-2,-1.8874885E-1,-7.331491E-2,-2.0556806E-1,6.499129E-2,-4.755107E-2,5.7392977E-2,1.0381096E-1,-5.5044745E-3,1.6947904E-1,1.705334E-2,2.934425E-1,4.5807837E-3,8.473108E-2,-1.0542178E-2,3.176181E-1,2.441365E-2,-9.526243E-3,1.2529123E-1,-4.9157947E-4,-1.9843249E-1,1.08595295E-2,-1.8624386E-2,-1.8568096E-1,-3.2673618E-1,3.4390914E-3,-1.4528327E-1,-1.2332242E-1,-2.2360787E-1,-2.8357996E-2,-1.4377968E-1,-5.0849398E-3,-1.2009749E-2,-4.880094E-2,8.863702E-2,-1.558001E-2,-1.5282294E-1,-2.546018E-2,9.690646E-2,1.2702686E-1,-2.4658425E-2,3.6861175E-3,9.977854E-3,2.113888E-1,1.599295E-2,-6.0455836E-2,1.0545327E-1,4.6006906E-1,2.690929E-1,2.036041E-1,-1.8816956E-3,-4.8382836E-3,-1.1247085E-2,-6.8514557E-3,2.563649E-2,-2.2101967E-1,-9.688479E-2,-2.7479225E-1,-2.13782E-2,-1.68762E-1,-1.8985339E-3,-1.1184934E-3,-1.7114916E-1,-1.5763903E-1,-2.7348912E-1,-4.3729156E-2,7.7849017E-3,-9.073622E-2,-2.73696E-1,-4.9986714E-3,1.3226691E-3,6.71694E-2,9.53378E-3,-8.465152E-2,2.3370089E-2,-9.30596E-3,-2.0551227E-3,-6.669334E-2,2.1946914E-3,1.2639599E-1,-3.5028288E-3,1.0315654E-1,1.0591707E-2,-8.797705E-2,4.939164E-3,1.6794829E-2,1.4658184E-1,8.65815E-2,-1.5820373E-2,1.8924217E-1,4.5217186E-2,1.019785E-2,2.3627738E-2,2.0017476E-1,3.3111122E-1,1.4480535E-2,2.450834E-3,7.2311927E-3,-2.2939708E-2,-1.0854046E-1,-2.4612446E-1,-9.645308E-6,-1.2104075E-1,-4.1258195E-3,-3.24094E-1,-1.0412486E-2,-4.6051564E-3,-1.02068875E-2,-4.148843E-3,-2.1493214E-3,-1.8824986E-1,-1.4288767E-2,-5.4772226E-3,-1.3185384E-2,-1.3305894E-1,-1.4188997E-1,-1.4728785E-2,-2.001718E-2,-1.665338E-1,9.54931E-2,2.9355342E-2,-6.8681487E-3,-4.646384E-4,-4.583179E-3,5.57374E-2,-3.7209212E-4,-5.7158275E-3,4.124596E-2,1.7703688E-1,1.1909425E-1,-1.7003579E-3,-7.177065E-4,-6.1760047E-3,1.02022635E-2,8.7373115E-2,-1.6423877E-3,8.035295E-3,2.1612059E-1,2.490472E-4,7.457233E-3,9.858267E-3,2.7113795E-1,1.2643786E-1,2.2591102E-1,4.1052514E-1,-4.1065407E-3,5.6956057E-4,-2.134921E-3,-7.4670273E-3,-1.2298635E-2,-7.2204466E-3,-6.5089376E-3,-2.2728448E-3,-1.7132578E-2,-9.439976E-3,-1.0521625E-2,-4.81459E-3,-2.1480166E-3,1.7590827E-3,-6.1685685E-5,-1.1617987E-2,-8.336323E-3,-3.0561646E-3,3.0857178E-3,-2.6728727E-3,-3.6108214E-3,-9.81382E-3,5.9812306E-4,5.0489143E-3,-4.0784515E-3,3.1945214E-3,1.0648882E-3,5.0846254E-3,5.369802E-3,-8.4117777E-4,2.8404922E-3,9.7217085E-3,6.401373E-3,-5.817069E-4,6.1227125E-3,7.2585413E-4,4.5768563E-3,1.3541119E-2,-2.2696569E-3,2.8209276E-3,1.4939151E-2,4.808647E-3,8.884772E-3,1.6050225E-3,1.2984555E-2,6.1633885E-3,2.1035686E-2,8.135332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,35,-1,37,39,-1,-1,41,43,45,47,49,-1,-1,-1,51,53,55,57,59,61,-1,63,-1,65,67,-1,-1,-1,69,71,-1,-1,73,75,77,79,81,83,85,87,89,91,93,-1,95,-1,97,-1,99,-1,101,-1,-1,103,-1,105,-1,107,109,111,-1,113,115,117,119,121,-1,-1,123,125,127,129,131,133,135,137,-1,-1,139,-1,141,143,145,147,149,-1,-1,-1,-1,151,153,155,157,-1,159,-1,-1,161,163,165,167,-1,169,171,-1,-1,173,-1,175,177,-1,-1,179,-1,181,-1,183,-1,185,-1,-1,187,189,-1,191,193,-1,-1,195,197,-1,-1,-1,199,201,203,-1,205,-1,207,-1,-1,-1,-1,-1,209,-1,-1,211,213,215,217,-1,219,221,223,-1,-1,-1,225,-1,-1,227,229,231,-1,-1,-1,-1,233,-1,-1,235,-1,-1,237,239,241,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8979494E1,7.0922275E0,5.126911E0,1.6670742E0,2.4453144E0,1.324079E-1,3.51021E0,2.7798653E-1,5.3042316E-1,1.5986202E0,7.9165006E-1,1.7974257E-2,0E0,1.0457177E0,2.4861736E0,1.0209775E-1,1.905551E-1,0E0,8.1788826E-1,1.0403669E0,0E0,3.8931125E-1,9.19435E-2,0E0,0E0,2.1548891E-1,7.495988E-1,6.2181473E-2,1.32302E0,5.347538E-2,0E0,0E0,0E0,2.755599E-1,3.4612656E-1,4.956305E-1,5.7869065E-1,2.624822E-1,2.264179E-1,0E0,1.1521256E-1,0E0,9.263992E-2,6.3901997E-1,0E0,0E0,0E0,3.9342785E-1,3.956443E-1,0E0,0E0,9.7783685E-2,2.635795E-1,3.254695E-1,2.4223363E-1,9.558189E-2,4.0457904E-1,1.8135667E-2,1.8191183E-1,1.72633E-1,1.4651519E-1,2.5963002E-1,0E0,2.4470925E-2,0E0,1.0855198E-1,0E0,3.2829434E-1,0E0,5.6744385E-1,0E0,0E0,1.7731379E-1,0E0,1.6686827E-2,0E0,1.3581477E-1,1.7941499E-1,6.358719E-2,0E0,4.1989148E-2,9.540084E-2,6.8768024E-2,2.4945322E-1,3.1972635E-1,0E0,0E0,5.8288127E-2,1.2514013E-1,1.12282336E-1,5.4234266E-2,4.8297122E-2,1.6239232E-1,1.5449905E-1,1.2603638E-1,0E0,0E0,1.340515E-1,0E0,6.101183E-1,4.658718E-1,1.8214226E-1,2.6283216E-1,1.3219628E-1,0E0,0E0,0E0,0E0,1.1803363E-1,1.0035491E-1,4.6680763E-2,1.72243E-1,0E0,4.0116966E-2,0E0,0E0,3.6854923E-2,4.9065024E-2,6.316447E-2,2.0288068E-1,0E0,1.4176857E-1,1.6726506E-1,0E0,0E0,4.860638E-2,0E0,7.012955E-2,1.1264998E-1,0E0,0E0,3.244455E-2,0E0,1.0481188E-1,0E0,1.4045393E-1,0E0,3.1033158E-2,0E0,0E0,4.6649694E-2,1.12860635E-1,0E0,1.9155884E-1,2.2849253E-1,0E0,0E0,1.6978943E-1,2.33078E-1,0E0,0E0,0E0,3.3658046E-2,2.5619522E-2,2.3350716E-2,0E0,1.5884325E-2,0E0,1.9402266E-2,0E0,0E0,0E0,0E0,0E0,1.9413322E-2,0E0,0E0,9.909747E-2,2.9603454E-1,5.230266E-2,6.011582E-2,0E0,2.3953825E-2,2.932635E-2,1.09083965E-1,0E0,0E0,0E0,3.652709E-2,0E0,0E0,5.1156364E-2,4.8802495E-2,1.4273459E-1,0E0,0E0,0E0,0E0,3.0277222E-2,0E0,0E0,2.7733946E-1,0E0,0E0,1.358888E-1,1.3211071E-1,1.20173424E-1,5.94247E-2,1.4038372E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,33,33,34,34,35,35,36,36,37,37,38,38,40,40,42,42,43,43,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,65,65,67,67,69,69,72,72,74,74,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,97,97,99,99,100,100,101,101,102,102,103,103,108,108,109,109,110,110,111,111,113,113,116,116,117,117,118,118,119,119,121,121,122,122,125,125,127,127,128,128,131,131,133,133,135,135,137,137,140,140,141,141,143,143,144,144,147,147,148,148,152,152,153,153,154,154,156,156,158,158,164,164,167,167,168,168,169,169,170,170,172,172,173,173,174,174,178,178,181,181,182,182,183,183,188,188,191,191,194,194,195,195,196,196,197,197,198,198],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,36,-1,38,40,-1,-1,42,44,46,48,50,-1,-1,-1,52,54,56,58,60,62,-1,64,-1,66,68,-1,-1,-1,70,72,-1,-1,74,76,78,80,82,84,86,88,90,92,94,-1,96,-1,98,-1,100,-1,102,-1,-1,104,-1,106,-1,108,110,112,-1,114,116,118,120,122,-1,-1,124,126,128,130,132,134,136,138,-1,-1,140,-1,142,144,146,148,150,-1,-1,-1,-1,152,154,156,158,-1,160,-1,-1,162,164,166,168,-1,170,172,-1,-1,174,-1,176,178,-1,-1,180,-1,182,-1,184,-1,186,-1,-1,188,190,-1,192,194,-1,-1,196,198,-1,-1,-1,200,202,204,-1,206,-1,208,-1,-1,-1,-1,-1,210,-1,-1,212,214,216,218,-1,220,222,224,-1,-1,-1,226,-1,-1,228,230,232,-1,-1,-1,-1,234,-1,-1,236,-1,-1,238,240,242,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.194E3,2.0976269E5,1E0,7.7E1,9.31E2,1.02006085E-1,1.087521E3,1.139605E6,3.07E2,5.433518E0,3.7951445E6,2E1,-5.852169E-3,8E0,2.670351E6,2.0376764E-7,7.7983914E4,-1.8602254E-2,1.4339622E0,2.0766992E6,-2.9546818E-2,2.437372E6,5.2692295E6,-2.1202961E-2,-1.15376385E-2,1.6869704E6,3.3778846E0,2.480065E6,2.3876712E7,3.1530054E0,-4.703944E-3,-2.2583077E-2,-1.0368227E-2,1.20267E5,8.57E2,4.33E2,1.1127184E-5,4.4970587E2,1E0,3.8242887E-3,7.82261E6,1.3354308E-3,2.8827406E5,1.0428572E1,1.9891214E-2,2.4160905E-2,4.2729717E-2,6.701152E0,2.9925186E-2,-1.5100135E-2,-6.533194E-3,5.445545E-1,1.4806053E4,7.485264E5,9.237895E0,1.6923077E0,4.066351E0,5E0,4.92E2,3E0,1E0,2.9789968E1,-5.5044745E-3,3.5364E4,1.705334E-2,5.691698E2,4.5807837E-3,1.4598765E0,-1.0542178E-2,1.0717949E1,2.441365E-2,-9.526243E-3,6.747343E8,-4.9157947E-4,1.0795E4,1.08595295E-2,3.06E2,1.7915686E2,1.3999298E5,3.4390914E-3,1.363176E7,2.8E1,1.7277137E4,1.97525E1,1.9197379E9,-5.0849398E-3,-1.2009749E-2,4.1973075E6,6.2521395E6,2.5084135E6,1.1362371E6,4.285714E0,3.7241504E0,7.955228E9,3.6966666E2,3.6861175E-3,9.977854E-3,1.2375429E8,1.599295E-2,1.9495455E6,4.159558E0,3.0238653E8,1.6466942E3,6.072304E5,-1.8816956E-3,-4.8382836E-3,-1.1247085E-2,-6.8514557E-3,9.384615E0,1.00019E5,2.11429E5,1.26E2,-2.13782E-2,6.829817E4,-1.8985339E-3,-1.1184934E-3,6.5998E4,2.6845297E5,1.7907799E2,2.3695534E9,7.7849017E-3,1.345672E6,3.4402E4,-4.9986714E-3,1.3226691E-3,1.5016073E4,9.53378E-3,5.3871745E-1,1E0,-9.30596E-3,-2.0551227E-3,1.01E2,2.1946914E-3,1.98E2,-3.5028288E-3,3.8435754E8,1.0591707E-2,2.4545455E0,4.939164E-3,1.6794829E-2,4.1290735E2,9.8933E4,-1.5820373E-2,2.4911E4,3.3382E4,1.019785E-2,2.3627738E-2,4.7837E4,1.5416006E5,1.4480535E-2,2.450834E-3,7.2311927E-3,1.8711711E2,8.657E2,1.02166664E2,-9.645308E-6,6.07E2,-4.1258195E-3,2.0262831E2,-1.0412486E-2,-4.6051564E-3,-1.02068875E-2,-4.148843E-3,-2.1493214E-3,9.917037E2,-1.4288767E-2,-5.4772226E-3,6.83E2,1.5043378E0,5.2083335E0,7.8E1,-2.001718E-2,8E0,1.09333336E2,1.3478261E1,-6.8681487E-3,-4.646384E-4,-4.583179E-3,4.4091418E4,-3.7209212E-4,-5.7158275E-3,2.28E4,4.6991256E7,1E0,-1.7003579E-3,-7.177065E-4,-6.1760047E-3,1.02022635E-2,3.1993368E6,-1.6423877E-3,8.035295E-3,5.6440186E2,2.490472E-4,7.457233E-3,7.1E1,1E0,2.72E0,3.09627E5,8.847283E8,-4.1065407E-3,5.6956057E-4,-2.134921E-3,-7.4670273E-3,-1.2298635E-2,-7.2204466E-3,-6.5089376E-3,-2.2728448E-3,-1.7132578E-2,-9.439976E-3,-1.0521625E-2,-4.81459E-3,-2.1480166E-3,1.7590827E-3,-6.1685685E-5,-1.1617987E-2,-8.336323E-3,-3.0561646E-3,3.0857178E-3,-2.6728727E-3,-3.6108214E-3,-9.81382E-3,5.9812306E-4,5.0489143E-3,-4.0784515E-3,3.1945214E-3,1.0648882E-3,5.0846254E-3,5.369802E-3,-8.4117777E-4,2.8404922E-3,9.7217085E-3,6.401373E-3,-5.817069E-4,6.1227125E-3,7.2585413E-4,4.5768563E-3,1.3541119E-2,-2.2696569E-3,2.8209276E-3,1.4939151E-2,4.808647E-3,8.884772E-3,1.6050225E-3,1.2984555E-2,6.1633885E-3,2.1035686E-2,8.135332E-3],"split_indices":[2,43,17,44,2,57,67,46,1,68,43,6,0,3,66,52,48,0,68,43,0,9,60,0,0,60,57,1,60,68,0,0,0,9,2,2,57,67,79,0,43,0,48,68,0,0,0,50,72,0,0,68,43,60,71,68,69,8,2,68,29,73,0,9,0,67,0,69,0,71,0,0,47,0,9,0,2,4,43,0,9,10,66,73,46,0,0,43,43,60,43,69,68,46,67,0,0,7,0,43,69,7,67,43,0,0,0,0,73,5,9,10,0,43,0,0,12,43,67,46,0,9,9,0,0,62,0,57,29,0,0,0,0,10,0,7,0,68,0,0,70,1,0,10,9,0,0,9,48,0,0,0,4,48,67,0,2,0,67,0,0,0,0,0,48,0,0,2,69,69,10,0,8,4,73,0,0,0,48,0,0,1,7,19,0,0,0,0,43,0,0,4,0,0,8,6,69,9,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,6.88E2,3.22E2,2.26E2,4.62E2,1.6E1,3.06E2,6.4E1,1.62E2,2.58E2,2.04E2,1.2E1,4E0,1.66E2,1.4E2,3.4E1,3E1,9E0,1.53E2,2.54E2,4E0,1.83E2,2.1E1,6E0,6E0,4.9E1,1.17E2,1.3E1,1.27E2,3E1,4E0,2.4E1,6E0,3.9E1,1.14E2,1.81E2,7.3E1,9.4E1,8.9E1,4E0,1.7E1,4E0,4.5E1,1.12E2,5E0,7E0,6E0,1.11E2,1.6E1,2.6E1,4E0,1.3E1,2.6E1,8.9E1,2.5E1,5.3E1,1.28E2,8E0,6.5E1,5.1E1,4.3E1,8.5E1,4E0,9E0,8E0,4.1E1,4E0,1.06E2,6E0,9.9E1,1.2E1,4E0,1.2E1,4E0,9E0,4E0,2.2E1,6.3E1,2.6E1,6E0,1.9E1,2E1,3.3E1,7.9E1,4.9E1,4E0,4E0,1.1E1,5.4E1,4E1,1.1E1,1.4E1,2.9E1,7.2E1,1.3E1,4E0,5E0,1.8E1,2.3E1,1.3E1,9.3E1,2.3E1,7.6E1,8E0,4E0,4E0,5E0,5E0,1.7E1,4.4E1,1.9E1,2.1E1,5E0,1.5E1,4E0,7E0,1.3E1,1.6E1,1.7E1,7.4E1,5E0,3.6E1,1.3E1,6E0,5E0,4.7E1,7E0,1.4E1,2.6E1,7E0,4E0,9E0,5E0,2.5E1,4E0,6E1,1.2E1,9E0,4E0,4E0,1.4E1,9E0,4E0,3.8E1,5.5E1,5E0,1.8E1,3.8E1,3.8E1,4E0,4E0,4E0,1.3E1,9E0,3.5E1,4E0,1.5E1,5E0,1.6E1,7E0,8E0,7E0,6E0,4E0,1.2E1,1.3E1,4E0,5.6E1,1.8E1,2.1E1,1.5E1,4E0,9E0,2.6E1,2.1E1,7E0,7E0,5E0,2.1E1,5E0,4E0,1E1,1.5E1,5.4E1,6E0,4E0,5E0,5E0,9E0,4E0,5E0,3.3E1,5E0,1.2E1,4.3E1,1.8E1,2E1,1.8E1,2E1,4E0,9E0,5E0,4E0,2.7E1,8E0,1.1E1,4E0,1E1,6E0,7E0,5E0,3.4E1,2.2E1,9E0,9E0,1.3E1,8E0,5E0,1E1,4E0,5E0,4E0,2.2E1,5E0,1.6E1,1.4E1,7E0,4E0,6E0,4E0,1.1E1,4.7E1,7E0,5E0,4E0,1.4E1,1.9E1,2E1,2.3E1,1.3E1,5E0,1.1E1,9E0,1E1,8E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"247","size_leaf_vector":"1"}},{"base_weights":[4.4282814E-3,-7.089756E-2,2.4218602E-1,-2.0437197E-1,-8.261404E-3,2.8415385E-1,-8.598903E-2,-1.8150964E-1,-4.5935646E-1,-9.294011E-2,5.2365724E-2,1.0600522E-1,3.4112406E-1,-1.5499134E-1,9.440856E-3,-1.9990972E-1,1.657073E-2,-2.907893E-2,-1.5777435E-2,-2.0898332E-1,-5.925382E-2,2.0580392E-2,1.360249E-1,3.554532E-2,2.1839052E-1,4.0324628E-1,1.7871171E-1,-1.398593E-2,-2.9483128E-1,-3.4354174E-1,-1.600544E-1,-1.3231076E-1,-2.5994238E-1,-8.98192E-2,2.8414747E-2,7.378349E-2,-6.5065715E-3,8.728703E-2,2.7384022E-1,9.920785E-2,-1.16205495E-2,4.738623E-3,2.4817392E-1,4.8966122E-1,3.1450874E-1,2.5352243E-1,-4.2203434E-2,-6.1995494E-3,4.9995813E-3,-1.6558258E-2,-7.9238815E-3,-4.6093622E-1,-2.9160646E-1,2.0438117E-1,-1.8067156E-1,-7.6184296E-3,-1.7975752E-3,-1.8328145E-1,-1.4750147E-2,-1.8464176E-1,-5.0108705E-2,9.0700425E-2,-3.2459646E-2,9.518787E-2,-3.891413E-2,-1.58949E-2,7.5983666E-3,-6.7246966E-2,1.1015578E-1,1.1786991E-1,1.8202102E-2,1.3118407E-1,1.3573053E-3,-4.3895687E-3,5.8842584E-2,5.5208523E-3,1.2986845E-2,3.9819297E-1,5.9044045E-1,4.2621452E-1,2.0997998E-1,1.5474588E-1,2.6790693E-2,-6.1836056E-3,2.5626253E-3,-1.3171322E-2,-2.4725582E-2,-7.5487415E-3,-3.0701473E-1,7.7513396E-5,1.8791415E-2,-2.0266195E-1,-3.2269064E-2,-1.1578427E-2,-5.7540867E-3,-7.8102365E-2,-3.1165498E-1,-6.324879E-2,5.5606193E-3,-2.0883323E-3,1.3096264E-1,2.9990165E-3,-7.7523096E-3,1.1098491E-1,-2.3818745E-3,-7.921826E-3,4.5830626E-3,-7.313378E-2,3.0233769E-2,4.5355887E-4,-6.0457014E-3,-5.4546865E-3,1.3486981E-1,-7.790265E-5,8.612825E-3,8.616152E-3,2.9208383E-3,-1.06385596E-4,4.1441983E-3,2.0027258E-2,1.0591589E-2,1.7179692E-2,2.9448232E-2,4.8606926E-1,1.9264043E-3,1.5037008E-1,1.3138838E-2,-4.5302436E-3,9.1990195E-3,-1.5037169E-2,-8.121841E-3,-1.8360811E-1,-3.5809505E-1,2.2791203E-2,-1.2045073E-1,-1.07284226E-1,1.9668234E-3,-1.685911E-3,-4.2326507E-1,-4.998092E-2,-9.597459E-3,6.3006915E-2,9.610854E-3,-3.9417487E-2,2.7359598E-3,1.3972326E-1,5.6301165E-2,-3.9379608E-2,-1.8409701E-2,7.110269E-2,-8.992044E-2,4.881099E-3,-7.3879417E-3,9.77548E-2,1.2968812E-2,1.2038146E-2,2.3867035E-2,8.8894E-3,-1.0467523E-3,-1.5443225E-1,-2.4996966E-1,-2.0567698E-2,-1.04516465E-2,-1.8106676E-3,3.784915E-3,-7.8015993E-3,-2.2148767E-3,-1.0186115E-3,-1.4806482E-1,-3.0158894E-2,-7.6714233E-3,-6.956638E-2,2.2432987E-2,7.837078E-5,4.518229E-3,-3.0988848E-3,5.8769487E-4,7.572765E-2,1.9353291E-1,-1.0267993E-3,1.1372247E-1,-6.88002E-2,6.220237E-2,4.0727045E-2,1.3610534E-1,-7.429002E-3,-2.3357257E-2,6.493661E-2,1.459348E-1,-7.599928E-3,2.7608543E-3,-1.2748771E-2,-2.8213009E-3,-9.0355445E-3,-3.98174E-3,-2.4131336E-3,-9.389699E-3,-3.1999135E-3,3.4211902E-3,-9.884589E-4,5.171651E-3,1.1303496E-2,5.980526E-3,2.4330695E-3,-2.166809E-3,2.1575398E-3,6.714982E-3,6.1021145E-3,-4.183727E-3,7.989451E-3,2.8440674E-5,4.2089876E-3,-1.0054508E-3,-6.8957923E-4,8.829163E-3,-5.13425E-3,2.5509289E-3,4.0809037E-3,-1.1029723E-3,8.457262E-3,2.759687E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,83,-1,-1,-1,-1,85,87,89,91,-1,-1,93,-1,95,97,99,101,103,105,107,-1,109,111,113,-1,115,-1,-1,117,-1,-1,119,121,123,125,127,-1,-1,-1,-1,-1,-1,129,-1,-1,131,133,-1,-1,135,137,139,-1,-1,141,143,-1,145,-1,-1,-1,147,149,-1,-1,151,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,155,-1,157,-1,-1,-1,-1,-1,159,161,163,165,167,-1,-1,169,171,-1,173,-1,175,-1,177,179,181,-1,183,185,-1,-1,187,-1,-1,-1,-1,-1,189,191,-1,-1,-1,-1,-1,-1,-1,193,-1,-1,195,197,-1,-1,-1,-1,199,201,203,205,207,209,211,213,-1,215,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8535387E1,6.5695257E0,3.4538612E0,1.379837E0,2.756598E0,2.2068024E0,6.126642E-1,2.3776798E0,1.8588877E-1,8.632474E-1,8.2950985E-1,4.277634E-1,1.6128674E0,4.6911293E-1,0E0,1.2330275E0,0E0,0E0,0E0,1.6096497E-1,4.7148627E-1,3.2905322E-1,5.537443E-1,1.0624728E-1,4.698515E-2,7.6618767E-1,8.039868E-1,2.0548818E-1,3.261125E-2,1.5823984E-1,1.374351E0,6.05582E-2,7.338679E-2,4.8025644E-1,1.7735785E-1,1.8944547E-1,2.4660249E-1,2.3711762E-1,3.6584604E-1,2.9638574E-2,1.2899071E-1,0E0,3.4207225E-2,3.0875874E-1,6.3444424E-1,1.070837E0,1.2378834E-1,0E0,0E0,0E0,0E0,1.7166376E-2,2.095294E-2,4.0952381E-1,5.449796E-1,0E0,0E0,2.8109133E-2,0E0,4.8987174E-1,2.1593298E-1,1.317706E-1,1.1568097E-1,1.5640038E-1,2.550196E-1,3.850623E-1,0E0,4.5406885E-2,1.635961E-1,9.047711E-2,0E0,2.5802672E-2,0E0,0E0,2.2850301E-2,0E0,0E0,1.0340643E-1,2.9957771E-2,6.47171E-1,1.0805762E-1,3.4482598E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7915487E-2,0E0,0E0,3.6960554E-1,1.1441724E-1,0E0,0E0,8.2040966E-2,5.066397E-1,1.5640551E-1,0E0,0E0,8.116335E-2,5.0029255E-2,0E0,8.485317E-2,0E0,0E0,0E0,6.9813865E-1,4.0415004E-1,0E0,0E0,2.0567635E-1,2.2697365E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.7575245E-2,0E0,1.5713403E-1,0E0,0E0,0E0,0E0,0E0,2.1890545E-1,7.156801E-2,5.819227E-2,2.350095E-2,5.9302866E-2,0E0,0E0,6.0008526E-1,1.1643958E-1,0E0,2.329855E-2,0E0,1.815864E-2,0E0,1.1620033E-1,7.25374E-2,1.8359569E-1,0E0,1.14901185E-1,9.532726E-2,0E0,0E0,5.218953E-2,0E0,0E0,0E0,0E0,0E0,2.1477342E-1,1.7457223E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.842454E-2,0E0,0E0,1.3767433E-1,9.033047E-2,0E0,0E0,0E0,0E0,6.922447E-2,3.386545E-2,3.202723E-2,1.6528234E-2,2.1739471E-1,9.536918E-2,1.3735428E-1,1.606979E-1,0E0,8.8898525E-2,5.4055437E-2,3.706959E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,51,51,52,52,53,53,54,54,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,71,71,74,74,77,77,78,78,79,79,80,80,81,81,88,88,91,91,92,92,95,95,96,96,97,97,100,100,101,101,103,103,107,107,108,108,111,111,112,112,123,123,125,125,131,131,132,132,133,133,134,134,135,135,138,138,139,139,141,141,143,143,145,145,146,146,147,147,149,149,150,150,153,153,159,159,160,160,168,168,171,171,172,172,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,84,-1,-1,-1,-1,86,88,90,92,-1,-1,94,-1,96,98,100,102,104,106,108,-1,110,112,114,-1,116,-1,-1,118,-1,-1,120,122,124,126,128,-1,-1,-1,-1,-1,-1,130,-1,-1,132,134,-1,-1,136,138,140,-1,-1,142,144,-1,146,-1,-1,-1,148,150,-1,-1,152,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,156,-1,158,-1,-1,-1,-1,-1,160,162,164,166,168,-1,-1,170,172,-1,174,-1,176,-1,178,180,182,-1,184,186,-1,-1,188,-1,-1,-1,-1,-1,190,192,-1,-1,-1,-1,-1,-1,-1,194,-1,-1,196,198,-1,-1,-1,-1,200,202,204,206,208,210,212,214,-1,216,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.0976269E5,2E0,4.217427E7,7.9E2,7.601393E2,1.465E4,6.68411E2,1.0479E4,3.95E2,2.0766992E6,5.4051904E8,6.9E1,3.91157E5,9.440856E-3,4.9E2,1.657073E-2,-2.907893E-2,-1.5777435E-2,3.1E1,2.8045622E6,1.592E3,2.831224E10,7.6937294E-1,1.2003246E6,2.1298597E0,5.712652E8,5.071E3,7.650379E8,8.0833334E-1,4E0,1E0,1.7142857E0,1E0,3.5364E4,2.3719013E0,2.0240436E0,2.9077E4,1E0,1.6673345E7,2.89196E6,4.738623E-3,6.1236826E8,1.6975454E3,1.684E0,3.5675005E3,4.7837E4,-6.1995494E-3,4.9995813E-3,-1.6558258E-2,-7.9238815E-3,1.00019E5,2.88E0,2.123E3,1.1939979E8,-7.6184296E-3,-1.7975752E-3,2.55E2,-1.4750147E-2,2.3514E4,2E1,8.612209E-6,7.45E2,1E0,4.6712E4,1.056178E6,7.5983666E-3,3.693316E7,1.9376624E0,1.87627E5,1.8202102E-2,4.4971E4,1.3573053E-3,-4.3895687E-3,1.5822886E0,5.5208523E-3,1.2986845E-2,8E0,1.1E1,3.2952412E5,1.5E1,5.94E2,2.6790693E-2,-6.1836056E-3,2.5626253E-3,-1.3171322E-2,-2.4725582E-2,-7.5487415E-3,5.513889E0,7.7513396E-5,1.8791415E-2,3.1E1,2.3412812E0,-1.1578427E-2,-5.7540867E-3,1.3E1,1.0865825E0,5.44E2,5.5606193E-3,-2.0883323E-3,2.6732E4,5.69E2,-7.7523096E-3,8E0,-2.3818745E-3,-7.921826E-3,4.5830626E-3,8.81059E5,3.84E2,4.5355887E-4,-6.0457014E-3,2.3670635E1,2.483E3,-7.790265E-5,8.612825E-3,8.616152E-3,2.9208383E-3,-1.06385596E-4,4.1441983E-3,2.0027258E-2,1.0591589E-2,1.7179692E-2,2.9448232E-2,2.0659652E0,1.9264043E-3,1.4E1,1.3138838E-2,-4.5302436E-3,9.1990195E-3,-1.5037169E-2,-8.121841E-3,1.2E1,4.6E1,4.306E4,1.862E3,3.3272727E0,1.9668234E-3,-1.685911E-3,2.6801266E2,6.953366E7,-9.597459E-3,7.597862E6,9.610854E-3,1.4162437E0,2.7359598E-3,1.047E3,4.0119403E2,2.8E1,-1.8409701E-2,9.751103E6,4.6816683E-1,4.881099E-3,-7.3879417E-3,5.144203E2,1.2968812E-2,1.2038146E-2,2.3867035E-2,8.8894E-3,-1.0467523E-3,4.7866177E2,7.997723E6,-2.0567698E-2,-1.04516465E-2,-1.8106676E-3,3.784915E-3,-7.8015993E-3,-2.2148767E-3,-1.0186115E-3,5.0149255E0,-3.0158894E-2,-7.6714233E-3,6.763314E7,4.358214E6,7.837078E-5,4.518229E-3,-3.0988848E-3,5.8769487E-4,2.7777777E0,3.3253515E6,2.12E2,3.1706784E0,8.118812E-1,1.9545455E0,1.7787506E4,1.2E2,-7.429002E-3,4.99E2,4.498404E2,1.2183674E1,-7.599928E-3,2.7608543E-3,-1.2748771E-2,-2.8213009E-3,-9.0355445E-3,-3.98174E-3,-2.4131336E-3,-9.389699E-3,-3.1999135E-3,3.4211902E-3,-9.884589E-4,5.171651E-3,1.1303496E-2,5.980526E-3,2.4330695E-3,-2.166809E-3,2.1575398E-3,6.714982E-3,6.1021145E-3,-4.183727E-3,7.989451E-3,2.8440674E-5,4.2089876E-3,-1.0054508E-3,-6.8957923E-4,8.829163E-3,-5.13425E-3,2.5509289E-3,4.0809037E-3,-1.1029723E-3,8.457262E-3,2.759687E-3],"split_indices":[2,43,6,60,2,67,44,67,9,2,43,7,8,1,0,1,0,0,0,10,43,44,46,53,66,53,47,2,5,68,10,8,68,29,9,53,53,1,29,47,62,0,7,67,69,67,9,0,0,0,0,5,69,1,7,0,0,2,0,1,3,53,2,19,1,9,0,7,69,1,0,9,0,0,57,0,0,32,3,48,3,0,0,0,0,0,0,0,69,0,0,3,68,0,0,8,71,10,0,0,1,2,0,8,0,0,0,9,0,0,0,73,2,0,0,0,0,0,0,0,0,0,0,57,0,3,0,0,0,0,0,8,3,1,2,69,0,0,70,7,0,60,0,68,0,2,4,71,0,9,53,0,0,4,0,0,0,0,0,67,9,0,0,0,0,0,0,0,71,0,0,7,5,0,0,0,0,69,60,10,69,68,68,48,0,0,0,4,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.033E3,7.85E2,2.48E2,2.5E2,5.35E2,2.2E2,2.8E1,2.31E2,1.9E1,2.23E2,3.12E2,5.4E1,1.66E2,2.3E1,5E0,2.24E2,7E0,6E0,1.3E1,4.9E1,1.74E2,2.27E2,8.5E1,3.4E1,2E1,1.19E2,4.7E1,1.2E1,1.1E1,4.7E1,1.77E2,2.1E1,2.8E1,1.29E2,4.5E1,7.6E1,1.51E2,6.4E1,2.1E1,1.4E1,2E1,5E0,1.5E1,5.8E1,6.1E1,3.5E1,1.2E1,6E0,6E0,6E0,5E0,1.2E1,3.5E1,9E0,1.68E2,1.5E1,6E0,1.4E1,1.4E1,3.7E1,9.2E1,2.2E1,2.3E1,6.4E1,1.2E1,1.44E2,7E0,8E0,5.6E1,1E1,1.1E1,9E0,5E0,9E0,1.1E1,4E0,1.1E1,3.3E1,2.5E1,2.8E1,3.3E1,2.8E1,7E0,6E0,6E0,5E0,7E0,5E0,3E1,5E0,4E0,1.46E2,2.2E1,5E0,9E0,2.1E1,1.6E1,8.6E1,6E0,5E0,1.7E1,1.9E1,4E0,5.8E1,6E0,6E0,6E0,6.4E1,8E1,4E0,4E0,1E1,4.6E1,4E0,6E0,4E0,5E0,4E0,7E0,2.6E1,7E0,6E0,1.9E1,2.4E1,4E0,2E1,1.3E1,4E0,2.4E1,2.5E1,5E0,1.32E2,1.4E1,1.4E1,8E0,1.7E1,4E0,5E0,1.1E1,8E1,6E0,1E1,7E0,1.1E1,8E0,3.7E1,2.1E1,5.9E1,5E0,6E1,2E1,6E0,4E0,3.8E1,8E0,4E0,2E1,1.6E1,4E0,9.4E1,3.8E1,7E0,7E0,7E0,7E0,4E0,4E0,6E0,1.1E1,5E0,6E0,6.3E1,1.7E1,4E0,6E0,7E0,4E0,1.8E1,1.9E1,1.1E1,1E1,4.6E1,1.3E1,4.2E1,1.8E1,9E0,1.1E1,2.4E1,1.4E1,9E1,4E0,3.3E1,5E0,5E0,6E0,5.7E1,6E0,6E0,1.1E1,5E0,1.3E1,9E0,1E1,5E0,6E0,4E0,6E0,4E0,4.2E1,4E0,9E0,2.3E1,1.9E1,5E0,1.3E1,5E0,6E0,1.9E1,5E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[2.5721124E-4,-6.941781E-2,2.1714778E-1,-2.0841785E-1,-1.1861273E-2,2.6116636E-1,-3.2985624E-2,-1.874136E-1,-4.045349E-1,-1.5475622E-1,1.889853E-2,2.1016075E-1,4.4465017E-1,-1.5201495E-2,7.024072E-2,-2.6033053E-1,-1.2859589E-1,-2.1156715E-2,-8.994043E-3,-1.3139848E-1,-2.250427E-2,2.5277276E-2,-1.5591447E-2,2.643263E-1,8.177779E-2,5.2652365E-1,2.0261356E-1,1.6935204E-1,-8.427964E-2,-2.9067558E-1,-1.4533873E-1,1.6115719E-1,-1.5745144E-1,-1.7316738E-1,-8.1896804E-2,-5.4016E-3,9.334349E-2,1.1552466E-1,3.166208E-1,4.711955E-2,1.3531619E-2,1.5834536E-2,6.438996E-1,3.1449094E-3,1.1773628E-2,3.99937E-2,1.5063236E-2,2.5682794E-3,-7.4456334E-3,-3.2147348E-1,-2.2857998E-1,-6.641187E-2,-2.1105298E-1,3.803326E-3,1.1055024E-2,-1.277601E-1,-3.1648016E-1,-1.4948812E-1,-1.2743434E-2,-2.2059928E-3,-1.08393565E-1,-6.1240386E-2,4.302475E-2,1.03435725E-1,-9.045641E-3,1.8546598E-1,-5.427583E-3,4.648768E-1,2.517334E-1,6.544048E-2,-5.1990584E-3,3.3385865E-2,1.8741027E-2,-6.1183185E-3,7.2737103E-3,-1.8258354E-1,-1.601116E-2,-4.6916767E-3,-2.5283355E-1,-8.951615E-4,-4.7298083E-3,-1.2539495E-2,-5.336325E-3,-1.424571E-1,3.1856492E-3,-1.9143803E-2,-6.0713263E-3,-2.0149945E-1,-1.1628604E-1,2.1152708E-3,-2.6716453E-3,-1.2655258E-1,8.596487E-4,-2.5652793E-1,-4.3023027E-2,6.0954146E-2,-9.001108E-2,3.0621938E-2,1.603076E-1,-1.2320047E-4,2.3263249E-1,2.3992082E-2,1.4339581E-2,1.7017592E-1,3.059089E-1,-2.7941069E-2,1.0504462E-1,-1.1340461E-2,-3.842874E-3,-2.7481338E-1,-5.688928E-3,-1.0213446E-1,-2.1679962E-1,-3.421872E-3,-1.1138808E-2,-3.179752E-3,-1.3882832E-1,-8.746116E-2,-7.4929795E-3,-3.431194E-3,-1.7375572E-2,-1.5040724E-2,-3.1650174E-2,7.369837E-2,-5.5258647E-2,-1.3014556E-1,2.6066182E-3,-1.0831684E-1,6.0576033E-2,1.7547688E-1,-2.0043545E-3,4.7190785E-3,1.2993889E-2,2.186632E-1,6.588124E-2,3.39173E-1,5.5275876E-3,2.6303676E-3,-6.3040005E-3,9.76267E-3,7.409952E-2,-8.616283E-3,-1.5098357E-2,-1.1394966E-1,7.986102E-4,-4.629351E-3,-2.4934334E-1,-7.442143E-3,-2.2514497E-3,-5.6348983E-3,-1.1858534E-3,-4.725372E-2,9.8782554E-2,1.3226067E-2,1.069804E-1,4.2094857E-2,-9.62348E-3,-2.9536115E-3,-8.063637E-3,1.2176153E-3,-1.2713758E-2,8.9630626E-2,-6.4605415E-2,1.0361151E-1,2.2378138E-1,1.197937E-2,5.4405904E-3,9.944889E-3,-5.8465656E-3,3.6956805E-1,5.233609E-3,9.348919E-2,6.3083536E-4,-2.6966045E-3,-6.9168294E-3,-1.2859753E-2,-5.801923E-3,-1.6738634E-3,-1.1154813E-2,8.751863E-3,3.2852514E-4,2.3848298E-3,-1.5936765E-3,5.5873496E-3,-4.5351812E-4,-2.6152588E-3,5.4206196E-3,2.7820508E-3,7.192777E-3,-8.781809E-3,2.34366E-3,2.8736636E-3,9.688542E-3,5.5319746E-3,1.1517445E-2,1.0970537E-2,1.8744828E-2,7.1072887E-4,5.65228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,-1,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,-1,73,-1,-1,-1,75,77,79,81,-1,-1,83,85,87,-1,89,91,93,95,97,-1,99,-1,101,103,105,-1,-1,-1,-1,-1,107,-1,-1,109,-1,-1,-1,-1,111,-1,-1,-1,113,115,-1,-1,117,-1,119,121,123,125,127,129,-1,131,-1,-1,133,135,137,139,-1,-1,141,-1,143,145,-1,-1,-1,147,149,-1,-1,-1,-1,151,153,155,157,-1,159,161,163,-1,-1,-1,165,167,169,-1,-1,-1,-1,171,-1,-1,173,-1,-1,175,-1,-1,-1,-1,177,179,181,183,185,-1,-1,-1,-1,-1,187,189,191,193,-1,-1,-1,-1,195,-1,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5414054E1,6.1738076E0,2.7392235E0,8.5514355E-1,2.4072196E0,1.881237E0,1.184429E0,8.500166E-1,1.4623117E-1,6.890199E-1,1.0337775E0,1.1396914E0,7.850561E-1,0E0,4.5839968E-1,2.7529144E-1,9.952483E-1,0E0,0E0,1.7603993E-1,0E0,9.276626E-1,0E0,8.738594E-1,3.593689E-1,4.911518E-1,7.400721E-2,3.5182512E-1,1.3800666E-1,6.229925E-2,9.4902545E-2,4.8054814E-2,4.6007466E-1,7.78923E-2,9.266901E-2,8.327839E-1,4.229468E-1,5.432877E-1,7.102909E-1,1.389912E-1,0E0,0E0,6.333256E-2,0E0,0E0,2.4361694E-1,0E0,0E0,0E0,9.357071E-2,5.750668E-2,1.6625367E-2,3.2097965E-2,0E0,0E0,2.729149E-1,2.3541677E-1,4.9097538E-2,0E0,3.468129E-2,7.845992E-2,4.977035E-1,3.9905316E-1,5.499928E-1,0E0,2.224595E-1,0E0,4.9082756E-2,2.218833E-1,1.5456519E-1,0E0,0E0,0E0,0E0,0E0,3.5841018E-2,0E0,0E0,2.887392E-2,0E0,0E0,0E0,0E0,2.3266053E-1,0E0,0E0,0E0,5.784613E-2,2.0400047E-2,0E0,0E0,2.6214063E-2,0E0,2.3358792E-1,4.1385034E-1,2.2030461E-1,1.2488291E-1,2.562685E-1,2.3935354E-1,0E0,9.817135E-2,0E0,0E0,1.221323E-1,1.8246865E-1,1.2750067E-1,7.9711616E-2,0E0,0E0,2.1702766E-2,0E0,8.239561E-2,8.505356E-2,0E0,0E0,0E0,2.8249264E-2,3.0987598E-2,0E0,0E0,0E0,0E0,2.6517516E-1,2.6425362E-1,2.3226213E-1,3.5874456E-2,0E0,2.5446662E-1,1.8764544E-1,2.1981311E-1,0E0,0E0,0E0,4.3266058E-2,3.1469896E-1,1.6411138E-1,0E0,0E0,0E0,0E0,2.685155E-2,0E0,0E0,9.463805E-2,0E0,0E0,4.403293E-2,0E0,0E0,0E0,0E0,2.3740658E-1,1.1500035E-1,8.972961E-2,1.3978243E-1,8.090342E-2,0E0,0E0,0E0,0E0,0E0,7.028574E-2,1.543665E-1,1.1876455E-1,7.900262E-2,0E0,0E0,0E0,0E0,2.7895212E-2,0E0,3.868772E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,42,42,45,45,49,49,50,50,51,51,52,52,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,65,65,67,67,68,68,69,69,75,75,78,78,83,83,87,87,88,88,91,91,93,93,94,94,95,95,96,96,97,97,98,98,100,100,103,103,104,104,105,105,106,106,109,109,111,111,112,112,116,116,117,117,122,122,123,123,124,124,125,125,127,127,128,128,129,129,133,133,134,134,135,135,140,140,143,143,146,146,151,151,152,152,153,153,154,154,155,155,161,161,162,162,163,163,164,164,169,169,171,171],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,-1,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,-1,74,-1,-1,-1,76,78,80,82,-1,-1,84,86,88,-1,90,92,94,96,98,-1,100,-1,102,104,106,-1,-1,-1,-1,-1,108,-1,-1,110,-1,-1,-1,-1,112,-1,-1,-1,114,116,-1,-1,118,-1,120,122,124,126,128,130,-1,132,-1,-1,134,136,138,140,-1,-1,142,-1,144,146,-1,-1,-1,148,150,-1,-1,-1,-1,152,154,156,158,-1,160,162,164,-1,-1,-1,166,168,170,-1,-1,-1,-1,172,-1,-1,174,-1,-1,176,-1,-1,-1,-1,178,180,182,184,186,-1,-1,-1,-1,-1,188,190,192,194,-1,-1,-1,-1,196,-1,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.3876712E7,4.217427E7,4.97E2,1.4176E4,1.4598765E0,2.71E2,1.6074808E5,1E0,4.4E1,8E0,1.2792593E2,-1.5201495E-2,7.7175174E8,1.9E1,1.1E1,-2.1156715E-2,-8.994043E-3,3.56E2,-2.250427E-2,2.0766992E6,-1.5591447E-2,6.9664386E2,2.8016653E3,1.9705E4,4.988E3,6.355E3,3.592428E7,1.16E2,1E0,1.07882355E2,3.6349E4,1.9530877E5,1.2051282E0,1.212945E6,1.0089981E3,4.75356E6,1.5282027E5,1.6677892E2,1.3531619E-2,1.5834536E-2,1.61E9,3.1449094E-3,1.1773628E-2,1.7431098E8,1.5063236E-2,2.5682794E-3,-7.4456334E-3,1.863E3,2.0395E4,3.83E2,6.44E2,3.803326E-3,1.1055024E-2,5.185192E2,1.775373E1,1.0538846E3,-1.2743434E-2,4.33E2,1E0,2.83408E5,7.1E1,4.6066E4,-9.045641E-3,1.02E2,-5.427583E-3,1.2909952E1,5.831829E6,1.0901037E10,-5.1990584E-3,3.3385865E-2,1.8741027E-2,-6.1183185E-3,7.2737103E-3,1.1E1,-1.601116E-2,-4.6916767E-3,3.328629E6,-8.951615E-4,-4.7298083E-3,-1.2539495E-2,-5.336325E-3,1.7E1,3.1856492E-3,-1.9143803E-2,-6.0713263E-3,2.4636364E1,3.806E3,2.1152708E-3,-2.6716453E-3,9.695652E0,8.596487E-4,4.5801528E-2,7E0,1E0,2.0316172E6,2.2007043E0,2.5345264E7,-1.2320047E-4,1.9376624E0,2.3992082E-2,1.4339581E-2,1.934E3,2.6912618E7,1.2473E4,1.0092541E8,-1.1340461E-2,-3.842874E-3,2.1E1,-5.688928E-3,2.0263722E8,5.236559E0,-3.421872E-3,-1.1138808E-2,-3.179752E-3,2E0,1.2123E4,-7.4929795E-3,-3.431194E-3,-1.7375572E-2,-1.5040724E-2,4.888E3,8.69E2,5.4007E7,1.4E1,2.6066182E-3,1E0,1.4E1,2.485E2,-2.0043545E-3,4.7190785E-3,1.2993889E-2,3.7951445E6,1.8667632E0,2.057143E0,5.5275876E-3,2.6303676E-3,-6.3040005E-3,9.76267E-3,1.213E5,-8.616283E-3,-1.5098357E-2,2.7777777E0,7.986102E-4,-4.629351E-3,8.43E2,-7.442143E-3,-2.2514497E-3,-5.6348983E-3,-1.1858534E-3,3.607317E1,2.6595745E0,1E0,3.471E3,1.14E3,-9.62348E-3,-2.9536115E-3,-8.063637E-3,1.2176153E-3,-1.2713758E-2,6.7723384E7,2.8266037E2,6.887749E7,1.0029973E8,1.197937E-2,5.4405904E-3,9.944889E-3,-5.8465656E-3,1.0775862E0,5.233609E-3,2.8802464E8,6.3083536E-4,-2.6966045E-3,-6.9168294E-3,-1.2859753E-2,-5.801923E-3,-1.6738634E-3,-1.1154813E-2,8.751863E-3,3.2852514E-4,2.3848298E-3,-1.5936765E-3,5.5873496E-3,-4.5351812E-4,-2.6152588E-3,5.4206196E-3,2.7820508E-3,7.192777E-3,-8.781809E-3,2.34366E-3,2.8736636E-3,9.688542E-3,5.5319746E-3,1.1517445E-2,1.0970537E-2,1.8744828E-2,7.1072887E-4,5.65228E-3],"split_indices":[2,43,60,60,2,2,69,2,48,106,3,32,73,0,7,3,10,0,0,2,0,43,0,67,4,2,44,2,60,44,20,67,44,66,68,9,4,62,48,73,0,0,5,0,0,5,0,0,0,9,9,12,1,0,0,67,71,48,0,2,28,7,8,1,0,0,0,71,60,46,0,0,0,0,0,5,0,0,9,0,0,0,0,3,0,0,0,67,1,0,0,73,0,73,3,19,62,69,60,0,69,0,0,44,62,44,7,0,0,0,0,7,69,0,0,0,32,1,0,0,0,0,10,2,7,3,0,8,8,67,0,0,0,43,57,68,0,0,0,0,10,0,0,71,0,0,10,0,0,0,0,73,68,26,10,2,0,0,0,0,0,7,4,7,5,0,0,0,0,68,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.71E2,2.47E2,2.25E2,5.46E2,2.1E2,3.7E1,2.05E2,2E1,9.6E1,4.5E2,1.66E2,4.4E1,9E0,2.8E1,9E1,1.15E2,1.5E1,5E0,9.1E1,5E0,4.43E2,7E0,1.16E2,5E1,3.2E1,1.2E1,1.7E1,1.1E1,7E1,2E1,1E1,1.05E2,4.8E1,4.3E1,3.06E2,1.37E2,3.1E1,8.5E1,4.4E1,6E0,1.4E1,1.8E1,4E0,8E0,1E1,7E0,4E0,7E0,4.4E1,2.6E1,1E1,1E1,6E0,4E0,9E1,1.5E1,4.1E1,7E0,1.1E1,3.2E1,1.42E2,1.64E2,1.33E2,4E0,2.4E1,7E0,2.4E1,6.1E1,4E1,4E0,1.2E1,6E0,4E0,6E0,8E0,3.6E1,5E0,2.1E1,5E0,5E0,5E0,5E0,8.4E1,6E0,9E0,6E0,1.4E1,2.7E1,6E0,5E0,2.8E1,4E0,1.1E1,1.31E2,1.45E2,1.9E1,5.9E1,7.4E1,5E0,1.9E1,1.6E1,8E0,2.6E1,3.5E1,1.2E1,2.8E1,4E0,4E0,1.7E1,4E0,5.6E1,2.8E1,4E0,1E1,1E1,1.7E1,1.5E1,1.3E1,5E0,6E0,4E0,1.27E2,1.31E2,1.4E1,1.5E1,4E0,1E1,4.9E1,6.9E1,5E0,6E0,1.3E1,1.7E1,9E0,2.9E1,6E0,7E0,5E0,5E0,2.3E1,8E0,9E0,5.1E1,5E0,7E0,2.1E1,1.3E1,4E0,9E0,6E0,1.14E2,1.3E1,4.7E1,8.4E1,9E0,5E0,7E0,8E0,6E0,4E0,4E1,9E0,2.9E1,4E1,1.1E1,6E0,5E0,4E0,2.5E1,4E0,1.7E1,6E0,2.1E1,3E1,1.6E1,5E0,1.09E2,5E0,6E0,7E0,2.6E1,2.1E1,7.5E1,9E0,4E0,5E0,2.9E1,1.1E1,4E0,5E0,2.2E1,7E0,9E0,3.1E1,7E0,1.8E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[-5.3168205E-3,-7.74734E-2,2.0169191E-1,-2.1571895E-1,-1.1063464E-2,1.3282856E-1,3.6131305E-1,-1.9371118E-1,-3.9438784E-1,-1.2515052E-1,2.0699512E-2,9.7009264E-2,5.8387464E-1,3.8022965E-1,-2.7943772E-4,-3.0554298E-1,-1.5399313E-1,-1.7895313E-1,-5.313147E-1,-1.519736E-1,3.9994195E-2,-6.532466E-2,4.1314524E-2,1.2865941E-1,-2.1402809E-1,3.1359896E-2,1.2808675E-2,4.9003658E-1,3.2540062E-1,-2.1329568E-1,-3.4083948E-1,1.1649526E-1,-1.7514001E-1,-1.3769596E-2,-2.3702849E-3,-3.6308873E-2,-2.9995766E-1,-1.2228492E-1,-2.4604942E-1,-2.210123E-2,4.511144E-3,-1.6732469E-1,-1.944176E-2,3.3107933E-2,1.5556166E-2,4.0220708E-1,1.0441745E-1,-1.5473763E-2,-6.933846E-2,1.6368799E-2,2.7545583E-2,2.344124E-1,3.8828143E-1,-1.144036E-2,-3.1243395E-3,-1.645332E-2,-7.3663923E-3,-4.727472E-3,2.2512001E-1,-1.5317227E-1,-3.496251E-1,-7.40218E-3,-1.6927782E-2,-1.6040571E-1,-8.918237E-2,-2.9165658E-1,-1.0871704E-3,2.1256988E-3,-3.9635994E-3,-6.320663E-2,-2.9409254E-1,-9.239928E-3,5.7223067E-3,1.0242157E-1,2.3927146E-3,2.2257049E-2,1.0584036E-2,-6.5492444E-2,1.2629864E-1,-4.2622528E-4,-5.3398423E-3,1.4539092E-2,8.228761E-3,1.890475E-2,9.131394E-3,4.570507E-3,1.4150088E-2,-1.6706988E-1,7.2310865E-2,-1.8060554E-2,-9.643882E-3,-1.01115964E-1,-2.0501201E-1,-1.04588315E-1,2.3978092E-3,-1.7310146E-2,-8.290944E-3,1.4051679E-3,-5.22323E-3,-4.314174E-3,-2.3851106E-2,9.6280344E-2,-2.7791766E-2,7.13368E-2,2.1080156E-1,-1.7589917E-2,1.1751634E-1,9.905234E-4,-1.2276243E-1,1.5467829E-1,3.0117992E-2,-2.1173723E-1,-1.0027598E-1,-1.6374715E-3,7.6507195E-3,-1.4358059E-1,-1.6352473E-3,-1.0641862E-2,-4.065583E-3,-7.0854135E-2,-7.4924002E-3,1.7874026E-3,7.697704E-3,-6.9604285E-2,9.6243724E-2,9.594987E-2,-7.492316E-3,1.5364879E-2,1.3179964E-1,-7.489912E-2,7.6373997E-3,4.0003918E-2,1.7660488E-1,-7.77869E-3,-1.5871506E-3,1.354796E-1,2.0589838E-2,-8.870381E-3,6.870691E-2,-1.8367597E-1,-3.554447E-1,-1.4009155E-1,-6.9369463E-4,-9.562683E-3,-3.0865753E-3,-5.7975124E-3,-2.4362074E-3,-6.7767436E-3,-3.841082E-2,1.8019229E-3,6.7304294E-3,7.2633E-2,1.812663E-1,1.791936E-1,5.4190175E-5,-4.928792E-2,-1.372968E-2,2.3710981E-2,-1.0721794E-1,7.766926E-3,-1.6710917E-2,1.1481164E-2,8.703412E-2,1.904918E-1,7.306057E-2,1.22991525E-1,-3.8058117E-3,-9.950286E-3,-3.0121743E-3,-2.0771096E-2,-8.609469E-3,-8.783466E-3,-5.0339014E-3,-2.618099E-3,1.2636482E-3,-3.939506E-3,2.731606E-5,2.0682104E-3,4.7070077E-3,1.0218954E-2,2.6412944E-3,1.0034836E-2,4.164094E-3,8.341585E-4,-4.223221E-3,-3.3481237E-3,2.1724629E-3,-8.852959E-3,-2.0215022E-3,1.9006011E-3,-3.761425E-3,7.8010594E-4,7.10566E-3,6.305115E-3,1.4021178E-2,-5.677469E-3,5.6100124E-3,3.424116E-3,9.708883E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,-1,49,51,53,55,57,59,-1,-1,-1,61,63,65,67,-1,69,71,73,-1,75,77,-1,79,-1,-1,81,83,-1,-1,-1,-1,-1,85,87,89,-1,-1,91,93,95,-1,-1,-1,97,99,-1,101,103,105,-1,-1,107,109,-1,-1,-1,-1,-1,-1,-1,-1,111,113,-1,-1,115,117,119,-1,-1,-1,-1,-1,-1,-1,121,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,143,-1,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,153,155,157,159,161,-1,-1,163,-1,-1,165,167,169,171,173,-1,-1,-1,-1,-1,175,-1,-1,177,179,181,-1,183,-1,185,187,-1,189,-1,191,193,195,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5833953E1,7.2137837E0,2.9638271E0,9.2628956E-1,1.9304225E0,3.0614972E0,5.7213783E-1,9.701071E-1,7.013869E-1,5.199224E-1,7.422021E-1,1.799525E0,2.1929169E-1,3.101492E-1,0E0,1.1940527E-1,9.985213E-1,1.6713586E-1,7.067418E-1,2.492106E-1,6.301643E-2,3.758315E-1,8.058327E-1,1.0455413E0,2.7417797E-1,0E0,0E0,1.346879E-1,2.2365904E-1,7.828534E-2,5.7002544E-2,3.320417E-1,5.5590105E-1,0E0,0E0,0E0,3.4148574E-2,8.412242E-2,2.2654927E-1,4.299224E-2,0E0,3.11382E-1,2.6013216E-1,7.001904E-1,0E0,6.866813E-2,5.7338154E-1,0E0,2.3952786E-2,0E0,0E0,5.6952596E-2,4.275036E-2,0E0,0E0,0E0,0E0,0E0,6.187865E-2,4.6019626E-1,1.7315626E-2,0E0,0E0,7.1834326E-2,1.0106379E-1,1.0975349E-1,0E0,0E0,0E0,7.344839E-2,4.4231218E-1,0E0,1.574592E-1,3.2522774E-1,5.288754E-1,0E0,0E0,9.224757E-2,3.6383367E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.81541E-1,9.6343994E-2,0E0,0E0,4.2361245E-2,3.605622E-2,6.621677E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.0267115E-2,2.0377596E-1,4.753556E-1,1.6782218E-1,2.8394052E-1,1.486584E-1,0E0,3.8571596E-2,5.206454E-1,2.8747773E-1,2.573223E-1,2.2193867E-1,0E0,0E0,3.48496E-2,0E0,0E0,0E0,2.0114452E-2,0E0,0E0,0E0,6.409441E-2,2.2617184E-2,1.3205391E-1,0E0,0E0,9.265354E-2,3.2693344E-1,2.5546297E-1,1.1906559E-1,1.00750804E-1,0E0,0E0,3.2602966E-1,0E0,0E0,2.3977716E-1,2.4512148E-1,1.0726166E-1,4.158306E-2,2.8238486E-2,0E0,0E0,0E0,0E0,0E0,4.0259857E-2,0E0,0E0,4.3531835E-2,6.1675876E-2,1.7663985E-2,0E0,1.5767989E-1,0E0,2.7472693E-1,8.2678944E-2,0E0,4.8269242E-2,0E0,4.4241786E-2,2.7597857E-1,4.698837E-1,7.3884726E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,51,51,52,52,58,58,59,59,60,60,63,63,64,64,65,65,69,69,70,70,72,72,73,73,74,74,77,77,78,78,87,87,88,88,91,91,92,92,93,93,101,101,102,102,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,112,112,115,115,119,119,123,123,124,124,125,125,128,128,129,129,130,130,131,131,132,132,135,135,138,138,139,139,140,140,141,141,142,142,148,148,151,151,152,152,153,153,155,155,157,157,158,158,160,160,162,162,163,163,164,164,165,165],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,-1,50,52,54,56,58,60,-1,-1,-1,62,64,66,68,-1,70,72,74,-1,76,78,-1,80,-1,-1,82,84,-1,-1,-1,-1,-1,86,88,90,-1,-1,92,94,96,-1,-1,-1,98,100,-1,102,104,106,-1,-1,108,110,-1,-1,-1,-1,-1,-1,-1,-1,112,114,-1,-1,116,118,120,-1,-1,-1,-1,-1,-1,-1,122,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,144,-1,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,154,156,158,160,162,-1,-1,164,-1,-1,166,168,170,172,174,-1,-1,-1,-1,-1,176,-1,-1,178,180,182,-1,184,-1,186,188,-1,190,-1,192,194,196,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,2.9251662E6,4.4895835E0,5.47E2,5.428175E3,2E0,4.54E2,3.89E2,2.9652428E6,1E0,3.9661028E7,1.3342042E0,3.3382E4,-2.7943772E-4,2.416E3,2.6333334E1,1.317484E1,1.753E3,6.2233735E-6,4.49239E6,8.1E2,7.168071E6,5.1E1,1.5900173E3,3.1359896E-2,1.2808675E-2,5.3700186E8,6.426937E6,3.07E2,2.7546012E0,1.3E2,4.217427E7,-1.3769596E-2,-2.3702849E-3,-3.6308873E-2,6.185E3,4.3827028E5,9E0,1.2E1,4.511144E-3,6.83E2,4.539777E5,1.5197045E0,1.5556166E-2,6.7321223E-1,2.8204132E7,-1.5473763E-2,2.6654E4,1.6368799E-2,2.7545583E-2,9.4658756E-1,3.3765998E0,-1.144036E-2,-3.1243395E-3,-1.645332E-2,-7.3663923E-3,-4.727472E-3,2.638E3,4.4970587E2,1.5416006E5,-7.40218E-3,-1.6927782E-2,3.1E1,7.9559E5,3.9595376E7,-1.0871704E-3,2.1256988E-3,-3.9635994E-3,1.4598765E0,6.278306E7,-9.239928E-3,1.3E1,4.797342E2,5.862504E2,2.2257049E-2,1.0584036E-2,1.3E1,1.6007428E7,-4.2622528E-4,-5.3398423E-3,1.4539092E-2,8.228761E-3,1.890475E-2,9.131394E-3,4.570507E-3,1.4150088E-2,1.339646E6,1.7500242E7,-1.8060554E-2,-9.643882E-3,3.3820656E7,1.9470909E2,5.1E1,2.3978092E-3,-1.7310146E-2,-8.290944E-3,1.4051679E-3,-5.22323E-3,-4.314174E-3,-2.3851106E-2,1.295313E6,2.364E3,4.5620965E6,2E0,4.4761734E5,2.8209653E-2,9.905234E-4,2.1340163E8,3.4915986E3,1.102794E3,2.0734E4,1.1995476E6,-1.6374715E-3,7.6507195E-3,4.017544E0,-1.6352473E-3,-1.0641862E-2,-4.065583E-3,6.14E2,-7.4924002E-3,1.7874026E-3,7.697704E-3,3.6966666E2,6.33E2,7.6569915E-1,-7.492316E-3,1.5364879E-2,9.07826E0,5.465798E7,6.4E1,1.125584E-3,9E0,-7.77869E-3,-1.5871506E-3,2.9308079E1,2.0589838E-2,-8.870381E-3,3.85374E8,5.57E2,6.160415E6,1.631108E6,1.030671E6,-9.562683E-3,-3.0865753E-3,-5.7975124E-3,-2.4362074E-3,-6.7767436E-3,1.6E1,1.8019229E-3,6.7304294E-3,8.7E1,3.7509E4,1.124641E0,5.4190175E-5,7.9146667E0,-1.372968E-2,2.5437157E2,9.884021E0,7.766926E-3,3.774872E7,1.1481164E-2,2.6662E4,7.177158E6,8.237181E9,3.2952412E5,-3.8058117E-3,-9.950286E-3,-3.0121743E-3,-2.0771096E-2,-8.609469E-3,-8.783466E-3,-5.0339014E-3,-2.618099E-3,1.2636482E-3,-3.939506E-3,2.731606E-5,2.0682104E-3,4.7070077E-3,1.0218954E-2,2.6412944E-3,1.0034836E-2,4.164094E-3,8.341585E-4,-4.223221E-3,-3.3481237E-3,2.1724629E-3,-8.852959E-3,-2.0215022E-3,1.9006011E-3,-3.761425E-3,7.8010594E-4,7.10566E-3,6.305115E-3,1.4021178E-2,-5.677469E-3,5.6100124E-3,3.424116E-3,9.708883E-3],"split_indices":[2,43,43,68,2,67,6,1,2,43,29,60,53,9,0,9,62,69,44,52,43,2,43,0,70,0,0,7,60,1,68,2,60,0,0,0,44,43,8,3,0,2,43,68,0,53,5,0,9,0,0,68,53,0,0,0,0,0,1,67,48,0,0,10,12,7,0,0,0,69,5,0,3,4,67,0,0,3,66,0,0,0,0,0,0,0,0,9,12,0,0,7,4,10,0,0,0,0,0,0,0,9,2,62,8,43,53,0,7,4,70,44,60,0,0,69,0,0,0,1,0,0,0,67,0,57,0,0,71,5,8,53,8,0,0,71,0,0,47,10,12,5,12,0,0,0,0,0,3,0,0,0,1,53,0,73,0,4,71,0,7,0,10,60,46,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.058E3,7.85E2,2.73E2,2.54E2,5.31E2,1.92E2,8.1E1,2.28E2,2.6E1,1.15E2,4.16E2,1.79E2,1.3E1,7.7E1,4E0,5.8E1,1.7E2,1.1E1,1.5E1,9.9E1,1.6E1,8E1,3.36E2,1.63E2,1.6E1,9E0,4E0,2.3E1,5.4E1,1.8E1,4E1,1.2E1,1.58E2,5E0,6E0,6E0,9E0,7.7E1,2.2E1,8E0,8E0,2.4E1,5.6E1,3.28E2,8E0,1.2E1,1.51E2,8E0,8E0,1.2E1,1.1E1,2.4E1,3E1,1.4E1,4E0,3.6E1,4E0,4E0,8E0,1.42E2,1.6E1,4E0,5E0,3.4E1,4.3E1,1.8E1,4E0,4E0,4E0,1.4E1,1E1,6E0,5E1,1E2,2.28E2,7E0,5E0,1.7E1,1.34E2,4E0,4E0,8E0,1.6E1,2.6E1,4E0,4E0,4E0,1.34E2,8E0,1.1E1,5E0,1.6E1,1.8E1,3.9E1,4E0,9E0,9E0,5E0,9E0,6E0,4E0,1.3E1,3.7E1,7.9E1,2.1E1,1.95E2,3.3E1,7E0,1E1,1.03E2,3.1E1,7.9E1,5.5E1,4E0,4E0,9E0,7E0,1.4E1,4E0,2.6E1,1.3E1,8E0,5E0,2.8E1,9E0,7.2E1,7E0,7E0,1.4E1,5.9E1,1.36E2,1.5E1,1.8E1,6E0,4E0,9.8E1,5E0,4E0,2.7E1,6.8E1,1.1E1,3.9E1,1.6E1,4E0,5E0,5E0,2.1E1,7E0,2.1E1,5E0,4E0,5.8E1,1.4E1,1E1,4E0,5.4E1,5E0,1.2E2,1.6E1,4E0,1.1E1,9E0,9E0,5.1E1,4.7E1,2E1,7E0,5.3E1,1.5E1,6E0,5E0,1.3E1,2.6E1,5E0,1.1E1,9E0,1.2E1,3.1E1,2.7E1,1E1,4E0,6E0,4E0,2.1E1,3.3E1,2.3E1,9.7E1,6E0,1E1,6E0,5E0,5E0,4E0,3.6E1,1.5E1,9E0,3.8E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[-9.581912E-3,-7.257247E-2,1.7942704E-1,-1.9418085E-1,-1.7906478E-2,-3.5965043E-1,2.0962237E-1,-1.6565062E-1,-3.8026592E-1,-1.2550124E-2,-2.6842918E-2,-2.3116974E-2,-1.0435031E-2,1.0108888E-1,2.8178123E-1,-1.5956992E-2,-1.3240612E-1,-5.664697E-1,-2.6462272E-1,-6.357436E-2,4.211078E-2,7.2013184E-2,2.8789636E-1,3.04593E-1,-9.930838E-2,6.466385E-2,-1.5498063E-1,-1.1142762E-2,-3.6836233E-2,-6.2297476E-3,-3.341127E-1,-4.581332E-2,-1.7267416E-1,2.0968283E-2,1.3817504E-1,5.1410656E-2,1.3865546E-2,1.6300699E-2,8.400301E-3,3.9637175E-1,2.3511447E-1,-8.551965E-3,2.9365002E-4,-8.7408796E-2,1.5608201E-2,-2.111568E-1,-1.0661881E-1,-1.8770915E-2,-8.416108E-3,-1.9438569E-2,-1.268761E-1,-2.324132E-1,-3.9913446E-2,-2.0647852E-2,7.2986744E-2,3.5533722E-2,1.8669394E-1,7.659787E-2,-8.622999E-2,3.3156115E-1,3.1152323E-2,3.555974E-1,1.6828786E-1,-1.3954026E-3,-6.8889037E-3,-2.4515393E-1,-9.3656994E-2,-7.471838E-2,-1.39719015E-2,-4.5223065E-2,3.9984792E-2,-9.4256885E-2,-2.0000146E-1,-1.6699068E-1,-1.783897E-2,1.1126087E-3,-7.103217E-2,3.2815162E-2,-7.65202E-2,1.4088078E-1,3.0322812E-2,3.80353E-3,-1.6813305E-3,1.0851359E-3,2.1000262E-1,9.989559E-3,1.5330155E-1,1.4128158E-4,-6.5795444E-3,3.895956E-1,1.8751533E-1,3.4678753E-3,3.9922535E-1,9.4451435E-2,2.5016007E-1,-1.6030985E-1,-3.1861877E-1,-6.0339727E-2,-7.7713244E-3,-1.06611826E-1,1.06922075E-1,1.0290782E-2,-7.21214E-2,-5.059808E-3,6.0526878E-2,-1.173335E-2,-7.076105E-2,-1.16590075E-2,-5.06098E-3,-1.0104442E-2,-2.8673087E-3,-8.059999E-4,-5.1009133E-3,-5.1831216E-2,1.4132656E-1,-2.2734904E-1,-8.436309E-3,2.059346E-3,1.6258979E-1,-2.3621519E-3,1.5853576E-1,1.6545305E-1,1.363455E-2,-4.0741548E-2,9.986728E-2,2.0863858E-1,7.6677024E-2,2.731075E-1,2.0024195E-2,3.3940226E-4,2.6874238E-1,1.0049854E-2,2.06835E-2,1.3586044E-1,-6.457043E-3,2.5187032E-3,2.84829E-1,-1.9281705E-1,-8.251519E-2,-6.3519846E-3,-3.5062477E-1,-3.810116E-3,-1.8515399E-4,-2.0755851E-1,-7.2851844E-2,8.5797915E-3,-1.87239E-3,3.1531107E-2,-4.5437623E-2,5.6683127E-2,-9.2034005E-2,4.04152E-2,9.143933E-3,-9.0005174E-2,1.2483862E-3,-1.189828E-1,1.0979742E-2,1.9397499E-1,-1.7005035E-3,-8.496999E-2,-2.6820421E-2,6.3791305E-2,-5.7213634E-2,1.8103935E-3,1.8076679E-1,3.0098017E-2,-1.0225083E-1,9.999024E-3,2.9828758E-3,9.143727E-3,3.5519006E-3,-1.002226E-1,1.0105338E-3,9.627098E-3,3.5622675E-2,5.269908E-3,1.091391E-2,2.9007126E-2,7.678993E-3,1.4638257E-2,6.860326E-3,5.6368955E-3,1.5483941E-2,1.6306031E-1,-5.0777383E-4,8.077706E-3,1.46527635E-2,-1.1176726E-2,-5.7484065E-3,-6.5746172E-3,-8.746909E-4,-1.8007085E-2,-9.23389E-3,-4.8850356E-3,-1.2704869E-2,-2.1737895E-3,-9.500405E-3,1.6851719E-4,4.5868917E-3,-5.4463306E-3,1.0696977E-3,-1.7372688E-3,5.013853E-3,-3.3590717E-3,-1.5900875E-2,2.7876692E-3,-2.6236782E-3,-1.9486265E-3,-6.2377723E-3,-2.3766218E-3,2.4804354E-3,-1.2117583E-2,-2.527635E-3,-2.295875E-3,3.259657E-3,4.09689E-3,1.1639896E-2,-8.153879E-3,1.467654E-3,-5.6455034E-4,4.4573303E-3,-7.0261434E-3,-1.6735867E-4,3.938377E-3,9.00426E-3,2.5920656E-3,-3.2891848E-3,-1.8424853E-3,-8.389362E-3,-2.5470501E-3,-6.6360096E-3,4.0666E-3,-1.5884526E-3,3.1150458E-3,-6.5819285E-4,1.2904538E-2,5.1943446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,49,51,53,55,57,-1,-1,-1,59,61,-1,-1,63,-1,65,67,-1,-1,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,-1,101,103,105,107,109,-1,-1,111,113,115,117,119,-1,-1,-1,121,123,125,-1,-1,127,129,-1,131,133,135,137,139,141,-1,143,145,147,149,-1,151,-1,153,-1,-1,-1,-1,-1,-1,155,157,159,161,-1,163,165,167,169,-1,171,173,175,177,179,-1,-1,181,-1,-1,183,-1,-1,185,187,189,-1,191,-1,-1,193,195,-1,-1,197,199,201,203,205,-1,207,209,211,213,215,-1,217,-1,219,221,-1,223,225,227,-1,-1,-1,-1,229,-1,-1,231,-1,-1,233,-1,-1,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2240258E1,5.1220093E0,4.2587376E0,1.2058716E0,1.6003832E0,1.505487E-1,1.8824387E0,1.2031603E0,5.1762867E-1,1.4780744E0,0E0,0E0,0E0,5.169921E-1,1.2994747E0,0E0,8.0190086E-1,6.3375187E-1,1.4432836E-1,5.230262E-1,5.170484E-1,3.934447E-1,2.5388718E-2,7.867861E-1,8.277353E-2,8.188601E-1,4.121716E-1,0E0,0E0,0E0,6.858122E-2,5.042422E-1,2.921015E-1,4.5828137E-1,2.2244811E-1,2.8741452E-1,0E0,0E0,0E0,8.21187E-1,5.9232616E-1,0E0,0E0,4.1383535E-2,0E0,2.6864958E-1,5.27087E-1,0E0,0E0,2.7760446E-1,1.1949545E-1,1.969943E-1,2.7599968E-2,3.5498926E-1,2.6801097E-1,5.684987E-2,1.1299467E-1,3.509967E-1,6.56547E-2,3.3656597E-1,0E0,3.127277E-1,3.0281007E-1,0E0,0E0,2.9797363E-1,3.4562618E-2,4.574708E-1,0E0,1.8888918E-1,1.7554389E-1,1.3964123E-1,5.2874148E-2,9.497136E-2,0E0,0E0,1.706541E-2,5.6761605E-1,5.9368473E-1,6.801742E-2,2.4762201E-1,0E0,0E0,0E0,5.3560495E-2,1.7733341E-1,1.19687736E-1,0E0,0E0,5.5248737E-2,2.3217899E-1,0E0,1.1400533E-1,3.0994967E-1,1.5403879E-1,5.489856E-2,1.1990929E-1,1.7123844E-2,0E0,2.098043E-1,1.4694121E-1,5.1147655E-2,2.2322768E-1,0E0,1.2740256E-1,0E0,5.2771196E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.4893135E-1,2.6035976E-1,8.5583895E-1,1.4809561E-1,0E0,5.7966113E-2,1.5871291E-1,5.13137E-2,4.2295992E-2,0E0,9.53024E-2,9.017858E-2,2.0031273E-2,5.9202842E-2,1.8362343E-2,0E0,0E0,6.630117E-2,0E0,0E0,1.05645E-1,0E0,0E0,1.9231796E-2,3.3710003E-2,3.532484E-2,0E0,6.3732624E-2,0E0,0E0,8.0796E-2,1.6504192E-1,0E0,0E0,5.820383E-2,6.2363863E-2,6.2739775E-2,3.3556515E-1,8.864807E-2,0E0,5.889103E-2,2.7631111E-2,1.4222875E-1,7.218829E-2,1.0585451E-1,0E0,1.5518844E-1,0E0,4.3420322E-2,1.265544E-1,0E0,1.7138124E-2,1.0147102E-1,5.213912E-2,0E0,0E0,0E0,0E0,1.7530844E-2,0E0,0E0,3.9709907E-2,0E0,0E0,1.9240392E-2,0E0,0E0,0E0,0E0,0E0,9.808725E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,43,43,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,76,76,77,77,78,78,79,79,80,80,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,104,104,106,106,113,113,114,114,115,115,116,116,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,127,127,130,130,133,133,136,136,137,137,138,138,140,140,143,143,144,144,147,147,148,148,149,149,150,150,151,151,153,153,154,154,155,155,156,156,157,157,159,159,161,161,162,162,164,164,165,165,166,166,171,171,174,174,177,177,183,183],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,50,52,54,56,58,-1,-1,-1,60,62,-1,-1,64,-1,66,68,-1,-1,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,-1,102,104,106,108,110,-1,-1,112,114,116,118,120,-1,-1,-1,122,124,126,-1,-1,128,130,-1,132,134,136,138,140,142,-1,144,146,148,150,-1,152,-1,154,-1,-1,-1,-1,-1,-1,156,158,160,162,-1,164,166,168,170,-1,172,174,176,178,180,-1,-1,182,-1,-1,184,-1,-1,186,188,190,-1,192,-1,-1,194,196,-1,-1,198,200,202,204,206,-1,208,210,212,214,216,-1,218,-1,220,222,-1,224,226,228,-1,-1,-1,-1,230,-1,-1,232,-1,-1,234,-1,-1,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,2.6E1,5.9E1,2.5330253E-2,9.616803E2,3.9E1,3.355864E0,9.48E2,-2.6842918E-2,-2.3116974E-2,-1.0435031E-2,5.6530495E6,3.206931E2,-1.5956992E-2,2.6333334E1,4E0,1.0772152E1,1.137832E-7,2.8971518E6,3.1639907E0,2.6846153E-1,5.932344E1,3.6086957E0,3.25E2,1.3823239E2,-1.1142762E-2,-3.6836233E-2,-6.2297476E-3,8.215373E6,2E0,2.8266037E2,2.437372E6,5.1942E4,2E0,1.3865546E-2,1.6300699E-2,8.400301E-3,1.9705E4,1.5277778E0,-8.551965E-3,2.9365002E-4,1.31306E5,1.5608201E-2,4.39776E6,4.217427E7,-1.8770915E-2,-8.416108E-3,2.732439E2,1.9E1,5.4E1,8.303459E5,3.72381E5,3.64E2,4.284608E6,4.745397E6,4.1032645E6,1.6E1,9.44363E5,3.1152323E-2,3.714144E8,5.620512E8,-1.3954026E-3,-6.8889037E-3,1E0,5.6842103E0,4.7866177E2,-1.39719015E-2,1.1772152E0,1.0909091E0,1.43E2,5.0710382E0,2.1578947E2,-1.783897E-2,1.1126087E-3,1.7352032E1,8.583682E4,1.048796E6,1.4979E4,5.7420593E2,3.80353E-3,-1.6813305E-3,1.0851359E-3,2.1111112E0,4.8709216E5,3.0977E4,1.4128158E-4,-6.5795444E-3,1.2562814E0,5.2141205E6,3.4678753E-3,7.659164E1,1.28636E7,1E0,2.590909E0,6.642857E0,7.2140925E5,-7.7713244E-3,1.9595902E3,6.743651E7,7.09E2,7.9634375E2,-5.059808E-3,3.7029E4,-1.173335E-2,4.629112E6,-1.16590075E-2,-5.06098E-3,-1.0104442E-2,-2.8673087E-3,-8.059999E-4,-5.1009133E-3,3.693316E7,8.695652E0,9.706078E4,3.1226995E0,2.059346E-3,5.4674416E7,6.4E1,4.857143E0,7E0,1.363455E-2,2.1340163E8,1.3333334E0,6.978874E5,6.624E3,1.0609756E0,2.0024195E-2,3.3940226E-4,2.5205562E5,1.0049854E-2,2.06835E-2,1.9407146E7,-6.457043E-3,2.5187032E-3,2.4344263E0,6.44E2,3.62E2,-6.3519846E-3,4.39E2,-3.810116E-3,-1.8515399E-4,9.6E1,4.76E2,8.5797915E-3,-1.87239E-3,5.57E2,1.994E3,1.1117E4,2.8302418E10,1E0,9.143933E-3,3.440772E7,2.08E2,2.3257812E2,2.3818183E0,1.609E3,-1.7005035E-3,1.373E3,-2.6820421E-2,1.071E3,1.058E3,1.8103935E-3,8.7578294E5,2.923409E7,8.7692904E7,9.999024E-3,2.9828758E-3,9.143727E-3,3.5519006E-3,1.5E1,1.0105338E-3,9.627098E-3,2.0394794E8,5.269908E-3,1.091391E-2,4.771E3,7.678993E-3,1.4638257E-2,6.860326E-3,5.6368955E-3,1.5483941E-2,2E0,-5.0777383E-4,8.077706E-3,1.46527635E-2,-1.1176726E-2,-5.7484065E-3,-6.5746172E-3,-8.746909E-4,-1.8007085E-2,-9.23389E-3,-4.8850356E-3,-1.2704869E-2,-2.1737895E-3,-9.500405E-3,1.6851719E-4,4.5868917E-3,-5.4463306E-3,1.0696977E-3,-1.7372688E-3,5.013853E-3,-3.3590717E-3,-1.5900875E-2,2.7876692E-3,-2.6236782E-3,-1.9486265E-3,-6.2377723E-3,-2.3766218E-3,2.4804354E-3,-1.2117583E-2,-2.527635E-3,-2.295875E-3,3.259657E-3,4.09689E-3,1.1639896E-2,-8.153879E-3,1.467654E-3,-5.6455034E-4,4.4573303E-3,-7.0261434E-3,-1.6735867E-4,3.938377E-3,9.00426E-3,2.5920656E-3,-3.2891848E-3,-1.8424853E-3,-8.389362E-3,-2.5470501E-3,-6.6360096E-3,4.0666E-3,-1.5884526E-3,3.1150458E-3,-6.5819285E-4,1.2904538E-2,5.1943446E-3],"split_indices":[2,43,17,3,3,53,67,2,73,2,0,0,0,43,73,0,62,8,73,52,43,57,53,73,69,2,67,0,0,0,12,32,4,9,1,6,0,0,0,2,68,0,0,9,0,9,60,0,0,67,8,0,43,9,10,43,60,60,3,9,0,7,7,0,0,8,71,67,0,68,68,2,69,70,0,0,73,48,9,1,67,0,0,0,68,43,44,0,0,68,60,0,73,60,14,68,73,60,0,48,7,2,48,0,1,0,9,0,0,0,0,0,0,7,61,48,69,0,7,8,69,8,0,7,68,43,2,68,0,0,48,0,0,9,0,0,69,1,1,0,2,0,0,10,0,0,0,10,12,1,46,19,0,7,0,48,68,2,0,2,0,2,10,0,60,9,5,0,0,0,0,3,0,0,7,0,0,2,0,0,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.026E3,7.7E2,2.56E2,2.38E2,5.32E2,1.3E1,2.43E2,2.08E2,3E1,5.28E2,4E0,5E0,8E0,9.8E1,1.45E2,3.1E1,1.77E2,1E1,2E1,2.73E2,2.55E2,8.6E1,1.2E1,1.37E2,8E0,1.8E1,1.59E2,5E0,5E0,8E0,1.2E1,2.36E2,3.7E1,2.1E2,4.5E1,8E1,6E0,6E0,6E0,5.7E1,8E1,4E0,4E0,1.2E1,6E0,7.2E1,8.7E1,7E0,5E0,1.79E2,5.7E1,2.5E1,1.2E1,1.17E2,9.3E1,1.5E1,3E1,6.8E1,1.2E1,4.8E1,9E0,2.7E1,5.3E1,7E0,5E0,5.5E1,1.7E1,7.6E1,1.1E1,1.25E2,5.4E1,4.1E1,1.6E1,1.9E1,6E0,4E0,8E0,6E1,5.7E1,3.5E1,5.8E1,9E0,6E0,4E0,2.6E1,3.7E1,3.1E1,5E0,7E0,3.3E1,1.5E1,4E0,2.3E1,2.9E1,2.4E1,2.7E1,2.8E1,1.3E1,4E0,6.5E1,1.1E1,4.1E1,8.4E1,6E0,4.8E1,4E0,3.7E1,9E0,7E0,1.2E1,7E0,4E0,4E0,3.4E1,2.6E1,1.7E1,4E1,7E0,2.8E1,4.7E1,1.1E1,1.9E1,7E0,2.4E1,1.3E1,1.7E1,1.4E1,1.1E1,2.2E1,5E0,1E1,6E0,1.7E1,2.5E1,4E0,4E0,2E1,1.8E1,9E0,5E0,2.3E1,9E0,4E0,1.5E1,5E1,7E0,4E0,3E1,1.1E1,1.1E1,7.3E1,4.3E1,5E0,2.9E1,8E0,1.6E1,1.8E1,2E1,6E0,1.3E1,4E0,1.6E1,2.4E1,4E0,2.4E1,3.6E1,1.1E1,6E0,5E0,1.3E1,6E0,1.2E1,1.2E1,4E0,9E0,5E0,1.2E1,1E1,4E0,7E0,4E0,4E0,6E0,2.1E1,4E0,6E0,1.4E1,9E0,9E0,4E0,5E0,1.7E1,6E0,7E0,8E0,4.3E1,7E0,2.2E1,8E0,5E0,6E0,4E0,7E0,6.9E1,4E0,3.6E1,7E0,1.5E1,1.4E1,4E0,4E0,4E0,1.2E1,9E0,9E0,8E0,1.2E1,7E0,6E0,5E0,1.1E1,8E0,1.6E1,4E0,2E1,2.9E1,7E0,7E0,4E0,7E0,5E0,5E0,4E0,5E0,5E0,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[8.72172E-3,-5.2378815E-2,1.8862124E-1,-1.4497288E-1,2.8107746E-2,2.2104456E-1,-4.3223172E-2,-2.8720894E-1,-1.0349912E-1,-5.3254906E-2,5.86618E-2,8.073839E-2,2.7825445E-1,-2.4189971E-1,8.807774E-2,-2.4866153E-1,-2.9690595E-2,-1.7638516E-1,-4.863835E-2,-9.128427E-2,8.682048E-2,4.8905227E-2,1.303921E-2,3.1544287E-2,2.4589539E-1,5.186504E-1,2.4941117E-1,-3.5997012E-3,-1.4142607E-2,1.5876757E-2,2.7034823E-3,-1.761185E-2,-2.2367401E-1,-1.62861E-1,-2.0514648E-2,2.0292308E-2,-7.1081586E-2,-2.1198758E-1,-4.1937176E-2,4.346774E-2,9.684896E-3,-1.4100128E-1,5.9390966E-2,1.0537662E-1,-4.0721823E-2,4.8275758E-3,1.4143654E-2,1.3774401E-2,3.0353699E-2,2.1816213E-2,2.1961127E-1,-5.3911596E-3,3.973649E-3,-1.5521282E-1,-2.4970925E-1,-1.4594304E-2,-1.430656E-1,-7.994017E-3,-1.3078807E-1,-8.0597945E-2,-3.4760985E-1,5.8276732E-3,-6.1842825E-2,3.9057387E-3,-4.0861764E-3,-5.4247882E-2,-1.40287345E-2,6.439775E-2,-9.971807E-3,-1.0422347E-3,1.5369838E-1,2.283647E-3,-1.0644382E-2,2.4693705E-1,9.715158E-2,-1.4571238E-3,-8.978063E-3,-1.2164489E-2,-4.2037857E-3,-8.9428425E-2,-2.0278858E-1,5.5312263E-3,-2.3124851E-2,-9.876157E-2,-2.8085992E-1,-5.663785E-3,9.016345E-4,-2.270514E-2,-8.068586E-3,-2.459085E-2,-1.1584179E-1,8.762837E-4,-4.564232E-3,4.2571846E-2,1.850129E-1,2.9195794E-3,8.805893E-3,4.2759046E-2,-4.4508157E-3,2.1421364E-1,4.018031E-1,1.703014E-2,1.22557245E-2,4.4476306E-3,-1.0588076E-1,-1.3681443E-1,-2.779559E-1,3.590126E-2,-6.642312E-2,-1.7129269E-1,-6.507932E-2,-1.5911153E-2,-8.186132E-3,-4.507132E-2,2.9461032E-3,-9.514544E-3,-5.4056033E-2,2.4281483E-2,1.2361709E-1,2.6249638E-1,8.5557684E-2,4.2141047E-3,-2.826333E-4,1.0423103E-1,2.4100588E-1,2.1852339E-2,1.0741991E-2,6.1002336E-3,-8.818392E-2,-2.001791E-1,-7.799874E-2,-1.7800367E-1,-7.733494E-4,-4.945717E-3,-3.118713E-1,5.4359782E-2,-2.9369479E-3,-1.3686177E-1,-5.772263E-3,-1.8939708E-1,-3.654349E-3,-3.6622614E-2,-7.570231E-3,-6.185178E-2,1.3657555E-3,-7.4566645E-3,1.6373991E-3,6.5352987E-3,1.1890642E-1,1.5681271E-1,-1.1835407E-2,1.6850555E-2,2.0135957E-1,-8.26217E-5,6.313497E-3,1.639012E-3,-1.9929293E-3,4.280685E-2,7.74868E-3,2.532417E-1,3.1546345E-3,2.6112532E-3,-8.966066E-3,-2.8425588E-3,-1.2502305E-2,-2.5173726E-3,-6.3366937E-3,-3.0830633E-3,-9.316043E-3,-1.608188E-2,-7.1748444E-3,8.384456E-4,3.765922E-3,-1.7980152E-3,-8.435483E-3,2.7885148E-3,-1.8783283E-3,-1.0050917E-2,-3.877687E-3,-6.5479623E-3,-3.4059043E-4,-1.1249786E-3,-3.8220037E-3,-6.0070277E-4,2.7662155E-3,6.9019375E-3,8.042589E-5,5.2760583E-3,9.539327E-3,2.090719E-3,-3.074931E-3,1.1207516E-2,5.9967865E-3,4.18515E-3,-1.5084115E-3,5.0713434E-3,1.2412042E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,43,45,47,49,-1,-1,-1,51,-1,53,55,-1,-1,57,59,61,63,-1,65,67,69,71,-1,-1,-1,-1,-1,73,-1,-1,75,77,-1,79,81,83,85,87,-1,89,-1,-1,91,-1,93,-1,-1,95,97,-1,99,101,-1,-1,-1,-1,103,105,-1,107,109,111,-1,-1,-1,-1,113,115,-1,-1,117,119,-1,-1,121,-1,123,125,127,-1,-1,129,131,133,135,137,139,141,-1,-1,143,-1,-1,145,147,149,151,153,-1,155,157,159,-1,-1,-1,161,163,165,167,-1,-1,169,171,-1,173,175,177,-1,179,-1,181,-1,-1,-1,183,185,187,189,-1,191,-1,-1,-1,-1,193,-1,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.101283E1,5.578603E0,1.9189138E0,2.0231476E0,9.9999356E-1,1.7675323E0,8.560711E-1,9.0894794E-1,1.0750372E0,5.9389293E-1,6.2315285E-1,5.2847695E-1,9.6618366E-1,1.1305791E-1,4.354735E-1,1.5321207E-1,0E0,3.475933E-1,1.7393423E0,5.093129E-1,1.1867106E-1,5.679323E-1,0E0,2.8174105E-1,9.5800996E-2,2.9226875E-1,8.4577847E-1,0E0,0E0,0E0,1.691855E-1,0E0,7.353306E-2,2.941072E-1,0E0,0E0,5.639649E-1,4.1746747E-1,2.1858262E-1,1.10424176E-1,0E0,1.951932E-1,3.8183808E-1,1.6420215E-1,2.2077356E-1,0E0,0E0,0E0,0E0,0E0,4.0433693E-1,0E0,0E0,9.0636015E-2,7.0994616E-2,0E0,3.0540562E-1,1.457744E-1,3.4234512E-1,6.1256036E-2,2.1274018E-1,0E0,1.1181073E-1,0E0,0E0,3.7750762E-2,0E0,6.844804E-1,0E0,0E0,5.142185E-2,9.593152E-2,0E0,4.4097376E-1,3.322513E-1,0E0,0E0,0E0,0E0,1.7822868E-1,1.9504929E-1,0E0,1.7361847E-1,1.476481E-1,2.4611235E-2,0E0,0E0,0E0,0E0,6.6212535E-2,1.1691299E-1,0E0,0E0,3.2904413E-1,2.8578675E-1,0E0,0E0,3.548234E-2,0E0,2.341876E-1,9.4928026E-2,2.2946034E-1,0E0,0E0,1.1847687E-1,1.3055676E-1,8.833504E-2,5.6703277E-2,1.658244E-1,1.840043E-2,1.23305246E-1,0E0,0E0,3.8147766E-2,0E0,0E0,1.4983103E-1,3.0820155E-1,1.8620956E-1,6.407273E-2,8.5050195E-2,0E0,1.6855368E-2,6.69201E-2,1.3064575E-1,0E0,0E0,0E0,1.6419947E-1,9.077996E-2,4.9550608E-2,3.9165437E-2,0E0,0E0,4.23671E-2,2.3103848E-2,0E0,7.351133E-2,5.32076E-2,2.9256344E-2,0E0,1.1271305E-1,0E0,1.6775578E-2,0E0,0E0,0E0,1.6335548E-1,1.03336334E-1,4.6262145E-2,3.1129247E-2,0E0,1.5717804E-2,0E0,0E0,0E0,0E0,4.2618744E-2,0E0,1.1433411E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,30,30,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,50,50,53,53,54,54,56,56,57,57,58,58,59,59,60,60,62,62,65,65,67,67,70,70,71,71,73,73,74,74,79,79,80,80,82,82,83,83,84,84,89,89,90,90,93,93,94,94,97,97,99,99,100,100,101,101,104,104,105,105,106,106,107,107,108,108,109,109,110,110,113,113,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,128,128,129,129,130,130,131,131,134,134,135,135,137,137,138,138,139,139,141,141,143,143,147,147,148,148,149,149,150,150,152,152,157,157,159,159],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,44,46,48,50,-1,-1,-1,52,-1,54,56,-1,-1,58,60,62,64,-1,66,68,70,72,-1,-1,-1,-1,-1,74,-1,-1,76,78,-1,80,82,84,86,88,-1,90,-1,-1,92,-1,94,-1,-1,96,98,-1,100,102,-1,-1,-1,-1,104,106,-1,108,110,112,-1,-1,-1,-1,114,116,-1,-1,118,120,-1,-1,122,-1,124,126,128,-1,-1,130,132,134,136,138,140,142,-1,-1,144,-1,-1,146,148,150,152,154,-1,156,158,160,-1,-1,-1,162,164,166,168,-1,-1,170,172,-1,174,176,178,-1,180,-1,182,-1,-1,-1,184,186,188,190,-1,192,-1,-1,-1,-1,194,-1,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.414E3,4.539777E5,2.856934E7,7.7E1,1E0,8.625455E2,1.3655363E2,1.05039425E1,5.41E2,6.33E2,7.82261E6,3.5510652E6,1.059448E7,3.6451373E8,1.2083E4,9.2058825E-1,-2.9690595E-2,3.1E1,2.00087E5,1.9750606E2,9.50283E5,1.6262903E2,1.303921E-2,1.5382514E1,5.2692295E6,5.467E3,7.714286E0,-3.5997012E-3,-1.4142607E-2,1.5876757E-2,1.01E3,-1.761185E-2,3.5675676E0,5.4E1,-2.0514648E-2,2.0292308E-2,4.064E3,1.0526316E0,1E1,2.5818555E2,9.684896E-3,1.5222145E2,2.3544883E8,1.1656439E9,1E0,4.8275758E-3,1.4143654E-2,1.3774401E-2,3.0353699E-2,2.1816213E-2,1.2792593E2,-5.3911596E-3,3.973649E-3,1.2E1,3E0,-1.4594304E-2,8.960784E0,3.2931E4,4.9735293E0,4.39E2,5.4E0,5.8276732E-3,5.524E3,3.9057387E-3,-4.0861764E-3,2.4481E4,-1.40287345E-2,5.817547E2,-9.971807E-3,-1.0422347E-3,1.29443414E5,6.0641945E1,-1.0644382E-2,4.73909E0,1.5E1,-1.4571238E-3,-8.978063E-3,-1.2164489E-2,-4.2037857E-3,7.3333335E-1,4.612245E0,5.5312263E-3,1E0,8.75E2,7.5391846E2,-5.663785E-3,9.016345E-4,-2.270514E-2,-8.068586E-3,1.535E3,1.5E1,8.762837E-4,-4.564232E-3,8.237181E9,3.8275862E0,2.9195794E-3,8.805893E-3,4.6816683E-1,-4.4508157E-3,1.000501E6,1.7655972E0,2.0277777E0,1.22557245E-2,4.4476306E-3,1E0,8E0,4.7721977E4,1E0,8.83425E7,2.026453E6,2.2587704E7,-1.5911153E-2,-8.186132E-3,1.28636E7,2.9461032E-3,-9.514544E-3,1.2218E4,1.427636E5,2.9411764E0,1.500502E6,1.98E2,4.2141047E-3,1.1E1,7.607E3,3.0546486E8,2.1852339E-2,1.0741991E-2,6.1002336E-3,1.6677892E2,1.1E1,3.373828E-1,2.992648E-2,-7.733494E-4,-4.945717E-3,1.9470909E2,1.2748E4,-2.9369479E-3,6.76E2,1.75E0,2.3847478E5,-3.654349E-3,2.4177586E2,-7.570231E-3,1.4E1,1.3657555E-3,-7.4566645E-3,1.6373991E-3,9.750085E6,2.2681375E6,1.7903225E0,3.3E1,1.6850555E-2,6.726722E2,-8.26217E-5,6.313497E-3,1.639012E-3,-1.9929293E-3,3.019898E8,7.74868E-3,1.0089981E3,3.1546345E-3,2.6112532E-3,-8.966066E-3,-2.8425588E-3,-1.2502305E-2,-2.5173726E-3,-6.3366937E-3,-3.0830633E-3,-9.316043E-3,-1.608188E-2,-7.1748444E-3,8.384456E-4,3.765922E-3,-1.7980152E-3,-8.435483E-3,2.7885148E-3,-1.8783283E-3,-1.0050917E-2,-3.877687E-3,-6.5479623E-3,-3.4059043E-4,-1.1249786E-3,-3.8220037E-3,-6.0070277E-4,2.7662155E-3,6.9019375E-3,8.042589E-5,5.2760583E-3,9.539327E-3,2.090719E-3,-3.074931E-3,1.1207516E-2,5.9967865E-3,4.18515E-3,-1.5084115E-3,5.0713434E-3,1.2412042E-2],"split_indices":[2,43,60,44,29,67,71,69,2,0,43,43,12,7,9,68,0,8,5,70,1,4,0,71,60,2,71,0,0,0,8,0,69,2,0,0,44,71,3,4,0,4,12,46,6,0,0,0,0,0,73,0,0,3,8,0,73,9,68,2,73,0,44,0,0,12,0,67,0,0,48,73,0,50,3,0,0,0,0,68,69,0,26,2,70,0,0,0,0,2,3,0,0,46,69,0,0,53,0,43,49,68,0,0,13,8,43,19,7,9,12,0,0,60,0,0,44,48,68,9,10,0,3,2,47,0,0,0,73,3,53,53,0,0,4,1,0,2,68,43,0,4,0,3,0,0,0,9,43,68,8,0,4,0,0,0,0,7,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.47E2,2.53E2,3.47E2,4E2,2.22E2,3.1E1,7.7E1,2.7E2,1.09E2,2.91E2,6.5E1,1.57E2,1.2E1,1.9E1,7.1E1,6E0,1.15E2,1.55E2,8.6E1,2.3E1,2.8E2,1.1E1,5.1E1,1.4E1,1.5E1,1.42E2,4E0,8E0,4E0,1.5E1,9E0,6.2E1,1.11E2,4E0,6E0,1.49E2,2.4E1,6.2E1,1.8E1,5E0,1.4E1,2.66E2,2.5E1,2.6E1,5E0,9E0,7E0,8E0,1.5E1,1.27E2,6E0,9E0,1.9E1,4.3E1,1.1E1,1E2,7.3E1,7.6E1,1.3E1,1.1E1,6E0,5.6E1,1.4E1,4E0,1E1,4E0,2.62E2,4E0,7E0,1.8E1,2.2E1,4E0,1.03E2,2.4E1,5E0,1.4E1,3.9E1,4E0,5.4E1,4.6E1,7E0,6.6E1,6.4E1,1.2E1,9E0,4E0,5E0,6E0,3.4E1,2.2E1,4E0,6E0,2.23E2,3.9E1,6E0,1.2E1,1.6E1,6E0,8.7E1,1.6E1,1.7E1,7E0,4E0,5E1,2.6E1,2E1,2.8E1,3.8E1,1.9E1,4.5E1,6E0,6E0,2.8E1,6E0,8E0,1.4E1,1.83E2,4E1,2.1E1,1.8E1,7E0,9E0,1.8E1,6.9E1,1E1,6E0,8E0,9E0,1E1,4E1,1.9E1,7E0,4E0,1.6E1,2.4E1,4E0,1.7E1,2.1E1,1.5E1,4E0,3.6E1,9E0,2.3E1,5E0,6E0,8E0,1.55E2,2.8E1,3.2E1,8E0,6E0,1.5E1,7E0,1.1E1,5E0,4E0,1E1,8E0,6.4E1,5E0,4E0,5E0,4E0,6E0,3E1,1E1,4E0,1.5E1,1.2E1,4E0,1.1E1,1.3E1,6E0,1.1E1,7E0,1.4E1,1.1E1,4E0,7E0,2.9E1,9E0,1.4E1,1.14E2,4.1E1,2.2E1,6E0,1.9E1,1.3E1,4E0,4E0,8E0,7E0,6E0,4E0,7E0,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-6.840664E-3,-7.564314E-2,1.9535907E-1,-1.9300523E-1,-2.1521175E-2,-3.0387625E-1,2.1447061E-1,-1.6691664E-1,-3.4692943E-1,-8.0028E-2,2.1416238E-2,-1.721789E-2,-7.416122E-3,5.991051E-1,1.8573333E-1,-2.0868088E-1,-8.339015E-2,-4.9642673E-1,-2.4282059E-1,-5.5640046E-2,-2.6173565E-1,1.0791406E-2,1.7121804E-1,1.8752169E-2,3.5375725E-2,6.153804E-2,2.328578E-1,-1.9223677E-1,-1.9543128E-2,1.2934754E-2,-1.2123926E-1,-1.0904279E-2,-3.5152256E-2,-1.9409508E-3,-2.851105E-1,-1.4052862E-1,-1.5613228E-2,-1.3557872E-1,-2.7016891E-2,-2.4180397E-2,6.0915094E-2,6.5613404E-2,1.1626313E-2,3.2972004E-2,9.303139E-3,2.5004095E-1,-9.377705E-3,-2.119346E-1,-7.990358E-2,-2.077344E-1,-3.6266204E-2,-1.5910942E-2,-6.5945326E-3,-1.8337767E-1,-9.328806E-2,1.5711425E-2,-8.075538E-2,-9.197322E-3,-5.216909E-2,-1.1941902E-1,-7.477994E-3,8.0042176E-2,-1.517489E-1,-5.2269764E-4,5.4877093E-3,1.5570532E-1,6.157E-3,2.2106372E-1,4.368353E-1,-2.1859212E-1,-1.3124169E-3,-1.2622821E-2,-8.669574E-3,-2.4108933E-1,-2.5025285E-3,1.6260386E-2,-1.4757274E-1,-3.908957E-3,-2.3108338E-1,-4.4649086E-4,-1.3085946E-1,-1.741995E-2,6.118278E-2,-1.1001991E-1,6.4516813E-3,-5.695077E-3,2.0137683E-3,-2.7138608E-2,-2.8045768E-1,9.3885595E-3,-1.0671488E-1,1.6980128E-1,3.4680944E-2,-1.2606748E-2,-1.1788013E-3,1.3482728E-3,1.0865502E-2,3.5373196E-2,-9.267539E-2,2.7034536E-1,1.5341043E-1,1.0501004E-2,2.1841403E-2,-2.0876442E-1,-1.7341226E-2,1.9915015E-3,-2.7630979E-3,-1.7403053E-1,-1.408249E-2,-2.2346012E-2,5.0646264E-2,-9.9995565E-3,-3.571021E-3,-1.3162834E-2,-1.562435E-1,-3.3883373E-3,-7.0597418E-3,5.0022574E-3,-3.538772E-2,2.5663525E-4,1.1904978E-1,-8.802719E-3,-3.23352E-2,1.7780726E-3,-2.1295685E-3,1.3409492E-3,-7.749313E-3,-1.9454127E-2,-3.868254E-3,5.086483E-4,8.497947E-3,-1.9072951E-1,-4.0342066E-2,1.0185063E-1,2.6930583E-1,8.930754E-2,-8.0569506E-2,7.236593E-2,-5.8642894E-2,1.1766521E-3,-9.026096E-3,1.7983173E-1,3.1910643E-1,1.029651E-1,3.3951056E-1,-1.3897629E-2,-1.907007E-1,-1.6402863E-3,-1.0218113E-2,7.861163E-4,-2.4435818E-3,4.6941247E-3,2.8159659E-4,-1.0388775E-2,-2.8447777E-3,-6.0307216E-2,7.929756E-2,-3.3707142E-2,5.030587E-3,7.5726593E-3,5.6892954E-4,6.3205147E-3,-6.1954847E-3,-7.278001E-3,8.822312E-3,-4.1713733E-3,-1.1689367E-2,8.8830624E-4,-3.185609E-3,1.4517672E-1,8.4653875E-4,1.6219297E-2,7.581044E-3,3.3748668E-2,1.530615E-1,1.1560535E-2,-2.000307E-2,3.919705E-2,5.767181E-3,-3.920738E-4,-5.4114377E-3,-3.4258162E-5,2.3252983E-1,2.515274E-1,3.9982662E-1,-2.1985779E-2,1.3787624E-1,8.8514695E-3,1.884963E-2,-1.124831E-2,-7.3551587E-3,-4.0056063E-3,1.2728353E-3,6.5676756E-3,2.6573543E-5,-3.5093222E-3,3.5097462E-4,3.1461755E-3,-1.1642853E-3,6.547251E-5,6.554649E-3,2.2026922E-3,8.144722E-3,-1.2287642E-3,3.7268945E-3,8.080138E-3,2.866218E-3,2.8895813E-3,-5.443667E-3,-1.6372912E-3,2.9130038E-3,1.332312E-2,2.5725174E-3,1.2843513E-2,7.203258E-3,2.037287E-2,1.0006106E-2,-6.540608E-3,5.5992655E-3,1.036429E-2,3.2748797E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,-1,-1,-1,51,53,55,57,-1,59,61,63,-1,65,-1,67,-1,69,71,73,75,-1,-1,77,79,81,83,-1,85,87,89,91,93,-1,-1,95,97,99,101,103,-1,105,-1,107,-1,109,111,-1,113,-1,115,117,119,121,123,-1,-1,125,127,129,131,133,135,-1,-1,-1,-1,137,139,141,143,-1,-1,145,-1,-1,-1,147,-1,149,151,-1,-1,-1,153,-1,-1,-1,155,157,159,-1,161,-1,-1,-1,-1,-1,-1,163,-1,165,167,169,171,173,175,177,179,-1,-1,181,183,185,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,191,193,195,-1,-1,-1,197,-1,-1,199,-1,-1,-1,-1,201,-1,-1,-1,203,205,207,-1,209,-1,-1,-1,-1,211,213,215,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4246592E1,4.8486733E0,2.5458717E0,9.098673E-1,1.3180709E0,3.8817823E-2,2.6631165E0,7.081895E-1,4.0863276E-1,9.6922505E-1,4.8041293E-1,0E0,0E0,2.364521E-1,1.3610392E0,4.0507698E-1,1.010421E0,7.156551E-1,1.7484891E-1,6.671763E-1,9.8036885E-1,4.9930102E-1,1.5338236E-1,0E0,0E0,2.569952E-1,1.3239107E0,2.7071905E-1,0E0,0E0,4.7020048E-1,0E0,0E0,0E0,1.0277665E-1,1.1041856E-1,2.7674374E-1,9.554705E-2,0E0,2.663469E-1,4.880603E-1,4.224673E-2,0E0,1.8334258E-1,0E0,7.9112816E-1,0E0,1.2819242E-1,1.4987557E-1,1.508503E-1,2.011824E-1,0E0,0E0,1.3122296E-1,9.9274665E-2,1.3966641E-1,1.1300105E-1,0E0,7.346066E-2,3.6390662E-1,2.4202682E-1,4.3384224E-1,1.4594364E-1,0E0,0E0,9.369987E-2,1.3899104E-1,4.3863106E-1,5.0651073E-2,8.494806E-2,0E0,3.939328E-2,0E0,5.8071494E-2,0E0,3.29921E-2,3.651063E-2,0E0,4.0088654E-2,0E0,1.6570061E-2,1.2578955E-1,1.3746837E-1,2.0025715E-1,2.1974608E-2,0E0,0E0,1.3911098E-1,2.1374452E-1,1.9210884E-1,1.09407604E-1,2.1598136E-1,4.6678448E-1,0E0,0E0,0E0,0E0,1.329265E-1,1.3844222E-1,3.045249E-1,5.56479E-1,0E0,0E0,1.1224699E-1,0E0,0E0,0E0,9.0109795E-2,0E0,1.5328394E-2,2.8894953E-2,0E0,0E0,0E0,5.7616442E-2,0E0,0E0,0E0,1.4196138E-1,7.7461936E-2,9.375256E-2,0E0,7.165874E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5774725E-1,0E0,3.346306E-2,2.3483142E-2,8.057438E-2,6.2831044E-2,1.719392E-1,7.842184E-1,4.209459E-2,3.12761E-2,0E0,0E0,2.9516435E-1,1.8309212E-1,2.2550237E-1,5.3544402E-2,0E0,1.07503414E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.515312E-2,4.4456627E-2,2.8501181E-2,0E0,0E0,0E0,2.898022E-2,0E0,0E0,1.1212287E-1,0E0,0E0,0E0,0E0,3.7106156E-2,0E0,0E0,0E0,8.1428446E-2,3.4864247E-2,1.393122E-1,0E0,3.4284696E-2,0E0,0E0,0E0,0E0,2.2100937E-1,3.512764E-2,7.039118E-2,2.2263767E-1,2.157852E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,30,30,34,34,35,35,36,36,37,37,39,39,40,40,41,41,43,43,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,68,68,69,69,71,71,73,73,75,75,76,76,78,78,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,97,97,98,98,99,99,100,100,103,103,107,107,109,109,110,110,114,114,118,118,119,119,120,120,122,122,129,129,131,131,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144,146,146,155,155,156,156,157,157,161,161,164,164,169,169,173,173,174,174,175,175,177,177,182,182,183,183,184,184,185,185,186,186],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,-1,-1,-1,52,54,56,58,-1,60,62,64,-1,66,-1,68,-1,70,72,74,76,-1,-1,78,80,82,84,-1,86,88,90,92,94,-1,-1,96,98,100,102,104,-1,106,-1,108,-1,110,112,-1,114,-1,116,118,120,122,124,-1,-1,126,128,130,132,134,136,-1,-1,-1,-1,138,140,142,144,-1,-1,146,-1,-1,-1,148,-1,150,152,-1,-1,-1,154,-1,-1,-1,156,158,160,-1,162,-1,-1,-1,-1,-1,-1,164,-1,166,168,170,172,174,176,178,180,-1,-1,182,184,186,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,192,194,196,-1,-1,-1,198,-1,-1,200,-1,-1,-1,-1,202,-1,-1,-1,204,206,208,-1,210,-1,-1,-1,-1,212,214,216,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.972052E5,1E0,2.6E1,7.785302E5,8.76E2,4.70127E5,7.24E2,3.355864E0,2.1E1,6.9307615E6,-1.721789E-2,-7.416122E-3,1.94261E5,8.399457E2,3.9E1,2.7E1,3.292496E-2,7.6255396E2,5.51E2,6.0692043E0,1.158E3,1E0,1.8752169E-2,3.5375725E-2,3.5510652E6,7.134432E7,9.566785E5,-1.9543128E-2,1.2934754E-2,1.4912975E-1,-1.0904279E-2,-3.5152256E-2,-1.9409508E-3,7.919006E7,1.561132E2,1.9E1,3.9E2,-2.7016891E-2,1.5961905E2,4.7220547E1,9.51E3,1.1626313E-2,3.414E3,9.303139E-3,6.663214E6,-9.377705E-3,1.317484E1,2.5493E4,1.7302156E5,3.38E2,-1.5910942E-2,-6.5945326E-3,1.001791E6,9.144571E5,5.5356906E4,1.862E3,-9.197322E-3,2.5E1,4.5418963E-4,6.346204E-7,2.308943E0,1.5714285E0,-5.2269764E-4,5.4877093E-3,3.3193566E5,2.115238E5,2.9308079E1,3.97371E5,4.233978E0,-1.3124169E-3,1.9903773E2,-8.669574E-3,6.0893228E7,-2.5025285E-3,6.0875E4,5.25E0,-3.908957E-3,5.7E1,-4.4649086E-4,2.087E2,2.6E1,9.61E2,3.011152E6,3.62349E5,-5.695077E-3,2.0137683E-3,7.89E2,7.3809524E0,3.2E1,5.5356906E4,1.28738E5,1.2629019E8,-1.2606748E-2,-1.1788013E-3,1.3482728E-3,1.0865502E-2,6.3055553E0,1E1,1.1772152E0,3.8537518E2,1.0501004E-2,2.1841403E-2,8.0833334E-1,-1.7341226E-2,1.9915015E-3,-2.7630979E-3,1.9166666E0,-1.408249E-2,2.29E2,8.910034E0,-9.9995565E-3,-3.571021E-3,-1.3162834E-2,1.1430505E3,-3.3883373E-3,-7.0597418E-3,5.0022574E-3,1.06314E5,1.4381613E7,1.8729467E8,-8.802719E-3,3.1316226E0,1.7780726E-3,-2.1295685E-3,1.3409492E-3,-7.749313E-3,-1.9454127E-2,-3.868254E-3,1.5881818E2,8.497947E-3,3.1E1,3.3272727E0,9.94E2,6E0,5.372237E2,2.364E3,4.924E3,2.5503825E6,1.1766521E-3,-9.026096E-3,3.714144E8,6.355E3,3.2608695E0,4.907764E3,-1.3897629E-2,1.0891155E4,-1.6402863E-3,-1.0218113E-2,7.861163E-4,-2.4435818E-3,4.6941247E-3,2.8159659E-4,-1.0388775E-2,-2.8447777E-3,1.3205625E7,5.342944E6,3.011152E6,5.030587E-3,7.5726593E-3,5.6892954E-4,2.99E2,-6.1954847E-3,-7.278001E-3,1.9650501E6,-4.1713733E-3,-1.1689367E-2,8.8830624E-4,-3.185609E-3,1.6446976E3,8.4653875E-4,1.6219297E-2,7.581044E-3,1.1E1,2.7012987E1,1.1962106E-1,-2.000307E-2,1.1E1,5.767181E-3,-3.920738E-4,-5.4114377E-3,-3.4258162E-5,2E0,1.6007428E7,1.853776E7,1.4342732E2,8.710612E1,8.8514695E-3,1.884963E-2,-1.124831E-2,-7.3551587E-3,-4.0056063E-3,1.2728353E-3,6.5676756E-3,2.6573543E-5,-3.5093222E-3,3.5097462E-4,3.1461755E-3,-1.1642853E-3,6.547251E-5,6.554649E-3,2.2026922E-3,8.144722E-3,-1.2287642E-3,3.7268945E-3,8.080138E-3,2.866218E-3,2.8895813E-3,-5.443667E-3,-1.6372912E-3,2.9130038E-3,1.332312E-2,2.5725174E-3,1.2843513E-2,7.203258E-3,2.037287E-2,1.0006106E-2,-6.540608E-3,5.5992655E-3,1.036429E-2,3.2748797E-3],"split_indices":[2,43,17,3,43,0,12,2,73,3,43,0,0,1,67,8,0,73,48,2,68,2,29,0,0,43,59,60,0,0,53,0,0,0,7,4,8,48,0,70,73,44,0,2,0,43,0,69,1,43,0,0,0,5,60,48,2,0,3,53,52,69,69,0,0,43,48,71,1,68,0,4,0,5,0,1,69,0,0,0,4,0,2,9,1,0,0,2,73,3,48,1,5,0,0,0,0,69,3,68,71,0,0,68,0,0,0,68,0,10,71,0,0,0,48,0,0,0,1,5,7,0,68,0,0,0,0,0,0,48,0,0,69,10,8,4,2,2,47,0,0,7,2,69,4,0,43,0,0,0,0,0,0,0,0,9,60,9,0,0,0,0,0,0,66,0,0,0,0,48,0,0,0,3,73,53,0,8,0,0,0,0,6,66,62,71,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,7.63E2,2.59E2,2.4E2,5.23E2,9E0,2.5E2,2.07E2,3.3E1,2.21E2,3.02E2,5E0,4E0,1.6E1,2.34E2,1.37E2,7E1,1.2E1,2.1E1,1.96E2,2.5E1,2.83E2,1.9E1,9E0,7E0,6.5E1,1.69E2,1.29E2,8E0,6E0,6.4E1,7E0,5E0,4E0,1.7E1,6.2E1,1.34E2,1.9E1,6E0,1.67E2,1.16E2,9E0,1E1,5.5E1,1E1,1.63E2,6E0,1.09E2,2E1,3.1E1,3.3E1,1.1E1,6E0,3.1E1,3.1E1,9.1E1,4.3E1,1E1,9E0,2.4E1,1.43E2,1.07E2,9E0,4E0,5E0,9E0,4.6E1,1.43E2,2E1,1.05E2,4E0,1.3E1,7E0,2.5E1,6E0,2.3E1,1E1,1.1E1,2E1,1E1,2.1E1,5.3E1,3.8E1,3.2E1,1.1E1,5E0,4E0,1.6E1,8E0,1.23E2,2E1,3.5E1,7.2E1,4E0,5E0,4E0,5E0,3.6E1,1E1,8.1E1,6.2E1,4E0,1.6E1,1.01E2,4E0,6E0,7E0,1.4E1,1.1E1,1.1E1,1.2E1,4E0,6E0,1E1,1E1,7E0,1.4E1,6E0,4.7E1,1.9E1,1.9E1,1.5E1,1.7E1,7E0,4E0,1.2E1,4E0,4E0,4E0,1.18E2,5E0,8E0,1.2E1,2.2E1,1.3E1,4.9E1,2.3E1,2.6E1,1E1,5E0,5E0,3E1,5.1E1,5E1,1.2E1,1.4E1,8.7E1,4E0,1E1,5E0,6E0,5E0,7E0,5E0,5E0,3.9E1,8E0,1.5E1,4E0,1.3E1,6E0,1.3E1,4E0,5E0,1.13E2,4E0,4E0,4E0,8E0,1.4E1,8E0,6E0,7E0,2.7E1,2.2E1,1.9E1,4E0,1.7E1,9E0,6E0,4E0,7E0,2.3E1,3E1,2.1E1,1.1E1,3.9E1,5E0,7E0,3E1,5.7E1,3E1,9E0,4E0,4E0,7E0,8E0,4E0,9E0,1.08E2,5E0,4E0,1E1,1.2E1,1.5E1,1.7E1,5E0,1.4E1,5E0,4E0,1.3E1,1.7E1,6E0,2.2E1,8E0,1.6E1,5E0,6E0,5E0,1.6E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-2.8888178E-3,-1.0378601E-1,1.03640884E-1,-1.7752095E-1,-4.3336123E-2,-2.8024974E-1,1.2771633E-1,-1.5946168E-1,-4.2828533E-1,-2.2663085E-2,-2.1770269E-1,-3.737383E-1,-1.400324E-1,7.256638E-2,2.6645783E-1,-2.0144477E-1,-7.7175714E-2,-1.762753E-1,-3.7833456E-2,-1.1101938E-2,-1.3162233E-1,-2.7249513E-2,-1.091418E-1,-1.882733E-2,-9.442346E-3,-2.26086E-3,-9.343133E-3,4.241862E-2,1.814922E-1,3.3945683E-2,2.3605664E-1,-2.6189423E-1,-1.6384618E-1,-5.902484E-2,-1.34531185E-2,-1.0062221E-2,-3.8291423E-3,8.8417195E-3,-1.6937181E-2,-1.0936117E-2,-7.1762465E-2,2.5273997E-3,-1.6103171E-1,5.24376E-2,-2.3033181E-1,1.3372271E-1,2.632323E-1,3.0090496E-1,1.1237551E-1,-1.8231781E-1,-2.9394698E-1,-1.1706685E-1,-2.0916119E-1,3.4688085E-2,-7.7425525E-2,-2.3373311E-2,1.3407475E-1,1.3794715E-4,-9.3332425E-2,-9.410978E-3,-1.2709263E-3,6.12179E-2,-2.0704196E-1,-1.3680625E-2,-4.8654717E-3,8.1075914E-2,2.4052992E-1,7.573041E-3,1.6519506E-2,3.369364E-1,9.153348E-2,4.2898804E-2,2.7307785E-1,-1.0592125E-2,-5.4114023E-3,-1.7695993E-1,-3.335862E-1,-1.3754368E-1,-4.8206348E-2,-2.647402E-3,-2.2166401E-1,6.4088334E-3,-2.2024706E-3,-1.1233871E-1,-1.6010825E-2,-1.1673879E-1,-1.2039128E-2,1.0470365E-3,9.451013E-3,-9.660795E-4,-5.5535776E-3,-7.306476E-3,1.055014E-1,-1.7359061E-2,1.4180505E-4,1.15015484E-1,-1.5274567E-2,4.1195396E-3,1.3099863E-2,1.6887656E-3,3.5382012E-1,-1.1621586E-3,7.3528434E-3,8.465699E-2,-9.989625E-3,6.3697295E-3,1.6504375E-2,-1.1958338E-2,-4.78911E-3,-9.370715E-3,-1.6632874E-2,-3.141064E-4,-1.5272544E-1,2.7652434E-4,-3.8264103E-3,-2.3514044E-1,-3.2117467E-3,-1.4402582E-1,-4.9925137E-2,-3.7213653E-2,3.326454E-3,3.2567137E-3,-1.795748E-1,3.361361E-2,-4.0257577E-2,1.3968155E-2,-1.63348E-2,1.5979484E-1,6.683572E-2,1.5146789E-1,1.3550721E-3,2.5317718E-3,-3.285927E-3,3.7045714E-1,8.346115E-3,-5.6934517E-2,1.3873275E-1,-1.704616E-1,-2.770229E-3,-1.5368074E-1,-2.6456314E-1,-8.46026E-3,-9.743069E-2,1.4720956E-4,-4.4718855E-3,-3.224681E-3,4.7905123E-4,-2.4685125E-1,2.6873744E-3,5.8459133E-2,-4.3845914E-2,-6.0763683E-2,1.2372842E-2,3.586358E-2,-1.3195297E-1,1.19522125E-1,2.7565986E-1,3.6398917E-2,1.5409309E-1,1.7750493E-1,2.2300573E-3,4.180528E-1,2.6491722E-1,2.59703E-3,-7.331607E-3,9.685445E-3,7.351236E-2,-5.6178286E-3,-1.0853061E-2,-8.500792E-3,-3.1583244E-3,-6.0907956E-3,-1.2990744E-2,-5.485076E-4,-6.0274336E-3,-4.691669E-3,-1.535808E-2,7.7819085E-4,4.014255E-3,8.9214253E-4,-3.339842E-3,-4.8270114E-3,-8.9502777E-4,-3.4885944E-3,2.916251E-3,8.705812E-3,1.0193696E-3,-9.336256E-3,9.5539633E-4,7.5618178E-3,1.1169775E-3,8.491481E-3,1.591753E-2,9.2139846E-4,7.552971E-3,9.486959E-3,3.960368E-3,9.777831E-3,3.405903E-3,2.0701755E-2,1.3645905E-2,3.7264237E-3,1.5198209E-2,5.622386E-4,5.097148E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,-1,-1,-1,-1,43,45,-1,47,49,51,53,-1,-1,-1,-1,55,-1,57,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,89,-1,-1,91,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,-1,107,109,111,113,-1,115,-1,-1,117,119,121,123,-1,-1,-1,-1,125,127,-1,-1,129,131,-1,-1,-1,133,-1,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,137,-1,-1,139,-1,141,143,145,-1,-1,147,149,151,153,-1,155,157,159,-1,-1,-1,161,-1,163,165,167,-1,169,171,-1,173,-1,-1,-1,-1,175,-1,177,179,181,183,185,187,189,191,193,195,197,-1,199,201,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1490206E1,2.4393477E0,4.8418083E0,1.0502748E0,1.0872861E0,3.5403466E-1,3.7299428E0,7.8592443E-1,1.430757E0,3.4117547E-1,1.2283708E0,1.9706964E-2,6.943387E-2,1.1482234E0,1.8055296E0,3.0483055E-1,2.9358885E-1,2.4608076E-2,0E0,2.9508433E-1,1.4657792E-1,0E0,2.2986373E-1,0E0,0E0,0E0,0E0,7.6903045E-1,2.6155567E-1,0E0,1.0248938E0,9.1795206E-2,1.8113518E-1,1.3201442E-1,0E0,0E0,0E0,0E0,2.3736948E-1,0E0,2.9300451E-2,0E0,1.0835254E-1,6.232615E-1,5.1064074E-2,2.616517E-1,1.943475E-1,6.179395E-1,5.130435E-1,3.129816E-2,1.2182188E-1,6.38718E-2,7.843256E-2,1.1800276E-1,1.332849E-1,2.4410196E-1,7.3905006E-2,0E0,2.473712E-2,0E0,0E0,7.897406E-1,3.2017112E-1,0E0,0E0,1.1621013E-1,7.890695E-2,0E0,0E0,3.5729122E-1,1.1447441E-1,3.8621104E-1,1.1005843E-1,0E0,0E0,4.5190156E-2,2.3774147E-2,7.438916E-2,2.4883984E-2,0E0,7.386875E-2,0E0,0E0,7.191771E-2,4.746671E-2,3.0678138E-1,2.6930907E-1,0E0,0E0,0E0,0E0,7.637111E-1,3.2100832E-1,0E0,0E0,7.676566E-2,4.3035794E-2,0E0,0E0,0E0,1.3185978E-1,0E0,0E0,2.375987E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.571712E-2,0E0,0E0,5.5212498E-2,0E0,2.9527068E-2,3.7649244E-2,3.173459E-2,0E0,0E0,3.1825638E-1,1.5652499E-1,1.3997442E-1,3.1784278E-1,0E0,2.7035546E-1,2.4403962E-1,3.979683E-2,0E0,0E0,0E0,2.0069122E-1,0E0,1.12548366E-1,8.633819E-2,5.961919E-2,0E0,2.1052778E-2,2.4721503E-2,0E0,3.8021833E-2,0E0,0E0,0E0,0E0,1.5276265E-1,0E0,7.070933E-2,3.732783E-2,1.6626453E-1,1.6979533E-1,1.7690632E-1,1.4334238E-1,2.0654905E-1,3.791082E-2,1.4521608E-1,6.650138E-2,3.4002006E-2,0E0,1.8774033E-2,2.3312235E-1,0E0,0E0,0E0,2.8017692E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,27,27,28,28,30,30,31,31,32,32,33,33,38,38,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,80,80,83,83,84,84,85,85,86,86,91,91,92,92,95,95,96,96,100,100,103,103,112,112,115,115,117,117,118,118,119,119,122,122,123,123,124,124,125,125,127,127,128,128,129,129,133,133,135,135,136,136,137,137,139,139,140,140,142,142,147,147,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,161,161,162,162,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,-1,-1,-1,-1,44,46,-1,48,50,52,54,-1,-1,-1,-1,56,-1,58,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,90,-1,-1,92,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,-1,108,110,112,114,-1,116,-1,-1,118,120,122,124,-1,-1,-1,-1,126,128,-1,-1,130,132,-1,-1,-1,134,-1,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,138,-1,-1,140,-1,142,144,146,-1,-1,148,150,152,154,-1,156,158,160,-1,-1,-1,162,-1,164,166,168,-1,170,172,-1,174,-1,-1,-1,-1,176,-1,178,180,182,184,186,188,190,192,194,196,198,-1,200,202,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,4.97E2,1E0,4.4895835E0,3.1316226E0,1.7540792E5,1.1459359E3,3.51E2,1E0,1E0,1E0,1.3141646E3,1.91E2,2.9652428E6,3.3817584E7,3.89E2,2.1E1,8.5714287E-1,-3.7833456E-2,1.2E1,8.314423E6,-2.7249513E-2,2.857143E0,-1.882733E-2,-9.442346E-3,-2.26086E-3,-9.343133E-3,1.2294118E1,2.593592E6,3.3945683E-2,4.1681065E1,5E1,3.1E1,8.638015E4,-1.34531185E-2,-1.0062221E-2,-3.8291423E-3,8.8417195E-3,7.4711205E6,-1.0936117E-2,3.202146E5,2.5273997E-3,2.0134516E6,1.5449402E7,1.594941E3,1E0,1.81E2,2.2040408E7,4.435876E2,3.5070792E2,2.416E3,2E0,3.3305264E2,2.90099E0,8.599521E5,1.6666667E-1,3.3852024E7,1.3794715E-4,2.5242718E-2,-9.410978E-3,-1.2709263E-3,3.850926E2,4.44E2,-1.3680625E-2,-4.8654717E-3,3.744E3,7.156528E7,7.573041E-3,1.6519506E-2,5.80418E7,4.771E3,3.206931E2,1.11436836E5,-1.0592125E-2,-5.4114023E-3,1.2571428E1,8.683527E3,1E1,3.927456E7,-2.647402E-3,1.901875E2,6.4088334E-3,-2.2024706E-3,1.339646E6,9.458613E3,3.151751E-1,1.5357143E0,1.0470365E-3,9.451013E-3,-9.660795E-4,-5.5535776E-3,2.954124E6,3.6828358E0,-1.7359061E-2,1.4180505E-4,2.4E0,7.617426E4,4.1195396E-3,1.3099863E-2,1.6887656E-3,1E0,-1.1621586E-3,7.3528434E-3,9.408E3,-9.989625E-3,6.3697295E-3,1.6504375E-2,-1.1958338E-2,-4.78911E-3,-9.370715E-3,-1.6632874E-2,-3.141064E-4,1.0666667E1,2.7652434E-4,-3.8264103E-3,1.625E0,-3.2117467E-3,1.9225262E2,1.2E1,9.356961E6,3.326454E-3,3.2567137E-3,8.1E2,1.5532663E1,1.6802971E6,6.16E2,-1.63348E-2,3.51E3,7.445455E2,1.5723623E0,1.3550721E-3,2.5317718E-3,-3.285927E-3,9.44363E5,8.346115E-3,3.0238653E8,8.710612E1,3.407316E7,-2.770229E-3,4.162742E5,9.185E3,-8.46026E-3,1.4E1,1.4720956E-4,-4.4718855E-3,-3.224681E-3,4.7905123E-4,2E0,2.6873744E-3,1.8791208E0,1E0,3.4289816E2,2.7857144E0,3.4662805E-4,4.6404468E2,5.556659E6,1E0,1.8953742E6,5.388794E6,5.1291872E8,2.2300573E-3,1.7103828E7,5.831829E6,2.59703E-3,-7.331607E-3,9.685445E-3,2.2365898E3,-5.6178286E-3,-1.0853061E-2,-8.500792E-3,-3.1583244E-3,-6.0907956E-3,-1.2990744E-2,-5.485076E-4,-6.0274336E-3,-4.691669E-3,-1.535808E-2,7.7819085E-4,4.014255E-3,8.9214253E-4,-3.339842E-3,-4.8270114E-3,-8.9502777E-4,-3.4885944E-3,2.916251E-3,8.705812E-3,1.0193696E-3,-9.336256E-3,9.5539633E-4,7.5618178E-3,1.1169775E-3,8.491481E-3,1.591753E-2,9.2139846E-4,7.552971E-3,9.486959E-3,3.960368E-3,9.777831E-3,3.405903E-3,2.0701755E-2,1.3645905E-2,3.7264237E-3,1.5198209E-2,5.622386E-4,5.097148E-3],"split_indices":[2,2,17,68,68,48,67,2,6,6,29,67,0,43,7,1,8,71,0,10,5,0,73,0,0,0,0,68,9,0,71,44,10,43,0,0,0,0,43,0,43,0,60,62,48,23,0,66,71,48,9,8,48,69,43,73,7,0,57,0,0,67,0,0,0,10,7,0,0,7,2,73,48,0,0,70,43,3,7,0,4,0,0,9,48,68,68,0,0,0,0,62,69,0,0,69,48,0,0,0,6,0,0,2,0,0,0,0,0,0,0,0,73,0,0,68,0,4,3,60,0,0,2,73,43,0,0,10,67,68,0,0,0,9,0,7,73,7,0,62,12,0,3,0,0,0,0,8,0,69,26,4,69,53,4,62,29,43,62,7,0,62,60,0,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.067E3,5.48E2,5.19E2,2.46E2,3.02E2,3E1,4.89E2,2.31E2,1.5E1,2.71E2,3.1E1,1.7E1,1.3E1,3.51E2,1.38E2,1.52E2,7.9E1,1E1,5E0,2.46E2,2.5E1,6E0,2.5E1,1.3E1,4E0,6E0,7E0,2.76E2,7.5E1,7E0,1.31E2,5.6E1,9.6E1,7.4E1,5E0,6E0,4E0,6E0,2.4E2,8E0,1.7E1,6E0,1.9E1,2.67E2,9E0,4.9E1,2.6E1,8.5E1,4.6E1,1.8E1,3.8E1,4.9E1,4.7E1,1.2E1,6.2E1,2.31E2,9E0,4E0,1.3E1,1.4E1,5E0,2.59E2,8E0,5E0,4E0,3.4E1,1.5E1,1.4E1,1.2E1,7.2E1,1.3E1,3.3E1,1.3E1,9E0,9E0,1.1E1,2.7E1,3.7E1,1.2E1,4E0,4.3E1,5E0,7E0,3.9E1,2.3E1,2.4E1,2.07E2,4E0,5E0,4E0,9E0,1.02E2,1.57E2,4E0,4E0,2.5E1,9E0,4E0,1.1E1,4E0,6.8E1,5E0,8E0,2.9E1,4E0,6E0,7E0,4E0,7E0,6E0,2.1E1,4E0,3.3E1,5E0,7E0,3.9E1,4E0,2.5E1,1.4E1,1.9E1,4E0,6E0,1.8E1,7.9E1,1.28E2,9.7E1,5E0,6.4E1,9.3E1,1.7E1,8E0,4E0,5E0,6.1E1,7E0,8E0,2.1E1,2.7E1,6E0,1.2E1,2.7E1,1.2E1,1.3E1,7E0,7E0,1.1E1,8E0,1.4E1,4E0,6E1,1.9E1,9.2E1,3.6E1,8.5E1,1.2E1,4.9E1,1.5E1,7E1,2.3E1,1.3E1,4E0,4E1,2.1E1,4E0,4E0,9E0,1.2E1,1.7E1,1E1,8E0,4E0,4E0,2.3E1,4E0,9E0,6E0,8E0,2.5E1,3.5E1,6E0,1.3E1,4.4E1,4.8E1,1.3E1,2.3E1,6E0,7.9E1,8E0,4E0,3.3E1,1.6E1,8E0,7E0,6.3E1,7E0,1.2E1,1.1E1,9E0,4E0,3E1,1E1,6E0,1.5E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"205","size_leaf_vector":"1"}},{"base_weights":[-2.3427939E-3,-5.623861E-2,1.5676703E-1,-1.3198337E-1,1.8639259E-2,-2.9502624E-1,1.8686032E-1,-1.1840211E-1,-3.6169827E-1,3.3162467E-2,-1.0273241E-1,-1.826237E-2,-5.0650304E-3,5.7936394E-1,1.6126613E-1,-2.2093423E-1,-8.746913E-2,-4.4488412E-1,-1.0701462E-3,9.307009E-3,9.7468786E-2,-2.6139313E-1,-4.1132398E-2,1.748581E-2,3.1540528E-2,7.998693E-2,2.140466E-1,-1.7831215E-1,-2.825526E-1,-1.4985313E-1,-3.467525E-2,-5.6049544E-1,-9.986859E-3,1.6116574E-2,-1.0327695E-2,1.4700647E-1,3.0877972E-3,-6.101707E-3,-1.8025063E-2,-6.814604E-3,-6.22091E-3,5.999928E-2,1.6007181E-2,1.5136747E-1,2.8882807E-1,-1.9712232E-1,-3.472857E-4,-3.3322403E-1,-1.8675955E-1,-1.2366861E-1,-2.0969711E-1,1.0904815E-2,-4.9897004E-2,-1.50895035E-2,-3.368131E-2,-1.0956182E-2,7.0743464E-2,1.2300843E-1,1.2319498E-2,4.746086E-2,-6.541598E-2,1.8706367E-3,-3.4743544E-2,7.546723E-2,-7.0990347E-3,2.2647834E-1,9.6260704E-2,3.17631E-1,-4.2286147E-3,-1.1765553E-3,-2.1324803E-1,-9.120772E-3,-3.676226E-1,-1.9430023E-3,-1.1383385E-2,-1.3343324E-2,-1.08061954E-1,-2.5948998E-1,-1.0164127E-1,-2.1878144E-2,-9.547743E-2,-6.876728E-2,1.8321907E-2,-7.173154E-2,9.2431605E-2,9.3851976E-2,2.2049987E-1,-1.6503694E-3,9.50001E-2,1.0518869E-3,-6.6644624E-3,-3.2587184E-3,2.4912076E-4,4.0731695E-2,1.7590047E-1,2.085931E-3,2.5166938E-1,1.1742903E-1,-7.5403583E-4,2.8857383E-1,2.0203322E-2,-4.8413756E-3,-1.0262682E-2,-2.3152672E-2,-1.2201006E-2,3.9538456E-4,-1.1738142E-1,-1.7621845E-1,-3.584483E-1,-1.6830821E-3,-6.368249E-3,-3.3268414E-2,4.4435696E-3,-1.056216E-2,-6.304716E-2,-4.1113973E-2,-1.3080865E-2,4.2379634E-3,1.17666805E-2,6.068921E-4,-8.140293E-3,1.7759284E-1,4.599027E-2,8.028012E-3,6.2466465E-2,6.238012E-3,1.3677929E-2,5.5254367E-3,1.4653923E-3,5.2801795E-2,-4.7400123E-3,2.0351318E-1,2.4189707E-3,1.3605975E-1,1.3544919E-2,1.4673632E-1,4.1701887E-2,3.1746763E-1,4.824167E-3,-1.3392566E-1,-5.2085243E-2,-9.941129E-3,-2.5528905E-3,-2.013959E-2,-8.952433E-3,-6.040955E-2,2.0263355E-2,-6.8422225E-3,-1.0169511E-1,-1.3024366E-1,3.0384844E-2,1.9406306E-2,-1.3725142E-1,5.4798927E-3,1.0472691E-2,-9.15994E-3,1.6057804E-1,8.553412E-2,-2.21314E-3,-2.292266E-3,6.6084415E-2,3.7688268E-3,1.0965808E-2,2.277029E-3,9.037467E-3,1.8887774E-3,1.8174064E-1,-4.352378E-3,5.6101177E-3,1.2731254E-2,2.1059021E-2,-4.447894E-3,-8.290691E-3,-9.5859176E-4,-4.9006348E-3,-3.3029483E-3,1.3655274E-4,-6.0596596E-4,6.691559E-3,1.4183397E-3,-5.456215E-3,-1.0743662E-3,-5.308265E-3,-2.358725E-3,-1.7983975E-2,3.2831766E-3,-2.9287226E-3,-1.443607E-3,2.3336082E-3,-1.4271461E-3,-1.09691955E-2,-2.4505972E-3,4.282196E-3,3.6186844E-3,8.956619E-3,6.3879024E-3,1.4070527E-3,3.7921395E-3,-2.8258574E-4,4.579095E-3,1.1755486E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,39,-1,-1,41,43,45,47,49,51,53,-1,55,-1,57,59,-1,-1,61,-1,63,-1,65,67,69,-1,71,73,75,77,-1,79,-1,-1,81,83,85,-1,87,89,-1,91,93,-1,95,97,99,-1,-1,101,-1,103,-1,-1,-1,105,107,109,111,113,115,117,119,121,123,125,-1,127,-1,-1,-1,-1,129,131,-1,133,135,-1,137,-1,-1,-1,-1,-1,-1,139,141,143,-1,-1,145,-1,-1,147,149,-1,151,-1,-1,-1,153,155,-1,157,-1,-1,-1,-1,159,-1,161,-1,163,-1,165,167,169,-1,171,173,-1,-1,-1,-1,175,177,179,181,183,185,187,189,-1,-1,191,193,195,-1,-1,197,-1,-1,-1,-1,-1,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.472691E0,4.1881056E0,3.4541159E0,1.102148E0,6.5913343E-1,2.5409234E-1,2.267335E0,1.0844002E0,5.5292034E-1,5.099337E-1,3.801884E-1,0E0,0E0,3.2236576E-2,9.3524885E-1,1.5972185E-1,8.807533E-1,3.0190825E-1,0E0,3.904989E-1,4.203334E-1,1.3164288E-1,9.536655E-2,0E0,0E0,4.5328116E-1,5.843706E-1,1.5915251E-1,8.8240385E-2,1.6680622E-1,6.175291E-1,1.5608215E-1,0E0,3.527534E-1,0E0,1.3661587E-1,1.0037583E-1,0E0,0E0,3.179416E-2,0E0,2.892081E-1,0E0,2.8863275E-1,6.8639946E-1,1.191318E-1,0E0,1.617837E-2,1.018261E-1,1.9560921E-1,1.673199E-1,0E0,1.7670676E-1,0E0,0E0,2.720757E-1,2.5074163E-1,1.242733E-1,0E0,8.248366E-2,9.366595E-2,0E0,2.192907E-2,2.6843485E-1,0E0,1.3023949E-1,1.091595E-1,6.8546295E-2,0E0,0E0,1.626277E-2,0E0,6.1178565E-2,0E0,0E0,0E0,9.012538E-2,1.3460457E-1,2.5668189E-2,1.2153788E-1,2.1443698E-1,3.0847922E-1,3.5513106E-1,9.9841386E-2,2.647276E-1,9.19843E-2,3.1072438E-2,0E0,1.6444102E-2,0E0,0E0,0E0,0E0,1.0921896E-1,5.756992E-2,0E0,8.970201E-2,7.8216374E-2,0E0,2.2225118E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.512534E-2,5.880198E-2,4.311502E-2,0E0,0E0,1.1891006E-1,0E0,0E0,9.488122E-2,3.1621808E-1,0E0,2.2157183E-1,0E0,0E0,0E0,4.2602062E-2,2.9216972E-1,0E0,8.302341E-2,0E0,0E0,0E0,0E0,7.946089E-2,0E0,4.6670675E-2,0E0,3.6872208E-2,0E0,9.26131E-2,1.4147733E-1,1.1204529E-1,0E0,8.557236E-2,2.6560873E-2,0E0,0E0,0E0,0E0,3.8138762E-2,1.1833511E-1,8.298921E-2,2.4258286E-2,4.31387E-1,1.11325234E-1,1.4811152E-1,9.921907E-2,0E0,0E0,1.4794476E-1,2.5888652E-2,7.130657E-2,0E0,0E0,5.825652E-2,0E0,0E0,0E0,0E0,0E0,9.493512E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,36,36,39,39,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,55,55,56,56,57,57,59,59,60,60,62,62,63,63,65,65,66,66,67,67,70,70,72,72,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,93,93,94,94,96,96,97,97,99,99,106,106,107,107,108,108,111,111,114,114,115,115,117,117,121,121,122,122,124,124,129,129,131,131,133,133,135,135,136,136,137,137,139,139,140,140,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,155,155,156,156,157,157,160,160,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,40,-1,-1,42,44,46,48,50,52,54,-1,56,-1,58,60,-1,-1,62,-1,64,-1,66,68,70,-1,72,74,76,78,-1,80,-1,-1,82,84,86,-1,88,90,-1,92,94,-1,96,98,100,-1,-1,102,-1,104,-1,-1,-1,106,108,110,112,114,116,118,120,122,124,126,-1,128,-1,-1,-1,-1,130,132,-1,134,136,-1,138,-1,-1,-1,-1,-1,-1,140,142,144,-1,-1,146,-1,-1,148,150,-1,152,-1,-1,-1,154,156,-1,158,-1,-1,-1,-1,160,-1,162,-1,164,-1,166,168,170,-1,172,174,-1,-1,-1,-1,176,178,180,182,184,186,188,190,-1,-1,192,194,196,-1,-1,198,-1,-1,-1,-1,-1,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,4.9963706E5,1E0,4E1,5.1071826E-8,9.3326636E-2,2.2968E4,7.7E1,6.0078123E-6,3.9742767E2,2.3238889E2,-1.826237E-2,-5.0650304E-3,1.63757E4,1.0665628E6,1E0,5.41E2,8E0,-1.0701462E-3,4.5620965E6,2.3818183E0,5.90352E5,2.01E2,1.748581E-2,3.1540528E-2,7.6255713E3,7.098E3,1.6576804E7,1.5389474E1,1.3306123E1,1.1E1,1E1,-9.986859E-3,6.946813E7,-1.0327695E-2,4.284608E6,1.2218E4,-6.101707E-3,-1.8025063E-2,2.1470589E1,-6.22091E-3,3.0070068E7,1.6007181E-2,1.1057851E1,4.1325716E7,5E0,-3.472857E-4,9E0,7.8918E4,3.9E1,1.9470909E2,1.0904815E-2,1.9E1,-1.50895035E-2,-3.368131E-2,2.1578947E2,5.4E1,2.483E3,1.2319498E-2,2.476353E-2,4.9E1,1.8706367E-3,5.1042255E2,1.5142261E3,-7.0990347E-3,1.2157188E6,1.0975045E7,2.1511E4,-4.2286147E-3,-1.1765553E-3,1.798E3,-9.120772E-3,2.1609572E7,-1.9430023E-3,-1.1383385E-2,-1.3343324E-2,8.8E1,3.50986E5,1.84E2,4.233978E0,1.0201538E-1,1.8E1,3.2E1,1.3287E4,9.538462E0,1.6939986E9,6.257143E2,-1.6503694E-3,2.0316172E6,1.0518869E-3,-6.6644624E-3,-3.2587184E-3,2.4912076E-4,1.1290322E1,3.7798166E0,2.085931E-3,1E0,2.9157894E0,-7.5403583E-4,2E0,2.0203322E-2,-4.8413756E-3,-1.0262682E-2,-2.3152672E-2,-1.2201006E-2,3.9538456E-4,3.201807E6,2.511352E6,1.6595395E5,-1.6830821E-3,-6.368249E-3,6.077143E5,4.4435696E-3,-1.056216E-2,3.401282E1,2.9652428E6,-1.3080865E-2,6.485856E7,1.17666805E-2,6.068921E-4,-8.140293E-3,1.14E3,1.2768175E6,8.028012E-3,1.9882522E0,6.238012E-3,1.3677929E-2,5.5254367E-3,1.4653923E-3,4.6712E4,-4.7400123E-3,3.8153846E0,2.4189707E-3,2.1512408E5,1.3544919E-2,3.6522612E6,4.488263E0,1.9717247E9,4.824167E-3,1E0,2.707673E6,-9.941129E-3,-2.5528905E-3,-2.013959E-2,-8.952433E-3,1.3205625E7,4.6134964E2,5.49E2,1.8571428E0,1.925508E6,2.6210527E0,1.7587205E6,3.2075E4,5.4798927E-3,1.0472691E-2,1.4E1,1.281E3,1.0092541E8,-2.21314E-3,-2.292266E-3,1.2820834E7,3.7688268E-3,1.0965808E-2,2.277029E-3,9.037467E-3,1.8887774E-3,5.306E3,-4.352378E-3,5.6101177E-3,1.2731254E-2,2.1059021E-2,-4.447894E-3,-8.290691E-3,-9.5859176E-4,-4.9006348E-3,-3.3029483E-3,1.3655274E-4,-6.0596596E-4,6.691559E-3,1.4183397E-3,-5.456215E-3,-1.0743662E-3,-5.308265E-3,-2.358725E-3,-1.7983975E-2,3.2831766E-3,-2.9287226E-3,-1.443607E-3,2.3336082E-3,-1.4271461E-3,-1.09691955E-2,-2.4505972E-3,4.282196E-3,3.6186844E-3,8.956619E-3,6.3879024E-3,1.4070527E-3,3.7921395E-3,-2.8258574E-4,4.579095E-3,1.1755486E-2],"split_indices":[2,43,17,3,52,53,12,44,52,67,67,0,0,43,43,15,2,32,0,62,68,9,0,0,0,67,2,5,73,73,10,8,0,7,0,43,44,0,0,61,0,58,0,71,66,0,0,0,1,2,4,0,8,0,0,70,0,2,0,53,8,0,4,67,0,62,60,2,0,0,9,0,60,0,0,0,44,60,10,68,53,3,3,1,73,46,4,0,62,0,0,0,0,69,68,0,29,69,0,32,0,0,0,0,0,0,9,5,43,0,0,62,0,0,73,43,0,7,0,0,0,2,62,0,68,0,0,0,0,1,0,69,0,48,0,60,69,5,0,8,5,0,0,0,0,9,4,0,68,43,71,43,1,0,0,3,44,7,0,0,9,0,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.86E2,7.37E2,2.49E2,3.66E2,3.71E2,1.5E1,2.34E2,3.47E2,1.9E1,3.32E2,3.9E1,9E0,6E0,1.3E1,2.21E2,7.9E1,2.68E2,1.5E1,4E0,2.43E2,8.9E1,1E1,2.9E1,6E0,7E0,8.8E1,1.33E2,4.9E1,3E1,1.22E2,1.46E2,9E0,6E0,2.37E2,6E0,5.8E1,3.1E1,6E0,4E0,2.2E1,7E0,8.3E1,5E0,7.4E1,5.9E1,4.4E1,5E0,1.8E1,1.2E1,8.7E1,3.5E1,7E0,1.39E2,5E0,4E0,1.59E2,7.8E1,5E1,8E0,1.9E1,1.2E1,8E0,1.4E1,7.8E1,5E0,3E1,4.4E1,5.5E1,4E0,4E0,4E1,5E0,1.3E1,4E0,8E0,6E0,8.1E1,2.3E1,1.2E1,8.7E1,5.2E1,5.3E1,1.06E2,1E1,6.8E1,4E1,1E1,7E0,1.2E1,6E0,6E0,7E0,7E0,5.9E1,1.9E1,4E0,2.6E1,3.7E1,7E0,4.7E1,8E0,4E0,3.6E1,4E0,9E0,6E0,7.5E1,1.4E1,9E0,5E0,7E0,8E1,7E0,9E0,4.3E1,4.8E1,5E0,1.01E2,5E0,6E0,4E0,2.3E1,4.5E1,1E1,3E1,6E0,4E0,8E0,4E0,5.5E1,4E0,1.5E1,4E0,8E0,1.8E1,2.6E1,1.1E1,4E1,7E0,5.9E1,1.6E1,1E1,4E0,5E0,4E0,5.3E1,2.7E1,1.8E1,2.5E1,2.1E1,2.7E1,9.2E1,9E0,1.2E1,1.1E1,3.1E1,1.4E1,2.5E1,5E0,6E0,4.9E1,4E0,1.1E1,4E0,4E0,7E0,1.9E1,4E0,7E0,3.3E1,7E0,3.4E1,2.5E1,1.1E1,5E0,4.5E1,8E0,2.2E1,5E0,1.4E1,4E0,4E0,2.1E1,1.7E1,4E0,1.9E1,8E0,3.5E1,5.7E1,5E0,4E0,2.2E1,9E0,5E0,9E0,1.2E1,1.3E1,4E1,9E0,1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[-3.7201808E-3,-5.2778203E-2,1.3622658E-1,-1.0779125E-1,2.8416028E-2,1.6271453E-1,-2.0433286E-1,-1.8276516E-1,-6.9586866E-2,-1.660413E-2,7.807986E-2,4.2367348E-1,1.4083019E-1,-1.9697698E-2,-1.0189746E-1,-1.9264838E-1,4.378566E-3,-1.3625574E-1,-3.219716E-2,-8.073628E-3,-9.6092345E-3,-1.9239467E-2,1.0486154E-1,2.2238046E-2,1.0093383E-2,9.120168E-2,2.448266E-1,-6.5098745E-3,1.9902634E-4,4.2371196E-3,-2.030865E-1,-1.2584434E-1,-1.5233027E-2,-1.4324683E-2,-1.6117576E-1,4.5197673E-2,-2.8210383E-2,6.2002182E-2,-7.5146645E-2,1.563233E-1,5.1622476E-2,5.3254947E-2,1.9003272E-1,2.681881E-1,-3.165949E-3,-1.8626359E-1,-3.1143346E-1,-1.5406427E-1,-8.2578465E-2,-1.1710958E-3,-1.3650678E-1,-2.4947971E-1,4.1078646E-3,7.8360856E-2,-5.258918E-3,-4.8107103E-2,3.313864E-2,3.8908428E-4,6.100071E-3,-1.3480869E-1,-2.0037354E-3,1.3065596E-1,1.4026982E-2,1.3842005E-1,1.3194412E-2,-2.2411019E-2,9.1375254E-2,2.3884837E-1,1.7470121E-4,2.2469002E-1,3.813664E-1,-2.4950086E-1,-1.489723E-1,-1.7351644E-2,-7.251859E-3,-1.0222929E-1,-1.8334144E-1,-1.1884007E-1,-1.799592E-2,-4.8711717E-2,2.3742288E-2,-8.246425E-3,-1.3399397E-6,-1.6387108E-1,-2.0294687E-2,2.10715E-2,1.376623E-1,-1.5259614E-2,-3.0737117E-2,1.5241931E-2,4.583055E-3,-1.5907167E-3,-9.827345E-3,2.0346134E-3,-2.6266773E-3,7.951623E-2,1.6847312E-1,1.6358165E-1,1.86106E-3,3.7603915E-2,-8.082834E-3,-1.2975219E-2,1.410646E-2,2.0414834E-1,4.896818E-2,3.3730572E-1,1.6533695E-1,8.374456E-3,-6.9652623E-3,2.3724182E-1,2.6215327E-3,2.0265052E-2,1.1777775E-2,-1.5467033E-2,-2.1312988E-1,-9.513864E-2,-2.3620494E-1,-2.4398442E-3,-1.3165367E-1,-2.6639467E-3,-1.9605726E-1,-1.4719027E-1,-5.896057E-4,-3.652433E-3,1.4035088E-2,3.0709058E-3,-9.1966264E-2,1.1734547E-2,6.3127475E-3,-2.4482403E-3,-2.0292285E-1,-2.0088296E-2,5.6573905E-2,4.427066E-3,1.049974E-2,5.0690905E-3,-8.101064E-2,5.6614406E-2,-3.879958E-2,-1.5586324E-3,4.797947E-3,1.0187592E-2,1.1876606E-1,3.6456757E-3,8.757406E-3,-4.4908606E-2,6.867655E-2,5.0229605E-2,-5.479357E-3,3.3455284E-3,2.4078888E-1,-7.904197E-3,6.843712E-2,1.9115336E-2,7.6878057E-3,9.614157E-2,1.0431385E-2,2.4852222E-1,4.077983E-3,-1.2565714E-1,-1.1456709E-2,1.870128E-3,-1.1665622E-1,-2.6607448E-1,-4.425153E-3,-7.7513563E-3,-3.652908E-3,-2.0956075E-1,-3.3782672E-3,-8.432478E-3,-3.505158E-3,2.7623072E-3,-1.2561356E-3,3.647485E-3,-1.8755503E-2,-2.1379977E-2,-7.443925E-3,7.988813E-2,-5.0174096E-3,-1.187403E-2,-4.500849E-3,1.4127657E-3,-2.7240717E-3,2.1736983E-4,3.951917E-3,3.2079108E-2,-4.9249128E-2,-1.2078163E-1,-3.1795375E-2,3.62021E-3,8.30691E-5,-3.098265E-3,3.940629E-4,8.457033E-2,8.49701E-3,2.0760712E-3,-4.7432054E-3,7.246409E-3,2.5563816E-2,-2.993375E-2,1.2704432E-1,6.986435E-3,1.3566222E-2,-2.964809E-3,9.289515E-2,4.3040756E-4,6.393469E-3,1.6590248E-1,2.6738778E-1,-1.8351442E-3,-8.37886E-3,-3.1916583E-3,-8.34454E-3,-1.3724857E-2,-5.4687564E-3,-5.771147E-3,-1.0581565E-2,-2.7714788E-3,1.506153E-3,-3.0180826E-3,2.8991923E-3,6.0761184E-3,-1.266455E-3,-1.4005963E-3,2.159663E-3,3.5238713E-3,-1.0352954E-3,-4.372605E-3,7.488627E-5,-1.9527575E-3,-6.417076E-3,1.552799E-4,-3.811379E-3,5.5943653E-3,1.0061627E-3,4.105663E-3,1.9326035E-4,-3.472781E-3,-3.0725062E-4,9.633285E-3,2.2896563E-3,5.334121E-3,-1.3267531E-3,3.5266308E-3,1.052502E-2,5.9843413E-3,1.3464372E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,35,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,-1,85,-1,87,89,-1,-1,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,-1,131,133,-1,135,137,-1,-1,-1,-1,-1,139,141,143,-1,145,-1,-1,147,149,151,153,155,-1,-1,157,-1,-1,-1,-1,159,161,163,-1,165,-1,167,169,-1,-1,171,173,175,177,-1,-1,179,181,183,-1,-1,185,187,189,191,-1,-1,-1,193,-1,-1,195,197,199,-1,-1,201,-1,203,-1,-1,205,-1,207,-1,209,-1,-1,211,213,-1,-1,-1,215,-1,-1,-1,-1,-1,-1,217,219,-1,221,223,-1,-1,-1,-1,-1,-1,225,227,229,231,-1,-1,-1,-1,233,-1,-1,-1,-1,235,237,239,-1,-1,-1,241,-1,-1,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.167918E0,3.4558551E0,2.4778628E0,1.3077388E0,7.011248E-1,1.3794842E0,4.1143525E-1,4.4342375E-1,7.6088107E-1,2.6771158E-1,3.903674E-1,1.1971378E-1,1.185389E0,0E0,6.275605E-2,4.638586E-1,0E0,1.8007803E-1,4.5411432E-1,1.7183612E-1,0E0,1.5454498E-1,3.1248403E-1,0E0,0E0,5.899303E-1,5.674114E-1,0E0,0E0,0E0,2.0628405E-1,1.1669934E-1,0E0,2.810884E-1,5.5220246E-1,2.4097246E-1,1.4318895E-1,5.1424187E-2,8.7006986E-2,1.8339312E-1,1.9496588E-1,3.3952376E-1,4.0775204E-1,2.5892353E-1,0E0,2.6468086E-1,1.1940634E-1,7.686496E-2,1.0090348E-1,1.8947577E-1,9.7418875E-2,2.204994E-1,0E0,1.228009E-1,0E0,4.2119813E-1,3.2775342E-2,0E0,0E0,7.869843E-2,2.775989E-2,8.716953E-2,0E0,3.7857294E-2,1.9947144E-1,3.814703E-1,3.6302853E-1,1.9755173E-1,3.0067497E-1,9.8263025E-2,1.8528938E-2,7.134223E-2,3.6730015E-1,0E0,0E0,2.9399306E-2,5.102122E-2,8.232668E-2,3.4359895E-2,1.2347327E-1,1.4074244E-1,0E0,0E0,4.986608E-2,0E0,3.0361911E-2,4.570231E-2,0E0,1.5161939E-1,5.585656E-2,0E0,0E0,0E0,0E0,0E0,6.965536E-2,4.882914E-2,1.5833259E-2,0E0,1.0051168E-1,0E0,0E0,1.785375E-1,8.392513E-2,2.6019835E-1,1.1407268E-1,7.032287E-2,0E0,0E0,6.040573E-2,0E0,0E0,0E0,0E0,7.555199E-2,1.5938577E-1,1.0453725E-1,0E0,1.5230447E-2,0E0,4.2776823E-2,4.1592747E-2,0E0,0E0,2.426122E-2,4.4542007E-2,1.4300957E-1,1.1000459E-1,0E0,0E0,3.2031745E-2,2.1500658E-2,1.6124956E-2,0E0,0E0,7.4936666E-2,6.466657E-2,1.6548164E-2,1.528679E-2,0E0,0E0,0E0,2.2322446E-2,0E0,0E0,6.1698616E-2,1.0243173E-1,1.8237695E-1,0E0,0E0,3.1340778E-2,0E0,1.8110633E-1,0E0,0E0,4.2729057E-2,0E0,2.5779963E-2,0E0,5.0241843E-2,0E0,0E0,1.2749565E-1,7.932532E-2,0E0,0E0,0E0,2.250719E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.642689E-2,6.325996E-2,0E0,1.0886927E-1,1.035655E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.358866E-2,3.941317E-2,2.115795E-2,3.0580873E-2,0E0,0E0,0E0,0E0,2.7175508E-2,0E0,0E0,0E0,0E0,2.6698302E-2,1.5298711E-2,8.4357694E-2,0E0,0E0,0E0,1.2954587E-1,0E0,0E0,4.3611705E-2,7.8309774E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,21,21,22,22,25,25,26,26,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,55,55,56,56,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,79,79,80,80,83,83,85,85,86,86,88,88,89,89,95,95,96,96,97,97,99,99,102,102,103,103,104,104,105,105,106,106,109,109,114,114,115,115,116,116,118,118,120,120,121,121,124,124,125,125,126,126,127,127,130,130,131,131,132,132,135,135,136,136,137,137,138,138,142,142,145,145,146,146,147,147,150,150,152,152,155,155,157,157,159,159,162,162,163,163,167,167,174,174,175,175,177,177,178,178,185,185,186,186,187,187,188,188,193,193,198,198,199,199,200,200,204,204,207,207,208,208],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,36,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,-1,86,-1,88,90,-1,-1,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,-1,132,134,-1,136,138,-1,-1,-1,-1,-1,140,142,144,-1,146,-1,-1,148,150,152,154,156,-1,-1,158,-1,-1,-1,-1,160,162,164,-1,166,-1,168,170,-1,-1,172,174,176,178,-1,-1,180,182,184,-1,-1,186,188,190,192,-1,-1,-1,194,-1,-1,196,198,200,-1,-1,202,-1,204,-1,-1,206,-1,208,-1,210,-1,-1,212,214,-1,-1,-1,216,-1,-1,-1,-1,-1,-1,218,220,-1,222,224,-1,-1,-1,-1,-1,-1,226,228,230,232,-1,-1,-1,-1,234,-1,-1,-1,-1,236,238,240,-1,-1,-1,242,-1,-1,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.9016544E5,4.217427E7,8.147158E4,1.071E3,3.6E1,2.5330253E-2,1.482958E0,5.47E2,2.8461537E0,5.036081E7,6.441311E-1,2.9251662E6,-1.9697698E-2,2.3308511E5,7E0,4.378566E-3,1E0,2.2E1,9.917037E2,-9.6092345E-3,3.1842105E0,8E0,2.2238046E-2,1.0093383E-2,1.5142261E3,5.2964807E0,-6.5098745E-3,1.9902634E-4,4.2371196E-3,4.217427E7,3.72E2,-1.5233027E-2,8.233192E9,3.7725E1,3.7103873E2,3.3486558E6,5.1942E4,1.2839025E1,3.549E3,2.1028344E4,6.648E3,2.856934E7,1.6466942E3,-3.165949E-3,3.175E1,1.5416006E5,3.1E1,1.339646E6,2.9211267E2,1.731E3,4E1,4.1078646E-3,1.1015625E-4,-5.258918E-3,1.2324481E2,3.1847827E2,3.8908428E-4,6.100071E-3,1.5115256E6,1.3254E4,2E0,1.4026982E-2,1.5E1,1.2619E4,1.4076087E0,1.5209424E1,5.6516E4,2.9622424E3,9.244374E6,1.5454545E0,9.2058825E-1,1.8797773E-1,-1.7351644E-2,-7.251859E-3,1.6E1,1.0443479E5,8E0,1.5292561E2,1.6808511E0,1.9099288E7,-8.246425E-3,-1.3399397E-6,3.7509E4,-2.0294687E-2,5.8899284E1,3.5589743E0,-1.5259614E-2,1.6153846E0,2.0487332E-3,4.583055E-3,-1.5907167E-3,-9.827345E-3,2.0346134E-3,-2.6266773E-3,5.441268E7,3.44E2,4.1388846E2,1.86106E-3,3.1609525E2,-8.082834E-3,-1.2975219E-2,1.1555496E8,4.6847186E2,3.97371E5,3.7936268E-3,1.3567028E5,8.374456E-3,-6.9652623E-3,1.562E5,2.6215327E-3,2.0265052E-2,1.1777775E-2,-1.5467033E-2,3.7452831E0,9.935484E-1,1.1408248E0,-2.4398442E-3,2.8003533E0,-2.6639467E-3,7.997723E6,1.9545455E0,-5.896057E-4,-3.652433E-3,2.1790265E2,1E1,9.966666E0,3.8E1,6.3127475E-3,-2.4482403E-3,4.88451E5,9.836066E-1,1.623E3,4.427066E-3,1.049974E-2,4.9390244E0,4.1361522E6,5.41E2,1.3809524E0,-1.5586324E-3,4.797947E-3,1.0187592E-2,1.9505818E8,3.6456757E-3,8.757406E-3,1E0,7E0,2.4358286E7,-5.479357E-3,3.3455284E-3,1E0,-7.904197E-3,2.5711747E5,1.9115336E-2,7.6878057E-3,9.67E2,1.0431385E-2,1.5137369E6,4.077983E-3,2.8877066E4,-1.1456709E-2,1.870128E-3,1.31E2,2.8027795E4,-4.425153E-3,-7.7513563E-3,-3.652908E-3,1.2608696E0,-3.3782672E-3,-8.432478E-3,-3.505158E-3,2.7623072E-3,-1.2561356E-3,3.647485E-3,2.0116506E4,1.4381613E7,-7.443925E-3,4.45E2,1.535E3,-1.187403E-2,-4.500849E-3,1.4127657E-3,-2.7240717E-3,2.1736983E-4,3.951917E-3,1E0,4.0119403E2,5.6666665E0,4.267606E0,3.62021E-3,8.30691E-5,-3.098265E-3,3.940629E-4,9.94E2,8.49701E-3,2.0760712E-3,-4.7432054E-3,7.246409E-3,2.2307692E0,1.5204346E8,8.818731E0,6.986435E-3,1.3566222E-2,-2.964809E-3,1.2792593E2,4.3040756E-4,6.393469E-3,6.000379E8,3.0238653E8,-1.8351442E-3,-8.37886E-3,-3.1916583E-3,-8.34454E-3,-1.3724857E-2,-5.4687564E-3,-5.771147E-3,-1.0581565E-2,-2.7714788E-3,1.506153E-3,-3.0180826E-3,2.8991923E-3,6.0761184E-3,-1.266455E-3,-1.4005963E-3,2.159663E-3,3.5238713E-3,-1.0352954E-3,-4.372605E-3,7.488627E-5,-1.9527575E-3,-6.417076E-3,1.552799E-4,-3.811379E-3,5.5943653E-3,1.0061627E-3,4.105663E-3,1.9326035E-4,-3.472781E-3,-3.0725062E-4,9.633285E-3,2.2896563E-3,5.334121E-3,-1.3267531E-3,3.5266308E-3,1.052502E-2,5.9843413E-3,1.3464372E-2],"split_indices":[2,43,60,43,2,0,53,57,2,68,7,53,43,0,48,3,0,106,3,48,0,69,8,0,0,67,53,0,0,0,60,2,0,46,71,70,43,1,73,10,48,44,60,67,0,70,48,10,9,4,2,3,0,53,0,67,4,0,0,43,44,8,0,3,44,69,71,9,67,60,68,68,57,0,0,3,43,8,67,68,9,0,0,1,0,62,69,0,68,53,0,0,0,0,0,7,10,4,0,67,0,0,5,4,1,72,48,0,0,10,0,0,0,0,69,68,53,0,68,0,9,68,0,0,4,3,73,0,0,0,9,71,44,0,0,71,60,2,68,0,0,0,5,0,0,26,3,5,0,0,23,0,43,0,0,0,0,62,0,60,0,0,44,48,0,0,0,68,0,0,0,0,0,0,48,5,0,44,2,0,0,0,0,0,0,26,4,73,71,0,0,0,0,10,0,0,0,0,71,7,71,0,0,0,73,0,0,5,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,7.72E2,2.7E2,4.6E2,3.12E2,2.51E2,1.9E1,1.54E2,3.06E2,1.64E2,1.48E2,1.8E1,2.33E2,5E0,1.4E1,1.49E2,5E0,1.09E2,1.97E2,1.58E2,6E0,3.2E1,1.16E2,1.3E1,5E0,1.59E2,7.4E1,1E1,4E0,5E0,1.44E2,1.05E2,4E0,1.74E2,2.3E1,4.3E1,1.15E2,1.3E1,1.9E1,5.8E1,5.8E1,1.16E2,4.3E1,6.9E1,5E0,1.27E2,1.7E1,6.2E1,4.3E1,1.58E2,1.6E1,1.7E1,6E0,3.6E1,7E0,8.7E1,2.8E1,8E0,5E0,1E1,9E0,5.1E1,7E0,1.7E1,4.1E1,3.9E1,7.7E1,3.4E1,9E0,5.2E1,1.7E1,4.5E1,8.2E1,1.1E1,6E0,2.4E1,3.8E1,2.7E1,1.6E1,5.4E1,1.04E2,1.2E1,4E0,1.3E1,4E0,1.9E1,1.7E1,4E0,8.3E1,2.3E1,5E0,5E0,5E0,5E0,4E0,2.3E1,2.8E1,1.3E1,4E0,3.7E1,4E0,4E0,3.5E1,2E1,5.7E1,1.3E1,2.1E1,4E0,5E0,4.8E1,4E0,1E1,7E0,1.1E1,3.4E1,5.2E1,3E1,1E1,1.4E1,4E0,3.4E1,2.1E1,6E0,5E0,1.1E1,2.5E1,2.9E1,9.5E1,9E0,4E0,9E0,9E0,1E1,1.3E1,4E0,4.9E1,3.4E1,1.3E1,1E1,4E0,1.9E1,1.2E1,1.6E1,4E0,9E0,1E1,2.7E1,2.8E1,7E0,5E0,1.5E1,4E0,5.3E1,8E0,5E0,1.1E1,1E1,4.4E1,4E0,1.1E1,2.3E1,7E0,4.5E1,2.4E1,6E0,7E0,7E0,3E1,4E0,1.3E1,8E0,5E0,6E0,5E0,2E1,1.5E1,1.4E1,1.8E1,7.7E1,5E0,4E0,4E0,5E0,4E0,6E0,3.3E1,1.6E1,1.8E1,1.6E1,9E0,4E0,6E0,4E0,1.2E1,4E0,4E0,6E0,8E0,1.9E1,1.4E1,1.4E1,7E0,8E0,8E0,4.5E1,4E0,7E0,1E1,3.4E1,5E0,6E0,2.7E1,1.8E1,1.9E1,5E0,7E0,2.3E1,1.1E1,9E0,1E1,5E0,1.2E1,6E0,5.2E1,2.5E1,1.8E1,1.5E1,8E0,8E0,4E0,1.4E1,1E1,6E0,7E0,5E0,4E0,1.5E1,4E0,1E1,6E0,8E0,3.8E1,7E0,5E0,5E0,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"247","size_leaf_vector":"1"}},{"base_weights":[-5.6042373E-3,-6.0386825E-2,1.2993024E-1,-1.5974328E-1,-1.3175338E-2,-3.243074E-1,1.5108468E-1,1.9640605E-1,-1.7365013E-1,-8.107625E-3,-2.2728685E-2,-2.524669E-2,-1.6484608E-1,6.4293586E-2,1.8931831E-1,1.4133215E-2,2.1995762E-3,-1.455832E-1,-2.8846595E-1,-5.3682692E-2,3.3824727E-2,-1.0733918E-2,-2.974399E-3,-2.7115382E-2,1.1999552E-1,2.8986552E-1,1.5005909E-1,-2.1175928E-1,-9.8635666E-2,-4.4349626E-1,-1.7600514E-1,-8.8702016E-2,5.1713306E-3,8.3594695E-2,-7.110025E-3,7.0060813E-3,-8.400357E-3,1.3772951E-1,-3.594129E-3,1.9723479E-2,2.1709171E-1,2.184235E-1,9.141413E-2,-2.9992566E-1,-1.8575497E-1,4.8802393E-3,-1.14146076E-1,-3.0517232E-2,-1.2244924E-2,-2.369582E-3,-2.1085495E-1,-7.84992E-2,-1.6382184E-2,-1.09717175E-1,3.299895E-2,6.779765E-2,2.2247683E-1,7.53344E-3,-1.04338124E-1,5.3205886E-3,-2.1514324E-2,6.215589E-2,1.7960593E-1,2.1804802E-3,2.429715E-1,1.6468567E-1,2.24881E-2,1.0880122E-1,-8.18845E-3,-6.986421E-3,-1.8994104E-2,-1.3678305E-1,-2.2019397E-1,-1.6975993E-1,-6.656845E-2,-1.4315485E-2,-1.4628866E-1,6.1695995E-3,-8.906914E-2,-2.9593755E-2,-1.0161552E-2,1.6191307E-2,9.682485E-3,8.215516E-3,1.0017252E-1,4.7066435E-3,1.2789598E-2,9.347914E-2,-1.1641701E-2,-4.5176525E-2,-9.938889E-3,-4.856461E-4,-4.486872E-3,1.120196E-1,6.076592E-3,2.1792844E-1,3.5728407E-3,1.1433973E-1,1.3062398E-2,1.870648E-1,-2.1032027E-3,1.7800152E-1,5.5134304E-2,-8.049488E-3,-4.2519826E-2,-5.101213E-3,-2.3152794E-1,-2.1118683E-1,-7.282702E-2,-1.3332805E-1,-1.44184865E-2,-2.095544E-3,-8.413984E-3,-6.916259E-2,-1.6833213E-1,-3.8833516E-3,6.253603E-4,6.3711755E-2,-9.6031455E-3,3.47807E-2,-7.8296244E-2,7.642113E-2,1.8293333E-1,3.4571737E-2,7.424689E-3,5.619764E-3,-2.1548659E-2,-3.5654642E-3,-3.106019E-4,-3.0075978E-3,1.2585211E-3,7.483405E-3,2.3891607E-3,-2.4689632E-3,2.5253573E-3,1.1354262E-2,4.7459104E-3,8.334024E-3,1.1742724E-3,1.5804507E-2,1.4363039E-1,3.170089E-3,2.0704596E-1,-5.1479917E-3,1.760666E-1,7.642351E-4,-4.300111E-3,-1.135412E-2,-5.268347E-3,-1.4717348E-1,-1.21041965E-2,-1.2980774E-1,1.5836431E-3,-2.2744222E-3,-7.625631E-3,3.1777162E-2,-4.873954E-3,-1.0042541E-1,3.3298023E-2,-3.259218E-1,-6.5730415E-2,1.2241098E-3,4.7077537E-3,-5.6773257E-3,1.20887365E-2,1.1478007E-2,5.6452784E-3,-5.6093144E-3,-9.016676E-4,1.0075476E-1,1.3852444E-2,3.4378578E-3,1.0112955E-2,4.5929444E-3,-4.114629E-3,-7.141527E-2,9.774558E-3,7.47523E-2,1.7047983E-1,2.2311167E-1,4.683377E-3,2.7128326E-2,-5.1033003E-3,4.0393667E-3,9.36807E-3,-8.357427E-3,-3.6020183E-3,-2.5368736E-3,-7.880929E-3,-6.4360676E-4,3.2814434E-3,1.7691898E-3,-4.986447E-3,-4.4692855E-4,4.286147E-3,-2.5783906E-2,-3.608875E-3,1.1859142E-4,-5.854548E-3,1.5286996E-3,-3.8179467E-3,-2.0952502E-3,1.3314765E-3,8.042869E-3,3.325956E-3,-1.8886509E-3,2.3460074E-3,-4.840639E-3,1.867129E-4,-3.2826783E-3,2.0799898E-3,-2.2710275E-3,6.25005E-3,6.351626E-3,1.0893004E-2,1.2034015E-2,6.213594E-3,-2.6580174E-3,2.4874485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,-1,-1,63,65,67,69,71,-1,73,-1,-1,-1,75,77,-1,79,81,83,85,87,89,-1,91,93,95,-1,97,99,-1,101,-1,-1,-1,103,105,107,109,-1,111,-1,113,115,-1,117,-1,119,121,-1,-1,123,125,127,-1,129,-1,131,133,135,-1,137,-1,139,-1,141,143,-1,145,-1,147,149,151,153,155,-1,-1,157,159,-1,-1,161,163,165,167,169,171,173,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,177,-1,179,181,183,-1,-1,-1,-1,185,-1,187,-1,-1,-1,189,-1,191,193,195,197,-1,-1,-1,199,201,-1,-1,-1,203,205,-1,-1,-1,-1,207,209,211,213,215,-1,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3511014E0,3.3048859E0,2.787603E0,1.1584172E0,1.1582612E0,3.7047112E-1,8.9373446E-1,1.3209131E-1,6.6373825E-1,9.09574E-1,0E0,0E0,4.606831E-2,4.360445E-1,7.025628E-1,0E0,0E0,5.321238E-1,6.542058E-1,4.703731E-1,5.058501E-1,0E0,0E0,1.7548345E-1,1.9661987E-1,4.259796E-1,5.3330636E-1,1.1165476E-1,3.478999E-1,4.8341203E-1,1.0218817E-1,3.601104E-1,2.785689E-1,2.3063636E-1,1.9584063E-1,8.885164E-2,0E0,1.4390928E-1,0E0,0E0,1.4681816E-1,8.283036E-1,3.7429255E-1,1.8588364E-1,6.9387436E-2,0E0,2.5065088E-1,0E0,0E0,0E0,8.2524896E-2,3.2343298E-1,0E0,1.4364082E-1,2.0436262E-1,1.9593135E-1,4.3910086E-2,1.9885078E-1,1.0826635E-1,0E0,3.652046E-2,5.2449413E-2,1.0464311E-1,0E0,1.2625837E-1,2.6254976E-1,0E0,2.575773E-1,0E0,0E0,0E0,8.791059E-2,1.5493631E-2,1.6532314E-1,1.895607E-1,0E0,4.2375714E-2,0E0,1.9821203E-1,2.6976772E-2,0E0,8.028647E-2,0E0,8.7756634E-2,1.1575526E-1,0E0,0E0,8.157377E-2,1.333448E-1,1.5477557E-2,0E0,3.581347E-2,0E0,2.1752939E-2,3.1838868E-2,4.5310736E-2,0E0,4.827781E-2,0E0,2.8081524E-1,0E0,8.42793E-2,3.0708963E-1,0E0,2.8414294E-2,0E0,2.0484447E-2,6.227684E-2,9.599407E-2,5.5920184E-2,1.3799325E-1,0E0,0E0,3.492903E-1,3.9897025E-1,0E0,0E0,2.998861E-2,1.0715246E-1,5.76699E-2,2.0441346E-2,7.929799E-2,3.8353354E-2,1.13076106E-1,0E0,0E0,1.4596242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1229825E-2,0E0,1.8127084E-2,1.0157322E-1,1.5413284E-2,0E0,0E0,0E0,0E0,2.324143E-2,0E0,2.1174312E-2,0E0,0E0,0E0,4.045987E-2,0E0,9.058535E-2,6.73502E-2,5.317664E-1,7.1142726E-2,0E0,0E0,0E0,7.634213E-2,2.4961041E-2,0E0,0E0,0E0,6.762332E-2,3.441491E-2,0E0,0E0,0E0,0E0,9.105116E-2,1.6844496E-1,1.0415837E-1,2.7917206E-2,3.1197786E-2,0E0,5.5115484E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,40,40,41,41,42,42,43,43,44,44,46,46,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,64,64,65,65,67,67,71,71,72,72,73,73,74,74,76,76,78,78,79,79,81,81,83,83,84,84,87,87,88,88,89,89,91,91,93,93,94,94,95,95,97,97,99,99,101,101,102,102,104,104,106,106,107,107,108,108,109,109,110,110,113,113,114,114,117,117,118,118,119,119,120,120,121,121,122,122,123,123,126,126,140,140,142,142,143,143,144,144,149,149,151,151,155,155,157,157,158,158,159,159,160,160,164,164,165,165,169,169,170,170,175,175,176,176,177,177,178,178,179,179,181,181],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,-1,-1,64,66,68,70,72,-1,74,-1,-1,-1,76,78,-1,80,82,84,86,88,90,-1,92,94,96,-1,98,100,-1,102,-1,-1,-1,104,106,108,110,-1,112,-1,114,116,-1,118,-1,120,122,-1,-1,124,126,128,-1,130,-1,132,134,136,-1,138,-1,140,-1,142,144,-1,146,-1,148,150,152,154,156,-1,-1,158,160,-1,-1,162,164,166,168,170,172,174,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,178,-1,180,182,184,-1,-1,-1,-1,186,-1,188,-1,-1,-1,190,-1,192,194,196,198,-1,-1,-1,200,202,-1,-1,-1,204,206,-1,-1,-1,-1,208,210,212,214,216,-1,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1.972052E5,1E0,7E0,5.6E1,1.2877E4,6.9664386E2,1.1634076E4,2.2E1,8.49E2,-2.2728685E-2,-2.524669E-2,4.27E3,4.7567694E5,1.0717949E1,1.4133215E-2,2.1995762E-3,2.71E2,2E0,2.61E2,3.3358974E0,-1.0733918E-2,-2.974399E-3,4.233978E0,1E0,1.4839433E5,5.932344E1,9.2058825E-1,2.2E1,3.428562E2,6.48334E5,1.925673E1,1.727279E-4,1.8857143E0,1E0,2.7662E4,-8.400357E-3,4.6134964E2,-3.594129E-3,1.9723479E-2,8.118812E-1,3.4915986E3,4.023131E2,3.959E3,1.1066655E-2,4.8802393E-3,1.4912975E-1,-3.0517232E-2,-1.2244924E-2,-2.369582E-3,7.336111E4,7E0,-1.6382184E-2,3.4919708E2,8.257819E9,1.295313E6,5.6530495E6,9E0,2.9690266E4,5.3205886E-3,3.257507E5,7.4010696E0,7.403968E2,2.1804802E-3,1.4831979E0,4.2964826E0,2.24881E-2,6.9E1,-8.18845E-3,-6.986421E-3,-1.8994104E-2,1.8187732E4,1.3947369E0,4.8E0,1.5152774E2,-1.4315485E-2,2.234E3,6.1695995E-3,6.83E2,1.1643481E6,-1.0161552E-2,1.4339622E0,9.682485E-3,1.683263E7,1.19216E8,4.7066435E-3,1.2789598E-2,3.1609525E2,3.757994E0,3.6572136E7,-9.938889E-3,1.5204346E8,-4.486872E-3,5E0,8.909296E4,1.4433751E0,3.5728407E-3,9.5139277E-1,1.3062398E-2,2.1412018E6,-2.1032027E-3,1.4320667E3,7.802E-2,-8.049488E-3,3.1416278E2,-5.101213E-3,5.0710382E0,5.449711E8,8.39E2,2.9157894E0,3.38E2,-2.095544E-3,-8.413984E-3,3.1540035E9,1.5077367E3,-3.8833516E-3,6.253603E-4,2.5677419E0,2.2E1,4.07E2,4.1957852E2,1.8454931E8,5.4007E7,3.7066666E2,7.424689E-3,5.619764E-3,3.35E2,-3.5654642E-3,-3.106019E-4,-3.0075978E-3,1.2585211E-3,7.483405E-3,2.3891607E-3,-2.4689632E-3,2.5253573E-3,1.1354262E-2,4.7459104E-3,8.334024E-3,1.1742724E-3,1.5804507E-2,4.186E4,3.170089E-3,2.0666666E0,1.7291568E2,7.555942E6,7.642351E-4,-4.300111E-3,-1.135412E-2,-5.268347E-3,5.15E2,-1.21041965E-2,4.0753E4,1.5836431E-3,-2.2744222E-3,-7.625631E-3,7.89E2,-4.873954E-3,8.414097E-1,4.49239E6,5.1942E4,1.9090909E0,1.2241098E-3,4.7077537E-3,-5.6773257E-3,1.6533333E1,4.0449125E5,5.6452784E-3,-5.6093144E-3,-9.016676E-4,1.8452264E6,3.472499E-1,3.4378578E-3,1.0112955E-2,4.5929444E-3,-4.114629E-3,1.1840488E8,7.56484E5,1.2586393E3,7.120907E6,8.847283E8,4.683377E-3,7.25E2,-5.1033003E-3,4.0393667E-3,9.36807E-3,-8.357427E-3,-3.6020183E-3,-2.5368736E-3,-7.880929E-3,-6.4360676E-4,3.2814434E-3,1.7691898E-3,-4.986447E-3,-4.4692855E-4,4.286147E-3,-2.5783906E-2,-3.608875E-3,1.1859142E-4,-5.854548E-3,1.5286996E-3,-3.8179467E-3,-2.0952502E-3,1.3314765E-3,8.042869E-3,3.325956E-3,-1.8886509E-3,2.3460074E-3,-4.840639E-3,1.867129E-4,-3.2826783E-3,2.0799898E-3,-2.2710275E-3,6.25005E-3,6.351626E-3,1.0893004E-2,1.2034015E-2,6.213594E-3,-2.6580174E-3,2.4874485E-3],"split_indices":[2,43,17,3,3,9,67,43,3,2,0,0,2,43,71,0,0,2,32,70,69,0,0,68,6,48,73,68,10,67,46,73,53,68,19,9,0,4,0,0,68,4,73,9,53,0,53,0,0,0,60,3,0,70,46,9,43,3,48,0,43,71,4,0,53,68,0,8,0,0,0,43,68,69,67,0,12,0,2,43,0,68,0,60,5,0,0,67,73,5,0,7,0,3,48,53,0,53,0,60,0,4,72,0,62,0,69,46,2,69,0,0,0,46,48,0,0,69,0,0,4,7,7,4,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,9,0,68,73,1,0,0,0,0,44,0,12,0,0,0,2,0,68,43,1,68,0,0,0,73,43,0,0,0,60,53,0,0,0,0,7,12,4,60,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.88E2,7.04E2,2.84E2,2.26E2,4.78E2,1.2E1,2.72E2,8E0,2.18E2,4.74E2,4E0,4E0,8E0,8.4E1,1.88E2,4E0,4E0,1.77E2,4.1E1,2.27E2,2.47E2,4E0,4E0,3.2E1,5.2E1,5.1E1,1.37E2,7.2E1,1.05E2,1.6E1,2.5E1,1.42E2,8.5E1,1.11E2,1.36E2,2.7E1,5E0,4.8E1,4E0,1.6E1,3.5E1,6.2E1,7.5E1,1.4E1,5.8E1,7E0,9.8E1,6E0,1E1,6E0,1.9E1,1.38E2,4E0,1.6E1,6.9E1,1.01E2,1E1,1.19E2,1.7E1,5E0,2.2E1,1.8E1,3E1,5E0,3E1,5.3E1,9E0,7.1E1,4E0,7E0,7E0,2.6E1,3.2E1,4.4E1,5.4E1,6E0,1.3E1,6E0,1.32E2,1E1,6E0,6.4E1,5E0,3.6E1,6.5E1,4E0,6E0,2.1E1,9.8E1,1.2E1,5E0,1.8E1,4E0,9E0,9E0,2.1E1,9E0,8E0,2.2E1,4.8E1,5E0,3E1,4.1E1,1.8E1,8E0,4E0,2.8E1,3E1,1.4E1,2.3E1,3.1E1,4E0,9E0,1.07E2,2.5E1,4E0,6E0,2.2E1,4.2E1,2.8E1,8E0,5.2E1,1.3E1,1.2E1,9E0,6E0,9.2E1,6E0,6E0,5E0,1.3E1,4E0,5E0,4E0,5E0,1.6E1,5E0,4E0,4E0,9E0,3.9E1,7E0,2.3E1,2.8E1,1.3E1,4E0,4E0,2.4E1,4E0,1.5E1,1.5E1,9E0,5E0,7E0,1.6E1,2.1E1,1E1,8.2E1,2.5E1,9E0,1.6E1,1.2E1,1E1,6E0,3.6E1,2.3E1,5E0,4E0,4E0,3.7E1,1.5E1,4E0,9E0,8E0,4E0,3.5E1,5.7E1,1.2E1,2.7E1,1.9E1,4E0,2.2E1,6E0,4E0,9E0,9E0,6E0,4E0,5E0,1E1,1.1E1,4E0,7.8E1,1.5E1,1E1,4E0,5E0,8E0,8E0,3E1,6E0,5E0,1.8E1,9E0,2.8E1,6E0,9E0,2.4E1,1.1E1,1.7E1,4E1,4E0,8E0,2E1,7E0,1.2E1,7E0,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-3.7557508E-3,-5.356728E-2,1.4186162E-1,-1.528055E-1,-2.857376E-3,-2.832127E-1,1.6357172E-1,-1.3140261E-1,-3.302597E-1,-1.1550596E-1,1.8353786E-2,-1.8730268E-2,-8.493633E-3,6.835811E-2,2.1737468E-1,-1.6718604E-1,-8.6870246E-2,-5.949048E-1,-1.648869E-1,-9.126658E-2,-1.8444842E-2,-9.19258E-3,6.997228E-2,1.09520346E-1,-7.306444E-3,2.8212065E-1,1.00879565E-1,-1.3432595E-2,-1.5529728E-1,1.274936E-2,-1.0777171E-1,-3.4556713E-2,-1.3500018E-2,6.179515E-4,-2.1677454E-1,-1.4211644E-1,-4.7929402E-2,-4.268616E-2,3.2748528E-2,7.840632E-2,-8.635644E-3,4.638027E-2,1.6900975E-1,-4.1179787E-2,4.0469444E-3,2.3710096E-1,4.330049E-1,2.6837327E-2,1.6909108E-1,-1.4538892E-1,-1.4599904E-2,-1.7023702E-1,-2.100985E-2,-1.3649008E-2,-5.582979E-3,-8.309649E-2,-1.7927557E-1,1.3527782E-3,-6.69579E-2,-7.4526265E-2,4.766257E-2,5.0661497E-2,-6.89106E-2,2.434058E-2,1.3213399E-1,-3.7332933E-2,9.68107E-2,2.0912513E-1,-3.390654E-3,1.5296387E-3,-7.80872E-2,2.8110296E-1,1.1583066E-1,2.4640094E-2,1.1977588E-2,-8.712043E-3,7.95725E-2,1.3394545E-1,1.4692688E-2,-1.6311975E-1,-8.5737474E-2,-1.1284824E-1,-1.3032554E-2,5.8899667E-2,-6.592208E-2,-6.0413913E-3,-1.1566941E-3,-9.1470005E-3,-3.8590443E-3,-8.554575E-2,1.5727554E-3,-5.64944E-2,-1.8141877E-2,-5.6810346E-2,1.16797656E-1,1.7292814E-2,8.1288956E-2,-2.2312049E-2,-7.676224E-3,-3.63934E-2,6.519692E-2,1.5005349E-1,-9.975056E-3,-3.116995E-3,8.4780186E-4,4.3281596E-2,6.0893246E-3,1.16314115E-2,1.1572087E-1,-2.0492954E-2,-7.273446E-3,1.6446128E-2,2.1358742E-1,-1.2744347E-3,1.4747547E-1,1.4035816E-1,-5.8292816E-3,9.476695E-3,5.8280297E-2,-2.1458541E-4,-1.707936E-1,2.1300022E-2,-6.3551646E-3,-1.4219256E-1,2.1160326E-3,2.1370817E-2,5.446219E-3,1.6521169E-3,-8.4359266E-2,-3.255745E-2,-1.2656389E-1,-4.2470887E-2,-1.78845E-1,-9.1749795E-2,1.0411858E-3,1.5287383E-1,1.01232676E-4,1.2553751E-1,-1.4810339E-2,1.0559643E-1,2.5457121E-2,-3.823277E-3,4.6852478E-4,-9.754159E-2,1.8690983E-3,1.0593614E-1,-5.7380903E-3,1.6477413E-1,3.0704768E-4,2.0880303E-3,-2.917539E-3,3.8176242E-3,-2.184008E-4,1.246536E-3,7.871543E-3,-3.0559883E-3,2.195523E-3,2.5345418E-1,1.3537777E-3,9.006914E-3,1.765918E-3,2.7478794E-3,8.435069E-3,-9.773464E-4,4.2581656E-3,-1.0102244E-1,-1.849062E-1,3.1487294E-3,-9.838736E-4,-7.738598E-2,-1.8368787E-1,2.8991008E-3,-1.0890296E-3,-4.64967E-3,-1.084472E-4,-3.0389505E-3,1.0488047E-3,-7.977407E-3,-3.3038724E-3,-7.425573E-2,1.3656159E-2,-1.2256857E-2,-1.0178325E-3,-6.5754154E-3,-2.2484954E-3,8.95076E-3,8.593619E-2,2.5366854E-3,8.639962E-3,5.5932276E-2,-4.6011843E-2,9.361874E-3,7.714603E-2,-1.8858064E-3,6.199686E-2,-1.0518959E-2,-1.4058007E-2,7.20199E-2,1.143643E-2,-2.843269E-3,2.264298E-2,1.04351275E-1,2.014991E-1,5.483878E-3,1.3614059E-2,-6.9777896E-3,-6.994794E-4,-1.0109652E-2,-7.368951E-3,-2.4036414E-4,-4.76596E-3,-1.0060209E-2,-4.8676725E-3,1.4966434E-3,-4.370702E-3,6.086785E-3,-1.6152324E-3,1.3124244E-3,5.8338824E-3,4.190093E-3,1.1915165E-5,-3.4006445E-3,2.697135E-4,7.1319556E-3,2.7713517E-3,4.112183E-4,4.0392824E-3,2.9411428E-3,-2.9955518E-3,2.2150865E-3,5.711781E-3,2.2634848E-3,-1.1109737E-3,2.9519508E-3,8.189894E-3,5.9749857E-3,1.2125979E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,-1,49,-1,51,-1,-1,-1,53,55,57,59,61,63,-1,65,67,69,-1,71,73,75,77,79,-1,81,83,-1,-1,85,87,-1,89,91,93,95,97,99,101,103,105,107,-1,-1,109,111,113,-1,-1,-1,115,117,-1,119,121,123,-1,125,127,-1,-1,-1,-1,129,-1,131,-1,133,135,137,139,141,-1,143,145,147,149,-1,-1,151,-1,-1,153,155,-1,-1,157,-1,159,161,-1,-1,163,-1,165,167,-1,169,-1,171,-1,-1,173,175,177,179,181,183,-1,185,-1,187,189,191,193,-1,-1,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,205,207,-1,-1,209,211,-1,-1,-1,-1,-1,-1,-1,-1,213,215,-1,-1,-1,-1,-1,217,-1,-1,219,221,-1,223,-1,225,227,-1,229,-1,-1,231,233,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.398838E0,3.8241575E0,2.4457111E0,9.3269825E-1,1.2063813E0,8.332908E-2,1.2555885E0,3.5357976E-1,1.0835652E0,5.137789E-1,6.049515E-1,0E0,0E0,2.842217E-1,1.163269E0,1.4141941E-1,8.260407E-1,2.4481821E-1,1.6939294E-1,1.5846509E-1,0E0,3.9192057E-1,3.3532184E-1,2.1377504E-1,1.096934E-1,5.8340263E-1,2.8839296E-1,0E0,1.4197063E-1,0E0,5.383841E-1,0E0,0E0,0E0,6.994367E-2,5.9669316E-2,6.366733E-2,4.4944263E-1,2.2929724E-1,4.150787E-1,0E0,1.3026503E-1,3.0706972E-1,7.1131304E-2,0E0,3.943305E-1,2.4526739E-1,3.4444174E-1,1.14438E-1,1.05531216E-1,0E0,3.404932E-1,1.5820612E-1,0E0,0E0,3.8243175E-2,1.9299865E-2,0E0,6.674351E-2,6.375787E-1,3.0276778E-1,1.06926024E-1,8.239066E-2,1.8351261E-1,1.848464E-1,2.137516E-2,2.9887542E-2,7.442236E-2,0E0,0E0,7.446534E-2,2.171936E-1,1.0845107E-1,0E0,0E0,0E0,3.1786966E-1,1.2808108E-1,0E0,1.0379219E-1,1.5824623E-1,1.9273269E-1,0E0,3.2899674E-2,5.550968E-2,0E0,0E0,0E0,0E0,5.8910385E-2,0E0,1.8114063E-1,0E0,4.9197737E-2,1.0302907E-1,1.8255906E-1,7.195404E-2,2.8302189E-2,0E0,1.4492512E-1,1.2776071E-1,1.3401198E-1,2.9258605E-2,0E0,0E0,1.7394174E-2,0E0,0E0,4.390958E-2,3.7301842E-2,0E0,0E0,2.3501801E-1,0E0,9.11127E-2,5.402416E-2,0E0,0E0,3.9638434E-2,0E0,6.2061787E-2,2.1410324E-2,0E0,7.721692E-2,0E0,2.1937333E-2,0E0,0E0,3.1817272E-2,2.7194906E-2,2.8265774E-2,1.7941299E-1,1.4329469E-1,1.9219846E-2,0E0,3.5328925E-2,0E0,4.1837946E-2,9.3071856E-2,8.928457E-2,4.6121582E-2,0E0,0E0,3.026628E-1,0E0,1.16652995E-1,2.8318616E-2,1.0767686E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1502409E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6439435E-2,2.282834E-2,0E0,0E0,2.7371578E-2,3.0427039E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4196849E-1,2.170464E-1,0E0,0E0,0E0,0E0,0E0,1.7299362E-2,0E0,0E0,2.5137812E-2,4.1925438E-2,0E0,3.181915E-2,0E0,1.5427865E-2,5.6512907E-2,0E0,2.3892798E-2,0E0,0E0,1.5942406E-2,5.9474915E-2,1.2006509E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,55,55,56,56,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,76,76,77,77,79,79,80,80,81,81,83,83,84,84,89,89,91,91,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,105,105,108,108,109,109,112,112,114,114,115,115,118,118,120,120,121,121,123,123,125,125,128,128,129,129,130,130,131,131,132,132,133,133,135,135,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,157,157,165,165,166,166,169,169,170,170,179,179,180,180,186,186,189,189,190,190,192,192,194,194,195,195,197,197,200,200,201,201,202,202],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,-1,50,-1,52,-1,-1,-1,54,56,58,60,62,64,-1,66,68,70,-1,72,74,76,78,80,-1,82,84,-1,-1,86,88,-1,90,92,94,96,98,100,102,104,106,108,-1,-1,110,112,114,-1,-1,-1,116,118,-1,120,122,124,-1,126,128,-1,-1,-1,-1,130,-1,132,-1,134,136,138,140,142,-1,144,146,148,150,-1,-1,152,-1,-1,154,156,-1,-1,158,-1,160,162,-1,-1,164,-1,166,168,-1,170,-1,172,-1,-1,174,176,178,180,182,184,-1,186,-1,188,190,192,194,-1,-1,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,206,208,-1,-1,210,212,-1,-1,-1,-1,-1,-1,-1,-1,214,216,-1,-1,-1,-1,-1,218,-1,-1,220,222,-1,224,-1,226,228,-1,230,-1,-1,232,234,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,2.0976269E5,1E0,4.586207E0,4.97E2,3.698E3,9.616803E2,1.3390654E2,2E0,1E0,1.9791039E6,-1.8730268E-2,-8.493633E-3,1.6942337E1,1.6007428E7,9E0,2.04115E5,3.3E1,3.7990784E7,3.51E2,-1.8444842E-2,1.852364E6,1.0358873E3,7.233077E2,4.439357E8,2.2365898E3,1.4E1,-1.3432595E-2,4.306E4,1.274936E-2,1.4912975E-1,-3.4556713E-2,-1.3500018E-2,6.179515E-4,9.237895E0,1.6667E4,1.2051282E0,1.409E3,5.2E1,5.4007E7,-8.635644E-3,1.1656439E9,3.2E1,4.383864E7,4.0469444E-3,2.7578741E1,1.159499E6,3.3580637E8,5.428175E3,2.71E2,-1.4599904E-2,5.884866E7,1E0,-1.3649008E-2,-5.582979E-3,1.4E1,5.0149255E0,1.3527782E-3,7.026624E7,2.511E3,1.594E3,1.6009701E7,1.6869704E6,2.5437157E2,1.3558896E7,3.164034E0,9.1233586E4,7.4390995E6,-3.390654E-3,1.5296387E-3,3.3803125E6,2.115238E5,9.812221E9,2.4640094E-2,1.1977588E-2,-8.712043E-3,3.85374E8,2.88721E5,1.4692688E-2,2.4E1,5.2117184E4,5.595208E2,-1.3032554E-2,3.7439062E2,2.5454E4,-6.0413913E-3,-1.1566941E-3,-9.1470005E-3,-3.8590443E-3,3.429012E7,1.5727554E-3,1.318849E6,-1.8141877E-2,1.6E1,2.453E4,7.9555137E3,1.8729467E8,2.9849844E7,-7.676224E-3,7.271717E6,2.964867E4,1E0,1.09E2,-3.116995E-3,8.4780186E-4,7.4010696E0,6.0893246E-3,1.16314115E-2,2.35184E5,6.5062125E5,-7.273446E-3,1.6446128E-2,2.3385885E0,-1.2744347E-3,8.710612E1,8E0,-5.8292816E-3,9.476695E-3,1.4608225E5,-2.1458541E-4,2.416E3,7.84664E1,-6.3551646E-3,6.23E2,2.1160326E-3,1.7161617E2,5.446219E-3,1.6521169E-3,1.946094E6,7.1E1,1.9903773E2,3.90225E2,1.3881669E6,2.2190831E5,1.0411858E-3,5.3386245E0,1.01232676E-4,1.177E3,3.1226995E0,1.4195632E6,1.6E1,-3.823277E-3,4.6852478E-4,2.6210527E0,1.8690983E-3,3.6977E4,3.5814576E7,1.1860938E8,3.0704768E-4,2.0880303E-3,-2.917539E-3,3.8176242E-3,-2.184008E-4,1.246536E-3,7.871543E-3,-3.0559883E-3,2.195523E-3,2.6402525E5,1.3537777E-3,9.006914E-3,1.765918E-3,2.7478794E-3,8.435069E-3,-9.773464E-4,4.2581656E-3,3.07E2,3.89E2,3.1487294E-3,-9.838736E-4,1E0,9.24E2,2.8991008E-3,-1.0890296E-3,-4.64967E-3,-1.084472E-4,-3.0389505E-3,1.0488047E-3,-7.977407E-3,-3.3038724E-3,1.5714285E0,4.3E1,-1.2256857E-2,-1.0178325E-3,-6.5754154E-3,-2.2484954E-3,8.95076E-3,3.4444444E0,2.5366854E-3,8.639962E-3,3.92E2,1.0000181E0,9.361874E-3,2E0,-1.8858064E-3,1.2783505E0,2.0399521E2,-1.4058007E-2,3.7035952E6,1.143643E-2,-2.843269E-3,3.2892792E2,1.609E3,4.4971E4,5.483878E-3,1.3614059E-2,-6.9777896E-3,-6.994794E-4,-1.0109652E-2,-7.368951E-3,-2.4036414E-4,-4.76596E-3,-1.0060209E-2,-4.8676725E-3,1.4966434E-3,-4.370702E-3,6.086785E-3,-1.6152324E-3,1.3124244E-3,5.8338824E-3,4.190093E-3,1.1915165E-5,-3.4006445E-3,2.697135E-4,7.1319556E-3,2.7713517E-3,4.112183E-4,4.0392824E-3,2.9411428E-3,-2.9955518E-3,2.2150865E-3,5.711781E-3,2.2634848E-3,-1.1109737E-3,2.9519508E-3,8.189894E-3,5.9749857E-3,1.2125979E-2],"split_indices":[2,43,17,68,2,2,67,67,32,106,43,0,0,71,66,70,5,3,7,2,0,9,4,70,12,67,3,0,1,0,53,0,0,0,71,9,68,10,8,7,0,46,3,12,0,71,9,7,67,2,0,5,26,0,0,0,71,0,7,2,2,5,60,4,9,53,48,62,0,0,66,48,46,0,0,0,47,9,0,2,62,67,0,4,9,0,0,0,0,7,0,9,0,3,44,62,7,5,0,60,62,6,0,0,0,71,0,0,1,43,0,0,53,0,73,3,0,0,48,0,9,67,0,2,0,67,0,0,60,0,4,67,43,62,0,69,0,2,69,60,73,0,0,71,0,12,7,5,0,0,0,0,0,0,0,0,0,48,0,0,0,0,0,0,0,1,1,0,0,17,2,0,0,0,0,0,0,0,0,69,0,0,0,0,0,0,68,0,0,10,57,0,8,0,68,4,0,43,0,0,4,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.59E2,2.59E2,2.56E2,5.03E2,1.2E1,2.47E2,2.3E2,2.6E1,7.9E1,4.24E2,4E0,8E0,9E1,1.57E2,1.26E2,1.04E2,9E0,1.7E1,7.4E1,5E0,2.77E2,1.47E2,5.8E1,3.2E1,1E2,5.7E1,9E0,1.17E2,5E0,9.9E1,5E0,4E0,4E0,1.3E1,3.3E1,4.1E1,1.54E2,1.23E2,1.43E2,4E0,2.9E1,2.9E1,2.4E1,8E0,7.9E1,2.1E1,2.8E1,2.9E1,1.12E2,5E0,5.7E1,4.2E1,6E0,7E0,1.4E1,1.9E1,8E0,3.3E1,1.14E2,4E1,1.05E2,1.8E1,7.2E1,7.1E1,1.1E1,1.8E1,2.5E1,4E0,8E0,1.6E1,5.7E1,2.2E1,1.2E1,9E0,5E0,2.3E1,2.5E1,4E0,8.5E1,2.7E1,3.9E1,1.8E1,1.5E1,2.7E1,7E0,7E0,1.5E1,4E0,2.8E1,5E0,1.09E2,5E0,1.6E1,2.4E1,5.1E1,5.4E1,1.3E1,5E0,2.9E1,4.3E1,6.3E1,8E0,7E0,4E0,8E0,1E1,1.6E1,9E0,1E1,6E0,2.5E1,3.2E1,4E0,1.8E1,1.8E1,5E0,1.2E1,1.3E1,4E0,8.1E1,9E0,1.8E1,3.3E1,6E0,1E1,5E0,4E0,2.3E1,1.3E1,1.5E1,9.9E1,1E1,1.1E1,5E0,1.8E1,6E0,1.1E1,4E1,3.7E1,1.7E1,4E0,9E0,1.6E1,1.3E1,2.7E1,1.6E1,5.7E1,6E0,4E0,4E0,4E0,4E0,4E0,5E0,6E0,4E0,2.6E1,6E0,1.2E1,6E0,7E0,1.1E1,4E0,9E0,1.5E1,6.6E1,4E0,5E0,1.4E1,1.9E1,5E0,5E0,1.9E1,4E0,8E0,5E0,7E0,8E0,6.3E1,3.6E1,6E0,4E0,4E0,7E0,1E1,8E0,6E0,5E0,1.2E1,2.8E1,7E0,3E1,6E0,1.1E1,1.2E1,4E0,2.3E1,4E0,5E0,1.1E1,2.3E1,3.4E1,7E0,1.9E1,9E0,6E0,2.5E1,4.1E1,4E0,1E1,1.2E1,7E0,1E1,5.3E1,1E1,2.6E1,4E0,4E0,7E0,5E0,1.8E1,1E1,4E0,2.6E1,4E0,7E0,5E0,7E0,1.7E1,6E0,7E0,4E0,1.6E1,7E0,1.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[4.803224E-4,-6.4988546E-2,1.21288955E-1,-1.1059974E-1,1.0745003E-2,-1.976288E-1,1.3855101E-1,-1.4435022E-1,-3.824352E-2,-1.4367324E-1,2.1822821E-2,-1.8122297E-2,-1.1957463E-1,9.712047E-2,2.5266978E-1,-1.3449155E-1,-4.3064338E-1,9.492502E-2,-5.5827163E-2,-4.490818E-2,-1.7173467E-2,7.91889E-3,9.406123E-2,-3.2540635E-4,-8.953963E-3,7.566279E-2,2.1347274E-1,2.8531857E-2,2.1964534E-1,-1.277048E-1,-2.202736E-2,-6.1311997E-3,-2.968037E-2,-5.8277004E-4,6.6967052E-3,-4.308805E-2,-2.0673989E-1,2.084897E-3,-5.9379395E-3,-4.7458272E-2,2.214873E-2,2.39022E-2,2.0004493E-1,9.030686E-2,-1.4208077E-1,4.2823735E-3,2.4661657E-1,2.5455597E-1,6.712854E-3,-1.7614657E-1,-9.713884E-2,-1.0351726E-1,-4.804799E-3,-1.4698131E-2,-2.4940043E-3,2.204983E-2,-9.282545E-2,1.5735382E-1,1.1036502E-2,6.765682E-2,-7.202281E-3,5.1831864E-3,1.0752917E-2,9.983039E-2,-5.824326E-3,-1.7747069E-2,-6.841284E-3,6.5612113E-3,2.7227536E-1,3.7616786E-1,2.0156719E-1,5.2179517E-3,-4.6420004E-3,-7.661795E-2,-2.0191501E-1,1.7577179E-1,-1.1540851E-1,1.4363092E-4,-1.17628835E-1,4.173631E-3,-2.0582218E-2,7.042988E-2,-1.3296179E-3,-1.3204718E-1,2.466316E-3,2.3801855E-3,9.416779E-3,3.913172E-2,-2.7490882E-2,1.97283E-2,6.333131E-3,5.607387E-2,1.4529963E-1,3.4835741E-3,-3.4297463E-3,1.3654399E-2,7.255779E-3,2.1689147E-2,1.1488178E-2,1.5455331E-1,2.8439677E-1,-5.7331333E-3,-2.4864047E-3,-2.1296485E-1,-4.0944275E-3,3.2724605E-3,1.0423297E-2,-9.6584104E-2,-2.0386173E-1,-9.7910434E-2,-1.0542079E-2,-6.9274396E-2,3.6720706E-3,5.900914E-3,-4.408583E-5,-1.0289645E-3,-1.6715394E-1,6.889242E-3,7.121398E-2,-4.984149E-2,4.7353823E-2,3.5678826E-3,-1.1948405E-3,-2.1087103E-2,7.6592214E-2,2.6549757E-1,8.5739076E-2,1.0424673E-2,7.90671E-2,1.5542647E-2,7.141104E-3,1.5845262E-3,-2.1311706E-3,-1.8088901E-1,-2.51622E-1,-8.0336615E-2,-1.1105385E-2,-3.5130077E-3,-2.2945558E-1,-7.057203E-2,-6.3128164E-3,1.1483617E-3,-4.359377E-3,3.632732E-2,-1.8513006E-3,-1.3865606E-3,-2.1429503E-1,4.0211953E-2,-3.510861E-2,7.59381E-3,5.463084E-2,-1.04452804E-1,-2.2676118E-2,-1.439573E-2,8.026256E-3,1.920989E-2,-6.0866047E-3,8.942626E-2,-3.2350274E-3,1.6472772E-2,9.477966E-3,-2.5429567E-2,1.0178783E-1,-4.0402506E-3,1.4478526E-1,-9.441583E-3,-3.7248184E-3,-1.8939136E-2,-9.839627E-3,-5.2954275E-3,3.3181882E-4,-1.1807099E-2,-4.137211E-3,-2.1972875E-3,-5.7290085E-3,4.3375455E-3,-5.3100684E-4,-1.5052858E-2,-4.805458E-3,-1.6188619E-3,2.5970554E-3,1.2557821E-4,-6.8897875E-3,2.012465E-3,5.4968237E-3,-2.4611356E-3,-6.8824627E-3,-3.7292093E-3,3.9702593E-4,2.7175676E-3,-3.9368607E-3,-2.8369667E-3,2.6757447E-3,5.9484793E-3,1.1066947E-3,-5.294183E-3,3.1795034E-3,9.655174E-3,2.7038043E-3,8.366757E-3,3.057394E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,-1,47,49,-1,-1,-1,-1,-1,51,53,-1,-1,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,-1,-1,81,83,85,87,89,-1,-1,-1,91,-1,-1,93,-1,95,97,99,-1,-1,101,103,105,107,-1,109,-1,111,113,-1,115,-1,-1,-1,117,119,121,-1,123,125,-1,-1,-1,-1,-1,-1,127,129,-1,131,133,-1,-1,-1,135,137,139,-1,141,143,-1,-1,-1,145,147,149,151,153,-1,-1,155,157,159,161,-1,163,-1,-1,-1,-1,165,167,169,-1,-1,171,173,-1,-1,-1,175,-1,-1,177,179,181,-1,183,185,187,189,-1,191,-1,193,-1,-1,-1,195,197,-1,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.146455E0,2.3086586E0,2.020256E0,1.0128174E0,4.3421087E-1,2.4174702E-1,1.6018939E0,7.4366236E-1,3.1907216E-1,3.6621127E-1,2.3629305E-1,0E0,1.23946235E-1,6.1799717E-1,9.610038E-1,5.802903E-1,4.826603E-1,8.790864E-2,2.1986881E-1,1.0367721E-1,0E0,1.5786476E-1,2.7777562E-1,0E0,0E0,7.015799E-1,1.3326001E-1,0E0,6.3014793E-1,3.8563108E-1,0E0,0E0,0E0,0E0,0E0,2.5629294E-1,1.3598934E-1,0E0,0E0,1.3133563E-1,2.3714969E-1,1.9935629E-1,1.698786E-2,4.3090224E-1,4.5568228E-1,0E0,4.184544E-2,3.9459372E-1,1.5938137E-1,2.5125837E-1,8.6291754E-1,6.506628E-2,1.0543821E-1,0E0,0E0,4.383269E-2,1.5041453E-1,4.85242E-2,1.6127618E-1,6.3846014E-2,0E0,0E0,0E0,3.798195E-1,0E0,0E0,6.1028562E-2,0E0,1.527667E-2,1.3308072E-1,1.6243505E-1,0E0,0E0,8.064334E-2,8.377552E-2,4.155594E-2,2.4505806E-1,0E0,6.0500085E-2,0E0,7.111403E-2,3.6933497E-2,0E0,7.335377E-2,0E0,0E0,0E0,8.8468775E-2,1.0774812E-1,3.6076866E-2,0E0,1.6151989E-1,6.5734696E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7535573E-1,6.958568E-2,0E0,1.7636184E-2,4.624152E-2,0E0,0E0,0E0,2.9142213E-1,7.088065E-2,2.6359677E-2,0E0,4.9573705E-2,6.0023487E-2,0E0,0E0,0E0,9.054324E-2,6.2989004E-2,5.744861E-2,7.032472E-2,1.19558014E-1,0E0,0E0,1.0063494E-1,1.5679595E-1,9.719443E-2,1.1855608E-1,0E0,2.2058652E-1,0E0,0E0,0E0,0E0,8.710039E-2,1.02329135E-1,3.6389846E-1,0E0,0E0,5.7490945E-2,2.1152787E-2,0E0,0E0,0E0,6.714228E-2,0E0,0E0,1.0140324E-1,3.2075748E-2,8.666871E-2,0E0,2.188699E-2,2.8521672E-2,6.2405165E-2,6.2051054E-2,0E0,5.6605503E-2,0E0,1.8747121E-1,0E0,0E0,0E0,8.352239E-2,2.5212848E-1,0E0,2.798155E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,28,28,29,29,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,59,59,63,63,66,66,68,68,69,69,70,70,73,73,74,74,75,75,76,76,78,78,80,80,81,81,83,83,87,87,88,88,89,89,91,91,92,92,99,99,100,100,102,102,103,103,107,107,108,108,109,109,111,111,112,112,116,116,117,117,118,118,119,119,120,120,123,123,124,124,125,125,126,126,128,128,133,133,134,134,135,135,138,138,139,139,143,143,146,146,147,147,148,148,150,150,151,151,152,152,153,153,155,155,157,157,161,161,162,162,164,164],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,-1,48,50,-1,-1,-1,-1,-1,52,54,-1,-1,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,-1,-1,82,84,86,88,90,-1,-1,-1,92,-1,-1,94,-1,96,98,100,-1,-1,102,104,106,108,-1,110,-1,112,114,-1,116,-1,-1,-1,118,120,122,-1,124,126,-1,-1,-1,-1,-1,-1,128,130,-1,132,134,-1,-1,-1,136,138,140,-1,142,144,-1,-1,-1,146,148,150,152,154,-1,-1,156,158,160,162,-1,164,-1,-1,-1,-1,166,168,170,-1,-1,172,174,-1,-1,-1,176,-1,-1,178,180,182,-1,184,186,188,190,-1,192,-1,194,-1,-1,-1,196,198,-1,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,7.785302E5,1E0,1.380863E6,5.862757E1,7.937523E-3,1.5142261E3,1.5126086E2,7E0,3.4E1,4.9321495E6,-1.8122297E-2,1E0,4.391553E6,3.3817584E7,4.8E1,4.68418E5,1.4979E4,1.2906634E7,1.3881669E6,-1.7173467E-2,2.3043478E2,3.741267E7,-3.2540635E-4,-8.953963E-3,1.7493458E7,4.4631204E2,2.8531857E-2,2.5130852E2,2.71E2,-2.202736E-2,-6.1311997E-3,-2.968037E-2,-5.8277004E-4,6.6967052E-3,1.5292561E2,2.956111E2,2.084897E-3,-5.9379395E-3,1.422162E3,2.1188422E2,1.7544615E0,2.9565217E0,1.6254545E1,3.167E3,4.2823735E-3,1.7068776E10,1.5277778E0,6.6477684E7,1E0,1.1E1,1.1010101E0,7.696402E3,-1.4698131E-2,-2.4940043E-3,1.4735735E2,1.605076E6,2.4568965E2,3.373913E0,3.074398E0,-7.202281E-3,5.1831864E-3,1.0752917E-2,6.9664386E2,-5.824326E-3,-1.7747069E-2,1.5043378E0,6.5612113E-3,2.5400869E5,8.053062E8,3.5675005E3,5.2179517E-3,-4.6420004E-3,3.9125E1,3.3502305E0,1.07882355E2,1.3922E4,1.4363092E-4,8.0509944E5,4.173631E-3,2.87834E5,3.9152692E6,-1.3296179E-3,1.1481482E0,2.466316E-3,2.3801855E-3,9.416779E-3,1.196966E6,6.2136955E0,5.51E2,6.333131E-3,1.6525911E9,1.0717949E1,3.4835741E-3,-3.4297463E-3,1.3654399E-2,7.255779E-3,2.1689147E-2,1.1488178E-2,2.7578741E1,3.2295492E0,-5.7331333E-3,6.44E2,2.4636364E1,-4.0944275E-3,3.2724605E-3,1.0423297E-2,2.1609572E7,1.9470909E2,3.0588236E0,-1.0542079E-2,2.94E2,1.474E3,5.900914E-3,-4.408583E-5,-1.0289645E-3,4.1E1,3.9914E4,1.552356E0,3.0291306E2,6.887749E7,3.5678826E-3,-1.1948405E-3,1.8578552E7,1E0,1.6521566E5,1.9E1,1.0424673E-2,2.453E4,1.5542647E-2,7.141104E-3,1.5845262E-3,-2.1311706E-3,6.1E1,2E0,3.35E2,-1.1105385E-2,-3.5130077E-3,1.96E2,4.3992E4,-6.3128164E-3,1.1483617E-3,-4.359377E-3,3.3924444E5,-1.8513006E-3,-1.3865606E-3,1.1E1,1.2E1,8.69E2,7.59381E-3,1.574E3,2.402121E4,4.8927066E4,1.43E2,8.026256E-3,1.5204346E8,-6.0866047E-3,1.3864E4,-3.2350274E-3,1.6472772E-2,9.477966E-3,1.2E2,1.1662862E8,-4.0402506E-3,1.2820834E7,-9.441583E-3,-3.7248184E-3,-1.8939136E-2,-9.839627E-3,-5.2954275E-3,3.3181882E-4,-1.1807099E-2,-4.137211E-3,-2.1972875E-3,-5.7290085E-3,4.3375455E-3,-5.3100684E-4,-1.5052858E-2,-4.805458E-3,-1.6188619E-3,2.5970554E-3,1.2557821E-4,-6.8897875E-3,2.012465E-3,5.4968237E-3,-2.4611356E-3,-6.8824627E-3,-3.7292093E-3,3.9702593E-4,2.7175676E-3,-3.9368607E-3,-2.8369667E-3,2.6757447E-3,5.9484793E-3,1.1066947E-3,-5.294183E-3,3.1795034E-3,9.655174E-3,2.7038043E-3,8.366757E-3,3.057394E-3],"split_indices":[2,43,17,9,62,53,67,61,3,3,43,0,15,43,7,3,9,1,12,43,0,4,7,0,0,66,4,0,73,2,0,0,0,0,0,67,4,0,0,48,67,68,73,69,2,0,46,68,59,20,10,68,62,0,0,67,12,70,69,71,0,0,0,67,0,0,69,0,48,7,67,0,0,4,68,67,44,0,60,0,12,43,0,68,0,0,0,9,69,2,0,46,71,0,0,0,0,0,0,71,53,0,1,67,0,0,0,60,4,68,0,44,10,0,0,0,10,9,69,67,7,0,0,5,6,48,73,0,44,0,0,0,0,2,10,67,0,0,8,1,0,0,0,43,0,0,3,3,2,0,2,48,62,0,0,7,0,10,0,0,0,0,12,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,6.67E2,3.61E2,4.16E2,2.51E2,1.8E1,3.43E2,2.83E2,1.33E2,1.6E1,2.35E2,4E0,1.4E1,2.53E2,9E1,2.75E2,8E0,1.5E1,1.18E2,1.2E1,4E0,1.98E2,3.7E1,6E0,8E0,2.15E2,3.8E1,6E0,8.4E1,2.71E2,4E0,4E0,4E0,5E0,1E1,1.1E2,8E0,6E0,6E0,4E1,1.58E2,2.3E1,1.4E1,2.02E2,1.3E1,9E0,2.9E1,7.2E1,1.2E1,1.03E2,1.68E2,4.2E1,6.8E1,4E0,4E0,1.6E1,2.4E1,1.1E1,1.47E2,1.9E1,4E0,5E0,9E0,1.94E2,8E0,4E0,9E0,7E0,2.2E1,2E1,5.2E1,6E0,6E0,2.2E1,8.1E1,1E1,1.58E2,5E0,3.7E1,9E0,5.9E1,8E0,8E0,1.9E1,5E0,4E0,7E0,8.5E1,6.2E1,1.2E1,7E0,1E2,9.4E1,4E0,5E0,1.7E1,5E0,1E1,1E1,3.5E1,1.7E1,1.3E1,9E0,7.3E1,8E0,4E0,6E0,1.32E2,2.6E1,3.3E1,4E0,1.9E1,4E1,4E0,4E0,5E0,1.4E1,4.3E1,4.2E1,4.8E1,1.4E1,5E0,7E0,2.1E1,7.9E1,3E1,6.4E1,1.7E1,1.8E1,1.1E1,6E0,5E0,4E0,4.3E1,3E1,1.2E2,1.2E1,5E0,2.1E1,2.1E1,1.2E1,4E0,1.5E1,2.3E1,1.7E1,4E0,1E1,2.4E1,1.9E1,5E0,3.7E1,1.5E1,3.3E1,1E1,4E0,1.6E1,5E0,7.3E1,6E0,1E1,2E1,8E0,5.6E1,5E0,1.3E1,3.4E1,9E0,4E0,2.6E1,8.6E1,3.4E1,1.7E1,4E0,1.6E1,5E0,1E1,1.3E1,4E0,6E0,4E0,2E1,1.5E1,4E0,3.3E1,4E0,8E0,7E0,1.1E1,2.2E1,5E0,5E0,5E0,1.1E1,4.5E1,2.8E1,4E0,4E0,1.5E1,4.1E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[-4.8119277E-3,-8.19449E-2,6.674725E-2,-6.0867727E-2,-2.3381202E-1,2.2733299E-2,1.7460561E-1,-1.2819494E-1,4.277983E-3,-1.6598009E-1,-2.7359715E-2,3.1408943E-2,-1.7334734E-1,1.8926386E-2,1.5694648E-1,-2.1095286E-1,-1.0319144E-1,1.5616835E-2,-1.4997721E-2,-1.1989513E-2,-1.11577906E-1,-1.2657218E-1,3.9631903E-2,-2.2440404E-1,-7.457351E-4,1.2680195E-1,2.6176533E-1,-1.866383E-1,-1.768178E-2,-9.140318E-2,-1.4859333E-2,4.396741E-3,-1.7916892E-1,-5.5355404E-2,-1.9814077E-1,-6.1191046E-3,-1.6930033E-2,1.6500445E-2,7.585727E-2,-5.640702E-3,-1.2937548E-2,6.5104023E-4,1.4319506E-1,6.5657133E-3,2.907409E-1,-1.1193723E-2,-1.2528831E-1,-7.586996E-2,-1.7860232E-1,-4.750737E-3,1.5629835E-2,-2.0091496E-2,-9.818152E-2,-9.136819E-2,3.94363E-3,-5.0382144E-3,-1.1096451E-2,-3.6236395E-3,2.2583473E-3,4.433173E-2,-6.4346627E-3,8.827237E-2,-7.4022366E-3,8.998234E-2,-6.165467E-3,1.2226552E-1,1.475295E-2,1.4521345E-2,6.388002E-3,-1.7084765E-3,-1.6992027E-1,4.4337574E-2,-8.4808104E-2,-1.2674099E-1,-1.5705595E-2,-6.724795E-2,2.1932188E-2,-7.828827E-3,-3.741805E-2,-1.2933938E-1,1.5349538E-3,2.5929652E-2,1.1204304E-1,-3.1998172E-2,3.6123767E-2,6.0704805E-2,1.4671421E-1,1.4191905E-4,7.3408578E-3,1.4411539E-1,1.8676901E-2,-3.9969385E-3,-9.377069E-3,-1.6101873E-3,4.755746E-3,-1.0512735E-1,-4.3142404E-2,-1.4917591E-1,-1.5748345E-3,-8.000582E-3,-4.8192218E-2,-2.1289203E-2,5.2353542E-2,-4.263576E-3,3.28917E-4,-2.9202914E-3,-7.90162E-3,3.888936E-2,-7.663315E-3,3.050958E-2,8.130047E-3,4.236355E-3,-1.1095308E-1,-2.3079947E-2,8.204179E-2,1.319442E-2,1.1738741E-1,2.2552916E-1,1.04881756E-1,1.1253367E-1,2.1508346E-1,-2.669938E-3,7.798438E-2,-6.846816E-2,-1.3848919E-1,6.092308E-2,-6.866946E-2,-8.476732E-3,-3.8694555E-3,-2.4941197E-2,-1.24132805E-1,-9.40167E-2,2.4256876E-2,7.8029774E-2,-1.8609444E-2,2.3689168E-2,8.667296E-2,-9.649653E-4,3.1533272E-3,-1.5281502E-2,4.482122E-3,-1.7335914E-1,1.2810153E-3,-4.263336E-3,2.1281246E-2,4.1914728E-2,8.280619E-3,3.435392E-2,-5.482454E-3,1.6697888E-1,4.7498997E-2,5.488922E-3,1.2000602E-2,3.7214328E-2,1.4040731E-1,1.7796963E-1,5.8243457E-2,1.3032056E-2,1.4699897E-1,6.570476E-3,5.292767E-4,-1.0127713E-2,-2.1635348E-3,-6.853345E-3,-1.5673038E-3,4.7726654E-3,2.9359382E-4,6.57511E-5,-4.13437E-3,-2.3574363E-3,1.3467293E-3,-1.02446E-3,-8.280099E-3,-5.945158E-3,1.9709137E-3,5.0180973E-3,-1.858477E-3,2.5033061E-3,6.4490414E-3,-2.0649957E-3,3.4100832E-3,2.2316587E-3,-5.1489077E-4,5.842822E-3,5.429285E-4,-1.4401516E-3,4.0613585E-3,-3.6001569E-3,-1.3163268E-2,3.9859163E-3,-1.3917804E-3,2.867904E-3,-7.010903E-5,-1.741727E-3,3.0876757E-3,9.491881E-3,5.2937944E-3,-2.8379129E-3,3.8349226E-3,-1.7605657E-3,3.9592516E-3,7.992866E-3,9.720368E-4,1.0105526E-2,3.1918418E-3,-1.1080394E-3,3.8449534E-3,9.017076E-3,2.6915255E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,-1,31,-1,33,35,37,39,-1,41,43,45,-1,47,-1,49,51,53,55,57,-1,59,61,-1,-1,63,65,-1,67,-1,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,81,83,85,-1,87,-1,89,-1,-1,-1,-1,91,93,95,97,-1,99,101,-1,103,105,-1,107,109,111,113,115,117,-1,-1,119,121,-1,-1,-1,-1,123,125,127,-1,-1,129,131,133,-1,-1,-1,-1,135,-1,137,-1,139,141,143,145,147,149,151,153,155,157,-1,159,161,163,165,167,-1,-1,169,171,173,175,177,179,181,183,-1,-1,185,-1,187,-1,-1,189,191,-1,193,-1,195,197,-1,-1,199,201,203,205,-1,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5857563E0,1.5446205E0,2.488349E0,1.8821764E0,1.3360584E0,6.416659E-1,5.6666136E-1,4.1484094E-1,1.413979E0,2.3056722E-1,0E0,4.7138312E-1,1.24800056E-1,0E0,4.186468E-1,1.2904906E-1,3.9291716E-1,0E0,6.630623E-1,0E0,1.5450239E-1,5.178684E-1,2.8532565E-1,3.4111798E-2,0E0,2.3359144E-1,6.615305E-2,1.1987877E-1,0E0,1.9910324E-1,0E0,5.7280076E-1,4.1292566E-1,1.1928689E-1,2.2502065E-2,5.554225E-2,0E0,1.343311E-1,4.0396422E-1,0E0,0E0,1.7972594E-1,3.2960844E-1,0E0,4.2825222E-2,0E0,8.436707E-2,1.4880842E-1,1.4937681E-1,3.0664724E-1,0E0,0E0,7.1617976E-2,8.945426E-2,0E0,0E0,0E0,0E0,0E0,1.154675E-1,1.2735727E-1,1.9570339E-1,0E0,5.3415954E-2,0E0,2.0327759E-1,0E0,0E0,0E0,0E0,2.378866E-2,4.9995825E-2,1.0213816E-1,3.4409463E-2,0E0,1.0267411E-1,1.7073062E-1,0E0,2.7538143E-2,2.7789697E-2,0E0,1.9536744E-1,9.8888785E-2,2.093153E-1,1.2150829E-1,2.3543623E-1,1.08227015E-1,0E0,0E0,1.4389288E-1,8.160494E-2,0E0,0E0,0E0,0E0,9.325147E-2,1.1841242E-1,1.8297434E-2,0E0,0E0,8.16574E-2,1.8115298E-1,1.3992882E-1,0E0,0E0,0E0,0E0,5.0241813E-2,0E0,2.082568E-2,0E0,9.383805E-2,2.0386156E-1,6.336149E-2,8.9992106E-2,1.4079614E-1,1.3070345E-1,1.662612E-2,6.2084675E-2,1.8147123E-1,5.8892608E-2,0E0,4.110597E-2,1.2673208E-1,3.927654E-2,2.015857E-2,5.1206112E-2,0E0,0E0,5.5108212E-2,5.7815403E-2,1.0499832E-1,1.8933123E-1,7.464054E-2,5.393332E-2,4.8557434E-2,4.9202308E-2,0E0,0E0,7.325361E-2,0E0,1.4814809E-1,0E0,0E0,4.603523E-2,1.6732477E-2,0E0,1.03154555E-1,0E0,2.2823393E-2,7.472238E-2,0E0,0E0,4.3438476E-2,6.640881E-2,9.1423035E-2,6.56774E-2,0E0,4.4210047E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,44,44,46,46,47,47,48,48,49,49,52,52,53,53,59,59,60,60,61,61,63,63,65,65,70,70,71,71,72,72,73,73,75,75,76,76,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,89,89,90,90,95,95,96,96,97,97,100,100,101,101,102,102,107,107,109,109,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,125,125,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,141,141,144,144,145,145,147,147,149,149,150,150,153,153,154,154,155,155,156,156,158,158],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,-1,32,-1,34,36,38,40,-1,42,44,46,-1,48,-1,50,52,54,56,58,-1,60,62,-1,-1,64,66,-1,68,-1,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,82,84,86,-1,88,-1,90,-1,-1,-1,-1,92,94,96,98,-1,100,102,-1,104,106,-1,108,110,112,114,116,118,-1,-1,120,122,-1,-1,-1,-1,124,126,128,-1,-1,130,132,134,-1,-1,-1,-1,136,-1,138,-1,140,142,144,146,148,150,152,154,156,158,-1,160,162,164,166,168,-1,-1,170,172,174,176,178,180,182,184,-1,-1,186,-1,188,-1,-1,190,192,-1,194,-1,196,198,-1,-1,200,202,204,206,-1,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.072304E5,1E0,3.921E3,6.83E2,9.639872E0,2.8088E1,7.9E1,3.89E2,2.00087E5,5E0,-2.7359715E-2,1.45064E5,1.1997242E0,1.8926386E-2,5.1167645E6,6.737403E7,1.2218E4,1.5616835E-2,1.1290322E1,-1.1989513E-2,1.8414415E1,1.1015E4,1E0,1.8E1,-7.457351E-4,1E0,8.0605554E2,3.6E1,-1.768178E-2,1.4E1,-1.4859333E-2,5.428175E3,1.5E1,1E0,1.25E0,2.0467092E6,-1.6930033E-2,1.5723623E0,3.174172E5,-5.640702E-3,-1.2937548E-2,2.4661344E5,4.43837E0,6.5657133E-3,1.6E-3,-1.1193723E-2,2.7444445E1,8.414097E-1,2E1,1.8988005E5,1.5629835E-2,-2.0091496E-2,6.593462E4,1.3E1,3.94363E-3,-5.0382144E-3,-1.1096451E-2,-3.6236395E-3,2.2583473E-3,1.047E3,1.2839025E1,2.308821E6,-7.4022366E-3,1.3388E4,-6.165467E-3,1.4078995E2,1.475295E-2,1.4521345E-2,6.388002E-3,-1.7084765E-3,5.3E1,3.56E2,1.3823239E2,7.997723E6,-1.5705595E-2,8.01E2,4.220534E5,-7.828827E-3,4.727453E7,1.3439851E7,1.5349538E-3,1.3062992E1,1.411E3,8E0,1.6802971E6,7.5491136E-1,7E0,1.4191905E-4,7.3408578E-3,1.6466942E3,3.4915986E3,-3.9969385E-3,-9.377069E-3,-1.6101873E-3,4.755746E-3,1.257E4,5.8114325E5,6.946813E7,-1.5748345E-3,-8.000582E-3,3.154992E7,1.159499E6,2.847E4,-4.263576E-3,3.28917E-4,-2.9202914E-3,-7.90162E-3,3.0291306E2,-7.663315E-3,1.1E1,8.130047E-3,4.39776E6,2.84E2,3.8343322E6,1.670046E1,4.76E2,3.8275862E0,2.379747E0,4.569524E2,4.320314E8,1.0851E4,-2.669938E-3,5.43E2,3.964269E3,2.3E1,1.4865319E5,1.1146532E8,-8.476732E-3,-3.8694555E-3,2.1340163E8,1.670046E1,5.821732E2,2.232008E-2,1.01573E5,8.076923E0,2.4915715E-3,1.500502E6,-9.649653E-4,3.1533272E-3,1.574E3,4.482122E-3,2.9077E4,1.2810153E-3,-4.263336E-3,1.1643481E6,4.646154E0,8.280619E-3,1.158E3,-5.482454E-3,9E0,4.857143E0,5.488922E-3,1.2000602E-2,4.0705118E6,3.43E2,1.3240695E3,7.618E3,1.3032056E-2,1.3091103E0,6.570476E-3,5.292767E-4,-1.0127713E-2,-2.1635348E-3,-6.853345E-3,-1.5673038E-3,4.7726654E-3,2.9359382E-4,6.57511E-5,-4.13437E-3,-2.3574363E-3,1.3467293E-3,-1.02446E-3,-8.280099E-3,-5.945158E-3,1.9709137E-3,5.0180973E-3,-1.858477E-3,2.5033061E-3,6.4490414E-3,-2.0649957E-3,3.4100832E-3,2.2316587E-3,-5.1489077E-4,5.842822E-3,5.429285E-4,-1.4401516E-3,4.0613585E-3,-3.6001569E-3,-1.3163268E-2,3.9859163E-3,-1.3917804E-3,2.867904E-3,-7.010903E-5,-1.741727E-3,3.0876757E-3,9.491881E-3,5.2937944E-3,-2.8379129E-3,3.8349226E-3,-1.7605657E-3,3.9592516E-3,7.992866E-3,9.720368E-4,1.0105526E-2,3.1918418E-3,-1.1080394E-3,3.8449534E-3,9.017076E-3,2.6915255E-3],"split_indices":[43,6,2,2,69,61,0,1,5,6,0,7,53,0,43,7,44,0,69,0,73,9,27,3,0,27,4,2,0,8,0,67,8,16,68,43,0,68,48,0,0,48,50,0,72,0,4,68,3,43,0,0,43,8,0,0,0,0,0,2,73,43,0,2,0,73,0,0,0,0,44,2,67,9,0,2,66,0,7,60,0,71,44,8,43,53,3,0,0,67,4,0,0,0,0,12,60,7,0,0,12,9,10,0,0,0,0,67,0,3,0,9,0,60,71,0,69,69,67,5,2,0,8,43,3,43,46,0,0,7,71,4,53,1,69,53,9,0,0,2,0,1,0,0,43,71,0,2,0,8,69,0,0,43,0,67,10,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,4.86E2,5.24E2,4.28E2,5.8E1,3.73E2,1.51E2,2.1E2,2.18E2,5E1,8E0,3.58E2,1.5E1,9E0,1.42E2,4.7E1,1.63E2,1.1E1,2.07E2,1.7E1,3.3E1,1.7E1,3.41E2,1.1E1,4E0,1.12E2,3E1,4.3E1,4E0,1.56E2,7E0,1.86E2,2.1E1,2.1E1,1.2E1,1.2E1,5E0,2.09E2,1.32E2,5E0,6E0,1.3E1,9.9E1,7E0,2.3E1,2.1E1,2.2E1,1.34E2,2.2E1,1.82E2,4E0,4E0,1.7E1,1.7E1,4E0,5E0,7E0,5E0,7E0,9.4E1,1.15E2,1.26E2,6E0,8E0,5E0,9E1,9E0,1.9E1,4E0,8E0,1.4E1,9E0,1.25E2,1.8E1,4E0,5.4E1,1.28E2,7E0,1E1,1.3E1,4E0,7.5E1,1.9E1,7.2E1,4.3E1,8.7E1,3.9E1,4E0,4E0,7.4E1,1.6E1,5E0,9E0,4E0,5E0,8.3E1,4.2E1,1.4E1,4E0,7E0,4.7E1,5.3E1,7.5E1,4E0,6E0,6E0,7E0,7.1E1,4E0,9E0,1E1,5E1,2.2E1,1.9E1,2.4E1,4.8E1,3.9E1,1.2E1,2.7E1,5.3E1,2.1E1,7E0,9E0,4.1E1,4.2E1,8E0,3.4E1,8E0,6E0,3.7E1,1E1,2E1,3.3E1,5.5E1,2E1,5.5E1,1.6E1,4E0,5E0,4.2E1,8E0,1.5E1,7E0,7E0,1.2E1,1.8E1,6E0,4.2E1,6E0,2.2E1,1.7E1,4E0,8E0,1E1,1.7E1,2.3E1,3E1,9E0,1.2E1,4E0,5E0,4E0,3.7E1,3.8E1,4E0,4E0,4E0,8E0,2.6E1,2.5E1,1.2E1,4E0,6E0,1.6E1,4E0,1.4E1,1.9E1,4.1E1,1.4E1,1.6E1,4E0,3.2E1,2.3E1,1E1,6E0,3.7E1,5E0,9E0,6E0,5E0,7E0,1.2E1,6E0,1.3E1,2.9E1,1.1E1,1.1E1,4E0,1.3E1,4E0,6E0,1.3E1,4E0,1.6E1,7E0,7E0,2.3E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"209","size_leaf_vector":"1"}},{"base_weights":[-5.0273514E-4,-1.2543014E-1,4.433076E-2,-1.0155056E-1,-2.711262E-1,1.177078E-2,2.009337E-1,-1.5354086E-1,-3.6586028E-2,-4.6052542E-1,-1.8392186E-1,-3.558003E-2,4.612676E-2,2.3762597E-2,1.7290747E-1,-1.427859E-1,-1.79971E-2,2.6996106E-1,-8.72707E-2,-6.3834502E-3,-2.8873853E-2,-2.0180507E-1,-2.9971502E-3,-2.6035894E-2,-1.488079E-1,3.1877626E-2,1.5861045E-1,2.0439933E-1,-3.5970823E-3,-1.948772E-1,-1.0245779E-1,5.1168823E-3,1.6806778E-2,-5.2321207E-2,-2.1511768E-1,-4.169671E-3,-2.3586677E-1,-9.047526E-3,-8.3322965E-2,-7.8002766E-2,-1.243493E-2,1.1171669E-1,1.732415E-2,-1.1329408E-3,1.796398E-1,3.4045517E-1,1.64429E-1,5.8520515E-2,-6.834624E-3,-2.168251E-1,-2.0474137E-3,-8.677662E-2,-1.3639349E-2,-8.476217E-2,3.0500285E-2,-1.1320735E-2,-3.8030786E-3,-1.2915283E-2,-6.1682626E-3,-3.2096844E-2,1.8989265E-2,-1.1183121E-2,-6.612635E-2,-5.1507945E-3,2.1135827E-4,7.6993674E-2,9.329905E-3,-9.390459E-2,2.8018365E-2,1.2619777E-2,1.4764638E-1,6.0773E-3,1.7535836E-2,-3.4592901E-3,1.8138115E-1,3.177586E-4,5.0440314E-3,-1.8143508E-1,-2.7184168E-1,-4.1874185E-2,-1.2552673E-1,-1.4925282E-1,-4.6230275E-2,6.256472E-2,-3.9093117E-3,1.5024849E-2,-6.891583E-2,8.406424E-3,8.694847E-3,1.886704E-2,-9.5699534E-2,9.940017E-2,1.499649E-2,3.931263E-2,-2.54917E-1,3.2674097E-2,-9.46087E-3,9.450849E-2,2.1504937E-1,1.3978823E-1,2.4620543E-1,-9.693711E-2,-9.545233E-3,-1.5372083E-2,-7.8240605E-3,3.4368723E-3,-7.603807E-2,-1.5857081E-1,-5.4866362E-2,-2.7530256E-3,-7.81642E-3,-6.396603E-2,3.207615E-3,5.728842E-4,4.4554253E-3,-2.3151629E-2,5.7724264E-2,-1.1677748E-2,-4.284088E-2,7.073717E-2,-1.8669238E-2,5.03168E-3,-1.295545E-3,-5.0552033E-2,-1.3818641E-1,7.60942E-4,1.1310651E-1,2.848938E-3,-1.5787615E-3,-1.7716123E-3,4.0062843E-3,-1.9272873E-2,-2.3181534E-3,9.634987E-3,7.565743E-2,6.3074734E-3,1.7241014E-3,1.1610202E-2,5.267861E-3,1.6217683E-1,4.9737714E-2,2.9040825E-1,1.514912E-1,-7.0272554E-3,-1.0338688E-3,-9.687571E-2,1.4254802E-3,-8.141663E-3,-3.5213497E-3,-8.661685E-4,-4.3371185E-3,-6.5384167E-3,-1.0749666E-1,2.9847318E-2,-6.0474083E-2,1.2997195E-2,6.2078577E-3,-7.10495E-2,2.183724E-3,1.2312038E-4,9.706163E-2,2.4653893E-2,-6.442012E-2,-4.548244E-3,-7.940892E-4,-4.016721E-3,-8.113032E-3,6.0717477E-3,9.642598E-4,6.811439E-2,-1.3134049E-2,-1.2004864E-2,9.938767E-2,1.7437801E-1,3.1434253E-3,5.0107064E-3,-7.980086E-4,8.947797E-3,1.6891176E-2,8.7557975E-3,3.1391324E-3,-5.445959E-3,-1.6181574E-3,-2.544946E-3,1.6982077E-3,-8.188024E-3,-2.9414995E-3,-1.0493287E-3,2.916391E-3,-4.929272E-3,-3.091876E-4,-2.236729E-3,2.2427244E-3,-1.4561897E-3,-5.2550673E-3,1.7367838E-3,-1.5441832E-3,5.4332195E-3,9.2994294E-4,-5.3042206E-3,2.3342576E-3,-1.824139E-3,-6.8961596E-3,1.2160112E-3,5.499531E-3,-4.0633734E-3,3.8748537E-4,-3.1079208E-3,1.5848759E-3,7.897111E-3,2.1629792E-3,8.711605E-3,4.3768445E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,61,63,-1,65,67,-1,69,71,73,75,-1,77,-1,79,-1,81,83,-1,-1,-1,-1,85,87,-1,89,-1,-1,91,-1,93,95,-1,97,-1,-1,-1,99,-1,-1,101,103,105,107,109,111,113,-1,115,117,-1,119,121,123,125,127,129,131,133,-1,135,137,139,141,143,-1,-1,-1,-1,145,147,149,-1,-1,151,-1,-1,-1,153,155,-1,157,159,161,-1,-1,163,165,-1,167,-1,-1,-1,-1,-1,-1,169,171,-1,-1,-1,-1,173,175,177,179,-1,-1,181,-1,-1,-1,-1,-1,183,185,187,189,191,-1,193,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,203,205,207,209,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6625023E0,9.0140295E-1,3.7912822E0,7.7459097E-1,5.264075E-1,1.0053666E0,1.0354099E0,2.6928306E-1,1.6394327E0,4.968083E-1,4.4389427E-2,2.7709788E-1,5.687446E-1,0E0,6.6219187E-1,2.4052572E-1,0E0,1.8377757E-1,3.908816E-1,0E0,0E0,6.610286E-2,0E0,2.337907E-1,1.4943555E-1,3.6872214E-1,1.5845251E-1,4.9327755E-1,1.790143E-1,1.6552782E-1,1.920293E-1,0E0,0E0,1.9603552E-1,5.5333853E-2,0E0,3.8695753E-2,1.214543E-1,1.3413253E-1,3.922581E-2,0E0,1.3797683E-1,3.249867E-1,0E0,6.738305E-2,1.2984681E-1,3.3812308E-1,3.5875898E-2,0E0,4.0241957E-2,0E0,1.13061905E-1,0E0,1.22317314E-1,8.281002E-2,0E0,0E0,0E0,0E0,1.7973764E-1,1.4102226E-1,0E0,1.2999536E-1,0E0,0E0,4.990752E-2,0E0,5.248125E-1,2.7549082E-1,0E0,8.354396E-2,0E0,0E0,0E0,1.6780066E-1,0E0,0E0,4.5918465E-2,4.350257E-2,1.3673691E-1,7.535648E-2,2.3882598E-2,7.418635E-2,2.7692549E-2,0E0,7.632162E-2,2.6722285E-1,0E0,1.3883686E-1,6.219068E-2,6.5905094E-2,2.8826743E-2,2.734955E-2,5.4866306E-2,3.4448367E-1,2.409037E-1,0E0,3.816046E-2,1.5300751E-2,8.870673E-2,7.32317E-2,3.3617347E-2,0E0,0E0,0E0,0E0,6.0901597E-2,2.1469176E-2,1.6557511E-2,0E0,0E0,7.429293E-2,0E0,0E0,0E0,5.1256258E-2,7.367456E-2,0E0,6.619941E-2,4.454039E-2,1.1456628E-1,0E0,0E0,3.026804E-2,2.1168083E-2,0E0,3.61194E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1393777E-1,1.7801422E-1,0E0,0E0,0E0,0E0,3.17809E-2,4.5158178E-2,6.0135722E-2,2.2072852E-2,0E0,0E0,2.4452105E-2,0E0,0E0,0E0,0E0,0E0,3.145268E-2,4.0673092E-2,2.0674776E-2,3.6574565E-2,3.5096187E-2,0E0,5.0426483E-2,2.7735056E-2,0E0,2.608031E-2,1.14049956E-1,5.33489E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.264222E-2,1.87523E-1,5.093741E-2,2.4400109E-1,1.9424796E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,49,49,51,51,53,53,54,54,59,59,60,60,62,62,65,65,67,67,68,68,70,70,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,98,98,99,99,100,100,101,101,106,106,107,107,108,108,111,111,115,115,116,116,118,118,119,119,120,120,123,123,124,124,126,126,133,133,134,134,139,139,140,140,141,141,142,142,145,145,151,151,152,152,153,153,154,154,155,155,157,157,158,158,160,160,161,161,162,162,169,169,170,170,171,171,172,172,173,173],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,62,64,-1,66,68,-1,70,72,74,76,-1,78,-1,80,-1,82,84,-1,-1,-1,-1,86,88,-1,90,-1,-1,92,-1,94,96,-1,98,-1,-1,-1,100,-1,-1,102,104,106,108,110,112,114,-1,116,118,-1,120,122,124,126,128,130,132,134,-1,136,138,140,142,144,-1,-1,-1,-1,146,148,150,-1,-1,152,-1,-1,-1,154,156,-1,158,160,162,-1,-1,164,166,-1,168,-1,-1,-1,-1,-1,-1,170,172,-1,-1,-1,-1,174,176,178,180,-1,-1,182,-1,-1,-1,-1,-1,184,186,188,190,192,-1,194,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,204,206,208,210,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,2.6E1,5.97E3,5.41E2,3.355864E0,9.24E2,1.059448E7,1E0,3E1,4E0,4.7721977E4,6.4153387E-6,4.5866325E6,2.3762597E-2,7.3274844E9,3.89E2,-1.79971E-2,7.635E3,4.217427E7,-6.3834502E-3,-2.8873853E-2,4.1665E3,-2.9971502E-3,2E0,1.8949389E6,6E0,4.745397E6,3.3382E4,2.5652106E7,2.135961E4,9.687E3,5.1168823E-3,1.6806778E-2,4.8614563E2,9.643839E6,-4.169671E-3,1E0,2.4568965E2,1.43E2,1.034073E6,-1.243493E-2,5.691698E2,1.4598765E0,-1.1329408E-3,1.4076087E0,7.0296685E6,1.36E2,2.8582963E-1,-6.834624E-3,9.55188E5,-2.0474137E-3,1.9668E4,-1.3639349E-2,7.026624E7,3.4779343E1,-1.1320735E-2,-3.8030786E-3,-1.2915283E-2,-6.1682626E-3,5.3061223E0,2.2871187E2,-1.1183121E-2,9.5E1,-5.1507945E-3,2.1135827E-4,2.2461708E6,9.329905E-3,5.404E2,3.8304348E1,1.2619777E-2,1.9290142E0,6.0773E-3,1.7535836E-2,-3.4592901E-3,1.4176E4,3.177586E-4,5.0440314E-3,1.957E3,1.2E1,9.9E1,1.1244019E2,6.07E2,3.2874417E2,3.88E2,-3.9093117E-3,1.1356604E2,8.860256E3,8.406424E-3,2.7563172E6,1.880118E-3,8E0,1.6964E4,4.577732E2,4.0695653E0,3.596E3,2.445E3,-9.46087E-3,2.041E3,6.7553375E4,5.657353E1,1.2792593E2,3.2E1,-9.545233E-3,-1.5372083E-2,-7.8240605E-3,3.4368723E-3,7.6811076E-3,3.3025316E5,2.241E2,-2.7530256E-3,-7.81642E-3,1.9545455E0,3.207615E-3,5.728842E-4,4.4554253E-3,3.5714287E-1,2.8361E4,-1.1677748E-2,1.6223962E7,2.7777777E0,2.90099E0,5.03168E-3,-1.295545E-3,6.23E2,3.125E0,7.60942E-4,1.4521204E0,2.848938E-3,-1.5787615E-3,-1.7716123E-3,4.0062843E-3,-1.9272873E-2,-2.3181534E-3,9.777778E0,3.7043128E5,6.3074734E-3,1.7241014E-3,1.1610202E-2,5.267861E-3,1.0905187E9,4.1104166E8,4.73909E0,6.2136906E-1,-7.0272554E-3,-1.0338688E-3,2.49E2,1.4254802E-3,-8.141663E-3,-3.5213497E-3,-8.661685E-4,-4.3371185E-3,1.4929079E2,5.25E0,4.49239E6,1.4141E4,1.407037E2,6.2078577E-3,1.5132743E0,1.901875E2,1.2312038E-4,2.4361508E6,3.7974394E-6,1.676E3,-4.548244E-3,-7.940892E-4,-4.016721E-3,-8.113032E-3,6.0717477E-3,9.642598E-4,1.9882522E0,3.0291306E2,1.02E2,7.202917E7,2.6E1,3.1434253E-3,5.0107064E-3,-7.980086E-4,8.947797E-3,1.6891176E-2,8.7557975E-3,3.1391324E-3,-5.445959E-3,-1.6181574E-3,-2.544946E-3,1.6982077E-3,-8.188024E-3,-2.9414995E-3,-1.0493287E-3,2.916391E-3,-4.929272E-3,-3.091876E-4,-2.236729E-3,2.2427244E-3,-1.4561897E-3,-5.2550673E-3,1.7367838E-3,-1.5441832E-3,5.4332195E-3,9.2994294E-4,-5.3042206E-3,2.3342576E-3,-1.824139E-3,-6.8961596E-3,1.2160112E-3,5.499531E-3,-4.0633734E-3,3.8748537E-4,-3.1079208E-3,1.5848759E-3,7.897111E-3,2.1629792E-3,8.711605E-3,4.3768445E-3],"split_indices":[43,3,2,2,73,2,12,6,0,8,43,52,43,0,5,1,0,1,60,0,0,62,0,32,60,3,60,9,58,43,44,0,0,4,62,0,26,70,2,9,0,67,69,0,69,62,0,49,0,46,0,12,0,7,71,0,0,0,0,73,67,0,10,0,0,43,0,4,68,0,68,0,0,0,2,0,0,9,3,44,67,2,67,10,0,67,62,0,60,53,8,1,4,71,2,2,0,2,48,71,73,2,0,0,0,0,53,62,4,0,0,68,0,0,0,68,1,0,5,69,69,0,0,2,71,0,53,0,0,0,0,0,0,73,43,0,0,0,0,7,7,50,49,0,0,2,0,0,0,0,0,67,69,43,9,67,0,68,4,0,60,53,44,0,0,0,0,0,0,68,67,0,12,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.009E3,2.66E2,7.43E2,2.3E2,3.6E1,6.16E2,1.27E2,1.27E2,1.03E2,1E1,2.6E1,2.59E2,3.57E2,9E0,1.18E2,1.23E2,4E0,1.4E1,8.9E1,4E0,6E0,2.2E1,4E0,2.4E2,1.9E1,3.18E2,3.9E1,1E2,1.8E1,5.2E1,7.1E1,6E0,8E0,7.1E1,1.8E1,6E0,1.6E1,1.86E2,5.4E1,1.3E1,6E0,4.8E1,2.7E2,4E0,3.5E1,2.1E1,7.9E1,1.3E1,5E0,4.5E1,7E0,6.7E1,4E0,5.1E1,2E1,1.4E1,4E0,1E1,6E0,1.02E2,8.4E1,4E0,5E1,9E0,4E0,3.6E1,1.2E1,2.3E1,2.47E2,7E0,2.8E1,4E0,1.7E1,5E0,7.4E1,7E0,6E0,3E1,1.5E1,3.2E1,3.5E1,1.8E1,3.3E1,1.6E1,4E0,4.5E1,5.7E1,4E0,8E1,1.3E1,3.7E1,2.6E1,1E1,1.3E1,1E1,2.43E2,4E0,1.7E1,1.1E1,4.7E1,2.7E1,8E0,2.2E1,8E0,7E0,7E0,2.5E1,2.3E1,1.2E1,4E0,1.4E1,2.9E1,4E0,7E0,9E0,2.4E1,2.1E1,6E0,5.1E1,2.4E1,5.6E1,4E0,9E0,1.9E1,1.8E1,4E0,2.2E1,5E0,5E0,5E0,8E0,5E0,5E0,1.59E2,8.4E1,9E0,8E0,7E0,4E0,3.7E1,1E1,1.7E1,1E1,4E0,4E0,2.1E1,4E0,1.8E1,5E0,7E0,5E0,1.3E1,1.6E1,1E1,1.4E1,1.4E1,7E0,3.1E1,2E1,7E0,1.7E1,2.9E1,2.7E1,7E0,1.2E1,9E0,9E0,1.8E1,4E0,4.4E1,1.15E2,1.8E1,6.6E1,3.2E1,5E0,5E0,5E0,9E0,8E0,6E0,4E0,1.5E1,6E0,6E0,7E0,5E0,1.1E1,4E0,6E0,7E0,7E0,5E0,9E0,1.7E1,1.4E1,1E1,1E1,1.3E1,4E0,4E0,2.5E1,2.2E1,5E0,2.5E1,1.9E1,2.5E1,9E1,8E0,1E1,2.7E1,3.9E1,2.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[1.1542375E-3,-5.76815E-2,1.068986E-1,-1.3268349E-1,-2.0550696E-2,-1.4995354E-1,1.2255918E-1,-1.2017352E-1,-2.4729492E-1,-8.86327E-2,1.6061844E-3,1.2959312E-3,-2.1845846E-1,1.8485563E-2,1.1080396E-1,-2.0795041E-1,-1.0434892E-1,-2.0038301E-2,-1.2309187E-1,-6.996615E-2,-1.6445434E-2,-3.311938E-3,1.0969806E-2,-2.9613188E-1,-3.524436E-3,9.177153E-2,2.6950178E-1,-6.725694E-3,-1.2040149E-2,9.711703E-3,-1.2355179E-1,-3.3570593E-4,-8.622217E-3,5.130638E-2,-9.074831E-2,-7.682282E-2,9.8082265E-3,-8.458637E-3,-1.721166E-2,8.068845E-2,2.7543744E-1,3.297408E-1,1.3756038E-1,-7.711062E-2,1.174303E-1,-1.127049E-1,-2.63053E-1,-9.3462586E-4,3.957514E-3,-8.916907E-3,-8.114612E-2,5.2357037E-2,-1.096071E-1,4.1632015E-2,-1.2792008E-2,3.5423167E-2,1.19343095E-1,1.7810117E-2,4.3378366E-4,6.9425576E-3,1.6544815E-2,3.8725317E-3,8.606136E-3,-1.6418867E-3,-7.0751617E-3,1.0048811E-2,6.870214E-4,-1.019315E-1,-1.0123459E-2,-1.6845658E-2,-6.216617E-3,2.5020705E-3,-9.610132E-2,-1.5455208E-3,4.904986E-3,-1.4406097E-1,-2.4317175E-2,1.4849249E-2,8.093883E-2,1.4323044E-3,-9.819756E-2,1.2038257E-1,1.4053195E-2,-4.906671E-2,1.3894041E-1,-1.3363954E-1,-6.323991E-2,-1.4279623E-1,-7.180374E-2,-2.0152305E-1,-1.0812687E-1,1.1029331E-3,-3.9046765E-3,-6.527125E-3,2.6111165E-2,1.0386447E-1,3.69592E-2,7.5321584E-3,-7.894034E-3,-3.9806597E-2,-8.828309E-3,1.9458464E-1,1.1414602E-2,2.3166211E-2,-8.321872E-3,3.370835E-2,-1.1756204E-2,1.6452897E-1,2.0416373E-2,-7.5284176E-2,-1.6244921E-1,-2.5253227E-2,-1.0038481E-1,-1.7571834E-1,-1.5262672E-3,8.3889463E-4,-8.176051E-2,-4.400068E-3,-1.3340265E-2,-7.920805E-4,-6.7401864E-3,3.2237347E-3,7.075076E-2,5.254402E-2,6.6081984E-3,3.900055E-3,8.0245466E-4,-1.1625355E-1,1.6852802E-2,-3.7231839E-3,1.1153442E-3,3.4418919E-3,1.2922096E-2,-5.374962E-3,4.150367E-3,2.9706245E-2,-4.1438416E-3,3.5956511E-3,-1.6613628E-4,2.2103499E-1,1.3144529E-1,-5.0971475E-3,7.439299E-2,-4.1886596E-3,2.020825E-4,-8.350153E-3,-4.717289E-3,3.4931733E-3,-2.1971525E-3,-1.4555298E-3,-8.170462E-3,-9.469792E-3,-2.4391005E-3,-4.382608E-3,-7.343796E-4,-1.5869667E-3,2.291051E-3,1.7959994E-3,5.8052777E-3,3.691448E-3,-8.370355E-5,-8.759674E-3,-2.2999875E-4,1.4840086E-3,-1.7780653E-3,-1.6564471E-3,2.0127238E-3,1.1070752E-2,3.3547985E-3,7.1514794E-3,-3.8565067E-3,5.220347E-3,-2.600876E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,-1,31,33,-1,35,-1,37,-1,39,41,-1,-1,43,45,-1,-1,47,49,51,53,-1,-1,55,57,59,61,63,65,67,69,-1,-1,-1,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,-1,-1,-1,-1,87,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,-1,-1,121,123,125,127,-1,129,-1,131,133,135,-1,137,-1,139,141,143,145,147,149,151,-1,-1,153,-1,-1,-1,-1,155,157,159,-1,-1,-1,161,163,-1,-1,-1,-1,-1,-1,165,-1,-1,-1,167,169,-1,171,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.526348E0,1.8742855E0,1.5267992E0,2.9001594E-1,6.8197757E-1,2.7440012E-1,1.1107273E0,2.602744E-1,4.3290412E-1,5.290116E-1,3.9746946E-1,0E0,1.540944E-1,0E0,1.0052257E0,6.059897E-2,3.8231063E-1,0E0,1.0167748E-1,2.69288E-1,0E0,3.2481703E-1,0E0,1.8177211E-2,0E0,6.006639E-1,2.4076939E-1,0E0,0E0,2.52217E-1,1.9581223E-1,0E0,0E0,4.10563E-2,7.271284E-2,2.2172725E-1,2.0623092E-1,0E0,0E0,5.031893E-1,4.942106E-1,6.841874E-2,1.8422857E-2,4.197003E-2,1.1993627E-1,1.3817811E-1,8.145499E-2,0E0,0E0,0E0,1.7517936E-1,5.4347377E-2,1.1697805E-1,1.2407483E-1,2.0423296E-1,2.4341156E-1,5.216763E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4981043E-1,0E0,0E0,0E0,0E0,7.5974226E-2,0E0,0E0,3.8090587E-2,3.9732635E-2,1.3028987E-1,4.455778E-2,1.5368478E-1,1.2257518E-1,2.1609136E-1,1.9705957E-1,2.9761422E-1,4.2124128E-1,1.0545385E-1,8.212565E-2,8.453268E-2,4.8062652E-2,6.521124E-2,6.622201E-2,0E0,0E0,0E0,6.9246545E-2,5.4805994E-2,1.5268533E-2,1.646397E-1,0E0,4.3674428E-2,0E0,1.427058E-1,1.3068056E-1,8.0166966E-2,0E0,2.1676803E-2,0E0,1.8593025E-1,1.9169639E-1,3.1027839E-2,3.2322764E-2,7.3339805E-2,1.5244117E-1,5.7515204E-2,0E0,0E0,3.9054394E-2,0E0,0E0,0E0,0E0,8.18454E-2,3.5369493E-2,2.2451662E-2,0E0,0E0,0E0,8.0680445E-2,1.12776116E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.243001E-2,0E0,0E0,0E0,9.576225E-2,3.8933206E-1,0E0,1.02219135E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,21,21,23,23,25,25,26,26,29,29,30,30,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,50,50,51,51,52,52,53,53,54,54,55,55,56,56,67,67,72,72,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,94,94,95,95,96,96,97,97,99,99,101,101,102,102,103,103,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,116,116,121,121,122,122,123,123,127,127,128,128,135,135,139,139,140,140,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,-1,32,34,-1,36,-1,38,-1,40,42,-1,-1,44,46,-1,-1,48,50,52,54,-1,-1,56,58,60,62,64,66,68,70,-1,-1,-1,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,-1,-1,-1,-1,88,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,-1,-1,122,124,126,128,-1,130,-1,132,134,136,-1,138,-1,140,142,144,146,148,150,152,-1,-1,154,-1,-1,-1,-1,156,158,160,-1,-1,-1,162,164,-1,-1,-1,-1,-1,-1,166,-1,-1,-1,168,170,-1,172,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,4.8465605E3,4.9735293E0,1E0,1E0,3.6E1,3.6E1,2E0,3.5531914E0,7.82261E6,1.2959312E-3,7.978062E6,1.8485563E-2,6.0052995E6,6.48334E5,1E0,-2.0038301E-2,4.0869565E0,5E-1,-1.6445434E-2,1.5292561E2,1.0969806E-2,1.1558494E8,-3.524436E-3,5.428175E3,1.9047619E0,-6.725694E-3,-1.2040149E-2,4.03E2,3.0977E4,-3.3570593E-4,-8.622217E-3,7.52E2,1.4076087E0,3.1488764E0,3.0416667E0,-8.458637E-3,-1.721166E-2,1.1806228E6,4.1325716E7,1.4081731E1,5.5177975E6,6.737099E7,2.00087E5,3.2856784E7,5.27E2,-9.3462586E-4,3.957514E-3,-8.916907E-3,1.6386554E0,1.525E4,1.585814E6,1.058E3,1E0,2.5959E4,7.549744E7,1.7810117E-2,4.3378366E-4,6.9425576E-3,1.6544815E-2,3.8725317E-3,8.606136E-3,-1.6418867E-3,-7.0751617E-3,1.0048811E-2,6.870214E-4,1.8224286E2,-1.0123459E-2,-1.6845658E-2,-6.216617E-3,2.5020705E-3,3.3820656E7,-1.5455208E-3,4.904986E-3,1.1430505E3,4.4091418E4,6.620166E-6,1.1939979E8,2.673585E1,6.763312E7,7.362415E1,3.0070068E7,8.620714E2,2E0,2.772925E3,1.2E1,4.8E0,1.2783505E0,8.238114E2,3.2E1,1.1029331E-3,-3.9046765E-3,-6.527125E-3,2.6086957E0,2.378914E6,7.3333335E-1,3.1706784E0,-7.894034E-3,3.3064186E7,-8.828309E-3,4.204547E5,1.5477113E2,1.6254545E1,-8.321872E-3,2.152E3,-1.1756204E-2,5.6516E4,8.399457E2,3.3857143E1,1.8E6,1.9741463E2,8.7578294E5,8.49E2,-1.5262672E-3,8.3889463E-4,1.2673605E7,-4.400068E-3,-1.3340265E-2,-7.920805E-4,-6.7401864E-3,1.24E3,6.7723384E7,2E0,6.6081984E-3,3.900055E-3,8.0245466E-4,9.24E2,4.3E1,-3.7231839E-3,1.1153442E-3,3.4418919E-3,1.2922096E-2,-5.374962E-3,4.150367E-3,8.2103E4,-4.1438416E-3,3.5956511E-3,-1.6613628E-4,1.683263E7,2.9366477E0,-5.0971475E-3,3.5510652E6,-4.1886596E-3,2.020825E-4,-8.350153E-3,-4.717289E-3,3.4931733E-3,-2.1971525E-3,-1.4555298E-3,-8.170462E-3,-9.469792E-3,-2.4391005E-3,-4.382608E-3,-7.343796E-4,-1.5869667E-3,2.291051E-3,1.7959994E-3,5.8052777E-3,3.691448E-3,-8.370355E-5,-8.759674E-3,-2.2999875E-4,1.4840086E-3,-1.7780653E-3,-1.6564471E-3,2.0127238E-3,1.1070752E-2,3.3547985E-3,7.1514794E-3,-3.8565067E-3,5.220347E-3,-2.600876E-3],"split_indices":[2,43,43,68,29,8,10,2,32,68,43,0,62,0,43,46,21,0,73,68,0,67,0,60,0,67,69,0,0,2,44,0,0,12,69,73,69,0,0,43,66,73,60,7,5,60,0,0,0,0,69,1,9,2,19,9,7,0,0,0,0,0,0,0,0,0,0,4,0,0,0,0,7,0,0,48,48,53,7,68,7,73,58,4,6,62,8,69,68,48,10,0,0,0,69,9,68,69,0,5,0,43,71,69,0,2,0,9,67,67,5,4,60,2,0,0,60,0,0,0,0,44,7,8,0,0,0,2,8,0,0,0,0,0,0,1,0,0,0,60,57,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,6.73E2,3.74E2,2.22E2,4.51E2,2.1E1,3.53E2,2.02E2,2E1,1.1E2,3.41E2,6E0,1.5E1,1.3E1,3.4E2,2.9E1,1.73E2,7E0,1.3E1,1.04E2,6E0,3.35E2,6E0,9E0,6E0,3.05E2,3.5E1,1.5E1,1.4E1,2.5E1,1.48E2,5E0,8E0,1.5E1,8.9E1,5E1,2.85E2,5E0,4E0,2.89E2,1.6E1,2.3E1,1.2E1,1.4E1,1.1E1,1.39E2,9E0,5E0,1E1,6E0,8.3E1,1E1,4E1,1.18E2,1.67E2,1.34E2,1.55E2,1.1E1,5E0,4E0,1.9E1,7E0,5E0,1E1,4E0,5E0,6E0,1.28E2,1.1E1,4E0,5E0,8E0,7.5E1,4E0,6E0,2.8E1,1.2E1,7.1E1,4.7E1,1.44E2,2.3E1,2.6E1,1.08E2,1.6E1,1.39E2,6.9E1,5.9E1,2.4E1,5.1E1,9E0,1.9E1,7E0,5E0,4E0,6.7E1,3E1,1.7E1,1.4E2,4E0,1.5E1,8E0,1.5E1,1.1E1,1.04E2,4E0,1.2E1,4E0,1.14E2,2.5E1,2.4E1,4.5E1,3E1,2.9E1,1.8E1,6E0,5E0,4.6E1,5E0,4E0,6E0,1.3E1,4.5E1,2.2E1,1.4E1,1.6E1,4E0,1.3E1,9E0,1.31E2,9E0,6E0,7E0,8E0,4E0,7E0,9.9E1,5E0,5E0,7E0,4E1,7.4E1,7E0,1.8E1,2E1,4E0,3.3E1,1.2E1,5E0,2.5E1,1.6E1,1.3E1,1.4E1,4E0,3.8E1,8E0,2.5E1,2E1,1.5E1,7E0,9E0,5E0,5E0,4E0,1.03E2,2.8E1,1.7E1,8.2E1,3.5E1,5E0,6.7E1,7E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"173","size_leaf_vector":"1"}},{"base_weights":[-1.3648425E-3,-4.643758E-2,1.2990181E-1,-3.1971816E-2,-2.3962098E-1,3.364901E-1,1.0956251E-1,-1.2884802E-1,-8.204965E-3,-1.8504345E-1,-2.939666E-2,1.7244022E-2,5.982114E-3,-7.7699996E-3,1.2363854E-1,-1.8769374E-1,-9.407046E-2,1.3923354E-2,-1.2019227E-2,-1.1522745E-1,-3.8488916E-1,1.03314966E-1,2.783451E-1,-2.3445815E-1,-1.574157E-1,-4.6898846E-2,-1.2055755E-1,-6.422348E-2,1.646271E-2,-1.4091845E-2,-8.297858E-2,-7.904948E-3,-2.1731677E-2,3.8110703E-2,1.3086286E-1,1.5881384E-2,8.615531E-3,-6.019431E-3,-1.22196E-2,-3.614537E-3,-7.7784923E-3,2.7469278E-3,-6.4881206E-2,-1.2851924E-1,-1.4535735E-3,-5.080084E-2,-1.8941809E-1,-6.767619E-3,9.0190314E-2,-1.3917547E-2,-1.12399995E-1,-1.3047696E-2,6.508588E-2,1.4266464E-1,-5.4888823E-3,2.6243364E-3,-2.8953925E-3,-8.2422554E-2,1.0056944E-3,-1.742598E-1,-1.07081674E-1,-4.0266406E-2,-1.3905594E-1,-1.1049805E-2,-4.96888E-3,-1.6899758E-3,-1.3199069E-2,1.02852836E-1,-4.620193E-3,2.962765E-3,-3.126654E-3,-1.696859E-1,-3.4480296E-2,4.01074E-2,1.11497855E-2,7.116663E-2,1.620457E-1,-4.4580832E-2,-6.3913036E-3,-4.2986376E-3,-9.496773E-3,-7.158731E-2,-7.2409557E-3,-6.275233E-2,5.4375557E-3,-1.781616E-1,-2.1513053E-3,-1.170128E-1,7.2911386E-3,1.4994432E-1,4.31407E-2,-1.079468E-2,-4.642897E-3,-4.1099167E-3,1.1894957E-3,-4.169197E-2,6.3070446E-2,1.0758131E-1,2.5152206E-2,2.0418164E-1,1.200258E-1,-3.435467E-3,3.4748582E-4,-6.092643E-4,-3.9851996E-3,-1.6120221E-2,-8.916895E-2,2.2807406E-2,-5.4019717E-3,-9.701979E-3,-3.9367694E-3,-2.4814813E-3,-1.3328407E-2,-2.5004733E-3,8.146435E-2,1.6666955E-1,4.9268856E-4,-2.0551838E-2,1.0149157E-1,1.0272469E-3,-3.5312236E-3,1.4029053E-1,1.312462E-2,2.1578704E-3,6.3079577E-3,-3.3562683E-4,2.9215373E-3,1.421513E-1,2.563523E-1,1.2189953E-2,1.5568084E-1,-3.504771E-3,9.823864E-4,-2.1566E-3,-5.3018993E-3,4.1064695E-3,-8.5137336E-4,-3.4009924E-3,5.8237165E-3,2.6133207E-3,-8.3855167E-4,5.628635E-3,-2.068917E-3,5.780465E-3,1.1389654E-2,2.8093476E-3,-2.9004205E-3,9.170078E-3,2.5837212E-3,8.167031E-3,1.1392484E-3,-3.0959942E-3,2.1333117E-3,8.1001725E-3,1.2493079E-3,1.4434329E-2,7.401501E-3,6.6161314E-3,-4.207273E-3,4.4999705E-3,1.0532091E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,61,63,65,67,69,71,-1,73,75,-1,-1,-1,77,-1,79,81,83,85,-1,-1,87,-1,89,-1,-1,-1,91,93,95,-1,97,99,101,-1,-1,-1,103,-1,105,107,109,-1,111,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,-1,-1,-1,-1,131,133,135,-1,-1,-1,137,-1,139,141,143,-1,145,147,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.064577E0,2.124577E0,1.0640211E0,1.6359792E0,1.0297062E0,1.2671018E-1,9.590945E-1,2.688551E-1,6.7905635E-1,6.2536657E-1,0E0,0E0,0E0,0E0,6.912625E-1,3.530526E-2,1.0679549E-1,0E0,8.4263045E-1,2.0115706E-1,1.3886237E-1,3.614459E-1,9.212375E-2,2.1862984E-2,1.631403E-2,3.053166E-2,3.6388755E-2,3.264659E-1,6.2917733E-1,0E0,6.5487504E-2,0E0,0E0,5.544626E-1,4.3831038E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.892436E-2,3.9793164E-2,3.232944E-2,0E0,1.6360933E-1,5.167353E-2,3.9608648E-1,2.2222221E-1,5.0494954E-2,9.6814275E-2,0E0,2.4567217E-1,1.7830944E-1,0E0,0E0,0E0,3.8271025E-2,0E0,1.9005895E-2,5.5750042E-2,1.688833E-1,6.0424447E-2,0E0,0E0,2.8671393E-1,0E0,2.2692168E-1,0E0,0E0,0E0,3.405574E-2,3.8384136E-2,1.0030442E-1,0E0,4.9237728E-2,1.6494155E-1,2.2282878E-2,0E0,0E0,0E0,1.9429564E-2,0E0,1.3406757E-1,1.1956794E-1,2.0580828E-2,0E0,3.9209622E-1,1.8685645E-1,1.05379105E-1,1.4331341E-1,0E0,0E0,0E0,0E0,2.8351832E-2,1.5628836E-1,2.3588896E-2,1.8978842E-2,1.3183975E-1,2.1341628E-1,0E0,0E0,0E0,0E0,9.31837E-2,7.07615E-2,1.3515425E-1,0E0,0E0,0E0,1.281458E-1,0E0,2.1179464E-1,1.5835415E-1,1.0520291E-1,0E0,6.909832E-2,7.729587E-2,0E0,0E0,6.314829E-2,7.1961775E-2,0E0,0E0,0E0,0E0,9.4563544E-2,9.55354E-2,2.1597436E-1,1.5473652E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,33,33,34,34,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,57,57,59,59,60,60,61,61,62,62,65,65,67,67,71,71,72,72,73,73,75,75,76,76,77,77,81,81,83,83,84,84,85,85,87,87,88,88,89,89,90,90,95,95,96,96,97,97,98,98,99,99,100,100,105,105,106,106,107,107,111,111,113,113,114,114,115,115,117,117,118,118,121,121,122,122,127,127,128,128,129,129,130,130],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,62,64,66,68,70,72,-1,74,76,-1,-1,-1,78,-1,80,82,84,86,-1,-1,88,-1,90,-1,-1,-1,92,94,96,-1,98,100,102,-1,-1,-1,104,-1,106,108,110,-1,112,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,-1,-1,-1,-1,132,134,136,-1,-1,-1,138,-1,140,142,144,-1,146,148,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,9.932432E0,5.6E1,3.56E2,2E1,1.84E2,8.147158E4,4.54E2,7.336111E4,4.9E1,-2.939666E-2,1.7244022E-2,5.982114E-3,-7.7699996E-3,6.9307615E6,1.2916666E0,1.9E1,1.3923354E-2,4.3827028E5,3.909839E-5,1.1E1,3.3580637E8,3.3382E4,6.2886596E-1,7E0,1.00019E5,7.997723E6,3.2856784E7,4.265829E2,-1.4091845E-2,4.1665E3,-7.904948E-3,-2.1731677E-2,1.3809524E0,7.134432E7,1.5881384E-2,8.615531E-3,-6.019431E-3,-1.22196E-2,-3.614537E-3,-7.7784923E-3,6.695E3,3E0,1.2139681E3,-1.4535735E-3,3.0084E4,1.731E3,2.511E3,1.1043767E3,1.3E1,3.541824E2,-1.3047696E-2,7.459E3,6.9664386E2,-5.4888823E-3,2.6243364E-3,-2.8953925E-3,1.7544615E0,1.0056944E-3,3.3305264E2,4.325E0,3.839424E2,1E0,-1.1049805E-2,-4.96888E-3,5.862757E1,-1.3199069E-2,3.5419354E0,-4.620193E-3,2.962765E-3,-3.126654E-3,2.56E2,1.0118329E4,6.726722E2,1.11497855E-2,4.569524E2,1.5918367E0,1.52E2,-6.3913036E-3,-4.2986376E-3,-9.496773E-3,3.2E1,-7.2409557E-3,1.5132743E0,1E0,1.266E3,-2.1513053E-3,9.0217394E-1,1.559287E5,3.4052496E8,1.552E3,-1.079468E-2,-4.642897E-3,-4.1099167E-3,1.1894957E-3,6.63651E4,1.6942337E1,2.6862775E6,5.53648E8,1.1618335E0,3.2608695E0,-3.435467E-3,3.4748582E-4,-6.092643E-4,-3.9851996E-3,7.56484E5,1.0142858E1,5.449711E8,-5.4019717E-3,-9.701979E-3,-3.9367694E-3,7.82E2,-1.3328407E-2,4E0,1.3333334E0,1.09652E5,4.9268856E-4,1.798E3,4.111111E0,1.0272469E-3,-3.5312236E-3,1.3205668E7,2.2715E4,2.1578704E-3,6.3079577E-3,-3.3562683E-4,2.9215373E-3,5E0,2.1089442E0,2.7578741E1,2.4077083E3,-3.504771E-3,9.823864E-4,-2.1566E-3,-5.3018993E-3,4.1064695E-3,-8.5137336E-4,-3.4009924E-3,5.8237165E-3,2.6133207E-3,-8.3855167E-4,5.628635E-3,-2.068917E-3,5.780465E-3,1.1389654E-2,2.8093476E-3,-2.9004205E-3,9.170078E-3,2.5837212E-3,8.167031E-3,1.1392484E-3,-3.0959942E-3,2.1333117E-3,8.1001725E-3,1.2493079E-3,1.4434329E-2,7.401501E-3,6.6161314E-3,-4.207273E-3,4.4999705E-3,1.0532091E-2],"split_indices":[2,69,0,2,6,10,43,1,60,3,0,0,0,0,43,68,10,0,43,53,8,7,9,68,0,5,9,60,67,0,62,0,0,69,59,0,0,0,0,0,0,9,8,48,0,44,2,2,67,0,67,0,2,67,0,0,0,68,0,48,69,4,19,0,0,62,0,69,0,0,0,10,48,4,0,67,68,44,0,0,0,10,0,68,19,2,0,68,48,7,2,0,0,0,0,48,71,62,7,68,69,0,0,0,0,12,73,46,0,0,0,2,0,73,68,1,0,44,69,0,0,9,44,0,0,0,0,6,53,71,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.023E3,7.62E2,2.61E2,7.1E2,5.2E1,2.2E1,2.39E2,1.39E2,5.71E2,4.7E1,5E0,1.8E1,4E0,1.1E1,2.28E2,5E1,8.9E1,6E0,5.65E2,3.6E1,1.1E1,2.03E2,2.5E1,1.7E1,3.3E1,3.3E1,5.6E1,1.99E2,3.66E2,4E0,3.2E1,4E0,7E0,6.1E1,1.42E2,1.3E1,1.2E1,5E0,1.2E1,5E0,2.8E1,9E0,2.4E1,5.1E1,5E0,1.81E2,1.8E1,2.79E2,8.7E1,1E1,2.2E1,4E0,5.7E1,1.36E2,6E0,5E0,4E0,2E1,4E0,1.4E1,3.7E1,1.63E2,1.8E1,1E1,8E0,2.75E2,4E0,8.2E1,5E0,4E0,6E0,1.2E1,1E1,5.1E1,6E0,3E1,1.06E2,1.3E1,7E0,5E0,9E0,2.3E1,1.4E1,1.09E2,5.4E1,1.2E1,6E0,1.9E1,2.56E2,4.5E1,3.7E1,5E0,7E0,5E0,5E0,1.1E1,4E1,1.6E1,1.4E1,5.1E1,5.5E1,8E0,5E0,5E0,1.8E1,4E1,6.9E1,4.8E1,6E0,8E0,4E0,1.2E1,7E0,2.27E2,2.9E1,4E1,5E0,1.8E1,1.9E1,4E0,7E0,1.5E1,2.5E1,6E0,1E1,8E0,6E0,2.5E1,2.6E1,1.4E1,4.1E1,1.5E1,2.5E1,2.7E1,4.2E1,1.8E1,3E1,8E0,4E0,4.7E1,1.8E2,2.2E1,7E0,2.8E1,1.2E1,6E0,1.2E1,5E0,1.4E1,1.1E1,4E0,7E0,1.8E1,1.9E1,6E0,1.5E1,1.1E1,6E0,8E0,2.4E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-2.3163706E-3,-4.3742355E-2,1.2968592E-1,-1.1102305E-1,-1.1658012E-2,8.893596E-2,2.077015E-1,-1.0250346E-1,-2.0608572E-2,-6.083685E-3,-2.345919E-1,1.02474086E-1,-8.679268E-3,3.111256E-1,1.2365165E-1,-8.4660165E-2,-1.9096068E-1,-4.335501E-2,2.5062487E-2,-8.14113E-2,-2.1408575E-2,1.307893E-1,2.9094229E-2,2.6379682E-3,3.5844988E-1,1.5902634E-1,4.275372E-3,-9.175615E-2,1.0311958E-2,-1.4437537E-1,-2.2984022E-2,-9.431513E-2,-2.3252202E-2,3.1502713E-2,-1.2919953E-2,1.5573751E-3,-8.327399E-3,-5.9242765E-3,1.4147119E-1,7.192187E-2,-8.044763E-3,1.0673161E-2,1.8483171E-2,2.7062166E-1,1.1890965E-1,4.742078E-3,-2.6675216E-3,-8.077384E-2,-2.2548053E-1,-9.565029E-3,-8.075886E-2,-1.251941E-1,4.3581687E-2,-1.1334203E-1,-1.4552204E-2,8.3757125E-2,7.1165347E-3,1.5431722E-1,-1.1361421E-3,8.1400096E-4,1.1968596E-1,-5.0875306E-2,2.390062E-2,4.9015544E-3,1.760288E-2,3.975848E-2,7.047192E-3,-1.2097469E-1,-2.7006762E-2,-1.4662112E-2,-3.4708255E-3,-4.3994233E-2,-6.594967E-3,-8.983631E-2,-2.1288743E-2,3.6841158E-3,-1.3964837E-3,-4.7106115E-4,-1.6389525E-1,2.4860755E-2,-3.3029135E-2,3.8901668E-2,1.1957992E-1,-3.081001E-3,1.4776576E-1,1.3359898E-2,1.4105667E-1,5.19689E-3,-6.3412455E-3,7.966141E-3,1.012034E-3,-4.2694362E-4,-4.5731384E-3,-1.2872173E-3,2.0908068E-3,-2.7223488E-3,5.3299638E-3,-1.0546174E-2,-1.0783026E-1,2.0469642E-1,-6.1550986E-2,-5.502692E-3,1.097008E-2,-1.0630176E-1,-5.4800945E-5,-3.7516362E-3,-9.49331E-3,-6.4669102E-3,5.0825156E-2,-4.7684547E-2,5.4219753E-2,5.8565225E-2,-2.2882926E-3,1.3279818E-1,1.2547638E-4,-3.92845E-2,2.6936859E-2,9.001348E-3,2.7892513E-3,9.377825E-2,1.853339E-1,-7.85614E-2,-7.4599828E-3,5.0631985E-3,1.3716438E-2,-4.0502585E-2,-1.473619E-1,2.7592587E-3,-1.8470054E-3,-4.9784373E-2,-1.4897035E-1,1.4266383E-3,-1.8684632E-3,3.2081798E-2,4.7341147E-3,-5.59169E-2,2.8386528E-3,4.7767456E-3,6.6411955E-4,-1.1531069E-3,7.7354796E-2,1.02882884E-1,1.962206E-1,-1.3367072E-2,-2.275188E-2,-3.5706526E-3,8.042419E-2,1.3515028E-1,5.812674E-2,1.0124369E-2,8.997898E-2,-4.4659856E-3,-2.0337415E-4,1.0180435E-4,-2.7191702E-3,-8.4482925E-3,-2.1657597E-3,-4.703047E-3,-6.4426457E-4,-1.0158487E-2,-4.609284E-3,-9.525904E-4,2.254283E-3,-8.0058066E-4,-3.372192E-3,9.4284205E-4,5.790955E-3,1.5408425E-4,5.404589E-3,4.3060756E-3,1.0701922E-2,-4.781559E-3,2.3125742E-4,8.74795E-4,-3.619816E-3,1.08540235E-2,2.4840198E-3,3.1417976E-3,9.341854E-3,-3.8126172E-3,4.335523E-3,5.1972517E-3,1.2449378E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,-1,41,43,45,47,-1,49,-1,51,53,55,-1,-1,-1,-1,57,59,61,-1,-1,63,65,-1,-1,67,69,-1,71,73,75,77,79,81,83,85,87,-1,89,91,93,-1,-1,95,-1,97,99,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,109,111,113,115,117,-1,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,121,123,125,-1,127,129,-1,-1,-1,131,133,135,137,139,-1,141,-1,143,145,-1,-1,147,149,151,-1,-1,-1,153,155,-1,-1,157,159,-1,-1,161,-1,163,-1,-1,-1,-1,165,167,169,-1,171,173,175,177,179,-1,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.654453E0,1.6975689E0,7.667732E-1,6.793349E-1,6.62178E-1,6.3382363E-1,6.9172263E-1,3.7684703E-1,0E0,6.0705507E-1,4.1790777E-1,3.2240415E-1,0E0,4.243307E-1,2.0201927E-1,4.7706652E-1,5.1366234E-1,2.4162403E-1,5.681142E-1,1.0848279E-1,0E0,3.30796E-1,7.215449E-2,0E0,7.2004795E-2,1.3346338E-1,7.913114E-2,2.8285062E-1,0E0,1.3077933E-1,0E0,2.907496E-1,1.3316101E-1,3.5620385E-1,0E0,0E0,0E0,0E0,1.9985628E-1,5.2359037E-2,3.54219E-2,0E0,0E0,1.23048484E-1,7.11292E-2,0E0,0E0,4.0957606E-1,1.8339062E-1,0E0,3.9302245E-2,6.1125803E-1,3.7109327E-2,7.654405E-2,1.1588156E-1,1.3835764E-1,2.7583617E-1,1.3804603E-1,1.7011037E-1,0E0,5.2954093E-2,2.1119406E-2,1.7842624E-2,0E0,0E0,8.141511E-2,0E0,1.291213E-1,6.7693174E-1,0E0,0E0,5.9106953E-2,0E0,7.43168E-2,0E0,0E0,0E0,0E0,1.7705292E-2,4.181229E-2,1.4068162E-1,7.419587E-2,7.533151E-2,1.9667351E-1,4.1715175E-2,0E0,1.7730188E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4275563E-1,6.391525E-2,1.2588277E-1,0E0,2.4744278E-2,9.627879E-2,0E0,0E0,0E0,3.1703446E-2,2.3817562E-2,8.698554E-2,2.986348E-2,5.529017E-2,0E0,6.329346E-2,0E0,3.3240622E-1,1.6183195E-1,0E0,0E0,6.1468035E-2,1.318196E-1,8.819911E-2,0E0,0E0,0E0,4.716234E-2,4.137495E-2,0E0,0E0,3.5863835E-2,6.179875E-2,0E0,0E0,2.0383798E-2,0E0,5.606109E-2,0E0,0E0,0E0,0E0,7.3967054E-2,4.453203E-2,2.5852442E-2,0E0,1.7534298E-1,1.0919836E-1,1.3564631E-1,8.080357E-2,1.433868E-1,0E0,1.6839348E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,38,38,39,39,40,40,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,65,65,67,67,68,68,71,71,73,73,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,98,98,99,99,100,100,102,102,103,103,107,107,108,108,109,109,110,110,111,111,113,113,115,115,116,116,119,119,120,120,121,121,125,125,126,126,129,129,130,130,133,133,135,135,140,140,141,141,142,142,144,144,145,145,146,146,147,147,148,148,150,150],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,-1,42,44,46,48,-1,50,-1,52,54,56,-1,-1,-1,-1,58,60,62,-1,-1,64,66,-1,-1,68,70,-1,72,74,76,78,80,82,84,86,88,-1,90,92,94,-1,-1,96,-1,98,100,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,110,112,114,116,118,-1,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,122,124,126,-1,128,130,-1,-1,-1,132,134,136,138,140,-1,142,-1,144,146,-1,-1,148,150,152,-1,-1,-1,154,156,-1,-1,158,160,-1,-1,162,-1,164,-1,-1,-1,-1,166,168,170,-1,172,174,176,178,180,-1,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8439855E5,1.6466942E3,3.8304348E1,1.2674157E1,9.7172376E7,1.7103828E7,1E0,-2.0608572E-2,8.23E2,1E0,8E0,-8.679268E-3,1.5599597E8,2.5130852E2,6.68411E2,7.45E0,1.5933333E2,1.4676277E7,1.4787234E1,-2.1408575E-2,6.6903913E-1,1.4E1,2.6379682E-3,4.48375E5,1.5277778E0,3.85374E8,3.0977E4,1.0311958E-2,5E0,-2.2984022E-2,4.57E2,5.8182236E1,9.538462E0,-1.2919953E-2,1.5573751E-3,-8.327399E-3,-5.9242765E-3,5.2409735E-7,7.601393E2,2.6844707E8,1.0673161E-2,1.8483171E-2,4.420569E1,1.270394E5,4.742078E-3,-2.6675216E-3,3.72E2,1.266E3,-9.565029E-3,1E0,4.5865917E8,3.5714287E-1,5.2224144E5,1.3619632E0,1.188E3,2.779E3,6.229955E4,2.6750835E6,8.1400096E-4,4.4522205E6,1.3342042E0,4.6750406E4,4.9015544E-3,1.760288E-2,8.93E2,7.047192E-3,6.6E0,1.6E1,-1.4662112E-2,-3.4708255E-3,7.026624E7,-6.594967E-3,4.92E2,-2.1288743E-2,3.6841158E-3,-1.3964837E-3,-4.7106115E-4,1.08712E5,2.4346602E2,3.7066666E2,1E0,1.1208894E0,1.273801E6,4.5E0,1.3359898E-2,1.606722E8,5.19689E-3,-6.3412455E-3,7.966141E-3,1.012034E-3,-4.2694362E-4,-4.5731384E-3,-1.2872173E-3,2.0908068E-3,-2.7223488E-3,5.3299638E-3,-1.0546174E-2,6.3623324E-2,1.9E1,4.142857E0,-5.502692E-3,1.6E1,3.1E1,-5.4800945E-5,-3.7516362E-3,-9.49331E-3,1.901875E2,2.12E2,1.6100003E8,9.207547E0,1.4953704E0,-2.2882926E-3,1.2E1,1.2547638E-4,5E0,1.5736016E6,9.001348E-3,2.7892513E-3,2.5054401E1,1.7103828E7,1.99E2,-7.4599828E-3,5.0631985E-3,1.3716438E-2,1.8E0,5.3776E4,2.7592587E-3,-1.8470054E-3,4.9E2,9.502128E1,1.4266383E-3,-1.8684632E-3,5.08E2,4.7341147E-3,1.1074469E6,2.8386528E-3,4.7767456E-3,6.6411955E-4,-1.1531069E-3,1.7143776E7,3.655004E7,2.8655008E2,-1.3367072E-2,3.64E2,1.3705882E1,1.4872362E-1,1.8863014E1,2.9355192E1,1.0124369E-2,1.3695652E0,-4.4659856E-3,-2.0337415E-4,1.0180435E-4,-2.7191702E-3,-8.4482925E-3,-2.1657597E-3,-4.703047E-3,-6.4426457E-4,-1.0158487E-2,-4.609284E-3,-9.525904E-4,2.254283E-3,-8.0058066E-4,-3.372192E-3,9.4284205E-4,5.790955E-3,1.5408425E-4,5.404589E-3,4.3060756E-3,1.0701922E-2,-4.781559E-3,2.3125742E-4,8.74795E-4,-3.619816E-3,1.08540235E-2,2.4840198E-3,3.1417976E-3,9.341854E-3,-3.8126172E-3,4.335523E-3,5.1972517E-3,1.2449378E-3],"split_indices":[2,43,67,68,69,60,62,6,0,2,6,32,0,46,73,67,69,4,62,71,0,68,3,0,1,68,47,44,0,6,0,0,62,73,0,0,0,0,52,67,7,0,0,71,48,0,0,2,2,0,106,5,68,43,68,2,2,48,43,0,62,53,48,0,0,8,0,67,10,0,0,7,0,2,0,0,0,0,7,4,4,19,53,9,69,0,12,0,0,0,0,0,0,0,0,0,0,0,53,0,68,0,3,10,0,0,0,4,10,5,73,69,0,8,0,3,62,0,0,73,62,2,0,0,0,68,1,0,0,1,67,0,0,2,0,60,0,0,0,0,5,7,4,0,10,71,57,73,73,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.032E3,7.86E2,2.46E2,2.53E2,5.33E2,1.63E2,8.3E1,2.48E2,5E0,5.21E2,1.2E1,1.56E2,7E0,3.6E1,4.7E1,2.08E2,4E1,2.37E2,2.84E2,8E0,4E0,1.12E2,4.4E1,6E0,3E1,3.6E1,1.1E1,2.04E2,4E0,3.6E1,4E0,6.6E1,1.71E2,2.79E2,5E0,4E0,4E0,4E0,1.08E2,2E1,2.4E1,9E0,2.1E1,8E0,2.8E1,4E0,7E0,1.9E2,1.4E1,1.7E1,1.9E1,5.4E1,1.2E1,1.4E1,1.57E2,8.8E1,1.91E2,9.9E1,9E0,1E1,1E1,1E1,1.4E1,4E0,4E0,9E0,1.9E1,1.08E2,8.2E1,8E0,6E0,1.3E1,6E0,5E1,4E0,8E0,4E0,5E0,9E0,5E1,1.07E2,4E1,4.8E1,1.79E2,1.2E1,7E0,9.2E1,5E0,4E0,6E0,4E0,6E0,4E0,4E0,1E1,4E0,5E0,1E1,9.8E1,1E1,7.2E1,5E0,8E0,4.2E1,8E0,4E0,5E0,2.3E1,2.7E1,9.2E1,1.5E1,3.3E1,7E0,4.3E1,5E0,8.1E1,9.8E1,7E0,5E0,4.6E1,4.6E1,6.5E1,3.3E1,6E0,4E0,5.9E1,1.3E1,4E0,4E0,1.9E1,2.3E1,1.1E1,1.2E1,2.1E1,6E0,8.6E1,6E0,6E0,9E0,6E0,2.7E1,3.1E1,1.2E1,4E0,7.7E1,6.3E1,3.5E1,2E1,2.6E1,3.3E1,1.3E1,5.2E1,1.3E1,1.8E1,4.1E1,9E0,4E0,7E0,1.2E1,8E0,1.5E1,5E0,1.6E1,2.7E1,5.9E1,1.3E1,1.4E1,4E0,2.7E1,4E0,8E0,1.9E1,5.8E1,4.9E1,1.4E1,4E0,3.1E1,1.1E1,9E0,5E0,2.1E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[1.7595345E-3,-4.604001E-2,9.6235365E-2,-1.1945923E-1,-1.0832919E-2,3.736891E-1,8.147172E-2,-9.8599575E-2,-2.7050975E-1,-6.886885E-3,-1.6953487E-2,8.473743E-3,1.9316973E-2,-1.4762089E-1,9.934878E-2,1.3500729E-1,-1.09373026E-1,-2.1028584E-2,-2.0486724E-1,-4.6014715E-2,1.782059E-2,-1.080661E-1,-1.1470033E-2,5.8647767E-2,1.4758804E-1,1.3014028E-2,-1.7870535E-3,-1.3100278E-1,-5.6821752E-2,-1.0959197E-2,-2.4903528E-3,-1.7599622E-2,-8.317143E-2,4.2048253E-2,-2.7355488E-2,-6.2773395E-3,-1.3967625E-4,6.88182E-2,-5.6689575E-2,1.5992443E-1,-3.3664703E-3,-1.1612602E-1,-2.1794435E-1,-1.2858361E-1,-1.6495831E-2,-9.081926E-2,4.7345733E-4,-6.795864E-2,-1.3194508E-2,1.1827146E-2,9.2539124E-2,-5.6345433E-2,3.4197066E-2,1.7180946E-1,4.736045E-2,2.1479174E-3,-6.4626858E-3,1.9397405E-1,9.4942704E-2,-1.5816578E-1,-7.565414E-2,-2.568005E-1,-3.0287288E-3,-2.9726156E-3,-7.6712733E-3,3.5225672E-3,-4.4709455E-2,-5.1574036E-2,-7.2225532E-3,1.642953E-2,-5.7172E-2,-1.4236178E-1,-3.4530643E-2,-6.482677E-3,2.0018592E-2,1.0792761E-1,-2.0332953E-2,-3.987955E-2,-1.4621301E-1,5.1721134E-3,-4.0996403E-2,1.9274902E-1,2.0934418E-3,-5.3314183E-2,6.4319156E-2,3.2232958E-1,1.6412465E-1,2.9443393E-2,1.5648606E-1,-2.4898688E-3,-1.8300876E-1,3.500623E-2,-1.07007146E-1,-1.3859208E-2,-6.8937135E-3,-6.735104E-2,2.4628008E-3,-4.575952E-3,-1.657172E-4,5.641105E-3,3.8313204E-3,-4.8381225E-3,7.0951093E-4,-7.486937E-2,-1.16012655E-2,-5.3010643E-2,1.7319338E-3,7.063823E-2,-1.6112518E-2,1.2887438E-1,2.989721E-2,-3.2878083E-3,1.5969548E-3,2.4510561E-3,-5.5810045E-2,-8.785496E-3,-2.9672084E-3,-7.760685E-2,1.8044072E-3,4.2414702E-3,9.705485E-3,-8.020313E-3,-5.9973323E-3,1.0236308E-1,2.6608733E-3,5.888042E-3,1.7629962E-2,8.491013E-2,2.1679874E-1,6.883945E-2,-4.6299775E-3,2.3849605E-4,1.8649268E-1,-2.0220388E-1,-2.432959E-3,-2.0935365E-3,4.6620397E-3,-1.6317834E-1,-6.973752E-2,-2.7008492E-2,-4.8713614E-3,3.6684677E-2,-2.1578815E-2,-1.1000487E-1,8.039836E-4,-2.8681986E-2,-5.0548567E-3,6.1379494E-3,4.7287244E-2,-7.388654E-2,2.96053E-2,4.853903E-2,1.5816322E-1,-2.2769286E-3,7.9363555E-2,-4.1126978E-2,-7.236221E-3,-5.611879E-3,-1.3361542E-3,-3.42289E-3,1.9650548E-3,1.2937905E-1,6.474979E-3,-2.7502578E-2,5.5280607E-2,7.678604E-3,4.2154018E-2,2.4456646E-1,6.6431584E-3,4.554645E-3,2.4831984E-2,1.19303465E-2,5.4118815E-3,-6.7189825E-3,-1.1882401E-2,-1.0256936E-2,-4.8764395E-3,1.4483294E-3,-3.9959946E-3,-3.5947496E-3,6.684414E-4,3.0958077E-3,-2.0195788E-3,1.06457E-3,-2.1839691E-3,-6.664084E-3,-1.8509475E-3,-2.122441E-3,7.712547E-4,1.1582002E-3,3.832561E-3,4.7430227E-4,-5.484657E-3,6.6732673E-4,5.716895E-3,-1.0076571E-3,5.6365225E-3,2.6749328E-3,8.013383E-3,6.4799343E-3,1.1980372E-4,-2.738094E-3,9.392785E-4,7.1157697E-3,3.6971763E-4,-5.3268857E-3,3.3406836E-3,-6.0323866E-5,-5.6981924E-3,-4.1920433E-4,5.579428E-3,4.774754E-3,-1.6400062E-3,8.504729E-3,1.454819E-2,-9.786241E-4,3.0436092E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,-1,29,31,33,35,-1,37,39,-1,-1,41,43,-1,-1,45,47,49,51,-1,-1,53,55,57,-1,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,-1,-1,85,87,89,91,93,-1,-1,-1,-1,95,97,-1,99,101,103,105,-1,107,109,111,113,115,-1,117,119,-1,121,123,125,127,129,131,-1,133,135,137,-1,-1,139,-1,-1,-1,141,-1,-1,-1,143,-1,145,-1,147,149,151,153,-1,-1,-1,155,-1,-1,157,-1,-1,-1,159,-1,161,163,-1,-1,165,167,169,-1,-1,171,173,-1,-1,-1,175,177,179,-1,181,183,185,-1,187,-1,-1,189,191,193,195,197,-1,199,201,-1,-1,-1,-1,-1,203,205,207,209,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5971026E0,1.7460828E0,1.3704648E0,6.6336656E-1,6.411064E-1,6.704068E-2,1.3498001E0,5.030161E-1,2.219162E-1,4.3972442E-1,0E0,0E0,0E0,6.872165E-2,5.8625054E-1,2.4019821E-1,2.0397711E-1,0E0,8.976817E-2,1.8435901E-1,3.065188E-1,5.509244E-2,0E0,1.9862425E-1,3.8696432E-1,0E0,0E0,1.441276E-1,1.5993524E-1,0E0,0E0,1.3369673E-1,2.146346E-1,2.766175E-1,1.7679854E-1,0E0,0E0,3.29987E-1,1.2668796E-1,2.7143478E-1,0E0,1.8199694E-1,8.9745104E-2,3.7450433E-2,1.0022424E-1,4.3361157E-2,7.636251E-2,1.7415881E-1,0E0,1.4613466E-1,1.2030578E-1,9.331225E-2,1.9168481E-1,6.0133874E-2,2.2196496E-1,0E0,0E0,2.727611E-1,1.848517E-1,1.326096E-1,2.1191806E-1,2.028513E-2,0E0,0E0,0E0,0E0,6.794954E-2,3.1539E-2,0E0,4.616598E-2,6.368832E-2,1.4267936E-1,6.9863915E-2,0E0,2.0232198E-1,9.421611E-2,2.757657E-2,8.882888E-2,2.382189E-2,0E0,5.2854706E-2,1.9199014E-2,0E0,6.479424E-2,2.5775716E-1,1.24649405E-1,2.7784264E-1,1.294945E-1,1.08014405E-1,0E0,9.943998E-2,7.868431E-2,8.70139E-2,0E0,0E0,3.4033477E-2,0E0,0E0,0E0,4.895681E-2,0E0,0E0,0E0,5.0544694E-2,0E0,5.3123437E-2,0E0,6.0551405E-2,1.7387208E-1,1.0291058E-1,5.8747545E-2,0E0,0E0,0E0,6.757656E-2,0E0,0E0,2.2113673E-2,0E0,0E0,0E0,4.6573278E-2,0E0,1.7531699E-1,6.976964E-2,0E0,0E0,9.7271815E-2,4.9973845E-2,2.2425406E-2,0E0,0E0,6.925994E-2,8.123171E-2,0E0,0E0,0E0,3.759569E-2,5.353117E-2,2.8518198E-2,0E0,6.87595E-2,3.6631614E-2,2.0564884E-2,0E0,2.4261696E-2,0E0,0E0,2.5657915E-2,1.10627115E-1,5.1019773E-2,7.501811E-2,4.2080402E-2,0E0,4.171247E-2,5.1374175E-2,0E0,0E0,0E0,0E0,0E0,1.5730798E-1,1.3661493E-1,7.013682E-2,6.852773E-2,0E0,1.03464484E-1,6.53888E-2,0E0,0E0,1.8395005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,27,27,28,28,31,31,32,32,33,33,34,34,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,66,66,67,67,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,80,80,81,81,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91,92,92,95,95,99,99,103,103,105,105,107,107,108,108,109,109,110,110,114,114,117,117,121,121,123,123,124,124,127,127,128,128,129,129,132,132,133,133,137,137,138,138,139,139,141,141,142,142,143,143,145,145,148,148,149,149,150,150,151,151,152,152,154,154,155,155,161,161,162,162,163,163,164,164,166,166,167,167,170,170],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,-1,30,32,34,36,-1,38,40,-1,-1,42,44,-1,-1,46,48,50,52,-1,-1,54,56,58,-1,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,-1,-1,86,88,90,92,94,-1,-1,-1,-1,96,98,-1,100,102,104,106,-1,108,110,112,114,116,-1,118,120,-1,122,124,126,128,130,132,-1,134,136,138,-1,-1,140,-1,-1,-1,142,-1,-1,-1,144,-1,146,-1,148,150,152,154,-1,-1,-1,156,-1,-1,158,-1,-1,-1,160,-1,162,164,-1,-1,166,168,170,-1,-1,172,174,-1,-1,-1,176,178,180,-1,182,184,186,-1,188,-1,-1,190,192,194,196,198,-1,200,202,-1,-1,-1,-1,-1,204,206,208,210,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.935E3,1.972052E5,3.6E1,2.6E1,5.6E1,2.00033E5,8.147158E4,6E0,5.6190475E1,6.99E2,-1.6953487E-2,8.473743E-3,1.9316973E-2,3.6672198E-8,5.709E3,1.89991E5,8.147158E4,-2.1028584E-2,4.4E1,1.53053E5,2E0,9.234473E-9,-1.1470033E-2,2.0319266E-7,7.06191E7,1.3014028E-2,-1.7870535E-3,2.7E1,1.463E2,-1.0959197E-2,-2.4903528E-3,6.613774E5,1.1949272E7,4.6066E4,9.750085E6,-6.2773395E-3,-1.3967625E-4,2.5959E4,1.1621469E1,2.5503825E6,-3.3664703E-3,3.8714287E1,6.593462E4,6.1E1,9.593451E4,3.1E1,3.3512E4,4.0832064E7,-1.3194508E-2,5.6591835E1,6.1114804E-8,2.8025E4,2.4E1,2.871795E0,5.9754E4,2.1479174E-3,-6.4626858E-3,2.5959E4,1.9616238E2,3.8E1,1.2E1,3.515484E1,-3.0287288E-3,-2.9726156E-3,-7.6712733E-3,3.5225672E-3,6.354E3,4.9E2,-7.2225532E-3,9E0,3.84E2,1.64388E5,1.8E7,-6.482677E-3,1E0,1.0988551E8,5E0,2.4243538E3,1.007E3,5.1721134E-3,1.594E3,2.297E3,2.0934418E-3,1.136E3,1.3922E4,1.4194314E3,8.439E3,1.3475722E2,2.234478E3,-2.4898688E-3,1.16E2,4.1142857E1,1.7467743E-2,-1.3859208E-2,-6.8937135E-3,4.8581E4,2.4628008E-3,-4.575952E-3,-1.657172E-4,7.619497E0,3.8313204E-3,-4.8381225E-3,7.0951093E-4,7.785302E5,-1.16012655E-2,9.776408E5,1.7319338E-3,3.63E2,2.026453E6,2.5912744E2,1.7728971E1,-3.2878083E-3,1.5969548E-3,2.4510561E-3,1.4172E4,-8.785496E-3,-2.9672084E-3,1.4260629E8,1.8044072E-3,4.2414702E-3,9.705485E-3,6.27E2,-5.9973323E-3,2.459733E0,3.2585382E10,5.888042E-3,1.7629962E-2,9.237895E0,1.7103828E7,1.752E3,-4.6299775E-3,2.3849605E-4,8.837831E-2,7.1E1,-2.432959E-3,-2.0935365E-3,4.6620397E-3,4.488263E0,1E0,1.4473684E-1,-4.8713614E-3,3.979933E0,3.6401E4,4.03E2,8.039836E-4,2.4332604E0,-5.0548567E-3,6.1379494E-3,1.0894737E1,2.1525E4,1.8002772E5,3E0,7.619497E0,-2.2769286E-3,1.7790684E5,3.19E2,-7.236221E-3,-5.611879E-3,-1.3361542E-3,-3.42289E-3,1.9650548E-3,4.797448E8,6.9508715E0,8.072743E4,2.3472653E-1,7.678604E-3,2.2007043E0,1.0494994E7,6.6431584E-3,4.554645E-3,2.0805957E-2,1.19303465E-2,5.4118815E-3,-6.7189825E-3,-1.1882401E-2,-1.0256936E-2,-4.8764395E-3,1.4483294E-3,-3.9959946E-3,-3.5947496E-3,6.684414E-4,3.0958077E-3,-2.0195788E-3,1.06457E-3,-2.1839691E-3,-6.664084E-3,-1.8509475E-3,-2.122441E-3,7.712547E-4,1.1582002E-3,3.832561E-3,4.7430227E-4,-5.484657E-3,6.6732673E-4,5.716895E-3,-1.0076571E-3,5.6365225E-3,2.6749328E-3,8.013383E-3,6.4799343E-3,1.1980372E-4,-2.738094E-3,9.392785E-4,7.1157697E-3,3.6971763E-4,-5.3268857E-3,3.3406836E-3,-6.0323866E-5,-5.6981924E-3,-4.1920433E-4,5.579428E-3,4.774754E-3,-1.6400062E-3,8.504729E-3,1.454819E-2,-9.786241E-4,3.0436092E-3],"split_indices":[2,43,10,3,3,5,43,3,62,2,0,0,0,52,2,9,43,0,3,9,32,52,0,52,59,0,0,8,70,0,0,60,60,1,9,0,0,9,71,47,0,70,43,10,43,10,1,7,0,62,52,44,8,68,1,0,0,9,71,44,33,73,0,0,0,0,10,1,0,8,0,62,5,0,26,7,8,62,2,0,2,2,0,44,44,67,2,73,4,0,44,67,53,0,0,1,0,0,0,73,0,0,0,43,0,62,0,44,9,4,73,0,0,0,44,0,0,7,0,0,0,44,0,53,46,0,0,71,62,0,0,0,72,44,0,0,0,69,21,71,0,69,9,2,0,71,0,0,73,9,48,8,73,0,48,0,0,0,0,0,0,7,71,48,53,0,69,62,0,0,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,6.75E2,3.41E2,2.18E2,4.57E2,1.6E1,3.25E2,1.93E2,2.5E1,4.53E2,4E0,4E0,1.2E1,2.3E1,3.02E2,8E0,1.85E2,5E0,2E1,1.75E2,2.78E2,1.8E1,5E0,1.65E2,1.37E2,4E0,4E0,1.3E2,5.5E1,1.6E1,4E0,1E2,7.5E1,1.81E2,9.7E1,1.4E1,4E0,1.52E2,1.3E1,1.3E2,7E0,1.13E2,1.7E1,1.9E1,3.6E1,1.9E1,8.1E1,7.1E1,4E0,1.14E2,6.7E1,6.6E1,3.1E1,2.5E1,1.27E2,6E0,7E0,8.4E1,4.6E1,5.4E1,5.9E1,1.3E1,4E0,8E0,1.1E1,8E0,2.8E1,1.3E1,6E0,6.4E1,1.7E1,2.1E1,5E1,5E0,1.09E2,5.9E1,8E0,5.7E1,9E0,1.5E1,1.6E1,2.1E1,4E0,1.8E1,1.09E2,1.4E1,7E1,2.3E1,2.3E1,1.1E1,4.3E1,1.3E1,4.6E1,8E0,5E0,2.3E1,5E0,6E0,7E0,5.6E1,8E0,1E1,7E0,1.4E1,7E0,4E1,1E1,4.5E1,6.4E1,4.6E1,1.3E1,4E0,4E0,8E0,4.9E1,5E0,4E0,1.1E1,5E0,4E0,1.7E1,1.2E1,6E0,6.7E1,4.2E1,4E0,1E1,2.9E1,4.1E1,1.8E1,5E0,4E0,1.9E1,3.7E1,6E0,6E0,7E0,1.7E1,2.9E1,1.2E1,1.1E1,2.6E1,3E1,1E1,4E0,2.9E1,1.1E1,1.1E1,3.4E1,2.8E1,3.6E1,1.3E1,3.3E1,5E0,8E0,4.4E1,5E0,5E0,6E0,5E0,7E0,5.2E1,1.5E1,2.7E1,1.5E1,9E0,2E1,2.8E1,1.3E1,1E1,8E0,8E0,1.1E1,2E1,1.7E1,7E0,1E1,4E0,2.5E1,5E0,7E0,1.9E1,7E0,1.1E1,1.9E1,6E0,4E0,2.1E1,8E0,2.2E1,1.2E1,1E1,1.8E1,3.2E1,4E0,7E0,6E0,5E0,2.8E1,4E0,4E0,3.4E1,1E1,4.3E1,9E0,5E0,1E1,2.2E1,5E0,8E0,7E0,1.1E1,9E0,1.7E1,1.1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-4.246676E-3,-6.808504E-2,5.8847517E-2,-1.24956235E-1,-3.197511E-2,7.25163E-2,-1.878514E-1,-1.0967272E-1,-2.89086E-1,-7.940078E-2,-1.0550625E-2,3.926635E-2,1.5051953E-1,-2.3990951E-1,-2.2365642E-4,-1.8054748E-1,-8.5415095E-2,-1.2362893E-1,-2.0285517E-2,-6.1006423E-2,-1.2663756E-2,-1.9150887E-2,1.4990428E-1,4.6509113E-2,-1.3270856E-1,1.8738891E-1,8.367987E-2,-1.4224235E-2,-8.497253E-3,-1.12483455E-2,-1.5302731E-1,3.4755883E-3,-9.085308E-2,-1.9021727E-3,-8.378598E-3,3.4661505E-2,-8.086697E-2,-7.2961594E-3,-1.2680723E-2,3.8711128E-3,9.816399E-3,3.8670145E-2,1.7892972E-1,-1.2233176E-3,-1.8878269E-1,1.6144902E-1,1.6803693E-2,1.1860967E-2,6.0304508E-2,-2.1567966E-3,-7.842399E-3,1.1266137E-2,-9.841681E-2,-2.2521977E-4,2.7197797E-3,-6.1837643E-2,-1.74032E-1,-1.715104E-2,7.461865E-3,1.2748428E-2,3.466672E-2,9.915375E-3,1.9959281E-3,-1.8064856E-3,-1.389236E-2,1.9271572E-1,1.03793584E-1,2.5398724E-2,1.1706977E-1,3.3249524E-3,-1.9033191E-3,-1.2271502E-1,-4.8634205E-2,-8.125813E-2,-4.508185E-3,-1.168239E-2,-6.0444773E-4,4.4353185E-3,-4.218284E-2,5.3904797E-3,6.13476E-2,1.6982117E-1,1.570477E-2,-3.8522992E-3,1.3461204E-1,-6.3506454E-2,5.9083413E-2,1.8047484E-3,1.6315807E-1,-9.565868E-2,-1.641413E-1,-9.196156E-2,-5.0656255E-3,-9.1702044E-2,1.1462504E-3,3.364774E-3,-4.2802643E-2,1.6971877E-2,-7.107972E-2,-6.7158826E-2,4.4043483E-3,-1.4942856E-1,1.9011347E-2,7.560417E-2,-8.150623E-2,2.0148331E-1,8.079732E-2,1.5145561E-1,1.3956367E-3,3.846139E-4,-5.665702E-3,1.752344E-2,1.02828525E-1,3.7424928E-3,9.171824E-3,-6.809631E-2,-1.6560681E-1,-2.2617683E-1,-5.223122E-3,-6.6056475E-2,-6.7318887E-3,1.8655114E-2,-4.1323737E-3,-6.2383365E-2,-1.3287278E-1,-1.1996457E-4,-3.8149245E-3,2.5052862E-2,-5.794073E-3,-4.846777E-3,-8.051778E-4,-9.425922E-2,-1.6991132E-3,-2.2909991E-2,5.1837415E-2,-1.8868674E-2,1.4377348E-3,-4.9310926E-2,3.409211E-2,1.0709327E-1,3.241865E-2,2.4693515E-3,-1.7741615E-1,7.040766E-3,1.1711209E-2,-1.824229E-4,5.9281304E-3,2.946607E-3,7.694757E-3,-9.253373E-4,3.7704797E-3,1.7401703E-3,6.9692265E-3,-4.020863E-3,2.170386E-4,-8.869812E-3,-3.6828346E-3,-5.5350126E-3,-1.2828617E-2,-4.4012032E-3,-6.8798754E-4,-7.7901117E-4,1.9768511E-3,-4.1316934E-3,-3.4449282E-4,-9.569709E-3,-4.346737E-3,-2.4460553E-4,2.775861E-3,-1.1375634E-3,-5.2239527E-3,-2.4104996E-3,1.2883787E-3,-2.7757033E-3,1.5397029E-3,5.1759507E-5,4.0610735E-3,-9.3171693E-4,-5.6071896E-3,1.0261538E-3,7.070875E-3,5.9179016E-3,4.5398576E-4,-2.8310283E-3,2.1788187E-3,-2.7921349E-3,-1.1961488E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,41,43,45,47,-1,-1,-1,49,-1,51,-1,-1,53,55,-1,57,-1,-1,59,61,-1,63,65,-1,-1,67,-1,-1,69,71,-1,-1,73,75,77,-1,-1,79,-1,-1,-1,-1,81,83,85,87,-1,-1,89,91,93,95,-1,-1,97,99,101,103,105,-1,-1,107,109,111,-1,113,115,117,119,121,123,-1,-1,125,127,129,131,133,135,137,139,141,143,145,147,-1,-1,-1,149,151,-1,-1,153,155,157,-1,159,-1,161,-1,163,165,-1,-1,167,-1,-1,-1,169,171,173,175,-1,-1,177,179,181,183,-1,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0761423E0,1.0289729E0,1.730008E0,4.5744872E-1,3.1312805E-1,1.2468004E0,2.5607204E-1,2.9404402E-1,3.4282207E-1,3.2711142E-1,2.9818097E-1,4.29667E-1,3.391838E-1,2.3290992E-2,0E0,4.0672302E-2,1.2564307E-1,3.537406E-2,0E0,1.719045E-1,0E0,1.8117213E-1,2.7703434E-2,3.3252007E-1,7.839517E-2,3.6209536E-1,1.9891614E-1,0E0,0E0,0E0,5.2546978E-2,0E0,1.0306585E-1,0E0,0E0,1.5601469E-2,1.2043044E-1,0E0,1.5539777E-1,0E0,0E0,2.8490737E-1,7.79292E-2,0E0,1.3901716E-1,1.2781835E-1,0E0,0E0,9.213552E-2,0E0,0E0,3.481302E-2,1.4281607E-1,0E0,0E0,6.9775745E-2,1.4895931E-1,1.0393341E-1,0E0,0E0,2.3869076E-1,0E0,0E0,0E0,0E0,1.1754084E-1,1.9004968E-1,9.658921E-2,5.8338553E-2,0E0,0E0,7.620764E-2,7.881805E-2,5.434078E-2,5.3466383E-2,0E0,0E0,9.9663034E-2,1.0385272E-1,3.1282997E-1,3.320192E-1,1.1619902E-1,0E0,0E0,4.2904586E-2,3.9247792E-2,4.0013142E-2,0E0,1.5880048E-2,8.897549E-2,7.2669566E-2,1.9616753E-2,4.570368E-2,4.3615013E-2,0E0,0E0,1.8974999E-2,1.0669867E-1,2.5397733E-2,1.027309E-1,4.2646885E-2,5.738407E-1,1.4180899E-1,1.9506961E-1,2.0363793E-1,4.816413E-2,5.6886256E-2,1.9828439E-2,0E0,0E0,0E0,3.2922707E-2,2.8740212E-2,0E0,0E0,5.417095E-2,1.6421348E-2,3.7951946E-2,0E0,2.2493325E-2,0E0,1.6151093E-2,0E0,3.9447434E-2,2.9479772E-2,0E0,0E0,9.174736E-2,0E0,0E0,0E0,5.0936073E-2,2.8337257E-2,4.5901388E-2,2.1851707E-2,0E0,0E0,5.0053746E-2,1.5489137E-1,1.703397E-1,8.9916594E-2,0E0,7.0151925E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26,30,30,32,32,35,35,36,36,38,38,41,41,42,42,44,44,45,45,48,48,51,51,52,52,55,55,56,56,57,57,60,60,65,65,66,66,67,67,68,68,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81,84,84,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,111,111,112,112,115,115,116,116,117,117,119,119,121,121,123,123,124,124,127,127,131,131,132,132,133,133,134,134,137,137,138,138,139,139,140,140,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,42,44,46,48,-1,-1,-1,50,-1,52,-1,-1,54,56,-1,58,-1,-1,60,62,-1,64,66,-1,-1,68,-1,-1,70,72,-1,-1,74,76,78,-1,-1,80,-1,-1,-1,-1,82,84,86,88,-1,-1,90,92,94,96,-1,-1,98,100,102,104,106,-1,-1,108,110,112,-1,114,116,118,120,122,124,-1,-1,126,128,130,132,134,136,138,140,142,144,146,148,-1,-1,-1,150,152,-1,-1,154,156,158,-1,160,-1,162,-1,164,166,-1,-1,168,-1,-1,-1,170,172,174,176,-1,-1,178,180,182,184,-1,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.071E3,1.972052E5,7.042942E7,4.6463413E0,1E0,1.087521E3,2.3308511E5,5.4E1,1.169375E2,3.7568388E0,2.0200005E8,2.1894855E1,6.6933334E1,1.1015E4,-2.2365642E-4,1.7857143E1,7E0,7.6811076E-3,-2.0285517E-2,6.728972E-1,-1.2663756E-2,2.35E2,7.4654E4,7.82261E6,3.4E1,2.1511E4,3.401282E1,-1.4224235E-2,-8.497253E-3,-1.12483455E-2,1.9363636E1,3.4755883E-3,5.7018555E4,-1.9021727E-3,-8.378598E-3,4.48E2,2.87E2,-7.2961594E-3,2.8034655E6,3.8711128E-3,9.816399E-3,2.3E1,1.3E1,-1.2233176E-3,2.4E1,2.3385885E0,1.6803693E-2,1.1860967E-2,7.802E-2,-2.1567966E-3,-7.842399E-3,1.16E2,1.339646E6,-2.2521977E-4,2.7197797E-3,5E0,1.409E3,1E0,7.461865E-3,1.2748428E-2,4.371613E2,9.915375E-3,1.9959281E-3,-1.8064856E-3,-1.389236E-2,3.478E4,1.5484655E6,3.3272727E0,1.14061164E5,3.3249524E-3,-1.9033191E-3,2.23607E5,1.901875E2,4.0119403E2,6.346204E-7,-1.168239E-2,-6.0444773E-4,1E0,3.011152E6,4E0,2.8827406E5,9.557399E6,1.570477E-2,-3.8522992E-3,2.5E1,1.6677892E2,7.073537E5,1.8047484E-3,2.101E3,1.2E1,2.135961E4,1.7559428E2,1.8E1,2.6686933E-2,1.1462504E-3,3.364774E-3,4.612245E0,1.2E1,2.988E4,4.831007E6,8.69E2,1.24E2,2.16577E5,2E0,2.67637E5,1.6E1,1.7026364E3,2.23462E5,1.3956367E-3,3.846139E-4,-5.665702E-3,8.25E2,3.3580637E8,3.7424928E-3,9.171824E-3,4.7181977E4,3.2216358E-1,4.6991256E7,-5.223122E-3,9.72E2,-6.7318887E-3,5.249058E5,-4.1323737E-3,4.2609915E6,1.48404255E1,-1.1996457E-4,-3.8149245E-3,5.4E1,-5.794073E-3,-4.846777E-3,-8.051778E-4,1.5132743E0,1.4E1,2.1818182E0,1.89965E4,-1.8868674E-2,1.4377348E-3,8.998703E-1,2.5176924E1,2E0,7.8461537E0,2.4693515E-3,1.2E2,7.040766E-3,1.1711209E-2,-1.824229E-4,5.9281304E-3,2.946607E-3,7.694757E-3,-9.253373E-4,3.7704797E-3,1.7401703E-3,6.9692265E-3,-4.020863E-3,2.170386E-4,-8.869812E-3,-3.6828346E-3,-5.5350126E-3,-1.2828617E-2,-4.4012032E-3,-6.8798754E-4,-7.7901117E-4,1.9768511E-3,-4.1316934E-3,-3.4449282E-4,-9.569709E-3,-4.346737E-3,-2.4460553E-4,2.775861E-3,-1.1375634E-3,-5.2239527E-3,-2.4104996E-3,1.2883787E-3,-2.7757033E-3,1.5397029E-3,5.1759507E-5,4.0610735E-3,-9.3171693E-4,-5.6071896E-3,1.0261538E-3,7.070875E-3,5.9179016E-3,4.5398576E-4,-2.8310283E-3,2.1788187E-3,-2.7921349E-3,-1.1961488E-2],"split_indices":[2,43,60,68,29,67,48,2,67,68,5,69,71,9,0,4,3,53,0,69,0,2,1,43,3,2,73,0,0,0,4,0,60,0,0,2,0,0,62,0,0,0,3,0,0,53,0,0,72,0,0,44,9,0,0,8,10,26,0,0,67,0,0,0,0,44,43,69,48,0,0,9,4,4,52,0,0,19,9,3,48,60,0,0,3,73,43,0,0,8,43,4,3,57,0,0,69,71,1,60,2,0,43,32,9,3,4,1,0,0,0,8,7,0,0,48,53,7,0,44,0,60,0,60,61,0,0,0,0,0,0,68,3,71,48,0,0,57,71,6,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,5.02E2,5.08E2,1.94E2,3.08E2,4.82E2,2.6E1,1.79E2,1.5E1,9.5E1,2.13E2,3.39E2,1.43E2,2E1,6E0,4.4E1,1.35E2,8E0,7E0,8.8E1,7E0,2.03E2,1E1,3.26E2,1.3E1,9.1E1,5.2E1,7E0,1.3E1,1.1E1,3.3E1,4E0,1.31E2,4E0,4E0,1.5E1,7.3E1,8E0,1.95E2,6E0,4E0,3.09E2,1.7E1,5E0,8E0,8.1E1,1E1,5E0,4.7E1,5E0,2.8E1,9E0,1.22E2,6E0,9E0,6.2E1,1.1E1,1.91E2,4E0,4E0,3.05E2,1.3E1,4E0,4E0,4E0,5.1E1,3E1,3E1,1.7E1,4E0,5E0,8.1E1,4.1E1,4.6E1,1.6E1,7E0,4E0,1.03E2,8.8E1,1.46E2,1.59E2,4.6E1,5E0,4E0,2.6E1,8E0,2.2E1,7E0,1E1,5.1E1,3E1,2E1,2.1E1,4.2E1,4E0,5E0,1.1E1,8.9E1,1.4E1,5.7E1,3.1E1,1.1E1,1.35E2,1.45E2,1.4E1,3.3E1,1.3E1,2.2E1,4E0,4E0,4E0,1.2E1,1E1,4E0,6E0,3.8E1,1.3E1,1.2E1,1.8E1,1.5E1,5E0,1.7E1,4E0,2.6E1,1.6E1,6E0,5E0,8.5E1,4E0,8E0,6E0,4E1,1.7E1,2E1,1.1E1,4E0,7E0,2.4E1,1.11E2,8.3E1,6.2E1,6E0,8E0,1.9E1,1.4E1,5E0,8E0,4E0,1.8E1,8E0,4E0,5E0,5E0,3E1,8E0,9E0,4E0,5E0,7E0,9E0,6E0,7E0,1E1,1.7E1,9E0,4E0,1.2E1,4.6E1,3.9E1,9E0,3.1E1,6E0,1.1E1,1.2E1,8E0,5E0,6E0,1.8E1,6E0,1.02E2,9E0,6.8E1,1.5E1,8E0,5.4E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[3.3297176E-3,-9.517032E-2,4.093917E-2,-7.589505E-2,-2.047397E-1,5.6218677E-3,1.35447E-1,-1.03723176E-1,3.9400604E-2,-1.5781309E-1,-2.3609966E-2,-3.5599433E-2,3.4445647E-2,2.018905E-2,1.2374736E-1,5.7373635E-2,-1.1288989E-1,1.1896653E-2,-1.1942283E-2,-1.7154962E-1,-1.6441952E-3,-2.5915174E-2,-2.3540896E-1,2.0166505E-2,1.0572814E-1,1.0986993E-1,1.3505884E-2,9.171345E-3,-2.3857206E-3,-1.0469012E-1,-1.2850202E-2,-4.5681067E-2,5.904921E-2,-1.9346522E-1,-9.161186E-2,-2.037825E-2,-9.737564E-3,-1.92913E-2,-2.050034E-3,2.4700554E-2,-8.195852E-3,7.24137E-2,9.736071E-3,1.1811017E-1,-5.0577535E-3,-1.2530367E-1,-5.4875832E-2,-4.7120946E-3,-5.9018005E-4,3.9337403E-3,1.2113429E-5,-3.963647E-3,-9.696643E-3,-1.2340919E-3,-6.384194E-3,1.3450192E-2,-3.5481364E-2,6.465759E-2,-4.6955356E-3,3.9165895E-2,1.2743217E-1,-3.98957E-3,1.247095E-1,-1.06488764E-1,-2.0684081E-1,-2.6475715E-2,-9.369074E-2,-2.8757644E-3,2.5149453E-2,-4.293659E-3,2.8145934E-2,-5.223333E-2,7.059656E-3,7.6398745E-2,-2.0471564E-2,-1.5931932E-2,1.3771853E-1,-1.8382893E-3,5.462749E-2,2.4348241E-3,8.603761E-3,1.4945285E-1,6.6638686E-2,-1.3527343E-1,-2.5583832E-2,-2.692773E-1,-4.0070787E-3,-4.8865356E-2,2.766803E-2,-8.288341E-3,-5.7927426E-2,3.6776708E-3,-5.5572303E-4,4.2710274E-2,-3.0675915E-3,-4.1916486E-2,-7.771451E-3,4.3633394E-3,-4.9401806E-3,6.3400075E-2,1.8184727E-1,-5.6452624E-2,2.567968E-3,-3.488993E-2,6.400086E-2,3.202812E-3,9.513486E-3,7.452817E-2,-6.550628E-4,1.7600492E-1,8.962082E-2,3.4909096E-2,1.9641206E-1,-1.5257972E-1,-7.524455E-3,6.721595E-2,-7.30793E-2,-1.6596582E-2,-9.118207E-3,-3.8823816E-3,-9.356743E-4,3.4212247E-3,-1.5489962E-3,-1.2179071E-3,-5.1467298E-3,2.8590102E-2,5.791706E-3,-1.2005941E-2,-8.84788E-2,-3.2254998E-3,5.03941E-3,4.7472272E-2,1.311127E-1,1.0810228E-2,3.8300692E-3,-5.5680884E-4,-4.5479448E-3,2.7110377E-2,-5.971857E-2,1.0920867E-1,6.9148587E-3,1.4481733E-3,5.27718E-3,1.3534814E-1,1.201341E-2,2.2169113E-2,1.1989972E-1,5.615528E-3,8.921606E-2,1.2231885E-2,3.469839E-3,-3.802598E-3,-7.929313E-3,2.7879747E-3,-2.9026833E-3,4.999125E-3,2.1167596E-4,-1.7738995E-3,-7.50784E-3,8.6023175E-4,3.6325655E-3,-2.8788622E-3,3.031062E-4,-8.70642E-3,-1.7986735E-3,2.8219619E-3,-2.2161666E-4,7.038843E-4,3.240193E-3,9.038444E-3,3.8805315E-3,-2.0097329E-3,4.4410056E-3,1.2077744E-3,-3.9940276E-3,1.1817028E-3,6.3428674E-3,-1.050409E-3,2.0927086E-3,5.188983E-3,1.0133847E-2,-1.5733469E-3,4.3168785E-3,6.6986037E-3,1.4355014E-3,-2.237358E-3,1.4948554E-3,9.953913E-4,6.7327907E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,45,-1,47,49,51,53,55,-1,-1,-1,57,-1,59,-1,61,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,69,71,73,75,77,79,-1,81,83,85,87,89,-1,91,-1,93,95,97,99,101,103,105,-1,107,-1,-1,109,111,113,115,117,-1,119,121,-1,123,-1,-1,125,-1,127,-1,-1,129,131,133,135,-1,137,139,-1,-1,141,-1,143,145,147,149,151,153,155,157,-1,-1,-1,-1,-1,-1,-1,-1,159,-1,161,163,-1,165,167,169,-1,-1,-1,-1,171,173,175,177,-1,-1,179,-1,181,183,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7269514E0,5.7159185E-1,2.4291286E0,7.677009E-1,5.086857E-1,6.3213587E-1,6.444812E-1,2.9000854E-1,5.293951E-1,5.6015015E-2,0E0,4.1766602E-1,3.1645578E-1,0E0,4.1709304E-1,1.7950486E-1,2.1750617E-1,0E0,9.610893E-2,4.0088475E-2,0E0,2.1065018E-1,3.2946062E-1,2.39079E-1,1.6665989E-1,3.3646822E-1,0E0,0E0,0E0,1.7282248E-1,0E0,6.8500035E-2,1.9922905E-2,2.6199758E-2,2.351398E-2,1.05214775E-1,0E0,0E0,0E0,3.0217084E-1,0E0,7.000798E-2,0E0,2.4420547E-1,0E0,1.6666985E-1,5.594392E-2,0E0,2.6957976E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0267708E-1,1.0156444E-1,1.10511065E-1,2.4062705E-1,3.51351E-2,5.609648E-2,0E0,2.3143435E-1,2.3407888E-1,1.4162177E-1,4.0144295E-2,5.704689E-2,0E0,2.5252316E-2,0E0,8.132444E-2,1.1500034E-1,4.346033E-2,1.20057166E-1,4.1450188E-2,2.128688E-1,3.7035853E-2,0E0,3.2550618E-2,0E0,0E0,1.7009997E-1,2.0726153E-1,1.6466653E-1,1.2832405E-1,1.9666433E-2,0E0,2.1443143E-2,3.086966E-2,0E0,2.5758862E-2,0E0,0E0,5.47702E-2,0E0,1.3124464E-1,0E0,0E0,2.4307435E-2,8.760643E-2,3.219551E-2,1.7457087E-2,0E0,1.7556931E-1,6.8924636E-2,0E0,0E0,2.727621E-2,0E0,2.3811865E-1,7.495177E-2,6.790532E-2,6.303039E-2,7.424271E-2,4.1328244E-2,2.5179472E-2,5.269234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0486768E-2,0E0,5.5518754E-2,1.7593735E-1,0E0,1.848474E-2,5.1533163E-2,3.2213032E-2,0E0,0E0,0E0,0E0,1.650054E-1,1.8818924E-1,3.172031E-2,1.5825287E-2,0E0,0E0,8.370173E-2,0E0,5.5276446E-2,5.345133E-2,4.347816E-2,5.4229297E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,19,19,21,21,22,22,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,39,39,41,41,43,43,45,45,46,46,48,48,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,68,68,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,81,81,82,82,83,83,84,84,85,85,87,87,88,88,90,90,93,93,95,95,98,98,99,99,100,100,101,101,103,103,104,104,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,125,125,127,127,128,128,130,130,131,131,132,132,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,148,148],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,46,-1,48,50,52,54,56,-1,-1,-1,58,-1,60,-1,62,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,70,72,74,76,78,80,-1,82,84,86,88,90,-1,92,-1,94,96,98,100,102,104,106,-1,108,-1,-1,110,112,114,116,118,-1,120,122,-1,124,-1,-1,126,-1,128,-1,-1,130,132,134,136,-1,138,140,-1,-1,142,-1,144,146,148,150,152,154,156,158,-1,-1,-1,-1,-1,-1,-1,-1,160,-1,162,164,-1,166,168,170,-1,-1,-1,-1,172,174,176,178,-1,-1,180,-1,182,184,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0976269E5,1.0657745E7,3.812E3,2.9994638E2,3.7798166E0,8.1E2,2.4481E4,7E0,1.176582E6,1.4916515E3,-2.3609966E-2,3.6E1,2.8530578E6,2.018905E-2,8.2471845E6,1.3050649E4,3.0977E4,1.1896653E-2,6.3351805E4,8.579633E-2,-1.6441952E-3,1E0,2.6676828E2,1.4676277E7,2.0833333E0,7.134432E7,1.3505884E-2,9.171345E-3,-2.3857206E-3,1.339646E6,-1.2850202E-2,8.147158E4,3.1095755E0,1.5933E4,2.4203823E0,2.6654E4,-9.737564E-3,-1.92913E-2,-2.050034E-3,1.7903225E0,-8.195852E-3,6E0,9.736071E-3,1.55E2,-5.0577535E-3,1.2E1,2.41E2,-4.7120946E-3,1.6E1,3.9337403E-3,1.2113429E-5,-3.963647E-3,-9.696643E-3,-1.2340919E-3,-6.384194E-3,1.62E2,6.887749E7,6.1114804E-8,8.7115955E2,2.02E4,2.742234E6,-3.98957E-3,8.7E1,2.55E2,7.709291E7,1.4276666E2,3.72E2,-2.8757644E-3,2.670362E7,-4.293659E-3,1.9E1,6.763314E7,2.4915715E-3,1.47176E5,6.6414773E-1,9.751103E6,1.7E1,-1.8382893E-3,2.682E3,2.4348241E-3,8.603761E-3,1.7103828E7,3.8537518E2,2.4E1,2.855464E5,1.746E3,-4.0070787E-3,1.33162E6,1.8E1,-8.288341E-3,6.642857E0,3.6776708E-3,-5.5572303E-4,5.6530495E6,-3.0675915E-3,1E0,-7.771451E-3,4.3633394E-3,3.000176E5,6.083812E2,1.72797E5,2.0211798E6,2.567968E-3,1.5123151E6,2.5417458E1,3.202812E-3,9.513486E-3,6.009E4,-6.550628E-4,1.4916515E3,5.97E3,1.4176E4,6.880842E-1,2.416E3,1E0,1.5E1,1.441215E5,-1.6596582E-2,-9.118207E-3,-3.8823816E-3,-9.356743E-4,3.4212247E-3,-1.5489962E-3,-1.2179071E-3,-5.1467298E-3,3.0751158E4,5.791706E-3,7.117E3,2.019537E2,-3.2254998E-3,3.617263E6,1.411E3,1.9197379E9,1.0810228E-2,3.8300692E-3,-5.5680884E-4,-4.5479448E-3,4.0911578E2,1E0,6.1786E4,5.460753E9,1.4481733E-3,5.27718E-3,4.9833015E1,1.201341E-2,3.1991906E0,9.0754684E1,2.453E4,1.9428571E0,1.2231885E-2,3.469839E-3,-3.802598E-3,-7.929313E-3,2.7879747E-3,-2.9026833E-3,4.999125E-3,2.1167596E-4,-1.7738995E-3,-7.50784E-3,8.6023175E-4,3.6325655E-3,-2.8788622E-3,3.031062E-4,-8.70642E-3,-1.7986735E-3,2.8219619E-3,-2.2161666E-4,7.038843E-4,3.240193E-3,9.038444E-3,3.8805315E-3,-2.0097329E-3,4.4410056E-3,1.2077744E-3,-3.9940276E-3,1.1817028E-3,6.3428674E-3,-1.050409E-3,2.0927086E-3,5.188983E-3,1.0133847E-2,-1.5733469E-3,4.3168785E-3,6.6986037E-3,1.4355014E-3,-2.237358E-3,1.4948554E-3,9.953913E-4,6.7327907E-3],"split_indices":[43,60,2,67,68,2,12,3,5,4,0,3,43,0,43,43,44,0,48,53,0,106,4,62,68,59,0,0,0,9,0,43,53,1,69,9,0,0,0,68,0,8,0,10,0,8,10,0,8,0,0,0,0,0,0,2,7,52,67,12,9,0,8,2,7,67,2,0,5,0,8,7,53,1,53,9,3,0,10,0,0,62,71,3,60,44,0,5,3,0,69,0,0,43,0,26,0,0,43,4,1,43,0,60,73,0,0,1,0,4,2,2,57,9,20,3,43,0,0,0,0,0,0,0,0,48,0,1,4,0,5,44,46,0,0,0,0,4,8,1,46,0,0,73,0,53,71,44,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.004E3,2.77E2,7.27E2,2.37E2,4E1,5.3E2,1.97E2,1.91E2,4.6E1,3.6E1,4E0,2.18E2,3.12E2,6E0,1.91E2,1E1,1.81E2,8E0,3.8E1,3.2E1,4E0,2.09E2,9E0,2.61E2,5.1E1,1.78E2,1.3E1,4E0,6E0,1.74E2,7E0,2.6E1,1.2E1,2.4E1,8E0,2.04E2,5E0,4E0,5E0,2.56E2,5E0,4E1,1.1E1,1.72E2,6E0,1.22E2,5.2E1,1.1E1,1.5E1,8E0,4E0,4E0,2E1,4E0,4E0,6.3E1,1.41E2,1.08E2,1.48E2,2.6E1,1.4E1,5E0,1.67E2,1.01E2,2.1E1,3.1E1,2.1E1,4E0,1.1E1,7E0,5.6E1,1.01E2,4E1,9.5E1,1.3E1,1.38E2,1E1,4E0,2.2E1,7E0,7E0,1.16E2,5.1E1,7.4E1,2.7E1,1.3E1,8E0,2.2E1,9E0,5E0,1.6E1,4E0,7E0,4.9E1,7E0,9.4E1,7E0,4E0,3.6E1,8.6E1,9E0,9E0,4E0,1.12E2,2.6E1,6E0,4E0,1.7E1,5E0,7.9E1,3.7E1,4.2E1,9E0,6.5E1,9E0,9E0,1.8E1,4E0,9E0,9E0,1.3E1,5E0,4E0,1.1E1,5E0,4.3E1,6E0,5.8E1,3.6E1,4E0,3.2E1,7.1E1,1.5E1,5E0,4E0,5E0,4E0,3.2E1,8E1,1.4E1,1.2E1,9E0,8E0,5.5E1,2.4E1,1.2E1,2.5E1,2.8E1,1.4E1,5E0,4E0,1.5E1,5E1,4E0,5E0,5E0,4E0,1.4E1,4E0,3.7E1,6E0,1.5E1,4.3E1,1.1E1,2.5E1,4E0,2.8E1,3E1,4.1E1,5E0,1E1,1.6E1,1.6E1,1.9E1,6.1E1,4E0,1E1,7E0,5E0,4.5E1,1E1,7E0,5E0,1.9E1,6E0,9E0,1.9E1,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[-2.5421267E-3,-4.54704E-2,7.471239E-2,-1.09337635E-1,-1.33620575E-2,1.0278751E-1,-8.404298E-2,-1.0111751E-1,-1.7944142E-2,-8.394736E-3,-2.2760764E-1,1.9427637E-2,9.310652E-2,-1.9356988E-2,-5.022023E-2,-8.435361E-2,-2.1126117E-1,-1.332153E-2,1.1324026E-1,-1.1398182E-3,-1.966273E-2,2.6305959E-2,1.217016E-1,-7.619313E-2,3.8039568E-3,-1.5917991E-1,-5.500472E-2,-2.5255677E-1,-3.532036E-3,-5.942694E-2,3.542078E-3,1.3865676E-2,9.241316E-3,-6.367484E-3,1.1543242E-1,1.5685134E-1,5.840395E-2,-2.7696842E-2,-1.5158741E-1,-7.2341606E-2,-1.9477642E-1,7.969432E-2,-7.33234E-2,-1.9888097E-1,-1.6335959E-2,-4.9246356E-2,-1.2355577E-2,1.2362886E-2,-8.839086E-2,2.4679608E-3,-1.3149148E-3,1.1127053E-2,-5.1141405E-3,2.3229497E-2,8.101115E-3,1.3353586E-1,1.3239269E-2,7.806513E-2,-8.309853E-2,-6.772005E-2,4.7144773E-3,-9.600239E-3,-6.4190663E-4,7.998607E-4,-1.1114062E-1,-1.215757E-2,-1.6683866E-1,8.65128E-3,-6.619609E-4,-4.2575836E-2,-9.408454E-2,-4.4017024E-3,-1.0416772E-2,-2.1093674E-2,-9.473748E-2,1.7409757E-2,-5.095121E-2,2.4142342E-3,-1.1512344E-1,-5.0133485E-2,3.266131E-2,3.584761E-3,-2.6576682E-3,1.8481413E-1,8.088663E-2,4.8281103E-2,1.9162512E-1,-8.544575E-3,2.5753442E-3,1.9589511E-3,-1.02603264E-1,-1.3831886E-3,-6.837394E-3,-8.4553035E-3,-2.4286485E-3,-2.4226228E-2,-1.0709931E-1,-7.313269E-2,-1.5756802E-1,-6.6714636E-3,-5.3081783E-3,-3.9686467E-2,-1.4527717E-1,7.143194E-3,4.8257496E-2,-9.067044E-2,1.1515733E-3,-2.9145833E-4,-7.0757326E-3,5.572348E-4,-6.090037E-3,8.611992E-2,1.2832938E-2,2.6268777E-1,1.3486584E-1,-7.571906E-3,1.06119685E-1,-3.2952398E-2,6.665692E-2,5.6689526E-3,1.2153334E-2,2.3366521E-4,-1.6178727E-1,-4.07812E-2,2.2711768E-3,-7.0660263E-3,-9.92913E-4,-1.2123521E-1,-4.080731E-2,-8.279315E-3,-3.224055E-3,-5.5296108E-2,2.0928886E-2,9.3635026E-4,-6.501933E-2,-1.8784513E-1,-3.487928E-4,-4.1923635E-2,1.800284E-2,8.1211045E-2,7.104659E-3,-5.814261E-3,-1.7329064E-3,1.084086E-3,6.6925497E-3,2.4810946E-3,-1.8613518E-4,1.3753464E-2,5.530243E-3,-1.822666E-3,1.5653424E-1,1.287163E-1,3.0749848E-2,3.8311942E-4,-3.88558E-3,1.2206044E-1,1.0784502E-2,-3.452554E-3,-1.1610145E-2,2.3218493E-3,-2.5222346E-3,-6.790782E-3,-1.2962726E-3,-5.549409E-3,-1.2334522E-3,-5.8116326E-3,-1.2548242E-3,-2.4443984E-3,2.2274922E-3,-1.0832128E-3,-4.5489715E-3,-1.0776526E-2,-3.3850789E-3,2.2743465E-3,-3.4247932E-3,1.0696541E-3,-4.5897663E-3,2.7066993E-3,7.071643E-3,-1.6484723E-3,2.4148275E-3,9.705331E-3,3.842353E-3,4.059716E-3,8.01699E-3,4.703545E-3,-9.25853E-4,7.4518058E-3,3.941479E-3,-1.1742665E-3,4.7277724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,35,37,-1,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,-1,-1,79,-1,81,-1,83,-1,85,87,89,-1,-1,-1,-1,91,-1,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,107,109,111,-1,-1,113,115,117,119,-1,-1,-1,121,-1,-1,-1,-1,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,-1,-1,-1,143,145,147,149,-1,151,153,155,-1,-1,-1,157,159,-1,-1,-1,161,163,-1,-1,165,167,-1,169,171,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,183,185,-1,-1,187,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.425925E0,1.3601847E0,1.6532998E0,4.699943E-1,4.6954185E-1,9.2670035E-1,6.133747E-1,3.8285613E-1,0E0,2.61592E-1,3.9131492E-1,0E0,5.8130956E-1,0E0,1.853061E-1,4.0849745E-1,1.3656747E-1,3.25207E-1,1.4385058E-1,0E0,0E0,2.7155736E-1,4.6696687E-1,1.5668058E-1,0E0,1.4973998E-1,3.4635958E-1,4.357028E-2,0E0,2.2401384E-1,2.5004607E-1,1.6588666E-2,0E0,1.2717147E-1,1.3404658E-1,3.7608337E-1,2.2349128E-1,1.523081E-1,1.2869799E-1,6.1059132E-2,2.6779413E-2,1.7461634E-1,7.387358E-2,2.2602916E-2,0E0,1.363169E-1,0E0,9.067723E-2,1.0753298E-1,0E0,0E0,8.078943E-2,0E0,5.2756384E-2,0E0,3.0362272E-1,0E0,2.2412497E-1,1.5172553E-1,8.8862E-2,0E0,0E0,0E0,0E0,3.1483725E-2,0E0,4.3997407E-2,0E0,0E0,5.77884E-2,8.4243536E-2,0E0,0E0,9.122833E-2,1.0830721E-1,8.231406E-2,6.5549836E-2,0E0,9.422311E-2,8.287967E-2,4.6594705E-2,0E0,0E0,1.9406223E-1,3.8799167E-1,8.578949E-2,3.664857E-2,0E0,0E0,0E0,1.09139115E-1,0E0,0E0,0E0,0E0,5.1816482E-2,3.9887086E-2,8.212891E-2,1.9789249E-2,8.1680715E-2,0E0,3.321284E-2,1.2170005E-1,1.0572588E-1,8.757159E-2,2.146358E-2,0E0,0E0,0E0,0E0,0E0,4.0767103E-2,2.3648227E-2,7.4499846E-2,1.4970922E-1,0E0,8.971262E-2,2.3905348E-2,1.41078E-1,0E0,0E0,0E0,6.694397E-2,4.5501724E-2,0E0,0E0,0E0,4.8669428E-2,3.595761E-2,0E0,0E0,4.0220015E-2,8.1127815E-2,0E0,1.8547483E-2,6.6167295E-2,0E0,1.09338894E-1,1.00211866E-1,4.9555868E-2,5.9839338E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1608368E-1,6.1625123E-2,5.256903E-2,0E0,0E0,1.939997E-2,8.214453E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,51,51,53,53,55,55,57,57,58,58,59,59,64,64,66,66,69,69,70,70,73,73,74,74,75,75,76,76,78,78,79,79,80,80,83,83,84,84,85,85,86,86,90,90,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,111,111,112,112,113,113,114,114,116,116,117,117,118,118,122,122,123,123,127,127,128,128,131,131,132,132,134,134,135,135,137,137,138,138,139,139,140,140,150,150,151,151,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,36,38,-1,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,-1,-1,80,-1,82,-1,84,-1,86,88,90,-1,-1,-1,-1,92,-1,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,108,110,112,-1,-1,114,116,118,120,-1,-1,-1,122,-1,-1,-1,-1,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,-1,-1,-1,144,146,148,150,-1,152,154,156,-1,-1,-1,158,160,-1,-1,-1,162,164,-1,-1,166,168,-1,170,172,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,184,186,-1,-1,188,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.8988005E5,1E0,6.9934395E-5,6.0692043E0,3.3817584E7,7.87E3,1.977157E7,-1.7944142E-2,7.82261E6,1.3586957E1,1.9427637E-2,5.817547E2,-1.9356988E-2,3.2462872E5,4.9E2,3.1126543E1,2.1222334E2,1.9186046E0,-1.1398182E-3,-1.966273E-2,7.127857E6,5.489183E1,4.01969E5,3.8039568E-3,5.3E1,9.83871E-1,3.2E1,-3.532036E-3,3.4E1,1.465E4,5.51E2,9.241316E-3,1.8E1,1E0,1.4176E4,3.206931E2,4.27E3,1.3026532E6,1E0,1.7857143E1,1.32E2,1.0142858E1,7.0093E4,-1.6335959E-2,1E0,-1.2355577E-2,2E0,2.5818555E2,2.4679608E-3,-1.3149148E-3,5.9754E4,-5.1141405E-3,1.1544592E1,8.101115E-3,3.72381E5,1.3239269E-2,3.8537518E2,6.9879E4,1E0,4.7144773E-3,-9.600239E-3,-6.4190663E-4,7.998607E-4,2.3E1,-1.215757E-2,1.76421E6,8.65128E-3,-6.619609E-4,2.0303884E6,3.1316226E0,-4.4017024E-3,-1.0416772E-2,1E0,1.703125E0,4.5432812E2,9E0,2.4142342E-3,1.9E1,1.136E3,1.1283241E6,3.584761E-3,-2.6576682E-3,2.124E3,6.27907E0,3.808001E5,4.43837E0,-8.544575E-3,2.5753442E-3,1.9589511E-3,2.511E3,-1.3831886E-3,-6.837394E-3,-8.4553035E-3,-2.4286485E-3,1.6580646E2,9.14E2,3.95E2,2.897656E8,8.038E3,-5.3081783E-3,8.454545E0,9.5E2,3.990487E5,2E0,7.89E2,1.1515733E-3,-2.9145833E-4,-7.0757326E-3,5.572348E-4,-6.090037E-3,5.9653606E5,9.47E4,1.3587301E1,2.9355192E1,-7.571906E-3,2.3922667E1,7.25723E2,8.710612E1,5.6689526E-3,1.2153334E-2,2.3366521E-4,1.3711089E6,2.6412E5,2.2711768E-3,-7.0660263E-3,-9.92913E-4,2.8461537E0,7.7E1,-8.279315E-3,-3.224055E-3,1.3E1,1.3020051E-5,9.3635026E-4,2.9016996E4,1.8149019E2,-3.487928E-4,1.5132743E0,1E0,2.0277777E0,1.404E3,-5.814261E-3,-1.7329064E-3,1.084086E-3,6.6925497E-3,2.4810946E-3,-1.8613518E-4,1.3753464E-2,5.530243E-3,-1.822666E-3,6E0,1.2E1,2.83E3,3.8311942E-4,-3.88558E-3,1.03339244E6,3.002566E0,-3.452554E-3,-1.1610145E-2,2.3218493E-3,-2.5222346E-3,-6.790782E-3,-1.2962726E-3,-5.549409E-3,-1.2334522E-3,-5.8116326E-3,-1.2548242E-3,-2.4443984E-3,2.2274922E-3,-1.0832128E-3,-4.5489715E-3,-1.0776526E-2,-3.3850789E-3,2.2743465E-3,-3.4247932E-3,1.0696541E-3,-4.5897663E-3,2.7066993E-3,7.071643E-3,-1.6484723E-3,2.4148275E-3,9.705331E-3,3.842353E-3,4.059716E-3,8.01699E-3,4.703545E-3,-9.25853E-4,7.4518058E-3,3.941479E-3,-1.1742665E-3,4.7277724E-3],"split_indices":[2,43,6,52,68,7,9,5,0,43,73,0,67,0,48,1,73,4,69,0,0,60,71,1,0,44,68,3,0,3,44,2,0,3,23,2,73,2,43,24,4,12,73,1,0,26,0,6,4,0,0,1,0,71,0,9,0,71,2,8,0,0,0,0,44,0,46,0,0,60,68,0,0,19,68,4,8,0,8,44,60,0,0,44,71,43,50,0,0,0,2,0,0,0,0,67,2,2,46,1,0,73,10,43,32,2,0,0,0,0,0,60,9,71,73,0,71,67,73,0,0,0,43,7,0,0,0,68,10,0,0,3,53,0,48,70,0,68,79,68,2,0,0,0,0,0,0,0,0,0,8,8,2,0,0,43,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.031E3,6.63E2,3.68E2,2.21E2,4.42E2,3.13E2,5.5E1,2.16E2,5E0,4.33E2,9E0,8E0,3.05E2,4E0,5.1E1,1.89E2,2.7E1,4.17E2,1.6E1,5E0,4E0,9.2E1,2.13E2,4.3E1,8E0,5.2E1,1.37E2,2E1,7E0,1.11E2,3.06E2,8E0,8E0,6.8E1,2.4E1,1.36E2,7.7E1,2.7E1,1.6E1,1.6E1,3.6E1,1.6E1,1.21E2,1.5E1,5E0,1.07E2,4E0,2.8E2,2.6E1,4E0,4E0,5.9E1,9E0,1E1,1.4E1,1.17E2,1.9E1,6.8E1,9E0,2.1E1,6E0,1.1E1,5E0,5E0,1.1E1,8E0,2.8E1,7E0,9E0,5E1,7.1E1,4E0,1.1E1,6.7E1,4E1,2.6E2,2E1,4E0,2.2E1,1.5E1,4.4E1,6E0,4E0,5.8E1,5.9E1,5.5E1,1.3E1,5E0,4E0,5E0,1.6E1,4E0,7E0,2.4E1,4E0,4E1,1E1,5.5E1,1.6E1,5.9E1,8E0,2E1,2E1,1.96E2,6.4E1,1.3E1,7E0,6E0,1.6E1,9E0,6E0,1.1E1,3.3E1,2.1E1,3.7E1,5E0,5.4E1,1E1,4.5E1,8E0,5E0,6E0,1E1,3.3E1,7E0,6E0,4E0,2.1E1,3.4E1,1.2E1,4E0,2.1E1,3.8E1,6E0,1.4E1,1.5E1,5E0,3.5E1,1.61E2,3.5E1,2.9E1,7E0,6E0,6E0,5E0,9E0,2.4E1,1.6E1,5E0,4E0,3.3E1,4.1E1,1.3E1,6E0,4E0,2.2E1,2.3E1,6E0,4E0,4E0,2.9E1,1.6E1,5E0,4E0,3E1,5E0,1.6E1,1E1,2.8E1,7E0,7E0,1E1,5E0,9E0,2.6E1,1.55E2,6E0,2.8E1,7E0,1.5E1,1.4E1,1.8E1,1.5E1,2.3E1,1.8E1,5E0,8E0,9E0,1.3E1,1.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-1.6768986E-3,-3.4966692E-2,9.1516346E-2,-6.739187E-2,1.5456984E-2,1.0763499E-1,-1.07800454E-1,-5.2370828E-2,-1.547599E-1,1.9478336E-2,-1.0464207E-2,2.0494487E-2,9.771774E-2,-8.86112E-3,3.7827522E-3,-1.3803354E-1,-3.8602564E-2,-9.705497E-2,-3.9143035E-1,7.961811E-3,9.7763024E-2,7.964134E-2,2.091503E-1,3.285471E-3,-3.5930462E-3,-7.501039E-2,-2.0544358E-1,1.0062904E-1,-4.6550117E-2,-1.309202E-1,4.237979E-3,-2.515075E-2,-1.7462966E-3,4.1632973E-2,-1.2664159E-2,-2.2505E-3,1.1642825E-1,5.4190245E-2,1.388433E-1,2.5647342E-1,1.0101846E-1,-6.518899E-3,-4.8759773E-2,-3.2511433E-3,-1.0478092E-2,1.1660193E-2,4.3333333E-2,-3.5339423E-2,-1.3117544E-1,-9.181514E-2,-8.221557E-3,-2.4021226E-3,2.194635E-3,1.4204666E-2,7.7023216E-2,8.0237497E-4,-1.4468968E-1,1.4548971E-1,1.7520034E-4,1.1522659E-2,4.4459067E-2,2.0912269E-1,7.068141E-2,1.3075406E-2,4.8180684E-3,2.2034561E-3,6.7632147E-3,6.2657185E-5,-7.07703E-2,-2.8989706E-3,4.047658E-3,-3.1303614E-2,-1.2008373E-2,-1.7737614E-1,2.968251E-3,4.992811E-4,-1.1801108E-1,2.9725919E-2,-5.6677135E-3,8.975173E-3,4.8828438E-2,-2.679227E-2,3.845852E-2,-1.4495105E-2,2.8808778E-3,1.6173092E-1,1.9256801E-3,2.2927333E-2,1.0094499E-1,2.3838468E-1,4.397981E-3,-3.2774562E-3,1.03562936E-1,-6.0479003E-3,-1.4568117E-3,-6.277974E-2,-4.0931082E-3,-2.3537235E-1,-5.9300605E-3,-6.481607E-3,-2.6139272E-3,5.352191E-2,-2.241417E-2,2.4598083E-2,5.0526774E-3,-1.4538056E-1,-1.2765515E-2,5.607442E-3,1.2290839E-2,-2.8369317E-3,3.0765005E-3,3.3858994E-3,8.153802E-3,-1.362983E-1,3.8940895E-2,1.2367919E-1,-1.104112E-3,6.4709918E-3,2.706289E-1,2.3289045E-2,7.1208873E-3,-1.2806566E-2,-8.651352E-2,-2.080394E-2,3.853658E-2,-4.4824574E-3,-1.3216604E-2,2.5242904E-2,1.0797634E-1,3.2110284E-3,-6.6505514E-2,-1.4431196E-3,4.570221E-2,-1.8732371E-3,-1.0216361E-2,1.2648586E-2,-4.126894E-2,-2.6493251E-2,6.1093733E-2,3.4651808E-3,-1.3382404E-2,8.724209E-3,7.902209E-2,1.6011976E-1,6.684649E-2,7.4967253E-3,1.4551862E-2,-1.36519E-3,3.0146171E-3,1.2837243E-3,-2.0682854E-3,-5.330412E-3,-2.7189185E-3,8.006539E-4,-1.9510069E-3,3.610932E-3,-8.1835163E-4,4.666905E-4,3.303524E-3,2.0861109E-3,7.5430125E-3,-5.617559E-3,4.0883533E-4,4.1358406E-3,7.0317305E-4,2.858946E-3,-1.2262574E-4,-2.4527307E-3,1.3239514E-3,7.4568996E-4,-5.464974E-3,4.5384523E-3,-2.5098824E-3,2.7692802E-3,-1.8188662E-3,8.140432E-3,1.7950365E-3,8.773918E-3,2.7231404E-3,6.678367E-3,-4.7211503E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,61,63,65,-1,67,-1,-1,-1,69,71,73,75,-1,-1,-1,77,79,81,83,85,-1,-1,87,89,91,-1,-1,-1,-1,-1,93,-1,-1,95,-1,97,-1,-1,99,101,-1,-1,103,105,107,-1,109,111,-1,113,115,117,-1,-1,119,-1,-1,121,123,125,-1,-1,-1,127,129,131,-1,133,135,-1,137,-1,-1,-1,-1,139,141,143,-1,-1,145,147,-1,149,151,153,155,-1,-1,157,159,-1,161,-1,163,-1,-1,165,167,169,171,-1,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.229679E0,1.2550843E0,8.9188766E-1,6.0476685E-1,2.972863E-1,8.0297303E-1,1.975645E-1,4.6644235E-1,8.9068925E-1,2.669872E-1,0E0,0E0,4.828894E-1,0E0,6.044476E-2,2.1825123E-1,3.8755083E-1,1.9265217E-1,6.826532E-1,1.8109734E-1,1.0956398E-1,3.164283E-1,1.4807415E-1,0E0,0E0,4.5170188E-2,5.9051156E-2,1.5427318E-1,3.0654085E-1,6.091845E-2,3.8964592E-2,0E0,0E0,9.500469E-2,2.8814012E-1,0E0,1.1074269E-1,2.7767336E-1,2.91803E-1,6.3414335E-2,2.2052176E-2,0E0,2.5466818E-2,0E0,0E0,0E0,7.595815E-2,2.5562325E-1,3.544889E-1,6.848256E-2,0E0,0E0,0E0,1.2475047E-1,1.3266331E-1,1.5481836E-1,3.7513608E-1,3.8779497E-2,0E0,0E0,1.7525658E-1,8.065164E-2,1.6567805E-1,0E0,0E0,0E0,0E0,0E0,3.36269E-2,0E0,0E0,2.4555314E-1,0E0,5.791509E-2,0E0,0E0,1.9443631E-2,6.5717354E-2,0E0,0E0,5.0117612E-2,1.4119728E-1,1.3520245E-1,0E0,4.095189E-2,1.7093003E-2,0E0,2.7813292E-1,1.1648506E-1,3.817749E-2,0E0,0E0,1.1007944E-1,0E0,0E0,1.5663701E-1,1.1122024E-1,6.344092E-2,0E0,0E0,0E0,5.2711368E-2,7.35201E-2,3.3243068E-2,0E0,6.2510654E-2,5.7024326E-2,0E0,9.436458E-2,0E0,0E0,0E0,0E0,3.4580347E-1,1.1800045E-1,5.971104E-2,0E0,0E0,3.0590415E-2,2.8536215E-2,0E0,5.8332376E-2,6.5071344E-2,9.224582E-2,9.945354E-2,0E0,0E0,1.678972E-2,3.431256E-2,0E0,5.089094E-2,0E0,2.5351577E-2,0E0,0E0,3.1808384E-2,3.2107137E-2,1.1149431E-1,9.96535E-2,0E0,0E0,1.4276846E-1,1.5710282E-1,4.9098134E-2,7.9258405E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,38,38,39,39,40,40,42,42,46,46,47,47,48,48,49,49,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,68,68,71,71,73,73,76,76,77,77,80,80,81,81,82,82,84,84,85,85,87,87,88,88,89,89,92,92,95,95,96,96,97,97,101,101,102,102,103,103,105,105,106,106,108,108,113,113,114,114,115,115,118,118,119,119,121,121,122,122,123,123,124,124,127,127,128,128,130,130,132,132,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,62,64,66,-1,68,-1,-1,-1,70,72,74,76,-1,-1,-1,78,80,82,84,86,-1,-1,88,90,92,-1,-1,-1,-1,-1,94,-1,-1,96,-1,98,-1,-1,100,102,-1,-1,104,106,108,-1,110,112,-1,114,116,118,-1,-1,120,-1,-1,122,124,126,-1,-1,-1,128,130,132,-1,134,136,-1,138,-1,-1,-1,-1,140,142,144,-1,-1,146,148,-1,150,152,154,156,-1,-1,158,160,-1,162,-1,164,-1,-1,166,168,170,172,-1,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.785302E5,3.5056704E7,1E0,4.4E1,3.3817584E7,1.3655363E2,3.89E2,9.639872E0,5.817547E2,-1.0464207E-2,2.0494487E-2,6.5590655E6,-8.86112E-3,2.342E4,6.7E1,6E0,5.7E1,1.753E3,2.308943E0,1.2895454E3,1.5142261E3,1.8029197E0,3.285471E-3,-3.5930462E-3,8.8324326E-1,1.3E1,1.89991E5,2.0734E4,5.6E1,5E0,-2.515075E-2,-1.7462966E-3,1.047E3,2.954124E6,-2.2505E-3,2.0319266E-7,5.6E1,8.710612E1,1E0,5.941442E6,-6.518899E-3,3.774648E0,-3.2511433E-3,-1.0478092E-2,1.1660193E-2,1.6595395E5,4.6E1,4.5179688E1,1.2E1,-8.221557E-3,-2.4021226E-3,2.194635E-3,1.683263E7,6.923077E-1,6.848509E7,4.1957852E2,3.361317E1,1.7520034E-4,1.1522659E-2,2.9251662E6,3.1095755E0,6.516E3,1.3075406E-2,4.8180684E-3,2.2034561E-3,6.7632147E-3,6.2657185E-5,2.8E1,-2.8989706E-3,4.047658E-3,1.5292561E2,-1.2008373E-2,2.0421052E0,2.968251E-3,4.992811E-4,3.3E1,6E0,-5.6677135E-3,8.975173E-3,1.28738E5,1.901875E2,3.168978E-2,-1.4495105E-2,1.535E3,1.8425926E0,1.9256801E-3,1.4598765E0,1E0,1.990351E1,4.397981E-3,-3.2774562E-3,1.270394E5,-6.0479003E-3,-1.4568117E-3,4.577342E0,7.09617E6,1.0609756E0,-5.9300605E-3,-6.481607E-3,-2.6139272E-3,2.6676828E2,3.5714287E-1,3.1096E4,5.0526774E-3,1.3938298E2,1E0,5.607442E-3,1.2768175E6,-2.8369317E-3,3.0765005E-3,3.3858994E-3,8.153802E-3,1.38132E5,1.0901037E10,1.500502E6,-1.104112E-3,6.4709918E-3,1.670046E1,2.337765E3,7.1208873E-3,1.31E2,2.511352E6,8.454545E0,2.5417458E1,-4.4824574E-3,-1.3216604E-2,4.28649E5,1.5E1,3.2110284E-3,8.1E2,-1.4431196E-3,2.3773398E3,-1.8732371E-3,-1.0216361E-2,3.491E3,1.9E1,1.1E1,1.990351E1,3.4651808E-3,-1.3382404E-2,1.0717949E1,5.8475E5,4.5866325E6,1.552356E0,7.4967253E-3,1.4551862E-2,-1.36519E-3,3.0146171E-3,1.2837243E-3,-2.0682854E-3,-5.330412E-3,-2.7189185E-3,8.006539E-4,-1.9510069E-3,3.610932E-3,-8.1835163E-4,4.666905E-4,3.303524E-3,2.0861109E-3,7.5430125E-3,-5.617559E-3,4.0883533E-4,4.1358406E-3,7.0317305E-4,2.858946E-3,-1.2262574E-4,-2.4527307E-3,1.3239514E-3,7.4568996E-4,-5.464974E-3,4.5384523E-3,-2.5098824E-3,2.7692802E-3,-1.8188662E-3,8.140432E-3,1.7950365E-3,8.773918E-3,2.7231404E-3,6.678367E-3,-4.7211503E-5],"split_indices":[2,43,60,6,3,7,71,1,69,67,0,0,43,0,9,44,3,6,44,69,48,67,69,0,0,68,5,9,44,0,8,0,0,2,62,0,52,0,73,19,60,0,69,0,0,0,43,3,71,3,0,0,0,60,68,7,4,71,0,0,43,53,44,0,0,0,0,0,2,0,0,67,0,68,0,0,3,8,0,0,1,4,53,0,2,69,0,69,6,73,0,0,48,0,0,73,12,68,0,0,0,4,68,9,0,4,26,0,62,0,0,0,0,9,46,9,0,0,71,67,0,44,5,73,73,0,0,9,3,0,2,0,48,0,0,12,3,8,73,0,0,71,1,43,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,7.66E2,2.73E2,4.66E2,3E2,2.53E2,2E1,3.99E2,6.7E1,2.96E2,4E0,6E0,2.47E2,1.1E1,9E0,5.4E1,3.45E2,5.5E1,1.2E1,2.59E2,3.7E1,2.14E2,3.3E1,5E0,4E0,2.9E1,2.5E1,1.8E1,3.27E2,4.1E1,1.4E1,8E0,4E0,9.8E1,1.61E2,4E0,3.3E1,1.51E2,6.3E1,2.2E1,1.1E1,7E0,2.2E1,4E0,2.1E1,4E0,1.4E1,2.9E2,3.7E1,2.4E1,1.7E1,6E0,8E0,5.6E1,4.2E1,1.47E2,1.4E1,2.6E1,7E0,6E0,1.45E2,3E1,3.3E1,1.8E1,4E0,6E0,5E0,7E0,1.5E1,4E0,1E1,2.86E2,4E0,3E1,7E0,5E0,1.9E1,5.1E1,5E0,7E0,3.5E1,8.5E1,6.2E1,6E0,8E0,2.2E1,4E0,1.06E2,3.9E1,2.3E1,7E0,6E0,2.7E1,5E0,1E1,1.32E2,1.54E2,1.2E1,1.8E1,1.3E1,6E0,3.5E1,1.6E1,2.6E1,9E0,8E0,7.7E1,1.4E1,4.8E1,4E0,4E0,4E0,1.8E1,9E0,9.7E1,3.3E1,6E0,7E0,1.6E1,1.1E1,1.6E1,4.3E1,8.9E1,1.11E2,4.3E1,4E0,8E0,2.4E1,1.1E1,5E0,1.1E1,7E0,1.9E1,4E0,4E0,4.1E1,3.6E1,2.7E1,2.1E1,4E0,5E0,5.6E1,4.1E1,1.9E1,1.4E1,6E0,1E1,5E0,6E0,1.9E1,2.4E1,4.2E1,4.7E1,4E1,7.1E1,2.5E1,1.8E1,1.9E1,5E0,6E0,5E0,6E0,5E0,7E0,1.2E1,9E0,3.2E1,3.1E1,5E0,1.9E1,8E0,1.6E1,5E0,2.7E1,2.9E1,1.1E1,3E1,1.4E1,5E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[-2.3257365E-3,-9.361101E-2,2.9138543E-2,-7.504461E-2,-2.1700005E-1,-4.3321406E-3,1.011941E-1,-1.1467243E-1,-3.931716E-2,-1.2133072E-1,-2.167969E-2,-3.6855094E-2,2.0264916E-2,1.1289062E-1,-7.688294E-4,-1.4771907E-1,-7.8278266E-2,1.1763657E-2,-6.197982E-2,-1.60153E-1,-4.319978E-2,-3.0141326E-2,-1.5133677E-2,2.4603236E-2,-9.040706E-3,1.0391565E-1,1.1384624E-2,1.3134174E-1,-1.0640603E-1,-1.0529617E-1,-1.7624575E-1,1.9840815E-3,-9.1524586E-2,-3.8676467E-2,-1.3649903E-1,-7.770432E-2,-1.232758E-2,2.9120783E-4,-3.568988E-3,-4.25794E-2,2.9313734E-2,6.865808E-2,6.948213E-3,6.336656E-2,1.3131529E-1,1.2039306E-2,-1.9101286E-3,-9.710885E-4,-9.285439E-3,-1.8202295E-3,-1.287337E-1,-2.0858306E-1,-3.2538017E-3,-1.2343109E-1,-5.3145684E-2,-1.4649846E-2,-9.6331365E-2,-8.9190565E-3,-3.6211764E-3,-7.5402656E-3,2.6586677E-3,-3.4477558E-2,-1.9628441E-1,6.643207E-2,-8.497266E-2,3.4993973E-2,9.441506E-2,-2.985897E-2,4.1015346E-2,7.423596E-2,-4.2558517E-3,1.493828E-1,2.5508147E-2,-7.6652523E-3,-3.3778017E-3,-2.4586946E-1,-2.575395E-3,-1.5095799E-1,-5.4662734E-2,-1.7122427E-2,-5.6848805E-3,2.5977049E-2,-3.9675407E-2,-7.3686386E-3,-9.345938E-3,-4.9747836E-2,9.4321783E-4,-3.1843367E-3,-1.313834E-2,4.1217147E-3,8.762426E-4,1.9442398E-3,-8.16296E-3,9.34223E-2,8.7435124E-4,8.1993686E-4,1.0738812E-1,-1.584113E-2,-1.3796739E-2,9.281836E-2,-2.1185244E-3,2.3200482E-3,9.659797E-2,1.7556436E-1,8.20661E-2,9.7870804E-2,-3.9834836E-3,-1.3162144E-2,-7.2342553E-3,-8.248221E-3,-2.011797E-3,4.8022768E-5,-4.593682E-3,6.649162E-4,-3.5711122E-3,-1.5925014E-2,6.588088E-2,-9.046201E-3,-9.591206E-2,2.3565071E-3,-3.6128052E-3,-5.725016E-2,4.075345E-2,1.3056016E-2,-3.7999048E-3,6.9344183E-4,6.946276E-3,-4.660225E-3,3.1354763E-2,2.837775E-2,1.262539E-1,3.9971434E-2,-4.8893124E-2,1.17042884E-1,-7.2920835E-4,-5.25034E-2,3.1982567E-2,3.6384843E-3,-2.1711405E-2,1.451414E-1,5.885555E-2,1.4906694E-1,1.3861813E-2,5.108682E-2,1.0761651E-2,9.2859525E-4,6.2019355E-3,-2.261614E-3,2.1580318E-3,4.2798687E-3,2.0968488E-4,-2.5722475E-3,5.3882714E-2,-6.505277E-3,-5.8119214E-4,-2.8470319E-2,-7.3642984E-2,3.9775637E-3,-1.766854E-4,-1.1563249E-2,6.301725E-2,5.4900598E-2,-2.3220961E-3,-8.796674E-4,3.239513E-3,1.4647314E-1,3.1010772E-3,1.08308464E-1,6.893903E-3,-1.3479684E-1,-1.6083207E-2,6.465944E-2,2.0353024E-1,4.8096015E-4,-8.272238E-2,9.167583E-2,-2.3340499E-2,-2.5334295E-3,4.6874394E-4,1.6548668E-1,2.2980636E-3,-1.4143132E-3,8.551146E-2,1.2755826E-2,1.2556845E-1,7.33895E-2,-2.8515377E-3,1.741479E-4,4.775488E-3,-4.848594E-4,-5.516882E-3,-7.39282E-3,-2.9940906E-3,-1.0144135E-3,2.3578054E-3,3.9681285E-3,4.8866356E-4,3.6606938E-3,-6.693271E-4,2.9359742E-3,7.4189096E-3,6.2821656E-3,1.9575302E-3,1.088083E-3,-2.4281596E-3,-2.3489571E-3,-1.0297211E-2,9.076303E-4,-4.2149E-3,5.7357843E-3,1.0329944E-3,1.2627515E-2,5.8501055E-3,-1.3251698E-3,-6.4646406E-3,1.3527403E-3,5.563906E-3,-3.0664308E-3,1.216456E-3,5.0566373E-3,9.165458E-3,6.077624E-3,2.2166981E-3,1.0801344E-2,5.1971795E-3,4.294069E-3,7.1184384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,35,37,39,-1,41,-1,43,-1,45,47,49,51,-1,53,55,57,59,-1,-1,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,-1,77,79,81,83,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,-1,109,111,113,-1,115,117,-1,119,121,123,-1,-1,-1,-1,-1,-1,125,127,-1,129,131,-1,133,135,137,139,141,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,153,-1,-1,155,157,159,-1,-1,-1,-1,161,163,165,167,169,171,-1,173,175,-1,177,179,181,183,-1,185,-1,-1,-1,-1,-1,-1,-1,-1,187,-1,-1,189,191,-1,-1,193,195,197,-1,-1,-1,199,-1,201,203,205,207,209,211,-1,213,215,217,-1,-1,219,-1,-1,221,-1,223,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9582477E0,5.885105E-1,1.8478379E0,3.2254136E-1,7.6362646E-1,4.199306E-1,2.8989124E-1,1.1953163E-1,8.351613E-1,7.194519E-2,0E0,4.3123996E-1,2.849426E-1,2.3565936E-1,3.768261E-1,4.9212337E-2,9.064275E-2,0E0,1.9442368E-1,1.287958E-1,1.734406E-2,1.6541861E-1,0E0,2.28193E-1,0E0,2.2055125E-1,0E0,2.7862227E-1,1.1691502E-1,3.3539206E-2,9.480685E-2,0E0,5.344248E-2,1.21876225E-1,7.3616385E-2,1.3660975E-1,0E0,0E0,0E0,2.2236672E-1,1.7108765E-1,6.953129E-2,2.6580077E-1,1.5049243E-1,2.2941351E-1,0E0,0E0,0E0,0E0,0E0,2.5096208E-2,1.2306404E-1,0E0,4.0965676E-2,5.8331028E-2,6.610015E-2,1.4136745E-1,0E0,0E0,0E0,0E0,9.5242694E-2,8.138168E-2,3.0751914E-2,1.2397021E-1,7.56733E-2,4.4840574E-2,3.7957314E-1,2.4598771E-1,1.2843063E-1,0E0,1.6888762E-1,1.6205882E-1,0E0,0E0,2.1597743E-2,0E0,4.5546174E-2,2.2547279E-2,3.3351928E-2,0E0,4.2850472E-2,6.8031736E-2,0E0,5.3856578E-2,8.584648E-2,5.5490278E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.0242854E-2,8.069519E-2,0E0,5.6040943E-2,1.8273656E-1,0E0,1.333895E-1,1.0648812E-1,3.844759E-2,1.03218555E-1,1.9935656E-1,1.3299358E-1,3.1246841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9181164E-2,2.091834E-2,8.223039E-2,5.0608307E-2,0E0,0E0,5.1686585E-2,2.054212E-2,5.9769113E-2,0E0,0E0,0E0,0E0,4.119456E-2,1.9149698E-2,2.8425872E-2,8.288889E-2,1.7257862E-1,1.7259061E-1,0E0,4.839112E-2,1.2432909E-1,0E0,1.744135E-2,4.2231977E-2,8.847342E-2,1.5339875E-1,0E0,7.246068E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9368546E-2,0E0,0E0,6.9041744E-2,4.457295E-2,0E0,0E0,2.259264E-2,1.8284276E-2,2.8167114E-2,0E0,0E0,0E0,1.671055E-2,0E0,1.6133487E-2,2.6987493E-2,1.1439049E-1,1.253054E-1,6.517204E-2,4.5903146E-2,0E0,4.85404E-2,2.877155E-2,4.449857E-2,0E0,0E0,1.7003655E-2,0E0,0E0,4.2070195E-2,0E0,5.1519215E-2,2.4649315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,25,25,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,39,39,40,40,41,41,42,42,43,43,44,44,50,50,51,51,53,53,54,54,55,55,56,56,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,77,77,78,78,79,79,81,81,82,82,84,84,85,85,86,86,93,93,94,94,96,96,97,97,99,99,100,100,101,101,102,102,103,103,104,104,105,105,115,115,116,116,117,117,118,118,121,121,122,122,123,123,128,128,129,129,130,130,131,131,132,132,133,133,135,135,136,136,138,138,139,139,140,140,141,141,143,143,152,152,155,155,156,156,159,159,160,160,161,161,165,165,167,167,168,168,169,169,170,170,171,171,172,172,174,174,175,175,176,176,179,179,182,182,184,184,185,185],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,36,38,40,-1,42,-1,44,-1,46,48,50,52,-1,54,56,58,60,-1,-1,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,-1,78,80,82,84,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,-1,110,112,114,-1,116,118,-1,120,122,124,-1,-1,-1,-1,-1,-1,126,128,-1,130,132,-1,134,136,138,140,142,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,154,-1,-1,156,158,160,-1,-1,-1,-1,162,164,166,168,170,172,-1,174,176,-1,178,180,182,184,-1,186,-1,-1,-1,-1,-1,-1,-1,-1,188,-1,-1,190,192,-1,-1,194,196,198,-1,-1,-1,200,-1,202,204,206,208,210,212,-1,214,216,218,-1,-1,220,-1,-1,222,-1,224,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,4.4895835E0,2.652E3,1.19111115E2,1E0,8.1E2,2.3876712E7,1.3E1,7.336111E4,8.237624E0,-2.167969E-2,1E0,1.2294118E1,9.059046E6,5.2224144E5,1.00033E5,2.88E0,1.1763657E-2,4.76E2,2.9E1,6.763314E7,3.2911258E6,-1.5133677E-2,8.960784E0,-9.040706E-3,1.1342433E3,1.1384624E-2,3.827E5,1.3784861E5,2.8282208E0,2.2470966E-1,1.9840815E-3,1.4473684E-1,1.3439851E7,9.3326636E-2,1.6254545E1,-1.232758E-2,2.9120783E-4,-3.568988E-3,8.84E2,8E0,1.071E3,3.850926E2,1.2118524E7,4.9036694E0,1.2039306E-2,-1.9101286E-3,-9.710885E-4,-9.285439E-3,-1.8202295E-3,4E-1,7.2896E5,-3.2538017E-3,6.44E2,7E0,8E0,1.0590052E8,-8.9190565E-3,-3.6211764E-3,-7.5402656E-3,2.6586677E-3,1.559733E6,2.124E3,9.244374E6,5.75E2,1.6580646E2,7.2864324E-1,2.41E3,3.8275862E0,4.9963706E5,-4.2558517E-3,8.7E1,4.88E2,-7.6652523E-3,-3.3778017E-3,2.72E0,-2.575395E-3,5.0406504E0,1.2445087E2,5.6321215E4,-5.6848805E-3,2.7976523E2,1.0885427E9,-7.3686386E-3,1.8729467E8,1.5512196E1,4.888889E0,-3.1843367E-3,-1.313834E-2,4.1217147E-3,8.762426E-4,1.9442398E-3,-8.16296E-3,8.9E2,1E1,8.1993686E-4,2.492E2,1.5522388E0,-1.3796739E-2,3.9888642E0,5.62E2,4.0663E4,9.318287E0,1.9E1,4.91027E0,2.8389828E7,-3.9834836E-3,-1.3162144E-2,-7.2342553E-3,-8.248221E-3,-2.011797E-3,4.8022768E-5,-4.593682E-3,6.649162E-4,-3.5711122E-3,2.0134516E6,1.6595395E5,8.39E2,2.58466E5,2.3565071E-3,-3.6128052E-3,1.4695653E0,1.8645384E1,1.142E3,-3.7999048E-3,6.9344183E-4,6.946276E-3,-4.660225E-3,1E0,4.07E2,1E0,4.05E2,3.49E2,1.95E2,-7.2920835E-4,3.8795E4,9.822E3,3.6384843E-3,2.1340163E8,6.5590655E6,1.5724638E1,1.0586236E5,1.3861813E-2,9.100503E0,1.0761651E-2,9.2859525E-4,6.2019355E-3,-2.261614E-3,2.1580318E-3,4.2798687E-3,2.0968488E-4,-2.5722475E-3,1.24272164E5,-6.505277E-3,-5.8119214E-4,1.137832E-7,2.8E0,3.9775637E-3,-1.766854E-4,5E0,2.7432E4,5.831829E6,-2.3220961E-3,-8.796674E-4,3.239513E-3,4.3827028E5,3.1010772E-3,3.7509E4,3.7253174E2,7.09415E5,4.2E2,3E0,1.5416006E5,4.8096015E-4,1.08E2,1.609E3,1.862E3,-2.5334295E-3,4.6874394E-4,5.893737E2,2.2980636E-3,-1.4143132E-3,3.307E3,1.2755826E-2,3.596E3,3.0546486E8,-2.8515377E-3,1.741479E-4,4.775488E-3,-4.848594E-4,-5.516882E-3,-7.39282E-3,-2.9940906E-3,-1.0144135E-3,2.3578054E-3,3.9681285E-3,4.8866356E-4,3.6606938E-3,-6.693271E-4,2.9359742E-3,7.4189096E-3,6.2821656E-3,1.9575302E-3,1.088083E-3,-2.4281596E-3,-2.3489571E-3,-1.0297211E-2,9.076303E-4,-4.2149E-3,5.7357843E-3,1.0329944E-3,1.2627515E-2,5.8501055E-3,-1.3251698E-3,-6.4646406E-3,1.3527403E-3,5.563906E-3,-3.0664308E-3,1.216456E-3,5.0566373E-3,9.165458E-3,6.077624E-3,2.2166981E-3,1.0801344E-2,5.1971795E-3,4.294069E-3,7.1184384E-4],"split_indices":[43,68,2,67,6,2,60,3,60,71,0,106,68,43,43,5,69,0,0,3,7,43,0,73,0,4,0,10,48,69,53,0,71,60,53,69,0,0,0,10,8,2,67,62,53,0,0,0,0,0,73,9,0,1,8,32,7,0,0,0,0,9,44,60,2,67,73,2,69,43,0,8,0,0,0,69,0,69,4,43,0,4,46,0,7,71,71,0,0,0,0,0,0,2,3,0,4,68,0,61,10,9,71,8,50,62,0,0,0,0,0,0,0,0,0,60,43,2,1,0,0,68,71,44,0,0,0,0,26,0,19,10,10,0,0,1,44,0,7,43,73,48,0,69,0,0,0,0,0,0,0,0,43,0,0,52,69,0,0,69,1,60,0,0,0,43,0,1,4,12,0,8,48,0,10,2,2,0,0,4,0,0,2,0,2,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,2.63E2,7.65E2,2.3E2,3.3E1,5.23E2,2.42E2,1.08E2,1.22E2,2.5E1,8E0,2.25E2,2.98E2,2.17E2,2.5E1,5.5E1,5.3E1,8E0,1.14E2,1.6E1,9E0,2.21E2,4E0,2.93E2,5E0,2.05E2,1.2E1,1.1E1,1.4E1,2.4E1,3.1E1,5E0,4.8E1,8.8E1,2.6E1,1E1,6E0,4E0,5E0,1.83E2,3.8E1,8.3E1,2.1E2,8.4E1,1.21E2,6E0,5E0,8E0,6E0,7E0,1.7E1,2.3E1,8E0,2.5E1,2.3E1,6.3E1,2.5E1,1.2E1,1.4E1,6E0,4E0,1.75E2,8E0,2.9E1,9E0,3.7E1,4.6E1,1.01E2,1.09E2,7.9E1,5E0,1.03E2,1.8E1,9E0,8E0,1.8E1,5E0,1.7E1,8E0,1.6E1,7E0,2.4E1,3.9E1,1.4E1,1.1E1,1.22E2,5.3E1,4E0,4E0,1.9E1,1E1,4E0,5E0,1.3E1,2.4E1,7E0,3.9E1,9.7E1,4E0,4.9E1,6E1,1.9E1,6E1,7.3E1,3E1,1.1E1,7E0,1.1E1,7E0,1.3E1,4E0,4E0,4E0,1.1E1,5E0,1.2E1,1.2E1,2.6E1,1.3E1,6E0,5E0,1.13E2,9E0,4.7E1,6E0,6E0,7E0,5E0,1.9E1,8E0,3.1E1,3.6E1,6.1E1,4E1,9E0,2.4E1,3.6E1,4E0,1.5E1,2.5E1,3.5E1,6.2E1,1.1E1,2.6E1,4E0,4E0,7E0,8E0,4E0,8E0,4E0,1.5E1,1.1E1,8E0,5E0,4.2E1,7.1E1,4E0,5E0,3.2E1,1.5E1,1.5E1,4E0,4E0,4E0,2.2E1,9E0,1.1E1,2.5E1,1.6E1,4.5E1,2.6E1,1.4E1,8E0,1.6E1,1.7E1,1.9E1,7E0,8E0,2E1,5E0,8E0,2.7E1,8E0,5.4E1,2.2E1,4E0,6E0,5E0,3.6E1,6E0,5E0,6.6E1,2.8E1,4E0,1E1,5E0,1.1E1,4E0,4E0,1.8E1,7E0,4E0,2E1,5E0,9E0,7E0,3.1E1,1.4E1,1E1,1.6E1,6E0,8E0,9E0,7E0,6E0,1.1E1,1E1,9E0,9E0,1.1E1,1.1E1,1.6E1,4E0,5E1,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[1.1009438E-3,-3.9112236E-2,7.401193E-2,-6.565131E-2,1.1799968E-2,9.642514E-2,-5.0982617E-2,-5.7125136E-2,-1.785663E-1,4.2089024E-3,1.4745581E-1,2.640706E-1,8.514786E-2,-1.3052501E-1,8.411319E-3,-8.500051E-2,-1.606398E-2,-1.3229026E-1,-1.7716803E-2,-3.3999672E-3,1.2328587E-1,2.1924123E-3,8.84822E-3,1.4566637E-2,4.643914E-3,1.23720154E-1,3.293409E-2,-2.8229157E-2,-2.015771E-1,8.455084E-2,-5.214153E-2,-7.6764576E-2,-2.5022778E-1,1.6954329E-2,-7.8922436E-2,-1.4988591E-1,-1.3125762E-3,7.1650865E-3,-8.685387E-2,2.9813452E-3,9.442373E-3,1.020845E-1,1.8986122E-1,2.0646023E-2,1.0955011E-2,-3.3969164E-3,1.0059561E-3,-1.0332464E-1,-1.4569315E-2,3.2375936E-4,6.2584677E-3,-8.223674E-3,1.9642137E-2,-1.6757555E-1,-6.591733E-2,-9.0988737E-4,-2.4158787E-2,8.109917E-3,6.0118563E-3,-9.401221E-3,-6.02769E-2,-3.3532484E-3,-7.7777305E-3,1.5964994E-2,-5.755993E-2,-1.1471005E-2,-1.5268955E-2,1.02831E-2,8.1624605E-2,2.6887571E-3,2.0287366E-1,2.8626302E-2,-6.2100897E-3,-1.9861078E-3,-6.6061923E-3,-3.379097E-3,5.064358E-3,-8.985815E-3,-4.1306373E-3,-5.707424E-2,-1.684672E-1,-4.080267E-2,2.5809642E-2,-3.4349825E-2,-6.545361E-3,2.1768924E-2,-7.385123E-2,-5.8555426E-3,-7.7722454E-4,2.6048787E-3,-4.7550974E-3,-2.1161705E-2,9.5475465E-2,2.1868771E-1,2.8222546E-3,-3.5680735E-3,3.522247E-2,3.0007742E-2,-6.511559E-2,-3.6720086E-3,-9.882433E-3,-1.5785502E-2,-9.1760166E-2,3.4780245E-2,-4.1845776E-2,-1.70479E-2,-5.985214E-3,-9.127189E-3,3.700356E-2,-5.6874775E-3,1.2595323E-6,-3.8020848E-3,5.323314E-4,1.0407894E-1,-3.943781E-3,1.0890828E-2,4.8748157E-3,-4.9988687E-2,4.4427574E-2,-1.7183678E-2,5.515672E-3,-6.930566E-2,2.0799518E-2,-1.9589649E-3,9.4224914E-4,-7.7101826E-3,-6.4023543E-4,4.5027938E-2,-2.0264322E-3,-3.6868136E-3,2.0699066E-4,-3.1238016E-2,1.605347E-3,6.6782977E-3,-7.2012E-3,-2.4768012E-2,5.0535183E-2,8.743726E-2,8.303924E-3,2.0616E-4,-5.36937E-3,5.4291193E-3,3.662511E-2,-2.3333067E-3,1.8280034E-3,-4.64554E-3,-2.6723002E-3,3.8047738E-3,-2.0751213E-3,-1.8756782E-3,2.4080144E-3,-3.35718E-3,-5.0970225E-4,1.0832227E-3,-1.5949009E-3,1.5376982E-3,-3.7117896E-3,6.5935426E-3,1.9816665E-3,5.313785E-3,1.0745503E-3,9.09989E-4,3.8047836E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,-1,-1,67,69,71,-1,-1,-1,73,-1,-1,-1,-1,75,77,79,-1,-1,-1,81,-1,83,-1,-1,85,87,-1,89,-1,91,-1,93,95,-1,-1,-1,-1,-1,-1,-1,97,99,101,103,105,-1,107,109,-1,-1,-1,-1,111,113,115,-1,-1,117,119,121,-1,-1,123,125,127,129,131,-1,133,135,-1,-1,-1,-1,137,-1,-1,-1,139,141,143,-1,145,147,-1,-1,-1,-1,149,-1,-1,-1,151,-1,153,-1,155,157,159,-1,-1,-1,-1,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0023127E0,8.925421E-1,1.0246727E0,4.0762532E-1,2.331377E-1,5.617058E-1,2.656607E-1,4.6219552E-1,2.3375797E-1,1.9612405E-1,4.3792754E-2,1.2750483E-1,5.8200073E-1,1.6874972E-1,1.5655395E-1,3.087443E-1,3.4379223E-1,4.3264776E-2,0E0,1.8048808E-1,4.8753113E-2,0E0,0E0,0E0,0E0,2.1959591E-1,3.087452E-1,2.6517067E-2,1.23927295E-1,5.8273993E-2,1.7516066E-1,2.1650028E-1,6.812315E-1,1.8809797E-1,1.2012792E-1,1.9186378E-2,0E0,1.0458145E-1,2.6201448E-1,0E0,0E0,2.974E-1,5.672252E-2,1.5110719E-1,0E0,0E0,0E0,1.5393563E-2,0E0,0E0,0E0,0E0,1.1654924E-1,3.3123553E-2,1.7901105E-1,0E0,0E0,0E0,9.651597E-2,0E0,1.02767706E-1,0E0,0E0,8.5534886E-2,5.98725E-2,0E0,1.1257855E-1,0E0,1.6004872E-1,0E0,6.735933E-2,8.2322314E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3710368E-1,4.7697335E-2,3.8070694E-2,4.5759827E-2,6.3679166E-2,0E0,7.188278E-2,3.6393583E-2,0E0,0E0,0E0,0E0,2.8922964E-2,1.6135865E-1,3.686881E-2,0E0,0E0,8.693366E-2,7.3758274E-2,6.515962E-2,0E0,0E0,2.2051597E-2,5.6215197E-2,5.43594E-2,1.6183186E-2,2.6810357E-2,0E0,1.1933007E-1,8.6510345E-2,0E0,0E0,0E0,0E0,1.026454E-1,0E0,0E0,0E0,4.01902E-2,5.1582277E-2,2.4691237E-2,0E0,5.4718196E-2,4.0106725E-2,0E0,0E0,0E0,0E0,3.7351184E-2,0E0,0E0,0E0,2.263704E-2,0E0,3.328061E-2,0E0,6.397338E-2,5.30473E-2,1.3716805E-1,0E0,0E0,0E0,0E0,6.805012E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,43,43,47,47,52,52,53,53,54,54,58,58,60,60,63,63,64,64,66,66,68,68,70,70,71,71,79,79,80,80,81,81,82,82,83,83,85,85,86,86,91,91,92,92,93,93,96,96,97,97,98,98,101,101,102,102,103,103,104,104,105,105,107,107,108,108,113,113,117,117,118,118,119,119,121,121,122,122,127,127,131,131,133,133,135,135,136,136,137,137,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,-1,-1,68,70,72,-1,-1,-1,74,-1,-1,-1,-1,76,78,80,-1,-1,-1,82,-1,84,-1,-1,86,88,-1,90,-1,92,-1,94,96,-1,-1,-1,-1,-1,-1,-1,98,100,102,104,106,-1,108,110,-1,-1,-1,-1,112,114,116,-1,-1,118,120,122,-1,-1,124,126,128,130,132,-1,134,136,-1,-1,-1,-1,138,-1,-1,-1,140,142,144,-1,146,148,-1,-1,-1,-1,150,-1,-1,-1,152,-1,154,-1,156,158,160,-1,-1,-1,-1,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,9.860918E5,1E0,1.0657745E7,7.82261E6,1.2083E4,1.9674084E5,6.99E2,7.627907E0,4.1E1,5.51E2,1.562E5,8E0,2.5345264E7,1.7234043E1,1.335955E2,7.453E3,5.1485147E0,-1.7716803E-2,2.8302418E10,1.0717949E1,2.1924123E-3,8.84822E-3,1.4566637E-2,4.643914E-3,6.46E2,4.91027E0,6E0,5E0,2.652E3,2.0900433E1,3.4E1,5.5E1,1.1634076E4,3.4402E4,1.4979E4,-1.3125762E-3,1E0,1.3254E4,2.9813452E-3,9.442373E-3,5.6E1,3.307E3,3.0392378E7,1.0955011E-2,-3.3969164E-3,1.0059561E-3,5.1917776E-2,-1.4569315E-2,3.2375936E-4,6.2584677E-3,-8.223674E-3,3.7827366E8,5.0363636E1,3.1E1,-9.0988737E-4,-2.4158787E-2,8.109917E-3,6.613774E5,-9.401221E-3,8.998703E-1,-3.3532484E-3,-7.7777305E-3,1E0,7.694314E-7,-1.1471005E-2,2.0217392E0,1.02831E-2,3.7043128E5,2.6887571E-3,4.7E1,8.147158E4,-6.2100897E-3,-1.9861078E-3,-6.6061923E-3,-3.379097E-3,5.064358E-3,-8.985815E-3,-4.1306373E-3,8.414097E-1,1.6E1,3.3E0,2.6789763E9,1.6533886E8,-6.545361E-3,1.7733E4,5.4814816E0,-5.8555426E-3,-7.7722454E-4,2.6048787E-3,-4.7550974E-3,1.6869566E1,1E0,2.8146256E5,2.8222546E-3,-3.5680735E-3,5.4814816E0,1.1E1,1.073125E1,-3.6720086E-3,-9.882433E-3,3.2638438E5,6.74502E0,1E0,3.57E2,6.5062125E5,-5.985214E-3,2.72E2,6.078218E2,-5.6874775E-3,1.2595323E-6,-3.8020848E-3,5.323314E-4,8.83E3,-3.943781E-3,1.0890828E-2,4.8748157E-3,6.1454544E0,3.8197617E9,9.183432E4,5.515672E-3,3.3820656E7,1.279012E7,-1.9589649E-3,9.4224914E-4,-7.7101826E-3,-6.4023543E-4,8.48E3,-2.0264322E-3,-3.6868136E-3,2.0699066E-4,8.818731E0,1.605347E-3,1E0,-7.2012E-3,3.6507E4,8.691756E-1,4.643602E8,8.303924E-3,2.0616E-4,-5.36937E-3,5.4291193E-3,1.836095E6,-2.3333067E-3,1.8280034E-3,-4.64554E-3,-2.6723002E-3,3.8047738E-3,-2.0751213E-3,-1.8756782E-3,2.4080144E-3,-3.35718E-3,-5.0970225E-4,1.0832227E-3,-1.5949009E-3,1.5376982E-3,-3.7117896E-3,6.5935426E-3,1.9816665E-3,5.313785E-3,1.0745503E-3,9.09989E-4,3.8047836E-3],"split_indices":[2,43,6,60,43,9,48,2,69,8,2,10,32,60,71,61,44,69,0,46,71,0,0,0,0,0,50,8,8,2,71,2,0,43,9,1,0,6,44,0,0,0,2,58,0,0,0,53,0,0,0,0,7,62,8,0,0,0,60,0,57,0,0,79,52,0,71,0,43,0,8,43,0,0,0,0,0,0,0,68,3,68,46,5,0,1,71,0,0,0,0,73,31,48,0,0,71,3,69,0,0,60,69,19,0,43,0,10,48,0,0,0,0,2,0,0,0,69,46,43,0,7,7,0,0,0,0,1,0,0,0,71,0,30,0,1,68,7,0,0,0,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,6.59E2,3.63E2,4.33E2,2.26E2,3.08E2,5.5E1,4.04E2,2.9E1,2.15E2,1.1E1,1.8E1,2.9E2,2.3E1,3.2E1,2.4E2,1.64E2,2.5E1,4E0,2.03E2,1.2E1,4E0,7E0,1.3E1,5E0,1.66E2,1.24E2,1E1,1.3E1,1.4E1,1.8E1,2.3E2,1E1,1.08E2,5.6E1,2.1E1,4E0,1.81E2,2.2E1,8E0,4E0,1.27E2,3.9E1,1.18E2,6E0,5E0,5E0,8E0,5E0,6E0,8E0,6E0,1.2E1,2.3E1,2.07E2,6E0,4E0,6E0,1.02E2,6E0,5E1,5E0,1.6E1,1.6E2,2.1E1,6E0,1.6E1,1.7E1,1.1E2,4E0,3.5E1,1.13E2,5E0,4E0,4E0,6E0,6E0,1.6E1,7E0,1.92E2,1.5E1,3E1,7.2E1,3.9E1,1.1E1,1.51E2,9E0,7E0,1.4E1,9E0,7E0,1.3E1,9.7E1,3.1E1,4E0,6E0,1.07E2,1.6E1,1.76E2,6E0,9E0,2.1E1,9E0,6.4E1,8E0,3.4E1,5E0,5E1,1.01E2,5E0,4E0,4E0,9E0,9.3E1,4E0,2.6E1,5E0,1E1,9.7E1,1.1E1,5E0,1.68E2,8E0,1.2E1,9E0,4E0,5E0,5.7E1,7E0,4E0,4E0,2.7E1,7E0,4.6E1,4E0,1.8E1,8.3E1,7.8E1,1.5E1,6E0,4E0,8E0,8.9E1,7E0,4E0,4.3E1,1.25E2,4E0,4E0,4E0,5.3E1,8E0,1.9E1,3.3E1,1.3E1,9E0,9E0,5E0,7.8E1,5.4E1,2.4E1,6.6E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[-1.7823981E-3,-4.0926073E-2,6.908415E-2,-8.466238E-2,-1.5719987E-2,3.4781877E-2,1.426687E-1,-6.65194E-2,-2.2224681E-1,-7.045499E-2,2.00722E-3,4.7250196E-2,-1.661988E-1,1.8625151E-2,1.224276E-1,8.938055E-2,-7.689234E-2,-3.3350907E-2,-4.3763816E-1,-5.9620846E-2,-9.797345E-3,-1.9559527E-3,8.344906E-3,1.6832297E-1,3.7220646E-2,-1.7415055E-3,-1.265517E-2,1.3706036E-1,-3.760608E-2,1.0930074E-2,6.8394765E-3,-1.0896457E-1,-5.3991854E-2,-5.618569E-3,1.782589E-2,-2.4994403E-2,-7.578095E-3,-9.8991886E-2,-2.288893E-2,1.421071E-2,-3.6013566E-2,9.800655E-3,3.1952346E-3,-6.388553E-2,5.154758E-2,-1.0179935E-3,1.491423E-1,-4.9310275E-3,2.442503E-3,-2.3401529E-3,3.440138E-3,-1.8716906E-1,-8.632807E-2,-3.3540778E-2,-9.6131526E-2,3.0979766E-3,-1.5881547E-3,4.572157E-4,-1.1829479E-1,-1.0720574E-2,-6.2556736E-2,2.6304418E-2,-2.0596504E-2,4.977003E-3,-4.773916E-2,1.3387327E-2,-1.9496838E-2,7.179611E-2,1.1792234E-2,1.7904058E-1,1.0037008E-1,-2.2123496E-1,-2.5109504E-3,1.3077036E-3,-9.602929E-2,-4.3801606E-2,2.5729572E-3,-2.8782912E-2,-1.6550991E-1,-8.080106E-3,-8.779007E-2,-5.083314E-4,-2.7835057E-3,-1.0223103E-3,-4.587428E-3,4.0390793E-2,-1.4233276E-2,-1.2336766E-1,1.3342861E-2,-1.1219512E-1,-2.3116386E-2,6.9491334E-2,-4.966726E-3,8.692444E-2,-1.0145965E-2,-1.0988743E-2,5.948917E-3,1.8852025E-1,3.0973414E-3,-1.1658761E-2,1.4079677E-1,-7.1715E-3,-1.4275366E-2,-8.131578E-2,-7.2465115E-3,-2.5789065E-2,-8.602322E-2,-1.4941373E-1,5.6944806E-2,-9.179282E-3,-2.7952767E-3,-5.4082677E-2,-5.434599E-3,-1.3082469E-2,3.5828743E-3,1.1836862E-2,6.163217E-2,-6.3834814E-3,2.4491977E-3,-1.09247165E-2,-2.5168979E-3,-1.6950825E-2,6.552288E-2,-6.0261615E-2,-1.6776241E-1,-8.509088E-3,-1.0564108E-2,5.828922E-4,6.093556E-3,5.5383097E-2,1.2170904E-1,6.0001258E-2,-9.330804E-2,6.4652073E-3,-9.366883E-2,1.6613105E-1,1.3439045E-2,-2.1326744E-3,1.8007722E-3,9.645941E-2,9.746061E-3,-9.264131E-2,2.0176284E-3,-3.802059E-2,9.327427E-4,-6.6219675E-3,-1.6317331E-3,-1.1597686E-2,-8.2730426E-4,-3.2230516E-4,4.5167725E-3,-3.4073547E-3,4.7528482E-7,1.1356301E-3,-1.9241748E-3,-4.3642693E-3,8.031804E-2,7.39972E-2,-8.077768E-4,-4.180844E-3,1.604562E-2,-1.5501073E-3,1.0083345E-3,4.472581E-3,-1.0474909E-3,-3.4120228E-4,-8.596113E-2,-3.4949712E-3,-1.029082E-2,2.2867149E-2,-5.1443856E-2,9.102348E-2,-4.608941E-2,5.3078935E-2,1.488305E-1,-1.2253443E-3,4.923842E-3,2.323397E-3,-9.120735E-3,-3.668458E-3,1.829681E-2,-5.8822064E-3,-1.5961746E-3,1.465743E-1,1.3548806E-2,1.9197954E-3,5.775997E-3,-6.1476664E-3,-3.3300633E-3,-7.5694895E-4,-5.4485486E-3,-1.4037599E-3,1.0096745E-3,6.1842E-3,1.0308623E-3,4.1973614E-3,1.0633453E-3,2.3505064E-3,-1.4537458E-3,-5.8272686E-3,-9.499235E-4,-3.1516328E-4,5.358671E-3,-3.3846407E-3,1.4188082E-3,5.24261E-4,5.2724076E-3,2.1776587E-3,-6.70844E-3,-4.4813997E-4,3.811271E-3,7.7153766E-3,1.392189E-3,1.4253506E-3,-2.3414674E-3,1.0679259E-2,5.7965866E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,-1,41,43,-1,-1,45,47,-1,49,51,53,-1,55,-1,-1,57,59,61,63,-1,-1,65,67,-1,69,-1,-1,-1,-1,71,73,75,77,-1,-1,-1,79,81,83,85,87,-1,89,91,-1,93,95,97,99,101,-1,-1,103,105,-1,107,109,-1,111,113,-1,-1,-1,115,117,119,121,123,125,127,-1,129,131,133,-1,135,-1,137,139,-1,-1,141,-1,143,145,147,149,-1,-1,151,-1,153,-1,155,157,-1,159,-1,-1,161,163,165,167,-1,169,-1,-1,171,173,175,177,179,181,183,-1,-1,-1,185,-1,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,191,193,195,-1,-1,197,-1,-1,-1,-1,-1,199,-1,-1,201,203,205,207,209,211,-1,-1,-1,-1,-1,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8766673E0,7.3508644E-1,9.26437E-1,5.942868E-1,4.125184E-1,6.4184344E-1,5.675175E-1,3.5779214E-1,1.1246276E0,1.4691907E-1,2.283005E-1,2.8395385E-1,1.9732544E-1,0E0,2.6274514E-1,1.6848947E-1,1.4394867E-1,7.444237E-2,2.7895808E-1,1.3945916E-1,0E0,1.7450075E-1,0E0,6.353611E-2,3.2486933E-1,0E0,0E0,1.9742656E-1,6.818693E-2,0E0,4.2650983E-2,1.3254523E-1,1.01053625E-1,0E0,3.0574983E-2,0E0,0E0,9.995845E-2,2.4429344E-2,9.104566E-2,1.7706119E-1,0E0,0E0,7.805198E-1,1.561259E-1,0E0,1.1910081E-1,0E0,0E0,0E0,0E0,6.941813E-2,7.722819E-2,7.843513E-2,1.7746517E-1,0E0,0E0,0E0,5.371189E-2,2.0707555E-2,1.5626907E-2,9.193957E-2,1.969734E-1,0E0,1.4817952E-1,1.7025469E-1,0E0,1.6079628E-1,1.7995669E-1,4.787779E-2,1.7374176E-1,3.3131897E-2,0E0,0E0,4.273194E-2,5.482045E-2,0E0,2.2567782E-1,5.4175317E-2,0E0,2.1805331E-2,3.5368964E-2,0E0,0E0,0E0,7.144691E-2,8.71788E-2,9.6197665E-2,6.908185E-2,6.492126E-2,1.3957712E-1,5.867447E-2,0E0,1.1284906E-1,1.2810396E-1,8.286195E-2,0E0,7.086134E-2,0E0,2.101843E-2,6.675249E-2,0E0,0E0,7.765496E-2,0E0,3.1136673E-2,5.8868155E-2,1.157819E-1,3.4604877E-2,0E0,0E0,1.5961781E-2,0E0,3.437814E-2,0E0,5.80431E-2,6.781703E-2,0E0,4.9473673E-2,0E0,0E0,1.8757867E-2,4.6347134E-2,1.9748509E-2,4.3928772E-2,0E0,9.137602E-2,0E0,0E0,2.1756987E-1,8.702904E-2,5.1187996E-2,1.5943608E-1,5.0220292E-2,1.5497029E-2,6.8903446E-2,0E0,0E0,0E0,2.460298E-2,0E0,3.0306697E-2,0E0,7.241744E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.000048E-2,2.8069824E-2,4.8611194E-2,0E0,0E0,5.7869907E-2,0E0,0E0,0E0,0E0,0E0,2.3679346E-2,0E0,0E0,1.0308711E-1,5.6024097E-2,7.968804E-2,1.5469018E-1,3.0349419E-2,7.26192E-2,0E0,0E0,0E0,0E0,0E0,3.880467E-2,0E0,0E0,4.418516E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,27,27,28,28,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,43,43,44,44,46,46,51,51,52,52,53,53,54,54,58,58,59,59,60,60,61,61,62,62,64,64,65,65,67,67,68,68,69,69,70,70,71,71,74,74,75,75,77,77,78,78,80,80,81,81,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,97,97,99,99,100,100,103,103,105,105,106,106,107,107,108,108,111,111,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,139,139,141,141,143,143,155,155,156,156,157,157,160,160,166,166,169,169,170,170,171,171,172,172,173,173,174,174,180,180,183,183],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,-1,42,44,-1,-1,46,48,-1,50,52,54,-1,56,-1,-1,58,60,62,64,-1,-1,66,68,-1,70,-1,-1,-1,-1,72,74,76,78,-1,-1,-1,80,82,84,86,88,-1,90,92,-1,94,96,98,100,102,-1,-1,104,106,-1,108,110,-1,112,114,-1,-1,-1,116,118,120,122,124,126,128,-1,130,132,134,-1,136,-1,138,140,-1,-1,142,-1,144,146,148,150,-1,-1,152,-1,154,-1,156,158,-1,160,-1,-1,162,164,166,168,-1,170,-1,-1,172,174,176,178,180,182,184,-1,-1,-1,186,-1,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,192,194,196,-1,-1,198,-1,-1,-1,-1,-1,200,-1,-1,202,204,206,208,210,212,-1,-1,-1,-1,-1,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1E0,7.098E3,3.7568388E0,5.02E2,4.217427E7,1.2646534E4,7E0,8.317E3,4.519E3,7.82261E6,4.125E0,1.7E1,1.8625151E-2,3.206931E2,1.3050649E4,9.9E1,3.3857143E1,2.277451E2,1.463E2,-9.797345E-3,1.4E1,8.344906E-3,1.94261E5,5.9754E4,-1.7415055E-3,-1.265517E-2,2.0778275E-1,6.9879E4,1.0930074E-2,6.228E1,9.2058825E-1,1.1891E4,-5.618569E-3,3.6E1,-2.4994403E-2,-7.578095E-3,2.824856E-3,2.1481E4,1E0,8.6E1,9.800655E-3,3.1952346E-3,2.466E3,4.6603775E0,-1.0179935E-3,7.153514E1,-4.9310275E-3,2.442503E-3,-2.3401529E-3,3.440138E-3,3.89E2,5E0,1.3E1,1.7717391E0,3.0979766E-3,-1.5881547E-3,4.572157E-4,9.917037E2,4.5726547E5,4.5087484E5,4.9390244E0,2.0217392E0,4.977003E-3,2.7534653E2,1.1E1,-1.9496838E-2,2E0,1.836095E6,1.7655972E0,1.3388E4,4.456432E6,-2.5109504E-3,1.3077036E-3,3.2856784E7,6.4153387E-6,2.5729572E-3,1.169375E2,3.8575E4,-8.080106E-3,4.325E0,3.8767453E4,-2.7835057E-3,-1.0223103E-3,-4.587428E-3,1.6494E4,5.633663E0,3.2962964E0,4.4970587E2,1.9578822E5,1.5E1,1E0,-4.966726E-3,1.3457517E10,1.8645384E1,2.2E1,5.948917E-3,2.5045E4,3.0973414E-3,3.8923203E8,3.8537518E2,-7.1715E-3,-1.4275366E-2,2.28E4,-7.2465115E-3,2.866353E2,2.0549193E2,1E1,1.4386049E6,-9.179282E-3,-2.7952767E-3,1.6341463E0,-5.434599E-3,8.201515E5,3.5828743E-3,4.0351807E2,4.519E3,-6.3834814E-3,7.5E-1,-1.09247165E-2,-2.5168979E-3,1.0003492E7,2.978142E0,2.0261577E2,3.9939122E5,-8.509088E-3,8.931E3,5.828922E-4,6.093556E-3,1.4872598E8,5.5177975E6,3.72381E5,1E1,2.5367088E2,9.525663E2,1.752E3,1.3439045E-2,-2.1326744E-3,1.8007722E-3,1E0,9.746061E-3,4.294737E1,2.0176284E-3,2.8387096E0,9.327427E-4,-6.6219675E-3,-1.6317331E-3,-1.1597686E-2,-8.2730426E-4,-3.2230516E-4,4.5167725E-3,-3.4073547E-3,4.7528482E-7,1.1356301E-3,-1.9241748E-3,4.5E1,8.82E2,1.5321098E8,-8.077768E-4,-4.180844E-3,1.6643229E1,-1.5501073E-3,1.0083345E-3,4.472581E-3,-1.0474909E-3,-3.4120228E-4,1.09924164E5,-3.4949712E-3,-1.029082E-2,6.1986052E7,2.593592E6,4.9963706E5,1.5822886E0,1.1481482E0,2.1E1,-1.2253443E-3,4.923842E-3,2.323397E-3,-9.120735E-3,-3.668458E-3,5.388794E6,-5.8822064E-3,-1.5961746E-3,7.0285716E0,1.3548806E-2,1.9197954E-3,5.775997E-3,-6.1476664E-3,-3.3300633E-3,-7.5694895E-4,-5.4485486E-3,-1.4037599E-3,1.0096745E-3,6.1842E-3,1.0308623E-3,4.1973614E-3,1.0633453E-3,2.3505064E-3,-1.4537458E-3,-5.8272686E-3,-9.499235E-4,-3.1516328E-4,5.358671E-3,-3.3846407E-3,1.4188082E-3,5.24261E-4,5.2724076E-3,2.1776587E-3,-6.70844E-3,-4.4813997E-4,3.811271E-3,7.7153766E-3,1.392189E-3,1.4253506E-3,-2.3414674E-3,1.0679259E-2,5.7965866E-3],"split_indices":[2,29,2,68,2,60,48,3,9,44,43,73,3,0,73,43,44,67,67,70,0,8,0,1,1,0,0,53,2,0,62,68,44,0,3,0,0,53,1,30,10,0,0,44,69,0,71,0,0,0,0,1,0,8,69,0,0,0,48,62,60,71,71,0,70,8,0,6,43,49,2,7,0,0,60,52,0,67,9,0,69,48,0,0,0,1,71,69,67,62,8,27,0,46,71,3,0,2,0,47,71,0,0,1,0,67,4,3,43,0,0,68,0,60,0,4,44,0,68,0,0,60,69,70,60,0,44,0,0,12,60,9,8,67,4,0,0,0,0,29,0,62,0,71,0,0,0,0,0,0,0,0,0,0,0,0,2,7,0,0,73,0,0,0,0,0,66,0,0,5,9,43,57,68,3,0,0,0,0,0,62,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.035E3,6.67E2,3.68E2,2.43E2,4.24E2,2.52E2,1.16E2,2.16E2,2.7E1,1.03E2,3.21E2,2.38E2,1.4E1,7E0,1.09E2,1.3E1,2.03E2,1.5E1,1.2E1,9.7E1,6E0,3.15E2,6E0,1.7E1,2.21E2,7E0,7E0,1E2,9E0,4E0,9E0,8.3E1,1.2E2,5E0,1E1,8E0,4E0,4.6E1,5.1E1,2.14E2,1.01E2,1.1E1,6E0,2.7E1,1.94E2,7E0,9.3E1,5E0,4E0,5E0,4E0,1.7E1,6.6E1,8.2E1,3.8E1,5E0,5E0,7E0,3.9E1,4E1,1.1E1,1.59E2,5.5E1,7E0,9.4E1,2.3E1,4E0,1.28E2,6.6E1,5.6E1,3.7E1,1.3E1,4E0,5E0,6.1E1,7.4E1,8E0,2E1,1.8E1,1.2E1,2.7E1,3.4E1,6E0,6E0,5E0,1.18E2,4.1E1,1.3E1,4.2E1,2.5E1,6.9E1,1.6E1,7E0,1.08E2,2E1,5.6E1,1E1,5.1E1,5E0,1E1,2.7E1,9E0,4E0,5.1E1,1E1,5.3E1,2.1E1,8E0,1.2E1,1.3E1,5E0,1.4E1,1.3E1,3E1,4E0,5.1E1,6.7E1,4E0,3.7E1,4E0,9E0,2.7E1,1.5E1,1.4E1,1.1E1,4E0,6.5E1,9E0,7E0,5.8E1,5E1,1.1E1,9E0,4.7E1,9E0,4.4E1,7E0,6E0,4E0,1.8E1,9E0,4.7E1,4E0,4.2E1,1.1E1,9E0,1.2E1,4E0,4E0,5E0,7E0,1E1,4E0,1.3E1,1.7E1,4.2E1,9E0,5.8E1,9E0,4E0,3.3E1,1.9E1,8E0,1.1E1,4E0,5E0,9E0,5E0,6E0,3.6E1,2.9E1,4.3E1,1.5E1,1.5E1,3.5E1,4E0,7E0,4E0,5E0,5E0,4.2E1,5E0,4E0,4E1,4E0,7E0,1.1E1,1.4E1,3.3E1,3.4E1,8E0,2.1E1,2.1E1,4E0,5E0,4.3E1,1.5E1,1.9E1,1.4E1,5E0,4E0,2.8E1,8E0,2.3E1,6E0,1E1,3.3E1,8E0,7E0,5E0,1E1,3E1,5E0,3.6E1,6E0,6E0,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-2.246716E-3,-7.778354E-2,2.6095033E-2,-6.0222115E-2,-2.0953187E-1,-1.3702271E-3,8.0813855E-2,-7.349078E-2,4.4343524E-2,-3.8790664E-1,-1.0271237E-1,1.966771E-3,-9.652232E-3,1.6765498E-2,7.482498E-2,-6.616697E-2,-1.587768E-1,1.0149643E-2,-3.2918174E-2,-2.2362106E-2,-7.230836E-3,-1.4324442E-1,2.1276942E-4,-5.9582265E-3,7.996801E-2,6.428155E-2,1.7563073E-1,-9.900151E-2,-4.2671546E-2,-1.3761507E-2,-9.30642E-2,2.6838355E-2,-4.6815434E-3,-1.2896681E-3,-1.793611E-1,-6.2180858E-2,2.1290486E-3,1.01124294E-1,-2.418476E-3,6.837701E-2,-6.06279E-3,1.0513901E-2,1.12644E-3,-8.052767E-2,-2.0621286E-1,6.570158E-3,-5.5054422E-2,-5.5885073E-3,-1.1193283E-3,-2.6713351E-3,5.596371E-3,-1.2341322E-2,-4.7161425E-3,-5.3587213E-2,-6.50655E-3,5.279459E-3,-8.045472E-3,6.594445E-2,1.4484148E-1,7.723972E-2,-3.0845776E-2,-9.442909E-2,3.2384848E-4,-1.3958449E-2,-4.3673394E-3,-4.132916E-2,-1.2295029E-1,-6.0812492E-2,1.2907062E-3,9.902494E-3,-9.700982E-2,9.194517E-2,5.7258084E-4,7.790057E-3,2.4970826E-3,6.985779E-2,1.8089138E-1,2.043353E-2,-8.247202E-3,-2.6567988E-2,-1.09578416E-1,-2.1340686E-3,1.85485E-3,-8.3295904E-2,-2.5017614E-2,-6.638835E-3,-1.6682023E-3,-9.239915E-4,-7.999419E-2,1.4523398E-2,-5.5371094E-2,-1.5578517E-2,2.8349573E-3,2.590712E-3,6.577774E-3,3.1344898E-2,8.6996354E-2,1.0408034E-2,2.9937655E-3,8.240165E-2,-2.8716796E-3,-3.4051596E-3,1.8808118E-3,-1.2041012E-1,-4.4448044E-2,-7.4528984E-4,-4.51952E-3,-4.939115E-2,3.6760774E-3,-6.321094E-3,-5.6336038E-2,7.0906878E-3,6.129466E-2,9.6387195E-4,-8.617695E-2,-2.5871943E-3,1.6265125E-3,-3.3372953E-3,3.9881147E-2,1.7698576E-1,6.657143E-2,-6.3252065E-4,7.4849008E-3,-3.7093188E-3,-6.952874E-3,9.4599277E-4,-4.642216E-3,1.5494056E-3,-2.7639477E-3,-2.230725E-3,1.4149114E-3,1.1848741E-3,-3.3805314E-3,1.6459491E-3,-2.0856343E-4,4.851957E-3,1.080785E-3,-5.902269E-3,-8.4228953E-4,5.3548994E-3,8.861318E-4,8.991867E-3,4.2320495E-3,4.317499E-3,-2.3936133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,31,-1,-1,33,-1,35,37,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,-1,59,-1,-1,-1,61,63,-1,65,-1,-1,-1,-1,-1,-1,67,-1,69,-1,71,73,75,77,79,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,95,97,99,-1,101,103,-1,-1,105,107,-1,-1,-1,109,111,113,-1,115,-1,-1,117,119,-1,-1,121,-1,-1,-1,123,125,-1,-1,127,129,-1,131,133,135,-1,137,-1,-1,-1,139,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2285092E0,6.444689E-1,1.1377809E0,3.528955E-1,5.88936E-1,3.5018238E-1,4.002546E-1,1.2940884E-1,4.0011975E-1,1.911695E-1,9.635559E-2,3.0832E-1,0E0,0E0,2.533568E-1,1.5623945E-1,1.2766632E-1,0E0,8.857402E-2,0E0,0E0,6.163782E-2,0E0,2.06106E-1,1.3406324E-1,1.9073576E-1,1.7516172E-1,1.5372622E-1,2.8942782E-1,0E0,2.203741E-2,1.0280403E-1,0E0,0E0,5.4124832E-2,3.0981809E-2,2.213914E-1,5.1967084E-2,0E0,1.984849E-1,0E0,0E0,0E0,8.432683E-2,9.999162E-2,0E0,1.0301763E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2918215E-2,0E0,1.8585946E-1,0E0,3.2277003E-2,2.7586013E-2,1.442188E-1,1.4752237E-1,6.288397E-2,2.4085097E-2,0E0,0E0,6.5001294E-2,2.904895E-2,3.6759675E-2,0E0,1.139128E-1,4.0865272E-1,2.0959139E-2,0E0,0E0,0E0,1.2466395E-1,5.2401185E-2,8.189341E-2,0E0,4.4229094E-2,3.1332552E-2,0E0,0E0,2.4773046E-2,5.0535973E-2,0E0,0E0,0E0,3.6659285E-2,1.2158606E-1,6.0509697E-2,0E0,2.6731653E-2,0E0,0E0,5.6817323E-2,2.3267531E-1,0E0,0E0,7.106991E-2,0E0,0E0,0E0,4.1030645E-2,3.479987E-2,0E0,0E0,3.5758115E-2,4.9091406E-2,0E0,3.6385737E-2,1.0083707E-1,7.70946E-2,0E0,4.8524186E-2,0E0,0E0,0E0,8.7484166E-2,1.833278E-2,3.549119E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,34,34,35,35,36,36,37,37,39,39,43,43,44,44,46,46,53,53,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,69,69,70,70,71,71,75,75,76,76,77,77,79,79,80,80,83,83,84,84,88,88,89,89,90,90,92,92,95,95,96,96,99,99,103,103,104,104,107,107,108,108,110,110,111,111,112,112,114,114,118,118,119,119,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,32,-1,-1,34,-1,36,38,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,-1,60,-1,-1,-1,62,64,-1,66,-1,-1,-1,-1,-1,-1,68,-1,70,-1,72,74,76,78,80,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,96,98,100,-1,102,104,-1,-1,106,108,-1,-1,-1,110,112,114,-1,116,-1,-1,118,120,-1,-1,122,-1,-1,-1,124,126,-1,-1,128,130,-1,132,134,136,-1,138,-1,-1,-1,140,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0162934E5,4.6463413E0,5.862504E2,2.83E3,2E0,1E0,1.4E1,2.8025E4,1.11791E5,1.2E1,8.499432E5,4.5866325E6,-9.652232E-3,1.6765498E-2,6.663214E6,2.71E2,6.15E2,1.0149643E-2,4.36853E5,-2.2362106E-2,-7.230836E-3,1.00011E5,2.1276942E-4,4.33E2,1.9E1,5.619497E7,1.213274E9,6.769866E7,2.00087E5,-1.3761507E-2,2.2E1,2.35184E5,-4.6815434E-3,-1.2896681E-3,8.2E1,3.310366E4,5.6E1,3.7788504E7,-2.418476E-3,7.34E0,-6.06279E-3,1.0513901E-2,1.12644E-3,1.9E1,1.76421E6,6.570158E-3,3.2856784E7,-5.5885073E-3,-1.1193283E-3,-2.6713351E-3,5.596371E-3,-1.2341322E-2,-4.7161425E-3,5.30577E-1,-6.50655E-3,4.75356E6,-8.045472E-3,7.6116E4,1.3922E4,4.907764E3,6.70019E1,1.957E3,7.701384E3,-1.3958449E-2,-4.3673394E-3,1.8401923E2,1.7443357E8,2.57E2,1.2907062E-3,3.0084E4,2.1512408E5,2.3043478E2,5.7258084E-4,7.790057E-3,2.4970826E-3,8.1407714E-1,3.2295492E0,2.7664675E5,-8.247202E-3,3.4E1,2.7546012E0,-2.1340686E-3,1.85485E-3,3.1E1,1.453785E6,-6.638835E-3,-1.6682023E-3,-9.239915E-4,3.01E2,1.812513E8,1.5215946E0,-1.5578517E-2,8.599521E5,2.590712E-3,6.577774E-3,1.0775862E0,3.3382E4,1.0408034E-2,2.9937655E-3,3E0,-2.8716796E-3,-3.4051596E-3,1.8808118E-3,1.5661134E0,7.9634375E2,-7.4528984E-4,-4.51952E-3,2.019537E2,1.5152774E2,-6.321094E-3,1.5310282E8,1.3364486E0,1.2E1,9.6387195E-4,1.3092E4,-2.5871943E-3,1.6265125E-3,-3.3372953E-3,2E0,8.7E1,2.9366477E0,-6.3252065E-4,7.4849008E-3,-3.7093188E-3,-6.952874E-3,9.4599277E-4,-4.642216E-3,1.5494056E-3,-2.7639477E-3,-2.230725E-3,1.4149114E-3,1.1848741E-3,-3.3805314E-3,1.6459491E-3,-2.0856343E-4,4.851957E-3,1.080785E-3,-5.902269E-3,-8.4228953E-4,5.3548994E-3,8.861318E-4,8.991867E-3,4.2320495E-3,4.317499E-3,-2.3936133E-3],"split_indices":[43,68,67,2,32,106,0,44,1,8,60,43,0,0,43,2,2,0,1,0,0,5,0,2,8,62,7,7,5,0,71,1,0,0,0,48,3,7,0,61,0,0,0,3,46,0,60,0,0,0,0,0,0,53,0,62,0,12,44,4,73,9,43,0,0,70,7,44,0,44,48,4,0,0,0,53,53,48,0,2,68,0,0,0,9,0,0,0,44,5,69,0,43,0,0,68,9,0,0,8,0,0,0,68,48,0,0,4,67,0,46,68,3,0,10,0,0,0,32,8,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,2.83E2,7.56E2,2.51E2,3.2E1,5.04E2,2.52E2,2.23E2,2.8E1,1.1E1,2.1E1,4.97E2,7E0,4E0,2.48E2,2.07E2,1.6E1,8E0,2E1,7E0,4E0,1.5E1,6E0,4.52E2,4.5E1,2.26E2,2.2E1,8.5E1,1.22E2,4E0,1.2E1,1.1E1,9E0,4E0,1.1E1,5.6E1,3.96E2,3.9E1,6E0,2.22E2,4E0,1.6E1,6E0,7.4E1,1.1E1,7E0,1.15E2,8E0,4E0,6E0,5E0,4E0,7E0,5.2E1,4E0,3.9E2,6E0,2.3E1,1.6E1,2.04E2,1.8E1,6.3E1,1.1E1,5E0,6E0,9.7E1,1.8E1,4.8E1,4E0,3.74E2,1.6E1,1.5E1,8E0,1.2E1,4E0,1.92E2,1.2E1,1.4E1,4E0,1.2E1,5.1E1,5E0,6E0,2.6E1,7.1E1,1.4E1,4E0,1.6E1,3.2E1,3.5E2,2.4E1,4E0,1.2E1,1E1,5E0,6E1,1.32E2,8E0,4E0,8E0,6E0,7E0,5E0,4.3E1,8E0,5E0,2.1E1,3.8E1,3.3E1,8E0,2.4E1,3.03E2,4.7E1,7E0,1.7E1,4E0,8E0,4E0,5.6E1,2.3E1,1.09E2,4E0,4E0,2E1,2.3E1,4E0,4E0,4E0,3.4E1,1.1E1,2.2E1,4E0,2E1,8.7E1,2.16E2,2.1E1,2.6E1,1E1,7E0,1.1E1,4.5E1,1.8E1,5E0,8.9E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[-2.567656E-3,-5.573658E-2,3.8479526E-2,-4.8533235E-2,-2.565958E-1,2.383949E-2,1.2916835E-1,-8.2375884E-2,-2.506622E-2,-1.01484045E-1,-2.524882E-2,3.0910918E-2,-1.4459175E-1,1.7562997E-1,2.1786032E-2,-7.585837E-2,-1.6404082E-1,7.0511973E-3,-3.0742949E-2,5.20484E-4,-9.116005E-3,1.9763673E-2,1.1887469E-1,-1.1419385E-2,-4.3389145E-3,1.33511E-1,3.1543034E-1,-2.4449607E-2,4.9294084E-3,-1.03909306E-1,-5.7610177E-2,-1.1580421E-2,-4.130033E-3,-2.3796668E-2,-1.12480275E-1,9.005744E-3,1.6761377E-2,-1.0729339E-3,1.3416049E-1,1.5866417E-1,-1.0213535E-3,1.8529927E-2,7.5269165E-3,-5.9708287E-3,4.4458758E-2,-8.619171E-2,-1.0779947E-2,-8.496841E-3,-7.923898E-2,-2.7775029E-2,4.8866454E-3,-8.242472E-3,-5.9152335E-2,2.1102395E-2,-8.611082E-2,7.2912045E-2,1.5482672E-1,1.2456751E-2,1.09403E-1,4.7205263E-3,-9.54947E-4,-2.3603722E-2,-9.9006094E-2,-5.842033E-2,1.3578452E-2,-8.600494E-2,3.3565337E-4,-4.5528298E-4,-4.440114E-2,-4.649631E-3,-4.0993546E-5,-2.8081415E-2,2.806836E-2,-2.7918817E-2,-8.692029E-3,5.204056E-3,-9.764041E-4,2.9035744E-3,7.7642733E-3,1.5350669E-3,1.2866788E-1,-2.9587694E-3,1.743244E-3,-1.1882344E-1,-2.7939728E-3,-4.856889E-3,4.3033942E-4,3.2742673E-3,-2.8316148E-2,-6.6414364E-3,-7.1578465E-2,1.208197E-2,-5.976793E-2,-3.6334645E-2,-1.1628438E-2,3.4833124E-3,-8.955902E-2,3.29158E-2,-3.3604197E-2,-4.1238335E-3,3.502559E-3,1.5145591E-1,9.654395E-4,-8.666436E-2,-1.5984003E-1,6.828456E-4,-2.4901757E-3,-5.582565E-2,-6.2511154E-3,-3.733035E-2,3.219132E-2,-6.5755304E-3,1.3982398E-3,-2.4311658E-2,-9.987207E-2,-3.6536732E-3,2.0446392E-2,-6.3733445E-3,-2.338618E-3,1.0011721E-2,5.3484317E-2,-5.435336E-3,-1.2975327E-2,7.96133E-3,2.8979066E-3,-1.6586041E-3,-4.958629E-3,-9.009088E-3,-3.3081567E-3,1.12197835E-4,-3.0778085E-3,-5.7873293E-3,-9.332417E-5,2.0442274E-3,-1.7622969E-3,4.8791593E-5,-2.178759E-3,-7.192644E-3,-1.6158909E-3,3.5146428E-3,-2.0182246E-4,1.2474451E-3,-2.26424E-3,9.338563E-4,3.910048E-3,2.6906333E-3,-1.7253913E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,-1,-1,35,37,-1,-1,39,41,43,-1,45,47,-1,-1,49,51,-1,53,-1,55,57,-1,-1,-1,-1,59,61,-1,63,65,67,-1,-1,69,71,73,75,77,-1,79,-1,-1,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,-1,-1,-1,-1,-1,-1,101,-1,-1,103,-1,-1,-1,-1,105,-1,107,109,111,113,-1,115,117,119,121,-1,-1,123,-1,125,127,-1,-1,129,-1,131,133,-1,-1,135,137,-1,139,-1,-1,141,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.204221E0,6.241642E-1,7.537026E-1,3.362584E-1,6.191022E-1,5.9046566E-1,3.9031708E-1,8.016801E-2,2.5989658E-1,1.21084E-1,0E0,4.6145254E-1,8.075446E-2,2.817402E-1,1.0075929E-1,7.745004E-2,5.5182368E-2,0E0,1.3699488E-1,0E0,0E0,2.1814148E-1,1.1823678E-1,0E0,0E0,1.7573696E-1,9.1415405E-2,1.2850341E-1,0E0,1.2435657E-1,1.06615275E-1,0E0,0E0,1.204668E-1,5.625072E-2,0E0,1.8714371E-1,0E0,4.9468398E-2,1.782251E-1,0E0,0E0,0E0,0E0,4.3463226E-2,4.337862E-2,0E0,3.6059678E-2,4.1594774E-2,1.00816876E-1,0E0,0E0,2.892045E-2,1.3749069E-1,9.568407E-2,5.4694273E-2,3.2042444E-2,0E0,3.6815196E-2,0E0,0E0,2.9701915E-2,2.870065E-2,3.2617927E-2,5.7214536E-2,4.3865412E-2,0E0,6.392962E-2,2.2161794E-1,0E0,0E0,9.7338155E-2,1.0574001E-1,8.312775E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.086428E-2,0E0,0E0,2.5719225E-2,0E0,0E0,0E0,0E0,1.6233398E-2,0E0,4.6658218E-2,7.175084E-2,1.1580853E-1,9.9935815E-2,0E0,4.9238116E-2,2.5052026E-2,1.5257299E-1,4.355167E-2,0E0,0E0,2.252385E-2,0E0,1.6080424E-2,2.3269981E-2,0E0,0E0,2.8460115E-2,0E0,6.4955235E-2,4.49384E-2,0E0,0E0,6.612628E-2,7.0620835E-2,0E0,4.036531E-2,0E0,0E0,1.567538E-1,1.7551258E-1,0E0,4.0358942E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,25,25,26,26,27,27,29,29,30,30,33,33,34,34,36,36,38,38,39,39,44,44,45,45,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,64,64,65,65,67,67,68,68,71,71,72,72,73,73,80,80,83,83,88,88,90,90,91,91,92,92,93,93,95,95,96,96,97,97,98,98,101,101,103,103,104,104,107,107,109,109,110,110,113,113,114,114,116,116,119,119,120,120,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,-1,-1,36,38,-1,-1,40,42,44,-1,46,48,-1,-1,50,52,-1,54,-1,56,58,-1,-1,-1,-1,60,62,-1,64,66,68,-1,-1,70,72,74,76,78,-1,80,-1,-1,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,-1,-1,-1,-1,-1,-1,102,-1,-1,104,-1,-1,-1,-1,106,-1,108,110,112,114,-1,116,118,120,122,-1,-1,124,-1,126,128,-1,-1,130,-1,132,134,-1,-1,136,138,-1,140,-1,-1,142,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.31E2,6.0692043E0,1.6466942E3,4.13E2,1.6493898E6,4.217427E7,2.2040408E7,3.578882E1,1.2E1,4.6683E7,-2.524882E-2,4.7429975E6,1.0479E4,3.8771296E3,2.6108465E0,3.9125E1,3.3817584E7,7.0511973E-3,8.7E1,5.20484E-4,-9.116005E-3,2.04115E5,1.0378262E10,-1.1419385E-2,-4.3389145E-3,1E1,1.159499E6,1.3655363E2,4.9294084E-3,2.161111E1,4.577342E0,-1.1580421E-2,-4.130033E-3,3.5E1,1.31306E5,9.005744E-3,1.2294118E1,-1.0729339E-3,1E0,1.6386554E0,-1.0213535E-3,1.8529927E-2,7.5269165E-3,-5.9708287E-3,2.5130852E2,1.957E3,-1.0779947E-2,5.684E3,3.95E2,7.9227E4,4.8866454E-3,-8.242472E-3,6.354E3,1.8439855E5,3.3924444E5,3.82E2,1.28738E5,1.2456751E-2,4.1032645E6,4.7205263E-3,-9.54947E-4,2.125E1,3.9E1,1.8187732E4,1.2E1,3.3817584E7,3.3565337E-4,1.1E1,2.8302418E10,-4.649631E-3,-4.0993546E-5,3.63E2,1.6673345E7,5.0563266E2,-8.692029E-3,5.204056E-3,-9.764041E-4,2.9035744E-3,7.7642733E-3,1.5350669E-3,1.0905187E9,-2.9587694E-3,1.743244E-3,2.6412E5,-2.7939728E-3,-4.856889E-3,4.3033942E-4,3.2742673E-3,2.3412812E0,-6.6414364E-3,1.5E1,7.5793734E8,1.994E3,9.07826E0,-1.1628438E-2,5.1E1,1.6E1,1.862E3,4.48375E5,-4.1238335E-3,3.502559E-3,1.2792593E2,9.654395E-4,2.88E0,4.631579E0,6.828456E-4,-2.4901757E-3,2.0563E4,-6.2511154E-3,3.3817584E7,1E0,-6.5755304E-3,1.3982398E-3,1E0,1.95E2,-3.6536732E-3,1.5853742E1,-6.3733445E-3,-2.338618E-3,8.009E3,5.2692295E6,-5.435336E-3,5.003246E1,7.96133E-3,2.8979066E-3,-1.6586041E-3,-4.958629E-3,-9.009088E-3,-3.3081567E-3,1.12197835E-4,-3.0778085E-3,-5.7873293E-3,-9.332417E-5,2.0442274E-3,-1.7622969E-3,4.8791593E-5,-2.178759E-3,-7.192644E-3,-1.6158909E-3,3.5146428E-3,-2.0182246E-4,1.2474451E-3,-2.26424E-3,9.338563E-4,3.910048E-3,2.6906333E-3,-1.7253913E-3],"split_indices":[2,68,67,2,60,60,66,61,10,7,0,43,9,67,50,4,7,0,8,0,0,5,46,0,0,6,9,71,0,67,73,0,0,3,9,0,68,0,27,69,0,0,0,0,73,9,0,9,2,9,0,0,10,43,43,0,1,0,60,0,0,4,2,43,33,7,0,8,46,0,0,0,47,67,0,0,0,0,0,0,7,0,0,7,0,0,0,0,68,0,8,46,12,71,0,0,3,2,1,0,0,73,0,69,69,0,0,12,0,7,19,0,0,26,10,0,73,0,0,44,60,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,4.39E2,5.69E2,4.25E2,1.4E1,4.91E2,7.8E1,1.73E2,2.52E2,1E1,4E0,4.72E2,1.9E1,5.4E1,2.4E1,1.62E2,1.1E1,7E0,2.45E2,5E0,5E0,4.2E2,5.2E1,5E0,1.4E1,4.3E1,1.1E1,1.6E1,8E0,6.2E1,1E2,4E0,7E0,2.27E2,1.8E1,6E0,4.14E2,5E0,4.7E1,3.7E1,6E0,6E0,5E0,6E0,1E1,5.6E1,6E0,3.1E1,6.9E1,2.21E2,6E0,7E0,1.1E1,3.98E2,1.6E1,1.3E1,3.4E1,1E1,2.7E1,5E0,5E0,1E1,4.6E1,9E0,2.2E1,6.4E1,5E0,8.4E1,1.37E2,6E0,5E0,4.9E1,3.49E2,1.1E1,5E0,9E0,4E0,5E0,2.9E1,6E0,2.1E1,6E0,4E0,2.9E1,1.7E1,5E0,4E0,9E0,1.3E1,1.1E1,5.3E1,7E1,1.4E1,1.33E2,4E0,3.3E1,1.6E1,3.24E2,2.5E1,7E0,4E0,1.7E1,4E0,1.8E1,1.1E1,5E0,8E0,4.4E1,9E0,2E1,5E1,7E0,7E0,1.13E2,2E1,5E0,2.8E1,6E0,1E1,1.54E2,1.7E2,4E0,2.1E1,1.3E1,4E0,6E0,1.2E1,7E0,4E0,7E0,3.7E1,5E0,1.5E1,4.3E1,7E0,5.4E1,5.9E1,1E1,1E1,8E0,2E1,1.2E2,3.4E1,8.3E1,8.7E1,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[3.7052329E-3,-1.3392239E-2,1.1293301E-1,-3.8338415E-2,2.3857282E-2,1.631091E-2,9.9645585E-2,-7.9138584E-2,-1.8889999E-2,1.0220474E-2,8.719398E-2,8.8257894E-2,1.1515891E-2,-6.900184E-2,-1.884737E-2,-1.07211374E-1,-7.146279E-3,-5.1901132E-2,3.0515952E-2,7.0185095E-2,1.627654E-1,-3.3053036E-3,9.6773274E-2,-6.064249E-2,-9.086762E-3,-7.579222E-3,-7.711027E-2,1.5004916E-1,-1.685171E-2,-1.7882802E-2,-1.104559E-1,4.8900127E-2,2.192085E-3,5.4561675E-2,6.166278E-3,9.617119E-3,3.0791692E-3,1.1367391E-1,3.3723872E-2,-6.82102E-2,5.340218E-3,-3.0035583E-2,-1.4521934E-1,3.5626607E-3,8.600763E-3,-8.644546E-3,-9.2343025E-2,-8.9038186E-2,3.185411E-3,-1.1556455E-2,-5.86331E-2,5.569412E-2,-6.5417853E-3,-7.152324E-2,2.7092975E-2,1.15076E-3,7.5273104E-2,1.3409376E-1,5.43253E-2,2.963826E-4,5.224055E-3,-4.7544133E-2,-1.0290723E-1,-7.5654134E-2,1.858653E-2,-1.0400868E-2,-3.5629508E-3,1.4883763E-3,-7.268112E-2,3.9379247E-3,-1.2294161E-1,-2.9599204E-4,-8.635267E-3,-5.6373373E-2,3.2842286E-2,-7.337421E-3,2.8057268E-3,4.40668E-2,1.69264E-1,1.3119429E-2,-1.071209E-2,4.7601566E-2,-3.7232894E-2,5.6640417E-3,1.9868102E-3,1.4637332E-1,1.2322719E-4,7.3623896E-2,7.856037E-5,1.1585838E-3,-3.6526981E-3,-8.506636E-2,-2.4752598E-2,-7.048921E-3,-5.9334535E-2,-5.853912E-3,-9.4698905E-4,3.216998E-3,-1.2496985E-3,-2.4807671E-2,1.7891357E-2,-9.8883994E-2,2.6558195E-3,-1.5565075E-1,2.1051473E-3,-8.9713087E-4,-4.4466844E-3,3.2608781E-3,4.6004853E-4,-2.9323524E-2,3.0723794E-3,1.0482689E-1,3.0395623E-2,9.968748E-3,2.8176096E-3,2.7873942E-3,-3.0307194E-2,7.570677E-3,8.164338E-2,-7.4500754E-2,2.0100423E-3,1.2345921E-1,1.1908433E-2,7.607614E-4,4.2900825E-3,3.0274796E-5,-9.848085E-2,-3.4731673E-4,-9.076999E-2,-5.730533E-3,-3.259769E-2,1.2830986E-3,-7.012057E-2,6.547583E-2,6.3389516E-3,-1.2570807E-1,-7.560943E-4,-5.411486E-3,-1.21586835E-2,1.4184571E-3,-1.7781866E-3,7.824094E-4,-3.6483293E-3,1.7556569E-3,1.2224585E-1,5.5783026E-2,3.3848137E-3,9.5232425E-4,-3.4726185E-3,1.0377073E-3,-2.0016988E-3,9.488311E-4,9.4034225E-2,-1.3008444E-3,-6.1791227E-3,1.1894656E-3,1.371726E-1,-7.4322456E-3,-3.5986681E-3,-2.763639E-3,1.4912166E-3,-1.2982892E-3,-5.291629E-3,-2.5039813E-3,1.416145E-4,-7.4322213E-4,1.4791973E-3,4.4492108E-4,-5.0209644E-3,3.8492386E-3,-1.7016523E-3,6.380402E-4,-2.895762E-3,-9.252627E-3,-4.145214E-3,2.6918112E-3,6.654921E-3,1.6192447E-3,7.643047E-3,-3.6111416E-3,9.59977E-4,5.3370306E-3,2.2070282E-3,3.2869226E-3,7.3763076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,35,-1,37,39,-1,-1,41,43,45,47,49,51,53,55,-1,-1,-1,57,59,61,-1,63,65,-1,-1,67,69,71,73,-1,75,77,-1,79,81,-1,83,85,87,89,-1,91,93,95,97,-1,-1,99,101,-1,103,-1,-1,105,107,-1,109,111,113,115,-1,117,119,-1,-1,121,-1,123,-1,-1,-1,125,127,-1,129,-1,-1,-1,-1,131,133,135,-1,137,-1,-1,-1,-1,139,141,-1,143,145,-1,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,157,159,161,-1,163,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,175,177,179,-1,-1,-1,-1,-1,181,-1,-1,-1,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9455996E0,8.381537E-1,4.1469884E-1,4.2699432E-1,3.1177196E-1,0E0,2.0801866E-1,5.5182314E-1,3.7989414E-1,3.7855935E-1,7.0841014E-2,1.7974281E-1,0E0,1.6952473E-1,0E0,6.092608E-2,4.9836463E-1,1.4542796E-1,1.17573485E-1,4.611045E-2,3.4127384E-2,0E0,1.2510109E-1,2.2260833E-1,0E0,0E0,9.1824666E-2,3.569433E-2,1.8940955E-1,7.224123E-2,1.8078822E-1,1.8519932E-1,1.6713972E-1,2.5761992E-2,0E0,0E0,0E0,1.0784495E-1,7.158613E-2,1.06250286E-1,0E0,4.3549493E-2,4.793948E-2,0E0,0E0,1.8049617E-1,1.7349139E-1,8.776646E-2,6.897188E-2,0E0,1.2971076E-1,1.6816932E-1,0E0,3.1519854E-1,9.1480196E-2,0E0,3.443353E-2,1.125797E-1,2.6280023E-2,4.131667E-2,0E0,8.255811E-2,1.1430943E-1,2.5563471E-2,2.5179796E-2,0E0,0E0,1.04395226E-1,1.3529351E-1,0E0,1.5039843E-1,0E0,0E0,1.7288294E-2,3.1743065E-2,0E0,3.0601783E-2,9.8154455E-2,4.72762E-2,3.4634795E-2,0E0,7.024355E-2,5.439528E-2,0E0,0E0,1.3153076E-1,0E0,1.9361801E-2,0E0,0E0,0E0,4.26113E-2,1.0147237E-1,0E0,5.1317275E-2,0E0,0E0,0E0,0E0,1.10039085E-1,8.14925E-2,6.877366E-2,0E0,6.0289502E-2,0E0,0E0,0E0,0E0,1.8851176E-2,2.4303338E-2,0E0,2.0962834E-2,6.907034E-2,0E0,0E0,0E0,2.2107825E-2,1.9921387E-2,1.92689E-2,2.9183663E-2,0E0,7.016677E-2,0E0,0E0,0E0,0E0,2.8570592E-2,9.3126304E-2,2.2104442E-2,0E0,1.8655686E-2,3.2547813E-2,1.0560502E-1,5.6007534E-2,6.3264325E-2,4.6427578E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5321314E-2,1.12425655E-1,7.254931E-2,0E0,0E0,0E0,0E0,0E0,1.7345369E-2,0E0,0E0,0E0,5.9919596E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,37,37,38,38,39,39,41,41,42,42,45,45,46,46,47,47,48,48,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,68,68,70,70,73,73,74,74,76,76,77,77,78,78,79,79,81,81,82,82,85,85,87,87,91,91,92,92,94,94,99,99,100,100,101,101,103,103,108,108,109,109,111,111,112,112,116,116,117,117,118,118,119,119,121,121,126,126,127,127,128,128,130,130,131,131,132,132,133,133,134,134,135,135,144,144,145,145,146,146,152,152,156,156],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,36,-1,38,40,-1,-1,42,44,46,48,50,52,54,56,-1,-1,-1,58,60,62,-1,64,66,-1,-1,68,70,72,74,-1,76,78,-1,80,82,-1,84,86,88,90,-1,92,94,96,98,-1,-1,100,102,-1,104,-1,-1,106,108,-1,110,112,114,116,-1,118,120,-1,-1,122,-1,124,-1,-1,-1,126,128,-1,130,-1,-1,-1,-1,132,134,136,-1,138,-1,-1,-1,-1,140,142,-1,144,146,-1,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,158,160,162,-1,164,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,176,178,180,-1,-1,-1,-1,-1,182,-1,-1,-1,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,8.995735E5,1.2646534E4,3.9595376E7,4.5866325E6,1.631091E-2,9.059046E6,5E0,2.35E2,2.485E2,6.459696E6,9.9688476E-1,1.1515891E-2,2.707256E7,-1.884737E-2,5.4E1,1.2E1,2.6210527E0,9.113598E6,2.712766E0,4.026E3,-3.3053036E-3,1.0907809E9,5E1,-9.086762E-3,-7.579222E-3,3.14403E5,1.4E1,1.7172844E7,1.241995E2,2.0671213E2,1.853776E7,1.0909091E0,2.8387096E0,6.166278E-3,9.617119E-3,3.0791692E-3,8.7E1,4.435876E2,8E0,5.340218E-3,1.0101351E4,8.960784E0,3.5626607E-3,8.600763E-3,2.2715E4,1.2E1,1.2131707E2,9.971819E8,-1.1556455E-2,9.582126E7,8.118189E6,-6.5417853E-3,2.41E3,6.346204E-7,1.15076E-3,1E0,4.6E1,5.712652E8,3.5056704E7,5.224055E-3,1.16408E5,4.9948E4,4.9E2,1.26E2,-1.0400868E-2,-3.5629508E-3,7.69E2,9.0754684E1,3.9379247E-3,2.3E1,-2.9599204E-4,-8.635267E-3,1.68991E6,1.7676923E2,-7.337421E-3,1.7E1,1.1149426E0,1E0,1.2E1,-1.071209E-2,1.157602E6,1.056178E6,5.6640417E-3,1.9868102E-3,1.3439851E7,1.2322719E-4,2.8646374E8,7.856037E-5,1.1585838E-3,-3.6526981E-3,1.11E2,1E0,-7.048921E-3,4.312981E5,-5.853912E-3,-9.4698905E-4,3.216998E-3,-1.2496985E-3,7.547528E6,2.23607E5,1E0,2.6558195E-3,7E0,2.1051473E-3,-8.9713087E-4,-4.4466844E-3,3.2608781E-3,9.695652E0,2.7005264E2,3.0723794E-3,1.1090909E1,1.676E3,9.968748E-3,2.8176096E-3,2.7873942E-3,1.3809524E0,2.6699028E0,1.4956522E1,7E0,2.0100423E-3,7.785302E5,1.1908433E-2,7.607614E-4,4.2900825E-3,3.0274796E-5,1E1,6.44E2,1.5365228E5,-5.730533E-3,8.315271E0,1.6507974E8,1.625E0,2E1,4.9469455E6,4.080551E-1,-7.560943E-4,-5.411486E-3,-1.21586835E-2,1.4184571E-3,-1.7781866E-3,7.824094E-4,-3.6483293E-3,1.7556569E-3,1.142E3,2.364E3,4.646154E0,9.5232425E-4,-3.4726185E-3,1.0377073E-3,-2.0016988E-3,9.488311E-4,2.295413E6,-1.3008444E-3,-6.1791227E-3,1.1894656E-3,1.2697166E3,-7.4322456E-3,-3.5986681E-3,-2.763639E-3,1.4912166E-3,-1.2982892E-3,-5.291629E-3,-2.5039813E-3,1.416145E-4,-7.4322213E-4,1.4791973E-3,4.4492108E-4,-5.0209644E-3,3.8492386E-3,-1.7016523E-3,6.380402E-4,-2.895762E-3,-9.252627E-3,-4.145214E-3,2.6918112E-3,6.654921E-3,1.6192447E-3,7.643047E-3,-3.6111416E-3,9.59977E-4,5.3370306E-3,2.2070282E-3,3.2869226E-3,7.3763076E-3],"split_indices":[2,43,48,7,43,0,43,6,2,67,9,69,0,60,0,2,33,71,60,68,2,0,7,3,0,0,9,3,60,67,4,62,68,71,0,0,0,8,71,32,0,43,73,0,0,44,3,70,46,0,5,60,0,2,52,0,26,6,47,60,0,7,9,1,44,0,0,2,71,0,8,0,0,43,67,0,3,68,19,3,0,12,9,0,0,60,0,12,0,0,0,1,26,0,60,0,0,0,0,5,9,28,0,8,0,0,0,0,73,70,0,73,44,0,0,0,69,68,73,6,0,43,0,0,0,0,3,1,43,0,71,46,68,3,62,53,0,0,0,0,0,0,0,0,44,2,71,0,0,0,0,0,43,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,9E2,1.4E2,5.39E2,3.61E2,6E0,1.34E2,1.73E2,3.66E2,2.98E2,6.3E1,1.26E2,8E0,1.69E2,4E0,4.2E1,3.24E2,7.3E1,2.25E2,5.3E1,1E1,6E0,1.2E2,1.6E2,9E0,1.3E1,2.9E1,1.8E1,3.06E2,4.7E1,2.6E1,1.36E2,8.9E1,4.4E1,9E0,6E0,4E0,9.4E1,2.6E1,1.54E2,6E0,1.8E1,1.1E1,7E0,1.1E1,2.77E2,2.9E1,1E1,3.7E1,6E0,2E1,1.32E2,4E0,2.2E1,6.7E1,1.9E1,2.5E1,6.9E1,2.5E1,1.9E1,7E0,9.8E1,5.6E1,9E0,9E0,4E0,7E0,2.4E2,3.7E1,4E0,2.5E1,6E0,4E0,1.2E1,2.5E1,7E0,1.3E1,1.21E2,1.1E1,1.5E1,7E0,5.1E1,1.6E1,9E0,1.6E1,6.3E1,6E0,1.8E1,7E0,1.5E1,4E0,3.6E1,6.2E1,2.5E1,3.1E1,4E0,5E0,4E0,5E0,9.2E1,1.48E2,3.1E1,6E0,2.1E1,4E0,7E0,5E0,1.1E1,1.4E1,9E0,4E0,2.1E1,1E2,7E0,4E0,7E0,8E0,2.4E1,2.7E1,1.1E1,5E0,5.4E1,9E0,5E0,1.3E1,5E0,3.1E1,4.6E1,1.6E1,8E0,2.3E1,5.9E1,3.3E1,2.8E1,1.2E2,2.3E1,8E0,1.7E1,4E0,8E0,6E0,5E0,4E0,5E0,1.6E1,5.1E1,4.9E1,4E0,4E0,1.9E1,5E0,5E0,2.2E1,7E0,4E0,7E0,4.7E1,6E0,2.5E1,1.6E1,3E1,5E0,1.1E1,1.4E1,9E0,3.8E1,2.1E1,1.1E1,2.2E1,2.4E1,4E0,1.09E2,1.1E1,6E0,1.7E1,5E0,1.1E1,4.4E1,7E0,8E0,4.1E1,1.4E1,8E0,1.3E1,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[-1.6896375E-3,-1.7278235E-2,1.0297271E-1,-4.4850104E-2,1.9811345E-2,1.6026318E-2,8.9046225E-2,-3.1389132E-2,-1.2141681E-1,2.8260003E-3,4.4210114E-2,4.547129E-2,1.4953184E-1,-6.733951E-2,-3.556275E-3,-3.775534E-2,-1.601674E-1,9.572094E-3,-1.3296606E-1,3.524323E-2,1.4289385E-1,2.8366694E-2,1.3575728E-1,1.662894E-1,-2.215249E-3,-1.4110914E-1,-5.950135E-2,1.08520485E-1,-2.2649748E-2,2.023681E-3,-6.4413175E-2,-2.904859E-1,-1.0108153E-1,3.0521434E-3,8.763196E-2,1.1556856E-3,-1.5132353E-2,-7.901481E-2,4.6831153E-2,9.348502E-3,3.716158E-3,4.7347613E-2,-3.2728433E-3,1.16381E-3,8.680404E-3,1.7969016E-3,1.8612354E-1,-1.0776713E-3,-7.996828E-3,3.7161887E-2,-6.4268805E-2,1.9950835E-1,6.2352218E-2,-8.378911E-2,-4.31869E-3,1.7515651E-3,-9.236442E-2,-1.1113867E-1,-2.1064032E-2,1.8901301E-3,-1.23835586E-1,7.6817484E-3,-9.8623894E-2,-6.587486E-4,6.6759977E-3,5.7972938E-2,-1.5577735E-2,-5.913482E-2,6.056969E-2,5.4327203E-3,2.1132376E-2,2.0330381E-1,3.952182E-3,4.524027E-3,-1.4337078E-3,-5.832092E-2,-1.6418974E-1,1.1305392E-2,4.032403E-3,1.1760663E-2,4.3074563E-3,-6.4046737E-3,-4.8607647E-2,1.0349239E-1,-1.2531519E-2,-6.113991E-3,-1.6560567E-3,-8.242859E-3,-9.99243E-4,-1.9602416E-1,-7.325049E-2,-1.1980067E-2,2.502334E-2,-6.9456054E-3,-1.255789E-3,6.4543122E-3,-2.3885854E-3,-2.346451E-2,-6.1406204E-3,7.270009E-2,-1.2208145E-2,-4.8326543E-3,3.712066E-2,3.1616518E-3,1.007249E-2,-7.388208E-2,-3.203063E-2,-2.8101031E-3,-1.0843636E-2,2.15063E-3,-1.1545226E-3,-1.7161662E-2,-7.3208436E-3,-5.3513714E-4,7.5063356E-3,1.9968376E-2,-3.025174E-2,-4.1439617E-3,-1.4096334E-2,-9.6212946E-2,-1.1046136E-3,-1.6949673E-3,-8.9006545E-3,3.688281E-2,-3.0597564E-2,-2.748403E-3,1.7957533E-3,8.150835E-2,-1.9443603E-3,4.0046684E-3,-5.6911085E-2,5.9090937E-3,2.2678224E-2,-5.28683E-2,-1.1962713E-1,3.2593027E-2,-4.6608146E-2,1.9286551E-3,-4.566239E-2,-1.0608669E-2,7.143529E-2,2.7843105E-2,-4.3032102E-2,-5.5495957E-3,-1.3466601E-3,-3.2579906E-2,2.3046885E-2,2.549738E-2,4.60747E-3,1.9420171E-3,-5.276887E-2,7.102301E-2,7.2105103E-3,-9.555584E-2,1.1102412E-3,1.7279412E-3,-2.23772E-2,8.9306E-4,-3.5921354E-3,-7.922401E-3,-4.008574E-3,-1.8745323E-3,4.1880566E-3,-9.312093E-4,-3.3869282E-3,5.7790417E-4,-3.8722993E-3,-1.973576E-3,1.102998E-3,1.9556483E-4,4.2582313E-3,2.9506234E-3,-2.8114258E-3,-3.4112425E-3,-7.950707E-4,1.059894E-3,-2.6423233E-3,-2.0099138E-3,2.0885812E-3,6.782996E-4,4.7667264E-3,-3.8253152E-4,-3.7655951E-3,3.587164E-3,-2.3929263E-3,-1.5564652E-3,-6.389795E-3,-3.2880423E-3,1.4271927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,-1,55,57,59,61,63,-1,-1,65,67,-1,-1,69,-1,-1,-1,-1,71,-1,-1,73,75,77,79,81,83,-1,85,87,-1,-1,89,91,93,-1,-1,95,-1,97,99,-1,101,103,-1,-1,-1,105,107,-1,-1,109,-1,-1,111,113,115,-1,-1,-1,-1,117,119,121,123,-1,-1,-1,-1,125,-1,127,129,-1,131,-1,-1,133,135,-1,-1,-1,-1,137,-1,-1,-1,139,141,-1,-1,143,-1,145,-1,147,149,-1,-1,151,-1,-1,153,-1,155,157,159,161,163,-1,165,167,169,171,173,-1,-1,175,177,179,-1,-1,181,183,-1,185,-1,-1,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7116323E0,9.3454885E-1,4.311037E-1,5.362557E-1,1.6149944E-1,0E0,3.3583617E-1,4.470295E-1,2.4682355E-1,2.1288401E-1,1.3654563E-1,1.1506051E-1,1.866765E-1,1.0435647E-1,5.438614E-1,5.935027E-2,3.7873638E-1,1.122097E-1,3.4633535E-1,1.9965431E-1,3.199938E-2,1.2763725E-1,6.4438E-2,1.190989E-1,0E0,6.377515E-2,8.441746E-2,1.4298806E-1,2.4284074E-1,0E0,6.0704984E-2,4.2410934E-1,1.2784246E-1,9.725519E-2,1.0057299E-1,0E0,0E0,5.1467836E-1,2.0011708E-1,0E0,0E0,1.0064004E-1,0E0,0E0,0E0,0E0,5.6909323E-2,0E0,0E0,4.0332474E-2,9.145838E-2,3.951463E-2,3.9528564E-2,8.9734316E-2,1.500498E-1,0E0,3.140591E-2,5.034226E-2,0E0,0E0,1.061461E-1,6.747031E-2,2.9166162E-2,0E0,0E0,9.686451E-2,0E0,3.8990267E-2,1.06826484E-1,0E0,8.831669E-2,5.6548595E-2,0E0,0E0,0E0,6.3908935E-2,5.133152E-2,0E0,0E0,1.5227805E-2,0E0,0E0,1.0782492E-1,8.333595E-2,9.102121E-2,0E0,0E0,0E0,0E0,1.2343022E-1,2.1492176E-2,1.7249525E-1,7.0390135E-2,0E0,0E0,0E0,0E0,2.9260274E-2,0E0,1.0757071E-1,8.45062E-2,0E0,4.6922967E-2,0E0,0E0,9.076446E-2,5.9926346E-2,0E0,0E0,0E0,0E0,4.5558415E-2,0E0,0E0,0E0,8.888538E-2,7.693039E-2,0E0,0E0,1.8948063E-2,0E0,6.8760276E-2,0E0,5.9710108E-2,3.2815367E-2,0E0,0E0,6.397331E-2,0E0,0E0,4.296054E-2,0E0,2.339222E-2,1.2906748E-1,3.7944794E-2,5.4655436E-2,3.4546264E-2,0E0,4.1000366E-2,4.0881924E-2,2.8873593E-2,6.459721E-2,6.554972E-2,0E0,0E0,5.5977616E-2,7.5711645E-2,6.1495405E-2,0E0,0E0,1.8503688E-2,7.397497E-2,0E0,1.9145362E-2,0E0,0E0,2.5591712E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,37,37,38,38,41,41,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,75,75,76,76,79,79,82,82,83,83,84,84,89,89,90,90,91,91,92,92,97,97,99,99,100,100,102,102,105,105,106,106,111,111,115,115,116,116,119,119,121,121,123,123,124,124,127,127,130,130,132,132,133,133,134,134,135,135,136,136,138,138,139,139,140,140,141,141,142,142,145,145,146,146,147,147,150,150,151,151,153,153,156,156],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,-1,56,58,60,62,64,-1,-1,66,68,-1,-1,70,-1,-1,-1,-1,72,-1,-1,74,76,78,80,82,84,-1,86,88,-1,-1,90,92,94,-1,-1,96,-1,98,100,-1,102,104,-1,-1,-1,106,108,-1,-1,110,-1,-1,112,114,116,-1,-1,-1,-1,118,120,122,124,-1,-1,-1,-1,126,-1,128,130,-1,132,-1,-1,134,136,-1,-1,-1,-1,138,-1,-1,-1,140,142,-1,-1,144,-1,146,-1,148,150,-1,-1,152,-1,-1,154,-1,156,158,160,162,164,-1,166,168,170,172,174,-1,-1,176,178,180,-1,-1,182,184,-1,186,-1,-1,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,7.785302E5,3.3817584E7,2.2E1,4.6134964E2,1.6026318E-2,2.36433E6,6.15E2,2.8266037E2,3.0812828E6,4.9321495E6,4.7263342E2,7.153514E1,7.785714E0,3.5E1,6.123E3,4.5E0,7.82261E6,3.9123364E2,1.4598765E0,1.4615384E0,3.592428E7,1.3E1,8.901939E2,-2.215249E-3,5.3E1,5.445545E-1,7.336111E4,1E0,2.023681E-3,6.5E1,1.08712E5,9.207547E0,6.5590655E6,1E0,1.1556856E-3,-1.5132353E-2,2.7664675E5,4.727453E7,9.348502E-3,3.716158E-3,3.0594551E1,-3.2728433E-3,1.16381E-3,8.680404E-3,1.7969016E-3,2.395631E7,-1.0776713E-3,-7.996828E-3,3.8575E4,5.178788E6,2.4595E5,8.754E3,1.7467743E-2,3.3382E4,1.7515651E-3,2.94E2,3E1,-2.1064032E-2,1.8901301E-3,3.88E2,3.1540035E9,7.9E2,-6.587486E-4,6.6759977E-3,1.65915E5,-1.5577735E-2,2.75E0,3.1530054E0,5.4327203E-3,2.0528813E8,1E0,3.952182E-3,4.524027E-3,-1.4337078E-3,1.6547934E2,2.241E2,1.1305392E-2,4.032403E-3,1.834412E6,4.3074563E-3,-6.4046737E-3,1.606722E8,6.999242E7,1.48E1,-6.113991E-3,-1.6560567E-3,-8.242859E-3,-9.99243E-4,1.7144266E4,1.3050649E4,1.857E3,3.9682608E2,-6.9456054E-3,-1.255789E-3,6.4543122E-3,-2.3885854E-3,1.835821E1,-6.1406204E-3,6.1657715E8,1.925673E1,-4.8326543E-3,5.7E3,3.1616518E-3,1.007249E-2,6.737099E7,3.1096E4,-2.8101031E-3,-1.0843636E-2,2.15063E-3,-1.1545226E-3,1.4915254E0,-7.3208436E-3,-5.3513714E-4,7.5063356E-3,5.0857143E0,4.5683857E-2,-4.1439617E-3,-1.4096334E-2,2E1,-1.1046136E-3,2.485E2,-8.9006545E-3,5.603014E6,4.6712E4,-2.748403E-3,1.7957533E-3,1E0,-1.9443603E-3,4.0046684E-3,3.1E2,5.9090937E-3,1.7291568E2,6.0921145E4,1.76421E6,7.9547346E-4,1.7161617E2,1.9286551E-3,9.75458E5,2.2190831E5,2.16577E5,4.2964826E0,1.3631483E8,-5.5495957E-3,-1.3466601E-3,6.4722223E0,1.1E1,1.3177083E1,4.60747E-3,1.9420171E-3,1.1262003E0,5.526E3,7.2105103E-3,2.427869E8,1.1102412E-3,1.7279412E-3,3.4915986E3,8.9306E-4,-3.5921354E-3,-7.922401E-3,-4.008574E-3,-1.8745323E-3,4.1880566E-3,-9.312093E-4,-3.3869282E-3,5.7790417E-4,-3.8722993E-3,-1.973576E-3,1.102998E-3,1.9556483E-4,4.2582313E-3,2.9506234E-3,-2.8114258E-3,-3.4112425E-3,-7.950707E-4,1.059894E-3,-2.6423233E-3,-2.0099138E-3,2.0885812E-3,6.782996E-4,4.7667264E-3,-3.8253152E-4,-3.7655951E-3,3.587164E-3,-2.3929263E-3,-1.5564652E-3,-6.389795E-3,-3.2880423E-3,1.4271927E-3],"split_indices":[2,43,7,3,4,0,43,2,4,66,43,71,71,67,0,43,73,43,4,69,68,60,3,67,0,44,68,60,29,0,44,7,73,43,29,0,0,48,7,0,0,73,0,0,0,0,62,0,0,9,12,9,1,53,9,0,44,3,0,0,10,46,2,0,0,1,0,68,68,0,7,29,0,0,0,4,4,0,0,5,0,0,12,7,73,0,0,0,0,48,43,44,4,0,0,0,0,71,0,7,73,0,10,0,0,7,9,0,0,0,0,68,0,0,0,69,53,0,0,6,0,67,0,9,1,0,0,23,0,0,8,0,73,60,46,53,67,0,66,62,43,68,7,0,0,73,3,71,0,0,68,2,0,7,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,9.12E2,1.35E2,5.23E2,3.89E2,6E0,1.29E2,4.46E2,7.7E1,2.3E2,1.59E2,7.6E1,5.3E1,1.94E2,2.52E2,2.5E1,5.2E1,2.2E2,1E1,1.47E2,1.2E1,6.5E1,1.1E1,4.9E1,4E0,1.7E1,1.77E2,3.6E1,2.16E2,6E0,1.9E1,1.5E1,3.7E1,2.04E2,1.6E1,6E0,4E0,1.3E1,1.34E2,5E0,7E0,5.5E1,1E1,4E0,7E0,7E0,4.2E1,4E0,1.3E1,8E0,1.69E2,1.1E1,2.5E1,4.9E1,1.67E2,4E0,1.5E1,8E0,7E0,5E0,3.2E1,1.96E2,8E0,6E0,1E1,9E0,4E0,1.5E1,1.19E2,1.4E1,4.1E1,3.5E1,7E0,4E0,4E0,1.61E2,8E0,7E0,4E0,1E1,1.5E1,1.8E1,3.1E1,1.1E1,1.56E2,8E0,7E0,4E0,4E0,1.2E1,2E1,9.2E1,1.04E2,4E0,4E0,5E0,4E0,1.1E1,4E0,1.02E2,1.7E1,4E0,3.7E1,4E0,3.1E1,1E2,6.1E1,4E0,4E0,5E0,5E0,2.5E1,6E0,4E0,7E0,5.5E1,1.01E2,7E0,5E0,1.3E1,7E0,8.8E1,4E0,8.6E1,1.8E1,7E0,4E0,9.5E1,7E0,5E0,1.2E1,4E0,3.3E1,7E1,3E1,1.1E1,5E1,8E0,1.7E1,3.5E1,2E1,1.8E1,8.3E1,9E0,4E0,3.9E1,4.9E1,7.4E1,1.2E1,4E0,1.4E1,8.5E1,1E1,8E0,4E0,2.5E1,8E0,1.8E1,5.2E1,1E1,2E1,5E0,6E0,2.6E1,2.4E1,7E0,1E1,1.8E1,1.7E1,5E0,1.5E1,1.3E1,5E0,3.7E1,4.6E1,1.2E1,2.7E1,1.2E1,3.7E1,6.6E1,8E0,6E0,8E0,8.1E1,4E0,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[3.7375215E-4,-2.2854786E-2,7.1063854E-2,-5.8360323E-2,1.0228556E-2,8.425252E-2,-3.8396563E-2,-4.3050762E-2,-1.4425461E-1,-3.8919474E-3,4.4126812E-2,2.1412544E-1,7.600787E-2,-8.458971E-3,1.855634E-2,-7.499242E-2,-9.817644E-3,-2.7764913E-1,-1.09219946E-1,1.0057027E-3,-1.2814783E-2,5.195271E-2,-6.6124233E-3,5.608803E-3,1.2762782E-2,4.6425205E-2,1.12194456E-1,-7.7419385E-2,1.3187157E-1,-7.061742E-2,-9.158675E-3,8.57492E-3,-1.88121E-2,-1.6626748E-3,-1.8969288E-2,-6.0778484E-2,-1.8828017E-1,-2.7380306E-2,2.4074294E-2,9.161428E-2,1.839152E-2,6.436518E-2,-5.232756E-2,1.2440169E-1,-9.069431E-4,-7.1217644E-4,-7.444602E-3,8.472259E-3,2.0179723E-3,-8.366807E-2,-3.0026326E-2,-2.7148295E-2,6.470269E-2,-8.010336E-2,2.396186E-3,-1.0680201E-1,-1.6745647E-2,-1.869584E-2,-7.0748855E-3,-5.630425E-2,3.18294E-2,5.9690524E-2,1.5382847E-1,4.316089E-2,-2.6562203E-2,1.940227E-2,1.1073856E-1,-8.700417E-3,-3.5067042E-3,8.8989146E-2,1.7802781E-1,-6.796249E-2,-1.1799029E-1,-3.81711E-3,-3.888446E-3,-1.6301822E-2,-7.552228E-2,-3.822054E-4,6.415212E-3,1.0924598E-3,-1.0593182E-1,-6.579887E-3,-9.84997E-4,2.7597444E-2,-3.753783E-2,-5.691611E-3,4.2805477E-4,-6.407973E-3,5.142304E-2,6.270709E-3,3.995773E-2,8.671582E-3,2.792569E-3,-4.433734E-3,6.541674E-2,-8.136432E-2,4.0618684E-2,-5.8471568E-2,3.6659908E-2,6.982494E-2,1.5573172E-1,1.5408626E-3,-2.328266E-3,1.165052E-1,3.0721609E-2,1.0859114E-2,1.315351E-1,-1.24975234E-1,-5.3757183E-2,-9.391959E-2,-8.30506E-3,4.103508E-3,-2.2957949E-2,-6.3306004E-2,-3.725974E-3,-4.7688512E-4,-1.0559738E-1,-7.248426E-3,-7.508661E-2,6.478108E-2,-4.0311858E-2,3.4639683E-2,-4.8735023E-2,1.5622701E-2,-5.602467E-3,6.097619E-2,4.8066482E-5,2.9032072E-3,5.1991437E-3,1.7157947E-3,-3.0516386E-3,-3.9130453E-5,4.044495E-3,-1.6181631E-3,-5.9868502E-3,-1.0374942E-3,3.6899229E-3,-6.05112E-3,5.4038176E-4,5.6653224E-2,-2.7900375E-3,4.9900594E-3,-9.71328E-3,2.879077E-3,1.7211762E-1,2.4420454E-3,1.4494091E-1,3.4271793E-3,-8.127193E-4,8.26875E-3,3.332406E-3,-1.8985095E-3,-7.909023E-3,4.1781426E-3,-8.0964744E-2,-1.172697E-1,5.1939E-4,2.1786975E-2,-2.8872078E-3,-1.2262E-2,-7.5702346E-3,-4.35586E-2,1.9688427E-2,-1.4707293E-1,-1.8735223E-3,-9.2844013E-4,-5.05943E-3,2.044092E-2,5.1416717E-3,-4.255829E-3,-3.2574005E-4,-7.035108E-4,3.3464157E-3,-1.23647325E-1,-3.5097986E-2,2.1621373E-3,5.303579E-3,4.0756516E-2,1.0062565E-1,2.6816274E-3,-1.142844E-3,4.214352E-3,2.893743E-2,-3.059964E-3,1.8020627E-3,9.515024E-3,4.612682E-3,7.3838206E-3,2.4686127E-3,-2.1000549E-3,1.2562141E-3,2.9393926E-3,-9.4918493E-4,-5.96852E-3,-2.1517708E-3,-7.7835103E-3,-3.8962588E-3,-8.35593E-4,2.9879336E-3,2.017854E-3,-2.5773207E-3,3.8914723E-4,-3.3399707E-3,-1.8202577E-3,1.6612908E-3,-3.5519698E-3,-9.326895E-3,-1.0680539E-3,2.821849E-3,-2.077275E-3,-9.6590705E-3,1.1493452E-3,-2.2182593E-3,6.698775E-4,-1.8111728E-3,8.8162813E-4,3.5410144E-3,1.6282144E-3,5.675965E-3,3.232429E-4,3.0703444E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,41,43,45,47,49,-1,-1,51,-1,-1,53,55,57,59,61,63,65,67,69,-1,-1,-1,-1,-1,71,73,75,77,79,-1,81,-1,83,-1,85,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,-1,113,115,-1,-1,-1,117,-1,-1,119,121,-1,-1,123,125,-1,127,-1,-1,129,131,133,135,137,139,141,143,-1,-1,145,147,-1,149,151,153,155,-1,-1,157,159,161,-1,163,-1,165,167,169,171,173,175,-1,177,-1,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,-1,-1,183,-1,185,-1,187,-1,189,-1,-1,-1,-1,191,193,195,-1,197,-1,199,-1,201,203,205,-1,-1,-1,207,-1,-1,-1,-1,-1,209,211,213,-1,215,217,-1,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6863446E0,9.08595E-1,3.6991894E-1,4.842862E-1,1.9211456E-1,2.2678661E-1,2.3415089E-1,3.368274E-1,2.3138726E-1,3.8068515E-1,1.8035841E-1,4.1351438E-2,2.2494793E-1,0E0,2.3859516E-1,7.335484E-2,2.7945817E-1,3.38638E-1,1.6472709E-1,1.8401024E-1,0E0,1.5005928E-1,0E0,0E0,0E0,2.1596271E-1,1.5692794E-1,5.98095E-2,3.7421122E-2,8.118659E-2,0E0,0E0,1.06882095E-1,0E0,0E0,6.946021E-2,2.0255941E-1,1.3474771E-1,9.8410495E-2,9.409565E-2,7.1295395E-2,2.105298E-1,1.2548997E-1,1.5127277E-1,0E0,0E0,0E0,0E0,0E0,5.6128144E-2,5.643835E-2,7.117568E-2,7.691259E-2,7.217732E-2,0E0,3.6549374E-2,0E0,1.04834594E-1,0E0,6.254794E-2,1.06717214E-1,4.859513E-2,4.2425722E-2,4.371965E-2,8.798298E-2,7.337125E-2,8.11677E-2,0E0,2.7605744E-2,8.460185E-2,5.9221506E-2,6.13119E-2,3.664863E-2,5.1688693E-2,0E0,6.715151E-2,4.7932282E-2,0E0,0E0,0E0,2.1230996E-2,0E0,0E0,9.0990454E-2,7.0383936E-2,0E0,0E0,1.2588629E-1,4.517995E-2,0E0,2.353527E-2,0E0,0E0,3.8317557E-2,4.1130565E-2,2.4511077E-2,2.902758E-2,5.240955E-2,8.879821E-2,8.609478E-2,2.4478793E-2,0E0,0E0,5.9301853E-2,2.6100218E-2,0E0,4.6277404E-2,5.499567E-2,1.0910152E-1,7.007365E-2,0E0,0E0,4.381951E-2,1.2117654E-1,8.572939E-2,0E0,3.9897308E-2,0E0,2.6196413E-2,4.5159906E-2,2.1751745E-2,2.3719955E-2,7.125728E-2,5.582016E-2,0E0,5.92331E-2,0E0,0E0,2.2002196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.280963E-2,0E0,0E0,3.0120505E-2,0E0,2.5905073E-2,0E0,2.5512457E-2,0E0,1.6238572E-2,0E0,0E0,0E0,0E0,3.511108E-2,7.192993E-2,2.4579614E-2,0E0,2.1691162E-2,0E0,4.382789E-2,0E0,5.1397614E-2,5.717989E-2,2.2950247E-2,0E0,0E0,0E0,2.4425022E-2,0E0,0E0,0E0,0E0,0E0,6.453876E-2,5.0777078E-2,1.9962048E-2,0E0,4.0541872E-2,3.3465147E-2,0E0,0E0,0E0,1.7646499E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,25,25,26,26,27,27,28,28,29,29,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,50,50,51,51,52,52,53,53,55,55,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,80,80,83,83,84,84,87,87,88,88,90,90,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,103,103,104,104,106,106,107,107,108,108,109,109,112,112,113,113,114,114,116,116,118,118,119,119,120,120,121,121,122,122,123,123,125,125,128,128,139,139,142,142,144,144,146,146,148,148,153,153,154,154,155,155,157,157,159,159,161,161,162,162,163,163,167,167,173,173,174,174,175,175,177,177,178,178,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,42,44,46,48,50,-1,-1,52,-1,-1,54,56,58,60,62,64,66,68,70,-1,-1,-1,-1,-1,72,74,76,78,80,-1,82,-1,84,-1,86,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,-1,114,116,-1,-1,-1,118,-1,-1,120,122,-1,-1,124,126,-1,128,-1,-1,130,132,134,136,138,140,142,144,-1,-1,146,148,-1,150,152,154,156,-1,-1,158,160,162,-1,164,-1,166,168,170,172,174,176,-1,178,-1,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,-1,-1,184,-1,186,-1,188,-1,190,-1,-1,-1,-1,192,194,196,-1,198,-1,200,-1,202,204,206,-1,-1,-1,208,-1,-1,-1,-1,-1,210,212,214,-1,216,218,-1,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.984E3,4.539777E5,1E0,2.2E1,4.577732E2,3.6E1,3.812E3,5.51E2,3.355864E0,2.511E3,1.1043767E3,1.0952997E3,2.839532E10,-8.458971E-3,2.2539758E5,5.0363636E1,2.04115E5,8E0,2.1772727E4,1.5859042E6,-1.2814783E-2,3.875E0,-6.6124233E-3,5.608803E-3,1.2762782E-2,4.88E2,3.7715748E7,4.9627705E0,3.08564E5,1.6840201E2,-9.158675E-3,8.57492E-3,8.13362E0,-1.6626748E-3,-1.8969288E-2,6.0078123E-6,3.7798166E0,1.70595E5,8.612209E-6,1.09652E5,9.822E3,1.1043767E3,5.11E2,1.6975454E3,-9.069431E-4,-7.1217644E-4,-7.444602E-3,8.472259E-3,2.0179723E-3,6.737099E7,2.732381E2,3.82E2,2.0480016E7,6.123E3,2.396186E-3,5.159652E-7,-1.6745647E-2,1.3902439E0,-7.0748855E-3,2.2715E4,1.185E3,9.46E2,3.43E2,4.4E0,3.4779343E1,3.64E2,1.4992306E8,-8.700417E-3,3.3266933E0,1.6673345E7,8.710612E1,7.785714E0,1.2380953E0,2.90099E0,-3.888446E-3,5.4E1,8.910034E0,-3.822054E-4,6.415212E-3,1.0924598E-3,7.336111E4,-6.579887E-3,-9.84997E-4,2.988E4,5.095006E5,-5.691611E-3,4.2805477E-4,1.2E1,3.7439062E2,6.270709E-3,1.0410659E5,8.671582E-3,2.792569E-3,1.01E2,4E0,2.2E1,3.53562E5,2.0939393E0,9.46898E2,2.059E3,8.822294E6,1.5408626E-3,-2.328266E-3,6.149315E6,1.752E3,1.0859114E-2,2.01E0,5.6E1,6.666982E4,4E0,-8.30506E-3,4.103508E-3,2.992648E-2,1.0439024E1,8.579633E-2,-4.7688512E-4,5.4051723E0,-7.248426E-3,9.99277E4,2.2084616E2,6.23E2,1.8736842E0,6.072304E5,7.6569915E-1,-5.602467E-3,2.8282208E0,4.8066482E-5,2.9032072E-3,3.588785E0,1.7157947E-3,-3.0516386E-3,-3.9130453E-5,4.044495E-3,-1.6181631E-3,-5.9868502E-3,-1.0374942E-3,3.6899229E-3,-6.05112E-3,5.4038176E-4,2.0588236E0,-2.7900375E-3,4.9900594E-3,1.3795834E3,2.879077E-3,1.853776E7,2.4420454E-3,1.4379113E9,3.4271793E-3,3.08E2,8.26875E-3,3.332406E-3,-1.8985095E-3,-7.909023E-3,3.5419354E0,3.774648E0,3.89E2,5.1939E-4,1.4392524E0,-2.8872078E-3,1.397998E6,-7.5702346E-3,5.3776E4,1.4929079E2,1.9067797E1,-1.8735223E-3,-9.2844013E-4,-5.05943E-3,3.0751158E4,5.1416717E-3,-4.255829E-3,-3.2574005E-4,-7.035108E-4,3.3464157E-3,3.2296E4,4E0,1E0,5.303579E-3,2.6138635E2,7.09E2,2.6816274E-3,-1.142844E-3,4.214352E-3,2.1111112E0,-3.059964E-3,1.8020627E-3,9.515024E-3,4.612682E-3,7.3838206E-3,2.4686127E-3,-2.1000549E-3,1.2562141E-3,2.9393926E-3,-9.4918493E-4,-5.96852E-3,-2.1517708E-3,-7.7835103E-3,-3.8962588E-3,-8.35593E-4,2.9879336E-3,2.017854E-3,-2.5773207E-3,3.8914723E-4,-3.3399707E-3,-1.8202577E-3,1.6612908E-3,-3.5519698E-3,-9.326895E-3,-1.0680539E-3,2.821849E-3,-2.077275E-3,-9.6590705E-3,1.1493452E-3,-2.2182593E-3,6.698775E-4,-1.8111728E-3,8.8162813E-4,3.5410144E-3,1.6282144E-3,5.675965E-3,3.232429E-4,3.0703444E-3],"split_indices":[2,43,6,3,4,10,2,2,73,2,67,48,46,0,48,61,5,0,48,43,0,69,0,0,0,0,62,61,1,4,0,0,69,0,0,52,68,1,53,1,44,67,0,67,0,0,0,0,0,7,4,0,5,43,0,52,0,68,0,44,44,44,0,69,71,10,5,0,68,47,73,67,71,69,0,0,71,0,0,0,60,0,0,1,43,0,0,8,4,0,48,0,0,0,8,71,1,68,67,44,62,0,0,60,0,0,68,44,60,8,0,0,53,73,53,0,69,0,43,67,2,68,43,57,0,69,0,0,71,0,0,0,0,0,0,0,0,0,0,69,0,0,67,0,62,0,12,0,8,0,0,0,0,69,69,1,0,68,0,60,0,1,67,73,0,0,0,48,0,0,0,0,0,12,73,30,0,4,2,0,0,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,7.72E2,2.53E2,3.72E2,4E2,2.26E2,2.7E1,3.17E2,5.5E1,2.83E2,1.17E2,1.2E1,2.14E2,7E0,2E1,1.61E2,1.56E2,1E1,4.5E1,2.79E2,4E0,1.13E2,4E0,6E0,6E0,1.19E2,9.5E1,1.1E1,9E0,1.57E2,4E0,6E0,1.5E2,4E0,6E0,2.9E1,1.6E1,1.25E2,1.54E2,5.1E1,6.2E1,1.01E2,1.8E1,8.7E1,8E0,7E0,4E0,5E0,4E0,1.18E2,3.9E1,1.37E2,1.3E1,2.5E1,4E0,1.2E1,4E0,1.18E2,7E0,1.3E1,1.41E2,3.5E1,1.6E1,4E1,2.2E1,5.2E1,4.9E1,4E0,1.4E1,5.4E1,3.3E1,8.3E1,3.5E1,2.7E1,1.2E1,1.13E2,2.4E1,7E0,6E0,5E0,2E1,8E0,4E0,3.4E1,8.4E1,6E0,7E0,4.8E1,9.3E1,6E0,2.9E1,1.1E1,5E0,1.3E1,2.7E1,1.2E1,1E1,9E0,4.3E1,2.7E1,2.2E1,8E0,6E0,3.6E1,1.8E1,1.3E1,2E1,1.5E1,6.8E1,2.7E1,8E0,4E0,2.3E1,2.3E1,9E1,8E0,1.6E1,6E0,1.4E1,2.2E1,1.2E1,1.1E1,7.3E1,4.1E1,7E0,7.8E1,1.5E1,1.7E1,1.2E1,8E0,5E0,7E0,2E1,7E0,5E0,4E0,6E0,4E0,5E0,3.6E1,7E0,1.8E1,9E0,4E0,1.8E1,1.2E1,2.4E1,7E0,1.1E1,1E1,1E1,6E0,9E0,2.2E1,4.6E1,2.2E1,5E0,1.1E1,1.2E1,1.6E1,7E0,3.3E1,5.7E1,9E0,7E0,6E0,8E0,1.2E1,1E1,4E0,8E0,5E0,6E0,1E1,6.3E1,3.7E1,4E0,5.3E1,2.5E1,4E0,8E0,1.5E1,2.1E1,4E0,5E0,1.1E1,7E0,2E1,4E0,4E0,7E0,6E0,1.6E1,1.8E1,2.8E1,7E0,1.5E1,6E0,5E0,7E0,9E0,1.2E1,2.1E1,1.2E1,4.5E1,5E0,4E0,6E0,6E0,6E0,4E0,1.1E1,5.2E1,2.9E1,8E0,3.4E1,1.9E1,7E0,1.8E1,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-4.676902E-3,-2.7015233E-2,6.269562E-2,-1.4433283E-1,-2.026201E-2,7.7048E-2,-5.1342774E-2,-9.494207E-2,-1.610266E-2,-6.3832276E-2,-1.9735934E-3,4.491648E-2,1.1286066E-1,-1.2745735E-1,-5.681625E-4,-1.1262371E-1,-2.938342E-2,-5.8491174E-2,-1.0141822E-2,8.9425035E-3,-5.341162E-3,1.5622101E-2,9.0033844E-2,1.543186E-1,4.7166493E-2,-1.5672048E-3,-8.751095E-3,4.3330923E-2,-3.0651113E-3,-1.3539714E-3,-5.7882154E-3,7.4794714E-4,-3.191422E-3,-9.907242E-2,-4.4012662E-2,-1.892396E-2,3.1855647E-2,1.0822931E-1,3.3164932E-3,1.4114897E-1,3.090772E-2,1.3716716E-1,1.3015919E-2,6.0195044E-2,-3.072016E-3,-5.882701E-4,3.93597E-3,-7.8674644E-2,-9.776861E-3,2.4785649E-2,-5.5658653E-2,-4.1122038E-2,1.3640688E-2,-2.4848082E-3,7.637008E-2,6.878172E-3,2.121992E-3,3.5483663E-3,-1.5115118E-2,9.114572E-3,1.0523726E-1,-2.7630284E-2,8.2565285E-2,1.4738992E-1,8.7286456E-4,1.8635593E-2,3.5730659E-3,-1.3134022E-1,-4.6591673E-2,3.951455E-3,-2.1455933E-4,-6.1301332E-2,2.0187856E-3,-1.1591104E-2,-3.529352E-2,2.9082855E-2,-5.7990752E-2,-4.078118E-2,2.527182E-2,8.7465055E-2,-3.0207979E-3,-4.6664895E-3,-6.565478E-3,5.4957136E-4,6.054689E-3,1.6593332E-3,-3.6105593E-3,1.4986608E-3,7.219949E-3,8.436715E-2,1.6638876E-1,-1.131794E-3,3.3744434E-3,-2.4045259E-3,-7.69905E-3,2.8641973E-2,-7.286297E-2,1.9043525E-3,-1.4012039E-3,-5.3395648E-2,-1.3524973E-1,-7.0584394E-2,-4.4139833E-3,3.4816034E-2,-5.546807E-3,-2.6899775E-2,-1.1126453E-1,-1.0510528E-2,-8.301122E-3,7.641999E-2,1.2568398E-3,8.1047835E-3,6.731646E-2,8.0370903E-4,-3.5109697E-3,2.2227303E-3,6.039655E-3,1.818477E-1,2.915891E-3,3.3756718E-3,-9.938544E-4,-5.5721467E-3,-5.0246067E-2,-5.949569E-2,5.75592E-4,-2.1886944E-3,-8.00468E-3,-1.6279727E-2,-5.578866E-2,4.6154104E-2,-2.8410831E-2,7.7605054E-2,2.4406698E-2,-2.8837048E-3,1.9822894E-3,-2.5136247E-3,-7.2642816E-3,2.9133026E-2,-6.688009E-2,6.576708E-4,4.515419E-3,2.1913093E-2,-5.6261286E-2,3.962466E-2,1.2559676E-1,-1.2319154E-2,4.6064172E-2,6.051044E-3,1.041517E-2,-1.1298243E-4,-3.4160744E-3,-1.8129412E-3,-3.7574493E-3,-2.947179E-3,5.0381785E-3,2.8153555E-3,-1.392568E-3,-2.459323E-3,2.901479E-3,4.643642E-3,-3.637462E-4,2.4517943E-4,2.3325812E-3,-6.185418E-4,2.5433982E-3,3.1915025E-3,-6.629384E-3,1.583631E-3,-1.5894301E-3,-5.4530525E-3,7.744604E-4,3.6098657E-3,-1.6096738E-4,7.241138E-4,7.5832903E-3,1.0984136E-4,-2.4812461E-3,4.3974826E-3,2.0313096E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,-1,67,-1,69,71,73,75,77,79,-1,-1,-1,81,-1,83,85,87,89,-1,91,-1,93,95,-1,97,99,-1,-1,101,103,105,107,109,111,-1,-1,113,-1,-1,-1,-1,-1,-1,115,117,-1,-1,-1,-1,119,121,-1,-1,123,125,127,129,131,-1,133,135,-1,137,139,141,-1,143,145,-1,-1,-1,147,-1,-1,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,161,-1,-1,-1,-1,163,165,-1,-1,167,169,171,173,175,177,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5894581E0,6.2534726E-1,4.3441367E-1,4.0062398E-1,5.9829426E-1,2.6419806E-1,1.15868226E-1,3.8421363E-2,0E0,1.7176777E-1,3.5003278E-1,1.6380414E-1,2.923113E-1,6.312376E-2,5.773878E-2,2.5873601E-2,1.7400166E-2,1.2255162E-1,0E0,0E0,2.6433644E-1,8.738933E-2,1.437271E-1,1.0777593E-1,6.915995E-2,0E0,0E0,2.9250408E-2,0E0,0E0,0E0,0E0,0E0,1.11916184E-1,1.303767E-1,2.7742112E-1,2.1420023E-1,1.5954398E-2,9.462356E-2,3.1656146E-2,7.485564E-2,7.096934E-2,0E0,2.7061254E-2,0E0,0E0,0E0,7.667437E-2,0E0,3.6465917E-2,8.0046386E-2,2.6991242E-1,1.7424512E-1,8.606886E-2,1.0165897E-1,0E0,0E0,0E0,4.0460773E-2,0E0,4.182124E-2,4.133342E-2,4.2965256E-2,5.3014398E-2,0E0,3.2296866E-2,0E0,4.1722536E-2,6.533447E-2,0E0,2.371108E-2,6.936258E-2,0E0,0E0,2.427063E-1,1.1448563E-1,4.338435E-2,2.04315E-1,5.76351E-2,8.921772E-2,0E0,0E0,2.6666917E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.8924244E-2,5.5575132E-2,0E0,0E0,0E0,0E0,2.1625318E-2,1.9858018E-2,0E0,0E0,4.8947155E-2,3.1878382E-2,4.139595E-1,1.4692824E-1,5.3781584E-2,0E0,4.9789E-2,1.7583951E-2,0E0,6.773403E-2,1.8493675E-2,4.0440455E-2,0E0,7.248983E-2,2.907692E-2,0E0,0E0,0E0,5.0305963E-2,0E0,0E0,0E0,0E0,2.0207822E-2,4.5457184E-2,0E0,0E0,0E0,0E0,1.3890344E-1,4.5832537E-2,1.8900472E-1,4.7788307E-2,5.0546292E-2,0E0,0E0,0E0,0E0,2.1678159E-2,1.3491501E-1,0E0,0E0,1.8588802E-2,4.2577066E-2,5.6720186E-2,6.098543E-2,2.3242194E-2,2.2095801E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,27,27,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,47,47,49,49,50,50,51,51,52,52,53,53,54,54,58,58,60,60,61,61,62,62,63,63,65,65,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,82,82,89,89,90,90,95,95,96,96,99,99,100,100,101,101,102,102,103,103,105,105,106,106,108,108,109,109,110,110,112,112,113,113,117,117,122,122,123,123,128,128,129,129,130,130,131,131,132,132,137,137,138,138,141,141,142,142,143,143,144,144,145,145,146,146],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,-1,68,-1,70,72,74,76,78,80,-1,-1,-1,82,-1,84,86,88,90,-1,92,-1,94,96,-1,98,100,-1,-1,102,104,106,108,110,112,-1,-1,114,-1,-1,-1,-1,-1,-1,116,118,-1,-1,-1,-1,120,122,-1,-1,124,126,128,130,132,-1,134,136,-1,138,140,142,-1,144,146,-1,-1,-1,148,-1,-1,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,162,-1,-1,-1,-1,164,166,-1,-1,168,170,172,174,176,178,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,3.964269E3,2.856934E7,4.6463413E0,5.02E2,7.098E3,1.3655363E2,5.7E1,-1.610266E-2,4.5E1,2.04115E5,5.9671835E6,8.710612E1,4.797448E8,2.342E4,1.2E1,1.86E2,3.89E2,-1.0141822E-2,8.9425035E-3,2.0766992E6,5.617715E6,1.1621469E1,8.604875E3,1.0534078E4,-1.5672048E-3,-8.751095E-3,3.592428E7,-3.0651113E-3,-1.3539714E-3,-5.7882154E-3,7.4794714E-4,-3.191422E-3,2.161111E1,8E0,2.026453E6,5.441268E7,7.6825094E0,3.9914E4,4.27E3,2.95E2,4.931361E-2,1.3015919E-2,1.6326721E0,-3.072016E-3,-5.882701E-4,3.93597E-3,1.2571428E1,-9.776861E-3,3.2226672E7,1.321E3,7E0,3.84E2,2.524828E2,1.8393531E0,6.878172E-3,2.121992E-3,3.5483663E-3,8.295515E7,9.114572E-3,3.0984934E8,1.0657745E7,1.4916515E3,6.7321223E-1,8.7286456E-4,5.063952E-2,3.5730659E-3,1.11E2,2.88E0,3.951455E-3,2.1277844E-4,2.7E1,2.0187856E-3,-1.1591104E-2,3.7439062E2,1E0,5.94E2,2.5247778E6,1.125584E-3,9.4658756E-1,-3.0207979E-3,-4.6664895E-3,1.7E1,5.4957136E-4,6.054689E-3,1.6593332E-3,-3.6105593E-3,1.4986608E-3,7.219949E-3,1.7379E4,4.262058E0,-1.131794E-3,3.3744434E-3,-2.4045259E-3,-7.69905E-3,1.7217412E4,3.887931E0,1.9043525E-3,-1.4012039E-3,1.605076E6,1.2E1,1.16408E5,1.798E3,6E0,-5.546807E-3,8.172E3,2.7012987E1,-1.0510528E-2,6.2521395E6,3.5070792E2,7.2744107E0,8.1047835E-3,6.1319604E2,9.643839E6,-3.5109697E-3,2.2227303E-3,6.039655E-3,1.562752E3,2.915891E-3,3.3756718E-3,-9.938544E-4,-5.5721467E-3,1.8E1,5.217E4,5.75592E-4,-2.1886944E-3,-8.00468E-3,-1.6279727E-2,2.4E1,3.807421E6,3.299862E1,1.6566709E6,1.5026077E7,-2.8837048E-3,1.9822894E-3,-2.5136247E-3,-7.2642816E-3,4.0705118E6,1.1626786E2,6.576708E-4,4.515419E-3,3.82E2,2.33E2,3.839424E2,9.68E2,4.5620965E6,4.6779457E1,6.051044E-3,1.041517E-2,-1.1298243E-4,-3.4160744E-3,-1.8129412E-3,-3.7574493E-3,-2.947179E-3,5.0381785E-3,2.8153555E-3,-1.392568E-3,-2.459323E-3,2.901479E-3,4.643642E-3,-3.637462E-4,2.4517943E-4,2.3325812E-3,-6.185418E-4,2.5433982E-3,3.1915025E-3,-6.629384E-3,1.583631E-3,-1.5894301E-3,-5.4530525E-3,7.744604E-4,3.6098657E-3,-1.6096738E-4,7.241138E-4,7.5832903E-3,1.0984136E-4,-2.4812461E-3,4.3974826E-3,2.0313096E-4],"split_indices":[2,43,60,68,2,2,71,6,0,6,5,60,73,7,9,3,6,1,0,0,43,12,71,4,4,0,0,60,0,0,0,0,0,67,10,9,7,71,9,2,0,72,0,53,0,0,0,70,0,7,10,3,0,4,53,0,0,0,7,0,7,60,4,53,0,72,0,1,69,0,53,8,0,0,4,79,0,43,53,68,0,0,3,0,0,0,0,0,0,2,53,0,0,0,0,43,69,0,0,12,3,7,44,3,0,44,73,0,43,48,71,0,67,62,0,0,0,67,0,0,0,0,70,12,0,0,0,0,3,62,71,43,5,0,0,0,0,43,4,0,0,4,0,4,10,62,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.054E3,7.92E2,2.62E2,4.2E1,7.5E2,2.33E2,2.9E1,3.5E1,7E0,2.21E2,5.29E2,1.24E2,1.09E2,1.1E1,1.8E1,2.7E1,8E0,2.15E2,6E0,8E0,5.21E2,7.6E1,4.8E1,6.6E1,4.3E1,5E0,6E0,1.1E1,7E0,4E0,2.3E1,4E0,4E0,5.5E1,1.6E2,3.82E2,1.39E2,8E0,6.8E1,2.5E1,2.3E1,6E1,6E0,3.9E1,4E0,5E0,6E0,4.8E1,7E0,2.3E1,1.37E2,2.27E2,1.55E2,7.9E1,6E1,4E0,4E0,1.3E1,5.5E1,8E0,1.7E1,1.1E1,1.2E1,5.5E1,5E0,1.2E1,2.7E1,1.7E1,3.1E1,6E0,1.7E1,1.3E2,7E0,5E0,2.22E2,1.28E2,2.7E1,3.3E1,4.6E1,5.6E1,4E0,4E0,5.1E1,4E0,1.3E1,5E0,6E0,8E0,4E0,1.4E1,4.1E1,7E0,5E0,6E0,1.1E1,8E0,2.3E1,7E0,1E1,1.19E2,1.1E1,1.03E2,1.19E2,1.24E2,4E0,1.8E1,9E0,4E0,2.9E1,1.4E1,3.2E1,9E0,4.7E1,4.7E1,4E0,9E0,5E0,3.5E1,6E0,4E0,4E0,6E0,1.7E1,1.09E2,1E1,4E0,7E0,4E0,9.9E1,3.8E1,8.1E1,2.3E1,1.01E2,1.2E1,6E0,5E0,4E0,1.8E1,1.1E1,4E0,1E1,2.4E1,8E0,3.3E1,1.4E1,3.7E1,1E1,1.8E1,1.7E1,6E0,1.1E1,5.8E1,5.1E1,9.5E1,4E0,3.2E1,6E0,6.4E1,1.7E1,1.8E1,5E0,5.9E1,4.2E1,7E0,1.1E1,4E0,7E0,2E1,4E0,4E0,4E0,1.7E1,1.6E1,4E0,1E1,2.8E1,9E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"179","size_leaf_vector":"1"}},{"base_weights":[1.8820602E-3,-2.3254154E-2,6.0664844E-2,-5.817044E-2,-4.1928925E-3,-9.252778E-2,6.8936184E-2,-4.7223926E-2,-2.159959E-1,-5.2362785E-2,8.787443E-3,-6.736836E-4,-5.850275E-3,4.4177786E-2,1.0824084E-1,-5.3769454E-2,4.6137717E-2,9.752715E-4,-3.6037058E-1,-3.6191095E-2,-1.0993559E-1,5.4022055E-3,6.5471455E-3,5.4822262E-2,-1.4119607E-1,1.394868E-1,-4.8247185E-3,7.382396E-2,-5.8908287E-2,3.6359744E-3,5.573894E-4,-2.163606E-2,-7.334285E-3,-6.334675E-2,-1.2614475E-2,-6.461285E-3,-2.5178462E-3,1.9779388E-2,-1.5173376E-2,6.0100798E-2,-3.5424551E-3,2.4684942E-3,-1.6008442E-2,1.2143163E-1,1.2305972E-2,3.0801974E-2,-7.091179E-3,-1.17766685E-4,6.256841E-3,-7.7221796E-2,-3.4447607E-2,-8.518524E-2,-2.7579462E-4,-6.2074084E-2,1.088716E-2,-6.835149E-4,3.4545116E-2,5.4946378E-2,-2.8622575E-2,6.372312E-2,-3.1338425E-3,1.4537615E-1,3.9426208E-2,-4.7660664E-2,8.2752876E-2,-6.494196E-2,-1.8335184E-1,-4.438151E-2,1.3905023E-3,-4.4974156E-2,-4.966688E-3,-4.166776E-3,3.0381273E-4,-7.781972E-3,3.3882523E-3,6.3535824E-2,-1.562934E-2,4.8185106E-2,-8.890355E-3,-1.5526025E-2,6.4513925E-3,-1.07028455E-1,-1.6015884E-2,9.540378E-2,4.4773288E-2,2.086556E-1,1.1762326E-1,-2.1872087E-3,7.379692E-2,7.913354E-4,-4.7547147E-3,5.4156943E-3,1.0624671E-3,-5.5279616E-2,-1.22451484E-1,-1.0283447E-2,-3.8761199E-3,-5.5431057E-2,1.4309211E-3,-3.989234E-3,-4.1560328E-4,9.28315E-4,-2.2553199E-3,1.5722326E-3,5.8921976E-3,-4.038363E-2,2.5596613E-2,3.7701253E-2,1.1580935E-1,-3.2595225E-2,1.9301457E-3,1.375967E-3,-3.0071088E-3,-2.1518843E-2,-1.8064602E-1,-5.7524736E-3,-4.8969863E-3,5.4504536E-2,1.3583387E-1,-3.12618E-3,5.37707E-2,4.7856667E-3,1.1263395E-2,9.045327E-3,1.049118E-1,6.038466E-3,1.2777855E-3,-1.8153297E-2,-7.628401E-2,-1.0370394E-2,-4.8994023E-2,-6.645577E-2,-1.1319116E-3,-4.728548E-3,-6.529911E-2,-1.1599367E-3,6.519381E-2,4.3444466E-2,-2.745755E-3,2.424189E-3,7.0940354E-3,-2.5433872E-3,9.325106E-4,1.3196211E-3,-3.1090914E-3,-4.4090957E-3,-1.0613119E-2,-3.6048897E-2,1.5763143E-2,-1.1187796E-3,6.706382E-2,7.248702E-3,1.537885E-4,1.026793E-1,3.6721017E-2,1.1499044E-1,1.8928207E-3,-5.172863E-5,-2.879527E-3,-2.4938239E-3,-5.6707766E-3,-4.5329537E-3,9.1259036E-4,-1.2672386E-3,-3.8949156E-3,-3.1517833E-3,1.6754561E-3,-1.8356799E-3,1.2716099E-3,4.4266082E-4,-4.026241E-3,4.5792023E-3,-1.3520306E-3,2.477007E-3,1.7522495E-4,-3.183424E-5,-3.7510216E-3,1.4656065E-3,-1.9369116E-3,7.3491996E-3,1.7981133E-3,2.4840212E-4,5.6303227E-3,4.980934E-4,3.1788892E-3,9.120172E-3,4.500747E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,37,-1,39,41,43,45,47,49,-1,-1,-1,-1,51,53,-1,-1,55,57,59,-1,-1,-1,61,-1,63,-1,-1,-1,65,67,69,-1,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,-1,99,-1,-1,-1,101,-1,103,105,107,109,111,-1,113,115,117,119,121,123,-1,125,-1,-1,-1,-1,127,129,-1,-1,131,-1,-1,-1,-1,-1,-1,-1,133,135,137,139,141,-1,-1,-1,143,145,147,-1,149,151,-1,153,-1,-1,-1,155,-1,-1,157,159,-1,161,163,165,167,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,175,177,-1,179,-1,-1,181,183,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4996736E0,4.7296107E-1,3.9157927E-1,4.2343718E-1,2.8870562E-1,4.0685713E-2,2.7645004E-1,1.467008E-1,5.553214E-1,8.726418E-2,1.6303253E-1,0E0,0E0,3.609448E-1,3.9281845E-1,1.4925438E-1,1.680354E-2,0E0,1.4330113E-1,4.913953E-2,2.7211338E-2,1.05611645E-1,0E0,1.2261045E-1,4.1811293E-1,1.6696012E-1,1.3714084E-1,4.2692162E-2,9.2755556E-2,0E0,0E0,0E0,0E0,4.460047E-2,5.065822E-2,0E0,0E0,6.3475035E-2,1.4020173E-1,8.032316E-2,0E0,0E0,0E0,1.4742768E-1,0E0,8.954436E-2,0E0,0E0,0E0,1.4578718E-1,6.096653E-2,1.8501371E-2,0E0,2.7617157E-2,3.5549108E-2,8.64162E-2,7.270843E-2,1.4541936E-1,1.20852254E-1,9.1783404E-2,0E0,8.082855E-2,6.0157947E-2,3.3983726E-2,2.408287E-2,5.4355353E-2,2.8152108E-2,6.916009E-2,0E0,1.5597679E-2,0E0,0E0,0E0,2.8434437E-2,0E0,2.7761877E-2,7.553158E-2,6.156738E-2,3.7395038E-2,3.3378836E-2,0E0,1.0312788E-1,9.913029E-2,9.0156436E-2,1.0682958E-1,3.7520826E-2,2.3178458E-2,0E0,3.314446E-2,0E0,0E0,0E0,0E0,7.347754E-2,1.0141474E-1,0E0,0E0,4.242766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0555954E-2,5.7853438E-2,4.8326865E-2,2.0629838E-2,2.6002686E-2,0E0,0E0,0E0,2.2556836E-2,1.5714139E-2,6.4449176E-2,0E0,3.2273136E-2,8.043331E-2,0E0,7.544774E-2,0E0,0E0,0E0,2.1066189E-2,0E0,0E0,2.7019009E-2,5.658865E-2,0E0,3.5709593E-2,3.879094E-2,3.5091456E-2,2.3674473E-2,4.5012444E-2,0E0,5.417341E-2,3.125538E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.539129E-2,5.47331E-2,0E0,6.372522E-2,0E0,0E0,4.4162273E-2,5.9186332E-2,3.000325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,37,37,38,38,39,39,43,43,45,45,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,73,73,75,75,76,76,77,77,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,88,88,93,93,94,94,97,97,105,105,106,106,107,107,108,108,109,109,113,113,114,114,115,115,117,117,118,118,120,120,124,124,127,127,128,128,130,130,131,131,132,132,133,133,134,134,136,136,137,137,147,147,148,148,150,150,153,153,154,154,155,155],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,38,-1,40,42,44,46,48,50,-1,-1,-1,-1,52,54,-1,-1,56,58,60,-1,-1,-1,62,-1,64,-1,-1,-1,66,68,70,-1,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,-1,100,-1,-1,-1,102,-1,104,106,108,110,112,-1,114,116,118,120,122,124,-1,126,-1,-1,-1,-1,128,130,-1,-1,132,-1,-1,-1,-1,-1,-1,-1,134,136,138,140,142,-1,-1,-1,144,146,148,-1,150,152,-1,154,-1,-1,-1,156,-1,-1,158,160,-1,162,164,166,168,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,176,178,-1,180,-1,-1,182,184,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1E0,1E0,9.369565E0,5.02E2,1.4E1,1.3627213E3,9.454259E10,1.654E3,1.706875E1,7.82261E6,-6.736836E-4,-5.850275E-3,3.3386944E5,1.8545505E2,7E0,8.238114E2,9.752715E-4,8.01E2,1.631108E6,2.1478824E2,9E0,6.5471455E-3,1.0428572E1,1.52179E5,6.5026445E3,8.127911E7,1.9E1,3.168978E-2,3.6359744E-3,5.573894E-4,-2.163606E-2,-7.334285E-3,1.6496063E2,2.6086957E0,-6.461285E-3,-2.5178462E-3,1.8E7,4.273336E-3,2E1,-3.5424551E-3,2.4684942E-3,-1.6008442E-2,1.2820834E7,1.2305972E-2,2.8181964E-1,-7.091179E-3,-1.17766685E-4,6.256841E-3,2.4E1,1.559733E6,1.5365228E5,-2.7579462E-4,4.0705118E6,6.763314E7,2.63E2,1E0,5.72202E2,4.727453E7,3.596E3,-3.1338425E-3,1.6666666E0,1.762852E7,1.8421053E0,7.184408E-1,7.24E2,3.8E1,1.3745962E0,1.3905023E-3,1.9666272E5,-4.966688E-3,-4.166776E-3,3.0381273E-4,2.3412812E0,3.3882523E-3,4.0351807E2,7.058982E-1,1.24147E5,1.1046409E7,2.732439E2,6.4513925E-3,3.14403E5,1.1251582E10,5.928425E2,1.7302156E5,1.7265024E3,1E0,-2.1872087E-3,3.3765998E0,7.913354E-4,-4.7547147E-3,5.4156943E-3,1.0624671E-3,2E0,1.5132743E0,-1.0283447E-2,-3.8761199E-3,1E2,1.4309211E-3,-3.989234E-3,-4.1560328E-4,9.28315E-4,-2.2553199E-3,1.5722326E-3,5.8921976E-3,3.3358974E0,8.15736E7,5.6818056E7,1.2E1,5.217E4,1.9301457E-3,1.375967E-3,-3.0071088E-3,1.16E2,2.75E0,6.0875E4,-4.8969863E-3,2.1924414E2,1.2979348E3,-3.12618E-3,1.552356E0,4.7856667E-3,1.1263395E-2,9.045327E-3,8.847283E8,6.038466E-3,1.2777855E-3,6.843911E6,2.0007E5,-1.0370394E-2,5E0,6.81531E-2,4.9390244E0,2.6043478E2,2.0162934E5,-1.1599367E-3,3.588785E0,1.14E3,-2.745755E-3,2.424189E-3,7.0940354E-3,-2.5433872E-3,9.325106E-4,1.3196211E-3,-3.1090914E-3,-4.4090957E-3,-1.0613119E-2,8E1,4.7220547E1,-1.1187796E-3,3.1157124E-1,7.248702E-3,1.537885E-4,2.5454E4,7.177158E6,1.3358E4,1.8928207E-3,-5.172863E-5,-2.879527E-3,-2.4938239E-3,-5.6707766E-3,-4.5329537E-3,9.1259036E-4,-1.2672386E-3,-3.8949156E-3,-3.1517833E-3,1.6754561E-3,-1.8356799E-3,1.2716099E-3,4.4266082E-4,-4.026241E-3,4.5792023E-3,-1.3520306E-3,2.477007E-3,1.7522495E-4,-3.183424E-5,-3.7510216E-3,1.4656065E-3,-1.9369116E-3,7.3491996E-3,1.7981133E-3,2.4840212E-4,5.6303227E-3,4.980934E-4,3.1788892E-3,9.120172E-3,4.500747E-3],"split_indices":[2,29,17,69,2,3,67,46,1,73,43,0,0,48,73,3,48,0,2,5,4,8,0,68,1,4,59,0,53,0,0,0,0,4,69,0,0,5,53,6,0,0,0,9,0,49,0,0,0,3,9,43,0,43,7,44,30,48,7,2,0,69,62,68,49,10,3,53,0,60,0,0,0,68,0,4,53,1,60,67,0,9,46,67,43,4,13,0,53,0,0,0,0,32,68,0,0,6,0,0,0,0,0,0,0,69,7,12,3,12,0,0,0,0,71,1,0,67,4,0,69,0,0,0,7,0,0,60,5,0,8,53,71,67,43,0,71,10,0,0,0,0,0,0,0,0,0,0,73,0,53,0,0,9,60,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.013E3,7.1E2,3.03E2,2.5E2,4.6E2,1.5E1,2.88E2,2.35E2,1.5E1,9.7E1,3.63E2,5E0,1E1,1.78E2,1.1E2,2.2E2,1.5E1,6E0,9E0,7.7E1,2E1,3.55E2,8E0,1.69E2,9E0,8.6E1,2.4E1,8E0,2.12E2,7E0,8E0,5E0,4E0,3.5E1,4.2E1,1.2E1,8E0,2.09E2,1.46E2,1.63E2,6E0,5E0,4E0,7.7E1,9E0,2E1,4E0,4E0,4E0,1.2E2,9.2E1,2.5E1,1E1,1.3E1,2.9E1,8.8E1,1.21E2,2.3E1,1.23E2,1.59E2,4E0,5.9E1,1.8E1,8E0,1.2E1,1.09E2,1.1E1,8E1,1.2E1,1E1,1.5E1,9E0,4E0,2.3E1,6E0,1.6E1,7.2E1,9.2E1,2.9E1,1.3E1,1E1,1.6E1,1.07E2,5.8E1,1.01E2,1.6E1,4.3E1,5E0,1.3E1,4E0,4E0,7E0,5E0,9.5E1,1.4E1,7E0,4E0,7E1,1E1,4E0,6E0,1.4E1,9E0,1.2E1,4E0,4.5E1,2.7E1,8.1E1,1.1E1,2E1,9E0,7E0,6E0,8E0,8E0,9.7E1,1E1,3E1,2.8E1,7E0,9.4E1,5E0,1.1E1,4E0,3.9E1,5E0,8E0,3.5E1,6E1,5E0,9E0,5.8E1,1.2E1,1.9E1,2.6E1,1.2E1,1.5E1,7.7E1,4E0,5E0,6E0,1.4E1,6E0,4E0,4E0,4E0,4E0,4E1,5.7E1,4E0,2.6E1,2.4E1,4E0,2.3E1,7.1E1,3.3E1,6E0,2.6E1,9E0,4.2E1,1.8E1,5E0,4E0,1.9E1,3.9E1,4E0,8E0,9E0,1E1,6E0,2E1,1.1E1,4E0,6.1E1,1.6E1,2.3E1,1.7E1,4.5E1,1.2E1,5E0,2.1E1,4E0,1.9E1,4E1,3.1E1,4E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[1.423794E-3,-5.0470393E-2,2.6536688E-2,-4.087611E-2,-1.5317702E-1,1.2591555E-2,1.1108404E-1,-8.011143E-2,-2.2470837E-2,-1.158281E-1,-1.3943634E-2,2.1700311E-2,-1.2029618E-1,1.2241098E-1,-5.627696E-3,-6.958135E-2,-1.2009375E-2,-1.1587812E-2,-5.7770766E-2,-7.2058383E-4,-1.4138368E-1,1.3850019E-2,8.25383E-2,-8.737344E-2,-1.478885E-2,9.8302804E-2,1.1615753E-2,-1.4650612E-1,-5.2909844E-2,2.0234497E-2,-2.6525741E-2,-2.550336E-2,-1.1192556E-1,-8.094936E-3,-3.8955284E-3,8.154739E-3,1.0790355E-2,1.0820098E-2,1.04724824E-1,-1.111358E-3,-4.6237097E-3,5.5580497E-2,1.286719E-1,-3.4692339E-3,-7.952699E-3,-6.1069567E-2,2.5494979E-3,1.702953E-3,6.182404E-2,-5.810492E-2,-8.361054E-3,-6.5425164E-3,6.7385836E-3,-6.0565653E-3,-1.656692E-3,-1.1868992E-2,2.1842804E-2,2.7411855E-3,-1.6625805E-2,7.854613E-2,7.5161792E-3,-1.8854549E-3,7.300622E-2,1.0140893E-2,9.734331E-2,-9.645782E-2,-2.5936764E-2,-2.9446915E-2,3.2763194E-2,4.7156163E-3,4.0735537E-4,5.032486E-4,-8.1467345E-2,3.6263962E-3,-1.4718059E-2,2.7741652E-2,-3.1366963E-3,-1.6080143E-2,4.536533E-3,1.0692011E-1,1.5913406E-2,-2.6327455E-3,1.2245688E-3,5.2073663E-3,4.0012702E-2,1.3121872E-1,4.1809496E-2,7.0301397E-3,7.624368E-2,1.1526459E-3,-1.2662338E-1,3.2327892E-3,-7.617263E-2,-4.1114476E-5,-4.2120614E-3,2.458413E-3,-9.431402E-4,-2.3647617E-3,-6.7651845E-3,-2.1264525E-2,3.266782E-3,2.0308178E-3,-1.5219009E-3,-6.8575954E-3,-8.124784E-2,1.862808E-5,1.319592E-1,6.750579E-2,7.871137E-3,8.38086E-4,4.308135E-3,3.1677843E-3,8.323019E-3,1.1964786E-2,3.3929597E-3,8.863961E-2,6.9045724E-5,-1.4481018E-1,-4.78835E-4,-2.9059453E-3,3.244578E-2,-5.766223E-3,-1.1129848E-3,-4.5741063E-2,9.578449E-4,-2.897477E-4,-5.906653E-3,-2.2751654E-2,-6.0806843E-3,7.307735E-3,1.7276135E-3,-1.7481922E-5,9.903697E-2,-6.351451E-2,1.8653069E-2,-1.6403464E-3,1.854359E-3,3.013373E-3,6.8379147E-3,-4.1314885E-3,-9.117624E-3,5.18E-3,7.723988E-5,-7.7845005E-4,-3.3960405E-3,-5.3273607E-4,2.3458803E-3,-2.8938593E-4,4.877667E-3,-2.8835372E-3,1.3580353E-3,5.1651024E-3,8.2982413E-4,-1.53484745E-2,-8.3737E-4,-7.1175564E-5,2.4650353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,31,-1,33,35,37,39,-1,41,-1,43,45,47,49,51,53,-1,-1,-1,55,57,59,-1,-1,61,63,-1,-1,65,-1,67,69,71,73,-1,75,-1,-1,77,79,-1,81,83,-1,-1,85,-1,87,89,91,93,95,-1,-1,-1,97,-1,99,101,-1,103,-1,105,107,-1,-1,-1,109,111,113,-1,115,-1,117,119,121,-1,-1,-1,-1,-1,-1,123,-1,-1,-1,125,127,-1,129,131,133,-1,-1,-1,-1,135,-1,137,-1,139,-1,-1,141,-1,-1,143,145,147,-1,149,-1,-1,-1,-1,151,153,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3267074E0,3.2089025E-1,8.074118E-1,2.1845156E-1,1.20483816E-1,7.167713E-1,2.708E-1,1.6560113E-1,7.962212E-2,5.9269696E-2,0E0,2.629674E-1,2.2201258E-1,2.6454866E-1,0E0,1.11516416E-1,0E0,7.706961E-2,8.2769796E-2,0E0,2.1909356E-2,2.4246335E-1,9.911105E-2,2.4952143E-2,0E0,9.664345E-2,0E0,1.6362786E-2,7.216802E-2,3.9972074E-2,6.261303E-2,1.2075242E-1,2.1548346E-2,0E0,0E0,0E0,1.2122238E-1,2.229071E-2,6.00605E-2,0E0,0E0,6.1769046E-2,1.09532654E-1,0E0,0E0,8.825824E-2,0E0,3.67631E-2,3.3234306E-2,6.536582E-2,4.0357567E-2,0E0,4.2639554E-2,0E0,0E0,7.527511E-2,1.6224712E-1,0E0,2.0660046E-2,4.157366E-2,0E0,0E0,4.899499E-2,0E0,3.0044079E-2,1.3500276E-1,5.592984E-2,3.3409826E-2,2.1748917E-2,0E0,0E0,0E0,5.1523507E-2,0E0,3.8973965E-2,2.2113571E-2,0E0,9.199871E-2,0E0,5.5955574E-2,1.260192E-1,0E0,0E0,0E0,1.926822E-2,1.8306315E-2,1.8946722E-2,0E0,2.581188E-2,0E0,5.9816033E-2,5.033967E-2,3.245856E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.4311194E-2,0E0,0E0,0E0,1.08038686E-1,5.2989602E-2,0E0,3.6236137E-2,8.779669E-2,2.0514865E-1,0E0,0E0,0E0,0E0,1.7474303E-2,0E0,2.3707837E-2,0E0,5.510235E-2,0E0,0E0,4.3948423E-2,0E0,0E0,2.2828713E-2,2.174061E-2,8.2883425E-2,0E0,2.2518221E-2,0E0,0E0,0E0,0E0,2.6967585E-2,4.2085797E-1,1.6196641E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,41,41,42,42,45,45,47,47,48,48,49,49,50,50,52,52,55,55,56,56,58,58,59,59,62,62,64,64,65,65,66,66,67,67,68,68,72,72,74,74,75,75,77,77,79,79,80,80,84,84,85,85,86,86,88,88,90,90,91,91,92,92,99,99,103,103,104,104,106,106,107,107,108,108,113,113,115,115,117,117,120,120,123,123,124,124,125,125,127,127,132,132,133,133,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,32,-1,34,36,38,40,-1,42,-1,44,46,48,50,52,54,-1,-1,-1,56,58,60,-1,-1,62,64,-1,-1,66,-1,68,70,72,74,-1,76,-1,-1,78,80,-1,82,84,-1,-1,86,-1,88,90,92,94,96,-1,-1,-1,98,-1,100,102,-1,104,-1,106,108,-1,-1,-1,110,112,114,-1,116,-1,118,120,122,-1,-1,-1,-1,-1,-1,124,-1,-1,-1,126,128,-1,130,132,134,-1,-1,-1,-1,136,-1,138,-1,140,-1,-1,142,-1,-1,144,146,148,-1,150,-1,-1,-1,-1,152,154,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.09E2,1.925673E1,1.5142261E3,8.147158E4,3.7377258E4,1E0,7.134432E7,1.2E1,2E0,1.375E0,-1.3943634E-2,4.5866325E6,3.7798166E0,6.5026445E3,-5.627696E-3,1.2571428E1,-1.2009375E-2,3.5364E4,1.89E2,-7.2058383E-4,2.5428572E0,2.04115E5,1.91E2,1.2E1,-1.478885E-2,1.5822886E0,1.1615753E-2,5.3E1,1.0428572E1,2.732381E2,1.5422421E2,1.43E2,1.162E3,-8.094936E-3,-3.8955284E-3,8.154739E-3,7.122041E7,2.0317461E0,3.201807E6,-1.111358E-3,-4.6237097E-3,1.1481482E0,2.5959E4,-3.4692339E-3,-7.952699E-3,7.7E1,2.5494979E-3,6.38E2,5.5177975E6,3.806E3,1.6379E2,-6.5425164E-3,3.4741783E0,-6.0565653E-3,-1.656692E-3,5E1,1.7039736E4,2.7411855E-3,3.6401E4,3.9914E4,7.5161792E-3,-1.8854549E-3,1.9514073E3,1.0140893E-2,1.6038339E0,1.2714286E1,2.5524476E0,4.878788E0,4.8064E4,4.7156163E-3,4.0735537E-4,5.032486E-4,1.703125E0,3.6263962E-3,3.7568388E0,2.1222334E2,-3.1366963E-3,9.6E1,4.536533E-3,1.2131076E2,3.3382E4,-2.6327455E-3,1.2245688E-3,5.2073663E-3,6.103178E8,1.599E3,1E0,7.0301397E-3,7.0993075E0,1.1526459E-3,2.1478824E2,5.4E1,9.876542E-2,-4.1114476E-5,-4.2120614E-3,2.458413E-3,-9.431402E-4,-2.3647617E-3,-6.7651845E-3,1.7594937E0,3.266782E-3,2.0308178E-3,-1.5219009E-3,2.511E3,1.2951E4,1.862808E-5,1.5E1,2.426015E-1,1E0,8.38086E-4,4.308135E-3,3.1677843E-3,8.323019E-3,7.3835544E5,3.3929597E-3,1.6327E4,6.9045724E-5,5.853E3,-4.78835E-4,-2.9059453E-3,9.2E1,-5.766223E-3,-1.1129848E-3,1.5357143E0,1E1,3.5E1,-5.906653E-3,1.2739792E1,-6.0806843E-3,7.307735E-3,1.7276135E-3,-1.7481922E-5,4.27E3,3.1157124E-1,1.1643481E6,-1.6403464E-3,1.854359E-3,3.013373E-3,6.8379147E-3,-4.1314885E-3,-9.117624E-3,5.18E-3,7.723988E-5,-7.7845005E-4,-3.3960405E-3,-5.3273607E-4,2.3458803E-3,-2.8938593E-4,4.877667E-3,-2.8835372E-3,1.3580353E-3,5.1651024E-3,8.2982413E-4,-1.53484745E-2,-8.3737E-4,-7.1175564E-5,2.4650353E-3],"split_indices":[2,73,67,43,48,83,59,8,32,68,0,43,68,4,0,70,0,9,10,0,68,5,0,3,0,53,0,44,68,4,67,2,10,0,0,0,7,69,9,0,0,68,9,0,0,44,0,44,60,1,67,0,68,0,0,3,62,0,9,9,0,0,70,0,57,67,68,73,44,0,0,0,68,0,68,4,0,8,0,48,9,0,0,0,7,10,28,0,50,0,4,2,53,0,0,0,0,0,0,71,0,0,0,2,44,0,3,53,8,0,0,0,0,43,0,2,0,9,0,0,44,0,0,68,8,3,0,71,0,0,0,0,2,53,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,3.31E2,6.85E2,3.04E2,2.7E1,5.89E2,9.6E1,9.6E1,2.08E2,2.3E1,4E0,5.52E2,3.7E1,9.2E1,4E0,9.2E1,4E0,1.6E2,4.8E1,5E0,1.8E1,4.9E2,6.2E1,3.3E1,4E0,7.9E1,1.3E1,1.5E1,7.7E1,5.1E1,1.09E2,3.1E1,1.7E1,1E1,8E0,8E0,4.82E2,1.5E1,4.7E1,6E0,2.7E1,3.4E1,4.5E1,5E0,1E1,7.2E1,5E0,3.6E1,1.5E1,3.9E1,7E1,6E0,2.5E1,1.3E1,4E0,1.58E2,3.24E2,5E0,1E1,3.4E1,1.3E1,5E0,2.9E1,1E1,3.5E1,3.5E1,3.7E1,1.8E1,1.8E1,8E0,7E0,1E1,2.9E1,4E0,6.6E1,2E1,5E0,1.53E2,5E0,2E1,3.04E2,5E0,5E0,1.7E1,1.7E1,9E0,2E1,8E0,2.7E1,7E0,2.8E1,2.4E1,1.3E1,1.3E1,5E0,1.3E1,5E0,2.1E1,8E0,6.2E1,4E0,1.6E1,4E0,1.35E2,1.8E1,4E0,1.6E1,4E1,2.64E2,1.3E1,4E0,5E0,4E0,1.1E1,9E0,2.3E1,4E0,2.4E1,4E0,7E0,1.7E1,6E0,7E0,2.9E1,3.3E1,1.29E2,6E0,9E0,9E0,1.2E1,4E0,1.3E1,2.7E1,3.4E1,2.3E2,4E0,7E0,1.8E1,5E0,1.3E1,1.1E1,4E0,1.3E1,1.5E1,1.4E1,2.7E1,6E0,1.23E2,6E0,5E0,4E0,2.3E1,4E0,4E0,3E1,1.46E2,8.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-2.3690518E-3,-4.6454236E-2,2.4270676E-2,-3.9487075E-2,-1.7790103E-1,1.2806815E-2,9.6692406E-2,-6.5399684E-2,-1.0367591E-2,-1.0324124E-2,-3.8157013E-1,2.2095663E-2,-4.8716985E-2,2.1900548E-1,7.713679E-2,-5.4802455E-2,-1.7812215E-1,-1.51273925E-2,8.280719E-2,-3.3877736E-3,2.086152E-3,-2.3489974E-2,-8.240744E-3,4.320459E-2,-9.208643E-3,-3.247114E-2,-1.4176326E-1,4.8005264E-3,1.323014E-2,9.144171E-2,-3.214184E-2,-1.17009215E-1,-4.7070682E-2,-1.5294679E-2,-6.460483E-2,-4.4045746E-3,-6.2572256E-2,-1.9115207E-5,6.90522E-3,1.4513594E-2,7.912101E-2,-3.743456E-2,5.928565E-3,-1.01075694E-1,-4.4577237E-4,-1.600472E-3,-9.148606E-3,1.02016166E-1,-3.1666807E-3,-4.510205E-3,2.4424011E-3,-1.4236003E-1,-5.767855E-4,-2.887059E-2,-7.6194726E-2,-3.0261488E-4,-4.4740615E-3,2.4722386E-2,-2.5758069E-2,-7.7478774E-2,6.9002045E-4,2.3605159E-2,-9.570162E-2,1.0650416E-1,3.300488E-2,-7.3764555E-2,-1.34646855E-2,1.05447076E-1,-4.812503E-3,-6.7789786E-2,-8.964123E-3,-3.090284E-2,8.842449E-2,1.1827256E-1,4.2806864E-2,-7.7682324E-3,-2.4023156E-3,-6.4046825E-3,-6.521758E-2,-1.3017711E-2,-5.7793178E-2,3.5074838E-2,-2.2289632E-3,2.9889024E-3,-3.2385856E-2,-1.9217745E-3,-5.2168015E-3,3.1509407E-2,-8.345172E-2,-1.2850322E-2,7.5271144E-4,7.9006106E-2,1.4786023E-1,-4.6273023E-2,6.134398E-2,-1.7573074E-3,-5.4621366E-3,-3.3920713E-2,3.7924357E-2,1.4922781E-1,2.5651226E-4,6.2303375E-3,-4.5400973E-2,-5.406925E-3,-3.0288946E-2,4.670073E-3,-4.595578E-3,7.395668E-4,6.3691614E-3,9.071165E-3,9.961424E-2,-4.0681256E-4,3.8774093E-3,-5.3135406E-2,1.0892414E-2,-7.025822E-3,-2.7026778E-2,-8.574363E-2,-2.4958434E-2,-1.1691692E-3,5.6429226E-2,-5.2172947E-3,-2.5322085E-2,4.2205244E-2,-3.798462E-2,-1.3420903E-3,-6.6029313E-3,3.759445E-2,1.22226916E-1,8.221263E-3,4.456291E-3,-5.069393E-3,6.9462217E-4,1.0558689E-1,1.6246017E-2,2.7053368E-3,-4.9223363E-2,-6.691448E-4,4.688343E-3,2.4992712E-3,9.910259E-3,-4.065815E-3,6.682679E-2,-4.6778736E-3,-6.907592E-4,9.347339E-4,-3.500155E-3,-1.4445928E-3,1.9964813E-3,5.2844435E-2,1.2996086E-1,-3.3647951E-3,-7.4026546E-5,5.5946827E-2,-6.9313142E-3,-4.1447632E-2,2.3322594E-3,1.15156574E-4,-1.0217599E-1,-3.503672E-2,1.4055921E-3,1.5535273E-3,-1.1627731E-3,5.110874E-3,4.086439E-2,-4.5741526E-3,-1.6006589E-2,4.6745043E-2,-3.5995934E-3,7.6965173E-3,-6.7384406E-3,6.0235422E-2,-1.2260028E-3,1.5047327E-1,-9.633956E-4,-3.4186582E-4,6.292773E-3,-1.760535E-3,5.5657398E-2,-8.316235E-2,9.3243754E-4,1.0042654E-2,-4.619761E-2,5.8167763E-3,3.131555E-4,1.087767E-4,7.070837E-2,8.7557705E-3,3.7370187E-3,6.297011E-4,4.4497224E-3,-2.2760548E-3,7.525329E-4,-3.0182349E-3,-2.3613703E-4,-6.422422E-3,-3.1489157E-3,-7.412458E-4,-4.9800836E-3,8.622268E-4,3.6478778E-3,-1.3873229E-3,3.78451E-4,3.99316E-3,1.4709192E-3,1.8237171E-3,-1.1570201E-3,4.405428E-3,9.0722676E-4,5.3884136E-3,9.9619245E-3,7.995641E-4,4.8918626E-3,-4.680241E-3,-1.2458394E-3,5.7625137E-5,4.3145483E-3,-3.5074928E-3,-2.730087E-4,4.252089E-3,6.424683E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,-1,-1,61,63,65,67,69,71,-1,-1,73,-1,-1,-1,75,-1,77,79,-1,-1,81,83,85,-1,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,-1,-1,113,115,-1,117,119,-1,-1,121,-1,-1,123,125,-1,-1,127,129,131,133,-1,-1,135,137,139,-1,141,143,-1,145,147,-1,-1,-1,-1,149,-1,-1,151,153,-1,155,157,159,161,163,-1,165,167,169,-1,-1,171,173,-1,-1,-1,-1,175,177,-1,179,-1,-1,-1,-1,181,183,-1,-1,-1,-1,-1,-1,185,187,-1,-1,189,191,193,-1,-1,195,197,-1,-1,-1,-1,199,-1,201,203,-1,205,-1,207,-1,209,-1,-1,-1,-1,211,213,-1,215,217,-1,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2565796E0,3.6204702E-1,5.5252683E-1,2.890737E-1,6.7866534E-1,3.3082342E-1,2.0077026E-1,2.3273116E-1,8.1968464E-2,4.5257922E-2,1.4134037E-1,3.321197E-1,1.10279575E-1,6.328726E-2,1.2799692E-1,8.443338E-2,2.7375138E-1,8.8245615E-2,4.9989052E-2,0E0,0E0,0E0,0E0,3.0799502E-1,8.695603E-2,1.4496261E-1,5.9435293E-2,0E0,0E0,1.2710792E-1,6.0827993E-2,5.121824E-2,8.6850375E-2,0E0,1.9953046E-2,8.957742E-2,3.8129225E-2,0E0,0E0,1.7059988E-1,1.6443783E-1,6.0835384E-2,1.4267583E-1,5.1823854E-2,1.2724204E-1,0E0,0E0,5.9426606E-2,0E0,0E0,0E0,2.8538644E-2,0E0,8.532194E-2,2.2582644E-1,0E0,0E0,4.777235E-2,5.185471E-2,2.9637948E-2,0E0,1.3463739E-1,2.7016523E-1,8.270675E-2,1.1741844E-1,3.9984316E-2,4.7539912E-2,5.602649E-2,5.451651E-2,2.784992E-2,0E0,8.58385E-2,4.0755257E-2,5.619371E-2,3.469716E-2,0E0,0E0,5.392696E-2,1.2857936E-1,0E0,5.2924722E-2,4.1860282E-2,0E0,0E0,4.1518524E-2,0E0,0E0,1.1012448E-1,3.1436995E-2,0E0,0E0,8.8476926E-2,3.2333255E-2,5.6115575E-2,7.41047E-2,0E0,0E0,4.8167996E-2,4.4771757E-2,4.4289827E-2,0E0,6.01671E-2,4.270429E-2,0E0,2.6804287E-2,3.459743E-2,0E0,0E0,0E0,0E0,5.558172E-2,0E0,0E0,1.8154286E-2,3.973663E-2,0E0,3.4507282E-2,4.6652466E-2,1.7252829E-2,1.8383233E-2,2.4161115E-2,0E0,4.852505E-2,7.289794E-2,9.9676974E-2,0E0,0E0,4.225469E-2,1.0417509E-1,0E0,0E0,0E0,0E0,6.674807E-2,4.4963993E-2,0E0,6.763481E-2,0E0,0E0,0E0,0E0,4.9800407E-2,4.8813827E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.697053E-2,6.0661912E-2,0E0,0E0,2.2603411E-2,3.61915E-2,2.1362014E-2,0E0,0E0,2.436024E-2,3.2816865E-2,0E0,0E0,0E0,0E0,2.2207748E-2,0E0,2.264145E-2,7.001647E-2,0E0,1.6595505E-2,0E0,2.8071135E-2,0E0,2.1151334E-2,0E0,0E0,0E0,0E0,2.1283206E-2,1.664035E-2,0E0,4.6066996E-2,2.4699166E-2,0E0,0E0,0E0,1.5682295E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,47,47,51,51,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,77,77,78,78,80,80,81,81,84,84,87,87,88,88,91,91,92,92,93,93,94,94,97,97,98,98,99,99,101,101,102,102,104,104,105,105,110,110,113,113,114,114,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,127,127,128,128,133,133,134,134,136,136,141,141,142,142,149,149,150,150,153,153,154,154,155,155,158,158,159,159,164,164,166,166,167,167,169,169,171,171,173,173,178,178,179,179,181,181,182,182,186,186],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,-1,-1,62,64,66,68,70,72,-1,-1,74,-1,-1,-1,76,-1,78,80,-1,-1,82,84,86,-1,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,-1,-1,114,116,-1,118,120,-1,-1,122,-1,-1,124,126,-1,-1,128,130,132,134,-1,-1,136,138,140,-1,142,144,-1,146,148,-1,-1,-1,-1,150,-1,-1,152,154,-1,156,158,160,162,164,-1,166,168,170,-1,-1,172,174,-1,-1,-1,-1,176,178,-1,180,-1,-1,-1,-1,182,184,-1,-1,-1,-1,-1,-1,186,188,-1,-1,190,192,194,-1,-1,196,198,-1,-1,-1,-1,200,-1,202,204,-1,206,-1,208,-1,210,-1,-1,-1,-1,212,214,-1,216,218,-1,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.01E2,9.639872E0,1.6466942E3,1.5422421E2,3.89E2,1E0,8.025477E0,6.9E2,1.5512196E1,1.317484E1,1.2916E4,8E0,2.614094E0,1.400229E6,3.206931E2,7.785714E0,1.5077367E3,8E0,6E0,-3.3877736E-3,2.086152E-3,-2.3489974E-2,-8.240744E-3,4.716184E2,6.953366E7,1.4598765E0,3.3512E4,4.8005264E-3,1.323014E-2,1.625E0,4.4615383E0,5.0363636E1,3.53969E5,-1.5294679E-2,3.8795E4,1.4392524E0,7.31E2,-1.9115207E-5,6.90522E-3,4.5620965E6,1.6485592E0,8.315271E0,3.168978E-2,2.060658E5,2.10377E5,-1.600472E-3,-9.148606E-3,1.3948464E7,-3.1666807E-3,-4.510205E-3,2.4424011E-3,1.4806053E4,-5.767855E-4,3.5318289E3,1.76421E6,-3.0261488E-4,-4.4740615E-3,3.0923106E-10,1.556431E6,5.524E3,6.9002045E-4,1.2300693E0,1.065548E0,1.0378262E10,1E0,6.185E3,1.935E3,5.1212654E0,5.1034E4,1.7851852E1,-8.964123E-3,8.625455E2,1.1772152E0,1.3208092E0,9.837502E1,-7.7682324E-3,-2.4023156E-3,1.0891155E4,1.2139681E3,-1.3017711E-2,2.240836E6,5.14E2,-2.2289632E-3,2.9889024E-3,1.952653E2,-1.9217745E-3,-5.2168015E-3,1E0,1.7544615E0,-1.2850322E-2,7.5271144E-4,6.4610266E2,1.6521566E5,6.9508715E0,4.7E3,-1.7573074E-3,-5.4621366E-3,1.06E2,6.0147805E0,2.6043478E2,2.5651226E-4,5.8165E4,1.9505818E8,-5.406925E-3,3.6451373E8,1.317E3,-4.595578E-3,7.395668E-4,6.3691614E-3,9.071165E-3,2.337765E3,-4.0681256E-4,3.8774093E-3,3.2E1,1.870878E6,-7.025822E-3,6E0,3.466E3,2.022131E6,8.6875E0,1.6877083E2,-5.2172947E-3,2.9726138E5,5.857143E0,3.3272727E0,-1.3420903E-3,-6.6029313E-3,2.297E3,1.4523518E8,8.221263E-3,4.456291E-3,-5.069393E-3,6.9462217E-4,4E0,9.812221E9,2.7053368E-3,2.558E3,-6.691448E-4,4.688343E-3,2.4992712E-3,9.910259E-3,6.627033E7,7.973882E7,-4.6778736E-3,-6.907592E-4,9.347339E-4,-3.500155E-3,-1.4445928E-3,1.9964813E-3,9.643839E6,1.4676277E7,-3.3647951E-3,-7.4026546E-5,5.4E1,1.43E2,5.6516E4,2.3322594E-3,1.15156574E-4,1.178E4,1.3935602E1,1.4055921E-3,1.5535273E-3,-1.1627731E-3,5.110874E-3,1.4E1,-4.5741526E-3,1.084437E6,3.0989232E-3,-3.5995934E-3,4.0903897E2,-6.7384406E-3,1.2526851E6,-1.2260028E-3,2.4545455E0,-9.633956E-4,-3.4186582E-4,6.292773E-3,-1.760535E-3,1.6438356E1,2.0130434E1,9.3243754E-4,6.63651E4,3.8104E4,5.8167763E-3,3.131555E-4,1.087767E-4,1.1863237E2,8.7557705E-3,3.7370187E-3,6.297011E-4,4.4497224E-3,-2.2760548E-3,7.525329E-4,-3.0182349E-3,-2.3613703E-4,-6.422422E-3,-3.1489157E-3,-7.412458E-4,-4.9800836E-3,8.622268E-4,3.6478778E-3,-1.3873229E-3,3.78451E-4,3.99316E-3,1.4709192E-3,1.8237171E-3,-1.1570201E-3,4.405428E-3,9.0722676E-4,5.3884136E-3,9.9619245E-3,7.995641E-4,4.8918626E-3,-4.680241E-3,-1.2458394E-3,5.7625137E-5,4.3145483E-3,-3.5074928E-3,-2.730087E-4,4.252089E-3,6.424683E-4],"split_indices":[2,69,67,67,2,6,71,2,71,69,1,32,68,1,73,67,48,32,8,0,0,0,0,70,7,69,1,0,0,61,69,62,9,0,1,68,2,0,0,62,53,71,53,48,1,0,0,9,0,0,0,43,0,62,46,0,0,52,5,44,0,53,53,46,27,44,2,68,44,73,0,67,68,68,73,0,0,43,48,0,9,2,0,0,70,0,0,19,68,0,0,70,48,71,2,0,0,0,69,67,0,10,5,0,7,2,0,0,0,0,67,0,0,4,46,0,8,1,12,73,67,0,43,68,69,0,0,2,12,0,0,0,0,3,46,0,10,0,0,0,0,12,47,0,0,0,0,0,0,62,62,0,0,2,2,9,0,0,1,73,0,0,0,0,3,0,9,53,0,67,0,43,0,68,0,0,0,0,71,71,0,48,10,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.068E3,4.02E2,6.66E2,3.83E2,1.9E1,5.76E2,9E1,2.02E2,1.81E2,1.1E1,8E0,5.01E2,7.5E1,1.1E1,7.9E1,1.86E2,1.6E1,1.73E2,8E0,5E0,6E0,4E0,4E0,2.99E2,2.02E2,6.5E1,1E1,5E0,6E0,7E1,9E0,1.9E1,1.67E2,6E0,1E1,1.42E2,3.1E1,4E0,4E0,1.67E2,1.32E2,7E1,1.32E2,2E1,4.5E1,4E0,6E0,6.6E1,4E0,5E0,4E0,1.5E1,4E0,1.04E2,6.3E1,4E0,6E0,6E1,8.2E1,2.6E1,5E0,1.55E2,1.2E1,8.2E1,5E1,2.7E1,4.3E1,1.2E1,1.2E2,1.6E1,4E0,3.4E1,1.1E1,5.1E1,1.5E1,1.1E1,4E0,6.5E1,3.9E1,4E0,5.9E1,5.3E1,7E0,5E0,7.7E1,1.4E1,1.2E1,1.45E2,1E1,4E0,8E0,5.1E1,3.1E1,1.3E1,3.7E1,1.6E1,1.1E1,3.1E1,1.2E1,8E0,4E0,9.5E1,2.5E1,6E0,1E1,2.3E1,1.1E1,5E0,6E0,8E0,4.3E1,7E0,8E0,1.7E1,4.8E1,1.1E1,2.8E1,3.1E1,2.8E1,2E1,3.3E1,5E0,7.2E1,1.26E2,1.9E1,6E0,4E0,2.7E1,2.4E1,1.8E1,1.3E1,6E0,7E0,1.8E1,1.9E1,4E0,2.7E1,7E0,5E0,4E0,4E0,8.2E1,1.3E1,8E0,1.7E1,5E0,5E0,1.2E1,1.1E1,1.8E1,2.5E1,1.2E1,5E0,1.3E1,3.5E1,2.4E1,4E0,5E0,2.6E1,2.4E1,4E0,8E0,1.2E1,6E0,2.7E1,7E0,6.5E1,1.22E2,4E0,1.4E1,5E0,2E1,7E0,2E1,4E0,4E0,1.4E1,8E0,1.1E1,1.8E1,9E0,6.2E1,2E1,6E0,7E0,5E0,1.3E1,1E1,1.5E1,7E0,6E0,1.2E1,2.3E1,1.4E1,1E1,1.1E1,1.5E1,2E1,4E0,1.8E1,9E0,4.1E1,2.4E1,3.2E1,9E1,7E0,7E0,1E1,1E1,1.5E1,5E0,7E0,4E0,1.3E1,5E0,5.7E1,5E0,1.1E1,9E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[3.1734907E-4,-2.5441412E-2,4.4444777E-2,-1.930243E-2,-1.1689011E-1,5.1820323E-2,-8.1651054E-2,-5.8866374E-2,-8.555098E-3,-2.7244818E-1,-2.264929E-2,3.028332E-2,8.010094E-2,-1.4328314E-1,-2.3486719E-2,-4.4790212E-2,-1.3263816E-1,-1.4765708E-2,3.70305E-2,6.172962E-4,-3.7759373E-1,-4.6482105E-2,1.6994231E-3,3.5040934E-2,-7.584718E-3,8.7630995E-2,-5.4392726E-3,-8.991346E-3,-2.4494752E-3,1.3180748E-3,-3.707882E-3,-6.523228E-3,-3.1407785E-2,-9.862959E-3,-9.145552E-2,-2.7298667E-2,1.1616464E-2,5.6367002E-2,-1.1782863E-2,-2.0760357E-2,-9.313337E-3,-4.9811127E-3,-1.7176581E-3,7.7325464E-3,6.706582E-2,1.6933034E-1,7.343165E-2,4.8997873E-3,-3.846071E-2,-2.1971301E-3,-5.725041E-3,-1.9333543E-2,-7.130554E-2,1.8790262E-2,-9.090453E-2,8.776974E-2,2.5256924E-2,1.4223348E-3,-5.5488846E-3,2.6358298E-3,-1.7664249E-3,2.3178311E-2,-1.1952087E-2,1.0394326E-2,5.3271286E-2,8.88285E-3,3.6925517E-3,6.51994E-2,1.0560282E-2,-4.358769E-2,1.4208389E-3,-2.5222566E-2,5.9746668E-2,-9.945379E-3,-5.1019914E-2,2.466904E-2,-3.960379E-2,-7.2662435E-3,-2.9322098E-4,5.6587663E-3,1.8851146E-3,2.463497E-3,-1.37553755E-2,7.4817287E-3,1.4445846E-2,4.443088E-3,7.0251875E-2,1.3766012E-1,5.296325E-2,-5.094637E-2,1.1885586E-2,-3.6514066E-3,-5.364996E-2,7.324804E-3,7.9231156E-4,1.415675E-3,-8.419614E-2,-1.2184613E-3,3.587004E-2,1.4462165E-3,-4.3761693E-3,-2.8119825E-3,2.1034083E-3,2.6040884E-2,-7.082199E-2,1.4593147E-3,-2.8696032E-3,7.726795E-2,-1.2283513E-3,4.527957E-3,9.850517E-3,-7.6740836E-3,6.893581E-2,-3.321744E-2,-8.0785654E-2,2.3800759E-3,-2.12408E-3,-3.5158604E-2,1.4836361E-2,-6.842919E-2,-5.845265E-3,-1.14167064E-1,-8.700106E-4,1.5398306E-2,-4.5961887E-2,6.464244E-2,1.8580774E-2,-4.6848338E-2,3.5051797E-2,-1.5942835E-3,-5.301727E-3,5.421723E-2,1.0862454E-1,5.122702E-3,-3.846949E-2,7.789783E-2,-2.7975119E-3,-4.4645816E-3,-1.0336055E-3,-4.408084E-3,-5.1937375E-4,-3.1197525E-3,-7.035668E-4,-9.4183546E-4,1.5935844E-3,-7.452188E-4,-3.982482E-3,-1.4736438E-3,1.5623561E-3,-1.8472258E-3,-6.0651773E-3,-5.055231E-4,2.2746518E-3,-4.2294003E-3,8.284892E-4,8.9585455E-4,3.8397154E-3,-6.036367E-4,1.4129496E-3,1.7207733E-4,-3.7506458E-3,2.0657287E-3,-1.007813E-3,1.0732767E-3,3.7291192E-3,6.259252E-3,1.1105543E-3,1.8018075E-3,-2.8517907E-3,4.9809343E-3,2.2626228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,47,-1,49,51,53,55,57,-1,-1,-1,59,61,63,65,67,-1,69,-1,-1,71,73,75,77,79,81,-1,-1,-1,-1,83,-1,-1,85,-1,-1,87,-1,89,-1,91,93,-1,95,97,99,-1,-1,-1,-1,-1,101,-1,103,105,107,109,111,113,115,117,119,-1,-1,-1,121,123,125,-1,-1,-1,-1,127,129,-1,-1,131,-1,-1,-1,133,135,137,139,-1,-1,141,143,145,147,149,-1,151,153,155,157,159,161,-1,-1,163,165,-1,167,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1503019E0,3.564796E-1,3.5113066E-1,2.546205E-1,5.8022606E-1,2.1257943E-1,7.191533E-2,1.2697732E-1,1.3445985E-1,4.5581985E-1,3.853836E-2,1.9263433E-1,2.3675478E-1,3.6021084E-2,3.7727233E-2,1.3619429E-1,4.706779E-2,1.3819677E-1,5.446531E-2,0E0,4.8367143E-2,5.1982917E-2,0E0,1.7264192E-1,0E0,1.5809727E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.812418E-2,0E0,1.548405E-2,9.781864E-2,1.00995876E-1,3.7837803E-2,8.209168E-2,0E0,0E0,0E0,2.7935635E-2,4.5045808E-1,1.8625724E-1,2.77071E-2,1.4730597E-1,0E0,3.4343094E-2,0E0,0E0,1.13741904E-1,1.145965E-1,4.4574272E-2,4.9515665E-2,2.7612E-2,2.482224E-2,0E0,0E0,0E0,0E0,1.22473866E-1,0E0,0E0,7.016404E-2,0E0,0E0,1.0087919E-1,0E0,3.611979E-2,0E0,1.3777676E-1,6.882788E-2,0E0,1.0824139E-1,3.367976E-2,4.98669E-2,0E0,0E0,0E0,0E0,0E0,3.0700976E-2,0E0,9.8735265E-2,4.3515056E-2,4.4859022E-2,3.003341E-2,1.0319358E-1,3.805095E-2,2.7593968E-2,7.566383E-2,6.81321E-2,0E0,0E0,0E0,5.270286E-2,2.7473414E-2,3.9092645E-2,0E0,0E0,0E0,0E0,5.8783587E-2,1.5185334E-2,0E0,0E0,3.6755413E-2,0E0,0E0,0E0,8.8239126E-2,1.02197796E-1,3.2226328E-2,2.741395E-2,0E0,0E0,2.9937278E-2,5.747947E-2,6.7394614E-2,2.587847E-2,1.8876106E-2,0E0,2.4603913E-2,3.0768512E-2,2.3462072E-2,2.015243E-2,1.7667213E-2,4.3831363E-2,0E0,0E0,2.7686536E-2,5.1270485E-2,0E0,3.664307E-2,6.307554E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,25,25,32,32,34,34,35,35,36,36,37,37,38,38,42,42,43,43,44,44,45,45,46,46,48,48,51,51,52,52,53,53,54,54,55,55,56,56,61,61,64,64,67,67,69,69,71,71,72,72,74,74,75,75,76,76,82,82,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,96,96,97,97,98,98,103,103,104,104,107,107,111,111,112,112,113,113,114,114,117,117,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,127,127,128,128,131,131,132,132,134,134,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,48,-1,50,52,54,56,58,-1,-1,-1,60,62,64,66,68,-1,70,-1,-1,72,74,76,78,80,82,-1,-1,-1,-1,84,-1,-1,86,-1,-1,88,-1,90,-1,92,94,-1,96,98,100,-1,-1,-1,-1,-1,102,-1,104,106,108,110,112,114,116,118,120,-1,-1,-1,122,124,126,-1,-1,-1,-1,128,130,-1,-1,132,-1,-1,-1,134,136,138,140,-1,-1,142,144,146,148,150,-1,152,154,156,158,160,162,-1,-1,164,166,-1,168,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.82E3,9.639872E0,4.217427E7,8.147158E4,2E0,1.2456025E3,1.2376862E10,5.17259E8,3.422592E6,8.317E3,1.6254545E1,1.5449402E7,4.1325716E7,4.5576923E1,2.5427E4,1.7857143E1,6.15E2,3.422351E6,1E0,6.172962E-4,5.75E2,6.76E2,1.6994231E-3,1.8062708E6,-7.584718E-3,9.318287E0,-5.4392726E-3,-8.991346E-3,-2.4494752E-3,1.3180748E-3,-3.707882E-3,-6.523228E-3,6E0,-9.862959E-3,1.5757076E9,2.87E2,1.6438356E1,2.1111E4,6.2521395E6,-2.0760357E-2,-9.313337E-3,-4.9811127E-3,3.37231E7,1.68991E6,1.1057693E0,5.1578946E0,8.363565E0,4.8997873E-3,1.3745962E0,-2.1971301E-3,-5.725041E-3,1.187E3,5.1942E4,5.4444447E0,1.23E2,1.67227E5,2.0487332E-3,1.4223348E-3,-5.5488846E-3,2.6358298E-3,-1.7664249E-3,4E0,-1.1952087E-2,1.0394326E-2,2.483E3,8.88285E-3,3.6925517E-3,1.524E3,1.0560282E-2,7.163082E7,1.4208389E-3,1E0,6.904904E3,-9.945379E-3,6E0,8.8790035E-1,5E0,-7.2662435E-3,-2.9322098E-4,5.6587663E-3,1.8851146E-3,2.463497E-3,1.3478261E1,7.4817287E-3,2.2E1,2.6844707E8,8.82E4,7.008608E7,1.0901037E10,2.04115E5,6.03E2,4.33E2,4E0,7.324804E-3,7.9231156E-4,1.415675E-3,1.034073E6,2.6950342E2,1E0,1.4462165E-3,-4.3761693E-3,-2.8119825E-3,2.1034083E-3,2.29E2,5.943433E4,1.4593147E-3,-2.8696032E-3,2.01E0,-1.2283513E-3,4.527957E-3,9.850517E-3,1.3358E4,1.514526E6,4.91271E5,3.257732E0,2.3800759E-3,-2.12408E-3,1.1430505E3,3.3832976E7,3.0052083E0,9E0,1.027972E0,-8.700106E-4,7.09E2,1.6153846E0,1.7733E4,2.9417648E2,2.6845297E5,2.831224E10,-1.5942835E-3,-5.301727E-3,6.9664386E2,5.8475E5,5.122702E-3,7.5456814E-4,9.4163445E1,-2.7975119E-3,-4.4645816E-3,-1.0336055E-3,-4.408084E-3,-5.1937375E-4,-3.1197525E-3,-7.035668E-4,-9.4183546E-4,1.5935844E-3,-7.452188E-4,-3.982482E-3,-1.4736438E-3,1.5623561E-3,-1.8472258E-3,-6.0651773E-3,-5.055231E-4,2.2746518E-3,-4.2294003E-3,8.284892E-4,8.9585455E-4,3.8397154E-3,-6.036367E-4,1.4129496E-3,1.7207733E-4,-3.7506458E-3,2.0657287E-3,-1.007813E-3,1.0732767E-3,3.7291192E-3,6.259252E-3,1.1105543E-3,1.8018075E-3,-2.8517907E-3,4.9809343E-3,2.2626228E-3],"split_indices":[2,69,60,43,32,70,46,46,43,9,69,62,66,73,9,4,2,9,30,0,2,2,0,43,0,71,0,0,0,0,0,0,3,0,46,0,71,9,43,0,0,0,5,43,69,69,50,0,53,0,0,10,1,69,0,1,53,0,0,0,0,73,0,0,2,0,0,44,0,7,0,26,48,0,8,71,8,0,0,0,0,0,73,0,3,7,10,5,46,5,10,2,6,0,0,0,9,67,26,0,0,0,0,10,48,0,0,68,0,0,0,9,10,46,68,0,0,48,7,69,8,68,0,2,68,1,4,43,46,0,0,67,1,0,53,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,6.38E2,3.72E2,5.99E2,3.9E1,3.52E2,2E1,1.27E2,4.72E2,1.4E1,2.5E1,2.01E2,1.51E2,9E0,1.1E1,1.08E2,1.9E1,4.16E2,5.6E1,4E0,1E1,1.8E1,7E0,1.97E2,4E0,1.46E2,5E0,5E0,4E0,6E0,5E0,1.2E1,9.6E1,5E0,1.4E1,2.82E2,1.34E2,4E1,1.6E1,6E0,4E0,7E0,1.1E1,1.07E2,9E1,2E1,1.26E2,4E0,9.2E1,7E0,7E0,2.4E2,4.2E1,1.26E2,8E0,1.9E1,2.1E1,1.2E1,4E0,4E0,7E0,1.02E2,5E0,6E0,8.4E1,1.5E1,5E0,1.21E2,5E0,8.6E1,6E0,2.24E2,1.6E1,4E0,3.8E1,1.15E2,1.1E1,4E0,4E0,1E1,9E0,1.2E1,9E0,5E0,9.7E1,2.2E1,6.2E1,1.6E1,1.05E2,7.6E1,1E1,1.28E2,9.6E1,4E0,1.2E1,1.1E1,2.7E1,3.5E1,8E1,5E0,6E0,5E0,4E0,8.6E1,1.1E1,1.6E1,6E0,5.8E1,4E0,1.2E1,4E0,2.2E1,8.3E1,4.9E1,2.7E1,6E0,4E0,4.7E1,8.1E1,7.3E1,2.3E1,1.8E1,9E0,2.6E1,9E0,2.9E1,5.1E1,9E0,7.7E1,7E0,4E0,3.5E1,2.3E1,4E0,1.8E1,7.8E1,5E0,6E0,4.3E1,2.2E1,5E0,1.7E1,3E1,2.9E1,5.2E1,1.9E1,5.4E1,1.4E1,9E0,4E0,1.4E1,1.5E1,1.1E1,5E0,4E0,9E0,2E1,1.4E1,3.7E1,4E0,5E0,6.6E1,1.1E1,1.7E1,1.8E1,1.7E1,6E0,4E0,1.4E1,3.7E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[2.8358696E-3,-2.1731937E-2,4.7262184E-2,-1.914325E-2,-1.0804873E-2,-1.07442856E-1,5.4537464E-2,-4.850239E-2,-4.782098E-3,-1.0055192E-2,-5.679092E-2,1.2139469E-2,4.8979595E-2,-4.3503724E-2,-1.5974875E-1,-3.4631126E-2,4.1432115E-3,2.755532E-4,-4.4701556E-3,3.7976187E-2,9.047904E-2,-3.45927E-2,-1.11122794E-1,-1.1393424E-2,-1.8910074E-3,-2.544626E-2,-8.264027E-3,1.8162321E-3,5.3703445E-3,6.246678E-2,1.574164E-2,1.1415972E-1,-6.775713E-2,-1.3162982E-1,-2.8366704E-2,-1.2663113E-2,-4.1569676E-2,2.0889657E-2,-4.246881E-2,-1.015258E-3,4.3923184E-3,1.04059055E-1,3.7159696E-2,-8.334721E-3,2.2537494E-2,1.2962312E-1,3.7638366E-2,-7.364217E-3,2.576431E-3,-8.075818E-3,-2.0724342E-3,6.308548E-2,-3.364144E-2,-8.0281794E-2,1.4198109E-3,-1.0233842E-2,1.9831716E-3,-6.898858E-2,-3.0415172E-3,-9.7341975E-3,3.1508435E-2,7.90356E-2,8.067667E-3,4.8366394E-2,-6.035828E-2,5.7871625E-2,-4.7866236E-3,9.6582316E-2,1.7765927E-1,-5.571634E-4,5.0647175E-3,7.6603927E-3,-1.5261787E-3,-5.3869475E-2,-1.464016E-2,-5.2134483E-3,-1.3355284E-3,-2.9182574E-3,1.6908598E-3,-4.7223542E-2,-1.0392852E-2,1.789234E-2,-4.146781E-3,-6.584291E-3,-5.6866505E-3,-2.8734442E-2,4.7082268E-2,9.949901E-2,1.976186E-2,1.5820231E-2,7.8003354E-2,-7.739879E-3,2.7205874E-3,1.496291E-2,1.1670819E-1,4.5652285E-2,-5.2709714E-2,4.4692498E-2,1.2049575E-1,9.118442E-3,3.6517426E-3,-4.1236207E-2,-1.503538E-1,8.923282E-2,-2.6142867E-2,-5.873108E-4,-2.8685401E-3,3.1307526E-3,-1.987494E-2,-5.7339385E-2,-1.0628119E-3,1.4283924E-3,-2.8482913E-3,3.0428344E-2,5.573799E-3,1.1780362E-1,1.2823052E-3,2.5670915E-3,-1.585374E-3,-1.5597975E-2,5.80315E-2,5.9674694E-3,4.745268E-2,-4.6728486E-3,4.6099126E-3,1.4337245E-1,1.7921563E-3,2.496411E-2,7.332925E-3,-3.266591E-2,-8.985747E-3,2.9854354E-4,3.7812009E-3,1.3690108E-1,1.7249534E-3,-1.3511529E-3,-4.6082414E-3,-9.808591E-3,-2.6945747E-3,6.3687614E-3,1.0516869E-3,-3.0566845E-3,-3.6080249E-4,1.376148E-3,-3.0977794E-3,-9.797035E-4,-9.130371E-3,-2.3079491E-3,2.611778E-4,-1.131352E-3,1.8780392E-3,1.5895964E-3,6.2102866E-3,-1.4444111E-3,2.0291891E-3,5.431144E-3,9.371942E-4,1.0213414E-3,6.905895E-3,-1.8031822E-3,1.8403545E-3,4.0617585E-3,9.6385805E-3,-8.982889E-4,2.262778E-3,1.116433E-3,-2.138018E-3,7.4168895E-3,2.904172E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,-1,39,-1,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,-1,65,67,69,-1,-1,-1,-1,71,73,75,-1,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,-1,-1,101,103,-1,-1,-1,-1,105,-1,107,-1,109,-1,111,113,115,117,119,121,-1,-1,123,125,127,129,131,133,-1,-1,135,137,139,141,-1,-1,-1,143,145,147,-1,-1,149,-1,151,-1,-1,-1,153,155,-1,157,159,-1,161,-1,163,-1,165,-1,-1,-1,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1372341E0,3.6402002E-1,4.2342842E-1,2.7973107E-1,0E0,8.25364E-2,3.979709E-1,1.1427647E-1,1.1922956E-1,0E0,3.3183564E-2,0E0,1.5501636E-1,1.22692555E-1,8.279735E-2,1.3191229E-1,8.978397E-2,0E0,0E0,1.4930898E-1,2.775293E-1,1.0924263E-1,2.6065782E-1,0E0,0E0,7.8185186E-2,0E0,8.9612134E-2,0E0,1.3402104E-1,1.9868553E-1,6.9667816E-2,1.22093916E-1,3.336823E-2,8.750458E-2,0E0,5.2848704E-2,1.8957775E-2,7.507357E-2,9.38855E-2,0E0,7.272667E-2,9.362622E-2,0E0,1.3737431E-1,6.537312E-2,4.4255488E-2,0E0,0E0,0E0,0E0,1.04219414E-1,6.37836E-2,1.6814038E-2,0E0,3.206031E-2,0E0,1.3534802E-1,5.6065947E-2,9.238241E-2,6.702777E-2,4.4125527E-2,0E0,7.1108654E-2,1.2488814E-1,1.5477902E-1,1.9819859E-1,3.556326E-2,2.3280442E-2,0E0,0E0,0E0,0E0,9.2531875E-2,1.0747701E-1,0E0,0E0,0E0,0E0,1.9221015E-2,0E0,4.8437238E-2,0E0,7.1401075E-2,0E0,3.136102E-2,6.493725E-2,3.350061E-2,2.3205133E-2,4.997466E-2,5.483286E-2,0E0,0E0,6.2090542E-2,4.845524E-2,8.960797E-2,1.13011725E-1,1.6515747E-2,2.4901718E-2,0E0,0E0,4.8295423E-2,3.893374E-2,2.5948353E-2,5.8179356E-2,0E0,0E0,0E0,3.7141807E-2,1.2253891E-1,7.613755E-2,0E0,0E0,2.7381256E-2,0E0,2.58694E-2,0E0,0E0,0E0,2.188846E-2,3.3565894E-2,0E0,6.3929886E-2,4.89602E-2,0E0,5.0415844E-2,0E0,3.827802E-2,0E0,3.0515894E-2,0E0,0E0,0E0,2.141428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,27,27,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,51,51,52,52,53,53,55,55,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,73,73,74,74,79,79,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,104,104,108,108,109,109,110,110,113,113,115,115,119,119,120,120,122,122,123,123,125,125,127,127,129,129,133,133],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,-1,40,-1,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,-1,66,68,70,-1,-1,-1,-1,72,74,76,-1,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,-1,-1,102,104,-1,-1,-1,-1,106,-1,108,-1,110,-1,112,114,116,118,120,122,-1,-1,124,126,128,130,132,134,-1,-1,136,138,140,142,-1,-1,-1,144,146,148,-1,-1,150,-1,152,-1,-1,-1,154,156,-1,158,160,-1,162,-1,164,-1,166,-1,-1,-1,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,5.8E1,1E0,1.972052E5,-1.0804873E-2,1.6427984E1,3.3817584E7,3.0977E4,1E0,-1.0055192E-2,1E1,1.2139469E-2,6.7947706E8,4.6463413E0,3.3064186E7,3.7568388E0,3.7E1,2.755532E-4,-4.4701556E-3,1.1736916E0,3.206931E2,6E0,2E0,-1.1393424E-2,-1.8910074E-3,1E0,-8.264027E-3,6.663214E6,5.3703445E-3,6E0,1.0280637E0,7.3274844E9,6.9879E4,1.4806053E4,6E0,-1.2663113E-2,1.0428572E1,1.1E1,5E0,9.751103E6,4.3923184E-3,7.8114265E-1,2.3308511E5,-8.334721E-3,2.67637E5,1.3E1,1.2567214E7,-7.364217E-3,2.576431E-3,-8.075818E-3,-2.0724342E-3,1.3050649E4,2.135961E4,8.579633E-2,1.4198109E-3,7.54E2,1.9831716E-3,1.339646E6,2E0,9.750042E6,8.0509944E5,2.936348E10,8.067667E-3,1.3026532E6,9.525663E2,1.3758875E3,4.519E3,1.4176E4,2.8389828E7,-5.571634E-4,5.0647175E-3,7.6603927E-3,-1.5261787E-3,1.965084E4,1.813506E-5,-5.2134483E-3,-1.3355284E-3,-2.9182574E-3,1.6908598E-3,1.4452E4,-1.0392852E-2,6.346204E-7,-4.146781E-3,1.5933333E2,-5.6866505E-3,1.6121496E1,1E0,6.441311E-1,1E0,5.691698E2,2.0778275E-1,-7.739879E-3,2.7205874E-3,3.13173E0,1.016E4,6.2868685E6,2.8146256E5,1.1223777E7,5.0554064E7,9.118442E-3,3.6517426E-3,3.2856784E7,2E0,6.0921145E4,1.9595902E3,-5.873108E-4,-2.8685401E-3,3.1307526E-3,6.76E2,1.5222145E2,3.202146E5,1.4283924E-3,-2.8482913E-3,5.55E2,5.573799E-3,3.7253174E2,1.2823052E-3,2.5670915E-3,-1.585374E-3,6.4356956E0,2.0421052E0,5.9674694E-3,6.2850784E1,4.1032645E6,4.6099126E-3,5E0,1.7921563E-3,7.0093E4,7.332925E-3,8E0,-8.985747E-3,2.9854354E-4,3.7812009E-3,1E0,1.7249534E-3,-1.3511529E-3,-4.6082414E-3,-9.808591E-3,-2.6945747E-3,6.3687614E-3,1.0516869E-3,-3.0566845E-3,-3.6080249E-4,1.376148E-3,-3.0977794E-3,-9.797035E-4,-9.130371E-3,-2.3079491E-3,2.611778E-4,-1.131352E-3,1.8780392E-3,1.5895964E-3,6.2102866E-3,-1.4444111E-3,2.0291891E-3,5.431144E-3,9.371942E-4,1.0213414E-3,6.905895E-3,-1.8031822E-3,1.8403545E-3,4.0617585E-3,9.6385805E-3,-8.982889E-4,2.262778E-3,1.116433E-3,-2.138018E-3,7.4168895E-3,2.904172E-3],"split_indices":[2,3,17,43,0,73,7,44,29,0,6,0,7,68,5,68,3,0,0,53,73,67,32,0,0,68,0,43,0,8,69,5,2,43,3,0,68,3,8,9,0,53,48,0,9,3,58,0,0,0,0,43,43,53,0,2,0,9,32,9,60,46,0,43,4,70,44,2,62,0,0,0,0,43,53,0,0,0,0,9,0,52,0,4,0,73,28,53,8,67,57,0,0,53,2,60,48,59,59,0,0,60,10,60,48,0,0,0,2,4,43,0,0,44,0,4,0,0,0,69,68,0,73,60,0,8,0,1,0,3,0,0,0,19,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,6.7E2,3.7E2,6.63E2,7E0,1.6E1,3.54E2,2.17E2,4.46E2,4E0,1.2E1,8E0,3.46E2,2.09E2,8E0,1.02E2,3.44E2,5E0,7E0,2.75E2,7.1E1,1.86E2,2.3E1,4E0,4E0,9.7E1,5E0,3.38E2,6E0,1.3E2,1.45E2,6.2E1,9E0,1E1,1.76E2,6E0,1.7E1,2.6E1,7.1E1,3.29E2,9E0,4.8E1,8.2E1,4E0,1.41E2,5.1E1,1.1E1,5E0,4E0,6E0,4E0,9E0,1.67E2,1.1E1,6E0,1.1E1,1.5E1,4.2E1,2.9E1,2.6E2,6.9E1,3.7E1,1.1E1,7.4E1,8E0,6.1E1,8E1,3.2E1,1.9E1,7E0,4E0,4E0,5E0,8E1,8.7E1,6E0,5E0,5E0,6E0,3.8E1,4E0,2.4E1,5E0,2.54E2,6E0,1.4E1,5.5E1,2.7E1,1E1,3.6E1,3.8E1,4E0,4E0,3.6E1,2.5E1,3.9E1,4.1E1,1.1E1,2.1E1,1.5E1,4E0,7.2E1,8E0,8E0,7.9E1,1.2E1,2.6E1,1E1,1.4E1,2.4E1,2.3E2,5E0,9E0,4.6E1,9E0,2.1E1,6E0,6E0,4E0,2.1E1,1.5E1,1.3E1,2.5E1,3E1,6E0,1.8E1,7E0,3.4E1,5E0,3.7E1,4E0,6E0,5E0,1.7E1,4E0,6.1E1,1.1E1,4E0,4E0,4E0,4E0,2.4E1,5.5E1,7E0,7E0,2E1,4E0,2.7E1,2.03E2,7E0,3.9E1,4E0,1.7E1,1.7E1,4E0,5E0,1E1,2.1E1,4E0,1.7E1,1.3E1,1.1E1,7E0,1.2E1,2.2E1,7E0,3E1,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[2.9454422E-3,-9.563474E-3,7.428527E-2,-2.9429536E-2,2.0322178E-2,4.1506413E-2,1.13783166E-1,-1.7095927E-2,-9.8724216E-2,2.3066541E-2,-6.2965574E-3,-4.1470453E-3,4.9289316E-2,1.3534188E-1,-4.0188786E-2,-5.0270803E-2,3.7640538E-3,-6.295235E-2,-1.7997329E-1,2.6324706E-2,-7.5294375E-2,6.173823E-2,-5.4395396E-2,1.6529757E-1,8.126788E-2,-6.065235E-3,2.7232047E-3,-4.616113E-2,-8.085477E-3,1.0416386E-1,-7.4531315E-3,-7.370393E-2,2.7397985E-3,-3.2464108E-1,-4.371797E-2,4.2655105E-3,4.3851E-2,-1.3487922E-2,4.167971E-3,-1.3443623E-3,7.2128266E-2,-1.4028888E-5,-4.509399E-3,1.3111286E-1,1.1876941E-2,9.881935E-2,-7.767354E-5,-1.0347914E-1,-3.748169E-2,1.3016593E-1,1.7415573E-3,-6.1022893E-2,-1.5484404E-3,1.5956135E-3,-8.622015E-2,-5.3663175E-3,-1.9735374E-2,-5.6289393E-3,9.3964546E-4,-8.858226E-3,5.2812744E-2,6.9785826E-3,3.736328E-2,8.584616E-2,5.409155E-4,1.5262698E-1,-1.2382794E-4,1.960861E-3,6.2812814E-3,-1.2023802E-3,-5.5518528E-3,-1.0085321E-2,-6.550774E-2,8.8766284E-2,9.474076E-3,-5.375932E-3,4.015815E-3,2.9827037E-3,-5.020529E-3,-1.0165612E-1,-8.88543E-6,-8.319991E-3,-1.0652757E-3,1.7654244E-3,5.5940794E-3,-1.7961536E-2,4.559733E-2,1.0166748E-1,-1.0409621E-3,9.179849E-3,4.7230236E-3,-1.6026724E-2,3.4700744E-3,2.086195E-4,-7.323711E-2,1.1003499E-3,5.1911105E-3,-2.7842843E-3,3.6193852E-3,1.4941422E-2,-1.7769432E-2,-1.1799691E-1,-2.7672232E-3,1.0791896E-2,-4.690106E-2,4.2978145E-2,-1.0148094E-1,5.5032045E-2,-1.847993E-2,9.680315E-3,8.564236E-2,3.5728882E-3,-3.6594528E-3,1.4066939E-2,-2.9211087E-2,-5.983574E-2,-6.85991E-3,5.559923E-3,4.667273E-3,1.2511676E-2,-3.4337644E-2,-3.7651986E-3,-6.74811E-3,3.1818952E-3,5.3375266E-3,-4.6710996E-3,-1.1250079E-2,-4.1055446E-4,4.440304E-3,-4.0270337E-5,-7.7803126E-3,3.386677E-2,8.028684E-2,-5.821384E-3,1.627714E-2,3.8258277E-2,4.8316563E-3,-8.3033217E-4,3.3440406E-3,-5.3379415E-3,-6.6886307E-4,-3.6172136E-3,-7.629177E-4,1.7429817E-3,-9.52301E-4,-1.1119111E-3,1.9207463E-3,-2.42766E-3,-2.3811366E-4,-1.1721759E-3,1.2622371E-3,7.354085E-4,-2.525038E-3,1.9855238E-3,-2.1034812E-3,1.6549384E-3,4.8858863E-3,-1.2219751E-3,3.4645093E-3,2.8443632E-3,-5.243642E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,55,57,59,61,-1,-1,-1,63,-1,-1,65,-1,67,-1,69,71,73,-1,75,77,-1,79,-1,-1,-1,-1,81,83,-1,85,87,-1,89,-1,-1,-1,-1,-1,91,93,95,-1,-1,97,-1,99,101,-1,-1,103,-1,-1,105,107,109,111,-1,-1,113,-1,-1,115,-1,-1,-1,-1,117,119,121,-1,123,125,127,129,131,133,-1,135,-1,-1,137,139,141,-1,143,-1,145,147,-1,-1,149,-1,-1,151,-1,-1,-1,-1,153,155,-1,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9130956E-1,5.0525177E-1,1.8818694E-1,4.3504015E-1,1.4927793E-1,8.922307E-2,2.2784162E-1,3.012211E-1,2.124654E-1,1.0934158E-1,0E0,0E0,1.05328545E-1,8.2136154E-2,8.9014895E-2,7.804695E-2,3.025988E-1,7.6665744E-2,4.3917787E-1,1.2546313E-1,4.2079395E-1,6.910908E-2,2.100739E-2,8.238542E-2,3.3651367E-2,0E0,0E0,7.7709496E-2,0E0,4.175678E-2,7.6453224E-2,7.207152E-2,0E0,1.8553889E-1,6.753443E-2,9.285038E-2,1.2024295E-1,0E0,0E0,0E0,5.204025E-2,0E0,0E0,8.388391E-2,0E0,3.401336E-2,0E0,2.4024278E-2,1.0990426E-1,4.071629E-2,0E0,8.685171E-2,5.059369E-2,0E0,6.106487E-2,0E0,0E0,0E0,0E0,1.5326431E-1,2.6272096E-2,0E0,7.917313E-2,7.1588576E-2,0E0,3.7735105E-2,0E0,0E0,0E0,0E0,0E0,3.8661696E-2,3.8758546E-2,1.8780746E-2,0E0,0E0,6.2057383E-2,0E0,5.3454533E-2,1.9544423E-2,0E0,0E0,6.0832735E-2,0E0,0E0,1.2142332E-1,9.16996E-2,5.811286E-2,6.126675E-2,0E0,0E0,2.818452E-2,0E0,0E0,5.565524E-2,0E0,0E0,0E0,0E0,6.566116E-2,6.4647704E-2,1.5755832E-2,0E0,7.057266E-2,4.331084E-2,3.943531E-2,6.718852E-2,6.730369E-2,7.679725E-2,0E0,3.1899065E-2,0E0,0E0,4.240682E-2,6.065211E-2,4.349293E-2,0E0,6.392194E-2,0E0,4.990501E-2,4.3838553E-2,0E0,0E0,5.8593817E-2,0E0,0E0,1.8558875E-2,0E0,0E0,0E0,0E0,5.4763347E-2,6.3031524E-2,0E0,4.2122483E-2,1.573117E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,40,40,43,43,45,45,47,47,48,48,49,49,51,51,52,52,54,54,59,59,60,60,62,62,63,63,65,65,71,71,72,72,73,73,76,76,78,78,79,79,82,82,85,85,86,86,87,87,88,88,91,91,94,94,99,99,100,100,101,101,103,103,104,104,105,105,106,106,107,107,108,108,110,110,113,113,114,114,115,115,117,117,119,119,120,120,123,123,126,126,131,131,132,132,134,134,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,56,58,60,62,-1,-1,-1,64,-1,-1,66,-1,68,-1,70,72,74,-1,76,78,-1,80,-1,-1,-1,-1,82,84,-1,86,88,-1,90,-1,-1,-1,-1,-1,92,94,96,-1,-1,98,-1,100,102,-1,-1,104,-1,-1,106,108,110,112,-1,-1,114,-1,-1,116,-1,-1,-1,-1,118,120,122,-1,124,126,128,130,132,134,-1,136,-1,-1,138,140,142,-1,144,-1,146,148,-1,-1,150,-1,-1,152,-1,-1,-1,-1,154,156,-1,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.467E3,8.995735E5,1.767635E3,2.2E1,4.9E1,1.1840488E8,2.5130852E2,5.51E2,6.0692043E0,1.9269184E7,-6.2965574E-3,-4.1470453E-3,6.364486E0,1.159499E6,4.591837E0,6.651E4,2.6E1,6.2233735E-6,1.746E3,1.158E3,2.395631E7,1.3935602E1,1.172766E3,3.5675005E3,1.7655972E0,-6.065235E-3,2.7232047E-3,3.4E1,-8.085477E-3,3.66025E5,1.84E2,1E0,2.7397985E-3,1.04109E5,2.6530768E2,4.5866325E6,6.923077E-1,-1.3487922E-2,4.167971E-3,-1.3443623E-3,1.4379113E9,-1.4028888E-5,-4.509399E-3,1.8776652E7,1.1876941E-2,1.2816234E7,-7.767354E-5,1.3888889E1,3.9961785E-2,6.743651E7,1.7415573E-3,1.1151079E0,5E0,1.5956135E-3,5.095006E5,-5.3663175E-3,-1.9735374E-2,-5.6289393E-3,9.3964546E-4,1.4110284E2,2.712766E0,6.9785826E-3,2.4294034E2,3.0561172E5,5.409155E-4,1.3947369E0,-1.2382794E-4,1.960861E-3,6.2812814E-3,-1.2023802E-3,-5.5518528E-3,7.2132964E0,1.2368386E0,1.7467743E-2,9.474076E-3,-5.375932E-3,6.819212E6,2.9827037E-3,1.4195632E6,2.8998098E1,-8.88543E-6,-8.319991E-3,9.03E2,1.7654244E-3,5.5940794E-3,1.0439024E1,2E0,1.43E2,5.3700186E8,9.179849E-3,4.7230236E-3,2.00033E5,3.4700744E-3,2.086195E-4,7.547528E6,1.1003499E-3,5.1911105E-3,-2.7842843E-3,3.6193852E-3,4.586207E0,3.14403E5,9.874E3,-2.7672232E-3,2.0097298E1,9.99E2,5.80418E7,1.1E1,6.5718125E6,1.4598765E0,9.680315E-3,1.606722E8,3.5728882E-3,-3.6594528E-3,4.9E2,4.9069305E1,1.3600995E2,-6.85991E-3,2.3412812E0,4.667273E-3,5.860227E5,1.3631483E8,-3.7651986E-3,-6.74811E-3,2.485E2,5.3375266E-3,-4.6710996E-3,3.0416667E0,-4.1055446E-4,4.440304E-3,-4.0270337E-5,-7.7803126E-3,2.23099E7,6.763312E7,-5.821384E-3,8.134772E2,2.4180895E6,4.8316563E-3,-8.3033217E-4,3.3440406E-3,-5.3379415E-3,-6.6886307E-4,-3.6172136E-3,-7.629177E-4,1.7429817E-3,-9.52301E-4,-1.1119111E-3,1.9207463E-3,-2.42766E-3,-2.3811366E-4,-1.1721759E-3,1.2622371E-3,7.354085E-4,-2.525038E-3,1.9855238E-3,-2.1034812E-3,1.6549384E-3,4.8858863E-3,-1.2219751E-3,3.4645093E-3,2.8443632E-3,-5.243642E-4],"split_indices":[2,43,67,3,3,7,73,2,68,62,0,0,69,9,69,1,0,52,44,2,62,73,4,67,49,0,0,2,0,9,44,20,0,5,67,43,68,0,0,0,12,0,0,60,0,9,0,4,53,7,0,61,3,0,43,0,0,0,0,4,68,0,67,48,0,68,0,0,0,0,0,69,68,53,0,0,12,0,60,73,0,0,2,0,0,73,6,0,7,0,0,5,0,0,5,0,0,0,0,68,9,1,0,73,2,7,8,60,69,0,12,0,0,1,62,67,0,68,0,62,7,0,0,67,0,0,69,0,0,0,0,9,7,0,70,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,8.49E2,1.48E2,5.1E2,3.39E2,8.2E1,6.6E1,4.34E2,7.6E1,3.34E2,5E0,4E0,7.8E1,5.8E1,8E0,1.67E2,2.67E2,5.4E1,2.2E1,3.24E2,1E1,7E1,8E0,3.6E1,2.2E1,4E0,4E0,1.63E2,4E0,2.6E1,2.41E2,5E1,4E0,1E1,1.2E1,1.44E2,1.8E2,4E0,6E0,7E0,6.3E1,4E0,4E0,2.8E1,8E0,1.8E1,4E0,2E1,1.43E2,1.8E1,8E0,2.3E1,2.18E2,5E0,4.5E1,4E0,6E0,5E0,7E0,1.14E2,3E1,9E0,1.71E2,5.1E1,1.2E1,2.4E1,4E0,8E0,1E1,4E0,1.6E1,7.3E1,7E1,1.3E1,5E0,1.2E1,1.1E1,1E1,2.08E2,3.8E1,7E0,4E0,1.1E2,2.6E1,4E0,2.2E1,1.49E2,4.3E1,8E0,1.1E1,1.3E1,6.9E1,4E0,7E0,6.3E1,4E0,9E0,6E0,5E0,8.1E1,1.27E2,2.6E1,1.2E1,8.8E1,2.2E1,1.3E1,9E0,1.3E2,1.9E1,4E0,3.9E1,4E0,4E0,2.1E1,4.8E1,5.5E1,8E0,7.4E1,7E0,4.5E1,8.2E1,1.3E1,1.3E1,8.3E1,5E0,8E0,1.4E1,7E0,6E0,4E0,5E0,7.2E1,5.8E1,4E0,1.5E1,1.2E1,2.7E1,1.4E1,7E0,6E0,4.2E1,3.8E1,1.7E1,3.3E1,4.1E1,2E1,2.5E1,5E1,3.2E1,3.8E1,4.5E1,9E0,5E0,6.5E1,7E0,2.2E1,3.6E1,9E0,6E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[2.0819542E-3,-1.7877074E-2,3.76012E-2,9.933179E-4,-4.062655E-2,5.258583E-2,-5.2450586E-2,-1.16662E-2,3.8404495E-2,-7.575788E-2,-2.2136638E-2,1.5701999E-1,4.148614E-2,-1.0884336E-2,-2.4057562E-2,-1.7698543E-2,5.362315E-2,2.4370957E-2,4.53019E-3,-4.9488153E-2,-2.3734702E-1,6.727523E-4,-4.7405884E-2,1.7948905E-1,4.0548347E-4,4.580467E-2,-9.699776E-2,-8.577646E-2,1.3775699E-2,-1.4497971E-2,-5.791575E-3,4.737488E-3,2.0695077E-2,-1.1189776E-2,3.9343145E-2,-5.96843E-2,2.8277691E-3,-1.9832907E-2,-5.4348363E-3,-2.0492876E-2,4.6831917E-2,-8.729253E-3,-3.721653E-2,3.3127645E-3,9.046999E-3,1.2169534E-2,6.482141E-2,-8.060399E-3,-5.7684047E-6,-4.000148E-4,-1.236139E-1,8.4833086E-2,-4.555248E-2,-6.140423E-2,-9.027213E-3,3.7792719E-3,-4.1408776E-4,8.329023E-4,-3.505223E-2,5.7367004E-2,1.1096895E-3,-4.2713508E-2,-1.390629E-1,-6.9562527E-3,-4.7919643E-3,4.4867317E-3,1.7274745E-2,2.6133945E-3,-4.476868E-2,5.975969E-3,4.7956347E-3,5.7432503E-2,7.495726E-3,-7.3635615E-3,-1.9243296E-3,1.2752093E-1,-8.089096E-4,2.5987994E-2,-6.9975727E-3,1.3696582E-3,-9.171758E-2,5.0598115E-2,-1.3074896E-2,-3.5596162E-3,5.521968E-4,4.8075696E-3,3.7947826E-2,-1.4810581E-3,1.161024E-3,-8.900546E-2,-1.6046567E-2,-1.6721735E-3,-9.779852E-3,-4.091469E-2,1.6461527E-2,-9.887124E-4,2.1111395E-3,-6.1900683E-2,-9.517646E-3,1.3114582E-2,-3.6036347E-3,8.496984E-2,2.7851483E-2,2.9190194E-3,8.279868E-3,3.5521688E-3,-2.331671E-3,-8.6891875E-3,-5.348176E-2,-2.4155518E-3,4.4803945E-3,-1.7102772E-2,5.2796353E-2,4.4994867E-5,6.330651E-2,-1.5464537E-3,-1.087415E-1,2.415385E-2,-4.13166E-2,-3.5429904E-3,3.5088528E-5,2.952218E-2,-1.672692E-3,-3.0730983E-2,-7.503631E-2,3.6828525E-2,-5.2139565E-2,-1.5952941E-2,3.6106307E-2,1.0857082E-1,-3.3979516E-2,3.496704E-2,-3.146467E-3,-8.7539974E-4,-4.7180164E-3,1.7066026E-3,-2.1535968E-2,3.934041E-4,5.2244924E-3,4.645524E-3,6.231866E-4,-1.5208583E-3,-1.2805347E-1,2.1235573E-3,-1.9287746E-3,-6.4249575E-2,2.951061E-4,-3.7301495E-4,5.7123598E-2,-2.2669241E-3,6.5244216E-4,-1.1241767E-1,-4.8546452E-2,-1.3279982E-3,4.1395235E-3,-1.6305559E-4,-4.3534413E-3,1.7959678E-3,-3.5524596E-2,5.1395256E-2,-3.9253917E-2,1.193095E-1,4.493497E-2,-9.086982E-3,4.3069743E-2,-2.7071675E-2,4.799165E-2,-5.2263984E-4,-3.4198577E-3,-7.490021E-3,-2.5688484E-3,-4.786176E-3,-1.2110481E-3,3.5636595E-3,1.9590536E-4,-6.549292E-3,-2.0684865E-3,-3.534229E-3,-8.781887E-4,5.2576273E-4,-2.7509732E-3,-4.5676634E-4,2.684451E-3,-3.661148E-3,3.968595E-4,5.820149E-3,1.7171294E-3,-1.5692565E-3,5.982705E-3,4.145085E-3,-4.971527E-4,-3.1300636E-3,1.7768291E-3,3.223041E-3,8.920452E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,51,53,-1,-1,55,57,59,61,-1,-1,-1,63,65,-1,67,-1,-1,69,71,-1,-1,-1,73,75,77,79,81,-1,-1,-1,83,85,87,89,91,93,-1,-1,95,-1,97,-1,99,101,-1,-1,-1,103,-1,105,-1,-1,107,109,111,-1,-1,-1,113,-1,-1,115,117,-1,-1,119,121,-1,-1,123,125,127,-1,129,131,-1,-1,-1,-1,-1,133,-1,-1,135,137,-1,139,-1,141,143,145,-1,-1,147,-1,149,151,153,155,157,159,161,163,165,-1,-1,-1,-1,167,-1,-1,-1,-1,-1,169,-1,-1,171,-1,-1,173,-1,-1,175,177,-1,-1,-1,-1,-1,179,181,183,185,187,-1,189,191,193,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.251983E-1,2.8122628E-1,4.9933475E-1,1.7047313E-1,1.9124687E-1,3.5978878E-1,2.699357E-1,1.06894106E-1,7.379095E-2,4.212342E-1,1.1300161E-1,9.835112E-2,1.7608231E-1,0E0,1.109279E-1,8.329445E-2,3.492868E-2,4.037189E-2,0E0,1.0410555E-1,2.711805E-1,1.02566436E-1,1.2709537E-1,3.174132E-2,0E0,1.776681E-1,6.752383E-2,5.1452726E-2,1.3033552E-1,6.1289374E-2,0E0,0E0,2.7918976E-2,1.6668027E-2,3.645941E-2,1.0452375E-1,0E0,0E0,0E0,8.0398895E-2,4.7370106E-2,0E0,6.486076E-2,0E0,0E0,8.7517515E-2,1.1882007E-1,0E0,0E0,0E0,2.8545663E-2,6.3395604E-2,1.3274671E-1,7.172075E-2,5.30333E-2,0E0,0E0,0E0,2.658392E-2,2.8695159E-2,1.5146844E-2,8.3749846E-2,9.69311E-2,5.077308E-2,0E0,0E0,2.516125E-2,0E0,4.893236E-2,0E0,6.78358E-2,1.3366872E-1,0E0,0E0,0E0,2.1939024E-2,0E0,4.668976E-2,0E0,0E0,5.9947595E-2,7.1878284E-2,5.5028502E-2,0E0,0E0,0E0,2.5253791E-2,0E0,0E0,2.3759425E-2,4.6711914E-2,0E0,0E0,3.8821492E-2,2.7208135E-2,0E0,0E0,2.0212904E-2,5.723038E-2,6.005582E-2,0E0,2.451188E-1,5.837079E-2,0E0,0E0,0E0,0E0,0E0,2.326142E-2,0E0,0E0,4.7035083E-2,3.1073507E-2,0E0,2.8333165E-2,0E0,2.2141352E-2,2.7793076E-2,3.1042457E-2,0E0,0E0,3.2630235E-2,0E0,1.5498072E-2,3.2171145E-2,5.1023245E-2,3.0359399E-2,4.4303596E-2,5.9626386E-2,4.3841183E-2,1.9652459E-1,6.351751E-2,0E0,0E0,0E0,0E0,9.5213816E-2,0E0,0E0,0E0,0E0,0E0,2.5543764E-2,0E0,0E0,2.5721401E-2,0E0,0E0,1.8723667E-2,0E0,0E0,2.3427978E-2,1.836222E-2,0E0,0E0,0E0,0E0,0E0,3.4479495E-2,1.7847449E-2,1.7901774E-2,2.8740168E-2,8.44657E-2,0E0,2.8590323E-2,3.9927527E-2,3.862101E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,40,40,42,42,45,45,46,46,50,50,51,51,52,52,53,53,54,54,58,58,59,59,60,60,61,61,62,62,63,63,66,66,68,68,70,70,71,71,75,75,77,77,80,80,81,81,82,82,86,86,89,89,90,90,93,93,94,94,97,97,98,98,99,99,101,101,102,102,108,108,111,111,112,112,114,114,116,116,117,117,118,118,121,121,123,123,124,124,125,125,126,126,127,127,128,128,129,129,130,130,131,131,136,136,142,142,145,145,148,148,151,151,152,152,158,158,159,159,160,160,161,161,162,162,164,164,165,165,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,52,54,-1,-1,56,58,60,62,-1,-1,-1,64,66,-1,68,-1,-1,70,72,-1,-1,-1,74,76,78,80,82,-1,-1,-1,84,86,88,90,92,94,-1,-1,96,-1,98,-1,100,102,-1,-1,-1,104,-1,106,-1,-1,108,110,112,-1,-1,-1,114,-1,-1,116,118,-1,-1,120,122,-1,-1,124,126,128,-1,130,132,-1,-1,-1,-1,-1,134,-1,-1,136,138,-1,140,-1,142,144,146,-1,-1,148,-1,150,152,154,156,158,160,162,164,166,-1,-1,-1,-1,168,-1,-1,-1,-1,-1,170,-1,-1,172,-1,-1,174,-1,-1,176,178,-1,-1,-1,-1,-1,180,182,184,186,188,-1,190,192,194,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,3.875E0,1E0,3.422351E6,1E0,1.8992E4,4.5E0,2.0277777E0,6.965855E9,1.8256016E7,1.45625E1,8.837831E-2,3.206931E2,-1.0884336E-2,1.4598765E0,5.2E1,3.1842105E0,3.011015E8,4.53019E-3,4.5E1,3.6255838E2,1.6223962E7,6.8E1,4.5496914E9,4.0548347E-4,5.817547E2,3.82E3,1.7234043E1,2.4E0,7.7E1,-5.791575E-3,4.737488E-3,1.4473684E-1,3.72E2,1.5321098E8,6.18826E5,2.8277691E-3,-1.9832907E-2,-5.4348363E-3,6.6358675E8,1.0486312E9,-8.729253E-3,2E0,3.3127645E-3,9.046999E-3,9.9688476E-1,4.907764E3,-8.060399E-3,-5.7684047E-6,-4.000148E-4,5.467E3,1.7E1,2E1,4.6874816E4,1.6E1,3.7792719E-3,-4.1408776E-4,8.329023E-4,8E0,1.5123151E6,4E0,2.4636364E1,5.3E1,4.1719616E5,-4.7919643E-3,4.4867317E-3,1.3881669E6,2.6133945E-3,3.885813E2,5.975969E-3,2E1,1.7951853E1,7.495726E-3,-7.3635615E-3,-1.9243296E-3,1.2697166E3,-8.089096E-4,8.0274E4,-6.9975727E-3,1.3696582E-3,7E0,2.66E2,1.775373E1,-3.5596162E-3,5.521968E-4,4.8075696E-3,1.5849056E0,-1.4810581E-3,1.161024E-3,3.8E1,8E0,-1.6721735E-3,-9.779852E-3,2.240836E6,1E0,-9.887124E-4,2.1111395E-3,7.106795E6,7.836E3,6.185E3,-3.6036347E-3,3.174172E5,1.514526E6,2.9190194E-3,8.279868E-3,3.5521688E-3,-2.331671E-3,-8.6891875E-3,7.042942E7,-2.4155518E-3,4.4803945E-3,3.5511714E-1,2.23607E5,4.4994867E-5,2.0467092E6,-1.5464537E-3,1.863E3,2.9E1,1.802361E6,-3.5429904E-3,3.5088528E-5,2.1651703E2,-1.672692E-3,1.7E1,1.0041E5,2.2166292E1,1.763E3,3.819455E-1,1.1108876E11,1.6063418E7,1.4598765E0,1.4407171E8,-3.146467E-3,-8.7539974E-4,-4.7180164E-3,1.7066026E-3,1.0292E4,3.934041E-4,5.2244924E-3,4.645524E-3,6.231866E-4,-1.5208583E-3,1.00011E5,2.1235573E-3,-1.9287746E-3,4.6748266E0,2.951061E-4,-3.7301495E-4,1.946094E6,-2.2669241E-3,6.5244216E-4,2.0518E2,5.4051723E0,-1.3279982E-3,4.1395235E-3,-1.6305559E-4,-4.3534413E-3,1.7959678E-3,1.136E3,5.036081E7,1.2193182E1,4.7E1,8.710612E1,-9.086982E-3,1.0236775E1,2.5205562E5,4.857143E0,-5.2263984E-4,-3.4198577E-3,-7.490021E-3,-2.5688484E-3,-4.786176E-3,-1.2110481E-3,3.5636595E-3,1.9590536E-4,-6.549292E-3,-2.0684865E-3,-3.534229E-3,-8.781887E-4,5.2576273E-4,-2.7509732E-3,-4.5676634E-4,2.684451E-3,-3.661148E-3,3.968595E-4,5.820149E-3,1.7171294E-3,-1.5692565E-3,5.982705E-3,4.145085E-3,-4.971527E-4,-3.1300636E-3,1.7768291E-3,3.223041E-3,8.920452E-4],"split_indices":[2,69,6,9,29,9,73,68,46,5,73,72,73,0,69,8,69,46,0,3,4,5,0,46,0,67,0,71,69,44,0,0,71,44,7,9,0,0,0,46,46,0,32,0,0,69,4,0,0,0,2,8,6,60,0,0,0,0,8,60,3,67,0,60,0,0,43,0,67,0,3,71,0,0,0,4,0,1,0,0,12,1,71,0,0,0,71,0,0,44,10,0,0,9,30,0,0,5,44,44,0,48,10,0,0,0,0,0,60,0,0,69,9,0,43,0,9,3,12,0,0,67,0,3,1,73,2,57,46,62,69,5,0,0,0,0,44,0,0,0,0,0,5,0,0,69,0,0,60,0,0,67,69,0,0,0,0,0,44,7,73,8,73,0,71,48,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.021E3,6.54E2,3.67E2,3.58E2,2.96E2,3.15E2,5.2E1,2.68E2,9E1,1.01E2,1.95E2,2.9E1,2.86E2,6E0,4.6E1,2.46E2,2.2E1,7.4E1,1.6E1,8.8E1,1.3E1,1.03E2,9.2E1,2.5E1,4E0,2.78E2,8E0,1.7E1,2.9E1,2.4E2,6E0,8E0,1.4E1,2.2E1,5.2E1,8.1E1,7E0,4E0,9E0,7.1E1,3.2E1,5E0,8.7E1,4E0,2.1E1,1.01E2,1.77E2,4E0,4E0,6E0,1.1E1,1.3E1,1.6E1,2.4E1,2.16E2,4E0,1E1,1E1,1.2E1,3.5E1,1.7E1,6.8E1,1.3E1,6.2E1,9E0,1.1E1,2.1E1,6E0,8.1E1,5E0,9.6E1,1.66E2,1.1E1,7E0,4E0,9E0,4E0,1E1,6E0,6E0,1.8E1,1.3E1,2.03E2,6E0,6E0,9E0,2.6E1,7E0,1E1,2.4E1,4.4E1,6E0,7E0,2.5E1,3.7E1,9E0,1.2E1,5.4E1,2.7E1,8.8E1,8E0,8.5E1,8.1E1,5E0,4E0,6E0,4E0,4E0,1.4E1,4E0,9E0,1.92E2,1.1E1,1.1E1,1.5E1,7E0,1.7E1,1.7E1,2.7E1,1.3E1,1.2E1,3E1,7E0,1.7E1,3.7E1,1.3E1,1.4E1,3.9E1,4.9E1,7.1E1,1.4E1,7.6E1,5E0,9E0,5E0,1.4E1,1.78E2,7E0,4E0,8E0,7E0,4E0,1.3E1,1.3E1,4E0,1.8E1,9E0,1.3E1,1.7E1,1.2E1,5E0,1.4E1,2.3E1,6E0,7E0,7E0,7E0,1E1,2.9E1,4.1E1,8E0,6E1,1.1E1,4E0,1E1,1.3E1,6.3E1,1.5E2,2.8E1,8E0,5E0,8E0,1E1,1.2E1,5E0,9E0,5E0,1.1E1,1.2E1,1E1,1.9E1,4E0,3.7E1,4E0,4E0,5.5E1,5E0,6E0,5E0,5E0,5E0,8E0,5E0,3.5E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"195","size_leaf_vector":"1"}},{"base_weights":[-5.098481E-4,-1.4738046E-2,4.0954E-2,-1.9372182E-2,5.9637774E-2,-1.23512626E-1,4.744676E-2,-1.7254204E-2,-1.8077473E-1,4.593562E-2,5.7980376E-3,-2.4887423E-3,-8.425841E-3,1.241756E-2,4.2697247E-2,-2.8662639E-2,8.007986E-3,-1.3674575E-2,-1.3583421E-3,3.2477606E-2,5.1880954E-3,5.212447E-2,-3.798862E-2,-1.542429E-2,-5.676903E-2,1.2686985E-2,-6.1987303E-3,-5.8552832E-5,4.940772E-2,3.7044518E-2,8.7050445E-2,3.6837216E-3,-7.729276E-2,-1.194912E-2,-1.3688593E-1,-1.1681E-2,-4.563655E-2,-2.3688074E-2,2.2412676E-2,3.8328255E-4,6.8039946E-2,1.10149E-2,5.3999268E-2,9.8913275E-2,2.2469626E-3,-2.5181672E-2,-1.3818614E-1,-3.8508173E-2,3.4171776E-3,-8.849732E-3,-2.5334677E-3,-8.2773045E-2,-1.645134E-2,-1.2277105E-2,-4.426478E-3,2.6620286E-2,-3.583389E-3,4.267272E-3,7.463321E-4,-3.0195527E-3,1.8861081E-2,6.5743993E-3,3.9159935E-2,6.2475264E-2,1.2962498E-1,2.8180457E-3,-2.631192E-3,-2.4598902E-3,1.1437379E-3,-1.0430779E-2,-1.0605438E-3,-3.1210838E-2,-9.042371E-3,1.0638501E-2,-2.299947E-3,-9.830764E-3,-7.055862E-2,-5.971306E-3,-4.361153E-3,-2.1943724E-4,-2.9575764E-3,3.4335148E-2,-2.6447143E-2,2.4948966E-2,-2.2842064E-3,9.353032E-2,2.4337528E-2,7.833544E-2,1.7111906E-4,7.3710205E-3,3.300594E-3,-4.6692163E-2,-4.9228165E-3,7.3657366E-3,-6.4146735E-2,-1.0125736E-1,-3.6484636E-2,-1.7468985E-2,5.0467397E-3,4.2490862E-2,-9.65095E-5,1.3811332E-3,-1.0489206E-1,-1.7991523E-3,3.0959442E-2,5.7429476E-3,-2.4022316E-4,4.5577753E-3,1.3667204E-2,4.3751905E-3,8.733776E-4,-1.5174905E-3,-5.508142E-3,2.0170852E-3,-1.6322191E-3,8.7473524E-4,-1.7178691E-3,1.8310734E-3,-4.1364287E-3,-5.4006767E-3,-7.667031E-4,-4.4859826E-4,-5.6857583E-3,-2.0381626E-4,-3.969211E-3,8.557374E-4,2.5274237E-3,2.0049165E-3,-1.0617899E-3,-6.7551206E-3,-1.9675244E-3,1.9234428E-3,-6.853169E-4,-3.649284E-3,1.1041055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,-1,27,-1,29,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,-1,51,53,55,-1,57,59,61,63,65,67,69,71,73,-1,-1,75,77,79,-1,81,-1,-1,-1,-1,83,-1,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,-1,93,-1,95,-1,97,-1,-1,99,101,103,-1,105,107,109,-1,-1,-1,111,113,115,117,119,121,123,-1,125,127,-1,129,-1,131,-1,-1,-1,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9114933E-1,2.5827742E-1,2.79228E-1,2.3719943E-1,3.3943772E-2,3.1172812E-2,2.4818105E-1,2.0052996E-1,1.4826483E-1,3.02912E-2,0E0,0E0,0E0,0E0,1.8681666E-1,1.7735177E-1,1.4606614E-1,0E0,0E0,1.8915746E-2,0E0,1.11591935E-1,1.2646657E-1,1.3639097E-1,3.224242E-1,7.517694E-2,0E0,0E0,1.6287457E-2,6.736934E-2,6.516829E-2,0E0,5.9984647E-2,1.3032006E-1,2.9865876E-2,0E0,1.5687838E-1,3.5859473E-2,7.231554E-2,0E0,1.7441131E-2,3.838615E-2,1.16295636E-1,5.56888E-2,3.4776725E-2,1.8135723E-2,8.5657164E-2,1.2812224E-1,2.6381576E-1,0E0,0E0,8.6624116E-2,1.1262984E-1,1.52747E-2,0E0,6.6822745E-2,0E0,0E0,0E0,0E0,2.4935605E-2,0E0,6.3001595E-2,2.580931E-2,4.0832877E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.5758143E-2,0E0,0E0,1.19433E-1,0E0,5.906737E-2,0E0,1.1507625E-1,0E0,0E0,3.959897E-2,9.595109E-2,2.1558948E-2,0E0,5.230528E-2,5.019623E-2,1.9609943E-2,0E0,0E0,0E0,6.738642E-2,6.506628E-2,8.986241E-2,7.468872E-2,3.9114714E-2,6.8354115E-2,6.1287977E-2,0E0,3.236623E-2,2.8869137E-2,0E0,1.694569E-2,0E0,2.45696E-2,0E0,0E0,0E0,5.6874275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,19,19,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,32,32,33,33,34,34,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,51,51,52,52,53,53,55,55,60,60,62,62,63,63,64,64,71,71,74,74,76,76,78,78,81,81,82,82,83,83,85,85,86,86,87,87,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,102,102,104,104,108,108],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,-1,28,-1,30,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,-1,52,54,56,-1,58,60,62,64,66,68,70,72,74,-1,-1,76,78,80,-1,82,-1,-1,-1,-1,84,-1,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,-1,94,-1,96,-1,98,-1,-1,100,102,104,-1,106,108,110,-1,-1,-1,112,114,116,118,120,122,124,-1,126,128,-1,130,-1,132,-1,-1,-1,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.984E3,4.5866325E6,1E0,5.9E1,1.339646E6,7.472532E9,3.3817584E7,2.026453E6,4.0568292E2,2.712766E0,5.7980376E-3,-2.4887423E-3,-8.425841E-3,1.241756E-2,3.1879792E8,3.39498E5,1E0,-1.3674575E-2,-1.3583421E-3,1.2E1,5.1880954E-3,1.4817301E3,7.556899E-1,4.217427E7,9E0,5.88E2,-6.1987303E-3,-5.8552832E-5,5.294E3,1.1251919E6,1.2820834E7,3.6837216E-3,1.3475722E2,1.7667647E2,1.0479E4,-1.1681E-2,1.084437E6,2.8608696E1,4.643602E8,3.8328255E-4,2.138772E4,3.167E3,3.443E3,1.6326721E0,3.359873E6,1.1863237E2,2.0120485E0,7.45E2,7.336111E4,-8.849732E-3,-2.5334677E-3,3.72424E5,1.318E2,1.0066326E8,-4.426478E-3,1E0,-3.583389E-3,4.267272E-3,7.463321E-4,-3.0195527E-3,5.433518E0,6.5743993E-3,7.5052085E0,1.4052098E0,1E0,2.8180457E-3,-2.631192E-3,-2.4598902E-3,1.1437379E-3,-1.0430779E-2,-1.0605438E-3,8.2765434E1,-9.042371E-3,1.0638501E-2,1.647E4,-9.830764E-3,3.1275E4,-5.971306E-3,1.9791039E6,-2.1943724E-4,-2.9575764E-3,1.7E1,5.036784E6,2.9585715E2,-2.2842064E-3,8.76E2,1.6208625E6,5.501002E9,1.7111906E-4,7.3710205E-3,3.300594E-3,4.577342E0,4.8E1,6.2222223E0,1.3E1,9.393264E-6,2.0734E4,2.4E1,5.0467397E-3,1.2200055E7,5.44E2,1.3811332E-3,1.11E2,-1.7991523E-3,7.973882E7,5.7429476E-3,-2.4022316E-4,4.5577753E-3,3.921E3,4.3751905E-3,8.733776E-4,-1.5174905E-3,-5.508142E-3,2.0170852E-3,-1.6322191E-3,8.7473524E-4,-1.7178691E-3,1.8310734E-3,-4.1364287E-3,-5.4006767E-3,-7.667031E-4,-4.4859826E-4,-5.6857583E-3,-2.0381626E-4,-3.969211E-3,8.557374E-4,2.5274237E-3,2.0049165E-3,-1.0617899E-3,-6.7551206E-3,-1.9675244E-3,1.9234428E-3,-6.853169E-4,-3.649284E-3,1.1041055E-3],"split_indices":[2,43,17,3,9,46,7,9,4,68,0,0,0,0,47,9,79,0,0,3,0,67,57,60,3,2,0,0,12,43,9,0,73,70,9,0,9,73,7,0,62,2,2,53,1,73,50,2,60,0,0,9,70,7,0,19,0,0,0,0,68,0,71,53,30,0,0,0,0,0,0,4,0,0,44,0,1,0,43,0,0,3,9,67,0,0,43,5,0,0,0,73,10,61,3,52,44,8,0,5,10,0,0,0,47,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.45E2,2.55E2,7.02E2,4.3E1,9E0,2.46E2,6.94E2,8E0,3.7E1,6E0,5E0,4E0,4E0,2.42E2,4.78E2,2.16E2,4E0,4E0,3.2E1,5E0,2.17E2,2.5E1,3.26E2,1.52E2,2.1E2,6E0,1.1E1,2.1E1,1.53E2,6.4E1,6E0,1.9E1,3.18E2,8E0,7E0,1.45E2,4.4E1,1.66E2,7E0,1.4E1,6.1E1,9.2E1,5.6E1,8E0,1.1E1,8E0,1.16E2,2.02E2,4E0,4E0,6.3E1,8.2E1,3.9E1,5E0,1.6E2,6E0,9E0,5E0,5E0,5.6E1,1.2E1,8E1,2.7E1,2.9E1,4E0,4E0,7E0,4E0,4E0,4E0,1.12E2,4E0,4E0,1.98E2,4E0,5.9E1,7E0,7.5E1,3.5E1,4E0,1.4E2,2E1,5.2E1,4E0,1.6E1,6.4E1,2.1E1,6E0,1.8E1,1.1E1,7E1,4.2E1,1.72E2,2.6E1,3E1,2.9E1,6.8E1,7E0,1.13E2,2.7E1,1.2E1,8E0,4E0,4.8E1,1.2E1,4E0,7E0,5.7E1,1.6E1,5E0,6E1,1E1,1.6E1,2.6E1,1.37E2,3.5E1,5E0,2.1E1,2.5E1,5E0,2.3E1,6E0,5.8E1,1E1,3.9E1,7.4E1,9E0,1.8E1,4E0,4E0,3.9E1,9E0,5E0,5.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-2.3366245E-3,-3.3935077E-2,2.497764E-2,-2.9708486E-2,-1.417064E-1,1.0916311E-2,7.048774E-2,-5.9631854E-2,-1.710505E-2,-5.928723E-2,-1.5898887E-2,1.8216379E-2,-7.442907E-2,1.065097E-2,6.237094E-2,-6.490355E-3,-5.35755E-2,2.1833261E-2,-2.692233E-2,2.0630606E-3,-6.598452E-3,1.22842915E-1,1.4626896E-2,2.0293012E-2,-1.0934075E-1,7.504802E-2,1.263924E-2,2.874033E-3,-5.7744954E-2,4.9296217E-3,1.3267819E-2,-9.8109305E-2,-2.0045375E-2,3.0784048E-3,8.191784E-3,-1.9137459E-2,2.371617E-2,2.562346E-3,-1.1997796E-3,-6.3487045E-2,-1.717726E-1,5.0587215E-2,9.540118E-2,4.512991E-2,-5.0172056E-3,-4.9768772E-2,-1.0947064E-1,-1.1092251E-2,3.1169321E-2,-1.1857892E-1,-3.2473574E-5,-1.3697813E-1,-1.5462298E-2,-4.862707E-3,-1.2178419E-2,7.0062275E-3,2.062097E-2,-3.93082E-3,4.8236318E-5,-1.1115208E-2,-2.5332111E-3,-1.5583293E-2,7.068069E-2,1.5074437E-4,1.04736455E-1,5.898369E-3,6.16405E-4,-5.9721004E-2,-9.094102E-3,-1.3707307E-1,-9.85457E-4,2.3244622E-3,-1.6215943E-3,7.4759037E-3,2.9494262E-3,-1.9061627E-3,-1.6024344E-1,-2.5501123E-4,-1.1135857E-2,-1.8156148E-2,5.153377E-3,9.026356E-4,-4.2929705E-2,5.9875626E-2,1.4100203E-2,-3.0645975E-3,8.354735E-4,3.4172386E-2,4.3436084E-3,1.3397242E-1,6.27663E-2,1.3335259E-3,-6.647257E-2,2.3376613E-3,-5.1083006E-2,-1.4053211E-3,-8.620643E-3,-3.4271897E-4,2.7534587E-3,-4.703675E-3,-1.0241547E-2,-2.724116E-2,2.0867791E-2,-3.3161156E-2,2.4616713E-2,-5.1386394E-3,-1.7778542E-2,3.0733211E-2,1.102831E-1,-1.4536855E-2,2.4328878E-2,2.9816085E-3,-1.9684598E-4,1.5591888E-1,2.7542224E-3,1.4732647E-2,4.383693E-3,-7.45126E-2,-1.2692969E-2,-5.083454E-3,-6.1066314E-3,-8.612028E-3,-2.2743389E-2,1.0033327E-2,3.5435753E-3,1.8349178E-3,-2.3374162E-3,-1.2451997E-2,4.3717068E-2,-2.0938818E-3,1.480186E-2,4.4817934E-3,8.248015E-3,6.5334155E-3,1.8484099E-3,2.1140661E-2,-8.525985E-2,1.2984869E-2,7.387153E-2,9.635364E-3,4.96547E-3,2.6512677E-3,-1.0749991E-3,-4.4982047E-3,-2.5811733E-3,1.6035568E-3,-3.0436222E-3,2.394242E-3,-2.902059E-3,-6.9392077E-4,-2.77782E-3,-8.605957E-4,1.2717951E-3,-1.9096925E-3,1.6748591E-3,2.8841703E-3,-7.6629693E-4,3.143531E-3,-1.9126337E-3,-8.562794E-4,3.1547972E-3,-1.0556738E-4,4.88332E-3,1.106909E-4,-1.8539676E-2,8.1247033E-4,-4.814637E-3,5.3307763E-3,5.0010026E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,-1,27,29,31,-1,-1,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,-1,53,55,-1,-1,57,59,61,63,65,-1,67,69,71,73,75,-1,77,79,-1,81,-1,83,-1,-1,-1,-1,85,87,-1,89,-1,-1,91,93,95,-1,-1,-1,97,-1,-1,99,-1,-1,101,-1,103,105,107,109,-1,-1,111,-1,113,115,-1,117,-1,119,-1,-1,-1,-1,-1,-1,121,123,125,127,-1,129,131,133,135,137,-1,-1,139,-1,141,-1,143,145,-1,147,-1,149,151,-1,-1,-1,153,155,-1,157,-1,159,-1,-1,161,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0104896E-1,2.1625483E-1,3.575926E-1,1.7510304E-1,2.7991465E-1,2.6863444E-1,1.5710902E-1,6.005004E-2,1.2673634E-1,1.2794484E-1,0E0,1.4692304E-1,1.1465365E-1,0E0,7.9119E-2,0E0,6.6351265E-2,4.729026E-2,1.2727444E-1,0E0,0E0,2.7988851E-2,1.1830098E-1,1.771107E-2,6.0662955E-2,4.5457244E-2,1.11257784E-1,0E0,4.653603E-2,0E0,2.7382812E-2,4.5775697E-2,1.2757272E-1,0E0,0E0,4.7860697E-2,1.1685783E-1,0E0,0E0,2.2388108E-2,6.7287296E-2,6.492203E-2,4.6105564E-2,5.6021012E-2,0E0,4.4390082E-2,3.600219E-2,4.1570447E-2,2.7391069E-2,5.4596215E-2,0E0,1.21788606E-1,8.208712E-2,0E0,3.1012716E-2,0E0,7.517898E-2,0E0,0E0,0E0,0E0,2.1703668E-2,2.8313294E-2,0E0,5.171591E-2,0E0,0E0,5.514896E-2,6.0233835E-2,5.7708204E-2,0E0,0E0,0E0,1.7521743E-2,0E0,0E0,2.0508885E-2,0E0,0E0,8.211104E-2,0E0,4.5244843E-2,3.7531644E-2,5.8308467E-2,7.5340986E-2,0E0,0E0,1.8947946E-2,0E0,3.6396503E-2,3.254538E-2,0E0,3.4942567E-2,0E0,3.644753E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2963577E-1,2.5847668E-2,3.123142E-2,2.3909206E-2,0E0,1.6824268E-2,4.0927082E-2,2.7162075E-2,1.7338146E-1,1.0526112E-1,0E0,0E0,3.1133235E-2,0E0,1.7545E-2,0E0,2.5233984E-2,3.254157E-2,0E0,3.282278E-2,0E0,5.1932782E-2,1.9567434E-2,0E0,0E0,0E0,1.8380256E-2,2.5860652E-2,0E0,2.9747263E-2,0E0,3.6641683E-2,0E0,0E0,9.136312E-2,6.381247E-1,8.542231E-2,9.036061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,35,35,36,36,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,54,54,56,56,61,61,62,62,64,64,67,67,68,68,69,69,73,73,76,76,79,79,81,81,82,82,83,83,84,84,87,87,89,89,90,90,92,92,94,94,101,101,102,102,103,103,104,104,106,106,107,107,108,108,109,109,110,110,113,113,115,115,117,117,118,118,120,120,122,122,123,123,127,127,128,128,130,130,132,132,135,135,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,-1,28,30,32,-1,-1,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,-1,54,56,-1,-1,58,60,62,64,66,-1,68,70,72,74,76,-1,78,80,-1,82,-1,84,-1,-1,-1,-1,86,88,-1,90,-1,-1,92,94,96,-1,-1,-1,98,-1,-1,100,-1,-1,102,-1,104,106,108,110,-1,-1,112,-1,114,116,-1,118,-1,120,-1,-1,-1,-1,-1,-1,122,124,126,128,-1,130,132,134,136,138,-1,-1,140,-1,142,-1,144,146,-1,148,-1,150,152,-1,-1,-1,154,156,-1,158,-1,160,-1,-1,162,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.48E2,6.0692043E0,5.97E3,3.06E2,1.6493898E6,2.856934E7,1.2646534E4,6E0,3.4E1,3.3817584E7,-1.5898887E-2,2.04115E5,8.620714E2,1.065097E-2,2.7021693E8,-6.490355E-3,2.4E1,2.00087E5,6.593462E4,2.0630606E-3,-6.598452E-3,1.5E1,3.3193566E5,2.476353E-2,1.9885094E-8,1.4817301E3,8.81059E5,2.874033E-3,6.769866E7,4.9296217E-3,2.4E1,9.7199225E-1,1.65E2,3.0784048E-3,8.191784E-3,2.5454E4,9.9688476E-1,2.562346E-3,-1.1997796E-3,1.6E1,1.7308458E1,1E0,1.990351E1,6.9E1,-5.0172056E-3,2E1,4E0,1.8949389E6,1.1913043E0,1.1732674E1,-3.2473574E-5,1.01E2,2.2E1,-4.862707E-3,2.3600838E6,7.0062275E-3,2.28E0,-3.93082E-3,4.8236318E-5,-1.1115208E-2,-2.5332111E-3,1.0292E4,8.83E3,1.5074437E-4,2.3385885E0,5.898369E-3,6.16405E-4,8E0,1E0,3.14403E5,-9.85457E-4,2.3244622E-3,-1.6215943E-3,3.1857144E2,2.9494262E-3,-1.9061627E-3,1.5E1,-2.5501123E-4,-1.1135857E-2,2.9652428E6,5.153377E-3,2.7491847E5,8.35684E4,8.7151864E7,3E0,-3.0645975E-3,8.354735E-4,1.925508E6,4.3436084E-3,2.4508352E9,1.4176E4,1.3335259E-3,5.2083335E0,2.3376613E-3,6.44E2,-1.4053211E-3,-8.620643E-3,-3.4271897E-4,2.7534587E-3,-4.703675E-3,-1.0241547E-2,1.2954546E2,2.8198E2,9.99E2,6E0,-5.1386394E-3,2.0227273E0,1.6446976E3,2.8543878E6,4.5620965E6,1.0213677E3,2.9816085E-3,-1.9684598E-4,1.3009709E0,2.7542224E-3,2.0669324E3,4.383693E-3,3.3817584E7,1.3085094E2,-5.083454E-3,1.0146154E2,-8.612028E-3,2.4E1,7.318515E6,3.5435753E-3,1.8349178E-3,-2.3374162E-3,8.6363555E4,3.9416668E1,-2.0938818E-3,3.164034E0,4.4817934E-3,1.5272727E0,6.5334155E-3,1.8484099E-3,2.517059E6,3.289E3,1E0,6.797565E1,9.635364E-3,4.96547E-3,2.6512677E-3,-1.0749991E-3,-4.4982047E-3,-2.5811733E-3,1.6035568E-3,-3.0436222E-3,2.394242E-3,-2.902059E-3,-6.9392077E-4,-2.77782E-3,-8.605957E-4,1.2717951E-3,-1.9096925E-3,1.6748591E-3,2.8841703E-3,-7.6629693E-4,3.143531E-3,-1.9126337E-3,-8.562794E-4,3.1547972E-3,-1.0556738E-4,4.88332E-3,1.106909E-4,-1.8539676E-2,8.1247033E-4,-4.814637E-3,5.3307763E-3,5.0010026E-4],"split_indices":[2,68,2,2,60,60,48,67,0,7,0,5,4,0,47,0,2,5,43,0,0,0,43,53,52,67,9,0,7,0,0,53,12,0,0,9,69,0,0,8,61,29,73,8,0,3,8,60,71,73,0,0,71,0,62,0,71,0,0,0,0,44,2,0,53,0,0,3,20,9,0,0,0,67,0,0,3,0,0,43,0,66,48,7,8,0,0,43,0,12,2,0,69,0,1,0,0,0,0,0,0,4,67,2,8,0,68,48,60,62,70,0,0,68,0,4,0,7,4,0,4,0,8,60,0,0,0,48,71,0,53,0,71,0,0,62,44,79,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,4.83E2,5.59E2,4.66E2,1.7E1,4.28E2,1.31E2,1.37E2,3.29E2,1.3E1,4E0,3.95E2,3.3E1,5E0,1.26E2,8E0,1.29E2,6.6E1,2.63E2,6E0,7E0,1.2E1,3.83E2,9E0,2.4E1,1E2,2.6E1,4E0,1.25E2,5E0,6.1E1,2.2E1,2.41E2,7E0,5E0,8.1E1,3.02E2,5E0,4E0,1.5E1,9E0,4.7E1,5.3E1,2.1E1,5E0,1.1E2,1.5E1,2.6E1,3.5E1,1.8E1,4E0,8E0,2.33E2,5E0,7.6E1,6E0,2.96E2,1.1E1,4E0,5E0,4E0,1.1E1,3.6E1,5E0,4.8E1,5E0,1.6E1,8.8E1,2.2E1,1.1E1,4E0,7E0,1.9E1,2.1E1,1.4E1,7E0,1.1E1,4E0,4E0,2.29E2,4E0,5.4E1,2.2E1,4.1E1,2.55E2,4E0,7E0,1.5E1,2.1E1,2.7E1,2.1E1,6E0,8.2E1,9E0,1.3E1,4E0,7E0,1.7E1,4E0,7E0,4E0,1.86E2,4.3E1,2.2E1,3.2E1,5E0,1.7E1,2.7E1,1.4E1,6.7E1,1.88E2,8E0,7E0,2E1,7E0,9E0,1.2E1,7.1E1,1.1E1,5E0,8E0,4E0,1.82E2,3.7E1,6E0,4E0,1.8E1,1.1E1,2.1E1,9E0,8E0,6E0,2.1E1,9E0,5E0,4.5E1,2.2E1,1.54E2,3.4E1,8E0,1.2E1,4E0,5E0,3E1,4.1E1,6E0,5E0,4E0,4E0,1.52E2,3E1,1.4E1,2.3E1,7E0,4E0,1.6E1,5E0,4E0,4E0,1.5E1,6E0,3.6E1,9E0,1.8E1,4E0,1.49E2,5E0,2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-4.5186665E-4,-1.232754E-2,6.791436E-2,-7.3019736E-3,-1.1986783E-1,5.622875E-2,9.821498E-3,-3.862031E-2,4.5574345E-3,-5.202672E-2,-2.2494516E-1,6.8715096E-2,-1.8988553E-2,-3.50905E-2,-8.798329E-3,6.2457863E-3,2.3689324E-3,-1.2038623E-2,-9.681999E-3,-1.6154714E-2,-1.4987651E-3,3.7945434E-2,1.0940608E-1,1.04084895E-4,-3.3479459E-3,-4.1402604E-2,3.1994343E-2,-1.7647162E-2,1.6856538E-2,2.3580734E-3,-1.8790762E-3,5.4157972E-2,-7.975848E-2,1.3890329E-1,5.924813E-2,-3.4592424E-2,-9.539265E-3,9.611605E-3,3.8169099E-3,-7.954144E-3,-5.5693094E-2,7.216566E-4,3.537732E-2,7.475748E-3,3.7811138E-2,-6.551623E-3,-8.095593E-5,1.509483E-3,1.5291207E-1,3.152177E-2,4.6366863E-3,-1.8541634E-2,-5.3614337E-2,2.7443261E-3,-1.911738E-3,-2.6344648E-2,1.2626461E-2,1.1205216E-3,-6.4910285E-2,7.618614E-3,-8.210358E-3,4.5564607E-2,-4.3693557E-2,4.064799E-4,5.8580212E-2,7.876785E-3,2.772825E-3,2.7803134E-3,-7.7249296E-4,-5.1793877E-2,7.0877206E-3,-7.6312795E-2,-3.074104E-2,-3.449613E-2,6.624884E-2,4.4400066E-2,-7.1777026E-3,-3.344104E-2,-1.2182505E-1,1.3140672E-2,-7.1245424E-2,6.1933473E-2,-9.346008E-5,-1.1465894E-2,2.7383037E-2,1.8956016E-3,-1.2820043E-3,6.8836E-2,-8.67289E-4,-2.6966687E-2,-8.826871E-2,1.7107772E-2,-5.2004624E-3,5.1355433E-5,-8.601864E-2,-4.171948E-2,8.981079E-3,1.6906897E-2,-4.8552163E-2,4.93053E-4,5.016087E-3,6.374375E-2,-6.5613067E-4,-4.958542E-2,1.5265792E-2,1.6842806E-3,-2.0857288E-3,-7.179622E-3,-2.607858E-3,1.9787349E-2,-5.6860816E-2,-1.5366844E-4,-4.8411638E-3,3.7294164E-2,9.580491E-2,-3.755522E-2,4.569833E-2,-2.7183776E-3,7.637889E-2,3.7102813E-3,7.568201E-4,-3.0836621E-3,1.0747998E-3,-5.392878E-3,-3.3006503E-4,3.1681757E-3,4.949889E-5,-1.0641998E-3,-4.9084323E-3,-2.3102786E-3,4.358501E-4,2.782167E-3,-1.3352493E-3,2.3087817E-3,-2.8223842E-3,-3.4552247E-3,-1.4551475E-3,2.0838044E-4,3.847151E-3,-6.8767555E-4,-3.985468E-3,4.282039E-3,1.8890017E-4,-3.0996418E-4,1.7690883E-3,-5.667811E-3,-1.0058801E-3,5.650749E-4,3.2458662E-3,5.065372E-3,-4.668861E-4,-2.5756788E-3,9.3603204E-4,3.759572E-3,-1.0161304E-3,8.382565E-4,5.182506E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,-1,-1,27,29,-1,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,-1,55,57,59,61,-1,63,-1,-1,-1,65,67,-1,69,71,-1,-1,73,75,-1,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,-1,-1,119,-1,121,123,125,-1,-1,127,129,131,133,135,-1,-1,137,-1,139,141,-1,-1,-1,-1,143,145,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.338413E-1,4.7201926E-1,2.4502254E-1,3.1105202E-1,2.6631075E-1,1.3466966E-1,0E0,1.1657637E-1,1.7350456E-1,1.5497485E-1,3.4882414E-1,1.4859837E-1,2.401888E-2,9.7023636E-2,0E0,0E0,1.7399865E-1,4.00082E-2,0E0,0E0,0E0,1.3980736E-1,6.870532E-2,0E0,0E0,2.2557074E-1,2.1868177E-2,9.248954E-2,1.0396752E-1,0E0,0E0,1.047533E-1,4.3317165E-2,4.2624593E-2,2.0775795E-2,6.0069248E-2,0E0,4.0541988E-2,0E0,7.678715E-2,3.915572E-2,2.3149186E-1,1.3248476E-1,0E0,4.3480344E-2,0E0,0E0,0E0,3.550458E-2,2.0362806E-2,0E0,9.4112806E-2,4.490024E-2,0E0,0E0,8.304616E-2,6.0726658E-2,0E0,7.831499E-2,7.976023E-2,0E0,1.0762718E-1,2.829521E-1,2.5159061E-2,2.9851675E-2,0E0,0E0,0E0,0E0,4.1086927E-2,7.745904E-2,3.3911705E-2,2.0870771E-2,7.235059E-2,2.0012569E-2,4.2847946E-2,5.78636E-2,2.7009934E-2,2.6354477E-2,8.0222264E-2,2.6990686E-2,8.489132E-2,6.861842E-2,0E0,6.7863144E-2,0E0,0E0,1.7945185E-2,0E0,6.149973E-2,4.303725E-2,4.862351E-2,0E0,0E0,4.851055E-2,1.6157545E-2,2.3093084E-2,5.9796385E-2,3.2547817E-2,0E0,0E0,3.201489E-2,0E0,2.5446285E-2,3.4234248E-2,0E0,0E0,0E0,0E0,7.6748595E-2,3.197217E-2,0E0,0E0,5.118967E-2,6.559163E-2,2.4274735E-2,4.526421E-2,0E0,1.8811878E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,42,42,44,44,48,48,49,49,51,51,52,52,55,55,56,56,58,58,59,59,61,61,62,62,63,63,64,64,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,84,84,87,87,89,89,90,90,91,91,94,94,95,95,96,96,97,97,98,98,101,101,103,103,104,104,109,109,110,110,113,113,114,114,115,115,116,116,118,118],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,-1,-1,28,30,-1,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,-1,56,58,60,62,-1,64,-1,-1,-1,66,68,-1,70,72,-1,-1,74,76,-1,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,-1,-1,120,-1,122,124,126,-1,-1,128,130,132,134,136,-1,-1,138,-1,140,142,-1,-1,-1,-1,144,146,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.467E3,6.0692043E0,1.0534078E4,1.5152774E2,4.9E1,7.3274844E9,9.821498E-3,4.5E1,7.336111E4,2.0778275E-1,2.8E1,1.6975454E3,3.9661028E7,8.8474586E8,-8.798329E-3,6.2457863E-3,6.285983E5,2.857143E0,-9.681999E-3,-1.6154714E-2,-1.4987651E-3,3.0561172E5,1.7493458E7,1.04084895E-4,-3.3479459E-3,4.5865917E8,1.9290142E0,2.4218928E7,4.4935583E2,2.3580734E-3,-1.8790762E-3,1.81E2,1.36E2,6.6358675E8,1.2E1,4.201202E-2,-9.539265E-3,3.5714287E-1,3.8169099E-3,1.7161779E-1,6.613774E5,2.511E3,2.9366477E0,7.475748E-3,1.3543621E6,-6.551623E-3,-8.095593E-5,1.509483E-3,2.53E2,3.01E2,4.6366863E-3,3.54E1,3.1157124E-1,2.7443261E-3,-1.911738E-3,5.7E3,1.3586957E1,1.1205216E-3,9.6203804E-1,1.70595E5,-8.210358E-3,2E0,6.9508715E0,4.9538185E6,1E0,7.876785E-3,2.772825E-3,2.7803134E-3,-7.7249296E-4,6.0921145E4,1E0,2.6412E5,1.547E3,5.539245E6,2.5393645E5,5.9908E6,1.9E1,8.7961E4,2.909091E0,6.346204E-7,1.162E3,6.965855E9,2.9308079E1,-1.1465894E-2,1.3E1,1.8956016E-3,-1.2820043E-3,2.4262331E5,-8.67289E-4,7.785714E0,3.5070792E2,2.3603575E5,-5.2004624E-3,5.1355433E-5,1.1010101E0,9.29023E5,1.5E1,4.313463E-2,1.8987958E0,4.93053E-4,5.016087E-3,5.4603375E5,-6.5613067E-4,9.783615E4,1.8916E2,1.6842806E-3,-2.0857288E-3,-7.179622E-3,-2.607858E-3,1.71E2,1.4E1,-1.5366844E-4,-4.8411638E-3,2.62E3,2.1199985E-7,3.1566668E1,1.8729467E8,-2.7183776E-3,3.307E3,3.7102813E-3,7.568201E-4,-3.0836621E-3,1.0747998E-3,-5.392878E-3,-3.3006503E-4,3.1681757E-3,4.949889E-5,-1.0641998E-3,-4.9084323E-3,-2.3102786E-3,4.358501E-4,2.782167E-3,-1.3352493E-3,2.3087817E-3,-2.8223842E-3,-3.4552247E-3,-1.4551475E-3,2.0838044E-4,3.847151E-3,-6.8767555E-4,-3.985468E-3,4.282039E-3,1.8890017E-4,-3.0996418E-4,1.7690883E-3,-5.667811E-3,-1.0058801E-3,5.650749E-4,3.2458662E-3,5.065372E-3,-4.668861E-4,-2.5756788E-3,9.3603204E-4,3.759572E-3,-1.0161304E-3,8.382565E-4,5.182506E-3],"split_indices":[2,68,4,67,3,5,0,6,60,53,8,67,60,5,0,0,43,73,0,0,0,48,66,0,0,5,68,12,70,0,0,0,0,46,8,57,0,68,0,53,60,2,57,0,43,0,0,0,8,0,0,70,53,0,0,10,73,0,53,1,0,32,71,60,6,0,0,0,0,60,30,7,10,5,43,9,73,1,68,52,10,46,71,0,3,0,0,48,0,67,48,60,0,0,68,62,3,53,68,0,0,60,0,48,67,0,0,0,0,10,3,0,0,2,52,73,7,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,8.74E2,1.51E2,8.36E2,3.8E1,1.41E2,1E1,2.29E2,6.07E2,2.4E1,1.4E1,1.21E2,2E1,2.25E2,4E0,9E0,5.98E2,2E1,4E0,8E0,6E0,7E1,5.1E1,1.5E1,5E0,2.06E2,1.9E1,2.51E2,3.47E2,6E0,1.4E1,6.2E1,8E0,3.1E1,2E1,1.99E2,7E0,1.4E1,5E0,2.01E2,5E1,1.86E2,1.61E2,7E0,5.5E1,4E0,4E0,4E0,2.7E1,1.3E1,7E0,1.09E2,9E1,7E0,7E0,1.06E2,9.5E1,5E0,4.5E1,1.8E2,6E0,1.43E2,1.8E1,2E1,3.5E1,2.2E1,5E0,8E0,5E0,4.7E1,6.2E1,4.4E1,4.6E1,9.8E1,8E0,3.6E1,5.9E1,3E1,1.5E1,1.69E2,1.1E1,1.05E2,3.8E1,4E0,1.4E1,8E0,1.2E1,3.1E1,4E0,2.9E1,1.8E1,5.8E1,4E0,5E0,3.9E1,3.6E1,1E1,2.1E1,7.7E1,4E0,4E0,2.7E1,9E0,2E1,3.9E1,4E0,2.6E1,9E0,6E0,1.55E2,1.4E1,4E0,7E0,6.2E1,4.3E1,2.1E1,1.7E1,5E0,9E0,2.5E1,6E0,1.6E1,1.3E1,1.3E1,5E0,1.3E1,4.5E1,1E1,2.9E1,3.1E1,5E0,4E0,6E0,1.5E1,6E0,2.9E1,4.8E1,7E0,2E1,1.1E1,9E0,4E0,3.5E1,6.4E1,9.1E1,4E0,1E1,3.6E1,2.6E1,3.8E1,5E0,1.6E1,5E0,1.1E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-5.610202E-3,-2.4882287E-2,1.9601159E-2,-2.1088654E-2,-1.0734692E-1,3.3337165E-2,-3.329409E-2,-1.920235E-2,-6.76749E-3,-5.0205804E-2,-1.2932322E-2,7.034963E-2,1.8605804E-2,-7.561385E-2,-4.219084E-3,-3.5982504E-2,-8.757486E-3,-8.6721204E-2,2.284558E-3,3.6221486E-2,1.01760015E-1,6.303687E-3,7.63384E-2,-1.0180549E-1,7.938385E-4,-3.498649E-2,5.13732E-2,-3.1398974E-2,-9.969479E-3,-4.586434E-3,-5.447526E-2,1.0931337E-3,-1.2901136E-1,5.254168E-2,-1.9473212E-3,1.251664E-1,2.3713157E-2,-1.3147849E-1,1.3926416E-2,1.0949343E-1,1.1473775E-2,-6.844725E-2,-1.0083909E-2,1.8404823E-2,-7.399956E-2,6.8098265E-3,1.8153418E-2,-2.3674041E-2,-1.0536943E-1,-1.2631728E-3,-6.1594028E-2,-7.3459116E-3,-1.8739425E-2,-8.61497E-3,-3.212482E-3,2.9867974E-3,-2.1978188E-3,8.739188E-3,9.657545E-2,-1.7832507E-3,3.1176666E-3,-1.6504169E-2,2.2435505E-3,1.8965283E-2,-5.8336486E-3,6.9539277E-3,7.605873E-2,-3.957373E-3,5.09897E-2,-4.846418E-3,1.8332672E-4,-4.652249E-4,3.1203397E-3,-9.6827786E-4,-1.2031306E-1,3.271095E-3,-1.2215601E-2,-3.4606993E-2,3.2107074E-2,-9.164775E-3,-4.7183715E-2,-4.5385556E-3,7.5813584E-2,-7.9530117E-4,-5.052743E-3,1.1326865E-3,-4.520894E-2,5.964977E-3,3.0496719E-3,1.5158699E-2,6.2111383E-3,2.0621126E-3,6.073613E-3,3.6040123E-3,-1.1279836E-4,-2.0216068E-3,-8.1687E-3,-2.8147774E-3,1.1237209E-3,-4.480446E-3,-3.0128418E-2,-1.9423816E-3,2.563411E-3,-6.5087766E-4,-4.4960114E-3,-8.701281E-3,5.6998145E-2,4.872609E-3,6.0986367E-4,-1.0892138E-2,-4.841695E-3,4.283715E-2,2.9996873E-3,-2.5197765E-2,-3.8575483E-3,4.241282E-2,-1.3967468E-2,-1.735478E-4,1.181702E-1,-2.3454702E-3,1.1158811E-3,7.434839E-2,2.5124997E-2,-8.852257E-3,4.6914063E-2,-1.4146487E-3,1.3151615E-3,5.5539194E-3,1.0102954E-3,-3.1213704E-3,-4.4343178E-4,7.643544E-3,2.1832879E-3,5.292591E-4,4.1281376E-3,-1.8285833E-3,1.8947485E-3,-2.8258418E-3,2.3657916E-4,6.076838E-3,8.2495087E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,43,45,47,-1,49,51,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,81,83,-1,85,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,89,-1,-1,91,-1,93,-1,-1,-1,-1,-1,95,-1,97,99,101,-1,103,105,107,-1,-1,-1,109,-1,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,-1,115,117,-1,-1,119,-1,121,123,125,-1,127,129,-1,131,-1,-1,133,135,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.921937E-1,1.7730671E-1,3.200739E-1,1.2728222E-1,2.3248035E-1,1.8898124E-1,1.1165167E-1,9.485452E-2,0E0,7.706805E-2,0E0,1.0243991E-1,1.777873E-1,9.1677904E-2,9.587104E-2,1.6465202E-1,6.3822396E-2,7.307405E-2,0E0,6.488806E-2,9.019852E-2,2.2028844E-1,9.33696E-2,9.772739E-2,0E0,7.634847E-2,6.1125174E-2,1.140745E-1,0E0,5.8462597E-2,1.0161383E-1,0E0,2.4627939E-2,5.2779123E-2,0E0,5.3091526E-2,3.8061373E-2,4.6445948E-1,1.4227024E-1,2.8332144E-2,6.578051E-2,6.3775264E-2,0E0,2.3066547E-2,4.9064934E-2,0E0,2.644825E-2,1.14543654E-1,9.346111E-2,7.431366E-2,3.4161016E-2,0E0,2.6298203E-2,0E0,0E0,0E0,0E0,0E0,2.031818E-2,0E0,0E0,0E0,0E0,8.236056E-2,0E0,0E0,2.5471538E-2,0E0,1.7515557E-2,0E0,0E0,0E0,0E0,0E0,3.984329E-2,0E0,2.1060588E-2,4.0656433E-2,5.607393E-2,0E0,2.0198671E-2,7.273318E-2,2.089008E-2,0E0,0E0,0E0,2.7488498E-2,0E0,0E0,6.2804975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.657894E-2,0E0,0E0,0E0,0E0,7.192415E-2,6.734069E-2,0E0,0E0,1.5201393E-2,0E0,2.9945523E-2,6.859696E-2,4.0450916E-2,0E0,3.7332352E-2,5.5786673E-2,0E0,2.2354305E-2,0E0,0E0,1.7852925E-2,4.0688187E-2,7.633678E-2,6.650075E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,52,52,58,58,63,63,66,66,68,68,74,74,76,76,77,77,78,78,80,80,81,81,82,82,86,86,89,89,100,100,105,105,106,106,109,109,111,111,112,112,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,44,46,48,-1,50,52,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,82,84,-1,86,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,90,-1,-1,92,-1,94,-1,-1,-1,-1,-1,96,-1,98,100,102,-1,104,106,108,-1,-1,-1,110,-1,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,-1,116,118,-1,-1,120,-1,122,124,126,-1,128,130,-1,132,-1,-1,134,136,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.850926E2,6.0692043E0,1E0,4.093403E6,1.6493898E6,2.165E3,5E0,1.8149019E2,-6.76749E-3,7.86127E6,-1.2932322E-2,7.257846E2,3.9152692E6,1.7308458E1,2.3751075E5,3.4E1,1E0,6.48334E5,2.284558E-3,4.1104166E8,1.9719212E1,1.2825651E0,1.5277778E0,4E0,7.938385E-4,8.7390656E2,3.2888E4,1E0,-9.969479E-3,4.2438595E1,2.1410204E2,1.0931337E-3,1.70698E6,2.4E1,-1.9473212E-3,1.55E2,1.10198E7,3.0984934E8,7.1E1,6.624E3,1E0,1.3026532E6,-1.0083909E-2,1.28636E7,4.3400474E0,6.8098265E-3,2.8827406E5,4.436827E7,3.7936746E3,2.89196E6,1.7181714E7,-7.3459116E-3,3.3272727E0,-8.61497E-3,-3.212482E-3,2.9867974E-3,-2.1978188E-3,8.739188E-3,6.9508715E0,-1.7832507E-3,3.1176666E-3,-1.6504169E-2,2.2435505E-3,4.1E1,-5.8336486E-3,6.9539277E-3,7.858646E-3,-3.957373E-3,2.7578741E1,-4.846418E-3,1.8332672E-4,-4.652249E-4,3.1203397E-3,-9.6827786E-4,4E0,3.271095E-3,1.332E3,9E0,1.45064E5,-9.164775E-3,6.666667E-1,1.9719212E1,1.7903225E0,-7.9530117E-4,-5.052743E-3,1.1326865E-3,1.1111689E8,5.964977E-3,3.0496719E-3,2E0,6.2111383E-3,2.0621126E-3,6.073613E-3,3.6040123E-3,-1.1279836E-4,-2.0216068E-3,-8.1687E-3,-2.8147774E-3,1.1237209E-3,-4.480446E-3,6.0147805E0,-1.9423816E-3,2.563411E-3,-6.5087766E-4,-4.4960114E-3,4.8679228E5,3.3E1,4.872609E-3,6.0986367E-4,4.7272725E0,-4.841695E-3,9.237895E0,1.9616238E2,5.25E0,-3.8575483E-3,1.9095E4,9.5679015E-2,-1.735478E-4,4.44E2,-2.3454702E-3,1.1158811E-3,1.0315458E8,5.658231E9,1E0,3.6828618E8,-1.4146487E-3,1.3151615E-3,5.5539194E-3,1.0102954E-3,-3.1213704E-3,-4.4343178E-4,7.643544E-3,2.1832879E-3,5.292591E-4,4.1281376E-3,-1.8285833E-3,1.8947485E-3,-2.8258418E-3,2.3657916E-4,6.076838E-3,8.2495087E-4],"split_indices":[67,68,6,66,60,44,6,70,0,5,0,4,43,61,48,3,19,46,0,7,71,69,68,6,0,4,9,30,0,73,67,0,5,3,0,10,62,7,3,2,27,43,0,60,61,0,48,5,48,62,12,0,69,0,0,0,0,0,71,0,0,0,0,3,0,0,49,0,71,0,0,0,0,0,8,0,44,70,7,0,71,71,68,0,0,0,7,0,0,32,0,0,0,0,0,0,0,0,0,0,69,0,0,0,0,60,8,0,0,69,0,71,71,69,0,12,73,0,0,0,0,7,46,29,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.011E3,5.73E2,4.38E2,5.49E2,2.4E1,3.48E2,9E1,5.42E2,7E0,1.9E1,5E0,9.8E1,2.5E2,3.6E1,5.4E1,2.07E2,3.35E2,1.4E1,5E0,4.8E1,5E1,2.07E2,4.3E1,2.8E1,8E0,3.5E1,1.9E1,2.03E2,4E0,3.08E2,2.7E1,4E0,1E1,4E1,8E0,3.8E1,1.2E1,1E1,1.97E2,2.8E1,1.5E1,2.3E1,5E0,1.5E1,2E1,4E0,1.5E1,1.85E2,1.8E1,2.92E2,1.6E1,6E0,2.1E1,4E0,6E0,3.6E1,4E0,1E1,2.8E1,5E0,7E0,4E0,6E0,1.91E2,6E0,1.1E1,1.7E1,4E0,1.1E1,1.5E1,8E0,1E1,5E0,1E1,1E1,5E0,1E1,1.55E2,3E1,6E0,1.2E1,2.81E2,1.1E1,9E0,7E0,8E0,1.3E1,1.2E1,1.6E1,1.86E2,5E0,1.2E1,5E0,7E0,4E0,5E0,5E0,4E0,6E0,9E0,1.46E2,7E0,2.3E1,8E0,4E0,2.64E2,1.7E1,7E0,4E0,9E0,4E0,5.6E1,1.3E2,1.35E2,1.1E1,2.4E1,2.4E2,9E0,8E0,4E0,5E0,1.9E1,3.7E1,1.03E2,2.7E1,1.23E2,1.2E1,4E0,2E1,1.7E1,2.23E2,4E0,4E0,4E0,1.5E1,7E0,3E1,2.1E1,8.2E1,6E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[-2.3832999E-3,-1.616509E-2,3.9556604E-2,-9.288826E-3,-1.17473E-1,4.8942212E-2,-3.1051794E-2,-7.5559868E-3,-1.0151071E-2,-1.9056906E-1,-1.5779432E-2,3.2307252E-2,8.266856E-2,4.129138E-3,-5.220338E-2,-5.2623376E-2,1.3472438E-4,4.1708484E-4,-2.4303037E-1,2.2713726E-3,-4.936446E-2,4.6477012E-2,-1.5822701E-2,7.083084E-2,9.563075E-3,-8.976092E-2,4.8905646E-4,-1.0384677E-1,-4.247589E-2,1.3308501E-1,-2.6712634E-3,-1.6859977E-2,-1.3859129E-1,-7.630779E-2,9.139906E-4,2.0981997E-2,7.079216E-2,-7.521709E-3,7.638667E-3,8.336396E-2,-1.6964907E-3,-4.3454796E-2,-8.879673E-3,-6.0306154E-2,-9.06809E-3,-1.346596E-2,-5.7981852E-2,2.0907472E-3,8.423091E-3,2.7398719E-5,-3.8388066E-3,-4.4804253E-2,-1.3787973E-2,-1.5046317E-3,-5.6492793E-3,5.1904593E-2,-3.101977E-2,8.074858E-2,-2.431322E-4,-1.4040481E-3,3.8905915E-2,1.3170636E-1,5.2274413E-2,-3.6403323E-3,1.3329152E-3,-4.2736847E-3,4.4880612E-4,3.5509577E-3,-2.8575424E-2,-8.383783E-2,-2.6361382E-2,-9.664212E-3,2.5248775E-2,1.3585996E-3,-4.5820232E-3,7.898426E-2,5.1168203E-3,9.873965E-4,-6.765529E-3,5.6036357E-3,5.527306E-2,3.8422006E-3,-2.226313E-4,9.6920095E-2,1.0159718E-2,3.888192E-2,6.386809E-3,-5.034832E-3,-1.1351237E-2,-5.5202052E-2,-6.970725E-3,1.6563621E-3,-6.29166E-2,-6.9045634E-3,-8.141564E-3,7.029964E-2,7.913334E-3,4.4405475E-2,6.5875757E-3,-4.0224628E-3,4.6553884E-2,1.0834249E-3,-3.0769308E-3,-1.620746E-3,7.15016E-2,5.501881E-3,1.3413954E-3,4.9138326E-2,-1.5955053E-3,1.1279745E-2,-2.8113066E-3,-3.6164382E-3,-2.0812389E-2,-1.0118706E-3,-4.020491E-3,1.1471406E-2,-2.001307E-2,4.5745578E-2,9.993264E-2,2.6743922E-3,4.2896364E-3,4.2437166E-3,9.256104E-5,3.5194184E-3,-1.9554637E-4,4.8606703E-3,4.9386084E-2,-1.0677816E-3,6.028534E-2,2.2145114E-3,-7.578226E-4,1.1115515E-3,-2.8743246E-3,7.0295564E-4,-4.546169E-3,-7.7078314E-3,-7.253777E-4,8.60132E-4,4.157949E-3,5.8099045E-3,1.5710271E-3,-8.8851004E-5,4.814243E-3,1.20445875E-5,3.7936168E-3,1.7642291E-3,4.20249E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,29,-1,31,-1,33,35,37,39,-1,41,-1,43,45,47,49,-1,51,53,-1,55,57,-1,59,61,-1,63,-1,65,-1,67,69,-1,-1,71,-1,73,-1,-1,-1,75,77,79,-1,-1,81,83,85,-1,-1,-1,-1,-1,87,89,91,93,95,-1,-1,97,99,101,-1,-1,103,-1,-1,105,-1,107,-1,-1,109,111,-1,-1,113,115,-1,117,119,121,-1,-1,123,-1,-1,-1,125,-1,-1,127,-1,129,-1,-1,131,-1,-1,133,135,137,139,141,-1,-1,-1,-1,-1,-1,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.052348E-1,5.478455E-1,1.7315972E-1,2.6781717E-1,3.6866283E-1,1.2580657E-1,8.363092E-2,2.5441167E-1,0E0,3.058567E-1,5.0412472E-2,1.0644382E-1,9.572089E-2,0E0,6.462185E-2,5.120641E-2,2.3463522E-1,0E0,2.4226606E-1,0E0,2.9455896E-2,7.290444E-2,1.2539126E-1,9.7418725E-2,0E0,7.229191E-2,0E0,5.6583375E-2,4.0153474E-2,4.7966853E-2,1.3369302E-1,0E0,1.9652194E-1,1.8136837E-2,0E0,9.809184E-2,4.6616286E-2,0E0,3.9233334E-2,8.780053E-2,0E0,3.549418E-2,0E0,3.0209124E-2,0E0,4.723507E-2,4.6027035E-2,0E0,0E0,1.4617217E-1,0E0,4.3269873E-2,0E0,0E0,0E0,4.7741756E-2,8.511515E-2,4.9072236E-2,0E0,0E0,3.523575E-2,5.232632E-2,4.0679403E-2,0E0,0E0,0E0,0E0,0E0,3.8925767E-2,5.310884E-2,6.593473E-2,1.9735524E-1,1.2908582E-1,0E0,0E0,4.645495E-2,6.1351433E-2,3.0409753E-2,0E0,0E0,5.382988E-2,0E0,0E0,2.5138259E-2,0E0,2.8804958E-2,0E0,0E0,2.869269E-2,1.7417833E-2,0E0,0E0,1.5892394E-2,1.0284511E-1,0E0,2.8898597E-2,5.3594798E-2,3.402076E-2,0E0,0E0,1.6872836E-2,0E0,0E0,0E0,1.7169341E-2,0E0,0E0,2.71656E-2,0E0,1.9051056E-2,0E0,0E0,2.1933664E-2,0E0,0E0,7.472952E-2,1.5502375E-1,3.0738868E-2,2.977541E-2,5.3883057E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.2134607E-2,0E0,1.5508845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,32,32,33,33,35,35,36,36,38,38,39,39,41,41,43,43,45,45,46,46,49,49,51,51,55,55,56,56,57,57,60,60,61,61,62,62,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,80,80,83,83,85,85,88,88,89,89,92,92,93,93,95,95,96,96,97,97,100,100,104,104,107,107,109,109,112,112,115,115,116,116,117,117,118,118,119,119,126,126,128,128],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,30,-1,32,-1,34,36,38,40,-1,42,-1,44,46,48,50,-1,52,54,-1,56,58,-1,60,62,-1,64,-1,66,-1,68,70,-1,-1,72,-1,74,-1,-1,-1,76,78,80,-1,-1,82,84,86,-1,-1,-1,-1,-1,88,90,92,94,96,-1,-1,98,100,102,-1,-1,104,-1,-1,106,-1,108,-1,-1,110,112,-1,-1,114,116,-1,118,120,122,-1,-1,124,-1,-1,-1,126,-1,-1,128,-1,130,-1,-1,132,-1,-1,134,136,138,140,142,-1,-1,-1,-1,-1,-1,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,9.932432E0,2.856934E7,1.6776881E7,9.4E1,7.650379E8,2.0263722E8,2.71E2,-1.0151071E-2,1E0,8.303E3,4.88E2,3.8771296E3,4.129138E-3,1.3655363E2,8.8324326E-1,2.04115E5,4.1708484E-4,3.2214062E2,2.2713726E-3,1.23E4,2.35184E5,5.11E2,2E1,9.563075E-3,1.7026364E3,4.8905646E-4,6.737099E7,3.8275862E0,1.766298E2,3.713753E7,-1.6859977E-2,5.7420593E2,8.620714E2,9.139906E-4,2.08E3,2.9E1,-7.521709E-3,2.6172416E10,1.4388715E0,-1.6964907E-3,9.3326636E-2,-8.879673E-3,2.66E2,-9.06809E-3,2.4E1,7.336111E4,2.0907472E-3,8.423091E-3,4.4935583E2,-3.8388066E-3,2.5144437E2,-1.3787973E-2,-1.5046317E-3,-5.6492793E-3,2.8827406E5,9.427676E2,4.029152E8,-2.431322E-4,-1.4040481E-3,3.0977E4,1.188E0,9.059046E6,-3.6403323E-3,1.3329152E-3,-4.2736847E-3,4.4880612E-4,3.5509577E-3,5.4E1,6.48334E5,3.466E3,2.483E3,2.8E0,1.3585996E-3,-4.5820232E-3,1.57922E5,3.2952412E5,1E0,-6.765529E-3,5.6036357E-3,8.484036E5,3.8422006E-3,-2.226313E-4,1.2376238E0,1.0159718E-2,3.116962E7,6.386809E-3,-5.034832E-3,8E0,2.2970297E0,-6.970725E-3,1.6563621E-3,4.8121494E2,1.5357143E0,-8.141564E-3,1.10566504E8,1.0358873E3,2.4434292E7,6.5875757E-3,-4.0224628E-3,1.83154E6,1.0834249E-3,-3.0769308E-3,-1.620746E-3,1.1621469E1,5.501881E-3,1.3413954E-3,4.6750406E4,-1.5955053E-3,2.855464E5,-2.8113066E-3,-3.6164382E-3,2.6412E5,-1.0118706E-3,-4.020491E-3,1.6942337E1,1.16408E5,6.083812E2,2.1609572E7,3.2585382E10,4.2896364E-3,4.2437166E-3,9.256104E-5,3.5194184E-3,-1.9554637E-4,4.8606703E-3,2.0619047E1,-1.0677816E-3,8.072373E1,2.2145114E-3,-7.578226E-4,1.1115515E-3,-2.8743246E-3,7.0295564E-4,-4.546169E-3,-7.7078314E-3,-7.253777E-4,8.60132E-4,4.157949E-3,5.8099045E-3,1.5710271E-3,-8.8851004E-5,4.814243E-3,1.20445875E-5,3.7936168E-3,1.7642291E-3,4.20249E-3],"split_indices":[2,69,60,62,0,5,7,2,0,8,44,0,67,0,71,68,5,0,70,0,10,1,0,3,0,4,0,7,69,4,60,0,67,4,0,44,8,0,46,69,0,53,0,1,0,2,60,0,0,70,0,48,0,0,0,48,4,7,0,0,44,68,43,0,0,0,0,0,2,46,1,2,69,0,0,1,48,6,0,0,43,0,0,69,0,62,0,0,32,68,0,0,48,68,0,5,4,12,0,0,9,0,0,0,71,0,0,48,0,60,0,0,7,0,0,71,7,4,60,46,0,0,0,0,0,0,71,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.045E3,7.87E2,2.58E2,7.38E2,4.9E1,2.28E2,3E1,7.33E2,5E0,2.8E1,2.1E1,1.54E2,7.4E1,4E0,2.6E1,1.06E2,6.27E2,6E0,2.2E1,7E0,1.4E1,1.19E2,3.5E1,6.9E1,5E0,1.6E1,1E1,1.6E1,9E1,1.2E1,6.15E2,9E0,1.3E1,1E1,4E0,5.9E1,6E1,4E0,3.1E1,6.2E1,7E0,1.2E1,4E0,1.2E1,4E0,3.2E1,5.8E1,5E0,7E0,5.96E2,1.9E1,9E0,4E0,6E0,4E0,3.7E1,2.2E1,5.3E1,7E0,1.4E1,1.7E1,2.3E1,3.9E1,8E0,4E0,8E0,4E0,4E0,2.8E1,3.1E1,2.7E1,4.31E2,1.65E2,4E0,5E0,2.3E1,1.4E1,1.8E1,4E0,1.9E1,3.4E1,8E0,9E0,1.8E1,5E0,3.5E1,4E0,4E0,2.4E1,2.3E1,8E0,1E1,1.7E1,4.25E2,6E0,4.5E1,1.2E2,1.6E1,7E0,4E0,1E1,1.4E1,4E0,5E0,2.9E1,1.3E1,5E0,3.1E1,4E0,1.7E1,7E0,1.3E1,1E1,7E0,1E1,1.77E2,2.48E2,2.6E1,1.9E1,1.14E2,6E0,7E0,9E0,6E0,4E0,1E1,1.9E1,4E0,2.7E1,7E0,1E1,5E0,5E0,1.72E2,5E0,6E0,2.42E2,1.7E1,9E0,1.3E1,6E0,1.1E2,4E0,8E0,1.1E1,1.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-2.1796166E-3,-1.5955565E-2,3.6903016E-2,-4.0135726E-2,-3.23034E-3,4.6634834E-2,-2.1685509E-2,-2.7762825E-2,-1.353065E-1,-1.2487384E-3,-9.219274E-3,3.0976562E-2,1.04161955E-1,-7.263966E-2,4.1796535E-2,1.394602E-3,-3.4222674E-2,1.6698814E-3,-2.2671686E-1,-4.4431672E-3,8.920827E-2,8.803175E-2,1.8302405E-2,9.845702E-3,8.400307E-2,-2.9315378E-3,-1.4208128E-1,5.1902547E-3,-1.3311481E-2,-5.879757E-2,-1.7828032E-2,-3.740043E-2,3.3107377E-3,-2.015563E-2,-8.557978E-2,-1.0228534E-2,5.048864E-2,1.3515306E-1,1.9705143E-4,4.662847E-3,-2.0242562E-4,3.177532E-2,-1.5183443E-2,5.7543978E-2,1.295571E-1,-2.3803602E-3,2.5062182E-3,-9.56011E-3,-1.0566904E-3,-3.3320778E-3,1.413662E-3,-4.298241E-2,-9.409306E-3,3.229396E-2,-2.9114697E-2,4.1061343E-4,-3.520769E-3,-1.0646119E-2,7.2326744E-4,-7.701555E-3,-6.67388E-3,8.639728E-2,7.388428E-3,2.4377177E-3,8.070683E-3,1.8092034E-2,9.4918504E-2,8.549523E-4,-3.324707E-2,4.28422E-3,-8.469955E-3,2.0201996E-3,7.243657E-3,-5.9466425E-2,-1.6953155E-3,-1.6933965E-3,5.7313085E-2,-3.6026187E-2,3.3629553E-3,-3.6745653E-3,-7.550389E-2,1.251894E-3,9.9867254E-2,1.8392664E-3,-1.8120952E-2,-4.08743E-3,2.4077382E-2,1.3038072E-1,1.3795211E-3,-3.1572036E-3,-1.1180756E-2,-2.708137E-3,1.3194776E-3,-4.7482144E-2,-6.2304595E-3,1.9251611E-2,-2.003988E-3,4.5912773E-3,1.2934699E-3,-5.9667155E-2,-9.664591E-3,-4.4623144E-2,3.0047023E-3,-8.255444E-4,-9.818465E-2,6.4058197E-3,4.7211863E-2,1.3401976E-3,-1.953801E-3,4.7398186E-3,1.7963966E-2,8.694866E-3,1.9029209E-3,9.350864E-4,-2.6294128E-3,1.8676422E-3,-5.6194555E-2,-1.5724283E-3,1.8410534E-3,-1.9769514E-2,-8.605686E-2,2.4962673E-4,-3.1029787E-2,-3.1112049E-2,-7.1276873E-3,7.8498935E-3,-3.1770267E-2,-5.567701E-3,-9.4650546E-4,-1.8824694E-4,3.7921108E-3,1.5570229E-3,4.896436E-2,-2.8982072E-3,4.0008436E-4,1.7682387E-3,-2.2826153E-3,-4.811528E-3,-9.714286E-4,9.5939456E-4,-3.1087901E-3,1.1182362E-3,-2.1502655E-3,-3.5281477E-3,4.648629E-4,-3.7354912E-4,-5.5599217E-3,-3.7694263E-4,4.5463024E-3,4.512656E-3,7.610985E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,-1,43,45,47,-1,49,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,73,-1,75,77,-1,-1,-1,-1,79,-1,81,83,-1,-1,85,87,-1,89,-1,91,-1,-1,93,95,-1,97,99,-1,101,103,-1,105,-1,107,-1,109,111,-1,-1,113,-1,-1,115,-1,117,-1,-1,-1,119,121,123,125,-1,127,-1,129,-1,-1,-1,131,-1,-1,-1,-1,-1,133,-1,-1,135,137,-1,139,141,-1,143,145,-1,-1,-1,-1,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5567956E-1,2.3464392E-1,1.5438774E-1,3.0593616E-1,1.9465898E-1,2.0485252E-1,1.2918834E-1,8.863005E-2,3.7601048E-1,1.4400873E-1,0E0,1.3072094E-1,8.971977E-2,1.0607766E-1,7.147612E-2,0E0,8.396293E-2,3.823042E-2,4.9504668E-1,1.5337855E-1,6.576924E-2,3.977576E-2,6.860643E-2,0E0,4.4373453E-2,3.613718E-2,8.030677E-2,0E0,3.061473E-2,1.8370345E-1,7.335113E-2,1.6844671E-2,0E0,0E0,1.7958742E-1,1.467779E-1,7.055449E-2,2.6616395E-2,0E0,0E0,0E0,9.159539E-2,2.739149E-2,6.8153255E-2,2.8207362E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.2303612E-2,0E0,4.3501407E-2,7.6577306E-2,0E0,0E0,0E0,0E0,1.1685641E-1,0E0,1.7262861E-2,1.8917616E-2,0E0,0E0,5.9535533E-2,3.9941788E-2,0E0,2.1692153E-2,0E0,2.2101728E-2,0E0,0E0,4.3291867E-2,2.091658E-2,0E0,1.9415442E-2,6.112148E-2,0E0,1.1117773E-1,2.9618219E-2,0E0,3.4303486E-2,0E0,1.603558E-2,0E0,3.9438218E-2,5.4136246E-2,0E0,0E0,2.8455315E-2,0E0,0E0,3.9800413E-2,0E0,1.8918943E-2,0E0,0E0,0E0,5.304885E-2,1.5620215E-2,7.965191E-2,5.9221376E-2,0E0,2.7026638E-2,0E0,1.8218156E-2,0E0,0E0,0E0,4.112212E-2,0E0,0E0,0E0,0E0,0E0,1.9625574E-2,0E0,0E0,3.973873E-2,3.4279972E-2,0E0,3.8961586E-2,4.551473E-2,0E0,5.815331E-2,8.9237005E-2,0E0,0E0,0E0,0E0,5.161716E-2,4.2707123E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,41,41,42,42,43,43,44,44,51,51,53,53,54,54,59,59,61,61,62,62,65,65,66,66,68,68,70,70,73,73,74,74,76,76,77,77,79,79,80,80,82,82,84,84,86,86,87,87,90,90,93,93,95,95,99,99,100,100,101,101,102,102,104,104,106,106,110,110,116,116,119,119,120,120,122,122,123,123,125,125,126,126,131,131,132,132],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,-1,44,46,48,-1,50,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,74,-1,76,78,-1,-1,-1,-1,80,-1,82,84,-1,-1,86,88,-1,90,-1,92,-1,-1,94,96,-1,98,100,-1,102,104,-1,106,-1,108,-1,110,112,-1,-1,114,-1,-1,116,-1,118,-1,-1,-1,120,122,124,126,-1,128,-1,130,-1,-1,-1,132,-1,-1,-1,-1,-1,134,-1,-1,136,138,-1,140,142,-1,144,146,-1,-1,-1,-1,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.167E3,1E0,2.7555334E7,3.7798166E0,8.432552E6,1.4176E4,3.416447E-2,4.2175066E-1,3.95E2,5.5303917E8,-9.219274E-3,3.9914E4,8.025477E0,1.480315E0,5.735198E-1,1.394602E-3,1.5222145E2,1.3E2,2.277451E2,5.817547E2,8.051603E7,9.033567E6,8E0,9.845702E-3,4.73909E0,8.969567E-9,1.3655363E2,5.1902547E-3,1.01E3,1.1244019E2,1.2E1,8E1,3.3107377E-3,-2.015563E-2,4.5E0,4.093403E6,4.612245E0,8.7568E4,1.9705143E-4,4.662847E-3,-2.0242562E-4,7.659164E1,3.9041237E1,1.4E1,4.29E2,-2.3803602E-3,2.5062182E-3,-9.56011E-3,-1.0566904E-3,-3.3320778E-3,1.413662E-3,3.3527696E-1,-9.409306E-3,3.7788504E7,4.4E1,4.1061343E-4,-3.520769E-3,-1.0646119E-2,7.2326744E-4,2.2968E4,-6.67388E-3,4.7773536E7,1.763E3,2.4377177E-3,8.070683E-3,8.118812E-1,1.3208092E0,8.549523E-4,2.4199E4,4.28422E-3,3.6349E4,2.0201996E-3,7.243657E-3,6.737509E7,4.08E2,-1.6933965E-3,4E0,7.5982756E5,3.3629553E-3,1.5933333E2,1.9067797E1,1.251894E-3,1.8729467E8,1.8392664E-3,6.232639E2,-4.08743E-3,2.3636363E0,4.2904656E8,1.3795211E-3,-3.1572036E-3,1.800554E-2,-2.708137E-3,1.3194776E-3,2.1126761E0,-6.2304595E-3,2.3857143E1,-2.003988E-3,4.5912773E-3,1.2934699E-3,3.7509E4,8.0274E4,1.4209677E2,1E0,-8.255444E-4,2.9308079E1,6.4058197E-3,1.935E3,1.3401976E-3,-1.953801E-3,4.7398186E-3,7.618E3,8.694866E-3,1.9029209E-3,9.350864E-4,-2.6294128E-3,1.8676422E-3,5.0406504E0,-1.5724283E-3,1.8410534E-3,1E0,1.8383686E5,2.4962673E-4,1.3358E4,3.757994E0,-7.1276873E-3,3.7974394E-6,1.2375429E8,-5.567701E-3,-9.4650546E-4,-1.8824694E-4,3.7921108E-3,2.8E1,6.03235E5,-2.8982072E-3,4.0008436E-4,1.7682387E-3,-2.2826153E-3,-4.811528E-3,-9.714286E-4,9.5939456E-4,-3.1087901E-3,1.1182362E-3,-2.1502655E-3,-3.5281477E-3,4.648629E-4,-3.7354912E-4,-5.5599217E-3,-3.7694263E-4,4.5463024E-3,4.512656E-3,7.610985E-4],"split_indices":[2,29,60,68,62,2,72,68,2,5,0,9,71,68,53,0,4,2,67,67,7,60,32,0,50,52,71,0,8,67,3,44,0,0,73,66,69,12,0,0,0,73,73,3,0,0,0,0,0,0,0,71,0,7,3,0,0,0,0,44,0,7,2,0,0,68,68,0,44,0,44,0,0,7,44,0,8,43,0,4,73,0,7,0,67,0,73,7,0,0,72,0,0,69,0,4,0,0,0,1,1,70,19,0,71,0,2,0,0,0,10,0,0,0,0,0,69,0,0,6,48,0,9,73,0,53,7,0,0,0,0,3,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,7.62E2,2.68E2,2.62E2,5E2,2.3E2,3.8E1,2.33E2,2.9E1,4.96E2,4E0,1.82E2,4.8E1,2.1E1,1.7E1,2.3E1,2.1E2,1.2E1,1.7E1,4.8E2,1.6E1,3.2E1,1.5E2,6E0,4.2E1,1.1E1,1E1,7E0,1E1,8.3E1,1.27E2,8E0,4E0,6E0,1.1E1,4.35E2,4.5E1,1E1,6E0,2.8E1,4E0,1.07E2,4.3E1,2.8E1,1.4E1,6E0,5E0,6E0,4E0,4E0,6E0,7.6E1,7E0,2.3E1,1.04E2,4E0,4E0,4E0,7E0,4.28E2,7E0,2.4E1,2.1E1,4E0,6E0,8.9E1,1.8E1,1.5E1,2.8E1,1.8E1,1E1,4E0,1E1,5.4E1,2.2E1,6E0,1.7E1,9.8E1,6E0,4.05E2,2.3E1,5E0,1.9E1,9E0,1.2E1,4E0,8.5E1,1.1E1,7E0,1E1,1.8E1,4E0,6E0,4.8E1,6E0,1.5E1,7E0,6E0,1.1E1,5.1E1,4.7E1,5.6E1,3.49E2,7E0,1.6E1,1E1,9E0,4E0,8E0,5E0,8E1,6E0,5E0,1.1E1,7E0,4E0,4.4E1,4E0,1.1E1,2.1E1,3E1,2.8E1,1.9E1,5.1E1,5E0,3.07E2,4.2E1,1.2E1,4E0,4E0,5E0,5.3E1,2.7E1,4E1,4E0,7E0,1.4E1,2.3E1,7E0,8E0,1.1E1,1.1E1,4E1,7E0,3E2,3.4E1,8E0,4.9E1,4E0,1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-4.394324E-3,-1.4657774E-2,2.9410226E-2,-4.259483E-2,-6.3885613E-3,3.9567206E-2,-2.9179737E-2,-2.3801986E-2,-9.635634E-2,-4.623298E-3,-8.238197E-3,2.7196303E-2,7.688229E-2,3.4070916E-2,-8.423665E-2,-4.785559E-3,-1.7770644E-2,-1.668953E-1,-4.314906E-2,-2.6686054E-2,7.5233844E-3,-4.5937024E-2,3.2708857E-2,1.1982903E-1,4.0392622E-2,4.835181E-3,-4.0303577E-2,-1.1631969E-1,2.1543658E-4,9.7151855E-3,-3.6102008E-2,-2.4053605E-1,-2.906722E-4,-7.035766E-2,2.0623659E-3,-1.682742E-2,-8.981411E-2,3.362829E-2,-2.304895E-3,-6.3896207E-3,1.2290553E-3,7.2906494E-2,1.9254409E-2,1.597043E-1,2.5875287E-3,3.2092799E-3,1.5024359E-3,-4.0444606E-3,6.928843E-4,-8.983178E-3,-3.0318138E-3,2.989017E-2,-2.617782E-3,-4.8118174E-2,3.9548497E-3,-4.6595233E-3,-1.4563231E-2,-6.685542E-3,-1.3469676E-2,-2.8876303E-2,1.8395469E-2,-9.334916E-3,-5.6627978E-2,-3.545487E-3,3.8810607E-2,-1.0196628E-2,5.080911E-2,4.8319746E-2,5.6435647E-3,2.8097006E-2,-3.703017E-3,3.3276656E-3,8.808678E-3,1.952945E-3,-3.3348324E-3,4.6422314E-2,-2.417001E-3,-5.433554E-2,1.2662942E-3,-2.1104075E-3,1.5051365E-3,1.993359E-3,-3.1490335E-3,-2.381475E-2,-6.217197E-3,7.877063E-2,-2.1794997E-2,-7.714783E-2,5.9857586E-5,5.0976045E-2,-4.1335197E-3,-5.2364026E-3,-1.1284767E-2,5.1628323E-3,3.2245267E-2,2.8856678E-3,1.4006702E-4,6.374135E-3,2.2018397E-2,5.0710775E-3,3.3900246E-2,-4.018954E-2,-8.749277E-2,-3.4528847E-3,-1.6585376E-2,4.3282816E-3,8.438426E-4,-3.298905E-3,-5.078036E-3,1.1687091E-4,-4.7057704E-3,7.995242E-2,1.9949615E-2,-5.5630016E-3,2.66841E-2,-2.8890943E-2,1.3977782E-2,8.5900427E-4,3.764203E-3,-1.6856841E-3,3.0916678E-2,1.0237182E-3,3.965133E-3,-3.1024155E-3,-7.5311546E-4,-5.815104E-3,-1.7638017E-3,-7.032845E-3,-3.0021719E-3,-1.7917902E-3,8.817341E-4,8.689986E-2,6.4403075E-4,-3.6498033E-3,4.0334754E-2,-2.3064863E-4,2.5729307E-3,-1.16037935E-1,-2.0676663E-2,-5.7259955E-2,2.5728125E-2,6.1875656E-2,5.8869147E-3,-6.147545E-5,-3.2808736E-3,4.558375E-3,1.8771517E-3,5.738897E-4,3.572522E-3,-7.2325376E-4,-8.926262E-3,-2.0561318E-4,-2.5929569E-3,-4.9445713E-3,4.587254E-4,2.0109846E-3,-2.6904224E-4,1.1480399E-3,5.290877E-3,-2.572983E-3,1.0134846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,53,55,-1,57,-1,59,61,63,65,-1,-1,67,69,71,-1,-1,73,-1,-1,-1,-1,75,-1,77,79,-1,-1,-1,81,83,85,-1,87,-1,89,91,93,95,-1,97,-1,-1,-1,-1,-1,99,-1,101,-1,-1,-1,-1,-1,103,-1,105,107,109,-1,111,113,115,-1,-1,117,-1,-1,-1,119,-1,121,123,125,-1,127,-1,-1,129,-1,-1,-1,131,133,-1,135,137,139,-1,-1,-1,141,-1,-1,-1,-1,-1,-1,143,-1,-1,-1,145,-1,-1,147,-1,-1,149,151,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4703547E-1,1.7691827E-1,1.3981211E-1,1.7451459E-1,1.791532E-1,8.977169E-2,1.2475352E-1,6.1196752E-2,1.61692E-1,1.5777847E-1,0E0,6.237842E-2,7.194832E-2,9.346831E-2,5.4733828E-2,0E0,6.232463E-2,2.2319901E-1,6.763412E-2,1.285612E-1,9.75727E-2,7.73851E-2,7.485424E-2,4.70424E-2,3.1554762E-2,0E0,2.4825234E-2,4.2687103E-2,0E0,6.865272E-2,3.5808697E-2,1.03833914E-1,0E0,8.687762E-2,0E0,7.777135E-2,9.285672E-2,6.2464677E-2,1.1662583E-1,0E0,0E0,3.6479726E-2,9.630768E-2,2.1558136E-2,0E0,0E0,4.2105917E-2,0E0,0E0,0E0,0E0,5.5432133E-2,0E0,2.8255582E-2,2.7062578E-2,0E0,0E0,0E0,4.3195836E-2,6.955447E-2,1.1576258E-1,0E0,2.7437381E-2,0E0,5.245413E-2,2.7968952E-1,3.7124515E-2,1.5822046E-2,0E0,6.308238E-2,0E0,0E0,0E0,0E0,0E0,2.245424E-2,0E0,2.1342948E-2,0E0,0E0,0E0,0E0,0E0,4.6988428E-2,0E0,1.5437402E-2,4.6904568E-2,3.3728853E-2,0E0,6.8529576E-2,8.563318E-2,1.0857547E-1,0E0,0E0,1.797624E-2,0E0,0E0,0E0,5.058517E-2,0E0,1.577929E-2,2.4025545E-2,2.3583218E-2,0E0,5.3172532E-2,0E0,0E0,2.059815E-2,0E0,0E0,0E0,1.7133206E-2,8.185482E-2,0E0,1.7736774E-2,7.419665E-2,1.1196615E-1,0E0,0E0,0E0,6.39521E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.5660185E-2,0E0,0E0,0E0,1.55928135E-2,0E0,0E0,3.3115365E-2,0E0,0E0,6.669875E-2,5.6694962E-2,6.459583E-2,6.428022E-2,6.927925E-2,4.7676142E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,46,46,51,51,53,53,54,54,58,58,59,59,60,60,62,62,64,64,65,65,66,66,67,67,69,69,75,75,77,77,83,83,85,85,86,86,87,87,89,89,90,90,91,91,94,94,98,98,100,100,101,101,102,102,104,104,107,107,111,111,112,112,114,114,115,115,116,116,120,120,127,127,131,131,134,134,137,137,138,138,139,139,140,140,141,141,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,54,56,-1,58,-1,60,62,64,66,-1,-1,68,70,72,-1,-1,74,-1,-1,-1,-1,76,-1,78,80,-1,-1,-1,82,84,86,-1,88,-1,90,92,94,96,-1,98,-1,-1,-1,-1,-1,100,-1,102,-1,-1,-1,-1,-1,104,-1,106,108,110,-1,112,114,116,-1,-1,118,-1,-1,-1,120,-1,122,124,126,-1,128,-1,-1,130,-1,-1,-1,132,134,-1,136,138,140,-1,-1,-1,142,-1,-1,-1,-1,-1,-1,144,-1,-1,-1,146,-1,-1,148,-1,-1,150,152,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,3.3832976E7,1E0,6E0,1.6776881E7,1.9851073E3,1.8645384E1,9E0,9.66E2,7.09E2,-8.238197E-3,7.549744E7,1.0284041E2,5.7E1,4.6E1,-4.785559E-3,3.373913E0,1.9874651E-5,4.83929E9,3.477E4,9.777778E0,2.0342858E0,1.684E0,1.137863E6,2.0666666E0,4.835181E-3,4.6779457E1,4.433E3,2.1543658E-4,2.4796897E3,2.7777777E0,2.7E1,-2.906722E-4,1.8154167E2,2.0623659E-3,5.51E2,1E1,6.2886596E-1,2.1512408E5,-6.3896207E-3,1.2290553E-3,4.455844E1,1.6553869E7,5.36763E5,2.5875287E-3,3.2092799E-3,8.72419E6,-4.0444606E-3,6.928843E-4,-8.983178E-3,-3.0318138E-3,8E0,-2.617782E-3,2.153533E6,4.36875E1,-4.6595233E-3,-1.4563231E-2,-6.685542E-3,4.9094446E2,1.0828667E5,2.1924414E2,-9.334916E-3,2.73E3,-3.545487E-3,1E0,2.0232125E5,5E0,1.0130841E1,5.6435647E-3,5.617715E6,-3.703017E-3,3.3276656E-3,8.808678E-3,1.952945E-3,-3.3348324E-3,2.6E1,-2.417001E-3,4.5E0,1.2662942E-3,-2.1104075E-3,1.5051365E-3,1.993359E-3,-3.1490335E-3,5.4E1,-6.217197E-3,6.69E2,1.7214285E1,3E0,5.9857586E-5,3.7936746E3,7.9E2,5.338167E5,-1.1284767E-2,5.1628323E-3,3.264E3,2.8856678E-3,1.4006702E-4,6.374135E-3,1.1724138E0,5.0710775E-3,3.1026666E2,1.8187732E4,4.353644E6,-3.4528847E-3,2.49E2,4.3282816E-3,8.438426E-4,1.7594937E0,-5.078036E-3,1.1687091E-4,-4.7057704E-3,1.8E1,2.53692E5,-5.5630016E-3,9.166947E6,5.4E1,1.2397727E1,8.5900427E-4,3.764203E-3,-1.6856841E-3,1.775373E1,1.0237182E-3,3.965133E-3,-3.1024155E-3,-7.5311546E-4,-5.815104E-3,-1.7638017E-3,1E0,-3.0021719E-3,-1.7917902E-3,8.817341E-4,1.2673605E7,6.4403075E-4,-3.6498033E-3,5.603014E6,-2.3064863E-4,2.5729307E-3,1.17E2,3.401282E1,3.270662E6,2E0,9.293198E2,1.4390917E8,-6.147545E-5,-3.2808736E-3,4.558375E-3,1.8771517E-3,5.738897E-4,3.572522E-3,-7.2325376E-4,-8.926262E-3,-2.0561318E-4,-2.5929569E-3,-4.9445713E-3,4.587254E-4,2.0109846E-3,-2.6904224E-4,1.1480399E-3,5.290877E-3,-2.572983E-3,1.0134846E-3],"split_indices":[2,7,6,8,62,67,71,70,12,2,0,7,73,6,6,0,69,57,5,1,73,68,69,9,68,0,73,10,0,48,68,3,0,67,0,2,3,68,48,0,0,73,66,1,0,0,58,0,0,0,0,32,0,43,70,0,0,0,4,48,67,0,10,0,30,48,3,71,0,12,0,0,0,0,0,4,0,73,0,0,0,0,0,2,0,2,73,8,0,48,2,43,0,0,10,0,0,0,68,0,67,43,60,0,10,0,0,71,0,0,0,3,12,0,60,0,73,0,0,0,71,0,0,0,0,0,0,19,0,0,0,60,0,0,9,0,0,10,73,12,32,4,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,7.66E2,2.32E2,1.74E2,5.92E2,1.98E2,3.4E1,1.3E2,4.4E1,5.87E2,5E0,1.5E2,4.8E1,1.6E1,1.8E1,8E0,1.22E2,1.8E1,2.6E1,2.08E2,3.79E2,1E1,1.4E2,2.1E1,2.7E1,8E0,8E0,1.3E1,5E0,4.9E1,7.3E1,1.2E1,6E0,2E1,6E0,1.81E2,2.7E1,1.03E2,2.76E2,4E0,6E0,3.4E1,1.06E2,1.2E1,9E0,1.5E1,1.2E1,4E0,4E0,4E0,9E0,3.8E1,1.1E1,5.6E1,1.7E1,5E0,7E0,8E0,1.2E1,1.35E2,4.6E1,5E0,2.2E1,4E0,9.9E1,2.41E2,3.5E1,2.4E1,1E1,9.8E1,8E0,4E0,8E0,8E0,4E0,3.2E1,6E0,5.2E1,4E0,6E0,1.1E1,6E0,6E0,1.3E2,5E0,1.8E1,2.8E1,1.6E1,6E0,7.7E1,2.2E1,2.37E2,4E0,7E0,2.8E1,1.8E1,6E0,4E0,9.4E1,4E0,2.8E1,3.8E1,1.4E1,1.5E1,1.15E2,1.4E1,4E0,2.4E1,4E0,4E0,1.2E1,3.9E1,3.8E1,4E0,1.8E1,1.06E2,1.31E2,2.3E1,5E0,1.2E1,8.2E1,2.4E1,4E0,1.7E1,2.1E1,7E0,7E0,9.7E1,1.8E1,9E0,1.5E1,3.5E1,4E0,6E0,3.2E1,9E0,9E0,8E0,9.8E1,1.8E1,1.13E2,3.6E1,4.6E1,9E1,7E0,2.7E1,8E0,1.9E1,1.3E1,4E0,4E0,6.8E1,3E1,1E1,8E0,7.2E1,4.1E1,2.2E1,1.4E1,9E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-3.5064819E-3,-1.7043624E-2,2.9958205E-2,-1.5313692E-2,-8.929629E-3,-8.798362E-3,3.3642247E-2,-4.0011212E-2,-2.3347524E-3,4.154353E-2,-2.1721793E-2,-2.9777834E-2,-1.2304945E-1,-7.003079E-4,-6.0048187E-3,3.3120025E-2,9.920983E-2,-4.1052856E-2,5.0914856E-3,-3.6685344E-2,2.6397053E-2,9.714439E-3,-2.299082E-1,-9.401194E-3,2.0300383E-2,5.3411988E-3,2.8317055E-2,6.0303295E-3,2.5415237E-3,3.367115E-3,-5.9281316E-2,-4.758875E-2,-1.1238481E-2,2.5520409E-3,-7.0842216E-4,-3.6390233E-3,2.669943E-3,-1.8471073E-2,-1.4443144E-3,-1.1500854E-2,4.9408553E-3,4.9678985E-2,8.2272326E-4,6.7663215E-2,1.9386671E-2,-2.7409794E-2,-1.270614E-1,-1.4830986E-2,-6.260465E-2,2.6524141E-2,-2.7662931E-2,-4.8351116E-3,-5.0283622E-2,6.231547E-2,-2.119309E-3,-1.5919986E-3,1.63569E-2,2.8714927E-2,1.0322834E-1,3.582367E-2,-7.643471E-3,-5.8292434E-2,9.3525334E-4,-2.2455424E-3,-8.320666E-3,-3.2136608E-2,3.8670804E-3,-1.07570544E-1,-5.0696123E-2,-4.5756376E-5,2.978936E-3,1.4405181E-3,-4.415626E-2,-5.1902514E-2,3.7245703E-4,-9.0679474E-2,-2.0690499E-2,6.923772E-2,5.0315005E-4,4.1536473E-2,-1.7199343E-2,2.7676343E-3,-1.04027036E-4,5.687251E-3,8.7700016E-4,-1.0285859E-2,5.2666947E-2,4.4412473E-3,-1.5758364E-2,-3.8619095E-3,-5.050631E-5,-3.2476212E-3,-7.955341E-3,-5.44112E-2,-7.4191424E-3,-2.3419643E-2,-7.66727E-2,-5.782504E-3,-2.918293E-2,-4.1498486E-3,-3.494274E-4,7.58996E-3,-3.7308473E-2,-7.7146096E-3,-5.4111432E-2,5.2095467E-4,-4.085223E-2,3.0023122E-2,3.7781356E-3,2.2920173E-2,4.1903337E-3,1.3029128E-3,-4.396736E-3,7.978218E-3,-3.9117597E-3,3.2966908E-2,9.231447E-2,5.1219616E-2,-2.9362127E-2,-1.3554444E-3,1.2498882E-3,-5.1819924E-3,5.734586E-4,-1.5715151E-3,2.5612765E-3,-4.2935116E-3,-9.4207666E-5,-1.8896698E-3,1.7369419E-3,7.4920733E-4,-1.1212428E-3,-4.456104E-3,-9.261613E-4,2.2897439E-4,-3.5130295E-3,-4.555337E-3,-7.9966645E-4,3.103407E-3,-1.6365559E-3,2.1641464E-3,-2.1283876E-4,2.514252E-3,-8.094858E-4,-1.3055975E-3,1.777273E-3,3.8738707E-3,7.031666E-4,2.6839764E-3,5.639245E-3,3.8731173E-3,-2.1650958E-4,-5.8842753E-4,-4.6003815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,33,35,37,39,41,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,-1,79,81,83,85,87,89,-1,-1,-1,91,-1,93,95,-1,-1,-1,97,99,101,103,105,107,-1,109,111,-1,-1,-1,-1,113,115,-1,117,-1,-1,-1,119,121,-1,123,125,-1,127,-1,-1,129,131,-1,133,-1,135,137,-1,139,-1,141,-1,143,-1,145,147,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.639462E-1,2.1930282E-1,2.4593905E-1,2.3166256E-1,0E0,0E0,1.2819946E-1,2.0792654E-1,9.9005535E-2,1.2058103E-1,9.888673E-2,8.749877E-2,3.8528195E-1,8.607142E-2,0E0,8.547725E-2,3.711492E-2,7.2857335E-2,0E0,5.4679185E-2,3.0938441E-2,5.9854865E-2,4.937685E-1,8.257921E-2,7.8926384E-2,0E0,7.3531374E-2,0E0,0E0,0E0,5.916466E-2,6.744385E-2,3.856778E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.455011E-2,0E0,6.92682E-2,4.660246E-2,5.105865E-2,7.802534E-2,3.208524E-2,2.869974E-2,7.9424985E-2,4.541698E-2,1.990654E-2,4.310949E-2,6.889432E-2,5.550488E-2,1.668863E-2,0E0,0E0,5.056537E-2,1.9453004E-2,3.1150386E-2,8.519499E-2,5.7937868E-2,1.8721722E-2,0E0,0E0,0E0,3.5427988E-2,0E0,4.4813946E-2,5.2762672E-2,0E0,0E0,0E0,3.758832E-2,4.6401568E-2,6.9361575E-2,4.6771973E-2,1.9151341E-2,1.869303E-2,0E0,2.9040456E-2,3.7632167E-2,0E0,0E0,0E0,0E0,4.1433286E-2,5.9441313E-2,0E0,5.890653E-2,0E0,0E0,0E0,1.950987E-2,4.384624E-2,0E0,3.447579E-2,4.7145486E-2,0E0,2.5017329E-2,0E0,0E0,5.9212293E-2,3.99247E-2,0E0,1.9806955E-2,0E0,2.276638E-2,3.1557165E-2,0E0,1.7338216E-2,0E0,2.2952452E-2,0E0,2.871148E-2,0E0,4.839037E-2,1.9376278E-2,2.0450532E-2,6.06014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,26,26,30,30,31,31,32,32,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,56,56,57,57,58,58,59,59,60,60,61,61,65,65,67,67,68,68,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,85,85,86,86,88,88,92,92,93,93,95,95,96,96,98,98,101,101,102,102,104,104,106,106,107,107,109,109,111,111,113,113,115,115,116,116,117,117,118,118],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,34,36,38,40,42,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,-1,80,82,84,86,88,90,-1,-1,-1,92,-1,94,96,-1,-1,-1,98,100,102,104,106,108,-1,110,112,-1,-1,-1,-1,114,116,-1,118,-1,-1,-1,120,122,-1,124,126,-1,128,-1,-1,130,132,-1,134,-1,136,138,-1,140,-1,142,-1,144,-1,146,148,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.56E3,2.483E3,2.7920364E2,1E0,-8.929629E-3,-8.798362E-3,2.5345264E7,3.7798166E0,4.4E1,1.7379E4,4.7263342E2,7.6937294E-1,4.327E3,3.6698947E2,-6.0048187E-3,1.0918E4,5.4123135E1,2.94857E5,5.0914856E-3,7.3131656E5,1.5E1,3.3857143E1,2.2596774E2,3.5E1,1E0,5.3411988E-3,3.222E3,6.0303295E-3,2.5415237E-3,3.367115E-3,1.4839433E5,6.0921145E4,5.51E2,2.5520409E-3,-7.0842216E-4,-3.6390233E-3,2.669943E-3,-1.8471073E-2,-1.4443144E-3,1.8E1,4.9408553E-3,1.59E2,1.14E3,6.232639E2,1.8645384E1,1.8987958E0,4.797448E8,2.135961E4,1E0,6E0,1.1015E4,8.234307E7,2.3266666E0,1.7172844E7,-2.119309E-3,-1.5919986E-3,1.688E3,4.1228E2,2.1199985E-7,7.653112E2,1.0092541E8,1.514526E6,9.3525334E-4,-2.2455424E-3,-8.320666E-3,5.62201E5,3.8670804E-3,1.3050649E4,6.76E2,-4.5756376E-5,2.978936E-3,1.4405181E-3,1.6716E5,2.026453E6,1E0,1.8149019E2,5.80563E5,2.2116136E5,5.0315005E-4,2.682E3,2.0766992E6,2.7676343E-3,-1.04027036E-4,5.687251E-3,8.7700016E-4,4.4522205E6,1.5E1,4.4412473E-3,2E0,-3.8619095E-3,-5.050631E-5,-3.2476212E-3,1.1E1,3.964269E3,-7.4191424E-3,1.5141701E1,5.7E1,-5.782504E-3,2.0130434E1,-4.1498486E-3,-3.494274E-4,3.5014236E2,1.2E1,-7.7146096E-3,1E0,5.2095467E-4,1.079E3,1.2E1,3.7781356E-3,1.5747747E1,4.1903337E-3,2.7662E4,-4.396736E-3,2.2E0,-3.9117597E-3,7.5052085E0,1.1772152E0,1.2707424E0,1.0975045E7,-1.3554444E-3,1.2498882E-3,-5.1819924E-3,5.734586E-4,-1.5715151E-3,2.5612765E-3,-4.2935116E-3,-9.4207666E-5,-1.8896698E-3,1.7369419E-3,7.4920733E-4,-1.1212428E-3,-4.456104E-3,-9.261613E-4,2.2897439E-4,-3.5130295E-3,-4.555337E-3,-7.9966645E-4,3.103407E-3,-1.6365559E-3,2.1641464E-3,-2.1283876E-4,2.514252E-3,-8.094858E-4,-1.3055975E-3,1.777273E-3,3.8738707E-3,7.031666E-4,2.6839764E-3,5.639245E-3,3.8731173E-3,-2.1650958E-4,-5.8842753E-4,-4.6003815E-3],"split_indices":[2,2,48,29,0,0,60,68,3,2,71,53,1,67,0,9,73,1,0,43,3,67,67,3,26,0,2,0,0,0,48,60,2,0,0,0,0,0,0,3,0,0,2,67,71,68,7,43,18,8,9,46,68,60,0,0,2,4,52,4,7,10,0,0,0,46,0,43,2,0,0,0,7,9,30,70,12,43,0,10,43,0,0,0,0,62,8,0,8,0,0,0,3,43,0,73,6,0,71,0,0,4,3,0,8,0,10,71,0,73,0,9,0,68,0,71,68,68,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,7.28E2,2.94E2,7.22E2,6E0,4E0,2.9E2,2.48E2,4.74E2,2.54E2,3.6E1,2.22E2,2.6E1,4.69E2,5E0,2.23E2,3.1E1,3.2E1,4E0,1.98E2,2.4E1,1.2E1,1.4E1,3.32E2,1.37E2,1.1E1,2.12E2,1.7E1,1.4E1,4E0,2.8E1,1.38E2,6E1,1.4E1,1E1,4E0,8E0,7E0,7E0,3.27E2,5E0,5.4E1,8.3E1,3.8E1,1.74E2,2E1,8E0,4.4E1,9.4E1,1.8E1,4.2E1,2.8E2,4.7E1,4.8E1,6E0,2.5E1,5.8E1,1.9E1,1.9E1,1.08E2,6.6E1,1.2E1,8E0,4E0,4E0,3.8E1,6E0,1.8E1,7.6E1,1.1E1,7E0,9E0,3.3E1,2.7E1,2.53E2,1.9E1,2.8E1,4.2E1,6E0,3.3E1,2.5E1,9E0,1E1,1.5E1,4E0,2.9E1,7.9E1,4E0,6.2E1,8E0,4E0,1.4E1,2.4E1,1E1,8E0,3.8E1,3.8E1,4E0,2.9E1,1.4E1,1.3E1,2.13E2,4E1,5E0,1.4E1,1.1E1,1.7E1,1.1E1,3.1E1,2.5E1,8E0,2.1E1,4E0,2.4E1,5E0,5.4E1,2.5E1,1E1,5.2E1,1.5E1,9E0,5E0,5E0,3.4E1,4E0,3.1E1,7E0,2.5E1,4E0,1.68E2,4.5E1,8E0,3.2E1,4E0,1E1,4E0,1.3E1,7E0,4E0,1.3E1,1.2E1,5E0,1.6E1,1.1E1,1.3E1,1.3E1,4.1E1,1.3E1,1.2E1,6E0,4E0,4.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-7.3198804E-5,-2.2783266E-2,1.5895527E-2,-1.5977498E-2,-7.454215E-2,-5.250131E-2,2.058859E-2,-1.09443525E-2,-6.320875E-2,-3.5612783E-3,-1.4833216E-1,-6.835955E-2,1.3748431E-3,7.889349E-3,1.8671151E-2,-1.6785571E-2,1.4599447E-2,9.362277E-4,-8.7782174E-2,-3.6571737E-2,3.799857E-2,-1.7136347E-2,-5.3527907E-2,-5.5754366E-3,-4.963083E-2,5.5908125E-2,1.0829558E-2,-1.8994963E-2,2.7791401E-3,2.3128772E-2,-3.3127256E-3,-6.019664E-3,-3.5958882E-2,9.1654784E-4,-3.084298E-3,3.637656E-3,-1.6070552E-3,-3.8415352E-3,2.2650431E-4,3.671403E-4,-2.7932285E-3,3.9474778E-2,1.01657175E-1,8.21374E-3,1.0845626E-1,-2.699716E-2,2.6730103E-3,4.0224258E-2,-5.378261E-3,-3.255703E-3,1.0072666E-3,4.0847105E-3,1.9338377E-2,1.0643075E-3,5.551289E-3,2.5339709E-3,5.7102513E-2,6.8270857E-3,2.0562187E-3,-1.9593183E-2,-7.7448964E-2,1.0233871E-2,-4.7914716E-3,1.7784748E-2,4.5752055E-3,-2.1229796E-2,1.9899188E-3,-3.2505472E-3,9.678808E-2,7.0530996E-3,-5.872616E-2,4.3609206E-2,7.471516E-3,-1.4337468E-2,-6.728788E-2,-6.964446E-3,-1.87423E-3,-4.020971E-2,2.0320127E-2,2.4518825E-3,2.578389E-3,5.384609E-4,-3.600521E-3,-4.117588E-2,4.4826217E-2,7.0764334E-3,6.87196E-4,5.2980117E-2,1.7797659E-3,-2.6662592E-2,-1.2345341E-1,5.576743E-2,-1.8785038E-3,-2.5240244E-2,1.54081015E-2,-1.2521219E-1,-8.718359E-5,-3.304146E-3,8.282728E-4,2.6197698E-2,-2.2359039E-3,1.3245983E-3,-2.0207013E-3,1.2220579E-3,-1.7750418E-3,-2.1131195E-3,-7.300003E-3,-1.9109527E-4,3.406367E-3,2.5541404E-2,1.05892815E-1,-1.014036E-1,6.1785076E-3,1.6401062E-3,-4.9241115E-3,-1.4431817E-3,-8.822992E-3,5.106643E-3,7.6217875E-2,-1.3711936E-3,2.223794E-3,-5.828585E-4,2.2298857E-3,-7.821379E-3,-2.591053E-3,1.6174281E-3,-5.652077E-4,-8.688377E-4,2.2355414E-3,-1.2461173E-3,2.4622704E-3,6.725709E-3,1.6320407E-3,3.4086595E-4,-1.3799031E-2,2.3074378E-3,-3.403599E-6,1.5976295E-3,-1.9899944E-3,4.2402097E-3,1.2835037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,31,33,35,-1,37,-1,39,41,43,45,-1,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,51,53,55,57,59,61,63,65,-1,-1,-1,67,-1,-1,69,71,-1,-1,73,75,77,-1,79,-1,81,-1,83,85,87,89,91,-1,93,95,-1,-1,97,99,101,-1,103,-1,105,107,-1,-1,109,111,113,115,117,-1,119,121,123,-1,-1,-1,125,-1,-1,-1,-1,-1,127,-1,-1,-1,129,131,133,135,-1,-1,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.724419E-1,1.4816678E-1,1.9463833E-1,8.8691615E-2,2.5611803E-1,5.2910015E-2,1.5950966E-1,5.112463E-2,7.562348E-2,3.705932E-2,4.8660952E-1,2.6348755E-2,0E0,0E0,1.6262098E-1,4.8400886E-2,4.8770323E-2,0E0,5.7007954E-2,2.6869562E-2,3.8321793E-2,0E0,3.156471E-2,0E0,1.6693085E-2,6.8912685E-2,1.1734719E-1,4.7061235E-2,0E0,2.8969493E-2,0E0,0E0,2.9445304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.023091E-2,3.3599466E-2,1.2536526E-1,2.3595676E-2,7.2444245E-2,6.058511E-2,4.7064945E-2,1.8578941E-2,0E0,0E0,0E0,9.3038395E-2,0E0,0E0,1.1281223E-1,5.929704E-2,0E0,0E0,4.2651787E-2,6.146738E-2,3.643215E-2,0E0,1.6188353E-2,0E0,2.226067E-2,0E0,7.8425504E-2,5.314546E-2,9.1695145E-2,5.4819234E-2,4.6324387E-2,0E0,5.1629774E-2,6.416471E-2,0E0,0E0,2.3125919E-2,2.509746E-2,2.6687378E-2,0E0,1.5150642E-2,0E0,1.0920067E-1,2.7196083E-2,0E0,0E0,5.3917736E-2,1.5539856E-1,1.0314853E-1,5.0075084E-2,3.898015E-2,0E0,3.9218865E-2,4.020536E-2,1.7932579E-2,0E0,0E0,0E0,1.9055326E-2,0E0,0E0,0E0,0E0,0E0,1.692801E-2,0E0,0E0,0E0,4.079175E-2,3.126581E-2,3.0265477E-1,8.975745E-2,0E0,0E0,0E0,0E0,1.8531075E-2,1.780349E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,19,19,20,20,22,22,24,24,25,25,26,26,27,27,29,29,32,32,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,52,52,55,55,56,56,59,59,60,60,61,61,63,63,65,65,67,67,68,68,69,69,70,70,71,71,73,73,74,74,77,77,78,78,79,79,81,81,83,83,84,84,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,99,99,105,105,109,109,110,110,111,111,112,112,117,117,118,118],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,32,34,36,-1,38,-1,40,42,44,46,-1,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,52,54,56,58,60,62,64,66,-1,-1,-1,68,-1,-1,70,72,-1,-1,74,76,78,-1,80,-1,82,-1,84,86,88,90,92,-1,94,96,-1,-1,98,100,102,-1,104,-1,106,108,-1,-1,110,112,114,116,118,-1,120,122,124,-1,-1,-1,126,-1,-1,-1,-1,-1,128,-1,-1,-1,130,132,134,136,-1,-1,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,3.5531914E0,1E0,2.8E1,8.62E2,1.1805571E-1,1.4E1,2.7224753E2,1.1E1,8.5714287E-1,1E0,3.421907E6,1.3748431E-3,7.889349E-3,1.6954315E0,1.3062992E1,5.0394736E2,9.362277E-4,1.084437E6,8.75477E5,6.763314E7,-1.7136347E-2,9.822E3,-5.5754366E-3,9.82E2,1.6E1,4.435876E2,8.93004E5,2.7791401E-3,1E0,-3.3127256E-3,-6.019664E-3,4.5172415E0,9.1654784E-4,-3.084298E-3,3.637656E-3,-1.6070552E-3,-3.8415352E-3,2.2650431E-4,3.671403E-4,-2.7932285E-3,3.887E3,2.5454E4,2.680152E5,2.4704508E7,4.28649E5,8.416667E0,3.541824E2,7.1E1,-3.255703E-3,1.0072666E-3,4.0847105E-3,1.9001543E1,1.0643075E-3,5.551289E-3,1.4127854E7,1.9E1,6.8270857E-3,2.0562187E-3,1E0,1.2E1,2.35E2,-4.7914716E-3,9.008784E7,4.5752055E-3,4.0119403E2,1.9899188E-3,3.659E3,2E0,1.4814814E0,1.7540792E5,1.4E1,7.471516E-3,5.5E1,2.0217392E0,-6.964446E-3,-1.87423E-3,7.773675E1,2.6043478E2,9.22E2,2.578389E-3,1.7638037E0,-3.600521E-3,2.62E3,1.7790684E5,7.0764334E-3,6.87196E-4,4.6604937E-1,1.027972E0,4.2619047E0,6.9756895E6,9.3411E4,-1.8785038E-3,3.1316226E0,1.6808511E0,1.2269693E4,-8.718359E-5,-3.304146E-3,8.282728E-4,1.0115696E0,-2.2359039E-3,1.3245983E-3,-2.0207013E-3,1.2220579E-3,-1.7750418E-3,4.080551E-1,-7.300003E-3,-1.9109527E-4,3.406367E-3,1.1E1,7E0,4.822581E0,6E0,1.6401062E-3,-4.9241115E-3,-1.4431817E-3,-8.822992E-3,4.8E2,1.4780379E7,-1.3711936E-3,2.223794E-3,-5.828585E-4,2.2298857E-3,-7.821379E-3,-2.591053E-3,1.6174281E-3,-5.652077E-4,-8.688377E-4,2.2355414E-3,-1.2461173E-3,2.4622704E-3,6.725709E-3,1.6320407E-3,3.4086595E-4,-1.3799031E-2,2.3074378E-3,-3.403599E-6,1.5976295E-3,-1.9899944E-3,4.2402097E-3,1.2835037E-3],"split_indices":[2,68,17,8,44,57,0,67,3,71,29,12,0,0,69,71,70,0,9,46,7,0,44,0,2,8,71,9,0,26,0,0,69,0,0,0,0,0,0,0,0,44,9,48,1,9,71,67,0,0,0,0,71,0,0,66,8,0,0,30,3,2,0,5,0,4,0,2,8,71,48,8,0,0,71,0,0,67,67,44,0,68,0,2,48,0,0,71,68,69,60,1,0,68,68,62,0,0,0,53,0,0,0,0,0,53,0,0,0,3,3,71,3,0,0,0,0,10,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,4.23E2,6.02E2,3.75E2,4.8E1,3.8E1,5.64E2,3.4E2,3.5E1,2.5E1,2.3E1,3.2E1,6E0,6E0,5.58E2,2.77E2,6.3E1,8E0,2.7E1,1.4E1,1.1E1,6E0,1.7E1,7E0,2.5E1,9.6E1,4.62E2,2.7E2,7E0,5.8E1,5E0,1.4E1,1.3E1,5E0,9E0,7E0,4E0,1.1E1,6E0,4E0,2.1E1,7.2E1,2.4E1,4.51E2,1.1E1,1.97E2,7.3E1,3.6E1,2.2E1,8E0,5E0,2E1,5.2E1,5E0,1.9E1,4.05E2,4.6E1,6E0,5E0,1.73E2,2.4E1,6.9E1,4E0,2.7E1,9E0,1.7E1,5E0,4.1E1,1.1E1,3.78E2,2.7E1,4.2E1,4E0,1.57E2,1.6E1,7E0,1.7E1,1.1E1,5.8E1,2E1,7E0,1.3E1,4E0,2.3E1,1.8E1,6E0,5E0,3.8E1,3.4E2,1.9E1,8E0,3.7E1,5E0,1.15E2,4.2E1,8E0,8E0,7E0,4E0,5.4E1,4E0,1.3E1,7E0,8E0,5E0,1.8E1,5E0,7E0,1.1E1,2.6E1,1.2E1,1.3E1,3.27E2,1.1E1,8E0,4E0,4E0,1.1E1,2.6E1,1.09E2,6E0,2.3E1,1.9E1,4E0,4E0,4.4E1,1E1,1.4E1,4E0,9E0,1.7E1,7E0,5E0,9E0,4E0,4E1,2.87E2,7E0,4E0,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[-5.0379704E-3,-2.6458643E-2,1.7478172E-2,-2.170328E-2,-1.7119959E-1,-6.794838E-2,2.3113383E-2,-2.4321122E-2,3.901759E-2,-9.5182745E-4,-2.5147036E-1,-4.120725E-2,-9.429642E-3,1.1287728E-2,5.8693375E-2,3.0864634E-2,-2.6794646E-2,3.650673E-3,-7.802085E-4,-1.5879592E-2,-5.4195095E-3,-2.7823346E-3,1.7815157E-3,1.353236E-2,-4.8417076E-3,1.2800871E-1,4.09381E-2,-9.640937E-4,2.3824265E-3,-1.214309E-1,-2.4114335E-2,2.5119218E-3,3.229817E-2,8.202278E-3,7.9946116E-2,1.2456085E-2,6.5346465E-2,-1.1591618E-3,-1.2499488E-2,-3.165019E-2,-5.995764E-3,9.0057375E-3,-1.3227522E-2,3.7086245E-2,-2.1506944E-3,5.17834E-3,8.505047E-4,-4.042029E-3,2.6778769E-2,8.8830836E-2,2.928456E-2,-2.4758635E-2,-6.505052E-2,1.4850303E-2,-2.3982411E-2,5.9431978E-2,2.3047836E-3,6.3128434E-2,1.946525E-2,6.9702476E-2,-9.84424E-3,1.0425684E-1,3.316943E-4,-2.2041834E-4,4.133811E-3,-2.7720671E-2,5.227569E-3,-8.053179E-2,3.5749776E-3,2.2346413E-2,-2.3308652E-3,5.5097076E-3,-3.8838666E-2,9.825463E-2,5.950998E-3,-3.4965403E-2,9.2094E-3,3.2930467E-2,8.763298E-2,4.5370262E-2,-1.9602356E-2,8.9224346E-2,-1.295253E-4,-3.2192608E-3,1.5374078E-2,6.2608216E-3,7.2297126E-2,-1.6361516E-2,-5.2813537E-2,-1.7768513E-2,-1.0871591E-1,-5.1212655E-3,1.9724157E-3,-8.45056E-4,2.4254594E-3,-3.6917245E-3,-1.1644814E-3,3.0869665E-4,6.0495976E-3,-2.4311212E-3,2.5551277E-3,-6.585039E-2,1.2863841E-2,6.7187324E-2,5.4816236E-3,7.04264E-2,-1.0434325E-2,6.5210555E-3,6.5620534E-2,2.6777036E-2,9.0508685E-2,-4.5372076E-2,1.4495606E-3,5.55659E-3,1.6888417E-3,-4.9947044E-3,3.0148623E-3,4.5743897E-3,7.261632E-4,-1.3532762E-3,1.0650757E-3,-1.0867582E-3,-3.4820798E-3,7.823806E-4,-3.8906117E-3,-1.1792188E-2,-3.2153344E-3,2.120461E-3,-1.4019826E-3,-4.938263E-3,1.8210933E-4,1.8892603E-3,-1.8554955E-3,5.717072E-3,-6.5250543E-4,-2.7998232E-3,5.6951516E-4,1.3544664E-3,5.243582E-3,1.0600457E-3,-2.2008317E-3,3.6425916E-3,3.4793493E-4,3.9226608E-4,3.5401762E-3,9.350862E-4,5.5203293E-3,-4.794298E-3,-6.972665E-4,2.1345895E-3,-1.5191894E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,-1,-1,37,39,41,43,-1,45,47,49,-1,-1,51,53,55,-1,57,-1,-1,-1,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,87,-1,89,-1,91,-1,93,95,97,99,101,103,105,107,109,111,113,-1,-1,115,-1,117,119,121,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,129,131,133,135,137,139,-1,141,143,145,147,-1,-1,-1,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.073713E-1,3.6727583E-1,2.4852696E-1,8.408375E-2,2.0032391E-1,1.08104065E-1,2.0214224E-1,6.93517E-2,5.053501E-2,0E0,9.582144E-2,4.4798974E-2,0E0,9.600747E-2,1.4244509E-1,2.4639979E-2,1.1907962E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.374912E-2,0E0,4.294312E-2,6.6620395E-2,0E0,0E0,1.7998101E-1,6.3846946E-2,4.2646283E-1,5.1562324E-2,0E0,2.5626764E-2,6.8192795E-2,4.143694E-2,0E0,0E0,7.457909E-2,5.2474316E-2,7.4877694E-2,0E0,5.613233E-2,0E0,0E0,0E0,0E0,6.5137975E-2,3.8297564E-2,4.552545E-2,1.1588268E-1,1.2971382E-1,3.294612E-2,3.3074666E-2,5.339838E-2,5.1017273E-2,3.411357E-2,7.780492E-2,2.7143799E-2,3.5778712E-2,1.6179293E-2,0E0,0E0,0E0,7.638003E-2,0E0,8.82836E-2,0E0,3.319358E-2,0E0,3.0039666E-2,2.5981337E-2,4.4030115E-2,3.7601948E-2,4.664046E-2,3.5725202E-2,3.9758928E-2,2.2545427E-2,3.6182806E-2,4.2148583E-2,2.0134136E-2,0E0,0E0,1.759925E-2,0E0,2.1118551E-2,9.6494034E-2,5.3727373E-2,4.062638E-2,1.8285868E-1,3.485449E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.484336E-2,2.0964954E-2,4.7738485E-2,7.222712E-2,1.9691184E-2,1.5995158E-2,0E0,1.5539132E-2,3.0343806E-2,2.4055652E-2,3.5570186E-2,0E0,0E0,0E0,2.008178E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,23,23,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,43,43,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,65,65,67,67,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,84,84,86,86,87,87,88,88,89,89,90,90,91,91,101,101,102,102,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,115,115],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,-1,-1,38,40,42,44,-1,46,48,50,-1,-1,52,54,56,-1,58,-1,-1,-1,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,88,-1,90,-1,92,-1,94,96,98,100,102,104,106,108,110,112,114,-1,-1,116,-1,118,120,122,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,130,132,134,136,138,140,-1,142,144,146,148,-1,-1,-1,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,6.0692043E0,3.964269E3,1.6521566E5,3.89E2,3.5E0,1.3141646E3,3.5714287E-1,4E0,-9.5182745E-4,6.6E1,1.24141544E-1,-9.429642E-3,1.853776E7,1.3587301E1,9.147E3,1.342376E0,3.650673E-3,-7.802085E-4,-1.5879592E-2,-5.4195095E-3,-2.7823346E-3,1.7815157E-3,1.8062708E6,-4.8417076E-3,2.5400869E5,1.6876252E0,-9.640937E-4,2.3824265E-3,1.0280637E0,1.559733E6,1.7587205E6,3.9935064E0,8.202278E-3,3.9813398E6,5.97E3,1.034073E6,-1.1591618E-3,-1.2499488E-2,4.064E3,1E0,2.3514E4,-1.3227522E-2,2.8045622E6,-2.1506944E-3,5.17834E-3,8.505047E-4,-4.042029E-3,6.6858566E-1,2.8827406E5,1.1863237E2,3.653E3,2.0130434E1,1E0,5E1,1.6869704E6,3.77E2,2.152E3,7.4010696E0,1.5859042E6,5.369854E5,5.932344E1,3.316943E-4,-2.2041834E-4,4.133811E-3,1.82407E5,5.227569E-3,1.480315E0,3.5749776E-3,6.83E2,-2.3308652E-3,5.195011E7,5.75E2,9.7573663E2,1.365243E7,1.4787234E1,1.0918E4,1.0092541E8,7.6825094E0,5.235714E2,1E0,5.801167E-1,-1.295253E-4,-3.2192608E-3,5.48352E5,6.2608216E-3,3.6666667E0,5.75E2,5.680271E-2,2.732439E2,1.8149019E2,1.2E1,1.9724157E-3,-8.45056E-4,2.4254594E-3,-3.6917245E-3,-1.1644814E-3,3.0869665E-4,6.0495976E-3,-2.4311212E-3,2.5551277E-3,1.7443357E8,2.3666666E0,1.6673345E7,7.9559E5,6.347708E7,4.3024124E2,6.5210555E-3,5.4567E5,2.01E0,1.0641107E8,3.921E3,1.4495606E-3,5.55659E-3,1.6888417E-3,9.408E3,3.0148623E-3,4.5743897E-3,7.261632E-4,-1.3532762E-3,1.0650757E-3,-1.0867582E-3,-3.4820798E-3,7.823806E-4,-3.8906117E-3,-1.1792188E-2,-3.2153344E-3,2.120461E-3,-1.4019826E-3,-4.938263E-3,1.8210933E-4,1.8892603E-3,-1.8554955E-3,5.717072E-3,-6.5250543E-4,-2.7998232E-3,5.6951516E-4,1.3544664E-3,5.243582E-3,1.0600457E-3,-2.2008317E-3,3.6425916E-3,3.4793493E-4,3.9226608E-4,3.5401762E-3,9.350862E-4,5.5203293E-3,-4.794298E-3,-6.972665E-4,2.1345895E-3,-1.5191894E-3],"split_indices":[2,68,43,48,2,68,67,68,8,0,10,53,0,62,71,44,69,0,0,0,0,0,0,43,0,48,53,0,0,69,9,43,68,0,43,2,9,0,0,44,26,1,0,43,0,0,0,0,57,48,73,44,71,19,0,60,10,2,71,43,43,73,0,0,0,9,0,68,0,2,0,7,2,48,5,71,9,7,71,4,28,57,0,0,44,0,69,2,53,67,70,3,0,0,0,0,0,0,0,0,0,7,68,47,12,7,4,0,1,68,5,2,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E3,5.38E2,5.12E2,5.22E2,1.6E1,3.1E1,4.81E2,5.01E2,2.1E1,6E0,1E1,2.7E1,4E0,3.62E2,1.19E2,2.1E1,4.8E2,1.2E1,9E0,5E0,5E0,2.2E1,5E0,3.56E2,6E0,2.3E1,9.6E1,6E0,1.5E1,1.2E1,4.68E2,2.25E2,1.31E2,1E1,1.3E1,4.5E1,5.1E1,8E0,4E0,3.3E2,1.38E2,2.21E2,4E0,1.24E2,7E0,8E0,5E0,5E0,4E1,3E1,2.1E1,2.75E2,5.5E1,6.4E1,7.4E1,2.5E1,1.96E2,4.9E1,7.5E1,1.8E1,2.2E1,2.5E1,5E0,1.4E1,7E0,2.7E2,5E0,5E1,5E0,5.8E1,6E0,2.5E1,4.9E1,1.4E1,1.1E1,3E1,1.66E2,2.3E1,2.6E1,4.5E1,3E1,1.4E1,4E0,6E0,1.6E1,1.1E1,1.4E1,1.87E2,8.3E1,1.6E1,3.4E1,2.5E1,3.3E1,1.7E1,8E0,1.1E1,3.8E1,4E0,1E1,5E0,6E0,1.8E1,1.2E1,9E0,1.57E2,1.2E1,1.1E1,6E0,2E1,3.3E1,1.2E1,2E1,1E1,8E0,6E0,1.2E1,4E0,9E0,5E0,1.41E2,4.6E1,3.7E1,4.6E1,1.1E1,5E0,6E0,2.8E1,8E0,1.7E1,1.1E1,7E0,8E0,4E0,5E0,4E0,1.4E1,1.43E2,7E0,5E0,6E0,5E0,1.6E1,4E0,2.5E1,8E0,4E0,8E0,6E0,1.4E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-6.0611346E-4,-2.3694906E-2,1.5831003E-2,-1.8948995E-2,-6.957293E-2,-5.7419207E-2,2.126269E-2,-1.0498687E-2,-4.1141726E-2,-2.0615581E-2,-2.0948768E-1,-3.4817286E-2,-9.849156E-3,1.8930182E-2,8.173714E-3,-4.462574E-2,-2.4489863E-3,-5.6143887E-2,1.47852795E-2,1.8199385E-3,-6.1517353E-3,-1.5598287E-2,-6.3831935E-4,-5.3048957E-2,1.7511126E-3,5.199987E-3,4.3018747E-2,-1.6522886E-2,-9.01693E-2,4.6123814E-2,-6.120625E-3,-5.1253475E-3,-8.202377E-2,-7.4939905E-3,3.0851678E-3,-1.9282785E-3,3.165261E-2,-2.6445054E-2,-3.4498938E-3,6.368992E-2,-2.6283106E-3,3.3134618E-3,5.5787474E-2,-3.9011112E-3,1.046837E-3,-5.0963473E-4,-5.047909E-3,2.9819328E-3,-3.3783424E-4,-4.590922E-3,-2.9065527E-3,-1.271066E-3,1.8554521E-3,-5.5347722E-2,-2.1221828E-1,3.139617E-2,-4.5551486E-2,-1.1024267E-3,5.2146208E-2,-2.4895126E-3,5.827955E-4,8.370852E-2,2.341045E-2,2.4086763E-3,-8.8695064E-2,-1.5617399E-2,3.2399737E-3,6.247584E-2,-4.3428443E-2,2.6933176E-3,-7.9541194E-4,-3.6056782E-3,-2.4883013E-4,3.695911E-2,-8.129441E-2,-2.8430296E-3,-1.480468E-2,3.1989787E-3,-5.8812666E-4,-3.7955975E-3,7.6137817E-6,3.7556288E-3,-2.2715343E-4,5.608295E-2,6.1319177E-3,-1.1956391E-3,2.3396944E-3,3.8379703E-2,-1.0034807E-2,-2.112811E-1,4.374598E-2,2.8258592E-3,-3.1431753E-2,3.160368E-2,7.7247284E-2,7.787996E-4,-3.992921E-3,-5.1907557E-3,4.4240944E-2,-6.8492896E-4,3.8153015E-3,-6.227962E-4,-9.9536784E-2,5.9769296E-5,3.754632E-3,2.757065E-2,5.2936114E-3,-7.1155108E-3,-4.5019267E-3,-1.4979242E-2,-2.590533E-3,3.8339172E-3,-1.960762E-4,-1.5056144E-2,-3.7293495E-3,7.981172E-2,8.733244E-3,-1.1164962E-3,8.3216265E-2,1.2045125E-3,-3.596816E-2,-8.078828E-4,2.8073252E-3,-8.479045E-3,-7.439039E-2,3.626753E-2,-2.4095168E-3,-8.627255E-3,-5.679496E-5,-3.1907673E-3,8.477124E-3,4.756977E-3,1.2897695E-3,-1.1739854E-3,2.9774604E-3,8.793451E-2,-6.30025E-4,-1.1709518E-3,3.593029E-4,8.3589846E-5,-3.1039051E-3,-2.1760524E-3,-5.29617E-3,5.036642E-4,3.1325482E-3,2.3443638E-3,-2.8171093E-4,1.148037E-3,-1.9558978E-3,1.1586294E-3,-9.785327E-4,4.343763E-3,3.077762E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,-1,-1,-1,37,-1,39,41,43,45,47,49,51,53,55,-1,-1,57,59,-1,61,63,65,67,-1,69,-1,-1,-1,-1,-1,71,-1,-1,73,75,77,79,-1,81,-1,-1,83,85,87,89,91,-1,93,95,-1,-1,-1,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,103,-1,-1,-1,105,107,109,111,-1,113,115,117,-1,-1,119,121,-1,-1,-1,123,-1,-1,125,-1,-1,127,-1,-1,-1,-1,129,-1,131,133,-1,135,137,139,-1,-1,-1,141,143,-1,-1,145,-1,147,-1,-1,149,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.833039E-1,9.0129495E-2,2.3596549E-1,7.12066E-2,2.6282564E-1,1.3692643E-1,1.9634533E-1,7.62608E-2,8.892782E-2,7.5990215E-2,2.4607664E-1,5.1066257E-2,0E0,1.7936848E-1,0E0,6.634506E-2,4.058755E-2,1.0916898E-1,2.7143696E-2,3.5107207E-2,0E0,0E0,0E0,1.5284985E-2,0E0,1.5905058E-1,9.969139E-2,4.0660903E-2,2.9017553E-2,1.6222056E-2,6.2951826E-2,3.0507263E-2,1.7927337E-1,2.65912E-2,0E0,0E0,1.9447623E-2,1.6757783E-2,0E0,3.1100214E-2,1.3330339E-1,6.307578E-2,1.0158855E-1,0E0,3.0281406E-2,0E0,0E0,0E0,0E0,0E0,4.0919363E-2,0E0,0E0,1.15572244E-1,1.2706017E-1,1.5912052E-2,1.507495E-2,0E0,2.0357706E-2,0E0,0E0,2.9119536E-2,2.1445882E-2,1.3063815E-1,2.8524488E-1,4.9280144E-2,0E0,6.140238E-2,2.7380034E-2,0E0,0E0,0E0,4.398281E-2,2.7213737E-2,4.443024E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6115246E-2,0E0,0E0,0E0,5.8617055E-2,1.7217854E-1,1.395979E-1,1.730082E-2,0E0,2.5421642E-2,5.1122315E-2,5.8913052E-2,0E0,0E0,3.547268E-2,2.1463603E-2,0E0,0E0,0E0,4.7600687E-2,0E0,0E0,4.8347026E-2,0E0,0E0,1.7010549E-1,0E0,0E0,0E0,0E0,3.3632E-2,0E0,1.514972E-2,1.8291134E-2,0E0,4.1870236E-2,2.6373472E-2,3.6680236E-2,0E0,0E0,0E0,2.095975E-2,4.8059255E-2,0E0,0E0,6.352902E-2,0E0,1.6859137E-2,0E0,0E0,1.5763158E-2,0E0,4.143244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,36,36,37,37,39,39,40,40,41,41,42,42,44,44,50,50,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,64,64,65,65,67,67,68,68,72,72,73,73,74,74,83,83,87,87,88,88,89,89,90,90,92,92,93,93,94,94,97,97,98,98,102,102,105,105,108,108,113,113,115,115,116,116,118,118,119,119,120,120,124,124,125,125,128,128,130,130,133,133,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,-1,-1,-1,38,-1,40,42,44,46,48,50,52,54,56,-1,-1,58,60,-1,62,64,66,68,-1,70,-1,-1,-1,-1,-1,72,-1,-1,74,76,78,80,-1,82,-1,-1,84,86,88,90,92,-1,94,96,-1,-1,-1,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,104,-1,-1,-1,106,108,110,112,-1,114,116,118,-1,-1,120,122,-1,-1,-1,124,-1,-1,126,-1,-1,128,-1,-1,-1,-1,130,-1,132,134,-1,136,138,140,-1,-1,-1,142,144,-1,-1,146,-1,148,-1,-1,150,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,3.5964913E0,1E0,2.56E2,2.0303884E6,3.5E0,1.0534078E4,4.9E2,2.7224753E2,6.5199727E-1,3.0536E4,1.24141544E-1,-9.849156E-3,1.7068776E10,8.173714E-3,5.853E3,1.2E1,1.1244019E2,1.9495455E6,4.586207E0,-6.1517353E-3,-1.5598287E-2,-6.3831935E-4,1.618692E6,1.7511126E-3,4.2E1,1.1806228E6,9E0,2E0,5.912179E3,1.43E2,1.4702564E11,2.8302418E10,1.946094E6,3.0851678E-3,-1.9282785E-3,7.8625E1,7.980211E5,-3.4498938E-3,8.295515E7,3.2462872E5,8.93E2,3.1E1,-3.9011112E-3,1.9363636E1,-5.0963473E-4,-5.047909E-3,2.9819328E-3,-3.3783424E-4,-4.590922E-3,5.0363636E1,-1.271066E-3,1.8554521E-3,1.7934177E2,1.4386049E6,7.547528E6,2E0,-1.1024267E-3,5.788E3,-2.4895126E-3,5.827955E-4,6.737099E7,2.8470706E2,2.379747E0,3.524172E5,1.9E1,3.2399737E-3,6.255748E6,2.67637E5,2.6933176E-3,-7.9541194E-4,-3.6056782E-3,4.3297894E9,3.6507E4,2.4203823E0,-2.8430296E-3,-1.480468E-2,3.1989787E-3,-5.8812666E-4,-3.7955975E-3,7.6137817E-6,3.7556288E-3,-2.2715343E-4,2.2116136E5,6.1319177E-3,-1.1956391E-3,2.3396944E-3,4.9833015E1,1.027972E0,6.27907E0,8.7692904E7,2.8258592E-3,3.7481603E8,1.5994397E0,1E0,7.787996E-4,-3.992921E-3,1E0,5.08E2,-6.8492896E-4,3.8153015E-3,-6.227962E-4,3.1706784E0,5.9769296E-5,3.754632E-3,2.19724E5,5.2936114E-3,-7.1155108E-3,1E0,-1.4979242E-2,-2.590533E-3,3.8339172E-3,-1.960762E-4,3.4779343E1,-3.7293495E-3,4.911E3,1.3388E4,-1.1164962E-3,1E0,2.590909E0,1.6808511E0,-8.078828E-4,2.8073252E-3,-8.479045E-3,1.6E1,1.915148E7,-2.4095168E-3,-8.627255E-3,1.7142857E0,-3.1907673E-3,2.2092846E7,4.756977E-3,1.2897695E-3,5.929839E8,2.9774604E-3,1.7655972E0,-6.30025E-4,-1.1709518E-3,3.593029E-4,8.3589846E-5,-3.1039051E-3,-2.1760524E-3,-5.29617E-3,5.036642E-4,3.1325482E-3,2.3443638E-3,-2.8171093E-4,1.148037E-3,-1.9558978E-3,1.1586294E-3,-9.785327E-4,4.343763E-3,3.077762E-4],"split_indices":[2,68,17,10,60,68,4,1,67,57,1,53,0,46,0,9,33,67,43,68,0,0,0,62,0,0,43,70,32,48,2,46,46,60,0,0,4,66,0,7,48,8,3,0,4,0,0,0,0,0,62,0,0,70,43,5,8,0,12,0,0,7,67,69,48,8,0,60,9,0,0,0,46,1,69,0,0,0,0,0,0,0,0,43,0,0,0,73,68,71,5,0,47,69,14,0,0,30,2,0,0,0,69,0,0,1,0,0,15,0,0,0,0,71,0,44,2,0,106,69,68,0,0,0,3,12,0,0,71,0,60,0,0,5,0,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,4.19E2,5.89E2,3.81E2,3.8E1,4E1,5.49E2,2.77E2,1.04E2,2.9E1,9E0,3.6E1,4E0,5.42E2,7E0,5.2E1,2.25E2,8.2E1,2.2E1,2.5E1,4E0,5E0,4E0,2.9E1,7E0,3.46E2,1.96E2,3.3E1,1.9E1,1.5E1,2.1E2,2.8E1,5.4E1,1.6E1,6E0,1E1,1.5E1,1.4E1,1.5E1,4E1,3.06E2,4.8E1,1.48E2,6E0,2.7E1,4E0,1.5E1,1.1E1,4E0,6E0,2.04E2,1.9E1,9E0,4.6E1,8E0,8E0,8E0,4E0,1.1E1,8E0,6E0,2.6E1,1.4E1,2.9E2,1.6E1,3.8E1,1E1,1.39E2,9E0,6E0,2.1E1,6E0,1.98E2,1E1,3.6E1,4E0,4E0,4E0,4E0,4E0,4E0,7E0,4E0,1.8E1,8E0,5E0,9E0,7.4E1,2.16E2,8E0,8E0,6E0,3.2E1,4.6E1,9.3E1,4E0,5E0,1.79E2,1.9E1,5E0,5E0,8E0,2.8E1,6E0,1.2E1,6.6E1,8E0,7E0,2.09E2,4E0,4E0,4E0,4E0,2.5E1,7E0,1.4E1,3.2E1,5E0,8.8E1,1.49E2,3E1,4E0,1.5E1,5E0,2.3E1,6E1,6E0,4E0,2.05E2,7E0,1.8E1,9E0,5E0,2.8E1,4E0,8.4E1,4E0,2.9E1,1.2E2,1.4E1,1.6E1,1.5E1,8E0,3.4E1,2.6E1,2.1E1,1.84E2,1.4E1,4E0,1.2E1,1.6E1,7.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[5.9554825E-4,-2.1876628E-2,1.7934639E-2,-1.5649695E-2,-8.1401184E-2,-6.374116E-3,3.300077E-2,-4.8421673E-2,-6.5315063E-3,-9.275872E-3,-4.74066E-2,-4.124915E-3,-4.7660996E-3,3.0714719E-2,7.466498E-3,1.7306872E-2,-5.5210866E-2,7.44051E-2,-1.2798433E-2,-2.7949918E-2,-7.1453936E-3,-8.725113E-3,5.8712777E-2,6.656207E-2,2.2821276E-2,-1.1501417E-3,2.5893515E-3,-8.200504E-2,-3.4653146E-2,1.3743595E-3,9.208301E-2,-1.5352372E-2,4.38133E-3,-4.439506E-2,2.9176925E-3,-5.7045175E-3,-3.083982E-3,-1.1341189E-4,3.7503797E-3,5.5552263E-2,7.4794665E-3,4.1974522E-2,1.0176706E-2,-4.349663E-3,-4.4510656E-4,2.5233554E-5,-5.1169906E-2,7.236533E-3,5.751145E-2,-7.110355E-3,-4.8197847E-2,-4.808201E-3,-1.6929764E-2,-1.8330231E-2,8.926301E-3,7.0742674E-2,1.1256013E-2,1.344279E-2,6.1497293E-2,1.8943131E-2,-6.7825414E-2,-6.1883503E-3,-3.0879207E-2,4.7989172E-4,4.1434774E-3,-1.0886417E-2,7.818445E-2,-9.160255E-2,-1.4865039E-2,2.2064399E-3,-1.7261893E-3,-2.8859276E-2,1.7495886E-3,1.5174884E-2,-2.2956925E-3,5.894271E-2,5.6828447E-3,-2.3596713E-3,1.6765526E-3,5.973904E-2,-9.411233E-3,1.3350567E-1,4.8250157E-2,-5.6302366E-3,2.3363248E-2,-1.2650138E-2,4.891592E-4,4.9221294E-4,-2.413337E-3,1.4825744E-3,-2.4918726E-2,5.2794244E-4,5.5801272E-3,-1.0495287E-1,-1.4707184E-3,2.0835115E-2,-1.8893981E-3,-1.6419897E-2,-1.0731031E-1,2.0827616E-2,-1.8675007E-3,6.684442E-2,-2.308612E-4,3.9988137E-3,2.8540035E-5,-2.1667664E-3,8.927682E-3,7.96291E-3,2.7801772E-3,5.6005865E-2,-1.3671643E-3,5.9227683E-2,9.508964E-3,-1.1720821E-2,3.2616075E-2,-4.370065E-3,-1.9740622E-2,-2.96228E-3,-6.4389375E-3,-3.0324302E-4,3.5548224E-3,-3.1264443E-2,9.441818E-4,-8.904431E-3,4.7323437E-4,2.8400443E-2,-2.19275E-2,3.49129E-3,1.322988E-4,4.7796655E-2,-1.1387231E-3,4.0509272E-2,4.937208E-3,-2.9512115E-2,8.0354266E-2,2.7870711E-2,-1.593654E-2,2.0722274E-4,-1.6170022E-3,2.2537686E-4,2.6415347E-3,-1.3797373E-3,1.1044816E-4,-2.77567E-4,-2.601295E-3,9.765567E-4,3.2912276E-3,-2.532683E-3,9.287375E-4,3.7208193E-3,-3.5119473E-4,7.6728576E-4,2.7136812E-3,-4.1455464E-3,1.6913696E-3,2.2544197E-3,5.7380674E-3,6.4408325E-4,3.4326196E-3,-1.1569837E-4,-5.0608525E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,45,-1,47,49,-1,51,-1,53,-1,-1,-1,55,-1,57,59,-1,-1,-1,61,-1,63,65,67,-1,69,71,73,75,77,79,81,83,85,-1,87,-1,-1,89,91,93,95,-1,-1,97,-1,99,-1,101,-1,-1,-1,103,105,107,109,-1,111,-1,-1,-1,-1,113,115,-1,-1,117,-1,119,-1,121,123,125,-1,127,-1,-1,-1,-1,129,-1,-1,131,-1,133,135,137,139,-1,141,-1,-1,-1,-1,143,-1,-1,-1,145,147,-1,-1,149,-1,151,-1,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9900133E-1,1.640384E-1,2.1189645E-1,1.2070628E-1,1.6092828E-1,4.7904048E-2,1.00001514E-1,4.022497E-2,1.6224943E-1,0E0,6.663172E-2,6.352863E-2,0E0,9.808254E-2,0E0,1.610988E-2,4.122442E-2,1.5417591E-2,8.282385E-2,4.7750674E-2,0E0,3.5376836E-2,2.0812105E-2,5.7131976E-2,6.980637E-2,0E0,0E0,2.8977945E-2,2.7352761E-2,0E0,2.6699856E-2,7.816404E-2,0E0,4.0855873E-2,0E0,3.6184903E-2,0E0,0E0,0E0,3.8230658E-2,0E0,6.3230455E-2,1.21633574E-1,0E0,0E0,0E0,4.9612872E-2,0E0,1.7263532E-2,7.589542E-2,8.255276E-2,0E0,2.6900316E-2,6.3225806E-2,3.4085285E-2,1.926884E-2,2.6820078E-2,5.134888E-2,5.8423817E-2,1.0180987E-1,2.9364616E-1,0E0,2.4508104E-2,0E0,0E0,3.891562E-2,2.7058162E-2,1.6607001E-2,3.2449704E-2,0E0,0E0,8.5091494E-2,0E0,2.6927112E-2,0E0,1.902768E-2,0E0,0E0,0E0,2.5313951E-2,2.310673E-2,1.8119186E-2,3.7277356E-2,0E0,7.645259E-2,0E0,0E0,0E0,0E0,4.971044E-2,3.601417E-2,0E0,0E0,1.9173503E-2,0E0,2.3565665E-2,0E0,4.331491E-2,1.2248604E-1,2.5229808E-2,0E0,1.8207744E-2,0E0,0E0,0E0,0E0,3.119448E-2,0E0,0E0,3.8233235E-2,0E0,8.298649E-2,5.324046E-2,3.2225855E-2,2.4246797E-2,0E0,2.2254046E-2,0E0,0E0,0E0,0E0,3.507976E-2,0E0,0E0,0E0,1.8716916E-2,1.7692465E-2,0E0,0E0,2.0564776E-2,0E0,1.775983E-2,0E0,3.912237E-2,4.236895E-2,4.164854E-2,6.0078975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,33,33,35,35,39,39,41,41,42,42,46,46,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,65,65,66,66,67,67,68,68,71,71,73,73,75,75,79,79,80,80,81,81,82,82,84,84,89,89,90,90,93,93,95,95,97,97,98,98,99,99,101,101,106,106,109,109,111,111,112,112,113,113,114,114,116,116,121,121,125,125,126,126,129,129,131,131,133,133,134,134,135,135,136,136],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,46,-1,48,50,-1,52,-1,54,-1,-1,-1,56,-1,58,60,-1,-1,-1,62,-1,64,66,68,-1,70,72,74,76,78,80,82,84,86,-1,88,-1,-1,90,92,94,96,-1,-1,98,-1,100,-1,102,-1,-1,-1,104,106,108,110,-1,112,-1,-1,-1,-1,114,116,-1,-1,118,-1,120,-1,122,124,126,-1,128,-1,-1,-1,-1,130,-1,-1,132,-1,134,136,138,140,-1,142,-1,-1,-1,-1,144,-1,-1,-1,146,148,-1,-1,150,-1,152,-1,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9236734E5,5.27439E0,1.194E3,2.08E2,3.964269E3,1.25E2,2.88755E5,1.863E3,1.1E1,-9.275872E-3,3.6440647E5,1.5512196E1,-4.7660996E-3,5E0,7.466498E-3,1.1E1,1.63757E4,3.806E3,4.435876E2,2.5393645E5,-7.1453936E-3,1.9E1,4.745397E6,6.439711E1,9.47E4,-1.1501417E-3,2.5893515E-3,5.619529E3,6.0921145E4,1.3743595E-3,4.1957852E2,3.9E2,4.38133E-3,1.317484E1,2.9176925E-3,2.5367088E2,-3.083982E-3,-1.1341189E-4,3.7503797E-3,4.643602E8,7.4794665E-3,6.5199727E-1,3.3386944E5,-4.349663E-3,-4.4510656E-4,2.5233554E-5,2.512195E0,7.236533E-3,9.7573663E2,5.202E3,4.5454545E0,-4.808201E-3,3.3817584E7,8.8474586E8,2E0,2.4545455E0,2.58466E5,2.16948E3,1.2877E4,1.8434782E2,6.27907E0,-6.1883503E-3,4E0,4.7989172E-4,4.1434774E-3,3.0751158E4,1.1481482E0,1.2152382E5,1.4E1,2.2064399E-3,-1.7261893E-3,2.8302418E10,1.7495886E-3,1.45752E8,-2.2956925E-3,2.0342858E0,5.6828447E-3,-2.3596713E-3,1.6765526E-3,1E0,1.8729467E8,1.8383686E5,3.1790426E8,-5.6302366E-3,1.3208092E0,-1.2650138E-2,4.891592E-4,4.9221294E-4,-2.413337E-3,5.117647E0,2.560356E5,5.2794244E-4,5.5801272E-3,2.1251968E1,-1.4707184E-3,6.364486E0,-1.8893981E-3,1.449E3,1.3254E4,3.9444446E2,-1.8675007E-3,1E0,-2.308612E-4,3.9988137E-3,2.8540035E-5,-2.1667664E-3,1.9495455E6,7.96291E-3,2.7801772E-3,9.356961E6,-1.3671643E-3,5.5933E5,8.998703E-1,4.0695653E0,1.2860047E5,-4.370065E-3,1.3631483E8,-2.96228E-3,-6.4389375E-3,-3.0324302E-4,3.5548224E-3,1.703125E0,9.441818E-4,-8.904431E-3,4.7323437E-4,1.978531E6,4.797342E2,3.49129E-3,1.322988E-4,5.94E2,-1.1387231E-3,2.0094643E0,4.937208E-3,2.2145481E5,2.7748251E1,2.6862775E6,3.7035952E6,2.0722274E-4,-1.6170022E-3,2.2537686E-4,2.6415347E-3,-1.3797373E-3,1.1044816E-4,-2.77567E-4,-2.601295E-3,9.765567E-4,3.2912276E-3,-2.532683E-3,9.287375E-4,3.7208193E-3,-3.5119473E-4,7.6728576E-4,2.7136812E-3,-4.1455464E-3,1.6913696E-3,2.2544197E-3,5.7380674E-3,6.4408325E-4,3.4326196E-3,-1.1569837E-4,-5.0608525E-3],"split_indices":[43,68,2,2,43,8,2,9,10,0,43,71,0,3,0,5,43,1,71,43,0,73,60,73,9,0,0,48,60,0,4,0,0,69,0,67,0,0,0,7,0,57,48,0,0,0,69,0,48,2,69,0,7,5,32,68,1,48,9,67,71,0,10,0,0,48,68,48,3,0,0,46,0,7,0,68,0,0,0,19,7,48,47,0,68,0,0,0,0,69,62,0,0,73,0,69,0,44,44,67,0,19,0,0,0,0,43,0,0,60,0,9,57,71,43,0,7,0,0,0,0,68,0,0,0,62,4,0,0,0,0,57,0,48,73,62,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,4.45E2,5.77E2,4.04E2,4.1E1,2.21E2,3.56E2,8.7E1,3.17E2,8E0,3.3E1,2.17E2,4E0,3.51E2,5E0,8E0,7.9E1,2.2E1,2.95E2,2.9E1,4E0,2.03E2,1.4E1,6.2E1,2.89E2,4E0,4E0,3.3E1,4.6E1,7E0,1.5E1,2.89E2,6E0,2.5E1,4E0,1.94E2,9E0,4E0,1E1,5.7E1,5E0,1.14E2,1.75E2,2.8E1,5E0,1.5E1,3.1E1,4E0,1.1E1,2.32E2,5.7E1,7E0,1.8E1,1.04E2,9E1,4.2E1,1.5E1,4.7E1,6.7E1,1.58E2,1.7E1,5E0,2.6E1,5E0,6E0,2.23E2,9E0,2.4E1,3.3E1,4E0,1.4E1,8.8E1,1.6E1,8.2E1,8E0,3.6E1,6E0,4E0,1.1E1,1.5E1,3.2E1,9E0,5.8E1,4E0,1.54E2,4E0,1.3E1,9E0,1.7E1,1.19E2,1.04E2,4E0,5E0,1.9E1,5E0,1.4E1,1.9E1,7.7E1,1.1E1,7.5E1,7E0,3.2E1,4E0,1E1,5E0,1E1,2.2E1,5E0,4E0,5.3E1,5E0,4.2E1,1.12E2,8.4E1,3.5E1,6E0,9.8E1,1E1,9E0,1E1,4E0,5.5E1,2.2E1,6E0,5E0,6.4E1,1.1E1,2.8E1,4E0,1E1,1.2E1,4.2E1,1.1E1,8E0,3.4E1,6.5E1,4.7E1,5E1,3.4E1,1.7E1,1.8E1,6.7E1,3.1E1,2.8E1,2.7E1,5.6E1,8E0,6E0,5E0,6E0,4E0,1.9E1,2.3E1,4E0,4E0,2.1E1,1.3E1,5.1E1,1.4E1,4.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[5.650882E-4,-6.382714E-3,4.63556E-2,-4.511776E-3,-1.3412294E-1,3.9423298E-2,7.0782066E-3,-1.4508722E-2,1.11286E-2,-5.4246124E-2,-1.1599501E-2,-4.332077E-3,4.4395857E-2,-1.2030394E-2,-1.2337878E-1,1.491511E-2,-8.759417E-2,-6.0621193E-3,1.5511045E-3,5.4352574E-2,-1.1092375E-2,5.0111458E-2,-1.4803842E-2,-1.0342491E-2,-2.085917E-3,-3.6253892E-3,2.6633024E-2,-1.04349945E-2,-1.0413991E-3,4.1141868E-2,1.173865E-1,2.0090997E-2,-2.3763739E-3,-2.0018022E-3,8.5639596E-2,-3.736683E-2,-8.735509E-3,-1.0657948E-2,4.758725E-2,7.7458583E-3,3.7162602E-2,-2.2213517E-3,2.1347508E-3,9.1278896E-2,3.0705065E-2,1.3925987E-3,7.3223677E-3,-4.681101E-4,2.97662E-3,5.3506605E-3,1.7191686E-3,-2.4529317E-2,-1.0019165E-1,-3.1257533E-3,-3.690509E-2,-3.4193262E-3,-4.987276E-3,3.6360817E-3,-1.6929218E-4,1.741027E-2,-6.506097E-2,4.6195902E-2,-4.227491E-2,2.1418915E-3,6.282564E-3,1.4381922E-2,3.4968413E-3,-3.7633654E-2,1.8281268E-2,-8.855379E-3,-5.0259758E-2,-6.129496E-3,7.898671E-2,-8.247789E-2,-1.7147567E-2,-1.028234E-2,3.904352E-3,8.705057E-3,4.8260647E-3,1.933137E-5,-5.4296907E-3,3.6354393E-2,9.309365E-2,-3.188885E-3,2.2618618E-4,-1.3390383E-3,2.2053603E-2,2.0223854E-3,-4.3230806E-2,1.3954345E-4,3.213783E-3,-3.52644E-3,3.105137E-4,-3.7718036E-3,-4.6047056E-3,6.1938735E-3,1.602496E-3,-6.2720566E-3,-2.571711E-3,-3.7010428E-2,9.81731E-4,2.1557739E-2,-2.5822636E-2,1.77644E-2,-3.0035512E-3,5.429795E-3,2.80181E-2,1.8132549E-3,5.5008247E-3,1.5818675E-3,-5.221393E-3,-2.0724467E-4,-2.56587E-3,-3.340695E-4,2.2396864E-3,-2.6626028E-3,-2.946884E-4,1.8733478E-4,2.2877564E-3,-4.240829E-3,-6.0451834E-4,2.439537E-3,1.6893951E-4,5.124633E-3,1.0620741E-3,-2.3530629E-3,1.6627699E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,39,-1,41,43,45,47,-1,-1,49,51,53,55,57,59,61,-1,-1,63,65,-1,-1,-1,-1,-1,-1,67,69,71,73,75,-1,-1,-1,77,79,81,83,-1,-1,85,-1,87,89,-1,91,93,95,97,99,101,-1,103,-1,-1,-1,105,107,-1,-1,-1,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,115,-1,117,119,121,-1,-1,123,-1,-1,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3084688E-1,2.1520646E-1,9.544331E-2,1.3947466E-1,1.07824564E-1,9.026338E-2,0E0,1.4529555E-1,1.3139401E-1,6.5242454E-2,0E0,0E0,7.0215434E-2,9.260291E-2,8.422525E-2,7.309455E-2,1.5535119E-1,0E0,0E0,8.427182E-2,2.6195515E-2,8.0066144E-2,6.963955E-2,0E0,0E0,4.7687605E-2,4.068017E-2,0E0,2.2225477E-2,4.42916E-2,5.9779823E-2,1.6733892E-2,0E0,0E0,2.0534642E-2,8.464593E-2,6.372653E-2,8.13267E-2,2.5805533E-2,5.3905368E-2,9.6673846E-2,0E0,0E0,2.3818687E-2,5.471883E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.173938E-2,7.259257E-2,8.385006E-2,5.891289E-2,6.664728E-2,0E0,0E0,0E0,4.9863756E-2,3.0968875E-2,5.153945E-2,1.8003406E-2,0E0,0E0,1.9602552E-2,0E0,3.378859E-2,1.7007353E-2,0E0,1.9825704E-2,7.1741335E-2,2.4631418E-2,2.1015555E-2,3.7395407E-2,5.102671E-2,0E0,4.256098E-2,0E0,0E0,0E0,6.437777E-2,2.3298353E-2,0E0,0E0,0E0,1.6465941E-2,0E0,3.1226076E-2,0E0,0E0,0E0,0E0,5.819079E-2,0E0,0E0,0E0,0E0,0E0,1.9847237E-2,0E0,1.6439136E-2,5.6781143E-2,2.748986E-2,0E0,0E0,3.580842E-2,0E0,0E0,0E0,3.1975724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,51,51,52,52,53,53,54,54,55,55,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,72,72,73,73,74,74,75,75,77,77,81,81,82,82,86,86,88,88,93,93,99,99,101,101,102,102,103,103,106,106,110,110],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,40,-1,42,44,46,48,-1,-1,50,52,54,56,58,60,62,-1,-1,64,66,-1,-1,-1,-1,-1,-1,68,70,72,74,76,-1,-1,-1,78,80,82,84,-1,-1,86,-1,88,90,-1,92,94,96,98,100,102,-1,104,-1,-1,-1,106,108,-1,-1,-1,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,116,-1,118,120,122,-1,-1,124,-1,-1,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,3.8304348E1,9.62385E3,9.860918E5,6.9756895E6,1.1770261E8,7.0782066E-3,6.425E1,1.9269184E7,2.0619047E1,-1.1599501E-2,-4.332077E-3,7.3274844E9,6E0,1.75E2,3.49E2,2.395631E7,-6.0621193E-3,1.5511045E-3,3.558194E6,2.948485E8,3.3527696E-1,3.3832976E7,-1.0342491E-2,-2.085917E-3,9.750085E6,1E0,-1.04349945E-2,1.9E1,2.01E2,1.0912606E5,4.435876E2,-2.3763739E-3,-2.0018022E-3,2.0649375E6,7E0,3.9E2,1.9E1,1.6108533E8,2.1391304E1,5.0741018E8,-2.2213517E-3,2.1347508E-3,6E0,5.1167645E6,1.3925987E-3,7.3223677E-3,-4.681101E-4,2.97662E-3,5.3506605E-3,1.7191686E-3,4.7659574E0,5.764631E1,1.23E4,8.638015E4,1.3E1,-4.987276E-3,3.6360817E-3,-1.6929218E-4,5.306E3,2.1557376E0,3.6451373E8,2.9251662E6,2.1418915E-3,6.282564E-3,3.532346E-1,3.4968413E-3,4.5054945E-1,5.667774E0,-8.855379E-3,2.7E1,3.53562E5,2.3266666E0,1.775373E1,2.364E3,1.0865825E0,3.904352E-3,1.0046621E7,4.8260647E-3,1.933137E-5,-5.4296907E-3,2.0115254E2,5.4204545E0,-3.188885E-3,2.2618618E-4,-1.3390383E-3,1.116241E2,2.0223854E-3,6.0921145E4,1.3954345E-4,3.213783E-3,-3.52644E-3,3.105137E-4,1.0564016E3,-4.6047056E-3,6.1938735E-3,1.602496E-3,-6.2720566E-3,-2.571711E-3,1.58E2,9.81731E-4,6.23E2,2.0217392E0,1.422162E3,-3.0035512E-3,5.429795E-3,1.1160929E0,1.8132549E-3,5.5008247E-3,1.5818675E-3,1.5477113E2,-2.0724467E-4,-2.56587E-3,-3.340695E-4,2.2396864E-3,-2.6626028E-3,-2.946884E-4,1.8733478E-4,2.2877564E-3,-4.240829E-3,-6.0451834E-4,2.439537E-3,1.6893951E-4,5.124633E-3,1.0620741E-3,-2.3530629E-3,1.6627699E-3],"split_indices":[2,68,67,43,60,7,0,61,62,71,0,0,5,3,6,10,62,0,0,1,47,71,7,0,0,9,29,0,3,0,48,71,0,0,62,8,0,8,7,71,7,0,0,8,43,0,0,0,0,0,0,69,62,10,43,8,0,0,0,2,69,7,43,0,0,53,0,68,68,0,3,1,68,71,2,71,0,62,0,0,0,67,71,0,0,0,71,0,60,0,0,0,0,4,0,0,0,0,0,8,0,2,71,48,0,0,69,0,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.038E3,9.02E2,1.36E2,8.9E2,1.2E1,1.29E2,7E0,5.43E2,3.47E2,8E0,4E0,4E0,1.25E2,5.32E2,1.1E1,3.35E2,1.2E1,4E0,4E0,1.06E2,1.9E1,2.2E1,5.1E2,4E0,7E0,1.3E2,2.05E2,4E0,8E0,8.9E1,1.7E1,1.1E1,8E0,6E0,1.6E1,1.07E2,4.03E2,1.15E2,1.5E1,7.4E1,1.31E2,4E0,4E0,1.4E1,7.5E1,6E0,1.1E1,7E0,4E0,9E0,7E0,9E1,1.7E1,3.37E2,6.6E1,1.08E2,7E0,9E0,6E0,6.6E1,8E0,1.18E2,1.3E1,8E0,6E0,5.6E1,1.9E1,6.9E1,2.1E1,5E0,1.2E1,3.26E2,1.1E1,1.9E1,4.7E1,1.01E2,7E0,6.1E1,5E0,4E0,4E0,9.9E1,1.9E1,8E0,5E0,8E0,4.8E1,4E0,6.5E1,1.7E1,4E0,8E0,4E0,3.19E2,7E0,4E0,7E0,5E0,1.4E1,3.1E1,1.6E1,3.3E1,6.8E1,5.5E1,6E0,8E0,9.1E1,7E0,1.2E1,3.3E1,1.5E1,1.6E1,4.9E1,3E2,1.9E1,1.8E1,1.3E1,2.1E1,1.2E1,1E1,5.8E1,1.5E1,4E1,4E0,8.7E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[1.4329132E-3,-3.3973034E-2,9.006472E-3,-2.6577396E-2,-1.0070072E-1,-5.4913033E-3,2.2044191E-2,-2.1321176E-2,-6.6375877E-3,-7.2451485E-3,-5.072872E-2,-8.932805E-3,5.2504815E-2,1.892582E-2,8.861062E-2,-4.7090568E-2,-9.810872E-4,1.3020491E-5,-3.6921296E-3,-1.193331E-2,5.3073812E-2,6.366194E-3,2.1156404E-2,2.0924678E-2,-5.8740634E-3,8.931953E-3,2.9918449E-2,-3.435887E-2,-1.1341266E-1,8.072683E-2,-2.2234092E-2,-9.192986E-2,-8.753858E-3,7.4712727E-3,-1.47254765E-2,2.4482051E-3,-1.1671192E-3,1.5946405E-2,6.4122155E-2,2.4216874E-3,-9.7320374E-4,-8.411323E-2,-1.7748777E-2,-9.182542E-3,-1.6746321E-3,5.583645E-3,2.1826164E-3,-2.9036723E-2,1.6939746E-3,1.4866677E-5,-8.511134E-3,-6.1108726E-3,-8.401018E-2,-2.6792507E-3,1.1302668E-3,-6.3615977E-3,1.8324176E-2,7.6316044E-2,-1.3391825E-5,-5.040966E-4,-5.0698896E-3,1.5586565E-3,-3.842713E-2,-1.1178046E-2,-5.4418586E-2,-6.673053E-2,-3.536216E-3,-7.7013066E-3,-1.0084688E-3,4.0267454E-3,1.546076E-2,2.6335793E-3,5.395029E-3,-4.196974E-3,-1.2800932E-2,-3.2464698E-2,3.552997E-2,-3.2223521E-3,-4.291627E-4,-4.294266E-3,-7.520036E-4,2.0293554E-2,-1.1433555E-2,3.2213912E-3,1.3270903E-2,1.3108187E-2,-5.2610193E-3,2.492809E-5,-3.026385E-3,-5.1812316E-5,3.100221E-3,3.5256784E-2,-2.2797983E-2,-6.3956957E-3,-7.7431314E-3,3.137669E-2,4.9798666E-3,-1.7102346E-3,1.9841224E-3,-2.089834E-3,1.9213211E-3,-2.2421593E-3,3.395017E-4,-3.8993752E-3,-2.3666656E-4,2.5502827E-6,2.2970787E-3,-2.6004745E-3,4.136652E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,21,23,25,27,29,-1,-1,31,33,-1,35,37,-1,-1,39,41,43,45,47,49,51,-1,53,-1,-1,55,57,-1,-1,59,61,-1,-1,-1,-1,63,-1,-1,-1,65,67,-1,-1,-1,69,71,-1,-1,-1,-1,73,75,77,79,81,-1,-1,-1,83,-1,-1,-1,85,87,89,-1,-1,-1,-1,91,93,-1,95,97,-1,-1,-1,-1,-1,99,101,-1,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7998543E-1,8.812645E-2,1.6267644E-1,9.972252E-2,4.252772E-2,8.190326E-2,9.235619E-2,8.4342234E-2,0E0,0E0,1.6401028E-2,7.2449096E-2,5.8583103E-2,1.293792E-1,1.1714721E-1,5.6055173E-2,1.5981992E-1,0E0,0E0,9.304864E-2,1.3497083E-1,0E0,2.7939316E-2,9.123343E-2,0E0,0E0,1.7164223E-2,4.8918992E-2,6.5239236E-2,1.982233E-2,3.0314434E-2,1.1926381E-1,7.029548E-2,0E0,2.1767298E-2,0E0,0E0,1.4354266E-1,3.45812E-2,0E0,0E0,2.694162E-2,5.13354E-2,0E0,0E0,0E0,0E0,2.9308613E-2,0E0,0E0,0E0,5.3609833E-2,5.806119E-2,0E0,0E0,0E0,7.3750466E-2,2.225335E-2,0E0,0E0,0E0,0E0,4.4508692E-2,4.0923756E-2,1.8032372E-2,1.7007913E-2,6.27117E-2,0E0,0E0,0E0,4.2688422E-2,0E0,0E0,0E0,6.434387E-2,3.0570865E-2,1.5120134E-2,0E0,0E0,0E0,0E0,5.4321237E-2,1.1610294E-1,0E0,5.298472E-2,3.1504724E-2,0E0,0E0,0E0,0E0,0E0,3.3476762E-2,1.7424518E-2,0E0,4.858224E-2,6.377583E-2,5.9914872E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,37,37,38,38,41,41,42,42,47,47,51,51,52,52,56,56,57,57,62,62,63,63,64,64,65,65,66,66,70,70,74,74,75,75,76,76,81,81,82,82,84,84,85,85,91,91,92,92,94,94,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,22,24,26,28,30,-1,-1,32,34,-1,36,38,-1,-1,40,42,44,46,48,50,52,-1,54,-1,-1,56,58,-1,-1,60,62,-1,-1,-1,-1,64,-1,-1,-1,66,68,-1,-1,-1,70,72,-1,-1,-1,-1,74,76,78,80,82,-1,-1,-1,84,-1,-1,-1,86,88,90,-1,-1,-1,-1,92,94,-1,96,98,-1,-1,-1,-1,-1,100,102,-1,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.147158E4,9.51E3,3.4734247E2,3.8304348E1,3.3025316E5,1.559287E5,4.435876E2,2.71E2,-6.6375877E-3,-7.2451485E-3,4.3E1,2.2503355E1,8.691756E-1,7.134432E7,4.8647133E-1,1.3E2,2.04115E5,1.3020491E-5,-3.6921296E-3,1.16408E5,1.68574E5,6.366194E-3,5.323779E6,5.1167645E6,-5.8740634E-3,8.931953E-3,2.7021693E8,7.785714E0,1.1626786E2,3.8275862E0,1.0652307E7,6.23E2,2.2051188E6,7.4712727E-3,4.232328E3,2.4482051E-3,-1.1671192E-3,1.0280637E0,1.213274E9,2.4216874E-3,-9.7320374E-4,1.11E2,1.1E1,-9.182542E-3,-1.6746321E-3,5.583645E-3,2.1826164E-3,4.217427E7,1.6939746E-3,1.4866677E-5,-8.511134E-3,1.88E2,6.763314E7,-2.6792507E-3,1.1302668E-3,-6.3615977E-3,3.63E2,1.400229E6,-1.3391825E-5,-5.040966E-4,-5.0698896E-3,1.5586565E-3,1.8857143E0,1E0,1.3141646E3,4.3827028E5,5.3061223E0,-7.7013066E-3,-1.0084688E-3,4.0267454E-3,8.118812E-1,2.6335793E-3,5.395029E-3,-4.196974E-3,3.5318289E3,1.26375E1,5.831829E6,-3.2223521E-3,-4.291627E-4,-4.294266E-3,-7.520036E-4,2.277451E2,5.475771E0,3.2213912E-3,5.6516E4,3.6E1,-5.2610193E-3,2.492809E-5,-3.026385E-3,-5.1812316E-5,3.100221E-3,8.612209E-6,3.8575E4,-6.3956957E-3,4.9558692E7,1E0,1.4598765E0,-1.7102346E-3,1.9841224E-3,-2.089834E-3,1.9213211E-3,-2.2421593E-3,3.395017E-4,-3.8993752E-3,-2.3666656E-4,2.5502827E-6,2.2970787E-3,-2.6004745E-3,4.136652E-4],"split_indices":[43,44,67,68,62,48,71,2,0,0,8,71,68,59,57,2,5,0,0,7,9,0,12,43,0,0,47,67,4,69,66,2,66,0,48,0,0,69,7,0,0,1,3,0,0,0,0,60,0,0,0,2,7,0,0,0,44,1,0,0,0,0,68,27,67,43,73,0,0,0,68,0,0,0,62,73,60,0,0,0,0,67,73,0,9,2,0,0,0,0,0,53,9,0,46,29,69,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,1.83E2,8.59E2,1.66E2,1.7E1,4.07E2,4.52E2,1.6E2,6E0,7E0,1E1,3.85E2,2.2E1,4.33E2,1.9E1,7E1,9E1,4E0,6E0,3.68E2,1.7E1,5E0,1.7E1,4.28E2,5E0,6E0,1.3E1,6E1,1E1,1.8E1,7.2E1,1.3E1,3.55E2,6E0,1.1E1,1E1,7E0,3.85E2,4.3E1,9E0,4E0,1.4E1,4.6E1,4E0,6E0,7E0,1.1E1,6.5E1,7E0,7E0,6E0,3.44E2,1.1E1,5E0,6E0,5E0,3.8E2,3.6E1,7E0,4E0,1E1,1.3E1,3.3E1,3.9E1,2.6E1,1.3E1,3.31E2,4E0,7E0,1.4E1,3.66E2,2.6E1,1E1,1E1,2.3E1,2.7E1,1.2E1,1.9E1,7E0,8E0,5E0,8.2E1,2.49E2,1.3E1,3.53E2,1.9E1,4E0,1.4E1,1.3E1,6E0,6E0,6.1E1,2.1E1,6E0,2.43E2,1.1E2,2.43E2,7E0,1.2E1,4E0,5.7E1,1.1E1,1E1,7E0,2.36E2,4.1E1,6.9E1,1.4E1,2.29E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[1.1671787E-3,-1.6318643E-2,1.8855238E-2,-1.17567815E-2,-8.3146155E-2,-5.6021255E-2,2.3787173E-2,-3.5236605E-2,-5.9766294E-3,-1.2162982E-1,1.7529247E-2,-7.91766E-3,-3.3420462E-2,1.1393533E-1,2.0647649E-2,8.585303E-4,-4.0340494E-2,7.558665E-3,-2.1797104E-2,-1.9651461E-2,-2.0640755E-1,-1.4655216E-3,3.378295E-3,8.368036E-4,-2.2313385E-3,2.0753755E-3,6.153532E-3,-8.023789E-4,3.193607E-2,-5.0421815E-2,-1.2918692E-2,1.0862727E-2,-3.6059632E-3,-2.4322916E-2,3.3252272E-3,-2.1773067E-3,1.3047814E-3,-1.3436436E-2,-2.7457245E-3,-8.843282E-3,6.401614E-2,4.073136E-2,5.2496777E-3,-9.2098214E-2,-4.2823434E-2,-3.801867E-2,1.6598296E-3,1.6413512E-2,-4.13027E-2,1.0997318E-2,-3.1897742E-2,-1.3973577E-3,-5.8147974E-2,9.946798E-2,-1.0840854E-3,2.2004254E-2,6.711193E-2,2.9541608E-2,-8.2224295E-2,-6.1208676E-3,-1.5378622E-3,-2.4805384E-2,-6.9542E-2,2.1254164E-3,-3.1143334E-3,-2.6794795E-2,2.237313E-2,3.6854463E-4,-4.22169E-3,3.7591243E-3,-1.9616357E-3,-5.253208E-2,-5.3759953E-3,-3.91986E-2,6.461366E-3,-5.898438E-3,-2.4197485E-2,6.4986926E-3,2.0354849E-3,2.6862154E-2,-4.052917E-3,2.3195792E-2,7.851363E-2,7.5571276E-2,3.486751E-3,-1.1895893E-2,-8.488627E-3,-3.164914E-3,-5.282204E-4,-6.216524E-3,-2.1999117E-3,-8.763408E-6,-3.5414186E-3,3.0253574E-2,-1.69838E-2,-1.9466795E-2,2.043597E-3,-6.993608E-2,-2.1643689E-2,-1.2486806E-2,3.809162E-3,-6.079649E-2,2.1433367E-3,-6.6817887E-3,3.449045E-2,3.1877044E-4,-3.7676631E-3,-2.6951255E-3,3.0706078E-2,2.968216E-3,-8.8688095E-5,5.6172274E-3,6.477177E-2,9.613334E-2,-6.267547E-4,2.070991E-2,-3.141287E-3,-2.6898896E-3,2.7368255E-3,3.3460625E-3,2.524807E-2,-4.9086116E-2,3.7091244E-2,2.0651363E-3,-3.719452E-2,-3.9838914E-2,-1.09166816E-1,-2.0034993E-3,6.615832E-5,-1.4908453E-3,-4.9295325E-2,-2.780346E-2,-5.6431512E-3,3.7475287E-3,-3.3615E-3,-2.1279707E-4,4.874945E-2,1.934768E-2,6.0871515E-2,7.938859E-2,5.598643E-4,2.6723088E-3,6.3184304E-3,-2.9879278E-3,3.471733E-2,-7.803605E-4,1.5939849E-3,-4.884244E-4,-3.6423355E-3,-5.622856E-4,4.5570764E-3,-4.549525E-4,-3.9587575E-3,5.481957E-4,-2.3209727E-3,-6.2672775E-3,-1.5254904E-3,8.0606254E-4,-1.446732E-3,-9.0561016E-4,-4.565492E-3,-2.7453857E-3,8.774453E-4,9.408216E-4,-1.5628253E-3,3.05456E-3,-2.4745456E-4,2.711664E-3,5.094988E-4,1.1849916E-3,4.369715E-3,4.298175E-3,9.6806703E-4,6.52018E-5,2.6862174E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,49,-1,-1,-1,-1,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,-1,79,81,83,85,-1,-1,87,89,-1,-1,91,93,-1,-1,-1,95,97,99,101,103,-1,105,-1,-1,107,-1,109,111,113,115,-1,117,-1,-1,-1,-1,-1,-1,119,121,123,-1,125,127,129,-1,131,-1,133,135,-1,-1,-1,137,-1,-1,-1,139,141,-1,143,-1,-1,-1,-1,145,147,149,-1,151,153,155,-1,-1,157,159,161,-1,163,-1,-1,165,167,169,171,-1,-1,-1,-1,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1918597E-1,1.5724371E-1,1.907763E-1,6.5883555E-2,1.299859E-1,7.770377E-2,1.3406426E-1,2.7120002E-2,8.410559E-2,2.034356E-1,2.9701091E-2,0E0,2.23427E-2,1.528433E-2,1.131036E-1,0E0,2.3676038E-2,6.0815345E-2,4.4621155E-2,1.7123068E-2,1.4403385E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.500042E-2,7.160881E-2,1.632312E-2,3.2017063E-2,6.0037516E-2,0E0,4.7706194E-2,0E0,0E0,0E0,0E0,0E0,5.3151E-2,5.815901E-2,1.1215052E-1,1.6604725E-1,1.6968362E-2,2.5234304E-2,4.5085967E-2,0E0,4.849542E-2,5.008562E-2,2.9341387E-2,7.961738E-2,3.7981108E-2,4.1759305E-2,2.3478836E-2,0E0,7.485337E-2,4.574713E-2,7.28936E-2,2.1751365E-1,0E0,0E0,1.966922E-2,2.4952158E-2,0E0,0E0,3.0496193E-2,5.1378436E-2,0E0,0E0,0E0,2.3503939E-2,4.2568E-2,4.1623466E-2,4.3489974E-2,3.927986E-2,0E0,2.5533155E-2,0E0,0E0,4.469379E-2,0E0,2.1775529E-2,3.6518157E-2,4.162696E-2,5.0653566E-2,0E0,4.7239628E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7092025E-2,5.0426826E-2,2.55511E-2,0E0,5.7049096E-2,1.5601579E-2,2.4615955E-2,0E0,3.3039324E-2,0E0,5.0581615E-2,1.9178785E-2,0E0,0E0,0E0,4.1993797E-2,0E0,0E0,0E0,4.444571E-2,2.0093426E-2,0E0,4.1714117E-2,0E0,0E0,0E0,0E0,4.894001E-2,1.9793436E-2,3.455285E-2,0E0,2.1701159E-2,1.7265175E-2,3.9899647E-2,0E0,0E0,2.765806E-2,1.8651254E-2,2.0365711E-2,0E0,4.065009E-2,0E0,0E0,2.384675E-2,2.9795833E-2,3.794667E-2,3.4734488E-2,0E0,0E0,0E0,0E0,2.2545572E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,27,27,28,28,29,29,30,30,31,31,33,33,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,65,65,66,66,70,70,71,71,72,72,73,73,74,74,76,76,79,79,81,81,82,82,83,83,84,84,86,86,93,93,94,94,95,95,97,97,98,98,99,99,101,101,103,103,104,104,108,108,112,112,113,113,115,115,120,120,121,121,122,122,124,124,125,125,126,126,129,129,130,130,131,131,133,133,136,136,137,137,138,138,139,139,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,50,-1,-1,-1,-1,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,-1,80,82,84,86,-1,-1,88,90,-1,-1,92,94,-1,-1,-1,96,98,100,102,104,-1,106,-1,-1,108,-1,110,112,114,116,-1,118,-1,-1,-1,-1,-1,-1,120,122,124,-1,126,128,130,-1,132,-1,134,136,-1,-1,-1,138,-1,-1,-1,140,142,-1,144,-1,-1,-1,-1,146,148,150,-1,152,154,156,-1,-1,158,160,162,-1,164,-1,-1,166,168,170,172,-1,-1,-1,-1,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,8.719101E0,5.3E1,2.71E2,3.535931E2,9.7573663E2,2.82E2,1.957E3,1E0,3.89E2,1.2674157E1,-7.91766E-3,1.2E1,4.814389E4,8.484036E5,8.585303E-4,1.6E1,1.6942337E1,6.048589E-4,1.0428572E1,1E0,-1.4655216E-3,3.378295E-3,8.368036E-4,-2.2313385E-3,2.0753755E-3,6.153532E-3,2.4693803E2,2.1356704E0,8E0,1.43E2,1E0,-3.6059632E-3,3.2931E4,3.3252272E-3,-2.1773067E-3,1.3047814E-3,-1.3436436E-2,-2.7457245E-3,4.4744192E8,1.3745962E0,7.956885E-1,3.0561172E5,1.8736842E0,1.62E2,1.7387315E2,1.6598296E-3,1.1379E4,6.763314E7,4.97E2,1.559733E6,1E0,1.48E2,8.036266E-2,-1.0840854E-3,3.49E2,2.297E3,1.684E0,1.2825651E0,-6.1208676E-3,-1.5378622E-3,6.48334E5,1.84E2,2.1254164E-3,-3.1143334E-3,8E0,6.764151E0,3.6854463E-4,-4.22169E-3,3.7591243E-3,1.7697E4,4.4E2,1.6957658E6,1.8383686E5,5.4275E2,-5.898438E-3,1.559733E6,6.4986926E-3,2.0354849E-3,1.1283241E6,-4.052917E-3,1.0586236E5,7.4010696E0,2.596275E3,3.1790426E8,-1.1895893E-2,1.4817301E3,-3.164914E-3,-5.282204E-4,-6.216524E-3,-2.1999117E-3,-8.763408E-6,-3.5414186E-3,1.84E2,1.0722478E5,1.3529412E0,2.043597E-3,1.17E2,2.8266037E2,3.4E1,3.809162E-3,7.601393E2,2.1433367E-3,4.8064E4,5.619529E3,3.1877044E-4,-3.7676631E-3,-2.6951255E-3,2.7777777E0,2.968216E-3,-8.8688095E-5,5.6172274E-3,4.1973075E6,6.46E2,-6.267547E-4,1.7717391E0,-3.141287E-3,-2.6898896E-3,2.7368255E-3,3.3460625E-3,8.754E3,1.2894E4,1.4E1,2.0651363E-3,9.489462E6,1.625E0,7.694314E-7,-2.0034993E-3,6.615832E-5,1.2751362E8,3.9939122E5,2.067E3,-5.6431512E-3,2.1068998E6,-3.3615E-3,-2.1279707E-4,6.0091515E6,1.125584E-3,1E0,9.837502E1,5.598643E-4,2.6723088E-3,6.3184304E-3,-2.9879278E-3,4.3948618E8,-7.803605E-4,1.5939849E-3,-4.884244E-4,-3.6423355E-3,-5.622856E-4,4.5570764E-3,-4.549525E-4,-3.9587575E-3,5.481957E-4,-2.3209727E-3,-6.2672775E-3,-1.5254904E-3,8.0606254E-4,-1.446732E-3,-9.0561016E-4,-4.565492E-3,-2.7453857E-3,8.774453E-4,9.408216E-4,-1.5628253E-3,3.05456E-3,-2.4745456E-4,2.711664E-3,5.094988E-4,1.1849916E-3,4.369715E-3,4.298175E-3,9.6806703E-4,6.52018E-5,2.6862174E-3],"split_indices":[2,69,44,2,70,48,44,9,26,2,69,0,3,60,43,0,3,71,52,68,29,0,0,0,0,0,0,71,53,0,2,19,0,9,0,0,0,0,0,7,53,53,48,68,2,48,0,9,7,2,9,29,0,72,0,8,2,69,69,0,0,46,44,0,0,8,71,0,0,0,9,10,66,48,67,0,9,0,0,60,0,48,71,67,47,0,67,0,0,0,0,0,0,44,66,68,0,10,4,8,0,67,0,44,48,0,0,0,68,0,0,0,43,0,0,69,0,0,0,0,1,12,3,0,12,68,52,0,0,7,60,2,0,66,0,0,62,53,29,73,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,5.18E2,5.12E2,4.86E2,3.2E1,3.1E1,4.81E2,9.5E1,3.91E2,2.3E1,9E0,4E0,2.7E1,1.5E1,4.66E2,8E0,8.7E1,2.11E2,1.8E2,1.1E1,1.2E1,5E0,4E0,6E0,2.1E1,4E0,1.1E1,1.61E2,3.05E2,6.3E1,2.4E1,2.04E2,7E0,1.76E2,4E0,7E0,4E0,7E0,5E0,1.44E2,1.7E1,2.29E2,7.6E1,8E0,5.5E1,1.6E1,8E0,1.85E2,1.9E1,3.1E1,1.45E2,1.26E2,1.8E1,1.2E1,5E0,1.35E2,9.4E1,6E1,1.6E1,4E0,4E0,3.4E1,2.1E1,4E0,1.2E1,2.2E1,1.63E2,1E1,9E0,4E0,2.7E1,8.1E1,6.4E1,2.1E1,1.05E2,5E0,1.3E1,6E0,6E0,1.3E2,5E0,2E1,7.4E1,2.1E1,3.9E1,4E0,1.2E1,7E0,2.7E1,4E0,1.7E1,1.5E1,7E0,1.36E2,2.7E1,2E1,7E0,5.1E1,3E1,6E1,4E0,1.7E1,4E0,7.2E1,3.3E1,9E0,4E0,5E0,1.25E2,7E0,1.3E1,1.6E1,5.8E1,1.7E1,4E0,3.2E1,7E0,7E0,5E0,1.3E1,1.23E2,1.7E1,1E1,4E0,1.6E1,3E1,2.1E1,1.5E1,1.5E1,4.7E1,1.3E1,1.2E1,5E0,6.3E1,9E0,9E0,2.4E1,9.2E1,3.3E1,4.5E1,1.3E1,1E1,7E0,4E0,2.8E1,2.2E1,1.01E2,8E0,9E0,6E0,4E0,1.1E1,5E0,5E0,2.5E1,1.5E1,6E0,2.9E1,1.8E1,9E0,4E0,7E0,5E0,4.4E1,1.9E1,1.8E1,6E0,1.5E1,7.7E1,1.7E1,1.6E1,3.6E1,9E0,1.2E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-3.4122604E-3,-2.430848E-2,9.615928E-3,-1.9493194E-2,-1.2938137E-1,1.4941398E-2,-4.113631E-2,-1.576189E-2,-6.713998E-2,-1.2116654E-2,-5.2154955E-2,3.404663E-2,3.869755E-3,-9.680642E-3,-2.5365511E-2,-4.5177612E-2,-1.0360505E-2,-2.1487218E-1,6.372106E-3,9.4395614E-4,-4.204237E-3,2.5462193E-2,6.391187E-2,-6.043369E-2,1.0937199E-2,-3.910601E-2,3.5298183E-3,-8.04002E-3,-6.352307E-2,2.92759E-4,-2.0525085E-2,-3.4781634E-3,-1.43902395E-2,-1.47886565E-2,2.5264043E-3,6.745667E-2,1.3960125E-2,7.677619E-2,2.1064932E-2,-1.634125E-1,2.0904155E-2,2.8469943E-2,-4.453927E-3,-5.89141E-2,5.100592E-4,-1.5182093E-3,1.8113337E-3,9.6618006E-4,-3.4525094E-3,-7.189631E-3,3.8627807E-2,-6.6283636E-2,-1.05567835E-2,1.8204711E-3,-2.276437E-3,3.5901018E-3,-4.4359217E-4,2.3128465E-3,4.5508087E-2,8.90509E-2,1.1188345E-3,-1.0882863E-3,4.0768897E-3,-9.455465E-3,-1.6631957E-2,-2.6570057E-4,4.088667E-3,7.813883E-3,4.2457435E-2,6.957198E-4,-8.301094E-3,-4.130163E-2,-7.3849675E-3,1.4438928E-2,-1.6662836E-2,-1.058075E-3,6.0410284E-2,-4.1354834E-3,-1.5883277E-3,-6.850375E-3,-4.5682453E-3,-6.2522357E-3,3.1907817E-3,6.395616E-2,-3.237498E-4,5.1193666E-2,1.23251446E-1,-2.5711127E-3,1.4145382E-3,1.4943029E-2,-3.59765E-3,4.8511443E-3,2.9721396E-2,-5.66817E-3,6.7258045E-2,-3.552426E-3,-1.2119684E-3,1.5243745E-4,2.5778625E-3,-3.6472242E-2,-2.3530026E-3,3.5207032E-3,1.8926304E-4,-1.917663E-2,1.7084794E-2,-3.6043897E-2,6.5483446E-3,-1.00886034E-4,3.6870416E-3,3.405424E-3,-3.9663675E-4,6.8591866E-3,2.225105E-3,-1.7099573E-3,5.296983E-2,1.3106582E-2,6.0004137E-2,1.35541065E-2,-1.9758547E-2,4.5102823E-4,4.032157E-3,-3.6705396E-4,-2.844864E-3,2.3753118E-2,-1.996189E-2,9.818433E-4,-3.0247677E-2,-1.1644831E-2,1.8071781E-3,-8.0444245E-4,-4.400291E-3,3.0311547E-2,-1.7426096E-2,-2.5337454E-2,3.1201009E-2,3.9751194E-3,-1.2712923E-3,5.095153E-2,-2.8290223E-2,3.9076055E-3,1.6565343E-2,-9.108564E-4,5.9718102E-2,-5.727447E-2,-6.9668833E-3,3.9136843E-3,-2.7331631E-5,-2.6043411E-3,2.8058012E-6,-5.1617913E-5,-1.8449372E-3,4.8178105E-4,-2.8277594E-3,6.156484E-4,2.9867042E-3,-2.7970145E-3,9.3909475E-6,2.2863483E-3,-2.1491125E-3,-3.2629122E-4,2.5960503E-3,2.5885573E-4,3.5710714E-3,-1.2076759E-4,-3.81259E-3,-1.8508105E-3,2.5246267E-3,-2.178779E-3,8.830343E-4,3.728812E-3,-1.2060668E-4,-6.442816E-4,-3.8868415E-3,2.7928483E-3,-7.7613455E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,29,31,33,-1,-1,35,37,39,41,43,-1,45,47,49,51,-1,-1,53,-1,55,57,59,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,77,79,-1,-1,-1,-1,81,83,85,-1,-1,-1,87,-1,-1,-1,89,91,93,-1,95,-1,97,99,-1,101,-1,-1,103,-1,105,-1,107,-1,109,111,-1,-1,113,-1,-1,115,117,119,-1,-1,-1,-1,121,123,-1,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,133,135,137,139,141,143,-1,-1,-1,-1,145,147,-1,149,151,-1,-1,-1,153,155,157,159,-1,-1,161,163,-1,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.779184E-1,1.9581272E-1,1.7062047E-1,6.578879E-2,1.6078484E-1,1.2043826E-1,1.5403841E-1,5.5141896E-2,2.9459193E-1,0E0,3.5697617E-2,5.1955268E-2,1.6517586E-1,0E0,8.129669E-2,3.6279805E-2,3.216694E-2,9.7919494E-2,2.0233218E-2,0E0,0E0,7.827414E-2,2.3737773E-2,3.049547E-1,8.836733E-2,5.0315738E-2,0E0,2.3571357E-2,3.5632312E-2,4.2156037E-2,6.863279E-2,0E0,0E0,2.8106462E-2,0E0,2.839829E-2,4.7662977E-2,2.0423293E-2,3.805142E-2,4.8086843E-1,3.8347248E-2,4.3873988E-2,1.5803763E-1,5.739686E-2,0E0,0E0,0E0,0E0,0E0,2.544228E-2,3.3550337E-2,1.6933404E-2,4.057107E-2,0E0,0E0,0E0,0E0,5.531943E-2,3.435672E-2,2.9942974E-2,0E0,0E0,0E0,2.0217083E-2,0E0,0E0,0E0,3.9648652E-2,6.937079E-2,7.281602E-2,0E0,1.5144557E-2,0E0,1.7323423E-2,2.4292538E-2,0E0,1.570034E-2,0E0,0E0,3.6321416E-2,0E0,3.3032175E-2,0E0,2.7433239E-2,0E0,2.0824704E-2,1.936005E-2,0E0,0E0,3.7452903E-2,0E0,0E0,3.7955493E-2,4.279112E-2,1.6413674E-2,0E0,0E0,0E0,0E0,2.5294248E-2,2.3918843E-2,0E0,0E0,3.6897816E-2,2.7040808E-2,2.7020108E-2,3.5280492E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.344991E-2,5.0283797E-2,8.134108E-2,2.7424872E-2,4.4794347E-2,4.327106E-2,0E0,0E0,0E0,0E0,3.1215465E-2,2.2603339E-2,0E0,1.8015005E-2,2.1424718E-2,0E0,0E0,0E0,1.7218739E-2,2.357597E-2,4.1924838E-2,1.7609848E-2,0E0,0E0,3.1825155E-2,3.459797E-2,0E0,2.5910014E-2,4.906027E-2,2.1146942E-2,2.559235E-2,4.6757687E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,50,50,51,51,52,52,57,57,58,58,59,59,63,63,67,67,68,68,69,69,71,71,73,73,74,74,76,76,79,79,81,81,83,83,85,85,86,86,89,89,92,92,93,93,94,94,99,99,100,100,103,103,104,104,105,105,106,106,113,113,114,114,115,115,116,116,117,117,118,118,123,123,124,124,126,126,127,127,131,131,132,132,133,133,134,134,137,137,138,138,140,140,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,30,32,34,-1,-1,36,38,40,42,44,-1,46,48,50,52,-1,-1,54,-1,56,58,60,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,78,80,-1,-1,-1,-1,82,84,86,-1,-1,-1,88,-1,-1,-1,90,92,94,-1,96,-1,98,100,-1,102,-1,-1,104,-1,106,-1,108,-1,110,112,-1,-1,114,-1,-1,116,118,120,-1,-1,-1,-1,122,124,-1,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,134,136,138,140,142,144,-1,-1,-1,-1,146,148,-1,150,152,-1,-1,-1,154,156,158,160,-1,-1,162,164,-1,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,6.0692043E0,2.856934E7,1.6631816E8,2E0,3.72381E5,2.7920364E2,1.63757E4,5.1942E4,-1.2116654E-2,3.3817584E7,8.260895E8,1E0,-9.680642E-3,2.3751075E5,5.4009717E3,3.66025E5,3E0,2.906237E2,9.4395614E-4,-4.204237E-3,1.6869704E6,2.8277853E11,1.048796E6,1.8497453E0,2.0618556E-2,3.5298183E-3,2.5192308E0,8E0,2E1,8.39777E5,-3.4781634E-3,-1.43902395E-2,3.5714287E-1,2.5264043E-3,3.89E2,1.38132E5,1.0284041E2,4.7083335E0,1.8669039E1,1.5971001E1,5.1191E4,1.9500381E2,1.6322028E3,5.100592E-4,-1.5182093E-3,1.8113337E-3,9.6618006E-4,-3.4525094E-3,1.2E1,3.2E1,1.1732674E1,7.814751E6,1.8204711E-3,-2.276437E-3,3.5901018E-3,-4.4359217E-4,1.6322028E3,1.3E1,1.0717949E1,1.1188345E-3,-1.0882863E-3,4.0768897E-3,1.03860024E8,-1.6631957E-2,-2.6570057E-4,4.088667E-3,2.152E3,1.5037551E4,1.3388E4,-8.301094E-3,1.775373E1,-7.3849675E-3,2.1924414E2,1.4473684E-1,-1.058075E-3,6.763314E7,-4.1354834E-3,-1.5883277E-3,3.11626E8,-4.5682453E-3,3.6522612E6,3.1907817E-3,3.741267E7,-3.237498E-4,9E0,2.5503825E6,-2.5711127E-3,1.4145382E-3,9.7199225E-1,-3.59765E-3,4.8511443E-3,1.7E1,1.9E1,3.8396227E0,-3.552426E-3,-1.2119684E-3,1.5243745E-4,2.5778625E-3,4.0869565E0,2.0625E0,3.5207032E-3,1.8926304E-4,2.8E1,1.7391304E0,5.556659E6,2.308943E0,-1.00886034E-4,3.6870416E-3,3.405424E-3,-3.9663675E-4,6.8591866E-3,2.225105E-3,2.732002E7,2.884462E0,5.812E3,2.1818182E0,1.0901037E10,7.919006E7,4.5102823E-4,4.032157E-3,-3.6705396E-4,-2.844864E-3,1.5365228E5,2.5437157E2,9.818433E-4,3.088104E5,2.49E2,1.8071781E-3,-8.0444245E-4,-4.400291E-3,1.225E0,3.5589743E0,3.53E2,5.036784E6,3.9751194E-3,-1.2712923E-3,9.8933E4,9E0,3.9076055E-3,1.0130841E1,2.3266666E0,1.7022608E7,3.0588236E0,1.4872362E-1,3.9136843E-3,-2.7331631E-5,-2.6043411E-3,2.8058012E-6,-5.1617913E-5,-1.8449372E-3,4.8178105E-4,-2.8277594E-3,6.156484E-4,2.9867042E-3,-2.7970145E-3,9.3909475E-6,2.2863483E-3,-2.1491125E-3,-3.2629122E-4,2.5960503E-3,2.5885573E-4,3.5710714E-3,-1.2076759E-4,-3.81259E-3,-1.8508105E-3,2.5246267E-3,-2.178779E-3,8.830343E-4,3.728812E-3,-1.2060668E-4,-6.442816E-4,-3.8868415E-3,2.7928483E-3,-7.7613455E-4],"split_indices":[2,68,60,5,32,9,48,43,1,0,7,5,8,0,48,43,9,8,4,0,0,60,46,9,68,72,0,68,3,3,9,0,0,68,0,8,9,73,69,73,71,1,73,4,0,0,0,0,0,3,12,73,60,0,0,0,0,4,8,71,0,0,0,7,0,0,0,2,48,2,0,71,0,67,71,0,7,0,0,46,0,60,0,7,0,8,47,0,0,53,0,0,8,8,69,0,0,0,0,73,71,0,0,10,71,62,69,0,0,0,0,0,0,5,71,44,69,46,7,0,0,0,0,43,4,0,60,10,0,0,0,68,69,44,9,0,0,1,8,0,71,68,9,68,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.019E3,3.91E2,6.28E2,3.75E2,1.6E1,5.69E2,5.9E1,3.49E2,2.6E1,5E0,1.1E1,2.08E2,3.61E2,4E0,5.5E1,5.3E1,2.96E2,8E0,1.8E1,4E0,7E0,1.63E2,4.5E1,3.5E1,3.26E2,4.9E1,6E0,1.8E1,3.5E1,1.45E2,1.51E2,4E0,4E0,1.3E1,5E0,3.4E1,1.29E2,3.4E1,1.1E1,1.5E1,2E1,1.52E2,1.74E2,3.5E1,1.4E1,1.2E1,6E0,4E0,3.1E1,1.22E2,2.3E1,2.6E1,1.25E2,5E0,8E0,3E1,4E0,9.5E1,3.4E1,2.7E1,7E0,7E0,4E0,9E0,6E0,1.5E1,5E0,6.2E1,9E1,1.7E2,4E0,3.1E1,4E0,3.7E1,8.5E1,6E0,1.7E1,1.4E1,1.2E1,1.21E2,4E0,8.5E1,1E1,2.5E1,9E0,1.4E1,1.3E1,4E0,5E0,5.8E1,4E0,1.4E1,7.6E1,1.56E2,1.4E1,8E0,2.3E1,3E1,7E0,3.5E1,5E1,1.3E1,4E0,8E1,4.1E1,2.5E1,6E1,5E0,2E1,1E1,4E0,9E0,4E0,4.1E1,1.7E1,5E1,2.6E1,6.6E1,9E1,4E0,1E1,1.7E1,1.8E1,2E1,3E1,1.7E1,6.3E1,1.8E1,2.3E1,2E1,5E0,3E1,3E1,2.4E1,1.7E1,1.2E1,5E0,2.6E1,2.4E1,1.6E1,1E1,5.1E1,1.5E1,2.2E1,6.8E1,5E0,1.5E1,1E1,2E1,1.6E1,4.7E1,1.3E1,5E0,2.1E1,9E0,8E0,2.2E1,5E0,1.9E1,7E0,1E1,1E1,1.6E1,1.7E1,7E0,4E0,6E0,1.5E1,3.6E1,1.1E1,4E0,9E0,1.3E1,8E0,6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[1.1045056E-4,-1.2815726E-2,1.8037228E-2,-1.0047119E-2,-9.884234E-2,2.6030619E-2,-2.7895272E-2,-1.3516349E-2,2.8885858E-2,1.9730919E-3,-1.849727E-1,5.351257E-3,2.3015238E-2,-3.895394E-2,2.6402446E-3,-2.3300089E-2,3.4412355E-3,-1.0403417E-2,5.4115847E-2,-1.2781561E-2,-2.1793044E-3,8.731328E-2,1.904624E-2,-5.320839E-2,1.5577832E-2,-8.889742E-3,-2.0766817E-2,8.538131E-3,-6.032947E-2,-1.9747985E-3,2.0074777E-3,1.0895453E-3,3.8380194E-3,1.1194588E-1,-6.421764E-4,1.0966069E-2,3.8813796E-2,-6.912194E-2,-6.271259E-3,4.667697E-3,-1.5536258E-3,-4.356134E-3,-1.850333E-2,-3.2385606E-3,1.1985731E-2,-8.4086595E-4,-6.511587E-3,2.1493388E-4,7.2192405E-3,1.346729E-2,-5.2106287E-3,5.8112144E-3,2.9883713E-2,-5.0095234E-2,-8.004593E-3,1.1779047E-3,-2.8735166E-3,1.4696375E-3,-2.8318046E-2,1.5294258E-2,-3.3248083E-3,5.098607E-4,3.7546866E-2,-3.442261E-2,3.904164E-2,-1.0026759E-2,-6.8698E-2,-3.5556927E-3,2.7173567E-3,-1.8700685E-2,-4.8779387E-2,-4.7632554E-3,2.4359833E-2,9.096348E-3,-1.4360896E-1,4.2818524E-2,-2.557734E-3,8.6203177E-4,-2.9242078E-3,-1.238015E-3,4.5631785E-2,-2.2741118E-3,1.4248629E-3,-5.178553E-3,-2.0266369E-3,7.542003E-3,-3.213402E-2,-8.339582E-3,-5.7941783E-2,-6.421848E-2,-6.1987992E-3,9.651417E-3,-5.2076153E-2,2.788856E-2,-2.1867906E-3,-4.310182E-2,1.693625E-2,-1.04953805E-2,-1.4468948E-3,3.3015613E-2,4.7215084E-3,2.8121676E-2,6.901285E-2,1.2956869E-3,-6.594403E-4,-7.989502E-4,-3.979578E-3,-8.8121585E-4,1.4106815E-3,-5.0460147E-3,7.296363E-4,-8.736107E-4,-3.9585796E-3,-2.6647928E-3,4.7477442E-4,8.5820846E-4,-2.8594034E-3,-3.5427369E-3,1.21667406E-4,1.0155893E-3,2.461328E-3,-3.6172033E-4,-3.9427355E-3,1.8423343E-4,1.8254595E-3,2.1788385E-3,1.546392E-4,4.2802566E-3,8.0725714E-4,3.6999774E-3,-2.1224305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,27,29,31,-1,-1,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,47,-1,49,51,53,55,-1,-1,-1,57,-1,59,-1,-1,-1,-1,61,-1,-1,63,65,-1,-1,-1,67,69,71,-1,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,-1,-1,-1,-1,101,-1,-1,-1,-1,103,105,107,109,111,113,115,117,119,-1,121,123,-1,-1,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.44469E-1,1.4500752E-1,1.6300927E-1,8.075556E-2,2.3954457E-1,9.941837E-2,6.4297885E-2,9.0832196E-2,4.9135122E-2,0E0,1.3856879E-1,0E0,9.173985E-2,4.6690777E-2,0E0,1.4467044E-1,6.584665E-2,3.6693778E-2,2.456697E-2,0E0,0E0,5.3912237E-2,5.4839283E-2,3.467253E-2,5.8063302E-2,0E0,5.533734E-2,5.1538415E-2,4.8378706E-2,0E0,0E0,0E0,0E0,8.010849E-2,0E0,7.797346E-2,7.390836E-2,6.0671598E-2,2.4682611E-2,0E0,0E0,0E0,6.55153E-2,0E0,5.1097475E-2,0E0,0E0,0E0,0E0,7.579915E-2,0E0,0E0,5.5523872E-2,2.2418097E-2,0E0,0E0,0E0,3.219898E-2,4.3167427E-2,3.1754952E-2,0E0,1.9806474E-1,4.400278E-2,1.9572716E-2,3.6367826E-2,1.9139552E-2,1.7710283E-2,3.2909174E-2,0E0,6.213027E-2,4.6277687E-2,3.8012605E-2,3.16787E-2,6.252349E-2,7.734038E-2,4.3776244E-2,0E0,0E0,0E0,0E0,2.8355211E-2,0E0,0E0,0E0,0E0,3.3948395E-2,2.1752847E-2,5.1911224E-2,1.2272502E-1,4.895675E-2,1.7556578E-2,2.860646E-2,1.7703831E-2,1.6071111E-2,0E0,2.9130146E-2,3.8430322E-2,0E0,0E0,2.9640816E-2,0E0,2.8613165E-2,2.5638789E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,28,28,33,33,35,35,36,36,37,37,38,38,42,42,44,44,49,49,52,52,53,53,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,80,80,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,96,96,99,99,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,28,30,32,-1,-1,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,48,-1,50,52,54,56,-1,-1,-1,58,-1,60,-1,-1,-1,-1,62,-1,-1,64,66,-1,-1,-1,68,70,72,-1,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,-1,-1,-1,-1,102,-1,-1,-1,-1,104,106,108,110,112,114,116,118,120,-1,122,124,-1,-1,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,6.0692043E0,2.856934E7,4.5866325E6,1.901875E2,1.2E1,4.435876E2,1.380863E6,1E0,1.9730919E-3,2.6235E4,5.351257E-3,1.0918E4,2.3308511E5,2.6402446E-3,4E0,1.2E1,1E0,1.81E2,-1.2781561E-2,-2.1793044E-3,1.4791E4,9.033567E6,2.0618556E-2,1E0,-8.889742E-3,6E0,3.7788504E7,1.3994015E7,-1.9747985E-3,2.0074777E-3,1.0895453E-3,3.8380194E-3,3.7509E4,-6.421764E-4,6.5026445E3,5E0,4.9E1,3.463702E2,4.667697E-3,-1.5536258E-3,-4.356134E-3,1.4615384E0,-3.2385606E-3,4.0903897E2,-8.4086595E-4,-6.511587E-3,2.1493388E-4,7.2192405E-3,1.7068776E10,-5.2106287E-3,5.8112144E-3,1.2895454E3,6.5E2,-8.004593E-3,1.1779047E-3,-2.8735166E-3,1.3E1,3.14403E5,1.0140845E0,-3.3248083E-3,2.1373269E8,2.5842668E7,1.194E3,1E1,5.17259E8,1.33E2,1E0,2.7173567E-3,1.65E2,9.5E2,1.2751362E8,5.123862E1,1E0,4.626E3,1.7743401E6,-2.557734E-3,8.6203177E-4,-2.9242078E-3,-1.238015E-3,1.9001543E1,-2.2741118E-3,1.4248629E-3,-5.178553E-3,-2.0266369E-3,3.883E4,8E0,5.5E1,2.2084616E2,3.56E2,2.8266037E2,1.8E1,1.3E1,7.884708E0,-2.1867906E-3,1.2585366E1,7.445455E2,-1.04953805E-2,-1.4468948E-3,4.076E4,4.7215084E-3,9.5596474E-1,2.7561485E6,1.2956869E-3,-6.594403E-4,-7.989502E-4,-3.979578E-3,-8.8121585E-4,1.4106815E-3,-5.0460147E-3,7.296363E-4,-8.736107E-4,-3.9585796E-3,-2.6647928E-3,4.7477442E-4,8.5820846E-4,-2.8594034E-3,-3.5427369E-3,1.21667406E-4,1.0155893E-3,2.461328E-3,-3.6172033E-4,-3.9427355E-3,1.8423343E-4,1.8254595E-3,2.1788385E-3,1.546392E-4,4.2802566E-3,8.0725714E-4,3.6999774E-3,-2.1224305E-4],"split_indices":[4,68,60,43,4,33,71,9,29,0,1,0,9,48,0,3,71,16,0,0,0,2,60,72,15,0,67,7,5,0,0,0,0,1,0,4,3,3,71,0,0,0,68,0,67,0,0,0,0,46,0,0,48,10,0,0,0,8,9,71,0,5,62,2,3,46,0,8,0,0,10,7,73,29,2,58,0,0,0,0,71,0,0,0,0,9,32,0,67,2,4,3,8,71,0,71,67,0,0,44,0,68,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.053E3,6.12E2,4.41E2,5.94E2,1.8E1,3.76E2,6.5E1,5.46E2,4.8E1,7E0,1.1E1,1.1E1,3.65E2,5.8E1,7E0,3.46E2,2E2,1.9E1,2.9E1,6E0,5E0,2E1,3.45E2,4.6E1,1.2E1,4E0,3.42E2,1.86E2,1.4E1,1.2E1,7E0,1.5E1,1.4E1,1.6E1,4E0,2.46E2,9.9E1,3.4E1,1.2E1,4E0,8E0,9E0,3.33E2,7E0,1.79E2,1E1,4E0,5E0,1.1E1,2.42E2,4E0,8E0,9.1E1,3E1,4E0,8E0,4E0,1.1E2,2.23E2,1.73E2,6E0,1.58E2,8.4E1,1.1E1,8E1,1E1,2E1,1.02E2,8E0,1.53E2,7E1,5.4E1,1.19E2,1.5E2,8E0,8E1,4E0,4E0,7E0,7E0,7.3E1,5E0,5E0,6E0,1.4E1,7.4E1,2.8E1,1.22E2,3.1E1,5.1E1,1.9E1,4.2E1,1.2E1,1.14E2,5E0,1.9E1,1.31E2,4E0,4E0,7E1,1E1,4.3E1,3E1,3.8E1,3.6E1,2.3E1,5E0,9.6E1,2.6E1,1.8E1,1.3E1,1.7E1,3.4E1,4E0,1.5E1,3.8E1,4E0,8E0,4E0,9.4E1,2E1,1.1E1,8E0,8.4E1,4.7E1,4.7E1,2.3E1,5E0,3.8E1,2.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[3.93045E-4,-1.3073961E-2,1.5106187E-2,6.039252E-2,-1.6067546E-2,1.2840821E-2,4.183979E-3,-1.3171644E-3,4.937781E-3,-1.2578861E-2,-6.315427E-2,1.4488584E-2,-3.915205E-3,-1.6600374E-2,2.839642E-2,-9.160539E-3,-1.0814054E-1,-2.6114536E-3,2.2298638E-2,2.7369767E-2,-2.0396769E-2,-5.7654246E-3,4.7308274E-2,2.9032074E-3,-3.614889E-2,-1.28113E-2,-4.454249E-2,9.089443E-3,-2.072299E-2,4.0063714E-3,1.9067867E-2,6.4171925E-2,-5.060151E-4,-1.4067255E-2,-5.613506E-2,5.745423E-2,-3.3637436E-4,5.193359E-4,-3.1033242E-3,-9.091091E-2,3.0665323E-3,-1.3860628E-2,3.2347526E-2,-2.3520426E-3,5.395782E-3,1.8976177E-3,3.028788E-2,1.9093285E-3,5.326736E-3,-5.4896926E-3,-1.1842894E-2,-1.08501256E-1,-6.9239717E-3,3.7535783E-3,1.9359307E-2,-6.4186123E-4,-7.0581343E-3,2.6896414E-3,-2.4070736E-2,3.5155264E-3,1.4133363E-2,2.9097486E-3,-7.587823E-4,7.510281E-3,-6.1715238E-2,4.9648867E-3,2.5730597E-2,-1.9521604E-3,-2.673824E-2,-1.3320321E-1,-5.468816E-4,-3.5595693E-2,3.7718985E-2,-1.2059099E-3,2.620339E-3,-3.5133367E-4,-3.1547733E-3,3.8654448E-3,-1.2719702E-3,-2.8051767E-3,1.2588836E-2,-1.2615975E-4,-5.550979E-3,5.3099122E-2,1.11167105E-2,-2.171132E-2,1.1895519E-2,-3.152733E-2,2.457812E-2,-1.7551357E-1,-4.7885578E-2,-2.3510227E-4,-3.729177E-3,5.113411E-3,-7.007122E-3,-1.6715389E-2,3.377651E-3,3.6446556E-2,-1.2793975E-3,6.6027895E-2,1.8545505E-2,3.801265E-2,-1.643462E-2,-6.074711E-4,-5.2675437E-3,8.450591E-4,-1.9476142E-3,-4.3062828E-4,-1.945756E-3,-1.6807514E-3,3.387191E-3,-1.25386E-2,-5.3577637E-3,-3.9051617E-3,-7.6920966E-5,1.7336402E-3,-2.3163406E-3,5.540255E-4,-2.39E-3,8.3220925E-4,4.0701777E-3,-1.7803643E-3,8.6021045E-4,4.569807E-3,1.8227851E-3,-1.7364054E-3,1.7217251E-3,2.5233924E-3,-2.8783127E-4,-6.54822E-3,1.6369525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,-1,19,21,23,25,27,29,31,33,-1,35,-1,37,-1,39,41,43,-1,45,47,-1,49,51,53,-1,-1,-1,55,-1,57,59,-1,61,63,65,-1,-1,-1,67,69,71,-1,73,-1,-1,-1,75,-1,77,-1,-1,79,81,-1,83,85,87,89,-1,91,93,-1,-1,-1,-1,-1,95,-1,97,-1,-1,99,101,103,105,107,109,111,113,-1,-1,-1,115,117,-1,119,121,123,125,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0150974E-1,1.1775222E-1,8.191388E-2,8.983888E-2,8.314797E-2,7.728656E-2,0E0,0E0,0E0,7.9072714E-2,8.402355E-2,6.2293567E-2,0E0,7.3171936E-2,1.3048084E-1,3.5306223E-2,1.9206207E-1,3.134078E-2,6.494801E-2,5.0065406E-2,8.9918986E-2,0E0,2.2018872E-2,0E0,1.9868134E-2,0E0,8.3181426E-2,4.848808E-2,4.6006266E-2,0E0,5.889356E-2,1.6257837E-2,0E0,7.828672E-2,1.5386857E-1,2.8338954E-2,0E0,0E0,0E0,5.028961E-2,0E0,3.5254423E-2,3.4781747E-2,0E0,4.1244015E-2,4.402208E-2,6.2304556E-2,0E0,0E0,0E0,4.9449306E-2,6.7668915E-2,4.230233E-2,0E0,2.5258288E-2,0E0,0E0,0E0,2.9330969E-2,0E0,3.5456855E-2,0E0,0E0,3.991979E-2,3.4141395E-2,0E0,6.980276E-2,5.580282E-2,3.3706546E-2,7.462129E-2,0E0,2.7121047E-2,4.3199234E-2,0E0,0E0,0E0,0E0,0E0,3.3386372E-2,0E0,3.511826E-2,0E0,0E0,2.6026368E-2,8.6582914E-2,6.48913E-2,4.2258702E-2,2.853109E-2,3.832032E-2,5.4167837E-2,1.5102407E-2,0E0,0E0,0E0,1.9168131E-2,2.4746615E-2,0E0,3.5546616E-2,5.1131885E-2,3.4549788E-2,2.0316612E-2,4.3690994E-2,1.4569485E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,26,26,27,27,28,28,30,30,31,31,33,33,34,34,35,35,39,39,41,41,42,42,44,44,45,45,46,46,50,50,51,51,52,52,54,54,58,58,60,60,63,63,64,64,66,66,67,67,68,68,69,69,71,71,72,72,78,78,80,80,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,94,94,95,95,97,97,98,98,99,99,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,-1,20,22,24,26,28,30,32,34,-1,36,-1,38,-1,40,42,44,-1,46,48,-1,50,52,54,-1,-1,-1,56,-1,58,60,-1,62,64,66,-1,-1,-1,68,70,72,-1,74,-1,-1,-1,76,-1,78,-1,-1,80,82,-1,84,86,88,90,-1,92,94,-1,-1,-1,-1,-1,96,-1,98,-1,-1,100,102,104,106,108,110,112,114,-1,-1,-1,116,118,-1,120,122,124,126,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.969174E7,7E0,9.059046E6,3.5443038E-1,6.0692043E0,1E0,4.183979E-3,-1.3171644E-3,4.937781E-3,2.008E3,1.2999985E6,3.808001E5,-3.915205E-3,7.3333335E-1,2.1126761E0,5.004138E-3,1.088E3,3.38E2,1.0164831E-2,4.31903E6,6.648E3,-5.7654246E-3,4.5620965E6,2.9032074E-3,1.1634076E4,-1.28113E-2,1.764442E7,4.062891E8,1.19E2,4.0063714E-3,8.1407714E-1,6.347708E7,-5.060151E-4,6E0,7.714286E0,3.015873E0,-3.3637436E-4,5.193359E-4,-3.1033242E-3,1.8471494E7,3.0665323E-3,7.807738E6,1.3209776E6,-2.3520426E-3,5.49E2,1.2769117E7,8.998703E-1,1.9093285E-3,5.326736E-3,-5.4896926E-3,1E0,2.378914E6,1.1362371E6,3.7535783E-3,5.6115704E0,-6.4186123E-4,-7.0581343E-3,2.6896414E-3,1.4195632E6,3.5155264E-3,3.9961785E-2,2.9097486E-3,-7.587823E-4,2.87834E5,7.3835544E5,4.9648867E-3,5.8606E4,4.1277572E5,2.3E1,8.39E2,-5.468816E-4,1.34214E5,7.82E2,-1.2059099E-3,2.620339E-3,-3.5133367E-4,-3.1547733E-3,3.8654448E-3,5.467E3,-2.8051767E-3,1.4327235E7,-1.2615975E-4,-5.550979E-3,1.016E4,1.5153E4,2.923526E5,1.3177083E1,1.7142857E0,7E0,2E0,5.095006E5,-2.3510227E-4,-3.729177E-3,5.113411E-3,1.3152658E7,1.393875E5,3.377651E-3,6.819212E6,2.194E3,2.1298597E0,1.3388E4,1.1E1,5.668E3,-6.074711E-4,-5.2675437E-3,8.450591E-4,-1.9476142E-3,-4.3062828E-4,-1.945756E-3,-1.6807514E-3,3.387191E-3,-1.25386E-2,-5.3577637E-3,-3.9051617E-3,-7.6920966E-5,1.7336402E-3,-2.3163406E-3,5.540255E-4,-2.39E-3,8.3220925E-4,4.0701777E-3,-1.7803643E-3,8.6021045E-4,4.569807E-3,1.8227851E-3,-1.7364054E-3,1.7217251E-3,2.5233924E-3,-2.8783127E-4,-6.54822E-3,1.6369525E-4],"split_indices":[7,3,43,71,68,79,0,0,0,2,60,43,0,68,69,53,44,0,53,60,44,0,62,0,43,0,60,46,8,0,53,7,0,67,71,68,0,0,0,5,0,46,60,0,0,62,57,0,0,0,26,9,43,0,69,0,0,0,60,0,53,0,0,12,43,0,9,60,3,2,0,1,2,0,0,0,0,0,2,0,12,0,0,2,10,60,71,68,8,32,43,0,0,0,60,48,0,12,2,53,2,8,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,5.3E2,4.85E2,2E1,5.1E2,4.72E2,1.3E1,7E0,1.3E1,4.76E2,3.4E1,4.65E2,7E0,4.34E2,4.2E1,1.6E1,1.8E1,1.46E2,3.19E2,3.4E1,4E2,4E0,3.8E1,4E0,1.2E1,4E0,1.4E1,8.9E1,5.7E1,1.4E1,3.05E2,1.7E1,1.7E1,3.41E2,5.9E1,3.2E1,6E0,5E0,7E0,1E1,4E0,4.5E1,4.4E1,2.6E1,3.1E1,1.21E2,1.84E2,1.3E1,4E0,6E0,3.35E2,2.8E1,3.1E1,1.9E1,1.3E1,5E0,5E0,5E0,4E1,1.2E1,3.2E1,8E0,2.3E1,1.12E2,9E0,9E0,1.75E2,2.02E2,1.33E2,2.2E1,6E0,1.9E1,1.2E1,6E0,7E0,3E1,1E1,5E0,2.7E1,7E0,1.05E2,5E0,4E0,6E1,1.15E2,8.3E1,1.19E2,1.22E2,1.1E1,1.4E1,8E0,1.2E1,7E0,4E0,8E0,2.3E1,4E0,3.8E1,6.7E1,4.3E1,1.7E1,5.8E1,5.7E1,7.7E1,6E0,1.07E2,1.2E1,4E1,8.2E1,5E0,6E0,4E0,1E1,4E0,4E0,4E0,4E0,1.3E1,1E1,2.9E1,9E0,2.3E1,4.4E1,1.8E1,2.5E1,4E0,1.3E1,4.2E1,1.6E1,7E0,5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-1.554028E-3,-1.8304912E-2,1.6564656E-2,-1.5394103E-2,-1.174105E-1,4.5295143E-3,1.3435578E-2,-1.7329253E-2,2.893446E-2,7.616974E-4,-2.0850763E-1,-2.7874842E-2,1.9149173E-2,-1.4193781E-2,-5.370661E-2,6.4994946E-2,-9.626782E-4,-1.3356135E-2,-3.9830017E-3,-2.8058556E-3,-1.0309225E-2,4.101179E-3,1.6330482E-2,-2.546953E-2,-5.6612757E-3,-5.8101793E-3,-3.6794227E-2,1.7421166E-4,4.7487365E-3,6.8761827E-3,-2.5261184E-3,1.7881889E-2,-3.4079968E-3,-2.1883378E-2,-9.79488E-2,4.765879E-3,-2.6690217E-2,1.0881517E-3,-6.842333E-2,2.1666206E-2,-2.4684125E-3,1.5434419E-2,1.02140196E-1,-1.8405905E-2,-2.817911E-3,-6.939745E-3,-1.2055098E-3,9.4584463E-4,3.9208382E-3,-8.255553E-2,-9.86674E-3,-1.238414E-3,2.4044833E-3,-4.386652E-3,-6.0855004E-4,-5.4762646E-4,5.094096E-2,1.0483627E-2,4.634253E-2,7.2312662E-3,2.2500358E-3,-4.120051E-2,-9.466208E-3,5.538158E-3,-5.959844E-2,-6.128115E-4,-1.1181261E-2,-3.3496052E-2,3.211552E-2,3.636293E-3,2.5069484E-4,1.4981445E-2,-2.6025381E-2,2.7629407E-2,3.5036304E-3,-2.5828788E-2,-8.644425E-2,3.2633454E-2,-2.5208378E-2,2.6760949E-2,-1.4966975E-2,-8.099305E-4,-5.3122505E-3,-4.056157E-3,-2.0849917E-2,5.952784E-3,7.554004E-3,1.8604428E-2,-1.9266376E-2,3.207527E-3,-3.969943E-2,5.221785E-2,-1.4170104E-6,-1.921377E-3,1.0375849E-3,-6.2345397E-3,-2.0938758E-3,2.3278622E-3,-2.1058281E-3,-1.5122887E-4,-1.7460182E-3,1.591008E-3,-2.065975E-3,-1.4087139E-3,5.7482865E-4,1.2719618E-3,-1.5271936E-3,-1.1451181E-3,2.4683645E-3,-6.922327E-5,1.1849086E-3,3.1147618E-4,-2.618261E-3,3.176464E-4,-3.248578E-3,3.0440209E-4,5.0093806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,35,-1,37,-1,-1,39,-1,41,-1,43,45,47,49,51,53,55,-1,57,59,61,-1,-1,-1,63,-1,65,67,-1,-1,-1,-1,-1,69,71,73,-1,-1,75,77,79,81,-1,-1,83,85,-1,-1,87,89,91,-1,93,95,97,99,101,103,-1,-1,-1,105,-1,107,109,111,-1,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.101808E-1,1.5137413E-1,1.2386489E-1,4.48509E-2,1.864033E-1,0E0,1.12412475E-1,5.5811554E-2,4.084006E-2,0E0,6.377882E-2,3.285103E-2,8.038448E-2,4.392018E-2,4.2900458E-2,2.9692862E-2,0E0,0E0,0E0,0E0,3.0113984E-2,0E0,5.7510197E-2,4.906112E-2,5.757647E-2,0E0,3.961458E-2,0E0,0E0,2.7023563E-2,0E0,8.0086514E-2,0E0,2.4583153E-2,2.9879995E-2,5.3774714E-2,8.09523E-2,2.413808E-2,2.5131643E-2,2.4154678E-2,0E0,5.8579735E-2,2.4344124E-2,3.5270933E-2,0E0,0E0,0E0,4.731854E-2,0E0,2.1778351E-1,6.853256E-2,0E0,0E0,0E0,0E0,0E0,1.600309E-2,5.523633E-2,2.7157843E-2,0E0,0E0,3.1845495E-2,8.499063E-2,6.915836E-2,2.5944151E-2,0E0,0E0,2.8384332E-2,5.8425784E-2,0E0,0E0,3.7334062E-2,5.1050145E-2,2.312562E-2,0E0,2.9853227E-2,1.7818898E-2,5.0899398E-2,2.5549069E-2,4.434356E-2,3.484807E-2,0E0,0E0,0E0,2.2762176E-2,0E0,3.24804E-2,3.826867E-2,2.8596187E-2,0E0,4.7541924E-2,4.5454074E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,20,20,22,22,23,23,24,24,26,26,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,47,47,49,49,50,50,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,84,84,86,86,87,87,88,88,90,90,91,91],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,36,-1,38,-1,-1,40,-1,42,-1,44,46,48,50,52,54,56,-1,58,60,62,-1,-1,-1,64,-1,66,68,-1,-1,-1,-1,-1,70,72,74,-1,-1,76,78,80,82,-1,-1,84,86,-1,-1,88,90,92,-1,94,96,98,100,102,104,-1,-1,-1,106,-1,108,110,112,-1,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.194E3,6.27957E0,1.2E1,1.559287E5,9E0,4.5295143E-3,1.8439855E5,1.0700264E6,4E0,7.616974E-4,6.6E1,1.3631483E8,6.923077E-1,5.02E2,2.3238889E2,5.4674416E7,-9.626782E-4,-1.3356135E-2,-3.9830017E-3,-2.8058556E-3,5.16175E5,4.101179E-3,5.619497E7,3.578882E1,4.064E3,-5.8101793E-3,8.39E2,1.7421166E-4,4.7487365E-3,4.7220547E1,-2.5261184E-3,4.435876E2,-3.4079968E-3,2.3364584E1,3.3817584E7,3.653E3,5.1942E4,1.7171982E6,3.8192307E2,1.896E3,-2.4684125E-3,4.9321495E6,1.213274E9,3.89E2,-2.817911E-3,-6.939745E-3,-1.2055098E-3,1.683263E7,3.9208382E-3,1.7431098E8,1.925508E6,-1.238414E-3,2.4044833E-3,-4.386652E-3,-6.0855004E-4,-5.4762646E-4,2.0734E4,2.3E1,1.3E1,7.2312662E-3,2.2500358E-3,1.9E1,1.0443479E5,1E0,1E0,-6.128115E-4,-1.1181261E-2,6.15E2,2.331083E6,3.636293E-3,2.5069484E-4,3.7568388E0,1.592E3,1.3784861E5,3.5036304E-3,1.00037E5,7.336111E4,2.952381E0,3.2E1,1E0,1E0,-8.099305E-4,-5.3122505E-3,-4.056157E-3,7.89E2,5.952784E-3,2.2715E4,1.1772152E0,1.0743623E0,3.207527E-3,5.7420593E2,6.203939E2,-1.4170104E-6,-1.921377E-3,1.0375849E-3,-6.2345397E-3,-2.0938758E-3,2.3278622E-3,-2.1058281E-3,-1.5122887E-4,-1.7460182E-3,1.591008E-3,-2.065975E-3,-1.4087139E-3,5.7482865E-4,1.2719618E-3,-1.5271936E-3,-1.1451181E-3,2.4683645E-3,-6.922327E-5,1.1849086E-3,3.1147618E-4,-2.618261E-3,3.176464E-4,-3.248578E-3,3.0440209E-4,5.0093806E-3],"split_indices":[2,68,33,48,0,0,43,66,8,0,10,7,68,2,67,7,0,0,0,0,1,0,62,61,44,0,2,0,0,73,0,71,0,73,7,44,1,60,67,10,0,43,7,1,0,0,0,60,0,5,43,0,0,0,0,0,44,3,3,0,0,67,43,26,8,0,0,2,43,0,0,68,44,48,0,5,60,68,10,23,27,0,0,0,2,0,44,68,53,0,67,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,5.3E2,4.9E2,5.16E2,1.4E1,1.7E1,4.73E2,4.95E2,2.1E1,6E0,8E0,5.7E1,4.16E2,4.57E2,3.8E1,1.2E1,9E0,4E0,4E0,1.9E1,3.8E1,1.5E1,4.01E2,1.96E2,2.61E2,6E0,3.2E1,5E0,7E0,2.8E1,1E1,3.95E2,6E0,1.88E2,8E0,1.75E2,8.6E1,1.5E1,1.7E1,2.3E1,5E0,3.85E2,1E1,1.74E2,1.4E1,4E0,4E0,1.68E2,7E0,1.9E1,6.7E1,1E1,5E0,1.1E1,6E0,1.1E1,1.2E1,3.33E2,5.2E1,4E0,6E0,4.8E1,1.26E2,1.57E2,1.1E1,1.4E1,5E0,4.3E1,2.4E1,7E0,5E0,2.97E2,3.6E1,3.3E1,1.9E1,3.7E1,1.1E1,3.4E1,9.2E1,7.7E1,8E1,7E0,4E0,7E0,3.6E1,4E0,2E1,2.69E2,2.8E1,4E0,3.2E1,1.7E1,1.6E1,2.8E1,9E0,4E0,7E0,2.8E1,6E0,3.4E1,5.8E1,7E1,7E0,5.1E1,2.9E1,7E0,2.9E1,1.2E1,8E0,7E1,1.99E2,1.7E1,1.1E1,1.3E1,1.9E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics index 288bf2ad2..14ec21817 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics @@ -1,109 +1,109 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,duration_max,0.2953008,6139.4632063074905,32138.962204191026,15.0,560.0,1136.5,3113.0,515161.0 -1,scan_bw,0.17895658,1593713.489873519,2294166.855457363,0.0,174981.0170488308,655006.879324951,2101650.4235552456,17385294.707427993 -2,executorCPUTime_mean,0.0827063,1343.4213374053284,11697.684240647743,3.875,163.23010869565218,315.8137480798771,686.9930131090191,227947.84615384616 -3,scan_time,0.028598357,46345.203022339025,243193.36454642922,0.0,473.5,2177.5,10481.0,3127801.0 -4,executorDeserializeTime_mean,0.025142837,6.00370659180027,19.71922884656477,0.04395604395604396,2.2201527924130664,3.6647940074906367,5.147844471947195,475.6 -5,executorDeserializeCPUTime_mean,0.024855148,3.021986101857972,9.16652187755674,0.01225114854517611,1.256188578359779,1.8571428571428572,2.7277823524313076,198.8 -6,output_recordsWritten_sum,0.024716686,192.27266754270696,2101.0595138559515,0.0,0.0,0.0,0.0,28800.0 -7,resultSize_max,0.021557935,3873052.346911958,7059374.819815673,1785.0,25612.5,468118.0,3422351.0,40000929.0 -8,sqlOp_SubqueryBroadcast,0.018513301,0.6484888304862023,0.4775991378028838,0.0,0.0,1.0,1.0,1.0 -9,input_bytesRead_mean,0.016610738,8879875.459031757,17843172.786585648,3240.8,947851.6444805195,3648864.1525423727,8947659.156362318,134037865.09764309 -10,numTasks_sum,0.015820967,494.50197109067017,1315.5857192434262,3.0,46.0,117.0,382.0,19553.0 -11,peakExecutionMemory_max,0.015749473,229247556.33114323,530192271.8769413,0.0,36933160.0,73662432.0,236688530.0,11576276592.0 -12,duration_mean,0.014476735,1469.099084463667,11726.571027982141,13.333333333333334,208.49642857142857,390.50629611411705,838.6961608927381,228041.3076923077 -13,numExecutors,0.014305489,3.3501971090670173,3.2308043283469328,1.0,1.0,1.0,8.0,8.0 -14,duration_min,0.014195058,16.144546649145862,17.18389150847028,2.0,10.0,13.0,17.0,346.0 -15,executorRunTime_mean,0.01412763,1458.0987197246566,11725.289746335016,6.0,201.34299660441428,378.56819026384244,812.0453680203045,228020.38461538462 -16,sqlOp_ColumnarToRow,0.01320624,0.9500657030223391,0.2178808241607047,0.0,1.0,1.0,1.0,1.0 -17,sw_writeTime_mean,0.012652813,29.836925576628065,53.41213678505095,0.0,5.9423076923076925,14.709041866423789,31.539893617021278,605.7253684210526 -18,sr_localBytesRead_mean,0.012182485,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 -19,resultSerializationTime_sum,0.011836103,78.88107752956635,346.6953575168781,0.0,0.0,5.0,19.0,7044.0 -20,jvmGCTime_mean,0.011720132,30.5415756714109,114.21214002514562,0.0,1.4274553571428572,5.642595559624628,15.659427859444222,2226.977412731006 -21,sw_recordsWritten_sum,0.011120409,361907079.3869908,2306099322.474557,0.0,19991.75,2012328.0,53483121.25,43595020340.0 -22,data_size,0.008518888,52257369616.6682,212466894399.25977,0.0,245400479.0,2237123204.5,14651629460.0,2535668591063.0 -23,output_bytesWrittenRatio,0.008387904,1.134598720485971e-05,0.00016771251117901053,0.0,0.0,0.0,0.0,0.004454741188373785 -24,duration_sum,0.007974725,1088436.8134034167,7051805.333730643,40.0,14202.0,54338.5,229829.0,191642768.0 -25,sr_localBytesReadRatio,0.0076882048,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 -26,input_recordsRead_sum,0.007035825,787220690.6005256,3437802880.181593,0.0,7106795.0,43264876.0,227006845.0,58530144920.0 -27,sr_localBlocksFetched_sum,0.00608383,46232.10183968463,242956.6186599344,0.0,70.0,449.0,4365.25,3860026.0 -28,sw_bytesWritten_mean,0.005955132,3740772.892818494,8646461.545663768,0.0,9715.138598022106,422415.42541797285,3103227.323872458,104601898.28278689 -29,shuffle_write_bw,0.005811749,76999.10285888401,100166.58594610717,0.0,1309.6095385860765,27263.050268104173,121175.31189824673,477599.3461538461 -30,sqlOp_Sort,0.004056511,0.5860709592641261,0.4926979651899333,0.0,0.0,1.0,1.0,1.0 -31,sqlOp_Expand,0.0029860404,0.09132720105124836,0.2881685338728861,0.0,0.0,0.0,0.0,1.0 -32,sw_bytesWrittenRatio,0.0026960408,0.7348621806914661,1.364829122538853,0.0,0.004226587490302376,0.1700493197577822,1.0750713385977424,16.779651636748564 -33,executorCores,0.0021443053,11.455978975032851,1.7234032412556695,6.0,12.0,12.0,12.0,12.0 -34,sqlOp_SortMergeJoin,0.0017613693,0.4126149802890933,0.4924664490141788,0.0,0.0,0.0,1.0,1.0 -35,shuffle_read_bw,0.0016664306,178564748.48950323,2870761935.402956,0.0,0.0,0.0,0.0,102339190130.0 -36,sqlOp_DeserializeToObject,0.0016348368,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 -37,memoryBytesSpilled_mean,0.001183661,2254905.721932443,13425137.78628761,0.0,0.0,0.0,0.0,292579823.9844638 -38,sr_fetchWaitTime_mean,0.0009891957,0.00431489459719567,0.02398992217438527,0.0,0.0,0.0,0.0,0.38974358974358975 -39,output_bytesWritten_mean,0.00083561294,42.247080700329185,784.3960049968721,0.0,0.0,0.0,0.0,25987.92 -40,sqlOp_HashAggregate,0.00069161056,0.9494086727989488,0.21923371988645474,0.0,1.0,1.0,1.0,1.0 -41,sqlOp_ObjectHashAggregate,0.0006604873,0.09461235216819974,0.29277495411872867,0.0,0.0,0.0,0.0,1.0 -42,diskBytesSpilled_mean,0.0005853335,560139.0351193441,3888224.5701207644,0.0,0.0,0.0,0.0,57763330.03389474 -43,memoryBytesSpilledRatio,0.0005463429,0.28201941146050324,1.760522704410585,0.0,0.0,0.0,0.0,20.350496922781154 -44,sqlOp_Subquery,0.00038015703,0.15703022339027595,0.3639488449522727,0.0,0.0,0.0,0.0,1.0 -45,sqlOp_BroadcastNestedLoopJoin,0.0003075349,0.03942181340341656,0.19466029547643687,0.0,0.0,0.0,0.0,1.0 -46,sqlOp_Window,0.00026852122,0.11038107752956636,0.31346715319289287,0.0,0.0,0.0,0.0,1.0 -47,sqlOp_Project,0.000210538,0.9697766097240473,0.1712577213783184,0.0,1.0,1.0,1.0,1.0 -48,sqlOp_BatchEvalPython,0.00020995157,0.012483574244415242,0.11106682510080343,0.0,0.0,0.0,0.0,1.0 -49,diskBytesSpilledRatio,0.0002064515,0.03870446764927755,0.22755637119531616,0.0,0.0,0.0,0.0,2.6257828841329087 -50,sqlOp_BroadcastExchange,0.00020307579,0.897503285151117,0.3034000972146485,0.0,1.0,1.0,1.0,1.0 -51,sqlOp_TakeOrderedAndProject,0.00017823598,0.022996057818659658,0.14994002302202078,0.0,0.0,0.0,0.0,1.0 -52,sqlOp_Filter,0.00017069223,0.9697766097240473,0.1712577213783184,0.0,1.0,1.0,1.0,1.0 -53,sqlOp_BroadcastHashJoin,0.0001656123,0.892904073587385,0.3093368062389466,0.0,1.0,1.0,1.0,1.0 -54,sqlOp_AQEShuffleRead,0.00014509328,0.8731931668856767,0.3328658271954145,0.0,1.0,1.0,1.0,1.0 -55,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sr_remoteBlocksFetched_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_WindowGroupLimit,0.0,0.017082785808147174,0.12962254302936355,0.0,0.0,0.0,0.0,1.0 -59,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_MapElements,0.0,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 -65,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_SerializeFromObject,0.0,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 -68,sqlOp_Scan text ,0.0,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 -69,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.13009198423127463,0.3365151758705616,0.0,0.0,0.0,0.0,1.0 -79,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,duration_max,0.3195818,6139.4632063074905,32138.962204191022,15.0,560.0,1136.5,3113.0,515161.0 +1,scan_bw,0.17451431,1593713.4898735192,2294166.855457363,0.0,174981.0170488308,655006.879324951,2101650.4235552456,17385294.707427993 +2,executorCPUTime_mean,0.06829664,1343.4213374053284,11697.68424064774,3.875,163.23010869565218,315.8137480798771,686.9930131090191,227947.84615384616 +3,scan_time,0.026964596,46345.203022339025,243193.36454642916,0.0,473.5,2177.5,10481.0,3127801.0 +4,executorDeserializeTime_mean,0.025398523,6.00370659180027,19.71922884656477,0.04395604395604396,2.2201527924130664,3.6647940074906367,5.147844471947195,475.6 +5,input_bytesRead_mean,0.02465917,8879875.459031755,17843172.786585648,3240.8,947851.6444805195,3648864.1525423727,8947659.156362318,134037865.09764309 +6,executorDeserializeCPUTime_mean,0.024618084,3.021986101857972,9.166521877556738,0.01225114854517611,1.256188578359779,1.8571428571428572,2.7277823524313076,198.8 +7,resultSize_max,0.022027511,3873052.346911958,7059374.819815673,1785.0,25612.5,468118.0,3422351.0,40000929.0 +8,sw_recordsWritten_sum,0.020693975,361907079.3869908,2306099322.4745564,0.0,19991.75,2012328.0,53483121.25,43595020340.0 +9,sqlOp_ColumnarToRow,0.017980693,0.9500657030223391,0.21788082416070467,0.0,1.0,1.0,1.0,1.0 +10,duration_min,0.016652323,16.144546649145862,17.183891508470282,2.0,10.0,13.0,17.0,346.0 +11,duration_mean,0.01661974,1469.0990844636665,11726.571027982141,13.333333333333334,208.49642857142857,390.50629611411705,838.6961608927381,228041.3076923077 +12,output_recordsWritten_sum,0.01523192,192.27266754270696,2101.059513855952,0.0,0.0,0.0,0.0,28800.0 +13,numTasks_sum,0.014689089,494.50197109067017,1315.5857192434264,3.0,46.0,117.0,382.0,19553.0 +14,peakExecutionMemory_max,0.013778389,229247556.33114323,530192271.8769413,0.0,36933160.0,73662432.0,236688530.0,11576276592.0 +15,sqlOp_SubqueryBroadcast,0.013648172,0.6484888304862023,0.4775991378028838,0.0,0.0,1.0,1.0,1.0 +16,jvmGCTime_mean,0.012468436,30.5415756714109,114.21214002514562,0.0,1.4274553571428572,5.642595559624628,15.659427859444222,2226.977412731006 +17,sw_writeTime_mean,0.011537033,29.836925576628065,53.41213678505095,0.0,5.9423076923076925,14.709041866423789,31.539893617021278,605.7253684210526 +18,resultSerializationTime_sum,0.011318291,78.88107752956635,346.69535751687823,0.0,0.0,5.0,19.0,7044.0 +19,executorRunTime_mean,0.010152615,1458.0987197246566,11725.289746335016,6.0,201.34299660441428,378.56819026384244,812.0453680203045,228020.38461538462 +20,sr_localBytesRead_mean,0.009579074,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 +21,input_recordsRead_sum,0.00918951,787220690.6005256,3437802880.1815925,0.0,7106795.0,43264876.0,227006845.0,58530144920.0 +22,duration_sum,0.009162015,1088436.8134034167,7051805.333730643,40.0,14202.0,54338.5,229829.0,191642768.0 +23,numExecutors,0.009127689,3.3501971090670173,3.2308043283469328,1.0,1.0,1.0,8.0,8.0 +24,sr_localBlocksFetched_sum,0.008925544,46232.10183968463,242956.6186599344,0.0,70.0,449.0,4365.25,3860026.0 +25,data_size,0.008175789,52257369616.6682,212466894399.25977,0.0,245400479.0,2237123204.5,14651629460.0,2535668591063.0 +26,sr_localBytesReadRatio,0.008023082,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 +27,sqlOp_Sort,0.005908362,0.5860709592641261,0.4926979651899333,0.0,0.0,1.0,1.0,1.0 +28,shuffle_write_bw,0.0053474535,76999.10285888403,100166.58594610717,0.0,1309.6095385860765,27263.050268104173,121175.31189824673,477599.3461538461 +29,sw_bytesWritten_mean,0.004812041,3740772.892818494,8646461.545663768,0.0,9715.138598022106,422415.42541797285,3103227.323872458,104601898.28278689 +30,output_bytesWrittenRatio,0.003886018,1.1345987204859713e-05,0.00016771251117901053,0.0,0.0,0.0,0.0,0.004454741188373785 +31,sqlOp_SortMergeJoin,0.003122781,0.4126149802890933,0.4924664490141788,0.0,0.0,0.0,1.0,1.0 +32,sqlOp_Expand,0.0025946551,0.09132720105124836,0.2881685338728861,0.0,0.0,0.0,0.0,1.0 +33,executorCores,0.0024388253,11.455978975032851,1.7234032412556695,6.0,12.0,12.0,12.0,12.0 +34,sw_bytesWrittenRatio,0.0019104867,0.7348621806914659,1.3648291225388531,0.0,0.004226587490302376,0.1700493197577822,1.0750713385977424,16.779651636748564 +35,shuffle_read_bw,0.0016794031,178564748.4895032,2870761935.402956,0.0,0.0,0.0,0.0,102339190130.0 +36,output_bytesWritten_mean,0.0013638405,42.247080700329185,784.3960049968721,0.0,0.0,0.0,0.0,25987.92 +37,sqlOp_Window,0.0010913209,0.11038107752956636,0.31346715319289287,0.0,0.0,0.0,0.0,1.0 +38,memoryBytesSpilled_mean,0.0010049371,2254905.7219324433,13425137.78628761,0.0,0.0,0.0,0.0,292579823.9844638 +39,sr_fetchWaitTime_mean,0.00097604207,0.004314894597195671,0.023989922174385267,0.0,0.0,0.0,0.0,0.38974358974358975 +40,diskBytesSpilled_mean,0.0008968401,560139.0351193441,3888224.570120765,0.0,0.0,0.0,0.0,57763330.03389474 +41,memoryBytesSpilledRatio,0.0008119394,0.28201941146050324,1.7605227044105844,0.0,0.0,0.0,0.0,20.350496922781154 +42,sqlOp_TakeOrderedAndProject,0.0007714525,0.022996057818659658,0.14994002302202078,0.0,0.0,0.0,0.0,1.0 +43,sqlOp_BroadcastHashJoin,0.0005698804,0.892904073587385,0.3093368062389466,0.0,1.0,1.0,1.0,1.0 +44,sqlOp_Filter,0.00056024315,0.9697766097240473,0.17125772137831843,0.0,1.0,1.0,1.0,1.0 +45,sqlOp_BatchEvalPython,0.0005573549,0.012483574244415242,0.11106682510080343,0.0,0.0,0.0,0.0,1.0 +46,sqlOp_ObjectHashAggregate,0.0004553541,0.09461235216819974,0.2927749541187287,0.0,0.0,0.0,0.0,1.0 +47,sqlOp_AQEShuffleRead,0.0003563175,0.8731931668856767,0.3328658271954145,0.0,1.0,1.0,1.0,1.0 +48,sqlOp_Project,0.00035407313,0.9697766097240473,0.1712577213783184,0.0,1.0,1.0,1.0,1.0 +49,sqlOp_DeserializeToObject,0.0003214633,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 +50,sqlOp_Subquery,0.00031257345,0.15703022339027595,0.3639488449522727,0.0,0.0,0.0,0.0,1.0 +51,sqlOp_HashAggregate,0.00030874432,0.9494086727989488,0.21923371988645474,0.0,1.0,1.0,1.0,1.0 +52,diskBytesSpilledRatio,0.00013834362,0.03870446764927755,0.22755637119531616,0.0,0.0,0.0,0.0,2.6257828841329087 +53,sqlOp_WindowGroupLimit,0.00012092391,0.017082785808147174,0.12962254302936352,0.0,0.0,0.0,0.0,1.0 +54,sqlOp_BroadcastExchange,9.207968e-05,0.897503285151117,0.3034000972146485,0.0,1.0,1.0,1.0,1.0 +55,sqlOp_Exchange,7.649039e-05,0.9579500657030223,0.20076907391845303,0.0,1.0,1.0,1.0,1.0 +56,sqlOp_BroadcastNestedLoopJoin,6.7585526e-05,0.03942181340341656,0.19466029547643685,0.0,0.0,0.0,0.0,1.0 +57,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.13009198423127463,0.3365151758705616,0.0,0.0,0.0,0.0,1.0 +62,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_MapElements,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 +65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_SerializeFromObject,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 +69,sqlOp_Scan text ,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 +70,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 80,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -85,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,maxOnHeapMem,0.0,10042985900.30749,854967375.2687464,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 -88,maxMem,0.0,10042985900.30749,854967375.2687464,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 -89,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -91,sr_remoteBytesReadRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sr_totalBytesReadRatio,0.0,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 -94,sqlOp_Scan parquet ,0.0,0.9500657030223391,0.2178808241607047,0.0,1.0,1.0,1.0,1.0 -95,sqlOp_LocalTableScan,0.0,0.0019710906701708277,0.044367767973722515,0.0,0.0,0.0,0.0,1.0 -96,sr_remoteBytesRead_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sr_totalBytesRead_mean,0.0,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 -99,sqlOp_Exchange,0.0,0.9579500657030223,0.20076907391845303,0.0,1.0,1.0,1.0,1.0 -100,executorMemory,0.0,16262.896189224704,1358.9340452651124,1024.0,16384.0,16384.0,16384.0,16384.0 -101,platform_onprem,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -102,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sr_remoteBytesReadRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sr_remoteBlocksFetched_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_LocalTableScan,0.0,0.0019710906701708277,0.044367767973722515,0.0,0.0,0.0,0.0,1.0 +86,sqlOp_Scan parquet ,0.0,0.9500657030223391,0.21788082416070467,0.0,1.0,1.0,1.0,1.0 +87,maxMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 +88,maxOnHeapMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 +89,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,executorMemory,0.0,16262.896189224704,1358.9340452651124,1024.0,16384.0,16384.0,16384.0,16384.0 +91,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +94,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +98,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sr_totalBytesReadRatio,0.0,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 +100,sr_remoteBytesRead_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sr_totalBytesRead_mean,0.0,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 +103,platform_onprem,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +104,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 From fb725201e1f812d6b29d5400e484aa6b739bf259 Mon Sep 17 00:00:00 2001 From: Lee Yang Date: Tue, 3 Dec 2024 10:21:44 -0800 Subject: [PATCH 07/52] More flexible regexes; fix default split function (#1443) Signed-off-by: Lee Yang --- user_tools/src/spark_rapids_tools/tools/qualx/model.py | 2 +- .../src/spark_rapids_tools/tools/qualx/preprocess.py | 10 +++++++--- user_tools/src/spark_rapids_tools/tools/qualx/util.py | 6 +++--- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/model.py b/user_tools/src/spark_rapids_tools/tools/qualx/model.py index 9f9470a1f..4491545f2 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/model.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/model.py @@ -347,7 +347,7 @@ def extract_model_features( default_df = default_df.loc[~default_df.appName.str.startswith(f'{ds_name}:')] modified_default_df = default_split_fn(default_df) if modified_default_df.index.equals(default_df.index): - cpu_aug_tbl.update(default_df) + cpu_aug_tbl.update(modified_default_df) cpu_aug_tbl.astype(df_schema) else: raise ValueError('Default split_function unexpectedly modified row indices.') diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py index d203bbc7f..320615b7c 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py @@ -30,7 +30,9 @@ get_logger, get_dataset_platforms, load_plugin, - run_profiler_tool, log_fallback, + log_fallback, + run_profiler_tool, + RegexPattern ) PREPROCESSED_FILE = 'preprocessed.parquet' @@ -269,10 +271,12 @@ def infer_app_meta(eventlogs: List[str]) -> Mapping[str, Mapping]: app_meta_inner = {} for e in eventlog_list: parts = Path(e).parts - app_id_inner = parts[-1] + app_id_part = parts[-1] + match = RegexPattern.app_id.search(app_id_part) + app_id = match.group() if match else app_id_part run_type = parts[-2].upper() job_name = parts[-4] - app_meta_inner[app_id_inner] = { + app_meta_inner[app_id] = { 'jobName': job_name, 'runType': run_type, 'scaleFactor': 1, diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/util.py b/user_tools/src/spark_rapids_tools/tools/qualx/util.py index 831721ff3..9bb8ee1a5 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/util.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/util.py @@ -45,9 +45,9 @@ def get_logger(name: str) -> logging.Logger: @dataclass class RegexPattern: - app_id = re.compile(r'^app.*[_-][0-9]+[_-][0-9]+$') - profile = re.compile(r'^prof_[0-9]+_[0-9a-zA-Z]+$') - qual_tool = re.compile(r'^qual_[0-9]+_[0-9a-zA-Z]+$') + app_id = re.compile(r'app.*[_-][0-9]+[_-][0-9]+') + profile = re.compile(r'prof_[0-9]+_[0-9a-zA-Z]+') + qual_tool = re.compile(r'qual_[0-9]+_[0-9a-zA-Z]+') rapids_profile = re.compile(r'rapids_4_spark_profile') rapids_qual = re.compile(r'rapids_4_spark_qualification_output') qual_tool_metrics = re.compile(r'raw_metrics') From 0eb5bf51fe99b6d7de3eeb234b71c97ad335d957 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Tue, 3 Dec 2024 12:53:14 -0600 Subject: [PATCH 08/52] Fix missing exec-to-stageId mapping in Qual tool (#1437) * Fix missing exec-to-stageId mapping in Qual tool Fixes #1156 This adds logic to walk the SparkGraph in order to assign execs to stages. For nodes that have no AccumIDs, the clusterization processes relies on adjacent nodes. --------- Signed-off-by: Ahmed Hussein (amahussein) --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 4 +- .../tool/planparser/SQLPlanParser.scala | 27 +- .../planparser/WholeStageExecParser.scala | 24 +- .../photon/PhotonStageExecParser.scala | 5 +- .../rapids/tool/AccumToStageRetriever.scala | 31 ++ .../spark/sql/rapids/tool/AppBase.scala | 8 +- .../sql/rapids/tool/util/ToolsPlanGraph.scala | 371 +++++++++++++++++- .../complex_dec_expectation.csv | 2 +- .../read_dsv1_expectation.csv | 2 +- .../read_dsv2_expectation.csv | 2 +- .../truncated_1_end_expectation.csv | 2 +- .../tool/planparser/BasePlanParserSuite.scala | 23 ++ .../tool/planparser/SqlPlanParserSuite.scala | 79 +++- 13 files changed, 539 insertions(+), 41 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/sql/rapids/tool/AccumToStageRetriever.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 7e17767b9..9580aa470 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -18,7 +18,6 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} -import com.nvidia.spark.rapids.tool.planparser.SQLPlanParser import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} import com.nvidia.spark.rapids.tool.qualification.QualSQLPlanAnalyzer @@ -88,7 +87,8 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap // Maps stages to operators by checking for non-zero intersection // between nodeMetrics and stageAccumulateIDs val nodeIdToStage = planGraph.allNodes.map { node => - val mappedStages = SQLPlanParser.getStagesInSQLNode(node, app) + val nodeAccums = node.metrics.map(_.accumulatorId) + val mappedStages = app.getStageIDsFromAccumIds(nodeAccums) ((sqlId, node.id), mappedStages) }.toMap sqlPlanNodeIdToStageIds ++= nodeIdToStage diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index 1b8d63a5f..ad2634c41 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -26,7 +26,6 @@ import com.nvidia.spark.rapids.tool.qualification.PluginTypeChecker import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlanInfo -//import org.apache.spark.sql.execution.joins.CartesianProductExec import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, SparkPlanGraphNode} import org.apache.spark.sql.rapids.tool.{AppBase, BuildSide, ExecHelper, JoinType, RDDCheckHelper, ToolUtils, UnsupportedExpr} import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph @@ -433,22 +432,19 @@ object SQLPlanParser extends Logging { sqlDesc: String, checker: PluginTypeChecker, app: AppBase): PlanInfo = { - val planGraph = ToolsPlanGraph(planInfo) + val toolsGraph = ToolsPlanGraph.createGraphWithStageClusters(planInfo, app) + // Find all the node graphs that should be excluded and send it to the parsePlanNode - val excludedNodes = buildSkippedReusedNodesForPlan(planGraph) + val excludedNodes = buildSkippedReusedNodesForPlan(toolsGraph.sparkGraph) // we want the sub-graph nodes to be inside of the wholeStageCodeGen so use nodes // vs allNodes - val execInfos = planGraph.nodes.flatMap { node => - parsePlanNode(node, sqlID, checker, app, reusedNodeIds = excludedNodes) + val execInfos = toolsGraph.nodes.flatMap { node => + parsePlanNode(node, sqlID, checker, app, reusedNodeIds = excludedNodes, + nodeIdToStagesFunc = toolsGraph.getNodeStageClusters) } PlanInfo(appID, sqlID, sqlDesc, execInfos) } - def getStagesInSQLNode(node: SparkPlanGraphNode, app: AppBase): Set[Int] = { - val nodeAccums = node.metrics.map(_.accumulatorId) - nodeAccums.flatMap(app.accumManager.getAccStageIds).toSet - } - // Set containing execs that refers to other expressions. We need this to be a list to allow // appending more execs in teh future as necessary. // Note that Spark graph may create duplicate nodes when any of the following execs exists. @@ -541,7 +537,8 @@ object SQLPlanParser extends Logging { sqlID: Long, checker: PluginTypeChecker, app: AppBase, - reusedNodeIds: Set[Long] + reusedNodeIds: Set[Long], + nodeIdToStagesFunc: Long => Set[Int] ): Seq[ExecInfo] = { // Avoid counting duplicate nodes. We mark them as shouldRemove to neutralize their impact on // speedups. @@ -560,9 +557,11 @@ object SQLPlanParser extends Logging { // For WholeStageCodegen clusters, use PhotonStageExecParser if the cluster is of Photon type. // Else, fall back to WholeStageExecParser to parse the cluster. case photonCluster: PhotonSparkPlanGraphCluster => - PhotonStageExecParser(photonCluster, checker, sqlID, app, reusedNodeIds).parse + PhotonStageExecParser(photonCluster, checker, sqlID, app, reusedNodeIds, + nodeIdToStagesFunc = nodeIdToStagesFunc).parse case cluster: SparkPlanGraphCluster => - WholeStageExecParser(cluster, checker, sqlID, app, reusedNodeIds).parse + WholeStageExecParser(cluster, checker, sqlID, app, reusedNodeIds, + nodeIdToStagesFunc = nodeIdToStagesFunc).parse case _ => // For individual nodes, use PhotonPlanParser if the node is of Photon type. // Else, fall back to the Spark node parsing logic to parse the node. @@ -587,7 +586,7 @@ object SQLPlanParser extends Logging { ExecInfo(node, sqlID, normalizedNodeName, expr = "", 1, duration = None, node.id, isSupported = false, None) } - val stagesInNode = getStagesInSQLNode(node, app) + val stagesInNode = nodeIdToStagesFunc(node.id) execInfo.setStages(stagesInNode) // shouldRemove is set to true if the exec is a member of "execsToBeRemoved" or if the node // is a duplicate diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala index 19eb46a52..cdef3b225 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala @@ -27,7 +27,8 @@ abstract class WholeStageExecParserBase( checker: PluginTypeChecker, sqlID: Long, app: AppBase, - reusedNodeIds: Set[Long]) extends Logging { + reusedNodeIds: Set[Long], + nodeIdToStagesFunc: Long => Set[Int]) extends Logging { val fullExecName = "WholeStageCodegenExec" @@ -38,16 +39,15 @@ abstract class WholeStageExecParserBase( // Perhaps take the max of those in Stage? val accumId = node.metrics.find(_.name == "duration").map(_.accumulatorId) val maxDuration = SQLPlanParser.getTotalDuration(accumId, app) - val stagesInNode = SQLPlanParser.getStagesInSQLNode(node, app) + val stagesInNode = nodeIdToStagesFunc.apply(node.id) // We could skip the entire wholeStage if it is duplicate; but we will lose the information of // the children nodes. val isDupNode = reusedNodeIds.contains(node.id) val childNodes = node.nodes.flatMap { c => - SQLPlanParser.parsePlanNode(c, sqlID, checker, app, reusedNodeIds) + // Pass the nodeToStagesFunc to the child nodes so they can get the stages. + SQLPlanParser.parsePlanNode(c, sqlID, checker, app, reusedNodeIds, + nodeIdToStagesFunc = nodeIdToStagesFunc) } - // For the childNodes, we need to append the stages. Otherwise, nodes without metrics won't be - // assigned to stage - childNodes.foreach(_.appendToStages(stagesInNode)) // if any of the execs in WholeStageCodegen supported mark this entire thing as supported val anySupported = childNodes.exists(_.isSupported == true) val unSupportedExprsArray = @@ -55,14 +55,11 @@ abstract class WholeStageExecParserBase( // average speedup across the execs in the WholeStageCodegen for now val supportedChildren = childNodes.filterNot(_.shouldRemove) val avSpeedupFactor = SQLPlanParser.averageSpeedup(supportedChildren.map(_.speedupFactor)) - // can't rely on the wholeStagecodeGen having a stage if children do so aggregate them together - // for now - val allStagesIncludingChildren = childNodes.flatMap(_.stages).toSet ++ stagesInNode.toSet - // Finally, the node should be marked as shouldRemove when all the children of the + // The node should be marked as shouldRemove when all the children of the // wholeStageCodeGen are marked as shouldRemove. val removeNode = isDupNode || childNodes.forall(_.shouldRemove) val execInfo = ExecInfo(node, sqlID, node.name, node.name, avSpeedupFactor, maxDuration, - node.id, anySupported, Some(childNodes), allStagesIncludingChildren, + node.id, anySupported, Some(childNodes), stagesInNode, shouldRemove = removeNode, unsupportedExprs = unSupportedExprsArray) Seq(execInfo) } @@ -73,5 +70,6 @@ case class WholeStageExecParser( checker: PluginTypeChecker, sqlID: Long, app: AppBase, - reusedNodeIds: Set[Long]) - extends WholeStageExecParserBase(node, checker, sqlID, app, reusedNodeIds) + reusedNodeIds: Set[Long], + nodeIdToStagesFunc: Long => Set[Int]) + extends WholeStageExecParserBase(node, checker, sqlID, app, reusedNodeIds, nodeIdToStagesFunc) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/photon/PhotonStageExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/photon/PhotonStageExecParser.scala index d49192d27..ceb59e712 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/photon/PhotonStageExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/photon/PhotonStageExecParser.scala @@ -35,5 +35,6 @@ case class PhotonStageExecParser( checker: PluginTypeChecker, sqlID: Long, app: AppBase, - reusedNodeIds: Set[Long]) - extends WholeStageExecParserBase(node, checker, sqlID, app, reusedNodeIds) + reusedNodeIds: Set[Long], + nodeIdToStagesFunc: Long => Set[Int]) + extends WholeStageExecParserBase(node, checker, sqlID, app, reusedNodeIds, nodeIdToStagesFunc) diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AccumToStageRetriever.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AccumToStageRetriever.scala new file mode 100644 index 000000000..7da118a0e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AccumToStageRetriever.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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.spark.sql.rapids.tool + +/** + * Trait that defines the interface for retrieving stage IDs from accumulables. + * This is used to map accumulables to stages. We use it as interface in order to allow to separate + * the logic and use dummy different implementations and mocks for testing when needed. + */ +trait AccumToStageRetriever { + /** + * Given a sequence of accumIds, return a set of stage IDs that are associated with the + * accumIds. Note that this method can only be called after the accumulables have been fully + * processed. + */ + def getStageIDsFromAccumIds(accumIds: Seq[Long]): Set[Int] +} diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index e3313b832..f8d4d7703 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -42,7 +42,9 @@ import org.apache.spark.util.Utils abstract class AppBase( val eventLogInfo: Option[EventLogInfo], - val hadoopConf: Option[Configuration]) extends Logging with ClusterTagPropHandler { + val hadoopConf: Option[Configuration]) extends Logging + with ClusterTagPropHandler + with AccumToStageRetriever { var appMetaData: Option[AppMetaData] = None @@ -105,6 +107,10 @@ abstract class AppBase( def sqlPlans: immutable.Map[Long, SparkPlanInfo] = sqlManager.getPlanInfos + def getStageIDsFromAccumIds(accumIds: Seq[Long]): Set[Int] = { + accumIds.flatMap(accumManager.getAccStageIds).toSet + } + // Returns the String value of the eventlog or empty if it is not defined. Note that the eventlog // won't be defined for running applications def getEventLogPath: String = { diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/ToolsPlanGraph.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/ToolsPlanGraph.scala index 6d6187d80..c9a1f1d3f 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/ToolsPlanGraph.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/ToolsPlanGraph.scala @@ -16,16 +16,375 @@ package org.apache.spark.sql.rapids.tool.util -import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper import java.util.concurrent.atomic.AtomicLong + import scala.collection.mutable +import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper + import org.apache.spark.sql.execution.SparkPlanInfo -import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, SparkPlanGraphEdge, SparkPlanGraphNode, SQLPlanMetric} +import org.apache.spark.sql.execution.ui._ +import org.apache.spark.sql.rapids.tool.AccumToStageRetriever import org.apache.spark.sql.rapids.tool.store.AccumNameRef import org.apache.spark.sql.rapids.tool.util.plangraph.PlanGraphTransformer import org.apache.spark.sql.rapids.tool.util.stubs.{GraphReflectionAPI, GraphReflectionAPIHelper} +/** + * A wrapper of the original SparkPlanGraph with additional information about the + * node-to-stage mapping. + * 1- The graph is constructed by visiting PlanInfos and creating GraphNodes and Edges. + * Although it is more efficient to assign stages during the construction of the nodes, + * the design is intentionally keeping those two phases separate to make the code more modular + * and easier to maintain. + * 2- Traverse the nodes and assign them to stages based on the metrics. + * 3- Nodes that belong to a graph cluster (childs of WholeStageCodeGen) while missing + * metrics, are assigned same as their WholeStageCodeGen node. + * 4- Iterate on all the orphanNodes and assign them to stages based on their adjacents nodes. + * 5- The iterative process is repeated until no assignment can be made. + * + * @param sparkGraph the original SparkPlanGraph to wrap + * @param accumToStageRetriever The object that eventually can retrieve StageIDs from AccumIds. + */ +class ToolsPlanGraph(val sparkGraph: SparkPlanGraph, + accumToStageRetriever: AccumToStageRetriever) { + // A map between SQLNode Id and the clusterIds that the node belongs to. + // Here, a clusterId means a stageId. + // Note: It is possible to represent the clusters as map [clusterId, Set[SQLNodeIds]]. + // While this is more memory efficient, it is more time-consuming to find the clusters a + // node belongs to since we have to iterate through all the keys. + private val nodeToStageCluster: mutable.Map[Long, Set[Int]] = mutable.HashMap[Long, Set[Int]]() + // shortcut to the nodes + def nodes: collection.Seq[SparkPlanGraphNode] = sparkGraph.nodes + // shortcut to the edges + def edges: collection.Seq[SparkPlanGraphEdge] = sparkGraph.edges + // delegate the call to the original graph + def allNodes: collection.Seq[SparkPlanGraphNode] = sparkGraph.allNodes + + /** + * Get stages that are associated with the accumulators of the node. + * Use this method if the purpose is to get raw information about the node-stage relationship + * based on the AccumIds without applying any logic. + * @param node the node to get the stages for + * @return a set of stageIds or empty if None + */ + private def getNodeStagesByAccum(node: SparkPlanGraphNode): Set[Int] = { + val nodeAccums = node.metrics.map(_.accumulatorId) + accumToStageRetriever.getStageIDsFromAccumIds(nodeAccums) + } + + /** + * Get the stages that the node belongs to. This function is used to get all the stages that can + * be assigned to a node. For example, if we want to get the "Exchange" node stages, then we call + * that method. + * @param node the node to get the stages for + * @return a set of stageIds or empty if None + */ + def getAllNodeStages(node: SparkPlanGraphNode): Set[Int] = { + val stageIdsByAccum = getNodeStagesByAccum(node) + nodeToStageCluster.get(node.id) match { + case Some(stageId) => stageIdsByAccum ++ stageId + case _ => stageIdsByAccum + } + } + + /** + * Check if a node exec is an epilogue. A.k.a, the exec has to be the tail of a stage. + * @param nodeName normalized node name (i.e., no GPU prefix) + * @return true if the node is an epilogue exec + */ + private def isEpilogueExec(nodeName: String): Boolean = { + nodeName match { + case "Exchange" | "BroadcastQueryStage" | "ShuffleQueryStage" | "TableCacheQueryStage" + | "ResultQueryStage" | "BroadcastExchange" => + true + case _ => false + } + } + + /** + * Check if a node exec is a prologue. A.k.a, the exec has to be the head of a stage. + * @param nodeName normalized node name (i.e., no GPU prefix) + * @return true if the node is a prologue exec + */ + private def isPrologueExec(nodeName: String): Boolean = { + nodeName match { + case nName if nName.contains("ShuffleRead") => + true + case _ => false + } + } + + /** + * Given a nodeName, this method returns a code that represents the node type. + * For example, an exchange node has to be at the end of a stage. ShuffleRead has to be at the + * beginning of a stage and so. + * + * @param nodeName the normalized name of the sparkNode (i.e., no GPU prefix). + * @return a code representing thenode type: + * (1) if the node can be assigned based on incoming edges. i.e., all nodes except the + * head of stage like shuffleRead. + * (2) if the node can be assigned based on outgoing edges. i.e., + * all nodes except the tail of stage like shuffleWrite/exchange. + * (3) if the node can be assigned based on both incoming and outgoing edges. + */ + private def multiplexCases(nodeName: String): Int = { + // nodes like shuffleRead should not be assigned to incoming edges + var result = 0 + if (!isPrologueExec(nodeName)) { + // Those are the nodes that can be assigned based on incoming edges. + result |= 1 + } + if (!isEpilogueExec(nodeName)) { + // Those are the nodes that can be assigned based on outgoing edges. + result |= 2 + } + result + } + + /** + * This method is used to assign a node to clusterID during the first walk of the graph. + * A cluster is used to wrap nodes together this could be a stageId. + * @param node the sparkNode to assign + * @return the clusterId that the node belongs to + */ + private def populateNodeClusters(node: SparkPlanGraphNode): Set[Int] = { + // First normalize the name. + val normalizedName = ToolsPlanGraph.processPlanInfo(node.name) + val stageIds = getNodeStagesByAccum(node) + normalizedName match { + case nName if isEpilogueExec(nName) => + // Cases that are tail of the stage cluster. + if (stageIds.size <= 1) { + stageIds + } else { + // Only use the smallest StageId because this would represent the stage that writes + // the data. + Set[Int](stageIds.min) + } + case nName if isPrologueExec(nName) => + // Cases that are head of a new stage. + if (stageIds.size <= 1) { + ToolsPlanGraph.EMPTY_CLUSTERS + } else { + // We should pick the stages associated with the reading metrics. This is likely to be + // the stage with the highest ID value. + Set[Int](stageIds.max) + } + case _ => + // Everything else goes here. + // It is possible to have multiple stages for a given node. + stageIds + } + } + + /** + * Updates the data structure that keeps track of the nodes cluster assignment. + * It adds the node to the map and remove the node from the orphans list if it exists. + * @param node the node to be assigned. + * @param orphanNodes the list of nodes that are not assigned to any cluster. + * @param clusters the clusterIds to assign the node to + */ + private def removeNodeFromOrphans(node: SparkPlanGraphNode, + orphanNodes: mutable.ArrayBuffer[SparkPlanGraphNode], + clusters: Set[Int]): Unit = { + nodeToStageCluster.put(node.id, clusters) + orphanNodes -= node + } + + /** + * Commits a wholeStageNode to a cluster. + * A WholeStageNode is visited after its children are. If any of the children is not assigned to + * a cluster, the wNode will transfer its assignment to the child. + * @param wNode the wholeStageCodeGen node to be visited + * @param orphanNodes the list of nodes that are not assigned to any cluster. + * @param clusters the clusterId to assign the node to. + * @return true if a change is made. + */ + private def commitNodeToStageCluster( + wNode: SparkPlanGraphCluster, + orphanNodes: mutable.ArrayBuffer[SparkPlanGraphNode], + clusters: Set[Int]): Boolean = { + if (nodeToStageCluster.contains(wNode.id) && clusters.subsetOf(nodeToStageCluster(wNode.id))) { + // Nothing to do since the node is assigned to the same cluster before. + false + } else { + val newClusterIds = + clusters ++ nodeToStageCluster.getOrElse(wNode.id, ToolsPlanGraph.EMPTY_CLUSTERS) + // Remove the wNode from orphanNodes if it exists + removeNodeFromOrphans(wNode, orphanNodes, newClusterIds) + // Assign the children to the same clusters if any of them is not assigned already. + wNode.nodes.foreach { childNode => + if (!nodeToStageCluster.contains(childNode.id)) { + // Assign the child node to the same stage of wNode and remove it from orphans + removeNodeFromOrphans(childNode, orphanNodes, newClusterIds) + } + } + true + } + } + + /** + * Assign a node to a clusterId. This method is used to assign a node to a clusterId during the + * first visit. + * @param node sparkNode to be assigned + * @param orphanNodes the list of nodes that are not assigned to any cluster + * @param clusters the clusterIds to assign the node to + * @return true if a change is made. + */ + private def commitNodeToStageCluster( + node: SparkPlanGraphNode, + orphanNodes: mutable.ArrayBuffer[SparkPlanGraphNode], + clusters: Set[Int]): Boolean = { + node match { + case cluster: SparkPlanGraphCluster => + // WholeCodeGen represents a special case because it propagates its assignment to + // children nodes. + commitNodeToStageCluster(cluster, orphanNodes, clusters) + case _ => + removeNodeFromOrphans(node, orphanNodes, clusters) + true + } + } + + /** + * Walk through the graph nodes and assign them to the correct stage cluster. + */ + protected def assignNodesToStageClusters(): Unit = { + // Keep track of nodes that have no assignment to any cluster. + val orphanNodes = mutable.ArrayBuffer[SparkPlanGraphNode]() + // Step(1): Visit all the nodes and assign them to the correct cluster based on AccumIDs. + // In the process, WholeStageCodeGens propagate their assignment to the child nodes if + // they are orphans. + allNodes.foreach { node => + if (!nodeToStageCluster.contains(node.id)) { + // Get clusterIDs based on AccumIds + val clusterIds = populateNodeClusters(node) + if (clusterIds.nonEmpty) { + // Found assignment + commitNodeToStageCluster(node, orphanNodes, clusterIds) + } else { + // This node has no assignment. Add it to the orphanNodes + orphanNodes += node + } + } + } + // Step(2): At this point, we made a quick visit handling all the straightforward cases. + // Iterate on the orphanNodes and try to assign them based on the adjacent nodes. + var changeFlag = orphanNodes.nonEmpty + while (changeFlag) { + // Iterate on the orphanNodes and try to assign them based on the adjacent nodes until no + // changes can be done in a single iteration. + changeFlag = false + // P.S: Copy the orphanNodes because we cannot remove objects inside the loop. + val orphanNodesCopy = orphanNodes.clone() + orphanNodesCopy.foreach { currNode => + if (orphanNodes.contains(currNode)) { // Avoid dup processing caused by wholeStageCodeGen + val currNodeName = ToolsPlanGraph.processPlanInfo(currNode.name) + val updatedFlag = currNode match { + case wNode: SparkPlanGraphCluster => + // WholeStageCodeGen is a corner case because it is not connected by edges. + // The only way to set the clusterID is to get it from the children if any. + wNode.nodes.find { childNode => nodeToStageCluster.contains(childNode.id) } match { + case Some(childNode) => + val clusterIDs = nodeToStageCluster(childNode.id) + commitNodeToStageCluster(wNode, orphanNodes, clusterIDs) + case _ => // do nothing if we could not find a child node with a clusterId + false + } + case _ => + // Handle all other nodes. + // Set the node type to determine the restrictions (i.e., exchange is + // positioned at the tail of a stage and shuffleRead should be the head of a stage). + val nodeCase = multiplexCases(currNodeName) + var clusterIDs = ToolsPlanGraph.EMPTY_CLUSTERS + if ((nodeCase & 1) > 0) { + // Assign cluster based on incoming edges. + val inEdgesWithIds = + edges.filter(e => e.toId == currNode.id && nodeToStageCluster.contains(e.fromId)) + if (inEdgesWithIds.nonEmpty) { + // For simplicity, assign the node based on the first incoming adjacent node. + clusterIDs = nodeToStageCluster(inEdgesWithIds.head.fromId) + } + } + if (clusterIDs.isEmpty && (nodeCase & 2) > 0) { + // Assign cluster based on outgoing edges (i.e., ShuffleRead). + // Corner case: TPC-DS Like Bench q2 (sqlID 24). + // A shuffleReader is reading on driver followed by an exchange without + // metrics. + // The metrics will not have a valid accumID. + // In that case, it is not feasible to match it to a cluster without + // considering the incoming node (exchange in that case). This corner + // case is handled later as a last-ditch effort. + val outEdgesWithIds = + edges.filter(e => e.fromId == currNode.id && nodeToStageCluster.contains(e.toId)) + if (outEdgesWithIds.nonEmpty) { + // For simplicity, assign the node based on the first outgoing adjacent node. + clusterIDs = nodeToStageCluster(outEdgesWithIds.head.toId) + } + } + if (clusterIDs.nonEmpty) { + // There is a possible assignment. Commit it. + commitNodeToStageCluster(currNode, orphanNodes, clusterIDs) + } else { + // nothing has changed + false + } + } // End of setting the UpdatedFlag variable. + changeFlag |= updatedFlag + } // End of if orphanNodes.contains(currNode). + } // End of iteration on orphanNodes. + // Corner case for shuffleRead when it is reading from the driver followed by an exchange that + // has no metrics. + if (!changeFlag && orphanNodes.nonEmpty) { + // This is to handle the special case of a shuffleRead that is reading from the driver. + // We could not assign any node to a cluster. This means that we have a cycle in the graph, + // and we need to break it. + // This is done by breaking the rule, allowing the shuffleRead to pick the highest stage + // order of the ancestor node. + changeFlag |= orphanNodes.filter( + n => isPrologueExec(ToolsPlanGraph.processPlanInfo(n.name))).exists { // Picks shuffleRead + orphanNode => + // Get adjacent nodes to the shuffleRead that have cluster assignment. + val inEdgesWithIds = + edges.filter(e => e.toId == orphanNode.id && nodeToStageCluster.contains(e.fromId)) + if (inEdgesWithIds.nonEmpty) { + // At this point, we need to get all the possible stageIDs that can be assigned to the + // adjacent nodes because and not only the logical ones. + val possibleIds = inEdgesWithIds.map { e => + val adjacentNode = allNodes.find(eN => eN.id == e.fromId).get + getAllNodeStages(adjacentNode) + }.reduce(_ ++ _) + // Assign the maximum value clusterId to the node. + val newIDs = Set[Int](possibleIds.max) + commitNodeToStageCluster(orphanNode, orphanNodes, newIDs) + } else { + false + } + } + } // end of corner case handling + } // end of changeFlag loop + } // end of assignNodesToStageClusters + // Start the construction of the graph + assignNodesToStageClusters() + + // Define public interface methods + /** + * Get the stage clusters that the node belongs to. + * Use this method if this logical representation of the node-to-stage relationship. + * For example, an "Exchange" node returns only a single stageID which is the stage that writes + * the data. + * @param node the node to get the stages for + * @return a set of stageIds or empty if None + */ + def getNodeStageClusters(node: SparkPlanGraphNode): Set[Int] = { + nodeToStageCluster.getOrElse(node.id, ToolsPlanGraph.EMPTY_CLUSTERS) + } + + def getNodeStageClusters(nodeId: Long): Set[Int] = { + nodeToStageCluster.getOrElse(nodeId, ToolsPlanGraph.EMPTY_CLUSTERS) + } +} // end of class ToolsPlanGraph + /** * This code is mostly copied from org.apache.spark.sql.execution.ui.SparkPlanGraph * with changes to handle GPU nodes. Without this special handle, the default SparkPlanGraph @@ -34,6 +393,8 @@ import org.apache.spark.sql.rapids.tool.util.stubs.{GraphReflectionAPI, GraphRef * Build a SparkPlanGraph from the root of a SparkPlan tree. */ object ToolsPlanGraph { + // Empty cluster set used to represent a node that is not assigned to any cluster. + private val EMPTY_CLUSTERS: Set[Int] = Set.empty // Captures the API loaded at runtime if any. var api: GraphReflectionAPI = _ @@ -89,6 +450,12 @@ object ToolsPlanGraph { } } + def createGraphWithStageClusters(planInfo: SparkPlanInfo, + accumStageMapper: AccumToStageRetriever): ToolsPlanGraph = { + val sGraph = ToolsPlanGraph(planInfo) + new ToolsPlanGraph(sGraph, accumStageMapper) + } + private def processPlanInfo(nodeName: String): String = { if (nodeName.startsWith("Gpu")) { nodeName.replaceFirst("Gpu", "") diff --git a/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv b/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv index f76c554be..8a9e1f666 100644 --- a/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly), Total Core Seconds -"Spark shell","local-1626104300434",1500,1469,131104,1315,88.35,"","","","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map;map>;map>;array>;array","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map>;map>;array>","NESTED COMPLEX TYPE",1260,1388,129598,181,1288,false,"CollectLimit","",30,1564 +"Spark shell","local-1626104300434",1500,1469,131104,996,88.35,"","","","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map;map>;map>;array>;array","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map>;map>;array>","NESTED COMPLEX TYPE",1260,1388,129598,493,976,false,"CollectLimit","",30,1564 diff --git a/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv b/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv index 4cb0d9940..b725defc0 100644 --- a/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1624371544219",4575,20421,175293,1557,72.15,"","JSON[string:double:date:int:bigint];Text[*]","JSON","","","",1859,5372,176916,13469,6952,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json;Scan json","",30,2096 +"Spark shell","local-1624371544219",4575,20421,175293,1523,72.15,"","JSON[string:double:date:int:bigint];Text[*]","JSON","","","",1859,5372,176916,13622,6799,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json;Scan json","",30,2096 diff --git a/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv b/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv index 404c02755..4c7726207 100644 --- a/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1624371906627",4917,21802,83738,1745,71.3,"","Text[*];json[double]","JSON","","","",1984,5438,83336,14064,7738,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json;BatchScan json","",30,997 +"Spark shell","local-1624371906627",4917,21802,83738,2687,71.3,"","Text[*];json[double]","JSON","","","",1984,5438,83336,9889,11913,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json;BatchScan json","",30,997 diff --git a/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv b/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv index 6ef5acd56..f7737d508 100644 --- a/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",395,14353,4872,142,62.67,"","","JSON","","","",1306,794,4477,9164,5189,true,"SerializeFromObject;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements;Scan","",30,49 +"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",395,14353,4872,164,62.67,"","","JSON","","","",1306,794,4477,8376,5977,true,"SerializeFromObject;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements;Scan","",30,49 diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala index 54cad1b1c..b7966d4d2 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala @@ -80,4 +80,27 @@ class BasePlanParserSuite extends BaseTestSuite { e.children.getOrElse(Seq.empty) :+ e } } + def verifyPlanExecToStageMap(toolsPlanInfo: PlanInfo): Unit = { + val allExecInfos = toolsPlanInfo.execInfo.flatMap { e => + e.children.getOrElse(Seq.empty) :+ e + } + // Test that all execs are assigned to stages + assert (allExecInfos.forall(_.stages.nonEmpty)) + // assert that exchange is assigned to a single stage + val exchangeExecs = allExecInfos.filter(_.exec == "Exchange") + if (exchangeExecs.nonEmpty) { + assert (exchangeExecs.forall(_.stages.size == 1)) + } + } + + def verifyExecToStageMapping(plans: Seq[PlanInfo], + qualApp: QualificationAppInfo, funcCB: Option[PlanInfo => Unit] = None): Unit = { + // Only iterate on plans with that are associated to jobs + val associatedSqls = qualApp.jobIdToSqlID.values.toSeq + val filteredPlans = plans.filter(p => associatedSqls.contains(p.sqlID)) + val func = funcCB.getOrElse(verifyPlanExecToStageMap(_)) + filteredPlans.foreach { plan => + func(plan) + } + } } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index a3d6211bf..6fc9357cf 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -109,7 +109,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { app.sqlPlans.foreach { case (sqlID, plan) => val planInfo = SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - + verifyPlanExecToStageMap(planInfo) val wholeStages = planInfo.execInfo.filter(_.exec.contains("WholeStageCodegen")) val allChildren = wholeStages.flatMap(_.children).flatten val sorts = allChildren.filter(_.exec == "Sort") @@ -205,6 +205,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { app.sqlPlans.foreach { case (sqlID, plan) => val planInfo = SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) + verifyPlanExecToStageMap(planInfo) val allExecInfo = planInfo.execInfo // Note that: // Spark320+ will generate the following execs: @@ -247,6 +248,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { app.sqlPlans.foreach { case (sqlID, plan) => val planInfo = SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) + verifyPlanExecToStageMap(planInfo) val wholeStages = planInfo.execInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 2) val numSupported = wholeStages.filter(_.isSupported).size @@ -267,6 +269,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val json = allExecInfo.filter(_.exec.contains("Scan json")) val orc = allExecInfo.filter(_.exec.contains("Scan orc")) @@ -289,6 +292,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) // Note that the text scan from this file is v1 so ignore it val json = allExecInfo.filter(_.exec.contains("BatchScan json")) @@ -319,6 +323,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val subqueryExecs = allExecInfo.filter(_.exec.contains(s"Subquery")) val summaryRecs = subqueryExecs.flatten { sqExec => @@ -354,6 +359,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val writeExecs = allExecInfo.filter(_.exec.contains(s"$dataWriteCMD")) val text = writeExecs.filter(_.expr.contains("text")) @@ -393,6 +399,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val writeExecs = allExecInfo.filter(_.exec.contains(s"$dataWriteCMD")) val text = writeExecs.filter(_.expr.contains("text")) @@ -425,6 +432,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val parquet = { allExecInfo.filter(_.exec.contains("CreateDataSourceTableAsSelectCommand")) @@ -454,6 +462,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val tableScan = allExecInfo.filter(_.exec == ("InMemoryTableScan")) assertSizeAndSupported(1, tableScan.toSeq) @@ -468,6 +477,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val broadcasts = allExecInfo.filter(_.exec == "BroadcastExchange") assertSizeAndSupported(3, broadcasts.toSeq, @@ -488,6 +498,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) // reusedExchange is added as a supportedExec val reusedExchangeExecs = allExecInfo.filter(_.exec == "ReusedExchange") @@ -509,6 +520,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val reader = allExecInfo.filter(_.exec == "CustomShuffleReader") assertSizeAndSupported(2, reader.toSeq) @@ -523,6 +535,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val reader = allExecInfo.filter(_.exec == "AQEShuffleRead") assertSizeAndSupported(2, reader.toSeq) @@ -552,15 +565,17 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) for (execName <- supportedExecs) { val execs = allExecInfo.filter(_.exec == execName) - assertSizeAndSupported(1, execs.toSeq, expectedDur = Seq.empty, extraText = execName) + assertSizeAndSupported(1, execs, expectedDur = Seq.empty, extraText = execName) } for (execName <- unsupportedExecs) { val execs = allExecInfo.filter(_.exec == execName) - assertSizeAndNotSupported(1, execs.toSeq) + assertSizeAndNotSupported(1, execs) } + verifyExecToStageMapping(parsedPlans.toSeq, app) } } @@ -581,6 +596,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) for (execName <- supportedExecs) { val supportedExec = allExecInfo.filter(_.exec == execName) @@ -617,6 +633,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val bhj = allExecInfo.filter(_.exec == "BroadcastHashJoin") assertSizeAndSupported(1, bhj) @@ -659,6 +676,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val bhj = allExecInfo.filter(_.exec == "BroadcastHashJoin") assertSizeAndNotSupported(1, bhj) @@ -692,6 +710,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val smj = allExecInfo.filter(_.exec == "SortMergeJoin") assertSizeAndNotSupported(1, smj) @@ -714,6 +733,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val sortAggregate = execInfo.filter(_.exec == "SortAggregate") assertSizeAndSupported(2, sortAggregate) @@ -734,6 +754,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val objectHashAggregate = execInfo.filter(_.exec == "ObjectHashAggregate") // OHA will get sql time metrics "time in aggregation build" @@ -760,6 +781,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val windowExecs = execInfo.filter(_.exec == "Window") assertSizeAndSupported(1, windowExecs) @@ -775,6 +797,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val flatMapGroups = allExecInfo.filter(_.exec == "FlatMapGroupsInPandas") assertSizeAndSupported(1, flatMapGroups) @@ -800,6 +823,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val supportedExecs = Array("GlobalLimit", "LocalLimit") app.sqlPlans.foreach { case (sqlID, plan) => val planInfo = SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) + verifyPlanExecToStageMap(planInfo) // GlobalLimit and LocalLimit are inside WholeStageCodegen. So getting the children of // WholeStageCodegenExec val wholeStages = planInfo.execInfo.filter(_.exec.contains("WholeStageCodegen")) @@ -825,6 +849,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { app.sqlPlans.foreach { case (sqlID, plan) => val planInfo = SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) + verifyPlanExecToStageMap(planInfo) val wholeStages = planInfo.execInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) val allChildren = wholeStages.flatMap(_.children).flatten @@ -849,6 +874,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { app.sqlPlans.foreach { case (sqlID, plan) => val planInfo = SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) + verifyPlanExecToStageMap(planInfo) val wholeStages = planInfo.execInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 2) val allChildren = wholeStages.flatMap(_.children).flatten @@ -873,6 +899,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val takeOrderedAndProject = execInfo.filter(_.exec == "TakeOrderedAndProject") assertSizeAndNotSupported(1, takeOrderedAndProject) @@ -897,6 +924,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val generateExprs = execInfo.filter(_.exec == "Generate") assertSizeAndSupported(1, generateExprs) @@ -924,6 +952,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val projectExprs = execInfo.filter(_.exec == "Project") assertSizeAndSupported(1, projectExprs) @@ -948,6 +977,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val sortAggregate = execInfo.filter(_.exec == "SortAggregate") assertSizeAndSupported(2, sortAggregate) @@ -974,6 +1004,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val sortExec = allExecInfo.filter(_.exec.contains("Sort")) assert(sortExec.size == 3) @@ -1001,6 +1032,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) @@ -1029,6 +1061,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "test desc", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) parsedPlans.foreach { pInfo => assert(pInfo.sqlDesc == "test desc") } @@ -1061,6 +1094,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) @@ -1091,6 +1125,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) @@ -1121,6 +1156,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) @@ -1183,6 +1219,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val projects = allExecInfo.filter(_.exec.contains("Project")) assertSizeAndNotSupported(1, projects) @@ -1211,6 +1248,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) @@ -1239,6 +1277,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val execInfo = getAllExecsFromPlan(parsedPlans.toSeq) val hashAggregate = execInfo.filter(_.exec == "HashAggregate") assertSizeAndSupported(2, hashAggregate, checkDurations = false) @@ -1265,6 +1304,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) assert(wholeStages.size == 1) @@ -1344,6 +1384,11 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + // TODO: Spark shows a weird behavior as the entire plan with SQL 65 loses the metrics + // and the associated jobs. This causes some execs to be isolated without stage. + // The UI won't be able to visualize that job anymore. So, we need to investigate + // what happens in that query before we test the execs-to-stage mapping. + val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val deltaLakeWrites = allExecInfo.filter(_.exec.contains(s"$dataWriteCMD")) assertSizeAndSupported(1, deltaLakeWrites) @@ -1406,6 +1451,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) // The promote_precision should be part of the project exec. val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val projExecs = allExecInfo.filter(_.exec.contains("Project")) @@ -1449,6 +1495,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) // The current_database should be part of the project-exec and the parser should ignore it. val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val projExecs = allExecInfo.filter(_.exec.contains("Project")) @@ -1697,6 +1744,18 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + // Note that the generated plan, there are skipped stages that causes some execs to appear + // without their relevant stages. so we skip the stage verification here. + verifyExecToStageMapping(parsedPlans.toSeq, app, Some( planInfo => + if (planInfo.sqlID == 73) { + // Nodes should not have any stages + val allExecInfos = planInfo.execInfo.flatMap { e => + e.children.getOrElse(Seq.empty) :+ e + } + // exclude all stages higher than 8 because those ones belong to a skipped stage + allExecInfos.filter(_.nodeId <= 8).forall(_.stages.nonEmpty) + }) + ) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val windowGroupLimitExecs = allExecInfo.filter(_.exec.contains(windowGroupLimitExecCmd)) // We should have two WindowGroupLimitExec operators (Partial and Final). @@ -1728,6 +1787,18 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + // Note that the generated plan, there are skipped stages that causes some execs to appear + // without their relevant stages. so we skip the stage verification here. + verifyExecToStageMapping(parsedPlans.toSeq, app, Some( planInfo => + if (planInfo.sqlID == 76) { + // Nodes should not have any stages + val allExecInfos = planInfo.execInfo.flatMap { e => + e.children.getOrElse(Seq.empty) :+ e + } + // exclude all stages higher than 8 because those ones belong to a skipped stage + allExecInfos.filter(_.nodeId <= 8).forall(_.stages.nonEmpty) + }) + ) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val windowExecNotSupportedExprs = allExecInfo.filter( _.exec.contains(windowGroupLimitExecCmd)).flatMap(x => x.unsupportedExprs) @@ -1793,6 +1864,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) // we should have 2 hash aggregates with min_by and max_by expressions // if the min_by and max_by were not recognized, the test would fail val hashAggExecs = @@ -1822,6 +1894,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val projectExecs = allExecInfo.filter(_.exec == "Project") assertSizeAndSupported(1, projectExecs) From 091af086eaf37b31211c7aa0abcdd3280e67e44b Mon Sep 17 00:00:00 2001 From: YanxuanLiu <104543031+YanxuanLiu@users.noreply.github.com> Date: Wed, 4 Dec 2024 04:11:08 +0800 Subject: [PATCH 09/52] enable license header check and add header for files (#1440) Signed-off-by: YanxuanLiu --- .github/workflows/license-header-check.yml | 50 +++++++++++++++++++ ...fo-autotuner-example-8cores-16gmemory.yaml | 14 ++++++ data_validation/pyproject.toml | 14 ++++++ data_validation/templates/datavalid_conf.yml | 14 ++++++ data_validation/tox.ini | 14 ++++++ user_tools/pyproject.toml | 14 ++++++ .../resources/bootstrap-conf.yaml | 14 ++++++ .../resources/cluster-configs.yaml | 14 ++++++ .../dev/process_databricks_azure_pricing.py | 14 ++++++ .../resources/diagnostic-conf.yaml | 14 ++++++ .../resources/profiling-conf.yaml | 14 ++++++ .../resources/qualification-conf.yaml | 14 ++++++ .../resources/cluster/dataproc/cpu-00.yaml | 14 ++++++ .../cluster/dataproc_gke/cpu-00.yaml | 14 ++++++ .../resources/cluster/onprem/cpu-00.yaml | 14 ++++++ .../invalid/tools_config_inv_00.yaml | 14 ++++++ .../invalid/tools_config_inv_01.yaml | 14 ++++++ .../invalid/tools_config_inv_02.yaml | 14 ++++++ .../tools_config/valid/tools_config_00.yaml | 14 ++++++ .../resources/worker_info.yaml | 14 ++++++ user_tools/tox.ini | 14 ++++++ 21 files changed, 330 insertions(+) create mode 100644 .github/workflows/license-header-check.yml diff --git a/.github/workflows/license-header-check.yml b/.github/workflows/license-header-check.yml new file mode 100644 index 000000000..1b27aa454 --- /dev/null +++ b/.github/workflows/license-header-check.yml @@ -0,0 +1,50 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + +# A workflow to check copyright/license header +name: license header check + +on: + pull_request: + types: [opened, synchronize, reopened] + +jobs: + license-header-check: + runs-on: ubuntu-latest + if: "!contains(github.event.pull_request.title, '[bot]')" + steps: + - name: Get checkout depth + run: | + echo "PR_FETCH_DEPTH=$(( ${{ github.event.pull_request.commits }} + 10 ))" >> $GITHUB_ENV + + - name: Checkout code + uses: actions/checkout@v4 + with: + fetch-depth: ${{ env.PR_FETCH_DEPTH }} + + - name: license-header-check + uses: NVIDIA/spark-rapids-common/license-header-check@main + with: + included_file_patterns: | + *.py, + *.toml, + *.ini, + *.yml, + *.yaml, + *.sh, + *.properties, + *.xml, + *.feature, + *.scala + \ No newline at end of file diff --git a/core/src/test/resources/worker_info-autotuner-example-8cores-16gmemory.yaml b/core/src/test/resources/worker_info-autotuner-example-8cores-16gmemory.yaml index 9ad530544..12091e726 100644 --- a/core/src/test/resources/worker_info-autotuner-example-8cores-16gmemory.yaml +++ b/core/src/test/resources/worker_info-autotuner-example-8cores-16gmemory.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + system: numCores: 8 memory: 16384MiB diff --git a/data_validation/pyproject.toml b/data_validation/pyproject.toml index 0516a2b98..8bff2bb93 100644 --- a/data_validation/pyproject.toml +++ b/data_validation/pyproject.toml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + [build-system] requires = ["setuptools"] build-backend = "setuptools.build_meta" diff --git a/data_validation/templates/datavalid_conf.yml b/data_validation/templates/datavalid_conf.yml index 5fe2d1f31..8c864d1fe 100644 --- a/data_validation/templates/datavalid_conf.yml +++ b/data_validation/templates/datavalid_conf.yml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + sparkConf: spark.executor.memory: 4g spark.executor.cores: 3 diff --git a/data_validation/tox.ini b/data_validation/tox.ini index 58c3a0212..4725be6de 100644 --- a/data_validation/tox.ini +++ b/data_validation/tox.ini @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + # tox (https://tox.readthedocs.io/) is a tool for running tests # in multiple virtualenvs. This configuration file will run the # test suite on all supported python versions. To use it, "pip install tox" diff --git a/user_tools/pyproject.toml b/user_tools/pyproject.toml index c92bff303..2c903919f 100644 --- a/user_tools/pyproject.toml +++ b/user_tools/pyproject.toml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + [build-system] requires = ["setuptools"] build-backend = "setuptools.build_meta" diff --git a/user_tools/src/spark_rapids_pytools/resources/bootstrap-conf.yaml b/user_tools/src/spark_rapids_pytools/resources/bootstrap-conf.yaml index ee24afdd4..a339a4d0f 100644 --- a/user_tools/src/spark_rapids_pytools/resources/bootstrap-conf.yaml +++ b/user_tools/src/spark_rapids_pytools/resources/bootstrap-conf.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + platform: shortName: 'boot' # workDir: rapids_user_tools_bootstrap diff --git a/user_tools/src/spark_rapids_pytools/resources/cluster-configs.yaml b/user_tools/src/spark_rapids_pytools/resources/cluster-configs.yaml index 700c181a5..d64b1ff95 100644 --- a/user_tools/src/spark_rapids_pytools/resources/cluster-configs.yaml +++ b/user_tools/src/spark_rapids_pytools/resources/cluster-configs.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + clusterConfigs: constants: # Maximum amount of pinned memory to use per executor in megabytes diff --git a/user_tools/src/spark_rapids_pytools/resources/dev/process_databricks_azure_pricing.py b/user_tools/src/spark_rapids_pytools/resources/dev/process_databricks_azure_pricing.py index 19953f393..9ca2ddc1c 100644 --- a/user_tools/src/spark_rapids_pytools/resources/dev/process_databricks_azure_pricing.py +++ b/user_tools/src/spark_rapids_pytools/resources/dev/process_databricks_azure_pricing.py @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + """Python script used to process Databricks Azure instance pricing information.""" import json diff --git a/user_tools/src/spark_rapids_pytools/resources/diagnostic-conf.yaml b/user_tools/src/spark_rapids_pytools/resources/diagnostic-conf.yaml index 359412120..7786e2aae 100644 --- a/user_tools/src/spark_rapids_pytools/resources/diagnostic-conf.yaml +++ b/user_tools/src/spark_rapids_pytools/resources/diagnostic-conf.yaml @@ -1,2 +1,16 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + platform: shortName: 'diag' diff --git a/user_tools/src/spark_rapids_pytools/resources/profiling-conf.yaml b/user_tools/src/spark_rapids_pytools/resources/profiling-conf.yaml index 847e33914..b183a332f 100644 --- a/user_tools/src/spark_rapids_pytools/resources/profiling-conf.yaml +++ b/user_tools/src/spark_rapids_pytools/resources/profiling-conf.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + toolOutput: subFolder: rapids_4_spark_profile textFormat: diff --git a/user_tools/src/spark_rapids_pytools/resources/qualification-conf.yaml b/user_tools/src/spark_rapids_pytools/resources/qualification-conf.yaml index 8e998011c..c149669c5 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualification-conf.yaml +++ b/user_tools/src/spark_rapids_pytools/resources/qualification-conf.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + toolOutput: completeOutput: true subFolder: rapids_4_spark_qualification_output diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc/cpu-00.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc/cpu-00.yaml index c6bc6038d..9a0d74652 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc/cpu-00.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc/cpu-00.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + clusterName: dataproc-test-nongpu-cluster clusterUuid: 11111111-1111-1111-1111-111111111111 config: diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc_gke/cpu-00.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc_gke/cpu-00.yaml index 39bd70ddf..405f6bea5 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc_gke/cpu-00.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/cluster/dataproc_gke/cpu-00.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + clusterName: dataproc-gke-test-nongpu-cluster clusterUuid: 11111111-1111-1111-1111-111111111111 config: diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/cluster/onprem/cpu-00.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/cluster/onprem/cpu-00.yaml index b170d85b0..9a900ae95 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/cluster/onprem/cpu-00.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/cluster/onprem/cpu-00.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + config: masterConfig: numCores: 2 diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_00.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_00.yaml index aa566a71f..b656ea7ee 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_00.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_00.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + # This yaml file is a configuration file for a tool that uses Spark 3.5.0 as dependency # invalid: API version is smaller than accepted API version api_version: '0.9' diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_01.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_01.yaml index a723f2e66..dd713e594 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_01.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_01.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + # This yaml file is a configuration file # invalid: empty file # Error:1 validation error for ToolsConfig diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_02.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_02.yaml index 210b4419a..dc8f1df49 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_02.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/invalid/tools_config_inv_02.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + # This yaml file is a configuration file for a tool that uses Spark 3.5.0 as dependency # invalid: Local file can be verified during the initialization and it should fail because no such # file exists. diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/valid/tools_config_00.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/valid/tools_config_00.yaml index f7dccf951..85ef4e15b 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/valid/tools_config_00.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/tools_config/valid/tools_config_00.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + # This yaml file is a configuration file for a tool that uses Spark 3.5.0 as dependency # Minimal file content. api_version: '1.0' diff --git a/user_tools/tests/spark_rapids_tools_ut/resources/worker_info.yaml b/user_tools/tests/spark_rapids_tools_ut/resources/worker_info.yaml index d9aaa14d5..7b3f8f32b 100644 --- a/user_tools/tests/spark_rapids_tools_ut/resources/worker_info.yaml +++ b/user_tools/tests/spark_rapids_tools_ut/resources/worker_info.yaml @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + system: numCores: 32 memory: 212992MiB diff --git a/user_tools/tox.ini b/user_tools/tox.ini index b7a766d98..f89d18857 100644 --- a/user_tools/tox.ini +++ b/user_tools/tox.ini @@ -1,3 +1,17 @@ +# Copyright (c) 2024, NVIDIA CORPORATION. +# +# Licensed 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. + # tox (https://tox.readthedocs.io/) is a tool for running tests # in multiple virtualenvs. This configuration file will run the # test suite on all supported python versions. To use it, "pip install tox" From 993bc8f82e5c512349c3485fb15dc144f0bbc9b5 Mon Sep 17 00:00:00 2001 From: Niranjan Artal <50492963+nartal1@users.noreply.github.com> Date: Wed, 4 Dec 2024 14:03:15 -0800 Subject: [PATCH 10/52] Report all operators in the output file (#1444) * first iterations all the UTs pass Signed-off-by: Ahmed Hussein (amahussein) * running end-to-end Signed-off-by: Ahmed Hussein (amahussein) * cleaned-up exec parsers Signed-off-by: Ahmed Hussein (amahussein) * update documentation and fix Signed-off-by: Niranjan Artal * qualx related changes * update qualx changes * Fix for Scan OneRowRelation Signed-off-by: Niranjan Artal * addressed review comments --------- Signed-off-by: Niranjan Artal Co-authored-by: Ahmed Hussein (amahussein) --- .../tool/planparser/BatchScanExecParser.scala | 28 ++++- .../BroadcastExchangeExecParser.scala | 3 +- .../BroadcastHashJoinExecParser.scala | 6 +- .../BroadcastNestedLoopJoinExecParser.scala | 6 +- .../DataWritingCommandExecParser.scala | 6 +- .../tool/planparser/DeltaLakeHelper.scala | 4 +- .../rapids/tool/planparser/ExecParser.scala | 5 +- .../planparser/FileSourceScanExecParser.scala | 35 +++++- .../tool/planparser/GenericExecParser.scala | 14 ++- .../rapids/tool/planparser/ReadParser.scala | 12 +- .../tool/planparser/SQLPlanParser.scala | 73 ++++++++---- .../ShuffleExchangeExecParser.scala | 4 +- .../ShuffledHashJoinExecParser.scala | 4 +- .../planparser/SortMergeJoinExecParser.scala | 4 +- .../SubqueryBroadcastExecParser.scala | 5 +- .../tool/planparser/SubqueryExecParser.scala | 3 +- .../planparser/WholeStageExecParser.scala | 30 ++++- .../planparser/WindowGroupLimitParser.scala | 11 +- .../planparser/WriteFilesExecParser.scala | 5 +- .../rapids/tool/planparser/ops/OpRef.scala | 64 +++++++++++ .../tool/planparser/ops/OperatorCounter.scala | 106 ++++++++++++++++++ .../tool/planparser/ops/OperatorRefBase.scala | 41 +++++++ .../planparser/ops/OperatorRefTrait.scala | 24 ++++ .../planparser/ops/UnsupportedExprOpRef.scala | 43 +++++++ .../qualification/PluginTypeChecker.scala | 6 +- .../tool/qualification/QualOutputWriter.scala | 73 +++++++++++- .../tool/qualification/Qualification.scala | 1 + .../spark/sql/rapids/tool/AppBase.scala | 4 +- .../spark/sql/rapids/tool/ToolUtils.scala | 2 - .../qualification/QualificationAppInfo.scala | 9 +- .../db_sim_test_expectation.csv | 2 +- .../directory_test_expectation.csv | 2 +- .../jdbc_expectation.csv | 2 +- .../qual_test_missing_sql_end_expectation.csv | 2 +- .../qual_test_simple_expectation.csv | 2 +- .../truncated_1_end_expectation.csv | 2 +- .../tool/planparser/SqlPlanParserSuite.scala | 19 ++-- .../PluginTypeCheckerSuite.scala | 2 +- .../qualification/QualificationSuite.scala | 4 +- .../tools/qualx/preprocess.py | 25 +++-- 40 files changed, 574 insertions(+), 119 deletions(-) create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRef.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefBase.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefTrait.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BatchScanExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BatchScanExecParser.scala index a8188065c..7c591b372 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BatchScanExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BatchScanExecParser.scala @@ -27,8 +27,8 @@ case class BatchScanExecParser( checker: PluginTypeChecker, sqlID: Long, app: AppBase) extends ExecParser with Logging { - - val fullExecName = "BatchScanExec" + val nodeName = "BatchScan" + val fullExecName = nodeName + "Exec" override def parse: ExecInfo = { val accumId = node.metrics.find(_.name == "scan time").map(_.accumulatorId) @@ -39,8 +39,26 @@ case class BatchScanExecParser( val speedupFactor = checker.getSpeedupFactor(fullExecName) val overallSpeedup = Math.max((speedupFactor * score), 1.0) - // TODO - add in parsing expressions - average speedup across? - ExecInfo(node, sqlID, s"${node.name} ${readInfo.format}", s"Format: ${readInfo.format}", - overallSpeedup, maxDuration, node.id, score > 0, None) + // 1- Set the exec name to be the batchScan + format + // 2- If the format cannot be found, then put the entire node description to make it easy to + // troubleshoot by reading the output files. + val readFormat = readInfo.getReadFormatLC + val execExpression = if (readInfo.hasUnknownFormat) { + node.desc + } else { + s"Format: $readFormat" + } + + ExecInfo.createExecNoNode( + sqlID = sqlID, + exec = s"$nodeName $readFormat", + expr = execExpression, + speedupFactor = overallSpeedup, + duration = maxDuration, + nodeId = node.id, + opType = OpTypes.ReadExec, + isSupported = score > 0.0, + children = None, + expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastExchangeExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastExchangeExecParser.scala index f9c348ea6..e6bbc99d8 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastExchangeExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastExchangeExecParser.scala @@ -44,8 +44,7 @@ case class BroadcastExchangeExecParser( } else { (1.0, false) } - // TODO - add in parsing expressions - average speedup across? ExecInfo(node, sqlID, node.name, "", filterSpeedupFactor, - duration, node.id, isSupported, None) + duration, node.id, isSupported, children = None, expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastHashJoinExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastHashJoinExecParser.scala index 8426f1047..8acd2db7b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastHashJoinExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastHashJoinExecParser.scala @@ -30,7 +30,7 @@ case class BroadcastHashJoinExecParser( override def parse: ExecInfo = { // BroadcastHashJoin doesn't have duration val duration = None - val exprString = node.desc.replaceFirst("BroadcastHashJoin ", "") + val exprString = node.desc.replaceFirst("^BroadcastHashJoin\\s*", "") val (expressions, supportedJoinType) = SQLPlanParser.parseEquijoinsExpressions(exprString) val notSupportedExprs = expressions.filterNot(expr => checker.isExprSupported(expr)) val (speedupFactor, isSupported) = if (checker.isExecSupported(fullExecName) && @@ -39,7 +39,7 @@ case class BroadcastHashJoinExecParser( } else { (1.0, false) } - // TODO - add in parsing expressions - average speedup across? - ExecInfo(node, sqlID, node.name, "", speedupFactor, duration, node.id, isSupported, None) + ExecInfo(node, sqlID, node.name, "", speedupFactor, duration, node.id, isSupported, + children = None, expressions = expressions) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastNestedLoopJoinExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastNestedLoopJoinExecParser.scala index 49f242efc..f6be5decd 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastNestedLoopJoinExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/BroadcastNestedLoopJoinExecParser.scala @@ -39,7 +39,7 @@ abstract class BroadcastNestedLoopJoinExecParserBase( override def parse: ExecInfo = { // BroadcastNestedLoopJoin doesn't have duration - val exprString = node.desc.replaceFirst("BroadcastNestedLoopJoin ", "") + val exprString = node.desc.replaceFirst("^BroadcastNestedLoopJoin\\s*", "") val (buildSide, joinType) = extractBuildAndJoinTypes(exprString) val (expressions, supportedJoinType) = SQLPlanParser.parseNestedLoopJoinExpressions(exprString, buildSide, joinType) @@ -51,8 +51,8 @@ abstract class BroadcastNestedLoopJoinExecParserBase( } else { (1.0, false) } - // TODO - add in parsing expressions - average speedup across? - ExecInfo(node, sqlID, node.name, "", speedupFactor, duration, node.id, isSupported, None) + ExecInfo(node, sqlID, node.name, "", speedupFactor, duration, node.id, isSupported, + children = None, expressions = expressions) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DataWritingCommandExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DataWritingCommandExecParser.scala index fedbd2ceb..86e553ac1 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DataWritingCommandExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DataWritingCommandExecParser.scala @@ -37,11 +37,11 @@ case class DataWritingCommandExecParser( val duration = None val speedupFactor = checker.getSpeedupFactor(wStub.mappedExec) val finalSpeedup = if (writeSupported) speedupFactor else 1 - // TODO - add in parsing expressions - average speedup across? // We do not want to parse the node description to avoid mistakenly marking the node as RDD/UDF ExecInfo.createExecNoNode(sqlID, s"${node.name.trim} ${wStub.dataFormat.toLowerCase.trim}", s"Format: ${wStub.dataFormat.toLowerCase.trim}", - finalSpeedup, duration, node.id, opType = OpTypes.WriteExec, writeSupported, None) + finalSpeedup, duration, node.id, opType = OpTypes.WriteExec, writeSupported, + children = None, expressions = Seq.empty) } } @@ -175,4 +175,4 @@ object DataWritingCommandExecParser { parsedString.split(",")(0) // return third parameter from the input string } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala index 82ece4f45..884ad8907 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala @@ -45,10 +45,10 @@ class DLWriteWithFormatAndSchemaParser(node: SparkPlanGraphNode, val finalSpeedupFactor = if (writeSupported) speedupFactor else 1.0 // execs like SaveIntoDataSourceCommand has prefix "Execute". So, we need to get rid of it. - val nodeName = node.name.replace("Execute ", "") + val nodeName = node.name.replaceFirst("Execute\\s*", "") ExecInfo.createExecNoNode(sqlID, nodeName, s"Format: $dataFormat", finalSpeedupFactor, None, node.id, OpTypes.WriteExec, - isSupported = writeSupported && isExecSupported, children = None) + isSupported = writeSupported && isExecSupported, children = None, expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ExecParser.scala index b43d9943f..c0ea0f369 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ExecParser.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tool.planparser -import org.apache.spark.sql.rapids.tool.UnsupportedExpr +import com.nvidia.spark.rapids.tool.planparser.ops.UnsupportedExprOpRef trait ExecParser { def parse: ExecInfo @@ -32,5 +32,6 @@ trait ExecParser { * @param expressions Array of expression strings to evaluate for support. * @return Empty Seq[UnsupportedExpr], indicating no unsupported expressions by default. */ - def getUnsupportedExprReasonsForExec(expressions: Array[String]): Seq[UnsupportedExpr] = Seq.empty + def getUnsupportedExprReasonsForExec( + expressions: Array[String]): Seq[UnsupportedExprOpRef] = Seq.empty } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/FileSourceScanExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/FileSourceScanExecParser.scala index 85387872f..520d05832 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/FileSourceScanExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/FileSourceScanExecParser.scala @@ -30,6 +30,9 @@ case class FileSourceScanExecParser( // The node name for Scans is Scan so here we hardcode val fullExecName = "FileSourceScanExec" + // Matches the first alphaneumeric characters of a string after trimming leading/trailing + // white spaces. + val nodeNameRegeX = """^\s*(\w+).*""".r override def parse: ExecInfo = { // Remove trailing spaces from node name @@ -37,7 +40,7 @@ case class FileSourceScanExecParser( val nodeName = node.name.trim val rddCheckRes = RDDCheckHelper.isDatasetOrRDDPlan(nodeName, node.desc) if (rddCheckRes.nodeNameRDD) { - // This is a scanRDD. we do not need to parse it as a normal node. + // This is a scanRDD. We do not need to parse it as a normal node. // cleanup the node name if possible: val newNodeName = if (nodeName.contains("ExistingRDD")) { val nodeNameLength = nodeName.indexOf("ExistingRDD") + "ExistingRDD".length @@ -46,7 +49,7 @@ case class FileSourceScanExecParser( nodeName } ExecInfo.createExecNoNode(sqlID, newNodeName, "", 1.0, duration = None, - node.id, OpTypes.ReadRDD, false, None) + node.id, OpTypes.ReadRDD, false, children = None, expressions = Seq.empty) } else { val accumId = node.metrics.find(_.name == "scan time").map(_.accumulatorId) val maxDuration = SQLPlanParser.getTotalDuration(accumId, app) @@ -57,14 +60,36 @@ case class FileSourceScanExecParser( // Use the default parser (fullExecName, ReadParser.parseReadNode(node)) } + // 1- Set the exec name to nodeLabel + format + // 2- If the format is not found, then put the entire node description to make it easy to + // troubleshoot by reading the output files. + val nodeLabel = nodeNameRegeX.findFirstMatchIn(nodeName) match { + case Some(m) => m.group(1) + // in case not found, use the full exec name + case None => execName + } + val readFormat = readInfo.getReadFormatLC + val exexExpr = if (readInfo.hasUnknownFormat) { + node.desc + } else { + s"Format: ${readFormat}" + } val speedupFactor = checker.getSpeedupFactor(execName) // don't use the isExecSupported because we have finer grain. val score = ReadParser.calculateReadScoreRatio(readInfo, checker) val overallSpeedup = Math.max(speedupFactor * score, 1.0) - // TODO - add in parsing expressions - average speedup across? - ExecInfo.createExecNoNode(sqlID, nodeName, "", overallSpeedup, maxDuration, - node.id, OpTypes.ReadExec, score > 0, None) + ExecInfo.createExecNoNode( + sqlID = sqlID, + exec = s"$nodeLabel $readFormat", + expr = exexExpr, + speedupFactor = overallSpeedup, + duration = maxDuration, + nodeId = node.id, + opType = OpTypes.ReadExec, + isSupported = score > 0, + children = None, + expressions = Seq.empty) } } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala index 62ffb5eaa..6295c5533 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala @@ -16,10 +16,11 @@ package com.nvidia.spark.rapids.tool.planparser +import com.nvidia.spark.rapids.tool.planparser.ops.UnsupportedExprOpRef import com.nvidia.spark.rapids.tool.qualification.PluginTypeChecker import org.apache.spark.sql.execution.ui.SparkPlanGraphNode -import org.apache.spark.sql.rapids.tool.{AppBase, UnsupportedExpr} +import org.apache.spark.sql.rapids.tool.AppBase class GenericExecParser( val node: SparkPlanGraphNode, @@ -46,7 +47,8 @@ class GenericExecParser( (1.0, false) } - createExecInfo(speedupFactor, isSupported, duration, notSupportedExprs) + createExecInfo(speedupFactor, isSupported, duration, + notSupportedExprs = notSupportedExprs, expressions = expressions) } protected def parseExpressions(): Array[String] = { @@ -63,7 +65,7 @@ class GenericExecParser( node.desc.replaceFirst(s"^${node.name}\\s*", "") } - protected def getNotSupportedExprs(expressions: Array[String]): Seq[UnsupportedExpr] = { + protected def getNotSupportedExprs(expressions: Array[String]): Seq[UnsupportedExprOpRef] = { checker.getNotSupportedExprs(expressions) } @@ -83,7 +85,8 @@ class GenericExecParser( speedupFactor: Double, isSupported: Boolean, duration: Option[Long], - notSupportedExprs: Seq[UnsupportedExpr] + notSupportedExprs: Seq[UnsupportedExprOpRef], + expressions: Array[String] ): ExecInfo = { ExecInfo( node, @@ -95,7 +98,8 @@ class GenericExecParser( node.id, isSupported, None, - unsupportedExprs = notSupportedExprs + unsupportedExprs = notSupportedExprs, + expressions = expressions ) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala index 131d6b2d3..ac43bf783 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala @@ -29,11 +29,21 @@ case class ReadMetaData(schema: String, location: String, format: String, def pushedFilters: String = tags(ReadParser.METAFIELD_TAG_PUSHED_FILTERS) def dataFilters: String = tags(ReadParser.METAFIELD_TAG_DATA_FILTERS) def partitionFilters: String = tags(ReadParser.METAFIELD_TAG_PARTITION_FILTERS) + + def hasUnknownFormat: Boolean = format.equals(ReadParser.UNKNOWN_METAFIELD) + + /** + * Returns the read format in lowercase. This is used to be consistent. + * @return the lower case of the read format + */ + def getReadFormatLC: String = format.toLowerCase } object ReadParser extends Logging { // It was found that some eventlogs could have "NativeScan" instead of "Scan" val SCAN_NODE_PREFIXES = Seq("Scan", "NativeScan") + // Do not include OneRowRelation in the scan nodes, consider it as regular Exec + val SCAN_ONE_ROW_RELATION = "Scan OneRowRelation" // DatasourceV2 node names that exactly match the following labels val DATASOURCE_V2_NODE_EXACT_PREF = Set( "BatchScan") @@ -58,7 +68,7 @@ object ReadParser extends Logging { ) def isScanNode(nodeName: String): Boolean = { - SCAN_NODE_PREFIXES.exists(nodeName.startsWith(_)) + SCAN_NODE_PREFIXES.exists(nodeName.startsWith(_)) && !nodeName.startsWith(SCAN_ONE_ROW_RELATION) } def isScanNode(node: SparkPlanGraphNode): Boolean = { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index ad2634c41..17159842b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -21,16 +21,18 @@ import scala.collection.mutable.{ArrayBuffer, WeakHashMap} import scala.util.control.NonFatal import scala.util.matching.Regex +import com.nvidia.spark.rapids.tool.planparser.ops.{OperatorRefBase, OpRef, UnsupportedExprOpRef} import com.nvidia.spark.rapids.tool.planparser.photon.{PhotonPlanParser, PhotonStageExecParser} import com.nvidia.spark.rapids.tool.qualification.PluginTypeChecker import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, SparkPlanGraphNode} -import org.apache.spark.sql.rapids.tool.{AppBase, BuildSide, ExecHelper, JoinType, RDDCheckHelper, ToolUtils, UnsupportedExpr} +import org.apache.spark.sql.rapids.tool.{AppBase, BuildSide, ExecHelper, JoinType, RDDCheckHelper, ToolUtils} import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph import org.apache.spark.sql.rapids.tool.util.plangraph.{PhotonSparkPlanGraphCluster, PhotonSparkPlanGraphNode} + object OpActions extends Enumeration { type OpAction = Value val NONE, IgnoreNoPerf, IgnorePerf, Triage = Value @@ -73,11 +75,10 @@ object UnsupportedReasons extends Enumeration { case class UnsupportedExecSummary( sqlId: Long, execId: Long, - execValue: String, + execRef: OperatorRefBase, opType: OpTypes.OpType, reason: UnsupportedReasons.UnsupportedReason, - opAction: OpActions.OpAction, - isExpression: Boolean = false) { + opAction: OpActions.OpAction) { val finalOpType: String = if (opType.equals(OpTypes.UDF) || opType.equals(OpTypes.DataSet)) { s"${OpTypes.Exec.toString}" @@ -85,14 +86,16 @@ case class UnsupportedExecSummary( s"${opType.toString}" } - val unsupportedOperator: String = execValue + val unsupportedOperatorCSVFormat: String = execRef.getOpNameCSV val details: String = UnsupportedReasons.reportUnsupportedReason(reason) + + def isExpression: Boolean = execRef.isInstanceOf[UnsupportedExprOpRef] } case class ExecInfo( sqlID: Long, - exec: String, + execRef: OpRef, expr: String, speedupFactor: Double, duration: Option[Long], @@ -103,10 +106,11 @@ case class ExecInfo( var stages: Set[Int], var shouldRemove: Boolean, var unsupportedExecReason: String, - unsupportedExprs: Seq[UnsupportedExpr], + unsupportedExprs: Seq[UnsupportedExprOpRef], dataSet: Boolean, udf: Boolean, - shouldIgnore: Boolean) { + shouldIgnore: Boolean, + expressions: Seq[OpRef]) { private def childrenToString = { val str = children.map { c => @@ -119,6 +123,12 @@ case class ExecInfo( } } + def exec: String = execRef.value + + def isClusterNode: Boolean = { + execRef.getOpName.contains("StageCodegen") || execRef.getOpName.contains("PhotonResultStage") + } + override def toString: String = { s"exec: $exec, expr: $expr, sqlID: $sqlID , speedupFactor: $speedupFactor, " + s"duration: $duration, nodeId: $nodeId, " + @@ -194,7 +204,7 @@ case class ExecInfo( getUnsupportedReason) // Initialize the result with the exec summary - val res = ArrayBuffer(UnsupportedExecSummary(sqlID, execId, exec, opType, + val res = ArrayBuffer(UnsupportedExecSummary(sqlID, execId, execRef, opType, execUnsupportedReason, getOpAction)) // TODO: Should we iterate on exec children? @@ -211,8 +221,8 @@ case class ExecInfo( unsupportedExprs.foreach { expr => val exprUnsupportedReason = determineUnsupportedReason(expr.unsupportedReason, exprKnownReason) - res += UnsupportedExecSummary(sqlID, execId, expr.exprName, OpTypes.Expr, - exprUnsupportedReason, getOpAction, isExpression = true) + res += UnsupportedExecSummary(sqlID, execId, expr, OpTypes.Expr, + exprUnsupportedReason, getOpAction) } } res @@ -235,9 +245,10 @@ object ExecInfo { stages: Set[Int] = Set.empty, shouldRemove: Boolean = false, unsupportedExecReason: String = "", - unsupportedExprs: Seq[UnsupportedExpr] = Seq.empty, + unsupportedExprs: Seq[UnsupportedExprOpRef] = Seq.empty, dataSet: Boolean = false, - udf: Boolean = false): ExecInfo = { + udf: Boolean = false, + expressions: Seq[String]): ExecInfo = { // Set the ignoreFlag // 1- we ignore any exec with UDF // 2- we ignore any exec with dataset @@ -259,7 +270,7 @@ object ExecInfo { val supportedFlag = isSupported && !udf && !finalDataSet ExecInfo( sqlID, - exec, + OpRef.fromExec(exec), expr, speedupFactor, duration, @@ -273,7 +284,9 @@ object ExecInfo { unsupportedExprs, finalDataSet, udf, - shouldIgnore + shouldIgnore, + // convert array of string expressions to OpRefs + expressions = expressions.map(OpRef.fromExpr) ) } @@ -290,10 +303,11 @@ object ExecInfo { stages: Set[Int] = Set.empty, shouldRemove: Boolean = false, unsupportedExecReason:String = "", - unsupportedExprs: Seq[UnsupportedExpr] = Seq.empty, + unsupportedExprs: Seq[UnsupportedExprOpRef] = Seq.empty, dataSet: Boolean = false, udf: Boolean = false, - opType: OpTypes.OpType = OpTypes.Exec): ExecInfo = { + opType: OpTypes.OpType = OpTypes.Exec, + expressions: Seq[String]): ExecInfo = { // Some execs need to be trimmed such as "Scan" // Example: Scan parquet . -> Scan parquet. // scan nodes needs trimming @@ -307,7 +321,7 @@ object ExecInfo { // if the expression is RDD because of the node name, then we do not want to add the // unsupportedExpressions because it becomes bogus. val finalUnsupportedExpr = if (rddCheckRes.nodeDescRDD) { - Seq.empty[UnsupportedExpr] + Seq.empty[UnsupportedExprOpRef] } else { unsupportedExprs } @@ -326,7 +340,8 @@ object ExecInfo { unsupportedExecReason, finalUnsupportedExpr, ds, - containsUDF + containsUDF, + expressions = expressions ) } } @@ -335,8 +350,18 @@ case class PlanInfo( appID: String, sqlID: Long, sqlDesc: String, - execInfo: Seq[ExecInfo] -) + execInfo: Seq[ExecInfo]) { + def getUnsupportedExpressions: Seq[OperatorRefBase] = { + execInfo.flatMap { e => + if (e.isClusterNode) { + // wholeStageCodeGen does not have expressions/unsupported-expressions + e.children.getOrElse(Seq.empty).flatMap(_.unsupportedExprs) + } else { + e.unsupportedExprs + } + } + } +} object SQLPlanParser extends Logging { @@ -528,7 +553,8 @@ object SQLPlanParser extends Logging { // supported but with shouldRemove flag set to True. // Setting the "shouldRemove" is handled at the end of the function. ExecInfo(node, sqlID, normalizedNodeName, expr = "", 1, duration = None, node.id, - isSupported = reuseExecs.contains(normalizedNodeName), None) + isSupported = reuseExecs.contains(normalizedNodeName), children = None, + expressions = Seq.empty) } } @@ -584,7 +610,8 @@ object SQLPlanParser extends Logging { logWarning(s"Unexpected error parsing plan node ${normalizedNodeName}. " + s" sqlID = ${sqlID}", e) ExecInfo(node, sqlID, normalizedNodeName, expr = "", 1, duration = None, node.id, - isSupported = false, None) + isSupported = false, children = None, + expressions = Seq.empty) } val stagesInNode = nodeIdToStagesFunc(node.id) execInfo.setStages(stagesInNode) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffleExchangeExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffleExchangeExecParser.scala index 2a1315718..2026a2a17 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffleExchangeExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffleExchangeExecParser.scala @@ -44,7 +44,7 @@ case class ShuffleExchangeExecParser( } else { (1.0, false) } - // TODO - add in parsing expressions - average speedup across? - ExecInfo(node, sqlID, node.name, "", filterSpeedupFactor, duration, node.id, isSupported, None) + ExecInfo(node, sqlID, node.name, "", filterSpeedupFactor, duration, node.id, isSupported, + children = None, expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffledHashJoinExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffledHashJoinExecParser.scala index b81f33fd6..4c90928a4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffledHashJoinExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ShuffledHashJoinExecParser.scala @@ -43,9 +43,7 @@ case class ShuffledHashJoinExecParser( } else { (1.0, false) } - - // TODO - add in parsing expressions - average speedup across? ExecInfo(node, sqlID, node.name, "", speedupFactor, - maxDuration, node.id, isSupported, None) + maxDuration, node.id, isSupported, children = None, expressions = expressions) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SortMergeJoinExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SortMergeJoinExecParser.scala index f98a0542b..4828d8ac9 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SortMergeJoinExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SortMergeJoinExecParser.scala @@ -42,8 +42,8 @@ case class SortMergeJoinExecParser( } else { (1.0, false) } - // TODO - add in parsing expressions - average speedup across? - ExecInfo(node, sqlID, opName, "", speedupFactor, duration, node.id, isSupported, None) + ExecInfo(node, sqlID, opName, "", speedupFactor, duration, node.id, isSupported, + children = None, expressions = expressions) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryBroadcastExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryBroadcastExecParser.scala index 64fc5ff5e..048a317b0 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryBroadcastExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryBroadcastExecParser.scala @@ -38,8 +38,7 @@ case class SubqueryBroadcastExecParser( } else { (1.0, false) } - // TODO - check is broadcast associated can be replaced - // TODO - add in parsing expressions - average speedup across? - ExecInfo(node, sqlID, node.name, "", filterSpeedupFactor, duration, node.id, isSupported, None) + ExecInfo(node, sqlID, node.name, "", filterSpeedupFactor, duration, node.id, isSupported, + children = None, expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryExecParser.scala index c60eff7ac..e01f7a27a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SubqueryExecParser.scala @@ -39,7 +39,8 @@ case class SubqueryExecParser( // TODO: Should we also collect the "data size" metric? val duration = SQLPlanParser.getDriverTotalDuration(collectTimeId, app) // should remove is kept in 1 place. So no need to set it here. - ExecInfo(node, sqlID, node.name, "", 1.0, duration, node.id, isSupported = false, None) + ExecInfo(node, sqlID, node.name, "", 1.0, duration, node.id, isSupported = false, + children = None, expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala index cdef3b225..5b8730938 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala @@ -31,6 +31,9 @@ abstract class WholeStageExecParserBase( nodeIdToStagesFunc: Long => Set[Int]) extends Logging { val fullExecName = "WholeStageCodegenExec" + // Matches the first alphanumeric characters of a string after trimming leading/trailing + // white spaces. + val nodeNameRegeX = """^\s*(\w+).*""".r def parse: Seq[ExecInfo] = { // TODO - does metrics for time have previous ops? per op thing, only some do @@ -50,17 +53,34 @@ abstract class WholeStageExecParserBase( } // if any of the execs in WholeStageCodegen supported mark this entire thing as supported val anySupported = childNodes.exists(_.isSupported == true) - val unSupportedExprsArray = - childNodes.filter(_.unsupportedExprs.nonEmpty).flatMap(x => x.unsupportedExprs).toArray // average speedup across the execs in the WholeStageCodegen for now val supportedChildren = childNodes.filterNot(_.shouldRemove) val avSpeedupFactor = SQLPlanParser.averageSpeedup(supportedChildren.map(_.speedupFactor)) // The node should be marked as shouldRemove when all the children of the // wholeStageCodeGen are marked as shouldRemove. val removeNode = isDupNode || childNodes.forall(_.shouldRemove) - val execInfo = ExecInfo(node, sqlID, node.name, node.name, avSpeedupFactor, maxDuration, - node.id, anySupported, Some(childNodes), stagesInNode, - shouldRemove = removeNode, unsupportedExprs = unSupportedExprsArray) + // Remove any suffix in order to get the node label without any trailing number. + val nodeLabel = nodeNameRegeX.findFirstMatchIn(node.name) match { + case Some(m) => m.group(1) + // in case not found, use the full exec name + case None => fullExecName + } + val execInfo = ExecInfo( + node = node, + sqlID = sqlID, + exec = nodeLabel, + expr = node.name, + speedupFactor = avSpeedupFactor, + duration = maxDuration, + nodeId = node.id, + isSupported = anySupported, + children = Some(childNodes), + stages = stagesInNode, + shouldRemove = removeNode, + // unsupported expressions should not be set for the cluster nodes. + unsupportedExprs = Seq.empty, + // expressions of wholeStageCodeGen should not be set. They belong to the children nodes. + expressions = Seq.empty) Seq(execInfo) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala index 01b5edc3d..853c1b767 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala @@ -16,10 +16,10 @@ package com.nvidia.spark.rapids.tool.planparser +import com.nvidia.spark.rapids.tool.planparser.ops.UnsupportedExprOpRef import com.nvidia.spark.rapids.tool.qualification.PluginTypeChecker import org.apache.spark.sql.execution.ui.SparkPlanGraphNode -import org.apache.spark.sql.rapids.tool.UnsupportedExpr case class WindowGroupLimitParser( node: SparkPlanGraphNode, @@ -36,10 +36,10 @@ case class WindowGroupLimitParser( } override def getUnsupportedExprReasonsForExec( - expressions: Array[String]): Seq[UnsupportedExpr] = { + expressions: Array[String]): Seq[UnsupportedExprOpRef] = { expressions.flatMap { expr => if (!supportedRankingExprs.contains(expr)) { - Some(UnsupportedExpr(expr, + Some(UnsupportedExprOpRef(expr, s"Ranking function $expr is not supported in $fullExecName")) } else { None @@ -57,7 +57,7 @@ case class WindowGroupLimitParser( * 3. Ranking function is supported by plugin's implementation of WindowGroupLimitExec. */ override def parse: ExecInfo = { - val exprString = node.desc.replaceFirst("WindowGroupLimit ", "") + val exprString = node.desc.replaceFirst("WindowGroupLimit\\s*", "") val expressions = SQLPlanParser.parseWindowGroupLimitExpressions(exprString) val notSupportedExprs = checker.getNotSupportedExprs(expressions) ++ getUnsupportedExprReasonsForExec(expressions) @@ -69,8 +69,7 @@ case class WindowGroupLimitParser( } else { (1.0, false) } - // TODO - add in parsing expressions - average speedup across? ExecInfo(node, sqlID, node.name, "", speedupFactor, None, node.id, isSupported, None, - unsupportedExprs = notSupportedExprs) + unsupportedExprs = notSupportedExprs, expressions = expressions) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WriteFilesExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WriteFilesExecParser.scala index bcd2272e0..0fa6cb254 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WriteFilesExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WriteFilesExecParser.scala @@ -44,7 +44,10 @@ case class WriteFilesExecParser( "", speedupFactor, duration, - node.id, opType = OpTypes.WriteExec, true, None) + node.id, opType = OpTypes.WriteExec, + isSupported = true, + children = None, + expressions = Seq.empty) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRef.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRef.scala new file mode 100644 index 000000000..3e5f22c08 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRef.scala @@ -0,0 +1,64 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + +import java.util.concurrent.ConcurrentHashMap + +import com.nvidia.spark.rapids.tool.planparser.OpTypes + + +/** + * A reference to an operator(either Exec operator or expression operator) in a Spark plan. + * + * @param value The name of the operator. + * @param opType The type of the operator (e.g., Exec, Expr). + */ +case class OpRef(override val value: String, + override val opType: OpTypes.OpType) extends OperatorRefBase(value, opType) + +object OpRef { + // Dummy OpNameRef to represent None accumulator names. This is an optimization to avoid + // storing an option[string] for all operator names which leads to "get-or-else" everywhere. + private val EMPTY_OP_NAME_REF: OpRef = new OpRef("", OpTypes.Exec) + // A global table to store reference to all operator names. The map is accessible by all + // threads (different applications) running in parallel. This avoids duplicate work across + // different threads. + val OP_NAMES: ConcurrentHashMap[String, OpRef] = { + val initMap = new ConcurrentHashMap[String, OpRef]() + initMap.put(EMPTY_OP_NAME_REF.value, EMPTY_OP_NAME_REF) + // Add the operator to the map because it is being used internally. + initMap + } + + /** + * Retrieves an `OpRef` for an expression operator. + * If the operator name already exists in the cache, it returns the existing `OpRef`. + * Otherwise, it creates a new `OpRef` with the given name and `OpTypes.Expr`. + */ + def fromExpr(name: String): OpRef = { + OP_NAMES.computeIfAbsent(name, OpRef.apply(_, OpTypes.Expr)) + } + + /** + * Retrieves an `OpRef` for an exec operator. + * If the operator name already exists in the cache, it returns the existing `OpRef`. + * Otherwise, it creates a new `OpRef` with the given name and `OpTypes.Exec`. + */ + def fromExec(name: String): OpRef = { + OP_NAMES.computeIfAbsent(name, OpRef.apply(_, OpTypes.Exec)) + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala new file mode 100644 index 000000000..fab9422aa --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala @@ -0,0 +1,106 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + +import scala.collection.mutable + +import com.nvidia.spark.rapids.tool.planparser.{ExecInfo, PlanInfo} + +/** + * `OperatorCounter` is responsible for counting the occurrences of execs and expressions + * in a given execution plan (`PlanInfo`). It maintains counts separately for supported and + * unsupported execs and expressions. + * + * @param planInfo The execution plan information to analyze. + */ +case class OperatorCounter(planInfo: PlanInfo) { + + /** + * Represents data for an exec or expression, including its reference, + * occurrence count, and stages where it appears. + * + * @param opRef The operator reference. + * @param count The number of times the operator appears. + * @param stages The set of stages where the operator appears. + */ + case class OperatorData( + opRef: OperatorRefBase, + var count: Int = 0, + var stages: Set[Int] = Set()) + + // Summarizes the count information for an exec or expression, including whether it is supported. + case class OperatorCountSummary( + opData: OperatorData, + isSupported: Boolean) + + private val supportedMap: mutable.Map[OperatorRefBase, OperatorData] = mutable.Map() + private val unsupportedMap: mutable.Map[OperatorRefBase, OperatorData] = mutable.Map() + + // Returns a sequence of `OperatorCountSummary`, combining both supported and + // unsupported operators. + def getOpsCountSummary(): Seq[OperatorCountSummary] = { + supportedMap.values.map(OperatorCountSummary(_, isSupported = true)).toSeq ++ + unsupportedMap.values.map(OperatorCountSummary(_, isSupported = false)).toSeq + } + + + // Updates the operator data in the given map (supported or unsupported). + // Increments the count and updates the stages where the operator appears. + private def updateOpRefEntry(opRef: OperatorRefBase, stages: Set[Int], + targetMap: mutable.Map[OperatorRefBase, OperatorData]): Unit = { + val operatorData = targetMap.getOrElseUpdate(opRef, OperatorData(opRef)) + operatorData.count += 1 + operatorData.stages ++= stages + } + + // Processes an `ExecInfo` node to update exec and expression counts. + // Separates supported and unsupported execs and expressions into their respective maps. + private def processExecInfo(execInfo: ExecInfo): Unit = { + val opMap = execInfo.isSupported match { + case true => supportedMap + case false => unsupportedMap + } + updateOpRefEntry(execInfo.execRef, execInfo.stages, opMap) + // update the map for supported expressions. We should exclude the unsupported expressions. + execInfo.expressions.filterNot( + e => execInfo.unsupportedExprs.exists(exp => exp.opRef.equals(e))).foreach { expr => + updateOpRefEntry(expr, execInfo.stages, supportedMap) + } + // update the map for unsupported expressions + execInfo.unsupportedExprs.foreach { expr => + updateOpRefEntry(expr, execInfo.stages, unsupportedMap) + } + } + + // Counts the execs and expressions in the execution plan. + private def countOperators(): Unit = { + planInfo.execInfo.foreach { exec => + exec.isClusterNode match { + // we do not want to count the cluster nodes in that aggregation + case true => + if (exec.children.nonEmpty) { + exec.children.get.foreach { child => + processExecInfo(child) + } + } + case false => processExecInfo(exec) + } + } + } + + countOperators() +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefBase.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefBase.scala new file mode 100644 index 000000000..04bd9a190 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefBase.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + +import com.nvidia.spark.rapids.tool.planparser.OpTypes + +import org.apache.spark.sql.rapids.tool.util.StringUtils + +/** + * Base class representing a reference to an operator (either exec operator or expression operator). + * It provides methods to retrieve the operator's name and type in both raw and + * CSV-friendly formats. + * + * @param value The name of the operator. + * @param opType The type of the operator (e.g., Exec, Expr). + */ + +class OperatorRefBase(val value: String, val opType: OpTypes.OpType) extends OperatorRefTrait { + // Preformatted values for CSV output to avoid reformatting multiple times. + val csvValue: String = StringUtils.reformatCSVString(value) + val csvOpType: String = StringUtils.reformatCSVString(opType.toString) + + override def getOpName: String = value + override def getOpNameCSV: String = csvValue + override def getOpType: String = opType.toString + override def getOpTypeCSV: String = csvOpType +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefTrait.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefTrait.scala new file mode 100644 index 000000000..40385ba7a --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorRefTrait.scala @@ -0,0 +1,24 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + +trait OperatorRefTrait { + def getOpName: String + def getOpNameCSV: String + def getOpType: String + def getOpTypeCSV: String +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala new file mode 100644 index 000000000..0e17d8f0d --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + + +/** + * Represents a reference to an unsupported expression operator. + * Extends `OperatorRefBase` and includes a reason why the expression is unsupported. + * + * @param opRef The underlying `OpRef` for the expression. + * @param unsupportedReason A string describing why the expression is unsupported. + */ +case class UnsupportedExprOpRef(opRef: OpRef, + unsupportedReason: String) extends OperatorRefBase(opRef.value, opRef.opType) + +// Provides a factory method to create an instance from an expression name and unsupported reason. +object UnsupportedExprOpRef { + /** + * Creates an `UnsupportedExprOpRef` for the given expression name and unsupported reason. + * + * @param exprName The name of the unsupported expression. + * @param unsupportedReason A string describing why the expression is unsupported. + * @return An instance of `UnsupportedExprOpRef`. + */ + def apply(exprName: String, unsupportedReason: String): UnsupportedExprOpRef = { + val opRef = OpRef.fromExpr(exprName) + UnsupportedExprOpRef(opRef, unsupportedReason) + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala index f04ddcac7..44bbf9fa3 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala @@ -21,11 +21,11 @@ import scala.io.BufferedSource import scala.util.control.NonFatal import com.nvidia.spark.rapids.tool.{Platform, PlatformFactory} +import com.nvidia.spark.rapids.tool.planparser.ops.UnsupportedExprOpRef import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.Logging -import org.apache.spark.sql.rapids.tool.UnsupportedExpr import org.apache.spark.sql.rapids.tool.util.UTF8Source object OpSuppLevel extends Enumeration { @@ -387,11 +387,11 @@ class PluginTypeChecker(platform: Platform = PlatformFactory.createInstance(), exprSupported.isSupported } - def getNotSupportedExprs(exprs: Seq[String]): Seq[UnsupportedExpr] = { + def getNotSupportedExprs(exprs: Seq[String]): Seq[UnsupportedExprOpRef] = { exprs.collect { case expr if !isExprSupported(expr) => val reason = unsupportedOpsReasons.getOrElse(expr, "") - UnsupportedExpr(expr, reason) + UnsupportedExprOpRef(expr, reason) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala index ed101d820..9dbe7c920 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{Buffer, LinkedHashMap, ListBuffer} import com.nvidia.spark.rapids.tool.ToolTextFileWriter import com.nvidia.spark.rapids.tool.planparser.{DatabricksParseHelper, ExecInfo, PlanInfo, UnsupportedExecSummary} +import com.nvidia.spark.rapids.tool.planparser.ops.OperatorCounter import com.nvidia.spark.rapids.tool.profiling.AppStatusResult import com.nvidia.spark.rapids.tool.profiling.ProfileUtils.replaceDelimiter import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.{CLUSTER_ID, CLUSTER_ID_STR_SIZE, JOB_ID, JOB_ID_STR_SIZE, RUN_NAME, RUN_NAME_STR_SIZE, TEXT_DELIMITER} @@ -172,6 +173,24 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, } } + def writeAllOpsSummaryCSVReport( + sums: Seq[QualificationSummaryInfo]): Unit = { + val csvFileWriter = new ToolTextFileWriter(outputDir, + s"${QualOutputWriter.LOGFILE_NAME}_operatorsStats.csv", + "All Operators CSV Report", hadoopConf) + try { + val headersAndSizes = QualOutputWriter.getAllOperatorsHeaderStrings + csvFileWriter.write(QualOutputWriter.constructOutputRowFromMap(headersAndSizes, + QualOutputWriter.CSV_DELIMITER)) + sums.foreach { sum => + QualOutputWriter.constructAllOperatorsInfo(csvFileWriter, sum.planInfo, sum.appId, + QualOutputWriter.CSV_DELIMITER) + } + } finally { + csvFileWriter.close() + } + } + def writePerSqlCSVReport(sums: Seq[QualificationSummaryInfo], maxSQLDescLength: Int): Unit = { val csvFileWriter = new ToolTextFileWriter(outputDir, s"${QualOutputWriter.LOGFILE_NAME}_persql.csv", @@ -282,7 +301,7 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, } else { ("", "") } - Seq(appIDCSVStr, sqlIDStr, StringUtils.reformatCSVString(info.exec), + Seq(appIDCSVStr, sqlIDStr, info.execRef.getOpNameCSV, StringUtils.reformatCSVString(info.expr), if (info.duration.isDefined) info.duration.get.toString else zeroDurationStr, info.nodeId.toString, @@ -445,6 +464,11 @@ object QualOutputWriter { val UNSUPPORTED_TASK_DURATION_STR = "Unsupported Task Duration" val SUPPORTED_SQL_TASK_DURATION_STR = "Supported SQL DF Task Duration" val LONGEST_SQL_DURATION_STR = "Longest SQL Duration" + val OPERATOR_TYPE = "Operator Type" + val OPERATOR_NAME = "Operator Name" + val COUNT = "Count" + val IS_SUPPORTED = "Supported" + val STAGES = "Stages" val EXEC_STR = "Exec Name" val EXPR_STR = "Expression Name" val EXEC_DURATION = "Exec Duration" @@ -838,6 +862,49 @@ object QualOutputWriter { mutable.LinkedHashMap(headersAndFields: _*) } + private def getAllOperatorsHeaderStrings: LinkedHashMap[String, Int] = { + val detailedHeaderAndFields = LinkedHashMap[String, Int]( + APP_ID_STR -> APP_ID_STR.size, + SQL_ID_STR -> SQL_ID_STR.size, + OPERATOR_TYPE -> OPERATOR_TYPE.size, + OPERATOR_NAME -> OPERATOR_NAME.size, + COUNT -> COUNT.size, + IS_SUPPORTED -> IS_SUPPORTED.size, + STAGES -> STAGES.size + ) + detailedHeaderAndFields + } + + private def constructAllOperatorsInfo( + csvWriter: ToolTextFileWriter, + planInfos: Seq[PlanInfo], + appId: String, + delimiter: String): Unit = { + // This method iterates on PlanInfo which are sorted by SqlID. It constructs the operators per + // SQLPlan and sumps it to the CSV as a bufferedString instead of writing one row at a time. + val appIDCSVStr = StringUtils.reformatCSVString(appId) + val supportedCSVStr = "true" + val unsupportedCSVStr = "false" + planInfos.foreach { planInfo => + val sqlIDCSVStr = planInfo.sqlID.toString + val allOpsCount = OperatorCounter(planInfo) + .getOpsCountSummary().sortBy(oInfo => (-oInfo.opData.count, oInfo.opData.opRef.getOpName)) + if (allOpsCount.nonEmpty) { + val planBuffer = allOpsCount.map { opInfo => + val supportFlag = if (opInfo.isSupported) supportedCSVStr else unsupportedCSVStr + val stageStr = StringUtils.reformatCSVString(opInfo.opData.stages.mkString(":")) + s"$appIDCSVStr$delimiter" + + s"$sqlIDCSVStr$delimiter" + + s"${opInfo.opData.opRef.getOpTypeCSV}$delimiter" + + s"${opInfo.opData.opRef.getOpNameCSV}$delimiter${opInfo.opData.count}$delimiter" + + s"$supportFlag$delimiter" + + s"$stageStr" + } + csvWriter.write(s"${planBuffer.mkString("\n")}\n") + } + } + } + def constructClusterInfo( sumInfo: QualificationSummaryInfo, headersAndSizes: LinkedHashMap[String, Int], @@ -940,7 +1007,7 @@ object QualOutputWriter { // need to remove the WholeStageCodegen wrappers since they aren't actual // execs that we want to get timings of execs.flatMap { e => - if (e.exec.contains("WholeStageCodegen")) { + if (e.isClusterNode) { e.children.getOrElse(Seq.empty) } else { e.children.getOrElse(Seq.empty) :+ e @@ -1048,7 +1115,7 @@ object QualOutputWriter { stageId.toString -> headersAndSizes(STAGE_ID_STR), reformatCSVFunc(unSupExecInfo.execId.toString) -> headersAndSizes(EXEC_ID), reformatCSVFunc(unSupExecInfo.finalOpType) -> headersAndSizes(UNSUPPORTED_TYPE), - reformatCSVFunc(unSupExecInfo.unsupportedOperator) -> headersAndSizes(UNSUPPORTED_OPERATOR), + unSupExecInfo.unsupportedOperatorCSVFormat -> headersAndSizes(UNSUPPORTED_OPERATOR), reformatCSVFunc(unSupExecInfo.details) -> headersAndSizes(DETAILS), stageAppDuration.toString -> headersAndSizes(STAGE_WALLCLOCK_DUR_STR), appDuration.toString -> headersAndSizes(APP_DUR_STR), diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index 49d70b80e..fa59e8b3e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -255,6 +255,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, qWriter.writeExecReport(allAppsSum) qWriter.writeStageReport(allAppsSum, order) qWriter.writeUnsupportedOpsSummaryCSVReport(allAppsSum) + qWriter.writeAllOpsSummaryCSVReport(allAppsSum) val appStatusResult = generateStatusResults(appStatusReporter.asScala.values.toSeq) logOutputPath() qWriter.writeStatusReport(appStatusResult, order) diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index f8d4d7703..b90917cd8 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -378,11 +378,11 @@ abstract class AppBase( allMetaWithSchema.foreach { plan => val meta = plan.metadata val readSchema = ReadParser.formatSchemaStr(meta.getOrElse("ReadSchema", "")) - val scanNode = allNodes.filter(node => { + val scanNode = allNodes.filter(ReadParser.isScanNode(_)).filter(node => { // Get ReadSchema of each Node and sanitize it for comparison val trimmedNode = AppBase.trimSchema(ReadParser.parseReadNode(node).schema) readSchema.contains(trimmedNode) - }).filter(ReadParser.isScanNode(_)) + }) // If the ReadSchema is empty or if it is PhotonScan, then we don't need to // add it to the dataSourceInfo diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala index fc276a064..021337495 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala @@ -401,8 +401,6 @@ object ExecHelper { } } -case class UnsupportedExpr(exprName: String, unsupportedReason: String) - object MlOps { val sparkml = "spark.ml." val xgBoost = "spark.XGBoost" diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index 887075c8c..1ef8b7315 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -281,7 +281,7 @@ class QualificationAppInfo( // need to remove the WholeStageCodegen wrappers since they aren't actual // execs that we want to get timings of execs.flatMap { e => - if (e.exec.contains("WholeStageCodegen")) { + if (e.isClusterNode) { e.children.getOrElse(Seq.empty) } else { e.children.getOrElse(Seq.empty) :+ e @@ -573,7 +573,7 @@ class QualificationAppInfo( val unSupportedExecs = planInfos.flatMap { p => // WholeStageCodeGen is excluded from the result. val topLevelExecs = p.execInfo.filterNot(_.isSupported).filterNot( - x => x.exec.startsWith("WholeStage")) + x => x.isClusterNode) val childrenExecs = p.execInfo.flatMap { e => e.children.map(x => x.filterNot(_.isSupported)) }.flatten @@ -581,9 +581,8 @@ class QualificationAppInfo( }.map(_.exec).toSet.mkString(";").trim.replaceAll("\n", "").replace(",", ":") // Get all the unsupported Expressions from the plan - val unSupportedExprs = origPlanInfos.map(_.execInfo.flatMap( - _.unsupportedExprs.map(_.exprName))).flatten.filter(_.nonEmpty).toSet.mkString(";") - .trim.replaceAll("\n", "").replace(",", ":") + val unSupportedExprs = origPlanInfos.flatMap(p => p.getUnsupportedExpressions) + .map(s => s.getOpName).toSet.mkString(";").trim.replaceAll("\n", "").replace(",", ":") // TODO - this is not correct as this is using the straight stage wall // clock time and hasn't been adjusted to the app duration wall clock diff --git a/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv b/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv index 5c618e929..35d5d6f1b 100644 --- a/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.14,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan;SerializeFromObject","",30,1599 +"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.14,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan unknown;SerializeFromObject","",30,1599 diff --git a/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv b/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv index 5c618e929..35d5d6f1b 100644 --- a/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.14,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan;SerializeFromObject","",30,1599 +"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.14,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan unknown;SerializeFromObject","",30,1599 diff --git a/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv b/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv index df554c136..50e8986ba 100644 --- a/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","app-20211019113801-0001",2942,19894,571967,2814,28.41,"","JDBC[*]","","","","",1812,2883,569025,859,19035,false,"CollectLimit;Scan JDBCRelation(TBLS) [numPartitions=1];Execute CreateViewCommand","",30,9110 +"Spark shell","app-20211019113801-0001",2942,19894,571967,2814,28.41,"","JDBC[*]","","","","",1812,2883,569025,859,19035,false,"CollectLimit;Scan jdbc;Execute CreateViewCommand","",30,9110 diff --git a/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv b/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv index 2561e6ca5..428f89654 100644 --- a/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622561780883",0,40448,7673,0,55.94,"","","","","","",0,5000,7673,8096,32352,false,"Scan;SerializeFromObject","",30,82 +"Rapids Spark Profiling Tool Unit Tests","local-1622561780883",0,40448,7673,0,55.94,"","","","","","",0,5000,7673,8096,32352,false,"Scan unknown;SerializeFromObject","",30,82 diff --git a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv index a3a847ad0..62421d9bf 100644 --- a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv @@ -1,5 +1,5 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",11600,132257,16319,9868,37.7,"","","JSON","","","",7143,13770,4719,19744,112513,false,"SerializeFromObject;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements;Scan","",1,186 +"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",11600,132257,16319,9868,37.7,"","","JSON","","","",7143,13770,4719,19744,112513,false,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",1,186 "Spark shell","local-1651187225439",224,180,355637,74,87.88,"","JSON[string:bigint:int]","","","","",498,228,355101,120,60,false,"SerializeFromObject;CollectLimit;DeserializeToObject;Scan json;Filter;MapElements","",1,2834 "Spark shell","local-1651188809790",347,283,166215,14,81.18,"","JSON[string:bigint:int]","","","","UDF",715,318,165572,271,12,false,"CollectLimit;Scan json;Project","UDF",1,1318 "Rapids Spark Profiling Tool Unit Tests","local-1623281204390",1156,4666,6240,1,46.27,"","JSON[string:bigint:int]","JSON","","","UDF",1209,1130,5809,4661,5,false,"Execute InsertIntoHadoopFsRelationCommand json;LocalTableScan;Project;Scan json;Execute CreateViewCommand","UDF",1,64 diff --git a/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv b/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv index f7737d508..79cb8dede 100644 --- a/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",395,14353,4872,164,62.67,"","","JSON","","","",1306,794,4477,8376,5977,true,"SerializeFromObject;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements;Scan","",30,49 +"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",395,14353,4872,164,62.67,"","","JSON","","","",1306,794,4477,8376,5977,true,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",30,49 diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index 6fc9357cf..e31064851 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -276,12 +276,12 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parquet = allExecInfo.filter(_.exec.contains("Scan parquet")) val text = allExecInfo.filter(_.exec.contains("Scan text")) val csv = allExecInfo.filter(_.exec.contains("Scan csv")) - assertSizeAndNotSupported(2, json.toSeq) - assertSizeAndNotSupported(1, text.toSeq) + assertSizeAndNotSupported(2, json) + assertSizeAndNotSupported(1, text) for (t <- Seq(parquet, csv)) { - assertSizeAndSupported(1, t.toSeq) + assertSizeAndSupported(1, t) } - assertSizeAndSupported(2, orc.toSeq) + assertSizeAndSupported(2, orc) } test("BatchScan") { @@ -299,10 +299,10 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val orc = allExecInfo.filter(_.exec.contains("BatchScan orc")) val parquet = allExecInfo.filter(_.exec.contains("BatchScan parquet")) val csv = allExecInfo.filter(_.exec.contains("BatchScan csv")) - assertSizeAndNotSupported(3, json.toSeq) - assertSizeAndSupported(1, csv.toSeq) + assertSizeAndNotSupported(3, json) + assertSizeAndSupported(1, csv) for (t <- Seq(orc, parquet)) { - assertSizeAndSupported(2, t.toSeq) + assertSizeAndSupported(2, t) } } @@ -1224,7 +1224,8 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val projects = allExecInfo.filter(_.exec.contains("Project")) assertSizeAndNotSupported(1, projects) val expectedExprss = Seq("parse_url_ref", "parse_url_userinfo") - projects(0).unsupportedExprs.map(_.exprName) should contain theSameElementsAs expectedExprss + projects(0) + .unsupportedExprs.map(_.getOpName) should contain theSameElementsAs expectedExprss } } } @@ -1802,7 +1803,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) val windowExecNotSupportedExprs = allExecInfo.filter( _.exec.contains(windowGroupLimitExecCmd)).flatMap(x => x.unsupportedExprs) - windowExecNotSupportedExprs.head.exprName shouldEqual "row_number" + windowExecNotSupportedExprs.head.getOpName shouldEqual "row_number" windowExecNotSupportedExprs.head.unsupportedReason shouldEqual "Ranking function row_number is not supported in WindowGroupLimitExec" val windowGroupLimitExecs = allExecInfo.filter(_.exec.contains(windowGroupLimitExecCmd)) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala index 3e9988c41..6b7821d3f 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala @@ -241,6 +241,6 @@ class PluginTypeCheckerSuite extends FunSuite with Logging { val expressions = SQLPlanParser.parseAggregateExpressions(hashAggregateExpr) assert(expressions.contains("decimalsum")) val notSupportedExprs = checker.getNotSupportedExprs(expressions) - assert(notSupportedExprs.find(_.exprName == "decimalsum").isEmpty) + assert(notSupportedExprs.find(_.getOpName == "decimalsum").isEmpty) } } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 57b6f1cb3..03943d463 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -1066,7 +1066,7 @@ class QualificationSuite extends BaseTestSuite { val stdOutunsupportedExecs = stdOutValues(stdOutValues.length - 3) // index of unsupportedExprs val stdOutunsupportedExprs = stdOutValues(stdOutValues.length - 2) - val expectedstdOutExecs = "Scan;Filter;SerializeF..." + val expectedstdOutExecs = "Scan unknown;Filter;Se..." assert(stdOutunsupportedExecs == expectedstdOutExecs) // Exec value is Scan;Filter;SerializeFromObject and UNSUPPORTED_EXECS_MAX_SIZE is 25 val expectedStdOutExecsMaxLength = 25 @@ -1096,7 +1096,7 @@ class QualificationSuite extends BaseTestSuite { val rows = outputActual.collect() assert(rows.size == 1) - val expectedExecs = "Scan;Filter;SerializeFromObject" // Unsupported Execs + val expectedExecs = "Scan unknown;Filter;SerializeFromObject" // Unsupported Execs val expectedExprs = "hex" //Unsupported Exprs val unsupportedExecs = outputActual.select(QualOutputWriter.UNSUPPORTED_EXECS).first.getString(0) diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py index 320615b7c..1bdbd76b7 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py @@ -108,7 +108,12 @@ 'sqlOp_CustomShuffleReader', 'sqlOp_DeserializeToObject', 'sqlOp_Exchange', - 'sqlOp_Execute InsertIntoHadoopFsRelationCommand', + 'sqlOp_Execute InsertIntoHadoopFsRelationCommand csv', + 'sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet', + 'sqlOp_Execute InsertIntoHadoopFsRelationCommand orc', + 'sqlOp_Execute InsertIntoHadoopFsRelationCommand json', + 'sqlOp_Execute InsertIntoHadoopFsRelationCommand text', + 'sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown', 'sqlOp_Expand', 'sqlOp_Filter', 'sqlOp_Generate', @@ -125,15 +130,17 @@ 'sqlOp_Project', 'sqlOp_ReusedSort', 'sqlOp_RunningWindowFunction', - 'sqlOp_Scan csv ', + 'sqlOp_Scan csv', 'sqlOp_Scan ExistingRDD Delta Table Checkpoint', 'sqlOp_Scan ExistingRDD Delta Table State', - 'sqlOp_Scan JDBCRelation', - 'sqlOp_Scan json ', + 'sqlOp_Scan ExistingRDD', + 'sqlOp_Scan jdbc', + 'sqlOp_Scan json', 'sqlOp_Scan OneRowRelation', - 'sqlOp_Scan orc ', - 'sqlOp_Scan parquet ', - 'sqlOp_Scan text ', + 'sqlOp_Scan orc', + 'sqlOp_Scan parquet', + 'sqlOp_Scan text', + 'sqlOp_Scan unknown', 'sqlOp_SerializeFromObject', 'sqlOp_Sort', 'sqlOp_SortAggregate', @@ -481,8 +488,8 @@ def combine_tables(table_name: str) -> pd.DataFrame: 'Scan DeltaCDFRelation', 'Scan ExistingRDD Delta Table Checkpoint', 'Scan ExistingRDD Delta Table State', - 'Scan JDBCRelation', - 'Scan parquet ', # trailing space is also in default sql op name + 'Scan jdbc', + 'Scan parquet', # GPU 'GpuScan parquet', ] From a414e095b85858ab73f0d8684d3e1627b04eefee Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Wed, 4 Dec 2024 18:25:25 -0800 Subject: [PATCH 11/52] [BUG] Fix sync plugin files script to handle empty or non-existing cvs files Signed-off-by: cindyyuanjiang --- scripts/sync_plugin_files/process_supported_files.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/scripts/sync_plugin_files/process_supported_files.py b/scripts/sync_plugin_files/process_supported_files.py index 8ff1ba3c9..9eecf8ac2 100644 --- a/scripts/sync_plugin_files/process_supported_files.py +++ b/scripts/sync_plugin_files/process_supported_files.py @@ -118,7 +118,11 @@ def unify_all_files(root_dir, file_name, key_names): for dir_name in os.listdir(root_dir): # List entries in root_dir if os.path.isdir(os.path.join(root_dir, dir_name)): csv_file_path = os.path.join(root_dir, dir_name, file_name) - cur_df = pd.read_csv(csv_file_path, keep_default_na=False) + try: + cur_df = pd.read_csv(csv_file_path, keep_default_na=False) + except Exception as e: + logging.debug(f"Encountered exception when reading file {csv_file_path}: {e}") + continue if final_df.empty: final_df = pd.DataFrame(columns=cur_df.columns.tolist()) From 746c8986e0c607d3dd6beb81952798837538185d Mon Sep 17 00:00:00 2001 From: Lee Yang Date: Thu, 5 Dec 2024 11:27:51 -0800 Subject: [PATCH 12/52] Update models for latest tools code (#1448) * updated models for latest tools code * remove deprecated models Signed-off-by: Lee Yang --------- Signed-off-by: Lee Yang --- .../qualx/models/xgboost/combined.cfg | 2 +- .../qualx/models/xgboost/combined.json | 2 +- .../qualx/models/xgboost/combined.metrics | 219 ++++++++--------- .../qualx/models/xgboost/databricks-aws.cfg | 2 +- .../qualx/models/xgboost/databricks-aws.json | 2 +- .../models/xgboost/databricks-aws.metrics | 123 +++++----- .../models/xgboost/databricks-aws_photon.cfg | 2 +- .../models/xgboost/databricks-aws_photon.json | 2 +- .../xgboost/databricks-aws_photon.metrics | 139 +++++------ .../qualx/models/xgboost/databricks-azure.cfg | 2 +- .../models/xgboost/databricks-azure.json | 2 +- .../models/xgboost/databricks-azure.metrics | 139 +++++------ .../xgboost/databricks-azure_photon.cfg | 2 +- .../xgboost/databricks-azure_photon.json | 2 +- .../xgboost/databricks-azure_photon.metrics | 135 ++++++----- .../qualx/models/xgboost/dataproc.cfg | 2 +- .../qualx/models/xgboost/dataproc.json | 2 +- .../qualx/models/xgboost/dataproc.metrics | 221 +++++++++--------- .../models/xgboost/dataproc_2.0-parquet.json | 1 - .../models/xgboost/dataproc_2.1-orc.json | 1 - .../resources/qualx/models/xgboost/emr.cfg | 2 +- .../resources/qualx/models/xgboost/emr.json | 2 +- .../qualx/models/xgboost/emr.metrics | 127 +++++----- .../resources/qualx/models/xgboost/onprem.cfg | 2 +- .../qualx/models/xgboost/onprem.json | 2 +- .../qualx/models/xgboost/onprem.metrics | 109 +++++---- 26 files changed, 650 insertions(+), 596 deletions(-) delete mode 100644 user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.0-parquet.json delete mode 100644 user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.1-orc.json diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg index f84abdfcb..8cbac1ad7 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0478905365","gamma":"0.100665279","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0478905365","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.100665279","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.799700022"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"93"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0452739485","gamma":"0.0653549582","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0452739485","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"8","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.0653549582","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.662151933"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json index d08771d24..2526694b7 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-2.3933945E-3,-5.9263664E-1,2.6889783E-1,-9.1858613E-1,-2.1448609E-1,4.9850196E-2,7.607807E-1,-9.524582E-1,1.082049E-1,-4.2231518E-1,3.841283E-1,-2.0045005E-1,2.5073895E-1,5.913415E-1,1.2293434E0,-1.190826E0,-6.499661E-1,-5.2304554E-1,-1.684352E-1,1.654881E-1,7.682251E-1,-6.403749E-1,-1.0299838E-1,2.2019845E-1,1.7205881E0,3.6116946E-1,8.6512417E-1,9.544972E-1,1.447596E0,-1.0850258E0,-1.5838625E0,-3.0255994E-1,-1.0274159E0,-6.542544E-1,-3.6114734E-1,2.1063672E-2,-3.827689E-1,-1.6058318E-1,3.2476264E-1,1.0358812E0,6.4064044E-1,-6.7629725E-1,1.9147405E-2,-3.798304E-1,3.208436E-3,2.7569512E-1,-5.907691E-1,9.582441E-2,3.9070837E-2,4.5508566E-1,-3.4440452E-1,6.501266E-1,1.1846459E0,1.0611738E0,4.215443E-1,9.425399E-3,1.4771441E0,-6.6020197E-1,-1.1486143E0,-8.24079E-2,-5.5061117E-2,4.055765E-2,-3.4508678E-1,-1.3218734E0,-6.474342E-1,-7.689773E-1,-5.070937E-1,-4.139987E-1,-2.1466441E-1,-6.077923E-2,2.2719845E-1,-4.9959147E-1,1.3236758E-1,-3.058238E-1,-1.4064024E-2,1.03090085E-1,4.891102E-1,5.986518E-2,1.9258194E-2,1.481983E-2,7.018445E-1,-6.000221E-1,-8.9893275E-1,-4.98677E-1,-2.3440248E-1,7.2169036E-2,-5.789142E-2,6.985793E-2,4.658221E-1,-7.8216726E-1,-2.4077396E-1,2.9188117E-1,6.169904E-1,-2.985779E-2,-1.7024064E-1,7.72275E-1,-1.2616562E-3,1.0792186E0,7.851262E-2,8.0599916E-1,1.2145987E0,1.318254E-2,2.8760185E-2,5.3556785E-2,1.5661538E0,-3.5902992E-2,-4.7232355E-3,-5.6290977E-2,-2.8847083E-2,-2.3156878E-2,-3.9114323E-3,-5.3385925E-2,-9.192074E-2,-1.2863264E-2,-3.9320957E-2,-3.7759576E-2,-6.2052947E-3,-2.586171E-2,-4.2914865E-3,-2.1443246E-2,-1.0495426E-2,-3.7772711E-3,-1.5231074E-2,-6.4352313E-3,1.1099501E-2,2.5020547E-3,1.4326472E-2,-3.558328E-2,-1.7009383E-2,-3.764246E-3,1.3154044E-2,-1.3399529E-3,-1.7454095E-2,7.5460942E-3,-4.2518647E-3,-1.3406342E-3,1.0195194E-2,4.4437973E-3,2.4880974E-2,1.3480483E-2,3.6080826E-2,-3.3894286E-2,-1.9205628E-2,-4.6135005E-2,-7.1078218E-3,-2.7161289E-2,-1.3822419E-2,9.33951E-3,-1.5613237E-2,-5.1571312E-3,1.618328E-2,-7.5569376E-3,8.243618E-3,3.0937996E-2,1.4483788E-2,-3.968076E-2,-5.711906E-3,-1.3762629E-2,3.4719768E-3,5.2895495E-3,2.007394E-2,1.8349549E-2,3.8173135E-2,-1.20216245E-2,-1.8143258E-3,3.894546E-2,1.0547749E-2,-3.2969955E-2,1.0913452E-2,5.2875895E-2,1.022835E-2,4.322535E-2,1.2377302E-2,3.060715E-2,6.0913827E-2,7.693966E-2,3.8323175E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,-1,-1,91,93,95,97,99,101,-1,103,105,107,-1,-1,-1,109,111,113,115,117,119,121,123,125,127,129,131,133,135,137,-1,-1,-1,139,141,143,145,147,-1,149,151,153,155,157,159,161,-1,163,165,167,169,-1,171,173,-1,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2257657E2,1.0223035E2,1.9476643E2,5.0161987E1,4.8184288E1,6.300396E1,4.3494873E1,3.109616E1,0E0,7.2514725E0,8.270768E0,2.3856287E1,3.0819637E1,2.5587585E1,7.4362183E0,8.510925E0,2.5521439E1,4.1184235E0,3.3791897E0,3.4279172E0,5.6572723E-1,4.0443726E0,1.3467082E1,3.085696E1,1.9182358E0,1.506632E1,1.207605E1,3.4187164E0,2.6659546E0,4.563431E0,9.930954E-1,5.375991E0,9.66169E0,1.5009193E0,6.3885975E-1,7.7298903E-1,2.4131331E0,4.6387058E-1,1.5525565E0,8.5456944E-1,3.3010578E-1,1.1248093E0,0E0,2.078125E0,3.059103E1,2.4974857E1,2.7591639E0,0E0,0E0,5.078205E0,1.2060947E0,9.0715065E0,1.7786636E0,1.21344E0,1.7507887E-1,0E0,7.3979187E-1,1.3331614E0,1.7802887E0,0E0,0E0,0E0,3.6439438E0,4.2959213E0,2.607813E0,6.575699E-1,6.663475E-1,3.4912968E-1,3.5392582E-1,7.416297E-1,1.4801759E-1,8.610544E-1,2.6175827E-1,1.6900599E-1,1.6788095E-1,2.938071E-1,2.5327396E-1,0E0,0E0,0E0,2.647257E-1,1.345295E0,9.688587E-1,8.287754E-1,2.3847637E0,0E0,6.3433666E0,7.198537E0,9.579071E0,7.625027E-1,2.6902348E-1,2.2860737E0,3.8638573E0,0E0,1.829462E-1,1.9906616E0,3.1487076E0,1.1210556E0,0E0,1.0359983E0,2.7605438E-1,0E0,0E0,0E0,5.718994E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,80,80,81,81,82,82,83,83,84,84,86,86,87,87,88,88,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,99,99,100,100,104,104],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,-1,-1,92,94,96,98,100,102,-1,104,106,108,-1,-1,-1,110,112,114,116,118,120,122,124,126,128,130,132,134,136,138,-1,-1,-1,140,142,144,146,148,-1,150,152,154,156,158,160,162,-1,164,166,168,170,-1,172,174,-1,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.14099E5,5.549173E6,2.2178302E5,1.8137958E6,3.386E3,1.082049E-1,3.3926086E2,2.496172E3,8.2E1,3.7002478E3,1.3664E4,5.964E3,4.0844156E7,6.747114E7,1.913925E-1,1.3304372E7,1.8140173E-1,5.015707E0,1.5427417E3,6.76E2,3.0820766E-1,1.7324902E8,3.9712732E7,1.5103707E0,1E0,1.545974E-1,2.461362E-3,9.6771875E2,2.66E2,1.0052E4,1.8122449E2,2.7538432E8,5.5871747E2,6.306E3,7.0014185E6,3.647E3,2.9956698E-1,1.3137E4,1.4504054E8,1.9147405E-2,7.2896E5,2.255481E6,9.52E2,2.6476662E6,9.582441E-2,3.9070837E-2,2.5682794E5,1.4813794E3,4.2620764E2,1.386749E1,2.9251662E6,1.3170732E0,9.425399E-3,2.3920168E6,2.701189E8,5.8426323E0,-8.24079E-2,-5.5061117E-2,4.055765E-2,6.316106E6,9.222717E-1,1.272E3,1.1994927E7,8.982707E9,5.426937E5,1E1,6.4860556E5,2.028994E8,1.221875E1,3.3E1,1.06158945E5,5.911314E1,2.4513194E5,2.028994E8,5.986518E-2,1.9258194E-2,1.481983E-2,5.7064955E6,5.222222E1,1E0,2.2247495E-1,3.28825E0,7.2169036E-2,8.426E3,4.97E2,3.71E0,1E0,3.989822E1,1.2939234E5,8.8543115E2,-2.985779E-2,3.4540886E3,6.971004E7,2.222456E0,6.13912E5,7.851262E-2,1.766486E7,1.460806E6,1.318254E-2,2.8760185E-2,5.3556785E-2,6.292039E0,-3.5902992E-2,-4.7232355E-3,-5.6290977E-2,-2.8847083E-2,-2.3156878E-2,-3.9114323E-3,-5.3385925E-2,-9.192074E-2,-1.2863264E-2,-3.9320957E-2,-3.7759576E-2,-6.2052947E-3,-2.586171E-2,-4.2914865E-3,-2.1443246E-2,-1.0495426E-2,-3.7772711E-3,-1.5231074E-2,-6.4352313E-3,1.1099501E-2,2.5020547E-3,1.4326472E-2,-3.558328E-2,-1.7009383E-2,-3.764246E-3,1.3154044E-2,-1.3399529E-3,-1.7454095E-2,7.5460942E-3,-4.2518647E-3,-1.3406342E-3,1.0195194E-2,4.4437973E-3,2.4880974E-2,1.3480483E-2,3.6080826E-2,-3.3894286E-2,-1.9205628E-2,-4.6135005E-2,-7.1078218E-3,-2.7161289E-2,-1.3822419E-2,9.33951E-3,-1.5613237E-2,-5.1571312E-3,1.618328E-2,-7.5569376E-3,8.243618E-3,3.0937996E-2,1.4483788E-2,-3.968076E-2,-5.711906E-3,-1.3762629E-2,3.4719768E-3,5.2895495E-3,2.007394E-2,1.8349549E-2,3.8173135E-2,-1.20216245E-2,-1.8143258E-3,3.894546E-2,1.0547749E-2,-3.2969955E-2,1.0913452E-2,5.2875895E-2,1.022835E-2,4.322535E-2,1.2377302E-2,3.060715E-2,6.0913827E-2,7.693966E-2,3.8323175E-2],"split_indices":[20,102,12,2,51,28,28,2,0,52,4,29,52,2,2,45,45,41,45,38,54,4,2,27,7,45,41,6,38,27,4,0,2,52,7,52,2,51,2,38,2,31,0,9,31,2,32,0,0,28,4,58,35,28,53,0,28,12,53,0,0,0,1,27,0,9,5,47,3,33,7,58,3,33,58,28,7,0,0,0,32,4,68,38,54,0,2,2,54,74,58,33,52,0,4,45,54,2,0,47,47,0,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.637E3,8.3E2,1.807E3,4.45E2,3.85E2,1.251E3,5.56E2,4.41E2,4E0,2.86E2,9.9E1,5.57E2,6.94E2,4.1E2,1.46E2,2.45E2,1.96E2,2.04E2,8.2E1,6.4E1,3.5E1,1E2,4.57E2,6.81E2,1.3E1,2.24E2,1.86E2,6.7E1,7.9E1,1.96E2,4.9E1,1.03E2,9.3E1,1.11E2,9.3E1,4.4E1,3.8E1,2.1E1,4.3E1,9E0,2.6E1,9.7E1,3E0,1.26E2,3.31E2,6.38E2,4.3E1,9E0,4E0,1.98E2,2.6E1,1.13E2,7.3E1,5.5E1,1.2E1,2E0,7.7E1,2.7E1,1.69E2,3.5E1,1.4E1,3E0,1E2,5.1E1,4.2E1,6E1,5.1E1,6.7E1,2.6E1,3.2E1,1.2E1,3.1E1,7E0,1E1,1.1E1,1.9E1,2.4E1,6E0,3E0,5E0,2.1E1,7.5E1,2.2E1,6.8E1,5.8E1,1.2E1,3.19E2,3.07E2,3.31E2,2.7E1,1.6E1,1E2,9.8E1,9E0,1.7E1,9.5E1,1.8E1,6.2E1,1.1E1,2.3E1,3.2E1,8E0,4E0,1.8E1,5.9E1,2.3E1,4E0,1.6E2,9E0,6.5E1,3.5E1,4E1,1.1E1,1.4E1,2.8E1,5.8E1,2E0,4.7E1,4E0,5.6E1,1.1E1,1.2E1,1.4E1,2.6E1,6E0,4E0,8E0,1E1,2.1E1,3E0,4E0,2E0,8E0,3E0,8E0,9E0,1E1,2E0,2.2E1,3E0,1.8E1,4.7E1,2.8E1,2E1,2E0,5E1,1.8E1,1E1,4.8E1,2.84E2,3.5E1,9.5E1,2.12E2,1.56E2,1.75E2,2.5E1,2E0,1.4E1,2E0,4.2E1,5.8E1,4.4E1,5.4E1,1E1,7E0,8.8E1,7E0,4E0,1.4E1,6E1,2E0,1.9E1,4E0,4E0,2.8E1,5.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-2.3714243E-3,-5.6937283E-1,2.632269E-1,-8.611649E-1,-2.1411869E-1,5.843058E-2,7.091735E-1,-9.001495E-1,1.0970145E-1,-4.0091532E-1,3.5433906E-1,-1.7773639E-1,2.4572168E-1,8.125841E-1,-2.1678058E-2,-1.1528195E0,-5.955599E-1,-4.9700186E-1,-1.660928E-1,1.5033236E-1,6.9017434E-1,-4.293215E-1,-1.3976596E-2,3.0475536E-1,-5.242693E-1,6.9015497E-1,1.2918309E0,-3.9108118E-1,5.0852233E-1,-1.047478E0,-1.5900015E0,-2.785633E-1,-9.6850884E-1,-6.040674E-1,-3.522944E-1,-5.0136417E-2,-1.10359766E-1,-1.9317546E-1,3.103282E-1,8.4424776E-1,4.6831298E-1,-6.551769E-1,-3.1109655E-1,1.366152E0,-7.375315E-2,7.9924315E-2,5.056863E-1,-6.0004544E-1,2.3809645E-2,4.9802318E-1,9.267325E-1,1.3728242E0,4.1750902E-1,-5.172177E-1,1.4290325E-1,6.738396E-1,-2.6505345E-1,-1.1137598E0,-6.214992E-1,-8.415815E-2,-5.3442948E-2,-3.4109232E-1,6.082151E-1,-1.257475E0,-5.9185696E-1,-7.346779E-1,-4.6774274E-1,-4.6425366E-1,-2.6155004E-1,2.4459746E-2,-2.3934168E-1,-2.3751712E-1,6.5737586E-3,1.5395092E-3,4.4827098E-1,5.2593436E-2,3.146254E-2,2.4698006E-2,3.600609E-3,-5.8996284E-1,-6.88253E-2,-6.0252374E-1,-2.3930793E-1,2.1151744E-2,7.104212E-2,-1.564757E-1,2.7012524E-1,-2.5477833E-1,1.5236986E-1,4.733558E-1,8.602649E-2,-3.6391646E-1,-3.977876E-2,2.5720194E-1,6.832873E-1,8.212044E-1,1.2729123E0,1.4012694E0,2.9624726E-3,2.8644571E-2,1.2195575E-2,-6.585593E-1,-2.8485823E-1,-3.0871911E-3,2.0029142E-2,6.137451E-3,8.038528E-1,2.440875E-3,-2.0870985E-2,-5.3829897E-2,-6.897391E-3,-3.3818252E-2,-2.7286527E-3,3.9100908E-2,-1.8272914E-2,2.4593093E-3,3.7607342E-2,-4.8421297E-2,-7.940924E-2,-8.505313E-4,-3.2663096E-2,-3.6118884E-2,-6.0071778E-3,-2.3973383E-2,-7.0663393E-3,-2.3054577E-2,-4.693853E-3,-1.621451E-2,-6.3083717E-3,-6.4284313E-3,5.80315E-3,-1.5587123E-2,1.13891335E-2,-2.1632304E-3,-1.475134E-2,1.2172776E-2,2.767851E-2,-2.8860107E-2,-5.0891167E-3,-2.229037E-2,-4.2480927E-2,-1.0742514E-3,-1.453044E-2,-5.429632E-3,-3.5057046E-2,8.080705E-3,2.9354332E-2,-7.101869E-3,-1.7226916E-2,7.898379E-3,-4.544447E-2,1.831193E-2,3.5269514E-2,-3.2701466E-2,-8.317099E-3,3.5670513E-4,1.553319E-2,1.7630273E-2,4.005947E-2,4.3862082E-2,2.4472235E-2,3.050482E-2,6.543778E-2,3.5831463E-2,6.891499E-2,-1.0401175E-2,-3.4483846E-2,-1.8825023E-2,-5.029285E-3,2.8034959E-2,5.8235735E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,87,89,91,-1,93,95,97,99,101,103,105,107,109,111,-1,-1,113,115,117,119,121,123,125,127,129,131,133,-1,-1,135,-1,-1,-1,-1,137,-1,139,141,-1,-1,143,145,147,149,151,-1,153,-1,155,157,159,161,163,-1,-1,-1,165,167,-1,-1,-1,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.026893E2,8.8237885E1,1.6626724E2,5.9566467E1,4.112986E1,5.528941E1,4.33237E1,3.4946228E1,0E0,6.483368E0,6.4734583E0,2.277734E1,3.1842194E1,2.8562683E1,1.4300626E1,9.939972E0,2.4793594E1,2.9787827E0,4.106191E0,3.412496E0,8.352127E-1,5.617153E0,2.7820534E1,2.923331E1,4.168356E0,1.781099E1,6.579071E0,2.9701748E0,4.0151677E0,5.1241455E0,1.766571E0,6.6395864E0,9.81321E0,1.7395973E0,7.929449E-1,0E0,1.4317751E0,3.2564998E-1,1.5972629E0,2.2972393E-1,2.7804232E-1,2.9257603E0,2.8893795E0,8.626785E-1,9.230486E0,7.482868E0,1.3420235E1,2.2566376E0,0E0,9.818138E0,5.493866E0,3.3440552E0,1.418258E-1,9.6187687E-1,5.407983E-1,1.6706333E0,3.459807E-1,1.5991211E0,1.3935089E0,0E0,0E0,5.416959E0,7.4846697E-1,3.6544037E0,2.2658472E0,5.9511185E-1,5.5712414E-1,1.9774723E-1,4.7538614E-1,6.4553094E-1,1.8035471E0,2.3019171E-1,0E0,0E0,5.578661E-1,0E0,0E0,0E0,0E0,3.6473846E-1,0E0,6.7917824E-1,1.6336422E0,0E0,0E0,6.391383E0,2.0864224E0,5.612302E-1,3.6548967E0,7.7378616E0,0E0,1.3982625E0,0E0,1.6578064E0,5.775181E0,3.6436539E0,1.5078278E0,1.1416321E0,0E0,0E0,0E0,4.136114E-1,2.506739E-1,0E0,0E0,0E0,1.0889463E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,74,74,79,79,81,81,82,82,85,85,86,86,87,87,88,88,89,89,91,91,93,93,94,94,95,95,96,96,97,97,101,101,102,102,106,106],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,88,90,92,-1,94,96,98,100,102,104,106,108,110,112,-1,-1,114,116,118,120,122,124,126,128,130,132,134,-1,-1,136,-1,-1,-1,-1,138,-1,140,142,-1,-1,144,146,148,150,152,-1,154,-1,156,158,160,162,164,-1,-1,-1,166,168,-1,-1,-1,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.14099E5,5.549173E6,2.1592189E5,3.9712732E7,3.386E3,1.0970145E-1,3.3926086E2,1.2060912E3,9.99E2,3.0820766E-1,2.9251662E6,3.970405E3,4.8297736E7,6.747114E7,1.6659623E-1,5.2143492E-5,1.4126303E0,3.1E1,3.439199E7,8E0,9.52E2,3.1998687E3,1.2679E4,1E0,7.418546E1,2.2576077E10,1.5E1,9.6771875E2,7.3610186E3,1.1879E4,1.8122449E2,2.1102592E8,-5.0136417E-2,1.1884672E7,1.0323588E3,2.091623E5,5.015707E0,3.2829E4,4E0,1.7E1,2.127E3,1.2817779E7,3.79E2,1.0409E4,4.2343444E7,2.3809645E-2,3.202814E5,6.082581E3,9.256843E3,2.84E2,2.28125E0,1.3664E4,7.3610186E3,2.14099E5,9.12179E6,9.7637427E-1,-8.415815E-2,-5.3442948E-2,2.66E2,8.06E2,1.0828989E8,7.6599895E9,1.1994927E7,3.906E3,5.1100562E5,1.2690893E0,2.612E3,6.624E3,7.44393E5,6.5737586E-3,1.5395092E-3,3.904E3,5.2593436E-2,3.146254E-2,2.4698006E-2,3.600609E-3,9.213145E0,-6.88253E-2,1.00033E5,1.3797468E0,2.1151744E-2,7.104212E-2,1E0,1.2502964E9,3.824611E0,5.432133E0,2.2214102E6,8.602649E-2,9.4168E4,-3.977876E-2,4.423E3,3.596E3,2.3058404E7,5.0279167E1,4.6253732E2,2.9624726E-3,2.8644571E-2,1.2195575E-2,2.652E3,3.474851E8,-3.0871911E-3,2.0029142E-2,6.137451E-3,2.3828971E-1,2.440875E-3,-2.0870985E-2,-5.3829897E-2,-6.897391E-3,-3.3818252E-2,-2.7286527E-3,3.9100908E-2,-1.8272914E-2,2.4593093E-3,3.7607342E-2,-4.8421297E-2,-7.940924E-2,-8.505313E-4,-3.2663096E-2,-3.6118884E-2,-6.0071778E-3,-2.3973383E-2,-7.0663393E-3,-2.3054577E-2,-4.693853E-3,-1.621451E-2,-6.3083717E-3,-6.4284313E-3,5.80315E-3,-1.5587123E-2,1.13891335E-2,-2.1632304E-3,-1.475134E-2,1.2172776E-2,2.767851E-2,-2.8860107E-2,-5.0891167E-3,-2.229037E-2,-4.2480927E-2,-1.0742514E-3,-1.453044E-2,-5.429632E-3,-3.5057046E-2,8.080705E-3,2.9354332E-2,-7.101869E-3,-1.7226916E-2,7.898379E-3,-4.544447E-2,1.831193E-2,3.5269514E-2,-3.2701466E-2,-8.317099E-3,3.5670513E-4,1.553319E-2,1.7630273E-2,4.005947E-2,4.3862082E-2,2.4472235E-2,3.050482E-2,6.543778E-2,3.5831463E-2,6.891499E-2,-1.0401175E-2,-3.4483846E-2,-1.8825023E-2,-5.029285E-3,2.8034959E-2,5.8235735E-2],"split_indices":[20,102,12,2,51,28,45,2,0,52,52,2,27,28,52,45,45,39,42,41,3,7,18,2,4,2,6,56,12,3,4,4,2,52,7,0,45,52,28,54,9,6,0,1,9,2,2,45,0,28,52,4,0,54,2,4,2,9,42,0,0,0,0,45,5,9,0,47,42,2,2,1,0,0,2,0,0,0,0,53,0,5,53,0,0,14,7,54,53,28,0,1,0,2,2,45,56,4,0,0,0,10,7,0,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.672E3,8.52E2,1.82E3,4.67E2,3.85E2,1.248E3,5.72E2,4.62E2,5E0,2.9E2,9.5E1,5.52E2,6.96E2,5.01E2,7.1E1,2.51E2,2.11E2,2.05E2,8.5E1,6E1,3.5E1,2.17E2,3.35E2,6.47E2,4.9E1,4.01E2,1E2,4.2E1,2.9E1,2.05E2,4.6E1,1.15E2,9.6E1,1.16E2,8.9E1,4E0,8.1E1,1.9E1,4.1E1,1.9E1,1.6E1,7.3E1,1.44E2,1.3E1,3.22E2,3.06E2,3.41E2,4.6E1,3E0,2.23E2,1.78E2,9.1E1,9E0,3.4E1,8E0,2.4E1,5E0,1.76E2,2.9E1,3.2E1,1.4E1,1.08E2,7E0,5.3E1,4.3E1,5.7E1,5.9E1,3.8E1,5.1E1,4E1,4.1E1,1.7E1,2E0,1.4E1,2.7E1,6E0,1.3E1,1.4E1,2E0,6.9E1,4E0,2.7E1,1.17E2,2E0,1.1E1,2.6E2,6.2E1,5.4E1,2.52E2,3.34E2,7E0,2.4E1,2.2E1,9.8E1,1.25E2,1.39E2,3.9E1,8.9E1,2E0,3E0,6E0,2E1,1.4E1,5E0,3E0,5E0,1.9E1,2E0,3E0,1.74E2,2E0,2.5E1,4E0,3E0,1.05E2,2E0,5E0,3.5E1,1.8E1,6E0,3.7E1,5.5E1,2E0,5.3E1,6E0,3.6E1,2E0,3.1E1,2E1,1.5E1,2.5E1,3.5E1,6E0,5E0,1.2E1,1.2E1,1.5E1,6.7E1,2E0,2E1,7E0,2.7E1,9E1,2.43E2,1.7E1,4.9E1,1.3E1,2.8E1,2.6E1,2.5E2,2E0,2.5E2,8.4E1,8E0,1.6E1,2.1E1,7.7E1,4.2E1,8.3E1,1.05E2,3.4E1,6E0,3.3E1,6E0,8.3E1,3E0,1.7E1,8E0,6E0,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[1.1071307E-3,-5.423456E-1,2.477801E-1,-8.41369E-1,-2.0129503E-1,1.2935925E-2,6.322977E-1,-1.017471E0,-3.7494534E-1,-4.0880737E-1,2.7471045E-1,-2.020943E-1,2.133659E-1,-4.2133304E-1,6.871309E-1,-8.673482E-1,-1.3602864E0,9.552879E-2,-4.5081005E-1,-5.1557684E-1,-2.1397609E-1,1.659792E-1,8.755924E-1,-3.7122184E-1,-2.086004E-3,2.631779E-1,-4.971364E-1,-1.14391066E-1,-5.080421E-1,4.3807676E-1,8.842364E-1,-9.792807E-1,-1.8303722E-1,-7.906136E-2,-1.0933759E0,-1.8364438E-1,-6.817106E-1,-6.383278E-1,-3.5833102E-1,-4.4184007E-2,-1.7400676E-1,5.937926E-2,4.08212E-1,4.4052668E-2,1.6647978E-2,-6.254603E-1,-2.617054E-1,7.01121E-2,-4.3626767E-2,9.80237E-2,4.5375454E-1,-6.410622E-1,2.5239038E-1,-1.3428412E-2,3.5675636E-3,-5.421011E-1,-2.6277422E-3,5.6580454E-1,2.8934479E-2,5.7499295E-1,1.0716008E0,-1.0271367E0,-5.5844575E-1,8.560279E-2,-4.738174E-1,-8.5298234E-1,-1.4473763E0,-2.5177908E-1,3.7454486E-2,-8.6696786E-1,-3.957296E-1,-7.214756E-1,-4.5520848E-1,-4.507801E-1,-2.412056E-1,-3.286371E-1,-8.4966786E-2,-1.4040019E-1,2.2271283E-1,2.6220378E-1,2.8265363E-2,-5.502463E-1,-4.5966916E-2,-2.0271505E-1,-5.411945E-1,-9.6603416E-2,4.7651988E-1,-2.3675628E-1,1.6351043E-1,3.493417E-1,8.2898444E-1,-6.9881845E-1,-8.517995E-3,-7.0675695E-4,2.9381186E-2,-2.7989402E-2,-8.317738E-3,3.3609986E-1,7.483398E-1,-7.860974E-1,1.3036136E-1,4.3768713E-1,9.2287767E-1,1.2026979E0,7.386259E-1,-5.167524E-2,-3.7510544E-2,-3.2135352E-2,-7.012149E-3,3.089976E-2,-1.1816963E-3,6.715307E-4,-2.6076894E-2,-4.425726E-2,-1.2234967E-2,-7.9474166E-2,-3.236669E-2,-3.444067E-3,-1.8337598E-2,-3.3463713E-2,-6.072954E-2,-1.3819378E-2,-4.500773E-2,-3.6008183E-2,-2.294122E-2,-2.9578067E-3,-2.4297133E-2,-5.169944E-4,-2.260335E-2,-2.353277E-3,-1.400775E-2,-4.175997E-3,-2.0382361E-2,-1.0091006E-2,2.084057E-3,-1.122155E-2,4.743048E-3,1.3938326E-2,7.7124435E-4,-9.910276E-4,1.515929E-2,-3.2590084E-2,-1.3340968E-2,-1.3620337E-2,1.8964648E-3,-3.496037E-2,-2.0541077E-2,-3.8837392E-3,-4.468688E-2,-9.579503E-3,2.6362702E-2,-3.6163644E-3,-1.46387955E-2,4.6837595E-3,1.9107463E-2,1.4221742E-2,3.8845684E-2,2.584793E-2,4.380022E-2,-1.3829837E-2,-3.5200674E-2,1.6466076E-3,1.9209769E-2,4.8949435E-2,2.8015452E-2,-4.0800646E-3,-4.8155468E-2,-1.5659416E-4,1.3060679E-2,-1.3518512E-2,2.3233196E-2,4.808888E-2,-1.878078E-3,6.864334E-2,5.0247166E-2,3.8324706E-2,-1.0213288E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,75,77,79,-1,-1,81,83,-1,85,87,89,91,93,-1,-1,95,-1,97,99,101,103,105,107,109,111,113,115,117,-1,119,121,123,125,127,129,131,133,135,137,139,-1,141,-1,143,145,147,149,151,153,155,157,159,-1,-1,-1,-1,-1,161,163,165,167,169,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5725714E2,8.467975E1,1.655149E2,3.6005707E1,3.863533E1,4.9132317E1,4.0442444E1,1.53402405E1,2.3011557E1,5.5478325E0,7.629492E0,1.8604027E1,2.1011847E1,8.7339735E-1,3.2061096E1,1.7131592E1,5.6408844E0,0E0,7.2774506E0,3.149128E0,2.6151524E0,2.6096046E0,1.0094166E-1,8.13361E0,1.54722395E1,1.7329464E1,4.368972E0,3.0309436E-1,3.8698864E-1,1.5347729E1,2.0706177E1,3.3334808E0,2.6183615E0,0E0,3.2756844E0,3.9474967E0,3.047638E0,1.1485023E0,7.466917E-1,0E0,1.266295E0,2.3806784E0,6.6900444E-1,0E0,0E0,1.7101631E0,3.3525963E0,0E0,6.852273E0,6.5500345E0,9.740181E0,7.5854015E-1,6.843557E-1,0E0,0E0,2.8850317E-1,0E0,9.140625E0,5.971773E0,6.3311577E0,9.064331E0,1.344223E0,9.2215633E-1,1.1267533E0,5.5942464E-1,1.2093525E0,2.1515617E0,1.2460558E0,0E0,1.6779404E0,1.3031058E0,1.1903381E-1,6.456218E-1,3.959837E-1,3.5011506E-1,7.588725E-1,9.9468243E-1,7.65288E-1,5.600376E-1,2.9861236E-1,0E0,2.4646645E0,0E0,3.4753609E0,4.5005894E-1,2.8429904E0,1.2609515E0,5.2141476E-1,3.8136616E0,4.645075E0,8.682747E-1,2.1825981E-1,0E0,0E0,0E0,0E0,0E0,1.9675417E0,4.947426E0,1.1447501E0,1.2183943E0,3.6552372E0,3.0885353E0,4.2310486E0,3.1550903E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,45,45,46,46,48,48,49,49,50,50,51,51,52,52,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,76,78,80,-1,-1,82,84,-1,86,88,90,92,94,-1,-1,96,-1,98,100,102,104,106,108,110,112,114,116,118,-1,120,122,124,126,128,130,132,134,136,138,140,-1,142,-1,144,146,148,150,152,154,156,158,160,-1,-1,-1,-1,-1,162,164,166,168,170,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.781059E4,1.983922E3,4.7931265E6,2.2178302E5,5.836513E3,3.9712732E7,1.4E1,2.728889E2,1.91E4,2.914E3,4.898012E-1,2.1506184E7,1.1410706E3,3.386E3,1.0443182E3,9.552879E-2,6.747114E7,1.7751849E-1,5.2143492E-5,4.7519747E5,3.07125E0,9.6E1,1.7E1,9.99E2,2.5422776E3,1.4E2,1E0,3.8398104E0,2.3421426E-1,2.1E1,2.1515152E0,-7.906136E-2,1.775894E10,1.5137865E2,3.0497742E3,1.998E3,2.264894E8,-4.4184007E-2,2.822937E5,1.7427321E-1,1.8835404E3,4.4052668E-2,1.6647978E-2,1.3597637E8,1.044E1,7.01121E-2,1.4777102E9,3.97E2,2.3314082E6,3.2177323E-3,8.426E3,-1.3428412E-2,3.5675636E-3,5.964E3,-2.6277422E-3,2.4434292E7,2.960909E2,5.57E4,8.5E1,2.5604828E2,6.3E0,2.66E2,1.0508E5,1.1879E4,8.289561E8,2.8839298E7,3.7454486E-2,1.2427474E8,2.12E2,2.4859156E1,2.8169732E10,1.59E2,1.822542E0,1.3062E4,7.59093E5,8.631E3,1.4187837E-1,6.143686E6,2.8265363E-2,4.3875E1,-4.5966916E-2,8.61E2,4.0911578E2,5.388794E6,2.142857E0,3.5791788E0,3.321532E6,9.908038E1,5.3038636E7,3.051499E5,-8.517995E-3,-7.0675695E-4,2.9381186E-2,-2.7989402E-2,-8.317738E-3,2.3161087E2,7E0,2.5672606E8,5.441E3,1.4972717E-2,7.218466E7,1.4307411E7,3.7284137E2,-5.167524E-2,-3.7510544E-2,-3.2135352E-2,-7.012149E-3,3.089976E-2,-1.1816963E-3,6.715307E-4,-2.6076894E-2,-4.425726E-2,-1.2234967E-2,-7.9474166E-2,-3.236669E-2,-3.444067E-3,-1.8337598E-2,-3.3463713E-2,-6.072954E-2,-1.3819378E-2,-4.500773E-2,-3.6008183E-2,-2.294122E-2,-2.9578067E-3,-2.4297133E-2,-5.169944E-4,-2.260335E-2,-2.353277E-3,-1.400775E-2,-4.175997E-3,-2.0382361E-2,-1.0091006E-2,2.084057E-3,-1.122155E-2,4.743048E-3,1.3938326E-2,7.7124435E-4,-9.910276E-4,1.515929E-2,-3.2590084E-2,-1.3340968E-2,-1.3620337E-2,1.8964648E-3,-3.496037E-2,-2.0541077E-2,-3.8837392E-3,-4.468688E-2,-9.579503E-3,2.6362702E-2,-3.6163644E-3,-1.46387955E-2,4.6837595E-3,1.9107463E-2,1.4221742E-2,3.8845684E-2,2.584793E-2,4.380022E-2,-1.3829837E-2,-3.5200674E-2,1.6466076E-3,1.9209769E-2,4.8949435E-2,2.8015452E-2,-4.0800646E-3,-4.8155468E-2,-1.5659416E-4,1.3060679E-2,-1.3518512E-2,2.3233196E-2,4.808888E-2,-1.878078E-3,6.864334E-2,5.0247166E-2,3.8324706E-2,-1.0213288E-3],"split_indices":[20,102,33,55,51,28,28,45,0,52,2,2,27,45,52,2,4,0,45,41,42,28,57,29,0,2,52,10,68,53,38,3,53,0,5,56,52,10,7,0,28,38,4,0,0,31,54,0,7,2,28,38,2,0,0,2,0,12,52,2,8,52,58,0,29,2,7,45,0,45,8,58,31,11,58,9,1,2,27,50,0,4,0,2,4,47,54,54,28,56,7,28,0,0,0,0,0,4,3,31,2,38,45,51,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.663E3,8.31E2,1.832E3,4.42E2,3.89E2,1.138E3,6.94E2,3.2E2,1.22E2,2.71E2,1.18E2,5.49E2,5.89E2,3.4E1,6.6E2,2.25E2,9.5E1,3E0,1.19E2,1.74E2,9.7E1,1.01E2,1.7E1,2.97E2,2.52E2,5.51E2,3.8E1,8E0,2.6E1,2.93E2,3.67E2,1.93E2,3.2E1,4.3E1,5.2E1,5.6E1,6.3E1,9.6E1,7.8E1,4E0,9.3E1,7.1E1,3E1,1.5E1,2E0,8.8E1,2.09E2,6E0,2.46E2,2.96E2,2.55E2,3.2E1,6E0,4E0,4E0,2.4E1,2E0,2.23E2,7E1,1.4E2,2.27E2,1.72E2,2.1E1,1.7E1,1.5E1,3.3E1,1.9E1,5.3E1,3E0,3.7E1,2.6E1,6.4E1,3.2E1,4.2E1,3.6E1,3.3E1,6E1,3.2E1,3.9E1,1.8E1,1.2E1,7.4E1,1.4E1,1.74E2,3.5E1,2.24E2,2.2E1,4.8E1,2.48E2,2.01E2,5.4E1,2.8E1,4E0,4E0,2E0,2.1E1,3E0,1E2,1.23E2,7E0,6.3E1,1.02E2,3.8E1,1.61E2,6.6E1,1.39E2,3.3E1,1.6E1,5E0,2E0,1.5E1,2E0,1.3E1,2.9E1,4E0,1.4E1,5E0,2.3E1,3E1,2.8E1,9E0,2.3E1,3E0,5.5E1,9E0,4E0,2.8E1,2E0,4E1,8E0,2.8E1,1E1,2.3E1,3E1,3E1,2.3E1,9E0,2.9E1,1E1,3E0,1.5E1,4.9E1,2.5E1,1.3E2,4.4E1,1.1E1,2.4E1,2.21E2,3E0,2E0,2E1,1.5E1,3.3E1,1.95E2,5.3E1,1.82E2,1.9E1,1.4E1,4E1,3E0,2.5E1,1.8E1,8.2E1,4.4E1,7.9E1,2E0,5E0,3.3E1,3E1,6E0,9.6E1,3.5E1,3E0,6.1E1,1E2,6.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-4.919165E-3,-5.127597E-1,2.3879103E-1,-7.9191464E-1,-1.7278804E-1,1.0360127E-2,5.744305E-1,-9.3548167E-1,-2.1089284E-1,-3.733383E-1,3.110595E-1,-3.844976E-1,9.87894E-2,-5.0353825E-2,6.7176384E-1,-5.1685107E-1,-1.0391344E0,9.2097595E-2,-3.004092E-1,-4.4944045E-1,-1.4406104E-1,1.22466266E-1,6.40427E-1,-5.205553E-1,-2.5133985E-1,2.6613197E-1,-8.421548E-2,-4.7398776E-1,2.2826439E-1,4.2708096E-1,8.6057603E-1,-7.2208554E-1,-1.9926143E-1,-9.6880615E-1,-1.6317617E0,-3.0516336E-2,-6.135284E-1,-5.636162E-1,-3.0649674E-1,-4.206998E-2,-9.874249E-2,-9.315113E-2,2.32138E-1,8.1261045E-1,4.103946E-1,-5.969056E-1,-3.8630342E-1,-2.7414072E-1,-1.4157817E-2,1.2882808E0,2.3131421E-1,-1.10023074E-1,8.274286E-2,-5.2044743E-1,-2.2309323E-1,4.899046E-1,-5.8427315E-2,8.5922915E-1,3.1470641E-1,5.6706154E-1,1.0399625E0,-1.4199626E-2,-7.7713436E-1,-4.8259678E-1,1.1423624E-1,-1.0583283E0,-6.140792E-1,-9.734037E-1,-9.093996E-2,-1.4891629E-1,4.698151E-1,-5.237221E-1,-5.7052564E-2,-5.186377E-1,-8.766293E-1,-4.1703E-1,-2.070744E-1,-1.3190204E-1,1.0012114E-2,-1.3954552E-2,-4.7818264E-3,2.5336344E-2,3.4767216E-1,1.32296225E-2,4.0732723E-2,3.2350381E-3,2.236119E-2,-2.9924694E-2,-1.15515925E-2,-1.9704202E-2,-1.3809703E-1,-1.8779661E-1,-3.5827935E-1,-5.774251E-3,5.5730506E-3,2.1002414E-2,6.634151E-2,-1.4144881E-1,3.2282695E-1,-4.6645034E-2,-5.456144E-1,-1.7495487E-2,-3.0887377E-2,2.9410354E-3,-1.5715923E-2,3.3676726E-1,3.5996564E-2,3.8872745E-2,-1.5845139E-2,4.7522017E-1,9.510711E-1,1.4229986E-1,4.537174E-1,4.4853738E-1,8.564297E-1,1.1075033E0,5.902776E-1,-4.0111583E-2,-1.3649273E-2,-7.0425444E-3,-2.9220473E-2,2.3889191E-2,1.1332703E-3,-5.2024394E-2,-2.5440563E-2,-1.2739209E-2,-3.600956E-2,-1.3584908E-2,-5.244782E-2,-1.1711691E-2,6.8403166E-3,3.4070555E-2,-3.374226E-3,-3.20843E-2,-1.1737632E-2,-2.8140582E-2,-1.7909383E-2,-4.5041524E-2,-1.6770821E-2,-2.0713795E-2,-2.9264675E-3,-1.0821908E-2,3.6486143E-3,-1.2323991E-2,-1.6360426E-3,8.529433E-3,-6.048879E-3,-7.3080612E-3,4.778744E-3,5.3772903E-3,2.018379E-2,1.7572939E-3,-1.2779862E-2,-1.315036E-2,-4.6435213E-3,2.2946522E-3,-1.8094562E-2,-1.24904215E-2,4.8844684E-3,8.581845E-3,2.4710488E-2,-9.423928E-3,4.403247E-3,-4.1492075E-2,-7.264296E-3,1.8601334E-2,-5.7371575E-3,-1.5103351E-2,4.1965568E-3,2.6947062E-2,1.0667516E-2,9.782774E-3,4.6983264E-2,8.852071E-3,-2.1807175E-2,2.488308E-2,5.222937E-3,2.3928111E-2,1.5389657E-3,2.7770624E-2,4.6568908E-2,5.967394E-2,4.4764254E-2,3.6715064E-2,3.5426198E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,83,85,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,113,115,-1,117,119,121,123,125,127,-1,129,131,133,-1,135,137,139,141,143,-1,-1,145,147,149,-1,-1,-1,-1,-1,-1,-1,151,153,155,-1,-1,-1,-1,157,159,161,163,-1,-1,-1,-1,165,-1,167,-1,169,171,173,175,177,179,181,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3243393E2,8.252524E1,1.3915088E2,3.9675354E1,3.834822E1,3.778324E1,4.4730896E1,1.6082123E1,1.8916203E1,4.803383E0,7.127082E0,3.456356E0,2.7094479E1,1.1911408E1,2.9015076E1,4.9401817E0,1.11727295E1,0E0,7.840581E0,3.2382812E0,2.2762773E0,1.7951586E0,1.3300343E0,7.636776E-1,5.4773045E-1,1.6144096E1,1.9970615E1,3.501501E-1,4.6042395E0,1.3278519E1,1.8259888E1,8.706894E-1,2.8975291E0,8.114746E0,3.6039124E0,3.0911632E0,1.6072922E0,1.1644135E0,9.5764446E-1,0E0,7.216298E-1,4.5406762E-1,1.1858242E0,2.3293018E-1,3.560667E-1,4.4579697E-1,1.8028927E-1,6.083498E-1,1.529316E-1,6.2875557E-1,1.53280945E1,1.1514244E1,0E0,4.0176678E-1,2.5929692E-1,1.0294905E0,8.080976E-1,1.5583191E0,5.2799416E0,4.3488655E0,6.0584717E0,0E0,9.806614E-1,6.37872E-1,5.455323E-1,2.6540527E0,2.5826187E0,6.13122E-1,0E0,1.2067871E0,1.3554792E0,1.424799E0,0E0,8.0791855E-1,1.0782242E-1,2.1621132E-1,2.955656E-1,7.5132704E-1,0E0,0E0,4.4476962E-1,2.6692042E-1,5.084567E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2232128E-1,3.51812E-1,3.8343716E-1,0E0,0E0,0E0,0E0,2.612753E0,9.894775E0,7.633507E0,6.565894E0,0E0,0E0,0E0,0E0,5.6528807E-1,0E0,4.2487645E-1,0E0,1.6172051E-1,8.0724335E-1,2.6520016E0,2.7355404E0,2.1119423E0,6.869602E-1,3.4075012E0,2.7661333E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,77,77,80,80,81,81,82,82,90,90,91,91,92,92,97,97,98,98,99,99,100,100,105,105,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,84,86,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,114,116,-1,118,120,122,124,126,128,-1,130,132,134,-1,136,138,140,142,144,-1,-1,146,148,150,-1,-1,-1,-1,-1,-1,-1,152,154,156,-1,-1,-1,-1,158,160,162,164,-1,-1,-1,-1,166,-1,168,-1,170,172,174,176,178,180,182,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,7.7344055E4,2.4519731E3,4.7931265E6,3.97E2,2.1328075E-1,1.2478469E-1,1.4E1,3.5734247E2,1.2060912E3,2.0550128E5,1.9E1,2.6393727E4,1.1410706E3,2.373E3,1.0017953E8,9.2097595E-2,8.229907E7,1.8329959E-1,5.2143492E-5,1.502494E0,8.7643677E-1,9.6E1,8.397659E6,2.04E5,6.7652373E3,1.6213043E3,3.5E0,8E0,2.3421426E-1,1.7515824E8,1E0,3.83E3,2.8681312E8,8.015419E3,9.22E4,1.5988282E1,2.1102592E8,-4.206998E-2,2.1446484E-1,3.4614954E8,2.821E3,2.6882867E8,1.4008022E11,6.2285713E1,2.73E2,9.25E0,3.598933E6,7.03E2,2.0407547E5,4.8297736E7,8.274286E-2,7.218466E7,1.92E2,2.3990374E8,3.9343938E-1,2.0131812E7,1E0,3.166E3,2.8091298E7,-1.4199626E-2,3.008162E6,1.68635E5,1.376072E8,5.8426323E0,2.7246006E7,2.7324794E11,-9.093996E-2,7.0764465E1,1.4433751E0,6.5317163E3,-5.7052564E-2,2.728E3,6.895858E-1,1.1075305E0,3.9382784E9,5.4878284E2,1.0012114E-2,-1.3954552E-2,9.035491E5,2.0783027E5,2.1924436E5,1.32296225E-2,4.0732723E-2,3.2350381E-3,2.236119E-2,-2.9924694E-2,-1.15515925E-2,-1.9704202E-2,3.1E1,1.109125E3,1.1485E4,-5.774251E-3,5.5730506E-3,2.1002414E-2,6.634151E-2,2.001177E6,1.009E3,7.59093E5,3.1998687E3,-1.7495487E-2,-3.0887377E-2,2.9410354E-3,-1.5715923E-2,1E0,3.5996564E-2,4.238096E6,-1.5845139E-2,1.3878379E0,1.5166431E5,4.7058824E-1,8.447369E0,3.5481934E1,5.57E4,2E1,7.4709034E-1,-4.0111583E-2,-1.3649273E-2,-7.0425444E-3,-2.9220473E-2,2.3889191E-2,1.1332703E-3,-5.2024394E-2,-2.5440563E-2,-1.2739209E-2,-3.600956E-2,-1.3584908E-2,-5.244782E-2,-1.1711691E-2,6.8403166E-3,3.4070555E-2,-3.374226E-3,-3.20843E-2,-1.1737632E-2,-2.8140582E-2,-1.7909383E-2,-4.5041524E-2,-1.6770821E-2,-2.0713795E-2,-2.9264675E-3,-1.0821908E-2,3.6486143E-3,-1.2323991E-2,-1.6360426E-3,8.529433E-3,-6.048879E-3,-7.3080612E-3,4.778744E-3,5.3772903E-3,2.018379E-2,1.7572939E-3,-1.2779862E-2,-1.315036E-2,-4.6435213E-3,2.2946522E-3,-1.8094562E-2,-1.24904215E-2,4.8844684E-3,8.581845E-3,2.4710488E-2,-9.423928E-3,4.403247E-3,-4.1492075E-2,-7.264296E-3,1.8601334E-2,-5.7371575E-3,-1.5103351E-2,4.1965568E-3,2.6947062E-2,1.0667516E-2,9.782774E-3,4.6983264E-2,8.852071E-3,-2.1807175E-2,2.488308E-2,5.222937E-3,2.3928111E-2,1.5389657E-3,2.7770624E-2,4.6568908E-2,5.967394E-2,4.4764254E-2,3.6715064E-2,3.5426198E-3],"split_indices":[20,102,33,4,51,2,42,27,0,52,52,45,3,28,52,2,45,0,45,41,42,41,57,29,9,5,52,4,53,3,38,31,16,2,7,4,10,54,7,0,38,12,2,7,31,4,2,58,5,1,28,45,0,45,0,7,27,12,100,0,45,0,32,29,12,53,45,31,0,56,42,4,0,10,27,38,5,52,0,0,47,33,28,0,0,0,0,0,0,0,0,33,9,0,0,0,0,9,2,1,4,0,0,0,0,74,0,45,0,53,28,57,53,57,2,3,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.684E3,8.7E2,1.814E3,4.77E2,3.93E2,1.08E3,7.34E2,3.82E2,9.5E1,2.78E2,1.15E2,1.97E2,8.83E2,9.9E1,6.35E2,7.7E1,3.05E2,3E0,9.2E1,2.08E2,7E1,7.4E1,4.1E1,9.6E1,1.01E2,4.61E2,4.22E2,3.9E1,6E1,2.78E2,3.57E2,4.6E1,3.1E1,2.75E2,3E1,5E1,4.2E1,1.14E2,9.4E1,3E0,6.7E1,2.5E1,4.9E1,2.2E1,1.9E1,5.9E1,3.7E1,9.2E1,9E0,1.4E1,4.47E2,4.17E2,5E0,3.2E1,7E0,3.1E1,2.9E1,5.6E1,2.22E2,1.37E2,2.2E2,6E0,4E1,1.6E1,1.5E1,2.18E2,5.7E1,1E1,2E1,4.1E1,9E0,3.8E1,4E0,1.02E2,1.2E1,4.3E1,5.1E1,6.1E1,6E0,7E0,1.8E1,1.8E1,3.1E1,2E0,2E1,3E0,1.6E1,5.4E1,5E0,3.3E1,4E0,4.7E1,4.5E1,5E0,4E0,2E0,1.2E1,8.8E1,3.59E2,3.65E2,5.2E1,1.6E1,1.6E1,2E0,5E0,2.1E1,1E1,2.2E1,7E0,1.2E1,4.4E1,1E2,1.22E2,9.9E1,3.8E1,1.9E2,3E1,3.5E1,5E0,5E0,1.1E1,2E0,1.3E1,2.06E2,1.2E1,1.7E1,4E1,2E0,8E0,3.1E1,1E1,6E0,3E0,2.4E1,1.4E1,6.7E1,3.5E1,1E1,2E0,4.1E1,2E0,4.8E1,3E0,2.6E1,3.5E1,7E0,1.1E1,5E0,1.3E1,8E0,2.3E1,2E0,2E0,2.3E1,2.4E1,2E0,4.3E1,5.9E1,2.9E1,2.07E2,1.52E2,1.75E2,1.9E2,2.8E1,2.4E1,1.9E1,2E0,2E0,2E1,8E0,4E0,2E0,4.2E1,9.4E1,6E0,1.02E2,2E1,8.8E1,1.1E1,1.3E1,2.5E1,1.02E2,8.8E1,2.2E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[1.7515587E-2,-4.6577635E-1,2.4062452E-1,-7.76435E-1,-1.8065654E-1,2.9554037E-2,5.8481735E-1,-9.203482E-1,-4.3783516E-1,-3.3206242E-1,4.0224496E-1,-7.918369E-2,3.9286548E-1,-3.128538E-2,6.600262E-1,-9.911532E-1,-5.4222184E-1,-2.9339272E-1,-6.0541105E-1,-3.7570697E-1,8.998138E-2,1.10121034E-1,6.057744E-1,-5.34453E-1,-2.2682665E-2,3.3058777E-1,8.8398254E-1,-4.3784672E-1,1.7259692E-1,5.795156E-1,1.1303889E0,-8.783423E-1,-1.4966913E0,6.742432E-2,-6.690063E-1,-3.5230425E-1,1.0886428E-2,-6.979792E-1,-3.0585936E-1,-2.740333E-1,-7.346481E-1,3.601082E-1,-2.8722864E-2,6.762017E-1,1.1629193E-1,-5.77323E-1,4.795034E-2,-3.5621222E-2,7.944069E-2,3.6245024E-1,-4.225629E-1,2.2578161E-2,4.8785433E-2,-5.0905085E-1,-2.1400604E-1,3.2812512E-1,-1.9784933E-1,5.1131433E-1,1.0553775E0,6.180007E-1,1.2166445E0,-9.576119E-1,-6.388863E-1,-7.4846774E-2,-4.0186353E-2,-6.3578864E-3,1.2170062E-2,-7.243893E-1,-1.1669412E-2,-2.3466805E-2,-2.7618378E-1,-1.22268766E-1,1.4738983E-2,-7.5107783E-1,-3.2518956E-1,-1.7730802E-2,3.5946374E-3,-3.2666022E-1,3.3884788E-1,-5.068287E-1,-1.0011654E0,6.2573394E-3,1.967377E-2,-1.7891224E-2,1.5581794E-2,3.160023E-1,7.650123E-1,-5.546192E-3,1.33478055E-2,-4.8409703E-1,-8.481132E-1,-1.1109948E-1,1.6773078E-1,2.156619E-1,5.625935E-1,-8.9622214E-2,-3.532306E-2,-2.3697845E-3,-5.6020683E-1,2.8471376E-3,-1.5088721E-2,7.415243E-2,4.3272308E-1,-1.3762907E-2,2.7106209E-3,2.4210797E-1,5.873564E-1,5.1964873E-1,1.2608923E0,7.1616924E-1,-6.4156565E-4,1.2823193E0,4.981187E-1,-4.640231E-2,-4.5691077E-3,4.3105887E-4,-3.2505058E-2,-1.7590953E-2,-3.847332E-2,-6.4417283E-4,-1.54947825E-2,-1.1906166E-2,-8.458317E-4,-3.7037075E-2,-1.1880717E-2,-3.48473E-3,-1.9712064E-2,-3.897682E-2,-1.3119226E-2,8.864441E-3,3.0739471E-2,-3.4407414E-2,-1.4815022E-2,-5.1590413E-2,-2.2342328E-2,-4.0430613E-3,4.455926E-3,1.9970935E-2,5.3344383E-3,2.938278E-2,4.7768764E-2,-2.537272E-2,1.0806537E-3,-4.3794144E-2,-4.8348783E-3,-1.0770009E-3,-1.2388245E-2,6.208467E-3,3.1455684E-2,1.1108762E-2,-2.694045E-2,1.908882E-2,3.362981E-2,4.281918E-3,-1.4293335E-2,-7.31503E-3,-2.8942747E-2,5.8901254E-3,-5.428953E-3,1.2272494E-2,2.7090317E-2,5.605291E-3,2.8689759E-2,2.9984437E-2,4.7241547E-3,3.1016782E-2,2.9132725E-4,6.332771E-2,1.291666E-2,1.2427995E-2,3.9615847E-2,3.2217834E-2,6.318827E-2,3.3706088E-2,1.1203404E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,-1,93,95,-1,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,-1,-1,117,-1,-1,119,121,-1,123,125,-1,-1,127,129,131,133,-1,-1,-1,135,137,139,-1,-1,141,143,145,147,149,151,153,-1,-1,155,-1,-1,157,159,-1,-1,161,163,165,167,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8984253E2,7.5059875E1,1.3358415E2,1.9380997E1,3.9318665E1,4.5107067E1,3.240976E1,7.066162E0,2.7996006E0,3.4769104E1,5.403226E0,2.2598555E1,7.751793E0,6.4581523E0,2.2937439E1,1.2258606E1,3.677909E0,1.2227731E0,1.3393364E0,1.2462399E1,0E0,1.3640018E0,1.7829018E0,6.964094E0,1.7093975E1,5.792507E0,1.1312428E0,3.0545807E-1,3.060891E0,1.6791E1,3.225792E0,3.1275635E0,5.162964E-1,3.7010124E-1,7.421913E-1,4.6215057E-1,5.119994E-1,5.871315E-1,3.9562404E-1,8.883785E0,4.107136E0,1.0918355E-1,4.0307605E-1,1.2617626E0,3.338671E-1,1.9876747E0,0E0,1.1959888E1,0E0,6.518255E0,9.827137E-1,0E0,0E0,4.18962E-1,2.3981732E-1,9.5081854E-1,3.807733E-1,9.458702E0,6.611389E0,9.393444E-1,2.9476242E0,1.3315887E0,1.348669E0,0E0,0E0,0E0,0E0,6.636505E-1,0E0,0E0,4.8094654E-1,1.1455661E-1,0E0,2.117939E-1,1.2274653E-1,0E0,0E0,6.2239876E0,9.0616107E-1,1.5748081E0,8.8058853E-1,0E0,0E0,0E0,1.9377315E-1,1.8089831E-1,6.907463E-1,0E0,0E0,1.7025833E0,1.0478802E0,7.421955E0,3.8459063E0,1.8048329E0,1.8955288E0,2.510165E-1,0E0,0E0,2.1347618E-1,0E0,0E0,1.2383856E-1,4.6500874E-1,0E0,0E0,4.6174064E0,6.849945E0,1.3109026E0,2.5060577E0,4.445734E-1,0E0,6.733322E-1,2.7007997E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,67,67,70,70,71,71,73,73,74,74,77,77,78,78,79,79,80,80,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,98,98,101,101,102,102,105,105,106,106,107,107,108,108,109,109,111,111,112,112],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,-1,94,96,-1,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,-1,-1,118,-1,-1,120,122,-1,124,126,-1,-1,128,130,132,134,-1,-1,-1,136,138,140,-1,-1,142,144,146,148,150,152,154,-1,-1,156,-1,-1,158,160,-1,-1,162,164,166,168,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,1.8532948E6,1E0,1.6990049E0,1.2727361E6,2.5998926E-1,5.121E3,5.1148495E6,1.3144558E4,1.3097832E3,8.2E1,7.064848E6,3.731E3,3.2196458E6,4.0844156E7,2.1693E5,2.3871907E2,1.998E3,6.747114E7,8.998138E-2,1.3510204E1,7.2234793E0,1.5445492E3,9.256843E3,6.3300834E0,1E0,1.6213043E3,1.8159722E-1,6.082581E3,1E0,1.1E1,1E0,1.4783E4,3.4409692E3,1.009E3,2.695E3,2.4859156E1,3.771981E6,2.0601392E3,1.4396E4,6.58542E5,2.5534486E4,2.7271206E5,1E0,1.4504054E8,4.795034E-2,3.422351E6,7.944069E-2,1.06E3,1E0,2.2578161E-2,4.8785433E-2,3.7246967E2,1.92E2,3.502483E-2,3.4540886E3,1E0,2.1150263E-1,2.777588E6,1E0,9.12179E6,2.08E2,-7.4846774E-2,-4.0186353E-2,-6.3578864E-3,1.2170062E-2,1.8357558E5,-1.1669412E-2,-2.3466805E-2,1.246372E0,1.732E3,1.4738983E-2,2.1989189E2,2.8E1,-1.7730802E-2,3.5946374E-3,1E0,7.3610186E3,2.8207336E3,3.4986075E6,6.2573394E-3,1.967377E-2,-1.7891224E-2,2.821E3,2.011653E7,3.3791902E0,-5.546192E-3,1.33478055E-2,6.8E1,1E0,5.024605E0,1.5963264E9,4.6E1,3.6129813E2,5E0,-3.532306E-2,-2.3697845E-3,1.401E3,2.8471376E-3,-1.5088721E-2,6.513793E1,4.3800272E8,-1.3762907E-2,2.7106209E-3,1.3508893E3,3.5301748E7,1.921032E4,1.515252E10,9.8E1,-6.4156565E-4,1.2062E5,1.85369E5,-4.640231E-2,-4.5691077E-3,4.3105887E-4,-3.2505058E-2,-1.7590953E-2,-3.847332E-2,-6.4417283E-4,-1.54947825E-2,-1.1906166E-2,-8.458317E-4,-3.7037075E-2,-1.1880717E-2,-3.48473E-3,-1.9712064E-2,-3.897682E-2,-1.3119226E-2,8.864441E-3,3.0739471E-2,-3.4407414E-2,-1.4815022E-2,-5.1590413E-2,-2.2342328E-2,-4.0430613E-3,4.455926E-3,1.9970935E-2,5.3344383E-3,2.938278E-2,4.7768764E-2,-2.537272E-2,1.0806537E-3,-4.3794144E-2,-4.8348783E-3,-1.0770009E-3,-1.2388245E-2,6.208467E-3,3.1455684E-2,1.1108762E-2,-2.694045E-2,1.908882E-2,3.362981E-2,4.281918E-3,-1.4293335E-2,-7.31503E-3,-2.8942747E-2,5.8901254E-3,-5.428953E-3,1.2272494E-2,2.7090317E-2,5.605291E-3,2.8689759E-2,2.9984437E-2,4.7241547E-3,3.1016782E-2,2.9132725E-4,6.332771E-2,1.291666E-2,1.2427995E-2,3.9615847E-2,3.2217834E-2,6.318827E-2,3.3706088E-2,1.1203404E-2],"split_indices":[20,56,51,102,41,28,42,2,45,52,4,29,28,29,28,45,29,52,10,45,0,58,57,52,4,53,102,4,27,52,100,3,105,9,52,2,2,58,1,52,9,1,33,28,102,31,0,9,0,2,84,0,0,4,0,38,4,100,38,1,6,9,2,0,0,0,0,32,0,0,53,2,0,52,3,0,0,75,4,55,32,0,0,0,2,32,35,0,0,0,68,53,7,3,52,8,0,0,2,0,0,56,7,0,0,52,45,4,5,0,0,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.686E3,8.48E2,1.838E3,4.05E2,4.43E2,1.14E3,6.98E2,2.83E2,1.22E2,3.52E2,9.1E1,8.78E2,2.62E2,7.6E1,6.22E2,2.37E2,4.6E1,6.7E1,5.5E1,3.46E2,6E0,3.8E1,5.3E1,9.6E1,7.82E2,2.34E2,2.8E1,2.5E1,5.1E1,5.33E2,8.9E1,1.96E2,4.1E1,8E0,3.8E1,5.6E1,1.1E1,4.1E1,1.4E1,2.71E2,7.5E1,1.3E1,2.5E1,4.6E1,7E0,9.4E1,2E0,7.77E2,5E0,2.25E2,9E0,8E0,2E1,1.8E1,7E0,3.6E1,1.5E1,4.68E2,6.5E1,1.4E1,7.5E1,1.45E2,5.1E1,3.6E1,5E0,4E0,4E0,3.3E1,5E0,1.8E1,3.8E1,8E0,3E0,3.5E1,6E0,1.2E1,2E0,2.5E2,2.1E1,4.2E1,3.3E1,3E0,1E1,2E0,2.3E1,1E1,3.6E1,3E0,4E0,7.2E1,2.2E1,5.67E2,2.1E2,1.31E2,9.4E1,5E0,4E0,2E0,1.6E1,2E0,5E0,1.1E1,2.5E1,1.1E1,4E0,1.04E2,3.64E2,1.9E1,4.6E1,1.2E1,2E0,6.8E1,7E0,1.43E2,2E0,3E0,4.8E1,7E0,2.6E1,6E0,3.2E1,3E0,5E0,3.3E1,2E0,2E0,4E0,2.3E1,2.27E2,1.5E1,6E0,1.9E1,2.3E1,2.8E1,5E0,1E1,1.3E1,6E0,4E0,2.4E1,1.2E1,6.6E1,6E0,2E1,2E0,3.55E2,2.12E2,1.96E2,1.4E1,1.29E2,2E0,4.5E1,4.9E1,3E0,2E0,2E0,1.4E1,9E0,2E0,1.2E1,1.3E1,7.8E1,2.6E1,3.37E2,2.7E1,1.5E1,4E0,4.3E1,3E0,3E0,9E0,5E0,6.3E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-4.4346023E-3,-4.9924287E-1,2.2344005E-1,-7.523059E-1,-1.8343851E-1,5.5417992E-2,5.948842E-1,-8.762053E-1,-1.7842771E-1,-3.6012796E-1,2.510525E-1,-1.6846073E-1,2.1308942E-1,4.954327E-1,1.0726805E0,-5.171616E-1,-9.753441E-1,-2.867368E-1,7.6155685E-2,-4.754012E-1,-2.4041353E-1,5.2564174E-2,5.664371E-1,-3.3135834E-1,2.7452286E-2,2.5753817E-1,-4.171702E-1,4.2617947E-1,1.04659E0,1.1405934E0,3.67778E-1,-6.154318E-1,-5.490702E-2,-1.048101E0,-5.6294364E-1,-1.8221585E-2,-5.4170924E-1,-5.681259E-1,-2.565554E-1,-2.683761E-1,6.2431926E-3,-1.5978965E-1,1.7202511E-1,6.069159E-1,1.4443612E-3,-5.2767235E-1,-2.4052514E-1,6.52581E-2,-2.514266E-2,5.6089576E-2,4.2269495E-1,-5.049324E-1,2.959479E-1,-9.540178E-2,4.8730674E-1,8.661617E-1,6.697101E-2,1.1675713E0,-6.2108174E-4,2.604694E-2,7.6493616E-3,-6.5836084E-1,-9.599027E-3,1.0822739E-1,-2.7363077E-2,-9.637717E-1,-1.4375284E0,-3.025072E-1,-6.755699E-1,-1.5328455E-1,4.640269E-1,-7.024536E-1,-2.466429E-1,-6.275368E-1,-1.4967823E-1,-3.3063972E-1,1.6110031E-1,-3.6441445E-1,-1.8654244E-1,1.9013456E-1,-2.3549592E-1,4.184815E-2,2.7876225E-1,4.4638932E-1,3.3484027E-2,-4.927532E-1,-4.1489117E-2,-1.8446854E-1,-5.118208E-1,-9.3524195E-2,3.210257E-1,-3.921947E-2,2.5558886E-1,3.909797E-1,7.441659E-2,-6.1830604E-1,-2.1782424E-1,1.6337793E-3,2.6167838E-2,1.2928998E-1,-4.7308764E-1,3.2622993E-1,5.9587157E-1,8.87934E-1,1.5489248E-2,2.7664749E-2,1.2045933E0,-3.422607E-2,-1.38134975E-2,2.1688746E-2,2.1033087E-4,-4.866716E-2,-2.8714808E-2,-7.213576E-2,-2.9399473E-2,-2.4058742E-2,-4.3695294E-3,-3.3436537E-2,-1.07376035E-2,1.9643265E-3,-1.3022219E-2,-4.7177344E-4,2.8773973E-2,-2.7872162E-2,-5.8400787E-2,-5.440206E-3,-2.474146E-2,-2.4827808E-2,-3.7059445E-2,2.475866E-3,-1.3505178E-2,-2.8926613E-2,-1.2251873E-2,-3.9099203E-3,1.7411487E-2,-1.478637E-2,-2.59938E-2,-1.14788655E-2,-2.5692945E-3,-1.5492676E-3,1.6725345E-2,-3.1319007E-3,-1.4327683E-2,7.6043764E-3,-4.9165785E-3,8.540398E-3,2.1901878E-2,2.8067967E-2,1.0342919E-2,-2.4330894E-2,-3.0451086E-3,-1.4212215E-2,-6.335672E-4,-4.2702634E-2,-1.7971318E-2,1.3271941E-2,-5.8422377E-3,4.5248043E-2,1.2457539E-2,-8.447121E-4,-4.4497415E-2,-2.314539E-3,1.4044928E-2,7.8089884E-3,2.2793485E-2,-3.14628E-2,-1.8400025E-3,-1.660995E-2,-3.7218041E-3,-8.3835813E-4,1.963284E-2,-7.1223923E-3,-2.5476398E-2,5.5143614E-3,2.0148948E-2,1.2880463E-2,3.2400552E-2,4.4973023E-2,2.3189435E-2,6.1774302E-2,4.1320153E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,-1,85,87,-1,89,91,93,95,97,99,101,103,-1,105,-1,-1,-1,107,-1,109,-1,111,113,115,117,119,121,123,125,127,129,131,133,135,137,139,141,143,145,147,-1,149,-1,151,153,155,157,159,161,163,-1,165,167,-1,-1,169,171,173,175,177,-1,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0037616E2,6.6993454E1,1.1380436E2,3.2977722E1,2.8879148E1,4.440709E1,2.6458542E1,1.3128571E1,1.6546911E1,3.5702744E0,6.793643E0,1.6604252E1,2.0781948E1,1.7569923E1,4.224098E0,3.827404E0,8.381439E0,5.467248E0,0E0,2.5937176E0,1.4139161E0,1.7525111E0,8.9342594E-1,4.9373226E0,1.667752E1,2.293301E1,3.2160912E0,1.3469162E1,2.089695E0,2.753769E0,2.7952147E-1,1.1151142E0,1.409876E0,6.9230957E0,1.1742392E0,2.6764843E0,1.7713852E0,2.2429447E0,1.352819E0,9.06559E-1,0E0,7.070623E-1,5.99702E-1,2.655058E-1,0E0,6.256027E-1,2.8863363E0,0E0,5.4508023E0,5.9424195E0,1.2989502E1,1.2975397E0,3.7483382E-1,3.8799963E0,6.3983154E0,1.5002823E-1,0E0,1.1823349E0,0E0,0E0,0E0,1.1062202E0,0E0,4.6324646E-1,0E0,3.420639E0,1.5096817E0,6.297536E-1,1.7012405E-1,7.48764E-1,5.8936167E-1,9.09482E-1,5.1603514E-1,9.077492E-1,3.609862E-1,5.817082E-1,3.73717E-1,3.8836956E-1,4.7448087E-1,1.882919E-1,2.079035E-1,3.711783E-1,3.488561E-1,4.2063236E-1,0E0,5.126629E-1,0E0,3.1497064E0,1.3772144E0,2.0597034E0,1.197828E0,4.0552034E0,1.1708708E0,7.087963E0,0E0,6.69857E-1,2.26098E-1,0E0,0E0,1.2019776E0,2.4558163E-1,3.023449E0,5.7263565E0,2.6797104E-1,0E0,0E0,1.2197952E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,61,61,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,96,96,99,99,100,100,101,101,102,102,103,103,106,106],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,-1,86,88,-1,90,92,94,96,98,100,102,104,-1,106,-1,-1,-1,108,-1,110,-1,112,114,116,118,120,122,124,126,128,130,132,134,136,138,140,142,144,146,148,-1,150,-1,152,154,156,158,160,162,164,-1,166,168,-1,-1,170,172,174,176,178,-1,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.867292E3,4.7931265E6,1.8108038E5,2.9251662E6,2.0304577E-1,1.9013363E4,1.5669752E-2,1.2060912E3,2.857E3,3.956147E-1,3.6966505E0,1E0,6.2880285E6,4.888E0,6.971004E7,7.6155685E-2,1.5345133E1,6.341735E2,1.482958E0,1E0,1.141E4,2.3E1,9.27E2,3.0795444E3,1.5833762E-1,1.146155E1,9.256843E3,2.84E2,3.4986075E6,8.749966E1,1.1034263E10,3E0,9.908038E1,3.970405E3,2.262586E6,4.3E1,2.1787034E8,6.2431926E-3,1.6254681E0,2.4513194E5,1.3137E4,1.4443612E-3,2.545455E7,1E0,6.52581E-2,8.32E3,1.5859042E6,4.4839956E3,2.6476662E6,3.09E2,6.496696E7,1.8108038E5,1.314806E8,6.697101E-2,4.6253732E2,-6.2108174E-4,2.604694E-2,7.6493616E-3,3.164136E6,-9.599027E-3,1.1766268E6,-2.7363077E-2,3.956E3,1.3999657E3,6.1463413E0,9.5732903E-1,5.418831E11,5.015233E4,1.5874315E10,5.5743E4,9.577834E7,6.822312E4,9.19E4,3.52E2,2.612E3,1.2690893E0,3.051499E5,5.48744E8,1.0824E4,2.106062E1,1.744993E6,3.3484027E-2,1.27384615E1,-4.1489117E-2,6.87E2,1E0,2.4E1,7.969174E7,1.062363E1,1.198E5,3.830216E5,7.441659E-2,1E0,8.896243E4,1.6337793E-3,2.6167838E-2,3.5799E4,8.657441E7,1.3933473E5,3.773E3,3.1066042E7,1.5489248E-2,2.7664749E-2,2.1226158E0,-3.422607E-2,-1.38134975E-2,2.1688746E-2,2.1033087E-4,-4.866716E-2,-2.8714808E-2,-7.213576E-2,-2.9399473E-2,-2.4058742E-2,-4.3695294E-3,-3.3436537E-2,-1.07376035E-2,1.9643265E-3,-1.3022219E-2,-4.7177344E-4,2.8773973E-2,-2.7872162E-2,-5.8400787E-2,-5.440206E-3,-2.474146E-2,-2.4827808E-2,-3.7059445E-2,2.475866E-3,-1.3505178E-2,-2.8926613E-2,-1.2251873E-2,-3.9099203E-3,1.7411487E-2,-1.478637E-2,-2.59938E-2,-1.14788655E-2,-2.5692945E-3,-1.5492676E-3,1.6725345E-2,-3.1319007E-3,-1.4327683E-2,7.6043764E-3,-4.9165785E-3,8.540398E-3,2.1901878E-2,2.8067967E-2,1.0342919E-2,-2.4330894E-2,-3.0451086E-3,-1.4212215E-2,-6.335672E-4,-4.2702634E-2,-1.7971318E-2,1.3271941E-2,-5.8422377E-3,4.5248043E-2,1.2457539E-2,-8.447121E-4,-4.4497415E-2,-2.314539E-3,1.4044928E-2,7.8089884E-3,2.2793485E-2,-3.14628E-2,-1.8400025E-3,-1.660995E-2,-3.7218041E-3,-8.3835813E-4,1.963284E-2,-7.1223923E-3,-2.5476398E-2,5.5143614E-3,2.0148948E-2,1.2880463E-2,3.2400552E-2,4.4973023E-2,2.3189435E-2,6.1774302E-2,4.1320153E-2],"split_indices":[20,102,12,4,51,28,28,27,52,38,52,2,27,35,6,50,54,45,0,58,52,41,84,9,0,2,4,42,35,4,0,32,58,5,8,56,52,1,3,7,0,53,28,2,0,45,6,0,2,28,52,32,0,45,28,1,0,4,0,0,0,1,0,32,0,2,52,58,27,31,28,12,10,7,33,1,0,2,42,28,5,10,56,47,0,56,0,2,16,3,7,54,7,28,0,74,33,0,0,10,45,33,2,45,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.662E3,8.39E2,1.823E3,4.65E2,3.74E2,1.256E3,5.67E2,3.82E2,8.3E1,2.66E2,1.08E2,5.19E2,7.37E2,4.71E2,9.6E1,8.4E1,2.98E2,7.9E1,4E0,1.34E2,1.32E2,6.7E1,4.1E1,2.83E2,2.36E2,6.89E2,4.8E1,4.2E2,5.1E1,8.7E1,9E0,6.9E1,1.5E1,2.52E2,4.6E1,3.9E1,4E1,9.3E1,4.1E1,1.23E2,9E0,2.4E1,4.3E1,3.8E1,3E0,8.8E1,1.95E2,8E0,2.28E2,3.11E2,3.78E2,4.3E1,5E0,4.4E1,3.76E2,3.6E1,1.5E1,8.5E1,2E0,4E0,5E0,6.2E1,7E0,1.2E1,3E0,2.1E2,4.2E1,1.5E1,3.1E1,3.1E1,8E0,2.5E1,1.5E1,8.1E1,1.2E1,3.5E1,6E0,5.5E1,6.8E1,4E0,2E1,2E1,2.3E1,1.6E1,2.2E1,8.2E1,6E0,1.63E2,3.2E1,1.91E2,3.7E1,2.11E2,1E2,3.69E2,9E0,3E1,1.3E1,3E0,2E0,2.8E1,1.6E1,1.53E2,2.23E2,3.4E1,2E0,6E0,7.9E1,5.3E1,9E0,2E0,1E1,1.82E2,2.8E1,3.8E1,4E0,7E0,8E0,2.9E1,2E0,1.2E1,1.9E1,2E0,6E0,2.2E1,3E0,1.1E1,4E0,4.9E1,3.2E1,5E0,7E0,6E0,2.9E1,3E0,3E0,4.4E1,1.1E1,4.8E1,2E1,2E0,2E0,6E0,1.4E1,1.1E1,9E0,1.6E1,7E0,9E0,7E0,7.9E1,3E0,9.8E1,6.5E1,7E0,2.5E1,1.3E1,1.78E2,2E0,3.5E1,2.07E2,4E0,1.1E1,8.9E1,1.01E2,2.68E2,2.8E1,2E0,6E0,7E0,1.9E1,9E0,3E0,1.3E1,4.8E1,1.05E2,4.5E1,1.78E2,2.9E1,5E0,6.1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[1.8860749E-4,-4.441118E-1,2.1098435E-1,-7.091485E-1,-2.0233929E-1,3.833865E-2,5.3233534E-1,-8.467057E-1,-3.8465324E-1,-3.2853097E-1,3.4043851E-1,-1.8696646E-1,2.0979714E-1,3.932558E-1,8.7977976E-1,-9.193671E-1,-3.918579E-1,-5.089913E-1,-2.1635105E-1,-3.6799085E-1,8.3161615E-2,2.4440913E-1,7.3928964E-1,-2.2847822E-1,3.9725527E-1,-3.4646764E-1,2.535628E-1,2.344202E-1,6.950682E-1,4.2269012E-1,9.999692E-1,-7.9712665E-1,-1.3801624E0,8.4622346E-2,-5.099326E-1,-5.600769E-1,-1.8407054E-1,-2.609325E-1,5.7592033E-3,-2.8763703E-1,-8.1502664E-1,6.661717E-2,3.923756E-1,1.2650997E-2,3.968274E-2,-4.495674E-1,-1.5057354E-1,5.7468224E-2,1.4469442E-1,-4.7625467E-1,2.0732722E-1,6.434489E-2,4.2175263E-1,3.0443656E-1,-2.9840046E-1,5.953819E-1,1.1456037E0,5.341327E-1,-8.133385E-3,1.0522008E0,5.261475E-1,-8.636042E-1,-4.0049723E-1,-1.8480714E-2,-1.4135507E0,-5.4962873E-3,1.27909845E-2,-5.8353454E-1,1.8945217E-2,-3.5019055E-1,-6.284403E-1,-1.1907787E-2,3.4008247E-3,-3.0906764E-1,-2.2562281E-3,-4.145218E-1,-1.15938716E-1,-7.00872E-1,-1.1561619E0,-2.2724554E-2,3.073867E-1,8.500362E-2,4.71508E-1,-3.8424906E-1,-7.1685374E-1,3.3643937E-1,-1.7629193E-1,-4.1071237E-3,2.1462257E-1,-6.267079E-1,-1.8371882E-1,-5.3983373E-3,2.1046968E-2,1.51384E-1,-1.6425556E-1,3.889705E-1,7.359985E-2,1.82212E-1,5.222423E-1,-3.0668985E-2,-3.9907258E-2,6.531286E-1,2.9862763E-2,5.765489E-2,2.4399783E-2,2.7261978E-1,6.849504E-1,1.35738505E-2,-2.1237178E-2,4.2470154E-1,1.091144E0,7.300193E-3,3.05549E-2,-2.5234437E-2,-4.3630842E-2,-8.500939E-3,-2.9292386E-2,-5.0847054E-2,-7.698113E-2,-3.441194E-2,-2.1357201E-2,-7.6197176E-3,9.131878E-3,3.8244042E-3,-1.9771801E-2,-1.8318975E-2,-3.3021484E-2,-1.6819876E-2,-3.171332E-3,-1.47554865E-2,-3.614761E-2,-3.6377327E-3,-1.9086443E-2,-3.5690863E-2,-1.4508022E-2,-2.7694875E-2,-6.1516788E-2,1.0454507E-2,-5.2836225E-3,6.4858077E-3,2.0011839E-2,8.006469E-3,-2.8502487E-3,8.727613E-3,2.6417486E-2,-2.5190484E-2,-9.394082E-3,-3.7013404E-2,-4.106744E-3,-5.6144674E-3,3.0448768E-2,-1.6382191E-2,-5.7627195E-3,7.1954224E-3,1.87048E-2,-1.5031501E-2,-3.4741882E-2,5.4933242E-3,-1.2829099E-2,-4.9090735E-3,9.933474E-3,-2.0423489E-2,-8.4694475E-4,1.5097896E-2,2.9911326E-2,6.607695E-3,2.0749427E-2,1.0217073E-2,2.7914017E-2,-6.7159845E-3,2.9156309E-2,2.5535481E-2,3.846386E-2,2.087439E-2,-1.3331809E-2,1.6034933E-2,-2.219152E-3,4.0116373E-2,1.4533908E-2,2.4894895E-2,2.9294665E-3,5.3171974E-2,-6.012525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,-1,83,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,-1,-1,119,121,123,125,-1,-1,127,-1,129,131,133,135,137,139,141,143,145,147,149,151,-1,153,155,157,-1,-1,159,161,163,-1,165,167,-1,169,171,173,-1,-1,175,177,-1,-1,179,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4856476E2,5.457962E1,9.9843376E1,1.784143E1,3.0797182E1,4.5314144E1,3.0048813E1,9.058945E0,2.4638233E0,3.0299782E1,3.074932E0,1.2386562E1,1.6301819E1,2.1457344E1,9.411118E0,1.270015E1,2.349822E0,1.0588207E0,8.4173036E-1,1.2625309E1,0E0,1.8153687E0,4.6014786E-1,8.096428E0,6.7195163E0,3.6206346E0,1.9640419E1,1.1170774E1,6.2793274E0,1.8768163E0,2.9359283E0,4.822052E0,1.1143723E0,3.5808948E-1,1.2956285E0,6.4084053E-1,1.9884986E-1,4.794681E-1,0E0,6.5966415E0,1.2925568E0,7.168405E-1,8.8419724E-1,0E0,0E0,1.8719711E0,4.4681764E0,0E0,4.5351923E-1,1.645381E0,7.9254586E-1,5.834842E0,1.1355621E1,6.9062214E0,3.068581E0,4.1932297E0,3.8768387E-1,1.0359316E0,1.2680173E0,2.6156158E0,5.7164717E-1,2.187851E0,1.3039918E0,0E0,1.567543E0,0E0,0E0,2.1824074E-1,1.8322968E-1,4.910903E-1,3.971653E-1,0E0,0E0,3.7592125E-1,0E0,6.104145E0,1.4498109E0,5.337391E-1,1.4731216E-1,5.519434E-1,1.1438781E-1,1.16838366E-1,5.947046E-1,2.5897245E0,7.468929E-1,2.5191545E0,3.0667686E0,0E0,1.7918271E-1,5.753937E-1,4.0748137E-1,0E0,0E0,3.037404E0,3.105428E0,5.3431816E0,0E0,1.8282342E0,1.6361084E0,0E0,1.4521673E0,1.6768799E0,1.7483073E0,0E0,0E0,2.7344447E-1,8.526659E-1,0E0,0E0,2.7943885E-1,2.5063477E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,64,64,67,67,68,68,69,69,70,70,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,89,89,90,90,93,93,94,94,95,95,97,97,98,98,100,100,101,101,102,102,105,105,106,106,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,-1,84,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,-1,-1,120,122,124,126,-1,-1,128,-1,130,132,134,136,138,140,142,144,146,148,150,152,-1,154,156,158,-1,-1,160,162,164,-1,166,168,-1,170,172,174,-1,-1,176,178,-1,-1,180,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,2.376302E8,1E0,1.5343539E0,2.3770695E5,1.2727361E6,5.121E3,1.5445488E8,1.3144558E4,2.5727358E0,1.5106794E3,1.2E1,1.3359244E9,3.564E3,3.1826714E7,2.1693E5,1.774E3,2.437E3,1E0,8.3161615E-2,2.0783027E5,1.8061392E0,1.24E2,1E0,2.5422776E3,9.27E2,4.3702424E7,7.466243E0,3.1984328E5,2E0,5.492982E2,1.0198864E0,1.4783E4,3.2703E4,3.3449508E9,3.771981E6,3.30399E6,5.7592033E-3,2.8593832E-1,1.3983857E10,2.1442623E1,2.4042394E5,1.2650997E-2,3.968274E-2,1.4504054E8,1E0,5.7468224E-2,5.6158E4,2.6476662E6,6.3997424E-1,4.12E0,8.426E3,2.6477592E6,1.2354571E0,5.0714142E2,9E0,1.0061821E5,1.20086E5,1.2045951E6,5.012E3,1.2327E4,1.6256282E7,-1.8480714E-2,4.3927447E11,-5.4962873E-3,1.27909845E-2,1.2068E4,4.9386E4,1.36E2,1.0376E4,-1.1907787E-2,3.4008247E-3,1.5377966E6,-2.2562281E-3,7.8099336E11,7.76264E7,2.9799202E3,1.5485038E5,1.0275E4,2.6976675E-1,1.0824E4,3.333E3,3.3151623E1,1E0,3.97E2,1.4735735E2,-4.1071237E-3,1.2204E4,4.877E4,1.8955729E1,-5.3983373E-3,2.1046968E-2,1.0046401E6,1E0,2.3314082E6,7.359985E-2,2.0601392E3,2.428818E6,-3.0668985E-2,5.094239E3,6.4683E4,1.9307388E7,5.765489E-2,2.4399783E-2,2.6666667E0,2E0,1.35738505E-2,-2.1237178E-2,1E1,1.3674345E4,7.300193E-3,3.05549E-2,-2.5234437E-2,-4.3630842E-2,-8.500939E-3,-2.9292386E-2,-5.0847054E-2,-7.698113E-2,-3.441194E-2,-2.1357201E-2,-7.6197176E-3,9.131878E-3,3.8244042E-3,-1.9771801E-2,-1.8318975E-2,-3.3021484E-2,-1.6819876E-2,-3.171332E-3,-1.47554865E-2,-3.614761E-2,-3.6377327E-3,-1.9086443E-2,-3.5690863E-2,-1.4508022E-2,-2.7694875E-2,-6.1516788E-2,1.0454507E-2,-5.2836225E-3,6.4858077E-3,2.0011839E-2,8.006469E-3,-2.8502487E-3,8.727613E-3,2.6417486E-2,-2.5190484E-2,-9.394082E-3,-3.7013404E-2,-4.106744E-3,-5.6144674E-3,3.0448768E-2,-1.6382191E-2,-5.7627195E-3,7.1954224E-3,1.87048E-2,-1.5031501E-2,-3.4741882E-2,5.4933242E-3,-1.2829099E-2,-4.9090735E-3,9.933474E-3,-2.0423489E-2,-8.4694475E-4,1.5097896E-2,2.9911326E-2,6.607695E-3,2.0749427E-2,1.0217073E-2,2.7914017E-2,-6.7159845E-3,2.9156309E-2,2.5535481E-2,3.846386E-2,2.087439E-2,-1.3331809E-2,1.6034933E-2,-2.219152E-3,4.0116373E-2,1.4533908E-2,2.4894895E-2,2.9294665E-3,5.3171974E-2,-6.012525E-4],"split_indices":[20,56,7,102,42,28,28,2,7,52,35,52,18,7,2,45,29,10,2,104,0,33,42,29,8,52,2,45,35,33,6,55,56,9,2,31,1,9,0,42,5,56,28,0,0,31,79,0,12,32,27,54,2,47,53,58,17,33,9,47,2,9,45,0,31,0,0,9,2,11,9,0,0,33,0,31,45,4,32,9,38,10,2,55,68,2,52,0,2,12,58,0,0,45,102,28,0,52,45,0,4,2,43,0,0,53,6,0,0,3,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.652E3,8.53E2,1.799E3,4.06E2,4.47E2,1.171E3,6.28E2,2.84E2,1.22E2,3.63E2,8.4E1,5.06E2,6.65E2,4.5E2,1.78E2,2.44E2,4E1,6.9E1,5.3E1,3.57E2,6E0,6.9E1,1.5E1,4.73E2,3.3E1,4.8E1,6.17E2,2.96E2,1.54E2,3.8E1,1.4E2,1.95E2,4.9E1,8E0,3.2E1,5.9E1,1E1,4.7E1,6E0,3.04E2,5.3E1,3.2E1,3.7E1,3E0,1.2E1,1.22E2,3.51E2,7E0,2.6E1,3.9E1,9E0,2.91E2,3.26E2,2.62E2,3.4E1,1.28E2,2.6E1,3E1,8E0,1.25E2,1.5E1,1.66E2,2.9E1,2E0,4.7E1,4E0,4E0,2.8E1,4E0,1.6E1,4.3E1,8E0,2E0,3.8E1,9E0,1.74E2,1.3E2,4.2E1,1.1E1,2.4E1,8E0,8E0,2.9E1,1E2,2.2E1,1.7E1,3.34E2,6E0,2E1,2.5E1,1.4E1,4E0,5E0,2.11E2,8E1,3.18E2,8E0,1.69E2,9.3E1,1.4E1,2E1,1.16E2,1.2E1,2.3E1,3E0,1.2E1,1.8E1,5E0,3E0,8E0,1.17E2,4E0,1.1E1,2.2E1,1.44E2,1.5E1,1.4E1,1.9E1,2.8E1,1.2E1,1.6E1,2E0,2E0,2E0,1.4E1,1E1,3.3E1,3.2E1,6E0,1.34E2,4E1,1.15E2,1.5E1,3.7E1,5E0,3E0,8E0,6E0,1.8E1,4E0,4E0,5E0,3E0,7E0,2.2E1,5.6E1,4.4E1,2E1,2E0,7E0,1E1,8.3E1,2.51E2,1.6E1,4E0,7E0,1.8E1,3E0,1.1E1,3.8E1,1.73E2,2.8E1,5.2E1,2.44E2,7.4E1,1.45E2,2.4E1,1.6E1,7.7E1,1.8E1,2E0,6.7E1,4.9E1,5E0,7E0,1E1,2E0,1.2E1,6E0,6E0,2E0,1.15E2,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[-1.0558123E-3,-4.143971E-1,1.9785956E-1,-6.403888E-1,-1.398694E-1,2.093154E-2,4.9939817E-1,-7.546526E-1,-1.3248697E-1,-3.2637885E-1,2.3420908E-1,-1.4117453E-1,1.8300101E-1,3.6262414E-1,7.9521173E-1,-2.7618825E-1,-8.307267E-1,8.0190375E-2,-2.1499006E-1,-3.6164734E-1,-3.9229605E-2,4.894443E-2,5.370642E-1,-4.9477413E-1,-7.929514E-2,2.2702083E-1,-4.08138E-1,4.4598362E-1,-1.8994002E-1,1.1016883E0,6.6323334E-1,-4.501161E-1,2.176271E-1,-7.8177416E-1,-1.3843228E0,2.1360764E-2,-4.4108316E-1,-4.6004504E-1,-2.4167927E-1,-1.9486237E-1,9.124618E-2,-1.9222733E-1,1.3373208E-1,3.4146073E-1,6.560845E-1,-5.3371245E-1,2.3498158E-1,-2.9008928E-1,-4.3340805E-3,6.1461017E-2,3.5540074E-1,-5.317152E-1,1.5263522E-1,3.1257325E-1,6.970049E-1,-1.1616825E-2,-4.140649E-1,1.166689E0,4.2429814E-1,4.58092E-1,8.3232784E-1,-5.623407E-1,-1.651446E-1,3.387619E-2,3.624483E-2,-8.504915E-1,-4.3898076E-1,-7.3438324E-2,-4.1045845E-2,-9.511301E-2,4.719409E-1,-3.549145E-1,-4.991228E-2,-5.6477106E-1,-2.7373558E-1,-2.6465765E-1,-1.4991928E-2,-7.134505E-4,-1.404091E-2,3.4773387E-2,1.6435781E-2,9.663403E-3,-2.6505923E-1,2.3398513E-1,-2.4001842E-2,4.2059473E-3,2.0931056E-2,8.351251E-1,4.472286E-1,-4.4688103E-1,-8.1279576E-1,2.1603137E-2,-2.847481E-3,-3.7856635E-1,-1.1257467E-1,1.1313146E0,-5.68371E-2,-2.5072476E-2,3.072867E-1,2.87979E-1,6.9488806E-1,-3.2454398E-1,-3.3887777E-2,-5.464966E-3,2.1450773E-2,1.6604684E-1,4.1044426E-1,3.4868482E-1,8.0315614E-1,-1.2776924E-2,1.1645597E-1,-2.6567418E-2,-2.1988136E-1,1.3049594E0,7.7271825E-1,2.8797045E-2,-5.82057E-4,4.8199612E-1,-2.7213732E-3,9.2071515E-1,1.1270674E-1,4.2334455E-3,-2.9396053E-2,3.4408124E-5,-1.4722298E-2,-9.859364E-3,5.0407504E-3,-5.138792E-2,-3.758987E-2,-2.3569454E-2,-5.672048E-3,1.1335611E-2,-7.5813043E-3,2.8859437E-2,4.6582654E-4,-2.4400096E-2,-5.545852E-3,-2.243816E-2,-3.3757295E-2,-2.9116474E-2,-1.0570923E-2,-1.4962618E-2,-6.98966E-3,3.8287563E-3,-5.1450375E-3,-2.4265123E-3,5.54898E-3,4.7967347E-5,-1.5085914E-2,1.6452364E-3,1.6111292E-2,3.6997618E-3,-5.910264E-3,2.0549277E-2,4.5197498E-2,4.894936E-3,2.4854504E-2,-2.4199603E-2,-1.0647177E-2,-2.8591735E-2,-5.4826915E-2,-1.4884704E-2,-3.0234348E-2,-2.488721E-4,-1.28053315E-2,1.906669E-2,5.783344E-2,-4.5019253E-3,1.7654154E-2,-6.1487914E-3,6.4180307E-3,1.683762E-2,6.3912785E-3,1.5927361E-2,-1.9491625E-3,2.493711E-2,4.2382292E-2,-1.7435376E-2,-8.97773E-4,1.1015172E-2,-7.805209E-3,1.8117836E-2,4.0461455E-2,2.0523792E-2,-4.5723403E-3,4.2356595E-2,1.0133508E-2,-1.7275623E-3,1.7943593E-2,-1.2590871E-2,5.3550483E-4,4.098259E-2,6.8299994E-2,1.6829154E-2,4.0127397E-2,2.503365E-2,1.087695E-2,4.5116853E-2,5.518465E-3,2.1798437E-2,-4.745412E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,-1,125,127,129,-1,-1,131,133,135,-1,137,139,141,143,-1,-1,145,-1,-1,147,149,151,-1,-1,153,155,157,159,-1,-1,161,163,165,167,169,171,173,175,177,-1,-1,-1,179,181,183,185,-1,187,-1,189,191,193,-1,-1,195,-1,197,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2067807E2,5.4010193E1,9.666088E1,2.7640625E1,2.763526E1,3.005524E1,2.681958E1,1.396608E1,1.36279335E1,2.6594276E0,7.380015E0,1.2476428E1,1.4977144E1,2.127298E1,7.8039246E0,4.821811E0,8.023346E0,0E0,4.6050286E0,2.6628933E0,6.2696964E-1,1.722923E0,9.4001293E-1,2.5407276E0,7.7098026E0,1.1306835E1,2.8531098E0,1.3177673E1,2.4331565E0,2.2348862E0,4.821953E0,1.2183456E0,1.3068138E0,6.9525146E0,7.301674E-1,2.3015606E0,1.9092503E0,2.3428688E0,5.585947E-1,2.4336326E-1,2.29946E-1,6.848045E-1,9.8604107E-1,4.7081447E-1,7.483177E-1,1.5996666E0,3.4498167E-1,1.9677477E0,2.1588919E1,4.9853888E0,6.6665344E0,9.7287464E-1,6.8123215E-1,3.7034492E0,4.8224487E0,1.1749153E0,6.156416E-1,2.0422058E0,5.481075E-1,8.889942E-1,5.0482635E0,1.0257034E0,3.0698845E-1,0E0,2.2217424E-1,2.9229736E0,8.6207867E-1,0E0,0E0,7.7134246E-1,5.377228E-1,1.4195013E0,0E0,7.619953E-1,7.16192E-1,5.101919E-1,1.05129294E-1,0E0,0E0,1.0980317E-1,0E0,0E0,2.5281549E-1,7.6550436E-1,2.6118183E-1,0E0,0E0,2.5672817E-1,3.4307337E-1,7.2739315E-1,6.0613155E-1,0E0,0E0,1.2595434E0,7.3004407E-1,4.1431427E-1,5.506197E0,2.877227E0,4.2280912E-1,3.7075863E0,1.1615791E0,1.9863915E-1,0E0,0E0,0E0,2.289271E0,1.8715801E0,1.2470903E0,4.862068E0,0E0,9.5879376E-1,0E0,1.3218832E-1,9.574661E-1,1.7239189E-1,0E0,0E0,5.768137E-1,0E0,1.130661E0,7.7042854E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,64,64,65,65,66,66,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,82,82,83,83,84,84,87,87,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,105,105,106,106,107,107,108,108,110,110,112,112,113,113,114,114,117,117,119,119,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,-1,126,128,130,-1,-1,132,134,136,-1,138,140,142,144,-1,-1,146,-1,-1,148,150,152,-1,-1,154,156,158,160,-1,-1,162,164,166,168,170,172,174,176,178,-1,-1,-1,180,182,184,186,-1,188,-1,190,192,194,-1,-1,196,-1,198,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,2.9799202E3,4.197889E6,2.2178302E5,1.3664E4,8.2608955E-3,1.4E1,2.262586E6,1.1925198E3,8.2E1,4.898012E-1,2.2075728E7,4.52737E5,6.325655E6,1.1855755E8,8.0190375E-2,7.218466E7,1.8329959E-1,2.34E0,6.7699623E-1,1.948181E0,1.6006084E3,6.76E2,8.46E2,2.5422776E3,6.167018E6,5.9767612E7,4.5481584E7,6.4683E4,2.692487E6,2.66E2,4.7777777E0,1.3528846E1,9.908038E1,9.22E4,1.4757962E1,9.705292E6,9.912942E6,9.075E3,6E0,1.4187837E-1,3.368326E11,2.4489644E0,1.4504054E8,5.202E3,1.325204E6,8E0,2.3314082E6,2.9251662E6,4.352552E9,8.32E3,1E0,3.171E3,4.492E3,9.53882E-2,7.8E1,5.5743E4,2.3817926E7,7.26E2,1.21E2,4.76E3,3.387619E-2,1.4E1,8.857896E4,3.65625E1,-7.3438324E-2,-4.1045845E-2,1E0,1.7390422E7,3.4409692E3,-4.991228E-2,1E0,1.06E3,1.4930757E0,1.43709E0,-7.134505E-4,-1.404091E-2,3.624992E5,1.6435781E-2,9.663403E-3,9.5411836E4,2.695E3,5.884028E0,4.2059473E-3,2.0931056E-2,1E0,2.862988E5,2.7523365E0,2.5544708E0,2.1603137E-2,-2.847481E-3,1.5E1,5.3296334E-1,1.279012E7,1.4777102E9,2.6883545E2,1.6E1,1.0238709E1,7.130317E7,1.6503105E3,-3.3887777E-2,-5.464966E-3,2.1450773E-2,5.743459E-3,1.7474695E0,2.067698E0,5.3411217E0,-1.2776924E-2,2.1872402E5,-2.6567418E-2,3.1E1,3.0720797E-1,2.73E2,2.8797045E-2,-5.82057E-4,3.5296965E-1,-2.7213732E-3,1.43761E8,4.0728608E3,4.2334455E-3,-2.9396053E-2,3.4408124E-5,-1.4722298E-2,-9.859364E-3,5.0407504E-3,-5.138792E-2,-3.758987E-2,-2.3569454E-2,-5.672048E-3,1.1335611E-2,-7.5813043E-3,2.8859437E-2,4.6582654E-4,-2.4400096E-2,-5.545852E-3,-2.243816E-2,-3.3757295E-2,-2.9116474E-2,-1.0570923E-2,-1.4962618E-2,-6.98966E-3,3.8287563E-3,-5.1450375E-3,-2.4265123E-3,5.54898E-3,4.7967347E-5,-1.5085914E-2,1.6452364E-3,1.6111292E-2,3.6997618E-3,-5.910264E-3,2.0549277E-2,4.5197498E-2,4.894936E-3,2.4854504E-2,-2.4199603E-2,-1.0647177E-2,-2.8591735E-2,-5.4826915E-2,-1.4884704E-2,-3.0234348E-2,-2.488721E-4,-1.28053315E-2,1.906669E-2,5.783344E-2,-4.5019253E-3,1.7654154E-2,-6.1487914E-3,6.4180307E-3,1.683762E-2,6.3912785E-3,1.5927361E-2,-1.9491625E-3,2.493711E-2,4.2382292E-2,-1.7435376E-2,-8.97773E-4,1.1015172E-2,-7.805209E-3,1.8117836E-2,4.0461455E-2,2.0523792E-2,-4.5723403E-3,4.2356595E-2,1.0133508E-2,-1.7275623E-3,1.7943593E-2,-1.2590871E-2,5.3550483E-4,4.098259E-2,6.8299994E-2,1.6829154E-2,4.0127397E-2,2.503365E-2,1.087695E-2,4.5116853E-2,5.518465E-3,2.1798437E-2,-4.745412E-3],"split_indices":[20,102,33,4,51,28,2,27,0,1,52,29,27,45,29,51,45,0,45,41,54,42,42,55,2,2,52,47,45,47,2,1,0,54,56,56,10,58,9,45,2,3,27,31,42,31,2,9,18,28,28,5,2,100,2,2,42,8,10,43,8,2,0,0,3,32,56,0,0,75,32,52,0,100,2,42,53,0,0,47,0,0,33,2,53,0,0,8,28,53,53,0,0,8,38,7,7,4,3,53,7,52,0,0,0,57,38,38,54,0,33,0,3,38,0,0,0,27,0,43,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.682E3,8.71E2,1.811E3,4.77E2,3.94E2,1.142E3,6.69E2,3.89E2,8.8E1,2.63E2,1.31E2,5.71E2,5.71E2,4.59E2,2.1E2,5.4E1,3.35E2,3E0,8.5E1,2.34E2,2.9E1,8.2E1,4.9E1,8.4E1,4.87E2,5.32E2,3.9E1,3.99E2,6E1,6.1E1,1.49E2,4E1,1.4E1,3.1E2,2.5E1,4.2E1,4.3E1,1.27E2,1.07E2,1.3E1,1.6E1,2.1E1,6.1E1,2E1,2.9E1,8E1,4E0,1.27E2,3.6E2,2.33E2,2.99E2,3.2E1,7E0,2.62E2,1.37E2,3.4E1,2.6E1,5.5E1,6E0,6.9E1,8E1,2.8E1,1.2E1,3E0,1.1E1,2.57E2,5.3E1,1.8E1,7E0,3.4E1,8E0,3.9E1,4E0,8E1,4.7E1,9.7E1,1E1,5E0,8E0,1.4E1,2E0,3E0,1.8E1,3.7E1,2.4E1,6E0,1.4E1,1.4E1,1.5E1,6.3E1,1.7E1,2E0,2E0,8.4E1,4.3E1,1.5E1,3.45E2,1.73E2,6E1,2.51E2,4.8E1,1.6E1,1.6E1,4E0,3E0,1.06E2,1.56E2,3.3E1,1.04E2,1.1E1,2.3E1,1.4E1,1.2E1,3.9E1,1.6E1,4E0,2E0,6.6E1,3E0,7.1E1,9E0,2E0,2.6E1,6E0,6E0,2E0,9E0,5.5E1,2.02E2,4.5E1,8E0,5E0,2.9E1,6E0,2E0,2.3E1,1.6E1,5E1,3E1,5E0,4.2E1,6.8E1,2.9E1,5E0,5E0,7E0,7E0,3E0,1.5E1,1.3E1,2.4E1,1.2E1,1.2E1,4E0,1E1,3E0,1.2E1,4.9E1,1.4E1,1.2E1,5E0,6.8E1,1.6E1,2.6E1,1.7E1,2E0,1.3E1,3.18E2,2.7E1,1.05E2,6.8E1,4.7E1,1.3E1,2.21E2,3E1,2.7E1,2.1E1,1.4E1,2E0,8.9E1,1.7E1,1.47E2,9E0,2.8E1,5E0,9.1E1,1.3E1,1.5E1,8E0,1E1,2E0,1E1,2.9E1,3E0,1.3E1,5.6E1,1E1,6.9E1,2E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[3.5006138E-3,-3.969561E-1,1.9679224E-1,-6.2508416E-1,-1.5343739E-1,4.1693296E-2,5.272067E-1,-7.534093E-1,-3.267084E-1,-1.9846436E-1,1.1858184E0,-1.0494695E-1,2.1011755E-1,3.9100263E-1,8.236596E-1,-8.107093E-1,-4.1659498E-1,-4.236018E-1,-1.3620822E-1,-3.8936618E-1,1.02756895E-1,8.169178E-2,2.790128E-2,-3.9404657E-1,-4.7355063E-2,2.4980399E-1,-3.0136E-1,1.8116954E-1,6.020342E-1,8.882204E-1,2.5630575E-1,-7.271806E-1,-1.1708956E0,-1.2824127E-1,-5.853325E-1,-4.7424787E-1,-2.0671944E-1,-2.752362E-1,-4.3616913E-2,-3.094976E-1,-7.947965E-1,-7.3902115E-2,2.8705993E-1,-2.509585E-2,-3.1526917E-1,9.063E-1,-8.476967E-2,6.858692E-2,4.2808235E-1,-4.4440764E-1,9.329305E-2,2.5746968E-1,-3.0217668E-1,-7.235705E-1,6.469938E-1,3.7384248E-1,9.4949603E-1,1.23258114E-1,2.3264548E-2,-8.129806E-1,-4.990952E-1,-8.2292154E-2,-1.0545468E0,1.9247614E-1,-2.9107055E-1,-1.1946179E-2,-7.174048E-1,-5.1234645E-1,-2.7709347E-1,1.2152324E-3,-3.442226E-1,-1.489933E-2,-3.013421E-3,-1.2386474E-1,1.062534E-1,-3.4466365E-1,2.580313E-1,-6.9528145E-1,-1.1183861E0,-3.0659411E-2,-3.2719985E-1,1.00163855E-1,4.189601E-1,-3.3341008E-1,-1.01779506E-1,1.0792022E0,-2.3506928E-2,-4.4020485E-2,-4.9643812E-1,1.4205422E-1,-1.4020038E-1,4.0156215E-1,7.3178805E-2,-2.2734977E-1,-2.8185548E-2,-7.2610513E-3,1.1239938E-2,2.1631588E-1,5.338431E-1,-3.4070706E-1,6.344477E-3,-4.2585578E-2,-9.879125E-3,5.5545664E-1,9.7280335E-1,5.2993053E-1,-1.6472928E-2,9.8488903E-1,3.0523643E-1,1.4037281E-2,-8.846869E-2,-4.1305944E-2,-2.4568625E-2,-1.3011487E-2,-3.1203024E-2,-6.1274856E-2,-4.379243E-2,-3.6355834E-3,1.9766707E-2,-1.6897934E-2,1.5544799E-3,-2.4753502E-2,-4.441104E-2,-1.2627219E-2,-2.63906E-2,-1.5624962E-2,-3.4768959E-3,-2.2406809E-2,-9.054172E-3,5.626869E-3,-9.173333E-3,8.435628E-3,-6.6489354E-3,-1.1311458E-2,-2.6499942E-2,-5.2402285E-3,2.4010714E-2,-3.616597E-2,-9.4264895E-3,-6.018833E-2,-2.2450307E-2,-4.5309556E-3,5.6982473E-3,-3.5137131E-3,-2.1308593E-2,-3.2591121E-3,9.339231E-3,-1.9096164E-3,2.340804E-2,-1.6591502E-2,-5.1571047E-3,-1.2474722E-2,6.6084776E-4,2.205711E-2,5.811348E-2,-4.5268578E-3,7.5387587E-3,-3.85709E-2,-1.4416413E-2,4.192362E-3,1.4656126E-2,-1.7306225E-2,-6.2065425E-5,9.208459E-3,2.2508822E-2,-2.0943552E-2,6.0102227E-3,5.4899757E-3,1.3670353E-2,3.6588386E-2,1.7983042E-2,-2.0728165E-2,-1.0310648E-2,2.9113524E-2,9.42248E-3,3.8484946E-2,7.520287E-2,2.9169668E-2,6.7132157E-3,3.7481792E-2,5.091276E-2,2.1644149E-2,4.2413422E-4,2.7449136E-3,-1.0159337E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,99,101,103,105,107,109,-1,111,113,-1,115,117,119,-1,121,123,125,-1,127,-1,-1,129,131,133,135,137,139,141,143,145,147,149,151,153,-1,155,157,159,161,163,-1,165,-1,-1,-1,167,169,171,-1,-1,-1,173,175,177,-1,179,181,-1,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.087618E2,4.8680023E1,9.319619E1,1.7098831E1,2.592427E1,3.0622591E1,2.3137619E1,5.7608795E0,2.4895363E0,2.3789032E1,3.3974476E0,1.1020151E1,1.1787647E1,1.76035E1,6.4388046E0,7.2416534E0,2.2604294E0,9.015293E-1,6.0588837E-1,7.9464417E0,5.26318E0,0E0,0E0,9.611225E-1,1.9843895E1,1.7287235E1,2.4125354E0,7.527437E0,1.232827E1,4.782608E0,5.507587E-1,3.9191513E0,1.3089523E0,1.0482457E0,1.1518774E0,4.063568E-1,6.1509585E-1,1.2514591E-1,3.7396407E-1,4.3399353E0,5.4268837E-1,8.990904E-1,1.9055133E0,0E0,2.4104118E-1,5.6026897E0,8.936079E0,4.1154394E0,7.2626076E0,8.3604E-1,4.5117146E-1,1.874939E0,5.1406455E-1,4.1640782E-1,5.215622E0,2.311079E0,3.0057068E0,5.294074E-1,0E0,1.9196701E0,2.0070906E0,0E0,1.8524551E-1,4.451652E-1,2.7126873E-1,0E0,4.0475273E-1,4.3234825E-1,1.1708343E-1,0E0,1.4177966E-1,0E0,0E0,3.4922487E-1,2.1287902E-1,4.419035E0,1.2172432E0,8.664665E-1,1.3009548E-1,7.002927E-1,3.2599795E-1,5.557446E-1,1.4931831E0,1.5396309E-1,1.3199444E-1,1.019228E0,0E0,4.969372E0,2.6554832E0,1.7431588E0,2.1500363E0,3.7054596E0,0E0,1.0921187E0,0E0,0E0,0E0,1.0498285E0,5.1486444E-1,2.0717835E-1,0E0,0E0,0E0,2.7595863E0,2.8831291E0,4.0826225E-1,0E0,1.3350525E0,3.8734543E-1,0E0,1.3836148E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,62,62,63,63,64,64,66,66,67,67,68,68,70,70,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,90,90,91,91,93,93,97,97,98,98,99,99,103,103,104,104,105,105,107,107,108,108,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,100,102,104,106,108,110,-1,112,114,-1,116,118,120,-1,122,124,126,-1,128,-1,-1,130,132,134,136,138,140,142,144,146,148,150,152,154,-1,156,158,160,162,164,-1,166,-1,-1,-1,168,170,172,-1,-1,-1,174,176,178,-1,180,182,-1,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.126489E7,1E0,1.3144558E4,3.2627738E5,1.2974394E6,5.121E3,1.19507775E-1,6.1392635E-1,1.5151515E-3,2.73E2,7.325843E0,1.2679E4,1.708775E7,4.0844156E7,5.9767612E7,3.011152E6,2.028994E8,1E0,1.5343539E0,8.169178E-2,2.790128E-2,4.1E1,8E0,9.52E2,1.945345E6,3.9712732E7,1.70752E4,3.333E3,1.7708E4,4.4866666E2,3E0,1.6956521E0,4.88954E5,5.638132E0,1.2608696E0,1.638058E2,2.437E3,1.1037509E2,1.0104842E12,1.9756216E-1,1.3097832E3,-2.509585E-2,1.8991614E5,1.9E1,1E0,4.12E0,5.847845E3,7.5179994E-1,4.366359E0,1.9725596E0,3.202814E5,3E1,8.791777E3,1.0963991E7,4.4263797E0,1.7321888E1,2.3264548E-2,3.340245E0,1.6256282E7,-8.2292154E-2,1.141E4,2.632E5,1.0409E4,-1.1946179E-2,1.6E1,2.200144E9,4.785384E5,1.2152324E-3,1.322E3,-1.489933E-2,-3.013421E-3,1.822542E0,4.768812E1,3.791269E7,1.003252E5,2.1169072E3,3.285E3,5.432133E0,4.651395E1,2.4513194E5,1E0,9.213145E0,3.8E1,3.192E3,-2.3506928E-2,2.1975278E8,3.3820656E7,3.321532E6,1E0,5.1586456E7,7.3178805E-2,2.4601164E-5,-2.8185548E-2,-7.2610513E-3,1.1239938E-2,5.794E3,4.1E2,1.6347875E3,6.344477E-3,-4.2585578E-2,-9.879125E-3,3.791269E7,4.516183E1,1.537007E7,-1.6472928E-2,1.067536E3,3.84E2,1.4037281E-2,4.06E2,-4.1305944E-2,-2.4568625E-2,-1.3011487E-2,-3.1203024E-2,-6.1274856E-2,-4.379243E-2,-3.6355834E-3,1.9766707E-2,-1.6897934E-2,1.5544799E-3,-2.4753502E-2,-4.441104E-2,-1.2627219E-2,-2.63906E-2,-1.5624962E-2,-3.4768959E-3,-2.2406809E-2,-9.054172E-3,5.626869E-3,-9.173333E-3,8.435628E-3,-6.6489354E-3,-1.1311458E-2,-2.6499942E-2,-5.2402285E-3,2.4010714E-2,-3.616597E-2,-9.4264895E-3,-6.018833E-2,-2.2450307E-2,-4.5309556E-3,5.6982473E-3,-3.5137131E-3,-2.1308593E-2,-3.2591121E-3,9.339231E-3,-1.9096164E-3,2.340804E-2,-1.6591502E-2,-5.1571047E-3,-1.2474722E-2,6.6084776E-4,2.205711E-2,5.811348E-2,-4.5268578E-3,7.5387587E-3,-3.85709E-2,-1.4416413E-2,4.192362E-3,1.4656126E-2,-1.7306225E-2,-6.2065425E-5,9.208459E-3,2.2508822E-2,-2.0943552E-2,6.0102227E-3,5.4899757E-3,1.3670353E-2,3.6588386E-2,1.7983042E-2,-2.0728165E-2,-1.0310648E-2,2.9113524E-2,9.42248E-3,3.8484946E-2,7.520287E-2,2.9169668E-2,6.7132157E-3,3.7481792E-2,5.091276E-2,2.1644149E-2,4.2413422E-4,2.7449136E-3,-1.0159337E-2],"split_indices":[20,56,12,102,52,28,28,2,38,42,57,2,53,2,45,45,45,9,7,104,42,0,0,2,18,2,12,45,33,2,10,4,3,53,32,56,53,52,2,56,31,38,4,0,28,3,6,54,4,27,56,38,28,3,52,47,42,56,0,54,45,0,9,33,2,0,3,31,28,0,2,0,0,58,58,45,28,52,0,53,58,28,16,53,0,1,0,7,7,28,102,7,0,39,0,0,0,2,0,4,0,0,0,45,54,9,0,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.695E3,8.77E2,1.818E3,4.52E2,4.25E2,1.238E3,5.8E2,3.15E2,1.37E2,4.12E2,1.3E1,6.62E2,5.76E2,3.99E2,1.81E2,2.68E2,4.7E1,9E1,4.7E1,2.52E2,1.6E2,6E0,7E0,1.09E2,5.53E2,5.35E2,4.1E1,2.01E2,1.98E2,1.62E2,1.9E1,2.2E2,4.8E1,1.8E1,2.9E1,7.2E1,1.8E1,1.8E1,2.9E1,2.12E2,4E1,8.2E1,7.8E1,3.9E1,7E1,2E1,5.33E2,2.66E2,2.69E2,3E1,1.1E1,1.74E2,2.7E1,6E0,1.92E2,1.8E1,1.44E2,1.3E1,6E0,1.58E2,6.2E1,6E0,4.2E1,6E0,1.2E1,9E0,2E1,5.9E1,1.3E1,7E0,1.1E1,1.5E1,3E0,1.9E1,1E1,2E2,1.2E1,3.3E1,7E0,7.1E1,1.1E1,3.3E1,4.5E1,6.4E1,6E0,1.8E1,2E0,4.86E2,4.7E1,1.97E2,6.9E1,2.64E2,5E0,1.3E1,1.7E1,4E0,7E0,1.53E2,2.1E1,2.5E1,2E0,4E0,2E0,1.52E2,4E1,1.5E1,3E0,1.36E2,8E0,7E0,6E0,1.34E2,2.4E1,2.6E1,3.6E1,1.3E1,2.9E1,3E0,3E0,1E1,2E0,1.2E1,8E0,9E0,5E1,1E1,3E0,5E0,6E0,4E0,1.5E1,8E0,2E0,1.33E2,6.7E1,5E0,7E0,2.9E1,4E0,5E0,2E0,5E1,2.1E1,4E0,7E0,1.2E1,2.1E1,6E0,3.9E1,6E1,4E0,2E0,4E0,4E0,1.4E1,3.89E2,9.7E1,1.7E1,3E1,1.49E2,4.8E1,2.6E1,4.3E1,6.6E1,1.98E2,8E0,5E0,6.3E1,9E1,7E0,1.4E1,1.3E1,1.2E1,1.32E2,2E1,3.3E1,7E0,1.2E1,3E0,4.1E1,9.5E1,5E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[-3.3234335E-3,-4.003951E-1,1.827159E-1,-6.184015E-1,-1.8858416E-1,1.0493759E-2,4.5210552E-1,-7.303263E-1,-3.3480394E-1,-2.915597E-1,2.9219142E-1,-1.3600777E-1,1.6380881E-1,2.9205087E-1,6.8141043E-1,-6.973475E-1,-1.394606E0,-4.8029998E-1,-1.8372782E-1,-3.1853017E-1,7.898548E-2,4.374563E-2,2.3557529E-1,-2.645788E-1,6.5557584E-3,2.0476851E-1,-4.2835072E-1,-1.3189842E-1,3.9556977E-1,7.987184E-1,3.7321103E-1,-7.577371E-1,-4.065924E-1,-7.445402E-2,-1.6418459E-2,-5.021286E-1,4.87623E-3,-2.1747012E-1,1.1059439E-2,-2.4748187E-1,-6.8584573E-1,1.3946673E-1,5.108134E-1,-4.6650845E-1,-1.8856515E-1,5.481633E-2,-3.301609E-2,-2.0880313E-1,2.4694636E-1,-5.3670335E-1,1.8480417E-1,1.5242167E-2,-4.1621393E-1,2.1529351E-1,5.403843E-1,6.5922534E-1,1.007569E0,2.0209235E-1,6.426898E-1,-6.7556447E-1,-9.0747494E-1,-7.862383E-2,-5.9886676E-1,-5.541575E-1,-1.9697428E-1,-1.1856262E-2,-2.837202E-4,-3.3748367E-1,-7.981044E-2,-9.450117E-1,-5.769568E-1,1.9328591E-1,-1.5110531E-1,5.682029E-1,1.6505509E-3,-3.7037188E-1,-6.415035E-1,-1.2850253E-1,-5.177456E-1,-7.8479916E-2,3.9566454E-1,-8.00756E-2,-3.1888035E-1,2.3583508E-1,6.9133006E-2,-3.1529808E-1,-3.1046988E-2,-3.7625376E-3,2.271746E-2,-2.9632473E-1,1.1873489E-1,-1.0991432E-2,-4.6751893E-1,-4.183315E-1,2.4459964E-1,2.646944E-1,6.5893066E-1,4.5572096E-1,8.463261E-1,1.0476203E0,3.323233E-1,2.869257E-1,-9.752654E-2,4.3432415E-1,9.820525E-1,-2.0829827E-2,-3.506599E-2,-4.4434495E-2,-1.3825222E-2,7.96538E-3,-1.06946E-2,-3.546534E-2,-1.7422864E-2,-6.277315E-3,-2.7267532E-2,-1.3810012E-2,-2.2943935E-3,-1.7560048E-2,1.4054912E-2,-2.1815042E-3,-1.7494887E-2,-4.9511332E-2,-2.1180192E-2,-9.699638E-3,-2.9263008E-2,4.9635353E-3,1.7797928E-2,1.8917877E-3,-1.2115964E-2,7.3791808E-3,3.1220611E-2,-2.1107988E-2,-6.5990756E-3,-3.271612E-2,-3.8631996E-3,-1.6484188E-2,-1.6219511E-3,-4.041705E-2,-1.8179415E-2,-3.0936569E-3,-4.1376576E-2,2.2838445E-2,-1.7189847E-2,-1.1571315E-2,-4.236853E-4,-8.802553E-3,-1.7797694E-2,8.224837E-3,2.0140592E-2,-1.7521737E-2,-1.9933828E-3,-1.7260598E-2,2.8170303E-3,-9.426454E-3,8.978151E-3,-2.4869086E-2,-1.2046495E-2,-9.204677E-3,-3.1405147E-2,1.3242372E-2,-1.268284E-2,2.3406796E-2,9.294755E-3,3.653343E-2,1.5484214E-2,1.7550675E-2,2.7231995E-2,3.490982E-2,5.1433202E-2,5.110159E-2,2.064687E-2,-5.770159E-4,2.4305474E-2,1.630929E-2,-6.964002E-4,-1.0069781E-2,-7.1228476E-4,2.44764E-2,-9.437193E-4,1.6457073E-2,5.199667E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,43,45,47,49,51,53,55,57,59,61,-1,-1,63,-1,65,-1,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,143,145,147,-1,149,-1,-1,-1,151,153,-1,155,157,159,161,163,165,167,169,171,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.947231E2,3.871753E1,8.327402E1,1.28965E1,2.127467E1,2.4639414E1,2.5491913E1,5.511978E0,2.5274916E0,1.9003387E1,2.457982E0,1.0284604E1,1.30882225E1,1.8221294E1,1.0078186E1,4.678543E0,1.6555805E0,8.1354046E-1,8.897147E-1,8.938171E0,0E0,0E0,1.8069031E0,4.440111E0,1.2062186E1,8.808229E0,2.424477E0,3.4367604E0,8.57716E0,5.3663635E0,3.60526E0,2.266159E0,3.168971E0,0E0,0E0,8.117027E-1,0E0,3.5511398E-1,0E0,4.4097385E0,1.0351067E0,8.7367725E-1,4.7248983E-1,1.1082783E0,4.204099E0,0E0,5.0788226E0,6.4045405E-1,5.7225113E0,5.367756E-1,4.94833E-1,1.8086538E0,1.6090441E-1,2.8876882E0,5.8390846E0,4.4499435E0,1.8424606E0,1.3242841E0,1.7655277E0,1.7942123E0,7.4002075E-1,7.4597263E-1,7.9631805E-1,2.4078274E-1,1.22418106E-1,0E0,0E0,3.6250343E0,1.0005627E0,2.1267414E-1,4.1221142E-1,6.96926E-1,1.8477874E-1,4.5422506E-1,0E0,8.232169E-1,5.754328E-1,3.74433E0,1.1666842E0,2.506602E0,1.7034743E0,2.6251405E-1,1.09829664E-1,5.290922E0,0E0,1.5085602E-1,0E0,0E0,0E0,3.3987474E-1,9.479378E-1,0E0,1.0418606E-1,2.498008E-1,2.4143496E0,8.069513E-1,4.2175865E0,4.2644405E-1,9.539528E-1,4.5030975E-1,3.6811483E-1,6.5163326E-1,1.0869281E-1,7.4201035E-1,3.5504723E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,44,46,48,50,52,54,56,58,60,62,-1,-1,64,-1,66,-1,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,144,146,148,-1,150,-1,-1,-1,152,154,-1,156,158,160,162,164,166,168,170,172,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.629299E4,1E0,1.6990049E0,2.3770695E5,8.631E3,1.4299594E8,2.3859661E-2,2.14099E5,1E0,2.657E3,4.898012E-1,1.545974E-1,2.5003334E7,3.773E3,7.431E3,6.54925E6,2.579E3,1E0,7.898548E-2,4.374563E-2,5.2610065E1,8.2E1,1.2E2,3.97E2,2.5422776E3,3.9712732E7,2.0802219E8,6.5029144E5,8.61491E3,6.4876E4,4.0844156E7,-7.445402E-2,-1.6418459E-2,2.064332E2,4.87623E-3,7.2123044E5,1.1059439E-2,4.0823767E-1,8.10114E4,6.298144E0,5.8E1,1.452881E6,4.602015E0,5.481633E-2,1.4777102E9,3.68E0,6.082581E3,1.0508E5,7.431E3,1E0,7.218466E7,4.54E2,3.7253947E5,4.0889E4,2.6284674E2,5.4106476E7,2.4347392E-1,1.2716E4,6.4446027E2,1E0,2.6835732E3,4E0,1.17E2,-1.1856262E-2,-2.837202E-4,1.1037509E2,7.218466E7,1.819149E1,5.503109E8,1.9776634E9,1.4307411E7,2.3292E4,1.6505509E-3,4.3875E1,1.75E2,2.73E2,8E0,5.388794E6,1.30399E5,3.3720784E5,1.6194E4,2.3314082E6,6.9133006E-2,2.8E2,-3.1046988E-2,-3.7625376E-3,2.271746E-2,1E0,4.1891047E5,-1.0991432E-2,9.2593566E-2,5.266129E0,2.015E3,1.2E1,1E0,1.6649964E5,6.7652373E3,1E0,5E0,2.6364462E7,6.0218544E1,1.17684984E8,1.2238596E9,-2.0829827E-2,-3.506599E-2,-4.4434495E-2,-1.3825222E-2,7.96538E-3,-1.06946E-2,-3.546534E-2,-1.7422864E-2,-6.277315E-3,-2.7267532E-2,-1.3810012E-2,-2.2943935E-3,-1.7560048E-2,1.4054912E-2,-2.1815042E-3,-1.7494887E-2,-4.9511332E-2,-2.1180192E-2,-9.699638E-3,-2.9263008E-2,4.9635353E-3,1.7797928E-2,1.8917877E-3,-1.2115964E-2,7.3791808E-3,3.1220611E-2,-2.1107988E-2,-6.5990756E-3,-3.271612E-2,-3.8631996E-3,-1.6484188E-2,-1.6219511E-3,-4.041705E-2,-1.8179415E-2,-3.0936569E-3,-4.1376576E-2,2.2838445E-2,-1.7189847E-2,-1.1571315E-2,-4.236853E-4,-8.802553E-3,-1.7797694E-2,8.224837E-3,2.0140592E-2,-1.7521737E-2,-1.9933828E-3,-1.7260598E-2,2.8170303E-3,-9.426454E-3,8.978151E-3,-2.4869086E-2,-1.2046495E-2,-9.204677E-3,-3.1405147E-2,1.3242372E-2,-1.268284E-2,2.3406796E-2,9.294755E-3,3.653343E-2,1.5484214E-2,1.7550675E-2,2.7231995E-2,3.490982E-2,5.1433202E-2,5.110159E-2,2.064687E-2,-5.770159E-4,2.4305474E-2,1.630929E-2,-6.964002E-4,-1.0069781E-2,-7.1228476E-4,2.44764E-2,-9.437193E-4,1.6457073E-2,5.199667E-2],"split_indices":[20,56,33,102,41,28,2,45,38,2,67,2,27,38,45,2,2,9,2,104,0,0,56,29,0,2,52,45,7,28,4,11,45,0,0,52,0,47,0,42,33,57,3,31,53,0,7,54,52,29,2,8,45,2,28,2,58,45,38,9,52,75,4,10,10,0,0,56,45,56,5,5,51,2,0,4,6,2,17,47,10,28,9,28,0,0,0,0,0,101,47,0,38,54,11,18,6,28,52,6,8,47,56,48,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.634E3,8.4E2,1.794E3,4.13E2,4.27E2,1.095E3,6.99E2,2.95E2,1.18E2,3.52E2,7.5E1,5.6E2,5.35E2,4.13E2,2.86E2,2.83E2,1.2E1,5.9E1,5.9E1,3.48E2,4E0,5E0,7E1,2.94E2,2.66E2,5.01E2,3.4E1,8.1E1,3.32E2,2.06E2,8E1,2.33E2,5E1,1E1,2E0,5.7E1,2E0,5.5E1,4E0,2.93E2,5.5E1,5.3E1,1.7E1,7.9E1,2.15E2,8E0,2.58E2,4.6E1,4.55E2,2.9E1,5E0,5.4E1,2.7E1,1.49E2,1.83E2,1.26E2,8E1,5E1,3E1,1.54E2,7.9E1,1.9E1,3.1E1,4.8E1,9E0,4.8E1,7E0,1.9E2,1.03E2,1.4E1,4.1E1,4.5E1,8E0,1.5E1,2E0,5.3E1,2.6E1,1.83E2,3.2E1,2.34E2,2.4E1,2.2E1,2.4E1,4.52E2,3E0,1.1E1,1.8E1,3E0,2E0,1.3E1,4.1E1,7E0,2E1,6E0,1.43E2,5.6E1,1.27E2,6.2E1,6.4E1,7.5E1,5E0,3.9E1,1.1E1,2E1,1E1,3.1E1,1.23E2,7.6E1,3E0,7E0,1.2E1,1.8E1,1.3E1,2E0,4.6E1,5E0,4E0,1.82E2,8E0,9.3E1,1E1,1.1E1,3E0,4E0,3.7E1,3.1E1,1.4E1,3E0,5E0,3E0,1.2E1,4E1,1.3E1,2.4E1,2E0,5.5E1,1.28E2,8E0,2.4E1,2.31E2,3E0,2.2E1,2E0,6E0,1.6E1,8E0,1.6E1,3.37E2,1.15E2,9E0,2E0,1.1E1,2E0,7E0,3.4E1,1.5E1,5E0,4E0,2E0,1.35E2,8E0,1.2E1,4.4E1,9.6E1,3.1E1,3.7E1,2.5E1,4.5E1,1.9E1,7.2E1,3E0,2E0,3E0,3.3E1,6E0,4E0,7E0,1.7E1,3E0,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[-3.940342E-3,-3.6695775E-1,1.6809095E-1,-5.6298876E-1,-1.3593155E-1,1.564209E-2,4.2950606E-1,8.910751E-1,-5.968302E-1,-3.0664882E-1,1.3832769E-1,-4.1079283E-2,4.251397E-1,5.829088E-3,5.047003E-1,1.0903615E-2,6.668626E-2,-6.972369E-1,-2.7466702E-1,-4.0406275E-1,-2.018088E-1,5.6622706E-2,5.175204E-1,-2.927398E-1,2.9284887E-2,3.3002976E-1,6.5466374E-1,-1.5693237E-1,4.209504E-1,3.0830398E-1,6.360723E-1,-6.7291516E-3,7.599452E-3,-5.989176E-1,-9.480622E-1,-6.0031887E-2,-4.9613988E-1,-3.3868358E-1,-5.465544E-1,-2.5439456E-1,-7.6970585E-2,1.328865E-1,-2.9399994E-1,1.0017676E-2,5.6205034E-1,-3.877959E-1,-1.73844E-1,6.728925E-2,-3.9622977E-1,2.3494028E-1,4.6916774E-1,7.094587E-1,-7.5932906E-4,-2.5370944E-1,2.0974137E-1,6.311919E-1,1.8264557E-1,6.446502E-1,2.2071388E-1,8.776205E-1,5.259085E-1,-7.13896E-1,-2.773012E-1,-1.0215458E0,-4.4469056E-1,-1.4809892E-1,4.2611438E-1,-4.001835E-1,-8.7832963E-1,-2.4752684E-2,-2.5275436E-1,-5.770845E-1,-1.0997533E-2,-2.8429908E-1,6.5235704E-2,-1.8614653E-1,2.9893992E-3,1.8249609E-1,-8.7079085E-2,-6.167658E-1,-8.0740415E-2,2.9726518E-2,1.08569395E-2,-3.567573E-1,-5.8150847E-2,-8.735131E-2,-2.6208434E-1,1.2578143E-1,-1.2658308E-1,-5.849912E-1,-1.4173315E-1,-1.1494045E-2,2.5477195E-1,5.151328E-1,-1.2903897E-2,2.7038762E-2,4.3442495E-2,1.1366823E-2,-2.7510363E-1,-2.273375E-2,3.4142634E-1,2.1020358E-2,4.053637E-2,-9.134987E-3,2.9188937E-1,3.196937E-1,7.2383595E-1,2.654588E-1,7.9278415E-3,4.9981683E-2,9.185026E-1,3.6276573E-1,6.8873954E-1,-3.543028E-2,-1.8861957E-2,8.036962E-4,-1.7629035E-2,-4.466214E-2,-7.040693E-2,-2.9637398E-2,-8.721627E-3,-1.9847872E-3,-1.3808355E-2,2.9624062E-2,1.4793124E-3,-2.4099316E-2,-7.6897545E-3,-1.650373E-2,-4.6390697E-2,-1.3556897E-2,-2.045462E-3,-1.9556813E-2,-3.3732682E-2,1.8238957E-3,-1.4265794E-2,-6.6158082E-3,7.473468E-3,-1.13970395E-2,-2.3217446E-3,2.4009808E-3,1.3477319E-2,-1.09154135E-2,3.666427E-3,-3.746451E-2,-1.5702527E-2,-7.675417E-3,8.526117E-3,-2.0725144E-2,-1.0639216E-2,-9.884179E-3,-2.5906113E-3,-1.7709967E-2,-4.829884E-3,3.5465024E-3,2.2730246E-2,-1.049206E-2,1.18358815E-2,-3.0183103E-2,-8.106523E-3,-1.7917014E-3,-3.7564598E-2,8.360126E-3,1.5763534E-2,2.942993E-2,1.3252408E-2,-1.7089332E-2,-6.645239E-3,1.886281E-2,8.538547E-4,1.702046E-2,-8.199157E-4,2.1661012E-2,6.407878E-3,2.7836595E-2,4.0326063E-2,9.415532E-3,2.2025635E-2,-7.3179584E-3,8.054962E-3,-7.926686E-3,9.535489E-3,4.5518387E-2,1.7147176E-2,1.1750841E-2,2.4499182E-2,3.9547026E-2,1.6478214E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,-1,61,63,65,67,69,71,73,75,77,79,-1,81,83,85,87,89,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,127,-1,129,131,-1,133,135,137,-1,139,141,143,145,-1,-1,147,-1,149,151,153,155,157,159,-1,161,163,-1,-1,-1,-1,165,-1,167,-1,-1,-1,169,171,173,175,177,179,181,183,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6742863E2,3.8961975E1,7.248487E1,2.34626E1,1.8630272E1,2.672882E1,2.1341965E1,4.8392897E0,1.4558884E1,2.4206104E0,4.710932E0,1.7911243E1,2.8401413E0,6.957768E0,1.448465E1,1.341773E-1,0E0,7.9895325E0,5.1991606E0,9.887047E-1,7.6818514E-1,3.4350245E0,2.5315285E-1,2.430275E0,1.2818249E1,1.2260599E0,1.4808922E0,2.6791487E0,1.3376608E0,6.6336536E0,8.585815E0,0E0,0E0,9.120728E0,3.0967255E0,2.5034742E0,1.6288776E0,1.2319984E0,2.3514557E-1,8.214512E-1,5.746952E-1,1.161694E0,1.5333585E0,0E0,3.1178427E-1,2.696331E0,7.4032116E-1,8.260114E0,3.0396318E0,6.451242E-1,1.4968157E0,5.4730225E-1,0E0,6.8513966E-1,1.6041712E0,2.4924755E-1,6.675398E-1,9.77993E-1,1.7497768E0,3.5068283E0,6.019417E0,1.243393E0,1.8221617E0,1.9413147E0,5.275402E-1,7.2091746E-1,6.654923E-1,1.030168E0,1.4840412E-1,0E0,3.8284183E-1,4.425316E-1,0E0,3.485546E-1,1.6932149E-1,1.3831574E-1,0E0,1.1119406E0,4.7840068E-1,1.7386341E-1,3.3723605E-1,0E0,0E0,1.1212406E0,0E0,1.9469771E-1,8.1370425E-1,1.0051939E1,5.896514E0,5.894995E-1,1.9148684E0,0E0,3.0050397E-1,7.303982E-1,0E0,0E0,0E0,0E0,5.857229E-1,0E0,2.3058152E-1,0E0,0E0,0E0,2.4160242E-1,2.1063948E-1,1.3417816E-1,1.9447184E0,8.757941E-1,2.2577769E-1,1.451149E0,1.970725E0,5.242012E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,73,73,74,74,75,75,77,77,78,78,79,79,80,80,83,83,85,85,86,86,87,87,88,88,89,89,90,90,92,92,93,93,98,98,100,100,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,-1,62,64,66,68,70,72,74,76,78,80,-1,82,84,86,88,90,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,128,-1,130,132,-1,134,136,138,-1,140,142,144,146,-1,-1,148,-1,150,152,154,156,158,160,-1,162,164,-1,-1,-1,-1,166,-1,168,-1,-1,-1,170,172,174,176,178,180,182,184,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,2E0,4.273439E2,2.3314082E6,2.5998926E-1,4.676E3,1.983922E3,1.913925E-1,1.4101E4,4.54E2,1E0,2.001177E6,1.0855529E3,5.9E1,6.668626E-2,3.9712732E7,6.747114E7,2.9005406E5,9.36494E5,4.0844156E7,2.1924436E5,7.805608E4,4.0844156E7,3.0872485E2,2.0189162E7,3.202814E5,4.4312353E0,8E0,2.1402586E4,-6.7291516E-3,7.599452E-3,2.291E3,4.7272725E0,9.637832E1,6.0435E4,1.074E3,1.16544716E-1,1.4000518E6,2.046E3,6.788246E0,1.7488463E3,1.0017676E-2,4.8765745E0,2.0116506E4,9.25E0,2.429717E1,3.0387878E3,2.657819E6,1.1328599E3,5.9570195E6,-7.5932906E-4,1.2897E4,4E0,5.8349776E-3,6.4625664E8,1.915148E7,6.205317E-2,3.051E3,2.6312E4,2.1E1,8.2608955E-3,1.2427474E8,6.823E3,5.418831E11,1.49E2,3.4409692E3,2.1897722E10,-2.4752684E-2,3.730007E6,4.914413E-3,-1.0997533E-2,2.28125E0,1.646E3,2.7738693E0,2.9893992E-3,3.386E3,4.0501267E8,1.116567E3,5.5325594E9,2.9726518E-2,1.08569395E-2,5.222222E1,-5.8150847E-2,1.56E2,1.5046341E2,8.5023944E2,7.06699E8,6.8E1,1.6338193E-1,-1.1494045E-2,4.5723195E6,2.0771764E4,-1.2903897E-2,2.7038762E-2,4.3442495E-2,1.1366823E-2,1.6213043E3,-2.273375E-2,8.48E2,2.1020358E-2,4.053637E-2,-9.134987E-3,2.342147E7,2.2444444E0,1.2774637E10,3.794679E8,4.646E3,1.2633843E3,1.4122614E2,2.4042394E5,3.5301748E7,-3.543028E-2,-1.8861957E-2,8.036962E-4,-1.7629035E-2,-4.466214E-2,-7.040693E-2,-2.9637398E-2,-8.721627E-3,-1.9847872E-3,-1.3808355E-2,2.9624062E-2,1.4793124E-3,-2.4099316E-2,-7.6897545E-3,-1.650373E-2,-4.6390697E-2,-1.3556897E-2,-2.045462E-3,-1.9556813E-2,-3.3732682E-2,1.8238957E-3,-1.4265794E-2,-6.6158082E-3,7.473468E-3,-1.13970395E-2,-2.3217446E-3,2.4009808E-3,1.3477319E-2,-1.09154135E-2,3.666427E-3,-3.746451E-2,-1.5702527E-2,-7.675417E-3,8.526117E-3,-2.0725144E-2,-1.0639216E-2,-9.884179E-3,-2.5906113E-3,-1.7709967E-2,-4.829884E-3,3.5465024E-3,2.2730246E-2,-1.049206E-2,1.18358815E-2,-3.0183103E-2,-8.106523E-3,-1.7917014E-3,-3.7564598E-2,8.360126E-3,1.5763534E-2,2.942993E-2,1.3252408E-2,-1.7089332E-2,-6.645239E-3,1.886281E-2,8.538547E-4,1.702046E-2,-8.199157E-4,2.1661012E-2,6.407878E-3,2.7836595E-2,4.0326063E-2,9.415532E-3,2.2025635E-2,-7.3179584E-3,8.054962E-3,-7.926686E-3,9.535489E-3,4.5518387E-2,1.7147176E-2,1.1750841E-2,2.4499182E-2,3.9547026E-2,1.6478214E-2],"split_indices":[20,102,33,17,52,28,42,1,55,41,2,2,100,9,52,29,0,45,45,33,9,45,28,28,45,4,45,28,54,3,32,0,0,2,54,56,10,2,39,33,2,53,4,0,57,33,58,53,4,28,4,45,0,1,3,34,7,12,39,2,2,3,27,45,2,31,8,52,5,0,9,38,0,54,2,56,0,2,7,4,5,0,0,4,0,2,55,52,7,3,38,0,28,47,0,0,0,0,4,0,0,0,0,0,45,56,31,7,2,4,56,28,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.679E3,8.61E2,1.818E3,4.65E2,3.96E2,1.149E3,6.69E2,1E1,4.55E2,2.44E2,1.52E2,1.01E3,1.39E2,1.01E2,5.68E2,4E0,6E0,3.46E2,1.09E2,1.25E2,1.19E2,1.26E2,2.6E1,2.2E2,7.9E2,1E2,3.9E1,7.3E1,2.8E1,2.29E2,3.39E2,2E0,2E0,2.51E2,9.5E1,5.6E1,5.3E1,8.8E1,3.7E1,8.3E1,3.6E1,1.04E2,2.2E1,4E0,2.2E1,1.21E2,9.9E1,7.26E2,6.4E1,6.1E1,3.9E1,3.6E1,3E0,5.8E1,1.5E1,1.4E1,1.4E1,4.6E1,1.83E2,1.04E2,2.35E2,1.84E2,6.7E1,8.2E1,1.3E1,4.8E1,8E0,4.4E1,9E0,2.7E1,6.1E1,3.3E1,4E0,7.6E1,7E0,2E1,1.6E1,8.5E1,1.9E1,8E0,1.4E1,1.8E1,4E0,1.18E2,3E0,5.1E1,4.8E1,5.58E2,1.68E2,3.6E1,2.8E1,2E0,5.9E1,3.7E1,2E0,2.3E1,1.3E1,2E0,5.6E1,2E0,1.3E1,9E0,5E0,3E0,1.1E1,1E1,3.6E1,1.51E2,3.2E1,5E0,9.9E1,1.19E2,1.16E2,1.69E2,1.5E1,1.6E1,5.1E1,7.1E1,1.1E1,7E0,6E0,2.8E1,2E1,5E0,3E0,3E1,1.4E1,2E0,7E0,5.3E1,8E0,1.6E1,1.7E1,3E0,7.3E1,2E0,5E0,1.4E1,6E0,3.7E1,4.8E1,1E1,9E0,4E0,4E0,1.1E1,3E0,7.4E1,4.4E1,1E1,4.1E1,2.8E1,2E1,4.87E2,7.1E1,1.35E2,3.3E1,3.2E1,4E0,2.5E1,3E0,3E1,2.9E1,2.5E1,1.2E1,3.4E1,2.2E1,1.1E1,2E0,9E0,2E0,5E0,5E0,1.9E1,1.7E1,1.13E2,3.8E1,1.6E1,1.6E1,2E0,3E0,9.3E1,6E0,6.8E1,5.1E1,8.2E1,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[4.1392664E-3,-3.3872607E-1,1.652893E-1,-5.57618E-1,-1.4421193E-1,1.926719E-2,4.1324338E-1,-6.5777576E-1,-3.1092346E-1,-2.491947E-1,3.047966E-1,-2.7809837E-1,9.0273544E-2,3.0791126E-3,4.8405278E-1,-5.0943136E-1,-7.7347136E-1,-3.9476103E-1,-1.3337931E-1,-2.6916647E-1,6.142199E-2,2.2677243E-1,5.9562546E-1,-4.024755E-1,-1.5346384E-1,2.1885608E-1,-5.881808E-2,1.4628533E-1,-3.6244982E-1,4.4217822E-1,1.001584E0,-5.823936E-1,-7.901949E-2,-9.6072274E-1,-6.072205E-1,-4.3235815E-1,-4.050857E-3,-3.961166E-2,-2.8701794E-1,-2.0821539E-1,-6.2311614E-1,1.2306538E-1,4.015321E-1,9.960072E-3,6.915271E-1,-3.707553E-1,-4.8232306E-2,5.581945E-2,-2.0989293E-1,5.3908084E-2,1.9266136E-1,-1.9294934E-1,8.057769E-2,1.8580418E-2,3.813336E-1,-1.9697133E-2,-5.10806E-3,1.5266545E-1,4.933439E-1,1.0429623E0,1.615854E-2,-8.7981194E-2,-6.1994034E-1,-1.844539E-1,1.8621355E-2,-8.2348293E-1,-1.4067496E0,-6.905495E-1,-3.8195306E-1,-4.835756E-1,-2.6403704E-1,-7.718894E-2,1.3132087E-2,-1.5709234E-2,-6.538714E-4,-2.941284E-1,-6.207406E-2,-8.669495E-1,-5.161007E-1,1.6730997E-1,-7.0562693E-3,4.6479002E-1,5.421565E-3,3.985905E-2,2.068201E-2,-4.3931797E-1,-2.4799885E-1,-1.8314597E-1,7.611014E-3,-9.0402186E-2,-2.8787932E-1,1.5248914E-2,2.9286507E-1,-4.4649893E-1,-1.092915E-1,6.1810523E-2,5.0528973E-2,-1.555497E-1,1.7860128E-1,2.7916614E-2,1.9463786E-1,-3.5069543E-1,2.0364206E-1,4.1897413E-1,7.446038E-1,5.9212983E-2,4.2020526E-2,-9.865256E-3,5.3175846E-3,-3.2971278E-2,-2.172833E-2,-1.9469347E-2,-1.2623376E-3,-4.178595E-2,-1.601854E-2,-4.2116594E-2,-7.831036E-2,-1.7896062E-2,-3.7041195E-2,-2.3268951E-2,-9.38974E-3,-1.0504885E-2,-2.4819799E-2,-2.1493426E-3,-1.4478454E-2,-6.2234136E-3,2.1348954E-3,-1.5292731E-2,1.3223458E-2,-1.324724E-3,-1.5238406E-2,-4.545038E-2,-1.0641752E-2,-7.231449E-3,-2.5938587E-2,3.9575254E-3,1.4507128E-2,8.22095E-3,2.5078539E-2,-2.2074101E-2,-1.1801965E-2,6.3827382E-3,-1.4590142E-2,-1.4283853E-2,3.128987E-4,-1.574744E-2,-2.858776E-3,-1.9985612E-2,-1.013208E-2,-1.688613E-2,2.4444999E-3,7.499787E-3,1.983767E-2,-2.6241193E-2,-1.7880421E-3,-7.779946E-3,1.3079105E-2,5.9997174E-3,-9.485768E-3,-1.4606297E-2,-1.2939525E-3,-2.2007927E-2,1.2017079E-2,1.4505352E-2,-8.561137E-3,-6.894331E-3,-3.0997986E-2,-1.385513E-2,1.1610181E-2,1.6169662E-2,2.5808292E-2,3.89333E-2,1.6615016E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,-1,83,85,-1,87,89,-1,91,93,95,97,99,-1,-1,101,103,105,-1,107,109,111,-1,113,115,117,119,121,123,125,-1,-1,-1,127,129,131,133,135,-1,137,-1,-1,-1,139,141,143,-1,145,147,149,151,153,155,157,-1,159,161,-1,163,165,167,169,171,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.503435E2,3.6962082E1,6.701078E1,9.901207E0,2.1853785E1,2.4645014E1,1.992234E1,4.5831833E0,1.73353E0,1.1817085E1,1.8617964E0,3.426115E0,1.8071558E1,5.3923144E0,1.2234894E1,4.0364723E0,4.428673E0,9.080267E-1,5.6488407E-1,7.874069E0,0E0,1.2373757E0,5.2535105E-1,1.7747555E0,1.3580179E0,1.1802286E1,8.186281E0,2.2141595E0,3.184576E-1,7.9682007E0,8.298454E-1,2.0060158E0,1.0582736E0,3.3712997E0,1.3881187E0,4.930277E-1,0E0,3.2777315E-1,1.6366434E-1,3.974372E0,9.616375E-1,5.749979E-1,4.2159176E-1,0E0,1.523037E-1,8.0003166E-1,0E0,6.450743E-1,8.126538E-1,0E0,8.759783E0,4.681346E0,3.8668952E0,1.3931395E0,8.650851E-1,0E0,0E0,2.2011654E0,8.302345E0,1.9610214E-1,0E0,2.3425594E-1,8.6101913E-1,5.8616227E-1,0E0,1.12323E0,4.89357E-1,1.3659668E0,4.0700865E-1,3.8121128E-1,1.4585733E-1,1.5998547E-1,0E0,0E0,0E0,2.9620323E0,1.0400273E0,5.83086E-1,2.9093552E-1,4.3427718E-1,0E0,2.7251625E-1,0E0,0E0,0E0,1.698513E-1,9.262886E-1,1.766277E-1,0E0,2.5126445E-1,4.4510698E-1,2.3748798E0,5.144413E0,2.262392E0,3.4799013E0,3.5216599E0,0E0,4.5275396E-1,1.3153472E0,0E0,6.604448E-1,3.973145E-1,1.5233591E0,3.3101387E0,2.5451584E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,47,47,48,48,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,75,75,76,76,77,77,78,78,79,79,81,81,85,85,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,98,98,100,100,101,101,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,-1,84,86,-1,88,90,-1,92,94,96,98,100,-1,-1,102,104,106,-1,108,110,112,-1,114,116,118,120,122,124,126,-1,-1,-1,128,130,132,134,136,-1,138,-1,-1,-1,140,142,144,-1,146,148,150,152,154,156,158,-1,160,162,-1,164,166,168,170,172,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.896243E4,1E0,1.5560952E0,4.54E2,2.5998926E-1,6.7033327E-1,2.0368582E8,1.3144558E4,3.3060927E0,6.6126086E4,2E1,5.9767612E7,7.0484486E0,3.74E3,1.36054E-1,3.7382904E7,5.59E2,1E0,6.142199E-2,5.148149E5,2.1924436E5,2.0116506E4,1.3170732E0,2.04E5,9.51668E5,2.001177E6,5.464732E8,1.0617118E8,6.292039E0,7.9E1,1.3141646E3,2.8681312E8,1.3928872E0,1E0,-4.050857E-3,2.67162E5,1.243766E0,3.4976208E-1,8.10114E4,4.5510144E9,2.8477445E0,9.960072E-3,2.093563E0,4.7E2,-4.8232306E-2,3.25E2,1.2747E4,5.3908084E-2,4.2648572E5,1.5139E4,8.791777E3,1.857051E5,4.822785E0,-1.9697133E-2,-5.10806E-3,4.97E2,2.2214102E6,2.231E3,1.615854E-2,1.4925373E-1,1.5053125E2,3.5912186E-1,1.8621355E-2,2.074E3,5.1160636E11,1.2E1,1.152553E6,2.9883E4,1.0376E4,1E0,1.3132087E-2,-1.5709234E-2,-6.538714E-4,3.0497742E3,7.526755E7,3.4613644E1,5.503109E8,1.4564946E3,-7.0562693E-3,1.308827E6,5.421565E-3,3.985905E-2,2.068201E-2,1.14E2,9.928469E-1,3.3820656E7,7.611014E-3,1.21E2,1.3325E2,1.8731416E4,9.99E2,6.650572E-3,9.9625344E5,4.0844156E7,5.0528973E-2,8.05E2,4E0,2.7916614E-2,1E0,4.6E0,1.2E1,1.3664E4,1E0,5.9212983E-2,4.2020526E-2,-9.865256E-3,5.3175846E-3,-3.2971278E-2,-2.172833E-2,-1.9469347E-2,-1.2623376E-3,-4.178595E-2,-1.601854E-2,-4.2116594E-2,-7.831036E-2,-1.7896062E-2,-3.7041195E-2,-2.3268951E-2,-9.38974E-3,-1.0504885E-2,-2.4819799E-2,-2.1493426E-3,-1.4478454E-2,-6.2234136E-3,2.1348954E-3,-1.5292731E-2,1.3223458E-2,-1.324724E-3,-1.5238406E-2,-4.545038E-2,-1.0641752E-2,-7.231449E-3,-2.5938587E-2,3.9575254E-3,1.4507128E-2,8.22095E-3,2.5078539E-2,-2.2074101E-2,-1.1801965E-2,6.3827382E-3,-1.4590142E-2,-1.4283853E-2,3.128987E-4,-1.574744E-2,-2.858776E-3,-1.9985612E-2,-1.013208E-2,-1.688613E-2,2.4444999E-3,7.499787E-3,1.983767E-2,-2.6241193E-2,-1.7880421E-3,-7.779946E-3,1.3079105E-2,5.9997174E-3,-9.485768E-3,-1.4606297E-2,-1.2939525E-3,-2.2007927E-2,1.2017079E-2,1.4505352E-2,-8.561137E-3,-6.894331E-3,-3.0997986E-2,-1.385513E-2,1.1610181E-2,1.6169662E-2,2.5808292E-2,3.89333E-2,1.6615016E-2],"split_indices":[20,56,33,102,42,2,42,27,7,52,35,28,3,45,35,2,42,12,0,104,0,28,28,33,53,5,1,9,7,7,38,11,52,7,41,105,0,1,39,39,33,5,57,0,39,1,0,2,12,0,28,9,52,28,54,0,0,2,28,0,0,56,52,27,0,2,31,18,32,1,9,8,0,0,0,52,45,56,5,4,0,1,0,0,0,29,53,7,0,2,55,28,2,41,28,45,0,10,3,0,74,54,18,2,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.719E3,8.69E2,1.85E3,4.08E2,4.61E2,1.165E3,6.85E2,2.89E2,1.19E2,3.74E2,8.7E1,2.24E2,9.41E2,1.01E2,5.84E2,1.29E2,1.6E2,8E1,3.9E1,3.7E2,4E0,7E1,1.7E1,1.11E2,1.13E2,5.05E2,4.36E2,7.3E1,2.8E1,5.42E2,4.2E1,1.1E2,1.9E1,7.3E1,8.7E1,7.1E1,9E0,2.5E1,1.4E1,3.17E2,5.3E1,4.5E1,2.5E1,4E0,1.3E1,1.07E2,4E0,2.4E1,8.9E1,1.3E1,4.92E2,2.22E2,2.14E2,4.8E1,2.5E1,2.3E1,5E0,8.2E1,4.6E2,3.9E1,3E0,8E0,1.02E2,1.6E1,3E0,5.8E1,1.5E1,6.2E1,2.5E1,5.3E1,1.8E1,2.3E1,2E0,1.2E1,2E0,1.99E2,1.18E2,1.4E1,3.9E1,3.9E1,6E0,2E1,5E0,7E0,6E0,6.7E1,4E1,7E0,1.7E1,3.6E1,5.3E1,1.78E2,3.14E2,5.4E1,1.68E2,2.11E2,3E0,2.3E1,2.5E1,1.1E1,1.4E1,7E0,7.5E1,3.57E2,1.03E2,1.5E1,2.4E1,5E0,3E0,7E1,3.2E1,6E0,1E1,5.2E1,6E0,6E0,9E0,1.4E1,4.8E1,1.5E1,1E1,7E0,4.6E1,3E0,1.5E1,1.6E1,7E0,1.91E2,8E0,1.05E2,1.3E1,1.2E1,2E0,3E0,3.6E1,2.5E1,1.4E1,4E0,1.6E1,5.9E1,8E0,5E0,3.5E1,4E0,3E0,3E0,3.3E1,1.8E1,3.5E1,1.5E1,1.63E2,1.49E2,1.65E2,4.3E1,1.1E1,1.48E2,2E1,1.7E2,4.1E1,1E1,1.3E1,2E0,2.3E1,1.1E1,3E0,5E0,2E0,5E0,7E1,2.15E2,1.42E2,8.7E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"173","size_leaf_vector":"1"}},{"base_weights":[-4.66578E-3,-3.454011E-1,1.5617731E-1,-5.204079E-1,-1.3271241E-1,7.499103E-3,3.8000122E-1,-6.225245E-1,-2.9486296E-1,-2.3698321E-1,2.4995245E-1,-2.9048836E-1,5.8320526E-2,2.520562E-1,5.814602E-1,-1.2812632E0,-5.90869E-1,-3.857814E-1,-1.22901596E-1,-2.5281426E-1,4.910802E-2,7.1488485E-2,3.7596777E-1,-4.3122056E-1,-2.3508063E-1,1.8283927E-1,-6.138081E-2,3.204046E-1,-1.08374886E-1,1.850741E-1,6.681728E-1,-2.1471651E-2,-6.62192E-2,-6.04459E-1,2.8891351E-2,-3.593749E-1,-3.478693E-2,-1.7141923E-1,4.926316E-3,-2.0135568E-1,-6.425572E-1,1.3153975E-1,-1.2328083E-1,5.1727355E-1,2.0722115E-1,-4.3892553E-3,-2.1321265E-2,-1.8168747E-1,-3.135421E-1,8.92581E-1,1.5690958E-1,-1.18823946E-1,4.45137E-1,2.0745718E-1,4.930029E-1,-3.5799405E-1,1.5118857E-1,5.4417275E-2,2.790752E-1,6.3266456E-1,6.0508516E-2,-4.842061E-1,-7.030904E-1,-3.9791638E-1,-1.9035153E-1,-1.7805612E-2,-2.1607602E-1,-2.7094245E-1,-4.67146E-2,-5.255081E-1,-4.654526E-2,2.00839E-2,1.0861377E-2,1.991162E-3,-1.0183419E-2,7.2082746E-1,3.1994554E-1,1.2434062E-3,2.8032938E-1,-2.9073268E-1,-1.0979963E-1,-3.35321E-1,-3.2513223E-3,2.335298E-2,4.72645E-2,9.465136E-2,3.9153588E-1,-2.0452768E-1,3.8235947E-2,6.392268E-2,3.1370714E-1,2.6482764E-1,-1.1810554E-1,9.123972E-1,3.599169E-1,-5.1740825E-1,-1.9662157E-1,2.5160512E-1,-6.4678583E-3,-4.714877E-3,1.425105E-1,2.662728E-3,1.5399018E-2,6.976434E-1,3.7170902E-1,-2.7662793E-2,-1.3851189E-2,-3.6894362E-2,-1.83417E-2,-1.9801801E-2,-1.0619635E-3,-1.0737708E-2,3.112108E-3,7.1985847E-3,-4.771783E-3,-1.2324916E-2,-2.3823907E-3,-1.0306834E-2,-2.9002203E-2,5.225078E-3,-1.7614838E-2,-1.6823642E-2,-3.0372806E-2,4.7458243E-3,-3.5001414E-3,1.16110025E-2,3.7941676E-2,1.8020436E-2,-1.4767415E-3,2.0525556E-4,1.6243292E-2,-1.001888E-2,-2.4205467E-2,-2.4753315E-3,-8.9364415E-3,-1.6837103E-2,-4.454629E-3,-6.087485E-3,7.2758067E-3,1.5741266E-2,3.6060594E-2,-2.1874592E-2,-6.798603E-3,-1.2197411E-3,9.1803605E-3,1.953457E-2,-4.689556E-3,1.5821604E-2,4.193816E-3,-4.0387303E-2,-2.3273611E-3,5.1823054E-2,3.3139817E-2,1.0876971E-2,2.2583893E-2,-2.670788E-2,-6.231729E-3,1.4904204E-3,-1.3151253E-2,1.833648E-4,2.481432E-2,-4.9353323E-5,1.0266969E-2,4.856328E-2,3.1363934E-2,6.8832855E-3,3.047126E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,-1,61,-1,63,-1,65,-1,67,69,71,73,75,77,-1,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,121,-1,123,-1,-1,-1,125,127,-1,129,131,133,135,-1,-1,-1,137,139,141,143,-1,145,147,149,151,153,155,157,159,-1,-1,161,-1,-1,163,165,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4715038E2,3.1979042E1,6.069575E1,1.0687485E1,1.5620537E1,1.6630074E1,1.8618118E1,5.9992294E0,2.2821217E0,6.394697E0,1.862215E0,1.1462517E0,1.3992168E1,1.10465145E1,9.569153E0,5.631275E-1,5.411705E0,6.113863E-1,5.981803E-1,5.9675255E0,0E0,4.372775E-1,1.0612082E0,1.6818619E-1,4.3318224E-1,8.325497E0,1.3989862E1,7.227665E0,4.7191496E0,6.234716E-1,4.0835495E0,0E0,0E0,3.3126678E0,0E0,5.235815E-1,0E0,2.9798782E-1,0E0,2.892805E0,8.385172E-1,2.9271695E-1,1.4142455E-1,8.1371355E-1,3.1075597E-1,0E0,0E0,5.3248763E-1,2.2333527E-1,1.3003063E-1,6.4632425E0,5.8093476E0,5.310542E0,4.308467E0,7.932312E0,8.500943E-1,1.0687503E0,3.237638E-1,2.6685762E-1,3.4711304E0,0E0,2.4194527E0,3.3025055E0,4.235878E-1,1.7884749E-1,1.6631295E-1,2.20644E-1,3.3383322E0,4.2929816E0,3.074684E-1,0E0,1.1030639E-1,0E0,0E0,0E0,2.179699E-1,3.0803967E-1,0E0,2.7469528E-1,3.84382E-1,1.9119602E-1,1.3270807E-1,0E0,0E0,0E0,4.502903E0,1.8649292E0,4.33817E0,1.500631E0,0E0,1.7459431E0,2.2365503E0,1.7233174E0,5.604229E-1,1.5832233E0,2.0821142E-1,3.637755E-1,1.7811681E0,0E0,0E0,1.5497303E-1,0E0,0E0,1.7021332E0,2.683004E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,37,37,39,39,40,40,41,41,42,42,43,43,44,44,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,75,75,76,76,78,78,79,79,80,80,81,81,85,85,86,86,87,87,88,88,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,100,100,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,-1,62,-1,64,-1,66,-1,68,70,72,74,76,78,-1,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,122,-1,124,-1,-1,-1,126,128,-1,130,132,134,136,-1,-1,-1,138,140,142,144,-1,146,148,150,152,154,156,158,160,-1,-1,162,-1,-1,164,166,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.330579E0,1.0402391E6,1E0,1.4930757E0,3.5E2,1.5283889E0,3E0,2.0368582E8,1.3144558E4,1.3097832E3,4.1E1,1.8E1,3.9712732E7,6.4446027E2,1.9041E4,4.9386E4,1E0,2.437E3,1E0,4.910802E-2,5.8426323E0,7.6247287E-1,4E0,9.25E0,3.92382E5,1.0409E4,6.071E3,6.513793E1,3.7253947E5,1.8114515E1,-2.1471651E-2,-6.62192E-2,6.748543E-1,2.8891351E-2,1.998E3,-3.478693E-2,2.25E2,4.926316E-3,1.0323588E3,1.3649979E10,3.1648794E-1,3.2794893E-1,1.4075E4,1.4157E4,-4.3892553E-3,-2.1321265E-2,2.5167784E-1,1.8224286E2,2.127E3,2.3314082E6,1.645614E6,7.969174E7,4.22088E0,2E0,1.7495675E5,1.5902449E0,3.101286E0,3.0291306E2,1.4285097E1,6.0508516E-2,2.046E3,1.0118026E0,3.098804E2,3.771981E6,1.0214149E6,2.04946E6,4.8297736E7,8.965631E7,2.9910075E5,-4.654526E-2,1.15062E5,1.0861377E-2,1.991162E-3,-1.0183419E-2,3.8910418E6,2.077501E6,1.2434062E-3,1.9776634E9,3.64299E5,3.9E2,2.4E1,-3.2513223E-3,2.335298E-2,4.72645E-2,1E0,2.5870967E0,6.061028E4,6.932264E8,6.392268E-2,2.342147E7,1E0,9.3E1,6.797394E8,4.7658E4,7.2530017E0,1E0,2.8541E4,-6.4678583E-3,-4.714877E-3,2.1515152E0,2.662728E-3,1.5399018E-2,1.0918E4,9.287868E6,-2.7662793E-2,-1.3851189E-2,-3.6894362E-2,-1.83417E-2,-1.9801801E-2,-1.0619635E-3,-1.0737708E-2,3.112108E-3,7.1985847E-3,-4.771783E-3,-1.2324916E-2,-2.3823907E-3,-1.0306834E-2,-2.9002203E-2,5.225078E-3,-1.7614838E-2,-1.6823642E-2,-3.0372806E-2,4.7458243E-3,-3.5001414E-3,1.16110025E-2,3.7941676E-2,1.8020436E-2,-1.4767415E-3,2.0525556E-4,1.6243292E-2,-1.001888E-2,-2.4205467E-2,-2.4753315E-3,-8.9364415E-3,-1.6837103E-2,-4.454629E-3,-6.087485E-3,7.2758067E-3,1.5741266E-2,3.6060594E-2,-2.1874592E-2,-6.798603E-3,-1.2197411E-3,9.1803605E-3,1.953457E-2,-4.689556E-3,1.5821604E-2,4.193816E-3,-4.0387303E-2,-2.3273611E-3,5.1823054E-2,3.3139817E-2,1.0876971E-2,2.2583893E-2,-2.670788E-2,-6.231729E-3,1.4904204E-3,-1.3151253E-2,1.833648E-4,2.481432E-2,-4.9353323E-5,1.0266969E-2,4.856328E-2,3.1363934E-2,6.8832855E-3,3.047126E-2],"split_indices":[20,56,47,102,42,2,41,3,7,52,4,2,3,45,52,9,2,77,2,104,0,53,57,0,58,5,2,2,56,28,35,0,0,27,0,10,0,10,0,52,5,38,38,9,9,0,0,58,4,1,28,9,7,53,17,33,35,41,52,54,0,2,39,52,1,48,9,45,45,28,0,29,0,0,0,45,29,0,5,9,12,3,0,0,0,102,53,28,7,0,45,14,0,7,10,53,67,2,0,0,53,0,0,9,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,8.6E2,1.823E3,4.71E2,3.89E2,1.096E3,7.27E2,3.23E2,1.48E2,3.06E2,8.3E1,1.59E2,9.37E2,4.46E2,2.81E2,1.3E1,3.1E2,9.6E1,5.2E1,3.03E2,3E0,3.5E1,4.8E1,4.3E1,1.16E2,4.59E2,4.78E2,3.75E2,7.1E1,5.1E1,2.3E2,2E0,1.1E1,3.07E2,3E0,9.1E1,5E0,4.3E1,9E0,2.69E2,3.4E1,2.7E1,8E0,2.5E1,2.3E1,2E0,4.1E1,7.1E1,4.5E1,1.5E1,4.44E2,4.3E2,4.8E1,2.28E2,1.47E2,3.6E1,3.5E1,2.2E1,2.9E1,2.19E2,1.1E1,1.41E2,1.66E2,7.3E1,1.8E1,1E1,3.3E1,1.85E2,8.4E1,2.7E1,7E0,1.3E1,1.4E1,3E0,5E0,1.1E1,1.4E1,7E0,1.6E1,2.7E1,4.4E1,4.1E1,4E0,4E0,1.1E1,3.52E2,9.2E1,2.78E2,1.52E2,5E0,4.3E1,1.94E2,3.4E1,3.4E1,1.13E2,1.7E1,1.9E1,2.6E1,9E0,8E0,1.4E1,5E0,2.4E1,1.74E2,4.5E1,9.4E1,4.7E1,1.36E2,3E1,7E1,3E0,1.6E1,2E0,3E0,7E0,2.6E1,7E0,1.6E2,2.5E1,5.7E1,2.7E1,1.2E1,1.5E1,7E0,6E0,2E0,9E0,1.2E1,2E0,3E0,1.3E1,2.1E1,6E0,2.6E1,1.8E1,3.8E1,3E0,7.2E1,2.8E2,8E1,1.2E1,5.4E1,2.24E2,1.08E2,4.4E1,3.5E1,8E0,1.41E2,5.3E1,2E0,3.2E1,1.7E1,1.7E1,5.3E1,6E1,1.5E1,2E0,5E0,1.4E1,1.4E1,1.2E1,5E0,9E0,1.8E1,1.56E2,2.5E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"167","size_leaf_vector":"1"}},{"base_weights":[2.6937397E-3,-3.194539E-1,1.5231116E-1,-4.913423E-1,-1.18760586E-1,6.9139555E-2,5.200848E-1,-5.9034604E-1,-2.7204713E-1,-2.3735517E-1,2.949747E-1,-5.319983E-2,3.0064428E-1,6.0685134E-1,1.6792245E-1,-6.339281E-1,-1.2254589E-1,-3.395377E-1,-1.0005719E-1,-1.441261E-1,-4.8610166E-1,9.58362E-2,4.1858348E-1,-3.1337887E-1,-1.2541638E-2,3.2553402E-1,-3.6755174E-1,9.192817E-1,5.263071E-1,-3.8451374E-2,4.6411547E-1,-5.5510205E-1,-8.616017E-1,3.1575095E-2,-2.8032812E-1,-4.1089502E-1,-2.0114641E-1,-1.856754E-1,3.6774024E-2,-2.1052109E-1,4.1793716E-1,-6.018875E-1,-3.3314762E-1,2.4706568E-1,-4.7569303E-3,-3.3420238E-3,4.6984115E-1,-3.394702E-1,4.1723487E-1,-9.149867E-2,1.0757618E-1,2.4472596E-1,5.8084095E-1,-4.9893895E-1,2.7701756E-4,1.0257202E0,5.0468916E-1,4.5080528E-1,9.336263E-1,-4.825508E-1,3.6633287E-2,3.1382203E-1,4.167263E-2,-5.891922E-1,-1.2138844E-1,-6.7874014E-1,-1.057143E0,-3.276434E-1,7.6669473E-3,-2.538854E-1,-4.871981E-1,-3.4732756E-1,-1.0749297E-1,-7.7762246E-2,-1.3773647E-2,6.223159E-3,-1.148917E-1,-4.6709093E-1,-1.5548837E-1,5.327621E-2,2.234101E-1,-3.6035302E-1,-6.962979E-1,-1.4601439E-2,-4.623849E-1,1.5146034E-2,4.889199E-3,8.138211E-3,-5.3017806E-2,2.287168E-1,5.505606E-1,-2.673376E-1,-4.8601162E-1,3.4250062E-2,-9.473923E-4,-2.4089362E-2,-2.132216E-1,2.1752708E-1,1.665665E-2,2.2128503E-1,6.161153E-1,6.3161933E-1,2.6082778E-1,-2.7001845E-2,-3.5000786E-3,8.009486E-3,-5.987215E-3,1.0762048E0,2.0923112E-2,2.8408049E-2,7.5318483E-3,3.5010093E-1,6.0399264E-1,9.851632E-1,4.30389E-3,-2.9383853E-2,-7.0407623E-3,2.1347733E-1,-9.984948E-2,5.3211357E-3,4.2172077E-1,-1.7243562E-2,-3.036037E-2,3.0039465E-3,-1.2285865E-2,-6.551776E-3,-3.3698786E-2,-3.5198424E-2,-5.776744E-2,-1.761137E-3,-1.7797781E-2,-1.5294387E-2,-2.0260303E-4,-2.4285527E-2,-7.7273E-3,-1.03474455E-2,-2.9762484E-2,4.0359926E-4,-7.948868E-3,5.157741E-3,-5.63419E-3,2.924477E-3,-9.4578015E-3,-7.666724E-3,-2.6441582E-2,-1.085656E-2,-2.1946235E-3,1.380741E-2,-1.1516257E-2,-1.9913303E-2,3.4777306E-6,-3.5714786E-2,-1.341211E-2,-6.0083307E-3,1.1420933E-2,-2.5039101E-2,-7.837343E-3,1.8195275E-4,-8.011385E-3,-2.954847E-3,1.3517644E-2,2.9255943E-2,1.2597094E-2,-1.8326944E-2,-1.5123065E-3,-2.9279768E-2,-1.6758831E-2,-5.0736316E-3,1.2241925E-2,-2.6952317E-2,-6.4601973E-3,7.910658E-3,2.0348242E-2,-6.9334256E-3,5.0431653E-3,6.0268547E-3,1.4960297E-2,1.5487161E-2,3.501578E-2,3.335104E-2,1.4940443E-2,5.4204254E-3,2.2828024E-2,3.1745672E-2,5.425941E-2,1.1852521E-2,1.9575668E-2,3.5509393E-2,2.3659457E-2,4.8983056E-2,1.4023721E-2,1.6855178E-2,1.4850127E-3,-8.805697E-3,1.5617014E-3,3.1090884E-2,1.4398778E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,121,123,125,127,129,-1,131,133,135,137,139,-1,-1,141,143,145,-1,147,149,151,153,155,-1,-1,-1,157,159,161,163,165,-1,-1,167,169,171,173,175,177,179,181,-1,-1,-1,-1,183,-1,-1,-1,185,187,189,-1,-1,-1,191,193,-1,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2902888E2,2.9229485E1,5.5879494E1,9.761345E0,1.9353254E1,4.227264E1,1.0195435E1,6.3240433E0,1.637536E0,7.026203E0,2.1260276E0,1.0331178E1,8.701355E0,6.31057E0,4.174447E0,4.6724014E0,3.6390243E0,9.3449783E-1,4.9886823E-1,8.447258E0,1.2694054E0,5.3361785E-1,1.3845491E0,2.6579113E0,8.03343E0,1.0132015E1,9.1837E-1,1.8652763E0,6.25259E0,1.3848246E0,1.4356647E0,3.1221085E0,1.8569756E0,0E0,5.5258095E-1,6.7898846E-1,4.6920836E-1,2.6362848E-1,2.5515524E-1,2.76544E0,2.9939399E0,7.980423E-1,1.5576339E0,1.0815942E-1,1.9448532E-1,0E0,8.2778454E-1,1.2230844E0,6.6517043E-1,4.187751E0,3.3552623E0,3.1718311E0,1.7689743E0,3.4563613E-1,1.4643167E-1,7.7833176E-1,3.064022E-1,2.6424713E0,1.3464565E0,1.7354846E-1,8.9021754E-1,4.3433404E-1,0E0,1.8229675E0,4.3910864E-1,4.439888E-1,6.98555E-1,2.6783872E-1,0E0,3.9096236E-1,2.0358467E-1,3.6950743E-1,1.6411495E-1,1.1622991E-1,0E0,0E0,1.1379375E-1,8.2531357E-1,1.2936702E0,0E0,6.714636E-1,2.9984498E-1,4.499483E-1,3.6659348E-1,4.0205765E-1,0E0,0E0,0E0,1.17497966E-1,2.3511845E-1,4.6378708E-1,2.3809853E0,4.8552322E-1,0E0,0E0,7.5825195E0,4.9073286E0,1.5796914E0,2.6627736E0,3.0820065E0,5.044613E-1,1.8949165E0,5.1967204E-1,0E0,0E0,0E0,0E0,1.4081955E-1,0E0,0E0,0E0,5.8497906E-1,7.221203E-1,5.08543E-1,0E0,0E0,0E0,3.9336407E-1,2.39715E-1,0E0,1.9036412E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,88,88,89,89,90,90,91,91,92,92,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,107,107,111,111,112,112,113,113,117,117,118,118,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,122,124,126,128,130,-1,132,134,136,138,140,-1,-1,142,144,146,-1,148,150,152,154,156,-1,-1,-1,158,160,162,164,166,-1,-1,168,170,172,174,176,178,180,182,-1,-1,-1,-1,184,-1,-1,-1,186,188,190,-1,-1,-1,192,194,-1,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.330579E0,9.075E3,1E0,1.4722673E0,7.733016E5,3.5301748E7,1.3999657E3,1.4887473E-1,5.9767612E7,1.289627E3,9.6E1,4.898012E-1,2E0,5.528107E3,3.9712732E7,6.2186885E-1,1.8122449E2,1.752512E2,2.0601392E3,2.8207336E3,1.3510204E1,1E0,1.6140062E3,1.1032986E8,3.171E3,5.98E2,2.3977574E7,6.7652373E3,2.9816154E1,2.4347392E-1,3.386E3,9.1569895E-1,3.1575095E-2,5.212854E8,3.5685265E6,2.8729197E8,4.05E2,7E0,2.368E0,8.417E3,2.8035452E11,9.35E2,4.571429E0,7.576E3,-3.3420238E-3,2.7271206E5,2.0189162E7,1.3E1,4.602015E0,2.1E1,7.064848E6,2E0,1.3715873E3,8E0,1E0,5.847845E3,1.6382656E-1,1.6259277E7,3.906E3,1.0503613E5,4.4E1,4.167263E-2,1.3062E4,2.1515152E0,7E0,8.91E2,7.9527026E-1,7.6669473E-3,9.705292E6,2.4906608E5,8.182648E6,1.6621418E0,3.9794872E0,-1.3773647E-2,6.223159E-3,1.9358906E6,6.699017E6,5.4878284E2,5.327621E-2,1.2342778E1,4.1E1,2.208825E-2,2.7271206E5,8.755996E-1,1.5146034E-2,4.889199E-3,8.138211E-3,5.884028E0,2.028994E8,3.07125E0,5.222222E1,7.7344055E4,3.4250062E-2,-9.473923E-4,3.5734247E2,8.8365E4,7.10309E2,1.6006084E3,1.06E3,1E0,2E0,9.015703E-7,-2.7001845E-2,-3.5000786E-3,8.009486E-3,-5.987215E-3,8.7757526E2,2.0923112E-2,2.8408049E-2,7.5318483E-3,5.3473564E1,1.0715278E7,2.4435181E12,4.30389E-3,-2.9383853E-2,-7.0407623E-3,1.7911884E3,5.094239E3,5.3211357E-3,1.3235373E1,-1.7243562E-2,-3.036037E-2,3.0039465E-3,-1.2285865E-2,-6.551776E-3,-3.3698786E-2,-3.5198424E-2,-5.776744E-2,-1.761137E-3,-1.7797781E-2,-1.5294387E-2,-2.0260303E-4,-2.4285527E-2,-7.7273E-3,-1.03474455E-2,-2.9762484E-2,4.0359926E-4,-7.948868E-3,5.157741E-3,-5.63419E-3,2.924477E-3,-9.4578015E-3,-7.666724E-3,-2.6441582E-2,-1.085656E-2,-2.1946235E-3,1.380741E-2,-1.1516257E-2,-1.9913303E-2,3.4777306E-6,-3.5714786E-2,-1.341211E-2,-6.0083307E-3,1.1420933E-2,-2.5039101E-2,-7.837343E-3,1.8195275E-4,-8.011385E-3,-2.954847E-3,1.3517644E-2,2.9255943E-2,1.2597094E-2,-1.8326944E-2,-1.5123065E-3,-2.9279768E-2,-1.6758831E-2,-5.0736316E-3,1.2241925E-2,-2.6952317E-2,-6.4601973E-3,7.910658E-3,2.0348242E-2,-6.9334256E-3,5.0431653E-3,6.0268547E-3,1.4960297E-2,1.5487161E-2,3.501578E-2,3.335104E-2,1.4940443E-2,5.4204254E-3,2.2828024E-2,3.1745672E-2,5.425941E-2,1.1852521E-2,1.9575668E-2,3.5509393E-2,2.3659457E-2,4.8983056E-2,1.4023721E-2,1.6855178E-2,1.4850127E-3,-8.805697E-3,1.5617014E-3,3.1090884E-2,1.4398778E-2],"split_indices":[20,56,2,102,42,28,45,52,38,45,4,29,27,17,52,45,27,52,52,52,55,58,16,52,7,2,10,47,52,56,38,2,27,0,32,45,5,10,54,58,12,31,0,54,9,0,28,45,0,53,3,28,6,52,8,6,4,35,9,0,33,3,0,9,53,3,0,27,0,9,47,45,58,54,0,0,45,45,52,0,54,3,38,28,42,0,0,0,53,7,57,4,33,0,0,52,1,52,55,2,102,17,37,0,0,0,0,52,0,0,0,56,45,31,0,0,0,52,4,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.675E3,8.48E2,1.827E3,4.56E2,3.92E2,1.491E3,3.36E2,3.13E2,1.43E2,3.05E2,8.7E1,9.76E2,5.15E2,2.69E2,6.7E1,2.86E2,2.7E1,1.02E2,4.1E1,2.23E2,8.2E1,3.4E1,5.3E1,1.31E2,8.45E2,4.97E2,1.8E1,5.3E1,2.16E2,4E1,2.7E1,2.15E2,7.1E1,4E0,2.3E1,6.6E1,3.6E1,2.5E1,1.6E1,2E2,2.3E1,4.5E1,3.7E1,1.3E1,2.1E1,5E0,4.8E1,1.27E2,4E0,5.1E2,3.35E2,3.79E2,1.18E2,1.3E1,5E0,4.1E1,1.2E1,1.84E2,3.2E1,5E0,3.5E1,2.1E1,6E0,1.99E2,1.6E1,3.9E1,3.2E1,2.1E1,2E0,2.3E1,4.3E1,1.3E1,2.3E1,1.3E1,1.2E1,1E1,6E0,3.4E1,1.66E2,4E0,1.9E1,1.4E1,3.1E1,1.1E1,2.6E1,8E0,5E0,4E0,1.7E1,1.3E1,3.5E1,8.7E1,4E1,2E0,2E0,3.29E2,1.81E2,1.51E2,1.84E2,3.58E2,2.1E1,1.01E2,1.7E1,1.1E1,2E0,2E0,3E0,3.7E1,4E0,9E0,3E0,1.13E2,7.1E1,3E1,2E0,3E0,2E0,1.5E1,2E1,8E0,1.3E1,3.4E1,1.65E2,7E0,9E0,2E0,3.7E1,1.2E1,2E1,3E0,1.8E1,1.8E1,5E0,4E1,3E0,1E1,3E0,8E0,1.5E1,2E0,1.1E1,2E0,4E0,8E0,2.6E1,1E2,6.6E1,1.7E1,2E0,1.2E1,2E0,2.7E1,4E0,8E0,3E0,2.1E1,5E0,1.2E1,5E0,2E0,1.1E1,2.8E1,7E0,5.8E1,2.9E1,1.9E1,2.1E1,2.55E2,7.4E1,3.2E1,1.49E2,1.22E2,2.9E1,6.5E1,1.19E2,1.76E2,1.82E2,7E0,1.4E1,8.3E1,1.8E1,1.1E1,6E0,6E0,3.1E1,4.3E1,7E1,2.9E1,4.2E1,2.8E1,2E0,8E0,7E0,1.2E1,8E0,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-4.247643E-3,-3.058124E-1,1.3960832E-1,-4.83575E-1,-1.4032428E-1,-5.5578197E-3,3.4042555E-1,-6.204453E-1,-3.3837742E-1,-2.2793765E-1,2.0594054E-1,-1.677511E-1,1.02495E-1,1.7850184E-1,4.6826115E-1,-5.861073E-1,-6.06181E-2,-2.3077953E-1,-4.7181672E-1,-2.5280342E-1,6.175792E-2,1.5057157E-1,5.6251943E-1,-1.9253857E-1,3.4382218E-1,-2.9788864E-1,1.3338059E-1,4.558468E-2,3.4065822E-1,2.992651E-1,6.070901E-1,-6.245098E-1,-1.2242591E-1,-2.6355356E-1,-9.469802E-3,-5.521135E-1,-3.275672E-1,-4.4889247E-1,-1.5776566E-1,2.2195434E-2,2.6350406E-1,3.672236E-2,1.4705444E-2,-1.6261622E-1,-5.0506926E-1,5.021334E-2,1.02120735E-1,-3.4369522E-1,2.2663586E-2,6.6342577E-3,2.4003159E-1,1.13434315E-1,-1.9297439E-1,2.513278E-1,5.1439106E-1,3.9056975E-1,8.21803E-2,5.7588106E-1,5.9795406E-2,-5.613824E-1,-8.3359253E-1,2.3499297E-1,-2.909332E-1,6.4825253E-3,-2.7714646E-1,8.840951E-3,-2.4127911E-3,-7.927887E-1,-4.553172E-1,-1.7926198E-1,-4.1790104E-1,-2.7021593E-1,-6.0556453E-1,-1.848087E-1,3.3282402E-1,1.8410621E-2,-3.0186702E-2,1.0764347E-1,3.6127278E-1,-3.093565E-1,-9.739997E-2,-3.3089048E-1,-9.074272E-1,1.1576631E-2,-2.7865483E-4,-2.3088285E-1,-2.3649285E-2,-4.8656188E-2,1.8346162E-1,1.8445863E-1,4.304313E-1,2.582306E-1,2.6582377E-2,-1.8688168E-2,-2.877568E-2,3.676333E-1,7.365476E-2,5.594331E-1,-7.409193E-3,4.4755977E-1,8.863831E-2,-6.2615713E-3,1.5133004E-1,7.2443664E-1,3.992038E-1,-2.9152242E-2,-1.3196349E-2,-4.2941295E-2,-1.4643722E-2,1.6191078E-2,9.197765E-4,-1.6214507E-2,-1.6834256E-3,-1.5316843E-2,-7.880858E-3,-4.2317837E-2,-1.5423025E-2,-2.4033567E-2,-1.0717409E-2,-3.558471E-4,-1.4031484E-2,-7.3647606E-3,-2.255646E-2,-1.8288078E-2,-4.4734506E-3,-2.1846795E-2,-3.842872E-2,-1.2852492E-2,-3.7025856E-3,-9.845759E-6,2.5908455E-2,6.641948E-3,-4.946377E-3,7.5539984E-3,-2.5928232E-3,2.1970824E-3,1.9176168E-2,-1.900498E-2,-1.1592528E-2,2.6534004E-2,-6.421102E-3,-2.4568362E-2,-7.812615E-3,-5.0317384E-2,-1.0677549E-2,-1.8034786E-2,-2.1553861E-3,-5.7003503E-3,4.5416444E-3,1.0764772E-2,-4.721906E-3,7.6967296E-3,3.546809E-2,1.0244192E-2,2.9621884E-2,2.1285497E-2,9.382852E-3,-5.95372E-3,5.1617357E-3,-5.894312E-3,8.391165E-3,1.1162952E-2,2.410681E-2,6.320504E-3,-1.8405957E-2,2.9070454E-2,1.3788845E-3,1.0779137E-2,2.4901416E-2,-9.139686E-3,7.543788E-3,4.830767E-3,1.5221984E-2,4.1526545E-2,2.2230864E-2,2.174772E-2,4.226578E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,-1,79,81,-1,83,85,-1,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,-1,113,-1,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,143,-1,-1,145,-1,147,149,151,153,155,157,-1,159,161,163,165,-1,167,169,-1,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.17130394E2,2.558429E1,5.3291496E1,8.13681E0,1.3789798E1,1.8629211E1,1.5793022E1,4.0223007E0,2.8488197E0,1.3985649E1,1.731694E0,5.4554577E0,7.9292946E0,7.3126154E0,9.859856E0,3.59169E0,0E0,8.3887625E-1,8.731766E-1,6.582184E0,0E0,1.1684762E0,3.907535E-1,3.7146397E0,3.29937E0,1.7874904E0,8.0143585E0,3.0648854E0,2.245222E0,3.8329697E0,3.9657516E0,2.0398178E0,1.092238E0,5.851793E-1,1.3634089E-1,9.669266E-1,3.939805E-1,3.0836277E0,3.272132E0,7.516176E-1,6.1635804E-1,0E0,0E0,3.5298529E0,2.1526184E0,0E0,2.4251765E-1,6.3116455E-1,0E0,2.6682334E0,3.3277092E0,1.8417839E0,1.3490914E0,2.1013503E0,1.6235743E0,2.3177013E0,8.884964E-1,5.6162415E0,0E0,1.848587E0,1.1238155E0,1.2698254E-1,1.3432741E-1,0E0,4.1832018E-1,0E0,0E0,3.5691643E-1,3.5089684E-1,2.909558E-1,2.220819E-1,1.0698562E0,1.4278355E0,2.0452685E0,9.0135574E-1,0E0,4.4638655E-1,1.5441114E-1,3.0262113E-1,5.750809E-1,6.2629795E0,7.1065474E-1,7.4623203E-1,0E0,0E0,7.1427345E-1,0E0,2.111622E0,7.7966475E-1,3.1954193E0,2.80013E0,5.5111885E-1,1.1529428E0,0E0,4.8325378E-1,1.0149403E0,1.1726044E0,1.1945648E0,0E0,1.7354832E0,4.7285166E-1,0E0,3.460467E-1,4.090164E0,1.7526894E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,87,87,88,88,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,99,99,100,100,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,-1,80,82,-1,84,86,-1,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,-1,114,-1,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,144,-1,-1,146,-1,148,150,152,154,156,158,-1,160,162,164,166,-1,168,170,-1,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,1.5938586E8,2.308943E0,1.4930757E0,1.8108038E5,2.1446484E-1,1.1855755E8,7.5891815E6,1.22159E5,7.75541E-1,1.5106794E3,1.2E1,9.099893E8,1.0855529E3,5.35E3,-6.06181E-2,2.728889E2,7.2764084E-2,1E0,6.175792E-2,2.0783027E5,2.093563E0,1E0,6.7388856E7,1.0409E4,8.15E2,4.0844156E7,3.166E3,5.236559E0,1.9013363E4,2.589369E7,1.5062694E9,2.08E2,1.04E2,1.2E1,1.9521575E5,1.7232166E8,1.3554651E2,6.862E3,3.0990322E5,3.672236E-2,1.4705444E-2,2.73E2,7.492813E0,5.021334E-2,6.1428223E1,5.097268E7,2.2663586E-2,2.3314082E6,2.2214102E6,6.5369595E3,1.46705E3,1.9761398E7,5.2686436E7,2E0,2.7165637E4,1.778482E7,5.9795406E-2,5.551643E2,9.173084E2,2.1358025E0,1.8489688E7,6.4825253E-3,9.490825E5,8.840951E-3,-2.4127911E-3,1.108E3,1E0,1.0830879E6,2.808311E0,1.221875E1,8.387235E11,3.7609413E-1,9.536863E4,1.8410621E-2,7.11E2,9.47817E7,3.333E3,4.1E1,8E0,1.5043668E0,1.752E3,1.1576631E-2,-2.7865483E-4,2.291E3,-2.3649285E-2,2.3446269E2,7.89E2,1.4685706E3,5.3038636E7,6E0,1.6342836E3,-1.8688168E-2,4.798624E3,3.0795444E3,3.0769388E7,3.00624E5,-7.409193E-3,2.0131812E7,1.2974394E6,-6.2615713E-3,8.985661E8,3.7E1,3.5301748E7,-2.9152242E-2,-1.3196349E-2,-4.2941295E-2,-1.4643722E-2,1.6191078E-2,9.197765E-4,-1.6214507E-2,-1.6834256E-3,-1.5316843E-2,-7.880858E-3,-4.2317837E-2,-1.5423025E-2,-2.4033567E-2,-1.0717409E-2,-3.558471E-4,-1.4031484E-2,-7.3647606E-3,-2.255646E-2,-1.8288078E-2,-4.4734506E-3,-2.1846795E-2,-3.842872E-2,-1.2852492E-2,-3.7025856E-3,-9.845759E-6,2.5908455E-2,6.641948E-3,-4.946377E-3,7.5539984E-3,-2.5928232E-3,2.1970824E-3,1.9176168E-2,-1.900498E-2,-1.1592528E-2,2.6534004E-2,-6.421102E-3,-2.4568362E-2,-7.812615E-3,-5.0317384E-2,-1.0677549E-2,-1.8034786E-2,-2.1553861E-3,-5.7003503E-3,4.5416444E-3,1.0764772E-2,-4.721906E-3,7.6967296E-3,3.546809E-2,1.0244192E-2,2.9621884E-2,2.1285497E-2,9.382852E-3,-5.95372E-3,5.1617357E-3,-5.894312E-3,8.391165E-3,1.1162952E-2,2.410681E-2,6.320504E-3,-1.8405957E-2,2.9070454E-2,1.3788845E-3,1.0779137E-2,2.4901416E-2,-9.139686E-3,7.543788E-3,4.830767E-3,1.5221984E-2,4.1526545E-2,2.2230864E-2,2.174772E-2,4.226578E-3],"split_indices":[20,56,7,54,42,28,38,45,45,2,34,52,18,7,52,2,0,52,39,16,0,33,39,6,7,2,2,45,0,54,52,45,12,2,10,18,33,7,56,9,28,0,0,2,54,0,56,45,0,28,28,48,52,45,43,6,33,51,0,4,52,56,32,0,33,0,0,0,105,32,58,58,31,42,28,0,0,32,2,2,18,53,29,0,0,2,0,52,2,55,7,3,55,0,4,4,1,10,0,12,28,0,5,8,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.698E3,8.71E2,1.827E3,4.19E2,4.52E2,1.061E3,7.66E2,2.14E2,2.05E2,3.61E2,9.1E1,4.24E2,6.37E2,3.39E2,4.27E2,2.05E2,9E0,1.15E2,9E1,3.56E2,5E0,8E1,1.1E1,4.05E2,1.9E1,4.5E1,5.92E2,1.87E2,1.52E2,1.94E2,2.33E2,1.89E2,1.6E1,1E2,1.5E1,5.6E1,3.4E1,1.15E2,2.41E2,3.8E1,4.2E1,5E0,6E0,3.71E2,3.4E1,4E0,1.5E1,4.3E1,2E0,2.71E2,3.21E2,1.46E2,4.1E1,1.02E2,5E1,1.36E2,5.8E1,2.24E2,9E0,1.48E2,4.1E1,5E0,1.1E1,3E0,9.7E1,2E0,1.3E1,1.4E1,4.2E1,1.4E1,2E1,5.5E1,6E1,2.29E2,1.2E1,4E0,3.4E1,1.7E1,2.5E1,1.13E2,2.58E2,2.5E1,9E0,6E0,9E0,2.6E1,1.7E1,2.07E2,6.4E1,2.5E2,7.1E1,5.4E1,9.2E1,1.8E1,2.3E1,6.1E1,4.1E1,4.7E1,3E0,1.14E2,2.2E1,1.4E1,4.4E1,1.2E2,1.04E2,1.26E2,2.2E1,3.6E1,5E0,3E0,2E0,9E0,2E0,6.9E1,2.8E1,1.1E1,3E0,3.4E1,8E0,6E0,8E0,4E0,1.6E1,3.3E1,2.2E1,3.6E1,2.4E1,1.28E2,1.01E2,5E0,7E0,1E1,2.4E1,1.3E1,4E0,3E0,2.2E1,4.7E1,6.6E1,1.3E1,2.45E2,1.1E1,1.4E1,7E0,2E0,1.4E1,1.2E1,1.39E2,6.8E1,5.6E1,8E0,2.41E2,9E0,3.4E1,3.7E1,1.2E1,4.2E1,3.2E1,6E1,1.6E1,7E0,3.2E1,2.9E1,3.7E1,4E0,4.3E1,4E0,2.9E1,8.5E1,4E0,1.8E1,3.5E1,9E0,7.6E1,4.4E1,8.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-9.633271E-4,-2.8656363E-1,1.3522477E-1,-4.427301E-1,-9.8753646E-2,-9.263564E-4,3.3237875E-1,-4.8258913E-1,4.3627083E-1,-2.2861637E-1,1.549249E-1,-5.902825E-2,2.369929E-1,2.4496846E-1,5.4095954E-1,-1.20156206E-1,-5.4981375E-1,6.432092E-2,6.89497E-2,-2.5363678E-1,1.03413925E-1,7.143452E-2,4.684444E-1,-6.6681206E-2,6.077724E-2,1.6654457E-1,4.860674E-1,1.5139186E-1,4.7727683E-1,7.4349344E-1,4.461697E-1,-2.763765E-1,1.911675E-2,-6.0303396E-1,-2.7678502E-1,2.0186953E-2,-6.208101E-3,-2.0026304E-1,-3.9671773E-1,-5.725977E-2,1.853529E-2,-1.2567244E-1,1.330607E-1,8.251597E-3,5.106224E-1,-3.1394044E-1,-2.3808453E-2,1.17333665E-1,5.411061E-1,3.1367743E-1,5.8568674E-1,2.1691048E-1,-6.799056E-2,2.68717E-1,6.252142E-1,7.851519E-1,1.8216412E-1,4.061159E-1,9.70317E-1,8.623198E-3,-3.105307E-1,1.2650074E-1,-2.0538205E-1,-5.4167235E-1,-9.065054E-1,-1.9976299E-1,-5.150728E-1,-2.4798037E-1,-2.2611583E-2,-4.4193938E-1,-2.0041464E-1,-9.394587E-3,6.584681E-3,-1.3730908E-2,1.1539308E-2,9.198243E-2,2.5580418E-1,5.3598255E-1,7.396243E-3,-3.4010336E-1,1.8794918E-2,-2.0184338E-1,1.48444995E-2,-1.544517E-1,1.5615357E-1,1.487895E-2,3.2586623E-2,3.40232E-1,2.8084107E-3,1.5138577E-2,3.0881722E-2,1.0328209E-1,3.3381253E-1,-2.923784E-1,2.2131456E-2,3.238783E-1,-1.5789692E-1,5.5347953E-2,5.7164955E-1,2.3442192E-2,8.753861E-1,1.6115185E-2,-4.0390394E-3,2.354291E-1,4.9632144E-1,3.0084045E-3,5.69581E-2,-1.7265059E-2,-5.532618E-3,1.7177178E-2,2.7503646E-3,-1.6294865E-2,4.154639E-4,-2.8388292E-2,-1.4751882E-2,-6.758118E-2,-3.8151737E-2,-2.5196272E-4,-1.2187545E-2,-7.535172E-3,-3.247957E-2,-1.07381465E-2,-2.3212058E-2,4.5194463E-3,-3.4452036E-3,-1.752935E-2,-3.468031E-2,-1.2019997E-2,-5.378835E-4,6.4026155E-3,-2.156702E-3,-2.4276385E-4,8.6377645E-3,9.229632E-3,2.2027062E-2,1.5012921E-2,2.964023E-2,-1.5120021E-2,-3.1183576E-2,-8.489763E-3,-3.276444E-2,4.361947E-2,-3.8118946E-4,1.6319944E-3,-2.0015331E-2,3.3912018E-3,1.1270735E-2,1.8272504E-2,5.295288E-3,1.311894E-2,5.139978E-4,2.2507358E-2,7.915439E-3,-3.4143146E-2,-1.0766284E-2,2.1613581E-2,-1.0846308E-3,8.454472E-3,2.6352236E-2,7.130458E-3,-2.4036609E-2,3.2023266E-3,2.8590823E-2,5.1839583E-2,3.3979844E-2,1.4570054E-2,-2.032371E-3,2.4907017E-2,-7.4918596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,-1,-1,67,69,71,-1,73,75,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,-1,107,109,111,113,115,117,119,121,123,125,127,-1,-1,-1,129,131,133,135,-1,137,-1,139,141,143,145,-1,-1,147,-1,-1,-1,149,151,153,155,157,159,-1,161,-1,163,-1,-1,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0420053E2,2.5326408E1,4.869303E1,1.6747597E1,1.3015112E1,1.4860492E1,1.3377472E1,1.0946648E1,6.902304E0,2.1944962E0,3.4741032E0,8.866329E0,3.621439E0,1.1318783E1,3.8534698E0,1.5718062E0,5.3692627E0,0E0,1.1748701E0,1.7762861E0,8.879984E-1,1.3164456E0,2.617588E-1,9.109114E0,0E0,2.9963074E0,6.0625935E-1,5.4048605E0,4.460064E0,1.447094E0,2.7760487E0,5.859487E-1,9.661764E-1,5.430603E0,1.0717263E0,0E0,0E0,1.511786E0,4.8844433E-1,3.7866026E-1,0E0,5.7782674E-1,3.8992167E-1,0E0,1.3400984E-1,2.4806118E0,5.0540733E0,1.5856192E0,3.9859486E-1,1.0872269E-1,2.4265957E-1,3.8044243E0,1.7658097E0,1.5572243E0,1.8115501E0,1.2522392E0,2.751667E-1,2.0695095E0,1.9129114E0,0E0,2.7023506E-1,4.123853E-1,3.7826127E-1,3.0052414E0,1.7587776E0,5.3086233E-1,7.686484E-1,6.842575E-1,2.318358E-1,8.4894085E-1,1.3113046E-1,0E0,0E0,0E0,1.0968002E-1,5.413058E-1,1.6890585E-1,2.1193743E-1,0E0,7.272587E-1,0E0,1.4310508E0,1.2348071E1,9.580694E-1,8.6190724E-1,0E0,0E0,1.1911678E-1,0E0,0E0,0E0,2.3345575E0,3.1808739E0,5.501399E-1,1.2209888E0,1.8041077E0,9.190344E-1,0E0,9.9673843E-1,0E0,8.46138E-1,0E0,0E0,9.879968E-1,1.5082893E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,74,74,75,75,76,76,77,77,79,79,81,81,82,82,83,83,84,84,87,87,91,91,92,92,93,93,94,94,95,95,96,96,98,98,100,100,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,-1,-1,68,70,72,-1,74,76,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,-1,108,110,112,114,116,118,120,122,124,126,128,-1,-1,-1,130,132,134,136,-1,138,-1,140,142,144,146,-1,-1,148,-1,-1,-1,150,152,154,156,158,160,-1,162,-1,164,-1,-1,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,7.7344055E4,6.7652373E3,2.3544883E8,1.2526851E6,1.3137E4,8.2608955E-3,2.46E2,9.026015E2,1.4101E4,8.791777E3,3.084419E6,1.8137958E6,4.68646E5,2.0605305E5,2.3023027E3,6.432092E-2,1.0040462E8,1.4274633E7,1.125584E-3,2.1592189E5,2.1924436E5,1.0376E4,6.077724E-2,7.780377E6,3.9892856E2,1.4274633E7,3.564E3,4.5481584E7,1.9013363E4,2.08E2,1.3983857E10,7.8351054E11,1.2427474E8,2.0186953E-2,-6.208101E-3,2.835051E2,7.07089E5,1E0,1.853529E-2,1.4887473E-1,8.715555E5,8.251597E-3,3.430961E7,3.78E2,4.54E2,1.1317E4,1E0,3.684685E2,3E0,6.308896E2,3.731E3,3.1984328E5,1.0918E4,3.0720797E-1,3.2E2,1.59769E5,5.673198E5,8.623198E-3,2.262586E6,6.4860556E5,2.5269838E6,3.83E3,3E0,9.07E2,3.046556E5,1E0,1.0001285E-1,1.3046257E11,2.184578E1,-9.394587E-3,6.584681E-3,-1.3730908E-2,9.619E3,5.2768206E5,1.2783833E3,2.4347392E-1,7.396243E-3,1.1855755E8,1.8794918E-2,4.5184364E0,8E0,1.4615384E0,2.3314082E6,1.487895E-2,3.2586623E-2,2.9138382E7,2.8084107E-3,1.5138577E-2,3.0881722E-2,3.1753032E0,2.4E1,4.0208153E-10,2.22747E5,5.314225E2,2.779E3,5.5347953E-2,1.9577E4,2.3442192E-2,5.4379158E1,1.6115185E-2,-4.0390394E-3,2.6025698E7,7.218466E7,3.0084045E-3,5.69581E-2,-1.7265059E-2,-5.532618E-3,1.7177178E-2,2.7503646E-3,-1.6294865E-2,4.154639E-4,-2.8388292E-2,-1.4751882E-2,-6.758118E-2,-3.8151737E-2,-2.5196272E-4,-1.2187545E-2,-7.535172E-3,-3.247957E-2,-1.07381465E-2,-2.3212058E-2,4.5194463E-3,-3.4452036E-3,-1.752935E-2,-3.468031E-2,-1.2019997E-2,-5.378835E-4,6.4026155E-3,-2.156702E-3,-2.4276385E-4,8.6377645E-3,9.229632E-3,2.2027062E-2,1.5012921E-2,2.964023E-2,-1.5120021E-2,-3.1183576E-2,-8.489763E-3,-3.276444E-2,4.361947E-2,-3.8118946E-4,1.6319944E-3,-2.0015331E-2,3.3912018E-3,1.1270735E-2,1.8272504E-2,5.295288E-3,1.311894E-2,5.139978E-4,2.2507358E-2,7.915439E-3,-3.4143146E-2,-1.0766284E-2,2.1613581E-2,-1.0846308E-3,8.454472E-3,2.6352236E-2,7.130458E-3,-2.4036609E-2,3.2023266E-3,2.8590823E-2,5.1839583E-2,3.3979844E-2,1.4570054E-2,-2.032371E-3,2.4907017E-2,-7.4918596E-3],"split_indices":[20,102,33,52,12,28,2,27,10,52,2,52,9,28,29,33,55,0,51,45,38,28,28,9,0,28,52,45,2,47,52,2,5,31,45,0,0,52,29,74,0,38,33,0,47,8,2,9,102,52,8,52,29,33,9,38,0,11,32,0,1,33,32,2,3,0,32,77,39,31,56,0,0,0,10,28,4,38,0,45,0,53,18,53,28,0,0,9,0,0,0,54,3,37,1,4,2,0,9,0,58,0,0,43,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.677E3,8.64E2,1.813E3,4.71E2,3.93E2,1.073E3,7.4E2,4.51E2,2E1,2.6E2,1.33E2,8.63E2,2.1E2,5.23E2,2.17E2,7.1E1,3.8E2,5E0,1.5E1,2.42E2,1.8E1,1.06E2,2.7E1,8.59E2,4E0,1.65E2,4.5E1,3.74E2,1.49E2,6.7E1,1.5E2,3.3E1,3.8E1,3.17E2,6.3E1,5E0,1E1,1.78E2,6.4E1,1.2E1,6E0,2.5E1,8.1E1,4E0,2.3E1,1.26E2,7.33E2,1.47E2,1.8E1,1.8E1,2.7E1,2.88E2,8.6E1,6.3E1,8.6E1,6.2E1,5E0,1.41E2,9E0,2E0,3.1E1,2.6E1,1.2E1,2.66E2,5.1E1,4.9E1,1.4E1,1.4E2,3.8E1,5.1E1,1.3E1,7E0,5E0,1.1E1,1.4E1,6.2E1,1.9E1,2.1E1,2E0,1.22E2,4E0,1.3E2,6.03E2,1.8E1,1.29E2,8E0,1E1,1.6E1,2E0,6E0,2.1E1,1.47E2,1.41E2,2.4E1,6.2E1,5.6E1,7E0,6E0,8E1,1.6E1,4.6E1,3E0,2E0,5E1,9.1E1,2E0,7E0,2.4E1,7E0,5E0,2.1E1,7E0,5E0,2.17E2,4.9E1,7E0,4.4E1,1.1E1,3.8E1,5E0,9E0,1.29E2,1.1E1,1.1E1,2.7E1,4.2E1,9E0,1E1,3E0,4E0,1E1,3E1,3.2E1,1.6E1,3E0,7E0,1.4E1,1.15E2,7E0,1.25E2,5E0,1.4E1,5.89E2,1.1E1,7E0,6.3E1,6.6E1,1.3E1,3E0,5.1E1,9.6E1,7.7E1,6.4E1,2E0,2.2E1,5E0,5.7E1,3.5E1,2.1E1,4E0,3E0,4E0,7.6E1,1.8E1,2.8E1,4E1,1E1,8.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[5.3309975E-3,-2.6363635E-1,1.317033E-1,-4.380478E-1,-1.08409576E-1,4.523528E-3,3.433563E-1,-5.462082E-1,-2.3884617E-1,-1.2755385E-1,6.504693E-2,-1.253692E-1,1.17934704E-1,1.9513133E-1,4.6135095E-1,-4.4440767E-1,-7.6162606E-1,-1.3689595E-1,-3.938413E-1,-2.0843865E-1,2.2098157E-1,-2.4990152E-1,-4.0793244E-2,1.4988588E-1,-3.1057602E-1,-1.7015627E-1,2.7698156E-1,3.827645E-1,7.3746014E-1,-4.6342984E-1,9.463305E-2,-9.117233E-1,-5.065781E-1,-1.7044578E-1,2.6828043E-2,-2.7030838E-1,-4.6945745E-1,-3.8242882E-1,-1.2715292E-1,1.4256208E-1,5.8266306E-1,-2.9542986E-1,-8.3449095E-2,8.262254E-1,-7.132833E-2,4.6901345E-2,2.3588608E-1,-3.9428294E-1,1.7624289E-1,-2.2792435E-1,2.7503353E-1,-2.5256416E-1,3.021997E-1,2.3394586E-1,5.107189E-1,8.157198E-1,3.538744E-1,-5.328269E-1,-3.142525E-1,8.679864E-3,-3.8917093E-3,-1.0345267E0,-4.7318056E-1,-5.9796566E-1,-2.6944238E-1,-9.145912E-2,-2.284485E-1,-3.1287079E-3,1.1620289E-2,-9.142162E-2,-1.7793521E-2,-5.1376194E-1,-8.480281E-3,-2.2164418E-1,-5.264009E-1,-2.7201477E-1,-7.025486E-2,1.12606116E-1,2.4972554E-2,3.5317462E-2,1.4386224E-2,-2.559147E-1,-5.2515066E-1,-1.4868414E-1,3.3536207E-2,1.4605067E-2,4.349299E-2,-1.0024972E-1,2.4955039E-1,-2.0992028E-2,1.7910972E-1,3.8047147E-1,1.4718372E-1,-2.3860666E-1,-2.534403E-2,-5.0408836E-3,1.4841042E-2,-1.5203847E-1,-1.7755546E-2,4.222626E-3,1.8827507E-2,-5.9512886E-3,-2.8544102E-2,2.0289478E-1,3.9739963E-1,3.240895E-1,8.271278E-2,6.181312E-1,3.2923582E-1,1.3939724E-2,8.4352756E-1,4.219235E-1,-3.9100843E-3,-1.8477492E-2,-2.9917318E-2,-2.714549E-3,-2.1977397E-2,-4.172783E-2,-6.365318E-2,-3.1380527E-2,-1.5298745E-2,-3.191983E-2,-1.0434379E-2,-1.7331542E-2,-8.222017E-4,-6.486354E-3,6.179507E-3,-1.6990216E-2,-7.642719E-3,-1.03235645E-2,1.3059198E-3,-1.5117565E-2,-2.9561939E-2,-5.7718926E-3,-1.8641692E-2,-2.954147E-2,-6.9426806E-3,-7.270126E-3,-1.5811894E-2,-2.7213676E-4,-1.5559657E-2,4.4568034E-4,1.0326746E-2,6.082619E-3,-1.3079393E-2,-3.36598E-2,-1.53597165E-2,-9.2995865E-3,1.6788843E-3,5.4491423E-3,-6.988274E-3,-4.3347143E-3,-3.6591392E-2,1.606166E-2,-1.4672387E-2,-4.3428224E-3,7.358545E-3,9.380508E-3,-2.1672035E-3,1.276732E-2,2.5908256E-2,5.6628776E-3,2.6014706E-2,-1.34601435E-2,3.9031385E-3,-1.5421444E-2,-3.8702958E-3,2.9167082E-2,8.437186E-3,9.918803E-3,2.2161607E-2,2.3269286E-2,1.1517048E-2,6.0425694E-3,-5.801302E-3,2.3799367E-2,4.235028E-2,8.48224E-3,2.4310464E-2,2.5109528E-2,4.3384533E-2,4.7504413E-3,2.3989476E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,-1,131,-1,133,135,137,139,141,-1,-1,-1,143,145,147,149,-1,-1,151,153,155,157,159,161,163,-1,-1,-1,165,-1,-1,-1,-1,-1,167,169,171,173,175,177,-1,179,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.197774E1,2.3370945E1,4.955445E1,8.638649E0,1.3061037E1,1.6970497E1,1.1995392E1,5.4290314E0,2.2373447E0,1.2844875E1,0E0,5.645726E0,8.466776E0,9.255985E0,8.049927E0,1.8997765E0,2.7155418E0,4.9344182E-1,4.0554333E-1,5.170109E0,2.3561425E0,1.6198368E0,8.557244E0,5.06382E0,1.8217444E0,1.5285993E0,3.4415722E0,5.6121635E0,2.232956E0,1.6442451E0,1.2136955E-1,2.2570648E0,5.5403423E-1,3.1813312E-1,3.340782E-1,4.006728E-1,3.4945345E-1,2.5916996E0,2.066206E0,7.5847924E-1,4.3388367E-1,1.4087753E0,3.7130028E-1,1.9292259E-1,2.911178E0,2.3554907E0,3.9454212E0,6.508956E-1,2.95982E-1,4.9640274E-1,1.1981785E-1,4.5471108E-1,2.1977386E0,1.8954039E0,2.934948E0,7.6112366E-1,5.0863695E-1,1.3526306E0,2.1350212E0,0E0,0E0,7.171631E-1,1.5454745E-1,4.540491E-1,2.5035542E-1,3.375024E-1,3.0873203E-1,0E0,0E0,1.532256E-1,0E0,3.9983082E-1,0E0,9.191897E-1,2.0151749E0,4.4619656E-1,3.0072365E0,7.216896E-1,0E0,0E0,0E0,1.0026236E0,6.464009E-1,2.6597434E-1,2.758211E-1,0E0,0E0,1.7793725E0,1.3433751E0,2.1317024E0,3.4361696E-1,2.012741E0,2.170474E0,2.8111386E-1,0E0,0E0,0E0,3.9769322E-1,0E0,0E0,0E0,0E0,0E0,1.1948719E0,1.4213829E0,1.0734844E0,4.937066E-1,2.7860794E0,1.5925484E0,0E0,8.194084E-1,3.0724907E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,69,69,71,71,73,73,74,74,75,75,76,76,77,77,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,97,97,103,103,104,104,105,105,106,106,107,107,108,108,110,110,111,111],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,-1,132,-1,134,136,138,140,142,-1,-1,-1,144,146,148,150,-1,-1,152,154,156,158,160,162,164,-1,-1,-1,166,-1,-1,-1,-1,-1,168,170,172,174,176,178,-1,180,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.896243E4,3.1140776E0,2.14099E5,2.0407547E5,5.964E3,8.592097E-1,5.751617E6,1.6018981E0,6.504693E-2,9.99E2,4.898012E-1,1.5988182E-1,1.546644E6,3.730007E6,1.36054E-1,2.728889E2,2.2056E4,1E0,7.75541E-1,1.838444E6,8E0,9E2,7.160724E1,9.750871E6,5.26E2,2.3292E4,2.4268942E7,1.729767E2,1E0,1E0,9.9668306E-1,2.21875E0,2.695E3,1.6007329E7,3.386E3,1.7232166E8,3.3926086E2,1.6316416E6,2.093563E0,1.5E1,7.997723E6,2.127E3,1.4777102E9,1.5859042E6,4.020408E0,1.37518E5,6.306E3,4.8297736E7,1.8465776E8,4.888E0,1.2974394E6,1.504779E7,2.5003334E7,1E0,9.256843E3,2.0304577E-1,1.9051096E7,8.679864E-3,-3.8917093E-3,2.7682406E8,8E0,1.9557823E0,1.5851064E0,1.6559546E0,8.542683E1,-3.1287079E-3,1.1620289E-2,1.9840922E8,-1.7793521E-2,3.1809045E2,-8.480281E-3,6.2186885E-1,5.676415E-1,6.952967E6,7.218466E7,2.3033286E5,2.4972554E-2,3.5317462E-2,1.4386224E-2,7E0,6.3039363E4,1.5561539E2,6.576923E0,1.4605067E-2,4.349299E-2,5.2228385E6,1.30399E5,1.566593E6,1.662639E7,4.3689142E2,9.908038E1,3.86569E5,-2.534403E-2,-5.0408836E-3,1.4841042E-2,3.270662E6,-1.7755546E-2,4.222626E-3,1.8827507E-2,-5.9512886E-3,-2.8544102E-2,3.5E1,1.8114872E7,1.0219829E1,5.9767612E7,6.7652373E3,1E0,1.3939724E-2,8.7757526E2,7.883E3,-3.9100843E-3,-1.8477492E-2,-2.9917318E-2,-2.714549E-3,-2.1977397E-2,-4.172783E-2,-6.365318E-2,-3.1380527E-2,-1.5298745E-2,-3.191983E-2,-1.0434379E-2,-1.7331542E-2,-8.222017E-4,-6.486354E-3,6.179507E-3,-1.6990216E-2,-7.642719E-3,-1.03235645E-2,1.3059198E-3,-1.5117565E-2,-2.9561939E-2,-5.7718926E-3,-1.8641692E-2,-2.954147E-2,-6.9426806E-3,-7.270126E-3,-1.5811894E-2,-2.7213676E-4,-1.5559657E-2,4.4568034E-4,1.0326746E-2,6.082619E-3,-1.3079393E-2,-3.36598E-2,-1.53597165E-2,-9.2995865E-3,1.6788843E-3,5.4491423E-3,-6.988274E-3,-4.3347143E-3,-3.6591392E-2,1.606166E-2,-1.4672387E-2,-4.3428224E-3,7.358545E-3,9.380508E-3,-2.1672035E-3,1.276732E-2,2.5908256E-2,5.6628776E-3,2.6014706E-2,-1.34601435E-2,3.9031385E-3,-1.5421444E-2,-3.8702958E-3,2.9167082E-2,8.437186E-3,9.918803E-3,2.2161607E-2,2.3269286E-2,1.1517048E-2,6.0425694E-3,-5.801302E-3,2.3799367E-2,4.235028E-2,8.48224E-3,2.4310464E-2,2.5109528E-2,4.3384533E-2,4.7504413E-3,2.3989476E-2],"split_indices":[20,56,33,54,2,28,2,27,45,41,0,2,27,38,28,9,42,52,9,16,34,9,18,2,56,9,2,2,50,52,77,16,39,53,2,45,2,7,52,28,39,8,9,1,7,28,54,29,2,45,5,54,28,45,45,67,4,27,45,0,0,7,3,53,53,56,52,0,0,5,0,4,0,27,42,45,45,33,0,0,0,3,28,52,56,0,0,47,10,9,45,52,56,1,0,0,0,12,0,0,0,0,0,0,12,54,45,52,102,0,52,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.704E3,8.64E2,1.84E3,4.06E2,4.58E2,1.15E3,6.9E2,2.62E2,1.44E2,4.53E2,5E0,5.36E2,6.14E2,3.07E2,3.83E2,1.8E2,8.2E1,8.8E1,5.6E1,3.68E2,8.5E1,2.16E2,3.2E2,5.72E2,4.2E1,5.6E1,2.51E2,3E2,8.3E1,1.74E2,6E0,5E1,3.2E1,7.3E1,1.5E1,2.3E1,3.3E1,1.16E2,2.52E2,7.1E1,1.4E1,1.69E2,4.7E1,1E1,3.1E2,2.61E2,3.11E2,3.6E1,6E0,5E1,6E0,1.1E1,2.4E2,1.4E2,1.6E2,6.8E1,1.5E1,1.17E2,5.7E1,4E0,2E0,3.8E1,1.2E1,2.2E1,1E1,3.2E1,4.1E1,1.1E1,4E0,9E0,1.4E1,2.8E1,5E0,5.6E1,6E1,7E1,1.82E2,6.7E1,4E0,8E0,6E0,1.46E2,2.3E1,3E1,1.7E1,2E0,8E0,2.85E2,2.5E1,1.73E2,8.8E1,1.17E2,1.94E2,1.8E1,1.8E1,2E0,4E0,3.4E1,1.6E1,3E0,3E0,9E0,2E0,1.19E2,1.21E2,8.7E1,5.3E1,9.9E1,6.1E1,4E0,6.4E1,1.3E1,2E0,4.7E1,7E1,2.1E1,3.6E1,2.7E1,1.1E1,4E0,8E0,1.8E1,4E0,7E0,3E0,2.7E1,5E0,1.3E1,2.8E1,4E0,5E0,1.1E1,1.7E1,3.6E1,2E1,4.8E1,1.2E1,2.4E1,4.6E1,1.46E2,3.6E1,3.4E1,3.3E1,6E0,1.4E2,1.1E1,1.2E1,2.4E1,6E0,1.2E1,5E0,2.82E2,3E0,2.2E1,3E0,1.24E2,4.9E1,8.2E1,6E0,7E1,4.7E1,1.82E2,1.2E1,1.6E1,2E0,9E0,2.5E1,6E0,1.13E2,3.2E1,8.9E1,2.8E1,5.9E1,4.4E1,9E0,7E1,2.9E1,3.4E1,2.7E1,1.2E1,5.2E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[-1.8346848E-3,-2.7554688E-1,1.2622665E-1,-4.229637E-1,-8.861101E-2,-7.9248287E-4,3.016821E-1,-4.3935767E-1,5.853826E-2,-2.0995888E-1,1.5193294E-1,-1.5124889E-1,9.422172E-2,1.504549E-1,4.1390675E-1,-5.0994503E-1,-2.026519E-1,-2.3025903E-1,1.2748733E-1,5.138487E-2,3.427205E-1,-1.7443928E-1,3.6734274E-1,1.21811755E-1,-3.212541E-1,6.1054513E-2,3.645707E-1,-7.6224945E-2,4.5234686E-1,-4.4532838E-1,-8.2572794E-1,-1.1482691E-1,-5.16233E-1,-4.5554867E-1,-2.0873152E-1,-2.931076E-2,1.7603105E-2,9.207189E-2,-1.61533E-1,2.546432E-2,2.5522974E-1,-3.2903945E-1,-1.18968785E-1,4.517485E-2,1.3540716E-1,1.994482E-2,2.1115692E-1,-3.5882297E-1,1.3132824E-2,1.1649986E-1,-2.4713102E-1,3.9903194E-1,-2.2459665E-1,1.43685E-2,-3.5277106E-2,3.1117746E-1,5.614743E-1,-5.0500077E-1,-1.7579465E-1,-8.5740876E-1,3.475467E-3,-1.5527041E-1,3.0901697E-1,-8.649813E-2,-6.5467256E-1,-1.5137451E-2,-2.9942818E-2,-2.832092E-1,-1.4878742E-1,1.0848282E-2,-5.822741E-3,1.6009694E-1,2.0375298E-3,-1.0867766E-2,7.3701893E-3,1.2997694E-1,1.599966E-2,-2.8618035E-1,-3.186528E-2,2.1560621E-1,-1.4826816E-1,-3.7199253E-4,1.1673757E-2,-1.6012955E-1,6.5071955E-2,1.9856204E-1,5.6441534E-2,-4.2865124E-1,-3.756253E-3,2.2968031E-1,2.2838583E-2,-3.1518298E-1,6.975698E-3,3.3517888E-1,5.869653E-1,9.261569E-3,-2.3080258E-2,1.3832317E-1,-1.0273704E-1,3.7621644E-1,1.2312745E-1,6.74767E-1,3.6750162E-1,-2.2046011E-2,-3.2298822E-2,-1.2424464E-2,3.624977E-3,-5.4098353E-2,-3.495284E-2,4.2476607E-3,-9.94074E-3,3.0624181E-2,4.4198935E-3,3.9982665E-3,-8.198379E-3,-8.504614E-3,-3.9486114E-2,-1.058987E-2,-2.266847E-2,-4.2960816E-3,-1.2355452E-2,1.4220296E-2,4.465118E-3,-5.197592E-3,3.53456E-3,1.2550784E-2,2.439139E-3,-1.50182415E-2,4.757736E-3,-1.0363526E-2,3.57183E-2,-1.1076832E-2,-2.8597994E-3,-1.1948968E-3,-9.899202E-3,3.748914E-3,-3.3719998E-2,8.0983555E-3,2.6059516E-2,-8.217187E-3,-2.2047872E-2,1.4118726E-2,3.5884478E-3,-1.080528E-2,3.2340689E-3,-1.7587384E-2,-8.100029E-3,3.6031646E-3,1.8522771E-2,3.117383E-2,1.1458572E-2,-1.5627183E-3,8.80137E-3,-9.345598E-3,-1.3302928E-3,2.0819508E-2,6.664944E-3,7.6534064E-3,-1.0742229E-2,3.8508825E-2,2.7432324E-2,1.9476097E-2,-3.3626049E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,-1,81,83,85,87,-1,89,91,93,95,97,-1,99,101,103,105,107,-1,109,111,113,115,-1,-1,117,119,-1,-1,121,123,-1,-1,125,-1,127,-1,129,131,-1,-1,133,135,137,-1,139,-1,141,143,145,-1,147,149,-1,-1,151,153,155,157,159,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.4464714E1,2.3634415E1,4.0917706E1,1.3484062E1,1.1123909E1,1.5253248E1,1.302269E1,7.849884E0,0E0,1.7624578E0,2.4378545E0,5.030961E0,7.535932E0,6.294613E0,8.362076E0,7.100441E0,2.9976916E0,1.064044E0,5.754404E-1,7.5122714E-1,5.918579E-1,3.3568516E0,2.2767751E0,5.583997E0,1.0156302E0,4.027255E0,2.0590544E0,1.9848891E0,6.1415176E0,4.8328476E0,1.798214E0,1.5564537E0,1.3711872E0,2.756753E-1,9.4005203E-1,2.6360813E-1,0E0,4.4064438E-1,3.2371464E-1,0E0,2.7264762E-1,1.3087568E0,2.9041991E0,0E0,2.1907917E-1,2.3507771E0,3.750229E0,7.299628E-1,0E0,2.1042788E0,1.0081942E0,9.072733E-1,7.385937E-1,4.4968432E-1,0E0,2.1674118E0,4.8064346E0,1.581173E0,1.2185994E0,1.2242622E0,0E0,1.0519558E0,5.139098E-1,1.15067065E-1,1.3120537E0,0E0,0E0,1.0402985E0,7.769079E-1,0E0,0E0,3.4742498E-1,2.6193646E-1,0E0,0E0,1.328337E-1,0E0,1.024653E0,0E0,5.658863E0,1.9712939E0,0E0,0E0,3.600098E-1,2.4020832E0,3.206685E0,0E0,1.7386103E-1,0E0,8.850384E-1,1.236707E0,1.7357731E-1,0E0,9.2311525E-1,3.3361435E-1,0E0,0E0,1.2256503E-1,1.0540861E-1,1.7981682E0,6.4760464E-1,1.4381943E0,1.5204964E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,68,68,71,71,72,72,75,75,77,77,79,79,80,80,83,83,84,84,85,85,87,87,89,89,90,90,91,91,93,93,94,94,97,97,98,98,99,99,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,-1,82,84,86,88,-1,90,92,94,96,98,-1,100,102,104,106,108,-1,110,112,114,116,-1,-1,118,120,-1,-1,122,124,-1,-1,126,-1,128,-1,130,132,-1,-1,134,136,138,-1,140,-1,142,144,146,-1,148,150,-1,-1,152,154,156,158,160,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5938586E8,2.14099E5,2.3544883E8,1.8108038E5,2.1446484E-1,2.0163035E3,5.853826E-2,9.026015E2,1.2060912E3,1.46705E3,3.956147E-1,1.4777102E9,2.943086E4,6.971004E7,1.2427474E8,2.750917E-4,1.125584E-3,1.1160287E7,1.4075E4,1.141E4,6.7388856E7,8.46E2,3.1998687E3,5.9767612E7,1.921032E4,2.1506184E7,5.964E3,3.904E3,1.1879E4,1.3554651E2,3.046556E5,9.374544E5,1.8122449E2,1.4E2,1.7603105E-2,2.3686748E0,1E0,2.546432E-2,3.3480989E3,1E0,7.330957E4,4.517485E-2,5.695621E8,3.97E2,1.3664E4,1.9505986E6,1.3132824E-2,9.74026E0,4.6E1,1.08874E5,5.1678925E1,2.6666667E0,-3.5277106E-2,4.020408E0,8.5E1,1E0,3.002239E5,1.8736842E0,3.475467E-3,2.467284E7,6.135771E9,1.141E4,1.5119754E0,-1.5137451E-2,-2.9942818E-2,2.01351E5,1.6256282E7,1.0848282E-2,-5.822741E-3,3.8434E4,6.019605E2,-1.0867766E-2,7.3701893E-3,3E0,1.599966E-2,1.91E2,-3.186528E-2,3.7761906E1,7.364257E4,-3.7199253E-4,1.1673757E-2,1.4396E4,5.8E1,5.5669255E6,5.6441534E-2,1.1880085E2,-3.756253E-3,2.5003334E7,1.4813794E3,1.5694646E3,6.975698E-3,1E0,2.7809634E5,9.261569E-3,-2.3080258E-2,1.4763578E3,1.4126303E0,1E0,2.1381798E-6,1.8903887E0,3.1570474E2,-2.2046011E-2,-3.2298822E-2,-1.2424464E-2,3.624977E-3,-5.4098353E-2,-3.495284E-2,4.2476607E-3,-9.94074E-3,3.0624181E-2,4.4198935E-3,3.9982665E-3,-8.198379E-3,-8.504614E-3,-3.9486114E-2,-1.058987E-2,-2.266847E-2,-4.2960816E-3,-1.2355452E-2,1.4220296E-2,4.465118E-3,-5.197592E-3,3.53456E-3,1.2550784E-2,2.439139E-3,-1.50182415E-2,4.757736E-3,-1.0363526E-2,3.57183E-2,-1.1076832E-2,-2.8597994E-3,-1.1948968E-3,-9.899202E-3,3.748914E-3,-3.3719998E-2,8.0983555E-3,2.6059516E-2,-8.217187E-3,-2.2047872E-2,1.4118726E-2,3.5884478E-3,-1.080528E-2,3.2340689E-3,-1.7587384E-2,-8.100029E-3,3.6031646E-3,1.8522771E-2,3.117383E-2,1.1458572E-2,-1.5627183E-3,8.80137E-3,-9.345598E-3,-1.3302928E-3,2.0819508E-2,6.664944E-3,7.6534064E-3,-1.0742229E-2,3.8508825E-2,2.7432324E-2,1.9476097E-2,-3.3626049E-3],"split_indices":[20,102,7,2,12,28,38,55,0,52,52,52,27,7,33,45,45,39,38,45,9,9,7,2,4,45,4,45,2,2,2,56,32,28,52,10,0,53,80,0,4,73,45,0,5,2,2,48,0,53,3,2,56,53,0,54,8,101,33,53,0,45,5,9,57,0,0,29,45,0,0,9,52,0,0,8,0,3,0,52,28,0,0,9,3,28,0,52,0,45,4,52,0,102,33,0,0,33,41,6,37,53,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.693E3,8.58E2,1.835E3,4.79E2,3.79E2,1.065E3,7.7E2,4.75E2,4E0,2.52E2,1.27E2,4.12E2,6.53E2,3.29E2,4.41E2,3.65E2,1.1E2,2.38E2,1.4E1,8.4E1,4.3E1,3.95E2,1.7E1,6.13E2,4E1,2.33E2,9.6E1,3.2E1,4.09E2,3.05E2,6E1,8.7E1,2.3E1,1.9E1,2.19E2,9E0,5E0,7.1E1,1.3E1,1.2E1,3.1E1,1.03E2,2.92E2,4E0,1.3E1,2.87E2,3.26E2,3.8E1,2E0,1.98E2,3.5E1,9.1E1,5E0,2.9E1,3E0,1.8E2,2.29E2,2.49E2,5.6E1,5.8E1,2E0,8E1,7E0,6E0,1.7E1,1.2E1,7E0,9.6E1,1.23E2,2E0,7E0,4E1,3.1E1,1.1E1,2E0,1.3E1,1.8E1,9.3E1,1E1,2.3E1,2.69E2,6E0,7E0,5.7E1,2.3E2,3.23E2,3E0,3E1,8E0,8.9E1,1.09E2,3E1,5E0,7E1,2.1E1,2E0,3E0,1.4E1,1.5E1,1.33E2,4.7E1,1.43E2,8.6E1,2E2,4.9E1,4.2E1,1.4E1,1.6E1,4.2E1,1.4E1,6.6E1,2E0,5E0,2E0,4E0,5E0,1.2E1,7.4E1,2.2E1,8.1E1,4.2E1,1.2E1,2.8E1,1.2E1,1.9E1,4E0,9E0,8.7E1,6E0,1.3E1,1E1,1.38E2,1.31E2,1.5E1,4.2E1,2.27E2,3E0,2.99E2,2.4E1,4E0,2.6E1,6.2E1,2.7E1,1.6E1,9.3E1,2.1E1,9E0,1.2E1,5.8E1,1.7E1,4E0,3E0,1.1E1,6E0,9E0,1.06E2,2.7E1,4.3E1,4E0,6E1,8.3E1,7.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[7.38637E-3,9.797268E-2,-3.0374405E-1,2.2218784E-2,4.0858358E-1,-4.8061123E-1,-6.537279E-2,-5.7118237E-2,2.4897574E-1,7.6588136E-1,3.38231E-1,-5.010012E-1,8.022406E-2,-1.753539E-1,1.251775E-1,-1.4859292E-1,3.807346E-2,1.8957101E-1,4.3930897E-1,-1.2470475E-2,8.020212E-1,2.7486053E-1,6.3646173E-1,-5.4284924E-1,-2.8811055E-1,1.5190777E-2,-5.1498762E-3,-1.4508761E-1,-2.6178703E-2,2.2252588E-1,-3.3456957E-1,-1.990346E-1,1.702818E-2,8.613689E-2,-3.4345978E-1,5.0753217E-2,2.4441539E-1,-1.753276E-1,5.0067514E-1,6.883975E-1,5.722066E-2,3.7108693E-1,6.775551E-2,9.0885025E-1,3.7823236E-1,-4.9316275E-1,-7.765605E-1,-2.3894012E-3,-3.416335E-1,-3.2083705E-1,-1.0990294E-1,1.4805111E-1,4.348239E-1,-4.1206703E-1,7.875164E-3,-1.7395695E-1,-4.080462E-1,8.9714095E-2,-1.6254705E-1,1.6209288E-1,-9.4796926E-2,-2.9368865E-1,-2.7521055E-2,7.3506035E-2,-2.3238877E-2,1.7324558E-1,3.5873446E-1,2.1934551E-3,-1.9425211E-2,6.515506E-1,3.6918586E-1,7.396438E-1,1.9117853E-2,2.5364107E-1,4.611726E-1,-3.49597E-1,1.3557611E-1,3.138944E-1,9.949306E-1,4.6753854E-1,-1.9752875E-1,-7.0240694E-1,-4.5242184E-1,-6.186981E-1,-1.1854341E0,-7.2022905E-3,8.4138885E-3,-4.0942836E-1,-1.2761447E-1,-1.6486576E-2,-1.645069E-3,-1.3379458E-1,4.4861846E-2,-9.201413E-3,2.104347E-1,2.3324244E-2,8.65822E-3,-1.0821477E-2,-6.53563E-1,-1.1240039E-2,-4.689002E-3,-4.793625E-3,-2.347739E-2,-5.644859E-3,6.8189558E-3,-1.15500195E-2,-7.0821797E-4,3.9879967E-2,6.6070436E-3,-1.1256724E-2,4.2018414E-4,-1.5014447E-2,2.507491E-3,1.1694624E-2,9.377558E-4,5.8075506E-3,1.679558E-2,1.8086782E-2,-1.5146022E-2,1.4626165E-4,3.2752294E-2,2.2956595E-2,1.1944999E-2,3.6738936E-2,1.3396466E-2,1.9379811E-2,9.357921E-3,1.7417908E-2,2.7773544E-2,-6.8150945E-3,-3.1263042E-2,2.8248017E-3,1.5867896E-2,2.6278581E-2,-1.2242954E-3,3.4389954E-2,5.4661196E-2,1.1415201E-2,2.6988776E-2,3.5388148E-3,-1.9305076E-2,-1.5133599E-2,-3.613515E-2,-2.6053755E-2,-1.7493192E-2,-3.0447835E-2,-1.0481941E-2,-2.3212695E-2,-6.116148E-2,-1.4548871E-2,-2.572024E-2,1.0482011E-3,-1.463924E-2,-1.4363955E-2,-5.3987317E-3,4.485082E-3,-4.8237806E-3,4.3340423E-3,-6.950795E-3,5.40154E-3,1.3909321E-2,-1.2239776E-2,-3.7769392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,-1,-1,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,-1,99,101,103,105,107,109,111,-1,113,-1,115,117,-1,-1,119,121,123,-1,125,127,129,131,133,135,137,139,141,143,145,147,-1,-1,149,151,-1,-1,153,155,157,159,-1,-1,-1,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.536602E1,4.8721527E1,2.5391102E1,2.9985928E1,9.946518E0,3.9968338E0,5.428839E0,1.0759581E1,4.8012104E0,2.656746E0,6.2696266E0,2.8192291E0,6.1252934E-1,1.7636127E0,4.326236E0,5.2665377E0,1.1142289E1,2.5076218E0,3.935007E0,0E0,2.0339012E0,5.6147537E0,3.8062286E0,2.845253E0,8.708639E-1,0E0,0E0,9.1084313E-1,0E0,1.1846805E0,7.251853E-1,2.4606724E0,1.9463661E0,7.422846E0,6.137295E-1,1.1942489E0,1.8660975E0,5.2781445E-1,1.6044407E0,4.753933E-1,0E0,1.925003E0,2.6231608E0,1.1216888E0,1.7377605E0,1.6599693E0,2.3002243E0,2.8998196E-1,6.4420176E-1,1.5590072E-1,4.8299074E-1,5.905349E-1,1.7720151E-1,5.2647066E-1,0E0,1.9772549E0,1.2416706E0,1.1757776E0,4.9600863E-1,6.034857E0,2.3262599E0,4.1436386E-1,0E0,8.420196E-1,0E0,1.3226519E0,1.2697268E0,0E0,0E0,8.7898254E-1,5.6653786E-1,3.2925606E-1,0E0,6.773553E-1,1.0392933E0,7.241758E-1,1.1649519E0,4.1259894E-1,1.5338516E-1,4.8178577E-1,3.0878198E-1,4.867096E-1,1.3739624E0,1.06713295E-1,1.518631E-1,0E0,0E0,3.2221603E-1,3.5276777E-1,0E0,0E0,3.5813546E-1,1.3729304E-1,2.5708976E-1,3.031633E-1,0E0,0E0,0E0,1.2103701E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,65,65,66,66,69,69,70,70,71,71,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,87,87,88,88,91,91,92,92,93,93,94,94,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,-1,-1,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,-1,100,102,104,106,108,110,112,-1,114,-1,116,118,-1,-1,120,122,124,-1,126,128,130,132,134,136,138,140,142,144,146,148,-1,-1,150,152,-1,-1,154,156,158,160,-1,-1,-1,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,8.426E3,1E0,1.022332E6,2E0,3.970405E3,1.067536E3,3.3030225E2,3.097E3,1E0,7.7101436E3,9.3026364E-1,1.1708007E7,1.2199979E10,9.302862E7,3.422351E6,4.0844156E7,1E0,1E0,-1.2470475E-2,4.162121E0,2.3907686E7,8.2474226E-1,9.222717E-1,3.94E2,1.5190777E-2,-5.1498762E-3,1.06E3,-2.6178703E-2,5.185489E3,3.3047943E0,5.024605E0,2.4505E4,2.2893274E1,1.2427474E8,3.7832818E0,7.472532E9,3.3012173E9,1.0993947E1,4.728721E0,5.722066E-2,1.5353E4,2.308943E0,6.3039363E4,1.56E2,1.5804776E4,7.95629E5,2.56E2,2.8880222E6,6.1E1,7.9203456E8,1E0,5.9767612E7,2.010856E0,7.875164E-3,1.4735735E2,3.97E2,4.26E2,5.4121188E7,8E0,7.44393E5,1E0,-2.7521055E-2,5.714286E-1,-2.3238877E-2,4.3689142E2,1.0289962E3,2.1934551E-3,-1.9425211E-2,7.549744E7,1.59551E5,1E0,1.9117853E-2,8.6E1,2.1924436E5,1.7231706E-1,7.023838E3,1.5435694E9,7.137324E0,2.1325744E7,9.188401E4,3.7930825E5,1.8903887E0,5.5E1,1.992945E5,-7.2022905E-3,8.4138885E-3,6.5928856E5,1.157E3,-1.6486576E-2,-1.645069E-3,2.9030122E-4,8.182648E6,8.1480905E-2,6.4506575E5,2.3324244E-2,8.65822E-3,-1.0821477E-2,6E0,-1.1240039E-2,-4.689002E-3,-4.793625E-3,-2.347739E-2,-5.644859E-3,6.8189558E-3,-1.15500195E-2,-7.0821797E-4,3.9879967E-2,6.6070436E-3,-1.1256724E-2,4.2018414E-4,-1.5014447E-2,2.507491E-3,1.1694624E-2,9.377558E-4,5.8075506E-3,1.679558E-2,1.8086782E-2,-1.5146022E-2,1.4626165E-4,3.2752294E-2,2.2956595E-2,1.1944999E-2,3.6738936E-2,1.3396466E-2,1.9379811E-2,9.357921E-3,1.7417908E-2,2.7773544E-2,-6.8150945E-3,-3.1263042E-2,2.8248017E-3,1.5867896E-2,2.6278581E-2,-1.2242954E-3,3.4389954E-2,5.4661196E-2,1.1415201E-2,2.6988776E-2,3.5388148E-3,-1.9305076E-2,-1.5133599E-2,-3.613515E-2,-2.6053755E-2,-1.7493192E-2,-3.0447835E-2,-1.0481941E-2,-2.3212695E-2,-6.116148E-2,-1.4548871E-2,-2.572024E-2,1.0482011E-3,-1.463924E-2,-1.4363955E-2,-5.3987317E-3,4.485082E-3,-4.8237806E-3,4.3340423E-3,-6.950795E-3,5.40154E-3,1.3909321E-2,-1.2239776E-2,-3.7769392E-2],"split_indices":[19,2,59,28,17,52,52,52,2,15,52,39,47,5,45,9,45,102,66,0,54,45,57,27,0,0,0,2,0,52,35,53,29,53,45,53,31,5,56,38,0,2,54,28,8,51,29,0,32,3,12,102,45,57,0,52,2,2,12,18,1,96,0,53,0,52,4,0,0,7,9,6,0,11,28,38,4,7,53,1,28,28,53,8,33,0,0,32,0,0,0,42,45,38,28,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.672E3,2.07E3,6.02E2,1.665E3,4.05E2,3.45E2,2.57E2,1.234E3,4.31E2,6.5E1,3.4E2,3.33E2,1.2E1,1.63E2,9.4E1,6.29E2,6.05E2,3.3E2,1.01E2,2E0,6.3E1,2.82E2,5.8E1,2.77E2,5.6E1,5E0,7E0,1.52E2,1.1E1,7.8E1,1.6E1,4.82E2,1.47E2,5.38E2,6.7E1,9.4E1,2.36E2,9E0,9.2E1,5.1E1,1.2E1,1.92E2,9E1,2.7E1,3.1E1,2.31E2,4.6E1,9E0,4.7E1,2.4E1,1.28E2,5.9E1,1.9E1,1.4E1,2E0,4.32E2,5E1,1.05E2,4.2E1,3.79E2,1.59E2,5.7E1,1E1,9.1E1,3E0,1.47E2,8.9E1,5E0,4E0,4.1E1,5.1E1,4.2E1,9E0,8.5E1,1.07E2,1.2E1,7.8E1,4E0,2.3E1,2.7E1,4E0,3.5E1,1.96E2,3.5E1,1.1E1,5E0,4E0,3.5E1,1.2E1,2.2E1,2E0,1.11E2,1.7E1,1.7E1,4.2E1,1.5E1,4E0,9E0,5E0,2.39E2,1.93E2,1.1E1,3.9E1,2.1E1,8.4E1,2.7E1,1.5E1,1.2E1,3.67E2,6.7E1,9.2E1,5.4E1,3E0,2.1E1,7E1,1.15E2,3.2E1,8.7E1,2E0,2E0,3.9E1,2.5E1,2.6E1,3.9E1,3E0,2.2E1,6.3E1,6.1E1,4.6E1,8E0,4E0,5.7E1,2.1E1,2E0,2E0,1E1,1.3E1,9E0,1.8E1,2E0,2E0,5E0,3E1,9.3E1,1.03E2,3.3E1,2E0,2E0,9E0,2.1E1,1.4E1,7E0,5E0,1.1E1,1E2,1.3E1,4E0,1E1,7E0,2E1,2.2E1,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[6.8415073E-3,-2.3900324E-1,1.2297631E-1,-3.7571606E-1,-7.774172E-2,5.3328514E-2,3.906239E-1,-4.2016906E-1,1.7988554E-1,-1.692995E-1,2.3792598E-1,-5.045602E-2,2.1727946E-1,7.023055E-1,3.264362E-1,-1.9315444E-1,-4.9880263E-1,5.37293E-2,-1.4517334E-2,-2.2276229E-1,1.0792277E-2,1.4221674E-1,4.2141357E-1,-2.6582333E-1,-1.7170008E-2,1.4972404E-1,3.7727678E-1,7.4605554E-1,4.5594167E-2,2.6415056E-1,5.684199E-1,-2.1588987E-1,2.2732444E-2,-5.5888563E-1,-3.1610426E-1,2.9770127E-1,-1.2248392E-1,-2.4003726E-1,6.8315E-2,-1.19798474E-1,6.4353324E-2,1.8888097E-1,-1.4320059E-1,4.503752E-1,5.774121E-3,-2.9944035E-1,8.84004E-2,-1.602112E-1,2.2433339E-2,1.7311618E-1,-2.5368458E-1,4.4319814E-1,1.0016107E-1,7.841902E-1,2.3024652E-2,8.910671E-3,-5.2714553E-3,3.2983577E-1,-5.0871864E-2,8.6411256E-1,3.5114542E-1,-2.9500383E-1,-2.4748562E-2,-6.543193E-1,-4.3387097E-1,-5.532174E-1,-2.1831264E-1,6.342485E-3,2.058604E-2,-7.385668E-2,-2.021204E-2,-1.722193E-1,-2.9578283E-1,1.0050049E-2,-4.447256E-4,-4.3488745E-2,-1.0378487E-2,1.5050128E-1,-2.4269756E-2,3.159909E-1,1.1811354E-1,-9.754068E-3,2.1856343E-3,2.4952112E-2,1.33217145E-2,-2.0226671E-1,-3.8940284E-1,2.2388028E-2,-5.372893E-3,-1.3021599E-1,-4.1545063E-1,1.591351E-1,-8.04573E-2,1.4886717E-1,4.7834903E-1,-2.1667829E-2,1.4401774E-3,5.063877E-1,1.8815206E-1,-7.6076365E-3,1.6360363E-1,8.075759E-1,1.5007019E-2,2.5759277E-1,4.7058856E-1,2.4739679E-2,-3.476656E-1,2.8670388E-1,9.56064E-1,2.3009215E-1,3.1947073E-2,-1.1084582E-2,-2.968877E-2,2.8764794E-3,-1.6682466E-2,-2.7135054E-2,-4.190036E-2,-6.0230624E-3,-2.2423416E-2,-1.416683E-2,-3.001126E-2,-6.677955E-4,-1.4735691E-2,6.7610373E-3,-4.9768877E-3,-1.0897801E-2,-2.295959E-3,-1.7846253E-2,-9.423455E-3,-4.5614108E-3,4.6347543E-3,9.785591E-4,9.027367E-3,5.5191903E-3,-2.6279788E-3,1.9164782E-2,8.17318E-4,2.9037767E-5,9.406794E-3,-1.0560492E-2,8.688622E-3,-1.6451137E-2,-3.3853706E-2,-7.821845E-3,2.4148838E-3,-3.179421E-3,-2.191309E-2,3.7854288E-2,5.8623464E-3,-1.0637188E-2,2.860109E-4,3.7811992E-3,1.1300808E-2,1.3559695E-2,2.848814E-2,1.2363486E-2,2.5925787E-2,-9.695655E-3,1.2532416E-2,-2.1256553E-3,1.1427601E-2,1.3736075E-2,3.9478403E-2,1.5234678E-2,7.193999E-3,2.6801009E-2,1.7358784E-2,-1.0702049E-3,2.6346546E-2,-2.4879666E-2,-5.090142E-3,2.5439585E-2,-2.5555792E-3,5.324557E-2,3.1510875E-2,2.2711856E-2,3.6698652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,99,101,-1,-1,-1,103,105,107,109,111,113,115,117,119,121,-1,-1,123,-1,125,127,-1,-1,129,-1,131,133,135,137,-1,-1,-1,-1,139,141,-1,-1,143,145,147,149,151,153,-1,-1,155,157,-1,159,161,-1,163,165,167,169,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.6175255E1,1.8842617E1,3.3756775E1,1.1510826E1,1.1427604E1,2.4496372E1,7.2530365E0,7.561722E0,6.425977E0,2.948635E0,1.5030379E0,6.318246E0,5.974758E0,1.7760468E0,4.552349E0,1.8030186E0,3.3140411E0,0E0,1.0477389E0,1.2030897E0,5.0432444E-1,8.261744E-1,2.0708704E-1,1.4316092E0,4.33725E0,3.7619314E0,2.9698048E0,2.19841E-1,1.298127E-1,5.1898575E0,3.7440987E0,1.6409903E0,0E0,2.5555878E0,1.7574272E0,1.1779481E-1,3.0386797E-1,7.921238E-1,1.5571994E-1,1.4588088E-1,3.9302218E-1,4.3042982E-1,1.120732E-1,1.8355083E-1,0E0,8.6537075E-1,8.888966E-1,1.2167907E0,8.4520035E0,2.6895847E0,1.2543973E0,2.0481815E0,5.682658E-1,2.974701E-1,0E0,0E0,0E0,1.9836407E0,9.960066E-1,1.1063251E0,1.29984E0,1.4464202E0,9.2660755E-1,2.0777664E0,1.0629921E0,2.494402E-1,1.0734615E0,0E0,0E0,1.5043825E-1,0E0,6.9743824E-1,8.4753895E-1,0E0,0E0,1.03596605E-1,0E0,1.2465125E-1,1.17340714E-1,4.461558E-1,3.219476E-1,0E0,0E0,0E0,0E0,4.180627E-1,5.5846596E-1,0E0,0E0,9.054322E-1,2.1953797E-1,5.8846197E0,4.200802E0,2.09448E0,4.4565868E-1,0E0,0E0,7.8030777E-1,8.572498E-1,0E0,4.3185705E-1,1.7403221E-1,0E0,8.5551167E-1,4.614334E-1,9.029104E-1,3.1810105E-1,4.4407624E-1,2.8987122E-1,1.0502359E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,71,71,72,72,75,75,77,77,78,78,79,79,80,80,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,97,97,98,98,100,100,101,101,103,103,104,104,105,105,106,106,107,107,108,108,109,109],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,100,102,-1,-1,-1,104,106,108,110,112,114,116,118,120,122,-1,-1,124,-1,126,128,-1,-1,130,-1,132,134,136,138,-1,-1,-1,-1,140,142,-1,-1,144,146,148,150,152,154,-1,-1,156,158,-1,160,162,-1,164,166,168,170,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.426E3,3.317425E3,7.2041174E2,6.0809356E5,2E0,2.0304577E-1,2.46E2,1.2690893E0,1.3664E4,8.2E1,2.9892595E6,4.5481584E7,6.7652373E3,1.7474695E0,5.676415E-1,5.37293E-2,3.3517068E7,1.1159378E12,1.752512E2,6.1061732E7,2.970339E0,1.6006084E3,5.52E2,9.837297E0,1E0,2.3977574E7,3.01E2,3.5301748E7,1.1336898E-1,3.4986075E6,2.2732444E-2,2.0521326E0,1.1880085E2,4.8765745E0,1.921032E4,8.3306855E6,1.8000048E10,1.2062E5,1.3510204E1,6.5346925E6,1.60264E6,1.5609541E0,5.774121E-3,9.82542E5,1.83E2,2.4E1,2.2E1,7.064848E6,3.1E2,2E0,7.3308685E2,1E0,2.3024652E-2,8.910671E-3,-5.2714553E-3,2.1019447E-1,1.1623708E0,6.3039363E4,3.2794893E-1,5.69815E5,1E0,7.8599895E9,2.5332516E5,1.3177E4,1.3264E4,6.342485E-3,2.058604E-2,7.364257E4,-2.021204E-2,5.980516E5,2.728E3,1.0050049E-2,-4.447256E-4,1E0,-1.0378487E-2,1.9563605E5,7.576E3,2.5482938E0,5.0079144E5,-9.754068E-3,2.1856343E-3,2.4952112E-2,1.33217145E-2,1.038E3,7.492813E0,2.2388028E-2,-5.372893E-3,3.422351E6,2.873721E2,2.04E5,7.44393E5,1.06E3,1E0,-2.1667829E-2,1.4401774E-3,8.490231E-1,3.14088E5,-7.6076365E-3,1.4523809E0,4.785384E5,1.5007019E-2,1.3152658E7,1.9493858E7,6.702495E-1,2.4832896E9,3.5762883E4,2.0967E4,4.3978744E0,3.1947073E-2,-1.1084582E-2,-2.968877E-2,2.8764794E-3,-1.6682466E-2,-2.7135054E-2,-4.190036E-2,-6.0230624E-3,-2.2423416E-2,-1.416683E-2,-3.001126E-2,-6.677955E-4,-1.4735691E-2,6.7610373E-3,-4.9768877E-3,-1.0897801E-2,-2.295959E-3,-1.7846253E-2,-9.423455E-3,-4.5614108E-3,4.6347543E-3,9.785591E-4,9.027367E-3,5.5191903E-3,-2.6279788E-3,1.9164782E-2,8.17318E-4,2.9037767E-5,9.406794E-3,-1.0560492E-2,8.688622E-3,-1.6451137E-2,-3.3853706E-2,-7.821845E-3,2.4148838E-3,-3.179421E-3,-2.191309E-2,3.7854288E-2,5.8623464E-3,-1.0637188E-2,2.860109E-4,3.7811992E-3,1.1300808E-2,1.3559695E-2,2.848814E-2,1.2363486E-2,2.5925787E-2,-9.695655E-3,1.2532416E-2,-2.1256553E-3,1.1427601E-2,1.3736075E-2,3.9478403E-2,1.5234678E-2,7.193999E-3,2.6801009E-2,1.7358784E-2,-1.0702049E-3,2.6346546E-2,-2.4879666E-2,-5.090142E-3,2.5439585E-2,-2.5555792E-3,5.324557E-2,3.1510875E-2,2.2711856E-2,3.6698652E-3],"split_indices":[20,102,2,52,52,28,17,27,10,42,2,29,51,47,52,38,42,0,51,31,52,45,34,55,2,54,6,47,0,45,57,32,0,53,52,57,4,45,5,1,58,45,1,57,0,31,0,8,3,28,10,17,4,6,0,0,0,35,41,28,38,9,104,5,28,9,10,0,0,28,0,28,10,0,0,8,0,28,9,42,28,0,0,0,0,0,54,0,0,9,33,5,1,2,102,0,0,53,1,0,54,28,0,45,51,39,7,28,10,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.666E3,8.55E2,1.811E3,4.62E2,3.93E2,1.438E3,3.73E2,4.28E2,3.4E1,3.05E2,8.8E1,8.81E2,5.57E2,6.2E1,3.11E2,1.11E2,3.17E2,5E0,2.9E1,2.35E2,7E1,5.9E1,2.9E1,1.17E2,7.64E2,3.93E2,1.64E2,5.8E1,4E0,2.49E2,6.2E1,1.08E2,3E0,2.37E2,8E1,7E0,2.2E1,2.22E2,1.3E1,2E1,5E1,5.1E1,8E0,2.6E1,3E0,1.07E2,1E1,1.65E2,5.99E2,3.72E2,2.1E1,1.32E2,3.2E1,4.9E1,9E0,2E0,2E0,2.06E2,4.3E1,2.5E1,3.7E1,7.6E1,3.2E1,1.32E2,1.05E2,2.2E1,5.8E1,4E0,3E0,2E1,2E0,1.02E2,1.2E2,4E0,9E0,1.2E1,8E0,2.5E1,2.5E1,1.7E1,3.4E1,6E0,2E0,1.7E1,9E0,5.3E1,5.4E1,3E0,7E0,1.49E2,1.6E1,2.57E2,3.42E2,3.46E2,2.6E1,1.2E1,9E0,1.05E2,2.7E1,6E0,2.6E1,4.6E1,3E0,1.38E2,6.8E1,3.5E1,8E0,4E0,2.1E1,2.8E1,9E0,6.5E1,1.1E1,2.6E1,6E0,9.7E1,3.5E1,1.1E1,9.4E1,6E0,1.6E1,1.8E1,4E1,2E0,1.8E1,7E1,3.2E1,6.6E1,5.4E1,9E0,3E0,6E0,1.9E1,4E0,2.1E1,1.3E1,4E0,1.4E1,2E1,5.1E1,2E0,4.9E1,5E0,1.26E2,2.3E1,2E0,1.4E1,1.3E1,2.44E2,1.29E2,2.13E2,1.93E2,1.53E2,1.1E1,1.5E1,1.4E1,9.1E1,4E0,2.3E1,7E0,1.9E1,2E0,4.4E1,8.7E1,5.1E1,3.5E1,3.3E1,3.3E1,2E0,4E0,4E0,2E0,2E0,1.2E1,9E0,1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-1.1700931E-3,-2.2083169E-1,1.02506E-1,-3.720529E-1,-8.384257E-2,4.123486E-2,3.690944E-1,-4.5417738E-1,-2.3221032E-1,-1.0038841E-1,5.5686913E-2,-4.622238E-2,2.0783722E-1,4.3822956E-1,1.05671436E-1,-3.3931053E-1,-5.5303556E-1,-1.4410281E-1,-3.5920516E-1,-2.0287484E-1,1.305622E-1,-1.03566065E-1,6.720343E-2,2.2560506E-1,-2.9035056E-1,6.212498E-1,3.6558983E-1,-2.5730671E-2,4.4860345E-1,-3.7221906E-1,1.7204954E-1,-6.011704E-1,-2.7628335E-1,-2.1252732E-1,-1.532787E-2,-6.1976504E-1,-3.1477454E-1,-3.136656E-1,-1.133257E-1,8.937629E-2,4.3901873E-1,-7.8104414E-2,-3.652995E-1,3.527504E-3,1.7417094E-1,1.6358863E-1,4.1913912E-1,-4.0033913E-1,-1.7481353E-4,4.7435302E-2,5.351954E-1,3.131524E-1,7.6370424E-1,-2.096719E-2,4.6495512E-2,3.2184553E-1,3.307948E-2,-2.7165666E-1,-4.581976E-1,1.507775E-2,-8.183383E-3,-5.5537945E-1,-8.694184E-1,-1.518201E-2,-5.499286E-3,-1.6607599E-1,-4.572986E-1,-4.7694817E-2,5.349649E-3,-5.8775647E-3,-3.6635727E-2,-3.0747766E-2,-2.6920688E-1,-9.51993E-3,-3.7441197E-1,-1.5878254E-1,1.6213012E-1,-5.486665E-2,1.42875E-1,4.830728E-3,5.0929374E-1,-1.7302787E-1,3.581178E-2,-4.3912455E-1,-9.902181E-3,-1.2418658E-2,2.196395E-2,-2.1266206E-1,2.0686261E-1,1.7208724E-1,-2.3932112E-2,5.390719E-1,2.9626742E-1,-2.2260321E-2,-1.4594613E-3,2.6001596E-1,6.2334126E-1,1.1662375E-1,3.6537823E-1,9.087846E-3,3.886845E-2,3.3189926E-1,-1.3241038E-2,2.4056094E-2,8.686377E-3,-1.5566328E-2,-5.8053113E-3,-1.5621327E-2,-2.774462E-2,-4.001646E-2,-2.416033E-2,-4.611535E-2,-8.351378E-3,6.4470153E-3,-9.287821E-3,-2.5623959E-2,-5.9115887E-3,-4.4842646E-3,2.6143896E-3,-7.2903335E-3,-1.508508E-2,-7.446386E-3,5.069164E-3,-2.3715764E-2,-1.0465999E-2,-3.5979275E-3,-1.1838676E-2,-1.0360867E-3,1.2779582E-2,-2.16084E-4,-1.657113E-2,-1.5609409E-3,1.0280591E-2,3.0951483E-2,1.434488E-2,1.9488862E-2,-9.056114E-3,6.256312E-3,-8.537145E-3,-1.6985603E-2,-2.674975E-2,-2.6305504E-3,2.9071986E-3,-2.102909E-2,5.500749E-4,1.4881802E-2,5.722099E-3,6.9257254E-3,1.674554E-2,4.0641394E-2,2.269407E-2,1.9985855E-2,8.996729E-3,1.5797975E-2,1.2630989E-3,3.2295052E-2,1.3906097E-2,8.794423E-3,-1.262952E-3,1.8084612E-2,-1.4724937E-2,1.4353474E-3,2.0475438E-2,5.3934744E-3,-6.0574627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,95,97,99,-1,101,103,-1,105,107,-1,-1,109,111,-1,-1,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,131,-1,133,135,137,139,-1,141,-1,143,145,147,-1,149,151,-1,-1,153,155,157,159,-1,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1010494E1,1.7763557E1,2.971298E1,4.5802116E0,9.439987E0,2.1588642E1,6.1456985E0,2.7193604E0,1.6601849E0,1.0606667E1,0E0,6.3290944E0,4.579878E0,3.3491135E0,3.2560062E0,2.1002417E0,1.6456223E0,8.073877E-1,5.226779E-1,3.0400305E0,1.7139225E0,4.276519E0,2.226666E0,5.848564E0,5.66146E-1,1.8530273E0,3.8316803E0,1.5978097E0,3.9152288E-1,8.611946E-1,4.4552532E-1,9.2972565E-1,1.08273864E-1,6.0625076E-1,1.4101359E-1,4.835651E-1,6.4736414E-1,2.5460873E0,2.1919372E0,9.570069E-1,3.2884192E-1,6.3860536E0,5.76313E-1,1.50066E0,1.5868309E0,2.1999617E0,1.593914E0,2.9585814E-1,0E0,0E0,1.366045E0,1.7433033E0,4.5088196E-1,0E0,7.886503E-1,2.891351E-1,0E0,4.0228987E-1,7.704973E-1,0E0,0E0,1.0186291E0,8.070116E-1,0E0,0E0,4.6177208E-1,1.6758335E-1,1.3198736E-1,0E0,0E0,0E0,0E0,2.1983433E-1,4.2062145E-1,2.0570736E0,1.0835431E0,4.868676E-1,4.969115E-1,1.1407752E0,0E0,1.7999959E-1,3.0785017E0,5.483966E0,1.8516874E-1,0E0,6.250286E-1,0E0,5.1261675E-1,9.886079E-1,1.7592697E0,0E0,7.8863335E-1,7.221265E-1,0E0,0E0,2.6877987E-1,6.024723E-1,3.6897516E-1,1.2449188E0,0E0,0E0,2.1821475E-1,5.7011884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51,52,52,54,54,55,55,57,57,58,58,61,61,62,62,65,65,66,66,67,67,72,72,73,73,74,74,75,75,76,76,77,77,78,78,80,80,81,81,82,82,83,83,85,85,87,87,88,88,89,89,91,91,92,92,95,95,96,96,97,97,98,98,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,96,98,100,-1,102,104,-1,106,108,-1,-1,110,112,-1,-1,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,132,-1,134,136,138,140,-1,142,-1,144,146,148,-1,150,152,-1,-1,154,156,158,160,-1,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.631E3,2.8636363E0,2.62797E5,7.733016E5,3.5301748E7,6.748543E-1,5.751617E6,9.451483E-1,5.5686913E-2,1.645614E6,4.898012E-1,4.4365574E-4,7.7101436E3,5.378E3,1.0118026E0,2.877528E2,1.9646037E-4,1E0,1.121807E0,4.0844156E7,1.8108038E5,3.097E3,5.98E2,1.7324902E8,7.765071E0,3.0972284E1,2.4347392E-1,1.6904992E-1,1.447E3,1.0017953E8,1.681727E6,1.2342778E1,8.2474226E-1,7.279E3,1.4921534E5,2.9173258E-4,6.624E3,8.890291E2,1.6081998E5,3.5259784E2,5.1861375E6,1.5435694E9,4.54E2,4.9E1,1.2909952E1,1.3715873E3,-1.7481353E-4,4.7435302E-2,1E0,1.2204E4,2.3936528E5,-2.096719E-2,1.6507992E9,1.19507775E-1,3.307948E-2,7.5646E4,1.8333334E0,1.507775E-2,-8.183383E-3,4.8543688E-2,3.724138E0,-1.518201E-2,-5.499286E-3,1.2E1,2.8107285E-1,3.1851864E7,5.349649E-3,-5.8775647E-3,-3.6635727E-2,-3.0747766E-2,3.5229592E0,5.2350176E4,1.067536E3,1.3304372E7,3.6873734E5,4.616371E8,1E0,4.830728E-3,2.093563E0,6E0,2.429717E1,1.13808104E8,-9.902181E-3,3.474851E8,2.196395E-2,5.033165E7,1E0,5.0083565E6,-2.3932112E-2,2.5737E4,1.46617E5,-2.2260321E-2,-1.4594613E-3,3.84E2,5.632258E0,1.4274633E7,1.314806E8,9.087846E-3,3.886845E-2,9.313E3,4.5843E7,2.4056094E-2,8.686377E-3,-1.5566328E-2,-5.8053113E-3,-1.5621327E-2,-2.774462E-2,-4.001646E-2,-2.416033E-2,-4.611535E-2,-8.351378E-3,6.4470153E-3,-9.287821E-3,-2.5623959E-2,-5.9115887E-3,-4.4842646E-3,2.6143896E-3,-7.2903335E-3,-1.508508E-2,-7.446386E-3,5.069164E-3,-2.3715764E-2,-1.0465999E-2,-3.5979275E-3,-1.1838676E-2,-1.0360867E-3,1.2779582E-2,-2.16084E-4,-1.657113E-2,-1.5609409E-3,1.0280591E-2,3.0951483E-2,1.434488E-2,1.9488862E-2,-9.056114E-3,6.256312E-3,-8.537145E-3,-1.6985603E-2,-2.674975E-2,-2.6305504E-3,2.9071986E-3,-2.102909E-2,5.500749E-4,1.4881802E-2,5.722099E-3,6.9257254E-3,1.674554E-2,4.0641394E-2,2.269407E-2,1.9985855E-2,8.996729E-3,1.5797975E-2,1.2630989E-3,3.2295052E-2,1.3906097E-2,8.794423E-3,-1.262952E-3,1.8084612E-2,-1.4724937E-2,1.4353474E-3,2.0475438E-2,5.3934744E-3,-6.0574627E-3],"split_indices":[20,56,2,54,2,28,45,27,45,42,0,9,27,57,52,2,39,4,42,102,34,45,28,2,10,7,35,56,38,27,0,45,28,54,57,9,28,27,2,4,28,52,50,7,2,3,56,52,0,0,100,2,32,0,5,38,0,9,56,0,0,58,56,0,0,18,27,32,0,0,0,0,54,28,52,45,28,7,102,0,39,3,53,45,0,7,0,7,16,28,0,9,9,0,0,0,54,45,1,0,0,2,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.677E3,8.58E2,1.819E3,4.07E2,4.51E2,1.48E3,3.39E2,2.55E2,1.52E2,4.46E2,5E0,9.71E2,5.09E2,2.68E2,7.1E1,1.2E2,1.35E2,9.1E1,6.1E1,3.09E2,1.37E2,6.45E2,3.26E2,4.92E2,1.7E1,7.4E1,1.94E2,5.2E1,1.9E1,1.13E2,7E0,1.14E2,2.1E1,5.9E1,3.2E1,7E0,5.4E1,1.37E2,1.72E2,1.22E2,1.5E1,5.89E2,5.6E1,2.05E2,1.21E2,3.74E2,1.18E2,1.2E1,5E0,1.2E1,6.2E1,1.73E2,2.1E1,7E0,4.5E1,1.4E1,5E0,5.4E1,5.9E1,5E0,2E0,1E2,1.4E1,1.6E1,5E0,5.1E1,8E0,2.6E1,6E0,2E0,5E0,5E0,4.9E1,2.3E1,1.14E2,1.48E2,2.4E1,3.3E1,8.9E1,3E0,1.2E1,3.21E2,2.68E2,3.7E1,1.9E1,1.99E2,6E0,9E0,1.12E2,3.7E2,4E0,5.8E1,6E1,1E1,2E0,1.6E1,4.6E1,3.7E1,1.36E2,2E0,1.9E1,7E0,3.8E1,5E0,9E0,3.9E1,1.5E1,3E1,2.9E1,1.3E1,8.7E1,1.2E1,2E0,4E0,4.7E1,6E0,2E0,1.8E1,8E0,1.5E1,3.4E1,1E1,1.3E1,6.3E1,5.1E1,7.7E1,7.1E1,9E0,1.5E1,2.9E1,4E0,2.6E1,6.3E1,6E0,6E0,8E0,3.13E2,1.86E2,8.2E1,2.4E1,1.3E1,1.26E2,7.3E1,4E0,5E0,5E1,6.2E1,3.22E2,4.8E1,8E0,5E1,2.7E1,3.3E1,1.2E1,4E0,3.9E1,7E0,2.5E1,1.2E1,1.34E2,2E0,2E0,5E0,1.8E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-1.8533156E-4,7.718929E-2,-2.7878785E-1,-1.8487187E-2,2.4208145E-1,-4.4018972E-1,-5.9117075E-2,-3.4032863E-2,5.139266E-1,1.9574815E-1,5.302279E-1,-8.904963E-1,-4.143258E-1,-1.526585E-1,8.821755E-2,-8.014152E-2,1.5181763E-1,3.463094E-1,5.1637992E-2,-2.3115821E-1,2.1888073E-1,6.236E-3,5.7139736E-1,-9.9007004E-1,-1.5294529E-2,-4.71162E-1,-2.4290088E-1,-1.0945926E-1,-4.4247124E-1,1.7611788E-1,-2.7265316E-1,-1.2911242E-1,5.5460073E-2,1.7094187E-1,-2.1686712E-2,3.837709E-1,-5.687864E-3,-1.5979981E-2,-1.09011136E-1,1.8838236E-1,4.9477762E-1,6.2277055E-1,2.59314E-1,-1.832884E-2,-5.3681314E-2,-4.278674E-1,-7.8724E-1,1.00098915E-1,-3.109471E-1,-7.179725E-2,-2.4543749E-1,-2.971236E-2,-2.566502E-1,1.1371355E-1,3.7434798E-1,-1.3172558E-1,-3.0693294E-2,-2.0500316E-1,-3.4374062E-2,1.755707E-1,1.0060853E-2,1.3205121E-1,4.4525653E-1,4.1548496E-1,4.291729E-3,-9.367991E-3,2.0425245E-3,2.105947E-1,-9.005501E-2,2.1238561E-1,6.6011E-1,6.3987553E-1,-1.4151166E-3,8.720725E-2,2.1241812E-2,-6.4799404E-1,-4.010029E-1,-2.470162E-2,-8.9430815E-1,1.8032188E-2,-4.564451E-3,-7.648353E-2,-3.6597568E-1,-2.2085959E-1,-4.4192813E-2,-3.0012348E-1,-1.8440108E-3,-1.4794427E-2,-6.841471E-4,1.5143657E-1,-4.9798228E-2,2.1796564E-2,3.5094381E-3,-2.904395E-1,4.3454006E-2,-8.659856E-3,-1.9866806E-2,3.760486E-2,-2.9778187E-3,9.84364E-3,-3.0528118E-3,1.2138577E-3,-1.6964309E-2,6.895193E-3,-2.5672827E-2,1.2013704E-2,2.6081199E-2,2.132424E-2,7.0779156E-3,4.6586744E-3,1.3347322E-2,2.756037E-3,-1.4870225E-2,1.6509587E-2,-4.5690886E-3,3.8387332E-2,1.7257964E-2,2.477213E-2,3.533877E-2,-2.5544963E-3,1.007202E-2,-3.3627503E-2,-1.1598347E-2,-1.3438389E-2,-2.1737168E-2,-1.3411587E-2,-4.5667835E-2,-9.312574E-3,8.369203E-3,-1.3977025E-2,-2.8786132E-2,-6.8995254E-3,-1.728781E-2,-3.4483615E-3,1.855611E-3,-6.2171645E-3,-1.7042968E-2,1.3834342E-2,5.2099684E-3,-1.0221809E-2,2.8113348E-3,-2.431578E-3,-2.0309621E-2,6.43387E-3,-3.880267E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,43,-1,45,47,49,51,53,55,57,59,61,-1,63,-1,-1,65,67,69,71,73,-1,-1,75,77,79,81,83,85,-1,87,89,91,93,-1,95,97,99,101,103,105,107,-1,-1,-1,109,111,113,115,117,-1,119,-1,121,123,-1,125,-1,-1,127,129,131,133,135,-1,-1,-1,137,139,-1,-1,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.662958E1,3.244055E1,2.0218937E1,1.0793796E1,9.976044E0,3.4628067E0,3.3624666E0,1.0860923E1,3.1264362E0,6.4893875E0,1.6383076E0,6.505928E-1,2.935154E0,1.8170011E0,3.0614812E0,6.747301E0,2.9914365E0,5.6825495E-1,0E0,3.893869E-1,5.1153107E0,0E0,1.345581E0,6.835699E-1,0E0,2.8653297E0,1.8983126E0,6.5325725E-1,4.6904206E-1,9.0995383E-1,9.016168E-1,5.356515E0,1.4705243E0,2.5549312E0,0E0,2.2397423E-1,0E0,0E0,2.3162185E-1,3.4762383E0,2.7078028E0,9.11541E-1,4.3694115E-1,0E0,0E0,9.701805E-1,3.2343102E-1,7.9212475E-1,8.364558E-1,4.181769E-1,3.0530095E-1,0E0,1.3494325E-1,3.7763065E-1,4.0443897E-1,4.2967233E-1,0E0,2.0252914E0,7.637721E0,5.439222E-1,1.1059606E0,1.7984223E0,4.4275188E-1,1.3352442E-1,0E0,0E0,0E0,3.9708347E0,1.3845136E0,1.0105644E0,1.3223648E0,5.5583763E-1,0E0,1.6693854E-1,0E0,2.9166794E-1,1.0697727E0,0E0,3.8838863E-1,0E0,0E0,4.489327E-1,7.656059E-1,1.201601E-1,2.0808992E-1,1.6267681E-1,0E0,0E0,0E0,2.557807E-1,2.255958E-1,0E0,0E0,2.3470825E-1,1.01396695E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,67,67,68,68,69,69,70,70,71,71,73,73,75,75,76,76,78,78,81,81,82,82,83,83,84,84,85,85,89,89,90,90,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,44,-1,46,48,50,52,54,56,58,60,62,-1,64,-1,-1,66,68,70,72,74,-1,-1,76,78,80,82,84,86,-1,88,90,92,94,-1,96,98,100,102,104,106,108,-1,-1,-1,110,112,114,116,118,-1,120,-1,122,124,-1,126,-1,-1,128,130,132,134,136,-1,-1,-1,138,140,-1,-1,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.0950326E6,1E0,1.6173E4,3.1100148E6,6E0,1.0323588E3,9.9625344E5,5.3534385E3,1.83E2,6.2671E4,1.9125667E3,5.121E3,4.8297736E7,9.6910744E7,4.264897E6,6.975062E-1,2.2668628E7,5.1637992E-2,9.53882E-2,6.7652373E3,6.236E-3,2.3487206E7,1.9041E4,-1.5294529E-2,1.1034263E10,1E0,1.1095187E1,6.654321E0,5.185489E3,1.4299594E8,2.6659616E2,1.07E2,6.057671E6,-2.1686712E-2,1.65E2,-5.687864E-3,-1.5979981E-2,8.829276E-8,3.5301748E7,2.1150263E-1,1E0,1.5311552E9,-1.832884E-2,-5.3681314E-2,5.85E2,1E1,1.447E3,1E1,1.287E3,2.2376953E1,-2.971236E-2,6.823E3,5.976267E6,5.9767612E7,6.08767E3,-3.0693294E-2,4.602015E0,8E0,9.47306E3,1E0,4.9E1,1E0,5.794597E7,4.291729E-3,-9.367991E-3,2.0425245E-3,5.862126E2,2.3432E4,1.5401015E1,2.60332E5,3.1547058E1,-1.4151166E-3,1.3948507E7,2.1241812E-2,1.74E4,3.8609805E10,-2.470162E-2,2.8107285E-1,1.8032188E-2,-4.564451E-3,2.6740572E5,1.2E1,8.595346E-1,2.491016E-1,2.69E2,-1.8440108E-3,-1.4794427E-2,-6.841471E-4,1.18827E5,9.877E3,2.1796564E-2,3.5094381E-3,1.747E3,1.10018056E8,-8.659856E-3,-1.9866806E-2,3.760486E-2,-2.9778187E-3,9.84364E-3,-3.0528118E-3,1.2138577E-3,-1.6964309E-2,6.895193E-3,-2.5672827E-2,1.2013704E-2,2.6081199E-2,2.132424E-2,7.0779156E-3,4.6586744E-3,1.3347322E-2,2.756037E-3,-1.4870225E-2,1.6509587E-2,-4.5690886E-3,3.8387332E-2,1.7257964E-2,2.477213E-2,3.533877E-2,-2.5544963E-3,1.007202E-2,-3.3627503E-2,-1.1598347E-2,-1.3438389E-2,-2.1737168E-2,-1.3411587E-2,-4.5667835E-2,-9.312574E-3,8.369203E-3,-1.3977025E-2,-2.8786132E-2,-6.8995254E-3,-1.728781E-2,-3.4483615E-3,1.855611E-3,-6.2171645E-3,-1.7042968E-2,1.3834342E-2,5.2099684E-3,-1.0221809E-2,2.8113348E-3,-2.431578E-3,-2.0309621E-2,6.43387E-3,-3.880267E-3],"split_indices":[19,47,59,2,28,3,52,28,4,29,1,4,2,45,45,9,27,45,0,42,52,0,50,9,0,5,64,54,54,52,45,52,10,28,0,3,0,0,37,45,38,104,12,0,0,2,3,0,3,2,56,0,2,47,45,4,0,53,18,48,85,3,102,1,0,0,0,52,9,54,11,58,0,9,0,9,31,0,27,0,0,33,18,27,38,0,0,0,0,29,9,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.625E3,2.055E3,5.7E2,1.301E3,7.54E2,3.28E2,2.42E2,1.265E3,3.6E1,6.51E2,1.03E2,1.6E1,3.12E2,1.48E2,9.4E1,1.014E3,2.51E2,2.9E1,7E0,3.3E1,6.18E2,1E1,9.3E1,1.3E1,3E0,2.33E2,7.9E1,1.3E2,1.8E1,7.6E1,1.8E1,7.45E2,2.69E2,2.44E2,7E0,2.7E1,2E0,1.7E1,1.6E1,5.58E2,6E1,7.9E1,1.4E1,3E0,1E1,2.07E2,2.6E1,1.3E1,6.6E1,1.03E2,2.7E1,8E0,1E1,5.9E1,1.7E1,1.4E1,4E0,4.13E2,3.32E2,7.3E1,1.96E2,2.15E2,2.9E1,2.4E1,3E0,1E1,6E0,5.17E2,4.1E1,2.3E1,3.7E1,7.7E1,2E0,8E0,6E0,2E1,1.87E2,9E0,1.7E1,5E0,8E0,1.3E1,5.3E1,1.5E1,8.8E1,2.1E1,6E0,8E0,2E0,4.8E1,1.1E1,1.3E1,4E0,7E0,7E0,3.72E2,4.1E1,1E1,3.22E2,6.5E1,8E0,1.89E2,7E0,2.12E2,3E0,1.1E1,1.8E1,2.1E1,3E0,1.95E2,3.22E2,2.5E1,1.6E1,1.6E1,7E0,2.4E1,1.3E1,3.7E1,4E1,4E0,4E0,1.7E1,3E0,5.9E1,1.28E2,2E0,1.5E1,9E0,4E0,4.2E1,1.1E1,1.1E1,4E0,6.6E1,2.2E1,6E0,1.5E1,1E1,3.8E1,4E0,7E0,3E0,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[-1.1812591E-4,-2.1217752E-1,9.950555E-2,-3.3008397E-1,-6.9691785E-2,1.2290686E-2,2.8372344E-1,-3.4806302E-1,5.140371E-2,-1.4491461E-1,1.8358131E-1,-1.5911955E-1,5.697012E-2,-1.6482951E-1,3.1343165E-1,-4.1546497E-1,-1.6751058E-1,-1.9065656E-1,8.568481E-3,-9.287021E-2,2.4718828E-1,-1.3739425E-1,-5.7674456E-1,1.4927594E-1,-5.2786626E-2,-2.2034866E-1,2.7312504E-3,2.4332842E-1,4.411571E-1,-3.023017E-1,-5.2885777E-1,-2.7848902E-1,-4.83281E-2,-2.2789668E-1,-7.950142E-2,6.396678E-2,-1.488694E-1,-1.8736912E-1,8.208487E-3,1.7515372E-1,4.2211872E-1,-1.9731843E-1,-2.5184227E-2,-5.4248464E-3,-4.8912223E-2,3.7033524E-2,1.3044512E-1,-8.483824E-2,3.296145E-1,-1.134739E-2,1.1789459E-4,1.726036E-1,4.52706E-1,3.2321626E-1,5.802711E-1,-3.9668986E-1,-1.6043806E-1,-5.4921764E-1,9.203449E-2,-2.2308059E-1,-4.7870055E-1,7.05412E-2,-1.3864648E-1,-2.0584533E-1,-4.5575833E-1,-1.9969797E-1,-5.008102E-4,-2.3493841E-2,1.4257969E-1,-1.190015E-2,-2.1228474E-3,-5.493766E-3,-1.8740091E-2,3.2574403E-1,7.8579165E-2,2.2445519E-2,9.684837E-5,-3.001944E-1,-1.5768589E-1,1.09362975E-1,-1.2837303E-1,-1.7208385E-1,1.5264909E-1,-5.5696737E-2,-3.640512E-1,3.9571773E-2,2.1548674E-1,1.8689074E-1,-2.2304924E-2,5.415034E-1,1.9705012E-1,3.748758E-1,3.6789384E-2,6.722104E-1,2.6806518E-1,-1.4159009E-2,-2.5497314E-2,2.1292619E-5,-1.6351983E-2,-2.8597225E-2,-1.1688683E-2,-5.369946E-3,1.597509E-2,-1.48337865E-2,-1.2660619E-3,-2.536114E-2,-6.3517992E-3,1.5126711E-3,1.3727201E-2,-1.1317879E-2,-1.5929495E-3,-1.0287399E-2,5.0801346E-3,-2.8260436E-2,-1.1355379E-2,-4.698423E-3,-1.2944702E-2,-2.591937E-3,5.3279162E-3,2.3805564E-3,-4.0496425E-3,7.998828E-3,-1.9024961E-3,1.8258393E-2,3.9733727E-3,-2.4188573E-4,7.5469175E-3,-6.9455393E-3,-1.6192617E-2,-4.960649E-3,-1.1414312E-2,1.993946E-2,3.3537608E-3,2.741936E-5,-8.691419E-3,8.416972E-3,-1.1298114E-2,3.9031713E-3,1.14904875E-2,-6.5663285E-3,1.5550986E-3,-2.3483902E-2,-3.9948937E-3,1.3105262E-2,-1.036583E-2,1.1249707E-2,4.1080574E-3,3.495401E-2,2.103797E-2,1.3737437E-2,-5.7545435E-3,3.8826056E-3,1.9912638E-2,-2.686017E-2,7.368813E-3,2.488727E-2,3.5861753E-2,4.460515E-3,1.8749028E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,-1,-1,-1,81,83,85,-1,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,-1,-1,-1,-1,123,125,-1,-1,127,129,131,133,135,137,139,141,-1,143,145,-1,147,149,151,153,155,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4991318E1,1.3954586E1,2.845035E1,1.1854408E1,7.2282295E0,9.222255E0,7.6339912E0,5.406723E0,0E0,2.0516534E0,1.5527854E0,2.1799421E0,9.683165E0,4.5599717E-1,4.6740723E0,4.036148E0,1.6266501E0,9.1325283E-1,6.026044E-1,4.5666212E-1,8.2073784E-1,1.595365E0,2.3569455E0,5.9977903E0,5.384086E0,1.0781622E-1,0E0,5.0603027E0,2.897007E0,2.1591387E0,2.0984383E0,6.03189E-1,6.646813E-1,7.5032806E-1,5.5064476E-1,3.54002E-1,1.7644098E-1,1.4773437E-1,0E0,7.3064935E-1,3.902881E-1,5.8463526E-1,1.1802753E0,0E0,0E0,0E0,3.4185266E0,3.2669442E0,1.7492402E0,0E0,0E0,2.467421E0,1.8683357E0,1.5250626E0,2.2596416E0,1.2001257E0,1.9803716E0,2.149994E0,3.3328712E-1,8.779831E-1,1.585958E-1,2.1823375E-1,3.5243243E-1,4.562459E-1,2.7132225E-1,1.3844365E-1,2.2174504E-1,1.1595838E-1,1.2695193E-1,0E0,0E0,0E0,0E0,2.3789167E-1,2.1884044E-1,0E0,0E0,1.981039E-1,4.727745E-1,4.1761756E-1,3.288352E-1,8.185698E-1,2.9053335E0,2.638853E0,1.3061304E0,0E0,7.877954E-1,1.2226553E0,0E0,9.1158867E-1,7.181251E-1,1.0242395E0,1.2758327E0,3.7052727E-1,4.1695678E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,-1,-1,-1,82,84,86,-1,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,-1,-1,-1,-1,124,126,-1,-1,128,130,132,134,136,138,140,142,-1,144,146,-1,148,150,152,154,156,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,3.3633875E6,2.14099E5,6.860185E2,4.97E2,2.1328075E-1,2.720437E6,5.140371E-2,1.2690893E0,1.5896305E-3,4.2E1,1.9E1,6E1,1.2974394E6,6.7033327E-1,5.4786605E-1,1.187875E6,5.9572783E0,1E0,1.3664E4,3.5E2,5.5E1,2.04E5,1.0052E4,4.0728608E3,2.7312504E-3,4.247868E3,5.964E3,7.5569354E2,6.08767E3,9.6910744E7,1.1297775E2,4.12204E5,2.1592189E5,2.3125623E5,5.124411E2,5.5325594E9,8.208487E-3,6.5346925E6,1.0881593E10,4.1E1,1.4492002E0,-5.4248464E-3,-4.8912223E-2,3.7033524E-2,7.805608E4,4.719849E7,7.969174E7,-1.134739E-2,1.1789459E-4,4.1822004E7,3.6604617E-2,1.9162654E7,2.3977574E7,2.5943396E0,7.6499896E9,3.6282136E0,1.3E1,6.494E3,3.757042E6,2.4137697E0,1.2103871E0,1E0,7.115101E2,2.0728285E6,3.673719E8,6.822511E5,5.4379158E1,-1.190015E-2,-2.1228474E-3,-5.493766E-3,-1.8740091E-2,2.5482938E0,3.6482175E5,2.2445519E-2,9.684837E-5,1.901E3,9.25E0,1.021793E6,9E0,6.9436204E-1,1.172E3,7.44393E5,3.1998687E3,3.9571773E-2,1E0,9.070543E0,-2.2304924E-2,1.417988E6,1.775894E10,1.8114872E7,1E0,1.6140062E3,6.72E2,-1.4159009E-2,-2.5497314E-2,2.1292619E-5,-1.6351983E-2,-2.8597225E-2,-1.1688683E-2,-5.369946E-3,1.597509E-2,-1.48337865E-2,-1.2660619E-3,-2.536114E-2,-6.3517992E-3,1.5126711E-3,1.3727201E-2,-1.1317879E-2,-1.5929495E-3,-1.0287399E-2,5.0801346E-3,-2.8260436E-2,-1.1355379E-2,-4.698423E-3,-1.2944702E-2,-2.591937E-3,5.3279162E-3,2.3805564E-3,-4.0496425E-3,7.998828E-3,-1.9024961E-3,1.8258393E-2,3.9733727E-3,-2.4188573E-4,7.5469175E-3,-6.9455393E-3,-1.6192617E-2,-4.960649E-3,-1.1414312E-2,1.993946E-2,3.3537608E-3,2.741936E-5,-8.691419E-3,8.416972E-3,-1.1298114E-2,3.9031713E-3,1.14904875E-2,-6.5663285E-3,1.5550986E-3,-2.3483902E-2,-3.9948937E-3,1.3105262E-2,-1.036583E-2,1.1249707E-2,4.1080574E-3,3.495401E-2,2.103797E-2,1.3737437E-2,-5.7545435E-3,3.8826056E-3,1.9912638E-2,-2.686017E-2,7.368813E-3,2.488727E-2,3.5861753E-2,4.460515E-3,1.8749028E-2],"split_indices":[20,102,50,2,52,2,42,32,0,42,38,6,3,3,28,27,39,1,53,89,2,2,0,5,2,4,0,52,2,4,4,45,58,29,28,28,4,5,0,45,12,2,53,0,0,0,28,45,7,0,0,51,27,47,47,56,5,53,3,2,29,35,39,89,55,32,7,32,58,0,0,0,0,42,28,0,0,9,58,5,58,53,2,1,4,0,84,53,0,29,5,12,8,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.601E3,8.31E2,1.77E3,4.54E2,3.77E2,1.202E3,5.68E2,4.49E2,5E0,2.91E2,8.6E1,2.48E2,9.54E2,3.5E1,5.33E2,3.26E2,1.23E2,2.24E2,6.7E1,1.6E1,7E1,2.37E2,1.1E1,5.18E2,4.36E2,2.8E1,7E0,3.46E2,1.87E2,1.65E2,1.61E2,6.3E1,6E1,1.67E2,5.7E1,5E1,1.7E1,1.2E1,4E0,5.1E1,1.9E1,1.54E2,8.3E1,6E0,5E0,1.4E1,5.04E2,4.03E2,3.3E1,2.6E1,2E0,2.6E2,8.6E1,1.03E2,8.4E1,9.8E1,6.7E1,1.56E2,5E0,5.1E1,1.2E1,2.6E1,3.4E1,1.54E2,1.3E1,2.2E1,3.5E1,2.4E1,2.6E1,8E0,9E0,1E1,2E0,1.9E1,3.2E1,1.7E1,2E0,4.1E1,1.13E2,3.6E1,4.7E1,3.4E1,4.7E2,3.66E2,3.7E1,5E0,2.8E1,2.55E2,5E0,6.3E1,2.3E1,8.7E1,1.6E1,6.4E1,2E1,5.8E1,4E1,3.6E1,3.1E1,1.34E2,2.2E1,3E0,2E0,3.5E1,1.6E1,1E1,2E0,2.3E1,3E0,1.7E1,1.7E1,1.5E2,4E0,7E0,6E0,1E1,1.2E1,2.4E1,1.1E1,1.1E1,1.3E1,2.3E1,3E0,1.5E1,4E0,1.6E1,1.6E1,9E0,3.2E1,6.9E1,4.4E1,3E0,3.3E1,1.4E1,3.3E1,5E0,2.9E1,2.6E2,2.1E2,1.9E2,1.76E2,2.5E1,1.2E1,2.5E1,3E0,1.72E2,8.3E1,2E1,4.3E1,1.8E1,5E0,1.1E1,7.6E1,2E0,1.4E1,2.4E1,4E1,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[5.063741E-3,-1.9028682E-1,9.715479E-2,-3.0770007E-1,-5.123238E-2,4.0889755E-2,3.3335114E-1,-3.1787968E-1,4.5255132E-2,-1.3289656E-1,1.9678284E-1,-5.3676285E-2,1.7596121E-1,3.9855924E-1,6.682498E-2,-3.740894E-1,-1.106596E-1,-1.7535107E-1,1.2653947E-2,1.10195026E-1,3.1447336E-1,-2.3090102E-1,-2.430489E-2,1.1812779E-1,3.188687E-1,6.002395E-1,3.2356128E-1,-4.1215703E-1,1.2658738E-1,-3.4826675E-1,-7.585101E-1,-5.296035E-3,-2.6857254E-1,-2.0605093E-1,-2.4375435E-2,8.423543E-2,-5.331391E-2,1.6784361E-1,-3.871292E-2,3.5326666E-1,3.619664E-3,-2.5422713E-1,1.9196434E-1,-6.651609E-2,4.4299565E-2,1.3478264E-1,-1.6999839E-2,7.8164E-1,2.828434E-1,7.7163047E-1,4.3929014E-1,2.104338E-1,4.5997095E-1,-2.78277E-2,-9.319314E-3,1.066173E-2,3.023679E-1,-3.9859042E-1,-2.1125762E-1,-4.055045E-2,-3.821295E-1,-4.61906E-2,2.2123571E-2,-2.0258436E-1,-4.4627866E-1,-2.012045E-2,-1.8541807E-1,3.0427357E-2,-1.4053719E-1,-4.5638094E-3,1.1969819E-1,5.1831612E-3,-3.6822548E-3,1.3388498E-2,5.3571593E-2,-7.18669E-3,6.790842E-3,4.4458872E-1,1.8452723E-1,-2.006734E-1,-3.902082E-1,1.9596996E-2,-3.5087622E-3,-1.0742121E-1,3.517041E-2,8.781587E-2,-3.072172E-2,-1.3778908E-2,1.8262824E-1,4.23824E-2,8.457966E-3,3.3276123E-1,6.809356E-2,7.9821265E-1,1.0632489E-2,1.527415E-2,3.227701E-2,1.7764118E-1,3.6677328E-1,7.369334E-1,3.6377224E-1,1.069791E-2,-4.285165E-2,3.9866355E-1,-1.6254738E-1,-1.6244465E-2,-2.8825775E-2,-8.6132187E-4,-1.3834382E-2,-2.6657287E-2,-1.0577224E-3,4.0802546E-3,-6.588688E-3,-1.2281722E-2,8.9785364E-4,-2.6249254E-2,-1.2437388E-3,-8.28177E-3,-2.0402577E-2,-2.790003E-3,8.274651E-3,1.8810356E-3,-9.501936E-3,5.973803E-4,9.019937E-3,5.160228E-3,-2.6201033E-3,1.2511615E-2,2.5415035E-2,-1.7566555E-3,1.1886012E-2,-1.2323977E-2,-4.3983194E-3,-1.348573E-2,-3.8979713E-2,1.1513348E-3,-7.781715E-3,8.143433E-3,-2.5627918E-3,2.4361168E-3,1.2971507E-2,-8.105712E-3,1.1669343E-3,-4.2938446E-3,6.8412223E-3,8.046124E-3,2.4497425E-2,1.7818317E-2,6.7237653E-3,-8.30653E-3,6.2499577E-3,3.1132214E-2,4.6746843E-2,-2.670691E-3,9.273692E-3,1.9320628E-2,1.7612483E-3,2.023467E-2,4.740859E-2,1.9158298E-2,-9.3394555E-3,9.158682E-3,-4.991569E-3,2.4183936E-2,5.453911E-3,-2.1601168E-2,8.627033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,101,103,105,107,-1,109,111,-1,113,115,-1,117,119,121,-1,123,-1,-1,-1,125,-1,-1,127,129,131,133,-1,-1,135,137,139,141,143,145,-1,-1,147,149,151,-1,-1,-1,153,155,157,159,-1,161,163,165,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.817782E1,1.39926605E1,2.4173431E1,6.229927E0,8.006211E0,1.879878E1,6.0562515E0,5.339878E0,0E0,1.8374634E0,9.6402216E-1,4.50221E0,4.9708176E0,4.05608E0,2.0559316E0,3.3392525E0,1.6624049E0,1.0610118E0,3.255922E-1,5.0438887E-1,3.5326242E-1,1.2620511E0,2.1574786E0,3.4596329E0,2.6797562E0,1.7299252E0,3.0975628E0,1.8091524E-1,1.280557E0,2.273819E0,4.0694904E-1,1.1869437E0,3.786509E-1,7.657747E-1,2.5891572E-1,2.2150809E-1,1.4542383E-1,5.2121973E-1,3.6298576E-1,4.5729828E-1,0E0,7.7372456E-1,4.3331096E-1,1.9213073E0,9.298584E-1,2.9807754E0,0E0,5.940223E-1,1.7246752E0,3.3413315E-1,9.4307184E-1,5.2442646E-1,2.2097626E0,0E0,0E0,4.5352525E-1,1.1840343E0,2.8025627E0,1.4049277E0,0E0,3.6467153E-1,6.961445E-1,0E0,3.7442434E-1,4.137684E-1,0E0,4.551053E-1,3.635608E-1,1.4307505E-1,0E0,2.037473E-1,0E0,0E0,0E0,1.3496E-1,0E0,0E0,2.0863199E-1,2.0626563E-1,5.0670767E-1,1.2435117E0,0E0,0E0,2.3856735E0,1.5999128E0,1.2035466E0,8.0543995E-1,1.2369834E0,1.4381056E0,0E0,0E0,9.4571877E-1,5.050356E-1,2.3518181E-1,0E0,0E0,0E0,3.7310076E-1,2.150178E-1,1.4113312E0,1.5170412E0,0E0,4.797084E-1,5.783272E-1,5.7559E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,60,60,61,61,63,63,64,64,66,66,67,67,68,68,70,70,74,74,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,93,93,97,97,98,98,99,99,100,100,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,102,104,106,108,-1,110,112,-1,114,116,-1,118,120,122,-1,124,-1,-1,-1,126,-1,-1,128,130,132,134,-1,-1,136,138,140,142,144,146,-1,-1,148,150,152,-1,-1,-1,154,156,158,160,-1,162,164,166,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.631E3,2.14099E5,6.860185E2,4.930349E5,3.5947604E7,1.945711E1,4.5255132E-2,1.1693485E0,2.160255E9,8.2E1,2.9892595E6,4.4365574E-4,3.0972284E1,1.2427474E8,7.218466E7,2.231E3,2.3686748E0,3.1826714E7,4.607796E-1,1.5694646E3,1.645614E6,7.5179994E-1,1.0918E4,7.06699E8,4.0889E4,4.1043E4,9.256843E3,9.357879E2,4.8083666E-1,8.230524E3,5.784E4,2.750917E-4,1.164249E7,1.8053533E8,7.576E3,6.4512783E-1,5.794E3,2.9722316E0,3.619664E-3,1.4504054E8,1.92E2,1.1968015E8,1.07E2,1E0,-1.6999839E-2,2E1,2E0,1.8222668E2,4.391553E6,1.0665628E6,1.417988E6,-2.78277E-2,-9.319314E-3,3.9382784E9,2.3580047E10,3.0161016E7,3.653602E7,-4.055045E-2,2.6E1,2.8091298E7,2.2123571E-2,9.408768E1,9.367855E-1,-2.012045E-2,1E0,5.810811E-1,1.0503613E5,-4.5638094E-3,1.8991614E5,5.1831612E-3,-3.6822548E-3,1.3388498E-2,2.8216E4,-7.18669E-3,6.790842E-3,1.948181E0,7.8377126E8,5.222222E1,3.5612745E0,1.9596996E-2,-3.5087622E-3,1E0,2.558106E10,7.298614E2,1.5676449E3,1.405E3,7.064848E6,4.23824E-2,8.457966E-3,2E0,7.697189E2,2.6295085E3,1.0632489E-2,1.527415E-2,3.227701E-2,1.260173E6,3.3517068E7,6.162955E0,1.5005797E4,1.069791E-2,3.7448005E3,1.0008265E1,9.627E3,-1.6244465E-2,-2.8825775E-2,-8.6132187E-4,-1.3834382E-2,-2.6657287E-2,-1.0577224E-3,4.0802546E-3,-6.588688E-3,-1.2281722E-2,8.9785364E-4,-2.6249254E-2,-1.2437388E-3,-8.28177E-3,-2.0402577E-2,-2.790003E-3,8.274651E-3,1.8810356E-3,-9.501936E-3,5.973803E-4,9.019937E-3,5.160228E-3,-2.6201033E-3,1.2511615E-2,2.5415035E-2,-1.7566555E-3,1.1886012E-2,-1.2323977E-2,-4.3983194E-3,-1.348573E-2,-3.8979713E-2,1.1513348E-3,-7.781715E-3,8.143433E-3,-2.5627918E-3,2.4361168E-3,1.2971507E-2,-8.105712E-3,1.1669343E-3,-4.2938446E-3,6.8412223E-3,8.046124E-3,2.4497425E-2,1.7818317E-2,6.7237653E-3,-8.30653E-3,6.2499577E-3,3.1132214E-2,4.6746843E-2,-2.670691E-3,9.273692E-3,1.9320628E-2,1.7612483E-3,2.023467E-2,4.740859E-2,1.9158298E-2,-9.3394555E-3,9.158682E-3,-4.991569E-3,2.4183936E-2,5.453911E-3,-2.1601168E-2,8.627033E-3],"split_indices":[20,102,2,2,52,28,45,56,0,42,12,29,51,57,56,45,45,0,53,45,38,52,9,27,9,7,2,10,4,55,42,52,10,39,45,7,9,57,2,57,0,31,0,7,3,102,0,3,6,58,28,28,29,0,0,5,5,45,45,0,8,45,0,56,27,0,77,57,33,0,28,0,0,0,9,0,0,42,7,4,53,0,0,8,19,52,55,10,28,0,0,17,4,4,0,0,0,50,51,53,4,0,4,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.676E3,8.57E2,1.819E3,4.64E2,3.93E2,1.47E3,3.49E2,4.61E2,3E0,2.96E2,9.7E1,8.65E2,6.05E2,2.8E2,6.9E1,3.62E2,9.9E1,2.29E2,6.7E1,5.7E1,4E1,1.22E2,7.43E2,4.32E2,1.73E2,7.4E1,2.06E2,7E0,6.2E1,3.41E2,2.1E1,6E1,3.9E1,1.9E2,3.9E1,3.2E1,3.5E1,4.1E1,1.6E1,3.4E1,6E0,1.16E2,6E0,4.6E2,2.83E2,4.18E2,1.4E1,1.1E1,1.62E2,3.4E1,4E1,1.14E2,9.2E1,3E0,4E0,3.8E1,2.4E1,2.48E2,9.3E1,1.6E1,5E0,5.6E1,4E0,3E1,9E0,1.5E1,1.75E2,2.7E1,1.2E1,5E0,2.7E1,4E0,3.1E1,2E1,2.1E1,1E1,6E0,2.1E1,1.3E1,8.5E1,3.1E1,3E0,3E0,3.28E2,1.32E2,1.79E2,1.04E2,1.02E2,3.16E2,9E0,2E0,1.31E2,3.1E1,3.2E1,2E0,2.8E1,1.2E1,9.6E1,1.8E1,2.2E1,7E1,7E0,3.1E1,2E1,4E0,1.94E2,5.4E1,2.7E1,6.6E1,3E0,2E0,2.3E1,3.3E1,2.4E1,6E0,7E0,2E0,1.68E2,7E0,1.7E1,1E1,3E0,9E0,1.1E1,1.6E1,1.4E1,7E0,8E0,1.3E1,3E0,1E1,5.5E1,3E1,2.6E1,5E0,9.7E1,2.31E2,5.2E1,8E1,1.5E2,2.9E1,2.9E1,7.5E1,6.9E1,3.3E1,3.04E2,1.2E1,1.08E2,2.3E1,6E0,2.5E1,2E1,1.2E1,6E0,9E1,1.6E1,2E0,1.1E1,1.1E1,6.6E1,4E0,6E0,2.5E1,1.4E1,6E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"167","size_leaf_vector":"1"}},{"base_weights":[2.534354E-4,-1.817672E-1,8.404214E-2,-3.109616E-1,-7.187962E-2,2.8869577E-2,2.9664963E-1,-2.77736E-1,-6.1055225E-1,-1.19388856E-1,2.5864786E-1,-4.3913096E-2,1.7333625E-1,3.5595453E-1,6.323017E-2,-3.7522808E-1,-1.9453834E-1,-8.328086E-1,-3.4446386E-1,-4.1534215E-2,-3.6376137E-1,4.559558E-2,1.7526606E-1,-2.115718E-2,-2.6970044E-1,1.8964878E-1,-2.6495045E-1,5.084102E-1,2.9189843E-1,-4.2525625E-1,1.2513228E-1,-7.0113844E-1,-3.442746E-1,-7.879981E-2,-3.2212946E-1,-4.2002924E-2,-1.387348E-2,-1.9726798E-1,-2.7231444E-2,-1.0362253E-1,1.2652984E-1,-3.909306E-1,-1.2637248E-2,2.468904E-1,-1.5740766E-1,-2.0026104E-1,9.947142E-3,-1.7888477E-1,-7.868624E-1,4.2348534E-2,2.5271308E-1,-3.225862E-1,6.2620286E-3,5.876966E-1,3.0751517E-1,1.997716E-1,4.2258406E-1,-2.8320055E-2,-8.338294E-3,6.199665E-3,3.38963E-1,-7.1059936E-3,-4.0808868E-2,-4.1303667E-1,-2.1193138E-1,-1.1484453E-1,1.7680207E-1,-3.7580168E-1,-9.896091E-2,-1.2855413E-2,-1.4231578E-3,-4.66029E-2,-2.185731E-1,2.683018E-1,3.231407E-2,-5.757786E-1,-3.4318265E-1,1.025269E-2,-5.9332848E-3,1.772871E-2,1.4728934E-1,8.637621E-4,-2.7737623E-1,-2.3111896E-1,-5.3056203E-2,5.7410216E-1,-6.975689E-3,-2.7371916E-1,-4.4144414E-2,-4.3356832E-2,-4.319082E-3,-2.7111015E-1,7.0523955E-2,2.8781077E-1,3.4509905E-2,-2.0273535E-2,-8.089927E-3,5.1899564E-1,4.4758625E-2,5.9207836E-3,3.9258856E-1,1.391346E-1,3.4514546E-1,1.4747322E-1,4.6830365E-1,-9.775152E-2,1.0556123E-1,2.3733936E-1,2.9750304E-2,-2.0808822E-2,-3.5871996E-3,1.1896432E-3,-1.2814185E-2,-4.351967E-3,-1.672296E-2,1.2292709E-2,-1.1229675E-4,-8.200372E-3,-2.1054527E-2,-1.03594195E-2,1.3547657E-3,-7.110817E-4,-1.1612182E-2,-5.4985033E-3,-1.5030434E-2,2.3009578E-2,7.4311094E-3,9.579997E-3,-1.1696282E-3,-1.2166969E-2,-3.0006858E-2,-1.4278742E-2,-2.8808668E-2,-6.5609803E-3,9.8274E-3,-4.3682167E-3,-2.0743106E-2,-1.5474554E-2,-9.761322E-3,2.0288483E-3,-1.0382548E-2,-1.7220875E-2,3.28266E-2,4.0469943E-3,-2.303593E-3,-1.4086619E-2,4.3822938E-4,-4.760197E-3,5.4710703E-3,4.2146565E-3,-2.587018E-2,-2.8679082E-3,5.6604687E-3,9.774477E-3,1.842159E-2,-1.6164748E-3,8.7346975E-3,2.6857993E-2,1.3355934E-2,2.0775834E-2,6.6963662E-3,2.833423E-3,8.859438E-3,1.2243242E-2,3.05354E-2,1.1947385E-2,-1.4830871E-2,1.9344917E-2,3.1805784E-2,-9.738684E-3,4.267631E-3,3.2484306E-3,1.6621247E-2,1.4479285E-2,-4.705413E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,95,-1,97,99,101,103,-1,-1,105,107,-1,-1,109,111,113,115,117,119,-1,-1,121,123,125,127,129,131,-1,-1,-1,133,-1,135,137,139,141,143,145,147,-1,-1,149,151,153,155,-1,-1,157,-1,-1,159,161,163,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0980194E1,1.200239E1,2.157482E1,3.7025604E0,7.2386065E0,1.538006E1,5.223957E0,2.7821178E0,1.9604416E0,7.633156E0,3.1866672E0,4.9935656E0,3.5535326E0,2.8008423E0,2.4118037E0,1.3827915E0,2.8099203E0,2.2660732E-1,5.1472807E-1,3.2061648E0,9.204178E-1,0E0,1.3017219E0,4.932243E0,4.0392523E0,4.3831787E0,4.527049E-1,1.2209587E0,2.5041046E0,2.7246082E-1,1.7781001E0,9.585514E-1,1.2683907E0,9.600407E-1,1.052804E0,0E0,0E0,1.4703268E-1,0E0,1.458008E0,1.0999739E0,5.967531E-1,2.2831708E-1,4.8583865E-1,2.154803E-1,5.827966E-1,7.2120647E0,9.718244E-1,9.913044E-1,1.2820715E0,2.5250053E0,1.7617643E-1,0E0,9.925785E-1,3.7384105E-1,1.0803032E0,1.0377979E0,0E0,0E0,4.8544133E-1,5.6964755E-1,0E0,0E0,6.716881E-1,7.048819E-1,4.7385192E-1,1.8663853E-1,8.780365E-1,2.8985032E-1,0E0,0E0,9.3559504E-1,6.955166E-1,7.1752477E-1,4.9021372E-1,1.1861515E-1,6.8156815E-1,0E0,0E0,0E0,4.5620066E-1,0E0,1.3447323E-1,2.05338E-1,3.8416702E-1,2.53371E0,2.7655163E0,2.6265717E-1,3.001476E-1,0E0,0E0,1.2071288E0,8.277011E-1,2.2320747E0,4.806082E-1,0E0,0E0,3.8589573E-1,0E0,0E0,1.2692833E-1,3.2645E-1,8.054304E-1,7.3007315E-1,6.7705727E-1,4.7198218E-1,1.9515228E-1,4.6598947E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,59,59,60,60,63,63,64,64,65,65,66,66,67,67,68,68,71,71,72,72,73,73,74,74,75,75,76,76,80,80,82,82,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,97,97,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,96,-1,98,100,102,104,-1,-1,106,108,-1,-1,110,112,114,116,118,120,-1,-1,122,124,126,128,130,132,-1,-1,-1,134,-1,136,138,140,142,144,146,148,-1,-1,150,152,154,156,-1,-1,158,-1,-1,160,162,164,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.382114E0,8.32E3,6.747114E7,2.9283184E3,7.733016E5,3.5301748E7,1.146E3,6.567044E2,4.0844156E7,6.737509E7,1E0,4.898012E-1,1.1317E4,3.2343243E1,4.39688E-5,7.44E2,2.7504245E10,1.0828989E8,6.860185E2,9.408768E1,4.559558E-2,1.1855755E8,2.73E2,9.457831E0,8.31E2,1E0,6.4E1,2.1019447E-1,4.1043E4,9.256843E3,1.9523809E0,2.765625E0,2.695E3,6.057963E2,-4.2002924E-2,-1.387348E-2,9.6910744E7,-2.7231444E-2,1.6256282E7,9.397667E6,3.131E3,1.1349584E2,8.7643677E-1,2.7523365E0,1.8108038E5,8E0,4.2E1,1.752E3,1.198E5,1E0,1.6709303E1,6.2620286E-3,1.5548611E7,9.433E3,1.8777761E0,5.673198E5,-2.8320055E-2,-8.338294E-3,1E0,2.4507338E4,-7.1059936E-3,-4.0808868E-2,3.86569E5,5.2E1,1.2342778E1,5.5814706E5,1.1978022E0,2.3E1,-1.2855413E-2,-1.4231578E-3,1.1231086E1,1.9563605E5,4.9716983E0,1.1317E4,5.5759512E-2,6.2105E4,1.025269E-2,-5.9332848E-3,1.772871E-2,5.505334E9,8.637621E-4,2.109E3,1.0972222E0,3.1E1,1.0005E5,5.372E3,1E0,1.8796511E5,-4.3356832E-2,-4.319082E-3,1.08504E5,5.79E2,4.7646948E2,1.4521204E0,-2.0273535E-2,-8.089927E-3,4.728721E0,4.4758625E-2,5.9207836E-3,2.79495E-1,1.2204E4,1.6195753E9,1.314806E8,7.983011E7,2.3274304E2,2.6312E4,1.5988282E1,2.9750304E-2,-2.0808822E-2,-3.5871996E-3,1.1896432E-3,-1.2814185E-2,-4.351967E-3,-1.672296E-2,1.2292709E-2,-1.1229675E-4,-8.200372E-3,-2.1054527E-2,-1.03594195E-2,1.3547657E-3,-7.110817E-4,-1.1612182E-2,-5.4985033E-3,-1.5030434E-2,2.3009578E-2,7.4311094E-3,9.579997E-3,-1.1696282E-3,-1.2166969E-2,-3.0006858E-2,-1.4278742E-2,-2.8808668E-2,-6.5609803E-3,9.8274E-3,-4.3682167E-3,-2.0743106E-2,-1.5474554E-2,-9.761322E-3,2.0288483E-3,-1.0382548E-2,-1.7220875E-2,3.28266E-2,4.0469943E-3,-2.303593E-3,-1.4086619E-2,4.3822938E-4,-4.760197E-3,5.4710703E-3,4.2146565E-3,-2.587018E-2,-2.8679082E-3,5.6604687E-3,9.774477E-3,1.842159E-2,-1.6164748E-3,8.7346975E-3,2.6857993E-2,1.3355934E-2,2.0775834E-2,6.6963662E-3,2.833423E-3,8.859438E-3,1.2243242E-2,3.05354E-2,1.1947385E-2,-1.4830871E-2,1.9344917E-2,3.1805784E-2,-9.738684E-3,4.267631E-3,3.2484306E-3,1.6621247E-2,1.4479285E-2,-4.705413E-3],"split_indices":[20,56,2,45,52,28,45,2,52,45,7,6,27,32,56,42,0,5,45,52,56,0,45,2,54,2,89,11,35,10,4,54,54,2,52,0,0,45,0,45,45,2,56,57,53,28,18,6,29,7,6,58,0,45,2,41,32,0,0,102,4,0,0,1,11,54,28,53,8,0,0,54,28,54,9,27,10,0,0,0,5,0,0,53,10,5,29,68,33,0,0,7,29,52,38,0,0,38,0,0,38,2,7,1,44,56,2,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.685E3,8.46E2,1.839E3,3.88E2,4.58E2,1.461E3,3.78E2,3.51E2,3.7E1,4.01E2,5.7E1,9.72E2,4.89E2,3.01E2,7.7E1,1.6E2,1.91E2,1.9E1,1.8E1,3.05E2,9.6E1,5E0,5.2E1,8.84E2,8.8E1,4.72E2,1.7E1,8.7E1,2.14E2,8E0,6.9E1,1.2E1,1.48E2,1.01E2,9E1,1.7E1,2E0,1.2E1,6E0,2.23E2,8.2E1,8.9E1,7E0,4.3E1,9E0,1.3E2,7.54E2,7.6E1,1.2E1,1.42E2,3.3E2,1.5E1,2E0,6.1E1,2.6E1,1.27E2,8.7E1,4E0,4E0,4.5E1,2.4E1,3E0,9E0,9.6E1,5.2E1,8.9E1,1.2E1,7.2E1,1.8E1,8E0,4E0,1.5E2,7.3E1,3.2E1,5E1,1.6E1,7.3E1,2E0,5E0,1.8E1,2.5E1,4E0,5E0,1.07E2,2.3E1,2.1E1,7.33E2,4.4E1,3.2E1,1E1,2E0,1.1E1,1.31E2,2.84E2,4.6E1,8E0,7E0,5.3E1,8E0,9E0,1.7E1,9.1E1,3.6E1,1.3E1,7.4E1,2.2E1,2.3E1,1.9E1,5E0,9E1,6E0,1E1,4.2E1,8.2E1,7E0,8E0,4E0,1.8E1,5.4E1,9E0,9E0,1.3E2,2E1,3.6E1,3.7E1,1E1,2.2E1,1.2E1,3.8E1,3E0,1.3E1,6.4E1,9E0,4E0,2.1E1,3E0,2E0,2.2E1,8.5E1,1.5E1,8E0,2E0,1.9E1,2.27E2,5.06E2,4.1E1,3E0,2.4E1,8E0,5E0,6E0,3.5E1,9.6E1,1.54E2,1.3E2,3.2E1,1.4E1,4.4E1,9E0,1.4E1,3E0,3.4E1,5.7E1,2.9E1,7E0,1.1E1,2E0,5.8E1,1.6E1,1.4E1,8E0,2.1E1,2E0,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-1.537491E-3,6.8239585E-2,-2.3639442E-1,-5.7615686E-2,1.6996337E-1,-3.746522E-1,-4.2344075E-2,-1.1156281E-1,8.400984E-2,4.9860086E-2,2.770806E-1,-5.7519734E-1,-3.3568943E-1,-8.887326E-2,1.8491934E-1,-1.4450718E-1,6.343844E-2,-2.271651E-1,1.070941E-1,9.3518734E-2,-2.1849242E-1,2.4747835E-1,5.323474E-1,-7.144298E-1,-4.741331E-1,-7.5807554E-1,-3.095004E-1,-5.172515E-2,-3.6764398E-1,2.5746548E-1,-2.3531961E-1,-1.1564013E-1,-3.2595497E-1,1.7792185E-1,-1.9342387E-2,6.2915278E-3,-4.1738197E-1,6.572452E-2,2.2413751E-1,1.977243E-1,-1.8016933E-2,-2.8789404E-1,8.293377E-2,2.0863147E-1,4.3856636E-1,5.845373E-1,1.069984E-1,-4.1911047E-2,-1.8440304E-2,-4.8672482E-3,-4.9305567E-1,-4.5865297E-2,-4.1377163E-1,-1.7508686E-1,-3.642573E-1,-1.0363793E-1,5.209747E-2,-4.9183795E-1,-4.225204E-2,2.875755E-1,-3.3697519E-3,-2.8361029E-3,-1.6885666E-2,-1.7697051E-1,-4.5807354E-2,-7.9863715E-1,-2.2640385E-1,2.1616499E-1,-5.1092533E-3,-1.2240728E-1,4.7220808E-2,-7.829505E-3,-5.2389324E-1,-3.9004873E-2,1.2343719E-1,8.13886E-2,2.9017538E-1,1.0389114E-1,3.5968548E-1,-1.702875E-1,4.0587112E-2,-1.8415479E-2,-1.7885107E-1,1.1892553E-2,-2.6301926E-3,9.0585425E-2,2.6206723E-1,6.4369905E-1,3.3223495E-1,9.816511E-3,6.268307E-1,-4.348767E-3,1.1677751E-2,-2.4563512E-2,-5.5998266E-3,-3.7726574E-3,-2.4534665E-2,-3.0466324E-1,-9.1103695E-2,-7.287307E-1,-3.20442E-1,-2.2872247E-1,-7.812004E-2,2.0821896E-1,-8.802197E-3,-6.0429436E-1,-1.0957017E-2,-6.612545E-3,4.2187353E-3,2.0904373E-1,4.0439516E-1,-7.2786696E-3,-1.3235975E-2,2.9199202E-2,-2.878566E-3,8.715093E-5,-4.5224957E-2,-6.868306E-4,-1.3616994E-2,4.2539425E-3,1.3841089E-2,-1.3838199E-3,-1.1436041E-2,-6.411396E-4,6.948284E-3,-2.9958751E-2,-6.9879084E-3,8.202395E-3,-4.112801E-3,7.0046107E-3,-1.511462E-2,-3.0721114E-3,8.423202E-3,1.6643105E-2,4.6436493E-3,1.3614329E-2,1.0534972E-3,3.3336535E-2,1.2306028E-2,-9.242589E-3,1.6674273E-2,5.562477E-4,1.0874078E-2,-1.1041956E-2,1.5036275E-3,5.1610977E-3,-1.5311847E-2,8.601215E-3,1.53613705E-2,3.4646858E-2,1.8064663E-2,1.2750096E-2,4.328682E-2,3.0667381E-2,9.858946E-3,-8.018865E-3,-1.7963847E-2,8.80625E-3,-5.918266E-3,-1.0597752E-2,-3.83608E-2,-1.3183085E-2,-2.552682E-2,-1.2210845E-2,6.5985403E-4,-9.302131E-4,-6.1906786E-3,1.3772639E-2,3.4989864E-3,-6.5293247E-3,1.6260952E-3,-3.294881E-2,-5.930562E-3,1.2114551E-3,1.2388291E-2,2.4783622E-2,8.916356E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,-1,-1,-1,93,-1,95,97,99,101,103,105,107,109,-1,-1,-1,111,113,115,117,119,-1,121,123,-1,125,127,129,131,133,135,137,139,141,-1,143,-1,-1,145,147,149,151,-1,153,-1,-1,-1,-1,-1,-1,155,157,159,161,163,165,167,169,171,-1,-1,-1,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4000534E1,2.6510914E1,1.6483955E1,7.08428E0,1.4716763E1,2.6120605E0,2.7340508E0,3.8776875E0,1.8632387E0,6.358739E0,4.4399185E0,4.6333885E-1,3.1171265E0,2.197126E0,1.4020021E0,2.9216986E0,1.0174838E0,1.2824328E0,1.1425157E0,5.4197116E0,1.6137555E0,3.9467087E0,1.3137875E0,8.2760334E-1,2.0364094E-1,7.7222633E-1,2.048563E0,1.0294461E0,9.895117E-1,3.9533067E-1,1.2367651E-1,2.0862179E0,3.442501E0,5.128554E-1,4.3798086E-1,0E0,2.3886156E-1,1.0806847E0,5.608468E-1,3.6251879E0,2.0236614E0,5.8456755E-1,3.556436E-1,2.8400383E0,1.7710361E0,7.544193E-1,2.3868991E-1,0E0,0E0,0E0,1.9392204E-1,0E0,2.237134E-1,8.972342E-1,3.0082932E0,3.8750255E-1,6.1032856E-1,3.9420176E-1,1.120836E-1,2.3655581E-1,0E0,0E0,0E0,6.037655E-1,2.1926115E0,1.5178947E0,7.9445815E-1,3.4539318E-1,0E0,2.6519156E-1,2.3327088E-1,0E0,2.1616602E-1,6.4553595E-1,1.1986372E0,3.0008873E-1,4.31679E-1,2.2687297E0,2.8622036E0,8.08862E-1,8.8338053E-1,0E0,3.443048E-1,0E0,0E0,1.0426731E0,1.4525166E0,3.6476898E-1,2.098517E0,0E0,1.4732933E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.5616264E-1,4.9602896E-1,5.9296894E-1,1.6103668E0,1.3719988E-1,3.187195E-1,1.7207384E-1,2.609213E-1,3.9741993E-1,0E0,0E0,0E0,2.0322573E-1,2.2451806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,63,63,64,64,65,65,66,66,67,67,69,69,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,82,82,85,85,86,86,87,87,88,88,90,90,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,-1,-1,-1,94,-1,96,98,100,102,104,106,108,110,-1,-1,-1,112,114,116,118,120,-1,122,124,-1,126,128,130,132,134,136,138,140,142,-1,144,-1,-1,146,148,150,152,-1,154,-1,-1,-1,-1,-1,-1,156,158,160,162,164,166,168,170,172,-1,-1,-1,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7810526E2,1E0,7.733016E5,1.4887473E-1,1.5804776E4,3.0497742E3,7.997723E6,5.8631687E1,5.7581736E7,4.391553E6,1.381869E2,6E0,8.672049E9,1.1855755E8,9.669789E0,2.25E2,3.0039525E-1,1E0,1.0238709E1,4.6726016E3,3.970405E3,4.728721E0,4.6547272E2,1.8108038E5,1.7997802E0,1.2E1,6.9664386E2,1.2151E4,1.2428079E1,2.603E3,1.4735735E2,2E0,2.0783027E5,1.9987492E5,6.2915278E-3,1.3860265E6,2.4757753E2,1E0,9.557794E2,7.44393E5,1.884E3,5.441732E9,2.2783158E5,3.2475834E7,1.752E3,8E0,-4.1911047E-2,-1.8440304E-2,-4.8672482E-3,1.8302105E6,-4.5865297E-2,2.1132307E0,3.9314518E1,1.2068E4,1.06E3,1.18827E5,8.977316E0,1E0,8.63558E-1,-3.3697519E-3,-2.8361029E-3,-1.6885666E-2,5.8036E4,2.04E5,8E0,2.4558064E2,1.5687E4,-5.1092533E-3,3.0273972E0,1.70244E7,-7.829505E-3,4.38E2,1.1779856E2,2.507E3,1.9925156E6,1.3581108E7,8E0,9.653717E3,6.071E3,1.5963264E9,-1.8415479E-2,4.6E1,1.1892553E-2,-2.6301926E-3,3.0161016E7,3.83E3,3.275343E6,4.2482185E0,9.816511E-3,2E1,-4.348767E-3,1.1677751E-2,-2.4563512E-2,-5.5998266E-3,-3.7726574E-3,-2.4534665E-2,3.512733E-1,1E0,1.7848537E0,2.72258E5,6.1E1,5.68E2,6.722251E1,5.378E3,6E1,-1.0957017E-2,-6.612545E-3,4.2187353E-3,1E0,1.0057E4,-7.2786696E-3,-1.3235975E-2,2.9199202E-2,-2.878566E-3,8.715093E-5,-4.5224957E-2,-6.868306E-4,-1.3616994E-2,4.2539425E-3,1.3841089E-2,-1.3838199E-3,-1.1436041E-2,-6.411396E-4,6.948284E-3,-2.9958751E-2,-6.9879084E-3,8.202395E-3,-4.112801E-3,7.0046107E-3,-1.511462E-2,-3.0721114E-3,8.423202E-3,1.6643105E-2,4.6436493E-3,1.3614329E-2,1.0534972E-3,3.3336535E-2,1.2306028E-2,-9.242589E-3,1.6674273E-2,5.562477E-4,1.0874078E-2,-1.1041956E-2,1.5036275E-3,5.1610977E-3,-1.5311847E-2,8.601215E-3,1.53613705E-2,3.4646858E-2,1.8064663E-2,1.2750096E-2,4.328682E-2,3.0667381E-2,9.858946E-3,-8.018865E-3,-1.7963847E-2,8.80625E-3,-5.918266E-3,-1.0597752E-2,-3.83608E-2,-1.3183085E-2,-2.552682E-2,-1.2210845E-2,6.5985403E-4,-9.302131E-4,-6.1906786E-3,1.3772639E-2,3.4989864E-3,-6.5293247E-3,1.6260952E-3,-3.294881E-2,-5.930562E-3,1.2114551E-3,1.2388291E-2,2.4783622E-2,8.916356E-3],"split_indices":[19,52,59,28,38,51,52,9,47,45,28,47,3,5,45,54,10,53,101,53,4,52,38,52,28,53,18,52,9,57,0,52,17,33,47,0,28,55,16,52,1,10,5,28,50,29,8,0,0,0,32,0,53,58,9,2,29,54,102,27,0,0,0,1,5,0,4,1,0,53,9,0,0,4,2,28,12,17,50,2,7,0,3,0,0,45,2,29,39,0,3,0,0,0,0,0,0,27,64,54,9,3,0,58,2,3,0,0,0,102,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,2.069E3,6.14E2,9.25E2,1.144E3,3.58E2,2.56E2,6.7E2,2.55E2,5.4E2,6.04E2,5.6E1,3.02E2,2.13E2,4.3E1,5.64E2,1.06E2,1.7E1,2.38E2,4.65E2,7.5E1,5.43E2,6.1E1,2.1E1,3.5E1,1.6E1,2.86E2,1.89E2,2.4E1,3.7E1,6E0,4.88E2,7.6E1,4.4E1,6.2E1,6E0,1.1E1,1.77E2,6.1E1,2.4E2,2.25E2,6.1E1,1.4E1,4.53E2,9E1,5.4E1,7E0,1.3E1,8E0,2E0,3.3E1,9E0,7E0,8.4E1,2.02E2,1.26E2,6.3E1,1.7E1,7E0,3.4E1,3E0,3E0,3E0,2.59E2,2.29E2,1.2E1,6.4E1,3.9E1,5E0,2.4E1,3.8E1,4E0,7E0,6.3E1,1.14E2,2E1,4.1E1,1.53E2,8.7E1,6.2E1,1.63E2,3.1E1,3E1,6E0,8E0,1.42E2,3.11E2,2.9E1,6.1E1,6E0,4.8E1,3E0,4E0,3.1E1,2E0,2E0,5E0,3.2E1,5.2E1,2E1,1.82E2,2E1,1.06E2,1.7E1,4.6E1,1.1E1,6E0,4E0,3E0,2.2E1,1.2E1,2.09E2,5E1,4E0,2.25E2,2E0,1E1,1.4E1,5E1,1.5E1,2.4E1,1.4E1,1E1,2.4E1,1.4E1,5E0,2E0,1.1E1,5.2E1,1.09E2,5E0,8E0,1.2E1,3.1E1,1E1,4.7E1,1.06E2,1.9E1,6.8E1,6E1,2E0,1.42E2,2.1E1,2.4E1,6E0,1.37E2,5E0,1.31E2,1.8E2,2.1E1,8E0,5.6E1,5E0,4.6E1,2E0,1.2E1,2E1,5E0,4.7E1,3E0,1.7E1,1.52E2,3E1,1.8E1,2E0,5E1,5.6E1,1E1,7E0,1.1E1,3.5E1,9E0,2E0,5E0,1.7E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[2.8231882E-3,6.418889E-2,-2.194715E-1,1.12586515E-2,2.538697E-1,-3.312235E-1,-6.4612046E-2,-5.092457E-2,1.5077528E-1,3.2521424E-1,4.441304E-2,-5.233784E-1,-2.8863823E-1,-1.3849673E-1,5.2404385E-2,-9.6434295E-2,4.1458823E-2,1.9353819E-1,9.043456E-4,4.880579E-1,2.3524341E-1,-4.0031947E-2,2.4925189E-1,-5.828715E-1,-2.1078362E-1,-3.0565947E-1,1.0846398E-1,-1.05985194E-1,-4.679638E-1,1.3483164E-1,-2.4054901E-1,-7.664969E-2,-3.2333928E-1,-1.9504818E-1,9.0468526E-2,7.910176E-2,2.7836934E-1,4.593802E-2,-2.5345665E-1,6.1077523E-1,3.002827E-1,1.24052174E-1,3.6632046E-1,-3.241094E-1,1.7355429E-2,2.9335463E-1,-7.4162004E-3,-1.1665747E-2,-6.531196E-1,-4.694509E-3,-1.5635677E-2,-2.2150363E-1,-3.8261747E-1,1.4311201E-2,-3.3620608E-3,-1.9982952E-1,-5.8931064E-2,-2.9841432E-2,-1.4036863E-2,-4.783056E-4,1.8845405E-1,-1.4084247E-1,-2.545102E-2,-1.0054427E-1,2.2611235E-1,-4.043859E-1,-1.0357394E-2,-1.3020298E-1,-3.8321268E-2,1.8889357E-1,4.6352413E-2,-3.1877365E-2,1.3537842E-1,3.2344848E-1,5.039328E-2,-7.822408E-2,1.0924132E-1,-3.2374144E-1,2.6772846E-3,4.5356223E-1,7.882047E-1,2.1442097E-2,1.9197415E-1,7.5913206E-2,2.1495621E-1,5.5645907E-1,2.883725E-1,-5.6801313E-3,-2.2192989E-2,6.804548E-2,-1.9026518E-1,1.4181435E-1,4.6723825E-1,-5.7816416E-1,-5.0868127E-2,-3.4390637E-1,-1.7536472E-1,-4.6278292E-1,-2.523147E-1,-1.2634554E-2,-1.209369E-1,-9.589752E-2,7.0617296E-2,1.9543554E-1,-1.04933806E-1,1.1726349E-1,1.3952951E-2,-2.529828E-1,8.111831E-4,-5.649903E-3,3.3903683E-3,3.207168E-2,4.865177E-3,-2.046667E-2,-6.2715844E-3,4.1759796E-3,-7.950544E-3,1.9628948E-2,6.974768E-3,-4.409762E-4,6.2632128E-3,7.926493E-3,-5.091463E-3,5.5325073E-3,2.220092E-2,1.251162E-2,2.1536443E-2,-2.4689494E-3,8.589683E-3,5.711626E-3,-5.2476726E-3,-7.939319E-3,6.731021E-3,-7.2629643E-3,-2.077717E-2,1.1725272E-2,2.385394E-2,3.9952338E-2,1.7851202E-2,-9.963025E-4,1.360835E-2,4.671612E-3,-9.921493E-3,1.3550245E-2,5.8112345E-3,1.8949054E-2,4.0799376E-2,1.4899093E-2,-1.6560933E-2,4.6695487E-4,2.0300016E-2,-5.1410017E-3,-1.77539E-2,9.432625E-3,1.29381E-3,2.457955E-2,6.1044954E-3,-2.9950356E-2,-1.1388044E-2,-1.7730225E-2,2.4639869E-3,-6.0007675E-3,-1.6081035E-2,-2.7156882E-2,-1.5220247E-2,-1.6891837E-2,-5.8301245E-3,-8.438276E-3,1.0849E-3,2.974882E-3,-6.155658E-3,6.7558414E-3,-2.9452348E-3,2.5408536E-3,1.5542958E-2,3.339106E-3,-7.394705E-3,1.3534941E-2,2.0821819E-3,-1.1062532E-3,-1.6675686E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,-1,93,-1,-1,95,97,-1,-1,99,101,-1,-1,103,105,107,-1,109,111,113,-1,115,-1,117,119,121,123,125,127,129,131,133,-1,135,137,-1,139,141,143,145,147,-1,-1,149,151,153,155,157,-1,159,161,163,165,-1,167,169,171,173,175,177,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6067963E1,2.0799715E1,9.877546E0,1.4070814E1,6.740059E0,2.5656776E0,2.090877E0,4.7212706E0,3.2042837E0,4.8226357E0,2.0165014E0,9.743767E-1,1.8851662E0,1.5357018E0,2.302284E0,3.3511138E0,4.3179584E0,3.754302E0,1.294582E0,2.5587444E0,3.1463976E0,1.3587292E0,6.51387E-1,1.0087605E0,1.1150786E-1,1.6185608E0,4.2912754E-1,5.8876765E-1,1.700561E-1,5.370077E-1,5.3423214E-1,5.0361543E0,4.3318796E-1,2.4071856E0,1.3286204E0,1.0473197E0,2.275465E0,7.6309294E-1,3.8773572E-1,1.6229916E0,6.9657516E-1,5.0880516E-1,1.3216238E0,3.4581196E-1,7.496476E-1,7.506006E-1,0E0,0E0,6.1232567E-1,0E0,0E0,6.6257477E-1,1.3038578E0,0E0,0E0,1.952765E-1,4.473548E-1,0E0,0E0,4.7072193E-1,3.540752E-1,3.050761E-1,0E0,1.9287453E0,2.7489617E0,1.5136576E-1,0E0,4.7382498E-1,0E0,8.483274E-1,1.0050641E0,8.5510725E-1,6.697409E-1,1.3430157E0,5.088664E-1,2.1739514E-1,5.7437956E-1,1.9946742E-1,0E0,2.4984074E-1,3.0109215E-1,0E0,5.9433496E-1,5.1015836E-1,2.266028E-1,9.931154E-1,1.1600943E0,0E0,0E0,1.1782517E0,1.6967851E-1,1.086348E-1,1.5341926E-1,4.29986E-1,0E0,3.7281275E-1,7.208929E-1,1.0745125E0,6.650131E-1,0E0,1.7856503E-1,3.7931722E-1,2.0413858E-1,1.2984943E-1,1.4038283E-1,3.9034855E-1,0E0,2.1085465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,51,51,52,52,55,55,56,56,59,59,60,60,61,61,63,63,64,64,65,65,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,82,82,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,95,95,96,96,97,97,98,98,100,100,101,101,102,102,103,103,104,104,105,105,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,-1,94,-1,-1,96,98,-1,-1,100,102,-1,-1,104,106,108,-1,110,112,114,-1,116,-1,118,120,122,124,126,128,130,132,134,-1,136,138,-1,140,142,144,146,148,-1,-1,150,152,154,156,158,-1,160,162,164,166,-1,168,170,172,174,176,178,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,7.431E3,1E0,7.733016E5,2.8091298E7,8.230099E4,1.0323588E3,1.5107028E-1,2E0,1.4232318E3,6.082581E3,1E0,3.970405E3,8.672049E9,9.6910744E7,4.0844156E7,5.08E2,1.06E3,4.898012E-1,8.606353E1,2.3292E4,2.308943E0,1.56E2,5.020181E-1,2.5E1,8.15E2,1.8140173E-1,5.124411E2,6.3300834E0,1E0,1.4299594E8,1.1210787E3,6.4587574E-3,4.5184364E0,8E0,1E0,1E0,5.033165E7,2.4519731E3,1.6503105E3,5.2610065E1,3.5432583E3,1.417988E6,1.2679E4,1.066317E12,1.3674345E4,-7.4162004E-3,-1.1665747E-2,7.0277144E7,-4.694509E-3,-1.5635677E-2,9.38E2,5.676415E-1,1.4311201E-2,-3.3620608E-3,1.8631586E4,8.0708375E5,-2.9841432E-2,-1.4036863E-2,1E0,1.3137E4,7.023838E3,-2.545102E-2,1.2817779E7,1.2E2,1.5324128E6,-1.0357394E-2,2.7913043E0,-3.8321268E-2,3.5E1,3.131E3,5E-1,7.064848E6,3.01E2,8.677148E2,1.07E2,1.7956694E-4,9.9625344E5,2.6772846E-3,1E0,1.766486E7,2.1442097E-2,2.7565938E3,2.1690162E7,8.5837976E7,1.921032E4,1.3144558E4,-5.6801313E-3,-2.2192989E-2,7.023838E3,7.644773E-1,2.6100147E5,1.1159378E12,2.5167784E-1,-5.0868127E-2,1.593E2,1.9041E4,2.0939393E0,6.057963E2,-1.2634554E-2,2.2927427E-1,1.8466014E-1,6.4888096E0,8.35338E-1,2.57636E6,6.5346925E6,1.3952951E-2,1.19839E5,8.111831E-4,-5.649903E-3,3.3903683E-3,3.207168E-2,4.865177E-3,-2.046667E-2,-6.2715844E-3,4.1759796E-3,-7.950544E-3,1.9628948E-2,6.974768E-3,-4.409762E-4,6.2632128E-3,7.926493E-3,-5.091463E-3,5.5325073E-3,2.220092E-2,1.251162E-2,2.1536443E-2,-2.4689494E-3,8.589683E-3,5.711626E-3,-5.2476726E-3,-7.939319E-3,6.731021E-3,-7.2629643E-3,-2.077717E-2,1.1725272E-2,2.385394E-2,3.9952338E-2,1.7851202E-2,-9.963025E-4,1.360835E-2,4.671612E-3,-9.921493E-3,1.3550245E-2,5.8112345E-3,1.8949054E-2,4.0799376E-2,1.4899093E-2,-1.6560933E-2,4.6695487E-4,2.0300016E-2,-5.1410017E-3,-1.77539E-2,9.432625E-3,1.29381E-3,2.457955E-2,6.1044954E-3,-2.9950356E-2,-1.1388044E-2,-1.7730225E-2,2.4639869E-3,-6.0007675E-3,-1.6081035E-2,-2.7156882E-2,-1.5220247E-2,-1.6891837E-2,-5.8301245E-3,-8.438276E-3,1.0849E-3,2.974882E-3,-6.155658E-3,6.7558414E-3,-2.9452348E-3,2.5408536E-3,1.5542958E-2,3.339106E-3,-7.394705E-3,1.3534941E-2,2.0821819E-3,-1.1062532E-3,-1.6675686E-2],"split_indices":[19,2,59,28,45,32,52,38,17,48,52,16,52,5,45,45,2,2,27,58,2,54,8,27,8,0,38,4,53,102,45,52,42,53,17,102,6,7,4,52,56,4,29,2,31,4,0,0,12,0,0,2,42,0,0,33,28,0,0,8,2,4,0,9,0,32,0,54,0,0,2,53,28,0,55,0,38,28,0,102,47,0,4,45,32,4,52,0,0,4,34,28,31,58,0,52,9,53,52,0,38,27,53,27,1,45,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.642E3,2.071E3,5.71E2,1.62E3,4.51E2,3.31E2,2.4E2,1.121E3,4.99E2,3.36E2,1.15E2,5.8E1,2.73E2,1.47E2,9.3E1,7.51E2,3.7E2,3.88E2,1.11E2,1.18E2,2.18E2,8.2E1,3.3E1,4.8E1,1E1,2.62E2,1.1E1,1.35E2,1.2E1,7.3E1,2E1,6.92E2,5.9E1,6.3E1,3.07E2,1.66E2,2.22E2,9.5E1,1.6E1,7E1,4.8E1,1.19E2,9.9E1,1.3E1,6.9E1,3E1,3E0,9E0,3.9E1,6E0,4E0,1.27E2,1.35E2,5E0,6E0,4.4E1,9.1E1,5E0,7E0,2.1E1,5.2E1,1.6E1,4E0,6.42E2,5E1,3.2E1,2.7E1,5.8E1,5E0,9.4E1,2.13E2,5.6E1,1.1E2,1.85E2,3.7E1,3.2E1,6.3E1,1.3E1,3E0,3.9E1,3.1E1,1.9E1,2.9E1,7.9E1,4E1,2.7E1,7.2E1,6E0,7E0,5.6E1,1.3E1,1.7E1,1.3E1,3.5E1,4E0,3.3E1,9.4E1,8.2E1,5.3E1,2.3E1,2.1E1,7.1E1,2E1,7E0,1.4E1,3.2E1,2E1,9E0,7E0,5.83E2,5.9E1,1E1,4E1,2.9E1,3E0,8E0,5E1,1.4E1,8E1,1.29E2,8.4E1,1.5E1,4.1E1,1.05E2,5E0,1.26E2,5.9E1,2.1E1,1.6E1,4E0,2.8E1,6E0,5.7E1,6E0,7E0,8E0,3.1E1,2.7E1,4E0,9E0,2E1,7.4E1,5E0,2.2E1,1.8E1,1.9E1,8E0,7E1,2E0,4.9E1,7E0,1E1,3E0,1.1E1,6E0,1.1E1,2E0,3E1,5E0,3.1E1,2E0,7.3E1,2.1E1,4.6E1,3.6E1,2.9E1,2.4E1,1.5E1,6E0,1.2E1,5.9E1,1.3E1,7E0,4E0,3E0,3E0,1.1E1,9E0,2.3E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[1.5107727E-3,6.2422026E-2,-2.1350959E-1,-1.00000445E-2,1.9007328E-1,-3.328192E-1,-4.5204706E-2,-1.5767956E-2,4.4709764E-2,1.5834345E-1,4.0813473E-1,-6.568949E-1,-3.028608E-1,-1.1140232E-1,6.276351E-2,-3.72615E-2,1.6657418E-1,1.2049971E-1,3.845171E-1,9.0460374E-4,4.3923622E-1,-1.9046584E-2,-3.729672E-2,-3.334063E-1,-7.223023E-2,-8.367229E-2,-4.2563576E-1,-1.0553695E-1,1.8096268E-1,-5.9970755E-2,1.6194424E-1,1.7866638E-1,-2.0934422E-2,1.4383371E-1,-1.7838275E-1,5.064117E-1,2.4896368E-1,4.9928537E-1,1.6260529E-1,-2.9447743E-1,-6.8295217E-1,2.1391213E-1,-1.5566526E-1,-2.2162172E-1,-5.32205E-2,-2.2608304E-2,-6.1746906E-3,-6.267876E-2,-2.2263179E-2,-1.3083662E-3,2.119064E-1,-1.0342472E-1,1.6715804E-2,9.091881E-2,2.7829954E-1,2.4257691E-1,6.0242068E-2,5.371444E-2,1.9908185E-1,-2.0731106E-1,2.4917552E-2,3.1085178E-1,6.133859E-1,3.2134047E-1,4.051953E-2,4.334938E-1,3.3444386E-2,2.2594444E-1,-4.7762725E-3,-2.3382933E-1,-3.779528E-1,-1.7781088E-2,-4.3462228E-2,1.641716E-2,2.0226995E-3,-1.6774682E-2,-2.4732706E-1,-1.1672439E-2,1.0421706E-3,-1.0437492E-2,-3.732832E-2,3.743706E-3,-1.19858816E-1,1.2219479E-2,6.0899656E-2,-5.7840534E-3,3.4516363E-3,3.7767633E-3,-3.1605663E-3,5.2367053E-3,-9.225471E-3,1.687312E-2,6.496214E-3,7.6703634E-3,1.8991891E-2,4.309809E-3,-1.0515035E-2,4.215987E-3,-9.035498E-3,2.998568E-3,1.2380063E-2,-1.6155442E-2,-5.231133E-3,-4.7013615E-3,8.655112E-3,1.7601652E-2,-5.459258E-3,1.9979272E-2,3.4529496E-2,-8.4089884E-4,1.6986944E-2,-2.368349E-3,9.3083335E-3,2.1377819E-2,2.7239432E-3,-8.5398984E-5,1.2645383E-2,-1.5937015E-2,-5.690159E-3,-2.2719162E-2,-1.4334592E-2,3.0329346E-3,-7.547988E-3,-1.4504537E-2,-4.107378E-3,3.7144008E-3,-3.6910535E-3,-7.7957194E-3,4.45263E-3,-2.0412346E-3,6.749374E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,79,-1,-1,81,-1,-1,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,115,-1,117,119,-1,-1,-1,-1,121,123,-1,-1,-1,125,-1,127,-1,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.458983E1,1.9027634E1,1.1691822E1,7.1661654E0,5.090067E0,3.1117516E0,1.743771E0,5.128574E0,0E0,5.5343018E0,1.1296806E0,5.7488537E-1,2.1879597E0,1.280155E0,1.8691449E0,5.3025746E0,1.0855181E0,3.9311676E0,1.4105825E0,0E0,1.364008E0,0E0,0E0,3.568142E0,9.40522E-1,5.7631826E-1,1.04504585E-1,5.7148695E-1,3.6026645E-1,3.5040128E0,9.6639943E-1,1.0147104E0,0E0,2.5783453E0,2.4629211E-1,8.134823E-1,6.80959E-1,6.2714577E-1,3.0501118E-1,1.1823425E0,1.5528879E0,1.8467337E-1,3.777799E-1,1.3983357E-1,2.9553246E-1,0E0,0E0,2.9930863E-1,0E0,0E0,2.979362E-1,2.052042E0,1.9636341E0,4.1820425E-1,4.274156E-1,1.0532775E0,4.2765802E-1,1.6719861E0,2.5951529E0,4.222721E-1,1.3280997E-1,4.9948335E-1,3.3267307E-1,3.906572E-1,1.8955931E-1,2.4657917E-1,0E0,1.21950984E-1,0E0,1.6447983E0,6.467123E-1,0E0,0E0,0E0,0E0,1.5664203E-1,1.3554454E-1,0E0,0E0,0E0,4.9455845E-1,0E0,2.5265825E-1,0E0,1.0543051E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,69,69,70,70,75,75,76,76,80,80,82,82,84,84],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,80,-1,-1,82,-1,-1,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,116,-1,118,120,-1,-1,-1,-1,122,124,-1,-1,-1,126,-1,128,-1,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.0950326E6,1E0,7.3610186E3,3.2196458E6,6.9121815E-2,1.067536E3,2.065904E6,4.4709764E-2,4.4839956E3,5.6553E4,1.2920592E0,1.1879E4,1.2199979E10,1E0,6.28205E8,4.4E1,4.2343444E7,2.5003334E7,9.0460374E-4,2.3487206E7,-1.9046584E-2,-3.729672E-2,1.0017953E8,1E0,1.06E3,1.2151E4,1.4299594E8,2.4418726E-3,1.273801E6,1.5435694E9,1.7463945E4,-2.0934422E-2,5.862126E2,3.202814E5,2.79495E-1,3.0990322E5,1.7921995E3,2.0288463E0,8.73E2,1.992945E5,1.447E3,4.928836E9,6.1E1,1.9646037E-4,-2.2608304E-2,-6.1746906E-3,1.4302E4,-2.2263179E-2,-1.3083662E-3,3.4570816E0,3.402546E6,5.107429E1,1E0,9.397667E6,1.048E3,2.84755E5,5.024605E0,1.5166431E5,1.7495675E5,4.286E3,5.68E2,6.7652373E3,5.0279167E1,4.8E1,5.6E1,3.3444386E-2,2.612E3,-4.7762725E-3,1.421E3,3.0608664E2,-1.7781088E-2,-4.3462228E-2,1.641716E-2,2.0226995E-3,1.16E2,3.970405E3,-1.1672439E-2,1.0421706E-3,-1.0437492E-2,5.2879925E5,3.743706E-3,6.9309726E0,1.2219479E-2,3.633605E6,-5.7840534E-3,3.4516363E-3,3.7767633E-3,-3.1605663E-3,5.2367053E-3,-9.225471E-3,1.687312E-2,6.496214E-3,7.6703634E-3,1.8991891E-2,4.309809E-3,-1.0515035E-2,4.215987E-3,-9.035498E-3,2.998568E-3,1.2380063E-2,-1.6155442E-2,-5.231133E-3,-4.7013615E-3,8.655112E-3,1.7601652E-2,-5.459258E-3,1.9979272E-2,3.4529496E-2,-8.4089884E-4,1.6986944E-2,-2.368349E-3,9.3083335E-3,2.1377819E-2,2.7239432E-3,-8.5398984E-5,1.2645383E-2,-1.5937015E-2,-5.690159E-3,-2.2719162E-2,-1.4334592E-2,3.0329346E-3,-7.547988E-3,-1.4504537E-2,-4.107378E-3,3.7144008E-3,-3.6910535E-3,-7.7957194E-3,4.45263E-3,-2.0412346E-3,6.749374E-3],"split_indices":[19,47,59,4,28,58,52,28,0,52,1,56,2,5,102,7,3,45,45,0,50,0,0,45,64,2,9,45,39,9,7,50,0,52,28,38,28,4,53,0,33,0,12,3,42,0,0,10,0,0,57,1,48,74,45,2,1,53,28,33,0,3,52,56,3,8,0,2,0,2,52,0,0,0,0,8,52,0,0,0,48,0,53,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.639E3,2.057E3,5.82E2,1.313E3,7.44E2,3.4E2,2.42E2,1.306E3,7E0,6.51E2,9.3E1,2.7E1,3.13E2,1.5E2,9.2E1,1.169E3,1.37E2,5.59E2,9.2E1,7E0,8.6E1,1E1,1.7E1,2.76E2,3.7E1,1.39E2,1.1E1,3.8E1,5.4E1,1.05E3,1.19E2,1.35E2,2E0,5.19E2,4E1,4.7E1,4.5E1,7E1,1.6E1,2.5E2,2.6E1,8E0,2.9E1,2.4E1,1.15E2,9E0,2E0,3.5E1,3E0,7E0,4.7E1,6.7E2,3.8E2,7.5E1,4.4E1,8.7E1,4.8E1,1.98E2,3.21E2,3.5E1,5E0,1.8E1,2.9E1,3.3E1,1.2E1,5.5E1,1.5E1,1.3E1,3E0,1.47E2,1.03E2,1.2E1,1.4E1,4E0,4E0,1.2E1,1.7E1,2.2E1,2E0,9E0,1.06E2,1E1,2.5E1,3.6E1,1.1E1,6.1E2,6E1,2.17E2,1.63E2,7.1E1,4E0,2.8E1,1.6E1,5.8E1,2.9E1,4.4E1,4E0,1.74E2,2.4E1,9.8E1,2.23E2,1.4E1,2.1E1,3E0,2E0,1.6E1,2E0,1.2E1,1.7E1,3E0,3E1,8E0,4E0,5.3E1,2E0,2E0,1.1E1,7.8E1,6.9E1,4.4E1,5.9E1,8E0,4E0,1.2E1,5E0,2.7E1,7.9E1,2.1E1,4E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-5.3433225E-5,-1.5606336E-1,7.3864155E-2,-2.7044863E-1,-6.729937E-2,1.4702409E-2,2.221406E-1,-1.8964024E-1,-3.4172866E-1,6.9088924E-1,-8.146373E-2,-9.232414E-2,8.218058E-2,1.5090178E-1,3.4294137E-1,-2.3640485E-1,1.3869165E-3,-3.8782865E-1,-9.947628E-2,7.990749E-3,3.9115895E-2,-1.6133048E-1,4.0026207E-2,-1.1340608E-1,2.1073145E-1,7.3953636E-2,3.799462E-2,1.0658173E-1,2.8640768E-1,1.4029399E-2,3.7708235E-1,-3.2219544E-1,-1.4733085E-1,-3.2927968E-2,8.546034E-3,-3.2338172E-1,-6.2852305E-1,2.9404778E-2,-1.847192E-1,-6.3770436E-2,-2.2038229E-1,-2.2137025E-2,1.138762E-1,-2.2129531E-1,-6.7894E-2,3.100706E-2,1.09621294E-1,1.124494E-1,-8.148124E-2,1.6623518E-1,1.6632285E-2,3.603062E-1,-3.1705517E-2,8.196518E-3,-1.8811304E-3,3.427386E-1,3.3171814E-2,-4.8889685E-1,-2.4396746E-1,-1.0648112E-1,-3.6418104E-1,-2.0523336E-4,-8.668755E-3,-4.0112618E-1,-1.9051564E-1,-4.0924692E-1,-7.2952604E-1,7.134041E-3,-1.5955258E-3,-3.5210484E-1,-3.2327783E-3,-8.8373445E-2,2.609646E-1,-3.0881885E-1,-1.1437352E-1,-6.4291414E-3,-2.0982799E-1,-7.090155E-2,1.4729528E-1,-2.9242274E-1,-1.6357294E-1,2.1406561E-1,-9.155216E-2,1.4698546E-1,-1.0355264E-4,1.6177587E-2,1.801334E-1,-1.7810671E-1,3.806661E-2,-3.739704E-3,1.9890027E-1,-2.2508079E-1,5.5337463E-2,-2.7964246E-3,3.7452504E-1,-1.0497735E-2,2.4067695E-1,3.5375687E-1,-1.6496254E-2,-2.4744457E-2,-6.736191E-3,-4.9003786E-3,-1.5235247E-2,-6.2534437E-3,4.8611E-3,-2.1608597E-2,-4.7450764E-3,-6.4338106E-3,-2.0677082E-2,1.5840215E-3,-1.2459256E-2,-1.7645089E-3,-2.4772966E-2,-3.7082773E-2,-8.989092E-3,-2.4174046E-2,-7.640748E-3,-7.2868373E-3,-4.3770584E-4,2.2648782E-2,1.8773497E-3,-1.5723255E-2,5.5362433E-3,-3.4656918E-3,-1.436615E-2,-1.3850961E-3,6.759027E-3,-1.94962E-3,-1.4518345E-2,-6.35474E-3,3.2020428E-3,1.8818734E-3,9.985261E-3,-1.0560763E-2,-2.5906444E-2,-8.920375E-3,5.5722246E-4,-8.974266E-3,2.4949055E-2,-7.0280256E-3,-9.643518E-4,8.134615E-3,-2.2447424E-3,1.7948869E-3,-1.043556E-2,1.9050927E-3,1.0468E-2,-1.7160157E-2,-5.344521E-3,7.7851377E-3,-2.7293684E-3,5.387367E-3,-6.6583795E-3,1.325398E-2,6.6879336E-3,-1.624695E-2,2.3120295E-3,3.8442214E-4,8.581635E-3,1.8562635E-2,4.05572E-3,-3.1724956E-3,1.8040104E-2,2.6949257E-2,1.5185237E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,-1,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,-1,-1,97,-1,99,101,103,105,-1,-1,107,109,111,113,-1,-1,115,-1,117,119,121,123,125,127,129,131,133,135,137,139,141,-1,143,145,147,149,151,153,155,157,-1,159,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0813173E1,8.703476E0,1.5900902E1,2.0952053E0,5.26482E0,9.374464E0,4.3974E0,1.5904565E0,2.1726513E0,4.574442E-1,4.6347914E0,3.231176E0,4.598614E0,1.9333153E0,2.1395187E0,1.0452957E0,2.246075E-1,2.3717403E0,3.6797616E-1,0E0,0E0,1.6451416E0,8.747642E-1,2.2927985E0,1.3862183E0,4.724227E0,0E0,1.3279517E0,1.8920703E0,1.6586736E-1,1.6375885E0,8.196564E-1,5.9901714E-1,1.3071056E-1,0E0,1.2974157E0,4.5446014E-1,1.0994836E-1,3.686362E-1,9.0190274E-1,1.6428127E0,3.0526668E-1,5.476774E-1,5.228524E-1,2.234253E0,0E0,1.172654E-1,4.115237E0,1.8203446E0,8.280287E-1,9.4801015E-1,4.088421E-1,8.698921E-1,0E0,0E0,1.3181839E0,0E0,1.4023733E-1,5.031569E-1,3.239842E-1,2.0907247E-1,0E0,0E0,6.202564E-1,8.027842E-1,5.039692E-1,3.8865185E-1,0E0,0E0,1.5465176E-1,0E0,5.191444E-1,3.575074E-1,8.4065056E-1,6.2541294E-1,3.2604E-1,1.13927275E-1,1.280484E-1,4.8096895E-1,9.7079086E-1,3.1627107E-1,3.30095E0,1.2072756E0,1.0107821E-1,0E0,1.3145682E0,1.9933529E0,1.0027645E0,8.4949625E-1,4.0878004E-1,5.3796625E-1,4.5331228E-1,5.0835E-1,0E0,2.0602036E-1,0E0,3.1718135E-1,1.0171356E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,57,57,58,58,59,59,60,60,63,63,64,64,65,65,66,66,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,94,94,96,96,97,97],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,-1,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,-1,-1,98,-1,100,102,104,106,-1,-1,108,110,112,114,-1,-1,116,-1,118,120,122,124,126,128,130,132,134,136,138,140,142,-1,144,146,148,150,152,154,156,158,-1,160,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.382114E0,3.794679E8,6.8503425E6,2.5E1,1.8108038E5,1.5589395E0,1.638058E2,7.621583E2,2.9328E4,5.517536E-1,1.5694646E3,7.3610186E3,4.7692296E-1,6.860185E2,2.68E0,2.579E3,4.3702424E7,4.422732E6,7.990749E-3,3.9115895E-2,1.683653E8,1.3640075E0,2.1525E4,1.5E1,3.7832818E0,3.799462E-2,1.5548611E7,2.6307288E7,4.6847186E2,1.6828056E1,7.08E2,1.2342778E1,4.822581E0,8.546034E-3,4.6253732E2,9.251576E3,2.2444444E0,2.116711E3,2.1169072E3,2.7009541E-2,2.0620978E7,2.821E3,3.5315048E2,7.330957E4,3.100706E-2,9.408768E1,8.01E2,1.176E3,1.4931408E6,1.0824417E0,2.1853803E4,6.43673E5,8.196518E-3,-1.8811304E-3,3.7826266E4,3.3171814E-2,9.824211E5,7.5408E4,9.544947E6,2.8107285E-1,-2.0523336E-4,-8.668755E-3,1.1317E4,1.18827E5,6.5029144E5,2.1907706E12,7.134041E-3,-1.5955258E-3,1.2068E4,-3.2327783E-3,3.24645E5,2.5927516E11,2.0601392E3,3.1933654E7,2.6312E4,3.4692544E7,3.3968747E-1,1.6052323E5,3E0,2.7E2,1.91E2,7.805608E4,1.7427321E-1,-1.0355264E-4,2E0,1E0,1E0,9.283875E6,3.37E2,1.03E2,2.2214102E6,2.2772677E5,-2.7964246E-3,8.476884E-1,-1.0497735E-2,5.0279167E1,1.0594161E1,-1.6496254E-2,-2.4744457E-2,-6.736191E-3,-4.9003786E-3,-1.5235247E-2,-6.2534437E-3,4.8611E-3,-2.1608597E-2,-4.7450764E-3,-6.4338106E-3,-2.0677082E-2,1.5840215E-3,-1.2459256E-2,-1.7645089E-3,-2.4772966E-2,-3.7082773E-2,-8.989092E-3,-2.4174046E-2,-7.640748E-3,-7.2868373E-3,-4.3770584E-4,2.2648782E-2,1.8773497E-3,-1.5723255E-2,5.5362433E-3,-3.4656918E-3,-1.436615E-2,-1.3850961E-3,6.759027E-3,-1.94962E-3,-1.4518345E-2,-6.35474E-3,3.2020428E-3,1.8818734E-3,9.985261E-3,-1.0560763E-2,-2.5906444E-2,-8.920375E-3,5.5722246E-4,-8.974266E-3,2.4949055E-2,-7.0280256E-3,-9.643518E-4,8.134615E-3,-2.2447424E-3,1.7948869E-3,-1.043556E-2,1.9050927E-3,1.0468E-2,-1.7160157E-2,-5.344521E-3,7.7851377E-3,-2.7293684E-3,5.387367E-3,-6.6583795E-3,1.325398E-2,6.6879336E-3,-1.624695E-2,2.3120295E-3,3.8442214E-4,8.581635E-3,1.8562635E-2,4.05572E-3,-3.1724956E-3,1.8040104E-2,2.6949257E-2,1.5185237E-2],"split_indices":[20,56,7,45,0,28,41,52,52,1,42,52,4,35,52,54,2,45,47,0,0,7,39,9,0,53,0,45,43,4,35,2,54,56,0,4,33,56,4,52,38,29,2,33,45,0,56,2,10,45,53,33,11,0,0,52,0,32,1,9,27,0,0,9,29,28,31,0,0,9,0,32,31,52,48,2,45,38,33,8,3,2,28,38,0,6,102,102,45,0,11,28,33,0,34,0,56,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E3,8.58E2,1.812E3,3.74E2,4.84E2,1.296E3,5.16E2,1.77E2,1.97E2,8E0,4.76E2,5.01E2,7.95E2,3.26E2,1.9E2,1.42E2,3.5E1,1.65E2,3.2E1,2E0,6E0,2.87E2,1.89E2,4.69E2,3.2E1,7.87E2,8E0,2.47E2,7.9E1,1.8E1,1.72E2,7.1E1,7.1E1,3E1,5E0,1.32E2,3.3E1,1.3E1,1.9E1,1.09E2,1.78E2,1.03E2,8.6E1,1.38E2,3.31E2,5E0,2.7E1,6.31E2,1.56E2,1.48E2,9.9E1,6.4E1,1.5E1,4E0,1.4E1,1.57E2,1.5E1,2.1E1,5E1,6.1E1,1E1,2.6E1,4E0,8.2E1,5E1,1.2E1,2.1E1,4E0,9E0,7E0,1.2E1,1.02E2,7E0,9.6E1,8.2E1,9.6E1,7E0,1.3E1,7.3E1,6E1,7.8E1,2.5E1,3.06E2,2E1,7E0,2.61E2,3.7E2,8.6E1,7E1,2.4E1,1.24E2,1.3E1,8.6E1,2E0,6.2E1,9E0,6E0,1.55E2,2E0,1.9E1,2E0,1.8E1,3.2E1,5.5E1,6E0,7E0,3E0,9E0,7.3E1,1.2E1,3.8E1,3E0,9E0,1.9E1,2E0,3E0,4E0,5.6E1,4.6E1,3E0,4E0,9.2E1,4E0,6.8E1,1.4E1,8.4E1,1.2E1,3E0,4E0,9E0,4E0,2.7E1,4.6E1,4.8E1,1.2E1,6.9E1,9E0,1.1E1,1.4E1,1.72E2,1.34E2,1.8E1,2E0,2.4E2,2.1E1,8E1,2.9E2,2.2E1,6.4E1,3E1,4E1,1.3E1,1.1E1,5.2E1,7.2E1,9E0,4E0,6.3E1,2.3E1,5.9E1,3E0,2E0,4E0,2.1E1,1.34E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[1.2119557E-3,-1.540139E-1,7.6362364E-2,-1.7639807E-1,2.5078934E-1,1.71523E-2,2.1235107E-1,-2.5985917E-1,-7.406544E-2,3.997826E-2,1.4232053E-1,-9.186061E-2,8.405321E-2,1.4797324E-1,3.5971838E-1,-2.3324619E-1,-6.0352886E-1,-1.5483174E-1,3.3917613E-2,2.2796384E-1,-3.301447E-2,-1.15178585E-1,1.9698398E-1,1.0628826E-1,-2.0412114E-1,1.2223314E-1,4.5618987E-1,3.9698562E-1,9.943642E-2,-3.019184E-1,-1.1961865E-1,-6.822117E-1,-2.2506E-1,-1.3383795E-1,-3.6214522E-1,-2.3699392E-2,5.064838E-2,2.8102335E-1,-2.194184E-4,-1.2456429E-2,2.6508642E-2,-9.159425E-2,-3.4361082E-1,2.9769607E-2,5.193369E-3,4.2846348E-2,1.9239607E-1,-2.8201234E-1,-6.087331E-2,3.489819E-2,1.7615661E-1,3.357693E-1,2.982226E-2,1.7352588E-1,4.4247276E-1,-6.3184793E-3,1.5818225E-1,-2.4881501E-1,-5.076333E-1,1.6581057E-2,-2.0508291E-1,-4.7581366E-1,-3.951628E-2,2.7363605E-3,-1.6731359E-2,-5.6544937E-2,-1.6535358E-1,-2.4306362E-2,-1.1256031E-2,-6.996627E-2,9.4258524E-2,1.8107957E-1,1.9408436E-2,-7.910122E-3,3.8964525E-3,2.3566937E-1,-1.0140389E-1,-2.215539E-1,-3.3515327E-2,-9.452073E-3,1.3040622E-1,1.7504188E-1,3.5708874E-2,-1.5014423E-1,-4.0553913E-1,-1.10911205E-1,5.8153574E-3,-2.5529439E-2,1.6182709E-1,2.0000014E-1,-8.409927E-2,1.7599303E-2,2.1117046E-3,1.12859376E-1,1.7093938E-2,4.5768884E-1,-5.2397903E-3,1.0657383E-2,-2.5691148E-3,-1.4343081E-2,-5.419665E-3,-1.48444865E-2,-2.9164245E-2,1.41752735E-2,-8.6316053E-4,-1.2008255E-2,-4.63748E-3,-5.770687E-3,-2.7314786E-2,-1.5841426E-2,-1.1700669E-3,-1.0532126E-2,-4.732732E-3,-5.443011E-3,4.925782E-3,1.6829729E-3,8.880654E-3,1.1461196E-2,1.3592008E-3,2.3971599E-2,1.9472501E-3,-1.1030624E-2,-3.8127773E-3,-1.5555846E-2,-5.8996133E-3,-2.0172992E-3,8.54556E-3,4.7921035E-3,1.6470768E-2,1.3572843E-2,5.0935447E-3,-1.039082E-2,3.8486072E-3,-3.1682875E-2,-1.5908632E-2,-6.78267E-3,3.8142996E-3,-3.5286003E-3,2.6995398E-3,1.7673096E-2,3.4765357E-3,7.505292E-3,1.6551051E-2,9.859318E-3,-9.691396E-3,6.5784464E-3,-3.8292196E-3,3.3776365E-2,2.0481696E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,-1,-1,73,75,77,-1,-1,79,81,83,85,87,89,91,-1,93,95,-1,97,99,101,103,105,107,-1,-1,-1,109,111,-1,-1,113,115,117,-1,-1,-1,119,121,123,-1,125,127,129,-1,131,133,135,-1,137,139,141,143,-1,-1,145,-1,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1111414E1,7.9274673E0,1.4465624E1,7.026909E0,2.7950737E0,9.145928E0,5.1172867E0,4.011549E0,3.2496092E0,0E0,6.081769E-1,3.2354126E0,5.00038E0,2.9669132E0,1.5610256E0,3.264574E0,7.993889E-1,8.785701E-1,1.4405768E0,3.344339E-1,1.961324E-1,2.3497667E0,1.2861078E0,3.9359312E0,6.076665E-1,1.6562715E0,3.724575E-1,1.3869114E0,3.1989732E-1,2.746748E0,1.8774741E0,3.743782E-1,2.6551554E-1,4.6565795E-1,2.3184705E-1,0E0,8.2715845E-1,1.9979608E-1,0E0,0E0,1.3960893E-1,1.3144538E0,1.6275992E0,0E0,0E0,1.9107769E0,2.807394E0,5.255234E-1,2.0503175E-1,1.047253E0,1.3696017E0,1.5958738E-1,0E0,2.4901712E-1,1.0493832E0,0E0,2.573137E-1,1.4167099E0,8.729477E-1,6.144797E-1,4.6504593E-1,3.679874E-1,0E0,0E0,0E0,5.027007E-1,4.792533E-1,0E0,0E0,3.2880622E-1,6.1334705E-1,1.1771414E-1,0E0,0E0,0E0,5.9949607E-1,1.0800714E0,2.8503466E-1,0E0,1.615833E0,9.762647E-1,2.1879911E0,0E0,3.1010932E-1,1.1827898E-1,1.10788494E-1,0E0,3.7089854E-1,7.8675246E-1,1.2062378E0,6.93244E-1,0E0,0E0,1.1020735E-1,0E0,5.603237E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,65,65,66,66,69,69,70,70,71,71,75,75,76,76,77,77,79,79,80,80,81,81,83,83,84,84,85,85,87,87,88,88,89,89,90,90,93,93,95,95],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,-1,-1,74,76,78,-1,-1,80,82,84,86,88,90,92,-1,94,96,-1,98,100,102,104,106,108,-1,-1,-1,110,112,-1,-1,114,116,118,-1,-1,-1,120,122,124,-1,126,128,130,-1,132,134,136,-1,138,140,142,144,-1,-1,146,-1,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.4839956E3,3.474851E8,1E0,2.46E2,1.8108038E5,1.1251919E6,1.2427474E8,3.3926086E2,3.997826E-2,6.747114E7,1.4845161E3,3.956147E-1,6.8545566E0,2E0,9.357879E2,4.8083666E-1,1.5988282E1,5.2143492E-5,2.7711287E0,5.704748E4,1E0,1.5E1,3.5259784E2,2.6476662E6,7.2456576E8,2.257944E1,1.460806E6,8.319452E2,3.0161016E7,3.791269E7,3.1372E4,1.7322648E5,2.8945708E6,2.3067484E0,-2.3699392E-2,2.3125623E5,3.5985E4,-2.194184E-4,-1.2456429E-2,2.9104478E0,7E0,7.492813E0,2.9769607E-2,5.193369E-3,1.5859042E6,4.4839956E3,7.31E2,2.0550244E0,4.5154482E-1,2.6974544E7,7.771383E0,2.982226E-2,8.32E3,5.1797544E7,-6.3184793E-3,3.8807898E6,4.31375E2,2.8035452E11,7.131107E6,3.0387878E3,5.980516E5,-3.951628E-2,2.7363605E-3,-1.6731359E-2,1.8631586E4,1.1771E4,-2.4306362E-2,-1.1256031E-2,1.0323588E3,1.6394367E0,1.0881593E10,1.9408436E-2,-7.910122E-3,3.8964525E-3,1.2068493E4,3.75E1,5E0,-3.3515327E-2,9.750871E6,7.780377E6,3.68E0,3.5708874E-2,1.415178E3,8.8365E4,1E0,5.8153574E-3,1.4571098E5,2.53383E5,6.4683E4,9.391714E4,1.7599303E-2,2.1117046E-3,1.964127E6,1.7093938E-2,1.0918E4,-5.2397903E-3,1.0657383E-2,-2.5691148E-3,-1.4343081E-2,-5.419665E-3,-1.48444865E-2,-2.9164245E-2,1.41752735E-2,-8.6316053E-4,-1.2008255E-2,-4.63748E-3,-5.770687E-3,-2.7314786E-2,-1.5841426E-2,-1.1700669E-3,-1.0532126E-2,-4.732732E-3,-5.443011E-3,4.925782E-3,1.6829729E-3,8.880654E-3,1.1461196E-2,1.3592008E-3,2.3971599E-2,1.9472501E-3,-1.1030624E-2,-3.8127773E-3,-1.5555846E-2,-5.8996133E-3,-2.0172992E-3,8.54556E-3,4.7921035E-3,1.6470768E-2,1.3572843E-2,5.0935447E-3,-1.039082E-2,3.8486072E-3,-3.1682875E-2,-1.5908632E-2,-6.78267E-3,3.8142996E-3,-3.5286003E-3,2.6995398E-3,1.7673096E-2,3.4765357E-3,7.505292E-3,1.6551051E-2,9.859318E-3,-9.691396E-3,6.5784464E-3,-3.8292196E-3,3.3776365E-2,2.0481696E-2],"split_indices":[20,52,7,102,10,28,28,45,52,0,45,52,27,35,6,55,42,54,42,42,28,6,0,52,32,7,54,47,52,45,45,11,28,45,58,0,28,2,0,0,54,3,54,0,0,28,52,11,39,38,43,53,0,2,50,0,28,55,31,45,4,28,0,0,0,33,11,0,0,52,42,12,0,0,0,28,4,6,0,9,28,54,0,52,1,89,0,28,9,2,28,0,0,1,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.665E3,8.69E2,1.796E3,8.24E2,4.5E1,1.252E3,5.44E2,4.53E2,3.71E2,6E0,3.9E1,4.76E2,7.76E2,3.8E2,1.64E2,4.22E2,3.1E1,2.12E2,1.59E2,2.6E1,1.3E1,4.41E2,3.5E1,7.21E2,5.5E1,3.52E2,2.8E1,1.43E2,2.1E1,2.62E2,1.6E2,2.5E1,6E0,1.94E2,1.8E1,4E0,1.55E2,2.1E1,5E0,2E0,1.1E1,4.01E2,4E1,5E0,3E1,4.16E2,3.05E2,3.5E1,2E1,1.35E2,2.17E2,1.8E1,1E1,2.5E1,1.18E2,4E0,1.7E1,2.1E2,5.2E1,6.2E1,9.8E1,1.2E1,1.3E1,2E0,4E0,5.7E1,1.37E2,7E0,1.1E1,4.1E1,1.14E2,1.3E1,8E0,2E0,9E0,1.1E1,3.9E2,3.1E1,9E0,2.61E2,1.55E2,2.97E2,8E0,1.8E1,1.7E1,1.6E1,4E0,9.2E1,4.3E1,1.99E2,1.8E1,1.6E1,2E0,2E1,5E0,1.15E2,3E0,1.3E1,4E0,1.52E2,5.8E1,1.9E1,3.3E1,6E0,5.6E1,6.8E1,3E1,3E0,9E0,5E0,5.2E1,7.4E1,6.3E1,3.3E1,8E0,7E1,4.4E1,9E0,4E0,4E0,7E0,5.5E1,3.35E2,1.4E1,1.7E1,2.23E2,3.8E1,1.37E2,1.8E1,1.14E2,1.83E2,1.4E1,4E0,2E0,1.5E1,1.4E1,2E0,5.8E1,3.4E1,1.2E1,3.1E1,1.55E2,4.4E1,5E0,1.3E1,1.8E1,2E0,1E1,1.05E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[-9.025267E-4,-1.3896793E-1,6.3401364E-2,-2.3047473E-1,-5.0123435E-2,2.4148004E-2,2.3427945E-1,-7.497217E-1,-2.1711773E-1,3.6045685E-2,-6.32899E-2,-1.48147E-2,1.8111321E-1,2.8776106E-1,3.6191937E-2,-4.3232553E-2,-1.4104466E-2,-2.8571758E-1,-1.5108955E-1,-1.039998E-1,1.09880276E-1,-6.32237E-2,7.2402485E-2,9.866355E-2,2.942659E-1,2.630818E-1,6.5693223E-1,-3.9383966E-1,9.517797E-2,-5.8497024E-1,-2.5799212E-1,-7.683761E-2,-2.1111427E-1,-6.1867006E-2,-2.3735094E-1,-1.0717568E-2,1.6390999E-1,-4.40953E-2,-2.60416E-1,-7.533637E-2,1.0049781E-1,6.1613254E-2,2.7795768E-1,3.3515242E-1,-2.2460647E-2,4.4898596E-1,2.1105194E-1,1.7014723E-2,4.0806178E-2,-2.5476264E-2,-8.473882E-3,1.4346276E-1,-1.8473464E-1,-8.07251E-3,-3.3989172E-2,-3.3321697E-1,-1.3463435E-1,1.1846715E-1,-1.0382356E-1,-1.9780806E-1,-3.074304E-2,-8.637672E-2,1.9657634E-1,-3.4236902E-1,-1.1278308E-1,1.1810268E-3,-6.3577644E-3,1.22817546E-1,1.7473219E-2,-8.554943E-2,1.0941487E-1,-3.7090725E-1,-1.6633303E-1,-4.9658455E-2,-2.9459309E-2,1.9624798E-1,3.422164E-2,-1.1727213E-1,1.1424032E-1,-1.2262575E-3,3.1396902E-1,3.8565356E-1,9.1139704E-2,-2.899652E-2,1.13808855E-1,4.9838895E-1,8.705254E-3,2.2165413E-1,-1.717199E-2,-1.2298125E-3,2.5202063E-1,-3.2229635E-1,4.8935506E-3,-1.4815088E-2,-3.3869617E-2,2.7267411E-3,-1.0900609E-2,1.0358575E-3,1.1958922E-2,-1.3460235E-2,-3.6432345E-3,-4.2113075E-3,-1.1780219E-2,-1.4100601E-2,-3.2161924E-3,1.2550069E-2,-4.385187E-3,-9.69781E-3,-1.9877557E-2,5.4907165E-3,-9.346425E-3,8.992355E-3,1.9796053E-4,-6.7965193E-3,-1.6988249E-3,3.0361772E-2,3.0012364E-3,-1.5751198E-2,-3.061828E-2,-1.0338508E-2,-3.1597777E-3,5.0719344E-4,-6.836698E-3,8.051416E-3,1.6193341E-2,9.442629E-4,1.0045677E-2,1.0519773E-2,-1.5337736E-2,-9.595826E-3,6.123958E-3,5.749683E-4,1.6217286E-2,2.3616776E-2,1.4429935E-2,-7.6268674E-3,7.839307E-3,1.5715433E-2,-8.580432E-4,2.521508E-2,8.04935E-3,7.982369E-3,1.7341759E-2,4.140153E-3,-4.9528796E-3,2.0140136E-2,6.882822E-3,-5.4836664E-3,-2.1527488E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,-1,-1,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,-1,-1,-1,89,91,-1,-1,93,95,97,99,101,-1,103,105,107,109,-1,-1,111,-1,113,115,117,119,121,-1,123,125,127,129,-1,131,133,135,-1,137,139,-1,141,-1,143,145,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3553644E1,6.8419323E0,1.2134988E1,2.7154903E0,4.5821385E0,9.01124E0,3.5703697E0,4.3066692E-1,1.7958202E0,0E0,2.9934678E0,4.9908733E0,2.7044792E0,2.2506676E0,1.8891926E0,0E0,0E0,1.4918709E0,9.152808E-1,1.912446E0,5.2954197E-1,2.8552167E0,1.7591968E0,1.1188104E0,1.6003599E0,2.3330536E0,6.39091E-1,1.7550969E-1,9.0501153E-1,7.115588E-1,1.6533003E0,5.129957E-1,5.322809E-1,1.676585E0,1.0341005E0,1.1553011E-1,4.1760468E-1,4.4257197E0,6.3479376E-1,9.1200054E-1,2.2460978E0,1.3596296E0,3.2505918E-1,1.3080502E0,1.2606144E0,6.2226295E-1,1.2752409E0,0E0,0E0,0E0,0E0,8.79931E-1,4.276203E-1,0E0,0E0,7.8978443E-1,1.2776433E0,1.4711373E-1,3.9175272E-1,5.764766E-1,0E0,9.3993115E-1,4.5795792E-1,3.633237E-1,7.530744E-1,0E0,0E0,3.6924082E-1,0E0,1.5389943E0,3.5894587E0,1.280036E-1,1.7469323E-1,3.71712E-1,0E0,5.21811E-1,5.331017E-1,2.3274164E0,5.017365E-1,0E0,1.8956709E-1,6.6733074E-1,3.894061E-1,0E0,3.729297E-1,3.2956886E-1,0E0,1.445365E0,0E0,2.3294386E-1,5.2360225E-1,1.3357317E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,51,51,52,52,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,80,80,81,81,82,82,84,84,85,85,87,87,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,-1,-1,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,-1,-1,-1,90,92,-1,-1,94,96,98,100,102,-1,104,106,108,110,-1,-1,112,-1,114,116,118,120,122,-1,124,126,128,130,-1,132,134,136,-1,138,140,-1,142,-1,144,146,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,9.075E3,3E0,2.5E1,1.8137958E6,3.5301748E7,1.36054E-1,2.308943E0,3.6045685E-2,1.5083523E0,1.645614E6,4.930435E2,4.5505118E2,3.0972284E1,-4.3232553E-2,-1.4104466E-2,1.0693E4,5.1148495E6,6.747114E7,2.3125623E5,4.0844156E7,6.1E2,1E0,2.0295714E7,2E0,4.22088E0,4.1043E4,2.3580047E10,1.7848537E0,1.752512E2,1.36E2,9.5032835E0,2.1169072E3,2.867292E3,1.3653578E7,7.6008E4,6.6473596E2,1.817757E1,1.744993E6,2.4E1,2.1033548E2,1.1693E4,2E0,1E0,4.728721E0,3.7826266E4,1.7014723E-2,4.0806178E-2,-2.5476264E-2,-8.473882E-3,7.023838E3,1E0,-8.07251E-3,-3.3989172E-2,4.5723195E6,6.748543E-1,2.72E-1,4.5555557E1,1.2E1,-3.074304E-2,1E0,8.996408E0,2.8277853E11,9.64E2,1.1810268E-3,-6.3577644E-3,8.7643677E-1,1.7473219E-2,1.7669278E5,2.04E5,1.6355225E1,5E0,5.8019036E-1,-2.9459309E-2,7.298614E2,1.2502964E9,5E-1,3.2801925E6,-1.2262575E-3,1.459995E2,7.857143E0,1.4523809E0,-2.899652E-2,1.3865336E0,1E0,8.705254E-3,6.9673E4,-1.717199E-2,6.8743766E-4,1.2695038E-3,2.14099E5,4.8935506E-3,-1.4815088E-2,-3.3869617E-2,2.7267411E-3,-1.0900609E-2,1.0358575E-3,1.1958922E-2,-1.3460235E-2,-3.6432345E-3,-4.2113075E-3,-1.1780219E-2,-1.4100601E-2,-3.2161924E-3,1.2550069E-2,-4.385187E-3,-9.69781E-3,-1.9877557E-2,5.4907165E-3,-9.346425E-3,8.992355E-3,1.9796053E-4,-6.7965193E-3,-1.6988249E-3,3.0361772E-2,3.0012364E-3,-1.5751198E-2,-3.061828E-2,-1.0338508E-2,-3.1597777E-3,5.0719344E-4,-6.836698E-3,8.051416E-3,1.6193341E-2,9.442629E-4,1.0045677E-2,1.0519773E-2,-1.5337736E-2,-9.595826E-3,6.123958E-3,5.749683E-4,1.6217286E-2,2.3616776E-2,1.4429935E-2,-7.6268674E-3,7.839307E-3,1.5715433E-2,-8.580432E-4,2.521508E-2,8.04935E-3,7.982369E-3,1.7341759E-2,4.140153E-3,-4.9528796E-3,2.0140136E-2,6.882822E-3,-5.4836664E-3,-2.1527488E-2],"split_indices":[20,56,2,3,0,28,45,42,54,0,42,9,4,56,56,0,0,9,45,45,28,45,2,84,50,17,53,10,5,54,52,11,53,52,4,32,10,52,58,47,8,4,1,6,8,38,52,0,0,0,0,4,102,0,0,28,27,56,52,18,0,75,57,31,0,0,0,57,0,28,5,53,6,41,0,52,7,53,45,0,33,56,54,0,54,6,0,2,0,39,27,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.651E3,8.42E2,1.809E3,4.14E2,4.28E2,1.472E3,3.37E2,9E0,4.05E2,6E0,4.22E2,1.18E3,2.92E2,2.65E2,7.2E1,6E0,3E0,1.97E2,2.08E2,3.42E2,8E1,7.59E2,4.21E2,1.7E2,1.22E2,2.5E2,1.5E1,8E0,6.4E1,1.5E1,1.82E2,9.4E1,1.14E2,2.61E2,8.1E1,2.5E1,5.5E1,6.93E2,6.6E1,6.7E1,3.54E2,1.42E2,2.8E1,1.08E2,1.4E1,5.3E1,1.97E2,7E0,8E0,4E0,4E0,5.5E1,9E0,4E0,1.1E1,1.12E2,7E1,1.1E1,8.3E1,1.12E2,2E0,2.39E2,2.2E1,4.3E1,3.8E1,2E1,5E0,4.7E1,8E0,5.46E2,1.47E2,2.9E1,3.7E1,6.5E1,2E0,1.44E2,2.1E2,3.2E1,1.1E2,3E0,2.5E1,8.9E1,1.9E1,2E0,1.2E1,4.4E1,9E0,1.94E2,3E0,2.4E1,3.1E1,6E0,3E0,1.07E2,5E0,2.3E1,4.7E1,7E0,4E0,1E1,7.3E1,3.5E1,7.7E1,1.9E1,2.2E2,1.8E1,4E0,1.6E1,2.7E1,1E1,2.8E1,3E1,1.7E1,2.56E2,2.9E2,1.1E1,1.36E2,2.7E1,2E0,2.4E1,1.3E1,4E1,2.5E1,1.22E2,2.2E1,1.95E2,1.5E1,1.2E1,2E1,4E0,1.06E2,2E0,2.3E1,3.7E1,5.2E1,4E0,1.5E1,4E0,8E0,4E1,4E0,1.41E2,5.3E1,1.3E1,1.1E1,1.1E1,2E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[-1.3425478E-3,4.8757233E-2,-1.7652026E-1,-4.3651097E-2,1.2142991E-1,-2.7242485E-1,-4.5142222E-2,-8.06858E-2,6.7806914E-2,1.5672283E-1,-1.3868158E-1,-4.6507066E-1,-2.3862034E-1,-7.999441E-2,1.078331E-1,-1.15254015E-1,2.983426E-2,7.533232E-2,-2.4984965E-2,2.087838E-1,1.4028188E-2,-2.125899E-1,7.081138E-2,-4.7950563E-1,-3.7875737E-3,-2.0027818E-1,-3.8116303E-1,-6.4674936E-2,-4.0643236E-1,8.05217E-3,3.201623E-1,-9.052396E-2,-2.9399326E-1,1.2081901E-1,-5.4091666E-2,3.457673E-2,1.6027793E-1,1.10237435E-1,2.7129313E-1,-1.842489E-2,6.1804426E-1,-2.5536036E-1,2.6509364E-3,1.3243365E-1,-2.6964265E-1,-1.7777385E-2,-5.642904E-1,-2.2838044E-1,5.28191E-3,-6.124084E-1,-2.7012056E-1,-7.6627746E-2,2.2728874E-1,-3.4849874E-3,-2.3531916E-2,5.021013E-3,-6.010439E-2,1.8268974E-2,5.8064847E-3,3.3546217E-2,-1.0676294E-1,-6.340038E-1,-2.2558402E-1,-1.4084831E-1,1.5647407E-1,-1.5914435E-2,-2.6724732E-1,9.3876295E-2,-4.180032E-2,1.620395E-2,1.0435567E-1,2.1467651E-1,4.8671722E-2,3.1491216E-2,2.5118002E-1,-7.185794E-2,1.1258791E-1,3.5531007E-2,5.884379E-3,-3.3652577E-1,-1.5212949E-1,4.63879E-2,3.4717417E-1,-3.817188E-3,-2.0737078E-2,-3.209351E-2,-1.2240141E-2,-4.5594755E-1,-1.8829298E-1,1.1369117E-1,-7.5557926E-3,-8.444118E-3,-6.831547E-1,-2.138393E-1,-3.665556E-2,-9.361729E-2,6.1215954E-3,1.6439272E-2,1.2208688E-3,-4.7428445E-3,5.7649314E-3,-1.0842792E-2,7.1337093E-3,-6.5562846E-3,-1.2527142E-3,4.285857E-3,-3.9560676E-2,-1.4964692E-2,-4.5439205E-3,-1.1996783E-2,-1.2448498E-3,9.62783E-3,-3.6240048E-3,1.8040574E-3,-5.296071E-3,-2.0547083E-2,-5.38744E-3,3.2815791E-3,1.2586527E-2,-3.4588396E-3,1.7048124E-2,5.9721293E-3,-1.4533703E-2,-2.421101E-3,1.1460163E-2,-8.5112394E-4,4.658486E-3,1.0437517E-2,1.9314304E-2,-7.317713E-3,-4.4669711E-4,2.6530349E-3,1.1587833E-2,-1.766145E-2,-7.28712E-3,-1.1596382E-2,-5.1019005E-3,9.353475E-3,-1.9290172E-3,1.959618E-2,4.154649E-3,-1.4382444E-2,-3.762815E-2,-1.3851483E-2,-6.8218824E-3,1.1440678E-2,-4.960935E-4,-3.517574E-2,-1.0335316E-2,-5.2380366E-3,-1.3033063E-2,2.754422E-3,-5.6999354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,-1,81,83,-1,85,87,89,91,93,95,97,-1,-1,-1,99,-1,-1,101,103,105,107,109,111,113,115,117,119,-1,121,123,125,-1,127,129,131,-1,-1,133,135,137,139,-1,-1,-1,-1,141,143,145,-1,-1,147,149,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3362513E1,1.3906513E1,7.4467583E0,3.7700377E0,1.0661463E1,2.11623E0,1.3468387E0,2.6204681E0,1.0507911E0,7.5818977E0,2.167229E0,2.406578E-1,1.5250225E0,9.9888957E-1,9.93183E-1,2.280747E0,1.2588359E0,7.7687216E-1,0E0,4.5778236E0,5.382385E0,1.1945238E0,8.119532E-1,2.3126316E-1,0E0,1.3471785E0,1.4009371E0,7.049046E-1,2.193042E-1,2.2400141E-1,1.3668656E-1,9.30377E-1,1.3223505E0,7.5909066E-1,6.9558835E-1,7.014095E-1,6.916549E-1,1.8670721E0,3.385334E0,1.8351983E0,7.6760626E-1,6.945162E-1,0E0,5.751755E-1,1.5166235E-1,0E0,6.4565086E-1,1.784277E0,5.305578E-1,4.6572447E-1,1.0349567E0,6.71942E-1,1.7848161E-1,0E0,0E0,0E0,1.5244716E-1,0E0,0E0,1.6538293E0,9.8515177E-1,1.46348E0,5.836375E-1,1.1943343E-1,7.393931E-1,3.7956917E-1,2.772318E-1,3.5464084E-1,8.4864074E-1,0E0,5.108661E-1,7.174125E-1,5.9915006E-1,0E0,2.1163883E0,9.399945E-1,5.489254E-1,0E0,0E0,2.3533487E-1,1.424827E-1,3.1762725E-1,1.0984659E-1,0E0,0E0,0E0,0E0,1.3036561E0,7.732606E-1,2.820363E-1,0E0,0E0,2.2544241E-1,2.1385908E-1,0E0,6.8254125E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,72,72,74,74,75,75,76,76,79,79,80,80,81,81,82,82,87,87,88,88,89,89,92,92,93,93,95,95],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,-1,82,84,-1,86,88,90,92,94,96,98,-1,-1,-1,100,-1,-1,102,104,106,108,110,112,114,116,118,120,-1,122,124,126,-1,128,130,132,-1,-1,134,136,138,140,-1,-1,-1,-1,142,144,146,-1,-1,148,150,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.7311627E2,1E0,9.9625344E5,4.0844156E7,1.5804776E4,1.7227725E0,4.264897E6,3.684685E2,1.0938637E1,3.4409692E3,1.8302105E6,4.7022E4,1.4299594E8,1.3137E4,5.024605E0,2.46E2,6.895302E7,-2.4984965E-2,6.9664386E2,5.847845E3,1E0,5.2686436E7,2.8035452E11,-3.7875737E-3,1.2204E4,1.7887324E0,2.3274304E2,4.1E2,1.5316E4,1.766486E7,9.6590906E-1,2E0,2.53E2,1.1347826E1,2E0,1.7039736E4,3.2449896E0,3.8E1,7.06699E8,1.417988E6,7.398769E4,2.6509364E-3,1.1803382E4,9.599108E6,-1.7777385E-2,1.191572E6,1.2068E4,1.1708007E7,2.2837209E2,3.6600095E10,1E0,5.1E1,-3.4849874E-3,-2.3531916E-2,5.021013E-3,3.8562928E6,1.8268974E-2,5.8064847E-3,4.0701206E1,2.2348747E2,8E0,2.373E3,4E0,3.3253515E6,1.5E1,2.558106E10,5.5669255E6,1.7331858E1,1.620395E-2,2E0,1.7148E4,1.19507775E-1,3.1491216E-2,3.2196458E6,4.646E3,2.9138382E7,3.5531007E-2,5.884379E-3,9.222717E-1,2.5933685E-2,7.0099896E9,7.7854166E0,-3.817188E-3,-2.0737078E-2,-3.209351E-2,-1.2240141E-2,9.71709E5,1.048E3,6.5884055E11,-7.5557926E-3,-8.444118E-3,9.099893E8,4.259842E8,-3.665556E-2,2.1129233E-1,6.1215954E-3,1.6439272E-2,1.2208688E-3,-4.7428445E-3,5.7649314E-3,-1.0842792E-2,7.1337093E-3,-6.5562846E-3,-1.2527142E-3,4.285857E-3,-3.9560676E-2,-1.4964692E-2,-4.5439205E-3,-1.1996783E-2,-1.2448498E-3,9.62783E-3,-3.6240048E-3,1.8040574E-3,-5.296071E-3,-2.0547083E-2,-5.38744E-3,3.2815791E-3,1.2586527E-2,-3.4588396E-3,1.7048124E-2,5.9721293E-3,-1.4533703E-2,-2.421101E-3,1.1460163E-2,-8.5112394E-4,4.658486E-3,1.0437517E-2,1.9314304E-2,-7.317713E-3,-4.4669711E-4,2.6530349E-3,1.1587833E-2,-1.766145E-2,-7.28712E-3,-1.1596382E-2,-5.1019005E-3,9.353475E-3,-1.9290172E-3,1.959618E-2,4.154649E-3,-1.4382444E-2,-3.762815E-2,-1.3851483E-2,-6.8218824E-3,1.1440678E-2,-4.960935E-4,-3.517574E-2,-1.0335316E-2,-5.2380366E-3,-1.3033063E-2,2.754422E-3,-5.6999354E-3],"split_indices":[19,52,59,28,45,51,39,9,52,53,52,32,9,45,2,53,10,7,0,52,4,96,43,31,0,2,53,56,0,9,47,53,17,2,53,8,50,54,0,7,29,33,0,4,29,0,29,9,47,4,5,89,3,0,0,0,47,0,0,52,52,0,2,8,45,3,19,28,58,0,6,9,38,0,28,2,9,0,0,27,38,5,54,0,0,0,0,29,2,31,0,0,7,7,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E3,2.069E3,5.91E2,9.11E2,1.158E3,3.41E2,2.5E2,6.84E2,2.27E2,1.02E3,1.38E2,4.9E1,2.92E2,2.04E2,4.6E1,5.21E2,1.63E2,2.25E2,2E0,7.47E2,2.73E2,1.02E2,3.6E1,4.7E1,2E0,2.32E2,6E1,1.96E2,8E0,3.2E1,1.4E1,4.59E2,6.2E1,7.8E1,8.5E1,1.53E2,7.2E1,2.91E2,4.56E2,2.6E2,1.3E1,8.8E1,1.4E1,3.1E1,5E0,2.3E1,2.4E1,2.04E2,2.8E1,1.8E1,4.2E1,1.89E2,7E0,2E0,6E0,1.3E1,1.9E1,1E1,4E0,5.3E1,4.06E2,9E0,5.3E1,9E0,6.9E1,7.3E1,1.2E1,8.6E1,6.7E1,1.6E1,5.6E1,1.07E2,1.84E2,2.1E1,4.35E2,1.85E2,7.5E1,1E1,3E0,4.8E1,4E1,2.3E1,8E0,3E0,2E0,1.7E1,7E0,2.9E1,1.75E2,1.7E1,1.1E1,3E0,1.5E1,3.9E1,3E0,1.75E2,1.4E1,4E0,3E0,1.6E1,3E0,1.6E1,3.7E1,2.95E2,1.11E2,2E0,7E0,3.1E1,2.2E1,4E0,5E0,5.8E1,1.1E1,4.7E1,2.6E1,5E0,7E0,7.6E1,1E1,6.3E1,4E0,5.4E1,2E0,9E0,9.8E1,7.8E1,1.06E2,3.59E2,7.6E1,8E1,1.05E2,5.3E1,2.2E1,4E1,8E0,1.2E1,2.8E1,8E0,1.5E1,6E0,2E0,2.1E1,8E0,5.3E1,1.22E2,8E0,9E0,1.3E1,2E0,1.5E1,2.4E1,2.5E1,1.5E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-1.0068289E-4,-1.3803886E-1,6.588948E-2,-1.8094383E-1,2.296315E-2,3.4195255E-2,2.6138955E-1,-2.7613327E-1,-1.3222906E-1,-8.3077E-2,1.01039864E-1,-1.4144013E-2,1.3907227E-1,4.5661634E-1,2.1273552E-1,-1.8471724E-1,-4.0934724E-1,-1.3992633E-1,2.8478988E-2,4.065788E-2,-1.2857856E-1,6.947225E-2,2.9002306E-1,-1.1912059E-1,2.1910315E-2,3.9241705E-2,1.971288E-1,3.0362066E-2,3.274529E-1,1.6397789E-1,3.5770518E-1,-2.5331122E-1,-1.8633373E-2,-3.4374425E-1,-6.672407E-1,-1.064535E-1,-2.6734373E-1,7.3735192E-3,-1.6803473E-3,-1.6668923E-2,-9.162848E-2,5.6957774E-2,1.3329638E-2,4.3484548E-1,2.9879345E-3,-1.0168853E-1,-3.4956014E-1,5.274668E-1,1.0975836E-2,-5.878817E-2,8.5982114E-2,2.314828E-1,1.5431064E-2,4.2184472E-2,3.9422992E-1,1.9641086E-1,-5.3872686E-2,2.8639588E-1,6.2143373E-1,-6.855983E-2,-3.252053E-1,4.693454E-2,-2.6426402E-1,-3.778572E-1,-1.4673752E-1,-7.545159E-1,-1.386058E-2,-1.4061412E-1,3.553743E-2,-4.0166473E-1,-1.3579045E-1,-4.7814015E-2,-2.1602352E-1,2.161548E-1,4.2845283E-2,2.4147045E-2,6.1318427E-3,-2.3349099E-1,-7.1223095E-2,-9.487379E-3,-2.0181065E-2,7.1193436E-3,2.7204888E-2,2.7873196E-2,-1.9451001E-1,7.0035267E-3,-1.3038078E-1,6.831947E-2,1.7909331E-2,1.9228734E-1,3.6737898E-1,1.3549495E-1,-6.0091086E-2,6.827845E-3,-6.665855E-3,2.2366568E-2,6.7188926E-3,1.1844995E-1,2.726299E-1,1.1218681E-2,-1.1490852E-1,2.1205842E-2,3.137526E-1,3.788239E-2,1.4062718E-2,1.613151E-3,-8.644862E-3,-2.18529E-2,-7.682095E-3,1.0435009E-2,-3.3175063E-3,-1.9056788E-2,-6.2790923E-3,-1.8621324E-2,-1.027549E-3,-1.0541899E-3,-1.3820314E-2,-1.5933609E-2,-4.0194903E-2,-1.8358912E-2,-5.993665E-3,-2.4255658E-3,7.4236374E-3,-6.4384355E-3,-2.136784E-2,-4.1184374E-3,-3.230067E-2,3.5648192E-3,-3.2556658E-3,-3.9216685E-3,-1.52913695E-2,2.0612013E-3,1.6054396E-2,1.0770377E-3,6.0667354E-3,-1.5448965E-2,-9.15574E-3,-2.9589625E-3,-2.1439444E-2,-1.3174274E-3,4.3732687E-3,-1.0613964E-2,4.535905E-4,-1.04925055E-2,2.1121812E-3,4.5598056E-3,-2.6140953E-3,1.0113632E-2,-7.834159E-3,3.13661E-2,1.4763115E-2,9.797779E-3,-2.8646956E-3,-5.9076822E-3,2.004941E-3,6.533834E-3,-6.657527E-3,1.4144981E-2,-5.4986967E-4,-1.045945E-2,-4.9194985E-4,-6.444248E-3,8.136847E-3,1.6316138E-2,6.980457E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,-1,-1,-1,71,73,-1,75,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,-1,119,121,123,125,127,129,131,133,-1,-1,135,137,-1,-1,-1,-1,139,141,-1,143,145,-1,147,149,151,153,-1,-1,-1,-1,155,157,-1,159,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4613325E1,6.0484657E0,1.1323956E1,3.1677475E0,1.5400584E0,7.9867105E0,2.324995E0,2.761835E0,2.651372E0,4.5000803E-1,6.1818624E-1,4.086474E0,2.8727837E0,9.3790054E-1,1.3974285E0,1.5902095E0,1.3609858E0,1.9121246E0,0E0,1.9421825E-1,4.3848556E-1,2.2723144E-1,4.6736324E-1,1.0731556E0,4.4387207E0,8.4813464E-1,1.9564304E0,0E0,5.739002E-1,1.1128054E0,7.811456E-1,1.2926745E0,6.8883365E-1,4.6203136E-1,3.7516212E-1,1.7608771E0,1.606473E0,0E0,0E0,0E0,2.6763645E-1,1.9069213E-1,0E0,1.2698376E-1,0E0,1.0193484E0,1.2324715E-1,1.7698717E-1,2.7414904E0,8.997783E-1,6.10266E-1,1.335557E0,4.7058365E-1,1.472981E-1,3.9666843E-1,7.758341E-1,3.9622873E-1,2.9804492E-1,3.236122E-1,3.380298E-1,1.4607882E0,6.9055843E-1,1.0799879E-1,2.4587631E-1,2.2363675E-1,2.2147608E-1,0E0,1.0478654E0,7.389406E-1,4.872203E-1,1.2563106E0,1.0121402E-1,1.5393978E-1,1.298689E-1,1.3879049E-1,0E0,0E0,1.1927652E-1,7.219465E-1,0E0,0E0,0E0,0E0,2.5640154E0,3.3401346E-1,0E0,7.062203E-1,3.9853263E-1,0E0,1.4264517E0,7.840328E-1,2.8212863E-1,2.1157895E-1,0E0,0E0,0E0,0E0,3.3843148E-1,4.4254112E-1,0E0,1.9268373E-1,1.3867562E-1,1.22344255E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,41,41,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,77,77,78,78,83,83,84,84,86,86,87,87,89,89,90,90,91,91,92,92,97,97,98,98,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,-1,-1,-1,72,74,-1,76,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,-1,120,122,124,126,128,130,132,134,-1,-1,136,138,-1,-1,-1,-1,140,142,-1,144,146,-1,148,150,152,154,-1,-1,-1,-1,156,158,-1,160,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.3344417E0,1.3137E4,2.308943E0,8.450237E0,8.9655056E5,1.5151515E-3,6.748543E-1,1.1458888E4,8.2608955E-3,8.498798E-1,6.76E2,1.172E3,6.797394E8,2.043739E0,3.402546E6,7.8351054E11,6.971004E7,2.8478988E-2,1.85221E5,2.2500739E-1,1.9015984E6,2.1367595E0,3.1E1,8E0,1E0,2E1,3.0362066E-2,1.4858E4,4.2620764E2,2.14099E5,4.01E2,4.0844156E7,1E0,9.824211E5,7.2041174E2,2.3023027E3,7.3735192E-3,-1.6803473E-3,-1.6668923E-2,3.2613106E5,1E0,1.3329638E-2,4.2E1,2.9879345E-3,6.1E1,3.8996825E5,1.279012E7,5.9767612E7,5.8365756E-1,6.6315195E6,1.0855529E3,2.9375E0,1.4225797E3,1.24900505E2,2.1019447E-1,1.2245615E0,6E2,6.2105E4,2.7772428E5,6.58542E5,1.8000048E10,2E1,7.8099336E11,1.9557823E0,1.4410892E-5,-1.386058E-2,1.04661034E-4,1E0,1E0,2.6572757E10,4.6263345E-2,1E0,6.6126086E4,1.02E2,2.4147045E-2,6.1318427E-3,1.0972222E0,1E0,-9.487379E-3,-2.0181065E-2,7.1193436E-3,2.7204888E-2,1.7669278E5,7.2E1,7.0035267E-3,1.2673605E7,3.9346433E0,1.7909331E-2,1.3543708E7,1.0918E4,5.5743E4,3.7E1,6.827845E-3,-6.665855E-3,2.2366568E-2,6.7188926E-3,8.4441924E-1,2.6572757E10,1.1218681E-2,1.7036236E5,2.6312E4,1.9840434E7,3.788239E-2,1.4062718E-2,1.613151E-3,-8.644862E-3,-2.18529E-2,-7.682095E-3,1.0435009E-2,-3.3175063E-3,-1.9056788E-2,-6.2790923E-3,-1.8621324E-2,-1.027549E-3,-1.0541899E-3,-1.3820314E-2,-1.5933609E-2,-4.0194903E-2,-1.8358912E-2,-5.993665E-3,-2.4255658E-3,7.4236374E-3,-6.4384355E-3,-2.136784E-2,-4.1184374E-3,-3.230067E-2,3.5648192E-3,-3.2556658E-3,-3.9216685E-3,-1.52913695E-2,2.0612013E-3,1.6054396E-2,1.0770377E-3,6.0667354E-3,-1.5448965E-2,-9.15574E-3,-2.9589625E-3,-2.1439444E-2,-1.3174274E-3,4.3732687E-3,-1.0613964E-2,4.535905E-4,-1.04925055E-2,2.1121812E-3,4.5598056E-3,-2.6140953E-3,1.0113632E-2,-7.834159E-3,3.13661E-2,1.4763115E-2,9.797779E-3,-2.8646956E-3,-5.9076822E-3,2.004941E-3,6.533834E-3,-6.657527E-3,1.4144981E-2,-5.4986967E-4,-1.045945E-2,-4.9194985E-4,-6.444248E-3,8.136847E-3,1.6316138E-2,6.980457E-3],"split_indices":[20,42,2,54,56,28,57,27,52,27,34,2,2,7,42,1,31,45,0,9,27,28,39,8,18,102,3,0,9,58,2,0,45,105,32,52,55,0,0,0,33,67,0,3,0,2,45,7,45,54,28,52,54,4,58,35,53,0,10,28,1,5,8,31,53,39,0,41,102,8,12,57,8,28,8,0,0,53,104,0,0,0,0,28,3,0,45,54,0,47,9,10,3,0,0,0,0,27,12,0,28,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.702E3,8.74E2,1.828E3,6.9E2,1.84E2,1.574E3,2.54E2,2.32E2,4.58E2,7.8E1,1.06E2,1.078E3,4.96E2,4.9E1,2.05E2,1.39E2,9.3E1,4.54E2,4E0,2.1E1,5.7E1,9.2E1,1.4E1,2.75E2,8.03E2,1.83E2,3.13E2,1.9E1,3E1,1.55E2,5E1,9.8E1,4.1E1,7.6E1,1.7E1,3.61E2,9.3E1,8E0,1.3E1,7E0,5E1,8.8E1,4E0,8E0,6E0,2.57E2,1.8E1,1.6E1,7.87E2,5.9E1,1.24E2,2.63E2,5E1,6E0,2.4E1,1.35E2,2E1,4.1E1,9E0,2.8E1,7E1,3.3E1,8E0,6.4E1,1.2E1,1.3E1,4E0,2.91E2,7E1,4.5E1,4.8E1,3.8E1,1.2E1,6E0,8.2E1,6E0,2E0,4.7E1,2.1E2,7E0,1.1E1,2E0,1.4E1,7.28E2,5.9E1,1.5E1,4.4E1,1.18E2,6E0,2.06E2,5.7E1,1.9E1,3.1E1,4E0,2E0,1.8E1,6E0,6.8E1,6.7E1,3E0,1.7E1,4E0,3.7E1,5E0,4E0,1.5E1,1.3E1,3.8E1,3.2E1,1.3E1,2E1,3E0,5E0,6.2E1,2E0,7E0,5E0,3E0,1E1,1.6E1,2.75E2,4.1E1,2.9E1,7E0,3.8E1,4.5E1,3E0,5E0,3.3E1,6E0,6E0,3E0,3E0,6.7E1,1.5E1,1.3E1,3.4E1,2.06E2,4E0,3.89E2,3.39E2,5.2E1,7E0,2.9E1,1.5E1,9.7E1,2.1E1,1.96E2,1E1,8E0,4.9E1,1.4E1,5E0,1.9E1,1.2E1,6.4E1,4E0,6.2E1,5E0,8E0,9E0,2E0,2E0,3.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-9.678152E-4,4.928742E-2,-1.3957706E-1,-1.6921453E-2,1.5478629E-1,-2.2277896E-1,-8.8423025E-3,-5.8182426E-2,8.2129434E-2,3.6538996E-2,2.6213652E-1,-2.6590958E-1,-6.486342E-2,-8.089233E-2,1.7841546E-1,-8.5551836E-2,3.4383427E-2,5.6254745E-2,2.0266624E-1,6.732515E-2,-2.1779166E-1,2.3705065E-1,5.5860996E-1,-2.4211429E-1,-5.977727E-1,-2.712564E-2,-3.791482E-1,-1.030613E-1,9.4193324E-2,8.5301645E-2,2.767736E-1,-1.2499692E-1,-2.4911646E-2,2.3563936E-1,7.271392E-3,-9.370268E-2,8.5404545E-2,1.22610964E-1,1.51450755E-2,2.4184413E-2,5.452534E-2,-1.9402E-2,-1.1952429E-1,1.9726111E-1,3.9350665E-1,6.612122E-1,3.7998985E-2,-4.779668E-1,-2.272058E-1,-1.4475487E-2,-7.15527E-1,-7.286107E-2,8.555383E-2,-4.3329066E-3,-3.1115733E-2,-1.7838975E-2,-8.650867E-2,7.0934873E-3,-6.2047946E-3,6.365458E-3,-2.9177105E-4,2.187177E-2,1.0123552E-2,-8.87959E-2,-2.3270309E-1,-7.423837E-2,5.688265E-2,-1.1886256E-3,1.2529451E-2,5.80037E-2,-8.599566E-2,3.8501327E-3,-2.718196E-1,1.5123035E-1,4.3425363E-2,1.5615915E-1,-1.3003883E-1,5.4774876E-3,1.8187405E-1,-8.121478E-3,-8.3636155E-4,2.242705E-1,1.5603318E-2,4.2339763E-1,6.412744E-3,3.430514E-2,1.75904E-2,8.568076E-3,-7.784519E-3,-3.0381117E-2,-2.5105944E-1,-3.2892048E-1,-1.9779742E-1,-3.8609985E-2,-1.7858945E-2,5.9416056E-2,-1.1352693E-1,1.921929E-2,6.298107E-3,-2.2762224E-1,-5.5652127E-2,-3.8878978E-3,-1.88534E-2,-1.5931077E-2,-6.6891075E-3,-6.0697235E-3,2.0362344E-3,6.299109E-3,-4.3006163E-4,-1.1281198E-3,4.642858E-3,-1.0475943E-2,5.303865E-6,9.091E-3,-5.6141345E-3,-3.4869347E-2,-3.6117095E-3,1.2818813E-2,3.9242506E-3,1.3128718E-3,9.489496E-3,9.580349E-3,1.4761825E-3,-1.4560707E-2,4.1813245E-3,8.647494E-4,-1.879472E-2,1.0155805E-2,-4.948288E-3,-4.0572826E-3,1.1360983E-2,-7.3753814E-3,7.359076E-3,1.6361479E-2,2.4572875E-2,-1.5204191E-3,-1.4807013E-2,-7.115673E-3,-1.9908711E-2,-1.2244772E-2,-6.78189E-3,-6.3968822E-3,6.787856E-3,5.205727E-3,-6.547938E-3,-8.030326E-3,2.1364067E-3,-8.430699E-3,-3.0322986E-2,-3.5273316E-3,3.0107903E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,-1,77,-1,79,81,83,85,87,89,91,-1,93,95,97,-1,-1,-1,99,-1,-1,-1,-1,-1,-1,101,103,105,107,-1,-1,109,111,113,115,117,119,121,123,125,127,-1,-1,129,131,133,-1,-1,-1,-1,-1,-1,135,137,139,-1,-1,141,143,-1,145,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8508152E1,1.3623296E1,7.6883373E0,4.9050145E0,9.536213E0,2.9251041E0,3.7382042E0,2.148793E0,1.0895541E0,2.8255842E0,2.7978745E0,2.5322762E0,1.104594E0,7.860018E-1,6.8159556E-1,1.5612259E0,1.0568395E0,1.2844017E0,5.249579E-1,1.7707306E0,6.695459E-1,2.1946163E0,1.5773497E0,1.0095577E0,5.492711E-1,4.4427627E-1,7.2230124E-1,7.616265E-1,2.9671097E-1,1.7923138E-1,3.3874655E-1,1.5233703E0,1.0491956E0,1.6217327E-1,8.1883234E-1,8.3508515E-1,6.704446E-1,3.4239244E-1,0E0,0E0,1.9540584E0,0E0,1.3498485E-1,1.4340944E0,5.0976944E-1,1.4183521E-1,1.9863677E-1,4.7960186E-1,8.3531094E-1,0E0,1.6541147E-1,3.348743E-1,6.231236E-1,0E0,0E0,0E0,7.232051E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.597054E-1,8.740816E-1,9.984674E-1,4.8300517E-1,0E0,0E0,3.580222E-1,6.973028E-1,7.428303E-1,1.4685639E0,7.4451065E-1,3.686471E-1,1.8085623E-1,2.1564221E-1,1.1433737E0,7.709658E-1,0E0,0E0,1.0410643E0,9.3690175E-1,3.0140877E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.04911625E-1,9.7126484E-1,7.313299E-1,0E0,0E0,2.5723752E-1,2.5718176E-1,0E0,1.4745577E-1,5.1609933E-1,3.0333337E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,56,56,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,90,90,91,91,92,92,95,95,96,96,98,98,99,99,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,-1,78,-1,80,82,84,86,88,90,92,-1,94,96,98,-1,-1,-1,100,-1,-1,-1,-1,-1,-1,102,104,106,108,-1,-1,110,112,114,116,118,120,122,124,126,128,-1,-1,130,132,134,-1,-1,-1,-1,-1,-1,136,138,140,-1,-1,142,144,-1,146,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,5.862126E2,1E0,7.733016E5,1.6649964E5,1.0052E4,6.553948E8,4.264897E6,4.5723195E6,1E0,6.7652373E3,1.2427474E8,1.5265896E8,1E0,4.7013435E1,2.631294E2,1.6E1,1.8122449E2,5.201557E7,1.102E3,1.6332493E1,3.2196458E6,2.3370175E0,7E0,8.8070023E-1,6.7652373E3,1.6992E4,3.6887734E-5,1.100592E6,1.5E1,8.628E3,1.2E1,1.19507775E-1,3.79E2,2.7661E4,1.6530488E8,1.3414634E0,1.8E1,1.51450755E-2,2.4184413E-2,1.3359244E9,-1.9402E-2,2.3898147E5,2.3907686E7,2.3487206E7,6.2105E4,5.5743E4,1.8081865E0,1.5804776E4,-1.4475487E-2,3.2049225E1,3.307766E6,3.3517068E7,-4.3329066E-3,-3.1115733E-2,-1.7838975E-2,9.4168E4,7.0934873E-3,-6.2047946E-3,6.365458E-3,-2.9177105E-4,2.187177E-2,1.0123552E-2,5.0272728E1,8.61E2,2.9076396E9,2.72258E5,-1.1886256E-3,1.2529451E-2,1.2513207E2,1.2460085E7,7.684162E-3,4.38E2,1.4136576E6,2.857E3,6.558106E-3,4.5454E4,9.192915E0,1.5005797E4,-8.121478E-3,-8.3636155E-4,1.3E0,1.3554651E2,1.3793921E3,6.412744E-3,3.430514E-2,1.75904E-2,8.568076E-3,-7.784519E-3,-3.0381117E-2,2.6011018E8,1.0376E4,1.7887324E0,-3.8609985E-2,-1.7858945E-2,4.3376322E5,2.5129596E11,1.921929E-2,6.1061732E7,1.5572634E6,7.123E3,-3.8878978E-3,-1.88534E-2,-1.5931077E-2,-6.6891075E-3,-6.0697235E-3,2.0362344E-3,6.299109E-3,-4.3006163E-4,-1.1281198E-3,4.642858E-3,-1.0475943E-2,5.303865E-6,9.091E-3,-5.6141345E-3,-3.4869347E-2,-3.6117095E-3,1.2818813E-2,3.9242506E-3,1.3128718E-3,9.489496E-3,9.580349E-3,1.4761825E-3,-1.4560707E-2,4.1813245E-3,8.647494E-4,-1.879472E-2,1.0155805E-2,-4.948288E-3,-4.0572826E-3,1.1360983E-2,-7.3753814E-3,7.359076E-3,1.6361479E-2,2.4572875E-2,-1.5204191E-3,-1.4807013E-2,-7.115673E-3,-1.9908711E-2,-1.2244772E-2,-6.78189E-3,-6.3968822E-3,6.787856E-3,5.205727E-3,-6.547938E-3,-8.030326E-3,2.1364067E-3,-8.430699E-3,-3.0322986E-2,-3.5273316E-3,3.0107903E-3],"split_indices":[27,52,102,28,28,2,12,9,28,14,52,45,45,89,56,52,10,52,7,29,58,28,38,3,27,52,2,38,29,8,9,8,38,2,29,5,53,3,0,0,7,0,33,45,50,10,10,53,51,0,58,1,51,0,0,0,1,0,0,0,0,0,0,46,2,5,9,0,0,52,12,38,0,28,2,38,1,57,4,0,0,54,56,4,0,0,0,0,0,0,7,9,53,0,0,28,31,0,45,48,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.655E3,1.949E3,7.06E2,1.198E3,7.51E2,4.31E2,2.75E2,8.46E2,3.52E2,3.58E2,3.93E2,3.38E2,9.3E1,1.99E2,7.6E1,6.53E2,1.93E2,2.91E2,6.1E1,3.2E2,3.8E1,3.64E2,2.9E1,3.17E2,2.1E1,8.4E1,9E0,1.77E2,2.2E1,4E1,3.6E1,3.95E2,2.58E2,2.2E1,1.71E2,4.7E1,2.44E2,3.7E1,2.4E1,8E0,3.12E2,1.2E1,2.6E1,2.92E2,7.2E1,2.4E1,5E0,1.7E1,3E2,7E0,1.4E1,6E1,2.4E1,5E0,4E0,9E0,1.68E2,1.8E1,4E0,2.6E1,1.4E1,8E0,2.8E1,2.97E2,9.8E1,1.61E2,9.7E1,2E0,2E1,1.11E2,6E1,3.1E1,1.6E1,9.4E1,1.5E2,3.3E1,4E0,2.26E2,8.6E1,1.7E1,9E0,2.54E2,3.8E1,6.4E1,8E0,1.9E1,5E0,3E0,2E0,9E0,8E0,6.5E1,2.35E2,1E1,4E0,1.4E1,4.6E1,4E0,2E1,2.9E1,1.39E2,2.91E2,6E0,4.6E1,5.2E1,1.11E2,5E1,4.5E1,5.2E1,3.6E1,7.5E1,2.3E1,3.7E1,1.2E1,1.9E1,4E0,1.2E1,3.4E1,6E1,1.37E2,1.3E1,2.4E1,9E0,2E0,2E0,2.2E2,6E0,7.8E1,8E0,1E1,2.44E2,1.7E1,2.1E1,3.6E1,2.8E1,2E0,6E0,2.2E1,4.3E1,1.14E2,1.21E2,4E0,1E1,4E0,4.2E1,3E0,1.7E1,2.7E1,2E0,1.21E2,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[1.142826E-3,-1.1853941E-1,5.7505704E-2,-1.3726164E-1,2.3136705E-1,-1.26056615E-2,1.2950072E-1,-2.058362E-1,-5.3220227E-2,3.3538356E-2,1.388937E-1,-9.216229E-2,4.1319672E-2,7.360901E-2,2.0648977E-1,-1.8715027E-1,-4.60578E-1,-1.1562095E-1,6.328669E-2,2.2607832E-1,-4.615969E-2,-7.102795E-2,-3.2371226E-1,3.1327907E-2,4.8833832E-1,3.631149E-2,1.7843041E-1,2.7092196E-2,1.8904985E-1,-2.4653E-1,-8.2984455E-2,-3.6778176E-1,-3.4143064E-2,-9.920873E-2,-3.2507485E-1,-2.0650366E-2,7.848238E-2,1.3667449E-2,-9.1902155E-4,-1.2871268E-1,8.225808E-3,-1.0522257E-1,7.736467E-2,-1.9778414E-1,-6.0956055E-1,5.592182E-2,-1.6311471E-1,2.7869472E-2,7.454037E-3,6.320274E-2,-1.11299224E-1,1.9976243E-1,-6.042506E-2,1.6337703E-1,4.1699335E-1,-2.2971192E-1,-6.1704946E-1,6.5067016E-2,-1.4470357E-1,-4.9703106E-1,-6.567212E-3,-6.938136E-2,-2.0713028E-1,-8.809176E-3,-2.0597063E-2,3.4751785E-1,5.973645E-2,-6.5330486E-4,-1.4267014E-2,2.1527879E-1,-1.20376654E-1,3.3336794E-1,-8.062184E-2,1.91107E-3,-1.1969005E-2,-3.5677258E-2,-4.329618E-3,8.569402E-3,1.7004418E-1,-4.4183695E-1,-9.2959076E-2,1.5395634E-2,1.1711859E-1,-2.680986E-1,-5.774945E-2,2.101925E-1,-1.4268249E-2,4.576699E-3,-8.883665E-3,2.1177794E-1,8.9343764E-2,4.6057552E-1,5.8579654E-3,-7.7388957E-3,-1.3801881E-2,-3.740023E-2,-6.464327E-3,1.1828843E-2,3.6730757E-4,-9.255528E-3,-7.798643E-4,-2.678754E-2,-8.898943E-3,-7.7473177E-3,-2.016138E-3,-1.0771496E-2,2.1443737E-3,4.903907E-3,2.2705382E-2,1.0601939E-3,6.8227123E-3,2.3272974E-2,-7.01278E-4,-6.820848E-3,-1.0245618E-3,1.9951154E-2,1.7692539E-5,-7.949823E-3,2.3790363E-3,-1.2338152E-3,6.3595483E-3,3.9244113E-3,1.2247906E-2,7.8099896E-4,-2.856019E-2,-1.4140213E-2,-1.0202689E-3,-3.40558E-3,2.8738207E-3,3.0114052E-3,9.519577E-3,-1.536612E-2,2.3895972E-3,-4.457168E-3,3.7695204E-3,9.310864E-3,2.6181804E-2,1.2208567E-2,8.763836E-4,-4.135167E-3,5.8319904E-3,4.51828E-3,2.3775013E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,-1,65,-1,-1,67,-1,69,71,73,75,77,79,-1,-1,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,109,-1,-1,111,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,131,133,135,-1,-1,-1,137,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8098612E1,5.659092E0,9.20858E0,4.69174E0,1.810791E0,3.9721665E0,3.857545E0,2.0510788E0,2.6827197E0,0E0,6.2352055E-1,1.807235E0,2.4502115E0,2.0382302E0,2.2611408E0,2.577385E0,4.4034052E-1,7.927747E-1,1.003155E0,3.837018E-1,2.5597993E-1,1.7536752E0,9.953251E-1,2.5962403E0,2.831123E-1,1.5432065E0,7.1012306E-1,0E0,2.0502806E0,1.5220451E0,1.4166068E0,6.473212E-1,0E0,7.066884E-1,1.7283428E-1,0E0,6.14699E-1,0E0,0E0,1.9063184E-1,0E0,1.3858478E0,2.6558058E0,2.9501504E-1,5.653579E-1,2.5978527E0,1.1517174E0,0E0,0E0,8.436369E-1,4.883117E-1,7.16691E-1,2.5185287E-1,1.152811E0,4.0599298E-1,9.7725296E-1,7.6372194E-1,4.784521E-1,6.765902E-1,1.8697643E-1,0E0,4.3934083E-1,2.2516513E-1,0E0,0E0,1.9969374E-1,3.6699164E-1,0E0,0E0,8.161845E-1,5.818596E-1,6.9274235E-1,4.6435565E-1,0E0,0E0,0E0,0E0,1.4577651E0,1.0436478E0,8.592746E-1,7.1850616E-1,6.796464E-1,6.695664E-1,2.676748E-1,2.2903879E-1,5.518627E-1,0E0,0E0,0E0,1.6399498E0,7.5699043E-1,3.6039162E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,-1,66,-1,-1,68,-1,70,72,74,76,78,80,-1,-1,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,110,-1,-1,112,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,132,134,136,-1,-1,-1,138,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.4839956E3,1.1803029E8,1E0,2.46E2,1.7669278E5,3.9556694E-1,1.2427474E8,4.5479352E2,3.3538356E-2,1.766486E7,1E0,9.0754684E1,1.3359244E9,1.141E4,3.83E3,7.6008E4,1.5401015E1,5.2143492E-5,5.1E1,1.4157E4,7.471519E2,3.5612745E0,5.9572783E0,2.8839298E7,3.9712732E7,3.0272608E10,2.7092196E-2,5.7039347E0,4.5723195E6,2.8091298E7,1.3528846E1,-3.4143064E-2,1.6527965E7,1.18827E5,-2.0650366E-2,1E0,1.3667449E-2,-9.1902155E-4,1.3674345E4,8.225808E-3,7E0,4.1391224E1,1.3446785E7,1.752E3,3.5734247E2,6.203E3,2.7869472E-2,7.454037E-3,2.1592189E5,3.0379269E1,3.7826266E4,9.2339675E5,8E0,4.728721E0,9.517604E6,1.4E1,3.3995522E5,3.4540886E3,9.277796E-1,-6.567212E-3,1.09E3,1E0,-8.809176E-3,-2.0597063E-2,1.16072E6,6.604754E5,-6.5330486E-4,-1.4267014E-2,1.3050649E4,3.2004956E5,2.6944466E7,4.06E2,1.91107E-3,-1.1969005E-2,-3.5677258E-2,-4.329618E-3,3.321532E6,1.273801E6,8.5539725E8,1.5944675E-4,1.6006084E3,3.596E3,1.7495675E5,6E1,1.9013363E4,-1.4268249E-2,4.576699E-3,-8.883665E-3,1E0,8.079531E4,1.401805E1,5.8579654E-3,-7.7388957E-3,-1.3801881E-2,-3.740023E-2,-6.464327E-3,1.1828843E-2,3.6730757E-4,-9.255528E-3,-7.798643E-4,-2.678754E-2,-8.898943E-3,-7.7473177E-3,-2.016138E-3,-1.0771496E-2,2.1443737E-3,4.903907E-3,2.2705382E-2,1.0601939E-3,6.8227123E-3,2.3272974E-2,-7.01278E-4,-6.820848E-3,-1.0245618E-3,1.9951154E-2,1.7692539E-5,-7.949823E-3,2.3790363E-3,-1.2338152E-3,6.3595483E-3,3.9244113E-3,1.2247906E-2,7.8099896E-4,-2.856019E-2,-1.4140213E-2,-1.0202689E-3,-3.40558E-3,2.8738207E-3,3.0114052E-3,9.519577E-3,-1.536612E-2,2.3895972E-3,-4.457168E-3,3.7695204E-3,9.310864E-3,2.6181804E-2,1.2208567E-2,8.763836E-4,-4.135167E-3,5.8319904E-3,4.51828E-3,2.3775013E-2],"split_indices":[20,52,7,102,10,28,38,45,52,0,47,6,56,7,9,2,10,54,42,3,9,52,53,53,45,45,5,0,35,28,45,56,0,45,29,0,67,0,0,4,0,3,56,45,29,52,29,0,0,28,56,52,47,17,38,45,8,48,4,27,0,2,74,0,0,1,28,0,0,28,47,45,0,0,0,0,0,28,9,31,42,55,2,33,3,52,0,0,0,6,28,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.681E3,8.58E2,1.823E3,8.15E2,4.3E1,9.24E2,8.99E2,4.48E2,3.67E2,6E0,3.7E1,3.73E2,5.51E2,5.22E2,3.77E2,4.19E2,2.9E1,2.39E2,1.28E2,2.5E1,1.2E1,3.43E2,3E1,5.4E2,1.1E1,3.86E2,1.36E2,1.6E1,3.61E2,2.66E2,1.53E2,2.3E1,6E0,2.23E2,1.6E1,3E0,1.25E2,2E1,5E0,9E0,3E0,2.79E2,6.4E1,2.2E1,8E0,4.8E2,6E1,8E0,3E0,3.27E2,5.9E1,1.25E2,1.1E1,3.26E2,3.5E1,2.56E2,1E1,4.5E1,1.08E2,1.4E1,9E0,1.76E2,4.7E1,8E0,8E0,7E0,1.18E2,6E0,3E0,1.2E1,2.67E2,2.4E1,4E1,4E0,1.8E1,6E0,2E0,3.4E2,1.4E2,1.1E1,4.9E1,1.74E2,1.53E2,1.4E1,4.5E1,1.23E2,2E0,5E0,6E0,1.96E2,1.3E2,3E1,5E0,1.2E2,1.36E2,7E0,3E0,1E1,3.5E1,7.8E1,3E1,1.1E1,3E0,3.9E1,1.37E2,4.4E1,3E0,3E0,4E0,8.2E1,3.6E1,5E0,7E0,2.18E2,4.9E1,1.9E1,5E0,2.4E1,1.6E1,2.67E2,7.3E1,7E1,7E1,3E0,8E0,1.2E1,3.7E1,5.9E1,1.15E2,9.3E1,6E1,1.2E1,2E0,3.6E1,9E0,1.19E2,4E0,1.6E2,3.6E1,2E1,1.1E2,3E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[-4.5213656E-4,4.105191E-2,-1.4387938E-1,-3.7552986E-2,1.00957386E-1,-2.3817492E-1,-1.4863122E-2,-7.9134434E-2,5.279529E-2,8.406425E-3,1.5901442E-1,-3.683517E-1,-1.9634594E-1,-4.9881138E-2,1.3646172E-1,-5.5431083E-2,-2.3011091E-1,-1.4316818E-1,6.824182E-2,-2.9016305E-2,1.688783E-1,1.8478848E-1,-3.631344E-2,-1.7717399E-1,-4.3439654E-1,-8.642746E-2,-2.3711523E-1,-3.2418083E-2,-2.8830624E-1,1.855081E-1,-4.630278E-3,-9.4142154E-2,1.4174249E-2,-6.108212E-1,-1.5068156E-1,7.448607E-2,-3.1261602E-1,9.587808E-2,-4.049624E-2,4.6638826E-1,-4.5540992E-2,1.9410118E-1,-2.7855268E-1,1.4284904E-1,3.302752E-1,-1.4746805E-1,1.0504799E-1,-2.083238E-1,1.0951897E-3,-3.3233306E-1,-5.885739E-1,-1.4820004E-1,-7.060679E-3,-5.066721E-1,-2.0282565E-1,-8.207635E-2,3.9370902E-2,-1.9382942E-2,1.0588976E-4,2.866507E-1,9.828748E-2,4.954074E-2,-1.10164106E-1,4.107475E-2,-1.3156094E-1,5.939747E-4,-7.157254E-1,3.0402867E-2,-1.9649558E-1,-1.0808054E-3,1.4772741E-2,-2.4466386E-2,-4.6939827E-3,7.660987E-2,1.6343003E-2,7.8213975E-2,-1.18409224E-1,6.400581E-3,2.4882525E-2,-1.9210975E-1,-2.3315761E-2,1.6187759E-1,3.151049E-2,-1.9583225E-2,-2.6501715E-3,1.2524231E-1,3.1274885E-1,4.9003834E-1,2.3252276E-1,3.147329E-2,-1.9538611E-1,2.3517495E-1,-1.1491689E-2,-6.4447173E-3,-1.7144002E-2,-3.6879358E-1,-5.4102684E-3,-6.31795E-1,-4.3022432E-3,8.0208825E-3,-1.701202E-1,1.9184507E-2,-9.680325E-3,-9.945265E-3,-2.811807E-2,-1.5413874E-1,-3.824723E-1,-7.264922E-2,-1.3167186E-2,6.160074E-3,1.0981631E-3,2.0335693E-2,9.622094E-3,-7.344622E-3,1.2955715E-1,4.6059564E-3,-3.6716233E-3,-2.8294143E-3,-6.976723E-3,-2.9289399E-3,4.054271E-3,-9.964525E-3,2.585566E-5,-3.84374E-2,-8.981231E-3,-1.8443529E-3,6.713946E-3,-1.7170938E-2,-4.263286E-3,4.0424117E-3,-1.5270849E-2,-7.8193017E-4,1.1766819E-2,-3.4992136E-3,-1.4859083E-2,-1.3647474E-2,-2.5140974E-3,-7.7851483E-4,-2.1719597E-2,1.06563335E-2,4.4152075E-3,1.1778935E-2,4.9965135E-3,1.699927E-2,4.7780434E-3,3.0475354E-2,1.5019408E-2,1.0059599E-2,2.2457328E-2,8.535874E-3,-3.7284812E-3,-1.2966271E-2,-4.5685284E-4,-3.0569548E-3,1.7005347E-2,-6.2989024E-3,5.2482467E-3,-1.5106979E-2,-2.4992632E-2,-3.2918274E-2,-1.0717031E-2,-6.4529106E-3,9.37213E-3,-5.709563E-3,-1.1569812E-2,6.534183E-3,-2.2156662E-3,-9.393368E-3,2.480797E-3,-1.6927895E-3,-2.2271125E-2,-2.9571084E-3,-1.2123139E-2,-2.2046359E-3,3.030028E-3,9.938658E-3,2.5757675E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,-1,95,97,99,101,103,105,107,109,-1,-1,111,113,115,117,119,121,-1,123,125,127,-1,-1,-1,-1,129,-1,131,133,-1,-1,135,137,139,-1,-1,-1,141,143,145,147,149,151,153,155,-1,-1,157,-1,159,-1,161,163,165,-1,-1,-1,167,169,171,-1,-1,173,-1,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5888006E1,9.751139E0,7.2836885E0,3.3702817E0,6.312213E0,1.8117008E0,1.3527772E0,2.1845348E0,8.663984E-1,2.7313044E0,3.6415443E0,9.710188E-1,1.1625681E0,8.525253E-1,5.66815E-1,1.4348068E0,2.4178448E0,7.9778755E-1,7.942275E-1,3.0420234E0,1.0246205E0,3.8456612E0,1.351094E0,1.480838E-1,7.6428604E-1,3.5685003E-1,1.6718893E0,6.9482523E-1,4.7443604E-1,3.246945E-1,0E0,7.927687E-1,7.5419056E-1,9.241805E-1,5.867084E-1,2.3405205E-1,4.6050203E-1,9.667405E-1,5.0871086E-1,1.4290977E-1,1.1617146E0,1.119184E0,1.2286675E-1,1.447547E0,2.1037025E0,4.1772652E-1,5.789708E-1,1.7257881E-1,0E0,2.7025986E-1,4.3903208E-1,1.4176434E-1,1.7282946E-1,3.832898E-1,1.4454336E0,1.891166E-1,2.7314848E-1,0E0,0E0,1.2760901E-1,2.0085667E-1,2.1337752E-1,5.5019736E-1,7.269821E-1,3.033148E-1,0E0,4.0017605E-1,1.1889205E-1,9.4338727E-1,0E0,0E0,0E0,0E0,6.3148844E-1,0E0,3.523727E-1,2.644127E-1,0E0,0E0,5.943862E-1,9.402103E-1,3.0906725E-1,0E0,0E0,0E0,1.1152554E0,3.7813807E-1,1.1685114E0,3.7956E-1,1.903752E-1,5.2963626E-1,6.680524E-1,3.1962562E-1,0E0,0E0,1.0485268E-1,0E0,3.2231808E-1,0E0,1.8713088E-1,1.02154136E-1,2.3654257E-1,0E0,0E0,0E0,1.1952455E0,1.015399E0,2.0278782E-1,0E0,0E0,1.6994216E-1,0E0,0E0,0E0,1.1290845E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,73,73,75,75,76,76,79,79,80,80,81,81,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,95,95,97,97,99,99,100,100,101,101,105,105,106,106,107,107,110,110,114,114],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,-1,96,98,100,102,104,106,108,110,-1,-1,112,114,116,118,120,122,-1,124,126,128,-1,-1,-1,-1,130,-1,132,134,-1,-1,136,138,140,-1,-1,-1,142,144,146,148,150,152,154,156,-1,-1,158,-1,160,-1,162,164,166,-1,-1,-1,168,170,172,-1,-1,174,-1,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.6129813E2,1E0,6.978874E5,1.5166431E5,7.2474093E-3,3.0497742E3,9.337562E0,5.8631687E1,1.3359244E9,2.8839298E7,5.0079144E5,1.2E1,1.8000048E10,1.1855755E8,1.325204E6,2E0,2.2444444E0,4.072289E0,8E0,1.314806E8,1.6503105E3,8.848604E1,1.9521575E5,8.672049E9,3.888319E-1,1.2068E4,6.218596E2,3.1827815E12,8.7643677E-1,-4.630278E-3,1.62283E0,5.894636E6,8E0,3.415965E2,7E0,2E0,2.4907975E0,9.87156E0,7.471519E2,1.54E2,2.093563E0,2.6422684E9,4.5723195E6,1.505516E7,7.621583E2,5.8019036E-1,5.065188E-4,1.0951897E-3,3.1743118E-1,1.4225797E3,1.060324E6,1E0,1.8058888E0,2.72258E5,5.077728E8,1.25218E5,-1.9382942E-2,1.0588976E-4,6.862E3,4.349136E6,3.3159972E5,3.2E1,1.8579784E2,6.97E3,5.939747E-4,3.9595376E7,9.213145E0,2.183E3,-1.0808054E-3,1.4772741E-2,-2.4466386E-2,-4.6939827E-3,2E0,1.6343003E-2,6.999242E7,5.743459E-3,6.400581E-3,2.4882525E-2,1.6848017E-2,5.1797544E7,5.670103E-2,3.151049E-2,-1.9583225E-2,-2.6501715E-3,8E0,2.5489312E5,5.473125E3,4.091015E9,5.1821535E4,1.8796511E5,4.8E1,2.3276744E5,-6.4447173E-3,-1.7144002E-2,1.1532471E-3,-5.4102684E-3,9.5082015E-1,-4.3022432E-3,6.395763E2,5.486433E5,2.46875E0,-9.680325E-3,-9.945265E-3,-2.811807E-2,3.647E3,1.411682E7,4.2343444E7,-1.3167186E-2,6.160074E-3,3.65625E1,2.0335693E-2,9.622094E-3,-7.344622E-3,5.323065E7,4.6059564E-3,-3.6716233E-3,-2.8294143E-3,-6.976723E-3,-2.9289399E-3,4.054271E-3,-9.964525E-3,2.585566E-5,-3.84374E-2,-8.981231E-3,-1.8443529E-3,6.713946E-3,-1.7170938E-2,-4.263286E-3,4.0424117E-3,-1.5270849E-2,-7.8193017E-4,1.1766819E-2,-3.4992136E-3,-1.4859083E-2,-1.3647474E-2,-2.5140974E-3,-7.7851483E-4,-2.1719597E-2,1.06563335E-2,4.4152075E-3,1.1778935E-2,4.9965135E-3,1.699927E-2,4.7780434E-3,3.0475354E-2,1.5019408E-2,1.0059599E-2,2.2457328E-2,8.535874E-3,-3.7284812E-3,-1.2966271E-2,-4.5685284E-4,-3.0569548E-3,1.7005347E-2,-6.2989024E-3,5.2482467E-3,-1.5106979E-2,-2.4992632E-2,-3.2918274E-2,-1.0717031E-2,-6.4529106E-3,9.37213E-3,-5.709563E-3,-1.1569812E-2,6.534183E-3,-2.2156662E-3,-9.393368E-3,2.480797E-3,-1.6927895E-3,-2.2271125E-2,-2.9571084E-3,-1.2123139E-2,-2.2046359E-3,3.030028E-3,9.938658E-3,2.5757675E-3],"split_indices":[19,52,59,28,28,41,52,54,47,7,45,28,18,5,45,9,17,56,54,18,1,52,56,33,5,39,9,52,31,57,0,54,45,0,4,8,17,53,58,52,0,39,7,28,50,52,41,38,0,58,4,51,105,54,9,7,29,0,0,9,32,28,10,55,10,0,7,53,2,0,0,0,0,6,0,7,57,0,0,41,50,57,0,0,0,3,33,32,7,33,33,8,33,0,0,39,0,27,0,4,28,53,0,0,0,2,12,45,0,0,56,0,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.667E3,2.069E3,5.98E2,8.95E2,1.174E3,3.45E2,2.53E2,6.13E2,2.82E2,4.53E2,7.21E2,8.2E1,2.63E2,2.06E2,4.7E1,5.31E2,8.2E1,2E1,2.62E2,3.68E2,8.5E1,6.37E2,8.4E1,2.2E1,6E1,7.2E1,1.91E2,1.93E2,1.3E1,3.9E1,8E0,3.41E2,1.9E2,1.3E1,6.9E1,9E0,1.1E1,2.09E2,5.3E1,1.1E1,3.57E2,8.1E1,4E0,4.96E2,1.41E2,4.7E1,3.7E1,1.9E1,3E0,3.8E1,2.2E1,4E1,3.2E1,2E1,1.71E2,1.14E2,7.9E1,9E0,4E0,1.7E1,2.2E1,3.4E1,3.07E2,1.61E2,2.9E1,2E0,1.1E1,1.4E1,5.5E1,7E0,2E0,5E0,6E0,1.95E2,1.4E1,2.1E1,3.2E1,2E0,9E0,4.6E1,3.11E2,7.7E1,4E0,2E0,2E0,4.51E2,4.5E1,5.2E1,8.9E1,1E1,3.7E1,1.7E1,2E1,1.4E1,5E0,3.2E1,6E0,2E1,2E0,5E0,3.5E1,2.9E1,3E0,5E0,1.5E1,1.36E2,3.5E1,1.1E2,4E0,2.3E1,5.6E1,5E0,1.2E1,2E0,2E1,2.5E1,9E0,1.27E2,1.8E2,4.8E1,1.13E2,1.8E1,1.1E1,9E0,2E0,9E0,5E0,2.1E1,3.4E1,1.92E2,3E0,1.4E1,7E0,2.7E1,5E0,2.7E1,1.9E1,3.07E2,4E0,4E1,3.7E1,6.5E1,3.86E2,3.7E1,8E0,2.7E1,2.5E1,8.3E1,6E0,4E0,6E0,2.6E1,1.1E1,5E0,1.2E1,1E1,1E1,2.6E1,6E0,1.7E1,3E0,3E0,2E0,2.2E1,1.3E1,1E1,1.9E1,1.13E2,2.3E1,7E0,2.8E1,1.05E2,5E0,3.2E1,2.4E1,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[7.4223237E-4,4.3862432E-2,-1.23617254E-1,3.4423836E-3,1.7984578E-1,-1.6719507E-1,5.0642032E-2,-7.446E-2,5.7787E-2,1.0162796E-1,2.7888247E-1,-2.2419123E-1,-6.775571E-2,1.3219929E-1,-9.3911566E-2,-1.3195291E-1,-4.3187395E-2,-2.4270918E-2,1.066255E-1,7.3395364E-2,2.9431573E-1,3.4827888E-2,2.561767E-1,-1.6783807E-1,-3.3305484E-1,-1.0919295E-1,2.7622792E-3,3.432525E-1,9.43586E-2,-6.046815E-2,-2.862429E-2,-1.541995E-1,7.93606E-3,7.811829E-2,-7.375492E-2,6.609801E-3,-1.3100047E-1,8.4829345E-2,2.1802732E-1,1.2499836E-1,9.18973E-3,3.539527E-2,1.8457709E-1,3.197639E-1,1.346694E-1,-2.1179755E-1,-5.05088E-2,-1.8805878E-1,-4.2106542E-1,-1.463907E-1,-2.2459723E-2,3.435573E-2,-5.549833E-3,-1.8415968E-3,1.9929629E-2,8.416342E-3,5.3417757E-2,-1.5524073E-1,9.096097E-4,-1.768985E-1,9.226949E-3,-5.938462E-3,7.6667905E-2,-1.4402078E-1,1.671559E-1,-3.7767002E-1,-6.486571E-2,1.4518505E-2,-2.4097411E-2,-1.0473795E-1,-1.3947306E-2,8.988294E-2,-1.9306183E-2,2.3857637E-1,-3.0000275E-3,1.943329E-1,5.364771E-2,2.3736939E-2,-1.6564261E-2,3.0457407E-1,-1.9363318E-2,-1.9709233E-3,3.4500962E-1,1.9179831E-1,9.096698E-3,-1.5406986E-1,-2.9357654E-1,8.272292E-2,-1.0045906E-1,1.7017075E-3,-2.291018E-1,-4.6124667E-1,-8.854711E-2,-9.6162826E-2,-2.3564318E-1,-7.121093E-2,1.2603986E-1,-7.7322006E-2,9.647252E-2,-1.6941411E-2,1.4286807E-1,3.1432898E-3,-9.270185E-3,-6.0414975E-3,-1.0557211E-2,5.735834E-3,-4.4788313E-3,8.851723E-3,-9.51018E-4,-1.5370347E-2,-3.8584997E-3,2.0372406E-2,1.6214466E-3,-9.257867E-3,-2.6808545E-2,-5.5061574E-3,-7.3232845E-4,3.877297E-3,-1.750448E-3,-6.719764E-3,7.284256E-4,2.9459181E-3,7.415229E-3,5.502135E-3,1.32832E-2,1.1610938E-2,4.589427E-3,2.6645002E-4,7.3917066E-3,-6.8728058E-3,2.8037634E-3,5.6780335E-3,1.616278E-2,1.7441098E-2,3.6660156E-3,-4.2355247E-3,1.0564346E-2,5.576741E-3,-4.3133283E-3,-1.3831443E-2,-4.802307E-3,-1.2023601E-2,-2.3516336E-2,7.795427E-3,-7.565273E-3,-6.526951E-3,1.3980351E-3,-1.5470681E-2,-7.833612E-3,3.3111576E-4,-2.3197426E-2,4.331042E-3,-9.248215E-3,-1.106831E-3,-7.222096E-3,-1.6203482E-2,-7.325904E-3,6.753086E-3,-4.3774694E-3,1.2384815E-2,-2.6048522E-4,-1.0631013E-2,-1.5728206E-3,3.2898919E-3,1.0231181E-2,-3.3546698E-3,8.024582E-3,-1.6604393E-3,8.719413E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,-1,-1,-1,-1,99,101,-1,103,105,-1,107,109,111,113,115,117,-1,119,-1,121,-1,123,-1,125,127,129,-1,131,-1,-1,133,135,137,139,141,143,145,-1,147,149,151,153,155,157,159,161,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4183623E1,1.0794416E1,5.1798706E0,6.418169E0,3.457553E0,3.0736866E0,1.6275619E0,1.1144259E0,3.5813618E0,1.3547378E0,1.8293943E0,2.0667458E0,5.847158E-1,6.704143E-1,8.0628264E-1,6.827636E-1,1.5075076E0,1.1016916E0,1.341259E0,7.343824E-1,1.4254563E0,0E0,1.4249372E0,1.175581E0,1.4138737E0,4.0322363E-1,2.8511903E-1,3.777318E-1,2.4383467E-1,3.644263E-1,0E0,7.027502E-1,2.9062292E-1,1.64593E0,8.5080945E-1,1.0555608E0,2.851051E-1,1.1952336E0,5.397892E-1,5.9806347E-1,5.2473384E-1,0E0,2.03114E0,1.1440439E0,4.7890186E-1,7.431612E-1,4.342276E-1,4.3027782E-1,9.211502E-1,3.7184072E-1,2.9131722E-1,4.2368317E-1,0E0,0E0,0E0,0E0,3.2969016E-1,1.989584E-1,0E0,3.390169E-1,2.8379738E-1,0E0,2.22925E-1,2.4357247E-1,2.01655E0,1.9395792E-1,7.8320444E-1,8.750166E-1,0E0,2.8378326E-1,0E0,8.491764E-1,0E0,3.7594128E-1,0E0,2.7060533E-1,2.993433E-1,5.7309794E-1,0E0,1.008265E-1,0E0,0E0,5.693226E-1,3.907112E-1,2.446836E-1,6.9586754E-1,4.4756126E-1,3.7254134E-1,2.223121E-1,0E0,1.8723512E-1,6.966057E-1,1.8590194E-1,2.2802854E-1,2.1144593E-1,1.414625E-1,1.7570427E-1,1.3427164E-1,1.122458E-1,3.0568996E-1,1.6638884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,56,56,57,57,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,69,69,71,71,73,73,75,75,76,76,77,77,79,79,82,82,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,-1,-1,-1,-1,100,102,-1,104,106,-1,108,110,112,114,116,118,-1,120,-1,122,-1,124,-1,126,128,130,-1,132,-1,-1,134,136,138,140,142,144,146,-1,148,150,152,154,156,158,160,162,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,6.624E3,1.998175E3,2.1592189E5,1.8108038E5,2.6476662E6,6.971004E7,1.20086E5,2.5275471E2,6.082581E3,6.737099E7,2.8277853E11,4.832493E-1,8.628E3,1.067509E6,1.4887473E-1,1.1558939E4,5.2E1,2.9251662E6,1.84556E7,1.417988E6,3.4827888E-2,8.2E1,2.011E3,7.5179994E-1,2.2100918E1,1.4540612E7,5.214E3,2.8091298E7,6.08767E3,-2.862429E-2,2.374722E6,2.4360857E4,2.53E2,3.3820656E7,4.4E1,2.5363636E0,3.7852024E1,2E0,5.003246E1,4.6577052E7,3.539527E-2,1.5005797E4,5.4091E4,3.5561565E8,1.411682E7,6.0910425E0,4.4444447E0,3.138649E6,2.74872E5,5.0079144E5,2.4684112E8,-5.549833E-3,-1.8415968E-3,1.9929629E-2,8.416342E-3,2.5882E4,3.9382784E9,9.096097E-4,5.61E2,9.287868E6,-5.938462E-3,3.1648794E-1,8.2E1,5.37798E5,5E0,1.19507775E-1,6.4736843E0,-2.4097411E-2,1.9143396E2,-1.3947306E-2,3.171E3,-1.9306183E-2,5.338169E9,-3.0000275E-3,1.65E2,1.3359244E9,2.496172E3,-1.6564261E-2,5.6981827E1,-1.9363318E-2,-1.9709233E-3,6.292039E0,1.0119178E10,1.19783E5,1.4431512E-1,8.7217E4,6.22198E5,8.0708375E5,1.7017075E-3,2.0939393E0,2.467284E7,4.242625E6,6.546031E5,3.2049225E1,5.3421542E-2,1.9125667E3,2.5534486E4,1.1251919E6,1E0,3.286908E-1,3.1432898E-3,-9.270185E-3,-6.0414975E-3,-1.0557211E-2,5.735834E-3,-4.4788313E-3,8.851723E-3,-9.51018E-4,-1.5370347E-2,-3.8584997E-3,2.0372406E-2,1.6214466E-3,-9.257867E-3,-2.6808545E-2,-5.5061574E-3,-7.3232845E-4,3.877297E-3,-1.750448E-3,-6.719764E-3,7.284256E-4,2.9459181E-3,7.415229E-3,5.502135E-3,1.32832E-2,1.1610938E-2,4.589427E-3,2.6645002E-4,7.3917066E-3,-6.8728058E-3,2.8037634E-3,5.6780335E-3,1.616278E-2,1.7441098E-2,3.6660156E-3,-4.2355247E-3,1.0564346E-2,5.576741E-3,-4.3133283E-3,-1.3831443E-2,-4.802307E-3,-1.2023601E-2,-2.3516336E-2,7.795427E-3,-7.565273E-3,-6.526951E-3,1.3980351E-3,-1.5470681E-2,-7.833612E-3,3.3111576E-4,-2.3197426E-2,4.331042E-3,-9.248215E-3,-1.106831E-3,-7.222096E-3,-1.6203482E-2,-7.325904E-3,6.753086E-3,-4.3774694E-3,1.2384815E-2,-2.6048522E-4,-1.0631013E-2,-1.5728206E-3,3.2898919E-3,1.0231181E-2,-3.3546698E-3,8.024582E-3,-1.6604393E-3,8.719413E-3],"split_indices":[27,2,52,28,28,32,45,9,52,52,7,31,39,9,9,38,51,11,28,45,29,0,8,2,27,56,45,9,45,4,0,29,28,2,7,3,53,53,6,58,48,0,4,33,32,12,56,56,29,29,28,7,0,0,0,0,10,5,0,2,1,0,38,29,5,6,38,58,0,52,0,2,0,31,0,3,7,4,0,56,0,0,38,20,10,58,9,1,28,0,53,45,29,48,58,27,4,33,28,89,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.643E3,1.963E3,6.8E2,1.514E3,4.49E2,5.44E2,1.36E2,6.22E2,8.92E2,2.52E2,1.97E2,3.45E2,1.99E2,8.7E1,4.9E1,2.18E2,4.04E2,3.33E2,5.59E2,2.21E2,3.1E1,8E0,1.89E2,2.29E2,1.16E2,1.25E2,7.4E1,1.2E1,7.5E1,4.7E1,2E0,1.88E2,3E1,8.1E1,3.23E2,2.59E2,7.4E1,4.69E2,9E1,1.22E2,9.9E1,5E0,2.6E1,1.23E2,6.6E1,1.66E2,6.3E1,4.5E1,7.1E1,8.7E1,3.8E1,5.9E1,1.5E1,2E0,1E1,2.4E1,5.1E1,2.1E1,2.6E1,1.65E2,2.3E1,1E1,2E1,2.3E1,5.8E1,8E0,3.15E2,2.56E2,3E0,6.5E1,9E0,4.65E2,4E0,8.4E1,6E0,6.1E1,6.1E1,9.6E1,3E0,2.2E1,4E0,8E0,1.15E2,4.5E1,2.1E1,9.9E1,6.7E1,1.7E1,4.6E1,7E0,3.8E1,6.3E1,8E0,5.7E1,3E1,2.9E1,9E0,2.1E1,3.8E1,2.9E1,2.2E1,3E0,1.8E1,7.8E1,8.7E1,1.1E1,1.2E1,9E0,1.1E1,5E0,1.8E1,1.9E1,3.9E1,5E0,3E0,1.56E2,1.59E2,1.11E2,1.45E2,5E1,1.5E1,3.25E2,1.4E2,2.1E1,6.3E1,4E1,2.1E1,4.2E1,1.9E1,1.6E1,8E1,4E0,1.8E1,1.07E2,8E0,4E0,4.1E1,1E1,1.1E1,2.7E1,7.2E1,5.7E1,1E1,1.3E1,4E0,3.6E1,1E1,1.4E1,2.4E1,3E0,6E1,3E0,5E0,2.5E1,3.2E1,1.2E1,1.8E1,2E0,2.7E1,4E0,5E0,4E0,1.7E1,3.2E1,6E0,2.3E1,6E0,4E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-1.9889919E-3,3.87132E-2,-1.1667124E-1,-4.4713423E-2,9.982473E-2,-1.4555909E-1,9.759356E-2,-6.508019E-2,1.150468E-1,1.863714E-2,1.5678556E-1,-9.81145E-2,-2.5548053E-1,2.3642708E-1,3.128151E-2,-9.8361135E-2,3.9561663E-2,7.911834E-2,1.4433761E-2,2.2502216E-2,5.844456E-3,1.4448035E-1,5.132219E-1,-1.7611371E-1,-7.659394E-3,-4.1859937E-1,-1.7829701E-1,2.9434854E-1,4.208564E-3,5.856077E-2,-2.695908E-1,-8.2598686E-2,-2.6608667E-1,1.1508546E-1,-4.24935E-2,2.9955331E-2,1.3677286E-1,-3.3627875E-2,1.5199356E-1,1.2394203E-1,3.102232E-1,5.9042376E-1,-8.262605E-3,-1.5642376E-1,-3.8965186E-1,4.9412098E-2,-8.215442E-2,-4.5626017E-1,-5.817708E-3,-1.3868155E-1,-3.3596548E-1,-2.203609E-3,1.6143795E-2,-5.691363E-3,7.908833E-2,-2.5300256E-3,-1.8988054E-2,-4.513431E-2,-1.3501313E-1,2.6033907E-3,-3.4143007E-1,1.3080752E-1,-1.2995703E-2,-2.2716054E-1,-1.2776421E-2,-7.714824E-2,4.3919086E-3,1.6992256E-1,2.05793E-3,-8.080126E-3,-1.8626769E-1,1.7368044E-1,-1.3702367E-2,8.736292E-2,1.7360179E-1,3.944957E-1,2.2212154E-1,3.149553E-2,1.2317148E-2,-2.126334E-1,-1.0569682E-1,-4.9034992E-1,-4.6106084E-4,1.7061938E-2,1.5949297E-1,-1.2741265E-1,-6.526266E-3,-1.51054375E-2,-6.074361E-1,-1.79534E-1,-4.9293123E-2,-3.6907598E-1,1.4304917E-3,2.8710915E-2,8.677636E-3,-5.0598397E-3,1.4346811E-3,-1.1604743E-2,-4.7129747E-3,-3.2481253E-2,-9.669161E-3,7.3323967E-3,-5.73213E-3,-1.7371178E-2,-3.7598596E-3,1.5589267E-3,-7.2063077E-3,9.731486E-4,-7.95197E-3,4.6239104E-3,1.2144572E-2,-6.01955E-5,-2.0438358E-2,-1.521743E-2,-5.4271477E-3,9.571553E-3,-2.0137613E-3,4.847699E-3,-4.2682453E-3,1.071768E-2,2.5033245E-3,1.9862695E-2,1.9241526E-3,5.3751995E-3,1.5339971E-2,-1.3109867E-2,-7.1731247E-3,4.7694944E-4,-7.910446E-3,-2.6578486E-2,-8.326242E-3,3.010049E-3,-3.0619202E-3,8.809152E-3,-4.2754523E-3,-7.481009E-3,1.7783456E-3,6.017749E-3,-1.8839579E-3,-3.2715898E-2,-1.5612066E-2,-2.097215E-3,-1.1669237E-2,9.051533E-3,-3.757066E-3,-5.438436E-3,-1.9238846E-2,6.4647617E-3,-1.0151259E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,-1,89,91,-1,-1,-1,93,-1,-1,95,97,-1,99,101,-1,103,105,107,-1,109,-1,111,113,115,-1,117,119,121,123,-1,-1,125,127,129,-1,131,133,135,137,-1,139,141,143,145,-1,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.25096035E1,1.0088957E1,4.359338E0,2.730918E0,5.2782707E0,3.2031279E0,7.6593655E-1,2.5914478E0,6.1218405E-1,2.7218497E0,2.8688717E0,3.0607061E0,2.2671213E0,2.0644438E-1,4.9182674E-1,1.4691052E0,1.1203716E0,2.2549498E-1,0E0,0E0,2.6587803E0,2.174201E0,1.2665772E0,9.1870975E-1,8.6287105E-1,6.0477066E-1,7.5291014E-1,2.9196537E-1,0E0,2.0604679E-1,1.16586596E-1,1.0091257E0,1.1774554E0,6.0661745E-1,4.7453332E-1,3.046983E-1,1.07749045E-1,1.4140512E0,9.80819E-1,1.0384102E0,4.395175E-1,2.7893543E-1,0E0,5.9056425E-1,7.223656E-1,4.0680912E-1,3.0037463E-1,9.156027E-1,0E0,3.709309E-1,3.1384897E-1,0E0,0E0,0E0,2.46853E-1,0E0,0E0,1.3801649E0,8.2057595E-1,0E0,1.6958523E0,5.266627E-1,0E0,2.1334726E-1,4.7926146E-1,1.506724E-1,0E0,1.412623E-1,0E0,8.907899E-1,4.6272945E-1,5.3935194E-1,0E0,8.31017E-1,1.4803481E0,2.3453617E-1,3.632605E-1,0E0,0E0,3.469453E-1,7.9276574E-1,2.1035218E-1,0E0,3.3782384E-1,1.745671E-1,2.6642317E-1,1.524097E-1,0E0,2.8864288E-1,6.074023E-1,2.5106937E-1,1.4625716E-1,0E0,1.8320295E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,54,54,57,57,58,58,60,60,61,61,63,63,64,64,65,65,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,80,80,81,81,83,83,84,84,85,85,86,86,88,88,89,89,90,90,91,91,93,93],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,-1,90,92,-1,-1,-1,94,-1,-1,96,98,-1,100,102,-1,104,106,108,-1,110,-1,112,114,116,-1,118,120,122,124,-1,-1,126,128,130,-1,132,134,136,138,-1,140,142,144,146,-1,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,3.3030225E2,3.4409692E3,2.5247778E6,1.5166431E5,4.987639E7,9.160584E-1,4.264897E6,2.0827537E0,8E0,8.791777E3,2.960909E2,6.7628815E2,1.7313588E7,3.103818E1,5.024605E0,2.53E2,4.5723195E6,1.4433761E-2,2.2502216E-2,1.3359244E9,4.391553E6,1.1896622E6,1.3142047E11,1.775044E7,3.1827815E12,1.2427474E8,8.838E3,4.208564E-3,1.3E1,1.341E3,5.372E3,1E0,1E0,2.5330253E-2,8.459239E6,3.2177323E-3,1E0,6.5880767E3,8.319452E2,1.6470588E0,4.6512E4,-8.262605E-3,1.074E3,2.0184135E0,1.51529E3,2.4559366E1,8.672049E9,-5.817708E-3,3.65625E1,6.8E1,-2.203609E-3,1.6143795E-2,-5.691363E-3,9.609747E3,-2.5300256E-3,-1.8988054E-2,4.16E2,8.74E2,2.6033907E-3,1E0,8.2465213E9,-1.2995703E-2,2.291E3,1.7E1,3E0,4.3919086E-3,1.7486666E3,2.05793E-3,2.7622065E1,7.937392E-3,2.8841177E10,-1.3702367E-2,2E0,1.9493858E7,2.1E1,1.2E1,3.149553E-2,1.2317148E-2,1.9121015E0,4.01E2,3.5799E4,-4.6106084E-4,9.669789E0,2.9933598E0,1E0,2.91E2,-1.51054375E-2,3.5791788E0,1.3E1,9.262083E2,8.7757526E2,1.4304917E-3,4.0601485E6,8.677636E-3,-5.0598397E-3,1.4346811E-3,-1.1604743E-2,-4.7129747E-3,-3.2481253E-2,-9.669161E-3,7.3323967E-3,-5.73213E-3,-1.7371178E-2,-3.7598596E-3,1.5589267E-3,-7.2063077E-3,9.731486E-4,-7.95197E-3,4.6239104E-3,1.2144572E-2,-6.01955E-5,-2.0438358E-2,-1.521743E-2,-5.4271477E-3,9.571553E-3,-2.0137613E-3,4.847699E-3,-4.2682453E-3,1.071768E-2,2.5033245E-3,1.9862695E-2,1.9241526E-3,5.3751995E-3,1.5339971E-2,-1.3109867E-2,-7.1731247E-3,4.7694944E-4,-7.910446E-3,-2.6578486E-2,-8.326242E-3,3.010049E-3,-3.0619202E-3,8.809152E-3,-4.2754523E-3,-7.481009E-3,1.7783456E-3,6.017749E-3,-1.8839579E-3,-3.2715898E-2,-1.5612066E-2,-2.097215E-3,-1.1669237E-2,9.051533E-3,-3.757066E-3,-5.438436E-3,-1.9238846E-2,6.4647617E-3,-1.0151259E-3],"split_indices":[27,52,52,28,28,45,57,9,53,18,52,52,52,1,54,53,10,28,0,0,7,28,28,31,45,31,45,2,0,3,0,29,66,65,38,45,38,14,57,52,54,10,0,2,53,4,56,5,0,56,3,0,0,0,52,0,0,2,2,0,102,31,0,2,3,8,0,33,0,57,42,5,0,6,51,3,3,0,0,53,0,10,0,54,39,105,0,0,54,3,52,52,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.678E3,1.977E3,7.01E2,8.36E2,1.141E3,6.18E2,8.3E1,7.42E2,9.4E1,4.71E2,6.7E2,4.33E2,1.85E2,2.6E1,5.7E1,5.63E2,1.79E2,8E1,1.4E1,1.2E1,4.59E2,6.49E2,2.1E1,2.32E2,2.01E2,5.8E1,1.27E2,1.8E1,8E0,5.3E1,4E0,5.16E2,4.7E1,9.3E1,8.6E1,4.4E1,3.6E1,3.62E2,9.7E1,5.79E2,7E1,1.9E1,2E0,2.14E2,1.8E1,1.14E2,8.7E1,5.1E1,7E0,1.03E2,2.4E1,2E0,1.6E1,5E0,4.8E1,2E0,2E0,3.02E2,2.14E2,9E0,3.8E1,9E1,3E0,1.1E1,7.5E1,1.6E1,2.8E1,2.6E1,1E1,3.11E2,5.1E1,9.3E1,4E0,3.35E2,2.44E2,3.4E1,3.6E1,1.5E1,4E0,1E2,1.14E2,1.4E1,4E0,8.9E1,2.5E1,5.4E1,3.3E1,2.8E1,2.3E1,7E1,3.3E1,2.2E1,2E0,3.3E1,1.5E1,1.67E2,1.35E2,5.3E1,1.61E2,1E1,2.8E1,8.3E1,7E0,5E0,6E0,5.7E1,1.8E1,8E0,8E0,1.5E1,1.1E1,3.07E2,4E0,1.7E1,3.4E1,8.3E1,1E1,3.11E2,2.4E1,1.72E2,7.2E1,3.2E1,2E0,1.8E1,1.8E1,4.9E1,5.1E1,3.9E1,7.5E1,1.1E1,3E0,5.7E1,3.2E1,2.3E1,2E0,4.6E1,8E0,6E0,2.7E1,1.7E1,6E0,2.3E1,4.7E1,3E0,3E1,3E0,1.9E1,1E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[2.3881637E-3,-9.557077E-2,4.910324E-2,-1.2084852E-1,7.416613E-2,8.779722E-3,1.4180945E-1,-8.720527E-2,-2.8043774E-1,1.4441279E-1,-1.5493113E-1,-9.081974E-2,3.9741147E-2,8.6879455E-2,2.4826741E-1,-1.5058519E-1,3.535118E-3,-2.2976854E-1,-4.8608428E-1,2.5196359E-2,1.0443652E-1,-7.8244574E-2,-2.760016E-2,2.1485205E-1,-1.03155054E-1,1.2522324E-2,1.2506633E-1,1.274118E-1,-7.0180963E-3,7.3524185E-2,2.880206E-1,-1.8908446E-1,-9.3372375E-2,5.124224E-2,-8.763057E-2,-4.63054E-1,-1.8967123E-1,-1.0670159E-2,-5.5749923E-1,2.9251019E-2,1.75241E-1,-1.7071894E-1,3.270513E-3,-1.5836745E-3,2.242638E-2,-1.21036105E-1,1.3165633E-1,8.782367E-3,3.3226635E-2,4.8096143E-2,1.655009E-1,9.487594E-2,2.5260225E-1,-8.892402E-2,5.9148263E-2,1.2534995E-1,-2.6654487E-3,1.9981887E-2,2.3814519E-1,-2.9976237E-1,-1.634389E-1,-5.3959325E-2,-2.4922591E-1,7.271923E-2,-1.1347116E-1,-2.0340782E-1,-5.3299487E-2,-2.4928471E-2,-8.374224E-3,-1.544753E-1,-3.37931E-1,-1.0755542E-2,-6.058839E-1,1.2255846E-1,-3.1635784E-2,2.694085E-1,1.3617378E-1,-1.097357E-3,-2.4233009E-1,-1.3195583E-1,8.432194E-3,1.733318E-2,2.6906496E-3,-1.205946E-1,2.4397044E-2,-4.6682186E-2,1.00836776E-1,2.3061757E-1,4.528177E-2,1.4823703E-1,5.0591905E-2,3.0478245E-1,1.4593229E-1,-1.3625433E-1,9.12702E-2,1.0224279E-1,-1.1114979E-1,1.0634754E-2,6.1085936E-2,2.0265873E-1,5.455519E-1,-1.5814949E-2,-9.786132E-4,-1.1502824E-3,-8.946975E-3,-4.359495E-3,6.2180805E-4,-1.8373866E-2,-4.272336E-3,1.0834874E-2,2.7698823E-3,-1.1349192E-3,-1.52766835E-2,-1.4361851E-3,-1.3198065E-2,-8.3153544E-4,-5.816487E-3,-1.1073316E-2,-2.9044272E-3,-5.106811E-3,-2.0373499E-2,-3.1819817E-2,-1.2650009E-2,1.4599806E-3,1.2975077E-2,-6.301207E-3,2.0957938E-3,2.4138685E-4,1.5688792E-2,1.411339E-3,8.237498E-3,-1.50540825E-2,-2.6686518E-3,-5.756288E-3,-1.40636545E-2,-7.2124125E-3,6.203269E-3,8.035083E-4,1.2694883E-2,-5.6949696E-3,1.1472185E-3,5.8515416E-3,-9.158331E-3,9.864568E-3,2.174665E-2,-4.659244E-3,3.7893988E-3,4.286363E-3,9.941699E-3,-5.4508564E-4,5.6757377E-3,1.5407113E-2,1.0809872E-3,2.5437484E-3,1.1645056E-2,8.197421E-3,-8.388856E-3,1.3458895E-2,-9.226298E-3,-3.345424E-3,7.5872005E-3,-9.338353E-3,6.046058E-3,8.501599E-3,-1.8280336E-3,1.941537E-2,8.06898E-3,1.2955044E-2,3.4944043E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,-1,-1,79,81,83,-1,85,87,89,91,93,95,97,-1,-1,99,101,103,105,107,109,111,113,115,-1,-1,117,119,-1,121,123,125,127,129,-1,131,133,-1,-1,-1,135,137,139,141,143,145,147,149,151,153,155,157,159,161,-1,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2287105E1,3.7288847E0,6.794876E0,4.029216E0,1.840919E0,3.9137928E0,3.200533E0,3.5949826E0,1.2533255E0,1.2726203E0,8.274024E-1,1.1571009E0,2.2465663E0,1.3900435E0,1.2778378E0,7.929201E-1,1.1265975E0,9.1306305E-1,3.4584284E-1,0E0,4.207515E-1,3.3687842E-1,0E0,7.6945835E-1,1.2335505E0,1.8666596E0,7.203076E-1,1.0146942E0,6.068149E-1,2.4799371E-1,8.802633E-1,5.726156E-1,9.0795135E-1,6.096347E-1,3.4291202E-1,1.574421E-1,4.330852E-1,0E0,1.6997719E-1,2.311249E-1,1.1414695E-1,1.5321466E-1,0E0,0E0,0E0,8.9920545E-1,3.44639E-1,1.4817135E0,0E0,4.1459817E-1,1.1872821E0,4.7456133E-1,2.4495745E-1,4.3886834E-1,4.6637696E-1,1.4803079E-1,0E0,0E0,1.1112213E0,3.3368373E-1,5.8230925E-1,3.014627E-1,6.111375E-1,3.3189648E-1,3.6144423E-1,2.4208409E-1,1.693358E-1,0E0,0E0,5.404495E-1,2.9863727E-1,0E0,1.4460516E-1,2.1031871E-1,1.941387E-1,1.6756356E-1,1.13459885E-1,0E0,1.168651E-1,4.609995E-1,0E0,0E0,0E0,6.124501E-1,1.1948441E0,1.5592131E-1,3.538438E-1,4.5150948E-1,2.7308688E-1,3.0214548E-1,4.770158E-1,1.5521073E-1,1.5854633E-1,5.0849515E-1,6.4680773E-1,4.9453622E-1,2.8337356E-1,0E0,2.0324592E-1,6.383705E-1,3.596351E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,72,72,73,73,74,74,75,75,76,76,78,78,79,79,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,98,98,99,99,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,-1,-1,80,82,84,-1,86,88,90,92,94,96,98,-1,-1,100,102,104,106,108,110,112,114,116,-1,-1,118,120,-1,122,124,126,128,130,-1,132,134,-1,-1,-1,136,138,140,142,144,146,148,150,152,154,156,158,160,162,-1,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.753919E3,3.474851E8,4.987639E7,1.0828989E8,7.805608E4,1.6789798E0,2.437E3,1.2427474E8,8.417E3,9.22E4,7E0,4.264897E6,2.3058404E7,1.0989723E3,3.68E0,1.9761398E7,3.2069644E2,8.8070023E-1,2.5196359E-2,1E0,3E1,-2.760016E-2,4.7E2,1.5106794E3,3.832274E2,9.52E2,4.1331047E-1,6.6692764E1,1.6474128E0,1.5277338E0,7.22E2,1.2342778E1,1.2342778E1,9.756616E2,8.92686E5,2.0295714E7,-1.0670159E-2,2.7324794E11,5.4786605E-1,6.211896E0,1.9589581E6,3.270513E-3,-1.5836745E-3,2.242638E-2,1.636E3,1.92E2,1.2E1,3.3226635E-2,5.4674416E7,2.4E1,5.68E2,5.2638875E6,6.923162E6,2.6025698E7,4.0248335E6,-2.6654487E-3,1.9981887E-2,1.9013363E4,2.5454E4,7.5408E4,1E0,2.8107285E-1,1E0,4.5154482E-1,2.65851E5,3.653602E7,-2.4928471E-2,-8.374224E-3,8.319452E2,1.2E1,-1.0755542E-2,1.4510472E1,5.033303E-1,5.4106476E7,3.7225406E4,7.6188426E0,-1.097357E-3,7.023838E3,9.55E0,8.432194E-3,1.733318E-2,2.6906496E-3,1.6943169E5,7.780377E6,2.04425E2,6.310469E6,1.4845161E3,1.1721691E5,4.1855645E-1,9.075E3,5.0738932E7,6.963859E1,9.4538574E2,1.30399E5,2.6477592E6,2.3553647E5,1.0634754E-2,2.185321E0,1.3062E4,1.6956E5,-1.5814949E-2,-9.786132E-4,-1.1502824E-3,-8.946975E-3,-4.359495E-3,6.2180805E-4,-1.8373866E-2,-4.272336E-3,1.0834874E-2,2.7698823E-3,-1.1349192E-3,-1.52766835E-2,-1.4361851E-3,-1.3198065E-2,-8.3153544E-4,-5.816487E-3,-1.1073316E-2,-2.9044272E-3,-5.106811E-3,-2.0373499E-2,-3.1819817E-2,-1.2650009E-2,1.4599806E-3,1.2975077E-2,-6.301207E-3,2.0957938E-3,2.4138685E-4,1.5688792E-2,1.411339E-3,8.237498E-3,-1.50540825E-2,-2.6686518E-3,-5.756288E-3,-1.40636545E-2,-7.2124125E-3,6.203269E-3,8.035083E-4,1.2694883E-2,-5.6949696E-3,1.1472185E-3,5.8515416E-3,-9.158331E-3,9.864568E-3,2.174665E-2,-4.659244E-3,3.7893988E-3,4.286363E-3,9.941699E-3,-5.4508564E-4,5.6757377E-3,1.5407113E-2,1.0809872E-3,2.5437484E-3,1.1645056E-2,8.197421E-3,-8.388856E-3,1.3458895E-2,-9.226298E-3,-3.345424E-3,7.5872005E-3,-9.338353E-3,6.046058E-3,8.501599E-3,-1.8280336E-3,1.941537E-2,8.06898E-3,1.2955044E-2,3.4944043E-2],"split_indices":[20,52,7,45,45,28,41,2,45,12,10,3,9,45,52,54,45,52,27,0,102,8,0,1,52,56,2,35,56,39,53,2,54,54,55,1,50,0,31,39,54,32,0,0,0,0,0,18,0,7,8,0,43,29,43,50,0,0,52,9,1,8,27,67,38,29,45,0,0,52,18,0,57,27,45,33,54,0,4,46,0,0,0,33,28,4,45,52,28,41,2,1,56,4,10,47,33,0,41,9,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,8.66E2,1.817E3,7.54E2,1.12E2,1.267E3,5.5E2,6.24E2,1.3E2,8.6E1,2.6E1,3E2,9.67E2,3.64E2,1.86E2,3.67E2,2.57E2,1.06E2,2.4E1,7E0,7.9E1,2.3E1,3E0,1.1E1,2.89E2,7.34E2,2.33E2,2.54E2,1.1E2,3.5E1,1.51E2,2.18E2,1.49E2,1.69E2,8.8E1,1.4E1,9.2E1,6E0,1.8E1,3.9E1,4E1,1.4E1,9E0,6E0,5E0,2.69E2,2E1,7.31E2,3E0,8.1E1,1.52E2,2.03E2,5.1E1,4.9E1,6.1E1,2.5E1,1E1,4E1,1.11E2,3.9E1,1.79E2,1.2E2,2.9E1,1.5E2,1.9E1,1.9E1,6.9E1,1.1E1,3E0,7.6E1,1.6E1,3E0,1.5E1,1.5E1,2.4E1,1E1,3E1,5E0,9E0,2.6E2,9E0,4E0,1.6E1,7.8E1,6.53E2,2.9E1,5.2E1,9.8E1,5.4E1,9.1E1,1.12E2,3.3E1,1.8E1,3.9E1,1E1,4.9E1,1.2E1,9E0,1.6E1,1.01E2,1E1,3.5E1,4E0,2.6E1,1.53E2,7.7E1,4.3E1,1.5E1,1.4E1,1.2E1,1.38E2,1.4E1,5E0,6E0,1.3E1,4.6E1,2.3E1,4.1E1,3.5E1,5E0,1.1E1,1.2E1,3E0,1E1,5E0,1E1,1.4E1,2E0,8E0,8E0,2.2E1,6E0,3E0,2.44E2,1.6E1,7E1,8E0,6.34E2,1.9E1,1.4E1,1.5E1,4.9E1,3E0,9E1,8E0,1E1,4.4E1,4.7E1,4.4E1,5.9E1,5.3E1,3.1E1,2E0,1E1,8E0,4E0,3.5E1,6E0,4E0,1.2E1,3.7E1,9E0,3E0,7E0,9E0,1.3E1,8.8E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-1.7043349E-3,-1.01871766E-1,4.5026742E-2,-1.4534566E-1,-2.0014042E-2,-1.0472367E-2,1.02395765E-1,-1.5360042E-1,3.198988E-1,-6.1272588E-2,7.360436E-2,-9.3126565E-2,1.9027775E-2,4.8570186E-2,1.611383E-1,-1.1913281E-1,-2.5563052E-1,2.0403912E-2,-3.3433423E-3,-4.6243764E-2,-1.2646888E-2,3.7049506E-2,1.6095517E-2,-1.2999433E-1,2.7345827E-2,1.4592828E-2,2.9614307E-2,1.03017795E-4,1.0783287E-1,2.0822385E-2,1.480638E-1,-1.5957874E-1,-4.5564314E-3,-3.9065E-1,-1.8534216E-1,-6.246214E-2,1.2533838E-1,-4.533743E-2,9.177847E-2,2.5322142E-1,-1.4630395E-1,2.2918869E-2,-6.529836E-2,-9.682538E-5,1.5125258E-1,-4.7874853E-2,1.303272E-1,1.2669975E-1,-1.3002592E-1,1.371924E-1,2.134815E-2,-2.0584288E-1,-1.0424374E-1,7.130941E-2,-9.074589E-2,-3.3783787E-1,-3.2873243E-2,-1.2154753E-1,-2.857635E-1,3.8481033E-3,-9.238765E-2,9.249305E-3,-1.4914662E-3,-4.412583E-3,3.425128E-3,1.15688734E-1,-3.266615E-3,3.5777825E-4,2.3806596E-2,-9.294487E-2,-2.3079544E-1,-1.4872979E-1,2.5394387E-3,8.470698E-3,-1.0125259E-2,1.0144756E-1,1.5755817E-2,-6.9717094E-2,7.094101E-2,1.9060145E-1,-2.9888535E-2,9.050476E-2,2.7008712E-1,3.9081164E-2,-3.5303235E-1,1.8254802E-1,7.978682E-2,-7.3868665E-3,-1.7333541E-2,-8.247547E-3,-2.9144012E-3,2.0590168E-3,1.300054E-2,-1.0256515E-3,-9.883833E-3,9.3349046E-4,-1.7080748E-2,5.5017318E-3,-6.76158E-3,-9.530306E-3,-2.4629956E-2,7.870523E-3,-1.2607207E-3,4.8867734E-3,-5.0337603E-3,4.161137E-3,1.02885775E-2,-8.52642E-3,7.726465E-4,-4.5746946E-4,-1.2054747E-2,-8.8999625E-3,-2.184501E-3,-2.022692E-3,2.393292E-3,6.6923546E-3,-8.149516E-3,-4.4535445E-3,4.304989E-3,6.8828417E-3,-3.2486126E-3,1.5161054E-2,6.8829376E-3,-3.4890994E-3,5.4396065E-3,3.3113577E-3,9.40047E-3,1.4731327E-2,-6.4694183E-3,-7.801616E-3,1.31051745E-2,-2.426871E-2,-5.3183828E-3,1.0382343E-2,8.226801E-4,3.0494467E-3,1.5187758E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,39,41,43,-1,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,-1,97,99,101,103,-1,-1,-1,-1,105,-1,-1,-1,107,109,111,-1,113,-1,115,-1,117,119,121,123,125,127,129,131,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2507357E1,3.021964E0,5.8021917E0,2.180605E0,1.1478276E0,2.2620857E0,2.8251915E0,1.8896828E0,4.4343865E-1,6.156642E-1,8.6070347E-1,1.0872927E0,1.8090976E0,1.347095E0,1.4669533E0,1.8992734E0,1.227766E0,0E0,0E0,5.4490614E-1,0E0,3.693861E-1,0E0,1.2058165E0,2.4558291E0,1.3651042E0,0E0,1.6244398E0,9.6336913E-1,0E0,1.2658014E0,7.542305E-1,7.126652E-1,4.866433E-1,5.5464983E-1,3.5158855E-1,1.8636471E-1,1.8707013E-1,1.9533414E-1,4.7576153E-1,7.9005885E-1,0E0,4.9161267E-1,1.115285E0,5.5089164E-1,4.9803475E-1,6.8167865E-1,9.9277663E-1,6.171884E-1,1.0190897E0,0E0,1.2650867E0,4.0241122E-1,3.17038E-1,4.0445212E-1,2.8273916E-1,0E0,2.8555024E-1,5.9163165E-1,2.755157E-1,3.124957E-1,0E0,0E0,0E0,0E0,1.04117334E-1,0E0,0E0,0E0,1.0424588E0,3.1694221E-1,1.0249442E-1,0E0,1.247972E0,0E0,5.7506615E-1,0E0,6.065151E-1,3.1274775E-1,2.556883E-1,1.3156761E-1,3.434503E-1,6.3439035E-1,5.183728E-1,2.0410544E-1,1.2478251E0,6.5665877E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,21,21,23,23,24,24,25,25,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,65,65,69,69,70,70,71,71,73,73,75,75,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,40,42,44,-1,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,-1,98,100,102,104,-1,-1,-1,-1,106,-1,-1,-1,108,110,112,-1,114,-1,116,-1,118,120,122,124,126,128,130,132,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.424835E6,1.1032986E8,8.230524E3,1.2554759E0,7.805608E4,2.9956698E-1,4.0844156E7,8.9712226E-1,1.1855755E8,1.19783E5,2.914E3,1.3664E4,7.06699E8,1.1485E4,2.9348E2,1.0908286E3,2.0403912E-2,-3.3433423E-3,3.0497742E3,-1.2646888E-2,7.204208E8,1.6095517E-2,7E0,3E0,4.5723195E6,2.9614307E-2,1.2677199E7,5.0714142E2,2.0822385E-2,1.6828056E1,1.1E1,1.6837112E7,1.13808104E8,1.0828989E8,9.1233586E4,5.1E1,3.3E1,1E0,4.7E2,4.9114623E0,2.2918869E-2,5.4E2,4.2343444E7,3.0608664E2,4.230839E6,3.63E2,9.3058E4,2.3580047E10,8E0,2.134815E-2,1.15062E5,1.11E3,2.8781365E6,4.1045683E8,4.107E3,-3.2873243E-2,5.684E3,5.2097E4,5.0461445E-2,2.22E2,9.249305E-3,-1.4914662E-3,-4.412583E-3,3.425128E-3,1.5859042E6,-3.266615E-3,3.5777825E-4,2.3806596E-2,3.3151623E1,1E0,7.5773336E7,2.5394387E-3,2.6659616E2,-1.0125259E-2,1.8E1,1.5755817E-2,1.8137958E6,4.0844156E7,1.5804776E4,1.4490348E9,4.1585636E1,5.68E2,4.90027E1,9.627E3,1E0,1.3674345E4,-7.3868665E-3,-1.7333541E-2,-8.247547E-3,-2.9144012E-3,2.0590168E-3,1.300054E-2,-1.0256515E-3,-9.883833E-3,9.3349046E-4,-1.7080748E-2,5.5017318E-3,-6.76158E-3,-9.530306E-3,-2.4629956E-2,7.870523E-3,-1.2607207E-3,4.8867734E-3,-5.0337603E-3,4.161137E-3,1.02885775E-2,-8.52642E-3,7.726465E-4,-4.5746946E-4,-1.2054747E-2,-8.8999625E-3,-2.184501E-3,-2.022692E-3,2.393292E-3,6.6923546E-3,-8.149516E-3,-4.4535445E-3,4.304989E-3,6.8828417E-3,-3.2486126E-3,1.5161054E-2,6.8829376E-3,-3.4890994E-3,5.4396065E-3,3.3113577E-3,9.40047E-3,1.4731327E-2,-6.4694183E-3,-7.801616E-3,1.31051745E-2,-2.426871E-2,-5.3183828E-3,1.0382343E-2,8.226801E-4,3.0494467E-3,1.5187758E-2],"split_indices":[20,32,7,52,39,28,38,45,27,45,10,2,2,7,9,52,4,0,0,52,0,12,0,3,8,28,0,9,58,0,35,3,45,45,45,33,3,3,104,1,56,0,0,45,52,1,0,2,5,17,0,29,2,47,7,12,0,9,10,27,11,0,0,0,0,28,0,0,0,55,79,7,0,52,0,3,0,28,45,47,5,53,3,56,0,6,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E3,8.49E2,1.821E3,5.54E2,2.95E2,9.26E2,8.95E2,5.45E2,9E0,2.05E2,9E1,2.43E2,6.83E2,4.68E2,4.27E2,4.09E2,1.36E2,7E0,2E0,1.92E2,1.3E1,8E1,1E1,1.86E2,5.7E1,6.79E2,4E0,2.58E2,2.1E2,1.8E1,4.09E2,3.02E2,1.07E2,4.5E1,9.1E1,1.76E2,1.6E1,3.2E1,4.8E1,7E0,1.79E2,9E0,4.8E1,6.14E2,6.5E1,1.89E2,6.9E1,1.95E2,1.5E1,3.96E2,1.3E1,1.63E2,1.39E2,5.7E1,5E1,4E1,5E0,5.7E1,3.4E1,5.5E1,1.21E2,1.1E1,5E0,2.3E1,9E0,4.2E1,6E0,4E0,3E0,1.11E2,6.8E1,2.8E1,2E1,5.91E2,2.3E1,5.2E1,1.3E1,1.6E2,2.9E1,5E1,1.9E1,1.57E2,3.8E1,9E0,6E0,2.2E2,1.76E2,1.24E2,3.9E1,5.3E1,8.6E1,5.1E1,6E0,3.2E1,1.8E1,2E0,3.8E1,4E0,5.3E1,2.6E1,8E0,8E0,4.7E1,7E0,1.14E2,3.4E1,8E0,6.2E1,4.9E1,6E0,6.2E1,2E1,8E0,2.66E2,3.25E2,4.6E1,6E0,1.4E2,2E1,1.9E1,1E1,1.2E1,3.8E1,1.5E1,4E0,1.32E2,2.5E1,3.5E1,3E0,5E0,4E0,3E0,3E0,1.82E2,3.8E1,1.66E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[3.6600014E-4,-9.0000324E-2,4.270803E-2,-1.3295661E-1,4.2422896E-4,3.193754E-3,1.15064375E-1,-1.4044258E-1,3.0853623E-1,1.0264827E-2,-1.1420575E-2,-3.3331245E-2,6.570521E-2,1.3753934E-1,-5.5307586E-2,-9.6795894E-2,-2.1410298E-1,4.1101402E-1,-3.236599E-3,-5.0370842E-2,5.1768117E-2,-3.7157886E-2,2.5992054E-2,9.161756E-2,-8.674673E-2,1.2211425E-1,3.3925453E-1,-1.0788301E-1,8.7527215E-2,-1.334115E-1,-2.3067346E-2,-2.541447E-1,-4.5978364E-2,2.3154907E-2,6.1800084E-3,5.571208E-3,-7.219319E-2,1.2606227E-1,9.0148905E-3,-6.9618985E-2,7.272048E-3,-8.873461E-3,1.3406385E-1,-2.985171E-2,-5.9383776E-2,1.6143043E-1,7.096846E-2,1.3421362E-2,2.108347E-2,-8.549321E-2,-2.274505E-2,1.5931176E-1,-5.6743626E-3,-5.3936925E-2,-1.7481942E-1,4.409454E-2,-1.2265409E-1,-2.1020396E-1,-3.7250754E-1,1.6508909E-2,-1.8664296E-1,-1.1447609E-2,-1.3657258E-1,6.673693E-2,2.201066E-1,-1.6391354E-2,1.5746386E-1,-5.577468E-2,-2.825929E-1,1.7293869E-1,-1.6763957E-2,-4.6827372E-2,1.2507924E-1,1.1566035E-1,1.6313186E-2,-1.7846546E-1,1.5709652E-3,7.308828E-2,2.2325622E-1,-1.7197456E-2,1.2861712E-1,-1.05558276E-1,6.333815E-3,2.2027093E-1,-3.7657998E-3,-1.9061929E-2,-1.5208513E-3,-1.034143E-2,-2.8651394E-3,6.4054615E-4,1.0345502E-2,-4.889726E-3,-1.9335803E-2,-2.1350281E-2,-8.8127395E-3,-2.201977E-2,-7.0106233E-3,-2.94504E-3,3.8205395E-3,-1.41563155E-2,-3.9077154E-3,2.49571E-3,-5.391128E-3,-1.077589E-2,-4.4171745E-3,4.8309406E-3,-3.3342168E-3,5.330021E-3,1.7475802E-2,3.8763832E-5,-7.2141523E-3,1.0145234E-2,-2.1568455E-3,8.29233E-3,-3.2024328E-3,-3.326295E-2,-1.0503451E-2,2.2041805E-2,5.686692E-3,-4.7360803E-4,-2.307186E-2,2.8772117E-3,-4.8078657E-3,-8.370184E-4,7.738746E-3,4.77222E-3,1.4328226E-2,-1.075212E-2,3.513805E-3,7.967701E-3,-2.0161609E-3,4.3595345E-3,-6.817637E-3,2.1918938E-2,9.661977E-3,-7.5876964E-3,1.9145287E-3,7.902323E-3,-2.665776E-3,-6.3729016E-3,2.6590808E-3,1.4639388E-2,3.3367507E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,-1,-1,61,63,65,67,69,71,73,-1,75,77,79,-1,-1,81,-1,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,-1,127,129,131,133,135,137,139,-1,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0277443E1,3.3289032E0,5.228943E0,1.9635372E0,6.536434E-1,2.7055357E0,2.481207E0,1.8182745E0,4.132166E-1,6.745343E-1,0E0,1.6690674E0,1.7342669E0,1.7352247E0,5.8017224E-1,9.7148395E-1,1.4154978E0,1.0111022E-1,0E0,4.049241E-1,5.0389755E-1,1.073536E0,0E0,1.5964761E0,9.026681E-1,1.05832E0,1.1496925E-1,4.1962034E-1,3.2943618E-1,7.836714E-1,8.147088E-1,8.1105995E-1,3.7225068E-1,0E0,0E0,0E0,3.7778062E-1,3.08905E-1,3.8727716E-1,1.2520828E0,1.2572178E0,5.752683E-1,9.604316E-1,0E0,4.5031315E-1,1.6222291E0,1.1864686E0,0E0,0E0,2.4989507E-1,0E0,2.4939978E-1,0E0,6.284517E-1,7.341747E-1,3.8266414E-1,2.4185264E-1,7.015004E-1,8.208599E-1,1.5295993E-1,1.2466821E-1,3.3366898E-1,1.6354054E-1,1.792569E-1,3.0602574E-1,2.0332743E-1,1.7471549E-1,1.040269E0,5.253737E-1,5.681915E-1,8.77515E-1,5.09962E-1,1.3295439E-1,6.8426704E-1,0E0,2.600069E-1,3.092768E-1,4.961785E-1,7.9495525E-1,7.573457E-1,9.5475316E-1,2.2898966E-1,0E0,1.4591998E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,49,49,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,81,81,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,-1,-1,62,64,66,68,70,72,74,-1,76,78,80,-1,-1,82,-1,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,-1,128,130,132,134,136,138,140,-1,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.0281758E6,9.772148E4,8.230524E3,1.1855755E8,4.5168175E5,1E0,6.856226E-1,8.9712226E-1,4.4844016E2,-1.1420575E-2,2.03148E2,3.7832818E0,7.096131E0,4.760537E8,2.914E3,6.654321E0,1.2003743E7,-3.236599E-3,1.8530114E-2,9.912942E6,3.131E3,2.5992054E-2,1E0,1.198E5,1.3226592E-2,1.7314173E1,1.566593E6,2.6878788E1,4.854E4,3.653602E7,1E0,1.1E1,2.3154907E-2,6.1800084E-3,5.571208E-3,5.650399E6,1.6525185E9,5.528107E3,4.004366E1,1.03E2,1.408E3,8.5023944E2,-2.985171E-2,1.6437647E-4,5.862126E2,4.894913E6,1.3421362E-2,2.108347E-2,4E1,-2.274505E-2,2.9375E0,-5.6743626E-3,1.04661034E-4,2E1,1.1598511E3,2.6572757E10,4.36E2,1.4510472E1,9.159699E7,5.37456E5,7.7854166E0,1.998E3,1.0931163E9,1.6594656E3,9.36494E5,1.766486E7,7E0,5.4210526E1,1E0,7.074777E6,2.294901E0,4.8858527E9,6.057671E6,1.6313186E-2,4.423676E1,5.6656016E3,1E0,1.0918E4,7.5935423E-1,2.6025698E7,1.5088776E0,6.333815E-3,7.1913286E8,-3.7657998E-3,-1.9061929E-2,-1.5208513E-3,-1.034143E-2,-2.8651394E-3,6.4054615E-4,1.0345502E-2,-4.889726E-3,-1.9335803E-2,-2.1350281E-2,-8.8127395E-3,-2.201977E-2,-7.0106233E-3,-2.94504E-3,3.8205395E-3,-1.41563155E-2,-3.9077154E-3,2.49571E-3,-5.391128E-3,-1.077589E-2,-4.4171745E-3,4.8309406E-3,-3.3342168E-3,5.330021E-3,1.7475802E-2,3.8763832E-5,-7.2141523E-3,1.0145234E-2,-2.1568455E-3,8.29233E-3,-3.2024328E-3,-3.326295E-2,-1.0503451E-2,2.2041805E-2,5.686692E-3,-4.7360803E-4,-2.307186E-2,2.8772117E-3,-4.8078657E-3,-8.370184E-4,7.738746E-3,4.77222E-3,1.4328226E-2,-1.075212E-2,3.513805E-3,7.967701E-3,-2.0161609E-3,4.3595345E-3,-6.817637E-3,2.1918938E-2,9.661977E-3,-7.5876964E-3,1.9145287E-3,7.902323E-3,-2.665776E-3,-6.3729016E-3,2.6590808E-3,1.4639388E-2,3.3367507E-3],"split_indices":[20,32,33,52,45,28,14,27,27,52,0,56,53,35,7,2,54,1,0,39,45,2,0,102,7,57,35,9,56,29,45,101,8,0,0,0,45,5,52,53,11,0,52,0,38,52,1,0,0,3,0,54,0,41,3,52,12,2,57,7,28,54,10,5,4,9,47,3,53,8,47,56,5,28,0,56,47,65,9,42,43,38,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.684E3,8.56E2,1.828E3,5.8E2,2.76E2,1.183E3,6.45E2,5.71E2,9E0,2.66E2,1E1,7.47E2,4.36E2,5.7E2,7.5E1,3.6E2,2.11E2,7E0,2E0,1.08E2,1.58E2,7.43E2,4E0,3.73E2,6.3E1,5.31E2,3.9E1,5.5E1,2E1,2.4E2,1.2E2,1.7E2,4.1E1,5E0,2E0,1.2E1,9.6E1,5.7E1,1.01E2,4.29E2,3.14E2,1.11E2,2.62E2,2E0,6.1E1,2.99E2,2.32E2,2.7E1,1.2E1,5.3E1,2E0,1.5E1,5E0,8.3E1,1.57E2,7.2E1,4.8E1,1.26E2,4.4E1,2.9E1,1.2E1,5E1,4.6E1,3.6E1,2.1E1,8.7E1,1.4E1,4.04E2,2.5E1,3.9E1,2.75E2,8.7E1,2.4E1,2.42E2,2E1,2E1,4.1E1,1.24E2,1.75E2,9.2E1,1.4E2,4.9E1,4E0,1.2E1,3E0,4E0,7.9E1,1.15E2,4.2E1,6.2E1,1E1,4.6E1,2E0,1.1E1,1.15E2,3.1E1,1.3E1,1.3E1,1.6E1,5E0,7E0,3.1E1,1.9E1,1.4E1,3.2E1,2.9E1,7E0,1.3E1,8E0,7.8E1,9E0,1.1E1,3E0,1.8E1,3.86E2,2E0,2.3E1,5E0,3.4E1,2.72E2,3E0,2.9E1,5.8E1,5E0,1.9E1,2.24E2,1.8E1,1.7E1,3E0,8E0,3.3E1,1.15E2,9E0,1.3E1,1.62E2,2.6E1,6.6E1,1.17E2,2.3E1,4.2E1,7E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[2.3761154E-3,3.6150422E-2,-9.07322E-2,-7.7931467E-3,1.116759E-1,-1.24647185E-1,6.4713754E-2,4.9105242E-2,-4.7877047E-2,3.223874E-2,1.820868E-1,-1.6664198E-1,-3.599859E-2,1.19691126E-1,-3.2581102E-2,6.4705454E-2,-3.386753E-1,-6.3339174E-2,4.550474E-2,4.0242793E-3,1.9204178E-1,2.9948002E-1,1.4696755E-1,-1.437786E-1,-3.6653155E-1,-1.3633028E-1,-9.846709E-3,1.6602917E-2,7.9640225E-2,-1.2695941E-1,3.5995588E-2,9.088688E-2,-6.4166844E-2,3.112114E-3,-5.163138E-1,-8.714618E-2,-1.3419466E-2,1.4166014E-1,-2.4599224E-2,1.8645272E-2,-9.702027E-3,2.4595402E-1,-1.2848076E-1,3.3136705E-1,-1.3692562E-2,1.2962659E-1,3.380972E-1,-1.3445573E-1,-4.248021E-1,-4.2190802E-1,8.4222294E-4,-8.5769184E-2,-1.163478E-2,-6.1884284E-2,2.2366336E-2,-1.0151187E-2,9.1164574E-2,-1.1953857E-2,-1.9978646E-3,7.144044E-2,-9.316321E-3,6.671786E-2,1.9161342E-1,-9.556616E-2,6.520495E-2,-3.2015383E-2,1.4994002E-3,-5.1636282E-2,-1.5779974E-1,-5.6298796E-2,6.709279E-2,1.204933E-2,6.3119225E-2,-1.003543E-2,7.2715334E-3,-8.751904E-2,3.5818603E-2,-7.8382984E-2,3.0943352E-1,-1.805706E-2,8.2451233E-4,2.2330992E-1,4.420784E-1,-1.693609E-2,1.33956E-1,1.5175901E-1,1.9973706E-2,2.0520618E-2,-4.3625194E-3,-1.8730003E-1,-1.0073802E-1,-3.010783E-2,-2.6058385E-3,-3.0291292E-1,-5.9998333E-1,-1.0750909E-1,3.4264494E-3,-4.51819E-2,-1.1009709E-2,4.31852E-2,-3.9186296E-3,9.627861E-3,7.1332775E-2,1.4402388E-2,1.8402552E-3,-1.7571129E-2,3.4371847E-3,7.0657423E-3,1.5470044E-2,-1.1654082E-3,-7.953744E-3,5.774435E-4,9.115826E-3,1.5161066E-2,-3.041008E-3,-1.2441723E-2,-5.137853E-3,2.1789765E-3,-3.8448637E-3,6.6124606E-3,-8.933309E-5,5.491202E-3,-3.5260103E-3,2.2230586E-3,-9.064561E-3,-5.696863E-3,2.4167304E-3,2.1423101E-3,-1.3485144E-2,4.403998E-3,-1.3012604E-2,1.04654785E-2,3.198155E-2,1.2918087E-2,-4.800513E-3,2.3689631E-2,1.3741009E-2,-6.599679E-4,1.1886611E-2,5.723315E-3,1.3934731E-2,-9.358705E-3,2.1337545E-3,-7.6022935E-3,-1.5586693E-2,9.958256E-4,-7.204988E-3,-4.011593E-3,-1.5880343E-2,-3.307813E-2,-1.5166525E-2,-2.6509203E-3,-1.0125017E-2,-7.1334117E-3,-1.1444285E-3,3.2335336E-3,-9.098512E-4,4.266838E-3,-2.6465668E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,55,57,59,61,63,-1,65,67,69,71,73,-1,75,77,79,81,83,85,87,89,91,93,-1,95,-1,97,99,-1,101,-1,-1,103,-1,105,107,109,111,-1,-1,113,115,117,119,-1,121,-1,123,125,127,129,131,-1,-1,133,135,-1,137,139,141,-1,-1,143,145,-1,-1,147,149,151,-1,153,-1,155,-1,-1,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.399546E0,6.5058346E0,3.756421E0,2.8304305E0,4.0268564E0,2.1681032E0,6.885736E-1,3.1252866E0,1.0540278E0,1.5322044E0,1.5308809E0,1.7570801E0,4.9456757E-1,7.178763E-1,3.0876112E-1,1.6726179E0,1.4676602E0,7.4160457E-1,7.0453566E-1,1.5375407E0,9.128847E-1,8.7333155E-1,9.372268E-1,8.7820435E-1,8.5826397E-1,1.8939507E-1,2.5450376E-1,0E0,2.5792474E-1,1.9868335E-1,2.426877E-1,9.877796E-1,3.4745502E-1,0E0,1.1877887E0,1.0531168E0,7.045966E-1,3.640545E-1,3.6108822E-1,0E0,9.985154E-1,9.301598E-1,2.962169E-1,8.463688E-1,4.9910942E-1,6.5950775E-1,9.701314E-1,5.9941196E-1,7.9644954E-1,5.599108E-1,0E0,1.0345341E-1,0E0,1.5236966E-1,2.0743275E-1,0E0,1.360107E-1,0E0,0E0,1.7769355E-1,0E0,7.560817E-1,4.0990996E-1,3.375169E-1,1.1077152E-1,0E0,0E0,1.2562664E0,6.986232E-1,3.2839704E-1,3.4778592E-1,0E0,1.987432E-1,0E0,4.1659343E-1,4.571284E-1,5.1626354E-1,2.8844914E-1,1.0594347E0,0E0,0E0,6.580069E-1,1.6809177E-1,0E0,1.2157057E-1,9.843831E-1,2.575524E-1,0E0,0E0,4.770398E-1,1.2959499E0,0E0,0E0,1.1680055E-1,1.1536217E-1,1.2378484E-1,0E0,1.15714125E-1,0E0,1.2082423E-1,0E0,0E0,1.3984582E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,56,56,59,59,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,72,72,74,74,75,75,76,76,77,77,78,78,81,81,82,82,84,84,85,85,86,86,89,89,90,90,93,93,94,94,95,95,97,97,99,99,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,56,58,60,62,64,-1,66,68,70,72,74,-1,76,78,80,82,84,86,88,90,92,94,-1,96,-1,98,100,-1,102,-1,-1,104,-1,106,108,110,112,-1,-1,114,116,118,120,-1,122,-1,124,126,128,130,132,-1,-1,134,136,-1,138,140,142,-1,-1,144,146,-1,-1,148,150,152,-1,154,-1,156,-1,-1,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,6.218596E2,2.0601392E3,2E0,1.5166431E5,1E0,6.971004E7,9.669789E0,9.750871E6,3.317425E3,1.0993947E1,1.0017953E8,2.067695E4,8.628E3,3.4570816E0,1E0,3.1E1,2.373E3,4.1E2,2.04E5,2.3580047E10,2.0295714E7,6.7652373E3,4.5723195E6,1.0052E4,1.48614E5,6.8177136E2,1.6602917E-2,9.877E3,3.0497742E3,2.12E2,7.0025E4,1.75E2,3.112114E-3,6.743895E5,2.2415E4,2.1924436E5,7.73756E0,1.322E3,1.8645272E-2,5.794E3,5.0279167E1,9.627E3,1.0201212E3,1.0855529E3,2.2807484E7,1.1896622E6,1.445E3,3E0,1.2905E4,8.4222294E-4,6.1E1,-1.163478E-2,1.9719212E1,3.0996E4,-1.0151187E-2,7.016421E5,-1.1953857E-2,-1.9978646E-3,5.491E3,-9.316321E-3,1.198E5,4.681779E2,2.20524E5,6.211896E0,-3.2015383E-2,1.4994002E-3,6E0,2.4063041E4,9.7549E4,4.7022E4,1.204933E-2,3.271242E0,-1.003543E-2,1.5548611E7,7.704183E-1,8.6206274E2,3.7137984E7,1.5005797E4,-1.805706E-2,8.2451233E-4,2E0,2.1298597E0,-1.693609E-2,1.0215407E1,5.3515047E-1,4.709E3,2.0520618E-2,-4.3625194E-3,1.087037E1,1.1662405E9,-3.010783E-2,-2.6058385E-3,1.4977E5,1.2659678E3,1.17E2,3.4264494E-3,1.009E3,-1.1009709E-2,7.2530017E0,-3.9186296E-3,9.627861E-3,1E0,1.4402388E-2,1.8402552E-3,-1.7571129E-2,3.4371847E-3,7.0657423E-3,1.5470044E-2,-1.1654082E-3,-7.953744E-3,5.774435E-4,9.115826E-3,1.5161066E-2,-3.041008E-3,-1.2441723E-2,-5.137853E-3,2.1789765E-3,-3.8448637E-3,6.6124606E-3,-8.933309E-5,5.491202E-3,-3.5260103E-3,2.2230586E-3,-9.064561E-3,-5.696863E-3,2.4167304E-3,2.1423101E-3,-1.3485144E-2,4.403998E-3,-1.3012604E-2,1.04654785E-2,3.198155E-2,1.2918087E-2,-4.800513E-3,2.3689631E-2,1.3741009E-2,-6.599679E-4,1.1886611E-2,5.723315E-3,1.3934731E-2,-9.358705E-3,2.1337545E-3,-7.6022935E-3,-1.5586693E-2,9.958256E-4,-7.204988E-3,-4.011593E-3,-1.5880343E-2,-3.307813E-2,-1.5166525E-2,-2.6509203E-3,-1.0125017E-2,-7.1334117E-3,-1.1444285E-3,3.2335336E-3,-9.098512E-4,4.266838E-3,-2.6465668E-3],"split_indices":[27,52,52,17,28,59,45,54,9,52,56,45,48,9,57,14,0,2,0,5,5,50,52,28,2,12,4,0,9,52,8,1,6,0,28,29,28,58,2,0,2,56,0,4,52,51,28,2,8,10,0,3,0,56,9,0,32,0,0,9,0,7,55,9,54,0,0,3,33,12,9,0,53,0,45,41,58,1,4,0,0,17,38,0,56,35,2,0,0,58,5,0,0,29,52,10,0,2,0,53,0,0,104,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.669E3,1.959E3,7.1E2,1.239E3,7.2E2,5.83E2,1.27E2,5.12E2,7.27E2,3.39E2,3.81E2,3.95E2,1.88E2,8.1E1,4.6E1,4.93E2,1.9E1,6.24E2,1.03E2,2.89E2,5E1,8.6E1,2.95E2,3.56E2,3.9E1,3.8E1,1.5E2,1.1E1,7E1,1.9E1,2.7E1,4.1E2,8.3E1,6E0,1.3E1,4.22E2,2.02E2,4.3E1,6E1,9E0,2.8E2,4.3E1,7E0,7.8E1,8E0,2.72E2,2.3E1,3.46E2,1E1,3.4E1,5E0,2.7E1,1.1E1,5.7E1,9.3E1,2E0,6.8E1,7E0,1.2E1,2.4E1,3E0,3.32E2,7.8E1,6.7E1,1.6E1,1E1,3E0,2.82E2,1.4E2,1.32E2,7E1,1.7E1,2.6E1,8E0,5.2E1,1.03E2,1.77E2,7E0,3.6E1,2E0,5E0,4.1E1,3.7E1,2E0,6E0,2.26E2,4.6E1,1.9E1,4E0,1.33E2,2.13E2,6E0,4E0,2.2E1,1.2E1,2.4E1,3E0,5.3E1,4E0,7.8E1,1.5E1,9E0,5.9E1,2E0,2.2E1,3E0,3.29E2,6E1,1.8E1,3.4E1,3.3E1,1.2E1,4E0,8E0,2.74E2,4.5E1,9.5E1,2.5E1,1.07E2,3.4E1,3.6E1,1.9E1,7E0,4.4E1,8E0,8.4E1,1.9E1,1.73E2,4E0,4E0,3E0,3E1,6E0,3.6E1,5E0,2.6E1,1.1E1,3E0,3E0,1.85E2,4.1E1,4E0,4.2E1,1.12E2,2.1E1,6.2E1,1.51E2,3E0,1.9E1,8E0,4E0,1.7E1,7E0,8E0,4.5E1,5.6E1,2.2E1,5.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[-2.689658E-3,-8.5344225E-2,3.592615E-2,-1.1440159E-1,2.2573067E-2,1.7378945E-4,1.0457496E-1,-6.312117E-2,-1.97686E-1,8.375912E-2,-1.4998297E-1,-4.9539793E-2,4.019781E-2,7.0687994E-2,1.81382E-1,-1.06992304E-1,3.0657876E-2,-7.701161E-2,-2.5411296E-1,4.4086424E-1,6.078568E-2,2.4764931E-3,-1.978538E-1,-3.7179343E-2,-2.5009495E-1,2.8207088E-2,2.4958497E-1,1.0112072E-1,-5.4341935E-2,6.739604E-2,2.1707827E-1,-1.1684477E-1,2.1859212E-3,4.5795154E-2,-7.7402536E-3,-9.852361E-2,1.10811956E-1,-2.874993E-1,-8.42752E-2,6.7936885E-3,2.4754122E-2,-1.0197091E-2,1.3192779E-1,-2.5078878E-1,-4.0311243E-2,-3.3013564E-2,-2.1494266E-2,-1.3010123E-1,-4.890716E-1,-1.0608999E-1,4.5123253E-2,2.261786E-2,1.3270679E-1,1.3858387E-2,1.475493E-1,-2.2095827E-2,-3.4070827E-2,1.9485065E-1,1.2932489E-2,1.9979894E-1,2.337115E-2,-8.573453E-2,-2.2559574E-1,5.3540813E-3,1.7841082E-2,-1.4515683E-1,1.3219172E-2,1.2624245E-2,-4.5589724E-4,-2.628267E-1,-4.9959093E-1,-8.308785E-3,1.6511317E-2,7.0440575E-2,-8.579145E-2,2.8853709E-1,1.01054676E-1,2.0844187E-3,-2.8642204E-1,-5.21243E-3,5.11591E-3,-6.0623802E-2,4.504721E-2,-1.068303E-2,4.8364103E-3,-2.8578794E-2,-1.862889E-3,2.9247839E-3,-1.4821883E-1,7.1820416E-3,9.90604E-2,8.438104E-3,-3.66304E-3,-2.6223424E-1,3.7955474E-2,1.8579046E-1,7.113639E-2,7.04476E-2,-6.836949E-2,2.3692087E-3,3.3316264E-1,1.0490476E-1,-3.5749655E-3,2.0932716E-1,-1.430713E-2,-7.178461E-3,-2.4162561E-3,-1.2608592E-2,-7.0005615E-4,1.2461193E-3,-1.0789694E-2,-1.1479266E-2,-4.4947234E-3,7.237082E-3,-5.003632E-3,-1.7370151E-2,-9.158028E-3,-2.8337546E-2,-1.0316811E-2,-3.5628073E-3,7.914851E-3,7.6364656E-3,7.767811E-4,-7.523199E-3,7.181083E-4,1.7752256E-2,-1.2786944E-3,2.0164715E-3,7.910838E-3,-1.6982008E-2,-8.492452E-3,9.469461E-3,-3.4136088E-3,4.8681027E-3,-1.077626E-3,-1.2106488E-2,-3.3543E-3,-1.3995846E-2,8.3427527E-4,1.0073852E-2,2.4727145E-3,-5.833648E-3,-2.1187464E-2,-1.01687275E-2,2.6969023E-3,2.510672E-2,7.941946E-3,1.5698733E-3,1.1017025E-2,-4.602676E-5,1.1930944E-2,-4.1452902E-3,5.6914943E-3,2.4354674E-3,2.0876192E-2,4.5670517E-4,9.032969E-3,7.4701076E-3,1.225837E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,51,53,55,57,59,61,-1,63,-1,65,67,69,71,-1,-1,73,75,77,79,81,-1,83,85,87,89,-1,91,93,95,-1,97,99,101,103,-1,105,107,-1,109,111,113,-1,-1,115,117,-1,119,121,123,125,127,-1,129,-1,-1,131,133,-1,-1,-1,-1,-1,135,137,139,-1,-1,141,143,145,147,149,151,-1,153,155,-1,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.560209E0,2.6805487E0,4.4865465E0,2.8618097E0,1.9341229E0,2.3956456E0,1.6165047E0,1.7230127E0,1.723608E0,1.0807326E0,4.736848E-1,1.3218992E0,1.6675988E0,1.6644757E0,7.625246E-1,4.3392944E-1,3.9742497E-1,3.4627593E-1,9.623966E-1,1.0853076E-1,6.477278E-1,0E0,3.1513143E-1,8.5458976E-1,8.253176E-1,1.4405594E0,8.743501E-1,1.4200201E0,6.9387937E-1,3.2373673E-1,5.8132076E-1,8.873749E-1,0E0,2.2863752E-1,0E0,3.9273828E-1,1.6798317E-1,6.2745667E-1,2.7263033E-1,0E0,0E0,4.0226716E-1,2.8028297E-1,3.165995E-1,1.2254145E-1,1.086835E0,0E0,4.9518853E-1,4.6153688E-1,5.0905895E-1,1.1493883E0,0E0,2.4091762E-1,8.2827574E-1,6.5594864E-1,0E0,3.023584E-1,2.650258E-1,2.817024E-1,7.1807003E-1,0E0,4.6778595E-1,4.6522617E-1,0E0,1.8285318E-1,2.3456502E-1,3.890917E-1,0E0,0E0,8.745918E-1,2.3902893E-1,0E0,2.1514085E-1,1.5186197E-1,2.466819E-1,2.6885527E-1,2.0096982E-1,0E0,1.3049841E-1,0E0,0E0,1.0381832E0,5.069903E-1,0E0,0E0,0E0,0E0,0E0,4.4829988E-1,1.0198298E0,1.2137032E0,0E0,0E0,1.9912136E-1,5.357752E-1,9.3967247E-1,4.678789E-1,2.6818764E-1,2.2480538E-1,0E0,1.8088567E-1,1.3392068E-1,0E0,2.9843283E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,53,53,54,54,56,56,57,57,58,58,59,59,61,61,62,62,64,64,65,65,66,66,69,69,70,70,72,72,73,73,74,74,75,75,76,76,78,78,81,81,82,82,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,100,100,101,101,103,103],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,52,54,56,58,60,62,-1,64,-1,66,68,70,72,-1,-1,74,76,78,80,82,-1,84,86,88,90,-1,92,94,96,-1,98,100,102,104,-1,106,108,-1,110,112,114,-1,-1,116,118,-1,120,122,124,126,128,-1,130,-1,-1,132,134,-1,-1,-1,-1,-1,136,138,140,-1,-1,142,144,146,148,150,152,-1,154,156,-1,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.0485785E3,9.772148E4,1.4540612E7,9.302862E7,1.7669278E5,1.4101E4,2.785857E2,2.6432162E5,3.541262E-5,3.046556E5,1E0,9.0754684E1,1.4274633E7,1E0,4.653159E9,1.044E1,5.189068E-1,6.568389E0,7.678E3,1E0,2.4764931E-3,8.2619705E1,7.074777E6,7.492813E0,1E0,1.8631586E4,1E0,5.1776E4,5.7075036E-1,4.2620764E2,1.0508E5,2.1859212E-3,4.1730586E-1,-7.7402536E-3,3.138649E6,9.8708276E2,1.2427474E8,9.538462E0,6.7936885E-3,2.4754122E-2,4.5158855E6,5.015707E0,1.7560976E0,4.73699E0,5.513E3,-2.1494266E-2,1.6812695E-2,1.752E3,1.6254681E0,1.146E3,2.261786E-2,4.8718534E0,4.97E2,4.4E0,-2.2095827E-2,3.53562E5,4.1043E4,2.589369E7,6.13912E5,2.337115E-2,9E2,3.2E1,5.3540813E-3,1E0,1.3E1,5.904E3,1.2624245E-2,-4.5589724E-4,3.0608664E2,4.822581E0,-8.308785E-3,9.3299186E-1,5.12861E5,8.454452E6,2.9956698E-1,3.368326E11,2.0844187E-3,5.2768206E5,-5.21243E-3,5.11591E-3,7E0,5.244755E-3,-1.068303E-2,4.8364103E-3,-2.8578794E-2,-1.862889E-3,2.9247839E-3,2.7754388E-4,1.198E5,3.1753032E0,8.438104E-3,-3.66304E-3,5.266129E0,1.5988182E-1,1.0918E4,1.00663277E9,5.012E3,1.7022566E7,2.3692087E-3,8.422367E6,1.7313588E7,-3.5749655E-3,2.1446484E-1,-1.430713E-2,-7.178461E-3,-2.4162561E-3,-1.2608592E-2,-7.0005615E-4,1.2461193E-3,-1.0789694E-2,-1.1479266E-2,-4.4947234E-3,7.237082E-3,-5.003632E-3,-1.7370151E-2,-9.158028E-3,-2.8337546E-2,-1.0316811E-2,-3.5628073E-3,7.914851E-3,7.6364656E-3,7.767811E-4,-7.523199E-3,7.181083E-4,1.7752256E-2,-1.2786944E-3,2.0164715E-3,7.910838E-3,-1.6982008E-2,-8.492452E-3,9.469461E-3,-3.4136088E-3,4.8681027E-3,-1.077626E-3,-1.2106488E-2,-3.3543E-3,-1.3995846E-2,8.3427527E-4,1.0073852E-2,2.4727145E-3,-5.833648E-3,-2.1187464E-2,-1.01687275E-2,2.6969023E-3,2.510672E-2,7.941946E-3,1.5698733E-3,1.1017025E-2,-4.602676E-5,1.1930944E-2,-4.1452902E-3,5.6914943E-3,2.4354674E-3,2.0876192E-2,4.5670517E-4,9.032969E-3,7.4701076E-3,1.225837E-2],"split_indices":[20,52,33,45,45,28,2,52,28,39,32,6,56,45,102,5,54,27,54,2,102,0,56,47,54,66,33,100,1,41,58,29,0,39,0,29,4,45,58,0,0,47,54,56,57,2,0,38,29,53,2,0,53,2,54,0,1,10,45,2,0,2,3,0,103,3,0,0,0,52,56,0,27,29,32,38,31,0,28,0,0,3,57,0,0,0,0,0,41,7,54,0,0,54,38,9,7,2,9,0,50,1,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E3,8.53E2,1.827E3,6.72E2,1.81E2,1.202E3,6.25E2,4.17E2,2.55E2,1.34E2,4.7E1,5.36E2,6.66E2,4.35E2,1.9E2,2.84E2,1.33E2,8.2E1,1.73E2,7E0,1.27E2,9E0,3.8E1,5.06E2,3E1,6.31E2,3.5E1,3.5E2,8.5E1,4.6E1,1.44E2,2.67E2,1.7E1,1.24E2,9E0,7.4E1,8E0,1.44E2,2.9E1,2E0,5E0,6.4E1,6.3E1,2.8E1,1E1,5.02E2,4E0,2.1E1,9E0,7E1,5.61E2,1.1E1,2.4E1,1.22E2,2.28E2,3E0,8.2E1,1.3E1,3.3E1,1.37E2,7E0,2.09E2,5.8E1,3.6E1,8.8E1,5.2E1,2.2E1,3E0,5E0,1.31E2,1.3E1,1.5E1,1.4E1,3.1E1,3.3E1,9E0,5.4E1,3E0,2.5E1,7E0,3E0,3.71E2,1.31E2,1.5E1,6E0,7E0,2E0,1.4E1,5.6E1,3.3E2,2.31E2,2E1,4E0,9E0,1.13E2,1.51E2,7.7E1,2E1,6.2E1,7E0,6E0,1.6E1,1.7E1,1.35E2,2E0,7.3E1,1.36E2,4.9E1,9E0,8.6E1,2E0,1.7E1,3.5E1,1E1,1.2E1,5.3E1,7.8E1,9E0,4E0,9E0,5E0,1.1E1,2E1,1.9E1,1.4E1,7E0,2E0,2.9E1,2.5E1,1.4E1,1.1E1,1.4E1,3.57E2,7.1E1,6E1,2.3E1,3.3E1,1E1,3.2E2,6.8E1,1.63E2,6E0,3E0,7E0,1.06E2,7E0,1.44E2,6.3E1,1.4E1,1.5E1,5E0,5.7E1,5E0,2E0,4E0,8E0,8E0,6.5E1,7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[5.0597626E-4,-8.41867E-2,4.101294E-2,-1.1360305E-1,2.9799454E-2,-8.410059E-2,6.045761E-2,-8.526692E-2,-2.5281742E-1,1.850927E-1,-2.4543645E-2,-7.097548E-2,-3.2975057E-1,8.762305E-2,-1.4870629E-2,-1.3224685E-1,-2.2598581E-2,-3.4586138E-1,-1.759242E-1,3.1965503E-1,8.717209E-2,-6.1389055E-2,1.5274066E-1,-1.0346897E-2,-5.467041E-2,-6.0777594E-3,-2.817622E-2,4.2938143E-2,1.4479825E-1,-5.7743948E-2,9.808434E-2,-8.080971E-2,-1.8225032E-1,-6.9750585E-2,1.040108E-2,-3.7848467E-1,-1.8249024E-3,-8.110903E-2,-2.5005007E-1,4.767929E-3,1.9806543E-2,1.228576E-3,1.0257844E-2,-4.8832316E-2,-1.8291049E-2,2.1164343E-3,1.190366E-2,-1.1119661E-2,-1.1326558E-1,-4.7590703E-2,7.5056955E-2,4.1716534E-1,1.3260593E-1,-2.9200855E-1,-4.1755397E-2,1.6251493E-1,-8.11238E-3,-5.8081806E-2,-3.3683863E-1,-2.1987046E-1,-7.465284E-2,-3.9180946E-2,-8.82817E-3,4.5421574E-2,-1.0526645E-1,-2.2318123E-1,-4.464321E-1,3.5225567E-3,-1.1498388E-1,-2.8802133E-1,6.0210535E-3,8.524108E-2,-6.9931984E-2,-6.965486E-2,6.789155E-2,-1.6562076E-1,-2.370217E-2,2.1726191E-1,-8.663913E-2,1.16041325E-1,1.0753073E-2,2.1997621E-2,9.116828E-3,1.568273E-1,2.9867036E-2,-3.035723E-2,-1.8828817E-1,-5.136941E-2,1.21288165E-1,1.3192953E-1,3.0737953E-2,-3.0320534E-2,1.0610115E-2,-4.6806918E-3,4.8982824E-4,-1.9385861E-2,1.1788341E-3,-1.3438413E-2,-6.5442445E-3,-5.1362077E-3,1.5748915E-3,7.2384044E-3,-2.5712966E-3,3.8240654E-3,-2.6755857E-3,-3.3317952E-4,-8.320269E-3,-1.6456159E-2,-6.570487E-3,-1.1288314E-3,-2.2737572E-2,-6.4813127E-3,3.8014012E-3,-1.0792331E-2,-2.550001E-2,7.62949E-3,-1.0616846E-3,-1.6603108E-3,-9.430072E-3,-9.605686E-4,-7.572817E-3,8.026952E-3,1.0983687E-3,2.2114484E-4,-8.563059E-3,-1.9437693E-3,7.868962E-3,1.6927205E-2,3.5588413E-3,4.759963E-3,-5.4982947E-3,6.0119973E-3,-9.307976E-3,1.093448E-2,-5.300097E-4,1.6704813E-2,6.6113416E-3,-1.6920137E-3,8.043709E-3,-5.221398E-3,-2.3725323E-2,-2.7548042E-3,1.081828E-2,1.1650656E-2,-5.870055E-3,3.512497E-3,9.020546E-3,-2.810305E-4,-1.6191516E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,-1,-1,49,51,53,55,57,59,61,63,65,-1,67,69,-1,-1,-1,-1,71,-1,-1,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,107,109,-1,111,113,-1,115,117,119,121,123,125,127,129,131,133,-1,-1,135,137,-1,139,141,143,145,-1,147,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.108379E0,2.8836598E0,4.374378E0,2.672739E0,1.4982665E0,7.56848E-1,3.1833372E0,1.6722355E0,7.6191664E-1,5.782504E-1,8.716154E-1,5.34619E-1,5.547795E-1,2.9139977E0,2.005721E0,8.211837E-1,3.8194552E-1,4.976182E-1,4.365554E-1,3.516004E-1,2.1330097E-1,4.2409977E-1,2.2823656E-1,0E0,5.3208E-1,0E0,0E0,1.87321E0,1.6083956E0,1.1114726E0,7.813425E-1,9.2311394E-1,6.474304E-1,3.4657785E-1,5.921378E-1,3.8577843E-1,0E0,1.656335E-1,5.4988337E-1,0E0,0E0,0E0,0E0,3.0865377E-1,0E0,0E0,0E0,5.643464E-1,4.1340506E-1,1.7666345E0,1.250593E0,1.1338353E-1,1.1922731E0,5.5438364E-1,4.4826642E-1,9.562019E-1,2.3113343E-1,4.0732902E-1,3.292855E-1,5.732508E-1,1.5776089E-1,2.3180953E-1,0E0,3.9503655E-1,2.2764912E-1,1.1854774E-1,3.604412E-1,0E0,1.0963377E-1,3.855195E-1,0E0,1.228696E-1,4.0959027E-1,3.0398896E-1,2.2927277E-1,1.2651408E-1,1.1460947E-1,4.0205097E-1,7.870209E-1,8.7909484E-1,8.855434E-1,0E0,0E0,1.3540936E0,8.432495E-1,0E0,3.35465E-1,4.6853864E-1,5.0579023E-1,2.1098292E-1,0E0,3.0657342E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,43,43,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,68,68,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84,86,86,87,87,88,88,89,89,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,-1,-1,50,52,54,56,58,60,62,64,66,-1,68,70,-1,-1,-1,-1,72,-1,-1,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,108,110,-1,112,114,-1,116,118,120,122,124,126,128,130,132,134,-1,-1,136,138,-1,140,142,144,146,-1,148,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,4.97E2,3.9712732E7,2.2668628E7,6.1313953E1,9.070543E0,2.074E3,9.966725E2,4.3800272E8,4.9386E4,8.333333E0,5.5E1,2.979E3,7.2456576E8,4.61E2,2.1592189E5,2.262586E6,6.971004E7,1E0,7.492813E0,2.6572757E10,3.5E0,-1.0346897E-2,8.836364E0,-6.0777594E-3,-2.817622E-2,1E0,6.8E1,6.2E1,1.6256282E7,9.367855E-1,1.568E3,1.8489688E7,1.9051096E7,2.8035452E11,-1.8249024E-3,1.116567E3,1.2679E4,4.767929E-3,1.9806543E-2,1.228576E-3,1.0257844E-2,3.046556E5,-1.8291049E-2,2.1164343E-3,1.190366E-2,4.0701206E1,1.8273557E2,1E0,2E0,7.8955513E-1,2.8839298E7,7.854E3,2.109E3,1.9745072E7,5.2076015E0,1.172E3,1.9E1,1E0,5.7118463E2,1.1472081E0,-8.82817E-3,5.9572783E0,2.65851E5,1.2939E4,3.830986E-1,3.5225567E-3,2.1944155E3,2.0605305E5,6.0210535E-3,3.28825E0,1.0828989E8,6.737088E7,3.4E2,1.1902924E0,9.544947E6,8E0,6.1E2,2.6204643E0,3.0769231E0,2.1997621E-2,9.116828E-3,1.0918E4,4.247868E3,-3.035723E-2,3.3720784E5,1.6173E4,2.342147E7,1.5435694E9,3.0737953E-2,9.3058E4,1.0610115E-2,-4.6806918E-3,4.8982824E-4,-1.9385861E-2,1.1788341E-3,-1.3438413E-2,-6.5442445E-3,-5.1362077E-3,1.5748915E-3,7.2384044E-3,-2.5712966E-3,3.8240654E-3,-2.6755857E-3,-3.3317952E-4,-8.320269E-3,-1.6456159E-2,-6.570487E-3,-1.1288314E-3,-2.2737572E-2,-6.4813127E-3,3.8014012E-3,-1.0792331E-2,-2.550001E-2,7.62949E-3,-1.0616846E-3,-1.6603108E-3,-9.430072E-3,-9.605686E-4,-7.572817E-3,8.026952E-3,1.0983687E-3,2.2114484E-4,-8.563059E-3,-1.9437693E-3,7.868962E-3,1.6927205E-2,3.5588413E-3,4.759963E-3,-5.4982947E-3,6.0119973E-3,-9.307976E-3,1.093448E-2,-5.300097E-4,1.6704813E-2,6.6113416E-3,-1.6920137E-3,8.043709E-3,-5.221398E-3,-2.3725323E-2,-2.7548042E-3,1.081828E-2,1.1650656E-2,-5.870055E-3,3.512497E-3,9.020546E-3,-2.810305E-4,-1.6191516E-2],"split_indices":[20,52,2,45,45,46,53,2,4,7,2,52,0,2,7,0,28,1,45,8,54,12,53,0,58,0,0,102,0,0,45,27,0,32,45,31,0,4,2,0,0,0,0,32,0,0,0,52,55,79,17,38,45,9,0,1,35,2,3,16,55,53,0,53,29,9,56,0,4,33,0,54,45,7,12,53,9,3,2,38,58,0,0,9,52,0,28,2,45,7,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.653E3,8.58E2,1.795E3,6.82E2,1.76E2,2.41E2,1.554E3,5.68E2,1.14E2,4.5E1,1.31E2,2.3E2,1.1E1,1.142E3,4.12E2,3.24E2,2.44E2,5E1,6.4E1,1.8E1,2.7E1,1.09E2,2.2E1,2.2E1,2.08E2,7E0,4E0,6.42E2,5E2,2.99E2,1.13E2,1.61E2,1.63E2,1E2,1.44E2,4.5E1,5E0,2.9E1,3.5E1,6E0,1.2E1,1.9E1,8E0,1.06E2,3E0,1.1E1,1.1E1,1.2E2,8.8E1,1.68E2,4.74E2,2E1,4.8E2,1.8E1,2.81E2,7E1,4.3E1,1.49E2,1.2E1,1.2E2,4.3E1,8E1,2E1,1.11E2,3.3E1,1.5E1,3E1,5E0,2.4E1,3.2E1,3E0,1.4E1,9.2E1,6.9E1,5.1E1,5.5E1,3.3E1,2.1E1,1.47E2,2.89E2,1.85E2,1.6E1,4E0,3.88E2,9.2E1,3E0,1.5E1,2.66E2,1.5E1,6.7E1,3E0,4E1,3E0,9.4E1,5.5E1,1E1,2E0,6.8E1,5.2E1,3.3E1,1E1,5E0,7.5E1,8.3E1,2.8E1,1.4E1,1.9E1,5E0,1E1,2E0,2.8E1,2.2E1,2E0,2.7E1,5E0,8E0,6E0,7.3E1,1.9E1,4.5E1,2.4E1,1.5E1,3.6E1,4E0,5.1E1,3.1E1,2E0,1E1,1.1E1,1.9E1,1.28E2,2.81E2,8E0,1.6E1,1.69E2,3.3E1,3.55E2,6.3E1,2.9E1,1.3E1,2E0,2.61E2,5E0,1E1,5E0,3.4E1,3.3E1,3.8E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"149","size_leaf_vector":"1"}},{"base_weights":[-2.8268914E-3,2.7982784E-2,-9.084398E-2,-1.3151068E-2,9.448543E-2,-1.8935782E-1,-5.0116386E-2,-4.360954E-2,5.6930955E-2,2.6189813E-1,6.884182E-2,-1.6766462E-1,-2.4675412E-2,-7.033677E-2,6.7127705E-2,-6.375882E-2,2.8510094E-2,2.038122E-2,1.03569865E-1,2.0987731E-1,2.3194913E-2,8.995504E-2,-1.1318532E-1,-1.3609564E-1,-3.2417434E-1,-3.6310185E-2,-1.4809369E-1,8.74176E-3,2.8137134E-2,-8.4559016E-2,8.769969E-3,1.02610774E-1,-2.4097817E-2,3.9317813E-2,-9.265528E-2,2.2212712E-1,6.513651E-2,2.3423636E-1,-2.1209913E-1,7.131105E-2,2.4271002E-1,-2.0057233E-1,-5.0745763E-2,-1.2016375E-1,-2.5014734E-2,-4.069689E-3,-4.0367797E-1,-1.5949805E-1,-1.1827354E-2,-1.282418E-1,-4.2487878E-1,7.299388E-3,5.078395E-4,-3.414583E-2,-1.1386132E-1,7.312087E-2,-5.483985E-2,-6.995687E-3,1.2568402E-1,-1.6832036E-1,2.5245606E-3,2.795005E-2,9.302387E-3,-1.4007173E-1,8.258309E-3,1.1411229E-1,3.4180486E-1,1.26976725E-2,4.7021013E-2,2.5835773E-1,-2.2667367E-3,-1.7024565E-2,9.532978E-5,3.0777E-2,1.4664218E-1,4.332064E-1,1.5060462E-1,3.2996075E-4,-2.257434E-1,-9.739994E-2,4.962522E-3,-2.75008E-1,-9.087806E-2,-5.376019E-1,-1.3561833E-1,-1.3302602E-1,-2.4531279E-2,-3.2334127E-2,4.988685E-2,-1.9282885E-2,-1.7159852E-1,-7.7844104E-3,-2.6328748E-2,-2.3619859E-2,7.916105E-3,-6.217722E-3,1.0896539E-3,-6.8739224E-3,-1.9834345E-6,5.359261E-3,-4.1615088E-3,-1.1298731E-2,3.1057285E-4,7.84211E-3,2.1557984E-4,1.6626782E-3,-1.2251022E-2,-1.2988701E-2,8.5603877E-4,3.209582E-3,-1.2365183E-3,-1.7771974E-2,-4.492304E-3,8.211742E-3,-3.6854608E-4,1.9175857E-2,7.559966E-3,3.487453E-3,-5.0034225E-3,1.3991252E-2,4.6688775E-3,1.9005117E-3,-1.2793418E-2,8.413275E-3,-1.6438293E-3,1.575776E-2,3.2561112E-2,4.456548E-3,2.392437E-2,1.609013E-4,-1.231445E-2,-5.8173263E-3,8.418222E-3,-1.6611684E-2,-5.588864E-3,-3.1400982E-3,-1.4257609E-2,-7.6651005E-3,-2.7938103E-2,3.8004331E-3,-9.979258E-3,-9.3590876E-4,-7.854351E-3,8.895771E-3,-1.9528002E-3,3.7538277E-3,-7.1213897E-3,-2.6633823E-3,7.930017E-3,-1.6724288E-2,-6.990558E-3,-3.6844598E-5,-1.1244445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,-1,83,85,87,89,91,-1,93,95,97,99,101,-1,103,105,107,109,-1,111,-1,113,115,-1,117,119,-1,-1,-1,121,123,125,127,-1,129,131,-1,133,135,137,139,141,-1,143,145,147,149,-1,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.164347E0,5.3546352E0,2.7388892E0,2.5853384E0,3.193183E0,1.3275242E0,1.1568915E0,1.2282836E0,6.23492E-1,1.0438967E0,2.5082471E0,8.8970184E-1,0E0,1.0926306E0,3.1885278E-1,9.9642634E-1,7.236548E-1,4.4673195E-1,7.1913636E-1,8.925543E-1,0E0,1.643054E0,3.608575E-1,9.156358E-1,5.552199E-1,8.7275E-1,6.3784575E-1,0E0,1.8878146E-1,7.53927E-1,6.0982555E-1,4.6018767E-1,4.189356E-1,3.080555E-1,4.0125248E-1,4.6514773E-1,4.331807E-1,5.419893E-1,1.5419926E-1,1.5863478E0,1.0401392E0,1.4777255E-1,3.0537108E-1,6.7335486E-1,0E0,0E0,7.527821E-1,3.8006902E-1,3.091439E-1,5.58859E-1,1.7274046E-1,0E0,1.8679816E-1,1.0373955E0,1.094255E0,4.6899492E-1,8.3442223E-1,0E0,3.26787E-1,3.10876E-1,3.9563566E-1,3.5255295E-1,0E0,2.4310529E-1,0E0,1.5502256E-1,1.3083482E-1,0E0,4.5319745E-1,3.629222E-1,0E0,0E0,0E0,9.1285324E-1,9.6232677E-1,2.9025817E-1,8.3711E-1,0E0,1.8067586E-1,2.3177886E-1,0E0,2.3254228E-1,6.662569E-1,1.6584682E-1,1.5730661E-1,1.5851927E-1,0E0,3.4450084E-1,3.5686076E-1,2.4428628E-1,3.3596492E-1,0E0,0E0,1.9766293E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,63,63,65,65,66,66,68,68,69,69,73,73,74,74,75,75,76,76,78,78,79,79,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,90,90,93,93],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,-1,84,86,88,90,92,-1,94,96,98,100,102,-1,104,106,108,110,-1,112,-1,114,116,-1,118,120,-1,-1,-1,122,124,126,128,-1,130,132,-1,134,136,138,140,142,-1,144,146,148,150,-1,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,5.862126E2,2.0666666E0,7.5939475E5,1.0594161E1,1.2427474E8,3.970405E3,4.264897E6,1.146E3,1.415178E3,1E0,2.0354E4,-2.4675412E-2,4.987639E7,4.0601485E6,6.067634E5,2.46E2,4.05E0,1.0437505E-2,1.9162654E7,2.3194913E-2,5.185489E3,5E0,4.5723195E6,5.2187E4,9.7043E4,8.91942E5,8.74176E-3,2.8091298E7,9.52E2,2.72258E5,2.35E2,2.5330253E-2,7.780377E6,2.1178565E6,7.130317E7,7.692308E-1,3.5E1,1.1E1,1E0,3.2475834E7,8.503172E2,1.4659271E7,7.22E2,-2.5014734E-2,-4.069689E-3,8.92686E5,5.551823E-1,8.3956606E5,4.1808826E-1,7.526755E7,7.299388E-3,5.545735E8,4.7E2,3.3012173E9,1.0715278E7,7.118233E7,-6.995687E-3,2.7309215E6,9.173799E4,2.511352E6,2.69E2,9.302387E-3,1.6437647E-4,8.258309E-3,6.445E3,1.3E1,1.26976725E-2,4.5158855E6,4.675E0,-2.2667367E-3,-1.7024565E-2,9.532978E-5,8.6206274E2,1.4521701E2,4.2E1,4.2482185E0,3.2996075E-4,1.7321888E1,1.9178673E3,4.962522E-3,1E0,5.7E1,2.2837209E2,6.77603E5,2.7627052E7,-2.4531279E-2,2.1515152E0,2.632E5,1.6992E4,8.1057306E2,-7.7844104E-3,-2.6328748E-2,7.75541E-1,7.916105E-3,-6.217722E-3,1.0896539E-3,-6.8739224E-3,-1.9834345E-6,5.359261E-3,-4.1615088E-3,-1.1298731E-2,3.1057285E-4,7.84211E-3,2.1557984E-4,1.6626782E-3,-1.2251022E-2,-1.2988701E-2,8.5603877E-4,3.209582E-3,-1.2365183E-3,-1.7771974E-2,-4.492304E-3,8.211742E-3,-3.6854608E-4,1.9175857E-2,7.559966E-3,3.487453E-3,-5.0034225E-3,1.3991252E-2,4.6688775E-3,1.9005117E-3,-1.2793418E-2,8.413275E-3,-1.6438293E-3,1.575776E-2,3.2561112E-2,4.456548E-3,2.392437E-2,1.609013E-4,-1.231445E-2,-5.8173263E-3,8.418222E-3,-1.6611684E-2,-5.588864E-3,-3.1400982E-3,-1.4257609E-2,-7.6651005E-3,-2.7938103E-2,3.8004331E-3,-9.979258E-3,-9.3590876E-4,-7.854351E-3,8.895771E-3,-1.9528002E-3,3.7538277E-3,-7.1213897E-3,-2.6633823E-3,7.930017E-3,-1.6724288E-2,-6.990558E-3,-3.6844598E-5,-1.1244445E-2],"split_indices":[27,52,53,28,56,45,52,9,2,52,14,9,0,45,47,47,10,54,41,47,0,52,6,28,29,1,9,0,45,29,9,2,38,28,28,7,53,3,8,63,50,4,50,2,0,0,1,38,28,27,45,0,44,1,5,45,7,0,45,33,5,10,0,38,0,29,3,0,47,54,0,0,0,58,58,8,39,0,56,46,0,16,8,4,29,5,0,53,33,2,4,0,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E3,1.956E3,6.84E2,1.209E3,7.47E2,1.99E2,4.85E2,8.43E2,3.66E2,9.8E1,6.49E2,1.88E2,1.1E1,4.14E2,7.1E1,6.59E2,1.84E2,2.06E2,1.6E2,8.1E1,1.7E1,5.82E2,6.7E1,1.58E2,3E1,2.89E2,1.25E2,1.7E1,5.4E1,5.12E2,1.47E2,7.6E1,1.08E2,1.77E2,2.9E1,3.8E1,1.22E2,7.7E1,4E0,5.2E2,6.2E1,2.7E1,4E1,1.53E2,5E0,8E0,2.2E1,4.7E1,2.42E2,1.18E2,7E0,9E0,4.5E1,1.89E2,3.23E2,7.3E1,7.4E1,6E0,7E1,1.6E1,9.2E1,1.66E2,1.1E1,2.5E1,4E0,2.1E1,1.7E1,9E0,1.13E2,7.1E1,6E0,2E0,2E0,3.39E2,1.81E2,1.9E1,4.3E1,3E0,2.4E1,3.1E1,9E0,2.3E1,1.3E2,1.4E1,8E0,4.5E1,2E0,1.82E2,6E1,3.4E1,8.4E1,3E0,4E0,4E1,5E0,7E1,1.19E2,2.56E2,6.7E1,5.9E1,1.4E1,1.8E1,5.6E1,5.3E1,1.7E1,5E0,1.1E1,4E0,8.8E1,9.6E1,7E1,3E0,2.2E1,1.4E1,7E0,1.2E1,5E0,9.7E1,1.6E1,5.8E1,1.3E1,3.3E2,9E0,1.56E2,2.5E1,1.5E1,4E0,3.8E1,5E0,3E0,2.1E1,2.9E1,2E0,1.5E1,8E0,1.17E2,1.3E1,2E0,1.2E1,2E0,6E0,1E1,3.5E1,6E0,1.76E2,5.3E1,7E0,2.9E1,5E0,9E0,7.5E1,3.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[7.590838E-4,-6.0613424E-2,3.8963288E-2,-7.306628E-2,2.0708703E-1,1.9886235E-2,1.790503E-1,-2.8392944E-2,-1.3904828E-1,2.465124E-2,8.336369E-2,1.01669416E-1,-1.6797764E-2,1.6225196E-1,1.7871466E-2,-7.882628E-2,5.4508377E-2,-1.22743964E-1,-3.0951062E-1,8.1719095E-3,-1.2204375E-3,6.022596E-2,1.766683E-1,-4.201876E-2,4.185695E-2,1.322178E-2,1.4261757E-1,-5.6855954E-2,-2.8497833E-1,2.084573E-1,-3.4388222E-3,-1.7096014E-1,-8.697128E-2,-2.4118785E-2,-1.9477753E-1,5.0335433E-2,3.656352E-1,2.1178412E-1,2.9586773E-2,-6.371543E-2,-2.8496743E-3,1.7469425E-1,1.6786452E-2,1.5100859E-1,-1.4602757E-1,-7.013273E-2,9.2563E-3,2.4425008E-3,-4.4109428E-1,1.8138494E-2,1.4469373E-1,-2.9542506E-2,1.4183521E-1,-3.6735502E-1,-1.5024747E-1,-1.7996979E-1,-5.9953704E-2,-1.2293349E-1,-2.419925E-2,7.4928336E-2,-1.211637E-1,2.0761304E-2,4.088355E-3,1.1542316E-1,2.6936242E-1,-2.9160032E-2,9.244815E-2,-7.582123E-2,1.5322201E-2,4.349946E-2,-3.31246E-2,2.1665667E-1,-6.2199935E-2,3.481671E-2,-1.3742635E-1,1.0386245E-1,2.2596057E-1,5.6169466E-3,-1.7272511E-2,7.999206E-3,-3.7560086E-3,-3.3741046E-2,-1.13047585E-2,5.678335E-3,2.2969475E-2,-2.1852208E-3,1.1473884E-2,3.9306437E-3,1.2382816E-2,-3.5809148E-3,-2.1088721E-2,-4.2263824E-3,-1.2045431E-2,-5.3500957E-3,-1.2411112E-2,-2.9652755E-4,-5.1761474E-3,3.0697947E-3,-9.945398E-3,-1.0154068E-3,5.466813E-3,-1.9512279E-3,-1.9038476E-2,1.1170663E-2,2.2821792E-3,8.794285E-3,1.4941511E-2,-7.0741E-3,7.0106536E-3,5.0430354E-3,-3.8941004E-3,7.4308133E-3,-1.2347978E-3,-3.4187704E-3,3.7710615E-3,-1.3014524E-2,-1.112332E-3,8.65011E-3,2.1070782E-2,3.7437095E-3,-7.7610286E-3,3.7379013E-3,-1.4790042E-3,-9.413969E-3,-8.2469475E-4,2.025146E-3,7.1890815E-3,9.3619395E-3,2.4710873E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,79,-1,-1,81,-1,83,85,87,89,91,93,95,97,-1,99,101,-1,-1,103,105,-1,107,109,111,113,115,117,119,121,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.2884703E0,3.4454403E0,4.413995E0,2.8965263E0,1.7080342E0,4.369903E0,5.851183E-1,2.462719E0,1.0653801E0,0E0,3.2733208E-1,1.3911643E0,1.4899651E0,3.7186766E-1,0E0,1.6414948E0,1.9900796E0,6.1296654E-1,6.591239E-1,0E0,0E0,8.6362755E-1,8.201165E-1,5.981046E-1,1.0061476E0,0E0,4.0715384E-1,1.1161286E0,1.8583112E0,6.075604E-1,6.223167E-1,5.712571E-1,5.193677E-1,0E0,4.5099825E-1,1.2068946E0,1.3421333E-1,6.8463945E-1,1.340626E0,4.3459845E-1,3.5505146E-1,4.906274E-1,7.165919E-1,5.242543E-1,3.248895E-1,6.348561E-1,0E0,0E0,1.1301212E0,0E0,3.456295E-1,6.127273E-1,1.531713E-1,2.660824E-1,8.6307955E-1,2.25075E-1,4.2627114E-1,3.302811E-1,0E0,9.413085E-1,7.839515E-1,0E0,0E0,3.881492E-1,2.3508358E-1,0E0,4.102611E-1,3.9944363E-1,3.5464492E-1,4.1052648E-1,3.5507125E-1,2.504648E-1,1.2480637E-1,6.557202E-1,1.8857777E-1,2.6990962E-1,4.205947E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,63,63,64,64,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,80,-1,-1,82,-1,84,86,88,90,92,94,96,98,-1,100,102,-1,-1,104,106,-1,108,110,112,114,116,118,120,122,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,6.082581E3,1.3137E4,1.6627582E-2,1.4232318E3,2E0,1.8114515E1,2.857E3,1.2427474E8,2.465124E-2,2.851369E2,5.9643E4,7.298614E2,2.386324E6,1.7871466E-2,9.837297E0,9.073025E6,1.9121015E0,2.0521326E0,8.1719095E-3,-1.2204375E-3,6.567044E2,2.3523688E0,3.5150378E5,5.1212654E0,1.322178E-2,5.68E2,4.564527E6,1.1880085E2,9.9341E4,1E0,5.85E2,1.5804776E4,-2.4118785E-2,7.6008E4,1.604E3,3.4971508E7,5.8084745E6,3.171E3,1.2817779E7,2.0282449E6,4.029152E8,6.747114E7,5.57E4,3.2917362E7,6E0,9.2563E-3,2.4425008E-3,3.475E2,1.8138494E-2,4.215533E3,4.5505118E2,1.82407E5,1E0,2.1358025E0,1.5476E4,1.6772E4,2.345785E5,-2.419925E-2,5.52E2,8.450237E0,2.0761304E-2,4.088355E-3,4.1681856E9,4.930435E2,-2.9160032E-2,1E0,3.8E2,2.4946861E4,9E2,1.074E3,1.5137865E2,7.0491E4,2.429717E1,6.1428223E1,2.7271206E5,1.0938637E1,5.6169466E-3,-1.7272511E-2,7.999206E-3,-3.7560086E-3,-3.3741046E-2,-1.13047585E-2,5.678335E-3,2.2969475E-2,-2.1852208E-3,1.1473884E-2,3.9306437E-3,1.2382816E-2,-3.5809148E-3,-2.1088721E-2,-4.2263824E-3,-1.2045431E-2,-5.3500957E-3,-1.2411112E-2,-2.9652755E-4,-5.1761474E-3,3.0697947E-3,-9.945398E-3,-1.0154068E-3,5.466813E-3,-1.9512279E-3,-1.9038476E-2,1.1170663E-2,2.2821792E-3,8.794285E-3,1.4941511E-2,-7.0741E-3,7.0106536E-3,5.0430354E-3,-3.8941004E-3,7.4308133E-3,-1.2347978E-3,-3.4187704E-3,3.7710615E-3,-1.3014524E-2,-1.112332E-3,8.65011E-3,2.1070782E-2,3.7437095E-3,-7.7610286E-3,3.7379013E-3,-1.4790042E-3,-9.413969E-3,-8.2469475E-4,2.025146E-3,7.1890815E-3,9.3619395E-3,2.4710873E-2],"split_indices":[102,52,2,27,48,17,35,2,45,0,58,1,52,1,0,54,45,53,53,0,0,52,41,47,53,0,3,9,52,1,96,2,51,0,10,2,5,45,2,9,50,7,45,2,1,3,0,0,55,0,4,56,9,104,56,9,9,28,0,2,56,0,0,31,4,0,16,12,50,2,2,56,10,53,56,28,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E3,1.028E3,1.652E3,9.83E2,4.5E1,1.455E3,1.97E2,5.87E2,3.96E2,1.2E1,3.3E1,4.5E2,1.005E3,1.83E2,1.4E1,3.65E2,2.22E2,3.63E2,3.3E1,1.8E1,1.5E1,2.91E2,1.59E2,7.03E2,3.02E2,2.5E1,1.58E2,3.31E2,3.4E1,6E1,1.62E2,1.53E2,2.1E2,1.1E1,2.2E1,2.83E2,8E0,1.28E2,3.1E1,4.52E2,2.51E2,4.7E1,2.55E2,1.54E2,4E0,3.15E2,1.6E1,1.1E1,2.3E1,1.5E1,4.5E1,1.38E2,2.4E1,1.3E1,1.4E2,4.6E1,1.64E2,1.9E1,3E0,2.48E2,3.5E1,6E0,2E0,4.9E1,7.9E1,2E0,2.9E1,3.92E2,6E1,9.9E1,1.52E2,4E1,7E0,2.29E2,2.6E1,9.6E1,5.8E1,2E0,2E0,1E1,3.05E2,9E0,1.4E1,4.3E1,2E0,1.31E2,7E0,1.7E1,7E0,3E0,1E1,8.8E1,5.2E1,2.6E1,2E1,7.8E1,8.6E1,6E0,1.3E1,7.2E1,1.76E2,2.8E1,7E0,1.7E1,3.2E1,2.8E1,5.1E1,5E0,2.4E1,1.1E1,3.81E2,1.3E1,4.7E1,2.3E1,7.6E1,5E0,1.47E2,3.6E1,4E0,3E0,4E0,1.38E2,9.1E1,1.7E1,9E0,4.2E1,5.4E1,5.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[-1.6417538E-5,-6.5911196E-2,3.1341176E-2,-8.880324E-2,1.3958083E-2,3.7770246E-3,1.08839154E-1,-5.681546E-2,-2.0157078E-1,6.581459E-2,-1.1001461E-1,-5.2184317E-2,3.6549885E-2,1.320488E-1,-1.8089226E-2,-1.1279587E-1,-1.0063984E-2,-1.6442397E-1,-3.9469168E-1,1.67138E-1,1.2169989E-2,-5.9666056E-2,-2.4701282E-1,1.9597353E-1,-5.9862524E-2,5.3300314E-2,-5.1319383E-2,1.1955666E-1,3.905179E-1,-2.5985873E-1,4.778198E-2,-5.030633E-2,-1.6384259E-1,-2.2273358E-2,-5.2048373E-3,-1.958364E-1,-6.701173E-2,-4.7418323E-1,-9.0435125E-2,1.3376762E-1,2.0147337E-2,-2.2717725E-2,1.7494553E-1,-1.3134602E-2,-1.0148311E-2,-1.8172564E-2,7.136646E-4,1.8471206E-2,2.9951928E-3,-6.969798E-2,7.062689E-2,1.4546054E-2,1.20950565E-1,-2.0676288E-1,-1.9226287E-2,2.087537E-1,9.531733E-2,4.6776813E-1,4.6029324E-3,-5.056277E-3,-1.534352E-2,3.129298E-4,2.0033874E-1,-8.773058E-2,3.3726748E-2,-1.2996623E-1,-2.6352745E-1,3.334921E-2,-3.865174E-2,-1.429598E-1,-3.0387446E-1,-4.3355394E-3,9.1236355E-3,-2.6795128E-2,-1.0070557E-2,-1.0822805E-2,1.6206236E-3,1.9425955E-1,7.778403E-2,-5.343405E-2,6.1245207E-3,1.227667E-2,1.6121519E-3,-1.1575131E-2,1.6573792E-3,-7.544189E-2,1.6378355E-1,9.3550505E-3,1.191143E-4,-1.4184625E-3,1.2689316E-1,1.3747165E-1,-3.7863847E-2,-4.896904E-2,-3.2437593E-1,2.4504896E-2,-1.4674091E-1,2.2530448E-1,-7.958212E-3,6.646334E-2,1.639498E-1,5.8351257E-3,2.4676496E-2,1.2976955E-1,-4.919439E-2,1.4598988E-2,2.5041006E-3,5.658634E-3,-5.2996897E-3,1.0443961E-2,8.989079E-5,-3.7187503E-3,-1.1862621E-2,-1.8293979E-2,-8.301669E-3,-2.5175244E-3,2.8319801E-3,-5.7597426E-3,-8.0205145E-4,-4.709894E-3,-1.3095168E-2,-4.0281406E-3,-1.6016947E-2,5.0866343E-3,1.301109E-2,-2.4264379E-4,7.713102E-3,-3.8572683E-3,1.180914E-3,-7.528414E-3,-2.8922237E-3,1.3757758E-2,6.223268E-4,-2.9050536E-3,1.1658467E-3,4.0732324E-3,1.2391443E-2,8.897558E-3,4.2133536E-3,-1.916104E-2,3.4816621E-3,5.138261E-3,-5.7937168E-3,-2.604672E-2,-7.968375E-3,4.686131E-4,1.2358862E-2,-8.26409E-3,2.7671985E-3,-4.8665362E-3,1.1265999E-2,8.466979E-3,2.5975942E-3,9.000484E-3,-6.891981E-3,1.1180475E-2,3.6117237E-4,-3.6288358E-3,4.1890126E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,-1,79,81,83,-1,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,-1,-1,-1,103,105,107,109,111,113,115,117,119,121,-1,-1,-1,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,129,131,-1,-1,133,135,137,139,141,143,145,147,149,-1,151,153,-1,-1,155,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4984283E0,1.5699813E0,3.8510962E0,2.3945394E0,1.2415684E0,2.442979E0,1.3958297E0,1.3624086E0,9.896555E-1,7.371875E-1,3.7842852E-1,9.491918E-1,1.2390875E0,1.2448449E0,1.1909146E0,7.477603E-1,6.2772375E-1,3.6937094E-1,5.13391E-1,3.4140623E-1,5.150723E-1,3.0182716E-1,5.260045E-1,3.556013E-1,6.183988E-1,1.849186E0,6.694943E-1,8.08177E-1,3.644116E-1,1.1558604E-1,4.2719913E-1,3.426053E-1,4.047556E-1,0E0,3.6622128E-1,4.9355054E-1,2.1105562E-1,2.876544E-1,1.0872464E-1,1.3051158E-1,0E0,3.537825E-1,1.7538336E-1,3.7938005E-1,0E0,0E0,0E0,0E0,0E0,6.09144E-1,2.884485E-1,8.0728537E-1,6.79518E-1,4.109637E-1,6.3372517E-1,5.400748E-1,5.891404E-1,1.6726112E-1,0E0,0E0,0E0,3.0120024E-1,2.0066291E-1,3.6718273E-1,1.9922504E-1,5.905472E-1,2.7565575E-1,2.9554182E-1,2.6972443E-1,3.514769E-1,1.7344451E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0748124E-1,1.6639714E-1,1.3535985E-1,0E0,0E0,0E0,0E0,0E0,5.2574134E-1,2.010994E-1,0E0,0E0,6.043315E-1,2.8692883E-1,5.408473E-1,1.0303489E0,1.367545E-1,3.6033487E-1,2.8970113E-1,1.6312015E-1,2.7738214E-1,0E0,2.803676E-1,6.843326E-1,0E0,0E0,1.6299799E-1,1.3013086E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,77,77,78,78,79,79,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,-1,80,82,84,-1,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,-1,-1,-1,104,106,108,110,112,114,116,118,120,122,-1,-1,-1,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,130,132,-1,-1,134,136,138,140,142,144,146,148,150,-1,152,154,-1,-1,156,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,6.624E3,2.589369E7,9.302862E7,1.8108038E5,4.0844156E7,1.638058E2,2.1827742E6,1.3977353E7,6.0435E4,7E0,7.382963E0,4.5505118E2,4.7013435E1,4.15E2,7E0,4.6E0,2.194E3,1E0,2.6108465E0,9.190713E-1,3.285E3,1.3050649E4,6.077036E8,4.5154498E2,4.745907E7,2E0,3.1029554E7,1.02E3,1.1803382E4,1.11E3,6.298144E0,-2.2273358E-2,2.1111E4,2.8636363E0,1.4622E4,1E0,2.0802219E8,2.5927516E11,2.0147337E-2,5.884028E0,3.0720797E-1,1.9469026E0,-1.0148311E-2,-1.8172564E-2,7.136646E-4,1.8471206E-2,2.9951928E-3,1.5694646E3,2.12062E5,4.5723195E6,1.6015803E7,8E0,6.856226E-1,2.2644286E2,1.4689211E7,3.4E1,4.6029324E-3,-5.056277E-3,-1.534352E-2,1.774775E11,7.7854166E0,3E0,1.71525E4,8.617448E5,9.192915E0,1E0,1.4571098E5,9.1569895E-1,1.5825138E5,-4.3355394E-3,9.1236355E-3,-2.6795128E-2,-1.0070557E-2,-1.0822805E-2,1.6206236E-3,5.59E2,1.3593E4,1.4777102E9,6.1245207E-3,1.227667E-2,1.6121519E-3,-1.1575131E-2,1.6573792E-3,3.983889E3,2.5057983E3,9.3550505E-3,1.191143E-4,2.0697437E2,2.0827537E0,3.68E0,3.171E3,9.61E2,2E0,3.9516E4,1.1364663E0,5.0040483E0,-7.958212E-3,1.7593515E1,1.314806E8,5.8351257E-3,2.4676496E-2,5.4106476E7,2.1325744E7,1.4598988E-2,2.5041006E-3,5.658634E-3,-5.2996897E-3,1.0443961E-2,8.989079E-5,-3.7187503E-3,-1.1862621E-2,-1.8293979E-2,-8.301669E-3,-2.5175244E-3,2.8319801E-3,-5.7597426E-3,-8.0205145E-4,-4.709894E-3,-1.3095168E-2,-4.0281406E-3,-1.6016947E-2,5.0866343E-3,1.301109E-2,-2.4264379E-4,7.713102E-3,-3.8572683E-3,1.180914E-3,-7.528414E-3,-2.8922237E-3,1.3757758E-2,6.223268E-4,-2.9050536E-3,1.1658467E-3,4.0732324E-3,1.2391443E-2,8.897558E-3,4.2133536E-3,-1.916104E-2,3.4816621E-3,5.138261E-3,-5.7937168E-3,-2.604672E-2,-7.968375E-3,4.686131E-4,1.2358862E-2,-8.26409E-3,2.7671985E-3,-4.8665362E-3,1.1265999E-2,8.466979E-3,2.5975942E-3,9.000484E-3,-6.891981E-3,1.1180475E-2,3.6117237E-4,-3.6288358E-3,4.1890126E-3],"split_indices":[20,52,2,45,45,28,45,52,28,50,10,3,54,56,56,0,10,54,0,66,35,27,0,28,7,55,7,17,47,11,4,2,57,0,9,54,2,16,7,31,0,53,38,53,0,0,0,0,0,52,29,28,51,8,27,58,1,8,0,0,0,31,54,11,48,32,57,64,28,27,28,0,0,0,0,0,0,0,10,7,0,0,0,0,0,28,55,0,0,4,53,54,2,12,17,10,39,56,0,58,1,0,0,45,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.659E3,8.57E2,1.802E3,6.66E2,1.91E2,1.33E3,4.72E2,5.2E2,1.46E2,1.35E2,5.6E1,4.91E2,8.39E2,3.99E2,7.3E1,2.36E2,2.84E2,1.24E2,2.2E1,4.6E1,8.9E1,4.2E1,1.4E1,1.4E1,4.77E2,7.05E2,1.34E2,3.82E2,1.7E1,1.5E1,5.8E1,1.07E2,1.29E2,2E0,2.82E2,9.3E1,3.1E1,1.7E1,5E0,4.2E1,4E0,7.4E1,1.5E1,3.3E1,9E0,9E0,5E0,5E0,9E0,4.44E2,3.3E1,4.49E2,2.56E2,2.2E1,1.12E2,8E1,3.02E2,1.3E1,4E0,5E0,1E1,4.5E1,1.3E1,7.4E1,3.3E1,9.8E1,3.1E1,1.31E2,1.51E2,6.4E1,2.9E1,2.9E1,2E0,1.2E1,5E0,2E0,3E0,1.9E1,2.3E1,6.2E1,1.2E1,9E0,6E0,5E0,2.8E1,4.34E2,1E1,1.1E1,2.2E1,3.94E2,5.5E1,2.32E2,2.4E1,1E1,1.2E1,8.4E1,2.8E1,7.7E1,3E0,2.14E2,8.8E1,2E0,1.1E1,1.2E1,3.3E1,7E0,6E0,7E0,6.7E1,4E0,2.9E1,6.9E1,2.9E1,1.2E1,1.9E1,3E1,1.01E2,3.1E1,1.2E2,4.9E1,1.5E1,4E0,2.5E1,1E1,9E0,1.2E1,1.1E1,4.6E1,1.6E1,6.6E1,3.68E2,5E0,5E0,1.19E2,2.75E2,4.3E1,1.2E1,1.16E2,1.16E2,5E0,1.9E1,3E0,7E0,4E0,8E0,8E1,4E0,2.5E1,3E0,2E0,7.5E1,2E1,1.94E2,8.2E1,6E0,6E0,6E0,2.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[-3.4797138E-3,-5.853659E-2,3.704608E-2,-3.2438904E-2,-1.3875745E-1,6.6020064E-2,-7.9248466E-2,-4.614918E-2,6.2207628E-2,-1.1811673E-1,-3.8868463E-1,2.9693453E-2,1.2439971E-1,-1.2554118E-1,6.306372E-2,-3.413721E-2,-1.7568286E-1,1.23296216E-1,-4.777981E-2,-9.760061E-2,-2.1985887E-1,-2.1758405E-2,-9.217794E-3,7.8779586E-2,-1.833606E-2,1.8346475E-2,1.0745199E-1,-9.7005E-2,-2.6222995E-1,8.82491E-2,-9.645017E-3,1.043304E-2,-6.672952E-2,-4.5982483E-1,-1.0126884E-1,4.3500736E-2,1.7424442E-1,2.883165E-2,-1.3397309E-1,-1.8481953E-2,-8.906079E-2,-6.5414324E-2,-2.9216263E-1,9.937331E-2,-2.2866428E-2,1.3372892E-2,-7.3199585E-2,9.6906304E-2,3.5516483E-1,-1.1197613E-1,3.3086605E-2,-3.1026304E-1,-1.4996774E-3,2.5030768E-1,5.1045537E-2,6.0554955E-2,-3.3689618E-2,-7.292131E-2,1.4006448E-1,1.459638E-3,-5.517417E-1,6.693444E-2,-1.5905286E-1,3.9417247E-4,8.492466E-3,1.2693601E-2,9.825902E-2,7.2399944E-2,-1.2989551E-2,-1.5539353E-2,-3.734213E-3,-2.0503278E-1,-6.9038E-2,-1.6765192E-2,2.542055E-4,-3.1722662E-1,1.4750673E-3,1.0675285E-1,-1.650324E-1,9.358433E-3,-4.837642E-2,-2.6477046E-2,7.5680286E-2,-2.939583E-2,-1.2531185E-1,8.695632E-2,3.352731E-1,8.901675E-3,4.720088E-1,-1.5354916E-1,-6.9612056E-2,-1.3314452E-4,1.1934996E-2,-1.0802491E-2,-2.310761E-2,1.4719202E-2,2.1609413E-3,1.24310724E-1,7.8414683E-4,-3.5303994E-3,3.6544942E-3,-2.9890074E-3,7.437525E-3,-3.6907475E-3,9.8273195E-3,-1.4600224E-3,9.5925825E-3,-3.0218448E-2,-6.2297494E-3,-2.8287754E-3,5.9021246E-3,-9.48032E-3,4.210484E-3,6.5397252E-3,-2.1920954E-3,6.4176233E-3,-8.51782E-4,-1.2384263E-2,-8.9633866E-4,-3.69534E-3,7.0596198E-3,-1.787673E-2,-7.836698E-3,6.811432E-3,2.5570607E-3,-1.6855827E-2,-2.937705E-3,5.970519E-3,-3.5471874E-3,-3.3738248E-3,1.2470211E-3,8.857069E-3,2.7091058E-3,-6.500371E-3,-3.0620597E-4,-7.824225E-3,-1.50835E-3,4.8974957E-3,-1.1927933E-3,1.759485E-2,3.0120611E-3,5.648826E-3,2.6137631E-2,-1.5954716E-2,-6.2940014E-3,-4.148103E-3,5.4299887E-3,1.9792689E-3,1.2356785E-2,-4.131064E-3,1.7997547E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,-1,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,95,97,99,101,103,105,-1,107,109,111,-1,-1,-1,113,115,-1,-1,-1,117,119,-1,-1,121,-1,123,125,-1,127,129,131,133,135,137,139,-1,141,143,145,-1,-1,-1,-1,-1,-1,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0309343E0,2.3947406E0,5.251569E0,1.1266184E0,1.4036779E0,2.6408625E0,2.0567906E0,1.1718599E0,7.449785E-1,5.233407E-1,1.8306017E-1,1.8167834E0,2.0545034E0,8.9119744E-1,5.3063536E-1,1.0088335E0,1.3066593E0,2.810663E-1,2.6799837E-1,5.0485206E-1,4.6106625E-1,0E0,0E0,7.9980826E-1,6.794481E-1,0E0,1.1411185E0,3.8543832E-1,4.3394732E-1,4.1529304E-1,0E0,6.5222466E-1,5.2366436E-1,6.0268974E-1,5.160581E-1,1.3542596E-1,2.7066624E-1,3.072512E-1,1.7739183E-1,0E0,4.8277068E-1,3.038345E-1,2.4451852E-1,6.3614583E-1,3.713933E-1,6.177743E-1,3.2274425E-1,9.978113E-1,2.6831317E-1,2.992828E-1,1.6348064E-1,3.7258816E-1,0E0,1.3828921E-1,2.1716471E-1,2.9754865E-1,8.6197627E-1,4.6541834E-1,1.2939002E-1,0E0,2.8549123E-1,1.0660765E-1,3.921355E-1,0E0,0E0,0E0,1.4414924E-1,1.1350362E-1,0E0,0E0,0E0,3.0452633E-1,3.3493537E-1,0E0,0E0,1.7137766E-1,0E0,5.77337E-1,1.5285826E-1,0E0,2.715453E-1,3.5283518E-1,1.9209367E-1,1.9236487E-1,2.269789E-1,7.1661973E-1,1.2564123E-1,0E0,1.9681072E-1,3.058889E-1,2.8676262E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.556293E-1,1.18547395E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,66,66,67,67,71,71,72,72,75,75,77,77,78,78,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,89,89,90,90,97,97,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,-1,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,96,98,100,102,104,106,-1,108,110,112,-1,-1,-1,114,116,-1,-1,-1,118,120,-1,-1,122,-1,124,126,-1,128,130,132,134,136,138,140,-1,142,144,146,-1,-1,-1,-1,-1,-1,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.449702E2,1.6204434E-2,4.0844156E7,9.750871E6,4.3702424E7,1.1210787E3,3.4409692E3,9.669789E0,3.61E2,5.426937E5,4.8052647E5,4.428154E-6,1.296E3,1.2427474E8,4.33E2,2E0,2E0,1.4462532E7,1.3E1,2.8972292E-5,4.302613E6,-2.1758405E-2,-9.217794E-3,1E0,1.3152658E7,1.8346475E-2,4.5505118E2,8.454878E1,4.551684E6,8.35471E-2,-9.645017E-3,1.5182724E0,1.681727E6,8E0,3.114E3,3.6648486E2,9.538462E0,4.0465977E1,4E0,-1.8481953E-2,4.0701206E1,1.2E1,9.3902206E-1,8.0752426E1,6.689559E0,2.5682794E5,1.1968433E-1,7.064848E6,3.8398104E0,1.13814086E5,1.3664E4,8.5354E4,-1.4996774E-3,5.0991332E7,9.83E2,1.02036265E-5,6.4547E4,3.65625E1,5.650399E6,1.459638E-3,2.6668E4,3.188889E1,9.756616E2,3.9417247E-4,8.492466E-3,1.2693601E-2,4.1384277E2,4.9363803E5,-1.2989551E-2,-1.5539353E-2,-3.734213E-3,4.4705097E5,5.964E3,-1.6765192E-2,2.542055E-4,1.8081865E0,1.4750673E-3,8E0,8.3E1,9.358433E-3,8.87E2,4.832E3,1.0074342E-1,3.63E2,5.6903213E-1,7.273333E1,3.7E1,8.901675E-3,3.4E1,1.7190635E0,1.681727E6,-1.3314452E-4,1.1934996E-2,-1.0802491E-2,-2.310761E-2,1.4719202E-2,2.1609413E-3,7.5E2,9.532E3,-3.5303994E-3,3.6544942E-3,-2.9890074E-3,7.437525E-3,-3.6907475E-3,9.8273195E-3,-1.4600224E-3,9.5925825E-3,-3.0218448E-2,-6.2297494E-3,-2.8287754E-3,5.9021246E-3,-9.48032E-3,4.210484E-3,6.5397252E-3,-2.1920954E-3,6.4176233E-3,-8.51782E-4,-1.2384263E-2,-8.9633866E-4,-3.69534E-3,7.0596198E-3,-1.787673E-2,-7.836698E-3,6.811432E-3,2.5570607E-3,-1.6855827E-2,-2.937705E-3,5.970519E-3,-3.5471874E-3,-3.3738248E-3,1.2470211E-3,8.857069E-3,2.7091058E-3,-6.500371E-3,-3.0620597E-4,-7.824225E-3,-1.50835E-3,4.8974957E-3,-1.1927933E-3,1.759485E-2,3.0120611E-3,5.648826E-3,2.6137631E-2,-1.5954716E-2,-6.2940014E-3,-4.148103E-3,5.4299887E-3,1.9792689E-3,1.2356785E-2,-4.131064E-3,1.7997547E-3],"split_indices":[52,27,45,9,45,52,52,54,0,47,47,39,29,45,8,17,17,5,3,42,45,0,0,14,45,0,56,56,29,27,0,53,28,0,29,4,58,58,8,0,52,18,27,54,56,28,41,28,53,33,2,11,0,43,0,38,1,56,45,0,1,52,55,0,0,0,4,33,0,0,0,33,2,0,0,53,0,17,3,0,2,2,42,0,39,54,8,0,8,53,28,0,0,0,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.701E3,1.145E3,1.556E3,8.65E2,2.8E2,1.246E3,3.1E2,7.56E2,1.09E2,2.6E2,2E1,7.69E2,4.77E2,2.34E2,7.6E1,6.93E2,6.3E1,7E1,3.9E1,2.18E2,4.2E1,1.4E1,6E0,3.8E2,3.89E2,2.8E1,4.49E2,1.95E2,3.9E1,7E1,6E0,2.93E2,4E2,1.2E1,5.1E1,2.8E1,4.2E1,2.1E1,1.8E1,5E0,2.13E2,1.4E1,2.8E1,3.16E2,6.4E1,2.47E2,1.42E2,4.32E2,1.7E1,1.75E2,2E1,3.2E1,7E0,1.2E1,5.8E1,1.37E2,1.56E2,3.89E2,1.1E1,2E0,1E1,1.3E1,3.8E1,2.3E1,5E0,1.8E1,2.4E1,1.9E1,2E0,3E0,1.5E1,3E1,1.83E2,2E0,1.2E1,2.6E1,2E0,3.08E2,8E0,6E0,5.8E1,1.51E2,9.6E1,7.8E1,6.4E1,4.16E2,1.6E1,8E0,9E0,8.7E1,8.8E1,1.8E1,2E0,2.3E1,9E0,9E0,3E0,2.3E1,3.5E1,1.4E1,1.23E2,1.36E2,2E1,3.84E2,5E0,3E0,8E0,8E0,2E0,4E0,9E0,3.3E1,5E0,1.9E1,5E0,1.1E1,8E0,2.3E1,7E0,1.77E2,6E0,1.8E1,8E0,1.84E2,1.24E2,2E0,6E0,7E0,5.1E1,8.2E1,6.9E1,1.3E1,8.3E1,1.3E1,6.5E1,4.5E1,1.9E1,3.66E2,5E1,1.4E1,2E0,2E0,7E0,8E0,7.9E1,8.1E1,7E0,1.5E1,8E0,1E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[9.887444E-4,-2.4525387E-2,7.195976E-2,-1.1019293E-2,-1.9117817E-1,1.3340597E-1,-2.9271409E-2,-6.7987114E-2,1.1136018E-2,-1.7343913E-1,-4.1863322E-1,3.5362384E-1,1.1822643E-1,-6.474622E-2,8.306102E-2,-1.4982256E-3,-9.7931124E-2,3.3972546E-2,-5.458534E-2,-1.8194837E-2,-1.562214E-1,-2.4213742E-2,-7.7473735E-3,1.9046277E-2,7.881211E-3,6.970366E-3,1.456019E-1,-8.371007E-2,5.8122E-2,1.4506757E-1,-7.4695244E-2,-3.9558712E-2,1.0298167E-1,-6.9661625E-2,-1.9670612E-1,5.482689E-2,-2.6974607E-2,-2.5557252E-2,-4.72938E-2,-1.6614743E-1,3.6687737E-3,-3.285133E-2,1.5259601E-1,2.7677628E-1,1.24673E-1,-5.927577E-2,-1.4682738E-1,1.1927485E-1,-1.5973743E-3,1.4993047E-2,8.50374E-2,6.691645E-3,-1.8273544E-2,-6.814294E-2,1.7764862E-1,1.7591112E-3,2.2419909E-1,-1.214927E-1,-3.5618484E-2,-2.5734082E-2,-1.7591558E-1,1.8430306E-1,4.4195753E-2,-3.9596695E-2,1.5105466E-2,-7.891135E-2,7.39839E-3,-1.4552577E-1,-2.613439E-1,-1.5080853E-2,-9.963273E-3,7.76593E-2,1.5894849E-2,2.5109738E-2,1.8126912E-1,1.1638704E-1,4.0116894E-1,-3.447174E-2,-1.4080681E-1,-7.1728475E-2,-2.4354361E-1,7.5916434E-3,-2.2941732E-3,1.5523177E-1,-1.2407547E-2,-2.583429E-3,5.590864E-3,5.163664E-3,-5.066285E-3,2.0692356E-2,1.5645848E-3,1.2811237E-2,1.8300931E-3,-1.3884139E-2,-5.0420095E-3,-2.4038514E-5,-5.909662E-3,-4.133331E-3,-1.2958851E-2,1.3251064E-2,4.697428E-3,3.7953406E-4,3.7476716E-3,-2.8889417E-3,4.056959E-3,-4.2277467E-3,6.8598106E-3,1.2664349E-3,-7.531224E-3,-9.270128E-3,-4.7905515E-3,-5.0881854E-3,-1.7662669E-2,-7.4840775E-3,5.9201405E-4,5.797252E-3,-5.8329953E-3,5.460449E-3,1.4918709E-2,7.4350233E-3,2.6396231E-3,7.1102446E-3,2.5051318E-2,-3.0133715E-3,3.37872E-3,-1.0752966E-2,-6.727715E-4,-6.44779E-3,2.9337471E-3,-1.9429612E-3,-1.3836312E-2,3.1966988E-3,1.1775641E-2,5.22427E-3,-2.9690512E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,-1,69,71,73,75,77,79,81,-1,-1,83,85,-1,87,89,-1,91,93,95,-1,97,99,101,103,-1,105,107,109,111,113,-1,115,-1,-1,117,119,121,123,125,127,129,-1,-1,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8618546E0,4.4432793E0,4.4202976E0,2.3079488E0,5.2454853E-1,1.4357123E0,1.0773354E0,1.0192163E0,1.9784707E0,4.348235E-1,1.4160573E-1,1.6789532E-1,1.2629447E0,4.8252642E-1,6.466608E-1,6.403227E-1,9.7198606E-1,1.2445295E0,1.162446E0,0E0,3.1316638E-1,0E0,0E0,0E0,0E0,4.8633495E-1,8.796902E-1,2.658131E-1,1.6061206E-1,4.4738E-1,4.770878E-1,7.447325E-1,3.370816E-1,4.8267245E-1,4.5788407E-1,9.953077E-1,1.0929122E0,0E0,5.816061E-1,2.0735645E-1,0E0,2.0233592E-1,2.1608049E-1,9.6883607E-1,6.1541796E-1,2.5804496E-1,3.3843172E-1,1.1913267E-1,0E0,0E0,2.4816921E-1,1.1306655E-1,0E0,7.0696527E-1,5.022956E-1,0E0,1.10607326E-1,2.6707292E-1,5.178083E-1,0E0,6.129334E-1,4.1097188E-1,8.332422E-1,6.2864506E-1,0E0,4.5592225E-1,3.9303046E-1,2.1013618E-1,3.108486E-1,2.3829067E-1,0E0,1.3591455E-1,0E0,0E0,2.622913E-1,6.629205E-1,1.6882348E-1,3.0621994E-1,3.152333E-1,2.5086E-1,1.8167233E-1,0E0,0E0,1.520341E-1,1.0316048E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51,53,53,54,54,56,56,57,57,58,58,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,71,71,74,74,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,-1,70,72,74,76,78,80,82,-1,-1,84,86,-1,88,90,-1,92,94,96,-1,98,100,102,104,-1,106,108,110,112,114,-1,116,-1,-1,118,120,122,124,126,128,130,-1,-1,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0855529E3,3.9712732E7,2.8839298E7,1.5288235E2,1.5265896E8,1.296E3,5.528107E3,2.4336365E3,6.015825E0,2.3446269E2,7.10309E2,3.4526646E1,8.079531E4,1.79E2,2.1897722E10,2.585E2,2.8169732E10,1.2471935E7,8.01E2,-1.8194837E-2,1.4622E4,-2.4213742E-2,-7.7473735E-3,1.9046277E-2,7.881211E-3,1.4847565E0,1.0918E4,1.0828989E8,8.848604E1,3.1664667E-3,1.921032E4,6.03368E5,5.52E2,4.54E2,5.2921E4,9.2058825E-1,1.4122614E2,-2.5557252E-2,4.832E3,9.6910744E7,3.6687737E-3,1.6018981E0,8.9550705E1,3.4246575E-3,2.4507338E4,1.8880464E-1,2.585911E5,1.13814086E5,-1.5973743E-3,1.4993047E-2,1.3235373E1,5.39978E5,-1.8273544E-2,5.8365756E-1,1.143316E6,1.7591112E-3,1.2094463E7,9.51505E-1,4.15E2,-2.5734082E-2,2.5943396E0,8E0,3.3030225E2,2.3314082E6,1.5105466E-2,2.3616017E11,1E0,3.488E3,4.6263345E-2,9.08912E5,-9.963273E-3,1.14E2,1.5894849E-2,2.5109738E-2,5.3534385E3,1.6015803E7,3.2917362E7,3.317425E3,7.683389E7,1.3664E4,6.99143E5,7.5916434E-3,-2.2941732E-3,9.609747E3,1.4777102E9,-2.583429E-3,5.590864E-3,5.163664E-3,-5.066285E-3,2.0692356E-2,1.5645848E-3,1.2811237E-2,1.8300931E-3,-1.3884139E-2,-5.0420095E-3,-2.4038514E-5,-5.909662E-3,-4.133331E-3,-1.2958851E-2,1.3251064E-2,4.697428E-3,3.7953406E-4,3.7476716E-3,-2.8889417E-3,4.056959E-3,-4.2277467E-3,6.8598106E-3,1.2664349E-3,-7.531224E-3,-9.270128E-3,-4.7905515E-3,-5.0881854E-3,-1.7662669E-2,-7.4840775E-3,5.9201405E-4,5.797252E-3,-5.8329953E-3,5.460449E-3,1.4918709E-2,7.4350233E-3,2.6396231E-3,7.1102446E-3,2.5051318E-2,-3.0133715E-3,3.37872E-3,-1.0752966E-2,-6.727715E-4,-6.44779E-3,2.9337471E-3,-1.9429612E-3,-1.3836312E-2,3.1966988E-3,1.1775641E-2,5.22427E-3,-2.9690512E-3],"split_indices":[52,45,45,52,45,29,52,51,53,52,52,58,28,3,5,47,31,45,2,0,2,0,0,0,0,39,9,45,56,27,4,9,2,2,1,53,56,0,2,45,0,41,58,57,4,38,28,33,0,0,57,11,0,54,9,0,45,58,0,0,56,3,52,28,0,31,74,2,57,1,0,3,0,0,4,51,1,52,32,2,1,0,0,52,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.683E3,1.974E3,7.09E2,1.827E3,1.47E2,4.41E2,2.68E2,5.11E2,1.316E3,1.38E2,9E0,2.7E1,4.14E2,2.04E2,6.4E1,1.59E2,3.52E2,9.77E2,3.39E2,9E0,1.29E2,6E0,3E0,2.1E1,6E0,8.2E1,3.32E2,1.77E2,2.7E1,4.6E1,1.8E1,1.17E2,4.2E1,2.75E2,7.7E1,7.28E2,2.49E2,4E0,3.35E2,1.24E2,5E0,6.5E1,1.7E1,4.4E1,2.88E2,1.29E2,4.8E1,1.6E1,1.1E1,1.1E1,3.5E1,1.5E1,3E0,1.04E2,1.3E1,2.8E1,1.4E1,1.08E2,1.67E2,3E0,7.4E1,5.4E1,6.74E2,2.41E2,8E0,2.12E2,1.23E2,1.04E2,2E1,6E1,5E0,1.3E1,4E0,1.1E1,3.3E1,2.81E2,7E0,1E2,2.9E1,2.8E1,2E1,1.3E1,3E0,2E1,1.5E1,1E1,5E0,1.8E1,8.6E1,4E0,9E0,1.1E1,3E0,8E0,1E2,1.2E2,4.7E1,3.9E1,3.5E1,2.5E1,2.9E1,3.27E2,3.47E2,2.07E2,3.4E1,2.04E2,8E0,1.11E2,1.2E1,4.9E1,5.5E1,9E0,1.1E1,9E0,5.1E1,1.1E1,2E0,2.3E1,1E1,1.71E2,1.1E2,3E0,4E0,7.9E1,2.1E1,1.7E1,1.2E1,1.9E1,9E0,4E0,1.6E1,1.1E1,9E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-1.6559662E-3,-1.8388318E-2,9.913462E-2,4.051779E-3,-9.596363E-2,1.2776917E-1,-9.4699524E-2,-6.961674E-3,1.17416374E-1,-1.5892436E-1,-3.8647637E-2,2.2792218E-2,1.133612E-1,-5.3493947E-2,-2.1783056E-2,-4.527749E-2,1.6807841E-2,1.0457537E-1,1.8261148E-2,-1.312889E-1,-2.9865563E-1,-4.8860487E-2,5.208041E-3,1.8454486E-1,8.333584E-2,-2.519828E-1,-5.075688E-3,-1.0027864E-2,-8.892604E-2,1.6322132E-2,1.2902867E-2,1.152017E-1,-5.890125E-2,-1.19931616E-1,-2.1084866E-2,-3.3253524E-1,4.2811506E-3,-3.704676E-2,-7.994268E-3,2.3501131E-1,8.260755E-2,5.440575E-2,1.8324074E-1,-2.0760732E-2,-6.6180197E-3,4.881125E-2,-3.937605E-3,-1.1342228E-1,1.1906468E-2,9.584322E-2,-1.0662254E-1,-6.39047E-2,2.568909E-2,4.779193E-2,1.4535975E-1,-1.3537004E-2,3.2373004E-2,-1.3231637E-1,2.67368E-3,-3.6732095E-1,-1.7452479E-3,-6.241089E-2,3.591071E-2,1.7035443E-1,3.264756E-1,1.10322244E-1,-1.1077728E-2,-4.060792E-3,1.00634925E-1,1.4375915E-1,2.4793867E-2,-8.750407E-4,6.60153E-3,-4.5219874E-3,-2.1515278E-2,7.4300603E-3,-5.661807E-4,8.693243E-3,1.2749265E-3,-1.8071787E-2,-4.560616E-3,-3.8497276E-3,9.119074E-3,4.759187E-3,4.4542472E-4,-4.4210185E-3,3.2148792E-3,9.4114365E-3,3.8121177E-3,6.9137393E-3,-5.5414536E-3,-4.543712E-3,-9.252605E-3,-1.0807908E-2,-2.09527E-2,-2.5205505E-3,-1.0417621E-2,9.208026E-3,7.095071E-5,9.077823E-3,-2.1563484E-3,1.8783804E-2,9.033643E-3,7.0402366E-3,-5.260249E-3,5.050018E-3,-1.1037405E-3,2.8665848E-3,8.697523E-3,1.1204367E-2,-2.1849193E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,-1,51,53,55,57,-1,59,-1,61,-1,63,65,67,69,-1,-1,71,-1,73,75,77,79,81,83,85,87,-1,89,91,-1,93,-1,95,97,99,101,103,-1,105,107,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5485287E0,4.0268536E0,2.1434352E0,2.241939E0,1.8667507E0,1.6264725E0,7.176807E-1,1.4916561E0,4.9892116E-1,9.145899E-1,4.162708E-1,0E0,6.7497396E-1,4.408034E-1,0E0,9.6380734E-1,1.2747858E0,2.7257037E-1,0E0,6.7711306E-1,5.556061E-1,3.508889E-1,0E0,4.6930575E-1,6.5189946E-1,1.2599498E-1,1.619839E-1,7.905754E-1,9.257548E-1,0E0,9.837203E-1,2.8579748E-1,2.1338989E-1,4.4709063E-1,0E0,3.6770678E-1,0E0,4.3525887E-1,0E0,3.1695247E-1,3.1262654E-1,4.842205E-1,5.9906685E-1,0E0,0E0,1.3989142E-1,0E0,3.4679163E-1,9.8072547E-1,1.4344007E-1,7.536564E-1,6.14693E-1,1.0351977E0,1.3082775E-1,3.1741643E-1,0E0,1.4937882E-1,4.1250682E-1,0E0,2.2934914E-1,0E0,2.525038E-1,3.2810283E-1,1.6980529E-1,1.5259004E-1,2.6651946E-1,0E0,1.6876599E-1,3.1933916E-1,8.1352985E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,35,35,37,37,39,39,40,40,41,41,42,42,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,59,59,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,-1,52,54,56,58,-1,60,-1,62,-1,64,66,68,70,-1,-1,72,-1,74,76,78,80,82,84,86,88,-1,90,92,-1,94,-1,96,98,100,102,104,-1,106,108,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,2.0304577E-1,2.3580047E10,3.2196458E6,4.7777777E0,7.969174E7,1.4299594E8,2.5275471E2,2.1944155E3,1.4400111E10,1E0,2.2792218E-2,4.52737E5,5.4091E4,-2.1783056E-2,2.5943396E0,8E0,3.2E1,1.8261148E-2,4.5723195E6,5.8530144E10,7.218466E7,5.208041E-3,2.8721826E7,1.08874E5,2.7504245E10,2.47E2,3.75E1,1.0195312E0,1.6322132E-2,3.7510395E4,5.6553E4,3.7057E5,2.3920168E6,-2.1084866E-2,9.099893E8,4.2811506E-3,1.0323588E3,-7.994268E-3,2.2216E4,5.6812387E0,1E0,4.516183E1,-2.0760732E-2,-6.6180197E-3,4.921397E0,-3.937605E-3,1.56E2,9.2058825E-1,6E0,5.7616882E1,1.2502964E9,1.823E3,4.3698645E6,9.1E3,-1.3537004E-2,2.52E2,8.15E2,2.67368E-3,8.853684E-1,-1.7452479E-3,7.4709034E-1,6.5346925E6,1E0,1.9921019E9,4.5723195E6,-1.1077728E-2,1.2059859E1,4.5575E4,4.13568E5,2.4793867E-2,-8.750407E-4,6.60153E-3,-4.5219874E-3,-2.1515278E-2,7.4300603E-3,-5.661807E-4,8.693243E-3,1.2749265E-3,-1.8071787E-2,-4.560616E-3,-3.8497276E-3,9.119074E-3,4.759187E-3,4.4542472E-4,-4.4210185E-3,3.2148792E-3,9.4114365E-3,3.8121177E-3,6.9137393E-3,-5.5414536E-3,-4.543712E-3,-9.252605E-3,-1.0807908E-2,-2.09527E-2,-2.5205505E-3,-1.0417621E-2,9.208026E-3,7.095071E-5,9.077823E-3,-2.1563484E-3,1.8783804E-2,9.033643E-3,7.0402366E-3,-5.260249E-3,5.050018E-3,-1.1037405E-3,2.8665848E-3,8.697523E-3,1.1204367E-2,-2.1849193E-3],"split_indices":[2,27,5,28,54,7,45,52,4,5,89,0,29,33,0,56,18,3,0,28,5,45,0,50,2,5,8,4,53,0,28,1,9,28,0,7,0,52,0,2,57,102,54,0,0,54,0,2,53,8,50,7,29,45,29,0,0,0,0,27,0,38,45,6,12,28,0,56,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.695E3,2.312E3,3.83E2,1.794E3,5.18E2,3.34E2,4.9E1,1.636E3,1.58E2,2.46E2,2.72E2,1.2E1,3.22E2,4.5E1,4E0,6.26E2,1.01E3,1.52E2,6E0,2.07E2,3.9E1,2.55E2,1.7E1,9.4E1,2.28E2,8E0,3.7E1,3.47E2,2.79E2,1.1E1,9.99E2,1.43E2,9E0,2.01E2,6E0,3.6E1,3E0,2.33E2,2.2E1,6.2E1,3.2E1,1.78E2,5E1,2E0,6E0,2.2E1,1.5E1,6E1,2.87E2,2.4E1,2.55E2,1.42E2,8.57E2,4.5E1,9.8E1,2E0,7E0,1.88E2,1.3E1,3.2E1,4E0,1.73E2,6E1,3.8E1,2.4E1,3E1,2E0,7.9E1,9.9E1,4.6E1,4E0,1.3E1,9E0,5.8E1,2E0,4E1,2.47E2,1E1,1.4E1,9E0,2.46E2,1.34E2,8E0,1.55E2,7.02E2,5E0,4E1,5.4E1,4.4E1,4E0,3E0,1.18E2,7E1,1.2E1,2E1,1.64E2,9E0,1E1,5E1,3.5E1,3E0,1.5E1,9E0,2.6E1,4E0,1.1E1,6.8E1,6.7E1,3.2E1,3.1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-6.4784044E-4,-2.5233878E-2,6.7146726E-2,-2.9729814E-3,-1.0441139E-1,1.2173669E-1,-2.4225794E-2,-2.8662782E-2,4.0451635E-2,-8.489691E-2,-2.6121217E-1,2.703225E-1,1.02194525E-1,-7.1208715E-2,7.001226E-2,-4.888E-2,1.6194943E-2,-3.0950438E-2,6.926519E-2,-1.6115832E-1,-5.157145E-2,-3.7481228E-4,-3.1298804E-1,2.271352E-1,2.2703102E-2,1.5177262E-1,6.309901E-2,-3.5791487E-2,-1.720627E-1,1.972347E-1,3.723425E-2,-2.778102E-2,-1.4111225E-1,1.462751E-1,-5.043668E-3,1.4437805E-1,-5.4920226E-2,2.2730062E-2,1.10283524E-1,-1.2401643E-1,-3.332386E-1,-5.7831737E-3,-9.155573E-2,-4.5460775E-1,-1.9470444E-1,1.5453053E-2,1.4858207E-1,1.0128849E-1,2.6114896E-1,3.5002645E-2,2.1221381E-1,-5.310686E-2,7.0895836E-2,-1.3940993E-1,-1.9727707E-2,1.16133345E-2,-1.0444602E-3,-8.78893E-3,5.6447662E-2,-1.2780482E-1,-1.4302397E-2,-2.0192072E-2,-1.2565279E-1,2.0587362E-1,5.9254025E-4,-1.0444039E-2,1.2944997E-2,1.2630209E-3,9.246121E-3,-1.6242872E-1,-3.3260952E-3,5.8488235E-2,-9.12574E-2,2.1339162E-1,8.904922E-2,-2.1956088E-1,-6.653874E-2,-2.1086987E-2,-1.5344176E-1,4.2145785E-2,-5.1411055E-2,-3.005143E-1,-7.012048E-2,-2.5847124E-2,-6.8202666E-3,1.640371E-3,-2.3076062E-1,-2.7464363E-3,9.033624E-3,7.130596E-2,1.1457292E-2,1.8583316E-2,2.2349165E-1,-4.1750398E-2,7.130681E-2,3.1848434E-1,6.611231E-2,-4.1562174E-2,-1.07891355E-2,1.4931704E-1,-4.0645422E-3,-1.8826324E-1,2.9039374E-3,1.971702E-1,3.402477E-2,-5.310257E-3,-2.082836E-2,7.7159996E-3,-1.5564454E-3,-1.6211657E-2,-5.027136E-3,1.22682145E-2,2.0027268E-3,1.594673E-3,-1.5081464E-3,-2.0092782E-3,-1.6175497E-2,1.1800812E-3,-7.1810232E-3,3.9254725E-3,-1.4969043E-3,-2.0730733E-3,-1.0706605E-2,-3.1575295E-3,1.3000084E-2,4.5381053E-3,-8.90347E-3,-5.0575035E-3,-1.24193635E-2,9.204534E-3,-4.6785614E-3,-1.5336561E-2,-7.982388E-4,1.0422741E-2,7.592345E-4,7.231377E-3,-3.2162883E-3,-4.489937E-3,-1.6913354E-2,-1.4330022E-2,-2.808984E-3,-6.045971E-3,-1.4596844E-2,4.0617674E-3,-4.532039E-3,5.2475873E-3,1.2153368E-2,-2.5568295E-2,-7.67144E-4,9.239419E-3,1.7663981E-3,1.872536E-2,4.027887E-3,6.738187E-3,-1.4558717E-2,-4.4571045E-3,-1.186265E-4,3.2826061E-3,1.2683579E-2,-1.2300761E-2,-2.7637053E-3,1.4255068E-2,2.7415876E-3,-6.081176E-4,6.023156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,-1,-1,-1,-1,103,105,107,-1,109,111,-1,113,-1,-1,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,-1,-1,-1,141,-1,-1,143,-1,-1,145,147,149,151,153,155,-1,157,-1,159,-1,161,163,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.440382E0,3.4454985E0,3.5394576E0,1.7046344E0,1.2905159E0,1.2616119E0,1.1827216E0,8.7141395E-1,1.170717E0,9.6506286E-1,6.1415577E-1,3.410716E-1,7.5410604E-1,6.2965196E-1,3.637948E-1,1.2843359E0,8.267916E-1,6.9715357E-1,7.7055454E-1,7.0007396E-1,4.9012786E-1,0E0,5.647862E-1,2.8227258E-1,0E0,9.288149E-1,9.226597E-1,2.4946004E-1,3.0286455E-1,1.8375039E-1,3.1574488E-1,7.26191E-1,4.8042893E-1,3.326288E-1,3.8646358E-1,1.098209E-1,8.037467E-1,7.8385025E-1,4.5172477E-1,5.18083E-1,3.17482E-1,2.7771577E-1,6.1943614E-1,3.7481427E-1,1.9777632E-1,0E0,2.2524536E-1,4.7927403E-1,1.7130351E-1,5.268307E-1,5.22364E-1,2.1923015E-1,2.4476656E-1,4.2187965E-1,0E0,0E0,0E0,0E0,2.0470348E-1,2.8437078E-1,1.5279305E0,0E0,4.8393548E-1,2.285862E-1,0E0,2.348979E-1,0E0,0E0,0E0,9.7859466E-1,4.0965247E-1,3.1020698E-1,2.8227097E-1,6.017543E-1,3.002596E-1,1.3575447E-1,5.1854295E-1,0E0,1.9998728E-1,2.8262094E-1,2.1733229E-1,1.0841322E-1,3.2658082E-1,0E0,0E0,0E0,1.1627698E-1,0E0,0E0,2.3217052E-1,0E0,0E0,1.239779E-1,7.639273E-1,5.287598E-1,3.070519E-1,4.844643E-1,2.1920122E-1,0E0,1.03156835E-1,0E0,2.89631E-1,0E0,1.09506875E-1,2.545042E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,58,58,59,59,60,60,62,62,63,63,65,65,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,79,79,80,80,81,81,82,82,86,86,89,89,92,92,93,93,94,94,95,95,96,96,97,97,99,99,101,101,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,-1,-1,-1,-1,104,106,108,-1,110,112,-1,114,-1,-1,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,-1,-1,-1,142,-1,-1,144,-1,-1,146,148,150,152,154,156,-1,158,-1,160,-1,162,164,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0855529E3,2.0304577E-1,2.8839298E7,4.930349E5,1.2199979E10,1.7742857E1,3.970405E3,3.386E3,1E0,1.9121015E0,2.0469315E0,6.5317163E3,5.8805E4,1.0828989E8,8.837831E-2,8.0391425E6,6.4E1,5E-1,6.9432104E7,1.9577E4,1.5316E4,-3.7481228E-4,2.4434783E0,9.397667E6,2.2703102E-2,1.3645918E0,4.4839956E3,4.559E3,2.778E3,4.1221875E-1,3.33231E5,4.7E2,9.073025E6,1.3645918E0,1.96E4,4.86E2,8.15E2,5.9378143E-2,1.3396206E-2,9.38E2,8E0,1.2471935E7,1.3441292E4,3.285E3,2.7777777E0,1.5453053E-2,2.184578E1,7.203918E9,1.0918E4,1.526038E6,3.4231E4,4.5575E4,8.962E3,2.8E1,-1.9727707E-2,1.16133345E-2,-1.0444602E-3,-8.78893E-3,2E0,1.56E2,1E0,-2.0192072E-2,1.394E3,1E0,5.9254025E-4,8.134936E0,1.2944997E-2,1.2630209E-3,9.246121E-3,2.2884E4,9.800051E6,6.710987E7,2.4E1,1.6841E4,1.3543708E7,1E0,2.014576E8,-2.1086987E-2,4.5254502E-2,1E0,5.684E3,1.2352941E0,1.71E2,-2.5847124E-2,-6.8202666E-3,1.640371E-3,1.387623E8,-2.7464363E-3,9.033624E-3,1.9221336E1,1.1457292E-2,1.8583316E-2,1.2974394E6,3.90106E0,1.5851064E0,1.7049885E0,1.5005797E4,1.2E1,-1.07891355E-2,2.8363848E7,-4.0645422E-3,1.2068E4,2.9039374E-3,1.1590965E0,1.5595E4,-5.310257E-3,-2.082836E-2,7.7159996E-3,-1.5564454E-3,-1.6211657E-2,-5.027136E-3,1.22682145E-2,2.0027268E-3,1.594673E-3,-1.5081464E-3,-2.0092782E-3,-1.6175497E-2,1.1800812E-3,-7.1810232E-3,3.9254725E-3,-1.4969043E-3,-2.0730733E-3,-1.0706605E-2,-3.1575295E-3,1.3000084E-2,4.5381053E-3,-8.90347E-3,-5.0575035E-3,-1.24193635E-2,9.204534E-3,-4.6785614E-3,-1.5336561E-2,-7.982388E-4,1.0422741E-2,7.592345E-4,7.231377E-3,-3.2162883E-3,-4.489937E-3,-1.6913354E-2,-1.4330022E-2,-2.808984E-3,-6.045971E-3,-1.4596844E-2,4.0617674E-3,-4.532039E-3,5.2475873E-3,1.2153368E-2,-2.5568295E-2,-7.67144E-4,9.239419E-3,1.7663981E-3,1.872536E-2,4.027887E-3,6.738187E-3,-1.4558717E-2,-4.4571045E-3,-1.186265E-4,3.2826061E-3,1.2683579E-2,-1.2300761E-2,-2.7637053E-3,1.4255068E-2,2.7415876E-3,-6.081176E-4,6.023156E-3],"split_indices":[52,27,45,28,5,58,52,2,102,53,56,4,9,45,57,45,11,53,7,9,9,0,53,45,0,38,52,0,0,38,29,1,45,38,2,12,2,38,41,2,3,45,51,0,53,0,56,12,9,9,10,2,0,8,0,0,0,0,8,2,53,0,12,101,0,58,0,0,0,1,50,7,8,9,47,104,7,0,39,75,9,56,0,0,0,0,7,0,0,53,0,0,28,56,53,38,4,18,0,1,0,9,0,42,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.662E3,1.954E3,7.08E2,1.526E3,4.28E2,4.43E2,2.65E2,9.59E2,5.67E2,3.82E2,4.6E1,5E1,3.93E2,1.77E2,8.8E1,6.61E2,2.98E2,1.63E2,4.04E2,1.15E2,2.67E2,8E0,3.8E1,4.3E1,7E0,1.72E2,2.21E2,1.32E2,4.5E1,1.7E1,7.1E1,5.39E2,1.22E2,4.1E1,2.57E2,1.9E1,1.44E2,1.9E2,2.14E2,9.6E1,1.9E1,1.25E2,1.42E2,1.6E1,2.2E1,1.8E1,2.5E1,1.19E2,5.3E1,1.87E2,3.4E1,1.14E2,1.8E1,4.1E1,4E0,1.4E1,3E0,5E0,6.6E1,6.3E1,4.76E2,5E0,1.17E2,2.8E1,1.3E1,2.53E2,4E0,6E0,1.3E1,4.6E1,9.8E1,1.45E2,4.5E1,3.5E1,1.79E2,3.5E1,6.1E1,1.1E1,8E0,6.1E1,6.4E1,1.2E1,1.3E2,1.2E1,4E0,3E0,1.9E1,4E0,2.1E1,9.9E1,2E1,1E1,4.3E1,6E1,1.27E2,1.9E1,1.5E1,1.08E2,6E0,1.2E1,6E0,3.3E1,8E0,8E0,5.8E1,6.1E1,2E0,4.4E1,4.32E2,9E0,1.08E2,2.1E1,7E0,8.2E1,1.71E2,2.8E1,1.8E1,8.3E1,1.5E1,1.15E2,3E1,3.4E1,1.1E1,6E0,2.9E1,1.76E2,3E0,1E1,2.5E1,6E0,5.5E1,3E0,5E0,7E0,5.4E1,4E0,6E1,3E0,9E0,5E0,1.25E2,9E0,1E1,9.2E1,7E0,1E1,3.3E1,2E0,5.8E1,2.7E1,1E2,1.4E1,5E0,1.3E1,2E0,4.6E1,6.2E1,8E0,4E0,2.1E1,1.2E1,4E0,4E0,3.9E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"165","size_leaf_vector":"1"}},{"base_weights":[-2.0076171E-3,-4.870162E-2,2.691012E-2,-1.46295745E-2,-9.4747275E-2,1.2061991E-2,1.4380443E-1,-6.1810456E-2,4.7111306E-2,-7.5717464E-2,-2.0249148E-1,7.641427E-2,-1.6111728E-2,1.11326136E-1,2.3553684E-1,-4.6479728E-2,-2.3554066E-1,1.4968385E-2,2.8191643E-2,-1.8357475E-1,-6.293492E-2,-3.5883608E-1,-1.197171E-1,5.0940625E-2,1.8490209E-1,-3.0853827E-2,3.151756E-2,1.1905103E-1,-1.3463357E-2,1.4403536E-2,3.4479382E-3,1.5837513E-1,-5.7432942E-2,2.2436916E-3,-4.326127E-1,3.5272256E-2,-1.9163532E-2,-2.1806498E-1,-1.085534E-3,2.7006788E-2,-8.186498E-2,-4.4853476E-1,-2.1495116E-3,-1.9094941E-1,-3.260793E-2,5.5944543E-2,-2.7370095E-2,6.835099E-2,2.4403836E-1,-3.582238E-2,8.74366E-3,1.6838523E-2,1.8589413E-1,8.9987904E-2,1.8116753E-1,1.102255E-2,1.6582279E-2,7.476529E-2,-6.783779E-2,-2.9447448E-2,-5.2877665E-3,2.2065572E-2,2.171359E-1,-2.3357237E-3,-2.7246377E-1,-6.7885496E-2,1.41986E-1,-1.414822E-1,-5.65054E-2,-2.3962684E-2,-1.9098137E-3,-2.5674424E-1,-3.5037773E-3,1.6865251E-3,-6.0798516E-3,-8.173761E-2,7.042154E-2,9.119173E-3,7.3310995E-4,1.08240165E-1,2.9565164E-1,1.11089855E-1,-4.1332815E-2,1.9497922E-1,2.1272635E-3,1.2944463E-2,3.1620425E-3,1.4643398E-1,3.28573E-2,2.9542074E-1,1.09409705E-1,-7.828165E-3,3.5954067E-3,-5.474116E-3,5.2983453E-3,-2.0778517E-3,-8.718658E-3,3.1603104E-3,-7.1065925E-4,-1.5013763E-3,1.5944395E-2,-1.690991E-2,-7.777802E-3,1.195298E-3,-9.370812E-3,2.592198E-3,9.993042E-3,-1.0332727E-2,-2.5888206E-3,-1.461737E-3,-7.23384E-3,-1.6775817E-2,-4.7788485E-3,5.0653187E-3,-5.9470087E-3,7.377344E-3,2.451832E-3,1.4215062E-2,2.3601492E-4,1.4931703E-2,3.4709016E-3,2.494668E-3,1.2072032E-2,-4.472754E-3,-1.4506669E-3,3.479822E-3,1.2216497E-2,1.4978254E-3,-2.4273016E-3,1.146933E-2,3.956574E-3,5.33965E-3,-1.4417202E-3,8.191524E-3,2.2709826E-2,-4.4733007E-3,7.0449086E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,57,-1,59,61,-1,63,-1,65,67,69,-1,71,73,75,-1,77,79,81,-1,83,85,87,89,91,-1,93,95,-1,-1,97,99,-1,101,103,105,107,109,-1,-1,111,-1,-1,-1,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,131,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6295266E0,1.6114118E0,2.879464E0,1.7274017E0,8.80672E-1,2.6736565E0,5.309098E-1,8.839581E-1,1.2818654E0,5.026982E-1,7.982273E-1,1.2314284E0,7.213841E-1,4.5790148E-1,4.9034142E-1,7.046798E-1,1.5228555E0,0E0,7.450532E-1,2.1149719E-1,5.725825E-1,5.9941244E-1,2.6679283E-1,1.1688423E0,5.691736E-1,8.379234E-1,5.475321E-1,2.328285E-1,0E0,0E0,0E0,4.391339E-1,4.1084504E-1,0E0,8.717849E-1,5.66841E-1,0E0,2.8140962E-1,0E0,6.5336657E-1,4.1079283E-1,3.401091E-1,0E0,1.67211E-1,1.3884737E-1,7.292963E-1,0E0,1.9132736E-1,3.60502E-1,6.2551636E-1,0E0,5.839956E-1,1.9460332E-1,3.0476916E-1,3.2021618E-1,1.24032766E-1,0E0,1.6184548E-1,7.572776E-1,0E0,0E0,3.6246973E-1,4.7082555E-1,0E0,1.5562928E-1,3.96403E-1,1.4687818E-1,5.208039E-1,4.7687E-1,0E0,0E0,1.8529743E-1,0E0,0E0,0E0,2.9166663E-1,5.2103865E-1,0E0,0E0,3.2974762E-1,1.2145424E-1,2.1977279E-1,4.2255914E-1,1.0646266E-1,3.2023963E-1,0E0,0E0,2.6526618E-1,2.4597E-1,2.7466238E-1,2.2780353E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,35,35,37,37,39,39,40,40,41,41,43,43,44,44,45,45,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,57,57,58,58,61,61,62,62,64,64,65,65,66,66,67,67,68,68,71,71,75,75,76,76,79,79,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,58,-1,60,62,-1,64,-1,66,68,70,-1,72,74,76,-1,78,80,82,-1,84,86,88,90,92,-1,94,96,-1,-1,98,100,-1,102,104,106,108,110,-1,-1,112,-1,-1,-1,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,132,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.9173258E-4,1.3664E4,2.41E3,4.7022E4,2E0,6.357613E-1,4.3978744E0,8E0,7.22E2,6.258335E5,2.84E2,1.3593E4,6.13912E5,3.1029554E7,7E0,1.3E1,1.4968385E-2,4.8916136E7,9.824211E5,5.5700205E8,1E0,1.010368E6,9.216E3,6.189387E6,1.5963264E9,2.3565867E3,8.015419E3,-1.3463357E-2,1.4403536E-2,3.4479382E-3,1.3984146E2,4.144144E-1,2.2436916E-3,9.99E2,3.832274E2,-1.9163532E-2,1.6358411E8,-1.085534E-3,1.21E3,4.1365924E2,1.91E2,-2.1495116E-3,2.5030267E5,2.9964134E9,1.5288235E2,-2.7370095E-2,1.2469E5,1.44657E5,9.61E2,8.74366E-3,8.963196E6,2.0189162E7,1.0254098E7,8.454878E1,3.6E1,1.6582279E-2,1E0,3.37E2,-2.9447448E-2,-5.2877665E-3,1.6837112E7,1E1,-2.3357237E-3,5.85E2,2.047619E0,3.8977896E2,1.2123E4,1.1855755E8,-2.3962684E-2,-1.9098137E-3,9.026015E2,-3.5037773E-3,1.6865251E-3,-6.0798516E-3,7.427971E2,3.5452E4,9.119173E-3,7.3310995E-4,1.366E3,2.8863275E0,2.5682794E5,8.87E2,1.3794063E1,1.1756504E5,1.2944463E-2,3.1620425E-3,1.526038E6,1.2469E5,1.21320024E8,2.4394053E9,-7.828165E-3,3.5954067E-3,-5.474116E-3,5.2983453E-3,-2.0778517E-3,-8.718658E-3,3.1603104E-3,-7.1065925E-4,-1.5013763E-3,1.5944395E-2,-1.690991E-2,-7.777802E-3,1.195298E-3,-9.370812E-3,2.592198E-3,9.993042E-3,-1.0332727E-2,-2.5888206E-3,-1.461737E-3,-7.23384E-3,-1.6775817E-2,-4.7788485E-3,5.0653187E-3,-5.9470087E-3,7.377344E-3,2.451832E-3,1.4215062E-2,2.3601492E-4,1.4931703E-2,3.4709016E-3,2.494668E-3,1.2072032E-2,-4.472754E-3,-1.4506669E-3,3.479822E-3,1.2216497E-2,1.4978254E-3,-2.4273016E-3,1.146933E-2,3.956574E-3,5.33965E-3,-1.4417202E-3,8.191524E-3,2.2709826E-2,-4.4733007E-3,7.0449086E-3],"split_indices":[102,27,2,2,9,17,38,53,18,2,32,0,10,2,47,3,0,0,48,32,5,16,9,10,45,7,52,4,0,0,0,52,53,0,2,56,0,7,0,2,52,8,0,33,12,52,0,9,1,12,0,12,45,45,56,2,0,15,0,0,0,45,3,0,2,58,4,10,45,0,0,52,0,0,0,33,9,0,0,10,53,28,2,56,33,0,0,9,9,32,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.686E3,1.027E3,1.659E3,5.91E2,4.36E2,1.473E3,1.86E2,3.35E2,2.56E2,3.72E2,6.4E1,4.48E2,1.025E3,1.39E2,4.7E1,3.09E2,2.6E1,1.6E1,2.4E2,3.8E1,3.34E2,2.1E1,4.3E1,3.64E2,8.4E1,7.83E2,2.42E2,1.37E2,2E0,3.3E1,1.4E1,1.5E1,2.94E2,1.1E1,1.5E1,2.37E2,3E0,3.1E1,7E0,5.8E1,2.76E2,1.6E1,5E0,2.3E1,2E1,3.62E2,2E0,2.9E1,5.5E1,7.66E2,1.7E1,2.22E2,2E1,9.5E1,4.2E1,9E0,6E0,2.1E1,2.73E2,9E0,6E0,2.22E2,1.5E1,8E0,2.3E1,3.2E1,2.6E1,8.1E1,1.95E2,1.4E1,2E0,1.4E1,9E0,1.2E1,8E0,3.4E1,3.28E2,8E0,2.1E1,1.6E1,3.9E1,2.7E1,7.39E2,1.6E1,2.06E2,1.1E1,9E0,4.7E1,4.8E1,1.5E1,2.7E1,2E0,7E0,3E0,1.8E1,2.26E2,4.7E1,1.01E2,1.21E2,5E0,1E1,1.2E1,1.1E1,1.9E1,1.3E1,1.2E1,1.4E1,4.3E1,3.8E1,1.54E2,4.1E1,8E0,6E0,6E0,2.8E1,6E1,2.68E2,5E0,1.1E1,3.6E1,3E0,2E1,7E0,1.28E2,6.11E2,6E0,1E1,1.33E2,7.3E1,1.8E1,2.9E1,2.1E1,2.7E1,1E1,5E0,4E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-8.2744757E-4,1.9621057E-2,-6.8965584E-2,-2.013101E-2,5.1293805E-2,-1.5249503E-1,-4.2346045E-2,-7.2814627E-3,-1.2626216E-1,7.463365E-2,-2.0917306E-2,-3.724339E-1,-1.24889635E-1,-6.0125694E-2,5.6030158E-2,3.5122536E-2,-3.7350655E-2,-3.8979778E-1,-8.137085E-2,9.5279776E-2,-4.5441445E-2,-1.3487735E-1,8.950262E-3,-5.543325E-3,-2.1471694E-2,-2.6584837E-1,-9.608623E-2,-2.9694492E-2,-1.3897687E-1,9.067925E-3,3.640702E-2,4.712802E-2,-8.9304514E-2,-4.887578E-2,3.481073E-2,1.8499856E-3,-4.6442145E-1,-4.080366E-2,-2.0306118E-1,1.9394626E-1,8.3773814E-2,-6.781334E-2,1.0330686E-1,-3.212199E-1,-9.863815E-2,2.2511403E-3,1.7886618E-2,6.152125E-3,-3.1170252E-1,-1.7401959E-1,4.0315334E-3,-6.1633077E-2,2.8837575E-2,-1.7458285E-1,-6.400877E-2,9.983201E-3,1.8293167E-2,-9.967301E-2,5.6046546E-2,4.318558E-3,-1.8355335E-1,-2.2776458E-2,-9.74474E-2,8.5357316E-2,-2.8690418E-2,-2.6462939E-2,-7.182562E-3,-1.0006911E-3,-1.0369102E-2,-1.4492958E-2,-3.6681868E-2,1.2030374E-1,1.6252365E-2,8.669115E-2,-1.6482532E-1,-4.4527132E-2,-2.399303E-1,2.4771935E-1,-4.315996E-3,2.738769E-3,-4.375062E-1,-5.327094E-3,3.5454326E-3,3.939288E-2,-5.0425075E-2,-7.967605E-3,-1.8062398E-2,-1.3924716E-1,-3.443887E-1,8.980047E-3,-3.5381686E-2,-2.6258139E-2,-9.313039E-2,5.2965283E-3,-6.5458374E-4,-9.470135E-2,-2.0655769E-1,2.707285E-3,-1.0477255E-1,9.516381E-3,8.476554E-3,1.3646448E-3,-2.1005223E-2,7.006528E-3,1.8649291E-3,-1.1596156E-2,-3.1324178E-3,-2.9935504E-3,2.6431184E-3,-1.2093748E-2,-3.2892714E-3,6.209021E-3,1.59456E-4,5.3032767E-3,-3.2642884E-3,3.3677008E-3,-4.457775E-3,-8.027926E-3,2.7168791E-3,4.0448373E-3,1.4696961E-2,2.9768997E-3,5.7863053E-3,-1.2466742E-2,3.8839544E-3,-3.032495E-3,1.0130233E-2,-6.3031698E-3,-1.649162E-2,1.8899888E-2,6.147796E-3,-2.523449E-2,-6.8314625E-3,-6.074895E-4,4.276421E-3,3.692465E-4,-3.723094E-3,-1.4121617E-2,-5.258141E-3,-2.2867095E-2,-8.432352E-3,-4.953844E-3,1.493814E-3,-2.7949866E-3,2.790207E-3,-6.4102923E-3,-2.3739735E-4,-9.020644E-3,-1.0075304E-3,-1.2667972E-2,-6.852777E-3,-6.663238E-4,-7.7939588E-3,-8.9466624E-4,3.51884E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,-1,-1,85,87,89,91,93,95,97,-1,99,101,103,-1,105,107,109,111,113,-1,-1,115,-1,-1,117,119,-1,121,123,125,127,129,-1,-1,131,-1,-1,133,135,-1,-1,137,139,-1,141,143,145,-1,-1,147,149,-1,151,-1,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.706199E0,2.5770636E0,1.3587806E0,1.2373542E0,1.921227E0,8.5206866E-1,8.2019585E-1,1.0353805E0,1.1276667E0,2.1388226E0,9.5084214E-1,2.5988436E-1,5.144477E-1,9.460896E-1,1.814652E-1,5.0762165E-1,3.967442E-1,4.6553183E-1,4.1132838E-1,8.1816244E-1,4.296337E-1,3.5998404E-1,5.41304E-1,0E0,0E0,4.3781924E-1,8.74583E-1,5.383041E-1,2.8244948E-1,0E0,1.9564386E-1,4.0883493E-1,5.2402586E-1,5.175318E-1,2.1429354E-1,0E0,2.4974322E-1,4.5438692E-1,3.367778E-1,7.7305627E-1,4.9295187E-1,4.3419904E-1,4.9719554E-1,4.210531E-1,1.1698821E-1,4.304569E-1,0E0,0E0,1.275121E-1,3.2526588E-1,3.6821216E-1,2.047987E-1,3.559861E-1,1.6944933E-1,1.877277E-1,0E0,1.02810085E-1,7.944324E-1,4.4356042E-1,0E0,1.225943E-1,8.3365345E-1,6.2055135E-1,1.3399854E-1,1.726489E-1,0E0,0E0,3.611862E-1,0E0,0E0,1.1960067E-1,3.2184684E-1,0E0,5.392399E-1,2.0898385E-1,4.8982006E-1,1.0172337E-1,1.2469363E-1,0E0,0E0,1.09387755E-1,0E0,0E0,3.363822E-1,1.4511311E-1,0E0,0E0,2.1513426E-1,1.3994789E-1,0E0,1.9352753E-1,2.4489534E-1,3.512199E-1,0E0,0E0,1.5471806E-1,1.5343356E-1,0E0,1.4522144E-1,0E0,1.0199347E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,67,67,70,70,71,71,73,73,74,74,75,75,76,76,77,77,80,80,83,83,84,84,87,87,88,88,90,90,91,91,92,92,95,95,96,96,98,98,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,-1,-1,86,88,90,92,94,96,98,-1,100,102,104,-1,106,108,110,112,114,-1,-1,116,-1,-1,118,120,-1,122,124,126,128,130,-1,-1,132,-1,-1,134,136,-1,-1,138,140,-1,142,144,146,-1,-1,148,150,-1,152,-1,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,3.6129813E2,1.9121015E0,9.669789E0,1.2328733E1,7E0,3.4409692E3,2E0,2E0,3.5301748E7,4.052E3,8.3933276E-1,1.2068E4,4.987639E7,1.4274633E7,2E0,2.437E3,8E0,1.4773677E5,9.42812E3,3.970405E3,4.745907E7,6.249511E9,-5.543325E-3,-2.1471694E-2,5.5544252E7,1.638058E2,5.674727E2,1.1832823E-2,9.067925E-3,1.5151515E-3,1.198E5,7.106795E6,2.4505E4,1.3152658E7,1.8499856E-3,1.1160287E7,2.970914E1,2.074E3,1.0485785E3,1.5005797E4,5.860658E-1,2.6422684E9,2.1325E4,1E0,2.7753687E-2,1.7886618E-2,6.152125E-3,6.41689E5,1.4977E5,8.640299E8,1.5773E4,1.25218E5,2.8277853E11,4.88954E5,9.983201E-3,4.3525005E3,1.08504E5,8.916961E2,4.318558E-3,2.2417E0,7.22E2,9E2,3.333E3,7.57657E4,-2.6462939E-2,-7.182562E-3,9.4E1,-1.0369102E-2,-1.4492958E-2,1.0238709E1,2.3369722E-3,1.6252365E-2,1.0855529E3,4.5505118E2,2.2216E4,8.637973E6,6.082581E3,-4.315996E-3,2.738769E-3,6.203E3,-5.327094E-3,3.5454326E-3,6.222E3,1.17E2,-7.967605E-3,-1.8062398E-2,4.26E2,1.05E2,8.980047E-3,7.804559E2,1.4157E4,1.3225807E0,5.2965283E-3,-6.5458374E-4,1E0,3.6282136E0,2.707285E-3,6.1238285E8,9.516381E-3,2.2153166E7,1.3646448E-3,-2.1005223E-2,7.006528E-3,1.8649291E-3,-1.1596156E-2,-3.1324178E-3,-2.9935504E-3,2.6431184E-3,-1.2093748E-2,-3.2892714E-3,6.209021E-3,1.59456E-4,5.3032767E-3,-3.2642884E-3,3.3677008E-3,-4.457775E-3,-8.027926E-3,2.7168791E-3,4.0448373E-3,1.4696961E-2,2.9768997E-3,5.7863053E-3,-1.2466742E-2,3.8839544E-3,-3.032495E-3,1.0130233E-2,-6.3031698E-3,-1.649162E-2,1.8899888E-2,6.147796E-3,-2.523449E-2,-6.8314625E-3,-6.074895E-4,4.276421E-3,3.692465E-4,-3.723094E-3,-1.4121617E-2,-5.258141E-3,-2.2867095E-2,-8.432352E-3,-4.953844E-3,1.493814E-3,-2.7949866E-3,2.790207E-3,-6.4102923E-3,-2.3739735E-4,-9.020644E-3,-1.0075304E-3,-1.2667972E-2,-6.852777E-3,-6.663238E-4,-7.7939588E-3,-8.9466624E-4,3.51884E-3],"split_indices":[27,52,53,54,53,3,52,17,17,45,2,56,9,45,45,6,2,0,33,47,52,7,7,0,0,45,52,52,38,0,57,7,5,29,45,0,45,58,2,52,4,41,7,1,101,57,0,0,29,29,5,9,29,31,32,0,4,7,33,0,53,2,2,2,28,0,0,3,0,0,53,41,0,52,56,2,47,52,0,0,29,0,0,2,3,0,0,2,52,0,55,9,57,0,0,16,53,0,7,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.658E3,2.045E3,6.13E2,9.07E2,1.138E3,1.47E2,4.66E2,8.1E2,9.7E1,8.6E2,2.78E2,1.5E1,1.32E2,3.95E2,7.1E1,3.36E2,4.74E2,1.3E1,8.4E1,7.34E2,1.26E2,5.7E1,2.21E2,4E0,1.1E1,2.1E1,1.11E2,2.86E2,1.09E2,8E0,6.3E1,3.07E2,2.9E1,4.09E2,6.5E1,2E0,1.1E1,6.4E1,2E1,7.5E1,6.59E2,1.1E2,1.6E1,8E0,4.9E1,2.18E2,3E0,2E0,1.9E1,6.2E1,4.9E1,1.85E2,1.01E2,7.3E1,3.6E1,5E0,5.8E1,1.7E1,2.9E2,1E1,1.9E1,2.67E2,1.42E2,3.6E1,2.9E1,8E0,3E0,5.3E1,1.1E1,1.2E1,8E0,5.1E1,2.4E1,6.52E2,7E0,9.8E1,1.2E1,9E0,7E0,2E0,6E0,4.6E1,3E0,1.28E2,9E1,7E0,1.2E1,5.3E1,9E0,8E0,4.1E1,8.8E1,9.7E1,3.4E1,6.7E1,2.2E1,5.1E1,9E0,2.7E1,2E0,5.6E1,1.3E1,4E0,4.5E1,2.45E2,1.2E1,7E0,1.77E2,9E1,2.1E1,1.21E2,2.3E1,1.3E1,6E0,2.3E1,3E1,2.3E1,3E0,5E0,4.4E1,7E0,3.81E2,2.71E2,5E0,2E0,9.2E1,6E0,7E0,5E0,3E0,6E0,4E0,2E0,6.3E1,6.5E1,2.9E1,6.1E1,7E0,4.6E1,4E0,5E0,2E1,2.1E1,6.4E1,2.4E1,6.6E1,3.1E1,9E0,1.3E1,2.5E1,2.6E1,1.1E1,1.6E1,4E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"155","size_leaf_vector":"1"}},{"base_weights":[2.430006E-3,2.2013843E-2,-6.165792E-2,-1.843684E-2,5.102397E-2,-1.5407607E-1,-3.4091175E-2,-3.291846E-2,4.70343E-2,5.74579E-3,8.36082E-2,-1.2179068E-1,-3.028829E-1,-1.3823505E-1,-1.906325E-2,-5.1987424E-2,3.1201081E-2,3.613268E-2,1.767337E-2,1.3111398E-2,-1.3770073E-3,7.613562E-2,2.268644E-1,-1.8837102E-1,-4.233509E-2,-4.034121E-1,-1.8596771E-3,-1.6830054E-1,7.921827E-3,-2.6918573E-2,6.036869E-3,-2.1479659E-2,-9.2484966E-2,9.497761E-2,-4.0435057E-2,7.694717E-2,-3.520908E-2,4.598737E-3,-2.886726E-1,6.544445E-2,2.0314649E-1,1.6793769E-2,4.666612E-2,1.1328447E-3,-2.1033391E-1,1.5810223E-1,-8.9345425E-2,-6.2022954E-3,-4.4955856E-1,-1.3184215E-1,-1.4628779E-2,-1.2782974E-2,-1.004445E-1,-9.7438216E-2,-3.3090517E-3,-1.4782019E-1,-2.817866E-2,1.11501314E-1,-1.3008677E-2,5.073272E-3,-1.2989652E-1,4.8806034E-2,8.756796E-3,-1.08773865E-1,3.9953263E-3,-1.58949E-2,9.026998E-2,-7.5401925E-3,-2.1366378E-2,7.489477E-2,-1.8247152E-2,1.1399026E-1,2.6685154E-1,-2.9221678E-2,1.2579071E-2,-1.3719773E-1,-2.7196664E-1,1.4517191E-2,-3.666547E-3,2.1300158E-3,-1.3788493E-1,-2.4576794E-2,-1.0519857E-2,-3.5506224E-3,-9.434639E-3,-6.7062594E-2,4.4514327E-3,-1.6625528E-1,-3.1589944E-2,-3.4413224E-3,-1.745361E-2,5.5479766E-3,-1.5632209E-3,1.1714168E-3,-8.756439E-3,2.4712048E-3,-2.9694063E-3,2.4527514E-3,8.382178E-3,-1.0500026E-2,1.114982E-3,-1.1319804E-3,-1.0501816E-2,4.5393975E-3,-1.0005847E-4,-1.6248632E-2,-1.4353594E-4,-2.4530757E-4,-5.813163E-3,1.4523773E-2,3.3239075E-3,2.0644653E-3,5.085096E-3,-2.1986805E-3,3.9200336E-3,-2.0021747E-3,7.6428503E-3,1.4582985E-3,1.3564273E-2,3.8757117E-3,-9.559799E-3,-1.7313039E-2,-3.6110622E-3,-7.91577E-3,-1.794636E-2,-1.3317644E-2,-4.6889824E-3,-1.5758965E-2,-2.1922418E-3,-1.9221816E-3,1.7531908E-3,-7.546665E-4,-9.535746E-3,3.8925186E-3,-6.8292455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,49,-1,51,-1,53,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,79,-1,81,83,-1,85,87,89,91,93,95,97,-1,99,101,103,-1,105,-1,107,109,-1,-1,111,113,115,117,119,-1,121,123,-1,-1,-1,125,-1,-1,-1,-1,127,129,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3675022E0,2.412441E0,1.5946827E0,8.1599045E-1,1.7655191E0,6.549289E-1,7.5642186E-1,8.62398E-1,5.342803E-1,9.587665E-1,7.2779083E-1,6.268468E-1,6.4795566E-1,5.8473253E-1,4.8787424E-1,6.688844E-1,7.4397063E-1,4.452995E-1,0E0,0E0,8.4256196E-1,8.869896E-1,7.4215996E-1,3.076613E-1,5.417927E-1,1.6929102E-1,0E0,2.3976731E-1,0E0,4.1860655E-1,0E0,4.289238E-1,8.249426E-1,5.5125594E-1,3.1414467E-1,2.8015834E-1,4.9780795E-1,8.459339E-1,1.3640732E-1,4.861958E-1,2.592466E-1,0E0,2.5354832E-1,0E0,2.2863412E-1,3.9757478E-1,3.0493316E-1,0E0,1.0668731E-1,1.5835255E-1,0E0,3.183607E-1,2.8928715E-1,3.8362014E-1,8.845579E-1,7.586863E-1,2.973447E-1,3.0890357E-1,0E0,2.175051E-1,2.4178302E-1,1.8273477E-1,0E0,8.523849E-1,0E0,4.4143808E-1,3.9223915E-1,0E0,0E0,5.433972E-1,1.7821369E-1,1.7007157E-1,1.03693485E-1,2.4138471E-1,0E0,3.747279E-1,2.779231E-1,0E0,0E0,0E0,1.6332322E-1,0E0,0E0,0E0,0E0,4.4225517E-1,3.7282962E-1,1.6005915E-1,4.2507306E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,63,63,65,65,66,66,69,69,70,70,71,71,72,72,73,73,75,75,76,76,80,80,85,85,86,86,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,50,-1,52,-1,54,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,80,-1,82,84,-1,86,88,90,92,94,96,98,-1,100,102,104,-1,106,-1,108,110,-1,-1,112,114,116,118,120,-1,122,124,-1,-1,-1,126,-1,-1,-1,-1,128,130,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,3.3030225E2,1.9121015E0,1.5859042E6,1.5166431E5,1.1034263E10,1.4123257E-4,4.264897E6,1.819149E1,8E0,5.528107E3,1.4268628E2,9.099893E8,3.1998687E3,9.609747E3,2.915723E0,2.46E2,8E0,1.767337E-2,1.3111398E-2,1.2427474E8,5.0083565E6,3.526947E7,1.3177E4,2.6011018E8,4.987639E7,-1.8596771E-3,1E0,7.921827E-3,7.218466E7,6.036869E-3,3.75E1,1.363E3,1E0,1.6E1,6.789622E7,1.63E3,1.4777102E9,1.6213043E3,2.7629982E7,7E0,1.6793769E-2,1.9013363E4,1.1328447E-3,1.16146E5,2.5927516E11,1.4195741E0,-6.2022954E-3,5.3842196E9,1.2283395E-2,-1.4628779E-2,1.513E3,1.2204E4,3.09627E5,3.9E2,4.09E2,5.07E2,7.59E2,-1.3008677E-2,6.061028E4,1.1383217E5,1.7463945E4,8.756796E-3,1.8122449E2,3.9953263E-3,7.806529E1,8.538E3,-7.5401925E-3,-2.1366378E-2,6.9664386E2,8.36E2,1.86E2,5.76E3,2.6284674E2,1.2579071E-2,5.23E2,2.4968571E2,1.4517191E-2,-3.666547E-3,2.1300158E-3,3.6482175E5,-2.4576794E-2,-1.0519857E-2,-3.5506224E-3,-9.434639E-3,4.2378342E-1,3.9134903E5,6.258335E5,4.1975945E1,-3.4413224E-3,-1.745361E-2,5.5479766E-3,-1.5632209E-3,1.1714168E-3,-8.756439E-3,2.4712048E-3,-2.9694063E-3,2.4527514E-3,8.382178E-3,-1.0500026E-2,1.114982E-3,-1.1319804E-3,-1.0501816E-2,4.5393975E-3,-1.0005847E-4,-1.6248632E-2,-1.4353594E-4,-2.4530757E-4,-5.813163E-3,1.4523773E-2,3.3239075E-3,2.0644653E-3,5.085096E-3,-2.1986805E-3,3.9200336E-3,-2.0021747E-3,7.6428503E-3,1.4582985E-3,1.3564273E-2,3.8757117E-3,-9.559799E-3,-1.7313039E-2,-3.6110622E-3,-7.91577E-3,-1.794636E-2,-1.3317644E-2,-4.6889824E-3,-1.5758965E-2,-2.1922418E-3,-1.9221816E-3,1.7531908E-3,-7.546665E-4,-9.535746E-3,3.8925186E-3,-6.8292455E-3],"split_indices":[27,52,53,28,28,5,42,9,56,18,52,52,7,4,52,56,10,8,0,0,45,28,50,9,7,45,0,101,0,45,0,4,2,74,3,7,10,7,4,45,3,0,52,0,1,31,53,0,12,57,0,2,2,9,12,29,11,2,0,28,47,50,0,52,0,53,2,0,0,52,8,0,12,58,0,29,4,0,0,0,28,0,0,0,0,27,28,32,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.681E3,2.054E3,6.27E2,8.58E2,1.196E3,1.43E2,4.84E2,7.03E2,1.55E2,5.01E2,6.95E2,1.19E2,2.4E1,6E1,4.24E2,5.42E2,1.61E2,1.51E2,4E0,1.2E1,4.89E2,6.62E2,3.3E1,6.4E1,5.5E1,1.7E1,7E0,5.5E1,5E0,4.03E2,2.1E1,3.1E2,2.32E2,8.5E1,7.6E1,9.6E1,5.5E1,4.8E2,9E0,6.12E2,5E1,1.9E1,1.4E1,6E0,5.8E1,1E1,4.5E1,3E0,1.4E1,4.5E1,1E1,3.39E2,6.4E1,5.9E1,2.51E2,1.24E2,1.08E2,8.2E1,3E0,5.1E1,2.5E1,7.7E1,1.9E1,3.4E1,2.1E1,3.88E2,9.2E1,6E0,3E0,5.5E2,6.2E1,2.2E1,2.8E1,1.1E1,3E0,2.8E1,3E1,6E0,4E0,1.2E1,3.3E1,1E1,4E0,2.5E1,2E1,8.1E1,2.58E2,3.2E1,3.2E1,5.5E1,4E0,4.9E1,2.02E2,2.1E1,1.03E2,3.2E1,7.6E1,4.3E1,3.9E1,3E0,4.8E1,1.2E1,1.3E1,4E1,3.7E1,1E1,2.4E1,3.53E2,3.5E1,7E0,8.5E1,2.74E2,2.76E2,4.9E1,1.3E1,5E0,1.7E1,2E0,2.6E1,7E0,4E0,5E0,2.3E1,1.6E1,1.4E1,6E0,2.7E1,5E0,7.6E1,1.08E2,1.5E2,6E0,2.6E1,1.6E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-2.8165912E-3,-3.134089E-2,4.0566783E-2,-1.644277E-2,-1.0840408E-1,-1.5399587E-2,6.9172494E-2,-2.97465E-2,4.663593E-2,-2.058015E-1,-7.214112E-2,-9.471575E-3,-2.5748866E-2,1.1089835E-2,9.146348E-2,-4.7077443E-2,2.8751638E-2,1.2124591E-1,-8.567262E-3,-1.4605333E-1,-3.834839E-1,1.5391331E-2,-9.6917994E-2,6.146762E-2,-4.371939E-2,-1.7754085E-2,2.1026853E-2,2.6861243E-2,1.0980594E-1,-5.8710698E-2,4.8222035E-2,6.253108E-2,-7.970315E-2,2.3208415E-2,1.5718843E-1,-7.647571E-2,2.9342748E-2,-1.3010666E-2,-9.5293835E-2,-4.463491E-1,-3.2366607E-3,5.2318115E-2,-6.222081E-3,-8.330702E-2,-2.453677E-1,8.23365E-2,-1.10211715E-1,-8.841E-2,1.7079742E-3,1.2479577E-1,-7.2880653E-3,1.4538383E-1,-2.3846624E-2,1.1438519E-1,-1.2854796E-2,-4.4655234E-2,-1.2179069E-1,2.7690897E-2,8.917724E-3,4.7147576E-2,1.6398251E-2,-4.96436E-2,-1.9464694E-2,-6.285667E-3,4.699354E-2,2.1169415E-1,6.202944E-2,-1.3899727E-1,1.3505295E-2,7.4186022E-3,1.3077052E-2,1.2543705E-2,-2.1514289E-1,-2.4759423E-2,-1.0333674E-2,-9.0728066E-4,5.4272506E-3,-1.5640152E-1,-5.1876597E-2,-1.5811814E-2,3.2420421E-4,1.0035618E-1,-1.4975682E-3,-8.164625E-3,3.2577426E-3,-7.626899E-2,-1.3181363E-2,5.542723E-2,-8.002565E-2,8.648715E-2,1.598817E-2,9.169883E-2,-2.958117E-2,2.7000636E-1,3.278678E-2,-4.5948718E-2,1.517529E-2,1.4945926E-1,8.2174994E-2,-1.6612294E-3,-7.2858394E-3,-9.670467E-3,-3.3549904E-3,-3.8810528E-3,2.1755775E-3,3.2062686E-4,4.276851E-3,-8.712922E-3,-8.441064E-4,3.3039635E-3,-6.5758387E-3,1.1058788E-2,3.771312E-4,-2.9250695E-3,6.4085117E-3,2.9682626E-3,-7.881385E-3,4.7932426E-3,-3.5584883E-3,3.0525445E-3,-1.5838477E-3,-2.7941186E-3,6.8909656E-3,-5.215821E-3,-1.3529135E-2,-8.940751E-3,-3.888412E-4,3.643264E-3,-3.3817843E-3,2.0058337E-3,6.243447E-3,-2.3550924E-3,-8.74005E-3,3.385721E-3,-7.968129E-3,-7.509882E-4,-7.7586165E-3,7.2912155E-3,1.9321596E-4,8.458297E-3,-9.515826E-4,-3.9939806E-3,6.2861905E-4,1.4442653E-2,1.5649493E-3,-8.676684E-4,6.9967853E-3,-1.6346594E-2,-1.187579E-3,1.7703604E-2,6.1230226E-3,2.1020526E-3,8.330972E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,-1,77,79,81,83,85,87,89,91,93,95,97,-1,99,101,103,-1,105,-1,107,-1,-1,109,111,113,115,117,-1,119,121,123,-1,-1,-1,-1,125,127,-1,-1,129,-1,-1,-1,131,-1,133,135,137,-1,139,141,143,145,147,-1,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2942064E0,1.8420608E0,1.6916767E0,1.1317654E0,9.0345216E-1,1.1032652E0,9.039948E-1,1.129744E0,9.689353E-1,6.8712497E-1,4.157288E-1,8.651642E-1,0E0,7.4763924E-1,5.954571E-1,9.551556E-1,9.3918717E-1,3.487234E-1,3.5236374E-1,3.18442E-1,3.0743837E-1,2.3753488E-1,2.8062046E-1,4.2467415E-1,4.873934E-1,0E0,5.621073E-1,6.826001E-1,7.113757E-1,6.7371106E-1,2.6115143E-1,8.234354E-1,5.789405E-1,1.0903517E-1,3.6549854E-1,2.7753904E-1,1.7770526E-1,0E0,5.196678E-1,1.1650586E-1,0E0,1.5431353E-1,0E0,3.089264E-1,2.5880408E-1,2.1604323E-1,1.5125522E-1,2.5347167E-1,5.31316E-1,2.994753E-1,3.359281E-1,4.6823967E-1,6.1701196E-1,4.271884E-1,0E0,6.67482E-1,5.621419E-1,1.637542E-1,0E0,3.166E-1,0E0,2.3822577E-1,0E0,0E0,1.10823065E-1,1.7841864E-1,2.5562197E-1,1.5925163E-1,1.6709521E-1,0E0,1.8681486E-1,2.1323298E-1,1.0518849E-1,0E0,0E0,0E0,0E0,1.8263352E-1,2.4033573E-1,0E0,0E0,1.5248334E-1,0E0,0E0,0E0,3.2200336E-1,0E0,2.5901332E-1,2.5066203E-1,1.945819E-1,0E0,2.687884E-1,2.8609362E-1,1.1002469E-1,1.1224151E-1,4.6635655E-1,0E0,8.3288336E-1,7.126676E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,59,59,61,61,64,64,65,65,66,66,67,67,68,68,70,70,71,71,72,72,77,77,78,78,81,81,85,85,87,87,88,88,89,89,91,91,92,92,93,93,94,94,95,95,97,97,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,-1,78,80,82,84,86,88,90,92,94,96,98,-1,100,102,104,-1,106,-1,108,-1,-1,110,112,114,116,118,-1,120,122,124,-1,-1,-1,-1,126,128,-1,-1,130,-1,-1,-1,132,-1,134,136,138,-1,140,142,144,146,148,-1,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5107028E-1,6.748543E-1,3.8172052E2,4.264897E6,1.9121015E0,1.0012501E7,1E0,2.092073E6,1.03E2,5.418831E11,1.4540612E7,1.5182724E0,-2.5748866E-2,1.3676985E4,1E0,2.3920168E6,5.7581736E7,7.68E2,1.4915413E3,7.33E2,8.289561E8,2.3432E4,1.5265896E8,5.59E2,2.6133334E2,-1.7754085E-2,8.3306855E6,1.3351997E0,5.68E2,6.72E2,1.862E3,8.791777E3,2.2576077E10,1.56E2,9.538462E0,2.5341389E8,3.4613644E1,-1.3010666E-2,3.3410483E8,1.3655606E6,-3.2366607E-3,7.105352E5,-6.222081E-3,9.3343524E2,6.773463E6,1.2453214E0,3.99704E5,1.2737473E1,3.401282E1,1.912391E6,4.5397964E-1,1.7E1,1.0980621E7,4.9716983E0,-1.2854796E-2,4.004366E1,6.81011E5,1.0203835E-5,8.917724E-3,8.027061E2,1.6398251E-2,1.2909952E1,-1.9464694E-2,-6.285667E-3,2.6120481E0,1.37E2,1.07E2,2.7487562E0,7.449623E2,7.4186022E-3,4.354E3,1E0,1.2252E4,-2.4759423E-2,-1.0333674E-2,-9.0728066E-4,5.4272506E-3,1.8344E4,6.239726E0,-1.5811814E-2,3.2420421E-4,1.5053125E2,-1.4975682E-3,-8.164625E-3,3.2577426E-3,2.6E1,-1.3181363E-2,1E0,2.0081E4,1.3E1,1.598817E-2,1.5851064E0,1.9007992E1,1.14E3,9.705292E6,1.22E2,1.517529E-2,1.4075E4,1.5435694E9,-1.6612294E-3,-7.2858394E-3,-9.670467E-3,-3.3549904E-3,-3.8810528E-3,2.1755775E-3,3.2062686E-4,4.276851E-3,-8.712922E-3,-8.441064E-4,3.3039635E-3,-6.5758387E-3,1.1058788E-2,3.771312E-4,-2.9250695E-3,6.4085117E-3,2.9682626E-3,-7.881385E-3,4.7932426E-3,-3.5584883E-3,3.0525445E-3,-1.5838477E-3,-2.7941186E-3,6.8909656E-3,-5.215821E-3,-1.3529135E-2,-8.940751E-3,-3.888412E-4,3.643264E-3,-3.3817843E-3,2.0058337E-3,6.243447E-3,-2.3550924E-3,-8.74005E-3,3.385721E-3,-7.968129E-3,-7.509882E-4,-7.7586165E-3,7.2912155E-3,1.9321596E-4,8.458297E-3,-9.515826E-4,-3.9939806E-3,6.2861905E-4,1.4442653E-2,1.5649493E-3,-8.676684E-4,6.9967853E-3,-1.6346594E-2,-1.187579E-3,1.7703604E-2,6.1230226E-3,2.1020526E-3,8.330972E-3],"split_indices":[38,27,52,9,53,45,102,1,11,31,45,53,0,33,100,28,45,2,55,2,7,9,45,0,55,0,45,41,3,0,2,52,12,2,58,7,56,0,7,32,0,28,0,4,1,38,1,54,58,1,41,8,1,54,0,53,1,42,0,52,0,56,0,0,53,3,10,54,4,0,10,16,9,0,0,0,0,9,56,0,0,52,0,0,0,8,0,74,29,8,0,53,56,10,9,10,0,9,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E3,1.605E3,1.055E3,1.346E3,2.59E2,3.57E2,6.98E2,1.112E3,2.34E2,6.9E1,1.9E2,3.54E2,3E0,1.94E2,5.04E2,8.58E2,2.54E2,9.9E1,1.35E2,5.3E1,1.6E1,4.2E1,1.48E2,1.15E2,2.39E2,4E0,1.9E2,1.12E2,3.92E2,7.65E2,9.3E1,1.94E2,6E1,2.7E1,7.2E1,4.8E1,8.7E1,1.4E1,3.9E1,1.3E1,3E0,3.4E1,8E0,1.37E2,1.1E1,1.03E2,1.2E1,1.2E2,1.19E2,4E1,1.5E2,3.3E1,7.9E1,3.88E2,4E0,6.27E2,1.38E2,8.2E1,1.1E1,1.85E2,9E0,5.6E1,4E0,3E0,2.4E1,4.5E1,2.7E1,2.8E1,2E1,9E0,7.8E1,2.1E1,1.8E1,9E0,4E0,1.6E1,1.8E1,4E1,9.7E1,8E0,3E0,8.9E1,1.4E1,9E0,3E0,1.14E2,6E0,7.2E1,4.7E1,3.5E1,5E0,2.7E1,1.23E2,1.5E1,1.8E1,7.5E1,4E0,1.84E2,2.04E2,5.75E2,5.2E1,5.3E1,8.5E1,1.1E1,7.1E1,9.5E1,9E1,1E1,4.6E1,2.2E1,2E0,4.1E1,4E0,1E1,1.7E1,3E0,2.5E1,1E1,1E1,3.7E1,4.1E1,1.4E1,7E0,8E0,1E1,3.3E1,7E0,1.2E1,8.5E1,3.1E1,5.8E1,9.2E1,2.2E1,6.8E1,4E0,2.7E1,2E1,1.9E1,1.6E1,1.5E1,1.2E1,5.4E1,6.9E1,1.3E1,2E0,1.3E1,5E0,4E0,7.1E1,1.5E1,1.69E2,1.45E2,5.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-2.501018E-4,1.9820053E-2,-5.7229545E-2,-2.5346614E-2,4.3810464E-2,-9.307421E-2,-2.211632E-4,-3.7346676E-2,9.7191855E-2,4.487065E-3,7.649069E-2,-8.2929105E-2,-3.0869815E-1,-3.5351593E-2,4.732796E-2,-6.8989885E-3,-7.623325E-2,1.1382237E-1,-1.2825838E-2,1.2981693E-2,-1.026676E-3,2.3598461E-1,6.713445E-2,-7.5338006E-2,-2.938941E-1,-1.8265557E-1,-2.388193E-2,-3.7684245E-3,-1.13874845E-1,2.8438682E-2,8.760066E-3,-9.982716E-2,1.4273972E-2,2.8881935E-2,-9.51427E-2,1.4953046E-1,2.8898044E-2,-1.9263823E-2,8.6886644E-2,2.785558E-1,3.0233264E-3,5.3398646E-2,1.354593E-1,-8.940438E-2,1.4522103E-2,-1.9726804E-2,4.4317087E-5,-1.2247542E-2,-2.622803E-3,-3.804913E-2,6.4892165E-2,1.676993E-3,-1.3767919E-1,7.2570935E-2,-2.782456E-2,-7.711848E-2,-1.5040548E-2,9.0018585E-2,-2.5457624E-2,-1.3062933E-1,7.494754E-2,-3.172035E-1,-8.177081E-2,3.0314142E-3,1.0132045E-2,3.9806557E-3,-3.6362307E-3,-2.383249E-2,9.518746E-3,1.0881697E-1,-1.00442074E-1,1.6182737E-2,1.5641755E-1,9.495584E-2,2.9816676E-2,1.8444574E-1,-3.3666797E-2,-6.1011225E-2,-1.4976597E-1,1.2868324E-1,-1.1435003E-3,-9.3914114E-2,-3.5046819E-3,9.305247E-3,1.886724E-3,-1.6268125E-2,-5.6309E-3,9.036303E-3,4.138032E-2,-5.931944E-2,3.7248824E-3,-7.378593E-3,-1.4251366E-3,6.9578853E-3,-2.4091017E-3,-1.662466E-2,-7.1567355E-4,-1.1537439E-2,4.550747E-3,4.4671344E-3,-4.1114283E-3,1.8136132E-3,-1.8447675E-2,2.464835E-3,-5.281913E-3,-1.543254E-4,-3.4180225E-3,5.917607E-3,-7.979386E-3,9.88886E-4,-1.1966934E-2,1.18489135E-2,1.1581079E-3,6.09773E-3,-2.572462E-3,4.3953714E-4,4.2202207E-3,9.930993E-3,1.1444441E-3,6.717889E-3,-3.4322632E-3,-4.200115E-3,5.990488E-4,-2.8794536E-3,-9.308791E-3,8.682683E-3,-2.3078928E-3,-3.9942772E-4,-7.3669273E-3,2.9346566E-3,-1.9638774E-3,7.757997E-3,7.7692827E-4,-8.598211E-3,-8.887975E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,49,51,53,-1,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,-1,-1,-1,-1,81,83,-1,85,87,89,91,-1,93,95,97,99,101,103,-1,-1,-1,-1,105,-1,107,109,-1,111,113,115,117,119,121,123,125,-1,127,129,-1,-1,-1,-1,-1,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0819554E0,2.1606593E0,1.4344788E0,1.0211945E0,1.672854E0,9.172244E-1,4.560277E-1,7.465822E-1,4.0683973E-1,8.7044126E-1,1.045157E0,6.3926005E-1,3.63878E-1,3.8893294E-1,2.918982E-1,6.999104E-1,5.5333245E-1,1.7762798E-1,0E0,0E0,9.3318415E-1,2.6940155E-1,6.2437487E-1,5.06901E-1,4.87507E-1,1.0750151E-1,0E0,2.68441E-1,1.6431361E-1,2.5777066E-1,0E0,2.9686803E-1,8.7503344E-1,3.2575366E-1,6.7186475E-1,2.1128666E-1,1.14304446E-1,4.8484772E-1,4.2174768E-1,1.7188048E-1,0E0,5.4792905E-1,9.341166E-1,5.838568E-1,2.441777E-1,0E0,0E0,0E0,0E0,1.463376E-1,1.17721885E-1,0E0,1.1842364E-1,2.0225179E-1,1.5760863E-1,2.1773213E-1,0E0,7.8285503E-1,6.407843E-1,2.7175054E-1,1.0822317E-1,3.2847977E-1,8.530878E-1,0E0,0E0,0E0,0E0,4.6320668E-1,0E0,3.861848E-1,2.0421249E-1,0E0,1.3841617E-1,9.8379123E-1,4.32398E-1,3.152249E-1,1.8191847E-1,4.6617013E-1,4.281907E-1,1.3903548E-1,0E0,1.4740264E-1,1.1913163E-1,0E0,0E0,0E0,0E0,0E0,1.3998526E-1,1.7307523E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,49,49,50,50,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,67,67,69,69,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,82,82,88,88,89,89],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,50,52,54,-1,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,-1,-1,-1,-1,82,84,-1,86,88,90,92,-1,94,96,98,100,102,104,-1,-1,-1,-1,106,-1,108,110,-1,112,114,116,118,120,122,124,126,-1,128,130,-1,-1,-1,-1,-1,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,2.6659616E2,2.7947538E6,1.3195207E7,2.3770695E5,1.5265896E8,6.726E3,2.5074627E0,1.0430133E7,8E0,1.0918E4,4.5723195E6,2.3580047E10,2.99665E5,1.4813511E6,3.75E1,1.1376811E0,2.8807552E7,-1.2825838E-2,1.2981693E-2,1.5435694E9,1E0,1.3664E4,3.0387878E3,3.0897958E0,2.867292E3,-2.388193E-2,3.222E3,1.0134536E7,1.29E3,8.760066E-3,3.09627E5,3.5508975E3,1.9E3,1.198E5,1.4462532E7,1.216E3,1.5137865E2,2.6572757E10,3.4246575E-3,3.0233264E-3,1.4811083E0,2.6364462E7,2.8035452E11,2.8277853E11,-1.9726804E-2,4.4317087E-5,-1.2247542E-2,-2.622803E-3,4.593E3,1.7270016E5,1.676993E-3,1.0595947E7,6.504065E-2,3.8248518E-1,8.333333E0,-1.5040548E-2,8E0,6.6861836E-4,1.8836284E2,2.0316172E6,1.5E1,3.4955364E7,3.0314142E-3,1.0132045E-2,3.9806557E-3,-3.6362307E-3,2.2075728E7,9.518746E-3,3.7826266E4,1.3674345E4,1.6182737E-2,9.532E3,3.1984328E5,1.8137958E6,2.1146836E7,1.2819376E2,2.011E3,1E1,1.0238709E1,-1.1435003E-3,4.0778715E6,1.9987492E5,9.305247E-3,1.886724E-3,-1.6268125E-2,-5.6309E-3,9.036303E-3,7.576E3,4.214286E0,3.7248824E-3,-7.378593E-3,-1.4251366E-3,6.9578853E-3,-2.4091017E-3,-1.662466E-2,-7.1567355E-4,-1.1537439E-2,4.550747E-3,4.4671344E-3,-4.1114283E-3,1.8136132E-3,-1.8447675E-2,2.464835E-3,-5.281913E-3,-1.543254E-4,-3.4180225E-3,5.917607E-3,-7.979386E-3,9.88886E-4,-1.1966934E-2,1.18489135E-2,1.1581079E-3,6.09773E-3,-2.572462E-3,4.3953714E-4,4.2202207E-3,9.930993E-3,1.1444441E-3,6.717889E-3,-3.4322632E-3,-4.200115E-3,5.990488E-4,-2.8794536E-3,-9.308791E-3,8.682683E-3,-2.3078928E-3,-3.9942772E-4,-7.3669273E-3,2.9346566E-3,-1.9638774E-3,7.757997E-3,7.7692827E-4,-8.598211E-3,-8.887975E-4],"split_indices":[27,52,32,9,28,45,2,56,45,18,9,28,5,29,28,4,53,9,0,0,7,6,2,4,56,4,0,2,45,0,0,9,50,12,7,5,10,56,12,57,0,53,47,31,31,0,0,0,0,11,47,0,45,57,34,52,0,17,41,52,47,0,7,0,0,0,0,45,0,52,4,0,9,33,28,9,58,2,3,53,0,32,47,0,0,0,0,0,9,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.693E3,1.992E3,7.01E2,6.91E2,1.301E3,4.3E2,2.71E2,6.3E2,6.1E1,5.91E2,7.1E2,4.12E2,1.8E1,1.56E2,1.15E2,3.54E2,2.76E2,5.9E1,2E0,1.1E1,5.8E2,3.8E1,6.72E2,3.99E2,1.3E1,1.2E1,6E0,1.12E2,4.4E1,1.02E2,1.3E1,6.5E1,2.89E2,4.2E1,2.34E2,4.1E1,1.8E1,4.81E2,9.9E1,3E1,8E0,5.61E2,1.11E2,3.45E2,5.4E1,9E0,4E0,7E0,5E0,7.5E1,3.7E1,6E0,3.8E1,5.7E1,4.5E1,6E1,5E0,9.9E1,1.9E2,9E0,3.3E1,1.2E1,2.22E2,1.8E1,2.3E1,1.2E1,6E0,4.72E2,9E0,8.9E1,1E1,1.9E1,1.1E1,2.02E2,3.59E2,8.6E1,2.5E1,2.36E2,1.09E2,1.3E1,4.1E1,2.8E1,4.7E1,5E0,3.2E1,2E0,3.6E1,1.1E1,4.6E1,3.5E1,1E1,2.2E1,3.8E1,7.1E1,2.8E1,5E0,1.85E2,6E0,3E0,3E1,3E0,2E0,1E1,3.9E1,1.83E2,3.3E2,1.42E2,8.5E1,4E0,6E0,4E0,6E0,5E0,1.66E2,3.6E1,2.66E2,9.3E1,7.5E1,1.1E1,4E0,2.1E1,1.73E2,6.3E1,3.7E1,7.2E1,1E1,3E0,1.2E1,1.6E1,1.7E1,3E1,7E0,3.9E1,8E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[9.7434997E-4,1.9878583E-2,-5.2678537E-2,2.8047527E-3,7.50732E-2,-6.991756E-2,5.993552E-2,-2.244531E-2,4.1533615E-2,8.164391E-2,-1.3027421E-2,-5.064338E-2,-1.4520691E-1,1.03148736E-1,-6.461553E-3,-4.289559E-2,1.806712E-2,5.3273868E-2,-8.9379095E-2,5.5105793E-3,1.1254196E-1,-4.3809943E-2,-2.6320383E-1,-2.3215084E-1,-8.3323844E-2,1.7148723E-1,4.825144E-2,-4.999566E-3,1.3185056E-3,-5.4477833E-2,1.323498E-2,-3.397296E-2,4.711923E-2,3.5841953E-2,1.0928806E-1,-2.1031407E-1,-3.6506772E-2,1.018185E-2,-1.2757965E-2,2.3564896E-1,9.426156E-2,-3.2955095E-2,-1.4914241E-1,-1.9007558E-2,-2.5723656E-3,-1.8752815E-1,-4.1955367E-1,-6.390718E-2,-1.621058E-2,1.9199818E-1,-2.055747E-3,9.6032483E-4,9.681312E-3,-1.5141088E-2,-7.625876E-2,5.9081227E-2,-8.145543E-2,-9.857383E-2,1.2545432E-2,7.924086E-2,-2.2377376E-2,4.359701E-2,-1.2299005E-1,1.1806432E-1,-1.4583651E-2,-4.8307846E-3,-1.7128825E-2,-9.88281E-2,3.8961926E-3,-6.1212685E-3,1.8750887E-2,9.745947E-2,3.0881128E-1,1.5213711E-2,8.4278665E-2,-7.990434E-2,2.3427417E-3,-1.4108649E-2,-1.0332131E-1,-2.6339105E-1,-1.4182222E-1,-2.3137957E-2,-6.289382E-3,-1.03638664E-1,5.6630354E-2,-9.254465E-4,1.0202807E-2,-2.4572844E-3,4.487157E-3,-3.36846E-3,-1.1519064E-2,3.883884E-3,-2.766488E-3,-6.713301E-3,-1.4560123E-3,2.7544512E-3,-5.881868E-3,7.4717607E-3,-6.151634E-4,5.13813E-3,1.6017283E-4,3.7629586E-3,-3.52027E-3,8.099108E-3,1.7214454E-3,-1.0601172E-3,-2.4944799E-2,8.556164E-3,3.0066827E-3,-9.5239293E-4,-9.114258E-3,1.9974964E-4,7.8582E-3,-4.34095E-5,1.0730326E-2,1.7317807E-2,3.052491E-3,5.22225E-3,1.3273908E-3,-3.2942193E-3,-1.6895313E-2,1.4604506E-3,-1.9526059E-3,-1.0853752E-2,-2.1517074E-3,-1.6212106E-2,-6.185731E-3,3.8440393E-3,-7.7221645E-3,-1.4253247E-3,-7.240574E-3,1.0444362E-2,4.6930666E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,-1,79,81,83,-1,85,-1,-1,-1,87,89,91,93,95,97,99,101,103,105,107,-1,-1,-1,109,-1,-1,111,113,115,-1,117,119,121,-1,123,125,127,-1,-1,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7405813E0,1.8826716E0,1.3709054E0,1.4941949E0,1.098691E0,8.792932E-1,2.703742E-1,7.6712865E-1,9.3154156E-1,1.0908799E0,0E0,6.9729817E-1,6.478207E-1,2.0556659E-1,1.2937982E-1,4.0054345E-1,4.7179368E-1,5.3702784E-1,3.1202775E-1,5.13464E-1,7.2043467E-1,5.370413E-1,3.9878953E-1,3.4780526E-1,3.450024E-1,1.1818725E-1,1.3916361E-1,0E0,0E0,4.358089E-1,4.6497968E-1,3.3781105E-1,4.4824204E-1,5.276244E-1,5.1170444E-1,2.0582086E-1,2.7247387E-1,0E0,4.5557255E-1,3.9453435E-1,6.177921E-1,7.144433E-1,2.6282674E-1,0E0,0E0,1.11742735E-1,1.0150123E-1,3.4110856E-1,0E0,1.0640985E-1,0E0,0E0,0E0,7.2519994E-1,3.021692E-1,1.8992347E-1,1.0066961E-1,1.8493143E-1,2.4243191E-1,2.9128045E-1,3.3611637E-1,3.828944E-1,7.9102623E-1,4.2091954E-1,0E0,0E0,0E0,1.6954419E-1,0E0,0E0,2.0909514E-1,1.9945297E-1,3.2633352E-1,0E0,3.864143E-1,5.3700817E-1,3.0105463E-1,0E0,2.4183768E-1,1.07548475E-1,1.3804442E-1,0E0,0E0,1.8168569E-1,1.339644E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,67,67,70,70,71,71,72,72,74,74,75,75,76,76,78,78,79,79,80,80,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,-1,80,82,84,-1,86,-1,-1,-1,88,90,92,94,96,98,100,102,104,106,108,-1,-1,-1,110,-1,-1,112,114,116,-1,118,120,122,-1,124,126,128,-1,-1,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.0989723E3,3.4409692E3,4.7519747E5,1.2367184E3,6.747114E7,1.663473E6,1.5107028E-1,2E0,7.805608E4,-1.3027421E-2,4.391553E6,5.35E3,3.3517068E7,5.528107E3,2.777588E6,3.8833074E4,6.072483E2,3.4446188E2,1.2E1,1.5552E4,6.5E1,3.0897958E0,1.97729E5,1.2502964E9,1.2982227E1,6.707523E8,-4.999566E-3,1.3185056E-3,3.8E1,7.8E1,6.061028E4,2.0687214E6,4.5158855E6,1.5449402E7,5.69815E5,1.038946E6,1.018185E-2,6.015E3,2.0601392E3,6.6666694E8,1.646E3,1.7446084E0,-1.9007558E-2,-2.5723656E-3,3.1665432E0,3.285E3,1.2204E4,-1.621058E-2,8.838E3,-2.055747E-3,9.6032483E-4,9.681312E-3,7.55552E5,1E0,2.116711E3,1.27E2,1.5555556E0,3.7129E4,1.8777761E0,2.72258E5,1.8574098E8,8.190217E0,2E0,-1.4583651E-2,-4.8307846E-3,-1.7128825E-2,9E0,3.8961926E-3,-6.1212685E-3,9.492855E8,2.0225275E5,8.861076E0,1.5213711E-2,2.2807484E7,2.1103458E0,2.65851E5,-1.4108649E-2,2.264894E8,8.126489E7,9.090909E-2,-2.3137957E-2,-6.289382E-3,6.5928856E5,2.060792E8,-9.254465E-4,1.0202807E-2,-2.4572844E-3,4.487157E-3,-3.36846E-3,-1.1519064E-2,3.883884E-3,-2.766488E-3,-6.713301E-3,-1.4560123E-3,2.7544512E-3,-5.881868E-3,7.4717607E-3,-6.151634E-4,5.13813E-3,1.6017283E-4,3.7629586E-3,-3.52027E-3,8.099108E-3,1.7214454E-3,-1.0601172E-3,-2.4944799E-2,8.556164E-3,3.0066827E-3,-9.5239293E-4,-9.114258E-3,1.9974964E-4,7.8582E-3,-4.34095E-5,1.0730326E-2,1.7317807E-2,3.052491E-3,5.22225E-3,1.3273908E-3,-3.2942193E-3,-1.6895313E-2,1.4604506E-3,-1.9526059E-3,-1.0853752E-2,-2.1517074E-3,-1.6212106E-2,-6.185731E-3,3.8440393E-3,-7.7221645E-3,-1.4253247E-3,-7.240574E-3,1.0444362E-2,4.6930666E-4],"split_indices":[27,52,52,28,58,45,29,38,6,28,0,28,2,51,52,1,33,4,4,18,9,8,56,11,7,53,32,0,0,10,8,28,47,47,50,9,9,0,2,52,31,2,53,0,0,56,0,2,0,2,0,0,0,9,85,4,8,53,9,41,9,31,56,17,0,0,0,8,0,0,32,33,53,0,51,42,29,0,7,12,58,0,0,32,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E3,1.997E3,7.03E2,1.526E3,4.71E2,6.1E2,9.3E1,9.24E2,6.02E2,4.63E2,8E0,4.87E2,1.23E2,5.6E1,3.7E1,6.14E2,3.1E2,5.53E2,4.9E1,1.34E2,3.29E2,4.73E2,1.4E1,5E1,7.3E1,2.4E1,3.2E1,9E0,2.8E1,5.09E2,1.05E2,1.11E2,1.99E2,4.23E2,1.3E2,1.4E1,3.5E1,1E1,1.24E2,4.1E1,2.88E2,4.3E2,4.3E1,8E0,6E0,4.2E1,8E0,6.9E1,4E0,2.2E1,2E0,2.8E1,4E0,1.82E2,3.27E2,7.1E1,3.4E1,4.6E1,6.5E1,1.36E2,6.3E1,4.04E2,1.9E1,1.28E2,2E0,9E0,5E0,2.3E1,1.2E1,2.6E1,9.8E1,1.5E1,2.6E1,1.1E1,2.77E2,1.84E2,2.46E2,9E0,3.4E1,1.4E1,2.8E1,6E0,2E0,5.2E1,1.7E1,2E0,2E1,1.37E2,4.5E1,3.17E2,1E1,6E1,1.1E1,1.5E1,1.9E1,6E0,4E1,9E0,5.6E1,9.9E1,3.7E1,2.1E1,4.2E1,2.2E1,3.82E2,1.6E1,3E0,6E1,6.8E1,1.3E1,1E1,9E1,8E0,9E0,6E0,2.1E1,5E0,1.92E2,8.5E1,1.78E2,6E0,1.49E2,9.7E1,1E1,2.4E1,8E0,6E0,2E0,2.6E1,2.1E1,3.1E1,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-2.4875929E-3,-3.7208196E-2,2.570828E-2,-4.9318664E-2,3.9249163E-2,5.1845904E-2,-6.6702664E-2,-4.1313488E-2,-1.3417709E-1,4.8208714E-2,-1.2140937E-2,1.0500792E-1,3.1266578E-2,-1.14903934E-1,-6.7390045E-3,-3.220296E-2,-1.2905051E-1,1.4085165E-2,-1.673199E-1,1.0561627E-1,7.388422E-3,1.2936331E-1,-4.100771E-3,2.0665443E-2,1.9495882E-1,-9.162128E-2,-2.1899548E-1,9.591572E-3,-2.545779E-1,-4.1825395E-2,3.665212E-2,-1.9779624E-2,-9.941118E-2,4.7033476E-3,-3.4292045E-3,-2.3750818E-1,-6.682866E-2,1.4946978E-1,1.7130578E-2,-3.4164144E-3,3.2434516E-2,1.6598189E-1,3.3285856E-2,8.2438216E-2,-5.4094374E-2,3.2670046E-3,6.371778E-2,3.2829952E-1,1.22554384E-1,-1.0920754E-2,-7.566726E-2,-1.7512725E-1,-1.819377E-2,-1.3050267E-1,2.6598256E-2,-1.8924491E-2,-3.0208786E-3,-3.8205303E-2,-2.8089824E-1,5.1417876E-2,-1.6715597E-2,3.1808133E-3,-1.2641895E-1,-8.3898336E-2,-2.6583305E-1,-7.6505536E-4,-1.0106618E-2,2.3747282E-3,1.0082044E-2,5.4952065E-3,-2.5773598E-3,7.779994E-2,-4.6055995E-2,9.4766065E-2,2.0463225E-1,-5.5271015E-2,1.16305895E-1,8.139922E-3,4.197697E-4,-9.161473E-2,7.723944E-3,9.536678E-3,-5.9471065E-3,1.6408002E-1,3.3005256E-2,6.182641E-3,1.9182136E-2,7.835693E-2,1.7395195E-2,-5.6434277E-2,-1.6146702E-1,-7.355841E-2,-1.0623323E-2,-9.622743E-4,-1.4020816E-2,7.650289E-2,-1.7660333E-2,-1.7213266E-3,-1.736891E-2,-2.0595968E-2,4.197928E-3,6.4516338E-3,9.4029435E-4,-1.4073057E-2,-4.152573E-3,-7.979636E-3,5.244776E-3,-1.016112E-2,-1.8139193E-2,1.1444332E-3,6.744535E-3,-7.282183E-4,-1.0518803E-2,5.3402972E-3,-6.633595E-3,1.111334E-2,5.2501056E-3,-1.826757E-2,4.6177144E-4,7.7056135E-3,-3.401982E-3,-2.4766324E-3,-1.2573513E-2,7.2525337E-4,-5.2127736E-3,1.0919113E-2,3.523322E-3,7.230783E-3,9.14296E-4,5.554188E-3,-9.346604E-3,-4.007623E-3,1.7176705E-3,2.2844438E-4,-9.839814E-3,1.4546271E-3,-1.1352478E-2,5.762611E-3,-3.0861396E-4,8.731044E-3,-1.2814606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,-1,-1,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,-1,89,91,-1,93,95,-1,-1,97,99,101,-1,-1,103,105,107,-1,-1,-1,-1,-1,-1,109,111,113,115,117,119,-1,-1,121,-1,123,-1,125,127,-1,-1,129,-1,131,133,135,-1,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5952477E0,1.101946E0,3.537192E0,6.9142556E-1,4.3977526E-1,1.2450144E0,9.328004E-1,7.4641573E-1,4.347732E-1,3.718192E-1,0E0,8.455789E-1,1.4258776E0,4.1374707E-1,5.8930784E-1,5.6602865E-1,6.9960105E-1,1.2958038E-1,4.91966E-1,2.5354648E-1,1.8766196E-1,9.092121E-1,2.597168E-1,5.8030045E-1,4.4248843E-1,3.0484617E-1,1.5934813E-1,3.298789E-1,2.1736807E-1,6.3663614E-1,6.159216E-1,0E0,3.7117028E-1,0E0,0E0,1.6043806E-1,2.239656E-1,2.5787985E-1,1.6546384E-1,0E0,2.599575E-1,4.980812E-1,5.43124E-1,1.4038119E-1,3.2185566E-1,4.4306386E-1,6.819548E-1,1.9292998E-1,3.3320242E-1,0E0,2.1201259E-1,1.1174512E-1,0E0,2.6285177E-1,2.730222E-1,0E0,0E0,5.287143E-1,6.424293E-1,2.6740253E-1,0E0,0E0,4.3994355E-1,1.4624792E-1,1.3172793E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.52926E-1,1.412496E-1,2.8106564E-1,2.8982592E-1,7.6590455E-1,3.2889986E-1,0E0,0E0,2.1149057E-1,0E0,3.5214373E-1,0E0,2.840724E-1,2.7946675E-1,0E0,0E0,3.3133107E-1,0E0,2.8217524E-1,1.7651731E-1,1.7712149E-1,0E0,0E0,0E0,2.1161881E-1,1.2495278E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,53,53,54,54,57,57,58,58,59,59,62,62,63,63,64,64,71,71,72,72,73,73,74,74,75,75,76,76,79,79,81,81,83,83,84,84,87,87,89,89,90,90,91,91,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,-1,-1,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,-1,90,92,-1,94,96,-1,-1,98,100,102,-1,-1,104,106,108,-1,-1,-1,-1,-1,-1,110,112,114,116,118,120,-1,-1,122,-1,124,-1,126,128,-1,-1,130,-1,132,134,136,-1,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7311627E2,8.397659E6,4.0844156E7,2.1094006E7,1E0,8E0,3.0387878E3,1.062363E1,6.043677E8,2.53E2,-1.2140937E-2,1.0424884E-7,8.015419E3,1.2427474E8,2.6572757E10,2.5247778E6,2E0,3.910327E-3,7.85939E5,2.7309215E6,1.0617118E8,2.1298597E0,1.752E3,3.771981E6,4.0601485E6,4.30453E2,9.374544E5,2.2579439E0,2.862988E5,2.298425E6,9.697451E-1,-1.9779624E-2,7.29E2,4.7033476E-3,-3.4292045E-3,1.2068E4,1.8600049E10,7.59E2,1.1217151E0,-3.4164144E-3,4.0895187E8,5.314225E2,2.9251662E6,1.1214854E1,5.794E3,1.0046621E7,9.397667E6,1.9745072E7,8.724166E9,-1.0920754E-2,1.93974E7,8.7757526E2,-1.819377E-2,1.9E1,6.747114E7,-1.8924491E-2,-3.0208786E-3,4.7279915E6,1.8081865E0,8.339375E2,-1.6715597E-2,3.1808133E-3,2.5040618E7,3.3926086E2,5.59277E5,-7.6505536E-4,-1.0106618E-2,2.3747282E-3,1.0082044E-2,5.4952065E-3,-2.5773598E-3,3.624992E5,3.7898205E1,1.6259277E7,5.8180006E8,1.3E0,2.2579439E0,8.139922E-3,4.197697E-4,9.557794E2,7.723944E-3,7.806529E1,-5.9471065E-3,1.1860538E8,6.85945E4,6.182641E-3,1.9182136E-2,5.68E2,1.7395195E-2,1.9701008E5,5.5544252E7,2.894E3,-1.0623323E-2,-9.622743E-4,-1.4020816E-2,1.5609541E0,2.99665E5,-1.7213266E-3,-1.736891E-2,-2.0595968E-2,4.197928E-3,6.4516338E-3,9.4029435E-4,-1.4073057E-2,-4.152573E-3,-7.979636E-3,5.244776E-3,-1.016112E-2,-1.8139193E-2,1.1444332E-3,6.744535E-3,-7.282183E-4,-1.0518803E-2,5.3402972E-3,-6.633595E-3,1.111334E-2,5.2501056E-3,-1.826757E-2,4.6177144E-4,7.7056135E-3,-3.401982E-3,-2.4766324E-3,-1.2573513E-2,7.2525337E-4,-5.2127736E-3,1.0919113E-2,3.523322E-3,7.230783E-3,9.14296E-4,5.554188E-3,-9.346604E-3,-4.007623E-3,1.7176705E-3,2.2844438E-4,-9.839814E-3,1.4546271E-3,-1.1352478E-2,5.762611E-3,-3.0861396E-4,8.731044E-3,-1.2814606E-3],"split_indices":[52,9,45,45,74,17,4,54,5,10,0,37,4,45,12,28,17,38,1,45,7,38,29,1,47,33,28,54,28,28,27,0,29,0,0,9,5,2,53,0,7,4,28,56,2,47,45,1,7,0,50,52,0,3,45,0,0,47,53,33,0,0,5,52,1,0,0,0,0,0,0,47,58,9,7,54,54,0,0,52,0,53,0,32,32,0,0,3,0,33,45,0,0,0,0,57,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.649E3,1.187E3,1.462E3,1.025E3,1.62E2,1.14E3,3.22E2,9.38E2,8.7E1,1.58E2,4E0,3.17E2,8.23E2,1.78E2,1.44E2,8.51E2,8.7E1,1.6E1,7.1E1,6.5E1,9.3E1,2.59E2,5.8E1,7.74E2,4.9E1,1.47E2,3.1E1,1.36E2,8E0,7.47E2,1.04E2,7E0,8E1,8E0,8E0,4.1E1,3E1,4.3E1,2.2E1,2.2E1,7.1E1,1.87E2,7.2E1,2.1E1,3.7E1,5.52E2,2.22E2,1.6E1,3.3E1,1.4E1,1.33E2,2.6E1,5E0,1.4E1,1.22E2,4E0,4E0,7.37E2,1E1,1.01E2,3E0,1.1E1,6.9E1,7E0,3.4E1,2.3E1,7E0,1.7E1,2.6E1,9E0,1.3E1,4.5E1,2.6E1,6.7E1,1.2E2,3.5E1,3.7E1,9E0,1.2E1,3.2E1,5E0,5.27E2,2.5E1,5.1E1,1.71E2,5E0,1.1E1,2.9E1,4E0,1.1E2,2.3E1,9E0,1.7E1,9E0,5E0,5.7E1,6.5E1,7.33E2,4E0,7E0,3E0,2.7E1,7.4E1,1.2E1,5.7E1,5E0,2E0,2.5E1,9E0,2.5E1,2E1,2.3E1,3E0,6.3E1,4E0,9.2E1,2.8E1,5E0,3E1,3E1,7E0,2.7E1,5E0,5.04E2,2.3E1,2.9E1,2.2E1,1.7E1,1.54E2,2.6E1,3E0,8.5E1,2.5E1,5E0,1.8E1,6E0,3E0,3.7E1,2E1,2E0,6.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[3.1497396E-4,-3.3174522E-2,2.499809E-2,-2.7953394E-2,-2.1275616E-1,4.7153536E-2,-5.956037E-2,1.0292534E-2,-4.5065027E-2,-2.5134924E-1,-4.919145E-3,4.080183E-2,2.126615E-1,-1.06342115E-1,-5.848964E-4,2.8620219E-2,-1.3330327E-1,-4.3079905E-2,-1.6195986E-2,-8.951981E-3,-1.7810443E-2,5.567284E-2,-1.0505596E-2,-7.920173E-3,2.4360216E-1,-8.218959E-2,-2.1899967E-1,1.6933598E-2,-1.7707242E-1,3.96871E-2,-1.15544096E-1,-3.297028E-2,-2.754713E-1,-5.31273E-2,8.46625E-3,9.934983E-2,3.5041973E-2,-2.779531E-2,7.06485E-2,3.209605E-1,6.822484E-3,-1.8270066E-2,-1.0880092E-1,-3.193961E-1,-1.5598251E-1,8.7244227E-4,5.3428584E-3,-2.4216084E-3,-1.7689347E-2,-2.3022565E-1,4.6340708E-2,-7.338118E-3,5.0194765E-4,2.6103456E-3,-5.985803E-3,-1.8790513E-2,-6.750598E-3,-2.9849296E-2,-1.0065427E-1,1.1894266E-1,-5.418893E-3,7.5983256E-2,1.8882364E-1,5.7601742E-2,-2.3972457E-2,-5.729735E-2,-2.2742713E-3,8.8453434E-2,-1.0534724E-2,8.2052057E-4,1.7512511E-2,-8.882978E-2,2.542221E-3,-5.856492E-2,-1.8578966E-1,-4.551935E-3,-1.8756103E-2,-2.1188618E-3,-1.1306932E-2,3.155813E-2,-4.6198335E-2,-8.5888733E-4,-2.4294993E-2,4.59315E-3,1.0011031E-3,8.592818E-3,-1.6647219E-3,-7.881456E-3,-2.8217298E-3,1.0193262E-2,1.0492948E-3,2.4480538E-3,-3.0107964E-3,7.123467E-3,2.7013319E-3,7.097018E-4,1.0130081E-2,1.5785316E-3,5.629085E-3,-2.5827736E-2,-7.0836325E-4,-4.342947E-3,-5.375952E-4,9.676845E-4,-3.483459E-3,-6.9591203E-3,4.9186293E-3,-6.3045938E-3,2.6063758E-3,8.097856E-3,-3.2793558E-3,-3.6459314E-3,-1.0701635E-2,6.456327E-3,-1.0330937E-4,-4.0139705E-3,7.1279675E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,-1,-1,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,-1,-1,-1,81,83,-1,-1,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,103,105,-1,-1,-1,107,-1,109,111,-1,-1,-1,-1,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.217011E0,1.0610625E0,2.895385E0,7.249404E-1,1.1007917E-1,1.2780626E0,8.8556874E-1,9.0651524E-1,4.3057227E-1,1.0910547E-1,0E0,9.0091693E-1,5.6664634E-1,4.6905327E-1,4.4512528E-1,4.911977E-1,5.472193E-1,3.94575E-1,0E0,0E0,0E0,8.2101965E-1,3.7519655E-1,0E0,2.8289223E-1,2.5145864E-1,1.4630747E-1,1.9900605E-1,2.9757395E-1,5.207024E-1,1.060836E-1,2.0000131E-1,1.9626868E-1,7.017294E-1,1.9239213E-1,5.982268E-1,8.3081305E-1,1.6551267E-1,2.6266068E-1,3.0622125E-1,0E0,2.3127791E-1,3.952179E-1,1.4107275E-1,1.7754954E-1,1.6467506E-1,0E0,0E0,0E0,4.0264893E-1,3.485152E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.4720387E-1,5.478208E-1,1.22898474E-1,3.6698896E-1,3.2492948E-1,2.33881E-1,6.622188E-1,8.105509E-1,1.5588102E-1,1.8997227E-1,1.6485071E-1,0E0,0E0,0E0,1.4823647E-1,0E0,1.5824027E-1,1.5235615E-1,0E0,0E0,0E0,0E0,2.4050154E-1,1.04645856E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,49,49,50,50,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,71,71,73,73,74,74,79,79,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,-1,-1,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,-1,-1,-1,82,84,-1,-1,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,104,106,-1,-1,-1,108,-1,110,112,-1,-1,-1,-1,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.494881E2,3.653602E7,4.0844156E7,2E0,7.080933E2,9.256843E3,3.0387878E3,2E0,7.780377E6,1.2199979E10,-4.919145E-3,1.0938637E1,2.64E0,1.2427474E8,2.6572757E10,1E0,5.5E1,2.437E3,-1.6195986E-2,-8.951981E-3,-1.7810443E-2,2.1068998E6,3.33E2,-7.920173E-3,1.10032365E-1,1.3E1,3.773E3,8.791777E3,3.869321E2,3.5902756E-6,1.5E1,2.943086E4,6.23E2,4.44E2,1.1451E4,8.027061E2,1.5548611E7,7.57657E4,4.543313E6,6.3039363E4,6.822484E-3,8.027061E2,3.325328E11,5E-1,8.7757526E2,5.7075036E-1,5.3428584E-3,-2.4216084E-3,-1.7689347E-2,2.2299826E-6,4.7022E4,-7.338118E-3,5.0194765E-4,2.6103456E-3,-5.985803E-3,-1.8790513E-2,-6.750598E-3,6E0,3.3720784E5,1.701E3,2.1E1,2.3663366E0,1.954E3,1.9125667E3,4.1724915E2,7.6E1,3.9150736E7,1.2561812E6,-1.0534724E-2,8.2052057E-4,1.7512511E-2,1.775894E10,2.542221E-3,4.198242E2,1.387623E8,-4.551935E-3,-1.8756103E-2,-2.1188618E-3,-1.1306932E-2,2.6970406E0,7.1308413E0,-8.5888733E-4,-2.4294993E-2,4.59315E-3,1.0011031E-3,8.592818E-3,-1.6647219E-3,-7.881456E-3,-2.8217298E-3,1.0193262E-2,1.0492948E-3,2.4480538E-3,-3.0107964E-3,7.123467E-3,2.7013319E-3,7.097018E-4,1.0130081E-2,1.5785316E-3,5.629085E-3,-2.5827736E-2,-7.0836325E-4,-4.342947E-3,-5.375952E-4,9.676845E-4,-3.483459E-3,-6.9591203E-3,4.9186293E-3,-6.3045938E-3,2.6063758E-3,8.097856E-3,-3.2793558E-3,-3.6459314E-3,-1.0701635E-2,6.456327E-3,-1.0330937E-4,-4.0139705E-3,7.1279675E-4],"split_indices":[52,45,45,17,4,4,4,6,28,5,0,53,54,45,12,74,0,2,0,0,0,51,8,0,57,3,2,52,57,38,3,33,2,0,12,52,45,28,1,28,0,52,31,53,52,41,0,0,0,41,9,0,0,0,0,0,0,3,28,11,3,54,2,4,4,8,32,45,0,0,0,5,0,52,7,0,0,0,0,53,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E3,1.137E3,1.543E3,1.106E3,3.1E1,1.223E3,3.2E2,3.42E2,7.64E2,2.2E1,9E0,1.179E3,4.4E1,1.78E2,1.42E2,3.04E2,3.8E1,7.6E2,4E0,1.6E1,6E0,9.14E2,2.65E2,3E0,4.1E1,1.48E2,3E1,1.3E2,1.2E1,2.83E2,2.1E1,2.3E1,1.5E1,6.36E2,1.24E2,2.92E2,6.22E2,2.19E2,4.6E1,2.2E1,1.9E1,4.4E1,1.04E2,1E1,2E1,1.12E2,1.8E1,8E0,4E0,6E0,2.77E2,1.6E1,5E0,1.2E1,1.1E1,7E0,8E0,4.28E2,2.08E2,1.3E1,1.11E2,2.33E2,5.9E1,4.5E2,1.72E2,1.01E2,1.18E2,4.4E1,2E0,3E0,1.9E1,2.2E1,2.2E1,6.4E1,4E1,3E0,7E0,9E0,1.1E1,6.8E1,4.4E1,4E0,2E0,9.3E1,1.84E2,9E0,4.19E2,8.1E1,1.27E2,6E0,7E0,5.6E1,5.5E1,4.8E1,1.85E2,7E0,5.2E1,3.2E2,1.3E2,2E0,1.7E2,5.8E1,4.3E1,9E1,2.8E1,2E0,4.2E1,1.7E1,5E0,2E0,6.2E1,1.1E1,2.9E1,1.6E1,5.2E1,2.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-1.7627389E-3,-3.744867E-2,2.0102818E-2,-3.0324143E-1,-3.3499368E-2,-4.797342E-3,5.4007232E-2,-1.8904464E-2,-6.8136784E-3,-6.0507372E-2,5.007274E-3,3.5184257E-2,-2.6916668E-2,1.0392677E-1,1.8612986E-2,4.8116397E-2,-7.404824E-2,1.7286214E-1,-1.3068992E-2,2.9007528E-2,2.932972E-1,-2.8535971E-2,1.5296751E-2,1.5342662E-1,2.6884891E-2,-4.936788E-3,1.0475534E-1,-5.9021775E-2,1.6028972E-1,-6.181677E-2,-2.1931821E-1,1.3459782E-2,1.1259303E-1,-2.4021113E-2,1.7582552E-1,1.263238E-1,1.3351499E-2,2.2614636E-3,2.0664686E-2,-3.5450075E-2,4.833383E-2,1.8487154E-1,4.8942305E-2,1.3114576E-1,-4.4992346E-2,5.7103388E-2,-2.6694005E-2,7.9027444E-2,3.026758E-1,-8.09971E-3,-1.4402532E-2,1.5072074E-2,4.7068493E-3,7.663518E-2,-6.752821E-2,3.1055901E-3,-2.7973026E-1,3.3996522E-3,1.2131323E-2,2.5881257E-2,-5.221674E-2,3.1676063E-1,9.0543233E-4,-6.7965663E-3,1.4783849E-1,-1.22896396E-1,2.5917593E-2,-3.8613707E-2,7.0322594E-3,7.1244105E-3,-2.0801563E-5,1.1957875E-1,2.3504403E-1,2.0107695E-1,-3.9739717E-2,9.344088E-3,3.9513446E-3,-3.6294648E-1,-1.4950543E-2,1.10954516E-1,1.0057674E-2,-3.1813286E-2,8.74128E-3,4.221578E-3,-8.982221E-3,4.1197054E-3,2.0725759E-2,1.1087245E-3,-1.0858509E-2,5.249893E-3,-3.961544E-3,-7.675833E-3,-2.6565394E-3,-1.9565515E-2,-6.8258285E-3,7.619883E-3,2.2116074E-4,-1.9650375E-3,-7.7636135E-3,1.7810792E-2,2.1903557E-3,5.339959E-3,1.18149305E-2,-3.1825502E-3,-1.3914727E-2,2.2246577E-3,-2.9245003E-3,-2.376027E-3,-6.82144E-5,4.772448E-3,1.2542534E-2,1.20764505E-2,1.3305168E-3,1.4040633E-3,1.2254552E-2,-4.531189E-3,4.1483776E-3,-2.2724539E-3,-2.4368213E-2,-1.4380303E-2,1.7712743E-4,3.2866574E-3,9.270804E-3,4.883596E-3,-1.1670967E-3,-1.8728632E-3,4.719962E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,-1,-1,67,69,71,73,75,77,79,81,83,85,-1,87,-1,-1,89,91,-1,93,-1,-1,95,97,99,-1,-1,101,103,105,107,-1,-1,-1,109,111,113,115,-1,-1,117,119,121,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1215885E0,1.0741779E0,1.4236231E0,1.6476667E-1,1.0600504E0,8.614352E-1,1.2596066E0,0E0,0E0,8.846085E-1,1.2796527E0,5.447122E-1,3.6074227E-1,1.1246893E0,8.5051435E-1,8.1466955E-1,9.33512E-1,2.356987E-1,7.9243463E-1,5.1633275E-1,2.516874E-1,3.3389193E-1,0E0,5.8234215E-1,8.826965E-1,4.4696543E-1,4.3436736E-1,1.6951665E-1,2.8506285E-1,3.9501965E-1,7.2198844E-1,0E0,1.4594895E-1,5.0928867E-1,4.5843518E-1,2.9222548E-1,5.0654376E-1,0E0,0E0,3.366959E-1,2.547513E-1,4.2396832E-1,5.881298E-1,1.3324183E-1,6.632079E-1,2.1676207E-1,2.6821584E-1,2.1353227E-1,2.4463135E-1,0E0,2.1257113E-1,0E0,0E0,1.14203446E-1,5.211859E-1,0E0,5.4267454E-1,0E0,0E0,3.7037247E-1,2.772845E-1,1.4739406E-1,0E0,0E0,1.3055658E-1,2.1726173E-1,4.851237E-1,2.3106104E-1,0E0,0E0,0E0,1.497218E-1,2.6627445E-1,1.4312279E-1,2.023536E-1,0E0,0E0,2.5201178E-1,3.469784E-1,1.2660825E-1,1.512156E-1,2.3207206E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,53,53,54,54,56,56,59,59,60,60,61,61,64,64,65,65,66,66,67,67,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,-1,-1,68,70,72,74,76,78,80,82,84,86,-1,88,-1,-1,90,92,-1,94,-1,-1,96,98,100,-1,-1,102,104,106,108,-1,-1,-1,110,112,114,116,-1,-1,118,120,122,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3E0,5.862126E2,5.101512E0,3.386E3,2E0,5.511673E0,-1.8904464E-2,-6.8136784E-3,1E0,7.4521235E6,2.1442623E1,1.3137E4,2.1351435E0,7.115465E6,1.56E2,9.457831E0,3.496664E9,4.5505118E2,4.296196E0,1.1668591E6,1.8149056E7,1.5296751E-2,1.3195207E7,4.0208E4,6.486406E6,4.516183E1,1.64E1,1.2920592E0,5.8365756E-1,1.3E1,1.3459782E-2,2.262586E6,2.8091298E7,6.7321223E-1,5.714286E-1,1.632095E2,2.2614636E-3,2.0664686E-2,3.15856E0,3.37E2,2.264389E-1,1.0594161E1,2.1155348E6,3.333E3,2.881356E-2,2.6312E4,1.33832655E1,6.4683E4,-8.09971E-3,3.64299E5,1.5072074E-2,4.7068493E-3,1.3483871E2,5.8941E1,3.1055901E-3,8E0,3.3996522E-3,1.2131323E-2,7.55E2,1.5265896E8,5.5544252E7,9.0543233E-4,-6.7965663E-3,6.7317666E-4,1.6521739E0,1.604E3,1.7248E4,7.0322594E-3,7.1244105E-3,-2.0801563E-5,8.66076E1,2.02E2,1.8820395E6,6.1707315E0,9.344088E-3,3.9513446E-3,2.821E3,1.683653E8,1.4122984E5,3E0,1E0,8.74128E-3,4.221578E-3,-8.982221E-3,4.1197054E-3,2.0725759E-2,1.1087245E-3,-1.0858509E-2,5.249893E-3,-3.961544E-3,-7.675833E-3,-2.6565394E-3,-1.9565515E-2,-6.8258285E-3,7.619883E-3,2.2116074E-4,-1.9650375E-3,-7.7636135E-3,1.7810792E-2,2.1903557E-3,5.339959E-3,1.18149305E-2,-3.1825502E-3,-1.3914727E-2,2.2246577E-3,-2.9245003E-3,-2.376027E-3,-6.82144E-5,4.772448E-3,1.2542534E-2,1.20764505E-2,1.3305168E-3,1.4040633E-3,1.2254552E-2,-4.531189E-3,4.1483776E-3,-2.2724539E-3,-2.4368213E-2,-1.4380303E-2,1.7712743E-4,3.2866574E-3,9.270804E-3,4.883596E-3,-1.1670967E-3,-1.8728632E-3,4.719962E-3],"split_indices":[102,3,52,56,2,17,53,0,0,79,45,56,2,41,1,2,54,31,56,58,28,9,0,9,9,45,54,4,56,54,0,0,1,45,41,53,4,0,0,38,0,38,56,45,2,57,2,53,2,0,9,0,0,52,50,0,17,0,0,10,45,45,0,0,42,53,2,11,0,0,0,56,8,45,54,0,0,2,7,33,8,89,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.717E3,1.032E3,1.685E3,1.4E1,1.018E3,9.72E2,7.13E2,8E0,6E0,5.98E2,4.2E2,3.46E2,6.26E2,2.95E2,4.18E2,6.6E1,5.32E2,4E1,3.8E2,3.39E2,7E0,6.24E2,2E0,1.79E2,1.16E2,3.29E2,8.9E1,3.4E1,3.2E1,4.92E2,4E1,1.3E1,2.7E1,3.6E2,2E1,4.6E1,2.93E2,3E0,4E0,5.73E2,5.1E1,1.37E2,4.2E1,4.7E1,6.9E1,8.5E1,2.44E2,8E1,9E0,9E0,2.5E1,8E0,2.4E1,1.9E1,4.73E2,7E0,3.3E1,2.2E1,5E0,1.3E2,2.3E2,1E1,1E1,3E0,4.3E1,2.4E1,2.69E2,5.64E2,9E0,1.6E1,3.5E1,6.1E1,7.6E1,1.5E1,2.7E1,1.9E1,2.8E1,5E0,6.4E1,3.9E1,4.6E1,2.39E2,5E0,7.8E1,2E0,4E0,5E0,2.2E1,3E0,1.6E1,3E0,5.3E1,4.2E2,1.6E1,1.7E1,1.7E1,1.13E2,2.1E2,2E1,8E0,2E0,3.3E1,1E1,1.9E1,5E0,2.18E2,5.1E1,4.35E2,1.29E2,5.5E1,6E0,7E1,6E0,4E0,1.1E1,1.9E1,8E0,2E0,3E0,3E0,6.1E1,2.7E1,1.2E1,1.2E1,3.4E1,2.27E2,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[4.8568082E-4,-3.258395E-2,2.14746E-2,-3.751854E-2,1.9723402E-1,6.969378E-2,9.344302E-4,-5.8749013E-2,-1.5686114E-3,2.1594968E-2,7.391775E-2,4.3773275E-2,1.5361112E-1,-2.5254713E-2,5.379143E-2,5.0714944E-2,-7.271401E-2,6.937837E-2,-5.205534E-2,7.054741E-3,-1.9868636E-3,4.8532557E-2,-1.9624734E-2,1.7196076E-1,-1.7195893E-4,-3.5252433E-2,2.370557E-2,4.557284E-2,9.571321E-3,-5.7779346E-2,1.4678124E-1,-6.41821E-2,-1.7803784E-1,1.7666084E-1,3.6273204E-2,-4.132565E-2,-1.9737004E-1,1.1688049E-2,4.189867E-2,1.9445686E-1,7.4858986E-2,5.4707425E-3,-3.952174E-2,-3.2989603E-2,5.5820603E-2,7.867378E-2,4.6786005E-3,-1.4746554E-2,-1.856222E-2,1.55604975E-2,8.25096E-2,-5.4471683E-2,-1.6453518E-1,-3.6175594E-1,-8.320711E-2,2.8390616E-1,3.3789493E-2,5.8580093E-2,-3.2462347E-3,1.0958357E-2,-4.6395212E-2,-1.1872441E-2,1.7173376E-4,-5.3460016E-3,7.047921E-2,1.6799125E-1,1.4117274E-2,-1.4391664E-3,6.787559E-3,-8.801597E-2,-2.7993293E-2,3.608498E-3,-7.119545E-2,1.1746106E-1,1.7550502E-2,8.8049635E-2,-1.3008593E-1,-2.0380553E-2,9.364785E-2,-8.570586E-3,1.4382713E-3,5.1622824E-3,-3.1352933E-3,-3.0331928E-3,4.127127E-3,-6.215174E-3,-2.0285577E-2,-1.9403074E-2,-3.2362451E-3,-1.136077E-2,-2.3502524E-3,1.4919993E-2,6.816437E-4,6.2146154E-3,-3.1580215E-3,-6.3167783E-3,3.5519456E-3,-4.4889227E-3,-1.0131429E-3,2.8760877E-4,-1.4665421E-2,1.094432E-2,2.8974812E-3,9.184335E-3,2.9953283E-3,-1.5149879E-3,-7.7856192E-3,-1.2036066E-4,-2.865393E-3,-1.4624825E-4,-6.321722E-3,7.5754356E-3,5.7737326E-4,3.1520605E-5,7.1722777E-3,9.348004E-3,3.5657373E-3,-1.1006491E-2,2.4924967E-3,4.6678516E-3,-1.5465537E-3,6.7038545E-3,-4.1196668E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,-1,39,-1,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,65,67,-1,69,71,73,75,77,79,-1,-1,81,83,85,87,89,91,93,95,-1,-1,97,-1,-1,99,101,103,-1,-1,-1,105,107,-1,109,111,113,115,117,119,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8574059E0,1.1856854E0,1.6212689E0,7.7688E-1,6.549176E-1,1.0569389E0,1.5918957E0,9.8203254E-1,1.3611443E0,0E0,1.4162874E-1,8.293636E-1,3.3234596E-1,3.7725434E-1,4.482684E-1,7.693339E-1,4.989853E-1,5.547454E-1,3.3798188E-1,0E0,0E0,4.7229213E-1,0E0,2.1043754E-1,0E0,4.1232765E-1,2.4036202E-1,4.896487E-1,0E0,4.9103808E-1,4.197076E-1,5.045836E-1,7.016283E-1,5.573313E-1,2.871576E-1,2.954775E-1,1.5322721E-1,0E0,4.8782724E-1,1.7085814E-1,1.5158515E-1,0E0,3.4527344E-1,2.0404436E-1,1.9609335E-1,4.0291417E-1,3.6531612E-1,2.4068373E-1,0E0,0E0,1.1929986E-1,6.0639644E-1,3.5919118E-1,1.513598E-1,1.4080627E-1,1.520803E-1,1.712818E-1,3.0967113E-1,0E0,0E0,2.4295044E-1,0E0,0E0,4.6975923E-1,3.409667E-1,1.5856147E-1,0E0,0E0,0E0,4.9507827E-1,4.0872228E-1,0E0,1.4876528E-1,1.3544405E-1,1.1827124E-1,2.651143E-1,1.754539E-1,1.8285187E-1,3.117428E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,60,60,63,63,64,64,65,65,69,69,70,70,72,72,73,73,74,74,75,75,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,-1,40,-1,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,66,68,-1,70,72,74,76,78,80,-1,-1,82,84,86,88,90,92,94,96,-1,-1,98,-1,-1,100,102,104,-1,-1,-1,106,108,-1,110,112,114,116,118,120,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,4.5505118E2,2E0,3.83E3,1.6907827E9,1.3572903E10,9.557794E2,1E0,2.8091298E7,2.1594968E-2,5.1797544E7,1.3361491E10,2.3977574E7,1.5083523E0,4.091015E9,4.1161446E1,9.457831E0,2.0131812E7,7.756375E-1,7.054741E-3,-1.9868636E-3,1.3E1,-1.9624734E-2,1E0,-1.7195893E-4,3.4E2,1.088E3,9.213145E0,9.571321E-3,3.64299E5,7.330957E4,6.496696E7,2E0,1.65E2,1.1953147E2,3.53562E5,8.985569E-1,1.1688049E-2,1.5687E4,1.8E1,4.1365924E2,5.4707425E-3,8.87E2,4.7022E4,9.1E3,2.3821254E7,1.5435694E9,1.64E1,-1.856222E-2,1.55604975E-2,1.0715278E7,1.0289962E3,2.55298E5,6E1,4.22088E0,6.5E1,1.0545219E5,3.290375E-2,-3.2462347E-3,1.0958357E-2,2.0288463E0,-1.1872441E-2,1.7173376E-4,2.4E1,4.1E1,1.5837E4,1.4117274E-2,-1.4391664E-3,6.787559E-3,1.6908121E1,3.5188188E5,3.608498E-3,4.5154482E-1,4.949877E6,1.30399E5,5.42974E5,6.117756E9,8.573816E0,1.100592E6,-8.570586E-3,1.4382713E-3,5.1622824E-3,-3.1352933E-3,-3.0331928E-3,4.127127E-3,-6.215174E-3,-2.0285577E-2,-1.9403074E-2,-3.2362451E-3,-1.136077E-2,-2.3502524E-3,1.4919993E-2,6.816437E-4,6.2146154E-3,-3.1580215E-3,-6.3167783E-3,3.5519456E-3,-4.4889227E-3,-1.0131429E-3,2.8760877E-4,-1.4665421E-2,1.094432E-2,2.8974812E-3,9.184335E-3,2.9953283E-3,-1.5149879E-3,-7.7856192E-3,-1.2036066E-4,-2.865393E-3,-1.4624825E-4,-6.321722E-3,7.5754356E-3,5.7737326E-4,3.1520605E-5,7.1722777E-3,9.348004E-3,3.5657373E-3,-1.1006491E-2,2.4924967E-3,4.6678516E-3,-1.5465537E-3,6.7038545E-3,-4.1196668E-3],"split_indices":[102,56,17,2,12,31,52,79,45,0,50,31,47,42,7,55,54,12,34,0,0,10,0,74,0,12,10,53,0,9,45,45,17,3,58,1,27,0,1,8,52,0,2,9,29,9,7,4,0,0,45,4,11,3,53,8,28,38,0,0,53,0,0,8,0,29,0,0,0,58,48,0,38,9,10,1,12,58,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.674E3,1.038E3,1.636E3,1.017E3,2.1E1,4.88E2,1.148E3,6.39E2,3.78E2,6E0,1.5E1,3.74E2,1.14E2,7.68E2,3.8E2,7.2E1,5.67E2,1.57E2,2.21E2,9E0,6E0,3.71E2,3E0,1.02E2,1.2E1,6.38E2,1.3E2,3.61E2,1.9E1,3.4E1,3.8E1,5.26E2,4.1E1,3.6E1,1.21E2,2.07E2,1.4E1,1.1E1,3.6E2,8.2E1,2E1,1.7E1,6.21E2,4.7E1,8.3E1,1.99E2,1.62E2,3.1E1,3E0,9E0,2.9E1,4.81E2,4.5E1,1.3E1,2.8E1,2E1,1.6E1,1E2,2.1E1,3E0,2.04E2,1.1E1,3E0,1.36E2,2.24E2,6.7E1,1.5E1,8E0,1.2E1,1.18E2,5.03E2,1.2E1,3.5E1,3.1E1,5.2E1,1.91E2,8E0,1.27E2,3.5E1,6E0,2.5E1,2.5E1,4E0,4.53E2,2.8E1,4.1E1,4E0,1.1E1,2E0,4E0,2.4E1,1.8E1,2E0,8E0,8E0,7E0,9.3E1,7E1,1.34E2,1.32E2,4E0,1.2E1,2.12E2,5.4E1,1.3E1,6.8E1,5E1,2.8E2,2.23E2,1.7E1,1.8E1,2.2E1,9E0,4.7E1,5E0,2E1,1.71E2,5E0,3E0,1.1E1,1.16E2,2.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[1.122273E-3,1.6461233E-2,-4.168877E-2,4.798923E-3,8.368394E-2,-9.899803E-2,-2.475647E-2,-5.716325E-3,6.0941726E-2,9.858676E-2,-1.340662E-1,-7.43938E-2,-2.2294313E-1,-1.291578E-1,-1.5764412E-2,-2.0147132E-2,3.460444E-2,2.4177296E-2,9.235781E-2,2.705808E-1,8.236896E-2,-1.6497461E-2,-4.264539E-3,-1.05353616E-1,2.85504E-2,-3.0005276E-1,-4.3952387E-2,-2.0805192E-1,-3.0665249E-3,-9.209467E-3,-1.5836276E-1,-4.0898856E-2,5.556926E-3,8.526216E-2,4.017814E-3,-1.74771E-2,7.772601E-2,1.0861144E-1,-6.400479E-2,1.4851156E-2,3.1736067E-3,6.284748E-2,1.8034804E-1,-5.514342E-2,-1.8762335E-1,-6.381103E-3,7.918243E-2,-3.3320247E-3,-1.6355086E-2,-9.450627E-3,1.3439644E-3,-1.9333801E-3,-1.2693598E-2,-2.5268849E-2,4.1540045E-2,-7.816136E-2,-1.7202582E-2,-3.8077388E-2,-2.7114008E-2,1.9261189E-2,-9.910443E-2,9.271901E-2,-1.3705013E-2,4.9940247E-2,-2.8525937E-2,1.4451918E-2,-4.354447E-1,1.2967029E-1,2.0979121E-2,-2.115313E-3,1.2043811E-1,-1.484644E-2,1.4975125E-3,4.4093594E-2,1.4657314E-1,3.4024152E-1,1.02452755E-1,-1.2364596E-1,2.6186744E-2,1.6441878E-3,-2.2901927E-1,1.0647811E-2,-7.297423E-4,2.8201488E-3,-5.59421E-2,1.06671676E-1,1.0897415E-2,-7.036022E-3,8.407499E-4,-1.3786291E-3,-7.141334E-3,2.4715096E-3,-1.0682301E-3,-3.8040362E-3,-1.9494329E-2,4.8629413E-3,-5.6668487E-3,3.0278447E-3,-7.761693E-3,-2.2699519E-3,4.366605E-3,-1.1986471E-3,4.5281085E-3,-1.316429E-3,-3.3439707E-2,8.342582E-3,-2.9999914E-3,-2.1727125E-3,5.157528E-3,6.544728E-3,3.0906938E-4,9.184618E-3,1.3820525E-3,9.825487E-3,2.547095E-3,2.0980364E-2,7.3306053E-3,-5.1103258E-3,7.741795E-3,1.5449917E-4,-7.662981E-3,6.671737E-3,-2.0591917E-3,2.1146953E-3,-1.23466E-2,1.954326E-3,-2.2776462E-3,1.2962887E-4,-3.4331006E-3,6.44116E-3,-1.5227275E-3,7.187468E-3,-2.1574109E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,-1,-1,73,75,77,79,-1,81,-1,-1,-1,-1,-1,-1,83,85,87,-1,89,-1,91,93,95,-1,97,99,101,103,105,107,-1,109,-1,-1,111,113,115,117,119,121,-1,123,-1,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7500734E0,1.5377517E0,6.797246E-1,9.880131E-1,9.5429397E-1,4.7134614E-1,5.0866187E-1,8.211216E-1,3.026855E-1,7.364199E-1,1.4930499E-1,4.3327093E-1,3.4566152E-1,2.0761406E-1,4.677731E-1,5.5443347E-1,5.760523E-1,2.7571476E-1,3.6783898E-1,1.6518772E-1,4.668398E-1,0E0,0E0,4.192549E-1,2.7269307E-1,1.482271E-1,1.04953475E-1,1.7090607E-1,0E0,3.9306748E-1,3.2629853E-1,8.306063E-1,6.6886145E-1,4.142977E-1,3.4964034E-1,9.404637E-1,1.5645418E-1,2.384088E-1,3.3782244E-1,0E0,0E0,3.232752E-1,4.7579432E-1,3.709452E-1,3.66197E-1,0E0,3.413034E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.1549633E-1,2.3035088E-1,1.144607E-1,0E0,5.868136E-1,0E0,5.5509883E-1,2.9481292E-1,2.690066E-1,0E0,2.8343478E-1,3.1382763E-1,2.1108393E-1,5.1687753E-1,2.507E-1,1.6022544E-1,0E0,2.210052E-1,0E0,0E0,3.8372576E-1,1.9701612E-1,1.7102921E-1,3.763581E-1,1.6660672E-1,2.48613E-1,0E0,2.709161E-1,0E0,0E0,3.6939448E-1,1.6178668E-1,1.468369E-1,1.7217603E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,46,46,53,53,54,54,55,55,57,57,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,70,70,73,73,74,74,75,75,76,76,77,77,78,78,80,80,83,83,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,-1,-1,74,76,78,80,-1,82,-1,-1,-1,-1,-1,-1,84,86,88,-1,90,-1,92,94,96,-1,98,100,102,104,106,108,-1,110,-1,-1,112,114,116,118,120,122,-1,124,-1,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.5106794E3,1.8903887E0,2.065904E6,2.3580047E10,2.0354E4,1.2209878E3,4.264897E6,6.789622E7,1.2684739E1,8.962E3,1.6772E4,1.038946E6,4.3256033E2,1.5265896E8,3.7311627E2,2.69E2,5.2921E4,3.1984328E5,1.4659271E7,9.256843E3,-1.6497461E-2,-4.264539E-3,1.8481675E0,7.68E2,2.04425E2,1.4560842E-2,4.0814578E2,-3.0665249E-3,2.0601392E3,2.894E3,4.5158855E6,3.7852024E1,1E0,1.5E1,2.829581E10,1.366E3,1.5139E4,1.4523809E0,1.4851156E-2,3.1736067E-3,4.43837E0,4.0601485E6,2.8884522E8,2.0304577E-1,-6.381103E-3,6.028929E5,-3.3320247E-3,-1.6355086E-2,-9.450627E-3,1.3439644E-3,-1.9333801E-3,-1.2693598E-2,1.4540612E7,2.5927516E11,3.931011E6,-1.7202582E-2,1.0219829E1,-2.7114008E-2,1.1394967E7,3.3720784E5,3.1826714E7,-1.3705013E-2,1E0,1E0,1.164249E7,3.8866E4,2.5652175E0,1.72797E5,-2.115313E-3,2.3821254E7,-1.484644E-2,1.4975125E-3,3.53562E5,1.745982E0,3.275343E6,5.0279167E1,4.7937644E-1,7.5891815E6,1.6441878E-3,4.107E3,1.0647811E-2,-7.297423E-4,1.8344E4,2.765625E0,1.4285097E1,2.99665E5,-7.036022E-3,8.407499E-4,-1.3786291E-3,-7.141334E-3,2.4715096E-3,-1.0682301E-3,-3.8040362E-3,-1.9494329E-2,4.8629413E-3,-5.6668487E-3,3.0278447E-3,-7.761693E-3,-2.2699519E-3,4.366605E-3,-1.1986471E-3,4.5281085E-3,-1.316429E-3,-3.3439707E-2,8.342582E-3,-2.9999914E-3,-2.1727125E-3,5.157528E-3,6.544728E-3,3.0906938E-4,9.184618E-3,1.3820525E-3,9.825487E-3,2.547095E-3,2.0980364E-2,7.3306053E-3,-5.1103258E-3,7.741795E-3,1.5449917E-4,-7.662981E-3,6.671737E-3,-2.0591917E-3,2.1146953E-3,-1.23466E-2,1.954326E-3,-2.2776462E-3,1.2962887E-4,-3.4331006E-3,6.44116E-3,-1.5227275E-3,7.187468E-3,-2.1574109E-4],"split_indices":[27,52,53,28,5,9,48,9,7,56,0,9,9,52,45,52,10,1,33,50,4,0,0,54,2,4,38,33,0,52,0,47,53,65,3,31,10,9,54,0,0,35,47,12,27,0,51,0,0,0,0,0,0,45,31,1,0,54,0,45,28,45,0,85,101,45,1,54,1,0,9,0,0,1,34,29,56,27,45,0,12,0,0,9,54,54,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.663E3,1.961E3,7.02E2,1.672E3,2.89E2,1.59E2,5.43E2,1.409E3,2.63E2,2.71E2,1.8E1,1.34E2,2.5E1,4.2E1,5.01E2,1.038E3,3.71E2,1.22E2,1.41E2,2.2E1,2.49E2,2E0,1.6E1,1.03E2,3.1E1,1.7E1,8E0,1.8E1,2.4E1,4.8E2,2.1E1,5.74E2,4.64E2,1.39E2,2.32E2,6.9E1,5.3E1,1.28E2,1.3E1,1.8E1,4E0,2.09E2,4E1,6.5E1,3.8E1,7E0,2.4E1,3E0,1.4E1,2E0,6E0,5E0,1.3E1,3.65E2,1.15E2,1.6E1,5E0,5.72E2,2E0,4.11E2,5.3E1,1.37E2,2E0,9.6E1,1.36E2,6.5E1,4E0,2.7E1,2.6E1,9E0,1.19E2,3E0,1E1,1.72E2,3.7E1,1.2E1,2.8E1,3.5E1,3E1,6E0,3.2E1,9E0,1.5E1,1.91E2,1.74E2,3.6E1,7.9E1,9E0,7E0,5.29E2,4.3E1,2.31E2,1.8E2,5.1E1,2E0,1.32E2,5E0,9.1E1,5E0,1.18E2,1.8E1,4.4E1,2.1E1,2E0,2E0,2.2E1,5E0,1.5E1,1.1E1,1.04E2,1.5E1,1.5E1,1.57E2,2.2E1,1.5E1,7E0,5E0,6E0,2.2E1,8E0,2.7E1,1.1E1,1.9E1,3E0,2.9E1,1.09E2,8.2E1,3.7E1,1.37E2,3E1,6E0,7E0,7.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-4.3682673E-4,-1.1988949E-2,5.3294897E-2,-3.5073673E-3,-1.0489852E-1,6.269076E-2,-1.5192501E-1,-4.4811577E-2,1.283393E-2,-9.74755E-2,-1.9853769E-2,1.3705349E-1,3.815474E-2,-1.5931306E-2,-3.7795585E-2,-2.5494812E-2,-9.230509E-2,2.7599087E-2,-2.8364308E-2,-1.5141769E-1,-7.455758E-2,1.692949E-1,-9.4448385E-4,1.8228639E-2,9.634931E-2,-4.62945E-3,4.190758E-3,-4.556368E-2,4.2970322E-2,-6.4062715E-2,-2.555597E-1,3.8849052E-2,-1.3412082E-2,-1.0509772E-1,-7.689571E-3,-1.0829665E-1,-2.757673E-1,-1.0070515E-2,-6.38462E-2,2.2418816E-1,4.0420484E-2,5.2466318E-2,-3.3138655E-2,2.20241E-1,6.747963E-2,-2.956571E-2,-1.5214601E-1,9.680287E-2,-7.344682E-3,-2.2085331E-1,-5.051151E-2,-3.4432095E-3,-3.269561E-1,-1.2965984E-2,4.875814E-2,-2.5820075E-2,1.4357159E-1,-2.114322E-2,-8.64994E-2,-2.0533808E-2,1.0132536E-1,-8.252397E-3,1.3160965E-3,-4.291358E-3,-1.6692175E-2,3.669709E-2,-7.6288424E-2,2.452613E-1,6.9656246E-4,7.6099024E-3,-4.0812775E-2,7.824821E-2,-2.0499572E-2,3.0964212E-3,-7.1609765E-2,2.0267943E-2,5.618303E-3,9.11328E-2,-1.6517588E-1,-1.0537513E-3,-1.2043106E-2,-3.348048E-3,-9.672948E-3,1.445865E-4,7.3576714E-3,4.12602E-4,-7.1604773E-3,1.8000358E-3,-1.79064E-2,-1.5744084E-3,-8.444079E-3,-2.1234956E-2,-4.657603E-3,1.0151772E-3,-7.519133E-3,8.624463E-3,2.1188895E-3,-1.8367084E-3,4.6769585E-3,1.2908282E-2,1.3620995E-3,-2.4866106E-3,-1.153468E-2,2.1141423E-3,-1.5131911E-3,8.246741E-3,9.428899E-4,8.08588E-3,-2.6567862E-3,-8.797418E-3,-2.9794667E-3,3.760851E-3,1.2415387E-2,-9.000331E-3,1.4477052E-3,5.7559083E-3,1.3270079E-3,-4.511221E-3,2.9497847E-3,-2.3017826E-3,2.339792E-3,-5.369397E-3,-2.5101026E-4,5.092409E-4,6.5552206E-3,-1.4664816E-2,5.983849E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,37,39,-1,41,43,-1,-1,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,97,99,-1,101,103,105,-1,-1,-1,-1,107,109,111,-1,-1,113,115,117,119,121,-1,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6591905E0,1.7332764E0,9.215708E-1,1.3619782E0,3.8254452E-1,8.212987E-1,4.205997E-1,5.2241147E-1,8.803824E-1,2.1247065E-1,0E0,5.694599E-1,3.9482367E-1,0E0,1.1173403E-1,5.625571E-1,7.443932E-1,4.918844E-1,6.0521317E-1,2.5432575E-1,1.7411381E-1,6.4885163E-1,0E0,4.519651E-1,2.9631233E-1,0E0,0E0,5.343136E-1,2.5260943E-1,2.9055208E-1,2.9120016E-1,4.2986798E-1,4.5171303E-1,4.6857512E-1,4.2494515E-1,3.6113536E-1,1.4490342E-1,0E0,1.541684E-1,2.8359222E-1,2.855998E-1,2.9130867E-1,1.4380412E-1,2.8460783E-1,4.1127178E-1,4.5763236E-1,1.5390396E-1,2.393839E-1,1.1247449E-1,4.4914806E-1,2.870865E-1,0E0,4.1160405E-1,6.67661E-1,4.0816402E-1,3.349024E-1,2.3887423E-1,0E0,4.037876E-1,1.9530866E-1,1.8176073E-1,0E0,0E0,0E0,0E0,1.7988849E-1,1.5183747E-1,1.1720204E-1,0E0,0E0,1.9381587E-1,2.3830849E-1,2.5386187E-1,1.2876807E-1,1.3383615E-1,0E0,0E0,2.4159354E-1,3.2469398E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,65,65,66,66,67,67,70,70,71,71,72,72,73,73,74,74,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,38,40,-1,42,44,-1,-1,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,98,100,-1,102,104,106,-1,-1,-1,-1,108,110,112,-1,-1,114,116,118,120,122,-1,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5106794E3,4.2343444E7,1.2367184E3,1.638058E2,4.1973075E6,1.1317E4,1.5166431E5,2.5943396E0,5.884028E0,1.0275E4,-1.9853769E-2,5.5544252E7,5.528107E3,-1.5931306E-2,5.7763332E7,1.04E3,3.2785368E7,1.4540612E7,2.373E3,1.8600049E10,3E0,2.2277021E0,-9.4448385E-4,2.589369E7,2.0399538E5,-4.62945E-3,4.190758E-3,1E0,9.887221E5,1.198E5,3.8609805E10,7E2,1.3238013E2,3.1964528E7,1.2502964E9,5.35E3,6.85945E4,-1.0070515E-2,1.6559546E0,7.273333E1,2.53383E5,1.0938637E1,3.6E1,2.4394053E9,1.56E2,1E0,1.71E2,1.0119178E10,4.438435E6,7.635605E-6,7.686371E5,-3.4432095E-3,8.17766E5,1E2,1.1693E4,1E0,3.7974813E8,-2.114322E-2,8.1407714E-1,1.4858E4,2.130881E6,-8.252397E-3,1.3160965E-3,-4.291358E-3,-1.6692175E-2,9.222717E-1,7.8040985E2,3.689E3,6.9656246E-4,7.6099024E-3,7.883E3,2.1374066E7,1.04E2,1.7E1,1E0,2.0267943E-2,5.618303E-3,4.9386E4,3.555E3,-1.0537513E-3,-1.2043106E-2,-3.348048E-3,-9.672948E-3,1.445865E-4,7.3576714E-3,4.12602E-4,-7.1604773E-3,1.8000358E-3,-1.79064E-2,-1.5744084E-3,-8.444079E-3,-2.1234956E-2,-4.657603E-3,1.0151772E-3,-7.519133E-3,8.624463E-3,2.1188895E-3,-1.8367084E-3,4.6769585E-3,1.2908282E-2,1.3620995E-3,-2.4866106E-3,-1.153468E-2,2.1141423E-3,-1.5131911E-3,8.246741E-3,9.428899E-4,8.08588E-3,-2.6567862E-3,-8.797418E-3,-2.9794667E-3,3.760851E-3,1.2415387E-2,-9.000331E-3,1.4477052E-3,5.7559083E-3,1.3270079E-3,-4.511221E-3,2.9497847E-3,-2.3017826E-3,2.339792E-3,-5.369397E-3,-2.5101026E-4,5.092409E-4,6.5552206E-3,-1.4664816E-2,5.983849E-3],"split_indices":[52,45,58,52,28,32,28,56,53,9,0,45,52,0,43,2,12,45,2,5,3,41,0,45,33,0,0,104,50,7,31,2,56,7,7,2,32,0,56,54,9,53,3,5,8,14,0,19,51,41,32,0,1,0,1,105,12,0,41,9,1,0,0,0,0,27,4,2,0,0,2,50,8,3,102,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.671E3,2.199E3,4.72E2,2.016E3,1.83E2,4.52E2,2E1,5.71E2,1.445E3,1.8E2,3E0,1.11E2,3.41E2,7E0,1.3E1,4.07E2,1.64E2,1.064E3,3.81E2,5.2E1,1.28E2,9.2E1,1.9E1,2.55E2,8.6E1,9E0,4E0,3.15E2,9.2E1,1.41E2,2.3E1,8.35E2,2.29E2,8E1,3.01E2,4E1,1.2E1,8E0,1.2E2,6.4E1,2.8E1,1.53E2,1.02E2,1.5E1,7.1E1,2.75E2,4E1,4.4E1,4.8E1,1E1,1.31E2,7E0,1.6E1,1.34E2,7.01E2,2.13E2,1.6E1,3E0,7.7E1,2.7E2,3.1E1,2.7E1,1.3E1,4E0,8E0,1.3E1,1.07E2,5.8E1,6E0,1.1E1,1.7E1,1.13E2,4E1,5.3E1,4.9E1,4E0,1.1E1,6.5E1,6E0,2.67E2,8E0,1.6E1,2.4E1,1.7E1,2.7E1,4.4E1,4E0,4E0,6E0,1.16E2,1.5E1,1E1,6E0,1.09E2,2.5E1,2.2E1,6.79E2,1.94E2,1.9E1,7E0,9E0,6.4E1,1.3E1,3.9E1,2.31E2,1.6E1,1.5E1,5E0,8E0,1.1E1,9.6E1,5E0,5.3E1,5E0,1.2E1,6.1E1,5.2E1,2.1E1,1.9E1,2.5E1,2.8E1,3E1,1.9E1,2.4E1,4.1E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[4.828569E-4,-2.9840367E-2,1.8985806E-2,-3.3553567E-2,1.3759737E-1,6.1527655E-2,7.5819873E-4,-2.3596663E-2,-9.0541855E-2,2.3813145E-1,-1.6319931E-3,4.5055754E-2,1.6233203E-1,-2.3350995E-2,3.950535E-2,-5.184071E-2,1.01660695E-2,-1.746807E-1,-6.3440055E-2,1.8066917E-2,6.932481E-3,5.0337013E-2,-2.0523557E-1,1.0892338E-2,8.861718E-2,-1.9702343E-2,-1.713575E-1,3.137077E-2,1.8021442E-1,-2.1582238E-2,-1.0093017E-1,9.763381E-2,-1.252284E-2,-1.3807252E-1,-2.0299643E-2,-1.3128415E-1,-4.140249E-2,3.3047426E-2,1.1387888E-1,-2.4418661E-2,-2.1719413E-3,-1.2022199E-3,7.269195E-3,-6.922451E-2,-9.265046E-3,-8.469876E-4,-1.1202243E-2,8.417833E-2,1.1659266E-2,1.1365007E-2,2.8103844E-3,-1.6678277E-2,-1.1781997E-2,-6.301815E-2,-1.9635841E-1,9.307029E-3,6.998885E-2,-3.2874502E-3,-1.03769824E-1,3.106797E-3,-7.3638717E-3,-4.8004016E-3,-1.6582068E-2,1.3620384E-2,-6.547274E-2,4.2975508E-2,-1.2298664E-1,1.6699685E-1,9.109882E-3,-1.2946209E-2,-1.4621538E-1,8.775503E-3,-3.6246713E-2,1.2075153E-1,1.5821797E-3,1.5889054E-2,-1.1828854E-2,-3.6831878E-3,1.6655715E-4,-6.8794546E-4,-7.499469E-3,-1.20777665E-2,-1.0695722E-3,-4.0482474E-3,4.3491E-3,2.9804683E-3,-1.0347534E-3,-7.3436983E-3,4.9594096E-3,-1.6268764E-3,4.6895538E-3,-6.8528187E-3,-1.1055571E-3,1.0663287E-2,1.6770991E-3,-2.3005684E-3,-2.5462562E-2,1.25526935E-2,6.010716E-3,-9.869995E-3,3.304582E-3,1.5312432E-3,-4.182445E-3,-5.6287018E-3,-1.4832485E-2,3.1385345E-3,-4.0906045E-4,-8.837213E-3,-1.3406385E-3,9.185842E-3,2.5662754E-3,-1.7604264E-3,7.934366E-3,-5.892375E-4,3.169401E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,-1,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,-1,-1,-1,-1,69,71,-1,-1,73,75,-1,-1,77,-1,79,81,-1,83,85,87,-1,-1,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4879606E0,6.2945545E-1,1.2770966E0,5.562117E-1,3.8781703E-1,8.129432E-1,1.0789461E0,8.005943E-1,3.218441E-1,1.2725759E-1,0E0,5.734404E-1,3.1101704E-1,3.8112104E-1,5.0140834E-1,6.766367E-1,7.613044E-1,2.6076472E-1,1.6201782E-1,0E0,0E0,4.5545673E-1,4.1525084E-1,0E0,2.4805948E-1,3.591054E-1,1.592854E-1,4.3620345E-1,1.5261304E-1,3.0847588E-1,6.171366E-1,1.9759923E-1,2.566828E-1,1.118837E-1,0E0,1.4549279E-1,1.1479141E-1,5.1649475E-1,4.9377394E-1,0E0,0E0,0E0,0E0,5.2238977E-1,2.8069422E-1,0E0,0E0,3.4418076E-1,3.3980227E-1,0E0,0E0,3.3897594E-1,0E0,5.7141125E-1,4.705001E-1,0E0,2.1039826E-1,3.350332E-1,3.0366218E-1,0E0,0E0,0E0,0E0,1.11651145E-1,1.9410613E-1,4.3680233E-1,5.870827E-1,1.9992173E-1,4.137878E-1,2.4003719E-1,2.0277059E-1,3.4064963E-1,2.7752355E-1,3.6675727E-1,2.3258983E-1,4.3042296E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,43,43,44,44,47,47,48,48,51,51,53,53,54,54,56,56,57,57,58,58,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,-1,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,-1,-1,-1,-1,70,72,-1,-1,74,76,-1,-1,78,-1,80,82,-1,84,86,88,-1,-1,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.832274E2,2E0,7.218466E7,5.1797544E7,2.831692E10,8.027061E2,2.074E3,7.298614E2,1.381E3,-1.6319931E-3,1.3888E4,1.8428E4,1E0,4.091015E9,2.7272727E0,7.4521235E6,2.2515285E6,8.10114E4,1.8066917E-2,6.932481E-3,5.862126E2,2.75E2,1.0892338E-2,5.8084745E6,8.01E2,2.18988E5,6.486406E6,8.931957E9,2.0136518E0,4.44E2,4.6E2,1.1075305E0,1.0804272E3,-2.0299643E-2,1.4285097E1,1.4302E4,4.5158855E6,5.077728E8,-2.4418661E-2,-2.1719413E-3,-1.2022199E-3,7.269195E-3,1.57383175E1,2.6540773E1,-8.469876E-4,-1.1202243E-2,6.587855E1,4.42384E5,1.1365007E-2,2.8103844E-3,4.7E2,-1.1781997E-2,2.3902743E0,4.352552E9,9.307029E-3,1.37E2,1.5548611E7,1.4771426E0,3.106797E-3,-7.3638717E-3,-4.8004016E-3,-1.6582068E-2,2.200423E3,1.7E1,1.3E1,3.324E3,4.8958065E6,5.3084288E8,1.22E2,3.0595E4,4.389587E1,2.23E2,7.21E3,6.1238285E8,6.520433E6,-1.1828854E-2,-3.6831878E-3,1.6655715E-4,-6.8794546E-4,-7.499469E-3,-1.20777665E-2,-1.0695722E-3,-4.0482474E-3,4.3491E-3,2.9804683E-3,-1.0347534E-3,-7.3436983E-3,4.9594096E-3,-1.6268764E-3,4.6895538E-3,-6.8528187E-3,-1.1055571E-3,1.0663287E-2,1.6770991E-3,-2.3005684E-3,-2.5462562E-2,1.25526935E-2,6.010716E-3,-9.869995E-3,3.304582E-3,1.5312432E-3,-4.182445E-3,-5.6287018E-3,-1.4832485E-2,3.1385345E-3,-4.0906045E-4,-8.837213E-3,-1.3406385E-3,9.185842E-3,2.5662754E-3,-1.7604264E-3,7.934366E-3,-5.892375E-4,3.169401E-3],"split_indices":[102,56,17,45,50,31,52,2,52,0,0,10,29,77,7,56,45,28,33,0,0,52,0,0,45,2,1,45,12,39,0,29,38,33,0,54,10,47,7,0,0,0,0,58,58,0,0,58,2,0,0,1,0,53,5,0,0,45,38,0,0,0,0,52,8,10,29,47,7,10,29,48,0,2,7,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E3,1.004E3,1.646E3,9.83E2,2.1E1,4.93E2,1.153E3,8.38E2,1.45E2,1.3E1,8E0,4.25E2,6.8E1,7.11E2,4.42E2,4.56E2,3.82E2,3.4E1,1.11E2,4E0,9E0,4.17E2,8E0,3.5E1,3.3E1,6.95E2,1.6E1,4.19E2,2.3E1,2.83E2,1.73E2,7.8E1,3.04E2,3.1E1,3E0,2.6E1,8.5E1,3.29E2,8.8E1,2E0,6E0,1.2E1,2.1E1,1.2E2,5.75E2,5E0,1.1E1,1.13E2,3.06E2,1.5E1,8E0,2.78E2,5E0,1.25E2,4.8E1,1.6E1,6.2E1,2.77E2,2.7E1,2E0,2.9E1,2.4E1,2E0,2.6E1,5.9E1,3.1E2,1.9E1,5.8E1,3E1,7E1,5E1,3.45E2,2.3E2,7.8E1,3.5E1,3.02E2,4E0,6.9E1,2.09E2,8.3E1,4.2E1,3.6E1,1.2E1,7E0,5.5E1,6E1,2.17E2,2.2E1,5E0,1.7E1,9E0,2E1,3.9E1,1.2E1,2.98E2,1.7E1,2E0,1.6E1,4.2E1,6E0,2.4E1,4.4E1,2.6E1,4.4E1,6E0,8E1,2.65E2,1.1E1,2.19E2,3.7E1,4.1E1,2.9E1,6E0,1.94E2,1.08E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-2.3375808E-3,-3.0542942E-2,1.5454932E-2,-3.4638602E-2,1.4939143E-1,-2.212469E-3,4.76343E-2,4.4114396E-2,-4.2260874E-2,1.16132265E-2,-5.5490103E-2,3.5243947E-2,-2.1806026E-2,8.734131E-2,2.2669258E-2,-4.7645573E-2,1.1664008E-1,-6.31152E-2,-8.571538E-3,1.4051355E-3,-9.896806E-3,1.4917633E-2,1.2757736E-1,7.2100144E-3,-4.1520588E-2,1.5163821E-1,1.2931002E-2,9.885526E-3,1.23094015E-1,-1.3774268E-2,-1.3504645E-2,1.3480528E-2,6.8391375E-2,-5.3653162E-2,-1.859632E-1,4.755254E-2,-3.7932757E-2,3.630101E-2,-1.1223769E-1,2.5926137E-1,7.1628995E-2,8.535104E-2,-2.6145602E-2,-8.877055E-2,-1.9345265E-2,1.666799E-1,-5.403987E-3,1.112712E-1,-4.778498E-2,2.9946636E-2,-2.4491461E-2,7.1088545E-2,1.5431758E-2,-6.500738E-3,1.2413923E-3,4.71735E-3,-1.5225871E-3,8.822755E-3,-7.768028E-2,-3.3398658E-1,-7.336484E-2,7.587133E-2,-3.7110776E-2,4.128978E-2,-5.6272432E-2,5.879385E-2,-5.7095513E-2,-6.568111E-2,-2.1691134E-2,1.411271E-2,2.525947E-3,-1.4946366E-3,4.801824E-3,4.702858E-2,9.157295E-3,-8.739121E-4,-9.137662E-2,-4.2899203E-2,-1.5763474E-1,-2.5695125E-2,6.6496413E-3,1.5048133E-2,1.426418E-1,2.0557644E-3,8.192295E-3,-2.1732806E-1,-1.300755E-2,2.0575918E-2,5.916967E-3,-4.396255E-3,-1.0878928E-5,5.6267856E-3,-1.5731198E-1,-5.337589E-3,2.2702196E-3,-8.587799E-4,-4.766917E-3,-1.7586932E-2,-2.7025172E-3,2.2699737E-3,-5.1549156E-3,5.361387E-3,8.4479264E-4,7.6859468E-3,-4.355861E-3,-1.902168E-4,7.1609295E-3,-1.3035922E-3,-5.0216704E-3,3.183361E-3,-8.092054E-3,-6.5388368E-3,-3.899639E-4,8.1785135E-3,-4.6733487E-3,-4.5870897E-6,5.073401E-3,6.1964325E-4,-5.727922E-3,4.2634456E-5,-6.9022705E-3,-3.6527151E-3,3.2690032E-3,-1.2348758E-3,-8.842099E-3,2.7422336E-4,-2.4829975E-3,8.011823E-3,2.0644777E-3,3.587738E-4,-1.9440027E-2,6.748865E-3,-1.8385946E-3,3.608036E-3,5.2507065E-5,3.499556E-3,-1.3925299E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,-1,-1,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,-1,-1,-1,-1,-1,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,115,-1,117,119,121,123,125,-1,-1,127,-1,-1,129,131,133,-1,-1,-1,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3298576E0,7.6054853E-1,9.2396E-1,6.039418E-1,4.5218682E-1,7.713682E-1,5.690013E-1,5.978484E-1,6.423093E-1,0E0,1.0865055E-1,6.75621E-1,3.9500594E-1,1.0601654E0,4.5453328E-1,3.1361765E-1,3.7842947E-1,6.4677453E-1,5.8020663E-1,0E0,0E0,8.116205E-1,4.6031713E-1,7.320353E-1,4.2892158E-1,4.9221492E-1,6.262825E-1,2.1865985E-1,3.9133924E-1,1.7781556E-1,0E0,0E0,1.2379941E-1,7.902726E-1,6.4218473E-1,2.929208E-1,3.3794692E-1,5.390489E-1,6.5229714E-1,1.2346053E-1,1.4154676E-1,3.3178645E-1,3.2467425E-1,4.0705252E-1,2.8583878E-1,3.5594106E-1,0E0,1.561153E-1,3.8043076E-1,1.7288755E-1,1.9105724E-1,3.682407E-1,0E0,0E0,0E0,0E0,0E0,6.8743205E-1,4.944737E-1,1.3614285E-1,1.044417E-1,1.8921226E-1,3.4442368E-1,2.161499E-1,2.625069E-1,3.7146688E-1,1.9333227E-1,3.1086656E-1,0E0,0E0,0E0,0E0,0E0,1.7490084E-1,0E0,2.3606525E-1,2.6781428E-1,3.0198303E-1,1.8033338E-1,2.2298668E-1,0E0,0E0,2.3571157E-1,0E0,0E0,4.6944743E-1,2.2031587E-1,1.9513503E-1,0E0,0E0,0E0,0E0,2.0573406E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,73,73,75,75,76,76,77,77,78,78,79,79,82,82,85,85,86,86,87,87,92,92],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,-1,-1,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,-1,-1,-1,-1,-1,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,116,-1,118,120,122,124,126,-1,-1,128,-1,-1,130,132,134,-1,-1,-1,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,8.791777E3,7.298614E2,1E0,1.3043794E8,2E0,5.2107143E0,1.91E2,3.83E3,1.16132265E-2,5.0738932E7,6.4547E4,9.969697E0,2.1351435E0,4.5575E4,3.09627E5,8E0,9.457831E0,2.3907686E7,1.4051355E-3,-9.896806E-3,1.604E3,1.5208336E4,1.03E2,1.246E3,3.410165E6,4.0208E4,2.2075728E7,4.516183E1,1.64E1,-1.3504645E-2,1.3480528E-2,1.3905958E7,8.2E1,2E0,4.8718534E0,5.473125E3,1E0,1.2936076E7,1E0,1.2045951E6,9.705292E6,1.4274633E7,5.372E3,2.8366232E0,7.696629E0,-5.403987E-3,3.15856E0,1.1660184E3,4.95E9,1.16072E6,1.3E2,1.5431758E-2,-6.500738E-3,1.2413923E-3,4.71735E-3,-1.5225871E-3,3.79E2,1.5756359E0,6E1,1.09E3,1.2471131E7,6.426937E6,2.2876644E-5,1.775894E10,1.4329944E0,1.9826213E0,2.8216E4,-2.1691134E-2,1.411271E-2,2.525947E-3,-1.4946366E-3,4.801824E-3,1.454213E2,9.157295E-3,1E0,2.64E0,8.2E2,5.28806E5,2.9206951E1,6.6496413E-3,1.5048133E-2,1.2817779E7,2.0557644E-3,8.192295E-3,7.792289E9,1.2236866E3,1.9E1,5.916967E-3,-4.396255E-3,-1.0878928E-5,5.6267856E-3,1.8215276E7,-5.337589E-3,2.2702196E-3,-8.587799E-4,-4.766917E-3,-1.7586932E-2,-2.7025172E-3,2.2699737E-3,-5.1549156E-3,5.361387E-3,8.4479264E-4,7.6859468E-3,-4.355861E-3,-1.902168E-4,7.1609295E-3,-1.3035922E-3,-5.0216704E-3,3.183361E-3,-8.092054E-3,-6.5388368E-3,-3.899639E-4,8.1785135E-3,-4.6733487E-3,-4.5870897E-6,5.073401E-3,6.1964325E-4,-5.727922E-3,4.2634456E-5,-6.9022705E-3,-3.6527151E-3,3.2690032E-3,-1.2348758E-3,-8.842099E-3,2.7422336E-4,-2.4829975E-3,8.011823E-3,2.0644777E-3,3.587738E-4,-1.9440027E-2,6.748865E-3,-1.8385946E-3,3.608036E-3,5.2507065E-5,3.499556E-3,-1.3925299E-2],"split_indices":[102,52,52,79,50,17,53,2,2,0,1,1,58,41,2,9,18,54,45,0,0,2,33,11,2,11,9,45,54,4,0,0,45,0,17,53,32,105,45,106,47,9,45,29,38,56,0,38,4,5,1,8,0,0,0,0,0,1,54,3,2,51,45,39,5,42,56,9,0,0,0,0,0,52,0,77,54,10,45,58,0,0,9,0,0,31,52,8,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.648E3,1.024E3,1.624E3,1.002E3,2.2E1,1.049E3,5.75E2,8.8E1,9.14E2,1.5E1,7E0,3.6E2,6.89E2,2.21E2,3.54E2,3.9E1,4.9E1,5.64E2,3.5E2,5E0,2E0,2.96E2,6.4E1,2.79E2,4.1E2,1.18E2,1.03E2,3.15E2,3.9E1,3.5E1,4E0,1E1,3.9E1,5.25E2,3.9E1,1.2E2,2.3E2,2.54E2,4.2E1,1.8E1,4.6E1,8.3E1,1.96E2,1.3E2,2.8E2,1.12E2,6E0,3.9E1,6.4E1,1.99E2,1.16E2,3.2E1,7E0,8E0,2.7E1,3E1,9E0,1.46E2,3.79E2,1.6E1,2.3E1,9E1,3E1,4.3E1,1.87E2,2.05E2,4.9E1,3.8E1,4E0,1.5E1,3E0,1E1,3.6E1,6.2E1,2.1E1,1.42E2,5.4E1,7.9E1,5.1E1,2.7E2,1E1,1.4E1,9.8E1,1.9E1,2E1,1E1,5.4E1,1.82E2,1.7E1,3E1,8.6E1,2.7E1,5E0,3.5E1,1.11E2,1.02E2,2.77E2,1.4E1,2E0,5E0,1.8E1,5.5E1,3.5E1,6E0,2.4E1,3.1E1,1.2E1,1.18E2,6.9E1,1.99E2,6E0,1.8E1,3.1E1,4E0,3.4E1,3.5E1,2.7E1,1.28E2,1.4E1,2E1,3.4E1,6.1E1,1.8E1,9E0,4.2E1,1.23E2,1.47E2,7.8E1,2E1,5E0,5E0,7E0,4.7E1,4.7E1,1.35E2,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-1.3486892E-3,-1.0264289E-2,5.7059854E-2,-4.4193747E-3,-9.535855E-2,8.872511E-2,9.160142E-3,-3.7424985E-2,8.649954E-3,-1.3783325E-1,-5.385191E-2,7.107852E-2,2.7594775E-1,3.1914353E-2,-8.4621795E-2,-1.2733996E-2,-8.871857E-2,2.1366874E-2,-2.1789758E-2,-1.1518908E-1,-2.6897258E-1,1.8833174E-3,-7.70291E-2,1.4325248E-1,3.417348E-2,8.576038E-3,2.106444E-2,2.9794497E-3,1.2611035E-1,-2.9318685E-2,-2.4634275E-1,-3.5267533E-3,-1.1976595E-1,-2.2989541E-1,-7.087159E-2,1.8671693E-2,1.9165023E-1,-3.910709E-2,6.463009E-2,-1.2559208E-1,3.460283E-3,-1.9784417E-2,-2.0211805E-3,-2.7956371E-3,-9.369476E-3,4.57036E-2,1.943669E-1,4.1260477E-2,-7.311492E-3,4.2258464E-3,-3.1704076E-2,-6.369279E-3,1.6404025E-1,5.152626E-3,-3.147392E-3,3.9091977E-4,-1.6751032E-2,-4.0570132E-2,2.8190786E-2,4.7966214E-3,-1.723475E-1,-3.59599E-1,-3.8291205E-2,-1.9988716E-1,-5.3386066E-2,3.1300582E-2,-1.961302E-2,2.2761813E-1,-2.4937887E-3,-1.5878223E-2,-3.514579E-2,-1.0590588E-2,9.686465E-2,-3.8138982E-3,-1.6749498E-1,4.418951E-3,-2.823961E-3,5.3103287E-3,1.2252159E-2,4.6575665E-2,-9.288807E-3,-3.002236E-3,1.6631221E-3,1.3519164E-2,5.217181E-3,7.4645504E-5,-4.30587E-3,4.2583463E-3,-9.0419117E-4,-3.3690054E-3,-1.9434111E-2,-8.119415E-3,-2.1895196E-2,-8.592947E-3,4.1044788E-3,2.9937704E-3,-1.2705624E-2,4.400748E-3,-3.1035289E-3,-5.612811E-3,1.6760298E-3,-3.815743E-3,1.5152643E-4,2.989912E-3,1.3505614E-2,-2.2327374E-3,2.063714E-3,7.4128984E-3,-4.204454E-3,2.7944434E-3,7.889506E-3,-4.35487E-3,-1.1642092E-2,1.3756546E-3,5.0130417E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,-1,-1,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,-1,-1,-1,-1,75,77,79,-1,-1,81,-1,83,-1,-1,-1,-1,85,87,-1,89,91,93,95,97,99,101,103,-1,-1,105,107,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3920223E0,1.1529766E0,5.361191E-1,9.368087E-1,2.5526166E-1,6.841043E-1,3.0571926E-1,7.790277E-1,6.0279226E-1,1.8579376E-1,1.6931768E-1,5.1716673E-1,2.0702195E-1,3.1306273E-1,2.4320038E-1,4.105183E-1,4.8762286E-1,5.003423E-1,6.894492E-1,1.3387269E-1,3.0943978E-1,0E0,1.2575915E-1,3.2020295E-1,1.7976497E-1,0E0,0E0,2.6598176E-1,2.8717008E-1,1.1671444E-1,1.8712819E-1,4.5348978E-1,3.9887214E-1,5.329691E-1,3.931359E-1,5.2375376E-1,1.6087812E-1,4.3203914E-1,1.8772817E-1,1.0412359E-1,0E0,0E0,0E0,0E0,0E0,1.22424945E-1,1.922921E-1,1.678138E-1,0E0,0E0,1.3391566E-1,0E0,1.2562609E-1,0E0,0E0,0E0,0E0,3.7014115E-1,5.9625006E-1,0E0,6.14385E-1,1.6361654E-1,1.9038406E-1,3.770663E-1,2.697146E-1,4.5093662E-1,3.6826143E-1,1.1716527E-1,0E0,0E0,3.4229505E-1,3.2001305E-1,1.3155234E-1,0E0,1.5732998E-1,0E0,0E0,0E0,0E0,1.2663692E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,45,45,46,46,47,47,50,50,52,52,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,74,74,79,79],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,-1,-1,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,-1,-1,-1,-1,76,78,80,-1,-1,82,-1,84,-1,-1,-1,-1,86,88,-1,90,92,94,96,98,100,102,104,-1,-1,106,108,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,6.747114E7,3.2475834E7,1.8122449E2,1.067536E3,8.791777E3,1.56E2,2.5074627E0,9.669789E0,4.1043E4,1.3E1,2E1,1.341E3,1.5005797E4,1.477425E6,1.2471935E7,8E0,1.3238013E2,7.06699E8,3.582157E12,3.138649E6,1.8833174E-3,4.4967708E7,2.6838362E-1,9.70107E7,8.576038E-3,2.106444E-2,1.10032365E-1,2.7258065E0,1E0,8.015419E3,4.26E2,3.3820656E7,2.6138031E0,1.141E4,1.25218E5,1.1953147E2,3.415965E2,6.1099392E1,2.8277853E11,3.460283E-3,-1.9784417E-2,-2.0211805E-3,-2.7956371E-3,-9.369476E-3,1.2804922E3,2.0718E4,1.4299594E8,-7.311492E-3,4.2258464E-3,5.2E1,-6.369279E-3,1.0317661E5,5.152626E-3,-3.147392E-3,3.9091977E-4,-1.6751032E-2,2.772925E3,1.75E2,4.7966214E-3,2.2214102E6,5.1086234E5,9.5751E4,4.580013E-3,1.0449754E0,2.402216E6,9.51668E5,2.4042394E5,-2.4937887E-3,-1.5878223E-2,3.307766E6,1.4663157E1,1.5435694E9,-3.8138982E-3,8.8070023E-1,4.418951E-3,-2.823961E-3,5.3103287E-3,1.2252159E-2,7.9684106E9,-9.288807E-3,-3.002236E-3,1.6631221E-3,1.3519164E-2,5.217181E-3,7.4645504E-5,-4.30587E-3,4.2583463E-3,-9.0419117E-4,-3.3690054E-3,-1.9434111E-2,-8.119415E-3,-2.1895196E-2,-8.592947E-3,4.1044788E-3,2.9937704E-3,-1.2705624E-2,4.400748E-3,-3.1035289E-3,-5.612811E-3,1.6760298E-3,-3.815743E-3,1.5152643E-4,2.989912E-3,1.3505614E-2,-2.2327374E-3,2.063714E-3,7.4128984E-3,-4.204454E-3,2.7944434E-3,7.889506E-3,-4.35487E-3,-1.1642092E-2,1.3756546E-3,5.0130417E-3],"split_indices":[2,45,50,52,52,52,8,56,54,10,3,3,0,4,9,45,3,56,7,31,29,0,50,41,1,0,0,57,54,64,4,2,7,54,9,29,58,4,54,31,0,0,0,0,0,52,2,45,0,0,8,0,28,0,0,0,0,50,11,0,28,51,9,58,53,7,1,28,0,0,1,54,7,0,27,0,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.671E3,2.318E3,3.53E2,2.17E3,1.48E2,2.12E2,1.41E2,6.15E2,1.555E3,7.2E1,7.6E1,1.95E2,1.7E1,1.14E2,2.7E1,4.16E2,1.99E2,1.097E3,4.58E2,6.3E1,9E0,1.5E1,6.1E1,6.5E1,1.3E2,1.2E1,5E0,8.8E1,2.6E1,2.1E1,6E0,3.84E2,3.2E1,2.1E1,1.78E2,1.081E3,1.6E1,3.82E2,7.6E1,6E1,3E0,5E0,4E0,5.4E1,7E0,2.3E1,4.2E1,1.26E2,4E0,2.5E1,6.3E1,3E0,2.3E1,4E0,1.7E1,2E0,4E0,1.77E2,2.07E2,6E0,2.6E1,1.2E1,9E0,2E1,1.58E2,8.13E2,2.68E2,1.4E1,2E0,4E0,3.78E2,2.3E1,5.3E1,3E1,3E1,1.6E1,7E0,1.9E1,2.3E1,1.24E2,2E0,4.3E1,2E1,6E0,1.7E1,9.6E1,8.1E1,9E1,1.17E2,1.9E1,7E0,5E0,7E0,4E0,5E0,4E0,1.6E1,1.1E1,1.47E2,1.9E1,7.94E2,7.3E1,1.95E2,4E0,1E1,3.3E2,4.8E1,7E0,1.6E1,3.5E1,1.8E1,1.6E1,1.4E1,9.6E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.981525E-3,-1.1756037E-2,5.9991084E-2,-5.6886408E-3,-1.082216E-1,1.3377568E-1,3.93587E-2,-1.9043704E-2,2.6035812E-2,-8.7427296E-2,-1.9871478E-1,1.4802474E-1,-3.1911135E-3,4.986983E-2,-6.8883516E-2,1.6252391E-2,-3.573239E-2,4.837493E-2,-2.110669E-2,-1.2102827E-2,-7.9119265E-2,-1.5646175E-2,-5.090761E-3,1.069153E-1,1.15991505E-2,3.5106856E-2,5.1066293E-3,-1.1131903E-3,-1.2256078E-2,3.7120417E-4,8.710284E-2,-4.4259466E-2,2.2390107E-2,1.0691169E-1,1.4455603E-2,-3.291213E-2,5.500505E-3,-9.315754E-3,-3.3230195E-3,2.8556147E-1,7.4232645E-2,4.3557014E-2,-5.257671E-2,8.06967E-3,-1.3471876E-1,5.5238385E-2,2.0494507E-1,-4.277222E-2,-1.4687151E-2,1.14483394E-1,-1.0987378E-2,1.3169283E-1,-1.3302945E-3,5.0503258E-2,-3.725445E-2,-9.326146E-2,-1.2982337E-2,1.7302072E-2,1.864368E-3,-2.7128668E-3,4.9103266E-3,2.649916E-2,1.3535473E-1,3.481158E-2,-8.480794E-3,2.2465116E-3,-1.119031E-3,-1.2480057E-3,-1.7424354E-2,6.42248E-3,1.5578313E-3,1.535914E-2,6.074591E-3,-1.2592523E-4,-2.5111728E-3,7.412858E-3,1.0045597E-3,2.2970563E-4,-4.8249215E-3,2.4704884E-3,7.863372E-3,3.7189212E-3,-2.563152E-6,-1.0300847E-2,-5.1418657E-4,-1.6115671E-2,-3.6095078E-3,1.0948219E-4,-4.028896E-3,2.9456173E-3,8.065669E-5,-9.803738E-4,8.388705E-3,-2.2566302E-3,1.0078854E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,-1,41,-1,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,57,59,61,63,65,67,69,71,73,-1,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6362358E0,1.3648772E0,5.5496037E-1,9.309017E-1,2.4230754E-1,2.3831475E-1,3.3224446E-1,9.112449E-1,6.866034E-1,1.3655502E-1,2.6098073E-1,2.6447093E-1,0E0,2.1748006E-1,2.1598998E-1,5.591867E-1,5.213866E-1,8.760576E-1,3.4085047E-1,0E0,1.1003637E-1,0E0,0E0,2.9408813E-1,0E0,1.5862843E-1,0E0,0E0,0E0,4.2436123E-1,3.303823E-1,3.443855E-1,4.1622657E-1,5.4437494E-1,5.256728E-1,2.3212266E-1,0E0,0E0,0E0,1.3533938E-1,1.7988935E-1,2.986265E-1,2.1289887E-1,4.7238928E-1,5.382266E-1,1.2638566E-1,1.311124E-1,3.5364342E-1,0E0,1.3242736E-1,1.4216156E-1,3.4822512E-1,0E0,2.2786993E-1,5.447277E-1,1.8353757E-1,1.5987402E-1,0E0,0E0,0E0,0E0,1.4236742E-1,1.89686E-1,1.8260802E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,25,25,29,29,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,53,53,54,54,55,55,56,56,61,61,62,62,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,-1,42,-1,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,58,60,62,64,66,68,70,72,74,-1,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,6.747114E7,1.2427981E9,5.862126E2,1.31496E5,4.5481584E7,3.0272608E10,2E0,1.0238709E1,5.815045E2,1E0,2.3292E4,-3.1911135E-3,4.328683E0,1.0828989E8,7.0025E4,6.808926E6,8.959817E7,4.4854636E7,-1.2102827E-2,3.5315048E2,-1.5646175E-2,-5.090761E-3,7.969174E7,1.15991505E-2,1.2367184E3,5.1066293E-3,-1.1131903E-3,-1.2256078E-2,4.5158855E6,4.681779E2,7.780377E6,2.25E2,3.5301748E7,5.8805E4,7.44393E5,5.500505E-3,-9.315754E-3,-3.3230195E-3,5.8E1,7.398769E4,9.3058E4,1.4858E4,1.4702222E0,6.4166665E0,1.4103535E0,1.6524802E6,1.704E3,-1.4687151E-2,1.9071268E3,1.0950326E6,1.954E3,-1.3302945E-3,1.6837112E7,3.564E3,5.542E3,1.9099288E7,1.7302072E-2,1.864368E-3,-2.7128668E-3,4.9103266E-3,1.6E1,1.36882E5,3.5791788E0,-8.480794E-3,2.2465116E-3,-1.119031E-3,-1.2480057E-3,-1.7424354E-2,6.42248E-3,1.5578313E-3,1.535914E-2,6.074591E-3,-1.2592523E-4,-2.5111728E-3,7.412858E-3,1.0045597E-3,2.2970563E-4,-4.8249215E-3,2.4704884E-3,7.863372E-3,3.7189212E-3,-2.563152E-6,-1.0300847E-2,-5.1418657E-4,-1.6115671E-2,-3.6095078E-3,1.0948219E-4,-4.028896E-3,2.9456173E-3,8.065669E-5,-9.803738E-4,8.388705E-3,-2.2566302E-3,1.0078854E-2],"split_indices":[2,45,12,52,11,47,5,17,53,4,67,2,0,35,45,1,9,12,9,0,33,0,0,7,0,58,0,0,0,47,55,28,10,45,9,1,0,0,0,3,33,2,9,53,56,56,47,29,0,48,47,2,0,45,2,29,9,0,0,0,0,18,11,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.699E3,2.332E3,3.67E2,2.195E3,1.37E2,7.9E1,2.88E2,1.545E3,6.5E2,1.13E2,2.4E1,7.4E1,5E0,2.63E2,2.5E1,4.96E2,1.049E3,4.41E2,2.09E2,4E0,1.09E2,9E0,1.5E1,5.3E1,2.1E1,2.1E2,5.3E1,2.1E1,4E0,4.06E2,9E1,9.15E2,1.34E2,1.61E2,2.8E2,1.93E2,1.6E1,7E0,1.02E2,7E0,4.6E1,1.92E2,1.8E1,3.85E2,2.1E1,7.2E1,1.8E1,9.11E2,4E0,3.5E1,9.9E1,1.36E2,2.5E1,1.65E2,1.15E2,4.7E1,1.46E2,5E0,2E0,8E0,3.8E1,1.63E2,2.9E1,1.1E1,7E0,1.72E2,2.13E2,1.5E1,6E0,1.5E1,5.7E1,6E0,1.2E1,1.77E2,7.34E2,2.4E1,1.1E1,8.5E1,1.4E1,4E1,9.6E1,1.07E2,5.8E1,1.4E1,1.01E2,2E0,4.5E1,1.21E2,2.5E1,6.7E1,9.6E1,6E0,2.3E1,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-2.1274015E-3,3.3004908E-3,-7.89385E-2,-1.644372E-2,3.401707E-2,-1.4766493E-1,-4.390631E-2,1.4348372E-2,-3.0972628E-2,4.9982652E-2,-1.5085784E-2,-1.14800654E-1,-2.972198E-1,-2.487709E-2,-1.5549889E-1,2.5526572E-2,-7.391543E-2,-3.861111E-2,2.5729429E-2,9.680996E-2,2.8908765E-2,-1.587504E-1,3.1106195E-3,-6.9739427E-3,-1.5887533E-3,-1.9633874E-2,-1.2257602E-3,-2.90397E-3,2.3639968E-2,-3.6622456E-3,-1.377513E-2,2.9122543E-2,-1.126579E-2,1.1143223E-1,-1.0783486E-1,-4.4243813E-2,4.328787E-2,9.994241E-2,-6.128154E-3,8.7163515E-2,1.8625112E-2,-1.29254805E-2,5.2835498E-2,-3.4343138E-1,-5.349381E-3,-6.166045E-3,2.1509755E-1,-6.401259E-3,3.841015E-2,5.668741E-2,-1.1391798E-3,-1.6822115E-3,9.126298E-3,-1.705108E-1,1.2799782E-2,-2.356119E-2,-6.482257E-2,2.0655436E-2,7.285168E-3,7.407709E-3,-2.6984276E-2,-9.260876E-2,2.4566954E-2,1.114113E-1,-5.118009E-4,-7.766E-3,-4.4210437E-3,8.394575E-2,8.659322E-3,-4.870825E-3,-2.2541031E-2,4.992273E-2,-3.1797063E-2,2.0142922E-3,1.6527753E-2,8.829968E-3,8.451066E-4,3.2331508E-3,-2.734144E-3,-1.7703656E-3,4.4315876E-3,-3.2963266E-3,-1.3140261E-2,6.8237297E-3,-3.395028E-3,-9.83119E-4,-1.6205305E-2,-6.7488733E-3,-2.2422266E-3,3.0329938E-3,-2.8127108E-3,-4.574992E-3,4.2184885E-3,-9.919357E-4,-9.680875E-3,1.8741095E-3,-7.569317E-3,9.166963E-3,4.265126E-3,-1.977096E-6,-1.23765115E-2,7.180333E-3,1.5543675E-3,-8.053363E-3,1.3830452E-3,3.8049025E-3,-7.5318565E-4,-1.8431812E-3,3.416862E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,69,-1,71,73,-1,75,77,79,-1,-1,81,83,85,87,89,-1,-1,91,93,95,97,-1,-1,99,101,103,-1,-1,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1117755E0,1.5112877E0,4.1726077E-1,6.7911303E-1,7.6511085E-1,2.5586474E-1,2.4670695E-1,4.823758E-1,4.4753772E-1,7.2351086E-1,6.278968E-1,1.209625E-1,2.9577494E-1,1.7840323E-1,1.5552884E-1,4.128794E-1,3.577951E-1,4.2124808E-1,2.9141474E-1,6.085429E-1,5.1017E-1,1.962493E-1,4.2216405E-1,0E0,0E0,0E0,0E0,0E0,1.0736427E-1,0E0,0E0,3.5708523E-1,0E0,1.1107306E-1,3.5754436E-1,3.6067784E-1,1.4218456E-1,2.6123744E-1,2.3321933E-1,5.2792025E-1,0E0,2.3534833E-1,4.445566E-1,1.0592145E-1,0E0,2.9773557E-1,1.8796545E-1,0E0,1.21009514E-1,2.797165E-1,6.9143397E-1,0E0,0E0,3.0852294E-1,1.9461212E-1,3.997844E-1,5.770284E-1,1.7311811E-1,0E0,0E0,1.030661E-1,1.76945E-1,1.7909223E-1,3.0125093E-1,0E0,0E0,1.9690412E-1,6.40674E-1,4.8752475E-1,0E0,0E0,1.2724966E-1,1.00735724E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,28,28,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,63,63,66,66,67,67,68,68,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,70,-1,72,74,-1,76,78,80,-1,-1,82,84,86,88,90,-1,-1,92,94,96,98,-1,-1,100,102,104,-1,-1,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,5.5871747E2,1.067536E3,2E0,1.0658642E1,3.9516E4,4.03424E5,2E0,7.997723E6,4.7931265E6,2.47E2,6.2408745E-3,3.138649E6,1.1879E4,1.4299594E8,4.6E1,3.5E1,2.852738E6,2.25E2,2.3274304E2,1E0,6.203E3,2.0136518E0,-6.9739427E-3,-1.5887533E-3,-1.9633874E-2,-1.2257602E-3,-2.90397E-3,1.8736842E0,-3.6622456E-3,-1.377513E-2,1E0,-1.126579E-2,2E0,3.449702E2,2.9676E4,4.8632206E5,1.9071268E3,1.7247218E-1,3.5301748E7,1.8625112E-2,2.7165637E4,2.1871264E0,4.5E1,-5.349381E-3,4.514673E-3,3.2917362E7,-6.401259E-3,3.307766E6,1E0,4.274E3,-1.6822115E-3,9.126298E-3,5.5E1,7.122057E8,6.6315195E6,1.2866874E3,7.8E1,7.285168E-3,7.407709E-3,4.116997E6,2.9005406E5,1.460806E6,9.42812E3,-5.118009E-4,-7.766E-3,9.6615314E-1,2.7203197E0,9.22E2,-4.870825E-3,-2.2541031E-2,2.773036E1,4.4854636E7,2.0142922E-3,1.6527753E-2,8.829968E-3,8.451066E-4,3.2331508E-3,-2.734144E-3,-1.7703656E-3,4.4315876E-3,-3.2963266E-3,-1.3140261E-2,6.8237297E-3,-3.395028E-3,-9.83119E-4,-1.6205305E-2,-6.7488733E-3,-2.2422266E-3,3.0329938E-3,-2.8127108E-3,-4.574992E-3,4.2184885E-3,-9.919357E-4,-9.680875E-3,1.8741095E-3,-7.569317E-3,9.166963E-3,4.265126E-3,-1.977096E-6,-1.23765115E-2,7.180333E-3,1.5543675E-3,-8.053363E-3,1.3830452E-3,3.8049025E-3,-7.5318565E-4,-1.8431812E-3,3.416862E-3],"split_indices":[45,52,52,17,53,10,11,6,9,51,0,38,29,2,45,3,0,1,10,56,102,29,39,0,0,0,0,0,53,0,0,16,0,8,52,29,28,48,41,45,0,33,42,3,0,57,1,0,1,74,29,0,0,0,31,28,48,8,0,0,12,33,47,47,0,0,27,53,10,0,0,58,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.665E3,2.49E3,1.75E2,1.516E3,9.74E2,5.8E1,1.17E2,4.86E2,1.03E3,7.35E2,2.39E2,4.9E1,9E0,1.01E2,1.6E1,4.32E2,5.4E1,9.08E2,1.22E2,2.27E2,5.08E2,2.6E1,2.13E2,3.5E1,1.4E1,6E0,3E0,5.8E1,4.3E1,1.1E1,5E0,4.27E2,5E0,8E0,4.6E1,8.5E2,5.8E1,3.6E1,8.6E1,2.21E2,6E0,1.85E2,3.23E2,4E0,2.2E1,2.05E2,8E0,3E0,4E1,2.23E2,2.04E2,3E0,5E0,3E1,1.6E1,4.25E2,4.25E2,4.9E1,9E0,2.5E1,1.1E1,2.2E1,6.4E1,1.77E2,4.4E1,9E0,1.76E2,1.89E2,1.34E2,2E0,2E0,6.4E1,1.41E2,4E0,4E0,4E0,3.6E1,2.04E2,1.9E1,1.48E2,5.6E1,1.6E1,1.4E1,6E0,1E1,4.22E2,3E0,8E1,3.45E2,3.2E1,1.7E1,7E0,4E0,1.4E1,8E0,6E1,4E0,3.7E1,1.4E2,1.74E2,2E0,8.2E1,1.07E2,1.3E1,1.21E2,4.4E1,2E1,1.33E2,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-9.0413727E-4,9.985263E-3,-3.600035E-2,6.0698995E-3,1.0957025E-1,-9.310574E-2,-1.819122E-2,3.3371456E-3,1.2788582E-1,2.2228993E-1,5.7443533E-2,-2.0946832E-1,-7.25392E-2,-8.529567E-2,-1.0761851E-2,-3.0970782E-2,1.2080123E-2,1.8869624E-3,7.5222133E-3,2.4988648E-1,1.0457162E-3,7.007239E-2,-8.649244E-3,-5.283069E-3,-3.1742015E-1,-1.3371855E-1,-3.929674E-2,3.8473662E-3,-1.2043056E-1,-6.0931145E-4,-5.6633342E-2,-3.0745827E-3,-7.55842E-2,1.1960138E-2,1.0352274E-2,8.329059E-3,1.967437E-2,1.2646501E-3,5.563665E-3,-1.7710485E-2,-3.4434453E-3,-6.491688E-2,-2.669941E-1,-8.699467E-3,-3.6539037E-3,-1.1770845E-2,-3.8988045E-3,-3.8455743E-2,1.281776E-2,-8.789539E-2,6.900316E-4,-1.3060038E-1,1.1631498E-2,-5.8017585E-2,-4.2615685E-1,1.23154465E-2,-1.4538646E-1,2.7375005E-3,-1.0986425E-1,-1.6228132E-2,-2.6575814E-3,-1.5061362E-1,1.5367574E-2,-5.4245394E-2,2.6648925E-3,5.7670258E-2,-4.5358995E-3,-1.4851986E-1,-1.3610998E-3,-3.2045601E-3,-1.050901E-2,2.1047816E-3,-1.4527445E-3,-3.423002E-3,3.45073E-3,5.445721E-4,-2.8899176E-2,1.994154E-3,2.2777833E-5,7.1992464E-3,-1.0067941E-2,-6.67738E-3,3.1335156E-3,-2.4775513E-3,-1.4279475E-2,2.040965E-3,-3.8841204E-3,-1.214641E-3,-1.12896E-2,-7.7077965E-3,3.3127998E-3,-6.951502E-3,7.515945E-5,-3.9557123E-4,-9.443911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,-1,39,41,43,-1,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,57,59,-1,61,-1,-1,63,65,67,-1,69,71,73,75,77,79,-1,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0200069E0,7.9358464E-1,6.4104843E-1,6.5237623E-1,4.396602E-1,3.437816E-1,2.398787E-1,5.759847E-1,1.0485554E-1,1.2730801E-1,1.7499414E-1,2.0046556E-1,2.5709927E-1,2.890096E-1,2.03016E-1,4.8514017E-1,6.256915E-1,0E0,0E0,1.8529117E-1,0E0,1.01611674E-1,0E0,0E0,1.0204041E-1,3.9529872E-1,4.3062514E-1,0E0,1.7564493E-1,1.824166E-1,1.7671868E-1,4.5349163E-1,8.990316E-1,0E0,4.665181E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7638035E-1,2.091881E-1,0E0,1.9519654E-1,0E0,0E0,1.426861E-1,2.0652564E-1,1.9408685E-1,0E0,1.2675977E-1,2.9564545E-1,2.5722283E-1,5.498333E-1,5.21468E-1,3.9592013E-1,0E0,1.2832597E-1,0E0,0E0,1.0129687E-1,1.6644372E-1,4.1861936E-1,0E0,1.9724318E-1,1.6506281E-1,1.8271565E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,21,21,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,41,41,42,42,44,44,47,47,48,48,49,49,51,51,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,65,65,66,66,67,67],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,-1,40,42,44,-1,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,58,60,-1,62,-1,-1,64,66,68,-1,70,72,74,76,78,80,-1,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.759343E-1,1.5137865E2,1.9121015E0,7.780377E6,9.640999E6,8.230099E4,5.2143492E-5,1.5288235E2,1E0,2.558106E10,1.1096592E8,7.5408E4,1.05E2,1E0,8.229907E7,4.1E0,7.330957E4,1.8869624E-3,7.5222133E-3,2.9799202E3,1.0457162E-3,6.5317163E3,-8.649244E-3,-5.283069E-3,9.334426E-1,6.6911E4,1.2068E4,3.8473662E-3,5.987692E2,3.891716E8,1.4843444E1,8.333333E0,2.7056329E10,1.1960138E-2,4.2620764E2,8.329059E-3,1.967437E-2,1.2646501E-3,5.563665E-3,-1.7710485E-2,-3.4434453E-3,2.7772428E5,1.5153E4,-8.699467E-3,7E0,-1.1770845E-2,-3.8988045E-3,1.3741055E0,1.15062E5,4.3597875E0,6.900316E-4,6.7E1,1.1846624E-5,1.6713532E9,3.4893394E-4,1.5277338E0,2.2536E4,2.7375005E-3,2.220153E6,-1.6228132E-2,-2.6575814E-3,2.080593E0,2.109E3,8.7643677E-1,2.6648925E-3,2.5757682E-1,4.2647058E-1,2.5652175E0,-1.3610998E-3,-3.2045601E-3,-1.050901E-2,2.1047816E-3,-1.4527445E-3,-3.423002E-3,3.45073E-3,5.445721E-4,-2.8899176E-2,1.994154E-3,2.2777833E-5,7.1992464E-3,-1.0067941E-2,-6.67738E-3,3.1335156E-3,-2.4775513E-3,-1.4279475E-2,2.040965E-3,-3.8841204E-3,-1.214641E-3,-1.12896E-2,-7.7077965E-3,3.3127998E-3,-6.951502E-3,7.515945E-5,-3.9557123E-4,-9.443911E-3],"split_indices":[27,56,53,28,51,32,42,52,102,19,43,1,52,67,45,58,45,0,0,4,0,4,0,0,27,29,9,0,4,5,57,52,31,0,58,0,0,0,0,0,0,28,10,0,3,0,0,57,29,54,0,29,39,31,38,53,9,0,51,0,0,54,0,57,0,27,58,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.667E3,2.036E3,6.31E2,1.96E3,7.6E1,1.49E2,4.82E2,1.918E3,4.2E1,2.3E1,5.3E1,2.1E1,1.28E2,4.7E1,4.35E2,3.89E2,1.529E3,1.1E1,3.1E1,2E1,3E0,5.1E1,2E0,1.2E1,9E0,4.4E1,8.4E1,8E0,3.9E1,3.57E2,7.8E1,2.4E2,1.49E2,1E1,1.519E3,1.5E1,5E0,2.7E1,2.4E1,7E0,2E0,3E1,1.4E1,1.6E1,6.8E1,8E0,3.1E1,9.3E1,2.64E2,5.4E1,2.4E1,2.4E1,2.16E2,1.43E2,6E0,1.501E3,1.8E1,8E0,2.2E1,1E1,4E0,7E0,6.1E1,8E1,1.3E1,7.3E1,1.91E2,2.6E1,2.8E1,1.5E1,9E0,1.22E2,9.4E1,1.3E2,1.3E1,2E0,4E0,4.31E2,1.07E3,3E0,1.5E1,1.9E1,3E0,5E0,2E0,4.8E1,1.3E1,7E1,1E1,3E0,7E1,7E0,1.84E2,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-3.7045596E-4,-3.6905535E-2,1.074292E-2,-1.5716402E-2,-7.9719685E-2,2.3399413E-2,-3.266609E-2,-1.0545039E-2,-2.266878E-1,-4.3506805E-2,-1.4847243E-1,9.215458E-3,5.6995597E-2,-8.137936E-2,-7.570754E-3,-1.3643672E-3,-1.2303829E-1,-5.394868E-3,-1.7699482E-2,6.037914E-3,-5.7852626E-2,-1.8682328E-1,1.772061E-3,2.3320531E-2,-2.6835205E-2,1.7143126E-1,4.6604823E-2,-1.176125E-2,-7.098443E-2,-2.1522405E-2,6.3156635E-2,1.7024914E-2,-5.931575E-2,4.275471E-3,-1.7756474E-1,5.974491E-3,-6.793847E-2,-1.7615017E-2,-1.468878E-1,2.0309126E-2,1.8096879E-1,-1.9845191E-2,-1.5665765E-1,1.0837881E-2,5.353572E-4,1.255252E-1,3.413027E-2,1.7568696E-3,-8.2337804E-2,-1.4919146E-2,-2.025549E-1,-6.1760414E-3,3.7283131E-3,-6.618896E-2,4.0882777E-2,-1.5164492E-1,-3.3207543E-2,-4.1114238E-1,-8.21643E-2,-1.2700996E-2,-5.1625438E-2,3.3750876E-3,-7.538584E-3,1.3424848E-1,1.642536E-2,1.4490446E-2,3.1741408E-3,-2.5601132E-2,5.7904683E-3,-1.7578412E-2,-4.8073903E-3,9.058901E-2,2.5480953E-1,8.0738366E-2,2.2288065E-2,-6.885432E-2,-2.0461586E-1,9.880613E-3,-1.9355161E-2,-1.3959523E-2,-8.867553E-4,3.1658085E-3,-4.2445837E-3,5.332567E-3,9.187295E-4,4.246692E-4,-9.85301E-3,-5.718539E-4,-6.719911E-3,-2.8425375E-2,-6.0318247E-3,-9.624904E-3,-5.683564E-4,-7.2848407E-4,-5.8950796E-3,7.933593E-3,-3.8281565E-3,1.7544639E-4,2.3679947E-3,-5.91567E-3,-8.7543053E-4,2.8594022E-3,1.13757E-2,1.4902215E-2,3.078912E-3,6.3625356E-4,5.1708897E-3,-4.047322E-3,1.4837121E-3,-3.899793E-3,3.958989E-3,-1.4766823E-2,-3.4316233E-3,-1.1583843E-3,7.141098E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,47,49,51,53,55,-1,57,-1,59,-1,61,63,65,67,69,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,85,87,89,91,-1,93,-1,-1,95,97,-1,-1,99,-1,-1,-1,101,103,105,107,109,111,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0897697E0,5.6665105E-1,1.1314414E0,4.5583373E-1,5.089154E-1,7.590585E-1,5.6783533E-1,4.24298E-1,1.2132001E-1,3.4059092E-1,5.1194394E-1,5.711598E-1,5.54443E-1,2.522602E-1,3.0528256E-1,4.0703553E-1,3.7473115E-1,0E0,0E0,0E0,2.4237558E-1,3.7243128E-1,0E0,3.7944213E-1,2.8364867E-1,3.404243E-1,4.2397177E-1,0E0,1.8638241E-1,3.053924E-1,1.540129E-1,5.784445E-1,2.167348E-1,0E0,5.23304E-1,0E0,3.7569594E-1,0E0,1.2548101E-1,3.487083E-1,1.891492E-1,2.4696006E-1,1.5259412E-1,0E0,0E0,2.4340254E-1,2.0628282E-1,0E0,2.1013916E-1,2.4934453E-1,1.4190704E-1,0E0,0E0,1.9884023E-1,3.4342602E-1,1.7548373E-1,1.6461027E-1,2.8938866E-1,1.5630521E-1,0E0,2.9220864E-1,0E0,0E0,1.8817165E-1,3.234579E-1,0E0,0E0,2.056516E-1,0E0,0E0,0E0,2.036412E-1,1.0885292E-1,1.3818195E-1,2.830071E-1,2.4303865E-1,1.5720004E-1,0E0,2.0390177E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,20,20,21,21,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,60,60,63,63,64,64,67,67,71,71,72,72,73,73,74,74,75,75,76,76,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,48,50,52,54,56,-1,58,-1,60,-1,62,64,66,68,70,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,86,88,90,92,-1,94,-1,-1,96,98,-1,-1,100,-1,-1,-1,102,104,106,108,110,112,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8122449E2,2.419355E0,3.0161016E7,1E0,3.8609805E10,1.0122174E3,5.121E3,1.2471935E7,5.5E1,1E0,6.9666364E2,5.666993E0,9.1875E0,3E0,6.082581E3,6.4953816E-1,3.3820656E7,-5.394868E-3,-1.7699482E-2,6.037914E-3,1.279012E7,6.929741E8,1.772061E-3,8.848604E1,9.2882293E-1,3E0,1.4075E4,-1.176125E-2,8E0,2.6572757E10,1.1E1,3.79E2,4.5555557E1,4.275471E-3,1.2694E4,5.974491E-3,4.1E1,-1.7615017E-2,1.1216729E2,2.831341E6,1.0133419E8,6.65E2,1.2920592E0,1.0837881E-2,5.353572E-4,1E0,2.6979439E1,1.7568696E-3,1.8600049E10,3.75163E5,1.5190727E5,-6.1760414E-3,3.7283131E-3,1.901E3,3.9E2,1.43E2,8.893E3,6.4E1,3.054842E-4,-1.2700996E-2,2.6E1,3.3750876E-3,-7.538584E-3,2.9600988E5,1.8137958E6,1.4490446E-2,3.1741408E-3,4.6501E4,5.7904683E-3,-1.7578412E-2,-4.8073903E-3,5.185489E3,8.861076E0,8.5518695E4,3.3633875E6,2.653986E6,9.330579E0,9.880613E-3,4.5505118E2,-1.3959523E-2,-8.867553E-4,3.1658085E-3,-4.2445837E-3,5.332567E-3,9.187295E-4,4.246692E-4,-9.85301E-3,-5.718539E-4,-6.719911E-3,-2.8425375E-2,-6.0318247E-3,-9.624904E-3,-5.683564E-4,-7.2848407E-4,-5.8950796E-3,7.933593E-3,-3.8281565E-3,1.7544639E-4,2.3679947E-3,-5.91567E-3,-8.7543053E-4,2.8594022E-3,1.13757E-2,1.4902215E-2,3.078912E-3,6.3625356E-4,5.1708897E-3,-4.047322E-3,1.4837121E-3,-3.899793E-3,3.958989E-3,-1.4766823E-2,-3.4316233E-3,-1.1583843E-3,7.141098E-3],"split_indices":[52,56,45,14,31,52,2,45,0,53,4,53,56,3,52,41,7,0,0,0,7,5,0,56,38,11,9,0,3,12,3,1,52,0,29,0,0,0,4,5,7,8,56,0,0,66,58,0,5,1,33,0,0,9,12,0,10,11,42,0,8,0,0,47,28,0,0,1,0,0,0,52,53,28,51,1,56,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.682E3,6.25E2,2.057E3,4.19E2,2.06E2,1.593E3,4.64E2,4.1E2,9E0,1.36E2,7E1,1.121E3,4.72E2,1.57E2,3.07E2,3.8E2,3E1,6E0,3E0,1E1,1.26E2,5.8E1,1.2E1,8.06E2,3.15E2,3.8E1,4.34E2,8E0,1.49E2,2.57E2,5E1,2.89E2,9.1E1,6E0,2.4E1,6E0,1.2E2,9E0,4.9E1,7.92E2,1.4E1,3E2,1.5E1,2.8E1,1E1,5.8E1,3.76E2,1.4E1,1.35E2,2.49E2,8E0,3E0,4.7E1,6.4E1,2.25E2,1.9E1,7.2E1,6E0,1.8E1,8E0,1.12E2,2E0,4.7E1,2.5E1,7.67E2,6E0,8E0,2.89E2,1.1E1,2E0,1.3E1,4.7E1,1.1E1,7.5E1,3.01E2,1.23E2,1.2E1,4E0,2.45E2,5E0,3E0,9E0,5.5E1,5.2E1,1.73E2,5E0,1.4E1,6.1E1,1.1E1,3E0,3E0,6E0,1.2E1,7.5E1,3.7E1,2.2E1,3E0,5.54E2,2.13E2,1.9E1,2.7E2,4E1,7E0,8E0,3E0,2.2E1,5.3E1,2.2E1,2.79E2,1.14E2,9E0,6E0,6E0,2.39E2,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.2291354E-3,-2.242416E-2,1.8934418E-2,-6.969995E-3,-5.7640344E-2,3.6321364E-2,-2.4335792E-2,-1.803986E-2,3.544245E-2,-7.275407E-2,1.4736947E-3,9.81766E-3,3.1442832E-2,-4.1047584E-2,4.1761704E-2,-3.7999442E-3,-4.986822E-2,4.317684E-2,-9.759392E-3,-1.5901522E-1,-5.6529354E-2,5.4578822E-2,-3.432106E-2,1.8879361E-2,7.302315E-2,-3.4678027E-2,-1.9686581E-1,7.046844E-3,1.877496E-2,1.1248517E-1,-9.729822E-3,-9.59718E-2,-1.1353953E-2,-8.336403E-3,6.632146E-2,-7.21133E-2,-2.4653335E-1,-3.1189358E-2,-1.1075971E-1,9.943153E-2,-6.414922E-4,6.8529444E-3,-1.0561256E-1,3.3028506E-2,-1.8614754E-2,8.402982E-3,5.314058E-2,-1.9789933E-1,-2.8367296E-2,-3.6642575E-3,-1.676441E-2,7.677766E-2,-2.8761689E-2,-1.6908859E-3,2.0351447E-1,-6.51642E-3,-2.2134253E-1,-6.9907725E-2,-3.2355767E-1,1.1944948E-1,-2.55202E-2,2.044738E-2,-8.964773E-3,1.315656E-1,2.6665814E-2,-8.43176E-3,-3.220332E-4,-4.0098327E-1,-1.4424694E-1,-5.408174E-2,2.7289912E-3,3.7399686E-3,-1.465993E-1,6.2734396E-3,-2.8529726E-3,2.0822876E-3,-3.7390497E-3,-6.5015554E-3,4.327909E-3,6.357335E-2,2.7048127E-3,-3.7986785E-2,6.1834265E-2,1.00490585E-1,1.9638369E-2,-1.8085772E-2,-3.9886828E-3,-3.1510174E-2,9.780491E-3,1.7903276E-1,1.6698474E-2,-8.229478E-2,1.8705702E-3,1.1436124E-2,-8.244502E-4,-1.1340085E-3,1.1428441E-3,-1.7697668E-2,-8.5270416E-4,1.0553922E-2,-4.108393E-3,-7.501211E-3,-3.164794E-2,1.2169124E-2,9.7504724E-4,-4.0778797E-3,3.4163802E-4,-4.035697E-4,5.9185997E-3,3.991354E-3,9.493994E-3,1.9446971E-3,-5.8681034E-3,-2.436361E-3,-2.3645807E-2,2.5859496E-3,-9.297908E-3,3.9000422E-4,-4.469447E-3,1.0018104E-2,-3.5217085E-3,4.531245E-3,-7.866371E-3,1.6731343E-3,5.982061E-3,-2.042094E-2,3.6345847E-4,-1.4006087E-3,-9.649833E-3,3.7779266E-3,-5.128896E-3,6.5392E-3,5.4942863E-4,1.6160468E-3,-4.654913E-3,-5.656984E-3,-1.1433343E-3,1.0782969E-2,-9.1099844E-4,-4.375089E-3,3.3299716E-3,-8.74991E-3,-1.4343066E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,83,85,87,-1,-1,89,91,-1,93,95,97,99,101,103,105,107,-1,109,111,-1,-1,113,115,117,-1,119,121,-1,-1,-1,-1,-1,-1,123,125,127,129,131,133,-1,-1,135,-1,137,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1299555E0,7.011194E-1,1.0203004E0,4.2178094E-1,3.5122657E-1,7.893125E-1,4.3140656E-1,3.225125E-1,3.5295117E-1,4.2907405E-1,1.5584488E-1,0E0,4.901932E-1,3.0183828E-1,1.886081E-1,3.4104583E-1,3.8955235E-1,2.16351E-1,0E0,3.5489082E-1,3.6070776E-1,1.0194257E-1,1.4455622E-1,3.847038E-1,4.3468785E-1,3.0339986E-1,1.9445935E-1,0E0,1.8418702E-1,3.3006826E-1,3.1815618E-1,5.7056236E-1,2.2726062E-1,2.9727328E-1,3.199041E-1,1.7554253E-1,3.2131433E-1,3.7044823E-1,3.452524E-1,1.081796E-1,0E0,1.0293615E-1,1.1861509E-1,4.8702633E-1,3.124826E-1,0E0,2.8991127E-1,1.9516134E-1,2.1842872E-1,0E0,0E0,1.8131255E-1,1.3754904E-1,0E0,1.20930076E-1,2.4249755E-1,2.0457199E-1,4.3837348E-1,4.59499E-1,1.5493946E-1,2.1479365E-1,1.4994422E-1,0E0,1.346001E-1,1.692963E-1,0E0,0E0,2.6717854E-1,1.6876867E-1,3.5485125E-1,0E0,3.48785E-1,2.881428E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5584238E-1,5.5868274E-1,2.2420923E-1,1.2079482E-1,2.423529E-1,1.829266E-1,0E0,0E0,1.8700176E-1,0E0,1.0477331E-1,1.2053237E-1,1.049052E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,46,46,47,47,48,48,51,51,52,52,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,67,67,68,68,69,69,71,71,72,72,79,79,80,80,81,81,82,82,83,83,84,84,87,87,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,84,86,88,-1,-1,90,92,-1,94,96,98,100,102,104,106,108,-1,110,112,-1,-1,114,116,118,-1,120,122,-1,-1,-1,-1,-1,-1,124,126,128,130,132,134,-1,-1,136,-1,138,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.273439E2,5.2187E4,3.1826714E7,4.264897E6,9.5822406E2,3.79E2,4.247868E3,3.38464E5,1E0,3.451532E8,1.0931163E9,9.81766E-3,1.8839418E3,1.5874315E10,9.042859E4,7E0,1.131645E6,6.9432104E7,-9.759392E-3,1.0407268E10,1.7291142E7,2.4013288E1,2.3012722E1,1.2328733E1,1.4075E4,3E0,2.8216E4,7.046844E-3,6.01324E11,1.8255814E0,6.8E1,9.195004E6,5.6656016E3,6.763314E7,9.538462E0,8.457023E3,1.3797468E0,4.95E9,2.1358025E0,2.342147E7,-6.414922E-4,1.4015416E6,3.3546498E-1,5.68197E6,4.33E2,8.402982E-3,5.9948825E6,2.4991455E9,4.5505118E2,-3.6642575E-3,-1.676441E-2,4.7777777E0,1E0,-1.6908859E-3,1.85221E5,8.01E2,8.15E2,4.814389E4,4.6501E4,1.296E3,5.033165E7,1.57383175E1,-8.964773E-3,1.1411955E-1,3.7307863E0,-8.43176E-3,-3.220332E-4,8.646E4,4.01E2,1.0508E5,2.7289912E-3,6.5029144E5,1.941926E2,6.2734396E-3,-2.8529726E-3,2.0822876E-3,-3.7390497E-3,-6.5015554E-3,4.327909E-3,8.027061E2,1.22E2,3.5085775E5,7.494145E-2,2.3977574E7,2.3821254E7,-1.8085772E-2,-3.9886828E-3,3.5315048E2,9.780491E-3,1.9013363E4,3.368326E11,1.4E1,1.8705702E-3,1.1436124E-2,-8.244502E-4,-1.1340085E-3,1.1428441E-3,-1.7697668E-2,-8.5270416E-4,1.0553922E-2,-4.108393E-3,-7.501211E-3,-3.164794E-2,1.2169124E-2,9.7504724E-4,-4.0778797E-3,3.4163802E-4,-4.035697E-4,5.9185997E-3,3.991354E-3,9.493994E-3,1.9446971E-3,-5.8681034E-3,-2.436361E-3,-2.3645807E-2,2.5859496E-3,-9.297908E-3,3.9000422E-4,-4.469447E-3,1.0018104E-2,-3.5217085E-3,4.531245E-3,-7.866371E-3,1.6731343E-3,5.982061E-3,-2.042094E-2,3.6345847E-4,-1.4006087E-3,-9.649833E-3,3.7779266E-3,-5.128896E-3,6.5392E-3,5.4942863E-4,1.6160468E-3,-4.654913E-3,-5.656984E-3,-1.1433343E-3,1.0782969E-2,-9.1099844E-4,-4.375089E-3,3.3299716E-3,-8.74991E-3,-1.4343066E-3],"split_indices":[52,29,45,9,55,29,52,9,65,5,5,0,52,12,28,3,9,7,0,31,45,58,56,53,9,3,9,0,31,54,3,45,47,7,58,48,53,5,56,45,0,32,42,50,8,0,48,12,56,0,0,54,102,0,9,2,2,45,1,29,7,58,0,38,53,0,0,1,0,29,0,28,33,0,0,0,0,0,0,52,10,28,57,47,9,0,0,33,0,52,31,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.642E3,1.288E3,1.354E3,8.96E2,3.92E2,9.66E2,3.88E2,7.11E2,1.85E2,3.12E2,8E1,2.6E1,9.4E2,3.1E2,7.8E1,4.92E2,2.19E2,1.8E2,5E0,4.8E1,2.64E2,3.2E1,4.8E1,7.23E2,2.17E2,2.99E2,1.1E1,1.3E1,6.5E1,2.3E1,4.69E2,9.9E1,1.2E2,5.6E1,1.24E2,2.5E1,2.3E1,1.81E2,8.3E1,1.9E1,1.3E1,3.1E1,1.7E1,5.25E2,1.98E2,3.4E1,1.83E2,1E1,2.89E2,7E0,4E0,2.9E1,3.6E1,9E0,1.4E1,4.63E2,6E0,9E1,9E0,1.1E1,1.09E2,4.9E1,7E0,4.6E1,7.8E1,9E0,1.6E1,8E0,1.5E1,1.44E2,3.7E1,2E1,6.3E1,1.6E1,3E0,2.2E1,9E0,1.5E1,2E0,2.61E2,2.64E2,1.6E2,3.8E1,7.5E1,1.08E2,3E0,7E0,2.86E2,3E0,1E1,1.9E1,2E1,1.6E1,1.2E1,2E0,2.96E2,1.67E2,3E0,3E0,4E0,8.6E1,7E0,2E0,4E0,7E0,3.8E1,7.1E1,3.9E1,1E1,2.8E1,1.8E1,7.2E1,6E0,2E0,6E0,3E0,1.2E1,5.6E1,8.8E1,5E0,1.5E1,4E0,5.9E1,1.79E2,8.2E1,2E0,2.62E2,1.53E2,7E0,3.5E1,3E0,5.3E1,2.2E1,9.7E1,1.1E1,2.2E1,2.64E2,8E0,2E0,6E0,1.3E1,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[-5.281632E-4,-1.1202844E-2,3.1433403E-2,2.2004566E-3,-4.7597833E-2,4.3088675E-2,-3.6814332E-2,-1.4853295E-2,2.5956249E-2,-5.2261353E-2,1.2035309E-1,3.283489E-2,1.0124883E-1,7.6310046E-2,-6.289518E-2,-4.3192427E-3,-5.7961255E-2,1.5090963E-1,1.8195909E-2,-2.1320546E-1,-4.8756264E-2,1.0116357E-2,-2.4511605E-3,7.7917054E-2,1.4502976E-2,1.70254E-1,4.566831E-2,5.8881886E-4,7.7962787E-3,-1.4975409E-1,-2.7968906E-2,9.269684E-2,-1.2170515E-2,-1.07313454E-1,1.3369906E-2,8.464524E-3,-2.137516E-3,2.159101E-2,-1.21332906E-1,-2.1642644E-2,-1.2237837E-2,-6.690859E-2,-1.3201713E-2,6.819413E-2,1.3741599E-2,9.904998E-2,6.002954E-3,6.4217607E-3,1.4905538E-2,-1.3023481E-2,7.229387E-2,-8.179164E-3,2.2703132E-3,-4.8799335E-4,-1.0437534E-2,1.342202E-2,6.677975E-2,-6.3569043E-3,-8.479879E-3,-1.9267292E-1,-4.4730213E-2,-3.9610814E-2,8.130084E-2,6.0581405E-2,5.691981E-3,-2.3208467E-2,-2.0401338E-2,7.645307E-3,-8.670941E-3,-3.5171766E-2,-1.0072776E-1,-6.903289E-2,3.2509472E-2,8.1979826E-2,-3.393903E-2,-1.7469164E-3,1.5366836E-1,8.539461E-3,-1.2684656E-2,-6.8481714E-3,1.0203649E-1,2.8374788E-4,7.1183303E-3,3.3448618E-5,-3.0145016E-3,-5.9549934E-3,-1.6068693E-2,-9.860446E-3,-7.3809933E-4,1.4741772E-3,-5.700591E-3,9.496523E-3,2.1689641E-3,2.340815E-3,9.852083E-3,-1.3587912E-4,4.231888E-3,-6.312675E-3,3.7359537E-3,-4.0171384E-3,-7.715464E-5,-2.4696998E-3,-6.9332467E-3,1.641281E-3,-4.982478E-3,-8.220929E-3,3.1490452E-3,-8.804818E-3,4.214012E-3,-8.184943E-3,2.6966832E-3,4.425255E-3,1.4934732E-2,-2.6951492E-4,2.7730812E-3,3.6451896E-3,1.1520508E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,51,53,55,57,59,61,-1,-1,63,65,-1,67,69,71,73,-1,75,77,-1,-1,-1,79,-1,-1,-1,-1,-1,81,-1,83,85,87,89,91,93,95,-1,97,-1,-1,99,101,103,105,107,109,-1,111,113,-1,-1,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.1983086E-1,9.8597467E-1,5.385123E-1,5.9916085E-1,4.3237388E-1,3.407762E-1,2.968872E-1,3.9073104E-1,5.965167E-1,2.885903E-1,2.446648E-1,4.0547955E-1,3.220644E-1,1.03187144E-1,2.4153784E-1,5.289861E-1,5.9637386E-1,1.897744E-1,2.7879518E-1,5.2218014E-1,3.347236E-1,0E0,0E0,2.69018E-1,2.5165206E-1,1.5200162E-1,4.3403172E-1,0E0,0E0,1.0348195E-1,1.9705895E-1,2.3406693E-1,2.8454608E-1,5.25589E-1,2.5515246E-1,0E0,0E0,3.5300353E-1,5.0344586E-1,0E0,2.3202029E-1,3.6567688E-1,4.5396763E-1,1.9567513E-1,0E0,2.4247831E-1,2.2212264E-1,0E0,0E0,0E0,3.091424E-1,0E0,0E0,0E0,0E0,0E0,2.33247E-1,0E0,3.1198424E-1,3.6823893E-1,2.7560422E-1,2.2729705E-1,1.2162542E-1,2.7020717E-1,2.865406E-1,0E0,1.4185527E-1,0E0,0E0,2.924831E-1,3.519523E-1,2.928674E-1,6.7648816E-1,2.095828E-1,2.1976674E-1,0E0,1.9407809E-1,2.227239E-1,0E0,0E0,1.2890899E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,40,40,41,41,42,42,43,43,45,45,46,46,50,50,56,56,58,58,59,59,60,60,61,61,62,62,63,63,64,64,66,66,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,52,54,56,58,60,62,-1,-1,64,66,-1,68,70,72,74,-1,76,78,-1,-1,-1,80,-1,-1,-1,-1,-1,82,-1,84,86,88,90,92,94,96,-1,98,-1,-1,100,102,104,106,108,110,-1,112,114,-1,-1,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,1.2471935E7,7.218466E7,3.0291306E2,1.3238013E2,6.082581E3,8.24666E5,5.2187E4,3.79E2,1.2673605E7,1.9768514E3,2.26182E5,3.3517068E7,1E0,3.0387878E3,7E0,6.5932993E10,4.2E1,1.9162654E7,7E2,1.022332E6,1.0116357E-2,-2.4511605E-3,1.95684E5,1E0,5.0738932E7,5.388794E6,5.8881886E-4,7.7962787E-3,5.377837E6,9.22E4,1.3050649E4,8.333333E0,3.6111112E0,1.6708623E2,8.464524E-3,-2.137516E-3,3.28825E0,3.171E3,-2.1642644E-2,2.3279E4,1.2886906E5,5.065188E-4,1.3900395E2,1.3741599E-2,9.536863E4,1.3056483E8,6.4217607E-3,1.4905538E-2,-1.3023481E-2,2.7258065E0,-8.179164E-3,2.2703132E-3,-4.8799335E-4,-1.0437534E-2,1.342202E-2,1E0,-6.3569043E-3,2.4E1,3.3162162E0,3E0,2.1818182E0,1.84442E5,7.562698E1,4.41E4,-2.3208467E-2,5.11E2,7.645307E-3,-8.670941E-3,1.159E3,2.1270142E5,1.7560976E0,4E0,7.549744E7,4.4200982E8,-1.7469164E-3,4.3525005E3,4.551684E6,-1.2684656E-2,-6.8481714E-3,5.68E2,2.8374788E-4,7.1183303E-3,3.3448618E-5,-3.0145016E-3,-5.9549934E-3,-1.6068693E-2,-9.860446E-3,-7.3809933E-4,1.4741772E-3,-5.700591E-3,9.496523E-3,2.1689641E-3,2.340815E-3,9.852083E-3,-1.3587912E-4,4.231888E-3,-6.312675E-3,3.7359537E-3,-4.0171384E-3,-7.715464E-5,-2.4696998E-3,-6.9332467E-3,1.641281E-3,-4.982478E-3,-8.220929E-3,3.1490452E-3,-8.804818E-3,4.214012E-3,-8.184943E-3,2.6966832E-3,4.425255E-3,1.4934732E-2,-2.6951492E-4,2.7730812E-3,3.6451896E-3,1.1520508E-2],"split_indices":[2,45,45,52,56,52,29,29,29,45,4,29,51,59,4,3,31,0,47,2,28,0,0,29,67,1,47,0,0,29,10,28,52,54,52,0,0,54,2,0,29,47,38,58,0,28,44,0,0,0,54,0,0,0,0,0,16,0,8,56,11,56,29,58,10,0,0,0,0,11,28,53,3,7,7,0,4,29,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.694E3,2.02E3,6.74E2,1.477E3,5.43E2,5.76E2,9.8E1,8.6E2,6.17E2,5.29E2,1.4E1,4.91E2,8.5E1,1.8E1,8E1,6.92E2,1.68E2,3.5E1,5.82E2,1E1,5.19E2,9E0,5E0,1.41E2,3.5E2,3.7E1,4.8E1,1.1E1,7E0,2.2E1,5.8E1,5.1E1,6.41E2,9.9E1,6.9E1,3.1E1,4E0,5.69E2,1.3E1,4E0,6E0,3.43E2,1.76E2,1.36E2,5E0,3.1E1,3.19E2,3.1E1,6E0,3E0,4.5E1,2E1,2E0,5.4E1,4E0,5E0,4.6E1,1.8E1,6.23E2,4.1E1,5.8E1,3.9E1,3E1,1.64E2,4.05E2,2E0,1.1E1,3E0,3E0,1.78E2,1.65E2,7.9E1,9.7E1,1.2E2,1.6E1,9E0,2.2E1,3.17E2,2E0,5E0,4E1,2.7E1,1.9E1,5.34E2,8.9E1,2.9E1,1.2E1,8E0,5E1,2.1E1,1.8E1,6E0,2.4E1,1.53E2,1.1E1,3.68E2,3.7E1,5E0,6E0,7.2E1,1.06E2,7.9E1,8.6E1,2E1,5.9E1,1.3E1,8.4E1,2E0,1.18E2,6E0,1E1,1.7E1,5E0,2.47E2,7E1,3.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-1.0142139E-3,-1.4178801E-2,2.1765398E-2,-1.8516176E-3,-5.9930407E-2,5.7788864E-2,3.1822545E-3,-1.82586E-2,2.210297E-2,-2.9770514E-1,-5.071695E-2,3.5540257E-2,1.0340945E-1,-1.9315339E-3,1.1046208E-1,-1.4197727E-3,-4.2384584E-2,5.658927E-2,-4.8949798E-3,-3.6004197E-2,-1.4679648E-1,-5.955818E-2,9.460475E-2,2.8082239E-2,1.0005599E-2,1.2385368E-1,-1.4481518E-3,2.7740391E-2,-1.866459E-2,1.5376389E-2,6.392128E-2,2.7887279E-3,-1.5912023E-1,-1.9818562E-1,-3.2475E-2,1.4442478E-1,1.1182662E-2,-1.01579785E-1,8.208584E-3,1.697075E-3,-1.051501E-2,-5.352984E-2,-2.639325E-1,6.961128E-3,-9.774782E-4,-1.031019E-2,3.2562774E-2,1.3937709E-1,-9.0648764E-4,7.696125E-2,-9.35537E-4,-4.1285637E-3,-7.7698976E-2,2.7140042E-2,9.847168E-3,-6.0941588E-2,1.4888449E-2,-1.4536853E-2,-6.033029E-4,5.992135E-4,-3.771519E-1,-4.377845E-2,6.133441E-2,9.820745E-2,1.819093E-1,2.0467034E-2,-2.246749E-2,3.5180133E-2,-1.7142558E-1,2.8312683E-2,-6.768737E-2,-9.617554E-3,-7.639413E-2,-1.8933129E-2,-3.8186358E-3,2.9477695E-2,1.1332613E-2,1.1275154E-2,1.1508262E-1,2.3307134E-3,7.7019357E-3,-7.882398E-3,1.5273821E-2,-2.6288193E-2,3.8986683E-2,-1.2880665E-1,-1.5915046E-2,5.9145787E-3,-1.4301324E-3,-3.7210542E-3,5.0651054E-3,3.0504225E-3,-5.1956775E-4,-1.0325392E-2,3.9078924E-3,-2.2271471E-2,-8.1609824E-4,1.7307271E-4,-3.3029001E-3,-1.2487058E-3,5.3888247E-3,6.050218E-3,-3.094054E-4,6.802989E-4,9.518027E-3,-2.6355565E-3,2.5930281E-3,5.8529396E-3,-2.7240338E-3,-6.950938E-4,-1.0675903E-2,-1.0815797E-3,2.54794E-3,-4.0288246E-3,2.8446435E-3,2.5977574E-3,-2.9097442E-3,-8.524616E-3,-3.1500182E-3,5.4907694E-4,3.0112644E-3,-4.9246913E-3,6.2293517E-3,-8.5760606E-4,6.5994277E-3,8.065654E-3,-1.4813286E-3,2.1677134E-3,-9.27698E-3,-4.557007E-3,-1.2448348E-2,-2.42437E-3,3.235921E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,-1,47,-1,49,51,-1,53,55,57,59,61,63,65,67,69,-1,-1,71,73,-1,-1,-1,75,77,-1,79,81,83,85,87,-1,89,91,-1,-1,93,95,97,99,101,103,105,-1,107,109,111,113,115,117,-1,-1,119,-1,-1,121,-1,-1,123,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8930396E-1,9.4062173E-1,6.4514595E-1,5.172282E-1,7.604189E-1,3.2924294E-1,3.4967557E-1,3.1722364E-1,4.9839377E-1,7.804909E-1,4.4504368E-1,2.8120485E-1,2.9610515E-1,3.0288747E-1,2.6373547E-1,3.0640134E-1,4.8916346E-1,9.356385E-1,3.821754E-1,0E0,1.5364654E-1,3.8381994E-1,1.1970799E-1,2.4036077E-1,0E0,2.0876932E-1,0E0,3.105918E-1,3.343481E-1,0E0,1.2471428E-1,3.479109E-1,2.5011125E-1,6.761659E-1,3.2397726E-1,1.2072992E-1,7.052867E-1,3.4990656E-1,4.077845E-1,0E0,0E0,3.155328E-1,1.8632191E-1,0E0,0E0,0E0,1.2586622E-1,1.6893625E-1,0E0,1.8436855E-1,3.136044E-1,3.010261E-1,2.4108773E-1,1.19038574E-1,0E0,2.0935574E-1,4.767825E-1,0E0,0E0,1.9272085E-1,3.0860424E-1,3.2376313E-1,1.5058888E-1,1.1268765E-1,1.1858702E-1,3.9455205E-1,0E0,1.11113794E-1,1.906972E-1,2.685103E-1,1.219857E-1,3.592794E-1,2.1163464E-1,0E0,0E0,1.2501813E-1,0E0,0E0,2.3880023E-1,0E0,0E0,2.0401992E-1,0E0,1.9326086E-1,1.650637E-1,1.605987E-1,1.0773334E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,46,46,47,47,49,49,50,50,51,51,52,52,53,53,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71,72,72,75,75,78,78,81,81,83,83,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,-1,48,-1,50,52,-1,54,56,58,60,62,64,66,68,70,-1,-1,72,74,-1,-1,-1,76,78,-1,80,82,84,86,88,-1,90,92,-1,-1,94,96,98,100,102,104,106,-1,108,110,112,114,116,118,-1,-1,120,-1,-1,122,-1,-1,124,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.423E3,1.3905958E7,8E-3,2.7604443E2,6E0,1.5106794E3,4.5505118E2,2.294901E0,8E0,1.6856086E8,1E0,4.0602388E6,8.657441E7,1.3446785E7,1.6907827E9,6.2233735E-6,5.8601086E1,1E0,1.074E3,-3.6004197E-2,3.01E2,4.387561E1,3.476768E-3,2.8672566E0,1.0005599E-2,4.728721E0,-1.4481518E-3,4.0208E4,1.8E2,1.5376389E-2,2.62797E5,4.7E2,1.4735735E2,7.8079817E-3,1.7022566E7,3.7810526E2,1.6904226E7,1.7143776E7,1.4383178E1,1.697075E-3,-1.051501E-2,1.36E2,6.203E3,6.961128E-3,-9.774782E-4,-1.031019E-2,2.03148E2,4.899767E6,-9.0648764E-4,2.3276744E5,1.921032E4,3.4409692E3,3.555E3,3.2475834E7,9.847168E-3,3.1530054E0,2.3067484E0,-1.4536853E-2,-6.033029E-4,3.0404909E0,4.38E2,2.6195612E-2,2.691875E2,9.38E2,5.9E1,1E0,-2.246749E-2,4.930435E2,1.3E1,1.954E3,3.474851E8,1E0,1.48298E5,-1.8933129E-2,-3.8186358E-3,7.019E3,1.1332613E-2,1.1275154E-2,5.794E3,2.3307134E-3,7.7019357E-3,4.091015E9,1.5273821E-2,5.473125E3,4.4543375E4,1.8215276E7,6.5900993E-1,5.9145787E-3,-1.4301324E-3,-3.7210542E-3,5.0651054E-3,3.0504225E-3,-5.1956775E-4,-1.0325392E-2,3.9078924E-3,-2.2271471E-2,-8.1609824E-4,1.7307271E-4,-3.3029001E-3,-1.2487058E-3,5.3888247E-3,6.050218E-3,-3.094054E-4,6.802989E-4,9.518027E-3,-2.6355565E-3,2.5930281E-3,5.8529396E-3,-2.7240338E-3,-6.950938E-4,-1.0675903E-2,-1.0815797E-3,2.54794E-3,-4.0288246E-3,2.8446435E-3,2.5977574E-3,-2.9097442E-3,-8.524616E-3,-3.1500182E-3,5.4907694E-4,3.0112644E-3,-4.9246913E-3,6.2293517E-3,-8.5760606E-4,6.5994277E-3,8.065654E-3,-1.4813286E-3,2.1677134E-3,-9.27698E-3,-4.557007E-3,-1.2448348E-2,-2.42437E-3,3.235921E-3],"split_indices":[2,45,57,52,3,52,56,56,17,5,74,48,45,45,12,37,50,16,2,0,0,53,38,56,0,38,0,9,8,0,2,1,52,58,9,52,47,5,53,0,0,11,29,0,0,0,56,45,0,33,4,52,0,50,0,53,58,0,0,56,0,42,55,2,29,100,0,4,3,2,7,14,1,0,0,2,0,0,2,0,0,7,0,32,4,1,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.63E3,1.667E3,9.63E2,1.314E3,3.53E2,3.27E2,6.36E2,7.8E2,5.34E2,1.2E1,3.41E2,2.21E2,1.06E2,6.08E2,2.8E1,4.6E2,3.2E2,2.34E2,3E2,2E0,1E1,3.22E2,1.9E1,2.13E2,8E0,9.2E1,1.4E1,2.19E2,3.89E2,4E0,2.4E1,4.49E2,1.1E1,1.8E1,3.02E2,7.9E1,1.55E2,3.5E1,2.65E2,3E0,7E0,3.14E2,8E0,1.3E1,6E0,3E0,2.1E2,8.3E1,9E0,8E1,1.39E2,3.13E2,7.6E1,2E1,4E0,7.1E1,3.78E2,5E0,6E0,9E0,9E0,2.7E2,3.2E1,3.7E1,4.2E1,1.53E2,2E0,1.2E1,2.3E1,2.1E2,5.5E1,1.08E2,2.06E2,4E0,4E0,2.08E2,2E0,1.5E1,6.8E1,6.1E1,1.9E1,1.37E2,2E0,2.07E2,1.06E2,4.1E1,3.5E1,7E0,1.3E1,6.5E1,6E0,1.3E2,2.48E2,2E0,7E0,7E0,2E0,9.4E1,1.76E2,1.2E1,2E1,2.9E1,8E0,4E0,3.8E1,4.7E1,1.06E2,6E0,6E0,6E0,1.7E1,6.9E1,1.41E2,4.9E1,6E0,4.8E1,6E1,1.8E1,1.88E2,1.36E2,7.2E1,4E0,6.4E1,1.29E2,8E0,4E0,2.03E2,1.04E2,2E0,3.4E1,7E0,2.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-1.9088521E-3,-7.848197E-3,3.692166E-2,-3.4132984E-3,-7.680386E-2,5.2463256E-2,-6.260142E-2,-3.2322776E-2,6.083315E-3,-6.2375523E-2,-1.7667925E-1,2.3234911E-1,4.494253E-2,4.9292248E-2,-1.2555254E-1,5.535428E-3,-5.661081E-2,4.2398136E-2,-4.54813E-3,-8.277883E-3,-2.547168E-3,-1.41818775E-2,-2.5284023E-4,1.3345404E-2,2.461529E-5,3.2221846E-2,9.72685E-2,-4.3512685E-3,6.079188E-3,-1.7648704E-1,-1.4462447E-3,-5.429388E-2,3.5198662E-2,-4.8784E-2,-1.7392181E-1,1.1567187E-1,1.28170205E-2,-1.7096302E-2,1.8499969E-2,1.15579575E-1,2.010245E-2,7.051919E-2,1.4984204E-2,-1.20289065E-2,-4.4320114E-3,-2.8239004E-2,-2.0779923E-1,-1.4610586E-2,9.366194E-2,-8.1413776E-2,-9.791397E-3,-2.8870687E-1,-2.5746275E-3,1.4669019E-1,3.9357938E-2,2.3352988E-2,-1.6363691E-1,-3.4045547E-3,-4.8496295E-2,1.446451E-1,1.1597278E-2,2.7891665E-3,8.911542E-3,2.5243545E-2,-6.3148984E-3,1.3379712E-2,3.7202824E-2,-4.8330007E-3,3.3725626E-4,-1.8986696E-2,-3.3670547E-3,2.8038216E-3,-4.369222E-3,1.216976E-2,3.4978057E-3,-3.3300105E-3,-1.351699E-2,4.499438E-4,-5.459559E-3,-6.5066223E-3,-1.9844709E-2,8.304198E-3,2.8179528E-3,-1.3988273E-3,4.864539E-3,-8.862896E-4,2.860393E-3,-2.3833254E-2,-1.8514056E-3,-7.4159104E-4,1.5696839E-3,-9.714607E-3,-1.9054809E-3,3.8441012E-3,1.34249E-2,1.4452289E-3,-1.0906295E-3,3.5853295E-3,5.6672667E-4,3.7972329E-3,-2.234857E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,65,-1,-1,-1,67,69,71,73,75,77,79,-1,81,83,85,87,89,91,93,95,-1,-1,97,-1,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1191475E-1,7.034098E-1,5.477392E-1,5.940312E-1,1.8928355E-1,4.0218377E-1,3.4429264E-1,4.9224657E-1,6.291155E-1,1.15659624E-1,3.4649473E-1,1.2870783E-1,1.9255877E-1,2.0741484E-1,1.4523068E-1,3.7460452E-1,2.908392E-1,7.9833186E-1,3.6501196E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.3811495E-1,3.0668765E-1,0E0,0E0,1.0214484E-1,0E0,2.7410135E-1,4.1196212E-1,3.8980502E-1,2.6033294E-1,2.4514234E-1,4.9462718E-1,3.5108012E-1,3.8580215E-1,1.1043805E-1,1.6661176E-1,3.5144922E-1,0E0,0E0,0E0,1.5712643E-1,2.3151964E-1,4.369655E-1,1.965431E-1,3.788203E-1,2.8299057E-1,1.357773E-1,0E0,1.6577554E-1,1.3948232E-1,3.8176298E-1,6.069449E-1,2.4964757E-1,3.252709E-1,1.7839438E-1,2.708249E-1,0E0,0E0,1.3419287E-1,0E0,0E0,1.6607031E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,25,25,26,26,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,63,63,66,66],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,66,-1,-1,-1,68,70,72,74,76,78,80,-1,82,84,86,88,90,92,94,96,-1,-1,98,-1,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,6.747114E7,2.3580047E10,1.5288235E2,2.25215E5,7.969174E7,5.393203E-1,2.8373447E3,1.4615384E0,3.5315048E2,3.138649E6,7.5E1,9.3058E4,1.6482393E-1,1.4580295E0,3.3151623E1,2.74872E5,3.3253515E6,6.7628815E2,-8.277883E-3,-2.547168E-3,-1.41818775E-2,-2.5284023E-4,1.3345404E-2,2.461529E-5,2.386324E6,4.516183E1,-4.3512685E-3,6.079188E-3,1.031322E6,-1.4462447E-3,6.737101E7,7.6593264E-5,8.31E2,6E0,3.0805944E7,2.3523688E0,2.6540773E1,1.296E3,9.994E3,4.5481584E7,1.3683E4,1.4984204E-2,-1.20289065E-2,-4.4320114E-3,8.333333E0,1.14E2,2.0042918E0,8E0,2.2214102E6,6.5260305E0,9.0067856E5,-2.5746275E-3,8E0,7.8586104E7,3.7382904E7,1.0449754E0,1.6908307E-1,1.2920592E0,5.5607376E0,1.1347826E1,2.7891665E-3,8.911542E-3,2.805891E4,-6.3148984E-3,1.3379712E-2,6.38041E5,-4.8330007E-3,3.3725626E-4,-1.8986696E-2,-3.3670547E-3,2.8038216E-3,-4.369222E-3,1.216976E-2,3.4978057E-3,-3.3300105E-3,-1.351699E-2,4.499438E-4,-5.459559E-3,-6.5066223E-3,-1.9844709E-2,8.304198E-3,2.8179528E-3,-1.3988273E-3,4.864539E-3,-8.862896E-4,2.860393E-3,-2.3833254E-2,-1.8514056E-3,-7.4159104E-4,1.5696839E-3,-9.714607E-3,-1.9054809E-3,3.8441012E-3,1.34249E-2,1.4452289E-3,-1.0906295E-3,3.5853295E-3,5.6672667E-4,3.7972329E-3,-2.234857E-3],"split_indices":[2,45,5,52,11,7,41,47,53,33,29,3,2,41,42,55,29,45,52,0,0,0,0,0,0,1,54,0,0,11,0,7,38,2,8,5,41,58,29,10,47,9,0,0,0,52,29,54,18,28,57,50,0,8,5,12,53,38,56,54,53,0,0,48,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.651E3,2.3E3,3.51E2,2.162E3,1.38E2,3.04E2,4.7E1,5.34E2,1.628E3,1.22E2,1.6E1,1.1E1,2.93E2,1.7E1,3E1,2.09E2,3.25E2,3.68E2,1.26E3,8E0,1.14E2,9E0,7E0,9E0,2E0,2.37E2,5.6E1,6E0,1.1E1,1.9E1,1.1E1,6.9E1,1.4E2,3.06E2,1.9E1,1.05E2,2.63E2,8.16E2,4.44E2,2.9E1,2.08E2,5.1E1,5E0,9E0,1E1,6E1,9E0,7.6E1,6.4E1,1.66E2,1.4E2,9E0,1E1,7.4E1,3.1E1,2.49E2,1.4E1,5.69E2,2.47E2,2.2E1,4.22E2,1.7E1,1.2E1,2.02E2,6E0,6E0,4.5E1,1.9E1,4.1E1,3E0,6E0,3.9E1,3.7E1,6E0,5.8E1,1.58E2,8E0,1.19E2,2.1E1,5E0,4E0,5.6E1,1.8E1,1.5E1,1.6E1,1.16E2,1.33E2,3E0,1.1E1,4.27E2,1.42E2,1.2E1,2.35E2,1.6E1,6E0,2.74E2,1.48E2,4.2E1,1.6E2,3E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-1.643766E-3,-1.0015222E-2,3.597123E-2,6.320102E-4,-4.0407643E-2,9.3945056E-2,2.0066757E-2,-1.7151E-2,2.501478E-2,-2.3165508E-1,-3.552816E-2,8.332106E-3,6.675863E-2,-8.546275E-4,7.443124E-2,-1.0311707E-2,-7.09938E-2,7.277038E-3,1.9791927E-2,-2.540778E-2,-1.01910606E-1,-3.9765988E-2,1.1024873E-1,1.1838934E-1,2.3970537E-2,5.670017E-3,-1.4650047E-1,1.6297033E-1,2.9127924E-2,-3.4932964E-2,5.0727045E-3,-1.6223269E-2,-4.5678854E-2,3.9606217E-2,-1.3185554E-3,5.082975E-3,-7.6594464E-3,-3.011627E-2,-8.428325E-2,9.550045E-3,-1.5746458E-3,2.0387109E-1,2.6208095E-3,-6.866172E-4,4.2100106E-3,-6.2549626E-4,1.6815682E-1,-1.1189128E-3,-1.1068337E-2,6.713864E-3,1.7566385E-2,-9.660369E-3,4.5673512E-2,-3.0318114E-3,-5.854067E-2,1.19891666E-1,-2.2615767E-3,-6.1536536E-2,6.7244126E-3,2.563644E-2,1.0876027E-1,1.3554903E-2,-5.4844167E-2,-2.5234088E-2,-1.3308915E-1,-6.647169E-2,-2.0342486E-1,1.3314665E-3,1.1760547E-2,-9.492422E-3,3.2511787E-3,-4.474717E-3,1.1744419E-2,-7.5467816E-3,6.788527E-2,-2.138244E-3,4.968909E-3,-1.2486031E-3,-4.7845966E-3,4.4085085E-3,1.3638431E-2,2.3808531E-4,-5.3814244E-3,3.1562378E-3,-3.8256682E-3,1.4190607E-3,-7.905045E-3,7.554166E-3,1.19842356E-4,6.589851E-3,-1.2285486E-4,-4.1635838E-4,-5.736866E-3,-3.8178337E-3,-6.788315E-4,-1.922376E-2,-1.9597139E-3,-4.263264E-3,-5.6885293E-4,-1.3603877E-2,1.3823479E-3,8.19591E-3,-3.7887126E-5,2.3132898E-3,9.291863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,33,-1,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,-1,-1,63,65,-1,-1,67,-1,-1,-1,69,71,-1,-1,-1,-1,-1,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,99,-1,-1,-1,101,-1,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.458461E-1,7.1096414E-1,4.4899398E-1,7.063283E-1,5.236871E-1,2.1685743E-1,4.3795496E-1,3.4615636E-1,4.5245412E-1,4.8883134E-1,3.48486E-1,0E0,1.7389649E-1,2.6596084E-1,4.244423E-1,3.1721902E-1,7.0303833E-1,0E0,2.7654144E-1,0E0,1.4976919E-1,2.3029351E-1,2.1075077E-1,1.8476015E-1,1.1148069E-1,2.742525E-1,1.1963341E-1,1.1572492E-1,2.8421548E-1,2.4232492E-1,4.3480226E-1,0E0,2.9783574E-1,3.266405E-1,2.5629503E-1,0E0,0E0,2.2136301E-1,1.9019943E-1,0E0,0E0,1.03304446E-1,0E0,0E0,0E0,1.9903173E-1,2.245442E-1,0E0,0E0,0E0,0E0,0E0,3.176977E-1,6.179244E-1,2.4587E-1,1.16081506E-1,3.876147E-1,2.1803933E-1,0E0,2.2201325E-1,2.962497E-1,5.0427073E-1,2.0877096E-1,2.567758E-1,4.781892E-1,1.0358152E-1,2.3297313E-1,0E0,0E0,0E0,1.7319117E-1,0E0,0E0,0E0,1.4376637E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,37,37,38,38,41,41,45,45,46,46,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,70,70,74,74],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,34,-1,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,-1,-1,64,66,-1,-1,68,-1,-1,-1,70,72,-1,-1,-1,-1,-1,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,100,-1,-1,-1,102,-1,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5106794E3,1.4540612E7,8.3306855E6,3.3030225E2,4E0,2.2376953E1,6.082581E3,9.837297E0,2.1761298E0,1E0,1.4122614E2,8.332106E-3,1.813943E6,2.6572757E10,3.3517068E7,1.1556145E2,2E0,7.277038E-3,2.773036E1,-2.540778E-2,2.56E2,6.971004E7,1.6826648E3,2.8E1,1.75E2,2.62797E5,3.1415796E0,1.3948507E7,1.5439131E2,3.2E1,7.692308E-1,-1.6223269E-2,1.2677199E7,8.027061E2,2.0399538E5,5.082975E-3,-7.6594464E-3,4.387561E1,2.55298E5,9.550045E-3,-1.5746458E-3,9.97958E5,2.6208095E-3,-6.866172E-4,4.2100106E-3,8.874407E6,2.142857E0,-1.1189128E-3,-1.1068337E-2,6.713864E-3,1.7566385E-2,-9.660369E-3,2.9104478E0,4.0701206E1,2.9138756E0,7.8955513E-1,1E0,7.29E2,6.7244126E-3,1E0,1.3195207E7,2.7487562E0,2.185321E0,2.2348747E2,2.90002E5,1.9701008E5,4.242625E6,1.3314665E-3,1.1760547E-2,-9.492422E-3,3.53562E5,-4.474717E-3,1.1744419E-2,-7.5467816E-3,4.42384E5,-2.138244E-3,4.968909E-3,-1.2486031E-3,-4.7845966E-3,4.4085085E-3,1.3638431E-2,2.3808531E-4,-5.3814244E-3,3.1562378E-3,-3.8256682E-3,1.4190607E-3,-7.905045E-3,7.554166E-3,1.19842356E-4,6.589851E-3,-1.2285486E-4,-4.1635838E-4,-5.736866E-3,-3.8178337E-3,-6.788315E-4,-1.922376E-2,-1.9597139E-3,-4.263264E-3,-5.6885293E-4,-1.3603877E-2,1.3823479E-3,8.19591E-3,-3.7887126E-5,2.3132898E-3,9.291863E-3],"split_indices":[52,45,45,52,3,56,52,54,58,8,56,0,9,12,51,52,17,0,58,0,0,45,4,3,8,2,54,9,58,10,53,0,9,52,33,0,0,53,11,0,0,1,0,0,0,45,54,0,0,0,0,0,54,52,58,38,74,29,0,85,9,54,41,52,1,33,29,0,0,0,1,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.684E3,2.196E3,4.88E2,1.627E3,5.69E2,1.04E2,3.84E2,9.41E2,6.86E2,1.3E1,5.56E2,2.5E1,7.9E1,2.78E2,1.06E2,8.36E2,1.05E2,2.6E1,6.6E2,3E0,1E1,5.41E2,1.5E1,3.5E1,4.4E1,2.67E2,1.1E1,3.5E1,7.1E1,3.21E2,5.15E2,8E0,9.7E1,3.4E2,3.2E2,2E0,8E0,4.46E2,9.5E1,9E0,6E0,1.4E1,2.1E1,2.8E1,1.6E1,2.58E2,9E0,5E0,6E0,3.3E1,2E0,4E0,6.7E1,1.37E2,1.84E2,3E1,4.85E2,9E1,7E0,2.84E2,5.6E1,2.51E2,6.9E1,4.27E2,1.9E1,8.4E1,1.1E1,3E0,1.1E1,4E0,2.54E2,2E0,7E0,6E0,6.1E1,9.9E1,3.8E1,1.04E2,8E1,2.7E1,3E0,4.56E2,2.9E1,1.1E1,7.9E1,2.79E2,5E0,3.8E1,1.8E1,2.8E1,2.23E2,4.1E1,2.8E1,7.1E1,3.56E2,4E0,1.5E1,5.9E1,2.5E1,8E0,3E0,5E0,2.49E2,5.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.191869E-4,5.7219495E-3,-4.7454964E-2,-1.0996642E-2,3.648694E-2,-1.24135844E-1,-3.2339003E-2,-1.755248E-2,4.385029E-2,9.7798556E-2,2.5957005E-2,-7.240969E-2,-1.2597989E-2,-1.5674135E-2,-7.2853506E-2,-1.0120915E-2,-5.7817478E-2,7.173417E-2,-1.0708805E-2,1.2414128E-1,-8.2834415E-2,2.2127982E-2,1.255441E-1,-1.9368847E-3,-1.10282086E-1,-4.2948057E-3,5.1709507E-5,1.964069E-3,-8.6261004E-2,-1.4697574E-2,4.4655595E-2,-4.4090685E-2,-1.8739015E-1,-2.816523E-2,9.202699E-2,-7.878122E-3,1.1036517E-2,1.417953E-1,-1.2434712E-3,-2.0806082E-2,-7.525068E-5,2.9536547E-2,-1.6315576E-2,1.5478192E-2,7.491589E-2,5.3847E-3,-3.3699577E-3,-6.200131E-3,3.8283244E-3,1.6751887E-2,-9.1467336E-2,-6.977814E-2,-9.19769E-3,-1.8928327E-2,5.0551966E-2,1.9228399E-2,1.3732272E-1,-4.843383E-2,1.18347015E-2,-1.5765702E-2,-3.1327477E-3,-3.3883967E-3,7.146558E-3,-1.96626E-3,1.0732103E-1,7.014859E-3,-1.8797957E-4,1.5092318E-1,-4.961201E-3,7.8300975E-2,1.6309122E-2,-6.486333E-2,6.498581E-2,1.2615668E-3,8.392244E-3,3.406414E-2,-2.6081933E-2,-8.886817E-3,-7.8370015E-4,-9.590577E-2,3.749297E-3,9.565159E-5,-1.8297962E-3,-1.2386943E-2,3.087671E-3,1.4099659E-3,-7.6241856E-3,1.45658245E-2,1.7686107E-3,-7.82894E-3,-1.9064385E-3,-7.8194705E-4,5.8907233E-3,7.533757E-3,-1.3868688E-3,4.36843E-3,-1.7724967E-3,-1.3026588E-3,1.447021E-3,-2.2805235E-3,-8.450147E-3,6.8340497E-3,-5.819276E-4,8.016655E-4,6.9945436E-3,-4.315518E-3,9.7502884E-4,1.7219377E-3,-5.2724998E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,-1,-1,-1,69,71,-1,73,-1,-1,-1,-1,75,77,79,-1,81,83,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,-1,93,-1,95,97,99,101,-1,-1,103,105,-1,-1,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.0522984E-1,1.2096314E0,3.7153566E-1,5.487508E-1,5.3202426E-1,3.6180532E-1,1.8250397E-1,4.0696877E-1,2.487432E-1,5.8604527E-1,2.6719734E-1,1.06545374E-1,0E0,2.2578159E-1,1.24142855E-1,2.8920442E-1,3.688047E-1,2.2153401E-1,1.8909621E-1,2.845794E-1,4.5649242E-1,1.9500822E-1,2.3596749E-1,1.2531355E-1,1.0522139E-1,0E0,2.476021E-1,0E0,1.1040938E-1,2.9440302E-1,2.0706712E-1,2.5606948E-1,3.2441854E-1,1.5409806E-1,1.8759203E-1,0E0,1.0189596E-1,2.2842407E-1,0E0,0E0,0E0,3.6832392E-1,4.4233218E-1,0E0,1.0258898E-1,0E0,0E0,0E0,0E0,1.024808E-1,1.73639E-1,2.512327E-1,0E0,4.031097E-1,2.9647428E-1,1.3407022E-1,3.0698025E-1,1.8302217E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5908122E-1,0E0,0E0,1.1028576E-1,0E0,1.8507981E-1,2.7430484E-1,1.262869E-1,2.535136E-1,0E0,0E0,1.876906E-1,1.2078086E-1,0E0,0E0,1.0499188E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,41,41,42,42,44,44,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,64,64,67,67,69,69,70,70,71,71,72,72,75,75,76,76,79,79],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,-1,-1,-1,70,72,-1,74,-1,-1,-1,-1,76,78,80,-1,82,84,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,-1,94,-1,96,98,100,102,-1,-1,104,106,-1,-1,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2343444E7,5.862126E2,1.1328599E3,9.750871E6,1.0594161E1,1E0,1.0828989E8,9.837297E0,4.1E2,1.93974E7,3.832274E2,2.3428571E0,-1.2597989E-2,9.173084E2,8.7757526E2,3.8E1,9.561769E-1,4E0,3.943114E7,3.159399E6,3.14088E5,2.7183437E1,4.8083666E-1,5.7581736E7,5.441E3,-4.2948057E-3,9E0,1.964069E-3,2.88001E10,4.5723195E6,1.2224265E1,1.0409E4,6.137763E2,3.324E3,6.62E2,-7.878122E-3,1.0927339E7,5.7E1,-1.2434712E-3,-2.0806082E-2,-7.525068E-5,1.683653E8,7.06699E8,1.5478192E-2,1.0787629E0,5.3847E-3,-3.3699577E-3,-6.200131E-3,3.8283244E-3,5.019624E7,1.27365E4,7.1E1,-9.19769E-3,2.3416922E9,2.35E2,8.67822E-1,1.6681991E-3,5.664E1,1.18347015E-2,-1.5765702E-2,-3.1327477E-3,-3.3883967E-3,7.146558E-3,-1.96626E-3,1.8481675E0,7.014859E-3,-1.8797957E-4,8.5E1,-4.961201E-3,7.2201815E6,1E0,1.912391E6,4.155E3,1.2615668E-3,8.392244E-3,2.2216E4,7.808427E9,-8.886817E-3,-7.8370015E-4,1.1E1,3.749297E-3,9.565159E-5,-1.8297962E-3,-1.2386943E-2,3.087671E-3,1.4099659E-3,-7.6241856E-3,1.45658245E-2,1.7686107E-3,-7.82894E-3,-1.9064385E-3,-7.8194705E-4,5.8907233E-3,7.533757E-3,-1.3868688E-3,4.36843E-3,-1.7724967E-3,-1.3026588E-3,1.447021E-3,-2.2805235E-3,-8.450147E-3,6.8340497E-3,-5.819276E-4,8.016655E-4,6.9945436E-3,-4.315518E-3,9.7502884E-4,1.7219377E-3,-5.2724998E-3],"split_indices":[45,52,4,9,56,101,45,54,0,50,56,56,0,52,52,3,38,3,5,32,1,53,42,45,2,0,17,0,5,28,56,2,4,29,2,0,9,6,0,0,0,7,7,0,34,0,0,0,0,51,33,8,0,31,2,27,41,50,0,0,0,0,0,0,54,0,0,8,0,50,102,1,10,0,0,2,12,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.673E3,2.35E3,3.23E2,1.523E3,8.27E2,5.2E1,2.71E2,1.361E3,1.62E2,1.2E2,7.07E2,3.9E1,1.3E1,1.93E2,7.8E1,1.15E3,2.11E2,1.07E2,5.5E1,1.05E2,1.5E1,6.82E2,2.5E1,1.4E1,2.5E1,3.3E1,1.6E2,8E0,7E1,1.062E3,8.8E1,1.92E2,1.9E1,1.8E1,8.9E1,6E0,4.9E1,9.4E1,1.1E1,2E0,1.3E1,5.72E2,1.1E2,4E0,2.1E1,5E0,9E0,2.3E1,2E0,1.36E2,2.4E1,6.2E1,8E0,9.98E2,6.4E1,7E1,1.8E1,1.9E2,2E0,8E0,1.1E1,1.5E1,3E0,9E0,8E1,4E0,4.5E1,9.1E1,3E0,1.21E2,4.51E2,6.9E1,4.1E1,1.5E1,6E0,9.7E1,3.9E1,1E1,1.4E1,5.3E1,9E0,4.79E2,5.19E2,2E0,6.2E1,6.7E1,3E0,6E0,1.2E1,1.2E1,1.78E2,9E0,7.1E1,8.8E1,3E0,1.09E2,1.2E1,1.09E2,3.42E2,6.1E1,8E0,2E1,2.1E1,8.5E1,1.2E1,1.6E1,2.3E1,5E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-1.3900019E-3,-1.1255132E-2,2.73545E-2,-2.2133246E-2,1.2760066E-2,3.9756186E-2,-2.8867034E-2,-1.7614743E-2,-9.140465E-2,1.66642E-2,-7.6859975E-3,6.704436E-3,3.252154E-2,-1.0310641E-2,-8.305423E-2,-1.6482973E-2,-1.4810501E-2,-3.7816742E-1,-6.483827E-2,3.440376E-2,-8.893297E-3,1.142074E-1,2.5648283E-2,-3.4827532E-3,6.8297156E-4,-4.5929883E-2,-1.0012942E-2,-2.652305E-2,1.0224242E-2,-3.2783188E-2,-5.6849867E-3,6.8132686E-3,-8.380953E-2,1.7285544E-2,8.4284075E-2,-2.1321741E-1,-4.7712415E-3,4.0985093E-2,8.715659E-3,-1.6266054E-1,2.8307777E-2,4.037223E-3,-4.2725164E-3,-2.1719882E-2,-9.600978E-2,3.783808E-2,-1.8710366E-2,1.0441071E-3,-5.2315337E-3,2.4602009E-2,-1.10197775E-1,3.3671193E-2,1.6719481E-1,-1.1866164E-3,-1.5831877E-2,-1.2086632E-2,3.8293311E-3,5.4091145E-3,-2.3690618E-3,-1.3126214E-2,3.7005902E-3,4.7383428E-2,8.231538E-3,-8.316518E-4,-4.1234116E-3,-3.1758456E-3,-1.2461748E-2,-5.831956E-4,3.8955768E-3,-5.7977834E-3,-1.5333088E-4,2.8020244E-3,-9.778156E-5,2.3723063E-4,-8.98023E-3,2.3188407E-3,-1.2344854E-2,1.6121937E-2,5.4025436E-3,1.5649153E-3,-1.2417021E-3,5.1025758E-3,1.3025291E-3,-1.3173674E-3,2.0882448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,41,-1,43,45,-1,-1,-1,47,49,51,53,55,57,-1,59,61,-1,-1,63,65,67,69,-1,-1,71,73,75,77,-1,-1,79,-1,-1,-1,-1,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.701957E-1,5.2850986E-1,4.850335E-1,4.3409014E-1,4.2947972E-1,4.0808153E-1,1.2583902E-1,4.2663708E-1,6.230314E-1,2.8049403E-1,0E0,0E0,2.95761E-1,1.467277E-1,1.4011271E-1,3.4992906E-1,0E0,4.751855E-1,3.2231903E-1,3.1020272E-1,2.1291046E-1,1.9603598E-1,2.5100112E-1,0E0,0E0,1.540818E-1,0E0,3.1431574E-1,2.8594932E-1,0E0,0E0,0E0,1.984368E-1,2.5734562E-1,3.8463098E-1,1.0239215E-1,1.5592712E-1,1.4228132E-1,0E0,2.0832443E-1,1.8573204E-1,0E0,0E0,2.4743545E-1,2.8018236E-1,3.9918926E-1,2.7739984E-1,0E0,0E0,2.3410234E-1,1.3447797E-1,2.6608258E-1,2.8455818E-1,0E0,0E0,1.4397147E-1,0E0,0E0,0E0,0E0,0E0,2.944659E-1,3.020816E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,27,27,28,28,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,55,55,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,42,-1,44,46,-1,-1,-1,48,50,52,54,56,58,-1,60,62,-1,-1,64,66,68,70,-1,-1,72,74,76,78,-1,-1,80,-1,-1,-1,-1,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0989723E3,1.273801E6,6.747114E7,9.60094E5,1E0,2.139E3,3.01968E5,1.6904226E7,4E0,2.7E1,-7.6859975E-3,6.704436E-3,9.532E3,1.3715873E3,1.4299594E8,1.5107028E-1,-1.4810501E-2,1.964127E6,3.7344142E-3,4.3689142E2,1.5687E4,4.9807886E3,1.43E2,-3.4827532E-3,6.8297156E-4,5.3474556E5,-1.0012942E-2,9.793811E-2,2.4013288E1,-3.2783188E-2,-5.6849867E-3,6.8132686E-3,2.5378451E0,1E0,8.84E2,7.56484E5,5.96E2,2.3058404E7,8.715659E-3,1.8E1,1.778482E7,4.037223E-3,-4.2725164E-3,6.971004E7,6.949118E-1,2.8080197E2,6.63E2,1.0441071E-3,-5.2315337E-3,1E0,2.315878E6,3.7852024E1,1.526038E6,-1.1866164E-3,-1.5831877E-2,9.87156E0,3.8293311E-3,5.4091145E-3,-2.3690618E-3,-1.3126214E-2,3.7005902E-3,2.7203197E0,1.3532244E7,-8.316518E-4,-4.1234116E-3,-3.1758456E-3,-1.2461748E-2,-5.831956E-4,3.8955768E-3,-5.7977834E-3,-1.5333088E-4,2.8020244E-3,-9.778156E-5,2.3723063E-4,-8.98023E-3,2.3188407E-3,-1.2344854E-2,1.6121937E-2,5.4025436E-3,1.5649153E-3,-1.2417021E-3,5.1025758E-3,1.3025291E-3,-1.3173674E-3,2.0882448E-3],"split_indices":[52,9,45,9,85,29,11,47,3,8,0,0,9,52,45,38,0,1,42,52,1,4,0,0,0,32,0,38,58,0,0,0,54,74,10,12,8,45,0,3,50,0,0,45,27,4,10,0,0,16,9,53,9,0,0,58,0,0,0,0,0,53,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.714E3,2.021E3,6.93E2,1.391E3,6.3E2,5.68E2,1.25E2,1.307E3,8.4E1,6.17E2,1.3E1,3.7E1,5.31E2,9.4E1,3.1E1,1.303E3,4E0,6E0,7.8E1,3.64E2,2.53E2,4E1,4.91E2,2.6E1,6.8E1,2.5E1,6E0,9.47E2,3.56E2,2E0,4E0,6E0,7.2E1,2.72E2,9.2E1,4E0,2.49E2,2E1,2E1,6E0,4.85E2,6E0,1.9E1,8.87E2,6E1,1.82E2,1.74E2,1.4E1,5.8E1,2.58E2,1.4E1,5.8E1,3.4E1,2E0,2E0,2.3E2,1.9E1,1.1E1,9E0,4E0,2E0,2.48E2,2.37E2,8.32E2,5.5E1,5.2E1,8E0,8.5E1,9.7E1,2.2E1,1.52E2,1.13E2,1.45E2,6E0,8E0,5.6E1,2E0,7E0,2.7E1,5.4E1,1.76E2,6.2E1,1.86E2,1.18E2,1.19E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-9.42093E-4,-1.0269936E-2,2.6848795E-2,-3.2133204E-3,-5.5798084E-2,3.9437436E-2,-1.9263199E-2,-6.2390894E-2,1.5729344E-3,-8.365326E-2,-2.2550879E-2,3.3939965E-2,1.4890884E-1,-1.02001026E-1,1.0026623E-2,-2.1810383E-1,-4.759728E-2,1.8253697E-1,-4.754161E-4,-6.2222775E-2,-7.1759513E-3,3.9225303E-2,-5.1356718E-2,6.950085E-2,1.9002955E-2,2.117591E-1,3.5728277E-3,-1.3749518E-1,-1.1395521E-4,3.181979E-2,-8.7454304E-2,-5.468184E-3,-1.7787682E-2,6.2922486E-3,-5.9847604E-2,-2.574821E-3,2.3338163E-1,1.02103844E-1,-2.9834833E-3,-1.060101E-2,-5.112676E-2,7.430462E-3,-4.8361078E-4,-2.1008137E-1,-3.2137517E-2,2.421696E-2,9.809699E-2,4.197099E-2,-9.2188185E-3,1.2005438E-2,-7.449099E-5,2.8561943E-4,-8.090687E-3,6.63662E-5,1.0551041E-1,-1.7437096E-1,-2.3407923E-5,-5.1662844E-2,-1.2928845E-2,1.1251019E-3,1.359593E-2,2.4677124E-3,9.35786E-3,-1.1543275E-2,2.9295556E-2,-3.68488E-2,-8.836297E-3,-4.380566E-3,-1.6066197E-2,2.7728886E-3,-4.9650706E-2,8.977288E-2,-1.1703331E-2,1.1604693E-1,3.2104415E-4,1.1539923E-2,3.4458287E-2,8.729137E-2,-1.9004084E-2,6.800164E-3,-3.6453793E-3,-8.204114E-5,-1.1882718E-2,-3.0424157E-3,4.2409617E-3,-4.929855E-4,-1.5100146E-2,3.1278091E-3,-4.9669173E-4,1.518936E-3,-3.1874238E-3,-5.4462412E-3,-5.677891E-4,8.036697E-3,8.2647754E-4,6.033123E-3,-1.4864083E-3,6.2632924E-3,-3.9223552E-4,4.3917028E-4,4.223964E-3,6.3702683E-3,-6.7023453E-3,-3.6129553E-3,3.1866322E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,-1,53,55,-1,-1,-1,57,-1,59,61,63,-1,65,-1,-1,67,69,71,73,75,77,-1,-1,-1,-1,-1,79,81,-1,83,-1,-1,-1,-1,-1,85,87,89,-1,-1,-1,-1,91,93,95,97,-1,-1,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.942318E-1,6.4397633E-1,3.9142767E-1,4.9208415E-1,2.4723619E-1,3.1330657E-1,3.524123E-1,2.8908122E-1,5.9613675E-1,1.9776702E-1,2.2267433E-1,2.6692176E-1,1.0095388E-1,1.3404006E-1,2.3221995E-1,1.3285857E-1,2.7097872E-1,2.2881186E-1,4.0963024E-1,1.8645003E-1,0E0,2.3028962E-1,2.5254622E-1,1.9057107E-1,2.3179387E-1,1.0833478E-1,0E0,1.2732202E-1,0E0,1.9945413E-1,1.510898E-1,0E0,0E0,0E0,1.803872E-1,0E0,1.5236598E-1,1.6923687E-1,4.297112E-1,0E0,1.950722E-1,0E0,0E0,1.0317394E-1,1.2429257E-1,1.3998312E-1,1.4850801E-1,2.8620774E-1,1.5370372E-1,0E0,0E0,0E0,0E0,0E0,1.8309674E-1,1.2691271E-1,0E0,1.886422E-1,0E0,0E0,0E0,0E0,0E0,4.6313572E-1,4.664166E-1,1.9969256E-1,0E0,0E0,0E0,0E0,1.5605463E-1,1.1594634E-1,1.07726395E-1,1.4021432E-1,0E0,0E0,2.5876594E-1,1.7447913E-1,1.6323543E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,34,34,36,36,37,37,38,38,40,40,43,43,44,44,45,45,46,46,47,47,48,48,54,54,55,55,57,57,63,63,64,64,65,65,70,70,71,71,72,72,73,73,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,-1,54,56,-1,-1,-1,58,-1,60,62,64,-1,66,-1,-1,68,70,72,74,76,78,-1,-1,-1,-1,-1,80,82,-1,84,-1,-1,-1,-1,-1,86,88,90,-1,-1,-1,-1,92,94,96,98,-1,-1,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,2.6944466E7,3.666495E7,4.0701206E1,1.6667E4,3.1699734E2,2.8316305E0,7.2727275E-1,3.9333332E1,2.7448502E8,2.1525E4,6.85945E4,5.7719237E-1,5.2E1,1.75E2,3.79E2,2E0,9.21641E3,1.0620689E0,1.752512E2,-7.1759513E-3,3.1665432E0,3E0,1.0989723E3,1.9051096E7,5.8E1,3.5728277E-3,2.777588E6,-1.1395521E-4,1.5005797E4,3.555E3,-5.468184E-3,-1.7787682E-2,6.2922486E-3,1.1603518E8,-2.574821E-3,1.7848537E0,1.5053125E2,5.862126E2,-1.060101E-2,1.1347826E1,7.430462E-3,-4.8361078E-4,2.2515285E6,2.0892E-5,6.341735E2,4.2343444E7,6.439778E5,4.35581E5,1.2005438E-2,-7.449099E-5,2.8561943E-4,-8.090687E-3,6.63662E-5,1.066317E12,2.656E3,-2.3407923E-5,1.16146E5,-1.2928845E-2,1.1251019E-3,1.359593E-2,2.4677124E-3,9.35786E-3,5.8018835E2,5.3506675E6,1.8159722E-1,-8.836297E-3,-4.380566E-3,-1.6066197E-2,2.7728886E-3,6.63E2,1.5736016E6,8E-1,2.5492362E7,3.2104415E-4,1.1539923E-2,4.95E9,5.0818306E-1,2.3023027E3,6.800164E-3,-3.6453793E-3,-8.204114E-5,-1.1882718E-2,-3.0424157E-3,4.2409617E-3,-4.929855E-4,-1.5100146E-2,3.1278091E-3,-4.9669173E-4,1.518936E-3,-3.1874238E-3,-5.4462412E-3,-5.677891E-4,8.036697E-3,8.2647754E-4,6.033123E-3,-1.4864083E-3,6.2632924E-3,-3.9223552E-4,4.3917028E-4,4.223964E-3,6.3702683E-3,-6.7023453E-3,-3.6129553E-3,3.1866322E-5],"split_indices":[2,45,51,52,9,56,54,53,50,7,9,32,42,8,8,1,17,28,54,52,0,56,3,52,45,3,0,1,0,4,0,0,0,0,12,0,54,52,52,0,53,0,0,28,38,52,45,51,29,0,0,0,0,0,31,0,0,1,0,0,0,0,0,52,50,27,0,0,0,0,10,50,53,50,0,0,5,39,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.676E3,2.004E3,6.72E2,1.736E3,2.68E2,5.28E2,1.44E2,1.29E2,1.607E3,1.45E2,1.23E2,5.04E2,2.4E1,3.7E1,1.07E2,1E1,1.19E2,1.7E1,1.59E3,1.11E2,3.4E1,3.9E1,8.4E1,1.48E2,3.56E2,1.2E1,1.2E1,2.7E1,1E1,8.8E1,1.9E1,7E0,3E0,7E0,1.12E2,3E0,1.4E1,3.7E1,1.553E3,6E0,1.05E2,1.1E1,2.8E1,8E0,7.6E1,5.8E1,9E1,1.96E2,1.6E2,1E1,2E0,5E0,2.2E1,6.3E1,2.5E1,9E0,1E1,1.09E2,3E0,3E0,1.1E1,2.5E1,1.2E1,1.228E3,3.25E2,9.6E1,9E0,5E0,3E0,1.2E1,6.4E1,2E1,3.8E1,7.5E1,1.5E1,6E0,1.9E2,1.4E1,1.46E2,2.1E1,4E0,3E0,6E0,1.01E2,8E0,1.224E3,4E0,1.7E2,1.55E2,2.9E1,6.7E1,2.3E1,4.1E1,9E0,1.1E1,4E0,3.4E1,6.7E1,8E0,1.3E2,6E1,1.2E1,2E0,3.7E1,1.09E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-4.1469975E-5,-8.570121E-3,2.4705455E-2,-1.9341419E-2,1.6644098E-2,6.571429E-2,1.2190902E-2,-9.68403E-3,-6.3361935E-2,6.0850527E-2,-3.7579853E-3,2.6992334E-3,7.975073E-2,1.7474692E-2,-1.14948966E-1,-2.2701949E-2,2.6384395E-2,1.2796186E-2,-6.847931E-2,5.1210597E-2,8.90467E-3,1.0108443E-3,-8.385356E-3,-4.354762E-3,2.5890209E-3,1.4707677E-1,6.311959E-2,2.5137894E-2,-4.062022E-2,-2.8776708E-3,-1.5951563E-2,-2.7899805E-2,2.4038649E-3,3.7899524E-2,-6.5496616E-2,-2.6724523E-1,-6.067522E-2,7.735741E-2,-1.5078407E-3,7.8688115E-2,-7.214078E-3,1.4918168E-2,3.7717083E-3,-2.2200262E-3,7.375504E-2,1.5048365E-2,6.9619E-2,-6.1465073E-3,-7.419049E-4,3.52969E-2,-3.4762565E-2,6.7535095E-2,-4.2461865E-3,4.4878755E-2,-4.845774E-3,-1.6547771E-2,-3.975008E-3,4.203245E-2,-7.2927445E-2,-3.9415895E-3,8.983077E-2,-1.3296394E-2,1.8242907E-2,1.2480075E-3,1.1751065E-2,-3.8920145E-2,1.3956699E-2,9.949706E-4,1.0468081E-1,4.6742108E-2,-1.0318309E-2,7.7429645E-2,-7.7521E-3,-1.3868338E-3,4.155791E-3,-3.8695594E-3,-1.1875863E-3,4.7419703E-4,4.470718E-3,1.8224358E-3,-2.0540163E-3,-7.7015925E-3,6.0640266E-3,9.652642E-3,1.8516665E-4,-4.0700664E-3,3.1215508E-4,4.883569E-3,-2.7019691E-3,3.7968731E-3,-2.6453065E-3,-2.7578396E-3,1.4835856E-3,3.2092386E-3,-7.0588005E-4,6.2476736E-3,-1.1659035E-3,1.1502858E-2,1.8084472E-3,8.186828E-6,-5.090935E-3,4.6291742E-3,-2.6528517E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,41,43,45,47,-1,-1,49,-1,51,53,55,57,59,61,63,65,-1,-1,-1,67,69,71,-1,-1,73,75,77,79,81,-1,-1,-1,83,85,-1,87,-1,89,-1,-1,91,93,-1,95,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.607814E-1,5.369835E-1,3.48693E-1,5.881821E-1,5.3415537E-1,1.4042139E-1,3.5334983E-1,5.344612E-1,4.3950295E-1,2.1596909E-1,3.319087E-1,1.4918782E-1,1.3627875E-1,2.249532E-1,2.3129207E-1,3.1816733E-1,3.2200485E-1,0E0,3.6537683E-1,2.4133995E-1,0E0,2.5356793E-1,0E0,0E0,0E0,2.5631076E-1,1.2683752E-1,1.9866538E-1,1.2807095E-1,0E0,0E0,3.399498E-1,0E0,3.3627102E-1,1.3846874E-1,1.01136446E-1,3.0256402E-1,2.4040091E-1,3.2684174E-1,3.26957E-1,2.4151912E-1,0E0,0E0,0E0,1.5675181E-1,2.931255E-1,1.6079688E-1,0E0,0E0,2.5721827E-1,3.2141292E-1,2.3417419E-1,1.8471186E-1,1.7169206E-1,0E0,0E0,0E0,1.5751877E-1,2.0462227E-1,0E0,1.9932544E-1,0E0,2.5554723E-1,0E0,0E0,1.8966702E-1,3.298437E-1,0E0,2.0605266E-1,2.723514E-1,2.0440143E-1,1.281878E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,21,21,25,25,26,26,27,27,28,28,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,57,57,58,58,60,60,62,62,65,65,66,66,68,68,69,69,70,70,71,71],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,42,44,46,48,-1,-1,50,-1,52,54,56,58,60,62,64,66,-1,-1,-1,68,70,72,-1,-1,74,76,78,80,82,-1,-1,-1,84,86,-1,88,-1,90,-1,-1,92,94,-1,96,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,1.273801E6,2.909019E3,3.38464E5,1.5277338E0,7.57657E4,1.2367184E3,1.5107028E-1,3.938775E4,4.94374E1,1E0,2.5629143E3,1.6081998E5,1.9840434E7,9.599108E6,3.454E3,2.1199985E-7,1.2796186E-2,2.9143541E-6,5.4803147E0,8.90467E-3,4.5680464E3,-8.385356E-3,-4.354762E-3,2.5890209E-3,4.8E1,2.4513194E5,8.015419E3,1.6108601E7,-2.8776708E-3,-1.5951563E-2,6E0,2.4038649E-3,2.558106E10,4.0434834E5,1.4E1,4.1504726E-3,7.8E3,7.79E2,6.0559247E2,6.98E2,1.4918168E-2,3.7717083E-3,-2.2200262E-3,1.3999657E3,1.9051096E7,1E0,-6.1465073E-3,-7.419049E-4,3.79E2,2.6216663E-5,1.8991614E5,1.5773E4,3.8527173E-1,-4.845774E-3,-1.6547771E-2,-3.975008E-3,2.2444444E0,4.832E3,-3.9415895E-3,2.507E3,-1.3296394E-2,9E0,1.2480075E-3,1.1751065E-2,1.1994927E7,1.9E1,9.949706E-4,1.24900505E2,6.439778E5,1.8834772E7,6.747114E7,-7.7521E-3,-1.3868338E-3,4.155791E-3,-3.8695594E-3,-1.1875863E-3,4.7419703E-4,4.470718E-3,1.8224358E-3,-2.0540163E-3,-7.7015925E-3,6.0640266E-3,9.652642E-3,1.8516665E-4,-4.0700664E-3,3.1215508E-4,4.883569E-3,-2.7019691E-3,3.7968731E-3,-2.6453065E-3,-2.7578396E-3,1.4835856E-3,3.2092386E-3,-7.0588005E-4,6.2476736E-3,-1.1659035E-3,1.1502858E-2,1.8084472E-3,8.186828E-6,-5.090935E-3,4.6291742E-3,-2.6528517E-4],"split_indices":[2,9,32,9,53,28,58,38,45,58,85,4,28,9,29,0,37,0,38,56,0,50,0,0,0,3,28,4,32,0,0,10,0,19,28,0,41,1,2,33,10,0,0,0,52,45,105,0,0,1,38,28,9,41,0,0,0,56,2,0,2,0,8,0,0,9,8,0,58,51,43,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.655E3,1.975E3,6.8E2,1.384E3,5.91E2,1.58E2,5.22E2,1.136E3,2.48E2,1.86E2,4.05E2,2.9E1,1.29E2,5.02E2,2E1,8.35E2,3.01E2,3E0,2.45E2,1.74E2,1.2E1,3.95E2,1E1,1E1,1.9E1,2.4E1,1.05E2,4.44E2,5.8E1,1.7E1,3E0,7.8E2,5.5E1,2.68E2,3.3E1,8E0,2.37E2,1.16E2,5.8E1,3.7E1,3.58E2,6E0,1.8E1,9E0,9.6E1,3.63E2,8.1E1,1.2E1,4.6E1,7.6E1,7.04E2,1.57E2,1.11E2,8E0,2.5E1,5E0,3E0,2.5E1,2.12E2,8E0,1.08E2,3E0,5.5E1,2.9E1,8E0,1.43E2,2.15E2,3.6E1,6E1,1.61E2,2.02E2,7.9E1,2E0,3.4E1,4.2E1,1.24E2,5.8E2,4.9E1,1.08E2,5.3E1,5.8E1,2E0,6E0,4E0,2.1E1,1.84E2,2.8E1,1E2,8E0,3E1,2.5E1,1.13E2,3E1,7.5E1,1.4E2,5E1,1E1,6E0,1.55E2,1.83E2,1.9E1,6.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[9.198005E-4,4.8439465E-3,-5.11713E-2,-6.579271E-3,2.8507423E-2,-7.375152E-2,-5.913907E-3,-1.1211438E-1,-4.5418767E-3,4.7929484E-2,5.823528E-3,-5.4088093E-2,-1.5603808E-1,3.4185268E-2,-9.84639E-2,3.7861988E-3,-1.3542083E-1,4.862542E-2,-9.444326E-3,7.563755E-2,2.715072E-2,-1.04328925E-2,5.579247E-2,-4.1107688E-2,-1.6680098E-1,-2.7865607E-3,-1.1536205E-2,9.122621E-3,9.261993E-3,-1.0941584E-2,-1.7103421E-3,-2.3526622E-3,-9.021094E-3,1.030198E-2,1.01559624E-1,-1.3420478E-1,-8.069678E-3,1.1045616E-1,2.9987702E-3,-1.4942914E-1,3.5300627E-2,-7.458258E-3,-1.3391635E-2,-3.4805841E-3,6.583414E-2,-7.464208E-2,7.702907E-5,1.86752E-3,-1.0685555E-2,3.777007E-3,-2.0395743E-3,2.9705465E-2,-1.3729803E-2,1.7369556E-1,5.4655455E-2,-4.4932544E-2,-2.4236135E-2,7.659578E-3,-1.9034194E-2,1.6395171E-1,8.069363E-2,-1.5343296E-3,6.9832434E-3,-2.1145646E-3,-1.9895859E-2,1.05102025E-1,1.8215546E-2,-1.0683364E-2,8.085559E-3,8.149354E-2,-1.672021E-3,-4.2315875E-3,3.3155053E-3,9.468568E-3,3.3678385E-4,9.86362E-3,1.0918102E-3,3.8370893E-3,-1.2077263E-2,-7.175806E-3,8.326449E-3,8.605302E-4,-2.202211E-3,-5.237995E-3,-5.688547E-4,3.2000553E-3,1.140098E-2,-5.939246E-3,4.317671E-3,6.7625375E-4,7.334658E-3,5.8310974E-4,1.1457703E-2,-9.494054E-3,-2.3749942E-4,8.325102E-3,2.8963252E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,-1,-1,-1,49,-1,-1,-1,-1,51,53,55,57,59,61,63,65,67,-1,-1,69,71,-1,-1,-1,-1,-1,73,-1,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.339704E-1,6.567779E-1,1.8657556E-1,3.5190305E-1,3.4821016E-1,1.8896055E-1,2.3345907E-1,1.488294E-1,4.1984504E-1,2.4321139E-1,2.978898E-1,1.3944909E-1,1.7694426E-1,1.6964215E-1,1.468419E-1,0E0,1.1868286E-1,2.7484617E-1,2.5171053E-1,4.5997965E-1,3.5785675E-1,2.1972586E-1,1.2088978E-1,1.3047382E-1,1.246129E-1,0E0,0E0,0E0,1.4411229E-1,0E0,0E0,0E0,0E0,4.695327E-1,1.8336475E-1,5.084218E-1,2.5177032E-1,1.8260646E-1,3.0490685E-1,2.897089E-1,2.783377E-1,1.5815821E-1,0E0,0E0,1.3536748E-1,1.0676971E-1,0E0,0E0,0E0,0E0,0E0,2.881122E-1,0E0,1.0242015E-1,3.0151618E-1,3.47392E-1,0E0,3.3263847E-1,5.546659E-1,2.936312E-1,1.6809994E-1,0E0,0E0,0E0,0E0,1.9894409E-1,2.507292E-1,2.8997067E-1,0E0,1.3115239E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,44,44,45,45,51,51,53,53,54,54,55,55,57,57,58,58,59,59,60,60,65,65,66,66,67,67,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,-1,-1,-1,50,-1,-1,-1,-1,52,54,56,58,60,62,64,66,68,-1,-1,70,72,-1,-1,-1,-1,-1,74,-1,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,7.298614E2,1.0052E4,1.0882353E1,1.504779E7,9.222717E-1,6.9998717E-4,1E0,1E0,6.627257E6,9.908038E1,1.0192391E2,1.2199979E10,8.24666E5,3.4409692E3,3.7861988E-3,2.645E3,5.69815E5,1.0195312E0,5.4770195E1,3.171E3,9.195E1,1E1,1.249246E6,6.13E2,-2.7865607E-3,-1.1536205E-2,9.122621E-3,4.7777777E0,-1.0941584E-2,-1.7103421E-3,-2.3526622E-3,-9.021094E-3,4.9422E5,3.011152E6,2.8781365E6,5.372E3,1.5938586E8,4.0602388E6,2.857E3,1.5595E4,6.001692E-7,-1.3391635E-2,-3.4805841E-3,2.8721272E1,7.221312E-2,7.702907E-5,1.86752E-3,-1.0685555E-2,3.777007E-3,-2.0395743E-3,2.6307692E1,-1.3729803E-2,3.06E2,1.3794063E1,2.7460318E0,-2.4236135E-2,1E0,7.59E2,3.275E3,2.41E3,-1.5343296E-3,6.9832434E-3,-2.1145646E-3,-1.9895859E-2,1.6554546E3,1.921032E4,9.8708276E2,8.085559E-3,2.1853803E4,-1.672021E-3,-4.2315875E-3,3.3155053E-3,9.468568E-3,3.3678385E-4,9.86362E-3,1.0918102E-3,3.8370893E-3,-1.2077263E-2,-7.175806E-3,8.326449E-3,8.605302E-4,-2.202211E-3,-5.237995E-3,-5.688547E-4,3.2000553E-3,1.140098E-2,-5.939246E-3,4.317671E-3,6.7625375E-4,7.334658E-3,5.8310974E-4,1.1457703E-2,-9.494054E-3,-2.3749942E-4,8.325102E-3,2.8963252E-3],"split_indices":[45,52,2,52,45,27,34,75,53,51,56,58,5,29,52,0,9,9,53,54,2,53,3,1,0,0,0,0,54,0,0,0,0,9,9,47,29,7,48,2,9,37,0,0,53,38,0,0,0,0,0,47,0,0,56,58,0,105,2,2,2,0,0,0,0,55,4,4,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E3,2.428E3,1.82E2,1.638E3,7.9E2,1.21E2,6.1E1,3E1,1.608E3,4.25E2,3.65E2,9.9E1,2.2E1,4.3E1,1.8E1,3E0,2.7E1,1.35E2,1.473E3,1.81E2,2.44E2,2.76E2,8.9E1,9E1,9E0,1.1E1,1.1E1,5E0,3.8E1,5E0,1.3E1,1.1E1,1.6E1,7.9E1,5.6E1,1.5E1,1.458E3,1.22E2,5.9E1,1E1,2.34E2,2.74E2,2E0,6E0,8.3E1,5E1,4E1,2E0,7E0,1.6E1,2.2E1,7.5E1,4E0,2.1E1,3.5E1,1.3E1,2E0,5.99E2,8.59E2,4.2E1,8E1,4.8E1,1.1E1,8E0,2E0,4.5E1,1.89E2,2.7E2,4E0,7.2E1,1.1E1,4.6E1,4E0,8E0,6.7E1,1.7E1,4E0,3.3E1,2E0,9E0,4E0,5.03E2,9.6E1,6.2E1,7.97E2,1.9E1,2.3E1,3E0,7.7E1,1.6E1,2.9E1,1.85E2,4E0,7E0,2.63E2,1.2E1,6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[5.3759076E-4,4.010311E-3,-4.798753E-2,-1.3766378E-2,1.7333908E-2,-7.588496E-2,6.7814942E-3,3.9472856E-2,-1.8568153E-2,4.7465187E-2,6.9286674E-3,-5.710561E-2,-1.5260199E-1,2.401453E-3,-1.4560716E-3,-1.1556385E-1,8.258921E-2,-6.3261084E-2,-1.2366538E-2,5.981223E-2,-1.1581077E-2,3.117737E-2,-4.953725E-3,-3.58478E-3,2.2072897E-3,-2.0790774E-1,1.682053E-4,1.5077149E-3,-9.442308E-3,1.5370573E-2,5.9668496E-2,-1.1195473E-1,2.7670525E-2,1.6494092E-2,-3.2572985E-2,7.973781E-2,1.2961323E-3,7.6934725E-2,-8.5022554E-2,5.0297886E-2,-1.928334E-2,-8.581442E-2,4.2595747E-3,-2.8248925E-3,-1.2693039E-2,1.0557215E-1,-5.050949E-4,-2.0541437E-2,-2.1516599E-1,5.9916847E-3,-1.8025972E-3,-1.5911703E-1,2.4037817E-2,-8.077747E-2,-1.4005123E-2,8.3405204E-2,-1.0743794E-2,1.259929E-2,-1.0238417E-2,6.086133E-3,7.5613025E-5,-3.1054965E-1,-2.956548E-2,4.4659078E-2,1.0546614E-2,-9.3632685E-3,-6.21206E-3,-7.827584E-3,-1.247326E-3,8.432361E-4,1.065469E-1,3.3608207E-3,9.219241E-3,-1.8018042E-3,1.0524379E-2,-1.360826E-2,-3.1582182E-3,-4.8306882E-3,2.0917684E-3,-1.2036117E-2,-8.539927E-4,-1.9537578E-3,1.6561705E-3,-2.9522663E-3,-1.3624409E-2,6.851132E-4,-1.8854869E-3,5.179243E-3,2.09883E-3,-2.1360675E-4,7.5745326E-3,-2.8918048E-3,-1.908626E-2,3.4969053E-3,-3.0631498E-3,3.4407133E-3,9.675387E-4,-9.6753903E-4,4.3889163E-3,2.6943293E-4,-4.893813E-3,6.841654E-4,9.057139E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,73,75,-1,77,79,81,83,85,87,-1,89,-1,-1,-1,91,93,95,-1,97,-1,-1,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.566973E-1,5.992308E-1,2.770672E-1,2.7784967E-1,4.5302567E-1,1.6416907E-1,1.0149876E-1,6.1115986E-1,2.7500015E-1,2.71169E-1,3.101851E-1,1.829392E-1,1.9931269E-1,0E0,0E0,2.4618852E-1,3.6875427E-1,5.388607E-1,5.108117E-1,3.5784864E-1,4.290887E-1,3.4252146E-1,5.389961E-1,0E0,0E0,1.2901258E-1,0E0,0E0,0E0,0E0,2.1384083E-1,7.409358E-1,1.2371752E-1,4.8206395E-1,4.5982152E-1,2.7408612E-1,1.9475277E-1,1.13171145E-1,4.41073E-1,2.3641646E-1,1.0614222E-1,3.3910662E-1,2.2705145E-1,0E0,0E0,1.0957217E-1,0E0,1.8477455E-1,3.62149E-1,0E0,1.5759714E-1,1.909334E-1,2.3863739E-1,5.397769E-1,2.684239E-1,2.175597E-1,0E0,1.8943374E-1,0E0,0E0,0E0,1.3001889E-1,1.1052498E-1,1.6476512E-1,0E0,1.0148863E-1,0E0,0E0,0E0,3.1102204E-1,1.5728511E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,57,57,61,61,62,62,63,63,65,65,69,69,70,70],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,74,76,-1,78,80,82,84,86,88,-1,90,-1,-1,-1,92,94,96,-1,98,-1,-1,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,3.3030225E2,1.0052E4,8.695652E-1,8E0,1.29175E5,1.00663277E9,2.35E2,1E0,2.3523688E0,2.1111E4,9.245905E-2,3.285E3,2.401453E-3,-1.4560716E-3,1.00033E5,2.04E5,1.307E3,3.98231E-2,1E0,2.53383E5,1.1900813E1,3E0,-3.58478E-3,2.2072897E-3,8.48106E5,1.682053E-4,1.5077149E-3,-9.442308E-3,1.5370573E-2,4.3440155E6,4.3E2,1.25218E5,1.418705E0,1.7247218E-1,3.7852024E1,4.5709072E1,2.4708056E5,3.90106E0,1.22159E5,3.275343E6,5.695737E0,3.832274E2,-2.8248925E-3,-1.2693039E-2,1.3181542E6,-5.050949E-4,7.920064E5,5.750032E3,5.9916847E-3,3.2E1,1.216E3,1.4431512E-1,1E0,1.0177216E1,2.462277E6,-1.0743794E-2,2.4079119E1,-1.0238417E-2,6.086133E-3,7.5613025E-5,1E1,2.8982632E0,2.342147E7,1.0546614E-2,4.0889E4,-6.21206E-3,-7.827584E-3,-1.247326E-3,1.0963991E7,1.6E1,3.3608207E-3,9.219241E-3,-1.8018042E-3,1.0524379E-2,-1.360826E-2,-3.1582182E-3,-4.8306882E-3,2.0917684E-3,-1.2036117E-2,-8.539927E-4,-1.9537578E-3,1.6561705E-3,-2.9522663E-3,-1.3624409E-2,6.851132E-4,-1.8854869E-3,5.179243E-3,2.09883E-3,-2.1360675E-4,7.5745326E-3,-2.8918048E-3,-1.908626E-2,3.4969053E-3,-3.0631498E-3,3.4407133E-3,9.675387E-4,-9.6753903E-4,4.3889163E-3,2.6943293E-4,-4.893813E-3,6.841654E-4,9.057139E-3],"split_indices":[45,52,2,53,17,11,7,2,64,42,9,41,0,0,0,5,5,0,42,14,9,54,8,0,0,29,0,0,0,0,45,29,29,54,41,53,56,33,56,2,29,54,56,0,0,47,0,28,33,0,3,10,58,105,58,28,0,56,0,0,0,3,41,45,0,2,0,0,0,47,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.708E3,2.528E3,1.8E2,1.083E3,1.445E3,1.19E2,6.1E1,8.9E1,9.94E2,3.7E2,1.075E3,9.7E1,2.2E1,2.8E1,3.3E1,1.9E1,7E1,1.2E2,8.74E2,3.06E2,6.4E1,3.53E2,7.22E2,8.3E1,1.4E1,1.6E1,6E0,7E0,1.2E1,5E0,6.5E1,7.8E1,4.2E1,3.6E2,5.14E2,2.28E2,7.8E1,2.9E1,3.5E1,2.56E2,9.7E1,7.3E1,6.49E2,5E0,1.1E1,3.9E1,2.6E1,4.2E1,3.6E1,9E0,3.3E1,1.4E1,3.46E2,1.42E2,3.72E2,2.26E2,2E0,7.5E1,3E0,1.7E1,1.2E1,6E0,2.9E1,2.49E2,7E0,9E1,7E0,3.1E1,4.2E1,6.29E2,2E1,2.9E1,1E1,4E1,2E0,2.4E1,1.2E1,1E1,2.3E1,8E0,6E0,4.8E1,2.98E2,1.31E2,1.1E1,1.76E2,1.96E2,1.38E2,8.8E1,6.8E1,7E0,2E0,4E0,7E0,2.2E1,1.17E2,1.32E2,8.2E1,8E0,6.02E2,2.7E1,1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[2.820341E-4,-6.144251E-3,3.573406E-2,-1.6163193E-2,1.505748E-2,7.712124E-2,2.0418983E-2,-6.4341314E-3,-6.0775254E-2,9.5607325E-2,1.02976505E-2,1.14065506E-1,-1.098812E-2,4.481024E-3,7.590156E-2,7.7521903E-3,-2.3062099E-2,-4.3109935E-2,-1.7515086E-1,1.8708764E-3,1.4266303E-1,6.81587E-2,4.440545E-3,1.3519557E-1,7.134519E-4,-7.697484E-3,2.0372026E-2,1.4732771E-2,-6.444944E-2,1.315196E-2,5.224077E-2,-9.380402E-3,3.3146963E-2,-1.2642719E-1,-1.490925E-2,-7.118995E-2,2.5917547E-2,-2.1300618E-2,-1.3827862E-1,1.0179894E-2,8.655641E-4,-1.1713222E-1,8.584661E-2,-1.5330375E-2,5.97412E-3,8.032844E-3,3.279822E-3,-2.9594412E-3,4.3578353E-3,6.358082E-3,8.727579E-3,-6.265976E-3,-4.421335E-4,7.5499415E-2,-2.9911632E-3,2.1967273E-3,-8.3430566E-2,4.3483034E-2,-8.3788574E-2,-3.5728917E-1,-7.933365E-2,-1.9258768E-3,-4.7521792E-2,-4.0776905E-1,-5.874551E-2,1.1426508E-1,-3.7804913E-2,-1.6229372E-3,-1.8673094E-1,6.5310707E-4,-1.2089859E-2,1.043574E-2,3.0591777E-3,2.266394E-2,-1.2222627E-2,1.3669271E-2,-1.1096397E-1,1.7682186E-1,4.7164977E-2,2.5868847E-4,-1.3581354E-2,-5.849613E-3,7.860139E-3,6.440143E-3,1.6470643E-3,-1.2815883E-2,-2.5639995E-3,-2.9823338E-3,-2.2165678E-2,-6.4637777E-3,5.394432E-4,-3.4253318E-3,2.4319848E-4,1.5310087E-3,-2.9456834E-3,-2.8528202E-2,-1.0189542E-3,1.1650296E-2,-3.1469376E-3,2.9259992E-3,1.0272048E-2,-3.239445E-3,3.0860791E-3,4.7335844E-3,-1.0470551E-2,-4.6514568E-4,2.1747581E-3,-1.0197393E-2,-3.2358564E-4,-3.131697E-3,9.700234E-4,5.5761257E-4,-1.1325263E-2,2.5457263E-3,1.2979112E-2,-4.655867E-3,3.3189987E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,-1,-1,47,49,51,-1,53,55,57,59,61,63,65,-1,67,-1,-1,69,71,-1,73,-1,-1,-1,-1,-1,75,-1,-1,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,-1,-1,-1,-1,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.176248E-1,4.877448E-1,2.62106E-1,6.762672E-1,2.815839E-1,3.6582857E-1,2.6936495E-1,3.0235347E-1,5.5616367E-1,1.0351166E-1,2.3590939E-1,1.6352439E-1,1.6293678E-1,1.6913404E-1,3.053868E-1,3.0138507E-1,4.953198E-1,4.728501E-1,3.063407E-1,0E0,1.8775928E-1,2.1980956E-1,3.1747133E-1,1.2960124E-1,0E0,0E0,1.7391042E-1,1.4627023E-1,1.1243436E-1,0E0,1.7011577E-1,3.5510066E-1,3.4085786E-1,4.391557E-1,2.3194307E-1,7.006622E-1,4.0403003E-1,0E0,1.6564202E-1,0E0,0E0,1.0961382E-1,1.572474E-1,0E0,1.9221567E-1,0E0,0E0,0E0,0E0,0E0,1.1922107E-1,0E0,0E0,1.4094779E-1,0E0,3.303517E-1,5.5676144E-1,2.0796502E-1,1.1356409E-1,1.8915164E-1,1.880193E-1,1.9194011E-1,1.754114E-1,4.231195E-1,3.6940622E-1,1.4675647E-1,1.3249125E-1,0E0,2.3116219E-1,0E0,0E0,0E0,0E0,2.434774E-1,3.3096546E-1,1.01295575E-1,1.2586933E-1,1.1372718E-1,1.4099056E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,41,41,42,42,44,44,50,50,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,68,68,73,73,74,74,75,75,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,-1,-1,48,50,52,-1,54,56,58,60,62,64,66,-1,68,-1,-1,70,72,-1,74,-1,-1,-1,-1,-1,76,-1,-1,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,-1,-1,-1,-1,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.753919E3,1.273801E6,1.0911331E4,3.38464E5,1.9E1,1.2819376E2,9.3058E4,5.5306E4,1.4329944E0,5.72E2,1.0655738E0,5.5544252E7,4.4200982E8,1.5410214E7,2.4394053E9,1.172E3,3.451532E8,3.83E3,1.1057693E0,1.8708764E-3,7.696402E3,3.5902756E-6,1.3314917E0,4.949877E6,7.134519E-4,-7.697484E-3,1.4E1,2.0588236E0,7.203918E9,1.315196E-2,2.6572757E10,1.2E1,4.0844156E7,1.3797468E0,7.03295E-1,4E0,4.7675516E4,-2.1300618E-2,8.896243E4,1.0179894E-2,8.655641E-4,2.9143541E-6,2E0,-1.5330375E-2,2.7E1,8.032844E-3,3.279822E-3,-2.9594412E-3,4.3578353E-3,6.358082E-3,1.6616056E7,-6.265976E-3,-4.421335E-4,1.3683E4,-2.9911632E-3,2.6777263E0,1.4229508E1,3.04386E-4,1.941926E2,9.19E4,4.312482E5,9.721951E1,8E0,6.058777E6,3.938775E4,3.9759712E-4,5.2646267E9,-1.6229372E-3,9.489796E-1,6.5310707E-4,-1.2089859E-2,1.043574E-2,3.0591777E-3,4E0,5.1586456E7,3.258768E-4,3.2703E4,3.285E3,4E0,2.5868847E-4,-1.3581354E-2,-5.849613E-3,7.860139E-3,6.440143E-3,1.6470643E-3,-1.2815883E-2,-2.5639995E-3,-2.9823338E-3,-2.2165678E-2,-6.4637777E-3,5.394432E-4,-3.4253318E-3,2.4319848E-4,1.5310087E-3,-2.9456834E-3,-2.8528202E-2,-1.0189542E-3,1.1650296E-2,-3.1469376E-3,2.9259992E-3,1.0272048E-2,-3.239445E-3,3.0860791E-3,4.7335844E-3,-1.0470551E-2,-4.6514568E-4,2.1747581E-3,-1.0197393E-2,-3.2358564E-4,-3.131697E-3,9.700234E-4,5.5761257E-4,-1.1325263E-2,2.5457263E-3,1.2979112E-2,-4.655867E-3,3.3189987E-3],"split_indices":[52,9,48,9,10,58,2,29,42,33,53,45,7,43,5,2,5,2,54,0,47,38,54,9,0,0,3,54,12,0,12,8,45,53,27,3,48,0,33,0,0,38,8,0,8,0,0,0,0,0,29,0,0,9,0,41,56,38,33,1,47,52,3,51,45,39,12,0,53,0,0,0,0,8,7,38,2,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.709E3,2.294E3,4.15E2,1.558E3,7.36E2,1.11E2,3.04E2,1.28E3,2.78E2,4E1,6.96E2,7.8E1,3.3E1,2.37E2,6.7E1,6.91E2,5.89E2,2.42E2,3.6E1,1.9E1,2.1E1,6.3E1,6.33E2,6.4E1,1.4E1,5E0,2.8E1,2.07E2,3E1,6E0,6.1E1,4.13E2,2.78E2,4.2E1,5.47E2,1.72E2,7E1,3E0,3.3E1,1.3E1,8E0,5E0,5.8E1,2E0,6.31E2,4.2E1,2.2E1,1.3E1,1.5E1,9E0,1.98E2,1.3E1,1.7E1,5.1E1,1E1,3.58E2,5.5E1,2.56E2,2.2E1,6E0,3.6E1,3.92E2,1.55E2,5E0,1.67E2,2.9E1,4.1E1,1.1E1,2.2E1,3E0,2E0,7E0,5.1E1,3.29E2,3.02E2,1.91E2,7E0,1E1,4.1E1,3.55E2,3E0,4.8E1,7E0,2.2E1,2.34E2,2E0,2E1,2E0,4E0,2.2E1,1.4E1,3.5E1,3.57E2,2.3E1,1.32E2,3E0,2E0,3E0,1.64E2,2E1,9E0,3.2E1,9E0,2E0,2E1,1.36E2,1.93E2,7E0,2.95E2,1.4E1,1.77E2,4E0,3E0,5E0,5E0,5E0,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-6.294603E-5,-1.727741E-2,1.0222398E-2,-3.505E-2,-6.1009557E-4,4.878508E-3,4.6097886E-2,-1.4355834E-2,-6.650365E-2,4.3033488E-2,-2.3789892E-2,-9.208187E-3,2.306798E-2,3.7939735E-2,7.3218797E-3,-1.0714821E-2,-1.0593722E-2,-3.7416838E-2,-1.3960508E-1,9.354763E-2,3.9831316E-3,2.1153677E-2,-4.9064547E-2,-2.3547415E-2,1.3225632E-2,3.0762877E-2,-4.251068E-2,4.3311536E-2,-1.1789063E-1,-8.442494E-2,-1.1712703E-3,4.9263656E-2,-6.965884E-2,-1.2344713E-2,-9.180333E-2,1.2020343E-1,-1.7256299E-2,5.2098878E-2,-5.1232476E-2,2.8438844E-2,-9.619698E-3,-1.3522124E-1,-3.551344E-2,1.6870071E-3,-3.4993473E-2,3.889838E-2,-2.4595294E-2,1.17102884E-1,2.31101E-2,-7.555908E-3,-2.4207536E-2,2.72482E-2,8.967151E-2,2.320947E-3,-1.2201177E-2,-1.5291064E-3,-7.7933003E-3,2.5221623E-2,-3.5875697E-2,7.0254825E-2,-7.026739E-3,1.8400339E-3,-9.681972E-2,5.9584836E-3,-1.19284086E-1,6.432693E-3,-5.710416E-4,-3.6506047E-3,6.116929E-3,-9.878926E-5,1.5100491E-1,-4.368468E-3,9.307828E-3,7.954867E-2,-8.474447E-3,-5.0137364E-3,-1.4360396E-2,8.669201E-3,-4.1204523E-2,-2.5828734E-2,-8.3619595E-2,-6.572732E-3,8.491732E-2,-9.423081E-2,6.2840013E-3,2.8359238E-3,1.8939397E-1,3.1801376E-2,-3.8619224E-2,-3.3815948E-3,1.1016147E-3,4.3890295E-3,6.05959E-4,3.337167E-3,1.4468497E-2,1.870738E-3,-4.705824E-3,-1.2885048E-3,-1.2887821E-2,5.151283E-3,-3.3096222E-3,-3.688303E-3,-1.2830272E-2,-8.990728E-3,-1.9349441E-3,-4.4528903E-3,1.5840662E-3,-2.2897387E-3,9.923408E-3,4.4260547E-3,-3.2132193E-3,1.1628698E-2,3.0631202E-3,-4.817128E-3,1.3320266E-3,-2.6121137E-3,3.1843717E-4,-1.4240765E-3,4.2147706E-3,-9.330566E-3,-1.8129586E-3,3.133464E-3,-2.1417285E-3,6.5327487E-3,1.7166693E-3,-2.3256952E-3,-1.08414E-2,9.952365E-3,-2.7765996E-5,1.0641067E-2,1.263565E-4,4.0660966E-3,1.0471422E-3,-5.5458113E-3,2.135637E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,-1,75,77,-1,79,81,83,85,87,-1,89,91,93,-1,-1,-1,-1,95,97,99,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,-1,109,111,113,-1,-1,-1,115,117,119,121,123,125,127,-1,129,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.715015E-1,2.9501513E-1,3.1936345E-1,3.1278247E-1,5.220232E-1,3.722577E-1,1.8238056E-1,2.1754895E-1,4.0089732E-1,3.527219E-1,3.8384736E-1,2.6377034E-1,3.2123747E-1,1.7419815E-1,0E0,2.0249651E-1,0E0,3.891075E-1,2.8135276E-1,2.3245257E-1,2.7365667E-1,2.030436E-1,2.4738193E-1,3.3759645E-1,3.1166616E-1,3.7259465E-1,1.3752997E-1,1.4351699E-1,1.7174295E-1,1.304593E-1,2.3538809E-1,1.6664132E-1,3.0021173E-1,0E0,2.5251913E-1,1.1911464E-1,1.4653552E-1,2.8396145E-1,1.17043674E-1,2.2518337E-1,0E0,1.1437082E-1,2.391345E-1,0E0,1.8456578E-1,4.0023184E-1,2.801306E-1,1.8127847E-1,2.818211E-1,0E0,1.3087659E-1,1.3700789E-1,1.9126758E-1,0E0,0E0,0E0,0E0,2.5337556E-1,2.2499655E-1,1.8941502E-1,0E0,0E0,2.538203E-1,0E0,1.86795E-1,0E0,0E0,0E0,0E0,1.1708765E-1,2.222237E-1,0E0,1.3320154E-1,1.1171991E-1,2.3677912E-1,0E0,0E0,0E0,1.1781296E-1,1.6007634E-1,3.2852516E-1,2.6936495E-1,2.3515922E-1,2.3040414E-1,1.2599748E-1,0E0,1.2193155E-1,2.4032572E-1,4.226345E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,57,57,58,58,59,59,62,62,64,64,69,69,70,70,72,72,73,73,74,74,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,-1,76,78,-1,80,82,84,86,88,-1,90,92,94,-1,-1,-1,-1,96,98,100,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,-1,110,112,114,-1,-1,-1,116,118,120,122,124,126,128,-1,130,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.074E3,1.2679E4,2.5074627E0,2.7682406E8,4.5168175E5,7.273333E1,2.0136518E0,4.44E2,2.5073426E0,2.3907686E7,1.5107028E-1,8.0073607E-1,5.68E2,7.3218797E-3,1.0882353E1,-1.0593722E-2,1.2E1,2.84755E5,1.0347911E7,3.0161016E7,2.03353E-6,3.83E3,1.11335E3,3.2982758E1,1.5316E4,1.47E2,6.9673E4,1.25E2,8.2E1,2.0304577E-1,2.271E3,1E0,-1.2344713E-2,2.5797562E2,3.8758245E-1,5.38732E4,2.653986E6,1.6E1,1.1662405E9,-9.619698E-3,1.4136576E6,2.97271E5,1.6870071E-3,3.5551153E5,1.96E2,2.373E3,1.7835504E-1,1E0,-7.555908E-3,9.123214E-1,2.0966542E0,1.2328733E1,2.320947E-3,-1.2201177E-2,-1.5291064E-3,-7.7933003E-3,2.59698E-6,4.5723195E6,4.072289E0,-7.026739E-3,1.8400339E-3,3.7832818E0,5.9584836E-3,2.3871907E2,6.432693E-3,-5.710416E-4,-3.6506047E-3,6.116929E-3,2.41E3,5.0713895E2,-4.368468E-3,1.775894E10,1.0976147E-1,1.02942106E11,-5.0137364E-3,-1.4360396E-2,8.669201E-3,4.247868E3,2.32661E5,6.126259E5,5.2770585E-1,2.9676E4,2.4505E4,4.162121E0,2.8359238E-3,4.3359457E10,4.7048388E0,8.874407E6,-3.3815948E-3,1.1016147E-3,4.3890295E-3,6.05959E-4,3.337167E-3,1.4468497E-2,1.870738E-3,-4.705824E-3,-1.2885048E-3,-1.2887821E-2,5.151283E-3,-3.3096222E-3,-3.688303E-3,-1.2830272E-2,-8.990728E-3,-1.9349441E-3,-4.4528903E-3,1.5840662E-3,-2.2897387E-3,9.923408E-3,4.4260547E-3,-3.2132193E-3,1.1628698E-2,3.0631202E-3,-4.817128E-3,1.3320266E-3,-2.6121137E-3,3.1843717E-4,-1.4240765E-3,4.2147706E-3,-9.330566E-3,-1.8129586E-3,3.133464E-3,-2.1417285E-3,6.5327487E-3,1.7166693E-3,-2.3256952E-3,-1.08414E-2,9.952365E-3,-2.7765996E-5,1.0641067E-2,1.263565E-4,4.0660966E-3,1.0471422E-3,-5.5458113E-3,2.135637E-3],"split_indices":[102,2,2,56,7,28,54,39,0,53,45,38,27,3,0,52,0,3,1,1,45,37,2,47,58,9,0,2,8,29,27,11,67,0,4,38,32,1,18,5,0,28,1,0,33,0,2,38,74,0,27,54,53,0,0,0,0,37,28,54,0,0,53,0,52,0,0,0,0,2,52,0,5,38,31,0,0,0,52,11,33,41,29,29,54,0,19,58,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.661E3,9.95E2,1.666E3,4.81E2,5.14E2,1.451E3,2.15E2,2.91E2,1.9E2,1.78E2,3.36E2,8.18E2,6.33E2,2.01E2,1.4E1,2.87E2,4E0,1.37E2,5.3E1,7.7E1,1.01E2,1.21E2,2.15E2,4.99E2,3.19E2,5.67E2,6.6E1,1.95E2,6E0,3.2E1,2.55E2,3.7E1,1E2,1.4E1,3.9E1,6.2E1,1.5E1,5.4E1,4.7E1,1.18E2,3E0,2.8E1,1.87E2,8.1E1,4.18E2,1.9E2,1.29E2,4.5E1,5.22E2,8E0,5.8E1,1.46E2,4.9E1,3E0,3E0,2E1,1.2E1,1.45E2,1.1E2,3.4E1,3E0,2E1,8E1,4E0,3.5E1,5.6E1,6E0,1.1E1,4E0,3.6E1,1.8E1,2.8E1,1.9E1,4.9E1,6.9E1,2.5E1,3E0,4E0,1.83E2,3.53E2,6.5E1,9.6E1,9.4E1,3.9E1,9E1,2.6E1,1.9E1,4.58E2,6.4E1,2.9E1,2.9E1,2.6E1,1.2E2,4.6E1,3E0,1.31E2,1.4E1,1.07E2,3E0,2.7E1,7E0,7.3E1,7E0,1.8E1,1.7E1,9E0,2.7E1,4E0,1.4E1,9E0,1E1,3E0,4.6E1,1.9E1,5E1,1.43E2,4E1,3.42E2,1.1E1,1.8E1,4.7E1,3.3E1,6.3E1,4.5E1,4.9E1,3E1,9E0,2E0,8.8E1,1.6E1,3E0,7.1E1,3.87E2,3.3E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-7.5653064E-5,4.819218E-3,-2.911583E-2,1.7605964E-3,6.334686E-2,-9.5471395E-3,-2.4996426E-2,-1.2136161E-2,1.4768766E-2,5.012812E-3,9.8605335E-2,-2.2179363E-2,-1.1075975E-2,1.1454574E-3,-5.2187376E-2,2.1899132E-2,-2.3409583E-2,-5.3894413E-3,2.054317E-3,5.0581647E-3,-4.5775794E-3,-4.9175028E-2,-2.29692E-3,2.1003865E-2,-1.7599018E-2,-8.87032E-2,-1.2981592E-2,5.3965304E-2,9.498208E-3,-5.7410114E-2,2.8639248E-2,-6.532234E-2,2.2435838E-2,1.0213037E-1,-1.1117651E-2,-6.346866E-4,4.9990255E-2,-5.085267E-2,1.6396571E-2,-2.0732778E-1,-6.885091E-2,-2.814371E-2,9.4912216E-2,1.2285399E-1,3.3203475E-2,-7.727915E-3,3.33904E-2,-1.2561522E-2,-4.9058154E-2,4.2185374E-2,-7.59576E-3,-7.004329E-2,4.832072E-3,4.7357148E-3,-9.929467E-4,6.722082E-3,-2.7402043E-3,8.3746435E-4,-4.2586204E-2,7.209575E-4,-8.585783E-3,3.3097416E-3,-3.7083114E-3,-1.6867602E-3,-7.81771E-3,-1.3057457E-2,9.879414E-4,-1.3451629E-2,-3.4040234E-3,-1.2405337E-3,-6.1682626E-3,-2.7626224E-3,2.1657292E-3,1.2098343E-2,1.8717676E-3,9.917709E-3,2.603371E-3,4.438887E-3,8.9410605E-4,3.8883185E-3,-7.258796E-4,2.2290077E-3,-4.329813E-3,-2.6166271E-3,7.6077883E-3,-1.8263707E-3,3.3646687E-3,-4.3422813E-3,-1.2517615E-3,-5.8266544E-3,-7.664384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,-1,-1,-1,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,-1,87,-1,-1,-1,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8138103E-1,4.1090444E-1,2.6641104E-1,3.948032E-1,2.3397744E-1,0E0,2.1510935E-1,5.617936E-1,3.0758306E-1,2.007664E-1,1.0680026E-1,2.0116444E-1,0E0,2.9592457E-1,3.7536764E-1,3.776527E-1,3.1664863E-1,0E0,0E0,0E0,0E0,1.8532097E-1,2.0101957E-1,2.4214002E-1,4.6319914E-1,3.060012E-1,2.1206619E-1,3.7493843E-1,2.8300646E-1,1.719422E-1,1.8677934E-1,1.0908061E-1,1.0106006E-1,1.1160186E-1,1.8167455E-1,6.249093E-1,4.0735283E-1,3.5790867E-1,2.5397953E-1,1.7044872E-1,3.004691E-1,2.475271E-1,1.3100271E-1,3.4069216E-1,1.7549866E-1,2.6546952E-1,4.731886E-1,0E0,1.30487E-1,1.5375102E-1,0E0,1.126495E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.9985647E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,51,51,58,58],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,-1,-1,-1,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,-1,88,-1,-1,-1,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7033327E-1,4.5723195E6,3E0,2.437E3,9E0,-9.5471395E-3,7.780377E6,2.2415E4,2.8721272E1,1.752E3,2.851369E2,1E0,-1.1075975E-2,9.658451E0,3.3720784E5,1.8174828E6,1.16E2,-5.3894413E-3,2.054317E-3,5.0581647E-3,-4.5775794E-3,4.6982746E8,5.1998E4,7E2,3.9E2,9.251576E3,4.222002E8,1.6E1,1.5106794E3,5.542E3,6.1603E4,5.5669255E6,1E0,9.83E2,2.34768E5,3.3751E4,2.7E1,3.2188842E0,1.2E1,3.7342235E6,6.13E2,1.058E3,1.032E3,1.988805E8,5.1547173E1,5E0,2.3580047E10,-1.2561522E-2,1.1879E4,9.750042E6,-7.59576E-3,5.695737E0,4.832072E-3,4.7357148E-3,-9.929467E-4,6.722082E-3,-2.7402043E-3,8.3746435E-4,6.624E3,7.209575E-4,-8.585783E-3,3.3097416E-3,-3.7083114E-3,-1.6867602E-3,-7.81771E-3,-1.3057457E-2,9.879414E-4,-1.3451629E-2,-3.4040234E-3,-1.2405337E-3,-6.1682626E-3,-2.7626224E-3,2.1657292E-3,1.2098343E-2,1.8717676E-3,9.917709E-3,2.603371E-3,4.438887E-3,8.9410605E-4,3.8883185E-3,-7.258796E-4,2.2290077E-3,-4.329813E-3,-2.6166271E-3,7.6077883E-3,-1.8263707E-3,3.3646687E-3,-4.3422813E-3,-1.2517615E-3,-5.8266544E-3,-7.664384E-4],"split_indices":[27,28,3,2,3,0,28,29,53,29,58,16,0,58,28,51,8,0,0,0,0,7,12,2,10,33,7,3,52,29,11,28,104,0,29,1,3,53,18,45,0,0,10,7,48,3,5,0,2,9,0,54,0,0,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.681E3,2.295E3,3.86E2,2.182E3,1.13E2,8E0,3.78E2,1.055E3,1.127E3,4.3E1,7E1,3.74E2,4E0,7.93E2,2.62E2,9.5E2,1.77E2,1E1,3.3E1,6.8E1,2E0,1.58E2,2.16E2,3.85E2,4.08E2,1.35E2,1.27E2,2.64E2,6.86E2,1.07E2,7E1,1.29E2,2.9E1,1.6E1,2E2,2.21E2,1.64E2,2.06E2,2.02E2,1.8E1,1.17E2,1.12E2,1.5E1,6E1,2.04E2,3.99E2,2.87E2,3E0,1.04E2,6.6E1,4E0,1.26E2,3E0,1E1,1.9E1,1.3E1,3E0,1.05E2,9.5E1,2.04E2,1.7E1,1.43E2,2.1E1,1.82E2,2.4E1,2E0,2E2,1.1E1,7E0,6.9E1,4.8E1,8E1,3.2E1,3E0,1.2E1,2.6E1,3.4E1,3.9E1,1.65E2,3E1,3.69E2,2.6E2,2.7E1,1.02E2,2E0,1.7E1,4.9E1,8.5E1,4.1E1,2.3E1,7.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-6.6464423E-4,-2.7832894E-2,4.8377756E-3,-2.0860024E-2,-1.40742E-1,1.6015174E-2,-1.3154069E-2,-1.8255502E-2,-2.3598868E-1,-2.537767E-3,-9.208126E-3,-1.5211444E-1,1.811273E-2,-3.148249E-2,9.159842E-3,1.052508E-2,-2.048962E-2,-2.12041E-2,2.3680597E-3,-2.523902E-1,2.7192412E-2,9.0135604E-2,1.4764362E-2,-2.6136182E-2,-1.4403789E-1,5.715214E-2,-9.487603E-4,3.2001864E-3,-2.6451226E-2,-1.4530828E-2,-7.2688895E-4,-6.2181046E-3,5.5540255E-3,1.2773897E-1,-2.1257337E-2,-2.0602247E-2,2.1084778E-2,-4.1227727E-3,-5.5199895E-2,-1.61742E-2,-2.988368E-3,1.2242728E-1,1.9707622E-4,-4.5340493E-2,8.882038E-3,-1.484533E-2,-1.5655364E-1,1.0207913E-2,4.3207295E-3,-6.281742E-3,2.7683792E-3,-4.218454E-2,2.7071852E-2,1.7809866E-2,1.0523898E-1,2.480049E-2,-7.295156E-2,-2.589239E-1,-4.7278676E-2,4.080509E-3,1.113729E-2,-1.31654E-1,-9.5598545E-4,1.8107383E-2,-2.89478E-3,-1.123359E-3,2.626708E-3,-8.885568E-3,4.4408715E-3,1.8578715E-3,-2.997918E-3,5.3859777E-3,3.088308E-4,2.6930822E-3,5.722779E-4,3.3773833E-3,9.856174E-3,1.575608E-3,-3.9463425E-3,4.557352E-3,-4.1248524E-3,4.3303633E-5,-2.174331E-2,-9.162349E-3,-1.8639447E-3,-1.0256394E-2,1.440142E-5,5.588564E-3,4.4393106E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,27,-1,-1,29,31,33,35,37,39,41,43,-1,45,-1,-1,-1,-1,47,49,51,53,55,57,-1,-1,59,-1,61,63,65,67,-1,-1,-1,-1,69,71,73,75,77,79,81,83,-1,-1,85,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9942542E-1,3.5134926E-1,4.4705933E-1,2.3382242E-1,1.0738385E-1,4.8625377E-1,3.4873825E-1,2.283847E-1,3.1699586E-1,0E0,0E0,3.125254E-1,3.254398E-1,2.7785712E-1,1.8683122E-1,0E0,2.193462E-1,0E0,0E0,1.2854195E-1,1.1264831E-1,2.5404033E-1,2.901315E-1,2.8639898E-1,3.109951E-1,2.3144999E-1,1.396163E-1,0E0,5.895783E-1,0E0,0E0,0E0,0E0,1.2497121E-1,1.4662251E-1,2.0383856E-1,3.012831E-1,5.1144207E-1,2.9921073E-1,0E0,0E0,1.0316393E-1,0E0,1.2442188E-1,1.6866258E-1,2.1794568E-1,2.4842411E-1,0E0,0E0,0E0,0E0,2.2725919E-1,1.08460724E-1,2.3801059E-1,1.2862909E-1,1.5987693E-1,1.7658296E-1,3.5525265E-1,2.1767321E-1,0E0,0E0,1.4160173E-1,0E0,2.0105559E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,33,33,34,34,35,35,36,36,37,37,38,38,41,41,43,43,44,44,45,45,46,46,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,63,63],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,28,-1,-1,30,32,34,36,38,40,42,44,-1,46,-1,-1,-1,-1,48,50,52,54,56,58,-1,-1,60,-1,62,64,66,68,-1,-1,-1,-1,70,72,74,76,78,80,82,84,-1,-1,86,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.08E2,3.1E1,1.4540612E7,1.9483356E1,3.90106E0,1.198E5,6.624E3,8.974133E9,1.7E1,-2.537767E-3,-9.208126E-3,3.05402E5,4.09E2,4.387561E1,2.060792E8,1.052508E-2,1.1057693E0,-2.12041E-2,2.3680597E-3,1.2684739E1,1.341E3,9.887221E5,7.805608E4,8.292435E5,2.90002E5,7.535486E0,3.1546112E-2,3.2001864E-3,8.581E3,-1.4530828E-2,-7.2688895E-4,-6.2181046E-3,5.5540255E-3,1.5043668E0,3.6020834E2,9.277255E6,1.3446785E7,2.8035452E11,1.6527965E7,-1.61742E-2,-2.988368E-3,1.7324902E8,1.9707622E-4,1E0,8.342E3,2.6659616E2,7.016421E5,1.0207913E-2,4.3207295E-3,-6.281742E-3,2.7683792E-3,2.1132307E0,1.1994927E7,1.2599255E9,1.9925156E6,1E0,5.107429E1,3.5259784E2,1.8122449E2,4.080509E-3,1.113729E-2,1.43E2,-9.5598545E-4,5.473125E3,-2.89478E-3,-1.123359E-3,2.626708E-3,-8.885568E-3,4.4408715E-3,1.8578715E-3,-2.997918E-3,5.3859777E-3,3.088308E-4,2.6930822E-3,5.722779E-4,3.3773833E-3,9.856174E-3,1.575608E-3,-3.9463425E-3,4.557352E-3,-4.1248524E-3,4.3303633E-5,-2.174331E-2,-9.162349E-3,-1.8639447E-3,-1.0256394E-2,1.440142E-5,5.588564E-3,4.4393106E-4],"split_indices":[2,8,45,54,56,7,2,19,0,0,0,1,29,53,7,0,54,0,0,56,0,50,28,51,1,53,38,0,29,0,0,0,0,53,4,9,45,31,45,0,0,7,0,64,0,52,32,0,0,0,0,53,9,31,28,73,48,52,52,0,0,3,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E3,4.49E2,2.221E3,4.24E2,2.5E1,1.37E3,8.51E2,4.2E2,4E0,1E1,1.5E1,1.6E1,1.354E3,4.67E2,3.84E2,3E0,4.17E2,2E0,2E0,1E1,6E0,5.9E1,1.295E3,4.47E2,2E1,6.6E1,3.18E2,2.6E1,3.91E2,8E0,2E0,2E0,4E0,4.4E1,1.5E1,1.96E2,1.099E3,2.55E2,1.92E2,5E0,1.5E1,2.9E1,3.7E1,5.7E1,2.61E2,3.6E2,3.1E1,1.2E1,3.2E1,6E0,9E0,1.35E2,6.1E1,1.059E3,4E1,1.8E2,7.5E1,6E0,1.86E2,2.3E1,6E0,1.2E1,4.5E1,2.31E2,3E1,3.21E2,3.9E1,2.8E1,3E0,2.7E1,1.08E2,1.1E1,5E1,1.39E2,9.2E2,3.1E1,9E0,1.68E2,1.2E1,5E0,7E1,3E0,3E0,9E0,1.77E2,7E0,5E0,1.8E1,2.13E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-8.383659E-4,-2.381538E-2,6.1179753E-3,-1.3333205E-2,-8.019522E-2,1.08433785E-2,-3.7119344E-2,8.672287E-2,-1.7890887E-2,-1.1045968E-1,6.698501E-3,8.460512E-2,8.74394E-3,-1.0223128E-1,-1.0948238E-3,-2.9623266E-3,1.314313E-1,-1.3693681E-2,-5.4295813E-3,-5.5662985E-4,-1.4319001E-1,-1.9401792E-3,4.132371E-3,1.05932295E-1,-4.4540158E-3,-1.3686826E-2,1.6630298E-2,-2.4129245E-3,-9.103278E-3,9.93876E-4,9.57697E-3,-5.9933453E-3,-6.213832E-2,3.5068055E-3,-1.6676763E-1,6.617293E-3,2.236415E-2,-2.8822197E-2,3.196563E-2,3.1019554E-2,2.366742E-3,2.2252256E-3,-5.589968E-2,-1.1611877E-1,-4.6391245E-3,5.1495857E-3,-1.8072762E-1,-1.6539942E-3,8.297093E-3,-1.3377881E-2,-2.5868315E-2,3.7506938E-2,-9.15219E-3,2.809833E-2,5.9090178E-3,-6.647859E-3,4.126814E-2,-9.513866E-4,1.5535924E-3,-4.5110825E-3,2.119393E-3,-1.9175769E-3,-1.3004711E-2,-6.443644E-3,1.1189374E-3,-1.4128609E-2,-6.3748513E-3,-2.5563754E-4,-2.3212074E-3,2.3278429E-3,-4.0185205E-3,1.9349075E-3,3.481105E-4,8.588709E-4,-9.1752154E-4,1.524389E-3,7.682512E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,-1,-1,35,-1,37,39,-1,-1,-1,-1,41,43,-1,45,-1,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,-1,67,69,-1,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2642817E-1,3.6534044E-1,4.188371E-1,2.4050803E-1,2.558297E-1,2.852835E-1,1.8519777E-1,1.6091278E-1,1.996319E-1,2.3125279E-1,1.01191156E-1,2.0303515E-1,3.181116E-1,1.544016E-1,0E0,0E0,1.324057E-1,1.7943284E-1,0E0,0E0,2.877499E-1,0E0,0E0,1.2085658E-1,0E0,3.242938E-1,2.7289838E-1,0E0,0E0,0E0,0E0,1.7116058E-1,2.0431426E-1,0E0,2.0706272E-1,0E0,1.2618256E-1,2.5325692E-1,1.5179282E-1,1.7515576E-1,2.3487964E-1,2.402726E-1,2.3046452E-1,3.911749E-1,1.2319624E-1,0E0,2.1649992E-1,0E0,0E0,0E0,1.6167752E-1,1.5907702E-1,0E0,1.6450304E-1,0E0,1.6764581E-1,1.3657723E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,20,20,23,23,25,25,26,26,31,31,32,32,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,50,50,51,51,53,53,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,-1,-1,36,-1,38,40,-1,-1,-1,-1,42,44,-1,46,-1,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,-1,68,70,-1,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8122449E2,1.15062E5,6.747114E7,6E0,6.172708E2,3.921901E6,6.7628815E2,1.0119178E10,1E0,1.5756359E0,1.6219908E5,1.6427443E2,3.9E2,9.18292E-1,-1.0948238E-3,-2.9623266E-3,1.4925373E-1,1.4136576E6,-5.4295813E-3,-5.5662985E-4,3.120286E6,-1.9401792E-3,4.132371E-3,2.5534486E4,-4.4540158E-3,4.264897E6,2.0483453E-2,-2.4129245E-3,-9.103278E-3,9.93876E-4,9.57697E-3,4.5586734E0,5.2921E4,3.5068055E-3,3.84E2,6.617293E-3,5.4E1,1.1057693E0,1E0,8.581219E6,3.4409692E3,1.6171625E2,3.3025316E5,4E0,8E0,5.1495857E-3,7.359468E8,-1.6539942E-3,8.297093E-3,-1.3377881E-2,1.0666667E1,3.29E2,-9.15219E-3,8.959817E7,5.9090178E-3,1.9577E4,4.516183E1,-9.513866E-4,1.5535924E-3,-4.5110825E-3,2.119393E-3,-1.9175769E-3,-1.3004711E-2,-6.443644E-3,1.1189374E-3,-1.4128609E-2,-6.3748513E-3,-2.5563754E-4,-2.3212074E-3,2.3278429E-3,-4.0185205E-3,1.9349075E-3,3.481105E-4,8.588709E-4,-9.1752154E-4,1.524389E-3,7.682512E-3],"split_indices":[52,29,45,3,55,5,52,19,14,54,32,54,10,27,0,0,56,28,0,0,45,0,0,33,0,9,39,0,0,0,0,56,1,0,0,0,0,54,84,28,52,55,50,8,3,0,5,0,0,0,58,10,0,12,0,9,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.666E3,6.19E2,2.047E3,5.23E2,9.6E1,1.846E3,2.01E2,2.2E1,5.01E2,7.1E1,2.5E1,5E1,1.796E3,3.5E1,1.66E2,5E0,1.7E1,4.81E2,2E1,1.8E1,5.3E1,1.6E1,9E0,4.5E1,5E0,4.67E2,1.329E3,2.3E1,1.2E1,7E0,1E1,4.16E2,6.5E1,5E0,4.8E1,3.2E1,1.3E1,3.51E2,1.16E2,6.61E2,6.68E2,3.58E2,5.8E1,3.3E1,3.2E1,2E0,4.6E1,1E1,3E0,3E0,3.48E2,1.14E2,2E0,6.42E2,1.9E1,5.43E2,1.25E2,2.07E2,1.51E2,4.2E1,1.6E1,2.3E1,1E1,5E0,2.7E1,1.2E1,3.4E1,1.83E2,1.65E2,1.05E2,9E0,4.03E2,2.39E2,1.83E2,3.6E2,1.17E2,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[2.0381191E-3,-5.4262234E-3,2.3171626E-2,-1.0532526E-3,-4.1642655E-2,5.476456E-2,8.906531E-3,-2.8416352E-2,5.0133783E-3,-4.6187565E-2,5.8473013E-3,8.3553664E-2,1.7095135E-3,7.017449E-4,6.978237E-2,-1.3026168E-2,-9.329572E-2,2.8296756E-2,-7.68294E-3,1.3202975E-2,-5.5169497E-2,1.1442048E-1,3.923786E-2,-1.1847753E-2,1.1650795E-2,1.1413612E-2,-4.4799294E-2,1.8347056E-1,3.8607568E-2,-6.913069E-3,-6.619193E-3,-3.1653725E-2,-1.0913822E-2,-9.350875E-2,3.2447968E-2,4.409743E-3,-1.1562681E-2,8.295782E-3,-2.9115638E-2,-9.267178E-2,-1.4588383E-3,1.5737617E-3,6.476988E-3,-6.881474E-4,9.51889E-2,5.6499415E-3,-8.467255E-3,5.8892958E-2,-2.0430556E-3,-1.1568737E-1,-1.6405651E-2,4.0393337E-3,1.4624676E-2,-9.79164E-3,3.1906976E-3,1.0181213E-2,-1.0171157E-2,8.849488E-4,-4.595159E-3,-7.96166E-4,-1.3155796E-2,6.507694E-2,1.780852E-2,-9.987941E-2,-7.971653E-3,-7.915936E-2,6.0278913E-3,-1.3066909E-1,5.1505363E-4,-4.301403E-4,7.6312474E-3,1.508409E-3,-2.3665496E-3,-8.3361035E-3,9.828358E-2,5.066466E-3,-7.4830525E-2,-4.332845E-4,-7.7475826E-3,8.130309E-3,-2.9410856E-2,4.3592092E-5,-3.3791366E-3,3.4702176E-3,-6.067235E-3,4.6111393E-4,4.4951793E-3,2.3385885E-3,-7.3384857E-3,1.4848536E-3,-6.9951016E-4,-6.713532E-3,7.013925E-4,-3.568548E-3,-9.493471E-3,4.8502125E-3,-1.9795166E-3,1.3994111E-3,7.227478E-3,4.8624116E-4,-5.0058644E-3,1.8276964E-4,-6.287046E-3,-7.57074E-4,-9.6877925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,-1,55,57,-1,59,61,-1,63,-1,65,67,-1,-1,-1,-1,69,-1,71,73,75,77,79,-1,-1,-1,-1,-1,81,-1,-1,-1,-1,83,85,87,89,91,-1,93,-1,-1,-1,-1,-1,95,97,99,101,-1,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.200502E-1,3.1163493E-1,3.1273225E-1,2.9180768E-1,1.6340625E-1,3.2977718E-1,2.3969816E-1,3.1730798E-1,4.2560473E-1,1.111508E-1,0E0,1.8701458E-1,1.9356449E-1,2.071694E-1,1.9493642E-1,2.169666E-1,4.9888635E-1,2.5989965E-1,3.6179802E-1,1.9465563E-1,1.6396332E-1,1.3483107E-1,1.7833057E-1,0E0,1.6142835E-1,2.1990669E-1,1.611365E-1,1.1925146E-1,3.268842E-1,0E0,1.9526389E-1,1.403786E-1,0E0,2.322625E-1,2.3385477E-1,0E0,2.8400254E-1,0E0,1.8835077E-1,2.804305E-1,0E0,0E0,0E0,0E0,1.9525173E-1,0E0,1.06291115E-1,2.020821E-1,1.3958234E-1,1.1158103E-1,1.4784586E-1,0E0,0E0,0E0,0E0,0E0,1.640154E-1,0E0,0E0,0E0,0E0,2.2442752E-1,2.1124339E-1,2.8519088E-1,2.2388569E-1,1.0509431E-1,0E0,1.8251652E-1,0E0,0E0,0E0,0E0,0E0,1.0907991E-1,1.7006871E-1,1.381993E-1,1.0704996E-1,0E0,0E0,0E0,1.2849063E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,30,30,31,31,33,33,34,34,36,36,38,38,39,39,44,44,46,46,47,47,48,48,49,49,50,50,56,56,61,61,62,62,63,63,64,64,65,65,67,67,73,73,74,74,75,75,76,76,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,-1,56,58,-1,60,62,-1,64,-1,66,68,-1,-1,-1,-1,70,-1,72,74,76,78,80,-1,-1,-1,-1,-1,82,-1,-1,-1,-1,84,86,88,90,92,-1,94,-1,-1,-1,-1,-1,96,98,100,102,-1,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0989723E3,2.8091298E7,9.517604E6,9.721951E1,9.637832E1,2.3523688E0,8.791777E3,2.5943396E0,9.538462E0,2.090909E0,5.8473013E-3,6.6E1,1.2375429E8,4.1755924E7,4.0601485E6,6E0,3.4246575E-3,4.034E3,8E-1,6.5029144E5,1.1328599E3,1.1E1,2.47E2,-1.1847753E-2,1.4523809E0,3.5229592E0,2.478592E0,3.5E1,5.388794E6,-6.913069E-3,8.974133E9,2.2174756E4,-1.0913822E-2,9.51505E-1,2.3629358E6,4.409743E-3,3.3870136E7,8.295782E-3,1.775894E10,1.5669752E-2,-1.4588383E-3,1.5737617E-3,6.476988E-3,-6.881474E-4,2.5787393E3,5.6499415E-3,3.7129E4,8.079531E4,2.9833334E1,1.5E1,9.042859E4,4.0393337E-3,1.4624676E-2,-9.79164E-3,3.1906976E-3,1.0181213E-2,2.9446045E-1,8.849488E-4,-4.595159E-3,-7.96166E-4,-1.3155796E-2,1E0,4.5723195E6,1.6118722E0,1.5476E4,2.183847E0,6.0278913E-3,5.93E2,5.1505363E-4,-4.301403E-4,7.6312474E-3,1.508409E-3,-2.3665496E-3,2.01E2,1.5190727E5,1.4602E4,1E0,-4.332845E-4,-7.7475826E-3,8.130309E-3,1.2367184E3,4.3592092E-5,-3.3791366E-3,3.4702176E-3,-6.067235E-3,4.6111393E-4,4.4951793E-3,2.3385885E-3,-7.3384857E-3,1.4848536E-3,-6.9951016E-4,-6.713532E-3,7.013925E-4,-3.568548E-3,-9.493471E-3,4.8502125E-3,-1.9795166E-3,1.3994111E-3,7.227478E-3,4.8624116E-4,-5.0058644E-3,1.8276964E-4,-6.287046E-3,-7.57074E-4,-9.6877925E-3],"split_indices":[52,45,45,52,56,42,52,56,58,56,0,8,7,50,47,52,57,9,53,28,4,3,8,0,54,54,54,8,47,0,19,47,0,58,45,0,7,0,5,38,0,0,0,0,55,0,9,28,53,3,28,0,0,0,0,0,39,0,0,0,0,84,28,53,9,54,0,0,0,0,0,0,0,0,33,0,16,0,0,0,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.661E3,1.967E3,6.94E2,1.756E3,2.11E2,2.15E2,4.79E2,3.18E2,1.438E3,2.06E2,5E0,1.39E2,7.6E1,4.23E2,5.6E1,2.58E2,6E1,5.07E2,9.31E2,2.7E1,1.79E2,8.1E1,5.8E1,2E0,7.4E1,3.43E2,8E1,1.1E1,4.5E1,1.1E1,2.47E2,4.2E1,1.8E1,1.6E1,4.91E2,3.4E1,8.97E2,5E0,2.2E1,7E1,1.09E2,1.7E1,6.4E1,3E1,2.8E1,1.1E1,6.3E1,7.5E1,2.68E2,2.2E1,5.8E1,7E0,4E0,4E0,4.1E1,3E0,2.44E2,2.4E1,1.8E1,1.2E1,4E0,1.51E2,3.4E2,3.4E1,8.63E2,1.7E1,5E0,5.1E1,1.9E1,1.1E1,1.7E1,3.2E1,3.1E1,2.8E1,4.7E1,2.45E2,2.3E1,7E0,1.5E1,3E0,5.5E1,2.07E2,3.7E1,1.46E2,5E0,3.08E2,3.2E1,9E0,2.5E1,1.25E2,7.38E2,1E1,7E0,2.9E1,2.2E1,6E0,2.2E1,2.1E1,2.6E1,2.35E2,1E1,1E1,1.3E1,5.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-1.991247E-3,1.7424638E-3,-3.3713963E-2,-2.1602359E-2,7.835268E-3,-6.872339E-2,-9.823067E-3,-1.8414395E-2,-1.1464835E-2,2.5429338E-2,1.6627641E-4,-9.987942E-2,-9.366645E-3,2.3047533E-3,-9.752515E-4,-5.843912E-3,-6.9958024E-2,7.188974E-2,1.0170992E-2,-2.5457695E-2,9.561559E-3,-1.5996728E-3,-1.419777E-1,3.8983996E-3,-2.0275114E-3,-1.1750292E-2,1.5404576E-1,-3.354256E-2,-1.6042873E-1,8.458685E-2,-9.373666E-4,-1.096299E-1,1.505399E-2,-1.3602316E-1,-1.731475E-2,1.1065564E-2,-1.9677873E-1,-1.13121234E-1,-1.2513735E-2,-2.5238995E-3,-7.512916E-2,-2.7870315E-3,9.303648E-3,-9.323772E-3,-1.7910954E-2,-2.4597889E-1,-2.3646594E-3,7.326807E-2,9.3003865E-3,2.017359E-3,-7.6415986E-3,-3.1508043E-2,2.804504E-2,-9.609712E-3,-5.466474E-2,-3.6294719E-3,-8.542633E-2,9.055196E-3,5.8110207E-3,-1.2782283E-4,-1.6363896E-2,-6.594157E-3,7.357751E-4,-2.3938161E-3,4.22373E-4,3.7857096E-3,-5.1693553E-3,-3.2443067E-3,1.2123494E-3,-3.2135076E-3,-1.4603035E-2,9.367656E-3,2.9174478E-3,1.4576565E-4,-5.436594E-3,1.703158E-3,-2.8631734E-3,-8.4158825E-3,2.7257449E-3,1.2665856E-4,-8.149095E-3,9.83604E-4,-6.5864776E-3,6.037901E-4,-2.9559324E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,25,27,29,31,33,35,-1,37,-1,-1,39,41,43,45,47,-1,49,51,53,55,57,59,61,-1,63,65,-1,-1,-1,67,69,-1,71,-1,-1,-1,73,75,-1,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1381541E-1,3.3740953E-1,2.3268238E-1,3.36503E-1,2.537968E-1,2.0803773E-1,1.0299062E-1,3.1383643E-1,0E0,4.0405583E-1,3.158594E-1,2.0235068E-1,1.2315136E-1,0E0,0E0,3.710584E-1,3.0759946E-1,1.6540086E-1,2.5378E-1,3.1456053E-1,3.0010077E-1,0E0,1.2538481E-1,0E0,0E0,2.20849E-1,1.3082439E-1,1.703226E-1,2.4436927E-1,1.3961786E-1,0E0,1.3554229E-1,2.519129E-1,1.152409E-1,3.0635935E-1,2.1039492E-1,1.9599378E-1,1.2208179E-1,0E0,1.7864276E-1,2.513592E-1,0E0,0E0,0E0,1.3957675E-1,1.3770223E-1,0E0,1.6030866E-1,0E0,0E0,0E0,2.582124E-1,2.1674252E-1,0E0,1.7210701E-1,2.8778517E-1,3.064998E-1,2.3662847E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,44,44,45,45,47,47,51,51,52,52,54,54,55,55,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,26,28,30,32,34,36,-1,38,-1,-1,40,42,44,46,48,-1,50,52,54,56,58,60,62,-1,64,66,-1,-1,-1,68,70,-1,72,-1,-1,-1,74,76,-1,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.9527026E-1,1.5561539E2,1E0,5.0363636E1,1.683274E0,9.357879E2,1.9598837E2,9.0067856E5,-1.1464835E-2,3.3253515E6,1.954E3,8.3306855E6,1E0,2.3047533E-3,-9.752515E-4,2.6713815E-1,6.699017E6,2.0494426E7,3.640909E6,2.0979407E-4,2.1470589E1,-1.5996728E-3,2.5882E4,3.8983996E-3,-2.0275114E-3,6.3251557E0,7.68E2,8.732036E1,1.05E2,2.4859156E1,-9.373666E-4,4.685271E8,1.5497989E5,6.3157897E0,4.8E1,4.5505118E2,8E0,2.155E3,-1.2513735E-2,4.7E2,1E0,-2.7870315E-3,9.303648E-3,-9.323772E-3,2.028994E8,1.40673E5,-2.3646594E-3,6.977611E5,9.3003865E-3,2.017359E-3,-7.6415986E-3,9.188401E4,2.3523688E0,-9.609712E-3,5.695737E0,2.6777263E0,9.2699325E5,1.9162654E7,5.8110207E-3,-1.2782283E-4,-1.6363896E-2,-6.594157E-3,7.357751E-4,-2.3938161E-3,4.22373E-4,3.7857096E-3,-5.1693553E-3,-3.2443067E-3,1.2123494E-3,-3.2135076E-3,-1.4603035E-2,9.367656E-3,2.9174478E-3,1.4576565E-4,-5.436594E-3,1.703158E-3,-2.8631734E-3,-8.4158825E-3,2.7257449E-3,1.2665856E-4,-8.149095E-3,9.83604E-4,-6.5864776E-3,6.037901E-4,-2.9559324E-3],"split_indices":[27,52,16,46,53,55,52,50,0,45,2,45,75,0,0,57,45,9,45,38,46,0,10,0,0,54,2,55,52,58,0,31,28,56,8,56,8,0,0,1,75,0,0,0,7,1,0,45,0,0,0,28,41,0,54,41,45,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.648E3,2.37E3,2.78E2,4.9E2,1.88E3,1.12E2,1.66E2,4.84E2,6E0,5.7E2,1.31E3,7.3E1,3.9E1,2.5E1,1.41E2,3.9E2,9.4E1,1.4E2,4.3E2,3.51E2,9.59E2,2.9E1,4.4E1,1E1,2.9E1,3.77E2,1.3E1,6.8E1,2.6E1,1.23E2,1.7E1,1.6E1,4.14E2,2.3E1,3.28E2,9.53E2,6E0,3.7E1,7E0,3.3E2,4.7E1,2E0,1.1E1,5E0,6.3E1,1.4E1,1.2E1,1.13E2,1E1,4E0,1.2E1,9E1,3.24E2,1.2E1,1.1E1,2.74E2,5.4E1,9.37E2,1.6E1,3E0,3E0,3.1E1,6E0,6.3E1,2.67E2,8E0,3.9E1,2.9E1,3.4E1,4E0,1E1,9E0,1.04E2,6.4E1,2.6E1,2.99E2,2.5E1,5E0,6E0,2.65E2,9E0,1.8E1,3.6E1,8.93E2,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.3315783E-5,-1.3907322E-2,9.146604E-3,-2.8851056E-3,-3.9034557E-2,2.154681E-2,-7.703026E-3,-1.1564173E-2,4.1380424E-2,-6.0566366E-2,3.139792E-4,5.509166E-3,1.7931344E-2,-1.4888161E-2,4.7808614E-2,-5.2741193E-3,-1.5114781E-1,5.4383982E-2,-2.7638033E-1,-9.597695E-3,-9.808754E-2,3.056069E-3,-2.2315942E-2,4.049813E-2,7.697429E-3,-1.1866749E-2,-1.0935853E-1,3.463327E-3,-3.4684192E-2,-1.6731056E-3,-1.21814E-1,-9.2970945E-2,-1.3299034E-2,8.3049955E-3,4.0654626E-2,-1.937949E-2,-2.6805138E-3,3.7855743E-3,-4.627143E-2,-6.9764994E-2,-1.6380903E-1,3.6925152E-3,-1.8064347E-3,8.157249E-2,1.4448547E-2,6.0080215E-3,1.2918135E-2,6.8916874E-3,-2.8133843E-2,-1.49690155E-2,-1.6441636E-3,-6.2057124E-3,4.0781097E-3,-4.2402903E-3,1.0082969E-2,-1.9852166E-1,3.0979565E-3,-1.4670172E-1,2.855686E-4,-5.101633E-2,5.7151925E-2,-7.754389E-3,-2.230061E-2,-8.657628E-2,6.452387E-3,-2.2504702E-1,-2.4665117E-3,4.2670045E-2,6.021785E-3,2.5528062E-2,-2.5405772E-3,1.3165819E-2,-4.587088E-2,5.4543737E-2,-1.1871515E-2,5.1311734E-3,-3.1933732E-2,4.7944504E-4,-1.2528927E-3,-2.3280394E-3,-1.7495116E-2,-3.837058E-3,-1.6375158E-2,-4.7037234E-3,6.0794335E-3,1.162812E-3,5.3882776E-3,-7.1586575E-3,1.2103326E-4,-5.6969575E-3,-6.46589E-4,-1.2497771E-2,5.1752344E-4,2.6203291E-3,-4.8257895E-3,3.858433E-3,5.371406E-4,4.3935038E-4,6.2561785E-3,-6.917418E-3,-1.1128811E-3,3.986624E-3,-3.379485E-5,-1.1942681E-3,3.0379645E-3,-3.1286587E-3,-8.285648E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,53,55,57,-1,-1,59,-1,-1,-1,61,63,65,-1,-1,67,69,71,-1,73,75,-1,-1,-1,-1,77,-1,79,-1,81,-1,83,85,-1,87,89,-1,91,-1,93,-1,95,-1,97,99,101,103,-1,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4531558E-1,2.97894E-1,3.4282574E-1,2.88232E-1,2.779155E-1,3.1735605E-1,2.783174E-1,5.4970163E-1,5.232692E-1,4.0448827E-1,1.6955213E-1,0E0,2.1008062E-1,1.7481492E-1,1.6425322E-1,2.5201285E-1,1.7296183E-1,1.8742391E-1,1.187225E-1,2.994904E-1,2.1566272E-1,0E0,1.3614592E-1,3.0269286E-1,2.7639604E-1,1.8297589E-1,2.7894515E-1,0E0,2.2710933E-1,3.1905323E-1,2.7035284E-1,1.0719068E-1,0E0,0E0,1.6624661E-1,0E0,0E0,0E0,1.7658001E-1,3.1075814E-1,2.359953E-1,0E0,0E0,1.8364185E-1,1.3222757E-1,2.326906E-1,0E0,2.5010282E-1,1.684283E-1,0E0,0E0,0E0,0E0,1.8083645E-1,0E0,3.072927E-1,0E0,1.4158186E-1,0E0,1.5526962E-1,1.6468695E-1,0E0,1.7404506E-1,1.9006288E-1,0E0,1.9766366E-1,0E0,1.0917481E-1,0E0,1.1804375E-1,0E0,2.5670156E-1,1.6619587E-1,1.2532903E-1,1.9935748E-1,0E0,1.5182093E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,34,34,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,53,53,55,55,57,57,59,59,60,60,62,62,63,63,65,65,67,67,69,69,71,71,72,72,73,73,74,74,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,54,56,58,-1,-1,60,-1,-1,-1,62,64,66,-1,-1,68,70,72,-1,74,76,-1,-1,-1,-1,78,-1,80,-1,82,-1,84,86,-1,88,90,-1,92,-1,94,-1,96,-1,98,100,102,104,-1,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2485715E2,4.61E2,1.6837112E7,8.8365E4,2.074E3,2.368E0,5.528107E3,3.8977896E2,3.099779E6,2.1818182E0,1E1,5.509166E-3,4.0208E4,4.1585636E1,1.4285097E1,2.829581E10,3.8833074E4,3.114E3,8E0,1.3298E4,3.6604617E-2,3.056069E-3,1.1451E4,4.6E0,1.662639E7,5.673198E5,6.203E3,3.463327E-3,3.3047943E0,2.5551208E10,8.87E2,4.6847186E2,-1.3299034E-2,8.3049955E-3,2.66E2,-1.937949E-2,-2.6805138E-3,3.7855743E-3,2.428818E6,1.9505986E6,2.3449652E0,3.6925152E-3,-1.8064347E-3,1.5E1,1.9221336E1,2.5482938E0,1.2918135E-2,1.3332031E1,1.22E2,-1.49690155E-2,-1.6441636E-3,-6.2057124E-3,4.0781097E-3,1E0,1.0082969E-2,1E0,3.0979565E-3,3.5E1,2.855686E-4,1.30727E5,1.1160287E7,-7.754389E-3,1.3841E4,6.5029144E5,6.452387E-3,3.8935616E8,-2.4665117E-3,1.9232916E7,6.021785E-3,1.4022468E6,-2.5405772E-3,5.3534385E3,6.972973E0,2.8316305E0,1E0,5.1311734E-3,3.6709285E6,4.7944504E-4,-1.2528927E-3,-2.3280394E-3,-1.7495116E-2,-3.837058E-3,-1.6375158E-2,-4.7037234E-3,6.0794335E-3,1.162812E-3,5.3882776E-3,-7.1586575E-3,1.2103326E-4,-5.6969575E-3,-6.46589E-4,-1.2497771E-2,5.1752344E-4,2.6203291E-3,-4.8257895E-3,3.858433E-3,5.371406E-4,4.3935038E-4,6.2561785E-3,-6.917418E-3,-1.1128811E-3,3.986624E-3,-3.379485E-5,-1.1942681E-3,3.0379645E-3,-3.1286587E-3,-8.285648E-4],"split_indices":[52,0,45,1,2,58,52,4,51,56,3,0,9,53,54,31,33,29,3,9,27,0,12,54,45,32,29,0,35,31,2,4,0,0,0,0,0,0,45,48,53,0,0,3,53,42,0,56,10,0,0,0,0,16,0,8,0,3,0,29,45,0,9,28,0,7,0,48,0,51,0,4,56,54,96,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.714E3,1.075E3,1.639E3,7.48E2,3.27E2,9.44E2,6.95E2,6.26E2,1.22E2,2.11E2,1.16E2,3.4E1,9.1E2,6.16E2,7.9E1,6E2,2.6E1,1.18E2,4E0,9E1,1.21E2,3E1,8.6E1,2.83E2,6.27E2,5.98E2,1.8E1,6.1E1,1.8E1,5.83E2,1.7E1,1.9E1,7E0,1.1E1,1.07E2,2E0,2E0,2.6E1,6.4E1,8.6E1,3.5E1,1.1E1,7.5E1,1.09E2,1.74E2,6.24E2,3E0,2.78E2,3.2E2,4E0,1.4E1,1E1,8E0,5.77E2,6E0,1.2E1,5E0,1.2E1,7E0,1.6E1,9.1E1,1E1,5.4E1,8E1,6E0,2.2E1,1.3E1,5.9E1,5E1,1.5E2,2.4E1,5.49E2,7.5E1,7.8E1,2E2,8E0,3.12E2,3.5E2,2.27E2,7E0,5E0,1E1,2E0,1.3E1,3E0,5.8E1,3.3E1,8E0,4.6E1,5.5E1,2.5E1,1.9E1,3E0,5.5E1,4E0,3E1,1.2E2,5.32E2,1.7E1,1.3E1,6.2E1,5.1E1,2.7E1,1.71E2,2.9E1,9.4E1,2.18E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[-1.637545E-3,-1.0796604E-2,1.2243679E-2,-2.8717825E-2,-6.9582346E-4,2.9338853E-2,9.0742606E-4,-1.6665215E-2,-8.119836E-2,4.7403865E-2,-9.961938E-3,2.3064915E-2,1.1791871E-1,-3.8015816E-2,1.0059724E-2,1.9722485E-3,-5.8109116E-2,-9.60461E-2,5.6635416E-3,1.2265814E-1,1.0681155E-2,-2.2164252E-2,1.1685215E-2,2.6454529E-2,-1.5282848E-2,2.4513416E-1,1.0144729E-3,-7.5179E-4,-8.888109E-2,-1.20399535E-1,1.2837485E-2,1.5115305E-2,-7.748733E-2,-1.7986799E-2,-1.166554E-1,-1.1795263E-1,-2.3855878E-2,9.092356E-3,6.0330473E-2,2.5494792E-2,-1.3034606E-1,-2.434309E-3,-4.5429908E-2,6.82748E-3,5.57928E-3,2.9383184E-2,-6.748306E-3,9.037837E-4,1.3816393E-2,-3.3347495E-2,2.962606E-3,-4.6264455E-3,-1.3725114E-1,6.5675895E-3,-9.236528E-3,2.6940256E-3,3.73667E-2,-4.4029806E-2,3.2174908E-2,-1.4459307E-2,-2.649444E-3,3.3549555E-3,-3.9747346E-2,-1.4104137E-1,2.8487018E-3,-6.172395E-3,5.5805114E-3,4.55719E-4,-7.6263743E-3,-4.346701E-3,4.169664E-3,1.4890461E-2,7.225985E-3,-1.065625E-2,4.146058E-3,-5.138739E-2,1.7203258E-2,-1.5376706E-1,-3.900131E-2,1.6335666E-2,-4.412293E-2,3.156216E-2,-9.066789E-3,-7.3993853E-3,-4.2591052E-4,2.6852307E-3,-4.917407E-3,-1.4322159E-2,-5.010318E-3,1.1849131E-2,-3.5926692E-2,1.2044706E-2,3.9980677E-3,-1.3024671E-3,-1.1360616E-2,1.9481808E-3,-3.7995146E-3,-5.9472066E-3,5.1132636E-4,-1.1180117E-2,-4.354136E-3,4.288706E-3,-3.3131114E-4,-1.7230488E-2,-1.5415894E-3,-2.3342695E-3,1.4440342E-3,2.0779527E-3,-1.0127966E-2,-2.3714113E-3,1.000224E-3,4.4066127E-4,5.142512E-3,2.981119E-4,-5.3758873E-3,-1.1740755E-4,2.2152297E-3,7.5036474E-4,-6.087997E-3,4.9505937E-3,-3.1861402E-3,-1.7477366E-3,2.2101083E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,-1,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,-1,79,81,-1,-1,-1,83,-1,85,87,-1,-1,89,91,93,95,-1,-1,-1,97,99,-1,-1,-1,-1,-1,-1,-1,101,-1,-1,-1,103,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,-1,117,119,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.425174E-1,2.9377276E-1,2.0752807E-1,3.6867535E-1,4.6355164E-1,2.3458305E-1,2.3015535E-1,3.685538E-1,3.3416277E-1,4.6296468E-1,2.3059373E-1,4.723925E-1,3.3938247E-1,2.3307288E-1,1.9107291E-1,3.4580404E-1,3.455556E-1,1.584928E-1,0E0,2.1890521E-1,2.414835E-1,2.560282E-1,2.5093338E-1,1.987324E-1,0E0,1.1229855E-1,0E0,1.4901075E-1,2.1128586E-1,2.3174436E-1,1.2753943E-1,2.879704E-1,2.1709871E-1,1.7329341E-1,2.6610893E-1,2.1497321E-1,1.1447038E-1,0E0,1.290921E-1,1.3604039E-1,2.3918398E-1,2.9216385E-1,1.7229575E-1,0E0,1.6217487E-1,1.9230881E-1,0E0,0E0,0E0,1.408415E-1,0E0,1.2478784E-1,1.4295173E-1,0E0,0E0,1.2912765E-1,1.7415974E-1,2.0167565E-1,2.1282588E-1,0E0,0E0,0E0,3.0787513E-1,2.2426343E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.585613E-1,0E0,0E0,0E0,5.0350624E-1,1.8677865E-1,1.6847938E-1,1.5406874E-1,1.607464E-1,1.8593124E-1,1.9403428E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5856686E-1,3.0528626E-1,1.6391341E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,44,44,45,45,49,49,51,51,52,52,55,55,56,56,57,57,58,58,62,62,63,63,71,71,75,75,76,76,77,77,78,78,79,79,80,80,81,81,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,-1,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,-1,80,82,-1,-1,-1,84,-1,86,88,-1,-1,90,92,94,96,-1,-1,-1,98,100,-1,-1,-1,-1,-1,-1,-1,102,-1,-1,-1,104,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,-1,118,120,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4887473E-1,1.172E3,2.5291866E1,6.523944E-2,5.107429E1,9.923929E2,7.2243604E2,6.68E3,1.02E3,1.3E1,6.624E3,1.3697374E7,9.433E3,5.5356906E4,1.3676985E4,1E0,2.64E0,1.566593E6,5.6635416E-3,1.0258198E-2,4.004366E1,1.7291142E7,1.363881E3,1E0,-1.5282848E-2,3.773E3,1.0144729E-3,1.0277911E0,1.3E1,8.3893835E2,1.3860265E6,3.3151623E1,1.632095E2,4.144144E-1,1.563E3,1.6410715E1,1.9E1,9.092356E-3,1.5669752E-2,1.3238013E2,6.0078123E-6,1.0317661E5,1.84556E7,6.82748E-3,9.302862E7,9.414044E2,-6.748306E-3,9.037837E-4,1.3816393E-2,1.2E1,2.962606E-3,1.3264E4,2.8599728E7,6.5675895E-3,-9.236528E-3,1.1502944E0,1.5E1,2.4636364E1,1E0,-1.4459307E-2,-2.649444E-3,3.3549555E-3,7.33E2,1E0,2.8487018E-3,-6.172395E-3,5.5805114E-3,4.55719E-4,-7.6263743E-3,-4.346701E-3,4.169664E-3,7.2419E4,7.225985E-3,-1.065625E-2,4.146058E-3,1.3797468E0,8E0,1.627E3,3.138649E6,1.08874E5,1.2E1,2.372255E2,-9.066789E-3,-7.3993853E-3,-4.2591052E-4,2.6852307E-3,-4.917407E-3,-1.4322159E-2,-5.010318E-3,1E0,5.7819215E6,6.952967E6,3.9980677E-3,-1.3024671E-3,-1.1360616E-2,1.9481808E-3,-3.7995146E-3,-5.9472066E-3,5.1132636E-4,-1.1180117E-2,-4.354136E-3,4.288706E-3,-3.3131114E-4,-1.7230488E-2,-1.5415894E-3,-2.3342695E-3,1.4440342E-3,2.0779527E-3,-1.0127966E-2,-2.3714113E-3,1.000224E-3,4.4066127E-4,5.142512E-3,2.981119E-4,-5.3758873E-3,-1.1740755E-4,2.2152297E-3,7.5036474E-4,-6.087997E-3,4.9505937E-3,-3.1861402E-3,-1.7477366E-3,2.2101083E-3],"split_indices":[38,2,58,41,48,52,55,29,0,3,2,45,2,33,33,105,54,9,0,38,53,45,4,65,0,2,0,38,3,4,28,55,4,53,11,54,8,0,38,56,37,28,45,0,45,52,0,0,0,3,0,10,5,0,0,38,8,52,74,0,0,0,2,102,0,0,0,0,0,0,0,1,0,0,0,53,3,2,29,2,18,4,0,0,0,0,0,0,0,105,51,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.692E3,1.622E3,1.07E3,5.84E2,1.038E3,4.26E2,6.44E2,4.76E2,1.08E2,1.67E2,8.71E2,3.99E2,2.7E1,1.22E2,5.22E2,3.29E2,1.47E2,1.01E2,7E0,5.4E1,1.13E2,5.57E2,3.14E2,3.96E2,3E0,1.1E1,1.6E1,7.1E1,5.1E1,1E1,5.12E2,2.83E2,4.6E1,8.8E1,5.9E1,7.7E1,2.4E1,2.5E1,2.9E1,1.03E2,1E1,3.02E2,2.55E2,1.3E1,3.01E2,3.9E2,6E0,2E0,9E0,4.7E1,2.4E1,1.9E1,3.2E1,2E0,8E0,3.63E2,1.49E2,6.3E1,2.2E2,3E0,4.3E1,1.7E1,7.1E1,5.2E1,7E0,7.4E1,3E0,2E1,4E0,4E0,2.5E1,9.6E1,7E0,7E0,3E0,8.6E1,2.16E2,1.3E1,2.42E2,2.48E2,5.3E1,3.87E2,3E0,7E0,4E1,1.2E1,7E0,4E0,2.8E1,2.94E2,6.9E1,9.7E1,5.2E1,5.9E1,4E0,2.05E2,1.5E1,2.6E1,4.5E1,1.7E1,3.5E1,2.1E1,7.5E1,4E0,8.2E1,3.5E1,1.81E2,3E0,1E1,2.06E2,3.6E1,2.31E2,1.7E1,3.1E1,2.2E1,1.17E2,2.7E2,2.87E2,7E0,1.2E1,5.7E1,4E1,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[-1.7857727E-3,-1.5245068E-2,6.3086837E-3,3.66996E-2,-2.0110637E-2,3.6427625E-3,4.9310643E-2,-3.1467382E-2,8.409787E-2,-3.9379105E-2,-4.458105E-3,-6.000741E-3,1.7881362E-2,2.7711175E-3,-4.312673E-3,6.5940915E-4,-1.24488855E-2,1.1632722E-2,2.3420372E-3,-7.1505303E-3,-7.629485E-2,2.4418577E-3,-1.4224856E-2,-8.568839E-3,3.3123365E-3,-7.371797E-3,3.1257723E-2,-2.173216E-2,3.948643E-3,1.0315223E-2,-8.368596E-2,-1.3127958E-2,-1.1762654E-2,1.1242803E-2,-1.9165298E-2,1.6421094E-3,-1.7878531E-1,6.985648E-2,1.3541595E-2,-8.759423E-2,-6.294956E-3,-1.07138894E-1,-1.3538097E-2,2.7627632E-2,-2.8103953E-2,-9.245398E-3,4.813086E-2,-1.333554E-2,-1.692019E-2,-6.0112447E-2,1.3913515E-2,-1.9818211E-3,-1.6062122E-2,8.991337E-2,-2.1527794E-3,-2.1557769E-1,1.7570259E-2,-4.883165E-4,-6.819266E-3,1.6833469E-3,-2.2824197E-3,-1.30273225E-2,-4.079154E-3,1.7118638E-3,-3.079885E-3,7.369151E-3,5.623253E-4,-5.8941813E-3,-1.0549527E-3,2.2223608E-4,-5.0040283E-3,4.917619E-3,2.27898E-4,-2.4701196E-3,-4.719342E-4,3.598542E-4,-5.0550676E-3,3.6102723E-4,7.032465E-3,7.3335418E-3,2.3568645E-3,4.9603516E-3,-2.2167237E-2,1.5070465E-3,-1.4856507E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,31,33,-1,35,37,39,-1,-1,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,-1,-1,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.901096E-1,2.5347847E-1,1.904664E-1,2.805121E-1,2.7578872E-1,2.1527204E-1,1.2083137E-1,3.7565735E-1,2.7315745E-1,4.8724848E-1,2.7474016E-1,1.8107793E-1,2.1405488E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.892312E-1,4.2979968E-1,0E0,2.7258313E-1,1.9020504E-1,0E0,3.4021673E-1,2.8215572E-1,1.9262663E-1,0E0,0E0,3.0619693E-1,0E0,2.762751E-1,2.392683E-1,3.38752E-1,1.5997756E-1,2.2282183E-1,3.033932E-1,2.668555E-1,1.5123424E-1,2.6739642E-1,4.8380184E-1,1.22407295E-1,2.508902E-1,1.7212185E-1,2.7028334E-1,2.6688185E-1,0E0,1.4299533E-1,1.0835156E-1,1.4803985E-1,0E0,0E0,2.7806234E-1,0E0,4.4257003E-1,1.9074884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,19,19,20,20,22,22,23,23,25,25,26,26,27,27,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,50,50,53,53,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,32,34,-1,36,38,40,-1,-1,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,-1,-1,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1E0,2.7702793E-1,3.6E2,2.074E3,4.930349E5,1E0,3.09627E5,4.676E3,2.2444444E0,8.182648E6,4.4854636E7,7.118233E7,2.7711175E-3,-4.312673E-3,6.5940915E-4,-1.24488855E-2,1.1632722E-2,2.3420372E-3,1.5733662E0,3.192E3,2.4418577E-3,5E1,2.2415E4,3.3123365E-3,1.4329944E0,1.5445488E8,3.79E2,3.948643E-3,1.0315223E-2,1.27384615E1,-1.3127958E-2,2.7682406E8,5.4468E4,6.23E2,5.08E2,3.577537E7,6.788246E0,1.5938586E8,6.7E1,1.5316E4,5.8601086E1,2.0189162E7,3.4015749E0,1.3456146E6,1.421E3,2.4517391E1,-1.333554E-2,3.26E2,3.429012E7,1E0,-1.9818211E-3,-1.6062122E-2,1.8048264E4,-2.1527794E-3,3.7246967E2,2.1298597E0,-4.883165E-4,-6.819266E-3,1.6833469E-3,-2.2824197E-3,-1.30273225E-2,-4.079154E-3,1.7118638E-3,-3.079885E-3,7.369151E-3,5.623253E-4,-5.8941813E-3,-1.0549527E-3,2.2223608E-4,-5.0040283E-3,4.917619E-3,2.27898E-4,-2.4701196E-3,-4.719342E-4,3.598542E-4,-5.0550676E-3,3.6102723E-4,7.032465E-3,7.3335418E-3,2.3568645E-3,4.9603516E-3,-2.2167237E-2,1.5070465E-3,-1.4856507E-3],"split_indices":[102,79,35,2,2,28,106,9,1,56,45,9,7,0,0,0,0,0,0,56,1,0,0,29,0,38,7,1,0,0,56,0,7,1,2,2,7,53,7,29,9,50,45,56,51,2,58,0,0,7,100,0,0,33,0,4,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.661E3,9.99E2,1.662E3,8.5E1,9.14E2,1.566E3,9.6E1,3.5E1,5E1,4.09E2,5.05E2,9.34E2,6.32E2,9.1E1,5E0,3E1,5E0,8E0,4.2E1,2.19E2,1.9E2,7.5E1,4.3E2,9.04E2,3E1,2.19E2,4.13E2,1.89E2,3E1,4E0,1.86E2,3E0,4.27E2,3.15E2,5.89E2,2.09E2,1E1,1.29E2,2.84E2,3.5E1,1.54E2,1.39E2,4.7E1,1.25E2,3.02E2,2.03E2,1.12E2,4E0,5.85E2,3.4E1,1.75E2,6E0,4E0,1.1E2,1.9E1,4E0,2.8E2,1.5E1,2E1,7.7E1,7.7E1,1.5E1,1.24E2,2.4E1,2.3E1,1.3E1,1.12E2,1.7E1,2.85E2,1.78E2,2.5E1,4.9E1,6.3E1,9.8E1,4.87E2,1.4E1,2E1,1.68E2,7E0,4.2E1,6.8E1,2E0,2E0,2.18E2,6.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.2462559E-3,3.9307628E-2,-3.8827376E-3,-3.226818E-3,5.32079E-2,-1.0608421E-2,1.4449323E-2,1.4283804E-2,4.029864E-2,-1.1262197E-2,9.006066E-3,3.072184E-2,-5.6940853E-3,4.2204163E-3,1.4342787E-2,-1.26485005E-2,8.4186085E-2,9.1464795E-2,2.0660782E-2,-2.1914434E-2,4.1145023E-2,-3.2763116E-2,6.433448E-2,-5.7149515E-3,-3.359276E-2,6.421183E-3,-4.3354635E-4,3.7462173E-3,1.3488212E-2,5.5227272E-2,3.2800797E-3,-1.0318107E-2,-3.564003E-3,1.5522914E-1,1.229263E-3,1.2057642E-2,-1.392733E-1,1.8997518E-3,9.053429E-3,-4.9020974E-3,-1.0620517E-2,-3.6577213E-1,-2.9732876E-2,3.510824E-2,5.145175E-3,5.5486675E-2,-1.2338233E-2,-1.18834985E-4,-4.151889E-3,6.8171846E-4,1.2699493E-2,-3.2080316E-3,4.120541E-3,-1.1222906E-2,1.2713218E-3,-1.1154166E-3,2.1528079E-4,-1.9811468E-4,-2.8996926E-2,-1.0156455E-3,-3.2158718E-3,3.3741598E-3,-4.947873E-4,6.722919E-4,5.756105E-3,-1.6915234E-3,1.3450813E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,7,9,11,-1,13,15,-1,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,-1,-1,-1,-1,43,45,47,-1,49,-1,51,53,-1,-1,55,-1,57,59,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.819926E-1,2.4234384E-1,3.0543324E-1,0E0,4.403516E-1,2.3746333E-1,2.1815127E-1,0E0,1.6880892E-1,2.4016763E-1,0E0,2.2300923E-1,2.2730617E-1,0E0,2.1404913E-1,2.5858656E-1,1.2216824E-1,1.08906806E-1,1.9026923E-1,1.3440105E-1,1.319715E-1,2.260511E-1,1.2784807E-1,2.3424143E-1,5.571277E-1,0E0,0E0,0E0,0E0,1.07569695E-1,1.7354494E-1,1.09319046E-1,0E0,1.3574398E-1,0E0,2.0459409E-1,2.2692758E-1,0E0,0E0,2.3097467E-1,0E0,4.3093556E-1,1.3861415E-1,1.2601018E-1,0E0,1.2972991E-1,1.5335764E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,29,29,30,30,31,31,33,33,35,35,36,36,39,39,41,41,42,42,43,43,45,45,46,46],"right_children":[2,4,6,-1,8,10,12,-1,14,16,-1,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,-1,-1,-1,-1,44,46,48,-1,50,-1,52,54,-1,-1,56,-1,58,60,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.489796E-1,2.35E2,1.0855529E3,-3.226818E-3,2.04E5,1.03992E3,2.3907686E7,1.4283804E-2,3.640909E6,1.0660479E2,9.006066E-3,1.5552E4,8.61491E3,4.2204163E-3,3.01E2,1.3905958E7,2.0237808E8,5.24422E5,6.89132E6,6.9998717E-4,8.0886055E3,2.2515285E6,1.067509E6,1.93974E7,1.0195312E0,6.421183E-3,-4.3354635E-4,3.7462173E-3,1.3488212E-2,1.1225189E5,2.28125E0,9E0,-3.564003E-3,1.9745072E7,1.229263E-3,5.828063E5,3.0121632E6,1.8997518E-3,9.053429E-3,2.0407547E5,-1.0620517E-2,2.915E4,7.8351054E11,4.949877E6,5.145175E-3,8.874407E6,3.4409692E3,-1.18834985E-4,-4.151889E-3,6.8171846E-4,1.2699493E-2,-3.2080316E-3,4.120541E-3,-1.1222906E-2,1.2713218E-3,-1.1154166E-3,2.1528079E-4,-1.9811468E-4,-2.8996926E-2,-1.0156455E-3,-3.2158718E-3,3.3741598E-3,-4.947873E-4,6.722919E-4,5.756105E-3,-1.6915234E-3,1.3450813E-3],"split_indices":[53,2,52,0,5,46,45,0,45,56,0,9,4,0,0,45,7,11,50,34,33,28,9,50,53,0,0,0,0,33,54,17,0,1,0,28,28,0,0,28,0,9,31,9,0,45,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.635E3,1.6E2,2.475E3,1.8E1,1.42E2,1.811E3,6.64E2,6E0,1.36E2,1.806E3,5E0,3.67E2,2.97E2,4.7E1,8.9E1,1.781E3,2.5E1,5.1E1,3.16E2,2.21E2,7.6E1,4.6E1,4.3E1,1.339E3,4.42E2,1.6E1,9E0,4.9E1,2E0,1.05E2,2.11E2,1.82E2,3.9E1,8E0,6.8E1,3.3E1,1.3E1,3.7E1,6E0,1.335E3,4E0,4E0,4.38E2,7.7E1,2.8E1,4.8E1,1.63E2,1.66E2,1.6E1,4E0,4E0,1.6E1,1.7E1,8E0,5E0,4.51E2,8.84E2,2E0,2E0,3.58E2,8E1,4.3E1,3.4E1,3E1,1.8E1,1.04E2,5.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-8.450246E-4,5.2264654E-3,-1.9518176E-2,3.0008398E-3,8.22117E-2,-5.9087586E-2,-7.964739E-3,4.0684803E-3,-8.77398E-2,7.0906975E-3,3.5626392E-3,-3.0349128E-2,-1.2277698E-1,-6.0651384E-2,-2.8816962E-3,-7.248077E-3,1.2767473E-2,2.402147E-3,-5.718662E-3,-2.9443514E-3,5.760498E-3,-1.31945945E-2,-6.6896277E-3,-7.1725324E-3,2.314325E-4,-9.0103E-3,-1.5427849E-3,6.0027998E-2,-1.0550387E-2,-5.555414E-3,-5.682088E-3,4.9527483E-3,1.0225812E-2,-3.687641E-2,8.489701E-2,6.80339E-3,-1.24300895E-2,-7.4487864E-3,-1.0660419E-1,4.484137E-2,-9.847016E-3,5.4754927E-3,5.2708063E-2,-2.640837E-2,-8.431305E-3,1.0968585E-2,1.95396E-3,-6.253427E-3,2.5727274E-2,8.901867E-3,-2.260177E-2,-9.886521E-3,4.0519098E-4,-3.6246088E-4,4.831039E-3,-1.6007894E-3,-4.352661E-5,2.9634868E-3,8.9957044E-5,3.570403E-3,-3.148785E-4,-2.2300882E-3,2.756089E-3,2.741482E-3,-6.2455297E-3,1.8083701E-3,-4.2680305E-4,-2.4799057E-3,-2.1004827E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,35,37,-1,39,-1,41,43,45,-1,47,49,51,53,55,57,59,61,-1,-1,-1,-1,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.049643E-1,3.4757414E-1,3.0125678E-1,1.9159487E-1,2.946725E-1,2.6899368E-1,1.3694315E-1,1.922526E-1,1.0644063E-1,0E0,2.1237947E-1,1.9322133E-1,1.5916973E-1,1.5708388E-1,2.264129E-1,1.4393237E-1,2.533076E-1,0E0,0E0,0E0,0E0,2.1452254E-1,0E0,0E0,0E0,0E0,0E0,3.0490848E-1,1.2373671E-1,0E0,1.7677902E-1,0E0,2.1665096E-1,1.0316228E-1,1.05062425E-1,0E0,1.2958884E-1,1.0079224E-1,1.5108547E-1,1.8871908E-1,1.6330263E-1,1.9628556E-1,1.4035648E-1,1.2114979E-1,0E0,0E0,0E0,0E0,1.15730435E-1,1.0109722E-1,1.1193089E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,21,21,27,27,28,28,30,30,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,48,48,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,36,38,-1,40,-1,42,44,46,-1,48,50,52,54,56,58,60,62,-1,-1,-1,-1,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3382226E-1,1.8552577E2,1.8903887E0,1E0,3.1972716E7,1.9166666E0,4.39688E-5,1.9563605E5,9E0,7.0906975E-3,1E0,2.285677E6,8.289561E8,5.987692E2,2.64E0,6E0,1.2E1,2.402147E-3,-5.718662E-3,-2.9443514E-3,5.760498E-3,3.449702E2,-6.6896277E-3,-7.1725324E-3,2.314325E-4,-9.0103E-3,-1.5427849E-3,4.88954E5,6.59148E-1,-5.555414E-3,1E0,4.9527483E-3,3.524269E6,6.5E1,2.4684112E8,6.80339E-3,3.8977896E2,7.1277297E-1,4.9885715E2,3.6E2,6.947568E7,2.3125623E5,1.21320024E8,4.3800272E8,-8.431305E-3,1.0968585E-2,1.95396E-3,-6.253427E-3,2.6E1,1.5418326E6,6.5039682E0,-9.886521E-3,4.0519098E-4,-3.6246088E-4,4.831039E-3,-1.6007894E-3,-4.352661E-5,2.9634868E-3,8.9957044E-5,3.570403E-3,-3.148785E-4,-2.2300882E-3,2.756089E-3,2.741482E-3,-6.2455297E-3,1.8083701E-3,-4.2680305E-4,-2.4799057E-3,-2.1004827E-4],"split_indices":[27,56,53,77,51,54,42,28,3,0,61,32,7,4,54,52,10,0,0,0,0,52,0,0,0,0,0,32,38,0,79,0,1,8,7,0,4,27,52,2,7,28,32,7,0,0,0,0,3,51,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.688E3,2.029E3,6.59E2,1.973E3,5.6E1,1.48E2,5.11E2,1.951E3,2.2E1,3E1,2.6E1,1.03E2,4.5E1,4.4E1,4.67E2,8.48E2,1.103E3,4E0,1.8E1,1.7E1,9E0,9E1,1.3E1,3.7E1,8E0,7E0,3.7E1,5E1,4.17E2,1.1E1,8.37E2,2.9E1,1.074E3,7.3E1,1.7E1,2.3E1,2.7E1,4.05E2,1.2E1,6.3E1,7.74E2,9.67E2,1.07E2,6.9E1,4E0,3E0,1.4E1,6E0,2.1E1,1.95E2,2.1E2,6E0,6E0,3.3E1,3E1,2.12E2,5.62E2,5.7E1,9.1E2,7.8E1,2.9E1,5.6E1,1.3E1,1.8E1,3E0,7.4E1,1.21E2,8E1,1.3E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[2.505353E-4,-1.5475386E-2,6.7282217E-3,-2.7822427E-4,-4.7753043E-2,1.96619E-2,-1.0524958E-2,4.2476733E-3,-5.5722357E-3,-7.911929E-2,-5.0942414E-3,4.7287423E-2,1.2083955E-2,-1.7698677E-2,2.0536177E-2,1.1404028E-2,-7.841582E-2,-2.2309543E-1,-6.434043E-2,1.0973172E-1,-2.0753644E-2,6.568712E-2,-2.6925992E-2,-2.2170506E-2,2.2216989E-2,-2.0602634E-2,4.175178E-3,3.7042093E-2,-5.1318146E-2,1.650263E-2,-6.1824527E-2,-1.09231554E-1,1.3426641E-3,-1.9525543E-3,-1.3108387E-2,1.7312178E-3,-8.932993E-2,1.1217811E-2,1.3585981E-3,2.1686828E-2,-7.965146E-2,5.5673987E-2,1.767928E-1,-5.375866E-3,6.070145E-4,-6.5569236E-4,-1.0053029E-1,2.3662772E-2,-8.205319E-3,-1.700037E-2,-1.0229141E-1,3.916233E-3,1.5029411E-2,-1.3214477E-2,-1.1090149E-2,-2.3420404E-3,4.9524613E-2,-9.2841027E-4,-2.16204E-2,2.6007663E-3,-6.1310385E-3,-1.898155E-1,-6.800329E-2,3.99772E-3,6.9412286E-3,-1.7718047E-1,-1.7503047E-3,6.7515234E-3,4.7276262E-2,1.3408653E-2,2.466413E-3,-6.518708E-2,2.8299095E-2,-3.264733E-1,-6.1173845E-2,2.2012698E-2,9.492001E-3,6.108415E-3,-3.3034977E-2,-1.41859995E-2,-2.1432433E-3,9.49959E-3,3.9375166E-4,-3.5722866E-3,4.9897404E-3,7.985907E-4,-1.7868241E-3,4.677161E-3,2.0279555E-4,-1.4820773E-2,-4.990099E-3,-1.1843284E-2,-2.7133075E-3,4.272176E-3,-1.0690631E-3,-1.1149878E-2,-3.6857784E-4,1.1044326E-3,3.6499936E-3,-1.3734085E-3,-6.2293797E-3,2.3263711E-3,-2.1001028E-3,-2.3100097E-2,-4.6983524E-4,-1.3840309E-3,-8.359507E-3,2.246093E-3,4.94886E-4,1.4918715E-3,-2.0677017E-3,-1.6896473E-2,-1.451139E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,-1,-1,-1,-1,61,-1,-1,63,65,67,69,-1,-1,71,73,75,-1,77,79,-1,81,83,-1,85,87,-1,-1,-1,-1,89,91,93,-1,95,-1,-1,97,-1,-1,99,101,103,105,107,-1,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7270177E-1,3.8308966E-1,4.2329377E-1,2.799779E-1,3.3410847E-1,2.263582E-1,1.8151927E-1,3.043693E-1,0E0,2.920497E-1,1.9467466E-1,3.1993115E-1,2.9622313E-1,2.0311564E-1,1.8336219E-1,1.7762111E-1,1.3755858E-1,1.08820915E-1,3.347066E-1,1.25048E-1,2.3919426E-1,1.9819784E-1,1.6023797E-1,3.2868382E-1,1.8818504E-1,1.8776739E-1,0E0,1.2185645E-1,1.9612253E-1,2.7578974E-1,5.03842E-1,1.0528651E-1,0E0,0E0,0E0,0E0,2.1425194E-1,0E0,0E0,1.2151341E-1,1.6167775E-1,1.1726552E-1,1.7951828E-1,0E0,0E0,2.895926E-1,3.5350552E-1,1.8399084E-1,0E0,2.2919564E-1,2.8801444E-1,0E0,1.0445136E-1,1.8792477E-1,0E0,1.8882E-1,3.5060126E-1,0E0,0E0,0E0,0E0,1.5215778E-1,1.670542E-1,1.14130214E-1,0E0,1.113686E-1,0E0,0E0,1.0966161E-1,0E0,0E0,1.0998201E-1,1.5867813E-1,2.9143363E-1,1.3034733E-1,1.8815726E-1,0E0,3.1484014E-1,3.083984E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,36,36,39,39,40,40,41,41,42,42,45,45,46,46,47,47,49,49,50,50,52,52,53,53,55,55,56,56,61,61,62,62,63,63,65,65,68,68,71,71,72,72,73,73,74,74,75,75,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,-1,-1,-1,-1,62,-1,-1,64,66,68,70,-1,-1,72,74,76,-1,78,80,-1,82,84,-1,86,88,-1,-1,-1,-1,90,92,94,-1,96,-1,-1,98,-1,-1,100,102,104,106,108,-1,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.146E3,5.116E3,1.4274633E7,1E0,5.1454154E8,1.1457406E-1,3.317425E3,1E0,-5.5722357E-3,1.763213E-5,2.831692E10,2.773036E1,3.2485715E2,7.26E2,1.75E2,1.3569831E7,5.117647E0,8.9928055E-1,4.0233E4,1.3569831E7,1.9928571E0,4.8992978E5,7.57657E4,5.3379045E6,1E0,4.1585636E1,4.175178E-3,1.5479116E-1,1.5005797E4,5.85E2,8.9E1,9.6590906E-1,1.3426641E-3,-1.9525543E-3,-1.3108387E-2,1.7312178E-3,4.2197E4,1.1217811E-2,1.3585981E-3,1.074E3,2.5070414E-5,7E0,2.65903E5,-5.375866E-3,6.070145E-4,1.04856936E8,9.536863E4,1.921032E4,-8.205319E-3,1.7181714E7,2.90002E5,3.916233E-3,8.0886055E3,8.61491E3,-1.1090149E-2,3.38464E5,1E0,-9.2841027E-4,-2.16204E-2,2.6007663E-3,-6.1310385E-3,5.339161E0,4.54E2,4.144144E-1,6.9412286E-3,2.1989189E2,-1.7503047E-3,6.7515234E-3,4.4377274E2,1.3408653E-2,2.466413E-3,1.4E1,3.2049225E1,9.5E2,6.6141135E-1,4.0208E4,9.492001E-3,2.8277853E11,5.1776E4,-1.41859995E-2,-2.1432433E-3,9.49959E-3,3.9375166E-4,-3.5722866E-3,4.9897404E-3,7.985907E-4,-1.7868241E-3,4.677161E-3,2.0279555E-4,-1.4820773E-2,-4.990099E-3,-1.1843284E-2,-2.7133075E-3,4.272176E-3,-1.0690631E-3,-1.1149878E-2,-3.6857784E-4,1.1044326E-3,3.6499936E-3,-1.3734085E-3,-6.2293797E-3,2.3263711E-3,-2.1001028E-3,-2.3100097E-2,-4.6983524E-4,-1.3840309E-3,-8.359507E-3,2.246093E-3,4.94886E-4,1.4918715E-3,-2.0677017E-3,-1.6896473E-2,-1.451139E-3],"split_indices":[2,29,45,104,5,42,52,74,0,38,31,58,52,8,8,45,54,53,1,45,54,51,28,45,85,53,0,57,4,2,0,53,0,0,0,0,12,0,0,2,42,3,32,0,0,7,28,4,0,12,1,0,33,4,0,9,16,0,0,0,0,58,2,53,0,52,0,0,52,0,0,8,58,0,41,9,0,31,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.675E3,7.8E2,1.895E3,5.31E2,2.49E2,1.083E3,8.12E2,5.12E2,1.9E1,1.43E2,1.06E2,2.32E2,8.51E2,6.6E2,1.52E2,4.72E2,4E1,1.2E1,1.31E2,1.2E1,9.4E1,1.86E2,4.6E1,1.94E2,6.57E2,6.43E2,1.7E1,1.24E2,2.8E1,4.42E2,3E1,3.1E1,9E0,3E0,9E0,2.6E1,1.05E2,4E0,8E0,5.5E1,3.9E1,1.72E2,1.4E1,1.4E1,3.2E1,1.53E2,4.1E1,6.53E2,4E0,6.17E2,2.6E1,4E1,8.4E1,2.4E1,4E0,2.82E2,1.6E2,2.8E1,2E0,3E0,2.8E1,1.7E1,8.8E1,4.9E1,6E0,1.1E1,2.8E1,1.4E1,1.58E2,7E0,7E0,4.7E1,1.06E2,5E0,3.6E1,6.48E2,5E0,2.53E2,3.64E2,5E0,2.1E1,2E0,8.2E1,1.6E1,8E0,1.83E2,9.9E1,7.7E1,8.3E1,6E0,1.1E1,4E0,8.4E1,1.1E1,3.8E1,8E0,3E0,8.7E1,7.1E1,3.1E1,1.6E1,8.3E1,2.3E1,3E0,2E0,2.9E1,7E0,2.06E2,4.42E2,1.68E2,8.5E1,2E0,3.62E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-1.0351321E-3,-9.22095E-3,1.1703669E-2,-8.717156E-4,-3.1310078E-2,3.020403E-2,1.228821E-4,5.6635885E-4,-1.7047733E-1,-8.9964405E-2,-2.1326074E-2,-1.9878922E-2,4.2211216E-2,3.2005638E-3,-4.4021127E-3,5.4684356E-3,-4.915344E-2,5.4527284E-3,-3.0860084E-1,-2.1504734E-1,-2.3053824E-3,4.4316027E-2,-3.125569E-2,-1.3446878E-1,1.3709729E-2,1.0522257E-2,3.820892E-2,-4.6437304E-3,4.1012384E-2,1.0765594E-3,5.2946784E-2,-6.586064E-2,8.720801E-2,-1.9719334E-2,-1.6189173E-3,-1.3470543E-2,-6.266413E-4,-2.23503E-3,1.0452187E-1,-1.3738455E-1,-2.1128677E-2,-3.4813446E-3,-1.5923979E-2,2.926725E-3,-1.862494E-3,4.313307E-2,-2.2545496E-3,-1.196464E-2,2.917685E-2,7.5314394E-3,1.3819087E-3,1.80894E-2,-1.6428579E-2,3.9676197E-2,7.493475E-3,3.7091756E-3,-7.992898E-2,9.141252E-3,-6.757506E-4,-1.8806949E-3,6.095457E-3,-6.1728597E-2,-2.2575122E-1,5.221671E-3,-4.5370694E-2,3.8966984E-2,6.388206E-3,-5.9049954E-3,-5.9940048E-2,2.1114575E-2,9.735371E-3,-4.9291115E-4,1.729651E-3,-1.0559111E-3,4.9375794E-3,3.421892E-3,-2.4118379E-4,-1.4083701E-2,-3.030732E-3,-5.3249407E-3,3.7937127E-3,-1.314917E-2,1.3644493E-3,-5.4131616E-3,7.8956806E-4,-1.2754747E-3,-5.341214E-3,1.9811473E-3,-6.908964E-3,-4.5285383E-3,-8.4814E-5,-7.714554E-4,-5.316256E-3,1.393565E-3,-5.7679266E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,35,-1,37,39,41,43,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,61,63,-1,-1,-1,-1,65,-1,67,69,-1,-1,71,73,75,-1,-1,77,-1,-1,-1,-1,79,81,83,85,87,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8040585E-1,3.0188024E-1,2.2538595E-1,2.901002E-1,2.610181E-1,2.4438795E-1,1.8385938E-1,2.8787053E-1,4.2787087E-1,3.2970822E-1,2.5232843E-1,3.059081E-1,2.239126E-1,1.8645968E-1,0E0,2.2410095E-1,2.4723464E-1,0E0,1.8451208E-1,2.1033645E-1,0E0,2.84249E-1,3.5714465E-1,1.9603825E-1,1.5696873E-1,0E0,1.3683805E-1,1.2934734E-1,1.4809252E-1,2.936421E-1,1.1880994E-1,1.977256E-1,1.2873745E-1,0E0,0E0,0E0,0E0,0E0,1.0643956E-1,1.799432E-1,1.963441E-1,0E0,0E0,0E0,0E0,1.0911393E-1,0E0,1.24362305E-1,1.2625167E-1,0E0,0E0,2.5619668E-1,3.28739E-1,1.1686435E-1,0E0,0E0,2.9149514E-1,0E0,0E0,0E0,0E0,1.2732032E-1,1.6916668E-1,1.99023E-1,1.9537485E-1,1.3444486E-1,0E0,1.3983572E-1,1.05654866E-1,1.0522493E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,38,38,39,39,40,40,45,45,47,47,48,48,51,51,52,52,53,53,56,56,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,36,-1,38,40,42,44,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,62,64,-1,-1,-1,-1,66,-1,68,70,-1,-1,72,74,76,-1,-1,78,-1,-1,-1,-1,80,82,84,86,88,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.83E3,5.5306E4,8E-3,1.6015803E7,3.451532E8,9.996076E2,2.6572757E10,1.062363E1,2.5489312E5,2.5454E4,3.2585382E10,3.114E3,1.05842E3,8.865177E-1,-4.4021127E-3,2.539E3,3.422351E6,5.4527284E-3,5.59E2,1.9939552E8,-2.3053824E-3,9.048701E7,7.359468E8,2.139E3,4.28586E5,1.0522257E-2,2.1751701E2,1.5515003E0,4.9469455E6,1E0,1.35E2,3.97E2,1.35419E5,-1.9719334E-2,-1.6189173E-3,-1.3470543E-2,-6.266413E-4,-2.23503E-3,2.75E2,6.09603E8,1.7291142E7,-3.4813446E-3,-1.5923979E-2,2.926725E-3,-1.862494E-3,1.8552577E2,-2.2545496E-3,1.4613059E0,7.9023E4,7.5314394E-3,1.3819087E-3,1.6171625E2,3.7E1,3.7057E5,7.493475E-3,3.7091756E-3,5.664E1,9.141252E-3,-6.757506E-4,-1.8806949E-3,6.095457E-3,5.0461445E-2,1.9E1,1.8108038E5,3.6401E4,2.52331E5,6.388206E-3,1.0989011E-2,1.46617E5,1E0,9.735371E-3,-4.9291115E-4,1.729651E-3,-1.0559111E-3,4.9375794E-3,3.421892E-3,-2.4118379E-4,-1.4083701E-2,-3.030732E-3,-5.3249407E-3,3.7937127E-3,-1.314917E-2,1.3644493E-3,-5.4131616E-3,7.8956806E-4,-1.2754747E-3,-5.341214E-3,1.9811473E-3,-6.908964E-3,-4.5285383E-3,-8.4814E-5,-7.714554E-4,-5.316256E-3,1.393565E-3,-5.7679266E-3],"split_indices":[2,29,57,51,5,55,12,54,33,9,31,29,4,35,0,2,9,0,0,12,0,7,5,29,1,0,54,39,51,16,8,2,1,0,0,0,0,0,11,5,45,0,0,0,0,56,0,41,2,0,0,55,3,9,0,0,47,0,0,0,0,27,3,28,9,11,0,57,9,105,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.687E3,1.636E3,1.051E3,1.188E3,4.48E2,4.04E2,6.47E2,1.179E3,9E0,6.4E1,3.84E2,7.8E1,3.26E2,6.27E2,2E1,1.074E3,1.05E2,3E0,6E0,1.5E1,4.9E1,5E1,3.34E2,1.7E1,6.1E1,6E0,3.2E2,5.2E2,1.07E2,9.84E2,9E1,9.4E1,1.1E1,4E0,2E0,1.1E1,4E0,2E1,3E1,2.8E1,3.06E2,1.4E1,3E0,3.2E1,2.9E1,3.03E2,1.7E1,4.28E2,9.2E1,9E0,9.8E1,4.99E2,4.85E2,8.1E1,9E0,8E0,8.6E1,5E0,6E0,4E0,2.6E1,1.6E1,1.2E1,1.47E2,1.59E2,2.91E2,1.2E1,3.81E2,4.7E1,8.9E1,3E0,1.94E2,3.05E2,4.64E2,2.1E1,4.7E1,3.4E1,5E0,8.1E1,1.2E1,4E0,1E1,2E0,1.2E1,1.35E2,1.25E2,3.4E1,2.88E2,3E0,1.6E1,3.65E2,2.6E1,2.1E1,8.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[5.0632947E-4,1.3622423E-3,-8.9590184E-2,-4.6076654E-3,1.3854655E-2,-2.4001086E-3,-9.876812E-3,1.0749808E-2,-1.2399439E-2,2.4207287E-2,-5.2403696E-3,5.940075E-3,7.139234E-2,-4.2675834E-2,-5.469837E-3,7.490368E-2,1.55816665E-2,-1.443045E-2,2.8017194E-3,8.222949E-3,-1.5175004E-1,8.902365E-2,-2.6392427E-3,-1.728675E-1,-3.366444E-2,-7.976705E-3,7.275965E-2,8.4878534E-2,-4.3913145E-3,5.231528E-2,4.699666E-3,-3.149252E-3,-2.731515E-3,1.5043318E-2,-4.289893E-2,9.097752E-4,-1.2355645E-2,5.8336486E-3,2.4417273E-4,6.6346987E-3,-2.3631346E-1,-2.059524E-2,-1.2385404E-1,6.167797E-4,-3.3686347E-2,-3.3701367E-3,4.9902312E-3,6.494585E-3,2.246757E-3,1.12606525E-1,3.4676395E-2,-5.9560616E-2,1.2545879E-2,-2.1838553E-2,2.5485573E-2,-6.1286177E-4,1.5563488E-3,-4.4888537E-3,2.2913506E-3,-6.4450456E-3,-1.9450184E-2,-1.683627E-4,-4.081503E-3,-1.1058426E-2,-3.0384173E-3,4.1796328E-3,-1.7521942E-4,-8.375788E-3,-1.3055303E-3,1.3188211E-3,8.738541E-3,-6.438924E-3,2.0355212E-3,-7.6779258E-3,-5.100913E-4,3.2033145E-3,1.5940925E-5,-1.4430815E-3,5.969806E-3,1.1723967E-2,6.800888E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,-1,49,51,53,-1,55,57,-1,-1,-1,-1,-1,59,61,63,65,67,-1,-1,-1,-1,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0486172E-1,1.9628002E-1,1.04776174E-1,2.1325123E-1,1.6835573E-1,0E0,0E0,1.7458057E-1,2.4775416E-1,2.4021581E-1,1.7658502E-1,2.0176044E-1,1.0304366E-1,2.522865E-1,1.8943527E-1,1.4072657E-1,1.8882889E-1,1.2623101E-1,0E0,1.9241388E-1,1.5003285E-1,1.06698215E-1,0E0,3.0934316E-1,2.4144126E-1,2.0637447E-1,1.4233911E-1,1.400209E-1,0E0,1.1094999E-1,1.8528438E-1,1.1240602E-1,0E0,2.3658636E-1,3.0431536E-1,0E0,0E0,0E0,0E0,0E0,1.5266758E-1,2.0060405E-1,1.5349635E-1,2.600457E-1,2.0755237E-1,0E0,0E0,0E0,0E0,1.3760918E-1,1.1823745E-1,1.9542041E-1,2.1700251E-1,1.5807047E-1,2.0194659E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,40,40,41,41,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,-1,50,52,54,-1,56,58,-1,-1,-1,-1,-1,60,62,64,66,68,-1,-1,-1,-1,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5265896E8,8.027061E2,1.7243155E12,2.2444444E0,2.8839298E7,-2.4001086E-3,-9.876812E-3,1.9826213E0,1.8122449E2,1.4075E4,1E0,2E0,6.731367E0,5.8941E1,6.32E2,2.0319266E-7,2.6979439E1,1.2737473E1,2.8017194E-3,1E0,5.3E1,2.1143505E6,-2.6392427E-3,1.57E2,4.4821704E7,6.015825E0,3.0972284E1,7.4456794E5,-4.3913145E-3,1.3E1,3.557179E1,4.7272725E0,-2.731515E-3,1E0,6.68E3,9.097752E-4,-1.2355645E-2,5.8336486E-3,2.4417273E-4,6.6346987E-3,2.5363636E0,1.593E2,2.1340163E8,6.647279E5,3.6789783E2,-3.3701367E-3,4.9902312E-3,6.494585E-3,2.246757E-3,2.6956522E0,1.954E3,5.2369475E6,1.5277338E0,1.7310865E6,4.7777777E0,-6.1286177E-4,1.5563488E-3,-4.4888537E-3,2.2913506E-3,-6.4450456E-3,-1.9450184E-2,-1.683627E-4,-4.081503E-3,-1.1058426E-2,-3.0384173E-3,4.1796328E-3,-1.7521942E-4,-8.375788E-3,-1.3055303E-3,1.3188211E-3,8.738541E-3,-6.438924E-3,2.0355212E-3,-7.6779258E-3,-5.100913E-4,3.2033145E-3,1.5940925E-5,-1.4430815E-3,5.969806E-3,1.1723967E-2,6.800888E-4],"split_indices":[45,52,31,56,45,0,0,56,52,9,96,6,54,50,8,37,58,54,0,105,0,50,0,29,12,53,56,32,0,3,58,54,0,66,29,0,0,0,0,0,53,52,7,45,55,0,0,0,0,54,2,45,53,28,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.654E3,2.63E3,2.4E1,1.78E3,8.5E2,1.9E1,5E0,5.99E2,1.181E3,5.51E2,2.99E2,5.56E2,4.3E1,2.19E2,9.62E2,7.9E1,4.72E2,2.62E2,3.7E1,5.49E2,7E0,3.8E1,5E0,1.3E1,2.06E2,9.33E2,2.9E1,7.5E1,4E0,1.07E2,3.65E2,2.08E2,5.4E1,4.85E2,6.4E1,3E0,4E0,2.7E1,1.1E1,2E0,1.1E1,1.81E2,2.5E1,7E2,2.33E2,5E0,2.4E1,3.1E1,4.4E1,2.3E1,8.4E1,3.9E1,3.26E2,1.26E2,8.2E1,1.87E2,2.98E2,4.1E1,2.3E1,8E0,3E0,1.44E2,3.7E1,8E0,1.7E1,3.2E1,6.68E2,9E0,2.24E2,1.1E1,1.2E1,3E0,8.1E1,1.2E1,2.7E1,5.9E1,2.67E2,1.2E2,6E0,3E0,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[1.04701E-3,-5.101969E-4,4.8529644E-2,1.3016697E-2,-6.923786E-3,1.0006952E-1,9.419537E-3,1.4987735E-2,-2.2480424E-1,-3.6442153E-2,-2.2554586E-3,1.1942421E-1,-2.283673E-3,-5.448645E-4,6.6470536E-3,9.980535E-3,6.5060176E-2,4.6291022E-4,-1.9303402E-2,-1.6704468E-1,-2.833399E-2,-1.299561E-3,-7.0967386E-3,1.28666805E-2,4.289764E-3,8.645347E-3,8.476427E-3,9.538927E-2,-4.6274066E-3,-9.994523E-3,2.6454101E-3,4.8131095E-3,-4.967722E-2,-2.0690986E-3,1.11905495E-2,1.58717E-2,-2.0751279E-2,2.0031498E-3,6.6220053E-3,-3.8318925E-3,2.6308717E-3,-6.761639E-3,2.2559604E-2,-4.39713E-3,-6.838517E-2,3.3913122E-3,-3.9289654E-3,-7.6697243E-4,1.287678E-3,-1.4811134E-3,2.4441313E-3,2.768058E-3,-1.1529619E-3,-2.054178E-3,4.569593E-3,-2.0818287E-3,-6.7235227E-3,4.0361355E-4,-5.9861975E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,-1,-1,29,31,33,-1,-1,-1,35,-1,37,39,-1,-1,41,43,45,-1,47,49,-1,-1,-1,-1,-1,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0025504E-1,2.2765094E-1,1.7241018E-1,3.9899778E-1,2.452043E-1,1.1150542E-1,1.3611096E-1,2.094563E-1,2.9648784E-1,2.5259098E-1,2.1435118E-1,1.2294665E-1,0E0,0E0,0E0,1.6861641E-1,1.6107205E-1,0E0,0E0,1.411429E-1,1.6295631E-1,2.769533E-1,0E0,0E0,0E0,1.6135159E-1,0E0,1.1595529E-1,1.1216009E-1,0E0,0E0,2.3896922E-1,1.1829293E-1,2.0699587E-1,0E0,2.143558E-1,1.1115179E-1,0E0,0E0,0E0,0E0,0E0,1.3615075E-1,1.6547644E-1,1.71509E-1,0E0,1.5769568E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,19,19,20,20,21,21,25,25,27,27,28,28,31,31,32,32,33,33,35,35,36,36,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,-1,-1,30,32,34,-1,-1,-1,36,-1,38,40,-1,-1,42,44,46,-1,48,50,-1,-1,-1,-1,-1,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8552577E2,8E0,2.6749698E7,4.004366E1,9E2,1.701E3,2.62797E5,1E0,1E1,2.8972292E-5,1.314806E8,2.0185E4,-2.283673E-3,-5.448645E-4,6.6470536E-3,1.8924E4,4.838871E8,4.6291022E-4,-1.9303402E-2,5.750032E3,1.04E2,2.4507338E4,-7.0967386E-3,1.28666805E-2,4.289764E-3,1E0,8.476427E-3,7.2488395E6,7.73E3,-9.994523E-3,2.6454101E-3,1.56E2,5.827505E5,2.139E3,1.11905495E-2,1E1,2.632E5,2.0031498E-3,6.6220053E-3,-3.8318925E-3,2.6308717E-3,-6.761639E-3,4.6E0,3.591E3,4.752366E0,3.3913122E-3,2.1525E4,-7.6697243E-4,1.287678E-3,-1.4811134E-3,2.4441313E-3,2.768058E-3,-1.1529619E-3,-2.054178E-3,4.569593E-3,-2.0818287E-3,-6.7235227E-3,4.0361355E-4,-5.9861975E-4],"split_indices":[56,17,50,53,2,11,2,84,8,42,1,10,0,0,0,6,7,0,0,33,10,4,0,0,0,14,0,45,29,0,0,2,45,29,0,3,33,0,0,0,0,0,54,29,56,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.707E3,2.622E3,8.5E1,8.43E2,1.779E3,3.6E1,4.9E1,8.37E2,6E0,2.42E2,1.537E3,3.2E1,4E0,4.3E1,6E0,7.62E2,7.5E1,3E0,3E0,1.3E1,2.29E2,1.528E3,9E0,4E0,2.8E1,7.57E2,5E0,5.2E1,2.3E1,1.1E1,2E0,9E1,1.39E2,1.524E3,4E0,6.08E2,1.49E2,2.4E1,2.8E1,1E1,1.3E1,9E0,8.1E1,4.1E1,9.8E1,3.7E1,1.487E3,1.56E2,4.52E2,1.31E2,1.8E1,4.6E1,3.5E1,3E1,1.1E1,7.4E1,2.4E1,6.09E2,8.78E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"93"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.2287959E-2,-6.075751E-1,2.6339018E-1,-9.5331484E-1,-2.163495E-1,6.359771E-2,7.384893E-1,-9.8467934E-1,9.649431E-2,-4.2805773E-1,4.266736E-1,-4.5035985E-1,1.5992919E-1,5.7733446E-1,1.2181703E0,-1.1201115E0,-4.83631E-1,-5.558979E-1,-2.2035332E-1,2.4327299E-1,8.5211366E-1,-6.6684306E-1,-3.5399643E-1,3.376331E-1,-1.11385696E-1,3.3770868E-1,8.6937344E-1,9.870721E-1,1.51824E0,-4.2884657E-1,-1.2151731E0,-1.9648427E-1,-7.58687E-1,-6.977819E-1,-3.862006E-1,-5.207925E-2,-4.314592E-1,1.2412835E-2,3.644517E-1,9.3588597E-1,1.5692027E-2,-6.866104E-1,-1.2127354E-2,-2.7863604E-1,-4.284348E-1,2.9723728E-1,1.629395E0,-1.4532039E-1,8.439421E-2,-1.9193913E-1,4.678388E-1,6.480753E-1,1.1868311E0,1.0541254E0,2.8089815E-1,4.402536E-2,7.355771E-2,-7.1746767E-1,2.0818464E-1,-1.1046746E0,-1.5868559E0,1.9852724E-2,-3.4153613E-1,-9.425081E-1,-3.4998298E-1,-7.8366053E-1,-5.1677E-1,-4.096129E-1,-2.9110531E-3,-1.3728237E-1,2.2719845E-1,-5.390091E-1,1.8329167E-1,-9.390399E-2,1.0796087E-2,8.208957E-2,4.9161538E-1,1.9194726E-2,4.5332123E-2,-3.171935E-2,-1.1216911E-2,-3.9702725E-1,-2.0167108E-1,-4.9056277E-1,-1.5457454E-1,-5.371406E-1,3.299525E-1,8.286849E-2,4.479216E-2,-9.136485E-2,-5.802398E-1,2.2376612E-2,-2.9009074E-1,3.0558974E-1,6.348574E-1,7.8146976E-1,1.4212093E-1,1.2209609E0,1.828047E-2,1.0794073E0,1.006871E-2,1.8257296E-2,-7.5496675E-4,-9.136754E-3,-7.7861136E-1,4.062588E-2,-1.9378807E-2,-1.2139422E0,-6.336755E-1,-1.6267382E0,-1.5996013E-2,-2.1964003E-1,3.9336237E-1,-1.6115408E-1,-2.6166333E-2,-7.0663184E-1,-1.2728345E0,-1.2109241E-1,-2.5424702E-2,-8.067704E-1,-5.866257E-3,-1.0149202E-2,-5.6838715E-1,-4.6854207E-1,-2.3918061E-1,-1.8568987E-1,8.335429E-3,2.3653503E-3,1.35437185E-2,-7.213844E-1,-3.5703966E-1,-1.3627921E-3,1.2435349E-2,9.4024575E-4,-1.031904E-2,-4.858914E-2,1.7578242E-2,1.4130814E-2,5.682835E-1,-2.2762196E-1,-2.3082001E-2,-3.6585186E-2,-2.4997734E-1,-2.2840735E-2,-5.551477E-3,1.0965692E-3,-1.293683E-2,-3.0379122E-2,-8.723121E-3,1.3708247E-1,5.1253307E-1,-2.2623189E-1,5.3102415E-2,-7.935744E-1,-2.3730625E-1,-5.536455E-1,-1.3314764E-1,5.736188E-2,3.697583E-1,3.554119E-1,8.046088E-1,8.202238E-1,1.0587163E-1,-2.3133924E-2,2.430749E-1,1.0254282E0,1.4146565E0,8.466854E-1,1.2014986E0,-1.3172391E-2,-3.942533E-2,-9.184141E-3,3.994716E-3,-5.6535237E-2,-2.8768888E-2,-1.4517033E-2,-3.791657E-2,-7.50437E-2,-2.5387041E-2,-4.481794E-4,-1.200435E-2,2.6661087E-2,-2.8806552E-3,-1.9818073E-2,-9.409865E-4,-1.732384E-2,-3.49455E-2,-2.318189E-2,-6.33396E-2,5.0738244E-4,-9.177636E-3,-3.8075015E-2,-1.8559834E-2,-2.9217733E-2,-1.6679043E-2,-2.2204112E-2,-4.756726E-3,-1.32419355E-2,1.5316522E-3,-1.0362046E-2,-2.091852E-3,-4.3962948E-2,-2.58003E-2,-1.8727234E-2,-1.2322275E-3,-6.611131E-3,1.8429055E-3,2.8439976E-2,1.3532753E-2,-6.7743934E-3,-1.7463084E-2,2.7385952E-3,-5.0427555E-3,-3.7742013E-3,-1.2675145E-2,9.547704E-3,-2.2837291E-3,2.9417343E-2,1.265296E-2,-1.4716425E-2,8.849982E-3,5.8162655E-4,2.3314774E-2,-3.7574686E-2,-1.2427102E-2,-1.5145935E-2,6.249124E-3,-2.9125908E-2,-9.6638575E-3,5.266232E-3,-8.088609E-3,5.1769316E-3,-1.8281813E-3,1.51933655E-2,2.8389513E-2,5.8117034E-3,2.1195123E-2,4.1381128E-2,2.1432921E-2,3.8050596E-2,4.9136905E-3,1.8617257E-2,-1.0628547E-2,2.4078928E-2,4.124287E-3,1.9025203E-2,4.761767E-2,2.11823E-2,6.673656E-2,2.5086207E-3,4.162516E-2,3.2413583E-2,5.782708E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,-1,81,83,85,87,89,-1,91,93,95,97,99,101,-1,-1,103,105,107,109,111,113,115,117,119,121,123,-1,125,127,129,131,133,-1,135,137,-1,-1,-1,-1,139,141,143,145,147,149,-1,-1,151,153,-1,155,157,159,161,163,165,-1,167,-1,-1,-1,-1,169,-1,171,173,175,177,-1,179,181,183,-1,185,187,189,-1,191,-1,-1,193,195,197,199,-1,-1,-1,201,203,-1,-1,-1,-1,205,-1,-1,207,209,-1,211,213,-1,-1,-1,-1,-1,-1,215,217,219,221,223,225,227,229,231,233,235,237,239,241,-1,243,245,247,249,251,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.577506E2,9.312471E1,1.4138565E2,3.750876E1,4.4425964E1,5.2057827E1,3.33873E1,2.4090729E1,0E0,6.3829803E0,6.1128063E0,3.2268753E0,4.270674E1,2.2939796E1,6.1293945E0,1.8207031E1,6.0904636E0,3.3661957E0,3.3522763E0,1.6180456E0,6.919708E-1,2.104168E-1,5.369568E-1,2.7432938E1,2.3800793E1,1.2771654E1,9.698502E0,2.7817345E0,1.0936737E0,6.768349E0,8.514648E0,1.2619839E0,2.675274E0,8.5417557E-1,5.087509E-1,1.3161381E0,2.8756852E0,5.270804E-1,1.3240795E0,2.4506378E-1,0E0,1.15161896E-1,0E0,4.757619E-1,8.939924E-1,1.4371593E1,2.4729538E-1,8.066328E0,0E0,2.6353445E0,3.8165855E0,5.9631805E0,1.1105423E0,1.0005798E0,2.6161188E-1,0E0,0E0,6.310978E-1,1.8992934E0,9.317902E0,2.1310577E0,1.6070967E0,9.303601E-1,1.2043095E0,6.100364E-1,6.966057E-1,3.1057024E-1,5.508013E-1,0E0,6.876172E-1,1.4801759E-1,9.3715954E-1,1.4476253E-1,2.3525925E-1,0E0,5.390015E-1,1.8043375E-1,0E0,0E0,0E0,0E0,3.2346582E-1,3.0186713E-1,1.740427E-1,2.8835416E-1,7.929621E-1,1.757211E1,0E0,0E0,6.032531E0,2.5917435E0,0E0,1.2912655E0,1.2052798E0,3.129448E0,1.7720947E0,1.4428611E0,7.081833E-1,0E0,6.022644E-1,0E0,0E0,0E0,0E0,7.358732E-1,0E0,2.153457E-1,2.386795E0,2.2196617E0,8.111572E-1,0E0,1.02371514E-1,6.629274E-1,5.7325983E-1,0E0,8.509064E-2,2.4373817E-1,8.8659026E-2,0E0,3.4781647E-1,0E0,0E0,1.3166332E-1,3.3050537E-1,2.863946E-1,2.1229851E-1,0E0,0E0,0E0,8.968449E-2,3.621204E-1,0E0,0E0,0E0,0E0,9.294924E-2,0E0,0E0,8.650017E-2,7.791662E-2,0E0,7.934735E-2,1.2655437E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.4015002E0,8.03791E0,6.7313147E0,2.7823033E0,2.0636845E-1,6.3009906E-1,2.2878361E-1,2.7056143E-1,9.930463E-2,3.2589436E-1,6.822226E-1,1.1616898E0,8.733444E-1,6.1658347E-1,0E0,7.415267E-1,6.812668E-2,7.726517E-1,1.2725296E0,3.7324905E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,75,75,76,76,81,81,82,82,83,83,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,99,99,104,104,106,106,107,107,108,108,109,109,111,111,112,112,113,113,115,115,116,116,117,117,119,119,122,122,123,123,124,124,125,125,129,129,130,130,135,135,138,138,139,139,141,141,142,142,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,160,160,161,161,162,162,164,164,165,165,166,166,167,167,168,168],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,-1,82,84,86,88,90,-1,92,94,96,98,100,102,-1,-1,104,106,108,110,112,114,116,118,120,122,124,-1,126,128,130,132,134,-1,136,138,-1,-1,-1,-1,140,142,144,146,148,150,-1,-1,152,154,-1,156,158,160,162,164,166,-1,168,-1,-1,-1,-1,170,-1,172,174,176,178,-1,180,182,184,-1,186,188,190,-1,192,-1,-1,194,196,198,200,-1,-1,-1,202,204,-1,-1,-1,-1,206,-1,-1,208,210,-1,212,214,-1,-1,-1,-1,-1,-1,216,218,220,222,224,226,228,230,232,234,236,238,240,242,-1,244,246,248,250,252,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.14099E5,5.549173E6,3.79E2,1.8137958E6,2.1790159E3,9.649431E-2,2.785857E2,1.4101E4,3.79E2,2E1,1.4101E4,7.431E3,2.461362E-3,7.218466E7,1.7751849E-1,1.2471935E7,3.386E3,4.8765745E0,5.2350176E4,9.25E0,1.3999657E3,9.256843E3,2.4344647E-1,1.5103707E0,1.4274633E7,2.5247778E6,1E0,6.971004E7,3.9401836E11,3.970405E3,1.8122449E2,9.705292E6,5.5871747E2,6.306E3,1.01E3,1.3118889E0,1.6081998E5,1.5692027E-2,1.5E1,-1.2127354E-2,1.56E2,3.30399E6,3.983889E3,1.86E2,3.956147E-1,8.439421E-2,3.8833074E4,3.1484194E5,2.851369E2,1.314806E8,1E0,1.9473622E7,4.402536E-2,7.355771E-2,1E0,2.5E1,3.386E3,1.0872E4,4E0,4.8765745E0,1.2427474E8,1.612E3,1.1994927E7,6.895302E7,1.1400756E6,-2.9110531E-3,8.715555E5,2.028994E8,6.568389E0,3.3E1,4.005E3,1.0796087E-2,2.2603108E3,4.492E3,1.9194726E-2,4.5332123E-2,-3.171935E-2,-1.1216911E-2,1.24E2,8E0,8.594445E5,1.75E2,4.1741933E-7,9.52E2,8.286849E-2,4.479216E-2,8.17766E5,2.6476662E6,2.2376612E-2,1.1902924E0,1.6162432E3,8.8543115E2,5.4106476E7,2.222456E0,7.7101436E3,1.828047E-2,2.9251662E6,1.006871E-2,1.8257296E-2,-7.5496675E-4,-9.136754E-3,2.127E3,4.062588E-2,1.4E1,5.8426323E0,2.7246006E7,4.286E3,-1.5996013E-2,5.84E2,1.942E3,1.7E1,-2.6166333E-2,1.0192E4,3.4183568E8,3.8E1,-2.5424702E-2,9.4538574E2,-5.866257E-3,-1.0149202E-2,4.217926E-2,3.275E3,6.025862E0,1.671E3,8.335429E-3,2.3653503E-3,1.35437185E-2,9.757E3,1E0,-1.3627921E-3,1.2435349E-2,9.4024575E-4,-1.031904E-2,1.1355795E7,1.7578242E-2,1.4130814E-2,2.9722316E0,4.9122566E1,-2.3082001E-2,1.8903887E0,1.0710737E2,-2.2840735E-2,-5.551477E-3,1.0965692E-3,-1.293683E-2,-3.0379122E-2,-8.723121E-3,8E0,8E0,6.743895E5,1.4777102E9,2.9049585E3,3.670858E1,1.4813794E3,1.8722344E8,1.8465776E8,1.4777102E9,9.112E3,1.2471935E7,3.3027109E10,5.491E3,-2.3133924E-2,1.0330753E5,1.82E4,5.3871745E-1,1.1057693E0,1.460806E6,-1.3172391E-2,-3.942533E-2,-9.184141E-3,3.994716E-3,-5.6535237E-2,-2.8768888E-2,-1.4517033E-2,-3.791657E-2,-7.50437E-2,-2.5387041E-2,-4.481794E-4,-1.200435E-2,2.6661087E-2,-2.8806552E-3,-1.9818073E-2,-9.409865E-4,-1.732384E-2,-3.49455E-2,-2.318189E-2,-6.33396E-2,5.0738244E-4,-9.177636E-3,-3.8075015E-2,-1.8559834E-2,-2.9217733E-2,-1.6679043E-2,-2.2204112E-2,-4.756726E-3,-1.32419355E-2,1.5316522E-3,-1.0362046E-2,-2.091852E-3,-4.3962948E-2,-2.58003E-2,-1.8727234E-2,-1.2322275E-3,-6.611131E-3,1.8429055E-3,2.8439976E-2,1.3532753E-2,-6.7743934E-3,-1.7463084E-2,2.7385952E-3,-5.0427555E-3,-3.7742013E-3,-1.2675145E-2,9.547704E-3,-2.2837291E-3,2.9417343E-2,1.265296E-2,-1.4716425E-2,8.849982E-3,5.8162655E-4,2.3314774E-2,-3.7574686E-2,-1.2427102E-2,-1.5145935E-2,6.249124E-3,-2.9125908E-2,-9.6638575E-3,5.266232E-3,-8.088609E-3,5.1769316E-3,-1.8281813E-3,1.51933655E-2,2.8389513E-2,5.8117034E-3,2.1195123E-2,4.1381128E-2,2.1432921E-2,3.8050596E-2,4.9136905E-3,1.8617257E-2,-1.0628547E-2,2.4078928E-2,4.124287E-3,1.9025203E-2,4.761767E-2,2.11823E-2,6.673656E-2,2.5086207E-3,4.162516E-2,3.2413583E-2,5.782708E-2],"split_indices":[19,109,12,2,50,2,27,54,0,51,2,1,3,2,2,26,44,40,44,2,56,27,57,51,4,41,40,44,27,15,44,30,51,51,9,51,2,0,38,27,0,0,0,2,9,27,10,26,0,32,27,57,1,111,44,0,0,52,0,2,2,8,56,44,0,9,7,32,0,32,7,53,3,10,0,4,2,0,0,0,0,28,10,27,10,36,2,0,0,1,31,0,52,54,51,44,53,51,0,27,0,0,0,0,1,0,3,52,44,0,0,0,0,3,0,9,7,3,0,4,0,0,56,2,53,0,0,0,0,9,94,0,0,0,0,49,0,0,56,51,0,52,4,0,0,0,0,0,0,16,16,27,7,4,57,4,7,5,7,10,44,12,9,0,32,10,41,53,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.178E3,6.89E2,1.489E3,3.65E2,3.24E2,1.049E3,4.4E2,3.62E2,3E0,2.44E2,8E1,1.65E2,8.84E2,3.31E2,1.09E2,2.84E2,7.8E1,1.5E2,9.4E1,5.7E1,2.3E1,4.9E1,1.16E2,5.34E2,3.5E2,1.83E2,1.48E2,6.4E1,4.5E1,3.5E1,2.49E2,3.9E1,3.9E1,8E1,7E1,5.3E1,4.1E1,2E1,3.7E1,1.9E1,4E0,4.6E1,3E0,6E1,5.6E1,5.19E2,1.5E1,3.45E2,5E0,3.6E1,1.47E2,8.9E1,5.9E1,5.8E1,6E0,9E0,3.6E1,2.4E1,1.1E1,1.95E2,5.4E1,1.6E1,2.3E1,2.6E1,1.3E1,5.2E1,2.8E1,6.5E1,5E0,4.1E1,1.2E1,3.5E1,6E0,1.4E1,6E0,1.2E1,2.5E1,3E0,1.6E1,4.4E1,2E0,2.2E1,3.8E1,4.5E1,1.1E1,1.9E1,5E2,1E1,5E0,3.08E2,3.7E1,4E0,3.2E1,7.6E1,7.1E1,7E1,1.9E1,5.6E1,3E0,5.6E1,2E0,4E0,2E0,3E0,2.1E1,2E0,9E0,1.57E2,3.8E1,5.2E1,2E0,1E1,6E0,1.4E1,9E0,1.7E1,9E0,7E0,6E0,5E1,2E0,5E0,2.3E1,4.7E1,1.8E1,3.6E1,5E0,4E0,8E0,1.6E1,1.9E1,2E0,4E0,8E0,6E0,9E0,3E0,9E0,1.6E1,1E1,1.2E1,9E0,2.9E1,4.3E1,2E0,5E0,6E0,1.3E1,6E0,2.44E2,2.56E2,1.59E2,1.49E2,2.2E1,1.5E1,1.1E1,2.1E1,1.6E1,6E1,2.8E1,4.3E1,6.6E1,4E0,2E0,1.7E1,3.1E1,2.5E1,2.2E1,3.4E1,4E0,1.7E1,3E0,6E0,1.47E2,1E1,1.6E1,2.2E1,5E1,2E0,2E0,8E0,4E0,2E0,4E0,1E1,4E0,1.3E1,2E0,7E0,3E0,4E0,4.5E1,5E0,1.5E1,8E0,4.4E1,3E0,1.5E1,3E0,2.7E1,9E0,4E0,1.2E1,1.6E1,3E0,4E0,5E0,1.2E1,4E0,8E0,2E0,4E0,5E0,5E0,2.4E1,1.75E2,6.9E1,1.6E2,9.6E1,1.29E2,3E1,1.38E2,1.1E1,2E1,2E0,1.2E1,3E0,8E0,3E0,3E0,1.8E1,1E1,6E0,5.5E1,5E0,1E1,1.8E1,3.1E1,1.2E1,6.4E1,2E0,2E0,2E0,5E0,1.2E1,2E0,2.9E1,2E0,2.3E1,2E0,2E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"253","size_leaf_vector":"1"}},{"base_weights":[-1.053385E-2,-5.8354586E-1,2.5445834E-1,-8.8465095E-1,-2.2395018E-1,1.4669156E-2,6.048662E-1,-9.6085006E-1,7.223453E-1,-4.3677148E-1,2.7129865E-1,-7.225671E-2,5.056615E-1,-5.5647574E-2,7.181664E-1,-1.1459941E0,-6.439725E-1,9.511374E-2,3.054766E-2,-5.153751E-1,-2.0421593E-1,1.0184818E-2,5.117333E-1,-4.055758E-1,2.3651041E-2,2.9032344E-1,7.6146835E-1,-5.1761335E-1,1.9936107E-1,3.3386216E-1,8.9050615E-1,-9.338133E-1,-1.3709036E0,5.9255282E-3,-7.6728016E-1,4.250893E-1,-2.26641E-2,-5.7641655E-1,-2.897737E-1,-4.7322117E-2,-1.2404624E-1,-9.712488E-2,4.2767417E-1,8.2738507E-1,3.699357E-1,-6.2531334E-1,-3.0597526E-1,1.1821808E0,-5.9435116E-3,3.2090682E-1,-2.5391487E-2,8.2152903E-1,-1.425195E-2,-5.5348974E-1,1.1611403E-3,4.213499E-1,-1.7607097E-1,3.9847496E-1,-3.9611056E-1,7.45255E-1,1.308875E0,-1.0585089E0,-7.0478326E-1,-1.4573431E0,-6.7408013E-1,-8.232144E-2,2.8145872E-2,-1.0580426E0,-5.942228E-1,-2.9642265E-3,2.714274E-2,-7.04544E-1,-4.922866E-1,-2.1017347E-2,-2.0150696E-1,-3.1093562E-1,-3.2255307E-2,-3.1529116E-3,-2.4726984E-1,4.8284302E-3,2.2392463E-2,8.118227E-3,9.132951E-1,9.3762174E-2,4.687495E-1,-6.4745927E-1,-1.8288628E-3,-3.3267248E-1,-8.760008E-3,-5.2699177E-3,6.865913E-2,4.4911407E-2,-4.898672E-1,-2.735291E-3,3.3787E-1,6.8421745E-1,4.9633026E-2,-2.646824E-2,-3.0798973E-3,6.528838E-2,5.2960044E-1,-2.8936493E-1,1.2422448E-1,9.725767E-2,4.9371445E-1,-4.656636E-2,-1.3012382E-1,5.7186735E-1,9.696331E-1,1.3786196E0,6.636318E-1,-8.367497E-1,-1.1976099E0,-2.865076E-1,-8.979325E-1,-1.3408912E0,-1.9630747E0,-3.9820068E-2,-1.1479197E-2,-2.1248704E-2,-1.5237781E-2,-3.3492766E-2,-5.254895E-2,-8.4076005E-1,-4.6171972E-1,-7.804736E-1,-2.0533925E-2,-5.1563144E-1,-7.477755E-3,-2.5505972E-1,-5.460944E-4,-2.2406413E-3,-3.8209912E-1,-1.2531851E-1,1.7982483E-1,5.6668986E-2,-9.146866E-3,3.4806177E-3,-3.3471364E-1,1.515239E-2,4.4733908E-2,-7.521694E-3,1.6997732E-1,3.012275E-1,2.5584042E-2,-3.0665223E-2,-1.0739863E-2,-2.3329228E-1,-4.2961794E-1,-4.994735E-3,4.267635E-3,-9.594819E-2,1.9318302E-1,-8.2036734E-1,-4.723236E-2,4.3519613E-1,2.6825634E-1,5.2991974E-1,8.0040085E-1,-2.039793E-1,1.2622228E-2,5.606161E-1,3.3469384E-3,-2.348054E-3,-1.7385287E-2,-9.371661E-4,8.436086E-3,-1.3575806E-1,2.2385617E-1,5.2751166E-1,3.329049E-2,-8.682717E-3,-8.048798E-4,3.4134597E-1,7.6813775E-1,6.3680804E-1,1.1385814E0,7.221236E-2,1.2039022E0,1.441484E-2,4.0668085E-2,-1.6742596E-2,-4.0221542E-2,-5.6183584E-2,-3.2096755E-2,-1.8167224E-2,-3.695578E-3,-4.214621E-2,-1.2667132E-2,-6.4084925E-2,-3.972503E-2,-3.7833113E-2,-9.346564E-2,-3.4450255E-3,4.578184E-3,-5.555724E-2,-2.2698905E-2,-1.636746E-2,-3.786643E-2,-3.671272E-2,-1.4791743E-2,-2.7980857E-2,-1.9790066E-2,-1.5954314E-2,-6.091029E-3,-1.939386E-2,-3.1538594E-3,1.6163456E-3,-1.0786567E-2,8.5472653E-4,1.528558E-2,-3.4495976E-3,4.285035E-3,-1.7662209E-2,-5.7214545E-3,2.3013377E-3,1.1550643E-2,1.7969858E-2,3.978848E-3,-1.7478567E-2,-5.715159E-3,-2.3246223E-2,-1.0356727E-2,-9.097973E-3,8.627695E-3,1.0117421E-2,-1.8086102E-2,-3.9548695E-2,-1.04021E-2,-6.446701E-3,3.1604744E-2,6.470288E-3,2.2298416E-2,-6.7832284E-3,1.4745652E-2,2.586435E-2,9.566112E-3,3.804318E-2,1.2119047E-2,-5.439874E-4,-1.4847592E-2,6.479319E-3,2.6634289E-2,-1.1321319E-2,5.4329885E-3,1.9107386E-2,3.8935903E-3,1.7151406E-2,2.9389523E-2,1.3119523E-2,-1.6288104E-2,9.681457E-3,2.8658394E-2,3.782751E-2,1.5204314E-2,3.6471706E-2,6.689279E-3,4.5332424E-2,7.628047E-2,5.6464683E-2,3.0726615E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,81,83,85,87,89,91,93,-1,95,-1,97,-1,99,101,103,105,107,109,111,113,115,117,119,-1,121,123,-1,-1,125,127,-1,129,131,133,135,137,-1,-1,-1,139,141,143,145,-1,147,149,-1,-1,151,153,-1,155,157,-1,-1,-1,159,161,163,165,167,169,-1,171,173,175,177,179,181,183,185,187,189,191,-1,-1,193,-1,-1,-1,195,197,199,-1,201,-1,203,-1,-1,205,207,209,211,-1,-1,213,-1,-1,-1,215,217,-1,-1,-1,219,221,-1,-1,223,225,227,229,231,233,235,237,239,-1,241,-1,-1,-1,-1,-1,243,245,247,249,-1,-1,251,253,255,257,-1,259,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.399747E2,7.6461334E1,1.286319E2,4.7837982E1,3.4267937E1,3.8875275E1,4.6588165E1,2.0851837E1,1.7101448E1,4.044239E0,6.1473064E0,2.4752735E1,7.3255615E0,1.0941277E1,3.4871307E1,9.7435E0,1.1063511E1,0E0,2.9370732E0,2.1602783E0,3.8483934E0,2.1919694E0,1.976614E0,3.594942E0,2.0647297E1,2.1583705E0,4.2663574E0,6.573734E-1,5.0746913E0,7.9873238E0,2.1265839E1,2.738182E0,5.64328E0,1.3009607E0,5.2082367E0,7.2381485E-1,0E0,1.0760384E0,4.9159217E-1,0E0,9.33482E-1,5.494876E-1,1.6209757E-1,6.7137146E-1,9.6577024E-1,6.603832E-1,9.5720005E-1,6.689148E0,1.4492622E1,4.9907923E-1,0E0,1.4757843E0,0E0,4.2877865E-1,0E0,1.430089E0,8.184444E-1,4.3412113E0,2.2212355E0,1.0105301E1,3.2703094E0,1.3731079E0,3.4590569E0,3.0803833E0,9.8659945E-1,3.189271E-1,0E0,5.134125E-1,2.0618267E0,0E0,0E0,5.98917E-1,5.486698E-1,0E0,2.708465E-1,3.1598556E-1,7.725359E-1,3.0888477E-1,4.5090485E-1,0E0,0E0,0E0,2.3177719E-1,2.4613069E-1,2.6555395E-1,4.642067E-1,0E0,9.691973E-1,1.2549469E-1,0E0,0E0,1.1150496E1,8.153542E0,0E0,3.642969E-1,3.287773E-1,0E0,0E0,0E0,6.317777E-1,3.6841393E-1,3.621844E-1,6.68656E-2,1.1513691E0,1.7875214E0,0E0,7.278761E-2,6.8411865E0,5.9101105E0,1.2177429E0,6.050596E-1,4.6740723E-1,7.369232E-2,3.434571E-1,3.715706E-1,1.5930786E0,1.0346985E-1,0E0,0E0,1.2849827E-1,0E0,0E0,0E0,2.6630669E0,1.6122265E0,2.45924E-1,0E0,3.4892654E-1,0E0,1.9894266E-1,0E0,0E0,1.7253995E-1,4.9948463E-1,2.9839185E-1,1.08128384E-1,0E0,0E0,8.921552E-2,0E0,0E0,0E0,7.833949E-2,2.100867E-1,0E0,0E0,0E0,8.8350844E-1,8.087425E-1,0E0,0E0,8.281763E0,4.7564535E0,8.042679E-1,1.8725101E0,3.987608E-1,1.1054807E0,1.5607548E-1,2.4254513E-1,1.1504902E-1,0E0,2.2778225E-1,0E0,0E0,0E0,0E0,0E0,4.3929234E-1,6.467203E-1,1.707201E0,1.0115944E0,0E0,0E0,2.6164837E0,2.1664963E0,3.347578E0,4.0878906E0,0E0,2.9003143E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,71,71,72,72,74,74,75,75,76,76,77,77,78,78,82,82,83,83,84,84,85,85,87,87,88,88,91,91,92,92,94,94,95,95,99,99,100,100,101,101,102,102,103,103,104,104,106,106,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,119,119,123,123,124,124,125,125,127,127,129,129,132,132,133,133,134,134,135,135,138,138,142,142,143,143,147,147,148,148,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,161,161,167,167,168,168,169,169,170,170,173,173,174,174,175,175,176,176,178,178],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,82,84,86,88,90,92,94,-1,96,-1,98,-1,100,102,104,106,108,110,112,114,116,118,120,-1,122,124,-1,-1,126,128,-1,130,132,134,136,138,-1,-1,-1,140,142,144,146,-1,148,150,-1,-1,152,154,-1,156,158,-1,-1,-1,160,162,164,166,168,170,-1,172,174,176,178,180,182,184,186,188,190,192,-1,-1,194,-1,-1,-1,196,198,200,-1,202,-1,204,-1,-1,206,208,210,212,-1,-1,214,-1,-1,-1,216,218,-1,-1,-1,220,222,-1,-1,224,226,228,230,232,234,236,238,240,-1,242,-1,-1,-1,-1,-1,244,246,248,250,-1,-1,252,254,256,258,-1,260,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,7.7344055E4,6.7652373E3,4.197889E6,1.6316416E6,2.1328075E-1,3.839604E-1,2.46E2,3.5259784E2,6.860185E2,4.36E2,1.028E3,2.6393727E4,3.596E3,7.03295E-1,1.0119178E10,9.511374E-2,1.0040462E8,4.387838E-1,5.2143492E-5,2.801121E0,4.9716983E0,9.6E1,8E0,4.6E1,5.99999E9,1.8314E4,1.4283712E6,5.884028E0,1.8137958E6,2.857E3,4.9716983E0,2.6108465E0,2.507E3,3.5912186E-1,-2.26641E-2,1.5288235E2,7.054E3,-4.7322117E-2,1.0396807E8,7.0988875E6,9.6771875E2,1.3617021E0,1.0855529E3,4.3906153E5,7.997723E6,1.3483871E2,4.0844156E7,3.5902756E-6,-2.5391487E-2,5.316494E6,-1.425195E-2,1E0,1.1611403E-3,2.8946466E2,2.778E3,2.5644116E2,8.8365E4,2.2216E4,2E0,2.1818182E0,2.707947E8,1.2427474E8,1.612289E6,8.251192E0,2.8145872E-2,1.3298E4,2.0184135E0,-2.9642265E-3,2.714274E-2,1.5669752E-2,6.8651576E9,-2.1017347E-2,2.026496E6,1.00137E5,7.885718E2,5.8426323E0,9E0,4.8284302E-3,2.2392463E-2,8.118227E-3,4E3,9.451483E-1,7.06699E8,1.5733662E0,-1.8288628E-3,9.25E0,6.977611E5,-5.2699177E-3,6.865913E-2,2.091623E5,5.1232258E1,-2.735291E-3,3.8575E4,4.3689142E2,4.9633026E-2,-2.646824E-2,-3.0798973E-3,2.1E2,2.5757682E-1,9.042859E4,2.5220325E6,2.9221216E8,1.4955851E7,-4.656636E-2,2.92E2,2.0407547E5,1E0,1.6695403E0,1.4433751E0,2.4E1,3.3E1,1.3206031E5,1.4689211E7,9.414044E2,1.08504E5,-3.9820068E-2,-1.1479197E-2,9.9668306E-1,-1.5237781E-2,-3.3492766E-2,-5.254895E-2,1.2151E4,2.1196339E1,1.16072E6,-2.0533925E-2,2E1,-7.477755E-3,1.5083523E0,-5.460944E-4,-2.2406413E-3,3.1110662E7,6.3300834E0,5.724838E5,3.1609525E2,-9.146866E-3,3.4806177E-3,2.6573448E9,1.515239E-2,4.4733908E-2,-7.521694E-3,1.5825138E5,1.5476E4,2.5584042E-2,-3.0665223E-2,-1.0739863E-2,1.56E2,1.3781E2,-4.994735E-3,4.267635E-3,3.274181E8,4.898012E-1,5.8E1,1.633077E2,2.3314082E6,1.632095E2,4.616371E8,1.2633843E3,9.27E2,1.2622228E-2,5.9186682E-2,3.3469384E-3,-2.348054E-3,-1.7385287E-2,-9.371661E-4,8.436086E-3,1.5E1,2.15E2,4.5479352E2,2.20524E5,-8.682717E-3,-8.048798E-4,9.6696576E8,1.504779E7,2.851369E2,1.0963991E7,7.221236E-2,2.6364462E7,1.441484E-2,4.0668085E-2,-1.6742596E-2,-4.0221542E-2,-5.6183584E-2,-3.2096755E-2,-1.8167224E-2,-3.695578E-3,-4.214621E-2,-1.2667132E-2,-6.4084925E-2,-3.972503E-2,-3.7833113E-2,-9.346564E-2,-3.4450255E-3,4.578184E-3,-5.555724E-2,-2.2698905E-2,-1.636746E-2,-3.786643E-2,-3.671272E-2,-1.4791743E-2,-2.7980857E-2,-1.9790066E-2,-1.5954314E-2,-6.091029E-3,-1.939386E-2,-3.1538594E-3,1.6163456E-3,-1.0786567E-2,8.5472653E-4,1.528558E-2,-3.4495976E-3,4.285035E-3,-1.7662209E-2,-5.7214545E-3,2.3013377E-3,1.1550643E-2,1.7969858E-2,3.978848E-3,-1.7478567E-2,-5.715159E-3,-2.3246223E-2,-1.0356727E-2,-9.097973E-3,8.627695E-3,1.0117421E-2,-1.8086102E-2,-3.9548695E-2,-1.04021E-2,-6.446701E-3,3.1604744E-2,6.470288E-3,2.2298416E-2,-6.7832284E-3,1.4745652E-2,2.586435E-2,9.566112E-3,3.804318E-2,1.2119047E-2,-5.439874E-4,-1.4847592E-2,6.479319E-3,2.6634289E-2,-1.1321319E-2,5.4329885E-3,1.9107386E-2,3.8935903E-3,1.7151406E-2,2.9389523E-2,1.3119523E-2,-1.6288104E-2,9.681457E-3,2.8658394E-2,3.782751E-2,1.5204314E-2,3.6471706E-2,6.689279E-3,4.5332424E-2,7.628047E-2,5.6464683E-2,3.0726615E-2],"split_indices":[19,109,32,51,50,27,41,41,10,51,51,2,2,27,2,26,18,0,50,41,41,38,53,28,17,3,5,6,47,52,27,2,53,34,2,26,0,51,28,0,7,44,4,52,51,27,9,51,44,37,0,27,0,67,0,51,0,54,1,2,6,55,7,44,31,56,0,9,52,0,0,37,5,0,9,28,51,52,3,0,0,0,2,40,7,55,0,57,44,0,0,27,55,0,9,51,0,0,0,10,26,27,49,30,46,0,0,27,109,53,41,10,3,31,1,51,7,0,0,38,0,0,0,9,53,1,0,3,0,41,0,0,31,52,47,51,0,0,12,0,0,0,27,9,0,0,0,2,51,0,0,7,26,3,55,27,4,7,4,2,0,41,0,0,0,0,0,3,10,51,9,0,0,7,44,57,46,0,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.237E3,7.07E2,1.53E3,3.84E2,3.23E2,9.09E2,6.21E2,3.67E2,1.7E1,2.26E2,9.7E1,7.73E2,1.36E2,9.1E1,5.3E2,2.3E2,1.37E2,5E0,1.2E1,1.68E2,5.8E1,4.7E1,5E1,1.72E2,6.01E2,7.5E1,6.1E1,3.2E1,5.9E1,1.65E2,3.65E2,1.21E2,1.09E2,2.2E1,1.15E2,7E0,5E0,1.31E2,3.7E1,4E0,5.4E1,3.8E1,9E0,1.4E1,3.6E1,5.2E1,1.2E2,1.4E1,5.87E2,7.3E1,2E0,5.8E1,3E0,3E1,2E0,3.7E1,2.2E1,1.52E2,1.3E1,2.73E2,9.2E1,7.6E1,4.5E1,9.6E1,1.3E1,2E1,2E0,4.1E1,7.4E1,2E0,5E0,4.9E1,8.2E1,1.1E1,2.6E1,1.7E1,3.7E1,2.4E1,1.4E1,2E0,7E0,2E0,1.2E1,1E1,2.6E1,5E1,2E0,1.1E2,1E1,3E0,1.1E1,5.32E2,5.5E1,3E0,7E1,4.1E1,1.7E1,2.8E1,2E0,9E0,2.8E1,1.6E1,6E0,3.7E1,1.15E2,3E0,1E1,1.56E2,1.17E2,8.2E1,1E1,3.2E1,4.4E1,1.5E1,3E1,8.1E1,1.5E1,8E0,5E0,1.7E1,3E0,1.2E1,2.9E1,2.4E1,5E1,3.6E1,1.3E1,7.6E1,6E0,2E1,6E0,4E0,1.3E1,2.6E1,1.1E1,1.9E1,5E0,3E0,1.1E1,2E0,1E1,2E0,8E0,1.1E1,1.5E1,4.6E1,4E0,5.6E1,5.4E1,5E0,5E0,2.73E2,2.59E2,3.1E1,2.4E1,2.7E1,4.3E1,2E1,2.1E1,4E0,5E0,2.6E1,2E0,5E0,1.1E1,2E0,4E0,1.3E1,2.4E1,1.07E2,8E0,6E0,4E0,7.3E1,8.3E1,4.1E1,7.6E1,3.3E1,4.9E1,5E0,5E0,4E0,2.8E1,3.9E1,5E0,9E0,6E0,2.8E1,2E0,6.8E1,1.3E1,2E0,1.3E1,1.2E1,5E0,1E1,1.4E1,4.1E1,9E0,3.3E1,3E0,3E1,4.6E1,1E1,1E1,1.1E1,2E0,1.1E1,1.5E1,6E0,5E0,4E0,1.5E1,8E0,3E0,4E0,4E0,7E0,4E0,2.2E1,3.4E1,3.7E1,1.7E1,2E2,7.3E1,2.47E2,1.2E1,2.8E1,3E0,2.2E1,2E0,5E0,2.2E1,5E0,3.8E1,1.7E1,3E0,1.9E1,2E0,2E0,2E0,2E0,2.4E1,9E0,4E0,9E0,1.5E1,5E1,5.7E1,5E0,3E0,5.2E1,2.1E1,7.1E1,1.2E1,3E1,1.1E1,6.3E1,1.3E1,4.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"261","size_leaf_vector":"1"}},{"base_weights":[-1.5764932E-4,-5.492678E-1,2.5056276E-1,-8.5224175E-1,-2.0532426E-1,4.447403E-2,7.1461064E-1,-1.0460614E0,-3.4033525E-1,-4.0811303E-1,3.0276525E-1,-9.456146E-2,3.616319E-1,6.1010647E-1,1.2751042E0,-9.3619174E-1,-1.5112791E0,1.2193446E0,-4.8604572E-1,-5.260646E-1,-2.1268463E-1,1.3870971E-1,7.2018874E-1,-5.765492E-1,-3.0798234E-2,4.0557244E-1,-7.038583E-1,4.4451123E-1,8.965001E-1,1.3360705E0,1.7285446E-2,-1.0442014E0,-4.164503E-1,-1.759377E0,-9.8625606E-1,8.619932E-2,2.908705E-1,-1.08363286E-1,-6.139888E-1,-6.5855235E-1,-3.725118E-1,-7.992044E-2,-3.5285282E-1,-7.172232E-2,2.7068484E-1,5.4611217E-2,5.7698685E-1,-6.141491E-1,1.5320052E-2,-4.7550127E-2,1.5004168E0,2.150421E-1,5.8388877E-1,-8.032924E-3,-7.8344274E-1,5.4396677E-1,-3.5744708E-2,7.1063554E-1,1.3122169E0,1.3763549E0,8.097304E-4,-1.0653807E0,-1.6823705E-2,3.4030102E-2,-5.729618E-1,-1.8491243E0,-3.0282142E-2,-4.924063E-2,-5.2935346E-3,-2.1047236E-3,2.4052816E-2,1.411696E-1,-1.7780498E-1,-7.730171E-1,-3.4783855E-1,-7.529195E-1,-4.7964987E-1,-9.450688E-2,-4.0742216E-1,-1.3740219E-1,1.4650229E-1,-4.8758858E-1,-3.8929548E-2,-2.4975982E-1,1.1143821E-1,1.17076576E-1,4.18718E-1,1.5722428E-2,3.347897E-2,-5.275127E-1,-8.406874E-1,-1.2798654E-1,2.1047673E-1,2.878268E-2,7.7832095E-2,1.3653415E-1,4.0107182E-1,5.1800644E-1,9.6761566E-1,-3.9313294E-2,-1.2115123E-2,3.6168742E-1,7.26274E-1,-4.3310046E-1,3.8129744E-1,4.6657786E-1,8.2752186E-1,1.1174519E0,1.6757166E0,1.407596E0,3.2594666E-2,-8.3645916E-1,-1.12794E0,-5.2858014E-2,1.2828423E-2,-1.9811556E-1,-6.6213405E-1,-1.9197241E0,-2.7044477E-2,4.9422606E-4,1.2123642E-2,-1.2351602E-2,-3.407514E-3,-6.4537483E-1,-6.0543355E-2,-2.5872365E-1,-4.5465074E-2,-3.0784521E-2,-4.3561026E-2,-5.0512064E-1,-4.098644E-3,-1.0228383E-4,-1.2597436E-2,-4.6563944E-1,-2.54224E-1,-1.8526846E-1,9.7596515E-3,-4.0636426E-3,9.814788E-3,-5.310074E-1,-7.4381926E-3,-1.8011037E-1,1.1373631E-2,-1.8587162E-3,-3.029471E-1,2.5786257E-1,-5.7394407E-3,1.8885124E-1,-7.4578663E-3,4.8724574E-1,4.6170405E-3,-6.38358E-1,-3.6018148E-1,-4.61792E-2,-2.2106266E-2,-4.2148584E-1,-8.097362E-2,3.1903076E-1,-6.322479E-2,-7.415931E-2,2.53643E-1,5.036105E-1,2.5284487E-1,6.346204E-1,3.709579E-1,1.3213231E-2,1.0205996E0,4.5663957E-2,2.9858506E-1,8.314431E-1,4.779929E-1,-3.179021E-2,-1.7210701E-1,5.149883E-1,-6.1042976E-2,4.257335E-2,5.5283487E-1,4.3227665E-2,5.2377576E-1,1.1707463E0,2.1053685E-2,3.8273387E-2,8.331684E-2,5.16931E-2,6.9651015E-2,-4.056676E-2,-1.5376485E-2,-5.864159E-2,-4.7177557E-2,3.2873359E-3,-7.5948997E-3,-1.5847793E-2,-3.0485876E-3,-3.164433E-2,-1.4148534E-2,-4.373541E-2,-9.048844E-2,-3.012315E-2,-8.926821E-3,9.755025E-4,-1.6439658E-2,-2.5146557E-2,-1.431582E-2,-2.4643099E-2,-1.656133E-2,-1.4077437E-2,-1.5046407E-3,-1.1288033E-2,-1.2975992E-3,-6.053867E-3,-2.6123712E-2,4.661254E-3,-1.1940855E-2,-1.7336236E-2,-5.903615E-3,1.576138E-2,2.625462E-3,1.0077667E-2,-2.4473118E-3,2.5130175E-2,9.663978E-3,-2.995469E-2,-7.3253415E-3,-1.9108012E-2,-3.3180583E-3,-2.245721E-2,5.162155E-3,-1.2844704E-2,-1.7450447E-4,1.6957108E-2,-2.4368542E-3,-5.918943E-3,1.4138946E-2,3.0515727E-4,-2.8324483E-2,-7.7030645E-3,1.2472246E-2,1.8303322E-2,3.399075E-2,-1.9867294E-3,1.3934861E-2,2.2224613E-2,3.4814134E-2,9.482934E-3,2.2373226E-2,2.3553908E-2,5.0397623E-2,1.1283892E-2,2.8589796E-2,3.2459784E-2,5.219673E-2,2.4465282E-2,-4.9544587E-3,-1.8959453E-2,-2.5457726E-3,3.0438215E-2,5.1843096E-3,5.630847E-3,-1.3035109E-2,1.5907058E-2,-1.2533997E-2,2.9195659E-2,1.4192832E-2,1.3914151E-2,2.735877E-2,3.1615682E-2,5.740049E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,-1,87,89,-1,91,93,95,97,-1,99,101,103,105,107,109,-1,111,-1,113,115,117,-1,-1,-1,-1,-1,119,121,123,125,127,129,131,133,135,137,139,141,143,145,147,149,-1,-1,151,153,155,157,-1,-1,159,161,163,165,-1,-1,167,169,171,173,175,177,179,181,183,-1,185,187,189,-1,191,193,195,-1,-1,-1,-1,-1,197,-1,199,-1,-1,-1,201,-1,-1,-1,203,205,207,-1,-1,-1,209,-1,211,-1,-1,213,215,-1,217,-1,219,-1,221,223,-1,-1,225,227,229,231,233,235,237,239,241,243,-1,245,-1,247,249,251,-1,253,255,257,259,261,-1,263,265,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0260547E2,7.1622894E1,1.442815E2,3.595761E1,3.350685E1,4.6158825E1,2.640036E1,1.2130188E1,2.377335E1,5.2359657E0,6.2688017E0,2.2355934E1,1.522047E1,1.8239685E1,3.4111252E0,1.1746902E1,4.6474075E0,5.1669416E0,4.4757214E0,2.6842003E0,1.6280289E0,1.9078959E0,1.1137037E0,3.1319408E0,1.6590427E1,1.0304665E1,4.056363E-1,1.200824E1,1.0116676E1,3.5332413E0,0E0,2.154007E0,2.769207E0,1.8950424E0,1.4589005E0,0E0,4.3020767E-1,4.583003E-1,2.6938953E0,9.2193604E-1,6.311598E-1,6.31843E-1,1.7997766E0,9.082878E-1,9.060209E-1,0E0,5.643687E-1,1.1658974E0,0E0,1.3268472E1,2.3091316E-1,2.135962E0,3.4931374E0,0E0,2.4938345E-1,6.64933E0,7.45656E0,2.4951248E0,1.1717377E0,3.195572E-1,0E0,1.5341034E0,0E0,2.5327072E-1,8.3817196E-1,1.4161224E0,0E0,0E0,0E0,0E0,0E0,9.6028455E-2,1.7432284E-1,2.2339134E0,1.3780024E0,7.8546524E-2,2.4169588E-1,1.5191951E-1,4.2566872E-1,6.819633E-1,1.855112E-1,3.3387232E-1,6.1873275E-1,1.4296883E-1,5.210053E-1,4.7608966E-1,4.0612245E-1,0E0,0E0,9.2004204E-1,8.289423E-1,6.6856236E0,4.5363045E0,0E0,0E0,2.6589355E0,5.4142046E-1,2.1138458E0,4.8984146E-1,0E0,0E0,4.026039E0,2.292057E0,1.4943614E0,1.3324478E0,1.2565322E0,2.0901794E0,5.334511E-1,2.206726E-1,3.649292E-1,0E0,9.091072E-1,5.6170654E-1,1.4192277E-1,0E0,1.1551118E-1,7.763958E-2,1.0253143E-1,0E0,0E0,0E0,0E0,0E0,2.0011425E-1,0E0,7.6852787E-1,0E0,0E0,0E0,8.099079E-2,0E0,0E0,0E0,1.3862419E-1,2.2780168E-1,3.3394516E-1,0E0,0E0,0E0,3.980751E-1,0E0,2.6390135E-1,0E0,0E0,1.0185766E-1,1.4219522E-1,0E0,1.595627E-1,0E0,1.9719744E-1,0E0,2.9223824E-1,4.4833302E-1,0E0,0E0,2.7311E0,6.581209E0,2.2763042E0,1.1563181E0,1.7441342E0,6.8552876E-1,3.1026506E-1,3.3957994E-1,1.0707226E0,1.1501665E0,0E0,3.4721947E-1,0E0,1.4525871E0,1.7862892E0,1.2685752E0,0E0,3.480271E-1,9.940939E-1,2.882043E-1,7.876824E-1,4.366398E-1,0E0,1.9768906E-1,3.9909363E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,59,59,61,61,63,63,64,64,65,65,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,95,95,96,96,97,97,98,98,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,111,111,112,112,113,113,115,115,116,116,117,117,123,123,125,125,129,129,133,133,134,134,135,135,139,139,141,141,144,144,145,145,147,147,149,149,151,151,152,152,155,155,156,156,157,157,158,158,159,159,160,160,161,161,162,162,163,163,164,164,166,166,168,168,169,169,170,170,172,172,173,173,174,174,175,175,176,176,178,178,179,179],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,-1,88,90,-1,92,94,96,98,-1,100,102,104,106,108,110,-1,112,-1,114,116,118,-1,-1,-1,-1,-1,120,122,124,126,128,130,132,134,136,138,140,142,144,146,148,150,-1,-1,152,154,156,158,-1,-1,160,162,164,166,-1,-1,168,170,172,174,176,178,180,182,184,-1,186,188,190,-1,192,194,196,-1,-1,-1,-1,-1,198,-1,200,-1,-1,-1,202,-1,-1,-1,204,206,208,-1,-1,-1,210,-1,212,-1,-1,214,216,-1,218,-1,220,-1,222,224,-1,-1,226,228,230,232,234,236,238,240,242,244,-1,246,-1,248,250,252,-1,254,256,258,260,262,-1,264,266,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,1.983922E3,4.7931265E6,7.733016E5,3.2196458E6,6.971004E7,6.737509E7,2.728889E2,2.496172E3,8.2E1,4.898012E-1,1.6018981E0,1E0,9.4538574E2,8.901186E0,9.83E2,3.4692544E7,1.16544716E-1,1.4274633E7,1.8708362E-1,4.7777777E0,1.6594656E3,5.528107E3,9.99E2,3.928E4,3.9712732E7,5.0744297E3,9.256843E3,1.7285446E-2,1E0,8.2608955E-3,4.099E3,8.68379E5,8.619932E-2,4.9807886E3,1.4540612E7,3.0497742E3,1.773747E2,1.2738854E0,2.1446484E-1,9.026015E2,2.7271206E5,3.564E3,5.4611217E-2,5.185489E3,1.3597637E8,1.5320052E-2,1.3736341E8,1.2E1,3.321532E6,3.051E3,-8.032924E-3,5.98E2,8.838E3,3.970405E3,1.1598511E3,2.4536058E1,6.292039E0,8.097304E-4,2.6575577E-1,-1.6823705E-2,7.806314E6,1.6837112E7,8.538E3,-3.0282142E-2,-4.924063E-2,-5.2935346E-3,-2.1047236E-3,2.4052816E-2,1.2679E4,3.5330936E1,1.2427474E8,4.33E2,1.0258198E-2,6.708E3,4.5880157E2,2.1150263E-1,6.860185E2,2.4684112E8,4.321761E8,2.386324E6,1E0,5.7E1,5.666993E0,2.0687214E6,1.5722428E-2,3.347897E-2,2.9636364E1,8.874974E7,1.2068E4,5.4849293E-3,2.878268E-2,7.7832095E-2,2.5644116E2,3.4653E4,3.5636363E0,8.695652E-1,-3.9313294E-2,-1.2115123E-2,6E0,2.3907686E7,1.2354571E0,1.2697302E8,1.0013907E1,8.861076E0,9.74026E0,1.3532244E7,5.748E3,3.2594666E-2,2.3871907E2,1.5294166E5,4.923077E0,1.2828423E-2,1.0443182E3,1.5092975E6,5.6039163E9,-2.7044477E-2,4.9422606E-4,1.2123642E-2,-1.2351602E-2,-3.407514E-3,8.454878E1,-6.0543355E-2,8.773955E6,-4.5465074E-2,-3.0784521E-2,-4.3561026E-2,5.511673E0,-4.098644E-3,-1.0228383E-4,-1.2597436E-2,1.85221E5,5.9986052E7,2.674421E9,9.7596515E-3,-4.0636426E-3,9.814788E-3,7.2591E4,-7.4381926E-3,5.1998E4,1.1373631E-2,-1.8587162E-3,2.7E1,2.010856E0,-5.7394407E-3,2.57937E5,-7.4578663E-3,3.1664667E-3,4.6170405E-3,5.2350176E4,3.7832818E0,-4.61792E-2,-2.2106266E-2,1.91E2,5.26E2,3.620103E1,2.93E2,1.4992306E8,5.8631687E1,3.3870136E7,9.232009E2,3.958068E2,6.806472E7,1.3213231E-2,1E0,4.5663957E-2,1.4100353E0,6.9673E4,3.0961128E7,-3.179021E-2,3.0200147E8,2.2418336E5,2.14099E5,3.324E3,6.211896E0,4.3227665E-2,1.8777761E0,4.423676E1,2.1053685E-2,3.8273387E-2,8.331684E-2,5.16931E-2,6.9651015E-2,-4.056676E-2,-1.5376485E-2,-5.864159E-2,-4.7177557E-2,3.2873359E-3,-7.5948997E-3,-1.5847793E-2,-3.0485876E-3,-3.164433E-2,-1.4148534E-2,-4.373541E-2,-9.048844E-2,-3.012315E-2,-8.926821E-3,9.755025E-4,-1.6439658E-2,-2.5146557E-2,-1.431582E-2,-2.4643099E-2,-1.656133E-2,-1.4077437E-2,-1.5046407E-3,-1.1288033E-2,-1.2975992E-3,-6.053867E-3,-2.6123712E-2,4.661254E-3,-1.1940855E-2,-1.7336236E-2,-5.903615E-3,1.576138E-2,2.625462E-3,1.0077667E-2,-2.4473118E-3,2.5130175E-2,9.663978E-3,-2.995469E-2,-7.3253415E-3,-1.9108012E-2,-3.3180583E-3,-2.245721E-2,5.162155E-3,-1.2844704E-2,-1.7450447E-4,1.6957108E-2,-2.4368542E-3,-5.918943E-3,1.4138946E-2,3.0515727E-4,-2.8324483E-2,-7.7030645E-3,1.2472246E-2,1.8303322E-2,3.399075E-2,-1.9867294E-3,1.3934861E-2,2.2224613E-2,3.4814134E-2,9.482934E-3,2.2373226E-2,2.3553908E-2,5.0397623E-2,1.1283892E-2,2.8589796E-2,3.2459784E-2,5.219673E-2,2.4465282E-2,-4.9544587E-3,-1.8959453E-2,-2.5457726E-3,3.0438215E-2,5.1843096E-3,5.630847E-3,-1.3035109E-2,1.5907058E-2,-1.2533997E-2,2.9195659E-2,1.4192832E-2,1.3914151E-2,2.735877E-2,3.1615682E-2,5.740049E-2],"split_indices":[19,109,12,54,50,27,27,44,7,51,4,28,26,40,6,4,55,0,44,38,44,37,53,4,51,2,12,44,54,4,0,58,26,0,46,0,4,44,51,51,52,37,51,27,2,0,51,30,0,7,17,27,2,0,10,2,51,51,53,37,0,26,0,47,44,2,0,0,0,0,0,2,55,44,8,37,0,4,37,51,7,7,1,66,8,52,46,0,0,54,7,9,38,0,0,54,9,53,52,0,0,3,44,52,43,55,52,52,1,2,0,51,32,57,0,4,27,5,0,0,0,0,0,55,0,1,0,0,0,52,0,0,0,9,5,5,0,0,0,28,0,12,0,0,3,56,0,28,0,26,0,27,52,0,0,3,2,57,3,5,46,7,32,51,7,0,109,0,37,2,42,0,7,27,2,28,53,0,40,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.196E3,6.88E2,1.508E3,3.65E2,3.23E2,1.045E3,4.63E2,2.64E2,1.01E2,2.31E2,9.2E1,7.27E2,3.18E2,3.92E2,7.1E1,2.16E2,4.8E1,8E0,9.3E1,1.43E2,8.8E1,6.7E1,2.5E1,8.4E1,6.43E2,3.06E2,1.2E1,2.5E2,1.42E2,6.6E1,5E0,1.78E2,3.8E1,3.1E1,1.7E1,4E0,4E0,2.4E1,6.9E1,7.5E1,6.8E1,4.6E1,4.2E1,2.6E1,4.1E1,4E0,2.1E1,8.1E1,3E0,6.37E2,6E0,1.49E2,1.57E2,2E0,1E1,2.07E2,4.3E1,1E2,4.2E1,6.4E1,2E0,1.72E2,6E0,1E1,2.8E1,2.8E1,3E0,1.5E1,2E0,2E0,2E0,5E0,1.9E1,4.2E1,2.7E1,4.7E1,2.8E1,8E0,6E1,3.7E1,9E0,2.9E1,1.3E1,1.3E1,1.3E1,2.1E1,2E1,1E1,1.1E1,6.1E1,2E1,4.86E2,1.51E2,2E0,4E0,1.06E2,4.3E1,1.36E2,2.1E1,8E0,2E0,1.05E2,1.02E2,2.2E1,2.1E1,3.4E1,6.6E1,3E1,1.2E1,6E1,4E0,4E1,1.32E2,8E0,2E0,6E0,2.2E1,2.6E1,2E0,3E0,2E0,9E0,1E1,3.6E1,6E0,2.5E1,2E0,3.8E1,9E0,2.6E1,2E0,6E0,2E0,4.2E1,1.8E1,3.3E1,4E0,2E0,7E0,2.5E1,4E0,9E0,4E0,3E0,1E1,8E0,5E0,1.7E1,4E0,1.6E1,4E0,3.5E1,2.6E1,1.2E1,8E0,6.6E1,4.2E2,1.08E2,4.3E1,3.8E1,6.8E1,2.4E1,1.9E1,7.4E1,6.2E1,2E0,1.9E1,8E0,9.7E1,7E1,3.2E1,1E1,1.2E1,1.6E1,5E0,6E0,2.8E1,4.5E1,2.1E1,2.7E1,3E0,3E0,9E0,2.3E1,3.7E1,3.5E1,5E0,4E1,9.2E1,4E0,4E0,2E0,4E0,1.9E1,3E0,3E0,2.3E1,3.4E1,2E0,7E0,1.8E1,1.9E1,7E0,2.1E1,2.1E1,1.4E1,4E0,2.3E1,1E1,3E0,2.2E1,2E0,7E0,6E0,4E0,5E0,3E0,1.5E1,2E0,1.2E1,4E0,3.3E1,2E0,2.1E1,5E0,5.8E1,8E0,1.15E2,3.05E2,9.4E1,1.4E1,3.7E1,6E0,3.4E1,4E0,3E0,6.5E1,1.9E1,5E0,3E0,1.6E1,3.8E1,3.6E1,2.8E1,3.4E1,4E0,1.5E1,8.6E1,1.1E1,5.4E1,1.6E1,2.9E1,3E0,3E0,9E0,1.1E1,5E0,3E0,2E0,3E0,3E0,1.9E1,9E0,7E0,1.4E1,6E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"267","size_leaf_vector":"1"}},{"base_weights":[2.4286276E-3,-5.1253885E-1,2.4983011E-1,-8.3475083E-1,-2.1044092E-1,1.481631E-2,5.8973676E-1,-9.848725E-1,-4.6009353E-1,-3.6180207E-1,3.7053424E-1,-3.85723E-1,1.042315E-1,-2.8115377E-2,6.8600285E-1,-9.2634004E-1,-1.6446683E0,-5.621533E-1,-1.7028722E-1,-3.9633742E-1,7.883265E-2,1.7117198E-1,6.672715E-1,-5.550774E-1,-2.7626705E-1,1.5753725E-1,-5.046786E-1,-4.116353E-1,3.13416E-1,3.6249304E-1,8.35208E-1,-9.978914E-1,-4.325282E-1,-8.481675E-2,-1.44210225E-2,-5.989222E-1,-1.7934836E-1,-2.3656517E-1,1.1215945E-1,-5.374592E-1,-1.87232E-1,2.0423448E-2,3.48401E-1,9.047523E-1,3.8808623E-1,-5.803477E-1,-9.882596E-3,-3.1813007E-2,-3.152388E-1,2.6329336E-1,-7.964606E-2,-8.13396E-1,-8.156053E-2,-4.9529165E-1,-1.5098523E-2,4.6031967E-1,-2.4677876E-1,4.3868664E-1,-2.661343E-1,6.9154423E-1,1.1693411E0,-6.862197E-1,-1.0783703E0,-7.5816244E-2,-6.2492895E-1,-6.5767384E-1,-3.2533213E-1,5.5857335E-4,-1.3438553E-2,-9.439681E-2,-1.5332663E-2,-3.5014944E-4,1.0622668E-2,-5.638186E-1,3.3066265E-2,-1.3175683E-1,-5.1559585E-1,-9.680141E-2,1.7502129E-1,5.9117537E-2,4.5318747E-1,1.8078387E-2,4.4851832E-2,-3.9803893E-3,4.6236664E-1,-5.948299E-1,-5.277344E-3,-1.22194275E-1,7.849977E-3,-3.5569766E-1,-1.403856E-1,1.9580695E-1,9.564259E-1,-3.0183944E-1,9.2381425E-2,-8.688636E-1,-8.157148E-3,5.5632282E-2,-3.848417E-2,-5.225999E-1,-2.173569E-3,4.846603E-3,-7.4253934E-3,3.6385143E-1,3.6581125E-2,1.3725706E-3,-1.6745253E-2,5.2485865E-2,5.179623E-1,-4.4689104E-2,-6.926395E-2,5.031277E-1,9.126997E-1,4.520511E-1,1.2145785E0,-1.4649249E-2,-7.349653E-1,-1.1377196E0,-8.3729357E-1,-8.349602E-3,8.489101E-3,-3.020412E-2,-7.395295E-3,-5.527494E-1,-4.077473E-2,-1.707012E-2,-4.5517852E-4,1.9394943E-3,-1.7893718E-1,-8.262561E-1,-3.756139E-1,-2.3813152E-1,1.3004024E-2,-1.8589555E-1,-6.785328E-1,-1.3284742E-2,-2.8853992E-2,-1.5770356E-3,9.981797E-3,6.881029E-3,-2.197181E-3,1.1200352E-2,2.4987891E-2,6.2445104E-3,5.0494593E-1,-9.706443E-3,-6.123189E-1,8.858775E-3,-9.714523E-3,-1.386481E-1,-3.814678E-1,-6.128662E-2,-1.8196069E-2,7.197764E-2,3.904957E-1,1.5119487E0,5.777801E-1,-7.033783E-1,-2.000889E-1,1.9478826E-2,4.963137E-1,-4.1443136E-2,-2.0095745E-2,-1.0719368E-1,6.632445E-1,-1.6362198E-2,-2.9479362E-2,4.0164274E-1,-4.98698E-3,1.4723408E-1,-1.9094542E-1,3.579307E-2,4.4606405E-1,-8.226424E-3,1.7375285E-3,3.8625327E-1,6.8989545E-1,1.0804882E0,5.7532376E-1,2.8269617E-3,3.0484252E-2,1.2571094E0,6.4013904E-1,-3.7473258E-2,-1.956277E-2,-4.7804113E-2,-5.9564378E-2,-4.026136E-2,-2.274086E-2,-2.6232269E-2,-7.301236E-3,-1.1944931E-2,1.0054356E-3,-1.5979717E-2,-4.2390585E-2,-2.1573909E-2,-5.322271E-3,-4.936804E-3,-1.403481E-2,-1.3250088E-3,2.0535957E-2,-1.6364865E-2,-1.9637977E-3,-3.637387E-2,-1.1731867E-2,4.0977504E-3,-3.3373344E-3,7.5369044E-3,2.5172004E-2,-2.827644E-2,-9.0575E-3,-1.0936613E-3,-1.2483088E-2,-1.95821E-2,-1.2177646E-2,-8.00867E-3,6.431711E-4,2.7273377E-2,1.8899054E-3,1.1451347E-2,2.8971309E-2,7.9663955E-2,2.7852543E-2,7.0021646E-3,3.4283254E-2,-5.679635E-2,-2.0658795E-2,1.1043712E-2,-1.1662814E-2,-2.238649E-4,1.5762484E-2,4.9220356E-3,2.5651554E-2,-1.0165459E-2,9.4460504E-4,1.15421545E-2,3.8504004E-2,1.9967427E-2,5.9864856E-3,8.843526E-3,-4.2556454E-3,-1.2526671E-2,2.0004017E-3,2.1259474E-2,-4.706013E-3,7.588175E-3,2.0432675E-2,3.5046183E-2,1.6809536E-2,4.0198326E-2,6.3576624E-2,2.9229015E-2,-2.7078497E-3,6.335294E-2,4.895846E-2,3.3168066E-2,9.552715E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,-1,65,67,69,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,125,-1,-1,127,-1,-1,-1,129,-1,131,133,135,137,139,141,-1,-1,-1,143,145,-1,147,-1,149,151,153,155,157,159,161,-1,163,-1,165,-1,-1,-1,167,-1,-1,-1,169,171,-1,173,175,177,179,181,-1,183,185,187,-1,-1,-1,-1,189,-1,-1,-1,-1,191,193,195,197,199,201,203,-1,205,-1,-1,-1,-1,-1,-1,-1,207,-1,209,-1,-1,211,213,215,-1,217,219,221,223,225,227,229,231,-1,-1,233,235,-1,-1,237,-1,239,241,-1,243,-1,-1,245,247,249,251,-1,-1,253,255,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8423834E2,7.02982E1,1.20374565E2,1.92323E1,3.3110462E1,3.1986559E1,3.6657898E1,8.092728E0,2.9211845E0,2.2333607E1,4.500123E0,2.8681564E0,2.3774147E1,1.1131678E1,2.5424774E1,7.7302856E0,5.166832E0,9.4823456E-1,5.506926E-1,8.575119E0,0E0,1.2753012E0,1.7042141E0,4.0950394E-1,9.521775E-1,1.6877138E1,7.6404676E0,1.3194332E0,3.8168564E0,8.259304E0,1.6664383E1,4.3016663E0,2.0802011E0,0E0,0E0,8.7905693E-1,1.83815E-1,3.018818E-1,8.9915805E-2,6.2600975E0,2.1179914E0,5.062555E-1,6.400943E-1,2.9928493E-1,6.0637474E-1,3.2022667E-1,0E0,2.993825E-1,5.655346E-1,2.1564045E1,7.9783444E0,1.0376472E0,2.7884266E0,3.7798977E-1,1.6307294E-1,9.134369E-1,3.522802E-1,4.6264954E0,2.6283622E0,1.0249817E1,2.9553528E0,5.400028E-1,1.3080292E0,3.7776542E-1,2.8071642E-1,9.042015E-1,2.2444534E-1,0E0,0E0,1.3528724E-1,0E0,0E0,0E0,8.18507E0,0E0,1.6045812E0,7.871094E-1,2.0530002E-1,1.2214512E-1,7.7356346E-2,1.8858147E-1,0E0,0E0,0E0,1.2375593E-1,2.4131584E-1,0E0,3.649854E-1,0E0,3.3864784E-1,3.4772244E-1,1.0217865E1,7.7506294E0,3.5772305E0,3.4651628E0,2.0661163E-1,0E0,2.3466015E0,0E0,4.069662E-1,0E0,0E0,0E0,5.8163667E-1,0E0,0E0,0E0,6.5086E-1,2.1171188E0,0E0,2.0062463E-1,2.811325E0,6.00148E0,6.4765215E-1,1.6628723E0,0E0,7.096348E-1,4.1630554E-1,1.628933E-1,0E0,0E0,0E0,0E0,3.3451176E-1,0E0,0E0,0E0,0E0,1.253991E-1,3.3679085E0,2.5999908E0,5.228214E-1,8.5135174E-1,1.5947406E-1,3.679552E-1,0E0,7.628302E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.14778996E-1,0E0,1.4258194E-1,0E0,0E0,1.3458522E-1,2.3975372E-1,1.3342541E-1,0E0,4.160498E0,5.499275E0,2.4707642E0,1.8571367E0,1.8785801E0,1.9526837E0,8.1249833E-1,4.2866993E-1,0E0,0E0,2.9018104E-1,1.654005E-1,0E0,0E0,2.3884773E-1,0E0,2.5042385E-1,1.735067E-1,0E0,1.3677444E0,0E0,0E0,1.1903763E0,1.1343594E0,3.554779E0,1.8687296E0,0E0,0E0,8.109436E-1,2.0256162E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,73,73,75,75,76,76,77,77,78,78,79,79,80,80,84,84,85,85,87,87,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,99,99,103,103,107,107,108,108,110,110,111,111,112,112,113,113,114,114,116,116,117,117,118,118,123,123,128,128,129,129,130,130,131,131,132,132,133,133,134,134,136,136,144,144,146,146,149,149,150,150,151,151,153,153,154,154,155,155,156,156,157,157,158,158,159,159,160,160,163,163,164,164,167,167,169,169,170,170,172,172,175,175,176,176,177,177,178,178,181,181,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,-1,66,68,70,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,126,-1,-1,128,-1,-1,-1,130,-1,132,134,136,138,140,142,-1,-1,-1,144,146,-1,148,-1,150,152,154,156,158,160,162,-1,164,-1,166,-1,-1,-1,168,-1,-1,-1,170,172,-1,174,176,178,180,182,-1,184,186,188,-1,-1,-1,-1,190,-1,-1,-1,-1,192,194,196,198,200,202,204,-1,206,-1,-1,-1,-1,-1,-1,-1,208,-1,210,-1,-1,212,214,216,-1,218,220,222,224,226,228,230,232,-1,-1,234,236,-1,-1,238,-1,240,242,-1,244,-1,-1,246,248,250,252,-1,-1,254,256,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.4166665E0,7.7344055E4,1E0,1.6990049E0,3.97E2,2.0261088E-1,1.0017953E8,1.4887473E-1,1.3144558E4,1.1925198E3,4.7E2,4.0844156E7,5.704748E4,4.052E3,3.773E3,6.520433E6,9.705292E6,2.3871907E2,4.8046228E-2,7.883265E-2,2.1212247E5,8.7643677E-1,5.2350176E4,1.2091255E0,1.062363E1,3.1998687E3,6.131E3,3.2570488E6,5.884028E0,1.2727361E6,2.0304577E-1,2.1693E5,-8.481675E-2,-1.44210225E-2,1.998E3,4.56E2,2.3067484E0,2.695E3,1.2424786E2,8.229907E7,3.6482175E5,2.091623E5,4.7233105E6,1.4E1,5.222222E1,-9.882596E-3,1E0,3.30399E6,8.5023944E2,7.44393E5,7.2E1,1.4563726E-1,1E0,6.47E2,9.075E3,3.328615E6,8.15E2,1.34E3,2.4848E4,1E0,1.7515824E8,2.1515152E0,6.071E3,1.2204E4,1.85369E5,7.6E1,5.5857335E-4,-1.3438553E-2,1.1844802E0,-1.5332663E-2,-3.5014944E-4,1.0622668E-2,1E0,3.3066265E-2,2.852738E6,1.272E3,2.41E3,6.254696E2,9.619E3,3.333E3,1.8078387E-2,4.4851832E-2,-3.9803893E-3,1.7612903E0,7E0,-5.277344E-3,1.1E1,7.849977E-3,2.905E1,1.3E1,1.208686E6,1.109125E3,9.5E1,1.4777102E9,2.9049585E3,-8.157148E-3,1.3664E4,-3.848417E-2,7.218466E7,-2.173569E-3,4.846603E-3,-7.4253934E-3,1E0,3.6581125E-2,1.3725706E-3,-1.6745253E-2,4.12E0,7E0,-4.4689104E-2,2.857E3,2.6100147E5,2.8091298E7,4E0,2E0,-1.4649249E-2,5.314225E2,2.01351E5,5.8426323E0,-8.349602E-3,8.489101E-3,-3.020412E-2,-7.395295E-3,2.3871907E2,-4.077473E-2,-1.707012E-2,-4.5517852E-4,1.9394943E-3,1.4000518E6,1.857051E5,5.4878284E2,1.9711539E0,9.557264E5,1.4101E4,1.4563726E-1,-1.3284742E-2,9.095E3,-1.5770356E-3,9.981797E-3,6.881029E-3,-2.197181E-3,1.1200352E-2,2.4987891E-2,6.2445104E-3,3.164136E6,-9.706443E-3,1.7E1,8.858775E-3,-9.714523E-3,1.7E1,1.6968552E2,2.08E2,-1.8196069E-2,5.486E1,1.172E3,3.64299E5,1.9563605E5,9.99E2,2.1E1,7.315484E9,7.2E1,-4.1443136E-2,-2.0095745E-2,1.17E2,4.38E2,-1.6362198E-2,-2.9479362E-2,7.128E3,-4.98698E-3,4.29E2,1.91E2,3.579307E-2,2.3279E4,-8.226424E-3,1.7375285E-3,1.0989723E3,1.4274633E7,6.7652373E3,3.0272608E10,2.8269617E-3,3.0484252E-2,1.9523809E0,2.8411612E6,-3.7473258E-2,-1.956277E-2,-4.7804113E-2,-5.9564378E-2,-4.026136E-2,-2.274086E-2,-2.6232269E-2,-7.301236E-3,-1.1944931E-2,1.0054356E-3,-1.5979717E-2,-4.2390585E-2,-2.1573909E-2,-5.322271E-3,-4.936804E-3,-1.403481E-2,-1.3250088E-3,2.0535957E-2,-1.6364865E-2,-1.9637977E-3,-3.637387E-2,-1.1731867E-2,4.0977504E-3,-3.3373344E-3,7.5369044E-3,2.5172004E-2,-2.827644E-2,-9.0575E-3,-1.0936613E-3,-1.2483088E-2,-1.95821E-2,-1.2177646E-2,-8.00867E-3,6.431711E-4,2.7273377E-2,1.8899054E-3,1.1451347E-2,2.8971309E-2,7.9663955E-2,2.7852543E-2,7.0021646E-3,3.4283254E-2,-5.679635E-2,-2.0658795E-2,1.1043712E-2,-1.1662814E-2,-2.238649E-4,1.5762484E-2,4.9220356E-3,2.5651554E-2,-1.0165459E-2,9.4460504E-4,1.15421545E-2,3.8504004E-2,1.9967427E-2,5.9864856E-3,8.843526E-3,-4.2556454E-3,-1.2526671E-2,2.0004017E-3,2.1259474E-2,-4.706013E-3,7.588175E-3,2.0432675E-2,3.5046183E-2,1.6809536E-2,4.0198326E-2,6.3576624E-2,2.9229015E-2,-2.7078497E-3,6.335294E-2,4.895846E-2,3.3168066E-2,9.552715E-3],"split_indices":[19,55,32,109,40,2,41,44,37,51,51,1,44,27,2,2,1,9,51,37,0,32,56,27,52,53,4,2,49,52,27,26,28,0,0,10,0,57,2,55,44,27,27,44,3,4,0,8,9,51,1,3,37,67,0,2,47,2,2,2,107,30,52,2,2,1,8,0,0,38,0,0,0,109,0,1,0,2,4,10,2,0,0,0,56,3,0,10,0,51,8,27,32,0,7,4,0,2,0,44,0,0,0,79,0,0,0,53,3,0,2,27,44,8,6,0,4,28,52,0,0,0,0,51,0,0,0,0,32,27,51,52,27,2,37,0,9,0,0,0,0,0,0,0,1,0,0,0,0,0,4,2,0,49,2,9,27,2,3,5,8,0,0,3,0,0,0,10,0,10,10,0,28,0,0,51,44,51,5,0,0,53,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.229E3,7.23E2,1.506E3,3.49E2,3.74E2,8.91E2,6.15E2,2.48E2,1.01E2,2.97E2,7.7E1,1.62E2,7.29E2,8.3E1,5.32E2,2.3E2,1.8E1,7.4E1,2.7E1,2.93E2,4E0,4.7E1,3E1,6.2E1,1E2,6.71E2,5.8E1,3.9E1,4.4E1,1.69E2,3.63E2,2E2,3E1,1.5E1,3E0,6.7E1,7E0,2.2E1,5E0,1.74E2,1.19E2,2.6E1,2.1E1,1.5E1,1.5E1,5.7E1,5E0,1.4E1,8.6E1,4.64E2,2.07E2,3.3E1,2.5E1,3.2E1,7E0,3.5E1,9E0,1.51E2,1.8E1,2.56E2,1.07E2,4.3E1,1.57E2,1.1E1,1.9E1,5.4E1,1.3E1,3E0,4E0,1E1,1.2E1,3E0,2E0,1.71E2,3E0,1.03E2,1.6E1,1.5E1,1.1E1,6E0,1.5E1,3E0,1.2E1,2E0,1.3E1,5.5E1,2E0,1E1,4E0,6.9E1,1.7E1,4.24E2,4E1,9E1,1.17E2,3E1,3E0,2.2E1,3E0,3E1,2E0,4E0,3E0,2.9E1,6E0,3E0,6E0,2.6E1,1.25E2,3E0,1.5E1,1.4E2,1.16E2,7E0,1E2,6E0,3.7E1,1.23E2,3.4E1,8E0,3E0,1.7E1,2E0,4E1,1.4E1,1.1E1,2E0,4E0,6E0,7E1,1.01E2,5.9E1,4.4E1,6E0,1E1,3E0,1.2E1,2E0,9E0,3E0,3E0,6E0,9E0,2E0,1.1E1,3E0,5.2E1,2E0,8E0,8E0,6.1E1,1.4E1,3E0,2.6E2,1.64E2,1.5E1,2.5E1,1.7E1,7.3E1,1E2,1.7E1,2.6E1,4E0,1.8E1,4E0,1.5E1,1.5E1,2.7E1,2E0,1.9E1,7E0,2.4E1,1.01E2,7E0,8E0,8.8E1,5.2E1,7.6E1,4E1,3E0,4E0,9.2E1,8E0,2.7E1,1E1,8.9E1,3.4E1,2.8E1,6E0,3.7E1,3E0,4E0,2E0,1.4E1,5.6E1,7.2E1,2.9E1,2.2E1,3.7E1,4.1E1,3E0,2E0,4E0,7E0,3E0,3E0,9E0,2E0,9E0,5E1,2E0,5E0,3E0,4E1,2.1E1,5E0,9E0,1.3E1,2.47E2,1.07E2,5.7E1,1.1E1,4E0,8E0,1.7E1,4E0,1.3E1,8E0,6.5E1,9.4E1,6E0,3E0,1.4E1,9E0,9E0,2E0,2E0,2.3E1,4E0,1.6E1,3E0,5E0,2E0,9.7E1,4E0,2.1E1,6.7E1,4E1,1.2E1,5E1,2.6E1,3.6E1,4E0,4.7E1,4.5E1,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"257","size_leaf_vector":"1"}},{"base_weights":[2.1324726E-2,-4.7183612E-1,2.469611E-1,-7.825479E-1,-1.6776597E-1,3.1492505E-2,5.94569E-1,-9.252218E-1,-4.5305964E-1,-3.1970337E-1,3.750731E-1,-2.0724477E-1,2.0825967E-1,-2.2094823E-2,6.7019534E-1,-7.9958206E-1,-1.1911638E0,-2.9286924E-1,-6.4451015E-1,-3.6592683E-1,8.176172E-2,1.1184257E-1,5.91769E-1,-3.5287496E-1,8.41734E-3,2.5093934E-1,-4.9510065E-1,-4.4897166E-1,1.547042E-1,4.2717597E-1,8.1020606E-1,-9.277102E-1,-4.053119E-1,-1.3486444E0,-4.818301E-1,-4.0390447E-1,-1.0151387E-1,-7.341502E-1,-4.2494822E-1,-2.659323E-1,-7.780977E-1,-2.681032E-2,3.316407E-1,4.575002E-1,8.893159E-1,-6.195118E-1,-2.5065374E-1,1.1250453E0,-7.298217E-2,6.9373034E-2,4.3512276E-1,-6.474612E-1,2.4764505E-1,-3.4827094E-3,-5.139226E-1,3.275224E-1,-2.2428821E-1,1.9412011E-1,6.100305E-1,5.287902E-1,9.811851E-1,-1.0951205E-4,-9.429843E-1,-5.501013E-1,-7.644026E-2,-9.683887E-1,-1.5285671E0,-5.556222E-1,-6.3381754E-3,2.3030257E-3,-4.313449E-1,-2.2020125E-1,1.4855012E-1,-3.6159147E-2,-1.7639177E-2,-2.3646409E-2,-7.952474E-3,-3.2092077E-1,3.6287963E-1,-8.957004E-1,-3.1144696E-1,2.323496E-1,-1.3194615E-1,1.5944161E-3,4.0928197E-1,6.025149E-3,5.147698E-1,4.4022363E-2,1.4310162E-2,-4.4482523E-1,-7.478753E-1,-1.9326402E-2,-3.4990743E-1,8.145203E-3,6.596336E-2,-1.20263964E-1,2.7423227E-2,-3.3083465E-2,2.7955878E-1,3.9098322E-1,1.2816534E0,-1.934345E-2,-3.3453338E-2,-3.019639E-3,2.6449924E-2,-1.9614517E-3,-5.7805634E-1,1.9253907E-1,5.677142E-1,-1.2140454E-2,1.8365661E-3,2.3096026E-1,-4.0286172E-2,4.3848914E-1,7.732984E-1,5.829365E-1,-8.535738E-2,1.0913203E0,6.530467E-1,-6.6295434E-3,-9.6199465E-1,-4.4754175E-3,-5.983665E-1,1.7425524E-2,-1.8540299E-1,-4.676011E-2,-1.9024761E-2,-7.261095E-2,-4.325785E-2,-3.25979E-2,-1.2279543E-2,-9.222166E-3,-4.7142047E-1,6.6972576E-4,-2.5482845E-1,2.9800006E-4,1.3078402E-2,-4.2912048E-1,-1.4479661E-1,4.7354615E-1,6.262219E-2,-9.535878E-1,-2.8819346E-1,-4.703104E-2,-4.9359676E-1,3.429966E-3,1.4978956E-2,-3.4154528E-3,-1.1430481E-2,2.140704E-2,8.1429E-3,3.800336E-1,3.0059744E-2,-2.22422E-2,-9.0761296E-4,-8.0186063E-1,-5.230385E-3,-3.6600655E-1,1.6942358E-1,-2.7835366E-1,-4.9321654E-1,-2.5302628E-1,-4.9778674E-2,-1.084743E-1,1.9041407E-1,3.0586502E-1,-2.4115829E-2,2.8738633E-1,6.7747945E-1,6.546677E-2,2.6776291E-2,-3.6072502E-1,-3.1509046E-2,-2.0674951E-3,2.4801062E-1,2.8282477E-2,9.395672E-3,-1.437643E-1,2.7284774E-1,5.6741923E-1,2.53684E-1,8.7783235E-1,3.4298804E-1,4.3449926E-1,8.8411516E-1,2.2546874E-1,-3.1043E-2,1.0930762E-1,1.1246588E0,7.062396E-1,5.1939292E-5,-4.4077568E-2,-8.76366E-3,-2.0328406E-2,-3.587679E-2,-2.9349606E-3,-1.5449486E-2,-2.3267588E-2,-1.1970625E-2,-1.34370215E-2,-9.166573E-5,-3.5716373E-2,-1.719856E-2,-1.5183254E-2,-2.894915E-3,7.3671434E-3,2.492424E-2,-2.8338903E-3,9.448827E-3,-1.5452728E-2,-4.6064414E-2,5.7366776E-4,-2.2319762E-2,-1.0586685E-2,9.857018E-3,-7.6910756E-3,-2.6671259E-2,2.1550123E-2,8.54125E-3,-3.8135156E-2,-5.274081E-3,-1.7806357E-2,-4.0442017E-3,3.2746736E-2,1.3530853E-3,-1.5601772E-2,-9.0350123E-4,-2.3611622E-2,9.217966E-4,-1.4547345E-2,-1.7744362E-3,-7.86391E-3,7.0575884E-6,-1.1427207E-2,-2.293948E-3,-7.264642E-3,1.1320199E-2,1.2209855E-2,2.747171E-2,1.5594409E-2,-1.3849166E-3,1.8967243E-2,3.5295494E-2,-2.0472737E-2,-3.9854785E-3,5.1897313E-3,1.55404E-2,-9.355936E-3,-6.1084505E-4,5.58401E-3,1.6498845E-2,1.1662946E-2,2.7938114E-2,1.5049098E-2,-1.714371E-3,4.3363165E-2,1.54470485E-2,-3.2371163E-4,1.873122E-2,2.3929302E-2,1.4921906E-2,4.1603927E-2,8.291929E-3,-3.2945066E-3,2.4533855E-2,1.9905329E-2,-1.6642863E-2,5.953383E-2,4.4758268E-2,1.1556755E-2,3.5676282E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,113,115,117,-1,119,121,123,125,127,129,-1,-1,131,133,135,-1,-1,-1,-1,137,139,141,143,145,147,-1,149,-1,151,-1,-1,153,155,157,159,-1,-1,161,-1,163,165,167,169,-1,-1,-1,-1,-1,171,173,175,-1,-1,177,-1,179,181,183,185,187,189,-1,191,-1,193,-1,195,-1,-1,-1,-1,-1,-1,-1,197,-1,199,-1,-1,201,203,205,207,209,211,213,215,-1,-1,-1,-1,-1,-1,217,-1,-1,-1,219,-1,221,223,225,227,229,231,233,235,237,-1,239,241,-1,-1,243,-1,-1,245,-1,-1,247,249,251,253,255,257,259,261,263,-1,265,267,269,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4458775E2,6.5058655E1,1.1292835E2,1.5611008E1,2.8987053E1,3.9374413E1,2.6923782E1,6.9664307E0,3.0314388E0,2.7582521E1,4.3259697E0,1.2471516E1,1.6214521E1,4.8996277E0,1.7142914E1,7.908798E0,7.616455E0,1.2125831E0,6.130581E-1,1.0879868E1,0E0,1.1060147E0,1.2710915E0,6.3043747E0,1.4715877E1,1.6891214E1,3.6684594E0,4.1959667E-1,3.0863113E0,7.978489E0,1.5118912E1,1.7360687E0,1.957869E0,2.6247482E0,3.0058503E-1,4.836712E-1,7.1533805E-1,4.002266E-1,2.9655242E-1,7.654331E0,2.5747643E0,6.573385E-1,3.077922E-1,5.07452E-1,2.638731E-1,1.1248474E0,3.965333E0,3.188861E0,4.918458E0,5.5221815E0,8.877628E0,2.0787525E-1,6.737628E-1,0E0,4.5785666E-1,9.2819357E-1,1.8515372E-1,3.599816E0,2.6296806E0,4.2064896E0,6.5513763E0,0E0,1.6914062E0,5.724611E-1,7.7511036E-1,3.4309387E-1,1.4942169E-1,3.6573958E-1,0E0,0E0,2.2328186E-1,1.3396448E-1,1.5746894E-1,0E0,0E0,0E0,0E0,3.768238E0,5.6845784E-1,1.1670303E0,5.547453E-1,8.2905054E-2,9.673342E-2,0E0,7.533944E-2,0E0,2.7028322E-1,0E0,0E0,5.8214426E-1,1.1363106E0,3.5301116E0,1.1034822E0,0E0,0E0,1.3162112E0,0E0,2.935647E0,1.9577127E0,6.9285316E0,5.0613785E-1,0E0,0E0,0E0,0E0,0E0,1.0769892E-1,3.0202538E-1,9.612298E-2,0E0,0E0,1.3420162E0,0E0,1.1427097E0,2.082964E0,4.757061E0,2.2071195E0,4.7551575E0,1.837038E0,0E0,8.615494E-1,0E0,3.9782524E-1,0E0,2.0336774E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1931324E-1,0E0,1.4792466E-1,0E0,0E0,1.8971405E0,1.1803973E0,2.2144699E-1,1.228143E-1,1.0996819E0,3.1433198E-1,3.4765103E-1,1.1635494E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.0403314E-1,0E0,0E0,0E0,8.1695557E-1,0E0,1.0835934E-1,2.6834104E0,1.414083E0,5.776787E-1,7.0295644E-1,5.8167684E-1,1.0687941E0,9.606186E-1,7.2904205E-1,0E0,3.240075E0,1.3366776E0,0E0,0E0,1.3480014E-1,0E0,0E0,1.893158E-1,0E0,0E0,7.0944026E-2,9.8189926E-1,3.18079E-1,5.633856E-1,1.4754868E0,3.0019462E-1,6.0521793E-1,7.348366E-1,7.943999E-1,0E0,1.1069261E0,2.4283905E0,1.6057835E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,77,77,78,78,79,79,80,80,81,81,82,82,84,84,86,86,89,89,90,90,91,91,92,92,95,95,97,97,98,98,99,99,100,100,106,106,107,107,108,108,111,111,113,113,114,114,115,115,116,116,117,117,118,118,120,120,122,122,124,124,132,132,134,134,137,137,138,138,139,139,140,140,141,141,142,142,143,143,144,144,151,151,155,155,157,157,158,158,159,159,160,160,161,161,162,162,163,163,164,164,165,165,167,167,168,168,171,171,174,174,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,185,185,187,187,188,188,189,189],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,114,116,118,-1,120,122,124,126,128,130,-1,-1,132,134,136,-1,-1,-1,-1,138,140,142,144,146,148,-1,150,-1,152,-1,-1,154,156,158,160,-1,-1,162,-1,164,166,168,170,-1,-1,-1,-1,-1,172,174,176,-1,-1,178,-1,180,182,184,186,188,190,-1,192,-1,194,-1,196,-1,-1,-1,-1,-1,-1,-1,198,-1,200,-1,-1,202,204,206,208,210,212,214,216,-1,-1,-1,-1,-1,-1,218,-1,-1,-1,220,-1,222,224,226,228,230,232,234,236,238,-1,240,242,-1,-1,244,-1,-1,246,-1,-1,248,250,252,254,256,258,260,262,264,-1,266,268,270,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,1.8532948E6,1E0,1.5083523E0,1.8108038E5,2.5998926E-1,2.6944466E7,5.1148495E6,1.3144558E4,2.1442623E1,3.131E3,4.898012E-1,3.731E3,9.923929E2,4.4866666E2,2.4519731E3,2.0368582E8,2E1,6.971004E7,8.176172E-2,2.0783027E5,3.7379084E0,9.6E1,1E0,9.27E2,2.5422776E3,1E0,1.8159722E-1,6.743895E5,2.6757258E-1,2.5004512E5,3.647E3,8.091679E-1,8.454452E6,2.7487562E0,2.437E3,1.998E3,1E0,2.0601392E3,8.454878E1,1.0275E4,1E0,1.5515003E0,1.515252E10,6.48334E5,1.9557823E0,4.0621606E2,1.5963264E9,1.5859042E6,1.0872E4,4.2343444E7,7.431E3,-3.4827094E-3,4.6501E4,1.0872E4,5.841568E1,5.7E1,2.465E3,5.0714142E2,1.02E2,-1.0951205E-4,9.532E3,4.4648915E6,3.68E2,1.3312868E6,2.5269838E6,2.711499E6,-6.3381754E-3,2.3030257E-3,1.02E2,2.2292539E8,2.821E3,-3.6159147E-2,-1.7639177E-2,-2.3646409E-2,-7.952474E-3,2.562723E-1,2.2243262E5,3.970405E3,8.387235E11,5.18E2,5.9572783E0,1.5944161E-3,2.1729739E0,6.025149E-3,5.863334E6,4.4022363E-2,1.4310162E-2,2.53E2,2.102858E4,2.35E2,4.602015E0,8.145203E-3,6.596336E-2,8.17766E5,2.7423227E-2,4.264897E6,4.4E1,2.2515285E6,1.9642437E1,-1.934345E-2,-3.3453338E-2,-3.019639E-3,2.6449924E-2,-1.9614517E-3,4.91271E5,5.704748E4,5.21929E5,-1.2140454E-2,1.8365661E-3,8.69032E-2,-4.0286172E-2,1.4492002E0,2E0,4.0889E4,6.5321615E6,3.171E3,4.5481584E7,-6.6295434E-3,6.77603E5,-4.4754175E-3,1.460806E6,1.7425524E-2,2.1693E5,-4.676011E-2,-1.9024761E-2,-7.261095E-2,-4.325785E-2,-3.25979E-2,-1.2279543E-2,-9.222166E-3,3.7382904E7,6.6972576E-4,1.5377966E6,2.9800006E-4,1.3078402E-2,1E0,9.994E3,7.299E3,1.3683E4,1E0,9.35E2,7.5E2,8.781059E4,3.429966E-3,1.4978956E-2,-3.4154528E-3,-1.1430481E-2,2.140704E-2,8.1429E-3,2.9E1,3.0059744E-2,-2.22422E-2,-9.0761296E-4,1E0,-5.230385E-3,1.8480903E0,2E0,2.4738782E2,4.96576E5,2.1975278E8,1.7027656E3,3.79E2,3.6E2,7.064848E6,-2.4115829E-2,1.0238709E1,5.8036E4,6.546677E-2,2.6776291E-2,7E0,-3.1509046E-2,-2.0674951E-3,3.403235E1,2.8282477E-2,9.395672E-3,2.84E2,1E0,2.8034655E6,1.9E1,2E0,7.3308685E2,6.852143E1,7.56E2,3.0972284E1,-3.1043E-2,2.821E3,1.4307411E7,3.3704692E10,5.1939292E-5,-4.4077568E-2,-8.76366E-3,-2.0328406E-2,-3.587679E-2,-2.9349606E-3,-1.5449486E-2,-2.3267588E-2,-1.1970625E-2,-1.34370215E-2,-9.166573E-5,-3.5716373E-2,-1.719856E-2,-1.5183254E-2,-2.894915E-3,7.3671434E-3,2.492424E-2,-2.8338903E-3,9.448827E-3,-1.5452728E-2,-4.6064414E-2,5.7366776E-4,-2.2319762E-2,-1.0586685E-2,9.857018E-3,-7.6910756E-3,-2.6671259E-2,2.1550123E-2,8.54125E-3,-3.8135156E-2,-5.274081E-3,-1.7806357E-2,-4.0442017E-3,3.2746736E-2,1.3530853E-3,-1.5601772E-2,-9.0350123E-4,-2.3611622E-2,9.217966E-4,-1.4547345E-2,-1.7744362E-3,-7.86391E-3,7.0575884E-6,-1.1427207E-2,-2.293948E-3,-7.264642E-3,1.1320199E-2,1.2209855E-2,2.747171E-2,1.5594409E-2,-1.3849166E-3,1.8967243E-2,3.5295494E-2,-2.0472737E-2,-3.9854785E-3,5.1897313E-3,1.55404E-2,-9.355936E-3,-6.1084505E-4,5.58401E-3,1.6498845E-2,1.1662946E-2,2.7938114E-2,1.5049098E-2,-1.714371E-3,4.3363165E-2,1.54470485E-2,-3.2371163E-4,1.873122E-2,2.3929302E-2,1.4921906E-2,4.1603927E-2,8.291929E-3,-3.2945066E-3,2.4533855E-2,1.9905329E-2,-1.6642863E-2,5.953383E-2,4.4758268E-2,1.1556755E-2,3.5676282E-2],"split_indices":[19,55,50,109,41,27,41,44,44,51,55,2,26,28,51,4,4,7,3,44,0,32,34,28,8,2,51,66,26,27,37,44,2,26,31,53,2,10,112,51,55,9,15,38,5,30,52,51,7,27,2,44,2,0,1,2,55,6,2,57,8,0,9,44,0,27,31,28,0,0,0,7,2,0,0,0,0,38,32,51,30,0,52,0,38,0,1,0,0,2,27,2,52,0,0,1,0,9,3,27,57,0,0,0,0,0,30,27,10,0,0,37,0,52,6,2,46,2,46,0,28,0,46,0,28,0,0,0,0,0,0,0,12,0,32,0,0,80,10,2,9,8,0,0,32,0,0,0,0,0,0,3,0,0,0,67,0,52,10,51,1,7,4,2,2,27,0,52,1,0,0,6,0,0,55,0,0,0,107,46,3,16,4,57,3,55,0,2,50,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.196E3,6.89E2,1.507E3,3.4E2,3.49E2,9.31E2,5.76E2,2.36E2,1.04E2,2.73E2,7.6E1,3.96E2,5.35E2,6.3E1,5.13E2,1.63E2,7.3E1,5.8E1,4.6E1,2.68E2,5E0,3.5E1,4.1E1,2.36E2,1.6E2,5.05E2,3E1,1.8E1,4.5E1,1.89E2,3.24E2,1.22E2,4.1E1,5.9E1,1.4E1,3.6E1,2.2E1,3.1E1,1.5E1,2.17E2,5.1E1,2.2E1,1.3E1,3E1,1.1E1,6.4E1,1.72E2,1E1,1.5E2,2.55E2,2.5E2,2.5E1,5E0,3E0,1.5E1,3.1E1,1.4E1,8.4E1,1.05E2,1.24E2,2E2,2E0,1.2E2,2.8E1,1.3E1,2.1E1,3.8E1,1.1E1,3E0,2E0,3.4E1,1.5E1,7E0,2.5E1,6E0,1E1,5E0,2E2,1.7E1,4E1,1.1E1,6E0,1.6E1,3E0,1E1,5E0,2.5E1,9E0,2E0,2.9E1,3.5E1,5.2E1,1.2E2,3E0,7E0,1.41E2,9E0,1.72E2,8.3E1,2.39E2,1.1E1,9E0,1.6E1,3E0,2E0,2E0,1.3E1,2.1E1,1E1,1.2E1,2E0,8.2E1,2E0,5.3E1,5.2E1,1.14E2,1E1,1.48E2,5.2E1,3E0,1.17E2,3E0,2.5E1,2E0,1.1E1,1.8E1,3E0,3.2E1,6E0,6E0,5E0,6E0,2.8E1,2E0,1.3E1,4E0,3E0,1.23E2,7.7E1,1.2E1,5E0,3.6E1,4E0,5E0,6E0,3E0,3E0,1.2E1,4E0,7E0,3E0,1.5E1,1E1,2.6E1,3E0,3.2E1,3E0,1.8E1,3.4E1,8.2E1,3.8E1,4.8E1,9.3E1,1.29E2,4.3E1,8.1E1,2E0,1.77E2,6.2E1,8E0,3E0,6E0,7E0,4E0,1.7E1,8E0,2E0,8E0,7.4E1,3E1,2.3E1,4.1E1,1.1E1,7.8E1,3.6E1,7E0,3E0,5E0,1.43E2,4.8E1,4E0,1.15E2,2E0,1.6E1,9E0,7E0,4E0,2.2E1,6E0,1.1E1,2E0,1.3E1,1.1E2,2.2E1,5.5E1,3E0,9E0,3E0,2E0,4E0,3.2E1,2E0,2E0,3E0,2E0,2E0,4E0,9E0,6E0,3E1,2E0,1.6E1,2E0,6E0,2.8E1,6.5E1,1.7E1,3.6E1,2E0,3.6E1,1.2E1,2.6E1,6.7E1,3.6E1,9.3E1,6E0,3.7E1,7.4E1,7E0,1.5E2,2.7E1,1.9E1,4.3E1,4E0,2E0,8E0,9E0,5E0,3E0,2.9E1,4.5E1,5E0,2.5E1,1.8E1,5E0,3.5E1,6E0,2E0,9E0,3.9E1,3.9E1,3.4E1,2E0,4E0,3E0,3E0,2E0,5.6E1,8.7E1,8E0,4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"271","size_leaf_vector":"1"}},{"base_weights":[-4.4414337E-4,-5.062649E-1,2.2674116E-1,-7.64031E-1,-1.830245E-1,5.230733E-2,6.023918E-1,-8.947878E-1,-1.6550012E-1,-3.772173E-1,2.5953144E-1,-1.7470218E-1,2.0949642E-1,3.269966E-1,7.7858114E-1,-4.90552E-1,-9.938687E-1,8.2822375E-2,-2.838932E-1,-4.8695496E-1,-2.2715715E-1,1.1842244E-3,5.017073E-1,-3.3765325E-1,1.2997794E-2,2.5356558E-1,-3.9462247E-1,7.8495204E-2,5.337211E-1,6.591071E-1,1.0923643E0,-7.0209646E-1,-1.8248913E-1,-9.398589E-1,-7.3877074E-2,2.2077829E-2,-5.7251525E-1,-5.389983E-1,-1.11732885E-1,-3.4749234E-1,-1.3432261E-1,-2.0983799E-1,1.321227E-1,3.6625254E-1,6.7303133E-1,-5.395346E-1,-2.442059E-1,6.0655165E-2,-4.2640466E-2,5.5899482E-2,4.1289774E-1,-5.069357E-1,3.6374626E-1,2.0177995E-1,-4.4101727E-1,6.043578E-1,-1.7912182E-3,4.0507227E-1,8.5100925E-1,1.1504705E0,1.4956143E-2,-1.113435E-2,-7.6568663E-1,6.807936E-2,-5.01373E-1,-1.0312523E0,-6.2865025E-1,-1.3989455E-1,4.6954802E-1,-4.6939942E-1,-5.5620596E-2,-5.7591236E-1,-1.2703216E-1,-2.7773154E-1,2.1122012E-1,-4.1969693E-1,-2.493982E-1,-1.8898909E-1,3.14618E-2,-3.3122748E-1,-7.960364E-2,3.1865785E-1,1.717484E-2,2.57915E-2,1.653659E-1,3.552033E-2,1.8456427E-2,-8.20078E-1,-4.8558354E-1,1.7319025E-1,-2.86281E-1,-1.0091755E-1,3.8742334E-1,7.4450955E-2,-4.0306255E-2,3.8249245E-1,6.952673E-2,-6.445733E-1,-2.0325303E-1,4.9320776E-3,2.6360353E-2,1.2546414E-1,4.78328E-1,-1.1714014E-2,-2.4571996E-2,4.4465014E-1,8.0590004E-1,1.8518114E-1,-3.3798322E-1,6.298481E-1,2.8986722E-1,7.3366964E-1,1.1979601E0,1.1789727E0,1.6482625E-2,-8.2628465E-1,-3.653811E-1,1.9877829E-2,-1.6278584E-2,-2.7197694E-2,-6.902594E-3,-9.436059E-1,-1.3171905E0,-2.3251817E-1,-7.457264E-1,-1.8205194E-1,1.0410458E-2,-1.0306372E-3,6.134608E-1,-6.1434627E-1,-1.8870474E-1,-6.2194043E-1,-3.5422134E-1,1.5146767E-3,-1.0847643E-2,-2.0541893E-2,1.0958445E-3,-1.96218E-3,1.5815787E-2,-4.590565E-3,-2.0276653E-2,1.193179E-3,-1.2830184E-2,-4.736619E-2,-2.8143075E-1,-6.9273864E-3,4.331922E-3,-1.8222408E-2,-2.4689767E-3,8.604711E-5,-9.160509E-3,6.4979796E-3,1.8029427E-2,1.4759189E-1,-1.149597E-1,2.538592E-1,-3.6260425E-3,-1.46677E-2,-4.4658702E-2,-5.829896E-1,-3.5463908E-1,-2.3508938E-1,3.048656E-2,-3.5201493E-1,-5.6500703E-2,-1.4183138E-1,1.7890814E-1,4.959588E-1,-1.2466952E-4,-1.8039365E-1,1.3223371E-1,5.8708113E-1,2.7121487E-1,-6.9375104E-1,-2.3853062E-3,-1.3549894E-2,-3.5396584E-3,-4.2918388E-2,1.9809969E-1,5.285681E-1,4.5807343E-3,2.3047337E-1,2.3344358E-2,8.4963036E-1,-9.387313E-5,2.1283282E-2,-6.1922953E-3,-5.2478965E-3,-2.0255163E-2,2.3940648E-1,7.55245E-1,3.4464297E-1,-1.366434E-1,7.78934E-1,-2.1877058E-1,6.0249414E-2,3.4179714E-2,3.4773875E-2,1.2270052E0,-4.032552E-2,-2.0481026E-2,-5.675424E-3,-2.5517259E-2,1.7991517E-3,-5.6618913E-3,-4.4995513E-2,-3.2540843E-2,-6.5732546E-2,-4.404971E-2,-4.5311754E-3,-2.438382E-2,-3.832336E-2,-2.1901833E-2,-1.5384949E-2,-4.1262136E-4,3.2908347E-2,9.95827E-3,-5.712617E-3,-2.9952208E-2,-1.026035E-2,2.753003E-5,-2.883221E-2,-1.4593805E-3,-1.9505152E-2,-2.4251384E-3,4.995751E-3,-9.046388E-3,-1.6095402E-2,-2.2969027E-3,-3.2996892E-3,9.589968E-3,-4.3799233E-4,-8.309472E-3,6.2110904E-3,1.5740143E-2,-9.295822E-3,-2.7172396E-2,-1.0805901E-2,-2.6521945E-2,-1.5017612E-2,3.1108211E-3,-1.33626E-2,-2.5206408E-2,-7.4702045E-3,4.6421257E-3,-1.4640287E-3,-1.1358721E-2,1.3569989E-2,2.4106188E-3,4.135053E-2,1.7507542E-2,-4.4795787E-3,-1.3859328E-2,8.590485E-3,-3.0044406E-3,2.2360114E-2,3.902671E-2,1.5216705E-2,4.15053E-3,-9.410216E-3,-3.305585E-2,-4.4352747E-3,5.423663E-3,1.5932763E-2,4.1530393E-3,8.323729E-3,2.6011234E-2,4.6085828E-4,1.288943E-2,3.979942E-2,1.0088945E-2,2.9660314E-3,1.4280961E-2,3.5757106E-2,1.1845993E-2,5.929745E-3,2.0284103E-2,9.139392E-3,-1.281821E-2,7.540332E-3,3.6481418E-2,-2.2104226E-2,5.596547E-3,5.6349386E-2,2.1340074E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,141,143,145,147,149,151,153,-1,155,-1,-1,157,159,161,163,165,167,169,-1,171,-1,173,175,-1,-1,177,179,-1,-1,181,183,185,187,189,191,193,195,197,-1,199,201,-1,203,-1,-1,205,207,209,211,213,-1,-1,215,217,219,221,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,-1,-1,229,231,233,-1,-1,-1,235,237,239,-1,241,243,245,247,249,-1,251,253,255,257,259,-1,-1,-1,261,263,265,-1,267,-1,269,-1,-1,-1,-1,-1,271,273,275,277,279,281,-1,-1,-1,283,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.521235E2,5.6518036E1,9.9175125E1,2.9434937E1,2.6138077E1,3.6968937E1,2.3035812E1,1.1928574E1,1.6815601E1,3.3639736E0,5.8179946E0,1.29728155E1,1.6393354E1,9.667288E0,1.0226791E1,3.9775229E0,7.010788E0,0E0,5.8410997E0,2.3026428E0,9.699087E-1,1.298747E0,8.6741924E-1,4.1506596E0,1.4672979E1,1.7960316E1,3.7617402E0,5.661161E0,3.8972435E0,1.0087738E1,3.117897E0,8.849716E-1,2.191494E0,5.921921E0,0E0,2.457455E0,1.7232895E0,1.5388203E0,9.164187E-1,1.7155933E-1,4.8627692E-1,2.5819653E-1,6.163396E-1,1.090884E0,2.721157E-1,6.764145E-1,2.8068113E0,0E0,4.8261046E0,4.5598316E0,1.020047E1,1.4155188E0,2.706166E-1,1.4384406E0,1.7479014E-1,2.5707512E0,8.797501E-1,2.2515879E0,4.0471344E0,1.2070236E0,0E0,0E0,4.7433662E-1,5.078653E-1,3.244357E-1,3.1929474E0,2.3773766E0,4.3164715E-1,6.516156E-1,1.1453295E0,0E0,7.6948166E-1,1.8867403E-1,5.9550714E-1,2.2609207E-1,1.4287877E-1,1.855619E-1,5.10344E-1,1.8049736E-1,1.5551662E-1,1.004159E-1,9.613502E-2,3.443865E-1,0E0,3.6787266E-1,0E0,0E0,5.056038E-1,5.8404446E-1,3.221562E0,2.1400347E0,1.9602119E0,9.7537065E-1,3.7347112E0,0E0,6.8755074E0,0E0,6.911411E-1,1.32743E-1,0E0,0E0,7.056439E-1,2.3074102E-1,0E0,0E0,6.868191E-1,1.3797455E0,8.898769E-1,6.96215E-2,1.3369532E0,1.5422893E0,4.1797905E0,7.01622E-1,4.391098E-1,0E0,2.4736977E-1,2.1483701E-1,0E0,9.028029E-2,0E0,0E0,8.5601044E-1,3.9565277E-1,5.135687E-1,6.9901466E-1,6.2443334E-1,0E0,0E0,1.5251446E-1,3.7765408E-1,8.629042E-2,6.73542E-1,4.1451788E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3077582E-1,3.921281E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5704672E-1,7.042694E-2,8.363557E-2,0E0,0E0,0E0,9.747124E-2,6.3757825E-1,2.8695965E-1,0E0,1.1279869E0,5.8633906E-1,1.759177E0,3.18065E-1,4.676385E-1,0E0,4.51015E-1,2.3653932E0,2.341507E0,2.3224726E0,2.547989E-1,0E0,0E0,0E0,1.7296101E-1,6.2217844E-1,1.0033965E-1,0E0,1.7628443E-1,0E0,4.870739E-1,0E0,0E0,0E0,0E0,0E0,9.8321795E-2,1.8557644E-1,1.2004828E0,4.5384198E-1,1.3296623E0,5.216531E-1,0E0,0E0,0E0,2.2582245E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,97,97,98,98,101,101,102,102,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124,125,125,128,128,129,129,130,130,131,131,132,132,143,143,144,144,153,153,154,154,155,155,159,159,160,160,161,161,163,163,164,164,165,165,166,166,167,167,169,169,170,170,171,171,172,172,173,173,177,177,178,178,179,179,181,181,183,183,189,189,190,190,191,191,192,192,193,193,194,194,198,198],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,142,144,146,148,150,152,154,-1,156,-1,-1,158,160,162,164,166,168,170,-1,172,-1,174,176,-1,-1,178,180,-1,-1,182,184,186,188,190,192,194,196,198,-1,200,202,-1,204,-1,-1,206,208,210,212,214,-1,-1,216,218,220,222,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,-1,-1,230,232,234,-1,-1,-1,236,238,240,-1,242,244,246,248,250,-1,252,254,256,258,260,-1,-1,-1,262,264,266,-1,268,-1,270,-1,-1,-1,-1,-1,272,274,276,278,280,282,-1,-1,-1,284,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.126489E7,2.867292E3,4.197889E6,1.8108038E5,2.1150263E-1,1.759343E-1,1.4E1,1.5669752E-2,8.8543115E2,2.857E3,3.956147E-1,1.2204E4,2.9251662E6,2.373E3,1.3521131E8,8.2822375E-2,6.971004E7,6.019605E2,2.1787034E8,1.6649964E5,1.3664E4,1.141E4,2.3E1,9.27E2,2.200423E3,5.9767612E7,1.1803382E4,1.3508893E3,1E0,1.7515824E8,1E0,3.83E3,-7.3877074E-2,9.908038E1,9.22E4,3.906E3,4.3E1,2.0939131E2,6.604754E5,1.4563726E-1,1.683274E0,1.8662969E6,2.4848485E0,3.983889E3,7.330957E4,6.0655165E-2,8.538E3,1.062363E1,3.0497742E3,2.6476662E6,3.09E2,3.21E2,1.0017953E8,2.1019447E-1,9.3161255E-1,1.2E1,7.7101436E3,3.2703E4,1.4956143E-2,-1.113435E-2,3.008162E6,2.34E2,6.8102196E-2,4.0844156E7,3.0161016E7,3.5985E4,5.015233E4,6.5317163E3,-5.5620596E-2,9.6771875E2,6.822312E4,1E0,3.52E2,2.07E2,1.4E1,5.03E2,7.27E2,3.38464E5,4.769198E-1,5.2430645E-2,2.7682406E8,2.57915E-2,4.4881E4,3.552033E-2,1.8456427E-2,3.833E3,6.1E1,3.188889E1,4.564527E6,6.28205E8,4.2343444E7,3.97E2,-4.0306255E-2,3.1753032E0,6.952673E-2,1E0,2.8980975E5,4.9320776E-3,2.6360353E-2,5.202E3,1.54E2,-1.1714014E-2,-2.4571996E-2,1E0,3.653602E7,1.74E4,2.3580047E10,2.6680525E6,2.5727873E5,3.7284137E2,7.6696295E-1,1.52179E5,1.6482625E-2,3.8E1,4.44E2,1.9877829E-2,5.377837E6,-2.7197694E-2,-6.902594E-3,1.892E3,9.6771875E2,8.9E1,4.7272725E0,3.3368218E6,1.0410458E-2,-1.0306372E-3,7.9684106E9,5.11E2,2.88001E10,4.0465977E1,3.8398104E0,1.5146767E-3,-1.0847643E-2,-2.0541893E-2,1.0958445E-3,-1.96218E-3,1.5815787E-2,-4.590565E-3,-2.0276653E-2,1.193179E-3,-1.2830184E-2,2.8050448E6,1.9840434E7,-6.9273864E-3,4.331922E-3,-1.8222408E-2,-2.4689767E-3,8.604711E-5,-9.160509E-3,6.4979796E-3,1.8029427E-2,3.09E2,8.286065E2,1.3569831E7,-3.6260425E-3,-1.46677E-2,-4.4658702E-2,4E0,3.891E3,2.2E1,3.048656E-2,2.2E1,1.0028155E7,2.8721272E1,1.3632204E-3,7.969174E7,-1.2466952E-4,4.278699E0,2E0,6.072483E2,2.2E1,2.345785E5,-2.3853062E-3,-1.3549894E-2,-3.5396584E-3,1.7488463E3,5.3E1,2.4239502E8,4.5807343E-3,1.6173E4,2.3344358E-2,1.7337298E9,-9.387313E-5,2.1283282E-2,-6.1922953E-3,-5.2478965E-3,-2.0255163E-2,1.4329944E0,8.229196E7,2.4898794E5,1.5552E4,1.8722344E8,1.1623708E0,6.0249414E-2,3.4179714E-2,3.4773875E-2,4.25E0,-4.032552E-2,-2.0481026E-2,-5.675424E-3,-2.5517259E-2,1.7991517E-3,-5.6618913E-3,-4.4995513E-2,-3.2540843E-2,-6.5732546E-2,-4.404971E-2,-4.5311754E-3,-2.438382E-2,-3.832336E-2,-2.1901833E-2,-1.5384949E-2,-4.1262136E-4,3.2908347E-2,9.95827E-3,-5.712617E-3,-2.9952208E-2,-1.026035E-2,2.753003E-5,-2.883221E-2,-1.4593805E-3,-1.9505152E-2,-2.4251384E-3,4.995751E-3,-9.046388E-3,-1.6095402E-2,-2.2969027E-3,-3.2996892E-3,9.589968E-3,-4.3799233E-4,-8.309472E-3,6.2110904E-3,1.5740143E-2,-9.295822E-3,-2.7172396E-2,-1.0805901E-2,-2.6521945E-2,-1.5017612E-2,3.1108211E-3,-1.33626E-2,-2.5206408E-2,-7.4702045E-3,4.6421257E-3,-1.4640287E-3,-1.1358721E-2,1.3569989E-2,2.4106188E-3,4.135053E-2,1.7507542E-2,-4.4795787E-3,-1.3859328E-2,8.590485E-3,-3.0044406E-3,2.2360114E-2,3.902671E-2,1.5216705E-2,4.15053E-3,-9.410216E-3,-3.305585E-2,-4.4352747E-3,5.423663E-3,1.5932763E-2,4.1530393E-3,8.323729E-3,2.6011234E-2,4.6085828E-4,1.288943E-2,3.979942E-2,1.0088945E-2,2.9660314E-3,1.4280961E-2,3.5757106E-2,1.1845993E-2,5.929745E-3,2.0284103E-2,9.139392E-3,-1.281821E-2,7.540332E-3,3.6481418E-2,-2.2104226E-2,5.596547E-3,5.6349386E-2,2.1340074E-2],"split_indices":[19,109,12,4,50,27,37,26,0,37,51,2,26,2,27,2,44,0,44,51,7,27,2,9,0,2,51,44,4,51,6,30,111,2,0,55,10,0,3,51,27,37,52,46,56,27,44,0,2,53,51,31,0,8,44,34,33,3,51,2,0,0,31,10,26,44,44,2,27,4,0,4,32,79,0,11,3,0,0,9,26,56,7,0,9,0,0,1,2,51,9,7,44,2,0,53,0,79,27,0,0,2,3,0,0,109,44,9,5,44,32,57,26,1,0,3,0,0,28,0,0,2,4,8,53,31,0,0,12,0,5,57,52,0,0,0,0,0,0,0,0,0,0,47,9,0,0,0,0,0,0,0,0,0,4,44,0,0,0,0,28,0,0,3,5,52,38,7,0,53,16,4,3,27,0,0,0,4,8,5,0,2,0,31,0,0,0,0,0,37,31,27,9,7,40,0,0,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.192E3,6.79E2,1.513E3,3.77E2,3.02E2,1.034E3,4.79E2,3.09E2,6.8E1,2.1E2,9.2E1,4.23E2,6.11E2,1.88E2,2.91E2,6.2E1,2.47E2,3E0,6.5E1,1.2E2,9E1,4.5E1,4.7E1,2.26E2,1.97E2,5.7E2,4.1E1,8.6E1,1.02E2,2.13E2,7.8E1,3.6E1,2.6E1,2.3E2,1.7E1,3.2E1,3.3E1,1.05E2,1.5E1,3.8E1,5.2E1,1.7E1,2.8E1,2.8E1,1.9E1,7E1,1.56E2,7E0,1.9E2,2.55E2,3.15E2,3.6E1,5E0,7E1,1.6E1,9E1,1.2E1,9.3E1,1.2E2,7.2E1,6E0,5E0,3.1E1,1.5E1,1.1E1,1.76E2,5.4E1,2.4E1,8E0,3E1,3E0,9.6E1,9E0,1E1,5E0,2E1,1.8E1,3.9E1,1.3E1,8E0,9E0,1E1,1.8E1,1.3E1,1.5E1,1.2E1,7E0,9E0,6.1E1,1.4E1,1.42E2,1.68E2,2.2E1,2.51E2,4E0,3.08E2,7E0,2.4E1,1.2E1,3E0,2E0,5.6E1,1.4E1,7E0,9E0,5.2E1,3.8E1,8E0,4E0,3E1,6.3E1,9.2E1,2.8E1,6.9E1,3E0,2.6E1,5E0,2E0,1.3E1,8E0,3E0,1.38E2,3.8E1,1.3E1,4.1E1,2.2E1,2E0,2E0,6E0,1.9E1,1.1E1,7.8E1,1.8E1,4E0,5E0,6E0,4E0,2E0,3E0,2E0,1.8E1,2E0,1.6E1,1.6E1,2.3E1,3E0,1E1,6E0,2E0,6E0,3E0,4E0,6E0,9E0,9E0,1.1E1,4E0,3E0,6E0,3.3E1,2.8E1,8E0,6E0,1.1E2,3.2E1,1.47E2,2.1E1,1.7E1,5E0,4.6E1,2.05E2,1.07E2,2.01E2,2.2E1,2E0,6E0,6E0,1.7E1,3.9E1,1.2E1,2E0,1.4E1,3.8E1,3.6E1,2E0,4E0,4E0,2E0,2E0,8E0,2.2E1,5.6E1,7E0,8.8E1,4E0,2E1,8E0,9E0,6E1,2.1E1,5E0,3E0,2E0,9E0,4E0,1.1E2,2.8E1,2.5E1,1.3E1,1E1,3E0,2.8E1,1.3E1,1.1E1,1.1E1,4E0,2E0,2E0,1.7E1,9E0,2E0,7.6E1,2E0,1.4E1,4E0,8E0,8E0,1.7E1,6E0,2E0,7E0,4E0,5E0,6E0,5E0,2E0,3.1E1,2E1,8E0,6E0,2E0,8.8E1,2.2E1,1.9E1,1.3E1,7.4E1,7.3E1,1E1,1.1E1,2E0,1.5E1,2.9E1,1.7E1,1.59E2,4.6E1,8.2E1,2.5E1,1.47E2,5.4E1,2E0,2E1,1.3E1,4E0,1.5E1,2.4E1,2E0,1E1,3E0,1.1E1,3.4E1,2E0,3E0,5E0,2E1,2E0,1.9E1,3.7E1,2E0,5E0,4E0,8.4E1,2E0,2E0,5.8E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"285","size_leaf_vector":"1"}},{"base_weights":[-1.4517497E-3,-4.450119E-1,2.1133822E-1,-7.073225E-1,-2.1664575E-1,2.3032013E-2,5.1278627E-1,-8.684884E-1,-4.5586833E-1,-3.3337128E-1,3.667193E-1,-2.9700893E-1,1.0991913E-1,-2.8685778E-1,5.689398E-1,-5.098753E-1,-9.7229564E-1,-5.1646775E-1,-1.3215013E-1,-3.7117073E-1,7.669686E-2,1.833075E-1,6.112723E-1,-4.600244E-1,-1.6686039E-1,1.4610146E-1,-4.759227E-1,2.6105797E-2,-4.920995E-1,3.0130953E-1,6.9270897E-1,-6.021196E-1,1.5080631E-2,-1.0491209E0,-4.6365657E-1,-6.5577877E-1,-4.021281E-1,-1.9886789E-1,6.087709E-3,-6.480803E-1,-2.2826889E-1,-3.139693E-2,2.9620966E-1,7.651228E-1,2.732141E-1,-4.104997E-1,-5.3433415E-2,-8.0320604E-2,-3.287168E-1,1.0018475E0,1.2519808E-1,-7.8137934E-1,-1.0085142E-1,1.0445598E-1,-8.955243E-2,-2.6087303E-2,-1.5775135E-1,3.8539463E-1,-2.8187194E-1,7.680832E-1,2.1653534E-1,-6.41751E-1,4.5538284E-3,7.1883556E-3,-2.6838318E-3,-9.58474E-1,-1.652072E0,1.2683767E-2,-2.5815867E-2,-7.291848E-1,-4.0205595E-1,-5.301541E-1,-2.6967555E-1,-5.803627E-4,-1.0649478E-2,-4.239422E-1,-8.194455E-1,-1.6281192E-1,-5.704555E-1,-1.0392242E-2,4.3509845E-2,1.5043612E-1,4.4159466E-1,4.460039E-2,2.5943302E-2,2.4219915E-4,1.7532287E-2,-2.3952365E-2,-2.4431767E-1,-1.4520264E-1,8.72541E-2,-6.1383094E-3,-3.928515E-1,-1.4646217E-2,1.2142904E0,-3.566654E-2,2.6696995E-1,-3.8648997E-2,-1.9661505E-2,-2.2604671E-1,7.728134E-2,-1.3496116E-3,7.5271134E-3,-8.146069E-3,1.0508824E-3,3.2215612E-3,-1.3129211E-2,3.066386E-1,6.427566E-1,-4.458116E-2,-1.19073234E-1,7.076459E-1,1.1517739E0,-2.6087543E-1,4.2270303E-1,-1.5219989E-2,-6.885409E-1,-1.0251505E0,-6.0167503E-1,-7.889809E-2,-3.0719984E-2,-5.2869217E-3,6.243996E-3,-5.3737384E-1,-3.805743E-2,-3.0936353E-2,-1.3075047E-2,2.8584866E-4,-5.8785826E-1,-3.312684E-1,-3.578933E-2,-5.4310364E-1,-2.134798E-1,-8.7521005E-1,-1.809901E-1,-2.1460992E-1,4.5189422E-1,-7.2687036E-1,-2.3615694E-1,-2.8408393E-3,5.124669E-3,8.994676E-3,-8.974274E-4,2.2206E-2,5.9509412E-3,3.133787E-2,-3.0049336E-1,-1.7921098E-1,9.3929603E-4,1.2397487E-1,-2.49221E-3,-1.9956382E-2,-1.4360537E-1,1.976785E-2,5.919638E-2,-1.407625E-1,2.3654331E-1,2.5093946E-1,7.2387956E-2,-1.2900008E-2,2.063919E-3,9.935169E-3,2.4056845E-4,3.2343495E-1,-1.9641465E-2,4.5146722E-1,8.297404E-1,6.5705115E-3,-8.04434E-3,6.440191E-1,1.1359625E0,1.2145219E0,-2.1674735E-3,-3.6092967E-2,-7.5781256E-2,5.189291E-1,-1.4491563E-1,-3.3610556E-2,-1.5008307E-2,-4.772963E-2,-3.1657178E-2,-7.0931576E-3,-3.0203981E-2,-2.6964169E-2,-3.9220667E-3,-1.8179469E-2,-3.0290915E-2,-1.6539589E-2,-5.214517E-3,-5.585641E-3,3.7414029E-3,-8.624645E-3,-2.8220035E-2,-4.7747223E-4,-1.2083269E-2,-1.372787E-2,-4.198654E-2,1.8646593E-3,-1.3689699E-2,-1.5061611E-2,-4.873595E-3,2.3735335E-2,5.3415843E-4,-9.219335E-3,-3.766032E-2,-8.923555E-4,-1.756385E-2,-7.587293E-3,1.0070175E-2,-8.391263E-3,-1.7097527E-2,-3.4497825E-3,-1.1662027E-2,2.6817133E-3,-5.246481E-3,-2.033823E-3,6.7679742E-3,-1.2043395E-2,-7.198268E-4,-1.0729425E-2,7.9328794E-4,6.645646E-3,2.4902904E-2,1.2548768E-2,-1.2573152E-2,1.9774145E-2,9.524917E-3,7.620296E-3,2.3350192E-2,1.6882332E-2,4.104005E-2,3.1302083E-2,1.368748E-2,5.3617436E-2,1.0948394E-2,3.2134358E-2,5.8167312E-2,-7.815204E-3,1.3788441E-2,1.4807812E-2,4.0254124E-2,-3.270904E-4,-1.2685684E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,83,85,87,-1,89,91,93,95,97,99,101,103,-1,105,107,109,111,113,115,-1,-1,-1,117,119,121,-1,123,125,127,129,-1,-1,131,133,135,137,-1,139,141,143,-1,-1,-1,-1,-1,145,147,149,-1,151,-1,153,155,157,-1,-1,159,161,-1,-1,-1,-1,-1,-1,163,165,-1,167,169,171,173,175,-1,177,179,181,-1,-1,-1,-1,183,-1,-1,-1,-1,185,187,189,191,193,195,197,199,201,203,205,-1,-1,-1,-1,-1,-1,207,209,211,213,215,-1,-1,217,-1,-1,219,221,223,-1,-1,-1,-1,-1,225,-1,227,229,-1,-1,231,233,235,-1,-1,237,239,241,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0670316E2,4.2377808E1,8.3999695E1,1.29776E1,2.6065914E1,2.5393028E1,2.5712936E1,6.9495697E0,2.5015926E0,2.4998981E1,2.7585993E0,4.060402E0,1.5289065E1,2.4537077E0,1.7413452E1,2.285016E0,5.5313416E0,1.4884586E0,4.18074E-1,1.2237755E1,0E0,9.304428E-1,1.1952381E0,2.5829372E0,1.5143692E0,1.1984736E1,4.733054E0,1.5434377E-1,5.559125E-1,8.444902E0,1.2862198E1,1.1785145E0,9.4893366E-2,5.68573E0,1.1239061E0,6.008167E-1,9.2534447E-1,1.18734E-1,0E0,3.7240067E0,4.5660114E0,2.1778032E-1,4.634006E-1,1.9618511E-1,3.033871E-1,1.4862757E0,0E0,8.0636847E-1,4.174304E-1,4.897806E0,1.5107145E1,2.0638561E-1,4.6349683E-1,8.7933846E-2,7.5514555E-2,0E0,2.0226504E-1,2.822958E0,2.3999698E0,6.4431915E0,5.1300335E0,3.118477E-1,0E0,0E0,0E0,2.1904297E0,2.2891998E-1,9.716804E-2,0E0,3.1764793E-1,2.1677017E-1,9.633608E-1,4.6353316E-1,0E0,0E0,1.0878506E0,1.9421806E0,5.700917E0,1.5538893E0,0E0,8.854995E-2,1.18914425E-1,1.1747503E-1,0E0,0E0,0E0,0E0,0E0,5.668819E-1,2.6358068E-1,1.1655827E-1,0E0,2.7126932E-1,0E0,4.44067E-1,8.930433E0,7.1144314E0,0E0,0E0,2.0493317E-1,9.091509E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5830412E0,8.415117E-1,0E0,3.1998914E-1,6.7083282E0,3.144436E0,1.5279528E0,2.041511E0,0E0,3.6905003E-1,1.919632E-1,5.072365E-1,0E0,0E0,0E0,0E0,3.6816835E-1,0E0,0E0,0E0,0E0,1.5495014E-1,1.492517E-1,9.317602E-2,7.2977066E-1,2.0312989E-1,1.330471E0,1.7426638E-1,2.0226908E0,4.39687E-1,1.0388432E0,3.8660485E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.0336234E-1,1.9652724E-1,3.2754815E-1,8.177419E-2,7.9974115E-2,0E0,0E0,8.956266E-2,0E0,0E0,3.4311604E0,2.3673534E0,4.917265E0,0E0,0E0,0E0,0E0,0E0,1.3687057E0,0E0,2.5904346E-1,2.2293186E-1,0E0,0E0,3.6759186E0,1.182766E0,4.7123718E-1,0E0,0E0,5.313817E-1,1.9109573E0,1.0973886E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,65,65,66,66,67,67,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,80,80,81,81,82,82,88,88,89,89,90,90,92,92,94,94,95,95,96,96,99,99,100,100,107,107,108,108,110,110,111,111,112,112,113,113,114,114,116,116,117,117,118,118,123,123,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,137,137,138,138,145,145,146,146,147,147,148,148,149,149,152,152,155,155,156,156,157,157,163,163,165,165,166,166,169,169,170,170,171,171,174,174,175,175,176,176],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,84,86,88,-1,90,92,94,96,98,100,102,104,-1,106,108,110,112,114,116,-1,-1,-1,118,120,122,-1,124,126,128,130,-1,-1,132,134,136,138,-1,140,142,144,-1,-1,-1,-1,-1,146,148,150,-1,152,-1,154,156,158,-1,-1,160,162,-1,-1,-1,-1,-1,-1,164,166,-1,168,170,172,174,176,-1,178,180,182,-1,-1,-1,-1,184,-1,-1,-1,-1,186,188,190,192,194,196,198,200,202,204,206,-1,-1,-1,-1,-1,-1,208,210,212,214,216,-1,-1,218,-1,-1,220,222,224,-1,-1,-1,-1,-1,226,-1,228,230,-1,-1,232,234,236,-1,-1,238,240,242,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.4166665E0,1.0950326E6,2.8636363E0,1.6990049E0,4.67E2,3.6E2,2.0304577E-1,1.1400756E6,1.3144558E4,1.1766373E3,7.364257E4,4.719849E7,2.1506184E7,6.308896E2,3.730007E6,4.888E3,1.4E1,6.025862E0,1E0,7.669686E-2,2.2783158E5,7.6247287E-1,2.0116506E4,4.278699E0,8E0,3.1998687E3,5.4674416E7,1.6213043E3,4.22088E0,3.5301748E7,1.096957E0,1.583E3,6.496696E7,1.84442E5,1.7325802E6,1.13E3,1.2570965E6,6.087709E-3,1.7232166E8,6.747114E7,7.5569354E2,3.426E3,8.229196E7,1.5552E4,3.79E2,-5.3433415E-2,8.314423E6,5.314985E3,1.3483871E2,2.3770695E5,7.937392E-3,2.6E1,5.85E2,1.9406E4,-2.6087303E-2,1.92E2,1.0407268E10,1.71E2,4.391553E6,6.6692764E1,1.1843003E0,4.5538284E-3,7.1883556E-3,-2.6838318E-3,1.1164689E0,2.1515152E0,1.9589581E6,-2.5815867E-2,1.5039847E5,9.536863E4,2.08E2,3.5E1,-5.803627E-4,-1.0649478E-2,1.42E1,7.804304E-1,1.8361508E3,3.0497742E3,-1.0392242E-2,4.020408E0,1.2165793E3,4.616371E8,4.460039E-2,2.5943302E-2,2.4219915E-4,1.7532287E-2,-2.3952365E-2,9.928469E-1,1.5288235E2,6.763314E7,-6.1383094E-3,1.838444E6,-1.4646217E-2,1.279012E7,9.750871E6,4.798624E3,-3.8648997E-2,-1.9661505E-2,1.17E2,2.53E2,-1.3496116E-3,7.5271134E-3,-8.146069E-3,1.0508824E-3,3.2215612E-3,-1.3129211E-2,1.3446785E7,2.2214102E6,-4.458116E-2,1.926994E1,6.082581E3,2.7565938E3,1.27365E4,2.3580047E10,-1.5219989E-2,2.308943E0,2.45659E5,2.5489312E5,-7.889809E-2,-3.0719984E-2,-5.2869217E-3,6.243996E-3,2.368E0,-3.805743E-2,-3.0936353E-2,-1.3075047E-2,2.8584866E-4,2.7627052E7,2.8050448E6,7.845511E-1,1.5043668E0,1E0,2.62888E-3,7.17E2,2.161716E8,8.518373E7,2.060792E8,2.310848E7,-2.8408393E-3,5.124669E-3,8.994676E-3,-8.974274E-4,2.2206E-2,5.9509412E-3,2.142857E0,1.14E2,3.1E1,2E0,3.6E2,-2.49221E-3,-1.9956382E-2,1.5782692E2,1.976785E-2,5.919638E-2,4.423E3,1.4777102E9,3.0820766E-1,7.2387956E-2,-1.2900008E-2,2.063919E-3,9.935169E-3,2.4056845E-4,1.4492002E0,-1.9641465E-2,1.829E3,3.5044186E2,6.5705115E-3,-8.04434E-3,9.74026E0,1.9840434E7,1.61225E5,-2.1674735E-3,-3.6092967E-2,6.015233E7,1.1803382E4,2.14099E5,-3.3610556E-2,-1.5008307E-2,-4.772963E-2,-3.1657178E-2,-7.0931576E-3,-3.0203981E-2,-2.6964169E-2,-3.9220667E-3,-1.8179469E-2,-3.0290915E-2,-1.6539589E-2,-5.214517E-3,-5.585641E-3,3.7414029E-3,-8.624645E-3,-2.8220035E-2,-4.7747223E-4,-1.2083269E-2,-1.372787E-2,-4.198654E-2,1.8646593E-3,-1.3689699E-2,-1.5061611E-2,-4.873595E-3,2.3735335E-2,5.3415843E-4,-9.219335E-3,-3.766032E-2,-8.923555E-4,-1.756385E-2,-7.587293E-3,1.0070175E-2,-8.391263E-3,-1.7097527E-2,-3.4497825E-3,-1.1662027E-2,2.6817133E-3,-5.246481E-3,-2.033823E-3,6.7679742E-3,-1.2043395E-2,-7.198268E-4,-1.0729425E-2,7.9328794E-4,6.645646E-3,2.4902904E-2,1.2548768E-2,-1.2573152E-2,1.9774145E-2,9.524917E-3,7.620296E-3,2.3350192E-2,1.6882332E-2,4.104005E-2,3.1302083E-2,1.368748E-2,5.3617436E-2,1.0948394E-2,3.2134358E-2,5.8167312E-2,-7.815204E-3,1.3788441E-2,1.4807812E-2,4.0254124E-2,-3.270904E-4,-1.2685684E-2],"split_indices":[19,55,46,53,40,2,28,26,32,51,51,27,44,44,51,9,2,3,53,15,0,27,56,32,53,17,4,7,4,52,44,41,2,44,28,47,2,32,0,7,44,4,2,31,9,1,0,5,49,51,27,41,8,2,1,0,0,30,0,27,55,53,0,0,0,38,52,31,0,31,27,2,3,0,0,57,38,51,51,0,53,4,7,0,0,0,0,0,52,51,7,0,9,0,7,9,4,0,0,3,3,0,0,0,0,0,0,44,27,0,57,51,4,32,5,0,53,11,32,0,0,0,0,57,0,0,0,0,5,47,26,52,109,26,0,12,43,7,31,0,0,0,0,0,0,53,28,10,8,2,0,0,4,0,0,2,7,26,0,0,0,0,0,52,0,2,4,0,0,52,9,1,0,0,31,4,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.188E3,7.09E2,1.479E3,3.29E2,3.8E2,9.11E2,5.68E2,1.99E2,1.3E2,3.17E2,6.3E1,1.94E2,7.17E2,3.7E1,5.31E2,4.6E1,1.53E2,1.09E2,2.1E1,3.12E2,5E0,3.7E1,2.6E1,8.5E1,1.09E2,6.76E2,4.1E1,1.5E1,2.2E1,1.69E2,3.62E2,3.9E1,7E0,1.32E2,2.1E1,4.7E1,6.2E1,1.7E1,4E0,1.05E2,2.07E2,1.3E1,2.4E1,1.7E1,9E0,8.1E1,4E0,7.2E1,3.7E1,1.5E1,6.61E2,2.2E1,1.9E1,9E0,6E0,1.7E1,5E0,1.48E2,2.1E1,3.12E2,5E1,3.7E1,2E0,2E0,5E0,1.17E2,1.5E1,4E0,1.7E1,3.5E1,1.2E1,3E1,3.2E1,3E0,1.4E1,4.7E1,5.8E1,1.75E2,3.2E1,3E0,1E1,1.3E1,1.1E1,6E0,1.1E1,3E0,6E0,4.6E1,3.5E1,5.2E1,2E1,1E1,2.7E1,2E0,1.3E1,3.1E2,3.51E2,1.7E1,5E0,1.1E1,8E0,3E0,6E0,3E0,3E0,2E0,3E0,1.15E2,3.3E1,3E0,1.8E1,2.72E2,4E1,1.5E1,3.5E1,6E0,3.1E1,9.7E1,2E1,1.3E1,2E0,2E0,2E0,1.5E1,2E1,2E0,1E1,3E0,2.7E1,2.5E1,7E0,2.9E1,1.8E1,5.3E1,5E0,1.62E2,1.3E1,2.1E1,1.1E1,4E0,6E0,1E1,3E0,9E0,2E0,6E0,2.9E1,4.2E1,1E1,1.6E1,4E0,2.2E1,5E0,2E0,1.1E1,2.24E2,8.6E1,3.48E2,3E0,9E0,2E0,2E0,6E0,1.13E2,2E0,1.8E1,1.5E1,3E0,1.5E1,2.39E2,3.3E1,3.8E1,2E0,3E0,1.2E1,3E1,5E0,2.6E1,5E0,8.7E1,1E1,3E0,1.7E1,1.3E1,2E0,1E1,1.7E1,2.1E1,4E0,4E0,3E0,6E0,2.3E1,4E0,1.4E1,5E0,4.8E1,2E0,3E0,7.6E1,8.6E1,1.1E1,2E0,4E0,1.7E1,5E0,6E0,3E0,3E0,1.3E1,1.6E1,1.9E1,2.3E1,7E0,3E0,2E0,1.4E1,2E0,3E0,1.39E2,8.5E1,6.8E1,1.8E1,3.32E2,1.6E1,5.5E1,5.8E1,4E0,1.4E1,3E0,1.2E1,2.09E2,3E1,3.1E1,2E0,6E0,3.2E1,1E1,2E0,2.1E1,9E0,3E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"243","size_leaf_vector":"1"}},{"base_weights":[7.888707E-3,-4.1342452E-1,2.1346398E-1,-6.4452255E-1,-1.336011E-1,3.3031657E-2,5.1955307E-1,-7.641993E-1,-9.960723E-2,-3.4288782E-1,2.1853793E-1,-3.5713884E-1,9.601477E-2,3.7103094E-2,6.04492E-1,-2.956965E-1,-8.4374213E-1,1.4344728E0,-2.3350242E-1,-4.544959E-1,-2.3120368E-1,9.199171E-2,6.697598E-1,-2.4044244E-2,-2.979052E-1,2.4239558E-1,-7.147575E-2,2.6815897E-1,-2.6364785E-1,3.7827116E-1,7.5610656E-1,-5.545441E-1,2.1024497E-2,-7.8577566E-1,-1.3353066E0,7.774144E-2,2.623325E-2,4.3915883E-2,-4.96323E-1,-4.8050398E-1,-8.5586876E-2,-2.7088252E-1,-5.7699725E-2,-1.3442825E-1,2.0380788E-1,1.0524529E-2,7.357043E-1,-3.3575457E-1,-1.9839819E-1,1.120728E0,2.0600697E-1,-8.624798E-2,6.0439963E-2,-3.629981E-3,4.4635278E-1,-1.2637958E-1,-2.29695E-2,6.968041E-1,3.0485126E-1,4.828219E-1,8.8027066E-1,1.7389223E-3,-6.035167E-1,2.7338145E-2,-6.323579E-2,-8.8127315E-1,-5.197276E-1,-7.4118294E-2,-7.9648083E-1,-3.918832E-2,4.3826687E-1,-3.9142492E-1,-4.742113E-2,-5.370165E-1,-3.2408005E-1,2.3102653E-3,-8.047981E-3,-3.180283E-1,-1.09810844E-1,-1.7285997E-1,1.5104318E-1,-2.1489137E-1,7.212608E-2,6.581886E-2,3.169704E-1,1.4651772E-2,3.7676603E-2,-3.5034963E-1,-1.6751341E-4,-1.0907384E-1,-1.4636852E-2,3.2171257E-2,6.052063E-2,4.142017E-3,3.408362E-1,-1.3484815E-1,2.5682223E-1,2.8894976E-1,-2.2276229E-1,3.2194832E-1,3.1104788E-2,-2.3609406E-1,9.490103E-2,1.3042518E-2,8.272055E-1,2.1310608E-1,5.0874954E-1,3.6815995E-1,7.509068E-1,9.2493606E-1,1.3321167E-1,-6.469551E-1,-9.676243E-3,-1.6634665E-2,7.874348E-3,-7.608828E-1,-1.0499495E0,-2.0250459E-1,-6.450736E-1,-9.208833E-3,-4.0509146E-2,1.562436E-2,-1.1447987E-1,2.4877084E-2,6.5146997E-3,-5.257E-1,-1.6506721E-1,-1.9990627E-1,-5.602297E-1,-7.3031907E-4,-3.603887E-1,-3.94842E-1,-2.2407496E-1,-1.2389337E-2,-3.77735E-4,4.3851407E-3,-1.0226681E-2,1.3016439E-3,1.2049586E-2,-2.8971538E-1,2.6198458E-3,-6.274877E-3,6.4348695E-3,-8.417943E-2,1.5252689E-1,2.3501972E-2,2.072666E-1,-2.7268633E-1,-4.044868E-1,-1.2182036E-2,-2.194379E-3,-5.38977E-2,2.432105E-1,1.7242494E-1,4.7848618E-1,-3.978236E-1,-6.884351E-2,8.078157E-2,4.370046E-1,1.8500313E-2,-3.9217076E-3,5.6909616E-3,-3.0502492E-1,-4.2817895E-3,3.7077054E-1,-1.2590892E-2,-1.3597459E-3,7.0720054E-3,-3.611713E-3,3.9253805E-2,1.054367E-2,2.4402669E-1,-1.2050266E-1,6.6389096E-1,1.23179056E-1,4.3704137E-1,1.479629E-1,8.3494127E-1,-7.7722147E-3,8.521738E-1,5.92359E-2,2.3725137E-2,-2.503147E-3,-3.1354167E-2,-1.2163877E-2,-1.959269E-3,4.2754686E-3,-4.017495E-2,-2.5482612E-2,-4.9075037E-2,-1.6472865E-2,-1.9590848E-3,-1.4411749E-2,-3.2506593E-2,-2.0363895E-2,-1.2713655E-2,-8.912348E-4,-1.5210296E-2,-2.8665189E-2,-9.631998E-3,2.9332682E-3,-1.5852645E-2,3.035766E-3,-2.7406653E-2,-1.5241386E-2,-2.4649767E-2,-1.3555078E-2,-1.8583508E-2,-4.13539E-3,-7.622553E-3,-1.6758554E-2,-9.992767E-3,-2.6445864E-2,1.38741E-3,-7.722583E-3,-3.3736385E-3,1.1551812E-2,1.4787203E-3,1.3016821E-2,-1.3288227E-2,8.8179845E-4,-2.1249037E-2,-9.173647E-3,-6.301938E-3,5.5092513E-3,-6.177466E-3,1.2935872E-2,3.480308E-3,1.412573E-2,1.7928528E-2,3.1642098E-2,-2.5857411E-2,-5.719722E-3,-7.1564536E-3,4.685964E-3,-5.81782E-3,6.9556446E-3,2.2803795E-2,9.626961E-3,-2.6470544E-2,-4.6020932E-3,8.377263E-3,2.0022912E-2,1.1777258E-2,-1.6698427E-2,-1.7423717E-2,-2.8564475E-4,1.6279366E-2,3.5938453E-2,-2.2360266E-3,8.621106E-3,1.2974781E-2,2.4432058E-2,7.821633E-4,1.3257383E-2,1.0933251E-3,4.108846E-2,4.6153E-2,3.2748926E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,-1,97,99,101,-1,103,105,107,109,-1,111,-1,113,115,117,-1,119,121,123,125,-1,127,129,-1,-1,131,133,135,137,139,141,143,145,-1,-1,147,-1,149,-1,-1,-1,151,153,155,157,159,161,163,-1,165,167,-1,169,171,173,175,177,179,181,183,-1,-1,185,187,189,191,193,-1,-1,-1,195,-1,-1,197,199,201,203,-1,205,207,209,-1,-1,-1,-1,-1,-1,211,-1,-1,-1,213,215,-1,217,219,221,-1,-1,223,225,227,229,231,233,235,237,-1,-1,-1,239,-1,241,-1,-1,-1,-1,-1,-1,243,245,247,249,251,253,255,-1,257,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9253918E2,4.7055862E1,8.2494E1,2.5944138E1,2.4468979E1,2.3158567E1,2.2679825E1,1.1936508E1,1.5329587E1,2.4876099E0,7.0188403E0,1.1915302E0,1.9900934E1,5.9078827E0,1.5868515E1,4.0288005E0,6.900635E0,4.551525E-1,4.979761E0,9.4711876E-1,7.141204E-1,2.5085037E0,6.0758305E-1,0E0,3.1001568E-1,1.3608362E1,7.9894485E0,2.3263137E0,1.1736E0,4.254431E0,9.109909E0,8.0721855E-1,1.1676695E0,5.9445343E0,3.2493706E0,0E0,0E0,1.1314306E0,1.6133499E0,6.745968E-1,1.0981116E-1,6.1814785E-1,5.309259E-1,5.639596E-1,1.0097399E0,0E0,6.451731E-1,3.4279346E-1,2.9942727E-1,5.5639076E-1,1.13482E1,6.3037796E0,0E0,1.3466406E0,6.471262E-1,6.293477E-1,0E0,1.6271362E0,2.8285303E0,2.4900684E0,6.2834015E0,0E0,2.850256E-1,0E0,4.5713907E-1,2.9375916E0,2.6129913E0,0E0,5.1742506E-1,8.785995E-1,1.1735904E-1,8.5544395E-1,0E0,4.2795944E-1,3.072605E-1,0E0,0E0,3.8784122E-1,3.468339E-1,2.2210181E-1,1.0496038E-1,5.065769E-1,1.6721815E-1,4.1602737E-1,6.907072E-1,0E0,0E0,1.7560673E-1,0E0,1.711973E-1,0E0,0E0,0E0,2.3441868E0,5.7129498E0,5.7006207E0,1.4555111E0,4.3993407E-1,3.8232744E-1,4.6211314E-1,0E0,1.3881141E-1,1.0883559E-1,0E0,4.134388E-1,1.1640983E0,2.789485E0,9.212208E-1,2.159071E0,3.9350739E0,9.130185E-1,1.93326E-1,0E0,0E0,8.378397E-2,2.280983E0,1.3322372E0,3.7351006E-1,3.748436E-1,0E0,0E0,0E0,3.8866612E-1,0E0,0E0,1.7464638E-1,1.5626422E-1,2.641316E-1,4.4186592E-1,0E0,1.17636204E-1,1.2849522E-1,1.9810426E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.5703585E-1,0E0,0E0,0E0,1.2427022E-1,4.8502666E-1,0E0,3.408922E-1,1.9429207E-1,4.023981E-1,0E0,0E0,2.0538132E0,5.716293E-1,1.4964707E0,2.2140484E0,3.0332232E0,4.0914845E0,4.0091705E-1,2.369957E-1,0E0,0E0,0E0,5.09306E-1,0E0,1.7045498E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0929165E0,2.994401E-1,1.0587053E0,1.834614E-1,6.0181427E-1,3.167793E-1,1.391016E0,0E0,2.6020432E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,57,57,58,58,59,59,60,60,62,62,64,64,65,65,66,66,68,68,69,69,70,70,71,71,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,87,87,89,89,93,93,94,94,95,95,96,96,97,97,98,98,99,99,101,101,102,102,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,114,114,115,115,116,116,117,117,118,118,122,122,125,125,126,126,127,127,128,128,130,130,131,131,132,132,139,139,143,143,144,144,146,146,147,147,148,148,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,162,162,164,164,171,171,172,172,173,173,174,174,175,175,176,176,177,177,179,179],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,-1,98,100,102,-1,104,106,108,110,-1,112,-1,114,116,118,-1,120,122,124,126,-1,128,130,-1,-1,132,134,136,138,140,142,144,146,-1,-1,148,-1,150,-1,-1,-1,152,154,156,158,160,162,164,-1,166,168,-1,170,172,174,176,178,180,182,184,-1,-1,186,188,190,192,194,-1,-1,-1,196,-1,-1,198,200,202,204,-1,206,208,210,-1,-1,-1,-1,-1,-1,212,-1,-1,-1,214,216,-1,218,220,222,-1,-1,224,226,228,230,232,234,236,238,-1,-1,-1,240,-1,242,-1,-1,-1,-1,-1,-1,244,246,248,250,252,254,256,-1,258,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.896243E4,2.9799202E3,4.5479352E2,3.5E2,2.5998926E-1,8.2608955E-3,2.1647402E3,1.913925E-1,1.4101E4,4.1E1,2E1,2.7246006E7,1.1410706E3,2.695E3,1.0017953E8,1.4E1,7.76264E7,8.982707E9,1.1400756E6,3.4047124E-1,2.1924436E5,-2.4044244E-2,2.0407547E5,2.04E5,6.7652373E3,1.8053533E8,9.302862E7,8E0,2.3421426E-1,1.21E2,2.66E2,3.83E3,1.1852991E1,7.774144E-2,2.623325E-2,4.5575E4,9.22E4,1.5345133E1,6.822312E4,9.36494E5,5.7082694E5,2.262586E6,3.0990322E5,1.0524529E-2,1.8061392E0,7.997723E6,1.4504054E8,1.287E3,4.3376322E5,2.89E2,6.0439963E-2,2.783636E6,6.9998717E-4,3.524269E6,-2.29695E-2,1.7605528E7,3.794679E8,6.4683E4,4.5481584E7,1.7389223E-3,7.221312E-2,2.7338145E-2,1.447E3,8.06E2,2.7246006E7,-7.4118294E-2,1.5683041E3,2.6713815E-1,1.671E3,3.7002478E3,-4.742113E-2,2.2539302E6,2.09633E1,2.3102653E-3,-8.047981E-3,2.9E1,5.101512E0,1.2570965E6,2.0699982E8,1E0,1.0800328E8,1.6018981E0,4.9716983E0,1.4651772E-2,3.7676603E-2,2.00059E5,-1.6751341E-4,1.21E2,-1.4636852E-2,3.2171257E-2,6.052063E-2,1.646E3,9.27E2,1.2068E4,7.0764465E1,3.3502305E0,2.34E0,3.0324321E-2,3.1104788E-2,3.9E1,4.987639E7,1.3042518E-2,1.1399403E3,3.0927835E-2,6.075589E6,2.589369E7,5.68E2,7.096131E0,3.2E2,7.5E1,-9.676243E-3,-1.6634665E-2,1.5851064E0,1.146E3,7.471519E2,1.504779E7,4.7272725E0,-9.208833E-3,-4.0509146E-2,1.562436E-2,3.2343243E1,2.4877084E-2,6.5146997E-3,1.8E1,1.5005797E4,1E0,2.41E3,-7.3031907E-4,1.06E3,3.454E3,1.945711E1,-1.2389337E-2,-3.77735E-4,4.3851407E-3,-1.0226681E-2,1.3016439E-3,1.2049586E-2,2.7864855E11,2.6198458E-3,-6.274877E-3,6.4348695E-3,3.80458E5,3.564E3,2.3501972E-2,6.860185E2,8.1445E1,1.654762E2,-1.2182036E-2,-2.194379E-3,3.422351E6,6.884212E4,2.657819E6,2.9251662E6,7.427971E2,1.5497989E5,3.647E3,2.1387602E-3,1.8500313E-2,-3.9217076E-3,5.6909616E-3,1E0,-4.2817895E-3,2.001177E6,-1.2590892E-2,-1.3597459E-3,7.0720054E-3,-3.611713E-3,3.9253805E-2,1.054367E-2,1.0485785E3,1.83E2,1.3E1,4.646E3,1.00663277E9,7.41688E-2,1.2E1,-7.7722147E-3,2E0,5.92359E-2,2.3725137E-2,-2.503147E-3,-3.1354167E-2,-1.2163877E-2,-1.959269E-3,4.2754686E-3,-4.017495E-2,-2.5482612E-2,-4.9075037E-2,-1.6472865E-2,-1.9590848E-3,-1.4411749E-2,-3.2506593E-2,-2.0363895E-2,-1.2713655E-2,-8.912348E-4,-1.5210296E-2,-2.8665189E-2,-9.631998E-3,2.9332682E-3,-1.5852645E-2,3.035766E-3,-2.7406653E-2,-1.5241386E-2,-2.4649767E-2,-1.3555078E-2,-1.8583508E-2,-4.13539E-3,-7.622553E-3,-1.6758554E-2,-9.992767E-3,-2.6445864E-2,1.38741E-3,-7.722583E-3,-3.3736385E-3,1.1551812E-2,1.4787203E-3,1.3016821E-2,-1.3288227E-2,8.8179845E-4,-2.1249037E-2,-9.173647E-3,-6.301938E-3,5.5092513E-3,-6.177466E-3,1.2935872E-2,3.480308E-3,1.412573E-2,1.7928528E-2,3.1642098E-2,-2.5857411E-2,-5.719722E-3,-7.1564536E-3,4.685964E-3,-5.81782E-3,6.9556446E-3,2.2803795E-2,9.626961E-3,-2.6470544E-2,-4.6020932E-3,8.377263E-3,2.0022912E-2,1.1777258E-2,-1.6698427E-2,-1.7423717E-2,-2.8564475E-4,1.6279366E-2,3.5938453E-2,-2.2360266E-3,8.621106E-3,1.2974781E-2,2.4432058E-2,7.821633E-4,1.3257383E-2,1.0933251E-3,4.108846E-2,4.6153E-2,3.2748926E-2],"split_indices":[19,109,32,4,51,2,41,26,32,40,2,2,3,44,51,2,44,0,44,5,32,40,27,0,27,5,51,7,44,3,37,2,0,2,55,0,0,2,10,57,32,9,46,1,27,0,41,9,30,2,27,8,0,12,33,1,0,12,7,2,46,0,37,0,0,0,44,0,4,56,0,51,0,44,57,0,0,3,55,32,12,94,7,40,53,0,0,5,0,2,0,0,0,2,2,9,55,52,53,37,0,3,44,0,4,56,31,44,3,34,0,3,0,0,52,2,51,44,53,0,0,0,55,0,0,3,4,8,2,0,2,0,55,0,0,0,0,0,0,30,0,0,0,28,2,0,51,51,4,0,0,9,27,27,27,32,27,2,38,0,0,0,8,0,9,0,0,0,0,0,0,51,0,3,2,7,37,17,0,16,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.221E3,7.28E2,1.493E3,3.98E2,3.3E2,9.4E2,5.53E2,3.26E2,7.2E1,2.07E2,1.23E2,1.3E2,8.1E2,8.3E1,4.7E2,4.8E1,2.78E2,5E0,6.7E1,1.02E2,1.05E2,9.7E1,2.6E1,3.1E1,9.9E1,4.32E2,3.78E2,4.7E1,3.6E1,1.9E2,2.8E2,2.6E1,2.2E1,2.51E2,2.7E1,3E0,2E0,3.3E1,3.4E1,9.5E1,7E0,8.5E1,2E1,3.2E1,6.5E1,4E0,2.2E1,7E1,2.9E1,1.6E1,4.16E2,3.75E2,3E0,1.9E1,2.8E1,2.4E1,1.2E1,3.4E1,1.56E2,8.9E1,1.91E2,2E0,2.4E1,2E0,2E1,1.83E2,6.8E1,1.6E1,1.1E1,2.8E1,5E0,3E1,4E0,6.8E1,2.7E1,3E0,4E0,6.5E1,2E1,1.3E1,7E0,2.3E1,9E0,3E1,3.5E1,5E0,1.7E1,6.7E1,3E0,1.8E1,1.1E1,7E0,9E0,1.67E2,2.49E2,3.29E2,4.6E1,8E0,1.1E1,2E1,8E0,1.6E1,8E0,9E0,2.5E1,1.09E2,4.7E1,6.4E1,2.5E1,1.8E2,1.1E1,2.1E1,3E0,3E0,1.7E1,1.1E2,7.3E1,2E1,4.8E1,2E0,9E0,4E0,2.4E1,3E0,2E0,1.8E1,1.2E1,5E0,6.3E1,3E0,2.4E1,3.4E1,3.1E1,7E0,1.3E1,2E0,1.1E1,4E0,3E0,1.8E1,5E0,2E0,7E0,1.1E1,1.9E1,1.1E1,2.4E1,3E1,3.7E1,4E0,1.4E1,1.35E2,3.2E1,1.13E2,1.36E2,6.5E1,2.64E2,2.4E1,2.2E1,6E0,2E0,2E0,9E0,2E0,1.8E1,1.3E1,3E0,6E0,2E0,2.3E1,2E0,1E2,9E0,3.3E1,1.4E1,4.8E1,1.6E1,2.3E1,2E0,1.54E2,2.6E1,3E0,8E0,1.8E1,3E0,1.1E1,6E0,6.5E1,4.5E1,6.9E1,4E0,9E0,1.1E1,3.3E1,1.5E1,8E0,1.6E1,8E0,1E1,1E1,2E0,3E0,2E0,5.1E1,1.2E1,4E0,2E1,3.2E1,2E0,2.4E1,7E0,1.6E1,2E0,5E0,6E0,6E0,1.3E1,8E0,1.6E1,2.8E1,2E0,2.7E1,1E1,9.1E1,4.4E1,3E0,2.9E1,6.8E1,4.5E1,1.01E2,3.5E1,3.9E1,2.6E1,1.74E2,9E1,6E0,1.8E1,1.6E1,6E0,3E0,6E0,6E0,1.2E1,9.8E1,2E0,2E0,7E0,1.1E1,2.2E1,4E0,1E1,2.1E1,2.7E1,9E0,7E0,2E0,2.1E1,6.4E1,9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"259","size_leaf_vector":"1"}},{"base_weights":[2.8116235E-3,-4.010859E-1,1.9643351E-1,-6.2831384E-1,-1.6634852E-1,3.5625793E-2,5.4152364E-1,-7.6357394E-1,-3.3022255E-1,-2.1286736E-1,1.1849309E0,-1.2619093E-1,2.2691303E-1,4.4369742E-1,9.700298E-1,-8.2473904E-1,-3.9902392E-1,-4.0352288E-1,-9.6407056E-2,-3.421476E-1,2.4921854E-1,7.653088E-2,2.6367532E-2,-4.4965562E-1,-5.7004564E-2,1.632127E-2,3.630779E-1,2.9889697E-1,6.465394E-1,1.0182608E0,1.4116168E-1,-7.564443E-1,-1.1764027E0,-7.9538584E-2,-6.039257E-1,-4.232564E-1,3.9411846E-4,-2.1734302E-1,6.915591E-2,-5.7715774E-1,-2.2534512E-1,1.7751251E-1,6.1685795E-1,-4.732967E-1,2.0678641E-2,-9.603002E-2,2.321107E-1,-1.9208172E-1,1.2462591E-1,4.341835E-1,-2.3582888E-1,3.593836E-1,-2.893124E-1,6.952883E-1,-3.787877E-1,1.0608379E0,2.9504886E-1,1.8735811E-2,-4.465438E-3,-8.124517E-1,-4.5742568E-1,-1.2263701E0,-2.2978641E-2,2.4809048E-1,-2.5101456E-1,-8.975754E-3,-7.055981E-1,-3.4381378E-1,-5.103761E-1,-1.2002531E-2,4.2527863E-3,1.4941412E-1,-6.7357956E-3,-3.1760293E-1,-7.922441E-1,-1.649197E-1,-5.034355E-1,1.1647028E-2,2.950578E-1,1.2891998E-2,3.6825594E-2,-3.9387906E-1,-7.082363E-1,-1.485204E-1,1.7662527E-1,5.984821E-2,4.564139E-2,2.1266775E-1,-2.8009892E-1,2.980733E-2,2.67213E-1,4.0502107E-1,7.161351E-2,4.6195097E-2,-5.397341E-1,2.1727571E-1,5.181255E-1,-2.0708095E-2,-1.3776296E-1,5.66134E-1,9.0741086E-1,-2.5880162E-3,-3.106341E-2,2.5525654E-2,1.1095874E0,1.911577E-2,5.700213E-3,-9.0426874E-1,-6.070596E-1,-1.8304399E-1,-5.6135255E-1,-9.706918E-1,-6.561276E-2,2.0434698E-2,-4.7821943E-3,-1.5152877E-2,-9.4640243E-4,-3.575417E-2,-1.48910815E-2,-4.3485087E-1,-2.5355878E-1,-5.3766865E-1,-5.7676923E-4,1.0747867E-2,6.3169416E-4,-4.8396537E-1,-1.4436592E-1,-8.602023E-1,-2.713168E-1,-2.2716813E-1,2.541659E-1,-7.0708823E-1,-2.2548991E-1,2.1362868E-1,-9.527211E-2,3.5827182E-2,3.8636062E-1,-5.2045536E-1,-2.015166E-1,-4.1933384E-2,-4.6931353E-1,-2.769136E-1,-8.28211E-2,6.1872184E-1,1.2210358E-1,2.655711E-1,-8.353203E-2,1.8569872E-3,1.2012417E-2,-2.0812535E-1,-7.23336E-1,6.38391E-2,-2.268522E-1,3.1168997E-1,3.0712783E-4,3.2888183E-1,5.9719706E-1,1.8728213E-1,-2.078791E-2,-3.263316E-3,-2.744219E-2,3.55921E-2,2.774132E-1,2.530827E-1,6.48792E-1,3.0421207E-3,-9.325043E-3,6.6168034E-1,3.3520395E-1,6.074789E-2,7.617604E-1,5.341036E-2,8.414566E-1,-4.17221E-2,-1.726263E-2,-2.0212356E-2,-3.341886E-2,5.9166044E-4,-1.2176841E-2,-3.2248935E-3,-2.874856E-2,-4.9138904E-2,-2.6238842E-2,-7.909164E-3,-2.1130867E-2,-1.9349369E-3,-1.2640437E-2,-2.8077517E-2,-1.6467348E-2,-2.8332174E-2,-1.4734148E-2,9.248894E-3,-8.576951E-3,-2.5871139E-2,-4.222179E-2,-1.8371306E-2,1.957153E-3,-1.5621888E-2,-4.873862E-3,1.7009526E-2,-8.402111E-3,-4.3923352E-2,-1.52216265E-2,4.8871705E-4,-1.4143259E-2,6.462033E-4,1.3447618E-2,-1.3915476E-2,-1.6432276E-3,4.409514E-3,-4.3584225E-3,2.7462017E-2,1.413069E-2,-2.4364416E-2,-9.458497E-3,4.430771E-3,-1.25536965E-2,-4.1379044E-3,-2.398237E-2,4.753214E-3,-1.4228926E-2,-6.56451E-3,1.8191313E-3,1.1164964E-2,3.25178E-2,2.6914736E-3,2.1303339E-2,1.6344002E-2,-2.4787816E-3,-1.2179285E-2,1.4457427E-3,-1.141334E-2,6.0825786E-3,-1.4635561E-2,-4.267392E-2,9.0012356E-4,8.257803E-3,-1.3080797E-2,-2.8962415E-4,7.401057E-3,1.8378453E-2,9.742953E-3,-1.9448826E-2,8.292334E-3,1.8066453E-2,2.217111E-2,3.5440862E-2,5.049882E-3,1.3641441E-2,-6.6297813E-3,5.651413E-3,9.888177E-3,1.8468602E-2,1.71909E-2,-6.3766195E-3,3.5100456E-2,1.81271E-2,2.2649352E-2,3.359413E-2,1.7838156E-2,-2.234799E-3,3.6553692E-2,-6.6177077E-3,4.0071126E-2,1.4863698E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,99,101,103,105,-1,-1,107,109,111,-1,113,115,-1,117,119,121,-1,-1,123,-1,125,127,129,131,133,135,-1,-1,137,139,141,143,-1,145,147,149,151,153,155,-1,157,159,161,163,-1,165,167,169,-1,-1,-1,171,-1,-1,173,175,177,179,181,-1,-1,-1,-1,-1,-1,-1,183,185,187,-1,-1,-1,189,191,193,195,197,199,201,203,205,207,209,211,213,215,-1,217,219,221,223,225,227,229,-1,-1,231,233,235,237,239,241,243,245,247,-1,-1,-1,249,251,253,255,-1,-1,257,259,-1,261,-1,263,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7400316E2,3.835282E1,8.344242E1,1.4504929E1,2.2630386E1,3.1817854E1,1.9606949E1,5.267639E0,1.949192E0,2.0695078E1,2.6702118E0,1.2439663E1,1.3500093E1,1.1291054E1,3.3742523E0,4.249695E0,2.4395633E0,7.1953964E-1,5.956835E-1,7.2813797E0,1.8940091E0,0E0,0E0,2.3161488E0,5.2124023E0,4.222146E0,1.2281883E1,8.2871E0,8.36071E0,2.2427673E0,4.331239E-1,2.6763687E0,4.804535E-1,9.6116257E-1,8.0876064E-1,4.0589237E-1,0E0,2.77667E-1,2.496131E-1,4.716957E0,2.9752522E0,1.2638493E0,5.516267E-1,1.5122433E0,0E0,5.836814E0,1.1116471E1,2.339563E0,1.6549835E0,7.926838E0,2.681899E0,4.6021442E0,4.9739122E-1,3.7327423E0,7.515458E-1,1.2643585E0,7.59058E-2,0E0,0E0,2.3175583E0,7.8123236E-1,3.027153E-1,0E0,4.790698E-1,2.0481211E-1,0E0,3.1069183E-1,2.7591753E-1,5.092592E-1,0E0,0E0,1.158696E-1,0E0,1.1405773E0,1.4658756E0,3.9969387E0,1.6518993E0,6.250326E-1,8.855603E-1,0E0,0E0,1.6969099E0,7.631531E-1,2.8535175E0,1.5153062E0,0E0,1.3858192E0,9.7586155E-2,1.5402269E0,6.6957533E-1,5.815921E-1,3.4762611E0,0E0,1.298072E0,4.2211008E-1,1.2138381E0,3.229763E0,0E0,1.9557127E-1,1.9625912E0,2.5840836E0,0E0,0E0,0E0,3.23555E-1,0E0,0E0,5.793686E-1,7.231865E-1,1.7200512E-1,7.487917E-1,1.3348389E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0094118E-1,1.2860954E-1,2.910385E-1,0E0,0E0,0E0,2.6847887E-1,3.8447934E-1,3.4495354E-1,3.136077E-1,1.8226027E0,1.138636E0,1.4300833E0,3.1137973E-1,1.6202107E-1,2.2650357E-1,9.8838754E-2,2.6847458E-1,1.3019562E-1,7.303134E-1,0E0,2.483418E-1,1.6743011E0,1.7418149E0,8.211136E-2,1.2819142E0,5.700021E-1,6.725056E-1,0E0,0E0,7.4162245E-1,3.09263E-1,3.455419E-1,1.2131095E-1,5.2035E-1,8.314848E-1,1.7878094E0,1.0132141E0,8.730367E-2,0E0,0E0,0E0,4.8933667E-1,5.578065E-1,1.7494481E0,1.7320652E0,0E0,0E0,5.2352524E-1,7.204454E-1,0E0,1.9822617E0,0E0,1.123724E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,63,63,64,64,66,66,67,67,68,68,71,71,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,84,84,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,96,96,98,98,99,99,100,100,104,104,107,107,108,108,109,109,110,110,111,111,119,119,120,120,121,121,125,125,126,126,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,137,137,138,138,140,140,141,141,142,142,143,143,144,144,145,145,146,146,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,161,161,162,162,163,163,164,164,167,167,168,168,170,170,172,172],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,100,102,104,106,-1,-1,108,110,112,-1,114,116,-1,118,120,122,-1,-1,124,-1,126,128,130,132,134,136,-1,-1,138,140,142,144,-1,146,148,150,152,154,156,-1,158,160,162,164,-1,166,168,170,-1,-1,-1,172,-1,-1,174,176,178,180,182,-1,-1,-1,-1,-1,-1,-1,184,186,188,-1,-1,-1,190,192,194,196,198,200,202,204,206,208,210,212,214,216,-1,218,220,222,224,226,228,230,-1,-1,232,234,236,238,240,242,244,246,248,-1,-1,-1,250,252,254,256,-1,-1,258,260,-1,262,-1,264,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.126489E7,1E0,1.3144558E4,3.2627738E5,2.9251662E6,5.121E3,1.294364E0,1.4722673E0,1.5151515E-3,9.6E1,8.15E2,1.2679E4,3.130603E3,4.8297736E7,5.9767612E7,3.771981E6,1.752512E2,1E0,4.080463E-1,7.653088E-2,2.6367532E-2,1.6594656E3,1.1598511E3,1E0,7.325843E0,3.9712732E7,1.2367184E3,1E0,3.01E2,1.5E1,5.159139E8,1.6666666E0,4.3161097E5,5.8084745E6,3.9411846E-4,1E0,4.5576923E1,1.7232166E8,7.76264E7,1.5190727E5,2.1924436E5,1.4504054E8,2.0678641E-2,2.6608378E8,1E0,4.144144E-1,4.624343E6,8.838E3,7.5179994E-1,5.5814706E5,1.6347875E3,7.9023E4,1.7427321E-1,4.6253732E2,2.84E2,1.8735811E-2,-4.465438E-3,1.593E2,3.2801925E6,3.1790106E8,-2.2978641E-2,9E0,1.0409E4,-8.975754E-3,1.0630361E8,1.09E3,4.0465977E1,-1.2002531E-2,4.2527863E-3,1.85221E5,-6.7357956E-3,1.221875E1,7.0231044E-1,2.0601392E3,3.0497742E3,7.11E2,2.4042394E5,1.2891998E-2,3.6825594E-2,5.222222E1,3.3820656E7,7.805608E4,2.53383E5,5.984821E-2,1.8071064E7,1.1251919E6,3.5612745E0,3.4782608E0,1.3569831E7,2.2515285E6,7.161351E-2,2.7183437E1,3.3265974E7,5.202E3,3.564E3,-2.0708095E-2,2.028994E8,2.545455E7,3.4231E4,-2.5880162E-3,-3.106341E-2,2.5525654E-2,2.2579439E0,1.911577E-2,5.700213E-3,4.22088E0,1.775044E7,2.08E2,1.3989231E5,1.918E3,-6.561276E-2,2.0434698E-2,-4.7821943E-3,-1.5152877E-2,-9.4640243E-4,-3.575417E-2,-1.48910815E-2,1.54E2,1.59E2,4.9114623E0,-5.7676923E-4,1.0747867E-2,6.3169416E-4,1.2599119E-3,5.0461445E-2,5.3421542E-2,3.677609E-1,2.6847225E-1,1.30307E0,1.333145E6,1.1855755E8,7.5569354E2,1.0771E5,2.8E1,1.3841E4,6.6875E-1,4E0,-4.1933384E-2,1.2E1,9.928469E-1,1.645614E6,1.440286E7,1.4777102E9,2.2333652E1,6.624E3,1.8569872E-3,1.2012417E-2,1.2673605E7,4.9E1,6.87E2,3.363849E2,3.136758E9,4.9E1,5.1586456E7,1.9121015E0,2.2700515E-3,-2.078791E-2,-3.263316E-3,-2.744219E-2,7.704183E-1,1.29E3,1.0963991E7,4.4E0,3.0421207E-3,-9.325043E-3,1.5166431E5,3.66221E-1,6.074789E-2,4.4543375E4,5.341036E-2,1.7997152E7,-4.17221E-2,-1.726263E-2,-2.0212356E-2,-3.341886E-2,5.9166044E-4,-1.2176841E-2,-3.2248935E-3,-2.874856E-2,-4.9138904E-2,-2.6238842E-2,-7.909164E-3,-2.1130867E-2,-1.9349369E-3,-1.2640437E-2,-2.8077517E-2,-1.6467348E-2,-2.8332174E-2,-1.4734148E-2,9.248894E-3,-8.576951E-3,-2.5871139E-2,-4.222179E-2,-1.8371306E-2,1.957153E-3,-1.5621888E-2,-4.873862E-3,1.7009526E-2,-8.402111E-3,-4.3923352E-2,-1.52216265E-2,4.8871705E-4,-1.4143259E-2,6.462033E-4,1.3447618E-2,-1.3915476E-2,-1.6432276E-3,4.409514E-3,-4.3584225E-3,2.7462017E-2,1.413069E-2,-2.4364416E-2,-9.458497E-3,4.430771E-3,-1.25536965E-2,-4.1379044E-3,-2.398237E-2,4.753214E-3,-1.4228926E-2,-6.56451E-3,1.8191313E-3,1.1164964E-2,3.25178E-2,2.6914736E-3,2.1303339E-2,1.6344002E-2,-2.4787816E-3,-1.2179285E-2,1.4457427E-3,-1.141334E-2,6.0825786E-3,-1.4635561E-2,-4.267392E-2,9.0012356E-4,8.257803E-3,-1.3080797E-2,-2.8962415E-4,7.401057E-3,1.8378453E-2,9.742953E-3,-1.9448826E-2,8.292334E-3,1.8066453E-2,2.217111E-2,3.5440862E-2,5.049882E-3,1.3641441E-2,-6.6297813E-3,5.651413E-3,9.888177E-3,1.8468602E-2,1.71909E-2,-6.3766195E-3,3.5100456E-2,1.81271E-2,2.2649352E-2,3.359413E-2,1.7838156E-2,-2.234799E-3,3.6553692E-2,-6.6177077E-3,4.0071126E-2,1.4863698E-2],"split_indices":[19,55,12,109,51,27,27,2,41,41,56,28,2,2,4,44,44,1,51,15,33,0,0,4,51,109,52,44,57,6,0,3,7,52,31,44,0,8,57,7,44,32,27,30,0,7,8,52,44,2,26,27,4,2,37,4,0,0,0,51,44,7,0,3,2,0,49,2,57,0,0,9,0,57,40,51,51,0,27,0,0,4,7,27,9,0,44,27,52,55,44,27,0,52,5,2,2,0,7,44,10,0,0,0,53,0,0,52,44,2,27,2,0,0,0,0,0,0,0,0,11,55,0,0,0,38,26,26,26,38,34,31,44,4,11,8,9,55,10,0,3,52,9,12,7,57,2,0,0,44,0,2,4,30,0,7,52,38,0,0,0,40,0,46,53,0,0,27,33,0,4,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.223E3,7.2E2,1.503E3,3.65E2,3.55E2,1.026E3,4.77E2,2.5E2,1.15E2,3.44E2,1.1E1,5.56E2,4.7E2,3.9E2,8.7E1,2.13E2,3.7E1,8.7E1,2.8E1,2.69E2,7.5E1,5E0,6E0,9.7E1,4.59E2,1.85E2,2.85E2,2.29E2,1.61E2,8.2E1,5E0,1.81E2,3.2E1,1.5E1,2.2E1,8.3E1,4E0,1.6E1,1.2E1,8.8E1,1.81E2,6.4E1,1.1E1,9.5E1,2E0,4.05E2,5.4E1,6.3E1,1.22E2,2.55E2,3E1,2.08E2,2.1E1,1.54E2,7E0,7.7E1,5E0,2E0,3E0,1.51E2,3E1,2.9E1,3E0,5E0,1E1,5E0,1.7E1,4.6E1,3.7E1,1.4E1,2E0,9E0,3E0,4.1E1,4.7E1,1.5E2,3.1E1,2.7E1,3.7E1,5E0,6E0,7.3E1,2.2E1,3.4E2,6.5E1,7E0,4.7E1,1.1E1,5.2E1,7.4E1,4.8E1,2.5E2,5E0,1.6E1,1.4E1,1.11E2,9.7E1,9E0,1.2E1,9.8E1,5.6E1,4E0,3E0,8E0,6.9E1,2E0,3E0,1.02E2,4.9E1,9E0,2.1E1,1.6E1,1.3E1,3E0,2E0,7E0,3E0,1.3E1,4E0,2.1E1,2.5E1,3.5E1,2E0,5E0,4E0,2E1,2.1E1,4.1E1,6E0,1.31E2,1.9E1,1.7E1,1.4E1,9E0,1.8E1,1E1,2.7E1,4.3E1,3E1,1E1,1.2E1,1.14E2,2.26E2,6E0,5.9E1,1.7E1,3E1,3E0,8E0,4.6E1,6E0,6.6E1,8E0,4.1E1,7E0,1.81E2,6.9E1,1.3E1,3E0,2E0,1.2E1,2.8E1,8.3E1,3.3E1,6.4E1,3E0,9E0,6.8E1,3E1,1.2E1,4.4E1,5.2E1,1.7E1,9.8E1,4E0,2.4E1,2.5E1,3E0,6E0,3E0,1.8E1,1.1E1,5E0,3E0,1.8E1,3E0,2.2E1,2.2E1,1.3E1,9E0,1.1E1,2E0,1.9E1,1E1,3.1E1,4E0,2E0,6.5E1,6.6E1,1.5E1,4E0,9E0,8E0,4E0,1E1,3E0,6E0,3E0,1.5E1,7E0,3E0,5E0,2.2E1,4E1,3E0,6E0,2.4E1,2E0,1E1,1E1,1.04E2,1.5E2,7.6E1,2E0,4E0,5.1E1,8E0,1.3E1,4E0,1.1E1,1.9E1,4.1E1,5E0,3E0,3E0,4.9E1,1.7E1,6E0,2E0,1.7E1,2.4E1,5E0,2E0,6E1,1.21E2,4.6E1,2.3E1,9E0,4E0,9E0,1.9E1,5.9E1,2.4E1,2.5E1,8E0,4.1E1,2.3E1,2.5E1,4.3E1,2.6E1,4E0,4.2E1,2E0,1.5E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"265","size_leaf_vector":"1"}},{"base_weights":[-9.089002E-3,-4.198235E-1,1.9412889E-1,-6.104423E-1,-1.5615392E-1,2.8010024E-2,5.184147E-1,-6.3740367E-1,7.700683E-2,-2.589812E-1,3.716619E-1,-1.4138377E-1,1.9415578E-1,-1.703273E-1,5.741402E-1,-7.422288E-1,-3.4496966E-1,-4.2709818E-1,-1.4215627E-1,-1.7929604E-2,4.597682E-1,-1.0343979E-1,-6.5632355E-1,2.3668298E-1,-4.0596342E-1,-3.9703572E-1,2.3048192E-1,5.0409126E-1,1.0554813E0,-8.390247E-1,-5.2655935E-1,-8.9383334E-2,-4.805034E-1,-4.9826786E-1,-2.2121163E-1,-2.2464527E-1,2.0756826E-2,-8.8867955E-3,6.0527567E-3,3.2549E-1,5.910499E-1,-3.688323E-1,-3.470408E-2,-9.7517973E-1,-4.428665E-1,8.155404E-2,4.0672347E-1,-5.205805E-1,3.170848E-1,-4.201599E-3,-4.3607816E-1,-4.31844E-3,2.9274175E-1,3.5621417E-1,6.2116593E-1,1.0828768E0,9.531731E-3,-7.8261E-1,-6.0990166E-2,-7.706339E-1,-3.6457092E-1,-1.268078E-1,1.8199192E-2,-6.4906824E-1,-1.9845496E-1,-4.1743392E-1,-7.310513E-1,-1.5196662E-2,-1.14396445E-1,-2.698285E-1,-1.3032949E-1,8.6134166E-2,-1.7697188E-1,3.6455154E-1,5.4224813E-3,2.8952368E-2,5.411287E-3,-4.7480974E-1,-2.617719E-1,8.238633E-1,-7.304381E-2,-5.774212E-2,-3.013653E-2,-2.7083619E-2,-1.1564585E-1,-2.0390485E-1,1.2919784E-1,3.5753942E-1,7.974813E-1,-3.4928283E-1,-3.1300765E-2,9.764173E-4,2.2949716E-2,-2.1745224E-2,-8.98676E-3,6.1666355E-3,1.7434727E-2,7.234238E-1,2.7271816E-1,8.4779423E-1,4.3923515E-1,1.1064562E0,1.4180206E-2,-6.4057666E-1,-8.6279756E-1,-1.142324E-2,-7.930432E-1,-1.4812766E-1,-5.425659E-1,-2.9556575E-1,-3.3981342E-2,-7.235828E-1,-6.87424E-3,1.3957259E-1,-4.606989E-1,-4.4952673E-1,-1.6066286E-1,-4.2089733E-1,-4.0301E-2,-1.6354352E-1,3.3471338E-3,-3.100312E-1,-9.8359E-2,-2.0714404E-1,1.256768E-3,-5.7104968E-2,1.78038E-1,-8.076363E-4,-1.2414123E-2,4.2386478E-1,7.935661E-3,-5.001044E-1,-1.04700085E-2,-1.3645734E-1,-2.9627195E-1,7.6382486E-3,4.407835E-2,-1.1220074E-1,4.1699022E-1,-8.881814E-3,6.307798E-4,-1.0655175E-1,-1.28814075E-2,1.0979617E-1,2.9608617E-2,5.8226955E-1,2.6000494E-1,4.79088E-2,2.5094993E-2,-4.2836854E-1,-2.2336403E-4,1.2072788E-2,3.4475204E-2,3.205123E-1,-4.2020895E-2,-1.15560675E-2,8.739786E-1,2.9114485E-1,5.400888E-1,3.7934322E-2,5.357512E-2,-3.0840168E-2,-1.2907935E-2,-4.235451E-2,-3.0569617E-2,-2.4423866E-2,-3.9519377E-2,3.818303E-3,-1.0396357E-2,-2.7329305E-2,-8.026664E-3,-2.6090138E-4,-1.5743261E-2,6.510755E-3,-7.3386696E-3,-3.5333097E-2,-1.63143E-2,1.9581106E-2,-1.7509441E-3,-1.1011163E-2,-3.1944416E-2,5.2266627E-5,-2.1185739E-2,-1.1714197E-2,-2.2667907E-3,-2.2907391E-2,-5.000375E-3,-1.939903E-3,-9.617319E-3,-2.651271E-3,-1.4911623E-2,5.089909E-3,-7.518458E-3,1.8334301E-3,-1.0533311E-2,-1.17971925E-2,4.479149E-4,2.938683E-3,1.3846368E-2,6.194486E-3,2.0989724E-2,-6.831204E-3,-2.3311697E-2,-1.0245158E-2,-1.8092287E-3,-1.527957E-2,-4.971868E-3,-4.6486123E-3,-3.700899E-2,2.1226427E-2,-5.41266E-3,-7.32424E-4,-8.379611E-3,5.551865E-3,-1.8332109E-2,1.8852407E-2,3.249154E-2,-5.3364686E-3,1.3332124E-2,-2.3220597E-2,-1.1059397E-2,-6.679963E-3,1.6004134E-2,-1.9329263E-2,2.309883E-3,4.1489072E-2,4.484677E-3,1.6460938E-2,2.0760696E-3,2.609213E-2,-9.954131E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,73,75,77,79,81,83,85,87,89,91,-1,93,-1,95,97,99,101,-1,103,-1,105,107,109,-1,111,113,115,117,-1,119,121,123,125,127,129,-1,-1,-1,131,133,135,137,-1,-1,-1,139,141,143,145,147,149,-1,-1,-1,-1,-1,-1,-1,151,153,155,157,159,-1,161,163,-1,165,167,169,171,173,175,-1,177,179,181,183,185,-1,187,-1,189,191,193,-1,195,197,-1,-1,199,-1,201,-1,203,205,-1,-1,207,209,-1,-1,211,-1,213,-1,215,217,-1,-1,219,-1,-1,-1,221,223,-1,225,227,229,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7828694E2,3.543876E1,7.69657E1,2.6634201E1,1.6269867E1,2.665173E1,1.8673965E1,1.21896515E1,0E0,4.84935E0,1.6901922E0,9.095835E0,1.2295309E1,3.4535818E0,1.4573349E1,5.7792206E0,3.7328749E0,1.3843708E0,2.0070055E0,2.9667392E-1,5.127497E-1,7.977171E0,1.7461128E0,1.1757887E1,2.8170247E0,2.4460125E-1,3.1196058E-1,6.582611E0,1.0682907E0,4.7933807E0,3.4750881E0,7.6814425E-1,3.2467728E0,1.1060944E0,3.0389714E-1,3.8672113E-1,6.745192E-1,0E0,0E0,1.331234E-1,3.5756636E-1,8.968468E-1,1.1548294E1,2.6614475E-1,9.983258E-1,3.2224255E0,3.808529E0,5.7765245E-1,2.6954806E-1,0E0,1.2530851E-1,0E0,1.2102342E-1,5.198965E0,8.615898E0,6.334572E-1,0E0,1.5657272E0,0E0,2.2966766E-1,2.1926541E0,5.695689E-1,0E0,1.52038E0,2.54638E0,4.057274E-1,5.135937E-1,0E0,1.6018254E-1,4.2447948E-1,4.1862947E-1,5.235717E-1,1.9624755E-1,1.277771E-1,0E0,0E0,0E0,1.5541077E-1,1.6276932E-1,1.307168E0,6.474843E0,0E0,0E0,0E0,8.621647E-2,2.3652494E-1,1.98492E0,4.035124E0,8.7172794E-1,4.3361604E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.5318565E-1,2.1946573E0,2.9958038E0,1.6798153E0,8.242035E-2,0E0,8.30801E-1,9.506531E-1,0E0,2.3778915E-1,5.498825E-1,6.0689735E-1,1.9453418E-1,5.927404E-1,5.034218E-1,0E0,6.9550776E-1,6.805315E-1,4.2388916E-1,7.381995E-2,1.5541077E-1,0E0,6.7595035E-2,0E0,2.3838234E-1,2.1847099E-1,1.5473807E-1,0E0,2.2069108E-1,3.2713044E-1,0E0,0E0,8.490038E-2,0E0,1.23859406E-1,0E0,9.498133E-2,2.3961759E-1,0E0,0E0,2.004826E0,7.515998E-1,0E0,0E0,1.1766389E-1,0E0,1.3441987E0,0E0,9.812069E-1,1.7932253E0,0E0,0E0,7.699013E-2,0E0,0E0,0E0,1.997345E0,7.380481E-1,0E0,2.9993591E0,9.073067E-1,2.1089802E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,52,52,53,53,54,54,55,55,57,57,59,59,60,60,61,61,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,72,72,73,73,77,77,78,78,79,79,80,80,84,84,85,85,86,86,87,87,88,88,89,89,97,97,98,98,99,99,100,100,101,101,103,103,104,104,106,106,107,107,108,108,109,109,110,110,111,111,113,113,114,114,115,115,116,116,117,117,119,119,121,121,122,122,123,123,125,125,126,126,129,129,131,131,133,133,134,134,137,137,138,138,141,141,143,143,145,145,146,146,149,149,153,153,154,154,156,156,157,157,158,158],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,74,76,78,80,82,84,86,88,90,92,-1,94,-1,96,98,100,102,-1,104,-1,106,108,110,-1,112,114,116,118,-1,120,122,124,126,128,130,-1,-1,-1,132,134,136,138,-1,-1,-1,140,142,144,146,148,150,-1,-1,-1,-1,-1,-1,-1,152,154,156,158,160,-1,162,164,-1,166,168,170,172,174,176,-1,178,180,182,184,186,-1,188,-1,190,192,194,-1,196,198,-1,-1,200,-1,202,-1,204,206,-1,-1,208,210,-1,-1,212,-1,214,-1,216,218,-1,-1,220,-1,-1,-1,222,224,-1,226,228,230,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.0330753E5,2.14099E5,1.0323588E3,2.3770695E5,1.6383727E-1,1.3528846E1,7.700683E-2,1.5669752E-2,5.2879925E5,1E0,2.5757682E-1,1E0,4.147809E0,5.676415E-1,3.653602E7,1.5345133E1,4.6547272E2,1.7860331E3,1.6173E4,2.73E2,1E0,1.06E3,2.5422776E3,1.8528142E6,1.5445488E8,1.0989723E3,6.292039E0,1.1855755E8,1.627E3,6.8654716E-1,2.4345527E3,5.236559E0,7.123717E7,6.38041E5,6.3300834E0,-8.8867955E-3,6.0527567E-3,4.080551E-1,2.970339E0,3.75E1,8E0,4.1724915E2,4.2E1,3.97E2,1.1392E4,1.96013E9,7.019E3,-4.201599E-3,5.387818E8,-4.31844E-3,2.001177E6,7E0,2.1E1,8.3998317E-1,9.531731E-3,4.459525E5,-6.0990166E-2,2.461362E-3,1.6837112E7,6.77603E5,1.8199192E-2,5.108923E6,1.0828989E8,6.285983E5,2.7056329E10,-1.5196662E-2,6.567044E2,2.1150263E-1,1.4328269E8,2.2418336E5,9.6771875E2,1.2E1,5.4224813E-3,2.8952368E-2,5.411287E-3,2.255481E6,1.00033E5,4.16E2,1.2799757E9,-5.774212E-2,-3.013653E-2,-2.7083619E-2,7E0,3.642857E0,7.780377E6,3.1140776E0,2.909019E3,1.3992102E3,-3.1300765E-2,9.764173E-4,2.2949716E-2,-2.1745224E-2,-8.98676E-3,6.1666355E-3,1.7434727E-2,1.513E3,9.433962E-3,8.490231E-1,1.4101E4,6.955642E0,1.4180206E-2,3.164136E6,2.0288463E0,-1.142324E-2,1.3298E4,1.2478469E-1,3.4409692E3,1E0,6.53143E6,4.8083666E-1,-6.87424E-3,4.895628E-2,5.784E4,4.5499244E9,1.6841E4,9.048701E7,-4.0301E-2,1.0573252E8,3.3471338E-3,5.249309E5,1.6772E4,1E0,1.256768E-3,1.19507775E-1,3.222E3,-8.076363E-4,-1.2414123E-2,5.1212654E0,7.935661E-3,4E0,-1.04700085E-2,2.5167784E-1,3.30399E6,7.6382486E-3,4.407835E-2,5.6202265E6,9.3161255E-1,-8.881814E-3,6.307798E-4,3.4402E4,-1.28814075E-2,4.6E1,2.9608617E-2,4.1365924E2,4.0328094E-5,4.79088E-2,2.5094993E-2,2.14E2,-2.2336403E-4,1.2072788E-2,3.4475204E-2,6.1E2,1.993E3,-1.15560675E-2,6.134093E7,2.0189162E7,6.249511E9,3.7934322E-2,5.357512E-2,-3.0840168E-2,-1.2907935E-2,-4.235451E-2,-3.0569617E-2,-2.4423866E-2,-3.9519377E-2,3.818303E-3,-1.0396357E-2,-2.7329305E-2,-8.026664E-3,-2.6090138E-4,-1.5743261E-2,6.510755E-3,-7.3386696E-3,-3.5333097E-2,-1.63143E-2,1.9581106E-2,-1.7509441E-3,-1.1011163E-2,-3.1944416E-2,5.2266627E-5,-2.1185739E-2,-1.1714197E-2,-2.2667907E-3,-2.2907391E-2,-5.000375E-3,-1.939903E-3,-9.617319E-3,-2.651271E-3,-1.4911623E-2,5.089909E-3,-7.518458E-3,1.8334301E-3,-1.0533311E-2,-1.17971925E-2,4.479149E-4,2.938683E-3,1.3846368E-2,6.194486E-3,2.0989724E-2,-6.831204E-3,-2.3311697E-2,-1.0245158E-2,-1.8092287E-3,-1.527957E-2,-4.971868E-3,-4.6486123E-3,-3.700899E-2,2.1226427E-2,-5.41266E-3,-7.32424E-4,-8.379611E-3,5.551865E-3,-1.8332109E-2,1.8852407E-2,3.249154E-2,-5.3364686E-3,1.3332124E-2,-2.3220597E-2,-1.1059397E-2,-6.679963E-3,1.6004134E-2,-1.9329263E-2,2.309883E-3,4.1489072E-2,4.484677E-3,1.6460938E-2,2.0760696E-3,2.609213E-2,-9.954131E-3],"split_indices":[19,109,32,2,51,27,41,55,0,37,47,6,26,109,34,41,44,57,51,32,2,2,15,2,51,49,7,51,37,44,2,37,51,53,7,9,52,0,0,37,33,4,17,4,6,2,2,5,2,0,7,0,9,3,3,26,0,27,0,26,44,28,0,31,44,27,30,0,51,37,12,27,4,8,0,0,0,30,5,2,7,0,0,0,8,53,27,53,31,4,0,0,0,0,0,0,0,2,56,52,2,34,0,1,52,0,9,26,51,8,28,41,0,37,10,30,9,7,0,7,0,50,9,15,0,37,2,0,0,52,0,0,0,57,9,0,0,46,33,0,0,9,0,3,0,51,37,0,0,0,0,0,0,2,2,0,49,44,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.134E3,7.06E2,1.428E3,4.09E2,2.97E2,9.45E2,4.83E2,4.05E2,4E0,2.49E2,4.8E1,4.68E2,4.77E2,3.6E1,4.47E2,2.97E2,1.08E2,1.01E2,1.48E2,9E0,3.9E1,4.37E2,3.1E1,4.46E2,3.1E1,2.3E1,1.3E1,3.92E2,5.5E1,2.03E2,9.4E1,3.8E1,7E1,7.4E1,2.7E1,9.8E1,5E1,4E0,5E0,2.1E1,1.8E1,8.9E1,3.48E2,1.1E1,2E1,2.34E2,2.12E2,2.7E1,4E0,3E0,2E1,2E0,1.1E1,1.75E2,2.17E2,5.3E1,2E0,1.85E2,1.8E1,3.6E1,5.8E1,3.6E1,2E0,4.3E1,2.7E1,5.7E1,1.7E1,1.2E1,1.5E1,6.5E1,3.3E1,3.8E1,1.2E1,1.7E1,4E0,1.6E1,2E0,4.3E1,4.6E1,1.4E1,3.34E2,4E0,7E0,1.3E1,7E0,3.3E1,2.01E2,1.9E2,2.2E1,1.5E1,1.2E1,2E0,2E0,1.6E1,4E0,5E0,6E0,3.1E1,1.44E2,9.5E1,1.22E2,5.1E1,2E0,7E1,1.15E2,2E0,3.4E1,2.7E1,3.1E1,1.2E1,2.4E1,3.7E1,6E0,1.2E1,1.5E1,5E1,7E0,7E0,1E1,1.2E1,3E0,5.2E1,1.3E1,2.2E1,1.1E1,1.5E1,2.3E1,5E0,7E0,1.2E1,5E0,3.8E1,5E0,1.1E1,3.5E1,3E0,1.1E1,3.1E2,2.4E1,4E0,3E0,1.6E1,1.7E1,1.95E2,6E0,5.6E1,1.34E2,9E0,1.3E1,1.2E1,3E0,3E0,2.8E1,1.25E2,1.9E1,2E0,9.3E1,5.1E1,7.1E1,1.4E1,3.7E1,6.2E1,8E0,8E1,3.5E1,1E1,2.4E1,7E0,2E1,2.6E1,5E0,2E0,1E1,1E1,1.4E1,3.1E1,6E0,4E0,8E0,9E0,6E0,2E0,4.8E1,3E0,4E0,5E0,2E0,4E0,8E0,4E0,4.8E1,3E0,1E1,2E0,2E1,3E0,1.2E1,1.3E1,1E1,2E0,1E1,2E0,3.6E1,5E0,6E0,2.8E1,7E0,3.07E2,3E0,2.2E1,2E0,8E0,8E0,1.91E2,4E0,2.7E1,2.9E1,1.1E1,1.23E2,7E0,5E0,8E0,1.17E2,3E0,1.6E1,8.8E1,5E0,3.9E1,1.2E1,6.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"231","size_leaf_vector":"1"}},{"base_weights":[-9.000763E-3,-3.7414548E-1,1.6467461E-1,-5.891176E-1,-1.5137042E-1,3.851567E-2,4.7865885E-1,-6.993949E-1,-3.4308112E-1,-2.619378E-1,2.9622057E-1,-1.8287939E-1,1.7599061E-1,3.504921E-1,8.0200493E-1,-6.190154E-1,-9.5928687E-1,-2.079398E-1,-5.0021225E-1,-2.9441282E-1,7.2259985E-2,1.053739E-1,4.2094278E-1,-4.0959674E-1,-1.0939188E-1,2.1312703E-1,-3.1289774E-1,1.6171855E-1,5.4951394E-1,8.816873E-1,2.2398402E-1,-6.822969E-1,-9.61319E-2,-1.1380745E0,-3.828098E-1,-3.1183445E-1,-8.3261184E-2,-4.194451E-2,-4.4539037E-1,-1.779793E-1,-5.0984055E-1,2.2066991E-1,-2.0959917E-3,4.5654023E-1,-8.783346E-3,-4.4262E-1,1.8153788E-1,-1.6330841E-1,1.2861139E-1,6.011321E-2,3.2534024E-1,-4.959604E-1,-6.263157E-2,2.1392883E-1,-3.2154453E-1,6.4954495E-1,2.0228194E-1,6.4235574E-1,1.0227678E0,7.789684E-2,2.0576097E-2,-4.754702E-1,-7.4755335E-1,3.0257168E-1,-2.6455647E-1,-3.7755694E-2,-1.2712209E0,-4.7735237E-3,-4.4336674E-1,3.4830326E-3,-1.53268855E-2,-1.21054456E-1,8.722419E-3,-2.5660159E-2,-3.384448E-1,-2.9164928E-1,-2.6612895E-4,-7.2643167E-1,-2.539149E-1,-1.2932593E-3,1.3185582E-2,-8.270584E-3,1.6638179E-3,6.4527524E-1,3.1748822E-1,-3.7569758E-1,-6.9053626E-1,1.848605E-2,-1.5361331E-3,-1.5110941E-1,-5.1322028E-2,5.1705707E-2,1.1454354E-2,-2.7457891E-2,2.5505465E-1,3.1040493E-1,5.9116255E-2,-5.46561E-1,4.1213413E-3,-1.7266831E-1,1.4669648E-2,3.6355445E-1,1.337704E-1,-2.1360623E-2,-6.89294E-3,5.48405E-1,9.3405443E-1,-3.6466336E-1,3.3215556E-1,6.9073224E-1,6.6201147E-3,1.0574489E0,-2.1032484E-3,1.3396165E-2,-8.8941574E-2,1.7460247E-3,-5.140232E-1,-8.004653E-1,-5.374961E-1,1.9096881E-2,1.3529793E-4,-1.2612802E-1,-3.582088E-1,-1.3211253E0,-1.8787615E-2,-2.1975176E-2,-7.0412643E-3,1.7315437E-3,-1.5899754E-1,-1.9013632E-2,-6.7777503E-3,-1.8485631E-1,-4.182396E-1,1.6291963E-1,-9.0533435E-2,-8.1675637E-1,-3.155164E-1,-4.0907827E-1,1.2996414E-1,5.558009E-3,3.2196607E-2,2.0719875E-1,4.9053633E-1,-4.828684E-1,-2.0989896E-1,-3.851076E-2,-1.6484207E-2,6.1637145E-2,-1.9517514E-1,1.4380996E-1,-1.31523E-1,-8.874715E-2,1.6539155E-1,-6.1103083E-2,2.7227196E-1,4.4279468E-1,2.1789461E-1,-9.028729E-3,-5.84833E-1,5.4274276E-3,-2.1966669E-1,3.171537E-2,2.6179966E-1,1.1679849E-2,2.506393E-1,4.2864493E-1,6.3927495E-1,6.7431484E-3,9.8669463E-1,-2.9195188E-2,-5.5965385E-3,-5.771461E-3,5.1031566E-1,1.5108023E-2,3.5079014E-2,1.121025E0,2.9654227E-2,2.9018912E-3,-9.94868E-3,-2.459895E-2,-4.8465235E-3,-3.7685633E-2,-1.9318448E-2,-1.4693273E-2,-2.793849E-2,2.182336E-3,-1.0882369E-2,-4.1070567E-3,-1.909094E-2,-4.527805E-2,-6.903656E-2,-1.1467997E-2,-4.880615E-3,-1.0249539E-2,3.1980695E-4,-9.8561E-3,-2.7620371E-2,-2.2896691E-3,1.2627827E-2,-5.858287E-3,1.185249E-2,-2.5352474E-2,-4.1884992E-2,-1.7517304E-2,-4.248242E-3,-2.0693985E-3,-2.2870163E-2,-5.1070913E-3,1.7604657E-2,1.3728442E-2,4.9644466E-3,2.5894357E-2,6.205379E-3,-1.1619693E-2,-2.3711719E-2,-3.287369E-3,-1.8334398E-2,-1.0304225E-2,9.244929E-3,-1.120144E-2,3.201437E-5,-2.234726E-4,1.23621505E-2,-1.0076123E-2,1.6475378E-3,-5.8548236E-3,7.794943E-3,-3.0171715E-3,9.906205E-3,-8.157962E-3,3.547332E-3,8.100557E-3,1.48520935E-2,1.3732316E-2,2.6809912E-2,1.1303604E-2,-3.4841925E-3,-1.2128597E-2,-2.8105002E-2,-1.2335215E-2,-4.304252E-3,3.9727376E-3,1.6421774E-2,-3.426869E-3,6.7467433E-3,8.004203E-3,1.9989893E-2,2.2416458E-2,1.2694667E-2,3.0957386E-2,9.678748E-3,3.585202E-2,5.6079585E-2,7.538821E-3,-5.795481E-3,2.6298804E-2,-2.7011898E-3,2.394933E-2,5.188248E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,115,117,119,121,-1,123,-1,125,-1,-1,127,-1,-1,129,131,133,135,137,-1,-1,-1,-1,139,141,143,145,-1,-1,147,-1,-1,149,151,153,155,-1,157,-1,159,-1,161,163,-1,-1,165,167,169,171,173,-1,175,-1,-1,177,-1,179,181,183,-1,-1,185,187,189,-1,-1,-1,-1,191,-1,-1,193,195,197,199,201,203,205,207,-1,-1,209,211,213,215,-1,-1,217,219,221,223,225,227,229,231,233,235,-1,237,-1,239,-1,241,243,245,247,249,-1,251,-1,-1,253,255,-1,-1,257,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4008557E2,3.4008797E1,5.9280613E1,9.581917E0,1.7448215E1,3.256856E1,1.7497604E1,4.6077957E0,2.3167658E0,1.7485373E1,1.621841E0,6.7861586E0,1.2049358E1,1.1520283E1,5.3755417E0,6.3205566E0,5.439152E0,7.9010296E-1,8.0248547E-1,6.8823357E0,0E0,3.5979718E-1,1.0079751E0,2.0324726E0,4.0118513E0,1.052293E1,2.1222138E0,4.105796E0,5.078705E0,2.9221115E0,5.033572E-1,1.9660034E0,1.5573797E0,6.811485E-1,2.052064E-1,3.7899446E-1,3.361596E-1,0E0,4.312935E-1,3.6764417E0,5.192377E0,2.545588E-1,1.1895206E-1,8.4489155E-1,0E0,1.3167286E0,3.0703336E-1,2.835163E0,6.873432E0,4.462596E0,4.7781982E0,8.517108E-1,9.4941056E-1,1.6904607E0,3.3518946E-1,2.8256912E0,2.6982188E0,9.070473E-1,2.4594421E0,4.2638528E-1,0E0,8.91799E-1,1.0035324E0,2.4878639E-1,1.6224623E-1,0E0,6.4821243E-1,0E0,6.9636345E-2,0E0,0E0,1.6927835E-1,0E0,0E0,3.6795735E-1,1.4232903E0,1.075334E0,1.6723213E0,2.7931082E0,0E0,0E0,0E0,0E0,4.6342754E-1,3.754959E-1,1.2668419E0,6.8315697E-1,0E0,0E0,2.3744884E0,0E0,0E0,1.0218786E0,2.1540253E0,4.57551E-1,4.190113E0,0E0,2.1551132E-1,0E0,2.6007134E-1,0E0,1.531004E0,1.3668572E0,0E0,0E0,6.8364716E-1,1.051569E0,3.930664E-1,1.7488933E0,8.8222885E-1,0E0,8.7768555E-1,0E0,0E0,1.5420884E-1,0E0,4.309435E-1,8.5002136E-1,2.9132557E-1,0E0,0E0,1.7090972E-1,1.14540815E-1,1.7557526E-1,0E0,0E0,0E0,0E0,7.8909576E-2,0E0,0E0,4.9580264E-1,1.7906961E0,6.580061E-1,6.774243E-1,6.50877E-1,1.3780701E-1,1.1289339E0,9.237448E-1,0E0,0E0,1.2599623E-1,1.8060017E-1,2.9087353E-1,8.2756805E-1,0E0,0E0,1.8607715E0,2.1388316E0,5.3962713E-1,4.0553907E-1,1.4831673E0,5.599153E-1,9.715594E-2,3.4222507E-1,2.799759E0,1.9680548E0,0E0,8.431959E-2,0E0,7.275903E-2,0E0,6.7078376E-1,5.872643E-1,6.126604E-1,2.652707E-1,7.6276016E-1,0E0,3.4443665E-1,0E0,0E0,2.5297534E-1,7.9885817E-1,0E0,0E0,1.76445E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,66,66,68,68,71,71,74,74,75,75,76,76,77,77,78,78,83,83,84,84,85,85,86,86,89,89,92,92,93,93,94,94,95,95,97,97,99,99,101,101,102,102,105,105,106,106,107,107,108,108,109,109,111,111,114,114,116,116,117,117,118,118,121,121,122,122,123,123,128,128,131,131,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,158,158,160,160,162,162,163,163,164,164,165,165,166,166,168,168,171,171,172,172,175,175],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,116,118,120,122,-1,124,-1,126,-1,-1,128,-1,-1,130,132,134,136,138,-1,-1,-1,-1,140,142,144,146,-1,-1,148,-1,-1,150,152,154,156,-1,158,-1,160,-1,162,164,-1,-1,166,168,170,172,174,-1,176,-1,-1,178,-1,180,182,184,-1,-1,186,188,190,-1,-1,-1,-1,192,-1,-1,194,196,198,200,202,204,206,208,-1,-1,210,212,214,216,-1,-1,218,220,222,224,226,228,230,232,234,236,-1,238,-1,240,-1,242,244,246,248,250,-1,252,-1,-1,254,256,-1,-1,258,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,3.673719E8,1E0,1.5283889E0,1.8108038E5,1.2974394E6,4.0844156E7,5.1148495E6,2.14099E5,7.621583E2,9.6E1,4.3271405E-1,1.2679E4,1.708775E7,3.596E3,2.0401025E3,2.060792E8,1.003252E5,4.0844156E7,7.2259985E-2,1.3510204E1,1E0,1.5141565E3,1.0989723E3,9.19E2,2.4832625E6,5.9767612E7,3.5301748E7,1.00860974E3,5E0,2.0304577E-1,1E0,4.3927447E11,1E0,2.7487562E0,3.7810526E2,-4.194451E-2,1.074E3,5.0713895E2,2.3023027E3,1.9563605E5,6.061028E4,3.1E1,-8.783346E-3,1.4504054E8,1.92E2,4.5E1,3.4175084E0,1.7310865E6,5.3534385E3,2.200423E3,5.6045805E6,3.61E2,1.6213043E3,6.082581E3,3.8976521E0,9.8708276E2,1.3674345E4,1.7321888E1,2.0576097E-2,1.4E1,2.5378451E0,9E0,1.1E1,-3.7755694E-2,4.551684E6,-4.7735237E-3,3.275343E6,3.4830326E-3,-1.53268855E-2,1.1331109E6,8.722419E-3,-2.5660159E-2,2.0939131E2,1.216918E7,8.2654175E5,3.3368218E6,1E0,-1.2932593E-3,1.3185582E-2,-8.270584E-3,1.6638179E-3,1E0,1.9464845E9,4.3875E1,1.45752E8,1.848605E-2,-1.5361331E-3,1.4286339E0,-5.1322028E-2,5.1705707E-2,2.3907686E7,2.04946E6,3.5902756E-6,3.5636363E0,5.9116255E-2,2.345785E5,4.1213413E-3,3.2775325E6,1.4669648E-2,2.53383E5,7.06699E8,-2.1360623E-2,-6.89294E-3,4.7658E4,4.423676E1,2.4832896E9,7.3610186E3,2.657E3,6.6201147E-3,2.6364462E7,-2.1032484E-3,1.3396165E-2,1.151E4,1.7460247E-3,3.730007E6,7.5569354E2,1.8344E4,1.9096881E-2,1.3529793E-4,4.1808826E-1,8.73E2,8.522844E5,-1.8787615E-2,-2.1975176E-2,-7.0412643E-3,1.7315437E-3,7.5646E4,-1.9013632E-2,-6.7777503E-3,4.41E4,1.9563605E5,5.68E2,2.9283184E3,2.060792E8,1.4032121E-1,6.747114E7,3.130603E3,5.558009E-3,3.2196607E-2,1.2516333E7,3.007E3,1.901E3,1.3446785E7,-3.851076E-2,-1.6484207E-2,7.860526E1,8.399036E5,6.726E3,7.678E3,3.5734247E2,4.67E2,1.02E2,3.136758E9,4.198242E2,1.4383178E1,-9.028729E-3,1.04E3,5.4274276E-3,4.1730586E-1,3.171537E-2,4.273927E8,1.9119574E6,6.065707E8,1.7548548E7,9.70107E7,6.7431484E-3,2.0455818E1,-2.9195188E-2,-5.5965385E-3,5.839812E-1,2.88001E10,1.5108023E-2,3.5079014E-2,1.8382221E-1,2.9654227E-2,2.9018912E-3,-9.94868E-3,-2.459895E-2,-4.8465235E-3,-3.7685633E-2,-1.9318448E-2,-1.4693273E-2,-2.793849E-2,2.182336E-3,-1.0882369E-2,-4.1070567E-3,-1.909094E-2,-4.527805E-2,-6.903656E-2,-1.1467997E-2,-4.880615E-3,-1.0249539E-2,3.1980695E-4,-9.8561E-3,-2.7620371E-2,-2.2896691E-3,1.2627827E-2,-5.858287E-3,1.185249E-2,-2.5352474E-2,-4.1884992E-2,-1.7517304E-2,-4.248242E-3,-2.0693985E-3,-2.2870163E-2,-5.1070913E-3,1.7604657E-2,1.3728442E-2,4.9644466E-3,2.5894357E-2,6.205379E-3,-1.1619693E-2,-2.3711719E-2,-3.287369E-3,-1.8334398E-2,-1.0304225E-2,9.244929E-3,-1.120144E-2,3.201437E-5,-2.234726E-4,1.23621505E-2,-1.0076123E-2,1.6475378E-3,-5.8548236E-3,7.794943E-3,-3.0171715E-3,9.906205E-3,-8.157962E-3,3.547332E-3,8.100557E-3,1.48520935E-2,1.3732316E-2,2.6809912E-2,1.1303604E-2,-3.4841925E-3,-1.2128597E-2,-2.8105002E-2,-1.2335215E-2,-4.304252E-3,3.9727376E-3,1.6421774E-2,-3.426869E-3,6.7467433E-3,8.004203E-3,1.9989893E-2,2.2416458E-2,1.2694667E-2,3.0957386E-2,9.678748E-3,3.585202E-2,5.6079585E-2,7.538821E-3,-5.795481E-3,2.6298804E-2,-2.7011898E-3,2.394933E-2,5.188248E-2],"split_indices":[19,55,7,109,40,27,27,44,44,2,51,28,26,2,44,2,4,7,27,44,0,57,111,54,51,2,31,44,44,4,6,26,80,30,63,53,51,0,2,51,54,27,27,3,0,30,0,6,52,27,4,51,47,0,4,51,53,4,4,55,0,0,53,3,8,0,28,0,28,0,0,44,0,0,51,44,46,31,109,0,0,0,0,15,7,4,7,0,0,52,0,0,44,9,37,53,0,27,0,31,0,9,7,0,0,10,55,7,4,2,0,46,0,0,28,0,9,4,9,0,0,26,0,47,0,0,0,0,9,0,0,10,27,0,51,7,56,44,4,0,0,31,0,9,44,0,0,51,49,2,2,51,2,0,30,51,52,0,2,0,38,0,30,46,12,44,1,0,53,0,0,41,5,0,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.207E3,7.11E2,1.496E3,3.61E2,3.5E2,1.068E3,4.28E2,2.48E2,1.13E2,2.81E2,6.9E1,4.09E2,6.59E2,3.08E2,1.2E2,1.92E2,5.6E1,6.2E1,5.1E1,2.77E2,4E0,2.8E1,4.1E1,9.9E1,3.1E2,6.13E2,4.6E1,1.59E2,1.49E2,1.05E2,1.5E1,1.71E2,2.1E1,4.2E1,1.4E1,3.3E1,2.9E1,4E0,4.7E1,1.81E2,9.6E1,1.3E1,1.5E1,3.9E1,2E0,9.4E1,5E0,2.53E2,5.7E1,2.6E2,3.53E2,2.6E1,2E1,1.44E2,1.5E1,1.15E2,3.4E1,4.1E1,6.4E1,1E1,5E0,4.3E1,1.28E2,6E0,1.5E1,1.5E1,2.7E1,3E0,1.1E1,2E0,3.1E1,2.6E1,3E0,2E1,2.7E1,1.1E2,7.1E1,5.1E1,4.5E1,3E0,1E1,2E0,1.3E1,1.5E1,2.4E1,7.6E1,1.8E1,2E0,3E0,2.51E2,2E0,5E0,5.2E1,1.8E2,8E1,3.49E2,4E0,2.4E1,2E0,1.6E1,4E0,4.9E1,9.5E1,7E0,8E0,8.7E1,2.8E1,6E0,2.8E1,3.7E1,4E0,6.2E1,2E0,4E0,6E0,3E0,4E1,1E2,2.8E1,4E0,2E0,7E0,8E0,2.5E1,2E0,9E0,2E0,5E0,2.1E1,1.8E1,9E0,6.1E1,4.9E1,2.5E1,4.6E1,4.1E1,1E1,3.2E1,1.3E1,2E0,1.3E1,1.6E1,8E0,4.5E1,3.1E1,1.1E1,7E0,4.3E1,2.08E2,2.7E1,2.5E1,1.37E2,4.3E1,4E0,7.6E1,1.42E2,2.07E2,3E0,2.1E1,2E0,1.4E1,1E1,3.9E1,4.7E1,4.8E1,4E1,4.7E1,2E0,2.6E1,2E0,4E0,1E1,1.8E1,8E0,2.9E1,5.2E1,1E1,3E0,3E0,3.7E1,3E0,9.1E1,9E0,9E0,1.9E1,3E0,4E0,2E0,6E0,1.2E1,1.3E1,6E0,1.5E1,5E1,1.1E1,2.5E1,2.4E1,9E0,1.6E1,4.2E1,4E0,1.4E1,2.7E1,7E0,3E0,7E0,2.5E1,7E0,6E0,7E0,9E0,6E0,2E0,8E0,3.7E1,1.9E1,1.2E1,1.4E1,2.9E1,1.64E2,4.4E1,1.3E1,1.4E1,1.6E1,9E0,1.19E2,1.8E1,8E0,3.5E1,2E0,2E0,3E1,4.6E1,7.5E1,6.7E1,1.87E2,2E1,3E0,1.8E1,9E0,5E0,1.5E1,2.4E1,2.9E1,1.8E1,3.6E1,1.2E1,2.6E1,1.4E1,4.2E1,5E0,1.7E1,9E0,4E0,6E0,1.6E1,2E0,3E0,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"259","size_leaf_vector":"1"}},{"base_weights":[9.259256E-3,-3.4183672E-1,1.7408596E-1,-5.307734E-1,-1.1395881E-1,2.2927234E-2,4.385728E-1,-6.371423E-1,-2.1483675E-1,-2.3412505E-1,3.3400092E-1,-2.7804908E-1,9.369408E-2,3.5129347E-1,7.884568E-1,-3.8404563E-1,-7.216796E-1,9.4473153E-1,-3.1123793E-1,-3.0184573E-1,6.933996E-3,-2.5852755E-2,4.4991344E-1,-4.0066332E-1,-1.6015275E-1,2.2013612E-1,-8.058494E-2,2.1154888E-1,5.759298E-1,5.9665316E-1,9.7986704E-1,-4.6782178E-1,-1.2608133E-1,-6.567015E-1,-1.0156091E0,5.3288326E-2,7.481286E-3,-1.0011283E-1,-5.202329E-1,-2.712863E-1,-6.008575E-1,-1.7953232E-1,1.345985E-1,-8.568654E-3,9.534083E-2,5.06584E-1,1.8748203E-1,-3.7264678E-1,-4.150994E-2,-2.0122069E-1,1.2068789E-1,4.5990452E-2,1.9842853E-1,-2.0557837E-1,8.0669105E-2,2.7207312E-1,-2.641248E-1,-3.919685E-2,6.1414254E-1,7.6220133E-3,6.4283484E-1,1.0181313E0,1.1184101E-2,-4.7442266E-3,-5.007136E-1,-1.2162829E-2,-8.648336E-3,-6.972989E-1,-2.4581988E-1,-8.0425143E-1,-1.32049E0,-2.1726185E-1,1.2629193E-1,-6.636098E-1,-2.2817902E-1,-5.402327E-1,-2.4103588E-1,-3.2707125E-2,-1.3881701E-2,3.4321037E-3,-2.4071112E-1,2.5599596E-1,3.0750593E-2,9.943654E-3,-4.484148E-2,2.8612745E-1,5.565246E-1,3.5946143E-3,1.1966813E-2,-4.0710065E-1,-2.0480412E-1,-7.251972E-2,-2.6316994E-1,1.4430088E-3,1.9226183E-1,-1.04751565E-1,2.5645423E-1,-3.0737516E-1,2.9641498E-2,5.44971E-2,4.5395922E-2,3.7398374E-1,1.3428448E-1,-3.0807224E-1,5.456655E-3,-2.365354E-2,1.2624395E-1,5.5347496E-1,4.6053044E-2,6.878864E-1,8.275696E-3,1.0708871E0,2.0100791E-2,-5.211838E-1,3.3327076E-4,5.508208E-3,-2.1223156E-3,-7.348816E-1,-4.7659972E-1,3.660534E-3,-1.679053E-2,-5.575634E-2,-6.6422015E-1,-6.570885E-2,-2.854395E-2,1.3369259E-1,-2.8720498E-1,-6.258329E-3,2.1831731E-1,-2.2861606E-2,-9.953097E-1,-8.549736E-4,-3.4062293E-1,-1.8790292E-2,-3.2663103E-2,-2.896667E-1,-1.1241088E-1,-7.280161E-3,4.585636E-3,-1.3228938E-2,-4.03113E-3,2.1404733E-3,1.3605945E-2,7.548258E-3,-1.1630357E-3,-8.571168E-3,3.3831499E-3,1.5222905E-2,2.5859808E-3,7.1008615E-3,5.901441E-1,-3.7447342E-1,-2.9725581E-2,8.3307475E-3,-2.6891285E-1,2.444829E-3,-1.5283786E-1,-2.9199246E-1,4.3002404E-3,4.332565E-4,1.19262785E-2,-3.963211E-1,-1.5954483E-2,2.2630593E-1,7.58199E-1,-3.3740956E-2,-2.5327605E-1,2.1539469E-1,-1.07813545E-1,9.143022E-2,-5.096586E-1,2.8633603E-1,5.565502E-1,-1.8879723E-2,2.3748668E-1,-3.747285E-1,-3.9052656E-3,1.5152307E-2,-4.864296E-3,4.9431083E-1,8.736519E-1,7.3331046E-1,3.747806E-1,2.6945613E-2,5.1040296E-2,-1.576372E-2,-2.6204072E-2,-3.414904E-2,-1.451455E-2,-2.779695E-2,-1.274151E-2,4.669611E-3,-7.3959893E-3,-3.158888E-2,-1.0440171E-2,1.3119557E-2,-5.3871595E-3,1.1780618E-3,-1.4824611E-2,2.3242418E-2,3.5523465E-3,-1.7078303E-2,-5.061707E-2,-8.519664E-3,-2.1999428E-2,-1.4693262E-2,-8.769839E-3,-1.1244684E-2,2.0902664E-3,3.337573E-2,2.0055698E-2,-1.988688E-2,-1.1676719E-2,-6.241204E-3,-1.8064337E-2,-8.944691E-3,1.3467025E-4,-1.38497865E-2,-2.038049E-3,-8.87578E-3,-2.5016593E-2,-9.071758E-3,5.192641E-3,7.0018834E-3,1.7308198E-2,1.6106015E-2,3.901399E-2,-2.0245846E-2,-8.144125E-3,1.8332466E-2,3.7793464E-3,8.245143E-4,-1.4161403E-2,2.2124895E-3,1.8453006E-2,-8.5808905E-3,-2.5989376E-2,1.41789345E-2,-8.035276E-3,-1.0495675E-3,2.6442414E-2,-1.7801289E-2,1.2849254E-3,7.895978E-3,2.1348808E-2,-1.0379115E-2,-1.9347116E-2,2.4214542E-2,8.5935015E-3,2.4358941E-2,4.541111E-2,1.0196421E-2,3.4142237E-2,7.4394094E-3,2.119695E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,73,75,77,79,81,-1,83,85,87,89,-1,91,93,-1,95,97,99,101,103,105,107,-1,109,111,-1,-1,113,-1,115,117,119,121,123,125,127,129,131,133,135,-1,-1,137,139,141,143,-1,145,147,149,-1,-1,151,153,155,157,-1,159,161,163,165,167,169,-1,171,173,175,-1,-1,177,179,-1,181,-1,183,-1,185,-1,-1,-1,187,189,191,-1,-1,193,-1,-1,195,197,-1,199,-1,201,-1,203,-1,-1,205,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,209,211,-1,-1,213,-1,215,217,-1,-1,-1,219,221,223,225,-1,227,229,231,233,235,237,239,241,243,245,-1,-1,-1,247,249,251,253,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3096144E2,3.106791E1,6.1558372E1,1.3137428E1,1.7788147E1,2.0920513E1,1.6824097E1,6.0346985E0,1.1584192E1,4.2465105E0,2.9314399E0,2.639062E0,1.7536516E1,1.4004433E1,3.4905167E0,1.5834723E0,3.5348206E0,1.2540278E0,4.095584E0,1.6919346E0,1.4048623E0,3.745199E-1,6.9324684E-1,9.7438526E-1,1.1430392E0,7.761055E0,6.7687426E0,8.104212E0,4.067051E0,1.0389957E0,1.262722E0,6.281557E-1,3.3252105E-1,2.8811646E0,1.386673E0,0E0,0E0,1.300547E0,1.8166447E0,1.3830328E0,4.2236757E-1,2.6947123E-1,4.333685E-1,0E0,2.0150788E-1,3.1418228E-1,7.03693E-2,4.2551327E-1,0E0,6.586137E-1,7.6511025E-2,0E0,7.9433727E0,4.534645E0,3.492649E0,3.432228E0,5.780709E-1,9.454109E-1,3.3983917E0,0E0,9.5199585E-1,1.0693207E0,0E0,0E0,5.446558E-1,0E0,6.5483555E-2,1.0266418E0,5.628917E-1,6.7780685E-1,3.4920692E-1,8.238965E-1,4.440088E-1,1.1287394E0,3.9364386E-1,1.0896397E-1,1.0245581E0,0E0,0E0,1.2878452E-1,1.1454654E-1,1.3449502E-1,1.5776834E-1,0E0,1.17795244E-1,9.555644E-2,3.6430454E-1,0E0,0E0,3.401785E-1,4.7322762E-1,3.0294824E-1,6.16137E-1,0E0,8.848548E-2,1.8836851E0,5.5463314E0,2.9382277E0,1.50412E0,3.046986E0,0E0,2.112856E0,1.6958201E0,3.964491E-1,0E0,0E0,4.7433567E-1,2.276989E0,0E0,3.9362717E-1,0E0,5.6495667E-1,0E0,2.9675674E-1,0E0,0E0,0E0,8.830414E-1,5.5110025E-1,1.3315703E-1,0E0,0E0,1.4422321E-1,0E0,0E0,2.711273E-1,3.488121E-1,0E0,4.9616003E-1,0E0,2.6085758E-1,0E0,1.522007E-1,0E0,0E0,3.4541702E-1,1.0463662E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.442669E-1,3.8508797E-1,0E0,0E0,2.0073307E-1,0E0,1.2603933E-1,1.6841602E-1,0E0,0E0,0E0,4.236169E-1,1.3963507E0,3.942213E0,5.6845E-1,0E0,1.6130605E0,5.9125054E-1,8.84007E-1,1.8100945E0,7.671118E-2,1.2757893E0,7.3104954E-1,7.895299E-1,8.627243E-1,7.039809E-2,0E0,0E0,0E0,1.4166241E0,3.7218475E-1,2.5319862E-1,8.034384E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,64,64,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,79,79,80,80,81,81,82,82,84,84,85,85,86,86,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,106,106,107,107,109,109,111,111,113,113,117,117,118,118,119,119,122,122,125,125,126,126,128,128,130,130,132,132,135,135,136,136,150,150,151,151,154,154,156,156,157,157,161,161,162,162,163,163,164,164,166,166,167,167,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,179,179,180,180,181,181,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,74,76,78,80,82,-1,84,86,88,90,-1,92,94,-1,96,98,100,102,104,106,108,-1,110,112,-1,-1,114,-1,116,118,120,122,124,126,128,130,132,134,136,-1,-1,138,140,142,144,-1,146,148,150,-1,-1,152,154,156,158,-1,160,162,164,166,168,170,-1,172,174,176,-1,-1,178,180,-1,182,-1,184,-1,186,-1,-1,-1,188,190,192,-1,-1,194,-1,-1,196,198,-1,200,-1,202,-1,204,-1,-1,206,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,210,212,-1,-1,214,-1,216,218,-1,-1,-1,220,222,224,226,-1,228,230,232,234,236,238,240,242,244,246,-1,-1,-1,248,250,252,254,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,2.0466278E6,1.0855529E3,8.8543115E2,4.54E2,2.2515285E6,2.0304577E-1,6.737509E7,1.294364E0,2.2898555E-1,7.805608E4,2E1,1.2679E4,5.964E3,1E0,6.9514094E11,1.942E3,7.76264E7,1.5401015E1,3.1294638E2,1.0800328E8,3.9715407E0,2.5534486E4,9.9625344E5,2.04E5,9.51668E5,3.8879236E7,2.943086E4,3.2212903E2,6.292039E0,6.473E3,1.2593641E6,4.759E3,6.971004E7,5.3288326E-2,7.481286E-3,6.316106E6,3.970405E3,5.0829604E3,1.5669752E-2,1.3E1,1.5E1,-8.568654E-3,1E0,2.4513194E5,6.2186885E-1,7.2896E5,-4.150994E-2,3.1E1,8.5E1,4.5990452E-2,1.7322648E5,1.477425E6,9.256843E3,2.2E1,3.202814E5,2.5E1,7.466243E0,7.6220133E-3,1.3697374E7,1E0,1.1184101E-2,-4.7442266E-3,1E0,-1.2162829E-2,2.4E1,1.1164689E0,1.2783833E3,6.41689E5,4.0950096E10,1.1108876E11,1.2399164E5,8.931957E9,1.272E3,9.374544E5,4.5808794E5,-3.2707125E-2,-1.3881701E-2,3.7333333E0,3E1,1.8991614E5,7.576E3,9.943654E-3,1.195E3,2.650885E6,1.51529E3,3.5946143E-3,1.1966813E-2,3.64299E5,1E0,1.6E1,2.653E3,1.4430088E-3,1E1,1.8731416E4,7.780377E6,2.6E1,5.1E1,5.2288747E-1,4.5395922E-2,5.202E3,2.3544883E8,5.557511E8,5.456655E-3,-2.365354E-2,2.5503985E9,1E0,4.6053044E-2,5.513E3,8.275696E-3,8.7757526E2,2.0100791E-2,2.7056329E10,3.3327076E-4,5.508208E-3,-2.1223156E-3,1E0,1.087037E1,9.34731E5,-1.679053E-2,-5.575634E-2,6.938E3,-6.570885E-2,-2.854395E-2,2.6835732E3,5.491E3,-6.258329E-3,1.348651E6,-2.2861606E-2,2.002E3,-8.549736E-4,2.0783027E5,-1.8790292E-2,-3.2663103E-2,3.5259784E2,8.4569194E10,-7.280161E-3,4.585636E-3,-1.3228938E-2,-4.03113E-3,2.1404733E-3,1.3605945E-2,7.548258E-3,-1.1630357E-3,-8.571168E-3,3.3831499E-3,1.5222905E-2,2.5859808E-3,7.1008615E-3,6.731367E0,3.79E2,-2.9725581E-2,8.3307475E-3,1.401805E1,2.444829E-3,4.6E2,7.997723E6,4.3002404E-3,4.332565E-4,1.19262785E-2,2.132956E8,8.61E2,3.6129813E2,1E0,-3.3740956E-2,1.72792E-4,1.3833216E-2,2.53E2,1.5963264E9,1.964127E6,2.2075728E7,2.7272727E0,1.6073539E7,9.6696576E8,1.8722344E8,-3.9052656E-3,1.5152307E-2,-4.864296E-3,2.8330308E7,2.8034655E6,1.513E3,4.1844E4,2.6945613E-2,5.1040296E-2,-1.576372E-2,-2.6204072E-2,-3.414904E-2,-1.451455E-2,-2.779695E-2,-1.274151E-2,4.669611E-3,-7.3959893E-3,-3.158888E-2,-1.0440171E-2,1.3119557E-2,-5.3871595E-3,1.1780618E-3,-1.4824611E-2,2.3242418E-2,3.5523465E-3,-1.7078303E-2,-5.061707E-2,-8.519664E-3,-2.1999428E-2,-1.4693262E-2,-8.769839E-3,-1.1244684E-2,2.0902664E-3,3.337573E-2,2.0055698E-2,-1.988688E-2,-1.1676719E-2,-6.241204E-3,-1.8064337E-2,-8.944691E-3,1.3467025E-4,-1.38497865E-2,-2.038049E-3,-8.87578E-3,-2.5016593E-2,-9.071758E-3,5.192641E-3,7.0018834E-3,1.7308198E-2,1.6106015E-2,3.901399E-2,-2.0245846E-2,-8.144125E-3,1.8332466E-2,3.7793464E-3,8.245143E-4,-1.4161403E-2,2.2124895E-3,1.8453006E-2,-8.5808905E-3,-2.5989376E-2,1.41789345E-2,-8.035276E-3,-1.0495675E-3,2.6442414E-2,-1.7801289E-2,1.2849254E-3,7.895978E-3,2.1348808E-2,-1.0379115E-2,-1.9347116E-2,2.4214542E-2,8.5935015E-3,2.4358941E-2,4.541111E-2,1.0196421E-2,3.4142237E-2,7.4394094E-3,2.119695E-2],"split_indices":[19,109,50,51,51,2,27,26,7,41,38,27,3,2,2,15,30,0,44,53,51,7,56,32,27,5,1,44,32,4,37,9,31,2,44,0,0,1,51,50,37,3,3,0,107,27,26,9,0,10,10,0,27,9,4,3,27,3,34,0,44,6,0,0,58,0,8,38,4,28,5,30,32,12,0,27,27,0,0,53,3,27,9,0,0,11,4,0,0,9,84,3,28,0,3,27,27,28,3,26,0,2,12,7,0,0,7,79,0,2,0,51,0,30,0,0,0,58,57,1,0,0,2,0,0,4,9,0,28,0,0,0,32,0,0,51,30,0,0,0,0,0,0,0,0,0,0,0,0,0,53,1,0,0,57,0,28,9,0,0,0,30,2,51,109,0,41,37,10,7,1,44,55,12,7,7,0,0,0,42,46,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.261E3,7.22E2,1.539E3,3.94E2,3.28E2,9.8E2,5.59E2,2.94E2,1E2,2.59E2,6.9E1,1.86E2,7.94E2,4.49E2,1.1E2,7.5E1,2.19E2,7E0,9.3E1,2.02E2,5.7E1,1.7E1,5.2E1,9E1,9.6E1,4.6E2,3.34E2,2.78E2,1.71E2,5.7E1,5.3E1,5.6E1,1.9E1,1.82E2,3.7E1,5E0,2E0,4.7E1,4.6E1,1.85E2,1.7E1,2.3E1,3.4E1,7E0,1E1,4.2E1,1E1,8.7E1,3E0,8.4E1,1.2E1,1.1E1,4.49E2,1.88E2,1.46E2,2.47E2,3.1E1,1E1,1.61E2,6E0,5.1E1,5E1,3E0,5E0,5.1E1,8E0,1.1E1,1.65E2,1.7E1,2.4E1,1.3E1,3.1E1,1.6E1,3E1,1.6E1,1.7E1,1.68E2,1.1E1,6E0,6E0,1.7E1,1.5E1,1.9E1,5E0,5E0,9E0,3.3E1,5E0,5E0,7.1E1,1.6E1,2.8E1,5.6E1,6E0,6E0,7.2E1,3.77E2,1.31E2,5.7E1,1.43E2,3E0,1.41E2,1.06E2,2.8E1,3E0,2E0,8E0,1.42E2,1.9E1,4.6E1,5E0,4.5E1,5E0,4.9E1,2E0,2E0,9E0,1.39E2,2.6E1,6E0,1.1E1,4E0,2E1,1E1,3E0,5E0,2.6E1,4E0,1.2E1,2.2E1,8E0,6E0,1E1,1.2E1,5E0,1.21E2,4.7E1,2E0,4E0,1.2E1,5E0,3E0,1.2E1,5E0,1.4E1,2E0,3E0,7E0,2E0,3E0,3E1,6.5E1,6E0,2E0,1.4E1,1.1E1,1.7E1,5.2E1,4E0,2E0,4E0,1.6E1,5.6E1,3.57E2,2E1,1.3E1,1.18E2,2.4E1,3.3E1,1.35E2,8E0,9.7E1,4.4E1,4.3E1,6.3E1,2.1E1,7E0,4E0,4E0,1.22E2,2E1,3.9E1,7E0,6E0,3.9E1,1.4E1,3.5E1,1.32E2,7E0,1.4E1,1.2E1,4E0,2E0,1.8E1,2E0,3E0,2E0,3E0,2.3E1,3E0,9E0,2E0,6E0,6E0,4E0,8.7E1,3.4E1,2.5E1,2.2E1,1.3E1,1.7E1,4E1,2.5E1,8E0,6E0,1.3E1,4E0,4.9E1,3E0,8E0,8E0,2.3E1,3.3E1,2.46E2,1.11E2,5E0,1.5E1,3.1E1,8.7E1,9E0,1.5E1,2.1E1,1.2E1,1.2E2,1.5E1,2E0,6E0,9.2E1,5E0,2E0,4.2E1,4E0,3.9E1,5.1E1,1.2E1,7E0,1.4E1,1.07E2,1.5E1,7E0,1.3E1,2E0,3.7E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"255","size_leaf_vector":"1"}},{"base_weights":[-9.375168E-3,-3.6737326E-1,1.5923801E-1,-5.445983E-1,-1.3405174E-1,3.2807466E-2,4.307237E-1,-6.3205004E-1,-1.9767769E-1,-2.2858284E-1,3.2358044E-1,-4.4081714E-2,2.5759265E-1,-1.3121367E-1,4.9388844E-1,-5.232042E-1,-8.604098E-1,-2.5837928E-1,4.8938837E-2,-3.1536302E-1,-6.9360785E-2,4.007362E-2,4.3159398E-1,-2.4310288E-1,2.906256E-2,2.8625008E-1,-3.752325E-1,-2.7139783E-1,1.330803E-1,4.2441007E-1,7.9486823E-1,-6.539568E-1,-2.0705335E-1,-1.1092181E0,-6.808803E-1,-1.5639412E-1,-5.829418E-1,-4.0856183E-1,-2.1263546E-1,4.798787E-2,-2.2604744E-1,1.3942817E-1,-9.344182E-3,4.8430607E-2,4.8651713E-1,-4.0609273E-1,-1.7823772E-1,5.9227858E-2,-4.992811E-1,1.6738147E-1,4.351955E-1,-1.7663339E-1,-2.7976789E-2,-4.3137237E-1,-1.5944228E-1,3.5614434E-1,-1.2955533E-1,3.5766482E-1,6.8917376E-1,2.3427749E-1,8.4473115E-1,-4.2685115E-1,-6.95456E-1,-3.13286E-1,4.0565144E-2,-6.762151E-1,-1.1912125E0,-5.543701E-1,-1.017856E0,-2.3001322E-1,3.108775E-1,-4.3512943E-1,-4.656224E-2,-3.7436405E-1,-3.166775E-2,-2.4761066E-1,-7.950833E-3,1.0178269E-2,1.1957791E-2,-3.967705E-1,4.6568695E-3,2.0915598E-3,1.0184774E-2,9.839821E-3,-4.090898E-3,2.9166487E-1,5.9325117E-1,-4.4376633E-1,-1.7711553E-1,-1.4409988E-1,-7.0463896E-1,3.270782E-2,6.845783E-1,-6.0429263E-1,-1.754426E-1,1.3626187E-1,5.3125834E-1,3.845994E-1,3.0119449E-2,6.016338E-3,-3.019549E-1,-5.889298E-1,-2.1219034E-1,1.5222444E-3,-2.068472E-1,1.0718734E-3,1.9753123E-2,-9.100167E-3,3.6372796E-3,3.740679E-1,-4.7562551E-1,2.193247E-1,7.7165735E-1,1.5330887E-2,-2.3773315E-3,8.9973503E-1,3.4751055E-1,-6.758322E-3,-5.165386E-1,-7.313113E-1,-4.7111067E-1,-2.0528586E-1,-4.488219E-1,2.3486529E-2,-6.7955054E-2,-9.538585E-3,-3.9563335E-2,-2.5152408E-2,-5.590454E-2,-6.160937E-1,-2.1844633E-1,-4.8578966E-2,-1.9070733E-2,-4.7896218E-2,-3.327116E-1,2.6120285E-2,3.6984533E-3,-5.950986E-1,-6.539399E-3,-2.236021E-2,-3.0415642E-1,-8.580108E-2,-2.8300932E-1,-8.150484E-3,1.7974454E-3,2.1139647E-1,-4.3041736E-2,-1.8931276E-1,-4.9479377E-1,-1.1010281E-1,1.6186932E-1,1.7625915E-2,6.2657367E-3,3.9310478E-2,4.5303702E-1,-5.6740693E-3,-4.544236E-1,1.2486315E-4,-1.0070222E-2,-6.266941E-2,-2.1219912E-1,-1.7509256E-2,-4.5515142E-2,8.738966E-1,1.2851012E-2,5.9219215E-2,4.7336805E-1,-2.915666E-2,-1.1529299E-2,-1.0900684E-2,1.6059879E-3,-1.3278149E-1,1.6381352E-1,3.0454265E-2,1.3641368E-2,3.354457E-1,2.6088301E-2,-1.969647E-2,-5.733683E-3,-3.13626E-2,-9.943076E-3,-1.3315985E-2,-2.2231227E-4,-1.0772671E-2,7.962125E-4,4.2959055E-1,1.692277E-1,-3.2880303E-2,-7.639939E-3,2.0809764E-2,-1.7992845E-2,6.411565E-1,1.0904624E0,7.582615E-1,1.0182309E0,2.3058347E-2,5.2613663E-3,-3.1170135E-2,-1.5598633E-2,-3.4399456E-3,-3.3617523E-2,-1.0581566E-2,-2.7128905E-2,-6.0668928E-3,-1.5433E-2,-2.701799E-2,-1.3603074E-2,1.970646E-3,-1.3499514E-2,-2.2032963E-2,-3.2651376E-2,-2.286501E-3,-1.5515368E-2,1.3772972E-3,-1.3790305E-2,-1.751728E-2,-8.116263E-3,-3.5530757E-2,-1.5295144E-2,-1.5980307E-2,-6.472636E-3,-7.079407E-3,3.2820643E-3,-1.5926791E-2,-7.5174095E-3,1.2881557E-2,1.3841569E-3,4.583498E-3,-4.4452823E-3,2.2321899E-4,-1.111544E-2,-3.1046921E-2,-1.6434947E-2,6.725689E-3,-8.334416E-3,1.0869938E-2,1.069504E-3,1.2148987E-2,2.4312949E-2,-2.2741843E-2,-1.4815605E-2,-5.342843E-3,1.0725192E-3,-1.3200755E-2,-4.8148553E-3,1.5107772E-2,4.294537E-2,5.9340126E-3,-3.531573E-3,2.5191072E-2,-5.764937E-3,9.4464386E-4,-2.951371E-2,4.9746362E-3,1.0607923E-2,2.0027423E-2,1.0991419E-2,1.3252E-2,2.2714082E-2,5.4689012E-3,1.851678E-2,4.957594E-3,-6.3831564E-3,3.1138834E-2,-1.0908002E-2,2.2860564E-2,5.3728495E-2,3.560499E-2,9.280992E-3,4.8474833E-2,2.3101576E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,113,115,117,119,121,123,125,127,129,131,133,135,137,-1,139,-1,141,143,145,-1,147,149,-1,-1,-1,-1,151,153,155,157,159,161,163,165,167,169,171,173,175,-1,-1,177,179,181,-1,183,-1,-1,-1,-1,185,187,189,191,-1,-1,193,195,-1,197,199,201,203,205,-1,207,-1,-1,-1,-1,209,211,-1,-1,213,215,-1,-1,217,-1,-1,219,221,223,-1,-1,225,227,229,231,233,235,-1,-1,-1,237,-1,239,-1,-1,241,243,-1,-1,245,247,-1,249,-1,-1,-1,-1,251,253,-1,-1,255,-1,-1,-1,-1,-1,-1,-1,-1,-1,257,259,-1,-1,-1,261,263,265,267,269,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3611713E2,2.978199E1,5.260469E1,1.2313972E1,1.3614494E1,1.810723E1,1.735067E1,7.6777496E0,6.7989297E0,3.5706282E0,1.6363239E0,1.1376285E1,4.949497E0,1.8886092E0,8.821922E0,9.124199E0,3.8902664E0,2.5899758E0,0E0,1.521574E0,1.7208357E0,4.205766E-1,7.9717445E-1,2.1478148E0,9.145904E0,4.4596195E0,4.6927917E-1,5.222523E-1,1.1006625E0,6.0799866E0,2.0662727E0,1.1922455E0,1.7751548E0,6.8564606E-1,2.1214771E0,2.2387133E0,8.036852E-1,6.036539E-1,5.850601E-1,4.3859428E-1,1.5773096E0,8.5295975E-2,0E0,1.6029516E-1,5.340805E-1,4.303589E-1,2.6299329E0,8.966308E0,9.481921E-1,1.5582232E0,1.0595627E0,3.5044295E-1,0E0,3.2397342E-1,1.99152E-1,2.561705E-1,1.5111542E-1,4.114456E0,2.5823479E0,2.5718546E-1,1.7111588E0,5.950575E-1,7.172775E-1,5.9584427E-1,1.1250334E0,6.4399624E-1,2.5613785E-1,8.606672E-1,7.4430466E-2,1.0127366E0,5.0166905E-1,6.580634E-1,0E0,5.210953E-1,0E0,3.7666607E-1,1.1374452E-1,5.124141E-1,0E0,3.740499E-1,3.42572E-1,0E0,0E0,0E0,0E0,1.5565836E-1,4.164586E-1,1.1273289E-1,8.211973E-2,7.8523254E-1,4.7149563E-1,8.691203E0,2.2777243E0,1.5535355E-1,1.3255087E-1,1.0150383E0,1.5500736E-1,7.0155334E-1,0E0,0E0,8.490217E-2,1.16202116E-1,1.1743277E-1,0E0,1.2283105E-1,0E0,0E0,0E0,0E0,3.1459389E0,3.389138E-1,6.9264555E-1,1.7971039E0,0E0,0E0,3.3433914E-1,2.7762282E-1,0E0,3.542652E-1,7.674179E-1,4.985757E-1,2.2495246E-1,2.5967884E-1,0E0,4.749543E-1,0E0,0E0,0E0,0E0,1.9317913E-1,1.7049938E-1,0E0,0E0,4.3145418E-1,2.2174692E-1,0E0,0E0,2.2288704E-1,0E0,0E0,3.665495E-1,1.6772118E-1,4.0013647E-1,0E0,0E0,1.2353432E-1,3.1228018E-1,9.946287E-2,1.2854695E-1,2.3785456E-1,8.084093E-2,0E0,0E0,0E0,1.04067326E-1,0E0,1.2798786E-1,0E0,0E0,3.230073E-1,6.243415E-1,0E0,0E0,1.6750908E-1,5.488013E0,0E0,9.6880007E-1,0E0,0E0,0E0,0E0,1.0504751E0,4.3809533E-1,0E0,0E0,6.6541576E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0445175E0,6.7056096E-1,0E0,0E0,0E0,1.2520896E-1,2.0166645E0,3.0274963E-1,3.687439E-1,3.3321E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,73,73,75,75,76,76,77,77,79,79,80,80,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,100,100,101,101,102,102,104,104,109,109,110,110,111,111,112,112,115,115,116,116,118,118,119,119,120,120,121,121,122,122,124,124,129,129,130,130,133,133,134,134,137,137,140,140,141,141,142,142,145,145,146,146,147,147,148,148,149,149,150,150,154,154,156,156,159,159,160,160,163,163,164,164,166,166,171,171,172,172,175,175,185,185,186,186,190,190,191,191,192,192,193,193,194,194],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,114,116,118,120,122,124,126,128,130,132,134,136,138,-1,140,-1,142,144,146,-1,148,150,-1,-1,-1,-1,152,154,156,158,160,162,164,166,168,170,172,174,176,-1,-1,178,180,182,-1,184,-1,-1,-1,-1,186,188,190,192,-1,-1,194,196,-1,198,200,202,204,206,-1,208,-1,-1,-1,-1,210,212,-1,-1,214,216,-1,-1,218,-1,-1,220,222,224,-1,-1,226,228,230,232,234,236,-1,-1,-1,238,-1,240,-1,-1,242,244,-1,-1,246,248,-1,250,-1,-1,-1,-1,252,254,-1,-1,256,-1,-1,-1,-1,-1,-1,-1,-1,-1,258,260,-1,-1,-1,262,264,266,268,270,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,2.9892595E6,2.6295085E3,8.795748E6,1.022332E6,2.8593832E-1,3.653602E7,1.9013363E4,3.3926086E2,3.8657358E6,5.52E2,6.4888096E0,2.6E1,2.657819E6,2.291E3,9.966725E2,1.2427474E8,4.8938837E-2,1.8329959E-1,1.3304372E7,1.5E1,2.2783158E5,2.255481E6,5.5544252E7,1.06E3,1.0658642E1,3.386E3,5.765574E0,5.185489E3,1E0,1.3177E4,3.3709784E5,6.5029144E5,1.13808104E8,1.3554651E2,7.6008E4,1E0,9.705292E6,7.2041174E2,1.7488463E3,1.4964847E3,-9.344182E-3,1.6081998E5,5.311576E6,4.8285713E1,4.5184364E0,1.1392E4,3.0795444E3,7.780377E6,5.316494E6,2.0919E4,-2.7976789E-2,2.148972E5,2.94E2,9.536863E4,2.7E2,1.1803382E4,1.5988182E-1,4.38E2,1E0,2.85E2,4.750825E2,2.2668628E7,3.68E2,1.9166666E-1,3.830986E-1,1.1879E4,4.5E1,5.418831E11,1.465E3,6.5317163E3,-4.656224E-2,1.074E3,-3.166775E-2,1.04E2,6E0,1.5851064E0,1.1957791E-2,2.65851E5,1.3593E4,2.0915598E-3,1.0184774E-2,9.839821E-3,-4.090898E-3,2.093563E0,4.7777777E0,4E0,1.00008E5,9.25E0,1E0,8E0,7.969174E7,7.494145E-2,1.0979587E8,1.198E5,4.61E2,2.732E3,3.0119449E-2,6.016338E-3,1.47E2,4.2E1,1.4887473E-1,1.5222444E-3,1.0052E4,1.0718734E-3,1.9753123E-2,-9.100167E-3,3.6372796E-3,9.74026E0,3E1,1.3651616E5,1E0,1.5330887E-2,-2.3773315E-3,7.123E3,3.8807898E6,-6.758322E-3,1.9987492E5,2.7549994E5,1.0894445E-2,1.2453214E0,1.0201212E3,2.3486529E-2,3.1790106E8,-9.538585E-3,-3.9563335E-2,-2.5152408E-2,-5.590454E-2,1.0509E4,1.612E3,-4.8578966E-2,-1.9070733E-2,5.0835518E1,2.2153166E7,2.6120285E-2,3.6984533E-3,2E1,-6.539399E-3,-2.236021E-2,1.8E1,1.11942E5,1.0681705E0,-8.150484E-3,1.7974454E-3,3.2931E4,2.1252015E6,1.2342285E1,1.116567E3,1.22E2,5.125467E-1,1.7625915E-2,6.2657367E-3,3.9310478E-2,7.6188426E0,-5.6740693E-3,2.1515152E0,1.2486315E-4,-1.0070222E-2,3.6E2,1.2513207E2,-1.7509256E-2,-4.5515142E-2,2.127E3,1.9E1,5.9219215E-2,1.6503105E3,-2.915666E-2,-1.1529299E-2,-1.0900684E-2,1.6059879E-3,1.3414634E0,2.3314082E6,3.0454265E-2,1.3641368E-2,3.1415796E0,2.6088301E-2,-1.969647E-2,-5.733683E-3,-3.13626E-2,-9.943076E-3,-1.3315985E-2,-2.2231227E-4,-1.0772671E-2,7.962125E-4,1E0,1.3359244E9,-3.2880303E-2,-7.639939E-3,2.0809764E-2,2.3E1,3.7826266E4,2.3553647E5,2.637815E1,3.0501078E8,2.3058347E-2,5.2613663E-3,-3.1170135E-2,-1.5598633E-2,-3.4399456E-3,-3.3617523E-2,-1.0581566E-2,-2.7128905E-2,-6.0668928E-3,-1.5433E-2,-2.701799E-2,-1.3603074E-2,1.970646E-3,-1.3499514E-2,-2.2032963E-2,-3.2651376E-2,-2.286501E-3,-1.5515368E-2,1.3772972E-3,-1.3790305E-2,-1.751728E-2,-8.116263E-3,-3.5530757E-2,-1.5295144E-2,-1.5980307E-2,-6.472636E-3,-7.079407E-3,3.2820643E-3,-1.5926791E-2,-7.5174095E-3,1.2881557E-2,1.3841569E-3,4.583498E-3,-4.4452823E-3,2.2321899E-4,-1.111544E-2,-3.1046921E-2,-1.6434947E-2,6.725689E-3,-8.334416E-3,1.0869938E-2,1.069504E-3,1.2148987E-2,2.4312949E-2,-2.2741843E-2,-1.4815605E-2,-5.342843E-3,1.0725192E-3,-1.3200755E-2,-4.8148553E-3,1.5107772E-2,4.294537E-2,5.9340126E-3,-3.531573E-3,2.5191072E-2,-5.764937E-3,9.4464386E-4,-2.951371E-2,4.9746362E-3,1.0607923E-2,2.0027423E-2,1.0991419E-2,1.3252E-2,2.2714082E-2,5.4689012E-3,1.851678E-2,4.957594E-3,-6.3831564E-3,3.1138834E-2,-1.0908002E-2,2.2860564E-2,5.3728495E-2,3.560499E-2,9.280992E-3,4.8474833E-2,2.3101576E-2],"split_indices":[19,109,50,4,49,27,41,44,51,51,31,2,52,8,27,2,4,44,0,40,44,8,27,30,44,2,52,2,52,51,107,9,32,27,44,55,10,82,9,51,4,4,0,27,1,54,52,2,4,27,27,12,0,32,0,27,3,4,37,0,6,0,54,44,0,57,55,2,3,30,10,4,0,2,0,10,3,52,0,28,10,0,0,0,0,38,53,0,5,57,6,17,7,56,7,7,0,2,0,0,0,6,37,0,2,0,0,0,0,52,3,27,79,0,0,2,27,0,46,44,37,41,4,0,7,0,0,0,0,10,0,0,0,56,1,0,0,8,0,0,8,1,41,0,0,9,31,55,4,10,56,0,0,0,53,0,52,0,0,2,51,0,0,1,3,0,51,0,0,0,0,52,27,0,0,53,0,0,0,0,0,0,0,0,0,109,7,0,0,0,3,51,32,55,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.253E3,7.21E2,1.532E3,4.09E2,3.12E2,1.046E3,4.86E2,3.26E2,8.3E1,2.59E2,5.3E1,7.8E2,2.66E2,4.9E1,4.37E2,2.23E2,1.03E2,8E1,3E0,1.67E2,9.2E1,1.5E1,3.8E1,2.09E2,5.71E2,2.55E2,1.1E1,3.2E1,1.7E1,3.57E2,8E1,1.57E2,6.6E1,4.1E1,6.2E1,6.2E1,1.8E1,8.6E1,8.1E1,5.3E1,3.9E1,1.1E1,4E0,5E0,3.3E1,5.8E1,1.51E2,5.41E2,3E1,1.43E2,1.12E2,7E0,4E0,1.2E1,2E1,9E0,8E0,2.87E2,7E1,7E0,7.3E1,2.6E1,1.31E2,4.6E1,2E1,8E0,3.3E1,4.7E1,1.5E1,5.4E1,8E0,1.5E1,3E0,7.9E1,7E0,6.9E1,1.2E1,4.6E1,7E0,2.2E1,1.7E1,6E0,5E0,2E0,3E0,1.3E1,2E1,4.9E1,9E0,1.43E2,8E0,5.2E2,2.1E1,2.2E1,8E0,1.33E2,1E1,9.4E1,1.8E1,2E0,5E0,6E0,6E0,4E0,1.6E1,2E0,7E0,6E0,2E0,2.82E2,5E0,1.1E1,5.9E1,5E0,2E0,6.5E1,8E0,7E0,1.9E1,1.11E2,2E1,2.7E1,1.9E1,3E0,1.7E1,3E0,5E0,3E0,3E1,3.9E1,8E0,1.3E1,2E0,2E1,3.4E1,3E0,5E0,9E0,6E0,2.7E1,5.2E1,1.3E1,5.6E1,2E0,1E1,9E0,3.7E1,8E0,1.4E1,1E1,7E0,7E0,6E0,5E0,1.5E1,2E0,4.7E1,2E0,7E0,6.6E1,7.7E1,5E0,3E0,1.1E1,5.09E2,4E0,1.7E1,1.9E1,3E0,6E0,2E0,1.2E1,1.21E2,5E0,5E0,7.7E1,1.7E1,2E0,3E0,4E0,2E0,4E0,2E0,1.4E1,2E0,2.21E2,6.1E1,2E0,3E0,5E0,6E0,4.4E1,1.5E1,3.3E1,3.2E1,4E0,4E0,8E0,1.1E1,2E0,1.09E2,8E0,1.2E1,1.9E1,8E0,8E0,1.1E1,1.2E1,5E0,2E1,1.9E1,4E0,4E0,1.6E1,4E0,2.4E1,1E1,4E0,5E0,3.9E1,1.3E1,9E0,4E0,3.4E1,2.2E1,6E0,3E0,1E1,2.7E1,2E0,6E0,4E0,1E1,2E0,8E0,4E0,3E0,6E0,9E0,3.2E1,1.5E1,4E1,2.6E1,4.3E1,3.4E1,2E0,9E0,2.21E2,2.88E2,1.5E1,2E0,1E1,2E0,7E1,5.1E1,3.4E1,4.3E1,7.8E1,1.43E2,5.2E1,9E0,3E0,3E0,4.2E1,2E0,3E0,1.2E1,3.1E1,2E0,2.8E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"271","size_leaf_vector":"1"}},{"base_weights":[3.0032033E-3,-3.401386E-1,1.5949398E-1,-5.1062E-1,-1.3708381E-1,7.225009E-2,5.3503484E-1,-6.16485E-1,-2.9129583E-1,-2.6315236E-1,2.980966E-1,-6.482942E-2,2.6066053E-1,6.432305E-1,2.0902763E-1,-6.325934E-1,2.4100633E-2,-3.8098517E-1,-1.328814E-1,-2.8006166E-1,5.3811796E-2,2.1975455E-1,6.6496557E-1,-3.769708E-1,-2.0995071E-2,1.22939676E-1,3.8661718E-1,9.485047E-1,5.480984E-1,4.75429E-2,5.107172E-1,-4.9761564E-2,-6.1033344E-1,-4.7779226E-1,-2.5477156E-1,-2.0225006E-1,1.0622347E-1,-1.8822658E-1,-5.0758696E-1,1.2801167E-1,4.2670947E-1,9.368054E-3,3.331632E-2,-7.415412E-1,-3.199936E-1,4.712747E-3,-3.847816E-1,1.2259719E-2,3.1928995E-1,4.2043105E-1,-7.865035E-2,1.0071908E0,3.6005548E-1,4.8653623E-1,9.043472E-1,-4.1584975E-1,1.1109669E-1,3.33614E-1,7.057927E-1,-6.3684326E-1,-3.7957716E-1,-1.295004E-1,-2.3098497E-2,1.2187199E-3,-2.7213332E-1,-3.1614396E-1,-8.3705865E-2,8.635209E-3,-1.5091491E-3,-2.4008968E-1,2.4750347E-1,-7.100118E-1,-3.909395E-1,2.58334E-1,-3.4775715E-2,1.026639E-2,5.333344E-1,-1.1479909E-2,-3.7206415E-2,-3.4009284E-1,5.9689647E-3,-1.5908366E-1,5.274922E-2,-2.93729E-1,-4.444156E-2,4.5912955E-2,-2.0133467E-2,2.4821734E-1,5.680723E-1,3.0702534E-1,6.022296E-1,9.072606E-2,-4.607536E-1,1.0502919E0,2.0868765E-2,2.098385E-2,4.6238E-3,3.8371345E-1,6.3759065E-1,4.3030486E-2,1.2685726E-2,-2.6838101E-2,-1.8701837E-3,2.644297E-1,-9.880267E-3,7.0417374E-3,2.33121E-2,3.7544508E-2,1.278455E-2,-2.026121E-1,-6.5068233E-1,-2.2808127E-3,-4.1862798E-1,-1.4075372E-2,4.3037143E-3,-3.2295048E-1,-8.431589E-3,-2.2441171E-2,-1.5683544E-1,3.1470316E-3,-1.4339119E-1,-3.8243443E-1,-1.6098136E-1,4.192598E-2,4.7126448E-1,-3.3995137E-2,-1.4395751E-2,-2.243943E-1,-5.477284E-1,2.8692275E-1,-1.694517E-3,-4.5746714E-3,7.725759E-2,2.7005963E-2,7.518002E-3,-3.4770748E-1,-1.9910135E-3,-1.1556669E-1,-4.9081156E-1,8.1725264E-1,2.8136414E-2,-2.1238027E-2,-1.6658555E-1,-1.16488166E-1,1.3417426E-1,2.1684211E-1,4.6819195E-1,1.2565063E-2,3.0823322E-2,3.306893E-1,-1.16229646E-1,7.030704E-2,6.3729954E-1,-9.111518E-3,1.0306298E-2,-2.7040545E-2,-5.666208E-3,1.1022291E0,2.9336793E-2,4.4764242E-1,2.7594352E-1,3.4521695E-2,2.3545856E-2,8.9812525E-2,2.030399E-2,-7.140765E-3,1.3872297E-1,-1.451142E-4,-1.4560788E-2,-3.023017E-2,-1.4536078E-2,-2.5109679E-2,-1.0703729E-2,-1.566595E-2,-3.4792551E-3,5.5146596E-4,-8.862284E-3,-7.930381E-3,1.4260593E-3,-1.4138156E-2,-3.1977125E-2,-9.187217E-3,-1.6731189E-3,5.9797186E-3,-1.0979255E-2,2.5886407E-2,3.6063904E-3,-1.3215752E-2,4.09877E-3,-2.792823E-2,-7.942737E-3,8.0277845E-3,1.645643E-2,-3.8787161E-3,6.603033E-3,-7.438108E-3,-1.6250774E-2,-1.0151817E-2,-1.9480491E-3,-2.477762E-2,-8.995811E-3,1.4644465E-2,3.9505847E-2,5.966267E-3,-1.9740958E-3,-3.419681E-3,-1.4843848E-2,-3.2444138E-3,-2.9323729E-2,7.0745866E-3,-1.0270991E-2,1.0623749E-2,-1.8235692E-3,5.230358E-3,2.5647352E-2,1.1204507E-2,2.1512562E-2,-1.8939724E-2,4.7332207E-3,-2.0090465E-3,7.579432E-3,3.0488659E-2,1.58173E-2,3.3027127E-2,5.382434E-2,2.1608593E-2,8.360387E-3,1.3903523E-2,-7.186467E-3,9.770272E-3,-5.890742E-4,1.4093764E-3,1.1728661E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,73,75,-1,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,115,117,119,-1,-1,121,123,125,127,129,131,-1,133,-1,-1,135,-1,137,139,141,-1,143,-1,145,147,149,151,153,155,157,-1,-1,-1,159,161,-1,-1,-1,-1,163,165,-1,-1,-1,-1,167,169,-1,171,-1,-1,173,-1,-1,175,-1,177,179,181,183,185,-1,-1,187,189,191,-1,-1,193,-1,-1,195,-1,197,199,201,203,-1,205,207,209,211,213,-1,-1,215,217,219,221,-1,-1,-1,-1,223,-1,225,227,-1,-1,229,-1,-1,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1743924E2,2.36427E1,4.9172337E1,8.457787E0,1.730529E1,3.1526093E1,9.847687E0,4.963135E0,1.696969E0,6.322542E0,1.8945017E0,9.665621E0,8.868752E0,5.636902E0,3.4834337E0,1.8942413E0,0E0,8.313055E-1,7.844576E-1,4.9610023E0,0E0,1.0845745E0,2.3753262E-1,1.5698948E0,5.8095036E0,5.3638997E0,4.249378E0,1.3295135E0,3.1567574E0,1.4598168E0,6.282501E-1,0E0,1.2141876E0,4.1919518E-1,1.8338847E-1,4.5958948E-1,1.3608994E-1,3.9915295E0,1.3516788E0,9.2122155E-1,2.6281E-1,0E0,0E0,2.2246027E-1,7.441168E-1,4.579817E0,1.9397407E0,2.465564E0,1.4326572E0,4.969345E0,1.2714686E0,5.517998E-1,1.1272937E-1,1.9608688E0,3.9546967E-1,3.731495E-1,8.002523E-1,4.0418994E-1,3.409438E-1,1.1421432E0,3.260088E-1,2.3322123E-1,0E0,0E0,8.8268995E-2,4.3707561E-1,1.8371825E-1,0E0,0E0,1.6950979E0,8.5424626E-1,1.4573765E-1,1.0782642E0,2.1367025E-1,1.5629205E-1,0E0,1.5614557E-1,0E0,0E0,1.5156555E-1,0E0,1.848803E0,8.444465E0,7.4685574E-1,0E0,2.151356E0,0E0,3.885007E-1,4.334321E-1,1.6009159E0,1.727375E0,6.054902E-1,2.0012915E-1,1.2156677E-1,0E0,0E0,0E0,4.7587204E-1,4.2549706E-1,0E0,0E0,0E0,0E0,5.703157E-1,5.711294E-1,0E0,0E0,0E0,0E0,1.8880934E-1,9.366455E-1,0E0,4.6354914E-1,0E0,0E0,8.698535E-2,0E0,0E0,7.472965E-2,0E0,8.331251E-2,1.0290766E0,5.2311516E-1,3.1409618E-1,3.0869198E-1,0E0,0E0,5.3669965E-1,4.8383284E-1,1.2016845E-1,0E0,0E0,1.0189594E-1,0E0,0E0,8.940697E-2,0E0,9.204699E-1,1.3869381E-1,1.00336075E-1,3.2550466E0,0E0,3.1299257E-1,1.215625E0,8.089683E-1,3.02572E-1,2.1189225E-1,0E0,0E0,1.66156E0,6.4401567E-1,8.53831E-2,6.9049835E-1,0E0,0E0,0E0,0E0,2.3058319E-1,0E0,3.2502556E-1,5.227151E-1,0E0,0E0,1.4531144E-1,0E0,0E0,1.6025174E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,64,64,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,76,76,79,79,81,81,82,82,83,83,85,85,87,87,88,88,89,89,90,90,91,91,92,92,93,93,97,97,98,98,103,103,104,104,109,109,110,110,112,112,115,115,118,118,120,120,121,121,122,122,123,123,124,124,127,127,128,128,129,129,132,132,135,135,137,137,138,138,139,139,140,140,142,142,143,143,144,144,145,145,146,146,149,149,150,150,151,151,152,152,157,157,159,159,160,160,163,163,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,74,76,-1,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,116,118,120,-1,-1,122,124,126,128,130,132,-1,134,-1,-1,136,-1,138,140,142,-1,144,-1,146,148,150,152,154,156,158,-1,-1,-1,160,162,-1,-1,-1,-1,164,166,-1,-1,-1,-1,168,170,-1,172,-1,-1,174,-1,-1,176,-1,178,180,182,184,186,-1,-1,188,190,192,-1,-1,194,-1,-1,196,-1,198,200,202,204,-1,206,208,210,212,214,-1,-1,216,218,220,222,-1,-1,-1,-1,224,-1,226,228,-1,-1,230,-1,-1,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.330579E0,9.075E3,1E0,1.4722673E0,5.0939494E5,2.8091298E7,4.9386E4,1.8441667E2,2.14099E5,7.75541E-1,8.317E3,7.969174E7,2E0,5.528107E3,2.9143541E-6,2.4100633E-2,1.5669752E-2,1.5589395E0,5.9767612E7,5.3811796E-2,6.978874E5,1.2414683E5,3.983889E3,1E0,2.5247778E6,6.9309726E0,4.4263797E0,6.7652373E3,2.28125E0,4.8E1,-4.9761564E-2,4.787007E6,2.2539302E6,1.59E2,2.36118E5,3.0200147E8,2.0601392E3,2.2100918E1,1.5211788E-1,3.1110662E7,9.368054E-3,3.331632E-2,3E1,1.65E2,7E2,9.161318E0,7.5179994E-1,7.064848E6,3.596E3,1.5092975E6,1E0,3.01E2,2.7702793E-1,2.4435181E12,3.906E3,1.0503613E5,8.791777E3,9E0,9.532E3,2.3770695E5,2.84E2,-2.3098497E-2,1.2187199E-3,9.490825E5,5.8426323E0,1.6966831E-3,8.635209E-3,-1.5091491E-3,8.4503644E5,8.848604E1,1.1879E4,1.2199979E10,2.8807552E7,6.4625664E8,1.026639E-2,1.1034263E10,-1.1479909E-2,-3.7206415E-2,2.1109243E-1,5.9689647E-3,3.1E1,8E0,7E0,-4.444156E-2,2.6133334E2,-2.0133467E-2,4.7646948E2,3.3870136E7,1.4955851E7,1E0,9.414044E2,4.7E2,2.3977574E7,2.0868765E-2,2.098385E-2,4.6238E-3,3.103818E1,1.0715278E7,4.3030486E-2,1.2685726E-2,-2.6838101E-2,-1.8701837E-3,2.7165637E4,7.492813E0,7.0417374E-3,2.33121E-2,3.7544508E-2,1.278455E-2,1.3984146E2,2.116711E3,-2.2808127E-3,3.4976208E-1,-1.4075372E-2,4.3037143E-3,3.2585382E10,-8.431589E-3,-2.2441171E-2,6.015825E0,3.1470316E-3,2.7538432E8,3.8879236E7,4.5808794E5,7.4611723E-1,4.76E3,-3.3995137E-2,-1.4395751E-2,3.317425E3,8.755996E-1,3.904E3,-1.694517E-3,-4.5746714E-3,1.4E1,2.7005963E-2,7.518002E-3,5.836513E3,-1.9910135E-3,2.97E2,4.3957013E-1,1.279012E7,5.666993E0,-2.1238027E-2,2.0368582E8,2.298425E6,2.8839298E7,6.6315195E6,2E0,1.2565063E-2,3.0823322E-2,5.314225E2,4.44E2,5.794E3,7.2E1,-9.111518E-3,1.0306298E-2,-2.7040545E-2,-5.666208E-3,1.0443182E3,2.9336793E-2,3.78E2,1.747E3,3.4521695E-2,2.3545856E-2,6.83E2,2.030399E-2,-7.140765E-3,7.683389E7,-1.451142E-4,-1.4560788E-2,-3.023017E-2,-1.4536078E-2,-2.5109679E-2,-1.0703729E-2,-1.566595E-2,-3.4792551E-3,5.5146596E-4,-8.862284E-3,-7.930381E-3,1.4260593E-3,-1.4138156E-2,-3.1977125E-2,-9.187217E-3,-1.6731189E-3,5.9797186E-3,-1.0979255E-2,2.5886407E-2,3.6063904E-3,-1.3215752E-2,4.09877E-3,-2.792823E-2,-7.942737E-3,8.0277845E-3,1.645643E-2,-3.8787161E-3,6.603033E-3,-7.438108E-3,-1.6250774E-2,-1.0151817E-2,-1.9480491E-3,-2.477762E-2,-8.995811E-3,1.4644465E-2,3.9505847E-2,5.966267E-3,-1.9740958E-3,-3.419681E-3,-1.4843848E-2,-3.2444138E-3,-2.9323729E-2,7.0745866E-3,-1.0270991E-2,1.0623749E-2,-1.8235692E-3,5.230358E-3,2.5647352E-2,1.1204507E-2,2.1512562E-2,-1.8939724E-2,4.7332207E-3,-2.0090465E-3,7.579432E-3,3.0488659E-2,1.58173E-2,3.3027127E-2,5.382434E-2,2.1608593E-2,8.360387E-3,1.3903523E-2,-7.186467E-3,9.770272E-3,-5.890742E-4,1.4093764E-3,1.1728661E-2],"split_indices":[19,55,2,109,41,27,44,2,51,2,33,9,7,16,51,41,0,37,40,44,0,27,27,27,6,27,52,41,51,53,8,0,31,44,11,1,7,51,55,26,31,0,0,3,3,2,53,26,27,2,27,6,0,34,30,0,32,51,16,9,27,0,0,0,32,52,37,0,0,31,55,2,5,9,7,0,5,0,0,38,0,8,17,6,0,54,0,51,7,46,66,51,0,46,0,0,0,53,44,0,0,0,0,32,53,0,0,0,0,51,4,0,38,0,0,30,0,0,52,0,7,44,27,26,0,0,0,51,41,2,0,0,3,0,0,27,0,2,37,7,52,0,7,27,44,27,8,0,0,4,0,2,8,0,0,0,0,4,0,8,0,0,0,11,0,0,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.185E3,6.84E2,1.501E3,3.71E2,3.13E2,1.219E3,2.82E2,2.49E2,1.22E2,2.43E2,7E1,7.06E2,5.13E2,2.11E2,7.1E1,2.46E2,3E0,7.7E1,4.5E1,2.41E2,2E0,5.9E1,1.1E1,8.6E1,6.2E2,2.46E2,2.67E2,4.8E1,1.63E2,4.7E1,2.4E1,9E0,2.37E2,4.2E1,3.5E1,3.5E1,1E1,1.73E2,6.8E1,4.2E1,1.7E1,2E0,9E0,1E1,7.6E1,5.8E2,4E1,1.58E2,8.8E1,2.49E2,1.8E1,4.3E1,5E0,1.41E2,2.2E1,5E0,4.2E1,1.4E1,1E1,2.11E2,2.6E1,4E0,3.8E1,2E0,3.3E1,1.7E1,1.8E1,6E0,4E0,1.55E2,1.8E1,2.3E1,4.5E1,2.3E1,1.9E1,7E0,1E1,2E0,8E0,7.3E1,3E0,1.31E2,4.49E2,3.6E1,4E0,1.48E2,1E1,7E1,1.8E1,1.55E2,9.4E1,1.3E1,5E0,3.9E1,4E0,3E0,2E0,8.6E1,5.5E1,2E1,2E0,3E0,2E0,1.8E1,2.4E1,8E0,6E0,7E0,3E0,7E0,2.04E2,3E0,2.3E1,2E0,2E0,1.9E1,1.4E1,7E0,1E1,5E0,1.3E1,5.4E1,1.01E2,1E1,8E0,2E1,3E0,2.3E1,2.2E1,2.1E1,2E0,1.2E1,7E0,8E0,2E0,7.1E1,2E0,1.17E2,1.4E1,1.3E1,4.36E2,1.4E1,2.2E1,5.2E1,9.6E1,6.3E1,7E0,6E0,1.2E1,1.47E2,8E0,6E0,8.8E1,4E0,9E0,3E0,2E0,3.3E1,6E0,5.2E1,3.4E1,2.4E1,3.1E1,1E1,8E0,1.2E1,1.2E1,3E0,4E0,1.93E2,1.1E1,1.2E1,1.1E1,1.7E1,2E0,2E0,8E0,1.1E1,2E0,4.6E1,8E0,7.5E1,2.6E1,8E0,2E0,6E0,2E0,1.9E1,4E0,1.8E1,4E0,1E1,1.1E1,2E0,5E0,5E0,6.6E1,4.6E1,7.1E1,1.1E1,3E0,2E0,1.1E1,1.78E2,2.58E2,1.5E1,7E0,4.9E1,3E0,9.1E1,5E0,5.9E1,4E0,2E0,5E0,9.5E1,5.2E1,3E0,5E0,3E0,3E0,7.7E1,1.1E1,8E0,2.5E1,4.6E1,6E0,3.2E1,2E0,4E0,6E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[-7.979776E-4,-3.2234123E-1,1.4744376E-1,-5.063728E-1,-1.530566E-1,-6.2811635E-3,3.5512537E-1,-3.8444722E-1,-6.6457707E-1,-2.3137991E-1,2.4894316E-1,-8.871361E-2,2.1207039E-1,1.9102733E-1,4.8003587E-1,-4.1261607E-1,-7.185899E-2,-8.2214814E-1,-4.9098787E-1,-2.5153315E-1,6.1259005E-2,3.8667016E-2,3.9803565E-1,-2.2906655E-1,-1.3394692E-2,1.6270234E-1,4.212859E-1,9.240107E-2,4.016477E-1,3.0091226E-1,6.346518E-1,-3.5786143E-1,-6.4611775E-1,1.5329092E-2,-1.8399598E-1,-9.2296E-1,-4.5723587E-1,-5.707429E-1,-3.1101993E-1,-4.450685E-1,-1.6235904E-1,7.468264E-3,-3.7895866E-2,2.2885494E-3,4.5426515E-1,-4.558213E-1,-1.9380772E-1,2.4176829E-2,-4.329177E-1,7.9017E-2,2.7467513E-1,2.1465987E-1,6.275467E-1,1.566765E-1,-1.8109538E-1,4.4048747E-1,-1.1625945E-1,3.9271736E-1,8.910621E-2,8.360236E-1,5.177416E-1,-3.9347076E-1,-1.3055588E-1,-2.1853039E-1,-7.363867E-1,-8.920746E-2,-1.3379028E-2,-7.4463844E-1,-1.1371255E0,-6.3977544E-3,-2.313498E-2,-6.633193E-1,-2.7047426E-1,-3.872877E-1,-1.1007346E-1,-3.4189066E-1,-7.202908E-1,-1.896594E-1,3.4628916E-1,1.8086817E-2,-1.0157721E-2,3.00826E-2,3.6140418E-1,-1.1054627E-2,-2.2172514E-2,-1.2919989E-1,-2.874064E-1,8.4627785E-2,-2.544309E-1,-5.0454247E-1,-1.3706943E-1,-2.147863E-1,1.1605092E-1,2.1286938E-1,2.818847E-2,-8.585696E-2,2.9826203E-1,6.7672753E-1,1.4713425E-2,1.23322085E-1,4.221437E-1,-2.756015E-1,1.9593957E-1,3.3967403E-1,5.905441E-1,1.2835161E-2,-1.50700305E-2,1.4545499E-1,4.6370247E-1,-9.365089E-3,2.179542E-1,9.338004E-1,4.1525537E-1,4.5371318E-1,5.5012386E-2,-4.2589614E-1,-1.6432884E-1,-1.762615E-1,5.9241266E-4,-1.423574E-2,6.40855E-4,-3.46173E-2,-1.2000746E-2,1.5804537E-3,-8.850248E-3,-3.5430767E-2,-1.19696865E-2,-1.1766425E0,-2.1201445E-2,-9.23398E-3,-7.143658E-1,-1.5794847E-2,-6.94668E-3,-1.8777275E-2,-5.4732035E-3,3.0474858E-3,-8.1683565E-3,-2.8846425E-1,-2.6282974E-2,-9.3061346E-1,-4.1489014E-1,-2.8245646E-1,-8.916706E-2,2.2307178E-2,-1.5001931E-3,-4.2617284E-3,4.114937E-3,6.2545836E-3,1.8822387E-2,-2.4976763E-1,-7.401918E-2,-2.3861973E-1,-4.309936E-2,4.4917487E-2,5.757166E-2,-4.6279337E-2,-2.1567698E-1,-2.3801237E-2,-7.24004E-3,8.656783E-4,-8.707071E-3,3.2812317E-3,-4.713639E-1,-2.8182107E-1,1.4867778E-1,1.7881158E-1,1.5447186E-2,4.8907255E-3,-9.498669E-3,4.089195E-3,3.6607513E-1,1.6370848E-2,3.3545148E-2,7.188173E-2,2.8158078E-1,2.545405E-2,1.9427675E-1,-2.0728363E-2,-1.7569596E-1,1.890024E-3,1.2681513E-2,4.1041985E-1,6.965298E-2,6.174911E-1,5.6241965E-3,2.171502E-1,-4.5569194E-3,4.9808332E-1,-1.12103574E-1,-4.4311685E-3,6.9282405E-2,2.9265627E-1,4.1305814E-2,7.5416833E-1,1.1120617E0,4.916041E-1,-2.445135E-3,5.071252E-1,1.3177368E-1,-2.8778275E-3,-2.0050379E-2,-5.1057446E-3,-1.663023E-2,-9.139718E-3,9.172689E-5,-3.9457045E-2,-6.349845E-2,-2.1655153E-2,-3.6215562E-2,-6.6176387E-3,-1.6885718E-2,-2.0635089E-2,-4.866802E-2,-6.5776957E-3,-2.2121532E-2,-1.8265145E-2,-7.8648E-3,-1.7977752E-3,-1.2054999E-2,-8.886161E-3,-2.4545925E-2,-1.1783359E-3,-9.124648E-3,-1.1502252E-2,1.1433127E-3,1.8504984E-3,4.7596276E-2,-1.1397471E-2,-2.2305239E-3,-3.605725E-3,-2.771327E-2,5.640513E-3,-3.2568946E-2,4.5702453E-3,1.320339E-2,9.806317E-3,-6.014284E-3,5.141121E-3,1.922426E-2,-2.4530324E-3,5.722235E-3,1.3935487E-2,3.6029998E-4,1.235436E-2,2.1484003E-3,-9.755357E-3,6.1027817E-3,4.694682E-3,2.037962E-2,-4.668347E-3,6.2794415E-3,1.6661733E-2,3.0669304E-2,1.1689184E-2,-1.4517653E-3,2.391555E-2,1.0942718E-2,1.0840872E-2,-1.574371E-2,5.145301E-3,-7.951941E-3,1.6698902E-2,6.429462E-3,-3.7478535E-3,5.990407E-3,3.5654645E-2,8.161603E-3,5.4721195E-2,2.6648E-2,3.6436517E-3,2.5309835E-2,1.6955566E-2,3.154166E-2,2.517788E-2,-5.738224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,-1,79,-1,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,121,123,-1,125,127,-1,-1,129,131,133,135,137,139,141,143,145,-1,-1,147,-1,-1,149,151,153,155,157,159,161,163,165,-1,167,169,171,-1,173,175,177,179,181,183,-1,-1,185,187,189,191,193,195,197,-1,199,201,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,205,-1,-1,207,-1,-1,-1,-1,-1,-1,209,-1,211,213,215,217,-1,-1,-1,-1,-1,-1,219,221,223,-1,-1,225,-1,227,-1,-1,-1,-1,-1,229,231,233,235,-1,-1,-1,-1,237,-1,-1,239,241,-1,243,-1,245,-1,-1,247,249,251,-1,253,-1,255,257,-1,259,261,263,265,267,269,-1,271,273,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0543734E2,2.1667786E1,4.833795E1,6.187088E0,1.15499735E1,1.5696162E1,1.3108742E1,1.652153E0,3.535118E0,1.01115055E1,1.8656447E0,6.6885214E0,2.3999815E0,5.7792425E0,9.917831E0,2.0062103E0,8.4474E-1,2.3002625E0,8.415737E-1,5.1570168E0,0E0,2.576489E-1,6.609025E-1,1.6785774E0,6.5202217E0,1.8055396E0,1.7831397E0,3.403561E0,1.8414822E0,3.290081E0,4.1579285E0,1.114191E0,1.0742092E0,0E0,1.21625155E-1,1.330101E0,2.3352551E-1,1.1548815E0,3.2518768E-1,2.4514408E0,2.9679089E0,0E0,1.8151815E-1,0E0,3.454218E-1,8.861303E-2,1.1299782E0,6.4215307E0,6.513052E-1,1.2544439E0,1.6523786E0,6.2858045E-1,1.0743523E-1,1.329243E0,1.3723894E0,1.0550079E0,7.004924E-1,2.0212345E0,6.7449605E-1,2.4921303E0,5.074692E0,8.5133743E-1,1.4017558E-1,1.6066101E-1,1.4328289E-1,1.2553763E-1,0E0,4.1273308E-1,8.33168E-2,0E0,0E0,6.9845676E-1,6.5752566E-2,7.4221134E-2,1.1197317E-1,7.377691E-1,1.2197142E0,1.8305683E0,6.2746274E-1,1.2293365E-1,0E0,0E0,2.2159362E-1,0E0,0E0,7.532618E-1,2.3017397E0,7.628095E0,1.8028278E0,1.1294603E-1,7.271376E-2,9.923249E-1,1.3530132E0,2.5366235E-1,0E0,1.6685118E-1,2.3485696E-1,9.630203E-2,0E0,1.1194832E0,4.3340325E-1,4.6576715E-1,9.2129916E-2,9.6476173E-1,3.2540035E-1,0E0,0E0,5.159083E-1,1.8771324E0,2.225735E-1,2.914195E-1,9.5451736E-1,5.639646E-1,1.9450378E0,0E0,6.359825E-1,1.364856E-1,7.347655E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.940426E-1,0E0,0E0,2.6363754E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.669507E-1,0E0,4.6965885E-1,1.6752887E-1,1.2849226E0,8.3905894E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5666122E-1,4.904884E-1,3.0789137E-1,0E0,0E0,5.0180454E0,0E0,3.8372874E-1,0E0,0E0,0E0,0E0,0E0,3.3722055E-1,1.5122253E0,6.119652E-1,7.236024E-1,0E0,0E0,0E0,0E0,1.5875638E-1,0E0,0E0,7.4211836E-1,2.4150634E-1,0E0,7.937744E-2,0E0,2.8841174E-1,0E0,0E0,4.5283413E-1,1.5744716E-1,1.9918251E-1,0E0,2.3327887E-1,0E0,5.4499817E-1,5.8030796E-1,0E0,2.0468652E-1,1.3124895E-1,1.0129769E-1,4.5158386E-1,6.0341644E-1,3.154142E-1,0E0,2.252924E0,2.0451212E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,82,82,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,96,96,97,97,99,99,100,100,101,101,102,102,103,103,104,104,107,107,108,108,109,109,110,110,111,111,112,112,113,113,115,115,116,116,117,117,127,127,130,130,137,137,139,139,140,140,141,141,142,142,149,149,150,150,151,151,154,154,156,156,162,162,163,163,164,164,165,165,170,170,173,173,174,174,176,176,178,178,181,181,182,182,183,183,185,185,187,187,188,188,190,190,191,191,192,192,193,193,194,194,195,195,197,197,198,198],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,-1,80,-1,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,122,124,-1,126,128,-1,-1,130,132,134,136,138,140,142,144,146,-1,-1,148,-1,-1,150,152,154,156,158,160,162,164,166,-1,168,170,172,-1,174,176,178,180,182,184,-1,-1,186,188,190,192,194,196,198,-1,200,202,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,206,-1,-1,208,-1,-1,-1,-1,-1,-1,210,-1,212,214,216,218,-1,-1,-1,-1,-1,-1,220,222,224,-1,-1,226,-1,228,-1,-1,-1,-1,-1,230,232,234,236,-1,-1,-1,-1,238,-1,-1,240,242,-1,244,-1,246,-1,-1,248,250,252,-1,254,-1,256,258,-1,260,262,264,266,268,270,-1,272,274,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.4166665E0,1.5938586E8,6.748543E-1,1.7551165E0,9.9625344E5,2.1150263E-1,3.74E3,1.9044118E0,1.22159E5,2.862988E5,5.52E2,2.742277E6,1.4777102E9,1.0855529E3,6.97E3,2.1358025E0,9.9668306E-1,2.983871E0,1E0,6.1259005E-2,1.8186971E0,3.426E3,3.6E1,5.5544252E7,2.657819E6,3.8866E4,4.719849E7,1.921032E4,5.236559E0,2E0,1.1400756E6,1.6621418E0,1.5329092E-2,1.279076E6,8.937236E-1,2.8477445E0,7.7329254E2,1.445E3,1.3983857E10,1.3554651E2,7.468264E-3,5.884028E0,2.2885494E-3,9.877E3,1E0,9.25E0,2.9833334E1,7.299E3,1.1058E4,6.789622E7,2.165022E2,2E0,9.773983E6,4.798624E3,3.166E3,1.4713E4,1.915148E7,6.308896E2,2.3487206E7,1.9013363E4,2.1989189E2,5.7082694E5,1.918E3,5.936784E-1,2.1515152E0,-1.3379028E-2,1E0,6.131E3,-6.3977544E-3,-2.313498E-2,7.7654165E-1,1.672E4,6.1E1,1.64935E5,6.971004E7,3.1827815E12,3.7609413E-1,1.0746194E4,2.2425473E0,-1.0157721E-2,3.00826E-2,1.992945E5,-1.1054627E-2,-2.2172514E-2,3.3151623E1,4.5184364E0,2.04E5,7.854E3,6.720137E7,5.441732E9,1.4615384E0,1E1,1.72797E5,2.818847E-2,3E0,1.6633E4,4.5412036E2,1.4713425E-2,3.1484194E5,1.5479116E-1,1.3141646E3,5.650613E3,3.565214E6,2.802E5,1.2835161E-2,-1.50700305E-2,1.7E1,1.5846036E7,6.7624075E5,1.51529E3,1.6503105E3,9.256843E3,3.5301748E7,5.5012386E-2,5.2E1,2.9924436E9,2E0,5.9241266E-4,-1.423574E-2,6.40855E-4,-3.46173E-2,-1.2000746E-2,1.5804537E-3,-8.850248E-3,-3.5430767E-2,-1.19696865E-2,5.418831E11,-2.1201445E-2,-9.23398E-3,3.0161016E7,-1.5794847E-2,-6.94668E-3,-1.8777275E-2,-5.4732035E-3,3.0474858E-3,-8.1683565E-3,1.4396E4,-2.6282974E-2,1.387623E8,2.8841177E10,1.9E1,4.7279915E6,2.2307178E-2,-1.5001931E-3,-4.2617284E-3,4.114937E-3,6.2545836E-3,1.8822387E-2,3.09627E5,5.9E1,7.997723E6,-4.309936E-2,4.4917487E-2,1.0409E4,-4.6279337E-2,7.44393E5,-2.3801237E-2,-7.24004E-3,8.656783E-4,-8.707071E-3,3.2812317E-3,5E0,3.114E3,1E0,6.057671E6,1.5447186E-2,4.8907255E-3,-9.498669E-3,4.089195E-3,5E0,1.6370848E-2,3.3545148E-2,1.7488463E3,6.439228E-1,2.545405E-2,1.2045951E6,-2.0728363E-2,6E1,1.890024E-3,1.2681513E-2,2.116711E3,1.9836358E5,2.4466434E-1,5.6241965E-3,3.6806385E6,-4.5569194E-3,2E0,8.235294E-1,-4.4311685E-3,2.5239267E5,7.612907E-1,7.88E2,2.6956522E0,1.7384394E7,4.709E3,-2.445135E-3,3.317425E3,5.7719237E-1,-2.8778275E-3,-2.0050379E-2,-5.1057446E-3,-1.663023E-2,-9.139718E-3,9.172689E-5,-3.9457045E-2,-6.349845E-2,-2.1655153E-2,-3.6215562E-2,-6.6176387E-3,-1.6885718E-2,-2.0635089E-2,-4.866802E-2,-6.5776957E-3,-2.2121532E-2,-1.8265145E-2,-7.8648E-3,-1.7977752E-3,-1.2054999E-2,-8.886161E-3,-2.4545925E-2,-1.1783359E-3,-9.124648E-3,-1.1502252E-2,1.1433127E-3,1.8504984E-3,4.7596276E-2,-1.1397471E-2,-2.2305239E-3,-3.605725E-3,-2.771327E-2,5.640513E-3,-3.2568946E-2,4.5702453E-3,1.320339E-2,9.806317E-3,-6.014284E-3,5.141121E-3,1.922426E-2,-2.4530324E-3,5.722235E-3,1.3935487E-2,3.6029998E-4,1.235436E-2,2.1484003E-3,-9.755357E-3,6.1027817E-3,4.694682E-3,2.037962E-2,-4.668347E-3,6.2794415E-3,1.6661733E-2,3.0669304E-2,1.1689184E-2,-1.4517653E-3,2.391555E-2,1.0942718E-2,1.0840872E-2,-1.574371E-2,5.145301E-3,-7.951941E-3,1.6698902E-2,6.429462E-3,-3.7478535E-3,5.990407E-3,3.5654645E-2,8.161603E-3,5.4721195E-2,2.6648E-2,3.6436517E-3,2.5309835E-2,1.6955566E-2,3.154166E-2,2.517788E-2,-5.738224E-3],"split_indices":[19,55,7,26,41,27,37,2,52,2,27,2,9,7,51,10,55,38,52,15,0,38,2,2,44,27,1,44,4,53,16,32,57,0,1,26,56,51,2,5,55,0,52,0,9,80,57,52,2,9,7,51,16,1,4,0,9,12,51,49,51,51,46,2,26,52,0,112,2,0,0,26,10,3,1,44,30,41,4,41,0,0,32,0,0,54,52,5,9,12,5,52,3,1,0,8,1,4,0,27,56,51,4,46,10,0,0,3,46,46,4,51,4,44,0,11,5,8,0,0,0,0,0,0,0,0,0,30,0,0,44,0,0,0,0,0,0,9,0,7,5,3,46,0,0,0,0,0,0,9,10,9,0,0,2,0,1,0,0,0,0,0,8,28,107,27,0,0,0,0,3,0,0,4,26,0,46,0,3,0,0,4,31,40,0,46,0,6,52,0,32,37,0,53,50,2,0,51,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.21E3,6.97E2,1.513E3,3.33E2,3.64E2,8.7E2,6.43E2,1.9E2,1.43E2,3.05E2,5.9E1,6.32E2,2.38E2,2.79E2,3.64E2,1.74E2,1.6E1,7.3E1,7E1,3.02E2,3E0,2.5E1,3.4E1,2.2E2,4.12E2,1.94E2,4.4E1,1.91E2,8.8E1,1.7E2,1.94E2,1.43E2,3.1E1,3E0,1.3E1,5.6E1,1.7E1,4.7E1,2.3E1,9.4E1,2.08E2,9E0,1.6E1,5E0,2.9E1,2.8E1,1.92E2,3.79E2,3.3E1,1.12E2,8.2E1,2.3E1,2.1E1,1.55E2,3.6E1,8.2E1,6E0,1.18E2,5.2E1,6.9E1,1.25E2,1.23E2,2E1,6E0,2.5E1,8E0,5E0,3.3E1,2.3E1,3E0,1.4E1,3.5E1,1.2E1,1.6E1,7E0,7E1,2.4E1,1.98E2,1E1,1.3E1,3E0,7E0,2.2E1,5E0,2.3E1,1.15E2,7.7E1,3.12E2,6.7E1,2.6E1,7E0,1.2E1,1E2,7.1E1,1.1E1,5E0,1.8E1,1.7E1,4E0,1.39E2,1.6E1,2.9E1,7E0,5.1E1,3.1E1,2E0,4E0,2.7E1,9.1E1,3E1,2.2E1,5.5E1,1.4E1,1.16E2,9E0,1.07E2,1.6E1,1.5E1,5E0,4E0,2E0,2.3E1,2E0,4E0,4E0,3E1,3E0,2.1E1,2E0,4E0,3.1E1,6E0,6E0,1.4E1,2E0,2E0,5E0,5.9E1,1.1E1,1.3E1,1.1E1,1.02E2,9.6E1,7E0,3E0,5E0,8E0,5E0,1.7E1,3.5E1,8E1,7.3E1,4E0,8E0,3.04E2,2E0,6.5E1,2.4E1,2E0,2E0,5E0,6E0,6E0,7E0,9.3E1,5.8E1,1.3E1,2E0,3E0,5E0,1.3E1,4E0,1.3E1,1.06E2,3.3E1,9E0,7E0,9E0,2E1,3E0,4E0,4E1,1.1E1,2.9E1,2E0,2.1E1,6E0,8.6E1,5E0,1.4E1,1.6E1,1.5E1,7E0,3E1,2.5E1,1.2E1,2E0,9.9E1,1.7E1,5E0,1.02E2,1.4E1,2E0,1.3E1,2E0,1.1E1,1E1,1E1,2.1E1,2.3E1,3.6E1,4E0,9E0,3E0,8E0,4.7E1,5.5E1,7.6E1,2E1,3.1E1,4E0,5.9E1,2.1E1,6.9E1,4E0,3E2,4E0,5.3E1,1.2E1,2E0,4E0,4E0,3E0,7.1E1,2.2E1,5.2E1,6E0,3E0,1E1,3.2E1,7.4E1,3E1,3E0,4E0,3E0,1.8E1,2E0,5E0,3.5E1,3E0,8E0,7E0,2.2E1,1.8E1,3E0,7.6E1,1E1,2E0,3E0,1.4E1,2E0,9E0,6E0,3E0,4E0,2.8E1,2E0,2E1,5E0,2E0,1E1,6E1,3.9E1,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"275","size_leaf_vector":"1"}},{"base_weights":[-3.282667E-3,-3.0403143E-1,1.4124826E-1,-4.6798652E-1,-1.09251566E-1,5.3295894E-3,3.5375798E-1,-5.086248E-1,5.111821E-1,-2.2746089E-1,1.4157993E-1,-6.9060035E-2,2.2588784E-1,1.9989431E-1,4.8535362E-1,-1.6583675E-1,-5.749003E-1,5.8027897E-2,1.4753193E-1,-4.6926573E-1,-1.9609798E-1,7.052865E-2,4.43574E-1,-2.4766725E-1,-9.273136E-3,1.5819094E-1,4.3340692E-1,-1.3112852E-1,2.5634894E-1,2.923178E-1,6.320234E-1,-3.2612804E-1,-1.5092057E-2,-6.233165E-1,-2.5639585E-1,2.0224646E-2,-1.2998848E-1,-2.7731076E-2,-1.586419E-2,-2.1962157E-1,7.004545E-2,-1.2027743E-1,1.4097892E-1,4.0593273E-3,5.0075495E-1,-3.3647445E-1,-1.3998705E-1,2.2651024E-2,-4.1972253E-1,9.4636366E-2,4.009534E-1,3.0296296E-1,5.444078E-1,-3.2495025E-1,1.1625563E-2,1.18877105E-1,3.4011567E-1,2.2526506E-1,6.325815E-1,6.8181694E-1,2.502138E-1,5.6775506E-3,-3.6285907E-1,-1.7077126E-1,1.414749E-1,-5.6947905E-1,-9.1999096E-1,-8.584092E-2,-4.0097225E-1,1.8776754E-3,-9.365734E-3,-2.496489E-1,-6.3369E-2,-1.03788055E-1,1.5085545E-2,1.9036557E-3,-2.3922719E-1,6.696974E-2,2.128953E-1,6.019609E-1,9.728644E-3,-3.1132248E-1,-3.704655E-2,4.6172403E-2,-1.6458143E-1,6.674601E-1,4.5364364E-3,-5.3171015E-1,2.385772E-3,1.2938626E-1,-1.8276198E-1,1.7250665E-1,5.7508546E-1,8.5006567E-4,3.3631435E-1,1.6309643E-2,6.532153E-1,-3.6506677E-1,-7.2669366E-4,1.2904524E-2,-7.532518E-2,2.457301E-1,-1.6443696E-2,4.0103814E-1,8.6798415E-2,2.955911E-1,7.313291E-2,3.1696666E-2,1.5204135E-2,5.46298E-1,7.88588E-1,5.542422E-1,3.6894534E-2,-3.8569745E-1,-1.9908557E-3,-1.2558163E-2,-3.097001E-3,5.822385E-2,1.799571E-2,-6.263206E-1,-3.554663E-1,-1.0139786E0,-1.8892514E-2,-2.353646E-1,4.591556E-2,-4.3969536E-1,1.6963255E-4,-2.2948332E-1,-4.9093926E-1,-1.11404106E-1,3.4095217E-2,-8.873955E-3,4.6074368E-2,-1.3234202E-2,-3.8577227E-3,1.2574191E-1,-6.2005702E-2,2.9122397E-1,5.8078848E-2,3.22661E-2,1.3781796E-2,-3.4895703E-1,-6.156722E-2,-1.1942897E-3,7.316024E-3,-8.2836606E-2,-2.4869843E-1,-1.6788531E-2,8.899495E-1,-2.2311777E-2,2.6791805E-1,-2.6406504E-2,-1.4353616E-2,7.598548E-2,2.543399E-1,-3.5972267E-1,1.7731106E-3,-1.4099987E-1,2.7981633E-1,2.9014682E-2,-5.121568E-4,1.635972E-2,5.1857843E-3,3.2051966E-2,1.3506078E-2,-1.9411534E-3,-1.8622827E-2,-7.92689E-3,5.95174E-4,1.6076757E-1,4.2844677E-1,-3.4023774E-1,3.379241E-2,4.2705837E-1,-1.4283898E-1,2.2329564E-1,1.4209877E-2,4.4899452E-1,2.2850564E-1,-5.0959926E-2,2.6680592E-1,6.9820714E-1,2.506816E-1,6.061168E-1,8.7225854E-1,1.1524299E-2,2.9956235E-2,2.2044364E-1,-5.8096778E-3,-8.226639E-3,-1.9269207E-2,-4.495472E-3,4.199229E-3,-2.6558317E-2,-3.8583178E-2,-7.629698E-3,-2.1763606E-2,-2.0440098E-2,-4.8714634E-2,-2.9385274E-3,-1.3543534E-2,-7.4773133E-3,4.7377283E-3,-1.29632605E-2,-2.95695E-2,-8.648996E-3,-1.4854131E-2,-1.0979704E-2,-2.7085116E-2,-7.9977984E-4,-7.593828E-3,3.7018356E-3,-5.445549E-3,6.6079716E-3,-3.1313577E-3,9.530846E-3,-2.496695E-4,-7.372847E-3,2.2262344E-3,1.5689595E-2,4.3080887E-3,4.928636E-3,-3.2010542E-3,-7.750561E-3,-1.7004313E-2,-8.922588E-3,1.3247448E-2,-6.78363E-3,-8.328628E-4,-1.28107285E-2,-1.8856513E-3,1.3403538E-2,4.4777352E-2,-4.4384203E-3,3.3218563E-3,-1.5814245E-2,1.6222708E-2,-1.4931551E-3,5.8494853E-3,-3.7450127E-3,1.2971588E-2,-2.9178888E-3,-2.6397027E-2,4.958563E-4,-1.1135224E-2,1.4209088E-2,2.7963677E-3,1.0955897E-2,-2.2748287E-3,5.429729E-3,2.2713251E-2,-2.5631282E-2,-3.8823981E-3,-9.693979E-3,3.4656054E-3,1.1154663E-2,2.4417955E-2,-1.8712271E-2,4.3338765E-3,4.649127E-3,2.1300578E-2,-8.194152E-3,5.0161295E-3,2.134825E-2,6.408372E-3,5.263564E-3,1.3881697E-2,-4.648211E-3,5.490707E-3,1.500629E-2,5.18791E-3,2.7608825E-2,5.0533745E-2,1.8941404E-2,4.5190756E-3,2.9584935E-2,1.617151E-2,4.3228976E-2,2.218075E-2,1.4256555E-2,3.209063E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,69,-1,-1,71,73,75,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,-1,113,115,117,119,121,123,125,-1,-1,127,129,131,-1,-1,133,135,137,139,-1,141,-1,143,145,147,149,151,-1,153,155,157,159,-1,161,-1,163,165,-1,-1,167,169,171,173,175,177,179,-1,-1,181,183,185,187,189,-1,-1,-1,191,-1,193,195,197,-1,199,201,203,-1,205,207,209,211,-1,213,-1,-1,215,217,219,221,-1,-1,223,225,-1,-1,227,229,-1,231,233,235,-1,-1,237,239,241,-1,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,247,249,251,253,255,257,259,261,263,265,267,269,271,273,275,277,-1,-1,279,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.601917E1,2.2846375E1,4.30931E1,1.5738983E1,9.787258E0,1.4962601E1,1.1687187E1,8.40136E0,4.294126E0,1.6067257E0,2.243164E0,7.2803535E0,3.162572E0,5.079314E0,8.677078E0,1.4931877E0,4.6668854E0,0E0,1.0544277E0,2.1104813E-1,1.2675128E0,1.18644E0,3.5984278E-1,1.5881329E0,6.718372E0,2.6620717E0,6.4591026E-1,1.1105316E0,2.6196623E0,2.9832401E0,3.1944504E0,5.43062E-1,8.2850903E-1,3.8045578E0,1.0258839E0,0E0,1.0085309E-1,0E0,0E0,8.474798E-1,8.1226647E-1,4.7346315E-1,3.2638073E-1,0E0,3.624897E-1,8.722496E-1,3.6929893E-1,5.5568666E0,1.9855065E0,1.3731859E0,1.3328705E0,2.5907016E-1,3.4579945E-1,2.0480359E-1,5.920195E-1,1.5318012E0,2.1720676E0,1.2194495E0,2.0574379E-1,1.8264389E0,1.406364E0,0E0,1.8695855E-1,1.6655025E-1,3.3917964E-1,2.599594E0,1.4667664E0,4.2452186E-1,3.6210275E-1,0E0,0E0,6.366644E-1,1.4893118E-1,1.6105053E-1,0E0,0E0,8.9561105E-2,2.579422E-1,3.711363E-1,1.7840862E-1,0E0,8.2446384E-1,0E0,8.800335E-2,4.6072757E-1,3.333097E0,3.287073E0,1.470747E-1,0E0,8.147516E-1,6.424431E-1,6.11373E-1,7.787595E-1,0E0,9.7412586E-2,0E0,1.19066715E-1,2.0597899E-1,0E0,0E0,1.6999707E-1,6.4522696E-1,7.2720814E-1,1.6884594E0,2.8416288E-1,7.1040154E-1,9.268669E-1,0E0,0E0,3.0582638E0,7.690468E-1,1.2132621E-1,4.528447E-1,1.3436699E-1,0E0,0E0,0E0,8.53524E-2,0E0,1.1325455E0,1.11307E0,6.654358E-1,0E0,8.930546E-2,1.6603914E-1,5.0037766E-1,0E0,4.832611E-1,1.4815521E-1,1.07916504E-1,9.07845E-2,0E0,6.7645855E-2,0E0,0E0,2.570825E-1,1.3133588E-1,1.9481623E-1,8.617787E-2,0E0,0E0,3.0494595E-1,6.816023E-1,0E0,0E0,1.5698826E-1,2.2940636E-1,0E0,3.5447884E-1,3.0647886E0,2.5245552E0,0E0,0E0,5.203533E-1,4.249401E-1,5.5590844E-1,0E0,8.2432404E-2,7.708013E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.7775515E-1,2.561643E-1,2.9637945E-1,4.2976803E-1,2.0986118E0,4.2671645E-1,2.4981049E-1,3.9712197E-1,9.264612E-2,4.6409893E-1,2.2619914E-1,1.0962939E-1,1.0557251E0,6.160096E-1,1.2016964E-1,1.3001518E0,0E0,0E0,7.656503E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,68,68,71,71,72,72,73,73,76,76,77,77,78,78,79,79,81,81,83,83,84,84,85,85,86,86,87,87,89,89,90,90,91,91,92,92,94,94,96,96,97,97,100,100,101,101,102,102,103,103,104,104,105,105,106,106,109,109,110,110,111,111,112,112,113,113,117,117,119,119,120,120,121,121,123,123,124,124,125,125,127,127,128,128,129,129,130,130,132,132,135,135,136,136,137,137,138,138,141,141,142,142,145,145,146,146,148,148,149,149,150,150,153,153,154,154,155,155,157,157,158,158,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,187,187],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,70,-1,-1,72,74,76,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,-1,114,116,118,120,122,124,126,-1,-1,128,130,132,-1,-1,134,136,138,140,-1,142,-1,144,146,148,150,152,-1,154,156,158,160,-1,162,-1,164,166,-1,-1,168,170,172,174,176,178,180,-1,-1,182,184,186,188,190,-1,-1,-1,192,-1,194,196,198,-1,200,202,204,-1,206,208,210,212,-1,214,-1,-1,216,218,220,222,-1,-1,224,226,-1,-1,228,230,-1,232,234,236,-1,-1,238,240,242,-1,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,248,250,252,254,256,258,260,262,264,266,268,270,272,274,276,278,-1,-1,280,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.629299E4,6.7652373E3,2.3544883E8,8.9655056E5,5.88E3,2.461362E-3,2.46E2,5.0829604E3,1.4101E4,4.67E2,2.742277E6,1.5988182E-1,2.3421426E-1,1E0,1.0052E4,5.8027897E-2,1.0040462E8,9.421523E-5,1.0122174E3,2.2783158E5,2.1924436E5,6.061028E4,4.719849E7,1E0,4.2404938E2,5.23E2,1E0,9.3058E4,5.07E2,2.08E2,5.893994E4,7.8351054E11,1.2199979E10,2.0224646E-2,1.0746194E4,-2.7731076E-2,-1.586419E-2,1.294364E0,1.125584E-3,8.182648E6,1.6394367E0,4.0593273E-3,2.4848485E0,2.0116506E4,1.0317661E5,8E0,1.32E2,3.7832818E0,2.1178565E6,6.763314E7,4.949877E6,1.566593E6,3.71E0,1E0,2E0,2.7629982E7,2.8290488E5,6.797394E8,3.0501078E8,5.6775506E-3,3.730007E6,2.5269838E6,6.9998717E-4,3.773E3,7.2456576E8,3.2627738E5,1E0,1.8776754E-3,-9.365734E-3,1E0,2.539E3,1E0,1.5085545E-2,1.9036557E-3,3.38464E5,7.6E1,2.9E1,1.21320024E8,9.728644E-3,7.2896E5,-3.704655E-2,3.114E3,9.25E0,1.04E3,8.426E3,4.5709072E1,2.385772E-3,4.1973075E6,2.4945455E-4,2.3189322E2,1.5E1,8.5006567E-4,4.616371E8,1.6309643E-2,3.44E2,3.908173E-2,-7.2669366E-4,1.2904524E-2,5.666351E7,2.1872402E5,1.0061821E5,1.9162654E7,1.2684739E1,4.1899456E7,6.082581E3,3.1696666E-2,1.5204135E-2,4.1179886E0,3.748732E6,1.4689211E7,2.6025698E7,5.2152296E-7,-1.9908557E-3,-1.2558163E-2,-3.097001E-3,1.4E1,1.799571E-2,4.0844156E7,4.8297736E7,9.4446504E7,-1.8892514E-2,1.6907827E9,1.1225189E5,1.96E4,1.6963255E-4,1.6256282E7,2.2806E5,2.2927427E-1,7.897508E-1,-8.873955E-3,1.2716E4,-1.3234202E-2,-3.8577227E-3,1.545974E-1,2.160255E9,4.7741196E-1,2.5086E4,3.22661E-2,1.3781796E-2,5.836513E3,3.79E2,-1.1942897E-3,7.316024E-3,1.1805985E3,2.057307E6,-1.6788531E-2,2.127E3,2.091623E5,2.637815E1,-2.6406504E-2,-1.4353616E-2,1E0,1.198E5,8E0,1.7731106E-3,1E1,5.93E2,2.9014682E-2,-5.121568E-4,1.635972E-2,5.1857843E-3,3.2051966E-2,1.3506078E-2,-1.9411534E-3,-1.8622827E-2,-7.92689E-3,5.95174E-4,1.2453214E0,1.074E3,1.71E2,4.54E2,7.612907E-1,2.75E2,2.657E3,1.3860265E6,2.7524946E0,1.2679E4,5.4E1,7.1277297E-1,4.77721E3,2.7753687E-2,1.8237082E1,2.7753594E7,1.1524299E-2,2.9956235E-2,1.7291568E2,-5.8096778E-3,-8.226639E-3,-1.9269207E-2,-4.495472E-3,4.199229E-3,-2.6558317E-2,-3.8583178E-2,-7.629698E-3,-2.1763606E-2,-2.0440098E-2,-4.8714634E-2,-2.9385274E-3,-1.3543534E-2,-7.4773133E-3,4.7377283E-3,-1.29632605E-2,-2.95695E-2,-8.648996E-3,-1.4854131E-2,-1.0979704E-2,-2.7085116E-2,-7.9977984E-4,-7.593828E-3,3.7018356E-3,-5.445549E-3,6.6079716E-3,-3.1313577E-3,9.530846E-3,-2.496695E-4,-7.372847E-3,2.2262344E-3,1.5689595E-2,4.3080887E-3,4.928636E-3,-3.2010542E-3,-7.750561E-3,-1.7004313E-2,-8.922588E-3,1.3247448E-2,-6.78363E-3,-8.328628E-4,-1.28107285E-2,-1.8856513E-3,1.3403538E-2,4.4777352E-2,-4.4384203E-3,3.3218563E-3,-1.5814245E-2,1.6222708E-2,-1.4931551E-3,5.8494853E-3,-3.7450127E-3,1.2971588E-2,-2.9178888E-3,-2.6397027E-2,4.958563E-4,-1.1135224E-2,1.4209088E-2,2.7963677E-3,1.0955897E-2,-2.2748287E-3,5.429729E-3,2.2713251E-2,-2.5631282E-2,-3.8823981E-3,-9.693979E-3,3.4656054E-3,1.1154663E-2,2.4417955E-2,-1.8712271E-2,4.3338765E-3,4.649127E-3,2.1300578E-2,-8.194152E-3,5.0161295E-3,2.134825E-2,6.408372E-3,5.263564E-3,1.3881697E-2,-4.648211E-3,5.490707E-3,1.500629E-2,5.18791E-3,2.7608825E-2,5.0533745E-2,1.8941404E-2,4.5190756E-3,2.9584935E-2,1.617151E-2,4.3228976E-2,2.218075E-2,1.4256555E-2,3.209063E-3],"split_indices":[19,109,32,51,12,27,2,26,10,50,2,2,9,37,37,15,2,0,50,41,51,27,27,27,44,107,51,28,107,2,8,2,27,30,5,0,4,0,0,41,37,44,41,0,56,32,27,17,3,52,27,7,9,9,53,15,6,44,32,7,31,0,9,31,33,2,7,27,89,0,0,82,2,79,0,0,9,8,3,31,0,9,0,28,57,2,2,55,0,27,37,51,3,0,7,0,0,37,0,0,12,32,32,46,55,31,51,0,0,52,46,1,42,37,0,0,0,3,0,44,44,7,0,12,32,2,0,44,1,37,26,0,9,0,0,37,12,37,9,0,0,27,2,0,0,32,12,0,1,27,55,0,0,109,7,8,0,3,0,0,0,0,0,0,0,0,0,0,0,37,2,0,2,37,0,2,27,56,2,3,26,51,56,53,46,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.207E3,7.16E2,1.491E3,3.88E2,3.28E2,9.1E2,5.81E2,3.73E2,1.5E1,2.23E2,1.05E2,6.81E2,2.29E2,2.69E2,3.12E2,6.1E1,3.12E2,4E0,1.1E1,2.4E1,1.99E2,8.6E1,1.9E1,1.7E2,5.11E2,1.74E2,5.5E1,3.9E1,2.3E2,1.36E2,1.76E2,2.9E1,3.2E1,2.7E2,4.2E1,5E0,6E0,9E0,1.5E1,1.83E2,1.6E1,2.3E1,6.3E1,3E0,1.6E1,9.2E1,7.8E1,4.75E2,3.6E1,1.39E2,3.5E1,2.7E1,2.8E1,1.6E1,2.3E1,8.8E1,1.42E2,1.15E2,2.1E1,1.55E2,2.1E1,2E0,2.7E1,1.6E1,1.6E1,2.31E2,3.9E1,2E1,2.2E1,2E0,4E0,1.53E2,3E1,1E1,6E0,1E1,1.3E1,3.2E1,3.1E1,1.1E1,5E0,8.9E1,3E0,9E0,6.9E1,1.2E1,4.63E2,2.9E1,7E0,1.24E2,1.5E1,1.6E1,1.9E1,3E0,2.4E1,1.2E1,1.6E1,1.4E1,2E0,5E0,1.8E1,4.5E1,4.3E1,1.14E2,2.8E1,7.8E1,3.7E1,1.6E1,5E0,7.1E1,8.4E1,8E0,1.3E1,2.5E1,2E0,7E0,9E0,1.3E1,3E0,1.81E2,5E1,3.2E1,7E0,9E0,1.1E1,2E1,2E0,1.43E2,1E1,2E1,1E1,6E0,4E0,9E0,4E0,2.2E1,1E1,2E1,1.1E1,7E0,4E0,7.7E1,1.2E1,6E0,3E0,3.6E1,3.3E1,2E0,1E1,4.21E2,4.2E1,2.2E1,7E0,8.8E1,3.6E1,8E0,7E0,4E0,1.2E1,1.7E1,2E0,2.1E1,3E0,1.3E1,3E0,2E0,1.2E1,8E0,1E1,3.2E1,1.3E1,5E0,3.8E1,1.09E2,5E0,9E0,1.9E1,2.2E1,5.6E1,2.3E1,1.4E1,4.6E1,2.5E1,2.9E1,5.5E1,3E0,5E0,6E0,7E0,5E0,2E1,2E0,1.1E1,1.57E2,2.4E1,2.1E1,2.9E1,4E0,2.8E1,3E0,6E0,2E0,9E0,1.3E1,7E0,1.05E2,3.8E1,4E0,6E0,8E0,1.2E1,8E0,2E0,2E0,2E0,1.3E1,9E0,5E0,5E0,1.5E1,5E0,8E0,3E0,1.1E1,6.6E1,9E0,3E0,1.7E1,1.9E1,2.8E1,5E0,2E0,8E0,2.35E2,1.86E2,5E0,3.7E1,2.9E1,5.9E1,3E0,3.3E1,4E0,4E0,2E0,2E0,1E1,2E0,2.3E1,9E0,3E0,1E1,2E0,3E0,5E0,3.3E1,4.3E1,6.6E1,2E0,3E0,7E0,2E0,6E0,1.3E1,2E1,2E0,2.4E1,3.2E1,1.8E1,5E0,9E0,5E0,4E1,6E0,1.1E1,1.4E1,2.3E1,6E0,4.4E1,1.1E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"281","size_leaf_vector":"1"}},{"base_weights":[5.909841E-3,-2.8718516E-1,1.425555E-1,-3.3481604E-1,2.8700888E-1,2.6419908E-2,3.8633937E-1,-4.8911145E-1,-1.4381926E-1,5.9867855E-2,1.08473256E-1,-1.1678973E-1,1.3438918E-1,-1.9159779E-1,4.2295226E-1,-1.2780869E-1,-5.559953E-1,-2.3187593E-1,6.260261E-2,2.00428E-1,-2.6099756E-1,-1.6569419E-1,1.9977552E-1,1.644091E-1,-3.94221E-1,-2.6424763E-1,1.8662029E-3,3.437884E-1,6.1274457E-1,-1.880166E-1,2.4810044E-2,-6.106594E-1,-3.0648583E-1,-1.4540157E-1,-3.0828956E-1,1.953712E-2,4.9619856E-1,-7.1859453E-3,2.5791237E-1,-3.3925606E-3,-1.9561708E-2,-1.8775749E-1,2.4745187E-1,4.4264966E-1,1.2317949E-1,5.504329E-2,2.572964E-1,-4.7907728E-1,-1.241352E-1,-1.5175478E-2,-1.5638149E-1,2.80971E-1,6.3861936E-1,4.5769083E-1,7.8792566E-1,-3.8130963E-1,-8.077813E-2,-7.3511356E-1,-5.184561E-1,-1.8215768E-1,-4.960007E-1,-1.9728775E-1,7.866908E-3,-4.932955E-1,-2.4735568E-1,-3.939076E-2,1.8204E-1,2.7004506E-2,5.897305E-3,3.621668E-1,1.01712316E-1,-3.4756324E-1,-1.448269E-1,3.448725E-2,7.126921E-2,7.5199623E-3,2.2771556E-2,1.061213E-2,2.2181429E-1,8.187138E-2,-2.0994194E-1,2.2641592E-1,6.5972507E-1,-5.0090986E-1,-7.2112097E-3,1.5523257E-3,-1.1668488E-2,-3.0093984E-4,-1.0910769E-2,3.2747248E-1,5.0400496E-2,9.2301536E-1,4.1908634E-1,1.4090618E-1,5.049868E-1,8.175481E-1,-3.0408066E-3,-9.027358E-3,-4.4993263E-1,2.3757338E-3,-1.1195516E-2,-5.356106E-1,-8.724889E-1,-2.9689252E-1,-5.8763987E-1,-2.8400376E-1,-7.608468E-2,-5.52716E-1,-8.720918E-3,-2.255557E-1,6.1826287E-3,-2.338099E-2,1.1555169E-2,-6.12037E-1,-3.0376065E-1,-3.2166648E-1,-1.1654858E-1,-2.0002599E-1,4.3846574E-2,-3.5703147E-3,2.1207304E-1,1.7136474E-1,2.15091E-2,1.9203341E-1,-1.31046E-3,-3.7366614E-1,-1.5009891E-3,1.6717525E-1,-1.6723579E-1,1.1512521E-1,-4.033008E-3,4.9371313E-2,-8.413566E-3,4.8822695E-3,1.3748069E-2,1.23330444E-1,-4.9760945E-2,6.8886736E-3,-2.6891562E-1,2.9298756E-1,5.4660186E-2,3.862438E-2,1.390207E-2,-1.3904558E-2,-2.5845546E-2,1.8516718E-1,4.129554E-1,-1.463478E-1,1.775666E-1,3.9171118E-1,4.7185227E-2,4.9322778E-1,-9.1117425E-3,-7.2746314E-3,2.2577876E-1,5.7538396E-1,2.3666388E-1,8.5514206E-1,4.1074485E-1,-2.3432797E-2,-7.6212464E-3,1.188723E-2,-1.4335964E-3,-2.8178168E-2,-7.680033E-3,-4.2470556E-2,-2.2412118E-2,-1.5528265E-2,-7.097474E-3,-2.7623553E-2,-1.003511E-2,-1.4403037E-2,-3.316323E-4,3.223588E-4,-9.948441E-3,-2.6938818E-2,-6.468115E-3,-1.8652033E-2,-8.871561E-3,3.4695787E-3,-8.319169E-3,3.8325562E-4,-8.36339E-3,-1.1624636E-2,-3.0761786E-2,-1.5546359E-2,-3.1885768E-3,-1.0983722E-2,-2.8856114E-2,-7.276153E-3,4.600744E-3,3.440946E-4,-1.2014992E-2,5.1190522E-3,-1.5614872E-3,1.1062611E-2,9.908237E-4,1.2002237E-2,1.9628077E-3,1.3855158E-2,3.804394E-3,-1.3178294E-2,-2.0598706E-2,2.8819777E-2,2.1232096E-3,-7.145443E-3,-4.3385975E-2,9.379838E-3,2.999118E-4,-2.0587936E-3,5.9356927E-3,-2.893057E-4,7.990612E-3,2.3518084E-3,-1.0889128E-2,-6.6784103E-3,-1.6564528E-2,1.8750386E-2,1.0544995E-2,-3.0971316E-3,7.34106E-3,1.1218278E-3,1.18103335E-2,2.3276994E-2,1.2981914E-2,-2.363631E-2,-1.9589923E-3,1.3400835E-2,-5.364714E-3,2.7661497E-2,-1.4133735E-3,1.1252294E-2,2.756871E-2,-1.5653737E-2,2.0046601E-2,-5.307628E-4,1.3369908E-2,2.6992192E-2,8.465507E-3,-1.3882897E-4,1.3439662E-2,4.0771056E-2,2.3908561E-2,6.0590752E-3,2.2399012E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,-1,57,59,61,63,65,67,-1,69,-1,-1,71,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,-1,121,123,125,127,-1,129,-1,-1,131,133,135,137,139,141,143,-1,-1,-1,-1,-1,145,147,149,151,153,155,157,-1,-1,159,161,-1,163,165,167,169,171,173,175,-1,177,179,181,-1,183,185,187,189,191,193,-1,195,197,-1,199,-1,201,-1,203,205,207,-1,209,-1,-1,-1,211,213,-1,215,217,219,-1,-1,-1,-1,221,223,225,227,229,-1,231,233,-1,235,237,239,241,243,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9553055E1,1.95415E1,4.3165565E1,1.9291397E1,1.0023019E1,1.600365E1,1.049173E1,8.700943E0,5.3766785E0,0E0,1.6830918E0,6.9223485E0,9.427426E0,5.183182E-1,6.7457886E0,2.4629896E0,3.9864502E0,1.3229923E0,1.6522048E0,8.4611E-1,2.8593433E-1,3.56499E0,1.0616663E0,5.66455E0,6.739211E-1,1.2359011E-1,0E0,5.9138145E0,3.306305E0,1.0885347E0,0E0,2.4876404E0,1.2526855E0,7.8885126E-1,1.1067505E0,7.9239863E-1,2.1584892E-1,0E0,5.168376E-1,0E0,0E0,2.4674873E0,1.7579052E0,1.5000963E-1,5.194899E-1,1.8513227E0,3.6036415E0,8.778334E-2,1.993179E-1,0E0,1.3776225E-1,2.9105682E0,3.1719322E0,1.0375595E0,1.6253624E0,1.4747453E-1,4.9902883E-1,2.3829918E0,2.0788422E0,3.6730397E-1,2.7206516E-1,4.270196E-1,2.0693468E-1,3.9923477E-1,7.7902174E-1,8.2439226E-1,1.8910795E-1,0E0,0E0,3.4949803E-1,1.8257526E-1,6.1955166E-1,2.062519E0,0E0,1.2277185E-1,0E0,0E0,1.8430164E-1,2.0047581E-1,1.2886149E0,5.534179E-1,3.2083912E0,1.170332E0,1.2294006E-1,0E0,0E0,0E0,0E0,0E0,2.6849155E0,1.2019132E0,1.1117153E0,1.0839E0,3.1783885E-1,1.0727348E0,5.663605E-1,0E0,0E0,1.605053E-1,2.2973707E-1,0E0,1.3236246E0,1.0210724E0,1.8520164E-1,7.789116E-1,1.6762972E-1,2.2866619E-1,2.27067E-1,0E0,2.779882E-1,1.4789556E-1,1.2488331E-1,0E0,1.8483973E-1,8.6761475E-2,1.1682239E0,3.246675E-1,2.8698874E-1,2.262256E-1,0E0,1.174196E-1,9.650174E-2,0E0,8.50912E-2,0E0,3.4435844E-1,0E0,1.0992932E0,1.8732772E0,1.2847123E-1,0E0,1.6100547E-1,0E0,0E0,0E0,1.2356005E0,1.118362E0,0E0,1.9322264E-1,1.3811207E0,1.0693603E0,0E0,0E0,0E0,0E0,1.0497916E0,1.6532154E0,7.132617E-1,1.0518677E0,5.7548636E-1,0E0,6.6968775E-1,1.0658649E0,0E0,1.5208906E-1,2.9271126E-1,2.173441E-1,3.2622528E-1,9.661126E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,38,38,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,89,89,90,90,91,91,92,92,93,93,94,94,95,95,98,98,99,99,101,101,102,102,103,103,104,104,105,105,106,106,107,107,109,109,110,110,111,111,113,113,114,114,115,115,116,116,117,117,118,118,120,120,121,121,123,123,125,125,127,127,128,128,129,129,131,131,135,135,136,136,138,138,139,139,140,140,145,145,146,146,147,147,148,148,149,149,151,151,152,152,154,154,155,155,156,156,157,157,158,158],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,-1,58,60,62,64,66,68,-1,70,-1,-1,72,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,-1,122,124,126,128,-1,130,-1,-1,132,134,136,138,140,142,144,-1,-1,-1,-1,-1,146,148,150,152,154,156,158,-1,-1,160,162,-1,164,166,168,170,172,174,176,-1,178,180,182,-1,184,186,188,190,192,194,-1,196,198,-1,200,-1,202,-1,204,206,208,-1,210,-1,-1,-1,212,214,-1,216,218,220,-1,-1,-1,-1,222,224,226,228,230,-1,232,234,-1,236,238,240,242,244,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.317425E3,3.3633875E6,1E0,1.5151515E-3,1.7669278E5,2.1328075E-1,8.2608955E-3,4.197889E6,5.9867855E-2,1.1855755E8,6.4625664E8,5.2288747E-1,6E1,1.546644E6,1.4387654E0,4.888E0,8.3306855E6,1.3664E4,1.3E1,2.0783027E5,1.415178E3,1.3510204E1,9.52E2,2.285677E6,4.366E3,1.8662029E-3,6.7652373E3,5.964E3,2.046E3,2.4810044E-2,7.2764084E-2,1.568E3,2.835051E2,4.752366E0,4.8632206E5,4.2E1,-7.1859453E-3,5.9767612E7,-3.3925606E-3,-1.9561708E-2,1.14E2,3.3341297E1,4.9375E1,9.309785E8,2E0,1.0872E4,1.176E3,4.0778715E6,-1.5175478E-2,3.7423557E1,2.7246006E7,1.9748653E-1,3.90106E0,9.256843E3,1.1846624E-5,2.8091298E7,6.355967E-1,1.2E1,9.403226E0,5.7950187E-1,5.9908E6,5.2E1,6.929741E8,1.221875E1,9.451483E-1,2.695E3,2.7004506E-2,5.897305E-3,5.185489E3,5.1E1,1.4285097E1,9.928469E-1,3.448725E-2,5.4210526E1,7.5199623E-3,2.2771556E-2,6.564E3,8.1590717E2,2E0,2.7913043E0,6.568389E0,1.8601036E1,2.07E2,-7.2112097E-3,1.5523257E-3,-1.1668488E-2,-3.0093984E-4,-1.0910769E-2,1.2804922E3,3.1152E4,6.3039363E4,1.27E2,9.2058825E-1,1.2673605E7,1E0,-3.0408066E-3,-9.027358E-3,2.308943E0,4.8E2,-1.1195516E-2,4.646E3,1E0,5.5949634E-1,2.3920168E6,1.0033929E1,4.7022E4,3.1029554E7,-8.720918E-3,1.8631586E4,1E0,2.637815E1,1.1555169E-2,1.9840922E8,1.1317E4,5.408015E0,1E0,9E0,1.2478469E-1,-3.5703147E-3,5.311576E6,3.8879236E7,2.15091E-2,7.64E2,-1.31046E-3,9.82542E5,-1.5009891E-3,1.3050649E4,4.5E1,1.348651E6,-4.033008E-3,1.3883365E6,-8.413566E-3,4.8822695E-3,1.3748069E-2,4.97E2,3.731E3,6.8886736E-3,1.5227116E6,9.538462E0,6.999242E7,3.862438E-2,1.390207E-2,-1.3904558E-2,-2.5845546E-2,1E0,1.8237082E1,4.2371875E4,2.3580047E10,5.4410156E4,4.7185227E-2,1.9745072E7,3.555E3,-7.2746314E-3,2.9328E4,1E0,1.4523809E0,4.728721E0,1.5427417E3,-2.3432797E-2,-7.6212464E-3,1.188723E-2,-1.4335964E-3,-2.8178168E-2,-7.680033E-3,-4.2470556E-2,-2.2412118E-2,-1.5528265E-2,-7.097474E-3,-2.7623553E-2,-1.003511E-2,-1.4403037E-2,-3.316323E-4,3.223588E-4,-9.948441E-3,-2.6938818E-2,-6.468115E-3,-1.8652033E-2,-8.871561E-3,3.4695787E-3,-8.319169E-3,3.8325562E-4,-8.36339E-3,-1.1624636E-2,-3.0761786E-2,-1.5546359E-2,-3.1885768E-3,-1.0983722E-2,-2.8856114E-2,-7.276153E-3,4.600744E-3,3.440946E-4,-1.2014992E-2,5.1190522E-3,-1.5614872E-3,1.1062611E-2,9.908237E-4,1.2002237E-2,1.9628077E-3,1.3855158E-2,3.804394E-3,-1.3178294E-2,-2.0598706E-2,2.8819777E-2,2.1232096E-3,-7.145443E-3,-4.3385975E-2,9.379838E-3,2.999118E-4,-2.0587936E-3,5.9356927E-3,-2.893057E-4,7.990612E-3,2.3518084E-3,-1.0889128E-2,-6.6784103E-3,-1.6564528E-2,1.8750386E-2,1.0544995E-2,-3.0971316E-3,7.34106E-3,1.1218278E-3,1.18103335E-2,2.3276994E-2,1.2981914E-2,-2.363631E-2,-1.9589923E-3,1.3400835E-2,-5.364714E-3,2.7661497E-2,-1.4133735E-3,1.1252294E-2,2.756871E-2,-1.5653737E-2,2.0046601E-2,-5.307628E-4,1.3369908E-2,2.6992192E-2,8.465507E-3,-1.3882897E-4,1.3439662E-2,4.0771056E-2,2.3908561E-2,6.0590752E-3,2.2399012E-2],"split_indices":[19,51,49,109,56,27,41,26,50,0,44,7,26,3,27,40,53,44,2,3,32,51,57,2,31,2,0,51,2,2,0,38,0,51,55,27,3,0,44,0,0,28,55,46,5,6,2,10,31,0,57,44,56,55,4,38,44,26,17,57,38,9,3,5,57,40,2,0,0,51,3,53,52,0,52,0,0,10,51,16,53,53,57,11,0,0,0,0,0,51,10,27,8,52,44,6,0,0,53,10,0,2,112,38,27,52,9,46,0,32,82,55,0,5,31,53,94,3,26,0,1,44,0,0,0,30,0,27,6,28,0,44,0,0,0,2,28,0,44,57,7,0,0,0,0,107,53,31,5,27,0,1,0,0,1,111,53,37,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.234E3,7.1E2,1.524E3,6.56E2,5.4E1,1.033E3,4.91E2,3.62E2,2.94E2,7E0,4.7E1,4.44E2,5.89E2,2.9E1,4.62E2,5.7E1,3.05E2,2.06E2,8.8E1,3.8E1,9E0,3.85E2,5.9E1,5.58E2,3.1E1,2.2E1,7E0,3.28E2,1.34E2,5.3E1,4E0,2.49E2,5.6E1,9.8E1,1.08E2,8.1E1,7E0,5E0,3.3E1,5E0,4E0,3.66E2,1.9E1,1.3E1,4.6E1,2.57E2,3.01E2,2.3E1,8E0,1.2E1,1E1,2.72E2,5.6E1,7.3E1,6.1E1,1.8E1,3.5E1,1.03E2,1.46E2,3.5E1,2.1E1,7.3E1,2.5E1,2.5E1,8.3E1,6E1,2.1E1,5E0,2E0,1.9E1,1.4E1,7.6E1,2.9E2,4E0,1.5E1,3E0,1E1,2.2E1,2.4E1,2.34E2,2.3E1,2.81E2,2E1,2.1E1,2E0,4E0,4E0,4E0,6E0,2.26E2,4.6E1,2.3E1,3.3E1,1E1,6.3E1,5.9E1,2E0,6E0,1.2E1,2.4E1,1.1E1,4.4E1,5.9E1,3.6E1,1.1E2,1.7E1,1.8E1,1.7E1,4E0,6.4E1,9E0,2.3E1,2E0,1.4E1,1.1E1,5.2E1,3.1E1,2E1,4E1,2E0,1.9E1,8E0,1.1E1,8E0,6E0,7E1,6E0,1.9E1,2.71E2,1.2E1,3E0,1.9E1,3E0,1.1E1,1.3E1,1.78E2,5.6E1,3E0,2E1,2.02E2,7.9E1,1.2E1,8E0,7E0,1.4E1,8.6E1,1.4E2,1.8E1,2.8E1,5E0,1.8E1,2.8E1,5E0,2E0,8E0,4.9E1,1.4E1,5.3E1,6E0,9E0,3E0,2E0,2.2E1,3.5E1,9E0,4.9E1,1E1,2.6E1,1E1,1.03E2,7E0,1.5E1,2E0,1.2E1,6E0,1.5E1,2E0,7E0,5.7E1,7E0,2E0,2E1,3E0,3E0,1.1E1,9E0,2E0,4.3E1,9E0,2.6E1,5E0,5E0,1.5E1,2.1E1,1.9E1,1.6E1,3E0,4E0,4E0,3E0,5E0,3.7E1,3.3E1,3E0,1.6E1,2.69E2,2E0,6E0,6E0,9E0,1E1,5.2E1,1.26E2,3.7E1,1.9E1,1E1,1E1,6.5E1,1.37E2,3.7E1,4.2E1,2.8E1,5.8E1,7.6E1,6.4E1,3E0,1.5E1,2E1,8E0,3E0,2E0,1E1,1.8E1,3E0,2E0,2E0,6E0,4.6E1,3E0,3E0,1.1E1,4.5E1,8E0,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"245","size_leaf_vector":"1"}},{"base_weights":[-5.037299E-3,-2.9901177E-1,1.3313596E-1,-4.4742757E-1,-1.0351855E-1,5.208736E-6,3.1119063E-1,-4.626219E-1,5.3942103E-2,-1.8683347E-1,2.6809883E-1,-1.4601353E-1,9.735102E-2,2.2816896E-1,5.150829E-1,-5.3424495E-1,-1.8511753E-1,-2.6629728E-1,-4.9588315E-2,-7.6756105E-2,3.1791365E-1,-2.2989215E-1,1.1252552E-2,1.310809E-1,-3.757213E-1,2.8110605E-1,-6.8062685E-2,2.432225E-1,6.1252666E-1,-1.1892462E0,-5.0253165E-1,-1.1372265E-1,-6.0400987E-1,-3.6466628E-1,-1.9691934E-1,-8.95375E-3,-2.331123E-1,2.9357334E-3,-6.1012693E-3,4.7233564E-1,2.0803128E-1,-1.9661774E-1,-4.7956675E-1,7.4456024E-1,-7.551338E-2,1.04818724E-1,5.5878747E-1,-4.6490738E-1,-1.3886921E-1,1.9991945E-1,4.14152E-1,-2.543552E-1,2.6414952E-1,4.810626E-1,7.425915E-2,4.5722485E-2,5.590857E-1,-2.3420747E-2,-6.0009383E-2,-5.5481875E-1,-3.4526598E-1,7.7866994E-2,-1.9259284E-1,-7.516183E-3,-3.312069E-2,-3.899034E-1,7.316181E-4,-1.6968517E-1,-2.6129905E-2,-1.115041E-1,6.1058268E-2,-1.3573522E-2,-2.7557148E-3,6.267944E-3,2.4771893E-2,2.7347806E-1,4.3245263E-2,-2.4900915E-1,-6.847458E-2,-3.622351E-1,-4.0742595E-2,4.243253E-2,3.4735135E-3,-1.7411637E-1,1.5891312E-2,5.466609E-2,3.3256605E-1,4.2892018E-1,5.5721413E-2,-1.5256828E-2,-2.605077E-2,7.953652E-4,-7.949426E-3,3.2696468E-1,7.632904E-2,3.7444443E-2,3.5061756E-1,-3.8522404E-1,-6.496445E-2,3.6555824E-1,-1.348059E-1,5.491843E-1,6.9376035E-3,2.9160506E-1,-4.312485E-2,8.2325184E-1,4.9662754E-1,-4.9358433E-1,-7.684386E-1,-1.281068E-1,-5.4341096E-1,2.3503188E-2,7.783779E-3,-2.882544E-1,4.0661603E-2,-5.14721E-1,-3.1150615E-1,-1.9258355E-1,2.7319507E-3,-1.3043997E-1,5.207526E-3,2.8282143E-3,1.8509383E-1,1.4742408E-3,1.4026027E-2,-5.7527944E-3,4.6046525E-3,1.5721495E-1,-2.6794755E-1,1.6365308E-2,-1.14132956E-1,-4.0768743E-1,2.7854939E-3,-1.9664411E-1,1.208335E-2,7.198807E-2,-1.1276749E-1,-4.4444337E-2,1.3047332E-1,2.2570345E-1,4.958251E-1,2.228177E-3,5.462367E-1,3.91437E-1,1.5047853E-1,4.270844E-2,2.4239829E-1,3.6708188E-1,8.875872E-3,-3.0636545E-2,-2.6972386E-1,-1.2441096E-2,6.9159865E-2,2.0054181E-1,3.0195111E-2,-7.4051315E-4,-1.12190405E-2,2.8685415E-2,3.3808175E-1,1.6648566E-2,-1.5379551E-3,1.6663553E-2,-1.844505E-2,3.9247643E-2,1.1676028E-2,3.90828E-1,6.79737E-1,-2.47894E-2,-1.3986858E-2,-3.745625E-2,-1.8779527E-2,3.0530826E-3,-1.1460226E-2,-1.5363582E-2,-3.807772E-2,-7.273339E-3,4.1401014E-3,-1.416678E-2,-2.987104E-4,2.1989234E-2,-4.500181E-3,-1.6550839E-2,-3.1218244E-2,-4.968276E-3,-1.4980088E-2,-2.2725642E-3,-9.883751E-3,-7.085196E-3,1.4098262E-3,-2.3488184E-3,5.295299E-3,4.29242E-4,9.570093E-3,-9.056621E-3,2.1512669E-2,-2.4065019E-4,-1.2579613E-2,-1.0046099E-2,-1.066008E-3,-4.916397E-3,-2.0558046E-2,-9.894859E-3,-1.2538554E-3,6.049642E-3,-1.9809185E-3,1.1584503E-3,-6.9778557E-3,-3.0053495E-3,1.2165547E-2,3.0779878E-3,1.0549207E-2,-3.252553E-3,1.1720778E-2,1.0298735E-2,2.6754236E-2,2.799214E-2,-1.9969502E-3,1.2362262E-2,2.5484711E-2,1.1244375E-2,-3.4755697E-3,3.979987E-5,8.402336E-3,2.7489644E-3,1.9747332E-2,1.2975744E-2,2.2515276E-2,8.994018E-3,-4.833827E-3,-1.5627274E-2,-4.707364E-3,7.5427312E-3,-3.272838E-3,3.0747873E-3,1.0983259E-2,1.907929E-2,3.9158734E-3,-4.100492E-3,3.3105784E-3,6.1620343E-3,1.960943E-2,2.4607973E-2,4.9280792E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,-1,-1,107,109,111,113,-1,-1,115,-1,117,-1,119,121,-1,-1,-1,-1,123,125,127,129,131,-1,-1,-1,133,135,137,139,141,-1,-1,-1,-1,-1,143,145,-1,147,149,151,153,155,157,-1,159,161,163,165,167,169,171,173,-1,175,177,179,181,183,185,-1,187,-1,189,191,-1,-1,-1,-1,193,195,-1,197,199,-1,201,-1,203,205,207,209,211,213,-1,215,217,219,221,223,225,227,-1,229,-1,231,233,-1,-1,-1,-1,235,-1,-1,237,-1,-1,-1,239,241,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.058048E1,2.0640865E1,3.595981E1,1.0636429E1,9.617123E0,1.2366452E1,1.0868408E1,7.8911285E0,0E0,2.7460184E0,1.0035982E0,4.593147E0,8.378632E0,7.2797318E0,4.8048553E0,6.028717E0,2.4434786E0,1.0201321E0,6.9891E-1,7.4449115E-2,7.501526E-1,1.7915745E0,7.8186116E0,5.4228916E0,6.747751E-1,4.1622124E0,4.461282E0,2.013776E0,2.344761E0,5.9580994E-1,2.3246613E0,1.1180785E0,5.48954E-1,6.310129E-1,8.961742E-1,5.6682926E-1,1.7704904E-1,0E0,0E0,4.2388248E-1,3.224995E-1,1.3390961E0,9.563365E-1,1.6009135E0,9.941723E-1,5.2474217E0,1.9155884E0,1.5945578E-1,6.626542E-2,3.8376303E0,3.5926666E0,1.1151004E0,1.1011759E0,3.8481903E-1,8.034763E-1,0E0,1.6205711E0,0E0,0E0,2.603157E0,3.3296757E0,6.763163E-1,1.1720222E0,0E0,0E0,4.365673E-1,0E0,4.90942E-1,0E0,1.5226686E-1,3.3893865E-1,0E0,0E0,0E0,0E0,1.8244183E-1,1.12489656E-1,1.1422577E0,1.2190986E0,4.4882274E-1,0E0,0E0,0E0,5.790367E-1,4.2672372E-1,2.8533347E0,1.336133E0,1.0867572E0,0E0,0E0,0E0,0E0,0E0,1.3178911E0,6.931494E-1,0E0,7.281904E-1,7.264261E-1,5.8187485E-1,9.1065E-1,7.625397E-2,8.872652E-2,0E0,2.851181E-1,4.641692E-1,3.1922436E-1,1.7103748E0,1.6242027E0,6.77227E-1,9.7052234E-1,2.204836E0,0E0,2.9652324E-1,2.549038E-1,1.0455781E0,3.2156086E-1,1.2002373E-1,2.9308033E-1,0E0,1.311776E-1,0E0,2.1212097E-1,6.7243636E-2,0E0,0E0,0E0,0E0,8.901822E-1,3.5501575E-1,0E0,5.316819E-1,2.2821307E-1,0E0,1.8274224E-1,0E0,2.9833072E-1,1.0694054E-1,1.151336E0,1.3616965E0,5.3200626E-1,6.823516E-1,0E0,8.353896E-1,1.6325407E0,7.753577E-1,6.333555E-1,7.1282494E-1,1.157053E0,1.7484063E-1,0E0,2.2672307E-1,0E0,1.9122452E-1,6.718582E-2,0E0,0E0,0E0,0E0,1.1031693E-1,0E0,0E0,1.2165666E-1,0E0,0E0,0E0,6.629629E-1,1.4126492E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,59,59,60,60,61,61,62,62,65,65,67,67,69,69,70,70,75,75,76,76,77,77,78,78,79,79,83,83,84,84,85,85,86,86,87,87,93,93,94,94,96,96,97,97,98,98,99,99,100,100,101,101,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,112,112,113,113,114,114,115,115,116,116,117,117,119,119,121,121,122,122,127,127,128,128,130,130,131,131,133,133,135,135,136,136,137,137,138,138,139,139,140,140,142,142,143,143,144,144,145,145,146,146,147,147,148,148,150,150,152,152,153,153,158,158,161,161,165,165,166,166],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,-1,-1,108,110,112,114,-1,-1,116,-1,118,-1,120,122,-1,-1,-1,-1,124,126,128,130,132,-1,-1,-1,134,136,138,140,142,-1,-1,-1,-1,-1,144,146,-1,148,150,152,154,156,158,-1,160,162,164,166,168,170,172,174,-1,176,178,180,182,184,186,-1,188,-1,190,192,-1,-1,-1,-1,194,196,-1,198,200,-1,202,-1,204,206,208,210,212,214,-1,216,218,220,222,224,226,228,-1,230,-1,232,234,-1,-1,-1,-1,236,-1,-1,238,-1,-1,-1,240,242,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5938586E8,2.14099E5,8.8543115E2,1.8108038E5,1.7654233E0,1.945711E1,5.3942103E-2,2.2198055E6,2.4418726E-3,2.914E3,3.956147E-1,4.2343444E7,1.1410706E3,3E0,1.4299594E8,1.5561539E2,1.3446785E7,4.9544224E1,1.4075E4,1E0,2.3E1,8.1590717E2,2.6476662E6,8.426E3,3.970405E3,2.53383E5,1.0918E4,1.0503613E5,4.423E3,2.8839298E7,1.3649979E10,9.705292E6,4.8297736E7,2.437E3,4.111033E8,2.9357334E-3,-6.1012693E-3,3.904E3,6.788246E0,8.91942E5,7.492813E0,6.2686886E-5,1.101214E6,3.321532E6,1.1392E4,7.31E2,4.3256033E2,2E0,2E0,2.3E1,3.036676E9,2.0992E4,2.458607E0,4.5722485E-2,1.6470588E0,-2.3420747E-2,-6.0009383E-2,3.9712732E7,6.496696E7,1.2E1,3.0497742E3,-7.516183E-3,-3.312069E-2,1.4E1,7.316181E-4,5.4878284E2,-2.6129905E-2,1E0,7.83495E5,-1.3573522E-2,-2.7557148E-3,6.267944E-3,2.4771893E-2,1E0,2.2783158E5,7E0,1E0,5.8E1,-4.0742595E-2,4.243253E-2,3.4735135E-3,7.299E3,8E-3,2.5275471E2,5.3038636E7,3.0996E4,5.5721413E-2,-1.5256828E-2,-2.605077E-2,7.953652E-4,-7.949426E-3,1E0,3.33E2,3.7444443E-2,2.6364462E7,5.1821535E4,6.0218544E1,1.2E1,5.794597E7,6.143686E6,6.9376035E-3,3.4E1,1.05842E3,3.6411794E5,1.3532244E7,2.5604828E2,7.298614E2,2.1515152E0,1.2E1,2.3503188E-2,1E0,9.255088E-1,3.8134545E6,2.12062E5,2.6668E4,2.8945708E6,2.7319507E-3,4.2648572E5,5.207526E-3,1.91E2,2.2292539E8,1.4742408E-3,1.4026027E-2,-5.7527944E-3,4.6046525E-3,4.7E2,9.833887E-1,1.6365308E-2,7.805608E4,5.26E2,2.7854939E-3,1.5485274E6,1.208335E-2,3.1066042E7,1.8071064E7,9.750871E6,1.645614E6,9.232009E2,3.6581E4,2.228177E-3,6.7507215E6,5.314225E2,1.2E1,3.2627738E5,1.348185E6,2.1019447E-1,4.0889E4,-3.0636545E-2,1.5106794E3,-1.2441096E-2,2.609323E3,3.8E1,3.0195111E-2,-7.4051315E-4,-1.12190405E-2,2.8685415E-2,1.9495119E6,1.6648566E-2,-1.5379551E-3,2.5534486E4,-1.844505E-2,3.9247643E-2,1.1676028E-2,1E0,2.1196339E1,-2.47894E-2,-1.3986858E-2,-3.745625E-2,-1.8779527E-2,3.0530826E-3,-1.1460226E-2,-1.5363582E-2,-3.807772E-2,-7.273339E-3,4.1401014E-3,-1.416678E-2,-2.987104E-4,2.1989234E-2,-4.500181E-3,-1.6550839E-2,-3.1218244E-2,-4.968276E-3,-1.4980088E-2,-2.2725642E-3,-9.883751E-3,-7.085196E-3,1.4098262E-3,-2.3488184E-3,5.295299E-3,4.29242E-4,9.570093E-3,-9.056621E-3,2.1512669E-2,-2.4065019E-4,-1.2579613E-2,-1.0046099E-2,-1.066008E-3,-4.916397E-3,-2.0558046E-2,-9.894859E-3,-1.2538554E-3,6.049642E-3,-1.9809185E-3,1.1584503E-3,-6.9778557E-3,-3.0053495E-3,1.2165547E-2,3.0779878E-3,1.0549207E-2,-3.252553E-3,1.1720778E-2,1.0298735E-2,2.6754236E-2,2.799214E-2,-1.9969502E-3,1.2362262E-2,2.5484711E-2,1.1244375E-2,-3.4755697E-3,3.979987E-5,8.402336E-3,2.7489644E-3,1.9747332E-2,1.2975744E-2,2.2515276E-2,8.994018E-3,-4.833827E-3,-1.5627274E-2,-4.707364E-3,7.5427312E-3,-3.272838E-3,3.0747873E-3,1.0983259E-2,1.907929E-2,3.9158734E-3,-4.100492E-3,3.3105784E-3,6.1620343E-3,1.960943E-2,2.4607973E-2,4.9280792E-2],"split_indices":[19,109,7,2,51,27,40,55,0,49,38,2,26,44,51,3,44,51,44,55,9,72,3,51,31,2,51,9,9,32,2,44,5,9,44,2,7,0,0,2,52,9,53,38,1,27,2,11,51,16,16,8,7,28,40,0,53,0,0,44,44,3,51,0,0,3,0,51,0,108,32,0,0,0,0,15,27,3,13,3,0,0,0,2,56,51,7,9,0,0,0,0,0,6,8,0,46,32,55,17,1,49,0,3,4,32,1,51,51,52,17,0,63,26,46,28,1,44,0,27,0,8,7,0,0,0,0,1,53,0,27,2,0,46,0,44,44,9,9,32,1,0,46,4,8,27,1,34,2,0,51,0,51,3,0,0,0,0,27,0,0,32,0,0,0,15,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.228E3,7.12E2,1.516E3,4.04E2,3.08E2,8.68E2,6.48E2,4.01E2,3E0,2.52E2,5.6E1,3.47E2,5.21E2,4.62E2,1.86E2,3.18E2,8.3E1,1.59E2,9.3E1,7E0,4.9E1,2.26E2,1.21E2,4.87E2,3.4E1,3.92E2,7E1,5E1,1.36E2,1.3E1,3.05E2,7.2E1,1.1E1,6.4E1,9.5E1,7.7E1,1.6E1,2E0,5E0,1.9E1,3E1,2.01E2,2.5E1,1.2E1,1.09E2,4.6E2,2.7E1,2.4E1,1E1,2.45E2,1.47E2,4.5E1,2.5E1,2E1,3E1,1.4E1,1.22E2,3E0,1E1,2.27E2,7.8E1,2.1E1,5.1E1,3E0,8E0,6E1,4E0,9E1,5E0,3.1E1,4.6E1,1.1E1,5E0,4E0,1.5E1,2.1E1,9E0,1.42E2,5.9E1,2.1E1,4E0,9E0,3E0,5.2E1,5.7E1,3.78E2,8.2E1,2.4E1,3E0,1.3E1,1.1E1,2E0,8E0,1.2E2,1.25E2,1.8E1,1.29E2,2.6E1,1.9E1,2E1,5E0,1.6E1,4E0,1E1,2E1,2.1E1,1.01E2,1.79E2,4.8E1,3.8E1,4E1,2E0,1.9E1,3.6E1,1.5E1,2.1E1,3.9E1,8.2E1,8E0,2.9E1,2E0,3.2E1,1.4E1,3E0,1.8E1,2E0,7E0,6E0,1.36E2,5E0,5.4E1,1.9E1,2E0,5E1,2E0,4E1,1.7E1,1.64E2,2.14E2,5.1E1,3.1E1,6E0,1.8E1,8.7E1,3.3E1,1.05E2,2E1,1.23E2,6E0,6E0,2E1,7E0,1.2E1,1.4E1,6E0,3E0,2E0,1E1,6E0,8E0,2E0,1.8E1,2E0,1.9E1,2E0,6.6E1,3.5E1,1.37E2,4.2E1,4E1,8E0,1.5E1,2.3E1,2.5E1,1.5E1,6E0,1.3E1,3.3E1,3E0,3E0,1.2E1,1.3E1,8E0,4E0,3.5E1,1.3E1,6.9E1,2.5E1,4E0,2.2E1,1E1,2E0,1.2E1,3E0,3E0,5E0,1.31E2,2.4E1,3E1,3E0,1.6E1,4.4E1,6E0,2.6E1,1.4E1,4E0,1.3E1,1.54E2,1E1,1.34E2,8E1,5E0,4.6E1,9E0,2.2E1,1.6E1,2E0,5.3E1,3.4E1,2.3E1,1E1,8.2E1,2.3E1,1.1E1,9E0,7.8E1,4.5E1,2E0,4E0,1.3E1,7E0,7E0,5E0,4E0,1E1,4E0,2E0,6E0,1.2E1,1E1,5.6E1,2.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"243","size_leaf_vector":"1"}},{"base_weights":[8.8383246E-4,-2.5269964E-1,1.2317646E-1,-4.1435224E-1,-9.3415946E-2,4.221171E-2,4.2596915E-1,-4.901445E-1,-2.4166197E-1,-1.7782678E-1,2.4209651E-1,-5.3461447E-2,2.438321E-1,5.113181E-1,1.4887221E-1,-3.7212598E-1,-5.849415E-1,-3.099462E-1,-8.6999945E-2,-2.0375182E-1,5.0412368E-2,-4.96665E-4,3.027531E-1,-2.1256068E-1,8.591587E-3,9.173802E-2,3.355914E-1,3.3814338E-1,6.476262E-1,-4.940987E-1,2.2022785E-1,-3.9576176E-1,1.7326664E-2,-6.5910244E-1,-3.646886E-1,-3.4781212E-1,-9.38619E-2,-5.087383E-3,-2.0133837E-1,-1.5170965E-1,-4.860342E-1,-2.0713643E-3,9.546442E-3,2.4237505E-1,5.496757E-1,-3.1648105E-1,-1.266051E-1,7.360139E-1,-6.5516704E-3,2.9416276E-2,2.3695901E-1,3.6835197E-1,-2.6731327E-1,2.5818148E-1,5.5258244E-1,5.3027764E-2,6.057483E-1,-1.0891285E-2,-3.112549E-2,3.7929498E-2,4.2347363E-1,-4.7158688E-1,-2.7238688E-1,-7.79269E-1,-5.019018E-1,-4.2155203E-1,-1.6900219E-1,-3.0039346E-1,-5.361633E-1,-6.3583683E-3,4.8995246E-3,-3.333399E-2,8.372019E-3,-1.382248E-2,-3.2688624E-3,-2.516323E-1,-4.8116438E-2,-4.077366E-1,-3.8611E-2,-8.136356E-3,2.6370612E-1,3.2965634E-2,1.5559675E-2,-2.7271828E-1,-6.577043E-1,-9.653321E-2,-3.2919544E-1,1.2440951E-2,3.625442E-2,2.5151871E-2,-3.2711917E-1,7.059766E-2,-1.4707541E-1,1.8242757E-1,4.041231E-1,2.7866364E-1,5.142446E-1,-3.9004543E-1,4.8146783E-3,3.0296746E-1,1.1366181E-1,6.2144583E-1,-1.1381085E-2,5.607248E-1,4.2209655E-2,2.05428E-1,-2.9228114E-2,5.183941E-1,1.1275298E-1,-3.5741445E-1,-7.472203E-1,-4.9063005E-2,-3.9390823E-1,-6.6691095E-1,-5.566091E-2,-8.038351E-1,-3.9857295E-1,-2.2025734E-2,-7.1175643E-3,-5.9702044E-4,-1.225481E-2,-3.5984352E-1,-1.7836368E-1,-1.2202126E-2,-2.8785015E-2,3.9731404E-3,-3.6014714E-3,-1.8284175E-1,-4.8586917E-1,-9.254413E-2,1.2974189E-1,-4.585242E-1,-1.10061325E-1,1.6350435E-1,3.4935158E-1,-3.319115E-1,-1.7415799E-1,-1.1039301E-2,-3.5423547E-2,-1.5785739E-1,-2.2738812E-2,-1.9275554E-2,-9.362135E-3,-5.667435E-2,1.298939E-1,-2.5324494E-1,-5.095461E-1,1.3293464E-1,-5.436115E-2,-5.2576803E-2,-2.3176897E-2,7.0916396E-4,9.49392E-3,2.2530679E-2,6.3119335E-3,1.9808735E-1,3.7088886E-1,5.622793E-1,1.7117417E-1,-2.1868743E-2,-5.463385E-4,1.8642198E-1,3.716693E-1,-1.1317419E-1,8.67546E-3,4.7785223E-2,5.0007373E-1,4.4076735E-1,6.75054E-1,1.0447379E-3,1.1019077E-2,-1.4981635E-1,1.214365E-1,6.2047327E-1,1.2132448E-2,5.374802E-3,1.4746528E-2,-2.2628487E-3,-1.8394776E-2,-1.1969168E-2,-3.718037E-2,-1.5320987E-2,1.6801613E-3,-2.4590167E-4,-2.0638006E-2,-2.0533808E-2,-3.1941198E-2,-4.019912E-2,-1.284287E-2,-1.2018668E-2,-2.4427759E-2,-1.7780194E-2,-7.58422E-3,-9.262631E-3,1.7409305E-3,-1.0456578E-2,3.6771037E-3,-2.6844682E-2,-6.522962E-3,-3.0810712E-3,-1.6416933E-2,-1.4147542E-4,1.3542756E-2,-5.242441E-3,-2.2136128E-2,-8.526245E-3,2.5836173E-3,1.0102396E-2,3.2932872E-3,2.0509915E-2,9.90648E-3,-1.38081135E-2,-2.4941156E-2,7.305986E-3,-9.440067E-3,-5.097038E-3,-1.1760378E-2,2.9983412E-4,-1.1270824E-2,-6.5708105E-3,1.3234456E-3,3.5451977E-3,1.6306655E-2,-1.2264011E-2,2.20597E-3,-3.1432821E-3,-2.6128205E-2,7.1505336E-3,-7.1850265E-3,-7.87327E-3,2.4282874E-3,-7.6081147E-3,4.4518798E-3,5.3916527E-3,1.5290055E-2,1.9210706E-2,1.1080183E-2,2.8345434E-2,1.3032452E-2,4.2176523E-4,1.2923739E-2,1.5386599E-2,3.570968E-3,5.7850867E-3,1.7608093E-2,4.9985687E-5,-1.0314332E-2,2.755202E-2,7.4775494E-3,1.2874476E-2,2.352955E-2,3.1315632E-2,1.2752607E-2,6.3600275E-3,-9.114609E-3,9.271488E-3,9.3891646E-4,3.17459E-2,8.277268E-3,3.395048E-3,-5.0906236E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,77,-1,-1,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,-1,-1,105,107,109,111,113,115,117,119,121,123,-1,-1,125,-1,-1,-1,127,129,131,-1,-1,133,-1,-1,135,137,139,141,-1,-1,143,145,147,149,151,153,155,157,159,-1,161,163,165,-1,167,-1,169,171,173,175,177,179,181,183,185,-1,187,189,-1,-1,-1,-1,191,193,-1,-1,-1,-1,195,197,199,201,203,205,207,209,211,213,-1,-1,215,217,-1,-1,219,221,223,225,227,229,231,-1,-1,-1,-1,-1,233,235,237,239,-1,-1,241,243,245,-1,-1,247,249,251,-1,-1,253,255,257,-1,259,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.996181E1,1.8860786E1,3.7294662E1,4.64579E0,1.0550101E1,2.3199207E1,7.5155067E0,2.601654E0,1.1664262E0,1.0169723E1,1.1037655E0,8.057859E0,5.370203E0,5.5642624E0,3.6415434E0,2.2031822E0,2.0137482E0,6.05567E-1,3.3614284E-1,4.2159166E0,0E0,1.6268641E-1,7.6129484E-1,2.0054836E0,6.4819613E0,1.323442E0,4.8585777E0,1.7434998E0,2.2728348E0,2.2686982E-1,2.5775335E0,9.192982E-1,0E0,1.5356903E0,3.404708E-1,4.2841005E-1,1.3756754E-1,1.24238946E-1,1.793496E-1,2.555017E0,9.4463634E-1,0E0,0E0,4.9757218E-1,1.541431E-1,1.358717E0,7.4194026E-1,1.3840342E-1,5.872351E0,7.662731E-1,3.2736754E-1,2.8531342E0,6.376588E-1,4.96305E-1,1.7617712E0,0E0,1.3592453E0,0E0,0E0,4.334587E-1,9.1484356E-1,1.8728733E0,1.2019114E0,2.0124855E0,1.1459389E0,4.0422535E-1,1.548622E-1,3.348241E-1,1.4744282E-1,0E0,0E0,1.1890474E-1,0E0,0E0,0E0,1.9414363E0,9.839326E-1,5.391445E-1,0E0,0E0,3.5480738E-1,0E0,0E0,4.8476648E-1,3.7702703E-1,5.032531E-1,1.0106933E-1,0E0,0E0,4.5155206E0,5.717459E-1,6.693387E-1,6.626416E-1,1.5589368E-1,1.7819679E-1,9.966192E-1,1.3209705E0,3.4563816E-1,0E0,4.2713976E-1,3.9305174E-1,8.944998E-1,0E0,1.2796402E0,0E0,7.052392E-2,5.269078E-1,4.7278118E-1,2.0405923E-1,7.311349E-1,4.6187305E-1,4.3643433E-1,6.953182E-1,6.708908E-2,0E0,2.2917843E-1,5.5109787E-1,0E0,0E0,0E0,0E0,1.6187E-1,1.21596575E-1,0E0,0E0,0E0,0E0,1.2767279E0,9.392452E-1,6.301129E-1,5.635831E-1,2.8736115E-1,1.0230796E-1,1.1367202E-1,2.3918843E-1,1.6801357E-1,4.601146E-1,0E0,0E0,2.5219035E-1,3.4702295E-1,0E0,0E0,2.2536292E0,2.716069E0,2.2093725E-1,3.762586E-1,4.407227E-1,3.8372013E-1,3.1275535E-1,0E0,0E0,0E0,0E0,0E0,8.31223E-1,3.4103584E-1,1.1330433E0,2.1931115E-1,0E0,0E0,3.8572794E-1,1.1020994E-1,7.8861E-2,0E0,0E0,7.351718E-1,5.807028E-1,1.782322E-1,0E0,0E0,2.6750746E-1,1.0486409E-1,4.7237825E-1,0E0,6.58431E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,71,71,75,75,76,76,77,77,80,80,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,103,103,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,113,113,115,115,116,116,121,121,122,122,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,140,140,143,143,144,144,145,145,146,146,147,147,148,148,149,149,155,155,156,156,157,157,158,158,161,161,162,162,163,163,166,166,167,167,168,168,171,171,172,172,173,173,175,175],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,78,-1,-1,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,-1,-1,106,108,110,112,114,116,118,120,122,124,-1,-1,126,-1,-1,-1,128,130,132,-1,-1,134,-1,-1,136,138,140,142,-1,-1,144,146,148,150,152,154,156,158,160,-1,162,164,166,-1,168,-1,170,172,174,176,178,180,182,184,186,-1,188,190,-1,-1,-1,-1,192,194,-1,-1,-1,-1,196,198,200,202,204,206,208,210,212,214,-1,-1,216,218,-1,-1,220,222,224,226,228,230,232,-1,-1,-1,-1,-1,234,236,238,240,-1,-1,242,244,246,-1,-1,248,250,252,-1,-1,254,256,258,-1,260,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.426E3,1E0,1.5343539E0,7.920064E5,2.8839298E7,6.748543E-1,2.0368582E8,1.1458888E4,2.3125623E5,6.76E2,1.06E3,1.7669278E5,2.28125E0,1.3141646E3,1.4072216E-1,1.998E3,5.59E2,1E0,5.0412368E-2,6.7E1,6.947958E-1,6.3039363E4,8E0,3.0121632E6,4.898012E-1,9.3058E4,1.059448E7,8.91942E5,7.3610186E3,4.433625E2,1.7326664E-2,1.8903887E0,1.5859042E6,1.98714E5,3.454E3,2.695E3,1.697E3,2.1446484E-1,2.2980049E10,-2.0713643E-3,9.546442E-3,1.2E1,1.9610367E0,1.3E1,1.8955729E1,2.127E3,3.9712732E7,2.1724138E0,6.057671E6,3.097E3,2.4519731E3,1.9253924E7,1.314806E8,5.3027764E-2,1.6828056E1,-1.0891285E-2,-3.112549E-2,3.5301748E7,1.1159378E12,2.5943396E0,1.5240682E9,7.95629E5,1.1863768E-4,2.5943396E0,1.157E3,1.09E3,1.48614E5,-6.3583683E-3,4.8995246E-3,1.822542E0,8.372019E-3,-1.382248E-2,-3.2688624E-3,1.2199979E10,6.0809356E5,2.9799202E3,-3.8611E-2,-8.136356E-3,1.51529E3,3.2965634E-2,1.5559675E-2,4.7E2,1.6E1,3.79E2,1.901E3,1.2440951E-2,3.625442E-2,1.7669278E5,1.0828989E8,2.53E2,1.8E1,4.3360384E7,3.471E3,2.065904E6,2E0,2.783636E6,4.8146783E-3,1.00663277E9,7.718584E5,1.417988E6,-1.1381085E-2,1.1251919E6,4.2209655E-2,5.84E2,1E0,6.496696E7,2.5016512E12,3.1E1,1.4523809E0,2.0339169E-2,1.4523809E0,1.3177E4,-5.566091E-2,1.517E3,9.222717E-1,-2.2025734E-2,-7.1175643E-3,-5.9702044E-4,-1.225481E-2,7.804304E-1,3.557179E1,-1.2202126E-2,-2.8785015E-2,3.9731404E-3,-3.6014714E-3,1.0122174E3,2.867292E3,1.2427474E8,4.653159E9,5.503109E8,2.7635896E-1,5.936784E-1,2.7457875E6,3.09627E5,8.490231E-1,-1.1039301E-2,-3.5423547E-2,1.83096E5,3.6581E4,-1.9275554E-2,-9.362135E-3,7.44393E5,8.027061E2,3.3E2,1.1E1,1.6714952E8,4.29E2,4.1828512E6,-2.3176897E-2,7.0916396E-4,9.49392E-3,2.2530679E-2,6.3119335E-3,2.742277E6,1.1E1,2E0,1.4523809E0,-2.1868743E-2,-5.463385E-4,1.6908307E-1,1.7227913E3,1.6627079E-2,8.67546E-3,4.7785223E-2,6.38041E5,1.5353E4,1E0,1.0447379E-3,1.1019077E-2,1.8835404E3,3.164557E-2,7.325843E0,1.2132448E-2,1.85224E5,1.4746528E-2,-2.2628487E-3,-1.8394776E-2,-1.1969168E-2,-3.718037E-2,-1.5320987E-2,1.6801613E-3,-2.4590167E-4,-2.0638006E-2,-2.0533808E-2,-3.1941198E-2,-4.019912E-2,-1.284287E-2,-1.2018668E-2,-2.4427759E-2,-1.7780194E-2,-7.58422E-3,-9.262631E-3,1.7409305E-3,-1.0456578E-2,3.6771037E-3,-2.6844682E-2,-6.522962E-3,-3.0810712E-3,-1.6416933E-2,-1.4147542E-4,1.3542756E-2,-5.242441E-3,-2.2136128E-2,-8.526245E-3,2.5836173E-3,1.0102396E-2,3.2932872E-3,2.0509915E-2,9.90648E-3,-1.38081135E-2,-2.4941156E-2,7.305986E-3,-9.440067E-3,-5.097038E-3,-1.1760378E-2,2.9983412E-4,-1.1270824E-2,-6.5708105E-3,1.3234456E-3,3.5451977E-3,1.6306655E-2,-1.2264011E-2,2.20597E-3,-3.1432821E-3,-2.6128205E-2,7.1505336E-3,-7.1850265E-3,-7.87327E-3,2.4282874E-3,-7.6081147E-3,4.4518798E-3,5.3916527E-3,1.5290055E-2,1.9210706E-2,1.1080183E-2,2.8345434E-2,1.3032452E-2,4.2176523E-4,1.2923739E-2,1.5386599E-2,3.570968E-3,5.7850867E-3,1.7608093E-2,4.9985687E-5,-1.0314332E-2,2.755202E-2,7.4775494E-3,1.2874476E-2,2.352955E-2,3.1315632E-2,1.2752607E-2,6.3600275E-3,-9.114609E-3,9.271488E-3,9.3891646E-4,3.17459E-2,8.277268E-3,3.395048E-3,-5.0906236E-3],"split_indices":[19,55,2,109,41,27,44,26,7,51,27,2,2,27,53,51,37,10,0,111,0,8,33,27,17,27,26,2,12,9,4,4,0,52,27,1,0,2,10,40,5,0,0,17,38,8,57,1,44,52,27,2,4,44,1,0,34,0,0,44,30,55,5,28,41,55,0,2,12,0,0,57,0,0,0,5,27,4,0,0,4,0,0,1,3,2,28,0,0,27,44,10,3,5,10,27,6,12,0,7,46,28,0,27,0,0,109,44,30,0,53,26,53,9,0,0,26,0,0,0,0,38,57,0,0,0,0,51,4,44,5,5,26,26,46,9,52,0,0,46,1,0,0,1,51,3,3,5,10,44,0,0,0,0,0,9,8,16,53,0,0,37,4,56,0,0,9,2,6,0,0,4,56,52,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.254E3,7.33E2,1.521E3,3.63E2,3.7E2,1.201E3,3.2E2,2.51E2,1.12E2,2.96E2,7.4E1,8.15E2,3.86E2,2.44E2,7.6E1,1.14E2,1.37E2,7.7E1,3.5E1,2.91E2,5E0,1.5E1,5.9E1,2.28E2,5.87E2,1.46E2,2.4E2,1.09E2,1.35E2,7E0,6.9E1,1.11E2,3E0,1.01E2,3.6E1,6.5E1,1.2E1,2.1E1,1.4E1,2.47E2,4.4E1,1.3E1,2E0,4.9E1,1E1,1.02E2,1.26E2,1.1E1,5.76E2,1.03E2,4.3E1,2.28E2,1.2E1,8.1E1,2.8E1,8E0,1.27E2,4E0,3E0,3.7E1,3.2E1,6.7E1,4.4E1,5.5E1,4.6E1,2.7E1,9E0,5.4E1,1.1E1,1E1,2E0,1.9E1,2E0,7E0,7E0,1.25E2,1.22E2,3.8E1,6E0,2E0,4.7E1,4E0,6E0,9.2E1,1E1,1.11E2,1.5E1,2E0,9E0,5.25E2,5.1E1,8.4E1,1.9E1,3.4E1,9E0,1.43E2,8.5E1,9E0,3E0,6.1E1,2E1,2.6E1,2E0,1.14E2,1.3E1,1E1,2.7E1,2.4E1,8E0,4.9E1,1.8E1,1.6E1,2.8E1,4.6E1,9E0,1E1,3.6E1,2.1E1,6E0,4E0,5E0,3.5E1,1.9E1,4E0,7E0,5E0,1.4E1,9.8E1,2.7E1,9.8E1,2.4E1,3.2E1,6E0,2.3E1,2.4E1,5.6E1,3.6E1,3E0,7E0,6E1,5.1E1,7E0,8E0,2.95E2,2.3E2,3.8E1,1.3E1,5.6E1,2.8E1,1.6E1,3E0,5E0,2.9E1,6E0,3E0,7.8E1,6.5E1,7.4E1,1.1E1,7E0,2E0,2.4E1,3.7E1,5E0,1.5E1,4E0,2.2E1,5.8E1,5.6E1,2E0,8E0,1.5E1,1.2E1,1.6E1,8E0,6E0,2E0,7E0,4.2E1,3E0,1.5E1,3E0,1.3E1,4E0,2.4E1,9E0,3.7E1,8E0,2E0,2E1,1.6E1,2.9E1,6E0,1.7E1,2E0,8.3E1,1.5E1,2E1,7E0,9.1E1,7E0,1.4E1,1E1,3E0,2.9E1,4E0,2E0,1.3E1,1E1,1.2E1,1.2E1,5.2E1,4E0,3E0,3.3E1,4.3E1,1.7E1,4.6E1,5E0,1.45E2,1.5E2,1.89E2,4.1E1,3.6E1,2E0,2E0,1.1E1,5.2E1,4E0,1.3E1,1.5E1,9E0,7E0,5.1E1,2.7E1,4.4E1,2.1E1,5.9E1,1.5E1,5E0,6E0,9E0,1.5E1,3E0,3.4E1,3E0,2E0,1.6E1,6E0,2.1E1,3.7E1,5.3E1,3E0,2E0,1.3E1,6E0,6E0,1.3E1,3E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"261","size_leaf_vector":"1"}},{"base_weights":[6.693766E-3,-2.458136E-1,1.2330852E-1,-4.0008393E-1,-9.735293E-2,1.0372925E-2,3.0531844E-1,-4.903753E-1,-2.1599165E-1,-1.7817634E-1,2.2958863E-1,-1.7868394E-1,5.7813544E-2,3.5636494E-1,-3.0780528E-2,-4.6264175E-1,-9.9542403E-1,-3.4665427E-1,-1.4609909E-1,-2.0195365E-1,5.426078E-2,1.6924964E-1,5.1215124E-1,-2.662025E-1,-9.648643E-2,1.5924984E-1,-3.7961856E-2,2.806829E-1,5.5340767E-1,-2.207923E-1,1.5050016E-1,-4.9293855E-1,-2.3006304E-1,-5.5695657E-2,-1.2542319E-2,-3.8025784E-1,-8.710319E-3,-2.894788E-1,-8.972279E-2,-4.560821E-1,-1.5806432E-1,9.080736E-3,2.4325277E-1,5.9107167E-1,3.5479446E-3,-3.2767773E-1,-1.974456E-1,4.213656E-2,-1.3483992E-1,1.1697899E-1,5.273863E-1,-1.6957138E-1,9.591359E-2,2.3498239E-1,5.503672E-1,2.718046E-1,6.1635303E-1,-1.7422292E-1,-3.196464E-2,6.110556E-2,1.602677E-2,-4.0933302E-1,-5.850963E-1,2.0261312E-1,-3.6333302E-1,-4.9039815E-3,-1.7895762E-2,-1.5129866E-1,-4.9510926E-1,-3.4558803E-3,-1.3049512E-1,-2.7632117E-1,-5.847506E-1,-1.0832011E-1,-3.377389E-1,-9.214218E-3,9.3516216E-2,1.2494322E-1,3.1397316E-1,3.0766912E-2,1.305827E-2,-3.5449982E-1,-1.2759751E-1,2.2643599E-1,-2.5609168E-1,2.1107816E-4,9.489666E-3,-6.3763484E-2,-2.2285531E-1,1.9160165E-1,8.627094E-3,3.6810555E-2,4.0652382E-1,-4.1020167E-1,-1.0637924E-1,5.253257E-2,4.4459176E-1,2.5767478E-1,-9.282038E-2,8.32024E-1,3.483309E-1,8.053797E-3,3.922353E-1,6.500258E-1,1.5876158E-1,7.264422E-2,-3.0068332E-1,-5.3568784E-2,1.15387455E-1,-4.906589E-1,-2.1949063E-1,-7.1001077E-1,-4.9098486E-1,1.4531539E-2,1.5213789E-3,-1.0835567E-2,-5.771418E-1,4.5054653E-3,-9.117902E-3,-2.7223803E-2,-5.327196E-3,-3.8764123E-3,2.9013583E-3,-1.6027951E-1,1.1003494E-3,1.2636831E-3,-3.1375518E-1,-6.6582924E-1,-1.1049393E-2,-1.4970776E-1,8.719664E-2,-4.4685477E-1,-8.6803E-2,-3.517197E-3,7.372156E-3,-9.5303816E-4,7.1411137E-3,3.5310245E-1,5.940543E-3,-3.9312732E-1,-9.023305E-3,-8.702785E-3,1.0253406E-3,1.45892715E-2,1.0509419E-3,-1.9286534E-1,-5.2660656E-1,-1.58614E-1,1.466746E-2,-2.1658496E-3,-2.5138098E-1,2.2288646E-1,-1.4539912E-1,-1.0440098E-1,1.01682305E-1,2.7930292E-1,2.6156876E-2,-4.968558E-1,-2.5588907E-3,-1.4959918E-1,1.9283535E-1,-2.516809E-1,8.5558996E-2,1.0806703E-2,5.529357E-1,1.22811146E-1,3.2905322E-1,-1.5069786E-1,1.3793935E-2,1.912996E-3,9.129338E-1,-1.1037575E-2,3.9593247E-1,1.9738097E-2,5.89853E-3,5.248377E-1,7.3661274E-1,1.381618E-2,-4.5518023E-3,1.8222312E-2,-6.65585E-2,-1.828006E-2,-1.4215295E-1,-5.1096855E-3,2.8787737E-3,-9.439254E-3,9.886767E-3,-2.420263E-2,-8.986338E-3,-5.5570262E-3,-1.9455923E-2,-4.8312217E-2,-2.7978916E-2,-7.9893E-3,-2.453158E-2,-3.1419896E-2,-8.602314E-3,-3.687215E-3,-8.950522E-3,-1.587607E-2,-1.1142879E-3,-1.0215627E-2,-3.199738E-2,-3.8756686E-3,-1.0465101E-2,-2.3818691E-3,1.0417451E-2,-1.5274186E-2,-3.5670035E-2,6.715964E-3,-8.925362E-3,1.8329253E-2,9.016757E-3,-1.2973456E-2,-2.2080164E-2,-3.449248E-3,-1.145637E-2,-3.0270057E-2,-5.180203E-3,-1.8862773E-3,-9.315549E-3,-1.0274316E-3,5.160364E-3,9.5563446E-4,-1.2330228E-2,7.7751623E-3,1.6914237E-2,-1.8078282E-2,-3.844842E-3,5.4834434E-3,-7.0825326E-3,-5.397137E-3,5.89754E-3,1.6063228E-2,3.1317773E-3,-1.7444815E-2,-3.3461377E-2,-8.361033E-3,1.8362813E-3,1.9170474E-2,4.9222484E-3,-1.3688302E-3,-1.3445192E-2,6.4891594E-4,8.5723465E-3,2.9612502E-2,1.1784813E-2,1.3203338E-2,2.3864168E-3,1.8235907E-2,7.825362E-3,-1.0256839E-2,2.327448E-3,4.9665898E-2,2.655393E-2,2.022652E-2,-9.526582E-3,2.5290484E-2,1.3842472E-3,3.517921E-2,2.0865222E-2,4.553241E-3,-6.9066607E-3,-8.215348E-3,1.0480199E-3,6.6120704E-3,-4.388829E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,-1,-1,65,-1,67,69,71,73,75,77,79,-1,81,83,85,87,89,91,93,95,97,99,101,103,105,-1,107,-1,109,111,113,115,-1,-1,117,119,121,123,125,127,129,131,-1,133,135,137,-1,-1,139,141,143,145,-1,-1,147,149,151,153,-1,155,157,159,161,163,165,167,169,171,-1,173,175,177,179,181,183,185,187,189,191,193,-1,-1,-1,195,-1,-1,-1,-1,-1,-1,197,-1,-1,199,201,-1,203,205,207,209,-1,-1,-1,-1,211,-1,213,-1,-1,-1,-1,-1,215,217,219,221,-1,223,225,227,229,231,233,-1,235,-1,237,239,241,243,-1,245,247,249,251,-1,-1,253,-1,255,-1,-1,257,259,-1,-1,-1,261,-1,263,-1,-1,265,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.487058E1,1.5901592E1,3.0973818E1,5.562168E0,9.448042E0,8.360446E0,9.910374E0,2.7330208E0,9.904413E-1,9.603861E0,1.1238363E0,1.3195572E0,7.2437963E0,7.31588E0,2.6855297E0,1.4227104E0,1.51441E0,1.2900925E-1,5.8835185E-1,3.073205E0,0E0,7.08971E-1,3.6449385E-1,3.157177E-1,5.2671605E-1,5.5715494E0,6.782069E0,4.356867E0,2.2785988E0,7.356831E-1,7.06024E-1,1.2398415E0,1.6305102E0,0E0,0E0,8.379936E-2,0E0,5.217855E-1,1.9651794E-1,7.867327E-1,2.1263566E0,3.7805587E-1,3.0381393E-1,7.183337E-2,0E0,2.0162916E-1,1.1766435E0,1.3709E-1,4.6748817E-1,2.634222E0,9.4699E-1,2.9063163E0,2.8657131E0,2.352209E0,2.6692343E0,2.355994E-1,1.6225395E0,1.1457238E0,0E0,1.8669865E-1,0E0,1.4979706E0,6.9766617E-1,1.29237E-1,4.1862726E-1,0E0,0E0,1.959467E-1,2.4993241E-1,1.1092713E-1,1.7909849E-1,2.2514212E-1,4.7282982E-1,1.5586219E0,1.3850279E0,0E0,1.906625E-1,8.469653E-2,1.3244987E-1,0E0,0E0,1.614399E-1,7.23231E-2,1.09342486E-1,5.825825E-1,0E0,0E0,3.2965195E-1,1.5771782E-1,2.0672069E0,1.4200078E0,0E0,4.485612E-1,1.1978364E0,2.0333138E0,1.739585E0,3.3647585E-1,2.7801094E0,5.2904344E-1,1.3033285E0,9.984832E-1,0E0,6.921959E-2,7.407265E-1,3.7235197E-1,6.2368965E-1,3.3480382E-1,7.664041E-2,2.5917697E-1,8.178673E-1,6.185081E-1,5.234928E-1,7.682371E-1,0E0,0E0,0E0,1.8480325E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.829547E-2,0E0,0E0,1.7280567E-1,1.6476345E-1,0E0,8.050871E-1,6.913779E-1,1.0782628E0,4.703145E-1,0E0,0E0,0E0,0E0,7.869911E-2,0E0,1.7269659E-1,0E0,0E0,0E0,0E0,0E0,2.194618E-1,3.3318853E-1,1.0170114E-1,9.534815E-2,0E0,1.7142665E-1,1.3003263E0,2.2712806E-1,7.3478925E-1,4.82718E-1,2.6101398E-1,0E0,5.797987E-1,0E0,9.146435E-1,3.4189278E-1,1.6153169E-1,1.1430732E0,0E0,2.1450043E-1,1.2035758E0,2.1303883E0,3.029219E-1,0E0,0E0,4.9820042E-1,0E0,1.0185623E0,0E0,0E0,7.507715E-1,2.9943848E-1,0E0,0E0,0E0,1.5906385E-1,0E0,7.5670585E-2,0E0,0E0,1.5020722E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,76,76,77,77,78,78,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,112,112,116,116,123,123,126,126,127,127,129,129,130,130,131,131,132,132,137,137,139,139,145,145,146,146,147,147,148,148,150,150,151,151,152,152,153,153,154,154,155,155,157,157,159,159,160,160,161,161,162,162,164,164,165,165,166,166,167,167,170,170,172,172,175,175,176,176,180,180,182,182,185,185],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,-1,-1,66,-1,68,70,72,74,76,78,80,-1,82,84,86,88,90,92,94,96,98,100,102,104,106,-1,108,-1,110,112,114,116,-1,-1,118,120,122,124,126,128,130,132,-1,134,136,138,-1,-1,140,142,144,146,-1,-1,148,150,152,154,-1,156,158,160,162,164,166,168,170,172,-1,174,176,178,180,182,184,186,188,190,192,194,-1,-1,-1,196,-1,-1,-1,-1,-1,-1,198,-1,-1,200,202,-1,204,206,208,210,-1,-1,-1,-1,212,-1,214,-1,-1,-1,-1,-1,216,218,220,222,-1,224,226,228,230,232,234,-1,236,-1,238,240,242,244,-1,246,248,250,252,-1,-1,254,-1,256,-1,-1,258,260,-1,-1,-1,262,-1,264,-1,-1,266,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,1.0950326E6,1E0,1.5343539E0,4.67E2,3.0923106E-10,1.2427474E8,1.172E3,2.14099E5,3.7379084E0,7.805608E4,2E1,1.8137958E6,6.743895E5,5.121E3,2.3902743E0,4.7325465E-1,1.1347137E2,8.728293E-1,5.426078E-2,1.5824156E5,7.083E3,4.7E2,1.44E0,2.914E3,1.023747E6,6.7652373E3,4.9969565E2,1.566593E6,3.564E3,1.0715278E7,1.5062694E9,-5.5695657E-2,-1.2542319E-2,7.6256483E-6,-8.710319E-3,3.790491E2,1.9358906E6,1.3448806E8,6.747114E7,2.1E1,3.0990322E5,5.857143E0,3.5479446E-3,2.6304092E0,9.928469E-1,4.56E2,4.25E0,8E0,4.296196E0,1.1058E4,1.6173E4,3.9712732E7,1.9748653E-1,6.392807E7,4.4263797E0,3.4692544E7,-3.196464E-2,4.2945E4,1.602677E-2,1.1880085E2,4.6847186E2,2.1358025E0,1.6E1,-4.9039815E-3,-1.7895762E-2,2.7E1,1.4E1,1.752512E2,3.730007E6,9.83253E0,1.7162472E-1,1.1210787E3,2.8326238E3,-9.214218E-3,5.351875E2,2.3033286E5,3.07125E0,3.0766912E-2,1.305827E-2,1.3E1,4.3875E1,2E1,2.132956E8,2.1107816E-4,9.489666E-3,1.5782692E2,2.6112197E-3,1E0,9.19E2,3.6810555E-2,7.7329254E2,1.757551E-2,8.594445E5,1.2E1,1.3989231E5,5.732733E2,9.599108E6,2.1150263E-1,5.0279167E1,8.053797E-3,2.4708056E5,1.2287474E3,3.84E2,1.5851064E0,9.2593566E-2,4.930435E2,1.4692029E-7,1.9E1,1.3264E4,1.7537523E-5,1.37518E5,1.4531539E-2,1.5213789E-3,-1.0835567E-2,5.311576E6,4.5054653E-3,-9.117902E-3,-2.7223803E-2,-5.327196E-3,-3.8764123E-3,2.9013583E-3,4.05E2,1.1003494E-3,1.2636831E-3,6.015E3,4.6568102E8,-1.1049393E-2,1.6256282E7,5.723202E5,4.29E4,6.6385515E6,-3.517197E-3,7.372156E-3,-9.5303816E-4,7.1411137E-3,4.2344445E-1,5.940543E-3,6.48334E5,-9.023305E-3,-8.702785E-3,1.0253406E-3,1.45892715E-2,1.0509419E-3,1.34E2,7.1214952E0,1.2944685E0,8.48E2,-2.1658496E-3,1.6254681E0,1.172E3,2.5853658E0,9.9E1,1E0,2.01351E5,2.6156876E-2,4.9E1,-2.5588907E-3,3.474851E8,7.22E2,3.653602E7,2.1547383E3,1.0806703E-2,2.4277832E6,2.9620786E0,7.535486E0,4.930349E5,1.3793935E-2,1.912996E-3,2.0967E4,-1.1037575E-2,3.0272608E10,1.9738097E-2,5.89853E-3,3.142868E5,2.3977574E7,1.381618E-2,-4.5518023E-3,1.8222312E-2,1.13814086E5,-1.828006E-2,2.6E1,-5.1096855E-3,2.8787737E-3,8.235294E-1,9.886767E-3,-2.420263E-2,-8.986338E-3,-5.5570262E-3,-1.9455923E-2,-4.8312217E-2,-2.7978916E-2,-7.9893E-3,-2.453158E-2,-3.1419896E-2,-8.602314E-3,-3.687215E-3,-8.950522E-3,-1.587607E-2,-1.1142879E-3,-1.0215627E-2,-3.199738E-2,-3.8756686E-3,-1.0465101E-2,-2.3818691E-3,1.0417451E-2,-1.5274186E-2,-3.5670035E-2,6.715964E-3,-8.925362E-3,1.8329253E-2,9.016757E-3,-1.2973456E-2,-2.2080164E-2,-3.449248E-3,-1.145637E-2,-3.0270057E-2,-5.180203E-3,-1.8862773E-3,-9.315549E-3,-1.0274316E-3,5.160364E-3,9.5563446E-4,-1.2330228E-2,7.7751623E-3,1.6914237E-2,-1.8078282E-2,-3.844842E-3,5.4834434E-3,-7.0825326E-3,-5.397137E-3,5.89754E-3,1.6063228E-2,3.1317773E-3,-1.7444815E-2,-3.3461377E-2,-8.361033E-3,1.8362813E-3,1.9170474E-2,4.9222484E-3,-1.3688302E-3,-1.3445192E-2,6.4891594E-4,8.5723465E-3,2.9612502E-2,1.1784813E-2,1.3203338E-2,2.3864168E-3,1.8235907E-2,7.825362E-3,-1.0256839E-2,2.327448E-3,4.9665898E-2,2.655393E-2,2.022652E-2,-9.526582E-3,2.5290484E-2,1.3842472E-3,3.517921E-2,2.0865222E-2,4.553241E-3,-6.9066607E-3,-8.215348E-3,1.0480199E-3,6.6120704E-3,-4.388829E-3],"split_indices":[19,55,46,109,41,2,36,44,2,2,34,27,3,27,27,2,52,38,51,57,0,32,0,1,57,2,1,51,4,9,2,44,12,0,0,37,0,4,44,7,44,3,27,53,0,52,52,0,53,16,57,9,2,44,56,12,41,44,0,1,0,51,4,55,3,0,0,10,3,51,9,55,56,51,51,0,51,32,56,0,0,0,4,0,30,0,0,4,40,6,2,0,51,37,27,17,27,51,28,37,55,0,32,54,0,52,37,4,36,3,10,41,28,0,0,0,1,0,0,0,0,0,0,10,0,0,2,5,0,44,27,10,31,0,0,0,0,37,0,30,0,0,0,0,0,28,55,52,28,0,52,2,53,10,109,28,0,3,0,7,2,44,54,0,47,53,52,27,0,0,10,0,5,0,0,32,46,0,0,0,32,0,3,0,0,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.201E3,6.95E2,1.506E3,3.4E2,3.55E2,9.3E2,5.76E2,2.27E2,1.13E2,2.85E2,7E1,1.86E2,7.44E2,5E2,7.6E1,2.17E2,1E1,3.8E1,7.5E1,2.81E2,4E0,5.9E1,1.1E1,8.9E1,9.7E1,3.61E2,3.83E2,3.63E2,1.37E2,3.7E1,3.9E1,1.91E2,2.6E1,7E0,3E0,3E1,8E0,2E1,5.5E1,4E1,2.41E2,1.9E1,4E1,9E0,2E0,4.5E1,4.4E1,2.1E1,7.6E1,3.25E2,3.6E1,1.93E2,1.9E2,3.12E2,5.1E1,2.6E1,1.11E2,3.5E1,2E0,2.8E1,1.1E1,1.03E2,8.8E1,6E0,2E1,2E0,2.8E1,1.3E1,7E0,1.8E1,3.7E1,1.8E1,2.2E1,1.9E2,5.1E1,5E0,1.4E1,1.6E1,2.4E1,6E0,3E0,3.9E1,6E0,5E0,3.9E1,1.8E1,3E0,4.3E1,3.3E1,1.92E2,1.33E2,9E0,2.7E1,3.9E1,1.54E2,1.7E2,2E1,2.92E2,2E1,2E1,3.1E1,1.6E1,1E1,1.03E2,8E0,1.2E1,2.3E1,9E0,1.9E1,7.1E1,3.2E1,3.5E1,5.3E1,3E0,3E0,1.4E1,6E0,2E0,1.1E1,5E0,2E0,8E0,1E1,3.1E1,6E0,2E0,1.6E1,1.7E1,5E0,1.57E2,3.3E1,3.5E1,1.6E1,4E0,1E1,3E0,1.3E1,1.9E1,5E0,3E1,9E0,4E0,2E0,3E0,2E0,3.3E1,6E0,1.9E1,2.4E1,5E0,2.8E1,1.76E2,1.6E1,6E1,7.3E1,1.7E1,1E1,3.1E1,8E0,1.35E2,1.9E1,1.6E1,1.54E2,8E0,1.2E1,1.02E2,1.9E2,1.8E1,2E0,2E0,1.8E1,2E0,2.9E1,8E0,2E0,4.5E1,5.8E1,5E0,3E0,3E0,9E0,1.3E1,1E1,6E0,3E0,9E0,1E1,6.1E1,1E1,2.3E1,9E0,5E0,3E1,8E0,4.5E1,4E0,2E0,1.1E1,2E1,1.4E1,2E0,2E0,1.5E1,8.9E1,6.8E1,1.7E1,1.6E1,2.8E1,7E0,5E0,1.1E1,1.3E1,6E0,1.6E1,1.4E1,1.2E1,2.1E1,4E0,2E0,6E0,1.3E1,1.8E1,6E0,2E0,2.6E1,1.33E2,4.3E1,2E0,1.4E1,1.1E1,4.9E1,8E0,6.5E1,1.2E1,5E0,2.3E1,8E0,1.14E2,2.1E1,4E0,1.5E1,3E0,1.3E1,9.2E1,6.2E1,8E0,4E0,2.9E1,7.3E1,1.28E2,6.2E1,1.3E1,5E0,1E1,8E0,2.7E1,2E0,4.2E1,3E0,4.9E1,9E0,3E0,6E0,8E0,2E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"267","size_leaf_vector":"1"}},{"base_weights":[-4.060782E-3,-2.4512474E-1,1.1081894E-1,-4.0089875E-1,-1.06054686E-1,4.5090027E-2,3.9896166E-1,-3.4603894E-1,-6.465721E-1,-2.0344979E-1,1.1483816E-1,-5.3952392E-2,2.051649E-1,2.966417E-1,5.689328E-1,-3.7138563E-1,-3.0319511E-3,-7.84134E-1,-3.7332284E-1,-2.2484761E-1,4.2619463E-2,7.180743E-2,4.817872E-1,-2.5470084E-1,-1.7310675E-2,1.3738419E-1,3.6478016E-1,4.617984E-2,2.6936615E-1,6.8638855E-1,3.6678025E-1,-4.5096627E-1,-2.3322476E-1,2.1532166E-1,-1.1199219E-1,-5.090627E-1,-8.6143225E-1,-7.264866E-3,-4.4311565E-1,-1.8003649E-1,-4.7954264E-1,3.254848E-2,3.2466573E-1,3.0130832E-2,1.1566275E-2,-1.7731981E-1,-3.4333643E-1,-1.4940655E-1,1.8176908E-2,1.6960399E-1,-2.4338448E-1,4.1838002E-1,7.8590296E-2,3.7873068E-1,1.445245E-1,7.323723E-1,3.2400212E-1,5.5596554E-1,1.2841935E-1,-2.9396084E-1,-4.9671018E-1,-2.8387275E-1,-6.9452696E-2,5.2778603E-4,1.653202E-2,6.872278E-3,-7.6334532E-3,-2.539809E-2,-8.312414E-3,-8.9611024E-1,-1.18702045E-2,-1.05775045E-2,-2.6262833E-2,-4.6665454E-1,-1.5538993E-1,-1.8960919E-3,-5.026939E-1,6.057912E-2,-1.0534675E-2,4.0511766E-1,1.007637E-3,-2.1914943E-1,7.0475094E-2,-3.6980146E-1,-3.3680869E-3,-3.0261067E-1,-1.1690124E-1,2.1713811E-1,-2.6295438E-2,9.3475685E-2,2.6620802E-1,-3.0367246E-1,7.430252E-3,5.198108E-1,2.584113E-1,-2.2145763E-2,1.5595369E-1,3.9183974E-1,-5.978195E-3,-8.1470825E-2,2.1995692E-1,6.099571E-1,4.3323442E-2,1.7354935E-2,3.5117175E-3,1.9796582E-2,3.3858765E-2,3.171303E-1,-3.765501E-2,-2.2769403E-1,-3.6983024E-2,-5.569618E-1,-3.745036E-1,-4.2508274E-1,-2.4002393E-1,-2.4088194E-4,-8.92132E-3,-1.576711E-2,-9.500807E-1,-2.591248E-3,-5.5029917E-1,2.3776596E-2,-1.6468203E-1,-5.7021743E-1,-3.2012865E-1,8.132009E-2,-1.2830287E-2,5.843345E-3,2.0469334E-2,-2.3255628E-1,-1.7461994E-3,1.51473265E-2,-4.097493E-3,-4.8784342E-1,-2.8933188E-1,-2.2594035E-1,-2.763112E-2,-9.091937E-2,-1.5825901E-2,3.159282E-2,1.6305208E-1,-9.594644E-2,5.5591233E-2,1.02668725E-1,-2.0624882E-2,3.8751793E-1,1.6722813E-1,-3.7980366E-1,-7.540067E-2,4.7524476E-1,4.146976E-2,4.1817197E-1,1.3506752E-1,-2.5146212E-2,2.3007007E-1,3.234097E-1,2.456475E-2,-1.425531E-2,-2.1732045E-2,5.4368425E-2,3.7845E-1,-2.321247E-3,2.9296089E-2,1.8523099E-2,4.520269E-3,2.9235433E-3,-6.048993E-3,-1.3109315E-2,1.1325345E-3,-2.82938E-2,-1.441344E-2,-1.17616E-2,-2.0793237E-2,-6.8214396E-3,-2.8150141E-2,-1.3527091E-2,-5.5355607E-3,-2.40106E-2,-4.675106E-2,-2.6960159E-2,-8.425707E-3,-5.968184E-3,-1.3968698E-2,-1.3122968E-2,-2.8333101E-2,-1.507747E-3,-1.693986E-2,2.203981E-3,9.4547365E-3,-1.0908895E-2,-8.099836E-4,-6.3093374E-3,-2.366442E-2,-1.4005008E-2,2.3383727E-5,-1.3676234E-2,-1.8738522E-3,-1.7942131E-3,-8.041017E-3,1.3473534E-2,1.8032797E-3,-6.454485E-3,3.697019E-3,-2.935307E-4,5.2143345E-3,1.6860397E-3,8.165549E-3,1.5654719E-2,2.9346203E-2,-4.555705E-4,9.585332E-3,-7.872197E-3,-2.4607645E-2,4.1805375E-3,-1.01544885E-2,2.2181518E-2,3.7259606E-4,7.343575E-4,2.0913493E-2,-5.631252E-3,9.57454E-3,-1.1802222E-2,5.4874774E-3,4.987537E-3,1.4542802E-2,1.5144954E-2,-4.646828E-4,-4.213342E-3,7.7873827E-3,8.764183E-3,-1.4432261E-3,1.8818045E-2,-4.857183E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,-1,-1,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,117,-1,-1,-1,119,121,-1,123,125,-1,127,-1,129,131,133,-1,135,137,139,141,143,145,147,-1,149,151,-1,153,155,-1,157,159,161,-1,-1,-1,-1,-1,163,165,167,-1,169,171,173,175,-1,-1,-1,177,-1,179,-1,181,183,185,187,-1,-1,-1,189,-1,-1,-1,191,193,195,-1,197,-1,-1,199,201,203,205,-1,207,209,211,213,215,-1,217,219,221,223,225,-1,227,-1,229,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1007946E1,1.53602295E1,2.8239454E1,4.291252E0,8.134096E0,1.9289778E1,4.6486588E0,2.3984451E0,1.9625854E0,6.629781E0,1.7970543E0,5.527128E0,4.9806423E0,3.1780386E0,2.1938744E0,2.7205238E0,4.9995157E-1,3.9071655E-1,2.557268E-1,2.858263E0,0E0,1.0317625E0,3.2895947E-1,7.347307E-1,2.988821E0,4.0652795E0,2.0840034E0,0E0,2.2663164E0,8.2919693E-1,1.7323785E0,1.0174751E0,7.765784E-1,2.0935324E-1,2.3466727E-1,8.2892895E-2,3.6769104E-1,0E0,3.2356167E-1,1.4886627E0,3.6397934E-1,6.980912E-1,3.315953E-1,0E0,0E0,6.788151E-1,3.4972143E-1,6.3691735E-1,4.45639E0,2.2034693E0,6.8859255E-1,1.7478619E0,1.0864986E0,6.543598E-1,1.3987572E0,9.556351E-1,1.0185492E-1,1.7412615E-1,6.0740256E-1,1.1654181E0,7.044449E-1,3.4997797E-1,1.9534874E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.674511E-1,0E0,0E0,0E0,5.5068135E-1,1.382874E0,0E0,2.6124477E-1,6.2232685E-1,0E0,8.437705E-2,0E0,1.241858E-1,4.5541957E-1,3.147931E-1,0E0,4.047079E-1,6.5128946E-1,2.2780256E0,2.355171E0,9.098567E-1,1.5321589E0,3.7107038E-1,0E0,8.005676E-1,8.696759E-1,0E0,2.876572E-1,7.3225975E-1,0E0,5.5570865E-1,1.5803473E0,9.906254E-1,0E0,0E0,0E0,0E0,0E0,1.3907194E-1,1.1653003E-1,5.700289E-1,0E0,1.1149635E0,3.0857563E-1,7.6990724E-1,3.6478353E-1,0E0,0E0,0E0,2.839012E-1,0E0,1.1958885E-1,0E0,9.188733E-1,2.0828533E-1,1.7352259E-1,3.2278872E-1,0E0,0E0,0E0,8.761644E-2,0E0,0E0,0E0,1.4576578E-1,1.7993188E-1,2.685206E-1,0E0,4.5091927E-1,0E0,0E0,1.3808024E0,1.8580196E0,7.0346475E-1,8.523253E-1,0E0,4.178257E-1,5.937102E-1,4.7864318E-1,1.9553271E-1,4.4205666E-1,0E0,3.4429693E-1,5.814097E-1,2.7289763E-1,1.2855184E-1,2.3694706E-1,0E0,2.9490578E-1,0E0,3.7837386E-1,6.047163E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,69,69,73,73,74,74,76,76,77,77,79,79,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,96,96,97,97,99,99,100,100,101,101,107,107,108,108,109,109,111,111,112,112,113,113,114,114,118,118,120,120,122,122,123,123,124,124,125,125,129,129,133,133,134,134,135,135,137,137,140,140,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,151,151,152,152,153,153,154,154,155,155,157,157,159,159,160,160],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,-1,-1,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,118,-1,-1,-1,120,122,-1,124,126,-1,128,-1,130,132,134,-1,136,138,140,142,144,146,148,-1,150,152,-1,154,156,-1,158,160,162,-1,-1,-1,-1,-1,164,166,168,-1,170,172,174,176,-1,-1,-1,178,-1,180,-1,182,184,186,188,-1,-1,-1,190,-1,-1,-1,192,194,196,-1,198,-1,-1,200,202,204,206,-1,208,210,212,214,216,-1,218,220,222,224,226,-1,228,-1,230,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.631E3,4.0844156E7,9.451483E-1,5.980516E5,3.7684184E-1,3.773E3,5.35E3,1.1458888E4,1.121807E0,8.2E1,2.9892595E6,2.714E2,8.5E1,1E0,1E0,2.8035452E11,5.3474556E5,1E0,4.2619463E-2,1.1490676E6,1.9041E4,1.139605E6,5.52E2,5.9572783E0,1.4274633E7,4.617984E-2,2.0189162E7,4.728721E0,3.5561565E8,1.6904992E-1,2.0802219E8,5.704E3,1.5789307E8,6.82E2,1E0,-7.264866E-3,1.4843444E1,3.5902756E-6,4.6568102E8,1E0,1.2166998E0,3.0130832E-2,1.1566275E-2,2.074E3,4.8810664E4,3.3820656E7,2.214E3,1.06E3,1E0,2.9375E0,1.24E2,7.065733E7,5.9272277E4,2.13019E6,2.2644286E2,4.9807886E3,1.3304372E7,1.2342778E1,1.9121015E0,1.2414683E5,5.59E2,5.2778603E-4,1.653202E-2,6.872278E-3,-7.6334532E-3,-2.539809E-2,-8.312414E-3,1.0198864E0,-1.18702045E-2,-1.05775045E-2,-2.6262833E-2,1.3651616E5,8.417E3,-1.8960919E-3,1.067536E3,1.062363E1,-1.0534675E-2,2.386324E6,1.007637E-3,9.837297E0,3.333E3,7.3686545E-3,-3.3680869E-3,1.3676985E4,3.1E1,2.255481E6,1.3448806E8,4.6E1,3.1753032E0,5.98E2,7.430252E-3,2.062993E3,5.388794E6,-2.2145763E-2,1.4523809E0,4.1331047E-1,-5.978195E-3,7.074777E6,3.650307E9,1E0,4.3323442E-2,1.7354935E-2,3.5117175E-3,1.9796582E-2,3.3858765E-2,3.09627E5,1E0,3.9794872E0,-3.6983024E-2,1.1164689E0,4.720179E10,9.9E1,1.287E3,-2.4088194E-4,-8.92132E-3,-1.576711E-2,1.4763578E3,-2.591248E-3,2.168033E3,2.3776596E-2,6.747114E7,8.35471E-2,3.9382784E9,1.3474802E2,-1.2830287E-2,5.843345E-3,2.0469334E-2,1.1502944E0,-1.7461994E-3,1.51473265E-2,-4.097493E-3,1E0,1E0,1.5E1,-2.763112E-2,4.675E0,-1.5825901E-2,3.159282E-2,1E0,6.783E3,3.4047124E-1,1.9925156E6,-2.0624882E-2,2.1132307E0,2.4198477E2,9.2199E4,4.44084E5,1E0,4.146976E-2,1.604E3,6.745516E2,3.564E3,1.819149E1,1E0,2.456475E-2,1.431413E-1,-2.1732045E-2,9.1856E4,1.2817779E7,-2.321247E-3,2.9296089E-2,1.8523099E-2,4.520269E-3,2.9235433E-3,-6.048993E-3,-1.3109315E-2,1.1325345E-3,-2.82938E-2,-1.441344E-2,-1.17616E-2,-2.0793237E-2,-6.8214396E-3,-2.8150141E-2,-1.3527091E-2,-5.5355607E-3,-2.40106E-2,-4.675106E-2,-2.6960159E-2,-8.425707E-3,-5.968184E-3,-1.3968698E-2,-1.3122968E-2,-2.8333101E-2,-1.507747E-3,-1.693986E-2,2.203981E-3,9.4547365E-3,-1.0908895E-2,-8.099836E-4,-6.3093374E-3,-2.366442E-2,-1.4005008E-2,2.3383727E-5,-1.3676234E-2,-1.8738522E-3,-1.7942131E-3,-8.041017E-3,1.3473534E-2,1.8032797E-3,-6.454485E-3,3.697019E-3,-2.935307E-4,5.2143345E-3,1.6860397E-3,8.165549E-3,1.5654719E-2,2.9346203E-2,-4.555705E-4,9.585332E-3,-7.872197E-3,-2.4607645E-2,4.1805375E-3,-1.01544885E-2,2.2181518E-2,3.7259606E-4,7.343575E-4,2.0913493E-2,-5.631252E-3,9.57454E-3,-1.1802222E-2,5.4874774E-3,4.987537E-3,1.4542802E-2,1.5144954E-2,-4.646828E-4,-4.213342E-3,7.7873827E-3,8.764183E-3,-1.4432261E-3,1.8818045E-2,-4.857183E-3],"split_indices":[19,55,2,44,41,27,37,2,2,51,33,28,50,32,8,109,80,30,31,111,0,27,9,30,2,52,44,0,44,37,31,26,7,2,5,0,112,0,56,37,5,111,56,0,0,2,27,7,28,2,94,53,0,1,32,9,57,4,44,53,52,27,0,0,0,0,0,0,0,55,0,0,0,27,12,0,51,53,0,1,0,53,2,41,0,32,8,30,7,3,53,10,0,54,46,0,53,34,0,46,12,107,0,0,0,0,0,9,89,53,0,38,30,10,2,0,0,0,32,0,4,0,44,26,5,57,0,0,0,37,0,0,0,80,67,3,0,53,0,0,15,10,40,27,0,52,51,12,1,111,0,2,4,2,55,89,0,37,0,11,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.201E3,7.1E2,1.491E3,3.34E2,3.76E2,1.215E3,2.76E2,2.75E2,5.9E1,2.61E2,1.15E2,7.51E2,4.64E2,1.74E2,1.02E2,2.56E2,1.9E1,3.8E1,2.1E1,2.57E2,4E0,1.04E2,1.1E1,1.15E2,6.36E2,3.27E2,1.37E2,5E0,1.69E2,6.3E1,3.9E1,1.61E2,9.5E1,6E0,1.3E1,1E1,2.8E1,6E0,1.5E1,2.2E2,3.7E1,9.1E1,1.3E1,5E0,6E0,6.3E1,5.2E1,1.34E2,5.02E2,3.02E2,2.5E1,1.15E2,2.2E1,8.9E1,8E1,5.5E1,8E0,2.1E1,1.8E1,3.8E1,1.23E2,7.2E1,2.3E1,3E0,3E0,2E0,1.1E1,8E0,2E0,2.6E1,2E0,7E0,8E0,1.6E1,2.04E2,2E0,3.5E1,8.3E1,8E0,1E1,3E0,5.4E1,9E0,4.7E1,5E0,2.2E1,1.12E2,9.1E1,4.11E2,1.7E2,1.32E2,2.2E1,3E0,6.9E1,4.6E1,2E0,2E1,8.7E1,2E0,2E1,6E1,3.8E1,1.7E1,6E0,2E0,1.5E1,6E0,8E0,1E1,3.5E1,3E0,8E1,4.3E1,1.5E1,5.7E1,1.6E1,7E0,3E0,2.3E1,3E0,1.3E1,2E0,2.02E2,2.4E1,1.1E1,7.9E1,4E0,2E0,8E0,5E1,4E0,3E0,6E0,1.7E1,3E1,1.9E1,3E0,1.02E2,1E1,8E0,8.3E1,2.22E2,1.89E2,1.68E2,2E0,5.8E1,7.4E1,1.6E1,6E0,6.4E1,5E0,1.9E1,2.7E1,6E0,1.4E1,6.2E1,2.5E1,1.8E1,2E0,3E1,3E1,2E0,3.6E1,5E0,3E0,5E0,5E0,2.8E1,7E0,6.1E1,1.9E1,2E1,2.3E1,7E0,8E0,3.7E1,2E1,5E0,1.8E1,1.1E1,2E0,1.66E2,3.6E1,5E0,1.9E1,2E0,9E0,6.4E1,1.5E1,4.8E1,2E0,2E0,1.5E1,2.8E1,2E0,1.3E1,6E0,6.5E1,3.7E1,3.9E1,4.4E1,1.76E2,4.6E1,9.3E1,9.6E1,9.2E1,7.6E1,5.2E1,6E0,1.5E1,5.9E1,8E0,8E0,3E0,3E0,6.2E1,2E0,2E0,1.7E1,6E0,2.1E1,2E0,4E0,7E0,7E0,6E1,2E0,1.3E1,5E0,1.1E1,1.9E1,2.8E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[-4.042339E-3,-2.3413792E-1,1.041683E-1,-3.7383464E-1,-6.629469E-2,-6.0911886E-3,2.5061885E-1,-4.0813756E-1,2.7088544E-1,-1.3908039E-1,2.2815849E-1,-7.845015E-2,1.7336026E-1,2.0515123E-1,5.3883547E-1,-2.1235849E-1,-4.7526258E-1,4.6893913E-2,1.49251325E-2,-1.9155195E-1,4.5797136E-3,1.22062646E-1,3.9989895E-1,-2.5704658E-1,-3.5992194E-2,1.2927487E-1,3.9063418E-1,2.4082102E-1,-7.338045E-2,5.675858E-1,2.5402433E-3,-2.417123E-1,2.3355423E-2,-9.131047E-1,-4.478583E-1,1.0394161E-1,-1.8356781E-2,-2.774843E-2,-1.7438807E-1,-1.5105043E-1,7.905708E-2,3.0597928E-1,4.4427242E-2,2.517359E-2,3.0290294E-1,-2.9099143E-1,1.3526268E-1,-9.322149E-3,-3.557874E-1,-1.1091729E-1,1.5286198E-1,-7.344361E-3,4.3854958E-1,1.7487377E-1,3.9530277E-1,-2.7323264E-1,2.1061082E-1,5.904739E-1,-1.1929594E-3,-3.2384098E-1,-3.2000992E-2,-1.0314382E0,-1.4932616E-2,-5.068248E-1,-3.0858162E-1,1.53188845E-2,9.249396E-3,-2.0206717E-1,-3.7462663E-2,-1.2738121E-3,-9.721641E-3,4.670179E-2,1.3480793E-2,1.577254E-2,2.8364302E-3,-1.382646E-2,7.3487447E-3,3.3502138E-1,1.8717659E-3,-2.2119765E-1,-4.1894174E-1,1.4117146E-2,2.360915E-3,1.05099194E-1,-5.7620224E-2,-2.3198996E-1,-4.4249643E-2,3.8133585E-3,-3.371047E-1,5.9761815E-2,2.1412526E-1,3.2885632E-1,2.4037505E-2,9.2681915E-2,2.7902234E-1,3.505161E-1,6.6217244E-1,-4.6921724E-1,-1.7600867E-1,2.6695082E-1,-8.153014E-3,4.2039812E-1,6.7069787E-1,-2.58718E-1,-5.6944096E-1,3.577662E-2,-1.245873E-2,-1.1575833E-2,-5.2563988E-2,-4.599657E-1,-8.56356E-1,-1.7132679E-1,-4.2541742E-1,-6.5373955E-3,2.6909455E-3,-6.0108732E-2,-2.1491143E-1,-1.3558762E-1,3.6811654E-2,9.4326116E-2,-2.4683453E-2,5.6326874E-2,-6.105938E-3,4.4557047E-3,1.6472999E-2,-1.4109251E-2,-1.3667725E-1,-6.2091845E-1,-2.9715616E-1,5.2641213E-2,3.800705E-2,-1.03125505E-1,5.0581023E-2,-3.143884E-1,-8.103941E-2,4.78699E-4,-2.3212224E-2,1.9684754E-1,1.3824042E-2,1.7044568E-1,3.6105955E-1,3.7006482E-1,5.9407502E-3,2.4084683E-1,6.860465E-2,1.8966094E-1,3.7435454E-1,3.6805958E-1,-6.754469E-2,1.819134E-2,3.994785E-2,-8.669997E-3,-2.6446227E-2,-2.3064171E-1,5.003E-3,5.1368624E-1,1.5616754E-1,4.6080413E-1,2.8414386E-3,1.4639789E-2,7.007687E-1,-1.3928638E-2,-4.388421E-3,-3.0711813E-2,8.930028E-4,-1.71385E-3,4.0594595E-3,-2.1770071E-2,-1.2155234E-2,-4.7237813E-2,-2.557964E-2,-1.5233667E-3,-9.9284025E-3,-2.1904262E-2,-2.9114662E-3,-9.090589E-3,1.8271666E-3,-1.7420182E-2,-8.79819E-3,-7.655005E-3,1.973281E-3,4.8031583E-3,-1.6366012E-3,5.8784713E-3,-2.0690816E-3,-5.3274897E-3,2.7832096E-3,4.2708186E-3,-2.8192617E-3,-6.932079E-3,2.5878247E-4,-1.6874341E-2,-3.8408976E-2,-1.50855575E-2,3.544006E-3,-9.87124E-3,5.4535097E-3,-7.0993723E-3,-6.015025E-4,-3.1939282E-3,7.1913907E-3,-1.5542201E-2,-1.2012947E-3,-7.0528965E-3,3.155881E-3,2.5874265E-3,1.1632098E-2,1.6509984E-3,-1.3626127E-2,4.1766046E-3,1.14067495E-2,1.8821707E-2,-1.3531971E-3,1.2093531E-2,2.2506647E-2,5.6541953E-3,1.5375115E-2,6.0100807E-5,5.568009E-3,1.16371885E-2,-3.582034E-3,1.8134268E-2,-2.8046705E-3,-6.6642737E-4,1.755792E-2,1.2160192E-2,-1.3707166E-2,-1.4863703E-2,-5.8974354E-3,8.346829E-3,2.7551148E-2,6.8371906E-4,9.474113E-3,1.081884E-2,2.3464667E-2,1.2095521E-2,3.2330267E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,49,51,53,55,57,-1,59,-1,61,63,65,-1,-1,67,69,71,73,75,-1,77,79,81,83,85,87,89,-1,91,93,95,97,99,101,-1,103,105,107,-1,109,111,-1,113,115,117,-1,-1,119,-1,-1,-1,121,-1,123,-1,125,127,-1,-1,129,131,133,-1,-1,135,137,139,141,-1,143,145,147,149,151,153,155,-1,157,159,161,163,165,-1,-1,-1,167,169,171,173,-1,-1,175,177,179,181,183,185,187,-1,-1,-1,-1,189,191,193,195,-1,197,199,201,203,-1,-1,205,207,209,211,213,-1,215,217,219,221,223,225,-1,-1,-1,-1,227,-1,229,231,233,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4801735E1,1.6482006E1,2.4174429E1,8.603024E0,6.911398E0,1.1115706E1,8.31567E0,4.6957817E0,3.9002008E0,1.9456997E0,1.1173282E0,4.61628E0,2.305838E0,5.5515575E0,1.1755867E0,2.1554356E0,2.84272E0,0E0,6.408836E-1,1.2599616E0,8.233062E-1,5.7182235E-1,1.950314E-1,1.597436E0,4.2099853E0,1.1840355E0,1.1572418E0,4.9635677E0,3.6899476E0,1.1302719E0,0E0,1.574182E0,0E0,6.9511986E-1,1.9542542E0,3.0763647E-1,0E0,0E0,6.859951E-1,1.7398101E-1,3.2575223E-1,1.0198176E-1,2.0827022E-1,0E0,1.2896442E-1,8.7178516E-1,1.3048261E-1,2.531589E0,2.6973963E0,8.670449E-1,1.059484E0,0E0,2.1049118E-1,2.9530773E0,1.5248737E0,6.414807E-1,6.423867E-1,7.9592514E-1,0E0,9.082923E-1,4.548716E-1,9.094343E-1,0E0,2.5378952E0,1.1939802E0,0E0,9.3404666E-2,2.642312E-1,2.3821244E-1,0E0,0E0,1.4851427E-1,0E0,0E0,0E0,1.8591519E-1,0E0,6.651151E-2,0E0,5.0780034E-1,7.2561073E-1,0E0,0E0,5.1951656E0,1.5906566E0,3.8497913E-1,0E0,0E0,5.548955E-1,4.7588217E-1,6.636963E-1,1.14328384E-1,0E0,6.826881E-1,1.2324095E0,9.636431E-1,7.9204464E-1,2.711215E-1,4.4812125E-1,5.7579863E-1,0E0,3.7569857E-1,3.1652832E-1,3.9769578E-1,8.4756947E-1,9.0686895E-2,0E0,0E0,0E0,5.108032E-1,5.003023E-1,2.4358761E-1,8.5459614E-1,0E0,0E0,2.0597757E-1,4.1111422E-1,9.138724E-2,9.9873506E-2,1.3523811E-1,1.5144561E-1,6.8978235E-2,0E0,0E0,0E0,0E0,9.2154324E-2,4.1769505E-1,3.6444402E-1,2.3757925E0,0E0,1.092721E0,1.2700893E0,1.6561389E-1,1.57333E-1,0E0,0E0,1.4478195E-1,4.2438984E-1,5.439632E-1,5.4675245E-1,7.6488495E-2,0E0,2.6677728E-1,6.2178916E-1,1.4872456E0,8.5899353E-1,9.358349E-1,5.557061E-1,0E0,0E0,0E0,0E0,1.8271697E-1,0E0,1.064775E-1,1.3165611E-1,1.949215E-1,0E0,0E0,8.096504E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,34,34,35,35,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,63,63,64,64,66,66,67,67,68,68,71,71,75,75,77,77,79,79,80,80,83,83,84,84,85,85,88,88,89,89,90,90,91,91,93,93,94,94,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,109,109,110,110,111,111,112,112,115,115,116,116,117,117,118,118,119,119,120,120,121,121,126,126,127,127,128,128,129,129,131,131,132,132,133,133,134,134,137,137,138,138,139,139,140,140,141,141,143,143,144,144,145,145,146,146,147,147,148,148,153,153,155,155,156,156,157,157,160,160],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,50,52,54,56,58,-1,60,-1,62,64,66,-1,-1,68,70,72,74,76,-1,78,80,82,84,86,88,90,-1,92,94,96,98,100,102,-1,104,106,108,-1,110,112,-1,114,116,118,-1,-1,120,-1,-1,-1,122,-1,124,-1,126,128,-1,-1,130,132,134,-1,-1,136,138,140,142,-1,144,146,148,150,152,154,156,-1,158,160,162,164,166,-1,-1,-1,168,170,172,174,-1,-1,176,178,180,182,184,186,188,-1,-1,-1,-1,190,192,194,196,-1,198,200,202,204,-1,-1,206,208,210,212,214,-1,216,218,220,222,224,226,-1,-1,-1,-1,228,-1,230,232,234,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.5125709E8,6.7652373E3,9.026015E2,9.9625344E5,2.9251662E6,2.0304577E-1,2.46E2,1.1997242E0,1.3664E4,1.141E4,5.0083565E6,3.9712732E7,1.708775E7,1.5088776E0,6E0,4.6893913E-2,8.4939235E-1,7.7660305E-5,1.752512E2,6.310469E6,1.504779E7,1.91E2,1E0,1.1058E4,4.7233105E6,1.2679E4,3.970405E3,6.292039E0,2.5402433E-3,3.008162E6,2.3355423E-2,1.9125667E3,7.298614E2,1E0,-1.8356781E-2,-2.774843E-2,7.105352E5,2.909019E3,8.7E1,1.3384724E8,3.212485E11,2.517359E-2,2.6654E4,6.23E2,2.7E2,8E0,8.645522E0,7.45E2,2.4738782E2,-7.344361E-3,3.774872E7,4.0434834E5,1E0,3.0379269E1,2.2576077E10,1.00860974E3,-1.1929594E-3,8.7217E4,1E0,1E0,-1.4932616E-2,7.218466E7,7.218466E7,1.53188845E-2,5.505334E9,4.8E1,2.2806E5,-1.2738121E-3,-9.721641E-3,1.0824E4,1.3480793E-2,1.577254E-2,2.8364302E-3,4.0844156E7,7.3487447E-3,7.0229135E6,1.8717659E-3,4.1E1,8E0,1.4117146E-2,2.360915E-3,8.9910974E2,2.821E3,2.2417E0,-4.4249643E-2,3.8133585E-3,3E0,9.232009E2,1E0,3.429012E7,2.4037505E-2,1.3642105E1,3.596E3,5.68E2,2.3553647E5,1.0485785E3,6E1,1.2695038E-3,-8.153014E-3,7.6842415E6,2.6882867E8,3.164136E6,4.8632206E5,1.5851064E0,-1.245873E-2,-1.1575833E-2,-5.2563988E-2,6.3251557E0,1.4713E4,2.1690162E7,2.203288E2,-6.5373955E-3,2.6909455E-3,4.2857143E-1,1.003252E5,3.557179E1,6.822312E4,5.608469E1,2.821E3,3.2829E4,-6.105938E-3,4.4557047E-3,1.6472999E-2,-1.4109251E-2,9.213145E0,4.9E1,4.116997E6,2.73E2,3.800705E-2,1.273801E6,2.091623E5,1.3E1,1.7427321E-1,4.78699E-4,-2.3212224E-2,3.3820656E7,2.4E1,1.645614E6,1E0,2.1434232E2,5.9407502E-3,6.4625664E8,8.5518695E4,1.4753189E8,1.5439131E2,2.943086E4,1.1803382E4,1.819134E-2,3.994785E-2,-8.669997E-3,-2.6446227E-2,1.6347875E3,5.003E-3,1.6E1,8.1980075E5,1.2045951E6,2.8414386E-3,1.4639789E-2,4.387838E-1,-1.3928638E-2,-4.388421E-3,-3.0711813E-2,8.930028E-4,-1.71385E-3,4.0594595E-3,-2.1770071E-2,-1.2155234E-2,-4.7237813E-2,-2.557964E-2,-1.5233667E-3,-9.9284025E-3,-2.1904262E-2,-2.9114662E-3,-9.090589E-3,1.8271666E-3,-1.7420182E-2,-8.79819E-3,-7.655005E-3,1.973281E-3,4.8031583E-3,-1.6366012E-3,5.8784713E-3,-2.0690816E-3,-5.3274897E-3,2.7832096E-3,4.2708186E-3,-2.8192617E-3,-6.932079E-3,2.5878247E-4,-1.6874341E-2,-3.8408976E-2,-1.50855575E-2,3.544006E-3,-9.87124E-3,5.4535097E-3,-7.0993723E-3,-6.015025E-4,-3.1939282E-3,7.1913907E-3,-1.5542201E-2,-1.2012947E-3,-7.0528965E-3,3.155881E-3,2.5874265E-3,1.1632098E-2,1.6509984E-3,-1.3626127E-2,4.1766046E-3,1.14067495E-2,1.8821707E-2,-1.3531971E-3,1.2093531E-2,2.2506647E-2,5.6541953E-3,1.5375115E-2,6.0100807E-5,5.568009E-3,1.16371885E-2,-3.582034E-3,1.8134268E-2,-2.8046705E-3,-6.6642737E-4,1.755792E-2,1.2160192E-2,-1.3707166E-2,-1.4863703E-2,-5.8974354E-3,8.346829E-3,2.7551148E-2,6.8371906E-4,9.474113E-3,1.081884E-2,2.3464667E-2,1.2095521E-2,3.2330267E-2],"split_indices":[19,109,7,51,51,27,27,26,10,40,2,9,27,44,44,37,3,0,33,41,51,44,44,3,6,9,44,2,51,37,0,31,0,4,51,80,0,0,27,31,8,31,30,0,9,2,3,16,53,2,51,0,7,27,79,55,12,4,0,9,111,80,0,44,44,0,5,10,1,0,0,10,0,0,0,44,0,31,0,2,16,0,0,4,10,52,0,0,8,32,107,7,0,57,2,3,32,51,3,26,0,46,7,1,27,52,0,0,0,53,9,44,56,0,0,55,27,57,32,57,2,9,0,0,0,0,52,3,12,2,0,9,27,8,37,0,0,7,8,9,6,4,0,7,27,12,57,32,4,0,0,0,0,4,0,3,31,46,0,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.199E3,7.03E2,1.496E3,3.83E2,3.2E2,8.54E2,6.42E2,3.64E2,1.9E1,2.57E2,6.3E1,6.09E2,2.45E2,5.56E2,8.6E1,9.4E1,2.7E2,4E0,1.5E1,1.88E2,6.9E1,4E1,2.3E1,1.16E2,4.93E2,2.05E2,4E1,4.93E2,6.3E1,8.1E1,5E0,9.1E1,3E0,1.4E1,2.56E2,1.3E1,2E0,6E0,1.82E2,2.2E1,4.7E1,1.1E1,2.9E1,7E0,1.6E1,1.07E2,9E0,4.56E2,3.7E1,1.8E1,1.87E2,3E0,3.7E1,3.47E2,1.46E2,3.7E1,2.6E1,7.8E1,3E0,6.5E1,2.6E1,1.1E1,3E0,1.78E2,7.8E1,3E0,1E1,1.51E2,3.1E1,8E0,1.4E1,4.2E1,5E0,9E0,2E0,2E1,9E0,1.4E1,2E0,7.1E1,3.6E1,2E0,7E0,1.35E2,3.21E2,3.2E1,5E0,1E1,8E0,7.5E1,1.12E2,1.9E1,1.8E1,1.95E2,1.52E2,1.27E2,1.9E1,1.1E1,2.6E1,2.3E1,3E0,2.7E1,5.1E1,5.3E1,1.2E1,2.1E1,5E0,2E0,9E0,1.59E2,1.9E1,3.7E1,4.1E1,2E0,8E0,1.3E1,1.38E2,1.3E1,1.8E1,2.5E1,1.7E1,1.3E1,7E0,2E0,1.2E1,3.3E1,3.8E1,1.2E1,2.4E1,1.27E2,8E0,2.26E2,9.5E1,2E1,1.2E1,3E0,5E0,1.8E1,5.7E1,8.8E1,2.4E1,1.5E1,4E0,2.6E1,1.69E2,8E1,7.2E1,1.22E2,5E0,1E1,9E0,4E0,7E0,2.2E1,4E0,6E0,1.7E1,2.4E1,3E0,5E0,4.6E1,4E1,1.3E1,1E1,2E0,9E0,1.2E1,1.42E2,1.7E1,1E1,9E0,1E1,2.7E1,3.5E1,6E0,5E0,8E0,1.3E1,1.25E2,1.1E1,2E0,9E0,9E0,2E1,5E0,8E0,9E0,1E1,3E0,3.4E1,4E0,7E0,5E0,2.2E1,2E0,2.5E1,1.02E2,1.41E2,8.5E1,4.5E1,5E1,1.8E1,2E0,8E0,4E0,6E0,1.2E1,5.4E1,3E0,4.6E1,4.2E1,2.1E1,3E0,1E1,5E0,1.3E1,1.3E1,7.6E1,9.3E1,6.4E1,1.6E1,6.8E1,4E0,6E0,1.16E2,2E0,3E0,1E1,1.2E1,2E0,4E0,5E0,1.2E1,6E0,1.8E1,2E0,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[-7.230466E-3,-2.3534985E-1,1.03154846E-1,-3.621592E-1,-7.899597E-2,4.898083E-3,2.8341016E-1,-3.803487E-1,5.0293256E-2,-2.1406388E-1,6.535554E-2,-4.8691314E-2,1.5385702E-1,1.9026382E-1,4.2751235E-1,-4.5660627E-1,-1.779542E-1,-2.3705164E-1,-6.283234E-2,2.3170875E-2,4.8554713E-1,-1.8293558E-1,-4.5334813E-3,1.0691586E-1,3.7734315E-1,2.2954199E-1,-1.6304816E-1,1.7323226E-1,5.0834453E-1,-3.6428508E-1,-6.595563E-1,-1.3148808E-1,-4.0039846E-1,-2.468254E-2,-2.1944895E-1,-8.83165E-3,-1.1194273E-3,8.3965056E-2,-1.3798223E-1,5.342133E-1,5.841848E-3,-1.6431072E-1,-5.442063E-1,7.0246464E-1,-2.5318654E-2,1.2669034E-1,-2.9728782E-1,2.3133758E-1,2.3510095E-2,1.7358151E-1,3.6004856E-1,-1.400007E-2,2.4607023E-3,2.0696825E-1,-1.9209174E-2,6.005566E-1,4.0303233E-1,-3.8044354E-1,2.2562379E-2,-7.381595E-1,-2.3829085E-1,-1.7317389E-1,1.2874119E-1,-4.8012152E-1,-7.286031E-3,-2.8123996E-1,-1.5161583E-1,1.61888E-2,2.0965451E-1,-3.6816818E-1,-4.5912325E-2,2.8184248E-2,1.2951475E-2,-1.8894711E-1,-3.9926753E-3,-4.935999E-3,-4.310674E-2,1.09836655E-2,3.416148E-2,-3.656794E-2,6.049547E-1,-4.049786E-2,1.8010445E-1,-1.766742E-2,2.2484756E-3,2.960955E-1,-6.8040597E-3,2.0454639E-1,-8.625004E-2,6.3065237E-1,2.8197515E-1,1.5527429E-1,-1.0475488E-1,2.988218E-2,3.5566097E-1,7.5752366E-1,4.9952525E-1,3.2696882E-1,5.6795657E-1,-2.0403215E-1,-4.1204795E-1,-4.7862446E-1,-9.020639E-1,3.8301575E-3,-2.9447398E-1,-2.5754754E-4,-2.1975684E-1,1.4669651E-2,5.94855E-3,-5.4092926E-1,-9.420229E-3,-2.6127368E-1,-2.211844E-2,5.53522E-3,-1.89551E-1,-1.4787649E-1,5.044485E-2,3.0051374E-1,1.0210008E-1,-2.3470433E-2,-3.8119317E-3,-1.0052932E-1,5.2599967E-2,-1.2616561E-2,-1.5840307E-1,1.0954756E-1,-8.480766E-2,-1.0434667E-1,6.295367E-2,3.9306916E-2,1.4878421E-2,-1.4198652E-1,6.4368825E-3,1.09397754E-1,2.4844135E-1,1.6615648E-2,3.1455895E-3,-1.1977399E-1,2.264946E-1,-1.4005019E-1,8.055872E-2,4.2996818E-1,3.547496E-2,3.4105164E-1,5.4439936E-2,1.2023978E-2,2.7831199E-4,-9.381158E-3,-8.6812855E-5,4.4482634E-3,-3.0267982E-2,6.758099E-3,4.1011435E-1,4.112161E-2,2.7059155E-2,1.6241843E-2,2.7679386E-2,4.274268E-1,1.3984185E-1,3.796514E-2,3.161074E-1,-1.1844961E-2,6.691275E-3,-1.7120115E-2,-2.5966363E-2,-2.751755E-2,-1.1904667E-2,-4.2190734E-2,-8.766571E-3,-1.5065842E-2,-2.5305722E-3,1.988027E-3,-1.0600982E-2,-7.307488E-3,3.4780419E-3,-1.1452075E-2,-2.8441677E-2,-1.4232021E-2,-8.99449E-3,4.8776898E-3,-3.8294264E-3,-4.68694E-3,-1.2030914E-2,-2.3963193E-4,-1.1271663E-2,3.0533336E-3,-1.25848735E-2,1.5748003E-2,4.4566705E-3,9.821616E-3,1.2044348E-3,-9.99895E-4,-8.386331E-3,6.627157E-3,-4.0389216E-3,-4.412853E-3,-1.071675E-2,-3.421709E-3,7.4827042E-3,-1.3309658E-3,-9.7783655E-3,-7.283677E-3,3.1564792E-3,7.800326E-3,3.1772358E-4,-3.3680317E-3,-2.7008096E-2,8.762404E-3,2.4158696E-3,5.5875233E-3,1.2548224E-2,-2.1485791E-4,-2.0974373E-2,6.545507E-3,1.3472839E-2,-9.913988E-3,-2.8478717E-3,7.0835324E-3,-2.1502874E-3,2.2806048E-2,8.755309E-3,2.7717527E-2,1.1941097E-2,7.2667086E-3,-5.949403E-3,5.8579776E-3,-3.2171358E-3,2.2248494E-2,1.0264938E-2,2.5165712E-2,1.1237225E-2,1.006999E-3,1.1554686E-2,-5.7536787E-3,1.8482693E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,67,69,71,-1,73,75,77,79,81,83,85,-1,87,89,-1,91,93,-1,95,97,99,-1,101,103,105,107,109,-1,111,113,115,117,119,121,-1,-1,123,125,-1,-1,-1,-1,127,129,131,133,-1,-1,135,-1,137,139,141,143,145,147,149,151,153,155,157,159,161,163,165,167,-1,169,-1,171,-1,173,175,-1,177,-1,179,181,183,185,187,189,-1,-1,191,193,-1,195,197,199,201,203,-1,-1,205,-1,207,209,-1,-1,211,213,215,217,219,-1,221,223,-1,-1,-1,-1,-1,225,-1,227,-1,-1,-1,-1,229,231,-1,233,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4440483E1,1.3949516E1,2.5803982E1,1.084523E1,6.194621E0,7.5430837E0,6.817871E0,5.860874E0,0E0,5.552473E-1,2.7184982E0,4.119086E0,2.5823312E0,4.394944E0,4.043293E0,4.9872437E0,1.0533173E0,6.5055656E-1,1.1107817E-1,1.3929868E0,1.7397094E-1,1.067903E0,7.7180066E0,1.7001901E0,7.710161E-1,2.0043106E0,7.749822E-1,1.09571E0,1.2340393E0,2.893736E0,2.6718025E0,9.986024E-1,2.66371E-1,0E0,5.27401E-1,0E0,0E0,8.740384E-1,8.0963135E-1,8.488822E-2,0E0,6.5093946E-1,1.3163638E0,1.7622519E-1,3.6380858E0,1.7844057E0,3.4185767E-1,6.1290216E-1,0E0,1.6267729E0,1.5968361E0,0E0,2.784097E-1,1.2532797E0,0E0,8.744278E-1,7.426052E-1,9.8082924E-1,0E0,2.631054E0,2.9702717E-1,6.0730267E-1,3.0979648E-1,1.0250354E-1,0E0,1.6209507E-1,4.0047312E-1,3.8963515E-1,3.2112086E-1,4.256569E-1,1.6132075E-1,0E0,0E0,3.477564E-1,2.2033277E-1,0E0,0E0,0E0,0E0,3.3927844E0,3.693545E-1,9.2861557E-1,7.0243454E-1,0E0,0E0,2.958747E-1,0E0,1.3012953E0,2.0871767E-1,2.154522E-1,8.8204527E-1,1.1351712E-1,1.0496354E-1,9.756062E-2,2.3098588E-1,1.5600395E-1,5.8372784E-1,9.365425E-1,1.2342782E0,6.66852E-1,6.6459465E-1,6.75992E-1,7.2839737E-1,0E0,1.0017836E-1,0E0,2.4180865E-1,0E0,1.1914874E-1,9.2451096E-2,0E0,1.568718E-1,0E0,1.3805708E-1,3.2640386E-1,1.7164624E-1,3.369158E-1,1.4745879E-1,1.5028615E-1,0E0,0E0,1.2215759E-1,1.5934618E-1,0E0,4.9701238E-1,1.1565903E-1,9.5407136E-2,2.9535434E0,1.2464383E0,0E0,0E0,9.388459E-1,0E0,3.480239E-1,2.3869705E-1,0E0,0E0,4.719519E-1,9.3534756E-1,8.97392E-2,6.57472E-2,7.7103734E-2,0E0,9.4946194E-1,3.167738E-1,0E0,0E0,0E0,0E0,0E0,9.385873E-2,0E0,1.867826E-1,0E0,0E0,0E0,0E0,6.4461136E-1,2.6568487E-1,0E0,5.8271194E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,52,52,53,53,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,73,73,74,74,79,79,80,80,81,81,82,82,85,85,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,104,104,106,106,108,108,109,109,111,111,113,113,114,114,115,115,116,116,117,117,118,118,121,121,122,122,124,124,125,125,126,126,127,127,128,128,131,131,133,133,134,134,137,137,138,138,139,139,140,140,141,141,143,143,144,144,150,150,152,152,157,157,158,158,160,160],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,68,70,72,-1,74,76,78,80,82,84,86,-1,88,90,-1,92,94,-1,96,98,100,-1,102,104,106,108,110,-1,112,114,116,118,120,122,-1,-1,124,126,-1,-1,-1,-1,128,130,132,134,-1,-1,136,-1,138,140,142,144,146,148,150,152,154,156,158,160,162,164,166,168,-1,170,-1,172,-1,174,176,-1,178,-1,180,182,184,186,188,190,-1,-1,192,194,-1,196,198,200,202,204,-1,-1,206,-1,208,210,-1,-1,212,214,216,218,220,-1,222,224,-1,-1,-1,-1,-1,226,-1,228,-1,-1,-1,-1,230,232,-1,234,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,9.772148E4,2.14099E5,3.3926086E2,8.594445E5,1.5283889E0,2.720437E6,5.0293256E-2,1.1400756E6,1.4101E4,4.67E2,5.0083565E6,6.971004E7,1.1410706E3,8.601996E-1,9.6910744E7,1E0,3.2449896E0,1.775044E7,6.9514094E11,1.5126086E2,8E0,4.898012E-1,3.774872E7,1.3664E4,1.184974E-1,1.5846036E7,3.09627E5,5.686354E-1,3.6282136E0,2.3292E4,1.1408248E0,-2.468254E-2,1.405E3,-8.83165E-3,-1.1194273E-3,7.2041174E2,9.538462E0,2.9956698E-1,5.841848E-3,1.901875E2,5.5E1,2.127E3,2.03148E2,2.6883545E2,2.4519731E3,1.8E1,2.3510095E-2,7.695788E-2,2.2100918E1,-1.400007E-2,1E0,5.0079144E5,-1.9209174E-2,3.4246575E-3,8.253647E6,2.2178302E5,2.2562379E-2,1.0198864E0,5.684E3,1.0119178E10,3.3517068E7,1.2068E4,-7.286031E-3,2.539E3,3.476768E-3,2.1592189E5,6.5346925E6,1.517E3,6.4506575E5,2.8184248E-2,1.2951475E-2,4.1E1,1.4492002E0,-4.935999E-3,-4.310674E-2,1.09836655E-2,3.416148E-2,1.645614E6,1.5461997E7,2.5247778E6,1.146E3,-1.766742E-2,2.2484756E-3,7.4338E4,-6.8040597E-3,9.3E1,1.6503105E3,1.8496495E5,5.21929E5,5.441732E9,5.3421542E-2,2.3770695E5,6.072483E2,2.5737E4,1.6195753E9,5.244755E-3,1.417988E6,2.0601392E3,8.7217E4,2.3679462E5,1.5106794E3,3.8301575E-3,3.0996E4,-2.5754754E-4,9.4446504E7,1.4669651E-2,7.6599895E9,5.684E3,-9.420229E-3,1.13E3,-2.211844E-2,6.822312E4,5.3766327E2,6.47E2,3.5330936E1,2.5482938E0,1.2592834E-1,-2.3470433E-2,-3.8119317E-3,4.9304895E6,3.4E1,-1.2616561E-2,9.25E0,1.08504E5,1.8955729E1,9.64E2,1.89E2,3.9306916E-2,1.4878421E-2,2.2515285E6,6.4368825E-3,3.295224E2,5.4674416E7,1.6615648E-2,3.1455895E-3,1.5361997E7,1.5294166E5,9.213145E0,1.6332837E9,1.2059859E1,3.547496E-2,4.556146E6,2.6025698E7,1.2023978E-2,2.7831199E-4,-9.381158E-3,-8.6812855E-5,4.4482634E-3,1.4925373E-1,6.758099E-3,1.3170732E0,4.112161E-2,2.7059155E-2,1.6241843E-2,2.7679386E-2,2.8982632E0,1.01E3,3.796514E-2,1.862272E6,-1.1844961E-2,6.691275E-3,-1.7120115E-2,-2.5966363E-2,-2.751755E-2,-1.1904667E-2,-4.2190734E-2,-8.766571E-3,-1.5065842E-2,-2.5305722E-3,1.988027E-3,-1.0600982E-2,-7.307488E-3,3.4780419E-3,-1.1452075E-2,-2.8441677E-2,-1.4232021E-2,-8.99449E-3,4.8776898E-3,-3.8294264E-3,-4.68694E-3,-1.2030914E-2,-2.3963193E-4,-1.1271663E-2,3.0533336E-3,-1.25848735E-2,1.5748003E-2,4.4566705E-3,9.821616E-3,1.2044348E-3,-9.99895E-4,-8.386331E-3,6.627157E-3,-4.0389216E-3,-4.412853E-3,-1.071675E-2,-3.421709E-3,7.4827042E-3,-1.3309658E-3,-9.7783655E-3,-7.283677E-3,3.1564792E-3,7.800326E-3,3.1772358E-4,-3.3680317E-3,-2.7008096E-2,8.762404E-3,2.4158696E-3,5.5875233E-3,1.2548224E-2,-2.1485791E-4,-2.0974373E-2,6.545507E-3,1.3472839E-2,-9.913988E-3,-2.8478717E-3,7.0835324E-3,-2.1502874E-3,2.2806048E-2,8.755309E-3,2.7717527E-2,1.1941097E-2,7.2667086E-3,-5.949403E-3,5.8579776E-3,-3.2171358E-3,2.2248494E-2,1.0264938E-2,2.5165712E-2,1.1237225E-2,1.006999E-3,1.1554686E-2,-5.7536787E-3,1.8482693E-2],"split_indices":[19,109,32,2,51,27,40,31,0,32,2,2,27,44,51,26,44,66,53,44,30,45,17,26,7,2,41,46,9,37,52,2,41,0,10,0,0,51,57,37,0,4,0,1,55,4,4,3,0,26,55,0,66,27,0,56,1,27,0,55,9,18,50,9,0,2,37,27,44,0,27,0,0,2,52,0,0,0,0,9,31,27,2,0,0,12,0,10,51,32,10,5,26,27,4,9,7,56,28,51,9,46,51,0,9,0,7,0,5,9,0,2,0,32,4,0,55,41,37,0,0,31,3,0,57,7,57,0,10,0,0,27,0,4,7,0,0,12,32,52,12,55,0,47,42,0,0,0,0,0,55,0,52,0,0,0,0,40,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E3,7.04E2,1.456E3,3.88E2,3.16E2,9.43E2,5.13E2,3.84E2,4E0,1.63E2,1.53E2,6.94E2,2.49E2,3.13E2,2E2,2.78E2,1.06E2,1.41E2,2.2E1,1.4E2,1.3E1,1.71E2,5.23E2,2.07E2,4.2E1,2.82E2,3.1E1,4.9E1,1.51E2,1.93E2,8.5E1,8.9E1,1.7E1,6E0,1.35E2,4E0,1.8E1,1.02E2,3.8E1,1.1E1,2E0,1.64E2,7E0,1.4E1,5.09E2,1.98E2,9E0,2.2E1,2E1,1.99E2,8.3E1,1.6E1,1.5E1,4.7E1,2E0,7.8E1,7.3E1,1.9E2,3E0,7.1E1,1.4E1,7.7E1,1.2E1,1.2E1,5E0,6.9E1,6.6E1,6.7E1,3.5E1,1E1,2.8E1,7E0,4E0,1.42E2,2.2E1,4E0,3E0,2E0,1.2E1,5.01E2,8E0,4.8E1,1.5E2,7E0,2E0,1.9E1,3E0,1.78E2,2.1E1,1.7E1,6.6E1,6E0,9E0,2.2E1,2.5E1,2.8E1,5E1,5.2E1,2.1E1,3E1,1.6E2,2.9E1,4.2E1,2E0,1.2E1,1.7E1,6E1,4E0,8E0,9E0,3E0,6.5E1,4E0,1.3E1,5.3E1,1.1E1,5.6E1,1.8E1,1.7E1,6E0,4E0,1.8E1,1E1,3.4E1,1.08E2,9E0,1.3E1,2.98E2,2.03E2,3E0,5E0,3.1E1,1.7E1,7.5E1,7.5E1,1.4E1,5E0,1.1E1,1.67E2,1.6E1,5E0,9E0,8E0,5.2E1,1.4E1,3E0,3E0,4E0,5E0,1E1,1.2E1,6E0,1.9E1,1.2E1,1.6E1,2.4E1,2.6E1,3.3E1,1.9E1,9E0,1.2E1,2.6E1,4E0,1.35E2,2.5E1,1.7E1,1.2E1,4E1,2E0,1E1,2E0,3E0,5.7E1,2E0,6E0,3E0,6E0,3.3E1,3.2E1,6E0,7E0,2.6E1,2.7E1,5E0,6E0,5.4E1,2E0,1.4E1,4E0,6E0,1.1E1,1E1,8E0,6E0,4E0,6.2E1,4.6E1,2E0,7E0,1E1,3E0,2.25E2,7.3E1,6.8E1,1.35E2,2.8E1,3E0,2.9E1,4.6E1,1.5E1,6E1,9E0,2E0,7.9E1,8.8E1,7E0,9E0,3E0,2E0,6E0,3E0,1E1,4.2E1,9E0,5E0,2E0,1E1,1.2E1,7E0,1.8E1,1.5E1,1E1,9E0,2E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"235","size_leaf_vector":"1"}},{"base_weights":[6.338032E-3,8.222665E-2,-2.5490075E-1,2.2233218E-2,3.3822837E-1,-4.123357E-1,-5.122351E-2,-6.951032E-2,1.726784E-1,7.06888E-1,2.72505E-1,-3.4595805E-1,-5.285216E-1,-1.4404967E-1,1.0078135E-1,-1.3514042E-1,4.954828E-3,1.05758496E-1,3.4194988E-1,6.1682904E-1,4.5112554E-2,1.9511276E-1,4.448187E-1,-2.8858843E-1,-5.3761405E-1,-5.9170526E-1,-3.01153E-1,-1.1363645E-1,-4.2540383E-1,1.7934081E-1,-2.4430513E-1,-1.6636902E-1,-5.652379E-3,4.6526868E-2,-2.8134796E-1,1.432568E-1,-2.0121013E-1,6.661857E-1,2.9826364E-1,2.9614111E-2,1.5738808E-2,2.6981556E-1,8.4354796E-2,3.8616955E-1,3.857047E-2,-3.0429342E-1,5.4656537E-3,-6.715169E-1,-3.781105E-1,-6.1449105E-1,-6.163345E-3,-1.7272027E-1,-4.1519237E-1,-2.6123178E-1,-8.13217E-2,-3.8101405E-3,-2.1722348E-2,1.0372523E-1,4.04534E-1,-3.024085E-1,4.902214E-3,-1.339152E-1,-3.999455E-1,6.6614285E-2,-1.2876455E-1,1.242391E-1,-7.97406E-2,-3.3236426E-1,1.8465961E-3,4.0687587E-2,2.1802472E-1,-2.977126E-1,-5.600058E-2,2.0177996E-1,3.592751E-2,3.2531884E-1,-3.9077327E-3,3.1315137E-2,2.524945E-1,1.4796592E-1,-1.6008444E-1,6.5853256E-1,3.450473E-1,-4.385947E-1,-2.6128468E-1,-7.274383E-1,-4.2335233E-3,-4.4299564E-1,-1.8505517E-1,-5.0905967E-1,-8.007742E-1,-1.0242408E-2,-1.8949583E-3,-4.9348753E-3,-4.6553075E-1,-2.8237647E-1,-1.5026409E-3,-5.242302E-2,-2.00075E-1,-7.369649E-2,1.756995E-1,2.1444762E-2,6.5141185E-3,-2.9558681E-3,-3.9072764E-1,-1.9035111E-1,-6.274725E-2,-5.641992E-1,-2.4597454E-1,1.574093E-1,6.7559606E-3,-1.8846585E-1,1.1553736E-2,2.9640745E-2,1.0392325E-1,-1.3123938E-1,9.334104E-2,-4.4945127E-1,-2.1118598E-1,-9.3259156E-2,9.9963985E-2,1.9166613E-1,4.4171488E-1,-3.627614E-1,-1.6386323E-1,-1.4891155E-1,2.42017E-3,-2.051878E-5,1.5246145E-2,3.5357636E-1,8.005854E-2,-6.695709E-3,5.4358905E-3,1.829449E-1,3.5740316E-1,4.0393595E-2,3.0363342E-1,-2.9896808E-1,1.4436358E-1,3.341658E-2,1.1471091E-2,1.6068763E-1,3.976712E-1,-2.484945E-3,-4.6234566E-1,-1.3499221E-1,-3.2888064E-1,-1.4470454E-2,-3.689292E-2,-2.173893E-2,-6.078776E-3,-1.1294817E-2,-7.2438695E-4,-6.1093366E-1,-3.257019E-1,-5.6481016E-1,-9.4583094E-1,-2.3659445E-2,-7.1628666E-3,-6.2920796E-3,-1.5117243E-2,-1.8592736E-2,-8.925944E-3,-1.0693765E-2,1.575779E-4,5.4672424E-4,-1.6097838E-1,-6.1972503E-4,2.1163748E-1,-9.549401E-3,-2.3920285E-2,-1.0251521E-2,-3.224346E-3,2.6183624E-2,-3.8357903E-3,-6.1587254E-3,-3.1216357E-2,-2.3387577E-2,-8.40253E-3,-2.105558E-3,9.180959E-3,-2.8646765E-3,4.935611E-3,-1.1255655E-2,-1.9609027E-3,3.5654341E-3,1.0666748E-2,-9.837695E-3,-4.200222E-3,5.656132E-3,-5.753507E-5,-1.6817054E-2,-3.2035388E-2,-1.0794274E-2,2.668229E-3,8.396356E-3,-8.5516395E-3,5.3456407E-3,-3.4696774E-3,2.860887E-3,1.0508984E-2,7.512867E-3,2.3659088E-2,-1.8695693E-2,-4.9335584E-3,-1.1390907E-2,5.6466316E-3,-8.77004E-3,9.1405987E-4,9.068838E-3,1.8738283E-2,-1.862939E-2,8.8860085E-3,9.034072E-3,-6.3745463E-3,1.1241205E-2,2.3365444E-2,8.390751E-3,-1.7749299E-3,1.6548121E-2,-5.7982635E-3,-6.732359E-3,-2.1713343E-2,1.2190704E-3,1.1446391E-2,7.8822015E-4,9.210252E-3,1.396272E-2,2.0867301E-2,-5.925586E-3,-2.1792486E-2,-3.933535E-3,-1.2934888E-2,-2.2560412E-2,-1.2526997E-2,-3.1214043E-2,-1.8358713E-2,-1.6077656E-2,-5.399018E-3,-2.7791971E-2,-9.644839E-3,-2.6246808E-2,-4.911335E-2,3.0155361E-3,-2.9198276E-3,-2.8834124E-3,-1.36681795E-2,1.6982775E-2,6.4070118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,81,-1,83,-1,85,87,89,-1,91,93,95,97,-1,-1,99,101,103,-1,105,107,109,111,113,115,117,-1,119,121,123,125,127,-1,129,131,-1,133,135,137,139,141,143,145,147,-1,149,151,153,155,-1,-1,-1,157,159,-1,161,163,165,167,-1,-1,-1,169,171,173,175,177,179,181,183,-1,-1,185,187,189,191,193,195,197,199,201,203,205,207,-1,-1,-1,209,211,-1,-1,213,215,217,219,221,223,-1,-1,225,227,-1,229,231,233,-1,-1,-1,-1,-1,-1,235,237,239,241,-1,-1,-1,-1,243,-1,-1,-1,-1,245,-1,247,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4211E1,2.653183E1,1.607597E1,1.9349125E1,7.754669E0,1.9996872E0,3.1187239E0,4.2573824E0,5.9788074E0,6.631088E-1,3.6472206E0,1.8569794E0,1.2727718E0,1.1239524E0,2.3210375E0,1.8713074E0,4.881086E0,4.4259524E0,1.9259987E0,1.7403316E-1,0E0,1.5906563E0,1.724432E0,9.559746E-1,6.217699E-1,7.2602654E-1,2.7980137E-1,5.7494426E-1,2.0092487E-1,1.1329796E0,3.6052555E-1,2.7739496E0,8.1819606E-1,3.5227911E0,8.6813116E-1,2.6062183E0,5.718771E-1,8.45469E-1,1.0967636E0,0E0,0E0,6.729841E-1,1.2676113E0,6.3518906E-1,0E0,6.825075E-1,0E0,6.107607E-1,1.8107605E-1,1.0457268E0,0E0,8.156723E-2,1.3421535E-1,9.670925E-2,3.4642E-1,0E0,0E0,6.897478E-1,2.3818016E-1,2.6768124E-1,0E0,1.3094611E0,9.9928474E-1,3.141571E-1,8.27476E-1,2.3243918E0,1.2333987E0,5.435605E-1,0E0,1.1606052E0,1.0768814E0,1.582253E-1,1.9101536E-1,1.3663292E-1,0E0,8.241377E-1,2.3196313E-1,0E0,7.591038E-1,1.0594481E0,7.619685E-1,1.0311723E-1,6.0551834E-1,2.66253E-1,8.690486E-1,4.1710854E-1,0E0,1.0765529E-1,7.224442E-2,7.349863E-1,3.2148933E-1,0E0,0E0,0E0,9.265685E-2,1.01379514E-1,0E0,4.1251925E-1,1.4789635E-1,1.2172599E-1,2.6153314E-1,0E0,0E0,0E0,1.3599956E-1,7.7050686E-1,2.1200774E0,1.0328426E0,3.1261444E-1,2.2225493E-1,2.6470605E-1,2.5114644E-1,0E0,0E0,6.9406843E-1,3.3071327E-1,9.580073E-2,1.8013239E-1,1.9061565E-1,1.2344092E0,3.3205855E-1,9.104552E-1,3.5926914E-1,1.5878248E-1,2.845735E-1,7.966645E-2,0E0,0E0,0E0,9.338827E-1,8.8049227E-1,0E0,0E0,4.0422964E-1,6.3369036E-1,4.5720294E-1,7.413435E-1,2.5439334E-1,6.961672E-2,0E0,0E0,1.0057595E-1,1.5705013E-1,0E0,1.2594938E-1,2.5794935E-1,4.7222042E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.204361E-1,7.911539E-2,7.629228E-2,8.3390236E-2,0E0,0E0,0E0,0E0,2.7803874E-1,0E0,0E0,0E0,0E0,9.045459E-2,0E0,3.1373E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,45,45,47,47,48,48,49,49,51,51,52,52,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,75,75,76,76,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,90,90,94,94,95,95,97,97,98,98,99,99,100,100,104,104,105,105,106,106,107,107,108,108,109,109,110,110,111,111,114,114,115,115,116,116,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,129,129,130,130,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,144,144,145,145,146,146,153,153,154,154,155,155,156,156,161,161,166,166,168,168],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,82,-1,84,-1,86,88,90,-1,92,94,96,98,-1,-1,100,102,104,-1,106,108,110,112,114,116,118,-1,120,122,124,126,128,-1,130,132,-1,134,136,138,140,142,144,146,148,-1,150,152,154,156,-1,-1,-1,158,160,-1,162,164,166,168,-1,-1,-1,170,172,174,176,178,180,182,184,-1,-1,186,188,190,192,194,196,198,200,202,204,206,208,-1,-1,-1,210,212,-1,-1,214,216,218,220,222,224,-1,-1,226,228,-1,230,232,234,-1,-1,-1,-1,-1,-1,236,238,240,242,-1,-1,-1,-1,244,-1,-1,-1,-1,246,-1,248,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,8.631E3,1E0,4.930349E5,2E0,8.294758E-1,1.067536E3,3.6129813E2,2.9892595E6,3.8773947E0,1.7654233E0,2.3887E4,2.4019E6,8.3510876E9,9.6910744E7,4.949877E6,3.9712732E7,3E0,1.3841E4,5.2638875E6,4.5112554E-2,2.8091298E7,1.9013363E4,3.4409692E3,6.258335E5,8.838E3,8.15E2,1.246E3,1.9E1,5.185489E3,3.3047943E0,9.457831E0,8.901186E0,2.2893274E1,1.32E2,8.15E2,1.63E3,7.576111E-1,1E0,2.9614111E-2,1.5738808E-2,1.7324902E8,2.3580047E10,1.4075E4,3.857047E-2,1.074E3,5.4656537E-3,6.9309726E0,1.645614E6,6.1061732E7,-6.163345E-3,3E0,8.592097E-1,6.1E1,2.7246006E7,-3.8101405E-3,-2.1722348E-2,1E0,1.4955851E7,8.781059E4,4.902214E-3,1.454213E2,9.74026E0,9.9E1,1.3992102E3,2.04E5,7.06699E8,2.073089E-2,1.8465961E-3,1E0,5.0083565E6,1.3325451E1,9.115134E5,4E0,3.592751E-2,1.9162654E7,7.697189E2,3.1315137E-2,1.6382656E-1,7.023838E3,1.4126303E0,9.0763354E-1,2.2615E4,6.7913E4,1.74123E5,1.120848E5,-4.2335233E-3,1.9E1,4.854E4,2.1631205E0,2.0605305E5,-1.0242408E-2,-1.8949583E-3,-4.9348753E-3,3.3546498E-1,5.033303E-1,-1.5026409E-3,1.1095187E1,1.0052E4,4E0,4.366E3,2.1444762E-2,6.5141185E-3,-2.9558681E-3,2.010856E0,7.2896E5,2.04E5,2.04425E2,3.983889E3,2.53E2,3.4015749E0,1.4571098E5,1.1553736E-2,2.9640745E-2,1.1598511E3,4.66085E5,4.2037987E1,4E0,1E0,5E-1,1E0,5.033165E7,1E0,7.5646E4,4.2E1,1E0,2.42017E-3,-2.051878E-5,1.5246145E-2,2.539E3,1E0,-6.695709E-3,5.4358905E-3,1.3958375E0,9.3058E4,3.7794448E3,3.72424E5,2.8216E4,5.3E1,3.341658E-2,1.1471091E-2,1.948181E0,3.0497742E3,-2.484945E-3,1.4462532E7,3.8615453E5,2.863783E2,-1.4470454E-2,-3.689292E-2,-2.173893E-2,-6.078776E-3,-1.1294817E-2,-7.2438695E-4,4.116997E6,6.135771E9,1.3933473E5,1.2199979E10,-2.3659445E-2,-7.1628666E-3,-6.2920796E-3,-1.5117243E-2,1.2068E4,-8.925944E-3,-1.0693765E-2,1.575779E-4,5.4672424E-4,5.432393E-1,-6.1972503E-4,1.2068E4,-9.549401E-3,-2.3920285E-2,-1.0251521E-2,-3.224346E-3,2.6183624E-2,-3.8357903E-3,-6.1587254E-3,-3.1216357E-2,-2.3387577E-2,-8.40253E-3,-2.105558E-3,9.180959E-3,-2.8646765E-3,4.935611E-3,-1.1255655E-2,-1.9609027E-3,3.5654341E-3,1.0666748E-2,-9.837695E-3,-4.200222E-3,5.656132E-3,-5.753507E-5,-1.6817054E-2,-3.2035388E-2,-1.0794274E-2,2.668229E-3,8.396356E-3,-8.5516395E-3,5.3456407E-3,-3.4696774E-3,2.860887E-3,1.0508984E-2,7.512867E-3,2.3659088E-2,-1.8695693E-2,-4.9335584E-3,-1.1390907E-2,5.6466316E-3,-8.77004E-3,9.1405987E-4,9.068838E-3,1.8738283E-2,-1.862939E-2,8.8860085E-3,9.034072E-3,-6.3745463E-3,1.1241205E-2,2.3365444E-2,8.390751E-3,-1.7749299E-3,1.6548121E-2,-5.7982635E-3,-6.732359E-3,-2.1713343E-2,1.2190704E-3,1.1446391E-2,7.8822015E-4,9.210252E-3,1.396272E-2,2.0867301E-2,-5.925586E-3,-2.1792486E-2,-3.933535E-3,-1.2934888E-2,-2.2560412E-2,-1.2526997E-2,-3.1214043E-2,-1.8358713E-2,-1.6077656E-2,-5.399018E-3,-2.7791971E-2,-9.644839E-3,-2.6246808E-2,-4.911335E-2,3.0155361E-3,-2.9198276E-3,-2.8834124E-3,-1.36681795E-2,1.6982775E-2,6.4070118E-3],"split_indices":[18,2,58,27,16,26,51,51,50,53,40,9,31,5,44,9,44,11,9,42,0,44,51,51,31,2,0,2,3,51,34,53,55,52,3,2,10,40,111,0,0,7,5,9,0,2,0,52,9,44,0,8,26,3,44,0,0,109,46,32,0,51,52,10,4,5,7,40,0,109,27,57,31,6,0,46,4,0,34,4,40,26,10,12,28,31,0,3,28,53,32,0,0,0,37,26,0,53,2,8,2,0,0,0,56,9,5,4,27,2,55,27,0,0,51,1,57,6,67,52,79,7,109,9,3,112,0,0,0,2,8,0,0,38,2,54,9,9,3,0,0,41,51,0,5,32,51,0,0,0,0,0,0,12,5,32,5,0,0,0,0,9,0,0,0,0,26,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.228E3,1.727E3,5.01E2,1.4E3,3.27E2,2.82E2,2.19E2,8.7E2,5.3E2,4.8E1,2.79E2,1.82E2,1E2,1.36E2,8.3E1,4.62E2,4.08E2,3.81E2,1.49E2,3.9E1,9E0,1.94E2,8.5E1,1.42E2,4E1,7.7E1,2.3E1,1.24E2,1.2E1,6.8E1,1.5E1,3.72E2,9E1,3.57E2,5.1E1,3.4E2,4.1E1,1.6E1,1.33E2,3.3E1,6E0,1.15E2,7.9E1,7.6E1,9E0,1.37E2,5E0,2E1,2E1,7.3E1,4E0,1.2E1,1.1E1,2.1E1,1.03E2,2E0,1E1,5.2E1,1.6E1,1.3E1,2E0,3.28E2,4.4E1,5.7E1,3.3E1,2.21E2,1.36E2,4.4E1,7E0,1.44E2,1.96E2,2.4E1,1.7E1,4E0,1.2E1,1.22E2,1.1E1,3E0,1.12E2,6.3E1,1.6E1,8E0,6.8E1,3.1E1,1.06E2,1.8E1,2E0,1.4E1,6E0,4.9E1,2.4E1,8E0,4E0,2E0,9E0,1.9E1,2E0,8.4E1,1.9E1,1.5E1,3.7E1,1.2E1,4E0,4E0,9E0,1.82E2,1.46E2,2E1,2.4E1,2.2E1,3.5E1,2.9E1,4E0,7E0,2.14E2,1.05E2,3.1E1,2.1E1,2.3E1,4.4E1,1E2,1.77E2,1.9E1,1.5E1,9E0,9E0,8E0,2E0,2E0,1.09E2,1.3E1,5E0,6E0,6.9E1,4.3E1,3.8E1,2.5E1,1.1E1,5E0,6E0,2E0,1.6E1,5.2E1,2E0,2.9E1,3.8E1,6.8E1,4E0,1.4E1,1.2E1,2E0,4E0,2E0,3E1,1.9E1,1.1E1,1.3E1,7E0,2E0,6E0,1.3E1,6.9E1,1.5E1,1.6E1,3E0,8E0,7E0,6E0,3.1E1,5E0,4E0,1.39E2,4.3E1,4E0,1.42E2,5E0,1.5E1,3E0,2.1E1,4E0,1.8E1,2.1E1,1.4E1,2E1,9E0,1.81E2,3.3E1,3.1E1,7.4E1,2.3E1,8E0,1.8E1,3E0,2.1E1,2E0,1.1E1,3.3E1,9.1E1,9E0,4.3E1,1.34E2,5E0,1.4E1,1.2E1,3E0,7E0,2E0,7E0,2E0,3.2E1,7.7E1,2E0,1.1E1,6.6E1,3E0,2.7E1,1.6E1,1.3E1,2.5E1,2.2E1,3E0,7E0,4E0,3E0,2E0,4E0,1.2E1,2.4E1,2.8E1,2E0,2.7E1,3E1,8E0,1.4E1,5.4E1,2E1,1E1,1.6E1,3E0,9E0,2E0,5E0,8E0,2.4E1,4.5E1,5E0,2E0,8E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"249","size_leaf_vector":"1"}},{"base_weights":[1.5090432E-3,-2.0455705E-1,9.516229E-2,-3.3318844E-1,-6.618688E-2,3.7665926E-2,3.3528122E-1,-3.9972454E-1,-1.7712644E-1,4.40584E-2,-8.4714934E-2,-8.713996E-2,1.2590407E-1,5.393393E-1,2.5885952E-1,-3.3199403E-1,-5.296861E-1,-2.6801482E-1,-8.7914184E-2,-1.534796E-1,1.8060899E-1,-2.7771842E-1,-4.9881324E-2,1.8090808E-3,1.8617508E-1,2.745857E-1,6.3502884E-1,1.9180167E-1,4.746803E-1,-3.9879382E-1,-2.0399451E-1,-6.751538E-1,-3.6467975E-1,-2.3403384E-1,-2.2675768E-2,-1.24042615E-1,5.790432E-2,-1.1489931E-1,-4.037244E-1,2.7826827E-2,2.7234176E-1,-2.0758505E-1,-8.3909816E-1,-1.9687957E-1,-1.8447855E-2,1.2754901E-2,-2.3225822E-2,-1.5177687E-1,2.2837962E-1,3.368495E-1,-7.66668E-4,4.4618983E-2,5.414752E-1,2.163123E-1,-5.0603753E-1,3.7369353E-1,2.9639583E-2,-3.2262936E-1,-5.5955464E-1,6.77284E-2,-2.8816515E-1,-5.315704E-1,-8.5302037E-1,-4.244381E-1,-1.5768458E-1,-3.3461303E-1,-1.46649E-1,-7.779368E-2,-1.0150319E-2,1.5803768E-4,7.509087E-3,-8.067562E-2,-3.0424696E-1,-6.741127E-1,-2.9864222E-1,-1.9367255E-3,9.644394E-2,1.7204797E-1,4.020805E-1,-2.4696131E-1,2.1524997E-3,-1.4166118E-2,-4.2772174E-2,-2.7836168E-1,-7.614289E-2,3.4544778E-1,-4.454081E-2,-2.6085753E-2,1.4034405E-1,-2.2749169E-1,1.7176351E-1,1.8575265E-1,3.814932E-1,3.8777822E-1,-2.8988726E-3,1.4402669E-2,5.848611E-1,-6.260385E-3,2.4398519E-1,-8.371568E-3,-2.8086798E-2,4.0325403E-1,-3.4424856E-3,-3.419023E-2,-2.8210205E-1,-6.125484E-1,-9.514728E-3,1.480768E-4,1.1054515E-2,4.5876376E-2,-3.3002087E-1,-3.1061612E-2,-1.53519465E-2,-1.5564349E-2,-8.951889E-1,-7.592223E-3,-4.6577215E-1,-1.3834494E-3,-1.1704723E-2,-1.1262208E-2,-1.9512514E-2,3.5284746E-3,-1.6692232E-1,-4.6055573E-3,2.1214532E-3,-1.7642789E-1,1.9856902E-2,-4.901293E-1,-1.7212881E-1,-1.2133532E-2,-3.7690595E-2,-3.2678458E-1,-2.495843E-3,6.0475827E-3,-3.8138072E-3,-6.3598426E-3,2.0394464E-1,1.2449289E-2,2.3840714E-2,-1.6461657E-1,-3.151919E-1,-2.3929259E-2,-2.3387532E-1,6.71296E-2,-1.1894437E-1,-1.09932065E-1,3.0721607E-2,-8.308109E-2,8.3015494E-2,-1.370986E-3,-3.4537113E-1,2.1996418E-1,7.624322E-2,-3.1245372E-1,-9.478295E-2,1.4117784E-2,1.3897267E-4,1.2704305E-1,2.7979377E-1,1.0070822E-1,4.2135555E-1,1.097579E-2,5.2099043E-1,2.7783034E-2,1.2054738E-2,9.7067595E-2,-1.3185453E-1,2.6186502E-1,-4.298313E-3,2.177266E-2,1.2586483E-2,-1.3881226E-2,4.8696934E-3,-1.7800676E-2,-3.2812998E-2,-8.738922E-3,9.669699E-3,-5.2483794E-3,-1.7132116E-2,-4.2903893E-2,-1.593509E-2,-2.522546E-2,-1.5442992E-2,-9.316329E-3,-3.8110449E-3,-1.0582812E-2,-1.7860206E-3,2.937516E-3,-9.604581E-3,-1.1131529E-2,-2.7096896E-2,3.2352686E-3,-1.432588E-2,-8.89777E-3,-1.9100202E-2,6.9786147E-3,1.8762546E-2,-9.714769E-3,5.4728467E-4,-5.014996E-3,-1.53024E-2,1.7135631E-3,-1.1632233E-2,-5.304792E-3,9.106589E-3,-7.132142E-3,3.0295881E-3,1.2930795E-3,-8.74472E-3,-5.259369E-3,1.5868624E-3,2.5830388E-3,1.5311353E-2,-4.403881E-3,2.431026E-3,-1.9426925E-2,3.474692E-3,1.1329895E-2,-3.4167308E-3,5.513958E-3,-6.403042E-3,-1.5703825E-2,3.551093E-3,8.252208E-3,-6.252317E-3,9.67235E-3,3.638335E-3,-2.9202786E-4,1.38577465E-2,6.0747564E-3,-2.516452E-3,1.2130526E-2,2.1268781E-2,7.021521E-3,2.7938966E-2,1.062029E-2,-6.8006985E-4,-1.5617353E-3,-1.1088942E-2,1.0096742E-2,2.0671034E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,87,-1,89,91,93,-1,-1,95,97,99,101,-1,103,105,107,109,111,113,115,117,119,121,123,-1,-1,-1,125,127,129,131,-1,133,135,137,139,-1,-1,-1,141,143,145,147,149,151,153,155,157,159,161,-1,-1,163,165,167,-1,-1,169,-1,-1,171,173,-1,-1,-1,175,177,-1,-1,-1,179,-1,181,-1,-1,-1,-1,-1,183,-1,-1,185,187,189,191,-1,-1,193,-1,-1,-1,-1,195,-1,-1,197,199,-1,201,203,205,207,-1,209,211,213,215,217,219,221,223,-1,-1,225,227,229,231,-1,233,-1,-1,235,237,239,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.317149E1,1.2419271E1,2.1220947E1,3.688118E0,6.591278E0,1.3688669E1,4.4919434E0,2.0511436E0,8.683574E-1,0E0,6.1032753E0,3.643004E0,5.4446163E0,1.8523636E0,3.0639544E0,1.363657E0,1.7940502E0,3.100412E-1,3.074696E-1,2.5120597E0,9.594655E-1,3.133017E0,1.9924952E0,1.3533267E0,7.0235844E0,4.2318034E-1,1.3682346E0,2.9830155E0,6.8236923E-1,1.1663399E0,1.3917825E0,6.702614E-1,4.5259953E-1,3.8159466E-1,0E0,1.9563407E-1,6.9911435E-2,1.4723756E0,7.931361E-1,1.3493243E-1,4.903388E-1,7.7523947E-1,2.0469856E-1,7.2648335E-1,3.4051418E0,1.1682378E0,0E0,1.386399E0,2.7748108E0,4.1273785E-1,0E0,0E0,2.7907848E-1,1.0054822E0,1.0558796E-1,4.9596167E-1,0E0,1.136797E0,5.008049E-1,1.697962E-1,6.5829134E-1,5.7034636E-1,7.877922E-2,2.6148367E-1,1.3312376E-1,7.482457E-2,1.404956E-1,1.0321437E-1,0E0,0E0,0E0,1.899868E0,7.7517986E-1,3.557949E-1,1.6701818E-1,0E0,1.0518378E-1,2.878282E-1,1.4005661E-1,3.1893063E-1,0E0,0E0,0E0,3.772223E-1,2.0421748E-1,3.7153428E0,1.6488595E0,1.4278092E0,2.6393354E-1,4.777689E-1,2.5895512E-1,1.8581963E0,1.0096846E0,2.028513E-1,0E0,0E0,1.9450188E-1,2.593692E-1,9.035511E-1,0E0,0E0,1.8726444E-1,0E0,0E0,7.2043085E-1,4.2893887E-1,0E0,0E0,0E0,2.8161502E-1,3.808899E-1,0E0,0E0,0E0,1.2242794E-1,0E0,1.1761665E-1,0E0,0E0,0E0,0E0,0E0,6.626856E-2,0E0,0E0,7.838292E-1,1.0270375E0,2.2346401E-1,7.958964E-1,0E0,0E0,2.1436381E-1,0E0,0E0,0E0,0E0,1.8769693E-1,0E0,0E0,2.8510046E-1,1.2571573E-1,0E0,2.6030445E-1,2.2115988E-1,1.898449E-1,1.3229495E-1,0E0,1.0085723E0,4.9744862E-1,8.9770585E-1,4.9238336E-1,2.3363793E-1,3.3448792E-1,3.903551E-1,2.5319356E-1,0E0,0E0,8.43863E-1,9.8790073E-1,7.546078E-2,4.8332214E-1,0E0,1.8565178E-1,0E0,0E0,1.730713E-1,9.06315E-2,9.4210243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,71,71,72,72,73,73,74,74,76,76,77,77,78,78,79,79,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,96,96,97,97,98,98,101,101,104,104,105,105,109,109,110,110,114,114,116,116,122,122,125,125,126,126,127,127,128,128,131,131,136,136,139,139,140,140,142,142,143,143,144,144,145,145,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,157,157,158,158,159,159,160,160,162,162,165,165,166,166,167,167],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,88,-1,90,92,94,-1,-1,96,98,100,102,-1,104,106,108,110,112,114,116,118,120,122,124,-1,-1,-1,126,128,130,132,-1,134,136,138,140,-1,-1,-1,142,144,146,148,150,152,154,156,158,160,162,-1,-1,164,166,168,-1,-1,170,-1,-1,172,174,-1,-1,-1,176,178,-1,-1,-1,180,-1,182,-1,-1,-1,-1,-1,184,-1,-1,186,188,190,192,-1,-1,194,-1,-1,-1,-1,196,-1,-1,198,200,-1,202,204,206,208,-1,210,212,214,216,218,220,222,224,-1,-1,226,228,230,232,-1,234,-1,-1,236,238,240,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,7.696629E0,8.631E3,1E0,2.1E1,2.0407547E5,5.473125E3,8.595346E-1,2.3859661E-2,4.40584E-2,1.5343539E0,3.3870136E7,8.46E2,2.345785E5,3.4606943E0,5.7118463E2,1.2835874E-2,1E0,5.124411E2,1E0,3.0990322E5,4E0,2.5454E4,5.8E1,1.2E1,4.5843E7,1.7324902E8,3.869321E2,4.091015E9,1.25218E5,1.3446785E7,6.496696E7,7.39235E8,1.074E3,-2.2675768E-2,1.902353E1,2.695E3,6.747114E7,6.0479047E4,2.3033286E5,8.5837976E7,3.83E3,2.9565217E0,6.884212E4,2.04E5,2.3314082E6,-2.3225822E-2,4.832493E-1,2.9251662E6,9.3161255E-1,-7.66668E-4,4.4618983E-2,9.9625344E5,3.0324321E-2,6.38041E5,7.065733E7,2.9639583E-2,1.727279E-4,4.020408E0,5.704E3,1.4105052E0,1.9121015E0,5.501614E9,1.2E1,1.4622E4,2.9813817E-3,9E0,5.2415428E5,-1.0150319E-2,1.5803768E-4,7.509087E-3,4.5971466E2,2.357309E3,1.120848E5,2.9799202E3,-1.9367255E-3,2.586199E7,1.2909952E1,2.939566E5,9.82542E5,2.1524997E-3,-1.4166118E-2,-4.2772174E-2,3.26985E4,1.1485E4,3.25E2,6.131E3,1.6714952E8,3.8575E4,1.0894445E-2,8.012376E0,5.862126E2,5.0338E4,8.088206E-2,-2.8988726E-3,1.4402669E-2,1E0,1.9599941E6,2.3619232E7,-8.371568E-3,-2.8086798E-2,2.0306142E7,-3.4424856E-3,-3.419023E-2,2.145E3,2.8107285E-1,-9.514728E-3,1.480768E-4,1.1054515E-2,2.183E3,2.65851E5,-3.1061612E-2,-1.53519465E-2,-1.5564349E-2,3.3953488E0,-7.592223E-3,8.69032E-2,-1.3834494E-3,-1.1704723E-2,-1.1262208E-2,-1.9512514E-2,3.5284746E-3,1.64354E9,-4.6055573E-3,2.1214532E-3,2.175722E6,1.2737473E1,2.8035452E11,9.64E2,-1.2133532E-2,-3.7690595E-2,2.8980975E5,-2.495843E-3,6.0475827E-3,-3.8138072E-3,-6.3598426E-3,1.9015984E6,1.2449289E-2,2.3840714E-2,1.6E1,6.377113E4,-2.3929259E-2,8E0,8.317E3,4.36E2,1.2944685E0,3.0721607E-2,9.277255E6,1.4777102E9,4.54E2,3.8398104E0,7.89E2,1.7560976E0,2.8326238E3,1.4560842E-2,1.4117784E-2,1.3897267E-4,1.5277338E0,1.0449754E0,3.55E2,1E0,1.097579E-2,3.52E2,2.7783034E-2,1.2054738E-2,2.5629143E3,5E0,6.4683E4,-4.298313E-3,2.177266E-2,1.2586483E-2,-1.3881226E-2,4.8696934E-3,-1.7800676E-2,-3.2812998E-2,-8.738922E-3,9.669699E-3,-5.2483794E-3,-1.7132116E-2,-4.2903893E-2,-1.593509E-2,-2.522546E-2,-1.5442992E-2,-9.316329E-3,-3.8110449E-3,-1.0582812E-2,-1.7860206E-3,2.937516E-3,-9.604581E-3,-1.1131529E-2,-2.7096896E-2,3.2352686E-3,-1.432588E-2,-8.89777E-3,-1.9100202E-2,6.9786147E-3,1.8762546E-2,-9.714769E-3,5.4728467E-4,-5.014996E-3,-1.53024E-2,1.7135631E-3,-1.1632233E-2,-5.304792E-3,9.106589E-3,-7.132142E-3,3.0295881E-3,1.2930795E-3,-8.74472E-3,-5.259369E-3,1.5868624E-3,2.5830388E-3,1.5311353E-2,-4.403881E-3,2.431026E-3,-1.9426925E-2,3.474692E-3,1.1329895E-2,-3.4167308E-3,5.513958E-3,-6.403042E-3,-1.5703825E-2,3.551093E-3,8.252208E-3,-6.252317E-3,9.67235E-3,3.638335E-3,-2.9202786E-4,1.38577465E-2,6.0747564E-3,-2.516452E-3,1.2130526E-2,2.1268781E-2,7.021521E-3,2.7938966E-2,1.062029E-2,-6.8006985E-4,-1.5617353E-3,-1.1088942E-2,1.0096742E-2,2.0671034E-2],"split_indices":[19,55,2,109,0,27,31,26,37,0,41,7,2,27,34,54,37,82,4,111,27,6,9,3,17,44,7,56,7,28,44,44,7,2,0,57,2,44,32,32,31,2,45,27,5,27,0,38,27,33,0,0,27,37,9,1,0,40,53,2,52,52,5,17,2,40,10,46,0,0,0,51,4,31,4,0,9,55,32,30,0,0,0,44,9,2,2,5,9,37,52,51,1,38,0,0,6,50,42,0,0,42,0,0,2,26,0,0,0,2,28,0,0,0,53,0,37,0,0,0,0,0,5,0,0,1,53,30,0,0,0,27,0,0,0,0,27,0,0,3,44,0,3,9,3,52,0,9,7,2,52,2,52,51,37,0,0,52,52,10,109,0,0,0,0,4,8,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.235E3,6.98E2,1.537E3,3.61E2,3.37E2,1.241E3,2.96E2,2.52E2,1.09E2,5E0,3.32E2,5.14E2,7.27E2,7.9E1,2.17E2,1.68E2,8.4E1,5.3E1,5.6E1,2.64E2,6.8E1,8.3E1,4.31E2,2.38E2,4.89E2,2.2E1,5.7E1,1.67E2,5E1,1.09E2,5.9E1,4.3E1,4.1E1,4.8E1,5E0,4.5E1,1.1E1,2.3E2,3.4E1,2.6E1,4.2E1,7.5E1,8E0,7.5E1,3.56E2,2.34E2,4E0,5.4E1,4.35E2,1.8E1,4E0,1E1,4.7E1,1.62E2,5E0,3.4E1,1.6E1,7.6E1,3.3E1,1.4E1,4.5E1,2.6E1,1.7E1,3.1E1,1E1,2.1E1,2.7E1,3.2E1,1.3E1,8E0,3E0,1.96E2,3.4E1,8E0,2.6E1,1.3E1,1.3E1,2.5E1,1.7E1,6.5E1,1E1,2E0,6E0,4.4E1,3.1E1,2.3E1,3.33E2,1.8E2,5.4E1,4.4E1,1E1,3.42E2,9.3E1,1.6E1,2E0,9E0,3.8E1,1.8E1,1.44E2,2E0,3E0,3.2E1,2E0,5E0,7.1E1,2.8E1,5E0,1.1E1,3E0,5E0,4E1,1.3E1,1.3E1,2E0,1.5E1,5E0,2.6E1,5E0,5E0,1.3E1,8E0,2E0,2.5E1,2.7E1,5E0,1E2,9.6E1,1.3E1,2.1E1,3E0,5E0,2.3E1,3E0,1.1E1,2E0,2E0,2.3E1,1E1,7E0,3.1E1,3.4E1,5E0,3.9E1,7E0,2.4E1,1E1,1.3E1,2.56E2,7.7E1,1.68E2,1.2E1,2.3E1,3.1E1,2.6E1,1.8E1,5E0,5E0,2.12E2,1.3E2,1.2E1,8.1E1,9E0,7E0,3.4E1,4E0,1E1,8E0,1.37E2,7E0,1.8E1,1.4E1,6.7E1,4E0,1.1E1,1.7E1,2E0,3E0,8E0,3.2E1,1.3E1,2E0,1.3E1,1.3E1,1.6E1,9E0,7E1,3E1,8.1E1,1.5E1,5E0,8E0,8E0,1.3E1,1.1E1,1.2E1,2E1,3E0,2.4E1,7E0,4E0,3E1,3E0,3.6E1,3E0,4E0,2E1,4E0,4E0,6E0,2E2,5.6E1,7.1E1,6E0,6.1E1,1.07E2,1E1,2E0,2.1E1,2E0,2.6E1,5E0,2.4E1,2E0,2E0,1.6E1,7.3E1,1.39E2,1.1E1,1.19E2,1E1,2E0,2.1E1,6E1,2E0,5E0,4E0,6E0,5E0,3E0,1.16E2,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"241","size_leaf_vector":"1"}},{"base_weights":[-1.1008189E-3,7.2276324E-2,-2.5231954E-1,-1.8715482E-2,2.369019E-1,-3.912274E-1,-4.339074E-2,-2.4520246E-2,4.721193E-2,8.790385E-2,3.0150646E-1,-3.5933644E-1,-6.870527E-1,-1.3611679E-1,1.0338739E-1,-1.3674074E-1,2.2257917E-2,1.1873899E-1,-2.9720664E-1,2.008829E-1,4.06528E-1,-4.0720335E-1,-1.974065E-1,-8.663829E-1,-2.776202E-1,-9.209208E-2,-4.469552E-1,1.8837197E-1,-3.3602566E-1,-1.5298355E-1,3.3639506E-1,3.0642394E-2,1.2792046E-2,1.2929794E-1,-1.616558E-2,1.164768E-3,-3.5317284E-1,1.12986766E-1,4.0159842E-1,4.429334E-1,1.7818516E-1,-3.4578553E-1,-5.649178E-1,1.4274451E-1,-2.2865225E-1,-2.3125615E-2,-4.487597E-2,-4.8454866E-2,-3.6046557E-2,-1.3134308E-1,-1.8273786E-2,-2.7655084E-2,-2.698285E-1,1.1554951E-1,4.087203E-1,-5.534799E-3,-2.0570364E-2,-1.2955937E-1,-3.346675E-1,9.023046E-3,2.0873586E-2,3.3124033E-2,-2.4953835E-1,-9.7503796E-2,1.5202215E-1,-1.7814651E-2,-4.5759464E-3,1.5583825E-1,-1.9743098E-1,4.398778E-1,1.6118364E-2,3.6672464E-1,5.5418503E-1,2.5466806E-1,-6.397633E-2,-4.9971017E-1,-3.057571E-1,-5.855115E-1,-6.5952097E-3,-1.3136558E-3,1.0679154E-2,-1.4095968E-1,-3.0331135E-1,3.6465034E-3,-6.427192E-3,-8.761333E-2,-1.8494235E-1,5.701784E-2,-1.0052907E-1,-1.4847976E-2,-2.0033296E-3,2.4177657E-1,2.6313461E-2,5.185483E-1,1.0597445E-2,-1.6507038E-1,-2.5665723E-2,-1.8799315E-1,-5.4125094E-1,-3.2823313E-2,1.0939399E-1,-3.8621742E-1,-2.6964398E-2,2.6995633E-2,-1.1836014E-2,1.7367972E-1,-6.024996E-2,2.504675E-2,2.0496313E-1,-2.4085757E-1,-1.2525063E-3,3.6186206E-1,8.4176004E-1,-1.1434692E-2,1.2711739E-2,3.9833355E-1,8.065888E-2,7.290814E-1,3.687528E-1,2.1853905E-2,1.8125509E-1,-6.847923E-3,5.97195E-3,-5.765439E-1,-1.30564775E-2,-2.6851615E-1,-4.6621436E-1,-6.0007095E-1,-6.2467386E-3,-8.03422E-3,3.1566485E-3,-4.2142776E-1,-2.0653109E-1,-2.1436621E-1,-1.3950126E-2,-1.4439564E-2,-1.532787E-1,5.5566058E-3,-4.423931E-2,-2.3267308E-4,-6.5266844E-3,1.5454889E-2,1.5423334E-1,6.0895856E-2,-1.0066285E-2,8.519734E-3,2.653305E-2,6.886119E-3,-8.314412E-3,2.7282017E-3,-4.523642E-3,-1.2742649E-2,-2.1591806E-3,-2.994663E-2,-8.719527E-3,-6.09792E-3,3.2809302E-3,1.3767896E-3,8.767337E-3,-8.609107E-3,-2.0336756E-2,-5.0223E-3,3.037303E-3,8.504983E-3,-1.66162E-3,8.69915E-3,-5.2715954E-3,2.1812625E-3,-1.6772863E-2,-1.1331975E-3,6.8018376E-3,3.2520862E-3,1.1525799E-2,-1.2647888E-2,-2.9484842E-3,5.056105E-3,1.9298509E-2,1.5460711E-2,4.2372297E-2,2.0176468E-2,8.777316E-3,6.845393E-3,-4.4468953E-3,3.61026E-2,2.0058183E-2,1.8209677E-2,2.4214653E-5,9.774098E-3,-1.17257194E-4,-2.8390497E-2,-1.3102472E-2,-6.490101E-3,-1.3889825E-2,-2.9821213E-2,-1.13607235E-2,-2.9310295E-2,-1.8484805E-2,-2.1957038E-2,-2.1695541E-3,-1.3546622E-2,-5.7738423E-3,-1.0963824E-2,-1.01612E-3,-5.6861974E-3,2.2116327E-3,-1.3963435E-3,-8.135877E-3,-6.4059296E-3,1.2662874E-3,1.6089582E-3,9.192332E-3,-2.3283812E-3,5.288405E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,-1,-1,65,67,69,71,73,75,77,79,81,-1,-1,83,-1,85,87,-1,89,91,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,-1,107,109,111,113,115,117,119,121,123,125,127,-1,-1,-1,129,131,-1,-1,133,135,137,139,-1,-1,141,143,145,-1,147,149,151,153,155,157,159,161,163,-1,165,167,169,171,173,-1,175,177,-1,-1,179,181,183,185,-1,187,-1,-1,189,-1,191,193,195,-1,-1,-1,197,199,201,203,-1,205,-1,207,-1,-1,-1,209,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0609272E1,2.5544613E1,1.4435907E1,6.7974753E0,5.816456E0,2.5739288E0,2.736104E0,5.745962E0,0E0,2.2321525E0,4.388447E0,2.0279121E0,1.8153362E0,1.6420407E0,2.9760776E0,2.5260482E0,4.7808633E0,9.102874E-1,2.6211345E-1,3.8006067E0,1.6335373E0,1.8291321E0,7.128484E-1,3.3705044E-1,1.1497976E0,3.1342113E-1,2.7060056E-1,1.0118287E0,2.732948E-1,1.2844496E0,9.610093E-2,0E0,4.0778775E0,8.858304E-1,0E0,0E0,8.2184196E-2,2.0659523E0,9.6680164E-1,1.297432E0,5.719292E-1,7.898998E-1,4.0683174E-1,1.0282452E-1,3.4352112E-1,0E0,0E0,1.0793218E-1,0E0,1.5021229E-1,2.4726418E-1,0E0,1.0350448E-1,5.681022E-1,1.8551874E-1,0E0,0E0,1.0214081E0,9.5876575E-1,0E0,0E0,3.5697198E0,1.6627517E0,3.3359903E-1,7.1844816E-1,0E0,0E0,8.622575E-1,1.3341141E-1,1.5765877E0,5.686769E-1,9.444313E-1,2.0013504E0,2.9754543E-1,1.5672353E-1,3.2083082E-1,6.157856E-1,2.6906013E-1,0E0,0E0,0E0,2.3536992E-1,2.673335E-1,0E0,0E0,3.7982756E-1,8.263981E-2,1.4516625E-1,7.833584E-2,0E0,0E0,1.3043237E-1,2.8011146E-1,9.043431E-2,0E0,1.2475438E0,4.6538773E-1,2.7506447E-1,4.5934057E-1,4.096234E0,2.1835809E0,3.3438683E-1,1.8118104E-1,1.0935796E-1,0E0,7.723551E-1,5.2241063E-1,2.4198663E-1,6.290474E-1,8.330721E-2,0E0,7.90195E-1,1.0429144E-1,0E0,0E0,8.520794E-1,1.6627121E-1,3.309307E-1,4.4330025E-1,0E0,1.2153351E-1,0E0,0E0,1.23263836E-1,0E0,4.4951057E-1,7.462864E-1,2.0766258E-1,0E0,0E0,0E0,2.853515E-1,1.0095835E-1,7.45967E-2,1.9544138E-1,0E0,8.181167E-2,0E0,6.716034E-2,0E0,0E0,0E0,6.809169E-2,1.8185914E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,49,49,50,50,52,52,53,53,54,54,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,81,81,82,82,85,85,86,86,87,87,88,88,91,91,92,92,93,93,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,105,105,106,106,107,107,108,108,109,109,111,111,112,112,115,115,116,116,117,117,118,118,120,120,123,123,125,125,126,126,127,127,131,131,132,132,133,133,134,134,136,136,138,138,142,142,143,143],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,-1,-1,66,68,70,72,74,76,78,80,82,-1,-1,84,-1,86,88,-1,90,92,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,-1,108,110,112,114,116,118,120,122,124,126,128,-1,-1,-1,130,132,-1,-1,134,136,138,140,-1,-1,142,144,146,-1,148,150,152,154,156,158,160,162,164,-1,166,168,170,172,174,-1,176,178,-1,-1,180,182,184,186,-1,188,-1,-1,190,-1,192,194,196,-1,-1,-1,198,200,202,204,-1,206,-1,208,-1,-1,-1,210,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.0950326E6,1E0,9.256843E3,3.426E3,1.2427474E8,1.067536E3,1.638058E2,4.721193E-2,2.2075728E7,4.0434834E5,5.121E3,1.0052E4,8.3510876E9,9.6910744E7,7.780377E6,7.330957E4,5.884028E0,7.576E3,3.970405E3,2.342147E7,2.65851E5,1.3446785E7,8.8070023E-1,2.332E3,4.273439E2,6.654321E0,5.185489E3,2.002E3,4.0448148E2,2.6970406E0,3.0642394E-2,4.0844156E7,2.064332E2,-1.616558E-2,1.164768E-3,1.6315161E5,3.653602E7,1.96E2,1.6503105E3,6.061914E-1,7.33E2,1.0122174E3,5.704E3,6.747114E7,-2.3125615E-2,-4.487597E-2,3.28825E0,-3.6046557E-2,5.03E2,6.3300834E0,-2.7655084E-2,5.748E3,1.775044E7,1.903E3,-5.534799E-3,-2.0570364E-2,2.04946E6,4.0578016E-4,9.023046E-3,2.0873586E-2,2.8980975E5,3.1998687E3,1.7019386E2,3.3018634E5,-1.7814651E-2,-4.5759464E-3,1E0,3.4E1,1.9013363E4,2.974979E6,2E0,1.93974E7,1.7725672E8,1.11E2,4.6176647E5,1E0,3.7891315E6,-6.5952097E-3,-1.3136558E-3,1.0679154E-2,6.1061732E7,2.0401025E3,3.6465034E-3,-6.427192E-3,1.06E3,3.1788826E-1,1.58362E5,3.2E1,-1.4847976E-2,-2.0033296E-3,5.810811E-1,1.2737473E1,7.5E2,1.0597445E-2,5.8365756E-1,2.6838362E-1,2.9E1,6.137763E2,1.2477659E3,3.0291306E2,4.8297736E7,8.2619705E1,8.695652E-1,-1.1836014E-2,2.0992E4,1.6368244E0,4.0501267E8,6.3039363E4,2E1,-1.2525063E-3,1.2E1,5.2E1,-1.1434692E-2,1.2711739E-2,7.2E1,1E0,2.23371E8,6.134093E7,2.1853905E-2,2.711499E6,-6.847923E-3,5.97195E-3,1E0,-1.30564775E-2,2.014576E8,8E0,2.1267729E-1,-6.2467386E-3,-8.03422E-3,3.1566485E-3,4.099E3,9.173799E4,6.1E1,7.4456794E5,-1.4439564E-2,2E1,5.5566058E-3,1.2511433E3,-2.3267308E-4,-6.5266844E-3,1.5454889E-2,7.382963E0,3.830216E5,-1.0066285E-2,8.519734E-3,2.653305E-2,6.886119E-3,-8.314412E-3,2.7282017E-3,-4.523642E-3,-1.2742649E-2,-2.1591806E-3,-2.994663E-2,-8.719527E-3,-6.09792E-3,3.2809302E-3,1.3767896E-3,8.767337E-3,-8.609107E-3,-2.0336756E-2,-5.0223E-3,3.037303E-3,8.504983E-3,-1.66162E-3,8.69915E-3,-5.2715954E-3,2.1812625E-3,-1.6772863E-2,-1.1331975E-3,6.8018376E-3,3.2520862E-3,1.1525799E-2,-1.2647888E-2,-2.9484842E-3,5.056105E-3,1.9298509E-2,1.5460711E-2,4.2372297E-2,2.0176468E-2,8.777316E-3,6.845393E-3,-4.4468953E-3,3.61026E-2,2.0058183E-2,1.8209677E-2,2.4214653E-5,9.774098E-3,-1.17257194E-4,-2.8390497E-2,-1.3102472E-2,-6.490101E-3,-1.3889825E-2,-2.9821213E-2,-1.13607235E-2,-2.9310295E-2,-1.8484805E-2,-2.1957038E-2,-2.1695541E-3,-1.3546622E-2,-5.7738423E-3,-1.0963824E-2,-1.01612E-3,-5.6861974E-3,2.2116327E-3,-1.3963435E-3,-8.135877E-3,-6.4059296E-3,1.2662874E-3,1.6089582E-3,9.192332E-3,-2.3283812E-3,5.288405E-3],"split_indices":[18,46,58,4,2,44,51,51,0,44,27,2,2,5,44,27,44,52,9,51,44,28,44,26,0,51,53,51,0,4,52,0,44,51,0,0,32,44,8,51,37,2,51,2,44,0,0,53,0,0,52,0,2,44,0,0,0,9,37,0,0,27,4,51,32,0,0,107,3,51,28,6,49,12,3,32,108,27,0,0,0,44,4,0,0,2,26,28,3,0,0,56,53,0,0,53,40,3,4,54,51,44,55,52,0,28,37,7,27,6,0,17,3,0,0,8,111,31,49,0,28,0,0,15,0,7,3,38,0,0,0,0,32,3,31,0,3,0,4,0,0,0,53,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.201E3,1.704E3,4.97E2,1.098E3,6.06E2,2.98E2,1.99E2,1.093E3,5E0,1.84E2,4.22E2,2.71E2,2.7E1,1.22E2,7.7E1,3.21E2,7.72E2,1.71E2,1.3E1,2.17E2,2.05E2,2.08E2,6.3E1,1.8E1,9E0,1.08E2,1.4E1,6.5E1,1.2E1,3.11E2,1E1,1E1,7.62E2,1.68E2,3E0,2E0,1.1E1,1.52E2,6.5E1,1.76E2,2.9E1,1.52E2,5.6E1,5E0,5.8E1,6E0,1.2E1,7E0,2E0,7E1,3.8E1,6E0,8E0,5E1,1.5E1,5E0,7E0,2.77E2,3.4E1,6E0,4E0,7.08E2,5.4E1,1.5E1,1.53E2,9E0,2E0,1.34E2,1.8E1,5.9E1,6E0,1.07E2,6.9E1,2.2E1,7E0,2.9E1,1.23E2,5.3E1,3E0,2E0,3E0,2.8E1,3E1,3E0,4E0,4E1,3E1,2E1,1.8E1,6E0,2E0,2E1,3E1,8E0,7E0,2.06E2,7.1E1,2.1E1,1.3E1,3.8E2,3.28E2,3.3E1,2.1E1,9E0,6E0,1.39E2,1.4E1,3.7E1,9.7E1,1.4E1,4E0,5.1E1,8E0,3E0,3E0,9.6E1,1.1E1,3.4E1,3.5E1,4E0,1.8E1,5E0,2E0,2E1,9E0,1.02E2,2.1E1,5.1E1,2E0,2.4E1,4E0,1.2E1,1.8E1,1.4E1,2.6E1,4E0,2.6E1,1.2E1,8E0,6E0,1.2E1,8E0,1.2E1,2.7E1,3E0,2E0,6E0,1.1E1,1.95E2,3.3E1,3.8E1,1.2E1,9E0,9E0,4E0,1.93E2,1.87E2,1.7E2,1.58E2,9E0,2.4E1,1.1E1,1E1,2E0,7E0,1.31E2,8E0,1.1E1,3E0,2.7E1,1E1,2.7E1,7E1,1.1E1,3E0,1.1E1,4E1,2E0,6E0,7.7E1,1.9E1,8E0,3E0,2.6E1,8E0,3.2E1,3E0,1.5E1,3E0,1.6E1,4E0,2.5E1,7.7E1,1E1,1.1E1,3.9E1,1.2E1,1E1,2E0,7E0,1.1E1,1.2E1,2E0,9E0,1.7E1,5E0,2.1E1,3E0,5E0,4E0,8E0,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[8.664744E-3,-1.7483367E-1,9.231984E-2,-2.792021E-1,-4.7971137E-2,4.1533384E-2,2.864663E-1,-2.926714E-1,3.574521E-2,-1.3596058E-1,1.4449038E-1,-1.8836658E-2,1.7905766E-1,3.419122E-1,6.795271E-2,-3.5955012E-1,-1.7200646E-1,-3.115724E-1,-1.1341838E-1,8.125981E-2,3.1889698E-1,2.3900787E-3,-2.8960374E-1,1.4103964E-1,3.718662E-1,4.7740418E-1,2.658343E-1,-1.6130462E-2,1.1713434E-1,-2.9194355E-1,-5.72667E-1,-2.6630247E-2,-2.9710332E-1,-1.0949567E-2,-2.3911076E-2,-1.6186202E-1,-3.833626E-2,8.001287E-3,1.6721098E-1,4.0070343E-1,1.7091514E-1,-1.6172336E-1,5.2692205E-2,-3.8897395E-1,-1.9350252E-1,1.5587714E-1,-2.3917848E-1,2.2516714E-1,4.9319136E-1,5.9321374E-1,3.2070708E-1,1.453584E-1,3.7467855E-1,-7.91158E-3,2.3827578E-1,-3.2201326E-1,-1.3217746E-1,-6.9539535E-1,-3.6100793E-1,-5.345203E-2,1.761426E-2,-5.868641E-1,-2.3048048E-1,-1.3783954E-1,-3.8082033E-1,4.4733632E-2,-1.3418624E-1,1.255608E-1,-5.1064234E-2,2.3780271E-1,9.125151E-2,2.1639755E-2,8.7042535E-3,9.492069E-4,1.0165151E-2,-1.3682736E-1,-4.1302353E-1,5.875884E-1,3.6001127E-2,-3.3487353E-1,-2.9128496E-2,-2.1928312E-1,-1.8536347E-3,7.576358E-2,2.2543702E-1,2.4170956E-2,-1.950122E-2,3.1113157E-1,-4.350543E-4,6.575847E-1,3.46102E-1,4.7213906E-1,7.659073E-1,1.2944369E-1,3.7205958E-1,2.383187E-1,9.525866E-2,3.124127E-1,6.9550383E-1,5.3300604E-2,-5.3643845E-3,3.9764112E-1,5.2401043E-2,-3.451171E-1,-1.8152496E-1,-1.5827823E-1,3.7836488E-3,-3.9301714E-1,-7.849487E-1,-8.6827455E-3,-4.384761E-1,8.931588E-2,-1.1064226E-1,-4.4994024E-3,-3.0173393E-2,-1.6860333E-1,-4.4598043E-1,-1.8688878E-1,-6.610752E-2,-2.2702157E-2,-6.147709E-3,-8.383028E-3,9.758584E-3,-1.6923434E-1,6.926001E-2,-2.9079856E-3,8.2924375E-3,-9.140994E-2,2.771954E-3,2.8074422E-1,-2.5431514E-3,2.0926797E-3,9.013234E-3,-1.755408E-1,-2.0205982E-2,-1.8545365E-1,-3.9686535E-2,9.154892E-3,2.8211214E-2,-1.1634586E-1,6.759925E-2,-1.0740565E-2,-1.8755332E-2,3.957855E-6,-2.3616114E-1,-3.7088297E-2,1.2617306E-1,3.509796E-1,1.8310976E-1,6.3299867E-3,-6.2513533E-3,3.6731276E-1,2.6208705E-3,1.655915E-1,-1.7552046E-2,3.252879E-2,8.818288E-3,4.0017724E-1,-3.4268622E-3,1.2520373E-2,2.335425E-2,3.7067432E-2,1.643958E-2,-4.0432837E-4,1.0104318E-2,2.0109186E-2,1.0923036E-1,1.4734767E-2,1.3415666E-1,-1.9082222E-2,1.5460776E-1,3.4729296E-1,-1.3496566E-1,1.4315607E-2,4.0431626E-2,-1.9298835E-2,1.0228099E-2,4.5338935E-1,2.234605E-4,5.3247386E-3,-1.1211856E-2,-2.1377688E-2,-1.3647419E-2,-8.174028E-4,-9.657672E-3,-2.9309592E-4,-9.900339E-3,-8.696809E-3,-2.2408389E-2,-4.030743E-2,-1.6833184E-2,-1.0908474E-2,-2.4601536E-2,1.0327839E-2,1.5105066E-3,-6.6252053E-3,8.920573E-3,-1.4612886E-2,-3.1093147E-3,-2.3844829E-2,-9.576677E-3,-5.9077255E-3,-1.2106154E-2,4.515688E-3,-4.26391E-3,2.4865156E-3,-2.6222272E-3,-8.333447E-3,1.3836235E-3,7.6308707E-3,-3.903146E-3,1.1258243E-3,-9.026575E-3,5.4976307E-3,1.5767023E-2,-1.2088233E-2,-6.4689703E-3,1.1401413E-2,-3.2201754E-3,-1.2039017E-2,-8.884985E-4,-8.015352E-3,5.5455463E-4,5.4906873E-4,6.062503E-3,-1.2743293E-2,-6.135465E-3,-4.830976E-3,6.208411E-3,7.5541055E-3,6.5765734E-4,3.6709556E-5,1.721034E-2,9.79843E-3,2.4020416E-3,9.682725E-3,2.0524308E-2,1.5943514E-3,1.15253655E-2,7.3212367E-3,1.9908166E-2,-6.263641E-3,9.72554E-3,-1.5273318E-3,7.467935E-3,-5.0796643E-3,3.438202E-3,9.4719315E-3,3.0719365E-3,1.8511632E-2,1.1798193E-2,6.6949255E-3,-1.4186836E-2,7.1973824E-3,-3.4912955E-3,3.7080164E-3,2.3020841E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,-1,-1,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,-1,113,115,117,119,121,123,125,127,129,131,-1,-1,-1,-1,133,135,137,139,141,-1,143,-1,145,147,149,-1,151,153,155,157,159,161,163,165,167,169,171,173,175,-1,177,179,181,183,185,-1,187,189,-1,191,193,195,-1,-1,197,199,201,203,-1,-1,205,-1,207,209,-1,-1,211,-1,213,-1,-1,-1,215,217,219,-1,-1,-1,221,223,-1,-1,-1,225,227,229,231,233,-1,-1,235,-1,237,-1,-1,-1,239,-1,-1,-1,-1,-1,-1,-1,-1,241,-1,243,245,247,249,251,-1,-1,253,-1,255,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3971615E1,9.162231E0,1.497532E1,5.7012215E0,5.3371034E0,1.0016002E1,3.7932053E0,2.9724464E0,0E0,8.19736E-1,1.0658417E0,4.8228455E0,2.6450653E0,2.4603844E0,1.3979414E0,3.3051643E0,2.4653988E0,2.3465514E-1,6.957841E-1,4.646146E-1,2.4626613E-1,6.4393654E0,5.0725794E-1,1.7805028E0,9.568472E-1,1.424839E0,2.0829735E0,0E0,8.9563155E-1,8.4619904E-1,1.22962E0,7.3436666E-1,1.273982E0,0E0,0E0,5.629184E-1,6.190957E-1,2.91104E-1,1.6061193E-1,1.6581273E-1,8.371964E-2,1.0884023E0,5.308859E0,1.8369389E-1,1.2081373E-1,1.6435809E0,6.157772E-1,5.6255436E-1,5.4618454E-1,6.8382454E-1,3.447132E-1,3.4522378E-1,1.4807148E0,2.0937274E-1,8.6782277E-1,4.404564E-1,1.87796E-1,6.3926697E-1,2.1742868E-1,5.0794286E-1,0E0,4.3820286E-1,7.3821187E-1,3.6411405E-1,2.570207E-1,3.828705E-1,2.6841575E-1,1.6481407E-1,1.3244894E-1,2.303738E-1,7.8196794E-2,0E0,0E0,0E0,0E0,7.549417E-1,1.5253217E0,1.204896E-1,2.7999148E0,1.03260994E-1,0E0,1.036917E-1,0E0,8.0111223E-1,7.806387E-1,1.3188468E-1,0E0,2.7751327E-1,6.4284176E-1,2.5466824E-1,4.6365333E-1,1.308856E-1,1.0701275E-1,1.3170265E-1,5.389929E-1,2.0140588E-1,3.6260965E-1,1.1860132E0,6.916809E-1,2.549952E-1,0E0,3.480444E-1,3.2261792E-1,5.9024334E-1,1.2087524E-1,2.5530767E-1,0E0,1.09733224E-1,8.006363E-1,0E0,1.7993927E-1,1.3218296E-1,5.1204765E-1,0E0,0E0,7.326673E-1,1.3455296E-1,2.511742E-1,2.1890587E-1,0E0,0E0,1.0666443E-1,0E0,9.749484E-2,1.0715052E-1,0E0,0E0,2.6859295E-1,0E0,1.1904955E-1,0E0,0E0,0E0,3.3817768E-1,6.135601E-1,1.5489486E-1,0E0,0E0,0E0,7.8267825E-1,1.7675898E0,0E0,0E0,0E0,8.0417514E-2,5.4828954E-1,4.381324E-1,3.9789343E-1,5.127897E-1,0E0,0E0,1.274395E-1,0E0,7.223922E-2,0E0,0E0,0E0,8.922362E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3884569E-1,0E0,7.794389E-2,1.7663845E-1,1.5161246E-1,2.5653076E-1,3.4907386E-1,0E0,0E0,1.6686478E-1,0E0,2.4486518E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,75,75,76,76,77,77,78,78,79,79,81,81,83,83,84,84,85,85,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,107,107,108,108,110,110,111,111,112,112,115,115,116,116,117,117,118,118,121,121,123,123,124,124,127,127,129,129,133,133,134,134,135,135,139,139,140,140,144,144,145,145,146,146,147,147,148,148,151,151,153,153,157,157,166,166,168,168,169,169,170,170,171,171,172,172,175,175,177,177],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,-1,-1,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,-1,114,116,118,120,122,124,126,128,130,132,-1,-1,-1,-1,134,136,138,140,142,-1,144,-1,146,148,150,-1,152,154,156,158,160,162,164,166,168,170,172,174,176,-1,178,180,182,184,186,-1,188,190,-1,192,194,196,-1,-1,198,200,202,204,-1,-1,206,-1,208,210,-1,-1,212,-1,214,-1,-1,-1,216,218,220,-1,-1,-1,222,224,-1,-1,-1,226,228,230,232,234,-1,-1,236,-1,238,-1,-1,-1,240,-1,-1,-1,-1,-1,-1,-1,-1,242,-1,244,246,248,250,252,-1,-1,254,-1,256,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,8.18E3,1.9013363E4,4.1393795E8,1.022332E6,3.5301748E7,1.3411796E3,3.574521E-2,5.0829604E3,1.2679E4,3.9712732E7,1.4580295E0,1.4232318E3,2.9816154E1,4.0844156E7,6.747114E7,1.692E3,2.728889E2,1.5822886E0,2.4848485E0,4.67E2,6.4587574E-3,6.3300834E0,3.544243E8,9.170864E1,2.8541E4,-1.6130462E-2,7.023838E3,3.008162E6,9.966725E2,1.3758811E0,1.9469026E0,-1.0949567E-2,-2.3911076E-2,1.5988282E1,1.6256282E7,2.8E1,4.9716983E0,8.518373E7,2.720437E6,4.602015E0,3.92382E5,2E0,2.3E1,1.06E3,1.5092975E6,3.0823356E5,1.6954315E0,1.6503105E3,1.5694646E3,4.8E1,4.516183E1,2.5492362E7,1.0008265E1,2.1E1,6.222E3,3.325328E11,1.1E1,2.2075728E7,1.761426E-2,5.3474556E5,6.0435E4,1E0,2.8E1,5.674727E2,2.777588E6,1.02E3,8.8543115E2,2.8807552E7,2.8352614E9,2.1639755E-2,8.7042535E-3,9.492069E-4,1.0165151E-2,3.5E2,1E0,1.279012E7,5.015233E4,7.218466E7,-2.9128496E-2,1.3794063E1,-1.8536347E-3,1E0,8.836364E0,9E0,-1.950122E-2,2.184578E1,1.4633E4,2.3977574E7,2.8454319E1,1E0,1.4955851E7,1.2E1,2.6284674E2,9.517604E6,2.400393E3,1.921032E4,1.3532244E7,7.683389E7,-5.3643845E-3,8.3998317E-1,1.4299594E8,1.5294166E5,1.0710737E2,6.5346925E6,3.7836488E-3,6.5029144E5,1.468963E6,-8.6827455E-3,8.3510876E9,1.5311552E9,7.023838E3,-4.4994024E-3,-3.0173393E-2,2.168033E3,3.166E3,4.44E2,1.1331109E6,-2.2702157E-2,-6.147709E-3,6.731367E0,9.758584E-3,4.1045683E8,3.402546E6,-2.9079856E-3,8.2924375E-3,1.5E1,2.771954E-3,7.9203456E8,-2.5431514E-3,2.0926797E-3,9.013234E-3,4.1E1,1.2E1,9.213145E0,-3.9686535E-2,9.154892E-3,2.8211214E-2,4.564527E6,1.7835504E-1,-1.0740565E-2,-1.8755332E-2,3.957855E-6,8.7999426E-2,2.2415E4,1E0,1.362E4,4.7272725E0,6.3299867E-3,-6.2513533E-3,6.678896E2,2.6208705E-3,1.98714E5,-1.7552046E-2,3.252879E-2,8.818288E-3,1.7848537E0,-3.4268622E-3,1.2520373E-2,2.335425E-2,3.7067432E-2,1.643958E-2,-4.0432837E-4,1.0104318E-2,2.0109186E-2,5E0,1.4734767E-2,1.5E1,1.0883807E5,7.56298E5,4.13568E5,5.3E1,1.4315607E-2,4.0431626E-2,1.1805985E3,1.0228099E-2,3.2343243E1,2.234605E-4,5.3247386E-3,-1.1211856E-2,-2.1377688E-2,-1.3647419E-2,-8.174028E-4,-9.657672E-3,-2.9309592E-4,-9.900339E-3,-8.696809E-3,-2.2408389E-2,-4.030743E-2,-1.6833184E-2,-1.0908474E-2,-2.4601536E-2,1.0327839E-2,1.5105066E-3,-6.6252053E-3,8.920573E-3,-1.4612886E-2,-3.1093147E-3,-2.3844829E-2,-9.576677E-3,-5.9077255E-3,-1.2106154E-2,4.515688E-3,-4.26391E-3,2.4865156E-3,-2.6222272E-3,-8.333447E-3,1.3836235E-3,7.6308707E-3,-3.903146E-3,1.1258243E-3,-9.026575E-3,5.4976307E-3,1.5767023E-2,-1.2088233E-2,-6.4689703E-3,1.1401413E-2,-3.2201754E-3,-1.2039017E-2,-8.884985E-4,-8.015352E-3,5.5455463E-4,5.4906873E-4,6.062503E-3,-1.2743293E-2,-6.135465E-3,-4.830976E-3,6.208411E-3,7.5541055E-3,6.5765734E-4,3.6709556E-5,1.721034E-2,9.79843E-3,2.4020416E-3,9.682725E-3,2.0524308E-2,1.5943514E-3,1.15253655E-2,7.3212367E-3,1.9908166E-2,-6.263641E-3,9.72554E-3,-1.5273318E-3,7.467935E-3,-5.0796643E-3,3.438202E-3,9.4719315E-3,3.0719365E-3,1.8511632E-2,1.1798193E-2,6.6949255E-3,-1.4186836E-2,7.1973824E-3,-3.4912955E-3,3.7080164E-3,2.3020841E-2],"split_indices":[19,109,2,51,12,27,44,4,0,50,2,44,41,47,55,44,44,2,51,41,56,2,41,52,7,57,2,0,4,31,4,33,52,0,0,53,44,8,53,43,31,52,5,6,8,2,27,32,53,51,51,3,53,49,53,3,2,30,3,44,0,31,10,8,3,51,1,0,51,9,5,0,0,0,0,2,6,7,27,44,0,55,0,109,57,16,0,55,10,46,55,109,46,3,57,44,4,4,1,31,0,26,44,32,4,44,0,27,28,0,5,12,4,0,0,4,0,0,44,0,0,53,0,7,1,0,0,3,0,12,0,0,0,2,10,52,0,0,0,9,37,0,0,0,40,28,112,1,53,0,0,4,0,1,0,0,0,53,0,0,0,0,0,0,0,0,8,0,3,27,28,9,3,0,0,32,0,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.211E3,6.92E2,1.519E3,3.79E2,3.13E2,1.205E3,3.14E2,3.75E2,4E0,2.15E2,9.8E1,8.38E2,3.67E2,2.5E2,6.4E1,2.4E2,1.35E2,2.3E1,1.92E2,7.3E1,2.5E1,7.78E2,6E1,3.08E2,5.9E1,8.8E1,1.62E2,6E0,5.8E1,1.84E2,5.6E1,6.3E1,7.2E1,1.9E1,4E0,1.16E2,7.6E1,4E1,3.3E1,1.5E1,1E1,1.82E2,5.96E2,2.8E1,3.2E1,2.97E2,1.1E1,2.8E1,3.1E1,4.9E1,3.9E1,7.8E1,8.4E1,2.9E1,2.9E1,1.54E2,3E1,3.4E1,2.2E1,6E1,3E0,1.2E1,6E1,1.06E2,1E1,4.1E1,3.5E1,1.3E1,2.7E1,1.6E1,1.7E1,1E1,5E0,3E0,7E0,1.67E2,1.5E1,1.7E1,5.79E2,2.5E1,3E0,2.7E1,5E0,1.39E2,1.58E2,5E0,6E0,2E1,8E0,1.3E1,1.8E1,3.1E1,1.8E1,9E0,3E1,2.6E1,5.2E1,7.2E1,1.2E1,1.9E1,1E1,1.5E1,1.4E1,1.31E2,2.3E1,2.7E1,3E0,9E0,2.5E1,8E0,1.4E1,1.7E1,4.3E1,2E0,1E1,4.8E1,1.2E1,6.2E1,4.4E1,6E0,4E0,3.2E1,9E0,3E1,5E0,3E0,1E1,2E1,7E0,1.4E1,2E0,1.3E1,4E0,1.25E2,4.2E1,1.1E1,4E0,2E0,1.5E1,9.9E1,4.8E2,1.3E1,1.2E1,2E0,2.5E1,4.3E1,9.6E1,3.8E1,1.2E2,3E0,2E0,1.6E1,4E0,6E0,2E0,1.1E1,2E0,1.6E1,2E0,7E0,2.4E1,1.5E1,3E0,4E0,5E0,2.3E1,7E0,1.3E1,1.3E1,1.8E1,3.4E1,6.7E1,5E0,5E0,7E0,1.4E1,5E0,1.3E1,2E0,1.2E1,2E0,3.1E1,1E2,4E0,1.9E1,8E0,1.9E1,4E0,5E0,1.9E1,6E0,6E0,8E0,4E0,1.3E1,3.9E1,4E0,1.8E1,3E1,8E0,4E0,3.8E1,2.4E1,6E0,3.8E1,1.4E1,1.8E1,2.8E1,2E0,3E0,2E0,1E1,1E1,5E0,9E0,3.1E1,9.4E1,6E0,3.6E1,7E0,4E0,6.7E1,3.2E1,2.62E2,2.18E2,1.6E1,9E0,3.1E1,1.2E1,7E1,2.6E1,3E0,3.5E1,9.5E1,2.5E1,7E0,9E0,3E0,3E0,3E0,1.3E1,2E0,5E0,2E0,1.1E1,9E0,9E0,2E1,1.4E1,3.7E1,3E1,2E0,3E0,3E0,1.1E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"257","size_leaf_vector":"1"}},{"base_weights":[-3.0037984E-3,-1.8668483E-1,8.252813E-2,-3.0221295E-1,-8.045721E-2,2.8112859E-2,2.6864496E-1,-2.4627735E-1,-5.1739824E-1,-1.2941326E-1,1.756439E-1,-3.6991507E-2,1.668445E-1,1.5938838E-1,3.7255943E-1,-2.6824388E-1,7.249569E-2,-6.422169E-1,-2.7702135E-1,-1.4329894E-1,3.3409502E-2,1.3207293E-1,1.9780181E-2,-1.2682103E-1,1.3577339E-2,1.0590123E-1,2.6550445E-1,7.465751E-2,3.4533823E-1,4.2810422E-1,1.23942E-1,-3.597326E-1,-1.9379038E-1,2.6736706E-1,-4.722059E-2,-3.436275E-1,-7.313338E-1,-3.7029317E-1,-3.1719415E-3,-3.4564793E-1,-1.0941271E-1,2.0483144E-1,6.2393073E-2,-1.7001876E-1,2.2002334E-2,-3.3752333E-2,9.974709E-2,6.582261E-2,2.8468782E-1,3.0736956E-1,-1.3622692E-1,-2.0603774E-1,1.0902656E-1,8.913092E-3,4.3197855E-1,4.0543195E-2,4.0069732E-1,-5.203681E-2,2.1429673E-1,-2.5165933E-1,-4.441917E-1,-1.3419221E-1,-2.9838324E-1,1.5636321E-2,2.1839112E-3,2.6110653E-3,-7.3219677E-3,-1.956314E-2,-6.0366746E-3,-1.2006735E-2,-7.691069E-1,-2.288333E-2,-4.754594E-3,-2.2554661E-1,-5.160648E-1,-7.1326405E-2,-2.7693468E-1,4.9293567E-3,1.3568932E-2,-2.775755E-3,9.891716E-2,1.9585189E-1,-1.8114004E-1,2.3135291E-2,-1.8440688E-2,-9.80664E-2,4.6104208E-2,1.0436381E-2,1.6199762E-1,-1.5655994E-1,8.4401675E-2,3.5500604E-1,8.583375E-3,3.5232252E-1,1.4176044E-1,-1.6392294E-2,2.9470807E-2,-1.7446246E-2,-3.7816241E-3,4.081091E-2,2.2643238E-1,7.25631E-2,5.1212966E-1,3.6696994E-1,3.3843905E-2,-6.5445453E-3,2.9234786E-3,4.1513127E-1,9.289851E-2,-3.5004404E-1,-1.3533747E-1,-4.6179685E-1,-3.525433E-3,-1.86478E-1,-3.1962857E-2,-2.5981846E-1,-3.0647E-2,-1.246325E-2,-8.1378025E-1,-2.2018435E-3,-2.7377725E-1,-7.6890797E-3,-5.78126E-1,-1.351735E-1,4.2048017E-3,-3.5755697E-1,-7.572856E-2,-2.3942327E-3,1.3558853E-1,-4.0873927E-3,1.9604621E-2,-1.9396794E-1,1.3402833E-1,2.6099501E-2,-9.167435E-3,-6.779564E-2,-3.878828E-1,1.8858656E-1,-6.004277E-2,-4.0800568E-2,2.0192267E-1,2.6984507E-1,1.2357413E-1,3.485E-3,-3.7052572E-1,1.00050226E-1,-1.3779944E-1,8.5523175E-3,1.9513149E-2,4.1213793E-1,1.6622715E-1,-3.2592246E-3,2.5979263E-1,-4.3318565E-3,6.733524E-3,-3.468078E-2,7.602844E-2,1.565484E-2,1.5804948E-1,-4.113506E-3,7.0674084E-3,3.3698633E-2,1.7539496E-2,-7.0377113E-3,3.7672833E-1,2.4690084E-2,6.7266584E-3,-7.6122896E-4,8.647258E-3,-5.5760755E-3,-1.7928919E-2,-7.547164E-3,1.0617796E-3,-1.6907496E-2,-2.5869371E-2,6.5727904E-3,-9.348191E-3,-3.7436732E-3,1.7730772E-3,-9.251057E-3,-2.013212E-2,-3.8688235E-2,-9.185458E-3,-1.4394353E-2,-2.990349E-4,-2.9052375E-2,-1.1715135E-2,-5.296727E-3,-1.3495288E-2,2.3503215E-3,-7.9442365E-3,-1.7568396E-2,-1.4430562E-3,-4.156557E-4,-1.534148E-2,2.2860293E-3,8.382461E-3,-1.3366394E-2,-7.3135686E-3,1.6590474E-2,6.915E-4,-8.9517026E-4,5.650771E-3,-5.894373E-3,2.5849822E-3,-3.114661E-2,-9.304757E-3,6.3302284E-3,2.292561E-2,-1.1597383E-3,-1.4292161E-2,-4.63868E-3,4.420652E-3,1.4157018E-2,3.6681371E-3,5.0825668E-3,1.550178E-2,-1.0482221E-3,7.1437946E-3,-2.3339499E-2,-4.3759514E-3,6.677233E-4,6.0169105E-3,-1.9404516E-2,2.7697352E-3,2.0108808E-2,4.0043076E-3,1.342692E-2,4.157081E-3,2.8109292E-3,-8.246508E-3,4.688672E-3,1.455067E-2,2.4265978E-3,-5.7874694E-3,4.599949E-3,-3.6658584E-3,-2.3408819E-3,8.988547E-3,1.7532725E-2,-1.5103965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,99,-1,101,-1,103,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,-1,117,-1,-1,119,121,123,125,-1,-1,-1,127,129,131,-1,133,135,137,139,141,143,145,147,-1,149,151,-1,153,-1,-1,155,157,159,161,163,-1,-1,-1,165,167,169,171,173,-1,175,177,179,-1,-1,181,-1,183,-1,185,187,189,191,193,-1,195,-1,-1,197,199,201,-1,203,205,207,209,211,213,215,217,-1,219,221,223,-1,-1,225,227,229,231,-1,-1,233,235,-1,237,-1,-1,-1,-1,-1,239,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.451587E1,8.541735E0,1.5173716E1,3.8927746E0,4.598991E0,1.0492365E1,3.789402E0,1.8890724E0,1.8623314E0,3.8031921E0,6.1390495E-1,3.594899E0,2.2025795E0,2.6054187E0,2.335966E0,1.6332016E0,4.3255225E-1,9.1950417E-1,4.5664597E-1,2.0398054E0,0E0,2.508496E-1,0E0,1.8359575E0,2.0711086E0,1.6364963E0,2.4132843E0,1.1415422E0,5.838485E-1,1.4473972E0,5.3586835E-1,9.0139866E-1,8.3265543E-1,1.02991164E-1,1.5345699E-1,1.6565895E-1,3.7707138E-1,5.563936E-1,0E0,7.636967E-1,1.6455033E0,1.9495893E-1,1.325728E-1,9.347143E-1,1.2949163E0,1.6890957E0,9.9904454E-1,7.982294E-1,2.1356153E-1,9.029665E-1,5.3457177E-1,2.5338876E-1,8.2368624E-1,0E0,8.846798E-1,0E0,1.3048992E0,1.3879582E-1,4.9781597E-1,5.362849E-1,3.5743904E-1,4.8044765E-1,5.540385E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3779755E-1,0E0,0E0,2.1706581E-1,2.671008E-1,1.0325724E0,7.515645E-1,0E0,0E0,0E0,1.2963843E-1,5.141299E-1,8.9523077E-1,0E0,5.053327E-1,1.5703824E0,2.2345617E0,7.440765E-1,4.0590453E-1,7.7805156E-1,6.2576115E-1,2.0891714E-1,0E0,1.0446291E0,4.9723947E-1,0E0,1.4855826E-1,0E0,0E0,1.8076497E-1,2.554809E-1,1.0974884E-1,4.6460295E-1,6.958046E-1,0E0,0E0,0E0,1.9665384E-1,1.6465323E-1,2.3280144E-1,1.2841037E-1,3.4822178E-1,0E0,4.28015E-1,1.18693754E-1,3.911872E-1,0E0,0E0,4.9886322E-1,0E0,2.4567473E-1,0E0,1.1820698E-1,3.0869126E-1,8.5774106E-1,3.2524824E-1,2.595496E-1,0E0,6.656486E-2,0E0,0E0,6.3205576E-1,2.428095E-1,2.2887585E-1,0E0,1.3010714E0,7.8888917E-1,9.055276E-1,7.399832E-1,5.221664E-1,1.8894958E-1,2.702613E-1,4.0528953E-1,0E0,2.678442E-1,4.6039426E-1,7.2057647E-1,0E0,0E0,7.380743E-1,2.2492725E-1,1.7496414E-1,1.2069428E-1,0E0,0E0,1.8776427E-1,1.92922E-1,0E0,2.3106766E-1,0E0,0E0,0E0,0E0,0E0,5.440769E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,70,70,73,73,74,74,75,75,76,76,80,80,81,81,82,82,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,96,96,99,99,100,100,101,101,102,102,103,103,107,107,108,108,109,109,110,110,111,111,113,113,114,114,115,115,118,118,120,120,122,122,123,123,124,124,125,125,126,126,128,128,131,131,132,132,133,133,135,135,136,136,137,137,138,138,139,139,140,140,141,141,142,142,144,144,145,145,146,146,149,149,150,150,151,151,152,152,155,155,156,156,158,158,164,164],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,100,-1,102,-1,104,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,-1,118,-1,-1,120,122,124,126,-1,-1,-1,128,130,132,-1,134,136,138,140,142,144,146,148,-1,150,152,-1,154,-1,-1,156,158,160,162,164,-1,-1,-1,166,168,170,172,174,-1,176,178,180,-1,-1,182,-1,184,-1,186,188,190,192,194,-1,196,-1,-1,198,200,202,-1,204,206,208,210,212,214,216,218,-1,220,222,224,-1,-1,226,228,230,232,-1,-1,234,236,-1,238,-1,-1,-1,-1,-1,240,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,7.883E3,4.0844156E7,1.5560952E0,8.9655056E5,1.8108038E5,3.689E3,7.2456576E8,1.3144558E4,1.3821641E0,7.805608E4,5.314225E2,3.7002478E3,2.8839298E7,2.222456E0,9E0,2.8035452E11,1.9469026E0,7.2210526E-1,3.3409502E-2,9.6755E4,1.9780181E-2,1.6006084E3,1.1032986E8,4.5723195E6,3.7832818E0,1.4702222E0,4.5575E4,1.059448E7,1.4122614E2,1.3878379E0,7.5891815E6,1E0,1.84442E5,1.710643E0,3.6873734E5,9.9625344E5,-3.1719415E-3,4.769198E-1,7.218466E7,3.426E3,1.249246E6,6E0,2.1934064E7,3.3926086E2,1.22871354E-1,2.86968E5,2.1383E4,1E0,9.757E3,2.0967E4,1.6195753E9,8.913092E-3,5.0279167E1,4.0543195E-2,1.6828056E1,1E0,5.7719237E-1,9.417789E1,2.7628E9,1.583E3,8.132321E0,1.5636321E-2,2.1839112E-3,2.6110653E-3,-7.3219677E-3,-1.956314E-2,-6.0366746E-3,-1.2006735E-2,1.0198864E0,-2.288333E-2,-4.754594E-3,1.216918E7,8.22E2,4.9885715E2,6.513793E1,4.9293567E-3,1.3568932E-2,-2.775755E-3,3.1415796E0,4.7E2,1.4100353E0,2.3135291E-2,9.908038E1,9.161318E0,7.2530017E0,2.653986E6,1.6521739E0,5E0,4.5158855E6,3.3820656E7,8.583375E-3,2E0,8.677148E2,-1.6392294E-2,1.1668591E6,-1.7446246E-2,-3.7816241E-3,1.067536E3,1.1795429E9,7.57657E4,3.275343E6,7.549744E7,3.3843905E-2,-6.5445453E-3,2.9234786E-3,4.1485362E5,2.1E1,1.1057693E0,4.222002E8,1.0508E5,-3.525433E-3,2.08E2,1.752512E2,4.7121394E-2,-3.0647E-2,-1.246325E-2,6.6E1,-2.2018435E-3,1.6988909E3,-7.6890797E-3,4.22088E0,1.2737473E1,5.6812387E0,5.377837E6,2.7504245E10,-2.3942327E-3,2.4343E5,-4.0873927E-3,1.9604621E-2,3.983889E3,2.3E1,1.0156423E-1,-9.167435E-3,1.365493E6,3.0805944E7,8.7757526E2,5.097268E7,1.2677199E7,2.1196338E4,3.3968747E-1,2.14E2,3.485E-3,4.38E2,2.6883545E2,2.148972E5,8.5523175E-3,1.9513149E-2,3.1984328E5,2.5737E4,1.3E1,2.657E3,-4.3318565E-3,6.733524E-3,9.173084E2,5.9767612E7,1.565484E-2,4.3467668E4,-4.113506E-3,7.0674084E-3,3.3698633E-2,1.7539496E-2,-7.0377113E-3,2.2893274E1,2.4690084E-2,6.7266584E-3,-7.6122896E-4,8.647258E-3,-5.5760755E-3,-1.7928919E-2,-7.547164E-3,1.0617796E-3,-1.6907496E-2,-2.5869371E-2,6.5727904E-3,-9.348191E-3,-3.7436732E-3,1.7730772E-3,-9.251057E-3,-2.013212E-2,-3.8688235E-2,-9.185458E-3,-1.4394353E-2,-2.990349E-4,-2.9052375E-2,-1.1715135E-2,-5.296727E-3,-1.3495288E-2,2.3503215E-3,-7.9442365E-3,-1.7568396E-2,-1.4430562E-3,-4.156557E-4,-1.534148E-2,2.2860293E-3,8.382461E-3,-1.3366394E-2,-7.3135686E-3,1.6590474E-2,6.915E-4,-8.9517026E-4,5.650771E-3,-5.894373E-3,2.5849822E-3,-3.114661E-2,-9.304757E-3,6.3302284E-3,2.292561E-2,-1.1597383E-3,-1.4292161E-2,-4.63868E-3,4.420652E-3,1.4157018E-2,3.6681371E-3,5.0825668E-3,1.550178E-2,-1.0482221E-3,7.1437946E-3,-2.3339499E-2,-4.3759514E-3,6.677233E-4,6.0169105E-3,-1.9404516E-2,2.7697352E-3,2.0108808E-2,4.0043076E-3,1.342692E-2,4.157081E-3,2.8109292E-3,-8.246508E-3,4.688672E-3,1.455067E-2,2.4265978E-3,-5.7874694E-3,4.599949E-3,-3.6658584E-3,-2.3408819E-3,8.988547E-3,1.7532725E-2,-1.5103965E-3],"split_indices":[19,55,2,44,41,27,27,2,7,51,33,27,4,51,44,53,3,30,52,57,0,28,0,54,7,27,52,52,2,12,55,52,44,80,28,52,27,27,0,26,44,2,1,3,5,51,37,7,9,6,9,10,7,0,55,0,34,109,41,51,12,2,52,0,0,0,0,0,0,0,55,0,0,44,11,51,55,0,0,0,53,1,37,0,55,53,52,1,52,8,46,7,0,16,54,0,27,0,0,51,5,27,28,7,0,0,0,27,3,53,7,28,0,2,51,37,0,0,8,0,51,0,52,53,56,28,5,0,28,0,0,27,0,37,0,9,5,51,44,9,32,37,0,0,0,4,32,0,0,32,9,8,2,0,0,51,44,0,32,0,0,0,0,0,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.195E3,6.97E2,1.498E3,3.33E2,3.64E2,1.16E3,3.38E2,2.66E2,6.7E1,3.06E2,5.8E1,7.9E2,3.7E2,1.66E2,1.72E2,2.49E2,1.7E1,4.3E1,2.4E1,3.02E2,4E0,5.1E1,7E0,2.84E2,5.06E2,2.3E2,1.4E2,1.15E2,5.1E1,1.4E2,3.2E1,1.1E2,1.39E2,6E0,1.1E1,1.1E1,3.2E1,1.6E1,8E0,4.2E1,2.6E2,2.4E1,2.7E1,2.2E2,6.4E1,3.27E2,1.79E2,1.89E2,4.1E1,1.27E2,1.3E1,1.2E1,1.03E2,2E1,3.1E1,6E0,1.34E2,1.1E1,2.1E1,5E1,6E1,9E1,4.9E1,4E0,2E0,6E0,5E0,7E0,4E0,3E0,2.9E1,1E1,6E0,2.6E1,1.6E1,2.13E2,4.7E1,1.3E1,1.1E1,6E0,2.1E1,6E0,2.14E2,4E0,6E1,1.81E2,1.46E2,7.4E1,1.05E2,1.4E1,1.75E2,2.2E1,1.9E1,9.9E1,2.8E1,5E0,8E0,4E0,8E0,6.6E1,3.7E1,6E0,2.5E1,1.24E2,1E1,6E0,5E0,7E0,1.4E1,2.6E1,2.4E1,5.7E1,3E0,5.9E1,3.1E1,4.6E1,3E0,3E0,2.6E1,6E0,2E1,3E0,1.3E1,1.15E2,9.8E1,3.3E1,1.4E1,4E0,1.7E1,3E0,3E0,2.06E2,8E0,4.9E1,1.1E1,1.65E2,1.6E1,6.2E1,8.4E1,5.9E1,1.5E1,2.6E1,7.9E1,7E0,7E0,1.64E2,1.1E1,8E0,1.4E1,7.4E1,2.5E1,1.3E1,1.5E1,4E0,4E0,2.1E1,4.5E1,1.2E1,2.5E1,2E0,4E0,7E0,1.8E1,2E0,1.22E2,4E0,3E0,7E0,7E0,5E0,2.1E1,2E1,4E0,3.4E1,2.3E1,3E0,5.6E1,1.8E1,1.3E1,3.7E1,9E0,2.4E1,2E0,1.7E1,3E0,1E1,3E0,1.05E2,1E1,7.8E1,2E1,3E1,3E0,1.2E1,2E0,7E0,1E1,4.8E1,1.58E2,2E0,6E0,3.4E1,1.5E1,1.1E2,5.5E1,5E0,1.1E1,5.5E1,7E0,7.5E1,9E0,4.1E1,1.8E1,7E0,8E0,9E0,1.7E1,1.5E1,6.4E1,4E0,3E0,4.6E1,1.18E2,4E0,7E0,6.7E1,7E0,8E0,1.7E1,1E1,3E0,5E0,1E1,1.1E1,1E1,3.9E1,6E0,4E0,2.1E1,1.19E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"241","size_leaf_vector":"1"}},{"base_weights":[-4.8184034E-4,-1.767885E-1,8.320908E-2,-2.9805884E-1,-7.731701E-2,7.1601314E-3,2.2254801E-1,-3.5461104E-1,-1.6912518E-1,-1.3502829E-1,1.3053626E-1,-2.531779E-2,1.8393756E-1,1.1250029E-1,2.956777E-1,-3.382885E-1,-7.5507385E-1,-2.1006995E-1,-4.2935666E-2,3.772795E-2,-1.4712663E-1,1.5884745E-1,-7.646442E-3,-3.305916E-2,4.0424515E-2,1.04319625E-1,2.6752537E-1,-6.3795666E-3,1.9272453E-1,-9.371697E-4,3.2841083E-1,-6.6919106E-1,-3.1916246E-1,-4.077979E-2,-9.600876E-3,-2.3000953E-1,1.9335778E-2,1.1477731E-2,-7.5517185E-3,-9.541222E-2,-2.3234618E-1,1.551261E-2,2.1641217E-1,-1.8198903E-1,-2.4446493E-3,-2.2582155E-1,1.2934E-1,3.003657E-1,-4.0952966E-2,1.0010902E-2,-3.509602E-2,1.4153618E-1,3.3158794E-1,-1.6633065E-2,3.455596E-2,3.04567E-1,3.249316E-2,-3.595607E-2,-1.6314732E-2,-3.3232936E-1,-2.5224429E-2,-3.8763672E-1,-1.7997235E-1,-3.3781365E-3,4.9100993E-3,-4.3579047E-3,3.7280368E-3,-1.5816924E-1,-5.3462118E-5,-4.935513E-1,-1.7278728E-1,-1.046008E-3,5.097679E-3,2.2456462E-2,1.8595149E-1,-2.1000561E-1,5.001585E-2,2.519164E-1,-1.4442744E-2,-1.6337123E-2,1.3351627E-3,1.6306569E-1,-8.170637E-2,3.199282E-1,-3.971694E-3,2.0897302E-3,-6.3203676E-3,-1.8106572E-1,-8.707645E-4,1.6657037E-1,-1.5728172E-2,1.6449133E-1,4.1525465E-1,1.22903965E-1,-2.7711414E-2,3.2716298E-1,9.355379E-2,-3.0804542E-1,-5.2349585E-1,-5.1136925E-3,3.852412E-3,-7.173902E-3,-2.3058021E-2,-2.198743E-1,-7.784601E-2,-2.6190054E-1,-5.8647044E-2,-2.3258552E-2,2.386351E-1,-1.4123268E-2,-2.631815E-2,-1.8725872E-1,1.5313771E-2,2.2451767E-1,6.2480148E-2,-1.87654E-1,-2.1719093E-2,-1.0331828E-1,2.619195E-1,-6.1014867E-3,5.3867567E-1,-1.4381588E-1,2.118793E-2,1.7834727E-1,-9.607267E-3,5.6680967E-3,-2.7499056E-1,8.489877E-3,3.676626E-1,-4.240009E-3,-1.21147605E-2,-6.0764417E-2,5.5525344E-2,1.4836177E-1,2.0542035E-2,-2.7906118E-4,9.792948E-3,4.483878E-1,2.533029E-3,1.1028872E-3,8.2149E-3,-1.5749155E-1,2.7670516E-3,2.56693E-1,4.222686E-1,1.5526246E-1,-1.13752574E-1,-1.584017E-2,-7.5885565E-3,-2.646516E-2,-5.335579E-3,8.6272624E-4,-1.0562614E-2,1.8919756E-3,-5.336509E-3,-2.0688903E-3,-1.3895887E-2,-3.771268E-3,8.204138E-3,-4.628127E-4,-1.4473192E-2,1.5325277E-2,-3.3290553E-4,-1.0095617E-2,-1.5610324E-3,6.2823542E-3,1.3055788E-2,1.9897397E-4,9.353352E-3,-7.543867E-3,-2.2513667E-2,-7.3756506E-3,4.0763123E-3,1.5655323E-3,1.5662061E-2,2.6467066E-2,5.170168E-3,-8.424539E-3,2.3497338E-3,-6.6359335E-4,5.4985876E-3,6.978703E-3,1.5843784E-2,-1.7269144E-2,-1.874289E-3,4.4977255E-3,1.7695384E-2,-4.162302E-3,4.0581175E-3,3.7399447E-3,-1.9384446E-3,1.3133359E-2,5.218569E-3,6.8492284E-3,2.1414334E-2,2.493341E-4,-9.631687E-3,1.0055945E-2,2.147439E-2,5.6292606E-3,2.018317E-2,1.8115266E-3,1.4364912E-2,2.2309904E-3,-1.1243539E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,-1,45,47,49,51,53,55,57,59,-1,-1,61,63,65,-1,67,69,71,73,75,77,79,81,83,85,-1,87,89,91,-1,93,95,-1,-1,-1,97,99,101,103,-1,-1,-1,-1,105,107,109,111,-1,-1,-1,113,115,117,119,121,-1,-1,123,125,127,-1,-1,-1,129,131,133,-1,135,137,139,141,143,145,147,149,-1,-1,-1,-1,151,153,155,157,159,161,-1,-1,163,-1,165,167,169,-1,171,173,-1,175,177,179,181,-1,-1,183,-1,185,-1,-1,187,189,191,-1,-1,-1,193,-1,-1,-1,195,-1,197,199,201,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.309602E1,8.686399E0,1.6115805E1,2.3097553E0,4.7918253E0,5.6599E0,4.288378E0,1.2152977E0,5.128274E-1,3.7778335E0,7.64344E-1,5.950619E0,9.903202E-1,2.0612311E0,3.1266518E0,1.1661072E0,4.4175148E-1,3.520906E-1,1.771521E-1,0E0,1.3396187E0,6.567254E-1,0E0,3.770191E0,0E0,6.900281E-1,7.6196E-1,5.826586E-1,8.700247E-1,4.4190842E-1,2.47324E0,1.3845062E-1,7.9663277E-1,0E0,0E0,4.8234105E-1,6.670105E-2,1.529856E-1,0E0,1.1609712E0,1.7193432E0,9.257838E-2,3.9322472E-1,9.27001E-1,2.1002576E0,2.1748891E-1,5.4714966E-1,5.367327E-1,7.51911E-2,0E0,3.9451456E-1,1.2329181E0,4.0702915E-1,0E0,1.7394301E-1,1.2806091E0,0E0,0E0,0E0,7.6255035E-1,1.06386006E-1,3.6542583E-1,2.0701087E-1,0E0,0E0,0E0,0E0,1.1924977E0,4.3757588E-1,1.5333223E-1,7.73298E-1,0E0,0E0,0E0,2.3834825E-1,6.695919E-1,5.46934E-1,3.5100431E0,3.0315883E0,0E0,0E0,4.1060162E-1,4.743271E-1,3.3555555E-1,0E0,0E0,0E0,8.838779E-2,2.2292896E-1,4.137268E-1,0E0,1.1628115E-1,2.3682141E-1,7.057963E-2,2.2866304E-1,1.5537243E0,3.758543E-1,9.928093E-1,4.512534E-1,0E0,0E0,0E0,0E0,1.295917E-1,8.632981E-2,5.4088616E-1,3.7578368E-1,2.749524E-1,1.7444995E-1,0E0,0E0,5.054498E-1,0E0,1.7902017E-1,1.1628497E-1,6.853671E-1,0E0,1.2989454E-1,1.2174627E-1,0E0,2.8411627E-1,1.1842272E0,1.8562679E0,2.0772433E-1,0E0,0E0,1.3339859E-1,0E0,2.4738121E-1,0E0,0E0,1.5826194E-1,9.4087146E-2,3.793198E-1,0E0,0E0,0E0,7.465887E-2,0E0,0E0,0E0,7.321812E-2,0E0,9.858475E-1,6.737156E-1,3.9812165E-1,1.6583571E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,54,54,55,55,59,59,60,60,61,61,62,62,67,67,68,68,69,69,70,70,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,87,87,88,88,89,89,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,103,103,104,104,105,105,106,106,107,107,108,108,111,111,113,113,114,114,115,115,117,117,118,118,120,120,121,121,122,122,123,123,126,126,128,128,131,131,132,132,133,133,137,137,141,141,143,143,144,144,145,145,146,146],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,-1,46,48,50,52,54,56,58,60,-1,-1,62,64,66,-1,68,70,72,74,76,78,80,82,84,86,-1,88,90,92,-1,94,96,-1,-1,-1,98,100,102,104,-1,-1,-1,-1,106,108,110,112,-1,-1,-1,114,116,118,120,122,-1,-1,124,126,128,-1,-1,-1,130,132,134,-1,136,138,140,142,144,146,148,150,-1,-1,-1,-1,152,154,156,158,160,162,-1,-1,164,-1,166,168,170,-1,172,174,-1,176,178,180,182,-1,-1,184,-1,186,-1,-1,188,190,192,-1,-1,-1,194,-1,-1,-1,196,-1,198,200,202,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.382114E0,2.376302E8,1E0,1.4580295E0,2.065904E6,2.1150263E-1,4.7772845E6,2.1340163E8,1.4E1,1E0,6.7652373E3,5.8036E4,7.727022E8,4.6187375E4,2.9143541E-6,1.11E3,4.264897E6,5.59E2,3.772795E-2,4.0844156E7,3.647E3,-7.646442E-3,1.1058E4,4.0424515E-2,1.3314917E0,2.545455E7,4.3531516E2,3.166E3,6E0,1.6828056E1,4.6547272E2,2.496172E3,-4.077979E-2,-9.600876E-3,1.2414683E5,4.01E2,1.96E2,-7.5517185E-3,1.49974E6,3.488E3,2.2376953E1,1E0,6.8656494E5,2.04E5,1E1,6.3844144E-1,1.3992102E3,3.5044186E2,1.0010902E-2,1.2245615E0,2.14099E5,9.3058E4,-1.6633065E-2,2.5040618E7,1E0,3.249316E-2,-3.595607E-2,-1.6314732E-2,6.747114E7,5.2E1,6.9E1,1.305E3,-3.3781365E-3,4.9100993E-3,-4.3579047E-3,3.7280368E-3,1.9E1,2.1169072E3,3.23153E5,3.1699734E2,-1.046008E-3,5.097679E-3,2.2456462E-2,1.535373E6,1E0,3.202814E5,2.97E2,7.805608E4,-1.6337123E-2,1.3351627E-3,5.503109E8,1.5182724E0,8.31E2,-3.971694E-3,2.0897302E-3,-6.3203676E-3,1.0828989E8,5.964E3,6.4683E4,-1.5728172E-2,3.7832818E0,3.5766065E0,6.0607E4,5.141184E2,1.3860265E6,2.64E0,2.5604828E2,3.724138E0,-5.1136925E-3,3.852412E-3,-7.173902E-3,-2.3058021E-2,4E0,1.0376E4,1.2839224E0,1E0,1.9483356E1,3.5E1,-1.4123268E-2,-2.631815E-2,1E0,1.5313771E-2,1.4935554E11,6.923162E6,1.0972E4,-2.1719093E-2,2.778E3,2.37E2,-6.1014867E-3,1.9E1,9.51668E5,4.264897E6,2.4222221E0,-9.607267E-3,5.6680967E-3,1.692E3,8.489877E-3,3.3870136E7,-4.240009E-3,-1.21147605E-2,1.22769325E2,2.5332516E5,2.26182E5,2.0542035E-2,-2.7906118E-4,9.792948E-3,3.652333E-3,2.533029E-3,1.1028872E-3,8.2149E-3,4.274E3,2.7670516E-3,1.0746194E4,8.84E2,4.760537E8,1.399844E8,-1.584017E-2,-7.5885565E-3,-2.646516E-2,-5.335579E-3,8.6272624E-4,-1.0562614E-2,1.8919756E-3,-5.336509E-3,-2.0688903E-3,-1.3895887E-2,-3.771268E-3,8.204138E-3,-4.628127E-4,-1.4473192E-2,1.5325277E-2,-3.3290553E-4,-1.0095617E-2,-1.5610324E-3,6.2823542E-3,1.3055788E-2,1.9897397E-4,9.353352E-3,-7.543867E-3,-2.2513667E-2,-7.3756506E-3,4.0763123E-3,1.5655323E-3,1.5662061E-2,2.6467066E-2,5.170168E-3,-8.424539E-3,2.3497338E-3,-6.6359335E-4,5.4985876E-3,6.978703E-3,1.5843784E-2,-1.7269144E-2,-1.874289E-3,4.4977255E-3,1.7695384E-2,-4.162302E-3,4.0581175E-3,3.7399447E-3,-1.9384446E-3,1.3133359E-2,5.218569E-3,6.8492284E-3,2.1414334E-2,2.493341E-4,-9.631687E-3,1.0055945E-2,2.147439E-2,5.6292606E-3,2.018317E-2,1.8115266E-3,1.4364912E-2,2.2309904E-3,-1.1243539E-2],"split_indices":[19,55,7,109,41,27,37,27,7,0,111,51,1,7,32,41,2,9,0,0,44,2,0,9,0,53,44,4,0,8,34,51,4,0,0,27,0,0,0,1,2,55,66,50,5,3,40,4,4,0,52,2,2,0,5,6,0,0,0,44,8,10,2,0,0,0,0,3,51,28,55,0,0,0,28,72,27,2,27,0,0,5,52,2,0,0,0,44,2,2,0,52,34,1,51,27,53,51,55,0,0,0,0,10,9,52,94,53,3,0,0,109,0,30,28,9,0,0,0,0,3,1,9,52,0,0,2,0,7,0,0,53,27,28,0,0,0,33,0,0,0,28,0,4,10,7,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.241E3,7.21E2,1.52E3,3.24E2,3.97E2,9.84E2,5.36E2,2.24E2,1E2,3.11E2,8.6E1,8.32E2,1.52E2,2.15E2,3.21E2,2.17E2,7E0,7.5E1,2.5E1,3E0,3.08E2,7.9E1,7E0,8.26E2,6E0,7.9E1,7.3E1,8.7E1,1.28E2,3.2E1,2.89E2,1E1,2.07E2,5E0,2E0,6.9E1,6E0,1.8E1,7E0,1.93E2,1.15E2,2.3E1,5.6E1,1.4E2,6.86E2,5E0,7.4E1,6.6E1,7E0,9E0,7.8E1,9.5E1,3.3E1,2E0,3E1,2.74E2,1.5E1,6E0,4E0,1.98E2,9E0,1.5E1,5.4E1,3E0,3E0,7E0,1.1E1,1.16E2,7.7E1,2E1,9.5E1,1.7E1,6E0,4E0,5.2E1,1.25E2,1.5E1,3E1,6.56E2,3E0,2E0,6.4E1,1E1,6.3E1,3E0,4E0,3E0,1.4E1,6.4E1,9.1E1,4E0,1.2E1,2.1E1,1.2E1,1.8E1,2.47E2,2.7E1,1.78E2,2E1,5E0,4E0,6E0,9E0,3.8E1,1.6E1,5.6E1,6E1,7.1E1,6E0,8E0,1.2E1,9.3E1,2E0,3.9E1,1.3E1,1.17E2,8E0,9E0,6E0,1.3E1,1.7E1,1.41E2,5.15E2,6.2E1,2E0,5E0,5E0,1.8E1,4.5E1,8E0,6E0,3.1E1,3.3E1,8.7E1,4E0,3E0,9E0,1.9E1,2E0,5E0,7E0,7E0,1.1E1,1.44E2,1.03E2,2.1E1,6E0,1.36E2,4.2E1,1.7E1,3E0,2E0,3.6E1,4E0,1.2E1,1E1,4.6E1,5.5E1,5E0,6.9E1,2E0,4E0,2E0,7.5E1,1.8E1,1.8E1,2.1E1,1E1,3E0,1.11E2,6E0,7E0,2E0,2E0,4E0,1.5E1,2E0,1.16E2,2.5E1,3.8E2,1.35E2,5.6E1,6E0,3E0,2E0,4E0,4.1E1,2.6E1,5E0,2.6E1,7E0,1.5E1,7.2E1,2E0,1.7E1,2E0,5E0,1.26E2,1.8E1,8E0,9.5E1,1.3E1,8E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"205","size_leaf_vector":"1"}},{"base_weights":[-2.3500046E-3,-1.6358349E-1,7.887699E-2,-1.8786521E-1,2.7394637E-1,-2.9163992E-3,1.8868469E-1,-2.7875698E-1,-7.894744E-2,4.0582456E-2,1.3361457E-1,-4.0427376E-2,1.0838591E-1,1.2718476E-1,3.654708E-1,-2.5219434E-1,-6.296962E-1,-1.6629964E-1,3.0491188E-2,2.635407E-1,-3.986616E-2,-7.4451E-2,1.2529889E-1,2.247914E-2,1.826902E-1,-1.2229991E-2,1.7436326E-1,3.1122893E-1,5.57331E-1,-3.1670198E-1,-1.4262375E-1,-7.36256E-1,-2.3528312E-1,-1.4574634E-1,-3.12041E-1,-4.1319918E-2,1.6345002E-1,1.3146672E-1,1.9409351E-2,-1.1255213E-2,1.4956148E-3,-1.3823012E-1,-2.2979788E-3,1.3505417E-1,-1.1676263E-2,1.145289E-1,-4.735628E-2,1.094674E-1,2.9502898E-1,5.8359127E-2,-1.883431E-1,3.100064E-2,2.2240011E-1,2.502842E-1,4.375266E-1,5.878387E-1,2.088878E-3,-2.655289E-1,-5.176856E-1,6.7997836E-2,-1.9200835E-1,-2.9077482E-1,-8.6536473E-1,2.2935276E-3,-1.6289191E-2,-7.5131655E-2,-1.8865095E-1,-8.352493E-3,-2.0899057E-2,-2.0402197E-2,-2.2668859E-2,9.89459E-2,2.9433715E-1,-1.2982044E-3,7.5318823E-3,-1.0088072E-1,-3.2771528E-1,2.699499E-2,-2.240788E-2,9.650446E-2,2.6113844E-1,7.689478E-2,1.2204332E-2,-8.9065254E-2,4.5574442E-3,1.263432E-1,-1.4014105E-2,2.102373E-1,5.036204E-1,-8.316245E-2,1.4537522E-1,-1.6554207E-2,-1.4516528E-1,5.5460047E-2,-2.2479055E-2,1.994296E-1,4.607902E-1,1.7119218E-3,2.740112E-1,8.687971E-3,4.6713364E-1,3.0217335E-2,1.902264E-2,-2.907589E-1,-1.19082764E-1,-3.3857274E-1,-5.8998585E-1,1.2540327E-2,5.0484505E-3,-1.4315233E-1,-2.6253137E-1,-1.9099077E-2,-5.422477E-3,-1.4861363E-2,-4.177639E-2,-5.890646E-2,-1.3202521E-2,-4.2080933E-1,-1.4442173E-1,-7.769328E-2,7.9490826E-2,-9.07139E-3,1.366854E-1,-1.6575031E-3,1.5584671E-2,2.573712E-1,-1.1317021E-1,-3.787143E-2,-2.1197964E-1,-3.3950236E-2,2.994E-2,1.8247482E-1,7.612122E-2,1.108308E-1,3.439343E-1,1.2651928E-1,-4.5833754E-4,-1.16805054E-1,7.0724315E-3,1.7060764E-1,1.4524943E-4,1.8329322E-2,7.974922E-2,1.2976306E-2,3.0462742E-2,-2.3834072E-3,-2.6871592E-1,2.2455168E-1,-5.1754806E-2,-8.159507E-3,-2.138255E-3,-5.5170305E-2,1.1999174E-1,3.2755712E-1,1.6447826E-1,2.39805E-2,1.3692489E-2,2.9754722E-1,4.5937593E-3,1.0423688E-2,4.9027413E-1,-1.1157766E-2,-1.7989393E-2,3.083493E-3,-9.568034E-3,2.2872041E-3,-1.9923057E-2,-2.9743277E-2,-1.4654291E-2,2.8363445E-3,-8.683605E-3,8.120237E-3,-7.554062E-3,-1.311958E-2,-2.0911156E-3,-3.5331212E-3,3.1884196E-3,-9.041162E-3,-2.3152854E-2,-7.7939783E-3,-7.9427747E-4,-4.9296767E-3,2.424422E-3,-2.7730805E-3,5.83839E-3,4.5869322E-3,-3.207815E-3,6.9423076E-3,-1.906632E-3,2.4952805E-3,2.018532E-2,-9.651501E-3,-4.0008677E-3,-2.4248643E-3,-1.242939E-2,1.7354528E-4,-9.401837E-3,1.3863338E-3,9.864584E-3,4.5766463E-3,-1.171128E-2,8.932803E-3,9.189105E-4,6.8172677E-3,1.9266535E-2,-8.735281E-4,6.761937E-3,-6.6800364E-3,-3.0561115E-4,8.7637175E-3,-6.42266E-3,1.2090449E-2,9.040912E-4,-1.0966421E-2,1.3777051E-3,-1.5712362E-2,-2.9179414E-3,2.2799388E-2,8.831188E-3,3.7308848E-3,-5.443238E-3,8.8507235E-3,-4.4350894E-3,-3.946524E-4,7.896423E-3,8.843427E-3,2.0015558E-2,2.0081396E-3,9.034558E-3,1.4440478E-2,7.0473007E-3,2.3509689E-2,1.0666886E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,-1,-1,75,77,79,-1,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,107,109,111,113,-1,-1,115,117,-1,-1,-1,119,121,123,-1,-1,125,127,-1,129,131,133,135,-1,137,-1,139,-1,141,143,145,147,-1,149,151,-1,153,155,-1,157,-1,159,-1,-1,161,163,165,167,-1,169,171,173,-1,-1,-1,-1,175,-1,177,179,181,183,185,187,-1,-1,189,191,-1,193,195,-1,197,199,201,203,205,-1,207,-1,209,-1,-1,211,-1,-1,213,215,217,219,-1,-1,221,223,225,227,-1,-1,229,-1,-1,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.869426E1,7.8494663E0,1.30866585E1,6.868206E0,3.286749E0,3.4947307E0,6.7161083E0,3.3741856E0,3.0450568E0,0E0,7.543158E-1,3.5390966E0,1.3427572E0,3.0474582E0,1.4916401E0,2.4569511E0,9.1951466E-1,4.847188E-1,1.3625585E0,3.581134E-1,2.3646854E-1,2.3926477E0,4.3482244E-1,6.418288E-1,8.972149E-1,1.4779285E0,2.3759012E0,8.6803627E-1,5.031185E-1,2.1457996E0,1.396451E0,8.9868355E-1,2.6744226E-1,4.6148872E-1,3.1548452E-1,6.9976807E-1,3.9301455E-1,7.182914E-2,0E0,0E0,0E0,1.9202352E0,2.962406E0,4.851526E-1,0E0,2.3016375E-1,3.6087012E-1,5.340848E-1,6.7234015E-1,1.0588354E0,2.0720863E-1,1.1629692E0,1.326374E0,4.3327427E-1,2.0125198E-1,1.4662838E-1,0E0,6.267519E-1,3.700428E-1,3.40652E-1,3.3452892E-1,8.396679E-2,1.5941334E-1,0E0,0E0,1.9708276E-1,9.3524003E-1,0E0,0E0,0E0,5.120909E-1,1.439181E-1,2.767955E-1,0E0,0E0,1.0505605E0,2.4724574E0,0E0,1.9006726E0,1.2980282E-1,2.60558E-1,1.5673769E-1,0E0,3.2442662E-1,0E0,3.7008655E-1,0E0,8.057519E-1,2.4783754E-1,4.939909E-1,8.388486E-1,0E0,9.106344E-2,6.13449E-1,0E0,1.0119419E0,7.978821E-2,0E0,2.906313E-1,0E0,7.898903E-2,0E0,0E0,6.1992836E-1,4.97925E-1,6.1452425E-1,3.224392E-1,0E0,2.5012577E-1,5.3439605E-1,2.401228E-1,0E0,0E0,0E0,0E0,1.4901134E-1,0E0,1.983149E-1,2.8899837E-1,2.473633E-1,2.3068145E-1,7.537753E-2,8.388093E-2,0E0,0E0,2.8039223E-1,5.392158E-1,0E0,3.6189044E-1,1.5432334E0,0E0,7.398909E-2,5.945445E-1,7.428453E-2,1.7264748E-1,7.946956E-2,0E0,1.3771933E-1,0E0,3.8622606E-1,0E0,0E0,2.2980148E-1,0E0,0E0,1.9625278E-1,1.3764328E-1,2.1446812E-1,1.5689334E-1,0E0,0E0,3.6238545E-1,3.7977177E-1,6.709113E-1,7.8746223E-1,0E0,0E0,1.5996599E-1,0E0,0E0,1.066165E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,70,70,71,71,72,72,75,75,76,76,78,78,79,79,80,80,81,81,83,83,85,85,87,87,88,88,89,89,90,90,92,92,93,93,95,95,96,96,98,98,100,100,103,103,104,104,105,105,106,106,108,108,109,109,110,110,115,115,117,117,118,118,119,119,120,120,121,121,122,122,125,125,126,126,128,128,129,129,131,131,132,132,133,133,134,134,135,135,137,137,139,139,142,142,145,145,146,146,147,147,148,148,151,151,152,152,153,153,154,154,157,157,160,160],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,-1,-1,76,78,80,-1,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,108,110,112,114,-1,-1,116,118,-1,-1,-1,120,122,124,-1,-1,126,128,-1,130,132,134,136,-1,138,-1,140,-1,142,144,146,148,-1,150,152,-1,154,156,-1,158,-1,160,-1,-1,162,164,166,168,-1,170,172,174,-1,-1,-1,-1,176,-1,178,180,182,184,186,188,-1,-1,190,192,-1,194,196,-1,198,200,202,204,206,-1,208,-1,210,-1,-1,212,-1,-1,214,216,218,220,-1,-1,222,224,226,228,-1,-1,230,-1,-1,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,4.4839956E3,1E0,1E0,2.46E2,3.422351E6,1.7730331E6,1.2427474E8,3.3030225E2,4.0582456E-2,1.0881593E10,1.681727E6,1.857051E5,3.4453398E-1,1.7027656E3,9.357879E2,4.8083666E-1,1.2342778E1,1.6394367E0,7.203918E9,8.755996E-1,3.5259784E2,3.0161016E7,2.46E2,4.8237036E2,3.9712732E7,7.7344055E4,4.391553E6,8.387235E11,3.0161016E7,2.2668628E7,4.56E2,1.7322648E5,5.1792985E6,5.23896E5,5.2143492E-5,2.160066E9,1E0,1.9409351E-2,-1.1255213E-2,1.4956148E-3,9.669789E0,2.00059E5,3.1517188E2,-1.1676263E-2,1.5435694E9,1.2060912E3,1E0,1.3304372E7,3.1609525E2,7.937392E-3,8.942177E6,6.8545566E0,1.5552E4,1.704E3,1.7398448E7,2.088878E-3,2.046E3,6.5029144E5,2.6713815E-1,1.2199979E10,4.8658E5,9.61E2,2.2935276E-3,-1.6289191E-2,5.159139E8,1.5494787E10,-8.352493E-3,-2.0899057E-2,-2.0402197E-2,6.604754E5,2.695E3,2.091623E5,-1.2982044E-3,7.5318823E-3,7E0,2E0,2.699499E-2,9.256843E3,1.0620689E0,2.298425E6,4.944E3,1.2204332E-2,8.36E2,4.5574442E-3,1E0,-1.4014105E-2,2.072E3,1.64935E5,3.05E3,2.7194722E6,-1.6554207E-2,3.794679E8,3.3030225E2,-2.2479055E-2,1.5851064E0,2.7148398E7,1.7119218E-3,2.0494426E7,8.687971E-3,7.130317E7,3.0217335E-2,1.902264E-2,2.2444444E0,2.9311974E8,1.9166666E-1,1.527638E12,1.2540327E-2,2.6281825E-1,1.3617021E0,2.7225998E-1,-1.9099077E-2,-5.422477E-3,-1.4861363E-2,-4.177639E-2,9.544947E6,-1.3202521E-2,8.182648E6,4.8E1,1.663473E6,3.56E2,1.5E1,2.160255E9,-1.6575031E-3,1.5584671E-2,3.192E3,3.75E1,-3.787143E-2,2.739E3,4.004366E1,2.994E-2,9.61E2,6.6315195E6,1.8472951E6,7.3E1,2.53E2,-4.5833754E-4,6.748383E8,7.0724315E-3,6.310469E6,1.4524943E-4,1.8329322E-2,4.9355304E2,1.2976306E-2,3.0462742E-2,7.549744E7,1.23E3,1.27365E4,2.1690162E7,-8.159507E-3,-2.138255E-3,2.9104478E0,1.431413E-1,6.072483E2,1E0,2.39805E-2,1.3692489E-2,2E0,4.5937593E-3,1.0423688E-2,1.4440248E9,-1.1157766E-2,-1.7989393E-2,3.083493E-3,-9.568034E-3,2.2872041E-3,-1.9923057E-2,-2.9743277E-2,-1.4654291E-2,2.8363445E-3,-8.683605E-3,8.120237E-3,-7.554062E-3,-1.311958E-2,-2.0911156E-3,-3.5331212E-3,3.1884196E-3,-9.041162E-3,-2.3152854E-2,-7.7939783E-3,-7.9427747E-4,-4.9296767E-3,2.424422E-3,-2.7730805E-3,5.83839E-3,4.5869322E-3,-3.207815E-3,6.9423076E-3,-1.906632E-3,2.4952805E-3,2.018532E-2,-9.651501E-3,-4.0008677E-3,-2.4248643E-3,-1.242939E-2,1.7354528E-4,-9.401837E-3,1.3863338E-3,9.864584E-3,4.5766463E-3,-1.171128E-2,8.932803E-3,9.189105E-4,6.8172677E-3,1.9266535E-2,-8.735281E-4,6.761937E-3,-6.6800364E-3,-3.0561115E-4,8.7637175E-3,-6.42266E-3,1.2090449E-2,9.040912E-4,-1.0966421E-2,1.3777051E-3,-1.5712362E-2,-2.9179414E-3,2.2799388E-2,8.831188E-3,3.7308848E-3,-5.443238E-3,8.8507235E-3,-4.4350894E-3,-3.946524E-4,7.896423E-3,8.843427E-3,2.0015558E-2,2.0081396E-3,9.034558E-3,1.4440478E-2,7.0473007E-3,2.3509689E-2,1.0666886E-2],"split_indices":[19,51,107,109,10,9,27,44,51,0,12,27,27,41,4,54,41,53,41,12,41,51,44,10,4,44,32,27,30,44,44,0,27,44,9,41,5,8,0,0,0,53,5,54,0,7,51,64,44,51,41,46,34,9,28,49,0,2,27,56,5,1,12,0,0,7,30,0,0,0,27,2,27,0,0,3,16,0,4,53,27,2,0,8,0,112,0,28,1,10,47,0,7,51,0,52,47,0,9,0,7,0,0,55,7,57,30,0,37,52,33,0,0,0,0,9,0,44,8,28,0,3,12,0,0,1,4,0,28,52,0,12,27,27,0,2,0,7,0,44,0,0,51,0,0,7,2,32,44,0,0,53,37,4,109,0,0,6,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.189E3,7.33E2,1.456E3,6.95E2,3.8E1,8.35E2,6.21E2,3.78E2,3.17E2,6E0,3.2E1,6.25E2,2.1E2,4.62E2,1.59E2,3.53E2,2.5E1,1.76E2,1.41E2,1.8E1,1.4E1,5.19E2,1.06E2,9.8E1,1.12E2,1.17E2,3.45E2,1.26E2,3.3E1,2.21E2,1.32E2,1.9E1,6E0,1.56E2,2E1,9.2E1,4.9E1,1.1E1,7E0,3E0,1.1E1,2.75E2,2.44E2,1.04E2,2E0,4.2E1,5.6E1,6.9E1,4.3E1,8.4E1,3.3E1,8.7E1,2.58E2,8.7E1,3.9E1,3.1E1,2E0,1.78E2,4.3E1,2.5E1,1.07E2,5E0,1.4E1,2E0,4E0,6E1,9.6E1,1.2E1,8E0,3E0,8.9E1,3.4E1,1.5E1,2E0,9E0,2.31E2,4.4E1,7E0,2.37E2,8.1E1,2.3E1,3.5E1,7E0,4.4E1,1.2E1,6.7E1,2E0,3.2E1,1.1E1,3.2E1,5.2E1,5E0,2.8E1,8.4E1,3E0,2.37E2,2.1E1,9E0,7.8E1,5E0,3.4E1,1.9E1,1.2E1,1.51E2,2.7E1,1.4E1,2.9E1,5E0,2E1,6.5E1,4.2E1,2E0,3E0,2E0,1.2E1,5.7E1,3E0,1.4E1,8.2E1,5.8E1,3.1E1,9E0,2.5E1,2E0,1.3E1,7E0,2.24E2,7E0,3.7E1,2.34E2,3E0,1.4E1,6.7E1,9E0,1.4E1,2.2E1,1.3E1,4E1,4E0,4.9E1,1.8E1,1.2E1,2E1,6E0,5E0,2.3E1,9E0,3.7E1,1.5E1,2E1,8E0,3.1E1,5.3E1,4.9E1,1.88E2,1.3E1,8E0,6.8E1,1E1,4E0,3E1,1.09E2,4.2E1,9E0,1.8E1,3E0,1.1E1,2.2E1,7E0,1.6E1,4E0,4E0,6.1E1,3.7E1,5E0,5E1,7E0,5E0,9E0,6.7E1,1.5E1,4.7E1,1.1E1,8E0,2.3E1,3E0,6E0,2.3E1,2E0,4E0,3E0,4.3E1,1.81E2,1.1E1,2.6E1,1.93E2,4.1E1,3E0,1.1E1,6.3E1,4E0,4E0,5E0,5E0,9E0,3E0,1.9E1,3.1E1,9E0,4.6E1,3E0,4E0,1.6E1,2E0,2.1E1,6E0,3E0,2E0,3.5E1,5E0,1E1,4E0,2.7E1,1.6E1,3.7E1,2.4E1,2.5E1,4.3E1,1.45E2,5.8E1,1E1,2.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[3.8062595E-4,5.320934E-2,-1.8402642E-1,-6.3878214E-3,2.0165795E-1,-2.988681E-1,-3.565619E-2,-6.94921E-2,9.9551484E-2,2.8959596E-1,-2.3549464E-2,-3.2098642E-1,5.048746E-2,-1.05141334E-1,7.7161215E-2,-1.03751816E-1,2.8222954E-2,5.468629E-2,1.9486056E-1,4.1115054E-1,1.8879957E-1,-1.046261E-1,2.6640886E-1,-2.937427E-1,-6.264132E-1,1.8590374E-1,-2.847474E-3,-8.240125E-2,-3.2378036E-1,1.4562353E-1,-2.100684E-1,-1.7385851E-1,-6.5805376E-2,1.1904589E-1,-2.431684E-2,-1.8484356E-1,7.9476506E-2,1.4417808E-1,3.093738E-1,4.5253307E-2,3.809512E-1,6.032607E-2,2.4861926E-1,-1.8608294E-1,1.1738511E-2,1.5596017E-1,3.814959E-1,-5.5549914E-1,-2.7847683E-1,-7.986653E-1,-3.2714236E-1,1.3811496E-3,1.2361635E-2,-9.79707E-2,8.078431E-2,-2.246061E-2,-1.7560522E-1,9.198135E-2,3.3829582E-1,-3.941216E-3,-1.8136108E-2,-6.0286883E-2,-2.0370364E-1,1.7059283E-2,-1.2558019E-1,8.350463E-2,2.7392507E-1,-7.5134295E-3,-1.6132567E-2,4.9115256E-3,-2.3611546E-1,-1.2716645E-2,9.11941E-2,9.8002724E-2,3.0553493E-1,3.474991E-1,7.402835E-4,4.3483827E-1,2.2189078E-1,-6.959066E-3,9.0361774E-2,2.030809E-1,4.072561E-1,-1.5747795E-1,-2.5744453E-2,8.297223E-2,-1.6998851E-1,2.6188323E-1,1.8387126E-4,2.3640838E-2,7.5255516E-3,-9.041922E-3,-6.066842E-1,-3.9714274E-1,-2.5146008E-1,-1.7087974E-2,-4.0616985E-2,-1.8830594E-2,-5.078919E-3,-1.2509735E-1,-3.626798E-2,-2.0948418E-3,6.0763797E-3,-1.0639268E-2,7.7625686E-5,1.992049E-1,3.5814535E-2,4.1609326E-1,1.8009467E-3,-1.4777297E-1,1.4624155E-1,-2.9101476E-1,-1.5892892E-1,-4.1301902E-2,1.4956114E-1,-1.6278535E-1,6.6159554E-2,1.1390679E-1,-4.7596246E-2,1.7294865E-2,7.1815634E-3,-5.7686806E-2,5.700172E-2,-1.7960206E-1,-2.2089623E-2,1.2011393E-1,-6.007854E-2,1.2690161E-1,-9.480863E-2,4.1749678E-3,3.478161E-1,1.6937912E-2,9.697394E-2,5.916851E-1,3.8236317E-1,-6.9459258E-3,2.5646785E-1,1.1335479E-1,-1.0464519E-2,2.2101082E-1,-7.438816E-2,3.1307933E-3,4.450625E-1,-8.8116124E-2,-3.1442684E-1,3.6303904E-2,1.8663807E-2,-1.107575E-2,-7.33843E-5,3.6850872E-3,1.4832077E-2,-3.026159E-2,-1.085906E-2,-2.383981E-1,-5.041106E-1,4.6816873E-3,-2.5784266E-1,-1.0877168E-1,-1.1330668E-2,1.5253128E-2,-4.798118E-3,1.127554E-2,8.9613954E-4,-2.4807267E-3,9.596494E-2,2.1161735E-2,5.588139E-3,-4.1474835E-3,-1.8574681E-2,1.4335625E-2,-2.0396083E-3,-1.0564093E-2,-2.8080083E-2,-5.427615E-3,-1.0834153E-2,-3.110297E-3,2.549402E-3,8.464849E-3,-6.212688E-3,-9.577204E-3,-2.5550344E-3,-1.8139195E-3,7.140646E-3,8.08185E-3,-1.3805854E-3,-7.1589057E-3,4.0423167E-3,-7.281036E-3,6.407991E-4,-2.18347E-3,4.6998844E-3,-4.04796E-3,-1.0646837E-2,3.4395452E-3,8.929996E-3,-5.109601E-3,4.9133413E-3,2.812256E-3,8.437351E-3,2.0857637E-3,-2.1995949E-2,5.112498E-3,1.7533077E-2,8.861058E-3,-1.5437362E-3,1.0137083E-3,2.8893536E-2,1.49163E-2,2.5529394E-2,-2.9439022E-4,1.2670473E-2,2.3822868E-3,9.412475E-3,9.0596E-3,2.97457E-2,4.168699E-3,-7.216204E-3,6.630461E-3,2.1037046E-2,-5.0151036E-3,7.2653648E-3,-2.315346E-3,-1.6464988E-2,-6.654608E-4,5.6925314E-3,-1.69544E-2,-9.724538E-4,-1.0068741E-2,-2.673233E-2,-5.3971354E-3,-1.23677915E-2,1.8402131E-4,-5.8965585E-3,3.4487266E-3,-2.6882763E-3,5.9899455E-4,6.8918234E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,81,83,85,87,89,91,93,95,97,-1,-1,99,101,-1,103,105,107,-1,-1,109,111,113,115,117,119,121,-1,-1,123,-1,125,127,129,131,-1,133,135,-1,137,139,141,143,-1,145,147,149,-1,-1,-1,-1,151,153,155,-1,-1,-1,-1,157,159,-1,-1,-1,-1,161,163,165,-1,167,169,171,173,175,177,179,181,183,185,-1,-1,187,189,191,-1,193,195,197,199,-1,201,-1,203,205,207,-1,209,211,-1,213,215,-1,217,219,221,223,-1,-1,-1,-1,-1,-1,-1,225,227,-1,229,231,-1,233,-1,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1042742E1,1.4855597E1,8.1851425E0,8.022501E0,9.532684E0,2.1102676E0,1.66217E0,2.520391E0,1.9050188E0,4.1616573E0,3.225076E0,1.9378414E0,2.7270585E-1,6.24635E-1,1.6234076E0,1.470346E0,9.3817127E-1,1.8317175E0,7.853403E-1,2.3912048E0,1.4528828E0,1.0164045E0,3.2183814E-1,7.8834724E-1,7.6972294E-1,9.999734E-2,0E0,3.1279963E-1,2.1929133E-1,6.510167E-1,3.4011453E-1,6.486831E-1,1.800161E0,3.7345707E-1,6.946965E-1,4.6176016E-1,1.1994805E0,7.225785E-1,4.7469568E-1,0E0,1.1892071E0,4.1247076E-1,8.564234E-1,5.9985924E-1,5.966335E-1,2.7245954E-1,3.3717966E-1,9.73649E-2,6.2147903E-1,1.5914583E-1,1.2510449E-1,0E0,0E0,1.7957962E-1,8.088297E-2,0E0,8.465445E-2,3.1205302E-1,3.085823E-1,0E0,0E0,7.804819E-1,5.557866E-1,1.1893208E0,1.5159822E0,2.4456659E-1,1.0140592E-1,3.9172143E-1,0E0,0E0,2.681147E-1,0E0,1.1884067E0,4.57794E-1,1.6635251E-1,2.171545E-1,0E0,6.8735313E-1,5.5843306E-1,0E0,4.3277386E-1,5.2886534E-1,3.271327E-1,6.2546945E-1,0E0,4.9212697E-1,1.6299567E-1,9.1958284E-2,0E0,0E0,0E0,0E0,6.611252E-2,5.590439E-1,4.4794846E-1,0E0,0E0,0E0,0E0,1.2808466E-1,1.2732336E-1,0E0,0E0,0E0,0E0,1.5604472E-1,2.0146264E-1,1.0752833E-1,0E0,4.0471E-1,4.3801537E-1,8.236728E-1,2.976823E-1,2.9002354E-1,5.298811E-1,9.0579987E-1,3.4673253E-1,4.6255195E-1,1.9566959E-1,0E0,0E0,5.061072E-1,2.6606387E-1,8.922225E-2,0E0,7.57874E-1,4.0380675E-1,2.5807512E-1,6.2835133E-1,0E0,1.230669E-1,0E0,7.1388245E-2,6.741104E-1,6.399727E-1,0E0,2.2944617E-1,2.8765845E-1,0E0,7.5263405E-1,1.1372492E-1,0E0,7.8258514E-2,2.5995922E-1,2.1479428E-1,1.3868463E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.233114E-1,4.281621E-1,0E0,3.564558E-1,1.6768903E-1,0E0,1.0000256E-1,0E0,0E0,0E0,0E0,1.00388125E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,53,53,54,54,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,72,72,73,73,74,74,75,75,77,77,78,78,80,80,81,81,82,82,83,83,85,85,86,86,87,87,92,92,93,93,94,94,99,99,100,100,105,105,106,106,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,121,121,122,122,123,123,125,125,126,126,127,127,128,128,130,130,132,132,133,133,134,134,136,136,137,137,139,139,140,140,142,142,143,143,144,144,145,145,153,153,154,154,156,156,157,157,159,159,164,164],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,82,84,86,88,90,92,94,96,98,-1,-1,100,102,-1,104,106,108,-1,-1,110,112,114,116,118,120,122,-1,-1,124,-1,126,128,130,132,-1,134,136,-1,138,140,142,144,-1,146,148,150,-1,-1,-1,-1,152,154,156,-1,-1,-1,-1,158,160,-1,-1,-1,-1,162,164,166,-1,168,170,172,174,176,178,180,182,184,186,-1,-1,188,190,192,-1,194,196,198,200,-1,202,-1,204,206,208,-1,210,212,-1,214,216,-1,218,220,222,224,-1,-1,-1,-1,-1,-1,-1,226,228,-1,230,232,-1,234,-1,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.0323588E3,1E0,5.0939494E5,2.8839298E7,4.3525005E3,1.067536E3,2.2292539E8,2.3920168E6,6.2686886E-5,6.082581E3,1.1855755E8,1.1708007E7,8.3510876E9,1.0828989E8,7.805608E4,6.14E2,1.3984146E2,4.6547272E2,3.3820656E7,8.5518695E4,3.8E1,1.1803382E4,5.85E2,2.8316305E0,7.023838E3,-2.847474E-3,1E0,6.567044E2,6.5317163E3,2.603E3,1E0,4.911E3,5.964E3,1E0,2.127E3,1.2E1,1E0,2E0,4.5253307E-2,1.778482E7,6.131E3,2.6422684E9,4.8916136E7,2.3580047E10,6.134093E7,6.971004E7,8E-3,8.857896E4,1.2E1,6.6992125E4,1.3811496E-3,1.2361635E-2,1.9505986E6,4.273439E2,-2.246061E-2,2.8373447E3,3.16617E5,5.9767612E7,-3.941216E-3,-1.8136108E-2,2.97E2,6.737088E7,7.68E2,1.2905E4,4.21E2,1.101214E6,8.755996E-1,-1.6132567E-2,4.9115256E-3,1.4813511E6,-1.2716645E-2,1E0,1.7560976E0,2.006639E0,7.6842415E6,7.402835E-4,8.901186E0,2.979E3,-6.959066E-3,1.4521701E2,5.794597E7,2.1384616E0,1.13808104E8,-2.5744453E-2,7.3610186E3,3.8085933E3,3.224158E6,1.8387126E-4,2.3640838E-2,7.5255516E-3,-9.041922E-3,1.3085492E-1,4.4365574E-4,3.0062932E4,-1.7087974E-2,-4.0616985E-2,-1.8830594E-2,-5.078919E-3,3.1110662E7,6.788246E0,-2.0948418E-3,6.0763797E-3,-1.0639268E-2,7.7625686E-5,2.5482938E0,5E0,6.2277466E2,1.8009467E-3,3.09627E5,5.37798E5,1.3027623E-7,1.9642437E1,3.422351E6,1.8019372E5,1E0,2.2783158E5,1.3933473E5,1.5789307E8,1.7294865E-2,7.1815634E-3,4.888E3,1.2561812E6,8.3933276E-1,-2.2089623E-2,3.9287415E-1,3.3E1,2.4738782E2,1.07E2,4.1749678E-3,5.2369475E6,1.6937912E-2,9.8E1,9.833887E-1,2.4345527E3,-6.9459258E-3,2.0802219E8,1.2799757E9,-1.0464519E-2,2.14099E5,3.5985E4,3.1307933E-3,8.548412E-2,2.6037296E-7,1E0,1.2679E4,1.8663807E-2,-1.107575E-2,-7.33843E-5,3.6850872E-3,1.4832077E-2,-3.026159E-2,-1.085906E-2,1.2866874E3,2.0155435E2,4.6816873E-3,1.988805E8,1.6649964E5,-1.1330668E-2,1.1789883E0,-4.798118E-3,1.127554E-2,8.9613954E-4,-2.4807267E-3,8.631E3,2.1161735E-2,5.588139E-3,-4.1474835E-3,-1.8574681E-2,1.4335625E-2,-2.0396083E-3,-1.0564093E-2,-2.8080083E-2,-5.427615E-3,-1.0834153E-2,-3.110297E-3,2.549402E-3,8.464849E-3,-6.212688E-3,-9.577204E-3,-2.5550344E-3,-1.8139195E-3,7.140646E-3,8.08185E-3,-1.3805854E-3,-7.1589057E-3,4.0423167E-3,-7.281036E-3,6.407991E-4,-2.18347E-3,4.6998844E-3,-4.04796E-3,-1.0646837E-2,3.4395452E-3,8.929996E-3,-5.109601E-3,4.9133413E-3,2.812256E-3,8.437351E-3,2.0857637E-3,-2.1995949E-2,5.112498E-3,1.7533077E-2,8.861058E-3,-1.5437362E-3,1.0137083E-3,2.8893536E-2,1.49163E-2,2.5529394E-2,-2.9439022E-4,1.2670473E-2,2.3822868E-3,9.412475E-3,9.0596E-3,2.97457E-2,4.168699E-3,-7.216204E-3,6.630461E-3,2.1037046E-2,-5.0151036E-3,7.2653648E-3,-2.315346E-3,-1.6464988E-2,-6.654608E-4,5.6925314E-3,-1.69544E-2,-9.724538E-4,-1.0068741E-2,-2.673233E-2,-5.3971354E-3,-1.23677915E-2,1.8402131E-4,-5.8965585E-3,3.4487266E-3,-2.6882763E-3,5.9899455E-4,6.8918234E-3],"split_indices":[18,51,58,27,44,4,51,7,27,38,51,44,46,5,44,27,11,51,51,7,27,8,4,2,53,4,0,94,51,4,0,84,28,2,82,1,17,89,6,0,50,2,7,47,5,49,44,56,31,17,32,0,0,47,51,0,46,28,44,0,0,2,7,2,10,0,1,41,0,0,27,0,6,52,52,46,0,55,2,0,57,1,52,44,0,4,51,46,0,0,0,0,37,56,27,0,0,0,0,31,52,0,0,0,0,41,8,55,0,9,5,36,57,9,32,60,27,32,5,0,0,2,44,55,0,40,3,51,0,0,44,0,0,53,51,0,7,7,0,2,2,0,38,36,8,2,0,0,0,0,0,0,0,47,4,0,7,27,0,56,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.158E3,1.678E3,4.8E2,1.198E3,4.8E2,2.7E2,2.1E2,7.51E2,4.47E2,3.45E2,1.35E2,2.54E2,1.6E1,1.3E2,8E1,5.56E2,1.95E2,3.05E2,1.42E2,1.55E2,1.9E2,1.06E2,2.9E1,2.35E2,1.9E1,7E0,9E0,1.19E2,1.1E1,6.5E1,1.5E1,1.94E2,3.62E2,7.1E1,1.24E2,2.8E1,2.77E2,1E2,4.2E1,6E0,1.49E2,6.1E1,1.29E2,6.2E1,4.4E1,1.6E1,1.3E1,1.1E1,2.24E2,1.1E1,8E0,3E0,4E0,1.09E2,1E1,4E0,7E0,5.2E1,1.3E1,1E1,5E0,4.1E1,1.53E2,1.52E2,2.1E2,5.9E1,1.2E1,1.19E2,5E0,4E0,2.4E1,8E0,2.69E2,7.9E1,2.1E1,3.7E1,5E0,1.1E2,3.9E1,7E0,5.4E1,1.02E2,2.7E1,5.9E1,3E0,3.2E1,1.2E1,9E0,7E0,7E0,6E0,2E0,9E0,3.9E1,1.85E2,3E0,8E0,5E0,3E0,7.5E1,3.4E1,3E0,7E0,5E0,2E0,1.7E1,3.5E1,1E1,3E0,2.9E1,1.2E1,5E1,1.03E2,1.06E2,4.6E1,1.76E2,3.4E1,4.8E1,1.1E1,5E0,7E0,6.7E1,5.2E1,2.1E1,3E0,2.26E2,4.3E1,6.9E1,1E1,4E0,1.7E1,3.3E1,4E0,2.5E1,8.5E1,3E0,3.6E1,5.1E1,3E0,9.6E1,6E0,3E0,2.4E1,4.2E1,1.7E1,2.9E1,3E0,8E0,4E0,3E0,6E0,7E0,2E0,1.7E1,2.2E1,3E0,1.82E2,6.8E1,7E0,2E1,1.4E1,1.3E1,4E0,1.4E1,2.1E1,8E0,2E0,2.5E1,4E0,6E0,6E0,4.4E1,6E0,7.1E1,3.2E1,8.3E1,2.3E1,4.1E1,5E0,1.2E2,5.6E1,1.6E1,1.8E1,3.3E1,1.5E1,6E0,5E0,2.7E1,4E1,1.6E1,3.6E1,9E0,1.2E1,1.45E2,8.1E1,3.3E1,1E1,3.4E1,3.5E1,8E0,2E0,3E0,1.4E1,2E0,2E0,2E0,2.3E1,6.8E1,1.7E1,3E0,3.3E1,3.2E1,1.9E1,9.3E1,3E0,2E0,4E0,2E0,2.2E1,3.9E1,3E0,3E0,1.4E1,1.9E1,1E1,1E1,7E0,6E0,1.6E1,1.9E1,1.63E2,1.1E1,5.7E1,1.1E1,9E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[-2.8564478E-3,-1.537361E-1,6.994541E-2,-2.4176301E-1,-4.977192E-2,3.5828486E-2,2.759324E-1,-3.2053325E-2,-2.7852395E-1,-1.0018577E-1,1.1438803E-1,-1.9434491E-2,1.4771004E-1,3.4823593E-1,2.31467E-2,-7.163382E-2,3.3079892E-2,-3.5105518E-1,-2.0113938E-1,-1.3900244E-1,1.661662E-2,-7.626E-2,1.5369086E-1,-1.8630233E-1,4.94038E-3,1.15917206E-1,2.9565588E-1,3.564368E-2,3.1766325E-1,-3.869324E-1,1.4124468E-1,-3.9154913E-2,-2.9120322E-2,-3.9329955E-1,-1.7175244E-1,-1.7875399E-1,-5.618295E-1,-1.537992E-1,5.7999782E-2,-6.5132745E-2,7.8031905E-2,-1.358091E-2,-1.3977041E-2,1.8022893E-1,7.0285825E-3,-1.661143E-1,-2.5941547E-2,-8.592557E-2,3.393826E-2,4.5235142E-2,1.7215681E-1,1.1038685E-1,3.770441E-1,2.6925808E-1,4.647593E-1,-2.2117909E-2,-7.574434E-3,5.1135972E-2,2.6194817E-1,-1.4959742E-1,5.1174685E-2,-3.4311232E-1,-5.4568577E-1,-6.018972E-2,-2.2986893E-1,-2.0827544E-1,2.9453479E-2,-3.0714463E-2,-4.6416395E-3,-1.4516851E-1,-1.9090267E-2,-4.618461E-3,9.316927E-2,9.160993E-4,-5.146886E-3,6.6586738E-3,2.2101335E-4,6.98251E-3,-2.8541079E-3,-3.784699E-3,1.9176185E-1,-4.252447E-3,3.5375094E-3,-2.9276484E-1,-1.3286987E-1,-5.6276646E-2,-4.9879155E-1,1.959007E-1,1.3370541E-3,5.6240097E-2,-1.7521616E-2,1.9108781E-1,-1.06119946E-1,6.671164E-3,-8.940681E-3,3.2892272E-1,3.0928185E-2,2.296413E-1,4.1650608E-1,5.418732E-1,2.5894916E-1,1.2459798E-2,1.380058E-2,4.8931004E-3,1.8214118E-2,8.572779E-3,-1.8730417E-1,2.9200616E-1,-6.3700695E-3,-3.7815607E-1,-1.2454684E-1,-6.576479E-1,-2.336043E-1,1.6933512E-3,-6.7541143E-3,-1.7105766E-2,-6.825954E-3,-3.473932E-1,-1.6168162E-1,2.3718795E-1,-3.7304383E-2,-1.1593211E-1,-1.9829823E-1,9.514788E-3,1.3953928E-3,1.4057326E-1,1.24939745E-2,-6.5768515E-3,-1.612373E-2,-1.5495051E-1,4.822997E-2,-2.9604478E-2,-2.9458472E-1,-7.381791E-3,-6.4628285E-1,1.4515188E-1,5.348173E-1,-3.087937E-1,1.0079706E-2,1.1291851E-2,1.6054815E-1,2.2227916E-1,5.3733304E-2,-1.3589102E-2,-8.10169E-4,3.8916337E-1,1.046515E-2,2.8205675E-1,1.4920829E-1,4.7115752E-1,-5.5735237E-3,6.329182E-1,1.3562948E-2,2.7723832E-3,1.4399365E-2,1.0481512E-1,-3.668468E-3,-1.542917E-2,-5.6924983E-3,1.2518836E-3,1.8891497E-2,-5.2158544E-3,2.1913522E-3,-1.12666935E-2,-1.907861E-2,2.5541282E-3,-1.0809854E-2,-3.3283427E-2,-1.4284603E-2,-3.1304946E-3,-1.5018236E-2,-8.62525E-3,-2.0507406E-2,-5.901259E-4,-8.835166E-3,1.599802E-2,1.899371E-3,7.2327664E-4,-9.106317E-3,-8.589704E-3,-2.3080322E-3,-3.4857125E-3,-1.1067949E-2,9.230316E-3,8.041137E-4,-5.093396E-3,-1.1806032E-2,9.516916E-3,-2.3188645E-3,2.4108205E-3,-3.3900312E-3,-6.0997684E-3,-1.7793108E-2,-3.557597E-2,-1.4049588E-2,1.5900008E-2,3.6002179E-3,2.7113505E-2,6.4748153E-3,-6.542221E-3,-2.1567708E-2,9.255502E-4,-9.935834E-3,-4.3275515E-3,2.4277768E-3,1.4398312E-2,4.328817E-3,1.2370631E-2,5.566158E-3,-9.0653886E-4,8.908507E-3,1.9795971E-2,9.974068E-3,9.232975E-3,1.6927667E-2,3.7480397E-3,1.1031809E-2,2.681444E-2,1.3410209E-2,1.079213E-2,3.0213019E-2,-1.3023932E-3,7.176114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,65,67,69,71,73,75,-1,77,79,81,83,-1,85,87,89,91,93,95,97,99,-1,-1,101,103,105,107,109,111,113,115,117,119,-1,-1,121,-1,-1,123,-1,-1,-1,-1,-1,-1,-1,125,-1,-1,127,129,131,133,135,137,139,-1,141,143,-1,-1,145,-1,147,149,151,153,155,-1,-1,-1,-1,157,159,161,163,165,167,169,-1,-1,-1,-1,171,173,175,177,179,181,-1,-1,183,-1,-1,-1,185,187,189,191,-1,193,195,197,199,201,203,205,207,209,-1,-1,211,-1,213,215,217,-1,219,-1,-1,-1,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4198225E1,6.549694E0,1.0433191E1,2.982212E0,2.7425184E0,7.892261E0,3.8484497E0,1.8334674E0,1.7816143E0,1.1493812E0,5.961752E-1,3.478532E0,1.950655E0,1.9512844E0,2.379269E0,1.0329597E0,0E0,1.2125435E0,1.2075896E0,5.627699E-1,3.2640472E-1,1.9087538E-1,2.5172186E-1,7.5572395E-1,1.9712013E0,1.3795338E0,1.0742049E0,0E0,9.784908E-1,1.6373193E-1,3.999095E-1,5.563481E-1,0E0,8.5287666E-1,2.0660782E-1,9.5270824E-1,4.1237712E-1,3.479824E-1,8.828951E-2,1.1834969E-1,1.8615165E-1,0E0,1.0875599E-1,1.8337703E-1,8.630427E-2,4.031949E-1,0E0,2.1875567E0,2.9935946E0,7.63838E-1,1.0422807E0,3.0221972E-1,5.26505E-1,5.985384E-1,4.4690275E-1,0E0,0E0,2.2187051E-1,3.020779E-1,3.5498512E-1,4.3147632E-1,7.5713444E-1,9.6449184E-1,1.18485585E-1,2.0050967E-1,8.189068E-1,2.8620633E-1,0E0,0E0,2.4573565E-1,0E0,0E0,8.10528E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8964648E-1,0E0,0E0,1.4455593E-1,3.4737134E-1,1.0718317E0,4.888575E-1,1.556334E0,1.2855883E0,7.1552205E-1,0E0,7.6779413E-1,2.1717027E-1,0E0,0E0,1.8097448E-1,0E0,3.6264467E-1,7.657776E-1,3.8395405E-1,1.2074357E-1,1.8980944E-1,0E0,0E0,0E0,0E0,1.7450994E-1,1.8714774E-1,1.6056754E-1,3.9840412E-1,3.4417713E-1,3.7796497E-1,1.3842148E-1,0E0,0E0,0E0,0E0,4.602449E-1,5.0986385E-1,9.858027E-2,1.4287813E-1,5.3675354E-1,3.172071E-1,0E0,0E0,2.6848525E-1,0E0,0E0,0E0,3.120445E-1,1.7185056E-1,5.8165854E-1,2.1452284E-1,0E0,1.3110805E-1,1.1079943E0,2.1553516E-1,2.890935E-1,1.1011436E0,4.9375886E-1,4.3997633E-1,7.1560526E-1,3.731844E-1,0E0,0E0,1.2290144E-1,0E0,3.3269262E-1,2.3079759E-1,2.907381E-1,0E0,8.536577E-2,0E0,0E0,0E0,8.345066E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,72,72,80,80,83,83,84,84,85,85,86,86,87,87,88,88,89,89,91,91,92,92,95,95,97,97,98,98,99,99,100,100,101,101,106,106,107,107,108,108,109,109,110,110,111,111,112,112,117,117,118,118,119,119,120,120,121,121,122,122,125,125,129,129,130,130,131,131,132,132,134,134,135,135,136,136,137,137,138,138,139,139,140,140,141,141,142,142,145,145,147,147,148,148,149,149,151,151,155,155],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,66,68,70,72,74,76,-1,78,80,82,84,-1,86,88,90,92,94,96,98,100,-1,-1,102,104,106,108,110,112,114,116,118,120,-1,-1,122,-1,-1,124,-1,-1,-1,-1,-1,-1,-1,126,-1,-1,128,130,132,134,136,138,140,-1,142,144,-1,-1,146,-1,148,150,152,154,156,-1,-1,-1,-1,158,160,162,164,166,168,170,-1,-1,-1,-1,172,174,176,178,180,182,-1,-1,184,-1,-1,-1,186,188,190,192,-1,194,196,198,200,202,204,206,208,210,-1,-1,212,-1,214,216,218,-1,220,-1,-1,-1,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1E0,1.2204E4,8.2608955E-3,6.860185E2,8.0708375E5,3.5301748E7,2.14099E5,2.0521326E0,1.1693485E0,1.5896305E-3,8.2E1,4.1973075E6,1.7324902E8,4.423676E1,5.884028E0,3.3079892E-2,1.0118026E0,4.4337E5,1.1159378E12,2.1924436E5,8.5023944E2,6.298144E0,1.5012488E1,7E2,3.7810526E2,5.2921E4,3.564368E-2,7.612907E-1,1.2799757E9,8.61491E3,2.222456E0,-2.9120322E-2,7.6599895E9,7.11E2,6.5317163E3,2.6476662E6,4.8297736E7,4.4821704E7,2.2927427E-1,1.3510204E1,-1.358091E-2,1.18827E5,1.2354571E0,4.838871E8,8.695652E-1,-2.5941547E-2,4.602015E0,2.44E3,3.4579455E6,3.0161016E7,5.503109E8,2.1944155E3,8.015419E3,2.7753594E7,-2.2117909E-2,-7.574434E-3,1E0,2.1325744E7,8E0,6.2686886E-5,6.057963E2,2.1907706E12,3.73802E5,1.5121101E9,2.4418726E-3,6.504065E-2,-3.0714463E-2,-4.6416395E-3,3.2613106E5,-1.9090267E-2,-4.618461E-3,1.75E2,9.160993E-4,-5.146886E-3,6.6586738E-3,2.2101335E-4,6.98251E-3,-2.8541079E-3,-3.784699E-3,1.3664E4,-4.252447E-3,3.5375094E-3,1.139605E6,1.6213043E3,2.4E1,3.97E2,9.923929E2,7.3E1,7.549744E7,-1.7521616E-2,2E0,7.1117523E2,6.671164E-3,-8.940681E-3,9.1E3,3.0928185E-2,1.505516E7,1.314806E8,7.9E1,6.626653E8,2.5492362E7,1.380058E-2,4.8931004E-3,1.8214118E-2,8.572779E-3,8.1445E1,8E0,1.1910478E5,1.3878379E0,6.258335E5,2.5269838E6,3.3E1,1.6933512E-3,-6.7541143E-3,-1.7105766E-2,-6.825954E-3,2.014576E8,2.7913043E0,9.83E2,9.123214E-1,5.23188E2,1.0508E5,9.514788E-3,1.3953928E-3,9.283875E6,1.24939745E-2,-6.5768515E-3,-1.612373E-2,2.0189162E7,1.92E2,1.4492002E0,5.542E3,-7.381791E-3,2.5789E4,5.088781E0,2.067698E0,2.7319324E7,5.9767612E7,1.8122449E2,9.538462E0,2E0,1.4433751E0,-1.3589102E-2,-8.10169E-4,2.5489312E5,1.046515E-2,4.1331047E-1,3.970405E3,1.1336898E-1,-5.5735237E-3,1.3664E4,1.3562948E-2,2.7723832E-3,1.4399365E-2,1.3664E4,-3.668468E-3,-1.542917E-2,-5.6924983E-3,1.2518836E-3,1.8891497E-2,-5.2158544E-3,2.1913522E-3,-1.12666935E-2,-1.907861E-2,2.5541282E-3,-1.0809854E-2,-3.3283427E-2,-1.4284603E-2,-3.1304946E-3,-1.5018236E-2,-8.62525E-3,-2.0507406E-2,-5.901259E-4,-8.835166E-3,1.599802E-2,1.899371E-3,7.2327664E-4,-9.106317E-3,-8.589704E-3,-2.3080322E-3,-3.4857125E-3,-1.1067949E-2,9.230316E-3,8.041137E-4,-5.093396E-3,-1.1806032E-2,9.516916E-3,-2.3188645E-3,2.4108205E-3,-3.3900312E-3,-6.0997684E-3,-1.7793108E-2,-3.557597E-2,-1.4049588E-2,1.5900008E-2,3.6002179E-3,2.7113505E-2,6.4748153E-3,-6.542221E-3,-2.1567708E-2,9.255502E-4,-9.935834E-3,-4.3275515E-3,2.4277768E-3,1.4398312E-2,4.328817E-3,1.2370631E-2,5.566158E-3,-9.0653886E-4,8.908507E-3,1.9795971E-2,9.974068E-3,9.232975E-3,1.6927667E-2,3.7480397E-3,1.1031809E-2,2.681444E-2,1.3410209E-2,1.079213E-2,3.0213019E-2,-1.3023932E-3,7.176114E-3],"split_indices":[19,109,2,26,51,27,44,2,52,41,37,28,27,7,55,52,0,38,11,30,27,51,56,53,2,51,1,0,37,7,4,53,0,5,0,4,31,44,12,37,57,0,28,52,7,52,0,52,28,50,44,5,4,4,46,0,0,79,1,3,38,51,30,32,5,38,56,0,0,32,0,0,8,0,0,0,0,0,0,0,2,0,0,30,4,8,2,51,0,7,0,6,32,0,0,28,0,49,1,8,7,49,0,0,0,0,51,0,32,52,31,31,8,0,0,0,0,7,53,0,26,4,28,0,0,44,0,0,0,44,0,52,28,0,1,57,37,5,44,51,57,16,41,0,0,32,0,34,51,56,0,2,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.201E3,7.16E2,1.485E3,3.87E2,3.29E2,1.275E3,2.1E2,5.8E1,3.29E2,2.52E2,7.7E1,8.54E2,4.21E2,1.63E2,4.7E1,5.6E1,2E0,1.68E2,1.61E2,1.89E2,6.3E1,1.3E1,6.4E1,1.08E2,7.46E2,3.48E2,7.3E1,9E0,1.54E2,1E1,3.7E1,5.4E1,2E0,1.35E2,3.3E1,1.53E2,8E0,1.76E2,1.3E1,2.7E1,3.6E1,2E0,1.1E1,5.4E1,1E1,1.04E2,4E0,1.8E2,5.66E2,1.55E2,1.93E2,2.3E1,5E1,1.18E2,3.6E1,6E0,4E0,2.2E1,1.5E1,2.4E1,3E1,1.04E2,3.1E1,1.2E1,2.1E1,1.34E2,1.9E1,6E0,2E0,1.72E2,4E0,2E0,1.1E1,1E1,1.7E1,1.8E1,1.8E1,2E0,9E0,2E0,5.2E1,4E0,6E0,2E1,8.4E1,1.69E2,1.1E1,9.4E1,4.72E2,1.52E2,3E0,1.81E2,1.2E1,2.1E1,2E0,4.5E1,5E0,9.5E1,2.3E1,2.5E1,1.1E1,2E1,2E0,8E0,7E0,2E0,2.2E1,5E0,2.5E1,8.9E1,1.5E1,2.2E1,9E0,6E0,6E0,6E0,1.5E1,3.2E1,1.02E2,4E0,1.5E1,1.13E2,5.9E1,3E0,8E0,3.4E1,1.8E1,7E0,1.3E1,7.5E1,9E0,1.53E2,1.6E1,4E0,7E0,8.3E1,1.1E1,1.2E1,4.6E2,1.07E2,4.5E1,1.47E2,3.4E1,3E0,9E0,2.6E1,1.9E1,5.6E1,3.9E1,2.1E1,2E0,1.7E1,8E0,3E0,8E0,1E1,1E1,5E0,1.7E1,2E0,3E0,8E0,1.7E1,2.4E1,6.5E1,6E0,9E0,1.7E1,5E0,4E0,5E0,1.4E1,1.8E1,1.9E1,8.3E1,2E0,2E0,1.2E1,3E0,5.2E1,6.1E1,1.7E1,4.2E1,2.2E1,1.2E1,5.5E1,2E1,3E0,6E0,5.4E1,9.9E1,7E0,9E0,4E0,3E0,1.9E1,6.4E1,9E0,2E0,7E0,5E0,4.41E2,1.9E1,3E1,7.7E1,1.2E1,3.3E1,9.6E1,5.1E1,2.3E1,1.1E1,1.9E1,7E0,3.2E1,2.4E1,2.4E1,1.5E1,1.1E1,1E1,2E0,1.5E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[5.732247E-4,-1.5517034E-1,7.2806165E-2,-2.016448E-1,6.1363755E-3,-5.72262E-2,1.2722972E-1,-1.5383407E-1,-4.271433E-1,1.0245144E-1,-1.4775601E-1,-1.633766E-1,1.8104797E-4,1.8417716E-1,4.687721E-3,-2.1128583E-1,-4.0532663E-2,-5.843205E-1,-2.9935086E-1,5.234279E-1,6.473119E-2,-3.4017494E-1,-1.086627E-1,-1.402817E-1,-2.0887358E-2,1.9951606E-2,-1.96799E-1,2.0062669E-1,-2.1071932E-1,-6.0313236E-2,1.7820424E-1,-2.6744837E-1,-1.529802E-1,8.565401E-3,-1.5705992E-1,-3.3162646E-3,-6.217555E-1,-2.0657045E-1,-3.9489168E-1,2.7133353E-2,8.927022E-3,-2.845414E-2,1.5455659E-1,-6.296516E-3,-2.0421341E-2,-5.2607253E-2,-2.9580155E-1,-2.4627066E-1,-8.4264345E-2,2.721757E-2,-2.11983E-2,-1.2083979E-1,-2.2947172E-2,1.0555239E-1,2.624338E-1,-1.4100821E-2,-1.1536485E-1,-7.130423E-2,1.5075674E-2,2.3128185E-1,-2.9747048E-2,-2.8808713E-1,-5.836587E-2,-1.8587762E-1,-5.7042886E-2,-1.1116512E-2,9.768983E-2,-2.477241E-1,-8.408425E-2,-2.1365104E-2,-7.277251E-1,-2.2413492E-1,1.1871512E-3,-3.06407E-1,-2.891977E-2,1.13101475E-1,-7.0916854E-2,9.375088E-2,2.780254E-1,-1.1376922E-1,2.741877E-2,-3.436611E-3,-2.0012226E-2,-1.4907387E-2,-9.471802E-3,-2.7370192E-2,-2.5157624E-1,4.732838E-2,-7.817869E-2,-1.7648771E-1,5.197008E-3,1.5522362E-1,1.6862445E-2,2.7339607E-2,2.4860428E-1,-7.164341E-3,-4.698813E-4,-1.1814073E-1,-1.644702E-2,1.7002861E-1,2.77269E-2,3.1062301E-2,-1.741632E-2,-3.2028297E-1,-1.3584049E-1,-1.359953E-1,5.618137E-3,-2.2393069E-1,-9.1993175E-2,-3.4752376E-2,-1.1271827E-2,3.0100802E-2,-1.049305E-1,9.3492E-3,2.1573922E-3,-1.4930243E-2,-5.0162286E-3,1.1839052E-3,-1.4065406E-1,-7.859601E-1,-1.2845869E-2,-6.1036637E-3,-2.8269792E-1,-3.6373782E-1,-2.2727903E-3,1.2648384E-2,-3.2607089E-3,-3.057463E-2,-1.0579667E-2,1.776512E-1,1.9796335E-3,3.3412644E-1,-2.6147661E-3,-1.6857727E-1,-1.5244131E-3,4.416259E-3,-2.7770095E-3,1.4775594E-1,-7.361432E-2,-7.313612E-2,-4.1066435E-1,-9.141076E-2,7.263649E-2,7.991444E-3,-1.607069E-1,-4.8486632E-4,-2.0690176E-1,2.9893064E-1,1.15950264E-1,6.9346E-2,-8.128369E-2,1.57891E-1,2.9097787E-1,-1.4850374E-1,-1.4425666E-2,1.0024495E-2,-3.407766E-2,8.5236624E-2,2.8205928E-1,-4.8949323E-3,4.9648467E-3,-1.1836973E-2,-1.9132027E-2,9.830577E-3,-7.891361E-3,-1.1111886E-3,-1.0176726E-2,-8.856546E-3,-1.5456278E-2,2.4929338E-3,-7.0428615E-3,1.303515E-3,-3.3812039E-3,1.8788666E-3,-6.8482026E-3,-6.5441737E-3,5.512941E-4,-1.1416614E-2,-3.5479749E-3,-1.3326315E-2,-3.7532773E-2,-4.6364656E-3,-1.5133274E-2,-2.0442728E-2,-9.440767E-3,-2.422093E-3,3.1708826E-3,1.5528771E-3,1.1093909E-2,2.2357203E-2,9.429764E-3,-3.0918363E-3,-1.0650921E-2,9.978516E-3,-4.5529322E-4,-4.3781586E-3,1.2396596E-2,3.9216317E-3,-6.961473E-3,-2.5010223E-2,-9.518516E-3,1.208877E-4,-8.168022E-3,4.502749E-3,-9.670722E-4,-9.2684645E-3,5.73658E-3,-1.070451E-2,-4.33655E-6,1.4566993E-2,1.1096272E-3,3.8886187E-3,1.2180842E-2,-8.244503E-3,3.9932304E-3,-1.2255789E-2,-1.9014991E-3,3.5546847E-2,5.5394596E-3,1.710596E-2,9.819816E-3,-5.8232564E-3,-1.810702E-2,-3.9746636E-3,3.6641166E-3,-6.25806E-4,-8.750454E-3,5.8271615E-3,1.2390777E-3,1.6124688E-2,5.1189186E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,-1,-1,75,77,-1,-1,79,81,83,85,87,-1,89,-1,91,93,-1,95,97,-1,99,101,103,105,107,109,111,113,115,117,-1,119,121,-1,123,-1,125,127,129,131,133,135,-1,-1,-1,-1,137,139,141,143,145,-1,147,149,-1,151,-1,-1,153,155,157,-1,159,-1,161,163,165,-1,167,169,171,-1,173,175,-1,-1,-1,-1,-1,177,179,-1,-1,181,183,-1,-1,-1,185,-1,187,-1,189,-1,191,-1,-1,-1,193,195,197,199,201,203,-1,205,-1,207,209,211,213,215,217,219,221,223,-1,225,227,229,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5244633E1,5.331352E0,1.085605E1,5.885538E0,2.3866825E0,2.7605217E0,7.542845E0,2.966217E0,1.7533436E0,1.5381726E0,4.2941773E-1,1.0294828E0,1.152776E0,4.8362217E0,3.8906343E0,9.512863E-1,8.881409E-1,7.580004E-1,4.0596437E-1,7.989359E-2,7.744755E-1,1.602602E-1,5.404168E-1,8.6389446E-1,0E0,9.6690255E-1,5.7463837E-1,4.1385746E0,2.522385E-1,1.1087592E0,1.0444105E0,6.434126E-1,4.659667E-1,1.9404128E-1,2.8351927E-1,0E0,2.7587605E-1,1.2854326E-1,3.6534882E-1,0E0,0E0,2.845192E-1,3.2679224E-1,0E0,0E0,2.0869046E-1,3.5229242E-1,8.7987185E-2,9.3550205E-1,5.675148E-1,0E0,3.218425E-1,0E0,1.2352829E0,1.875576E0,0E0,7.488729E-2,6.2754023E-1,0E0,1.6468391E0,4.436232E-1,6.378298E-1,2.2933751E-1,3.7953067E-1,1.6247328E-1,3.550337E-1,9.9055186E-2,1.8816268E-1,1.7204787E-1,0E0,3.1996346E-1,1.0834932E-1,0E0,2.901988E-1,0E0,3.587119E-1,2.2924104E-1,1.3073748E-1,3.018613E-1,9.9684685E-2,1.24242924E-1,0E0,0E0,0E0,0E0,6.1826867E-1,6.796582E-1,7.9315436E-1,9.321948E-1,9.3581796E-2,0E0,9.812331E-1,5.310446E-1,0E0,1.5454102E0,0E0,0E0,4.1353905E-1,4.8562983E-1,6.0291207E-1,0E0,2.0876896E-1,0E0,5.756149E-1,3.9082712E-1,1.03331536E-1,0E0,1.9591737E-1,3.2639986E-1,9.60329E-2,0E0,1.3858938E-1,1.3165998E-1,0E0,0E0,0E0,0E0,0E0,1.0526392E-1,1.3697243E-1,0E0,0E0,1.14506006E-1,1.3750362E-1,0E0,0E0,0E0,7.2939925E-2,0E0,1.0751799E-1,0E0,1.584009E-1,0E0,7.754934E-2,0E0,0E0,0E0,1.8565023E-1,5.077918E-1,1.80768E-1,2.5319886E-1,2.9355204E-1,4.8109877E-1,0E0,4.433869E-1,0E0,9.771174E-2,2.3009777E-1,6.349788E-1,3.3372152E-1,2.5651062E-1,2.8908947E0,1.7260551E0,4.584446E-1,2.2398093E-1,0E0,3.423608E-1,9.458828E-2,3.1035423E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,41,41,42,42,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,56,56,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,73,73,75,75,76,76,77,77,78,78,79,79,80,80,85,85,86,86,87,87,88,88,89,89,91,91,92,92,94,94,97,97,98,98,99,99,101,101,103,103,104,104,105,105,107,107,108,108,109,109,111,111,112,112,118,118,119,119,122,122,123,123,127,127,129,129,131,131,133,133,137,137,138,138,139,139,140,140,141,141,142,142,144,144,146,146,147,147,148,148,149,149,150,150,151,151,152,152,153,153,154,154,156,156,157,157,158,158],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,-1,-1,76,78,-1,-1,80,82,84,86,88,-1,90,-1,92,94,-1,96,98,-1,100,102,104,106,108,110,112,114,116,118,-1,120,122,-1,124,-1,126,128,130,132,134,136,-1,-1,-1,-1,138,140,142,144,146,-1,148,150,-1,152,-1,-1,154,156,158,-1,160,-1,162,164,166,-1,168,170,172,-1,174,176,-1,-1,-1,-1,-1,178,180,-1,-1,182,184,-1,-1,-1,186,-1,188,-1,190,-1,192,-1,-1,-1,194,196,198,200,202,204,-1,206,-1,208,210,212,214,216,218,220,222,224,-1,226,228,230,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.067536E3,9.99E2,3.9712732E7,6.747114E7,1.7036236E5,9.070543E0,2.7604443E2,1.0443182E3,1.4232318E3,1.9557823E0,1.6711E4,2E0,1E0,7.2456576E8,1E0,1.9761398E7,1.6592433E9,5.642369E11,9.83E2,1E0,1.2E1,5.5743E4,5.222222E1,-2.0887358E-2,5.8E1,1.359E3,5.732733E2,3.1240672E6,1.6173E4,1.417988E6,7.44393E5,1.1411955E-1,8.1335E4,7.857143E0,-3.3162646E-3,1E0,9.313E3,1E0,2.7133353E-2,8.927022E-3,1E0,7.3963095E9,-6.296516E-3,-2.0421341E-2,2.4285715E0,3.7333333E0,9.2058825E-1,2.2E1,8.009E3,-2.11983E-2,1.3E1,-2.2947172E-2,1.5E1,6.8E1,-1.4100821E-2,1.6347875E3,1.101214E6,1.5075674E-2,8.773955E6,1.6956E5,3.71E0,8.230099E4,2.652E3,1.2342778E1,1.195E3,2.24196E5,1E0,2.1442623E1,-2.1365104E-2,2.964E3,6.971004E7,1.1871512E-3,2.2065516E5,-2.891977E-2,3.0416E4,1.1658537E1,1.10597E5,5.0714142E2,4.3525005E3,1.141E4,-3.436611E-3,-2.0012226E-2,-1.4907387E-2,-9.471802E-3,1.3797468E0,2.8331464E2,3.5E2,5.8365756E-1,2.2178302E5,5.197008E-3,1.0258198E-2,3.271242E0,2.7339607E-2,1E0,-7.164341E-3,-4.698813E-4,2.0368582E8,2.3012722E1,1.5435694E9,2.77269E-2,6.129333E-2,-1.741632E-2,1.4103535E0,2.08E2,1.3984146E2,5.618137E-3,6.1102856E2,1E0,1.283561E6,-1.1271827E-2,1.708775E7,6.0809356E5,9.3492E-3,2.1573922E-3,-1.4930243E-2,-5.0162286E-3,1.1839052E-3,4.923077E0,3.44E2,-1.2845869E-2,-6.1036637E-3,7.43361E0,4.571429E0,-2.2727903E-3,1.2648384E-2,-3.2607089E-3,2.4480316E7,-1.0579667E-2,4.366E3,1.9796335E-3,6.5884055E11,-2.6147661E-3,2.909019E3,-1.5244131E-3,4.416259E-3,-2.7770095E-3,1.4343751E6,3.342857E2,3.8452727E3,4.1724915E2,3.1E1,1E0,7.991444E-3,1.838444E6,-4.8486632E-4,2.1325E4,3.6E3,6.392807E7,4.004E3,2E0,3.7577084E2,4.4312353E0,1E0,1.4685706E3,1.0024495E-2,5.6039163E9,1.23E2,1.65E2,-4.8949323E-3,4.9648467E-3,-1.1836973E-2,-1.9132027E-2,9.830577E-3,-7.891361E-3,-1.1111886E-3,-1.0176726E-2,-8.856546E-3,-1.5456278E-2,2.4929338E-3,-7.0428615E-3,1.303515E-3,-3.3812039E-3,1.8788666E-3,-6.8482026E-3,-6.5441737E-3,5.512941E-4,-1.1416614E-2,-3.5479749E-3,-1.3326315E-2,-3.7532773E-2,-4.6364656E-3,-1.5133274E-2,-2.0442728E-2,-9.440767E-3,-2.422093E-3,3.1708826E-3,1.5528771E-3,1.1093909E-2,2.2357203E-2,9.429764E-3,-3.0918363E-3,-1.0650921E-2,9.978516E-3,-4.5529322E-4,-4.3781586E-3,1.2396596E-2,3.9216317E-3,-6.961473E-3,-2.5010223E-2,-9.518516E-3,1.208877E-4,-8.168022E-3,4.502749E-3,-9.670722E-4,-9.2684645E-3,5.73658E-3,-1.070451E-2,-4.33655E-6,1.4566993E-2,1.1096272E-3,3.8886187E-3,1.2180842E-2,-8.244503E-3,3.9932304E-3,-1.2255789E-2,-1.9014991E-3,3.5546847E-2,5.5394596E-3,1.710596E-2,9.819816E-3,-5.8232564E-3,-1.810702E-2,-3.9746636E-3,3.6641166E-3,-6.25806E-4,-8.750454E-3,5.8271615E-3,1.2390777E-3,1.6124688E-2,5.1189186E-3],"split_indices":[19,51,2,44,44,27,52,51,4,47,52,28,6,72,7,109,44,5,30,0,109,3,10,4,0,3,28,51,46,2,28,1,37,10,55,0,108,2,15,0,0,63,5,0,0,56,53,52,3,28,0,8,0,3,0,0,4,1,0,1,2,53,31,10,53,0,10,15,55,0,0,44,0,32,0,10,56,28,57,4,9,0,0,0,0,52,4,2,53,27,0,37,52,0,109,0,0,7,55,7,0,37,0,55,2,51,0,54,15,47,0,44,27,0,0,0,0,0,57,0,0,0,55,53,0,0,0,1,0,2,0,30,0,31,0,0,0,44,51,46,4,10,112,0,9,0,1,10,12,9,8,32,53,112,54,0,5,3,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.242E3,7.1E2,1.532E3,5.51E2,1.59E2,4.52E2,1.08E3,4.56E2,9.5E1,9.8E1,6.1E1,1.58E2,2.94E2,7.37E2,3.43E2,3.02E2,1.54E2,4.1E1,5.4E1,7E0,9.1E1,9E0,5.2E1,1.48E2,1E1,2.68E2,2.6E1,7.08E2,2.9E1,2.5E2,9.3E1,1.52E2,1.5E2,1.09E2,4.5E1,3E0,3.8E1,2.9E1,2.5E1,5E0,2E0,4.5E1,4.6E1,4E0,5E0,4.1E1,1.1E1,5E1,9.8E1,2.65E2,3E0,2.2E1,4E0,2.8E2,4.28E2,1.3E1,1.6E1,2.44E2,6E0,7.4E1,1.9E1,1.38E2,1.4E1,1.11E2,3.9E1,9E1,1.9E1,1.9E1,2.6E1,1.8E1,2E1,2.7E1,2E0,2E1,5E0,1E1,3.5E1,3.2E1,1.4E1,2.3E1,1.8E1,5E0,6E0,1.3E1,3.7E1,7.4E1,2.4E1,2.23E2,4.2E1,1.8E1,4E0,1.79E2,1.01E2,1.5E1,4.13E2,1.1E1,5E0,1.31E2,1.13E2,6.5E1,9E0,1.7E1,2E0,1.13E2,2.5E1,1E1,4E0,7.8E1,3.3E1,3.6E1,3E0,6.3E1,2.7E1,5E0,1.4E1,1.1E1,8E0,9E0,1.7E1,1.7E1,3E0,1.2E1,1.5E1,1.6E1,4E0,5E0,5E0,2.9E1,6E0,1.1E1,2.1E1,1.2E1,2E0,1.3E1,1E1,1E1,8E0,1.5E1,5.9E1,1.2E1,1.2E1,3.4E1,1.89E2,1E1,3.2E1,3E0,1.5E1,3.7E1,1.42E2,6.6E1,3.5E1,1.33E2,2.8E2,1.01E2,3E1,7E0,1.06E2,3.8E1,2.7E1,6E0,1.1E1,7.4E1,3.9E1,2E0,2.3E1,5E0,5E0,6.5E1,1.3E1,1E1,2.3E1,1.4E1,2.2E1,6E1,3E0,2E1,7E0,5E0,1.2E1,2E0,1.5E1,4E0,1.1E1,9E0,7E0,2.4E1,5E0,4E0,7E0,4E0,8E0,6E0,7E0,1E1,5E0,5.6E1,3E0,4E0,8E0,6E0,6E0,1.7E1,1.7E1,1.47E2,4.2E1,2.8E1,4E0,1.3E1,2E0,3.4E1,3E0,1.2E2,2.2E1,4E0,6.2E1,5E0,3E1,6E0,1.27E2,1.27E2,1.53E2,9.5E1,6E0,1.7E1,1.3E1,9.5E1,1.1E1,2.1E1,1.7E1,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"231","size_leaf_vector":"1"}},{"base_weights":[-2.1779037E-3,-1.2826072E-1,5.914255E-2,-1.3914213E-1,5.180832E-1,-8.235415E-3,1.496039E-1,-1.2431342E-1,-5.522556E-1,3.0005855E-2,8.284246E-3,-1.3080375E-1,3.153988E-2,5.0432604E-2,2.178154E-1,-1.8483147E-1,-2.8116383E-2,-6.592374E-1,-2.5588724E-1,-9.56307E-2,-3.6044616E-1,4.7035832E-2,-2.1314752E-1,-8.995037E-3,1.886994E-1,1.3707097E-1,2.9358867E-1,-1.5418117E-1,-4.1660497E-1,8.831137E-2,-9.3397565E-2,-7.530111E-1,-1.2838711E-2,-2.15454E-2,-1.2997214E-3,-1.4329827E-1,4.6342198E-2,-2.537528E-1,-2.847885E-2,-2.030725E-2,1.0068816E-1,-2.3216811E-1,3.9002958E-3,-2.091549E-1,2.019909E-2,2.2179084E-1,-1.7389944E-1,1.5346025E-1,-1.5853164E-1,3.6520487E-1,1.5350364E-1,-2.1773908E-1,-8.011926E-2,-1.8206137E-3,-4.4080412E-1,-1.8785093E-3,1.4337432E-1,-1.504244E-1,3.083411E-2,-2.130511E-2,-4.2008206E-2,-1.070727E-1,-2.5655332E-1,4.124022E-1,-5.384543E-2,-1.3144124E-2,-5.1583676E-3,-2.5860593E-3,-2.7971846E-1,1.551746E-1,-1.2284735E-3,-1.6514538E-1,-1.595687E-2,-1.4589049E-1,-2.1423101E-2,5.5125006E-3,7.2367573E-3,2.3330294E-1,-5.293329E-3,9.788148E-3,-1.6895248E-2,1.1867036E-1,2.7301747E-1,6.12276E-3,-2.488788E-1,1.9535023E-1,4.1222116E-1,1.1740495E-1,2.2407116E-2,-2.6734352E-1,-1.4278767E-1,-3.319278E-2,-1.6284055E-1,-2.223043E-2,-3.292116E-1,6.108273E-2,-1.22813314E-1,1.8053746E-1,1.1025815E-2,-1.3148595E-1,-3.3802336E-1,1.7090356E-1,-3.8141135E-2,-1.705783E-1,-3.2309715E-2,-3.9312196E-1,-1.6629727E-1,5.7025948E-3,2.1200372E-2,-4.794727E-3,1.338686E-3,6.1396122E-2,-4.5650154E-2,-2.710481E-2,-9.692557E-2,1.161037E-1,2.5319028E-1,2.6608003E-2,-1.9229038E-1,-1.2337496E-2,-5.2924803E-3,-3.2772855E-3,-1.1187007E-2,2.0495236E-2,-1.0050527E-1,1.7192192E-1,2.8479326E-1,1.3481623E-1,-1.5449718E-1,1.7204593E-1,3.7564993E-1,-1.4416112E-2,-1.2150472E-3,2.9074398E-1,2.5162182E-3,4.3268523E-1,2.9735396E-2,1.6457628E-1,-3.8225524E-2,-1.2929298E-2,-3.7334503E-3,4.1043446E-3,-7.5197867E-3,-4.141865E-3,1.0700929E-3,-2.1447265E-3,-1.0400449E-2,-3.846028E-3,-1.6999708E-2,6.224188E-4,9.448005E-3,8.708746E-4,-7.5766663E-3,1.0285267E-2,4.3174946E-3,-6.264987E-3,3.8313484E-3,-9.50498E-3,-4.3789432E-3,-5.99012E-3,-2.0514319E-2,3.1669987E-3,1.8987328E-2,6.442094E-4,-7.3895683E-3,-6.401254E-3,-1.766837E-2,1.24420095E-2,-4.076493E-3,-7.17727E-3,-2.4359224E-2,2.4821418E-3,-8.713371E-3,-6.3821487E-3,4.3273144E-3,2.135326E-3,-4.4526807E-3,-6.703657E-3,1.9186185E-3,-3.191616E-2,5.942031E-3,8.7040225E-3,1.9343287E-2,7.323346E-4,1.6704362E-2,-2.2328664E-2,-4.892322E-3,4.648462E-3,-7.5892494E-5,-8.435046E-3,-1.9049621E-3,1.1913794E-2,3.895851E-3,1.70863E-2,8.685334E-3,4.278698E-3,1.0476449E-2,-1.9164244E-2,2.5585823E-4,1.7250422E-3,1.1388292E-2,4.840473E-3,1.8225461E-2,1.5153988E-3,1.5550827E-2,1.6963867E-2,2.3569228E-2,8.449543E-3,-8.5735805E-3,-1.4111747E-3,8.53964E-3,-6.706459E-3,3.4615458E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,-1,-1,61,63,65,-1,67,69,71,-1,73,75,77,79,81,83,85,87,89,91,-1,93,95,97,99,101,-1,-1,103,105,107,109,-1,-1,111,113,115,117,119,-1,121,-1,123,-1,125,-1,-1,-1,127,129,-1,131,133,135,137,-1,139,141,143,145,-1,147,149,151,153,155,157,159,161,163,165,167,169,171,-1,-1,-1,-1,173,175,-1,177,179,181,183,185,-1,-1,-1,-1,187,189,191,193,195,197,199,201,-1,-1,203,-1,205,207,209,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6823502E1,5.0796776E0,8.924461E0,4.2256308E0,4.6051502E-1,4.099286E0,4.2169037E0,3.941887E0,5.9225416E-1,0E0,0E0,1.6294386E0,2.4177327E0,2.1052158E0,2.2235165E0,2.8910017E0,2.0071905E0,3.8532066E-1,3.8617802E-1,1.2247913E0,6.1410785E-1,2.1626863E0,2.3632717E-1,1.0560534E0,9.62486E-1,9.012213E-1,1.8500862E0,1.7195196E0,4.1947556E-1,4.7205073E-1,1.2019253E0,2.0535946E-1,0E0,0E0,0E0,5.2639866E-1,1.7092414E0,7.430017E-2,0E0,1.2220987E0,1.8493643E0,2.3201609E-1,0E0,3.1515634E-1,3.2316166E-1,3.0412364E-1,6.2484765E-1,6.830144E-1,2.9907048E-1,9.177151E-1,7.553294E-1,6.979761E-1,6.6226757E-1,0E0,9.0519905E-2,2.8922737E-1,2.8820014E-1,3.676834E-1,5.2822995E-1,0E0,0E0,4.8826694E-1,3.290615E-1,1.01044536E-1,1.6380313E-1,0E0,0E0,6.9163644E-1,9.3315244E-1,8.016186E-1,6.273799E-1,9.764004E-2,0E0,1.3089049E-1,0E0,2.307947E-1,0E0,1.6911411E-1,0E0,0E0,0E0,6.162493E-1,3.291633E-1,0E0,1.1498165E-1,3.705107E-1,7.4637413E-1,4.5411003E-1,0E0,3.742771E-1,4.5577836E-1,3.698659E-1,4.6792853E-1,0E0,1.602521E-1,1.7179978E-1,8.546509E-2,1.6107428E-1,1.656579E-1,2.715447E-1,1.768105E-1,4.2219067E-1,2.4468583E-1,3.0390096E-1,8.93546E-1,3.2266593E-1,1.3203448E-1,0E0,0E0,0E0,0E0,7.121619E-1,7.38333E-1,0E0,9.1767974E-2,2.0444796E0,5.6487703E-1,3.619029E-1,3.2687718E-1,0E0,0E0,0E0,0E0,2.3157008E-1,8.240464E-2,2.4370468E-1,2.374866E-1,4.764712E-1,3.4677297E-1,2.1402293E-1,8.7127924E-2,0E0,0E0,2.1916342E-1,0E0,2.739334E-1,2.4160437E-1,2.270968E-1,2.0038997E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,35,35,36,36,37,37,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,73,73,75,75,77,77,81,81,82,82,84,84,85,85,86,86,87,87,89,89,90,90,91,91,92,92,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,111,111,112,112,114,114,115,115,116,116,117,117,118,118,123,123,124,124,125,125,126,126,127,127,128,128,129,129,130,130,133,133,135,135,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,-1,-1,62,64,66,-1,68,70,72,-1,74,76,78,80,82,84,86,88,90,92,-1,94,96,98,100,102,-1,-1,104,106,108,110,-1,-1,112,114,116,118,120,-1,122,-1,124,-1,126,-1,-1,-1,128,130,-1,132,134,136,138,-1,140,142,144,146,-1,148,150,152,154,156,158,160,162,164,166,168,170,172,-1,-1,-1,-1,174,176,-1,178,180,182,184,186,-1,-1,-1,-1,188,190,192,194,196,198,200,202,-1,-1,204,-1,206,208,210,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.3144558E4,1.5938586E8,1.4299594E8,1.8215276E7,7.805608E4,1.857051E5,9.756616E2,1E0,3.0005855E-2,8.284246E-3,1E0,3.956147E-1,1.3359244E9,5.964E3,3.0161016E7,2.342147E7,2.9414747E1,1.2015E4,3.051E3,3.7832818E0,8.61E2,3.1998687E3,1.2944685E0,1.5005797E4,2.2075728E7,8.2E1,1.638058E2,7.158023E7,2.0407547E5,2.0601392E3,4.511E3,-1.2838711E-2,-2.15454E-2,-1.2997214E-3,1.2E1,5.511673E0,1.5532663E1,-2.847885E-2,3.6616542E0,2.8E1,4.352552E9,3.9002958E-3,1.6173E4,3.4409692E3,8.6206274E2,5.9E1,2.9251662E6,2.97271E5,2.4347392E-1,7.17892E0,3.1979167E0,9.912942E6,-1.8206137E-3,3.8172052E2,1.26248375E5,3.1907563E0,1.13808104E8,1.1297775E2,-2.130511E-2,-4.2008206E-2,3.3151623E1,1.9676556E3,2.5207965E2,5.28E2,-1.3144124E-2,-5.1583676E-3,1.5043668E0,1E0,2.1827742E6,4.1585636E1,8.447369E0,-1.595687E-2,1.0828989E8,-2.1423101E-2,3.1240672E6,7.2367573E-3,6.4683E4,-5.293329E-3,9.788148E-3,-1.6895248E-2,1.766486E7,1.5088776E0,6.12276E-3,1.2956525E8,4.4844124E-1,4.5481584E7,3.5301748E7,2.2407116E-2,4.95E9,7E0,2.785857E2,1.0931163E9,-2.223043E-2,2.7864855E11,1.0855529E3,1.1399403E3,3E1,1.3912211E9,1E0,2.6608378E8,4.77721E3,1.0104842E12,3.09627E5,7E0,6.872425E1,3.535297E3,5.7025948E-3,2.1200372E-2,-4.794727E-3,1.338686E-3,5.798305E1,6.714286E0,-2.710481E-2,6.789622E7,4E0,2.1132307E0,1.0409E4,2.657E3,-1.2337496E-2,-5.2924803E-3,-3.2772855E-3,-1.1187007E-2,3.94E2,4.247E3,1.4490348E9,5.0795E4,7.2041174E2,1.1660184E3,1.9360614E0,1.9577E4,-1.4416112E-2,-1.2150472E-3,5.751617E6,2.5162182E-3,2.0718E4,5.5743E4,7.44E2,3.555E3,-1.2929298E-2,-3.7334503E-3,4.1043446E-3,-7.5197867E-3,-4.141865E-3,1.0700929E-3,-2.1447265E-3,-1.0400449E-2,-3.846028E-3,-1.6999708E-2,6.224188E-4,9.448005E-3,8.708746E-4,-7.5766663E-3,1.0285267E-2,4.3174946E-3,-6.264987E-3,3.8313484E-3,-9.50498E-3,-4.3789432E-3,-5.99012E-3,-2.0514319E-2,3.1669987E-3,1.8987328E-2,6.442094E-4,-7.3895683E-3,-6.401254E-3,-1.766837E-2,1.24420095E-2,-4.076493E-3,-7.17727E-3,-2.4359224E-2,2.4821418E-3,-8.713371E-3,-6.3821487E-3,4.3273144E-3,2.135326E-3,-4.4526807E-3,-6.703657E-3,1.9186185E-3,-3.191616E-2,5.942031E-3,8.7040225E-3,1.9343287E-2,7.323346E-4,1.6704362E-2,-2.2328664E-2,-4.892322E-3,4.648462E-3,-7.5892494E-5,-8.435046E-3,-1.9049621E-3,1.1913794E-2,3.895851E-3,1.70863E-2,8.685334E-3,4.278698E-3,1.0476449E-2,-1.9164244E-2,2.5585823E-4,1.7250422E-3,1.1388292E-2,4.840473E-3,1.8225461E-2,1.5153988E-3,1.5550827E-2,1.6963867E-2,2.3569228E-2,8.449543E-3,-8.5735805E-3,-1.4111747E-3,8.53964E-3,-6.706459E-3,3.4615458E-3],"split_indices":[19,51,7,44,1,27,27,54,15,0,0,6,26,7,2,44,44,55,9,2,52,2,4,52,4,44,8,51,7,27,51,10,0,0,0,8,52,57,0,52,3,5,0,2,51,57,8,27,1,37,34,53,44,0,51,27,56,44,57,0,0,54,32,4,0,0,0,52,109,27,52,52,0,44,0,46,0,2,0,0,0,46,37,0,31,56,46,44,0,5,10,51,5,0,30,51,4,3,12,80,7,51,30,9,12,51,32,0,0,0,0,46,57,0,7,3,52,2,2,0,0,0,0,0,2,5,9,51,4,53,9,0,0,44,0,2,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.174E3,7.11E2,1.463E3,7E2,1.1E1,8.39E2,6.24E2,6.77E2,2.3E1,7E0,4E0,2.05E2,6.34E2,2.55E2,3.69E2,4.15E2,2.62E2,1.6E1,7E0,1.79E2,2.6E1,5.97E2,3.7E1,1.79E2,7.6E1,1.8E2,1.89E2,3.68E2,4.7E1,9.4E1,1.68E2,1.2E1,4E0,3E0,4E0,1.34E2,4.5E1,2E1,6E0,2.65E2,3.32E2,3.5E1,2E0,2.2E1,1.57E2,7E1,6E0,1.71E2,9E0,1.24E2,6.5E1,1.97E2,1.71E2,3E0,4.4E1,3.6E1,5.8E1,1.15E2,5.3E1,6E0,6E0,1.03E2,3.1E1,9E0,3.6E1,1.5E1,5E0,2.49E2,1.6E1,2.16E2,1.16E2,2.4E1,1.1E1,1.9E1,3E0,1.43E2,1.4E1,6.8E1,2E0,2E0,4E0,1.34E2,3.7E1,2E0,7E0,2.8E1,9.6E1,6E1,5E0,1.17E2,8E1,1.1E2,6.1E1,2.8E1,1.6E1,2.4E1,1.2E1,4.5E1,1.3E1,1.06E2,9E0,1.7E1,3.6E1,5.5E1,4.8E1,1.1E1,2E1,2E0,7E0,2.2E1,1.4E1,1E2,1.49E2,5E0,1.1E1,1.56E2,6E1,1.02E2,1.4E1,6E0,1.8E1,1.2E1,7E0,1.26E2,1.7E1,3.3E1,3.5E1,1.27E2,7E0,2E1,1.7E1,5E0,2E0,1.6E1,1.2E1,9.1E1,5E0,4.6E1,1.4E1,1.06E2,1.1E1,7E0,7.3E1,5.4E1,5.6E1,2.3E1,3.8E1,3E0,1.3E1,1.9E1,5E0,3E0,9E0,2.8E1,1.7E1,4E0,9E0,3.1E1,7.5E1,4E0,5E0,1.3E1,4E0,2.6E1,1E1,5E1,5E0,7E0,4.1E1,5E0,6E0,2E0,1.8E1,1.4E1,8.6E1,5.4E1,9.5E1,8E0,3E0,2E0,1.54E2,4.6E1,1.4E1,1E2,2E0,2E0,1.2E1,2.6E1,1E2,6E0,1.1E1,1.5E1,1.8E1,1.6E1,1.9E1,9.1E1,3.6E1,2E0,5E0,8E0,1.2E1,2E0,1.5E1,3E0,1.3E1,5.8E1,3.3E1,3E0,2E0,5E0,4.1E1,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[5.361184E-3,5.1004842E-2,-1.5969804E-1,-2.0827627E-2,1.5153995E-1,-2.5760275E-1,-2.4897886E-2,-6.533084E-2,9.71324E-2,1.9174495E-1,-9.385324E-2,-4.5017064E-1,-2.2202767E-1,-6.997239E-2,1.7622024E-1,-9.623876E-2,7.263273E-3,1.0280233E-1,-1.46066705E-2,2.5073388E-1,6.2428627E-2,-2.2786114E-1,6.723641E-2,-4.7924528E-1,-6.037469E-3,-5.8480716E-1,-2.0036907E-1,-4.431759E-2,-2.8272557E-1,2.8730503E-1,-3.473262E-2,-7.2104834E-2,-2.5701764E-1,7.442513E-2,-7.379723E-2,1.1848655E-1,-4.797218E-2,2.7092978E-2,2.2976978E-1,2.6275115E-2,5.776808E-1,-2.680513E-1,7.3660095E-4,-1.68384E-2,1.7613442E-1,-5.8628243E-1,-1.7172145E-2,-3.247492E-3,-6.7157304E-1,-1.00323565E-1,-2.4779144E-1,-8.449129E-2,3.456436E-2,-3.6084583E-1,-3.3700766E-4,3.5461688E-1,1.6939755E-1,2.4974376E-3,-5.9991796E-3,-1.0645356E-1,-3.8313398E-3,-6.969979E-1,-1.86737E-1,1.0432247E-1,-1.1813017E-1,-2.7527368E-1,-4.1275132E-2,9.154859E-2,2.160626E-1,-9.3652025E-2,4.3427018E-3,4.4940317E-1,2.1061929E-1,-1.1391727E-1,5.5953007E-2,6.57334E-3,6.929089E-1,-3.5324532E-1,-7.793241E-3,-7.6971196E-2,6.975293E-2,2.4589163E-1,-9.626698E-2,-2.9131467E-2,-1.0848976E-2,-1.3636647E-2,-3.4988225E-2,-1.7880791E-1,-5.5183778E-3,-5.5618453E-1,-2.080054E-1,-6.2310845E-2,-1.7120655E-1,7.6855565E-3,-1.6661228E-3,-7.5359573E-3,-2.038901E-2,1.7544672E-2,4.247615E-3,9.519935E-3,-4.3905296E-4,-1.1025605E-1,1.4117118E-2,1.0478733E-1,-5.7059422E-2,-3.587774E-2,-1.0952815E-2,-2.1597627E-1,8.062847E-3,1.2707452E-1,-3.3022024E-2,-1.7928427E-1,5.6792437E-3,-8.532745E-3,-2.0154495E-2,-5.6628224E-2,1.3543029E-2,1.0126573E-1,-2.3951638E-1,3.030873E-1,1.4676166E-1,-1.1094317E-2,-2.038591E-5,2.08812E-1,6.2492895E-1,2.6004007E-1,1.6061327E-1,-6.6253087E-3,2.5519428E-3,9.518318E-2,-1.1297158E-2,1.4746955E-2,3.6388487E-2,-4.492221E-3,-3.7102014E-1,-5.2427785E-3,3.5393676E-3,1.1042535E-2,4.7682822E-4,1.8922523E-2,1.1404227E-1,5.02429E-3,-1.2288268E-2,-8.8288635E-2,-2.2247387E-1,2.0279184E-2,-9.531039E-3,-1.5862837E-2,-3.0116867E-2,-1.7046247E-1,-4.3417102E-1,-2.2856086E-2,-1.2993723E-1,-2.1488017E-3,-2.2084968E-1,4.6400484E-2,-7.508175E-2,2.536584E-3,-5.366494E-3,6.202045E-3,-7.2238585E-3,-4.232328E-3,3.7554298E-3,-2.0032153E-3,-1.232203E-2,7.6366314E-3,2.3155874E-3,-4.8707584E-3,4.9429573E-3,1.6013904E-3,-9.870073E-3,3.1486552E-4,-4.9591484E-3,4.9218475E-3,-1.2744595E-2,2.610931E-3,-2.5168944E-2,4.9995854E-3,1.4780336E-2,1.178735E-2,4.1195583E-3,1.1608845E-2,6.2500424E-4,3.5365067E-2,1.909884E-2,1.0415429E-2,1.8356502E-2,8.232271E-3,8.583857E-4,2.5418357E-3,7.7136983E-3,-4.33222E-3,9.221032E-4,-2.1326214E-2,-1.2362959E-2,-5.0422986E-4,7.351437E-3,6.682588E-3,-6.1542606E-3,-5.638642E-3,-1.2767975E-2,4.588389E-3,-2.1914465E-3,-9.278533E-3,3.0090557E-3,-3.5311293E-3,-2.5163524E-2,-4.473403E-3,1.5498279E-3,2.989338E-3,-6.979277E-3,-5.444506E-3,-1.377518E-2,8.6543465E-4,8.678515E-3,2.828645E-3,-4.935842E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,79,81,83,-1,-1,85,87,89,91,93,95,-1,97,99,-1,-1,101,103,105,107,109,111,113,115,117,119,121,-1,123,125,127,129,-1,131,133,-1,135,137,139,141,-1,-1,-1,-1,143,145,147,149,151,153,-1,155,-1,-1,-1,-1,-1,-1,157,-1,159,161,-1,-1,163,-1,165,167,169,-1,-1,-1,171,-1,173,175,177,179,-1,-1,181,183,185,187,-1,-1,189,191,-1,-1,-1,193,-1,-1,-1,-1,-1,195,-1,-1,197,199,201,-1,-1,-1,203,205,207,209,-1,211,213,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6936832E1,1.2720545E1,6.420805E0,5.402934E0,7.2596626E0,1.8319244E0,1.8797822E0,1.6767673E0,7.0433164E-1,4.797762E0,2.260398E0,3.3217716E-1,1.7685719E0,9.09896E-1,9.004576E-1,2.0147061E0,1.2249875E0,6.654582E-1,0E0,3.018633E0,3.6918283E0,5.6362605E-1,4.4243366E-1,1.9486809E-1,0E0,5.3055716E-1,1.0568638E0,4.8460525E-1,3.844055E-1,1.383431E-1,1.3036388E-1,1.0713067E0,1.9725475E0,7.204212E-1,6.569867E-1,6.4523077E-1,1.864541E-1,0E0,1.6298428E0,7.844089E-1,5.472522E-1,3.3959627E-1,0E0,1.5116437E-1,4.227124E-1,1.826973E-1,0E0,0E0,1.7967987E-1,5.570675E-1,1.7738504E0,1.8359369E-1,2.5498322E-1,1.686182E-1,0E0,9.236753E-2,8.256775E-2,0E0,0E0,5.1788116E-1,8.963398E-1,1.9926214E-1,6.785002E-1,3.3855736E-1,2.7571195E-1,1.2256539E-1,4.8208466E-1,6.676552E-1,2.8118658E-1,2.9620856E-1,0E0,1.2208657E0,8.9487076E-1,1.9043395E-1,4.1000175E-1,0E0,1.4515162E-1,7.6066494E-2,0E0,1.11467704E-1,1.2119229E-1,3.3800733E-1,2.1161678E-1,0E0,0E0,0E0,0E0,1.4241517E-1,1.893675E-1,1.4839554E-1,1.0977373E0,2.1582115E-1,1.1237913E-1,0E0,1.5169512E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.2233086E-1,0E0,4.5599335E-1,5.381082E-1,0E0,0E0,5.28327E-1,0E0,2.819767E-1,1.8139064E-1,1.2408206E-1,0E0,0E0,0E0,2.9007095E-1,0E0,5.8642983E-1,5.9625435E-1,6.753564E-2,1.7903441E-1,0E0,0E0,1.3569915E-1,2.414751E-1,7.308512E-1,5.7065344E-1,0E0,0E0,2.781229E-1,1.5693729E-1,0E0,0E0,0E0,9.67195E-2,0E0,0E0,0E0,0E0,0E0,6.8363875E-2,0E0,0E0,1.8865106E-1,1.223104E-1,1.8308233E-1,0E0,0E0,0E0,9.939115E-1,7.7979493E-1,2.3318692E-1,1.5256298E-1,0E0,8.088094E-2,9.940476E-2,8.61808E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,43,43,44,44,45,45,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,76,76,77,77,79,79,80,80,81,81,82,82,87,87,88,88,89,89,90,90,91,91,92,92,94,94,101,101,103,103,104,104,107,107,109,109,110,110,111,111,115,115,117,117,118,118,119,119,120,120,123,123,124,124,125,125,126,126,129,129,130,130,134,134,140,140,143,143,144,144,145,145,149,149,150,150,151,151,152,152,154,154,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,80,82,84,-1,-1,86,88,90,92,94,96,-1,98,100,-1,-1,102,104,106,108,110,112,114,116,118,120,122,-1,124,126,128,130,-1,132,134,-1,136,138,140,142,-1,-1,-1,-1,144,146,148,150,152,154,-1,156,-1,-1,-1,-1,-1,-1,158,-1,160,162,-1,-1,164,-1,166,168,170,-1,-1,-1,172,-1,174,176,178,180,-1,-1,182,184,186,188,-1,-1,190,192,-1,-1,-1,194,-1,-1,-1,-1,-1,196,-1,-1,198,200,202,-1,-1,-1,204,206,208,210,-1,212,214,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,5.862126E2,1E0,9.9625344E5,4.0844156E7,1.5804776E4,3.0497742E3,3.3030225E2,4.6E1,9.74026E0,2.200423E3,3.7E1,6E0,8.3510876E9,6.747114E7,9.457831E0,2.3E1,6.1114804E-8,-1.46066705E-2,3.8E1,4.5575E4,1.17E2,9.256843E3,2.8972292E-5,-6.037469E-3,1.9041E4,1.2E1,6.9664386E2,1.2151E4,1.5609541E0,1.1855755E8,1.477425E6,2E0,1.1884672E7,5.5E1,3.958068E2,3.3E1,2.7092978E-2,1.0918E4,8.00791E5,6.3039363E4,2.09633E1,7.3660095E-4,4.0434834E5,2.8159826E0,1.8600049E10,-1.7172145E-2,-3.247492E-3,1.0503613E5,3.3459526E-1,1.2068E4,1.6007329E7,1.18827E5,7.5436196E-7,-3.3700766E-4,6.2277466E2,3.0364912E5,2.4974376E-3,-5.9991796E-3,3.65625E1,2E0,3.3E1,2.1146836E7,3.8134545E6,1.5033076E6,7.55552E5,9.6696576E8,3.8992128E6,2.3663366E0,7.5773336E7,4.3427018E-3,2.1169072E3,8.986545E6,2.014576E8,1.5548611E7,6.57334E-3,1.22159E5,4.360204E1,-7.793241E-3,1E0,3.7448005E3,4.1179886E0,1.568E3,-2.9131467E-2,-1.0848976E-2,-1.3636647E-2,-3.4988225E-2,3.3125E0,1E0,7.07089E5,2.72258E5,2.1111E4,3E1,7.6855565E-3,6.585265E-1,-7.5359573E-3,-2.038901E-2,1.7544672E-2,4.247615E-3,9.519935E-3,-4.3905296E-4,1.1057693E0,1.4117118E-2,1.1304221E0,1.7022566E7,-3.587774E-2,-1.0952815E-2,1.481E3,8.062847E-3,2.4434783E0,3.957351E6,9.6E2,5.6792437E-3,-8.532745E-3,-2.0154495E-2,1.2982479E-2,1.3543029E-2,9.1E2,3.5044186E2,1.0398213E6,8E0,-1.1094317E-2,-2.038591E-5,4.4844124E-1,3E0,5.3515047E-1,1.600814E6,-6.6253087E-3,2.5519428E-3,1.108E3,2.1944155E3,1.4746955E-2,3.6388487E-2,-4.492221E-3,1.1863768E-4,-5.2427785E-3,3.5393676E-3,1.1042535E-2,4.7682822E-4,1.8922523E-2,3.9401836E11,5.02429E-3,-1.2288268E-2,1.0894445E-2,4.029937E-1,2.46875E0,-9.531039E-3,-1.5862837E-2,-3.0116867E-2,4.832E3,1.09E3,5.124411E2,1.5E1,-2.1488017E-3,7.688607E-1,5.8871865E-1,1.2783833E3,2.536584E-3,-5.366494E-3,6.202045E-3,-7.2238585E-3,-4.232328E-3,3.7554298E-3,-2.0032153E-3,-1.232203E-2,7.6366314E-3,2.3155874E-3,-4.8707584E-3,4.9429573E-3,1.6013904E-3,-9.870073E-3,3.1486552E-4,-4.9591484E-3,4.9218475E-3,-1.2744595E-2,2.610931E-3,-2.5168944E-2,4.9995854E-3,1.4780336E-2,1.178735E-2,4.1195583E-3,1.1608845E-2,6.2500424E-4,3.5365067E-2,1.909884E-2,1.0415429E-2,1.8356502E-2,8.232271E-3,8.583857E-4,2.5418357E-3,7.7136983E-3,-4.33222E-3,9.221032E-4,-2.1326214E-2,-1.2362959E-2,-5.0422986E-4,7.351437E-3,6.682588E-3,-6.1542606E-3,-5.638642E-3,-1.2767975E-2,4.588389E-3,-2.1914465E-3,-9.278533E-3,3.0090557E-3,-3.5311293E-3,-2.5163524E-2,-4.473403E-3,1.5498279E-3,2.989338E-3,-6.979277E-3,-5.444506E-3,-1.377518E-2,8.6543465E-4,8.678515E-3,2.828645E-3,-4.935842E-3],"split_indices":[18,51,58,27,44,50,51,51,3,52,51,3,3,5,44,53,3,36,0,0,2,3,4,41,0,9,17,51,9,56,44,9,16,44,0,51,3,0,9,1,27,57,0,27,34,5,0,0,32,38,9,44,28,37,0,55,32,0,0,55,16,3,9,46,46,9,7,50,53,7,0,51,44,7,44,0,2,46,0,60,4,52,0,0,0,0,0,53,112,28,9,9,3,0,26,0,0,0,0,0,0,53,0,37,9,0,0,28,0,52,44,2,0,0,0,38,0,11,4,27,3,0,0,56,11,34,11,0,0,0,4,0,0,0,41,0,0,0,0,0,30,0,0,37,26,52,0,0,0,2,2,4,3,0,26,26,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.246E3,1.76E3,4.86E2,1.027E3,7.33E2,2.81E2,2.05E2,7.46E2,2.81E2,6.3E2,1.03E2,4.2E1,2.39E2,1.68E2,3.7E1,5.23E2,2.23E2,2.78E2,3E0,4.32E2,1.98E2,5.6E1,4.7E1,3.8E1,4E0,1.2E1,2.27E2,1.51E2,1.7E1,2.4E1,1.3E1,4.56E2,6.7E1,1.22E2,1.01E2,2.52E2,2.6E1,2.3E1,4.09E2,1.86E2,1.2E1,4.8E1,8E0,2.7E1,2E1,1.6E1,2.2E1,2E0,1E1,7.4E1,1.53E2,1E2,5.1E1,1.3E1,4E0,1.4E1,1E1,7E0,6E0,3.03E2,1.53E2,8E0,5.9E1,1.06E2,1.6E1,1.3E1,8.8E1,1.99E2,5.3E1,2E1,6E0,3.1E1,3.78E2,3.2E1,1.54E2,3E0,9E0,2.4E1,2.4E1,1.6E1,1.1E1,1.6E1,4E0,1.3E1,3E0,3E0,7E0,4E1,3.4E1,1.6E1,1.37E2,8.1E1,1.9E1,1E1,4.1E1,5E0,8E0,1.2E1,2E0,8E0,2E0,3.01E2,2E0,5E1,1.03E2,6E0,2E0,5.5E1,4E0,9.1E1,1.5E1,1.3E1,3E0,1E1,3E0,8.5E1,3E0,1.94E2,5E0,2.2E1,3.1E1,7E0,1.3E1,1.4E1,1.7E1,1.88E2,1.9E2,2.7E1,5E0,9.7E1,5.7E1,3E0,6E0,2E0,2.2E1,1.3E1,3E0,2E0,9E0,6E0,1E1,2E0,2E0,1.4E1,2.6E1,3.1E1,3E0,7E0,9E0,1.19E2,1.8E1,5.2E1,2.9E1,6E0,1.3E1,2.5E1,1.6E1,1.4E1,2.87E2,4.5E1,5E0,8.2E1,2.1E1,1.4E1,4.1E1,5.8E1,3.3E1,1E1,5E0,2E0,1.1E1,3.9E1,4.6E1,1.91E2,3E0,3E0,2E0,3E0,1.9E1,9E0,2.2E1,1.1E1,3E0,8E0,9E0,1.58E2,3E1,1.65E2,2.5E1,6.5E1,3.2E1,1.5E1,4.2E1,9E0,1.3E1,3E0,7E0,2E0,1.2E1,1.1E1,1.5E1,1.4E1,1.7E1,1.04E2,1.5E1,5E0,1.3E1,2.2E1,3E1,3E0,2.6E1,7E0,6E0,2.2E1,3E0,3E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-1.1312908E-3,4.832235E-2,-1.6758035E-1,-4.362211E-2,1.18571214E-1,-2.638097E-1,-2.7561313E-2,-2.2120044E-2,-2.6091394E-1,1.9006044E-2,1.8043098E-1,-2.225876E-1,-4.171295E-1,-5.2317727E-2,2.3391224E-1,4.963244E-2,-7.8848965E-2,-4.4960308E-1,-8.094348E-2,-2.5940502E-2,2.0477083E-1,1.422481E-1,3.098751E-1,-2.567972E-1,-7.496083E-2,-6.1585325E-1,-2.5649974E-1,-3.8985092E-2,-3.4909552E-1,1.43990675E-2,5.724129E-4,7.1369514E-2,-1.04455985E-1,-1.029839E-1,8.350865E-3,-7.386601E-2,-6.170351E-1,-1.3513556E-1,7.0233457E-3,2.0695142E-2,-4.430025E-2,2.3891704E-1,-9.826343E-2,1.6035396E-1,-1.7848395E-1,3.5661325E-2,2.7733725E-1,-3.581857E-1,-2.2861199E-1,7.988798E-3,-1.1708451E-1,-4.0821832E-1,-8.0805874E-1,-9.985239E-2,-4.2570248E-1,-9.094509E-2,1.5140074E-2,-4.0886714E-3,-1.9028908E-2,3.7739627E-2,1.6964918E-1,-1.2993822E-1,6.259438E-3,-1.14409804E-1,5.8700185E-2,5.592696E-2,-9.063477E-2,1.6223945E-5,-9.229014E-3,-3.4996524E-2,-3.9080325E-1,5.3668683E-3,-1.6896537E-1,-1.390442E-2,-1.8198316E-1,2.0291659E-1,3.3375714E-2,-1.7515615E-2,2.826109E-3,1.3158904E-1,3.0354953E-1,-2.2493756E-1,2.7656637E-4,3.3126867E-1,8.366322E-2,-2.3097849E-2,-2.9051504E-1,-2.8098828E-1,-1.354697E-1,-1.9574504E-1,1.3140343E-2,-4.5976123E-1,-2.209045E-3,-4.0158946E-2,-1.1281239E-2,-8.1596365E-3,-6.582368E-5,-4.9077478E-1,-4.143647E-3,-1.5598641E-1,-6.197384E-2,7.0065215E-2,-4.929974E-2,8.235801E-2,-6.402436E-2,-8.293005E-3,1.83537E-1,-2.6272537E-2,-1.9286112E-1,-1.369957E-1,-1.7646054E-2,-2.9144322E-3,5.8820955E-3,8.7383695E-2,-9.751241E-2,-2.3058753E-2,-2.4824546E-1,-4.114219E-3,-2.0253623E-2,-1.4734678E-2,-5.6917993E-3,3.663901E-2,-6.69423E-2,-1.5161815E-1,-2.3019934E-2,2.910684E-1,1.5719676E-1,9.390119E-2,2.0743848E-1,8.634391E-4,3.291222E-1,-5.6569646E-3,-2.953868E-1,3.5175598E-1,1.17563605E-2,2.1535279E-1,-5.3603906E-2,-3.3771378E-1,-5.365156E-3,-2.2827785E-1,-4.6087942E-1,-4.5229856E-2,-1.9089961E-1,-1.638521E-2,-6.648939E-3,8.165047E-3,-4.7636684E-3,-6.329256E-3,-2.2873627E-2,-1.1236688E-2,-2.5488695E-2,-1.19637065E-1,-1.5582082E-2,1.0763277E-2,-1.19274914E-1,1.1546721E-1,-2.3400942E-3,-5.2569313E-3,-1.5896867E-1,4.735875E-3,-4.1213403E-3,1.16387E-4,-5.540417E-3,6.6394494E-3,1.4992805E-2,-7.933603E-3,5.956956E-4,-1.4508968E-2,-6.0193418E-3,-4.2228713E-3,-9.875057E-3,-3.486672E-3,6.937487E-3,6.566441E-3,-1.0363559E-3,-9.1961855E-3,3.0553542E-4,-2.9757223E-3,5.3464686E-3,-1.6038032E-2,-5.1520574E-3,-1.6180829E-4,4.6702595E-3,-4.5704935E-3,3.7559462E-3,-1.0597036E-2,-9.2210877E-4,2.4749458E-4,1.4600769E-2,1.0250497E-2,2.6192137E-3,5.462392E-3,-1.1068576E-3,-1.55967185E-2,1.0387679E-2,1.771169E-2,8.473529E-3,-3.1556985E-3,-1.5293277E-2,1.9732157E-5,1.6828887E-2,7.85127E-3,-6.9198203E-3,1.730575E-2,2.789758E-3,-1.046909E-2,4.561211E-4,-9.419462E-3,-1.9080583E-2,4.520207E-3,-1.0791449E-2,-7.1400264E-3,-2.7218733E-2,2.644369E-3,-5.860457E-3,-5.569008E-3,-1.285565E-2,-3.1407052E-3,-9.113231E-3,-5.1436066E-3,2.4246841E-3,-3.1717012E-3,-8.834348E-3,3.8223462E-3,9.66845E-3,5.913897E-3,-2.2787184E-3,-1.3090549E-2,-1.1174828E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,-1,59,61,63,65,67,69,71,-1,-1,73,75,77,79,81,-1,83,85,87,-1,89,91,93,95,97,99,101,-1,-1,103,105,107,-1,109,111,113,115,-1,-1,-1,117,-1,119,121,123,125,-1,-1,-1,127,129,131,-1,133,135,-1,137,139,141,143,145,147,-1,-1,-1,-1,-1,149,-1,151,153,155,157,159,161,-1,163,165,167,169,171,-1,-1,173,175,177,179,-1,-1,-1,-1,181,183,185,-1,187,189,191,193,-1,195,-1,197,199,201,203,205,207,-1,209,211,213,215,-1,-1,-1,-1,-1,-1,-1,-1,217,-1,219,221,223,-1,225,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8076267E1,1.0940249E1,6.7701206E0,3.4236405E0,5.9092855E0,1.7905617E0,1.3457062E0,2.7270386E0,2.204867E0,3.086272E0,2.8819084E0,1.177661E0,1.8211327E0,7.3423153E-1,3.3105862E-1,9.9888664E-1,7.879665E-1,1.967555E0,4.6872956E-1,2.6631143E0,7.6849484E-1,2.6940947E0,1.8648043E0,4.6678448E-1,5.078252E-1,7.476673E-1,9.1167855E-1,5.132234E-1,1.0946834E-1,0E0,0E0,8.5293245E-1,2.468049E-1,5.480001E-1,3.912356E-1,1.0619937E-1,4.741373E-1,2.6890367E-1,0E0,0E0,1.2027457E0,1.0163949E0,3.9516363E-1,1.7518253E0,2.157489E-1,0E0,1.2988138E0,2.6320648E-1,7.0833635E-1,0E0,4.123143E-1,2.5545478E-1,3.5267162E-1,1.5791488E-1,3.247602E-1,1.6557544E-1,3.2199714E-1,0E0,0E0,8.90679E-1,3.5701513E-1,2.1671325E-1,0E0,5.964086E-1,1.8367454E-1,2.7881628E-1,2.8184474E-1,0E0,0E0,0E0,1.4593089E-1,0E0,1.3504261E-1,6.3778245E-1,4.3840694E-1,2.0412278E-1,0E0,0E0,0E0,1.0210524E0,5.1751375E-1,1.0397291E-1,0E0,6.4391327E-1,5.3576654E-1,0E0,2.0354033E-1,8.1455946E-1,2.764789E-1,1.4992648E-1,3.3562392E-1,1.1848545E-1,0E0,0E0,0E0,0E0,0E0,1.0551667E-1,0E0,1.503573E-1,2.763661E-1,2.768311E-1,2.0202969E-1,5.3412354E-1,2.3346813E-1,0E0,2.9992795E-1,8.453645E-2,1.179319E-1,7.670493E-1,5.494625E-1,0E0,0E0,3.021364E-1,1.1148158E-1,1.2821457E-1,7.369852E-2,0E0,0E0,0E0,0E0,3.266979E-1,5.9975934E-1,5.25471E-1,0E0,1.7776525E-1,2.8256893E-1,7.786038E-1,1.5362854E0,0E0,5.085211E-1,0E0,8.173257E-2,6.518698E-1,2.1277057E-1,3.613001E-1,1.720027E-1,1.6202235E-1,0E0,2.784977E-1,7.9855824E-1,2.083006E-1,1.8618822E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.864629E-2,0E0,1.6575298E-1,1.2447202E-1,9.157732E-2,0E0,1.9644478E-1,2.0262691E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,63,63,64,64,65,65,66,66,70,70,72,72,73,73,74,74,75,75,79,79,80,80,81,81,83,83,84,84,86,86,87,87,88,88,89,89,90,90,91,91,97,97,99,99,100,100,101,101,102,102,103,103,104,104,106,106,107,107,108,108,109,109,110,110,113,113,114,114,115,115,116,116,121,121,122,122,123,123,125,125,126,126,127,127,128,128,130,130,132,132,133,133,134,134,135,135,136,136,137,137,139,139,140,140,141,141,142,142,151,151,153,153,154,154,155,155,157,157,158,158],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,-1,60,62,64,66,68,70,72,-1,-1,74,76,78,80,82,-1,84,86,88,-1,90,92,94,96,98,100,102,-1,-1,104,106,108,-1,110,112,114,116,-1,-1,-1,118,-1,120,122,124,126,-1,-1,-1,128,130,132,-1,134,136,-1,138,140,142,144,146,148,-1,-1,-1,-1,-1,150,-1,152,154,156,158,160,162,-1,164,166,168,170,172,-1,-1,174,176,178,180,-1,-1,-1,-1,182,184,186,-1,188,190,192,194,-1,196,-1,198,200,202,204,206,208,-1,210,212,214,216,-1,-1,-1,-1,-1,-1,-1,-1,218,-1,220,222,224,-1,226,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.6129813E2,1E0,9.669789E0,1.5166431E5,1.2199979E10,6.082581E3,2E0,1.503E3,1.3359244E9,1.3664E4,4.646E3,3.402546E6,1.3521131E8,5.1E1,1.137832E-7,3.30399E6,1.31625E2,1.3528182E-1,8E0,9.70107E7,3.9712732E7,6.737509E7,5.3125258E4,2.1693E5,6.747114E7,1.2E1,6.019605E2,4.1E2,1.43990675E-2,5.724129E-4,1E0,1.048E3,1.5083523E0,1.5E1,3.6E1,8E0,6.4E1,7.0233457E-3,2.0695142E-2,3.791269E7,2.093563E0,9.627E3,3.2196458E6,1.6644816E6,3.5661325E-2,2.8091298E7,1.4713E4,1.7314286E0,7.988798E-3,3.3487454E11,2.653986E6,1E0,1.7384394E7,9.9625344E5,1.4015416E6,3.1826714E7,-4.0886714E-3,-1.9028908E-2,1E0,3.97E2,8.7217E4,6.259438E-3,2.674421E9,2.7302878E2,4.2037987E1,1.470375E1,1.6223945E-5,-9.229014E-3,-3.4996524E-2,2.8132E4,5.3668683E-3,2.183E3,7.273333E1,3.5985E4,1.2469E5,3.3375714E-2,-1.7515615E-2,2.826109E-3,8.8543115E2,6.4547E4,7.987328E-1,2.7656637E-4,1E0,4.1485362E5,-2.3097849E-2,8E0,1.9577E4,2.2065516E5,1.9121015E0,6.135771E9,3.9343938E-1,-2.209045E-3,-4.0158946E-2,-1.1281239E-2,-8.1596365E-3,-6.582368E-5,4.1043E4,-4.143647E-3,9.4801545E-1,5.59E2,2.5086E4,1.1490676E6,1E0,1.6118722E0,-8.293005E-3,7.8955513E-1,2.35E2,1.48404255E1,1.3887E4,2.5956412E2,-2.9144322E-3,5.8820955E-3,9.296418E9,1.2E1,1.7232166E8,2.3737179E1,-4.114219E-3,-2.0253623E-2,-1.4734678E-2,-5.6917993E-3,9.923929E2,7.06699E8,1.7935375E3,-2.3019934E-2,1.4689211E7,1.5548611E7,8.185918E0,6.3157897E0,8.634391E-4,9.1E3,-5.6569646E-3,6.747114E7,4.7326977E4,5.5743E4,5.8019036E-1,2.7753687E-2,1.7770598E-2,-5.365156E-3,1.8722344E8,1.4390244E0,1.3152658E7,1.333145E6,-1.638521E-2,-6.648939E-3,8.165047E-3,-4.7636684E-3,-6.329256E-3,-2.2873627E-2,-1.1236688E-2,-2.5488695E-2,6.822511E5,-1.5582082E-2,9.19E4,9.837297E0,1.1251919E6,-2.3400942E-3,1E0,9.877E3,4.735875E-3,-4.1213403E-3,1.16387E-4,-5.540417E-3,6.6394494E-3,1.4992805E-2,-7.933603E-3,5.956956E-4,-1.4508968E-2,-6.0193418E-3,-4.2228713E-3,-9.875057E-3,-3.486672E-3,6.937487E-3,6.566441E-3,-1.0363559E-3,-9.1961855E-3,3.0553542E-4,-2.9757223E-3,5.3464686E-3,-1.6038032E-2,-5.1520574E-3,-1.6180829E-4,4.6702595E-3,-4.5704935E-3,3.7559462E-3,-1.0597036E-2,-9.2210877E-4,2.4749458E-4,1.4600769E-2,1.0250497E-2,2.6192137E-3,5.462392E-3,-1.1068576E-3,-1.55967185E-2,1.0387679E-2,1.771169E-2,8.473529E-3,-3.1556985E-3,-1.5293277E-2,1.9732157E-5,1.6828887E-2,7.85127E-3,-6.9198203E-3,1.730575E-2,2.789758E-3,-1.046909E-2,4.561211E-4,-9.419462E-3,-1.9080583E-2,4.520207E-3,-1.0791449E-2,-7.1400264E-3,-2.7218733E-2,2.644369E-3,-5.860457E-3,-5.569008E-3,-1.285565E-2,-3.1407052E-3,-9.113231E-3,-5.1436066E-3,2.4246841E-3,-3.1717012E-3,-8.834348E-3,3.8223462E-3,9.66845E-3,5.913897E-3,-2.2787184E-3,-1.3090549E-2,-1.1174828E-3],"split_indices":[18,51,58,53,27,5,51,16,2,7,2,2,1,44,3,36,9,51,37,17,1,44,7,47,28,44,17,51,0,0,0,107,6,41,3,3,16,11,0,0,44,38,0,27,31,0,44,9,52,0,30,1,111,50,27,31,44,0,0,15,2,9,0,5,4,57,55,0,0,0,1,0,2,53,2,9,0,0,0,51,1,26,0,112,27,0,8,9,32,52,5,26,0,0,0,0,0,10,0,26,0,9,27,79,52,0,37,2,45,28,51,0,0,30,3,7,55,0,0,0,0,51,7,54,0,1,44,53,55,0,28,0,44,32,10,40,56,56,0,7,55,44,31,0,0,0,0,0,0,0,0,31,0,1,53,27,0,8,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.194E3,1.692E3,5.02E2,7.33E2,9.59E2,2.97E2,2.05E2,6.68E2,6.5E1,3.68E2,5.91E2,2.36E2,6.1E1,1.88E2,1.7E1,2.95E2,3.73E2,3.1E1,3.4E1,2.97E2,7.1E1,4.58E2,1.33E2,1.91E2,4.5E1,2.6E1,3.5E1,1.81E2,7E0,1.2E1,5E0,2.59E2,3.6E1,2.92E2,8.1E1,1E1,2.1E1,2.8E1,6E0,1E1,2.87E2,6.4E1,7E0,4.34E2,2.4E1,7E0,1.26E2,3.9E1,1.52E2,6E0,3.9E1,1.4E1,1.2E1,1.9E1,1.6E1,9.2E1,8.9E1,2E0,5E0,1.94E2,6.5E1,3.3E1,3E0,2.73E2,1.9E1,5.5E1,2.6E1,7E0,3E0,1.1E1,1E1,3E0,2.5E1,2.36E2,5.1E1,6.1E1,3E0,2E0,5E0,3.63E2,7.1E1,1.9E1,5E0,9.8E1,2.8E1,1E1,2.9E1,9.6E1,5.6E1,2.4E1,1.5E1,1.2E1,2E0,1E1,2E0,1E1,9E0,1.3E1,3E0,2.7E1,6.5E1,4.8E1,4.1E1,1.35E2,5.9E1,2E0,6.3E1,1.3E1,2E1,2.21E2,5.2E1,7E0,1.2E1,4.6E1,9E0,1.9E1,7E0,2E0,8E0,4E0,2.1E1,1.21E2,1.15E2,4.8E1,3E0,1.9E1,4.2E1,2.44E2,1.19E2,6E0,6.5E1,9E0,1E1,9.2E1,6E0,1.4E1,1.4E1,2.2E1,7E0,7.6E1,2E1,2.2E1,3.4E1,4E0,2E1,6E0,9E0,2E0,1E1,4E0,9E0,2.4E1,3E0,2.9E1,3.6E1,3.5E1,1.3E1,3E1,1.1E1,1.2E2,1.5E1,2.8E1,3.1E1,5.2E1,1.1E1,2E0,1.1E1,5E0,1.5E1,1.45E2,7.6E1,3.9E1,1.3E1,3E1,1.6E1,4E0,5E0,1.5E1,4E0,3E0,4E0,7.6E1,4.5E1,9.4E1,2.1E1,2.9E1,1.9E1,2E0,1.7E1,2.4E1,1.8E1,1.99E2,4.5E1,4E0,1.15E2,4.4E1,2.1E1,2E0,8E0,5E0,8.7E1,3E0,3E0,6E0,8E0,3E0,1.1E1,1E1,1.2E1,2E0,7.4E1,7E0,1.3E1,1E1,1.2E1,2.1E1,1.3E1,1.6E1,8E0,7E0,2.2E1,2.3E1,1.3E1,2.8E1,7E0,7E0,2.3E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[-1.2215471E-3,4.1829035E-2,-1.5867159E-1,6.9077974E-3,2.0244406E-1,-2.4567199E-1,-3.8185403E-2,-4.0432587E-2,8.341533E-2,1.6011055E-1,4.4991997E-1,-1.8491939E-1,-3.4451777E-1,-9.903258E-2,5.7927504E-2,-7.150255E-2,2.6825564E-2,4.264262E-2,1.7118558E-1,1.170468E-1,3.6517245E-1,7.418052E-1,2.7336535E-1,-7.8067675E-2,-2.251142E-1,-4.1212484E-1,-1.6012327E-1,-7.5015485E-2,-3.3238393E-1,1.1122607E-1,-1.6978829E-1,-7.739865E-2,3.700614E-1,-3.558814E-2,7.907424E-2,-3.723156E-3,1.2269409E-1,4.5709112E-1,1.4460047E-1,1.781421E-1,3.6069278E-2,4.1149804E-1,1.1096102E-2,3.7312265E-2,1.792081E-2,7.4946694E-2,4.5355964E-1,-1.3908074E-2,-5.0255857E-2,-4.559034E-1,-1.9128895E-1,-4.4057074E-1,-7.3839934E-4,-2.3684312E-1,-6.100606E-2,-1.6897187E-1,-5.74126E-2,-2.1202153E-2,-8.502987E-3,7.038745E-2,1.34863155E-2,-2.1554208E-3,-2.7074462E-1,-1.03804596E-1,-1.1460744E-3,2.0975428E-2,2.7268524E-3,7.3972665E-2,-8.9305386E-2,1.41566E-1,-4.5383745E-4,-6.500397E-2,1.0056746E-1,9.354912E-2,2.6814106E-1,2.3128532E-2,4.0593715E-3,-3.4559824E-2,1.6951789E-1,2.155779E-1,6.83774E-2,-9.851034E-3,1.7232732E-1,1.0013484E-2,2.02976E-2,2.139697E-1,-1.1132558E-2,2.603741E-2,1.3972741E-2,4.0930633E-2,-1.1495922E-1,-2.5303936E-2,-6.2529803E-3,-1.7680515E-1,-2.9161615E-2,-3.5742253E-1,-6.7388165E-1,-3.1840885E-3,-1.3860059E-2,8.4612286E-4,-6.4909677E-3,-8.908391E-3,1.1918234E-3,1.4683981E-3,-8.180404E-2,1.8866819E-1,3.491706E-2,-1.50546525E-2,-2.5778343E-3,-1.2320423E-1,-1.4578313E-2,1.08123966E-1,-1.2732913E-1,1.0913278E-1,-1.586222E-3,5.666162E-2,-1.321847E-1,1.8427338E-1,1.2832715E-2,-6.3305777E-3,3.9000317E-2,-4.6411503E-2,-3.1693888E-1,4.6559986E-2,1.8338618E-1,1.8123688E-1,6.1499916E-2,5.4288814E-3,1.4909643E-2,4.2428255E-2,-1.2405502E-2,1.9463706E-1,1.1583005E-2,3.2249394E-1,1.8521327E-1,-9.125475E-3,1.334701E-1,-1.6023692E-2,5.035827E-3,2.4048232E-1,-1.2471415E-3,-2.1744438E-3,1.256479E-2,-2.0660101E-3,5.5868835E-3,-3.888263E-3,-1.7353277E-1,-3.483257E-1,-1.3820183E-1,-4.1540694E-1,-2.5683507E-1,-7.2382545E-1,-9.033751E-3,-1.6826883E-2,-1.1395608E-1,2.466303E-3,1.1271192E-2,-7.653572E-4,5.4412275E-3,-4.4112247E-3,-8.761472E-3,3.7641264E-3,-3.7522889E-3,1.5009568E-2,4.5115643E-4,-9.38104E-3,-7.958242E-4,-2.8101099E-3,6.4803166E-3,-6.903076E-4,1.0276753E-2,-4.3579265E-3,-1.3721333E-2,6.32773E-3,1.8389795E-2,-1.2741996E-3,9.24788E-3,3.4037773E-3,-2.6110196E-3,-7.748841E-3,-2.6991186E-4,-7.2379857E-3,-2.8528953E-2,-8.783014E-3,2.9208155E-3,-2.685305E-3,9.857182E-3,1.2915232E-2,5.2045863E-3,1.6956768E-3,9.265763E-3,1.0597671E-2,-8.9664385E-4,9.777176E-3,2.8530343E-3,-5.836738E-3,3.0339062E-3,5.456351E-3,1.5974093E-2,1.3188537E-2,5.9605874E-3,-3.737985E-3,3.3789405E-3,7.2650234E-3,-5.1602787E-5,-1.7535131E-3,4.0989392E-3,7.1506337E-3,1.9173529E-2,-4.5939786E-3,7.59937E-3,-9.174767E-4,-1.0671322E-2,-1.8198721E-2,-2.1129989E-3,-9.4687176E-4,-7.882892E-3,-2.2680871E-2,-1.3515798E-2,-1.2737112E-2,-3.5745863E-4,-1.5051846E-2,-3.5945762E-2,7.814431E-4,-5.0623096E-3,-7.326742E-3,-3.0414914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,-1,-1,85,87,-1,89,91,93,95,-1,97,99,101,103,-1,-1,105,-1,-1,107,109,111,-1,-1,113,115,117,119,121,123,125,127,-1,-1,129,131,133,135,137,139,-1,-1,141,-1,-1,-1,143,145,-1,-1,147,-1,149,151,-1,-1,-1,-1,-1,-1,-1,153,155,157,-1,-1,159,161,163,165,167,-1,169,171,173,175,-1,177,179,181,183,185,187,189,-1,-1,191,-1,193,195,197,199,201,203,-1,205,207,-1,-1,-1,-1,-1,209,211,213,215,217,219,221,-1,223,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4742695E1,9.578561E0,4.8858747E0,5.0885143E0,3.1233044E0,1.5636845E0,1.1578319E0,1.8157849E0,1.9144251E0,2.2709098E0,2.0646982E0,7.1255684E-1,1.2055779E0,6.532011E-1,9.5342803E-1,1.5751235E0,8.9959824E-1,1.3663433E0,1.2260456E0,1.0720959E0,1.4948177E-1,1.2957478E-1,1.0028708E0,2.904432E-1,8.7299013E-1,8.26005E-1,2.060746E-1,1.7388213E-1,1.2820518E-1,4.5926857E-1,1.7213279E-1,1.1818182E0,2.0320177E-1,7.4815446E-1,7.4554163E-1,1.5020344E0,5.471873E-1,2.3076892E-1,7.067399E-1,4.947915E-1,5.961138E-1,1.1814022E-1,0E0,0E0,0E0,7.216757E-1,8.675218E-2,0E0,2.6430854E-1,4.138093E-1,6.010082E-1,1.0976982E0,0E0,1.6308528E-1,9.457643E-2,9.745878E-2,2.1084368E-1,0E0,0E0,2.1577185E-1,0E0,0E0,8.67362E-2,7.5290155E-1,2.1095474E0,0E0,0E0,1.6479954E-1,5.3950185E-1,4.5945573E-1,3.738796E-1,6.7853606E-1,3.8059777E-1,3.0923074E-1,1.6001976E-1,0E0,0E0,3.803215E-1,5.45218E-1,2.38945E-1,1.6769877E-1,3.7058198E-1,3.3356208E-1,0E0,0E0,1.9650322E-1,0E0,0E0,0E0,1.4127307E-1,1.6847658E-1,0E0,0E0,6.60336E-1,0E0,2.024312E-1,2.5831223E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5476859E-1,8.351961E-2,1.8761869E-1,0E0,0E0,6.2986946E-1,5.341657E-1,1.7948804E0,6.18786E-1,1.9516769E-1,0E0,2.484244E-1,3.7604547E-1,5.6548214E-1,1.7832355E-1,0E0,1.8971568E-1,7.003492E-1,3.9119864E-1,2.4574314E-1,2.962531E-1,1.740604E-1,2.849435E-1,0E0,0E0,1.9546802E-1,0E0,3.2030344E-1,1.6439906E-1,7.39001E-2,3.8211536E-1,1.0457944E-1,6.559834E-2,0E0,2.6545376E-1,2.0502806E-1,0E0,0E0,0E0,0E0,0E0,1.8462148E-1,1.5721792E-1,2.8869176E-1,3.7124586E-1,1.7044306E-1,1.3195288E-1,1.477685E-1,0E0,8.61149E-2,1.0109973E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,45,45,46,46,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,59,59,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81,82,82,85,85,89,89,90,90,93,93,95,95,96,96,104,104,105,105,106,106,109,109,110,110,111,111,112,112,113,113,115,115,116,116,117,117,118,118,120,120,121,121,122,122,123,123,124,124,125,125,126,126,129,129,131,131,132,132,133,133,134,134,135,135,136,136,138,138,139,139,145,145,146,146,147,147,148,148,149,149,150,150,151,151,153,153,154,154],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,-1,-1,86,88,-1,90,92,94,96,-1,98,100,102,104,-1,-1,106,-1,-1,108,110,112,-1,-1,114,116,118,120,122,124,126,128,-1,-1,130,132,134,136,138,140,-1,-1,142,-1,-1,-1,144,146,-1,-1,148,-1,150,152,-1,-1,-1,-1,-1,-1,-1,154,156,158,-1,-1,160,162,164,166,168,-1,170,172,174,176,-1,178,180,182,184,186,188,190,-1,-1,192,-1,194,196,198,200,202,204,-1,206,208,-1,-1,-1,-1,-1,210,212,214,216,218,220,222,-1,224,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,8.631E3,1E0,4.930349E5,8.791777E3,8.228668E-1,1.067536E3,1.4887473E-1,6.7699623E-1,1.4813511E6,4.0601485E6,4.01E2,2.4019E6,6.8651576E9,8.965631E7,1.5137865E2,2.960909E2,1.113742E6,1.3841E4,2.0189162E7,2.0568174E7,2.0399538E5,2.8332526E9,7.9555137E3,7.55E2,9.099893E8,3.888319E-1,1.06E3,9E0,4.5575E4,1.272E3,7.2041174E2,2.0827537E0,1.7997802E0,2.2E1,2.462277E6,1.8071064E7,7.131107E6,7.118233E7,3.6848484E1,1.5137865E2,1E0,1.1096102E-2,3.7312265E-2,1.792081E-2,1.4285097E1,1.772114E6,-1.3908074E-2,4.6268812E5,2.6E1,8.609194E6,7.8351054E11,-7.3839934E-4,6.75784E5,1.0963991E7,6.1E1,3.52E2,-2.1202153E-2,-8.502987E-3,1.18827E5,1.34863155E-2,-2.1554208E-3,1.0526E4,4.264897E6,2.3058404E7,2.0975428E-2,2.7268524E-3,1.8174828E6,9.25E0,2.970914E1,9.342733E7,7.84892E5,2.9254214E2,1.4634147E0,1E0,2.3128532E-2,4.0593715E-3,1.3171996E0,1E0,3.1126543E1,1.5963264E9,1.0972222E0,3.526947E7,1.0013484E-2,2.02976E-2,1.5E1,-1.1132558E-2,2.603741E-2,1.3972741E-2,3.790491E2,2.6432162E5,-2.5303936E-2,-6.2529803E-3,9.417789E1,-2.9161615E-2,3.0823356E5,5.8530144E10,-3.1840885E-3,-1.3860059E-2,8.4612286E-4,-6.4909677E-3,-8.908391E-3,1.1918234E-3,1.4683981E-3,6.015825E0,4.366E3,3.368326E11,-1.50546525E-2,-2.5778343E-3,4.8718534E0,3.4764814E3,1.33832655E1,1.6E1,8.777E3,-1.586222E-3,1.4735735E2,9.669789E0,6.341735E2,2.1E1,-6.3305777E-3,8.747313E6,3.6887734E-5,4.0058823E2,5.8631687E1,1.5139E4,2.4481E4,3.1100148E6,5.4288814E-3,1.4909643E-2,6E0,-1.2405502E-2,2.0295714E7,1.3860265E6,2.0757051E-1,1.9577E4,1.2679E4,2.8352614E9,-1.6023692E-2,1.138871E0,5.528107E3,-1.2471415E-3,-2.1744438E-3,1.256479E-2,-2.0660101E-3,5.5868835E-3,2.091623E5,2.824856E-3,1.7314286E0,1.775044E7,1E0,6.5884055E11,1.4410892E-5,-9.033751E-3,9.912942E6,3.3502703E5,2.466303E-3,1.1271192E-2,-7.653572E-4,5.4412275E-3,-4.4112247E-3,-8.761472E-3,3.7641264E-3,-3.7522889E-3,1.5009568E-2,4.5115643E-4,-9.38104E-3,-7.958242E-4,-2.8101099E-3,6.4803166E-3,-6.903076E-4,1.0276753E-2,-4.3579265E-3,-1.3721333E-2,6.32773E-3,1.8389795E-2,-1.2741996E-3,9.24788E-3,3.4037773E-3,-2.6110196E-3,-7.748841E-3,-2.6991186E-4,-7.2379857E-3,-2.8528953E-2,-8.783014E-3,2.9208155E-3,-2.685305E-3,9.857182E-3,1.2915232E-2,5.2045863E-3,1.6956768E-3,9.265763E-3,1.0597671E-2,-8.9664385E-4,9.777176E-3,2.8530343E-3,-5.836738E-3,3.0339062E-3,5.456351E-3,1.5974093E-2,1.3188537E-2,5.9605874E-3,-3.737985E-3,3.3789405E-3,7.2650234E-3,-5.1602787E-5,-1.7535131E-3,4.0989392E-3,7.1506337E-3,1.9173529E-2,-4.5939786E-3,7.59937E-3,-9.174767E-4,-1.0671322E-2,-1.8198721E-2,-2.1129989E-3,-9.4687176E-4,-7.882892E-3,-2.2680871E-2,-1.3515798E-2,-1.2737112E-2,-3.5745863E-4,-1.5051846E-2,-3.5945762E-2,7.814431E-4,-5.0623096E-3,-7.326742E-3,-3.0414914E-3],"split_indices":[18,2,58,27,51,26,51,37,41,27,46,0,31,5,44,55,51,9,9,44,46,32,7,49,10,7,38,2,8,2,0,51,52,52,3,27,44,44,7,53,55,109,0,0,0,53,28,0,31,3,28,30,0,12,46,3,0,0,0,28,0,0,9,9,44,0,0,50,57,57,5,9,54,55,15,0,0,41,6,57,7,52,49,0,0,3,0,0,0,4,27,0,0,51,0,32,5,0,0,0,0,0,0,0,52,2,30,0,0,52,47,52,17,1,0,51,53,51,3,0,44,37,4,46,9,12,27,0,0,3,0,49,27,40,9,2,5,0,40,51,0,0,0,0,0,27,41,52,44,15,30,38,0,44,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.173E3,1.707E3,4.66E2,1.403E3,3.04E2,2.7E2,1.96E2,8.67E2,5.36E2,2.61E2,4.3E1,1.69E2,1.01E2,1.2E2,7.6E1,5.93E2,2.74E2,3.67E2,1.69E2,2.17E2,4.4E1,1.5E1,2.8E1,4.7E1,1.22E2,7.3E1,2.8E1,1.1E2,1E1,6.2E1,1.4E1,5.86E2,7E0,1.25E2,1.49E2,2.33E2,1.34E2,1.3E1,1.56E2,1.23E2,9.4E1,3E1,1.4E1,1.1E1,4E0,1.4E1,1.4E1,4E0,4.3E1,1.4E1,1.08E2,6.8E1,5E0,1.5E1,1.3E1,1.6E1,9.4E1,4E0,6E0,5.2E1,1E1,7E0,7E0,4.35E2,1.51E2,5E0,2E0,4.1E1,8.4E1,8.3E1,6.6E1,1.47E2,8.6E1,1.13E2,2.1E1,1.1E1,2E0,1.9E1,1.37E2,9.1E1,3.2E1,7.1E1,2.3E1,6E0,2.4E1,1E1,4E0,6E0,8E0,1.8E1,2.5E1,1E1,4E0,1.06E2,2E0,5.2E1,1.6E1,5E0,1E1,7E0,6E0,1.4E1,2E0,2E1,7.4E1,1.1E1,4.1E1,5E0,2E0,3.57E2,7.8E1,8.1E1,7E1,3.1E1,1E1,1.9E1,6.5E1,6.2E1,2.1E1,1.4E1,5.2E1,1.38E2,9E0,5.3E1,3.3E1,2.9E1,8.4E1,7E0,1.4E1,1.5E1,4E0,1.18E2,1.9E1,1.8E1,7.3E1,1.5E1,1.7E1,2E0,6.9E1,1.7E1,6E0,2E0,8E0,9E0,9E0,9E0,1.6E1,1.8E1,8.8E1,3.1E1,2.1E1,1.4E1,2E0,2.5E1,4.9E1,4E0,7E0,2.6E1,1.5E1,2.63E2,9.4E1,3.2E1,4.6E1,2.4E1,5.7E1,4E1,3E1,5E0,2.6E1,1.4E1,5E0,5.5E1,1E1,5.3E1,9E0,1.8E1,3E0,3.8E1,1.4E1,3.3E1,1.05E2,7E0,2E0,3E0,5E1,4E0,2.9E1,1E1,1.9E1,7.3E1,1.1E1,3E0,1.2E1,1.01E2,1.7E1,5E0,1.4E1,3E0,1.5E1,2.3E1,5E1,8E0,7E0,1.4E1,3E0,4.6E1,2.3E1,1.3E1,4E0,6E0,3E0,5E0,1.1E1,1.5E1,3E0,2.1E1,6.7E1,1.6E1,1.5E1,1.9E1,2E0,3E0,1.1E1,1.9E1,6E0,2.3E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[-1.7208812E-3,-1.1579152E-1,5.1854327E-2,-1.9027619E-1,-4.6816997E-2,2.3089772E-2,1.8273805E-1,-1.4880584E-1,-3.3346447E-1,-8.235581E-2,1.2823555E-1,-2.3973238E-2,1.03909634E-1,2.3548041E-1,-8.42204E-3,-1.779683E-1,-2.1804674E-2,-4.0467075E-1,-1.517861E-1,-9.106548E-2,2.736914E-2,7.9978565E-3,1.8816398E-1,-1.4649056E-1,2.6874527E-4,1.3034408E-1,-3.7290413E-2,2.1293889E-1,5.422923E-1,5.3958535E-2,-4.206406E-1,-1.312349E-1,-2.4123393E-1,3.5164446E-2,-1.6087486E-1,-2.0742181E-1,-4.8226446E-1,2.4410784E-4,-1.9574617E-1,-2.1216302E-1,-5.7523977E-2,5.1215284E-2,-5.242021E-3,1.22532584E-1,3.6198586E-1,-1.08028926E-1,-4.3416113E-1,9.914609E-3,-1.8873829E-1,5.5321485E-2,1.8493314E-1,-1.8966301E-1,4.1211735E-2,3.654753E-1,1.7277162E-1,6.9827425E-1,1.2973558E-2,-1.1171083E-1,1.3529436E-1,-2.403516E-2,-8.044173E-3,-2.0671853E-1,-8.2754724E-2,-1.9254415E-1,-4.5743328E-1,1.6725269E-3,1.391757E-1,-1.3680694E-2,-3.7134667E-3,-1.2736895E-2,-2.5036195E-2,-3.7778714E-1,-7.370222E-1,-3.0356667E-3,-2.4008651E-1,-1.6170995E-1,-3.5972196E-1,-3.2326136E-2,-1.5076445E-1,7.755188E-3,-1.3516459E-3,9.640968E-3,7.175553E-2,2.0854529E-2,6.858422E-3,-1.4561222E-1,3.4598768E-2,-2.3152411E-3,-4.8915395E-1,-7.7881557E-3,1.2288739E-1,-2.177375E-1,5.163456E-4,-8.765586E-3,7.1024805E-2,2.1251309E-1,7.111659E-2,4.414971E-3,-2.4900758E-1,7.834764E-2,-7.194176E-3,3.8680595E-1,-1.5233946E-3,1.29274E-1,2.9293692E-1,1.1648436E-2,3.6327295E-2,-1.876068E-1,3.230246E-3,3.21529E-1,6.1307676E-2,-2.1709707E-2,-1.6482036E-1,9.535917E-3,-9.311045E-2,-2.461048E-1,-4.9890503E-2,-1.0363839E-2,-2.4470357E-2,-6.587997E-2,2.41002E-3,-1.3983035E-3,8.700602E-3,-5.176355E-3,4.6348316E-3,-5.3273386E-1,-1.9765462E-1,-1.1942911E-2,-3.7231293E-2,-2.830109E-3,-1.4128343E-2,7.338555E-4,-1.8579271E-1,-4.562931E-3,-1.7634839E-2,-1.1499636E-1,5.496133E-3,-2.1220276E-1,-6.0132E-2,1.2537965E-3,8.359316E-3,-9.4416216E-2,-3.142075E-1,2.6890773E-3,-7.562736E-3,-7.3599657E-3,-2.7899697E-2,-1.3167761E-2,2.8026158E-1,3.9064708E-1,6.458763E-2,-1.2393916E-2,-5.9782113E-3,2.0408677E-1,4.6293654E-2,2.3857611E-1,1.5673561E-2,-5.1944017E-2,1.2836796E-1,-2.883781E-1,3.7731961E-3,1.5099409E-1,-1.753987E-2,7.8120707E-3,4.28282E-1,1.5382141E-1,2.9840944E-2,3.32947E-1,-7.7144797E-3,-1.031082E-2,-1.8274753E-3,6.144211E-3,1.9141303E-2,2.1433151E-1,-1.4404015E-2,-1.2889226E-2,-5.4867505E-3,-6.5835966E-3,-2.5638358E-3,-2.7522482E-2,-9.667838E-3,4.7948677E-3,-5.2035223E-3,1.7326403E-3,-5.2708453E-3,-1.1263606E-2,-2.639417E-2,-1.1064977E-2,3.2052107E-3,-1.2497431E-2,-4.3157157E-3,-3.7075137E-3,-1.4485265E-2,1.0052584E-3,-7.3992074E-3,-1.2039402E-2,-4.3117874E-3,9.601597E-3,-5.994504E-3,-6.0409484E-3,2.2852025E-3,-3.8712856E-3,-1.9125957E-2,-1.7039364E-3,2.2249273E-3,1.6601738E-2,5.455014E-3,2.024993E-2,5.573224E-3,5.8041895E-3,-3.3387626E-3,3.4995892E-3,1.570557E-2,4.3579354E-3,-9.716872E-5,8.649683E-3,1.4784926E-2,-2.1729723E-3,8.758636E-3,5.8116147E-4,-8.333688E-3,1.1285148E-2,3.0596354E-3,-1.5135634E-2,-1.4735383E-3,7.7257743E-3,-1.9668886E-3,-5.430492E-3,2.3630043E-3,2.044731E-2,6.8771434E-3,2.5152662E-3,7.95851E-3,-9.509675E-3,3.2642789E-3,8.097105E-3,2.0668296E-2,6.0644053E-4,1.2216197E-2,1.1762524E-3,-1.0795565E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,75,77,79,-1,81,83,85,87,89,91,93,95,97,99,101,103,105,-1,107,109,-1,-1,111,113,115,117,119,121,-1,-1,-1,123,125,127,-1,129,131,133,135,137,-1,-1,-1,139,-1,-1,141,143,-1,145,147,149,151,-1,-1,153,155,157,-1,159,161,-1,163,-1,165,167,-1,-1,169,-1,171,173,-1,175,-1,177,179,181,-1,-1,183,-1,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,189,-1,-1,191,193,195,197,-1,-1,199,201,-1,-1,-1,-1,203,205,207,209,-1,-1,211,213,215,217,219,221,223,-1,225,227,-1,229,231,233,235,-1,-1,-1,-1,-1,237,239,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3616E1,3.649024E0,5.7032356E0,1.9811974E0,2.3186114E0,4.7342014E0,2.7544336E0,9.849982E-1,9.217615E-1,1.9036398E0,4.5245135E-1,2.3374786E0,1.714366E0,1.3645506E0,1.565563E0,6.0992765E-1,4.097151E-1,7.264528E-1,1.5997505E-1,1.2307041E0,0E0,1.2362013E-1,4.3478823E-1,1.3914537E0,1.2014948E0,1.5705991E0,8.810115E-1,1.1683769E0,3.5836697E-1,7.292812E-1,1.19951844E-1,4.4764447E-1,8.698535E-1,1.2865807E-1,1.4234433E-1,2.2211128E-1,7.329855E-1,0E0,8.661693E-2,4.331863E-1,5.6134593E-1,1.7060643E-1,0E0,1.3163176E-1,1.5862918E-1,6.268468E-1,2.909379E-1,1.2549136E0,1.8821108E-1,6.54841E-1,6.848955E-1,4.5034653E-1,3.7659067E-1,3.62072E-1,8.036628E-1,1.6085911E-1,0E0,2.6346838E-1,4.7632766E-1,0E0,0E0,4.810276E-1,2.5962776E-1,5.684092E-1,1.7997336E-1,1.0451295E-1,8.705622E-2,0E0,0E0,0E0,7.996898E-2,7.041197E-1,1.8689299E-1,0E0,1.4665025E-1,2.2153556E-1,8.4171295E-2,5.9784615E-1,2.712233E-1,0E0,0E0,0E0,1.0431137E-1,0E0,0E0,7.644228E-1,1.3492459E-1,0E0,4.3939137E-1,8.4511995E-1,1.3016468E0,9.825671E-2,0E0,0E0,5.0062233E-1,9.145727E-1,3.2371038E-1,0E0,2.988137E-1,2.9521573E-1,0E0,2.7267027E-1,0E0,2.8830862E-1,8.353269E-1,0E0,0E0,6.76336E-2,0E0,1.307801E-1,3.1590095E-1,0E0,1.9562376E-1,0E0,1.386962E-1,5.181854E-1,2.351137E-1,0E0,0E0,7.288574E-2,0E0,0E0,0E0,0E0,0E0,6.9001675E-2,2.0553708E-1,0E0,0E0,0E0,0E0,0E0,3.4109056E-1,0E0,0E0,3.8090134E-1,3.7618703E-1,1.6317713E-1,4.6089622E-1,0E0,0E0,4.137438E-1,4.8372984E-1,0E0,0E0,0E0,0E0,8.160175E-1,9.391427E-2,1.7220163E-1,6.342491E-1,0E0,0E0,4.077583E-1,3.1967562E-1,5.983057E-1,2.5831455E-1,1.307532E-1,2.1097183E-1,2.1239495E-1,0E0,9.9115014E-2,1.4194362E-1,0E0,1.3793707E-1,1.974318E-1,2.6858455E-1,6.609936E-1,0E0,0E0,0E0,0E0,0E0,9.6747935E-2,1.7810793E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,82,82,85,85,86,86,88,88,89,89,90,90,91,91,94,94,95,95,96,96,98,98,99,99,101,101,103,103,104,104,107,107,109,109,110,110,112,112,114,114,115,115,116,116,119,119,125,125,126,126,132,132,135,135,136,136,137,137,138,138,141,141,142,142,147,147,148,148,149,149,150,150,153,153,154,154,155,155,156,156,157,157,158,158,159,159,161,161,162,162,164,164,165,165,166,166,167,167,173,173,174,174],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,76,78,80,-1,82,84,86,88,90,92,94,96,98,100,102,104,106,-1,108,110,-1,-1,112,114,116,118,120,122,-1,-1,-1,124,126,128,-1,130,132,134,136,138,-1,-1,-1,140,-1,-1,142,144,-1,146,148,150,152,-1,-1,154,156,158,-1,160,162,-1,164,-1,166,168,-1,-1,170,-1,172,174,-1,176,-1,178,180,182,-1,-1,184,-1,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,190,-1,-1,192,194,196,198,-1,-1,200,202,-1,-1,-1,-1,204,206,208,210,-1,-1,212,214,216,218,220,222,224,-1,226,228,-1,230,232,234,236,-1,-1,-1,-1,-1,238,240,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.689559E0,9.075E3,3.1066042E7,1.7897441E0,6.285983E5,3.5301748E7,2.5604828E2,5.121E3,2.14099E5,2.4513194E5,3.3820656E7,2E1,4.5505118E2,8.6206274E2,6.8503425E6,1.6524802E6,2.8035452E11,1.8357558E5,2.368E0,2.736914E-2,1.3653578E7,1.3712785E-1,2E0,7.218466E7,3.2069644E2,1.0738284E4,2E0,1.24681935E-1,5.094239E3,3E1,2.308943E0,3.290375E-2,9.479827E0,7.5569354E2,1E0,8.965631E7,2.4410784E-4,1.2E1,5.8871865E-1,6.747114E7,2.62888E-3,-5.242021E-3,5.015707E0,2.093563E0,7.2E1,7.9037575E1,1.1410706E3,2.3276744E5,3.79E2,2E0,1.4286339E0,2.03353E-6,6.404426E7,5.847845E3,1.6E1,1.2973558E-2,9.74026E0,6.135771E9,-2.403516E-2,-8.044173E-3,5.23E2,2.08E2,4.7436612E2,8.3479E4,4.387838E-1,2.090909E0,-1.3680694E-2,-3.7134667E-3,-1.2736895E-2,1.732E3,3.1294638E2,1.2199979E10,-3.0356667E-3,8.631404E2,8.0469E4,5.2E1,8.677148E2,3.7930825E5,7.755188E-3,-1.3516459E-3,9.640968E-3,7.7854166E0,2.0854529E-2,6.858422E-3,6E0,8.2056335E1,-2.3152411E-3,4.571429E0,1.3238013E2,4.714286E0,5.1232258E1,5.163456E-4,-8.765586E-3,7E0,1.4569893E0,1.829E3,4.414971E-3,2.4519731E3,4.5865917E8,-7.194176E-3,1E0,-1.5233946E-3,1.9051096E7,1.314806E8,1.1648436E-2,3.6327295E-2,2.0676695E-1,3.230246E-3,9.35E2,1.2424786E2,-2.1709707E-2,1.13814086E5,9.535917E-3,1.09E3,1.8991614E5,1E1,-1.0363839E-2,-2.4470357E-2,4.7931477E4,2.41002E-3,-1.3983035E-3,8.700602E-3,-5.176355E-3,4.6348316E-3,1.8255814E0,6.4446027E2,-1.1942911E-2,-3.7231293E-2,-2.830109E-3,-1.4128343E-2,7.338555E-4,3.0300303E9,-4.562931E-3,-1.7634839E-2,2.0724944E7,1.5793961E1,4.6726016E3,1E0,1.2537965E-3,8.359316E-3,1E0,8.393294E8,2.6890773E-3,-7.562736E-3,-7.3599657E-3,-2.7899697E-2,4.264897E6,1.8631586E4,1.6E1,6.963859E1,-1.2393916E-2,-5.9782113E-3,1E0,1.5182724E0,1.3572903E10,4.2E1,1.3058E4,1.0315458E8,2.3084E4,3.7731961E-3,3.8177986E1,1.7801905E5,7.8120707E-3,4.7E1,1.8835404E3,2.316491E-2,1.01268125E5,-7.7144797E-3,-1.031082E-2,-1.8274753E-3,6.144211E-3,1.9141303E-2,1.0057E4,1.6956E5,-1.2889226E-2,-5.4867505E-3,-6.5835966E-3,-2.5638358E-3,-2.7522482E-2,-9.667838E-3,4.7948677E-3,-5.2035223E-3,1.7326403E-3,-5.2708453E-3,-1.1263606E-2,-2.639417E-2,-1.1064977E-2,3.2052107E-3,-1.2497431E-2,-4.3157157E-3,-3.7075137E-3,-1.4485265E-2,1.0052584E-3,-7.3992074E-3,-1.2039402E-2,-4.3117874E-3,9.601597E-3,-5.994504E-3,-6.0409484E-3,2.2852025E-3,-3.8712856E-3,-1.9125957E-2,-1.7039364E-3,2.2249273E-3,1.6601738E-2,5.455014E-3,2.024993E-2,5.573224E-3,5.8041895E-3,-3.3387626E-3,3.4995892E-3,1.570557E-2,4.3579354E-3,-9.716872E-5,8.649683E-3,1.4784926E-2,-2.1729723E-3,8.758636E-3,5.8116147E-4,-8.333688E-3,1.1285148E-2,3.0596354E-3,-1.5135634E-2,-1.4735383E-3,7.7257743E-3,-1.9668886E-3,-5.430492E-3,2.3630043E-3,2.044731E-2,6.8771434E-3,2.5152662E-3,7.95851E-3,-9.509675E-3,3.2642789E-3,8.097105E-3,2.0668296E-2,6.0644053E-4,1.2216197E-2,1.1762524E-3,-1.0795565E-2],"split_indices":[19,55,2,44,40,27,44,51,2,2,27,7,3,55,57,44,46,30,31,57,0,31,33,6,44,51,46,16,56,4,3,53,37,56,4,15,44,0,8,26,44,26,0,53,38,3,32,51,32,2,16,52,36,43,4,3,0,52,5,0,0,28,2,4,28,41,55,0,0,0,2,51,5,0,51,28,10,54,27,0,0,0,53,0,0,8,50,0,53,55,52,55,0,0,3,45,2,0,4,5,0,109,0,44,1,0,0,38,0,0,55,0,32,0,2,27,3,0,0,46,0,0,0,0,0,53,51,0,0,0,0,0,5,0,0,44,56,4,66,0,0,109,30,0,0,0,0,9,32,8,55,0,0,108,52,30,6,28,7,9,0,55,32,0,8,4,37,27,0,0,0,0,0,9,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.226E3,7.11E2,1.515E3,3.41E2,3.7E2,1.243E3,2.72E2,2.66E2,7.5E1,3.08E2,6.2E1,7.86E2,4.57E2,2.13E2,5.9E1,2.16E2,5E1,5.3E1,2.2E1,3.05E2,3E0,2.1E1,4.1E1,1.29E2,6.57E2,3.85E2,7.2E1,2E2,1.3E1,5.2E1,7E0,1.26E2,9E1,3.6E1,1.4E1,1.6E1,3.7E1,5E0,1.7E1,6.5E1,2.4E2,1.6E1,5E0,3.1E1,1E1,1.15E2,1.4E1,6.26E2,3.1E1,1.63E2,2.22E2,2.4E1,4.8E1,4E1,1.6E2,7E0,6E0,1.7E1,3.5E1,4E0,3E0,4.8E1,7.8E1,7.5E1,1.5E1,2.8E1,8E0,4E0,1E1,1.1E1,5E0,2.8E1,9E0,5E0,1.2E1,5E1,1.5E1,1.9E2,5E1,6E0,1E1,1E1,2.1E1,6E0,4E0,9.1E1,2.4E1,2E0,1.2E1,5.42E2,8.4E1,2.7E1,4E0,9E0,1.54E2,1.78E2,4.4E1,4E0,2E1,4.1E1,7E0,3.8E1,2E0,1.19E2,4.1E1,2E0,5E0,1.2E1,5E0,9E0,2.6E1,5E0,4.3E1,2E0,7.6E1,5.4E1,2.1E1,5E0,1E1,1.2E1,1.6E1,2E0,6E0,3E0,2E0,1.4E1,1.4E1,2E0,7E0,4E0,8E0,6E0,4.4E1,2E0,1.3E1,5.9E1,1.31E2,2.9E1,2.1E1,1.6E1,5E0,7.1E1,2E1,2.2E1,2E0,4E0,8E0,5.33E2,9E0,1.4E1,7E1,1.5E1,1.2E1,2.3E1,1.31E2,1.57E2,2.1E1,1.4E1,3E1,1.8E1,2E0,2.3E1,1.8E1,7E0,3.1E1,9.5E1,2.4E1,3.8E1,3E0,9E0,3E0,4E0,5E0,8E0,1.8E1,1E1,3.3E1,3E1,4.6E1,3E0,5.1E1,6E0,1.5E1,4E0,8E0,3E0,1.1E1,1.2E1,2E0,2.1E1,2.3E1,5.2E1,7E0,1.2E2,1.1E1,1.9E1,1E1,4E0,1.7E1,5.6E1,1.5E1,7E0,1.3E1,3.83E2,1.5E2,5E0,4E0,1.1E1,3E0,4.8E1,2.2E1,1.3E1,1E1,6.4E1,6.7E1,1.04E2,5.3E1,1.6E1,5E0,1E1,4E0,9E0,2.1E1,1.5E1,3E0,2.1E1,2E0,7E0,1.1E1,2.8E1,3E0,1.8E1,7.7E1,3E0,2.1E1,1.8E1,2E1,2E0,6E0,1.6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"241","size_leaf_vector":"1"}},{"base_weights":[4.5838444E-3,-1.0641354E-1,5.732315E-2,-1.848399E-1,-3.6673147E-2,2.2667022E-2,1.9882733E-1,-1.4188932E-1,-2.8349587E-1,-7.263729E-2,9.945327E-2,-2.3834364E-2,1.124309E-1,3.1448394E-1,1.4516772E-1,-1.6061941E-1,1.41291255E-2,-3.216745E-1,-2.599317E-2,-8.156352E-2,2.7337825E-2,4.99692E-2,1.9290522E-1,-1.5716378E-1,-4.9453047E-3,1.4785872E-1,1.8320322E-2,3.3071792E-1,-3.4131E-2,1.06688835E-1,3.7103534E-1,-1.8138117E-1,-5.2221514E-2,1.5629552E-1,-4.552112E-3,-2.8885853E-1,-2.967475E-2,-4.2946837E-3,3.6961273E-3,-6.0812883E-2,-2.6283368E-1,7.13362E-2,-5.8536977E-3,3.0622405E-3,1.0960343E-2,-4.8632395E-2,-2.0207362E-1,-3.4522418E-2,5.435837E-2,1.3935088E-1,2.1750486E-2,-3.017753E-1,3.593378E-2,2.6630628E-1,4.7407243E-1,5.99897E-3,-8.574379E-3,2.0434478E-2,1.6594733E-1,3.156017E-2,2.1362779E-1,-1.9007099E-1,2.4596248E-3,-1.364759E-1,8.055629E-4,-3.3721782E-3,9.502993E-3,-2.1403554E-1,-4.026364E-1,-7.210889E-2,1.4147967E-2,-2.0659102E-1,-2.2760514E-2,1.0081972E-1,-6.762789E-2,-6.709257E-3,3.6218967E-2,-1.6892463E-1,-2.682915E-2,-7.0174327E-3,-1.1580197E-1,2.3028943E-1,4.4863136E-3,1.2225931E-1,3.0885112E-1,2.731678E-4,-2.0698715E-2,5.07755E-2,-1.2674928E-2,3.204217E-1,1.4281724E-1,5.0670016E-1,2.4912665E-3,5.0950624E-2,-1.9883972E-1,1.3483853E-2,2.0482841E-1,2.8963283E-1,-8.7514974E-2,-1.6255303E-1,-3.0732667E-1,-3.1924662E-3,-1.805544E-2,-2.4667297E-1,-3.43194E-2,-2.898391E-4,-4.297422E-1,-2.052935E-1,-5.1753197E-2,-2.8138554E-1,-3.1302068E-3,1.1824547E-4,1.2924746E-1,1.2806639E-3,-6.7918706E-3,-4.7915387E-3,3.3601674E-3,-1.85727E-1,3.1623545E-3,-7.311694E-2,2.9148154E-2,-2.4865603E-1,-5.8383968E-2,1.9072253E-2,1.2241238E-1,2.4363851E-2,-1.151166E-2,9.052726E-2,1.887649E-1,3.302499E-1,1.5714861E-3,-5.086458E-2,9.2668295E-2,3.3666962E-1,3.5239905E-3,1.0523867E-2,2.5454366E-3,2.5781175E-2,1.2358062E-2,4.1811457E-3,6.073071E-3,-1.3082748E-2,-6.3096127E-4,1.1094687E-1,-1.7094478E-1,2.2751933E-1,-3.3502482E-2,6.8583624E-3,1.5900858E-2,-1.2852184E-2,6.248602E-3,-1.175227E-2,-5.8809607E-3,-2.258052E-2,-1.0930681E-2,-6.253093E-4,-1.2233113E-2,3.165715E-3,-4.441139E-3,-1.5090827E-2,-3.367463E-2,-6.3325865E-3,-1.5769655E-2,-1.8220483E-3,-1.1482816E-2,-1.6016005E-2,-6.23215E-3,-2.9334559E-3,5.489E-3,3.5652816E-3,9.684265E-3,-9.442249E-3,-3.4009842E-3,-2.7478638E-3,-1.935559E-2,1.6173076E-2,7.685189E-4,1.5661793E-3,-1.451953E-2,-3.471573E-3,6.845643E-3,1.2501864E-3,1.33064035E-2,1.3259855E-4,1.037468E-2,3.4213813E-3,1.046956E-2,9.498077E-3,-5.4785353E-4,7.80122E-3,1.7437145E-2,-3.9010218E-3,6.787907E-3,1.0218906E-2,3.20423E-3,2.165214E-3,1.5873656E-2,-2.8096198E-3,4.3439427E-3,-9.82882E-4,8.124233E-3,-1.3224927E-2,-1.8030107E-3,5.9252386E-3,1.2369923E-2,2.4892364E-3,-6.145125E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,-1,-1,-1,69,71,73,-1,-1,-1,75,77,79,81,83,-1,85,87,89,91,-1,-1,93,95,-1,97,99,-1,101,-1,-1,-1,103,105,107,-1,109,-1,111,113,-1,115,117,-1,119,121,123,125,127,129,-1,-1,131,-1,133,135,137,-1,139,141,143,145,147,149,151,153,-1,-1,155,157,-1,159,161,163,165,167,-1,169,-1,-1,-1,-1,171,-1,173,175,177,179,-1,181,183,-1,185,187,189,-1,191,193,195,-1,-1,-1,-1,-1,197,-1,-1,-1,199,201,203,205,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2867174E1,3.8656168E0,7.302434E0,1.3708067E0,1.8479958E0,5.0033054E0,1.7727308E0,6.860728E-1,9.747505E-1,1.8486673E0,3.556869E-1,1.988812E0,1.3609519E0,5.319538E-1,1.7160239E0,4.6088934E-1,4.3959138E-1,7.635889E-1,1.1151635E-1,1.09365E0,0E0,2.1223147E-1,1.4997709E-1,4.678371E-1,1.2174205E0,7.733774E-1,6.479369E-1,6.8631744E-1,1.5445149E-1,8.859756E-1,1.3394775E0,3.6877298E-1,2.0889461E-1,1.6408324E-1,0E0,6.033907E-1,0E0,0E0,0E0,1.1442316E0,3.009107E-1,2.0348142E-1,0E0,0E0,0E0,2.5880158E-1,7.9312944E-1,1.0342668E0,2.0257711E0,8.0527496E-1,0E0,2.8978348E-1,5.205642E-1,3.7524557E-1,3.2760334E-1,0E0,0E0,4.9191207E-1,6.078167E-1,0E0,5.0600296E-1,4.9534512E-1,0E0,2.487356E-1,0E0,0E0,0E0,2.906735E-1,3.3019638E-1,6.915413E-1,0E0,3.9478076E-1,0E0,1.11335605E-1,7.536477E-2,0E0,9.962033E-2,2.7375126E-1,0E0,8.3164424E-1,8.7798357E-1,9.9911857E-1,9.3714565E-1,5.442674E-1,1.4105988E-1,0E0,0E0,4.4754413E-1,0E0,1.4327908E-1,1.4992383E-1,1.9656229E-1,0E0,2.48447E-1,1.4595878E-1,4.1441515E-1,4.5107388E-1,9.1420054E-2,2.6060882E-1,4.0670705E-1,2.7060413E-1,0E0,0E0,2.318654E-1,6.669235E-2,0E0,6.3022614E-1,2.6940715E-1,5.148454E-1,1.4894426E-1,6.920797E-2,0E0,1.1568034E-1,0E0,0E0,0E0,0E0,1.3837504E-1,0E0,5.1558363E-1,8.925412E-1,7.253573E-1,3.2985705E-1,0E0,5.446425E-1,7.4118376E-1,0E0,3.655839E-1,3.6153364E-1,1.4172506E-1,0E0,2.3264928E-1,1.9957852E-1,1.4433622E-1,0E0,0E0,0E0,0E0,0E0,2.6125443E-1,0E0,0E0,0E0,1.4019877E-1,1.15463644E-1,2.9552197E-1,7.9828314E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,39,39,40,40,41,41,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,57,57,58,58,60,60,61,61,63,63,67,67,68,68,69,69,71,71,73,73,74,74,76,76,77,77,79,79,80,80,81,81,82,82,83,83,84,84,87,87,89,89,90,90,91,91,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,103,103,104,104,106,106,107,107,108,108,109,109,110,110,112,112,117,117,119,119,120,120,121,121,122,122,124,124,125,125,127,127,128,128,129,129,131,131,132,132,133,133,139,139,143,143,144,144,145,145,146,146],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,-1,-1,-1,70,72,74,-1,-1,-1,76,78,80,82,84,-1,86,88,90,92,-1,-1,94,96,-1,98,100,-1,102,-1,-1,-1,104,106,108,-1,110,-1,112,114,-1,116,118,-1,120,122,124,126,128,130,-1,-1,132,-1,134,136,138,-1,140,142,144,146,148,150,152,154,-1,-1,156,158,-1,160,162,164,166,168,-1,170,-1,-1,-1,-1,172,-1,174,176,178,180,-1,182,184,-1,186,188,190,-1,192,194,196,-1,-1,-1,-1,-1,198,-1,-1,-1,200,202,204,206,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,6.576923E0,8.631E3,8.294758E-1,1.4580295E0,7.3582644E5,3.61172E5,4.832E3,6.1707315E0,2.14099E5,3.2343243E1,8.2E1,4.072289E0,4.5481584E7,8.791777E3,9.490825E5,1.5062694E9,4.5723195E6,9.4801545E-1,1.0828989E8,2.7337825E-2,1.775044E7,1.7227725E0,1E0,5.862126E2,1.8839418E3,7.5740985E2,2.0718E4,5.5743E4,3.0416E4,9.8092645E-2,7.856E3,2.028994E8,1.0257407E2,-4.552112E-3,1.6559546E0,-2.967475E-2,-4.2946837E-3,3.6961273E-3,5.185489E3,7.6008E4,5.884028E0,-5.8536977E-3,3.0622405E-3,1.0960343E-2,4.3875E1,4E0,5.024605E0,2.558106E10,2.040457E0,2.1750486E-2,2E1,9.367855E-1,8.2619705E1,1.8485916E-1,5.99897E-3,-8.574379E-3,7.6599895E9,4.470588E0,3.156017E-2,1.27E2,9.619E3,2.4596248E-3,1.454213E2,8.055629E-4,-3.3721782E-3,9.502993E-3,6.4446027E2,2.6432162E5,1.2E1,1.4147967E-2,6.08767E3,-2.2760514E-2,2.2783158E5,2.8216E4,-6.709257E-3,1.6254681E0,1E0,-2.682915E-2,1.3984146E2,9.1875E0,1.592E1,7.526755E7,2.45009E6,4.944E3,2.731678E-4,-2.0698715E-2,1.246E3,-1.2674928E-2,1E0,4.1032645E6,1.381E3,2.4912665E-3,1.5963264E9,1.7452E4,2.1845728E7,3.0272608E10,1.74E4,5.3474556E5,2.0074467E5,6.6985805E8,-3.1924662E-3,-1.805544E-2,8.5315496E-1,1.2882936E0,-2.898391E-4,1.5265896E8,5.2288747E-1,1.5793961E1,3.4463277E0,5.6812387E0,1.1824547E-4,1.51529E3,1.2806639E-3,-6.7918706E-3,-4.7915387E-3,3.3601674E-3,2.4222221E0,3.1623545E-3,3.52E2,1.1E1,2.1E1,1.9125667E3,1.9072253E-2,8.99653E9,2.231E3,-1.151166E-2,2.0619047E1,1.3794063E1,6.789622E7,1.5714861E-3,2.1178565E6,1.0303075E1,1.2068493E4,3.5239905E-3,1.0523867E-2,2.5454366E-3,2.5781175E-2,1.2358062E-2,3.317425E3,6.073071E-3,-1.3082748E-2,-6.3096127E-4,6.9998717E-4,2.8332526E9,6.0435E4,6.90802E5,6.8583624E-3,1.5900858E-2,-1.2852184E-2,6.248602E-3,-1.175227E-2,-5.8809607E-3,-2.258052E-2,-1.0930681E-2,-6.253093E-4,-1.2233113E-2,3.165715E-3,-4.441139E-3,-1.5090827E-2,-3.367463E-2,-6.3325865E-3,-1.5769655E-2,-1.8220483E-3,-1.1482816E-2,-1.6016005E-2,-6.23215E-3,-2.9334559E-3,5.489E-3,3.5652816E-3,9.684265E-3,-9.442249E-3,-3.4009842E-3,-2.7478638E-3,-1.935559E-2,1.6173076E-2,7.685189E-4,1.5661793E-3,-1.451953E-2,-3.471573E-3,6.845643E-3,1.2501864E-3,1.33064035E-2,1.3259855E-4,1.037468E-2,3.4213813E-3,1.046956E-2,9.498077E-3,-5.4785353E-4,7.80122E-3,1.7437145E-2,-3.9010218E-3,6.787907E-3,1.0218906E-2,3.20423E-3,2.165214E-3,1.5873656E-2,-2.8096198E-3,4.3439427E-3,-9.82882E-4,8.124233E-3,-1.3224927E-2,-1.8030107E-3,5.9252386E-3,1.2369923E-2,2.4892364E-3,-6.145125E-3],"split_indices":[19,55,2,26,41,27,28,2,53,2,55,28,53,46,51,32,12,27,26,44,0,44,38,80,51,51,50,2,10,10,56,0,7,51,0,55,0,0,0,51,10,52,0,0,0,4,52,52,18,52,0,0,26,55,56,0,0,5,53,0,8,10,0,51,0,0,0,51,27,17,0,4,0,27,9,0,52,67,0,51,55,57,44,9,2,0,0,2,0,6,44,0,0,7,10,42,5,9,31,27,5,0,0,26,38,0,44,26,56,52,56,0,4,0,0,0,0,52,0,0,10,3,4,0,30,0,0,55,55,7,0,27,57,27,0,0,0,0,0,51,0,0,0,33,7,10,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.196E3,7.07E2,1.489E3,3.32E2,3.75E2,1.197E3,2.92E2,2.33E2,9.9E1,2.97E2,7.8E1,7.89E2,4.08E2,9.1E1,2.01E2,2.08E2,2.5E1,8.6E1,1.3E1,2.94E2,3E0,5.2E1,2.6E1,9.7E1,6.92E2,2.96E2,1.12E2,8.7E1,4E0,1.73E2,2.8E1,1.74E2,3.4E1,1.1E1,1.4E1,8E1,6E0,8E0,5E0,2.65E2,2.9E1,4.7E1,5E0,8E0,1.8E1,2.9E1,6.8E1,4.62E2,2.3E2,2.9E2,6E0,5E0,1.07E2,6.2E1,2.5E1,2E0,2E0,7.1E1,1.02E2,8E0,2E1,1.68E2,6E0,1.5E1,1.9E1,2E0,9E0,5E1,3E1,2.58E2,7E0,2.5E1,4E0,3.9E1,8E0,1.3E1,1.6E1,6.4E1,4E0,3.46E2,1.16E2,5E1,1.8E2,2.65E2,2.5E1,2E0,3E0,1.03E2,4E0,4.2E1,2E1,2.3E1,2E0,6.3E1,8E0,2.1E1,8.1E1,1.6E1,4E0,1.38E2,3E1,1.3E1,2E0,4.2E1,8E0,2E0,2.8E1,3.3E1,2.25E2,1.8E1,7E0,9E0,3E1,4E0,4E0,3E0,1.3E1,6E1,4E0,1.22E2,2.24E2,3.4E1,8.2E1,1.7E1,3.3E1,1.68E2,1.2E1,1.81E2,8.4E1,2.3E1,2E0,3E1,7.3E1,3.9E1,3E0,9E0,1.1E1,1.7E1,6E0,4.1E1,2.2E1,5E0,3E0,1.4E1,7E0,7.4E1,7E0,6E0,1E1,2E0,2E0,3.3E1,1.05E2,6E0,2.4E1,4E0,3.8E1,3E0,5E0,2.3E1,5E0,2.4E1,9E0,2.14E2,1.1E1,1.1E1,7E0,5E0,2E0,2E1,1E1,4.9E1,1.1E1,1.19E2,3E0,7E0,2.17E2,7E0,2.7E1,7.6E1,6E0,2.2E1,1.1E1,1.53E2,1.5E1,1.65E2,1.6E1,7.6E1,8E0,7E0,1.6E1,2.6E1,4E0,9E0,6.4E1,2E0,3.7E1,2.4E1,1.7E1,5E0,9E0,3E0,4E0,2.5E1,4.9E1,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"207","size_leaf_vector":"1"}},{"base_weights":[-2.4856357E-5,-1.2119065E-1,5.5706616E-2,-1.5179855E-1,2.9763216E-2,1.3620107E-2,1.5681626E-1,-1.3558857E-1,-3.6643305E-1,-1.021857E-1,1.067764E-1,-5.452125E-2,5.7180423E-2,8.769741E-2,2.362463E-1,-1.4478299E-1,2.802647E-1,-2.8537562E-1,-6.139225E-1,-1.9742502E-1,-1.5804303E-3,6.783582E-2,1.5986528E-2,-7.829083E-2,9.1034666E-2,7.309063E-2,-1.4010412E-1,1.015566E-1,-1.6105129E-1,2.1815301E-1,2.1224452E-2,-1.3597599E-1,-4.8321626E-1,2.0395203E-2,7.853719E-3,-3.4709424E-1,1.8073699E-3,-3.310852E-2,-6.1308676E-3,-1.0928684E-1,-1.4501947E-2,8.353741E-2,-6.357864E-3,1.7505608E-1,-8.605164E-2,4.2208412E-1,1.4859589E-2,4.218706E-2,1.8882932E-1,-1.8402523E-1,1.4971185E-1,7.1852624E-2,2.3455872E-1,-3.2495922E-1,1.6619326E-1,2.1003416E-2,1.9929609E-1,-1.1333865E-1,-2.3510204E-1,-2.9153284E-2,-4.404568E-3,-4.7153583E-1,-1.9786017E-1,-8.947946E-3,5.4357736E-4,1.3392863E-1,3.6012582E-3,1.7226385E-2,1.4976469E-4,-1.0903892E-1,-2.0651987E-2,2.1591043E-2,5.2947267E-3,-6.514966E-2,1.5627271E-1,-2.8001022E-2,6.579201E-2,2.1307962E-1,-1.162072E-1,-2.0682211E-1,1.3851804E-3,-8.975853E-4,1.00278165E-2,8.575309E-2,-1.3800856E-1,2.5373495E-1,-2.8320073E-4,-4.3376475E-3,-2.214429E-2,1.6416928E-2,-3.8765522E-3,1.2759066E-1,2.4769434E-1,-1.3089496E-1,-1.1217843E-2,-2.81501E-3,-2.617587E-1,-2.2991005E-2,-7.6613217E-3,-2.8736554E-3,-1.2018758E-2,4.0345443E-3,1.06768105E-2,-3.435419E-3,5.0552147E-3,-1.3016383E-1,-2.8702188E-2,-4.7493648E-2,9.2335604E-2,-6.4984653E-3,1.7168588E-4,2.1535565E-1,-1.8297033E-3,1.20490074E-1,-5.859817E-2,1.8877083E-1,4.2362917E-2,1.2679423E-1,3.0015424E-1,-1.2942022E-2,-1.7231903E-3,-3.111622E-3,-2.3488225E-1,2.2864003E-2,1.2237405E-1,2.3060157E-3,-9.355819E-3,2.0801246E-3,2.8382596E-1,8.650954E-2,2.2910829E-1,2.69521E-1,1.3241707E-1,-4.2636762E-3,-8.799709E-3,6.253411E-4,-5.9217517E-3,-2.515705E-3,-1.2826059E-2,-4.66037E-3,-9.067556E-3,-2.102599E-3,5.6099286E-3,1.5839688E-3,-4.538825E-3,8.325441E-3,-8.8969467E-4,2.5197049E-3,1.1492835E-2,2.4088766E-4,7.524672E-3,-1.0344085E-2,-1.2964222E-3,5.2730613E-3,1.4338426E-2,-1.5833772E-3,3.04099E-3,-6.921519E-3,8.664545E-3,4.4956533E-3,1.5621786E-2,-1.3025136E-2,-6.1374456E-3,-2.0783555E-3,3.687458E-3,4.828948E-3,1.2242688E-2,1.5412334E-2,7.0813387E-3,9.284397E-3,1.2709319E-3,-2.3814554E-3,1.1840512E-2,7.6246816E-3,1.4763638E-2,-3.8712714E-3,8.086932E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,47,49,51,53,55,-1,57,59,-1,-1,61,-1,-1,-1,63,-1,65,-1,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,-1,-1,97,99,-1,-1,101,103,-1,-1,105,107,-1,-1,109,111,113,115,117,119,121,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,129,131,133,135,-1,137,-1,-1,-1,-1,-1,-1,-1,-1,139,141,143,145,-1,-1,147,-1,149,151,153,155,157,159,-1,-1,-1,161,163,165,-1,-1,-1,167,169,171,173,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4896562E1,3.215683E0,6.4277015E0,1.9613056E0,1.2100042E0,3.1732183E0,2.416892E0,2.1014585E0,6.272359E-1,2.7503675E-1,6.954645E-1,1.4498646E0,2.0567617E0,8.402976E-1,7.645111E-1,1.5080423E0,1.3999444E-1,6.563151E-1,4.0643215E-1,1.5995795E-1,0E0,2.2755906E-1,0E0,7.2092223E-1,1.4815733E0,2.1523838E0,6.512737E-1,8.7995315E-1,7.5129974E-1,8.0197716E-1,0E0,1.1312313E0,7.4367094E-1,0E0,0E0,4.0811706E-1,0E0,0E0,0E0,1.3097273E-1,0E0,2.4925867E-1,0E0,3.868573E-1,5.224209E-1,1.2820876E-1,5.645847E-1,7.937859E-1,9.642091E-1,2.1671307E-1,8.95772E-2,5.573451E-1,1.9092155E-1,2.916909E-1,2.783584E-1,0E0,5.9654236E-1,7.5512075E-1,4.2163658E-1,0E0,0E0,6.7634106E-2,1.10348105E-1,0E0,0E0,1.5251249E-1,2.2318746E-1,0E0,0E0,4.3411732E-1,2.8340062E-1,0E0,0E0,1.7347653E-1,2.178784E-1,5.568124E-1,1.023502E0,8.485074E-1,1.2169646E-1,1.3497877E-1,0E0,0E0,0E0,4.0254033E-1,1.6615556E-1,2.2455812E-1,0E0,0E0,0E0,0E0,0E0,2.9296422E-1,2.3353577E-1,8.207278E-1,1.903734E-1,0E0,3.4845448E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.6883688E-1,1.5465494E-1,3.2987332E-1,1.8816648E-1,0E0,0E0,7.194036E-2,0E0,1.0967535E-1,5.075594E-1,4.903431E-1,5.816999E-1,1.1311092E0,4.887581E-1,0E0,0E0,0E0,1.2885392E-1,2.6964056E-1,2.2936642E-1,0E0,0E0,0E0,1.8473125E-1,3.7530884E-1,2.0556235E-1,4.541793E-1,2.0658916E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,39,39,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,61,61,62,62,65,65,66,66,69,69,70,70,73,73,74,74,75,75,76,76,77,77,78,78,79,79,83,83,84,84,85,85,91,91,92,92,93,93,94,94,96,96,105,105,106,106,107,107,108,108,111,111,113,113,114,114,115,115,116,116,117,117,118,118,122,122,123,123,124,124,128,128,129,129,130,130,131,131,132,132],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,48,50,52,54,56,-1,58,60,-1,-1,62,-1,-1,-1,64,-1,66,-1,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,-1,-1,98,100,-1,-1,102,104,-1,-1,106,108,-1,-1,110,112,114,116,118,120,122,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,130,132,134,136,-1,138,-1,-1,-1,-1,-1,-1,-1,-1,140,142,144,146,-1,-1,148,-1,150,152,154,156,158,160,-1,-1,-1,162,164,166,-1,-1,-1,168,170,172,174,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.4580295E0,3.474851E8,1.1855755E8,2.539E3,2.0407547E5,3.3968747E-1,4.77721E3,1.7243155E12,1E0,1.19783E5,1.0989723E3,3.881848E-1,4.2620764E2,1.6828056E1,4.5723195E6,3.368326E11,4.551684E6,4.0950096E10,1.8989474E0,-1.5804303E-3,7.137324E0,1.5986528E-2,7E0,1E0,5.862126E2,2.4345527E3,6.117756E9,2.4507338E4,1.1485E4,2.1224452E-2,7.2234793E0,1.4E1,2.0395203E-2,7.853719E-3,1.3528846E1,1.8073699E-3,-3.310852E-2,-6.1308676E-3,2.4222438E2,-1.4501947E-2,2.26182E5,-6.357864E-3,1.3050649E4,1.445113E6,1.0804272E3,9.042859E4,1E0,1.778482E7,8.360387E-1,1E0,2.8E2,5.4106476E7,3.791269E7,1.3683E4,2.1003416E-2,1.2974394E6,1.747325E3,3.835264E-2,-2.9153284E-2,-4.404568E-3,4.3E1,2.2E1,-8.947946E-3,5.4357736E-4,1.1766373E3,3.6E1,1.7226385E-2,1.4976469E-4,3.647E3,6.241865E5,2.1591043E-2,5.2947267E-3,6.071E3,3.291238E7,5E-1,4.296196E0,3.426E3,1.2804922E3,4.4E1,1.3851804E-3,-8.975853E-4,1.00278165E-2,7.06699E8,8.9190274E-2,1.1041459E-1,-2.8320073E-4,-4.3376475E-3,-2.214429E-2,1.6416928E-2,-3.8765522E-3,9.1854416E-2,2.1938796E7,1.6527965E7,1E0,-2.81501E-3,3.036836E7,-2.2991005E-2,-7.6613217E-3,-2.8736554E-3,-1.2018758E-2,4.0345443E-3,1.06768105E-2,-3.435419E-3,5.0552147E-3,1.2E1,1.6286094E5,1.1255588E1,2.5054495E1,-6.4984653E-3,1.7168588E-4,9.073025E6,-1.8297033E-3,4.86E2,1.4431512E-1,6.895302E7,2.3161087E2,1E0,2.5332516E5,-1.2942022E-2,-1.7231903E-3,-3.111622E-3,1.3849624E1,1.7036236E5,5.57E4,2.3060157E-3,-9.355819E-3,2.0801246E-3,2.4480248E5,1.1852991E1,7E0,5.1148495E6,2.6364462E7,-4.2636762E-3,-8.799709E-3,6.253411E-4,-5.9217517E-3,-2.515705E-3,-1.2826059E-2,-4.66037E-3,-9.067556E-3,-2.102599E-3,5.6099286E-3,1.5839688E-3,-4.538825E-3,8.325441E-3,-8.8969467E-4,2.5197049E-3,1.1492835E-2,2.4088766E-4,7.524672E-3,-1.0344085E-2,-1.2964222E-3,5.2730613E-3,1.4338426E-2,-1.5833772E-3,3.04099E-3,-6.921519E-3,8.664545E-3,4.4956533E-3,1.5621786E-2,-1.3025136E-2,-6.1374456E-3,-2.0783555E-3,3.687458E-3,4.828948E-3,1.2242688E-2,1.5412334E-2,7.0813387E-3,9.284397E-3,1.2709319E-3,-2.3814554E-3,1.1840512E-2,7.6246816E-3,1.4763638E-2,-3.8712714E-3,8.086932E-3],"split_indices":[19,41,7,44,2,27,37,51,30,109,10,51,26,57,34,27,30,28,5,53,0,52,0,3,8,51,51,12,4,9,0,56,8,0,0,55,0,0,0,4,0,28,0,27,9,32,27,109,49,38,15,3,44,44,9,0,27,54,26,0,0,3,57,0,0,51,3,0,0,2,46,0,0,2,44,52,57,2,51,10,0,0,0,7,37,38,0,0,0,0,0,56,50,44,111,0,12,0,0,0,0,0,0,0,0,8,27,57,55,0,0,44,0,12,57,7,4,109,27,0,0,0,53,27,2,0,0,0,32,55,3,44,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.204E3,6.94E2,1.51E3,5.77E2,1.17E2,1.067E3,4.43E2,5.38E2,3.9E1,4.3E1,7.4E1,4.16E2,6.51E2,2.38E2,2.05E2,5.27E2,1.1E1,3.1E1,8E0,1.7E1,2.6E1,6.5E1,9E0,3.58E2,5.8E1,6.03E2,4.8E1,2.26E2,1.2E1,1.92E2,1.3E1,5.15E2,1.2E1,3E0,8E0,2.6E1,5E0,6E0,2E0,1.1E1,6E0,6.1E1,4E0,1E1,3.48E2,1E1,4.8E1,4.77E2,1.26E2,4.2E1,6E0,1.86E2,4E1,8E0,4E0,1.2E1,1.8E2,4.21E2,9.4E1,8E0,4E0,1.3E1,1.3E1,6E0,5E0,3.7E1,2.4E1,4E0,6E0,2.57E2,9.1E1,8E0,2E0,3.1E1,1.7E1,1.2E2,3.57E2,1.17E2,9E0,3.8E1,4E0,2E0,4E0,1.75E2,1.1E1,3.7E1,3E0,4E0,4E0,2E0,2E0,7.4E1,1.06E2,3.59E2,6.2E1,1.3E1,8.1E1,1.1E1,2E0,5E0,8E0,2.7E1,1E1,1.4E1,1E1,2.03E2,5.4E1,7.4E1,1.7E1,1.4E1,1.7E1,1.3E1,4E0,2E1,1E2,5.6E1,3.01E2,6E1,5.7E1,2E0,7E0,7E0,3.1E1,6.5E1,1.1E2,3E0,8E0,5E0,3.2E1,5.4E1,2E1,8.8E1,1.8E1,2.29E2,1.3E2,5.2E1,1E1,8E0,7.3E1,1.48E2,5.5E1,4.9E1,5E0,2.9E1,4.5E1,9E0,8E0,3E0,1E1,6E0,1.4E1,1.4E1,8.6E1,3.7E1,1.9E1,7.3E1,2.28E2,1.1E1,4.9E1,1.1E1,4.6E1,1.9E1,1.2E1,3E1,3.5E1,1.01E2,9E0,2.1E1,1.1E1,1.7E1,3.7E1,2E0,1.8E1,3.3E1,5.5E1,3E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"177","size_leaf_vector":"1"}},{"base_weights":[-1.7429258E-4,-1.0135215E-1,4.607109E-2,-1.7973429E-1,-3.5125747E-2,1.357837E-4,1.2880424E-1,-1.1061714E-1,-2.2977939E-1,-7.541063E-2,1.0169267E-1,-3.698152E-2,6.478478E-2,1.4937118E-1,-2.7904058E-2,-1.425145E-1,1.9783715E-2,-3.2151914E-1,-1.8498123E-1,7.9279505E-3,-1.3008775E-1,3.1910968E-1,7.896392E-2,-7.634327E-2,1.4681066E-2,9.136058E-2,-7.995436E-2,1.09250076E-1,2.2538084E-1,-7.4967675E-2,1.2091375E-1,-2.2019154E-1,-7.281916E-2,6.258284E-2,-5.3221374E-3,-2.8925415E-2,-2.7442345E-1,-4.9524743E-2,-2.2594213E-1,-1.7143799E-2,2.829244E-1,-2.7607137E-1,-7.0233196E-2,1.7478453E-2,3.5693762E-3,2.19324E-2,1.4619242E-1,-5.287882E-2,-2.0832922E-1,2.0366195E-1,-1.1043419E-2,2.4268962E-3,1.2635969E-1,-3.8768235E-1,-2.91188E-2,8.370624E-2,2.2653797E-1,2.749603E-1,9.188985E-2,-1.0043515E-1,7.892456E-3,1.8440399E-1,-9.764623E-3,-3.811445E-1,-1.3806953E-1,-1.2836218E-1,1.5843796E-2,9.578613E-3,1.2641969E-3,-3.352094E-1,-1.4480673E-1,-1.2913866E-1,7.0643656E-2,8.5326337E-4,-2.5090784E-1,-4.4179386E-3,-9.225229E-3,2.0962598E-2,3.8796768E-3,-2.2560497E-1,-2.0086419E-2,-1.3355054E-2,-1.4413942E-1,2.6514768E-3,-1.028735E-3,2.5989902E-1,8.808966E-2,-1.2431773E-1,-1.827897E-2,-1.3923945E-1,-5.38307E-1,1.497499E-1,2.1533169E-2,-5.621868E-3,-2.005154E-2,-4.992929E-2,1.6176496E-1,1.0700957E-1,3.2644993E-1,-8.293739E-3,-2.4736647E-2,1.6534192E-2,-1.6032611E-1,1.6876873E-1,5.9699874E-2,2.526521E-1,-8.268718E-3,1.1363161E-3,2.946387E-1,1.1562622E-1,-8.700868E-3,-1.3436466E-1,3.5098894E-3,2.5640859E-3,1.376701E-2,-7.952514E-3,-2.2973048E-2,-6.860116E-4,-1.8476254E-1,-4.703512E-2,-1.7045453E-1,1.578193E-1,-2.9431537E-3,-2.3718503E-1,-4.2005682E-1,-2.4621664E-1,-1.3091475E-3,-7.4414467E-3,1.0719651E-4,-2.5155894E-3,8.455321E-3,-3.017765E-1,-1.2292139E-1,1.3736567E-2,-1.447472E-1,-2.7040267E-1,-9.107715E-2,-4.3616883E-2,1.2791016E-2,-3.02372E-1,-8.41959E-2,1.5255798E-2,5.73488E-3,-5.5287295E-4,1.7493454E-1,-1.395344E-1,3.0838728E-3,2.4471208E-1,-3.9986625E-2,1.3859688E-2,-2.2946514E-1,-3.26104E-2,-5.919786E-3,1.5935682E-2,9.295192E-2,-2.5393728E-2,1.1550075E-1,2.9304444E-3,-1.7429619E-1,1.23972865E-2,1.02626964E-1,9.015535E-2,1.4136283E-2,1.6871016E-2,7.4366084E-3,-8.20988E-2,1.0028941E-1,-9.5289415E-3,2.3191873E-4,1.9226055E-1,-2.9851744E-2,-8.371139E-2,7.964005E-2,2.159165E-1,1.7778952E-2,3.732035E-1,2.1640643E-1,1.8749388E-1,1.1195418E-2,-7.498297E-2,-2.243765E-1,-5.157691E-3,-1.2269556E-2,-7.7076307E-3,1.8259181E-3,-8.638646E-3,1.6128749E-3,1.2481356E-2,2.3967505E-3,-1.6573418E-2,-7.4705174E-3,-1.08449515E-2,-2.2200225E-2,-5.1292987E-3,-1.6139241E-2,-1.4570791E-2,3.375715E-3,-8.929455E-3,-1.2551434E-3,-6.8167206E-3,1.3066427E-3,-1.0530622E-2,-8.208301E-4,-9.76789E-3,-2.0923823E-2,-3.7493292E-4,-9.29632E-3,-7.35245E-4,-1.0018386E-2,-9.349297E-3,-2.0500273E-2,-1.2001969E-3,-8.432899E-3,1.4870106E-3,1.03444625E-2,-6.7597334E-3,2.8219773E-3,1.4000991E-2,-1.1868943E-3,-3.9955783E-3,1.0728047E-3,4.995557E-3,-4.2956527E-3,-3.1952597E-2,-7.6080444E-3,-7.5619965E-4,6.296989E-3,-2.314387E-3,2.300228E-3,8.698694E-3,-1.8565014E-3,2.5555422E-3,-4.047417E-3,4.9498E-3,-1.0448278E-2,5.9326477E-3,-2.9052743E-3,2.2959518E-3,6.1937226E-3,-1.2314918E-2,-1.8725067E-3,7.3166867E-3,2.594674E-5,4.4192313E-3,1.146469E-2,-7.90549E-3,5.5403532E-3,-3.867589E-4,-1.2971913E-2,1.465831E-2,3.0856924E-3,1.1760196E-2,3.985092E-3,1.2783641E-2,2.4016162E-2,8.518025E-3,2.526186E-2,-2.6796982E-3,9.530933E-3,-3.4192512E-3,3.6730561E-3,3.3572547E-3,-4.127876E-3,-1.3336453E-2,-3.168606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,77,79,81,-1,-1,83,85,87,89,91,93,95,97,99,101,103,105,107,109,111,-1,113,-1,115,117,119,121,-1,-1,123,125,127,129,-1,131,133,-1,-1,-1,135,-1,137,139,-1,-1,141,143,145,147,149,151,153,-1,155,-1,157,159,161,163,-1,-1,165,167,169,171,173,-1,-1,175,177,-1,179,-1,-1,-1,-1,-1,-1,181,183,185,187,-1,189,191,193,-1,-1,-1,-1,-1,195,197,199,201,203,205,207,-1,209,211,-1,-1,-1,213,215,-1,217,219,221,223,-1,-1,-1,225,227,229,231,233,-1,235,237,-1,-1,-1,239,241,-1,-1,243,245,247,249,251,-1,253,255,257,259,261,263,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0560513E1,3.6688914E0,5.886759E0,1.0952911E0,2.1296787E0,2.394793E0,1.7864933E0,5.768225E-1,7.090521E-1,1.3589318E0,4.0966415E-1,1.2903788E0,1.4074521E0,1.4666281E0,4.643184E-1,5.8488965E-1,1.7193915E-1,7.2150135E-1,6.99008E-1,8.253129E-1,1.553412E0,9.825218E-2,3.0735928E-1,1.1038952E0,1.3391439E0,9.58374E-1,8.803922E-1,9.4574404E-1,1.086628E0,3.3495346E-1,3.8169724E-1,6.3470006E-1,2.9751018E-1,1.07696615E-1,0E0,0E0,3.769188E-1,3.057022E-1,6.036382E-1,2.6033995E-1,3.1552052E-1,3.3971262E-1,5.403458E-1,0E0,0E0,7.494314E-2,2.2155231E-1,7.565627E-1,1.1561434E0,4.1261482E-1,5.664454E-1,7.4220216E-1,8.2072806E-1,1.5952384E-1,3.0215117E-1,5.331073E-1,6.552038E-1,5.924616E-1,3.3913535E-1,2.8817022E-1,0E0,1.968551E-1,0E0,3.52005E-1,2.0381767E-1,1.1864287E-1,2.857537E-1,0E0,0E0,1.9978476E-1,2.1617699E-1,8.8406175E-2,2.0088714E-1,0E0,5.423393E-1,2.669396E-1,0E0,0E0,0E0,2.2848701E-1,0E0,6.74909E-1,5.0570273E-1,0E0,0E0,8.052695E-2,2.2861347E-1,3.0232453E-1,1.1972405E0,6.3860023E-1,5.7332873E-1,3.0038983E-1,0E0,5.8008456E-1,0E0,4.4096887E-1,1.17696226E-1,6.7402244E-1,8.114076E-2,0E0,0E0,3.2215267E-1,1.08935684E-1,2.763648E-1,6.043329E-1,1.9302082E-1,0E0,0E0,6.137686E-1,3.2812953E-1,0E0,1.9163275E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2797642E-1,1.5756534E-1,1.0746539E-1,9.6661985E-2,0E0,1.1634612E-1,1.1095381E-1,1.06176496E-1,0E0,0E0,0E0,0E0,0E0,4.995966E-1,1.8553528E-1,2.3544604E-1,1.292634E-1,2.2516608E-1,1.11820154E-1,3.271569E-1,0E0,1.2281692E-1,2.4235463E-1,0E0,0E0,0E0,9.9022985E-2,1.9277894E-1,0E0,2.8787148E-1,5.9532416E-1,1.9931519E-1,7.297764E-1,0E0,0E0,0E0,1.2373571E-1,4.103455E-1,4.1566506E-1,2.4221641E-1,3.4563136E-1,0E0,8.4233716E-2,3.4082532E-1,0E0,0E0,0E0,1.2801243E-1,1.2831363E-1,0E0,0E0,2.75357E-1,1.7562462E-1,3.9391994E-1,4.9379325E-1,2.2944593E-1,0E0,6.359315E-1,4.6458292E-1,1.6012418E-1,1.21355E-1,6.7133844E-2,1.4194578E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,79,79,81,81,82,82,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,95,95,96,96,97,97,98,98,101,101,102,102,103,103,104,104,105,105,108,108,109,109,111,111,118,118,119,119,120,120,121,121,123,123,124,124,125,125,131,131,132,132,133,133,134,134,135,135,136,136,137,137,139,139,140,140,144,144,145,145,147,147,148,148,149,149,150,150,154,154,155,155,156,156,157,157,158,158,160,160,161,161,165,165,166,166,169,169,170,170,171,171,172,172,173,173,175,175,176,176,177,177,178,178,179,179,180,180],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,78,80,82,-1,-1,84,86,88,90,92,94,96,98,100,102,104,106,108,110,112,-1,114,-1,116,118,120,122,-1,-1,124,126,128,130,-1,132,134,-1,-1,-1,136,-1,138,140,-1,-1,142,144,146,148,150,152,154,-1,156,-1,158,160,162,164,-1,-1,166,168,170,172,174,-1,-1,176,178,-1,180,-1,-1,-1,-1,-1,-1,182,184,186,188,-1,190,192,194,-1,-1,-1,-1,-1,196,198,200,202,204,206,208,-1,210,212,-1,-1,-1,214,216,-1,218,220,222,224,-1,-1,-1,226,228,230,232,234,-1,236,238,-1,-1,-1,240,242,-1,-1,244,246,248,250,252,-1,254,256,258,260,262,264,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.638132E0,9.772148E4,6.8503425E6,1.4329944E0,4.5168175E5,3.0923106E-10,1.7019386E2,3.8410318E2,1.6837112E7,1E0,2.857E3,3.7832818E0,6.167018E6,4.760537E8,2.308943E0,4.1100917E0,2.5070414E-5,1.18827E5,1.067536E3,1.3097832E3,4.1391224E1,1.6525185E9,9.669789E0,7.9E1,1E0,3.6887734E-5,2.6396152E-3,8.5E1,4E1,1.43709E0,1.4773677E5,1E0,2.2805734E9,-5.3221374E-3,-2.8925415E-2,1.9121015E0,1.1910478E5,1.5756359E0,1.9283049E-1,6.585265E-1,8.672049E9,6.747114E7,1.7478453E-2,3.5693762E-3,1.4302E4,8E0,2.53E2,3.624992E5,1.89777E5,6.5321615E6,1.408E3,8.5023944E2,8E0,8.3490986E-1,1.3878379E0,4.92E2,9.4168E4,8.724166E9,1.5088776E0,7.892456E-3,1.7E1,-9.764623E-3,1.3563734E0,3.3018634E5,1.1129032E0,1.1331109E6,9.578613E-3,1.2641969E-3,1.5039847E5,1.3617646E-3,1.4688152E9,1.4103535E0,8.5326337E-4,5.121E3,1.1231086E1,-9.225229E-3,2.0962598E-2,3.8796768E-3,1.6408995E11,-2.0086419E-2,5.185489E3,1.3528846E1,2.6514768E-3,-1.028735E-3,8.7643677E-1,5.3E1,3.2188842E0,1.2E1,4.5586734E0,6.203E3,1E0,2.1533169E-2,4.894913E6,-2.005154E-2,1.0258E4,6.4506575E5,6.057671E6,1.4E2,-8.293739E-3,-2.4736647E-2,6.789622E7,3.1998687E3,5.68197E6,2.960909E2,9.317209E0,-8.268718E-3,1.1363161E-3,1.6918114E7,3.5561565E8,-8.700868E-3,9E0,3.5098894E-3,2.5640859E-3,1.376701E-2,-7.952514E-3,-2.2973048E-2,-6.860116E-4,7.983101E-1,5.152349E1,6.1E1,1.0142518E0,-2.9431537E-3,3.0966794E8,1.7258642E0,2.1402586E4,-1.3091475E-3,-7.4414467E-3,1.0719651E-4,-2.5155894E-3,8.455321E-3,2.7485228E6,5.3E1,1.074E3,7.221312E-2,1.3046257E11,1.3593E4,1.401692E1,1.2791016E-2,1.0828989E8,1.0104842E12,1.5255798E-2,5.73488E-3,-5.5287295E-4,2.603E3,2.4E1,3.0838728E-3,5.1853933E0,2.0407547E5,2.6E1,7.854E3,-3.26104E-2,-5.919786E-3,1.5935682E-2,1E0,7.21E3,2.88001E10,1E0,5.52E2,1.23972865E-2,2.78E2,6.9432104E7,1.4136283E-2,1.6871016E-2,7.4366084E-3,2.2690948E2,1.7270016E5,-9.5289415E-3,2.3191873E-4,2.7457875E6,2.1872402E5,5.408015E0,4.2E1,2.3619232E7,1.7778952E-2,1.6503105E3,5.68E2,5.441E3,1.2E1,1.0937254E5,1.1328599E3,-5.157691E-3,-1.2269556E-2,-7.7076307E-3,1.8259181E-3,-8.638646E-3,1.6128749E-3,1.2481356E-2,2.3967505E-3,-1.6573418E-2,-7.4705174E-3,-1.08449515E-2,-2.2200225E-2,-5.1292987E-3,-1.6139241E-2,-1.4570791E-2,3.375715E-3,-8.929455E-3,-1.2551434E-3,-6.8167206E-3,1.3066427E-3,-1.0530622E-2,-8.208301E-4,-9.76789E-3,-2.0923823E-2,-3.7493292E-4,-9.29632E-3,-7.35245E-4,-1.0018386E-2,-9.349297E-3,-2.0500273E-2,-1.2001969E-3,-8.432899E-3,1.4870106E-3,1.03444625E-2,-6.7597334E-3,2.8219773E-3,1.4000991E-2,-1.1868943E-3,-3.9955783E-3,1.0728047E-3,4.995557E-3,-4.2956527E-3,-3.1952597E-2,-7.6080444E-3,-7.5619965E-4,6.296989E-3,-2.314387E-3,2.300228E-3,8.698694E-3,-1.8565014E-3,2.5555422E-3,-4.047417E-3,4.9498E-3,-1.0448278E-2,5.9326477E-3,-2.9052743E-3,2.2959518E-3,6.1937226E-3,-1.2314918E-2,-1.8725067E-3,7.3166867E-3,2.594674E-5,4.4192313E-3,1.146469E-2,-7.90549E-3,5.5403532E-3,-3.867589E-4,-1.2971913E-2,1.465831E-2,3.0856924E-3,1.1760196E-2,3.985092E-3,1.2783641E-2,2.4016162E-2,8.518025E-3,2.526186E-2,-2.6796982E-3,9.530933E-3,-3.4192512E-3,3.6730561E-3,3.3572547E-3,-4.127876E-3,-1.3336453E-2,-3.168606E-3],"split_indices":[19,55,32,44,41,27,36,51,4,44,66,2,52,46,7,53,55,41,28,51,4,55,5,53,11,109,40,33,8,3,52,32,8,30,0,0,52,32,53,37,26,5,44,0,0,10,8,2,46,28,46,0,51,8,26,52,3,1,7,37,0,3,0,53,32,57,44,0,0,31,40,5,55,0,2,53,0,0,0,30,0,51,55,0,0,56,8,52,10,55,28,8,0,1,0,28,27,27,8,0,0,7,4,49,51,34,0,0,49,31,0,8,0,0,0,0,0,0,38,51,3,57,0,5,53,31,0,0,0,0,0,27,8,2,37,30,10,56,0,44,30,0,0,0,0,3,0,53,27,3,9,0,0,0,15,2,5,15,2,0,8,7,0,0,0,32,46,0,0,46,32,53,0,42,0,51,3,2,3,32,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.255E3,7.07E2,1.548E3,3.23E2,3.84E2,9.96E2,5.52E2,1.37E2,1.86E2,2.97E2,8.7E1,6.33E2,3.63E2,4.88E2,6.4E1,1.1E2,2.7E1,5.9E1,1.27E2,1.18E2,1.79E2,7E0,8E1,3.59E2,2.74E2,3.07E2,5.6E1,3.21E2,1.67E2,4.9E1,1.5E1,5.1E1,5.9E1,2.1E1,6E0,6E0,5.3E1,3E1,9.7E1,1.09E2,9E0,5.1E1,1.28E2,5E0,2E0,4.4E1,3.6E1,3.06E2,5.3E1,3.2E1,2.42E2,8.7E1,2.2E2,7E0,4.9E1,2.65E2,5.6E1,1.21E2,4.6E1,4.5E1,4E0,1.3E1,2E0,1.6E1,3.5E1,3.6E1,2.3E1,3E0,1.8E1,3.5E1,1.8E1,1.8E1,1.2E1,9E0,8.8E1,1.03E2,6E0,4E0,5E0,4.1E1,1E1,7.3E1,5.5E1,2.4E1,2E1,1.1E1,2.5E1,9.9E1,2.07E2,4.5E1,8E0,2.8E1,4E0,2.4E2,2E0,6.6E1,2.1E1,2.02E2,1.8E1,4E0,3E0,3.7E1,1.2E1,5.7E1,2.08E2,5.3E1,3E0,9E0,1.12E2,4.3E1,3E0,3.8E1,7E0,7E0,6E0,7E0,9E0,1E1,2.5E1,1.3E1,2.3E1,8E0,1.5E1,1.8E1,1.7E1,9E0,9E0,1.4E1,4E0,6E0,6E0,6.2E1,2.6E1,9.2E1,1.1E1,3E1,1.1E1,6.7E1,6E0,1.4E1,4.1E1,6E0,5E0,1.2E1,1.3E1,9.2E1,7E0,1.5E1,1.92E2,1.7E1,2.8E1,5E0,3E0,5E0,2.3E1,2.07E2,3.3E1,4.7E1,1.9E1,6E0,1.5E1,1.88E2,1.4E1,1.3E1,5E0,1.7E1,2E1,9E0,3E0,5.1E1,6E0,2.5E1,1.83E2,4.4E1,9E0,5.4E1,5.8E1,2.5E1,1.8E1,2.4E1,1.4E1,1.5E1,1E1,5E0,8E0,2.1E1,2E0,3E0,5E0,5E0,1.3E1,6E0,1.1E1,5E0,4E0,5.9E1,3E0,1.4E1,1.2E1,7E0,8.5E1,6E0,5E0,2.5E1,5E0,7E0,4E0,5.9E1,8E0,1E1,4E0,2.7E1,1.4E1,4E0,9E0,8.8E1,4E0,1.2E1,3E0,1.09E2,8.3E1,9E0,8E0,2E0,2.6E1,7E0,1.6E1,1.55E2,5.2E1,2.2E1,1.1E1,3E1,1.7E1,3E0,1.6E1,1.3E1,2E0,1.03E2,8.5E1,2E0,1.5E1,1.2E1,8E0,2.1E1,3E1,3E0,3E0,1.9E1,6E0,7E0,1.76E2,3.2E1,1.2E1,3.6E1,1.8E1,5.5E1,3E0,2E0,2.3E1,8E0,1E1,2E0,2.2E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"265","size_leaf_vector":"1"}},{"base_weights":[5.930355E-3,4.663497E-2,-1.047688E-1,1.2026157E-2,1.6052756E-1,-1.4352623E-1,5.1466994E-2,-5.444699E-2,6.0935825E-2,1.2803192E-1,3.915716E-1,-1.8977799E-1,-4.1432187E-2,-1.8398765E-2,1.3590539E-1,-7.333279E-2,2.8974835E-2,-2.4614898E-3,1.0261939E-1,1.8454863E-1,-2.1408947E-2,5.286307E-1,2.1802399E-1,-1.6725822E-1,-3.9285898E-1,-1.48686E-1,-2.2819659E-2,8.90687E-2,-9.508313E-2,1.382507E-2,9.979582E-2,-6.340147E-2,-2.03637E-1,4.7070134E-2,-1.3497081E-1,9.077016E-3,-1.4479184E-2,1.0730296E-1,-1.9257177E-2,3.3315122E-1,1.4088327E-1,-5.023949E-3,-1.3831333E-2,4.11364E-3,2.6448883E-2,1.2569483E-2,-5.415608E-3,-1.5567668E-1,-4.8408076E-1,-2.962207E-1,-2.9235488E-2,-3.1916164E-3,-8.936854E-3,2.502272E-3,-1.1424885E-1,2.9839715E-1,1.8415619E-2,-1.16576254E-1,9.131441E-3,-3.0843718E-3,1.3168263E-1,-6.8144776E-2,8.064731E-3,-1.21297054E-1,-4.568544E-1,5.6284066E-2,-1.067668E-2,3.2372763E-3,-8.852307E-3,-2.1685916E-3,-1.7699446E-1,7.862713E-2,1.5748714E-1,3.7757924E-1,3.126517E-3,9.7824134E-2,2.3337148E-1,-1.19186185E-1,4.4772897E-2,-2.0225999E-1,-9.90914E-2,-2.9165978E-2,-3.7501543E-3,-3.2203764E-1,2.035327E-5,2.8004816E-2,-2.983439E-3,4.9738474E-3,-1.455365E-1,4.760349E-3,1.8881481E-2,1.5797038E-1,-2.5915394E-3,-9.229449E-2,-1.634411E-2,1.6637021E-1,3.2335143E-3,-5.273688E-2,-1.3192078E-1,-8.971343E-3,2.8820025E-2,-2.7974684E-2,-1.6372661E-3,7.969179E-2,-3.1685255E-2,2.7467158E-2,-9.5412776E-2,-8.409597E-2,-1.649142E-2,1.1020447E-1,5.5143144E-3,2.0366699E-1,9.212845E-2,4.1256174E-1,2.1038607E-1,1.6767283E-1,6.491724E-2,2.888852E-1,1.1916161E-1,4.6294957E-2,-8.123022E-3,-1.1700546E-1,7.8134224E-2,-1.6763648E-1,-2.90154E-1,1.6038928E-2,-1.741266E-1,-2.879858E-3,-3.5762137E-1,3.6924707E-3,-2.3520505E-2,-7.7620853E-4,-7.65925E-3,9.142128E-4,1.0711739E-2,-1.1689133E-1,1.0318675E-3,1.0352416E-2,4.7783647E-3,-4.8617697E-3,-8.911709E-4,-1.5999919E-2,-5.0410726E-3,-1.8899624E-3,5.770735E-3,2.4327245E-3,8.051201E-3,-8.6801E-3,-7.150093E-5,1.542632E-3,-1.811304E-2,-7.53114E-3,-9.6894056E-4,2.7262077E-3,-5.837075E-3,5.451583E-3,-1.3386873E-2,1.4635076E-2,-4.6381852E-4,1.2125771E-2,5.1110955E-3,5.948754E-3,-9.0009655E-4,1.2214011E-2,2.062011E-2,-1.14578106E-4,1.1949227E-2,1.2372728E-2,2.4482072E-3,7.9261395E-4,5.725627E-3,4.2306297E-3,1.3698818E-2,6.259363E-3,-1.8782145E-3,-2.2605092E-3,1.0110233E-2,-1.4399278E-3,-9.63816E-3,1.4849822E-2,2.2853755E-3,-1.5734792E-2,-6.3047344E-3,-1.9745992E-2,-8.89036E-3,9.2633255E-3,-5.6079915E-4,-1.3631343E-2,-6.233423E-3,-2.5626002E-3,-1.7562823E-2,-2.8152359E-3,2.65243E-3,2.6083496E-4,-6.4598685E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,59,61,63,65,67,-1,69,71,-1,73,75,77,-1,-1,-1,-1,-1,79,81,83,-1,-1,-1,85,87,89,91,93,-1,-1,95,97,-1,99,101,103,-1,-1,-1,105,107,109,111,113,-1,115,117,119,121,123,125,-1,-1,127,-1,129,-1,-1,131,-1,-1,133,-1,135,-1,137,-1,139,141,-1,143,-1,-1,145,147,149,151,153,-1,155,157,159,161,163,165,167,169,171,173,175,-1,177,179,181,183,185,187,-1,189,-1,191,-1,-1,-1,-1,193,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.972083E0,6.375719E0,3.6102877E0,4.041283E0,2.7803545E0,2.244545E0,7.045233E-1,8.32407E-1,1.8923726E0,2.8079925E0,9.7891426E-1,1.4343462E0,2.960468E-1,5.523799E-1,2.9995763E-1,5.4354906E-1,2.970844E-1,6.980237E-1,1.1138778E0,1.5168619E0,4.267144E-1,5.550442E-1,4.759841E-1,1.0144196E0,5.907521E-1,6.987327E-2,2.9894197E-1,4.086165E-1,2.7227983E-1,0E0,2.55279E-1,4.7155142E-1,5.700822E-1,2.4525756E-1,1.3901561E-1,0E0,5.397942E-1,6.0497904E-1,0E0,6.075158E-1,7.258735E-1,5.056092E-1,0E0,0E0,0E0,0E0,0E0,7.391052E-1,5.891588E-1,1.9162297E-1,0E0,0E0,0E0,1.7977095E-1,2.1232125E-1,1.16657615E-1,2.4132463E-1,1.8628192E-1,0E0,0E0,6.96404E-2,3.8020873E-1,0E0,2.826221E-1,4.519018E-1,1.8167216E-1,0E0,0E0,0E0,6.989878E-1,2.9990172E-1,6.34287E-1,4.4906592E-1,1.8224907E-1,0E0,2.879207E-1,3.3964896E-1,2.774274E-1,3.4376484E-1,4.29142E-1,1.1431181E0,0E0,0E0,1.8875599E-1,0E0,2.0887072E-1,0E0,0E0,7.297641E-2,0E0,0E0,8.188763E-2,0E0,1.017406E-1,0E0,6.6802144E-2,0E0,5.740957E-1,3.0697715E-1,0E0,6.79674E-2,0E0,0E0,1.6622773E-1,9.1241464E-2,5.5591786E-1,3.1668025E-1,9.834034E-2,0E0,8.311837E-1,4.2392483E-1,4.9396443E-1,2.9289275E-1,1.0371399E-1,1.1467543E-1,4.7711384E-1,2.6041046E-1,7.608032E-2,7.014984E-2,1.4741725E-1,0E0,8.308856E-2,3.4276915E-1,5.387535E-1,5.0310516E-1,2.8662276E-1,3.2870054E-1,0E0,1.6047263E-1,0E0,1.2744422E-1,0E0,0E0,0E0,0E0,9.256461E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,37,37,39,39,40,40,41,41,47,47,48,48,49,49,53,53,54,54,55,55,56,56,57,57,60,60,61,61,63,63,64,64,65,65,69,69,70,70,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,83,83,85,85,88,88,91,91,93,93,95,95,97,97,98,98,100,100,103,103,104,104,105,105,106,106,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,119,119,121,121,122,122,123,123,124,124,125,125,126,126,128,128,130,130,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,60,62,64,66,68,-1,70,72,-1,74,76,78,-1,-1,-1,-1,-1,80,82,84,-1,-1,-1,86,88,90,92,94,-1,-1,96,98,-1,100,102,104,-1,-1,-1,106,108,110,112,114,-1,116,118,120,122,124,126,-1,-1,128,-1,130,-1,-1,132,-1,-1,134,-1,136,-1,138,-1,140,142,-1,144,-1,-1,146,148,150,152,154,-1,156,158,160,162,164,166,168,170,172,174,176,-1,178,180,182,184,186,188,-1,190,-1,192,-1,-1,-1,-1,194,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.0989723E3,1.8361508E3,2.1924436E5,6.082581E3,1E0,1E0,9.544947E6,2.5604828E2,2.3058404E7,3.4231E4,1.0017953E8,1.246E3,5.418831E11,1.0526E4,1E0,2.1690162E7,5E0,3.7852024E1,1.7589172E1,4.6577052E7,3.1140776E0,1.5005797E4,4.5723195E6,2.6998322E5,4.8856924E9,2.8839298E7,1E0,1.3758811E0,1.382507E-2,1.3707545E-4,2.8366232E0,4E0,9.2882293E-1,7.232191E0,9.077016E-3,2E0,2.612E3,-1.9257177E-2,2.2731024E7,2.7702793E-1,6.131E3,-1.3831333E-2,4.11364E-3,2.6448883E-2,1.2569483E-2,-5.415608E-3,2.074E3,3E0,1.0052E4,-2.9235488E-2,-3.1916164E-3,-8.936854E-3,7.325843E0,5.684E3,5.441732E9,1E0,3.8312717E8,9.131441E-3,-3.0843718E-3,8.037839E-1,7.1E1,8.064731E-3,4.2E1,1.752E3,5.670103E-2,-1.067668E-2,3.2372763E-3,-8.852307E-3,1.03E2,1.8820395E6,2E0,1.1693485E0,2.6120481E0,3.126517E-3,1.46617E5,1.5641598E7,3.53562E5,2.222456E0,1.8738E4,2.342147E7,-2.9165978E-2,-3.7501543E-3,1.4977E5,2.035327E-5,1.6504E4,-2.983439E-3,4.9738474E-3,1.2414683E5,4.760349E-3,1.8881481E-2,3.164136E6,-2.5915394E-3,7.2E1,-1.634411E-2,1.5609541E0,3.2335143E-3,9.721951E1,6.76E2,-8.971343E-3,7E0,-2.7974684E-2,-1.6372661E-3,6.823E3,2.1442623E1,4.4E1,1.322E3,1.6748592E0,-1.649142E-2,2.507E3,1.02036265E-5,6.6911E4,7.6842415E6,1.1666666E0,1.5427417E3,1.15062E5,1.49E2,1.4912975E-1,7.771383E0,1.6213043E3,-8.123022E-3,4.9807886E3,1.3035715E0,5.85E2,4.5089152E-1,8.7999426E-2,9.195571E2,-2.879858E-3,7.427971E2,3.6924707E-3,6.978874E5,-7.7620853E-4,-7.65925E-3,9.142128E-4,1.0711739E-2,1E0,1.0318675E-3,1.0352416E-2,4.7783647E-3,-4.8617697E-3,-8.911709E-4,-1.5999919E-2,-5.0410726E-3,-1.8899624E-3,5.770735E-3,2.4327245E-3,8.051201E-3,-8.6801E-3,-7.150093E-5,1.542632E-3,-1.811304E-2,-7.53114E-3,-9.6894056E-4,2.7262077E-3,-5.837075E-3,5.451583E-3,-1.3386873E-2,1.4635076E-2,-4.6381852E-4,1.2125771E-2,5.1110955E-3,5.948754E-3,-9.0009655E-4,1.2214011E-2,2.062011E-2,-1.14578106E-4,1.1949227E-2,1.2372728E-2,2.4482072E-3,7.9261395E-4,5.725627E-3,4.2306297E-3,1.3698818E-2,6.259363E-3,-1.8782145E-3,-2.2605092E-3,1.0110233E-2,-1.4399278E-3,-9.63816E-3,1.4849822E-2,2.2853755E-3,-1.5734792E-2,-6.3047344E-3,-1.9745992E-2,-8.89036E-3,9.2633255E-3,-5.6079915E-4,-1.3631343E-2,-6.233423E-3,-2.5626002E-3,-1.7562823E-2,-2.8152359E-3,2.65243E-3,2.6083496E-4,-6.4598685E-3],"split_indices":[26,51,51,27,51,58,109,9,51,44,10,44,2,30,9,6,44,3,52,55,47,53,4,27,32,30,44,66,33,0,37,37,52,37,46,0,6,2,0,46,34,2,0,0,0,0,0,2,8,2,0,0,0,52,9,5,80,43,0,0,26,8,0,6,28,56,0,0,0,11,44,16,41,52,0,9,47,1,53,9,44,0,0,28,0,9,0,0,27,0,0,1,0,3,0,56,0,51,2,0,8,0,0,2,55,3,2,52,0,2,37,28,46,52,4,28,8,40,52,4,0,4,52,2,26,40,4,0,32,0,27,0,0,0,0,80,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.211E3,1.617E3,5.94E2,1.241E3,3.76E2,4.76E2,1.18E2,5.26E2,7.15E2,3.31E2,4.5E1,3.27E2,1.49E2,6.5E1,5.3E1,4.29E2,9.7E1,2.84E2,4.31E2,2.4E2,9.1E1,2.4E1,2.1E1,2.96E2,3.1E1,2.1E1,1.28E2,2.7E1,3.8E1,8E0,4.5E1,4E2,2.9E1,8.8E1,9E0,1.5E1,2.69E2,4.28E2,3E0,5.3E1,1.87E2,8.7E1,4E0,3E0,2.1E1,1.8E1,3E0,2.87E2,9E0,2.4E1,7E0,9E0,1.2E1,1.01E2,2.7E1,6E0,2.1E1,3.6E1,2E0,7E0,3.8E1,3.93E2,7E0,2.3E1,6E0,8.6E1,2E0,2E0,7E0,2.51E2,1.8E1,2.74E2,1.54E2,4.5E1,8E0,1.29E2,5.8E1,2.6E1,6.1E1,1.56E2,1.31E2,6E0,3E0,2.2E1,2E0,7.4E1,2.7E1,3E0,2.4E1,3E0,3E0,7E0,1.4E1,3.4E1,2E0,2.3E1,1.5E1,3.18E2,7.5E1,1.5E1,8E0,4E0,2E0,6.8E1,1.8E1,1.91E2,6E1,1.3E1,5E0,1.91E2,8.3E1,8.9E1,6.5E1,3.6E1,9E0,4E1,8.9E1,3.8E1,2E1,7E0,1.9E1,1E1,5.1E1,1.14E2,4.2E1,5.2E1,7.9E1,3E0,1.9E1,3.6E1,3.8E1,4E0,2E1,3E0,4E0,2.8E1,6E0,1E1,1.3E1,1.19E2,1.99E2,5E0,7E1,5E0,3E0,5.5E1,1.3E1,2E0,1.6E1,1.89E2,2E0,3E1,3E1,3E0,1E1,1.87E2,4E0,3E0,8E1,5.1E1,3.8E1,4.8E1,1.7E1,1E1,2.6E1,2E0,7E0,2E1,2E1,5.1E1,3.8E1,3E0,3.5E1,1.8E1,2E0,5E0,2E0,6E0,4E0,4E0,4.7E1,1.4E1,1E2,1.5E1,2.7E1,6E0,4.6E1,1.6E1,6.3E1,2E0,1.7E1,2.6E1,1.2E1,5E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"195","size_leaf_vector":"1"}},{"base_weights":[-6.033452E-3,-1.0358769E-1,3.9813936E-2,-1.942817E-1,-6.1745383E-2,-1.1697877E-2,1.0803956E-1,-1.7949425E-1,-2.5856238E-2,-7.706516E-2,1.6658376E-1,-8.4362164E-2,1.3752901E-2,4.7002964E-2,1.563358E-1,-2.1742313E-1,-3.3771116E-2,-6.466159E-2,-2.9866877E-1,2.4500603E-2,7.373523E-2,-7.2687134E-2,-2.8456378E-1,4.9538348E-2,-6.816537E-2,1.4379666E-2,1.3205157E-1,-8.085103E-2,1.7187527E-1,-2.018274E-1,-2.7855562E-2,8.077387E-2,-1.17368475E-1,-9.588699E-2,6.962153E-3,-4.0495488E-1,-7.653028E-2,1.4326742E-1,-6.132889E-3,-1.10393455E-2,-6.3941255E-2,-6.942214E-2,-2.5891159E-2,3.126548E-1,3.77422E-2,-3.9642763E-1,-5.015038E-2,1.8486622E-1,3.1728367E-4,1.5917955E-1,-1.3885697E-2,-5.0375998E-2,-9.64396E-3,1.6148823E-1,4.43196E-1,-4.189315E-1,-1.8383333E-1,1.6499135E-1,-4.491185E-3,-1.6317861E-3,-1.7102742E-1,-5.6529712E-2,-1.9616179E-1,9.066534E-2,-5.140035E-2,-3.014924E-1,-2.8973939E-2,7.3709857E-4,-1.21143805E-2,2.1881682E-1,2.6616275E-2,-3.6694556E-2,-1.093476E-1,-7.1900254E-3,2.7015032E-3,4.2177117E-3,1.652391E-2,-1.15150504E-1,5.781316E-2,-2.4401745E-2,-2.1665555E-3,-5.8480393E-2,2.1617135E-2,-1.2560143E-3,2.580477E-1,2.0202989E-2,-9.4441585E-2,1.6997845E-1,-7.255301E-3,-5.6321383E-5,-5.610279E-3,3.5307115E-1,1.4598504E-1,1.2670928E-1,2.6512457E-2,-1.0880584E-2,-2.31881E-2,-3.6541125E-1,-1.6335796E-1,1.1284778E-5,9.502927E-3,-9.364954E-3,-5.362463E-4,-3.8829934E-2,-1.7115478E-1,-2.2934198E-2,-2.2160192E-1,1.2558613E-3,6.6156737E-3,-1.3598321E-1,-7.3411455E-3,-1.6537942E-2,-3.0168334E-3,3.803043E-3,1.2483151E-2,-3.6595382E-3,4.247228E-3,1.7822646E-3,-8.6883664E-2,-1.7862163E-2,-8.370714E-2,-1.911771E-1,3.0628834E-2,1.546593E-1,3.4596283E-2,-1.09961525E-1,-2.2931036E-3,3.133448E-3,1.5565852E-2,5.7942765E-3,1.7210978E-1,-1.0371802E-2,-2.0084156E-2,1.33192735E-2,1.436205E-1,1.9756125E-2,-5.073247E-2,1.2333345E-1,2.3006323E-1,1.2470756E-2,-2.9097062E-3,-1.8698698E-2,-5.9424415E-3,-7.881517E-3,4.925954E-3,-3.1361948E-3,1.0919695E-3,-3.4881614E-3,-1.4403788E-2,-7.0736916E-3,6.1759967E-3,-1.0425772E-2,-5.4923556E-4,-6.991151E-3,2.8511148E-3,-2.04874E-3,5.7026544E-3,-3.0981607E-3,2.7148423E-3,-7.576134E-3,-1.1652274E-3,7.952357E-3,-4.609466E-3,4.332542E-3,-1.112934E-2,4.7172927E-3,-4.5210174E-3,4.8691877E-3,1.32485675E-2,-3.2183055E-3,2.3158875E-3,-9.427165E-3,-3.3645353E-3,2.6591509E-3,-2.5192527E-3,5.5566574E-3,-2.4196651E-4,1.1659497E-3,9.493223E-3,-4.031898E-3,2.2890943E-3,2.9956459E-3,1.0778557E-2,-7.533326E-3,3.705227E-3,-4.4085383E-3,5.8975257E-3,-2.9257669E-3,1.0893436E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,-1,-1,71,73,-1,75,77,79,81,83,85,87,-1,89,-1,91,93,95,97,99,-1,-1,101,103,105,107,109,111,-1,-1,-1,113,115,117,119,-1,-1,-1,-1,121,123,-1,-1,125,-1,-1,127,129,131,133,-1,-1,-1,135,137,139,-1,-1,-1,141,143,-1,-1,-1,-1,145,147,149,151,-1,-1,153,155,-1,-1,-1,-1,-1,-1,157,159,-1,161,163,165,167,169,171,173,-1,-1,175,177,-1,179,-1,181,-1,183,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.946729E0,2.68614E0,5.318589E0,1.1456804E0,1.7198379E0,1.5971601E0,1.911694E0,1.1920414E0,0E0,1.2395813E0,1.0382514E0,5.028846E-1,1.8795252E0,7.999507E-1,1.3518276E0,9.4302464E-1,4.499606E-1,9.735366E-1,5.356543E-1,0E0,3.9760128E-1,3.0323446E-1,7.0205724E-1,1.3724767E0,1.1332788E0,5.034053E-1,1.001025E0,8.1905276E-2,8.8944244E-1,5.7538605E-1,0E0,3.1950638E-1,1.1166012E-1,1.1847136E0,6.5438217E-1,1.9609714E-1,1.6393986E-1,1.7043442E-1,0E0,0E0,2.4837089E-1,1.0179216E-1,0E0,1.8420005E-1,1.3200622E0,4.7110617E-1,8.4541523E-1,2.5610328E-1,3.6935565E-1,2.8940332E-1,0E0,7.211161E-2,0E0,9.286051E-1,4.6662927E-1,7.675052E-2,5.202961E-1,1.0352072E-1,0E0,0E0,8.878958E-2,4.3766963E-1,3.6950684E-1,1.8789056E-1,2.9340738E-1,1.7043054E-1,0E0,0E0,0E0,6.890428E-2,7.256262E-2,2.4926896E-1,5.227125E-1,0E0,0E0,0E0,0E0,5.600108E-1,8.468176E-1,0E0,0E0,5.319835E-1,0E0,0E0,1.7055607E-1,3.535853E-1,3.358103E-1,2.0002675E-1,0E0,0E0,0E0,8.293965E-1,5.5630875E-1,1.5973507E-1,0E0,0E0,0E0,1.1298108E-1,4.3350863E-1,0E0,0E0,0E0,0E0,3.6767825E-1,3.766551E-1,2.7622172E-1,1.3175344E-1,0E0,0E0,1.0875815E-1,2.735368E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.0640897E-1,2.7119103E-1,0E0,3.5128647E-1,5.408393E-1,1.831944E-1,4.4301558E-1,5.4079795E-1,3.2078922E-1,2.9304573E-1,0E0,0E0,1.9473833E-1,7.158929E-2,0E0,1.1657012E-1,0E0,4.4513988E-1,0E0,9.028568E-2,3.858664E-1,2.1449065E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,63,63,64,64,65,65,69,69,70,70,71,71,72,72,77,77,78,78,81,81,84,84,85,85,86,86,87,87,91,91,92,92,93,93,97,97,98,98,103,103,104,104,105,105,106,106,109,109,110,110,117,117,118,118,120,120,121,121,122,122,123,123,124,124,125,125,126,126,129,129,130,130,132,132,134,134,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,-1,-1,72,74,-1,76,78,80,82,84,86,88,-1,90,-1,92,94,96,98,100,-1,-1,102,104,106,108,110,112,-1,-1,-1,114,116,118,120,-1,-1,-1,-1,122,124,-1,-1,126,-1,-1,128,130,132,134,-1,-1,-1,136,138,140,-1,-1,-1,142,144,-1,-1,-1,-1,146,148,150,152,-1,-1,154,156,-1,-1,-1,-1,-1,-1,158,160,-1,162,164,166,168,170,172,174,-1,-1,176,178,-1,180,-1,182,-1,184,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,2.308943E0,1.5125709E8,1.2427474E8,5.185489E3,4.97E2,2.1150263E-1,3.402546E6,-2.5856238E-2,1.3521131E8,1.5151515E-3,6.1313953E1,5.666993E0,1.5435694E9,2.5534486E4,4.1973075E6,1.8000048E10,6.860185E2,1E0,2.4500603E-2,1.1855755E8,6E0,5.5E1,1.3E1,3.983889E3,7E0,2.4507338E4,6.4596E4,1.9013363E4,1E0,-2.7855562E-2,4.0844156E7,2.8091298E7,1.3697374E7,2.467284E7,7.6008E4,2.9E1,1.0040462E8,-6.132889E-3,-1.10393455E-2,3.09627E5,1.048E3,-2.5891159E-2,1.279012E7,1.279012E7,1E2,8.253647E6,2.8946466E2,4.719849E7,6.5880767E3,-1.3885697E-2,2.9138382E7,-9.64396E-3,1.0918E4,1.0238709E1,1.8058888E0,5.750032E3,5.530237E9,-4.491185E-3,-1.6317861E-3,6.538605E12,1.2342778E1,1.2414683E5,4.2648572E5,6.823E3,4.230839E6,-2.8973939E-2,7.3709857E-4,-1.21143805E-2,2.804366E-1,2.1601E4,4.1E0,1.76421E6,-7.1900254E-3,2.7015032E-3,4.2177117E-3,1.652391E-2,2.723E3,1.125584E-3,-2.4401745E-2,-2.1665555E-3,1.33027E5,2.1617135E-2,-1.2560143E-3,2.45009E6,6.2105E4,1.3715873E3,3.471E3,-7.255301E-3,-5.6321383E-5,-5.610279E-3,4.4844124E-1,3.2196458E6,4.4543375E4,2.6512457E-2,-1.0880584E-2,-2.31881E-2,5.987692E2,4.888E3,1.1284778E-5,9.502927E-3,-9.364954E-3,-5.362463E-4,3.2485715E2,3.38464E5,2.2E1,1E0,1.2558613E-3,6.6156737E-3,1.6052323E5,1E0,-1.6537942E-2,-3.0168334E-3,3.803043E-3,1.2483151E-2,-3.6595382E-3,4.247228E-3,3.79E2,1.109125E3,-1.7862163E-2,1E0,1.1111111E-1,8.192143E1,4.9355304E2,1.2414683E5,2.657E3,4.7675516E4,3.133448E-3,1.5565852E-2,5.884286E2,3.8466454E0,-1.0371802E-2,3.1546112E-2,1.33192735E-2,4.1331047E-1,1.9756125E-2,7.576E3,1.172E3,2.507E3,1.2470756E-2,-2.9097062E-3,-1.8698698E-2,-5.9424415E-3,-7.881517E-3,4.925954E-3,-3.1361948E-3,1.0919695E-3,-3.4881614E-3,-1.4403788E-2,-7.0736916E-3,6.1759967E-3,-1.0425772E-2,-5.4923556E-4,-6.991151E-3,2.8511148E-3,-2.04874E-3,5.7026544E-3,-3.0981607E-3,2.7148423E-3,-7.576134E-3,-1.1652274E-3,7.952357E-3,-4.609466E-3,4.332542E-3,-1.112934E-2,4.7172927E-3,-4.5210174E-3,4.8691877E-3,1.32485675E-2,-3.2183055E-3,2.3158875E-3,-9.427165E-3,-3.3645353E-3,2.6591509E-3,-2.5192527E-3,5.5566574E-3,-2.4196651E-4,1.1659497E-3,9.493223E-3,-4.031898E-3,2.2890943E-3,2.9956459E-3,1.0778557E-2,-7.533326E-3,3.705227E-3,-4.4085383E-3,5.8975257E-3,-2.9257669E-3,1.0893436E-2],"split_indices":[19,53,7,44,51,2,37,1,0,44,56,45,52,7,32,27,5,51,15,0,44,51,0,10,27,3,4,28,51,66,0,44,44,44,44,10,3,50,0,0,9,6,0,7,7,3,1,51,44,56,0,9,0,9,52,53,32,5,0,0,30,53,27,27,2,1,0,0,0,56,10,57,30,0,0,0,0,0,37,0,0,9,0,0,9,10,51,10,0,0,0,56,27,4,0,0,0,4,2,0,0,0,0,51,9,3,79,0,0,32,109,0,0,0,0,0,0,1,32,0,84,52,49,51,27,2,47,0,0,46,53,0,37,0,34,0,9,2,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.222E3,7.1E2,1.512E3,2.23E2,4.87E2,8.62E2,6.5E2,2.16E2,7E0,4.57E2,3E1,2.23E2,6.39E2,2.88E2,3.62E2,1.71E2,4.5E1,4.34E2,2.3E1,5E0,2.5E1,2.12E2,1.1E1,4.45E2,1.94E2,2.09E2,7.9E1,2.2E1,3.4E2,1.66E2,5E0,1.9E1,2.6E1,3.02E2,1.32E2,1.5E1,8E0,1.9E1,6E0,9E0,2.03E2,7E0,4E0,1.8E1,4.27E2,9E0,1.85E2,1.5E1,1.94E2,7.5E1,4E0,1.9E1,3E0,3.29E2,1.1E1,1.1E1,1.55E2,1.3E1,6E0,1.1E1,1.5E1,2.18E2,8.4E1,5.4E1,7.8E1,1.2E1,3E0,6E0,2E0,1.1E1,8E0,1.28E2,7.5E1,4E0,3E0,4E0,1.4E1,4.9E1,3.78E2,6E0,3E0,1.83E2,2E0,4E0,1.1E1,1.61E2,3.3E1,7.3E1,2E0,1.2E1,7E0,2.3E1,3.06E2,4E0,7E0,5E0,6E0,1.4E1,1.41E2,3E0,1E1,1.2E1,3E0,1.9E2,2.8E1,1.1E1,7.3E1,2.6E1,2.8E1,2.6E1,5.2E1,9E0,3E0,4E0,7E0,3E0,5E0,7.3E1,5.5E1,5E0,7E1,3.2E1,1.7E1,7.2E1,3.06E2,9.5E1,8.8E1,4E0,7E0,1.48E2,1.3E1,1.1E1,2.2E1,1.1E1,6.2E1,1.9E1,4E0,2.43E2,6.3E1,2E0,2E0,1.1E1,3E0,1.36E2,5E0,1.28E2,6.2E1,1.8E1,1E1,6E0,5E0,7E1,3E0,2.4E1,2E0,4.1E1,1.1E1,3.3E1,4E1,2.3E1,3.2E1,4E0,6.6E1,5E0,2.7E1,1.1E1,6E0,5.5E1,1.7E1,4.1E1,2.65E2,2.4E1,7.1E1,4.1E1,4.7E1,1.2E1,1.36E2,3E0,1E1,1.1E1,1.1E1,3.5E1,2.7E1,2E0,2E0,7E0,2.36E2,2E0,6.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[9.405747E-4,-9.67801E-2,4.8019942E-2,-1.1061384E-1,2.2231622E-1,1.4772786E-2,1.3318026E-1,-1.3521636E-1,3.1332105E-2,2.3476081E-2,1.423313E-1,-7.67966E-2,4.3327887E-2,2.2221E-1,8.672425E-2,-9.8686665E-2,-2.0928381E-1,-9.581241E-2,7.838163E-2,9.805465E-2,1.585811E-2,-8.360994E-2,9.903962E-3,1.6626535E-2,1.1579673E-1,2.7384904E-1,1.1552805E-2,4.5455545E-2,2.092735E-1,-1.5854001E-1,-7.515654E-2,-2.6364112E-1,-9.1056526E-2,-2.1589801E-3,-1.0565859E-2,4.9547978E-2,2.6084554E-1,-4.893036E-3,5.617209E-3,1.7633352E-1,-9.075394E-2,-1.9952772E-2,7.617595E-2,1.5105568E-1,1.047059E-2,2.4274541E-2,2.4062862E-1,5.8150417E-3,-5.672748E-2,9.813695E-2,-3.2853376E-2,3.5430688E-1,1.0760187E-1,-2.1986845E-1,-8.843453E-2,-5.9061125E-2,-2.0461074E-1,-3.312722E-1,-1.8060099E-1,-2.2937706E-1,-6.004533E-2,5.7742433E-3,1.0317442E-2,1.2812127E-3,1.428143E-2,-6.6304486E-4,1.1574467E-2,-1.0809489E-1,3.1322677E-2,1.1152836E-2,-9.856362E-2,8.616144E-2,-1.4912547E-1,-2.3041887E-2,1.7402737E-1,-7.152209E-2,6.562827E-2,2.0336077E-1,3.8512215E-1,-8.4130075E-3,2.6517618E-3,1.9134104E-1,7.022095E-2,6.9260605E-2,-5.9796635E-2,1.0219506E-2,2.2951068E-2,1.2843148E-1,-9.32614E-3,-3.0268786E-2,-3.2666102E-1,-3.3537056E-2,-2.1328887E-1,-6.7715675E-2,1.06589876E-1,-1.2345217E-1,-1.91287E-2,-2.4097782E-1,-3.99181E-1,-2.024529E-2,-1.591029E-1,-2.206123E-4,-1.2619002E-2,-1.0981261E-1,8.288105E-2,-4.092118E-3,3.453395E-2,-1.5973859E-1,-6.1845202E-2,6.685519E-2,-7.552483E-3,2.3567133E-2,-1.514734E-1,-1.6579416E-1,-4.333651E-2,5.6754872E-2,1.7021698E-1,-1.9095097E-2,-2.8243892E-2,1.5848946E-3,-7.4395607E-3,1.8695997E-1,1.5111138E-2,3.2183488E-3,-5.210339E-3,7.591875E-3,1.611661E-2,2.6483583E-1,1.2911731E-1,2.0532392E-2,6.789018E-3,3.938522E-3,-4.261057E-3,1.4840216E-2,1.3025013E-1,-1.7489352E-3,8.9046784E-2,1.18325E-1,-2.1998168E-3,9.935767E-3,-1.0425952E-1,5.6046788E-2,2.2306685E-1,-9.924409E-3,5.025849E-4,-1.6412143E-2,-1.1201835E-3,2.0441306E-3,-3.4526126E-3,-6.5117846E-3,-1.5272292E-2,-1.6655797E-3,-5.141632E-3,1.0267158E-2,2.1666042E-3,-9.442331E-3,-9.4914244E-4,-1.7462416E-2,-6.334247E-3,-1.9948708E-2,-4.4285674E-3,-8.379555E-3,2.5643287E-3,-1.3643001E-3,-8.330419E-3,6.4344266E-3,-2.362268E-3,-2.4026248E-3,2.7404379E-3,-8.1261415E-3,4.066437E-3,7.7382787E-3,-3.5617154E-3,3.284804E-4,7.5810906E-3,2.828255E-3,-1.0969698E-3,-8.575842E-3,5.1637082E-3,-1.5710667E-3,-1.3128491E-2,2.248056E-3,-5.1237745E-3,2.1711679E-3,1.4723121E-2,8.529587E-3,-5.127499E-3,-9.001964E-3,5.155631E-3,7.4679838E-3,1.272023E-2,-4.00101E-3,3.526843E-3,-5.415323E-3,2.6317812E-3,5.2212635E-3,1.3774957E-2,1.0923387E-2,3.5429439E-3,8.44318E-3,2.8146096E-3,5.6745755E-3,1.2355165E-3,-7.5169967E-4,9.268966E-3,-6.364355E-4,5.356693E-3,-9.986012E-3,-2.6994746E-3,4.336994E-3,-2.4524105E-3,1.1897639E-2,-2.2598696E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,-1,41,43,45,47,49,51,53,55,57,59,-1,-1,61,63,-1,-1,65,67,69,71,73,75,-1,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,105,-1,-1,-1,-1,107,109,111,113,115,117,119,121,123,125,127,129,-1,131,133,135,137,139,-1,-1,141,-1,143,145,147,149,151,153,155,-1,157,159,-1,161,-1,-1,163,165,-1,167,169,171,173,-1,175,177,179,181,183,185,-1,187,-1,-1,189,191,-1,-1,-1,193,195,197,-1,-1,-1,-1,-1,199,-1,201,203,-1,205,207,209,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.01030245E1,3.1763968E0,4.194076E0,2.3964128E0,6.3698494E-1,2.7932901E0,1.7012911E0,1.559453E0,6.17683E-1,0E0,1.9582522E-1,5.297705E-1,1.5731777E0,1.5415316E0,1.3812861E0,5.4156876E-1,1.2098846E0,1.7423958E-1,3.816582E-1,1.3139266E-1,0E0,4.7840536E-1,0E0,1.2997727E0,8.117161E-1,8.644247E-1,2.387402E-1,8.5718685E-1,9.7933483E-1,4.5349646E-1,5.796634E-1,6.758814E-1,2.5217503E-1,0E0,0E0,2.0062245E-1,1.18041754E-1,0E0,0E0,1.0978377E-1,5.177982E-1,9.058375E-1,5.2354133E-1,6.5989804E-1,2.579553E-1,0E0,4.7633266E-1,0E0,1.4614071E-1,3.0919766E-1,2.3524937E-1,4.350822E-1,3.0080298E-1,1.1644866E0,3.5611895E-1,3.7136114E-1,4.9605668E-1,3.3916235E-1,2.784319E-1,1.2050998E-1,3.7898037E-1,0E0,1.12861924E-1,0E0,0E0,0E0,0E0,4.9887276E-1,2.2940548E-1,5.405947E-1,3.826747E-1,5.2918863E-1,3.1766206E-1,1.6977625E-1,2.951789E-1,1.510658E-1,1.6910356E-1,3.4919143E-1,2.5366735E-1,0E0,1.22450314E-1,1.9070292E-1,2.0221335E-1,1.0982927E-1,2.0876963E-1,0E0,0E0,2.6150393E-1,0E0,1.7439352E-1,3.9147115E-1,1.3119124E-1,8.933294E-2,3.3898675E-1,8.091718E-2,2.0496765E-1,0E0,4.2874098E-1,4.4396687E-1,0E0,3.356074E-1,0E0,0E0,2.2498989E-1,1.1942937E-1,0E0,8.74735E-2,5.0898814E-1,4.5990682E-1,1.6066056E-1,0E0,4.6265006E-1,2.1305183E-1,7.5508463E-1,3.887737E-1,3.6940402E-1,3.0997336E-1,0E0,2.1659592E-1,0E0,0E0,2.3088408E-1,8.4856436E-2,0E0,0E0,0E0,1.4381196E-1,2.3014069E-1,2.0929599E-1,0E0,0E0,0E0,0E0,0E0,6.8814516E-2,0E0,1.8260539E-1,1.5496065E-1,0E0,7.187461E-2,1.9969428E-1,1.10328056E-1,1.9747066E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,87,87,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,98,98,100,100,103,103,104,104,106,106,107,107,108,108,109,109,111,111,112,112,113,113,114,114,115,115,116,116,118,118,121,121,122,122,126,126,127,127,128,128,134,134,136,136,137,137,139,139,140,140,141,141,142,142],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,-1,42,44,46,48,50,52,54,56,58,60,-1,-1,62,64,-1,-1,66,68,70,72,74,76,-1,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,106,-1,-1,-1,-1,108,110,112,114,116,118,120,122,124,126,128,130,-1,132,134,136,138,140,-1,-1,142,-1,144,146,148,150,152,154,156,-1,158,160,-1,162,-1,-1,164,166,-1,168,170,172,174,-1,176,178,180,182,184,186,-1,188,-1,-1,190,192,-1,-1,-1,194,196,198,-1,-1,-1,-1,-1,200,-1,202,204,-1,206,208,210,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,5.185489E3,6.306E3,1.5343539E0,2.1583429E-3,8.242408E4,3.14E2,7.0116967E-1,1.752512E2,2.3476081E-2,1.6316416E6,1.5106794E3,4.264897E6,2.5492362E7,4.247868E3,2.222456E0,5.7417827E0,7.44E2,9.22E4,1.1E1,1.585811E-2,7E0,9.903962E-3,9.76E2,2.4E1,1.1485E4,1.3900395E2,1.8071064E7,1.1336898E-1,7.44393E5,1.5988282E1,1.36054E-1,2.5918032E-4,-2.1589801E-3,-1.0565859E-2,5.26337E5,1.4492002E0,-4.893036E-3,5.617209E-3,3.79E2,9.705292E6,3.0947855E0,1.766486E7,5.08E2,7.451E3,2.4274541E-2,2.0718E4,5.8150417E-3,2.3274304E2,2.773036E1,5.473125E3,7.7101436E3,3.6600095E10,3.68E2,3.653602E7,2.0601392E3,1.645614E6,2.01351E5,4.26E2,1.17E2,9.348536E-1,5.7742433E-3,1.9928571E0,1.2812127E-3,1.428143E-2,-6.6304486E-4,1.1574467E-2,1.6812695E-2,5.961E3,1.8E1,1.993E3,2.517059E6,2.75E2,3.8833074E4,4.5478998E8,1.3797468E0,4.28586E5,2.4E1,8.387235E11,-8.4130075E-3,6.2395184E7,6.1E1,3.2004956E5,1.9214156E3,2.8352614E9,1.0219506E-2,2.2951068E-2,1.1803382E4,-9.32614E-3,1E0,3.3926086E2,2.778E3,4.4768806E1,1.4540612E7,3.046556E5,8.088206E-2,-1.91287E-2,1.4713E4,4.230839E6,-2.024529E-2,1.9495119E6,-2.206123E-4,-1.2619002E-2,1.440286E7,1.1900813E1,-4.092118E-3,2.821E3,1.01E3,2.6195612E-2,1.3607962E5,-7.552483E-3,1E0,1.803E3,8.5539725E8,1.4540612E7,1.3238013E2,1E0,-1.9095097E-2,1.3860265E6,1.5848946E-3,-7.4395607E-3,2.145E3,2.108E3,3.2183488E-3,-5.210339E-3,7.591875E-3,3.1615953E1,2.702696E6,3.0457064E1,2.0532392E-2,6.789018E-3,3.938522E-3,-4.261057E-3,1.4840216E-2,2.4242423E-2,-1.7489352E-3,1.0685876E7,2.3175376E3,-2.1998168E-3,3.9343938E-1,1.70752E4,6.7033327E-1,1.27E2,-9.924409E-3,5.025849E-4,-1.6412143E-2,-1.1201835E-3,2.0441306E-3,-3.4526126E-3,-6.5117846E-3,-1.5272292E-2,-1.6655797E-3,-5.141632E-3,1.0267158E-2,2.1666042E-3,-9.442331E-3,-9.4914244E-4,-1.7462416E-2,-6.334247E-3,-1.9948708E-2,-4.4285674E-3,-8.379555E-3,2.5643287E-3,-1.3643001E-3,-8.330419E-3,6.4344266E-3,-2.362268E-3,-2.4026248E-3,2.7404379E-3,-8.1261415E-3,4.066437E-3,7.7382787E-3,-3.5617154E-3,3.284804E-4,7.5810906E-3,2.828255E-3,-1.0969698E-3,-8.575842E-3,5.1637082E-3,-1.5710667E-3,-1.3128491E-2,2.248056E-3,-5.1237745E-3,2.1711679E-3,1.4723121E-2,8.529587E-3,-5.127499E-3,-9.001964E-3,5.155631E-3,7.4679838E-3,1.272023E-2,-4.00101E-3,3.526843E-3,-5.415323E-3,2.6317812E-3,5.2212635E-3,1.3774957E-2,1.0923387E-2,3.5429439E-3,8.44318E-3,2.8146096E-3,5.6745755E-3,1.2355165E-3,-7.5169967E-4,9.268966E-3,-6.364355E-4,5.356693E-3,-9.986012E-3,-2.6994746E-3,4.336994E-3,-2.4524105E-3,1.1897639E-2,-2.2598696E-3],"split_indices":[19,51,2,41,26,27,11,26,51,0,27,51,9,49,51,53,53,0,10,3,0,3,0,10,8,9,57,44,56,1,53,41,38,0,0,1,52,0,0,1,9,52,46,2,28,0,2,0,55,57,31,51,5,0,44,51,9,28,2,10,26,0,53,0,0,0,0,37,10,8,2,46,0,32,7,52,1,8,30,0,43,8,46,51,5,0,0,4,0,111,51,0,57,44,31,38,0,9,1,0,27,0,0,12,53,0,2,0,41,46,0,15,2,30,44,55,111,0,27,0,0,2,2,0,0,0,55,50,57,0,0,0,0,0,56,0,47,4,0,26,32,26,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.194E3,7.13E2,1.481E3,6.84E2,2.9E1,1.066E3,4.15E2,5.83E2,1.01E2,5E0,2.4E1,2.53E2,8.13E2,1.41E2,2.74E2,3.92E2,1.91E2,2.7E1,7.4E1,2.1E1,3E0,2.48E2,5E0,5.95E2,2.18E2,1.13E2,2.8E1,2.06E2,6.8E1,1.09E2,2.83E2,1.3E2,6.1E1,2.1E1,6E0,6.5E1,9E0,2E0,1.9E1,6E0,2.42E2,3.69E2,2.26E2,1.63E2,5.5E1,1.1E1,1.02E2,1E1,1.8E1,1.23E2,8.3E1,2.7E1,4.1E1,5.7E1,5.2E1,2.53E2,3E1,7E1,6E1,1E1,5.1E1,2.1E1,4.4E1,2E0,7E0,2E0,4E0,2.12E2,3E1,2.65E2,1.04E2,2.17E2,9E0,1.9E1,1.44E2,2.2E1,3.3E1,8.3E1,1.9E1,5E0,1.3E1,2.7E1,9.6E1,1.7E1,6.6E1,1.6E1,1.1E1,3.9E1,2E0,2.1E1,3.6E1,3.7E1,1.5E1,2.41E2,1.2E1,2.3E1,7E0,3.2E1,3.8E1,3E0,5.7E1,2E0,8E0,3.8E1,1.3E1,8E0,3.6E1,9.9E1,1.13E2,2.6E1,4E0,2.47E2,1.8E1,4.6E1,5.8E1,1.62E2,5.5E1,2E0,7E0,1.4E1,5E0,1.33E2,1.1E1,5E0,1.7E1,1E1,2.3E1,4.4E1,3.9E1,1.4E1,5E0,7E0,6E0,7E0,2E1,1.4E1,8.2E1,1.2E1,5E0,2.6E1,4E1,2.3E1,1.6E1,3E0,1.8E1,3.2E1,4E0,1.3E1,2.4E1,1.1E1,4E0,1.45E2,9.6E1,3E0,9E0,1.2E1,1.1E1,1.2E1,2E1,3.3E1,5E0,5.1E1,6E0,1.9E1,1.9E1,9E0,4E0,8E0,2.8E1,9.2E1,7E0,7E0,1.06E2,1.7E1,9E0,1.36E2,1.11E2,1.6E1,2E0,2.3E1,2.3E1,2.5E1,3.3E1,1.58E2,4E0,5.2E1,3E0,3E0,4E0,1.1E2,2.3E1,4E0,7E0,5E0,1.8E1,1E1,3.4E1,1.1E1,2.8E1,1E1,1E1,5.1E1,3.1E1,5E0,7E0,2.2E1,4E0,1E1,3E1,1.7E1,6E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[-3.877208E-3,3.2799385E-2,-1.0712241E-1,-2.8278986E-2,8.805983E-2,-2.1894452E-1,-6.049587E-2,-4.528675E-2,1.05012156E-1,2.2848764E-2,1.4284408E-1,-1.855749E-1,-5.0835484E-1,-8.4395505E-2,7.5011685E-2,-7.432164E-2,3.8957037E-2,6.5099865E-2,1.212777E-2,4.234979E-3,2.321215E-1,9.4077975E-2,2.1815579E-1,-1.6814755E-2,-1.6133863E-1,-4.0432224E-3,-2.679186E-2,-4.964765E-2,-1.6805308E-1,1.2503134E-2,3.5642684E-2,-6.43103E-2,-2.1317783E-1,6.509883E-2,-7.290695E-2,9.957802E-2,-1.2683086E-2,1.4485032E-2,-5.6710355E-3,2.952321E-1,-3.5449438E-2,1.1087754E-1,-4.1683566E-2,2.7703708E-1,1.1295598E-1,-3.934417E-1,-1.3813725E-1,-1.818673E-1,-2.4104906E-2,-1.4426537E-1,-4.7027683E-1,6.838933E-3,1.8186176E-3,-6.1162595E-2,-2.4599988E-2,-5.4643625E-1,-1.3687685E-1,7.416276E-2,-9.132253E-3,-1.2421441E-1,2.0674136E-3,-2.7074102E-2,1.2434686E-1,-1.2793535E-1,3.5519794E-2,1.4261257E-2,-1.3814083E-1,2.1092315E-1,2.748785E-2,7.7870707E-3,-1.6723301E-2,1.1553278E-1,-1.428707E-2,8.9393005E-2,-9.686105E-2,3.174891E-1,1.3856335E-1,1.7393078E-1,-1.967116E-2,-3.7967658E-3,-2.4346009E-2,-3.0482876E-1,-1.05989404E-1,-1.5063068E-1,-1.925679E-2,-1.6590921E-2,-1.892389E-1,-2.918266E-1,-1.01593636E-1,-2.6356533E-2,-5.7521192E-3,-2.2469398E-2,1.07829925E-2,-7.9764225E-2,1.0667508E-2,-3.191535E-2,-6.924849E-3,-5.363943E-2,-3.1588894E-1,1.5935253E-2,1.1053892E-1,-5.24308E-2,-1.0776473E-2,3.816475E-3,-1.2582364E-2,1.5233763E-1,4.167514E-2,-1.2292154E-2,-7.337011E-4,-1.5980955E-3,4.0122825E-3,2.6506364E-2,-5.127422E-3,-1.9152091E-1,-1.1088175E-4,8.413576E-4,1.1437685E-2,1.4580747E-1,5.1359214E-2,1.3528468E-1,-2.842409E-3,1.4211052E-2,-2.1932253E-1,1.5549107E-1,3.6836168E-1,2.0632823E-1,-5.6639887E-2,1.1089031E-1,1.9050192E-2,5.6296196E-3,-2.6522835E-3,-3.391526E-1,-3.7462027E-3,-5.55788E-2,-1.7805077E-1,-1.6504104E-1,3.0650687E-3,-7.1126424E-2,2.9189608E-3,-1.2791733E-2,-8.948216E-5,-1.4940485E-1,-1.8459035E-2,3.234313E-2,-1.2721214E-1,3.4704008E-3,-1.9955412E-1,4.132589E-3,-3.9417082E-3,-7.331195E-3,1.3458334E-3,1.967879E-3,-5.285289E-3,-1.7558018E-2,-1.9359108E-3,-1.2455842E-2,1.4537241E-3,8.635877E-3,2.6927034E-3,4.5932024E-3,-4.334831E-3,4.60416E-3,1.0143459E-2,-2.4432836E-3,5.328754E-3,6.8748607E-3,6.6543475E-4,-1.2864823E-2,-4.461266E-3,5.375371E-3,9.605146E-3,-1.6847282E-3,4.2170873E-3,1.6177094E-3,1.3097572E-2,-4.260309E-3,5.4551773E-3,-1.4611319E-2,-1.3892674E-3,9.40548E-3,2.3145315E-3,-4.8830933E-3,1.7415706E-2,-6.604581E-5,1.0335016E-2,4.680858E-3,-5.759511E-3,5.6863166E-3,-4.643532E-3,-1.6918475E-2,-2.4180296E-3,-6.256155E-5,-6.439706E-3,6.514512E-3,-1.0096763E-2,-2.961473E-3,-8.493939E-3,-9.300171E-3,-1.4063612E-3,1.1974643E-3,-2.4372826E-3,-1.1861205E-2,-2.1570863E-3,-2.433282E-3,1.0842185E-2,-7.190833E-3,-1.7017887E-3,-1.0963035E-3,3.8999275E-3,-1.5716087E-2,6.584162E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,-1,45,-1,-1,47,49,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,-1,95,97,99,-1,101,-1,103,105,107,109,111,113,115,-1,-1,-1,117,-1,119,121,123,125,127,129,-1,-1,131,133,135,-1,137,139,141,143,-1,-1,145,-1,147,149,-1,-1,151,153,155,157,159,-1,-1,-1,161,163,-1,-1,-1,-1,165,-1,167,-1,-1,-1,169,171,173,-1,175,177,179,181,183,185,187,-1,-1,-1,189,-1,191,193,195,-1,197,199,-1,-1,201,-1,203,205,207,209,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.364508E0,5.5040956E0,3.0055895E0,1.7616682E0,3.05483E0,1.5474396E0,1.3341485E0,1.6857494E0,5.545824E-1,1.5259972E0,1.6872616E0,6.3365126E-1,5.4140186E-1,1.006172E0,4.811504E-1,6.980405E-1,5.2235174E-1,1.9435349E-1,0E0,1.1308049E0,5.508357E-1,6.5235066E-1,1.0960703E0,0E0,6.887963E-1,0E0,0E0,8.331749E-1,6.6545486E-1,0E0,2.0662515E-1,6.89052E-1,7.897557E-1,3.6320114E-1,2.1251465E-1,1.6021785E-1,1.3232052E-1,0E0,9.287598E-1,5.5566216E-1,5.4844683E-1,5.3871655E-1,2.4009793E-1,6.057863E-1,5.452446E-1,4.9099767E-1,6.4825535E-1,2.4014068E-1,2.5619054E-1,5.748348E-1,1.9483614E-1,0E0,2.46455E-1,6.3819766E-1,0E0,2.663833E-1,4.151706E-1,2.9477167E-1,0E0,1.8359903E-1,0E0,2.748575E-1,9.0841234E-2,1.0788771E-1,6.705882E-2,4.8079923E-1,3.3271968E-1,1.7179823E-1,0E0,0E0,0E0,4.8016334E-1,0E0,7.8329355E-2,3.1596404E-1,6.7619514E-1,3.8206828E-1,6.691574E-1,1.3057165E-1,0E0,0E0,1.2439203E-1,3.8243103E-1,1.2758398E-1,0E0,2.1406785E-1,1.566889E-1,2.8570163E-1,2.648902E-1,0E0,0E0,1.7984065E-1,0E0,4.8112726E-1,3.3047587E-1,0E0,0E0,1.3375023E-1,1.6022825E-1,2.6775357E-1,3.4088397E-1,1.1636404E-1,0E0,0E0,0E0,8.979106E-2,9.3605995E-2,0E0,0E0,0E0,0E0,4.121992E-1,0E0,2.562374E-1,0E0,0E0,0E0,2.8022623E-1,3.0695644E-1,1.1232145E-1,0E0,1.6098848E-1,2.0467025E-1,1.1481577E-1,5.6195736E-1,9.6108556E-2,1.03505105E-1,1.3218051E-1,0E0,0E0,0E0,1.4781773E-1,0E0,3.0508894E-1,6.690259E-1,6.643319E-2,0E0,2.7877E-1,2.0040129E-1,0E0,0E0,1.13537565E-1,0E0,2.446888E-1,1.7538297E-1,8.2192115E-2,1.6302948E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,24,24,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,55,55,56,56,57,57,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,71,71,73,73,74,74,75,75,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86,87,87,88,88,91,91,93,93,94,94,97,97,98,98,99,99,100,100,101,101,105,105,106,106,111,111,113,113,117,117,118,118,119,119,121,121,122,122,123,123,124,124,125,125,126,126,127,127,131,131,133,133,134,134,135,135,137,137,138,138,141,141,143,143,144,144,145,145,146,146],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,-1,46,-1,-1,48,50,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,-1,96,98,100,-1,102,-1,104,106,108,110,112,114,116,-1,-1,-1,118,-1,120,122,124,126,128,130,-1,-1,132,134,136,-1,138,140,142,144,-1,-1,146,-1,148,150,-1,-1,152,154,156,158,160,-1,-1,-1,162,164,-1,-1,-1,-1,166,-1,168,-1,-1,-1,170,172,174,-1,176,178,180,182,184,186,188,-1,-1,-1,190,-1,192,194,196,-1,198,200,-1,-1,202,-1,204,206,208,210,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,3.7810526E2,2.0666666E0,2.3314082E6,1.8991614E5,2.72258E5,3.970405E3,4.264897E6,2.0827537E0,4.247868E3,5.964E3,4.8543688E-2,2.2837209E2,5.097268E7,2.8781365E6,6.015825E0,5.8730896E2,1E0,1.212777E-2,2.04E5,1.96E2,2E0,8.2E1,-1.6814755E-2,1.0192E4,-4.0432224E-3,-2.679186E-2,9.7043E4,8.91942E5,1.2503134E-2,2.8091298E7,4.5158855E6,1E0,1E0,2.1584728E6,1.198E5,1.9826213E0,1.4485032E-2,4.0844156E7,2.4507338E4,1.1803382E4,3.7852024E1,1.1644968E1,2.1298597E0,3.0501078E8,9.35E2,7.22E2,1.5572634E6,1.08E2,8.92686E5,3.1827815E12,6.838933E-3,6.318545E8,2.631294E2,-2.4599988E-2,4.0911578E2,8.285757E4,8.31E2,-9.132253E-3,7.7344055E4,2.0674136E-3,1.08504E5,2.2174756E4,8.01E2,1.1860938E8,7.806529E1,7.2E1,5.3473564E1,2.748785E-2,7.7870707E-3,-1.6723301E-2,2.8E1,-1.428707E-2,1.8E1,9E0,1.0989723E3,1.8222668E2,4.5505118E2,4.90027E1,-3.7967658E-3,-2.4346009E-2,9.824211E5,3.5551153E5,4.416098E5,-1.925679E-2,2.046E3,8.088206E-2,2.8035452E11,2.7864855E11,-2.6356533E-2,-5.7521192E-3,3.8312717E8,1.07829925E-2,1.0620689E0,5.8601086E1,-3.191535E-2,-6.924849E-3,2.1E1,2.3E1,3.5902756E-6,2.53E2,2.0979407E-4,-1.0776473E-2,3.816475E-3,-1.2582364E-2,1.7486666E3,1.45625E1,-1.2292154E-2,-7.337011E-4,-1.5980955E-3,4.0122825E-3,1.2224265E1,-5.127422E-3,1.2152382E5,-1.1088175E-4,8.413576E-4,1.1437685E-2,2.98E2,2.507E3,2.657E3,-2.842409E-3,5.542E3,1E0,1.1323028E0,3.258768E-4,1E0,1.4386049E6,1.894826E6,1.9050192E-2,5.6296196E-3,-2.6522835E-3,2.1383E4,-3.7462027E-3,7.4456794E5,2.014576E8,9.318E3,3.0650687E-3,1E0,2.3058404E7,-1.2791733E-2,-8.948216E-5,1E0,-1.8459035E-2,9.711E3,4.982003E6,5.2E1,5.0162595E8,4.132589E-3,-3.9417082E-3,-7.331195E-3,1.3458334E-3,1.967879E-3,-5.285289E-3,-1.7558018E-2,-1.9359108E-3,-1.2455842E-2,1.4537241E-3,8.635877E-3,2.6927034E-3,4.5932024E-3,-4.334831E-3,4.60416E-3,1.0143459E-2,-2.4432836E-3,5.328754E-3,6.8748607E-3,6.6543475E-4,-1.2864823E-2,-4.461266E-3,5.375371E-3,9.605146E-3,-1.6847282E-3,4.2170873E-3,1.6177094E-3,1.3097572E-2,-4.260309E-3,5.4551773E-3,-1.4611319E-2,-1.3892674E-3,9.40548E-3,2.3145315E-3,-4.8830933E-3,1.7415706E-2,-6.604581E-5,1.0335016E-2,4.680858E-3,-5.759511E-3,5.6863166E-3,-4.643532E-3,-1.6918475E-2,-2.4180296E-3,-6.256155E-5,-6.439706E-3,6.514512E-3,-1.0096763E-2,-2.961473E-3,-8.493939E-3,-9.300171E-3,-1.4063612E-3,1.1974643E-3,-2.4372826E-3,-1.1861205E-2,-2.1570863E-3,-2.433282E-3,1.0842185E-2,-7.190833E-3,-1.7017887E-3,-1.0963035E-3,3.8999275E-3,-1.5716087E-2,6.584162E-4],"split_indices":[26,51,52,27,27,9,51,9,52,51,2,57,4,44,46,52,4,15,0,5,8,6,8,0,9,0,0,1,9,0,44,46,63,64,50,7,55,0,44,4,4,52,55,37,31,0,2,47,8,1,30,0,43,51,0,4,32,2,0,32,0,7,46,2,5,52,3,55,0,0,0,8,0,3,8,51,57,55,55,0,0,31,32,46,0,2,38,30,30,0,0,43,0,53,49,0,0,8,3,37,10,37,0,0,0,32,57,0,0,0,0,55,0,32,0,0,0,0,2,2,0,28,111,41,37,109,27,11,0,0,0,9,0,31,7,9,0,80,44,0,0,15,0,2,28,8,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.207E3,1.629E3,5.78E2,7.74E2,8.55E2,1.69E2,4.09E2,6.87E2,8.7E1,3.91E2,4.64E2,1.53E2,1.6E1,3.48E2,6.1E1,5.11E2,1.76E2,7.1E1,1.6E1,3.6E2,3.1E1,2.83E2,1.81E2,1.6E1,1.37E2,3E0,1.3E1,2.47E2,1.01E2,9E0,5.2E1,4.78E2,3.3E1,1.43E2,3.3E1,4.9E1,2.2E1,1E1,3.5E2,2.5E1,6E0,2.52E2,3.1E1,1.15E2,6.6E1,1.1E1,1.26E2,3.9E1,2.08E2,9.5E1,6E0,1.1E1,4.1E1,4.76E2,2E0,5E0,2.8E1,1.39E2,4E0,2.3E1,1E1,8E0,4.1E1,6E0,1.6E1,3.05E2,4.5E1,2.1E1,4E0,4E0,2E0,2.5E2,2E0,9E0,2.2E1,8.8E1,2.7E1,4.5E1,2.1E1,4E0,7E0,1.9E1,1.07E2,3.6E1,3E0,2E2,8E0,2E1,7.5E1,4E0,2E0,3.8E1,3E0,3.78E2,9.8E1,3E0,2E0,2E1,8E0,5.4E1,8.5E1,1.5E1,8E0,6E0,2E0,3E1,1.1E1,2E0,4E0,7E0,9E0,2.79E2,2.6E1,3.2E1,1.3E1,4E0,1.7E1,1.69E2,8.1E1,7E0,2E0,1.2E1,1E1,2.2E1,6.6E1,2E1,7E0,3.7E1,8E0,4E0,1.7E1,1.6E1,3E0,6.4E1,4.3E1,3.4E1,2E0,5.2E1,1.48E2,5E0,3E0,1E1,1E1,1.2E1,6.3E1,3.4E1,4E0,1.5E1,3.63E2,9E0,8.9E1,8E0,1.2E1,6E0,2E0,2E0,5.2E1,3.2E1,5.3E1,3E0,1.2E1,1.9E1,1.1E1,5E0,6E0,2.3E1,2.56E2,1.5E1,1.7E1,1.22E2,4.7E1,2.6E1,5.5E1,5E0,2E0,6E0,6E0,6E0,4E0,1.4E1,8E0,2E0,6.4E1,2E0,1.8E1,2E0,5E0,3.5E1,2E0,1.4E1,2E0,4E1,2.4E1,5E0,3.8E1,7E0,2.7E1,1.1E1,4.1E1,1.05E2,4.3E1,4E0,6E0,9E0,3E0,4.6E1,1.7E1,2.6E1,8E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"211","size_leaf_vector":"1"}},{"base_weights":[4.8600553E-4,3.0312635E-2,-1.05012886E-1,-2.6249927E-2,7.810339E-2,-1.564129E-1,-3.0728808E-2,-1.13342125E-2,-1.5280707E-1,9.830686E-2,-8.195923E-2,-1.7212874E-1,-2.107791E-2,-5.3765528E-2,1.3267887E-1,-6.1565977E-2,1.9727366E-2,-5.043195E-1,-8.6293444E-2,8.465633E-2,3.1836426E-1,-1.6871345E-1,9.636958E-2,-1.4672238E-1,-3.594795E-1,6.7292308E-3,-7.677017E-2,-3.9622173E-2,-2.0292942E-1,2.2355819E-1,-3.3092028E-3,-7.838762E-2,1.1412199E-1,2.1138065E-1,1.2414144E-2,-2.8783822E-2,-7.5537055E-3,-1.1473015E-1,1.519869E-1,1.1720951E-1,-1.0369851E-3,2.331787E-2,1.4631914E-1,-2.1097803E-1,-2.9422985E-3,2.4377951E-1,8.860935E-3,-2.6329866E-1,-1.290852E-1,-2.3713661E-2,-2.5113937E-1,3.1123562E-3,-7.104029E-3,-1.4569263E-1,-2.1913297E-2,-1.6382577E-2,-8.4382773E-4,4.6082973E-3,1.3284705E-2,2.7509755E-3,-4.180971E-3,-5.809792E-2,-2.1289344E-1,-8.4635E-2,1.8507627E-1,1.4540623E-2,5.133622E-3,-5.7048993E-3,8.224845E-2,5.7360525E-3,-1.3293687E-1,1.0006647E-2,-1.6638365E-3,5.5435788E-2,1.4563861E-1,-8.6574934E-2,3.0910965E-2,2.148533E-1,-1.9820185E-1,-2.5290787E-1,3.6837618E-4,3.6757253E-3,1.5178106E-2,1.256521E-2,-6.2309742E-2,-1.8899329E-2,-2.137916E-1,-3.9718494E-2,-1.6021594E-1,1.6522965E-4,-2.79652E-1,4.4152634E-3,-3.0824707E-3,-4.736614E-3,-1.1075133E-2,-5.2414995E-2,2.342102E-2,7.628729E-2,-7.219992E-2,-2.2421569E-2,-1.5487069E-1,1.9570917E-3,-1.2202594E-2,1.1656537E-2,2.8253123E-3,-1.0058999E-3,-2.3857497E-2,-1.3356374E-2,1.18881345E-1,-1.8156689E-2,-1.1179035E-1,1.21958025E-1,6.758394E-3,1.7729849E-1,3.2767434E-2,3.4678152E-3,-1.1367231E-1,8.557083E-3,9.889955E-2,1.6453731E-1,1.8685462E-2,-1.4390402E-3,-1.3516593E-2,-2.7268752E-1,-5.208963E-3,-9.993713E-3,-2.4203902E-4,-3.3084774E-1,-4.8238514E-3,-8.477558E-2,5.491331E-3,-6.543089E-2,-1.8579948E-1,-2.6022848E-3,-1.375564E-2,1.6427243E-3,-8.4474094E-2,5.589041E-2,-5.0034355E-2,-1.9804868E-3,8.574156E-3,-2.6855955E-3,-9.251338E-3,5.483597E-5,-7.861774E-3,-1.6278843E-3,2.053578E-3,1.0890389E-3,-1.3388429E-2,1.1368011E-2,3.897389E-3,-3.4236624E-3,-1.0340235E-2,7.0559285E-3,-1.1887952E-3,-3.239281E-3,1.7013979E-3,1.2187664E-2,6.6231573E-3,-1.8863032E-2,2.4461937E-3,-4.1931067E-3,-1.9001286E-2,-4.2810032E-4,5.032438E-3,1.8000887E-3,9.130217E-3,9.516071E-3,2.251441E-3,-9.309955E-3,-1.4864108E-2,-8.190865E-3,-1.9270238E-2,-2.1116459E-3,-1.2168858E-2,-7.692361E-3,4.9683347E-4,-1.29549885E-2,-5.7555744E-3,-7.78201E-3,-3.0056112E-3,5.8894013E-3,1.2143925E-3,-5.858001E-3,4.217352E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,-1,77,79,-1,81,83,85,87,-1,89,91,-1,93,95,-1,-1,-1,-1,-1,-1,97,99,101,103,-1,-1,105,107,-1,109,-1,-1,111,113,115,117,119,121,123,-1,-1,-1,-1,125,-1,127,129,131,-1,133,-1,-1,-1,-1,135,137,139,141,-1,143,-1,-1,-1,-1,145,-1,147,149,-1,151,153,155,157,159,-1,161,163,165,167,-1,-1,-1,169,-1,-1,-1,171,-1,173,-1,175,177,-1,-1,-1,179,181,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9384575E0,4.648795E0,1.8508396E0,1.485768E0,3.021978E0,6.067929E-1,7.6000893E-1,1.1027707E0,1.8970509E0,2.4602776E0,1.6406901E0,1.1705513E0,3.044108E-1,3.6326295E-1,3.0970237E-1,8.068929E-1,6.098764E-1,4.706223E-1,4.9852693E-1,2.1787915E0,1.5488162E0,2.9754424E-1,4.4932747E-1,4.347973E-1,4.0247297E-1,0E0,1.5413485E-1,3.011063E-1,4.3258268E-1,9.775716E-2,6.836546E-2,6.6171455E-1,3.541277E-1,1.3571507E-1,5.3443885E-1,0E0,0E0,2.9816025E-1,1.12362355E-1,9.859195E-1,5.929527E-1,0E0,6.800937E-1,4.6143007E-1,0E0,1.6612178E-1,4.5709652E-1,1.32061E-1,5.500467E-1,0E0,1.4631462E-1,9.402318E-2,0E0,6.786272E-2,1.9481592E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.1484958E-1,4.4754624E-1,1.7713755E-1,1.4377725E-1,0E0,0E0,8.224585E-1,3.0628633E-1,0E0,2.9533494E-1,0E0,0E0,5.8203906E-1,1.3771296E0,2.6957595E-1,2.3928347E-1,1.6107678E-1,7.400957E-2,8.314729E-2,0E0,0E0,0E0,0E0,1.7028904E-1,0E0,2.6603317E-1,3.947778E-1,3.467338E-1,0E0,7.923198E-2,0E0,0E0,0E0,0E0,2.4192855E-1,1.3874437E-1,2.9336348E-1,3.2149637E-1,0E0,8.227688E-2,0E0,0E0,0E0,0E0,5.428679E-1,0E0,2.7119997E-1,2.5156164E-1,0E0,2.2312188E-1,3.8586104E-1,2.5611132E-1,8.211994E-1,8.443292E-1,0E0,2.8975415E-1,2.2282146E-1,2.2900707E-1,9.4370306E-2,0E0,0E0,0E0,6.6762686E-2,0E0,0E0,0E0,7.9322934E-2,0E0,2.8217512E-1,0E0,2.6528633E-1,6.4657784E-1,0E0,0E0,0E0,8.661267E-2,8.2774356E-2,2.1658063E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,42,42,43,43,45,45,46,46,47,47,48,48,50,50,51,51,53,53,54,54,61,61,62,62,63,63,64,64,67,67,68,68,70,70,73,73,74,74,75,75,76,76,77,77,78,78,79,79,84,84,86,86,87,87,88,88,90,90,95,95,96,96,97,97,98,98,100,100,105,105,107,107,108,108,110,110,111,111,112,112,113,113,114,114,116,116,117,117,118,118,119,119,123,123,127,127,129,129,131,131,132,132,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,-1,78,80,-1,82,84,86,88,-1,90,92,-1,94,96,-1,-1,-1,-1,-1,-1,98,100,102,104,-1,-1,106,108,-1,110,-1,-1,112,114,116,118,120,122,124,-1,-1,-1,-1,126,-1,128,130,132,-1,134,-1,-1,-1,-1,136,138,140,142,-1,144,-1,-1,-1,-1,146,-1,148,150,-1,152,154,156,158,160,-1,162,164,166,168,-1,-1,-1,170,-1,-1,-1,172,-1,174,-1,176,178,-1,-1,-1,180,182,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,3.8637936E2,1E0,9.669789E0,4.0844156E7,1.1879E4,4.247868E3,1.454213E2,2E0,6.082581E3,3.4409692E3,1.3983857E10,1E0,1.8000048E10,5.9767612E7,2.3920168E6,1E0,7.696629E0,1.2677199E7,1.0938637E1,3.2475834E7,1.3933473E5,1.058E3,4.5555557E1,1.8736842E0,6.7292308E-3,1.00663277E9,1.06E3,1.7243155E12,6.08767E3,1.1855755E8,1.8828108E10,8.6906275E-6,6E0,2.065904E6,-2.8783822E-2,-7.5537055E-3,6.48334E5,9.112E3,5.862126E2,4.423E3,2.331787E-2,1.3205668E7,1.32E2,-2.9422985E-3,9.0754684E1,9.042859E4,2.35E2,3.6628513E10,-2.3713661E-2,1.2920592E0,2.8878505E0,-7.104029E-3,1.2471935E7,5.148149E5,-1.6382577E-2,-8.4382773E-4,4.6082973E-3,1.3284705E-2,2.7509755E-3,-4.180971E-3,2.183847E0,4.925824E8,1.1779856E2,1.9041E4,1.4540623E-2,5.133622E-3,3.0491832E6,1.002E3,5.7360525E-3,3.983889E3,1.0006647E-2,-1.6638365E-3,5.500988E2,1.9493858E7,4.8215595E1,3.33E2,3.1699734E2,4.93E2,8.9712226E-1,3.6837618E-4,3.6757253E-3,1.5178106E-2,1.256521E-2,1.0937254E5,-1.8899329E-2,6.0292965E-1,1.56221E5,1.00137E5,1.6522965E-4,1.10280626E-1,4.4152634E-3,-3.0824707E-3,-4.736614E-3,-1.1075133E-2,5.18E2,2.7246006E7,3.97E2,1.8955729E1,-2.2421569E-2,3.3704692E10,1.9570917E-3,-1.2202594E-2,1.1656537E-2,2.8253123E-3,1.566593E6,-2.3857497E-2,2.69E2,2.1383E4,-1.8156689E-2,3.45E3,3.0923106E-10,5.61E2,1.0594161E1,1.1660184E3,3.4678152E-3,3.202814E5,1.2679E4,7.8726E-1,1E0,1.8685462E-2,-1.4390402E-3,-1.3516593E-2,1E0,-5.208963E-3,-9.993713E-3,-2.4203902E-4,3.68E2,-4.8238514E-3,9.9341E4,5.491331E-3,9E2,3.2069644E2,-2.6022848E-3,-1.375564E-2,1.6427243E-3,3.0966794E8,2.5330253E-2,6.823E3,-1.9804868E-3,8.574156E-3,-2.6855955E-3,-9.251338E-3,5.483597E-5,-7.861774E-3,-1.6278843E-3,2.053578E-3,1.0890389E-3,-1.3388429E-2,1.1368011E-2,3.897389E-3,-3.4236624E-3,-1.0340235E-2,7.0559285E-3,-1.1887952E-3,-3.239281E-3,1.7013979E-3,1.2187664E-2,6.6231573E-3,-1.8863032E-2,2.4461937E-3,-4.1931067E-3,-1.9001286E-2,-4.2810032E-4,5.032438E-3,1.8000887E-3,9.130217E-3,9.516071E-3,2.251441E-3,-9.309955E-3,-1.4864108E-2,-8.190865E-3,-1.9270238E-2,-2.1116459E-3,-1.2168858E-2,-7.692361E-3,4.9683347E-4,-1.29549885E-2,-5.7555744E-3,-7.78201E-3,-3.0056112E-3,5.8894013E-3,1.2143925E-3,-5.858001E-3,4.217352E-3],"split_indices":[18,51,58,53,44,2,51,51,16,51,51,5,63,5,44,27,84,55,9,52,49,32,0,51,52,0,7,2,30,4,44,30,37,3,27,0,0,30,10,51,2,0,9,3,0,55,27,2,30,0,55,53,0,44,27,0,0,0,0,0,0,53,5,4,9,0,0,46,28,0,27,0,0,4,50,53,8,55,0,26,0,0,0,0,32,0,26,1,28,0,26,0,0,0,0,0,44,2,57,0,30,0,0,0,0,9,0,10,9,0,11,36,10,55,4,0,27,2,38,63,0,0,0,15,0,0,0,0,0,1,0,2,51,0,0,0,5,37,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.203E3,1.718E3,4.85E2,7.87E2,9.31E2,2.86E2,1.99E2,7.05E2,8.2E1,8.27E2,1.04E2,2.56E2,3E1,1.75E2,2.4E1,2.69E2,4.36E2,1.2E1,7E1,7.8E2,4.7E1,7E1,3.4E1,2.27E2,2.9E1,7E0,2.3E1,1.61E2,1.4E1,1.4E1,1E1,2.46E2,2.3E1,1.5E1,4.21E2,8E0,4E0,6.3E1,7E0,5.65E2,2.15E2,2.1E1,2.6E1,4.9E1,2.1E1,1.2E1,2.2E1,2.8E1,1.99E2,1E1,1.9E1,1.2E1,1.1E1,2.2E1,1.39E2,7E0,7E0,6E0,8E0,6E0,4E0,2.15E2,3.1E1,6E0,1.7E1,6E0,9E0,3.35E2,8.6E1,4E0,5.9E1,5E0,2E0,1.79E2,3.86E2,5.8E1,1.57E2,2.2E1,4E0,4.1E1,8E0,5E0,7E0,4E0,1.8E1,5E0,2.3E1,5.2E1,1.47E2,2E0,1.7E1,5E0,7E0,1.7E1,5E0,8.3E1,5.6E1,2E1,1.95E2,4E0,2.7E1,4E0,2E0,1E1,7E0,3.33E2,2E0,2.4E1,6.2E1,3E0,5.6E1,7.5E1,1.04E2,3.01E2,8.5E1,8E0,5E1,1.19E2,3.8E1,1.9E1,3E0,2E0,2E0,3.5E1,6E0,4E0,1.4E1,1E1,1.3E1,4.1E1,1.1E1,3.2E1,1.15E2,2E0,1.5E1,2.2E1,6.1E1,3.9E1,1.7E1,1E1,1E1,1.79E2,1.6E1,3E0,2.4E1,1.9E2,1.43E2,2.2E1,2E0,1.1E1,5.1E1,4.4E1,1.2E1,6.1E1,1.4E1,2.9E1,7.5E1,7.4E1,2.27E2,3E0,8.2E1,4.8E1,2E0,1.02E2,1.7E1,2.5E1,1.3E1,1.3E1,6E0,1.8E1,1.7E1,5E0,5E0,3.5E1,6E0,1.3E1,1.9E1,4.1E1,7.4E1,9E0,5.2E1,1E1,2.9E1,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[-5.0356344E-4,-8.229352E-2,3.891362E-2,-8.811393E-2,3.008688E-1,1.2961738E-3,1.1127497E-1,-1.2909487E-1,-3.5121992E-2,1.8601544E-2,3.9251796E-3,1.21062165E-2,-1.7150047E-1,-2.1146152E-2,1.3374707E-1,-1.6795933E-1,-7.155599E-2,-6.291972E-2,6.705063E-2,-3.798077E-2,4.6507623E-2,-2.691418E-1,-7.777604E-2,9.590485E-4,-1.8732242E-2,-3.842214E-2,1.516697E-1,-1.3486518E-1,-2.5041062E-1,1.373989E-3,-1.454033E-1,-5.0604206E-2,-2.1542068E-1,1.8475208E-2,5.0279453E-2,-2.1767538E-2,-1.7835303E-1,8.4158495E-2,-2.7029404E-2,-6.375492E-3,-3.438898E-1,-8.077621E-3,-1.3502925E-2,7.616548E-2,-6.463013E-2,1.31138135E-2,-1.5837891E-2,1.4118643E-1,3.918231E-1,-7.202191E-2,-1.662063E-1,-3.6633918E-1,-1.465134E-1,1.0219929E-2,-1.9040339E-2,-1.0257885E-1,-3.2699233E-1,-7.1656495E-2,9.536483E-3,1.6312733E-4,-1.259776E-2,3.0684628E-2,7.6419436E-3,-6.8410225E-2,1.9583834E-2,-2.2140409E-1,1.5867504E-1,6.105293E-2,1.8755858E-1,-5.9901386E-2,4.185209E-2,-8.972279E-3,-1.8351935E-2,3.4808584E-3,-3.0825755E-3,9.868613E-2,-8.576841E-3,-8.059432E-2,6.9633583E-3,7.875751E-3,-1.1653857E-2,1.7852335E-1,9.80547E-2,1.1630899E-2,2.2393996E-2,-9.533803E-2,8.853565E-2,-2.23785E-1,-9.3251236E-2,-3.9344087E-1,-6.6426992E-3,-1.1504145E-1,-1.7086236E-2,1.5097186E-2,-3.2886602E-2,-1.633375E-2,-1.6407959E-1,-2.2173735E-1,-2.5399098E-2,1.787528E-2,-8.2447544E-2,-7.4258493E-3,2.4862334E-2,1.5534877E-2,1.0301413E-2,-5.5511415E-2,-3.0654874E-1,8.5593954E-2,-4.379772E-2,-2.4488778E-1,-1.4272323E-3,1.3856833E-2,-1.8837565E-3,1.244506E-1,2.2959352E-2,2.1730113E-1,-4.2635202E-3,-8.527193E-2,1.8082174E-2,8.023962E-2,-3.6771032E-3,1.6039258E-1,-1.7376714E-4,4.095944E-3,-9.834914E-2,6.458216E-2,-7.122683E-2,1.614403E-1,1.9783596E-2,-1.2796664E-2,1.0614994E-1,-9.865184E-3,-2.5869978E-3,8.262383E-3,-1.9653287E-4,-8.056315E-3,-1.8280363E-2,1.1029874E-3,-6.21266E-3,-2.108769E-2,-1.2663495E-2,-6.66789E-3,2.7139774E-3,-7.054288E-4,-5.5433773E-3,-3.5507025E-3,1.7665327E-3,-1.1328411E-2,-1.7152507E-3,-1.4510162E-2,-4.772305E-3,-7.303217E-3,4.228265E-3,-8.117309E-3,-3.1529434E-3,-1.0051802E-3,3.4514782E-3,3.367898E-3,-2.2368314E-4,-7.2403E-3,-1.7479437E-3,3.8511849E-3,-2.5286822E-2,4.437717E-3,-3.3485468E-3,-1.4602367E-2,-1.2516569E-3,-1.3067505E-2,-5.621303E-3,-5.322231E-3,6.3047647E-3,-1.7932117E-4,5.4136124E-3,1.6569655E-2,7.623609E-3,-4.3378095E-3,4.020553E-3,-1.2977666E-3,4.0117055E-3,5.529317E-3,7.9595204E-4,-4.710383E-4,8.1469575E-3,-9.38947E-3,-2.5725001E-3,-1.0584757E-4,9.004533E-3,-5.0244327E-3,4.7803125E-3,5.4267813E-3,9.096542E-3,-2.1439625E-3,5.373258E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,-1,71,-1,73,75,77,79,-1,81,83,85,87,89,91,-1,93,95,97,99,101,-1,-1,103,-1,105,107,109,111,113,115,117,119,-1,-1,-1,-1,121,-1,123,-1,-1,125,127,129,-1,-1,131,133,135,137,139,-1,141,-1,-1,143,145,147,149,-1,151,153,-1,155,157,-1,159,161,163,165,167,-1,-1,-1,169,171,173,-1,175,177,179,-1,181,-1,-1,183,185,187,189,-1,-1,191,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.137758E0,1.6322274E0,4.066597E0,1.5373445E0,2.2351909E-1,1.8401283E0,1.5234528E0,8.824687E-1,8.8751066E-1,0E0,0E0,1.5990455E0,5.0984716E-1,6.5129626E-1,1.3542895E0,6.1564827E-1,8.780661E-1,4.4924015E-1,3.6268172E-1,8.5640335E-1,1.525078E0,2.2014308E-1,1.981137E-1,3.6007902E-1,0E0,6.7861897E-1,9.416466E-1,3.26797E-1,7.533183E-1,3.8445345E-1,5.9634995E-1,2.8935403E-1,2.4872744E-1,0E0,1.4576751E-1,6.5711105E-1,6.009985E-1,8.581877E-1,4.2592573E-1,0E0,7.191944E-2,0E0,1.03909284E-1,2.2207247E-1,1.4839427E-1,1.4942521E-1,0E0,5.9605885E-1,9.4870806E-2,2.2931212E-1,4.5800686E-1,1.267848E-1,2.254386E-1,0E0,3.8679215E-1,3.526972E-1,2.4507475E-1,1.6475165E-1,1.6367231E-1,0E0,0E0,1.664527E-1,0E0,4.744557E-1,7.61021E-1,1.4834774E-1,1.6033469E-1,7.192409E-1,5.7350206E-1,2.5280955E-1,2.9470497E-1,0E0,0E0,0E0,0E0,2.0306069E-1,0E0,1.212748E-1,0E0,0E0,1.5447299E-1,8.2572985E-1,5.8166206E-1,0E0,0E0,2.3098686E-1,7.060711E-2,4.4792843E-1,2.7227524E-1,9.185457E-2,0E0,2.0062646E-1,0E0,0E0,1.1432406E-1,1.0273369E-1,4.1446018E-1,1.0407841E-1,0E0,2.7216902E-1,1.7478609E-1,0E0,1.3709211E-1,6.5441646E-2,0E0,2.6385218E-1,8.36941E-1,1.836943E-1,4.0976736E-1,1.2787509E-1,0E0,0E0,0E0,4.1772485E-1,4.8944214E-1,3.7472844E-1,0E0,1.8465579E-1,1.0804205E-1,1.2121919E-1,0E0,6.867224E-2,0E0,0E0,1.4218098E-1,1.3528335E-1,1.4526145E-1,2.910757E-1,0E0,0E0,3.442309E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,40,40,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,75,75,77,77,80,80,81,81,82,82,85,85,86,86,87,87,88,88,89,89,91,91,94,94,95,95,96,96,97,97,99,99,100,100,102,102,103,103,105,105,106,106,107,107,108,108,109,109,113,113,114,114,115,115,117,117,118,118,119,119,121,121,124,124,125,125,126,126,127,127,130,130],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,-1,72,-1,74,76,78,80,-1,82,84,86,88,90,92,-1,94,96,98,100,102,-1,-1,104,-1,106,108,110,112,114,116,118,120,-1,-1,-1,-1,122,-1,124,-1,-1,126,128,130,-1,-1,132,134,136,138,140,-1,142,-1,-1,144,146,148,150,-1,152,154,-1,156,158,-1,160,162,164,166,168,-1,-1,-1,170,172,174,-1,176,178,180,-1,182,-1,-1,184,186,188,190,-1,-1,192,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,1.3144558E4,1.0061821E5,1E0,1.8215276E7,4.0844156E7,3.0802163E-1,3.83E3,7.2041174E2,1.8601544E-2,3.9251796E-3,8.15E2,1E0,2.8524646E-1,2.960909E2,2.589369E7,6.747114E7,1E0,1E0,2.4E1,1.0938637E1,1.279012E7,2.6646748E3,2.342147E7,-1.8732242E-2,5.266129E0,1.6828056E1,8.728293E-1,1.0937254E5,4.44084E5,3.6401E4,1.294364E0,9E0,1.8475208E-2,1.6316416E6,1.654762E2,7.358598E6,7.885718E2,6.222E3,-6.375492E-3,1.363881E3,-8.077621E-3,1.7331858E1,1E0,1.7971E4,1.15E0,-1.5837891E-2,5.911314E1,8.349059E6,4.05E0,2.2741232E6,3.5612745E0,2.1525E4,1.0219929E-2,8.417E3,1.02422056E6,3.6600095E10,1.1978022E0,2.5534486E4,1.6312733E-4,-1.259776E-2,1.45E2,7.6419436E-3,6.62E2,1.6521739E0,1E0,1.3569831E7,3.1140776E0,2.4030858E7,8.5023944E2,4.915391E-2,-8.972279E-3,-1.8351935E-2,3.4808584E-3,-3.0825755E-3,1.6526531E1,-8.576841E-3,1.06158945E5,6.9633583E-3,7.875751E-3,2.15E2,5.094239E3,2.979E3,1.1630899E-2,2.2393996E-2,3.7342235E6,2.241E2,9.517604E6,1.3298E4,3E0,-6.6426992E-3,1.9015984E6,-1.7086236E-2,1.5097186E-2,1.4843444E1,1.3498145E5,3.0497742E3,7.123E3,-2.5399098E-2,7.8079817E-3,9.536863E4,-7.4258493E-3,2.779E3,1.18827E5,1.0301413E-2,8.333333E0,2.47E2,6.2233735E-6,1.198E5,3.5192856E1,-1.4272323E-3,1.3856833E-2,-1.8837565E-3,1.1058E4,9.750871E6,1.109125E3,-4.2635202E-3,7.2456576E8,1.3339816E5,2.773036E1,-3.6771032E-3,1.2886906E5,-1.7376714E-4,4.095944E-3,2.5933685E-2,1E0,4.5723195E6,8.319452E2,1.9783596E-2,-1.2796664E-2,2.9311974E8,-9.865184E-3,-2.5869978E-3,8.262383E-3,-1.9653287E-4,-8.056315E-3,-1.8280363E-2,1.1029874E-3,-6.21266E-3,-2.108769E-2,-1.2663495E-2,-6.66789E-3,2.7139774E-3,-7.054288E-4,-5.5433773E-3,-3.5507025E-3,1.7665327E-3,-1.1328411E-2,-1.7152507E-3,-1.4510162E-2,-4.772305E-3,-7.303217E-3,4.228265E-3,-8.117309E-3,-3.1529434E-3,-1.0051802E-3,3.4514782E-3,3.367898E-3,-2.2368314E-4,-7.2403E-3,-1.7479437E-3,3.8511849E-3,-2.5286822E-2,4.437717E-3,-3.3485468E-3,-1.4602367E-2,-1.2516569E-3,-1.3067505E-2,-5.621303E-3,-5.322231E-3,6.3047647E-3,-1.7932117E-4,5.4136124E-3,1.6569655E-2,7.623609E-3,-4.3378095E-3,4.020553E-3,-1.2977666E-3,4.0117055E-3,5.529317E-3,7.9595204E-4,-4.710383E-4,8.1469575E-3,-9.38947E-3,-2.5725001E-3,-1.0584757E-4,9.004533E-3,-5.0244327E-3,4.7803125E-3,5.4267813E-3,9.096542E-3,-2.1439625E-3,5.373258E-3],"split_indices":[19,51,32,109,1,44,40,2,51,0,0,2,109,37,51,44,44,90,66,8,52,7,54,44,0,53,34,57,32,1,9,41,3,0,27,4,44,51,2,0,4,0,57,64,6,53,0,57,46,53,50,52,9,0,12,31,5,52,32,0,0,8,0,2,52,89,44,53,47,51,56,0,0,0,0,57,0,32,0,0,10,4,2,0,0,44,4,44,9,8,0,27,0,0,56,32,51,2,0,57,27,0,2,28,0,51,0,36,7,57,0,0,0,9,9,32,0,7,27,57,0,46,0,0,37,107,27,51,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.212E3,7.19E2,1.493E3,7.09E2,1E1,9.83E2,5.1E2,3.99E2,3.1E2,6E0,4E0,9.26E2,5.7E1,7.4E1,4.36E2,2.37E2,1.62E2,2.44E2,6.6E1,3.77E2,5.49E2,2.7E1,3E1,7.1E1,3E0,4.1E1,3.95E2,1.71E2,6.6E1,8.2E1,8E1,2.27E2,1.7E1,2E0,6.4E1,3.39E2,3.8E1,3.63E2,1.86E2,1.1E1,1.6E1,1.1E1,1.9E1,3.3E1,3.8E1,3.6E1,5E0,3.8E2,1.5E1,5.8E1,1.13E2,3E1,3.6E1,6E0,7.6E1,6.6E1,1.4E1,1.68E2,5.9E1,4E0,1.3E1,5.6E1,8E0,1.59E2,1.8E2,3.4E1,4E0,2.98E2,6.5E1,1.26E2,6E1,6E0,1E1,7E0,1.2E1,3.1E1,2E0,3.6E1,2E0,4E0,3.2E1,2.02E2,1.78E2,8E0,7E0,5.1E1,7E0,6.2E1,5.1E1,2.6E1,4E0,3.3E1,3E0,2E0,7.4E1,2.8E1,3.8E1,1.1E1,3E0,1.8E1,1.5E2,4E0,5.5E1,5.3E1,3E0,1.52E2,7E0,8.8E1,9.2E1,3E1,4E0,2E0,2E0,1.11E2,1.87E2,5.9E1,6E0,9.5E1,3.1E1,4.6E1,1.4E1,1.9E1,1.2E1,3E0,3.3E1,1.4E1,1.8E1,1.91E2,1.1E1,3E0,1.75E2,1.1E1,4E1,3E0,4E0,5.1E1,1.1E1,1.4E1,3.7E1,1.4E1,1.2E1,2.8E1,5E0,6.3E1,1.1E1,1.3E1,1.5E1,2.2E1,1.6E1,5E0,6E0,5E0,1.3E1,1.6E1,1.34E2,2.9E1,2.6E1,1.3E1,4E1,2E1,1.32E2,3E0,4E0,8.2E1,6E0,4E0,8.8E1,2.1E1,9E0,6E0,1.05E2,1.47E2,4E1,1.3E1,4.6E1,9E1,5E0,1.9E1,1.2E1,2.7E1,1.9E1,2E0,1.7E1,8E0,2.5E1,1E1,4E0,1.5E1,3E0,9.5E1,9.6E1,1.3E1,1.62E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"193","size_leaf_vector":"1"}},{"base_weights":[-2.9457933E-3,-6.34363E-2,4.0619034E-2,-2.1412687E-2,-1.24910094E-1,7.14938E-2,-7.794418E-2,1.3531782E-1,-3.726128E-2,-1.1168488E-1,-3.8247487E-1,3.3122517E-2,1.3226159E-1,-1.3099615E-1,8.560795E-2,2.07181E-1,-9.0680785E-2,-9.1820784E-2,-5.2319467E-3,-1.6930781E-1,-8.1803374E-2,-4.7975034E-1,-7.2926227E-3,6.564693E-2,-3.7486263E-2,1.7567452E-2,1.16904065E-1,-9.80463E-2,-2.346676E-1,1.7563483E-1,-1.5534188E-2,-6.5743E-4,2.3293328E-1,-1.5573898E-2,6.012358E-2,-7.985571E-2,-1.9805714E-2,-1.12311E-2,2.629063E-1,-1.791042E-1,7.4458295E-3,-8.744771E-2,6.879817E-3,-2.6436538E-2,-9.829998E-3,9.494242E-2,6.6000773E-3,-7.601862E-2,1.0025345E-2,1.026384E-1,4.112846E-1,-1.0586511E-1,1.0122358E-2,-2.649028E-1,-3.3599946E-3,1.9778462E-2,1.278358E-1,-5.994434E-2,1.06968535E-2,1.716896E-1,1.4640216E-2,-4.4344934E-3,5.4002525E-3,-5.9688047E-2,-2.306718E-1,-1.3790102E-1,-2.546358E-3,8.8208E-4,1.6134681E-2,-1.4190559E-1,-2.609258E-1,-7.830321E-2,-2.2555399E-1,1.4608577E-1,5.2529987E-2,5.7940252E-2,-1.0403638E-1,4.99194E-3,-8.408822E-2,3.3560764E-2,-8.598082E-2,1.4068481E-1,4.068955E-2,1.0204729E-2,2.4967285E-2,-8.216799E-2,-2.175447E-1,-2.7973118E-1,5.0817424E-4,7.385483E-3,1.848414E-1,-9.056582E-4,-1.3337055E-2,4.062551E-3,1.0314805E-2,-1.4000238E-1,-2.3300499E-2,-1.9024534E-2,-8.1709504E-2,-2.6955172E-1,2.0704854E-2,1.4457779E-1,-1.2702039E-2,-3.2271922E-1,-1.10238895E-1,-3.3210754E-1,-6.895756E-2,-9.933789E-2,-2.0204514E-2,-2.0797865E-3,-1.3256514E-2,1.7704505E-1,1.5439218E-2,1.5671158E-2,1.485094E-1,9.4327435E-2,-2.4919624E-2,-2.0816604E-2,-6.8523385E-2,-9.722766E-2,3.8227685E-2,-1.3578102E-3,6.728544E-2,4.0558497E-3,-8.204949E-3,5.0200637E-2,1.7163873E-1,-2.1148993E-2,8.714048E-2,-1.2386972E-2,-7.371815E-2,-1.4664614E-1,-4.294322E-1,-6.513625E-3,-1.4087657E-2,4.8242803E-3,-2.9204763E-3,1.0725077E-2,3.198427E-3,-9.994898E-3,-4.4858484E-3,4.2395066E-3,-2.323281E-3,-6.4731077E-3,4.1600047E-3,-3.0147745E-3,-1.7023787E-2,3.6385786E-3,-6.5782405E-3,3.7894302E-4,8.72092E-3,-1.7290907E-3,2.0695329E-3,-2.8655296E-3,-1.6673269E-2,-1.618382E-2,-4.10754E-3,-6.5727234E-3,-1.8432043E-2,1.301062E-3,-8.689845E-3,-3.5551863E-3,-1.0037205E-2,9.7609294E-4,-2.96963E-3,5.0203805E-3,1.0093661E-2,9.944128E-3,-9.39246E-4,-2.2826558E-3,2.0844964E-3,3.638901E-3,9.928814E-3,5.001277E-3,-2.9158182E-3,-2.2977784E-3,7.3408238E-3,1.3301448E-3,-5.288004E-3,-4.877626E-3,7.3895365E-4,5.9115994E-3,-4.4686594E-3,7.4620387E-3,2.237787E-3,-1.8128388E-2,3.3251154E-3,1.1045205E-2,5.5319234E-3,1.769071E-3,-2.8243521E-3,4.824165E-3,-1.9270097E-3,1.2629121E-3,-3.8926792E-3,-9.069525E-3,-2.3911581E-3,-2.4922041E-2,-5.6547965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,-1,49,51,53,55,57,-1,59,-1,61,63,-1,65,67,69,-1,71,-1,-1,-1,73,75,77,79,81,83,85,-1,87,-1,-1,89,91,-1,93,-1,-1,-1,95,97,99,101,-1,-1,103,105,107,109,111,113,115,117,-1,119,121,123,125,127,-1,-1,129,131,133,-1,135,137,-1,-1,-1,-1,139,141,-1,143,145,147,149,151,153,155,157,159,161,163,-1,-1,165,167,169,171,173,175,-1,177,179,181,-1,183,-1,-1,185,187,189,191,-1,193,195,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.84758E0,2.3972585E0,4.7277613E0,1.3786712E0,1.241477E0,2.3833184E0,2.3279097E0,8.4707725E-1,8.7851423E-1,6.060195E-1,3.1038046E-1,1.4469652E0,1.5100865E0,6.670153E-1,6.0369194E-1,2.2604644E-1,5.172887E-1,7.342962E-1,5.1448834E-1,4.2836332E-1,3.2863414E-1,2.0152307E-1,0E0,7.4492E-1,3.6507815E-1,0E0,1.5326118E0,4.1345358E-1,2.132039E-1,3.8117743E-1,3.7200746E-1,0E0,1.3833356E-1,0E0,9.584099E-2,5.367627E-1,0E0,3.428787E-1,1.5233046E-1,3.2516742E-1,0E0,2.7786028E-1,0E0,0E0,0E0,6.1703086E-1,8.239616E-1,1.7462379E-1,2.0624466E-1,8.4041405E-1,3.1869507E-1,3.843906E-1,0E0,1.6692066E-1,0E0,0E0,2.1149147E-1,2.5651368E-1,0E0,8.538306E-2,0E0,0E0,0E0,4.6725106E-1,5.585778E-1,4.2319244E-1,4.3962118E-1,0E0,0E0,4.4538856E-1,4.690256E-1,2.6807427E-1,1.5566444E-1,5.230808E-1,5.6016517E-1,3.0083936E-1,5.4965526E-1,0E0,1.7407608E-1,1.5847608E-1,3.172363E-1,6.1317015E-1,3.9824212E-1,0E0,0E0,1.852625E-1,3.2761085E-1,1.1751366E-1,0E0,8.530041E-2,1.0808927E-1,0E0,0E0,0E0,0E0,1.4336646E-1,3.7173837E-1,0E0,1.5140037E-1,2.1265435E-1,1.107203E-1,1.2273288E-1,4.1120416E-1,1.1851144E-1,3.197205E-1,3.0548334E-1,1.376776E-1,3.9513385E-1,1.15149684E-1,0E0,0E0,2.941258E-1,1.966744E-1,2.3515615E-1,1.9461256E-1,1.8388236E-1,1.5803356E-1,0E0,2.0666917E-1,1.1630529E-1,1.5198313E-1,0E0,7.545696E-2,0E0,0E0,6.3689953E-1,5.603204E-1,1.5143596E-1,2.0346802E-1,0E0,1.556589E-1,9.5192045E-2,1.5240681E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,27,27,28,28,29,29,30,30,32,32,34,34,35,35,37,37,38,38,39,39,41,41,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,56,56,57,57,59,59,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,79,79,80,80,81,81,82,82,85,85,86,86,87,87,89,89,90,90,95,95,96,96,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,111,111,112,112,113,113,114,114,115,115,116,116,118,118,119,119,120,120,122,122,125,125,126,126,127,127,128,128,130,130,131,131,132,132],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,-1,50,52,54,56,58,-1,60,-1,62,64,-1,66,68,70,-1,72,-1,-1,-1,74,76,78,80,82,84,86,-1,88,-1,-1,90,92,-1,94,-1,-1,-1,96,98,100,102,-1,-1,104,106,108,110,112,114,116,118,-1,120,122,124,126,128,-1,-1,130,132,134,-1,136,138,-1,-1,-1,-1,140,142,-1,144,146,148,150,152,154,156,158,160,162,164,-1,-1,166,168,170,172,174,176,-1,178,180,182,-1,184,-1,-1,186,188,190,192,-1,194,196,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.449702E2,1.1846624E-5,3.9712732E7,7E0,4.3702424E7,1.0855529E3,3.4409692E3,3.6789783E2,1.8991614E5,2.222456E0,7.080933E2,6.788246E0,7.9E1,1.13808104E8,9.827109E9,7.3E1,1.5125709E8,4.5E1,3.1930302E1,6.494E3,1.6213043E3,4.8052647E5,-7.2926227E-3,1.9203481E5,4.832E3,1.7567452E-2,4.5505118E2,1.0242964E8,4.551684E6,3.541262E-5,9.02496E5,-6.5743E-4,5E0,-1.5573898E-2,7.66E2,1.980293E7,-1.9805714E-2,1.198E5,1.732E3,1.8544E4,7.4458295E-3,1E0,6.879817E-3,-2.6436538E-2,-9.829998E-3,8E0,1.504779E7,1.421E3,5.382E4,1.504779E7,3.8398104E0,4.5397964E-1,1.0122358E-2,7E0,-3.3599946E-3,1.9778462E-2,2.3292E4,5.39978E5,1.06968535E-2,1.3531746E0,1.4640216E-2,-4.4344934E-3,5.4002525E-3,3.3151623E1,3.7246967E2,6.239726E0,1.15E0,8.8208E-4,1.6134681E-2,4.36E2,4.95E9,3.8657358E6,2.18988E5,3.0923106E-10,7.298614E2,2.185321E0,8.05E2,4.99194E-3,1.60264E6,3.7510395E4,7.5891815E6,3.544243E8,1.479E3,1.0204729E-2,2.4967285E-2,2E1,6.9514094E11,2.2222222E-1,5.0817424E-4,5.441732E9,1.8552577E2,-9.056582E-4,-1.3337055E-2,4.062551E-3,1.0314805E-2,1.0972222E0,1.3797468E0,-1.9024534E-2,3.596E3,1.08504E5,1.819149E1,3.55E2,4.784047E6,4.5168175E5,5.750032E3,2.2837209E2,2.9005406E5,2.3362158E1,2.2E1,-2.0797865E-3,-1.3256514E-2,4.8237036E2,4.5586734E0,2.221E3,3.38464E5,8.7690773E-7,3.2343243E1,-2.0816604E-2,2.65903E5,1.9056322E3,7.1E1,-1.3578102E-3,4.923077E0,4.0558497E-3,-8.204949E-3,8.490231E-1,9.047619E-3,1.3E1,1.117E4,-1.2386972E-2,5.4803147E0,3.9401836E11,1.07259315E9,-6.513625E-3,-1.4087657E-2,4.8242803E-3,-2.9204763E-3,1.0725077E-2,3.198427E-3,-9.994898E-3,-4.4858484E-3,4.2395066E-3,-2.323281E-3,-6.4731077E-3,4.1600047E-3,-3.0147745E-3,-1.7023787E-2,3.6385786E-3,-6.5782405E-3,3.7894302E-4,8.72092E-3,-1.7290907E-3,2.0695329E-3,-2.8655296E-3,-1.6673269E-2,-1.618382E-2,-4.10754E-3,-6.5727234E-3,-1.8432043E-2,1.301062E-3,-8.689845E-3,-3.5551863E-3,-1.0037205E-2,9.7609294E-4,-2.96963E-3,5.0203805E-3,1.0093661E-2,9.944128E-3,-9.39246E-4,-2.2826558E-3,2.0844964E-3,3.638901E-3,9.928814E-3,5.001277E-3,-2.9158182E-3,-2.2977784E-3,7.3408238E-3,1.3301448E-3,-5.288004E-3,-4.877626E-3,7.3895365E-4,5.9115994E-3,-4.4686594E-3,7.4620387E-3,2.237787E-3,-1.8128388E-2,3.3251154E-3,1.1045205E-2,5.5319234E-3,1.769071E-3,-2.8243521E-3,4.824165E-3,-1.9270097E-3,1.2629121E-3,-3.8926792E-3,-9.069525E-3,-2.3911581E-3,-2.4922041E-2,-5.6547965E-3],"split_indices":[51,38,44,3,44,51,51,54,27,53,4,52,0,44,5,2,7,6,55,2,4,46,0,32,2,0,55,31,28,38,11,0,3,0,28,5,0,7,2,9,0,82,0,0,0,16,44,2,11,44,52,40,0,6,0,0,2,11,0,52,0,0,0,54,4,55,53,0,0,2,5,31,1,36,51,40,10,0,1,27,44,7,0,0,0,0,30,57,0,5,55,0,0,0,0,52,52,0,2,7,55,10,44,27,32,4,32,57,57,0,0,4,55,10,9,36,55,0,31,54,8,0,57,0,0,52,56,8,0,0,55,30,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.217E3,9.28E2,1.289E3,5.52E2,3.76E2,1.023E3,2.66E2,5E1,5.02E2,3.59E2,1.7E1,6.28E2,3.95E2,2.01E2,6.5E1,3.8E1,1.2E1,1.85E2,3.17E2,1.21E2,2.38E2,1.1E1,6E0,4.3E2,1.98E2,2.1E1,3.74E2,1.54E2,4.7E1,3.4E1,3.1E1,4E0,3.4E1,4E0,8E0,1.8E2,5E0,3.11E2,6E0,1.18E2,3E0,2.33E2,5E0,7E0,4E0,2.87E2,1.43E2,1.09E2,8.9E1,3.58E2,1.6E1,1.51E2,3E0,3.9E1,8E0,4E0,3E1,2.7E1,4E0,2.2E1,1.2E1,2E0,6E0,1.6E2,2E1,1.9E1,2.92E2,2E0,4E0,8.3E1,3.5E1,2.2E2,1.3E1,1.29E2,1.58E2,9.8E1,4.5E1,4E0,1.05E2,7.2E1,1.7E1,2.21E2,1.37E2,8E0,8E0,1.26E2,2.5E1,3.7E1,2E0,1E1,2E1,2.4E1,3E0,1E1,1.2E1,4.9E1,1.11E2,8E0,1.2E1,1E1,9E0,1.8E1,2.74E2,1.1E1,7.2E1,2.5E1,1E1,1.61E2,5.9E1,4E0,9E0,1.04E2,2.5E1,1.15E2,4.3E1,6.8E1,3E1,3E0,4.2E1,9.5E1,1E1,2.5E1,4.7E1,6E0,1.1E1,5.7E1,1.64E2,5.9E1,7.8E1,4E0,1.22E2,2E1,5E0,8E0,2.9E1,4E0,6E0,1.3E1,7E0,1.5E1,3.4E1,2.1E1,9E1,9E0,3E0,4E0,6E0,7E0,2E0,5E0,1.3E1,1.91E2,8.3E1,2E0,9E0,4E0,6.8E1,8E0,1.7E1,6E0,4E0,1.39E2,2.2E1,3.1E1,2.8E1,4.4E1,6E1,3E0,2.2E1,3.6E1,7.9E1,2.3E1,2E1,6.2E1,6E0,2.7E1,3E0,1.4E1,2.8E1,8.7E1,8E0,6E0,4E0,6E0,4.1E1,2E0,5.5E1,6.5E1,9.9E1,2.4E1,3.5E1,6.8E1,1E1,1.3E1,1.09E2,1.2E1,8E0,3E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[1.196977E-3,-7.002284E-2,3.48611E-2,-8.445867E-2,1.2812419E-1,1.6168639E-2,1.5065771E-1,-5.621116E-2,-1.7204551E-1,3.4726888E-1,3.680563E-2,-1.40984785E-2,7.7164434E-2,2.4751794E-2,1.3604255E-1,-9.945093E-2,1.2188328E-2,-4.3389177E-1,-1.5344213E-1,1.9553687E-2,4.91488E-3,-2.9170928E-3,1.7123446E-1,-3.8339756E-2,2.594703E-2,2.4683494E-2,1.2435719E-1,-1.2275177E-1,1.576894E-1,-1.15598015E-1,-1.511177E-2,7.9624906E-2,-1.1947714E-2,-2.56775E-2,-8.474989E-3,-2.1493337E-1,-9.545717E-2,2.2160418E-2,-7.195038E-3,1.1219749E-2,1.4802342E-3,-5.2506447E-2,6.6295944E-2,7.411587E-2,-4.04739E-3,3.5361826E-2,-8.324822E-3,1.4746173E-1,4.22463E-3,-1.8325452E-2,-1.9533416E-2,1.3873501E-1,1.7172132E-2,-1.6408963E-2,-1.1022035E-1,5.0241105E-2,-9.401476E-2,4.062163E-2,2.1776213E-1,1.9198349E-2,-7.2089106E-2,-1.0745402E-1,-2.849205E-1,6.385787E-2,-1.1947178E-1,-8.658813E-4,5.9334263E-3,-4.833299E-3,-8.4554106E-2,1.15565084E-1,-2.167464E-3,-9.341044E-2,9.289914E-2,-2.161345E-2,1.09306015E-1,-3.258934E-2,6.5523826E-2,1.592292E-1,-3.7416484E-2,-5.0111737E-2,1.2160137E-1,-6.0824095E-3,1.00313805E-1,2.1206416E-1,1.07798494E-1,-6.74125E-2,-1.4367934E-1,6.0771094E-3,-9.570781E-4,-7.2119515E-3,-1.0333327E-2,7.590681E-2,-1.86602E-3,3.2360302E-3,1.3181188E-2,4.2676892E-2,-1.0886567E-1,-9.82322E-3,-4.682688E-2,-1.335299E-1,8.6493604E-4,-3.2677191E-1,-1.3346444E-1,4.169743E-5,1.0475599E-2,-8.387721E-2,-1.8701409E-1,-1.9030621E-2,2.073462E-1,-1.1592292E-1,-2.83684E-2,-3.728232E-3,1.4236297E-1,-8.882188E-3,1.0282206E-3,1.0362807E-1,-4.1401617E-2,9.233576E-3,-2.9493695E-2,1.7648E-1,4.8134738E-4,4.4074194E-3,-7.42524E-2,1.3749767E-2,5.5859115E-2,3.1878263E-1,1.331605E-1,3.2338768E-3,-4.562161E-3,2.2443647E-3,-7.620681E-2,-3.0669733E-3,7.526541E-3,-1.9376238E-3,9.885428E-3,2.446402E-1,5.017392E-3,5.7995733E-2,1.5430912E-1,-1.6747325E-3,-8.940572E-3,-8.253628E-3,-2.0811583E-3,4.9792184E-3,-2.7085268E-3,6.1929934E-3,8.0997037E-4,4.067778E-3,-1.4778264E-4,2.6950252E-3,-7.946073E-3,5.2507455E-3,-2.8041152E-3,-9.223426E-3,-3.3118639E-3,-8.991683E-3,-1.7681835E-2,-8.066414E-3,2.043546E-3,-1.3929489E-3,-5.2418704E-3,-1.383423E-2,-4.0282584E-3,-4.5061726E-3,3.238731E-4,1.2809572E-2,-6.593934E-4,8.994695E-3,-5.7066293E-3,-2.9730252E-3,3.0689724E-3,3.1062441E-3,1.1259828E-2,8.242826E-3,3.40225E-3,-6.553583E-3,3.1796363E-3,-1.0725292E-3,-1.5517548E-2,1.0767471E-2,1.5661584E-3,-1.1070707E-2,-6.225624E-4,2.168357E-3,1.3345954E-2,9.7330764E-4,1.5611875E-2,7.31405E-3,3.2346032E-3,-4.9542515E-3,1.1356957E-3,8.771227E-3,1.5146144E-2,5.572054E-3,1.1601514E-3,1.0016033E-2,8.8437134E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,55,57,59,-1,-1,61,63,65,-1,-1,-1,67,69,71,73,75,-1,77,79,81,-1,83,-1,-1,85,87,89,91,93,95,97,99,101,103,105,-1,-1,107,109,111,-1,113,115,117,119,121,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,143,-1,145,-1,-1,-1,147,149,-1,151,153,-1,155,157,-1,-1,159,161,163,165,167,169,-1,171,-1,-1,173,175,-1,177,179,-1,-1,181,-1,183,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,193,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.2362986E0,2.0159738E0,3.2075531E0,1.608397E0,9.477935E-1,2.3604891E0,1.1180334E0,1.468171E0,7.0794773E-1,2.359364E-1,1.8739529E-1,8.309897E-1,1.04706E0,0E0,1.1397727E0,4.1225004E-1,3.1508818E-1,2.2259808E-1,5.1431775E-1,0E0,0E0,1.1306398E-1,7.677744E-2,7.9356915E-1,4.669693E-1,4.563713E-1,6.183374E-1,5.07872E-1,7.244482E-1,2.985437E-1,2.6255023E-1,2.6593217E-1,2.6924694E-1,0E0,0E0,5.1098084E-1,3.0987448E-1,1.1490259E-1,0E0,0E0,0E0,7.176634E-1,3.6553746E-1,3.988675E-1,4.005971E-1,3.9769715E-1,0E0,4.12786E-1,2.4207321E-1,1.9241774E-1,0E0,3.6733055E-1,0E0,0E0,3.4982324E-1,1.7070249E-1,2.6632994E-1,1.2152275E-1,9.727007E-2,2.9028794E-1,1.6918409E-1,1.0155481E-1,2.2894931E-1,1.1576511E-1,1.5019935E-1,0E0,0E0,5.762983E-1,4.9196792E-1,2.5038803E-1,0E0,1.5960598E-1,1.6567147E-1,3.1529793E-1,1.7708868E-1,3.320487E-1,2.9149264E-1,6.895962E-1,8.994401E-2,7.141109E-2,1.126965E-1,0E0,1.275881E-1,1.3778996E-1,2.7596283E-1,4.3585134E-1,5.199907E-1,0E0,0E0,9.9067286E-2,0E0,9.998612E-2,0E0,0E0,0E0,1.758281E-1,1.7878635E-1,0E0,1.1296009E-1,9.105331E-2,0E0,1.9219017E-1,9.586948E-2,0E0,0E0,7.6075554E-2,2.4013668E-1,3.7801695E-1,2.0540768E-1,5.9831977E-1,3.702739E-1,0E0,2.9665613E-1,0E0,0E0,2.1864367E-1,1.1400367E-1,0E0,2.991401E-1,1.3104758E-1,0E0,0E0,4.7052285E-1,0E0,2.3327363E-1,1.7843938E-1,2.5653362E-1,0E0,0E0,0E0,7.341137E-2,0E0,0E0,0E0,0E0,1.045146E-1,0E0,1.2535049E-1,5.6390786E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,51,51,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,67,67,68,68,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,82,82,83,83,84,84,85,85,86,86,89,89,91,91,95,95,96,96,98,98,99,99,101,101,102,102,105,105,106,106,107,107,108,108,109,109,110,110,112,112,115,115,116,116,118,118,119,119,122,122,124,124,125,125,126,126,130,130,135,135,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,56,58,60,-1,-1,62,64,66,-1,-1,-1,68,70,72,74,76,-1,78,80,82,-1,84,-1,-1,86,88,90,92,94,96,98,100,102,104,106,-1,-1,108,110,112,-1,114,116,118,120,122,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,144,-1,146,-1,-1,-1,148,150,-1,152,154,-1,156,158,-1,-1,160,162,164,166,168,170,-1,172,-1,-1,174,176,-1,178,180,-1,-1,182,-1,184,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,194,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,3.970405E3,1.2679E4,3.791269E7,1.1297775E2,7.733016E5,7.969174E7,2.437E3,3E0,1.100592E6,1.3144558E4,1.645614E6,1E0,2.4751794E-2,2.1853803E4,2.863783E2,1.497006E-1,7.016421E5,5.121E3,1.9553687E-2,4.91488E-3,8.4939235E-1,1.5137865E2,1.3264E4,1.3325451E1,4.898012E-1,1E0,7.074777E6,1.6941456E1,2.9143541E-6,2.1111E4,9.414044E2,1.775044E7,-2.56775E-2,-8.474989E-3,2.8035452E11,5.28587E5,5.2E1,-7.195038E-3,1.1219749E-2,1.4802342E-3,1E0,2.0447222E8,3.25E2,1.46705E3,2.5464671E2,-8.324822E-3,1.3226592E-2,3.097E3,3.2343243E1,-1.9533416E-2,4.52737E5,1.7172132E-2,-1.6408963E-2,4.15E2,1.7257734E6,1.4274633E7,2.0754387E1,4.3906153E5,2.3567346E7,7.5569354E2,9.18E2,1.7243155E12,1.6992E4,2.93446E5,-8.658813E-4,5.9334263E-3,1.1598511E3,8.2E2,4.3531516E2,-2.167464E-3,4E0,3.1066042E7,1E0,7.418546E1,1.0620689E0,2.1288E2,1.9041E4,9.8376294E5,1E0,1.2E1,-6.0824095E-3,1.8008E4,2.8721826E7,5.57E4,8.592097E-1,1.8122449E2,6.0771094E-3,-9.570781E-4,2.208825E-2,-1.0333327E-2,3.8434E4,-1.86602E-3,3.2360302E-3,1.3181188E-2,2.322963E0,4.624343E6,-9.82322E-3,6.425392E-2,4.6547272E2,8.6493604E-4,1.02E3,3.7891315E6,4.169743E-5,1.0475599E-2,3.512733E-1,1.2015E4,3.1E2,1.92E2,4.580013E-3,2.6757258E-1,-3.728232E-3,2.5682794E5,-8.882188E-3,1.0282206E-3,3.076635E3,7.21E3,9.233576E-3,1E0,2.0575744E8,4.8134738E-4,4.4074194E-3,1.632095E2,1.3749767E-2,6.6315195E6,1.459995E2,4.6982746E8,3.2338768E-3,-4.562161E-3,2.2443647E-3,1.3261502E0,-3.0669733E-3,7.526541E-3,-1.9376238E-3,9.885428E-3,2.0718E4,5.017392E-3,1.07259315E9,3.3517068E7,-1.6747325E-3,-8.940572E-3,-8.253628E-3,-2.0811583E-3,4.9792184E-3,-2.7085268E-3,6.1929934E-3,8.0997037E-4,4.067778E-3,-1.4778264E-4,2.6950252E-3,-7.946073E-3,5.2507455E-3,-2.8041152E-3,-9.223426E-3,-3.3118639E-3,-8.991683E-3,-1.7681835E-2,-8.066414E-3,2.043546E-3,-1.3929489E-3,-5.2418704E-3,-1.383423E-2,-4.0282584E-3,-4.5061726E-3,3.238731E-4,1.2809572E-2,-6.593934E-4,8.994695E-3,-5.7066293E-3,-2.9730252E-3,3.0689724E-3,3.1062441E-3,1.1259828E-2,8.242826E-3,3.40225E-3,-6.553583E-3,3.1796363E-3,-1.0725292E-3,-1.5517548E-2,1.0767471E-2,1.5661584E-3,-1.1070707E-2,-6.225624E-4,2.168357E-3,1.3345954E-2,9.7330764E-4,1.5611875E-2,7.31405E-3,3.2346032E-3,-4.9542515E-3,1.1356957E-3,8.771227E-3,1.5146144E-2,5.572054E-3,1.1601514E-3,1.0016033E-2,8.8437134E-4],"split_indices":[19,51,2,44,57,27,7,2,3,28,51,9,107,0,32,51,56,31,2,0,0,33,55,10,57,26,6,46,34,41,9,51,44,0,0,30,28,8,0,0,0,8,7,2,51,4,0,56,2,55,0,28,0,0,0,31,44,57,27,31,4,0,30,2,11,0,0,51,10,4,0,8,44,84,55,53,51,9,27,66,3,0,2,49,2,26,51,0,0,37,0,9,0,0,0,52,44,0,41,51,0,0,27,0,0,26,9,1,0,57,37,0,27,0,0,49,2,0,6,31,0,0,4,0,27,32,7,0,0,0,37,0,0,0,0,2,0,7,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.182E3,7E2,1.482E3,6.53E2,4.7E1,1.277E3,2.05E2,4.95E2,1.58E2,1.3E1,3.4E1,8.54E2,4.23E2,6E0,1.99E2,3.03E2,1.92E2,9E0,1.49E2,9E0,4E0,2.7E1,7E0,5.32E2,3.22E2,2.01E2,2.22E2,1.5E1,1.84E2,2.54E2,4.9E1,5E1,1.42E2,5E0,4E0,7.1E1,7.8E1,2.4E1,3E0,4E0,3E0,4.69E2,6.3E1,1.23E2,1.99E2,1.92E2,9E0,1.86E2,3.6E1,1.2E1,3E0,1.71E2,1.3E1,4E0,2.5E2,2.7E1,2.2E1,4E1,1E1,9.4E1,4.8E1,2.9E1,4.2E1,1E1,6.8E1,1.8E1,6E0,1.89E2,2.8E2,4.4E1,1.9E1,1.2E1,1.11E2,1.73E2,2.6E1,5.9E1,1.33E2,1.75E2,1.1E1,2.5E1,1.1E1,6E0,6E0,4.9E1,1.22E2,1.11E2,1.39E2,1.2E1,1.5E1,1.4E1,8E0,2.8E1,1.2E1,4E0,6E0,8E1,1.4E1,6E0,4.2E1,2.4E1,5E0,3.2E1,1E1,8E0,2E0,4.6E1,2.2E1,1.78E2,1.1E1,1.79E2,1.01E2,5E0,3.9E1,6E0,6E0,1.03E2,8E0,5E0,1.68E2,1.5E1,1.1E1,1.4E1,4.5E1,4E0,1.29E2,2.3E1,1.52E2,4E0,7E0,5E0,2E1,2E0,9E0,3E0,3E0,3.6E1,1.3E1,6E1,6.2E1,9.1E1,2E1,9.9E1,4E1,4E0,1E1,1.3E1,1.5E1,3.9E1,4.1E1,4E0,1E1,3E0,3.9E1,1E1,1.4E1,1.2E1,2E1,8E0,2E0,1.8E1,2.8E1,9E0,1.3E1,4.3E1,1.35E2,8E0,3E0,5E0,1.74E2,7.3E1,2.8E1,2.4E1,1.5E1,2.6E1,7.7E1,4E0,4E0,1.66E2,2E0,1E1,5E0,1.1E1,3.4E1,1.26E2,3E0,2E0,2.1E1,1.03E2,4.9E1,1.5E1,5E0,2.5E1,1.1E1,1.9E1,4.1E1,4.1E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-6.515465E-4,-2.8052762E-2,7.989848E-2,-3.9556115E-3,-1.2021458E-1,1.2323665E-1,-2.7208932E-2,-4.397477E-2,3.963275E-2,-2.0012726E-1,-7.170784E-2,1.0256535E-1,2.8449038E-1,-1.8036357E-1,8.693342E-3,-6.8235494E-2,7.27053E-3,2.6369832E-2,1.48664E-1,-2.4134666E-2,-1.7955165E-1,-4.311384E-2,-1.7582244E-1,7.6184295E-2,2.1157345E-1,4.7976893E-1,1.2333417E-1,-2.5821784E-1,-6.738208E-2,1.10235125E-1,-4.10459E-2,-1.0111526E-1,-4.1463293E-2,-1.2377964E-1,2.3118678E-2,3.3217238E-3,9.78452E-2,1.6749957E-1,2.9769068E-4,-5.019195E-2,-2.0880246E-1,-1.4047423E-1,-1.402165E-2,-1.2596832E-2,-1.356858E-1,1.4608516E-1,6.235086E-2,2.2636896E-1,-3.8684697E-3,5.1958347E-3,5.139492E-1,1.98188E-1,-1.5920995E-1,-1.5870078E-2,-4.482362E-3,-1.9198829E-7,-8.541443E-3,3.2193474E-2,2.2117342E-1,-2.1904157E-2,-1.3825741E-2,-1.23942696E-1,-2.4513124E-2,1.4925513E-1,-5.8992162E-2,1.7190806E-3,-6.347867E-3,-3.488131E-2,4.0405482E-2,1.9651774E-2,-8.935218E-2,1.28097E-1,-8.5532695E-2,-2.071183E-3,1.795257E-1,-1.2935907E-2,-2.938389E-3,-2.575647E-1,-1.3817428E-1,-9.2974186E-2,-2.6892695E-1,-3.418959E-2,7.815865E-2,-5.839534E-4,-1.6064537E-1,1.8402319E-1,-3.454464E-2,3.5344433E-2,9.871548E-2,1.5063018E-1,2.7183342E-1,2.720959E-2,1.4940961E-2,4.9862275E-3,1.3734097E-2,3.581769E-5,-1.0838958E-2,-2.6994105E-4,7.54865E-3,2.6476784E-3,2.7924877E-1,1.1028775E-3,-1.2934384E-1,-6.977648E-2,-1.6973947E-1,5.5314314E-2,-9.160638E-2,-3.7094986E-3,1.5788883E-2,-2.2786409E-2,-5.3069107E-2,1.9308308E-2,-1.1290383E-1,1.9368297E-1,2.5986949E-2,2.3948496E-2,-1.6840374E-2,-1.895112E-2,-6.629818E-2,1.3988373E-1,-5.217184E-3,-3.161175E-1,1.2403404E-2,1.3564344E-1,2.2662982E-1,5.8039087E-3,-5.1929887E-2,-1.9636887E-1,-4.0593064E-1,-8.97492E-2,-1.9098245E-2,-5.567827E-3,2.1527365E-3,-3.8891712E-1,-3.2213938E-3,-2.0886198E-2,-1.0189139E-2,6.5054637E-3,-5.5920734E-4,-4.7791284E-3,-9.974866E-3,2.3420525E-1,2.971931E-3,-8.637092E-3,3.4121028E-3,1.8504137E-2,6.170976E-3,1.3035505E-1,-5.214172E-3,1.0928024E-2,4.514611E-3,-8.201E-4,2.875705E-1,1.5108494E-2,3.3175214E-3,-2.6292508E-2,6.331499E-2,-8.975087E-3,3.18711E-4,-4.004622E-3,2.124651E-3,-1.3877447E-2,-4.615631E-3,3.8259982E-3,-3.0190253E-3,-3.2223968E-4,-6.378512E-3,1.620857E-3,-3.471318E-3,2.7583928E-3,-4.1812095E-3,-3.10304E-3,-1.4789589E-2,1.1439229E-2,2.2082487E-3,6.451199E-3,4.973953E-4,1.2168776E-3,-1.5104907E-2,1.7316192E-3,-5.9119803E-3,3.5707883E-3,8.890946E-3,-1.8327579E-2,-4.187001E-3,-2.448638E-3,7.0716566E-3,4.233175E-3,1.2350443E-2,1.1219516E-2,2.1805586E-3,-4.1630734E-3,4.125705E-3,-1.3539398E-2,-4.645791E-3,-1.9794906E-2,-5.64022E-3,1.6637825E-4,-8.772188E-3,-2.1689204E-2,-6.2966887E-3,6.0113873E-3,-1.59473E-3,1.3970402E-2,3.4824456E-3,-2.8096118E-3,1.496046E-3,9.842644E-3,4.3050884E-3,5.2180234E-3,-1.8604897E-3,1.5154412E-2,9.136732E-3,-4.0841821E-4,-8.146209E-3,5.867572E-4,6.0943146E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,-1,83,85,87,89,-1,-1,91,93,95,-1,-1,-1,-1,97,99,101,-1,103,105,107,109,-1,-1,111,113,115,117,119,121,-1,123,-1,125,127,129,131,133,135,137,-1,139,141,143,145,147,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,153,155,157,159,161,163,165,-1,-1,-1,167,169,171,173,175,177,-1,-1,179,181,-1,183,185,187,189,-1,191,193,195,197,-1,-1,-1,199,-1,201,-1,-1,-1,-1,-1,203,-1,-1,-1,205,-1,207,209,-1,-1,-1,211,-1,-1,213,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8955336E0,3.6750107E0,2.6167092E0,2.2920997E0,1.3152213E0,1.3074837E0,8.978397E-1,8.5212123E-1,9.056593E-1,7.8802776E-1,6.32486E-1,1.0105236E0,1.3569942E0,2.5527954E-1,6.7625594E-1,4.051299E-1,4.618571E-1,9.25498E-1,1.8088841E-1,0E0,4.5493627E-1,4.7954294E-1,1.5133154E-1,2.7034557E-1,3.2256603E-1,1.9053602E-1,5.814408E-1,2.2985852E-1,1.14400685E-1,3.7382323E-1,4.4869363E-1,3.6067677E-1,8.7131906E-1,6.9107205E-2,1.9993097E-1,6.464514E-1,7.709656E-1,1.6208553E-1,0E0,2.6627523E-1,3.0880976E-1,2.1295565E-1,2.4807164E-1,0E0,1.0519326E-1,3.2843566E-1,2.3515594E-1,1.8631983E-1,0E0,0E0,8.987236E-2,1.6105741E-1,7.718031E-2,0E0,0E0,0E0,0E0,1.395976E-1,1.537404E-1,2.1025148E-1,0E0,3.850584E-1,2.675054E-1,1.0431933E0,5.9699446E-1,0E0,0E0,1.9666848E-1,3.3310235E-1,6.197183E-1,4.5802116E-1,3.5455525E-1,4.555725E-1,0E0,8.708012E-2,0E0,1.4157036E-1,4.4689178E-1,5.528059E-1,1.3234651E-1,2.0963615E-1,2.7206796E-1,1.441275E-1,0E0,7.207501E-2,2.163074E-1,1.6891882E-1,2.3720217E-1,3.3937347E-1,1.0067266E-1,1.8772626E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.213814E-1,1.22671776E-1,1.4233899E-1,1.6762567E-1,7.661886E-1,8.75505E-2,1.114161E-1,0E0,0E0,0E0,4.957136E-1,1.3574807E-1,1.5779893E-1,9.879008E-2,2.4485372E-1,3.8609374E-1,0E0,0E0,4.1495362E-1,3.7403083E-1,0E0,8.1578076E-2,1.518569E-1,1.5944695E-1,8.636105E-2,0E0,9.998903E-2,3.8328218E-1,8.640599E-2,3.6995527E-1,0E0,0E0,0E0,6.850684E-2,0E0,2.3160754E-1,0E0,0E0,0E0,0E0,0E0,2.9209125E-1,0E0,0E0,0E0,1.449599E-1,0E0,2.2183645E-1,1.1279623E-1,0E0,0E0,0E0,8.612704E-2,0E0,0E0,1.3105534E-1,7.723621E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,50,50,51,51,52,52,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,74,74,76,76,77,77,78,78,79,79,80,80,81,81,82,82,84,84,85,85,86,86,87,87,88,88,89,89,90,90,100,100,101,101,102,102,103,103,104,104,105,105,106,106,110,110,111,111,112,112,113,113,114,114,115,115,118,118,119,119,121,121,122,122,123,123,124,124,126,126,127,127,128,128,129,129,133,133,135,135,141,141,145,145,147,147,148,148,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,-1,84,86,88,90,-1,-1,92,94,96,-1,-1,-1,-1,98,100,102,-1,104,106,108,110,-1,-1,112,114,116,118,120,122,-1,124,-1,126,128,130,132,134,136,138,-1,140,142,144,146,148,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,154,156,158,160,162,164,166,-1,-1,-1,168,170,172,174,176,178,-1,-1,180,182,-1,184,186,188,190,-1,192,194,196,198,-1,-1,-1,200,-1,202,-1,-1,-1,-1,-1,204,-1,-1,-1,206,-1,208,210,-1,-1,-1,212,-1,-1,214,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.726E3,2.0304577E-1,4.0844156E7,3.2627738E5,2.0827537E0,7.7101436E3,2.5652175E0,1.5107028E-1,4.5723195E6,3E0,4.0844156E7,1.546644E6,3.2475834E7,2E1,1.038E3,7.5646E4,2.2505386E5,2.539E3,2.9E1,-2.4134666E-2,1.3878379E0,1.07279E5,3.131E3,5.473125E3,1.9213884E2,5.893994E4,1.3205668E7,6.5029144E5,2.778E3,1.8008E4,1.5265896E8,1.224625E6,2.04E5,1.3970588E0,1.5288235E2,2.85898E10,1.6218617E7,1.136E3,2.9769068E-4,1.1216729E2,2.863783E2,2.0473914E2,9.026015E2,-1.2596832E-2,3.4736843E0,2.1871264E0,2.3761498E9,1.4521204E0,-3.8684697E-3,5.1958347E-3,2.0399538E5,2.6284674E2,2.909019E3,-1.5870078E-2,-4.482362E-3,-1.9198829E-7,-8.541443E-3,3.970405E3,4.349136E6,6.5029144E5,-1.3825741E-2,2.650027E2,1.6256282E7,4.0701206E1,1.4E1,1.7190806E-3,-6.347867E-3,5.3296334E-1,6.485856E7,2.507E3,1.3058E4,1.958583E3,3.564E3,-2.071183E-3,2.650027E2,-1.2935907E-2,5.1270317E-2,1.0508E5,1.2427474E8,1.06E3,2.960909E2,1.1979123E7,8.228668E-1,-5.839534E-4,7.218466E7,1.79E2,2.3737179E1,5.37456E5,1.9699374E10,4E0,7.123E3,2.720959E-2,1.4940961E-2,4.9862275E-3,1.3734097E-2,3.581769E-5,-1.0838958E-2,-2.6994105E-4,7.54865E-3,2.6476784E-3,1.6293218E8,4.0434834E5,6.520433E6,2.0115286E6,3.3654268E0,9.5E1,8.751669E4,-3.7094986E-3,1.5788883E-2,-2.2786409E-2,1.1558939E4,4.5E2,2.1E1,1.1494032E7,9E0,5.8E1,-1.6840374E-2,-1.895112E-2,1.3E1,6.9664386E2,-5.217184E-3,4.44E2,1.3887E4,2.6970406E0,1E0,5.8039087E-3,5.387818E8,7E2,5.3766327E2,1.02422056E6,-1.9098245E-2,-5.567827E-3,2.1527365E-3,1.71E2,-3.2213938E-3,3.1415796E0,-1.0189139E-2,6.5054637E-3,-5.5920734E-4,-4.7791284E-3,-9.974866E-3,4.396066E1,2.971931E-3,-8.637092E-3,3.4121028E-3,2.3737179E1,6.170976E-3,1.8648378E7,4.942935E6,1.0928024E-2,4.514611E-3,-8.201E-4,1.505516E7,1.5108494E-2,3.3175214E-3,5.96E2,7.9684106E9,-8.975087E-3,3.18711E-4,-4.004622E-3,2.124651E-3,-1.3877447E-2,-4.615631E-3,3.8259982E-3,-3.0190253E-3,-3.2223968E-4,-6.378512E-3,1.620857E-3,-3.471318E-3,2.7583928E-3,-4.1812095E-3,-3.10304E-3,-1.4789589E-2,1.1439229E-2,2.2082487E-3,6.451199E-3,4.973953E-4,1.2168776E-3,-1.5104907E-2,1.7316192E-3,-5.9119803E-3,3.5707883E-3,8.890946E-3,-1.8327579E-2,-4.187001E-3,-2.448638E-3,7.0716566E-3,4.233175E-3,1.2350443E-2,1.1219516E-2,2.1805586E-3,-4.1630734E-3,4.125705E-3,-1.3539398E-2,-4.645791E-3,-1.9794906E-2,-5.64022E-3,1.6637825E-4,-8.772188E-3,-2.1689204E-2,-6.2966887E-3,6.0113873E-3,-1.59473E-3,1.3970402E-2,3.4824456E-3,-2.8096118E-3,1.496046E-3,9.842644E-3,4.3050884E-3,5.2180234E-3,-1.8604897E-3,1.5154412E-2,9.136732E-3,-4.0841821E-4,-8.146209E-3,5.867572E-4,6.0943146E-3],"split_indices":[2,26,44,27,52,51,53,37,27,3,44,27,49,8,0,9,44,2,8,0,52,1,2,31,57,27,9,27,0,2,44,1,5,52,51,30,49,28,0,4,51,51,51,0,53,41,12,40,0,0,32,57,31,0,0,0,0,51,31,27,0,4,44,51,0,0,0,37,7,2,28,4,2,0,4,0,37,28,44,2,51,49,26,0,44,3,55,27,5,3,2,0,0,0,0,0,0,0,0,0,49,27,1,44,57,8,27,0,0,0,50,10,3,12,3,3,0,0,3,51,0,0,28,52,108,0,7,2,4,31,0,0,0,0,0,53,0,0,0,0,0,57,0,0,0,55,0,49,49,0,0,0,49,0,0,8,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.216E3,1.654E3,5.62E2,1.312E3,3.42E2,4E2,1.62E2,6.84E2,6.28E2,1.28E2,2.14E2,3.56E2,4.4E1,3E1,1.32E2,4.64E2,2.2E2,5.61E2,6.7E1,6E0,1.22E2,1.69E2,4.5E1,2.88E2,6.8E1,1.9E1,2.5E1,1.7E1,1.3E1,4.3E1,8.9E1,2.07E2,2.57E2,2.3E1,1.97E2,4.25E2,1.36E2,5.9E1,8E0,2.3E1,9.9E1,3.8E1,1.31E2,1.1E1,3.4E1,4.6E1,2.42E2,6.5E1,3E0,2E0,1.7E1,2E1,5E0,1E1,7E0,9E0,4E0,2.6E1,1.7E1,8.4E1,5E0,1.59E2,4.8E1,2.1E1,2.36E2,2E0,2.1E1,4.5E1,1.52E2,3.62E2,6.3E1,1.17E2,1.9E1,3E0,5.6E1,3E0,2E1,5.7E1,4.2E1,2.9E1,9E0,1.08E2,2.3E1,6E0,2.8E1,3.8E1,8E0,1.4E2,1.02E2,2.6E1,3.9E1,1E1,7E0,1.2E1,8E0,2E0,3E0,2.1E1,5E0,5E0,1.2E1,7E1,1.4E1,7.4E1,8.5E1,2.2E1,2.6E1,1E1,1.1E1,2E0,2.34E2,2.7E1,1.8E1,1.2E1,1.4E2,3.59E2,3E0,3E0,6E1,1.12E2,5E0,5E0,1.4E1,3.1E1,2.5E1,5E0,1.5E1,4.2E1,1.5E1,3.7E1,5E0,2.4E1,5E0,5E0,4E0,1.02E2,6E0,1.3E1,1E1,1.6E1,1.2E1,2.6E1,1.2E1,3E0,5E0,1.21E2,1.9E1,7.8E1,2.4E1,8E0,1.8E1,2E0,3.7E1,9E0,3E0,4.9E1,2.1E1,9E0,5E0,6.4E1,1E1,2.7E1,5.8E1,1.8E1,4E0,1E1,1.6E1,4.9E1,1.85E2,2E1,7E0,1.6E1,2E0,8E0,4E0,1.5E1,1.25E2,3.57E2,2E0,2.3E1,3.7E1,5.5E1,5.7E1,3E0,2E0,1E1,4E0,2.5E1,6E0,2.2E1,3E0,1.2E1,3E0,1.9E1,2.3E1,1.3E1,2E0,2E1,1.7E1,3E0,2E0,8E0,9.4E1,1.7E1,9E0,1.8E1,1.03E2,2.1E1,5.7E1,5E0,1.9E1,2.2E1,1.5E1,4.5E1,4E0,1.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-2.2123435E-3,-6.282932E-2,3.527434E-2,-1.5756033E-2,-1.2013223E-1,1.4247902E-2,1.4617184E-1,-8.166596E-2,7.843369E-2,-9.608571E-2,-2.4385706E-1,4.7191735E-3,1.3647707E-1,-1.08840875E-1,1.588336E-1,-5.588132E-2,-2.920968E-1,5.0812E-2,4.046651E-1,-1.2106296E-1,-1.1547081E-2,-2.0721322E-1,-2.8076824E-2,6.291072E-2,-1.4588156E-2,2.0233822E-1,8.032386E-2,-5.897126E-4,-1.6298935E-2,3.153363E-1,1.3302365E-1,1.6202217E-1,-6.57177E-2,-7.890462E-2,-4.7370842E-1,1.1265273E-2,3.3907875E-2,2.2496276E-2,5.4705576E-3,-4.9500246E-2,-1.475287E-1,3.5069056E-2,-1.0074937E-1,-4.1006088E-1,-1.3950032E-1,1.1208065E-2,1.070481E-1,-5.192594E-2,1.2202745E-2,1.2202922E-1,2.4940805E-1,4.3317273E-2,9.6395705E-3,1.5852531E-2,5.593051E-3,1.6388454E-2,1.2179674E-1,6.3237E-4,1.2815834E-2,-7.5872585E-2,4.088683E-3,1.2925625E-3,-1.7273901E-2,-2.6600534E-2,-4.7879987E-3,-6.2939905E-2,5.644626E-2,-1.4205119E-1,8.305456E-2,-2.1896443E-1,-1.1404166E-1,6.487193E-2,-7.649312E-2,-1.9480434E-1,-1.8788775E-2,-3.2722442E-3,-2.243837E-2,-1.5594178E-2,-1.2195799E-1,6.270985E-2,-5.547985E-2,1.1315942E-1,-9.957573E-3,-6.446079E-2,8.429009E-2,2.587883E-2,-8.420026E-2,1.5390132E-1,-5.461125E-4,9.1078235E-3,1.733203E-2,2.6361416E-3,-4.43537E-3,5.8333397E-2,1.6559312E-1,2.997649E-3,-8.4940374E-2,6.79159E-3,-4.4693984E-3,-2.440564E-2,7.955912E-2,-1.9451533E-1,-1.4088802E-2,5.518526E-4,8.25939E-3,-9.7755894E-2,-2.9808328E-1,-8.724863E-2,-2.4385214E-1,9.972714E-3,3.6377933E-2,-7.6935315E-3,5.8634114E-3,-1.4005547E-2,-4.4758758E-3,1.2425223E-3,-9.637473E-3,-8.3442626E-4,-1.4157635E-1,8.272012E-2,-7.599445E-3,3.2159504E-2,-1.1055591E-1,9.7831845E-2,2.4740402E-1,-1.5588826E-1,-5.2675873E-2,1.916753E-1,-2.8714474E-2,-1.0803746E-3,7.716543E-2,-1.1149253E-1,1.4481087E-3,-7.5561577E-4,8.440656E-3,8.4692715E-3,2.1814872E-2,2.7674356E-1,1.1829438E-1,-2.7359421E-3,-6.592591E-3,4.3677003E-3,-4.333277E-3,4.9485085E-3,-2.0840212E-3,-1.3080354E-2,-5.4679774E-3,-2.2884982E-3,5.6271185E-3,-2.483461E-3,-9.8388875E-3,-1.5425081E-2,-1.9194084E-3,-1.809661E-3,-5.5609248E-3,-5.0387247E-3,-1.3104805E-2,-1.1721448E-3,3.5822007E-3,-9.344969E-3,-3.2916046E-3,2.2212523E-3,8.777951E-3,8.390253E-3,-5.5108534E-4,-3.4596329E-3,-1.5732916E-2,5.8823335E-3,2.207139E-3,1.3677011E-2,-2.4369126E-3,-1.4567687E-2,-5.320419E-3,-2.2074739E-3,-1.3938531E-2,-1.527429E-4,1.1902926E-2,1.5511665E-3,-5.8353213E-3,2.9462085E-3,-9.801758E-4,1.3620511E-3,6.8792715E-3,-3.6314914E-3,-1.0066176E-2,-6.0936424E-4,4.8751016E-3,3.721168E-3,1.4576253E-2,3.4958655E-3,1.1878896E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,-1,-1,53,55,57,59,61,63,-1,65,-1,-1,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,-1,-1,-1,93,-1,-1,95,-1,-1,-1,-1,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,-1,117,119,121,123,-1,125,127,129,131,133,-1,-1,-1,-1,-1,135,137,-1,139,-1,-1,141,143,145,147,-1,-1,149,151,153,155,-1,157,-1,-1,-1,-1,-1,-1,-1,159,161,-1,163,165,167,169,171,173,175,177,179,181,183,-1,-1,-1,-1,185,187,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.078601E0,2.301718E0,3.2181048E0,2.9241834E0,1.1220016E0,1.352724E0,7.276468E-1,1.486647E0,1.7275847E0,6.827419E-1,7.264118E-1,1.2131995E0,2.9440606E-1,2.600322E-1,8.063121E-1,5.422061E-1,1.1241677E0,5.9406424E-1,3.326156E-1,4.6618724E-1,3.1540316E-1,7.394495E-1,0E0,6.126635E-1,8.118874E-1,1.1320984E-1,2.2139663E-1,0E0,0E0,1.5036511E-1,4.2374563E-1,1.9318956E-1,3.839494E-1,4.9868846E-1,6.080532E-1,0E0,3.6807913E-1,0E0,0E0,8.582951E-1,4.1040373E-1,1.7120591E-1,1.9676873E-1,3.687544E-1,1.1977351E-1,4.3279752E-1,3.14646E-1,5.842582E-1,6.2578964E-1,7.023692E-2,7.10057E-2,8.574896E-2,0E0,0E0,0E0,0E0,4.7674656E-1,0E0,0E0,2.9422247E-1,0E0,0E0,0E0,0E0,0E0,2.58623E-1,2.5608736E-1,2.7251035E-1,2.0200524E-1,5.174837E-1,4.1695416E-1,1.6910888E-1,2.3374519E-1,1.012955E-1,1.4043748E-1,0E0,0E0,0E0,8.472532E-2,3.4145465E-1,2.6903108E-1,2.7003896E-1,0E0,3.2688022E-1,3.574194E-1,5.738815E-1,1.9065344E-1,7.5261384E-2,0E0,0E0,0E0,0E0,0E0,3.3910453E-1,5.0876236E-1,0E0,3.0498302E-1,0E0,0E0,2.7643624E-1,4.009788E-1,1.669774E-1,7.186978E-2,0E0,0E0,1.11186504E-1,3.5608673E-1,1.7339426E-1,9.473276E-2,0E0,9.520574E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4574015E-1,2.3743021E-1,0E0,1.5163478E-1,2.4848905E-1,1.9845402E-1,2.4690235E-1,1.8496704E-1,2.6192868E-1,2.0929784E-1,9.904539E-2,3.7290362E-1,4.9890387E-1,1.4860094E-1,0E0,0E0,0E0,0E0,1.7712204E-1,2.374258E-1,4.1772795E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,56,56,59,59,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,78,78,79,79,80,80,81,81,83,83,84,84,85,85,86,86,87,87,93,93,94,94,96,96,99,99,100,100,101,101,102,102,105,105,106,106,107,107,108,108,110,110,118,118,119,119,121,121,122,122,123,123,124,124,125,125,126,126,127,127,128,128,129,129,130,130,131,131,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,-1,-1,54,56,58,60,62,64,-1,66,-1,-1,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,-1,-1,-1,94,-1,-1,96,-1,-1,-1,-1,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,-1,118,120,122,124,-1,126,128,130,132,134,-1,-1,-1,-1,-1,136,138,-1,140,-1,-1,142,144,146,148,-1,-1,150,152,154,156,-1,158,-1,-1,-1,-1,-1,-1,-1,160,162,-1,164,166,168,170,172,174,176,178,180,182,184,-1,-1,-1,-1,186,188,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,8.2608955E-3,3.8562928E6,2.857E3,4.7022E4,1.3137E4,1.55E2,3.7832818E0,5.528107E3,1.0052E4,3.0272608E10,2E0,5.6045805E6,7.8E1,1.4523809E0,6E0,1E0,4.561156E7,4.084E4,5.5700205E8,1.8600049E10,1.7190635E0,-2.8076824E-2,1.6633E4,3.3030225E2,2.0718E4,4.317904E0,-5.897126E-4,-1.6298935E-2,2.6998322E5,1.0918E4,1.3984146E2,1.2739522E0,2.507E3,9.76E2,1.1265273E-2,5.2350176E4,2.2496276E-2,5.4705576E-3,1.074E3,3.0291306E2,2.3828971E-1,2.7E1,2.2837209E2,2.5672606E8,1E0,2.7183437E1,1.7022566E7,3.5301748E7,1.14E2,4.7013435E1,5.96E2,9.6395705E-3,1.5852531E-2,5.593051E-3,1.6388454E-2,7.0988875E6,6.3237E-4,1.2815834E-2,3.0039525E-1,4.088683E-3,1.2925625E-3,-1.7273901E-2,-2.6600534E-2,-4.7879987E-3,2.66E2,4.179E3,1.9121015E0,4.4596E2,1.3333334E0,1.0828989E8,2.322963E0,7.8599895E9,5.684E3,1.1458888E4,-3.2722442E-3,-2.243837E-2,-1.5594178E-2,1.44E0,1E0,8.836364E0,2.831692E10,-9.957573E-3,5.015233E4,3.37E2,2.585911E5,1.998175E3,3.178082E6,-5.461125E-4,9.1078235E-3,1.733203E-2,2.6361416E-3,-4.43537E-3,5.388794E6,1.7398448E7,2.997649E-3,1.0258E4,6.79159E-3,-4.4693984E-3,1.405E3,2.9283184E3,8.857896E4,1.1880085E2,5.518526E-4,8.25939E-3,5.198287E5,4.020408E0,4.52737E5,2.2222222E-1,9.972714E-3,4.9807886E3,-7.6935315E-3,5.8634114E-3,-1.4005547E-2,-4.4758758E-3,1.2425223E-3,-9.637473E-3,-8.3442626E-4,5.012E3,3.2069644E2,-7.599445E-3,2.8316305E0,8.31E2,4.2782607E0,8.629299E4,3.5229592E0,1.6259277E7,4.6501E4,5.954224E0,1.0911331E4,7.298614E2,2.8277853E11,1.4481087E-3,-7.5561577E-4,8.440656E-3,8.4692715E-3,2.7809634E5,5.7835355E6,1.22159E5,-2.7359421E-3,-6.592591E-3,4.3677003E-3,-4.333277E-3,4.9485085E-3,-2.0840212E-3,-1.3080354E-2,-5.4679774E-3,-2.2884982E-3,5.6271185E-3,-2.483461E-3,-9.8388875E-3,-1.5425081E-2,-1.9194084E-3,-1.809661E-3,-5.5609248E-3,-5.0387247E-3,-1.3104805E-2,-1.1721448E-3,3.5822007E-3,-9.344969E-3,-3.2916046E-3,2.2212523E-3,8.777951E-3,8.390253E-3,-5.5108534E-4,-3.4596329E-3,-1.5732916E-2,5.8823335E-3,2.207139E-3,1.3677011E-2,-2.4369126E-3,-1.4567687E-2,-5.320419E-3,-2.2074739E-3,-1.3938531E-2,-1.527429E-4,1.1902926E-2,1.5511665E-3,-5.8353213E-3,2.9462085E-3,-9.801758E-4,1.3620511E-3,6.8792715E-3,-3.6314914E-3,-1.0066176E-2,-6.0936424E-4,4.8751016E-3,3.721168E-3,1.4576253E-2,3.4958655E-3,1.1878896E-2],"split_indices":[109,26,46,2,9,2,10,52,51,2,5,16,47,0,53,3,65,5,11,5,5,52,0,1,51,2,34,0,0,32,9,51,37,2,2,0,27,0,0,2,51,37,8,4,30,15,52,9,44,3,55,8,0,0,0,0,44,0,0,52,0,0,0,0,0,0,2,52,54,55,44,52,5,9,51,0,0,0,57,79,57,30,0,27,0,27,51,44,0,0,0,0,0,46,49,0,28,0,0,10,51,31,51,0,0,31,53,28,57,0,4,0,0,0,0,0,0,0,2,51,0,53,2,55,32,53,9,1,53,47,51,30,0,0,0,0,32,46,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.233E3,8.53E2,1.38E3,4.69E2,3.84E2,1.161E3,2.19E2,2.76E2,1.93E2,3.23E2,6.1E1,1.078E3,8.3E1,1E1,2.09E2,2.47E2,2.9E1,1.79E2,1.4E1,2.49E2,7.4E1,5.7E1,4E0,2.68E2,8.1E2,3.7E1,4.6E1,8E0,2E0,2.8E1,1.81E2,1E1,2.37E2,1.4E1,1.5E1,1.3E1,1.66E2,1E1,4E0,6.8E1,1.81E2,4.9E1,2.5E1,1.3E1,4.4E1,1.24E2,1.44E2,3.38E2,4.72E2,1.5E1,2.2E1,3.7E1,9E0,2.3E1,5E0,7E0,1.74E2,5E0,5E0,2.23E2,1.4E1,1.1E1,3E0,1.1E1,4E0,3.1E1,1.35E2,4E1,2.8E1,5.6E1,1.25E2,3.9E1,1E1,1.1E1,1.4E1,3E0,1E1,2E0,4.2E1,7E1,5.4E1,1.42E2,2E0,3.1E2,2.8E1,4.14E2,5.8E1,1.2E1,3E0,1.8E1,4E0,3.4E1,3E0,7.2E1,1.02E2,1.3E1,2.1E2,4E0,2.7E1,3E1,1.05E2,2.8E1,1.2E1,1.7E1,1.1E1,2.3E1,3.3E1,1.05E2,2E1,5E0,3.4E1,7E0,3E0,4E0,7E0,1.2E1,2E0,7E0,3.5E1,6.5E1,5E0,2.1E1,3.3E1,1.29E2,1.3E1,3.4E1,2.76E2,1.4E1,1.4E1,2.72E2,1.42E2,4.7E1,1.1E1,2E0,1E1,1.5E1,5.7E1,2.9E1,7.3E1,1.51E2,5.9E1,1.1E1,1.9E1,8.5E1,2E1,1.1E1,1.7E1,1E1,2E0,1.8E1,5E0,2.8E1,5E0,4.6E1,5.9E1,6E0,1.4E1,1.4E1,2E1,1.7E1,1.8E1,5.1E1,1.4E1,4E0,1.7E1,3E1,3E0,7.7E1,5.2E1,1.1E1,2E0,5E0,2.9E1,2.73E2,3E0,4E0,1E1,9E0,5E0,6.4E1,2.08E2,8.8E1,5.4E1,3.8E1,9E0,4.1E1,1.6E1,6E0,2.3E1,5.8E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-1.7873282E-3,2.4592288E-2,-7.348617E-2,-1.969372E-2,6.3377745E-2,-1.2363748E-1,-2.904285E-2,-2.5161242E-2,2.089101E-1,1.616755E-3,1.0745497E-1,-2.934173E-1,-1.0748997E-1,-2.4756845E-2,-2.1509973E-2,-3.678318E-2,4.8086867E-2,3.7943332E-3,1.3609833E-2,-3.756679E-3,1.46575915E-2,8.346931E-2,1.900414E-1,2.7400258E-3,-3.2834446E-1,-1.1823629E-1,4.3556914E-2,-3.8884863E-2,1.03168964E-1,-2.9992582E-2,-1.4792566E-1,9.9237375E-2,-4.2018935E-2,1.0413858E-2,-1.1093174E-2,2.3341495E-1,7.405006E-2,2.1316572E-1,-4.5865092E-2,-3.757961E-1,-4.158397E-3,-1.0581302E-1,-2.3246695E-1,9.017316E-3,-3.1616095E-2,-2.7003998E-2,-1.21300794E-1,-2.1963974E-2,1.6976069E-1,-6.163719E-2,-3.5638595E-3,8.8059035E-4,-2.0241973E-1,1.0108533E-3,1.3985598E-1,2.035353E-2,-1.6353165E-1,-4.262002E-3,-1.09982565E-2,5.7016127E-2,1.5517266E-2,1.053381E-1,3.0883044E-2,2.8874952E-1,1.6890444E-1,-5.8992016E-3,3.6905962E-3,-9.537237E-3,-4.408114E-1,-1.4069349E-1,-6.521967E-2,-2.9778248E-1,2.8171914E-3,-4.221494E-3,2.527323E-3,-1.172949E-1,-1.245064E-2,1.4195321E-3,-1.5287696E-1,-5.0164983E-3,3.8637894E-3,1.0201304E-2,3.898482E-3,-4.5500286E-2,-1.4642209E-1,1.7780276E-3,-2.9588336E-1,-4.265249E-1,-7.381596E-2,2.1619834E-1,4.9583115E-2,2.9499556E-3,-4.6666797E-2,-1.374125E-2,-3.060239E-3,-5.3526476E-2,2.1374466E-2,5.1446436E-3,-5.114565E-3,1.10295705E-1,-1.290623E-2,-4.4931214E-2,5.760606E-2,3.1949574E-1,5.0406978E-3,1.7827551E-1,-1.1590812E-3,-9.001485E-3,-2.2609038E-2,-1.09833516E-1,-2.78007E-1,3.425326E-2,-1.07930824E-1,-1.5389277E-2,-2.4813893E-3,-8.0328375E-2,-2.195009E-2,-2.6428888E-2,6.1790116E-2,-1.1979907E-2,-5.15754E-3,-3.349687E-3,-1.2409329E-4,-4.5747794E-3,-2.17379E-2,2.2672445E-3,-1.305561E-3,-2.3149505E-4,-2.6482956E-2,-2.538652E-2,-1.1028678E-2,-1.6311076E-4,-6.8558487E-3,8.4234285E-5,1.0708325E-2,3.5098863E-3,-4.0091663E-3,-4.3571545E-3,2.9041357E-3,9.775302E-3,-2.720508E-3,1.41421E-3,-9.905511E-3,5.9837573E-3,5.1164813E-4,-5.235343E-3,5.847727E-4,5.769122E-3,3.5804787E-4,8.732311E-3,1.7181106E-2,9.227137E-3,4.540433E-3,-9.256222E-3,-3.4582294E-3,-1.0840168E-3,-1.3740206E-2,5.863668E-3,-1.708757E-3,-1.9143022E-3,-7.573587E-3,-5.6732656E-4,-5.8686286E-3,4.1346986E-5,-2.6854442E-3,8.141524E-3,1.1707033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,-1,35,37,-1,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,-1,69,71,-1,73,75,77,79,81,83,85,-1,87,-1,89,91,93,95,-1,97,-1,99,101,103,105,-1,-1,-1,107,109,111,113,-1,-1,-1,115,117,-1,119,-1,-1,-1,-1,121,123,125,127,129,131,133,135,-1,137,-1,-1,139,141,-1,-1,143,-1,145,147,149,-1,151,-1,-1,-1,153,155,157,159,-1,-1,161,-1,163,165,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.240408E0,2.8164816E0,1.3404305E0,9.6267724E-1,2.3790755E0,7.4431133E-1,6.0193634E-1,6.393813E-1,1.8130058E-1,6.330238E-1,9.9439096E-1,3.1656146E-1,4.27181E-1,5.805526E-1,0E0,4.8386967E-1,4.7889256E-1,0E0,0E0,6.199849E-1,0E0,5.434289E-1,6.3166666E-1,0E0,2.1236849E-1,3.202479E-1,2.1686241E-1,2.7872607E-1,2.703788E-1,5.107759E-1,3.4141386E-1,2.0305955E-1,2.9133222E-1,0E0,5.5325806E-1,4.240154E-1,5.032263E-1,2.9915428E-1,1.2893231E-1,1.0039973E-1,0E0,3.0374122E-1,4.6123338E-1,0E0,7.525722E-2,3.3099067E-1,1.7832279E-1,1.2385243E-1,7.820511E-2,3.7368786E-1,5.220797E-1,0E0,7.754973E-1,0E0,2.8562093E-1,8.1113614E-2,1.5406847E-1,4.318159E-1,0E0,1.09078616E-1,0E0,4.5144272E-1,3.2472003E-1,1.662538E-1,1.2850952E-1,0E0,0E0,0E0,7.07953E-2,4.6948934E-1,4.4555175E-1,1.7587328E-1,0E0,0E0,0E0,4.365796E-1,2.2890155E-1,0E0,1.0044366E-1,0E0,0E0,0E0,0E0,2.8604516E-1,6.17761E-1,4.8794022E-1,5.0131905E-1,1.0890496E-1,1.0299596E-1,9.975517E-2,8.792241E-2,0E0,6.6599816E-2,0E0,0E0,2.1988428E-1,5.3976244E-1,0E0,0E0,4.6402788E-1,0E0,1.7328006E-1,4.0844408E-1,1.6230035E-1,0E0,1.0898328E-1,0E0,0E0,0E0,2.9453337E-1,1.286956E-1,2.2435933E-1,2.773639E-1,0E0,0E0,1.0866736E-1,0E0,1.6640417E-1,1.437964E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,54,54,55,55,56,56,57,57,59,59,61,61,62,62,63,63,64,64,68,68,69,69,70,70,71,71,75,75,76,76,78,78,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,92,92,95,95,96,96,99,99,101,101,102,102,103,103,105,105,109,109,110,110,111,111,112,112,115,115,117,117,118,118],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,-1,36,38,-1,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,-1,70,72,-1,74,76,78,80,82,84,86,-1,88,-1,90,92,94,96,-1,98,-1,100,102,104,106,-1,-1,-1,108,110,112,114,-1,-1,-1,116,118,-1,120,-1,-1,-1,-1,122,124,126,128,130,132,134,136,-1,138,-1,-1,140,142,-1,-1,144,-1,146,148,150,-1,152,-1,-1,-1,154,156,158,160,-1,-1,162,-1,164,166,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,3.7311627E2,3.8398104E0,7.780377E6,1.5166431E5,7E0,2.6572757E10,9.750871E6,1E0,2.6585266E2,1.3137E4,9.824011E-4,3.4409692E3,1.5963264E9,-2.1509973E-2,5.511673E0,3.37E2,3.7943332E-3,1.3609833E-2,2.04E5,1.46575915E-2,1.0918E4,3.7284137E2,2.7400258E-3,1.9125667E3,1.2427474E8,4.928836E9,6.747114E7,1E0,1.5288235E2,2.5207965E2,1.296E3,1.6E1,1.0413858E-2,1.1855755E8,1.0700461E6,2E0,1.2427981E9,2.14099E5,2.915E4,-4.158397E-3,3.2069644E2,4.551684E6,9.017316E-3,2.332E3,9.4168E4,4.3978744E0,2.1252015E6,2.3898147E5,5.507653E6,3.684685E2,8.8059035E-4,7.696629E0,1.0108533E-3,3.55E2,4.9363803E5,4.470588E0,5.306E3,-1.09982565E-2,1.8472951E6,1.5517266E-2,2.7183437E1,1.954E3,1.24900505E2,2.0620978E7,-5.8992016E-3,3.6905962E-3,-9.537237E-3,4.2378342E-1,3.8095238E0,2.7864855E11,1.6667E4,2.8171914E-3,-4.221494E-3,2.527323E-3,4.5154482E-1,1.9214156E3,1.4195321E-3,1.1863768E-4,-5.0164983E-3,3.8637894E-3,1.0201304E-2,3.898482E-3,3.5E2,3.7991562E0,1.4615384E0,2.5330253E-2,2.5789E4,1.592E1,4.8404854E-5,9.1233586E4,2.9499556E-3,1.0744694E9,-1.374125E-2,-3.060239E-3,4.2320535E2,8.6206274E2,5.1446436E-3,-5.114565E-3,2E0,-1.290623E-2,2.4125186E3,1.5125709E8,1.6140062E3,5.0406978E-3,7.8E1,-1.1590812E-3,-9.001485E-3,-2.2609038E-2,7.22E2,4.6229E4,4.752366E0,1.3E1,-1.5389277E-2,-2.4813893E-3,3.052775E2,-2.195009E-2,1.0008265E1,1.884E3,-1.1979907E-2,-5.15754E-3,-3.349687E-3,-1.2409329E-4,-4.5747794E-3,-2.17379E-2,2.2672445E-3,-1.305561E-3,-2.3149505E-4,-2.6482956E-2,-2.538652E-2,-1.1028678E-2,-1.6311076E-4,-6.8558487E-3,8.4234285E-5,1.0708325E-2,3.5098863E-3,-4.0091663E-3,-4.3571545E-3,2.9041357E-3,9.775302E-3,-2.720508E-3,1.41421E-3,-9.905511E-3,5.9837573E-3,5.1164813E-4,-5.235343E-3,5.847727E-4,5.769122E-3,3.5804787E-4,8.732311E-3,1.7181106E-2,9.227137E-3,4.540433E-3,-9.256222E-3,-3.4582294E-3,-1.0840168E-3,-1.3740206E-2,5.863668E-3,-1.708757E-3,-1.9143022E-3,-7.573587E-3,-5.6732656E-4,-5.8686286E-3,4.1346986E-5,-2.6854442E-3,8.141524E-3,1.1707033E-3],"split_indices":[26,51,52,27,27,3,12,9,109,55,2,41,51,7,0,52,0,0,0,5,0,9,57,0,4,44,12,44,109,51,4,28,3,0,44,46,16,12,2,9,0,51,28,0,0,1,52,31,32,44,51,0,55,0,10,32,53,2,0,27,0,52,2,57,28,0,0,0,26,55,30,9,0,0,0,37,51,0,41,0,0,0,0,2,57,52,37,1,57,37,32,0,12,0,0,4,57,0,0,6,0,32,7,51,0,3,0,0,0,2,28,55,3,0,0,4,0,53,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E3,1.638E3,6.02E2,7.65E2,8.73E2,2.82E2,3.2E2,7.48E2,1.7E1,3.64E2,5.09E2,2.3E1,2.59E2,3.18E2,2E0,6.46E2,1.02E2,8E0,9E0,3.59E2,5E0,3.96E2,1.13E2,2E0,2.1E1,2.42E2,1.7E1,2.87E2,3.1E1,6.1E2,3.6E1,6.5E1,3.7E1,1E1,3.49E2,2.2E1,3.74E2,1.03E2,1E1,1.7E1,4E0,2.2E2,2.2E1,5E0,1.2E1,2.52E2,3.5E1,1.1E1,2E1,2.77E2,3.33E2,9E0,2.7E1,2.3E1,4.2E1,2.5E1,1.2E1,3.4E2,9E0,9E0,1.3E1,2.16E2,1.58E2,3.6E1,6.7E1,6E0,4E0,6E0,1.1E1,1.17E2,1.03E2,1.8E1,4E0,7E0,5E0,3.4E1,2.18E2,6E0,2.9E1,6E0,5E0,1.1E1,9E0,2.34E2,4.3E1,3.28E2,5E0,9E0,1.8E1,2.2E1,2E1,1.5E1,1E1,4E0,8E0,1.16E2,2.24E2,7E0,2E0,2.14E2,2E0,4.1E1,1.17E2,3E1,6E0,6.4E1,3E0,3E0,8E0,9.7E1,2E1,3.1E1,7.2E1,1.5E1,3E0,3.2E1,2E0,1.84E2,3.4E1,6E0,2.3E1,1.4E2,9.4E1,3.9E1,4E0,1.27E2,2.01E2,3E0,2E0,4E0,5E0,1E1,8E0,2E0,2E1,1.7E1,3E0,7E0,3E0,2E0,1.14E2,2.16E2,8E0,1.75E2,3.9E1,1.8E1,2.3E1,4.8E1,6.9E1,1.1E1,1.9E1,4.7E1,1.7E1,2.4E1,7.3E1,2E0,1.8E1,1.3E1,1.8E1,3.5E1,3.7E1,1.4E1,1.8E1,1.01E2,8.3E1,7E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"167","size_leaf_vector":"1"}},{"base_weights":[5.4595435E-5,2.6183298E-2,-7.112061E-2,1.9251583E-3,1.0453584E-1,-8.927301E-2,5.791506E-2,-2.9330457E-2,4.8677076E-2,9.357345E-2,3.8287416E-1,-1.3177781E-1,-4.6422195E-2,7.3650174E-2,-7.7333553E-3,-4.9979646E-2,1.5195614E-2,5.395367E-2,-2.0999473E-1,1.400884E-1,2.6723878E-2,-1.8750106E-3,4.5704022E-1,-3.4634015E-1,-1.2172957E-1,-1.3830128E-1,-3.1381525E-2,1.5243606E-1,3.352835E-2,-5.9882086E-2,3.3099484E-2,-1.5191718E-2,2.1229487E-2,2.0511169E-2,8.326637E-2,-1.671159E-2,-1.9191583E-3,1.3475882E-2,1.224016E-1,3.768526E-2,-1.1067537E-2,7.275873E-3,2.2647655E-2,-1.935956E-2,-7.393026E-3,-1.0981647E-1,-2.4527511E-1,-9.739938E-2,-2.9383788E-1,-9.772879E-3,-9.414051E-2,2.1253967E-1,5.9645454E-4,1.6354265E-2,6.707738E-3,-3.5587233E-2,-1.0296927E-1,7.351093E-2,-5.917902E-2,-1.1409006E-2,6.453304E-2,2.9116305E-2,-1.3537344E-1,8.898363E-2,-1.6978353E-1,3.450459E-1,1.11031555E-1,1.5053266E-2,9.441279E-3,-1.570058E-1,-7.614526E-2,-4.3486685E-2,-3.4306315E-1,-5.3476566E-3,3.0242756E-3,-1.4090678E-3,-2.1871533E-2,-3.674669E-2,3.894286E-2,-2.5567975E-2,-6.328203E-3,1.6425159E-3,1.1337699E-2,-6.1313855E-4,3.2177072E-3,-6.748439E-2,4.6760872E-2,-1.5981846E-1,-5.859521E-2,1.6901079E-1,2.2933662E-2,-9.941161E-3,-9.123678E-3,1.4267582E-2,-1.2754814E-1,9.2830054E-2,-1.4363833E-2,2.043252E-2,7.891894E-3,-6.408486E-4,-8.792805E-3,8.035739E-2,2.1934603E-1,-1.4776702E-2,-4.4762937E-4,2.0610387E-3,1.9201599E-2,1.7709273E-1,8.4273964E-2,3.4137037E-2,-1.4550775E-1,-1.2061818E-1,-2.706099E-1,1.1302034E-1,-9.7228654E-2,-7.4595218E-3,2.817515E-3,-5.117483E-3,-1.790687E-2,-2.669274E-2,-1.0798719E-2,9.408615E-2,-4.3284107E-4,-2.013628E-3,5.506099E-3,-1.0572583E-2,-2.4999133E-3,3.7605767E-3,-1.7148751E-3,-4.541476E-3,-1.4766371E-2,-2.1150652E-3,-1.3955042E-2,1.1777472E-3,9.476222E-3,-5.3867567E-3,3.2351515E-3,3.12813E-3,-3.4599775E-3,-5.8891234E-4,4.0727737E-3,3.2429746E-3,-6.741796E-3,5.9188637E-3,1.2334085E-3,-2.6110413E-3,6.647435E-3,1.3744388E-3,-6.3795205E-3,2.8175327E-3,5.766087E-3,2.6101242E-3,1.1543285E-2,-8.291601E-3,9.162077E-3,4.4718646E-3,-4.8373896E-3,2.1472832E-3,-3.898864E-3,-8.662072E-3,7.6371175E-4,-3.8890282E-3,-9.431767E-3,-2.620979E-3,-1.3634253E-2,-4.335952E-3,7.053678E-3,-3.2865854E-3,-9.059839E-3,-3.742942E-4,-3.9735283E-3,6.7359745E-3,1.0156374E-3,-2.1003124E-3,2.1778035E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,-1,59,61,63,-1,-1,-1,65,67,-1,-1,-1,-1,-1,69,71,73,75,77,79,81,-1,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,-1,111,113,115,117,-1,-1,-1,-1,119,121,123,-1,-1,-1,-1,-1,125,127,129,131,133,135,-1,137,139,141,143,145,147,-1,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,161,163,165,167,-1,-1,-1,-1,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.065341E0,3.0389972E0,1.3801043E0,1.7870872E0,1.117547E0,9.318862E-1,2.7531123E-1,6.746997E-1,6.8013394E-1,1.1312253E0,4.594779E-1,5.140815E-1,3.5219604E-1,2.1300036E-1,0E0,4.1414773E-1,5.002356E-1,4.6981168E-1,2.5105724E-1,5.627279E-1,4.5806262E-1,0E0,7.321167E-2,9.3808174E-2,3.3807373E-1,2.0040321E-1,3.0149847E-1,1.8971294E-1,9.0104684E-2,4.6530044E-1,2.0572212E-1,0E0,3.258097E-1,3.0709982E-1,3.854766E-1,0E0,0E0,0E0,4.5532703E-1,5.614003E-1,0E0,0E0,0E0,0E0,0E0,3.503189E-1,3.9946365E-1,1.0994816E-1,3.3300567E-1,2.208772E-1,1.7479795E-1,9.775573E-2,0E0,6.967035E-2,0E0,7.5963676E-1,3.9677894E-1,1.7966859E-1,1.3448665E-1,3.9545128E-1,2.2235551E-1,2.6752484E-1,8.284077E-2,2.6463294E-1,1.4701231E-1,1.9385326E-1,3.15624E-1,4.0330142E-1,0E0,3.5593486E-1,5.494027E-1,1.1282459E-1,1.242336E-1,0E0,0E0,0E0,0E0,2.1272354E-1,1.3031907E-1,7.22026E-2,0E0,0E0,0E0,0E0,0E0,4.1076E-1,2.5202173E-1,6.601654E-1,2.5840494E-1,6.946695E-2,1.8754394E-1,0E0,7.897586E-2,2.2619992E-1,1.12787634E-1,1.7791837E-1,1.9695169E-1,3.3097118E-1,0E0,0E0,0E0,1.939727E-1,7.169223E-2,0E0,0E0,0E0,0E0,5.1966333E-1,3.8466012E-1,1.9109432E-1,1.07384086E-1,2.0439589E-1,1.2548804E-1,1.3996956E-1,2.9468906E-1,0E0,0E0,0E0,0E0,1.1630628E-1,0E0,9.464139E-2,8.2533784E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,38,38,39,39,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,77,77,78,78,79,79,85,85,86,86,87,87,88,88,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,101,101,102,102,107,107,108,108,109,109,110,110,111,111,112,112,113,113,114,114,119,119,121,121,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,-1,60,62,64,-1,-1,-1,66,68,-1,-1,-1,-1,-1,70,72,74,76,78,80,82,-1,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,-1,112,114,116,118,-1,-1,-1,-1,120,122,124,-1,-1,-1,-1,-1,126,128,130,132,134,136,-1,138,140,142,144,146,148,-1,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,162,164,166,168,-1,-1,-1,-1,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.0855529E3,3.4409692E3,4.459525E5,4.5505118E2,3.6282136E0,1.4299594E8,1.5107028E-1,1.0889231E1,1.4274633E7,8E0,3E0,9.7043E4,4.0601485E6,-7.7333553E-3,2.974979E6,3.3820656E7,6.9432104E7,4.05E2,1.296E3,1.921032E4,-1.8750106E-3,4.215533E3,1.1087069E7,1.2427474E8,1.305E3,4.5843E7,9.11898E5,5.545735E8,2.0116506E4,7.8E1,-1.5191718E-2,3.2485715E2,6.1114804E-8,1.5449402E7,-1.671159E-2,-1.9191583E-3,1.3475882E-2,1.3841E4,6.7652373E3,-1.1067537E-2,7.275873E-3,2.2647655E-2,-1.935956E-2,-7.393026E-3,1.7019386E2,1.2E1,3E0,4.0233E4,1.2633843E3,5.692891E-1,6.3997424E-1,5.9645454E-4,1.5595E4,6.707738E-3,1.273801E6,2.316491E-2,1.4400111E10,4.4195585E0,3.2982758E1,4.4768806E1,1.8297922E7,1.08504E5,7.780377E6,3.564E3,1.2905E4,2.2579439E0,1.0828989E8,9.441279E-3,1.4540612E7,7.1913286E8,8.63332E5,4.1E2,-5.3476566E-3,3.0242756E-3,-1.4090678E-3,-2.1871533E-2,6.005171E8,2.65851E5,3.1851864E7,-6.328203E-3,1.6425159E-3,1.1337699E-2,-6.1313855E-4,3.2177072E-3,3.983889E3,1.37E2,1.3833216E-2,5.495303E-6,3.138649E6,8.342E3,-9.941161E-3,2.5659466E1,3.730007E6,2.915723E0,2.2E1,4.6368E4,1.25E2,7.891894E-3,-6.408486E-4,-8.792805E-3,5.862126E2,1E0,-1.4776702E-2,-4.4762937E-4,2.0610387E-3,1.9201599E-2,2.3520866E8,1.8222668E2,3.4306667E1,1.3715873E3,5.7724595E0,5.4347825E-1,2.0992E4,2.603E3,-7.4595218E-3,2.817515E-3,-5.117483E-3,-1.790687E-2,3.8636575E6,-1.0798719E-2,4.4844124E-1,2.8454319E1,-2.013628E-3,5.506099E-3,-1.0572583E-2,-2.4999133E-3,3.7605767E-3,-1.7148751E-3,-4.541476E-3,-1.4766371E-2,-2.1150652E-3,-1.3955042E-2,1.1777472E-3,9.476222E-3,-5.3867567E-3,3.2351515E-3,3.12813E-3,-3.4599775E-3,-5.8891234E-4,4.0727737E-3,3.2429746E-3,-6.741796E-3,5.9188637E-3,1.2334085E-3,-2.6110413E-3,6.647435E-3,1.3744388E-3,-6.3795205E-3,2.8175327E-3,5.766087E-3,2.6101242E-3,1.1543285E-2,-8.291601E-3,9.162077E-3,4.4718646E-3,-4.8373896E-3,2.1472832E-3,-3.898864E-3,-8.662072E-3,7.6371175E-4,-3.8890282E-3,-9.431767E-3,-2.620979E-3,-1.3634253E-2,-4.335952E-3,7.053678E-3,-3.2865854E-3,-9.059839E-3,-3.742942E-4,-3.9735283E-3,6.7359745E-3,1.0156374E-3,-2.1003124E-3,2.1778035E-3],"split_indices":[26,51,51,27,55,52,44,37,53,44,3,3,1,46,0,28,7,7,10,28,4,0,4,50,44,11,44,28,43,32,8,0,51,36,49,0,0,0,9,51,0,0,0,0,0,51,17,8,1,4,26,26,0,9,0,9,37,5,57,57,57,44,7,27,2,10,53,44,0,44,5,1,0,0,0,0,0,7,28,31,0,0,0,0,0,27,3,37,36,28,0,0,55,9,55,3,11,8,0,0,0,51,109,0,0,0,0,7,57,52,51,56,55,28,0,0,0,0,0,49,0,56,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.184E3,1.598E3,5.86E2,1.221E3,3.77E2,5.14E2,7.2E1,7.32E2,4.89E2,3.64E2,1.3E1,2.57E2,2.57E2,6.8E1,4E0,5E2,2.32E2,4.8E2,9E0,2.14E2,1.5E2,2E0,1.1E1,1E1,2.47E2,3.5E1,2.22E2,2.2E1,4.6E1,4.47E2,5.3E1,3E0,2.29E2,2.25E2,2.55E2,4E0,5E0,2E1,1.94E2,1.45E2,5E0,2E0,9E0,6E0,4E0,2.27E2,2E1,2.9E1,6E0,1.66E2,5.6E1,1.5E1,7E0,4.1E1,5E0,2.87E2,1.6E2,3.7E1,1.6E1,1.31E2,9.8E1,2.14E2,1.1E1,2.5E2,5E0,8E0,1.86E2,1.29E2,1.6E1,9.3E1,1.34E2,7E0,1.3E1,2.6E1,3E0,3E0,3E0,1.07E2,5.9E1,2.3E1,3.3E1,3E0,1.2E1,2.7E1,1.4E1,2.07E2,8E1,6.9E1,9.1E1,1.2E1,2.5E1,3E0,1.3E1,1.08E2,2.3E1,7.2E1,2.6E1,2.03E2,1.1E1,4E0,7E0,2.36E2,1.4E1,2E0,3E0,2E0,6E0,5.2E1,1.34E2,1.16E2,1.3E1,7.2E1,2.1E1,1.3E1,1.21E2,3E0,4E0,3E0,1E1,1.03E2,4E0,2.4E1,3.5E1,2.1E1,2E0,1.3E1,1.94E2,5.6E1,2.4E1,5.2E1,1.7E1,8.8E1,3E0,3E0,9E0,6E0,1.9E1,6E0,7E0,8E1,2.8E1,2E0,2.1E1,4.5E1,2.7E1,2.1E1,5E0,1.92E2,1.1E1,1.72E2,6.4E1,3E0,1.1E1,3E0,4.9E1,1.25E2,9E0,1.05E2,1.1E1,1E1,3E0,5.3E1,1.9E1,3E0,1.8E1,2E0,1.1E1,9.9E1,2.2E1,8E1,2.3E1,1.3E1,1.1E1,1.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-8.1245875E-4,2.4001084E-2,-7.897361E-2,-4.5534056E-3,7.430332E-2,-1.6807044E-1,-4.0681645E-2,-2.7319374E-2,6.268484E-2,3.735567E-2,1.2897824E-1,-1.4586365E-1,-3.4890103E-1,-6.083329E-2,5.4066822E-2,-3.196659E-2,8.695547E-2,7.4901335E-2,-3.0670264E-1,4.7569685E-2,-1.1836692E-1,1.43368E-1,-1.8024689E-2,-1.3024184E-1,-3.9905962E-1,-1.8918386E-3,-4.4018346E-1,-3.1924408E-2,-1.3210838E-1,1.7150348E-1,2.261904E-2,-1.2451247E-1,-2.6802827E-2,1.997936E-4,5.390445E-3,5.501929E-2,3.176936E-1,-5.4960465E-3,-1.9498106E-2,5.581662E-2,-5.2239075E-2,-3.320589E-3,-1.4210947E-2,2.8338242E-1,1.2687929E-1,7.61471E-2,-6.3984044E-2,-2.4378482E-1,-9.539353E-2,-2.2325959E-2,-3.526761E-3,-2.3622802E-2,-9.29395E-3,-1.17367454E-1,-1.2140077E-2,1.6250346E-3,-1.5595536E-1,-1.2600736E-3,2.0704545E-1,5.459934E-3,3.433609E-4,-6.5702915E-2,-4.0077275E-1,-9.514196E-2,-2.0751286E-2,9.875731E-2,5.203582E-3,4.7268933E-1,1.1542894E-1,4.3889605E-2,1.4093274E-1,5.5892393E-3,-9.77808E-2,2.5214974E-3,1.4250018E-2,1.9332354E-1,1.0105541E-1,7.397371E-3,-5.6014615E-3,-8.082205E-3,1.9493331E-4,-2.8465426E-1,-8.7113905E-4,-1.3384163E-1,1.032718E-4,-1.5203898E-2,-8.553508E-2,3.6145363E-3,-6.855318E-2,-1.5226841E-2,-1.3374805E-1,1.3106213E-2,4.8354142E-3,-3.359536E-2,1.0829899E-1,7.5771273E-3,-1.2104181E-1,-2.424435E-2,-1.9300684E-3,-7.800803E-2,-1.78689E-2,8.355377E-2,-3.2628287E-2,1.8878414E-1,4.8408262E-2,-8.612351E-2,4.2107362E-2,7.7806916E-3,2.3562618E-2,8.288615E-3,-4.683228E-3,1.1318538E-2,4.003854E-2,1.3854563E-3,1.8275075E-1,-7.504965E-3,4.191752E-2,2.3761687E-3,2.2658461E-1,-1.5904985E-2,1.09599814E-1,-3.3608943E-1,-1.4444068E-1,-8.5743405E-2,-2.4303994E-1,1.2708922E-1,-3.4089226E-2,-1.09392345E-1,7.969897E-4,-1.06772985E-2,3.9765146E-3,-9.377051E-2,2.8850508E-3,-7.267583E-2,-1.7564069E-1,3.8949964E-3,-2.553493E-3,8.608753E-3,3.803052E-4,-4.0395088E-3,5.2305404E-3,-7.6091955E-3,-1.335159E-3,-1.8916818E-3,-7.718632E-3,1.4721495E-2,2.714887E-3,-2.4978311E-3,-3.6855065E-4,1.27917E-2,6.1575617E-3,-2.0775478E-3,3.3522628E-3,-1.6468895E-3,-1.4135226E-2,1.5945313E-2,1.3694982E-3,6.12595E-4,3.0521387E-3,-5.1882374E-3,3.0543145E-3,9.668468E-3,2.3103151E-3,4.6373983E-3,-4.2140717E-3,1.0887948E-2,1.0833238E-3,-9.197331E-3,5.2207476E-3,-8.915972E-3,-1.8148169E-2,-1.6301105E-3,-1.0140809E-2,-7.5037796E-3,2.8314618E-3,-8.347464E-4,-1.2486935E-2,1.2849086E-4,9.940714E-3,-4.3937676E-3,1.2136984E-3,-8.673408E-3,-3.5060025E-3,1.8681104E-3,-1.3566789E-3,-5.0907666E-3,1.6761427E-3,-5.4800723E-5,-5.0671184E-3,-9.124616E-3,6.048325E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,-1,-1,65,67,-1,-1,69,71,-1,-1,73,75,77,79,81,83,-1,-1,-1,-1,85,87,-1,89,-1,91,-1,93,95,97,99,101,103,105,107,109,111,113,-1,115,-1,-1,117,119,-1,-1,-1,-1,121,-1,123,125,-1,127,129,131,-1,133,-1,-1,135,137,139,141,-1,-1,143,-1,145,147,149,151,153,155,-1,-1,-1,-1,-1,157,159,161,-1,163,-1,165,-1,167,169,171,173,175,177,179,181,-1,183,-1,185,-1,187,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3462753E0,2.4435775E0,1.8354764E0,1.6641003E0,1.2395697E0,5.9964657E-1,7.2692806E-1,4.340003E-1,1.2639427E0,5.925448E-1,5.271883E-1,5.255108E-1,4.581871E-1,6.407136E-1,2.4355994E-1,3.698908E-1,8.428167E-2,1.2713848E0,1.5446633E-1,2.8853726E-1,1.7944378E-1,4.858017E-1,1.0441792E-1,5.267372E-1,2.0328116E-1,0E0,1.4204502E-1,3.7723222E-1,3.6616814E-1,1.04949415E-1,1.17815815E-1,6.397845E-1,3.0447143E-1,0E0,0E0,5.3983855E-1,5.7244086E-1,0E0,0E0,3.196876E-1,2.28108E-1,0E0,0E0,1.4674187E-1,3.3049655E-1,1.59716E-1,1.2591219E-1,2.8818548E-1,3.965543E-1,0E0,0E0,0E0,0E0,2.6333797E-1,1.6289209E-1,0E0,2.7145648E-1,0E0,6.8250954E-2,0E0,1.6851276E-1,1.4285043E-1,3.1493962E-1,2.771514E-1,8.476092E-1,5.891862E-1,3.9795333E-1,6.858897E-2,1.6699964E-1,2.146213E-1,2.273348E-1,0E0,2.1589746E-1,0E0,0E0,2.5043774E-1,6.144109E-1,0E0,0E0,0E0,0E0,1.4617658E-1,0E0,3.8358378E-1,1.432753E-1,0E0,9.5428884E-2,1.7425999E-1,1.394029E-1,0E0,1.7144549E-1,0E0,0E0,1.00338034E-1,8.834569E-2,1.7750831E-1,8.115327E-2,0E0,0E0,1.871801E-1,0E0,3.7943977E-1,3.379478E-1,1.9413757E-1,2.1129076E-1,3.7105745E-1,2.982229E-1,0E0,0E0,0E0,0E0,0E0,2.0155367E-1,8.436902E-2,1.1082852E-1,0E0,7.4885786E-2,0E0,1.2105942E-1,0E0,2.8615797E-1,9.228897E-2,6.95665E-2,6.628014E-1,1.6419172E-1,7.9811074E-2,1.02286585E-1,6.6570014E-2,0E0,1.2546568E-1,0E0,8.72013E-2,0E0,8.680998E-2,2.0829964E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,53,53,54,54,56,56,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,72,72,75,75,76,76,81,81,83,83,84,84,86,86,87,87,88,88,90,90,93,93,94,94,95,95,96,96,99,99,101,101,102,102,103,103,104,104,105,105,106,106,112,112,113,113,114,114,116,116,118,118,120,120,121,121,122,122,123,123,124,124,125,125,126,126,127,127,129,129,131,131,133,133,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,-1,-1,66,68,-1,-1,70,72,-1,-1,74,76,78,80,82,84,-1,-1,-1,-1,86,88,-1,90,-1,92,-1,94,96,98,100,102,104,106,108,110,112,114,-1,116,-1,-1,118,120,-1,-1,-1,-1,122,-1,124,126,-1,128,130,132,-1,134,-1,-1,136,138,140,142,-1,-1,144,-1,146,148,150,152,154,156,-1,-1,-1,-1,-1,158,160,162,-1,164,-1,166,-1,168,170,172,174,176,178,180,182,-1,184,-1,186,-1,188,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,5.0939494E5,2.0827537E0,1.0989723E3,2.373E3,1.0017953E8,3.4409692E3,4.51E2,1.1855755E8,2E0,2.7629982E7,4.7772845E6,4.56E2,4.8297736E7,6.504065E-2,3.983889E3,1.2511433E3,8.791777E3,1.4101E4,6.5E1,1.273801E6,1.059448E7,6.722251E1,4.650193E-1,1.4E1,-1.8918386E-3,4.230839E6,1.246E3,2.765625E0,6.3997424E-1,2.8091298E7,6.162955E0,4.7E2,1.997936E-4,5.390445E-3,1.6837112E7,3.4231E4,-5.4960465E-3,-1.9498106E-2,5.314225E2,1.9523809E0,-3.320589E-3,-1.4210947E-2,2.183847E0,4.0208E4,4.9E1,1.6E1,1.1693485E0,6.172708E2,-2.2325959E-2,-3.526761E-3,-2.3622802E-2,-9.29395E-3,2.8980975E5,2.1616615E6,1.6250346E-3,8.1057306E2,-1.2600736E-3,2.5927516E11,5.459934E-3,7.9023E4,2.0189162E7,1E2,1.56E2,3.79E2,4.1391224E1,4.470588E0,4.7019145E4,1.5005797E4,5.148149E5,4.7779464E7,5.5892393E-3,3.01E2,2.5214974E-3,1.4250018E-2,6.9664386E2,1.22E2,7.397371E-3,-5.6014615E-3,-8.082205E-3,1.9493331E-4,8.251192E0,-8.7113905E-4,1.4196E5,7.16173E5,-1.5203898E-2,1.44657E5,1.1210787E3,2.6481817E0,-1.5226841E-2,7.218466E7,1.3106213E-2,4.8354142E-3,2E0,6.747114E7,1.9406E4,1.667E3,-2.424435E-2,-1.9300684E-3,6E0,-1.78689E-2,9.2058825E-1,2.41E3,1E0,2.496172E3,1.2367184E3,4.7777777E0,7.7806916E-3,2.3562618E-2,8.288615E-3,-4.683228E-3,1.1318538E-2,4.899767E6,4.3698645E6,1.9701008E5,-7.504965E-3,1.5789307E8,2.3761687E-3,1.9473622E7,-1.5904985E-2,4.1844E4,3.68E2,2.074E3,2.4973914E2,1.4195741E0,5.49E2,7.7329254E2,3.1254683E0,7.969897E-4,5.0260034E5,3.9765146E-3,3.325328E11,2.8850508E-3,3.3487454E11,4.242625E6,3.8949964E-3,-2.553493E-3,8.608753E-3,3.803052E-4,-4.0395088E-3,5.2305404E-3,-7.6091955E-3,-1.335159E-3,-1.8916818E-3,-7.718632E-3,1.4721495E-2,2.714887E-3,-2.4978311E-3,-3.6855065E-4,1.27917E-2,6.1575617E-3,-2.0775478E-3,3.3522628E-3,-1.6468895E-3,-1.4135226E-2,1.5945313E-2,1.3694982E-3,6.12595E-4,3.0521387E-3,-5.1882374E-3,3.0543145E-3,9.668468E-3,2.3103151E-3,4.6373983E-3,-4.2140717E-3,1.0887948E-2,1.0833238E-3,-9.197331E-3,5.2207476E-3,-8.915972E-3,-1.8148169E-2,-1.6301105E-3,-1.0140809E-2,-7.5037796E-3,2.8314618E-3,-8.347464E-4,-1.2486935E-2,1.2849086E-4,9.940714E-3,-4.3937676E-3,1.2136984E-3,-8.673408E-3,-3.5060025E-3,1.8681104E-3,-1.3566789E-3,-5.0907666E-3,1.6761427E-3,-5.4800723E-5,-5.0671184E-3,-9.124616E-3,6.048325E-4],"split_indices":[26,27,52,51,2,44,51,8,44,6,44,27,0,44,56,27,4,51,2,8,9,12,57,26,8,0,1,2,53,26,44,52,1,0,0,44,10,0,0,4,53,0,0,53,9,3,8,41,54,0,0,0,0,27,46,0,4,0,30,0,2,44,3,2,28,55,53,27,4,27,7,0,8,0,0,51,10,0,0,0,0,56,0,28,1,0,1,51,40,0,44,0,0,8,44,1,2,0,0,10,0,52,2,15,4,57,53,0,0,0,0,0,44,44,32,0,5,0,44,0,9,0,2,54,52,0,51,52,0,47,0,30,0,30,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.239E3,1.7E3,5.39E2,1.085E3,6.15E2,1.61E2,3.78E2,8.11E2,2.74E2,3.68E2,2.47E2,1.45E2,1.6E1,3.12E2,6.6E1,7.8E2,3.1E1,2.66E2,8E0,3.46E2,2.2E1,2.25E2,2.2E1,1.38E2,7E0,4E0,1.2E1,2.23E2,8.9E1,1.3E1,5.3E1,4E1,7.4E2,9E0,2.2E1,2.47E2,1.9E1,4E0,4E0,3.2E2,2.6E1,1.9E1,3E0,2.2E1,2.03E2,7E0,1.5E1,3.1E1,1.07E2,5E0,2E0,8E0,4E0,4.1E1,1.82E2,1.1E1,7.8E1,2E0,1.1E1,9E0,4.4E1,3.4E1,6E0,5.9E1,6.81E2,1.31E2,1.16E2,1E1,9E0,2.82E2,3.8E1,5E0,2.1E1,3E0,1.9E1,5.5E1,1.48E2,5E0,2E0,5E0,1E1,2.6E1,5E0,7.6E1,3.1E1,4E0,3.7E1,1.43E2,3.9E1,7E0,7.1E1,5E0,6E0,3.4E1,1E1,1.5E1,1.9E1,4E0,2E0,5.7E1,2E0,6.9E1,6.12E2,4.6E1,8.5E1,3.3E1,8.3E1,2E0,8E0,7E0,2E0,4E0,2.78E2,9E0,2.9E1,1.4E1,7E0,1.1E1,4.4E1,2E0,1.46E2,1.8E1,8E0,5.4E1,2.2E1,6E0,2.5E1,3E1,7E0,1.23E2,2E1,3.3E1,6E0,3E1,4.1E1,5E0,2.9E1,5E0,5E0,8E0,7E0,1.2E1,7E0,4.2E1,1.5E1,5E0,6.4E1,3.18E2,2.94E2,1.5E1,3.1E1,1.8E1,6.7E1,2.8E1,5E0,2E0,8.1E1,1.42E2,1.36E2,3E0,6E0,2.3E1,6E0,5E0,2E0,4.1E1,3E0,2E0,1.44E2,7E0,1.1E1,4E0,4E0,3.5E1,1.9E1,3E0,1.9E1,3E0,3E0,1.2E1,1.3E1,7E0,2.3E1,3.3E1,9E1,2.9E1,4E0,1.1E1,1.9E1,3.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-3.4656876E-3,-5.351952E-2,2.6274245E-2,-1.2747617E-2,-1.0637433E-1,1.0414309E-3,9.445376E-2,-6.201751E-2,5.0764576E-2,-9.689437E-2,-3.337438E-1,3.0271124E-2,-3.8930897E-2,1.9532904E-1,7.693018E-2,-4.9521476E-2,-2.3004547E-1,1.6166791E-1,7.997764E-3,-1.0736081E-1,4.8439845E-2,-4.4899836E-1,-4.7645415E-3,-3.3688564E-2,5.9715867E-2,-3.307642E-2,-2.1052995E-1,2.9973128E-1,1.10923335E-1,5.867242E-2,1.7745456E-1,1.4771736E-1,-5.846707E-2,-5.7111215E-2,-4.3907437E-1,1.1712479E-1,1.7485587E-2,2.0536672E-2,-1.6173217E-2,-1.516325E-1,-8.0225304E-2,1.8415232E-1,-8.2734734E-4,-7.119522E-3,-2.3084685E-2,-1.29561145E-2,-1.9596943E-1,1.2606665E-1,3.4375753E-2,-3.575486E-2,1.9477805E-1,-2.5166985E-1,1.0959139E-3,1.5014525E-1,1.5449821E-2,-7.137163E-3,1.3408262E-1,1.2336628E-2,5.3685177E-2,1.4392537E-1,2.0089244E-2,2.0970746E-3,1.0163797E-2,-2.2896782E-2,-1.0146932E-1,1.8801771E-3,-5.4061483E-3,-2.4777403E-2,-3.454876E-3,3.8096733E-2,1.5086961E-1,3.3447903E-2,-9.269801E-3,-1.03746496E-1,-2.403738E-1,-5.061049E-3,-1.0160081E-1,1.1184586E-2,-1.363661E-4,-3.079251E-3,-1.6672726E-1,-9.730367E-3,-4.8050654E-4,9.4198555E-2,1.9546798E-1,-5.7571217E-2,5.7804994E-2,-4.9089182E-2,1.4049054E-2,1.5650908E-2,-9.536218E-4,-1.3867115E-3,-1.3256893E-2,1.0829271E-2,1.0666483E-3,1.8457612E-1,1.3621524E-3,6.1828047E-2,-5.078496E-2,7.961807E-3,-9.29495E-2,-5.917231E-2,8.233553E-2,9.074741E-2,-1.2443944E-1,3.6271904E-3,-4.88921E-3,1.7295112E-1,-2.6249574E-3,2.3454081E-2,1.3337374E-2,-1.5958723E-1,-2.1861143E-2,-1.9818157E-1,-1.9151937E-2,-2.120463E-1,3.7527233E-2,-2.2355671E-1,-8.4323935E-2,1.4419652E-2,-7.188495E-2,-1.3832505E-3,-1.3436902E-2,1.1276612E-1,-2.9146262E-2,1.2305596E-2,1.2844332E-1,6.92199E-2,-1.0525026E-1,-8.401025E-4,1.0472754E-1,-5.5705395E-2,2.4866327E-3,-4.7581743E-2,3.5313588E-2,6.188448E-4,9.685555E-3,1.4684148E-1,5.087646E-2,4.2309323E-3,-3.2598618E-3,8.313079E-3,-1.2547096E-2,-3.1360013E-3,8.845411E-3,5.348045E-3,-4.7255056E-3,5.9330775E-3,-3.3429514E-3,-7.3165824E-3,-1.6880009E-3,4.5586363E-3,1.1015917E-2,1.6156326E-3,-5.2772127E-3,2.6108688E-5,-8.899179E-3,2.1444068E-3,-7.737903E-3,-6.798065E-3,-1.5733095E-2,-1.8039045E-3,-1.6945466E-2,3.9480445E-3,-1.7807395E-3,-2.6035826E-3,-1.3390733E-2,-2.6101957E-3,-6.5729874E-3,-1.18792865E-4,4.5166286E-3,-1.274024E-3,-6.449166E-3,5.622615E-3,-4.5762467E-3,-5.1036775E-3,4.2421026E-3,7.4217045E-3,-4.8890914E-4,4.570664E-3,-2.6784255E-3,-5.9742127E-3,2.4794163E-3,6.640582E-4,-2.4130547E-2,6.2596607E-3,1.8329261E-3,-2.32988E-3,-8.624543E-3,-4.758888E-3,1.7077521E-3,5.1533934E-3,5.42672E-4,4.8990124E-3,1.5917825E-2,3.7005963E-3,1.1580137E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,-1,73,75,77,-1,-1,-1,79,81,83,85,87,89,91,-1,93,-1,-1,95,-1,97,99,-1,-1,-1,101,103,-1,-1,-1,-1,105,107,109,-1,111,113,115,117,-1,-1,119,121,-1,-1,123,125,127,129,131,133,-1,-1,-1,-1,-1,-1,135,-1,137,139,-1,141,143,145,147,149,-1,-1,151,-1,153,-1,155,157,159,-1,161,163,165,167,169,171,-1,-1,173,175,-1,177,179,181,183,185,187,-1,189,191,-1,-1,193,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3031158E0,1.780822E0,2.394644E0,1.4676322E0,7.4362564E-1,1.1894199E0,6.496272E-1,5.4292405E-1,9.710184E-1,5.330334E-1,3.214568E-1,1.1093783E0,4.2502433E-1,4.5142078E-1,5.805708E-1,4.4444793E-1,6.255927E-1,5.2488613E-1,6.896906E-1,3.770678E-1,2.2065967E-1,7.9698086E-2,0E0,6.232543E-1,6.7324495E-1,2.6145712E-1,1.4129043E-1,9.7824335E-2,2.2017169E-1,2.77552E-1,3.736025E-1,7.5237826E-2,3.6019254E-1,7.255536E-2,2.7225387E-1,1.2492496E-1,0E0,4.2861232E-1,0E0,4.9349713E-1,3.2564056E-1,9.491038E-2,0E0,0E0,0E0,2.516035E-1,7.149142E-2,2.2708452E-1,6.345225E-1,2.7496356E-1,1.7015046E-1,1.0016817E-1,0E0,7.327077E-2,0E0,0E0,1.5197998E-1,0E0,2.3197472E-1,3.5673267E-1,0E0,0E0,0E0,5.0494593E-1,4.8372328E-1,0E0,0E0,0E0,0E0,1.0686955E-1,1.6682541E-1,3.532476E-1,0E0,3.6688685E-1,2.3983312E-1,4.1345423E-1,3.140576E-1,0E0,0E0,1.9006835E-1,1.6804567E-1,0E0,0E0,1.8192333E-1,1.3904393E-1,3.700022E-1,6.440039E-1,2.2740436E-1,1.17008716E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.7418964E-2,0E0,2.2528726E-1,6.793735E-2,0E0,3.565284E-1,2.6535776E-1,2.3953605E-1,8.927197E-2,3.049513E-1,0E0,0E0,1.3312638E-1,0E0,2.3365074E-1,0E0,2.8437757E-1,3.5891753E-1,2.0600569E-1,0E0,2.085928E-1,1.5233594E-1,2.1082336E-1,2.1886855E-1,1.8353346E-1,9.4767064E-2,0E0,0E0,1.786871E-1,1.2350698E-1,0E0,1.0059181E-1,7.52349E-2,1.9525945E-1,8.7431914E-1,2.7446437E-1,1.6579992E-1,0E0,1.16838664E-1,1.1951425E-1,0E0,0E0,1.8343711E-1,1.720162E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,56,56,58,58,59,59,63,63,64,64,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,95,95,97,97,98,98,100,100,101,101,102,102,103,103,104,104,107,107,109,109,111,111,112,112,113,113,115,115,116,116,117,117,118,118,119,119,120,120,123,123,124,124,126,126,127,127,128,128,129,129,130,130,131,131,133,133,134,134,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,-1,74,76,78,-1,-1,-1,80,82,84,86,88,90,92,-1,94,-1,-1,96,-1,98,100,-1,-1,-1,102,104,-1,-1,-1,-1,106,108,110,-1,112,114,116,118,-1,-1,120,122,-1,-1,124,126,128,130,132,134,-1,-1,-1,-1,-1,-1,136,-1,138,140,-1,142,144,146,148,150,-1,-1,152,-1,154,-1,156,158,160,-1,162,164,166,168,170,172,-1,-1,174,176,-1,178,180,182,184,186,188,-1,190,192,-1,-1,194,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.6104508E10,1.0855529E3,2.41E3,5.24422E5,1.03E2,1.8238512E0,9.161318E0,9.073025E6,7.3610186E3,3.285E3,7E2,1E0,2.2277021E0,6.4683E4,6E0,3E0,1.998175E3,4.3930836E7,1.8736842E0,3.81094E7,6.862E3,-4.7645415E-3,2.3362158E1,9.87156E0,1.4622E4,4.3927447E11,1.0884912E7,2.0802219E8,1.296E3,4.516183E1,1.3984146E2,6E2,1.279012E7,8.01E2,3.3743455E0,1.7485587E-2,1.0828989E8,-1.6173217E-2,2.1818182E0,1.1795429E9,6.2277466E2,-8.2734734E-4,-7.119522E-3,-2.3084685E-2,2.4E1,2.064332E2,9.750871E6,3.64E2,1.214049E0,9.5E2,2.18988E5,1.0959139E-3,1.2E2,1.5449821E-2,-7.137163E-3,2.3487206E7,1.2336628E-2,5.9767612E7,4.42384E5,2.0089244E-2,2.0970746E-3,1.0163797E-2,3.79E2,1E0,1.8801771E-3,-5.4061483E-3,-2.4777403E-2,-3.454876E-3,2.0966542E0,1.0938637E1,8.791777E3,-9.269801E-3,3.001419E2,9.6910744E7,1.1216729E2,1.513E3,1.1184586E-2,-1.363661E-4,6.59148E-1,1.89E2,-9.730367E-3,-4.8050654E-4,1.1968015E8,1.5445488E8,8.74E2,3.8172052E2,2.407741E7,1.8053533E8,1.5650908E-2,-9.536218E-4,-1.3867115E-3,-1.3256893E-2,1.0829271E-2,1.0666483E-3,2.60656E5,1.3621524E-3,1.1317E4,2.47E2,7.961807E-3,8.5E1,9E0,8E0,1.0715278E7,7.096141E6,3.6271904E-3,-4.88921E-3,7.10309E2,-2.6249574E-3,9.9618355E8,1.3337374E-2,1.3177E4,1.460806E6,2.0354E4,-1.9151937E-2,1.152553E6,5.929839E8,1.9859155E0,2.1525E4,1.85209E5,4.3304763E0,-1.3832505E-3,-1.3436902E-2,2.6E1,6.1463413E0,1.2305596E-2,1.4813511E6,5.4468E4,9.750871E6,2.0783027E5,8.959817E7,4.987639E7,2.4866327E-3,2.6510325E-1,4.25E0,6.188448E-4,9.685555E-3,4.0601485E6,7.2201815E6,4.2309323E-3,-3.2598618E-3,8.313079E-3,-1.2547096E-2,-3.1360013E-3,8.845411E-3,5.348045E-3,-4.7255056E-3,5.9330775E-3,-3.3429514E-3,-7.3165824E-3,-1.6880009E-3,4.5586363E-3,1.1015917E-2,1.6156326E-3,-5.2772127E-3,2.6108688E-5,-8.899179E-3,2.1444068E-3,-7.737903E-3,-6.798065E-3,-1.5733095E-2,-1.8039045E-3,-1.6945466E-2,3.9480445E-3,-1.7807395E-3,-2.6035826E-3,-1.3390733E-2,-2.6101957E-3,-6.5729874E-3,-1.18792865E-4,4.5166286E-3,-1.274024E-3,-6.449166E-3,5.622615E-3,-4.5762467E-3,-5.1036775E-3,4.2421026E-3,7.4217045E-3,-4.8890914E-4,4.570664E-3,-2.6784255E-3,-5.9742127E-3,2.4794163E-3,6.640582E-4,-2.4130547E-2,6.2596607E-3,1.8329261E-3,-2.32988E-3,-8.624543E-3,-4.758888E-3,1.7077521E-3,5.1533934E-3,5.42672E-4,4.8990124E-3,1.5917825E-2,3.7005963E-3,1.1580137E-3],"split_indices":[109,18,51,2,11,11,52,53,44,4,0,2,90,40,2,3,8,51,47,52,50,9,0,57,57,2,30,49,7,28,53,51,2,7,2,53,0,44,0,55,5,55,0,0,0,8,51,9,10,41,0,1,0,0,0,0,49,0,44,2,0,0,0,2,84,0,0,0,0,53,52,51,0,4,44,4,2,0,0,37,10,0,0,7,7,2,51,9,7,0,0,0,0,0,0,1,0,9,0,0,8,8,16,44,12,0,0,51,0,31,0,9,46,9,0,31,5,52,9,12,53,0,0,3,57,0,27,1,9,32,12,44,0,26,53,0,0,46,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.217E3,8.26E2,1.391E3,4.67E2,3.59E2,1.016E3,3.75E2,2.63E2,2.04E2,3.46E2,1.3E1,5.87E2,4.29E2,5.4E1,3.21E2,2.46E2,1.7E1,5.6E1,1.48E2,3.23E2,2.3E1,8E0,5E0,1.85E2,4.02E2,4.16E2,1.3E1,2.3E1,3.1E1,2.73E2,4.8E1,1E1,2.36E2,1E1,7E0,4.8E1,8E0,1.44E2,4E0,1.21E2,2.02E2,7E0,1.6E1,2E0,6E0,1.65E2,2E1,1.1E2,2.92E2,4.12E2,4E0,1.1E1,2E0,6E0,1.7E1,2E0,2.9E1,5E0,2.68E2,4.4E1,4E0,5E0,5E0,1.3E2,1.06E2,4E0,6E0,5E0,2E0,1.5E1,3.3E1,1.37E2,7E0,8E1,4.1E1,4.5E1,1.57E2,5E0,2E0,1.56E2,9E0,1.8E1,2E0,7.7E1,3.3E1,5.9E1,2.33E2,3.25E2,8.7E1,2E0,2E0,2E0,9E0,3E0,3E0,1.9E1,1E1,2.49E2,1.9E1,3.9E1,5E0,9.7E1,3.3E1,1.1E1,9.5E1,1.2E1,3E0,3E1,3E0,1.33E2,4E0,4.7E1,3.3E1,3.5E1,6E0,7E0,3.8E1,1.8E1,1.39E2,1.25E2,3.1E1,5E0,4E0,6.7E1,1E1,1.4E1,1.9E1,1.6E1,4.3E1,1.04E2,1.29E2,3.06E2,1.9E1,2.2E1,6.5E1,3E0,1.6E1,2.7E1,2.22E2,2E0,1.7E1,2E0,3E0,9.4E1,3E0,2.8E1,5E0,9E0,2E0,6.6E1,2.9E1,1.6E1,1.4E1,1.23E2,1E1,9E0,3.8E1,2.3E1,1E1,2.8E1,7E0,4E0,3E0,2.3E1,1.5E1,6E0,1.2E1,9.8E1,4.1E1,1.05E2,2E1,2E1,1.1E1,6.4E1,3E0,6E0,4E0,1.5E1,4E0,1.3E1,3E0,3.7E1,6E0,1.02E2,2E0,8.4E1,4.5E1,2.98E2,8E0,1.3E1,9E0,1.4E1,5.1E1,2.4E1,3E0,9.9E1,1.23E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[2.063109E-3,-2.8614266E-2,4.8111178E-2,-1.0271923E-2,-1.1678773E-1,-3.483864E-2,7.49128E-2,-1.2822072E-2,1.7843323E-2,-9.6462354E-2,-3.2639736E-1,4.690898E-2,-7.6881625E-2,5.4508965E-2,1.3438389E-1,-2.3303373E-2,6.086781E-2,-1.9693539E-1,-6.1732758E-2,-1.5821628E-1,-3.871738E-1,1.0801983E-1,-2.9960567E-2,-9.79726E-2,8.7684155E-2,4.2759355E-2,1.6057523E-1,5.4102473E-2,1.6502425E-1,-3.709995E-2,2.7351888E-2,4.6706513E-2,2.851506E-1,-1.4432716E-1,-4.1397882E-1,-7.5985044E-2,7.03961E-2,3.3674602E-4,-1.0230354E-2,-8.054075E-3,-2.0513404E-2,1.9226278E-3,1.6568343E-1,2.0794948E-3,-3.090116E-3,-7.481016E-2,-2.4617161E-1,1.4188847E-1,-3.2046663E-3,-1.6390873E-2,6.564224E-2,2.2701393E-1,6.569807E-2,1.02904685E-1,-2.5341144E-1,2.8035098E-1,1.440812E-1,-4.070755E-2,2.129016E-1,3.6177855E-2,-1.9550057E-2,7.850489E-2,-1.5808279E-2,2.8051E-3,1.581064E-2,-8.19791E-2,-2.5220934E-1,-2.6406983E-2,-9.231607E-3,-1.7369522E-1,-5.1951963E-2,1.438835E-1,-5.435266E-3,9.959374E-3,4.1375146E-3,-8.872661E-2,2.2507415E-3,-3.651361E-1,1.4843503E-3,-2.0048413E-3,8.138599E-3,1.14505425E-1,-4.7698725E-2,-1.5227061E-2,6.912933E-2,1.691216E-2,1.8092057E-1,2.280635E-2,1.1454727E-2,2.6505712E-2,1.518384E-1,-1.8019201E-2,-2.058438E-3,7.685227E-3,1.5929969E-2,-1.1675003E-3,1.5442774E-1,-3.434543E-2,-1.2934487E-1,1.3976193E-2,3.695117E-3,1.1386113E-1,1.7007649E-3,4.480916E-2,1.7031871E-1,-3.410027E-3,1.962293E-2,-1.0749179E-2,-4.8932567E-2,-2.4787064E-3,-1.3290876E-2,-9.301923E-3,-1.0406815E-3,-4.4988874E-2,-1.0748581E-2,1.1651442E-2,3.5071464E-3,-1.5283486E-1,-6.116532E-2,-6.0006725E-3,-2.1046309E-2,3.2704134E-2,8.488802E-3,-6.715577E-2,7.4766744E-3,1.1037536E-1,3.9182555E-2,8.909937E-3,1.8837051E-4,2.2800027E-3,-6.3806362E-3,3.9758515E-3,-2.766755E-3,-3.5567518E-4,7.832832E-3,-3.294426E-3,1.6212517E-1,-2.3422656E-3,-8.282296E-5,-1.3492651E-2,-5.0814264E-3,8.246472E-3,1.4793783E-3,-3.9213044E-3,1.6140693E-3,4.1255048E-3,-1.4912402E-3,-5.1837E-3,9.185909E-3,4.5327833E-3,-7.157729E-4,-3.6657494E-3,3.230185E-3,-2.725035E-3,2.0503243E-3,-1.0280076E-2,-4.1078967E-3,3.6283254E-3,-3.0952082E-3,4.9519325E-3,-2.202163E-3,-1.41198635E-2,-2.4052148E-3,7.467701E-3,2.5257561E-3,9.438092E-4,7.7965427E-3,9.228923E-3,5.7585877E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,-1,-1,75,77,79,-1,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,109,-1,-1,111,113,115,-1,-1,-1,117,-1,119,-1,-1,-1,121,123,-1,125,-1,127,129,-1,131,133,-1,-1,-1,-1,-1,135,137,139,-1,-1,141,143,145,147,-1,149,-1,151,-1,-1,-1,-1,153,-1,-1,-1,155,157,-1,-1,159,-1,161,-1,163,165,-1,-1,-1,-1,-1,-1,-1,-1,-1,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1586134E0,2.1692276E0,1.9901414E0,1.1518112E0,9.526665E-1,7.5629056E-1,8.1295943E-1,8.5565513E-1,0E0,7.2759295E-1,1.4317155E-1,3.5555357E-1,5.118644E-1,6.2248373E-1,4.141662E-1,6.78219E-1,4.2498827E-1,5.5683804E-1,3.0476803E-1,8.024143E-2,1.1253762E-1,1.5172559E-1,1.0194383E-1,4.2439568E-1,1.560675E-1,6.182847E-1,3.002615E-1,7.614378E-1,2.5910306E-1,6.959193E-1,8.5856503E-1,2.616518E-1,9.276396E-2,2.820052E-1,2.3670554E-1,3.2912803E-1,2.4120314E-1,0E0,0E0,0E0,0E0,0E0,6.864399E-2,0E0,0E0,2.0023227E-1,5.761341E-1,9.9626094E-2,0E0,5.3001624E-1,4.8441637E-1,1.2829483E-1,1.7097992E-1,1.567303E-1,1.923751E-1,8.262932E-2,1.920402E-1,4.196266E-1,1.1315414E-1,5.488565E-1,0E0,2.61523E-1,9.636351E-2,0E0,0E0,1.4249091E-1,1.1458707E-1,0E0,0E0,1.2931406E-1,1.4040998E-1,7.050274E-2,0E0,0E0,0E0,1.7087632E-1,0E0,2.1674514E-1,0E0,0E0,0E0,1.4314142E-1,4.417825E-1,0E0,3.9958072E-1,0E0,6.654775E-2,9.2819124E-2,0E0,1.0206221E-1,8.926356E-2,0E0,0E0,0E0,0E0,0E0,1.8842769E-1,3.9741892E-1,1.10833764E-1,0E0,0E0,3.43302E-1,4.318422E-1,2.3705263E-1,2.3663735E-1,0E0,8.4296584E-2,0E0,1.0596134E-1,0E0,0E0,0E0,0E0,1.591667E-1,0E0,0E0,0E0,1.17687464E-1,7.975903E-2,0E0,0E0,8.6400665E-2,0E0,3.1303957E-1,0E0,3.9889145E-1,4.6228713E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.16892576E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,42,42,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,69,69,70,70,71,71,75,75,77,77,81,81,82,82,84,84,86,86,87,87,89,89,90,90,96,96,97,97,98,98,101,101,102,102,103,103,104,104,106,106,108,108,113,113,117,117,118,118,121,121,123,123,125,125,126,126,136,136],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,-1,-1,76,78,80,-1,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,110,-1,-1,112,114,116,-1,-1,-1,118,-1,120,-1,-1,-1,122,124,-1,126,-1,128,130,-1,132,134,-1,-1,-1,-1,-1,136,138,140,-1,-1,142,144,146,148,-1,150,-1,152,-1,-1,-1,-1,154,-1,-1,-1,156,158,-1,-1,160,-1,162,-1,164,166,-1,-1,-1,-1,-1,-1,-1,-1,-1,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4887473E-1,6.748543E-1,2.960909E2,2.4507338E4,1.4299594E8,1.5277338E0,1.8137958E6,1.8137958E6,1.7843323E-2,1.8903887E0,9.1002154E-1,1.5036375E6,9.750871E6,3.3791902E0,3.564E3,1.067536E3,2.2376953E1,7.8351054E11,3.0497742E3,2.7487562E0,1.2199979E10,1.5053125E2,2.25E2,9.669789E0,3.37E2,1E0,2.4848485E0,1.505516E7,1.0918E4,1.1037509E2,1.6904226E7,2.227328E4,2.8E2,5.1270317E-2,9.410836E5,5.2143492E-5,8.965631E7,3.3674602E-4,-1.0230354E-2,-8.054075E-3,-2.0513404E-2,1.9226278E-3,1.8839E4,2.0794948E-3,-3.090116E-3,2.373E3,3.25855E7,8.2363125E4,-3.2046663E-3,6.189387E6,1.1057693E0,1.13814086E5,1.386749E1,5.815045E2,4.44E2,9.496754E6,1E0,3.791269E7,2.6970406E0,1.4540612E7,-1.9550057E-2,4.2197E4,2.8080197E2,2.8051E-3,1.581064E-2,8.764843E3,1.3177E4,-2.6406983E-2,-9.231607E-3,1.5012488E1,1.6992E4,1.0526E4,-5.435266E-3,9.959374E-3,4.1375146E-3,2.0184135E0,2.2507415E-3,9.6203804E-1,1.4843503E-3,-2.0048413E-3,8.138599E-3,1.0828514E5,3.832274E2,-1.5227061E-2,3.271242E0,1.691216E-2,1E0,9.70107E7,1.1454727E-2,4.5412036E2,1.5756359E0,-1.8019201E-2,-2.058438E-3,7.685227E-3,1.5929969E-2,-1.1675003E-3,1.9577E4,3.5734247E2,3.3E1,1.3976193E-2,3.695117E-3,2.6540773E1,9.309785E8,1.7314286E0,5.08E2,-3.410027E-3,8.31E2,-1.0749179E-2,1E0,-2.4787064E-3,-1.3290876E-2,-9.301923E-3,-1.0406815E-3,1.97729E5,-1.0748581E-2,1.1651442E-2,3.5071464E-3,6.7699623E-1,5.836513E3,-6.0006725E-3,-2.1046309E-2,1.2E1,8.488802E-3,1.3676985E4,7.4766744E-3,1.6143107E0,1.5435694E9,8.909937E-3,1.8837051E-4,2.2800027E-3,-6.3806362E-3,3.9758515E-3,-2.766755E-3,-3.5567518E-4,7.832832E-3,-3.294426E-3,1.710933E0,-2.3422656E-3,-8.282296E-5,-1.3492651E-2,-5.0814264E-3,8.246472E-3,1.4793783E-3,-3.9213044E-3,1.6140693E-3,4.1255048E-3,-1.4912402E-3,-5.1837E-3,9.185909E-3,4.5327833E-3,-7.157729E-4,-3.6657494E-3,3.230185E-3,-2.725035E-3,2.0503243E-3,-1.0280076E-2,-4.1078967E-3,3.6283254E-3,-3.0952082E-3,4.9519325E-3,-2.202163E-3,-1.41198635E-2,-2.4052148E-3,7.467701E-3,2.5257561E-3,9.438092E-4,7.7965427E-3,9.228923E-3,5.7585877E-3],"split_indices":[37,26,51,4,44,52,27,27,0,52,26,49,9,34,2,51,55,30,51,53,5,51,10,53,0,109,56,49,9,55,46,49,0,37,31,41,44,0,0,0,0,0,1,0,0,2,5,46,0,44,53,32,34,4,0,46,15,44,52,44,0,12,4,0,0,32,9,0,0,53,2,9,0,0,0,52,0,41,0,0,0,32,55,0,52,0,113,1,0,4,53,0,0,0,0,0,9,51,10,0,0,57,5,52,2,0,2,0,15,0,0,0,0,11,0,0,0,41,27,0,0,3,0,32,0,41,7,0,0,0,0,0,0,0,0,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.234E3,1.341E3,8.93E2,1.111E3,2.3E2,2.18E2,6.75E2,1.105E3,6E0,2.11E2,1.9E1,7.4E1,1.44E2,5.04E2,1.71E2,9.68E2,1.37E2,5.3E1,1.58E2,6E0,1.3E1,4.1E1,3.3E1,1.28E2,1.6E1,4.55E2,4.9E1,4.8E1,1.23E2,7.61E2,2.07E2,1.3E2,7E0,4.4E1,9E0,1.43E2,1.5E1,2E0,4E0,4E0,9E0,2E1,2.1E1,1.1E1,2.2E1,1.12E2,1.6E1,1.2E1,4E0,1.27E2,3.28E2,2.8E1,2.1E1,4.2E1,6E0,1.7E1,1.06E2,7.51E2,1E1,2.04E2,3E0,8.6E1,4.4E1,2E0,5E0,2.9E1,1.5E1,4E0,5E0,2.7E1,1.16E2,1.1E1,4E0,1.1E1,1E1,1.01E2,1.1E1,1.1E1,5E0,2E0,1E1,2.4E1,1.03E2,2E0,3.26E2,5E0,2.3E1,1.8E1,3E0,1.7E1,2.5E1,3E0,3E0,8E0,9E0,6E0,1E2,7.02E2,4.9E1,5E0,5E0,6.2E1,1.42E2,6.4E1,2.2E1,1.6E1,2.8E1,4E0,2.5E1,3E0,1.2E1,2.2E1,5E0,1.13E2,3E0,3E0,8E0,2.9E1,7.2E1,4E0,7E0,1.2E1,1.2E1,9.5E1,8E0,1.36E2,1.9E2,2.1E1,2E0,1.6E1,2E0,1E1,7E0,3E0,2.2E1,3E0,9.7E1,4.57E2,2.45E2,3E0,4.6E1,3.3E1,2.9E1,3.9E1,1.03E2,4E1,2.4E1,2E0,2E1,8E0,2E1,2E1,5E0,9.7E1,1.6E1,1.2E1,1.7E1,3E0,6.9E1,6E0,6E0,4E0,9.1E1,6.7E1,6.9E1,1.68E2,2.2E1,4.2E1,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-2.0491793E-4,1.9032683E-2,-6.949952E-2,4.041497E-3,8.878441E-2,-1.13051526E-1,-8.441809E-3,-2.9968456E-3,1.6308793E-1,4.483897E-2,1.4151256E-1,-1.0398305E-1,-4.1178244E-1,-3.936506E-2,5.4915316E-2,2.528618E-3,-1.2666024E-1,3.4413764E-1,1.06884114E-1,5.219359E-2,-1.2379083E-2,1.5341726E-1,-2.8590262E-2,-8.8031754E-2,-2.7561992E-1,-3.8269958E-3,-2.2943858E-2,-2.7409712E-2,-1.8845703E-1,-7.946378E-3,6.4705834E-2,-3.9511982E-2,2.5336495E-2,-3.201696E-1,-9.4453804E-2,5.2301693E-3,1.7927809E-2,1.4415267E-1,-1.7802124E-1,-7.868533E-2,6.562694E-2,1.2353768E-1,2.2073479E-1,1.6390216E-3,-8.685371E-3,-2.9625085E-1,-7.828404E-2,-3.3272535E-1,-2.642503E-3,-2.1458944E-2,-9.349922E-3,-1.1402984E-2,-1.3752619E-3,8.24029E-2,-9.201071E-4,-2.839035E-2,-1.4799894E-1,3.9446734E-2,-2.6131218E-2,-3.4165573E-3,-2.04593E-2,-1.2403849E-1,-1.1883408E-3,1.78657E-1,1.8167716E-3,1.8098373E-4,-1.2225324E-2,-3.8283266E-2,-1.0911014E-2,-4.723425E-2,7.652212E-2,1.3372709E-1,-6.1848033E-3,1.1697134E-2,4.593727E-3,-1.6119046E-2,-4.646135E-3,-2.5876945E-2,-1.1703019E-1,-1.9385207E-1,-4.0347636E-1,-8.318837E-4,-2.8161362E-3,4.6857197E-2,7.1084974E-3,-4.481583E-2,3.2881886E-2,-1.8657852E-2,-2.4961656E-1,2.0412825E-1,3.304229E-2,-5.8785412E-2,5.2120246E-2,-3.774728E-3,-8.983119E-3,-3.3442231E-3,1.979541E-1,-7.648506E-2,3.4563255E-3,-1.390592E-1,2.9181915E-3,1.208332E-1,3.111786E-2,1.4202829E-2,1.1944925E-1,-7.1970165E-2,5.0823797E-2,-2.1539038E-1,-6.660508E-2,1.3446158E-3,-1.2374239E-2,-1.0034365E-2,-2.3455076E-2,8.166903E-3,-8.278312E-3,3.7688424E-3,-9.719623E-4,-9.0480736E-4,-3.6389101E-3,2.4113886E-3,-1.1491954E-3,5.008794E-4,-8.636708E-3,-1.796093E-2,-4.2843004E-3,1.966667E-3,1.0533759E-2,2.1517216E-4,2.8449595E-3,-3.1414556E-3,1.0063043E-2,3.7212358E-3,-1.1916577E-3,1.2189899E-2,6.1331238E-3,-5.2598E-3,3.0825369E-3,-1.0158232E-2,3.5277721E-3,3.7181734E-3,9.125057E-3,1.0350201E-2,5.617966E-4,9.024506E-3,4.156311E-3,-5.706509E-3,-1.5244231E-3,3.912716E-3,-2.925445E-3,4.419002E-3,-1.0479577E-2,-4.227507E-3,1.6804165E-3,2.4783858E-3,-6.721243E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,-1,-1,49,51,-1,53,55,57,59,61,-1,-1,63,65,67,69,71,73,-1,-1,75,77,79,-1,81,-1,-1,-1,83,-1,85,87,89,91,-1,-1,93,-1,95,-1,-1,-1,97,-1,99,101,103,-1,-1,-1,-1,-1,105,107,109,111,113,-1,115,-1,117,119,121,123,125,127,129,131,-1,-1,-1,133,135,-1,137,-1,139,141,-1,143,145,147,149,151,-1,-1,-1,-1,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.975384E0,1.8264112E0,1.2887506E0,1.611339E0,7.096579E-1,7.2383785E-1,3.9988106E-1,9.4258773E-1,5.8393383E-1,4.121934E-1,2.8788948E-1,7.313783E-1,2.058636E-1,2.3902965E-1,1.6251293E-1,1.2676511E0,3.3746135E-1,1.2019825E-1,5.353267E-1,2.9877362E-1,0E0,2.362268E-1,1.12719014E-1,4.92342E-1,2.659253E-1,0E0,0E0,1.3199542E-1,9.2586994E-2,0E0,9.9484235E-2,5.5684507E-1,6.2340283E-1,2.2377914E-1,1.0167259E-1,0E0,0E0,1.4717007E-1,1.0156338E-1,9.569918E-2,1.9052017E-1,2.647866E-1,1.4881551E-1,0E0,0E0,9.079462E-2,4.9243152E-1,9.8642945E-2,0E0,1.0499978E-1,0E0,0E0,0E0,1.3585034E-1,0E0,4.271918E-1,5.60506E-1,7.0306385E-1,4.75783E-1,0E0,0E0,9.233773E-2,0E0,1.7055726E-1,0E0,0E0,0E0,6.714378E-2,0E0,1.5254697E-1,2.7589577E-1,1.9380498E-1,0E0,0E0,0E0,0E0,0E0,3.7487316E-1,6.8002903E-1,1.5023747E-1,1.1439109E-1,1.3907996E-1,0E0,9.6264265E-2,0E0,2.9330462E-1,1.0822318E-1,1.04415275E-1,5.096215E-1,1.0454023E-1,5.467638E-1,4.0578097E-1,1.3190873E-1,0E0,0E0,0E0,9.717369E-2,7.103187E-2,0E0,1.6557343E-1,0E0,2.0215464E-1,2.5580886E-1,0E0,1.6941369E-1,1.3244048E-1,1.699683E-1,2.5914216E-1,2.644794E-1,0E0,0E0,0E0,0E0,8.7629676E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,49,49,53,53,55,55,56,56,57,57,58,58,61,61,63,63,67,67,69,69,70,70,71,71,77,77,78,78,79,79,80,80,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,96,96,97,97,99,99,101,101,102,102,104,104,105,105,106,106,107,107,108,108,113,113],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,-1,-1,50,52,-1,54,56,58,60,62,-1,-1,64,66,68,70,72,74,-1,-1,76,78,80,-1,82,-1,-1,-1,84,-1,86,88,90,92,-1,-1,94,-1,96,-1,-1,-1,98,-1,100,102,104,-1,-1,-1,-1,-1,106,108,110,112,114,-1,116,-1,118,120,122,124,126,128,130,132,-1,-1,-1,134,136,-1,138,-1,140,142,-1,144,146,148,150,152,-1,-1,-1,-1,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3359457E10,1.8137958E6,1E0,6.082581E3,2.539E3,4.5723195E6,1.067536E3,4.987639E7,1E0,4.9E1,4.4263797E0,1.2427474E8,7.0988875E6,3.0121632E6,2.8316305E0,2.6659616E2,5.4106476E7,1.6E1,4.92E2,3.4048372E6,-1.2379083E-2,6.852143E1,3.84E2,9.2699325E5,4.8083666E-1,-3.8269958E-3,-2.2943858E-2,5.077728E8,1.0526E4,-7.946378E-3,5.1E1,4.3978744E0,2.1722221E1,3.5902756E-6,1.4E1,5.2301693E-3,1.7927809E-2,3.5296965E-1,3.7137984E7,2.909019E3,8.6906275E-6,2.0295714E7,3.404013E0,1.6390216E-3,-8.685371E-3,8.625696E-1,1.30727E5,1.2E1,-2.642503E-3,7.2530017E0,-9.349922E-3,-1.1402984E-2,-1.3752619E-3,1.96E4,-9.201071E-4,3.30399E6,1.1880085E2,3.921901E6,7.06699E8,-3.4165573E-3,-2.04593E-2,1.13808104E8,-1.1883408E-3,1.2566517E5,1.8167716E-3,1.8098373E-4,-1.2225324E-2,1.5156479E1,-1.0911014E-2,2.2415E4,2.2174756E4,1.0620689E0,-6.1848033E-3,1.1697134E-2,4.593727E-3,-1.6119046E-2,-4.646135E-3,3.124379E2,7.5569354E2,2.6432162E5,1.5265896E8,1.1563158E1,-2.8161362E-3,8.4342186E1,7.1084974E-3,2.3428571E0,2.064332E2,1.2627974E8,1.13E3,3.0872485E2,6.567044E2,1.2204E4,4.6368E4,-3.774728E-3,-8.983119E-3,-3.3442231E-3,3.494351E6,1.8496495E5,3.4563255E-3,3.790491E2,2.9181915E-3,4.551E4,6.9436204E-1,1.4202829E-2,7.43361E0,7.22E2,4.9355304E2,4.59E3,2.9799202E3,1.3446158E-3,-1.2374239E-2,-1.0034365E-2,-2.3455076E-2,2.6185484E1,-8.278312E-3,3.7688424E-3,-9.719623E-4,-9.0480736E-4,-3.6389101E-3,2.4113886E-3,-1.1491954E-3,5.008794E-4,-8.636708E-3,-1.796093E-2,-4.2843004E-3,1.966667E-3,1.0533759E-2,2.1517216E-4,2.8449595E-3,-3.1414556E-3,1.0063043E-2,3.7212358E-3,-1.1916577E-3,1.2189899E-2,6.1331238E-3,-5.2598E-3,3.0825369E-3,-1.0158232E-2,3.5277721E-3,3.7181734E-3,9.125057E-3,1.0350201E-2,5.617966E-4,9.024506E-3,4.156311E-3,-5.706509E-3,-1.5244231E-3,3.912716E-3,-2.925445E-3,4.419002E-3,-1.0479577E-2,-4.227507E-3,1.6804165E-3,2.4783858E-3,-6.721243E-4],"split_indices":[18,27,58,51,2,27,51,44,107,3,41,44,44,27,53,51,44,3,3,44,0,57,0,44,41,0,0,7,9,0,3,52,52,37,8,0,0,26,1,31,37,49,40,0,0,26,28,17,0,52,0,0,0,2,0,9,51,5,7,0,0,44,0,32,0,0,0,57,0,28,46,53,0,0,0,0,0,4,4,27,44,53,0,57,0,55,51,5,2,4,51,2,11,0,0,0,28,32,0,4,0,12,52,0,55,2,51,12,4,0,0,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.23E3,1.746E3,4.84E2,1.438E3,3.08E2,2.82E2,2.02E2,1.378E3,6E1,1.69E2,1.39E2,2.75E2,7E0,1.36E2,6.6E1,1.32E3,5.8E1,1.3E1,4.7E1,1.66E2,3E0,1.3E2,9E0,2.53E2,2.2E1,2E0,5E0,1.27E2,9E0,2E0,6.4E1,4.64E2,8.56E2,7E0,5.1E1,3E0,1E1,4.2E1,5E0,1.5E1,1.51E2,9.2E1,3.8E1,7E0,2E0,1E1,2.43E2,1.7E1,5E0,1.24E2,3E0,6E0,3E0,5.3E1,1.1E1,4.22E2,4.2E1,6.72E2,1.84E2,3E0,4E0,3.5E1,1.6E1,3.1E1,1.1E1,2E0,3E0,1.3E1,2E0,1.3E1,1.38E2,8.9E1,3E0,2.8E1,1E1,7E0,3E0,1.04E2,1.39E2,7E0,1E1,8.3E1,4.1E1,3.7E1,1.6E1,3.33E2,8.9E1,1.9E1,2.3E1,2.4E1,6.48E2,1.3E2,5.4E1,2.4E1,1.1E1,2E0,2.9E1,1E1,3E0,7E0,6E0,6.9E1,6.9E1,5E0,8.4E1,6.5E1,3.9E1,4.6E1,9.3E1,2E0,5E0,5E0,5E0,8E1,3E0,2.4E1,1.3E1,1.97E2,1.36E2,6.6E1,2.3E1,1.7E1,2E0,1.1E1,1.2E1,4E0,2E1,3.33E2,3.15E2,1.26E2,4E0,3.9E1,1.5E1,1.2E1,1.7E1,8E0,2E0,5E0,2E0,4.8E1,2.1E1,5E0,6.4E1,2E1,6.4E1,2.6E1,3.9E1,3E1,9E0,2E0,4.4E1,7.4E1,1.9E1,2.6E1,5.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"155","size_leaf_vector":"1"}},{"base_weights":[-3.3392487E-3,-4.2383276E-2,2.7456071E-2,-5.566944E-2,4.2788107E-2,5.4939747E-2,-6.647911E-2,-4.570974E-2,-1.4169043E-1,4.99383E-2,-8.986996E-3,3.4757454E-2,1.0998993E-1,-1.00141674E-1,4.6406955E-2,-2.2317322E-2,-8.880507E-2,-3.9694065E-1,-1.071495E-1,8.056376E-2,-1.775938E-2,8.8112846E-2,1.0653663E-2,1.6846415E-2,9.670298E-2,-7.72114E-2,-1.9613159E-1,1.3823109E-2,2.0510738E-1,-1.9583754E-2,-2.0887626E-2,-1.1965659E-1,-1.5579584E-2,-2.1191524E-2,-3.392834E-3,-1.6954513E-1,-6.005808E-2,6.0327824E-2,1.9422558E-1,-1.5181927E-1,1.934057E-2,1.0811771E-1,-3.1544056E-2,3.8430646E-2,-1.2639393E-2,4.5051016E-2,1.332306E-1,-4.4983316E-2,-1.4370541E-1,-2.2412549E-1,2.2749973E-3,1.6997834E-1,-5.19198E-3,1.2263801E-2,1.0089535E-3,-3.9525025E-2,3.1567346E-2,-3.2371268E-2,-1.6812114E-1,9.630988E-2,-4.9617007E-2,-4.007125E-3,-3.039837E-1,3.894151E-4,-6.6089844E-3,1.517008E-1,3.577467E-2,1.3788728E-2,6.67307E-4,-1.4942675E-3,-1.0877957E-2,-1.4839321E-2,5.7954113E-3,1.3480149E-1,3.3997009E-3,-1.8584367E-2,2.1811662E-2,2.6521023E-2,1.8936512E-1,-3.1151429E-2,4.1176938E-2,-6.634847E-3,9.788042E-2,1.2044012E-1,1.8345295E-2,-8.2594536E-2,1.731723E-2,-2.9553536E-1,-9.6649975E-2,-1.0980773E-2,-4.009327E-3,1.1468306E-2,1.0010585E-4,3.015265E-4,-8.7421145E-3,-3.0408315E-2,-1.4847462E-1,1.815958E-1,6.0872077E-3,9.200379E-3,-1.7987111E-1,-2.8100738E-1,-1.3232614E-1,3.2700348E-4,1.6044389E-1,3.4528743E-2,-9.6760385E-2,-2.6977416E-2,-8.421825E-3,4.2372528E-3,-4.8684765E-2,1.1485466E-3,1.0184969E-2,-7.448083E-4,8.21686E-2,2.517561E-3,-3.1785623E-3,8.930909E-2,2.1092835E-1,8.1573926E-2,-6.0715556E-2,-8.366747E-3,5.2759573E-2,-5.2952934E-2,3.8294777E-2,3.012146E-3,1.1608789E-2,-3.692701E-2,4.9761776E-3,-2.8333622E-3,6.229865E-2,4.724168E-2,-6.886622E-2,1.2841727E-1,-5.089281E-3,2.1466269E-1,8.248892E-2,-1.20662235E-1,-4.6268754E-2,5.9449945E-2,-1.0701296E-1,-1.9787839E-3,-1.5273121E-2,-1.557299E-1,-7.6074447E-3,-4.781768E-3,-8.022749E-4,-2.1258226E-2,-3.269728E-3,9.492495E-3,7.64693E-4,-4.9003833E-3,1.0123922E-3,-9.302518E-4,4.735661E-3,-1.0089959E-2,-2.2325157E-3,-1.4146796E-2,-6.387138E-4,-4.171825E-3,-1.2422058E-2,1.0574118E-2,-1.172435E-3,1.0930947E-2,1.5056987E-4,-1.6413188E-2,-3.3155114E-3,4.2325724E-3,-3.6755733E-3,4.733069E-3,-2.8991597E-3,4.815621E-3,-6.3528023E-3,5.209341E-3,1.0398289E-2,-1.893382E-3,5.424574E-3,3.225199E-3,-4.2555737E-3,3.477904E-3,-4.4718264E-3,-5.685123E-3,6.1180774E-4,2.4558231E-3,-1.6825502E-3,-2.586738E-3,1.6980778E-4,6.2875752E-3,1.1898121E-3,3.2350505E-3,-2.6025842E-3,-2.2171634E-4,-6.783673E-3,8.627686E-3,1.8654122E-3,1.1546434E-2,5.1748278E-3,8.432294E-3,2.6708269E-3,-3.7787082E-3,-8.516259E-3,3.050215E-3,-3.0924198E-3,-1.6763483E-4,5.728749E-3,-1.21650715E-4,-8.201269E-3,6.976625E-4,-9.297493E-3,-3.9634756E-3,1.6307974E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,53,55,-1,57,59,-1,-1,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,101,103,105,-1,107,109,-1,111,113,-1,-1,-1,-1,115,-1,117,119,-1,121,123,125,127,129,131,133,135,-1,137,139,141,143,-1,-1,-1,-1,-1,-1,145,147,149,151,153,155,157,159,-1,161,163,165,-1,-1,-1,167,-1,-1,-1,169,-1,-1,171,173,175,177,-1,179,181,183,-1,-1,185,-1,-1,187,189,191,193,-1,195,197,199,201,203,205,-1,-1,207,209,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6560037E0,1.1045614E0,3.1920319E0,7.157583E-1,2.3764914E-1,1.0583305E0,1.0686569E0,7.6115286E-1,7.2451055E-1,2.6888296E-1,0E0,9.005945E-1,8.5557246E-1,4.6028876E-1,3.3328432E-1,5.8144104E-1,5.99061E-1,1.9396996E-1,2.1900475E-1,1.9298756E-1,2.0685449E-1,5.2524114E-1,3.13639E-1,0E0,4.5624495E-1,3.706993E-1,2.9404628E-1,1.6419433E-1,1.2623447E-1,5.010565E-1,0E0,7.856133E-1,3.094529E-1,0E0,0E0,3.3261198E-1,2.3896347E-1,1.6732207E-1,2.5296152E-1,8.665238E-2,1.2446909E-1,5.220709E-1,6.559541E-1,3.9224142E-1,2.6429266E-1,2.818944E-1,4.4865108E-1,2.826143E-1,3.7866879E-1,7.0892215E-2,0E0,8.332099E-2,1.12636805E-1,0E0,0E0,3.4610653E-1,5.264335E-1,4.1861805E-1,4.4770694E-1,1.0739787E-1,2.4928975E-1,0E0,3.0234408E-1,1.3262321E-1,0E0,1.4237893E-1,1.5157536E-1,0E0,0E0,0E0,0E0,1.053167E-1,0E0,4.9717808E-1,2.0042706E-1,0E0,1.9518429E-1,1.9478343E-1,1.1471039E-1,1.6455999E-1,1.5292624E-1,1.8095233E-1,3.4193867E-1,4.7576118E-1,0E0,9.812999E-2,2.4732682E-1,1.22143626E-1,2.3554352E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.1017032E-1,6.271275E-1,8.893126E-2,2.2382782E-1,1.5539636E-1,7.320851E-2,2.3072529E-1,5.05517E-1,0E0,1.5591472E-1,1.4769088E-1,2.2357458E-1,0E0,0E0,0E0,9.5382646E-2,0E0,0E0,0E0,1.14644825E-1,0E0,0E0,3.8702226E-1,7.2416306E-2,8.9951806E-2,1.0330822E-1,0E0,1.0237825E-1,1.3201788E-1,2.186417E-1,0E0,0E0,1.5639886E-1,0E0,0E0,1.5357214E-1,7.764544E-2,1.2875532E-1,2.3606116E-1,0E0,1.2887132E-1,2.3281431E-1,7.461727E-2,1.047691E-1,1.4874992E-1,9.232333E-2,0E0,0E0,2.3249745E-1,6.941577E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,52,52,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,65,65,66,66,71,71,73,73,74,74,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,104,104,105,105,106,106,110,110,114,114,117,117,118,118,119,119,120,120,122,122,123,123,124,124,127,127,130,130,131,131,132,132,133,133,135,135,136,136,137,137,138,138,139,139,140,140,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,54,56,-1,58,60,-1,-1,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,102,104,106,-1,108,110,-1,112,114,-1,-1,-1,-1,116,-1,118,120,-1,122,124,126,128,130,132,134,136,-1,138,140,142,144,-1,-1,-1,-1,-1,-1,146,148,150,152,154,156,158,160,-1,162,164,166,-1,-1,-1,168,-1,-1,-1,170,-1,-1,172,174,176,178,-1,180,182,184,-1,-1,186,-1,-1,188,190,192,194,-1,196,198,200,202,204,206,-1,-1,208,210,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7311627E2,8.397659E6,4.0844156E7,9.669789E0,1E0,1.5445492E3,7.418546E1,2.642E4,2E0,3.68E2,-8.986996E-3,8E0,6.7388856E7,1.2427474E8,2.4347392E-1,3.684685E2,2.074E3,3.9595376E7,1.646E3,1E0,1.322E3,2.878424E5,6.952967E6,1.6846415E-2,2.3421426E-1,1.2199979E10,4.4839956E3,2.4748051E2,2.7753594E7,8.46E2,-2.0887626E-2,1.8058888E0,1.10597E5,-2.1191524E-2,-3.392834E-3,3.9552E2,2.2E1,2.4946861E4,2.3140822E8,2.5E1,4.1891047E5,6.1114804E-8,1.3E0,3.771981E6,5.460954E9,1.568E3,4.42384E5,6.938E3,4E0,4.551684E6,2.2749973E-3,5.5544252E7,3.5551153E5,1.2263801E-2,1.0089535E-3,1.216918E7,2.227328E4,4.4705097E5,8.819892E2,1E0,2.5853658E0,-4.007125E-3,2.1934064E7,3.29E2,-6.6089844E-3,1.6633E4,2.057307E6,1.3788728E-2,6.67307E-4,-1.4942675E-3,-1.0877957E-2,1.5660624E1,5.7954113E-3,2.37E2,1.752E3,-1.8584367E-2,2.5040618E7,7.857143E0,5.1148495E6,1E0,4.878616E2,6.722251E1,2.6085916E7,1.4307411E7,1.8345295E-2,9.414044E2,8.996408E0,8.91E2,2.4519731E3,-1.0980773E-2,-4.009327E-3,1.1468306E-2,1.0010585E-4,3.015265E-4,-8.7421145E-3,3.79E2,1.2673605E7,3E0,7.661245E6,1.6316416E6,8.747313E6,5.7417827E0,9.112E3,3.2700348E-4,2.6281825E-1,9.83E2,1.3797468E0,-2.6977416E-2,-8.421825E-3,4.2372528E-3,1.3E1,1.1485466E-3,1.0184969E-2,-7.448083E-4,6.952967E6,2.517561E-3,-3.1785623E-3,1.241112E8,4.7436612E2,2.9883E4,1.01644066E2,-8.366747E-3,1.51529E3,1.2E2,6.722251E1,3.012146E-3,1.1608789E-2,1.645614E6,4.9761776E-3,-2.8333622E-3,2.0237808E8,3.6930276E3,4.4E1,1.4072216E-1,-5.089281E-3,2.1351435E0,2.2579439E0,2.8489833E11,1.1E1,3.6873734E5,1.1392E4,-1.9787839E-3,-1.5273121E-2,3.4015749E0,1.602114E6,-4.781768E-3,-8.022749E-4,-2.1258226E-2,-3.269728E-3,9.492495E-3,7.64693E-4,-4.9003833E-3,1.0123922E-3,-9.302518E-4,4.735661E-3,-1.0089959E-2,-2.2325157E-3,-1.4146796E-2,-6.387138E-4,-4.171825E-3,-1.2422058E-2,1.0574118E-2,-1.172435E-3,1.0930947E-2,1.5056987E-4,-1.6413188E-2,-3.3155114E-3,4.2325724E-3,-3.6755733E-3,4.733069E-3,-2.8991597E-3,4.815621E-3,-6.3528023E-3,5.209341E-3,1.0398289E-2,-1.893382E-3,5.424574E-3,3.225199E-3,-4.2555737E-3,3.477904E-3,-4.4718264E-3,-5.685123E-3,6.1180774E-4,2.4558231E-3,-1.6825502E-3,-2.586738E-3,1.6980778E-4,6.2875752E-3,1.1898121E-3,3.2350505E-3,-2.6025842E-3,-2.2171634E-4,-6.783673E-3,8.627686E-3,1.8654122E-3,1.1546434E-2,5.1748278E-3,8.432294E-3,2.6708269E-3,-3.7787082E-3,-8.516259E-3,3.050215E-3,-3.0924198E-3,-1.6763483E-4,5.728749E-3,-1.21650715E-4,-8.201269E-3,6.976625E-4,-9.297493E-3,-3.9634756E-3,1.6307974E-3],"split_indices":[51,9,44,53,79,51,55,28,16,0,0,16,7,44,37,51,2,7,2,108,2,32,44,0,37,5,51,32,46,2,0,53,28,0,0,4,57,49,7,8,46,36,53,1,5,0,2,2,8,28,0,44,32,0,0,44,49,32,47,8,53,0,5,0,0,1,12,0,0,0,0,55,0,0,28,0,5,55,44,94,51,57,28,50,0,51,56,0,4,0,0,0,0,0,0,1,44,11,5,27,44,53,10,0,37,0,52,0,0,0,10,0,0,0,44,0,0,12,4,1,32,0,4,0,57,0,0,9,0,0,7,54,8,37,0,40,53,30,3,27,2,0,0,55,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.207E3,9.73E2,1.234E3,8.42E2,1.31E2,9.55E2,2.79E2,7.56E2,8.6E1,1.28E2,3E0,7E2,2.55E2,2.15E2,6.4E1,4.91E2,2.65E2,9E0,7.7E1,8.8E1,4E1,2.17E2,4.83E2,1.1E1,2.44E2,1.75E2,4E1,5.4E1,1E1,4.89E2,2E0,1.86E2,7.9E1,7E0,2E0,3.2E1,4.5E1,7.6E1,1.2E1,8E0,3.2E1,1.86E2,3.1E1,2.2E2,2.63E2,1.02E2,1.42E2,1.19E2,5.6E1,3.6E1,4E0,5E0,4.9E1,7E0,3E0,3.52E2,1.37E2,6.7E1,1.19E2,1.8E1,6.1E1,2.1E1,1.1E1,2.7E1,1.8E1,1.5E1,6.1E1,7E0,5E0,4E0,4E0,2.5E1,7E0,1.48E2,3.8E1,3E0,2.8E1,2.05E2,1.5E1,1.96E2,6.7E1,5.2E1,5E1,1.37E2,5E0,7.4E1,4.5E1,1.2E1,4.4E1,3.1E1,5E0,3E0,2E0,4.7E1,2E0,3.26E2,2.6E1,1.9E1,1.18E2,5.3E1,1.4E1,2.7E1,9.2E1,8E0,1E1,2.2E1,3.9E1,2E0,9E0,9E0,1.8E1,6E0,9E0,2.9E1,3.2E1,1.1E1,1.4E1,9.4E1,5.4E1,1.7E1,2.1E1,3E0,2.5E1,2.6E1,1.79E2,6E0,9E0,1.89E2,7E0,1.1E1,5.6E1,2.8E1,2.4E1,4.4E1,6E0,3.8E1,9.9E1,3.5E1,3.9E1,3.4E1,1.1E1,2E0,1E1,2.6E1,1.8E1,4.6E1,2.8E2,4E0,2.2E1,1.6E1,3E0,1.4E1,1.04E2,4.1E1,1.2E1,1E1,4E0,2.4E1,3E0,7.3E1,1.9E1,7E0,3E0,2E0,2E1,2E0,3.7E1,3E0,1.5E1,2.8E1,4E0,8.8E1,6E0,1E1,4.4E1,4E0,1.3E1,4E0,1.7E1,2.2E1,3E0,1.2E1,1.4E1,1.48E2,3.1E1,1.26E2,6.3E1,1.7E1,3.9E1,2.3E1,5E0,1.4E1,1E1,2.5E1,1.9E1,2.6E1,1.2E1,1.7E1,8.2E1,2.4E1,1.1E1,6E0,3.3E1,1.8E1,1.6E1,5E0,6E0,6E0,2E1,6E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"211","size_leaf_vector":"1"}},{"base_weights":[2.7840398E-3,2.3784393E-2,-5.628875E-2,-2.4886202E-2,4.8031136E-2,-9.0653E-2,-4.795526E-3,9.919745E-3,-6.539439E-2,-8.759549E-3,6.479611E-2,-7.254651E-2,-1.8409327E-1,-2.363545E-2,8.189259E-2,-9.085871E-3,1.651253E-2,-4.902445E-2,-1.7254566E-1,-2.5911188E-2,1.2815385E-1,4.3760784E-2,1.072377E-1,-1.9958952E-1,-5.789053E-2,-3.1040773E-1,-1.345978E-1,-8.9357095E-3,-1.7965017E-2,1.2993126E-1,4.0449787E-2,2.6652722E-2,-9.529509E-2,2.2993548E-2,-6.715959E-2,-3.752611E-1,-1.1359922E-1,9.0250615E-3,-3.6316022E-2,8.0575235E-2,1.1743109E-2,3.379515E-2,1.2005287E-1,1.1460954E-1,-1.9558378E-1,-3.8090914E-3,-2.620039E-1,8.180952E-3,-6.1312158E-2,-7.708246E-4,-1.881333E-2,-1.7431224E-2,-9.81059E-2,-5.2249372E-2,3.7653528E-3,8.124331E-3,1.7844441E-3,-1.5974873E-3,3.6455689E-3,3.4347523E-2,-1.3715914E-1,-1.5099735E-2,-2.647547E-3,-1.5530275E-2,1.4347902E-1,-1.2410529E-2,-9.0018794E-2,-3.324152E-3,-2.179088E-2,3.3465256E-3,-1.5957646E-1,-7.5425975E-2,-8.402514E-3,7.104177E-3,-4.712884E-2,2.8853921E-2,2.1744196E-1,-2.6249269E-3,1.2960243E-1,2.6706958E-1,1.0150175E-1,-5.4946815E-4,-1.2067108E-2,-1.3339902E-2,-2.9846549E-3,-2.8996537E-2,-8.6008E-2,-4.7010297E-4,-6.4880373E-3,-9.152849E-2,8.1767264E-4,1.8277077E-2,-4.6753697E-3,3.8664456E-2,-1.100015E-2,-1.015412E-2,-5.729987E-4,-1.3972309E-1,3.9735775E-2,8.5708285E-3,-1.8943227E-3,-5.910333E-2,6.155485E-2,-1.5906624E-1,-4.2786915E-2,-1.856475E-3,-1.0798285E-2,-9.9648535E-2,2.8722205E-3,-8.3558016E-2,4.8970883E-3,2.5078433E-3,-6.3485187E-3,3.1863168E-2,-1.8333615E-1,1.3909221E-2,2.7840282E-3,1.5943068E-1,8.216983E-2,1.268336E-1,1.7341122E-2,9.245467E-2,1.5234929E-2,-1.1142316E-2,-1.6822639E-1,-1.0990774E-1,-3.0623633E-2,-5.664201E-2,-8.536881E-3,5.5055365E-2,-1.611536E-2,-1.580707E-3,2.4535137E-3,-8.717088E-3,2.9566057E-3,2.60531E-3,-4.982415E-3,-4.229169E-3,8.1528455E-4,5.315159E-3,-5.258459E-4,-1.8353894E-3,-9.448904E-3,3.997951E-3,-3.0196928E-3,-6.6432483E-3,-2.0492082E-3,6.1617084E-3,-1.1466331E-3,2.481809E-3,-5.1127663E-3,1.5948896E-3,-1.5944341E-3,2.859754E-3,6.161838E-4,-1.4672903E-2,1.4728804E-4,3.152806E-3,9.043265E-3,4.553641E-3,-4.1878056E-3,-4.0026493E-3,9.896929E-3,5.7358644E-3,1.7201916E-3,-4.5128823E-3,2.300805E-4,-1.8764015E-2,-2.395549E-3,-7.5706155E-3,-2.481348E-3,-2.4280918E-3,5.9624896E-3,-3.7270822E-3,8.31255E-4,5.176742E-3,1.1943092E-3,-2.5747886E-3,1.6841553E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,81,-1,83,-1,85,-1,-1,-1,87,89,91,-1,-1,-1,-1,93,95,-1,-1,97,99,101,103,-1,-1,-1,105,107,109,-1,111,113,115,-1,117,119,121,-1,-1,-1,-1,123,125,-1,-1,127,-1,129,-1,131,-1,-1,-1,133,135,-1,-1,137,139,141,143,-1,-1,145,147,149,151,-1,-1,153,155,-1,-1,157,159,161,-1,163,-1,165,167,169,171,173,-1,175,177,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7528775E0,1.9330246E0,1.0294297E0,7.690582E-1,1.0412068E0,5.771091E-1,3.8410378E-1,4.115275E-1,4.3356395E-1,5.903928E-1,7.4880767E-1,5.35856E-1,3.083315E-1,1.8658802E-1,7.836029E-2,0E0,3.2679257E-1,2.8887677E-1,3.5169923E-1,5.298925E-1,1.5391865E-1,4.262718E-1,6.4524174E-1,1.9413853E-1,2.2741568E-1,4.556024E-1,3.4603447E-1,0E0,1.4025894E-1,7.808754E-2,7.56786E-2,3.3622244E-1,1.9099148E-1,2.1109852E-1,2.1915025E-1,1.8873143E-1,2.4457571E-1,0E0,2.3298913E-1,2.2187197E-1,0E0,4.492616E-1,1.1802238E-1,5.184591E-1,8.787629E-2,0E0,1.02193475E-1,0E0,2.0659822E-1,0E0,0E0,0E0,1.4918485E-1,2.0325804E-1,1.8206382E-1,0E0,0E0,0E0,0E0,3.1107762E-1,1.2732561E-1,0E0,0E0,2.4577963E-1,1.01349294E-1,1.8668157E-1,3.975879E-1,0E0,0E0,0E0,1.9612277E-1,1.6908216E-1,1.2635101E-1,0E0,9.3666196E-2,3.1872272E-1,1.6312534E-1,0E0,7.4619174E-2,2.9290736E-1,5.0679064E-1,0E0,0E0,0E0,0E0,2.8399727E-1,1.9253755E-1,0E0,0E0,1.4841962E-1,0E0,1.3117144E-1,0E0,2.8638503E-1,0E0,0E0,0E0,1.3169426E-1,7.302399E-2,0E0,0E0,8.898852E-2,8.717303E-2,2.8571534E-1,2.4098489E-1,0E0,0E0,1.6749424E-1,8.6096786E-2,8.341229E-2,1.3264343E-1,0E0,0E0,2.7568048E-1,1.8489859E-1,0E0,0E0,1.2002832E-1,9.2910156E-2,2.366017E-1,0E0,4.5200562E-1,0E0,1.4846104E-1,3.4718743E-1,3.1608975E-1,1.7885523E-1,7.085224E-2,0E0,8.1989005E-2,1.19451426E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,41,41,42,42,43,43,44,44,46,46,48,48,52,52,53,53,54,54,59,59,60,60,63,63,64,64,65,65,66,66,70,70,71,71,72,72,74,74,75,75,76,76,78,78,79,79,80,80,85,85,86,86,89,89,91,91,93,93,97,97,98,98,101,101,102,102,103,103,104,104,107,107,108,108,109,109,110,110,113,113,114,114,117,117,118,118,119,119,121,121,123,123,124,124,125,125,126,126,127,127,129,129,130,130],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,82,-1,84,-1,86,-1,-1,-1,88,90,92,-1,-1,-1,-1,94,96,-1,-1,98,100,102,104,-1,-1,-1,106,108,110,-1,112,114,116,-1,118,120,122,-1,-1,-1,-1,124,126,-1,-1,128,-1,130,-1,132,-1,-1,-1,134,136,-1,-1,138,140,142,144,-1,-1,146,148,150,152,-1,-1,154,156,-1,-1,158,160,162,-1,164,-1,166,168,170,172,174,-1,176,178,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6204434E-2,2.5275471E2,1E0,2.6725664E0,7.57657E4,4.7022E4,7.06699E8,6E0,4.3978744E0,1.6195753E9,1.0989723E3,1.381869E2,1.7314286E0,5.2143492E-5,9.3915343E-1,-9.085871E-3,1E0,1.1978022E0,2.5789E4,8E0,1.4100353E0,2.9251662E6,1.2367184E3,9.318E3,4.107E3,1.6093831E0,1.5166431E5,-8.9357095E-3,3.6360288E8,1.903E3,3.16617E5,6.2233735E-6,9.232009E2,6.4166665E0,1.13E2,4E0,2.739E3,9.0250615E-3,1.3581437E8,4.8810664E4,1.1743109E-2,1.3238013E2,1.7148E4,1.1485E4,2.1325744E7,-3.8090914E-3,2.1907706E12,8.180952E-3,6.895858E-1,-7.708246E-4,-1.881333E-2,-1.7431224E-2,1.9826213E0,2.4347392E-1,7.218466E7,8.124331E-3,1.7844441E-3,-1.5974873E-3,3.6455689E-3,3.3768854E2,3.4814104E7,-1.5099735E-2,-2.647547E-3,1.141E4,4.382419E6,6E0,9.76E2,-3.324152E-3,-2.179088E-2,3.3465256E-3,3.7057E5,9.34731E5,2.308943E0,7.104177E-3,1.1392E4,1.6218617E7,3.7974813E8,-2.6249269E-3,4.0501267E8,5.841568E1,1.9013363E4,-5.4946815E-4,-1.2067108E-2,-1.3339902E-2,-2.9846549E-3,5.099062E5,1.3565588E6,-4.7010297E-4,-6.4880373E-3,4.4844124E-1,8.1767264E-4,1.7268195E6,-4.6753697E-3,3.3151623E1,-1.100015E-2,-1.015412E-2,-5.729987E-4,5.904E3,2.3446269E2,8.5708285E-3,-1.8943227E-3,6.5E1,1.2E1,5.3134368E8,2E0,-1.856475E-3,-1.0798285E-2,7.6588124E-2,2.3432E4,1E0,2.7183437E1,2.5078433E-3,-6.3485187E-3,1.3E1,3.564E3,1.3909221E-2,2.7840282E-3,3.7934E4,6.5321615E6,3.0324321E-2,1.7341122E-2,1.7384394E7,1.5234929E-2,2.797723E-3,1.7801818E-1,2.1267729E-1,4.5162E4,7.43361E0,-8.536881E-3,5.432393E-1,1.7488463E3,-1.580707E-3,2.4535137E-3,-8.717088E-3,2.9566057E-3,2.60531E-3,-4.982415E-3,-4.229169E-3,8.1528455E-4,5.315159E-3,-5.258459E-4,-1.8353894E-3,-9.448904E-3,3.997951E-3,-3.0196928E-3,-6.6432483E-3,-2.0492082E-3,6.1617084E-3,-1.1466331E-3,2.481809E-3,-5.1127663E-3,1.5948896E-3,-1.5944341E-3,2.859754E-3,6.161838E-4,-1.4672903E-2,1.4728804E-4,3.152806E-3,9.043265E-3,4.553641E-3,-4.1878056E-3,-4.0026493E-3,9.896929E-3,5.7358644E-3,1.7201916E-3,-4.5128823E-3,2.300805E-4,-1.8764015E-2,-2.395549E-3,-7.5706155E-3,-2.481348E-3,-2.4280918E-3,5.9624896E-3,-3.7270822E-3,8.31255E-4,5.176742E-3,1.1943092E-3,-2.5747886E-3,1.6841553E-3],"split_indices":[26,51,109,55,27,9,7,51,52,7,51,46,52,41,56,0,112,52,1,17,41,27,57,9,12,53,27,0,5,0,28,36,32,55,10,8,28,0,7,27,0,55,9,9,1,0,30,0,26,0,0,0,55,37,44,0,0,0,0,54,7,0,0,9,1,8,10,0,0,0,9,1,53,0,2,49,12,0,7,55,51,0,0,0,0,32,27,0,0,56,0,50,0,54,0,0,0,0,51,0,0,10,8,30,16,0,0,41,9,8,52,0,0,3,2,0,0,1,46,37,0,50,0,38,38,38,10,55,0,26,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.217E3,1.636E3,5.81E2,5.44E2,1.092E3,3.48E2,2.33E2,2.93E2,2.51E2,2.49E2,8.43E2,2.93E2,5.5E1,1.92E2,4.1E1,8E0,2.85E2,2.19E2,3.2E1,2.22E2,2.7E1,5.65E2,2.78E2,2.9E1,2.64E2,1.4E1,4.1E1,5E0,1.87E2,1.8E1,2.3E1,2.62E2,2.3E1,4.4E1,1.75E2,6E0,2.6E1,9E0,2.13E2,2.1E1,6E0,5.01E2,6.4E1,2.72E2,6E0,1.1E1,1.8E1,3E0,2.61E2,4E0,1E1,4E0,3.7E1,7.2E1,1.15E2,1.1E1,7E0,8E0,1.5E1,2.51E2,1.1E1,2E0,2.1E1,3.4E1,1E1,5.2E1,1.23E2,2E0,4E0,5E0,2.1E1,8.8E1,1.25E2,1.3E1,8E0,4.89E2,1.2E1,3E0,6.1E1,2E1,2.52E2,2E0,4E0,1.5E1,3E0,1.14E2,1.47E2,1.3E1,2.4E1,4.6E1,2.6E1,1.02E2,1.3E1,2.48E2,3E0,6E0,5E0,1E1,2.4E1,8E0,2E0,3.2E1,2E1,4.9E1,7.4E1,9E0,1.2E1,6.7E1,2.1E1,1.8E1,1.07E2,4E0,4E0,4.83E2,6E0,7E0,5E0,3.6E1,2.5E1,1E1,1E1,2.44E2,8E0,1.02E2,1.2E1,1.02E2,4.5E1,3.5E1,1.1E1,4.9E1,5.3E1,4.3E1,2.05E2,8E0,2E0,2.2E1,2E0,2.2E1,1E1,1.1E1,9E0,1.5E1,3.4E1,1.1E1,6.3E1,3.5E1,3.2E1,3E0,1.8E1,3E0,1.5E1,6.1E1,4.6E1,1.77E2,3.06E2,3E0,3E0,1.2E1,2.4E1,2.3E1,2E0,3E0,7E0,1.49E2,9.5E1,1.5E1,8.7E1,3E0,9E0,4.9E1,5.3E1,4E1,5E0,2.6E1,9E0,1.5E1,3.4E1,3E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"179","size_leaf_vector":"1"}},{"base_weights":[-3.428958E-3,-4.3598868E-2,2.1174334E-2,-5.316877E-2,9.1816224E-2,-1.3126392E-2,4.8737757E-2,2.671062E-1,-5.6324784E-2,2.412351E-1,-2.0457576E-3,2.7986374E-2,-3.568557E-2,8.522843E-2,9.994357E-3,2.470171E-3,1.4888852E-2,-5.109132E-2,-2.467824E-1,1.492516E-1,1.8880185E-2,2.1445978E-2,-1.2115468E-2,3.5881396E-2,-2.6221463E-1,-1.2712023E-3,-5.2760586E-2,1.1183401E-1,3.0976357E-2,-1.166967E-2,1.20599225E-1,-7.3416136E-2,-1.4716311E-2,-1.5841001E-1,-2.254154E-2,1.2795128E-2,2.849507E-3,8.795826E-2,-8.525788E-4,1.9812956E-2,1.5316059E-1,2.1898956E-3,-2.6662845E-2,-2.5935804E-2,6.727689E-2,-1.2166322E-1,-3.031993E-2,1.23167515E-1,-3.48598E-2,9.767076E-2,-3.0443897E-2,-3.152725E-2,4.5623735E-2,8.486341E-2,2.1803573E-1,2.4186434E-2,-8.636408E-2,9.840477E-2,-4.152125E-2,-2.129544E-1,6.830667E-4,6.5811286E-3,-1.4922648E-3,5.371597E-2,-2.6321767E-2,2.0788842E-1,2.6998103E-3,1.4505099E-2,-6.0654074E-2,-3.4500281E-3,1.0453776E-1,-1.05015956E-1,-1.6001709E-2,2.4835698E-2,-4.426027E-2,8.8501334E-2,1.7308922E-1,2.664024E-3,-4.628419E-3,3.0003956E-3,7.39932E-3,-1.1053454E-1,2.5581637E-2,-2.0441804E-2,-1.0994953E-1,-5.0223204E-3,5.297308E-2,1.08099714E-1,-1.3218585E-2,4.443862E-3,1.817649E-2,-5.9044678E-2,1.1249883E-1,-7.68554E-2,-2.1809925E-1,1.1813221E-1,-2.688437E-3,-9.377955E-3,-3.266133E-2,-1.1102978E-2,-1.0679908E-3,6.156387E-2,-7.769301E-3,2.7785802E-3,-1.5648855E-1,2.6513548E-3,1.1444509E-2,-7.305636E-3,4.417794E-2,-5.8240215E-3,-3.1207351E-2,1.8196827E-1,3.0252455E-2,-2.7953738E-3,-1.6422118E-1,-2.9696326E-3,7.2392404E-2,-1.137873E-2,-7.949511E-2,1.2620474E-1,4.414741E-2,2.3545368E-1,9.932319E-2,-2.3057799E-1,-1.2234572E-2,5.2286692E-2,-5.22521E-3,-9.9493444E-2,-1.05567E-2,-9.2726825E-3,-2.900166E-3,5.597717E-3,3.160218E-2,2.2686895E-3,1.6644731E-1,-6.9285207E-3,-8.108581E-4,1.268491E-2,3.2994545E-3,3.5560832E-3,-3.7843403E-3,-1.54373E-2,-4.0237196E-3,6.7064096E-3,8.626099E-4,-4.2968744E-4,-3.9049897E-3,-2.3746716E-3,3.6453402E-3,-1.1589016E-3,5.6502083E-3,-1.2061038E-2,-2.4272837E-3,4.36785E-3,-1.9724749E-3,2.1460634E-3,-2.8876816E-3,2.9833252E-3,1.08264405E-2,-1.9306564E-3,4.151069E-3,-1.1049439E-2,-2.970487E-3,-1.8567311E-3,4.2105573E-3,2.9847375E-3,-1.9984744E-3,-1.0433325E-2,-2.5713185E-3,7.253912E-3,5.919242E-6,4.803137E-3,6.988145E-4,2.8143849E-3,1.1581976E-2,1.0356461E-2,3.0659E-3,-1.6465012E-2,-4.6149013E-3,2.8963469E-3,-3.9426163E-3,8.47785E-3,1.1081614E-3,-1.0313552E-3,-8.199545E-3,2.1210506E-3,-1.3224526E-3,2.1398724E-4,3.633335E-3,9.534652E-3,3.5889843E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,-1,37,-1,39,41,43,45,47,49,51,53,55,57,59,-1,-1,-1,61,-1,63,65,-1,-1,67,69,71,73,75,77,79,81,83,85,87,89,91,93,95,97,99,-1,-1,-1,101,103,105,-1,107,109,-1,111,113,-1,115,117,119,121,-1,-1,-1,-1,123,125,127,129,-1,131,133,-1,-1,-1,135,137,139,141,143,-1,-1,145,-1,-1,147,-1,149,151,-1,-1,-1,153,-1,155,157,159,-1,161,-1,163,165,167,169,171,173,175,177,179,181,-1,183,185,-1,-1,-1,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2325315E0,1.1167718E0,1.3252089E0,8.2570004E-1,7.9990435E-1,5.8066756E-1,1.0976496E0,8.706349E-2,7.798102E-1,2.9348576E-1,2.3032187E-1,5.1804245E-1,2.3733377E-1,5.738988E-1,9.0715086E-1,0E0,0E0,6.2827015E-1,3.8714325E-1,1.6638798E-1,0E0,9.274071E-2,0E0,4.0590054E-1,6.3730836E-1,2.299909E-1,4.1379064E-1,4.4985843E-1,5.47793E-1,3.6209202E-1,1.9737911E-1,6.0869026E-1,9.0161824E-1,1.640892E-1,0E0,0E0,0E0,9.5027514E-2,0E0,3.0172226E-1,1.20685995E-1,0E0,0E0,1.4128363E-1,2.0042935E-1,2.1754265E-1,1.5872136E-1,4.1559958E-1,1.3227277E-1,1.21322155E-1,3.1692612E-1,2.0291808E-1,1.0022403E-1,4.4310257E-1,3.1245202E-1,4.2561173E-1,5.1435256E-1,1.8443996E-1,3.4539893E-1,7.3693514E-2,0E0,0E0,0E0,2.0684698E-1,3.1104252E-1,9.246051E-2,0E0,2.524015E-1,1.0409081E-1,0E0,1.6130155E-1,1.5126354E-1,0E0,1.8536808E-1,1.8910187E-1,2.4360025E-1,4.4125223E-1,0E0,0E0,0E0,0E0,3.3938673E-1,1.641761E-1,1.6148788E-1,1.1312875E-1,0E0,1.1540142E-1,1.4391881E-1,0E0,0E0,0E0,1.1324748E-1,1.6560718E-1,4.219036E-1,4.1429615E-1,1.4875513E-1,0E0,0E0,2.8353834E-1,0E0,0E0,2.3760974E-1,0E0,2.3864502E-1,1.553269E-1,0E0,0E0,0E0,1.9234893E-1,0E0,1.0289052E-1,7.713467E-2,7.55645E-2,0E0,1.8867528E-1,0E0,6.925492E-2,2.2072631E-1,2.5762224E-1,3.4308088E-1,1.2285894E-1,1.7467761E-1,1.8085226E-1,1.8551964E-1,1.025411E-1,1.3054259E-1,0E0,1.3987787E-1,2.0053934E-1,0E0,0E0,0E0,8.068587E-2,0E0,6.798148E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,37,37,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,63,63,64,64,65,65,67,67,68,68,70,70,71,71,73,73,74,74,75,75,76,76,81,81,82,82,83,83,84,84,86,86,87,87,91,91,92,92,93,93,94,94,95,95,98,98,101,101,103,103,104,104,108,108,110,110,111,111,112,112,114,114,116,116,117,117,118,118,119,119,120,120,121,121,122,122,123,123,124,124,125,125,127,127,128,128,132,132,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,-1,38,-1,40,42,44,46,48,50,52,54,56,58,60,-1,-1,-1,62,-1,64,66,-1,-1,68,70,72,74,76,78,80,82,84,86,88,90,92,94,96,98,100,-1,-1,-1,102,104,106,-1,108,110,-1,112,114,-1,116,118,120,122,-1,-1,-1,-1,124,126,128,130,-1,132,134,-1,-1,-1,136,138,140,142,144,-1,-1,146,-1,-1,148,-1,150,152,-1,-1,-1,154,-1,156,158,160,-1,162,-1,164,166,168,170,172,174,176,178,180,182,-1,184,186,-1,-1,-1,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.3238013E2,3.8637936E2,8.974133E9,4.387838E-1,2E0,6.015825E0,8E0,1.5265896E8,3.832274E2,8.724166E9,2.465E3,5.116E3,2.185321E0,1.5435694E9,2.470171E-3,1.4888852E-2,3.131E3,1.707E4,2.1853803E4,1.8880185E-2,2.2729468E0,-1.2115468E-2,1.8465776E8,1.3642105E1,1.1803029E8,1.074E3,2.02E2,4.0208E4,1.7911884E3,1.5439131E2,1E0,9.912942E6,1.141E4,-2.254154E-2,1.2795128E-2,2.849507E-3,3.0501078E8,-8.525788E-4,1.5182724E0,1.366E3,2.1898956E-3,-2.6662845E-2,1E0,1.13929E7,1.1698891E6,1.48298E5,8.1590717E2,1.0128521E6,3.15856E0,1.365493E6,4.5843E7,8.874974E7,2.62797E5,2.62797E5,2.97E2,9.837297E0,3.4613644E1,8.1057306E2,4.242625E6,6.830667E-4,6.5811286E-3,-1.4922648E-3,1.1644968E1,2.096295E6,6.62E2,2.6998103E-3,8.732036E1,3.6610576E7,-3.4500281E-3,1.6358411E8,2.0081E4,-1.6001709E-2,2.56E2,7.629372E4,2E0,2.6120481E0,2.664024E-3,-4.628419E-3,3.0003956E-3,7.39932E-3,2.98E2,1.8931546E8,1.71E2,6.6404694E2,-5.0223204E-3,1.1900813E1,2.3130558E6,-1.3218585E-2,4.443862E-3,1.817649E-2,9.2E0,4.676E3,5.8365756E-1,6.737088E7,8.2608955E-3,-2.688437E-3,-9.377955E-3,1.775894E10,-1.1102978E-2,-1.0679908E-3,1.654762E2,-7.769301E-3,2.2401E4,2.81E2,2.6513548E-3,1.1444509E-2,-7.305636E-3,1.2868687E1,-5.8240215E-3,4.470588E0,5.9908E6,1.829E3,-2.7953738E-3,2.755479E8,-2.9696326E-3,9.653717E3,2.09633E1,1.003252E5,9.544947E6,3.0996E4,1.1217151E0,6.720137E7,2.0966542E0,1.8648378E7,2.3523688E0,-5.22521E-3,1.6181818E1,3.1909382E-3,-9.2726825E-3,-2.900166E-3,5.597717E-3,1E0,2.2686895E-3,3.94E2,-6.9285207E-3,-8.108581E-4,1.268491E-2,3.2994545E-3,3.5560832E-3,-3.7843403E-3,-1.54373E-2,-4.0237196E-3,6.7064096E-3,8.626099E-4,-4.2968744E-4,-3.9049897E-3,-2.3746716E-3,3.6453402E-3,-1.1589016E-3,5.6502083E-3,-1.2061038E-2,-2.4272837E-3,4.36785E-3,-1.9724749E-3,2.1460634E-3,-2.8876816E-3,2.9833252E-3,1.08264405E-2,-1.9306564E-3,4.151069E-3,-1.1049439E-2,-2.970487E-3,-1.8567311E-3,4.2105573E-3,2.9847375E-3,-1.9984744E-3,-1.0433325E-2,-2.5713185E-3,7.253912E-3,5.919242E-6,4.803137E-3,6.988145E-4,2.8143849E-3,1.1581976E-2,1.0356461E-2,3.0659E-3,-1.6465012E-2,-4.6149013E-3,2.8963469E-3,-3.9426163E-3,8.47785E-3,1.1081614E-3,-1.0313552E-3,-8.199545E-3,2.1210506E-3,-1.3224526E-3,2.1398724E-4,3.633335E-3,9.534652E-3,3.5889843E-3],"split_indices":[109,55,51,18,41,16,52,0,44,55,7,2,28,40,7,0,0,2,9,32,0,52,0,5,57,7,2,8,9,51,57,84,44,9,0,0,0,31,0,52,10,0,0,15,5,46,1,51,46,37,9,44,7,2,2,2,53,55,4,28,0,0,0,55,12,2,0,54,7,0,7,28,0,0,32,16,52,0,0,0,0,0,31,0,32,0,53,47,0,0,0,54,1,53,7,26,0,0,5,0,0,4,0,1,10,0,0,0,57,0,53,9,2,0,5,0,49,57,27,9,9,52,12,53,49,40,0,57,38,0,0,0,79,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.257E3,8.57E2,1.4E3,8.01E2,5.6E1,6.24E2,7.76E2,7E0,7.94E2,2.1E1,3.5E1,2.21E2,4.03E2,3.99E2,3.77E2,2E0,5E0,7.74E2,2E1,1.5E1,6E0,3.3E1,2E0,2.16E2,5E0,1.34E2,2.69E2,2.67E2,1.32E2,3.16E2,6.1E1,4.79E2,2.95E2,1.6E1,4E0,5E0,1E1,1.2E1,2.1E1,1.91E2,2.5E1,3E0,2E0,9.9E1,3.5E1,6.5E1,2.04E2,2.48E2,1.9E1,6.3E1,6.9E1,2.35E2,8.1E1,4.6E1,1.5E1,5.6E1,4.23E2,5.6E1,2.39E2,1.2E1,4E0,8E0,4E0,1.1E2,8.1E1,1.5E1,1E1,4.6E1,5.3E1,7E0,2.8E1,6.2E1,3E0,4.1E1,1.63E2,1.48E2,1E2,8E0,1.1E1,4.4E1,1.9E1,2.8E1,4.1E1,2.07E2,2.8E1,3E0,7.8E1,4.4E1,2E0,1E1,5E0,2.9E1,2.7E1,3.96E2,2.7E1,5E1,6E0,1.1E1,2.28E2,1E1,2E0,1.07E2,3E0,6.7E1,1.4E1,4E0,1.1E1,6E0,4E1,1.5E1,3.8E1,1.3E1,1.5E1,3.7E1,2.5E1,1.4E1,2.7E1,8.5E1,7.8E1,7.9E1,6.9E1,5.3E1,4.7E1,1.2E1,1.6E1,3.5E1,6E0,2.2E1,1.85E2,8E0,2E1,1.7E1,6.1E1,2.3E1,2.1E1,8E0,2.1E1,4E0,2.3E1,1.6E1,3.8E2,1.3E1,1.4E1,3.8E1,1.2E1,1.6E2,6.8E1,1.5E1,9.2E1,5.5E1,1.2E1,6E0,8E0,2.5E1,1.5E1,1.1E1,2.7E1,5E0,8E0,7E0,8E0,1.3E1,1.2E1,4E0,2.3E1,2.5E1,6E1,9E0,6.9E1,6.2E1,1.7E1,2.1E1,4.8E1,6E0,4.7E1,8E0,3.9E1,5E0,7E0,8E0,8E0,5E0,3E1,1.2E1,1E1,4.5E1,1.4E2,4E1,2.1E1,1.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-1.3889718E-3,-3.925648E-2,2.2856528E-2,-5.0839797E-2,7.9571195E-2,-7.0371376E-3,6.048012E-2,-2.7205054E-2,-8.666889E-2,2.517393E-1,2.4767797E-2,-2.0689169E-2,7.37823E-2,3.55837E-2,9.261625E-2,-5.542447E-2,2.3144482E-2,-7.670096E-2,-3.72056E-1,1.3046117E-1,1.9181753E-2,6.817782E-2,-8.393742E-2,9.596091E-2,-2.7539192E-2,6.9828015E-3,1.3797843E-1,8.356415E-2,1.6035927E-2,8.003614E-2,2.2106947E-1,-3.7344526E-2,-1.4190137E-1,9.983386E-2,-5.567168E-2,-9.0899155E-2,1.5118779E-4,-2.0213395E-2,-2.2457575E-3,8.453954E-3,6.4925355E-4,8.965648E-3,1.7416202E-1,-8.691042E-3,-3.4234018E-4,1.2986583E-1,-2.6354434E-3,1.3412568E-1,-3.0542236E-2,2.436711E-2,-9.6011E-3,1.9155315E-1,-1.23356E-3,1.1413351E-1,-6.340899E-2,-7.0357854E-3,2.1206446E-2,-1.03723835E-2,8.508644E-2,2.2334824E-3,1.1612429E-2,9.494317E-2,-4.5700625E-2,-8.457661E-2,-2.620133E-1,1.1886452E-2,6.94903E-2,-1.0047502E-1,2.2723438E-2,-3.5553977E-2,-1.259842E-1,5.0935883E-2,-1.11585215E-1,-4.53703E-2,3.8570887E-3,2.954029E-3,1.0844092E-2,1.9140682E-1,1.2102717E-3,-6.11886E-4,8.655492E-3,-3.260979E-2,7.865105E-3,8.437652E-3,1.0061282E-2,1.5664284E-1,1.6183292E-2,2.5448122E-3,-6.286263E-3,9.510434E-2,1.2275097E-2,1.5041318E-3,-6.9267075E-3,4.250804E-2,-1.9777697E-2,2.2007939E-1,7.459591E-2,1.097928E-2,3.7843075E-2,-8.375222E-3,-3.992594E-2,-4.3233223E-2,-1.8774608E-1,-1.6304448E-2,-1.741513E-3,3.796683E-2,1.2179078E-1,-6.6861965E-2,-1.5965115E-2,9.192751E-3,-3.8486565E-3,-9.081418E-2,3.7239634E-2,-1.6644148E-2,-1.1696919E-1,-1.9044604E-2,1.2648043E-1,-9.01949E-3,-4.5282877E-4,6.12313E-4,-5.950717E-3,1.6647992E-3,1.0547098E-2,-6.1478415E-3,-5.1786225E-2,-5.5086777E-2,4.2320896E-2,8.526561E-3,2.795852E-3,1.5218386E-1,4.823484E-2,-1.865362E-2,6.703216E-2,-4.368109E-3,1.9671109E-2,1.3675801E-2,4.5424504E-3,6.817399E-2,1.2942021E-2,5.110708E-3,-1.4435122E-3,5.6241774E-3,-1.9675656E-3,-3.0118064E-3,4.858801E-3,-2.3062646E-3,-1.4232495E-2,2.2126061E-3,-4.1414974E-3,2.0698616E-3,9.063747E-3,-1.7251562E-3,-8.477201E-3,4.4456897E-3,-2.5966172E-3,-5.441715E-3,1.2193552E-3,-1.5784638E-3,4.8737987E-3,-4.2762393E-3,-8.857414E-3,5.3011504E-4,-8.000627E-3,1.0058693E-2,2.8425585E-3,-1.2509153E-3,1.9728686E-3,-9.99131E-3,-1.8273409E-3,-4.519466E-3,2.8659313E-3,3.6650163E-3,-5.944439E-4,-2.744375E-3,7.981254E-3,-7.1125524E-4,5.573364E-3,2.6998075E-3,-4.511572E-3,5.674451E-3,1.4606016E-3,-3.2134973E-5,6.0849264E-3,3.299856E-3,-8.672673E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,-1,-1,73,75,-1,-1,77,-1,79,81,83,-1,85,87,89,91,-1,93,-1,95,-1,-1,97,99,101,103,-1,105,107,109,111,113,115,117,119,-1,-1,-1,121,-1,-1,-1,123,-1,-1,125,127,-1,-1,-1,129,-1,-1,-1,131,133,135,137,-1,139,-1,141,143,145,-1,-1,147,149,151,-1,-1,153,155,157,-1,159,161,163,-1,-1,-1,-1,-1,-1,165,167,169,171,-1,-1,173,175,177,179,-1,181,-1,-1,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9757663E0,1.1602368E0,1.4760413E0,6.462996E-1,7.0141965E-1,8.091783E-1,4.615476E-1,6.5912926E-1,8.3625746E-1,3.2007778E-1,2.7941865E-1,5.036506E-1,4.5359308E-1,3.072326E-1,3.9114356E-1,4.590596E-1,1.0149664E0,3.219067E-1,2.1779847E-1,7.572833E-2,0E0,2.620118E-1,1.3881122E-1,1.905112E-1,2.9199007E-1,2.069928E-1,4.0298128E-1,4.3406022E-1,2.1160828E-1,3.8170242E-1,1.211822E-1,2.7737495E-1,3.2899463E-1,4.014209E-1,2.938093E-1,4.7902608E-1,2.7238515E-1,0E0,0E0,0E0,0E0,1.1984028E-1,9.115064E-2,0E0,0E0,1.789447E-1,0E0,9.524353E-2,2.5188255E-1,1.14003316E-1,0E0,1.6988027E-1,1.3428518E-1,2.102269E-1,1.522504E-1,0E0,2.0076405E-1,0E0,3.0788088E-1,0E0,0E0,1.1721198E-1,1.8002802E-1,1.4543658E-1,3.3708203E-1,0E0,1.15730554E-1,4.2564064E-1,1.4932048E-1,3.9759567E-1,2.8812408E-1,1.7686805E-1,1.3154125E-1,8.79498E-2,0E0,0E0,0E0,1.0709894E-1,0E0,0E0,0E0,2.9298532E-1,0E0,0E0,1.0110893E-1,9.060776E-2,0E0,0E0,0E0,1.852802E-1,0E0,0E0,0E0,2.2774073E-1,2.4094261E-1,1.2677836E-1,2.7085042E-1,0E0,6.638782E-2,0E0,1.3589314E-1,1.0181512E-1,1.5720683E-1,0E0,0E0,7.081285E-2,1.5189135E-1,1.5903135E-1,0E0,0E0,1.585074E-1,1.969713E-1,2.2206368E-1,0E0,2.4369049E-1,8.9350626E-2,8.357337E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.6183903E-1,6.374978E-1,9.106291E-2,6.961092E-2,0E0,0E0,1.7482638E-1,1.9821906E-1,2.8494415E-1,2.1490106E-1,0E0,1.24026686E-1,0E0,0E0,2.6977384E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,41,41,42,42,45,45,47,47,48,48,49,49,51,51,52,52,53,53,54,54,56,56,58,58,61,61,62,62,63,63,64,64,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,77,77,81,81,84,84,85,85,89,89,93,93,94,94,95,95,96,96,98,98,100,100,101,101,102,102,105,105,106,106,107,107,110,110,111,111,112,112,114,114,115,115,116,116,123,123,124,124,125,125,126,126,129,129,130,130,131,131,132,132,134,134,137,137],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,-1,-1,74,76,-1,-1,78,-1,80,82,84,-1,86,88,90,92,-1,94,-1,96,-1,-1,98,100,102,104,-1,106,108,110,112,114,116,118,120,-1,-1,-1,122,-1,-1,-1,124,-1,-1,126,128,-1,-1,-1,130,-1,-1,-1,132,134,136,138,-1,140,-1,142,144,146,-1,-1,148,150,152,-1,-1,154,156,158,-1,160,162,164,-1,-1,-1,-1,-1,-1,166,168,170,172,-1,-1,174,176,178,180,-1,182,-1,-1,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,8.848604E1,6.067634E5,1.8159722E-1,7.9203456E8,1.7730331E6,1.0989723E3,2.857E3,4.5723195E6,1.1879E4,2.851369E2,1.6E1,6.8670874E9,1.402418E0,5.316494E6,2.6120481E0,1.775044E7,1.1879E4,1.4E1,2.853839E4,1.9181753E-2,8.015419E3,1.2934837E0,6.9676765E1,5.486E1,9.4010997E-1,2E0,7.9715155E6,2.09E2,8.490231E-1,4E0,7E0,8E0,9.3262803E8,1.9701008E5,1.25218E5,1.5963264E9,-2.0213395E-2,-2.2457575E-3,8.453954E-3,6.4925355E-4,1.8552577E2,1.0192391E2,-8.691042E-3,-3.4234018E-4,2.2539302E6,-2.6354434E-3,3.4E1,3.0612E4,1.5792079E0,-9.6011E-3,1.2224265E1,3.4E1,2.836111E1,1.9032773E0,-7.0357854E-3,7.65244E5,-1.03723835E-2,1.0594161E1,2.2334824E-3,1.1612429E-2,1.3050649E4,6E0,1.2469E5,2.43406E5,1.1886452E-2,3.7307863E0,9.496754E6,1.92E2,2.4973914E2,7.07541E1,3.8142116E0,7.203918E9,1.3900395E2,3.8570887E-3,2.954029E-3,1.0844092E-2,5.08E2,1.2102717E-3,-6.11886E-4,8.655492E-3,2.2415E4,7.865105E-3,8.437652E-3,2.8331464E2,9.403226E0,1.6183292E-2,2.5448122E-3,-6.286263E-3,4.020325E0,1.2275097E-2,1.5041318E-3,-6.9267075E-3,2.991942E2,7.420515E-1,2.0784864E0,8.724166E9,1.097928E-2,2E0,-8.375222E-3,2.3E1,7.932662E-5,9.457831E0,-1.6304448E-2,-1.741513E-3,3.8807898E6,2.130881E6,4.8718534E0,-1.5965115E-2,9.192751E-3,2.3907686E7,3.5452E4,1.028E3,-1.6644148E-2,9.6910744E7,5.8E1,9.11898E5,-9.01949E-3,-4.5282877E-4,6.12313E-4,-5.950717E-3,1.6647992E-3,1.0547098E-2,5.9643E4,1.04E3,1.102E3,8.15E2,8.526561E-3,2.795852E-3,5.5E1,5.314225E2,1.0288468E6,1.3E1,-4.368109E-3,6.2105E4,1.3675801E-2,4.5424504E-3,2.62797E5,1.2942021E-2,5.110708E-3,-1.4435122E-3,5.6241774E-3,-1.9675656E-3,-3.0118064E-3,4.858801E-3,-2.3062646E-3,-1.4232495E-2,2.2126061E-3,-4.1414974E-3,2.0698616E-3,9.063747E-3,-1.7251562E-3,-8.477201E-3,4.4456897E-3,-2.5966172E-3,-5.441715E-3,1.2193552E-3,-1.5784638E-3,4.8737987E-3,-4.2762393E-3,-8.857414E-3,5.3011504E-4,-8.000627E-3,1.0058693E-2,2.8425585E-3,-1.2509153E-3,1.9728686E-3,-9.99131E-3,-1.8273409E-3,-4.519466E-3,2.8659313E-3,3.6650163E-3,-5.944439E-4,-2.744375E-3,7.981254E-3,-7.1125524E-4,5.573364E-3,2.6998075E-3,-4.511572E-3,5.674451E-3,1.4606016E-3,-3.2134973E-5,6.0849264E-3,3.299856E-3,-8.672673E-3],"split_indices":[109,55,46,26,12,27,51,2,27,2,57,10,30,52,27,52,44,2,8,32,0,4,40,52,49,26,16,49,28,52,3,3,8,30,32,28,7,0,0,0,0,55,57,0,0,44,0,10,2,57,0,55,3,57,41,0,47,0,55,0,0,27,51,9,1,0,52,46,0,54,51,56,12,57,0,0,0,2,0,0,0,28,0,0,4,57,0,0,0,55,0,0,0,51,38,40,7,0,16,0,2,26,53,0,0,27,1,52,0,0,44,9,2,0,44,3,28,0,0,0,0,0,0,1,2,28,2,0,0,0,4,50,8,0,10,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E3,8.39E2,1.311E3,7.65E2,7.4E1,7.31E2,5.8E2,4.62E2,3.03E2,1.7E1,5.7E1,6.26E2,1.05E2,3.28E2,2.52E2,2.96E2,1.66E2,2.94E2,9E0,1.1E1,6E0,4.1E1,1.6E1,3.4E1,5.92E2,5.2E1,5.3E1,9.4E1,2.34E2,2.31E2,2.1E1,2.46E2,5E1,8.4E1,8.2E1,2.48E2,4.6E1,7E0,2E0,7E0,4E0,2.7E1,1.4E1,6E0,1E1,2.8E1,6E0,1E1,5.82E2,4.9E1,3E0,3.8E1,1.5E1,7.8E1,1.6E1,6E0,2.28E2,3E0,2.28E2,4E0,1.7E1,1.4E1,2.32E2,3.5E1,1.5E1,1.2E1,7.2E1,5.2E1,3E1,9.7E1,1.51E2,3.2E1,1.4E1,1.6E1,1.1E1,6E0,8E0,1.7E1,1.1E1,3E0,7E0,5.77E2,5E0,3E0,4.6E1,3.3E1,5E0,1.1E1,4E0,7.1E1,7E0,8E0,8E0,1.5E2,7.8E1,1.5E1,2.13E2,3E0,1.1E1,8E0,2.24E2,2.6E1,9E0,1E1,5E0,4.6E1,2.6E1,4.7E1,5E0,3E0,2.7E1,5.5E1,4.2E1,4E0,1.47E2,1.7E1,1.5E1,7E0,7E0,1E1,6E0,4E0,1.3E1,2.43E2,3.34E2,1.5E1,3.1E1,2.4E1,9E0,3.1E1,4E1,4.3E1,1.07E2,2.6E1,5.2E1,8E0,7E0,2.08E2,5E0,5E0,6E0,4E0,2.2E2,2.3E1,3E0,5E0,4E0,4.3E1,3E0,1.4E1,1.2E1,3.9E1,8E0,9E0,1.8E1,4.4E1,1.1E1,2.1E1,2.1E1,1.16E2,3.1E1,1.5E1,2E0,5E0,1E1,1.7E2,7.3E1,2E1,3.14E2,1.1E1,4E0,1.8E1,1.3E1,3E0,2.8E1,2.2E1,1.8E1,2.2E1,2.1E1,3.9E1,6.8E1,4.5E1,7E0,2.05E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[2.607636E-3,-4.1336708E-2,2.3150178E-2,-6.664873E-2,-7.3387106E-3,1.1307151E-2,9.966347E-2,-5.3212367E-2,-2.2660202E-1,7.0485524E-3,-1.4838009E-1,1.5592424E-4,6.324899E-2,1.8509028E-2,8.811358E-2,-1.4071275E-2,-4.9587596E-2,-3.7873168E-2,-2.9060018E-1,1.1373961E-2,-2.6660578E-4,-8.943872E-2,-1.582741E-2,-1.4497423E-2,4.211961E-2,2.7507637E-2,1.0898036E-1,-7.7406764E-2,9.932908E-2,-7.722252E-2,-1.9668644E-2,6.3279714E-3,-4.916561E-3,-2.028804E-2,-1.936791E-1,-3.7918273E-2,2.0291872E-2,-1.5430875E-1,2.3307567E-3,-4.08465E-2,6.5552085E-3,8.929376E-2,1.4004659E-2,1.8578647E-3,7.486357E-2,1.7600851E-1,4.7381625E-2,7.914651E-3,-1.3915806E-1,7.390922E-2,1.6076477E-1,-4.174119E-2,-1.2234446E-1,-8.862871E-4,-9.425172E-2,-2.4675292E-1,-2.9019355E-3,3.3785067E-3,-5.3544283E-2,6.514604E-2,-4.497952E-3,-8.668358E-3,2.2749869E-4,-3.3748083E-2,-4.481131E-1,2.0702513E-2,-9.2087716E-2,-2.8309408E-3,1.06110334E-1,6.2290583E-2,-1.2802515E-2,1.6256554E-2,-1.346354E-2,8.661555E-2,-5.543946E-3,2.0453905E-1,6.1447695E-2,-1.285752E-3,8.635838E-2,-2.473924E-3,-1.2377798E-2,1.5446554E-1,2.7549075E-2,2.7381575E-1,8.395156E-2,-1.3026895E-1,-8.02151E-3,-2.5985314E-2,-1.9764878E-1,-5.7617743E-2,2.8079625E-2,-1.7607339E-1,-1.2415581E-3,-3.6467605E-3,-1.3537403E-2,-6.301353E-2,4.9200375E-3,-4.096677E-3,7.4593216E-2,-1.973816E-2,9.638729E-2,-1.272349E-2,-8.3357E-2,-4.523295E-3,-3.4544643E-2,4.0452592E-2,-5.3974032E-2,-5.5208183E-3,-7.435904E-4,1.2608163E-1,3.728546E-4,1.4963622E-3,9.720985E-2,-2.890538E-2,5.7262774E-3,3.19606E-2,-1.0157904E-1,2.918271E-3,7.49834E-3,2.2203566E-1,1.3983822E-5,-3.301526E-3,5.4759854E-3,3.9329432E-2,-9.2509195E-2,9.326011E-3,4.7867365E-2,1.1732735E-1,1.4736244E-2,-1.2966491E-3,5.565322E-2,4.115354E-1,9.283759E-3,1.2606089E-1,-9.0463206E-2,-7.1728714E-3,7.9689035E-3,3.7247401E-3,-1.0688422E-3,-3.5455378E-3,4.960878E-3,-1.8310005E-2,-5.9232963E-3,-1.4127112E-3,-7.13121E-3,3.3311378E-3,-4.6278542E-4,-9.437097E-3,1.5249219E-3,-1.8013548E-3,-6.1339852E-3,2.350183E-3,7.644698E-3,9.851647E-4,-2.754541E-3,8.337387E-3,-2.9860619E-3,-1.4877917E-4,-6.73536E-3,-6.5381285E-3,-7.730732E-4,9.798173E-3,1.5044949E-3,-1.8745117E-3,-1.6805239E-2,2.9639E-3,7.361661E-3,2.1672053E-3,-4.4020726E-3,1.0820817E-2,2.1882185E-3,-1.9499151E-3,3.7166772E-3,2.256361E-3,-1.1964369E-3,6.0344266E-4,-8.167676E-3,1.0949966E-2,3.5904988E-4,3.1363133E-3,-6.0434113E-3,1.5128596E-4,-8.5656205E-3,4.521304E-3,-3.1063373E-3,6.0842587E-3,-3.3537655E-3,3.2059383E-3,-3.848208E-3,6.1481483E-3,2.3336638E-2,3.5530357E-3,9.620969E-3,-1.013665E-2,2.9693034E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,31,33,-1,35,37,-1,39,41,43,45,47,49,51,53,-1,-1,-1,55,57,59,61,-1,63,65,67,69,71,73,75,77,-1,79,81,83,85,87,89,91,93,-1,-1,95,97,99,-1,-1,101,103,105,107,-1,109,111,113,115,-1,117,-1,119,121,123,125,-1,-1,127,129,131,133,135,137,139,141,143,145,147,-1,-1,-1,149,-1,-1,151,153,155,157,159,-1,-1,161,163,-1,-1,165,-1,167,169,171,-1,173,175,-1,-1,177,-1,-1,-1,179,181,-1,183,185,-1,-1,187,189,-1,191,193,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9977925E0,6.062043E-1,1.3653457E0,8.562151E-1,6.1386406E-1,7.570244E-1,6.8527055E-1,3.3379424E-1,3.6342132E-1,4.9126965E-1,3.0080467E-1,6.62867E-1,3.7457204E-1,0E0,3.7154174E-1,0E0,3.0440003E-1,1.282495E-1,2.6434398E-1,0E0,2.0821777E-1,2.1819805E-1,0E0,4.4348702E-1,3.687868E-1,1.5898252E-1,4.0867412E-1,2.2680396E-1,2.7546823E-1,3.0228257E-1,2.5066787E-1,0E0,0E0,0E0,9.422225E-2,1.7106086E-1,1.937683E-1,9.5684916E-2,0E0,1.0111638E0,6.229179E-1,2.7216172E-1,2.286135E-1,3.7537608E-1,1.17274135E-1,1.4431882E-1,1.02543466E-1,0E0,1.0985802E-1,4.887122E-1,4.3725455E-1,3.24334E-1,6.043899E-1,2.3824997E-1,1.9202018E-1,7.145548E-2,0E0,0E0,1.3587794E-1,9.790751E-2,1.7564139E-1,0E0,0E0,3.6399198E-1,5.816649E-1,5.7738173E-1,1.2449655E-1,0E0,1.827023E-1,1.3359657E-1,2.5934175E-1,1.5683767E-1,0E0,6.6349596E-2,0E0,1.357181E-1,9.6772775E-2,9.626476E-2,1.2828283E-1,0E0,0E0,2.6668608E-1,1.3582146E-1,1.14468575E-1,2.53772E-1,2.8666943E-1,1.1453233E-1,2.781355E-1,5.964483E-1,1.24377936E-1,1.6804484E-1,1.1566049E-1,0E0,0E0,0E0,1.2926683E-1,0E0,0E0,1.1738366E-1,1.701904E-1,2.2587065E-1,3.1663465E-1,4.1840738E-1,0E0,0E0,3.854779E-1,3.1160572E-1,0E0,0E0,1.6202712E-1,0E0,1.1453264E-1,2.6567405E-1,1.6546895E-1,0E0,7.8558914E-2,9.2982456E-2,0E0,0E0,1.4664042E-1,0E0,0E0,0E0,1.0239504E-1,7.477175E-2,0E0,1.5221295E-1,1.438371E-1,0E0,0E0,1.2754878E-1,1.0192692E-1,0E0,9.438035E-2,1.60438E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,58,58,59,59,60,60,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,73,73,75,75,76,76,77,77,78,78,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,95,95,98,98,99,99,100,100,101,101,102,102,105,105,106,106,109,109,111,111,112,112,113,113,115,115,116,116,119,119,123,123,124,124,126,126,127,127,130,130,131,131,133,133,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,32,34,-1,36,38,-1,40,42,44,46,48,50,52,54,-1,-1,-1,56,58,60,62,-1,64,66,68,70,72,74,76,78,-1,80,82,84,86,88,90,92,94,-1,-1,96,98,100,-1,-1,102,104,106,108,-1,110,112,114,116,-1,118,-1,120,122,124,126,-1,-1,128,130,132,134,136,138,140,142,144,146,148,-1,-1,-1,150,-1,-1,152,154,156,158,160,-1,-1,162,164,-1,-1,166,-1,168,170,172,-1,174,176,-1,-1,178,-1,-1,-1,180,182,-1,184,186,-1,-1,188,190,-1,192,194,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0119178E10,9.357879E2,1.3664E4,4.3702424E7,1.3521131E8,2.065904E6,1.1032986E8,2.9143541E-6,2.8035452E11,9.3262803E8,7.6008E4,4.264897E6,2.342E3,1.8509028E-2,2.943086E4,-1.4071275E-2,1.7887324E0,2.3140822E8,1.9044118E0,1.1373961E-2,6.860185E2,5.3E1,-1.582741E-2,3.7311627E2,2.46E2,6.0607E4,5.542E3,1.3683E4,9.3058E4,6.8503425E6,1.044E1,6.3279714E-3,-4.916561E-3,-2.028804E-2,2.5341389E8,1.1214854E1,2.467284E7,4.982003E6,2.3307567E-3,2.906587E6,7.218466E7,1.5782692E2,1.5E1,7.451E3,6.745516E2,2.8366232E0,6.97E2,7.914651E-3,7.074777E6,4.52737E5,2.3329864E0,9E2,5.4225586E-2,1.06E3,2.8107285E-1,2.0288463E0,-2.9019355E-3,3.3785067E-3,2.9887363E-1,1.2354571E0,6.7652373E3,-8.668358E-3,2.2749869E-4,5.839E3,1.86E2,2.6491638E1,1.7495675E5,-2.8309408E-3,1.37E2,6.0607E4,1.5963264E9,5.393203E-1,-1.346354E-2,4.4377274E2,-5.543946E-3,1.9007992E1,5E0,4.4768806E1,1.43709E0,-2.473924E-3,-1.2377798E-2,1E0,2.8541E4,1.694761E9,1.56E2,5.1538463E0,1.151E4,1.8836284E2,5.4121188E7,3.724138E0,5.49E2,2.7453306E5,-1.2415581E-3,-3.6467605E-3,-1.3537403E-2,6.5884055E11,4.9200375E-3,-4.096677E-3,3.3487454E11,1.778482E7,2.9956698E-1,5.024605E0,1.263E3,-4.523295E-3,-3.4544643E-2,3.598933E6,3.3720784E5,-5.5208183E-3,-7.435904E-4,1.6633E4,3.728546E-4,1.563E3,2.23E2,4.3348173E8,5.7262774E-3,1.6E1,1.2478948E1,2.918271E-3,7.49834E-3,2.3821254E7,1.3983822E-5,-3.301526E-3,5.4759854E-3,3.7173E4,3.88672E5,9.326011E-3,9.966725E2,6.5794776E7,1.4736244E-2,-1.2966491E-3,2.3817926E7,2.1325744E7,9.283759E-3,9.317209E0,3.555E3,-7.1728714E-3,7.9689035E-3,3.7247401E-3,-1.0688422E-3,-3.5455378E-3,4.960878E-3,-1.8310005E-2,-5.9232963E-3,-1.4127112E-3,-7.13121E-3,3.3311378E-3,-4.6278542E-4,-9.437097E-3,1.5249219E-3,-1.8013548E-3,-6.1339852E-3,2.350183E-3,7.644698E-3,9.851647E-4,-2.754541E-3,8.337387E-3,-2.9860619E-3,-1.4877917E-4,-6.73536E-3,-6.5381285E-3,-7.730732E-4,9.798173E-3,1.5044949E-3,-1.8745117E-3,-1.6805239E-2,2.9639E-3,7.361661E-3,2.1672053E-3,-4.4020726E-3,1.0820817E-2,2.1882185E-3,-1.9499151E-3,3.7166772E-3,2.256361E-3,-1.1964369E-3,6.0344266E-4,-8.167676E-3,1.0949966E-2,3.5904988E-4,3.1363133E-3,-6.0434113E-3,1.5128596E-4,-8.5656205E-3,4.521304E-3,-3.1063373E-3,6.0842587E-3,-3.3537655E-3,3.2059383E-3,-3.848208E-3,6.1481483E-3,2.3336638E-2,3.5530357E-3,9.620969E-3,-1.013665E-2,2.9693034E-3],"split_indices":[19,54,2,44,44,27,7,41,30,30,10,9,2,0,32,0,52,7,52,0,51,8,0,51,10,1,28,9,2,44,53,0,0,0,7,55,44,28,0,49,44,4,3,28,4,37,0,0,46,28,34,2,41,2,26,52,0,0,33,52,51,0,0,28,10,52,32,0,3,1,7,40,0,51,0,55,3,57,52,0,0,89,2,7,8,53,28,51,12,55,0,46,0,0,0,30,0,0,30,49,37,52,2,0,0,5,27,0,0,1,0,2,0,31,0,3,57,0,0,9,0,0,0,28,9,0,4,49,0,0,42,1,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.211E3,7.04E2,1.507E3,4.03E2,3.01E2,1.306E3,2.01E2,3.73E2,3E1,2.74E2,2.7E1,1.076E3,2.3E2,6E0,1.95E2,4E0,3.69E2,8E0,2.2E1,7E0,2.67E2,2.2E1,5E0,7.98E2,2.78E2,1.3E2,1E2,1.2E1,1.83E2,1.91E2,1.78E2,2E0,6E0,7E0,1.5E1,9.4E1,1.73E2,1.5E1,7E0,3.54E2,4.44E2,1.03E2,1.75E2,8.5E1,4.5E1,4.7E1,5.3E1,2E0,1E1,1.31E2,5.2E1,1.08E2,8.3E1,1.43E2,3.5E1,1E1,5E0,1.1E1,8.3E1,6.1E1,1.12E2,1.2E1,3E0,3.49E2,5E0,3.89E2,5.5E1,1E1,9.3E1,6.2E1,1.13E2,8.2E1,3E0,4.3E1,2E0,3.7E1,1E1,2.4E1,2.9E1,7E0,3E0,4.7E1,8.4E1,2E1,3.2E1,2.9E1,7.9E1,3.7E1,4.6E1,4.8E1,9.5E1,1.5E1,2E1,3E0,7E0,7.9E1,4E0,3E0,5.8E1,9.8E1,1.4E1,2.46E2,1.03E2,3E0,2E0,3.08E2,8.1E1,3.9E1,1.6E1,7.7E1,1.6E1,2.3E1,3.9E1,1.02E2,1.1E1,7.3E1,9E0,3.5E1,8E0,3.4E1,3E0,3E0,7E0,1.7E1,7E0,6E0,2.3E1,4E1,7E0,2.8E1,5.6E1,5E0,1.5E1,2.6E1,6E0,2.7E1,2E0,1.1E1,6.8E1,2.7E1,1E1,1E1,3.6E1,3.9E1,9E0,4.3E1,5.2E1,1.3E1,2E0,6.1E1,1.8E1,4.8E1,1E1,4.9E1,4.9E1,9E0,5E0,2.31E2,1.5E1,5.3E1,5E1,1.1E1,2.97E2,7.9E1,2E0,3E1,4.7E1,1.6E1,7E0,9E0,3E1,9.1E1,1.1E1,5.6E1,1.7E1,4E0,5E0,3.1E1,3E0,1.5E1,2E0,4E0,3E0,1.6E1,7E0,3.7E1,3E0,5.1E1,5E0,2E0,3E0,1.8E1,8E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"195","size_leaf_vector":"1"}},{"base_weights":[1.4645935E-3,-1.0857945E-2,7.3451586E-2,6.2206234E-3,-6.376993E-2,1.1466428E-1,6.352119E-5,-2.8338438E-2,2.4957033E-2,-1.2115865E-1,-4.450906E-2,3.313305E-1,9.931702E-2,-4.9191486E-2,1.0116423E-1,-3.9204285E-2,4.255431E-2,-1.38248345E-2,5.053017E-2,-1.0121777E-1,-1.8501692E-2,-2.2061665E-2,-9.740873E-2,1.6824156E-2,3.3142052E-3,2.0425293E-1,8.154377E-2,-2.5549496E-2,-1.2918417E-2,1.810517E-2,1.6754037E-1,-4.7031064E-2,1.7061241E-2,-3.8562168E-3,4.4538467E-3,-2.4279399E-2,7.338295E-2,8.710945E-3,4.5625687E-2,-6.42019E-2,-2.0851657E-1,7.2813575E-4,-5.3433053E-2,-2.2227606E-1,-6.5765366E-2,1.2051071E-2,1.082084E-3,-2.8628012E-2,9.11547E-2,7.131542E-2,-5.2291412E-2,3.4908387E-3,-5.1044063E-3,2.1935011E-1,1.4752586E-3,-2.6572458E-2,-8.599776E-2,3.7497429E-3,-4.2063044E-3,-4.112356E-2,2.424154E-3,-3.584265E-2,5.640944E-2,1.1370177E-1,-7.244268E-4,1.530847E-2,6.525609E-2,-1.6886988E-1,-1.9784084E-2,-1.3500886E-1,-1.6580284E-2,-4.7237758E-2,2.2478355E-2,-6.960268E-2,1.6669091E-3,-1.2239065E-2,-2.9467228E-3,1.6873313E-2,-1.002516E-1,3.6879247E-3,-5.4951147E-3,2.1435618E-2,1.11061595E-1,7.0642508E-3,-5.3936837E-4,-3.0985627E-2,-5.8677737E-3,1.31004695E-2,5.519867E-3,-1.2884098E-1,-1.848845E-2,-1.2710097E-1,-4.091977E-2,2.7743927E-3,-3.679929E-2,2.71759E-3,-4.0351697E-3,6.725981E-3,-4.1725975E-2,-9.166164E-3,6.0133543E-3,1.828035E-1,1.6515246E-3,5.294044E-2,-1.1739672E-2,8.402992E-3,5.793968E-2,-1.9184266E-3,-2.0868117E-1,3.5081495E-2,-6.545866E-2,-1.8743078E-1,-4.924916E-4,-3.6975122E-3,2.4775423E-2,6.1406167E-3,6.7251775E-4,-1.5203482E-1,-3.134818E-2,1.9672192E-3,-6.0451967E-3,-4.8029818E-2,-1.5448816E-1,4.464429E-2,-1.0911246E-2,8.2798816E-2,1.7069635E-1,-3.7910081E-3,2.6780146E-4,-9.078702E-3,-2.9424457E-3,1.2108512E-3,-1.972202E-3,-3.7545154E-3,-1.0609003E-2,-9.2994515E-4,-4.7430974E-3,-6.3747405E-3,2.0707399E-4,1.367387E-3,-2.6333139E-3,1.82192E-3,-1.1495784E-2,1.4599507E-3,1.00025E-2,1.6418525E-3,8.841355E-3,-5.2733775E-3,7.9388166E-4,5.154025E-3,1.9693181E-3,-4.8123915E-3,-1.2674923E-2,-3.3955963E-3,4.268859E-3,-1.6723044E-3,-1.1584371E-2,-5.162516E-3,-1.34350145E-2,3.3270034E-3,-2.7408027E-3,3.2639748E-3,-7.825551E-3,-2.585224E-3,1.9487331E-3,-3.4346594E-3,1.2875206E-3,-1.2313763E-2,-2.8279366E-3,-2.2308751E-4,4.4197463E-3,5.642007E-3,2.066474E-3,5.6764944E-3,1.4455599E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,-1,-1,45,47,49,-1,51,53,55,57,59,-1,61,63,-1,65,67,69,71,73,75,77,-1,-1,79,81,83,85,-1,-1,87,-1,89,91,-1,93,95,-1,97,99,101,-1,103,105,107,109,111,-1,113,115,117,-1,-1,-1,119,121,-1,-1,123,125,-1,-1,127,-1,-1,-1,129,131,133,135,-1,137,-1,-1,-1,139,141,-1,143,-1,145,147,-1,149,-1,151,153,155,157,-1,-1,159,-1,-1,161,163,-1,-1,165,167,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.923949E0,1.6739768E0,9.5876527E-1,9.0787256E-1,4.942286E-1,6.4211535E-1,5.7764375E-1,3.8143104E-1,9.0222114E-1,6.064706E-1,4.0158737E-1,1.07953906E-1,3.3956945E-1,4.2818335E-1,2.064659E-1,1.8928415E-1,1.7136493E-1,3.3182412E-1,3.7253284E-1,4.1159093E-1,0E0,1.7162639E-1,3.8445532E-1,0E0,0E0,2.9519796E-1,1.7732036E-1,1.9023463E-1,0E0,1.4765505E-1,1.3968432E-1,2.9746366E-1,7.447416E-2,8.1350796E-2,0E0,3.0432206E-1,1.4236964E-1,0E0,3.1502008E-1,3.732381E-1,2.635548E-1,1.4711945E-1,1.4995506E-1,1.3348746E-1,2.3538303E-1,0E0,0E0,1.5282862E-1,2.085787E-1,1.1467467E-1,8.987716E-2,0E0,0E0,6.703532E-2,0E0,2.0264933E-1,2.3381668E-1,0E0,8.976538E-2,1.1702845E-1,0E0,3.1431374E-1,2.0665896E-1,1.3741073E-1,0E0,2.1437523E-1,2.7277017E-1,1.1202073E-1,1.476422E-1,1.2810466E-1,0E0,1.1112396E-1,8.166528E-2,2.664265E-1,0E0,0E0,0E0,1.0573446E-1,1.568551E-1,0E0,0E0,2.2394083E-1,1.8474889E-1,0E0,0E0,9.0033725E-2,0E0,0E0,0E0,6.9355994E-2,2.6315165E-1,2.979126E-1,7.981821E-2,0E0,1.2112759E-1,0E0,0E0,0E0,3.280886E-1,2.8834966E-1,0E0,7.3244184E-2,0E0,2.0180294E-1,3.791801E-1,0E0,2.4127972E-1,0E0,1.0160625E-1,1.830517E-1,1.643356E-1,7.850158E-2,0E0,0E0,6.6783905E-2,0E0,0E0,1.38417E-1,1.1740862E-1,0E0,0E0,6.813867E-2,2.857625E-1,8.714611E-2,0E0,1.2172693E-1,2.0911193E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,48,48,49,49,50,50,53,53,55,55,56,56,58,58,59,59,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,77,77,78,78,81,81,82,82,85,85,89,89,90,90,91,91,92,92,94,94,98,98,99,99,101,101,103,103,104,104,106,106,108,108,109,109,110,110,111,111,114,114,117,117,118,118,121,121,122,122,123,123,125,125,126,126],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,-1,-1,46,48,50,-1,52,54,56,58,60,-1,62,64,-1,66,68,70,72,74,76,78,-1,-1,80,82,84,86,-1,-1,88,-1,90,92,-1,94,96,-1,98,100,102,-1,104,106,108,110,112,-1,114,116,118,-1,-1,-1,120,122,-1,-1,124,126,-1,-1,128,-1,-1,-1,130,132,134,136,-1,138,-1,-1,-1,140,142,-1,144,-1,146,148,-1,150,-1,152,154,156,158,-1,-1,160,-1,-1,162,164,-1,-1,166,168,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,8.2608955E-3,3.5301748E7,1.5497989E5,1.7314286E0,1.3510204E1,6.7652373E3,6.932264E8,2.6659616E2,7.8099336E11,2.8035452E11,3.392E3,1.5277338E0,5.24422E5,1.2E1,3.307766E6,1.7022566E7,1.3195207E7,7.501215E6,2.294901E0,-1.8501692E-2,1.8544E4,8.92686E5,1.6824156E-2,3.3142052E-3,2.5422776E3,3.2449896E0,5.441732E9,-1.2918417E-2,3.6181125E9,6.971004E7,3.136758E9,2.081E3,1.3359244E9,4.4538467E-3,4.5723195E6,9.87156E0,8.710945E-3,9.76E2,8.31E2,2.72258E5,1.2E1,5.441E3,9.3299186E-1,5.3042426E0,1.2051071E-2,1.082084E-3,1.8222668E2,8.079531E4,1.3062E4,2.6654E4,3.4908387E-3,-5.1044063E-3,2.109E3,1.4752586E-3,8.333333E0,4.423E3,3.7497429E-3,3.304531E3,6.222E3,2.424154E-3,2.0223091E1,8.15E2,1.9203481E5,-7.244268E-4,5.101512E0,1.176E3,1E0,7.11E2,2.9E1,-1.6580284E-2,3.3737933E8,1.9044118E0,4.1808826E-1,1.6669091E-3,-1.2239065E-2,-2.9467228E-3,4.4967708E7,1.8000048E10,3.6879247E-3,-5.4951147E-3,3.7826266E4,4.77721E3,7.0642508E-3,-5.3936837E-4,1.0526E4,-5.8677737E-3,1.31004695E-2,5.519867E-3,1.2068493E4,2.7460318E0,3.154992E7,2.9290114E7,2.7743927E-3,3.1152E4,2.71759E-3,-4.0351697E-3,6.725981E-3,1.1217151E0,7.89E2,6.0133543E-3,6.2E1,1.6515246E-3,4.2782607E0,3.7311627E2,8.402992E-3,1.6954315E0,-1.9184266E-3,1.71E2,6.29E2,5.099062E5,1.5476604E0,-4.924916E-4,-3.6975122E-3,2.0201733E9,6.1406167E-3,6.7251775E-4,3.114E3,2.539E3,1.9672192E-3,-6.0451967E-3,1.1034263E10,3.138649E6,3.7002478E3,-1.0911246E-2,1.4307411E7,1.2328733E1,-3.7910081E-3,2.6780146E-4,-9.078702E-3,-2.9424457E-3,1.2108512E-3,-1.972202E-3,-3.7545154E-3,-1.0609003E-2,-9.2994515E-4,-4.7430974E-3,-6.3747405E-3,2.0707399E-4,1.367387E-3,-2.6333139E-3,1.82192E-3,-1.1495784E-2,1.4599507E-3,1.00025E-2,1.6418525E-3,8.841355E-3,-5.2733775E-3,7.9388166E-4,5.154025E-3,1.9693181E-3,-4.8123915E-3,-1.2674923E-2,-3.3955963E-3,4.268859E-3,-1.6723044E-3,-1.1584371E-2,-5.162516E-3,-1.34350145E-2,3.3270034E-3,-2.7408027E-3,3.2639748E-3,-7.825551E-3,-2.585224E-3,1.9487331E-3,-3.4346594E-3,1.2875206E-3,-1.2313763E-2,-2.8279366E-3,-2.2308751E-4,4.4197463E-3,5.642007E-3,2.066474E-3,5.6764944E-3,1.4455599E-2],"split_indices":[51,26,44,27,52,57,51,7,51,30,30,11,52,11,17,1,9,9,5,55,0,9,1,0,0,51,53,5,0,7,44,30,0,7,0,27,57,0,10,2,9,17,2,26,55,0,0,57,27,9,9,0,0,0,0,51,2,0,32,2,0,51,2,32,0,55,10,111,0,8,0,7,52,26,0,0,0,49,5,0,0,51,51,0,0,9,0,0,0,27,57,12,31,0,10,0,0,0,52,2,0,0,0,55,51,0,53,0,0,10,32,52,0,0,5,0,0,28,2,0,0,5,28,51,0,50,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.167E3,1.851E3,3.16E2,1.4E3,4.51E2,2.02E2,1.14E2,4.92E2,9.08E2,1.12E2,3.39E2,1.2E1,1.9E2,7.7E1,3.7E1,4.27E2,6.5E1,3.61E2,5.47E2,1.06E2,6E0,2.39E2,1E2,1E1,2E0,2.6E1,1.64E2,7.1E1,6E0,1.7E1,2E1,3.75E2,5.2E1,3.6E1,2.9E1,3.23E2,3.8E1,1.7E1,5.3E2,8E1,2.6E1,1.39E2,1E2,1.9E1,8.1E1,1.9E1,7E0,1.3E1,1.51E2,1.5E1,5.6E1,1.2E1,5E0,1.4E1,6E0,2.47E2,1.28E2,1.2E1,4E1,2.2E1,1.4E1,2.83E2,4E1,2.6E1,1.2E1,2.09E2,3.21E2,2.3E1,5.7E1,1.9E1,7E0,4.3E1,9.6E1,8.5E1,1.5E1,1.4E1,5E0,2.4E1,5.7E1,6E0,7E0,3.4E1,1.17E2,7E0,8E0,4.5E1,1.1E1,7E0,7E0,1.7E1,2.3E2,6.6E1,6.2E1,1.3E1,2.7E1,7E0,1.5E1,8E0,2.75E2,2.2E1,1.8E1,1.3E1,1.3E1,8.7E1,1.22E2,1.7E1,3.04E2,6E0,1.7E1,2.6E1,3.1E1,1.3E1,6E0,2.9E1,1.4E1,5E0,9.1E1,2.6E1,5.9E1,2.1E1,3E0,3E1,2.7E1,3.2E1,2E0,8.1E1,3.6E1,1.8E1,2.7E1,7E0,1E1,8.2E1,1.48E2,4.8E1,1.8E1,4.8E1,1.4E1,7E0,2E1,5.1E1,2.24E2,1.9E1,3E0,3E0,1E1,7.9E1,8E0,2.6E1,9.6E1,6.1E1,2.43E2,8E0,9E0,9E0,1.7E1,2.8E1,3E0,9E0,4E0,9E0,5E0,2E0,2.4E1,4.4E1,1.5E1,2.2E1,8E0,1.1E1,1.6E1,1.7E1,1.5E1,3.7E1,4.4E1,2.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-2.3424716E-3,1.445892E-2,-4.9134076E-2,-5.9882957E-3,4.6883993E-2,-2.1808055E-1,-4.0841468E-2,2.2335846E-2,-2.6293218E-2,5.2172323E-3,8.136837E-2,-8.713294E-2,-3.026612E-1,-1.8090619E-1,-3.3629037E-2,2.8644167E-2,-1.754255E-1,-4.0562052E-2,1.3092696E-2,-3.5247672E-2,5.043933E-2,1.03758946E-1,1.8223498E-2,1.8614677E-3,-6.0871216E-3,-2.0899894E-2,-7.294343E-3,-2.2699752E-1,-1.6788455E-2,-4.3098174E-2,3.5991516E-2,4.2876564E-2,-4.231116E-2,-1.5660489E-2,-6.1222915E-2,-5.2656963E-2,2.2779098E-2,-5.0629415E-3,8.036696E-2,1.06015384E-1,-4.389187E-2,3.9710797E-2,1.0908543E-2,9.457187E-2,1.2694137E-2,-2.3284802E-1,3.5528213E-2,-2.5883037E-1,-1.2224163E-3,-5.325981E-3,7.103116E-3,-3.0976271E-2,-1.0409253E-1,1.2975658E-2,7.1976334E-3,1.9512981E-1,3.517803E-2,-5.8970224E-2,8.12156E-3,1.2916289E-3,-1.1329209E-2,-3.149511E-2,-8.247111E-2,1.12251945E-1,-2.149015E-2,8.86116E-3,-1.0590721E-1,8.483929E-3,2.2475547E-3,8.298356E-3,-1.6481304E-3,-6.642656E-2,2.0123122E-2,4.9599513E-2,-9.754238E-3,1.0572513E-1,5.5532316E-3,-1.6236886E-2,5.6528335E-4,5.41713E-2,-8.395177E-2,-4.2462726E-3,-1.3549824E-2,-5.993327E-2,4.372493E-3,-8.896121E-2,-1.3771941E-2,6.425278E-2,-2.3156142E-2,1.4449733E-3,1.117699E-2,1.7400602E-2,1.0010975E-1,3.6804148E-3,-1.0780073E-1,-1.3089628E-2,-8.476494E-2,-1.647074E-1,-3.8833126E-2,7.3330323E-3,-5.5059274E-5,9.385213E-3,-4.2982255E-3,2.7849618E-2,-6.335002E-2,-9.763178E-3,-1.3343494E-3,-8.09881E-2,6.597037E-4,-3.5226487E-3,5.319837E-2,2.9709054E-2,1.2419361E-1,1.6235411E-1,8.326703E-2,-1.1129327E-1,8.7706454E-2,4.0854555E-2,1.0523244E-2,1.3847762E-3,-6.5667275E-3,-5.403789E-2,-1.34342145E-2,9.691031E-3,-1.0325948E-2,-1.0986262E-1,-1.3647006E-2,1.0600918E-3,1.1426159E-2,-3.5008385E-3,9.429666E-4,1.2580496E-3,-7.363592E-3,2.9499067E-3,9.589883E-3,4.0012402E-3,-1.4847136E-3,-9.146893E-3,-2.8878653E-3,-2.6503922E-3,1.3469071E-3,-5.0270204E-3,1.8983164E-3,-2.267965E-3,-9.984745E-3,3.590269E-3,-2.5461644E-3,-5.265092E-3,1.1453923E-3,7.1329917E-3,7.3782797E-4,-6.8697482E-3,1.950756E-4,-2.9235408E-3,-6.476674E-3,-6.4340944E-4,4.600637E-3,5.099793E-3,2.5466645E-5,-3.161833E-3,7.864863E-3,2.3944397E-3,8.949192E-3,4.6734054E-5,4.927035E-3,-2.6534856E-3,-1.1309791E-2,-1.0406877E-3,9.738847E-3,1.1233486E-2,1.4148474E-3,-4.6670954E-3,-1.6909261E-3,2.754433E-3,-2.7305254E-4,2.2588618E-4,-6.0850615E-3,8.385951E-3,-2.3774244E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,-1,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,73,-1,75,-1,77,79,81,-1,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,95,97,99,101,103,105,-1,-1,-1,-1,107,109,111,-1,113,115,-1,-1,117,119,-1,-1,121,123,125,-1,127,129,-1,-1,131,133,135,137,139,141,143,145,-1,-1,147,-1,149,151,-1,-1,153,-1,-1,155,157,159,161,163,165,167,169,-1,-1,-1,171,-1,173,-1,175,177,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7318437E0,1.0750078E0,8.064792E-1,5.7283115E-1,9.0048206E-1,2.726704E-1,5.555916E-1,5.2400243E-1,3.2648593E-1,5.233017E-1,4.8305964E-1,7.834953E-2,2.8564918E-1,2.0042801E-1,3.5095185E-1,4.097994E-1,2.3192239E-1,3.2756042E-1,1.89725E-1,1.8904012E-1,2.663515E-1,3.8402104E-1,4.0364233E-1,0E0,0E0,0E0,0E0,1.2670743E-1,1.4106688E-1,3.420602E-1,1.7910042E-1,3.8814253E-1,2.6172185E-1,0E0,1.5981863E-1,2.2327286E-1,2.757141E-1,1.7365085E-1,9.920637E-2,1.05723426E-1,2.0762259E-1,3.361796E-1,0E0,2.3968196E-1,0E0,1.8963882E-1,1.9611071E-1,9.21098E-2,0E0,0E0,0E0,4.0048736E-1,2.0607609E-1,1.0338826E-1,0E0,1.2724805E-1,3.6994877E-1,1.9618955E-1,0E0,0E0,0E0,2.0576687E-1,5.2472055E-1,1.2995008E-1,1.0747451E-1,1.512228E-1,1.19462445E-1,0E0,0E0,0E0,0E0,1.2620229E-1,1.2722415E-1,1.8177417E-1,0E0,2.5991392E-1,2.78542E-1,0E0,0E0,1.6854583E-1,8.840565E-2,0E0,0E0,2.8410882E-1,2.210339E-1,1.1341536E-1,0E0,1.707907E-1,8.0236375E-2,0E0,0E0,4.7952923E-1,2.556135E-1,9.2488274E-2,1.3642737E-1,3.0938917E-1,1.8438646E-1,3.153124E-1,2.0597084E-1,0E0,0E0,7.09439E-2,0E0,1.2779355E-1,1.3819796E-1,0E0,0E0,8.2449615E-2,0E0,0E0,9.609849E-2,2.4166045E-1,2.6196933E-1,2.2148407E-1,3.2778895E-1,7.29285E-2,2.4468865E-1,1.3357493E-1,0E0,0E0,0E0,1.6886699E-1,0E0,1.4029075E-1,0E0,1.6134113E-1,1.4112346E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,45,45,46,46,47,47,51,51,52,52,53,53,55,55,56,56,57,57,61,61,62,62,63,63,64,64,65,65,66,66,71,71,72,72,73,73,75,75,76,76,79,79,80,80,83,83,84,84,85,85,87,87,88,88,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,101,101,103,103,104,104,107,107,110,110,111,111,112,112,113,113,114,114,115,115,116,116,117,117,121,121,123,123,125,125,126,126],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,-1,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,74,-1,76,-1,78,80,82,-1,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,96,98,100,102,104,106,-1,-1,-1,-1,108,110,112,-1,114,116,-1,-1,118,120,-1,-1,122,124,126,-1,128,130,-1,-1,132,134,136,138,140,142,144,146,-1,-1,148,-1,150,152,-1,-1,154,-1,-1,156,158,160,162,164,166,168,170,-1,-1,-1,172,-1,174,-1,176,178,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,5.862126E2,7E0,2E0,1.5166431E5,2.915E4,7.81893E-2,5.9572783E0,3.7186194E5,6.626653E8,1.7398448E7,9.824011E-4,1.36054E-1,3.9986582E4,6.513793E1,3.0923106E-10,3.415965E2,6.54925E6,3.1930302E1,2.3066575E6,5.686354E-1,4.502969E3,3.171E3,1.8614677E-3,-6.0871216E-3,-2.0899894E-2,-7.294343E-3,2.914E3,1.7026253E8,7.218466E7,1.4813511E6,1.3E1,8.8692E2,-1.5660489E-2,4.9E1,1.1E1,4.204986E-2,1.9809574E6,1.2E1,4.3E1,1.772114E6,9.496754E6,1.0908543E-2,2.0319266E-7,1.2694137E-2,5.59E2,4.2620764E2,4.5555557E1,-1.2224163E-3,-5.325981E-3,7.103116E-3,7.449623E2,5.58901E5,2.7864855E11,7.1976334E-3,6.04E2,7.2419E4,2.20524E5,8.12156E-3,1.2916289E-3,-1.1329209E-2,1E0,1.246E3,1.07E2,2.9273078E0,2.1442623E1,2.41E3,8.483929E-3,2.2475547E-3,8.298356E-3,-1.6481304E-3,1.8839418E3,7.298614E2,4.247868E3,-9.754238E-3,3.3382E4,9.015703E-7,-1.6236886E-2,5.6528335E-4,4.2343444E7,4.50847E5,-4.2462726E-3,-1.3549824E-2,7.780377E6,3.036676E9,1.2204E4,-1.3771941E-2,5.441732E9,1.3532244E7,1.4449733E-3,1.117699E-2,2.465E3,4.681779E2,1.3414634E0,1.48404255E1,4.0701206E1,1.7589172E1,4.4444447E0,8.192143E1,7.3330323E-3,-5.5059274E-5,4.7121394E-2,-4.2982255E-3,3.9387238E5,4.5763E4,-9.763178E-3,-1.3343494E-3,2.5249E4,6.597037E-4,-3.5226487E-3,6.061028E4,6.255748E6,5.0279167E1,9.035491E5,4.1032645E6,2.2075728E7,3.131E3,3.564E3,1.0523244E-2,1.3847762E-3,-6.5667275E-3,4.2378342E-1,-1.34342145E-2,1.01696544E6,-1.0325948E-2,9E0,8.24666E5,1.0600918E-3,1.1426159E-2,-3.5008385E-3,9.429666E-4,1.2580496E-3,-7.363592E-3,2.9499067E-3,9.589883E-3,4.0012402E-3,-1.4847136E-3,-9.146893E-3,-2.8878653E-3,-2.6503922E-3,1.3469071E-3,-5.0270204E-3,1.8983164E-3,-2.267965E-3,-9.984745E-3,3.590269E-3,-2.5461644E-3,-5.265092E-3,1.1453923E-3,7.1329917E-3,7.3782797E-4,-6.8697482E-3,1.950756E-4,-2.9235408E-3,-6.476674E-3,-6.4340944E-4,4.600637E-3,5.099793E-3,2.5466645E-5,-3.161833E-3,7.864863E-3,2.3944397E-3,8.949192E-3,4.6734054E-5,4.927035E-3,-2.6534856E-3,-1.1309791E-2,-1.0406877E-3,9.738847E-3,1.1233486E-2,1.4148474E-3,-4.6670954E-3,-1.6909261E-3,2.754433E-3,-2.7305254E-4,2.2588618E-4,-6.0850615E-3,8.385951E-3,-2.3774244E-3],"split_indices":[26,51,3,16,27,9,57,52,46,7,49,41,41,32,55,36,4,9,55,44,37,4,2,0,0,0,0,2,7,44,27,10,45,0,3,8,40,46,3,3,28,46,0,36,0,0,57,51,0,0,0,4,9,30,0,1,1,9,0,0,0,15,2,3,52,55,2,0,0,0,0,51,51,51,0,9,36,0,0,44,31,0,0,27,7,2,0,5,1,0,0,2,54,52,45,51,55,55,49,0,0,37,0,46,11,0,0,11,0,0,27,44,55,46,44,44,2,2,0,0,0,26,0,49,0,3,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.201E3,1.62E3,5.81E2,9.94E2,6.26E2,2.6E1,5.55E2,4.15E2,5.79E2,2.84E2,3.42E2,1.1E1,1.5E1,2.6E1,5.29E2,4.03E2,1.2E1,4.25E2,1.54E2,1.5E2,1.34E2,2.52E2,9E1,3E0,8E0,6E0,9E0,2E1,6E0,4.66E2,6.3E1,3.36E2,6.7E1,4E0,8E0,3.57E2,6.8E1,1.22E2,3.2E1,8E0,1.42E2,1.28E2,6E0,2.41E2,1.1E1,5E0,8.5E1,1.7E1,3E0,4E0,2E0,3.9E2,7.6E1,5.4E1,9E0,1.5E1,3.21E2,6.3E1,4E0,6E0,2E0,2.1E2,1.47E2,2.2E1,4.6E1,1.08E2,1.4E1,6E0,2.6E1,5E0,3E0,1.05E2,3.7E1,1.24E2,4E0,2.14E2,2.7E1,3E0,2E0,7.4E1,1.1E1,4E0,1.3E1,2.14E2,1.76E2,7.2E1,4E0,2.2E1,3.2E1,4E0,1.1E1,2.53E2,6.8E1,2.8E1,3.5E1,1.57E2,5.3E1,5E1,9.7E1,1.5E1,7E0,3.3E1,1.3E1,8.6E1,2.2E1,5E0,9E0,8.9E1,1.6E1,9E0,2.8E1,9.9E1,2.5E1,5.9E1,1.55E2,1.1E1,1.6E1,7E1,4E0,4E0,7E0,2.1E2,4E0,1.73E2,3E0,5.6E1,1.6E1,1.9E1,3E0,1.4E1,1.8E1,2.4E2,1.3E1,5.3E1,1.5E1,8E0,2E1,1E1,2.5E1,7.6E1,8.1E1,4.4E1,9E0,1.7E1,3.3E1,1.2E1,8.5E1,3E0,3E1,6E0,8E1,9E0,1.3E1,7.2E1,1.7E1,1.2E1,1.6E1,2.5E1,7.4E1,5E0,2E1,1.5E1,4.4E1,3.7E1,1.18E2,9E0,2E0,9E0,7E0,2E0,6.8E1,5.2E1,1.58E2,4E1,1.33E2,1E1,4.6E1,2E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"179","size_leaf_vector":"1"}},{"base_weights":[-1.274962E-3,-3.6095917E-2,2.0345991E-2,-4.296477E-2,1.3002029E-1,2.1225023E-3,5.818056E-2,-2.0727657E-2,-7.815486E-2,2.314117E-1,-5.1566724E-2,4.3302555E-2,-1.8404324E-2,1.05979815E-1,3.2012172E-2,-6.255902E-2,5.1641474E-3,-2.0697369E-1,-6.921704E-2,1.2834967E-2,3.5903999E-3,1.3220676E-3,-1.0892714E-2,2.2152686E-2,1.3218671E-1,6.336275E-2,-2.4324277E-2,1.6632712E-1,3.77824E-2,2.2050405E-2,1.8076709E-1,-3.7415568E-3,-9.941285E-2,-9.880525E-2,1.620641E-2,-2.5986958E-1,-3.2422596E-3,-5.619664E-2,-1.4088248E-1,7.6220883E-3,1.5359452E-2,8.375588E-3,8.791379E-2,1.1630708E-2,3.0266427E-2,-2.0658324E-2,-1.6029812E-1,2.1005364E-1,6.785533E-2,-1.5540857E-2,5.292422E-2,2.6082106E-2,-1.1526485E-2,3.361876E-3,1.0947065E-2,-5.7606094E-2,5.6211863E-2,-1.7659375E-1,-4.7650144E-2,-6.239382E-2,-1.4482278E-2,2.0716111E-1,2.4020683E-3,-1.3413034E-2,-1.252699E-3,-2.0771687E-1,-5.167503E-2,-1.7332299E-1,3.082027E-2,2.0837577E-2,-6.615017E-3,9.253961E-3,2.2701945E-3,-2.0401765E-2,4.1233776E-3,-3.8152985E-2,9.286263E-4,9.634157E-4,-8.596168E-3,1.3913782E-2,1.801123E-1,4.581188E-3,-2.9733349E-3,9.016112E-3,2.6553415E-2,-3.5080284E-2,4.0900193E-2,5.2590417E-3,-6.980856E-2,6.8952697E-3,-1.9675702E-2,-3.4527475E-4,-2.7208772E-1,-2.8959896E-2,-9.827404E-3,-6.9731656E-3,4.6299936E-4,1.0885527E-2,2.9836262E-3,5.579109E-2,-2.1113763E-2,-2.9004344E-3,-1.3556851E-2,-9.913677E-2,-3.6585517E-2,-1.2845839E-2,-1.392383E-1,-3.6260304E-3,5.1333928E-3,-2.1231055E-2,4.297301E-2,-6.88345E-3,3.0237675E-2,-3.3605367E-2,-2.0619337E-1,5.670622E-2,-2.0296156E-2,1.2115413E-1,1.1070901E-2,-5.247719E-3,5.2443862E-2,-2.0734638E-3,4.874343E-3,3.5120536E-2,8.4890295E-3,-4.7148257E-3,-1.5919283E-4,-6.385449E-3,2.312633E-3,-2.3484859E-3,-1.4218546E-2,-2.6010834E-3,3.248122E-3,4.1791046E-4,6.7263497E-3,-6.286417E-4,-5.594267E-3,-8.943681E-3,-2.6225091E-3,6.9084513E-4,-2.6003746E-3,-7.52767E-3,-1.844479E-3,-2.22417E-3,2.320336E-3,2.300709E-3,-7.1390974E-3,4.180145E-3,-1.9647053E-3,-1.1054336E-3,-3.772728E-3,-1.8053204E-2,-3.4203161E-3,-7.1593275E-4,3.989031E-3,-1.955015E-3,1.5468494E-3,3.5773893E-3,1.0152213E-2,8.935426E-3,1.6707452E-3,-4.405424E-3,1.8184873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,-1,69,-1,71,-1,73,75,77,79,81,-1,83,85,-1,-1,-1,87,89,91,93,95,-1,97,99,-1,-1,101,103,105,107,109,-1,-1,-1,111,-1,113,115,-1,-1,-1,117,-1,-1,-1,119,121,123,-1,125,-1,127,-1,129,131,-1,-1,-1,-1,-1,133,135,-1,-1,137,139,-1,141,-1,-1,143,145,-1,147,149,151,153,155,157,-1,-1,159,-1,-1,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.670583E0,9.754603E-1,9.437718E-1,6.377038E-1,6.379195E-1,7.826956E-1,5.532265E-1,5.4403406E-1,3.495034E-1,1.5180993E-1,2.0550366E-1,5.760772E-1,3.006886E-1,6.398542E-1,4.2427206E-1,4.158166E-1,3.5864708E-1,1.2458879E-1,2.6921737E-1,0E0,0E0,0E0,0E0,2.4571055E-1,1.2168455E-1,2.6124507E-1,2.8430793E-1,3.4313035E-1,4.5089555E-1,3.1001526E-1,9.904963E-2,2.4544038E-1,4.6300626E-1,2.2153196E-1,7.428554E-1,1.101132E-1,0E0,1.6260046E-1,2.559976E-1,0E0,2.1533774E-1,0E0,1.3934013E-1,0E0,1.1585954E-1,2.1265596E-1,8.4584534E-2,1.1065459E-1,1.2682033E-1,0E0,2.761984E-1,2.4523206E-1,0E0,0E0,0E0,9.292415E-2,2.6550773E-1,7.562871E-1,2.2169542E-1,1.8123445E-1,0E0,7.346469E-2,3.3262658E-1,0E0,0E0,7.3051006E-2,1.7326814E-1,1.035018E-1,8.234002E-2,2.178837E-1,0E0,0E0,0E0,1.4458512E-1,0E0,2.307319E-1,3.0066934E-1,0E0,0E0,0E0,1.4508343E-1,0E0,0E0,0E0,2.3845117E-1,8.6015955E-2,1.7728263E-1,0E0,8.611828E-2,0E0,1.8753862E-1,0E0,2.6096916E-1,1.9349352E-1,0E0,0E0,0E0,0E0,0E0,3.485341E-1,1.3454825E-1,0E0,0E0,2.2783619E-1,2.0518747E-1,0E0,7.576364E-2,0E0,0E0,1.6629581E-1,2.4884644E-1,0E0,7.716009E-2,1.3686922E-1,1.7113465E-1,1.6080418E-1,2.3081604E-1,9.521958E-2,0E0,0E0,1.1433409E-1,0E0,0E0,1.4422902E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,42,42,44,44,45,45,46,46,47,47,48,48,50,50,51,51,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,67,67,68,68,69,69,73,73,75,75,76,76,80,80,84,84,85,85,86,86,88,88,90,90,92,92,93,93,99,99,100,100,103,103,104,104,106,106,109,109,110,110,112,112,113,113,114,114,115,115,116,116,117,117,120,120,123,123],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,-1,70,-1,72,-1,74,76,78,80,82,-1,84,86,-1,-1,-1,88,90,92,94,96,-1,98,100,-1,-1,102,104,106,108,110,-1,-1,-1,112,-1,114,116,-1,-1,-1,118,-1,-1,-1,120,122,124,-1,126,-1,128,-1,130,132,-1,-1,-1,-1,-1,134,136,-1,-1,138,140,-1,142,-1,-1,144,146,-1,148,150,152,154,156,158,-1,-1,160,-1,-1,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,9.256843E3,8.027061E2,2.0304577E-1,6.4391124E7,2E0,4.3978744E0,5.033165E7,7E0,1.0118026E0,1.22159E5,8.2465213E9,7E0,2.1351435E0,2.8332526E9,1.07882355E2,4.0701206E1,3.7E1,9.302862E7,1.2834967E-2,3.5903999E-3,1.3220676E-3,-1.0892714E-2,1.5316E4,6E0,1.0460138E4,1E0,3.4246575E-3,3.171E3,4.42384E5,7.4259315E0,3.79E2,8.61E2,3.09627E5,8E0,8.92686E5,-3.2422596E-3,1.0824417E0,4.7563137E1,7.6220883E-3,4.123E3,8.375588E-3,4.57091E5,1.1630708E-2,3.4453398E-1,1.1968015E8,5.5306E4,7.696629E0,1.600814E6,-1.5540857E-2,2.3066575E6,4.1E2,-1.1526485E-2,3.361876E-3,1.0947065E-2,4E0,1.46E2,1E0,4E0,3.79E2,-1.4482278E-2,2.368E0,8.182648E6,-1.3413034E-2,-1.252699E-3,4.1485362E5,4.762529E-1,1.9044118E0,5.2E1,2.7547675E2,-6.615017E-3,9.253961E-3,2.2701945E-3,4.1844E4,4.1233776E-3,1.8361429E6,2.2806E5,9.634157E-4,-8.596168E-3,1.3913782E-2,1.4964847E3,4.581188E-3,-2.9733349E-3,9.016112E-3,1.2224206E7,1.7227725E0,1.1458888E4,5.2590417E-3,4.3E0,6.8952697E-3,1.69E2,-3.4527475E-4,6.86E2,1.1490676E6,-9.827404E-3,-6.9731656E-3,4.6299936E-4,1.0885527E-2,2.9836262E-3,3.7057E5,9.9618355E8,-2.9004344E-3,-1.3556851E-2,2.4684112E8,6.748543E-1,-1.2845839E-2,2.3898147E5,-3.6260304E-3,5.1333928E-3,7.130317E7,5.077728E8,-6.88345E-3,1.0927339E7,9.5E1,1.71E2,2.691875E2,1.27E2,2.632E5,1.1070901E-2,-5.247719E-3,4.492E3,-2.0734638E-3,4.874343E-3,3.541262E-5,8.4890295E-3,-4.7148257E-3,-1.5919283E-4,-6.385449E-3,2.312633E-3,-2.3484859E-3,-1.4218546E-2,-2.6010834E-3,3.248122E-3,4.1791046E-4,6.7263497E-3,-6.286417E-4,-5.594267E-3,-8.943681E-3,-2.6225091E-3,6.9084513E-4,-2.6003746E-3,-7.52767E-3,-1.844479E-3,-2.22417E-3,2.320336E-3,2.300709E-3,-7.1390974E-3,4.180145E-3,-1.9647053E-3,-1.1054336E-3,-3.772728E-3,-1.8053204E-2,-3.4203161E-3,-7.1593275E-4,3.989031E-3,-1.955015E-3,1.5468494E-3,3.5773893E-3,1.0152213E-2,8.935426E-3,1.6707452E-3,-4.405424E-3,1.8184873E-3],"split_indices":[109,4,51,26,50,16,52,7,3,38,2,30,3,40,7,51,51,8,44,0,0,0,0,9,8,49,82,56,2,2,52,1,2,9,17,1,0,52,56,0,2,0,12,0,41,7,28,55,11,0,44,0,0,0,0,0,28,8,6,1,0,57,44,0,0,27,26,52,8,54,0,0,0,9,0,47,1,0,0,0,4,0,0,0,49,38,51,0,53,0,10,0,28,27,0,0,0,0,0,9,31,0,0,7,26,0,32,0,0,7,7,0,9,8,0,54,8,32,0,0,2,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.217E3,8.49E2,1.368E3,8.16E2,3.3E1,9.24E2,4.44E2,5.01E2,3.15E2,2.1E1,1.2E1,3.07E2,6.17E2,1.56E2,2.88E2,1.91E2,3.1E2,1.9E1,2.96E2,1.5E1,6E0,9E0,3E0,2.49E2,5.8E1,4.1E1,5.76E2,8.2E1,7.4E1,2.71E2,1.7E1,7.4E1,1.17E2,2.9E1,2.81E2,1.3E1,6E0,2.52E2,4.4E1,1E1,2.39E2,2.5E1,3.3E1,5E0,3.6E1,5.62E2,1.4E1,5.6E1,2.6E1,2E0,7.2E1,2.68E2,3E0,7E0,1E1,3.9E1,3.5E1,4.6E1,7.1E1,2.6E1,3E0,1.8E1,2.63E2,1.1E1,2E0,6E0,2.46E2,3.7E1,7E0,2.32E2,7E0,7E0,2.6E1,2E1,1.6E1,3.1E2,2.52E2,2E0,1.2E1,1.1E1,4.5E1,2.1E1,5E0,1E1,6.2E1,5.2E1,2.16E2,2E0,3.7E1,1.5E1,2E1,1.7E1,2.9E1,6.5E1,6E0,1.1E1,1.5E1,1.4E1,4E0,8E1,1.83E2,3E0,3E0,5.8E1,1.88E2,7E0,3E1,3E0,4E0,8E1,1.52E2,5E0,1.5E1,3.03E2,7E0,6.9E1,1.83E2,2.5E1,2E1,9E0,5.3E1,4.9E1,3E0,2.09E2,7E0,2.4E1,1.3E1,7E0,1.3E1,5E0,2.4E1,5.1E1,1.4E1,5.4E1,2.6E1,1.72E2,1.1E1,1.6E1,4.2E1,5.4E1,1.34E2,2.3E1,7E0,5.8E1,2.2E1,1.47E2,5E0,8E0,7E0,2.57E2,4.6E1,2E0,5E0,2.1E1,4.8E1,1.29E2,5.4E1,1.9E1,6E0,4E0,4.9E1,7E0,2.02E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[-1.4290494E-3,-1.1163305E-2,6.1064437E-2,-1.3452504E-3,-7.927302E-2,6.813854E-2,-2.0388897E-1,-1.4651097E-2,3.981182E-2,-5.9380114E-2,-1.748633E-1,2.1593547E-1,5.818057E-2,-1.1765307E-3,-1.3828149E-2,-4.4301584E-2,-3.5725527E-3,3.1075437E-2,1.6226453E-1,-3.1861488E-2,-1.20561205E-1,-2.3066878E-1,-3.69231E-2,1.1540174E-2,7.1748113E-4,5.1097896E-2,1.009046E-2,-2.481065E-2,-1.08311534E-1,1.4387006E-2,-3.942176E-2,1.06978575E-2,6.163848E-2,2.095473E-3,9.4469655E-3,-1.1527771E-2,-1.1278465E-1,-9.777071E-2,-1.6335685E-2,-2.6312652E-1,-1.0071818E-3,-4.8221597E-3,6.622522E-3,2.4541235E-2,9.171832E-2,-4.607729E-2,2.3212178E-2,2.9281145E-2,-1.3794151E-1,9.336707E-3,1.119759E-2,-4.62386E-2,1.9218642E-1,8.657725E-3,5.857641E-3,1.1527449E-1,2.6746996E-2,-2.846821E-2,3.5975436E-3,-6.30617E-3,1.6748607E-3,-1.0322032E-2,-7.237574E-2,-4.0852115E-3,-1.4169627E-2,3.2683294E-2,-1.0685904E-2,1.2126772E-1,1.1115811E-2,-1.0783391E-2,-9.2850395E-2,3.2957584E-2,-6.042197E-3,3.6013708E-3,-2.688102E-3,-2.0220502E-1,-4.8977565E-2,1.5576373E-2,-1.4143062E-1,-1.2515974E-1,-3.8608827E-2,1.3794638E-2,3.8551309E-3,7.066267E-2,-8.458831E-3,1.4599612E-1,-1.977835E-2,1.127274E-1,3.46226E-3,-4.4866065E-3,-1.1390314E-1,3.08888E-2,-1.175145E-1,2.0847754E-3,7.2700694E-2,1.0447521E-1,1.3571673E-2,5.2942676E-3,-1.9174115E-3,5.9282984E-6,-1.3090131E-2,-4.5560165E-3,3.5411606E-3,2.5896966E-3,-6.769951E-4,-1.2097667E-2,-1.7698779E-3,-4.350446E-4,-7.7569713E-3,1.2116284E-3,-9.781681E-4,-1.070818E-2,-2.3187636E-3,-6.963555E-3,3.2083488E-3,3.8302406E-3,-1.95111E-3,-6.1466694E-3,4.0987628E-3,-1.2320446E-2,1.4868734E-5,4.0020226E-3,9.159526E-3,-3.0394867E-3,4.152513E-3,8.3244005E-3,1.0503888E-3,2.6511056E-3,-1.7611679E-3,-1.2905349E-3,3.372442E-3,-8.749498E-3,-2.7355547E-3,3.784964E-3,-3.2098799E-3,-1.6557581E-4,-6.841552E-3,4.1136737E-3,-7.0013636E-4,2.231137E-3,6.8891365E-3,5.1467875E-3,-3.5377967E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,-1,45,47,49,51,53,55,-1,-1,57,59,61,-1,63,-1,-1,-1,65,67,69,71,73,75,-1,77,79,81,-1,83,85,87,89,-1,-1,-1,-1,91,-1,-1,93,-1,95,97,99,101,103,-1,-1,-1,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,129,131,133,135,137,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3450856E0,1.2791258E0,5.72436E-1,9.167678E-1,4.4789457E-1,4.1571808E-1,1.365816E-1,4.1537428E-1,4.3250322E-1,3.3456624E-1,3.0709362E-1,1.3628203E-1,3.0822754E-1,0E0,0E0,4.2649186E-1,5.941839E-1,2.3787329E-1,1.356352E-1,2.2906542E-1,3.0420876E-1,1.9032896E-1,1.8130963E-1,0E0,0E0,2.8253728E-1,0E0,2.7170983E-1,3.3018845E-1,3.73983E-1,4.9516776E-1,2.0073073E-1,2.832471E-1,0E0,0E0,1.7644046E-1,1.1630747E-1,1.7530096E-1,0E0,1.8722236E-1,0E0,0E0,0E0,3.4928653E-1,2.4635464E-1,3.0274692E-1,1.2959771E-1,7.1566924E-2,3.6742747E-1,0E0,4.0720838E-1,1.758129E-1,8.243421E-2,0E0,2.1027587E-1,2.5183016E-1,1.8780066E-1,1.9618016E-1,0E0,0E0,0E0,0E0,2.3669153E-1,0E0,0E0,1.9209073E-1,0E0,1.9648898E-1,1.6903552E-1,3.2206434E-1,1.13036335E-1,9.1452636E-2,0E0,0E0,0E0,3.9247453E-1,1.3804609E-1,2.4596436E-1,1.3371909E-1,1.5691656E-1,1.5567201E-1,0E0,0E0,1.8046097E-1,4.310426E-1,1.4106739E-1,6.952033E-2,1.2311259E-1,1.7732416E-1,1.462866E-1,7.744491E-2,9.1775246E-2,1.3334376E-1,1.4162222E-1,1.19053096E-1,1.2824386E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,39,39,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,54,54,55,55,56,56,57,57,62,62,65,65,67,67,68,68,69,69,70,70,71,71,75,75,76,76,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,-1,46,48,50,52,54,56,-1,-1,58,60,62,-1,64,-1,-1,-1,66,68,70,72,74,76,-1,78,80,82,-1,84,86,88,90,-1,-1,-1,-1,92,-1,-1,94,-1,96,98,100,102,104,-1,-1,-1,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,130,132,134,136,138,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,6.975062E-1,1.5265896E8,9.9625344E5,7.8351054E11,2.060792E8,1.272E3,1.5288235E2,7.780377E6,1E0,2.1907706E12,8.35471E-2,1.6828056E1,-1.1765307E-3,-1.3828149E-2,4.21E2,1.3446785E7,5.314225E2,1E0,6.747114E7,4.7772845E6,9.075E3,2.2065516E5,1.1540174E-2,7.1748113E-4,6.2105E4,1.009046E-2,7.08E2,7.102631E1,1.3403519E-6,7.26E2,2.2E1,1.0993947E1,2.095473E-3,9.4469655E-3,5.5012004E9,2.1109243E-1,4.36E2,-1.6335685E-2,1.4410892E-5,-1.0071818E-3,-4.8221597E-3,6.622522E-3,2.62797E5,5.21929E5,4.212173E-2,1.2737473E1,3.7930825E5,3.2449896E0,9.336707E-3,1E0,1.56221E5,1.27365E4,8.657725E-3,9.489796E-1,1.1253046E9,1.6470588E0,1.25218E5,3.5975436E-3,-6.30617E-3,1.6748607E-3,-1.0322032E-2,6.504065E-2,-4.0852115E-3,-1.4169627E-2,4.4839956E3,-1.0685904E-2,3.832274E2,2.3058404E7,3.124379E2,1.3332031E1,3.7991562E0,-6.042197E-3,3.6013708E-3,-2.688102E-3,1.568E3,3.51875E5,3.6848484E1,2.402216E6,6.0078123E-6,2.22747E5,1.3794638E-2,3.8551309E-3,4.9375E1,2.2222222E-1,3.171E3,5.3641737E-1,2.9251662E6,3.7129E4,5.674727E2,2.4019E6,1.9859155E0,1.3298E4,4.961E3,4.7932118E8,3.3027109E10,1.3571673E-2,5.2942676E-3,-1.9174115E-3,5.9282984E-6,-1.3090131E-2,-4.5560165E-3,3.5411606E-3,2.5896966E-3,-6.769951E-4,-1.2097667E-2,-1.7698779E-3,-4.350446E-4,-7.7569713E-3,1.2116284E-3,-9.781681E-4,-1.070818E-2,-2.3187636E-3,-6.963555E-3,3.2083488E-3,3.8302406E-3,-1.95111E-3,-6.1466694E-3,4.0987628E-3,-1.2320446E-2,1.4868734E-5,4.0020226E-3,9.159526E-3,-3.0394867E-3,4.152513E-3,8.3244005E-3,1.0503888E-3,2.6511056E-3,-1.7611679E-3,-1.2905349E-3,3.372442E-3,-8.749498E-3,-2.7355547E-3,3.784964E-3,-3.2098799E-3,-1.6557581E-4,-6.841552E-3,4.1136737E-3,-7.0013636E-4,2.231137E-3,6.8891365E-3,5.1467875E-3,-3.5377967E-3],"split_indices":[2,26,44,27,30,7,0,51,27,111,30,26,34,0,0,0,44,4,109,44,27,2,32,0,0,10,0,2,46,37,8,0,55,0,0,5,38,2,0,38,0,0,0,2,10,41,53,27,53,0,67,1,32,0,52,5,53,28,0,0,0,0,56,0,0,51,0,55,44,4,55,57,0,0,0,0,28,53,7,36,1,0,0,46,57,2,26,27,9,51,31,52,9,10,31,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.209E3,1.912E3,2.97E2,1.672E3,2.4E2,2.9E2,7E0,1.264E3,4.08E2,2E2,4E1,1.7E1,2.73E2,3E0,4E0,3.43E2,9.21E2,3.82E2,2.6E1,1.39E2,6.1E1,2.8E1,1.2E1,1.4E1,3E0,2.63E2,1E1,2.64E2,7.9E1,6.14E2,3.07E2,2.3E2,1.52E2,8E0,1.8E1,1.12E2,2.7E1,5.7E1,4E0,2.4E1,4E0,9E0,3E0,1.6E2,1.03E2,1.83E2,8.1E1,1.4E1,6.5E1,9E0,6.05E2,2.99E2,8E0,5E0,2.25E2,5.9E1,9.3E1,9.5E1,1.7E1,2.3E1,4E0,8E0,4.9E1,6E0,1.8E1,1.56E2,4E0,7.5E1,2.8E1,1.05E2,7.8E1,7.7E1,4E0,9E0,5E0,3.7E1,2.8E1,5.89E2,1.6E1,2.5E1,2.74E2,3E0,5E0,4E1,1.85E2,4.8E1,1.1E1,1.9E1,7.4E1,7.5E1,2E1,1.5E1,3.4E1,8.9E1,6.7E1,7E1,5E0,9E0,1.9E1,1.02E2,3E0,7.5E1,3E0,5.1E1,2.6E1,2.6E1,1.1E1,2.2E1,6E0,4.53E2,1.36E2,7E0,9E0,2.2E1,3E0,9E0,2.65E2,3E0,3.7E1,5E0,1.8E2,2.5E1,2.3E1,8E0,3E0,1E1,9E0,3.2E1,4.2E1,5.8E1,1.7E1,7E0,1.3E1,1E1,5E0,8E0,2.6E1,1.4E1,7.5E1,5.3E1,1.4E1,6.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[-8.294164E-4,-1.055248E-2,6.452846E-2,-2.536246E-3,-8.548448E-2,7.838636E-2,-1.08478956E-1,-1.7412623E-2,3.5756767E-2,-1.3679624E-2,-7.549093E-2,6.735803E-2,2.912265E-1,-6.1424777E-2,-1.747637E-2,1.5308867E-2,-3.2957904E-2,1.5380347E-1,2.3610312E-2,-1.09643996E-1,-4.013298E-2,1.5868095E-1,5.325876E-2,3.1272094E-3,1.4729368E-2,-5.2680094E-3,1.6081603E-3,5.2181616E-2,-1.5404522E-2,-3.928895E-2,2.5526488E-2,2.0811832E-1,1.7636792E-3,4.3679215E-2,-1.847616E-2,-8.638547E-2,-1.7194375E-1,-2.3655967E-4,-5.3644725E-3,3.9995134E-2,9.046098E-3,3.835525E-2,1.2122862E-1,6.0061075E-2,-1.10330075E-1,-2.278878E-1,-4.4616363E-3,-4.0445726E-2,1.2395954E-2,1.2985472E-1,-2.8000625E-2,2.3022032E-1,2.1343878E-3,1.3287498E-1,3.086907E-2,-3.2045577E-2,1.22814454E-1,-1.9136474E-1,-3.0536905E-3,-9.83474E-2,-1.703847E-2,4.112104E-3,-4.929818E-3,5.8870944E-3,2.7758658E-2,1.5666164E-1,6.1447173E-4,-4.0065005E-2,7.582296E-2,-7.985167E-3,7.3882454E-4,-4.65247E-3,-1.9067856E-2,5.00962E-3,-2.5195323E-2,-3.889848E-2,-1.2308657E-2,1.3473754E-4,7.070151E-3,-1.0733338E-1,3.7332255E-2,7.5938953E-3,1.4473042E-2,2.2079176E-1,5.889952E-2,4.277937E-2,-3.4238245E-2,-9.2802785E-2,-2.1200448E-2,7.7840732E-3,4.454352E-4,1.0045487E-4,-1.0867467E-2,-5.1110727E-4,-6.709173E-3,3.40373E-2,-3.9363857E-3,2.9462844E-3,2.1410283E-1,1.805154E-3,-9.730036E-3,4.2302073E-3,1.1124593E-3,-6.8785655E-3,-1.5516984E-5,-5.186952E-4,-2.2598058E-3,-2.1152834E-3,-9.807717E-3,-1.8892713E-3,3.172975E-3,1.1052974E-2,1.3797838E-3,-3.113097E-3,3.7430471E-3,7.346521E-3,1.5193736E-3,-9.221713E-3,-7.235223E-4,2.123132E-3,-5.4425593E-3,1.1614446E-3,-2.0636579E-3,2.1181416E-3,-1.5149487E-3,5.1209433E-3,1.2839057E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,35,37,39,41,-1,-1,-1,-1,43,45,47,49,51,-1,53,55,57,59,-1,-1,61,-1,63,65,67,69,71,73,75,-1,77,79,81,-1,83,85,87,89,91,-1,93,-1,-1,-1,-1,95,97,-1,99,101,-1,-1,-1,-1,-1,103,105,-1,-1,-1,107,109,-1,-1,111,113,115,117,119,121,-1,-1,-1,-1,-1,-1,123,-1,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4279661E0,1.1746985E0,7.0618176E-1,1.007784E0,3.8642824E-1,6.141466E-1,2.6359102E-1,6.485435E-1,7.0612377E-1,0E0,2.1537971E-1,3.2374644E-1,7.6043725E-2,1.1064462E-1,0E0,4.6625602E-1,3.2091385E-1,2.788378E-1,3.808902E-1,1.18454576E-1,2.4828869E-1,1.5796953E-1,2.2327292E-1,0E0,0E0,0E0,0E0,2.4608088E-1,5.2214885E-1,2.9356933E-1,4.7830915E-1,9.944022E-2,0E0,3.443945E-1,2.8385144E-1,1.24259055E-1,3.2401818E-1,0E0,0E0,8.557067E-2,0E0,1.7708364E-1,1.5045726E-1,2.8553128E-1,7.815835E-2,2.2274369E-1,5.159316E-1,2.6800454E-1,0E0,9.633899E-2,2.9944012E-1,9.730959E-2,0E0,2.3413539E-1,2.0928463E-1,8.632778E-2,7.044768E-2,9.475601E-2,0E0,8.077294E-2,0E0,0E0,0E0,0E0,1.2395114E-1,1.4388913E-1,0E0,3.579922E-1,1.3761944E-1,0E0,0E0,0E0,0E0,0E0,5.7634974E-1,2.3339713E-1,0E0,0E0,0E0,1.6211984E-1,8.578575E-2,0E0,0E0,6.808478E-2,7.2460674E-2,2.4463218E-1,1.2667897E-1,8.222309E-2,1.325404E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3923892E-1,0E0,0E0,9.4234645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,59,59,64,64,65,65,67,67,68,68,74,74,75,75,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,95,95,98,98],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,36,38,40,42,-1,-1,-1,-1,44,46,48,50,52,-1,54,56,58,60,-1,-1,62,-1,64,66,68,70,72,74,76,-1,78,80,82,-1,84,86,88,90,92,-1,94,-1,-1,-1,-1,96,98,-1,100,102,-1,-1,-1,-1,-1,104,106,-1,-1,-1,108,110,-1,-1,112,114,116,118,120,122,-1,-1,-1,-1,-1,-1,124,-1,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2679E4,4.5843E7,1.2367184E3,5.862126E2,2.0008473E2,2.4507338E4,9.599108E6,2E0,1.8372351E9,-1.3679624E-2,1.067536E3,2.495333E6,1.08874E5,3.0036328E7,-1.747637E-2,1.4702222E0,9.750871E6,4.1179886E0,1.0938637E1,1.3888E4,1.13808104E8,2.0754387E1,4.499888E0,3.1272094E-3,1.4729368E-2,-5.2680094E-3,1.6081603E-3,6.346204E-7,1.2859046E-4,1.3238013E2,1.40673E5,2.8366232E0,1.7636792E-3,1.1317E4,4.4854636E7,4.30453E2,8.657441E7,-2.3655967E-4,-5.3644725E-3,1.101214E6,9.046098E-3,2.1402586E4,3.7134724E0,1.1556145E2,9.393264E-6,4.6E1,4.1E0,6.6315195E6,1.2395954E-2,1.2E2,2.657E3,3.689E3,2.1343878E-3,3.392E3,2.342147E7,2.47E2,1.23E2,1.08504E5,-3.0536905E-3,6.747114E7,-1.703847E-2,4.112104E-3,-4.929818E-3,5.8870944E-3,3.430961E7,1.2516333E7,6.1447173E-4,1.07882355E2,1.6358411E8,-7.985167E-3,7.3882454E-4,-4.65247E-3,-1.9067856E-2,5.00962E-3,1.9913513E2,4.911E3,-1.2308657E-2,1.3473754E-4,7.070151E-3,3.673719E8,6.2E1,7.5938953E-3,1.4473042E-2,3.9712732E7,2.97271E5,1.125584E-3,1.4593E4,1.01644066E2,2.8169013E-3,7.7840732E-3,4.454352E-4,1.0045487E-4,-1.0867467E-2,-5.1110727E-4,-6.709173E-3,1.0033929E1,-3.9363857E-3,2.9462844E-3,1.9563605E5,1.805154E-3,-9.730036E-3,4.2302073E-3,1.1124593E-3,-6.8785655E-3,-1.5516984E-5,-5.186952E-4,-2.2598058E-3,-2.1152834E-3,-9.807717E-3,-1.8892713E-3,3.172975E-3,1.1052974E-2,1.3797838E-3,-3.113097E-3,3.7430471E-3,7.346521E-3,1.5193736E-3,-9.221713E-3,-7.235223E-4,2.123132E-3,-5.4425593E-3,1.1614446E-3,-2.0636579E-3,2.1181416E-3,-1.5149487E-3,5.1209433E-3,1.2839057E-2],"split_indices":[2,44,57,51,51,4,28,16,30,0,51,1,2,42,0,52,9,52,52,10,44,57,34,0,0,0,0,36,40,55,1,37,0,9,9,32,44,0,0,1,0,31,40,51,36,0,57,27,0,0,2,2,0,11,44,0,3,7,0,44,0,0,0,0,46,31,0,51,7,0,0,0,0,0,54,28,0,0,0,7,8,0,0,44,1,37,28,32,56,0,0,0,0,0,0,52,0,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.245E3,1.955E3,2.9E2,1.767E3,1.88E2,2.69E2,2.1E1,1.273E3,4.94E2,7E0,1.81E2,2.57E2,1.2E1,1.9E1,2E0,4.1E2,8.63E2,4.5E1,4.49E2,9.1E1,9E1,3.3E1,2.24E2,2E0,1E1,1.2E1,7E0,1.86E2,2.24E2,7.79E2,8.4E1,3E1,1.5E1,3.04E2,1.45E2,6.8E1,2.3E1,6.3E1,2.7E1,9E0,2.4E1,1.85E2,3.9E1,1.78E2,8E0,1E1,2.14E2,7.77E2,2E0,2.8E1,5.6E1,2.6E1,4E0,3.7E1,2.67E2,1.33E2,1.2E1,9E0,5.9E1,1.8E1,5E0,7E0,2E0,1.8E1,1.67E2,2.9E1,1E1,2.4E1,1.54E2,5E0,3E0,7E0,3E0,3.2E1,1.82E2,7.73E2,4E0,5E0,2.3E1,2.5E1,3.1E1,1.7E1,9E0,1.6E1,2.1E1,2.26E2,4.1E1,1.9E1,1.14E2,8E0,4E0,2E0,7E0,7E0,1.1E1,1.59E2,8E0,1.2E1,1.7E1,1.7E1,7E0,1.14E2,4E1,2.9E1,1.53E2,2.22E2,5.51E2,1.7E1,8E0,9E0,2.2E1,1.4E1,2E0,3E0,1.8E1,1.5E1,2.11E2,3E0,3.8E1,3E0,1.6E1,3.9E1,7.5E1,1.34E2,2.5E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[-4.5639044E-3,9.861498E-3,-4.6488423E-2,-1.8874628E-2,3.5280757E-2,-6.775051E-2,-3.3807806E-3,-2.6980061E-2,4.1029625E-2,5.0118126E-2,-3.37109E-2,-6.1412714E-2,-2.4243484E-1,9.618622E-3,-8.524699E-2,-2.5039298E-2,-2.0818483E-2,-7.062103E-2,5.686838E-2,4.6202768E-2,2.2024027E-1,-6.286647E-2,3.5242796E-2,-9.5200166E-2,-3.985757E-2,-3.0228063E-1,-1.4583494E-3,-2.5538113E-2,3.7410177E-2,-3.0108698E-2,-1.1953371E-2,-2.2552043E-2,-3.0192587E-1,7.225658E-4,-1.1217651E-2,1.4048779E-1,2.3866469E-2,5.56165E-2,-3.1663243E-2,1.5564955E-2,5.5554096E-4,-1.4710947E-2,-5.500478E-2,-1.9348647E-3,6.962011E-2,-5.6446627E-2,-1.4786845E-1,1.0983103E-2,-7.212348E-2,-1.273986E-3,-1.6629053E-2,-3.731783E-2,6.4347703E-3,7.900161E-2,2.4061112E-4,1.9738018E-3,-3.568473E-3,-7.7622836E-3,-6.0497522E-2,-2.4688637E-2,-1.9875853E-3,1.0244732E-3,8.183455E-3,3.8869347E-2,-7.851091E-2,8.42027E-3,5.127232E-2,-6.5843016E-2,5.1447663E-2,-4.5323104E-2,-2.4538264E-1,3.850631E-3,-2.8759192E-3,-8.366918E-3,-3.6603324E-2,-1.8544462E-1,-6.925344E-2,-5.053696E-2,4.8138443E-2,-5.8655027E-2,-2.2351508E-1,-4.590212E-3,-3.4991098E-3,1.0058013E-1,-9.0027094E-4,-1.3207443E-4,-1.1672708E-1,-1.0507203E-1,3.4625067E-3,-1.5944303E-3,6.904089E-2,-9.853431E-3,2.1955464E-3,5.257097E-2,-1.04270335E-2,-4.7924966E-2,-1.8186322E-1,4.730685E-3,-1.5916066E-2,3.8794314E-3,-2.266546E-3,-1.5729161E-2,-2.786574E-3,1.0818294E-1,-4.866295E-2,-9.549104E-3,-2.588997E-3,1.4299126E-3,-1.0273898E-1,9.3058473E-4,-9.5524766E-2,7.4998066E-3,2.5101406E-2,-4.964449E-2,-2.3509702E-1,-1.6024621E-2,1.717688E-4,-3.090515E-2,4.5836847E-3,6.630954E-3,8.080048E-3,-1.9468763E-3,9.550051E-4,1.9392093E-3,-1.1011022E-2,-6.024005E-3,-1.0931493E-3,-6.324895E-4,1.1964171E-2,5.617791E-3,1.6051824E-3,4.406305E-3,2.053236E-3,-3.973541E-3,2.2460283E-4,-1.1032179E-2,-8.249231E-4,1.5387727E-3,-7.258456E-3,-9.7102224E-4,1.1090434E-2,-3.4208775E-3,2.2258535E-4,-1.9572556E-3,-1.0187063E-2,-8.934089E-3,-8.0105016E-4,2.0537062E-3,-2.554543E-3,-1.7359044E-3,-9.450232E-3,-1.5687643E-2,-3.7068463E-4,3.6011122E-3,-2.458556E-3,-5.209856E-3,2.8032458E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,-1,51,53,55,-1,57,59,-1,-1,61,63,65,67,-1,-1,-1,69,-1,71,73,75,77,79,-1,-1,81,-1,83,-1,-1,-1,85,87,-1,-1,-1,-1,89,91,-1,93,95,97,99,101,-1,-1,-1,103,105,107,109,111,113,115,117,-1,119,-1,121,123,125,127,-1,129,-1,-1,131,-1,133,135,-1,137,-1,-1,-1,-1,139,141,-1,-1,-1,143,-1,145,-1,147,149,151,-1,-1,153,-1,-1,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3442752E0,1.208832E0,5.212928E-1,3.7837917E-1,9.004582E-1,4.0692186E-1,2.0195597E-1,5.639312E-1,1.6847801E-1,4.7140646E-1,3.1583554E-1,2.6494575E-1,1.5381563E-1,1.6119328E-1,2.4834336E-1,4.6356443E-1,0E0,1.8800987E-1,2.229343E-1,5.2073205E-1,4.069113E-1,2.0875764E-1,1.2896965E-1,2.835282E-1,3.7316766E-1,1.6790509E-1,0E0,1.4948474E-1,1.22391224E-1,7.8831784E-2,0E0,3.8002706E-1,3.5286474E-1,0E0,0E0,1.05062574E-1,9.520899E-2,3.483938E-1,2.2212228E-1,0E0,0E0,0E0,1.8794158E-1,0E0,7.32051E-2,2.0719042E-1,1.6440034E-1,2.0586887E-1,2.7031225E-1,0E0,0E0,8.9904934E-2,0E0,8.735305E-2,0E0,0E0,0E0,4.06551E-1,5.420513E-1,0E0,0E0,0E0,0E0,1.2090292E-1,1.51916E-1,0E0,2.3626709E-1,1.0562129E-1,8.384541E-2,6.887041E-2,7.24076E-2,0E0,0E0,0E0,1.3447353E-1,1.18628144E-1,7.4153066E-2,1.1070396E-1,1.4744551E-1,1.9341782E-1,3.2747728E-1,1.11744046E-1,0E0,1.3887388E-1,0E0,4.176686E-1,6.562221E-1,2.5043535E-1,3.628618E-1,0E0,6.585088E-2,0E0,0E0,1.9300544E-1,0E0,1.0368145E-1,6.636657E-2,0E0,8.472057E-2,0E0,0E0,0E0,0E0,1.1163906E-1,9.980741E-2,0E0,0E0,0E0,1.0668883E-1,0E0,1.6291146E-1,0E0,8.184285E-2,2.1495792E-1,1.4883986E-1,0E0,0E0,8.701696E-2,0E0,0E0,8.6688586E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,35,35,36,36,37,37,38,38,42,42,44,44,45,45,46,46,47,47,48,48,51,51,53,53,57,57,58,58,63,63,64,64,66,66,67,67,68,68,69,69,70,70,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,83,83,85,85,86,86,87,87,88,88,90,90,93,93,95,95,96,96,98,98,103,103,104,104,108,108,110,110,112,112,113,113,114,114,117,117,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,-1,52,54,56,-1,58,60,-1,-1,62,64,66,68,-1,-1,-1,70,-1,72,74,76,78,80,-1,-1,82,-1,84,-1,-1,-1,86,88,-1,-1,-1,-1,90,92,-1,94,96,98,100,102,-1,-1,-1,104,106,108,110,112,114,116,118,-1,120,-1,122,124,126,128,-1,130,-1,-1,132,-1,134,136,-1,138,-1,-1,-1,-1,140,142,-1,-1,-1,144,-1,146,-1,148,150,152,-1,-1,154,-1,-1,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,3.7810526E2,4.0281758E6,2.3920168E6,2.2893274E1,5.10034E5,1E0,4.5158855E6,1.4431512E-1,4.5505118E2,7.06699E8,7.2474093E-3,3.454E3,3.2627738E5,1.8544E4,2.298425E6,-2.0818483E-2,6.87E2,2.1383E4,3.5947604E7,3.1029554E7,7.854E3,1.22E2,2.8277853E11,4.50847E5,8E0,-1.4583494E-3,3.0612E4,1.2015E4,1.8662969E6,-1.1953371E-2,2.9676E4,4.06E2,7.225658E-4,-1.1217651E-2,6E2,6.6315195E6,2.3E1,1.7495675E5,1.5564955E-2,5.5554096E-4,-1.4710947E-2,3.3720784E5,-1.9348647E-3,5.8921628E10,4.0701206E1,1.8606549E3,1.34E3,2.0211798E6,-1.273986E-3,-1.6629053E-2,5.188105E10,6.4347703E-3,1.3849624E1,2.4061112E-4,1.9738018E-3,-3.568473E-3,1.0008265E1,2.373E3,-2.4688637E-2,-1.9875853E-3,1.0244732E-3,8.183455E-3,6.088569E6,5.03E2,8.42027E-3,3.7826266E4,8.6206274E2,2.5727873E5,1.5052E4,2.4063041E4,3.850631E-3,-2.8759192E-3,-8.366918E-3,1.9523809E0,1.3439535E1,2E0,2.4626505E0,6.439778E5,5.3296334E-1,9.35E2,7.089473E6,-3.4991098E-3,4.9122E5,-9.0027094E-4,3.5E2,1.1556145E2,1.1041459E-1,4.33E2,-1.5944303E-3,1.1055555E1,-9.853431E-3,2.1955464E-3,1.710933E0,-1.04270335E-2,1.819795E6,2.9816154E1,4.730685E-3,1.0828989E8,3.8794314E-3,-2.266546E-3,-1.5729161E-2,-2.786574E-3,9E0,1E0,-9.549104E-3,-2.588997E-3,1.4299126E-3,9.403226E0,9.3058473E-4,6.798644E-1,7.4998066E-3,1.9577E4,2.93446E5,4.2378342E-1,-1.6024621E-2,1.717688E-4,5.348301E6,4.5836847E-3,6.630954E-3,3.4976208E-1,-1.9468763E-3,9.550051E-4,1.9392093E-3,-1.1011022E-2,-6.024005E-3,-1.0931493E-3,-6.324895E-4,1.1964171E-2,5.617791E-3,1.6051824E-3,4.406305E-3,2.053236E-3,-3.973541E-3,2.2460283E-4,-1.1032179E-2,-8.249231E-4,1.5387727E-3,-7.258456E-3,-9.7102224E-4,1.1090434E-2,-3.4208775E-3,2.2258535E-4,-1.9572556E-3,-1.0187063E-2,-8.934089E-3,-8.0105016E-4,2.0537062E-3,-2.554543E-3,-1.7359044E-3,-9.450232E-3,-1.5687643E-2,-3.7068463E-4,3.6011122E-3,-2.458556E-3,-5.209856E-3,2.8032458E-3],"split_indices":[26,51,31,27,52,11,111,46,57,55,7,40,0,27,9,27,0,2,9,44,46,9,8,30,31,3,0,2,9,46,0,28,0,0,0,2,27,0,32,0,0,0,27,0,30,51,4,2,27,0,0,30,0,53,0,0,0,53,2,0,0,0,0,44,0,0,51,57,32,9,32,0,0,0,53,53,8,57,50,37,0,47,0,28,0,2,51,38,8,0,57,0,0,53,0,1,55,0,44,0,0,0,0,3,15,0,0,0,57,0,26,0,9,11,26,0,0,12,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.221E3,1.653E3,5.68E2,7.76E2,8.77E2,3.8E2,1.88E2,6.84E2,9.2E1,7.22E2,1.55E2,3.68E2,1.2E1,1.63E2,2.5E1,6.82E2,2E0,1.1E1,8.1E1,7.07E2,1.5E1,1.09E2,4.6E1,1.42E2,2.26E2,9E0,3E0,7.2E1,9.1E1,2E1,5E0,6.77E2,5E0,8E0,3E0,2.2E1,5.9E1,6.31E2,7.6E1,9E0,6E0,2E0,1.07E2,1.4E1,3.2E1,8.3E1,5.9E1,8.8E1,1.38E2,2E0,7E0,6.8E1,4E0,3.9E1,5.2E1,8E0,1.2E1,4.88E2,1.89E2,2E0,3E0,6E0,1.6E1,5.2E1,7E0,1.9E1,6.12E2,5.4E1,2.2E1,1.03E2,4E0,2.9E1,3E0,1E1,7.3E1,3.9E1,2E1,3.3E1,5.5E1,1.28E2,1E1,3.8E1,3E1,3.2E1,7E0,4.57E2,3.1E1,1.11E2,7.8E1,1.5E1,3.7E1,3E0,4E0,6.1E2,2E0,4.8E1,6E0,1.2E1,1E1,3E0,1E2,2E0,2E0,5E0,6.8E1,3.2E1,7E0,5E0,1.5E1,1.3E1,2E1,8E0,4.7E1,1.23E2,5E0,6E0,4E0,3.1E1,7E0,2.1E1,1.1E1,1.51E2,3.06E2,1.4E1,1.7E1,8.2E1,2.9E1,7.4E1,4E0,1.3E1,2.4E1,8.3E1,5.27E2,2.7E1,2.1E1,4E0,2E0,8E0,2E0,3E0,2E0,4.5E1,2.3E1,1.1E1,4E0,8E0,1.2E1,3.8E1,9E0,1.16E2,7E0,3E0,2E0,5E0,2.6E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-1.1604184E-3,1.23341065E-2,-3.8910985E-2,-6.748511E-3,4.3247756E-2,-1.00298904E-1,-2.0218384E-2,-1.3221645E-2,9.502215E-2,-2.3176247E-2,5.4692127E-2,-3.256143E-1,-8.566181E-2,-3.0903209E-2,3.3902712E-2,-3.0648068E-2,5.397539E-3,1.2675573E-1,-1.0241145E-1,5.042198E-2,-7.270404E-2,6.419896E-2,-1.6423905E-2,-2.101665E-2,-3.212877E-3,-1.0522561E-2,-6.594602E-2,-2.8089302E-2,-1.2992773E-2,5.9658576E-2,-1.0555295E-3,-8.272696E-3,-7.187922E-2,9.345352E-3,-8.838592E-3,1.391821E-2,8.780567E-2,-1.15983095E-2,-9.906011E-4,-1.3401109E-3,9.321743E-2,4.9824025E-3,-1.0545577E-1,7.256863E-2,-1.1419282E-2,-7.155175E-2,5.7010163E-2,-1.022383E-1,-8.496074E-3,-1.01924635E-1,-1.9215781E-2,7.2828025E-2,-2.4196736E-3,-3.2914463E-2,2.9332126E-2,-2.1261798E-1,-5.628152E-2,5.747002E-2,-5.5738743E-3,3.6273312E-4,1.0753155E-1,5.3288084E-3,-5.684851E-3,-6.608843E-2,-3.7001246E-1,5.569943E-2,1.10550046E-1,-8.329673E-2,3.854514E-2,-1.2155777E-1,4.4795312E-4,5.8560967E-3,5.64691E-3,-6.7384794E-2,-1.9408028E-1,3.180486E-2,-9.3905E-2,-5.11813E-2,-1.7484081E-1,-5.0464664E-3,-6.034697E-2,1.0202843E-1,9.1067085E-4,-2.5673416E-2,-2.2083895E-2,5.3094488E-2,-7.069258E-2,-1.1423984E-2,-9.2679216E-4,-3.1593055E-2,-1.17574915E-1,1.7934188E-2,1.3285138E-1,-1.4266932E-2,-2.881581E-3,-1.1206563E-3,5.6520277E-3,-1.3839916E-2,-1.7332892E-1,-2.5081012E-2,-6.3171266E-3,-1.2793781E-1,5.9925415E-2,1.2618287E-1,-1.7713772E-2,-2.5466704E-1,-9.195494E-3,7.507147E-3,-9.487239E-3,-3.6688806E-3,-1.1373896E-2,2.4990318E-3,-4.246866E-3,-3.208251E-2,-1.2402943E-1,-2.4557206E-1,-3.57142E-5,-4.0897685E-3,7.628166E-2,-6.692329E-4,-8.249808E-3,4.126648E-3,-3.7410825E-3,-1.0028811E-2,-2.7224696E-3,-1.110369E-2,9.09941E-2,-1.4090171E-1,-3.9921578E-2,5.6754537E-3,-5.5157504E-4,-8.137145E-4,-1.0309485E-2,9.08888E-3,1.5153585E-3,-1.5663257E-3,-8.114465E-3,-2.3534752E-3,2.3243774E-3,-1.3071194E-2,-2.7358718E-3,-5.2129827E-3,1.4386574E-3,7.183194E-3,8.810388E-4,-3.64E-4,6.419516E-3,-4.913663E-3,2.1140175E-3,-1.590871E-2,-1.60108E-3,-9.309954E-3,2.0013407E-3,3.4293924E-3,7.8086636E-4,3.3417866E-3,8.108372E-3,-3.4368152E-3,3.7230947E-3,-1.5089179E-2,-2.9406582E-3,4.119555E-3,-4.900148E-3,1.0796563E-3,-7.490753E-3,-1.0484733E-2,-5.603396E-4,-1.1701341E-2,-2.1711893E-3,-5.643789E-4,-1.3709395E-2,4.4775913E-3,-2.7859204E-3,-1.2262573E-3,1.3135584E-3,9.230931E-3,-3.135096E-4,-1.0594549E-2,-3.1256495E-4,1.458201E-3,-3.1428146E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,49,-1,51,-1,53,55,57,-1,-1,59,-1,-1,-1,61,-1,63,65,67,69,71,73,75,77,79,81,-1,83,85,87,89,91,93,-1,95,-1,-1,97,99,101,103,105,107,109,-1,111,-1,113,115,117,119,121,123,125,127,129,-1,131,-1,133,135,-1,-1,137,139,141,143,-1,145,-1,-1,147,149,-1,-1,151,153,155,157,159,161,-1,163,-1,-1,-1,-1,165,167,169,-1,-1,171,-1,-1,-1,-1,-1,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1375188E0,9.706993E-1,6.727021E-1,6.718893E-1,4.7866118E-1,4.2392814E-1,2.624277E-1,3.1100363E-1,3.9516312E-1,3.4266555E-1,3.634379E-1,2.4940455E-1,3.6082125E-1,2.6431808E-1,1.11916386E-1,4.560738E-1,3.693212E-1,3.4596568E-1,1.0584383E-1,1.3289325E-1,3.467943E-1,3.0018997E-1,2.6316792E-1,0E0,0E0,0E0,2.4099022E-1,2.4373373E-1,0E0,8.134702E-2,0E0,2.993465E-1,3.699987E-1,3.2772052E-1,0E0,0E0,6.996852E-2,0E0,0E0,0E0,1.4869627E-1,0E0,4.7546136E-1,2.6678753E-1,1.7559382E-1,1.5189603E-1,9.6262164E-2,2.1419394E-1,1.6132137E-1,1.390518E-1,1.9551653E-1,7.029252E-2,0E0,6.2975436E-1,3.0798674E-1,1.2455469E-1,2.3537731E-1,3.1948593E-1,2.8994897E-1,0E0,8.552003E-2,0E0,0E0,2.3853685E-1,1.7711318E-1,2.3815304E-1,2.6290143E-1,2.5163323E-1,1.7885375E-1,1.0048634E-1,0E0,8.8773705E-2,0E0,1.02004126E-1,1.889118E-1,1.807256E-1,1.0212259E-1,1.1944144E-1,7.233983E-2,1.4702691E-1,1.3587615E-1,8.146253E-2,0E0,2.9617006E-1,0E0,2.9463044E-1,9.1743596E-2,0E0,0E0,1.9544342E-1,4.2264497E-1,1.3629192E-1,1.0519457E-1,0E0,2.6010004E-1,0E0,0E0,1.7702962E-1,3.3149502E-1,0E0,0E0,1.02716535E-1,1.9519293E-1,3.0822384E-1,9.349435E-2,6.784332E-2,1.5868315E-1,0E0,1.1832494E-1,0E0,0E0,0E0,0E0,1.307998E-1,1.9212979E-1,1.963622E-1,0E0,0E0,8.089785E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5273535E-1,1.6723225E-1,2.1048126E-1,1.5116477E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26,27,27,29,29,31,31,32,32,33,33,36,36,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,83,83,85,85,86,86,89,89,90,90,91,91,92,92,94,94,97,97,98,98,101,101,102,102,103,103,104,104,105,105,106,106,108,108,113,113,114,114,115,115,118,118,125,125,126,126,127,127,128,128],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,50,-1,52,-1,54,56,58,-1,-1,60,-1,-1,-1,62,-1,64,66,68,70,72,74,76,78,80,82,-1,84,86,88,90,92,94,-1,96,-1,-1,98,100,102,104,106,108,110,-1,112,-1,114,116,118,120,122,124,126,128,130,-1,132,-1,134,136,-1,-1,138,140,142,144,-1,146,-1,-1,148,150,-1,-1,152,154,156,158,160,162,-1,164,-1,-1,-1,-1,166,168,170,-1,-1,172,-1,-1,-1,-1,-1,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,4.930349E5,1.9121015E0,3.4409692E3,1.8122449E2,3E0,3.0497742E3,2.657E3,2.3580047E10,1.1333333E0,1E0,1.36054E-1,1.1485E4,6.9998717E-4,4.4E1,1.1280869E9,7.023838E3,1.3683E4,2.5086E4,4.54E2,1.1779856E2,2E0,9.195004E6,-2.101665E-2,-3.212877E-3,-1.0522561E-2,1.3781E2,1.0558809E-4,-1.2992773E-2,9E0,-1.0555295E-3,6.62E2,1.0526649E3,2.1442623E1,-8.838592E-3,1.391821E-2,1.3532244E7,-1.15983095E-2,-9.906011E-4,-1.3401109E-3,3E0,4.9824025E-3,1.216918E7,1.6E1,8.976923E0,5.209486E2,1.61225E5,9.517604E6,1.903E3,1.2151E4,2.8216E4,6.4391124E7,-2.4196736E-3,1.9483356E1,2.3E1,6.7143274E2,1.4229508E1,6.7628815E2,6.2E1,3.6273312E-4,4.90027E1,5.3288084E-3,-5.684851E-3,8E0,4.06E2,2.86968E5,2.1938796E7,1.3860265E6,2.64E0,1.0950326E6,4.4795312E-4,1.421E3,5.64691E-3,9.9341E4,2.0588236E0,8E0,8.988928E5,5.3042426E0,1.305E3,1.3137E4,1.01813E5,1.0238709E1,9.1067085E-4,3.9E1,-2.2083895E-2,8.2884795E3,1.22871354E-1,-1.1423984E-2,-9.2679216E-4,6.172708E2,1.7812634E1,6.0607E4,9.342223E6,-1.4266932E-2,1.5137865E2,-1.1206563E-3,5.6520277E-3,4.97E2,2E0,-2.5081012E-2,-6.3171266E-3,7.88E2,5.68197E6,2.183E3,1.980198E-2,1.6118722E0,3.794679E8,7.507147E-3,2.878424E5,-3.6688806E-3,-1.1373896E-2,2.4990318E-3,-4.246866E-3,1.1057693E0,1.56221E5,1.710933E0,-3.57142E-5,-4.0897685E-3,3.4348795E6,-6.692329E-4,-8.249808E-3,4.126648E-3,-3.7410825E-3,-1.0028811E-2,-2.7224696E-3,8.3956606E5,5.3084288E8,1.2166998E0,6.699017E6,5.6754537E-3,-5.5157504E-4,-8.137145E-4,-1.0309485E-2,9.08888E-3,1.5153585E-3,-1.5663257E-3,-8.114465E-3,-2.3534752E-3,2.3243774E-3,-1.3071194E-2,-2.7358718E-3,-5.2129827E-3,1.4386574E-3,7.183194E-3,8.810388E-4,-3.64E-4,6.419516E-3,-4.913663E-3,2.1140175E-3,-1.590871E-2,-1.60108E-3,-9.309954E-3,2.0013407E-3,3.4293924E-3,7.8086636E-4,3.3417866E-3,8.108372E-3,-3.4368152E-3,3.7230947E-3,-1.5089179E-2,-2.9406582E-3,4.119555E-3,-4.900148E-3,1.0796563E-3,-7.490753E-3,-1.0484733E-2,-5.603396E-4,-1.1701341E-2,-2.1711893E-3,-5.643789E-4,-1.3709395E-2,4.4775913E-3,-2.7859204E-3,-1.2262573E-3,1.3135584E-3,9.230931E-3,-3.135096E-4,-1.0594549E-2,-3.1256495E-4,1.458201E-3,-3.1428146E-3],"split_indices":[26,27,52,51,51,3,51,2,5,55,79,41,9,33,3,30,4,9,9,2,4,6,44,0,0,0,51,38,0,16,0,2,32,55,0,0,1,0,0,0,11,0,44,3,45,51,1,44,0,9,9,50,0,53,3,47,55,51,0,0,55,0,0,8,0,7,50,27,53,46,0,2,0,1,53,3,31,55,11,2,1,52,0,8,0,46,37,0,0,54,55,1,31,0,55,0,0,2,16,0,0,0,49,2,56,52,7,0,32,0,0,0,0,53,1,53,0,0,46,0,0,0,0,0,0,27,7,56,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.231E3,1.644E3,5.87E2,1.017E3,6.27E2,1.36E2,4.51E2,9.57E2,6E1,9.2E1,5.35E2,7E0,1.29E2,3.77E2,7.4E1,4.94E2,4.63E2,5.2E1,8E0,3.7E1,5.5E1,4.72E2,6.3E1,4E0,3E0,1.4E1,1.15E2,3.74E2,3E0,5.1E1,2.3E1,3.21E2,1.73E2,4.55E2,8E0,8E0,4.4E1,2E0,6E0,1.3E1,2.4E1,8E0,4.7E1,4.25E2,4.7E1,3.6E1,2.7E1,7E1,4.5E1,3.9E1,3.35E2,4.6E1,5E0,1.94E2,1.27E2,1.6E1,1.57E2,1.07E2,3.48E2,9E0,3.5E1,2.2E1,2E0,4.2E1,5E0,2.96E2,1.29E2,1.9E1,2.8E1,2.2E1,1.4E1,1.6E1,1.1E1,5.2E1,1.8E1,3.1E1,1.4E1,2.4E1,1.5E1,2.5E2,8.5E1,2.9E1,1.7E1,1.92E2,2E0,1.03E2,2.4E1,1.3E1,3E0,1.13E2,4.4E1,7.1E1,3.6E1,2E0,3.46E2,4E0,3.1E1,2.9E1,1.3E1,2E0,3E0,6E0,2.9E2,1.15E2,1.4E1,5E0,1.4E1,7E0,2.1E1,1.8E1,4E0,1.1E1,5E0,3.3E1,1.9E1,1.4E1,4E0,8E0,2.3E1,8E0,6E0,4E0,2E1,1E1,5E0,2.36E2,1.4E1,1.6E1,6.9E1,2.4E1,5E0,1.86E2,6E0,1.1E1,9.2E1,1.9E1,5E0,9.1E1,2.2E1,1E1,3.4E1,6E0,6.5E1,2.9E1,7E0,3.35E2,1.1E1,1.1E1,1.8E1,5E0,8E0,4E0,2E0,2.11E2,7.9E1,5.9E1,5.6E1,9E0,5E0,3E0,2E0,7E0,7E0,1.8E1,3E0,2E0,3.1E1,6E0,1.3E1,3E0,1.1E1,2E1,3E0,1.69E2,6.7E1,6E0,8E0,9E0,7E0,2E1,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[-2.571658E-3,1.1800047E-2,-4.378604E-2,-3.0463526E-3,5.3342838E-2,-7.220961E-2,-2.53086E-3,-2.158536E-2,2.4167871E-2,1.6956288E-1,4.179139E-2,-6.1646156E-2,-1.9277544E-1,7.2084524E-2,-2.1965025E-2,-1.8629273E-2,-1.703817E-1,-1.6315026E-2,4.4892516E-2,1.8051639E-1,-9.98352E-4,2.9234862E-2,1.24536924E-1,-1.0310207E-1,-3.7945066E-2,-3.5997516E-1,-8.917697E-2,8.850757E-2,-2.7329724E-3,-1.7425876E-2,-1.0250448E-2,4.1472666E-2,-2.5039839E-2,1.0652514E-3,-9.2658065E-3,3.6565093E-3,-3.0732889E-2,1.1264466E-1,3.4237634E-2,5.3154575E-3,1.0096512E-2,6.62117E-2,1.9107133E-3,1.9924803E-1,6.7498825E-2,-5.5130344E-2,-1.4535779E-1,-1.2987998E-2,-9.155581E-2,-1.9104185E-2,-3.3804995E-3,-1.2334029E-1,5.5575166E-3,6.78026E-2,8.523366E-3,-2.3703104E-2,4.2917854E-3,-2.1229337E-3,5.724429E-2,-2.81051E-2,1.05370484E-1,-1.9281855E-2,-1.9542797E-2,9.239118E-3,7.19824E-2,6.516219E-4,5.9165586E-2,1.3453721E-2,5.2541565E-2,-3.996067E-2,1.8068554E-2,5.501761E-3,3.2007915E-1,-4.250027E-2,5.891197E-3,-3.2719523E-2,-1.0535194E-2,-1.2715705E-2,-1.056306E-1,2.7810557E-2,-6.410573E-2,6.415167E-3,-1.01365514E-1,-7.933129E-3,-3.4686946E-4,3.8300573E-3,-1.6239355E-3,2.819529E-2,-3.819828E-2,1.1436265E-2,4.226799E-2,-2.9484088E-2,8.831926E-3,-1.6508007E-4,6.580315E-3,-9.9194616E-2,9.947428E-3,-2.6143899E-3,9.814492E-2,-1.6956115E-1,1.9047132E-2,6.8001077E-3,7.7873215E-2,9.706596E-2,2.581166E-3,6.500209E-2,-3.172083E-3,8.410519E-2,-4.51332E-3,5.6594876E-3,1.8658403E-2,5.5875134E-4,-9.559125E-3,-6.593479E-2,1.9007747E-3,3.1951673E-3,-1.3743564E-1,5.8346717E-3,-4.656634E-3,5.1832248E-2,-8.224158E-2,-2.0303984E-1,-7.290287E-2,-1.7526018E-3,2.3814687E-3,-7.996907E-3,-3.1665865E-2,2.4642006E-3,-4.1949325E-3,-1.1921325E-3,-7.499185E-3,-1.6860472E-3,-8.866345E-3,8.0833066E-4,-8.879992E-3,7.2214114E-3,1.7395781E-3,-2.0092614E-2,-1.3055525E-3,1.483228E-3,-2.916892E-3,1.3307723E-3,-3.8179585E-3,4.0352046E-3,-1.3968298E-3,1.653377E-4,5.7509993E-3,2.9345618E-3,-1.3258961E-3,6.2513663E-3,-7.85978E-4,5.06876E-3,-6.784875E-4,-2.959476E-3,8.3139597E-4,-3.7272063E-3,3.6977772E-3,-4.023E-3,-9.766095E-3,-1.9687668E-3,2.9151114E-3,-2.7267935E-3,6.9507645E-3,-8.506241E-3,-2.8300793E-3,-1.2495702E-2,-4.575925E-3,-1.7247619E-3,-6.5414617E-3,-1.1457524E-3,-5.7886443E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,-1,55,-1,57,59,-1,-1,-1,61,63,65,-1,-1,67,69,71,73,75,77,79,81,-1,-1,83,-1,85,-1,87,-1,-1,89,91,93,-1,95,-1,97,99,101,-1,103,105,107,-1,109,111,-1,113,-1,-1,115,117,119,-1,121,-1,-1,-1,-1,123,125,-1,127,129,-1,-1,-1,131,133,-1,135,137,139,141,143,145,147,149,-1,151,153,-1,-1,-1,-1,155,-1,-1,157,-1,159,161,163,165,167,-1,-1,-1,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3196154E0,1.0190516E0,6.752461E-1,6.150328E-1,5.7690537E-1,4.2241073E-1,3.4380406E-1,3.1572074E-1,4.1519535E-1,8.5835576E-2,4.0884084E-1,3.0559123E-1,4.3558264E-1,1.1192894E-1,1.7051929E-1,2.7536315E-1,9.787491E-2,2.3762721E-1,2.3220164E-1,7.304108E-2,0E0,3.494582E-1,2.081427E-1,2.2307336E-1,2.6908022E-1,1.4536452E-1,1.4546463E-1,7.8113526E-2,0E0,1.3256869E-1,0E0,9.884513E-2,2.6039517E-1,0E0,0E0,0E0,6.416929E-1,1.4888555E-1,2.3773488E-1,0E0,0E0,4.4848448E-1,1.3603224E-1,1.6772264E-1,2.1907811E-1,2.1125624E-1,2.9332328E-1,2.9167143E-1,1.6080391E-1,0E0,0E0,9.468885E-2,0E0,6.919786E-2,0E0,1.3347763E-1,0E0,0E0,1.6194099E-1,1.9964415E-1,6.5897346E-2,0E0,3.392355E-1,0E0,1.14276215E-1,3.8523006E-1,1.5915078E-1,0E0,3.1117338E-1,1.8150029E-1,2.1660809E-1,0E0,9.2124104E-2,1.1358768E-1,0E0,1.2683281E-1,0E0,0E0,2.787009E-1,2.5653934E-1,1.3462609E-1,0E0,1.6462082E-1,0E0,0E0,0E0,0E0,6.572001E-2,1.19202584E-1,0E0,9.692658E-2,2.647717E-1,0E0,0E0,0E0,2.2621489E-1,1.7622697E-1,0E0,9.2812926E-2,4.5030883E-1,1.2918094E-1,9.3073316E-2,1.4960057E-1,2.066437E-1,1.347802E-1,8.064118E-2,0E0,1.037952E-1,1.5294763E-1,0E0,0E0,0E0,0E0,9.140502E-2,0E0,0E0,1.3672876E-1,0E0,1.6365671E-1,1.1181067E-1,1.00817084E-1,6.882143E-2,1.1907482E-1,0E0,0E0,0E0,7.811564E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,31,31,32,32,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,51,51,53,53,55,55,58,58,59,59,60,60,62,62,64,64,65,65,66,66,68,68,69,69,70,70,72,72,73,73,75,75,78,78,79,79,80,80,82,82,87,87,88,88,90,90,91,91,95,95,96,96,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,107,107,108,108,113,113,116,116,118,118,119,119,120,120,121,121,122,122,126,126],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,-1,56,-1,58,60,-1,-1,-1,62,64,66,-1,-1,68,70,72,74,76,78,80,82,-1,-1,84,-1,86,-1,88,-1,-1,90,92,94,-1,96,-1,98,100,102,-1,104,106,108,-1,110,112,-1,114,-1,-1,116,118,120,-1,122,-1,-1,-1,-1,124,126,-1,128,130,-1,-1,-1,132,134,-1,136,138,140,142,144,146,148,150,-1,152,154,-1,-1,-1,-1,156,-1,-1,158,-1,160,162,164,166,168,-1,-1,-1,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.2608955E-3,1.0122174E3,4.888E3,4.5168175E5,1.0993947E1,7.8351054E11,2.99665E5,1E0,8.15E2,1.708775E7,5.185489E3,1.263E3,1.8736842E0,1E0,2.6572757E10,1.9928571E0,9E0,1.1057693E0,2.7319324E7,3.09627E5,-9.98352E-4,1.0595947E7,3.379416E7,7.1047956E-1,2.1111E4,3.285E3,1.631E4,5.5325594E9,-2.7329724E-3,7.9023E4,-1.0250448E-2,3.97E2,6.65E2,1.0652514E-3,-9.2658065E-3,3.6565093E-3,1.3314917E0,9.99E2,2.37E2,5.3154575E-3,1.0096512E-2,1.4075E4,6.624E3,3.7E1,7.6767676E-2,9.655878E-1,4.650193E-1,2.1506184E7,1.89E2,-1.9104185E-2,-3.3804995E-3,3.773E3,5.5575166E-3,2.9924436E9,8.523366E-3,3.9382784E9,4.2917854E-3,-2.1229337E-3,2.04E5,1.7971E4,9.34731E5,-1.9281855E-2,1E0,9.239118E-3,4.7779464E7,4E0,3.4048372E6,1.3453721E-2,5.003246E1,1.1598511E3,8.579088E-3,5.501761E-3,1.003252E5,1.6956E5,5.891197E-3,9E2,-1.0535194E-2,-1.2715705E-2,5.033303E-1,7.505689E4,1.2302839E-1,6.415167E-3,1.857051E5,-7.933129E-3,-3.4686946E-4,3.8300573E-3,-1.6239355E-3,1.64354E9,1.22E2,1.1436265E-2,1.99607E3,6.425E1,8.831926E-3,-1.6508007E-4,6.580315E-3,2.1325E4,5.503109E8,-2.6143899E-3,8E0,1.8137958E6,1.99E2,1E0,6.1095314E-8,9.772148E4,5.4582E4,3.24403E5,-3.172083E-3,3.960925E3,4.6E0,5.6594876E-3,1.8658403E-2,5.5875134E-4,-9.559125E-3,6.015825E0,1.9007747E-3,3.1951673E-3,2.1102592E8,5.8346717E-3,3.7974813E8,1.1E2,1.1863768E-4,9.6909856E5,1.708775E7,-1.7526018E-3,2.3814687E-3,-7.996907E-3,9.4996613E-1,2.4642006E-3,-4.1949325E-3,-1.1921325E-3,-7.499185E-3,-1.6860472E-3,-8.866345E-3,8.0833066E-4,-8.879992E-3,7.2214114E-3,1.7395781E-3,-2.0092614E-2,-1.3055525E-3,1.483228E-3,-2.916892E-3,1.3307723E-3,-3.8179585E-3,4.0352046E-3,-1.3968298E-3,1.653377E-4,5.7509993E-3,2.9345618E-3,-1.3258961E-3,6.2513663E-3,-7.85978E-4,5.06876E-3,-6.784875E-4,-2.959476E-3,8.3139597E-4,-3.7272063E-3,3.6977772E-3,-4.023E-3,-9.766095E-3,-1.9687668E-3,2.9151114E-3,-2.7267935E-3,6.9507645E-3,-8.506241E-3,-2.8300793E-3,-1.2495702E-2,-4.575925E-3,-1.7247619E-3,-6.5414617E-3,-1.1457524E-3,-5.7886443E-3],"split_indices":[26,51,2,27,55,30,28,90,2,44,51,2,52,89,12,53,3,53,5,9,0,44,49,55,9,0,9,5,0,2,0,2,8,0,0,0,53,2,0,0,0,9,2,8,56,26,26,44,10,0,0,2,0,5,0,5,0,0,5,6,1,0,109,0,7,3,44,0,57,51,56,0,27,2,0,2,0,0,26,46,57,0,27,0,0,0,0,5,10,0,4,45,0,0,0,1,5,0,8,27,0,108,36,32,9,12,0,4,53,0,0,0,0,52,0,0,7,0,12,10,41,47,44,0,0,0,26,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.226E3,1.651E3,5.75E2,1.217E3,4.34E2,3.4E2,2.35E2,7.24E2,4.93E2,3.8E1,3.96E2,3.14E2,2.6E1,4.8E1,1.87E2,7.11E2,1.3E1,1.67E2,3.26E2,3.6E1,2E0,3.45E2,5.1E1,1.13E2,2.01E2,9E0,1.7E1,4.3E1,5E0,1.84E2,3E0,6.8E1,6.43E2,2E0,1.1E1,2.1E1,1.46E2,4.3E1,2.83E2,1.6E1,2E1,1.46E2,1.99E2,2.1E1,3E1,5.4E1,5.9E1,1.38E2,6.3E1,7E0,2E0,1.5E1,2E0,3.7E1,6E0,1.75E2,9E0,1E1,5.8E1,6.29E2,1.4E1,3E0,1.43E2,1.2E1,3.1E1,1.21E2,1.62E2,7E0,1.39E2,5.5E1,1.44E2,1.4E1,7E0,1.1E1,1.9E1,4.9E1,5E0,1.2E1,4.7E1,7.7E1,6.1E1,2E0,6.1E1,1E1,5E0,3.2E1,5E0,3.8E1,1.37E2,3E0,5.5E1,6.26E2,3E0,4E0,1E1,3.8E1,1.05E2,5E0,2.6E1,1.1E1,1.1E2,4.3E1,1.19E2,7.3E1,6.6E1,1.2E1,4.3E1,3.6E1,1.08E2,3E0,4E0,9E0,2E0,3.4E1,1.5E1,7E0,4E1,1.8E1,5.9E1,8E0,5.3E1,1.2E1,4.9E1,1E1,2.8E1,5E0,1.32E2,5.1E1,4E0,6.13E2,1.3E1,2.4E1,1.4E1,1.02E2,3E0,1.2E1,1.4E1,3E0,8E0,9.5E1,1.5E1,3.5E1,8E0,1.08E2,1.1E1,1.8E1,5.5E1,2.2E1,4.4E1,6E0,6E0,2.8E1,8E0,2.9E1,7.9E1,3.1E1,3E0,2.6E1,1.4E1,3.8E1,2.1E1,4E0,4E0,7E0,4.6E1,6E0,6E0,3.4E1,1.5E1,1.25E2,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[7.2125404E-5,-6.141248E-3,7.5112976E-2,2.9421425E-3,-6.0185056E-2,1.325638E-1,2.8475476E-2,-2.058867E-2,2.888792E-2,-5.5333182E-2,-1.2777918E-2,1.0370241E-1,2.9105714E-1,8.290975E-3,6.9541996E-3,-8.656692E-3,-6.221978E-2,5.2181244E-2,4.3699597E-3,-1.15816295E-1,-4.081857E-2,1.6082042E-1,4.6894204E-2,8.146884E-3,1.9943826E-2,1.980213E-2,-1.0011298E-2,6.380386E-2,-1.5200428E-2,2.3947638E-2,-7.730053E-2,6.188513E-2,-2.1641526E-2,8.333786E-5,1.01948746E-1,-8.75469E-3,-7.603183E-2,-3.0429395E-2,-1.1285175E-1,1.14832325E-2,5.6450143E-3,-2.2303448E-3,3.7795347E-3,-5.0040867E-2,5.3063534E-2,9.991064E-2,-7.1644016E-2,-7.734379E-2,-9.002349E-3,-2.8853698E-3,3.4117382E-3,-7.1899556E-2,-1.6270986E-2,6.76068E-2,-4.8879124E-2,7.503325E-3,-4.1537706E-2,-4.4544535E-3,4.0325355E-3,7.430816E-3,1.2875814E-3,-3.389225E-2,-1.5446238E-1,-3.456471E-2,5.092686E-3,-1.4269555E-1,2.6683363E-3,-5.7108025E-3,-1.0565973E-4,9.1769084E-2,-1.2129422E-2,1.2129533E-1,-1.7935504E-3,-1.2215178E-2,1.4785145E-2,-1.5667784E-1,-4.9180362E-2,9.6357994E-2,-1.4189452E-2,-1.4113924E-1,-5.4319486E-2,8.27327E-2,1.6676305E-2,-4.8317853E-3,-1.277575E-2,-1.4111115E-1,-1.3503321E-2,2.9854717E-2,-1.1193378E-2,5.5889734E-3,-7.5348453E-3,-1.4505728E-2,-4.308491E-3,-2.2884194E-2,-7.210162E-2,-1.9308654E-4,-8.429376E-3,8.728998E-3,3.065085E-3,-1.757086E-3,5.023866E-3,8.0924714E-4,7.401105E-3,4.3622935E-3,-3.8959885E-3,-4.6872674E-4,-1.0358901E-2,-5.301833E-3,1.9937977E-5,1.0824953E-2,2.4869773E-3,-4.639036E-3,-3.5465002E-4,-9.646097E-3,3.8354873E-4,1.3344771E-3,-3.0761943E-3,2.6129521E-3,5.93419E-3,5.211345E-3,-1.3691428E-3,-2.1857999E-3,4.939316E-3,-1.2062497E-3,-9.642557E-3,-3.395069E-3,1.2307015E-3,8.054608E-3,1.0933708E-3,-2.2296722E-3,4.251002E-4,1.9982385E-3,-3.5228352E-3,-1.5597964E-3,2.7673377E-3,-6.718126E-3,-1.5954967E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,31,33,35,37,39,41,-1,-1,43,-1,45,47,49,51,53,55,57,59,-1,61,63,65,-1,-1,-1,-1,67,69,71,73,75,77,-1,-1,79,-1,81,83,-1,85,-1,87,-1,-1,89,91,93,-1,95,-1,-1,-1,97,99,101,-1,-1,103,105,107,109,111,113,115,117,119,121,-1,123,125,127,129,131,-1,-1,-1,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0243613E0,9.9621516E-1,4.4635826E-1,1.0616877E0,2.989725E-1,3.1289196E-1,2.3613569E-1,4.5248958E-1,4.7207558E-1,2.4722666E-1,0E0,2.033686E-1,1.0340041E-1,2.2092992E-1,0E0,3.3762538E-1,2.6564342E-1,3.0507922E-1,1.6897799E-1,1.5499961E-1,1.7093512E-1,7.570183E-2,1.2372929E-1,0E0,0E0,1.8617837E-1,0E0,2.9570264E-1,2.5043422E-1,1.4453755E-1,2.3773563E-1,2.409631E-1,1.9328459E-1,1.5128464E-1,7.303719E-2,0E0,1.2035605E-1,1.2504311E-1,1.5708861E-1,0E0,0E0,0E0,0E0,9.409631E-2,1.3737288E-1,1.4504018E-1,2.2663969E-1,1.2411937E-1,3.2599714E-1,0E0,0E0,2.0048106E-1,0E0,2.7364242E-1,1.938923E-1,0E0,1.2650263E-1,0E0,1.4737837E-1,0E0,0E0,1.369795E-1,9.742439E-2,8.604011E-2,0E0,1.4971638E-1,0E0,0E0,0E0,6.9625825E-2,7.3276184E-2,1.7592555E-1,0E0,0E0,9.054387E-2,1.5594071E-1,1.5172414E-1,1.5243608E-1,3.17334E-1,3.6765295E-1,1.5957868E-1,3.250196E-1,3.8678566E-1,8.941715E-2,0E0,7.659495E-2,9.460032E-2,1.13101125E-1,1.8531731E-1,7.094707E-2,0E0,0E0,0E0,1.5176736E-1,1.2693372E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,43,43,44,44,45,45,46,46,47,47,48,48,51,51,53,53,54,54,56,56,58,58,61,61,62,62,63,63,65,65,69,69,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,32,34,36,38,40,42,-1,-1,44,-1,46,48,50,52,54,56,58,60,-1,62,64,66,-1,-1,-1,-1,68,70,72,74,76,78,-1,-1,80,-1,82,84,-1,86,-1,88,-1,-1,90,92,94,-1,96,-1,-1,-1,98,100,102,-1,-1,104,106,108,110,112,114,116,118,120,122,-1,124,126,128,130,132,-1,-1,-1,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4409692E3,3.388159E7,3.379416E7,3.6129813E2,1.2367184E3,9.609747E3,1E0,5.5758075E5,5.59175E3,9.6771875E2,-1.2777918E-2,1.135479E6,1.341E3,4.4543375E4,6.9541996E-3,7E0,2.25E2,2E0,2.5503985E9,2.25E2,1.2427474E8,1E0,1.40449E5,8.146884E-3,1.9943826E-2,7.023838E3,-1.0011298E-2,3.7467392E2,3.3151623E1,1.346757E6,2.6104508E10,3.7852024E1,2.2679293E8,5.72E2,1.4411988E5,-8.75469E-3,6.747114E7,3.5985E4,4.982003E6,1.14832325E-2,5.6450143E-3,-2.2303448E-3,3.7795347E-3,1.214049E0,9.452547E7,5.2187E4,1.5125709E8,2.5086691E0,6.9436204E-1,-2.8853698E-3,3.4117382E-3,1.1041459E-1,-1.6270986E-2,2.1351435E0,1.504779E7,7.503325E-3,5E0,-4.4544535E-3,4.787007E6,7.430816E-3,1.2875814E-3,8.286065E2,2.3446269E2,1.6383727E-1,5.092686E-3,2.2222222E-1,2.6683363E-3,-5.7108025E-3,-1.0565973E-4,2.9E1,9.178234E0,3.830986E-1,-1.7935504E-3,-1.2215178E-2,6.0479047E4,1.00033E5,1.00011E5,2.545E3,5.8601086E1,8.890291E2,2.3067484E0,8.027061E2,4.0208E4,9.08912E5,-1.277575E-2,8E0,1.0012501E7,2.5228915E0,1.51529E3,1.592E1,-7.5348453E-3,-1.4505728E-2,-4.308491E-3,2.3898147E5,6.071E3,-1.9308654E-4,-8.429376E-3,8.728998E-3,3.065085E-3,-1.757086E-3,5.023866E-3,8.0924714E-4,7.401105E-3,4.3622935E-3,-3.8959885E-3,-4.6872674E-4,-1.0358901E-2,-5.301833E-3,1.9937977E-5,1.0824953E-2,2.4869773E-3,-4.639036E-3,-3.5465002E-4,-9.646097E-3,3.8354873E-4,1.3344771E-3,-3.0761943E-3,2.6129521E-3,5.93419E-3,5.211345E-3,-1.3691428E-3,-2.1857999E-3,4.939316E-3,-1.2062497E-3,-9.642557E-3,-3.395069E-3,1.2307015E-3,8.054608E-3,1.0933708E-3,-2.2296722E-3,4.251002E-4,1.9982385E-3,-3.5228352E-3,-1.5597964E-3,2.7673377E-3,-6.718126E-3,-1.5954967E-3],"split_indices":[51,44,49,51,57,51,79,47,47,4,0,28,0,4,0,3,10,6,7,10,44,109,11,0,0,4,0,4,54,49,19,52,30,32,32,0,44,2,28,0,0,0,0,41,47,28,7,53,52,0,0,38,0,40,44,0,6,0,31,0,0,4,51,41,0,57,0,0,0,3,34,55,0,0,32,5,5,28,49,4,57,51,9,1,0,8,44,57,4,57,0,0,0,32,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.195E3,2.028E3,1.67E2,1.737E3,2.91E2,7.4E1,9.3E1,9.11E2,8.26E2,2.86E2,5E0,6.4E1,1E1,8.1E1,1.2E1,7.09E2,2.02E2,4.23E2,4.03E2,5.4E1,2.32E2,3.1E1,3.3E1,7E0,3E0,7.8E1,3E0,5.8E1,6.51E2,3E1,1.72E2,3.74E2,4.9E1,3.87E2,1.6E1,1.7E1,3.7E1,2.04E2,2.8E1,7E0,2.4E1,9E0,2.4E1,2.5E1,5.3E1,4.6E1,1.2E1,5.8E1,5.93E2,1.1E1,1.9E1,1.7E2,2E0,3.56E2,1.8E1,4E0,4.5E1,1.4E1,3.73E2,8E0,8E0,2.5E1,1.2E1,1.99E2,5E0,2.4E1,4E0,9E0,1.6E1,3.3E1,2E1,4E1,6E0,3E0,9E0,1.4E1,4.4E1,2.7E1,5.66E2,3.3E1,1.37E2,2.74E2,8.2E1,1.6E1,2E0,9E0,3.6E1,1.38E2,2.35E2,2E1,5E0,2E0,1E1,1.53E2,4.6E1,6E0,1.8E1,5E0,2.8E1,1.7E1,3E0,1.2E1,2.8E1,5E0,4E0,5E0,9E0,1.8E1,2.6E1,5E0,2.2E1,3.7E1,5.29E2,2.2E1,1.1E1,1.9E1,1.18E2,1.82E2,9.2E1,2.6E1,5.6E1,1.2E1,4E0,4E0,5E0,1.4E1,2.2E1,4E0,1.34E2,8.2E1,1.53E2,1.4E1,6E0,1.35E2,1.8E1,1.4E1,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-3.3885555E-3,-1.2369084E-2,4.6991613E-2,-5.258677E-3,-7.697449E-2,5.6586206E-2,-1.3863426E-1,-2.7744696E-2,1.3581384E-2,-5.4288294E-2,-1.2043521E-1,1.7446148E-1,4.6999518E-2,-1.0562546E-2,1.3553078E-3,-9.889033E-3,-6.3281976E-2,5.4305684E-2,4.987531E-3,-4.782577E-2,-1.373204E-2,-1.6491984E-1,-4.4442385E-2,9.38662E-3,2.943756E-3,3.158113E-2,1.05925545E-1,-2.7655805E-2,2.4701932E-2,-7.891883E-2,1.3845293E-3,1.2530011E-1,3.5320174E-2,-4.4328213E-2,1.7011616E-2,-9.2286635E-3,-3.9557584E-2,-1.1124412E-3,-1.9394615E-1,4.787093E-3,-6.9727466E-2,1.656151E-2,9.811031E-2,1.5126336E-1,-4.6423157E-3,-1.9211493E-2,-1.643375E-1,5.9024394E-2,-2.0057943E-2,-1.3168706E-1,-5.1787883E-2,4.5330994E-2,-7.119636E-2,-7.358296E-4,6.5518464E-3,-6.95272E-3,8.179614E-2,-7.8349754E-2,2.878542E-2,3.929634E-2,-7.581367E-3,-7.4870745E-3,-3.374338E-2,-1.2223518E-1,-2.825371E-1,-7.299331E-3,-1.509085E-3,4.7467876E-2,-1.5144595E-2,2.70444E-1,2.5033798E-2,1.6177194E-2,1.2747112E-1,5.201247E-3,-3.8002767E-3,-5.708382E-3,-1.2577788E-2,-2.1595454E-3,-2.109181E-1,7.3198095E-2,-4.255545E-3,-1.2451426E-1,1.3354191E-2,-2.2503331E-1,-8.518398E-2,-2.8155735E-2,-1.7180692E-1,-1.7980302E-3,3.3563443E-3,-1.1156593E-1,8.925165E-4,4.908342E-2,-5.9682608E-2,4.433502E-3,1.2131277E-1,-4.001092E-2,-1.8620454E-1,8.686564E-3,9.232176E-3,4.7096748E-2,-8.279475E-2,-1.5555104E-2,8.8086426E-2,-5.17346E-3,-2.498588E-2,-6.8740416E-3,-1.393433E-4,-7.5961566E-3,-1.6901817E-2,8.635527E-2,1.7787941E-3,2.9839461E-3,-1.0823986E-1,8.210638E-3,1.9266628E-2,-1.2996918E-1,9.133797E-2,6.4389994E-3,-3.254037E-4,1.7349165E-3,-1.7300494E-3,-1.5307809E-2,-5.7148286E-3,3.7672822E-3,-7.7141253E-3,1.4269417E-4,-7.7117113E-3,1.5776663E-3,-6.9883084E-3,-4.0603043E-3,-1.1664753E-2,-8.055684E-3,-2.7104325E-3,-9.0433275E-3,-8.546655E-4,-2.9767882E-3,-1.1931769E-2,3.101038E-3,-6.668128E-3,-2.5460622E-4,6.7842286E-3,-5.531622E-3,-4.175464E-4,3.7324107E-3,-1.6962267E-3,6.3229855E-3,-4.071414E-4,-3.6524865E-3,1.2323521E-4,-1.2506593E-3,-1.2579914E-2,2.218698E-3,-3.0539727E-3,3.0650382E-3,8.818652E-4,-1.5769005E-2,-1.536772E-3,1.4464757E-3,-1.4598654E-3,1.2368024E-3,7.4409232E-3,1.6700623E-3,-1.7695144E-3,4.5852163E-3,-3.6280088E-3,8.0470537E-4,-7.905495E-3,8.5016736E-4,-3.7806921E-3,-1.35996025E-2,-2.686309E-3,-3.3178224E-4,-9.530544E-3,-4.329359E-3,5.621387E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,-1,37,39,-1,-1,41,43,45,47,49,51,53,55,57,59,-1,61,-1,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,-1,91,93,95,97,99,101,-1,103,105,107,-1,-1,109,111,113,115,-1,117,-1,-1,-1,119,-1,121,123,-1,125,127,129,131,133,135,-1,-1,137,-1,139,141,143,145,147,149,151,-1,153,155,157,159,-1,161,-1,-1,-1,-1,163,165,167,169,-1,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0167627E0,8.756153E-1,6.133865E-1,7.2861046E-1,1.7980778E-1,3.5793126E-1,2.7827513E-1,4.9685484E-1,3.273092E-1,1.8826693E-1,2.0902342E-1,7.2849095E-2,2.7012658E-1,0E0,0E0,3.2210314E-1,2.650385E-1,2.1527639E-1,4.597181E-1,1.5189141E-1,0E0,1.5818644E-1,1.0230462E-1,0E0,0E0,2.3819351E-1,3.1150228E-1,3.9619005E-1,2.746398E-1,2.9525888E-1,1.6909043E-1,9.579486E-2,2.5597626E-1,3.8034594E-1,3.417318E-1,0E0,8.146997E-2,0E0,1.7511117E-1,0E0,6.788815E-2,1.9376323E-1,5.4734576E-1,1.701622E-1,1.8972723E-1,2.302472E-1,9.8600686E-2,2.259349E-1,2.7389127E-1,2.8903377E-1,3.95082E-1,8.3780155E-2,7.559937E-2,0E0,0E0,2.0680921E-1,1.8900305E-1,4.2259252E-1,1.7070697E-1,3.153212E-1,2.2793067E-1,0E0,7.824437E-2,7.006669E-2,8.321726E-2,0E0,0E0,1.775193E-1,1.6571575E-1,1.0246891E-1,3.4074616E-1,0E0,8.2467675E-2,0E0,0E0,0E0,4.0464187E-1,0E0,1.08133554E-1,2.4100113E-1,0E0,1.1168072E-1,2.2165385E-1,7.842541E-2,1.0440281E-1,1.856817E-1,2.0645052E-1,0E0,0E0,1.0012819E-1,0E0,1.8826973E-1,1.12405345E-1,7.4994504E-2,9.927958E-2,1.3576055E-1,3.8477147E-1,1.41456E-1,0E0,1.7394012E-1,2.4758261E-1,2.1906465E-1,1.0210605E-1,0E0,9.303765E-2,0E0,0E0,0E0,0E0,1.4227027E-1,1.3571289E-1,1.1495176E-1,1.3252734E-1,0E0,0E0,9.723127E-2,1.5302996E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,67,67,68,68,69,69,70,70,72,72,76,76,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,89,89,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,104,104,109,109,110,110,111,111,112,112,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,-1,38,40,-1,-1,42,44,46,48,50,52,54,56,58,60,-1,62,-1,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,-1,92,94,96,98,100,102,-1,104,106,108,-1,-1,110,112,114,116,-1,118,-1,-1,-1,120,-1,122,124,-1,126,128,130,132,134,136,-1,-1,138,-1,140,142,144,146,148,150,152,-1,154,156,158,160,-1,162,-1,-1,-1,-1,164,166,168,170,-1,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.753919E3,3.9712732E7,2.6572757E10,2.785857E2,6.9514094E11,4.695E3,1.4613059E0,2.7661E4,1.823E3,2.8721272E1,2.1907706E12,1.6904226E7,9.4615E4,-1.0562546E-2,1.3553078E-3,7.68E2,3.3012173E9,1.9E1,2.47E2,2.0008473E2,-1.373204E-2,3.051499E5,1.2920592E0,9.38662E-3,2.943756E-3,8.791777E3,3.3968816E8,1E0,7.980211E5,1.06E3,1.3905958E7,1.3E1,7.3E1,9.5E2,1.3226592E-2,-9.2286635E-3,1.4634147E0,-1.1124412E-3,2.2615E4,4.787093E-3,2.852738E6,2.589369E7,3.3517068E7,1E0,2.802E5,8.333333E0,4.5168175E5,1.8114872E7,4.552196E6,2.9910075E5,3.73524E5,5.9570195E6,8.503172E2,-7.358296E-4,6.5518464E-3,5.69815E5,1.2354571E0,1.9566955E5,1E0,3.142868E5,1.5963264E9,-7.4870745E-3,4.30453E2,4.804E3,2.0966542E0,-7.299331E-3,-1.509085E-3,1.46617E5,2.264389E-1,1.62E2,5.2630428E7,1.6177194E-2,1.42248E6,5.201247E-3,-3.8002767E-3,-5.708382E-3,3.0769231E0,-2.1595454E-3,4.0701206E1,4.4E1,-4.255545E-3,1.7019386E2,4.3298E4,1E0,1.1E2,3.5476844E7,1.03270106E6,-1.7980302E-3,3.3563443E-3,9.908197E1,8.925165E-4,9.5411836E4,4.930349E5,9.489796E-1,1.363176E7,4E0,2.44E3,2.2515285E6,9.232176E-3,5.666993E0,3.14088E5,1.9711539E0,3.7898205E1,-5.17346E-3,1E0,-6.8740416E-3,-1.393433E-4,-7.5961566E-3,-1.6901817E-2,2.429717E1,1.6368244E0,9E0,1.7384394E7,8.210638E-3,1.9266628E-2,3.164557E-2,2.1E1,6.4389994E-3,-3.254037E-4,1.7349165E-3,-1.7300494E-3,-1.5307809E-2,-5.7148286E-3,3.7672822E-3,-7.7141253E-3,1.4269417E-4,-7.7117113E-3,1.5776663E-3,-6.9883084E-3,-4.0603043E-3,-1.1664753E-2,-8.055684E-3,-2.7104325E-3,-9.0433275E-3,-8.546655E-4,-2.9767882E-3,-1.1931769E-2,3.101038E-3,-6.668128E-3,-2.5460622E-4,6.7842286E-3,-5.531622E-3,-4.175464E-4,3.7324107E-3,-1.6962267E-3,6.3229855E-3,-4.071414E-4,-3.6524865E-3,1.2323521E-4,-1.2506593E-3,-1.2579914E-2,2.218698E-3,-3.0539727E-3,3.0650382E-3,8.818652E-4,-1.5769005E-2,-1.536772E-3,1.4464757E-3,-1.4598654E-3,1.2368024E-3,7.4409232E-3,1.6700623E-3,-1.7695144E-3,4.5852163E-3,-3.6280088E-3,8.0470537E-4,-7.905495E-3,8.5016736E-4,-3.7806921E-3,-1.35996025E-2,-2.686309E-3,-3.3178224E-4,-9.530544E-3,-4.329359E-3,5.621387E-3],"split_indices":[51,44,12,51,30,28,40,28,28,52,30,46,10,0,0,2,5,10,0,51,0,27,55,0,0,51,31,111,49,2,44,0,0,10,56,0,55,0,10,0,1,44,50,8,10,51,27,12,12,27,28,44,4,0,0,9,52,46,89,32,7,0,32,11,53,0,0,9,37,8,47,0,11,0,0,0,57,0,51,3,0,51,11,80,10,7,49,0,0,4,0,32,27,52,9,8,28,27,0,52,1,52,57,0,66,0,0,0,0,52,37,16,50,0,0,56,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.245E3,1.906E3,3.39E2,1.718E3,1.88E2,3.23E2,1.6E1,7.83E2,9.35E2,1.25E2,6.3E1,2.3E1,3E2,1E1,6E0,5.22E2,2.61E2,1.62E2,7.73E2,1.23E2,2E0,3.9E1,2.4E1,1.7E1,6E0,2.39E2,6.1E1,3.45E2,1.77E2,2.1E2,5.1E1,3.3E1,1.29E2,1.51E2,6.22E2,5E0,1.18E2,7E0,3.2E1,3E0,2.1E1,1.96E2,4.3E1,4.3E1,1.8E1,3.26E2,1.9E1,1E2,7.7E1,7E1,1.4E2,3.2E1,1.9E1,4E0,2.9E1,6.8E1,6.1E1,1.03E2,4.8E1,3.26E2,2.96E2,4E0,1.14E2,1.9E1,1.3E1,5E0,1.6E1,9.9E1,9.7E1,1.2E1,3.1E1,3E0,4E1,7E0,1.1E1,1.8E1,3.08E2,6E0,1.3E1,9.2E1,8E0,1.8E1,5.9E1,2.2E1,4.8E1,1.18E2,2.2E1,8E0,2.4E1,1.3E1,6E0,3.3E1,3.5E1,2.1E1,4E1,7.7E1,2.6E1,4.4E1,4E0,3.07E2,1.9E1,2.74E2,2.2E1,1E1,1.04E2,1.5E1,4E0,7E0,6E0,5.3E1,4.6E1,8.2E1,1.5E1,9E0,3E0,9E0,2.2E1,3.6E1,4E0,1.03E2,2.05E2,4E0,9E0,8.9E1,3E0,5E0,1.3E1,5.3E1,6E0,5E0,1.7E1,9E0,3.9E1,5E0,1.13E2,1.1E1,1.1E1,2E0,1.1E1,2.2E1,1.1E1,1.5E1,2E1,7E0,1.4E1,3.5E1,5E0,3.9E1,3.8E1,1E1,1.6E1,2.9E1,1.5E1,1.75E2,1.32E2,2E0,1.7E1,7.1E1,2.03E2,1.3E1,9E0,1.9E1,8.5E1,4.9E1,4E0,4.3E1,3E0,7E1,1.2E1,2E0,1.3E1,4E0,5E0,3E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-1.3519886E-3,1.7927189E-3,-1.3466054E-1,-2.1791903E-2,2.0420073E-2,-1.6177797E-1,-2.380987E-2,-1.2400635E-2,-7.131222E-2,5.357488E-2,3.0067293E-3,-1.3538279E-1,-1.3074521E-2,-8.398075E-3,2.716833E-3,-1.9336438E-2,3.7796363E-2,-6.1349254E-2,-2.425447E-2,1.6373502E-1,4.5026932E-2,-2.3025492E-2,2.6915627E-2,-7.660074E-3,1.0880263E-3,-1.3299093E-2,-9.8412976E-2,8.4039986E-2,2.7280799E-4,-4.7640882E-2,-1.7880686E-1,8.282251E-3,-9.0880676E-5,-3.2540653E-2,5.1052045E-2,1.1490838E-2,-4.8444428E-2,7.909334E-2,1.6369903E-2,-1.7495676E-3,-5.207183E-2,-7.2415456E-2,-1.6877279E-2,5.431642E-3,1.6623968E-3,1.1458771E-3,-8.946601E-2,-9.2217416E-2,-2.5318569E-2,-1.2496889E-2,-4.644805E-3,3.947736E-3,-2.8233584E-3,4.702646E-2,1.61527E-1,7.035431E-3,-4.578522E-3,-2.1012159E-1,-4.279761E-2,-3.909265E-3,1.1626278E-1,3.610779E-3,6.448025E-2,-4.417152E-3,1.6219214E-1,-8.0459096E-2,1.911707E-2,-2.2759433E-3,-1.4245601E-1,-5.172219E-4,-8.893596E-3,-5.1211778E-2,-1.223942E-2,-8.723587E-2,4.0511144E-3,8.987076E-2,3.66483E-2,8.836046E-3,-2.0162573E-4,1.2531949E-3,-1.2635368E-2,-1.3397705E-2,1.3242781E-4,-6.2342625E-2,-4.741923E-3,2.1590455E-3,-2.7092982E-3,1.3719596E-1,-3.939521E-3,2.8067539E-2,-2.0512873E-2,1.6383335E-1,2.2305831E-2,6.4936967E-6,-3.8929686E-3,1.1981138E-3,1.3188514E-2,-1.119726E-2,-2.8263268E-3,-1.3990821E-3,4.774439E-3,-5.718501E-3,9.714772E-4,-1.6386481E-2,-1.9784821E-3,-1.620824E-4,-4.1434607E-3,-8.124113E-3,-1.3247692E-3,5.9575993E-3,-1.0415803E-3,5.3085107E-3,4.68002E-4,6.274818E-4,2.8728526E-3,8.941279E-4,-2.5288328E-3,-4.7384184E-3,-1.149944E-3,6.393676E-3,-8.03275E-4,7.5177797E-3,4.4089483E-4,2.080613E-3,-1.0795343E-3,-5.1056535E-4,-6.1461227E-3,1.0965362E-2,2.4180687E-3,-7.3388405E-3,2.076882E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,29,-1,31,33,35,37,-1,-1,39,41,43,45,47,49,-1,-1,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,69,71,73,-1,-1,-1,-1,75,77,-1,79,81,83,85,87,89,91,93,95,97,99,101,103,-1,-1,105,-1,107,109,111,113,-1,-1,115,-1,-1,-1,117,119,-1,-1,121,-1,123,125,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.095218E-1,9.318014E-1,1.4644724E-1,4.3441048E-1,6.8398213E-1,9.500015E-2,1.6020544E-1,2.7507812E-1,6.565488E-1,3.768177E-1,4.848483E-1,1.9271564E-1,0E0,0E0,0E0,3.2953793E-1,1.6655867E-1,2.2817391E-1,0E0,9.265888E-2,1.7925566E-1,3.2781512E-1,2.2210738E-1,0E0,0E0,2.8874418E-1,3.2251677E-1,6.871152E-2,1.23591304E-1,1.3000703E-1,8.2825124E-2,0E0,0E0,1.0541695E-1,1.4939249E-1,3.6839554E-1,1.8729389E-1,2.1024543E-1,2.0786095E-1,2.1850595E-1,3.0062518E-1,2.2565584E-1,0E0,0E0,0E0,0E0,9.793668E-2,3.0753505E-1,1.6401763E-1,0E0,0E0,0E0,0E0,1.4877498E-1,6.7875E-2,0E0,2.3087293E-1,1.2882686E-1,1.551584E-1,6.634621E-2,2.1193516E-1,1.5928207E-1,2.9384688E-1,1.8266721E-1,1.3248508E-1,3.0349928E-1,1.8896197E-1,7.336053E-2,4.8271674E-1,0E0,0E0,7.071012E-2,0E0,1.4977473E-1,2.169805E-1,1.4252383E-1,1.6833577E-1,0E0,0E0,1.5109786E-1,0E0,0E0,0E0,2.1308619E-1,1.3913435E-1,0E0,0E0,1.5652734E-1,0E0,1.2553224E-1,1.4321725E-1,1.6878891E-1,2.2908266E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,46,46,47,47,48,48,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,71,71,73,73,74,74,75,75,76,76,79,79,83,83,84,84,87,87,89,89,90,90,91,91,92,92],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,30,-1,32,34,36,38,-1,-1,40,42,44,46,48,50,-1,-1,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,70,72,74,-1,-1,-1,-1,76,78,-1,80,82,84,86,88,90,92,94,96,98,100,102,104,-1,-1,106,-1,108,110,112,114,-1,-1,116,-1,-1,-1,118,120,-1,-1,122,-1,124,126,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2427474E8,3.7311627E2,4.4839956E3,1.3905958E7,7.358598E6,5.39978E5,2.7635896E-1,2.437E3,1.2592834E-1,3.79E2,6.624E3,3.995065E6,-1.3074521E-2,-8.398075E-3,2.716833E-3,1.0008265E1,1.6E1,4.8297736E7,-2.425447E-2,3.748732E6,2.2679293E8,7.4946815E1,2.909019E3,-7.660074E-3,1.0880263E-3,2.7661E4,9.561769E-1,4.1384277E2,3.7186194E5,1.8122449E2,1.1863768E-4,8.282251E-3,-9.0880676E-5,6.763312E7,8.581219E6,1.1E1,1.0784314E0,1.3141646E3,8.015419E3,1.3304372E7,6.285983E5,1E0,-1.6877279E-2,5.431642E-3,1.6623968E-3,1.1458771E-3,1.7322648E5,1.7270016E5,5.5987865E-4,-1.2496889E-2,-4.644805E-3,3.947736E-3,-2.8233584E-3,1.9352394E9,4.084361E0,7.035431E-3,9.195E1,5.5012004E9,2.6713815E-1,1.393168E6,1.066317E12,2.342147E7,4.7279915E6,1E0,1.5331442E-3,5.750032E3,4.230769E0,4.22088E0,1E0,-5.172219E-4,-8.893596E-3,4.785384E5,-1.223942E-2,1.9253924E7,6.504016E4,5.454369E6,1.101214E6,8.836046E-3,-2.0162573E-4,3.388159E7,-1.2635368E-2,-1.3397705E-2,1.3242781E-4,1.5524965E3,7.697189E2,2.1590455E-3,-2.7092982E-3,5.5544252E7,-3.939521E-3,9.74026E0,3.0392378E7,1.0258741E2,2.0521326E0,6.4936967E-6,-3.8929686E-3,1.1981138E-3,1.3188514E-2,-1.119726E-2,-2.8263268E-3,-1.3990821E-3,4.774439E-3,-5.718501E-3,9.714772E-4,-1.6386481E-2,-1.9784821E-3,-1.620824E-4,-4.1434607E-3,-8.124113E-3,-1.3247692E-3,5.9575993E-3,-1.0415803E-3,5.3085107E-3,4.68002E-4,6.274818E-4,2.8728526E-3,8.941279E-4,-2.5288328E-3,-4.7384184E-3,-1.149944E-3,6.393676E-3,-8.03275E-4,7.5177797E-3,4.4089483E-4,2.080613E-3,-1.0795343E-3,-5.1056535E-4,-6.1461227E-3,1.0965362E-2,2.4180687E-3,-7.3388405E-3,2.076882E-3],"split_indices":[44,51,51,44,44,11,26,2,37,28,2,28,0,0,0,53,3,44,0,46,30,47,31,0,0,28,37,4,46,51,41,0,0,7,27,3,55,51,4,44,27,65,0,0,0,0,27,46,37,0,0,0,0,30,37,0,52,5,56,1,30,44,46,79,40,32,55,52,109,0,0,27,0,44,46,49,1,0,0,44,0,0,0,54,4,0,0,44,0,52,42,55,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.168E3,2.119E3,4.9E1,9.35E2,1.184E3,3.9E1,1E1,7.87E2,1.48E2,4.07E2,7.77E2,3.4E1,5E0,3E0,7E0,6.92E2,9.5E1,1.46E2,2E0,2.8E1,3.79E2,3.72E2,4.05E2,2.8E1,6E0,6.44E2,4.8E1,4.2E1,5.3E1,1.32E2,1.4E1,2.5E1,3E0,2.7E1,3.52E2,1.58E2,2.14E2,6.7E1,3.38E2,4.97E2,1.47E2,4.5E1,3E0,2.3E1,1.9E1,4.2E1,1.1E1,4.3E1,8.9E1,5E0,9E0,5E0,2.2E1,3.41E2,1.1E1,1.5E1,1.43E2,6E0,2.08E2,2.1E1,4.6E1,2.68E2,7E1,4.9E2,7E0,1.05E2,4.2E1,2.3E1,2.2E1,7E0,4E0,3.6E1,7E0,2.8E1,6.1E1,6.5E1,2.76E2,9E0,2E0,1.41E2,2E0,4E0,2E0,1.37E2,7.1E1,1.1E1,1E1,4.2E1,4E0,1.33E2,1.35E2,2E1,5E1,4.65E2,2.5E1,4E0,3E0,9E0,9.6E1,2.7E1,1.5E1,3E0,2E1,6E0,1.6E1,1.7E1,1.9E1,1E1,1.8E1,1E1,5.1E1,4.8E1,1.7E1,1.5E2,1.26E2,1.07E2,3.4E1,6.3E1,7.4E1,5E0,6.6E1,3.4E1,8E0,9.9E1,3.4E1,1.26E2,9E0,1.1E1,9E0,5E0,4.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-2.779153E-3,2.4407338E-3,-6.9299445E-2,-8.076954E-3,3.9362133E-2,-5.1948834E-2,-1.7189884E-1,-2.289276E-3,-6.231699E-2,3.0720515E-2,1.2422386E-1,-8.236172E-2,3.185318E-2,-9.901333E-2,-1.6384892E-2,-2.7155625E-2,1.3226993E-2,-1.6330734E-1,-4.5271434E-2,1.9917822E-2,1.0961394E-1,2.3805174E-1,6.498888E-2,-6.781761E-2,-1.7783497E-1,6.0942937E-3,-1.49959475E-2,-9.623934E-3,-2.3263833E-3,-2.5157528E-2,-1.8112579E-1,6.4930334E-3,6.5582596E-2,-4.6054553E-3,-1.0909126E-2,4.838525E-2,-6.614549E-2,2.6965376E-2,-1.1095386E-1,1.7747025E-1,4.9960043E-2,6.420739E-3,1.733564E-2,-6.9389176E-3,9.6066855E-2,-8.129534E-3,-5.781718E-2,-1.2592376E-2,-2.360023E-3,3.3585003E-3,-4.4007752E-2,-5.1245526E-3,-2.2154283E-2,-1.35138165E-2,3.3890419E-3,9.108016E-3,-1.21393174E-1,1.15833625E-1,1.3060317E-2,4.2419606E-3,-5.0124056E-3,3.263136E-2,-8.455235E-2,-1.1986554E-2,3.0250449E-2,-7.038281E-3,2.0262282E-3,8.90321E-3,-7.224993E-4,-3.8302908E-4,5.34318E-3,-4.2128577E-3,1.2478037E-1,1.5422106E-3,-6.649964E-2,-3.5317843E-3,2.4446752E-3,3.7123997E-2,-2.9986303E-2,1.25055825E-2,-8.9902386E-2,-1.51422005E-2,-2.250038E-2,1.4348294E-3,1.459991E-1,-4.1019903E-3,3.743128E-2,6.677814E-2,-7.2602173E-3,-5.647272E-2,-1.5109985E-1,5.8818284E-2,7.941463E-3,3.529884E-3,1.1065998E-2,-5.766785E-3,-5.1192842E-2,-8.152237E-4,3.9745206E-3,-1.4792926E-3,3.8991037E-3,1.1905552E-4,2.104101E-3,-5.0730687E-3,5.3091687E-3,-4.670856E-3,3.4119703E-3,9.256498E-3,3.47623E-3,-2.697015E-3,2.8073217E-3,-2.3276554E-3,5.0009363E-3,-9.000913E-3,-1.5303994E-3,-9.56681E-3,-2.0061978E-3,2.9724964E-3,-3.2807195E-3,-1.1213254E-3,1.4345625E-3,-8.884972E-4,-4.123519E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,-1,-1,51,53,55,57,-1,-1,59,61,63,65,67,69,-1,-1,-1,71,-1,73,-1,-1,-1,75,-1,77,-1,-1,79,81,83,85,-1,-1,87,89,-1,91,-1,-1,-1,-1,-1,-1,-1,93,-1,95,-1,-1,97,99,101,103,-1,105,-1,107,-1,109,111,-1,113,115,117,119,-1,-1,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.730995E-1,8.0221474E-1,2.7945632E-1,5.046469E-1,3.3205467E-1,3.59436E-1,2.826875E-1,5.613603E-1,2.594295E-1,3.5320985E-1,2.6771814E-1,1.3096279E-1,1.8495186E-1,8.6432174E-2,0E0,1.6741806E-1,3.1546843E-1,8.069235E-2,2.651697E-1,3.4240758E-1,1.9352823E-1,1.408503E-1,2.1142754E-1,9.131098E-2,1.4941508E-1,0E0,7.3119126E-2,0E0,0E0,1.4369604E-1,2.3264568E-1,2.669697E-1,2.6769564E-1,0E0,0E0,1.9059221E-1,2.0275795E-1,3.4342322E-1,1.3862698E-1,8.847493E-2,1.1232692E-1,0E0,0E0,0E0,1.5279853E-1,0E0,7.007441E-2,0E0,0E0,0E0,7.4190944E-2,0E0,2.4996126E-1,0E0,0E0,2.6351193E-1,3.296026E-1,1.2735766E-1,1.3223751E-1,0E0,0E0,1.2933046E-1,1.652494E-1,0E0,2.2068274E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0976261E-1,0E0,6.616062E-2,0E0,0E0,1.7754881E-1,1.5074697E-1,2.5316352E-1,1.3245109E-1,0E0,1.02508605E-1,0E0,1.3805556E-1,0E0,1.0381949E-1,8.8282734E-2,0E0,2.0947073E-1,1.6423613E-1,1.4119905E-1,1.5301201E-1,0E0,0E0,0E0,7.879707E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,44,44,46,46,50,50,52,52,55,55,56,56,57,57,58,58,61,61,62,62,64,64,72,72,74,74,77,77,78,78,79,79,80,80,82,82,84,84,86,86,87,87,89,89,90,90,91,91,92,92,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,-1,-1,52,54,56,58,-1,-1,60,62,64,66,68,70,-1,-1,-1,72,-1,74,-1,-1,-1,76,-1,78,-1,-1,80,82,84,86,-1,-1,88,90,-1,92,-1,-1,-1,-1,-1,-1,-1,94,-1,96,-1,-1,98,100,102,104,-1,106,-1,108,-1,110,112,-1,114,116,118,120,-1,-1,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.971004E7,1.0855529E3,1.5265896E8,1E0,8.791777E3,1.2679E4,1.7243155E12,2.1592189E5,5.85E2,3.2196458E6,3.2475834E7,2.25215E5,7.39235E8,6.218596E2,-1.6384892E-2,6.9934395E-5,1.0927339E7,8E-3,1.988805E8,3.0961128E7,9.51E3,3.5E1,5.388794E6,3.2849124E2,1.2068E4,6.0942937E-3,7.76264E7,-9.623934E-3,-2.3263833E-3,8.333333E0,2.3E1,1.4659271E7,1.3581108E7,-4.6054553E-3,-1.0909126E-2,1.8544E4,1.3177E4,8.490231E-1,2.3245156E0,2.0647826E6,1.96013E9,6.420739E-3,1.733564E-2,-6.9389176E-3,1.3E1,-8.129534E-3,8E0,-1.2592376E-2,-2.360023E-3,3.3585003E-3,9.609747E3,-5.1245526E-3,5.643341E-2,-1.35138165E-2,3.3890419E-3,2.6104508E10,6.244737E1,1.8839E4,3.4453398E-1,4.2419606E-3,-5.0124056E-3,7.8091515E6,8.9259076E-1,-1.1986554E-2,6.6385515E6,-7.038281E-3,2.0262282E-3,8.90321E-3,-7.224993E-4,-3.8302908E-4,5.34318E-3,-4.2128577E-3,9.612346E7,1.5422106E-3,1.2019585E3,-3.5317843E-3,2.4446752E-3,3.6E2,6.65E2,2.539E3,1.7227725E0,-1.51422005E-2,1.2974394E6,1.4348294E-3,1.5445488E8,-4.1019903E-3,6.4547E4,2.2783158E5,-7.2602173E-3,2.1914425E-3,8.677148E2,5.4210526E1,4.382419E6,3.529884E-3,1.1065998E-2,-5.766785E-3,3.841808E-2,-8.152237E-4,3.9745206E-3,-1.4792926E-3,3.8991037E-3,1.1905552E-4,2.104101E-3,-5.0730687E-3,5.3091687E-3,-4.670856E-3,3.4119703E-3,9.256498E-3,3.47623E-3,-2.697015E-3,2.8073217E-3,-2.3276554E-3,5.0009363E-3,-9.000913E-3,-1.5303994E-3,-9.56681E-3,-2.0061978E-3,2.9724964E-3,-3.2807195E-3,-1.1213254E-3,1.4345625E-3,-8.884972E-4,-4.123519E-3],"split_indices":[44,51,44,111,51,2,30,27,2,27,49,11,7,51,0,36,9,56,7,42,28,8,46,32,9,0,44,0,0,51,3,49,12,0,0,9,9,52,38,42,5,0,0,0,3,0,3,0,0,0,51,0,57,0,0,19,57,1,41,0,0,44,40,0,31,0,0,0,0,0,0,0,42,0,4,0,0,2,8,2,38,0,27,0,7,0,1,27,0,40,54,52,1,0,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.225E3,2.064E3,1.61E2,1.607E3,4.57E2,1.39E2,2.2E1,1.453E3,1.54E2,4.16E2,4.1E1,1.02E2,3.7E1,1.7E1,5E0,5.58E2,8.95E2,2.1E1,1.33E2,3.67E2,4.9E1,1.3E1,2.8E1,9E1,1.2E1,1.1E1,2.6E1,4E0,1.3E1,5.52E2,6E0,7.94E2,1.01E2,1.3E1,8E0,2.4E1,1.09E2,3.49E2,1.8E1,2.2E1,2.7E1,9E0,4E0,3E0,2.5E1,6E0,8.4E1,6E0,6E0,6E0,2E1,1.7E1,5.35E2,4E0,2E0,7.79E2,1.5E1,5.1E1,5E1,1.9E1,5E0,1.7E1,9.2E1,3E0,3.46E2,1.4E1,4E0,2E1,2E0,1.5E1,1.2E1,3E0,2.2E1,7E0,7.7E1,1.5E1,5E0,6.2E1,4.73E2,7.54E2,2.5E1,4E0,1.1E1,1.4E1,3.7E1,9E0,4.1E1,1.5E1,2E0,6.6E1,2.6E1,1.51E2,1.95E2,1.7E1,5E0,1.4E1,6.3E1,3E1,3.2E1,4.63E2,1E1,5.85E2,1.69E2,2.3E1,2E0,6E0,5E0,1.9E1,1.8E1,8E0,3.3E1,4E0,1.1E1,8E0,5.8E1,1.6E1,1E1,1.44E2,7E0,8.2E1,1.13E2,3.6E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[-2.7947447E-3,-2.5897203E-2,1.5817322E-2,-1.530771E-2,-8.198472E-2,2.7127696E-2,-6.1021537E-2,-2.3902616E-2,5.101462E-2,-7.299542E-2,-2.350188E-2,2.1269197E-2,1.03625014E-1,-4.678365E-2,-1.6272064E-1,-1.568402E-2,-8.867027E-2,6.278702E-2,-6.976155E-2,-7.882137E-2,1.124876E-2,4.0907454E-2,1.0288988E-3,2.1014825E-1,3.7108377E-2,-8.057922E-2,-1.466197E-2,-4.022346E-3,-1.5927585E-2,-2.0768264E-2,7.259607E-2,-1.429238E-2,-6.36071E-2,7.133269E-2,-6.0895886E-3,-6.9678994E-3,1.2828141E-3,-4.4190995E-2,-1.2033963E-1,3.057521E-2,9.987574E-2,5.092556E-3,-7.842461E-2,7.0664478E-3,1.6197814E-2,-5.9784655E-3,6.920998E-2,-8.8722914E-2,1.8008121E-3,4.924045E-3,-2.5336625E-2,4.2950604E-3,-2.4569197E-2,4.9979556E-2,1.3795785E-2,-5.0457217E-2,-9.444725E-3,1.18615225E-1,4.1992437E-2,-8.28927E-3,-1.0973452E-1,-1.0357754E-1,-1.4224121E-2,6.6394038E-3,2.662386E-2,1.5661563E-1,3.7369493E-2,3.3407733E-2,-1.0530306E-2,-1.6297352E-2,-1.6510066E-3,-9.583038E-2,1.0600742E-1,-3.1688637E-3,-1.6405271E-1,-1.553963E-3,5.751386E-3,-1.3359355E-2,-6.525301E-2,-5.7370793E-3,3.2220182E-3,-1.0095351E-2,-3.9566584E-2,1.5712537E-1,8.230975E-4,6.194344E-2,-1.1633066E-3,1.4968238E-2,-8.636935E-2,-1.9811195E-1,-3.049814E-2,-8.911629E-4,-1.2622124E-1,5.5189032E-2,1.1781102E-2,1.034975E-2,3.115209E-3,2.6726115E-3,-2.1974198E-3,2.6168058E-2,8.83511E-3,-5.570134E-2,3.2794534E-3,1.4411918E-3,-1.0478729E-2,3.423715E-3,1.0648499E-2,-9.977982E-4,-9.459619E-3,-7.360313E-4,8.253267E-3,-4.7831098E-3,-8.712809E-4,-5.8485046E-3,-9.0576767E-4,-1.3265833E-3,8.960404E-3,-3.8603976E-4,4.7826343E-3,1.9822514E-3,-3.9561265E-3,-6.3757757E-3,1.3096635E-3,-1.1611597E-2,5.393076E-4,-2.5474208E-3,4.912733E-3,-9.490764E-3,-4.172909E-3,-2.178073E-3,2.9830497E-3,1.58622E-3,-1.1039061E-3,1.5771784E-3,-2.0206633E-3,-1.1553047E-3,-6.7945556E-3,9.922367E-6,8.078063E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,-1,-1,51,53,-1,55,57,-1,-1,-1,59,61,63,65,67,69,-1,-1,-1,71,73,-1,-1,75,-1,77,79,-1,81,-1,83,85,87,89,91,-1,-1,93,95,97,99,101,-1,-1,103,105,-1,107,-1,-1,109,111,-1,-1,-1,113,115,-1,117,-1,119,121,123,125,-1,127,129,131,-1,-1,-1,-1,133,-1,135,137,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.541152E-1,5.86744E-1,1.0697503E0,4.766279E-1,5.79023E-1,4.7822303E-1,2.2116822E-1,3.9197838E-1,1.408753E-1,3.098653E-1,0E0,3.962496E-1,5.3070486E-1,1.5079844E-1,2.3415077E-1,2.9639393E-1,4.5216936E-1,1.5796238E-1,7.864833E-2,2.1466446E-1,0E0,3.0523932E-1,1.5925235E-1,1.7562258E-1,2.6979285E-1,7.017821E-2,9.893304E-2,0E0,0E0,2.7571636E-1,1.7376277E-1,0E0,1.3439131E-1,1.1293799E-1,0E0,0E0,0E0,1.9910607E-1,1.916498E-1,1.9423258E-1,2.6014227E-1,2.0782058E-1,2.7038598E-1,0E0,0E0,0E0,2.591543E-1,7.881588E-2,0E0,0E0,9.74765E-2,0E0,2.741319E-1,1.3630822E-1,0E0,1.245808E-1,0E0,1.2091172E-1,7.45148E-2,1.0298589E-1,2.0403919E-1,1.20993614E-1,0E0,0E0,1.7699993E-1,2.3079008E-1,7.171958E-2,1.9044124E-1,1.8919441E-1,0E0,0E0,1.4587834E-1,1.15683645E-1,0E0,7.1893424E-2,0E0,0E0,2.7171937E-1,2.3957384E-1,0E0,0E0,0E0,1.166103E-1,1.8069851E-1,0E0,1.3022465E-1,0E0,1.3359275E-1,8.569751E-2,1.7465967E-1,6.700106E-2,0E0,1.2552583E-1,1.6122353E-1,2.338084E-1,0E0,0E0,0E0,0E0,1.0036272E-1,0E0,1.984173E-1,1.2484895E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,32,32,33,33,37,37,38,38,39,39,40,40,41,41,42,42,46,46,47,47,50,50,52,52,53,53,55,55,57,57,58,58,59,59,60,60,61,61,64,64,65,65,66,66,67,67,68,68,71,71,72,72,74,74,77,77,78,78,82,82,83,83,85,85,87,87,88,88,89,89,90,90,92,92,93,93,94,94,99,99,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,-1,-1,52,54,-1,56,58,-1,-1,-1,60,62,64,66,68,70,-1,-1,-1,72,74,-1,-1,76,-1,78,80,-1,82,-1,84,86,88,90,92,-1,-1,94,96,98,100,102,-1,-1,104,106,-1,108,-1,-1,110,112,-1,-1,-1,114,116,-1,118,-1,120,122,124,126,-1,128,130,132,-1,-1,-1,-1,134,-1,136,138,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7311627E2,1.4540612E7,6.747114E7,9.750871E6,1.2592834E-1,6.082581E3,1.5265896E8,9.837297E0,4.70092E1,7.26E2,-2.350188E-2,9.517604E6,3.379416E7,1.3715873E3,1.7243155E12,3.8E1,2E0,1.0430133E7,1.1214854E1,5.3421542E-2,1.124876E-2,1.753919E3,4.387561E1,1.49E2,2.7258065E0,4.099E3,1.92E2,-4.022346E-3,-1.5927585E-2,4.144144E-1,1.3332031E1,-1.429238E-2,9.561769E-1,7.948417E0,-6.0895886E-3,-6.9678994E-3,1.2828141E-3,1.5148404E1,4.6176647E5,2.1761298E0,1.4659271E7,1.683653E8,2.90002E5,7.0664478E-3,1.6197814E-2,-5.9784655E-3,5.388794E6,2.175722E6,1.8008121E-3,4.924045E-3,1.4299594E8,4.2950604E-3,4.85E2,1.741674E2,1.3795785E-2,4.034E3,-9.444725E-3,6.13E2,6.568389E0,3.0288122E0,2.011E3,1.9928571E0,-1.4224121E-2,6.6394038E-3,5.4582E4,1.6E1,8.3306855E6,1.4122614E2,4.423E3,-1.6297352E-2,-1.6510066E-3,1.3674345E4,9.760584E7,-3.1688637E-3,1.8159722E-1,-1.553963E-3,5.751386E-3,7.359468E8,9.309785E8,-5.7370793E-3,3.2220182E-3,-1.0095351E-2,1.9E1,1.6633E4,8.230975E-4,2.765625E0,-1.1633066E-3,3.325328E11,2.7E1,4.602015E0,1.16544716E-1,-8.911629E-4,2.183847E0,3.928E4,1.4387654E0,1.034975E-2,3.115209E-3,2.6726115E-3,-2.1974198E-3,1.2E2,8.83511E-3,2.46875E0,2.6037296E-7,1.4411918E-3,-1.0478729E-2,3.423715E-3,1.0648499E-2,-9.977982E-4,-9.459619E-3,-7.360313E-4,8.253267E-3,-4.7831098E-3,-8.712809E-4,-5.8485046E-3,-9.0576767E-4,-1.3265833E-3,8.960404E-3,-3.8603976E-4,4.7826343E-3,1.9822514E-3,-3.9561265E-3,-6.3757757E-3,1.3096635E-3,-1.1611597E-2,5.393076E-4,-2.5474208E-3,4.912733E-3,-9.490764E-3,-4.172909E-3,-2.178073E-3,2.9830497E-3,1.58622E-3,-1.1039061E-3,1.5771784E-3,-2.0206633E-3,-1.1553047E-3,-6.7945556E-3,9.922367E-6,8.078063E-3],"split_indices":[51,44,44,9,37,51,44,53,57,8,0,44,49,51,30,3,16,44,55,26,0,51,52,8,53,0,0,0,0,52,55,0,37,57,0,0,0,55,32,57,49,7,1,0,0,0,46,1,0,0,44,0,0,4,0,9,0,0,53,53,2,53,0,0,9,17,44,55,2,0,0,4,42,0,26,0,0,5,5,0,0,0,3,1,0,53,0,30,3,52,38,0,53,12,40,0,0,0,0,8,0,52,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.217E3,9.89E2,1.228E3,8.33E2,1.56E2,1.071E3,1.57E2,7.38E2,9.5E1,1.54E2,2E0,9.96E2,7.5E1,1.39E2,1.8E1,6.56E2,8.2E1,8.7E1,8E0,1.52E2,2E0,5.05E2,4.91E2,2.8E1,4.7E1,6.7E1,7.2E1,1.4E1,4E0,6.21E2,3.5E1,7E0,7.5E1,8.4E1,3E0,4E0,4E0,8.4E1,6.8E1,4.31E2,7.4E1,4.68E2,2.3E1,2.2E1,6E0,7E0,4E1,6.3E1,4E0,5E0,6.7E1,1.9E1,6.02E2,3.3E1,2E0,7E1,5E0,3.1E1,5.3E1,5.5E1,2.9E1,6.4E1,4E0,1.3E1,4.18E2,3.8E1,3.6E1,1.66E2,3.02E2,2E0,2.1E1,7E0,3.3E1,5.2E1,1.1E1,6.4E1,3E0,4.73E2,1.29E2,3E0,3E1,3E0,6.7E1,2.2E1,9E0,4.1E1,1.2E1,4.3E1,1.2E1,1.3E1,1.6E1,1.4E1,5E1,1.42E2,2.76E2,2E1,1.8E1,2.9E1,7E0,1.6E2,6E0,7E1,2.32E2,4E0,3E0,2.8E1,5E0,3E0,8E0,4.67E2,6E0,6.8E1,6.1E1,1.1E1,5.6E1,4E0,1.8E1,1.6E1,2.5E1,3.4E1,9E0,8E0,4E0,1E1,3E0,1.4E1,2E0,1.3E1,3.7E1,1.3E1,1.29E2,1.68E2,1.08E2,1.43E2,1.7E1,5.4E1,1.6E1,2.29E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"139","size_leaf_vector":"1"}},{"base_weights":[-1.0132832E-3,-1.7621595E-2,2.3291795E-2,-1.47186015E-2,-1.4414509E-1,4.378324E-2,-3.6479738E-2,-2.1002004E-2,3.9611038E-2,-8.997806E-2,-2.7508628E-1,9.566626E-2,3.4473483E-2,-5.90461E-2,5.7240287E-3,-2.1934716E-2,8.82499E-3,6.922338E-2,-1.365272E-2,-1.2691917E-1,3.310776E-4,-3.2172573E-3,-1.7352983E-2,-5.968463E-2,1.1359063E-1,2.9783826E-2,1.5386966E-1,-4.9432267E-2,-2.4563444E-1,4.0155865E-2,-3.3388942E-2,-1.6955193E-2,-6.1323725E-2,8.978277E-2,-1.1341291E-3,-5.009575E-3,2.5383412E-4,-8.610113E-3,-1.8058869E-3,-1.292672E-2,2.7478915E-3,1.4207423E-1,5.992555E-4,2.2633588E-2,9.398166E-2,1.5766997E-2,8.764559E-2,-7.4500516E-2,-3.3334133E-4,-1.0457373E-3,-1.4941735E-2,1.7781774E-2,5.0802887E-3,5.3620287E-3,-2.478024E-3,-1.7706724E-2,1.09092435E-2,-1.5542698E-1,-3.6735408E-2,7.191965E-2,1.08782705E-2,1.574343E-1,4.5376178E-2,5.7791688E-2,1.0139469E-2,7.340432E-2,2.809587E-1,-2.0765941E-4,6.2646065E-3,-5.266063E-2,-1.3600598E-1,4.058522E-3,-1.746388E-2,-2.3285144E-4,5.687405E-3,5.895898E-2,-2.0202212E-2,-2.2775126E-1,-3.6098294E-2,2.4409594E-3,-4.8285667E-2,8.162694E-3,5.0907295E-2,7.22202E-2,1.7509012E-1,-6.0546934E-3,4.529114E-3,1.8607463E-4,7.795029E-2,1.2004732E-2,-9.827702E-3,8.958598E-2,-9.5061865E-3,3.8803173E-3,1.7319866E-2,-8.051012E-2,-9.550683E-3,-1.8453065E-1,1.8255871E-3,4.080689E-3,-1.7159026E-3,3.8172407E-3,-4.292991E-3,-7.9102983E-4,-5.071265E-3,-1.1694835E-2,-4.1716782E-4,3.3211978E-3,-6.317437E-3,-8.1906666E-4,-5.080425E-3,2.7975603E-3,-3.1825982E-3,5.826026E-3,-1.4743768E-3,9.693795E-3,5.3180815E-3,5.1501123E-3,1.968426E-3,1.7569008E-3,-2.549368E-4,8.073715E-3,2.4923736E-3,-6.1294767E-3,-1.9457171E-3,5.5697013E-4,-6.7551853E-3,-2.1200834E-3,-9.875584E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,-1,-1,39,41,43,45,47,49,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,61,-1,63,65,-1,67,69,71,-1,-1,73,-1,-1,-1,75,-1,77,79,81,-1,83,85,87,89,91,93,-1,-1,95,97,-1,99,-1,-1,101,103,105,107,-1,109,-1,111,113,115,-1,-1,-1,117,119,-1,121,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.8201773E-1,4.7462985E-1,1.0879996E0,4.3437812E-1,1.8093652E-1,3.159771E-1,2.1652949E-1,2.3366702E-1,2.0914403E-1,7.9996824E-2,1.4936972E-1,2.852363E-1,3.1009609E-1,2.5334448E-1,1.0907928E-1,2.215774E-1,0E0,1.6699696E-1,9.028963E-2,8.050215E-2,0E0,0E0,0E0,3.1637788E-1,2.553805E-1,2.4666709E-1,2.3981315E-1,1.7475218E-1,1.238437E-1,6.732905E-2,1.2962878E-1,1.9984314E-1,2.8814808E-1,1.7089969E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.6826434E-2,0E0,2.142168E-1,1.8714085E-1,0E0,8.016972E-2,1.191054E-1,7.709189E-2,0E0,0E0,8.470195E-2,0E0,0E0,0E0,1.9384864E-1,0E0,2.1616793E-1,1.0940133E-1,1.3656461E-1,0E0,7.762635E-2,1.21074215E-1,1.3819462E-1,1.5602472E-1,2.4808401E-1,6.6397935E-2,0E0,0E0,8.447297E-2,2.1279234E-1,0E0,9.577055E-2,0E0,0E0,1.3276859E-1,2.426829E-1,1.04480565E-1,1.3477942E-1,0E0,1.7362796E-1,0E0,7.690233E-2,7.518981E-2,8.5039735E-2,0E0,0E0,0E0,1.0983914E-1,1.7037593E-1,0E0,1.3729253E-1,0E0,0E0,0E0,8.274883E-2,9.034048E-2,7.769191E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,41,41,43,43,44,44,46,46,47,47,48,48,51,51,55,55,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,72,72,75,75,76,76,77,77,78,78,80,80,82,82,83,83,84,84,88,88,89,89,91,91,95,95,96,96,97,97],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,-1,-1,40,42,44,46,48,50,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,62,-1,64,66,-1,68,70,72,-1,-1,74,-1,-1,-1,76,-1,78,80,82,-1,84,86,88,90,92,94,-1,-1,96,98,-1,100,-1,-1,102,104,106,108,-1,110,-1,112,114,116,-1,-1,-1,118,120,-1,122,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.862126E2,4.3702424E7,4.2343444E7,9.750871E6,9.112E3,1.0993947E1,2.6476662E6,1.3137E4,3.52E2,4.6547272E2,3E0,1.3314917E0,3.832274E2,1.2367184E3,6.747114E7,6.4418E4,8.82499E-3,1.655567E6,6.203E3,6.9666364E2,3.310776E-4,-3.2172573E-3,-1.7352983E-2,3.596E3,5.3474556E5,9.3058E4,4.8083666E-1,3.126405E3,1.4713E4,1.9745072E7,1.289627E3,1.3238013E2,9.2641E4,7.704183E-1,-1.1341291E-3,-5.009575E-3,2.5383412E-4,-8.610113E-3,-1.8058869E-3,-1.292672E-2,2.7478915E-3,3.2613106E5,5.992555E-4,2.5291866E1,4.516183E1,1.5766997E-2,1.6E1,1.8600049E10,3.2449896E0,-1.0457373E-3,-1.4941735E-2,5.212854E8,5.0802887E-3,5.3620287E-3,-2.478024E-3,1.0620689E0,1.09092435E-2,6.067634E5,3.307175E-2,2.7460318E0,1.08782705E-2,1.732E3,8.646E4,7.7329254E2,3.5997456E7,1.3144558E4,8.91E2,-2.0765941E-4,6.2646065E-3,2.5807297E-2,3.582157E12,4.058522E-3,5.4106476E7,-2.3285144E-4,5.687405E-3,1E0,4.5158855E6,1.7027656E3,5.894636E6,2.4409594E-3,3.6401E4,8.162694E-3,1.91E2,6.567044E2,7.696629E0,-6.0546934E-3,4.529114E-3,1.8607463E-4,1.34748E8,4.0208E4,-9.827702E-3,2.3887E4,-9.5061865E-3,3.8803173E-3,1.7319866E-2,3.6002816E-6,3.650307E9,6.85945E4,1.8255871E-3,4.080689E-3,-1.7159026E-3,3.8172407E-3,-4.292991E-3,-7.9102983E-4,-5.071265E-3,-1.1694835E-2,-4.1716782E-4,3.3211978E-3,-6.317437E-3,-8.1906666E-4,-5.080425E-3,2.7975603E-3,-3.1825982E-3,5.826026E-3,-1.4743768E-3,9.693795E-3,5.3180815E-3,5.1501123E-3,1.968426E-3,1.7569008E-3,-2.549368E-4,8.073715E-3,2.4923736E-3,-6.1294767E-3,-1.9457171E-3,5.5697013E-4,-6.7551853E-3,-2.1200834E-3,-9.875584E-3],"split_indices":[51,44,44,9,10,55,31,2,0,51,8,53,55,57,44,11,0,49,28,4,0,0,0,2,31,2,41,54,9,1,4,55,11,40,0,0,0,0,0,0,0,32,0,57,53,0,8,5,53,0,0,31,0,0,0,53,0,46,56,57,0,2,1,51,42,51,0,0,0,40,30,0,44,0,0,112,46,4,44,0,9,0,3,51,55,0,0,0,7,9,0,9,0,0,0,38,12,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.183E3,1.297E3,8.86E2,1.269E3,2.8E1,6.6E2,2.26E2,1.138E3,1.31E2,2.1E1,7E0,9.9E1,5.61E2,1.47E2,7.9E1,1.134E3,4E0,8.4E1,4.7E1,1.5E1,6E0,3E0,4E0,1E1,8.9E1,5.41E2,2E1,1.41E2,6E0,4.2E1,3.7E1,1.008E3,1.26E2,6.9E1,1.5E1,7E0,4E1,8E0,7E0,3E0,7E0,6.9E1,2E1,4.88E2,5.3E1,4E0,1.6E1,9.3E1,4.8E1,2E0,4E0,3.3E1,9E0,4E0,3.3E1,1.006E3,2E0,2.5E1,1.01E2,6.3E1,6E0,5.9E1,1E1,1.27E2,3.61E2,4.9E1,4E0,6E0,1E1,7E1,2.3E1,7E0,4.1E1,2.8E1,5E0,3.1E1,9.75E2,1.5E1,1E1,1.1E1,9E1,9E0,5.4E1,1.1E1,4.8E1,2E0,8E0,3.5E1,9.2E1,3.59E2,2E0,4.7E1,2E0,2E0,2E0,4.2E1,2.8E1,1.8E1,5E0,6E0,3.5E1,2.7E1,4E0,9.48E2,2.7E1,1.3E1,2E0,5E0,5E0,6.2E1,2.8E1,5E1,4E0,7E0,4E0,2.7E1,2.1E1,4.4E1,4.8E1,1.42E2,2.17E2,1.2E1,3.5E1,1.6E1,2.6E1,2.5E1,3E0,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[-2.6762218E-3,8.415443E-3,-3.876609E-2,-1.4397979E-2,2.9371157E-2,-4.7898028E-2,2.8616913E-2,5.500085E-3,-3.1956952E-2,2.5254073E-2,1.56834E-1,-3.5208896E-2,-1.1835402E-1,5.0397057E-2,-3.6629036E-3,1.3909803E-2,-6.6219226E-2,-3.2694623E-1,-2.8500753E-2,2.7394248E-2,-2.0984417E-1,2.2026882E-1,-1.5582177E-3,-1.6004857E-1,-3.0240659E-2,-7.6352686E-2,-2.216398E-1,3.6632991E-3,1.0552323E-4,-4.2863335E-2,2.6565647E-2,-1.1694919E-2,-2.0036462E-3,-4.359979E-3,-2.0310264E-2,-2.5476601E-2,-2.1019854E-1,-3.4101729E-3,3.749464E-2,-3.9414437E-3,-1.4074145E-2,1.2653641E-2,1.6073604E-3,7.3140515E-3,-3.2252965E-3,-8.642815E-4,-1.1404499E-2,-2.6915845E-2,-1.5189712E-1,-1.09343424E-1,-9.0484704E-5,-1.6188645E-3,-2.5233927E-1,5.5697545E-2,-7.969807E-2,2.1446435E-1,1.8980676E-2,-4.723673E-2,3.8724637E-3,-7.600755E-4,-2.0938417E-2,-8.2098935E-3,1.689722E-3,1.0333642E-1,3.004047E-2,-1.0150731E-1,-2.1753596E-2,-1.3801552E-2,-3.4206456E-3,4.8541403E-3,-1.2523572E-1,-1.4212485E-2,-3.9441483E-3,5.928802E-3,-1.855229E-4,-1.4068399E-3,-1.2908526E-1,3.4978756E-4,1.3175873E-2,3.3518907E-2,-3.3915594E-2,-1.62487E-1,-3.7468206E-2,5.898109E-2,-1.9472511E-2,5.363849E-3,-5.361121E-3,1.374597E-1,2.129857E-4,3.1940695E-2,-8.960337E-3,4.0812886E-4,-5.9338976E-3,5.130218E-2,-3.0238755E-2,-6.7823683E-3,2.1387825E-3,-6.656403E-3,1.7892933E-3,1.7094454E-3,-6.9614295E-3,-3.3287916E-3,1.1082769E-3,-1.2718927E-2,-3.168393E-3,1.5111907E-3,-2.6007325E-3,3.3904824E-3,-5.2095465E-3,-1.082052E-2,-3.1341962E-4,-1.3942762E-3,1.0848003E-3,-2.387203E-3,7.0417738E-3,5.683543E-3,1.270089E-3,5.6975614E-3,8.6166506E-4,-2.5052202E-3,-5.4031797E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,-1,-1,-1,57,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,65,67,69,-1,-1,71,73,75,77,79,81,83,-1,-1,-1,85,87,89,91,93,-1,-1,-1,95,-1,-1,-1,-1,-1,97,-1,-1,99,101,103,105,107,109,-1,111,113,-1,115,-1,-1,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0419626E-1,8.2653046E-1,3.281147E-1,2.8947037E-1,4.6886182E-1,4.1423678E-1,1.5674365E-1,2.3554225E-1,4.3928027E-1,4.4550288E-1,2.8167552E-1,2.4201292E-1,2.9108524E-1,7.902072E-2,0E0,2.5189933E-1,1.6154796E-1,9.710109E-2,2.3421884E-1,2.698834E-1,7.216996E-2,2.0892024E-1,1.1379985E-1,1.8939954E-1,1.5123388E-1,1.2733728E-1,1.0510713E-1,0E0,0E0,2.3678869E-1,4.0353817E-1,0E0,0E0,0E0,0E0,2.747979E-1,3.337959E-1,1.9540039E-1,3.173855E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.421892E-1,9.373242E-2,1.3732657E-1,0E0,0E0,1.4562368E-1,8.153627E-2,1.0885501E-1,1.7186087E-1,2.1338579E-1,2.7199066E-1,2.3786627E-1,0E0,0E0,0E0,1.7355703E-1,2.2148603E-1,2.6102197E-1,7.958807E-2,2.1975252E-1,0E0,0E0,0E0,1.5116978E-1,0E0,0E0,0E0,0E0,0E0,7.5380266E-2,0E0,0E0,1.7807512E-1,1.4058109E-1,1.8851733E-1,3.2600975E-1,1.6089006E-1,3.5649985E-1,0E0,1.4915457E-1,1.7666715E-1,0E0,2.0895714E-1,0E0,0E0,0E0,8.660968E-2,1.4449885E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,35,35,36,36,37,37,38,38,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,57,57,58,58,62,62,63,63,64,64,65,65,66,66,70,70,76,76,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,89,89,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,-1,-1,-1,58,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,66,68,70,-1,-1,72,74,76,78,80,82,84,-1,-1,-1,86,88,90,92,94,-1,-1,-1,96,-1,-1,-1,-1,-1,98,-1,-1,100,102,104,106,108,110,-1,112,114,-1,116,-1,-1,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,3.8637936E2,3.4409692E3,2.5074627E0,2.62797E5,1.2199979E10,1.3439535E1,1E0,8E0,1.0040462E8,2.5239267E5,4.36E2,9.222717E-1,9.827109E9,-3.6629036E-3,4.7E2,9.42812E3,2E0,4.597929E6,7.805608E4,1.2E1,2.2576077E10,2.081E3,1E0,7.8955513E-1,2.1907706E12,4.8543688E-2,3.6632991E-3,1.0552323E-4,2.645E3,3.2351852E1,-1.1694919E-2,-2.0036462E-3,-4.359979E-3,-2.0310264E-2,2.5275471E2,3.3030225E2,7.854E3,1.4075E4,-3.9414437E-3,-1.4074145E-2,1.2653641E-2,1.6073604E-3,7.3140515E-3,-3.2252965E-3,-8.642815E-4,-1.1404499E-2,5.2143492E-5,3.6709285E6,1.1451E4,-9.0484704E-5,-1.6188645E-3,1.0894445E-2,1.2068493E4,6.7E1,4.1161446E1,1.0155283E-5,1.198E5,3.47E3,-7.600755E-4,-2.0938417E-2,-8.2098935E-3,1.2E1,5.054757E11,1E0,1.279012E7,5.524862E-3,-1.3801552E-2,-3.4206456E-3,4.8541403E-3,2.203288E2,-1.4212485E-2,-3.9441483E-3,5.928802E-3,-1.855229E-4,-1.4068399E-3,8.888114E4,3.4978756E-4,1.3175873E-2,6.1313953E1,1.01036356E6,1.5561539E2,1.2839224E0,1E0,5.8601086E1,5.363849E-3,8.482736E-2,7.449623E2,2.129857E-4,4.1E1,-8.960337E-3,4.0812886E-4,-5.9338976E-3,6.7913E4,2.074E3,-6.7823683E-3,2.1387825E-3,-6.656403E-3,1.7892933E-3,1.7094454E-3,-6.9614295E-3,-3.3287916E-3,1.1082769E-3,-1.2718927E-2,-3.168393E-3,1.5111907E-3,-2.6007325E-3,3.3904824E-3,-5.2095465E-3,-1.082052E-2,-3.1341962E-4,-1.3942762E-3,1.0848003E-3,-2.387203E-3,7.0417738E-3,5.683543E-3,1.270089E-3,5.6975614E-3,8.6166506E-4,-2.5052202E-3,-5.4031797E-4],"split_indices":[26,51,51,55,2,5,53,112,17,50,32,2,26,5,0,1,46,10,46,27,17,12,0,111,37,30,57,0,0,9,49,0,0,0,0,51,51,9,9,0,0,0,0,0,0,0,0,41,31,12,0,0,37,27,28,54,38,7,28,0,0,0,17,30,90,7,56,0,0,0,56,0,0,0,0,0,44,0,0,45,50,51,52,79,49,0,37,4,0,0,0,0,0,12,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.257E3,1.727E3,5.3E2,8.27E2,9E2,4.67E2,6.3E1,3.88E2,4.39E2,8.73E2,2.7E1,3.97E2,7E1,5.3E1,1E1,3.48E2,4E1,4E0,4.35E2,8.66E2,7E0,1.9E1,8E0,1.4E1,3.83E2,5.1E1,1.9E1,3.2E1,2.1E1,6.3E1,2.85E2,3E0,3.7E1,2E0,2E0,4.29E2,6E0,2.14E2,6.52E2,4E0,3E0,1.4E1,5E0,2E0,6E0,6E0,8E0,3.74E2,9E0,3.5E1,1.6E1,3E0,1.6E1,1.7E1,4.6E1,1E1,2.75E2,2.46E2,1.83E2,4E0,2E0,5E0,2.09E2,6.5E1,5.87E2,2.3E1,3.51E2,2E0,7E0,2E0,3.3E1,1.1E1,5E0,7E0,1E1,2.4E1,2.2E1,3E0,7E0,2.16E2,5.9E1,1.8E1,2.28E2,5.4E1,1.29E2,1.1E1,1.98E2,4.8E1,1.7E1,5.83E2,4E0,5E0,1.8E1,3.6E1,3.15E2,2.9E1,4E0,2E1,2E0,2.12E2,4E0,3.5E1,2.4E1,7E0,1.1E1,5E1,1.78E2,5E1,4E0,6E0,1.23E2,1.06E2,9.2E1,4E0,4.4E1,2.2E1,5.61E2,1E1,2.6E1,1.32E2,1.83E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[1.05575E-3,7.1668113E-3,-4.7468845E-2,-2.4919056E-3,3.6245976E-2,-6.5505944E-2,2.5628911E-2,-1.0326963E-2,2.6293753E-2,-1.18272295E-2,4.960261E-2,-5.208409E-2,-2.0692447E-1,5.3363102E-3,-3.773218E-4,-1.2764206E-2,1.377339E-1,4.609664E-2,-6.765156E-3,-4.4010594E-3,-1.22364685E-2,7.466401E-2,4.0236837E-3,-1.1870335E-1,-3.275441E-2,-1.3409043E-2,-5.9452346E-3,-8.169097E-3,1.38478475E-2,7.0232354E-2,-1.5355707E-2,2.0273517E-1,1.9460624E-3,7.4517794E-2,6.299613E-3,-1.2241185E-1,1.2203679E-2,-2.7817072E-2,6.1822727E-2,6.405378E-2,1.6320354E-1,-7.758806E-2,2.6448717E-2,-2.25235E-2,-1.8589227E-1,9.140135E-3,-3.7480917E-2,5.7523586E-3,-5.138292E-4,2.0592175E-3,7.3674354E-3,-3.572216E-2,-5.2880924E-3,1.391934E-2,4.481162E-3,-1.8428518E-3,8.820406E-2,4.580615E-3,-2.298989E-2,-6.914144E-3,3.1656697E-3,2.3321403E-2,-5.619592E-3,3.8215711E-3,-3.6540654E-2,-2.4901556E-3,3.7842346E-3,8.489935E-2,2.7618539E-2,1.0379215E-2,4.084341E-3,-2.0338899E-2,-9.45816E-3,4.1445863E-2,-1.1093E-1,-5.7724393E-3,1.8376645E-3,-6.3924524E-3,-1.4827358E-2,-1.9555956E-2,-7.696024E-2,-1.0561227E-2,-7.541313E-2,7.966492E-3,-3.3549808E-2,7.4262195E-3,7.255383E-2,2.4723558E-2,-8.692279E-2,-2.3855441E-3,4.2512853E-2,-3.764677E-3,-5.66084E-4,9.9990435E-2,1.3724113E-2,3.9390717E-2,-8.766946E-2,-5.213938E-3,1.4325263E-2,2.1216482E-2,2.0135212E-1,1.9373503E-3,-7.874254E-3,-9.175689E-3,-1.3691015E-2,1.0619371E-3,-4.524066E-3,-1.2013326E-4,-9.136722E-3,6.0637044E-3,-3.8015943E-3,8.187884E-4,-1.5982927E-3,-1.3052067E-2,-1.2167197E-3,1.0094667E-3,4.840615E-3,3.4152982E-3,-3.5586997E-3,-5.985842E-3,1.0585986E-3,-3.8756267E-4,3.5416977E-3,6.465147E-3,2.9313166E-3,7.78307E-3,-2.997603E-4,8.272174E-3,1.4033928E-3,-7.890009E-3,1.924226E-3,-2.5846364E-3,5.6366487E-3,-6.6828274E-4,2.2444122E-3,-3.1229411E-3,1.2182764E-2,1.3005502E-4,-3.0382269E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,41,43,45,-1,-1,-1,47,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,77,-1,79,-1,-1,-1,-1,81,83,-1,-1,-1,85,-1,87,-1,-1,89,-1,-1,91,-1,-1,93,95,-1,-1,97,-1,99,101,-1,-1,-1,-1,103,105,107,109,111,113,-1,115,117,119,-1,121,-1,-1,123,125,127,129,-1,131,133,135,-1,-1,-1,137,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.6013455E-1,5.553652E-1,3.299781E-1,3.3519438E-1,3.1714004E-1,3.688423E-1,1.199829E-1,4.2324296E-1,2.0849434E-1,2.0609334E-1,4.4061643E-1,2.3305407E-1,7.582426E-2,0E0,1.04953855E-1,2.4842696E-1,1.08303845E-1,2.2461501E-1,2.6473582E-1,1.6609362E-1,0E0,2.2346294E-1,2.5461227E-1,2.60764E-1,1.6726647E-1,0E0,0E0,0E0,1.0983869E-1,7.2720975E-2,2.2863752E-1,8.907899E-2,0E0,1.8651795E-1,2.3575093E-1,1.0976976E-1,1.6052295E-1,8.0427065E-2,7.578456E-2,1.677863E-1,1.0466331E-1,2.2129238E-1,2.2916904E-1,1.2443309E-1,1.0566926E-1,0E0,9.898345E-2,0E0,0E0,0E0,0E0,3.677098E-1,2.8046608E-1,0E0,0E0,0E0,1.1151284E-1,0E0,2.0042886E-1,0E0,0E0,1.4374311E-1,0E0,0E0,8.187238E-2,0E0,0E0,1.5115154E-1,1.161002E-1,0E0,0E0,7.396463E-2,0E0,3.1597358E-1,1.1396259E-1,0E0,0E0,0E0,0E0,1.0354681E-1,1.03815675E-1,3.42767E-1,2.6737052E-1,2.2393093E-1,3.9865908E-1,0E0,1.4959693E-1,2.0504981E-1,1.4297445E-1,0E0,1.4294922E-1,0E0,0E0,1.6728508E-1,8.481471E-2,8.575656E-2,9.7592846E-2,0E0,1.4086522E-1,9.1532186E-2,2.1998942E-1,0E0,0E0,0E0,8.601578E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,51,51,52,52,56,56,58,58,61,61,64,64,67,67,68,68,71,71,73,73,74,74,79,79,80,80,81,81,82,82,83,83,84,84,86,86,87,87,88,88,90,90,93,93,94,94,95,95,96,96,98,98,99,99,100,100,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,42,44,46,-1,-1,-1,48,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,78,-1,80,-1,-1,-1,-1,82,84,-1,-1,-1,86,-1,88,-1,-1,90,-1,-1,92,-1,-1,94,96,-1,-1,98,-1,100,102,-1,-1,-1,-1,104,106,108,110,112,114,-1,116,118,120,-1,122,-1,-1,124,126,128,130,-1,132,134,136,-1,-1,-1,138,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.688607E-1,1.0855529E3,3.4540886E3,4.264897E6,8.079531E4,1.2427474E8,6.504065E-2,1.1037509E2,1.4952381E1,1.921032E4,2.1938796E7,1.7314286E0,3.773E3,5.3363102E-3,7.341606E1,1.0620689E0,2.177294E-2,3.8773947E0,3.28825E0,1.3359244E9,-1.22364685E-2,5.3534385E3,2.64E0,1.4615384E0,1.7997802E0,-1.3409043E-2,-5.9452346E-3,-8.169097E-3,3.3517068E7,1.926994E1,1.8122449E2,2.7523365E0,1.9460624E-3,8.777E3,9.9E1,2.4848485E0,1E0,1.3794063E1,5.2217197E-2,6.5E1,1E0,5.1797544E7,2.6572757E10,8.31E2,7.8099336E11,9.140135E-3,3.6E1,5.7523586E-3,-5.138292E-4,2.0592175E-3,7.3674354E-3,1.9090909E0,1.3446785E7,1.391934E-2,4.481162E-3,-1.8428518E-3,2.6006784E2,4.580615E-3,1.7E1,-6.914144E-3,3.1656697E-3,1.4343751E6,-5.619592E-3,3.8215711E-3,6.131E3,-2.4901556E-3,3.7842346E-3,2.8091298E7,1E0,1.0379215E-2,4.084341E-3,9.1233586E4,-9.45816E-3,1.921032E4,3.0161016E7,-5.7724393E-3,1.8376645E-3,-6.3924524E-3,-1.4827358E-2,1.8736842E0,1.53E2,3.0923106E-10,2.00059E5,1.1095187E1,4E0,7.4262195E-3,1.2738854E0,6.184386E5,9.330579E0,-2.3855441E-3,3.3737933E8,-3.764677E-3,-5.66084E-4,1.6622689E5,1.2245615E0,1.845044E10,1.612E3,-5.213938E-3,2.653986E6,3.368326E11,4.423676E1,1.9373503E-3,-7.874254E-3,-9.175689E-3,9.4996613E-1,1.0619371E-3,-4.524066E-3,-1.2013326E-4,-9.136722E-3,6.0637044E-3,-3.8015943E-3,8.187884E-4,-1.5982927E-3,-1.3052067E-2,-1.2167197E-3,1.0094667E-3,4.840615E-3,3.4152982E-3,-3.5586997E-3,-5.985842E-3,1.0585986E-3,-3.8756267E-4,3.5416977E-3,6.465147E-3,2.9313166E-3,7.78307E-3,-2.997603E-4,8.272174E-3,1.4033928E-3,-7.890009E-3,1.924226E-3,-2.5846364E-3,5.6366487E-3,-6.6828274E-4,2.2444122E-3,-3.1229411E-3,1.2182764E-2,1.3005502E-4,-3.0382269E-3],"split_indices":[26,51,4,9,27,44,56,55,55,4,50,52,2,0,57,53,56,53,53,7,0,4,53,52,52,0,0,0,50,57,51,52,0,1,10,56,79,55,37,8,109,49,12,2,30,0,3,0,0,0,0,55,44,0,0,0,4,0,8,0,0,44,0,0,2,0,0,44,89,0,0,32,0,4,44,0,0,0,0,52,11,36,5,53,3,0,52,27,55,0,7,0,0,32,52,30,0,0,1,30,55,0,0,0,26,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.224E3,1.976E3,2.48E2,1.484E3,4.92E2,1.99E2,4.9E1,1.167E3,3.17E2,1.07E2,3.85E2,1.83E2,1.6E1,1E1,3.9E1,1.149E3,1.8E1,1.98E2,1.19E2,1.05E2,2E0,2.48E2,1.37E2,4E1,1.43E2,6E0,1E1,2E0,3.7E1,3.4E1,1.115E3,1E1,8E0,1.15E2,8.3E1,1.6E1,1.03E2,7.8E1,2.7E1,2.23E2,2.5E1,2.9E1,1.08E2,1.7E1,2.3E1,2E0,1.41E2,6E0,3.1E1,2.8E1,6E0,3.68E2,7.47E2,4E0,6E0,1.2E1,1.03E2,1.9E1,6.4E1,1.4E1,2E0,9.6E1,7E0,5E0,7.3E1,4E0,2.3E1,1.41E2,8.2E1,1.2E1,1.3E1,2.1E1,8E0,9.8E1,1E1,6E0,1.1E1,1.9E1,4E0,9.8E1,4.3E1,2.26E2,1.42E2,5.09E2,2.38E2,1.6E1,8.7E1,3.7E1,2.7E1,1.9E1,7.7E1,2.4E1,4.9E1,1.16E2,2.5E1,7.5E1,7E0,5E0,1.6E1,8.8E1,1E1,3E0,7E0,2E0,9.6E1,8E0,3.5E1,2.18E2,8E0,5E0,1.37E2,4.13E2,9.6E1,5E0,2.33E2,3.6E1,5.1E1,2.5E1,1.2E1,1.9E1,8E0,3.2E1,4.5E1,5.1E1,6.5E1,2E0,2.3E1,3E0,7.2E1,4E0,3E0,1E1,6E0,3.9E1,4.9E1,2E0,8E0,7.4E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"139","size_leaf_vector":"1"}},{"base_weights":[-9.887988E-4,-1.3045841E-2,2.410885E-2,-2.3945458E-2,2.0014267E-2,5.7790324E-2,1.0655029E-2,-1.4459855E-2,-5.126121E-2,2.9468842E-2,-4.070621E-2,5.0910905E-2,2.0042324E-1,5.064518E-3,1.4736931E-1,-2.8540099E-2,1.8264154E-2,-2.3308828E-1,-4.721077E-2,3.267704E-2,-6.489486E-3,-7.093135E-2,5.587588E-4,1.1196592E-1,3.392895E-2,1.3866127E-2,2.4669853E-3,-4.586358E-3,4.4821627E-2,1.1369554E-2,7.2181754E-2,-8.333062E-3,-5.462456E-2,3.8973592E-2,-2.7055945E-2,-2.1819016E-2,5.3502154E-4,-3.670557E-2,-1.157802E-1,1.17990635E-1,2.6159713E-2,-4.4492744E-3,4.4662505E-4,-1.7341005E-3,1.2450713E-1,1.9447566E-3,7.032838E-2,5.7441657E-3,-1.049756E-1,5.3985454E-2,-8.124084E-3,6.657419E-4,8.956492E-3,2.7538491E-2,-2.7829621E-2,-1.8724587E-2,-4.8662707E-2,2.749132E-2,1.4204776E-1,-3.8601056E-2,6.8604485E-3,-7.285146E-2,-4.418329E-3,7.4496E-4,-1.4718345E-1,-2.3084148E-3,6.671342E-3,5.5252384E-2,6.227214E-3,6.4335396E-3,1.8331748E-4,-8.659714E-2,2.4381584E-2,1.07097305E-1,-1.3865464E-3,6.109628E-2,-2.7626895E-3,-2.2920049E-3,-6.979978E-3,7.3006555E-2,-1.6001873E-3,-5.1839724E-2,6.99986E-2,-2.3412077E-2,-1.14449505E-2,-5.803645E-2,2.8086666E-3,1.1545986E-1,1.590193E-2,3.2978482E-3,9.543699E-3,-3.4131324E-3,-1.3770134E-2,-1.5648872E-1,-5.271597E-2,1.5339794E-2,-7.764784E-2,-5.5186666E-4,-7.582613E-3,-9.1212094E-2,6.663011E-2,1.4768472E-2,-6.9247633E-3,-6.5789158E-3,8.168529E-4,1.0540476E-2,1.0854154E-2,7.0334272E-3,2.3094653E-3,1.2627825E-1,1.403643E-2,2.784182E-4,-3.906247E-3,-2.3141E-3,5.4480038E-3,9.215619E-3,2.8552003E-3,-4.6929014E-3,-1.0765994E-4,-2.1420906E-3,3.5733117E-3,-3.023613E-3,5.3359934E-5,-6.4598667E-3,-1.933331E-3,1.4672253E-3,8.08868E-3,1.0490201E-3,-4.7998186E-3,-3.8770614E-3,3.8805476E-4,-8.936801E-4,-8.853229E-3,-6.2805265E-3,-1.4574773E-3,1.781261E-3,-2.8109446E-3,-1.17147835E-2,-1.6607487E-3,-9.65566E-4,-1.0135608E-2,2.0287111E-3,5.8725635E-3,-2.2890035E-3,1.3329915E-3,2.328191E-3,-1.5682942E-3,3.587842E-3,1.1387366E-2,-2.415474E-3,3.2653557E-3,-3.254178E-4,2.120924E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,-1,-1,47,49,-1,51,53,55,57,59,-1,-1,61,63,65,67,-1,-1,-1,69,71,73,75,77,79,-1,-1,-1,81,83,-1,85,87,89,91,-1,93,95,-1,97,-1,-1,99,101,-1,-1,103,105,107,-1,109,111,113,-1,115,-1,117,119,121,-1,123,-1,125,127,-1,-1,-1,129,131,133,135,137,-1,-1,139,141,143,-1,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.639238E-1,5.345069E-1,3.2202315E-1,2.881624E-1,2.1246332E-1,1.8898618E-1,3.8904804E-1,3.8249296E-1,2.0036536E-1,1.8153706E-1,8.1044495E-2,1.9838369E-1,1.22332275E-1,1.8859781E-1,1.3790035E-1,3.052568E-1,2.3569292E-1,3.7135634E-1,1.9888484E-1,1.7150256E-1,0E0,7.1664646E-2,0E0,8.391988E-2,1.7915103E-1,0E0,0E0,4.1128415E-1,2.0697887E-1,0E0,9.097461E-2,2.3016037E-1,5.213101E-1,1.9966137E-1,1.6873282E-1,0E0,0E0,2.8677267E-1,1.5606797E-1,1.1667597E-1,1.6956362E-1,0E0,0E0,0E0,8.208513E-2,1.6699912E-1,2.7054197E-1,1.6968288E-1,1.865584E-1,1.606018E-1,0E0,0E0,0E0,3.9459604E-1,2.0575818E-1,0E0,2.630183E-1,1.5720624E-1,6.553048E-2,6.747173E-2,0E0,1.8710345E-1,1.9073208E-1,0E0,8.016771E-2,0E0,0E0,2.0438299E-1,2.40188E-1,0E0,0E0,1.0707076E-1,1.9719017E-1,1.3551801E-1,0E0,1.4546528E-1,7.952158E-2,8.3978646E-2,0E0,8.7096065E-2,0E0,1.0483274E-1,8.333281E-2,2.2405747E-1,0E0,2.9373187E-1,0E0,8.869155E-2,1.256352E-1,0E0,0E0,0E0,7.459043E-2,1.1364919E-1,1.6322494E-1,1.9573557E-1,1.9824311E-1,0E0,0E0,7.866427E-2,1.4697573E-1,1.6158073E-1,0E0,0E0,0E0,1.19999886E-1,0E0,0E0,0E0,9.787345E-2,1.17367916E-1,1.056852E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,27,27,28,28,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,44,44,45,45,46,46,47,47,48,48,49,49,53,53,54,54,56,56,57,57,58,58,59,59,61,61,62,62,64,64,67,67,68,68,71,71,72,72,73,73,75,75,76,76,77,77,79,79,81,81,82,82,83,83,85,85,87,87,88,88,92,92,93,93,94,94,95,95,96,96,99,99,100,100,101,101,105,105,109,109,110,110,111,111],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,-1,-1,48,50,-1,52,54,56,58,60,-1,-1,62,64,66,68,-1,-1,-1,70,72,74,76,78,80,-1,-1,-1,82,84,-1,86,88,90,92,-1,94,96,-1,98,-1,-1,100,102,-1,-1,104,106,108,-1,110,112,114,-1,116,-1,118,120,122,-1,124,-1,126,128,-1,-1,-1,130,132,134,136,138,-1,-1,140,142,144,-1,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.202E3,1.838444E6,2.26182E5,1.3569831E7,2.01351E5,1.95684E5,4.5505118E2,3.6129813E2,4E0,1E0,2.386324E6,9.330579E0,6.610734E6,4.247868E3,2.771816E11,2.1818182E0,8.185918E0,2.148972E5,3.794679E8,1.9E1,-6.489486E-3,1.6342836E3,5.587588E-4,1.8820395E6,1.6503105E3,1.3866127E-2,2.4669853E-3,4.1822004E7,4.4543375E4,1.1369554E-2,1.066E3,2.7835715E3,3.2351852E1,3.2343243E1,2.155E3,-2.1819016E-2,5.3502154E-4,3.9E2,9E0,2E1,1.5277338E0,-4.4492744E-3,4.4662505E-4,-1.7341005E-3,5.4656273E-1,3.324E3,1.2819376E2,3.0791914E0,1.2E1,7.218466E7,-8.124084E-3,6.657419E-4,8.956492E-3,3.79E2,5.0647113E2,-1.8724587E-2,3.3012173E9,2.54E2,1.8E1,1.1132721E5,6.8604485E-3,2.419355E0,1.1905188E6,7.4496E-4,1.0258E4,-2.3084148E-3,6.671342E-3,1.7699115E-2,2.8034655E6,6.4335396E-3,1.8331748E-4,3.7057E5,1.504779E7,3.72424E5,-1.3865464E-3,2.374722E6,1.8956E4,1.408E3,-6.979978E-3,8.0886055E3,-1.6001873E-3,1.3E1,8.2E1,5.79E2,-1.14449505E-2,1E0,2.8086666E-3,1.139605E6,2.6878788E1,3.2978482E-3,9.543699E-3,-3.4131324E-3,2E1,1.24E2,2.7090813E9,4.8297736E7,5.314225E2,-5.5186666E-4,-7.582613E-3,8.315271E0,5.314225E2,7.1E1,-6.9247633E-3,-6.5789158E-3,8.168529E-4,7.55552E5,1.0854154E-2,7.0334272E-3,2.3094653E-3,3.1998687E3,1.0104842E12,1.4689211E7,-3.906247E-3,-2.3141E-3,5.4480038E-3,9.215619E-3,2.8552003E-3,-4.6929014E-3,-1.0765994E-4,-2.1420906E-3,3.5733117E-3,-3.023613E-3,5.3359934E-5,-6.4598667E-3,-1.933331E-3,1.4672253E-3,8.08868E-3,1.0490201E-3,-4.7998186E-3,-3.8770614E-3,3.8805476E-4,-8.936801E-4,-8.853229E-3,-6.2805265E-3,-1.4574773E-3,1.781261E-3,-2.8109446E-3,-1.17147835E-2,-1.6607487E-3,-9.65566E-4,-1.0135608E-2,2.0287111E-3,5.8725635E-3,-2.2890035E-3,1.3329915E-3,2.328191E-3,-1.5682942E-3,3.587842E-3,1.1387366E-2,-2.415474E-3,3.2653557E-3,-3.254178E-4,2.120924E-3],"split_indices":[2,9,28,44,28,28,55,51,3,64,1,55,44,51,30,55,53,32,7,10,0,54,0,44,51,0,0,50,4,0,8,50,49,55,0,0,0,10,3,0,52,0,0,0,26,28,57,53,17,44,0,0,0,1,4,0,5,28,3,27,0,55,47,0,28,0,0,57,46,0,0,9,44,9,0,28,0,0,0,32,0,3,28,28,0,63,0,30,55,0,0,0,3,0,30,44,4,0,0,55,4,0,0,0,0,9,0,0,0,4,30,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.192E3,1.481E3,7.11E2,1.114E3,3.67E2,2.02E2,5.09E2,8.28E2,2.86E2,3.18E2,4.9E1,1.94E2,8E0,4.9E2,1.9E1,5.79E2,2.49E2,5E0,2.81E2,3.13E2,5E0,3.1E1,1.8E1,4.1E1,1.53E2,4E0,4E0,3.95E2,9.5E1,7E0,1.2E1,3.27E2,2.52E2,1.71E2,7.8E1,2E0,3E0,2.45E2,3.6E1,2.1E1,2.92E2,2.3E1,8E0,3E0,3.8E1,8.2E1,7.1E1,3.59E2,3.6E1,9.2E1,3E0,9E0,3E0,1.15E2,2.12E2,3E0,2.49E2,1.55E2,1.6E1,7.4E1,4E0,1.15E2,1.3E2,7E0,2.9E1,3E0,1.8E1,1.18E2,1.74E2,3.3E1,5E0,1.6E1,6.6E1,5.2E1,1.9E1,4.7E1,3.12E2,1.2E1,2.4E1,7.6E1,1.6E1,4E1,7.5E1,2.09E2,3E0,2.3E2,1.9E1,1.7E1,1.38E2,9E0,7E0,2.9E1,4.5E1,2.1E1,9.4E1,1.03E2,2.7E1,4E0,2.5E1,8E0,1.1E2,1.66E2,8E0,1E1,6E0,6.3E1,3E0,2.7E1,2.5E1,1.9E1,2.8E1,3.02E2,1E1,9E0,3E0,4E0,7.2E1,1.9E1,2.1E1,5E0,7E1,7.5E1,1.34E2,3.4E1,1.96E2,8E0,9E0,1.31E2,7E0,1E1,3.5E1,5E0,1.6E1,1.7E1,7.7E1,7.9E1,2.4E1,4E0,2.3E1,6E0,2E0,8.3E1,2.7E1,3E1,1.36E2,3.3E1,3E1,1.5E1,4E0,1.3E1,1.5E1,2.61E2,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-5.983469E-4,6.2242495E-3,-4.7979463E-2,-7.186882E-3,3.6666036E-2,-1.1124597E-2,-4.1852083E-2,-2.5528986E-2,4.362337E-3,5.314415E-2,6.716853E-3,-2.6642088E-2,-1.0702642E-1,-1.0661843E-2,-5.7597954E-2,1.3200395E-2,-3.0212874E-2,4.843308E-2,2.1978377E-1,-2.0334437E-2,3.2683972E-2,-6.590908E-2,-2.3028112E-3,-7.641497E-2,-2.581846E-1,-7.673301E-3,-1.2065749E-2,-2.1393439E-1,-4.1253436E-2,5.827128E-2,3.1027333E-3,-1.474657E-2,-2.458876E-2,1.268726E-1,3.999255E-2,6.280224E-3,1.6421001E-2,2.174834E-2,-4.7835395E-2,2.4987662E-2,7.098558E-3,-7.570028E-2,4.616752E-3,5.6880023E-3,-5.1977304E-3,-4.987232E-3,2.8655555E-3,-1.51241645E-2,-9.224384E-4,-9.943503E-2,4.0507462E-4,-2.1098938E-2,-1.393024E-1,4.3154493E-2,-5.9747323E-2,1.3189435E-1,3.052175E-2,-2.7024942E-3,6.547223E-2,-1.13323135E-2,-2.0230135E-2,3.1827022E-3,8.946938E-3,6.476326E-2,9.670799E-3,6.000375E-2,-1.8913634E-3,-1.0605936E-1,2.665296E-2,1.4364413E-3,-9.315795E-3,-3.5879612E-2,-4.6201535E-3,-1.0843346E-2,4.6192452E-2,-1.2729916E-3,-1.3749377E-1,9.07002E-3,-8.655714E-2,-8.722852E-3,1.3101298E-3,1.1967221E-2,6.805372E-3,-2.8163852E-2,-1.2299976E-1,1.5102165E-3,7.3955054E-3,1.4535874E-1,1.0209716E-2,3.6949392E-2,-1.2102807E-2,8.259937E-2,-3.987041E-3,-1.0803698E-2,-9.4517745E-2,8.180635E-2,1.5480362E-2,2.8601924E-3,-9.046558E-3,8.249485E-2,-3.8175436E-3,-6.9364175E-2,-1.7061144E-1,2.7433163E-3,-2.9778087E-3,-6.170812E-2,2.3790547E-3,2.4939282E-3,-2.5642589E-2,7.544411E-3,2.6543185E-2,8.739349E-4,-7.7029127E-3,1.7007045E-3,-9.3643344E-4,-1.4629422E-3,-1.142815E-2,-1.7836747E-3,4.0990133E-3,2.3486058E-5,-4.0729227E-3,-1.0609545E-2,-4.17791E-3,9.535717E-3,1.5444661E-3,5.8651837E-3,-9.2590206E-5,-3.9444678E-3,2.2479384E-3,-1.0250078E-3,1.8240052E-3,-5.0441315E-4,5.7280487E-3,-7.193339E-4,4.925544E-3,3.069216E-3,-6.5979664E-3,-3.986413E-3,4.0301345E-3,-8.267689E-4,4.877294E-3,-4.0582847E-3,4.7703343E-4,5.744419E-3,-7.5109344E-4,-3.6861614E-4,-5.881446E-3,-1.2624671E-2,-2.4204315E-3,1.2741486E-3,-4.6725203E-3,-5.1145104E-4,-5.3314827E-3,-8.50197E-4,3.1829376E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,-1,59,61,63,-1,-1,65,67,69,-1,71,-1,73,-1,-1,-1,-1,-1,75,77,-1,79,81,83,85,87,89,91,-1,93,-1,-1,95,97,99,-1,101,103,-1,-1,105,-1,107,109,-1,111,113,115,-1,-1,117,-1,119,121,-1,-1,123,125,127,129,131,-1,133,135,137,139,-1,141,143,-1,145,147,-1,-1,149,-1,-1,151,-1,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0113117E-1,7.743222E-1,3.2048726E-1,2.791007E-1,2.8587663E-1,0E0,2.6083478E-1,2.4206385E-1,2.476112E-1,2.8296423E-1,1.4607054E-1,2.072417E-1,2.1162933E-1,2.6480016E-1,4.0256876E-1,2.933655E-1,2.66214E-1,2.3658204E-1,6.554869E-2,1.1908664E-1,9.733E-2,1.4415267E-1,1.2201715E-1,2.096752E-1,1.375429E-1,2.565251E-1,0E0,2.1729517E-1,2.3204988E-1,2.3704901E-1,1.9134942E-1,0E0,1.5529801E-1,1.2618917E-1,2.4798495E-1,0E0,0E0,1.02142826E-1,2.717212E-1,1.636135E-1,0E0,7.922661E-2,0E0,8.553921E-2,0E0,0E0,0E0,0E0,0E0,7.213315E-2,2.4115886E-1,0E0,1.22293085E-1,8.843668E-2,2.3826137E-1,9.3737006E-2,2.0144977E-1,1.8085343E-1,1.2560838E-1,0E0,1.1172193E-1,0E0,0E0,1.5160131E-1,1.5079077E-1,9.135259E-2,0E0,7.107195E-2,9.18217E-2,0E0,0E0,7.861309E-2,0E0,9.020066E-2,8.314436E-2,0E0,9.542343E-2,2.4749392E-1,2.52762E-1,0E0,0E0,9.2353605E-2,0E0,1.4539364E-1,1.1518705E-1,0E0,0E0,8.7198436E-2,1.0995088E-1,1.5089872E-1,2.1731985E-1,1.7021188E-1,0E0,8.929909E-2,1.5951267E-1,1.7317516E-1,1.5137309E-1,0E0,1.7770769E-1,1.0325955E-1,0E0,8.7901354E-2,1.3725421E-1,0E0,0E0,9.984784E-2,0E0,0E0,9.778245E-2,0E0,6.677016E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,32,32,33,33,34,34,37,37,38,38,39,39,41,41,43,43,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,63,63,64,64,65,65,67,67,68,68,71,71,73,73,74,74,76,76,77,77,78,78,81,81,83,83,84,84,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,96,96,98,98,99,99,101,101,102,102,105,105,108,108,110,110],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,-1,60,62,64,-1,-1,66,68,70,-1,72,-1,74,-1,-1,-1,-1,-1,76,78,-1,80,82,84,86,88,90,92,-1,94,-1,-1,96,98,100,-1,102,104,-1,-1,106,-1,108,110,-1,112,114,116,-1,-1,118,-1,120,122,-1,-1,124,126,128,130,132,-1,134,136,138,140,-1,142,144,-1,146,148,-1,-1,150,-1,-1,152,-1,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2343444E7,6.9664386E2,2.3446269E2,1.8122449E2,1.4540612E7,-1.1124597E-2,1.2427474E8,2.5943396E0,1.1231086E1,1.921032E4,5.841568E1,1.0323588E3,3.6600095E10,9.5992924E-5,1.141E4,1.4343751E6,1.1563158E1,1.3358E4,1.272E3,9E0,9.370472E2,1.1879E4,1.2367184E3,6.5E1,4.0950096E10,1.0882353E1,-1.2065749E-2,1.92E2,2.426232E3,1E0,1E0,-1.474657E-2,5.85E2,2.9347712E6,5.1063637E1,6.280224E-3,1.6421001E-2,9.3544586E1,2.332E3,1.921032E4,7.098558E-3,2.8636363E0,4.616752E-3,2.3292E4,-5.1977304E-3,-4.987232E-3,2.8655555E-3,-1.51241645E-2,-9.224384E-4,2.645E3,1.2471935E7,-2.1098938E-2,2.1132307E0,7.474142E-4,4.44E2,5.85E2,2.9620786E0,1.8989474E0,4.2037987E1,-1.13323135E-2,8.8867E4,3.1827022E-3,8.946938E-3,1.7682243E1,3.7129E4,1.0658642E1,-1.8913634E-3,7.6842415E6,9.02496E5,1.4364413E-3,-9.315795E-3,2.0288463E0,-4.6201535E-3,1.4E1,7.401584E6,-1.2729916E-3,5E1,3.8488917E-2,3.4653E4,-8.722852E-3,1.3101298E-3,6.03368E5,6.805372E-3,2.9E1,1.13E3,1.5102165E-3,7.3955054E-3,1.5445488E8,1.1317E4,1.6713532E9,3.2E1,2.0521326E0,-3.987041E-3,2.155E3,6.895004E3,1E0,4.0602388E6,2.8601924E-3,6.244737E1,2.011653E7,-3.8175436E-3,1.131645E6,1.1693485E0,2.7433163E-3,-2.9778087E-3,5.5544252E7,2.3790547E-3,2.4939282E-3,3.705174E7,7.544411E-3,5.57E4,8.739349E-4,-7.7029127E-3,1.7007045E-3,-9.3643344E-4,-1.4629422E-3,-1.142815E-2,-1.7836747E-3,4.0990133E-3,2.3486058E-5,-4.0729227E-3,-1.0609545E-2,-4.17791E-3,9.535717E-3,1.5444661E-3,5.8651837E-3,-9.2590206E-5,-3.9444678E-3,2.2479384E-3,-1.0250078E-3,1.8240052E-3,-5.0441315E-4,5.7280487E-3,-7.193339E-4,4.925544E-3,3.069216E-3,-6.5979664E-3,-3.986413E-3,4.0301345E-3,-8.267689E-4,4.877294E-3,-4.0582847E-3,4.7703343E-4,5.744419E-3,-7.5109344E-4,-3.6861614E-4,-5.881446E-3,-1.2624671E-2,-2.4204315E-3,1.2741486E-3,-4.6725203E-3,-5.1145104E-4,-5.3314827E-3,-8.50197E-4,3.1829376E-3],"split_indices":[44,51,51,51,44,0,44,55,53,4,55,51,5,36,9,44,53,9,0,8,55,2,57,8,5,51,0,12,46,15,89,0,2,46,57,0,0,57,0,4,0,53,0,2,0,0,0,0,0,9,44,0,52,38,0,2,53,53,57,0,11,0,0,52,9,52,0,46,11,0,0,52,0,3,1,0,28,37,9,0,0,9,0,8,2,0,0,7,9,30,3,52,0,0,32,66,47,0,57,31,0,9,41,0,0,44,0,0,49,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.167E3,1.895E3,2.72E2,1.316E3,5.79E2,7E0,2.65E2,5.08E2,8.08E2,3.73E2,2.06E2,2.16E2,4.9E1,3.48E2,1.6E2,6.44E2,1.64E2,3.64E2,9E0,1.01E2,1.05E2,8.2E1,1.34E2,4.2E1,7E0,3.45E2,3E0,1.4E1,1.46E2,1.17E2,5.27E2,2E0,1.62E2,3.4E1,3.3E2,7E0,2E0,4E1,6.1E1,1E2,5E0,7.8E1,4E0,1.26E2,8E0,3.4E1,8E0,5E0,2E0,2.7E1,3.18E2,2E0,1.2E1,2.6E1,1.2E2,3.1E1,8.6E1,4.83E2,4.4E1,2E0,1.6E2,2E1,1.4E1,1.81E2,1.49E2,2.5E1,1.5E1,3.4E1,2.7E1,9.8E1,2E0,3.2E1,4.6E1,9E1,3.6E1,1E1,1.7E1,2.9E2,2.8E1,9E0,3E0,2.1E1,5E0,8.1E1,3.9E1,8E0,2.3E1,1.2E1,7.4E1,9.2E1,3.91E2,4E1,4E0,1.43E2,1.7E1,1.34E2,4.7E1,3.8E1,1.11E2,2.2E1,3E0,2.3E1,1.1E1,2E1,7E0,2.5E1,7E0,1.6E1,7.4E1,4E0,3.2E1,3E0,1.4E1,1.48E2,1.42E2,2.2E1,6E0,1.3E1,8E0,5.6E1,2.5E1,7E0,3.2E1,7E0,5E0,6E0,6.8E1,8E0,8.4E1,3.26E2,6.5E1,1.3E1,2.7E1,1.38E2,5E0,4E0,1.3E1,5E0,1.29E2,3.5E1,1.2E1,2.1E1,9E1,1.5E1,7E0,1.2E1,1.1E1,5E0,6E0,8E0,1.7E1,6.5E1,9E0,1.6E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"155","size_leaf_vector":"1"}},{"base_weights":[1.2405826E-3,7.851492E-3,-3.726772E-2,4.0792213E-3,7.3685385E-2,-6.873137E-2,-1.0902465E-2,-4.455841E-3,2.6365586E-2,1.37055E-1,3.3202965E-2,-5.7342358E-2,-2.1054022E-1,2.8249333E-2,-4.129086E-2,-1.9328682E-2,9.666864E-3,-4.5823776E-3,4.144918E-2,3.994138E-3,3.274653E-1,1.0658234E-1,3.4652308E-3,-1.619034E-1,-4.7402397E-2,-1.5066243E-2,1.2236937E-4,1.5615763E-3,-4.5518195E-3,-9.705197E-2,-1.28204385E-2,1.0842019E-4,-6.1806392E-2,5.2912854E-2,1.0760645E-3,1.0431746E-1,-1.8254658E-2,5.009812E-2,-2.3901457E-2,1.7787393E-2,3.9602728E-3,-6.6896235E-3,6.5736203E-3,5.6219865E-3,-6.093362E-4,-1.215364E-3,-1.0235406E-2,-3.4806445E-2,-1.2048326E-1,3.95173E-3,-5.5982596E-3,2.0360129E-2,-4.6582725E-2,1.9561546E-3,-1.2461209E-2,-8.39052E-2,-9.483257E-3,6.575992E-2,-1.3530323E-3,4.763951E-3,-9.1840535E-2,1.7071602E-1,-3.7668247E-4,2.920881E-2,-4.3480005E-2,1.5605833E-2,6.401386E-2,-1.8992059E-1,-8.204545E-4,-7.184559E-2,-7.751438E-3,-6.4763147E-3,2.1523517E-3,4.5959905E-2,-2.786424E-3,-1.0848847E-1,-2.2957347E-4,8.000545E-3,-8.232756E-2,-1.0240822E-1,6.747743E-5,-2.2395771E-2,5.195917E-3,7.7949926E-2,-2.2875166E-3,-3.5961764E-3,4.907549E-2,-8.683355E-3,-1.9182595E-3,9.534791E-3,-8.66683E-4,-4.941555E-4,5.198286E-3,-7.320592E-2,3.4510918E-2,-7.726863E-2,2.8340178E-2,2.3750627E-2,1.0595224E-1,-1.4483205E-2,1.5244668E-4,4.4728573E-3,-4.13743E-2,-1.0738182E-1,-2.1042736E-2,5.243452E-2,-5.796801E-2,8.68112E-2,-7.127367E-4,-1.5669205E-3,-9.110255E-3,-6.458497E-4,1.8366056E-3,-1.3907437E-3,-1.51318945E-2,-2.1740594E-3,-6.0711545E-3,1.9475314E-3,-4.858518E-3,3.0381877E-3,-2.1620288E-3,5.4419525E-3,2.4819758E-3,-3.892357E-4,3.1483842E-3,7.925168E-4,4.2644064E-3,-2.3244238E-3,-9.219241E-3,3.3496262E-3,-3.3242097E-3,8.603549E-4,-1.5408004E-2,2.2044503E-3,-2.4826347E-3,4.665077E-3,-3.5617998E-4,5.6207017E-3,9.733926E-4,6.701815E-4,-4.663733E-3,-7.730944E-4,-6.18762E-3,7.531881E-4,-6.562128E-3,-2.1923578E-4,6.4980136E-3,-5.4143984E-3,-2.4093031E-4,5.718683E-3,-1.5056042E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,-1,-1,-1,-1,49,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,69,71,-1,-1,73,75,77,-1,79,81,83,-1,85,87,89,-1,91,93,95,97,99,101,103,105,-1,-1,107,-1,109,-1,111,113,115,117,119,-1,121,-1,123,125,-1,-1,-1,-1,-1,-1,127,129,131,133,135,137,-1,-1,-1,139,141,143,145,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8377826E-1,4.857071E-1,2.7797845E-1,3.5239664E-1,2.6737708E-1,2.3503178E-1,2.2008234E-1,2.8164682E-1,2.396617E-1,3.5234457E-1,1.4365166E-1,1.4095E-1,2.8763306E-1,6.8587676E-2,1.6386014E-1,5.3915197E-1,2.555472E-1,2.5352994E-1,1.9619614E-1,0E0,9.1232E-2,2.0805635E-1,9.973237E-2,9.793779E-2,1.1731544E-1,0E0,0E0,0E0,0E0,1.824173E-1,7.936606E-2,2.289766E-1,2.362414E-1,1.2322885E-1,1.974704E-1,1.435208E-1,1.8205965E-1,1.4539665E-1,1.5695158E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1368184E-1,7.883513E-2,0E0,0E0,7.831254E-2,8.939389E-2,2.2838E-1,0E0,2.2374678E-1,1.0204136E-1,1.442737E-1,0E0,2.0540585E-1,1.0066001E-1,9.490451E-2,0E0,1.8330652E-1,2.3204316E-1,1.07449695E-1,3.6390424E-1,1.2669581E-1,1.5356041E-1,8.4396794E-2,2.055358E-1,0E0,0E0,7.1944386E-2,0E0,8.8953346E-2,0E0,3.03675E-1,3.797465E-1,1.969403E-1,1.291384E-1,1.3257518E-1,0E0,8.135837E-2,0E0,1.7141245E-1,1.2369208E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.9486931E-1,1.2473263E-1,2.850452E-1,1.3639575E-1,2.8159395E-1,1.602242E-1,0E0,0E0,0E0,9.5014885E-2,7.172492E-2,1.0044818E-1,1.624367E-1,1.19401805E-1,8.482878E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,47,47,48,48,51,51,52,52,53,53,55,55,56,56,57,57,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,73,73,75,75,77,77,78,78,79,79,80,80,81,81,83,83,85,85,86,86,93,93,94,94,95,95,96,96,97,97,98,98,102,102,103,103,104,104,105,105,106,106,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,-1,-1,-1,-1,50,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,70,72,-1,-1,74,76,78,-1,80,82,84,-1,86,88,90,-1,92,94,96,98,100,102,104,106,-1,-1,108,-1,110,-1,112,114,116,118,120,-1,122,-1,124,126,-1,-1,-1,-1,-1,-1,128,130,132,134,136,138,-1,-1,-1,140,142,144,146,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7033327E-1,3.970405E3,1E0,7.733016E5,3.2475834E7,3.0996E4,1.84442E5,2.41E3,3.55E2,1.71E2,9.188401E4,4.36E2,1.964127E6,4.391553E6,6.624E3,1.7417E4,1.2059859E1,7.501215E6,1.6015803E7,3.994138E-3,1.694761E9,2.0724944E7,1.3264E4,2.6610387E-3,6E1,-1.5066243E-2,1.2236937E-4,1.5615763E-3,-4.5518195E-3,2.41E3,3.1152E4,2.6104508E10,1.996528E-1,2.545455E7,3.1972716E7,7.3139534E0,4.9E1,1E0,3.171E3,1.7787393E-2,3.9602728E-3,-6.6896235E-3,6.5736203E-3,5.6219865E-3,-6.093362E-4,-1.215364E-3,-1.0235406E-2,1.6681991E-3,8.253647E6,3.95173E-3,-5.5982596E-3,2.9407698E-1,8.773955E6,1.0715278E7,-1.2461209E-2,2.7E1,7.9203456E8,2.1196339E1,-1.3530323E-3,9.287868E6,2.308943E0,1.4702222E0,-3.7668247E-4,1.19E3,3.2196458E6,5.2921E4,5.8084745E6,2.75E2,1.418705E0,4.3256033E2,5.3474556E5,-6.4763147E-3,2.1523517E-3,1E0,-2.786424E-3,7.8351054E11,-2.2957347E-4,7E2,5.1212654E0,1.52179E5,1.1231086E1,2.8593832E-1,5.195917E-3,2E0,-2.2875166E-3,4.4854636E7,1.6286094E5,-8.683355E-3,-1.9182595E-3,9.534791E-3,-8.66683E-4,-4.941555E-4,5.198286E-3,2.2515285E6,1E0,1E0,1.845406E1,3.3382E4,1E0,-1.4483205E-2,1.5244668E-4,4.4728573E-3,8E0,4.781639E10,2.223036E3,1.7446084E0,1.6644816E6,1.8552577E2,-7.127367E-4,-1.5669205E-3,-9.110255E-3,-6.458497E-4,1.8366056E-3,-1.3907437E-3,-1.51318945E-2,-2.1740594E-3,-6.0711545E-3,1.9475314E-3,-4.858518E-3,3.0381877E-3,-2.1620288E-3,5.4419525E-3,2.4819758E-3,-3.892357E-4,3.1483842E-3,7.925168E-4,4.2644064E-3,-2.3244238E-3,-9.219241E-3,3.3496262E-3,-3.3242097E-3,8.603549E-4,-1.5408004E-2,2.2044503E-3,-2.4826347E-3,4.665077E-3,-3.5617998E-4,5.6207017E-3,9.733926E-4,6.701815E-4,-4.663733E-3,-7.730944E-4,-6.18762E-3,7.531881E-4,-6.562128E-3,-2.1923578E-4,6.4980136E-3,-5.4143984E-3,-2.4093031E-4,5.718683E-3,-1.5056042E-3],"split_indices":[26,51,15,27,49,9,28,2,10,8,27,2,1,27,2,28,55,5,50,0,7,44,10,38,8,0,0,0,0,2,10,19,38,44,50,57,10,109,2,0,0,0,0,0,0,0,0,40,1,0,0,37,1,44,0,3,12,53,0,1,53,52,0,28,27,1,44,0,53,51,31,0,0,108,0,30,0,2,52,1,53,41,0,8,0,9,27,0,0,0,0,0,0,27,79,8,55,9,79,0,0,0,8,30,4,52,31,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.291E3,1.956E3,3.35E2,1.851E3,1.05E2,1.52E2,1.83E2,1.339E3,5.12E2,4E1,6.5E1,1.42E2,1E1,8E1,1.03E2,6.52E2,6.87E2,1.68E2,3.44E2,3.3E1,7E0,1.8E1,4.7E1,1.1E1,1.31E2,6E0,4E0,7.7E1,3E0,3.4E1,6.9E1,4.48E2,2.04E2,1.13E2,5.74E2,1.8E1,1.5E2,3.04E2,4E1,5E0,2E0,2E0,1.6E1,5E0,4.2E1,4E0,7E0,1.13E2,1.8E1,4E0,3E1,3.5E1,3.4E1,4.46E2,2E0,1.43E2,6.1E1,9.8E1,1.5E1,5.53E2,2.1E1,1.1E1,7E0,5.2E1,9.8E1,8.8E1,2.16E2,4E0,3.6E1,4.7E1,6.6E1,1.6E1,2E0,2.7E1,8E0,1.3E1,2.1E1,4.17E2,2.9E1,1.17E2,2.6E1,5.6E1,5E0,8.9E1,9E0,4.66E2,8.7E1,6E0,1.5E1,9E0,2E0,3.6E1,1.6E1,7.1E1,2.7E1,1E1,7.8E1,1.11E2,1.05E2,2E0,2E0,1E1,2.6E1,2.7E1,2E1,3E1,3.6E1,1.6E1,1.1E1,8E0,5E0,2.48E2,1.69E2,2.5E1,4E0,4.4E1,7.3E1,1.9E1,7E0,1.2E1,4.4E1,3E1,5.9E1,4.37E2,2.9E1,5.2E1,3.5E1,6.2E1,9E0,2E1,7E0,8E0,2E0,6.3E1,1.5E1,3.1E1,8E1,8.6E1,1.9E1,1.4E1,1.2E1,7E0,2E1,1.6E1,4E0,1.9E1,1.1E1,1.6E1,2E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-6.1806507E-4,-9.26298E-3,2.4482142E-2,1.2340171E-2,-2.2086779E-2,3.5315227E-2,-2.487254E-2,2.2204192E-2,-3.0825729E-2,-5.78102E-2,-1.441488E-2,2.754785E-2,1.0413969E-1,2.32925E-3,-3.9499838E-2,9.514172E-3,5.9208516E-2,-4.0274505E-2,8.380309E-3,-3.6749914E-2,-1.3462895E-1,1.1471169E-2,-2.8500028E-2,6.3413635E-2,1.2187589E-2,7.907745E-2,1.1871678E-2,-2.795676E-2,-1.2558587E-1,1.703037E-2,-1.24749854E-1,7.0061594E-2,-6.7413417E-3,-1.3084336E-3,-8.138113E-2,-2.980326E-2,-8.724903E-3,-1.6808186E-1,-2.0117151E-4,3.6959954E-2,-1.7508868E-2,-5.7153136E-2,-8.730721E-3,7.286159E-2,-7.86606E-4,1.6111413E-2,-9.093978E-2,4.313155E-3,-4.4617015E-3,-6.435014E-2,-3.870269E-3,-2.5894712E-3,-1.2047304E-2,2.3666825E-2,-1.16700515E-1,-4.179458E-2,-2.6299557E-2,5.7806257E-2,8.45873E-3,-1.2815861E-1,5.915576E-2,2.724764E-2,-1.0225235E-1,-1.5331909E-1,-1.8720115E-2,-6.014035E-3,-8.1570335E-2,-9.003563E-3,-3.2021867E-3,1.3310779E-3,6.724967E-2,3.0331139E-2,-4.226048E-2,-2.4556556E-1,-4.6371263E-2,7.533207E-3,-3.711984E-2,3.3620156E-2,9.8335765E-2,1.3514932E-2,1.1614624E-2,-1.1943043E-3,-1.1513499E-2,1.4693994E-3,-4.2529223E-3,-2.0148684E-3,3.2223742E-2,-8.549527E-3,4.795619E-2,-2.950151E-1,1.7032599E-3,-4.495436E-5,-6.686068E-3,1.04918145E-1,3.737344E-2,1.6279125E-3,-9.099893E-3,-2.342604E-3,6.109052E-3,5.219049E-2,-1.8147372E-3,-1.1514652E-2,-1.412317E-3,-7.4327335E-2,-2.2051707E-1,5.2425726E-3,-5.0255336E-2,-5.078289E-3,1.0292033E-2,-1.1756834E-1,-1.872675E-4,-1.0996632E-2,5.5531636E-3,-6.8841287E-4,8.194052E-2,-1.2736836E-2,3.2681392E-3,2.1642616E-3,-5.7305392E-2,-6.0823973E-4,-3.214498E-1,-3.7936766E-2,-1.6475679E-1,1.988539E-2,-4.7136243E-2,-5.114441E-2,2.7677994E-2,2.9399488E-3,-4.259926E-2,1.8267244E-1,3.3991828E-3,8.406086E-3,1.0457317E-1,6.554375E-3,2.3451932E-4,7.35298E-3,-5.993843E-4,6.10609E-3,1.2968319E-3,-1.2196762E-3,-2.0394655E-2,3.4959011E-3,9.184039E-3,-1.2940656E-3,3.0318121E-3,1.0764576E-3,7.1709645E-3,-6.0620205E-3,7.916608E-4,-2.821202E-3,-1.1850301E-2,-1.3093817E-3,-8.921887E-3,1.9566163E-3,-1.5207435E-3,-6.0045402E-3,2.8145493E-3,3.7640755E-3,-1.284021E-3,3.0619132E-3,9.473277E-3,4.318905E-3,-1.7147951E-3,-3.8284592E-3,1.632293E-4,-1.9203786E-2,-6.9921566E-3,3.4719922E-3,-1.977536E-3,-2.5927406E-3,-1.1955249E-2,1.1982921E-3,-3.5703117E-3,-1.9349322E-4,-4.38727E-3,-2.7440041E-3,2.9016188E-3,-1.8385252E-3,3.1510538E-3,3.1881677E-3,-4.294175E-3,-9.459274E-5,9.672334E-3,-3.8647421E-3,5.417745E-4,9.151143E-3,1.6109747E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,-1,67,-1,69,71,73,75,77,-1,79,81,-1,-1,83,85,-1,-1,87,89,91,-1,93,-1,95,97,99,101,103,105,107,109,-1,-1,111,113,115,117,119,121,123,125,127,129,131,-1,-1,-1,-1,-1,-1,133,135,137,139,-1,-1,-1,141,143,-1,-1,-1,-1,145,-1,-1,-1,147,149,-1,151,-1,153,155,-1,157,-1,-1,159,161,-1,-1,163,-1,165,167,169,171,173,175,177,-1,179,181,-1,183,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8194268E-1,4.579335E-1,3.051019E-1,2.629707E-1,2.8337353E-1,2.467826E-1,1.1701746E-1,2.3516868E-1,2.4208415E-1,2.9111427E-1,3.1206745E-1,2.3115936E-1,1.654697E-1,0E0,8.294074E-2,3.8055766E-1,9.2089504E-2,1.7768927E-1,0E0,1.5135318E-1,1.6910154E-1,2.2372422E-1,3.1331548E-1,9.743935E-2,1.21242635E-1,1.3017446E-1,0E0,6.812489E-2,8.087693E-2,3.1871158E-1,7.3251987E-1,1.4661968E-1,1.5960121E-1,1.6997471E-1,2.4054599E-1,1.7191924E-1,0E0,7.8846335E-2,0E0,1.7365988E-1,1.6942647E-1,4.4771785E-1,1.5218505E-1,1.0965365E-1,0E0,1.7507423E-1,1.0930909E-1,0E0,0E0,9.1839984E-2,7.18324E-2,0E0,0E0,2.66465E-1,4.7890157E-1,7.761893E-2,0E0,9.3304336E-2,0E0,9.090371E-2,1.1623189E-1,7.934575E-2,1.3254999E-1,1.1631173E-1,1.3307992E-1,1.6905698E-1,1.2216237E-1,0E0,0E0,1.1352406E-1,1.066322E-1,8.924675E-2,1.3011011E-1,2.070623E-1,2.0843202E-1,1.427476E-1,1.1046697E-1,1.1256561E-1,1.1965233E-1,1.3123848E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.838655E-2,1.200035E-1,3.2176253E-1,3.2125783E-1,0E0,0E0,0E0,6.572613E-2,1.424448E-1,0E0,0E0,0E0,0E0,9.883117E-2,0E0,0E0,0E0,7.344139E-2,6.848371E-2,0E0,7.260142E-2,0E0,1.2426134E-1,8.944139E-2,0E0,1.1514786E-1,0E0,0E0,1.206522E-1,7.157168E-2,0E0,0E0,1.3478315E-1,0E0,8.888543E-2,1.3631493E-1,1.3432053E-1,1.1388082E-1,8.278091E-2,1.1185753E-1,6.5805845E-2,0E0,8.935152E-2,8.05496E-2,0E0,9.116539E-2,9.376307E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,42,42,43,43,45,45,46,46,49,49,50,50,53,53,54,54,55,55,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,86,86,87,87,88,88,89,89,93,93,94,94,99,99,103,103,104,104,106,106,108,108,109,109,111,111,114,114,115,115,118,118,120,120,121,121,122,122,123,123,124,124,125,125,126,126,128,128,129,129,131,131,132,132],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,-1,68,-1,70,72,74,76,78,-1,80,82,-1,-1,84,86,-1,-1,88,90,92,-1,94,-1,96,98,100,102,104,106,108,110,-1,-1,112,114,116,118,120,122,124,126,128,130,132,-1,-1,-1,-1,-1,-1,134,136,138,140,-1,-1,-1,142,144,-1,-1,-1,-1,146,-1,-1,-1,148,150,-1,152,-1,154,156,-1,158,-1,-1,160,162,-1,-1,164,-1,166,168,170,172,174,176,178,-1,180,182,-1,184,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,8E0,6.747114E7,1E0,9E2,8.791777E3,8.24666E5,6.0607E4,1.03992E3,2.642E4,6.6911E4,8E-3,1.71E2,2.32925E-3,9.60094E5,4.597929E6,1.6904226E7,2.08395E5,8.380309E-3,9.697451E-1,2.3033286E5,2.1230708E7,1.9E1,2.0568174E7,4.255079E7,1E0,1.1871678E-2,9.757E3,3.6600095E10,6.203E3,2.545E3,4.8279068E1,1.3715873E3,3.6022064E1,7.549744E7,1.1E1,-8.724903E-3,8.3304755E8,-2.0117151E-4,2.47E2,1.8544E4,2.4576474E8,2.9206951E1,9.414044E2,-7.86606E-4,3.7561736E7,4.8916136E7,4.313155E-3,-4.4617015E-3,9.5E2,1.3664E4,-2.5894712E-3,-1.2047304E-2,5.52E2,5.339161E0,7E0,-2.6299557E-2,1.0258198E-2,8.45873E-3,1.2939234E5,2.979178E0,1.8955729E1,4.4195585E0,5.4E1,1.3E1,1.91E2,9.159699E7,-9.003563E-3,-3.2021867E-3,9.750871E6,4.745907E7,1.4930757E0,2.25E2,7E0,6.4625664E8,4.5843E7,7.8726E-1,7.07089E5,7.557E3,3.832274E2,1.1614624E-2,-1.1943043E-3,-1.1513499E-2,1.4693994E-3,-4.2529223E-3,-2.0148684E-3,6.932264E8,8.974133E9,1.9E1,2E0,1.7032599E-3,-4.495436E-5,-6.686068E-3,4.88E2,1.954E3,1.6279125E-3,-9.099893E-3,-2.342604E-3,6.109052E-3,1.803E3,-1.8147372E-3,-1.1514652E-2,-1.412317E-3,2.6757258E-1,4.91271E5,5.2425726E-3,1.4E1,-5.078289E-3,1E0,9.9341E4,-1.872675E-4,3.1140776E0,5.5531636E-3,-6.8841287E-4,3.75163E5,1.8579784E2,3.2681392E-3,2.1642616E-3,1.16E2,-6.0823973E-4,4.7E2,7.1047956E-1,6.798644E-1,1.11696E5,6.5029144E5,7.733016E5,1.2511433E3,2.9399488E-3,2.983871E0,1.4225797E3,3.3991828E-3,1.0989011E-2,8.837831E-2,6.554375E-3,2.3451932E-4,7.35298E-3,-5.993843E-4,6.10609E-3,1.2968319E-3,-1.2196762E-3,-2.0394655E-2,3.4959011E-3,9.184039E-3,-1.2940656E-3,3.0318121E-3,1.0764576E-3,7.1709645E-3,-6.0620205E-3,7.916608E-4,-2.821202E-3,-1.1850301E-2,-1.3093817E-3,-8.921887E-3,1.9566163E-3,-1.5207435E-3,-6.0045402E-3,2.8145493E-3,3.7640755E-3,-1.284021E-3,3.0619132E-3,9.473277E-3,4.318905E-3,-1.7147951E-3,-3.8284592E-3,1.632293E-4,-1.9203786E-2,-6.9921566E-3,3.4719922E-3,-1.977536E-3,-2.5927406E-3,-1.1955249E-2,1.1982921E-3,-3.5703117E-3,-1.9349322E-4,-4.38727E-3,-2.7440041E-3,2.9016188E-3,-1.8385252E-3,3.1510538E-3,3.1881677E-3,-4.294175E-3,-9.459274E-5,9.672334E-3,-3.8647421E-3,5.417745E-4,9.151143E-3,1.6109747E-3],"split_indices":[2,16,44,6,2,51,28,1,45,28,28,56,8,0,9,46,46,9,0,26,32,12,3,46,42,112,0,9,5,28,28,57,51,55,7,8,0,5,0,0,9,5,57,51,0,42,47,0,0,0,2,0,0,2,57,8,0,37,0,32,41,57,57,0,3,2,7,0,0,9,7,41,10,3,7,44,38,28,2,55,0,0,0,0,0,0,7,18,10,8,0,0,0,4,2,0,0,0,0,2,0,0,0,37,30,0,8,0,15,1,0,53,0,0,1,54,0,0,8,0,0,55,26,10,27,27,4,0,52,4,0,56,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.219E3,1.651E3,5.68E2,6.15E2,1.036E3,4.66E2,1.02E2,5.01E2,1.14E2,1.82E2,8.54E2,4.2E2,4.6E1,1.6E1,8.6E1,3.74E2,1.27E2,1.1E2,4E0,1.44E2,3.8E1,3.01E2,5.53E2,1.25E2,2.95E2,4.1E1,5E0,7.7E1,9E0,3.55E2,1.9E1,1.09E2,1.8E1,5.7E1,5.3E1,1.39E2,5E0,3E1,8E0,1.6E2,1.41E2,2.25E2,3.28E2,1.12E2,1.3E1,2.85E2,1E1,3.8E1,3E0,3E1,4.7E1,7E0,2E0,3.39E2,1.6E1,1.7E1,2E0,1E2,9E0,6E0,1.2E1,4.5E1,1.2E1,2.4E1,2.9E1,9.6E1,4.3E1,2.2E1,8E0,7.4E1,8.6E1,4.8E1,9.3E1,1.1E1,2.14E2,2.09E2,1.19E2,4.5E1,6.7E1,2.83E2,2E0,8E0,2E0,7E0,2.3E1,2.2E1,2.5E1,1.46E2,1.93E2,7E0,9E0,1.3E1,4E0,2.9E1,7.1E1,2E0,4E0,5E0,7E0,3.3E1,1.2E1,3E0,9E0,1.2E1,1.2E1,5E0,2.4E1,1.2E1,8.4E1,2.9E1,1.4E1,6.8E1,6E0,1.3E1,7.3E1,2.4E1,2.4E1,1.3E1,8E1,3E0,8E0,2.01E2,1.3E1,1.71E2,3.8E1,9.8E1,2.1E1,3.2E1,1.3E1,1.3E1,5.4E1,2.69E2,1.4E1,4E0,2.1E1,3E0,1.43E2,3.4E1,1.59E2,3E0,4E0,2.4E1,5E0,2.2E1,4.9E1,2.7E1,6E0,7E0,5E0,3E0,9E0,2.2E1,2E0,4.8E1,3.6E1,2.7E1,2E0,1E1,5.8E1,6.7E1,6E0,4E0,2E1,5.5E1,2.5E1,4E0,4E0,9E0,1.92E2,7E0,6E0,1.61E2,1E1,2.1E1,1.7E1,9.1E1,7E0,8E0,1.3E1,4E0,9E0,2E0,1.1E1,9E0,2.6E2,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[-2.6461885E-3,5.3340443E-3,-2.9143486E-2,-8.741717E-3,2.0043992E-2,-7.551606E-2,-1.4362561E-2,1.149244E-2,-2.7613558E-2,3.531612E-2,-5.023137E-3,-6.4150564E-2,-1.1604518E-2,-5.76093E-2,-4.4209473E-3,1.4843461E-2,-1.8929577E-1,-9.478924E-3,-2.5112122E-2,3.3149924E-2,1.3850504E-2,-2.6134552E-2,4.4262322E-4,-5.3875137E-2,-9.932792E-3,-6.457724E-2,5.727232E-3,-4.5244113E-2,7.027223E-3,2.636092E-2,-4.149294E-2,1.2383362E-3,-1.2741236E-2,-1.01461895E-2,-5.558582E-2,7.6249555E-2,2.0737965E-2,-3.3169277E-3,7.53072E-3,-3.663931E-2,-1.6289087E-1,-6.878391E-4,-9.108632E-2,9.4261614E-4,-6.851295E-2,4.4179358E-2,-8.84458E-3,3.0125476E-2,-7.2553656E-3,2.3012904E-2,-6.067669E-2,-6.7616515E-2,2.9777302E-3,-1.4460176E-1,-2.955876E-2,1.0528145E-1,1.8245974E-3,3.1603314E-2,-4.296404E-2,-6.1086277E-3,1.9938927E-3,-9.660027E-2,-1.8011134E-2,-1.1380956E-2,-1.170298E-3,-5.3217687E-2,-6.23217E-3,-4.856257E-3,-1.4246376E-3,2.2372916E-2,4.8566926E-3,4.6160785E-3,-1.6849404E-2,3.984268E-2,-4.3749623E-2,-2.058964E-3,4.026969E-3,-3.39406E-3,9.368649E-4,-5.0780527E-2,-1.5761012E-2,9.135929E-3,-6.301663E-3,-1.7008118E-1,7.555593E-3,-5.0022963E-2,3.5812978E-2,1.1573071E-1,-1.7691279E-3,7.02221E-2,2.0574948E-2,4.1796683E-4,-1.3785355E-1,2.6636766E-2,-2.0010153E-2,-1.354782E-1,5.259735E-4,-7.277294E-2,3.05117E-2,-5.3356397E-3,-3.7418128E-4,-5.269012E-3,3.7952695E-2,-5.2243937E-2,4.4131945E-3,4.0616635E-3,1.3486956E-3,1.4073189E-3,-5.4391213E-3,-6.2305206E-3,-1.3742818E-3,2.0719124E-3,-8.3628716E-4,-8.315773E-3,8.165883E-4,-1.4122E-3,-6.6701327E-3,3.0837357E-3,-5.72212E-3,7.709993E-3,3.4471038E-3,-1.4749433E-4,4.812872E-3,1.0335499E-3,-8.460618E-3,1.05331485E-4,-7.740666E-3,3.0200353E-3,-3.410796E-4,-3.5764184E-3,6.043825E-5,-2.1808934E-3,-9.404857E-3,-1.308793E-3,-7.6892036E-3,1.0758337E-4,4.4507743E-3,3.8524766E-3,-1.0722354E-4,-2.6965924E-3,5.049204E-3,-1.0182625E-3,1.3911157E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,35,-1,-1,37,39,-1,41,-1,43,45,47,49,-1,-1,51,53,55,57,59,-1,61,63,-1,65,-1,67,69,71,73,-1,75,77,79,81,83,85,87,-1,89,91,-1,93,95,97,-1,-1,99,-1,-1,-1,101,-1,-1,103,105,107,-1,-1,-1,-1,109,-1,-1,111,113,-1,115,117,119,-1,121,123,-1,125,127,129,131,-1,133,135,-1,-1,-1,137,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.695847E-1,3.5361585E-1,3.5122624E-1,3.3367747E-1,3.1983915E-1,2.383647E-1,1.6785276E-1,2.8675064E-1,2.0024285E-1,2.9409415E-1,9.917284E-1,1.764119E-1,0E0,1.0181899E-1,1.4940275E-1,2.7101627E-1,1.4741525E-1,0E0,2.0329812E-1,2.7445328E-1,0E0,0E0,1.9701321E-1,2.044771E-1,0E0,9.1564775E-2,0E0,1.09033346E-1,1.478086E-1,2.479014E-1,8.92563E-2,0E0,0E0,2.2722638E-1,3.3629873E-1,1.1577648E-1,2.7959996E-1,2.1624507E-1,0E0,1.0772252E-1,1.7540199E-1,0E0,7.3794365E-2,0E0,7.127392E-2,1.0078928E-1,1.5782323E-1,2.4581537E-1,0E0,8.081473E-2,6.550616E-2,2.4679008E-1,4.5511144E-1,2.9028964E-1,1.5563624E-1,9.955335E-2,0E0,1.4506552E-1,2.934909E-1,0E0,1.6212139E-1,9.834148E-2,2.0416747E-1,0E0,0E0,7.747479E-2,0E0,0E0,0E0,1.2763672E-1,0E0,0E0,1.2440622E-1,1.4823312E-1,2.3239273E-1,0E0,0E0,0E0,0E0,9.160365E-2,0E0,0E0,1.5038443E-1,8.259767E-2,0E0,1.566487E-1,1.5530321E-1,1.1449522E-1,0E0,2.018534E-1,1.3004076E-1,0E0,9.789702E-2,1.9355653E-1,1.9903354E-1,9.723428E-2,0E0,1.4836723E-1,7.7972546E-2,0E0,0E0,0E0,9.9039376E-2,8.0944896E-2,7.431738E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,18,18,19,19,22,22,23,23,25,25,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,39,39,40,40,42,42,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,60,60,61,61,62,62,65,65,69,69,72,72,73,73,74,74,79,79,82,82,83,83,85,85,86,86,87,87,89,89,90,90,92,92,93,93,94,94,95,95,97,97,98,98,102,102,103,103,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,36,-1,-1,38,40,-1,42,-1,44,46,48,50,-1,-1,52,54,56,58,60,-1,62,64,-1,66,-1,68,70,72,74,-1,76,78,80,82,84,86,88,-1,90,92,-1,94,96,98,-1,-1,100,-1,-1,-1,102,-1,-1,104,106,108,-1,-1,-1,-1,110,-1,-1,112,114,-1,116,118,120,-1,122,124,-1,126,128,130,132,-1,134,136,-1,-1,-1,138,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0304577E-1,2.437E3,2.1631205E0,2E0,1.3152658E7,4.5723195E6,1.6681991E-3,5.8E1,3.983889E3,4.42384E5,5.1776E4,6.5E1,-1.1604518E-2,1E0,3.451532E8,1E0,2.7E1,-9.478924E-3,2.2415E4,3.7129E4,1.3850504E-2,-2.6134552E-2,2.6037296E-7,2.0354E4,-9.932792E-3,1E0,5.727232E-3,4.3360384E7,1.4540612E7,2.6204643E0,1.7810288E5,1.2383362E-3,-1.2741236E-2,3.3151623E1,6.4E1,3.1530054E0,2.6186154E0,8.588109E0,7.53072E-3,7.22E2,4.222002E8,-6.878391E-4,9.2199E4,9.4261614E-4,4.5402945E6,1.5683041E3,5.348301E6,1E0,-7.2553656E-3,1E0,1.75E2,1.21E2,1.2E1,1.307E3,1.477425E6,1E0,1.8245974E-3,1.2224265E1,4.4648915E6,-6.1086277E-3,3.8338925E8,9.115134E5,1.3841E4,-1.1380956E-2,-1.170298E-3,4.6547272E2,-6.23217E-3,-4.856257E-3,-1.4246376E-3,2.5757682E-1,4.8566926E-3,4.6160785E-3,3.502483E-2,2.0606E4,5.4674416E7,-2.058964E-3,4.026969E-3,-3.39406E-3,9.368649E-4,6E0,-1.5761012E-2,9.135929E-3,4.8E1,4.4993465E1,7.555593E-3,1.7291142E7,1.945711E1,2.1351435E0,-1.7691279E-3,1.16E0,1E0,4.1796683E-4,1.63E3,4.4E1,3.3743455E0,7.73E3,5.259735E-4,2.0469315E0,1.544E3,-5.3356397E-3,-3.7418128E-4,-5.269012E-3,7.88E2,9.5732903E-1,7.315484E9,4.0616635E-3,1.3486956E-3,1.4073189E-3,-5.4391213E-3,-6.2305206E-3,-1.3742818E-3,2.0719124E-3,-8.3628716E-4,-8.315773E-3,8.165883E-4,-1.4122E-3,-6.6701327E-3,3.0837357E-3,-5.72212E-3,7.709993E-3,3.4471038E-3,-1.4749433E-4,4.812872E-3,1.0335499E-3,-8.460618E-3,1.05331485E-4,-7.740666E-3,3.0200353E-3,-3.410796E-4,-3.5764184E-3,6.043825E-5,-2.1808934E-3,-9.404857E-3,-1.308793E-3,-7.6892036E-3,1.0758337E-4,4.4507743E-3,3.8524766E-3,-1.0722354E-4,-2.6965924E-3,5.049204E-3,-1.0182625E-3,1.3911157E-3],"split_indices":[26,2,53,16,44,27,40,3,27,2,1,8,0,113,5,6,0,0,28,9,0,0,36,9,0,8,0,5,44,37,46,0,0,54,11,52,41,55,0,2,7,0,12,0,31,4,12,79,0,8,6,2,12,0,9,67,0,55,44,0,7,31,9,0,0,51,0,0,0,26,0,0,37,9,7,0,0,0,0,51,0,0,10,55,0,44,55,40,0,52,90,0,10,3,53,28,0,55,2,0,0,0,0,26,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.219E3,1.706E3,5.13E2,8.72E2,8.34E2,1.23E2,3.9E2,4.21E2,4.51E2,5.18E2,3.16E2,1.17E2,6E0,7.2E1,3.18E2,4.15E2,6E0,5E0,4.46E2,5.15E2,3E0,2E0,3.14E2,1.11E2,6E0,7E1,2E0,6.9E1,2.49E2,3.45E2,7E1,2E0,4E0,3E2,1.46E2,1.14E2,4.01E2,3.08E2,6E0,9.7E1,1.4E1,2.5E1,4.5E1,1.8E1,5.1E1,7.4E1,1.75E2,3.39E2,6E0,1.6E1,5.4E1,5.5E1,2.45E2,3.2E1,1.14E2,6.2E1,5.2E1,3.43E2,5.8E1,1.1E1,2.97E2,2.2E1,7.5E1,8E0,6E0,2.6E1,1.9E1,2.4E1,2.7E1,5.6E1,1.8E1,1.1E1,1.64E2,3E2,3.9E1,8E0,8E0,4.6E1,8E0,5.3E1,2E0,1E1,2.35E2,3E1,2E0,8.7E1,2.7E1,5.8E1,4E0,7.5E1,2.68E2,3.8E1,2E1,1.4E2,1.57E2,1.6E1,6E0,3.5E1,4E1,1E1,1.6E1,5E0,5.1E1,6.1E1,1.03E2,4.9E1,2.51E2,2E1,1.9E1,9E0,4.4E1,4.4E1,1.91E2,2.8E1,2E0,7.4E1,1.3E1,2.3E1,4E0,2.3E1,3.5E1,2.5E1,5E1,2.66E2,2E0,4E0,1.6E1,6.4E1,7.6E1,4.1E1,1.16E2,8E0,8E0,2.5E1,1E1,2.9E1,1.1E1,2.3E1,2.8E1,5.9E1,2E0,5.1E1,5.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[-7.509865E-4,-6.585007E-3,3.173534E-2,9.988615E-3,-2.201297E-2,4.1956633E-2,-6.7316964E-2,2.5831072E-3,5.200677E-2,-1.7538367E-1,-1.9758081E-2,9.004033E-2,2.3140622E-2,-1.403479E-1,6.5582595E-4,-1.012405E-2,2.5118805E-2,4.0821426E-2,1.2790291E-1,-5.0713043E-3,-1.676957E-2,-9.4196305E-4,-3.141362E-2,1.1337362E-1,-8.660307E-4,3.164636E-3,7.3505975E-2,-9.534026E-3,-3.174756E-3,1.713025E-2,-4.2376395E-2,-9.01992E-3,2.910653E-2,5.111861E-2,-4.0794272E-2,6.82877E-3,-1.3300561E-3,-1.1769104E-2,7.000608E-2,-1.0825748E-1,-2.5966993E-2,8.6363E-3,6.980031E-2,1.8442E-2,-4.0196575E-2,1.674722E-1,4.5725312E-2,1.938912E-3,6.477785E-2,-2.564852E-2,-9.817889E-2,1.7938276E-1,2.3347624E-2,4.3822885E-2,7.543823E-3,-5.4416927E-3,7.5481215E-4,-1.7964917E-1,-8.544263E-3,9.270714E-2,-2.7798787E-2,-8.919095E-2,-1.3913839E-2,-1.01376474E-1,-2.1776004E-2,1.067183E-3,4.959246E-3,-4.4843974E-3,2.6669646E-2,-8.262256E-2,1.5352195E-3,4.191088E-3,1.5011955E-2,6.4809926E-2,-1.050302E-1,1.6991211E-2,-1.0234389E-1,1.0168559E-1,-5.460746E-2,-3.3400107E-2,1.1143538E-2,-1.3355863E-1,-2.9700488E-2,-5.8134596E-5,1.0173508E-2,1.0602316E-1,1.50860585E-2,-1.07879976E-4,5.8070574E-2,-1.1559555E-2,-8.5411454E-4,-1.1862556E-2,5.4642414E-3,1.9132172E-3,5.9789824E-3,-6.1639645E-3,1.7243212E-3,3.849444E-3,-1.0157549E-1,1.1358969E-3,-1.8005504E-1,5.5704033E-3,-3.887413E-2,1.1962825E-1,1.738561E-2,-5.3315493E-3,-5.0296466E-4,4.7754258E-2,8.679217E-3,-1.1401391E-2,1.4183447E-3,1.7743948E-3,-1.036356E-3,-1.017448E-2,-1.1015588E-3,-1.429137E-4,6.841616E-3,-3.9431094E-3,5.2156025E-3,-1.0348374E-3,-9.14121E-3,-2.7676714E-3,-8.807682E-3,-3.1779134E-3,3.1570462E-3,6.911061E-3,-2.857431E-4,1.539805E-3,-7.400004E-4,-1.211466E-3,3.277704E-3,-1.9083773E-3,-5.7751393E-5,-8.071214E-3,-3.086276E-3,-4.1666906E-3,-1.2612356E-2,1.3764704E-3,-1.7841986E-3,-3.852914E-5,-2.2410282E-3,8.027352E-3,-5.077595E-4,1.406785E-3,-1.3112471E-3,4.1609304E-3,4.167519E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,41,-1,43,45,-1,-1,47,49,-1,51,53,55,-1,-1,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,85,87,-1,-1,-1,89,91,93,95,97,-1,99,101,-1,-1,-1,103,105,-1,-1,-1,107,109,111,113,115,117,119,-1,121,123,-1,-1,125,127,-1,129,-1,-1,131,-1,-1,-1,-1,-1,-1,133,-1,135,137,139,141,143,-1,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2076692E-1,4.8144203E-1,3.450301E-1,2.8240418E-1,3.3372274E-1,2.756726E-1,1.9320513E-1,2.2161418E-1,1.1023253E-1,1.3152617E-1,2.1094856E-1,2.2123414E-1,2.2312102E-1,6.908375E-2,0E0,4.358384E-1,2.5540355E-1,1.0293475E-1,6.5738976E-2,0E0,0E0,2.842766E-1,2.4580145E-1,2.2608507E-1,0E0,1.067325E-1,1.5680027E-1,0E0,0E0,1.9458489E-1,2.093767E-1,0E0,2.3380417E-1,8.286804E-2,6.588395E-2,0E0,0E0,1.7225009E-1,1.1137059E-1,1.2061527E-1,1.7351079E-1,0E0,8.375925E-2,1.1767714E-1,1.3485077E-1,1.4139E-1,1.5066254E-1,3.235266E-1,2.9195854E-1,3.7901062E-1,1.2201613E-1,8.2179785E-2,1.8008636E-1,6.7287475E-2,0E0,0E0,0E0,6.8187475E-2,1.3694724E-1,7.383549E-2,7.6571554E-2,8.846736E-2,0E0,2.93072E-1,2.4709475E-1,0E0,0E0,0E0,9.4579645E-2,6.65409E-2,0E0,0E0,0E0,8.823751E-2,1.2799448E-1,1.6027415E-1,2.4277517E-1,2.5959653E-1,9.829837E-2,3.0095413E-1,0E0,1.382798E-1,8.1393875E-2,0E0,0E0,1.2667036E-1,1.4509836E-1,0E0,9.6731365E-2,0E0,0E0,9.9033475E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.752657E-2,0E0,1.3046241E-1,2.290581E-1,1.3125834E-1,7.745643E-2,6.6240266E-2,0E0,0E0,6.8935886E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,25,25,26,26,29,29,30,30,32,32,33,33,34,34,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,57,57,58,58,59,59,60,60,61,61,63,63,64,64,68,68,69,69,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,82,82,85,85,86,86,88,88,91,91,98,98,100,100,101,101,102,102,103,103,104,104,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,42,-1,44,46,-1,-1,48,50,-1,52,54,56,-1,-1,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,86,88,-1,-1,-1,90,92,94,96,98,-1,100,102,-1,-1,-1,104,106,-1,-1,-1,108,110,112,114,116,118,120,-1,122,124,-1,-1,126,128,-1,130,-1,-1,132,-1,-1,-1,-1,-1,-1,134,-1,136,138,140,142,144,-1,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,4.839905E1,2.3580047E10,3.2196458E6,5E0,1.4497429E7,2.8980975E5,1E0,4.94374E1,2.2515285E6,7.2488395E6,5.5544252E7,2.1325744E7,3.4409692E3,6.5582595E-4,9.658451E0,3.5E2,1.4952381E1,3.142868E5,-5.0713043E-3,-1.676957E-2,1.8E9,2.8729197E8,7.492813E0,-8.660307E-4,2.5189628E5,1.251806E6,-9.534026E-3,-3.174756E-3,4.0448148E2,3.1254683E0,-9.01992E-3,5.116325E3,1.2909952E1,1.86E2,6.82877E-3,-1.3300561E-3,3.5476844E7,1.4E2,6.4860556E5,9.417789E1,8.6363E-3,2.609323E3,5.794E3,1.3758811E0,1.27734E5,1.3205668E7,3.363849E2,1E0,2.7E1,4.87514E6,1.3797468E0,6.647279E5,5.3379045E6,7.543823E-3,-5.4416927E-3,7.5481215E-4,4.7E2,3.1998687E3,5.751617E6,1.91E2,6.9121815E-2,-1.3913839E-2,5.84E2,1.7291142E7,1.067183E-3,4.959246E-3,-4.4843974E-3,1.5277338E0,1.571654E7,1.5352195E-3,4.191088E-3,1.5011955E-2,2.4507338E4,3.555E3,4E0,1.763213E-5,3.9739098E4,4.423676E1,3.171E3,1.1143538E-2,4.57091E5,3.523E3,-5.8134596E-5,1.0173508E-2,1.5445488E8,1.0824E4,-1.07879976E-4,1.0412E4,-1.1559555E-2,-8.5411454E-4,1.364172E6,5.4642414E-3,1.9132172E-3,5.9789824E-3,-6.1639645E-3,1.7243212E-3,3.849444E-3,8.61E2,1.1358969E-3,6.0599957E4,6.129333E-2,1.0317661E5,8.05953E8,8.8E1,-5.3315493E-3,-5.0296466E-4,3.8538412E7,8.679217E-3,-1.1401391E-2,1.4183447E-3,1.7743948E-3,-1.036356E-3,-1.017448E-2,-1.1015588E-3,-1.429137E-4,6.841616E-3,-3.9431094E-3,5.2156025E-3,-1.0348374E-3,-9.14121E-3,-2.7676714E-3,-8.807682E-3,-3.1779134E-3,3.1570462E-3,6.911061E-3,-2.857431E-4,1.539805E-3,-7.400004E-4,-1.211466E-3,3.277704E-3,-1.9083773E-3,-5.7751393E-5,-8.071214E-3,-3.086276E-3,-4.1666906E-3,-1.2612356E-2,1.3764704E-3,-1.7841986E-3,-3.852914E-5,-2.2410282E-3,8.027352E-3,-5.077595E-4,1.406785E-3,-1.3112471E-3,4.1609304E-3,4.167519E-4],"split_indices":[51,47,5,27,3,49,27,107,57,27,44,44,1,51,0,57,2,55,32,0,0,5,5,53,0,42,28,0,0,4,52,0,32,55,0,0,0,7,8,32,51,0,51,2,33,11,9,4,15,3,12,52,44,44,0,0,0,0,4,44,8,57,0,0,44,0,0,0,52,1,0,0,0,4,0,57,37,27,55,2,0,12,10,0,0,7,10,0,12,0,0,44,0,0,0,0,0,0,2,0,31,37,27,7,3,0,0,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.218E3,1.881E3,3.37E2,9.07E2,9.74E2,3.06E2,3.1E1,7.72E2,1.35E2,1.3E1,9.61E2,8.5E1,2.21E2,1.6E1,1.5E1,4.94E2,2.78E2,1.19E2,1.6E1,1.1E1,2E0,3.68E2,5.93E2,7E1,1.5E1,1.59E2,6.2E1,7E0,9E0,2.68E2,2.26E2,4E0,2.74E2,1.06E2,1.3E1,1.4E1,2E0,3.2E2,4.8E1,3.8E1,5.55E2,2.4E1,4.6E1,1.18E2,4.1E1,1.3E1,4.9E1,2.04E2,6.4E1,1.75E2,5.1E1,9E0,2.65E2,1.01E2,5E0,5E0,8E0,5E0,3.15E2,3.9E1,9E0,3.6E1,2E0,2.8E1,5.27E2,2.2E1,2.4E1,7E0,1.11E2,2.6E1,1.5E1,1E1,3E0,4.4E1,5E0,1.79E2,2.5E1,4.9E1,1.5E1,1.71E2,4E0,3.3E1,1.8E1,2E0,7E0,2.3E1,2.42E2,2.4E1,7.7E1,3E0,2E0,3.08E2,7E0,1.8E1,2.1E1,3E0,6E0,2E0,3.4E1,1.1E1,1.7E1,2.03E2,3.24E2,9E0,1.02E2,1.7E1,9E0,4E1,4E0,2E0,3E0,1.15E2,6.4E1,9E0,1.6E1,1.6E1,3.3E1,1.3E1,2E0,1.62E2,9E0,1.6E1,1.7E1,1.3E1,5E0,1.6E1,7E0,1.51E2,9.1E1,1.1E1,6.6E1,7.9E1,2.29E2,9E0,2.5E1,1E1,7E0,1.31E2,7.2E1,7.1E1,2.53E2,6E0,3E0,7.9E1,2.3E1,1.8E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-1.0919131E-4,-8.1789885E-3,2.4649268E-2,-3.0117766E-3,-6.783182E-2,6.414383E-2,1.0935965E-2,-1.4838298E-2,1.2683944E-2,-2.4279706E-1,-5.7653956E-2,7.091927E-2,-3.3829128E-3,1.05540805E-1,6.005828E-3,-9.294692E-3,-7.3142685E-2,2.1212718E-2,-4.0971152E-2,-2.9857382E-3,-1.6250929E-2,-1.1415423E-1,-4.125734E-2,1.2116837E-2,9.239653E-2,1.418679E-1,-3.406167E-3,-2.3122698E-2,1.9554324E-2,6.2003177E-2,-1.3415656E-2,-8.5468575E-2,1.2727979E-3,3.6822364E-2,-4.612497E-3,-3.2666374E-2,-1.2390993E-2,-7.779289E-2,-1.2036432E-2,-5.4660894E-2,1.0340577E-3,-3.0936021E-3,2.6503243E-3,6.5769386E-3,5.9352834E-2,8.263916E-3,1.6704433E-3,-4.9628075E-3,-1.4498246E-2,2.9242232E-2,-2.8243326E-2,4.675366E-3,2.2717118E-3,-5.5171862E-2,-3.7994802E-3,-3.2445227E-3,-8.8522285E-3,3.0158134E-2,1.1691024E-1,-3.6326654E-3,3.0885756E-3,8.8889373E-4,-5.227921E-2,-9.238565E-3,-1.1032132E-3,-7.413115E-3,-4.5569487E-2,2.2074383E-2,1.1012014E-1,1.9688653E-4,-6.031979E-2,1.1364527E-2,2.3973178E-2,-1.3855888E-2,-7.7984645E-3,-3.4414604E-3,2.4571517E-3,-4.7261633E-2,-1.837784E-2,-1.4663034E-3,-5.994955E-3,5.886391E-3,2.565282E-2,-1.4787274E-3,6.2010665E-3,-1.8399034E-2,4.1480657E-2,-6.5668154E-4,-4.340236E-3,2.643215E-3,-2.4920157E-3,5.3690765E-3,3.2877196E-3,7.1830037E-3,1.3360769E-3,-1.1216839E-2,-6.1119813E-3,2.8237069E-2,-7.7233754E-2,2.0018304E-3,-4.7755644E-2,7.653854E-4,-3.3505764E-3,9.667835E-5,-3.578855E-3,1.2451815E-3,-8.022316E-3,1.0056025E-4,-6.879576E-3,-1.3054169E-3,2.6827855E-3,-5.6924783E-3,9.4941555E-4,-6.8154726E-3,5.1484077E-4,4.905023E-3,8.878703E-4,6.670711E-4,-6.9610896E-3,-3.300004E-3,2.370625E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,-1,-1,37,39,41,43,45,-1,47,49,51,53,55,-1,57,59,61,-1,63,-1,65,-1,-1,-1,-1,67,-1,-1,-1,69,71,73,-1,75,77,79,-1,-1,81,83,-1,85,-1,87,-1,-1,-1,89,91,93,-1,95,-1,97,99,-1,-1,-1,101,-1,103,-1,-1,105,-1,-1,107,109,-1,-1,-1,-1,-1,111,-1,-1,113,-1,115,117,-1,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4214982E-1,5.142463E-1,2.9392934E-1,2.85495E-1,2.2284454E-1,1.37021E-1,1.8826412E-1,2.8278106E-1,3.0335072E-1,1.20114386E-1,1.1294478E-1,1.6811109E-1,0E0,1.4201432E-1,1.5278617E-1,2.3647217E-1,9.7256124E-2,2.3035276E-1,1.6693275E-1,0E0,0E0,1.337702E-1,8.7161705E-2,1.4218055E-1,1.625964E-1,6.582984E-2,0E0,9.016486E-2,1.2307354E-1,1.0821791E-1,3.0455545E-1,8.929813E-2,0E0,1.864799E-1,1.2604474E-1,9.231841E-2,0E0,1.569274E-1,0E0,7.484752E-2,0E0,0E0,0E0,0E0,1.14905804E-1,0E0,0E0,0E0,9.78491E-2,2.510617E-1,9.0848625E-2,0E0,8.147381E-2,3.748E-1,1.8525715E-1,0E0,0E0,1.4513052E-1,8.547434E-2,0E0,1.6327916E-1,0E0,1.0563044E-1,0E0,0E0,0E0,8.002767E-2,6.669934E-2,9.70422E-2,0E0,1.19497284E-1,0E0,9.5392466E-2,8.479364E-2,0E0,0E0,0E0,2.4151355E-1,0E0,1.7006224E-1,0E0,0E0,1.2386221E-1,0E0,0E0,3.5008746E-1,9.04524E-2,0E0,0E0,0E0,0E0,0E0,7.564338E-2,0E0,0E0,6.780034E-2,0E0,1.4159843E-1,6.560278E-2,0E0,7.175824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,37,37,39,39,44,44,48,48,49,49,50,50,52,52,53,53,54,54,57,57,58,58,60,60,62,62,66,66,67,67,68,68,70,70,72,72,73,73,77,77,79,79,82,82,85,85,86,86,92,92,95,95,97,97,98,98,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,-1,-1,38,40,42,44,46,-1,48,50,52,54,56,-1,58,60,62,-1,64,-1,66,-1,-1,-1,-1,68,-1,-1,-1,70,72,74,-1,76,78,80,-1,-1,82,84,-1,86,-1,88,-1,-1,-1,90,92,94,-1,96,-1,98,100,-1,-1,-1,102,-1,104,-1,-1,106,-1,-1,108,110,-1,-1,-1,-1,-1,112,-1,-1,114,-1,116,118,-1,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,4.2343444E7,1.858736E-3,3.2485715E2,3E0,1.8222668E2,7.019E3,3.1126543E1,8.0752426E1,2.2515285E6,1E0,9.883509E4,-3.3829128E-3,4.8767575E9,1.2204E4,6E0,1.9554291E0,4.942935E6,3.2627738E5,-2.9857382E-3,-1.6250929E-2,1.568E3,1.7994144E0,7.431E3,8.959817E7,1E0,-3.406167E-3,1.6181818E1,2.3580047E10,2.090909E0,1E0,1.502494E0,1.2727979E-3,9.923929E2,4.256737E-1,1.07E2,-1.2390993E-2,1.28E2,-1.2036432E-2,5.23188E2,1.0340577E-3,-3.0936021E-3,2.6503243E-3,6.5769386E-3,1.4421636E3,8.263916E-3,1.6704433E-3,-4.9628075E-3,1.03E2,5.473125E3,1.0008265E1,4.675366E-3,5.088781E0,1.9483356E1,1E0,-3.2445227E-3,-8.8522285E-3,9.489796E-1,5.9E1,-3.6326654E-3,1.2287474E3,8.8889373E-4,2.5291866E1,-9.238565E-3,-1.1032132E-3,-7.413115E-3,9E0,4.5478998E8,8.238194E1,1.9688653E-4,1E0,1.1364527E-2,1E0,6.896236E-2,-7.7984645E-3,-3.4414604E-3,2.4571517E-3,1.62283E0,-1.837784E-2,6.2222223E0,-5.994955E-3,5.886391E-3,9.262083E2,-1.4787274E-3,6.2010665E-3,2.1298597E0,1.176E3,-6.5668154E-4,-4.340236E-3,2.643215E-3,-2.4920157E-3,5.3690765E-3,2.4513194E5,7.1830037E-3,1.3360769E-3,2.1944155E3,-6.1119813E-3,7.9203456E8,7.401584E6,2.0018304E-3,8.791777E3,7.653854E-4,-3.3505764E-3,9.667835E-5,-3.578855E-3,1.2451815E-3,-8.022316E-3,1.0056025E-4,-6.879576E-3,-1.3054169E-3,2.6827855E-3,-5.6924783E-3,9.4941555E-4,-6.8154726E-3,5.1484077E-4,4.905023E-3,8.878703E-4,6.670711E-4,-6.9610896E-3,-3.300004E-3,2.370625E-3],"split_indices":[2,44,56,51,3,57,2,57,53,27,66,27,0,12,2,3,40,49,27,0,0,0,56,2,12,108,0,57,5,55,63,40,0,51,41,3,0,10,0,4,0,0,0,0,51,0,0,0,3,31,53,0,57,53,82,0,0,52,28,0,54,0,57,0,0,0,3,7,57,0,14,0,112,37,0,0,0,53,0,45,0,0,51,0,0,37,10,0,0,0,0,0,27,0,0,4,0,12,1,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.211E3,1.668E3,5.43E2,1.536E3,1.32E2,1.39E2,4.04E2,8.76E2,6.6E2,6E0,1.26E2,1.33E2,6E0,1.9E1,3.85E2,8.01E2,7.5E1,5.7E2,9E1,3E0,3E0,2.7E1,9.9E1,3.6E1,9.7E1,1.6E1,3E0,1.22E2,2.63E2,4.3E1,7.58E2,6.7E1,8E0,3.55E2,2.15E2,8.8E1,2E0,2.3E1,4E0,8.2E1,1.7E1,1.3E1,2.3E1,3.6E1,6.1E1,1.1E1,5E0,1E1,1.12E2,2.19E2,4.4E1,2.5E1,1.8E1,1.41E2,6.17E2,6.1E1,6E0,3.29E2,2.6E1,1.9E1,1.96E2,2.4E1,6.4E1,6E0,1.7E1,5E0,7.7E1,3.6E1,2.5E1,8E1,3.2E1,4E0,2.15E2,4.1E1,3E0,7E0,1.1E1,1.39E2,2E0,6.07E2,1E1,1.3E1,3.16E2,3E0,2.3E1,1.26E2,7E1,3.5E1,2.9E1,6E0,7.1E1,5E0,3.1E1,1.5E1,1E1,2E1,1.2E1,2.07E2,8E0,1.5E1,2.6E1,4.1E1,9.8E1,5.81E2,2.6E1,3.14E2,2E0,1.1E2,1.6E1,1.4E1,5.6E1,3E0,2.8E1,2E0,1.8E1,1.9E1,1.88E2,4E0,4E0,2.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[-9.667238E-4,-3.8980942E-3,7.215456E-2,6.8815437E-4,-3.934884E-2,1.4886375E-1,3.320698E-2,-1.486098E-2,1.1412045E-2,-1.0237732E-2,-3.193347E-2,4.8692403E-3,1.2786277E-2,-5.0182804E-2,2.9058575E-3,-5.7574045E-3,-5.8389105E-2,9.514727E-3,6.3040387E-3,-2.750609E-2,-8.788295E-3,3.3579307E-4,-5.672787E-3,1.0619909E-2,-3.227831E-2,-3.8443893E-2,-1.5037341E-1,6.080031E-2,6.1836545E-3,-5.582367E-2,-4.76923E-3,5.0926437E-3,9.941887E-3,-1.2473218E-1,-2.3306448E-2,-5.2071285E-2,7.605012E-2,-1.9388698E-1,4.744158E-3,6.6554146E-3,1.7305564E-2,3.8256932E-3,6.2336344E-2,-6.1810724E-2,4.745394E-3,-9.853698E-3,6.9740717E-3,9.231826E-3,-1.6655944E-1,1.6063516E-3,-3.4229165E-1,3.7163824E-2,-3.77835E-2,-7.13496E-2,2.687402E-3,7.620463E-3,-2.0043622E-3,-2.7911758E-1,-4.4720937E-3,3.7773654E-3,-1.5496232E-3,1.24901265E-1,2.6995286E-3,9.234749E-3,3.2227594E-2,3.0763634E-3,-6.8055324E-2,1.7320004E-3,-2.363945E-2,6.6073687E-4,-3.8879262E-3,-1.5336275E-4,-1.3420084E-2,-1.7964046E-2,-4.5745787E-3,-3.948045E-5,4.590952E-3,-5.353698E-3,-1.0522264E-3,-8.80348E-3,-1.9898734E-3,-1.9928183E-2,-8.322876E-3,8.884518E-3,-6.0353806E-4,1.5761153E-4,-6.747273E-3,2.861974E-4,4.8155705E-3,-1.9176764E-3,-5.07789E-3,-3.6334582E-3,-4.4023627E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,-1,23,25,27,-1,29,-1,-1,-1,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,-1,-1,59,61,63,65,-1,67,-1,69,71,-1,73,75,77,79,-1,-1,-1,81,-1,-1,-1,83,85,-1,87,-1,89,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6712586E-1,3.4064466E-1,2.460182E-1,3.0965242E-1,3.2518873E-1,1.241194E-1,1.5046084E-1,3.000301E-1,2.6522022E-1,0E0,1.6124447E-1,0E0,0E0,7.015622E-2,0E0,2.7310905E-1,2.3455772E-1,1.8481025E-1,0E0,1.4588834E-1,0E0,0E0,0E0,4.4819814E-1,1.9616896E-1,1.7402779E-1,2.756855E-1,2.4491458E-1,1.3468309E-1,1.0361579E-1,1.040664E-1,2.7152744E-1,0E0,7.4411726E-1,1.9399905E-1,2.1475703E-1,1.4067543E-1,1.3455218E-1,0E0,0E0,1.562063E-1,1.3315049E-1,1.6827331E-1,8.4034175E-2,0E0,8.314468E-2,0E0,1.9097362E-1,1.897017E-1,0E0,7.822728E-2,1.05155654E-1,2.0554289E-1,2.7475622E-1,0E0,0E0,0E0,7.253814E-2,0E0,0E0,0E0,9.136662E-2,1.15696155E-1,0E0,6.654924E-2,0E0,1.01787716E-1,0E0,7.564005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,15,15,16,16,17,17,19,19,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,40,40,41,41,42,42,43,43,45,45,47,47,48,48,50,50,51,51,52,52,53,53,57,57,61,61,62,62,64,64,66,66,68,68],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,-1,24,26,28,-1,30,-1,-1,-1,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,-1,-1,60,62,64,66,-1,68,-1,70,72,-1,74,76,78,80,-1,-1,-1,82,-1,-1,-1,84,86,-1,88,-1,90,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7652373E3,4.2343444E7,3.526947E7,2.7604443E2,3.0608664E2,1.3E2,5.673198E5,1.18827E5,8.581219E6,-1.0237732E-2,2.6572757E10,4.8692403E-3,1.2786277E-2,4.7658E4,2.9058575E-3,2.1969697E0,2.1584728E6,1.1331109E6,6.3040387E-3,1.067536E3,-8.788295E-3,3.3579307E-4,-5.672787E-3,1.6122807E1,1E0,9.289174E2,1.477425E6,4.930435E2,1.8552577E2,1.4622E4,1.2013854E8,6.6315195E6,9.941887E-3,4.3E2,2.772925E3,1.16544716E-1,3.426E3,1.5153E4,4.744158E-3,6.6554146E-3,3.7307863E0,1.08504E5,2.6277744E8,1.0198864E0,4.745394E-3,1.3E1,6.9740717E-3,1.1304221E0,5.8084745E6,1.6063516E-3,1.67E2,4.64109E2,3.6610576E7,1.6286094E5,2.687402E-3,7.620463E-3,-2.0043622E-3,1.7232166E8,-4.4720937E-3,3.7773654E-3,-1.5496232E-3,2.6944466E7,1E0,9.234749E-3,4.5505118E2,3.0763634E-3,7.8099336E11,1.7320004E-3,2.28125E0,6.6073687E-4,-3.8879262E-3,-1.5336275E-4,-1.3420084E-2,-1.7964046E-2,-4.5745787E-3,-3.948045E-5,4.590952E-3,-5.353698E-3,-1.0522264E-3,-8.80348E-3,-1.9898734E-3,-1.9928183E-2,-8.322876E-3,8.884518E-3,-6.0353806E-4,1.5761153E-4,-6.747273E-3,2.861974E-4,4.8155705E-3,-1.9176764E-3,-5.07789E-3,-3.6334582E-3,-4.4023627E-4],"split_indices":[51,44,49,51,51,8,31,28,27,0,12,0,0,10,0,52,50,44,0,51,0,0,0,55,63,54,9,4,55,2,43,27,0,28,49,38,2,10,0,0,52,7,12,55,0,3,0,37,44,0,0,32,7,27,0,0,0,7,0,0,0,44,90,0,55,0,30,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.177E3,2.094E3,8.3E1,1.855E3,2.39E2,2.7E1,5.6E1,7.57E2,1.098E3,8E0,2.31E2,2.2E1,5E0,1.5E1,4.1E1,6.27E2,1.3E2,1.083E3,1.5E1,2.26E2,5E0,9E0,6E0,3.88E2,2.39E2,1.08E2,2.2E1,6.5E1,1.018E3,1E2,1.26E2,3.79E2,9E0,2E1,2.19E2,9.7E1,1.1E1,1.9E1,3E0,2.1E1,4.4E1,9.78E2,4E1,9.7E1,3E0,1.23E2,3E0,3.71E2,8E0,1.2E1,8E0,4.2E1,1.77E2,8.3E1,1.4E1,6E0,5E0,9E0,1E1,1.9E1,2.5E1,8E0,9.7E2,6E0,3.4E1,4E0,9.3E1,2.7E1,9.6E1,3.52E2,1.9E1,4E0,4E0,6E0,2E0,2.7E1,1.5E1,2.6E1,1.51E2,1.4E1,6.9E1,2E0,7E0,5E0,3E0,9.66E2,4E0,2.6E1,8E0,6E1,3.3E1,1.8E1,7.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[6.756343E-4,-1.7435513E-2,1.1795097E-2,-2.0926513E-2,1.2726544E-1,-4.455398E-2,1.6018663E-2,-1.7721999E-1,-1.8398507E-2,2.2885911E-1,1.8319982E-3,1.9987945E-2,-6.073949E-2,2.851508E-2,-5.2379523E-3,-1.4753313E-2,-3.3931409E-3,6.499801E-3,-1.9985236E-2,1.2493942E-2,1.9315358E-3,-5.0568283E-3,3.2293089E-3,-6.984907E-3,-8.644872E-2,1.7480996E-2,5.61926E-2,-1.2077801E-2,7.136235E-2,-2.2522477E-2,7.5810954E-2,-2.844618E-3,2.870598E-3,-4.807346E-3,-5.350254E-4,2.4869319E-2,-3.8188502E-2,7.548973E-2,4.1479296E-3,-3.6374066E-2,1.5358307E-4,9.693866E-2,-8.5463844E-2,-2.4192903E-2,1.0935477E-1,4.5423596E-3,-2.4212466E-3,3.8363226E-2,-1.9790186E-2,-6.9821216E-2,4.1221067E-2,1.2428743E-1,5.862471E-2,3.925051E-3,-1.3943332E-3,4.009554E-2,-4.7440853E-2,3.5219442E-2,-9.6901385E-3,6.377495E-2,1.0914349E-2,2.5896328E-3,-1.1191415E-2,-2.1341339E-2,-1.0562876E-1,-7.919952E-5,8.317537E-3,4.5907706E-2,-2.3937497E-2,-8.168893E-2,1.1406566E-2,-9.313092E-2,2.7744954E-3,-2.8985916E-4,5.136622E-3,1.3312166E-2,1.6323632E-1,-4.3880655E-3,6.999166E-2,4.553746E-3,-2.5193205E-3,-1.3509335E-1,-3.279954E-2,2.527884E-3,-9.3117036E-4,-3.1634096E-2,1.176984E-2,5.889017E-3,1.24214E-3,-1.1142745E-3,2.395514E-3,-1.0442031E-3,-7.453838E-3,1.7579062E-3,6.558906E-3,2.1439989E-3,-2.4023554E-3,-8.567798E-4,-7.1238494E-3,2.302183E-3,-9.23666E-4,-3.1314702E-3,-7.846556E-3,-6.1968407E-3,3.557744E-3,8.688255E-3,2.5728513E-3,2.3630874E-3,5.353761E-3,6.502844E-4,-8.0959415E-3,-7.550549E-3,-1.0664498E-3,3.3432868E-4,-2.134216E-3,-6.222353E-5,4.002055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,-1,-1,29,-1,-1,-1,-1,31,33,35,37,39,41,43,45,-1,-1,-1,-1,47,49,51,53,55,57,59,61,63,65,-1,-1,67,69,71,73,75,77,-1,-1,79,81,83,85,87,-1,-1,-1,89,91,-1,-1,93,95,97,99,101,-1,-1,-1,103,105,-1,107,-1,-1,109,111,-1,-1,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4627616E-1,4.2850548E-1,3.2747942E-1,3.223108E-1,1.6710728E-1,1.014999E-1,3.3962646E-1,1.7321175E-1,2.115483E-1,6.716168E-2,0E0,1.4317861E-1,1.0580927E-1,2.4486148E-1,2.4920683E-1,0E0,0E0,0E0,1.9736922E-1,0E0,0E0,0E0,0E0,1.0619039E-1,7.533237E-2,2.382723E-1,2.2967452E-1,1.2957473E-1,1.6471519E-1,1.7557415E-1,7.1938984E-2,0E0,0E0,0E0,0E0,3.0810112E-1,1.7369002E-1,1.3015044E-1,1.8440479E-1,1.2545797E-1,1.00788236E-1,1.4714712E-1,1.5257612E-1,1.7794818E-1,8.293947E-2,0E0,0E0,1.8534929E-1,2.3072553E-1,1.5545058E-1,6.952831E-2,1.7936623E-1,2.2956413E-1,0E0,0E0,1.1562748E-1,1.5983745E-1,7.475445E-2,1.0774138E-1,6.630024E-2,0E0,0E0,0E0,1.8367732E-1,1.22878015E-1,0E0,0E0,2.3958987E-1,9.179965E-2,1.8697536E-1,1.0140781E-1,6.8713784E-2,0E0,0E0,0E0,1.2838876E-1,8.365905E-2,0E0,9.517425E-2,0E0,0E0,1.2020603E-1,1.3354214E-1,0E0,0E0,6.871554E-2,1.16956264E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,18,18,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,59,59,63,63,64,64,67,67,68,68,69,69,70,70,71,71,75,75,76,76,78,78,81,81,82,82,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,-1,-1,30,-1,-1,-1,-1,32,34,36,38,40,42,44,46,-1,-1,-1,-1,48,50,52,54,56,58,60,62,64,66,-1,-1,68,70,72,74,76,78,-1,-1,80,82,84,86,88,-1,-1,-1,90,92,-1,-1,94,96,98,100,102,-1,-1,-1,104,106,-1,108,-1,-1,110,112,-1,-1,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.832274E2,1.05E2,3E0,4.8083666E-1,3.1E1,6.015825E0,3.83E3,8.974133E9,2.1325744E7,1.8319982E-3,6.9121815E-2,5.6656016E3,6.9664386E2,2.6312E4,-1.4753313E-2,-3.3931409E-3,6.499801E-3,5.13657E6,1.2493942E-2,1.9315358E-3,-5.0568283E-3,3.2293089E-3,6.0875E1,1.1537302E0,1E0,2.4489644E0,1.2633843E3,1.96E2,1.7971E4,8.282879E0,-2.844618E-3,2.870598E-3,-4.807346E-3,-5.350254E-4,1.9203481E5,9.195004E6,8.959817E7,2.5737E4,4.531E3,1.7331858E1,2.62797E5,2.14099E5,5.0363636E1,2.1E1,4.5423596E-3,-2.4212466E-3,5.2187E4,2.9686664E7,1.2152382E5,5.500988E2,1.954E3,1.0407268E10,3.925051E-3,-1.3943332E-3,1.259E3,2.3E1,1.774775E11,4.1891047E5,1.3062E4,1.0914349E-2,2.5896328E-3,-1.1191415E-2,7E0,4.278699E0,-7.919952E-5,8.317537E-3,1.9719212E1,2.9375E0,7.1214952E0,1E1,4.949877E6,2.7744954E-3,-2.8985916E-4,5.136622E-3,2.02E4,3.4780384E5,-4.3880655E-3,2.043739E0,4.553746E-3,-2.5193205E-3,1.4858E4,5.5E1,2.527884E-3,-9.3117036E-4,2.1111E4,3.78E2,5.889017E-3,1.24214E-3,-1.1142745E-3,2.395514E-3,-1.0442031E-3,-7.453838E-3,1.7579062E-3,6.558906E-3,2.1439989E-3,-2.4023554E-3,-8.567798E-4,-7.1238494E-3,2.302183E-3,-9.23666E-4,-3.1314702E-3,-7.846556E-3,-6.1968407E-3,3.557744E-3,8.688255E-3,2.5728513E-3,2.3630874E-3,5.353761E-3,6.502844E-4,-8.0959415E-3,-7.550549E-3,-1.0664498E-3,3.3432868E-4,-2.134216E-3,-6.222353E-5,4.002055E-3],"split_indices":[109,55,51,3,41,10,52,2,18,1,0,57,46,51,2,0,0,0,9,0,0,0,0,51,40,79,41,4,8,6,52,0,0,0,0,32,44,12,9,28,57,2,2,45,3,0,0,28,12,32,4,2,30,0,0,11,3,30,46,9,0,0,0,6,53,0,0,55,53,55,3,9,0,0,0,12,32,0,41,0,0,9,0,0,0,9,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.214E3,8.42E2,1.372E3,8.23E2,1.9E1,9.5E1,1.277E3,1.2E1,8.11E2,8E0,1.1E1,1.9E1,7.6E1,8.04E2,4.73E2,4E0,8E0,7E0,8.04E2,6E0,2E0,5E0,1.4E1,2.5E1,5.1E1,5.76E2,2.28E2,4.35E2,3.8E1,7.84E2,2E1,1.4E1,1.1E1,4E1,1.1E1,5.09E2,6.7E1,1.66E2,6.2E1,1.45E2,2.9E2,3.3E1,5E0,7.75E2,9E0,1.7E1,3E0,3.91E2,1.18E2,4.8E1,1.9E1,4.1E1,1.25E2,1.8E1,4.4E1,1.8E1,1.27E2,6.3E1,2.27E2,2.8E1,5E0,3E0,2E0,7.5E2,2.5E1,4E0,5E0,3.49E2,4.2E1,3.9E1,7.9E1,4.1E1,7E0,1.2E1,7E0,1.1E1,3E1,8E0,1.17E2,1.1E1,7E0,1.7E1,1.1E2,4.6E1,1.7E1,1.12E2,1.15E2,9E0,1.9E1,7.19E2,3.1E1,1.1E1,1.4E1,3.27E2,2.2E1,1.2E1,3E1,2.2E1,1.7E1,3.5E1,4.4E1,3.3E1,8E0,3E0,8E0,2.3E1,7E0,8.7E1,3E1,4E0,1.3E1,6E0,1.04E2,3.2E1,8E1,9.9E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-4.6061465E-4,3.1461872E-3,-5.3271208E-2,-7.108056E-3,2.0892352E-2,-1.5692157E-1,-3.36613E-2,1.7852472E-2,-1.3699072E-2,3.8983587E-2,7.8039505E-3,-2.1381336E-1,-1.3869343E-3,-2.2947364E-2,-1.4814846E-1,-4.596008E-3,5.4730035E-2,-3.9693564E-3,-3.314693E-2,-1.2214052E-3,5.408774E-2,-1.6571568E-2,2.8067231E-2,-1.4355667E-2,-5.9068627E-3,-5.6544226E-2,1.1819123E-3,-2.8485423E-3,-1.1783216E-2,7.624282E-3,-2.5300992E-1,1.1699581E-1,2.096095E-2,-7.0039206E-3,1.1604016E-1,1.2439269E-2,-4.551816E-2,4.834881E-2,-4.2504475E-2,9.481845E-2,2.553793E-2,2.7197134E-2,-3.390195E-2,-8.996306E-3,3.1791955E-2,6.999664E-4,-3.865413E-3,4.826884E-3,-9.3811E-3,-2.9447386E-2,2.7836854E-2,-5.077108E-3,-1.656313E-2,8.934659E-4,1.3979213E-1,-3.4316098E-3,3.895012E-2,3.52823E-4,-5.9125826E-2,1.9048094E-3,8.78288E-3,2.7491437E-2,-5.2145426E-3,-7.735745E-2,-2.3814453E-2,-8.9589483E-4,4.50799E-3,-1.0038756E-2,-2.1171514E-2,7.808992E-2,8.368083E-3,-3.148695E-2,4.485492E-2,-6.1165616E-2,2.476233E-3,-3.9520584E-2,4.571628E-3,4.1007828E-2,-3.0474553E-2,-1.189959E-3,2.4334511E-3,1.3521218E-3,-5.275379E-2,8.133706E-3,2.1670729E-2,9.503576E-3,6.0352124E-2,4.07148E-3,3.434088E-4,-1.1150375E-2,3.0484801E-2,-1.686133E-2,-4.2721596E-2,6.044405E-3,4.900997E-3,-1.0178145E-1,-9.648196E-3,2.6196502E-2,-3.7303273E-2,-9.550366E-2,6.4999186E-3,7.610678E-3,6.304441E-2,1.4676438E-2,-1.3354129E-1,4.6535498E-3,2.571205E-2,3.0893937E-4,-5.891491E-3,-8.186023E-2,-2.3199527E-2,3.0956013E-2,1.2808543E-1,4.8196707E-2,-3.4477734E-3,5.848226E-3,-2.946408E-3,4.4475594E-3,5.605608E-4,5.741018E-3,-4.433103E-3,5.6876196E-4,-1.6811836E-3,1.6515437E-3,-7.525631E-3,-5.9444224E-5,-4.468828E-3,1.0956415E-3,-5.4019247E-3,-6.6321036E-3,-2.1964053E-3,3.835905E-4,-5.8653257E-3,1.973992E-3,-3.936616E-3,-9.409063E-4,-3.5741841E-3,1.2589415E-3,-6.3246335E-3,5.300012E-3,-5.789282E-4,9.056214E-3,2.3488363E-3,-2.686175E-3,2.7170787E-3,9.513244E-5,-1.1179359E-2,3.4369305E-3,-2.512081E-4,3.2779176E-4,-5.361728E-3,-7.8712933E-4,-7.2362684E-3,-1.15161856E-4,2.40461E-3,9.669505E-3,2.6732346E-3,4.1899476E-3,-5.2382704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,-1,-1,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,-1,-1,-1,79,81,83,-1,-1,-1,85,-1,87,89,91,-1,-1,93,-1,95,97,-1,-1,-1,99,101,-1,103,105,107,-1,109,-1,111,113,-1,-1,-1,115,-1,117,-1,119,-1,-1,121,123,-1,125,127,-1,129,131,133,135,137,139,-1,141,143,145,-1,147,-1,-1,149,151,153,155,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1712254E-1,3.7319416E-1,2.7927822E-1,2.1411438E-1,1.7752159E-1,1.5151364E-1,1.4256677E-1,2.2542545E-1,1.9457804E-1,1.913538E-1,2.1631165E-1,8.773565E-2,0E0,8.924028E-2,7.918209E-2,5.179963E-1,2.1412149E-1,2.5100204E-1,1.9400355E-1,1.8009277E-1,2.6387203E-1,1.5186915E-1,2.0819487E-1,0E0,0E0,9.682499E-2,7.339011E-2,0E0,0E0,1.2295786E-1,8.6129665E-2,7.706034E-2,1.21312216E-1,2.5731415E-1,8.986072E-2,1.4531374E-1,1.8473601E-1,1.415767E-1,1.7883298E-1,1.2711143E-1,1.51223E-1,1.4184046E-1,1.1236121E-1,0E0,1.36676E-1,0E0,0E0,0E0,6.543632E-2,8.175277E-2,9.533666E-2,0E0,0E0,0E0,1.488722E-1,0E0,9.2058636E-2,2.0449564E-1,4.1066748E-1,0E0,0E0,1.1539473E-1,0E0,1.7927462E-1,1.10260606E-1,0E0,0E0,0E0,9.112335E-2,9.909958E-2,0E0,1.6685101E-1,1.1047274E-1,6.653167E-2,0E0,9.3462735E-2,0E0,1.7634428E-1,1.1507277E-1,0E0,0E0,0E0,1.0258739E-1,0E0,7.204096E-2,0E0,1.7018129E-1,0E0,0E0,2.635587E-1,1.9825579E-1,0E0,1.8533057E-1,1.2579328E-1,0E0,1.8340808E-1,6.628034E-2,7.271601E-2,8.641881E-2,6.9278754E-2,7.186919E-2,0E0,9.820303E-2,8.744464E-2,1.69698E-1,0E0,1.2283048E-1,0E0,0E0,1.2467104E-1,7.7448756E-2,1.3819051E-1,1.11338645E-1,9.7934164E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,48,48,49,49,50,50,54,54,56,56,57,57,58,58,61,61,63,63,64,64,68,68,69,69,71,71,72,72,73,73,75,75,77,77,78,78,82,82,84,84,86,86,89,89,90,90,92,92,93,93,95,95,96,96,97,97,98,98,99,99,100,100,102,102,103,103,104,104,106,106,109,109,110,110,111,111,112,112,113,113],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,-1,-1,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,-1,-1,-1,80,82,84,-1,-1,-1,86,-1,88,90,92,-1,-1,94,-1,96,98,-1,-1,-1,100,102,-1,104,106,108,-1,110,-1,112,114,-1,-1,-1,116,-1,118,-1,120,-1,-1,122,124,-1,126,128,-1,130,132,134,136,138,140,-1,142,144,146,-1,148,-1,-1,150,152,154,156,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.218466E7,5.862126E2,1.0908286E3,1.3035715E0,1.858736E-3,3.98231E-2,5.24422E5,7.2896E5,4.85E2,2.81E2,3.771981E6,2.1226158E0,-1.3869343E-3,1.8333675E-2,2E1,4.47682E5,3E0,1.2477659E3,7.759459E7,7.73756E0,8.561815E7,1.18827E5,1.0972222E0,-1.4355667E-2,-5.9068627E-3,2.2222222E-1,1.8722344E8,-2.8485423E-3,-1.1783216E-2,9.417789E1,6.806472E7,3.7990784E7,4E0,1.2342778E1,4.247E3,5.863334E6,2.046E3,4.30453E2,3.4782608E0,6.564E3,2.98E2,3.5432E4,1E0,-8.996306E-3,1.600814E6,6.999664E-4,-3.865413E-3,4.826884E-3,5.2E1,4E0,2.04E5,-5.077108E-3,-1.656313E-2,8.934659E-4,9.6590906E-1,-3.4316098E-3,1.0315458E8,6.678E4,1E0,1.9048094E-3,8.78288E-3,1.819795E6,-5.2145426E-3,1.108E3,1E1,-8.9589483E-4,4.50799E-3,-1.0038756E-2,3.928E4,2.2805734E9,8.368083E-3,4.531E3,9.6203804E-1,5.704846E1,2.476233E-3,4.759E3,4.571628E-3,8.17824E5,4.4200982E8,-1.189959E-3,2.4334511E-3,1.3521218E-3,9E0,8.133706E-3,2.784091E-1,9.503576E-3,1E0,4.07148E-3,3.434088E-4,1E0,1.6958537E9,-1.686133E-2,1.42E1,1.393168E6,4.900997E-3,3.7939188E0,4.9363803E5,1.116567E3,2.5927516E11,2.0606E4,1.3794063E1,7.610678E-3,8.417E3,3.2049225E1,1.0963991E7,4.6535498E-3,5.5465E4,3.0893937E-4,-5.891491E-3,1.6268E4,1.1502944E0,5.2610065E1,4.7777777E0,1E0,-3.4477734E-3,5.848226E-3,-2.946408E-3,4.4475594E-3,5.605608E-4,5.741018E-3,-4.433103E-3,5.6876196E-4,-1.6811836E-3,1.6515437E-3,-7.525631E-3,-5.9444224E-5,-4.468828E-3,1.0956415E-3,-5.4019247E-3,-6.6321036E-3,-2.1964053E-3,3.835905E-4,-5.8653257E-3,1.973992E-3,-3.936616E-3,-9.409063E-4,-3.5741841E-3,1.2589415E-3,-6.3246335E-3,5.300012E-3,-5.789282E-4,9.056214E-3,2.3488363E-3,-2.686175E-3,2.7170787E-3,9.513244E-5,-1.1179359E-2,3.4369305E-3,-2.512081E-4,3.2779176E-4,-5.361728E-3,-7.8712933E-4,-7.2362684E-3,-1.15161856E-4,2.40461E-3,9.669505E-3,2.6732346E-3,4.1899476E-3,-5.2382704E-3],"split_indices":[44,51,4,52,56,41,11,9,0,10,1,53,0,41,3,9,8,54,7,57,12,28,52,0,0,57,7,0,0,51,7,7,8,53,2,1,2,32,55,10,0,28,112,0,11,0,0,0,8,10,5,0,0,0,52,0,7,1,63,0,0,1,0,0,3,0,0,0,12,30,0,28,41,57,0,2,0,11,7,0,0,0,0,0,53,0,15,0,0,15,5,0,57,1,0,53,32,4,30,9,55,0,12,57,46,0,9,0,0,11,37,55,53,90,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.188E3,2.049E3,1.39E2,1.299E3,7.5E2,2.1E1,1.18E2,2.71E2,1.028E3,3.14E2,4.36E2,1.4E1,7E0,1.09E2,9E0,1.69E2,1.02E2,6.86E2,3.42E2,8.6E1,2.28E2,1.98E2,2.38E2,5E0,9E0,4.5E1,6.4E1,6E0,3E0,1.62E2,7E0,3.5E1,6.7E1,6.7E2,1.6E1,7.3E1,2.69E2,3.9E1,4.7E1,9.3E1,1.35E2,5.6E1,1.42E2,3E0,2.35E2,1.3E1,3.2E1,5E0,5.9E1,5.7E1,1.05E2,4E0,3E0,7E0,2.8E1,1E1,5.7E1,5.88E2,8.2E1,9E0,7E0,6.6E1,7E0,1.08E2,1.61E2,1.7E1,2.2E1,4E0,4.3E1,8E1,1.3E1,3.4E1,1.01E2,1.3E1,4.3E1,1.37E2,5E0,2.05E2,3E1,4.7E1,1.2E1,1.6E1,4.1E1,3E0,1.02E2,1.4E1,1.4E1,2.1E1,3.6E1,4.26E2,1.62E2,3E0,7.9E1,5.3E1,1.3E1,7.9E1,2.9E1,3.4E1,1.27E2,1.1E1,3.2E1,1E1,7E1,2.4E1,1E1,2.4E1,7.7E1,7E0,6E0,3.7E1,1E2,1.85E2,2E1,1.1E1,1.9E1,2E0,3.9E1,1E1,9.2E1,1E1,4E0,2.23E2,2.03E2,1.58E2,4E0,4.6E1,3.3E1,4.7E1,6E0,4.2E1,3.7E1,2.6E1,3E0,3E1,4E0,9.2E1,3.5E1,3E0,8E0,4E0,2.8E1,4E0,6.6E1,9E0,1.5E1,5E0,5E0,2.9E1,4.8E1,1.1E1,2.6E1,9.7E1,3E0,7.4E1,1.11E2,8E0,1.2E1,9E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[6.921225E-4,-1.6143285E-2,9.858751E-3,4.390124E-2,-2.2980327E-2,1.5704637E-2,-3.6680512E-2,9.5815495E-2,1.46670025E-2,-4.352165E-3,-4.284492E-2,1.3916972E-2,1.1342114E-1,-2.1277644E-2,-9.196299E-2,-3.4933032E-3,1.2657782E-1,-2.4811178E-4,6.6935727E-3,4.860228E-3,-7.208996E-2,-1.0237584E-1,-3.0117102E-2,2.8252397E-2,2.4329366E-3,9.212054E-3,1.2902258E-3,-1.6945304E-3,3.774027E-2,-9.097935E-3,-5.7538606E-2,6.9516175E-3,-5.434053E-4,8.005398E-3,-6.2801614E-3,-1.0180622E-3,-1.6555203E-1,-1.6621832E-2,-8.69161E-2,-1.2653929E-2,-8.13607E-2,3.6176216E-2,-7.0669237E-3,-5.2626897E-2,7.738188E-3,8.791271E-2,-1.2184815E-3,-1.0345192E-3,-1.2823075E-1,4.828001E-3,5.90943E-3,-2.4995771E-3,3.5268045E-3,-3.1170112E-1,-3.092532E-2,-1.2531064E-1,-1.4135829E-3,2.419633E-2,-3.1249456E-2,-1.0437931E-1,3.1137565E-2,2.5742816E-2,6.725257E-2,-3.4754634E-2,1.2947988E-3,-1.6843744E-1,-1.8464155E-2,-9.8582625E-2,9.7446935E-3,6.0926774E-3,-1.8859279E-3,-2.4948523E-3,-1.1595216E-2,1.2850538E-2,-4.5471776E-2,-3.4853322E-3,-1.665021E-2,-4.684295E-3,3.477768E-3,-7.250908E-3,3.6025848E-4,3.0433116E-3,-4.710045E-3,-1.049762E-2,-7.432943E-2,-1.4791166E-2,-8.2665004E-2,8.5216776E-2,-5.0636586E-3,3.194019E-2,-2.687687E-2,1.2496413E-1,4.446658E-2,2.599686E-3,-5.4737896E-2,-1.088908E-2,-2.806992E-3,5.319699E-2,-2.202511E-3,2.761053E-3,-8.420495E-3,2.5498042E-2,-1.9995177E-3,3.1925473E-4,5.3409664E-3,-1.0911545E-2,-1.360705E-3,2.459072E-3,-1.6802696E-3,-1.9935425E-3,2.6710234E-3,1.6731515E-3,-4.996469E-3,-1.3086431E-3,-6.3083214E-3,6.950727E-3,-6.24673E-4,4.7562935E-4,2.4647512E-3,-6.179749E-3,-2.482476E-4,-7.4547605E-4,6.345715E-3,-9.8430675E-5,3.01514E-3,5.8412686E-4,-3.8464079E-3,-3.899051E-3,3.6787477E-3,9.73711E-4,8.942294E-3,2.5995696E-5,-8.438683E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,-1,33,35,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,49,-1,51,53,-1,55,57,59,61,63,65,67,69,-1,-1,71,73,-1,-1,-1,75,77,79,-1,81,83,85,87,89,91,93,-1,95,97,99,101,-1,-1,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,107,109,111,-1,113,115,-1,117,119,121,123,-1,125,-1,-1,127,-1,-1,-1,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4538177E-1,3.2491463E-1,3.9479566E-1,1.2152624E-1,2.6217246E-1,2.2341505E-1,1.3598767E-1,1.6407868E-1,1.4301921E-1,2.2936834E-1,2.5680745E-1,2.0835099E-1,1.700944E-1,1.2341991E-1,1.19754374E-1,0E0,9.527326E-2,0E0,0E0,1.4741835E-1,2.9210258E-1,2.1367997E-1,2.5311524E-1,1.57819E-1,2.0605004E-1,0E0,0E0,0E0,9.3193516E-2,0E0,6.573143E-2,0E0,0E0,1.23166196E-1,0E0,1.154946E-1,3.6419642E-1,0E0,1.2010971E-1,1.4671437E-1,1.8969801E-1,1.4741546E-1,1.0369535E-1,2.4218708E-1,1.3813397E-1,1.02338344E-1,0E0,0E0,7.004888E-2,1.2608273E-1,0E0,0E0,0E0,9.012264E-2,9.391931E-2,1.5916389E-1,0E0,8.985986E-2,1.2616608E-1,2.626502E-1,1.1005195E-1,1.1369035E-1,1.4567E-1,1.1181961E-1,0E0,9.249291E-2,1.08577244E-1,1.7871153E-1,1.16977714E-1,0E0,0E0,0E0,0E0,1.6314484E-1,1.223846E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.6171165E-2,2.2865802E-1,1.88528E-1,0E0,1.6658598E-1,6.975424E-2,0E0,1.4893866E-1,8.60228E-2,7.1092606E-2,8.685042E-2,0E0,1.012581E-1,0E0,0E0,6.563333E-2,0E0,0E0,0E0,1.8111014E-1,1.7237617E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,19,19,20,20,21,21,22,22,23,23,24,24,28,28,30,30,33,33,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,49,49,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,73,73,74,74,82,82,83,83,84,84,86,86,87,87,89,89,90,90,91,91,92,92,94,94,97,97,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,-1,34,36,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,50,-1,52,54,-1,56,58,60,62,64,66,68,70,-1,-1,72,74,-1,-1,-1,76,78,80,-1,82,84,86,88,90,92,94,-1,96,98,100,102,-1,-1,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,108,110,112,-1,114,116,-1,118,120,122,124,-1,126,-1,-1,128,-1,-1,-1,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.835051E2,1E0,6.747114E7,5.8601086E1,2.6E2,4.5505118E2,3.01968E5,1.2909091E1,1.836923E1,3.1E1,7.08E2,2.5291866E1,4.1485362E5,1.3664E4,1E0,-3.4933032E-3,2.2348747E2,-2.4811178E-4,6.6935727E-3,1.4688152E9,1E0,1.3314917E0,4.1401203E5,2.8091298E7,5.31E2,9.212054E-3,1.2902258E-3,-1.6945304E-3,8.352216E-2,-9.097935E-3,9.60094E5,6.9516175E-3,-5.434053E-4,8.638343E0,-6.2801614E-3,2.3428571E0,5.7616882E1,-1.6621832E-2,1.838444E6,2.294901E0,2.2348747E2,6.9664386E2,5.033057E-4,1.1508651E1,1.0784314E0,5.2E1,-1.2184815E-3,-1.0345192E-3,1.6E1,2.3871907E2,5.90943E-3,-2.4995771E-3,3.5268045E-3,1.2836386E2,4.8658E5,1.3860265E6,-1.4135829E-3,6.439778E5,2.122448E5,9.536863E4,9.610984E-1,1E0,1.3794063E1,3.4015749E0,1.2947988E-3,3.1875726E-2,2.5369E4,2.0687214E6,4.0208E4,6.0926774E-3,-1.8859279E-3,-2.4948523E-3,-1.1595216E-2,9.750871E6,1.727279E-4,-3.4853322E-3,-1.665021E-2,-4.684295E-3,3.477768E-3,-7.250908E-3,3.6025848E-4,3.0433116E-3,1.3298E4,6.554713E4,1.5773E4,-1.4791166E-2,7.27E2,2.0074467E5,-5.0636586E-3,4.5168175E5,1.5476604E0,1E0,8.5518695E4,2.599686E-3,1E0,-1.088908E-2,-2.806992E-3,1.803E3,-2.202511E-3,2.761053E-3,-8.420495E-3,3.119384E2,6.134093E7,3.1925473E-4,5.3409664E-3,-1.0911545E-2,-1.360705E-3,2.459072E-3,-1.6802696E-3,-1.9935425E-3,2.6710234E-3,1.6731515E-3,-4.996469E-3,-1.3086431E-3,-6.3083214E-3,6.950727E-3,-6.24673E-4,4.7562935E-4,2.4647512E-3,-6.179749E-3,-2.482476E-4,-7.4547605E-4,6.345715E-3,-9.8430675E-5,3.01514E-3,5.8412686E-4,-3.8464079E-3,-3.899051E-3,3.6787477E-3,9.73711E-4,8.942294E-3,2.5995696E-5,-8.438683E-3],"split_indices":[51,52,44,49,10,55,11,54,57,3,2,57,27,2,66,0,51,0,0,5,65,53,32,44,10,0,0,0,37,0,9,0,0,52,0,55,49,0,9,55,51,51,38,55,55,3,0,0,3,51,0,0,0,51,1,27,0,50,31,27,56,79,55,55,0,26,28,46,9,0,0,0,0,9,40,0,0,0,0,0,0,0,9,32,9,0,0,27,0,27,52,65,27,0,14,0,0,2,0,0,0,56,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.236E3,7.88E2,1.448E3,8E1,7.08E2,1.287E3,1.61E2,2.8E1,5.2E1,3.66E2,3.42E2,1.265E3,2.2E1,1.27E2,3.4E1,4E0,2.4E1,4.6E1,6E0,3.23E2,4.3E1,5.9E1,2.83E2,5.62E2,7.03E2,1E1,1.2E1,1E2,2.7E1,7E0,2.7E1,2E1,4E0,3.17E2,6E0,2.5E1,1.8E1,2E0,5.7E1,2.12E2,7.1E1,4.59E2,1.03E2,6.1E1,6.42E2,1.5E1,1.2E1,1.9E1,8E0,3.1E2,7E0,1.5E1,1E1,8E0,1E1,3.3E1,2.4E1,7.1E1,1.41E2,5.9E1,1.2E1,3.45E2,1.14E2,5.8E1,4.5E1,1.3E1,4.8E1,1.1E1,6.31E2,1.1E1,4E0,6E0,2E0,2.68E2,4.2E1,2E0,6E0,6E0,4E0,2.6E1,7E0,2.8E1,4.3E1,9.6E1,4.5E1,4E0,5.5E1,9E0,3E0,3.09E2,3.6E1,3.1E1,8.3E1,1E1,4.8E1,7E0,6E0,1.4E1,3.4E1,4E0,7E0,2.69E2,3.62E2,2.55E2,1.3E1,2E0,4E1,1.5E1,2.8E1,6.5E1,3.1E1,1.1E1,3.4E1,2.9E1,2.6E1,5E0,4E0,1.59E2,1.5E2,5E0,3.1E1,3E0,2.8E1,2.7E1,5.6E1,1.5E1,3.3E1,2E0,1.2E1,2.64E2,5E0,3.58E2,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[-1.2296082E-3,-1.6800497E-2,8.470626E-3,3.277566E-2,-2.2789782E-2,4.9688637E-2,2.5167242E-3,-1.8056071E-3,1.1311808E-1,-8.509765E-2,-1.718952E-2,4.731578E-3,3.2544147E-2,-3.8302955E-4,7.403156E-2,-1.629216E-2,9.1752805E-2,1.4240924E-1,-4.44731E-3,-4.6750396E-2,-1.0367321E-2,1.0411265E-2,-2.7173607E-2,-6.5101884E-2,4.3573074E-2,4.1374657E-2,-5.375865E-3,5.591359E-2,1.4538551E-2,-6.7195157E-3,-8.581715E-3,6.3064573E-3,-2.2530323E-3,2.900185E-3,9.027392E-3,-9.280513E-2,2.7071403E-3,-1.3225558E-2,2.0217937E-3,-5.501135E-2,-3.1297638E-3,9.5748255E-4,-1.0167739E-2,9.0622865E-3,3.6508664E-2,7.2242916E-2,5.0031412E-3,-1.9551972E-2,7.408055E-3,5.5395258E-3,1.2697111E-3,-2.2723796E-3,2.2885818E-2,-1.2290927E-2,-3.3643174E-3,1.0552789E-3,-8.318204E-2,-5.8105905E-2,9.454426E-3,1.4489698E-2,-5.3048927E-2,3.030691E-2,9.241841E-3,-1.9224647E-3,8.422157E-2,-2.3442917E-3,3.399449E-2,-2.1728354E-2,1.1483515E-1,1.669174E-2,-2.016032E-2,3.9433315E-3,-9.513417E-4,1.4003231E-2,-4.939743E-3,9.0657733E-4,-1.3235605E-1,-6.393402E-2,2.5799219E-3,8.318773E-3,5.599849E-3,-3.7014127E-2,-8.090677E-3,3.836444E-2,-2.5954586E-3,4.590979E-3,-1.1918052E-3,6.702512E-2,-8.002672E-4,3.0664825E-3,-2.4531268E-2,1.1084555E-2,-5.491783E-4,4.411487E-2,-5.27186E-3,3.3993926E-2,-4.4206902E-2,8.96155E-4,-6.83081E-3,1.1351898E-3,-7.3913834E-3,-4.2756367E-3,-2.1284518E-3,4.6597472E-3,4.8687E-5,-3.8543064E-3,-1.6337284E-4,5.594113E-4,3.9782426E-3,-1.3824181E-3,4.934538E-3,-1.6724138E-3,-1.8485055E-4,-4.2403582E-3,2.5021615E-3,-6.121497E-3,-2.4645196E-5,-1.0761938E-3,2.70818E-3,-6.044369E-3,-9.909411E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,-1,37,39,41,43,45,47,49,-1,51,-1,-1,-1,-1,-1,53,-1,55,-1,57,59,-1,-1,-1,61,63,65,67,69,-1,-1,-1,71,-1,-1,73,75,77,-1,79,81,83,-1,-1,85,-1,87,89,91,93,95,-1,-1,97,-1,-1,99,101,-1,103,-1,105,-1,107,-1,-1,-1,109,-1,-1,111,-1,-1,113,115,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3802158E-1,2.5580615E-1,3.3829257E-1,2.5859013E-1,2.661745E-1,1.6013852E-1,2.5012308E-1,9.0947226E-2,1.873275E-1,3.3692285E-1,1.945588E-1,0E0,1.4714545E-1,2.4206994E-1,1.9245505E-1,9.492186E-2,6.748497E-2,9.884846E-2,0E0,2.5580138E-1,0E0,1.5276867E-1,3.465389E-1,1.9732368E-1,1.2654643E-1,1.3885653E-1,1.8807644E-1,7.989247E-2,0E0,7.322127E-2,0E0,0E0,0E0,0E0,0E0,9.879038E-2,0E0,1.1197995E-1,0E0,2.0808506E-1,2.4616279E-1,0E0,0E0,0E0,1.1544147E-1,9.563574E-2,9.724124E-2,1.4639416E-1,1.4007764E-1,0E0,0E0,0E0,9.767276E-2,0E0,0E0,1.3560823E-1,9.90766E-2,1.6328436E-1,0E0,1.3845909E-1,1.4047079E-1,8.269703E-2,0E0,0E0,1.1853856E-1,0E0,6.774652E-2,1.232748E-1,1.350115E-1,2.465776E-1,1.8120977E-1,0E0,0E0,8.4143944E-2,0E0,0E0,6.7339346E-2,1.12734735E-1,0E0,1.3488506E-1,0E0,1.05783746E-1,0E0,1.3410883E-1,0E0,0E0,0E0,1.0159267E-1,0E0,0E0,1.1935845E-1,0E0,0E0,2.848198E-1,1.3998306E-1,6.5564856E-2,1.8841545E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,35,35,37,37,39,39,40,40,44,44,45,45,46,46,47,47,48,48,52,52,55,55,56,56,57,57,59,59,60,60,61,61,64,64,66,66,67,67,68,68,69,69,70,70,73,73,76,76,77,77,79,79,81,81,83,83,87,87,90,90,93,93,94,94,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,-1,38,40,42,44,46,48,50,-1,52,-1,-1,-1,-1,-1,54,-1,56,-1,58,60,-1,-1,-1,62,64,66,68,70,-1,-1,-1,72,-1,-1,74,76,78,-1,80,82,84,-1,-1,86,-1,88,90,92,94,96,-1,-1,98,-1,-1,100,102,-1,104,-1,106,-1,108,-1,-1,-1,110,-1,-1,112,-1,-1,114,116,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.247716E5,2.183847E0,3.7761906E1,9.61E2,3.7129E4,6.9673E4,6.6875E-1,3E1,8E0,7.667683E6,4.731578E-3,6.363636E-1,7.3244873E3,1.2328733E1,3.09627E5,6E0,1.139605E6,-4.44731E-3,7.920064E5,-1.0367321E-2,1.421E3,4.423E3,5.4803147E0,6.9436204E-1,4.839905E1,3.5150378E5,1.9E1,1.4538551E-2,3.6E1,-8.581715E-3,6.3064573E-3,-2.2530323E-3,2.900185E-3,9.027392E-3,1.0119178E10,2.7071403E-3,6.298144E0,2.0217937E-3,7.160724E1,4.1755924E7,9.5748255E-4,-1.0167739E-2,9.0622865E-3,5.1678925E1,9.577143E1,9.421523E-5,1.96E4,1.2453214E0,5.5395258E-3,1.2697111E-3,-2.2723796E-3,1.5444445E1,-1.2290927E-2,-3.3643174E-3,4.2320535E2,1.3358E4,1.6347875E3,9.454426E-3,4.5505118E2,6.03368E5,3.3709784E5,9.241841E-3,-1.9224647E-3,1E0,-2.3442917E-3,9.969697E0,3.02E2,9.5E2,6.34E2,3.09627E5,3.9433315E-3,-9.513417E-4,5.334262E0,-4.939743E-3,9.0657733E-4,3.392348E7,2.3871907E2,2.5799219E-3,4.2782607E0,5.599849E-3,7.808427E9,-8.090677E-3,8.182648E6,-2.5954586E-3,4.590979E-3,-1.1918052E-3,1.0119178E10,-8.002672E-4,3.0664825E-3,3.891716E8,1.1084555E-2,-5.491783E-4,7.92582E5,1.892E3,1.1E1,1.273801E6,8.96155E-4,-6.83081E-3,1.1351898E-3,-7.3913834E-3,-4.2756367E-3,-2.1284518E-3,4.6597472E-3,4.8687E-5,-3.8543064E-3,-1.6337284E-4,5.594113E-4,3.9782426E-3,-1.3824181E-3,4.934538E-3,-1.6724138E-3,-1.8485055E-4,-4.2403582E-3,2.5021615E-3,-6.121497E-3,-2.4645196E-5,-1.0761938E-3,2.70818E-3,-6.044369E-3,-9.909411E-4],"split_indices":[109,44,53,51,12,9,2,55,3,8,44,0,52,49,52,9,10,30,0,27,0,2,2,55,52,47,46,8,0,2,0,0,0,0,0,18,0,56,0,55,49,0,0,0,55,54,41,2,37,0,0,0,51,0,0,4,9,4,0,55,9,32,0,0,112,0,57,28,0,0,9,0,0,52,0,0,12,51,0,55,0,12,0,44,0,0,0,19,0,0,5,0,0,12,2,3,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.236E3,8.58E2,1.378E3,9.2E1,7.66E2,1.73E2,1.205E3,6.5E1,2.7E1,6.2E1,7.04E2,4E1,1.33E2,1.159E3,4.6E1,5.7E1,8E0,2.4E1,3E0,5E1,1.2E1,1.87E2,5.17E2,1.3E1,1.2E2,1.23E2,1.036E3,4.4E1,2E0,5.5E1,2E0,6E0,2E0,1.1E1,1.3E1,3.5E1,1.5E1,1.11E2,7.6E1,2.39E2,2.78E2,9E0,4E0,4E0,1.16E2,6.6E1,5.7E1,4.91E2,5.45E2,1.2E1,3.2E1,2.2E1,3.3E1,2E0,3.3E1,9.3E1,1.8E1,2.37E2,2E0,2.06E2,7.2E1,1.13E2,3E0,6E0,6E1,1.9E1,3.8E1,4.84E2,7E0,4.08E2,1.37E2,1.3E1,2E1,8.4E1,9E0,6E0,1.2E1,2.26E2,1.1E1,1.96E2,1E1,6.5E1,7E0,1.04E2,9E0,5.2E1,8E0,2.3E1,1.5E1,1.4E1,4.7E2,3E0,4E0,1.81E2,2.27E2,4.2E1,9.5E1,8.2E1,2E0,2E0,1E1,7.9E1,1.47E2,1.3E1,1.83E2,2.6E1,3.9E1,6.9E1,3.5E1,7E0,1.6E1,2.92E2,1.78E2,1.3E1,1.68E2,7E0,2.2E2,1.3E1,2.9E1,1.8E1,7.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[2.6073382E-4,-1.5067672E-2,9.713436E-3,-1.7722046E-2,3.2435588E-3,6.241364E-3,7.144892E-2,-1.9940434E-2,3.6725355E-3,7.363547E-3,-5.622719E-3,3.5566327E-3,-3.1259698E-3,-1.1497951E-2,-4.9092527E-2,7.68519E-2,5.3369366E-3,-3.8035854E-3,-8.421448E-3,-1.3921466E-2,-4.138506E-2,5.738603E-4,1.1257453E-1,-4.250373E-3,6.9604195E-3,8.773068E-2,-1.17570255E-2,2.95046E-2,-6.634273E-2,8.684184E-3,2.6044196E-3,2.6687033E-2,-8.884872E-4,1.3250221E-1,-5.635747E-3,4.937832E-3,-2.0629598E-2,6.952758E-3,-1.387538E-2,-1.5156855E-1,-4.6420436E-2,7.58246E-2,8.923974E-3,-2.015144E-2,1.1736267E-2,7.3340167E-3,-1.737391E-4,5.2192545E-4,-4.9504368E-3,-5.381474E-3,-6.28092E-4,-3.4892212E-3,6.8732994E-3,-4.215596E-3,-1.0145662E-2,-6.549246E-3,-1.2381809E-3,2.4259929E-3,8.439829E-3,7.1846583E-4,-3.148948E-3,-4.851127E-4,-4.949777E-3,4.2482363E-3,3.0682207E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,19,21,23,-1,25,-1,27,-1,29,-1,31,33,35,37,39,-1,-1,41,43,45,-1,47,49,-1,51,53,55,57,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.171763E-1,1.9345534E-1,2.8978395E-1,1.7944202E-1,0E0,1.8870921E-1,7.7337205E-2,1.948517E-1,0E0,1.7886114E-1,0E0,0E0,0E0,1.369401E-1,3.4178293E-1,8.110517E-2,2.0006925E-1,0E0,1.9122007E-1,0E0,3.0980527E-1,0E0,8.9292675E-2,0E0,1.8876435E-1,2.0651491E-1,8.512637E-2,2.4970242E-1,2.1199238E-1,0E0,0E0,3.021966E-1,2.1254143E-1,6.897229E-2,0E0,1.518916E-1,2.4724633E-1,0E0,3.7817255E-1,8.0883384E-2,1.9464223E-1,2.1629018E-1,1.4063644E-1,2.8999397E-1,2.1433598E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13,14,14,15,15,16,16,18,18,20,20,22,22,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,20,22,24,-1,26,-1,28,-1,30,-1,32,34,36,38,40,-1,-1,42,44,46,-1,48,50,-1,52,54,56,58,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,5.13657E6,6.396218E6,1.5787431E0,3.2435588E-3,1E0,1E0,3.64299E5,3.6725355E-3,4.86E2,-5.622719E-3,3.5566327E-3,-3.1259698E-3,1.0882353E1,2.35E2,1.857051E5,2.53E2,-3.8035854E-3,7E0,-1.3921466E-2,1.9588852E5,5.738603E-4,2.3066575E6,-4.250373E-3,3.3382E4,1.4409872E5,1E0,1.8333334E0,6.0479047E4,8.684184E-3,2.6044196E-3,1.03E2,1.273801E6,5.148E1,-5.635747E-3,1.9624242E7,5.86E2,6.952758E-3,6.341735E2,1.113742E6,3.502483E-2,1.0509E4,1E0,1.010368E6,7.3244873E3,7.3340167E-3,-1.737391E-4,5.2192545E-4,-4.9504368E-3,-5.381474E-3,-6.28092E-4,-3.4892212E-3,6.8732994E-3,-4.215596E-3,-1.0145662E-2,-6.549246E-3,-1.2381809E-3,2.4259929E-3,8.439829E-3,7.1846583E-4,-3.148948E-3,-4.851127E-4,-4.949777E-3,4.2482363E-3,3.0682207E-4],"split_indices":[109,9,43,37,0,82,113,9,0,12,0,0,0,51,2,27,2,0,12,0,50,0,44,0,9,44,8,55,32,0,0,11,9,32,0,31,12,0,51,9,37,10,89,9,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.187E3,8.34E2,1.353E3,8.1E2,2.4E1,1.282E3,7.1E1,7.93E2,1.7E1,1.272E3,1E1,6.8E1,3E0,6.16E2,1.77E2,3.5E1,1.237E3,2.4E1,5.92E2,4E0,1.73E2,1.3E1,2.2E1,1.9E1,1.218E3,1.9E1,5.73E2,4.5E1,1.28E2,8E0,1.4E1,3.46E2,8.72E2,1.6E1,3E0,1.99E2,3.74E2,1.1E1,3.4E1,2.3E1,1.05E2,9.1E1,2.55E2,3.45E2,5.27E2,1.3E1,3E0,1.89E2,1E1,2.3E1,3.51E2,2.5E1,9E0,1.4E1,9E0,1.6E1,8.9E1,7.7E1,1.4E1,2.35E2,2E1,3.13E2,3.2E1,2.9E1,4.98E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-3.174873E-3,-1.4236334E-3,-7.8621954E-2,-9.474421E-3,1.8957613E-2,-5.6786627E-2,-2.0776021E-1,-6.773036E-3,-6.2010847E-2,2.157733E-2,-4.014294E-3,-4.35394E-3,2.1528502E-3,-1.2914283E-2,-4.238197E-4,-2.0299008E-2,8.492702E-5,-8.382301E-3,-4.705236E-2,4.919933E-2,7.5425464E-3,4.604187E-3,-3.15473E-2,-2.586498E-3,6.2437825E-2,-2.5060682E-3,4.6973345E-3,1.7856015E-2,8.460105E-2,5.8108054E-2,9.015395E-4,-1.0804364E-2,6.1117854E-2,-1.5071575E-1,-2.7210303E-2,4.1342573E-3,-2.7885132E-2,1.5992655E-1,1.1694682E-3,4.103444E-2,-7.1603E-2,7.570156E-2,1.0477822E-2,4.940478E-4,6.3724252E-3,-1.6032167E-2,2.7196433E-2,7.585048E-3,-9.26872E-2,1.14443965E-1,7.358869E-3,1.139116E-3,-2.7993602E-1,2.754958E-2,-3.3245254E-2,8.81946E-3,-6.089658E-2,-9.413541E-3,-2.3174068E-2,9.482607E-3,-1.8992973E-3,4.424757E-3,2.1804564E-2,-1.3099383E-3,-1.2090374E-2,6.695028E-2,9.715051E-3,-1.366191E-2,-8.246047E-3,1.5057461E-2,7.6643433E-3,8.55599E-4,-5.8458475E-3,-6.3120974E-3,1.5414853E-3,1.2727331E-3,7.790075E-3,-4.374806E-3,1.9907858E-3,-1.9005556E-2,-3.173611E-3,-9.339837E-4,6.5757055E-3,-7.4826097E-3,-1.2763778E-3,1.7273712E-3,5.3564247E-5,-1.8950873E-3,-1.0307871E-2,-4.5053936E-3,-6.111938E-4,4.966782E-4,8.588391E-3,3.5294008E-3,-2.3930443E-3,-3.389202E-4,-2.9622288E-3,8.9528423E-4,-6.3230684E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,-1,61,63,65,-1,-1,-1,67,69,71,73,75,77,-1,79,81,83,85,87,-1,89,-1,-1,-1,91,-1,-1,93,-1,95,-1,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9703927E-1,3.610027E-1,1.3201061E-1,2.2338961E-1,1.7756523E-1,1.8948087E-1,1.0494357E-1,1.3923544E-1,1.3311616E-1,2.357192E-1,0E0,0E0,0E0,0E0,0E0,1.4160503E-1,1.6623285E-1,0E0,9.337264E-2,2.2595888E-1,1.3594899E-1,1.3824575E-1,1.7575702E-1,1.628301E-1,1.421881E-1,0E0,0E0,2.310253E-1,1.0792518E-1,1.8178576E-1,1.6028796E-1,1.8892519E-1,9.662759E-2,2.7976468E-1,1.12285405E-1,2.3118185E-1,1.6566461E-1,1.1876014E-1,0E0,9.381586E-2,1.8408355E-1,9.744996E-2,0E0,0E0,0E0,8.373427E-2,2.4065447E-1,1.617392E-1,1.4096902E-1,7.973066E-2,7.27812E-2,0E0,1.7600054E-1,1.9578832E-1,1.9814888E-1,1.5832768E-1,1.5240698E-1,0E0,1.441461E-1,0E0,0E0,0E0,1.1978991E-1,0E0,0E0,1.201601E-1,0E0,6.888716E-2,0E0,9.837466E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,58,58,62,62,65,65,67,67,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,-1,62,64,66,-1,-1,-1,68,70,72,74,76,78,-1,80,82,84,86,88,-1,90,-1,-1,-1,92,-1,-1,94,-1,96,-1,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2427474E8,9.557794E2,2.1907706E12,4.2343444E7,1.89039E2,4.2E1,4.0950096E10,1.8991614E5,2.3446269E2,1.858736E-3,-4.014294E-3,-4.35394E-3,2.1528502E-3,-1.2914283E-2,-4.238197E-4,2.9138756E0,3.7075245E1,-8.382301E-3,1.1879E4,5.34E2,6.426937E6,2.437E3,3.3654268E0,1.2471935E7,6.745516E2,-2.5060682E-3,4.6973345E-3,2.4235377E0,7.285458E-1,5.0079144E5,1.3532244E7,1.86E2,1.17E2,6.5030734E4,1.2068E4,1E0,1.2673605E7,3.3135505E0,1.1694682E-3,9.6603775E0,6.5346925E6,9.821139E3,1.0477822E-2,4.940478E-4,6.3724252E-3,8.598387E0,2.6557522E1,4.6E1,3.7382904E7,1.3448806E8,3.83E3,1.139116E-3,3.68E2,3.5029E4,3.3870136E7,7.235588E3,5.485425E6,-9.413541E-3,1.0449754E0,9.482607E-3,-1.8992973E-3,4.424757E-3,1.3238013E2,-1.3099383E-3,-1.2090374E-2,1.1037509E2,9.715051E-3,1.3195207E7,-8.246047E-3,4.4543375E4,7.6643433E-3,8.55599E-4,-5.8458475E-3,-6.3120974E-3,1.5414853E-3,1.2727331E-3,7.790075E-3,-4.374806E-3,1.9907858E-3,-1.9005556E-2,-3.173611E-3,-9.339837E-4,6.5757055E-3,-7.4826097E-3,-1.2763778E-3,1.7273712E-3,5.3564247E-5,-1.8950873E-3,-1.0307871E-2,-4.5053936E-3,-6.111938E-4,4.966782E-4,8.588391E-3,3.5294008E-3,-2.3930443E-3,-3.389202E-4,-2.9622288E-3,8.9528423E-4,-6.3230684E-3],"split_indices":[44,51,30,44,53,8,5,27,51,56,0,0,0,0,0,57,55,0,2,0,44,2,57,44,4,0,0,37,38,27,1,10,3,27,9,79,44,52,0,55,44,4,0,0,0,34,53,3,12,7,2,0,0,1,7,46,9,0,52,0,0,0,55,0,0,55,0,9,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.248E3,2.198E3,5E1,1.576E3,6.22E2,4.4E1,6E0,1.5E3,7.6E1,6.08E2,1.4E1,3.2E1,1.2E1,4E0,2E0,5.04E2,9.96E2,7E0,6.9E1,2.04E2,4.04E2,1.57E2,3.47E2,9.56E2,4E1,6.6E1,3E0,1.09E2,9.5E1,4.6E1,3.58E2,1.24E2,3.3E1,1.1E1,3.36E2,7.56E2,2E2,1E1,3E1,8.7E1,2.2E1,9.1E1,4E0,3E1,1.6E1,2.18E2,1.4E2,1.02E2,2.2E1,1.6E1,1.7E1,5E0,6E0,3.3E1,3.03E2,7.06E2,5E1,4E0,1.96E2,8E0,2E0,2.1E1,6.6E1,1.9E1,3E0,8.7E1,4E0,2.16E2,2E0,1.3E2,1E1,9.5E1,7E0,1.6E1,6E0,7E0,9E0,4E0,1.3E1,3E0,3E0,2.4E1,9E0,1E1,2.93E2,1.45E2,5.61E2,4.6E1,4E0,2.1E1,1.75E2,6.3E1,3E0,8E1,7E0,1.94E2,2.2E1,1.27E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[-5.1463777E-3,8.954737E-3,-1.7470097E-2,5.902977E-2,3.8366101E-3,-3.348083E-2,-6.67378E-3,-4.2881686E-4,6.987301E-2,-7.2951266E-4,6.1552387E-2,-2.1962838E-2,-6.310659E-2,1.1648524E-2,-2.1862201E-2,7.6140664E-3,5.251845E-2,3.9932327E-3,-5.1032763E-2,5.6970813E-3,1.1801055E-2,-4.387499E-2,-2.7746262E-3,-4.803654E-2,-1.2628774E-1,-5.4609133E-3,1.411527E-2,-1.4764705E-2,-6.5488115E-2,1.0386772E-2,3.8084795E-3,2.2189928E-2,-1.1284439E-2,-9.554299E-2,2.2864247E-2,1.3975877E-3,-5.7883887E-3,-3.3723276E-2,-1.4805919E-1,2.5440953E-3,-1.3649129E-2,-7.050211E-2,5.838764E-3,-8.6739235E-2,-1.41360015E-2,3.5815684E-3,8.279834E-3,-2.0757047E-2,4.6849355E-2,-5.09114E-2,-1.0637984E-2,2.4453334E-3,-6.682282E-2,2.6942857E-2,-1.332173E-1,3.2834273E-2,-2.5421709E-2,3.0039875E-3,-1.1639592E-1,6.4230356E-3,-2.1386265E-3,3.3449472E-4,-5.175635E-2,-2.324658E-3,-1.2153868E-2,-5.922861E-2,-8.756783E-5,-8.357401E-2,3.26679E-3,6.00433E-3,-2.1537637E-2,3.7216034E-4,-1.2286278E-1,6.1465125E-3,7.25933E-3,-5.692945E-2,-5.463578E-4,4.362859E-3,-1.6624091E-2,-6.913678E-2,1.8934574E-3,4.0870896E-4,-6.5274932E-3,8.976405E-3,6.341328E-2,-1.1924907E-2,-9.599253E-4,-3.2019426E-4,6.0184404E-2,-5.7042323E-2,-2.1426737E-5,-8.776773E-3,-4.397582E-2,-3.8965322E-2,-1.302875E-1,-9.7957835E-2,1.7067197E-3,1.89811E-2,-5.6675628E-2,-4.7189253E-3,-1.1828152E-2,7.9949194E-4,-3.7903166E-3,-1.6106796E-1,-2.987074E-4,2.1971405E-2,-1.5695523E-2,-3.0896724E-3,3.5728288E-3,3.6065804E-3,-2.8569836E-3,-5.313147E-3,-8.0476626E-4,9.5441565E-4,-4.3228166E-3,9.07048E-3,2.393996E-3,9.028354E-3,1.90286E-3,-9.157272E-4,-5.0530904E-3,-5.2370806E-4,4.8489305E-3,4.081845E-3,-5.5019106E-3,-1.4926522E-3,-9.138956E-3,-1.0770318E-2,-2.8729225E-3,-5.389253E-3,2.1611096E-3,1.6527753E-3,-1.5927887E-3,-5.9535136E-3,2.1141008E-4,5.251694E-3,-2.7201609E-3,-1.1303687E-3,-1.0231794E-2,7.7654594E-3,6.5212016E-4,-1.2414728E-3,1.9017725E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,29,31,33,-1,35,37,39,41,43,-1,45,47,49,51,-1,53,55,57,59,-1,-1,61,63,-1,65,67,69,71,-1,-1,73,75,77,79,-1,-1,81,83,85,87,89,-1,91,-1,-1,-1,93,-1,-1,95,97,99,-1,-1,101,-1,103,105,-1,107,-1,-1,109,111,-1,-1,-1,113,115,-1,-1,-1,117,119,121,-1,123,125,127,129,-1,131,133,-1,135,-1,-1,137,-1,139,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8594404E-1,2.6529038E-1,2.0461205E-1,7.1968436E-2,2.4803351E-1,1.6138029E-1,1.9756007E-1,0E0,1.3276836E-1,2.0760189E-1,2.1918961E-1,1.4493527E-1,1.2056863E-1,1.0676191E-1,1.1917138E-1,0E0,9.49913E-2,2.223749E-1,2.4887247E-1,0E0,1.1046625E-1,1.6528827E-1,1.1939778E-1,1.3233864E-1,1.567632E-1,0E0,1.1957221E-1,1.2465868E-1,1.214432E-1,1.1139422E-1,0E0,2.7319267E-1,2.721521E-1,1.6639364E-1,2.486352E-1,0E0,0E0,1.1010471E-1,1.4922476E-1,0E0,9.704106E-2,1.4962101E-1,1.173952E-1,7.647592E-2,0E0,0E0,9.3738705E-2,9.549196E-2,9.589129E-2,8.9086846E-2,0E0,0E0,7.1710214E-2,2.320424E-1,1.5481465E-1,1.16119556E-1,2.650457E-1,0E0,2.2709072E-1,0E0,0E0,0E0,9.83862E-2,0E0,0E0,1.3883235E-1,1.3272095E-1,1.03990346E-1,0E0,0E0,6.9723435E-2,0E0,7.000813E-2,1.0023709E-1,0E0,9.6069634E-2,0E0,0E0,6.768216E-2,1.04406714E-1,0E0,0E0,0E0,3.0420643E-1,1.5819374E-1,0E0,0E0,0E0,1.5086246E-1,2.930677E-1,2.2882092E-1,0E0,2.502028E-1,8.124854E-2,8.558291E-2,8.552794E-2,0E0,8.861011E-2,1.429072E-1,0E0,1.11827865E-1,0E0,0E0,1.00400686E-1,0E0,1.8765941E-1,8.3696656E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,37,37,38,38,40,40,41,41,42,42,43,43,46,46,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,58,58,62,62,65,65,66,66,67,67,70,70,72,72,73,73,75,75,78,78,79,79,83,83,84,84,88,88,89,89,90,90,92,92,93,93,94,94,95,95,97,97,98,98,100,100,103,103,105,105,106,106],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,30,32,34,-1,36,38,40,42,44,-1,46,48,50,52,-1,54,56,58,60,-1,-1,62,64,-1,66,68,70,72,-1,-1,74,76,78,80,-1,-1,82,84,86,88,90,-1,92,-1,-1,-1,94,-1,-1,96,98,100,-1,-1,102,-1,104,106,-1,108,-1,-1,110,112,-1,-1,-1,114,116,-1,-1,-1,118,120,122,-1,124,126,128,130,-1,132,134,-1,136,-1,-1,138,-1,140,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.839905E1,6E0,1.0097089E3,3.830986E-1,1.9056322E3,1.6256282E7,1.775044E7,-4.2881686E-4,1.8294791E6,4.5184364E0,8.238194E1,2.755479E8,2.08395E5,1.43E2,1.8E2,7.6140664E-3,5.0647113E2,1E0,2.66E2,5.6970813E-3,3.7284137E2,7.0988875E6,4.06E2,1.6504E4,1.3312868E6,-5.4609133E-3,7.3139534E0,6.6558E5,1.3674345E4,4.2320535E2,3.8084795E-3,4.123E3,4.0208E4,1E0,3.0608664E2,1.3975877E-3,-5.7883887E-3,1.0119178E10,9.723319E9,2.5440953E-3,1.05E2,9.3343524E2,2.090909E0,1.64354E9,-1.41360015E-2,3.5815684E-3,2.4507338E4,2.0666666E0,2.802E5,1.5988182E-1,-1.0637984E-2,2.4453334E-3,1.1092533E0,8.07966E5,6.972973E0,2.9407698E-1,1.113742E6,3.0039875E-3,1E0,6.4230356E-3,-2.1386265E-3,3.3449472E-4,1.1603518E8,-2.324658E-3,-1.2153868E-2,1.5E1,5.68197E6,1E0,3.26679E-3,6.00433E-3,2.4906608E5,3.7216034E-4,8.387235E11,4.0672535E1,7.25933E-3,1.7310865E6,-5.463578E-4,4.362859E-3,1.9761398E7,3.01E2,1.8934574E-3,4.0870896E-4,-6.5274932E-3,3.64299E5,1.3E1,-1.1924907E-2,-9.599253E-4,-3.2019426E-4,4.3957013E-1,3.38464E5,2.5551208E10,-8.776773E-3,9.5E1,7.780377E6,1.0911816E6,9.60094E5,1.7067197E-3,1.062363E1,8.164331E-1,-4.7189253E-3,8.2892E4,7.9949194E-4,-3.7903166E-3,2E0,-2.987074E-4,1.4560842E-2,4.33E2,-3.0896724E-3,3.5728288E-3,3.6065804E-3,-2.8569836E-3,-5.313147E-3,-8.0476626E-4,9.5441565E-4,-4.3228166E-3,9.07048E-3,2.393996E-3,9.028354E-3,1.90286E-3,-9.157272E-4,-5.0530904E-3,-5.2370806E-4,4.8489305E-3,4.081845E-3,-5.5019106E-3,-1.4926522E-3,-9.138956E-3,-1.0770318E-2,-2.8729225E-3,-5.389253E-3,2.1611096E-3,1.6527753E-3,-1.5927887E-3,-5.9535136E-3,2.1141008E-4,5.251694E-3,-2.7201609E-3,-1.1303687E-3,-1.0231794E-2,7.7654594E-3,6.5212016E-4,-1.2414728E-3,1.9017725E-3],"split_indices":[47,3,54,55,54,44,44,0,44,52,57,5,9,0,8,0,4,15,0,0,57,44,0,9,27,0,57,11,4,4,0,2,9,80,51,0,0,18,30,0,51,4,55,5,0,0,4,52,10,37,0,0,52,9,55,37,9,0,109,0,0,0,12,0,0,8,49,15,0,0,46,0,30,53,0,27,0,0,44,8,0,0,0,9,10,0,0,0,37,9,30,0,10,27,47,9,0,53,38,0,12,0,0,8,0,37,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.219E3,1.035E3,1.184E3,9.5E1,9.4E2,4.76E2,7.08E2,1.3E1,8.2E1,8.72E2,6.8E1,3.44E2,1.32E2,3.21E2,3.87E2,1.1E1,7.1E1,7.98E2,7.4E1,2.9E1,3.9E1,1.6E2,1.84E2,1.08E2,2.4E1,5E0,3.16E2,3.34E2,5.3E1,3.1E1,4E1,3.64E2,4.34E2,4.6E1,2.8E1,3.5E1,4E0,1.47E2,1.3E1,2.8E1,1.56E2,7.6E1,3.2E1,2.1E1,3E0,2.5E1,2.91E2,3.05E2,2.9E1,5E1,3E0,2E1,1.1E1,3.54E2,1E1,1.05E2,3.29E2,5E0,4.1E1,1E1,1.8E1,4.5E1,1.02E2,8E0,5E0,3.5E1,1.21E2,7E1,6E0,5E0,2.7E1,6E0,1.5E1,2.88E2,3E0,5.8E1,2.47E2,1.6E1,1.3E1,4.2E1,8E0,6E0,5E0,2.38E2,1.16E2,4E0,6E0,4.3E1,6.2E1,1.46E2,1.83E2,1.9E1,2.2E1,8.9E1,1.3E1,2.5E1,1E1,9.1E1,3E1,5.4E1,1.6E1,1.7E1,1E1,1.1E1,4E0,1.67E2,1.21E2,5.4E1,4E0,4E0,9E0,2.1E1,2.1E1,2.14E2,2.4E1,7E0,1.09E2,6E0,5.6E1,8.8E1,5.8E1,1.66E2,1.7E1,8E0,1.4E1,8.7E1,2E0,4E0,9E0,2.2E1,3E0,6.9E1,2.2E1,1.3E1,1.7E1,4E0,1.2E1,4E0,7E0,7E0,1.6E2,1.01E2,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"143","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"5.681664E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics index 4895b8807..ade7f596d 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/combined.metrics @@ -1,109 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,maxOnHeapMem,0.24525835,12891675333.097458,7443156274.854312,455501414.0,8974132838.0,10119177830.0,10119177830.0,26104509235.0 -1,duration_max,0.122840814,19727.637005649718,132756.56647030966,15.0,1000.75,2626.5,6606.5,3183955.0 -2,executorCPUTime_mean,0.116636105,1600.0121232217436,8490.83160303521,3.875,196.08367473240125,438.1275815373426,1135.2854603729604,227947.84615384616 -3,scan_bw,0.09914459,991028.5544311537,1739237.4464631807,0.0,111648.31626667196,344807.51426898653,1052087.8925818277,17385294.707427993 -4,input_bytesRead_mean,0.08676977,19051772.3871117,29839180.431289956,3240.8,2282876.103862428,7538112.224328557,20442663.2280394,240185411.32144082 -5,sqlOp_SubqueryBroadcast,0.066191964,0.6070621468926554,0.48847220931507623,0.0,0.0,1.0,1.0,1.0 -6,cache_hit_ratio,0.057533674,0.18990465580425084,0.32957735575345715,0.0,0.0,0.0,0.30864069264631583,1.0 -7,shuffle_write_bw,0.042041168,165536.28360480044,275668.61585567426,0.0,3445.5108454312553,83232.4505533861,213410.4162089666,4415434.0 -8,maxMem,0.033244178,21686267078.83305,13741098115.368979,455501414.0,10119177830.0,10119177830.0,26104509235.0,43359456460.0 -9,executorDeserializeCPUTime_mean,0.02722386,7.730842747906887,16.187183743296377,0.01225114854517611,1.5320794973899026,2.4677083333333334,6.553110708090206,245.76923076923077 -10,peakExecutionMemory_max,0.026185669,514434918.05706215,1343644906.5355828,0.0,68478953.0,185118727.5,448760080.5,25501360736.0 -11,jvmGCTime_mean,0.023969578,37.70700489432971,112.05477273251788,0.0,2.6427434456928838,10.6552217453505,35.56270150609702,2226.977412731006 -12,sw_recordsWritten_sum,0.023896383,1307752725.568644,4224481965.3663344,0.0,139053.75,16308835.0,463500001.0,44204763473.0 -13,sw_bytesWrittenRatio,0.023290131,0.7500153334529006,1.312331627655776,0.0,0.004156014806987636,0.20888208704072536,1.1450582662697832,19.10476372587304 -14,resultSize_max,0.022359248,3958608.534180791,11103446.031355042,1785.0,15101.25,54348.0,1857380.5,263794922.0 -15,sr_localBytesReadRatio,0.020905973,0.4186158681422342,1.0093096602351777,0.0,0.0020370799199269708,0.06772948216509181,0.3323507501682036,16.779651636748564 -16,numExecutors,0.018349394,6.029378531073446,3.1526548701636963,1.0,1.0,8.0,8.0,14.0 -17,duration_mean,0.016111324,2226.988951208108,8813.182199843044,13.333333333333334,319.1502445988216,776.5181426587314,1918.973089868859,228041.3076923077 -18,duration_min,0.0159653,53.92768361581921,116.25739909610253,0.0,11.0,16.0,39.0,1779.0 -19,executorDeserializeTime_mean,0.01589882,18.885789433104375,44.57893656009577,0.04395604395604396,2.56471764117941,4.46117303124429,10.272696053946055,475.6 -20,scan_time,0.015392276,1520882.0240112995,7956731.9533324735,0.0,2684.5,49494.0,516427.25,217849366.0 -21,sw_bytesWritten_mean,0.012503516,8701193.262865013,23683912.279130522,0.0,21461.232201626925,1031447.9115545701,6399900.097575081,314037255.60981447 -22,sr_localBytesRead_mean,0.010507526,2910251.7965611313,7109002.559062359,0.0,9821.28396762487,375550.6402773764,2249818.307875526,104601898.28278689 -23,duration_sum,0.010165823,4842764.72118644,20681577.709232815,40.0,51743.25,347622.0,2003098.25,515693115.0 -24,platform_databricks-aws,0.009524851,0.08813559322033898,0.283532043314531,0.0,0.0,0.0,0.0,1.0 -25,executorRunTime_mean,0.009243112,2199.3590155298366,8810.870593599537,6.0,307.07921482662414,748.9813360530342,1864.906949267008,228020.38461538462 -26,numTasks_sum,0.008401272,1383.4225988700564,3662.17887830497,3.0,130.0,419.5,1177.0,86122.0 -27,sr_totalBytesReadRatio,0.0075071068,0.806944875525602,1.3776145394257457,0.0,0.004556759536252387,0.2297007756287479,1.1873950080800997,19.18647529336742 -28,output_recordsWritten_sum,0.007329679,82.66638418079096,1380.6943192952579,0.0,0.0,0.0,0.0,28800.0 -29,executorCores,0.0070164194,12.945762711864408,3.103634601050894,6.0,12.0,12.0,16.0,16.0 -30,input_recordsRead_sum,0.0068921465,4254947665.6189265,13496209064.983372,0.0,41728108.0,454111279.5,2835348935.0,230401504404.0 -31,sw_writeTime_mean,0.0067101875,65.46063250813955,282.56757535974504,0.0,4.31657853171011,17.049135220125788,45.60306478963834,6475.538414708389 -32,resultSerializationTime_sum,0.006529984,73.159604519774,251.12253258121743,0.0,0.0,8.0,49.0,7044.0 -33,sr_fetchWaitTime_mean,0.0058721746,53.589055836211855,852.5182710790448,0.0,0.0,0.0,0.199677245831092,26216.59561667383 -34,data_size,0.005667293,354067441856.6409,1436943956289.1838,0.0,1947746994.0,25418067831.0,172467501151.5,22044104226233.0 -35,memoryBytesSpilledRatio,0.0055548237,0.38330444729758456,2.2121805527381846,0.0,0.0,0.0,0.0,43.971493342592005 -36,sr_totalBytesRead_mean,0.0048567513,9202557.355776286,24510563.626570754,0.0,23858.06464995155,1237051.5861350573,7001522.360789239,314039311.06463194 -37,shuffle_read_bw,0.0046514957,200340881.16569823,2736126536.080987,0.0,0.0,0.0,4142971.1076581576,107487374955.0 -38,sr_localBlocksFetched_sum,0.004350457,44000.88644067797,208616.4344751978,0.0,145.0,1412.5,15105.5,3860026.0 -39,sr_remoteBlocksFetched_sum,0.0043445663,165629.2963276836,948144.8162985918,0.0,0.0,368.0,45471.0,21016512.0 -40,sqlOp_SortMergeJoin,0.0042423685,0.4788135593220339,0.49962150606394323,0.0,0.0,0.0,1.0,1.0 -41,sqlOp_TakeOrderedAndProject,0.0037542866,0.10423728813559321,0.3056112883826109,0.0,0.0,0.0,0.0,1.0 -42,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0032178024,0.05593220338983051,0.22982322022384652,0.0,0.0,0.0,0.0,1.0 -43,sr_remoteBytesReadRatio,0.0028581598,0.38832900738336773,0.9402463724732919,0.0,0.0,0.0002836891594572681,0.31458116619117393,16.78961797640585 -44,sr_remoteBytesRead_mean,0.0027610108,6292305.559215155,21016177.50230388,0.0,0.0,4620.1696005917165,2150364.996089182,274787384.05385995 -45,sqlOp_Sort,0.002645748,0.581638418079096,0.49335983507473885,0.0,0.0,1.0,1.0,1.0 -46,sqlOp_HashAggregate,0.0014733697,0.9709039548022599,0.1680995171957513,0.0,1.0,1.0,1.0,1.0 -47,sqlOp_BroadcastExchange,0.0011297737,0.6274011299435028,0.4835648944825208,0.0,0.0,1.0,1.0,1.0 -48,sqlOp_Expand,0.0009254089,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 -49,diskBytesSpilled_mean,0.00091879274,2012928.3715561333,13605693.750735264,0.0,0.0,0.0,0.0,374126772.9802514 -50,sqlOp_Window,0.0006066229,0.08870056497175141,0.2843512182402978,0.0,0.0,0.0,0.0,1.0 -51,sqlOp_Subquery,0.00060174114,0.16638418079096046,0.3724777539299263,0.0,0.0,0.0,0.0,1.0 -52,sqlOp_Filter,0.00053117686,0.9059322033898305,0.2919642550557146,0.0,1.0,1.0,1.0,1.0 -53,sqlOp_BroadcastHashJoin,0.00048093367,0.9096045197740112,0.28678802468260717,0.0,1.0,1.0,1.0,1.0 -54,platform_onprem,0.00047731653,0.42994350282485877,0.49513769985720407,0.0,0.0,0.0,1.0,1.0 -55,sqlOp_RunningWindowFunction,0.0004683478,0.011016949152542373,0.1043966220714521,0.0,0.0,0.0,0.0,1.0 -56,sqlOp_Scan orc ,0.00044496782,0.0768361581920904,0.26636892844122,0.0,0.0,0.0,0.0,1.0 -57,diskBytesSpilledRatio,0.0004446417,0.07709541615220665,0.425212311809422,0.0,0.0,0.0,0.0,7.517807102047801 -58,sqlOp_AQEShuffleRead,0.0004128795,0.8387005649717514,0.3678588769077452,0.0,1.0,1.0,1.0,1.0 -59,sqlOp_GenerateBloomFilter,0.00035020744,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 -60,output_bytesWrittenRatio,0.00034191555,4.878133481863412e-06,0.00011009204907379927,0.0,0.0,0.0,0.0,0.004454741188373785 -61,sqlOp_ObjectHashAggregate,0.00033151326,0.06016949152542373,0.23783419152332647,0.0,0.0,0.0,0.0,1.0 -62,output_bytesWritten_mean,0.0003017444,18.163857860424017,514.6579916903138,0.0,0.0,0.0,0.0,25987.92 -63,sqlOp_OutputAdapter,0.0003016191,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 -64,sqlOp_BroadcastNestedLoopJoin,0.00019190965,0.0384180790960452,0.19223050973607964,0.0,0.0,0.0,0.0,1.0 -65,sqlOp_Scan parquet ,0.00017240988,0.8997175141242938,0.3004187138796259,0.0,1.0,1.0,1.0,1.0 -66,sqlOp_BatchEvalPython,0.00015781133,0.005367231638418079,0.07307484463606036,0.0,0.0,0.0,0.0,1.0 -67,sqlOp_DeserializeToObject,0.00011716436,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -68,memoryBytesSpilled_mean,8.455912e-05,7570388.018003908,50085507.71137506,0.0,0.0,0.0,0.0,874196883.2566372 -69,sqlOp_WindowGroupLimit,4.368235e-05,0.011864406779661017,0.10829106764028926,0.0,0.0,0.0,0.0,1.0 -70,platform_dataproc,2.9462633e-05,0.16751412429378532,0.3734870106579372,0.0,0.0,0.0,0.0,1.0 -71,sqlOp_HashAggregatePrefixGroupingSets,2.4900206e-05,0.009322033898305085,0.09611317867107047,0.0,0.0,0.0,0.0,1.0 -72,sqlOp_SubqueryOutputBroadcast,1.1342636e-05,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 -73,sqlOp_Project,0.0,0.9850282485875707,0.1214568451648366,0.0,1.0,1.0,1.0,1.0 -74,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_ReusedSort,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 -76,sqlOp_SerializeFromObject,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -77,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,sqlOp_Scan text ,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -83,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,maxOnHeapMem,0.2573325,12891675333.097458,7443156274.854312,455501414.0,8974132838.0,10119177830.0,10119177830.0,26104509235.0 +1,duration_max,0.116825044,19727.637005649718,132756.56647030966,15.0,1000.75,2626.5,6606.5,3183955.0 +2,executorCPUTime_mean,0.114755705,1600.0121232217436,8490.83160303521,3.875,196.08367473240125,438.1275815373426,1135.2854603729604,227947.84615384616 +3,scan_bw,0.096296355,991028.5544311537,1739237.4464631807,0.0,111648.31626667196,344807.51426898653,1052087.8925818277,17385294.707427993 +4,input_bytesRead_mean,0.0830621,19051772.3871117,29839180.431289956,3240.8,2282876.103862428,7538112.224328557,20442663.2280394,240185411.32144082 +5,sqlOp_SubqueryBroadcast,0.06529896,0.6070621468926554,0.48847220931507623,0.0,0.0,1.0,1.0,1.0 +6,cache_hit_ratio,0.05493743,0.18990465580425084,0.32957735575345715,0.0,0.0,0.0,0.30864069264631583,1.0 +7,sw_bytesWrittenRatio,0.029399674,0.7500153334529006,1.312331627655776,0.0,0.004156014806987636,0.20888208704072536,1.1450582662697832,19.10476372587304 +8,shuffle_write_bw,0.029113747,165536.28360480044,275668.61585567426,0.0,3445.5108454312553,83232.4505533861,213410.4162089666,4415434.0 +9,jvmGCTime_mean,0.027562521,37.70700489432971,112.05477273251788,0.0,2.6427434456928838,10.6552217453505,35.56270150609702,2226.977412731006 +10,peakExecutionMemory_max,0.026136443,514434918.05706215,1343644906.5355828,0.0,68478953.0,185118727.5,448760080.5,25501360736.0 +11,executorDeserializeCPUTime_mean,0.026075294,7.730842747906887,16.187183743296377,0.01225114854517611,1.5320794973899026,2.4677083333333334,6.553110708090206,245.76923076923077 +12,maxMem,0.025987305,21686267078.83305,13741098115.368979,455501414.0,10119177830.0,10119177830.0,26104509235.0,43359456460.0 +13,resultSize_max,0.020297138,3958608.534180791,11103446.031355042,1785.0,15101.25,54348.0,1857380.5,263794922.0 +14,sw_recordsWritten_sum,0.019516138,1307752725.568644,4224481965.3663344,0.0,139053.75,16308835.0,463500001.0,44204763473.0 +15,duration_mean,0.019102091,2226.988951208108,8813.182199843044,13.333333333333334,319.1502445988216,776.5181426587314,1918.973089868859,228041.3076923077 +16,executorDeserializeTime_mean,0.017916078,18.885789433104375,44.57893656009577,0.04395604395604396,2.56471764117941,4.46117303124429,10.272696053946055,475.6 +17,duration_min,0.017819798,53.92768361581921,116.25739909610253,0.0,11.0,16.0,39.0,1779.0 +18,sw_bytesWritten_mean,0.016554512,8701193.262865013,23683912.279130522,0.0,21461.232201626925,1031447.9115545701,6399900.097575081,314037255.60981447 +19,sr_localBytesReadRatio,0.016466133,0.4186158681422342,1.0093096602351777,0.0,0.0020370799199269708,0.06772948216509181,0.3323507501682036,16.779651636748564 +20,scan_time,0.014725033,1520882.0240112995,7956731.9533324735,0.0,2684.5,49494.0,516427.25,217849366.0 +21,numExecutors,0.01422912,6.029378531073446,3.1526548701636963,1.0,1.0,8.0,8.0,14.0 +22,duration_sum,0.011375834,4842764.72118644,20681577.709232815,40.0,51743.25,347622.0,2003098.25,515693115.0 +23,sr_localBytesRead_mean,0.010557457,2910251.7965611313,7109002.559062359,0.0,9821.28396762487,375550.6402773764,2249818.307875526,104601898.28278689 +24,executorRunTime_mean,0.010003867,2199.3590155298366,8810.870593599537,6.0,307.07921482662414,748.9813360530342,1864.906949267008,228020.38461538462 +25,sr_totalBytesReadRatio,0.009838409,0.806944875525602,1.3776145394257457,0.0,0.004556759536252387,0.2297007756287479,1.1873950080800997,19.18647529336742 +26,sw_writeTime_mean,0.00836185,65.46063250813955,282.56757535974504,0.0,4.31657853171011,17.049135220125788,45.60306478963834,6475.538414708389 +27,platform_databricks-aws,0.0077426797,0.08813559322033898,0.283532043314531,0.0,0.0,0.0,0.0,1.0 +28,output_recordsWritten_sum,0.0072907945,82.66638418079096,1380.6943192952579,0.0,0.0,0.0,0.0,28800.0 +29,input_recordsRead_sum,0.0072506084,4254947665.6189265,13496209064.983372,0.0,41728108.0,454111279.5,2835348935.0,230401504404.0 +30,resultSerializationTime_sum,0.0069350842,73.159604519774,251.12253258121743,0.0,0.0,8.0,49.0,7044.0 +31,sr_fetchWaitTime_mean,0.0064337407,53.589055836211855,852.5182710790448,0.0,0.0,0.0,0.199677245831092,26216.59561667383 +32,numTasks_sum,0.0063541057,1383.4225988700564,3662.17887830497,3.0,130.0,419.5,1177.0,86122.0 +33,sr_localBlocksFetched_sum,0.0061586583,44000.88644067797,208616.4344751978,0.0,145.0,1412.5,15105.5,3860026.0 +34,sqlOp_SortMergeJoin,0.00575969,0.4788135593220339,0.49962150606394323,0.0,0.0,0.0,1.0,1.0 +35,data_size,0.0055217906,354067441856.6409,1436943956289.1838,0.0,1947746994.0,25418067831.0,172467501151.5,22044104226233.0 +36,executorCores,0.0052096336,12.945762711864408,3.103634601050894,6.0,12.0,12.0,16.0,16.0 +37,sr_totalBytesRead_mean,0.0051096017,9202557.355776286,24510563.626570754,0.0,23858.06464995155,1237051.5861350573,7001522.360789239,314039311.06463194 +38,shuffle_read_bw,0.00477962,200340881.16569823,2736126536.080987,0.0,0.0,0.0,4142971.1076581576,107487374955.0 +39,sr_remoteBlocksFetched_sum,0.0047682696,165629.2963276836,948144.8162985918,0.0,0.0,368.0,45471.0,21016512.0 +40,sr_remoteBytesReadRatio,0.004553455,0.38832900738336773,0.9402463724732919,0.0,0.0,0.0002836891594572681,0.31458116619117393,16.78961797640585 +41,sr_remoteBytesRead_mean,0.0044714212,6292305.559215155,21016177.50230388,0.0,0.0,4620.1696005917165,2150364.996089182,274787384.05385995 +42,memoryBytesSpilledRatio,0.004234966,0.38330444729758456,2.2121805527381846,0.0,0.0,0.0,0.0,43.971493342592005 +43,sqlOp_Sort,0.003654605,0.581638418079096,0.49335983507473885,0.0,0.0,1.0,1.0,1.0 +44,sqlOp_TakeOrderedAndProject,0.0026442565,0.10423728813559321,0.3056112883826109,0.0,0.0,0.0,0.0,1.0 +45,output_bytesWrittenRatio,0.0023144677,4.878133481863412e-06,0.00011009204907379927,0.0,0.0,0.0,0.0,0.004454741188373785 +46,sqlOp_Expand,0.0010848013,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 +47,diskBytesSpilledRatio,0.0010659718,0.07709541615220665,0.425212311809422,0.0,0.0,0.0,0.0,7.517807102047801 +48,diskBytesSpilled_mean,0.0007492866,2012928.3715561333,13605693.750735264,0.0,0.0,0.0,0.0,374126772.9802514 +49,sqlOp_Window,0.0006744248,0.08870056497175141,0.2843512182402978,0.0,0.0,0.0,0.0,1.0 +50,sqlOp_HashAggregate,0.0006506306,0.9709039548022599,0.1680995171957513,0.0,1.0,1.0,1.0,1.0 +51,sqlOp_AQEShuffleRead,0.00061880174,0.8387005649717514,0.3678588769077452,0.0,1.0,1.0,1.0,1.0 +52,sqlOp_DeserializeToObject,0.0006076176,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 +53,sqlOp_Filter,0.0004885468,0.9059322033898305,0.2919642550557146,0.0,1.0,1.0,1.0,1.0 +54,sqlOp_GenerateBloomFilter,0.00042495588,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 +55,output_bytesWritten_mean,0.00042176832,18.163857860424017,514.6579916903138,0.0,0.0,0.0,0.0,25987.92 +56,sqlOp_BroadcastHashJoin,0.00031076177,0.9096045197740112,0.28678802468260717,0.0,1.0,1.0,1.0,1.0 +57,sqlOp_OutputAdapter,0.00029876328,0.006779661016949152,0.08207070077249075,0.0,0.0,0.0,0.0,1.0 +58,sqlOp_ObjectHashAggregate,0.00029279114,0.06016949152542373,0.23783419152332647,0.0,0.0,0.0,0.0,1.0 +59,sqlOp_RunningWindowFunction,0.0002772561,0.011016949152542373,0.1043966220714521,0.0,0.0,0.0,0.0,1.0 +60,sqlOp_BroadcastExchange,0.00025236784,0.6274011299435028,0.4835648944825208,0.0,0.0,1.0,1.0,1.0 +61,sqlOp_Subquery,0.0002474833,0.16638418079096046,0.3724777539299263,0.0,0.0,0.0,0.0,1.0 +62,platform_dataproc,0.0002451532,0.16751412429378532,0.3734870106579372,0.0,0.0,0.0,0.0,1.0 +63,memoryBytesSpilled_mean,0.0002162816,7570388.018003908,50085507.71137506,0.0,0.0,0.0,0.0,874196883.2566372 +64,sqlOp_BroadcastNestedLoopJoin,0.00012899202,0.0384180790960452,0.19223050973607964,0.0,0.0,0.0,0.0,1.0 +65,sqlOp_BatchEvalPython,0.00010320502,0.005367231638418079,0.07307484463606036,0.0,0.0,0.0,0.0,1.0 +66,sqlOp_Exchange,7.392883e-05,0.980225988700565,0.13924251056124148,0.0,1.0,1.0,1.0,1.0 +67,sqlOp_Scan parquet,6.862866e-05,0.8997175141242938,0.3004187138796259,0.0,1.0,1.0,1.0,1.0 +68,sqlOp_WindowGroupLimit,3.076295e-05,0.011864406779661017,0.10829106764028926,0.0,0.0,0.0,0.0,1.0 +69,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_ReusedSort,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 +71,sqlOp_SubqueryOutputBroadcast,0.0,0.000847457627118644,0.029102898519035535,0.0,0.0,0.0,0.0,1.0 +72,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_SerializeFromObject,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 +75,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Project,0.0,0.9850282485875707,0.1214568451648366,0.0,1.0,1.0,1.0,1.0 +77,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_LocalTableScan,0.0,0.0011299435028248588,0.03360038173899415,0.0,0.0,0.0,0.0,1.0 +85,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 86,sqlOp_MapElements,0.0,0.01807909604519774,0.13325636384843126,0.0,0.0,0.0,0.0,1.0 -87,sqlOp_LocalTableScan,0.0,0.0011299435028248588,0.03360038173899415,0.0,0.0,0.0,0.0,1.0 -88,executorOffHeap,0.0,8387.176271186441,15015.577753655323,0.0,0.0,0.0,0.0,35262.0 -89,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -92,executorMemory,0.0,21657.8790960452,12250.464509917423,1024.0,16384.0,16384.0,16384.0,47016.0 -93,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,maxOffHeapMem,0.0,8794591745.735594,15744974458.616884,0.0,0.0,0.0,0.0,36974886912.0 +87,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,platform_emr,0.0,0.08926553672316384,0.2851669203761615,0.0,0.0,0.0,0.0,1.0 +90,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +91,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 95,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,sqlOp_Exchange,0.0,0.980225988700565,0.13924251056124148,0.0,1.0,1.0,1.0,1.0 -97,platform_databricks-azure,0.0,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 -98,platform_emr,0.0,0.08926553672316384,0.2851669203761615,0.0,0.0,0.0,0.0,1.0 -99,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -100,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_ColumnarToRow,0.0,0.9765536723163842,0.1513375934616419,0.0,1.0,1.0,1.0,1.0 -102,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.000847457627118644,0.02910289851903554,0.0,0.0,0.0,0.0,1.0 +96,executorOffHeap,0.0,8387.176271186441,15015.577753655323,0.0,0.0,0.0,0.0,35262.0 +97,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,executorMemory,0.0,21657.8790960452,12250.464509917423,1024.0,16384.0,16384.0,16384.0,47016.0 +99,maxOffHeapMem,0.0,8794591745.735594,15744974458.616884,0.0,0.0,0.0,0.0,36974886912.0 +100,platform_databricks-azure,0.0,0.07542372881355933,0.2641111407370466,0.0,0.0,0.0,0.0,1.0 +101,platform_onprem,0.0,0.42994350282485877,0.49513769985720407,0.0,0.0,0.0,1.0,1.0 +102,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.009322033898305085,0.09611317867107047,0.0,0.0,0.0,0.0,1.0 +103,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_ColumnarToRow,0.0,0.9765536723163842,0.1513375934616419,0.0,1.0,1.0,1.0,1.0 +105,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +114,sqlOp_WindowSort,0.0,0.000847457627118644,0.02910289851903554,0.0,0.0,0.0,0.0,1.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg index bb4140578..787e26392 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0486664437","gamma":"0.00858792849","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0486664437","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.00858792849","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.654426217"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"3.6910433E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0486664437","gamma":"0.00858792849","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0486664437","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.00858792849","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.654426217"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"3.6910433E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json index bfe96460c..d833ef180 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[1.5103106E-2,-2.2944748E-1,4.0485805E-1,-5.2396727E-1,-5.360323E-2,7.115805E-1,-2.6379624E-1,-3.949695E-1,-4.2585783E-2,-1.771069E-1,8.511205E-2,4.611889E-1,1.0766281E0,2.0126725E-2,-6.011713E-1,-4.39699E-1,2.4540755E-3,-2.5400817E-1,-1.0922299E-1,2.1650563E-1,-2.3319207E-1,-1.3541527E-2,5.946551E-1,5.5333883E-2,1.7301071E-2,-1.22435205E-1,1.9814072E-2,-4.192214E-2,-1.1715452E-2,-2.594052E-1,-5.554998E-1,-3.1053467E-4,-3.0310825E-1,8.845983E-2,-1.6904932E-1,2.8078866E-1,6.388342E-2,-1.6651804E-2,-3.3995867E-2,7.99645E-3,-1.5311554E-1,6.501042E-1,6.8630027E-3,-1.1348052E-2,5.7094876E-2,-4.0573996E-2,-3.4288028E-1,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,-3.68585E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,-1.9150509E-1,7.322452E-3,1.7035466E-2,-6.4032795E-3,1.3852504E-1,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,45,47,-1,49,51,53,55,57,-1,59,-1,61,63,-1,-1,65,67,69,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8298021E1,6.0830874E0,1.536447E1,1.6471167E0,1.2999624E0,4.1651173E0,2.3110242E0,6.9953203E-1,0E0,1.810708E-1,1.5561198E0,2.0355868E0,4.665451E-1,8.22276E-1,8.244722E-1,5.031233E-1,0E0,2.1691906E-1,2.8696674E-1,2.3714864E-1,2.271381E-1,2.2374773E-1,5.371609E-1,0E0,0E0,2.3201612E-1,0E0,0E0,0E0,2.418254E-1,3.252077E-2,0E0,1.0391843E-1,5.7198662E-2,4.2966306E-2,1.09799504E-1,1.495303E-1,0E0,2.0135637E-2,0E0,2.2688761E-2,2.5058937E-1,0E0,0E0,6.3281864E-2,1.3045962E-2,8.805108E-2,0E0,0E0,0E0,2.4855494E-2,0E0,0E0,0E0,2.7623773E-2,0E0,0E0,0E0,6.379056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,34,34,35,35,36,36,38,38,40,40,41,41,44,44,45,45,46,46,50,50,54,54,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,46,48,-1,50,52,54,56,58,-1,60,-1,62,64,-1,-1,66,68,70,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.039435E-3,6.627576E7,6.8652206E9,4.5570032E2,4.7162656E3,4E0,4.1211517E1,-4.2585783E-2,1.1383354E6,9.518453E6,1E0,7.317201E0,1.5312085E0,7.588906E-2,5.1147053E5,2.4540755E-3,4.7812922E5,1.630886E6,8.729897E0,1.2534044E3,8.78E2,2.709E4,5.5333883E-2,1.7301071E-2,6.7995167E0,1.9814072E-2,-4.192214E-2,-1.1715452E-2,2.77E2,1E0,-3.1053467E-4,2.8880866E0,4.35E2,2.6374866E1,5.716463E2,2.831E3,-1.6651804E-2,9.17E2,7.99645E-3,3.349E4,1.5145266E6,6.8630027E-3,-1.1348052E-2,6.695E3,9.513E3,1.4377E4,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,1.0491824E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,4.2658337E2,7.322452E-3,1.7035466E-2,-6.4032795E-3,5.586E3,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"split_indices":[53,39,46,5,53,53,8,59,0,52,46,102,54,58,39,29,0,52,46,55,4,0,9,0,0,55,0,0,0,0,105,0,57,0,59,53,2,0,0,0,10,49,0,0,2,9,9,0,0,0,40,0,0,0,53,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.17E2,7.3E1,4.3E1,7.4E1,5E1,2.3E1,3.3E1,1E1,3.9E1,3.5E1,3.1E1,1.9E1,1.3E1,1E1,3E1,3E0,1.7E1,2.2E1,2.5E1,1E1,7E0,2.4E1,1.7E1,2E0,1E1,3E0,5E0,5E0,1.3E1,1.7E1,3E0,1.4E1,5E0,1.7E1,1.7E1,8E0,6E0,4E0,3E0,4E0,2.1E1,3E0,6E0,4E0,4E0,9E0,1.4E1,3E0,5E0,9E0,2E0,3E0,3E0,1.4E1,7E0,1E1,2E0,6E0,2E0,2E0,2E0,2E0,5E0,1.6E1,2E0,2E0,2E0,2E0,5E0,4E0,2E0,7E0,1.2E1,2E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-3.3002432E-2,-1.7783551E-1,5.082772E-1,-2.7808794E-1,6.895006E-2,7.5295E-2,8.787938E-1,-1.8593931E-1,-9.442671E-1,3.6505783E-1,-2.5721067E-1,3.1757823E-1,-1.1269199E-1,4.4750738E-1,1.012557E0,-2.7686968E-1,-1.1995372E-3,-1.790164E-2,-4.942715E-2,1.0660491E-1,5.7554835E-1,3.059845E-2,-4.588169E-1,5.580259E-3,1.7479694E-2,-1.3313731E-2,4.1303214E-2,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,-2.2212821E-1,-4.4453266E-1,7.171647E-2,-2.8227565E-1,2.1667868E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,-7.6152295E-2,1.4846555E-2,-9.6043665E-3,-5.975643E-1,7.728357E-3,-3.7457045E-2,-2.4542458E-1,2.0091075E-3,-7.810127E-3,-2.595061E-2,-1.12920456E-1,1.1881216E-1,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,-1,-1,57,-1,59,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5220662E1,3.804525E0,6.511341E0,6.5156603E0,4.4399395E0,9.4671035E-1,8.160591E-1,1.6291182E0,1.6117287E-1,1.2442894E0,1.2905694E0,3.4778595E-2,3.0859464E-1,2.4943113E-2,1.920681E-1,4.978342E-1,6.9657713E-1,0E0,0E0,3.9868122E-1,7.959266E-1,3.1622946E-1,3.536806E-1,0E0,0E0,0E0,7.072824E-2,0E0,0E0,0E0,0E0,3.2309484E-1,2.9235983E-1,2.4819759E-1,3.8475174E-1,7.987356E-2,0E0,0E0,0E0,3.175994E-2,0E0,0E0,1.1517382E-1,0E0,1.07335895E-2,1.10358E-1,0E0,0E0,0E0,1.0522947E-2,4.5623663E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,26,26,31,31,32,32,33,33,34,34,35,35,39,39,42,42,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,-1,-1,58,-1,60,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5085194E0,1.2871602E3,1.020266E3,8.260109E9,6.971004E7,2.3238E4,1.9107767E0,4.684559E2,2.6E1,5.5326223E9,8.462096E11,5.380165E0,3.236715E0,5.8229775E3,2.8626094E5,1.03431056E6,5.5015685E9,-1.790164E-2,-4.942715E-2,6.4348924E-1,5.6759316E7,5.4827127E0,1.5214152E5,5.580259E-3,1.7479694E-2,-1.3313731E-2,5.884028E0,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,1.1556807E0,5.7625353E-1,7.1568984E7,2.533331E6,4.1808826E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,8.307621E7,1.4846555E-2,-9.6043665E-3,2.012E3,7.728357E-3,1.009E3,5.1146146E-2,2.0091075E-3,-7.810127E-3,-2.595061E-2,4.5E1,7.384995E4,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"split_indices":[43,53,53,5,46,9,43,53,3,5,32,54,57,4,29,29,5,0,0,28,46,58,34,0,0,0,54,0,0,0,0,43,28,7,1,28,0,0,0,46,0,0,0,0,0,28,0,0,0,3,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.92E2,1.52E2,4E1,1.08E2,4.4E1,1.9E1,2.1E1,9.6E1,1.2E1,2.3E1,2.1E1,8E0,1.1E1,6E0,1.5E1,6.4E1,3.2E1,2E0,1E1,1.1E1,1.2E1,9E0,1.2E1,2E0,6E0,5E0,6E0,4E0,2E0,2E0,1.3E1,5E1,1.4E1,2.6E1,6E0,8E0,3E0,9E0,3E0,7E0,2E0,5E0,7E0,2E0,4E0,4.6E1,4E0,4E0,1E1,5E0,2.1E1,4E0,2E0,5E0,3E0,4E0,3E0,2E0,5E0,2E0,2E0,6E0,4E1,3E0,2E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-7.2186072E-3,-1.7554772E-1,4.6836394E-1,-1.3343666E-1,-7.98081E-1,4.8964757E-2,7.305873E-1,-2.4706829E-1,1.5109251E-1,-1.1278819E-2,-9.1069216E-1,-2.2222637E-1,2.0841512E-1,5.310632E-1,1.0273924E0,-2.0253772E-1,-7.193482E-1,8.116926E-2,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,2.4997413E-1,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,-4.6462825E-1,-1.3208786E-1,-1.10341525E-2,-4.028151E-2,-1.6114761E-1,2.4269797E-1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,-5.287513E-1,-9.0207495E-2,-3.1034762E-1,1.1962035E-2,-2.5665852E-1,3.4205204E-1,-1.1185924E-2,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7454023E1,4.1142073E0,6.232873E0,4.9517193E0,4.2668343E-1,1.0408703E0,1.5270805E0,2.1466722E0,2.0214696E0,0E0,3.1103182E-1,1.4763036E-1,1.022948E-1,7.396126E-2,9.1228485E-3,1.8035164E0,2.927456E-1,1.6438966E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2034745E-2,0E0,0E0,0E0,0E0,2.2022247E-1,5.7671654E-1,0E0,0E0,2.841458E-1,6.34436E-1,0E0,0E0,0E0,1.3254166E-2,7.471074E-1,2.088238E-1,2.8850803E-1,9.1903985E-2,4.5164657E-1,1.8377072E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24,29,29,30,30,33,33,34,34,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,7.7567725E2,1.0523087E3,4.14E2,2.6263257E2,1.5605432E9,6.8652206E9,2.3274304E2,-1.1278819E-2,3.7284137E2,2.5546576E4,2.5086822E8,6.8734935E6,3.4456624E7,1.142E3,1.9E1,7.157224E0,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,1.630886E6,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,2.7373068E7,3.0143394E7,-1.10341525E-2,-4.028151E-2,2E0,1.0943299E1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,5.9215684E0,7.54579E5,1.5641167E5,6.802721E-3,1.0180757E0,2.4794E4,9.875E3,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"split_indices":[43,46,53,53,0,53,7,5,57,0,59,34,7,48,48,2,3,55,0,0,0,0,0,0,46,0,0,0,0,5,46,0,0,8,55,0,0,0,57,29,34,58,40,9,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.6E2,5.6E1,1.51E2,9E0,2.2E1,3.4E1,1.08E2,4.3E1,2E0,7E0,8E0,1.4E1,2.2E1,1.2E1,1E2,8E0,4E1,3E0,5E0,2E0,3E0,5E0,3E0,1.1E1,1.8E1,4E0,1E1,2E0,2E1,8E1,2E0,6E0,1.6E1,2.4E1,5E0,6E0,5E0,1.5E1,6.6E1,1.4E1,6E0,1E1,1.7E1,7E0,1.3E1,2E0,4.6E1,2E1,1.2E1,2E0,3E0,3E0,6E0,4E0,1.4E1,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-9.860434E-3,-1.3550225E-1,4.1085625E-1,-4.535459E-1,-6.382216E-2,2.3670215E-2,7.524566E-1,-5.6793636E-1,9.8671935E-2,-9.873285E-2,4.3589786E-1,2.2890316E-1,-1.3927497E-1,9.374098E-1,4.9539495E-1,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,-4.404742E-2,-6.1913097E-1,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,-2.5028974E-1,-2.472767E-2,1.5874594E-2,4.9255155E-2,5.498805E-1,6.3501373E-3,-1.7045158E-1,8.096087E-2,-4.8548445E-2,-3.0521184E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,-6.917094E-2,1.6786462E-2,3.2274876E-2,-2.1812399E-1,-5.0437325E-3,1.5704603E-1,-2.1648994E-2,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,39,-1,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1261909E1,3.7098274E0,6.463071E0,1.9510937E0,2.3981898E0,8.344817E-1,7.7176666E-1,1.549511E-1,3.8158774E-1,3.5869198E0,2.744478E-1,1.8382758E-1,1.7184111E-1,3.0896282E-1,1.867218E-1,0E0,0E0,0E0,0E0,1.8467818E0,1.121139E0,0E0,0E0,0E0,0E0,1.3394743E-2,3.954195E-2,0E0,0E0,2.6690483E-2,0E0,4.5625246E-1,4.6401808E-1,0E0,7.3043644E-2,0E0,0E0,0E0,1.0194156E-2,0E0,0E0,3.1877828E-1,1.343909E-1,8.3303446E-1,9.282285E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,25,25,26,26,29,29,31,31,32,32,34,34,38,38,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,40,-1,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.7575762E4,1.1382212E3,2.0199957E3,1.7730763E12,1.5593E4,8.812601E-1,4.6490747E2,3.20084E6,1.775894E10,4.6673E4,8.147317E0,3.2851852E2,4.558E3,1.4118051E7,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,5.470729E2,8.92851E6,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,6.314915E-1,4.1546512E2,1.5874594E-2,4.9255155E-2,3.76835E7,6.3501373E-3,3.9E1,3.291238E7,-4.8548445E-2,6.881048E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,8.27E2,1.6786462E-2,3.2274876E-2,4.7095413E2,6.9327216E0,1.4154823E-1,2.4654E4,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"split_indices":[43,52,4,4,32,9,58,53,7,5,10,54,4,2,1,0,0,0,0,53,1,0,0,0,0,39,4,0,0,49,0,3,46,0,35,0,0,0,0,0,0,4,54,39,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.63E2,4.8E1,2.9E1,1.34E2,2.3E1,2.5E1,2.4E1,5E0,1.26E2,8E0,1E1,1.3E1,1.3E1,1.2E1,1.4E1,1E1,3E0,2E0,1.15E2,1.1E1,4E0,4E0,8E0,2E0,6E0,7E0,2E0,1.1E1,1E1,2E0,5.7E1,5.8E1,4E0,7E0,4E0,2E0,2E0,5E0,5E0,5E0,4.4E1,1.3E1,3.3E1,2.5E1,5E0,2E0,3E0,2E0,1.2E1,3.2E1,7E0,6E0,1.9E1,1.4E1,2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.037411E-2,-1.7082255E-1,4.0125197E-1,-1.368692E-1,-4.7962084E-2,1.3270798E-1,7.9803574E-1,-2.2494297E-1,7.1677774E-2,2.673719E-1,-1.0539308E-1,1.9987479E-2,4.1916054E-2,-5.089811E-1,-1.4352345E-1,1.7057969E-1,-1.6686903E-2,1.1773016E-1,1.8603355E-2,-1.25300465E-2,-9.196732E-3,-3.748573E-1,-3.2896925E-2,-2.6696217E-1,-5.671464E-2,6.511845E-2,4.1393185E-1,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,-4.2942005E-1,-5.2706E-2,-3.20558E-1,5.7432473E-2,-1.8276703E-1,2.8034353E-1,-4.025757E-2,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,-1,27,-1,-1,29,31,-1,33,35,37,39,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3262604E1,4.125049E0,5.7545595E0,2.7528837E0,0E0,1.1183422E0,1.8529606E-1,2.355206E0,1.903528E0,3.3322942E-1,1.8780968E-1,0E0,0E0,2.6173544E-1,8.7425184E-1,9.226215E-1,0E0,4.9472928E-2,0E0,0E0,1.1085327E-1,8.237338E-2,0E0,3.7353468E-1,7.2992265E-1,6.2363416E-1,1.9650996E-1,0E0,0E0,0E0,0E0,0E0,7.688856E-2,1.9526544E-1,2.2509766E-1,3.0900213E-1,1.7987198E-1,2.4989247E-1,3.8682532E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,20,20,21,21,23,23,24,24,25,25,26,26,32,32,33,33,34,34,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,-1,28,-1,-1,30,32,-1,34,36,38,40,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,1.3453049E3,1.0648859E3,-4.7962084E-2,2.4608E4,2.7350403E5,1.7575762E4,9.820072E7,5.433479E2,2.5546576E4,1.9987479E-2,4.1916054E-2,1.3900659E6,2.802166E2,1.6E1,-1.6686903E-2,2.668E3,1.8603355E-2,-1.25300465E-2,1.9E1,2.9270105E2,-3.2896925E-2,2.124077E1,9.222491E0,3.10214E5,2.4986116E7,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,6.1E1,1.5497297E1,3.079376E8,6.735744E1,3.1E1,5.1E1,6.8125E0,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"split_indices":[43,46,53,53,0,9,29,52,46,53,34,0,0,29,53,8,0,2,0,0,3,34,0,59,55,30,49,0,0,0,0,0,3,59,5,59,3,3,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.53E2,5.4E1,1.48E2,5E0,3.3E1,2.1E1,1.04E2,4.4E1,2.1E1,1.2E1,4E0,1.7E1,2.2E1,8.2E1,3.6E1,8E0,1E1,1.1E1,4E0,8E0,1.4E1,8E0,3.3E1,4.9E1,2.6E1,1E1,4E0,6E0,3E0,5E0,4E0,1E1,7E0,2.6E1,2.6E1,2.3E1,8E0,1.8E1,8E0,2E0,8E0,2E0,3E0,4E0,1.3E1,1.3E1,2.1E1,5E0,9E0,1.4E1,6E0,2E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.8539885E-2,-1.315011E-1,4.7729447E-1,-3.645807E-1,-4.856526E-2,3.3575363E-2,7.048596E-1,-4.006475E-1,1.0404514E-2,-1.2890418E-1,1.0923791E-1,2.1161272E-1,-2.1542291E-1,4.2066466E-2,7.948288E-1,-5.199407E-1,-2.0412306E-1,-1.0585207E-1,-3.185486E-2,-2.5976712E-1,1.9807549E-1,1.8924989E-3,3.4906948E-1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,-5.954765E-1,-2.6221934E-1,-4.2201504E-1,1.2239022E-2,-1.8437941E-1,-8.044045E-3,-5.61047E-2,-2.8251013E-2,2.2316225E-2,1.2032818E-1,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,-1.095318E-1,-2.132905E-1,-1.3484026E-2,1.0698679E-1,-1.6088222E-1,-6.4098774E-3,3.0856705E-3,3.494343E-1,6.278915E-2,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.39668255E1,2.9398837E0,5.0250864E0,9.200578E-1,1.4593433E0,8.4354407E-1,1.9343872E0,7.847419E-1,0E0,8.516334E-1,1.3299325E0,2.4759096E-1,8.105555E-2,1.5741335E-1,1.7285347E-1,2.9159975E-1,7.59462E-1,5.668815E-1,0E0,4.9000573E-1,5.9287274E-1,0E0,1.3428587E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.4581165E-2,1.9908011E-2,1.5304077E-1,2.1783859E-1,1.9969559E-1,6.1496985E-1,6.256449E-2,0E0,0E0,3.2080543E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8989278E-2,1.5572238E-1,2.168184E-2,1.3415802E-1,2.2903234E-1,0E0,0E0,5.3615987E-2,2.547765E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,48,48,49,49,50,50,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.130317E8,6.794883E4,1E0,4.5342876E3,6.787619E2,9.820072E7,3.788E3,1.4862166E1,1.0404514E-2,4.462437E7,7.443E3,3.5073215E3,1.2895599E4,6.09E2,4.2414474E0,6.1E1,5.0548022E2,2.98022E2,-3.185486E-2,9.64E2,1.20171E5,1.8924989E-3,5.1E1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,5.0446276E7,1.0804272E3,1.0363E4,4.1749332E4,2.0890524E6,7.6420575E6,1.1484E4,-2.8251013E-2,2.2316225E-2,1.2117E4,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,1.7488E4,2.124077E1,4.35E2,1.725E3,6.9789816E7,-6.4098774E-3,3.0856705E-3,4E0,6.1108776E7,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"split_indices":[7,51,102,4,53,46,2,59,0,46,9,53,4,0,58,3,4,53,0,0,30,0,3,0,0,0,0,0,0,46,34,9,33,51,46,2,0,0,9,0,0,0,0,0,0,0,0,0,9,59,0,2,12,0,0,8,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.52E2,4.9E1,3.9E1,1.13E2,1.7E1,3.2E1,3.7E1,2E0,7.5E1,3.8E1,1E1,7E0,4E0,2.8E1,2.2E1,1.5E1,7.3E1,2E0,7E0,3.1E1,5E0,5E0,3E0,4E0,2E0,2E0,2.4E1,4E0,1.6E1,6E0,7E0,8E0,4E1,3.3E1,5E0,2E0,6E0,2.5E1,2E0,3E0,1.1E1,5E0,3E0,3E0,2E0,5E0,3E0,5E0,3.4E1,6E0,1.9E1,1.4E1,3E0,2E0,4E0,2.1E1,3E0,2E0,8E0,2.6E1,2E0,4E0,2E0,1.7E1,5E0,9E0,2E0,2E0,1.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-2.776328E-2,-1.2117824E-1,3.86292E-1,-5.909288E-2,-5.795699E-1,5.5683386E-1,-2.6300025E-1,-2.0977455E-1,9.621003E-2,-8.023927E-1,-1.1718366E-1,2.3014233E-1,6.94869E-1,-1.3781548E-3,-3.7562144E-1,-3.571755E-1,-1.0406611E-1,1.914385E-1,-8.06779E-2,-2.3914319E-2,-4.741177E-2,9.839836E-3,-3.492453E-1,3.794242E-1,-6.1298814E-3,3.9063934E-2,3.491601E-1,-2.482055E-2,-3.466254E-3,-4.0868425E-1,-5.727207E-2,-1.4600524E-1,2.0078115E-2,7.405951E-2,3.1055027E-1,-1.1359027E-1,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,-4.7098124E-1,-1.5451024E-1,2.9516693E-3,-6.394594E-3,4.4541497E-2,-1.712155E-1,7.7366033E-3,-5.9316065E-2,1.5788026E-1,-6.221433E-2,3.575082E-1,-1.2339829E-3,-7.365338E-2,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,61,63,-1,65,67,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.401466E0,5.003068E0,4.575707E0,3.694187E0,2.0696063E0,1.2831182E0,2.2723442E-1,1.2093525E0,1.3293368E0,2.6940823E-1,6.6352755E-1,6.3254225E-1,5.397568E-1,0E0,2.0912564E-1,4.8636627E-1,2.528435E-1,6.9004285E-1,3.1393635E-1,0E0,0E0,0E0,6.1558306E-2,2.0225549E-1,0E0,0E0,7.9099655E-2,0E0,0E0,3.7058783E-1,6.041512E-2,1.8056059E-1,1.5278538E-1,3.1823194E-1,4.0340137E-1,1.3923776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.6962032E-2,1.4837602E-1,0E0,0E0,2.0146374E-2,1.326319E-1,0E0,4.3555457E-2,1.6426831E-1,1.09528095E-1,6.0819864E-2,0E0,7.7050425E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,22,22,23,23,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,62,64,-1,66,68,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0105532E3,7.6599895E9,1.2055723E8,4.684559E2,1.5461028E12,8.601656E6,1.3611247E5,1.822764E4,2.2971932E7,3.6509445E1,2.5E1,2.4716E4,5.9283892E7,-1.3781548E-3,1.3382495E8,1E0,2.0806985E-1,7.7567725E2,7.4297624E0,-2.3914319E-2,-4.741177E-2,9.839836E-3,1.776E3,5.1E1,-6.1298814E-3,3.9063934E-2,1.046E4,-2.482055E-2,-3.466254E-3,1.01978E5,1.685E3,4.914413E-3,5.1E2,6.2742615E0,2.8314E4,8.539363E7,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,7.774228E-2,7.359468E8,2.9516693E-3,-6.394594E-3,3.76E2,5.2560944E1,7.7366033E-3,3.892E3,2.7883E4,4.4589956E5,7.3586698E0,-1.2339829E-3,1E0,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"split_indices":[53,5,46,53,32,1,34,34,46,57,3,2,46,0,33,105,39,53,54,0,0,0,0,3,0,0,9,0,0,30,2,39,0,54,9,46,0,0,0,0,0,0,0,58,5,0,0,0,59,0,11,9,29,54,0,84,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.76E2,3.9E1,1.56E2,2E1,3.1E1,8E0,7.9E1,7.7E1,1.3E1,7E0,1E1,2.1E1,3E0,5E0,3.2E1,4.7E1,5E1,2.7E1,6E0,7E0,3E0,4E0,7E0,3E0,1.5E1,6E0,3E0,2E0,2.7E1,5E0,3.5E1,1.2E1,2.6E1,2.4E1,2.5E1,2E0,2E0,2E0,5E0,2E0,3E0,3E0,2.1E1,6E0,2E0,3E0,4E0,3.1E1,4E0,8E0,1.6E1,1E1,2.1E1,3E0,2.1E1,4E0,1.9E1,2E0,2E0,4E0,2E0,2E0,2.3E1,8E0,3E0,5E0,1.4E1,2E0,4E0,6E0,1.8E1,3E0,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[7.874234E-3,-1.0717477E-1,4.6521157E-1,-1.982936E-1,9.943495E-2,1.02683544E-1,7.759888E-1,-1.3372196E-1,-6.0561585E-1,2.801086E-1,-2.3861834E-1,-1.4555064E-1,1.8935364E-1,4.1564103E-2,4.7552985E-1,-1.7659279E-2,-9.404589E-2,-7.024602E-3,-6.612021E-1,1.8482257E-1,3.9656345E-2,-7.397643E-2,-5.8301306E-1,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.4021983E-1,9.948502E-3,2.6430182E-2,-1.4144325E-1,2.806465E-2,-2.1147303E-2,-3.7808225E-2,2.9104125E-1,-3.4760606E-2,5.0945976E-3,-1.12476826E-1,-1.1272882E-2,-3.41051E-2,9.7476035E-2,1.4251037E-2,-8.572106E-2,-2.9373407E-1,9.719249E-2,-1.7345269E-1,5.5071485E-4,3.198904E-1,-1.1351033E-2,3.5261977E-2,-1.3395034E-1,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,35,37,-1,-1,-1,39,-1,-1,41,43,-1,-1,45,47,-1,49,-1,-1,51,-1,53,55,57,59,-1,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1257111E1,3.2389693E0,4.7649326E0,3.0488906E0,3.303937E0,4.7936463E-1,1.3339043E-1,9.1138387E-1,3.148079E-1,1.6327975E0,1.0342052E0,4.6653435E-2,1.7866707E-1,0E0,1.7190337E-2,0E0,5.248641E-1,0E0,2.9516697E-2,7.331557E-1,0E0,1.08073495E-1,8.598232E-2,0E0,0E0,0E0,6.9084704E-2,0E0,0E0,5.2640927E-1,3.790502E-1,0E0,0E0,1.658467E-1,1.6110414E-1,0E0,1.4867976E-2,0E0,0E0,1.5488528E-2,0E0,2.9356018E-1,1.905942E-2,1.5623116E-1,6.481768E-2,0E0,1.2719929E-1,0E0,4.8637956E-2,9.308532E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,18,18,19,19,21,21,22,22,26,26,29,29,30,30,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,46,46,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,36,38,-1,-1,-1,40,-1,-1,42,44,-1,-1,46,48,-1,50,-1,-1,52,-1,54,56,58,60,-1,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7987691E0,1.0648859E3,1.5262272E3,5.0446276E7,7.294459E7,1.0437421E8,2.98476E0,1.295E3,6.856226E-1,6.2150537E3,2.09E3,7.44E2,1.5675428E5,4.1564103E-2,3.145611E6,-1.7659279E-2,5.3287085E2,-7.024602E-3,9.180692E7,2.249985E5,3.9656345E-2,1.1341179E3,1.4233672E8,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.398624E8,9.948502E-3,2.6430182E-2,6.7841024E10,3.0143394E7,-2.1147303E-2,-3.7808225E-2,7.91144E5,2.3276744E5,5.0945976E-3,3.7387386E-1,-1.1272882E-2,-3.41051E-2,1.144E3,1.4251037E-2,6.0130353E0,7.273E3,9.4390506E5,2.9673992E9,5.5071485E-4,1E0,-1.1351033E-2,2.6387034E5,9.9764734E4,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"split_indices":[43,53,4,46,46,7,58,2,28,4,0,0,29,0,1,0,53,0,7,34,0,34,46,0,0,0,7,0,0,32,46,0,0,1,34,0,58,0,0,0,0,54,9,33,5,0,102,0,29,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.7E2,4.2E1,1.18E2,5.2E1,2E1,2.2E1,1.03E2,1.5E1,3.4E1,1.8E1,5E0,1.5E1,1.6E1,6E0,1.4E1,8.9E1,2E0,1.3E1,3E1,4E0,1.3E1,5E0,3E0,2E0,3E0,1.2E1,2E0,4E0,6.4E1,2.5E1,6E0,7E0,2E1,1E1,2E0,1.1E1,2E0,3E0,4E0,8E0,4.8E1,1.6E1,1.9E1,6E0,2E0,1.8E1,2E0,8E0,8E0,3E0,2E0,2E0,1.6E1,3.2E1,2E0,1.4E1,4E0,1.5E1,3E0,3E0,3E0,1.5E1,3E0,5E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.3249438E-2,-1.2597813E-1,3.6839122E-1,-3.741495E-1,-6.773875E-2,5.7275885E-1,-2.5949249E-2,-4.510603E-1,-5.7005443E-2,-1.6321269E-1,4.9919948E-2,2.9108667E-1,7.353367E-1,1.081298E-1,-2.378517E-2,-4.51294E-3,-4.978342E-1,1.5801355E-3,-4.8320345E-3,-1.921461E-2,-1.2809235E-1,1.7340758E-1,-1.6564597E-1,3.5022026E-1,-1.1079086E-3,4.0072575E-2,4.0726957E-1,-5.00003E-2,3.2730332E-1,-8.940148E-3,-5.212322E-1,-7.0953816E-3,-1.7170624E-1,2.1817629E-1,-2.174709E-2,-2.512538E-1,1.0451725E-1,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,4.3051414E-2,1.952225E-2,5.8277114E-3,-1.2334295E-2,-5.556018E-1,-1.2370801E-1,4.7271628E-2,-9.585648E-2,-2.4668616E-1,3.1131923E-2,2.7143374E-1,6.132134E-4,-4.048703E-3,-1.6482335E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,65,67,69,71,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.865889E0,2.272738E0,3.8026128E0,7.0604134E-1,1.4684818E0,1.2055569E0,1.1077032E0,3.5608482E-1,2.9706666E-2,5.3508985E-1,1.5992881E0,2.492423E-1,2.136507E-1,5.015763E-1,0E0,0E0,5.8598995E-2,0E0,0E0,0E0,3.3651924E-1,3.3674383E-1,5.350784E-1,1.20595574E-1,0E0,0E0,1.174857E-1,1.1404645E-1,4.39198E-2,0E0,3.405571E-2,1.1996572E-1,2.4365234E-1,3.003564E-1,1.7932303E-2,2.712661E-1,2.0998505E-1,0E0,0E0,0E0,0E0,0E0,1.23745585E-2,0E0,0E0,0E0,3.083849E-2,2.4845354E-2,9.429864E-2,2.3359656E-1,2.8410673E-2,1.0068264E-1,1.6567254E-1,0E0,0E0,6.312257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,42,42,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,66,68,70,72,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.4862166E1,6.627576E7,1.7497415E-3,5.3287085E2,3.5073215E3,5.7006E4,3.7453184E-3,9.096081E7,9.4418E4,3.1821228E7,5.2E1,1.0096877E8,1.0252E4,-2.378517E-2,-4.51294E-3,6.519E3,1.5801355E-3,-4.8320345E-3,-1.921461E-2,4.06E2,2.8314E4,4.4E1,1.075839E9,-1.1079086E-3,4.0072575E-2,1.5605432E9,3.9880952E-1,1E0,-8.940148E-3,2.0521739E0,1.369E3,9.258696E0,5.716463E2,1.3717402E0,7.6599895E9,1.2633E4,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,2E0,1.952225E-2,5.8277114E-3,-1.2334295E-2,1.0595825E3,1.36135E5,9.087618E6,1.1947E4,2.4734788E7,8.5093097E2,5.0691235E6,6.132134E-4,-4.048703E-3,1.8159722E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"split_indices":[53,59,46,40,53,53,10,59,7,1,46,3,52,9,0,0,9,0,0,0,0,9,3,7,0,0,7,58,8,0,57,2,55,53,43,5,2,0,0,0,0,0,8,0,0,0,34,1,33,9,12,4,46,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.58E2,4.6E1,2.9E1,1.29E2,3E1,1.6E1,2.3E1,6E0,7.1E1,5.8E1,1.2E1,1.8E1,1.3E1,3E0,3E0,2E1,2E0,4E0,8E0,6.3E1,3.7E1,2.1E1,1E1,2E0,1.3E1,5E0,8E0,5E0,2E0,1.8E1,1.7E1,4.6E1,3E1,7E0,1.6E1,5E0,8E0,2E0,2E0,3E0,3E0,5E0,3E0,2E0,3E0,1.5E1,5E0,1.2E1,2.4E1,2.2E1,7E0,2.3E1,5E0,2E0,1.3E1,3E0,3E0,2E0,3E0,2E0,9E0,6E0,3E0,2E0,9E0,3E0,3E0,2.1E1,7E0,1.5E1,5E0,2E0,7E0,1.6E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[6.0231384E-2,-6.6708066E-2,4.1548002E-1,-1.802126E-1,9.4408296E-2,6.121976E-1,4.159176E-2,-1.3381815E-1,-4.551602E-1,-2.3027116E-1,1.6258606E-1,2.7157372E-1,6.904492E-1,1.487131E-1,-1.9334765E-2,-3.371152E-1,-7.73753E-2,-1.0917176E-2,-2.8615614E-2,-1.011948E-1,-2.631605E-2,7.940243E-2,3.767569E-1,1.6384088E-2,2.4758964E-3,8.306936E-1,4.7551715E-1,3.2327557E-1,1.1648051E-2,-9.826521E-3,-2.175756E-2,-1.1204586E-1,1.6198908E-1,-1.3703959E-1,7.871614E-4,1.0334385E-1,-7.2968313E-3,4.7740215E-1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,-8.063122E-2,5.84258E-3,-3.1697118E-1,-6.857089E-2,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.4921291E-1,-1.0298919E-2,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,-1,35,37,-1,-1,39,41,43,45,-1,-1,47,49,51,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,57,-1,59,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.911444E0,2.6869793E0,3.7991571E0,1.0361109E0,1.3877299E0,7.1874046E-1,9.523439E-1,8.3451545E-1,2.3604941E-1,3.861003E-1,8.7867534E-1,9.7789764E-2,4.188528E-1,3.7905708E-1,0E0,1.4759302E-1,5.16084E-1,0E0,0E0,4.008031E-2,0E0,2.2414063E-1,3.420025E-1,0E0,0E0,4.281807E-2,4.2568207E-2,1.4053464E-1,1.0971589E-1,0E0,0E0,4.504485E-1,6.2172145E-2,1.6357675E-2,0E0,1.8398082E-1,0E0,1.7374277E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1939041E-2,0E0,2.7805746E-2,2.6995105E-1,0E0,0E0,0E0,0E0,2.1720147E-1,1.2581149E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,37,37,45,45,47,47,48,48,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,-1,36,38,-1,-1,40,42,44,46,-1,-1,48,50,52,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,58,-1,60,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0601392E3,2.1836595E8,6.627576E7,8.93182E5,1E0,1.2737E4,5.7006E4,1.042E3,4E0,7.6499896E9,1.068371E3,3.5E1,1.5213319E0,1E0,-1.9334765E-2,8.601996E-1,1E0,-1.0917176E-2,-2.8615614E-2,2.3276744E5,-2.631605E-2,4.759155E1,7.9762895E6,1.6384088E-2,2.4758964E-3,1.929E3,9.299267E9,3.6392304E-1,6.331E3,-9.826521E-3,-2.175756E-2,1.5497297E1,6.98247E5,1.1534339E9,7.871614E-4,7.183439E0,-7.2968313E-3,8.383825E1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,1.181E3,5.84258E-3,3.5E1,5.974511E0,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.8613E4,2.770862E8,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"split_indices":[53,7,46,30,102,2,10,2,8,5,53,3,58,8,0,28,89,0,0,34,0,57,46,0,0,0,12,28,9,0,0,59,29,7,0,54,0,59,0,0,0,0,0,0,0,0,0,10,54,0,0,0,0,9,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.45E2,5.1E1,8.5E1,6E1,3.3E1,1.8E1,7.4E1,1.1E1,1E1,5E1,7E0,2.6E1,1.5E1,3E0,1.5E1,5.9E1,5E0,6E0,8E0,2E0,3.7E1,1.3E1,5E0,2E0,1.4E1,1.2E1,6E0,9E0,8E0,7E0,5.2E1,7E0,6E0,2E0,3.4E1,3E0,9E0,4E0,8E0,6E0,1E1,2E0,2E0,4E0,5E0,4E0,8E0,4.4E1,3E0,4E0,4E0,2E0,2.4E1,1E1,5E0,4E0,2E0,3E0,2E0,6E0,1.7E1,2.7E1,1E1,1.4E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[6.2760315E-3,-8.942039E-2,3.3899674E-1,-3.464915E-1,-2.9866556E-2,-3.944495E-2,5.6220204E-1,-7.096159E-2,-4.272631E-1,-3.1877894E-2,-1.0096922E-2,-2.7154645E-1,1.3294205E-1,3.9848673E-1,3.567469E-2,-8.072219E-3,6.935243E-3,-2.918099E-1,-2.5652623E-2,-1.33155E-1,6.0559504E-2,-4.3664877E-3,-2.650756E-2,1.6947778E-2,8.673523E-3,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,5.782375E-2,-1.6802189E-1,2.2789724E-1,-4.5959535E-3,3.984736E-3,-5.162517E-2,6.689567E-3,-5.874931E-3,-2.1503107E-1,-5.4912195E-3,7.9473734E-2,3.573293E-1,-3.5994206E-2,2.3827569E-1,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,27,-1,29,31,-1,-1,-1,33,-1,-1,-1,-1,35,37,39,41,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5575504E0,2.4391828E0,3.907806E0,6.373303E-1,1.6124437E0,7.563031E-1,5.1900864E-1,1.857129E-1,1.5725279E-1,0E0,1.1210039E0,3.4842473E-1,2.9128677E-1,7.89125E-2,0E0,0E0,0E0,4.2960286E-2,0E0,3.2268065E-1,8.950907E-1,0E0,0E0,0E0,3.954031E-2,0E0,0E0,0E0,0E0,1.3033742E-1,3.0443656E-1,4.1344917E-1,4.6612126E-1,0E0,1.9462278E-2,0E0,0E0,1.0997474E-1,4.840184E-2,1.0358285E-1,2.309668E-2,5.0568026E-1,1.8324465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,17,17,19,19,20,20,24,24,29,29,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,28,-1,30,32,-1,-1,-1,34,-1,-1,-1,-1,36,38,40,42,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7108864E8,1.4133928E1,1E0,2.862988E5,3.673257E0,3.067586E5,1.5435694E9,5.34E2,3.203027E7,-3.1877894E-2,3.3969492E2,2.565E3,1E0,1.4008022E11,3.567469E-2,-8.072219E-3,6.935243E-3,1.3448806E8,-2.5652623E-2,2.0589474E1,1.10597E5,-4.3664877E-3,-2.650756E-2,1.6947778E-2,4.078932E5,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,7.269755E0,1.7869681E6,6.185678E2,3.1982856E3,3.984736E-3,1.743E3,6.689567E-3,-5.874931E-3,1.5373E4,7.343501E-1,4.0759E4,4.3411046E8,9.518453E6,1.4139E4,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"split_indices":[7,59,102,29,54,29,7,0,46,0,53,0,8,32,0,0,0,7,0,59,30,0,0,0,29,0,0,0,0,54,51,53,4,0,0,0,0,9,28,30,7,46,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.59E2,4.5E1,2.9E1,1.3E2,1.7E1,2.8E1,7E0,2.2E1,3E0,1.27E2,7E0,1E1,1.6E1,1.2E1,5E0,2E0,1.1E1,1.1E1,4.6E1,8.1E1,5E0,2E0,3E0,7E0,4E0,1.2E1,9E0,2E0,7E0,3.9E1,2.2E1,5.9E1,3E0,4E0,5E0,2E0,3E1,9E0,1.1E1,1.1E1,5.3E1,6E0,2E0,2E0,1.3E1,1.7E1,7E0,2E0,5E0,6E0,9E0,2E0,1.8E1,3.5E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.2222775E-2,-9.139186E-2,4.9998534E-1,-5.1995587E-2,-4.944162E-1,6.0979265E-1,-1.5150189E-2,-1.3674228E-1,1.0816623E-1,3.8813213E-3,-6.330518E-1,8.045889E-1,3.992678E-1,-3.3645314E-1,-1.01294234E-1,2.436709E-1,-8.044677E-2,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,-3.9906213E-1,-5.7864846E-3,-5.545319E-2,-2.0736766E-1,-8.008159E-2,3.159065E-1,-1.8992147E-1,-6.997158E-3,-2.360766E-2,-1.1299601E-2,-5.494948E-3,-2.1435164E-1,-2.4299191E-1,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.7862383E-1,4.077045E-1,-5.167938E-3,-1.228408E-2,-5.0322257E-2,4.2618223E-2,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,37,39,41,43,-1,-1,45,47,49,-1,-1,-1,51,53,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.480311E0,2.8481483E0,2.6842346E0,2.2679396E0,1.3240938E0,7.0399E-1,0E0,7.3756933E-1,1.5019631E0,0E0,4.9333286E-1,1.782751E-2,2.6048756E-1,1.7048275E-1,4.4557756E-1,8.2043767E-1,1.9969293E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.9644938E-2,0E0,5.306874E-1,1.2524438E-1,5.6175735E-2,2.8003812E-1,2.552101E-2,3.65381E-2,0E0,0E0,2.7235347E-1,1.0511035E-1,6.087303E-2,0E0,0E0,0E0,1.4527509E-1,1.0674596E-1,0E0,0E0,2.577142E-2,9.493223E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,23,23,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,39,39,40,40,43,43,44,44],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,38,40,42,44,-1,-1,46,48,50,-1,-1,-1,52,54,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,1.650269E10,1.2055723E8,6.978001E2,1.8159722E-1,8.812601E-1,-1.5150189E-2,1.142E3,3.291238E7,3.8813213E-3,5.535E3,1.929E3,1.7595541E0,1.3448806E8,1.06403265E1,1E0,2.1845242E3,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,1.22E2,-5.7864846E-3,1.9540394E7,9.744047E0,9.31E2,1.068371E3,5.5053414E9,7.951E3,-2.360766E-2,-1.1299601E-2,1.1888112E-1,4.22631E5,4.187504E7,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.617623E5,2.9E1,-5.167938E-3,-1.228408E-2,2.5821698E0,7.157903E7,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"split_indices":[53,5,46,53,28,58,0,2,46,0,9,0,58,7,55,102,4,0,0,0,0,0,0,10,0,46,54,0,53,5,9,0,0,58,29,33,0,0,0,34,8,0,0,54,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.8E2,2.7E1,1.65E2,1.5E1,2.4E1,3E0,1.08E2,5.7E1,3E0,1.2E1,1.1E1,1.3E1,1.5E1,9.3E1,3.3E1,2.4E1,5E0,7E0,8E0,3E0,3E0,1E1,1.1E1,4E0,6.6E1,2.7E1,6E0,2.7E1,9E0,1.5E1,6E0,5E0,5.1E1,1.5E1,2.1E1,6E0,3E0,3E0,1.2E1,1.5E1,5E0,4E0,8E0,7E0,1.9E1,3.2E1,6E0,9E0,1.8E1,3E0,1E1,2E0,1.2E1,3E0,2E0,6E0,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.469039E-3,-6.84523E-2,5.0491726E-1,-1.8420036E-1,8.1181556E-2,6.261244E-1,3.6942E-2,-2.2080618E-1,7.735341E-2,-3.1455643E-2,2.747695E-1,3.8490146E-2,4.343899E-1,5.4492303E-3,-3.6699688E-3,-1.701605E-1,-4.6023417E-1,-3.2597054E-2,1.7538356E-2,-2.7009997E-1,4.3159403E-2,1.4272796E-1,3.6220354E-1,7.502832E-3,2.3337092E-2,-2.6047137E-1,-8.6393684E-2,-3.3155903E-2,-2.9781944E-1,3.291199E-4,-5.0207926E-3,-1.3054396E-1,-2.0009596E-2,-3.049531E-3,2.1293099E-1,1.6921525E-1,-5.2545854E-4,2.1539481E-2,2.3263586E-1,-3.6815163E-1,-1.5577061E-1,-1.2372288E-2,-4.669773E-2,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,-5.099285E-2,7.831989E-2,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,-1,31,33,35,37,-1,-1,39,41,-1,43,-1,-1,45,-1,47,49,51,-1,-1,53,55,57,-1,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009293E0,3.3230407E0,1.3381958E0,1.0511143E0,1.8402805E0,2.5670147E-1,5.9022054E-2,1.0736556E0,4.4741017E-1,9.7320604E-1,3.0619597E-1,0E0,6.5217495E-2,0E0,0E0,5.8270144E-1,3.7153912E-1,3.1251714E-2,0E0,2.0223719E-1,3.3013782E-1,5.8757484E-2,8.452511E-2,0E0,0E0,3.710723E-1,2.7460602E-1,0E0,4.2746305E-2,0E0,0E0,6.497468E-2,0E0,1.3663186E-1,6.1325133E-2,2.643928E-2,0E0,0E0,1.6941309E-2,5.598998E-2,1.3381499E-1,0E0,1.9927172E-1,0E0,0E0,0E0,0E0,1.089835E-1,1.0501201E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28,31,31,33,33,34,34,35,35,38,38,39,39,40,40,42,42,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,-1,32,34,36,38,-1,-1,40,42,-1,44,-1,-1,46,-1,48,50,52,-1,-1,54,56,58,-1,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,2.2896422E6,9.298568E7,2.096851E3,1.537424E0,8.812601E-1,1.2055723E8,5.0446276E7,1.0625348E8,5.6E0,8.57703E2,3.8490146E-2,4.11964E5,5.4492303E-3,-3.6699688E-3,4.9658173E2,5.511724E0,5.9527163E3,1.7538356E-2,4.3927447E11,5.4732365E9,1.1216892E3,6.518218E6,7.502832E-3,2.3337092E-2,1.822764E4,1E0,-3.3155903E-2,2.0529972E1,3.291199E-4,-5.0207926E-3,6.4336755E8,-2.0009596E-2,7.9472524E-1,3.9952212E3,8.3154816E2,-5.2545854E-4,2.1539481E-2,7.8114974E-1,4.3117376E8,2.524E3,-1.2372288E-2,1.3474202E8,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,1.03235014E-1,9.445656E6,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"split_indices":[53,51,46,4,43,58,46,46,7,55,53,0,30,0,0,4,54,34,0,32,5,4,46,0,0,34,65,0,57,0,0,7,0,40,4,4,0,0,28,5,2,0,7,0,0,0,0,58,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.9E2,2.3E1,1.07E2,8.3E1,1.8E1,5E0,9.4E1,1.3E1,5.3E1,3E1,8E0,1E1,3E0,2E0,7.9E1,1.5E1,1E1,3E0,1.2E1,4.1E1,1.3E1,1.7E1,2E0,8E0,3.7E1,4.2E1,5E0,1E1,7E0,3E0,7E0,5E0,3.3E1,8E0,1.1E1,2E0,9E0,8E0,1.7E1,2E1,7E0,3.5E1,7E0,3E0,4E0,3E0,2.1E1,1.2E1,6E0,2E0,6E0,5E0,6E0,2E0,1.2E1,5E0,1.7E1,3E0,1.2E1,2.3E1,4E0,1.7E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-2.0219442E-2,-1.16520904E-1,2.8449756E-1,-3.0797613E-1,-5.9537902E-2,9.543412E-2,5.2078587E-1,-3.3963862E-1,7.4714734E-3,-2.8888127E-2,-3.689822E-1,2.1040678E-1,-8.497771E-2,3.1196967E-2,3.1238148E-1,-2.7664992E-1,-2.8590444E-2,-1.8818352E-1,6.0974874E-2,-4.8011044E-1,-2.6513548E-3,1.2447058E-2,4.99297E-3,2.9411163E-2,-1.22154895E-2,5.4155197E-3,1.8623266E-2,-3.116507E-1,-1.2212595E-1,-3.6836183E-1,-1.402538E-1,3.1627953E-2,3.3315614E-1,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,-3.2800922E-1,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,-1.5982786E-1,-2.9887237E-2,1.3616896E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.107501E0,1.705395E0,2.1839733E0,5.781839E-1,1.1559219E0,6.194103E-1,3.51089E-1,3.682499E-1,0E0,1.6290425E0,3.58294E-1,6.1777413E-2,2.3528156E-1,0E0,9.8772764E-2,1.18784666E-1,0E0,3.0240762E-1,5.7256365E-1,1.4015281E-1,0E0,0E0,0E0,2.9680826E-2,0E0,0E0,0E0,5.1579952E-2,1.1075765E-2,5.4989576E-2,1.3172704E-1,4.3493733E-1,5.3087175E-2,0E0,0E0,0E0,0E0,0E0,3.895402E-2,0E0,0E0,0E0,0E0,0E0,8.195257E-2,2.843659E-1,3.718426E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,23,23,27,27,28,28,29,29,30,30,31,31,32,32,38,38,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.5497297E1,1.3453049E3,4.5342876E3,9.996903E7,2.68E4,8.812601E-1,3.013037E6,7.4714734E-3,3.2384683E2,1.4539318E-1,1.5331E4,6.079755E0,3.1196967E-2,1.5545E4,1.3448806E8,-2.8590444E-2,8.2955E4,9.467578E6,2.1897722E10,-2.6513548E-3,1.2447058E-2,4.99297E-3,5.5566853E-1,-1.22154895E-2,5.4155197E-3,1.8623266E-2,3.7453184E-3,9.279E3,2.700379E2,2.0589474E1,6.5963456E5,6.2357716E-2,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,5.46562E5,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,1.2705392E1,1.617E3,7.6760286E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"split_indices":[43,59,53,4,46,9,58,29,0,53,39,9,54,0,9,7,0,1,1,5,0,0,0,28,0,0,0,59,9,4,59,29,39,0,0,0,0,0,12,0,0,0,0,0,57,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.57E2,4.9E1,3.5E1,1.22E2,2.8E1,2.1E1,3.3E1,2E0,1.12E2,1E1,1.7E1,1.1E1,1.2E1,9E0,2.8E1,5E0,4E1,7.2E1,7E0,3E0,1.1E1,6E0,7E0,4E0,3E0,6E0,2.2E1,6E0,7E0,3.3E1,6.6E1,6E0,3E0,4E0,4E0,3E0,2E0,2E1,4E0,2E0,2E0,5E0,3E0,3E1,4.2E1,2.4E1,2E0,4E0,1.7E1,3E0,2.1E1,9E0,3.5E1,7E0,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.1936052E-2,-1.5182267E-1,1.5760238E-1,-1.214642E-1,-3.187172E-2,1.07080944E-1,3.2834876E-2,-1.8811886E-1,6.19821E-2,-8.8205084E-2,1.7667614E-1,-3.115475E-1,-1.0234942E-1,1.5685846E-1,-1.4203048E-1,-1.3068446E-2,-2.3196185E-2,-3.587127E-2,2.4979593E-1,-2.4225932E-1,-2.5490196E-2,-2.0880654E-2,-1.4597873E-1,3.4969583E-2,3.1181702E-1,1.3681039E-2,-1.3338024E-2,-9.279402E-2,4.1761484E-2,2.3741236E-2,-1.5669461E-2,9.474784E-2,3.6113718E-1,5.085498E-3,-2.820379E-1,-8.3250865E-2,8.308102E-2,-1.855153E-1,-1.1149166E-2,-5.6521293E-2,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,-2.4819752E-2,-8.349065E-3,6.2872446E-3,1.8276206E-4,-3.816804E-2,4.864078E-3,1.2524661E-1,-6.326331E-3,3.7593523E-1,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,-1,-1,-1,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.107609E0,1.8600852E0,2.331173E0,1.4911933E0,0E0,1.1708164E0,0E0,9.103031E-1,6.6102564E-1,2.6339516E-1,9.880699E-1,4.0248227E-1,1.8889087E-1,4.235136E-1,2.2991076E-1,0E0,8.525165E-2,2.97587E-1,7.771559E-1,2.8676605E-1,0E0,1.3659878E-1,1.8450612E-1,2.3734151E-1,9.613538E-2,8.857786E-3,0E0,4.3926656E-2,3.2743193E-2,7.4582696E-2,0E0,1.6022247E-1,5.321765E-2,5.2886583E-2,9.587407E-2,5.3622194E-2,1.212705E-1,8.459413E-2,9.3104824E-2,2.9008571E-2,0E0,0E0,0E0,0E0,0E0,2.897961E-2,0E0,0E0,0E0,5.3322386E-2,0E0,4.675585E-2,0E0,3.5073757E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,45,45,49,49,51,51,53,53],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,-1,-1,-1,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.0728035E-1,1.3917024E8,1.1038146E0,1.0648859E3,-3.187172E-2,4.2447827E2,3.2834876E-2,6.794883E4,2.2882867E5,2.5546576E4,1E0,2.47565E6,8.623692E0,3.851071E7,4.1808826E-1,-1.3068446E-2,2.3739047E-1,1.019731E0,1.0861458E3,2.9377524E2,-2.5490196E-2,3.8E1,2.4079119E1,2.0601392E3,3.353193E6,4E1,-1.3338024E-2,6.741443E-1,4.35E2,8.836347E7,-1.5669461E-2,4.6E1,4.8420677E-1,1.4416069E9,4.2E1,1.4178E4,7.47E2,3.8718798E6,8.2092184E7,4.83E9,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,9.576E3,-8.349065E-3,6.2872446E-3,1.8276206E-4,9.31E2,4.864078E-3,6.98247E5,-6.326331E-3,1E0,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"split_indices":[40,46,35,53,0,53,0,51,34,34,102,29,55,12,28,0,39,35,53,34,0,3,57,53,1,3,0,28,0,49,0,3,39,5,3,9,0,46,7,5,0,0,0,0,0,9,0,0,0,0,0,29,0,84,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.26E2,9.1E1,1.2E2,6E0,8.4E1,7E0,8.8E1,3.2E1,2.2E1,6.2E1,3.5E1,5.3E1,2.2E1,1E1,5E0,1.7E1,1.6E1,4.6E1,2.8E1,7E0,1.9E1,3.4E1,1.3E1,9E0,5E0,5E0,8E0,9E0,1.4E1,2E0,2E1,2.6E1,4E0,2.4E1,1.2E1,7E0,2.6E1,8E0,9E0,4E0,4E0,5E0,3E0,2E0,5E0,3E0,2E0,7E0,8E0,6E0,1.8E1,2E0,2.4E1,2E0,2E0,2E0,1.7E1,7E0,7E0,5E0,5E0,2E0,4E0,2.2E1,4E0,4E0,3E0,6E0,3E0,2E0,2E0,6E0,1.2E1,6E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-1.6992955E-2,-8.875833E-2,2.2156855E-1,-6.0390815E-2,-5.453842E-1,6.565264E-2,3.5614845E-1,-8.633612E-2,3.0628803E-1,-3.5136674E-2,-2.5970185E-1,1.033548E-2,1.3761056E-3,4.3397513E-3,3.885008E-1,-1.4680506E-1,-1.1059224E-2,2.477328E-2,1.7252595E-3,-9.932688E-4,-2.007134E-2,-5.6842774E-2,1.1758452E-1,2.6560768E-1,4.3975347E-1,-1.07489556E-1,-2.2993198E-2,-1.1149532E-1,1.1053811E-1,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,-2.6751718E-1,-5.8981713E-2,-4.527185E-2,-3.2934055E-1,1.45027E-2,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,29,-1,-1,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4779978E0,1.9877847E0,9.62826E-1,1.4344559E0,2.6657557E-1,2.1661456E-1,1.8128777E-1,6.3109255E-1,6.166359E-1,0E0,1.7431176E-1,0E0,1.2169773E-1,0E0,2.1767378E-2,9.350836E-1,7.817284E-1,0E0,4.614174E-2,0E0,0E0,3.312366E-2,1.0790542E-2,2.8682768E-2,3.3157587E-2,5.27603E-1,0E0,4.9001646E-1,6.9180393E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0392995E-2,2.4817911E-1,1.5262431E-1,9.5196724E-2,2.9514036E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,39,39,40,40,41,41,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,30,-1,-1,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5064244E0,1.4233672E8,8.57703E2,4.9192E4,2.849513E3,1.1244E4,1E0,6.978001E2,7.424212E-1,-3.5136674E-2,2.09E3,1.033548E-2,1.4872659E1,4.3397513E-3,2.8626094E5,4.462437E7,1E0,2.477328E-2,1.3174513E4,-9.932688E-4,-2.007134E-2,5.9857685E6,3.261056E8,1.2519051E7,3.007E3,1.142E3,-2.2993198E-2,3.4521E4,3.212485E11,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,4.543028E2,1.06403265E1,2.7577372E11,5.4656273E-1,2.4394053E9,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"split_indices":[43,46,53,2,4,9,102,53,43,0,0,0,57,0,29,46,102,0,4,0,0,33,7,33,0,2,0,10,32,0,0,0,0,0,0,0,0,0,0,4,55,32,28,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.55E2,4.6E1,1.47E2,8E0,2.2E1,2.4E1,1.38E2,9E0,4E0,4E0,6E0,1.6E1,3E0,2.1E1,7.6E1,6.2E1,5E0,4E0,2E0,2E0,1.1E1,5E0,8E0,1.3E1,6.9E1,7E0,3.4E1,2.8E1,2E0,2E0,6E0,5E0,2E0,3E0,6E0,2E0,1.1E1,2E0,1.5E1,5.4E1,2.7E1,7E0,2.1E1,7E0,1.3E1,2E0,3.6E1,1.8E1,7E0,2E1,2E0,5E0,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.02255E-2,-6.558096E-2,3.4569636E-1,-1.2843427E-2,-4.0469694E-1,5.4956263E-1,4.3343384E-2,-1.403701E-1,1.0433215E-1,2.2149876E-2,-4.8705584E-1,6.838387E-1,3.7301254E-1,1.4347048E-1,-1.6691085E-2,-2.8438172E-1,-7.497648E-2,3.6531553E-2,2.410501E-1,-3.8605474E-3,5.6571406E-3,-5.657334E-1,-2.4888176E-1,9.962667E-3,7.5234216E-1,6.084204E-3,1.9794445E-2,7.620034E-2,1.50580425E-2,-3.147017E-1,-2.6331637E-3,-3.2688044E-2,-1.6758075E-1,3.0311698E-3,1.3236392E-1,-3.7357313E-4,2.712043E-1,-6.3281095E-1,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,2.171496E-3,8.731073E-3,-1.6129533E-2,-5.054191E-3,2.5078287E-2,-1.376335E-1,-2.738632E-3,-1.0184393E-2,6.306253E-2,-5.9866663E-2,2.3515062E-1,-2.9690387E-3,1.6760947E-2,1.7714213E-1,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,-1,41,-1,-1,43,-1,45,-1,47,49,51,53,-1,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,-1,-1,65,67,69,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7954755E0,3.3212526E0,2.565043E0,2.4354134E0,8.896537E-1,3.3758068E-1,7.454521E-1,7.132385E-1,7.791066E-1,5.6962486E-2,2.5271702E-1,2.7266836E-1,5.7875514E-2,1.438675E-1,0E0,1.5053928E-1,2.0990708E-1,1.8580317E-1,2.1208572E-1,0E0,0E0,2.0361853E-1,6.189382E-2,0E0,4.8636913E-2,0E0,0E0,9.129316E-2,0E0,3.965044E-2,0E0,2.3988187E-1,6.710768E-2,1.699124E-1,3.1248683E-1,0E0,1.11125946E-1,2.6826096E-1,0E0,0E0,0E0,0E0,0E0,3.272756E-2,0E0,0E0,0E0,2.1187598E-1,8.620715E-2,0E0,0E0,1.1212682E-1,1.685599E-1,7.086277E-2,0E0,0E0,4.61289E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,24,24,27,27,29,29,31,31,32,32,33,33,34,34,36,36,37,37,43,43,47,47,48,48,51,51,52,52,53,53,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,-1,42,-1,-1,44,-1,46,-1,48,50,52,54,-1,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,-1,-1,66,68,70,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6499896E9,6.627576E7,3.7763092E2,3.3478114E-1,8E0,5.7006E4,1.278E3,1.5022863E0,1.776E3,9.006135E0,2.6E1,1.625E3,8.129243E5,-1.6691085E-2,4.5715973E4,5.531948E8,1.8331E4,7.555544E2,-3.8605474E-3,5.6571406E-3,1.2728E4,1.2187347E3,9.962667E-3,2.2047469E4,6.084204E-3,1.9794445E-2,1.0252E4,1.50580425E-2,3.4322312E-1,-2.6331637E-3,7.269755E0,1.2421726E7,4.278E3,2.4794E4,-3.7357313E-4,1.5489E4,8.260109E9,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,4E0,8.731073E-3,-1.6129533E-2,-5.054191E-3,5.1E2,2.037744E10,-2.738632E-3,-1.0184393E-2,4.541179E1,5.382E3,5.586E3,-2.9690387E-3,1.6760947E-2,7.4353185E0,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"split_indices":[53,5,46,53,28,8,10,2,42,0,55,3,0,29,0,34,5,10,4,0,0,9,4,0,4,0,0,9,0,39,0,54,46,2,9,0,9,5,0,0,0,0,0,8,0,0,0,0,32,0,0,57,2,2,0,0,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.85E2,4.1E1,1.61E2,2.4E1,2.4E1,1.7E1,7.7E1,8.4E1,4E0,2E1,1.2E1,1.2E1,1.4E1,3E0,2.3E1,5.4E1,5.7E1,2.7E1,2E0,2E0,1.4E1,6E0,2E0,1E1,2E0,1E1,1.1E1,3E0,2E1,3E0,3.8E1,1.6E1,4.3E1,1.4E1,3E0,2.4E1,1.1E1,3E0,4E0,2E0,8E0,2E0,7E0,4E0,1.8E1,2E0,2.5E1,1.3E1,5E0,1.1E1,2.2E1,2.1E1,9E0,5E0,1.2E1,1.2E1,2E0,9E0,5E0,2E0,1.5E1,1E1,1E1,3E0,1.8E1,4E0,9E0,1.2E1,5E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-5.735387E-3,-6.410505E-2,2.797867E-1,-1.760073E-2,-3.442871E-1,5.04675E-1,2.686096E-2,-1.2676698E-1,1.1911592E-1,-4.2207205E-1,5.1267102E-2,6.152395E-1,1.3419471E-2,1.5410598E-1,-1.9511065E-1,-2.5507975E-1,-8.848773E-2,2.6512688E-1,4.352096E-2,-3.2537884E-1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,2.4225692E-1,2.5578894E-2,-4.040385E-2,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,-4.242069E-2,-1.8737647E-1,3.087647E-1,4.6277814E-2,-5.1333107E-2,1.4060691E-1,-3.4747878E-3,-3.5448268E-1,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.1831463E-3,-1.2673235E-1,-2.0977704E-1,8.250273E-4,2.3295416E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,5.527623E-2,-1.189838E-1,1.975704E-1,-1.7819712E-2,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,45,47,49,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,65,-1,-1,-1,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5344255E0,2.287163E0,2.0341249E0,2.2836764E0,7.9084253E-1,3.1127357E-1,5.384738E-1,3.9492452E-1,7.390245E-1,2.7931428E-1,7.600727E-2,2.1988153E-1,0E0,1.2976074E-1,1.931273E-1,6.892848E-2,2.9727066E-1,2.0536327E-1,4.308317E-1,1.00349784E-1,0E0,0E0,0E0,0E0,0E0,8.028874E-2,1.2697376E-2,1.2392008E-2,0E0,0E0,0E0,1.8929411E-1,9.967518E-2,7.952249E-2,1.2078894E-2,1.7966965E-1,2.110849E-1,0E0,1.2337124E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1306114E-1,7.93834E-2,1.05109274E-1,0E0,1.7682552E-2,0E0,0E0,0E0,2.9296512E-2,1.0169087E-1,1.4039552E-1,1.7305832E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,27,27,31,31,32,32,33,33,34,34,35,35,36,36,38,38,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,46,48,50,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,66,-1,-1,-1,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6599895E9,6.627576E7,5.242538E2,1.5593E4,8.812601E-1,1.3917024E8,1.056E3,1.26881E5,1.3917024E8,3.729811E6,8E0,1.3419471E-2,5.1E1,2.565E3,7.107292E6,5.4127317E10,6.3300834E0,1.5181E4,1.9E1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,1.739E3,7.7573473E9,1.3174513E4,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,1.0466548E1,1E0,4.361E3,3.098E3,5.7277904E0,3.2395E4,-3.4747878E-3,3.20084E6,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.998024E-1,2.124077E1,7.2899837E0,8.250273E-4,9.607843E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,2.7060036E11,5.36E3,1.7790995E0,4.0953446E8,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"split_indices":[53,5,46,53,9,58,46,2,30,46,1,8,0,3,0,46,32,54,10,3,0,0,0,0,0,0,5,4,0,0,0,55,105,2,2,54,9,0,7,0,0,0,0,0,0,58,59,54,0,58,0,0,0,32,2,58,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.75E2,3.5E1,1.51E2,2.4E1,1.8E1,1.7E1,8.4E1,6.7E1,2E1,4E0,1.1E1,7E0,1.1E1,6E0,1.8E1,6.6E1,2.2E1,4.5E1,1.6E1,4E0,2E0,2E0,5E0,6E0,6E0,5E0,4E0,2E0,1.1E1,7E0,4.6E1,2E1,1.8E1,4E0,2.3E1,2.2E1,2E0,1.4E1,4E0,2E0,2E0,3E0,2E0,2E0,3E1,1.6E1,1.8E1,2E0,1.3E1,5E0,2E0,2E0,9E0,1.4E1,1.6E1,6E0,2E0,1.2E1,2.2E1,8E0,7E0,9E0,1.1E1,7E0,1.1E1,2E0,5E0,4E0,9E0,5E0,7E0,9E0,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-2.2588685E-2,-1.4545202E-1,1.1374607E-1,-8.6952515E-2,-3.9759496E-1,-3.064367E-2,2.3289914E-1,-1.2433577E-1,1.7574582E-1,-5.7842267E-1,-9.222219E-2,-6.9834334E-3,-1.8266106E-2,6.878721E-2,3.451957E-1,-2.7263525E-1,-8.135537E-2,-5.932932E-4,1.1667584E-2,-6.281898E-1,-9.885883E-3,-1.32481875E-2,1.0622433E-1,-2.1569806E-1,3.714026E-2,-4.582277E-2,1.5392454E-1,2.7393732E-2,4.457966E-1,-1.735199E-1,-1.8242102E-2,-2.038975E-1,-3.7180442E-2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,-3.8278066E-2,8.7990396E-2,-4.001707E-3,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,-1.1330966E-1,2.5172912E-2,3.078277E-1,-2.0457634E-1,-1.095509E-3,-1.3675371E-1,-1.4804438E-2,3.1190168E-2,-1.298265E-1,-8.181965E-2,8.646828E-3,-2.6838033E-4,1.4474735E-1,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,61,-1,63,65,-1,67,-1,69,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6011605E0,1.6322045E0,1.7619938E0,9.3515986E-1,1.1062734E0,3.806838E-1,1.0082908E0,5.004853E-1,1.4725864E-1,1.1519289E-1,3.504039E-1,4.224278E-1,0E0,2.4123535E-1,1.0428362E0,1.1729789E-1,3.442828E-1,0E0,0E0,1.5132141E-1,0E0,0E0,1.4618695E-2,8.2455575E-2,1.4909932E-1,5.0066087E-2,1.4621168E-1,4.4345292E-1,8.724737E-2,4.6982825E-2,0E0,7.2891474E-2,3.1442845E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6498463E-1,1.1525272E-1,1.6504118E-2,0E0,0E0,0E0,0E0,9.535315E-2,0E0,1.2730312E-1,2.6230484E-2,0E0,1.0405913E-2,0E0,2.094729E-1,1.0832605E-1,9.527324E-2,0E0,3.1808536E-2,1.478824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,39,39,40,40,41,41,46,46,48,48,49,49,51,51,53,53,54,54,55,55,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,62,-1,64,66,-1,68,-1,70,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3905947E-1,1.4446067E10,4.4589956E5,1E0,1.5461028E12,1.4146589E8,1.2838955E3,6.639351E2,4.22631E5,8.92851E6,1.244125E3,2.853839E4,-1.8266106E-2,1.3012535E1,1E0,8.3949E4,4.7095413E2,-5.932932E-4,1.1667584E-2,6.1826086E0,-9.885883E-3,-1.32481875E-2,1.5011E4,1.338E3,1.0176675E0,1.1350955E0,6.695E3,1E0,3.1E1,1E0,-1.8242102E-2,7.1399055E6,7.11E2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,4.8008E4,1.2364354E3,4.1376923E2,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,2.5591E4,2.5172912E-2,5.86883E6,1.4008755E2,-1.095509E-3,6.1E1,-1.4804438E-2,6.38E2,9.388849E-1,6.8992513E-1,8.646828E-3,2.7953E4,1.1946838E7,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"split_indices":[40,5,29,89,32,46,53,34,29,1,34,34,0,57,102,12,4,0,0,54,0,0,9,2,40,58,2,8,3,105,0,46,0,0,0,0,0,0,0,2,4,53,0,0,0,0,2,0,1,34,0,3,0,0,28,28,0,9,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.12E2,1.01E2,9.2E1,2E1,4.6E1,5.5E1,8.1E1,1.1E1,1.2E1,8E0,4.4E1,2E0,2.3E1,3.2E1,1.7E1,6.4E1,3E0,8E0,1E1,2E0,4E0,4E0,7E0,3.7E1,1E1,1.3E1,8E0,2.4E1,1E1,7E0,1.6E1,4.8E1,7E0,3E0,2E0,2E0,2E0,5E0,1.5E1,2.2E1,8E0,2E0,1.1E1,2E0,2E0,6E0,1.4E1,1E1,8E0,2E0,1.1E1,5E0,2.8E1,2E1,1.3E1,2E0,9E0,1.3E1,6E0,2E0,4E0,2E0,2E0,8E0,2E0,6E0,9E0,2E0,2.4E1,4E0,1.8E1,2E0,1E1,3E0,7E0,2E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.9344994E-3,-8.175323E-2,2.3140544E-1,-4.4790298E-2,-4.8490012E-1,2.5693094E-2,3.847721E-1,-9.1722965E-2,4.259129E-2,-2.0991474E-1,-3.360973E-2,2.7427834E-2,-5.6275822E-2,3.184738E-2,3.1256717E-1,-1.0761265E-1,1.2113636E-1,-2.8683331E-3,2.0815976E-1,-1.3282049E-2,-1.7001278E-3,7.0364326E-2,-1.4656629E-1,4.4856225E-3,1.5982723E-2,-2.2537349E-1,-8.0327906E-2,8.497628E-3,1.8191043E-3,-6.4350456E-2,7.170837E-2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,-3.6205932E-2,-4.63827E-3,-1.3423674E-2,5.86282E-3,-1.1076284E-1,1.3306548E-2,-1.0665464E-1,-2.4310345E-2,1.2974514E-1,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,-1,-1,33,35,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6567056E0,2.3029141E0,1.5360355E0,5.94458E-1,5.9041786E-1,1.0017112E0,3.4057522E-1,3.3096892E-1,3.8445777E-1,6.7638606E-2,0E0,0E0,2.3900099E-1,0E0,5.3549767E-2,2.6464677E-1,2.4824701E-2,1.9260496E-1,3.0953288E-2,0E0,0E0,8.459512E-2,1.8438596E-1,0E0,0E0,7.871467E-2,1.9214371E-1,0E0,0E0,7.697916E-2,1.0903385E-1,0E0,0E0,0E0,0E0,0E0,4.2805668E-2,0E0,0E0,1.03736766E-1,1.2159997E-1,6.966427E-2,5.4593235E-2,3.1910963E-2,5.4574564E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26,29,29,30,30,36,36,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,-1,-1,34,36,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.8000048E10,1E0,2.4277832E6,6.0671224E7,1.7727281E2,1.2117E4,1E0,8.373168E5,5.441E3,-3.360973E-2,2.7427834E-2,3.1047E4,3.184738E-2,5.8E2,6.554877E-4,3.5E1,1.0415988E0,1.521636E5,-1.3282049E-2,-1.7001278E-3,3.5073215E3,5.785788E-1,4.4856225E-3,1.5982723E-2,3E2,3.7504044E5,8.497628E-3,1.8191043E-3,2E0,8.3154816E2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,6.804E3,-4.63827E-3,-1.3423674E-2,1.2599119E-3,5.948464E0,2.7954546E-1,2.1845242E3,8.412314E0,1.579932E5,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"split_indices":[4,5,102,49,46,34,9,89,29,2,0,0,10,0,0,40,3,40,34,0,0,53,43,0,0,0,49,0,0,8,4,0,0,0,0,0,9,0,0,40,54,58,4,55,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.55E2,4.8E1,1.43E2,1.2E1,2.1E1,2.7E1,9.3E1,5E1,6E0,6E0,2E0,1.9E1,4E0,2.3E1,8.7E1,6E0,4E1,1E1,4E0,2E0,8E0,1.1E1,2E0,2.1E1,1.5E1,7.2E1,3E0,3E0,2.2E1,1.8E1,7E0,3E0,5E0,3E0,4E0,7E0,5E0,1E1,1.9E1,5.3E1,8E0,1.4E1,7E0,1.1E1,5E0,2E0,7E0,1.2E1,1.2E1,4.1E1,5E0,3E0,1E1,4E0,5E0,2E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.2111462E-2,-1.5450773E-1,1.031491E-1,-1.09187275E-1,-4.5132187E-1,-2.8939374E-2,1.8243721E-1,-2.6891872E-1,-6.3274145E-2,-2.8736094E-2,-2.1376169E-1,-1.9380113E-1,1.3079939E-2,-5.3318623E-2,2.8723615E-1,-3.048581E-1,-3.2565857E-3,-2.1935193E-1,-2.3797065E-2,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,-1.15509786E-1,4.264821E-2,1.4384617E-1,-1.3136774E-1,1.7543119E-1,4.10858E-1,-3.4710836E-1,-7.3455116E-3,-2.187251E-3,-2.6718003E-1,-4.2986523E-2,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.1813E-2,-8.5979775E-2,1.9637176E-3,1.3061595E-2,-5.950475E-2,-2.1748714E-2,2.684922E-1,1.24782175E-1,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.991617E-2,-1.04395926E-1,8.574969E-3,4.3539573E-2,-5.5793906E-3,-9.2947256E-4,-9.633975E-2,3.2059713E-3,4.982372E-3,1.4664561E-2,1.5088551E-1,3.8726836E-2,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,-1,53,55,-1,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,63,65,-1,67,-1,-1,69,-1,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4133608E0,1.2017343E0,1.2096798E0,5.8435357E-1,2.704537E-1,3.0733958E-1,1.7942166E0,1.1027694E-1,3.9527723E-1,0E0,1.3330251E-2,9.3468994E-2,1.4180462E-1,3.728331E-1,6.189995E-1,4.515779E-2,0E0,9.6432924E-2,2.5955042E-1,0E0,0E0,0E0,0E0,1.570262E-2,1.1871672E-1,7.939862E-2,3.5887626E-1,9.847349E-2,1.3197088E-1,3.6898494E-2,0E0,0E0,1.9970298E-2,2.5871733E-1,0E0,0E0,0E0,6.611198E-2,9.313557E-3,0E0,0E0,7.5622745E-2,0E0,1.8241048E-2,3.9083302E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.800409E-1,1.35196E-1,0E0,3.6446035E-2,0E0,0E0,6.822489E-2,0E0,0E0,0E0,3.4981847E-2,1.2126244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,37,37,38,38,41,41,43,43,44,44,51,51,52,52,54,54,57,57,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,-1,54,56,-1,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,64,66,-1,68,-1,-1,70,-1,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8601307E-2,1.8000048E10,6.787619E2,1.142E3,1.527638E12,2.853839E4,1E0,1.3448806E8,4.059859E-5,-2.8736094E-2,2.906E3,3.6E1,1.7218965E2,2E0,1.3173E4,2.23695E5,-3.2565857E-3,1.7727281E2,3.3E1,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,3.0345E4,4.6E1,4.078932E5,8.476884E-1,9.686559E6,4.747723E6,3.7790768E2,-7.3455116E-3,-2.187251E-3,6.802721E-3,4.454691E5,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.751E3,1.251E3,1.9637176E-3,1.3061595E-2,1.257013E0,-2.1748714E-2,8.366892E4,8.442676E0,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.5E1,5.979E3,8.574969E-3,2.702E3,-5.5793906E-3,-9.2947256E-4,2.012E3,3.2059713E-3,4.982372E-3,1.4664561E-2,7.595543E-1,1.3235667E9,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"split_indices":[39,5,53,2,32,34,102,7,39,0,0,3,53,8,2,12,0,34,8,0,0,0,0,9,3,29,35,52,33,34,0,0,58,49,0,0,0,9,0,0,0,40,0,34,55,0,0,0,0,0,0,3,2,0,2,0,0,0,0,0,0,28,5,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,9.2E1,1.14E2,8.1E1,1.1E1,4.3E1,7.1E1,1.7E1,6.4E1,6E0,5E0,8E0,3.5E1,2.2E1,4.9E1,1.4E1,3E0,1.2E1,5.2E1,2E0,3E0,5E0,3E0,6E0,2.9E1,6E0,1.6E1,2.7E1,2.2E1,1E1,4E0,3E0,9E0,4.9E1,3E0,4E0,2E0,2.4E1,5E0,4E0,2E0,1.4E1,2E0,8E0,1.9E1,2E0,2E1,3E0,7E0,5E0,4E0,2.1E1,2.8E1,4E0,2E1,3E0,2E0,1.1E1,3E0,2E0,6E0,1.4E1,5E0,1E1,1.1E1,2E1,8E0,1.3E1,7E0,5E0,6E0,9E0,5E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.2747186E-3,-5.4162722E-2,1.8111008E-1,-3.108007E-2,-4.4845343E-1,7.1374275E-2,3.275183E-1,-9.3600295E-2,1.1067671E-1,-2.5658727E-2,-6.88157E-3,1.2687185E-1,-2.088327E-2,2.448392E-3,3.571717E-1,-2.3487736E-1,-5.843745E-2,1.925611E-1,-3.9538275E-2,9.78743E-3,3.3138932E-3,-7.214252E-2,6.806273E-3,1.9431083E-2,1.0387621E-2,-1.5502349E-1,-1.7014932E-2,5.5956807E-5,-1.2483689E-1,2.5137913E-1,-3.1633927E-3,-1.7023765E-1,8.103208E-2,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,2.859666E-2,-1.6140592E-1,6.8674185E-3,1.9025224E-1,2.8979616E-2,-1.5382758E-2,-9.495701E-4,-2.327769E-2,1.6913544E-1,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9040357E0,1.3148481E0,6.996037E-1,1.2422698E0,1.18970394E-1,1.4089137E-1,1.3821435E-1,4.6814162E-1,5.365249E-1,0E0,0E0,6.0488462E-2,1.007224E-1,0E0,1.7639875E-2,1.198957E-1,3.0683744E-1,4.4052112E-1,2.659302E-1,0E0,0E0,2.7664423E-2,0E0,0E0,0E0,2.1903068E-2,0E0,2.2701421E-1,1.7931145E-1,3.7047946E-1,0E0,1.6969696E-1,8.6647436E-2,0E0,0E0,0E0,0E0,0E0,1.13961965E-1,6.886476E-2,1.559542E-2,2.4640507E-1,0E0,0E0,0E0,1.2072772E-2,1.1905044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,25,25,27,27,28,28,29,29,31,31,32,32,38,38,39,39,40,40,41,41,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4436649E0,1.4233672E8,1.4008022E11,1.0648859E3,3.7284137E2,2.7883E4,1E0,6.126268E0,2.249985E5,-2.5658727E-2,-6.88157E-3,3.5037E4,5.53839E5,2.448392E-3,2.7E1,7.0712406E5,7.269755E0,9.040637E7,9.875E3,9.78743E-3,3.3138932E-3,1.009E3,6.806273E-3,1.9431083E-2,1.0387621E-2,2.44412E3,-1.7014932E-2,1.053E3,2.1102592E8,5.902126E3,-3.1633927E-3,4.6E1,3.858136E6,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,1E0,7.3376025E6,1.9519106E-1,2.8880222E6,2.8979616E-2,-1.5382758E-2,-9.495701E-4,4.895358E8,8.51E2,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"split_indices":[43,46,32,53,59,9,102,55,34,0,0,30,29,0,8,29,54,46,9,0,0,0,0,0,0,48,0,2,7,4,0,3,48,0,0,0,0,0,89,33,40,33,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.89E2,1.45E2,4.4E1,1.38E2,7E0,2.6E1,1.8E1,9.6E1,4.2E1,5E0,2E0,1.6E1,1E1,2E0,1.6E1,1.8E1,7.8E1,2.7E1,1.5E1,6E0,1E1,8E0,2E0,1.1E1,5E0,1.2E1,6E0,4.2E1,3.6E1,2.2E1,5E0,7E0,8E0,6E0,2E0,4E0,8E0,5E0,3.7E1,2.8E1,8E0,2E1,2E0,3E0,4E0,4E0,4E0,3.3E1,4E0,2.2E1,6E0,2E0,6E0,9E0,1.1E1,2E0,2E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.826838E-3,-6.721812E-2,1.9046006E-1,-7.924094E-2,2.6099376E-2,5.0246296E-3,2.6243427E-1,-2.5692692E-1,-4.875577E-2,-3.755318E-2,3.6572933E-3,2.1474557E-1,1.9588318E-2,-4.4567264E-3,-2.904459E-1,-3.542672E-2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,2.688706E-1,1.3754945E-1,-4.928484E-3,-3.0891788E-1,-1.2508754E-1,4.5123346E-2,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,-1.1071097E-1,1.8496023E-2,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,-1,21,23,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,-1,-1],"loss_changes":[2.477284E0,1.1091111E0,6.6600955E-1,7.729075E-1,0E0,4.7609035E-2,1.5403104E-1,8.979154E-2,6.413361E-1,3.7017375E-2,0E0,8.418608E-2,0E0,0E0,2.8120995E-2,8.8675094E-1,0E0,0E0,0E0,1.0490298E-2,3.518188E-2,0E0,1.5468597E-2,1.0648471E-1,4.4222242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2309903E-1,2.3909211E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,14,14,15,15,19,19,20,20,22,22,23,23,24,24,32,32,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,-1,22,24,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,7.8040985E2,5.5200674E3,2.6099376E-2,6.2958473E4,1.2790948E0,5.4E0,1.4146589E8,4.71E2,3.6572933E-3,9.6494156E-1,1.9588318E-2,-4.4567264E-3,4.146E3,5.3287085E2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,7.810231E4,1.5593E4,-4.928484E-3,1.2187347E3,6.35E2,1.2241036E12,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,4.2E1,2.122531E6,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"split_indices":[43,57,4,49,0,34,36,55,46,0,0,58,0,0,30,53,0,0,0,34,9,0,4,2,32,0,0,0,0,0,0,0,3,48,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.46E2,4.9E1,1.44E2,2E0,1.4E1,3.5E1,2E1,1.24E2,9E0,5E0,2.8E1,7E0,4E0,1.6E1,1.21E2,3E0,4E0,5E0,1.5E1,1.3E1,2E0,1.4E1,5.7E1,6.4E1,4E0,1.1E1,5E0,8E0,1.2E1,2E0,3E0,5.4E1,5.9E1,5E0,4.8E1,6E0,3.8E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.4262974E-3,-8.005432E-2,1.4594802E-1,-2.888352E-1,-4.6757724E-2,-1.8107321E-2,2.4755892E-1,-4.6657827E-3,-3.4083346E-1,-1.0410578E-1,1.8106598E-2,2.52526E-2,-7.499621E-2,7.3558204E-2,3.0511466E-1,-1.0108946E-2,-1.9372413E-2,-1.6241044E-1,-4.8085254E-2,6.1593957E-2,-1.17093466E-1,8.193563E-2,-1.12674244E-1,-1.9586612E-3,1.11724325E-1,3.5032213E-1,1.3559058E-1,-6.2673695E-2,-2.1433787E-1,7.613574E-2,-8.7488696E-2,8.202132E-2,-6.9475058E-3,-5.7769872E-2,-1.4384934E-2,7.815317E-5,7.870827E-3,-1.6748264E-1,-8.807537E-3,7.7555277E-3,2.0889724E-5,3.6501428E-1,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,-3.6418545E-3,-1.605696E-1,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,-1.1916163E-1,1.1715029E-1,1.5941553E-2,1.3056229E-3,-9.284757E-2,-2.2564225E-1,-2.0749527E-3,-2.448091E-3,2.7077156E-3,3.8617578E-1,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,53,-1,55,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,67,69,71,-1,73,75,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4140606E0,8.554564E-1,1.2713884E0,1.3789642E-1,4.0734783E-1,9.553926E-1,4.482491E-1,0E0,1.6353011E-2,1.8163341E-1,3.1294593E-1,0E0,1.7479275E-1,6.096188E-2,2.2073102E-1,0E0,0E0,1.3158071E-1,1.5839781E-1,1.786837E-1,1.20575935E-1,3.8199473E-2,1.2945855E-1,0E0,5.294875E-2,5.433297E-2,3.3570766E-2,4.5042265E-2,5.2512467E-2,7.294734E-2,5.8373034E-2,8.402106E-2,0E0,3.5133395E-2,0E0,0E0,0E0,9.929475E-2,2.6866937E-2,0E0,0E0,4.9077034E-2,0E0,0E0,0E0,0E0,2.0661239E-2,1.6366512E-2,0E0,0E0,0E0,0E0,5.8940127E-2,1.0018352E-1,5.5888433E-2,0E0,2.880384E-2,6.1261773E-2,0E0,0E0,0E0,1.6714334E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,37,37,38,38,41,41,46,46,47,47,52,52,53,53,54,54,56,56,57,57,61,61],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,54,-1,56,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,68,70,72,-1,74,76,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.6770158E-4,1E0,6.051637E-1,3.3030225E2,6.948189E7,6.201765E5,-4.6657827E-3,1.57525425E1,2.0562818E4,9.959017E0,2.52526E-2,9.31E2,1.0363E4,5.03433E0,-1.0108946E-2,-1.9372413E-2,1.0363E4,2.0589474E1,6.579211E9,3.2776688E7,8.319244E8,1.5574268E7,-1.9586612E-3,3.4E1,5.2E1,1.5E1,5.3943863E0,3.62043E1,9.075745E7,5.948464E0,4.9212202E-1,-6.9475058E-3,2E1,-1.4384934E-2,7.815317E-5,7.870827E-3,5.840393E-1,6.804E3,7.7555277E-3,2.0889724E-5,1.0528413E10,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,6.948189E7,1.2421726E7,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,2E0,1.4265206E1,2.1150263E-1,1.3056229E-3,1.5996247E1,2.6023E4,-2.0749527E-3,-2.448091E-3,2.7077156E-3,1.4456249E7,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"split_indices":[53,40,102,28,53,7,49,0,57,34,55,0,0,9,58,0,0,9,59,5,33,12,1,0,3,8,8,54,59,7,54,58,0,3,0,0,0,43,9,0,0,12,0,0,0,0,7,46,0,0,0,0,8,57,39,0,59,10,0,0,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.24E2,7.5E1,1.6E1,1.08E2,2.9E1,4.6E1,4E0,1.2E1,5.7E1,5.1E1,2E0,2.7E1,1.2E1,3.4E1,5E0,7E0,2.7E1,3E1,3.9E1,1.2E1,5E0,2.2E1,3E0,9E0,2.6E1,8E0,1E1,1.7E1,7E0,2.3E1,3.6E1,3E0,1E1,2E0,3E0,2E0,1.4E1,8E0,6E0,3E0,2.4E1,2E0,5E0,3E0,4E0,6E0,1.3E1,4E0,4E0,3E0,7E0,1.6E1,2.3E1,1.3E1,3E0,7E0,9E0,5E0,5E0,3E0,2.1E1,3E0,2E0,4E0,7E0,6E0,1.3E1,3E0,1.3E1,1E1,8E0,5E0,2E0,5E0,2E0,7E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.6450368E-2,-1.1724766E-1,1.2559639E-1,-7.511801E-2,-3.363559E-1,1.8126132E-1,-2.3499241E-1,-1.6364163E-1,-1.0940317E-2,-1.8125798E-1,-4.375745E-1,1.2667201E-1,3.6933804E-1,-4.659123E-2,-2.6914816E-2,-2.2536351E-1,-1.00311846E-1,7.6694086E-2,-4.540101E-2,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,-1.531599E-2,2.0203556E-1,4.1224214E-1,2.9349497E-3,1.9489948E-3,-3.9977366E-3,-2.5541717E-1,2.9264723E-3,1.9217292E-2,-1.2399441E-1,1.2778491E-1,-2.9145163E-2,-7.644804E-2,6.597971E-2,5.250523E-2,-1.4877042E-1,2.3447502E-1,7.5913966E-4,2.1494467E-2,8.123851E-3,-2.7855477E-1,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,-7.0292644E-2,7.10427E-2,8.864286E-3,-2.6523983E-3,1.1110839E-3,1.0315416E-2,-1.1217816E-1,-7.6833105E-4,1.1495934E-1,9.3826205E-2,-1.8227108E-2,-1.1465208E-2,-1.2050162E-3,2.9752174E-1,1.3666672E-1,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,-1,-1,65,-1,-1,-1,-1,67,69,-1,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.913867E0,1.1932194E0,1.6282953E0,6.345087E-1,2.5036097E-1,6.616366E-1,6.333728E-1,1.611085E-1,2.0279819E-1,5.5682838E-2,9.7411156E-2,5.916339E-1,1.740191E-1,2.7933039E-2,0E0,2.1270418E-1,7.315171E-2,1.06465176E-1,1.7102996E-1,0E0,0E0,0E0,0E0,1.8883207E-1,2.3487675E-1,2.0117044E-2,0E0,0E0,0E0,1.0595679E-1,0E0,2.1876162E-2,3.615102E-2,2.7157709E-2,1.046974E-2,1.1895977E-1,4.5869254E-2,4.262839E-2,6.952983E-2,1.5049982E-1,3.6906146E-2,0E0,0E0,4.3109298E-2,0E0,0E0,0E0,0E0,1.5443023E-2,1.32172E-2,0E0,0E0,0E0,9.080692E-2,1.4392808E-1,0E0,9.912826E-3,3.9224803E-2,1.2763529E-2,0E0,0E0,1.4601588E-2,1.00580364E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,48,48,49,49,53,53,54,54,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,-1,-1,66,-1,-1,-1,-1,68,70,-1,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.352552E9,1.2055723E8,5.162122E2,3.4E1,5.0561914E3,1.5119754E0,1.822764E4,5.415E4,9.285558E4,1.2E1,1E0,1.02E2,5.4438036E9,-2.6914816E-2,1E0,2.1768606E5,6.0511904E0,2.2583E4,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,3.20566E7,6.1108776E7,9.298568E7,2.9349497E-3,1.9489948E-3,-3.9977366E-3,6.1E1,2.9264723E-3,8.637E3,2.8774E4,2E0,4.411222E6,1.0092749E8,8.3154816E2,1.4364E4,5.840393E-1,8.383825E1,1.91E2,2.1494467E-2,8.123851E-3,3.5623631E0,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,5.6058385E6,6.314179E2,8.864286E-3,-2.6523983E-3,1.1110839E-3,9.075745E7,2.0053221E6,-7.6833105E-4,4.7E1,9.355911E-1,5.314E3,-1.1465208E-2,-1.2050162E-3,6.518218E6,2.6678162E9,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"split_indices":[53,5,46,56,3,53,58,34,30,34,8,102,3,5,0,105,29,54,10,0,0,0,0,51,46,46,0,0,0,3,0,9,9,8,33,7,4,10,43,59,10,0,0,57,0,0,0,0,33,4,0,0,0,7,49,0,8,28,9,0,0,46,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.31E2,7.8E1,1.11E2,2E1,6.8E1,1E1,4.6E1,6.5E1,9E0,1.1E1,5.4E1,1.4E1,7E0,3E0,2.2E1,2.4E1,1.8E1,4.7E1,7E0,2E0,7E0,4E0,1.9E1,3.5E1,1.2E1,2E0,2E0,5E0,2E1,2E0,4E0,2E1,1.2E1,6E0,3.7E1,1E1,1.3E1,6E0,3E1,5E0,1E1,2E0,1.8E1,2E0,2E0,2E0,1E1,1E1,7E0,5E0,4E0,2E0,1.1E1,2.6E1,4E0,6E0,8E0,5E0,3E0,3E0,1.7E1,1.3E1,2E0,3E0,1E1,8E0,7E0,3E0,3E0,4E0,7E0,4E0,1.2E1,1.4E1,4E0,2E0,6E0,2E0,2E0,3E0,7E0,1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.8897833E-2,-6.780625E-2,2.0897096E-1,-1.5521011E-2,-2.9606915E-1,3.5466662E-1,-4.421246E-3,-9.93627E-2,9.932131E-2,-1.4433515E-1,-4.58412E-1,4.7411785E-1,2.1561413E-1,9.5952526E-2,-1.4718074E-1,-2.0804802E-1,-6.7736566E-2,1.9759446E-1,2.8317139E-2,-1.7917114E-1,-4.480132E-2,-5.3120285E-1,-6.104539E-3,5.165622E-1,9.185563E-3,2.6545724E-1,2.4120836E-3,7.911076E-3,-3.163457E-3,-1.2183908E-2,-3.5092747E-4,-2.297472E-1,-1.1178532E-3,1.8829303E-2,-9.4082914E-2,1.3437358E-1,1.6593615E-2,-1.20509595E-1,6.209947E-2,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,-2.9204038E-1,-6.37282E-3,-4.2878713E-2,1.0429371E-1,-4.7475312E-2,-1.8176384E-1,2.5304276E-2,1.6177177E-1,6.3902856E-4,-8.5941E-3,1.6473201E-1,1.7290728E-2,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,-1,-1,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,67,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3322675E0,2.048187E0,1.1519101E0,1.3674978E0,7.220886E-1,2.631929E-1,2.4339132E-1,2.6776582E-1,4.1307282E-1,5.4385334E-2,2.9693174E-1,3.5738707E-2,8.616072E-2,6.653001E-2,9.928414E-2,6.7011595E-2,1.5001667E-1,1.8078548E-1,1.8848129E-1,1.2413353E-2,1.4049292E-2,2.2869587E-2,0E0,3.3781767E-2,0E0,5.2916467E-2,0E0,0E0,1.2511282E-2,0E0,0E0,6.0228944E-2,0E0,8.88534E-2,1.9570062E-1,5.2684218E-2,0E0,5.4782815E-2,1.3511673E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1208367E-2,0E0,1.15634445E-2,1.6972624E-2,1.08784124E-1,5.8658242E-2,1.0622809E-2,1.1181682E-2,0E0,0E0,4.1976526E-2,3.8918935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,28,28,31,31,33,33,34,34,35,35,37,37,38,38,51,51,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,-1,-1,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,68,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.2829284E7,6.627576E7,5.3287085E2,8.67241E9,8.812601E-1,3.511154E7,1.056E3,1.26881E5,2.1845242E3,5.770668E6,3.2707875E8,1.7414924E7,5.1E1,1.4539318E-1,6.1E1,2.5126263E1,1.467574E3,2.035483E6,1.0892118E3,7.6336784E1,6.1826086E0,-6.104539E-3,4.5E1,9.185563E-3,4.747723E6,2.4120836E-3,7.911076E-3,7.358531E3,-1.2183908E-2,-3.5092747E-4,2.23695E5,-1.1178532E-3,6.21E2,9.348848E0,5.9734796E2,1.6593615E-2,9.31E2,2.7E1,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,2.2290532E3,-6.37282E-3,5.093772E-1,3.396881E4,1.1556807E0,7.688406E0,3.7E1,1.617623E5,6.3902856E-4,-8.5941E-3,1.38635E5,1.8454842E-1,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"split_indices":[53,46,46,53,5,58,33,2,30,4,30,33,33,3,39,3,59,4,33,53,57,54,0,3,0,33,0,0,4,0,0,12,0,11,55,53,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,49,0,28,34,43,54,3,34,0,0,11,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.71E2,3.6E1,1.4E2,3.1E1,2.1E1,1.5E1,8.1E1,5.9E1,1.7E1,1.4E1,1E1,1.1E1,9E0,6E0,1.7E1,6.4E1,2.4E1,3.5E1,1.2E1,5E0,1.1E1,3E0,8E0,2E0,8E0,3E0,5E0,4E0,3E0,3E0,1.5E1,2E0,1.5E1,4.9E1,1.8E1,6E0,6E0,2.9E1,8E0,4E0,3E0,2E0,9E0,2E0,6E0,2E0,2E0,6E0,2E0,2E0,8E0,7E0,9E0,6E0,3.3E1,1.6E1,4E0,1.4E1,2E0,4E0,8E0,2.1E1,2E0,6E0,6E0,3E0,4E0,2E0,2E1,1.3E1,1E1,6E0,2E0,2E0,1.2E1,2E0,2E0,6E0,3E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[9.597382E-3,-7.689029E-2,1.3135731E-1,-5.166708E-2,-3.934461E-1,1.9286239E-1,-1.5746945E-1,-1.8152632E-1,-2.5963075E-2,-2.2153566E-2,-5.7512084E-3,1.2324249E-1,4.3318063E-1,-8.0194466E-2,-2.265891E-2,-2.6121805E-3,-2.3640567E-1,5.765824E-4,-9.947969E-2,-1.1233864E-2,1.7927498E-1,5.3216517E-1,2.2989117E-1,-1.9807076E-1,6.597765E-2,-4.3076635E-3,-1.2422029E-2,2.8069925E-2,-1.1729321E-1,6.657329E-2,-1.4218627E-1,9.798071E-2,-6.077645E-2,-1.3545295E-3,1.9635805E-1,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,7.5570874E-2,-6.374473E-2,-1.7920145E-1,-1.4495179E-3,2.901012E-4,4.642252E-3,-8.290076E-2,-1.9911025E-1,1.871269E-3,7.0417197E-3,-5.9855576E-3,-1.9459605E-2,2.7096123E-1,1.4672595E-1,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,63,65,-1,-1,-1,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2320848E0,9.697564E-1,1.5936292E0,3.824837E-1,9.9226475E-2,1.1679718E0,3.4362957E-1,1.2062222E-1,1.912657E-1,0E0,0E0,4.3904662E-1,1.9962025E-1,2.5429142E-1,0E0,0E0,1.3633668E-2,2.3985873E-1,1.9384623E-1,1.0334882E-1,1.5052485E-1,1.5721798E-2,2.7384043E-2,1.3692349E-2,4.27702E-2,0E0,0E0,2.664942E-1,7.050744E-2,9.9109635E-3,5.4363728E-2,1.1121236E-2,3.102351E-2,0E0,1.0094595E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2414542E-1,1.4501756E-1,3.8122535E-2,0E0,0E0,0E0,4.5270823E-2,2.6841223E-2,0E0,0E0,0E0,1.3282825E-2,1.0726178E-1,4.9301982E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,34,34,43,43,44,44,45,45,49,49,50,50,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,64,66,-1,-1,-1,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.2109607E2,8.260109E9,9.298568E7,1.295E3,9.4819355E0,5.08711E3,2.88001E10,5.7625353E-1,1.5989196E7,-2.2153566E-2,-5.7512084E-3,1E0,2.7E1,1.5574268E7,-2.265891E-2,-2.6121805E-3,2.9585883E2,1.193419E1,1.3729923E5,1E0,7.056125E7,9.886E3,1.917786E7,1.15E2,3.9E1,-4.3076635E-3,-1.2422029E-2,5.998024E-1,8.401064E0,2.68E2,1.09137096E8,1.9253E4,2.1845242E3,-1.3545295E-3,1.2951E4,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,4.1376923E2,8.05E2,2.6239506E5,-1.4495179E-3,2.901012E-4,4.642252E-3,3.1274893E2,1.0209414E1,1.871269E-3,7.0417197E-3,-5.9855576E-3,4E1,1.1865996E1,1.4456249E7,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"split_indices":[53,5,46,2,54,53,5,28,46,0,0,102,8,1,0,0,34,55,29,8,7,9,1,10,3,0,0,58,54,0,7,2,4,0,9,0,0,0,0,0,0,0,0,53,0,29,0,0,0,53,55,0,0,0,3,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.23E2,8.7E1,1.15E2,8E0,7.2E1,1.5E1,1.8E1,9.7E1,6E0,2E0,5.7E1,1.5E1,1.3E1,2E0,6E0,1.2E1,7.2E1,2.5E1,1.7E1,4E1,9E0,6E0,7E0,6E0,2E0,1E1,5.9E1,1.3E1,5E0,2E1,5E0,1.2E1,3E0,3.7E1,4E0,5E0,3E0,3E0,2E0,5E0,2E0,4E0,3.9E1,2E1,7E0,6E0,2E0,3E0,1.1E1,9E0,3E0,2E0,4E0,8E0,1.3E1,2.4E1,2.4E1,1.5E1,1.1E1,9E0,4E0,3E0,4E0,7E0,7E0,2E0,4E0,4E0,1.1E1,2E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-1.772178E-2,-7.057884E-2,8.537469E-2,-5.372641E-2,-3.503544E-1,-7.3327735E-2,1.9524743E-1,-1.9434269E-1,-2.6594644E-2,-2.087923E-2,-8.001967E-3,1.7575795E-2,-2.1872297E-1,1.2774226E-1,3.8615E-1,-2.1584399E-1,-2.1231158E-3,2.6761827E-3,-1.002963E-1,-2.3103694E-2,1.2448591E-2,-3.2898784E-1,7.034172E-5,-6.594587E-2,1.6341075E-1,2.4855476E-2,1.1777236E-2,-2.5548145E-1,-6.297792E-3,6.468007E-2,-6.0867704E-2,3.847441E-3,-1.2086692E-1,1.0591442E-1,-6.7531325E-2,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,1.8735889E-1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,1.1528385E-1,-3.7465915E-2,-1.02739446E-1,5.237328E-2,-1.373252E-1,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,-2.5229553E-2,2.3754518E-1,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,-1,-1,41,-1,43,45,-1,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,55,57,59,61,63,-1,-1,-1,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1885403E0,6.6012216E-1,1.3046603E0,5.1750946E-1,4.6478212E-2,4.1356945E-1,5.176964E-1,6.1877012E-2,2.5051618E-1,0E0,0E0,1.9972178E-1,2.9179746E-1,2.4566746E-1,7.394695E-2,3.203535E-2,0E0,3.3489424E-1,1.3130611E-1,1.10328E-1,0E0,9.977138E-2,0E0,1.4139006E-2,1.0379839E-1,0E0,0E0,6.927937E-2,0E0,2.2658505E-1,2.045279E-1,0E0,7.257894E-2,1.1413369E-2,8.147253E-2,0E0,0E0,0E0,0E0,8.3029985E-2,0E0,0E0,0E0,9.5561266E-2,5.3016253E-2,1.7131087E-1,4.7250487E-2,5.080968E-2,0E0,0E0,0E0,0E0,2.8906034E-2,3.853917E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,23,23,24,24,27,27,29,29,30,30,32,32,33,33,34,34,39,39,43,43,44,44,45,45,46,46,47,47,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,-1,-1,42,-1,44,46,-1,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,56,58,60,62,64,-1,-1,-1,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.142E3,6.645232E2,8.307621E7,1.9253E4,1.3448806E8,1.6278354E7,-2.087923E-2,-8.001967E-3,1.9142364E7,1.4539318E-1,7.5773336E7,1.2117E4,2.23695E5,-2.1231158E-3,1.643E4,5.577E3,8.87E2,1.2448591E-2,2.2982106E0,7.034172E-5,1.33511E5,7.4308796E0,2.4855476E-2,1.1777236E-2,3.7790768E2,-6.297792E-3,1.0466548E1,6.3004294E5,3.847441E-3,9.568003E2,1.4783E4,2.9E1,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,7.4009796E1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,5.3055557E1,6.693824E5,4.966E3,1.175E1,1.30468176E11,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,7.7573473E9,4.60675E5,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"split_indices":[53,5,102,2,34,46,2,7,46,0,0,1,39,7,9,12,0,9,9,0,0,54,0,12,54,0,0,34,0,55,29,0,53,9,3,0,0,0,0,59,0,0,0,59,49,2,57,32,0,0,0,0,5,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.43E2,7.3E1,1.36E2,7E0,3E1,4.3E1,2.1E1,1.15E2,4E0,3E0,1.9E1,1.1E1,3.3E1,1E1,1.8E1,3E0,8.3E1,3.2E1,1.7E1,2E0,7E0,4E0,5E0,2.8E1,4E0,6E0,1.1E1,7E0,4.2E1,4.1E1,3E0,2.9E1,4E0,1.3E1,2E0,5E0,3E0,2E0,2.4E1,4E0,4E0,7E0,2.8E1,1.4E1,3E1,1.1E1,2.6E1,3E0,2E0,2E0,2E0,1.1E1,1.4E1,1E1,1.7E1,1.1E1,4E0,1E1,2.5E1,5E0,6E0,5E0,1.3E1,1.3E1,7E0,4E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.1619442E-2,-6.1717395E-2,1.3433132E-1,-2.3387735E-1,-4.0964365E-2,1.77865E-3,2.3689537E-1,1.3427171E-2,-3.110039E-1,-1.2818594E-1,-3.7928703E-3,5.2376132E-2,-8.1358645E-3,2.3620052E-2,1.8211213E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,-1.5684637E-1,1.68092E-2,7.7523957E-3,-1.8851958E-2,-1.6274573E-2,1.226543E-1,2.2858551E-1,6.3767E-2,-1.8596564E-1,-4.8825707E-2,8.553022E-2,-6.1434526E-3,-7.6919543E-3,1.647666E-1,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,1.4144917E-1,1.4252614E-2,4.003073E-4,4.388098E-3,-1.9882518E-1,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,-2.6464459E-2,1.329478E-1,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,-1,-1,-1,-1,27,29,31,-1,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,53,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5730232E0,5.620738E-1,7.473112E-1,3.2842952E-1,4.6638432E-1,2.2247311E-1,3.3355105E-1,3.4519482E-2,7.46156E-2,1.8132824E-1,4.54711E-1,9.848905E-2,0E0,0E0,1.3392115E-1,0E0,0E0,0E0,0E0,1.0415751E-1,8.9438625E-2,2.4376561E-1,0E0,1.8215474E-2,2.6880935E-2,6.499237E-2,1.245489E-2,5.9958518E-2,5.9645988E-2,1.10448785E-2,0E0,2.4641316E-1,2.660959E-1,0E0,0E0,0E0,0E0,1.1662543E-2,0E0,0E0,0E0,2.202785E-2,0E0,0E0,0E0,0E0,0E0,1.3542011E-1,6.190428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,37,37,41,41,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,-1,-1,-1,-1,28,30,32,-1,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,54,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,6.794276E-5,6.48045E6,1.7727281E2,2.802166E2,1.0430015E1,6.677E3,1.94122E5,3.2445145E2,1.01978E5,1.0590684E1,2.806E3,-8.1358645E-3,2.3620052E-2,8.269529E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,2.1974122E5,4.87374E5,4.9192E4,-1.8851958E-2,1.5878E4,2.0837E4,1.980161E0,4.602015E0,6.1E1,3.7E1,3.52777E5,-6.1434526E-3,1E0,1.0670794E10,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,5.649635E0,1.4252614E-2,4.003073E-4,4.388098E-3,1.8E2,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,3.2441288E5,3.9360254E2,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"split_indices":[43,43,49,34,53,55,9,1,34,30,58,2,0,0,28,0,0,0,0,48,1,2,0,10,9,43,54,3,3,1,0,89,12,0,0,0,0,54,0,0,0,11,0,0,0,0,0,29,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.59E2,5.4E1,1.6E1,1.43E2,2.4E1,3E1,4E0,1.2E1,4.2E1,1.01E2,1.9E1,5E0,4E0,2.6E1,2E0,2E0,5E0,7E0,3.5E1,7E0,9.9E1,2E0,1E1,9E0,1.8E1,8E0,2.7E1,8E0,5E0,2E0,9.1E1,8E0,6E0,4E0,7E0,2E0,9E0,9E0,3E0,5E0,2.5E1,2E0,6E0,2E0,2E0,3E0,8.1E1,1E1,4E0,4E0,4E0,5E0,4E0,2.1E1,2.7E1,5.4E1,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.0766466E-3,-4.316265E-2,2.0560837E-1,-1.2818186E-2,-2.3354582E-1,2.703695E-1,-9.532744E-2,-5.3655125E-2,1.0286662E-1,-2.9023632E-1,-1.5101601E-2,2.0894068E-1,4.177894E-1,1.6216272E-2,-1.39497025E-2,-1.1859325E-1,-2.1754269E-2,2.239287E-1,4.9224846E-2,-7.534239E-3,-3.746514E-1,-5.9629036E-3,3.3697663E-3,7.089449E-2,2.384329E-1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,-1.6322732E-1,-5.6480993E-2,2.9111031E-2,-7.753057E-2,1.5182119E-2,4.083155E-3,8.881563E-2,-4.8623074E-2,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,2.5431147E-1,5.2253725E-3,6.213173E-4,-1.8242495E-1,-1.1897854E-2,-5.765586E-3,5.034584E-2,-1.3384877E-1,7.912595E-3,-1.0946924E-1,-9.919967E-3,1.289712E-1,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,37,-1,-1,39,41,-1,-1,-1,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,-1,55,-1,-1,57,59,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.055721E0,1.0079945E0,8.2482004E-1,7.235998E-1,2.9282618E-1,2.2060513E-1,1.7536506E-1,2.3150888E-1,2.487365E-1,1.5920115E-1,6.284737E-2,8.894992E-2,2.0777345E-2,1.1685234E-2,0E0,9.335673E-2,2.2071204E-1,1.2175274E-1,1.1666089E-1,0E0,1.730293E-1,0E0,0E0,3.1039445E-2,1.6396642E-2,0E0,0E0,0E0,0E0,7.328111E-2,4.5573883E-2,1.4861596E-1,1.0183704E-1,0E0,0E0,8.453986E-2,1.6937668E-2,0E0,0E0,0E0,0E0,1.6923666E-2,0E0,0E0,3.164649E-2,3.1010306E-2,0E0,1.0485908E-1,1.7051622E-2,8.5487984E-2,7.8597695E-2,2.3515034E-2,7.874721E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,20,20,23,23,24,24,29,29,30,30,31,31,32,32,35,35,36,36,41,41,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,38,-1,-1,40,42,-1,-1,-1,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,-1,56,-1,-1,58,60,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.0099896E9,1.2055723E8,8.297185E2,1.2728E4,1.4243088E6,2.374331E5,2.1015242E2,1.10597E5,5.033165E7,1.4792529E10,4.0492815E-1,1.4735E4,9.4908775E1,-1.39497025E-2,3.5E1,7.2929355E6,6.675932E1,8.7022424E-1,-7.534239E-3,8.260109E9,-5.9629036E-3,3.3697663E-3,8.601656E6,5.1E1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,7.36E2,3.5223506E5,1.996866E1,4.008916E10,1.5182119E-2,4.083155E-3,9.1014955E10,2.7577372E11,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,1.6503E4,5.2253725E-3,6.213173E-4,1.13002876E-4,6.693824E5,-5.765586E-3,2.579E3,6.09E2,1.438E3,1.8272479E3,5.0895057E10,1E0,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"split_indices":[53,5,46,53,9,29,34,53,30,7,5,28,2,57,0,3,46,59,28,0,5,0,0,1,3,0,0,0,0,30,29,57,32,0,0,32,32,0,0,0,0,2,0,0,39,49,0,2,0,2,4,32,102,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.74E2,4E1,1.51E2,2.3E1,3.3E1,7E0,1.12E2,3.9E1,1.8E1,5E0,2.5E1,8E0,5E0,2E0,3.6E1,7.6E1,1.1E1,2.8E1,8E0,1E1,2E0,3E0,5E0,2E1,2E0,6E0,2E0,3E0,2E1,1.6E1,4E1,3.6E1,6E0,5E0,2E1,8E0,3E0,7E0,3E0,2E0,1.7E1,3E0,2E0,1.8E1,1E1,6E0,3.6E1,4E0,1E1,2.6E1,6E0,1.4E1,2E0,6E0,3E0,1.4E1,2E0,1.6E1,4E0,6E0,1.8E1,1.8E1,2E0,2E0,3E0,7E0,2.4E1,2E0,2E0,4E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[5.1350137E-3,-3.3741128E-2,2.475987E-1,-2.1715815E-1,-6.9968994E-3,3.2988298E-1,2.8091062E-2,-6.1420463E-2,-2.4671237E-1,-9.5820725E-2,3.2276653E-2,2.3344913E-3,3.5703343E-1,-5.512472E-2,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,-3.0382487E-1,-1.4058068E-1,-2.9446453E-2,-2.1209379E-1,5.59181E-2,4.091285E-1,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,-6.627251E-2,-2.2693902E-1,5.2225657E-2,-9.5052674E-2,1.0859418E-2,-3.6253917E-1,2.6486173E-2,1.5046827E-1,2.1380894E-2,7.393621E-3,2.4758913E-3,-8.528379E-2,-3.1659063E-3,-2.504589E-1,-1.3437112E-3,5.355821E-3,-7.7356305E-3,-4.3233983E-2,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,1.0883494E-2,1.4663106E-1,1.9825523E-1,8.6160064E-2,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,59,-1,-1,-1,-1,61,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9783993E0,8.8634837E-1,5.145322E-1,9.070897E-2,5.577526E-1,1.4360905E-1,6.3942134E-2,9.758426E-3,6.0073018E-2,1.4143506E-1,6.5313697E-1,0E0,1.07219934E-1,1.2388382E-2,0E0,0E0,0E0,0E0,3.4586906E-2,1.7105693E-1,1.1748636E-1,3.393597E-1,2.7985966E-1,4.202509E-2,0E0,0E0,0E0,0E0,0E0,4.2199805E-2,3.3204734E-2,4.9204696E-2,3.2862037E-2,3.151442E-2,7.372153E-2,1.4649613E-1,5.6672513E-2,0E0,0E0,0E0,3.002061E-2,0E0,3.334278E-2,0E0,0E0,0E0,2.2537217E-2,0E0,0E0,0E0,0E0,1.6933751E-1,1.3314232E-2,2.8548539E-2,3.267061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,18,18,19,19,20,20,21,21,22,22,23,23,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,42,42,46,46,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,60,-1,-1,-1,-1,62,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7162656E3,2.750917E-4,9.298568E7,2.56917E-2,2.802166E2,4.6111366E1,5.547E3,6.054E3,2.799504E11,1.8716404E10,3.791641E0,2.3344913E-3,4.5E1,4.8008E4,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,6.645232E2,8.1446184E7,4.57E2,8.640031E9,1.5085194E0,3.2707875E8,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,1.5E1,2.2057E4,2.3168E5,3.1E1,8.601996E-1,8.92851E6,7.6599895E9,2.5661135E0,2.1380894E-2,7.393621E-3,2.4758913E-3,3.07E2,-3.1659063E-3,2.0562818E4,-1.3437112E-3,5.355821E-3,-7.7356305E-3,2.7898628E5,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,6.7835927E0,1.743E3,2.6198992E-1,1.51E3,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"split_indices":[53,40,46,59,53,57,9,2,32,32,54,0,3,2,0,0,0,0,34,5,0,5,43,33,0,0,0,0,0,3,12,12,3,28,1,5,43,0,0,0,11,0,34,0,0,0,29,0,0,0,0,54,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.8E2,2.8E1,2.2E1,1.58E2,2E1,8E0,4E0,1.8E1,4.8E1,1.1E2,2E0,1.8E1,4E0,4E0,2E0,2E0,8E0,1E1,2.8E1,2E1,9E0,1.01E2,1.3E1,5E0,2E0,2E0,5E0,5E0,1.6E1,1.2E1,9E0,1.1E1,4E0,5E0,7.8E1,2.3E1,1.1E1,2E0,2E0,1.4E1,2E0,1E1,4E0,5E0,4E0,7E0,2E0,2E0,3E0,2E0,7E1,8E0,1.2E1,1.1E1,5E0,9E0,5E0,5E0,2E0,5E0,4E1,3E1,2E0,6E0,6E0,6E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-6.245155E-3,-2.833662E-2,3.4589776E-1,-7.005383E-2,5.9592E-2,4.2491645E-1,-1.5961174E-3,-4.3598708E-2,-2.8613925E-1,-8.835135E-2,1.3214482E-1,5.290952E-1,1.0459326E-2,-9.431229E-2,1.7313663E-2,-4.463223E-3,-3.3082795E-1,2.9486376E-3,-2.370824E-1,-2.7359384E-3,1.4807577E-1,3.203436E-2,8.785884E-3,-1.4088191E-1,-5.170084E-2,4.7399618E-2,-1.0460413E-1,-2.1237668E-2,-1.0130584E-2,6.996463E-3,-4.852486E-2,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,1.24671705E-1,-2.0670156E-1,-8.551871E-2,5.622916E-3,-6.533785E-2,6.141754E-2,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,-8.802993E-2,6.8448015E-2,1.8364535E-1,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,51,53,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5812309E0,7.0478874E-1,3.730235E-1,7.306783E-1,6.77165E-1,1.050607E-1,0E0,3.6305583E-1,9.1302514E-2,2.8586107E-1,1.3435823E-1,1.5126264E-1,0E0,1.1971778E-1,2.0303094E-1,0E0,5.1597834E-2,1.0845146E-1,1.1068231E-1,0E0,6.660807E-2,0E0,0E0,9.164274E-2,8.737236E-2,1.02572635E-1,6.1553992E-2,0E0,0E0,0E0,2.9702732E-2,0E0,0E0,0E0,1.00164235E-1,4.0271103E-2,6.0178608E-2,0E0,5.5570483E-2,7.76602E-2,0E0,0E0,0E0,0E0,1.725753E-2,3.47196E-2,2.8324187E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,30,30,34,34,35,35,36,36,38,38,39,39,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,52,54,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,1.0648859E3,1.2055723E8,4.934166E7,1E0,3.0291426E-1,-1.5961174E-3,3.3969492E2,8.7793425E-2,5E0,7.340097E7,1.04E3,1.0459326E-2,2.0562818E4,1.2670352E1,-4.463223E-3,3E0,8.78E2,9.875E3,-2.7359384E-3,3.835264E-2,3.203436E-2,8.785884E-3,1.4357987E10,4.914413E-3,1.7328871E6,1.018E3,-2.1237668E-2,-1.0130584E-2,6.996463E-3,2E0,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,2.6673E4,7.3353534E2,5.571429E0,5.622916E-3,3.606519E-1,2.5356756E1,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,9.281264E-2,6.7835927E0,7.1847E4,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"split_indices":[4,53,46,46,102,39,0,53,59,8,7,0,0,34,55,0,8,0,9,0,28,0,0,32,39,48,0,0,0,0,8,0,0,0,10,4,57,0,39,57,0,0,0,0,39,54,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.9E2,1.1E1,1.29E2,6.1E1,9E0,2E0,1.16E2,1.3E1,2E1,4.1E1,5E0,4E0,6.3E1,5.3E1,3E0,1E1,1.3E1,7E0,3E0,3.8E1,3E0,2E0,2.9E1,3.4E1,4.3E1,1E1,4E0,6E0,3E0,1E1,4E0,3E0,5E0,3.3E1,1.2E1,1.7E1,2E0,3.2E1,4E1,3E0,8E0,2E0,4E0,6E0,1.8E1,1.5E1,1E1,2E0,6E0,1.1E1,2.6E1,6E0,3.1E1,9E0,3E0,3E0,1.3E1,5E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.3435227E-3,-3.277699E-2,1.774612E-1,-6.964357E-3,-1.5355606E-1,1.9180963E-2,2.988968E-1,-6.0709182E-2,7.332652E-2,-6.922103E-2,-2.4452102E-1,1.5958549E-1,-3.233358E-2,3.533102E-1,1.318238E-1,-1.3005522E-1,-4.060321E-2,1.0830404E-1,-3.1816754E-2,-1.1131207E-2,-1.2607405E-1,-2.9532254E-1,2.7801134E-3,1.0396815E-3,1.1904416E-2,1.8165827E-2,-1.0879546E-2,3.7512168E-1,6.5092915E-3,2.031597E-3,9.1953445E-3,-1.5207392E-1,-1.9281486E-4,-4.925895E-3,-9.33342E-2,5.5923946E-2,1.5398896E-1,-1.0388468E-1,2.1812472E-2,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,-3.3411142E-1,-2.0121949E-2,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,4.4080522E-2,-6.593762E-2,-4.5851987E-2,-1.4824677E-1,-2.6830027E-3,8.220542E-2,1.7252632E-3,1.8331738E-1,9.3954586E-4,-7.641705E-3,4.75524E-3,-9.375982E-3,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,67,69,71,73,-1,75,-1,77,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4195844E0,5.663384E-1,7.4388015E-1,6.560496E-1,2.2856313E-1,1.353517E-1,1.5342748E-1,1.21597886E-1,2.2778258E-1,5.8043107E-2,2.467323E-1,5.353722E-2,1.3925183E-1,2.644825E-2,2.8130919E-2,5.492565E-2,1.3499385E-1,1.01465404E-1,6.410514E-2,3.194621E-2,4.1086197E-2,4.452479E-2,0E0,0E0,0E0,3.9501768E-2,0E0,4.0338874E-2,0E0,0E0,0E0,3.306678E-2,0E0,1.3446508E-1,6.8816334E-2,7.1661495E-2,7.693821E-2,4.8853584E-2,2.4587668E-2,0E0,0E0,0E0,0E0,0E0,4.43269E-2,1.6359001E-2,0E0,0E0,0E0,0E0,0E0,1.0982905E-1,8.9442335E-2,3.789381E-2,2.6375294E-2,0E0,2.339761E-2,0E0,3.7862718E-2,0E0,0E0,0E0,1.0178003E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,27,27,31,31,33,33,34,34,35,35,36,36,37,37,38,38,44,44,45,45,51,51,52,52,53,53,54,54,56,56,58,58,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,68,70,72,74,-1,76,-1,78,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.075104E7,1E0,5.470729E2,7.6599895E9,1E0,3.3326385E0,1.142E3,2.7883E4,1.0363E4,7.2418816E12,3.5912186E-1,8.129243E5,4.4E1,3.6E1,1.3448806E8,5.549974E8,7.8761675E5,1.6595541E1,1.1484E4,1.0731037E-1,8.67241E9,2.7801134E-3,1.0396815E-3,1.1904416E-2,2.613E3,-1.0879546E-2,2.608036E6,6.5092915E-3,2.031597E-3,9.1953445E-3,1.8E2,-1.9281486E-4,4.61E2,1.2845426E1,3.3997994E5,3.88E2,8.45E2,1.3050505E0,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,6.1826086E0,1.970492E6,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,7.269755E0,3.6050353E5,9.5157854E-2,1E0,-2.6830027E-3,7E1,1.7252632E-3,7.3586698E0,9.3954586E-4,-7.641705E-3,4.75524E-3,1.82E9,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"split_indices":[53,46,102,53,5,8,58,2,9,9,32,28,29,3,3,7,5,29,57,2,28,5,0,0,0,0,0,1,0,0,0,11,0,0,57,48,0,0,58,0,0,0,0,0,54,30,0,0,0,0,0,54,29,39,65,0,8,0,54,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.81E2,3.8E1,1.5E2,3.1E1,1.7E1,2.1E1,9E1,6E1,1.7E1,1.4E1,4E0,1.3E1,1.5E1,6E0,1.9E1,7.1E1,4.5E1,1.5E1,9E0,8E0,1.2E1,2E0,2E0,2E0,1.1E1,2E0,1.3E1,2E0,3E0,3E0,1.6E1,3E0,4.3E1,2.8E1,2.2E1,2.3E1,6E0,9E0,7E0,2E0,2E0,6E0,3E0,9E0,8E0,3E0,2E0,1.1E1,5E0,1.1E1,2.4E1,1.9E1,1.6E1,1.2E1,4E0,1.8E1,5E0,1.8E1,2E0,4E0,2E0,7E0,5E0,4E0,4E0,4E0,1.7E1,7E0,1.3E1,6E0,1.2E1,4E0,2E0,1E1,1.6E1,2E0,1.6E1,2E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-1.0673481E-2,-2.922265E-2,3.2056907E-1,-9.558372E-2,6.192814E-3,7.489175E-3,1.9760614E-2,-6.519865E-2,-2.9253313E-1,-5.28204E-2,5.102089E-2,-1.4650565E-1,-1.8217016E-2,-8.463386E-3,-2.1037353E-2,2.8328342E-2,-8.429096E-2,-5.649434E-2,8.870596E-2,-1.687275E-1,-7.512345E-4,8.831442E-2,-5.69984E-2,7.728642E-2,-2.82216E-3,-1.0359726E-1,6.4482796E-4,4.802586E-3,-9.4108865E-2,1.5494351E-2,1.22702755E-1,-2.0045006E-1,-3.7551587E-3,5.5987155E-4,6.6966885E-3,-2.3729175E-3,-8.077933E-3,6.7342306E-3,6.569003E-4,-1.23539165E-1,-4.9365353E-2,-6.873523E-3,-9.76779E-4,4.8460853E-3,-3.942172E-2,7.549796E-2,1.7108953E-1,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,-1,39,-1,-1,41,43,45,47,-1,-1,-1,49,-1,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.158347E0,4.18929E-1,7.998216E-2,3.5195512E-1,3.1221244E-1,0E0,0E0,2.0742762E-1,5.744326E-2,1.3263142E-1,2.764765E-1,5.5647433E-2,1.537655E-1,0E0,0E0,6.787142E-2,7.104939E-2,1.143521E-1,1.2254682E-1,3.3913255E-2,0E0,3.627623E-2,1.603502E-1,3.7099797E-2,0E0,2.7427554E-2,0E0,0E0,4.950477E-2,8.2423344E-2,6.474209E-2,3.1638563E-2,0E0,0E0,0E0,9.902678E-2,0E0,0E0,0E0,5.4692864E-2,1.5442751E-2,0E0,0E0,0E0,3.445577E-2,8.0237605E-2,6.781274E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,28,28,29,29,30,30,31,31,35,35,39,39,40,40,44,44,45,45,46,46],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,-1,40,-1,-1,42,44,46,48,-1,-1,-1,50,-1,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3060927E0,1.04201E-2,5.9165E4,1.810504E10,5.470729E2,7.489175E-3,1.9760614E-2,5.162122E2,5.752E3,1.0275E4,3.8834286E0,1E0,9.8298E4,-8.463386E-3,-2.1037353E-2,2.8945708E6,4E1,1.4139E4,3.996671E5,3.5523141E-3,-7.512345E-4,2.7123368E10,5.347626E5,7.751E3,-2.82216E-3,7.2899837E0,6.4482796E-4,4.802586E-3,1.1334876E0,7.6420575E6,2.6673E4,3.0353E4,-3.7551587E-3,5.5987155E-4,6.6966885E-3,8.591E3,-8.077933E-3,6.7342306E-3,6.569003E-4,5.8910837E0,5.69434E-1,-6.873523E-3,-9.76779E-4,4.8460853E-3,5.754E3,7.6221466E8,7.6E0,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"split_indices":[36,39,2,5,53,0,0,56,2,9,54,105,30,0,0,46,3,10,29,40,0,32,29,9,0,54,0,0,40,46,10,12,0,0,0,2,0,0,0,54,28,0,0,0,2,7,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.77E2,9E0,6.1E1,1.16E2,4E0,5E0,5.4E1,7E0,5E1,6.6E1,1.9E1,3.5E1,5E0,2E0,1.4E1,3.6E1,1.7E1,4.9E1,1.6E1,3E0,9E0,2.6E1,9E0,5E0,3E1,6E0,3E0,1.4E1,1.6E1,3.3E1,1.1E1,5E0,4E0,5E0,1.8E1,8E0,4E0,5E0,2.1E1,9E0,8E0,6E0,6E0,1E1,1.8E1,1.5E1,3E0,8E0,1.5E1,3E0,1E1,1.1E1,5E0,4E0,3E0,7E0,1.1E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[3.5595038E-4,-4.4311643E-2,2.0678306E-1,-1.9426705E-2,-1.8886262E-1,3.1928593E-1,5.56452E-2,-7.896872E-2,2.8734265E-2,-9.082003E-2,-3.8585064E-1,2.895736E-2,2.4027427E-1,1.074728E-1,-3.3747476E-2,-1.00550115E-1,1.675159E-2,7.017641E-2,-3.8212094E-2,-1.407528E-1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.6446205E-1,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,-1.5302381E-1,-7.644423E-2,7.8863114E-2,-4.7753017E-2,1.0884225E-1,-1.3557769E-2,-5.902727E-2,4.1850857E-3,-9.2872605E-3,-4.909929E-2,1.4215367E-2,5.618002E-3,-3.536953E-3,-2.0744017E-1,-4.741403E-2,-1.308574E-1,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.7041807E-2,1.5840885E-1,-8.000341E-2,2.9958343E-2,-1.2175934E-1,-2.2903696E-2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9178276E0,6.114423E-1,6.1136055E-1,4.238065E-1,4.477874E-1,3.14538E-1,8.238807E-2,1.3802564E-1,2.3018244E-1,2.9552594E-1,3.0219913E-2,0E0,4.197371E-2,6.660308E-2,4.066568E-2,5.66082E-2,5.5850185E-2,1.6659597E-1,8.780883E-2,6.407347E-2,0E0,0E0,0E0,0E0,2.8056264E-2,0E0,0E0,0E0,0E0,5.4878265E-2,5.5400893E-2,1.0032747E-2,1.9345354E-2,9.9761456E-2,5.1551525E-2,6.0662225E-2,0E0,0E0,1.02921E-2,0E0,0E0,0E0,4.0799797E-2,7.5311735E-2,1.8520892E-2,0E0,0E0,0E0,0E0,3.455124E-2,3.763908E-2,3.8690723E-2,1.8420529E-2,3.3416137E-2,2.2927333E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,6.6932364E7,5.9337012E7,3.3969492E2,1.4792529E10,5.0655737E0,1.2055723E8,2E0,2.2971932E7,2E1,1.8942408E7,2.895736E-2,2.819228E-1,2.1516852E5,9.4908775E1,1.142E3,1.06403265E1,9.607843E-1,1.4792529E10,5.9862564E1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.5868E4,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,5.244755E-3,7.44E2,9.950143E7,1.2067E4,5.6426154E2,3.634E3,2.2502415E1,4.1850857E-3,-9.2872605E-3,1.2281632E3,1.4215367E-2,5.618002E-3,-3.536953E-3,5.322348E7,1.4300815E6,3.1001872E5,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.58881E5,2.94E2,5.4452E4,6.568221E0,2.88446E5,9.38E2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"split_indices":[53,46,46,53,5,55,46,8,46,8,33,0,58,34,57,2,55,58,5,57,0,0,0,0,9,0,0,0,0,58,0,12,9,53,2,57,0,0,53,0,0,0,5,33,29,0,0,0,0,1,0,30,54,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.7E2,3.6E1,1.46E2,2.4E1,2E1,1.6E1,6.5E1,8.1E1,1.7E1,7E0,3E0,1.7E1,1E1,6E0,5.3E1,1.2E1,5E1,3.1E1,1.5E1,2E0,5E0,2E0,3E0,1.4E1,3E0,7E0,3E0,3E0,1.5E1,3.8E1,6E0,6E0,3.4E1,1.6E1,2.7E1,4E0,9E0,6E0,1.1E1,3E0,7E0,8E0,2.6E1,1.2E1,4E0,2E0,3E0,3E0,1.6E1,1.8E1,6E0,1E1,9E0,1.8E1,3E0,3E0,3E0,5E0,1.4E1,1.2E1,9E0,3E0,9E0,7E0,3E0,1.5E1,2E0,4E0,6E0,4E0,3E0,6E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[2.6763344E-4,-3.562319E-2,1.7865828E-1,-1.8097268E-2,-2.4454215E-1,4.47172E-1,1.16943076E-1,-3.3486996E-2,1.414897E-1,-3.481418E-1,3.6788161E-3,2.790042E-2,6.3239834E-3,-1.0595338E-2,1.9547185E-1,-6.566554E-2,2.4707085E-2,1.8834342E-1,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,5.0848205E-2,2.2159144E-1,4.051122E-3,-5.639541E-2,-1.0586935E-2,-4.5325225E-3,4.210238E-2,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,-6.8116255E-2,7.0792675E-2,7.365901E-2,-2.802525E-2,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,37,-1,-1,-1,-1,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.299687E0,6.143675E-1,5.111331E-1,3.9033002E-1,4.5747256E-1,1.6556728E-1,2.925187E-1,2.7127695E-1,7.63599E-2,5.6399226E-2,0E0,0E0,0E0,1.5918176E-1,3.4319043E-2,1.1866668E-1,1.1051539E-1,3.6700696E-2,0E0,0E0,0E0,0E0,5.9108846E-2,4.338783E-2,0E0,1.3749906E-1,0E0,0E0,1.0386192E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3594824E-1,6.001374E-2,6.339806E-2,7.879637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,22,22,23,23,25,25,28,28,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,38,-1,-1,-1,-1,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,1.8000048E10,1.4164306E-3,7.918757E7,3.582157E12,8.50999E9,1E0,2.0053221E6,1.2117E4,6.1826086E0,3.6788161E-3,2.790042E-2,6.3239834E-3,5.9084116E3,7.1528664E0,4.187504E7,5.6865673E0,8.81832E5,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,7.644773E-1,2.819228E-1,4.051122E-3,1E0,-1.0586935E-2,-4.5325225E-3,6.773399E0,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,1.2936847E4,6.7111494E8,2.7883E4,3.4288502E8,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"split_indices":[53,5,58,33,32,5,102,49,9,54,0,0,0,4,54,33,55,1,0,0,0,0,35,58,0,89,0,0,54,0,0,0,0,0,0,52,5,9,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.68E2,3.3E1,1.56E2,1.2E1,5E0,2.8E1,1.43E2,1.3E1,9E0,3E0,3E0,2E0,1.1E1,1.7E1,9.2E1,5.1E1,9E0,4E0,6E0,3E0,2E0,9E0,1.3E1,4E0,8.8E1,4E0,6E0,4.5E1,4E0,5E0,7E0,2E0,3E0,1E1,8.1E1,7E0,3.1E1,1.4E1,2.4E1,5.7E1,4E0,3E0,2.4E1,7E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[7.671295E-3,-4.4080827E-2,1.6683623E-1,-2.5097556E-2,-3.2062513E-1,2.2227801E-1,-1.2322681E-2,-1.1087152E-1,-5.909076E-3,-4.2326204E-3,-1.9872338E-2,1.3815795E-1,3.3456573E-1,-9.755224E-2,3.63379E-3,-1.355743E-1,1.2135234E-3,-9.3863435E-2,9.225092E-3,9.646266E-2,1.2028726E-2,3.6554006E-1,3.8201683E-3,-7.6256203E-3,-6.825402E-4,-9.017518E-2,-2.0942573E-1,-7.649883E-3,-5.741548E-2,5.0508834E-2,-2.6426286E-2,1.13049895E-1,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,-1.8938167E-2,9.144381E-2,1.4660759E-2,-8.3531834E-2,1.4068319E-3,1.457002E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7124388E0,8.161584E-1,5.0931644E-1,2.4293393E-1,1.6942883E-1,3.2215738E-1,1.0366744E-1,9.3960494E-2,1.6326205E-1,0E0,0E0,8.524135E-2,9.821105E-2,3.2380693E-2,0E0,5.822915E-2,0E0,3.252116E-2,1.558761E-1,4.3112665E-2,0E0,9.481287E-2,0E0,0E0,0E0,2.1951303E-2,2.0213217E-2,0E0,1.926352E-2,1.4102952E-1,1.3496363E-1,4.2491734E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.587293E-2,1.4206815E-1,5.812499E-2,9.924857E-2,0E0,3.1057805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,19,19,21,21,25,25,26,26,28,28,29,29,30,30,31,31,41,41,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.8000048E10,9.298568E7,1.4862166E1,5.709406E7,5.8038013E3,9.4908775E1,3.068397E6,2.0293878E2,-4.2326204E-3,-1.9872338E-2,1.197355E6,5E1,6.97021E3,3.63379E-3,8.3949E4,1.2135234E-3,8.587809E4,7.9762895E6,1.127938E6,1.2028726E-2,6.677E3,3.8201683E-3,-7.6256203E-3,-6.825402E-4,1.3868161E5,1.0595825E3,-7.649883E-3,1.96897E5,2.495E3,4.497684E1,4.078932E5,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,2.7309215E6,5.511358E1,5.8454223E0,1.6594656E3,1.4068319E-3,2.819228E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"split_indices":[53,5,46,59,46,53,57,12,53,0,0,29,3,4,0,12,0,29,46,29,0,9,0,0,0,29,34,0,1,2,59,29,0,0,0,0,0,0,0,0,0,46,59,54,4,0,58,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.56E2,5E1,1.47E2,9E0,3.8E1,1.2E1,2.6E1,1.21E2,3E0,6E0,2.3E1,1.5E1,6E0,6E0,2.2E1,4E0,1.7E1,1.04E2,1.8E1,5E0,1.3E1,2E0,3E0,3E0,1.5E1,7E0,5E0,1.2E1,4.8E1,5.6E1,1.6E1,2E0,2E0,1.1E1,2E0,1.3E1,5E0,2E0,1E1,2E0,1.8E1,3E1,3.3E1,2.3E1,5E0,1.1E1,6E0,1.2E1,1.4E1,1.6E1,7E0,2.6E1,1.3E1,1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.0088654E-2,-4.6401702E-2,1.01957306E-1,-3.6083024E-2,-2.5793928E-1,3.6505505E-2,1.9830973E-1,-8.227722E-2,2.2556398E-2,-2.2164036E-3,-1.9751323E-2,1.0576863E-1,-6.8776435E-3,1.892742E-2,1.5552393E-1,-6.712239E-2,-1.4856316E-2,-5.168168E-2,6.346703E-2,6.68196E-3,1.6625825E-3,2.28483E-2,-3.8955654E-3,1.8000433E-1,1.2089454E-3,-7.764623E-2,3.5982527E-2,-1.81113E-2,-1.3062242E-1,-3.4493096E-2,1.2884454E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,-1.4065999E-1,-6.0010847E-2,-2.1526885E-4,3.771369E-3,-5.376633E-2,7.684118E-4,-8.152548E-3,-1.2452523E-3,-7.3128425E-2,5.791923E-3,1.6153444E-1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,-1,27,29,-1,-1,31,-1,33,-1,35,37,39,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.306833E-1,3.268422E-1,3.0808258E-1,4.0275842E-1,2.0142797E-1,9.3877874E-2,1.0262287E-1,2.599991E-1,2.0373048E-1,0E0,0E0,2.2404373E-2,4.5329027E-2,0E0,5.311328E-2,8.85137E-2,0E0,6.123595E-2,2.7983302E-1,0E0,0E0,9.2295585E-3,0E0,3.8849056E-2,0E0,7.1570426E-2,1.3761323E-2,2.2606054E-2,2.2841543E-2,1.1545663E-1,9.22758E-2,0E0,0E0,0E0,0E0,5.3088874E-2,9.419921E-2,0E0,0E0,2.1270107E-2,0E0,0E0,0E0,8.4302165E-2,0E0,8.3529234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,25,25,26,26,27,27,28,28,29,29,30,30,35,35,36,36,39,39,43,43,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,-1,28,30,-1,-1,32,-1,34,-1,36,38,40,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.537424E0,1.4146589E8,1.2236866E3,6.978001E2,1.5214152E5,1.1244E4,4.9695907E0,8.260109E9,3.41673E5,-2.2164036E-3,-1.9751323E-2,5.6352734E1,8.412314E0,1.892742E-2,4.8420677E-1,3.4741312E8,-1.4856316E-2,8.062193E-1,6.6217735E-2,6.68196E-3,1.6625825E-3,4.9852173E1,-3.8955654E-3,1.08E2,1.2089454E-3,8.8916E4,1.09524E5,4.737429E8,4.6E1,1E0,1.5107028E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,9.135018E0,5.580003E8,-2.1526885E-4,3.771369E-3,9.042926E3,7.684118E-4,-8.152548E-3,-1.2452523E-3,9.2312753E-1,5.791923E-3,1.234543E1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"split_indices":[43,46,53,53,34,9,55,5,29,0,0,59,55,0,39,7,0,28,58,0,0,59,0,3,0,1,30,7,3,89,39,0,0,0,0,59,5,0,0,34,0,0,0,28,0,55,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.53E2,4.9E1,1.47E2,6E0,3E1,1.9E1,8.2E1,6.5E1,3E0,3E0,1.1E1,1.9E1,2E0,1.7E1,7.8E1,4E0,2.3E1,4.2E1,7E0,4E0,1.4E1,5E0,1.4E1,3E0,7.1E1,7E0,1.7E1,6E0,1.7E1,2.5E1,4E0,1E1,1.2E1,2E0,1.4E1,5.7E1,4E0,3E0,8E0,9E0,4E0,2E0,1.4E1,3E0,1.9E1,6E0,6E0,8E0,2.4E1,3.3E1,3E0,5E0,8E0,6E0,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.7253997E-2,-2.9623577E-2,9.3434505E-2,-2.2292453E-1,-1.1781458E-2,1.2750259E-1,-1.9930653E-1,-3.3444623E-3,-1.2307007E-2,1.7842796E-2,-6.838117E-2,8.674666E-2,3.2339618E-1,-8.01149E-2,-1.8067626E-2,-4.725675E-3,3.552434E-2,-1.2093892E-1,-3.2683358E-2,2.175943E-3,1.3742693E-1,2.7282907E-2,2.0839193E-1,-5.710558E-3,-1.8373676E-4,4.8699785E-2,-4.864452E-3,-1.5385593E-1,3.9161125E-4,1.3277833E-3,-5.8980435E-2,6.1291102E-2,-4.3464635E-2,4.9143904E-3,1.6467385E-1,1.2900099E-2,5.3549507E-3,1.10007E-1,1.44394925E-2,-1.135703E-2,-1.014236E-1,-6.185148E-3,-1.7182616E-3,1.0451842E-1,7.325834E-4,-7.334734E-2,9.2549727E-4,-2.225592E-3,2.6441324E-3,2.1629092E-1,7.352542E-2,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,35,-1,-1,37,-1,39,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,-1,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.848589E-1,4.077926E-1,7.5345045E-1,2.8820753E-2,1.8565977E-1,4.9887896E-1,1.3421485E-1,0E0,0E0,1.5181659E-1,6.734279E-2,2.3972067E-1,1.832602E-1,1.6607814E-2,0E0,0E0,1.1910778E-1,6.492545E-2,3.9455958E-2,6.2021095E-2,1.2556028E-1,0E0,2.1007508E-2,0E0,0E0,1.2233533E-1,0E0,2.6357263E-2,0E0,0E0,2.2924516E-2,1.8414091E-2,2.566748E-2,2.000419E-2,1.20536804E-1,0E0,0E0,4.2832226E-2,5.8858946E-2,0E0,2.2556297E-2,0E0,0E0,1.198972E-2,0E0,1.4668021E-2,0E0,0E0,0E0,3.9754987E-2,2.4631858E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,22,22,25,25,27,27,30,30,31,31,32,32,33,33,34,34,37,37,38,38,40,40,43,43,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,36,-1,-1,38,-1,40,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,-1,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,1.6770158E-4,1.2055723E8,1.9E1,7.183439E0,8.258924E3,3.312524E6,-3.3444623E-3,-1.2307007E-2,1.056E3,7.8206635E0,1E0,1.6232848E7,1.5574268E7,-1.8067626E-2,-4.725675E-3,4.759155E1,2.2235178E1,2.7E1,9.32784E5,1.1114967E-3,2.7282907E-2,1.5307794E2,-5.710558E-3,-1.8373676E-4,1.2029E4,-4.864452E-3,2.9239766E-2,3.9161125E-4,1.3277833E-3,1.1079511E1,8.78E2,3.0558723E-1,2.69336E3,8.637931E1,1.2900099E-2,5.3549507E-3,4E1,4.711463E5,-1.135703E-2,3.4996462E-1,-6.185148E-3,-1.7182616E-3,5.88E2,7.325834E-4,9.042926E3,9.2549727E-4,-2.225592E-3,2.6441324E-3,1.5213319E0,4.6379595E6,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"split_indices":[53,40,46,3,54,53,30,0,0,2,54,102,1,1,0,0,57,57,3,30,39,0,57,0,0,9,0,58,0,0,55,0,40,4,59,0,0,10,52,0,58,0,0,0,0,34,0,0,0,58,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.18E2,7.2E1,9E0,1.09E2,6.5E1,7E0,2E0,7E0,7.2E1,3.7E1,5.5E1,1E1,5E0,2E0,9E0,6.3E1,1.4E1,2.3E1,2.1E1,3.4E1,2E0,8E0,3E0,2E0,5.8E1,5E0,1.1E1,3E0,7E0,1.6E1,9E0,1.2E1,6E0,2.8E1,4E0,4E0,2E1,3.8E1,3E0,8E0,3E0,1.3E1,4E0,5E0,8E0,4E0,3E0,3E0,1.7E1,1.1E1,3E0,1.7E1,1.1E1,2.7E1,2E0,6E0,2E0,2E0,2E0,6E0,1.2E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.821953E-2,-5.8290947E-2,5.3941235E-2,-1.5808642E-2,-4.884159E-2,-6.909937E-2,1.1836419E-1,-1.4827322E-2,-8.093995E-2,-2.6398048E-2,-1.7481435E-2,4.550153E-2,1.5699527E-1,-6.594871E-2,3.347421E-2,-9.2051856E-2,-6.509846E-3,6.4362705E-2,-7.304391E-2,8.8149734E-2,-1.3496414E-2,1.2912967E-2,1.3428916E-1,-1.2285006E-1,-2.9534832E-2,9.6844934E-2,3.1477788E-3,-6.093492E-2,-1.3656648E-1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,-1.4490403E-1,5.8896925E-2,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.758437E-2,1.9397938E-1,-1.2398881E-3,-9.721017E-3,-4.254263E-2,2.629008E-3,6.676271E-3,9.927296E-5,-2.7347464E-2,5.9932888E-2,-1.4137745E-1,-3.3839725E-2,-1.0662791E-3,-1.5445867E-1,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,1.0476206E-1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,63,65,67,69,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.496537E-1,2.9141405E-1,5.4719037E-1,0E0,1.2872437E-1,2.878313E-1,1.1952078E-1,1.4789282E-1,4.9851447E-2,9.7840816E-2,0E0,4.396437E-2,3.570825E-2,5.645959E-2,5.92712E-2,6.509176E-2,3.340755E-2,1.0004889E-2,1.5075365E-1,1.9013055E-2,1.2354755E-2,0E0,7.2952986E-2,7.7276245E-2,2.294885E-2,3.7963897E-2,3.9735876E-2,6.714282E-2,3.9696425E-2,0E0,0E0,0E0,0E0,4.4010997E-2,2.6443997E-2,0E0,0E0,0E0,0E0,4.359097E-2,9.496659E-3,0E0,0E0,1.2809919E-2,0E0,0E0,0E0,2.1849878E-2,1.7759107E-2,1.1468753E-2,4.3800417E-2,0E0,9.729177E-3,0E0,0E0,0E0,0E0,1.81911E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,39,39,40,40,43,43,47,47,48,48,49,49,50,50,52,52,57,57],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,64,66,68,70,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,5.243E3,1E0,-1.5808642E-2,7.6420575E6,6.3639E4,4.0679638E6,2.802166E2,5.754E3,1.4364E4,-1.7481435E-2,1.9071735E6,7.573E3,1.822764E4,4.585366E-2,3E0,5.178571E0,7.406885E8,1.0807868E7,9.1797715E-1,3.508923E6,1.2912967E-2,1.56E4,1.17E2,3.65102E5,1.7027842E8,2.2583E4,1.8716404E10,7E1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,2.6654E4,1.00152E5,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.8114974E-1,4.2414474E0,-1.2398881E-3,-9.721017E-3,1.0437421E8,2.629008E-3,6.676271E-3,9.927296E-5,1.8465776E8,8.3154816E2,1.1272E4,2.7048333E10,-1.0662791E-3,7.64764E5,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,3.6E1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"split_indices":[53,9,102,0,46,10,33,53,2,10,0,33,9,34,58,8,55,7,1,28,33,0,2,10,1,33,10,32,10,0,0,0,0,2,2,0,0,0,0,28,58,0,0,7,0,0,0,5,4,30,32,0,9,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,1.21E2,6.7E1,3E0,1.18E2,2.3E1,4.4E1,5.8E1,6E1,2.1E1,2E0,1.6E1,2.8E1,2.8E1,3E1,5.2E1,8E0,7E0,1.4E1,9E0,7E0,3E0,2.5E1,1E1,1.8E1,9E0,2.1E1,3.2E1,2E1,4E0,4E0,5E0,2E0,9E0,5E0,6E0,3E0,4E0,3E0,1.4E1,1.1E1,5E0,5E0,1.6E1,2E0,6E0,3E0,1.4E1,7E0,7E0,2.5E1,3E0,1.7E1,6E0,3E0,3E0,2E0,1.1E1,3E0,8E0,3E0,8E0,8E0,6E0,8E0,3E0,4E0,3E0,4E0,3E0,2.2E1,1.5E1,2E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.81344E-3,-2.7871855E-2,1.0370427E-1,-1.6869712E-1,-1.2145221E-2,1.3553725E-1,-1.1319494E-2,-6.7867875E-2,-2.5078225E-1,1.1689531E-2,-7.812123E-2,2.2571503E-1,4.7493696E-2,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,-2.2323536E-2,5.4143388E-2,-9.7720034E-2,1.1783024E-4,1.8657329E-2,1.5811518E-1,9.230261E-2,-1.6328286E-3,-4.379925E-2,5.0750427E-2,1.1329648E-1,1.5480721E-2,-1.1520765E-1,-1.8632218E-3,8.1961945E-2,9.722441E-3,1.8049882E-3,8.147564E-3,-2.9605734E-3,1.3710238E-3,-1.09546825E-1,-2.015405E-2,-1.4587996E-3,6.54756E-2,1.231898E-1,7.869888E-4,7.951029E-2,-2.6912533E-2,-1.2801106E-1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,37,39,41,43,45,-1,47,-1,49,-1,-1,-1,51,53,-1,55,57,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8306116E-1,3.744514E-1,4.811502E-1,1.2368178E-1,2.4308157E-1,2.989232E-1,0E0,2.157684E-2,3.9408505E-2,1.6582306E-1,6.4956516E-2,1.4187676E-1,4.637814E-2,0E0,0E0,0E0,0E0,1.0265903E-1,1.14883915E-1,2.9888332E-2,0E0,0E0,3.1216264E-2,7.44717E-2,2.112299E-2,7.5522915E-2,1.9793212E-2,1.7225742E-2,8.900131E-2,4.644516E-2,0E0,3.160474E-2,0E0,1.4609658E-2,0E0,0E0,0E0,6.457342E-2,9.696657E-2,0E0,1.6802516E-2,1.4416695E-2,0E0,1.5758365E-2,1.763527E-2,4.027137E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,37,37,38,38,40,40,41,41,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,38,40,42,44,46,-1,48,-1,50,-1,-1,-1,52,54,-1,56,58,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.178571E0,1.3578947E1,8.062193E-1,7.3808947E0,5.861021E7,-1.1319494E-2,3.662451E6,2.692487E6,5.716463E2,2.770862E8,6.661E3,2.1516852E5,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,3.9E1,1.26881E5,1.6259928E1,1.1783024E-4,1.8657329E-2,6.917027E3,4.078932E5,6.47008E1,1.4300815E6,2.1229417E-4,2.4025E4,3.6E1,4.2845196E1,-1.8632218E-3,5.59076E3,9.722441E-3,6.804E3,8.147564E-3,-2.9605734E-3,1.3710238E-3,2.07533E5,1.2670352E1,-1.4587996E-3,1.43E2,2.458E3,7.869888E-4,1E0,4.983287E-2,5.3E1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"split_indices":[53,55,55,28,54,46,0,1,1,53,7,9,34,0,0,0,0,3,30,55,0,0,4,29,57,33,39,9,3,57,0,4,0,9,0,0,0,1,55,0,10,2,0,84,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.69E2,4.1E1,1.6E1,1.53E2,3.8E1,3E0,8E0,8E0,1.13E2,4E1,1.8E1,2E1,6E0,2E0,4E0,4E0,6.3E1,5E1,3.2E1,8E0,4E0,1.4E1,1E1,1E1,4.9E1,1.4E1,1.9E1,3.1E1,2.4E1,8E0,6E0,8E0,5E0,5E0,3E0,7E0,1.2E1,3.7E1,2E0,1.2E1,1.7E1,2E0,1.2E1,1.9E1,2.2E1,2E0,4E0,2E0,2E0,3E0,7E0,5E0,3.5E1,2E0,4E0,8E0,4E0,1.3E1,1E1,2E0,3E0,1.6E1,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.1425839E-2,-5.037939E-2,8.030544E-2,-3.6732145E-2,-1.940263E-1,2.1132613E-2,5.40595E-2,-8.394264E-2,-1.2433366E-2,-2.411859E-3,-2.3667553E-1,-7.583577E-2,1.2632278E-1,-9.6114695E-2,3.007908E-3,3.00965E-2,-5.099469E-2,-1.4926101E-2,-5.8065807E-3,-2.905628E-2,-1.687885E-2,9.731201E-4,1.4231151E-1,-1.0521082E-1,6.9725106E-4,-3.51654E-2,6.765553E-2,-1.512059E-2,-1.1731359E-1,-7.190969E-2,1.2920251E-3,7.135396E-2,1.5594196E-1,-5.4826945E-2,-1.3231695E-1,1.7662038E-3,-8.1669055E-2,-1.3722117E-3,8.081185E-2,-6.1539885E-2,5.3886116E-2,-1.3972227E-1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,1.7186429E-1,7.422348E-2,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,33,-1,35,37,39,41,43,-1,45,47,49,51,-1,53,-1,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1869767E-1,2.692873E-1,5.338764E-1,1.4808676E-1,6.220767E-2,0E0,5.446193E-1,8.3999515E-2,1.4421546E-1,0E0,3.7372112E-2,2.5613624E-1,6.0055733E-2,4.269564E-2,0E0,1.05328426E-1,1.0766618E-1,0E0,0E0,4.71834E-2,0E0,0E0,1.9822717E-2,4.4642508E-2,0E0,5.613244E-2,3.6327116E-2,1.0261665E-1,3.450933E-2,6.169112E-2,0E0,1.8945247E-2,2.107352E-2,2.1910824E-2,4.9286067E-2,0E0,3.790599E-2,0E0,1.6026601E-2,3.391008E-2,3.6774833E-2,1.0349661E-2,0E0,0E0,0E0,0E0,0E0,1.815337E-2,3.8996745E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,19,19,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,38,38,39,39,40,40,41,41,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,34,-1,36,38,40,42,44,-1,46,48,50,52,-1,54,-1,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3453049E3,8.67241E9,1.4164306E-3,4.9054803E2,4.944487E7,2.1132613E-2,1E0,1.4864864E0,9.785595E6,-2.411859E-3,6.096518E0,4.4816768E8,1.6463401E3,9.741151E-1,3.007908E-3,2.4242616E5,8.78E2,-1.4926101E-2,-5.8065807E-3,3.0687964E3,-1.687885E-2,9.731201E-4,2.8626094E5,5.2646E4,6.9725106E-4,1.3585858E1,1.29E2,3.307175E-2,3.170081E6,7.6599895E9,1.2920251E-3,2.6295085E3,3.8014093E2,6.492337E0,2.0289855E-2,1.7662038E-3,1.1565725E0,-1.3722117E-3,1.2075471E7,1.2067E4,1.2117E4,4E1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,3.2014463E0,1.0807868E7,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"split_indices":[53,5,58,56,46,0,102,58,46,0,54,45,53,28,0,29,0,0,0,53,0,0,29,1,0,57,0,58,1,5,0,4,59,54,58,0,43,0,49,9,9,3,0,0,0,0,0,36,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.4E2,5.9E1,1.29E2,1.1E1,3E0,5.6E1,4.3E1,8.6E1,3E0,8E0,2E1,3.6E1,4E1,3E0,4.1E1,4.5E1,4E0,4E0,1.8E1,2E0,5E0,3.1E1,3.7E1,3E0,1.5E1,2.6E1,3E1,1.5E1,1E1,8E0,6E0,2.5E1,1.4E1,2.3E1,6E0,9E0,3E0,2.3E1,1.8E1,1.2E1,1.2E1,3E0,6E0,4E0,2E0,4E0,2E1,5E0,5E0,9E0,1.2E1,1.1E1,6E0,3E0,2E1,3E0,1.3E1,5E0,6E0,6E0,1E1,2E0,1.7E1,3E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[4.6584974E-3,-1.20946085E-2,1.9950889E-1,-1.0169338E-1,3.8820638E-3,3.8749582E-1,8.888968E-2,-5.8970988E-2,-1.9482663E-1,-9.452532E-2,1.551003E-2,2.2913612E-2,7.1646026E-3,1.3861427E-1,-4.1111445E-4,-8.537462E-2,2.566781E-2,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,-1.250599E-1,-3.5212412E-2,4.8972625E-2,2.1056903E-3,7.941639E-3,-4.738872E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,-8.0461E-2,-1.16186915E-2,8.0707185E-2,-5.17336E-2,5.499706E-2,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,-4.0313277E-2,4.2702768E-2,1.3941039E-1,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009837E-1,2.8382987E-1,3.1906998E-1,1.07877135E-1,1.9484903E-1,5.082476E-2,5.925162E-2,5.135636E-2,2.9909045E-2,1.1930962E-1,2.5971475E-1,0E0,0E0,1.16812885E-2,0E0,1.3007693E-2,1.56768E-2,0E0,0E0,0E0,6.1911702E-2,1.2099819E-1,1.173628E-1,0E0,0E0,1.1479644E-2,0E0,0E0,0E0,1.0476619E-2,0E0,6.582728E-2,7.438664E-2,8.7449014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.335335E-2,8.5745096E-2,9.818435E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,20,20,21,21,22,22,25,25,29,29,31,31,32,32,33,33,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1.4862166E1,2E0,1E0,4.060294E0,9.886E3,6.627576E7,1.3448806E8,8.3949E4,7.56E2,4.2658337E2,2.2913612E-2,7.1646026E-3,1.625E3,-4.1111445E-4,8.351808E-1,9.282E3,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,5.936149E1,2.124077E1,6.7948157E-1,2.1056903E-3,7.941639E-3,2.2970924E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,1E0,-1.16186915E-2,1.2623029E7,1.38E2,1.6974416E6,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,5.138372E8,3.810132E-2,1.2951E4,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"split_indices":[53,59,8,84,54,9,46,7,12,0,53,0,0,0,0,28,9,0,0,0,57,59,35,0,0,28,0,0,0,65,0,12,0,29,0,0,0,0,0,0,0,0,7,28,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.97E2,1.6E1,2.9E1,1.68E2,5E0,1.1E1,2.1E1,8E0,1.7E1,1.51E2,3E0,2E0,7E0,4E0,1.6E1,5E0,2E0,6E0,2E0,1.5E1,6E1,9.1E1,2E0,5E0,8E0,8E0,3E0,2E0,1.2E1,3E0,7E0,5.3E1,8.9E1,2E0,4E0,4E0,2E0,1E1,5E0,2E0,3E0,5E1,7.9E1,1E1,4.8E1,2E0,4E0,7.5E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-9.7492535E-4,2.4562754E-2,-1.2644652E-1,1.0812377E-2,3.050939E-1,-1.6741821E-1,3.276021E-2,-4.319679E-2,5.309041E-2,1.990497E-2,7.832562E-3,-1.8774702E-1,3.1424365E-3,3.0188295E-3,-1.7861315E-3,-2.2519317E-2,-9.010733E-2,8.511394E-2,-1.3702277E-2,-1.5522692E-1,-1.8752424E-2,-3.5709422E-2,8.45564E-2,-1.2079619E-1,5.7729695E-4,3.1312965E-2,1.5568885E-1,2.2016572E-2,-9.091352E-2,-1.9204946E-1,-1.8670933E-3,-5.38557E-2,2.1705188E-2,5.703837E-3,6.2500284E-4,-1.3545471E-1,-3.5225743E-4,8.462742E-2,-2.3649544E-2,1.15246095E-1,1.1550232E-2,-3.8249083E-2,4.199171E-2,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,27,29,-1,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.63182E-1,6.601755E-1,2.3378861E-1,3.7905455E-1,5.4006636E-2,1.4417273E-1,1.8542241E-2,6.878023E-2,2.0008749E-1,0E0,0E0,1.0724431E-1,0E0,0E0,0E0,7.6461144E-2,7.027824E-2,2.343359E-1,8.749639E-2,9.590763E-2,0E0,5.013643E-2,1.2541946E-2,2.7317584E-2,0E0,1.1042236E-1,6.1801553E-2,2.8090071E-2,2.3553073E-2,3.9512277E-2,0E0,1.5470095E-2,4.2363483E-2,0E0,0E0,2.9321939E-2,0E0,1.5620723E-2,5.383783E-2,5.4485172E-2,0E0,1.0460219E-2,2.2912873E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,37,37,38,38,39,39,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,28,30,-1,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,8.861832E3,1.4539318E-1,4.2658337E2,1.04E3,1E0,1.3507566E8,1.3839568E1,2.4934822E7,1.990497E-2,7.832562E-3,3.074E3,3.1424365E-3,3.0188295E-3,-1.7861315E-3,1.91126E5,7.407092E0,5.370622E5,1.06403265E1,8.66076E1,-1.8752424E-2,3.8E1,4.61E2,5.979E3,5.7729695E-4,9.6494156E-1,1.8907035E3,4.602015E0,1.27718E6,3.27E2,-1.8670933E-3,2.4397528E8,4.3E1,5.703837E-3,6.2500284E-4,1.7218965E2,-3.5225743E-4,8.047134E4,4.649E3,7.4245725E0,1.1550232E-2,8.062193E-1,3.706834E-4,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"split_indices":[46,4,39,53,0,89,46,57,46,0,0,0,0,0,0,30,54,29,55,57,0,3,0,2,0,58,4,54,29,0,0,32,10,0,0,53,0,34,2,55,0,28,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.71E2,3.4E1,1.64E2,7E0,2.7E1,7E0,7.2E1,9.2E1,3E0,4E0,2.5E1,2E0,5E0,2E0,5.1E1,2.1E1,6.2E1,3E1,2.3E1,2E0,4.6E1,5E0,1.6E1,5E0,3.6E1,2.6E1,2.1E1,9E0,1.7E1,6E0,3.5E1,1.1E1,3E0,2E0,1.4E1,2E0,1.8E1,1.8E1,1.9E1,7E0,5E0,1.6E1,7E0,2E0,4E0,1.3E1,2E0,3.3E1,4E0,7E0,4E0,1E1,8E0,1E1,7E0,1.1E1,8E0,1.1E1,2E0,3E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[6.475444E-3,-1.306737E-2,1.7832997E-1,2.339655E-3,-1.5614031E-1,2.8744474E-1,5.3003386E-2,-1.6489496E-2,6.7189105E-2,-2.4180317E-1,1.30160665E-2,3.55285E-1,3.888601E-3,9.1399415E-4,6.7161336E-3,-1.0044899E-2,-1.06382035E-2,1.3008821E-2,1.2666671E-1,-2.8520468E-1,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,-1.6523695E-2,1.5263823E-1,-4.6523552E-2,7.135857E-2,1.455868E-1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,-2.6452178E-2,7.834649E-2,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,-1,33,35,37,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9082713E-1,4.0816608E-1,2.7193904E-1,2.06246E-1,2.639749E-1,1.2648803E-1,2.9406311E-2,1.6842958E-1,1.2080774E-1,5.8472157E-2,2.9649243E-2,6.8154216E-2,0E0,0E0,0E0,1.3743296E-1,0E0,7.626754E-2,3.144613E-2,2.2196114E-2,0E0,0E0,0E0,0E0,0E0,1.1887014E-1,4.850761E-2,1.9805137E-2,2.4082582E-2,1.8560827E-2,0E0,0E0,0E0,7.060035E-2,5.3812273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,-1,34,36,38,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.993295E3,1.8000048E10,5.9337012E7,1.5085194E0,1.5461028E12,6.6737964E2,6.543368E6,1.3201905E0,7.7567725E2,4.577256E3,2.6023E4,1.04E3,3.888601E-3,9.1399415E-4,6.7161336E-3,2.3329E4,-1.06382035E-2,2.2906428E5,6.4487465E6,5.709406E7,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,1E0,9.39E2,3E1,5.1E2,3.5E1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,4.56567E7,3.2834557E-1,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"split_indices":[4,5,46,43,32,57,30,40,53,4,10,0,0,0,0,2,0,29,48,46,0,0,0,0,0,89,0,3,0,3,0,0,0,46,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.84E2,2E1,1.67E2,1.7E1,1E1,1E1,1.3E2,3.7E1,1.1E1,6E0,7E0,3E0,8E0,2E0,1.27E2,3E0,2E1,1.7E1,8E0,3E0,4E0,2E0,4E0,3E0,1.23E2,4E0,1E1,1E1,1.4E1,3E0,2E0,6E0,1.12E2,1.1E1,2E0,2E0,4E0,6E0,4E0,6E0,5E0,9E0,9.5E1,1.7E1,2E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0223907E-3,-2.7888758E-2,9.6585326E-2,-1.5584077E-2,-2.1336152E-1,1.575512E-1,-3.900375E-2,-3.563399E-2,2.183398E-2,-3.6633124E-3,-1.6371118E-2,1.8702927E-1,-5.3675026E-3,2.0629173E-2,-1.4183015E-1,-3.0176764E-2,-7.741039E-3,5.2294556E-2,-3.702995E-2,5.3999938E-2,2.235795E-1,4.8674736E-2,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,-4.079073E-2,3.0565633E-2,6.8358756E-3,7.473907E-2,2.9559305E-3,-5.9157714E-2,-3.6169306E-4,3.6848264E-3,1.9294271E-2,1.8000335E-1,4.2783604E-3,7.0099917E-4,-3.1242667E-2,-1.5811709E-1,-3.2913506E-2,1.07815154E-1,2.1910788E-3,-7.367315E-4,9.088779E-2,5.4238463E-4,-5.502338E-3,-4.669484E-3,8.802838E-2,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8288884E-1,3.6638862E-1,4.1119415E-1,1.1556303E-1,1.445714E-1,2.890743E-1,1.0103525E-1,6.2558055E-2,9.8760046E-2,0E0,0E0,1.3865662E-1,0E0,3.1503566E-2,8.29455E-2,6.378441E-2,0E0,3.621982E-2,4.469773E-2,1.12352E-2,1.0943866E-1,1.083583E-2,0E0,0E0,0E0,8.803986E-2,7.710456E-2,1.1590129E-2,2.3634195E-2,0E0,4.6466663E-2,0E0,0E0,0E0,4.115665E-2,0E0,0E0,7.36302E-2,2.4497524E-2,2.882525E-2,2.4574421E-2,0E0,0E0,1.8441036E-2,0E0,0E0,1.4278462E-2,2.0401236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,25,25,26,26,27,27,28,28,30,30,34,34,37,37,38,38,39,39,40,40,43,43,46,46,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.775894E10,9.040637E7,2.0053221E6,6.0671224E7,1.3578947E1,5.3868E4,5.138372E8,3.8279E5,-3.6633124E-3,-1.6371118E-2,1.2737E4,-5.3675026E-3,9.123462E-1,2.6022625E5,2.0997734E8,-7.741039E-3,5.716463E2,8.75E2,5.22E2,1.6064256E-2,3.9E1,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,5.1383884E7,4.016202E1,7.25E2,1.5562991E6,2.9559305E-3,2.7102E4,-3.6169306E-4,3.6848264E-3,1.9294271E-2,3.383436E5,4.2783604E-3,7.0099917E-4,2.47565E6,5.88E2,3.887674E6,1.631E4,2.1910788E-3,-7.367315E-4,7.3586698E0,5.4238463E-4,-5.502338E-3,1.617E3,9.851E3,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"split_indices":[4,5,46,49,46,55,10,7,30,0,0,2,0,28,34,7,0,53,0,0,58,3,0,0,0,33,59,0,48,0,10,0,0,0,29,0,0,29,0,46,9,0,0,54,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.61E2,4.8E1,1.52E2,9E0,3.3E1,1.5E1,9.9E1,5.3E1,5E0,4E0,3E1,3E0,1E1,5E0,9.6E1,3E0,3.5E1,1.8E1,7E0,2.3E1,8E0,2E0,3E0,2E0,8.2E1,1.4E1,1.2E1,2.3E1,3E0,1.5E1,2E0,5E0,3E0,2E1,3E0,5E0,7.7E1,5E0,8E0,6E0,4E0,8E0,1.8E1,5E0,7E0,8E0,6E0,1.4E1,7.3E1,4E0,2E0,3E0,4E0,4E0,4E0,2E0,1.5E1,3E0,2E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[8.825459E-3,-2.0716727E-2,9.291196E-2,-9.286735E-3,-2.2815323E-1,4.1376296E-2,1.8566836E-1,-4.1899815E-2,3.8700562E-2,-1.1017629E-3,-1.425366E-2,-6.1172745E-3,7.967702E-2,2.2176522E-1,7.0497654E-2,-1.486095E-1,-2.7861884E-2,-2.4466898E-2,7.862809E-2,2.2698525E-2,-3.884307E-3,1.145321E-1,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,-1.2128689E-2,-1.3520806E-1,1.7996103E-2,-7.628645E-2,5.7908777E-2,9.716253E-3,-1.2009995E-3,3.937936E-2,2.097745E-3,6.499496E-3,-2.237607E-2,1.0090606E-1,-8.440877E-3,5.7535426E-4,5.9123063E-3,-9.376124E-3,-4.9414644E-3,-3.266944E-4,8.224998E-3,4.116531E-2,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,-1,49,-1,-1,51,53,-1,-1,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1613337E-1,3.6301482E-1,2.5045186E-1,2.3167856E-1,9.979531E-2,6.5972775E-2,6.335485E-2,1.2768199E-1,1.5325102E-1,0E0,0E0,3.7913833E-2,3.5779223E-2,8.4248126E-2,1.1696836E-2,5.5736482E-2,1.3178307E-1,5.418279E-2,8.121562E-2,1.124549E-2,0E0,1.054576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.3339594E-2,5.8268234E-2,4.0797673E-2,1.8643163E-2,5.442866E-2,0E0,0E0,9.034543E-3,0E0,0E0,8.4523104E-2,1.790364E-2,0E0,0E0,0E0,1.3487471E-2,0E0,0E0,0E0,5.1540002E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,29,29,30,30,31,31,32,32,33,33,36,36,39,39,40,40,44,44,48,48],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,-1,50,-1,-1,52,54,-1,-1,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5585917E0,1.4233672E8,1.6059814E3,1.2534044E3,8.667E3,5.433479E2,2.7304773E0,8.35E2,7.157224E0,-1.1017629E-3,-1.425366E-2,1.1350955E0,1.2224206E7,8E0,2.8352614E9,4.1749332E4,1.448907E6,7.794012E7,5.2086E4,2.9539914E4,-3.884307E-3,7.810231E4,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,1E0,2.18E3,5.491E3,1.47245E7,6.489288E6,9.716253E-3,-1.2009995E-3,8.842206E7,2.097745E-3,6.499496E-3,1.5989196E7,2.94E2,-8.440877E-3,5.7535426E-4,5.9123063E-3,1.424506E6,-4.9414644E-3,-3.266944E-4,8.224998E-3,9.1487586E-1,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"split_indices":[42,46,53,4,2,53,43,2,55,0,0,58,51,8,5,33,1,46,2,34,0,34,0,0,0,0,0,0,0,89,0,9,1,46,0,0,5,0,0,46,0,0,0,0,30,0,0,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.53E2,5.3E1,1.46E2,7E0,3.5E1,1.8E1,8.7E1,5.9E1,2E0,5E0,1.6E1,1.9E1,1.3E1,5E0,9E0,7.8E1,2.3E1,3.6E1,1.2E1,4E0,1.1E1,8E0,9E0,4E0,2E0,3E0,6E0,3E0,6.9E1,9E0,1.3E1,1E1,3.2E1,4E0,3E0,9E0,3E0,8E0,6.4E1,5E0,7E0,2E0,2E0,1.1E1,7E0,3E0,3E0,2.9E1,4E0,5E0,4.5E1,1.9E1,2E0,3E0,9E0,2E0,2.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.4462977E-3,-2.893987E-2,6.9793515E-2,-3.6714904E-2,1.3322261E-2,2.450053E-2,1.3456057E-1,-3.0940516E-2,-1.5242134E-2,6.13726E-2,-2.383746E-2,1.9235724E-1,5.9276864E-2,-1.44404005E-2,-1.02318145E-1,6.015054E-3,1.4756152E-3,-5.1880803E-2,1.3078791E-3,2.2946529E-1,2.957695E-3,4.6630064E-3,-1.3928295E-3,-7.5536236E-2,1.9160501E-3,-1.6728845E-1,-1.4484057E-2,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,-4.1924756E-2,-1.2142303E-2,9.4770074E-2,-1.28825195E-2,-1.1275523E-1,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,27,-1,29,-1,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9038587E-1,3.7482685E-1,1.5511182E-1,2.329789E-1,0E0,6.027959E-2,8.43229E-2,1.7518334E-1,0E0,3.29639E-2,2.2726193E-2,4.4647485E-2,3.807029E-2,1.2305999E-1,1.5736201E-1,0E0,0E0,1.2245912E-2,0E0,1.3593733E-2,0E0,0E0,0E0,8.423267E-2,1.2907745E-1,5.515635E-2,1.3991938E-2,0E0,0E0,0E0,0E0,0E0,5.6269135E-2,7.5751625E-2,1.7076403E-2,0E0,2.499877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,23,23,24,24,25,25,26,26,32,32,33,33,34,34,36,36],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,28,-1,30,-1,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,1.3453049E3,2.8159826E0,1.3322261E-2,2.4608E4,8.812601E-1,7.361366E7,-1.5242134E-2,7.614E3,2.449092E5,2.5501368E9,8.2731545E-1,6.974669E7,8.66076E1,6.015054E-3,1.4756152E-3,3.239E3,1.3078791E-3,3.6E1,2.957695E-3,4.6630064E-3,-1.3928295E-3,8.35E2,5.506676E11,1E0,2.313233E6,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,3.54E2,8E0,3.3478114E-1,-1.28825195E-2,3.890129E5,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"split_indices":[43,57,53,36,0,9,58,46,0,10,29,7,28,7,57,0,0,2,0,3,0,0,0,2,32,65,1,0,0,0,0,0,0,8,28,0,29,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.54E2,5.3E1,1.51E2,3E0,3.2E1,2.1E1,1.49E2,2E0,1.8E1,1.4E1,1.1E1,1E1,1.22E2,2.7E1,5E0,1.3E1,9E0,5E0,8E0,3E0,7E0,3E0,2.5E1,9.7E1,1.5E1,1.2E1,5E0,4E0,6E0,2E0,5E0,2E1,8.5E1,1.2E1,4E0,1.1E1,3E0,9E0,1E1,1E1,6.1E1,2.4E1,3E0,9E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.4985325E-2,-3.9879307E-2,5.6348145E-2,-2.3429228E-2,-1.0598515E-1,-1.7923791E-2,9.930136E-2,-5.1214058E-2,3.0147022E-2,-3.333782E-2,-1.8125312E-1,3.6635974E-3,-3.763992E-2,1.5140602E-1,5.2652568E-2,-6.5377824E-2,1.8297803E-2,1.5232481E-2,8.540196E-3,-7.670071E-2,8.6073264E-2,-2.6167667E-1,-8.275727E-4,-9.86009E-3,-4.455716E-3,8.21394E-3,2.7782663E-3,8.663035E-2,1.0011745E-2,-1.3291118E-1,-5.3235017E-2,-3.8194293E-3,3.9224032E-2,7.2662863E-3,-3.2975117E-3,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,-1.8245183E-1,-3.2175884E-2,-1.0584831E-1,4.2874664E-3,-1.5806843E-3,9.24955E-3,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,-1,-1,-1,43,45,47,49,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.769355E-1,1.6824591E-1,1.7724186E-1,1.9059703E-1,1.6346598E-1,4.1423395E-2,7.709977E-2,8.389847E-2,9.2703044E-2,9.462805E-2,1.9369146E-1,0E0,2.6057394E-2,1.6584098E-2,2.810949E-2,5.03276E-2,3.3456847E-2,1.0235344E-1,0E0,3.1034954E-2,2.0551417E-2,4.1319788E-2,0E0,1.02055315E-2,0E0,0E0,0E0,1.1555165E-2,9.544139E-3,5.704157E-2,6.4181656E-2,0E0,4.842038E-2,0E0,6.882077E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.789392E-3,5.7711136E-2,5.315976E-2,0E0,0E0,3.5858765E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,27,27,28,28,29,29,30,30,32,32,34,34,48,48,49,49,50,50,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,-1,-1,-1,44,46,48,50,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,7.294459E7,3.239E3,1.2943141E3,1.3907746E10,7.751E3,6.4512783E-1,4.3E1,4.1983E4,1.753919E3,3.7284137E2,3.6635974E-3,8.724138E0,2.2151E4,2E0,8.35E2,3.99E2,2.73782E5,8.540196E-3,2.7864855E11,1.7837838E0,9.301866E7,-8.275727E-4,5.0900903E0,-4.455716E-3,8.21394E-3,2.7782663E-3,3.1E1,2.2621576E-1,4.569361E7,7E0,-3.8194293E-3,1.2623029E7,7.2662863E-3,1.2001309E0,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,4.1749332E4,1E0,3.677609E-1,4.2874664E-3,-1.5806843E-3,1E1,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"split_indices":[43,46,2,56,5,9,58,3,2,53,59,0,55,9,8,2,2,30,0,32,58,7,0,57,0,0,0,3,39,5,8,0,12,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,33,74,28,0,0,3,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.56E2,5.4E1,1.26E2,3E1,2E1,3.4E1,8.3E1,4.3E1,1.6E1,1.4E1,3E0,1.7E1,1.5E1,1.9E1,6.9E1,1.4E1,4E1,3E0,1.2E1,4E0,9E0,5E0,1.2E1,5E0,1.2E1,3E0,1E1,9E0,9E0,6E1,2E0,1.2E1,4E0,3.6E1,6E0,6E0,2E0,2E0,2E0,7E0,5E0,7E0,5E0,5E0,2E0,7E0,3E0,6E0,4.4E1,1.6E1,7E0,5E0,3.4E1,2E0,3E0,3E0,3.9E1,5E0,2E0,1.4E1,2E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.3111593E-4,8.332628E-3,-1.9431697E-1,-3.4451315E-3,1.5696207E-1,-1.8588053E-3,-1.294587E-2,-3.1280745E-2,2.9869221E-2,2.6512837E-1,5.53925E-2,-1.5708588E-1,-2.058258E-2,1.0180442E-2,9.531762E-2,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,2.4204077E-3,-7.460408E-2,-4.4569806E-3,1.9710528E-2,1.22437425E-1,-5.1242317E-4,-1.3157908E-2,7.7513896E-2,-1.2132418E-1,-5.7141267E-2,9.711912E-2,6.6852947E-3,1.4493192E-3,1.5084128E-1,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5398465E-1,3.5764718E-1,9.0578735E-2,1.7712615E-1,1.5010995E-1,0E0,0E0,1.367387E-1,1.1113774E-1,3.353101E-2,1.3905274E-2,3.2876134E-2,1.20718196E-1,6.775201E-2,5.870013E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.17382E-2,1.6727448E-2,0E0,6.287142E-2,3.7616774E-2,0E0,7.575147E-2,3.0201033E-2,3.716676E-2,8.603498E-3,2.5892913E-2,4.6929307E-2,0E0,2.140811E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,34,34],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4233672E8,4.9192E4,8.901E3,4.2350553E5,8.812601E-1,-1.8588053E-3,-1.294587E-2,5.160874E0,9.855582E5,1.8136424E7,6.804E3,5.3012047E-2,7.501344E0,2.884462E0,3.9247762E6,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,1E0,2.6239506E5,-4.4569806E-3,1.5324128E6,2.8244882E6,-5.1242317E-4,1.6278354E7,3.57E2,4.888E3,1.3593E4,1.7417817E-1,1.5489E4,1.4493192E-3,1.02952E5,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"split_indices":[46,2,2,48,58,0,0,55,29,1,9,58,54,54,48,0,0,0,0,0,0,74,29,0,33,33,0,46,0,2,10,39,9,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.03E2,8E0,1.89E2,1.4E1,3E0,5E0,1.03E2,8.6E1,6E0,8E0,7E0,9.6E1,6.7E1,1.9E1,3E0,3E0,2E0,6E0,5E0,2E0,6.8E1,2.8E1,5E0,6.2E1,1.5E1,4E0,5.7E1,1.1E1,6E0,2.2E1,8E0,5.4E1,4E0,1.1E1,3.5E1,2.2E1,6E0,5E0,4E0,2E0,2E1,2E0,2E0,6E0,2.7E1,2.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[7.821103E-3,-2.8981427E-3,1.7867728E-1,-5.675793E-2,1.8588716E-2,2.1517694E-1,-1.2397172E-4,-3.6930315E-2,-1.2052149E-2,-1.4890818E-2,4.163438E-2,2.7627808E-1,2.6501652E-3,-5.6514364E-2,6.0741283E-2,-4.3238487E-2,2.5888829E-2,-2.5325203E-2,7.3175E-2,1.5455141E-2,5.614201E-3,-7.2515686E-3,-4.0423278E-2,-8.0662005E-4,6.1275316E-3,1.2608571E-2,-6.090714E-2,-9.619795E-3,7.656288E-2,7.5304456E-2,-7.165898E-2,1.6621768E-1,5.4205015E-2,-1.64469E-2,-1.2827884E-1,2.7917859E-3,-9.4076915E-4,1.9582137E-3,-7.1012415E-2,3.3104515E-3,-3.469917E-2,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,-2.7355356E-2,-1.6309094E-1,1.9180217E-1,2.4875687E-3,5.2126553E-3,3.92498E-2,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,27,29,31,-1,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,-1,59,61,63,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7230206E-1,2.2325236E-1,7.992381E-2,1.9902351E-1,1.0709499E-1,8.315757E-2,0E0,1.0098582E-1,0E0,6.701185E-2,1.7503278E-1,9.8798275E-3,0E0,5.797288E-2,4.7433127E-2,3.4317195E-2,4.39213E-2,1.3128516E-1,8.98312E-2,0E0,0E0,0E0,7.7935606E-2,0E0,0E0,1.3974639E-2,2.9431812E-2,3.1568956E-2,1.9552685E-2,2.7546454E-2,7.250317E-2,1.6699761E-2,3.3773705E-2,4.245451E-2,5.6236163E-2,0E0,0E0,0E0,2.1543108E-2,0E0,1.2517829E-2,0E0,0E0,0E0,0E0,2.2041425E-2,2.2109985E-2,1.6221613E-2,0E0,0E0,3.1004522E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,40,40,45,45,46,46,47,47,50,50],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,28,30,32,-1,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,-1,60,62,64,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,4.039435E-3,4.6E1,2.0604518E8,5.470729E2,2.0867E4,-1.2397172E-4,1E0,-1.2052149E-2,3.5E1,1E0,9.299267E9,2.6501652E-3,9.342733E7,3.52777E5,6.027375E0,5.935E3,1.4364E4,7.573E3,1.5455141E-2,5.614201E-3,-7.2515686E-3,6.6932364E7,-8.0662005E-4,6.1275316E-3,1.625E3,6.558106E-3,6.693824E5,1.0275E4,9.123462E-1,5.4827127E0,1.81025E5,7.03482E5,1.3696512E7,9.64E2,2.7917859E-3,-9.4076915E-4,1.9582137E-3,4.053E3,3.3104515E-3,4E1,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,2.3541917E3,3.800374E6,2.6807916E0,2.4875687E-3,5.2126553E-3,2.1225555E-1,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"split_indices":[2,39,3,7,53,9,0,74,0,3,102,12,0,5,1,54,10,10,9,0,0,0,46,0,0,0,39,49,9,28,58,30,48,46,0,0,0,0,11,0,3,0,0,0,0,4,1,40,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.91E2,1.1E1,5.4E1,1.37E2,9E0,2E0,5E1,4E0,5.6E1,8.1E1,6E0,3E0,4.2E1,8E0,3.3E1,2.3E1,2.6E1,5.5E1,4E0,2E0,5E0,3.7E1,4E0,4E0,8E0,2.5E1,1.4E1,9E0,8E0,1.8E1,8E0,4.7E1,3E1,7E0,3E0,5E0,2E0,2.3E1,3E0,1.1E1,4E0,5E0,5E0,3E0,1.3E1,5E0,6E0,2E0,9E0,3.8E1,7E0,2.3E1,5E0,2E0,9E0,1.4E1,6E0,5E0,2E0,1.1E1,3E0,2E0,4E0,2E0,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[1.6695334E-3,-2.430477E-2,6.987881E-2,-1.628185E-1,-1.665332E-2,3.3794872E-2,1.4406632E-1,-2.0858636E-3,-2.262213E-1,-1.3216189E-3,-8.5313104E-2,7.368082E-2,5.320639E-3,2.1326382E-1,6.0462188E-2,-1.3826551E-2,-4.5224247E-3,-3.0901669E-2,5.3741764E-2,-4.580613E-2,-1.4649896E-2,6.6577974E-3,3.5978813E-2,1.8743427E-2,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,-1.9552542E-2,-8.157249E-3,-1.2044725E-2,9.7219266E-2,-6.857402E-2,-3.44251E-4,5.0435625E-2,-1.1778983E-3,-4.252783E-4,5.2595764E-2,-4.5224298E-2,2.5154488E-2,-2.6128737E-3,4.6722338E-2,-8.63488E-4,1.1892799E-1,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,-1,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,-1,49,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7906808E-1,1.6156235E-1,1.5307292E-1,5.1149473E-2,1.556826E-1,4.617256E-2,9.702435E-2,0E0,1.2178838E-2,2.0037158E-1,2.1631023E-1,3.5536163E-2,2.652885E-2,6.27754E-2,2.7581971E-2,0E0,0E0,1.2084925E-1,1.2416096E-1,2.0679474E-2,0E0,0E0,1.1661252E-2,2.092692E-2,0E0,0E0,0E0,0E0,0E0,8.721569E-2,0E0,4.656056E-2,6.699793E-2,2.1412484E-2,0E0,8.723641E-3,0E0,0E0,1.359814E-2,4.531187E-2,9.016034E-2,0E0,1.4617754E-2,0E0,5.3301275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,22,22,23,23,29,29,31,31,32,32,33,33,35,35,38,38,39,39,40,40,42,42,44,44],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,-1,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,-1,50,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.3125E0,1.3173E4,5.34E2,1.3845824E1,1.5489E4,8E0,-2.0858636E-3,4.233395E6,8.3764136E2,2.1024984E5,5.2560944E1,3.0794094E0,2.2047469E4,2.4711658E5,-1.3826551E-2,-4.5224247E-3,8.260109E9,7.157224E0,8.071663E5,-1.4649896E-2,6.6577974E-3,7.4809E4,1.5383295E6,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,3.7E1,-8.157249E-3,5E1,7.157903E7,1.17E2,-3.44251E-4,2.398624E8,-1.1778983E-3,-4.252783E-4,2.8314E4,1.4218562E5,7.47E2,-2.6128737E-3,5.5326223E9,-8.63488E-4,2.4039803E5,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"split_indices":[43,54,2,0,55,9,8,0,1,53,34,59,43,4,34,0,0,5,55,49,0,0,30,48,0,0,0,0,0,3,0,3,7,10,0,7,0,0,9,29,0,0,5,0,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.54E2,5.8E1,7E0,1.47E2,4E1,1.8E1,3E0,4E0,1.21E2,2.6E1,1.6E1,2.4E1,9E0,9E0,2E0,2E0,7.9E1,4.2E1,2.3E1,3E0,5E0,1.1E1,2.1E1,3E0,7E0,2E0,4E0,5E0,7.4E1,5E0,1.7E1,2.5E1,1.4E1,9E0,9E0,2E0,1.2E1,9E0,4.7E1,2.7E1,1E1,7E0,4E0,2.1E1,8E0,6E0,2E0,7E0,7E0,2E0,7E0,4E1,1.6E1,1.1E1,3E0,4E0,1.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[3.2811111E-3,-1.1759641E-2,9.609429E-2,-5.231051E-3,-1.926002E-1,4.223107E-2,1.7549825E-1,-2.3685513E-2,3.7570078E-2,-7.758012E-4,-1.34779E-2,7.0535643E-3,2.5339068E-3,1.425658E-2,4.615414E-3,-1.5669417E-2,-9.816772E-3,4.6593472E-2,-5.953628E-3,1.4607337E-3,-3.911037E-2,-1.08244635E-1,-6.703496E-3,1.3439576E-3,8.3413675E-2,-3.8560897E-3,3.702331E-5,-1.4499448E-1,1.2573712E-3,4.405412E-3,-4.881462E-2,2.0919163E-3,-1.956045E-2,9.7872205E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,23,-1,-1,25,27,29,31,33,-1,-1,35,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.80326E-1,2.0324215E-1,1.1234322E-1,1.3358834E-1,8.498618E-2,7.30211E-2,7.642406E-2,1.6523767E-1,7.928742E-2,0E0,0E0,0E0,1.7196054E-2,0E0,0E0,9.389689E-2,0E0,8.1526615E-2,0E0,0E0,9.659185E-3,5.3020775E-2,4.9334954E-2,2.086177E-2,4.4374347E-2,0E0,0E0,2.1213919E-2,0E0,4.3695264E-2,4.650379E-2,0E0,1.4095474E-2,3.8369343E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,17,17,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,32,32,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,24,-1,-1,26,28,30,32,34,-1,-1,36,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.918757E7,1.4623123E8,1.7227725E0,1.0892118E3,8.667E3,1.1944959E8,3.0291426E-1,6.6932364E7,1.8323593E1,-7.758012E-4,-1.34779E-2,7.0535643E-3,8.0756325E-1,1.425658E-2,4.615414E-3,8.35E2,-9.816772E-3,7.316901E0,-5.953628E-3,1.4607337E-3,1.467E3,6.1E1,1.1103951E1,4.8759037E1,2.3524794E7,-3.8560897E-3,3.702331E-5,1.0907971E6,1.2573712E-3,1.643E4,6.87E2,2.0919163E-3,6.879445E1,1.9652087E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"split_indices":[33,46,40,53,2,33,39,46,58,0,0,0,28,0,0,2,0,55,0,0,0,3,55,57,33,0,0,29,0,9,0,0,57,40,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.72E2,2.7E1,1.67E2,5E0,1.7E1,1E1,1.17E2,5E1,2E0,3E0,4E0,1.3E1,3E0,7E0,1.13E2,4E0,4.8E1,2E0,8E0,5E0,9E0,1.04E2,2.2E1,2.6E1,2E0,3E0,7E0,2E0,8.3E1,2.1E1,7E0,1.5E1,2.3E1,3E0,4E0,3E0,4.2E1,4.1E1,1.2E1,9E0,9E0,6E0,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.0387296E-3,-1.8574167E-2,8.5663736E-2,-8.023423E-3,-1.4978984E-1,1.4511184E-1,-3.3413577E-3,-2.7192928E-2,3.1242838E-2,-1.0438316E-2,-2.401442E-3,3.2174537E-1,9.6915334E-2,2.1752674E-2,-7.842911E-3,-2.143232E-2,-6.5535344E-3,9.6770816E-2,-2.4400586E-3,1.8892974E-2,6.1257724E-3,7.6597073E-4,1.1642241E-1,-6.7683046E-5,7.212856E-2,-7.260881E-2,-7.379258E-3,1.2818316E-1,1.6344215E-3,-7.859502E-2,1.6685827E-2,2.2511675E-3,1.3107267E-1,6.177627E-3,-1.2163549E-3,-2.2213712E-2,-1.08554E-1,2.886258E-2,-3.7549652E-2,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,4.6269923E-2,-4.119254E-2,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,-1,-1,-1,31,-1,33,35,37,39,-1,41,43,-1,45,-1,-1,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1510934E-1,2.230747E-1,2.6504415E-1,1.14509515E-1,6.739807E-2,2.2421151E-1,8.667014E-2,6.0260966E-2,1.10594735E-1,0E0,0E0,2.9250383E-2,3.775531E-2,2.2251856E-2,0E0,6.994447E-2,0E0,2.9437363E-2,5.0833475E-2,0E0,0E0,0E0,1.449576E-2,0E0,3.511203E-2,3.5629295E-2,8.637136E-2,1.2054026E-2,0E0,1.4433451E-2,4.9849793E-2,0E0,1.9013852E-2,0E0,0E0,1.4445642E-2,3.5027638E-2,4.9891964E-2,7.3443145E-2,0E0,0E0,0E0,0E0,2.8110322E-2,1.6329382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,22,22,24,24,25,25,26,26,27,27,29,29,30,30,32,32,35,35,36,36,37,37,38,38,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,-1,-1,-1,32,-1,34,36,38,40,-1,42,44,-1,46,-1,-1,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6503105E3,8.67241E9,5.9337012E7,6.90321E2,6.1826086E0,5.0655737E0,1.4623123E8,6.0073395E8,1.10597E5,-1.0438316E-2,-2.401442E-3,2E0,2.819228E-1,4.078932E5,-7.842911E-3,1.142E3,-6.5535344E-3,7.393122E1,1.6595541E1,1.8892974E-2,6.1257724E-3,7.6597073E-4,3.559158E6,-6.7683046E-5,2.1516852E5,5.12334E9,5.62E2,7.44905E0,1.6344215E-3,1E0,6.1093975E6,2.2511675E-3,6.797394E8,6.177627E-3,-1.2163549E-3,1.1086122E3,5.244755E-3,1.5989196E7,2.7816156E5,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,5.6E0,7.3376025E6,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"split_indices":[53,5,46,53,54,55,46,7,30,0,0,8,58,29,0,2,0,59,57,0,0,0,1,0,34,32,0,55,0,17,33,0,7,0,0,34,58,46,29,0,0,0,0,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.61E2,4.9E1,1.5E2,1.1E1,2.9E1,2E1,1.01E2,4.9E1,6E0,5E0,5E0,2.4E1,1.8E1,2E0,9.7E1,4E0,1.6E1,3.3E1,3E0,2E0,5E0,1.9E1,1.3E1,5E0,2E1,7.7E1,1E1,6E0,6E0,2.7E1,4E0,1.5E1,3E0,2E0,9E0,1.1E1,3.5E1,4.2E1,3E0,7E0,2E0,4E0,1.8E1,9E0,2E0,1.3E1,4E0,5E0,7E0,4E0,2.5E1,1E1,1.9E1,2.3E1,2E0,1.6E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.4796249E-3,-2.8328227E-2,4.2443145E-2,-1.2839314E-2,-1.4659679E-1,6.714284E-2,-6.827072E-2,-2.624263E-2,3.197302E-2,-5.994708E-2,-2.0733023E-1,1.3621603E-2,5.6769222E-2,-2.7255906E-2,-8.979017E-3,-2.0926751E-2,-6.972742E-3,6.93552E-2,-2.6764661E-2,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,3.61498E-2,1.4122447E-1,1.9069649E-3,-5.1106434E-2,-1.4165286E-2,-5.6748535E-3,8.369632E-2,-1.472121E-3,-3.9843167E-3,1.1822013E-3,8.772388E-2,2.698954E-3,1.7185566E-1,1.427893E-3,-4.393489E-3,-4.8403983E-4,-7.145611E-2,5.5866083E-4,1.9922133E-3,5.4703923E-3,1.7163315E-1,4.553083E-2,1.823748E-2,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,33,35,-1,37,39,-1,41,-1,-1,-1,43,45,47,-1,-1,-1,49,51,-1,-1,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6417843E-1,2.5274518E-1,2.370248E-1,7.531766E-2,7.177171E-2,1.3860276E-1,7.124599E-2,5.672677E-2,6.5532796E-2,1.3237113E-2,3.143558E-2,0E0,1.1331123E-1,2.2602845E-2,0E0,5.8513593E-2,0E0,2.8243646E-2,3.5750594E-2,0E0,0E0,0E0,0E0,9.637425E-2,3.9510548E-2,0E0,1.52250845E-2,7.427207E-2,0E0,1.5040562E-2,0E0,0E0,0E0,7.007395E-2,7.020194E-2,1.9569606E-2,0E0,0E0,0E0,2.5643677E-2,5.1912256E-2,0E0,0E0,1.0038182E-2,2.4570972E-2,2.6236098E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,18,18,23,23,24,24,26,26,27,27,29,29,33,33,34,34,35,35,39,39,40,40,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,34,36,-1,38,40,-1,42,-1,-1,-1,44,46,48,-1,-1,-1,50,52,-1,-1,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,4.56567E7,1.09495016E8,5.470729E2,7.683833E-1,1.7727281E2,1.4623123E8,5.138372E8,1.1424464E3,3.97E2,2.1E1,1.3621603E-2,4.2245E4,1E0,-8.979017E-3,9.85054E5,-6.972742E-3,2.4848485E0,1.2707859E3,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,2.1637352E7,3.832274E2,1.9069649E-3,8.427037E6,1.056E3,-5.6748535E-3,6.685237E0,-1.472121E-3,-3.9843167E-3,1.1822013E-3,7.573E3,1.28885E1,4.759155E1,1.427893E-3,-4.393489E-3,-4.8403983E-4,7.8206635E0,4.570007E7,1.9922133E-3,5.4703923E-3,5.142232E-1,2.3033286E5,2.249985E5,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"split_indices":[53,46,46,53,28,34,46,7,4,0,3,0,2,8,0,30,0,58,4,0,0,0,0,46,57,0,48,2,0,55,0,0,0,9,55,57,0,0,0,54,5,0,0,58,34,34,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.38E2,8.4E1,1.23E2,1.5E1,6.9E1,1.5E1,9.5E1,2.8E1,7E0,8E0,2E0,6.7E1,1.2E1,3E0,9.2E1,3E0,1.7E1,1.1E1,2E0,5E0,2E0,6E0,5.5E1,1.2E1,3E0,9E0,8.7E1,5E0,1.5E1,2E0,5E0,6E0,2.1E1,3.4E1,9E0,3E0,4E0,5E0,1.7E1,7E1,7E0,8E0,6E0,1.5E1,3.1E1,3E0,2E0,7E0,1.2E1,5E0,6E0,6.4E1,4E0,2E0,8E0,7E0,1.9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.5098556E-3,-3.430795E-2,4.6132654E-2,-7.989392E-2,-7.3299413E-3,6.539497E-2,-1.2817919E-1,-5.846227E-2,-1.0367309E-2,4.5335226E-2,-3.8782444E-2,4.477679E-2,1.7191277E-1,3.5083215E-4,-1.0261507E-2,1.7973488E-3,-6.9773324E-2,6.968256E-2,-1.4760661E-3,-5.74948E-2,8.712715E-3,1.1792199E-1,2.4033781E-2,2.3350367E-1,2.4967322E-3,2.0957861E-2,-8.140613E-2,-2.9711837E-3,8.504071E-2,-2.0592897E-3,-7.8736946E-2,-3.0431528E-2,3.5283656E-3,1.3522897E-3,1.7078307E-1,-1.0893706E-2,4.7292955E-2,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,-9.249218E-2,1.0750794E-3,7.799928E-4,1.05605684E-1,-2.1099192E-3,1.9261859E-3,-2.492124E-2,-1.10814184E-1,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,-3.874176E-2,3.5721097E-2,3.354856E-4,7.9686604E-2,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,29,31,33,35,37,-1,39,41,-1,43,45,47,49,-1,-1,51,53,55,-1,-1,-1,-1,57,-1,-1,59,-1,-1,61,63,-1,-1,-1,-1,65,67,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0736163E-1,1.5191185E-1,2.669213E-1,1.18796796E-1,1.3261785E-1,1.4603654E-1,9.106535E-2,4.6816275E-2,0E0,5.738108E-2,4.4938266E-2,8.8200994E-2,6.973371E-2,0E0,0E0,0E0,4.075806E-2,5.1372252E-2,0E0,4.223033E-2,3.9660476E-2,5.719474E-2,3.9509647E-2,2.0871669E-2,0E0,1.7594527E-2,3.99065E-2,0E0,2.8111845E-2,2.0630281E-2,4.2121127E-2,1.1639008E-2,0E0,0E0,1.40551925E-2,2.734653E-2,3.7402496E-2,0E0,0E0,0E0,0E0,5.2814364E-2,0E0,0E0,8.88367E-3,0E0,0E0,9.187069E-3,3.1335637E-2,0E0,0E0,0E0,0E0,2.286155E-2,2.1192856E-2,0E0,1.3821408E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,41,41,44,44,47,47,48,48,53,53,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,30,32,34,36,38,-1,40,42,-1,44,46,48,50,-1,-1,52,54,56,-1,-1,-1,-1,58,-1,-1,60,-1,-1,62,64,-1,-1,-1,-1,66,68,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,4.039435E-3,1.3917024E8,2.0604518E8,1.2903E4,5.4349E4,1.5214152E5,2.9270105E2,-1.0367309E-2,1.3845824E1,4.311809E5,1.20171E5,1.1406721E2,3.5083215E-4,-1.0261507E-2,1.7973488E-3,2.1013361E-1,9.33E2,-1.4760661E-3,5.948464E0,1.3012535E1,4.361E3,5.590909E0,1.671E3,2.4967322E-3,4E1,4.1211517E1,-2.9711837E-3,6.0775385E0,2.5848E4,1.2753862E1,5.8348556E0,3.5283656E-3,1.3522897E-3,1.57727E3,1.96E4,4.485647E5,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,6.258335E5,1.0750794E-3,7.799928E-4,3.0597075E8,-2.1099192E-3,1.9261859E-3,2E0,1.7261968E7,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,8.062193E-1,6.75741E5,3.354856E-4,1.184E3,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"split_indices":[53,39,46,7,9,2,34,34,0,55,29,30,57,0,0,0,28,2,0,54,57,2,54,0,0,3,59,0,55,9,57,55,0,0,4,2,29,0,0,0,0,33,0,0,7,0,0,8,46,0,0,0,0,28,30,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.23E2,7.6E1,4.5E1,7.8E1,6.9E1,7E0,4E1,5E0,2.9E1,4.9E1,5.9E1,1E1,3E0,4E0,4E0,3.6E1,2.2E1,7E0,3.5E1,1.4E1,1.2E1,4.7E1,6E0,4E0,4E0,3.2E1,2E0,2E1,1E1,2.5E1,9E0,5E0,5E0,7E0,1.9E1,2.8E1,4E0,2E0,2E0,2E0,2.9E1,3E0,5E0,1.5E1,5E0,5E0,1E1,1.5E1,3E0,6E0,3E0,4E0,1.2E1,7E0,1.3E1,1.5E1,1.5E1,1.4E1,1.1E1,4E0,6E0,4E0,9E0,6E0,5E0,7E0,3E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.6206973E-3,-1.9603977E-2,4.7420494E-2,-4.8373155E-2,9.0342695E-3,7.200922E-2,-6.0652774E-2,-2.3101505E-2,-7.6871574E-2,8.582439E-2,-5.1944912E-3,5.357398E-2,1.6620213E-1,1.5464444E-2,-1.2918839E-1,4.415707E-2,-5.9842244E-2,-5.267666E-4,-1.0224566E-1,5.1139877E-3,-2.7146392E-5,7.1116164E-2,-2.1878945E-2,1.2166778E-1,4.110889E-2,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,-3.735069E-2,-1.0302123E-2,5.6916097E-4,4.160516E-3,-7.803266E-2,1.388256E-3,-1.3427517E-1,-4.7179796E-2,5.5395655E-4,5.7911566E-3,-3.5666935E-2,2.1849289E-3,1.78514E-3,9.610851E-3,1.667288E-2,9.926257E-2,6.1937334E-4,-3.6489156E-3,-5.62302E-2,-5.528736E-3,-7.2530344E-2,-9.035339E-3,-3.0848335E-3,5.726469E-4,-9.4292425E-2,-1.1822653E-2,4.17147E-2,-1.5671453E-2,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,47,-1,49,51,-1,-1,53,-1,-1,-1,55,57,-1,-1,59,-1,61,-1,-1,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.23844E-1,1.1413506E-1,2.1164352E-1,4.791811E-2,7.69298E-2,1.0222992E-1,8.024894E-2,9.65454E-2,5.252196E-2,1.8357664E-2,7.7948034E-2,4.1682094E-2,4.1363686E-2,1.208929E-2,5.2706182E-2,1.7784836E-2,4.241229E-2,0E0,3.423077E-2,0E0,0E0,3.0106395E-2,4.7635924E-2,4.4303104E-2,6.655512E-2,0E0,0E0,0E0,0E0,1.0375681E-2,0E0,0E0,0E0,9.898692E-3,0E0,3.0951291E-2,1.0299623E-2,0E0,0E0,5.7596285E-2,0E0,0E0,0E0,2.9002367E-2,3.170462E-2,0E0,0E0,2.2735469E-2,0E0,9.1954805E-3,0E0,0E0,0E0,2.8296992E-2,2.1532139E-2,1.2805883E-2,1.26161575E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,29,29,33,33,35,35,36,36,39,39,43,43,44,44,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,48,-1,50,52,-1,-1,54,-1,-1,-1,56,58,-1,-1,60,-1,62,-1,-1,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,3.1749217E4,9.820072E7,1.4455165E7,8.637E3,1.6974416E6,1.3611247E5,1.1086122E3,6.9163686E-1,7.105731E0,3.9986582E4,6.518218E6,1.884E3,9.276886E1,2.1000721E0,1.9789175E2,9.946E3,-5.267666E-4,2.19E2,5.1139877E-3,-2.7146392E-5,3.067E3,1.3411796E3,5.62E2,1.6992E4,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,2.012E3,-1.0302123E-2,5.6916097E-4,4.160516E-3,1.6126542E8,1.388256E-3,6.7E1,7.0229135E6,5.5395655E-4,5.7911566E-3,2.4861076E-1,2.1849289E-3,1.78514E-3,9.610851E-3,6.9089854E-1,7.438459E6,6.1937334E-4,-3.6489156E-3,7.774228E-2,-5.528736E-3,7.856E3,-9.035339E-3,-3.0848335E-3,5.726469E-4,1.0092749E8,1.4519928E1,1.8156171E3,1.521636E5,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"split_indices":[53,34,46,46,9,29,34,34,28,54,34,46,0,57,58,53,10,0,10,0,0,2,4,0,2,0,0,0,0,0,0,0,0,5,0,10,33,0,0,40,0,0,0,28,1,0,0,58,0,9,0,0,0,7,57,53,34,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.37E2,7.7E1,6.8E1,6.9E1,6.3E1,1.4E1,3.7E1,3.1E1,1E1,5.9E1,5.4E1,9E0,7E0,7E0,1.3E1,2.4E1,9E0,2.2E1,8E0,2E0,1E1,4.9E1,7E0,4.7E1,4E0,5E0,3E0,4E0,4E0,3E0,8E0,5E0,2E1,4E0,1.3E1,9E0,5E0,5E0,4.1E1,8E0,4E0,3E0,3.4E1,1.3E1,2E0,2E0,1.4E1,6E0,7E0,6E0,7E0,2E0,1.1E1,3E1,1.9E1,1.5E1,3E0,1E1,8E0,6E0,4E0,3E0,3E0,8E0,2.1E1,9E0,1.2E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.1845648E-4,-1.3225078E-2,1.3611369E-1,-1.417116E-1,-6.7631784E-3,1.953729E-2,8.1282414E-2,-1.8065992E-1,-1.7495391E-4,-5.01936E-2,7.976252E-3,7.00604E-4,1.2666328E-1,-1.9851862E-3,-1.1117795E-2,-3.9532438E-2,-9.23816E-3,1.1314319E-2,-6.9974E-3,7.236729E-3,1.2495602E-3,-2.5365911E-2,-6.0845776E-3,-5.2135065E-4,6.29387E-2,-5.9177596E-3,-7.433232E-2,-5.97098E-3,4.050646E-3,-1.3454848E-3,8.696749E-2,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,-1,23,-1,-1,-1,25,-1,27,29,31,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8503218E-1,1.5596202E-1,2.437979E-1,4.776524E-2,1.1626564E-1,0E0,4.97777E-2,3.7470296E-2,0E0,6.0973354E-2,7.052197E-2,0E0,1.8436834E-2,0E0,0E0,5.011916E-2,0E0,8.190803E-2,0E0,0E0,0E0,3.6700055E-2,0E0,6.1938964E-2,5.6821078E-2,3.1497225E-2,1.2199353E-2,0E0,4.7399078E-2,0E0,2.0637825E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,17,17,21,21,23,23,24,24,25,25,26,26,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,-1,24,-1,-1,-1,26,-1,28,30,32,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,5.314E3,1.4164306E-3,5.3E1,3.5523141E-3,1.953729E-2,4.1151314E0,9.34E2,-1.7495391E-4,4.0515396E-3,3.167407E7,7.00604E-4,1.7652562E4,-1.9851862E-3,-1.1117795E-2,2.47565E6,-9.23816E-3,9.855582E5,-6.9974E-3,7.236729E-3,1.2495602E-3,2.2345362E1,-6.0845776E-3,1E1,2.55E2,1.4740072E1,6.2174755E-1,-5.97098E-3,5.6E0,-1.3454848E-3,1.548321E-1,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"split_indices":[53,9,58,3,40,0,54,0,0,42,44,0,4,0,0,29,0,29,0,0,0,59,0,3,0,59,28,0,55,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.88E2,1.8E1,8E0,1.8E2,2E0,1.6E1,6E0,2E0,4.5E1,1.35E2,7E0,9E0,2E0,4E0,4.3E1,2E0,1.33E2,2E0,7E0,2E0,3.8E1,5E0,1.09E2,2.4E1,2.8E1,1E1,3E0,1.06E2,5E0,1.9E1,1.6E1,1.2E1,3E0,7E0,8E0,9.8E1,9E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.666469E-3,-1.7545465E-2,1.19480304E-1,-8.49554E-3,-8.6954616E-2,1.4809049E-2,7.858761E-2,-1.9953433E-2,4.7460496E-2,-1.262078E-2,-4.3817658E-2,1.2613867E-1,2.3892207E-2,-3.333415E-2,2.116914E-2,-2.6011164E-3,9.102063E-2,7.2902715E-4,-9.959214E-2,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,-2.6154688E-2,-6.2321755E-3,6.586735E-2,-6.205673E-3,-4.2967666E-3,1.9037895E-2,1.7304541E-1,1.7110182E-2,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,-4.9978763E-2,-6.258132E-3,5.289967E-3,1.691339E-3,8.535292E-3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,37,39,-1,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9837632E-1,1.14212275E-1,1.5746966E-1,1.0545254E-1,1.447895E-1,0E0,5.1603585E-2,7.553959E-2,6.1296083E-2,0E0,4.4794835E-2,2.756174E-2,1.8413085E-2,6.669822E-2,4.2011324E-2,2.7640013E-2,8.797648E-2,1.5424975E-2,1.9613385E-2,0E0,0E0,0E0,0E0,4.566688E-2,0E0,1.3546545E-2,2.2841673E-2,0E0,3.328212E-2,1.718907E-2,1.4277115E-2,0E0,0E0,0E0,0E0,2.4923556E-2,6.463696E-2,0E0,0E0,1.3547961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,25,25,26,26,28,28,29,29,30,30,35,35,36,36,39,39],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,38,40,-1,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,7.810581E11,1.4164306E-3,4.5E1,8.0522484E11,1.4809049E-2,1.0096877E8,1.5064244E0,6.787619E2,-1.262078E-2,7.294459E7,3.2635366E8,1.1038146E0,1.4114665E0,1.5489E4,8.27725E3,5.1270317E-2,6.324075E-5,7.5213313E-1,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,3.52777E5,-6.2321755E-3,3.0597075E8,4.187504E7,-4.2967666E-3,6.58542E5,8.63332E5,2.0323718E7,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,2.1034482E-1,4.95715E5,5.289967E-3,1.691339E-3,1.46E3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"split_indices":[53,32,58,3,32,0,52,43,53,0,46,33,35,42,9,49,39,39,28,0,0,0,0,1,0,7,33,0,1,1,46,0,0,0,0,58,1,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.82E2,2.3E1,1.62E2,2E1,3E0,2E1,1.35E2,2.7E1,3E0,1.7E1,1E1,1E1,1.02E2,3.3E1,1.3E1,1.4E1,1E1,7E0,8E0,2E0,6E0,4E0,9.6E1,6E0,1.2E1,2.1E1,2E0,1.1E1,6E0,8E0,3E0,7E0,3E0,4E0,4.3E1,5.3E1,4E0,8E0,1.8E1,3E0,6E0,5E0,2E0,4E0,4E0,4E0,3E1,1.3E1,1E1,4.3E1,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-2.7402204E-3,-1.4488179E-2,8.47742E-2,-1.21823445E-1,-8.342185E-3,1.8740107E-1,-9.2469845E-3,-1.6784839E-3,-1.3934572E-1,1.2403458E-2,-4.2587325E-2,2.9215527E-1,4.7938665E-3,2.4776671E-2,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,4.4642862E-2,-1.8922342E-2,-3.3003904E-2,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,-3.79182E-2,5.552622E-2,-4.235295E-2,3.8699843E-2,1.6639564E-2,-5.707555E-2,-3.353905E-3,1.2455323E-4,7.502207E-2,-1.0837827E-2,-2.996247E-2,-1.0963698E-1,6.998143E-2,-2.54994E-4,7.42517E-2,-9.605715E-3,-8.033993E-2,-1.4847374E-3,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,-1,25,27,29,-1,-1,-1,-1,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1713255E-1,1.21684544E-1,2.4296036E-1,1.0496929E-2,1.2627566E-1,8.29688E-2,6.688262E-2,0E0,1.0810778E-2,1.1294642E-1,1.1641891E-1,3.8371354E-2,0E0,1.0111526E-2,0E0,0E0,0E0,5.1525243E-2,7.859604E-2,7.862471E-2,0E0,0E0,0E0,0E0,0E0,8.959427E-3,6.421043E-2,3.0895934E-2,2.3733113E-2,3.3978265E-2,5.6785285E-2,0E0,0E0,2.0940512E-2,1.2563176E-2,2.8267324E-2,2.118297E-2,8.720197E-3,0E0,1.4422674E-2,9.381125E-3,2.5498733E-2,8.510427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,-1,26,28,30,-1,-1,-1,-1,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,8.35E2,5.9337012E7,9.279E3,2.4934822E7,1.04E3,1.4233672E8,-1.6784839E-3,1.22E2,1.6672E4,6.3522E4,1.6232848E7,4.7938665E-3,9.5589536E-1,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,5.277751E0,1.02942106E11,6.4348924E-1,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,3.4E1,8.019512E0,4.305809E7,3.2395E4,2.7778377E11,3.9880952E-1,-3.353905E-3,1.2455323E-4,4.091697E7,6.2958473E4,2.237806E8,1.0519024E8,7.2023706E0,-2.54994E-4,7.951E3,4.83E9,4.944487E7,1.5107028E-1,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"split_indices":[53,2,46,9,46,0,46,0,10,9,10,1,0,28,0,0,0,55,32,28,0,0,0,0,0,3,54,33,9,32,58,0,0,33,34,7,7,55,0,9,5,46,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.85E2,2.4E1,9E0,1.76E2,1.1E1,1.3E1,2E0,7E0,1.1E2,6.6E1,4E0,7E0,1.1E1,2E0,5E0,2E0,5.4E1,5.6E1,6.4E1,2E0,2E0,2E0,9E0,2E0,6E0,4.8E1,4E1,1.6E1,2.1E1,4.3E1,3E0,3E0,3.7E1,1.1E1,3.5E1,5E0,9E0,7E0,6E0,1.5E1,3E1,1.3E1,2.7E1,1E1,4E0,7E0,2.1E1,1.4E1,3E0,2E0,6E0,3E0,2E0,4E0,4E0,1.1E1,1.2E1,1.8E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.9001366E-3,-1.343855E-2,1.1601315E-1,-1.0010185E-2,-1.111417E-2,2.2562835E-1,3.3268586E-2,-4.3225788E-2,2.3105354E-3,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,-5.3559467E-2,4.5828316E-3,4.5007914E-2,-1.1152626E-2,-4.4588957E-2,-8.577182E-3,-4.0048826E-3,5.9981905E-2,-3.185049E-2,2.3447553E-2,-2.0497924E-2,-8.2263514E-2,8.222998E-3,3.7578024E-2,-1.694235E-2,-1.1662346E-1,3.2336626E-2,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,23,-1,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.899919E-1,1.4066838E-1,1.6482484E-1,7.867399E-2,0E0,1.0884556E-1,1.5947495E-2,7.8872435E-2,8.1554644E-2,0E0,0E0,0E0,0E0,4.606816E-2,0E0,6.974669E-2,7.8090996E-2,4.0971868E-2,0E0,0E0,6.934259E-2,8.396302E-2,4.81951E-2,3.422363E-2,2.2953227E-2,0E0,6.309295E-2,6.78614E-2,3.393674E-2,2.4240296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,24,-1,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.5303E4,1.897887E6,3.5523141E-3,-1.111417E-2,3.0291426E-1,3.9637537E0,1E0,1.19855255E-1,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,4.0515396E-3,4.5828316E-3,5.178571E0,1.5064244E0,1.3431159E6,-8.577182E-3,-4.0048826E-3,3.849802E-1,1.4330416E0,5.46229E5,3.59E2,4.277001E3,8.222998E-3,6.448948E6,9.717001E5,4.3590187E1,1.3047E4,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"split_indices":[4,10,30,40,0,39,54,89,42,0,0,0,0,42,0,55,43,29,0,0,28,42,9,0,48,0,33,29,59,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.93E2,1.8E1,1.91E2,2E0,7E0,1.1E1,5.1E1,1.4E2,4E0,3E0,9E0,2E0,4.8E1,3E0,3.3E1,1.07E2,4.6E1,2E0,3E0,3E1,6.7E1,4E1,2.9E1,1.7E1,4E0,2.6E1,5.8E1,9E0,3.8E1,2E0,1.4E1,1.5E1,1.3E1,4E0,2E1,6E0,4.8E1,1E1,4E0,5E0,1.3E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.2056913E-3,-9.473315E-3,7.512071E-2,4.3374414E-4,-7.8392126E-2,1.5720385E-1,-1.8191284E-2,-2.5484065E-2,2.0686356E-2,-1.2111978E-2,-4.889973E-2,2.523382E-1,8.440286E-2,1.3624126E-2,-6.7835744E-3,1.453452E-2,-4.12981E-2,5.900995E-2,5.9917844E-3,-9.201298E-3,-1.0729721E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,2.635762E-2,-6.7390345E-2,-9.481888E-5,5.3193234E-3,3.617373E-2,-4.9073147E-3,6.533765E-2,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,1.4299699E-3,3.6974133E-3,-3.8031008E-2,-1.2230593E-1,7.3048964E-2,-2.9346012E-2,2.2003332E-3,-1.1724293E-3,-1.1925893E-2,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,-1,45,47,49,-1,-1,-1,-1,51,-1,53,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7272978E-1,1.1952097E-1,2.1732129E-1,8.136228E-2,9.84464E-2,8.259007E-2,5.5882495E-2,4.3634407E-2,4.864037E-2,0E0,4.537849E-2,8.354357E-2,1.8657587E-2,1.5502267E-2,0E0,1.9534279E-2,5.267284E-2,2.3609497E-2,4.1716885E-2,2.1403857E-2,2.0815924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2154378E-2,4.371713E-2,4.4934556E-2,0E0,1.089422E-2,2.8820923E-2,5.3309254E-2,0E0,0E0,0E0,0E0,1.3273055E-2,0E0,1.3264997E-2,2.0031199E-2,2.1552466E-2,2.3880532E-2,0E0,0E0,2.0580877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,28,28,29,29,30,30,32,32,33,33,34,34,39,39,41,41,42,42,43,43,44,44,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,-1,46,48,50,-1,-1,-1,-1,52,-1,54,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.8236975E11,5.9337012E7,3.9360254E2,8.0522484E11,1.04E3,3.6030095E10,1.9689655E1,1.15062E5,-1.2111978E-2,7.294459E7,7.65E2,5.8199444E0,3.6377484E7,-6.7835744E-3,1.7081589E2,4.2060562E1,1E0,8.657441E7,1.6491606E3,5.534958E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,8.351808E-1,3.2022728E1,4.497684E1,5.3193234E-3,2.6921875E1,1E0,6.695E3,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,5.093772E-1,3.6974133E-3,5.1184835E0,4.8887E5,3.7E1,2.0133628E6,2.2003332E-3,-1.1724293E-3,5.0123274E-1,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"split_indices":[53,32,46,53,32,0,5,59,30,0,46,0,54,1,0,4,59,65,46,4,28,0,0,0,0,0,0,0,28,59,59,0,57,89,2,0,0,0,0,28,0,54,1,3,46,0,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.74E2,2.7E1,1.53E2,2.1E1,1.4E1,1.3E1,6.7E1,8.6E1,2E0,1.9E1,5E0,9E0,1.1E1,2E0,1.9E1,4.8E1,2.3E1,6.3E1,1.2E1,7E0,2E0,3E0,4E0,5E0,8E0,3E0,2E0,1.7E1,2.9E1,1.9E1,6E0,1.7E1,5.4E1,9E0,6E0,6E0,3E0,4E0,1.2E1,5E0,2E1,9E0,5E0,1.4E1,1.5E1,2E0,5E1,4E0,4E0,5E0,7E0,5E0,4E0,1.6E1,7E0,2E0,3E0,2E0,7E0,7E0,2.4E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-3.738608E-4,-7.6801833E-3,1.01941355E-1,7.692978E-3,-4.625046E-2,1.483836E-1,7.440007E-4,-9.925026E-3,6.1333936E-2,-8.378208E-2,-1.7553208E-2,1.8119766E-1,1.0880003E-3,-9.9033825E-2,-3.6509018E-3,1.2566844E-1,3.761748E-2,-1.050452E-1,-2.485187E-3,-5.857001E-3,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,1.0020773E-4,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.9409139E-2,5.540788E-3,-1.2027661E-1,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,1.2911957E-2,-6.143175E-2,2.4736589E-2,-1.5001127E-2,2.155236E-3,-5.720792E-4,-9.470102E-4,-1.3429011E-1,3.1634532E-2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,39,-1,41,-1,-1,-1,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5702043E-1,1.1657343E-1,5.407299E-2,1.3386032E-1,5.8844402E-2,3.31683E-2,0E0,5.945997E-2,4.867518E-2,4.122594E-2,4.3614455E-2,1.10411495E-2,0E0,1.9426428E-2,4.644964E-2,1.4112622E-2,3.5269197E-2,1.4692083E-2,8.812408E-3,3.3128686E-2,0E0,0E0,0E0,0E0,0E0,3.7203256E-2,0E0,0E0,0E0,1.8351674E-2,0E0,1.8578902E-2,0E0,0E0,0E0,1.6290672E-2,2.7178083E-2,4.342707E-2,2.3945123E-2,0E0,0E0,0E0,1.3021171E-2,2.1517647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,29,29,31,31,35,35,36,36,37,37,38,38,42,42,43,43],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,40,-1,42,-1,-1,-1,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,3.0143394E7,7.7434E4,7.7567725E2,2.3443186E3,2.0867E4,7.440007E-4,1.0135136E0,5.63905E6,1.07431E1,1.3844339E1,1.04E3,1.0880003E-3,5.045749E-1,1.3237829E7,1E0,1.3526E4,4.5E1,1.476E3,1.09495016E8,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,6.0130353E0,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.7115049E6,5.540788E-3,2.95E2,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,2.249985E5,2.884462E0,2.7496E4,9.0636E0,2.155236E-3,-5.720792E-4,-9.470102E-4,2.1E1,5.57E2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"split_indices":[2,46,2,53,4,9,0,57,51,55,58,0,0,28,51,102,2,3,0,46,0,0,0,0,0,54,0,0,0,48,0,0,0,0,0,34,54,30,55,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.95E2,1.3E1,1.4E2,5.5E1,8E0,5E0,1.06E2,3.4E1,2.3E1,3.2E1,6E0,2E0,6E0,1E2,8E0,2.6E1,1.8E1,5E0,3E1,2E0,3E0,3E0,3E0,3E0,9.8E1,2E0,2E0,6E0,2.2E1,4E0,1.4E1,4E0,3E0,2E0,2.3E1,7E0,3.7E1,6.1E1,1.2E1,1E1,2E0,1.2E1,1.5E1,8E0,4E0,3E0,9E0,2.8E1,2.1E1,4E1,1E1,2E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.0470916E-4,-1.1145463E-2,9.843046E-2,-6.1118633E-2,4.996951E-3,1.2766055E-2,5.4078806E-2,-2.5317255E-2,-1.1859635E-1,-1.923276E-2,6.010194E-2,-2.131724E-4,3.8350024E-3,-5.8678132E-3,-2.225848E-3,-1.1466437E-2,-6.446723E-2,-1.1111286E-2,-6.9062617E-3,1.0727382E-1,4.0534303E-2,2.531128E-3,-8.6874526E-4,7.2951976E-4,-8.027679E-2,-5.5201775E-3,-5.2759657E-3,3.1679564E-3,8.534004E-3,2.4647668E-2,1.0526845E-1,-5.7665943E-3,-2.0202387E-3,-5.2320316E-2,9.702222E-3,3.7629638E-2,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.165946E-1,1.4873593E-1,1.3118732E-1,8.959766E-2,1.8808083E-1,0E0,2.4914969E-2,1.1248944E-2,9.248865E-2,9.6062645E-2,3.534253E-2,0E0,0E0,1.2025353E-2,0E0,0E0,1.753379E-2,4.9994655E-2,0E0,2.3044884E-2,3.0314423E-2,0E0,0E0,0E0,1.1376724E-2,6.391215E-2,0E0,0E0,0E0,1.0773087E-2,2.1219812E-2,0E0,0E0,2.3858346E-2,4.077713E-2,1.2596151E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,16,16,17,17,19,19,20,20,24,24,25,25,29,29,30,30,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1E0,1.4164306E-3,6.107924E11,8.3764136E2,1.2766055E-2,6.804E3,4.327473E5,8.0522484E11,4.462437E7,6.518218E6,-2.131724E-4,3.8350024E-3,8.6E1,-2.225848E-3,-1.1466437E-2,1E0,6.0073395E8,-6.9062617E-3,4.6008915E6,1.56E4,2.531128E-3,-8.6874526E-4,7.2951976E-4,3.7614656E5,1.292E3,-5.2759657E-3,3.1679564E-3,8.534004E-3,9.087618E6,9.56E2,-5.7665943E-3,-2.0202387E-3,2.8434807E3,7.269755E0,8.461558E10,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"split_indices":[53,102,58,32,53,0,9,29,32,46,46,0,0,10,0,0,8,7,0,46,2,0,0,0,29,2,0,0,0,33,0,0,0,48,54,32,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.83E2,1.9E1,4.4E1,1.39E2,3E0,1.6E1,2.8E1,1.6E1,9.7E1,4.2E1,5E0,1.1E1,1.5E1,1.3E1,4E0,1.2E1,9.2E1,5E0,1.1E1,3.1E1,2E0,1.3E1,2E0,1E1,8.8E1,4E0,8E0,3E0,2.6E1,5E0,4E0,6E0,2.1E1,6.7E1,1.8E1,8E0,2E0,3E0,6E0,1.5E1,4.2E1,2.5E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.050566E-3,4.2812275E-3,-1.5856923E-1,-1.1301997E-2,5.6136116E-2,-2.1941631E-4,-1.0672151E-2,-7.0011653E-3,-1.2052315E-1,8.916989E-3,1.02983706E-1,-5.7596188E-2,1.8009902E-3,-2.245222E-3,-8.737236E-3,-5.627105E-3,5.7158438E-3,1.7363462E-1,6.978703E-2,-1.5238659E-2,-8.90067E-2,1.7688595E-2,-2.4926206E-2,-3.5939007E-3,4.3866397E-3,1.0011509E-2,3.031237E-3,8.032361E-2,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,3.1169657E-2,-2.7009284E-2,-6.0450863E-2,-1.2030017E-3,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7956841E-1,1.679042E-1,6.4495996E-2,7.4314885E-2,1.0526259E-1,0E0,0E0,6.9132246E-2,1.8055126E-2,4.0123533E-2,4.4171542E-2,2.9244505E-2,5.6916848E-2,0E0,0E0,1.6036227E-2,0E0,1.2192935E-2,1.5846536E-2,1.4111153E-2,3.3006206E-2,5.1458158E-2,4.2065214E-2,0E0,8.906513E-3,0E0,0E0,9.820163E-3,0E0,0E0,0E0,0E0,0E0,2.6585214E-2,3.6565024E-2,2.0105965E-2,2.8099384E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,3.0455479E3,8.667E3,1.8600049E10,1E0,-2.1941631E-4,-1.0672151E-2,1.292E3,3.240192E6,8.129243E5,1.2117E4,8.137731E7,2.73782E5,-2.245222E-3,-8.737236E-3,2.3125E0,5.7158438E-3,1.884E3,7.317201E0,1.1086122E3,4.1749332E4,1.06403265E1,2.2009405E1,-3.5939007E-3,1.3174513E4,1.0011509E-2,3.031237E-3,3.2707875E8,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,2.0589474E1,5.3153326E2,6.704386E6,4.462437E7,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"split_indices":[46,4,2,5,102,0,0,2,7,29,9,5,30,0,0,54,0,0,54,34,33,55,57,0,4,0,0,33,0,0,0,0,0,59,4,46,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.06E2,6E0,1.59E2,4.7E1,2E0,4E0,1.54E2,5E0,2.4E1,2.3E1,2.2E1,1.32E2,3E0,2E0,2.2E1,2E0,6E0,1.7E1,1E1,1.2E1,8.3E1,4.9E1,2E0,2E1,4E0,2E0,1.5E1,2E0,3E0,7E0,6E0,6E0,6.4E1,1.9E1,1.9E1,3E1,1.5E1,5E0,1.2E1,3E0,8E0,5.6E1,1E1,9E0,3E0,1.6E1,2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.4155842E-3,-1.3810372E-3,1.2642601E-2,-1.6525343E-2,4.4375908E-2,-1.2493689E-2,-8.009961E-3,1.661861E-2,9.891294E-2,-8.0245316E-2,-4.4310438E-3,4.9835492E-2,-6.8227405E-4,3.6446485E-3,9.054623E-3,-2.6625823E-2,-1.3053659E-1,-7.4558124E-2,1.1256639E-3,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,5.806915E-3,-7.995392E-2,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,9,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[2.0164502E-1,1.420966E-1,0E0,8.987266E-2,7.556003E-2,8.2129285E-2,0E0,2.0147853E-2,2.1653563E-2,3.9669834E-2,5.307982E-2,1.9236067E-2,1.3649627E-2,0E0,0E0,9.523076E-3,1.3058096E-2,3.7478834E-2,4.8667632E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2858007E-2,2.4820045E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,29,29,30,30],"right_children":[2,4,-1,6,8,10,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.8375797E2,1.6572577E0,1.2642601E-2,7.5656E4,1.9846272E9,4.994744E-5,-8.009961E-3,1.5489E4,9.246969E-1,8.7793425E-2,1.9E1,7.614E3,5E0,3.6446485E-3,9.054623E-3,2.8800612E9,3.2445145E2,2.75108E5,1.4655151E0,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,1E0,3.14608E6,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"split_indices":[57,42,0,10,5,43,0,9,28,59,3,10,8,0,0,5,34,9,42,0,0,0,0,0,0,0,0,0,0,89,49,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,2.03E2,2E0,1.53E2,5E1,1.5E2,3E0,3.4E1,1.6E1,1.5E1,1.35E2,1.1E1,2.3E1,1.4E1,2E0,8E0,7E0,9E0,1.26E2,6E0,5E0,1.6E1,7E0,2E0,6E0,5E0,2E0,6E0,3E0,1.2E2,6E0,1.07E2,1.3E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.000657E-3,-1.0153504E-2,9.496438E-2,-3.1673333E-3,-7.777414E-2,1.5645174E-2,6.2693365E-2,5.9380285E-3,-5.026301E-2,-9.674218E-3,-2.5248412E-2,9.625793E-2,2.6888123E-2,-6.8429126E-3,4.4476364E-2,-2.1850644E-2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,8.983803E-3,-4.372294E-2,6.858251E-3,1.07610226E-1,-6.777975E-2,-4.1921283E-4,-1.2492558E-2,3.5249196E-2,-9.645313E-3,-8.684807E-2,-1.2344087E-3,1.4300514E-3,1.2848052E-1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,29,31,33,35,37,-1,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5137326E-1,8.573215E-2,1.699006E-1,7.148311E-2,1.03037156E-1,0E0,2.6254699E-2,6.926221E-2,8.881589E-2,0E0,1.071448E-2,2.5661737E-2,2.7722035E-2,6.226391E-2,8.238617E-2,1.3398929E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2746734E-2,4.618959E-2,1.7440343E-2,9.797484E-3,1.0284005E-2,0E0,3.8003672E-2,4.3830436E-2,3.493194E-2,1.7143585E-2,0E0,0E0,1.4258161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,30,32,34,36,38,-1,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.3983983E10,1.4164306E-3,1.2670352E1,5.535E3,1.5645174E-2,1.095857E10,6.695E3,2.0662694E-1,-9.674218E-3,2.1000721E0,1.7173885E8,3.6377484E7,1.6278354E7,4.485647E5,2.650027E2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,5.097052E8,7.149117E-1,4.602015E0,8.564948E6,2.3968115E2,-4.1921283E-4,1.5498047E1,6.034607E6,9.737113E0,5.8426323E0,-1.2344087E-3,1.4300514E-3,6.879445E1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"split_indices":[53,5,58,55,9,0,12,2,39,0,58,33,1,46,29,4,0,0,0,0,0,0,0,5,28,54,33,4,0,57,49,55,54,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.81E2,2.5E1,1.65E2,1.6E1,2E0,2.3E1,1.39E2,2.6E1,4E0,1.2E1,1.1E1,1.2E1,1.05E2,3.4E1,2.2E1,4E0,8E0,4E0,8E0,3E0,7E0,5E0,7.4E1,3.1E1,2.2E1,1.2E1,4E0,1.8E1,4.1E1,3.3E1,1.8E1,1.3E1,9E0,1.3E1,8E0,4E0,2E0,2E0,3.5E1,6E0,2.5E1,8E0,1.2E1,6E0,8E0,5E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.156695E-3,-7.7511407E-3,6.4903505E-2,1.10561E-3,-6.161546E-2,8.889031E-2,9.5223775E-3,-1.1973289E-2,2.9857075E-2,-4.3537878E-2,-9.037242E-3,8.965046E-4,1.1017409E-1,-6.7902607E-4,2.1986503E-3,7.537986E-3,-3.928328E-2,6.545862E-2,1.1178152E-2,5.3028204E-2,-6.55025E-2,1.2060009E-3,6.3895895E-3,3.2280452E-4,6.727481E-2,-7.918549E-2,-8.271147E-3,3.5601782E-4,7.7395335E-2,2.8093105E-2,-2.6743395E-2,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,6.4077914E-2,-8.103772E-3,-7.31776E-5,4.6275426E-3,2.5301395E-5,-9.328173E-2,-5.3631708E-2,1.3650758E-2,2.564571E-3,6.1303745E-3,2.7586694E-3,-5.2228123E-3,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,-1,-1,-1,-1,49,51,-1,-1,-1,53,55,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5891957E-1,8.784768E-2,4.6976537E-2,6.014745E-2,5.0235853E-2,3.546658E-2,1.0674542E-2,5.890991E-2,3.251822E-2,5.437839E-2,0E0,0E0,3.1012043E-2,0E0,0E0,2.7994063E-2,5.6267872E-2,1.12324655E-2,2.2560306E-2,1.894341E-2,1.4103778E-2,0E0,0E0,3.2216277E-2,1.35331005E-2,2.251944E-2,2.7580641E-2,0E0,1.0156989E-2,2.3157822E-2,1.0649132E-2,0E0,0E0,0E0,0E0,1.371366E-2,2.1627188E-2,0E0,0E0,0E0,1.5584886E-2,1.2574337E-2,9.651603E-3,0E0,0E0,0E0,1.44525645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,35,35,36,36,40,40,41,41,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,-1,-1,-1,-1,50,52,-1,-1,-1,54,56,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,5.075104E7,5.1E1,6.787619E2,6.3538E4,6.19999E9,9.532E3,1.7086E4,1E0,5.577E3,-9.037242E-3,8.965046E-4,1.5214152E5,-6.7902607E-4,2.1986503E-3,1E0,2.5259212E5,2.695E3,7.6760286E-1,2.083122E6,5.534958E-1,1.2060009E-3,6.3895895E-3,4.570007E7,2.77E2,5.8910837E0,2.368447E6,3.5601782E-4,1.4854577E5,8.729897E0,2.3117E4,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,1.297E3,2.0200117E-1,-7.31776E-5,4.6275426E-3,2.5301395E-5,1.2318777E3,2.187288E1,3.3E1,2.564571E-3,6.1303745E-3,2.7586694E-3,7.9206874E8,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"split_indices":[53,46,3,53,10,5,9,9,8,9,0,0,34,0,0,74,29,2,28,1,28,0,0,5,0,54,33,0,34,55,10,0,0,0,0,2,28,0,0,0,4,59,3,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.83E2,3.5E1,1.58E2,2.5E1,2.4E1,1.1E1,1.09E2,4.9E1,2.3E1,2E0,6E0,1.8E1,7E0,4E0,6.4E1,4.5E1,1.6E1,3.3E1,4E0,1.9E1,4E0,1.4E1,5.8E1,6E0,1.9E1,2.6E1,3E0,1.3E1,2.3E1,1E1,2E0,2E0,5E0,1.4E1,6E0,5.2E1,2E0,4E0,3E0,1.6E1,8E0,1.8E1,1E1,3E0,1.2E1,1.1E1,4E0,6E0,3E0,3E0,6E0,4.6E1,1.4E1,2E0,3E0,5E0,1.1E1,7E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-8.12835E-4,2.8342765E-3,-1.5601721E-1,-1.7404577E-2,2.4213504E-2,-2.3796102E-3,-1.0275062E-2,-1.0378984E-2,-1.0568353E-1,9.81486E-3,1.8756628E-2,-2.6070654E-2,1.303203E-2,-6.017882E-4,-7.747756E-3,5.4938868E-2,-4.513947E-3,-1.7527992E-2,-4.4134418E-3,-1.0665796E-2,3.9632928E-2,1.0806894E-1,3.8279194E-2,1.5463653E-3,-6.14344E-3,-6.364611E-2,-6.1187022E-3,2.6128873E-2,-4.657165E-2,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.8797043E-2,-8.169748E-4,-2.9431539E-2,2.5399424E-2,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,-1,-1,21,23,25,-1,27,29,31,33,35,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2327455E-1,9.2584625E-2,1.9196324E-2,6.712644E-2,9.692997E-2,0E0,0E0,3.8177617E-2,3.798466E-2,0E0,8.61301E-2,3.2634463E-2,2.6898822E-2,0E0,0E0,3.1268403E-2,4.6668507E-2,2.9076368E-2,0E0,3.1603176E-2,3.4378827E-2,1.0986567E-2,2.5717452E-2,4.5821525E-2,0E0,1.2384877E-2,2.5712674E-2,1.1580934E-2,1.3976639E-2,0E0,0E0,0E0,0E0,1.3670683E-2,1.1835324E-2,1.6557261E-2,5.3196885E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,-1,-1,22,24,26,-1,28,30,32,34,36,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,6.978001E2,2.9E1,4.462437E7,1.7727281E2,-2.3796102E-3,-1.0275062E-2,7.555544E2,9.214292E7,9.81486E-3,2.4611046E7,1.9284746E1,9.258696E0,-6.017882E-4,-7.747756E-3,7.573E3,1.3844339E1,2.547512E2,-4.4134418E-3,6.855402E6,1.2670352E1,1.884E3,6.5829754E-1,2.819228E-1,-6.14344E-3,3.7790768E2,1.3696512E7,6.735744E1,6.8539324E0,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.982359E-1,2.709E4,7.340202E1,1.3888E4,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"split_indices":[46,53,3,46,34,0,0,4,7,0,46,57,55,0,0,9,58,56,0,46,55,0,28,58,0,34,46,59,55,0,0,0,0,28,9,57,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,2.12E2,4E0,1.09E2,1.03E2,2E0,2E0,1.02E2,7E0,2E0,1.01E2,6.1E1,4.1E1,3E0,4E0,3.9E1,6.2E1,5.5E1,6E0,2.2E1,1.9E1,8E0,3.1E1,6E1,2E0,1E1,4.5E1,1.1E1,1.1E1,1E1,9E0,5E0,3E0,2E1,1.1E1,2.6E1,3.4E1,2E0,8E0,3.7E1,8E0,7E0,4E0,5E0,6E0,1.7E1,3E0,8E0,3E0,1.5E1,1.1E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.6912411E-3,-1.9064195E-2,2.4100082E-2,-6.059589E-3,-7.711967E-2,4.095765E-2,-4.2764593E-2,-5.51559E-2,1.8715139E-3,-1.06207624E-1,-1.2241886E-3,1.0488696E-2,3.2134976E-2,-7.6291934E-3,-1.7113544E-2,5.660094E-4,-7.0297986E-2,5.156652E-3,-5.481648E-3,-6.6033505E-2,-8.217553E-3,7.82574E-3,2.3170885E-2,-4.49811E-2,1.0298288E-3,-4.236804E-3,9.818355E-4,2.7860066E-2,-4.866169E-3,-4.3798788E-3,-7.048719E-4,9.212592E-4,4.4673063E-2,-4.4644894E-3,-3.072789E-4,4.526607E-2,-1.4959747E-2,-3.7038807E-2,1.067402E-2,-6.5524145E-3,3.5458216E-3,2.2589561E-4,8.1912346E-2,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,25,27,-1,29,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,43,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.82962E-2,1.1127101E-1,9.417995E-2,4.7608547E-2,3.814037E-2,9.47225E-2,4.7029093E-2,1.8009417E-2,4.045555E-2,3.1635314E-2,0E0,0E0,7.009145E-2,0E0,1.6951097E-2,0E0,2.2984944E-2,2.3801006E-2,0E0,1.3520818E-2,0E0,0E0,2.9142186E-2,1.5691848E-2,0E0,0E0,0E0,2.4527468E-2,3.6893085E-2,0E0,0E0,1.7144382E-2,4.5832694E-2,0E0,0E0,1.8606756E-2,1.2692557E-2,1.1324577E-2,3.5423E-2,2.0130176E-2,0E0,0E0,1.3577841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,16,16,17,17,19,19,22,22,23,23,27,27,28,28,31,31,32,32,35,35,36,36,37,37,38,38,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,26,28,-1,30,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,44,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,2.325945E9,1.09495016E8,1.142E3,7.888087E0,4.164E3,2.2970924E-2,4.569361E7,1.2224206E7,8.260109E9,-1.2241886E-3,1.0488696E-2,1.1307017E3,-7.6291934E-3,8.66076E1,5.660094E-4,5.9215684E0,2.6374866E1,-5.481648E-3,5.754E3,-8.217553E-3,7.82574E-3,7.6E0,2.884462E0,1.0298288E-3,-4.236804E-3,9.818355E-4,7.438972E0,4.2060562E1,-4.3798788E-3,-7.048719E-4,7.361366E7,2.6023E4,-4.4644894E-3,-3.072789E-4,6.8623977E0,7.444958E7,6.37E2,6.191606E0,8.062193E-1,3.5458216E-3,2.2589561E-4,7.3586698E0,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"split_indices":[53,5,46,2,54,12,28,5,51,5,0,0,53,0,57,0,57,59,0,2,0,0,55,54,0,0,0,54,59,0,0,46,10,0,0,54,7,0,54,28,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.47E2,8.1E1,1.21E2,2.6E1,6.5E1,1.6E1,1.6E1,1.05E2,1.6E1,1E1,2E0,6.3E1,2E0,1.4E1,3E0,1.3E1,1.03E2,2E0,1.1E1,5E0,3E0,6E1,8E0,6E0,1.1E1,2E0,3.1E1,7.2E1,7E0,4E0,3E1,3E1,3E0,5E0,2.2E1,9E0,2.3E1,4.9E1,2.8E1,2E0,1.5E1,1.5E1,1.5E1,7E0,4E0,5E0,1.1E1,1.2E1,2.7E1,2.2E1,1.4E1,1.4E1,1.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.5013376E-3,5.938352E-3,-1.24217264E-1,-2.7496833E-3,8.3399355E-2,-8.301269E-3,-2.700483E-4,-3.2061845E-2,9.8344935E-3,1.079024E-2,4.594788E-2,-1.9474141E-2,-7.3742555E-3,6.1866198E-2,-1.161049E-3,4.8218677E-3,7.569026E-4,-8.262385E-2,-3.5967962E-3,9.7572885E-2,-1.3912333E-3,3.422547E-2,-7.907227E-3,-1.2076516E-3,-6.5928353E-3,5.179689E-2,-2.5634378E-2,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,-5.0402116E-2,2.8263181E-3,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,29,31,-1,-1,33,35,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.14531964E-1,1.3348015E-1,3.7561677E-2,6.634294E-2,9.341748E-2,0E0,0E0,7.825357E-2,7.215855E-2,0E0,2.5913164E-2,4.983531E-2,0E0,7.3844664E-2,2.5336621E-2,0E0,0E0,2.7188063E-2,5.1378243E-2,3.849563E-2,0E0,1.8305652E-2,4.0737204E-2,0E0,0E0,2.3600813E-2,3.192762E-2,0E0,0E0,0E0,0E0,2.3702014E-2,4.1213304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,22,22,25,25,26,26,31,31,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,30,32,-1,-1,34,36,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,8.861832E3,6.45153E5,4.039435E-3,1.4164306E-3,-8.301269E-3,-2.700483E-4,2.0604518E8,7.573E3,1.079024E-2,3.0904205E2,7.5999E4,-7.3742555E-3,8.19162E5,1.2402962E-1,4.8218677E-3,7.569026E-4,4.1749332E4,5.4127317E10,1.946E3,-1.3912333E-3,7.317201E0,3.0738266E4,-1.2076516E-3,-6.5928353E-3,7.356E3,1.9990385E1,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,3.7E1,2.2759746E7,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"split_indices":[46,4,29,39,58,0,0,7,9,0,59,1,0,30,42,0,0,33,32,2,0,54,34,0,0,9,59,0,0,0,0,3,46,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.97E2,6E0,1.78E2,1.9E1,4E0,2E0,5.3E1,1.25E2,3E0,1.6E1,4.9E1,4E0,2.1E1,1.04E2,5E0,1.1E1,9E0,4E1,1.5E1,6E0,1.6E1,8.8E1,5E0,4E0,1.1E1,2.9E1,3E0,1.2E1,1E1,6E0,1.7E1,7.1E1,2E0,9E0,2.1E1,8E0,1.3E1,4E0,5.2E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.211318E-5,-7.6537346E-3,7.079598E-2,6.928152E-3,-3.6480606E-2,1.4365091E-2,3.16417E-2,-9.043619E-3,5.276066E-2,-6.891165E-2,-9.199198E-3,6.183768E-2,-4.7232076E-3,7.881159E-3,-3.276541E-2,7.939335E-3,8.772806E-2,-3.0282011E-2,-1.1089681E-1,-1.8124288E-2,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,1.8954586E-2,-4.7628164E-2,2.5371954E-2,2.6573949E-3,-1.2165993E-2,5.2440288E-3,4.916817E-4,-1.1942704E-2,-4.229069E-3,-3.970585E-5,-1.548476E-1,2.0617556E-3,-4.2224076E-2,6.205118E-2,4.711567E-4,-7.918416E-2,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,2.3531545E-2,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1673426E-1,8.264138E-2,1.751531E-1,9.6372604E-2,5.776053E-2,0E0,7.98365E-2,3.9603867E-2,5.2591182E-2,4.529798E-2,3.7174046E-2,1.3060965E-2,0E0,4.1106842E-2,3.6635753E-2,1.5675733E-2,2.8447852E-2,1.6613591E-2,6.760767E-2,1.7113835E-2,0E0,0E0,0E0,0E0,3.9799407E-2,3.9343596E-2,1.4240889E-2,0E0,1.0493089E-2,0E0,0E0,1.1072869E-2,0E0,0E0,1.0195777E-2,1.9533398E-2,1.2358395E-2,2.6103884E-2,2.7533289E-2,3.5623252E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1920147E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,25,25,26,26,28,28,31,31,34,34,35,35,36,36,37,37,38,38,39,39,50,50],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,2.2971932E7,1.4164306E-3,7.7567725E2,9.875E3,1.4365091E-2,1.402511E8,1.7086E4,4.5470334E5,6.107924E11,3.5632E4,8.0426145E-1,-4.7232076E-3,5.421359E0,1.5071542E0,3.4706132E-3,4.493248E6,7.951E3,2.358119E-1,7.4500206E5,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,4.573327E5,4.4589956E5,1.69228E5,2.6573949E-3,1.8590495E9,5.2440288E-3,4.916817E-4,9.1487586E-1,-4.229069E-3,-3.970585E-5,1.96995E10,1.1036333E3,1.1192292E1,1.4214917E3,4.570007E7,5.6111975E0,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,3.3478114E-1,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"split_indices":[53,46,58,53,9,0,46,9,29,32,2,28,0,55,58,28,1,9,28,49,0,0,0,0,49,29,30,0,5,0,0,28,0,0,5,34,55,49,5,54,0,0,0,0,0,0,0,0,0,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.95E2,2E1,1.3E2,6.5E1,2E0,1.8E1,9.7E1,3.3E1,2.9E1,3.6E1,1.5E1,3E0,5.7E1,4E1,1.5E1,1.8E1,1.6E1,1.3E1,3.4E1,2E0,8E0,7E0,8E0,4.9E1,3.2E1,8E0,4E0,1.1E1,1.4E1,4E0,1.3E1,3E0,4E0,9E0,1.9E1,1.5E1,1.4E1,3.5E1,1.7E1,1.5E1,2E0,6E0,5E0,6E0,9E0,4E0,5E0,4E0,6E0,1.3E1,1.2E1,3E0,2E0,1.2E1,6E0,2.9E1,3E0,1.4E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.1967006E-3,8.5786525E-3,-1.4402847E-1,8.315427E-4,8.315179E-2,-9.1837915E-3,-1.7736525E-3,-1.9747915E-2,1.3641278E-2,1.3899025E-2,4.2534754E-2,-1.29205575E-2,-7.1898274E-2,3.6052603E-2,-8.618837E-3,3.79517E-4,7.254509E-2,-1.8197084E-2,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,2.5952812E-2,-1.9911975E-2,1.2206658E-3,1.6528742E-4,4.4683916E-3,-2.9289007E-2,1.7770747E-2,5.043673E-2,-5.733453E-3,2.2968883E-3,-2.780274E-2,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.356853E-1,1.1663157E-1,2.196373E-2,4.877499E-2,1.4408445E-1,0E0,0E0,2.4350522E-2,5.7182897E-2,0E0,1.7155953E-2,2.955227E-2,1.8783536E-2,5.395387E-2,2.2573741E-2,0E0,1.1681624E-2,2.485658E-2,0E0,0E0,0E0,0E0,4.1406788E-2,2.4611853E-2,0E0,0E0,0E0,1.7023295E-2,2.698188E-2,2.68047E-2,2.1004716E-2,0E0,2.7273526E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,22,22,23,23,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,9.417025E3,1.6330187E8,8.1913336E2,1.4164306E-3,-9.1837915E-3,-1.7736525E-3,1.996866E1,4.91101E5,1.3899025E-2,8.476884E-1,1.6482397E1,2.25813E5,6.37E3,1.0807868E7,3.79517E-4,3.5766065E0,3.8E1,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,6.995449E-1,2E1,1.2206658E-3,1.6528742E-4,4.4683916E-3,1.5662762E10,6.2734294E0,7.269755E0,1E0,2.2968883E-3,1.01618E5,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"split_indices":[46,4,46,56,58,0,0,57,30,0,35,57,1,9,1,0,36,3,0,0,0,0,28,3,0,0,0,32,55,54,101,0,11,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,2.01E2,5E0,1.83E2,1.8E1,3E0,2E0,7E1,1.13E2,2E0,1.6E1,6.3E1,7E0,5.6E1,5.7E1,8E0,8E0,6E1,3E0,2E0,5E0,4E0,5.2E1,4.3E1,1.4E1,2E0,6E0,4.6E1,1.4E1,2.9E1,2.3E1,4E0,3.9E1,2.2E1,2.4E1,4E0,1E1,2.3E1,6E0,1.7E1,6E0,2.1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.9543333E-3,-2.559017E-2,1.9161552E-2,-2.0923022E-2,-5.902389E-3,2.6990097E-2,-7.9133324E-2,-8.834234E-3,-4.04624E-2,1.3379109E-2,7.079206E-2,3.3201702E-4,-8.117297E-3,8.160095E-3,-2.9381018E-2,-7.949769E-2,-1.22999055E-2,4.4339493E-2,-1.4131938E-2,1.0944012E-1,-7.471799E-4,7.50054E-2,-6.7173177E-3,-5.5795915E-2,-1.1298074E-3,-5.1226197E-3,1.4864717E-3,-1.065279E-3,-4.5908066E-3,-3.7182914E-3,6.852332E-2,-5.39476E-2,8.511279E-3,1.29912235E-2,5.2558865E-2,4.981872E-4,4.811432E-3,-3.245346E-3,1.7857874E-3,-2.4376418E-2,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,8.3817184E-2,-7.3740636E-3,-2.728049E-2,5.5922586E-2,-1.4394672E-2,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,-1,45,47,49,51,-1,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0851875E-1,4.7482766E-2,8.293395E-2,2.5152944E-2,0E0,5.747927E-2,6.141771E-2,2.3983913E-2,4.4094764E-2,6.53598E-2,7.825514E-2,0E0,0E0,3.8294274E-2,2.30708E-2,5.1408485E-2,2.2939531E-2,4.2094745E-2,3.7422903E-2,1.2931412E-1,0E0,9.805635E-3,1.6436392E-2,1.2211446E-2,2.4566848E-2,0E0,0E0,9.913108E-3,0E0,1.5312212E-2,2.8017767E-2,3.489702E-2,3.0127015E-2,0E0,2.5919788E-2,0E0,0E0,0E0,1.9405605E-2,2.0426273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.549913E-2,0E0,1.4559356E-2,1.8774323E-2,1.5052079E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,34,34,38,38,39,39,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,-1,46,48,50,52,-1,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,8.260109E9,1.4623123E8,1.7086E4,-5.902389E-3,9.276886E1,1.5214152E5,4.327473E5,2.5259212E5,2.4934822E7,5.2E1,3.3201702E-4,-8.117297E-3,4.570007E7,4.9054803E2,4.758E3,4.1969254E8,2.8626094E5,5.8816573E9,1.6064256E-2,-7.471799E-4,7.444958E7,8.8916E4,2.79E2,6.027375E0,-5.1226197E-3,1.4864717E-3,1.194E3,-4.5908066E-3,1E0,1E0,2.9E1,3.800374E6,1.29912235E-2,3.5935396E-1,4.981872E-4,4.811432E-3,-3.245346E-3,2.0558643E8,3.6E1,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,1.5262272E3,-7.3740636E-3,4E0,5.57E2,2.5848E4,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"split_indices":[53,5,46,9,0,57,34,29,29,46,3,0,0,5,56,2,7,29,5,58,0,7,1,0,54,0,0,10,0,8,102,3,1,0,58,0,0,0,7,10,0,0,0,0,0,0,0,0,4,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.11E2,1.04E2,1.07E2,4E0,9.7E1,7E0,6.7E1,4E1,7.5E1,2.2E1,4E0,3E0,3.7E1,3E1,1.6E1,2.4E1,3.5E1,4E1,1.5E1,7E0,6E0,3.1E1,1.5E1,1.5E1,1.3E1,3E0,2.2E1,2E0,1.2E1,2.3E1,1.4E1,2.6E1,3E0,1.2E1,2E0,4E0,3E0,2.8E1,8E0,7E0,7E0,8E0,1.2E1,1E1,5E0,7E0,4E0,1.9E1,2E0,1.2E1,8E0,1.8E1,4E0,8E0,1.6E1,1.2E1,4E0,4E0,5E0,1.4E1,7E0,5E0,2E0,6E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.1127363E-3,-5.290152E-3,6.0248755E-2,-5.542424E-2,-1.2838513E-3,6.840342E-3,8.6930476E-4,1.3661059E-4,-7.317434E-2,-5.0637187E-3,6.27141E-4,-3.2042078E-4,-4.331261E-3,-7.5224843E-3,3.989758E-2,6.3668867E-3,-2.6071595E-2,-3.1950418E-2,6.536836E-2,-2.7029186E-2,1.4747367E-2,-7.087237E-2,-1.15542095E-2,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,7.527232E-2,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,-1,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8252895E-2,3.5278834E-2,3.8336206E-2,1.3642367E-2,3.2272287E-2,0E0,0E0,0E0,9.951666E-3,0E0,5.21465E-2,0E0,0E0,3.4946784E-2,5.3329237E-2,2.2228872E-2,3.690375E-2,1.3936162E-2,1.901877E-2,1.1226752E-2,2.018778E-2,3.677965E-2,1.6500767E-2,0E0,0E0,0E0,1.2472153E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,-1,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8008E4,1.9388489E3,7.9684106E9,6.051637E-1,1E1,6.840342E-3,8.6930476E-4,1.3661059E-4,7.891632E1,-5.0637187E-3,9.198947E-1,-3.2042078E-4,-4.331261E-3,8.729897E0,7.5773336E7,4.817292E0,2.9E1,6.7995167E0,1.4740072E1,5.9145586E3,7.137324E0,1.6753285E-1,1.6144748E5,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,1.2633E4,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"split_indices":[2,49,12,28,3,0,0,0,52,0,28,0,0,55,7,54,3,55,59,4,54,40,29,0,0,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.75E2,1.1E1,1.2E1,1.63E2,3E0,8E0,3E0,9E0,2E0,1.61E2,2E0,7E0,1.34E2,2.7E1,7.7E1,5.7E1,7E0,2E1,1.5E1,6.2E1,1.3E1,4.4E1,4E0,3E0,2E0,1.8E1,9E0,6E0,5.5E1,7E0,5E0,8E0,1.5E1,2.9E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-9.433804E-4,-1.2022093E-2,5.917883E-2,-6.3716695E-2,-6.556876E-3,1.185642E-2,3.261322E-2,-7.952249E-2,-4.8074138E-4,-1.3831026E-2,3.4149677E-2,4.693409E-2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,-4.5358995E-4,-3.4879703E-2,-6.250601E-4,2.4170529E-3,7.417909E-2,8.9431816E-4,1.8055465E-2,-2.7939798E-2,-7.63545E-2,-1.986685E-2,4.411847E-3,-8.005692E-4,-7.7093607E-3,6.321066E-2,1.0480897E-2,-5.4052684E-2,-4.8673116E-3,-6.69903E-4,-7.4792397E-3,-7.8966774E-2,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,27,29,31,33,-1,-1,35,37,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4125362E-1,5.015369E-2,1.5375312E-1,1.3680756E-2,4.8737537E-2,0E0,6.7347795E-2,8.856602E-3,0E0,3.9123423E-2,1.886667E-2,2.0422306E-2,0E0,0E0,0E0,4.4256426E-2,3.210645E-2,0E0,0E0,2.239582E-2,0E0,6.0984924E-2,3.5681155E-2,1.9354679E-2,2.9425774E-2,0E0,0E0,1.626718E-2,2.6215322E-2,1.7650409E-2,2.1931183E-2,0E0,0E0,1.2872141E-2,1.4290784E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,15,15,16,16,19,19,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,28,30,32,34,-1,-1,36,38,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,4.060294E0,1.4164306E-3,4.668851E6,2.6493E4,1.185642E-2,1.18E2,7.7573473E9,-4.8074138E-4,1.7086E4,7.4353185E0,3.0904205E2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,1.5989196E7,2.2741E4,-6.250601E-4,2.4170529E-3,1.2003011E8,8.9431816E-4,4.984018E8,7.149117E-1,4.4589956E5,1.3000701E7,4.411847E-3,-8.005692E-4,2.80373E5,7.11E2,2.2809816E5,5.36E3,-4.8673116E-3,-6.69903E-4,1.2318777E3,1.1075563E3,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"split_indices":[53,54,58,30,10,0,3,5,0,9,55,59,0,0,0,46,9,0,0,33,0,5,28,29,51,0,0,1,0,29,2,0,0,4,53,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.78E2,3.2E1,1.6E1,1.62E2,3E0,2.9E1,1.2E1,4E0,1.38E2,2.4E1,2.7E1,2E0,8E0,4E0,8.5E1,5.3E1,6E0,1.8E1,1.3E1,1.4E1,5.1E1,3.4E1,1.3E1,4E1,1.1E1,2E0,3.3E1,1.8E1,1.4E1,2E1,9E0,4E0,3.4E1,6E0,1.9E1,1.4E1,1.3E1,5E0,4E0,1E1,1.5E1,5E0,2.8E1,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.7782936E-4,-1.382795E-2,2.3196248E-2,-7.2672334E-3,-1.0122567E-1,-7.0918067E-3,4.573724E-2,-4.658554E-3,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,2.595186E-2,-2.5460321E-2,5.915973E-3,3.5621542E-2,-5.366179E-2,-6.7920005E-4,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,1.8883888E-2,8.107459E-2,-4.6013836E-3,-9.936781E-5,2.8179128E-2,-8.45359E-3,7.830855E-4,5.3368773E-2,7.481408E-3,2.3286117E-3,-1.5134966E-3,4.5415547E-2,-1.37019865E-2,1.673101E-3,3.2098924E-3,-1.2978952E-2,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,-1,-1,31,33,35,37,-1,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.788673E-2,7.380529E-2,5.4942183E-2,3.1929776E-2,2.121576E-2,2.1904452E-2,3.0952565E-2,2.3363702E-2,0E0,0E0,0E0,9.656656E-3,1.7785855E-2,0E0,3.0421652E-2,1.8802509E-2,2.5366073E-2,0E0,0E0,0E0,0E0,1.993274E-2,1.9198395E-2,0E0,0E0,2.5934385E-2,2.0478453E-2,2.058842E-2,1.0298643E-2,0E0,0E0,0E0,1.0949407E-2,2.1243747E-2,0E0,0E0,1.2440793E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,15,15,16,16,21,21,22,22,25,25,26,26,27,27,28,28,32,32,33,33,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,-1,-1,32,34,36,38,-1,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.2224206E7,3E0,1.256E3,5.0655737E0,1.1475E2,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,1.7837838E0,2.3894766E7,5.915973E-3,4.4E1,7.8206635E0,3.06123E5,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,3.4170952E11,1.2117E4,-4.6013836E-3,-9.936781E-5,5.1184835E0,2.6493E4,2.95E2,3.5935396E-1,7.481408E-3,2.3286117E-3,-1.5134966E-3,7.856E3,2.325945E9,1.673101E-3,3.2098924E-3,3.5120192E-1,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"split_indices":[53,5,102,51,8,0,55,53,0,0,0,58,49,0,3,54,12,0,0,0,0,32,9,0,0,54,10,0,58,0,0,0,9,5,0,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.29E2,7.9E1,1.21E2,8E0,3.4E1,4.5E1,1.19E2,2E0,4E0,4E0,1.2E1,2.2E1,4E0,4.1E1,8E0,1.11E2,6E0,6E0,7E0,1.5E1,3.1E1,1E1,4E0,4E0,2.3E1,8.8E1,2.1E1,1E1,2E0,8E0,5E0,1.8E1,7.9E1,9E0,3E0,1.8E1,2E0,8E0,3E0,1.5E1,7E1,9E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.1277195E-4,4.217555E-3,-1.0916315E-1,-2.9917555E-3,5.3992495E-2,5.0713454E-4,-7.7418964E-3,3.2400577E-3,-3.849527E-2,1.5364976E-1,2.9980375E-2,-2.0002978E-2,1.4616362E-2,-1.2040921E-2,-9.3005456E-2,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,-2.5916894E-3,-7.337681E-2,2.9357852E-2,-1.9776661E-2,-3.767159E-2,6.012406E-4,-6.2919264E-3,-7.461606E-4,4.0725708E-2,-2.1920465E-2,-5.1689288E-3,-1.9261596E-3,3.854466E-2,-1.4079762E-2,-5.6224447E-3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7122716E-2,7.65951E-2,4.344309E-2,4.149871E-2,6.000404E-2,0E0,0E0,4.2599544E-2,3.9257012E-2,6.30681E-2,1.8668272E-2,4.906311E-2,5.533707E-2,1.2970728E-2,2.3380555E-2,0E0,0E0,0E0,0E0,3.5220888E-2,1.0222927E-2,3.087072E-2,1.9976046E-2,2.0381412E-2,0E0,0E0,0E0,1.7245183E-2,1.5136076E-2,0E0,0E0,2.189073E-2,2.3911959E-2,1.0170521E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,22,22,23,23,27,27,28,28,31,31,32,32,33,33],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,4.7162656E3,8.667E3,1E0,1.3277E4,5.0713454E-4,-7.7418964E-3,2.427E3,2.7577372E11,4.2E1,4.970186E7,1.3017557E7,8.40547E5,1.6391889E5,1.485E3,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,6.522946E5,6.5657372E0,2.7883E4,1.7657008E7,2.1692245E1,6.012406E-4,-6.2919264E-3,-7.461606E-4,1E0,2.85898E10,-5.1689288E-3,-1.9261596E-3,1.2670352E1,1.5324128E6,2.016E3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"split_indices":[46,53,2,84,10,0,0,2,32,3,52,46,30,29,0,0,0,0,0,49,55,9,33,57,0,0,0,69,32,0,0,55,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,2.12E2,6E0,1.86E2,2.6E1,2E0,4E0,1.59E2,2.7E1,4E0,2.2E1,5.2E1,1.07E2,1.9E1,8E0,2E0,2E0,6E0,1.6E1,4E1,1.2E1,7.5E1,3.2E1,9E0,1E1,5E0,3E0,1.2E1,2.8E1,5E0,7E0,6.2E1,1.3E1,2.5E1,7E0,5E0,4E0,1E1,2E0,2.3E1,5E0,5.5E1,7E0,3E0,1E1,1.4E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.3611307E-3,-9.2408955E-3,4.410103E-2,-6.9312286E-3,-7.04401E-3,1.3789503E-1,4.378376E-3,-2.0679065E-3,-5.6651037E-2,1.4739492E-3,1.1579285E-2,3.072128E-2,-8.0794595E-2,-4.1738063E-2,2.7708148E-3,-8.478865E-2,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,8.560338E-3,-6.760063E-2,-1.666495E-3,6.8934974E-3,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,-8.954448E-2,3.4993854E-3,4.3400084E-3,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,33,-1,35,-1,-1,-1,-1],"loss_changes":[6.6894464E-2,5.595534E-2,9.999141E-2,4.340747E-2,0E0,7.891193E-2,4.7356185E-2,3.178787E-2,3.738624E-2,0E0,0E0,2.251287E-2,2.4343904E-2,2.3993906E-2,2.2820415E-2,1.3125837E-2,0E0,0E0,0E0,0E0,0E0,1.6774707E-2,1.7332826E-2,0E0,2.1932926E-2,0E0,0E0,0E0,0E0,0E0,9.104654E-3,0E0,2.9089697E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,21,21,22,22,24,24,30,30,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,34,-1,36,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.4311E4,1.04E3,2.3361332E6,-7.04401E-3,7.65E2,3.093317E5,1.278E3,6.8125E0,1.4739492E-3,1.1579285E-2,5.8199444E0,9.532E3,2.8434807E3,1.0458333E1,1E0,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,1.7081589E2,8.137731E7,-1.666495E-3,2.2413793E0,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,2.1367E4,3.4993854E-3,5.3473053E0,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"split_indices":[53,10,0,29,0,0,34,2,54,0,0,54,9,48,59,84,0,0,0,0,0,4,5,0,57,0,0,0,0,0,9,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.81E2,2.6E1,1.79E2,2E0,7E0,1.9E1,1.64E2,1.5E1,4E0,3E0,1.5E1,4E0,1.7E1,1.47E2,1.1E1,4E0,1E1,5E0,2E0,2E0,6E0,1.1E1,1.4E1,1.33E2,4E0,7E0,2E0,4E0,3E0,8E0,4E0,1.29E2,6E0,2E0,7E0,1.22E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4610173E-3,9.7932376E-2,-4.4504614E-3,8.870057E-3,4.9648836E-4,-3.314924E-2,7.020033E-3,-2.3121499E-2,-1.00224335E-2,-5.438503E-3,5.8383252E-2,1.6109983E-2,-4.7826532E-2,7.8370846E-5,-3.111012E-3,7.136404E-3,4.1926984E-2,-1.6382972E-2,4.455269E-2,-6.4636454E-2,-4.9594564E-5,-5.14796E-2,5.3385887E-3,-4.3677186E-4,4.9276084E-2,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,-1.0062985E-1,-8.23799E-2,7.526664E-4,2.452092E-2,-9.486034E-3,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,-1,35,-1,-1,-1,-1,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.105056E-2,4.253041E-2,6.5741345E-2,0E0,0E0,9.4003044E-2,9.1824725E-2,5.4077312E-2,0E0,3.7433997E-2,3.4517996E-2,2.1062033E-2,2.668634E-2,2.929389E-2,0E0,0E0,9.794302E-3,1.1263341E-2,8.910602E-3,2.421417E-2,0E0,2.1960407E-2,2.8100949E-2,0E0,1.2502909E-2,0E0,0E0,0E0,0E0,0E0,1.0933511E-2,1.6032834E-2,0E0,1.9169066E-2,2.5302555E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,21,21,22,22,24,24,30,30,31,31,33,33,34,34],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,-1,36,-1,-1,-1,-1,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.813798E2,8.1356537E2,1E0,8.870057E-3,4.9648836E-4,4.4816768E8,2.7123457E3,2E0,-1.00224335E-2,4.462437E7,1.6064256E-2,6.822511E5,1.3088399E7,8.35E2,-3.111012E-3,7.136404E-3,5.4907064E0,8.901E3,5.1270317E-2,2.041E3,-4.9594564E-5,6.974669E7,1.6533424E6,-4.3677186E-4,7.317201E0,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,3.606E3,4.1749332E4,7.526664E-4,1.4653994E6,6.191606E0,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"split_indices":[34,52,102,0,0,45,4,8,0,46,58,33,1,2,0,0,55,2,39,0,0,7,51,0,54,0,0,0,0,0,10,33,0,51,54,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,5E0,1.98E2,2E0,3E0,5.6E1,1.42E2,5.4E1,2E0,1.15E2,2.7E1,2.1E1,3.3E1,1.06E2,9E0,3E0,2.4E1,1E1,1.1E1,2.4E1,9E0,9E0,9.7E1,3E0,2.1E1,8E0,2E0,4E0,7E0,1.4E1,1E1,6E0,3E0,4.2E1,5.5E1,1.8E1,3E0,3E0,7E0,3E0,3E0,3.7E1,5E0,3.7E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[4.282986E-4,4.1160197E-3,-9.1818266E-2,-1.1841053E-3,7.4616544E-2,-6.1210035E-3,3.2609148E-4,-1.09339515E-2,2.124421E-2,1.0531628E-1,7.8503106E-4,-7.0198895E-3,-9.130692E-2,-6.0027227E-4,4.235613E-2,6.323839E-3,6.2048313E-4,-3.122984E-2,2.2667213E-3,-1.7209452E-3,-6.5925727E-3,4.2787664E-2,-5.5588E-4,8.872538E-3,5.4689284E-2,3.1641412E-3,-7.729943E-2,-7.908227E-3,3.704432E-2,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,19,21,23,-1,-1,25,27,-1,-1,29,-1,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0795104E-2,7.4820675E-2,2.7605474E-2,4.112406E-2,2.3438096E-2,0E0,0E0,4.056964E-2,2.6312627E-2,1.8859334E-2,0E0,2.8399166E-2,1.00320205E-2,1.410548E-2,1.1490557E-2,0E0,0E0,5.5702664E-2,3.28481E-2,0E0,0E0,9.601074E-3,0E0,9.688978E-3,9.379521E-3,1.8371563E-2,2.2842146E-2,2.3135118E-2,2.1546472E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,20,22,24,-1,-1,26,28,-1,-1,30,-1,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,5.4349E4,6.45153E5,1.1565725E0,2.0216698E7,-6.1210035E-3,3.2609148E-4,9.964687E-1,7.6126127E0,2.3274304E2,7.8503106E-4,6.126268E0,9.31E2,7.614E3,6.314179E2,6.323839E-3,6.2048313E-4,8.351808E-1,1.0685499E0,-1.7209452E-3,-6.5925727E-3,1.3559702E8,-5.5588E-4,2.6164518E7,1.5107028E-1,1.424506E6,9.313E3,6.4745765E0,6.6352534E-1,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"split_indices":[46,2,29,43,48,0,0,40,55,57,0,55,0,10,4,0,0,28,58,0,0,7,0,33,39,30,2,55,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.99E2,7E0,1.86E2,1.3E1,5E0,2E0,1.3E2,5.6E1,8E0,5E0,1.25E2,5E0,2.8E1,2.8E1,6E0,2E0,3.4E1,9.1E1,3E0,2E0,5E0,2.3E1,8E0,2E1,2E1,1.4E1,7.1E1,2E1,3E0,2E0,6E0,2E0,2E0,1.8E1,1E1,1E1,8E0,6E0,3E0,6.8E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.1866337E-3,-1.4964924E-3,7.432874E-2,-7.1397156E-2,2.0264261E-3,1.4760831E-1,2.5388267E-2,2.4828428E-4,-5.377491E-3,-2.0973684E-2,1.2397231E-2,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,-8.560193E-2,-1.29591115E-2,2.8582712E-4,3.782403E-2,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,6.9721565E-2,-6.839452E-3,-3.537567E-3,4.555346E-2,5.6208028E-5,4.49603E-3,-2.1360328E-2,1.4701819E-2,6.0154725E-2,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.437666E-2,4.619424E-2,6.0806543E-2,2.7483527E-2,4.295682E-2,4.940191E-2,1.3439575E-2,0E0,0E0,2.7896874E-2,3.8175512E-2,0E0,0E0,0E0,0E0,1.4757402E-2,1.4061235E-2,4.253052E-2,3.747972E-2,0E0,0E0,0E0,0E0,1.2324721E-2,2.471229E-2,0E0,2.2664323E-2,0E0,0E0,2.4652388E-2,2.1451397E-2,1.7983623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,23,23,24,24,26,26,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,5.314E3,1.256E3,6.035227E9,1.05190264E8,5.3696685E0,9.532E3,2.4828428E-4,-5.377491E-3,1.2338027E5,8.1350375E5,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,1.4084E4,1.7669278E5,1.3448397E8,6.0599937E-6,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,2.8E1,1.1812092E0,-3.537567E-3,2.7991664E-1,5.6208028E-5,4.49603E-3,5.791575E-1,3.4E1,1.3839568E1,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"split_indices":[4,9,0,5,7,54,9,0,0,29,29,0,0,0,0,9,29,7,39,0,0,0,0,3,43,0,39,0,0,42,3,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.86E2,1.7E1,8E0,1.78E2,6E0,1.1E1,3E0,5E0,5.5E1,1.23E2,3E0,3E0,4E0,7E0,5E0,5E1,8.4E1,3.9E1,2E0,3E0,3E0,4.7E1,7E0,7.7E1,2E0,3.7E1,2E0,5E0,4.6E1,3.1E1,2.7E1,1E1,3E1,1.6E1,8E0,2.3E1,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4136843E-3,-1.7078152E-2,1.7051285E-2,-4.9061432E-2,-3.8076276E-3,2.0602716E-2,-4.516626E-3,-3.270243E-4,-7.544991E-2,6.5981923E-3,-3.537236E-2,-7.553134E-3,3.3188965E-2,-5.2146493E-3,-2.832102E-2,-1.7100578E-2,3.0886639E-2,1.3335078E-3,-4.885492E-2,4.1569836E-4,-4.885816E-2,2.7083127E-2,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,4.4731785E-2,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,4.3737285E-2,-3.549587E-3,6.7125736E-3,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6693766E-2,4.4803295E-2,3.7026227E-2,3.4130327E-2,2.5196122E-2,3.143581E-2,0E0,0E0,2.535192E-2,3.3913348E-2,1.7419232E-2,1.9031767E-2,2.084884E-2,0E0,1.2429329E-2,1.4870551E-2,2.5628617E-2,0E0,1.2944881E-2,0E0,1.3053814E-2,2.9234491E-2,0E0,0E0,0E0,0E0,0E0,2.8370071E-2,0E0,0E0,0E0,0E0,0E0,1.1751018E-2,1.2866924E-2,1.5719058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,27,27,33,33,34,34,35,35],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9640406E-1,5.511724E0,1.2083455E1,4.958582E-1,9.952575E5,3.225E3,-4.516626E-3,-3.270243E-4,1.0702152E-2,1.643614E6,5.916872E0,2.806E3,1.6974416E6,-5.2146493E-3,5.815166E0,1E0,5.2192265E1,1.3335078E-3,4.4589956E5,4.1569836E-4,1.8465776E8,8.062193E-1,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,1.01978E5,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,1.0252E4,1.0266666E1,7.1706E4,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"split_indices":[40,54,36,28,52,2,0,0,39,33,54,2,29,0,55,106,57,0,29,0,5,28,0,0,0,0,0,30,0,0,0,0,0,9,55,30,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.94E2,1.05E2,8.9E1,3E1,7.5E1,8.7E1,2E0,1.2E1,1.8E1,5.7E1,1.8E1,2.7E1,6E1,1E1,8E0,2.9E1,2.8E1,3E0,1.5E1,2E1,7E0,5.6E1,4E0,6E0,2E0,2.7E1,2E0,2.3E1,5E0,1.2E1,3E0,3E0,4E0,3.6E1,2E1,1.1E1,1.2E1,1E1,2.6E1,1.6E1,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.348882E-3,-1.4007735E-2,2.0349778E-2,-8.801855E-3,-7.60288E-2,3.1053293E-2,-1.0958469E-2,-5.7257973E-2,-5.0193043E-3,-4.4670764E-5,-4.9110665E-3,2.440679E-2,1.0895903E-1,-3.7627418E-2,2.2707816E-2,-4.087384E-3,3.10375E-4,3.1430066E-2,-1.0879586E-2,4.352828E-3,4.6011005E-2,6.7795296E-3,1.5659243E-3,-6.1382737E-2,-3.0169598E-4,4.8260357E-5,2.7864138E-3,6.0025465E-2,-8.0559475E-5,-1.663934E-2,1.3024335E-3,1.8343031E-2,-1.7778914E-2,-1.4250753E-4,6.0098227E-2,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,8.744232E-3,-2.3721121E-2,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,6.921383E-2,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,-1,41,43,-1,45,-1,-1,-1,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.254653E-2,3.32326E-2,3.619688E-2,1.76748E-2,1.4859837E-2,3.8764708E-2,2.5994083E-2,1.244238E-2,1.9993477E-2,0E0,0E0,3.2293484E-2,9.498283E-3,1.1564847E-2,9.696336E-3,0E0,0E0,1.2398537E-2,1.7790027E-2,1.2711223E-2,2.4995036E-2,0E0,0E0,1.1450622E-2,0E0,0E0,0E0,1.1509789E-2,0E0,1.27906855E-2,0E0,9.788271E-3,8.809424E-3,0E0,1.9199878E-2,0E0,0E0,0E0,0E0,8.899177E-3,1.8784469E-2,0E0,0E0,0E0,0E0,1.800523E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,27,27,29,29,31,31,32,32,34,34,39,39,40,40,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,-1,42,44,-1,46,-1,-1,-1,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,3.6509445E1,1.3983983E10,1.1475E2,2.1434378E5,5.2086E4,1.5461028E12,8.842206E7,4.1740056E7,-4.4670764E-5,-4.9110665E-3,4.9754506E5,1.8136424E7,1.3088399E7,6.1921E4,-4.087384E-3,3.10375E-4,4.327473E5,2.6493E4,5.0123274E-1,1E0,6.7795296E-3,1.5659243E-3,1.775894E10,-3.0169598E-4,4.8260357E-5,2.7864138E-3,1.0380544E8,-8.0559475E-5,1.43E2,1.3024335E-3,1.1201525E2,2.513265E5,-1.4250753E-4,1.3578947E1,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,6.1579504E2,2.109E3,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,2.1525126E0,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"split_indices":[53,57,5,53,29,2,32,5,5,0,0,29,1,1,10,0,0,29,10,28,17,0,0,5,0,0,0,7,0,10,0,57,34,0,55,0,0,0,0,4,2,0,0,0,0,40,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.04E2,1.06E2,9.7E1,7E0,7.9E1,2.7E1,6E0,9.1E1,2E0,5E0,7.4E1,5E0,1.5E1,1.2E1,4E0,2E0,1.2E1,7.9E1,3.9E1,3.5E1,3E0,2E0,8E0,7E0,8E0,4E0,6E0,6E0,6.9E1,1E1,2.4E1,1.5E1,8E0,2.7E1,2E0,6E0,3E0,3E0,1.5E1,5.4E1,1.9E1,5E0,1.2E1,3E0,2.4E1,3E0,8E0,7E0,1.8E1,3.6E1,1.6E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.8318873E-3,-1.0389051E-2,6.847433E-2,-9.090252E-2,-7.818896E-3,1.4557077E-1,1.5369026E-2,-1.1227432E-3,-5.793796E-3,-3.6059227E-3,-5.7291586E-2,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,-1.0661409E-2,3.3012316E-2,-1.4323252E-3,-7.734146E-3,-6.9573727E-3,-4.916107E-2,4.209554E-3,2.0994669E-2,-5.233113E-2,-3.3860938E-3,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.4740467E-2,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[8.666744E-2,4.0054876E-2,5.917617E-2,8.709792E-3,3.9811417E-2,4.4877455E-2,9.926351E-3,0E0,0E0,4.6339944E-2,3.7794035E-2,0E0,0E0,0E0,0E0,2.1048589E-2,1.682442E-2,0E0,0E0,2.2184646E-2,1.6582798E-2,0E0,2.2704527E-2,1.6044017E-2,2.14594E-2,0E0,0E0,0E0,9.11136E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,19,19,20,20,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,2.3125E0,2E0,5.34E2,7.8032995E6,6.677E3,6.881048E-1,-1.1227432E-3,-5.793796E-3,1.6503105E3,5.8199444E0,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,9.5096946E-1,6.0775385E0,-1.4323252E-3,-7.734146E-3,1.2425E2,6.128552E0,4.209554E-3,5.590909E0,7.005162E5,1.2804077E7,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.842571E7,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"split_indices":[4,54,8,0,48,9,35,0,0,53,54,0,0,0,0,28,55,0,0,53,54,0,54,29,46,0,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.96E2,1.4E1,5E0,1.91E2,5E0,9E0,2E0,3E0,1.77E2,1.4E1,2E0,3E0,7E0,2E0,1.49E2,2.8E1,1.2E1,2E0,1.37E2,1.2E1,4E0,2.4E1,9E0,1.28E2,9E0,3E0,4E0,2E1,5E0,4E0,7.1E1,5.7E1,1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6622468E-4,-4.8488043E-3,5.08311E-2,-7.834397E-4,-6.2016364E-2,7.5922403E-3,1.9395813E-2,2.0835144E-3,-6.411687E-2,-4.91167E-3,-1.6307954E-3,4.5530796E-2,-4.4596207E-4,-8.775512E-3,2.092946E-2,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,-4.312229E-3,-4.002928E-2,-6.387113E-4,3.425055E-2,-6.662406E-4,-2.56261E-3,-3.120549E-3,1.352972E-4,4.647988E-2,-1.3092252E-3,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1110663E-2,4.5503184E-2,5.6485604E-2,3.354321E-2,1.1037149E-2,0E0,1.1613244E-2,3.640647E-2,3.848807E-2,0E0,0E0,9.618994E-3,0E0,1.5590253E-2,2.9935649E-2,0E0,0E0,0E0,0E0,1.7591223E-2,1.4617277E-2,0E0,2.0499848E-2,1.4878065E-2,0E0,0E0,0E0,2.9250316E-2,9.96951E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,22,22,23,23,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.8600049E10,1.4164306E-3,7.8032995E6,2.884462E0,7.5922403E-3,5.861021E7,6.978001E2,7E0,-4.91167E-3,-1.6307954E-3,2.819228E-1,-4.4596207E-4,1E0,2.8626094E5,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,3.6509445E1,4.966E3,-6.387113E-4,5.2192265E1,4.454691E5,-2.56261E-3,-3.120549E-3,1.352972E-4,9.42191E-1,2.6673E4,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"split_indices":[53,5,58,48,54,0,46,53,8,0,0,58,0,84,29,0,0,0,0,57,2,0,57,49,0,0,0,28,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.95E2,1.7E1,1.83E2,1.2E1,3E0,1.4E1,1.76E2,7E0,4E0,8E0,7E0,7E0,1.12E2,6.4E1,5E0,2E0,2E0,5E0,9.9E1,1.3E1,1.8E1,4.6E1,9.3E1,6E0,8E0,5E0,3.4E1,1.2E1,2.7E1,6.6E1,2.9E1,5E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.3941145E-4,-2.0441902E-3,8.919721E-3,-1.1523694E-2,2.1440972E-2,-9.1706775E-3,-5.6796446E-3,5.424757E-3,1.2543296E-2,-1.505701E-2,4.4461366E-2,1.9426743E-2,-3.729582E-3,-7.865752E-3,-3.6841E-2,3.1923247E-4,3.1530529E-3,-4.811276E-3,5.0493825E-2,-1.1715324E-2,1.7492641E-3,-1.6764663E-2,-3.0829909E-3,-2.7253285E-2,1.2633276E-2,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0149213E-1,4.410473E-2,0E0,3.3826478E-2,4.4260666E-2,4.4551205E-2,0E0,0E0,3.4023255E-2,1.944866E-2,1.0361386E-2,3.789317E-2,0E0,1.6611358E-2,1.5435919E-2,0E0,0E0,1.1697531E-2,2.4396595E-2,1.604245E-2,0E0,1.6484369E-2,0E0,1.106496E-2,1.5908504E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,23,23,24,24],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9028345E2,8.1350375E5,8.919721E-3,2.61E2,5.0655737E0,3.2432E4,-5.6796446E-3,5.424757E-3,7.251039E0,1.8747724E7,1.0252E4,7.6188426E0,-3.729582E-3,2.513265E5,6.995449E-1,3.1923247E-4,3.1530529E-3,1.2153278E-3,9.161318E0,4.8E1,1.7492641E-3,4.588941E-1,-3.0829909E-3,6.809655E0,2.0350546E-3,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"split_indices":[57,29,0,3,55,2,0,0,58,33,9,55,0,34,28,0,0,40,55,10,0,39,0,55,43,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.96E2,2E0,1.4E2,5.6E1,1.38E2,2E0,4E0,5.2E1,1.25E2,1.3E1,4.9E1,3E0,9.5E1,3E1,5E0,8E0,2.8E1,2.1E1,8.8E1,7E0,1.8E1,1.2E1,1.2E1,1.6E1,7E0,1.4E1,7E0,8.1E1,1.5E1,3E0,1E1,2E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.881544E-3,-6.655184E-3,2.188296E-2,-2.8302656E-3,-2.666419E-3,-2.7059872E-2,3.2757495E-2,-9.625506E-3,2.9928401E-2,-5.4137424E-2,7.742816E-7,2.5661085E-2,7.91566E-2,1.4555868E-3,-2.7334698E-2,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,3.091641E-2,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,-5.2867554E-2,1.0829677E-2,-4.0220562E-2,6.7792195E-3,6.600478E-2,1.1288791E-3,-7.709266E-4,-4.6325806E-3,-1.4827026E-2,2.3715133E-2,-2.952948E-4,-5.428135E-2,2.0235393E-3,-9.2693866E-4,8.701906E-2,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,35,37,-1,-1,-1,39,41,-1,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3877617E-2,2.2490527E-2,5.198053E-2,2.5887422E-2,0E0,1.3197487E-2,2.3980856E-2,1.8878557E-2,3.1391144E-2,9.128876E-3,0E0,1.9829843E-2,2.8112642E-2,3.112634E-2,1.6537398E-2,0E0,0E0,0E0,0E0,1.5805755E-2,0E0,0E0,0E0,1.2595711E-2,1.7542738E-2,1.2636464E-2,1.0678305E-2,1.2879096E-2,0E0,0E0,0E0,9.937875E-3,1.8333944E-2,0E0,1.3910651E-2,0E0,0E0,1.5306905E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,19,19,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,37,37],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,36,38,-1,-1,-1,40,42,-1,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2936785E3,3.0978662E6,3.673257E0,1.0907971E6,-2.666419E-3,4.8601307E-2,5.4349E4,2.2906403E-1,1E0,3.26E2,7.742816E-7,1.1778515E7,7.65E2,9.7043E4,4.4589956E5,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,7.573E3,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,4.0289572E2,3.1E1,1.0485419E8,1.2857332E6,1.1865996E1,1.1288791E-3,-7.709266E-4,-4.6325806E-3,1.5403727E-1,1.9540394E7,-2.952948E-4,1.6680729E0,2.0235393E-3,-9.2693866E-4,1.884E3,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"split_indices":[4,29,54,29,0,39,2,58,17,0,0,48,0,1,29,0,0,0,0,9,0,0,0,4,3,7,49,55,0,0,0,58,46,0,40,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.22E2,9.5E1,1.14E2,8E0,1.7E1,7.8E1,9.5E1,1.9E1,8E0,9E0,6.9E1,9E0,5.9E1,3.6E1,1E1,9E0,2E0,6E0,6.3E1,6E0,2E0,7E0,8E0,5.1E1,2.6E1,1E1,1E1,5.3E1,5E0,3E0,1.7E1,3.4E1,8E0,1.8E1,4E0,6E0,7E0,3E0,1.5E1,2E0,2.8E1,6E0,1.3E1,5E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.403546E-3,-9.570334E-4,-4.4911288E-2,-7.611177E-2,1.5534413E-3,-2.1605039E-2,-8.161089E-3,-5.0543556E-3,-6.690376E-4,-5.41183E-3,1.55741805E-2,-2.0407424E-3,4.387262E-4,-1.5254784E-3,-6.8485916E-2,3.5775187E-3,7.775442E-3,5.1245918E-3,-2.5670715E-2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,1.9513896E-2,2.5408894E-2,-7.420587E-3,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4000924E-2,3.5022937E-2,7.077891E-2,8.954152E-3,1.7662771E-2,1.4694375E-2,0E0,0E0,0E0,2.955602E-2,2.6538467E-2,0E0,0E0,1.8596815E-2,1.0216929E-2,0E0,1.7291361E-2,2.3345338E-2,1.594018E-2,0E0,0E0,0E0,8.91481E-3,1.4992485E-2,2.6731981E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,13,13,14,14,16,16,17,17,18,18,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.361366E7,1.0135136E0,7.5656E4,1.38E2,1.500621E3,3.0687964E3,-8.161089E-3,-5.0543556E-3,-6.690376E-4,1.2473612E7,4.965E3,-2.0407424E-3,4.387262E-4,2.1067844E7,1E0,3.5775187E-3,5.1034484E0,1.2623029E7,8.103274E2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,9.43017E5,2.1768606E5,1.1565725E0,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"split_indices":[46,57,10,0,4,53,0,0,0,51,2,0,0,46,8,0,54,12,4,0,0,0,49,29,43,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.84E2,2.5E1,5E0,1.79E2,2.2E1,3E0,3E0,2E0,1.2E2,5.9E1,1.3E1,9E0,1.14E2,6E0,6E0,5.3E1,9E1,2.4E1,2E0,4E0,1.6E1,3.7E1,3.4E1,5.6E1,7E0,1.7E1,6E0,3.1E1,7E0,2.7E1,2.8E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.779959E-6,1.8453373E-3,-5.9890207E-3,-1.285875E-3,6.8091586E-2,-5.2687E-3,3.303071E-2,8.70132E-3,5.314312E-4,-5.49993E-2,-3.4504228E-3,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,-8.831604E-3,1.3855618E-2,-1.8325018E-2,8.489807E-3,-3.5646225E-3,4.3126993E-2,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.611087E-2,4.1366342E-2,0E0,2.6279328E-2,5.5009544E-2,1.53631605E-2,2.091176E-2,0E0,0E0,1.1431506E-2,1.5674567E-2,0E0,0E0,0E0,0E0,2.1203808E-2,2.0550307E-2,1.3397509E-2,1.3573993E-2,1.2377562E-2,1.1399828E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6085916E7,2.0494681E2,-5.9890207E-3,1E0,9.900047E6,2.3125E0,1.04E3,8.70132E-3,5.314312E-4,3.26E2,2.6673E4,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,3.9E1,5.8167644E0,5.540166E-3,7.31E3,3.7988422E5,2.3E1,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"split_indices":[30,57,0,74,1,54,0,0,0,0,10,0,0,0,0,3,54,58,9,29,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.98E2,2E0,1.9E2,8E0,1.71E2,1.9E1,2E0,6E0,5E0,1.66E2,1.3E1,6E0,2E0,3E0,1.27E2,3.9E1,8.2E1,4.5E1,2.5E1,1.4E1,2.1E1,6.1E1,1.2E1,3.3E1,1E1,1.5E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.3686826E-4,-3.519467E-3,5.055597E-2,-2.117268E-3,-4.640776E-3,7.0017064E-3,1.523785E-2,-4.871748E-2,3.465182E-4,-9.892045E-4,4.3544497E-2,6.1831385E-4,-3.1955007E-3,-5.2515082E-2,2.6555143E-3,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,-1.2239224E-3,2.5115369E-2,9.1643715E-3,-1.2135973E-2,5.6670893E-2,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,15,-1,-1,17,19,-1,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0816244E-2,2.5682297E-2,4.724356E-2,2.2864658E-2,0E0,0E0,1.2965487E-2,1.1241557E-2,2.333033E-2,0E0,8.747165E-3,0E0,0E0,2.0471422E-2,1.5975818E-2,0E0,0E0,0E0,0E0,1.790836E-2,3.341707E-2,2.1425955E-2,2.8729388E-2,1.4656175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,16,-1,-1,18,20,-1,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,6.7948157E-1,1.0038867E4,1.2425E2,-4.640776E-3,7.0017064E-3,6.881048E-1,3.7790768E2,2.3125E0,-9.892045E-4,2.7445236E-1,6.1831385E-4,-3.1955007E-3,8.591E3,4.4E1,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,3.3E1,8.87E2,1.127938E6,4.570007E7,1.7E1,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"split_indices":[4,35,4,53,0,0,35,34,54,0,39,0,0,2,3,0,0,0,0,3,0,29,5,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,2E2,1.4E1,1.98E2,2E0,3E0,1.1E1,9E0,1.89E2,5E0,6E0,2E0,7E0,7E0,1.82E2,3E0,3E0,3E0,4E0,1.56E2,2.6E1,8E1,7.6E1,1.4E1,1.2E1,5.3E1,2.7E1,7E0,6.9E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.8720286E-3,2.203888E-3,-3.8517896E-2,7.5318073E-3,-6.002819E-4,-2.1684619E-2,-8.622627E-3,1.0079312E-3,-4.2458405E-3,-2.1083932E-3,1.5030096E-3,-9.4883353E-4,3.8221332E-3,2.3723133E-3,-7.695643E-4,2.9969336E-3,-3.9027993E-2,-1.1395355E-2,1.3584136E-2,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,13,15,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0922936E-2,7.110839E-2,6.3847005E-2,0E0,2.2939872E-2,1.3612385E-2,0E0,2.4651024E-2,0E0,0E0,1.2223788E-2,2.4006953E-2,0E0,0E0,0E0,2.2249037E-2,1.5829908E-2,9.478996E-3,2.061506E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,10,10,11,11,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,14,16,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,3.7902024E0,7.5656E4,7.5318073E-3,1.3105492E1,4.6E1,-8.622627E-3,5.9165E4,-4.2458405E-3,-2.1083932E-3,1E0,2.72258E5,3.8221332E-3,2.3723133E-3,-7.695643E-4,7.6077003E0,5.1737704E7,3.1347964E-2,2.6673E4,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"split_indices":[46,55,10,0,36,3,0,2,0,0,8,9,0,0,0,55,33,58,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.65E2,2.8E1,2E0,1.63E2,2.6E1,2E0,1.61E2,2E0,1.3E1,1.3E1,1.58E2,3E0,3E0,1E1,1.44E2,1.4E1,6.1E1,8.3E1,1.1E1,3E0,2.2E1,3.9E1,6.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-6.800024E-4,1.8960047E-3,-6.0736863E-3,-3.6225263E-2,6.760957E-3,-4.00987E-3,-1.3761937E-2,4.2979596E-3,5.311085E-3,1.561032E-3,-1.3046184E-3,-4.1869044E-2,6.551281E-3,-3.4434297E-3,1.4532126E-3,2.6219185E-3,3.280929E-2,-3.7609767E-3,2.3818783E-2,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,11,-1,-1,-1,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.290568E-2,3.6038343E-2,0E0,2.1682842E-2,4.2899136E-2,0E0,1.0427506E-2,1.7855216E-2,0E0,0E0,0E0,1.8689083E-2,1.662869E-2,0E0,0E0,1.9301808E-2,1.2423562E-2,2.420176E-2,2.810772E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,11,11,12,12,15,15,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,-1,10,12,-1,-1,-1,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.02E2,3.8834286E0,-6.0736863E-3,2.3443186E3,5.2086E4,-4.00987E-3,1.2633E4,1.8E2,5.311085E-3,1.561032E-3,-1.3046184E-3,1.3448806E8,4.6E1,-3.4434297E-3,1.4532126E-3,1.1482285E6,1.10597E5,8.351808E-1,3.1E1,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"split_indices":[8,54,0,4,2,0,2,11,0,0,0,7,3,0,0,29,30,28,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.92E2,3E0,2.1E1,1.71E2,6E0,1.5E1,1.68E2,3E0,3E0,1.2E1,7E0,1.61E2,5E0,2E0,1.41E2,2E1,1.09E2,3.2E1,3E0,1.7E1,8.7E1,2.2E1,2.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.1066506E-4,4.845112E-3,-3.1581253E-2,6.452437E-2,-6.428997E-4,-5.0642762E-2,5.648247E-4,1.1988164E-1,-7.721919E-4,-4.0026516E-2,3.5905282E-3,-2.9436452E-2,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,-2.7368726E-2,7.2603608E-3,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,1.2873769E-2,-1.3507972E-2,3.7806993E-3,1.0231802E-2,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1],"loss_changes":[3.406493E-2,5.8340125E-2,2.528421E-2,6.86559E-2,2.7487058E-2,4.734615E-2,0E0,2.8751194E-2,0E0,2.1383194E-2,1.7127382E-2,1.0340074E-2,0E0,0E0,0E0,0E0,0E0,2.1022052E-2,1.5795767E-2,0E0,0E0,0E0,0E0,1.7355684E-2,1.4121132E-2,0E0,1.356209E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,17,17,18,18,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1],"split_conditions":[1.8000048E10,6.677E3,3.244E3,5.1E1,1.142E3,3.08E3,5.648247E-4,5.8229775E3,-7.721919E-4,5.244755E-3,1.13002876E-4,7.520736E6,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,8.3949E4,7.384892E0,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,3.0404909E0,3.501498E7,3.7806993E-3,4.3E1,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"split_indices":[5,9,0,3,2,0,0,4,0,58,39,30,0,0,0,0,0,12,54,0,0,0,0,57,33,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.77E2,2.9E1,1.4E1,1.63E2,2E1,9E0,8E0,6E0,1.5E1,1.48E2,1.8E1,2E0,6E0,2E0,8E0,7E0,1.5E1,1.33E2,1.6E1,2E0,8E0,7E0,1.05E2,2.8E1,3E0,1.02E2,2.5E1,3E0,7.8E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-2.5543855E-3,-3.670569E-4,-9.478624E-2,-4.503718E-3,3.9570075E-2,-2.891256E-4,-7.399056E-3,7.92847E-3,-1.9427676E-2,1.2930362E-1,9.055213E-3,-2.102676E-2,1.859223E-2,-4.328215E-3,-3.512878E-2,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,6.881532E-3,-3.8464133E-2,3.4366657E-3,1.07829925E-2,-2.6562288E-3,8.551802E-5,-1.3136177E-2,-5.552462E-2,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,-1.9891918E-2,1.758985E-2,-5.6563243E-2,1.090067E-2,-8.1699066E-2,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,-1,-1,-1,-1,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3541346E-2,3.5201967E-2,2.452866E-2,3.5962764E-2,5.359327E-2,0E0,0E0,3.312072E-2,2.0710837E-2,1.6731478E-2,1.720418E-2,1.4377463E-2,3.0927734E-2,1.4162992E-2,1.8558595E-2,0E0,0E0,0E0,0E0,9.007291E-3,9.359658E-3,0E0,1.4757078E-2,0E0,0E0,2.358127E-2,2.1488503E-2,0E0,0E0,0E0,0E0,1.4124965E-2,1.671442E-2,8.771379E-3,1.1919808E-2,1.0015354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,25,25,26,26,31,31,32,32,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,-1,-1,-1,-1,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.25E2,5.2062725E3,2.133183E1,1.6278354E7,1.1823068E7,-2.891256E-4,-7.399056E-3,1.549E3,8.062193E-1,6.23E2,2.0090337E0,4.0431567E-3,1.26747945E5,1.775044E7,7.573E3,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,7.107292E6,8E-3,3.4366657E-3,3.104892E4,-2.6562288E-3,8.551802E-5,3.7614656E5,1.2067E4,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,4.9973932E-1,5.1821535E4,2.849513E3,9.8613055E4,9.377185E-1,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"split_indices":[8,53,55,46,1,0,0,2,28,0,35,43,48,46,9,0,0,0,0,46,58,0,34,0,0,29,9,0,0,0,0,28,34,4,34,28,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.11E2,4E0,1.92E2,1.9E1,2E0,2E0,1.05E2,8.7E1,4E0,1.5E1,2.8E1,7.7E1,4.5E1,4.2E1,2E0,2E0,1.3E1,2E0,1.1E1,1.7E1,9E0,6.8E1,4E0,4.1E1,2.1E1,2.1E1,5E0,6E0,4E0,1.3E1,1.2E1,5.6E1,7E0,1.4E1,1.2E1,9E0,9E0,3E0,1.2E1,4.4E1,4E0,3E0,1.1E1,3E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.7936366E-3,2.4298213E-3,-4.112654E-2,-4.0990007E-3,4.8309796E-2,-2.183913E-2,-8.530635E-3,2.3222237E-3,-3.93802E-2,1.3060242E-1,8.8548794E-4,-4.1647155E-2,3.9303087E-4,-8.698616E-3,2.1892874E-2,-5.8111317E-2,-5.4723965E-3,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,-4.9995827E-3,-3.3989844E-3,3.6737766E-2,-1.1948632E-2,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,-4.3598227E-2,7.2821265E-4,4.7861084E-2,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1747835E-2,5.5313785E-2,6.086246E-2,3.6792308E-2,5.4622788E-2,1.3965219E-2,0E0,2.9954968E-2,1.5560266E-2,4.6919495E-2,0E0,1.526756E-2,0E0,1.9780226E-2,2.5557641E-2,8.726589E-3,9.571268E-3,0E0,0E0,0E0,0E0,1.8823985E-2,0E0,1.4614824E-2,9.260894E-3,0E0,0E0,0E0,0E0,1.8716276E-2,2.0947833E-2,9.567626E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,21,21,23,23,24,24,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3849624E1,9.276886E1,1.019731E0,4.9386452E7,1.6064256E-2,1.7810251E1,-8.530635E-3,8.4182155E10,4.4E1,7.257E3,8.8548794E-4,2.6239506E5,3.9303087E-4,4.991491E8,1.7486866E7,8.3764136E2,9.39616E-1,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,6.974669E7,-3.3989844E-3,4.493248E6,2.6678162E9,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,1.0363E4,7.25E2,1.7747324E7,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"split_indices":[55,57,35,46,58,57,0,32,3,2,0,29,0,7,33,53,28,0,0,0,0,7,0,1,12,0,0,0,0,9,0,46,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.83E2,2.4E1,1.61E2,2.2E1,2.2E1,2E0,1.37E2,2.4E1,5E0,1.7E1,1.3E1,9E0,8.8E1,4.9E1,1.5E1,9E0,2E0,3E0,4E0,9E0,8.4E1,4E0,3.4E1,1.5E1,6E0,9E0,6E0,3E0,1E1,7.4E1,2.6E1,8E0,7E0,8E0,6E0,4E0,5E1,2.4E1,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.3284292E-3,-3.6234653E-3,6.495424E-2,-3.3305318E-4,-4.9157396E-2,9.216801E-3,3.1242464E-2,-1.132699E-2,1.368838E-2,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,-1.8146961E-3,-3.4096006E-2,5.2584518E-2,5.3565246E-3,-6.317367E-3,3.768521E-3,-8.287692E-2,-1.5646296E-2,4.7426983E-3,3.285665E-4,-1.2222636E-3,1.7330685E-3,-9.491318E-3,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,4.4616484E-3,-1.594493E-2,1.9072331E-2,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,27,29,-1,-1,31,-1,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5454505E-2,2.8899927E-2,5.700592E-2,2.8056335E-2,2.499102E-2,0E0,9.989295E-3,2.2029854E-2,2.564449E-2,0E0,0E0,0E0,0E0,2.65384E-2,2.5703099E-2,2.8132629E-2,1.3379129E-2,1.2622242E-2,0E0,1.1075061E-2,1.793515E-2,0E0,0E0,1.7036377E-2,0E0,1.3837478E-2,0E0,0E0,0E0,0E0,8.933199E-3,9.179806E-3,2.1313196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,28,30,-1,-1,32,-1,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.8032995E6,1.4164306E-3,4.7132755E6,5.7592998E0,9.216801E-3,6.5772717E8,2.1557302E6,6.505428E6,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,2.0041434E5,5.9083433E0,7.56E2,9.855582E5,1E0,3.768521E-3,1.1485E4,2.831E3,4.7426983E-3,3.285665E-4,3.6E1,1.7330685E-3,4.43E2,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,2.8037289E1,7.894435E-1,1.1156E4,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"split_indices":[4,48,58,33,54,0,33,33,33,0,0,0,0,34,54,0,29,106,0,9,2,0,0,3,0,0,0,0,0,0,59,28,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.92E2,1.4E1,1.8E2,1.2E1,2E0,1.2E1,1.01E2,7.9E1,1E1,2E0,1E1,2E0,7.2E1,2.9E1,1.3E1,6.6E1,6.9E1,3E0,7E0,2.2E1,6E0,7E0,5.5E1,1.1E1,6.6E1,3E0,3E0,4E0,7E0,1.5E1,3.2E1,2.3E1,3.1E1,3.5E1,3E0,1.2E1,2.1E1,1.1E1,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.613512E-4,9.168064E-3,-2.0914895E-2,2.484408E-3,9.1796964E-2,-1.50884595E-2,-5.1584444E-3,-3.610229E-2,6.524373E-3,2.7067963E-2,8.863301E-3,-3.4663703E-2,1.7349996E-2,-6.1991275E-4,-4.9615046E-3,2.1708814E-2,-6.561107E-3,2.5338226E-3,-1.149334E-3,-2.8247127E-2,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,3.0649362E-2,2.0936963E-3,-1.2641752E-2,-4.455137E-2,1.7206483E-3,3.736369E-3,4.129001E-2,-4.1820602E-3,-8.335901E-3,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,23,25,-1,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.960969E-2,7.377521E-2,3.0622425E-2,1.9721659E-2,5.3537726E-2,3.9970838E-2,0E0,1.7162973E-2,2.2793744E-2,1.0098371E-2,0E0,1.4024261E-2,1.5488645E-2,0E0,0E0,1.6034028E-2,1.923645E-2,0E0,0E0,1.8154921E-2,0E0,0E0,0E0,0E0,1.1873342E-2,0E0,1.6956246E-2,9.147577E-3,1.221883E-2,1.0724097E-2,1.0640074E-2,0E0,1.465915E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,19,19,24,24,26,26,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,24,26,-1,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7864855E11,1.3173E4,1.4146589E8,7.5999E4,5.4E9,1.0807868E7,-5.1584444E-3,5.6978E4,8.3333336E-2,7.438459E6,8.863301E-3,1.226189E0,3.093317E5,-6.1991275E-4,-4.9615046E-3,2.7E1,7.720203E4,2.5338226E-3,-1.149334E-3,8.305745E-1,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,5.2682075E5,2.0936963E-3,9.484755E4,7.4074075E-2,3.7791205E6,3.63E2,7.501344E0,-4.1820602E-3,3.6360288E8,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"split_indices":[32,2,46,1,5,1,0,1,58,1,0,40,34,0,0,3,29,0,0,58,0,0,0,0,33,0,29,58,48,0,54,0,5,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.33E2,6.4E1,1.24E2,9E0,6.1E1,3E0,1.1E1,1.13E2,6E0,3E0,3.8E1,2.3E1,9E0,2E0,5.2E1,6.1E1,4E0,2E0,3.6E1,2E0,1.9E1,4E0,1.1E1,4.1E1,6E0,5.5E1,2.3E1,1.3E1,1.2E1,2.9E1,2E0,5.3E1,1.9E1,4E0,6E0,7E0,9E0,3E0,2.3E1,6E0,1.6E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"3.6910433E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[1.5103106E-2,-2.2944748E-1,4.0485805E-1,-5.2396727E-1,-5.360323E-2,7.115805E-1,-2.6379624E-1,-3.949695E-1,-4.2585783E-2,-1.771069E-1,8.511205E-2,4.611889E-1,1.0766281E0,2.0126725E-2,-6.011713E-1,-4.39699E-1,2.4540755E-3,-2.5400817E-1,-1.0922299E-1,2.1650563E-1,-2.3319207E-1,-1.3541527E-2,5.946551E-1,5.5333883E-2,1.7301071E-2,-1.22435205E-1,1.9814072E-2,-4.192214E-2,-1.1715452E-2,-2.594052E-1,-5.554998E-1,-3.1053467E-4,-3.0310825E-1,8.845983E-2,-1.6904932E-1,2.8078866E-1,6.388342E-2,-1.6651804E-2,-3.3995867E-2,7.99645E-3,-1.5311554E-1,6.501042E-1,6.8630027E-3,-1.1348052E-2,5.7094876E-2,-4.0573996E-2,-3.4288028E-1,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,-3.68585E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,-1.9150509E-1,7.322452E-3,1.7035466E-2,-6.4032795E-3,1.3852504E-1,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,45,47,-1,49,51,53,55,57,-1,59,-1,61,63,-1,-1,65,67,69,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8298021E1,6.0830874E0,1.536447E1,1.6471167E0,1.2999624E0,4.1651173E0,2.3110242E0,6.9953203E-1,0E0,1.810708E-1,1.5561198E0,2.0355868E0,4.665451E-1,8.22276E-1,8.244722E-1,5.031233E-1,0E0,2.1691906E-1,2.8696674E-1,2.3714864E-1,2.271381E-1,2.2374773E-1,5.371609E-1,0E0,0E0,2.3201612E-1,0E0,0E0,0E0,2.418254E-1,3.252077E-2,0E0,1.0391843E-1,5.7198662E-2,4.2966306E-2,1.09799504E-1,1.495303E-1,0E0,2.0135637E-2,0E0,2.2688761E-2,2.5058937E-1,0E0,0E0,6.3281864E-2,1.3045962E-2,8.805108E-2,0E0,0E0,0E0,2.4855494E-2,0E0,0E0,0E0,2.7623773E-2,0E0,0E0,0E0,6.379056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,34,34,35,35,36,36,38,38,40,40,41,41,44,44,45,45,46,46,50,50,54,54,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,46,48,-1,50,52,54,56,58,-1,60,-1,62,64,-1,-1,66,68,70,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.039435E-3,6.627576E7,6.8652206E9,4.5570032E2,4.7162656E3,4E0,4.1211517E1,-4.2585783E-2,1.1383354E6,9.518453E6,1E0,7.317201E0,1.5312085E0,7.588906E-2,5.1147053E5,2.4540755E-3,4.7812922E5,1.630886E6,8.729897E0,1.2534044E3,8.78E2,2.709E4,5.5333883E-2,1.7301071E-2,6.7995167E0,1.9814072E-2,-4.192214E-2,-1.1715452E-2,2.77E2,1E0,-3.1053467E-4,2.8880866E0,4.35E2,2.6374866E1,5.716463E2,2.831E3,-1.6651804E-2,9.17E2,7.99645E-3,3.349E4,1.5145266E6,6.8630027E-3,-1.1348052E-2,6.695E3,9.513E3,1.4377E4,-2.889748E-2,-1.3288353E-2,-6.9817984E-3,1.0491824E-1,9.059357E-3,-3.3698004E-4,-2.0721143E-3,4.2658337E2,7.322452E-3,1.7035466E-2,-6.4032795E-3,5.586E3,1.5071915E-3,-4.2646215E-3,-2.1270025E-3,-1.0292312E-2,1.6440231E-2,3.5141196E-2,-2.7886136E-3,7.419621E-3,-4.0311287E-3,7.4014184E-4,-9.255774E-3,-2.22666E-2,-7.0775696E-3,-1.9768063E-2,-1.0192754E-2,-2.4307563E-3,9.682183E-3,-4.067571E-4],"split_indices":[52,38,45,5,52,52,8,58,0,51,45,109,53,57,38,28,0,51,45,54,4,0,9,0,0,54,0,0,0,0,112,0,56,0,58,52,2,0,0,0,10,48,0,0,2,9,9,0,0,0,39,0,0,0,52,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.17E2,7.3E1,4.3E1,7.4E1,5E1,2.3E1,3.3E1,1E1,3.9E1,3.5E1,3.1E1,1.9E1,1.3E1,1E1,3E1,3E0,1.7E1,2.2E1,2.5E1,1E1,7E0,2.4E1,1.7E1,2E0,1E1,3E0,5E0,5E0,1.3E1,1.7E1,3E0,1.4E1,5E0,1.7E1,1.7E1,8E0,6E0,4E0,3E0,4E0,2.1E1,3E0,6E0,4E0,4E0,9E0,1.4E1,3E0,5E0,9E0,2E0,3E0,3E0,1.4E1,7E0,1E1,2E0,6E0,2E0,2E0,2E0,2E0,5E0,1.6E1,2E0,2E0,2E0,2E0,5E0,4E0,2E0,7E0,1.2E1,2E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-3.3002432E-2,-1.7783551E-1,5.082772E-1,-2.7808794E-1,6.895006E-2,7.5295E-2,8.787938E-1,-1.8593931E-1,-9.442671E-1,3.6505783E-1,-2.5721067E-1,3.1757823E-1,-1.1269199E-1,4.4750738E-1,1.012557E0,-2.7686968E-1,-1.1995372E-3,-1.790164E-2,-4.942715E-2,1.0660491E-1,5.7554835E-1,3.059845E-2,-4.588169E-1,5.580259E-3,1.7479694E-2,-1.3313731E-2,4.1303214E-2,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,-2.2212821E-1,-4.4453266E-1,7.171647E-2,-2.8227565E-1,2.1667868E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,-7.6152295E-2,1.4846555E-2,-9.6043665E-3,-5.975643E-1,7.728357E-3,-3.7457045E-2,-2.4542458E-1,2.0091075E-3,-7.810127E-3,-2.595061E-2,-1.12920456E-1,1.1881216E-1,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,-1,-1,57,-1,59,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5220662E1,3.804525E0,6.511341E0,6.5156603E0,4.4399395E0,9.4671035E-1,8.160591E-1,1.6291182E0,1.6117287E-1,1.2442894E0,1.2905694E0,3.4778595E-2,3.0859464E-1,2.4943113E-2,1.920681E-1,4.978342E-1,6.9657713E-1,0E0,0E0,3.9868122E-1,7.959266E-1,3.1622946E-1,3.536806E-1,0E0,0E0,0E0,7.072824E-2,0E0,0E0,0E0,0E0,3.2309484E-1,2.9235983E-1,2.4819759E-1,3.8475174E-1,7.987356E-2,0E0,0E0,0E0,3.175994E-2,0E0,0E0,1.1517382E-1,0E0,1.07335895E-2,1.10358E-1,0E0,0E0,0E0,1.0522947E-2,4.5623663E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,26,26,31,31,32,32,33,33,34,34,35,35,39,39,42,42,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,-1,-1,58,-1,60,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5085194E0,1.2871602E3,1.020266E3,8.260109E9,6.971004E7,2.3238E4,1.9107767E0,4.684559E2,2.6E1,5.5326223E9,8.462096E11,5.380165E0,3.236715E0,5.8229775E3,2.8626094E5,1.03431056E6,5.5015685E9,-1.790164E-2,-4.942715E-2,6.4348924E-1,5.6759316E7,5.4827127E0,1.5214152E5,5.580259E-3,1.7479694E-2,-1.3313731E-2,5.884028E0,2.5019243E-2,9.117976E-3,1.8951705E-2,5.225612E-2,1.1556807E0,5.7625353E-1,7.1568984E7,2.533331E6,4.1808826E-1,-8.161961E-3,3.465062E-2,4.4055944E-3,8.307621E7,1.4846555E-2,-9.6043665E-3,2.012E3,7.728357E-3,1.009E3,5.1146146E-2,2.0091075E-3,-7.810127E-3,-2.595061E-2,4.5E1,7.384995E4,-2.1005087E-2,2.9546588E-3,5.0991937E-3,1.6077418E-2,-6.082024E-3,1.9040752E-4,-1.07611865E-2,-3.3394508E-2,-3.6866798E-3,6.485111E-4,-4.6994775E-3,-1.2889486E-2,-7.0435596E-3,-1.5994612E-3,1.3460631E-2,-5.691292E-4],"split_indices":[42,52,52,5,45,9,42,52,3,5,31,53,56,4,28,28,5,0,0,27,45,57,33,0,0,0,53,0,0,0,0,42,27,7,1,27,0,0,0,45,0,0,0,0,0,27,0,0,0,3,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.92E2,1.52E2,4E1,1.08E2,4.4E1,1.9E1,2.1E1,9.6E1,1.2E1,2.3E1,2.1E1,8E0,1.1E1,6E0,1.5E1,6.4E1,3.2E1,2E0,1E1,1.1E1,1.2E1,9E0,1.2E1,2E0,6E0,5E0,6E0,4E0,2E0,2E0,1.3E1,5E1,1.4E1,2.6E1,6E0,8E0,3E0,9E0,3E0,7E0,2E0,5E0,7E0,2E0,4E0,4.6E1,4E0,4E0,1E1,5E0,2.1E1,4E0,2E0,5E0,3E0,4E0,3E0,2E0,5E0,2E0,2E0,6E0,4E1,3E0,2E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-7.2186072E-3,-1.7554772E-1,4.6836394E-1,-1.3343666E-1,-7.98081E-1,4.8964757E-2,7.305873E-1,-2.4706829E-1,1.5109251E-1,-1.1278819E-2,-9.1069216E-1,-2.2222637E-1,2.0841512E-1,5.310632E-1,1.0273924E0,-2.0253772E-1,-7.193482E-1,8.116926E-2,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,2.4997413E-1,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,-4.6462825E-1,-1.3208786E-1,-1.10341525E-2,-4.028151E-2,-1.6114761E-1,2.4269797E-1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,-5.287513E-1,-9.0207495E-2,-3.1034762E-1,1.1962035E-2,-2.5665852E-1,3.4205204E-1,-1.1185924E-2,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7454023E1,4.1142073E0,6.232873E0,4.9517193E0,4.2668343E-1,1.0408703E0,1.5270805E0,2.1466722E0,2.0214696E0,0E0,3.1103182E-1,1.4763036E-1,1.022948E-1,7.396126E-2,9.1228485E-3,1.8035164E0,2.927456E-1,1.6438966E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2034745E-2,0E0,0E0,0E0,0E0,2.2022247E-1,5.7671654E-1,0E0,0E0,2.841458E-1,6.34436E-1,0E0,0E0,0E0,1.3254166E-2,7.471074E-1,2.088238E-1,2.8850803E-1,9.1903985E-2,4.5164657E-1,1.8377072E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24,29,29,30,30,33,33,34,34,38,38,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,7.7567725E2,1.0523087E3,4.14E2,2.6263257E2,1.5605432E9,6.8652206E9,2.3274304E2,-1.1278819E-2,3.7284137E2,2.5546576E4,2.5086822E8,6.8734935E6,3.4456624E7,1.142E3,1.9E1,7.157224E0,4.0394735E-2,-5.11399E-2,-1.5907263E-2,-1.7994083E-2,-4.226395E-3,1.5395293E-3,1.630886E6,2.7758019E-2,1.3406339E-2,5.3184077E-2,2.1656366E-2,2.7373068E7,3.0143394E7,-1.10341525E-2,-4.028151E-2,2E0,1.0943299E1,1.5833307E-2,7.2834827E-3,-1.0521455E-2,5.9215684E0,7.54579E5,1.5641167E5,6.802721E-3,1.0180757E0,2.4794E4,9.875E3,-2.7091272E-2,-1.0813779E-2,-7.753554E-3,3.3467524E-3,-1.735609E-2,-3.0784312E-4,-8.734573E-3,9.753335E-3,-1.6083255E-2,-4.962887E-3,2.0094408E-2,-4.4497824E-4,-8.261265E-3,5.7380055E-3],"split_indices":[42,45,52,52,0,52,7,5,56,0,58,33,7,47,47,2,3,54,0,0,0,0,0,0,45,0,0,0,0,5,45,0,0,8,54,0,0,0,56,28,33,57,39,9,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.6E2,5.6E1,1.51E2,9E0,2.2E1,3.4E1,1.08E2,4.3E1,2E0,7E0,8E0,1.4E1,2.2E1,1.2E1,1E2,8E0,4E1,3E0,5E0,2E0,3E0,5E0,3E0,1.1E1,1.8E1,4E0,1E1,2E0,2E1,8E1,2E0,6E0,1.6E1,2.4E1,5E0,6E0,5E0,1.5E1,6.6E1,1.4E1,6E0,1E1,1.7E1,7E0,1.3E1,2E0,4.6E1,2E1,1.2E1,2E0,3E0,3E0,6E0,4E0,1.4E1,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-9.860434E-3,-1.3550225E-1,4.1085625E-1,-4.535459E-1,-6.382216E-2,2.3670215E-2,7.524566E-1,-5.6793636E-1,9.8671935E-2,-9.873285E-2,4.3589786E-1,2.2890316E-1,-1.3927497E-1,9.374098E-1,4.9539495E-1,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,-4.404742E-2,-6.1913097E-1,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,-2.5028974E-1,-2.472767E-2,1.5874594E-2,4.9255155E-2,5.498805E-1,6.3501373E-3,-1.7045158E-1,8.096087E-2,-4.8548445E-2,-3.0521184E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,-6.917094E-2,1.6786462E-2,3.2274876E-2,-2.1812399E-1,-5.0437325E-3,1.5704603E-1,-2.1648994E-2,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,39,-1,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1261909E1,3.7098274E0,6.463071E0,1.9510937E0,2.3981898E0,8.344817E-1,7.7176666E-1,1.549511E-1,3.8158774E-1,3.5869198E0,2.744478E-1,1.8382758E-1,1.7184111E-1,3.0896282E-1,1.867218E-1,0E0,0E0,0E0,0E0,1.8467818E0,1.121139E0,0E0,0E0,0E0,0E0,1.3394743E-2,3.954195E-2,0E0,0E0,2.6690483E-2,0E0,4.5625246E-1,4.6401808E-1,0E0,7.3043644E-2,0E0,0E0,0E0,1.0194156E-2,0E0,0E0,3.1877828E-1,1.343909E-1,8.3303446E-1,9.282285E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,25,25,26,26,29,29,31,31,32,32,34,34,38,38,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,40,-1,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.7575762E4,1.1382212E3,2.0199957E3,1.7730763E12,1.5593E4,8.812601E-1,4.6490747E2,3.20084E6,1.775894E10,4.6673E4,8.147317E0,3.2851852E2,4.558E3,1.4118051E7,-2.1002399E-2,-3.4177277E-2,-5.8313124E-3,1.7379109E-2,5.470729E2,8.92851E6,8.820802E-3,2.9363675E-2,1.411719E-2,-1.5052634E-3,6.314915E-1,4.1546512E2,1.5874594E-2,4.9255155E-2,3.76835E7,6.3501373E-3,3.9E1,3.291238E7,-4.8548445E-2,6.881048E-1,-1.413894E-2,-4.856759E-3,3.52352E-3,8.27E2,1.6786462E-2,3.2274876E-2,4.7095413E2,6.9327216E0,1.4154823E-1,2.4654E4,-1.773156E-2,-4.1464125E-3,-4.790018E-3,-3.459164E-4,-1.7294485E-2,-7.6624565E-3,4.080249E-3,-5.1540625E-3,1.4019506E-2,-1.3688318E-3,-5.838996E-3,1.5870973E-2],"split_indices":[42,51,4,4,31,9,57,52,7,5,10,53,4,2,1,0,0,0,0,52,1,0,0,0,0,38,4,0,0,48,0,3,45,0,34,0,0,0,0,0,0,4,53,38,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.63E2,4.8E1,2.9E1,1.34E2,2.3E1,2.5E1,2.4E1,5E0,1.26E2,8E0,1E1,1.3E1,1.3E1,1.2E1,1.4E1,1E1,3E0,2E0,1.15E2,1.1E1,4E0,4E0,8E0,2E0,6E0,7E0,2E0,1.1E1,1E1,2E0,5.7E1,5.8E1,4E0,7E0,4E0,2E0,2E0,5E0,5E0,5E0,4.4E1,1.3E1,3.3E1,2.5E1,5E0,2E0,3E0,2E0,1.2E1,3.2E1,7E0,6E0,1.9E1,1.4E1,2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.037411E-2,-1.7082255E-1,4.0125197E-1,-1.368692E-1,-4.7962084E-2,1.3270798E-1,7.9803574E-1,-2.2494297E-1,7.1677774E-2,2.673719E-1,-1.0539308E-1,1.9987479E-2,4.1916054E-2,-5.089811E-1,-1.4352345E-1,1.7057969E-1,-1.6686903E-2,1.1773016E-1,1.8603355E-2,-1.25300465E-2,-9.196732E-3,-3.748573E-1,-3.2896925E-2,-2.6696217E-1,-5.671464E-2,6.511845E-2,4.1393185E-1,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,-4.2942005E-1,-5.2706E-2,-3.20558E-1,5.7432473E-2,-1.8276703E-1,2.8034353E-1,-4.025757E-2,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,-1,27,-1,-1,29,31,-1,33,35,37,39,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3262604E1,4.125049E0,5.7545595E0,2.7528837E0,0E0,1.1183422E0,1.8529606E-1,2.355206E0,1.903528E0,3.3322942E-1,1.8780968E-1,0E0,0E0,2.6173544E-1,8.7425184E-1,9.226215E-1,0E0,4.9472928E-2,0E0,0E0,1.1085327E-1,8.237338E-2,0E0,3.7353468E-1,7.2992265E-1,6.2363416E-1,1.9650996E-1,0E0,0E0,0E0,0E0,0E0,7.688856E-2,1.9526544E-1,2.2509766E-1,3.0900213E-1,1.7987198E-1,2.4989247E-1,3.8682532E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,20,20,21,21,23,23,24,24,25,25,26,26,32,32,33,33,34,34,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,-1,28,-1,-1,30,32,-1,34,36,38,40,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.4233672E8,1.3453049E3,1.0648859E3,-4.7962084E-2,2.4608E4,2.7350403E5,1.7575762E4,9.820072E7,5.433479E2,2.5546576E4,1.9987479E-2,4.1916054E-2,1.3900659E6,2.802166E2,1.6E1,-1.6686903E-2,2.668E3,1.8603355E-2,-1.25300465E-2,1.9E1,2.9270105E2,-3.2896925E-2,2.124077E1,9.222491E0,3.10214E5,2.4986116E7,1.1070694E-3,8.21275E-3,5.874856E-3,-4.587929E-3,-8.752552E-3,6.1E1,1.5497297E1,3.079376E8,6.735744E1,3.1E1,5.1E1,6.8125E0,2.328887E-2,3.996887E-3,-2.3179155E-2,-7.08981E-3,-1.034502E-2,4.1719945E-3,-2.0279834E-2,-9.806688E-3,5.3788214E-3,-7.1446914E-3,-3.0508717E-3,-1.2197481E-2,1.798379E-2,-1.0322091E-3,-7.398417E-3,6.444544E-3],"split_indices":[42,45,52,52,0,9,28,51,45,52,33,0,0,28,52,8,0,2,0,0,3,33,0,58,54,29,48,0,0,0,0,0,3,58,5,58,3,3,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.53E2,5.4E1,1.48E2,5E0,3.3E1,2.1E1,1.04E2,4.4E1,2.1E1,1.2E1,4E0,1.7E1,2.2E1,8.2E1,3.6E1,8E0,1E1,1.1E1,4E0,8E0,1.4E1,8E0,3.3E1,4.9E1,2.6E1,1E1,4E0,6E0,3E0,5E0,4E0,1E1,7E0,2.6E1,2.6E1,2.3E1,8E0,1.8E1,8E0,2E0,8E0,2E0,3E0,4E0,1.3E1,1.3E1,2.1E1,5E0,9E0,1.4E1,6E0,2E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.8539885E-2,-1.315011E-1,4.7729447E-1,-3.645807E-1,-4.856526E-2,3.3575363E-2,7.048596E-1,-4.006475E-1,1.0404514E-2,-1.2890418E-1,1.0923791E-1,2.1161272E-1,-2.1542291E-1,4.2066466E-2,7.948288E-1,-5.199407E-1,-2.0412306E-1,-1.0585207E-1,-3.185486E-2,-2.5976712E-1,1.9807549E-1,1.8924989E-3,3.4906948E-1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,-5.954765E-1,-2.6221934E-1,-4.2201504E-1,1.2239022E-2,-1.8437941E-1,-8.044045E-3,-5.61047E-2,-2.8251013E-2,2.2316225E-2,1.2032818E-1,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,-1.095318E-1,-2.132905E-1,-1.3484026E-2,1.0698679E-1,-1.6088222E-1,-6.4098774E-3,3.0856705E-3,3.494343E-1,6.278915E-2,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.39668255E1,2.9398837E0,5.0250864E0,9.200578E-1,1.4593433E0,8.4354407E-1,1.9343872E0,7.847419E-1,0E0,8.516334E-1,1.3299325E0,2.4759096E-1,8.105555E-2,1.5741335E-1,1.7285347E-1,2.9159975E-1,7.59462E-1,5.668815E-1,0E0,4.9000573E-1,5.9287274E-1,0E0,1.3428587E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.4581165E-2,1.9908011E-2,1.5304077E-1,2.1783859E-1,1.9969559E-1,6.1496985E-1,6.256449E-2,0E0,0E0,3.2080543E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8989278E-2,1.5572238E-1,2.168184E-2,1.3415802E-1,2.2903234E-1,0E0,0E0,5.3615987E-2,2.547765E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,48,48,49,49,50,50,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.130317E8,6.794883E4,1E0,4.5342876E3,6.787619E2,9.820072E7,3.788E3,1.4862166E1,1.0404514E-2,4.462437E7,7.443E3,3.5073215E3,1.2895599E4,6.09E2,4.2414474E0,6.1E1,5.0548022E2,2.98022E2,-3.185486E-2,9.64E2,1.20171E5,1.8924989E-3,5.1E1,-1.566189E-2,-4.2446754E-3,-6.2135113E-3,9.625554E-3,4.0909473E-2,1.9805295E-2,5.0446276E7,1.0804272E3,1.0363E4,4.1749332E4,2.0890524E6,7.6420575E6,1.1484E4,-2.8251013E-2,2.2316225E-2,1.2117E4,2.4938837E-2,6.7778295E-3,-2.2307793E-2,-3.7493635E-2,-1.603496E-2,-6.2972833E-3,-5.454884E-3,-2.465652E-2,9.335951E-3,1.7488E4,2.124077E1,4.35E2,1.725E3,6.9789816E7,-6.4098774E-3,3.0856705E-3,4E0,6.1108776E7,-7.7258847E-3,-3.5986607E-4,-3.809706E-3,-1.218577E-2,2.7086088E-3,-2.5438727E-3,-4.9488633E-3,6.6099954E-3,5.6492415E-4,-1.2083303E-2,6.3400804E-3,2.2002792E-2,4.921263E-3,-1.0399772E-2],"split_indices":[7,50,109,4,52,45,2,58,0,45,9,52,4,0,57,3,4,52,0,0,29,0,3,0,0,0,0,0,0,45,33,9,32,50,45,2,0,0,9,0,0,0,0,0,0,0,0,0,9,58,0,2,12,0,0,8,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.52E2,4.9E1,3.9E1,1.13E2,1.7E1,3.2E1,3.7E1,2E0,7.5E1,3.8E1,1E1,7E0,4E0,2.8E1,2.2E1,1.5E1,7.3E1,2E0,7E0,3.1E1,5E0,5E0,3E0,4E0,2E0,2E0,2.4E1,4E0,1.6E1,6E0,7E0,8E0,4E1,3.3E1,5E0,2E0,6E0,2.5E1,2E0,3E0,1.1E1,5E0,3E0,3E0,2E0,5E0,3E0,5E0,3.4E1,6E0,1.9E1,1.4E1,3E0,2E0,4E0,2.1E1,3E0,2E0,8E0,2.6E1,2E0,4E0,2E0,1.7E1,5E0,9E0,2E0,2E0,1.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-2.776328E-2,-1.2117824E-1,3.86292E-1,-5.909288E-2,-5.795699E-1,5.5683386E-1,-2.6300025E-1,-2.0977455E-1,9.621003E-2,-8.023927E-1,-1.1718366E-1,2.3014233E-1,6.94869E-1,-1.3781548E-3,-3.7562144E-1,-3.571755E-1,-1.0406611E-1,1.914385E-1,-8.06779E-2,-2.3914319E-2,-4.741177E-2,9.839836E-3,-3.492453E-1,3.794242E-1,-6.1298814E-3,3.9063934E-2,3.491601E-1,-2.482055E-2,-3.466254E-3,-4.0868425E-1,-5.727207E-2,-1.4600524E-1,2.0078115E-2,7.405951E-2,3.1055027E-1,-1.1359027E-1,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,-4.7098124E-1,-1.5451024E-1,2.9516693E-3,-6.394594E-3,4.4541497E-2,-1.712155E-1,7.7366033E-3,-5.9316065E-2,1.5788026E-1,-6.221433E-2,3.575082E-1,-1.2339829E-3,-7.365338E-2,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,61,63,-1,65,67,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.401466E0,5.003068E0,4.575707E0,3.694187E0,2.0696063E0,1.2831182E0,2.2723442E-1,1.2093525E0,1.3293368E0,2.6940823E-1,6.6352755E-1,6.3254225E-1,5.397568E-1,0E0,2.0912564E-1,4.8636627E-1,2.528435E-1,6.9004285E-1,3.1393635E-1,0E0,0E0,0E0,6.1558306E-2,2.0225549E-1,0E0,0E0,7.9099655E-2,0E0,0E0,3.7058783E-1,6.041512E-2,1.8056059E-1,1.5278538E-1,3.1823194E-1,4.0340137E-1,1.3923776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.6962032E-2,1.4837602E-1,0E0,0E0,2.0146374E-2,1.326319E-1,0E0,4.3555457E-2,1.6426831E-1,1.09528095E-1,6.0819864E-2,0E0,7.7050425E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,22,22,23,23,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,62,64,-1,66,68,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0105532E3,7.6599895E9,1.2055723E8,4.684559E2,1.5461028E12,8.601656E6,1.3611247E5,1.822764E4,2.2971932E7,3.6509445E1,2.5E1,2.4716E4,5.9283892E7,-1.3781548E-3,1.3382495E8,1E0,2.0806985E-1,7.7567725E2,7.4297624E0,-2.3914319E-2,-4.741177E-2,9.839836E-3,1.776E3,5.1E1,-6.1298814E-3,3.9063934E-2,1.046E4,-2.482055E-2,-3.466254E-3,1.01978E5,1.685E3,4.914413E-3,5.1E2,6.2742615E0,2.8314E4,8.539363E7,1.1264107E-2,-6.1374507E-3,-2.2190096E-2,2.3005225E-2,3.2301592E-3,7.452221E-3,2.2284444E-2,7.774228E-2,7.359468E8,2.9516693E-3,-6.394594E-3,3.76E2,5.2560944E1,7.7366033E-3,3.892E3,2.7883E4,4.4589956E5,7.3586698E0,-1.2339829E-3,1E0,-1.2974207E-2,-2.3886124E-2,-8.846369E-3,2.559347E-3,-1.2062858E-2,4.739848E-3,-1.1270544E-3,-1.0161947E-2,-2.5279624E-3,-6.672173E-3,1.1806205E-4,9.479755E-3,-3.8591044E-3,-8.365138E-3,1.2172054E-3,1.8634392E-2,7.179232E-3,-4.7638407E-3,2.9137798E-3],"split_indices":[52,5,45,52,31,1,33,33,45,56,3,2,45,0,32,112,38,52,53,0,0,0,0,3,0,0,9,0,0,29,2,38,0,53,9,45,0,0,0,0,0,0,0,57,5,0,0,0,58,0,11,9,28,53,0,89,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.76E2,3.9E1,1.56E2,2E1,3.1E1,8E0,7.9E1,7.7E1,1.3E1,7E0,1E1,2.1E1,3E0,5E0,3.2E1,4.7E1,5E1,2.7E1,6E0,7E0,3E0,4E0,7E0,3E0,1.5E1,6E0,3E0,2E0,2.7E1,5E0,3.5E1,1.2E1,2.6E1,2.4E1,2.5E1,2E0,2E0,2E0,5E0,2E0,3E0,3E0,2.1E1,6E0,2E0,3E0,4E0,3.1E1,4E0,8E0,1.6E1,1E1,2.1E1,3E0,2.1E1,4E0,1.9E1,2E0,2E0,4E0,2E0,2E0,2.3E1,8E0,3E0,5E0,1.4E1,2E0,4E0,6E0,1.8E1,3E0,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[7.874234E-3,-1.0717477E-1,4.6521157E-1,-1.982936E-1,9.943495E-2,1.02683544E-1,7.759888E-1,-1.3372196E-1,-6.0561585E-1,2.801086E-1,-2.3861834E-1,-1.4555064E-1,1.8935364E-1,4.1564103E-2,4.7552985E-1,-1.7659279E-2,-9.404589E-2,-7.024602E-3,-6.612021E-1,1.8482257E-1,3.9656345E-2,-7.397643E-2,-5.8301306E-1,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.4021983E-1,9.948502E-3,2.6430182E-2,-1.4144325E-1,2.806465E-2,-2.1147303E-2,-3.7808225E-2,2.9104125E-1,-3.4760606E-2,5.0945976E-3,-1.12476826E-1,-1.1272882E-2,-3.41051E-2,9.7476035E-2,1.4251037E-2,-8.572106E-2,-2.9373407E-1,9.719249E-2,-1.7345269E-1,5.5071485E-4,3.198904E-1,-1.1351033E-2,3.5261977E-2,-1.3395034E-1,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,35,37,-1,-1,-1,39,-1,-1,41,43,-1,-1,45,47,-1,49,-1,-1,51,-1,53,55,57,59,-1,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1257111E1,3.2389693E0,4.7649326E0,3.0488906E0,3.303937E0,4.7936463E-1,1.3339043E-1,9.1138387E-1,3.148079E-1,1.6327975E0,1.0342052E0,4.6653435E-2,1.7866707E-1,0E0,1.7190337E-2,0E0,5.248641E-1,0E0,2.9516697E-2,7.331557E-1,0E0,1.08073495E-1,8.598232E-2,0E0,0E0,0E0,6.9084704E-2,0E0,0E0,5.2640927E-1,3.790502E-1,0E0,0E0,1.658467E-1,1.6110414E-1,0E0,1.4867976E-2,0E0,0E0,1.5488528E-2,0E0,2.9356018E-1,1.905942E-2,1.5623116E-1,6.481768E-2,0E0,1.2719929E-1,0E0,4.8637956E-2,9.308532E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,18,18,19,19,21,21,22,22,26,26,29,29,30,30,33,33,34,34,36,36,39,39,41,41,42,42,43,43,44,44,46,46,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,36,38,-1,-1,-1,40,-1,-1,42,44,-1,-1,46,48,-1,50,-1,-1,52,-1,54,56,58,60,-1,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7987691E0,1.0648859E3,1.5262272E3,5.0446276E7,7.294459E7,1.0437421E8,2.98476E0,1.295E3,6.856226E-1,6.2150537E3,2.09E3,7.44E2,1.5675428E5,4.1564103E-2,3.145611E6,-1.7659279E-2,5.3287085E2,-7.024602E-3,9.180692E7,2.249985E5,3.9656345E-2,1.1341179E3,1.4233672E8,-2.0160642E-3,-1.1478778E-2,-1.1339232E-3,2.398624E8,9.948502E-3,2.6430182E-2,6.7841024E10,3.0143394E7,-2.1147303E-2,-3.7808225E-2,7.91144E5,2.3276744E5,5.0945976E-3,3.7387386E-1,-1.1272882E-2,-3.41051E-2,1.144E3,1.4251037E-2,6.0130353E0,7.273E3,9.4390506E5,2.9673992E9,5.5071485E-4,1E0,-1.1351033E-2,2.6387034E5,9.9764734E4,-1.7540465E-3,6.992818E-3,9.135353E-4,1.1705131E-3,-6.7973924E-3,-5.3676493E-3,-1.5127462E-2,1.2226892E-2,2.0916124E-3,-2.1916362E-3,-1.2580683E-2,4.4793054E-3,1.7367091E-2,-2.659917E-3,4.3473905E-3,-2.8596863E-3,-7.871873E-3],"split_indices":[42,52,4,45,45,7,57,2,27,4,0,0,28,0,1,0,52,0,7,33,0,33,45,0,0,0,7,0,0,31,45,0,0,1,33,0,57,0,0,0,0,53,9,32,5,0,109,0,28,28,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.7E2,4.2E1,1.18E2,5.2E1,2E1,2.2E1,1.03E2,1.5E1,3.4E1,1.8E1,5E0,1.5E1,1.6E1,6E0,1.4E1,8.9E1,2E0,1.3E1,3E1,4E0,1.3E1,5E0,3E0,2E0,3E0,1.2E1,2E0,4E0,6.4E1,2.5E1,6E0,7E0,2E1,1E1,2E0,1.1E1,2E0,3E0,4E0,8E0,4.8E1,1.6E1,1.9E1,6E0,2E0,1.8E1,2E0,8E0,8E0,3E0,2E0,2E0,1.6E1,3.2E1,2E0,1.4E1,4E0,1.5E1,3E0,3E0,3E0,1.5E1,3E0,5E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.3249438E-2,-1.2597813E-1,3.6839122E-1,-3.741495E-1,-6.773875E-2,5.7275885E-1,-2.5949249E-2,-4.510603E-1,-5.7005443E-2,-1.6321269E-1,4.9919948E-2,2.9108667E-1,7.353367E-1,1.081298E-1,-2.378517E-2,-4.51294E-3,-4.978342E-1,1.5801355E-3,-4.8320345E-3,-1.921461E-2,-1.2809235E-1,1.7340758E-1,-1.6564597E-1,3.5022026E-1,-1.1079086E-3,4.0072575E-2,4.0726957E-1,-5.00003E-2,3.2730332E-1,-8.940148E-3,-5.212322E-1,-7.0953816E-3,-1.7170624E-1,2.1817629E-1,-2.174709E-2,-2.512538E-1,1.0451725E-1,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,4.3051414E-2,1.952225E-2,5.8277114E-3,-1.2334295E-2,-5.556018E-1,-1.2370801E-1,4.7271628E-2,-9.585648E-2,-2.4668616E-1,3.1131923E-2,2.7143374E-1,6.132134E-4,-4.048703E-3,-1.6482335E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,65,67,69,71,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.865889E0,2.272738E0,3.8026128E0,7.0604134E-1,1.4684818E0,1.2055569E0,1.1077032E0,3.5608482E-1,2.9706666E-2,5.3508985E-1,1.5992881E0,2.492423E-1,2.136507E-1,5.015763E-1,0E0,0E0,5.8598995E-2,0E0,0E0,0E0,3.3651924E-1,3.3674383E-1,5.350784E-1,1.20595574E-1,0E0,0E0,1.174857E-1,1.1404645E-1,4.39198E-2,0E0,3.405571E-2,1.1996572E-1,2.4365234E-1,3.003564E-1,1.7932303E-2,2.712661E-1,2.0998505E-1,0E0,0E0,0E0,0E0,0E0,1.23745585E-2,0E0,0E0,0E0,3.083849E-2,2.4845354E-2,9.429864E-2,2.3359656E-1,2.8410673E-2,1.0068264E-1,1.6567254E-1,0E0,0E0,6.312257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,20,20,21,21,22,22,23,23,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,42,42,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,66,68,70,72,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.4862166E1,6.627576E7,1.7497415E-3,5.3287085E2,3.5073215E3,5.7006E4,3.7453184E-3,9.096081E7,9.4418E4,3.1821228E7,5.2E1,1.0096877E8,1.0252E4,-2.378517E-2,-4.51294E-3,6.519E3,1.5801355E-3,-4.8320345E-3,-1.921461E-2,4.06E2,2.8314E4,4.4E1,1.075839E9,-1.1079086E-3,4.0072575E-2,1.5605432E9,3.9880952E-1,1E0,-8.940148E-3,2.0521739E0,1.369E3,9.258696E0,5.716463E2,1.3717402E0,7.6599895E9,1.2633E4,1.9539105E-2,3.8772572E-3,6.1657904E-3,2.5106201E-2,-8.617747E-3,2E0,1.952225E-2,5.8277114E-3,-1.2334295E-2,1.0595825E3,1.36135E5,9.087618E6,1.1947E4,2.4734788E7,8.5093097E2,5.0691235E6,6.132134E-4,-4.048703E-3,1.8159722E-1,-2.3892645E-2,-3.1008373E-3,1.4307415E-2,3.6782157E-3,-7.139691E-4,-3.1046964E-2,-1.7450994E-2,-8.264143E-3,-1.0220006E-3,-4.175107E-4,8.520538E-3,6.535376E-3,-6.4893803E-3,-7.3677725E-3,-1.3573787E-2,5.002019E-3,-5.9638247E-3,1.9579979E-2,9.434901E-3,-1.1248127E-5,-9.355448E-3],"split_indices":[52,58,45,39,52,52,10,58,7,1,45,3,51,9,0,0,9,0,0,0,0,9,3,7,0,0,7,57,8,0,56,2,54,52,42,5,2,0,0,0,0,0,8,0,0,0,33,1,32,9,12,4,45,0,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.58E2,4.6E1,2.9E1,1.29E2,3E1,1.6E1,2.3E1,6E0,7.1E1,5.8E1,1.2E1,1.8E1,1.3E1,3E0,3E0,2E1,2E0,4E0,8E0,6.3E1,3.7E1,2.1E1,1E1,2E0,1.3E1,5E0,8E0,5E0,2E0,1.8E1,1.7E1,4.6E1,3E1,7E0,1.6E1,5E0,8E0,2E0,2E0,3E0,3E0,5E0,3E0,2E0,3E0,1.5E1,5E0,1.2E1,2.4E1,2.2E1,7E0,2.3E1,5E0,2E0,1.3E1,3E0,3E0,2E0,3E0,2E0,9E0,6E0,3E0,2E0,9E0,3E0,3E0,2.1E1,7E0,1.5E1,5E0,2E0,7E0,1.6E1,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[6.0231384E-2,-6.6708066E-2,4.1548002E-1,-1.802126E-1,9.4408296E-2,6.121976E-1,4.159176E-2,-1.3381815E-1,-4.551602E-1,-2.3027116E-1,1.6258606E-1,2.7157372E-1,6.904492E-1,1.487131E-1,-1.9334765E-2,-3.371152E-1,-7.73753E-2,-1.0917176E-2,-2.8615614E-2,-1.011948E-1,-2.631605E-2,7.940243E-2,3.767569E-1,1.6384088E-2,2.4758964E-3,8.306936E-1,4.7551715E-1,3.2327557E-1,1.1648051E-2,-9.826521E-3,-2.175756E-2,-1.1204586E-1,1.6198908E-1,-1.3703959E-1,7.871614E-4,1.0334385E-1,-7.2968313E-3,4.7740215E-1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,-8.063122E-2,5.84258E-3,-3.1697118E-1,-6.857089E-2,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.4921291E-1,-1.0298919E-2,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,-1,35,37,-1,-1,39,41,43,45,-1,-1,47,49,51,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,57,-1,59,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.911444E0,2.6869793E0,3.7991571E0,1.0361109E0,1.3877299E0,7.1874046E-1,9.523439E-1,8.3451545E-1,2.3604941E-1,3.861003E-1,8.7867534E-1,9.7789764E-2,4.188528E-1,3.7905708E-1,0E0,1.4759302E-1,5.16084E-1,0E0,0E0,4.008031E-2,0E0,2.2414063E-1,3.420025E-1,0E0,0E0,4.281807E-2,4.2568207E-2,1.4053464E-1,1.0971589E-1,0E0,0E0,4.504485E-1,6.2172145E-2,1.6357675E-2,0E0,1.8398082E-1,0E0,1.7374277E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1939041E-2,0E0,2.7805746E-2,2.6995105E-1,0E0,0E0,0E0,0E0,2.1720147E-1,1.2581149E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,37,37,45,45,47,47,48,48,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,-1,36,38,-1,-1,40,42,44,46,-1,-1,48,50,52,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,58,-1,60,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0601392E3,2.1836595E8,6.627576E7,8.93182E5,1E0,1.2737E4,5.7006E4,1.042E3,4E0,7.6499896E9,1.068371E3,3.5E1,1.5213319E0,1E0,-1.9334765E-2,8.601996E-1,1E0,-1.0917176E-2,-2.8615614E-2,2.3276744E5,-2.631605E-2,4.759155E1,7.9762895E6,1.6384088E-2,2.4758964E-3,1.929E3,9.299267E9,3.6392304E-1,6.331E3,-9.826521E-3,-2.175756E-2,1.5497297E1,6.98247E5,1.1534339E9,7.871614E-4,7.183439E0,-7.2968313E-3,8.383825E1,4.8722425E-3,4.6811514E-2,2.6442848E-2,2.4902169E-2,8.972875E-3,3.0673465E-3,2.0185333E-2,1.181E3,5.84258E-3,3.5E1,5.974511E0,1.7042852E-3,1.1250064E-2,-8.248987E-3,-1.8132239E-3,1.8613E4,2.770862E8,2.746903E-2,1.3504093E-2,-5.5276375E-4,-5.471479E-3,-5.8001215E-3,-1.7347483E-2,1.4247099E-3,-6.2790834E-3,1.2426594E-2,2.989934E-3,-7.5093266E-3,3.0654971E-3],"split_indices":[52,7,45,29,109,2,10,2,8,5,52,3,57,8,0,27,94,0,0,33,0,56,45,0,0,0,12,27,9,0,0,58,28,7,0,53,0,58,0,0,0,0,0,0,0,0,0,10,53,0,0,0,0,9,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.45E2,5.1E1,8.5E1,6E1,3.3E1,1.8E1,7.4E1,1.1E1,1E1,5E1,7E0,2.6E1,1.5E1,3E0,1.5E1,5.9E1,5E0,6E0,8E0,2E0,3.7E1,1.3E1,5E0,2E0,1.4E1,1.2E1,6E0,9E0,8E0,7E0,5.2E1,7E0,6E0,2E0,3.4E1,3E0,9E0,4E0,8E0,6E0,1E1,2E0,2E0,4E0,5E0,4E0,8E0,4.4E1,3E0,4E0,4E0,2E0,2.4E1,1E1,5E0,4E0,2E0,3E0,2E0,6E0,1.7E1,2.7E1,1E1,1.4E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[6.2760315E-3,-8.942039E-2,3.3899674E-1,-3.464915E-1,-2.9866556E-2,-3.944495E-2,5.6220204E-1,-7.096159E-2,-4.272631E-1,-3.1877894E-2,-1.0096922E-2,-2.7154645E-1,1.3294205E-1,3.9848673E-1,3.567469E-2,-8.072219E-3,6.935243E-3,-2.918099E-1,-2.5652623E-2,-1.33155E-1,6.0559504E-2,-4.3664877E-3,-2.650756E-2,1.6947778E-2,8.673523E-3,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,5.782375E-2,-1.6802189E-1,2.2789724E-1,-4.5959535E-3,3.984736E-3,-5.162517E-2,6.689567E-3,-5.874931E-3,-2.1503107E-1,-5.4912195E-3,7.9473734E-2,3.573293E-1,-3.5994206E-2,2.3827569E-1,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,27,-1,29,31,-1,-1,-1,33,-1,-1,-1,-1,35,37,39,41,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5575504E0,2.4391828E0,3.907806E0,6.373303E-1,1.6124437E0,7.563031E-1,5.1900864E-1,1.857129E-1,1.5725279E-1,0E0,1.1210039E0,3.4842473E-1,2.9128677E-1,7.89125E-2,0E0,0E0,0E0,4.2960286E-2,0E0,3.2268065E-1,8.950907E-1,0E0,0E0,0E0,3.954031E-2,0E0,0E0,0E0,0E0,1.3033742E-1,3.0443656E-1,4.1344917E-1,4.6612126E-1,0E0,1.9462278E-2,0E0,0E0,1.0997474E-1,4.840184E-2,1.0358285E-1,2.309668E-2,5.0568026E-1,1.8324465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,17,17,19,19,20,20,24,24,29,29,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,28,-1,30,32,-1,-1,-1,34,-1,-1,-1,-1,36,38,40,42,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7108864E8,1.4133928E1,1E0,2.862988E5,3.673257E0,3.067586E5,1.5435694E9,5.34E2,3.203027E7,-3.1877894E-2,3.3969492E2,2.565E3,1E0,1.4008022E11,3.567469E-2,-8.072219E-3,6.935243E-3,1.3448806E8,-2.5652623E-2,2.0589474E1,1.10597E5,-4.3664877E-3,-2.650756E-2,1.6947778E-2,4.078932E5,9.4667105E-3,2.1718945E-2,-1.568656E-2,-4.5168605E-3,7.269755E0,1.7869681E6,6.185678E2,3.1982856E3,3.984736E-3,1.743E3,6.689567E-3,-5.874931E-3,1.5373E4,7.343501E-1,4.0759E4,4.3411046E8,9.518453E6,1.4139E4,8.319301E-4,-5.0192857E-3,-6.300102E-3,-1.31226275E-2,1.7344939E-3,-5.5161105E-3,8.392618E-3,-5.633232E-4,1.882791E-2,6.8000876E-3,4.7115507E-3,-5.114217E-3,1.8424604E-3,1.8450594E-2],"split_indices":[7,58,109,28,53,28,7,0,45,0,52,0,8,31,0,0,0,7,0,58,29,0,0,0,28,0,0,0,0,53,50,52,4,0,0,0,0,9,27,29,7,45,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.59E2,4.5E1,2.9E1,1.3E2,1.7E1,2.8E1,7E0,2.2E1,3E0,1.27E2,7E0,1E1,1.6E1,1.2E1,5E0,2E0,1.1E1,1.1E1,4.6E1,8.1E1,5E0,2E0,3E0,7E0,4E0,1.2E1,9E0,2E0,7E0,3.9E1,2.2E1,5.9E1,3E0,4E0,5E0,2E0,3E1,9E0,1.1E1,1.1E1,5.3E1,6E0,2E0,2E0,1.3E1,1.7E1,7E0,2E0,5E0,6E0,9E0,2E0,1.8E1,3.5E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.2222775E-2,-9.139186E-2,4.9998534E-1,-5.1995587E-2,-4.944162E-1,6.0979265E-1,-1.5150189E-2,-1.3674228E-1,1.0816623E-1,3.8813213E-3,-6.330518E-1,8.045889E-1,3.992678E-1,-3.3645314E-1,-1.01294234E-1,2.436709E-1,-8.044677E-2,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,-3.9906213E-1,-5.7864846E-3,-5.545319E-2,-2.0736766E-1,-8.008159E-2,3.159065E-1,-1.8992147E-1,-6.997158E-3,-2.360766E-2,-1.1299601E-2,-5.494948E-3,-2.1435164E-1,-2.4299191E-1,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.7862383E-1,4.077045E-1,-5.167938E-3,-1.228408E-2,-5.0322257E-2,4.2618223E-2,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,37,39,41,43,-1,-1,45,47,49,-1,-1,-1,51,53,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.480311E0,2.8481483E0,2.6842346E0,2.2679396E0,1.3240938E0,7.0399E-1,0E0,7.3756933E-1,1.5019631E0,0E0,4.9333286E-1,1.782751E-2,2.6048756E-1,1.7048275E-1,4.4557756E-1,8.2043767E-1,1.9969293E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.9644938E-2,0E0,5.306874E-1,1.2524438E-1,5.6175735E-2,2.8003812E-1,2.552101E-2,3.65381E-2,0E0,0E0,2.7235347E-1,1.0511035E-1,6.087303E-2,0E0,0E0,0E0,1.4527509E-1,1.0674596E-1,0E0,0E0,2.577142E-2,9.493223E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,23,23,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,39,39,40,40,43,43,44,44],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,38,40,42,44,-1,-1,46,48,50,-1,-1,-1,52,54,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,1.650269E10,1.2055723E8,6.978001E2,1.8159722E-1,8.812601E-1,-1.5150189E-2,1.142E3,3.291238E7,3.8813213E-3,5.535E3,1.929E3,1.7595541E0,1.3448806E8,1.06403265E1,1E0,2.1845242E3,-4.2579796E-2,-1.8128771E-2,4.3203447E-2,2.026169E-2,4.804704E-3,2.2983126E-2,1.22E2,-5.7864846E-3,1.9540394E7,9.744047E0,9.31E2,1.068371E3,5.5053414E9,7.951E3,-2.360766E-2,-1.1299601E-2,1.1888112E-1,4.22631E5,4.187504E7,-3.201366E-3,8.6682016E-4,-7.6870713E-3,1.617623E5,2.9E1,-5.167938E-3,-1.228408E-2,2.5821698E0,7.157903E7,4.219129E-3,-2.9784364E-3,-4.3842704E-3,-1.3621782E-2,-9.7163925E-3,-1.888767E-2,1.0912775E-2,-2.343902E-3,2.2033297E-2,7.7579026E-3,1.7331039E-3,-3.89148E-3,-5.742622E-4,3.0525676E-3],"split_indices":[52,5,45,52,27,57,0,2,45,0,9,0,57,7,54,109,4,0,0,0,0,0,0,10,0,45,53,0,52,5,9,0,0,57,28,32,0,0,0,33,8,0,0,53,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.8E2,2.7E1,1.65E2,1.5E1,2.4E1,3E0,1.08E2,5.7E1,3E0,1.2E1,1.1E1,1.3E1,1.5E1,9.3E1,3.3E1,2.4E1,5E0,7E0,8E0,3E0,3E0,1E1,1.1E1,4E0,6.6E1,2.7E1,6E0,2.7E1,9E0,1.5E1,6E0,5E0,5.1E1,1.5E1,2.1E1,6E0,3E0,3E0,1.2E1,1.5E1,5E0,4E0,8E0,7E0,1.9E1,3.2E1,6E0,9E0,1.8E1,3E0,1E1,2E0,1.2E1,3E0,2E0,6E0,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.469039E-3,-6.84523E-2,5.0491726E-1,-1.8420036E-1,8.1181556E-2,6.261244E-1,3.6942E-2,-2.2080618E-1,7.735341E-2,-3.1455643E-2,2.747695E-1,3.8490146E-2,4.343899E-1,5.4492303E-3,-3.6699688E-3,-1.701605E-1,-4.6023417E-1,-3.2597054E-2,1.7538356E-2,-2.7009997E-1,4.3159403E-2,1.4272796E-1,3.6220354E-1,7.502832E-3,2.3337092E-2,-2.6047137E-1,-8.6393684E-2,-3.3155903E-2,-2.9781944E-1,3.291199E-4,-5.0207926E-3,-1.3054396E-1,-2.0009596E-2,-3.049531E-3,2.1293099E-1,1.6921525E-1,-5.2545854E-4,2.1539481E-2,2.3263586E-1,-3.6815163E-1,-1.5577061E-1,-1.2372288E-2,-4.669773E-2,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,-5.099285E-2,7.831989E-2,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,-1,31,33,35,37,-1,-1,39,41,-1,43,-1,-1,45,-1,47,49,51,-1,-1,53,55,57,-1,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009293E0,3.3230407E0,1.3381958E0,1.0511143E0,1.8402805E0,2.5670147E-1,5.9022054E-2,1.0736556E0,4.4741017E-1,9.7320604E-1,3.0619597E-1,0E0,6.5217495E-2,0E0,0E0,5.8270144E-1,3.7153912E-1,3.1251714E-2,0E0,2.0223719E-1,3.3013782E-1,5.8757484E-2,8.452511E-2,0E0,0E0,3.710723E-1,2.7460602E-1,0E0,4.2746305E-2,0E0,0E0,6.497468E-2,0E0,1.3663186E-1,6.1325133E-2,2.643928E-2,0E0,0E0,1.6941309E-2,5.598998E-2,1.3381499E-1,0E0,1.9927172E-1,0E0,0E0,0E0,0E0,1.089835E-1,1.0501201E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28,31,31,33,33,34,34,35,35,38,38,39,39,40,40,42,42,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,-1,32,34,36,38,-1,-1,40,42,-1,44,-1,-1,46,-1,48,50,52,-1,-1,54,56,58,-1,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.478327E3,2.2896422E6,9.298568E7,2.096851E3,1.537424E0,8.812601E-1,1.2055723E8,5.0446276E7,1.0625348E8,5.6E0,8.57703E2,3.8490146E-2,4.11964E5,5.4492303E-3,-3.6699688E-3,4.9658173E2,5.511724E0,5.9527163E3,1.7538356E-2,4.3927447E11,5.4732365E9,1.1216892E3,6.518218E6,7.502832E-3,2.3337092E-2,1.822764E4,1E0,-3.3155903E-2,2.0529972E1,3.291199E-4,-5.0207926E-3,6.4336755E8,-2.0009596E-2,7.9472524E-1,3.9952212E3,8.3154816E2,-5.2545854E-4,2.1539481E-2,7.8114974E-1,4.3117376E8,2.524E3,-1.2372288E-2,1.3474202E8,-1.6784841E-2,-6.288304E-3,-9.754274E-3,-5.1337713E-4,1.03235014E-1,9.445656E6,1.2546066E-2,1.8136286E-3,4.728268E-3,1.0953895E-2,1.2770799E-2,4.166148E-3,-2.0230724E-2,-9.916658E-3,-9.174111E-3,1.4842916E-3,2.667299E-3,-4.8537054E-3,4.0545748E-3,-4.159387E-3,6.2763602E-3,-3.303361E-3],"split_indices":[52,50,45,4,42,57,45,45,7,54,52,0,29,0,0,4,53,33,0,31,5,4,45,0,0,33,64,0,56,0,0,7,0,39,4,4,0,0,27,5,2,0,7,0,0,0,0,57,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.9E2,2.3E1,1.07E2,8.3E1,1.8E1,5E0,9.4E1,1.3E1,5.3E1,3E1,8E0,1E1,3E0,2E0,7.9E1,1.5E1,1E1,3E0,1.2E1,4.1E1,1.3E1,1.7E1,2E0,8E0,3.7E1,4.2E1,5E0,1E1,7E0,3E0,7E0,5E0,3.3E1,8E0,1.1E1,2E0,9E0,8E0,1.7E1,2E1,7E0,3.5E1,7E0,3E0,4E0,3E0,2.1E1,1.2E1,6E0,2E0,6E0,5E0,6E0,2E0,1.2E1,5E0,1.7E1,3E0,1.2E1,2.3E1,4E0,1.7E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-2.0219442E-2,-1.16520904E-1,2.8449756E-1,-3.0797613E-1,-5.9537902E-2,9.543412E-2,5.2078587E-1,-3.3963862E-1,7.4714734E-3,-2.8888127E-2,-3.689822E-1,2.1040678E-1,-8.497771E-2,3.1196967E-2,3.1238148E-1,-2.7664992E-1,-2.8590444E-2,-1.8818352E-1,6.0974874E-2,-4.8011044E-1,-2.6513548E-3,1.2447058E-2,4.99297E-3,2.9411163E-2,-1.22154895E-2,5.4155197E-3,1.8623266E-2,-3.116507E-1,-1.2212595E-1,-3.6836183E-1,-1.402538E-1,3.1627953E-2,3.3315614E-1,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,-3.2800922E-1,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,-1.5982786E-1,-2.9887237E-2,1.3616896E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.107501E0,1.705395E0,2.1839733E0,5.781839E-1,1.1559219E0,6.194103E-1,3.51089E-1,3.682499E-1,0E0,1.6290425E0,3.58294E-1,6.1777413E-2,2.3528156E-1,0E0,9.8772764E-2,1.18784666E-1,0E0,3.0240762E-1,5.7256365E-1,1.4015281E-1,0E0,0E0,0E0,2.9680826E-2,0E0,0E0,0E0,5.1579952E-2,1.1075765E-2,5.4989576E-2,1.3172704E-1,4.3493733E-1,5.3087175E-2,0E0,0E0,0E0,0E0,0E0,3.895402E-2,0E0,0E0,0E0,0E0,0E0,8.195257E-2,2.843659E-1,3.718426E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,23,23,27,27,28,28,29,29,30,30,31,31,32,32,38,38,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,1.5497297E1,1.3453049E3,4.5342876E3,9.996903E7,2.68E4,8.812601E-1,3.013037E6,7.4714734E-3,3.2384683E2,1.4539318E-1,1.5331E4,6.079755E0,3.1196967E-2,1.5545E4,1.3448806E8,-2.8590444E-2,8.2955E4,9.467578E6,2.1897722E10,-2.6513548E-3,1.2447058E-2,4.99297E-3,5.5566853E-1,-1.22154895E-2,5.4155197E-3,1.8623266E-2,3.7453184E-3,9.279E3,2.700379E2,2.0589474E1,6.5963456E5,6.2357716E-2,-1.0115377E-2,-2.9292125E-2,-1.2595921E-3,4.4371635E-3,-4.538515E-3,5.46562E5,-7.2706686E-3,-1.7502357E-3,-6.321306E-3,-2.0741828E-2,2.263406E-3,1.2705392E1,1.617E3,7.6760286E-1,5.577835E-3,1.9352233E-2,-1.7078329E-2,-6.95349E-3,-5.6405556E-3,-1.1703364E-2,4.25698E-4,-9.733608E-3,1.2807233E-2,8.560488E-4],"split_indices":[42,58,52,4,45,9,57,28,0,52,38,9,53,0,9,7,0,1,1,5,0,0,0,27,0,0,0,58,9,4,58,28,38,0,0,0,0,0,12,0,0,0,0,0,56,0,27,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.57E2,4.9E1,3.5E1,1.22E2,2.8E1,2.1E1,3.3E1,2E0,1.12E2,1E1,1.7E1,1.1E1,1.2E1,9E0,2.8E1,5E0,4E1,7.2E1,7E0,3E0,1.1E1,6E0,7E0,4E0,3E0,6E0,2.2E1,6E0,7E0,3.3E1,6.6E1,6E0,3E0,4E0,4E0,3E0,2E0,2E1,4E0,2E0,2E0,5E0,3E0,3E1,4.2E1,2.4E1,2E0,4E0,1.7E1,3E0,2.1E1,9E0,3.5E1,7E0,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.1936052E-2,-1.5182267E-1,1.5760238E-1,-1.214642E-1,-3.187172E-2,1.07080944E-1,3.2834876E-2,-1.8811886E-1,6.19821E-2,-8.8205084E-2,1.7667614E-1,-3.115475E-1,-1.0234942E-1,1.5685846E-1,-1.4203048E-1,-1.3068446E-2,-2.3196185E-2,-3.587127E-2,2.4979593E-1,-2.4225932E-1,-2.5490196E-2,-2.0880654E-2,-1.4597873E-1,3.4969583E-2,3.1181702E-1,1.3681039E-2,-1.3338024E-2,-9.279402E-2,4.1761484E-2,2.3741236E-2,-1.5669461E-2,9.474784E-2,3.6113718E-1,5.085498E-3,-2.820379E-1,-8.3250865E-2,8.308102E-2,-1.855153E-1,-1.1149166E-2,-5.6521293E-2,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,-2.4819752E-2,-8.349065E-3,6.2872446E-3,1.8276206E-4,-3.816804E-2,4.864078E-3,1.2524661E-1,-6.326331E-3,3.7593523E-1,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,-1,-1,-1,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.107609E0,1.8600852E0,2.331173E0,1.4911933E0,0E0,1.1708164E0,0E0,9.103031E-1,6.6102564E-1,2.6339516E-1,9.880699E-1,4.0248227E-1,1.8889087E-1,4.235136E-1,2.2991076E-1,0E0,8.525165E-2,2.97587E-1,7.771559E-1,2.8676605E-1,0E0,1.3659878E-1,1.8450612E-1,2.3734151E-1,9.613538E-2,8.857786E-3,0E0,4.3926656E-2,3.2743193E-2,7.4582696E-2,0E0,1.6022247E-1,5.321765E-2,5.2886583E-2,9.587407E-2,5.3622194E-2,1.212705E-1,8.459413E-2,9.3104824E-2,2.9008571E-2,0E0,0E0,0E0,0E0,0E0,2.897961E-2,0E0,0E0,0E0,5.3322386E-2,0E0,4.675585E-2,0E0,3.5073757E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,45,45,49,49,51,51,53,53],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,-1,-1,-1,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.0728035E-1,1.3917024E8,1.1038146E0,1.0648859E3,-3.187172E-2,4.2447827E2,3.2834876E-2,6.794883E4,2.2882867E5,2.5546576E4,1E0,2.47565E6,8.623692E0,3.851071E7,4.1808826E-1,-1.3068446E-2,2.3739047E-1,1.019731E0,1.0861458E3,2.9377524E2,-2.5490196E-2,3.8E1,2.4079119E1,2.0601392E3,3.353193E6,4E1,-1.3338024E-2,6.741443E-1,4.35E2,8.836347E7,-1.5669461E-2,4.6E1,4.8420677E-1,1.4416069E9,4.2E1,1.4178E4,7.47E2,3.8718798E6,8.2092184E7,4.83E9,1.0266547E-2,2.0700308E-2,8.0414545E-3,2.0834575E-3,-1.4463284E-3,9.576E3,-8.349065E-3,6.2872446E-3,1.8276206E-4,9.31E2,4.864078E-3,6.98247E5,-6.326331E-3,1E0,5.7154456E-3,-4.3637455E-3,4.776234E-3,-1.5839484E-2,-7.254232E-3,-6.5225866E-3,-8.151858E-5,7.688388E-3,-4.5947568E-3,-1.936269E-3,-1.0177594E-2,-5.187018E-3,4.210355E-3,9.670185E-4,-4.48214E-3,-3.7717258E-3,2.6131899E-3,3.8229923E-3,-4.026643E-3,3.807702E-3,9.472959E-3,1.9017018E-2,6.6647767E-3],"split_indices":[39,45,34,52,0,52,0,50,33,33,109,28,54,12,27,0,38,34,52,33,0,3,56,52,1,3,0,27,0,48,0,3,38,5,3,9,0,45,7,5,0,0,0,0,0,9,0,0,0,0,0,28,0,89,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.26E2,9.1E1,1.2E2,6E0,8.4E1,7E0,8.8E1,3.2E1,2.2E1,6.2E1,3.5E1,5.3E1,2.2E1,1E1,5E0,1.7E1,1.6E1,4.6E1,2.8E1,7E0,1.9E1,3.4E1,1.3E1,9E0,5E0,5E0,8E0,9E0,1.4E1,2E0,2E1,2.6E1,4E0,2.4E1,1.2E1,7E0,2.6E1,8E0,9E0,4E0,4E0,5E0,3E0,2E0,5E0,3E0,2E0,7E0,8E0,6E0,1.8E1,2E0,2.4E1,2E0,2E0,2E0,1.7E1,7E0,7E0,5E0,5E0,2E0,4E0,2.2E1,4E0,4E0,3E0,6E0,3E0,2E0,2E0,6E0,1.2E1,6E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-1.6992955E-2,-8.875833E-2,2.2156855E-1,-6.0390815E-2,-5.453842E-1,6.565264E-2,3.5614845E-1,-8.633612E-2,3.0628803E-1,-3.5136674E-2,-2.5970185E-1,1.033548E-2,1.3761056E-3,4.3397513E-3,3.885008E-1,-1.4680506E-1,-1.1059224E-2,2.477328E-2,1.7252595E-3,-9.932688E-4,-2.007134E-2,-5.6842774E-2,1.1758452E-1,2.6560768E-1,4.3975347E-1,-1.07489556E-1,-2.2993198E-2,-1.1149532E-1,1.1053811E-1,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,-2.6751718E-1,-5.8981713E-2,-4.527185E-2,-3.2934055E-1,1.45027E-2,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,29,-1,-1,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4779978E0,1.9877847E0,9.62826E-1,1.4344559E0,2.6657557E-1,2.1661456E-1,1.8128777E-1,6.3109255E-1,6.166359E-1,0E0,1.7431176E-1,0E0,1.2169773E-1,0E0,2.1767378E-2,9.350836E-1,7.817284E-1,0E0,4.614174E-2,0E0,0E0,3.312366E-2,1.0790542E-2,2.8682768E-2,3.3157587E-2,5.27603E-1,0E0,4.9001646E-1,6.9180393E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0392995E-2,2.4817911E-1,1.5262431E-1,9.5196724E-2,2.9514036E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,39,39,40,40,41,41,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,30,-1,-1,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5064244E0,1.4233672E8,8.57703E2,4.9192E4,2.849513E3,1.1244E4,1E0,6.978001E2,7.424212E-1,-3.5136674E-2,2.09E3,1.033548E-2,1.4872659E1,4.3397513E-3,2.8626094E5,4.462437E7,1E0,2.477328E-2,1.3174513E4,-9.932688E-4,-2.007134E-2,5.9857685E6,3.261056E8,1.2519051E7,3.007E3,1.142E3,-2.2993198E-2,3.4521E4,3.212485E11,-4.197916E-3,4.337853E-3,-4.928059E-3,2.167307E-4,8.100901E-3,2.507955E-3,1.4692348E-2,4.496399E-3,2.2878537E-2,8.358296E-3,4.543028E2,1.06403265E1,2.7577372E11,5.4656273E-1,2.4394053E9,1.7559739E-2,-1.4046805E-2,-3.8834924E-3,-4.8160978E-4,-7.3712687E-3,3.6298623E-3,-4.3204315E-3,-4.2046723E-3,-1.9268107E-2,5.511888E-3,-5.6127054E-3],"split_indices":[42,45,52,2,4,9,109,52,42,0,0,0,56,0,28,45,109,0,4,0,0,32,7,32,0,2,0,10,31,0,0,0,0,0,0,0,0,0,0,4,54,31,27,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.55E2,4.6E1,1.47E2,8E0,2.2E1,2.4E1,1.38E2,9E0,4E0,4E0,6E0,1.6E1,3E0,2.1E1,7.6E1,6.2E1,5E0,4E0,2E0,2E0,1.1E1,5E0,8E0,1.3E1,6.9E1,7E0,3.4E1,2.8E1,2E0,2E0,6E0,5E0,2E0,3E0,6E0,2E0,1.1E1,2E0,1.5E1,5.4E1,2.7E1,7E0,2.1E1,7E0,1.3E1,2E0,3.6E1,1.8E1,7E0,2E1,2E0,5E0,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.02255E-2,-6.558096E-2,3.4569636E-1,-1.2843427E-2,-4.0469694E-1,5.4956263E-1,4.3343384E-2,-1.403701E-1,1.0433215E-1,2.2149876E-2,-4.8705584E-1,6.838387E-1,3.7301254E-1,1.4347048E-1,-1.6691085E-2,-2.8438172E-1,-7.497648E-2,3.6531553E-2,2.410501E-1,-3.8605474E-3,5.6571406E-3,-5.657334E-1,-2.4888176E-1,9.962667E-3,7.5234216E-1,6.084204E-3,1.9794445E-2,7.620034E-2,1.50580425E-2,-3.147017E-1,-2.6331637E-3,-3.2688044E-2,-1.6758075E-1,3.0311698E-3,1.3236392E-1,-3.7357313E-4,2.712043E-1,-6.3281095E-1,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,2.171496E-3,8.731073E-3,-1.6129533E-2,-5.054191E-3,2.5078287E-2,-1.376335E-1,-2.738632E-3,-1.0184393E-2,6.306253E-2,-5.9866663E-2,2.3515062E-1,-2.9690387E-3,1.6760947E-2,1.7714213E-1,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,-1,41,-1,-1,43,-1,45,-1,47,49,51,53,-1,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,63,-1,-1,65,67,69,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7954755E0,3.3212526E0,2.565043E0,2.4354134E0,8.896537E-1,3.3758068E-1,7.454521E-1,7.132385E-1,7.791066E-1,5.6962486E-2,2.5271702E-1,2.7266836E-1,5.7875514E-2,1.438675E-1,0E0,1.5053928E-1,2.0990708E-1,1.8580317E-1,2.1208572E-1,0E0,0E0,2.0361853E-1,6.189382E-2,0E0,4.8636913E-2,0E0,0E0,9.129316E-2,0E0,3.965044E-2,0E0,2.3988187E-1,6.710768E-2,1.699124E-1,3.1248683E-1,0E0,1.11125946E-1,2.6826096E-1,0E0,0E0,0E0,0E0,0E0,3.272756E-2,0E0,0E0,0E0,2.1187598E-1,8.620715E-2,0E0,0E0,1.1212682E-1,1.685599E-1,7.086277E-2,0E0,0E0,4.61289E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,24,24,27,27,29,29,31,31,32,32,33,33,34,34,36,36,37,37,43,43,47,47,48,48,51,51,52,52,53,53,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,-1,42,-1,-1,44,-1,46,-1,48,50,52,54,-1,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,64,-1,-1,66,68,70,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6499896E9,6.627576E7,3.7763092E2,3.3478114E-1,8E0,5.7006E4,1.278E3,1.5022863E0,1.776E3,9.006135E0,2.6E1,1.625E3,8.129243E5,-1.6691085E-2,4.5715973E4,5.531948E8,1.8331E4,7.555544E2,-3.8605474E-3,5.6571406E-3,1.2728E4,1.2187347E3,9.962667E-3,2.2047469E4,6.084204E-3,1.9794445E-2,1.0252E4,1.50580425E-2,3.4322312E-1,-2.6331637E-3,7.269755E0,1.2421726E7,4.278E3,2.4794E4,-3.7357313E-4,1.5489E4,8.260109E9,-1.0855893E-2,-1.5139293E-2,-3.0296221E-3,3.967288E-2,1.5232022E-2,4E0,8.731073E-3,-1.6129533E-2,-5.054191E-3,5.1E2,2.037744E10,-2.738632E-3,-1.0184393E-2,4.541179E1,5.382E3,5.586E3,-2.9690387E-3,1.6760947E-2,7.4353185E0,-8.7453835E-3,-3.4332376E-2,2.1692228E-3,-4.056635E-3,4.754953E-3,-4.031546E-3,-8.69389E-3,4.6473034E-4,4.676335E-3,-3.6525405E-3,-7.5851143E-3,9.04169E-4,5.741595E-3,1.5997974E-2,4.914819E-3,1.2125476E-2],"split_indices":[52,5,45,52,27,8,10,2,41,0,54,3,0,28,0,33,5,10,4,0,0,9,4,0,4,0,0,9,0,38,0,53,45,2,9,0,9,5,0,0,0,0,0,8,0,0,0,0,31,0,0,56,2,2,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.85E2,4.1E1,1.61E2,2.4E1,2.4E1,1.7E1,7.7E1,8.4E1,4E0,2E1,1.2E1,1.2E1,1.4E1,3E0,2.3E1,5.4E1,5.7E1,2.7E1,2E0,2E0,1.4E1,6E0,2E0,1E1,2E0,1E1,1.1E1,3E0,2E1,3E0,3.8E1,1.6E1,4.3E1,1.4E1,3E0,2.4E1,1.1E1,3E0,4E0,2E0,8E0,2E0,7E0,4E0,1.8E1,2E0,2.5E1,1.3E1,5E0,1.1E1,2.2E1,2.1E1,9E0,5E0,1.2E1,1.2E1,2E0,9E0,5E0,2E0,1.5E1,1E1,1E1,3E0,1.8E1,4E0,9E0,1.2E1,5E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-5.735387E-3,-6.410505E-2,2.797867E-1,-1.760073E-2,-3.442871E-1,5.04675E-1,2.686096E-2,-1.2676698E-1,1.1911592E-1,-4.2207205E-1,5.1267102E-2,6.152395E-1,1.3419471E-2,1.5410598E-1,-1.9511065E-1,-2.5507975E-1,-8.848773E-2,2.6512688E-1,4.352096E-2,-3.2537884E-1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,2.4225692E-1,2.5578894E-2,-4.040385E-2,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,-4.242069E-2,-1.8737647E-1,3.087647E-1,4.6277814E-2,-5.1333107E-2,1.4060691E-1,-3.4747878E-3,-3.5448268E-1,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.1831463E-3,-1.2673235E-1,-2.0977704E-1,8.250273E-4,2.3295416E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,5.527623E-2,-1.189838E-1,1.975704E-1,-1.7819712E-2,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,45,47,49,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,65,-1,-1,-1,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5344255E0,2.287163E0,2.0341249E0,2.2836764E0,7.9084253E-1,3.1127357E-1,5.384738E-1,3.9492452E-1,7.390245E-1,2.7931428E-1,7.600727E-2,2.1988153E-1,0E0,1.2976074E-1,1.931273E-1,6.892848E-2,2.9727066E-1,2.0536327E-1,4.308317E-1,1.00349784E-1,0E0,0E0,0E0,0E0,0E0,8.028874E-2,1.2697376E-2,1.2392008E-2,0E0,0E0,0E0,1.8929411E-1,9.967518E-2,7.952249E-2,1.2078894E-2,1.7966965E-1,2.110849E-1,0E0,1.2337124E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1306114E-1,7.93834E-2,1.05109274E-1,0E0,1.7682552E-2,0E0,0E0,0E0,2.9296512E-2,1.0169087E-1,1.4039552E-1,1.7305832E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,27,27,31,31,32,32,33,33,34,34,35,35,36,36,38,38,45,45,46,46,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,46,48,50,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,66,-1,-1,-1,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.6599895E9,6.627576E7,5.242538E2,1.5593E4,8.812601E-1,1.3917024E8,1.056E3,1.26881E5,1.3917024E8,3.729811E6,8E0,1.3419471E-2,5.1E1,2.565E3,7.107292E6,5.4127317E10,6.3300834E0,1.5181E4,1.9E1,-3.2432023E-2,-3.476694E-3,7.6350067E-3,3.8638406E-2,1.8209685E-2,1.739E3,7.7573473E9,1.3174513E4,-1.8878609E-2,-8.580401E-3,-1.6612234E-2,1.0466548E1,1E0,4.361E3,3.098E3,5.7277904E0,3.2395E4,-3.4747878E-3,3.20084E6,1.5151722E-2,2.2566218E-3,-1.3786369E-3,2.9012284E-3,6.913387E-4,-3.9685247E-3,5.998024E-1,2.124077E1,7.2899837E0,8.250273E-4,9.607843E-1,2.1130733E-2,-4.6251816E-4,4.216146E-3,2.7060036E11,5.36E3,1.7790995E0,4.0953446E8,-3.7352948E-3,-1.9043483E-2,2.7285856E-3,-6.1042076E-3,-1.8814462E-3,-8.979784E-3,-1.3104318E-2,-4.590143E-3,1.2223603E-2,4.011823E-3,4.84449E-3,-4.331928E-4,-8.624007E-3,-1.0295074E-4,1.4480397E-2,4.761265E-3,-3.7043525E-3,1.0085007E-3],"split_indices":[52,5,45,52,9,57,45,2,29,45,1,8,0,3,0,45,31,53,10,3,0,0,0,0,0,0,5,4,0,0,0,54,112,2,2,53,9,0,7,0,0,0,0,0,0,57,58,53,0,57,0,0,0,31,2,57,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.75E2,3.5E1,1.51E2,2.4E1,1.8E1,1.7E1,8.4E1,6.7E1,2E1,4E0,1.1E1,7E0,1.1E1,6E0,1.8E1,6.6E1,2.2E1,4.5E1,1.6E1,4E0,2E0,2E0,5E0,6E0,6E0,5E0,4E0,2E0,1.1E1,7E0,4.6E1,2E1,1.8E1,4E0,2.3E1,2.2E1,2E0,1.4E1,4E0,2E0,2E0,3E0,2E0,2E0,3E1,1.6E1,1.8E1,2E0,1.3E1,5E0,2E0,2E0,9E0,1.4E1,1.6E1,6E0,2E0,1.2E1,2.2E1,8E0,7E0,9E0,1.1E1,7E0,1.1E1,2E0,5E0,4E0,9E0,5E0,7E0,9E0,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-2.2588685E-2,-1.4545202E-1,1.1374607E-1,-8.6952515E-2,-3.9759496E-1,-3.064367E-2,2.3289914E-1,-1.2433577E-1,1.7574582E-1,-5.7842267E-1,-9.222219E-2,-6.9834334E-3,-1.8266106E-2,6.878721E-2,3.451957E-1,-2.7263525E-1,-8.135537E-2,-5.932932E-4,1.1667584E-2,-6.281898E-1,-9.885883E-3,-1.32481875E-2,1.0622433E-1,-2.1569806E-1,3.714026E-2,-4.582277E-2,1.5392454E-1,2.7393732E-2,4.457966E-1,-1.735199E-1,-1.8242102E-2,-2.038975E-1,-3.7180442E-2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,-3.8278066E-2,8.7990396E-2,-4.001707E-3,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,-1.1330966E-1,2.5172912E-2,3.078277E-1,-2.0457634E-1,-1.095509E-3,-1.3675371E-1,-1.4804438E-2,3.1190168E-2,-1.298265E-1,-8.181965E-2,8.646828E-3,-2.6838033E-4,1.4474735E-1,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,61,-1,63,65,-1,67,-1,69,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6011605E0,1.6322045E0,1.7619938E0,9.3515986E-1,1.1062734E0,3.806838E-1,1.0082908E0,5.004853E-1,1.4725864E-1,1.1519289E-1,3.504039E-1,4.224278E-1,0E0,2.4123535E-1,1.0428362E0,1.1729789E-1,3.442828E-1,0E0,0E0,1.5132141E-1,0E0,0E0,1.4618695E-2,8.2455575E-2,1.4909932E-1,5.0066087E-2,1.4621168E-1,4.4345292E-1,8.724737E-2,4.6982825E-2,0E0,7.2891474E-2,3.1442845E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6498463E-1,1.1525272E-1,1.6504118E-2,0E0,0E0,0E0,0E0,9.535315E-2,0E0,1.2730312E-1,2.6230484E-2,0E0,1.0405913E-2,0E0,2.094729E-1,1.0832605E-1,9.527324E-2,0E0,3.1808536E-2,1.478824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,39,39,40,40,41,41,46,46,48,48,49,49,51,51,53,53,54,54,55,55,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,62,-1,64,66,-1,68,-1,70,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3905947E-1,1.4446067E10,4.4589956E5,1E0,1.5461028E12,1.4146589E8,1.2838955E3,6.639351E2,4.22631E5,8.92851E6,1.244125E3,2.853839E4,-1.8266106E-2,1.3012535E1,1E0,8.3949E4,4.7095413E2,-5.932932E-4,1.1667584E-2,6.1826086E0,-9.885883E-3,-1.32481875E-2,1.5011E4,1.338E3,1.0176675E0,1.1350955E0,6.695E3,1E0,3.1E1,1E0,-1.8242102E-2,7.1399055E6,7.11E2,-3.513965E-2,-1.37930475E-2,1.2286393E-3,7.3872944E-3,-1.2584665E-3,-1.3367111E-2,4.8008E4,1.2364354E3,4.1376923E2,-7.5925346E-3,9.569238E-3,-3.3191377E-3,1.6866349E-2,2.5591E4,2.5172912E-2,5.86883E6,1.4008755E2,-1.095509E-3,6.1E1,-1.4804438E-2,6.38E2,9.388849E-1,6.8992513E-1,8.646828E-3,2.7953E4,1.1946838E7,-1.4701461E-3,2.8460945E-3,-4.7384037E-4,-1.2077148E-2,2.4715671E-3,1.7486108E-2,-3.0180616E-3,-1.1507121E-2,-7.3731127E-3,-2.0442237E-3,-3.5540518E-4,1.0580931E-2,-7.4852505E-3,3.1792286E-3,-1.3020845E-3,-1.0355819E-2,1.5907116E-3,-4.285435E-3,7.978947E-3,2.713082E-3],"split_indices":[39,5,28,94,31,45,52,33,28,1,33,33,0,56,109,12,4,0,0,53,0,0,9,2,39,57,2,8,3,112,0,45,0,0,0,0,0,0,0,2,4,52,0,0,0,0,2,0,1,33,0,3,0,0,27,27,0,9,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.12E2,1.01E2,9.2E1,2E1,4.6E1,5.5E1,8.1E1,1.1E1,1.2E1,8E0,4.4E1,2E0,2.3E1,3.2E1,1.7E1,6.4E1,3E0,8E0,1E1,2E0,4E0,4E0,7E0,3.7E1,1E1,1.3E1,8E0,2.4E1,1E1,7E0,1.6E1,4.8E1,7E0,3E0,2E0,2E0,2E0,5E0,1.5E1,2.2E1,8E0,2E0,1.1E1,2E0,2E0,6E0,1.4E1,1E1,8E0,2E0,1.1E1,5E0,2.8E1,2E1,1.3E1,2E0,9E0,1.3E1,6E0,2E0,4E0,2E0,2E0,8E0,2E0,6E0,9E0,2E0,2.4E1,4E0,1.8E1,2E0,1E1,3E0,7E0,2E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.9344994E-3,-8.175323E-2,2.3140544E-1,-4.4790298E-2,-4.8490012E-1,2.5693094E-2,3.847721E-1,-9.1722965E-2,4.259129E-2,-2.0991474E-1,-3.360973E-2,2.7427834E-2,-5.6275822E-2,3.184738E-2,3.1256717E-1,-1.0761265E-1,1.2113636E-1,-2.8683331E-3,2.0815976E-1,-1.3282049E-2,-1.7001278E-3,7.0364326E-2,-1.4656629E-1,4.4856225E-3,1.5982723E-2,-2.2537349E-1,-8.0327906E-2,8.497628E-3,1.8191043E-3,-6.4350456E-2,7.170837E-2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,-3.6205932E-2,-4.63827E-3,-1.3423674E-2,5.86282E-3,-1.1076284E-1,1.3306548E-2,-1.0665464E-1,-2.4310345E-2,1.2974514E-1,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,-1,-1,33,35,-1,-1,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6567056E0,2.3029141E0,1.5360355E0,5.94458E-1,5.9041786E-1,1.0017112E0,3.4057522E-1,3.3096892E-1,3.8445777E-1,6.7638606E-2,0E0,0E0,2.3900099E-1,0E0,5.3549767E-2,2.6464677E-1,2.4824701E-2,1.9260496E-1,3.0953288E-2,0E0,0E0,8.459512E-2,1.8438596E-1,0E0,0E0,7.871467E-2,1.9214371E-1,0E0,0E0,7.697916E-2,1.0903385E-1,0E0,0E0,0E0,0E0,0E0,4.2805668E-2,0E0,0E0,1.03736766E-1,1.2159997E-1,6.966427E-2,5.4593235E-2,3.1910963E-2,5.4574564E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26,29,29,30,30,36,36,39,39,40,40,41,41,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,-1,-1,34,36,-1,-1,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.8000048E10,1E0,2.4277832E6,6.0671224E7,1.7727281E2,1.2117E4,1E0,8.373168E5,5.441E3,-3.360973E-2,2.7427834E-2,3.1047E4,3.184738E-2,5.8E2,6.554877E-4,3.5E1,1.0415988E0,1.521636E5,-1.3282049E-2,-1.7001278E-3,3.5073215E3,5.785788E-1,4.4856225E-3,1.5982723E-2,3E2,3.7504044E5,8.497628E-3,1.8191043E-3,2E0,8.3154816E2,1.1927086E-2,4.004414E-3,-6.6781894E-4,8.706586E-3,-1.4299641E-2,6.804E3,-4.63827E-3,-1.3423674E-2,1.2599119E-3,5.948464E0,2.7954546E-1,2.1845242E3,8.412314E0,1.579932E5,-3.9715716E-3,3.2444396E-3,-4.089218E-3,2.9553997E-3,-9.804311E-4,-6.6270907E-3,-2.7375359E-3,5.563364E-3,-6.9615273E-3,-2.5614531E-4,1.0142556E-3,-5.1834392E-3,8.654046E-3,1.3076891E-3],"split_indices":[4,5,109,48,45,33,9,94,28,2,0,0,10,0,0,39,3,39,33,0,0,52,42,0,0,0,48,0,0,8,4,0,0,0,0,0,9,0,0,39,53,57,4,54,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.55E2,4.8E1,1.43E2,1.2E1,2.1E1,2.7E1,9.3E1,5E1,6E0,6E0,2E0,1.9E1,4E0,2.3E1,8.7E1,6E0,4E1,1E1,4E0,2E0,8E0,1.1E1,2E0,2.1E1,1.5E1,7.2E1,3E0,3E0,2.2E1,1.8E1,7E0,3E0,5E0,3E0,4E0,7E0,5E0,1E1,1.9E1,5.3E1,8E0,1.4E1,7E0,1.1E1,5E0,2E0,7E0,1.2E1,1.2E1,4.1E1,5E0,3E0,1E1,4E0,5E0,2E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.2111462E-2,-1.5450773E-1,1.031491E-1,-1.09187275E-1,-4.5132187E-1,-2.8939374E-2,1.8243721E-1,-2.6891872E-1,-6.3274145E-2,-2.8736094E-2,-2.1376169E-1,-1.9380113E-1,1.3079939E-2,-5.3318623E-2,2.8723615E-1,-3.048581E-1,-3.2565857E-3,-2.1935193E-1,-2.3797065E-2,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,-1.15509786E-1,4.264821E-2,1.4384617E-1,-1.3136774E-1,1.7543119E-1,4.10858E-1,-3.4710836E-1,-7.3455116E-3,-2.187251E-3,-2.6718003E-1,-4.2986523E-2,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.1813E-2,-8.5979775E-2,1.9637176E-3,1.3061595E-2,-5.950475E-2,-2.1748714E-2,2.684922E-1,1.24782175E-1,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.991617E-2,-1.04395926E-1,8.574969E-3,4.3539573E-2,-5.5793906E-3,-9.2947256E-4,-9.633975E-2,3.2059713E-3,4.982372E-3,1.4664561E-2,1.5088551E-1,3.8726836E-2,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,-1,53,55,-1,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,63,65,-1,67,-1,-1,69,-1,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4133608E0,1.2017343E0,1.2096798E0,5.8435357E-1,2.704537E-1,3.0733958E-1,1.7942166E0,1.1027694E-1,3.9527723E-1,0E0,1.3330251E-2,9.3468994E-2,1.4180462E-1,3.728331E-1,6.189995E-1,4.515779E-2,0E0,9.6432924E-2,2.5955042E-1,0E0,0E0,0E0,0E0,1.570262E-2,1.1871672E-1,7.939862E-2,3.5887626E-1,9.847349E-2,1.3197088E-1,3.6898494E-2,0E0,0E0,1.9970298E-2,2.5871733E-1,0E0,0E0,0E0,6.611198E-2,9.313557E-3,0E0,0E0,7.5622745E-2,0E0,1.8241048E-2,3.9083302E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.800409E-1,1.35196E-1,0E0,3.6446035E-2,0E0,0E0,6.822489E-2,0E0,0E0,0E0,3.4981847E-2,1.2126244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,23,23,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,37,37,38,38,41,41,43,43,44,44,51,51,52,52,54,54,57,57,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,-1,54,56,-1,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,64,66,-1,68,-1,-1,70,-1,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8601307E-2,1.8000048E10,6.787619E2,1.142E3,1.527638E12,2.853839E4,1E0,1.3448806E8,4.059859E-5,-2.8736094E-2,2.906E3,3.6E1,1.7218965E2,2E0,1.3173E4,2.23695E5,-3.2565857E-3,1.7727281E2,3.3E1,-4.050389E-3,-1.256674E-2,-1.29712615E-2,-1.7642345E-3,3.0345E4,4.6E1,4.078932E5,8.476884E-1,9.686559E6,4.747723E6,3.7790768E2,-7.3455116E-3,-2.187251E-3,6.802721E-3,4.454691E5,1.0804944E-2,-7.124008E-3,-1.243371E-3,7.751E3,1.251E3,1.9637176E-3,1.3061595E-2,1.257013E0,-2.1748714E-2,8.366892E4,8.442676E0,4.9170884E-3,2.1196842E-2,-7.858449E-3,-1.9298002E-2,-1.5722835E-2,-7.138E-3,3.5E1,5.979E3,8.574969E-3,2.702E3,-5.5793906E-3,-9.2947256E-4,2.012E3,3.2059713E-3,4.982372E-3,1.4664561E-2,7.595543E-1,1.3235667E9,6.3743466E-3,-2.2817587E-3,-7.1317567E-3,2.7001402E-4,4.8852706E-4,4.707233E-3,-2.8777638E-4,-7.7907853E-3,9.120767E-3,3.1563747E-3,-8.443057E-4,3.4602752E-3],"split_indices":[38,5,52,2,31,33,109,7,38,0,0,3,52,8,2,12,0,33,8,0,0,0,0,9,3,28,34,51,32,33,0,0,57,48,0,0,0,9,0,0,0,39,0,33,54,0,0,0,0,0,0,3,2,0,2,0,0,0,0,0,0,27,5,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,9.2E1,1.14E2,8.1E1,1.1E1,4.3E1,7.1E1,1.7E1,6.4E1,6E0,5E0,8E0,3.5E1,2.2E1,4.9E1,1.4E1,3E0,1.2E1,5.2E1,2E0,3E0,5E0,3E0,6E0,2.9E1,6E0,1.6E1,2.7E1,2.2E1,1E1,4E0,3E0,9E0,4.9E1,3E0,4E0,2E0,2.4E1,5E0,4E0,2E0,1.4E1,2E0,8E0,1.9E1,2E0,2E1,3E0,7E0,5E0,4E0,2.1E1,2.8E1,4E0,2E1,3E0,2E0,1.1E1,3E0,2E0,6E0,1.4E1,5E0,1E1,1.1E1,2E1,8E0,1.3E1,7E0,5E0,6E0,9E0,5E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.2747186E-3,-5.4162722E-2,1.8111008E-1,-3.108007E-2,-4.4845343E-1,7.1374275E-2,3.275183E-1,-9.3600295E-2,1.1067671E-1,-2.5658727E-2,-6.88157E-3,1.2687185E-1,-2.088327E-2,2.448392E-3,3.571717E-1,-2.3487736E-1,-5.843745E-2,1.925611E-1,-3.9538275E-2,9.78743E-3,3.3138932E-3,-7.214252E-2,6.806273E-3,1.9431083E-2,1.0387621E-2,-1.5502349E-1,-1.7014932E-2,5.5956807E-5,-1.2483689E-1,2.5137913E-1,-3.1633927E-3,-1.7023765E-1,8.103208E-2,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,2.859666E-2,-1.6140592E-1,6.8674185E-3,1.9025224E-1,2.8979616E-2,-1.5382758E-2,-9.495701E-4,-2.327769E-2,1.6913544E-1,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9040357E0,1.3148481E0,6.996037E-1,1.2422698E0,1.18970394E-1,1.4089137E-1,1.3821435E-1,4.6814162E-1,5.365249E-1,0E0,0E0,6.0488462E-2,1.007224E-1,0E0,1.7639875E-2,1.198957E-1,3.0683744E-1,4.4052112E-1,2.659302E-1,0E0,0E0,2.7664423E-2,0E0,0E0,0E0,2.1903068E-2,0E0,2.2701421E-1,1.7931145E-1,3.7047946E-1,0E0,1.6969696E-1,8.6647436E-2,0E0,0E0,0E0,0E0,0E0,1.13961965E-1,6.886476E-2,1.559542E-2,2.4640507E-1,0E0,0E0,0E0,1.2072772E-2,1.1905044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,25,25,27,27,28,28,29,29,31,31,32,32,38,38,39,39,40,40,41,41,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4436649E0,1.4233672E8,1.4008022E11,1.0648859E3,3.7284137E2,2.7883E4,1E0,6.126268E0,2.249985E5,-2.5658727E-2,-6.88157E-3,3.5037E4,5.53839E5,2.448392E-3,2.7E1,7.0712406E5,7.269755E0,9.040637E7,9.875E3,9.78743E-3,3.3138932E-3,1.009E3,6.806273E-3,1.9431083E-2,1.0387621E-2,2.44412E3,-1.7014932E-2,1.053E3,2.1102592E8,5.902126E3,-3.1633927E-3,4.6E1,3.858136E6,-4.97187E-3,1.0682712E-3,-3.3360675E-3,-9.044156E-3,-8.794569E-3,1E0,7.3376025E6,1.9519106E-1,2.8880222E6,2.8979616E-2,-1.5382758E-2,-9.495701E-4,4.895358E8,8.51E2,3.4365128E-4,8.240074E-3,-9.09646E-3,-2.6539892E-3,3.2405166E-3,-9.5909066E-4,2.8134463E-3,1.3858536E-2,-3.1675037E-3,1.279433E-3,3.103089E-3,1.0615611E-2],"split_indices":[42,45,31,52,58,9,109,54,33,0,0,29,28,0,8,28,53,45,9,0,0,0,0,0,0,47,0,2,7,4,0,3,47,0,0,0,0,0,94,32,39,32,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.89E2,1.45E2,4.4E1,1.38E2,7E0,2.6E1,1.8E1,9.6E1,4.2E1,5E0,2E0,1.6E1,1E1,2E0,1.6E1,1.8E1,7.8E1,2.7E1,1.5E1,6E0,1E1,8E0,2E0,1.1E1,5E0,1.2E1,6E0,4.2E1,3.6E1,2.2E1,5E0,7E0,8E0,6E0,2E0,4E0,8E0,5E0,3.7E1,2.8E1,8E0,2E1,2E0,3E0,4E0,4E0,4E0,3.3E1,4E0,2.2E1,6E0,2E0,6E0,9E0,1.1E1,2E0,2E0,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.826838E-3,-6.721812E-2,1.9046006E-1,-7.924094E-2,2.6099376E-2,5.0246296E-3,2.6243427E-1,-2.5692692E-1,-4.875577E-2,-3.755318E-2,3.6572933E-3,2.1474557E-1,1.9588318E-2,-4.4567264E-3,-2.904459E-1,-3.542672E-2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,2.688706E-1,1.3754945E-1,-4.928484E-3,-3.0891788E-1,-1.2508754E-1,4.5123346E-2,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,-1.1071097E-1,1.8496023E-2,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,-1,21,23,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,-1,-1],"loss_changes":[2.477284E0,1.1091111E0,6.6600955E-1,7.729075E-1,0E0,4.7609035E-2,1.5403104E-1,8.979154E-2,6.413361E-1,3.7017375E-2,0E0,8.418608E-2,0E0,0E0,2.8120995E-2,8.8675094E-1,0E0,0E0,0E0,1.0490298E-2,3.518188E-2,0E0,1.5468597E-2,1.0648471E-1,4.4222242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2309903E-1,2.3909211E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,14,14,15,15,19,19,20,20,22,22,23,23,24,24,32,32,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,-1,22,24,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,7.8040985E2,5.5200674E3,2.6099376E-2,6.2958473E4,1.2790948E0,5.4E0,1.4146589E8,4.71E2,3.6572933E-3,9.6494156E-1,1.9588318E-2,-4.4567264E-3,4.146E3,5.3287085E2,-2.1564234E-2,-4.807155E-3,9.5999654E-4,7.810231E4,1.5593E4,-4.928484E-3,1.2187347E3,6.35E2,1.2241036E12,7.1910643E-3,1.4450356E-2,2.7101396E-3,8.6061945E-3,-1.5987605E-2,-5.890048E-3,-1.4185957E-2,4.2E1,2.122531E6,1.4788565E-2,-6.195292E-3,1.0334674E-3,3.1753597E-3,-3.1741308E-3],"split_indices":[42,56,4,48,0,33,35,54,45,0,0,57,0,0,29,52,0,0,0,33,9,0,4,2,31,0,0,0,0,0,0,0,3,47,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.46E2,4.9E1,1.44E2,2E0,1.4E1,3.5E1,2E1,1.24E2,9E0,5E0,2.8E1,7E0,4E0,1.6E1,1.21E2,3E0,4E0,5E0,1.5E1,1.3E1,2E0,1.4E1,5.7E1,6.4E1,4E0,1.1E1,5E0,8E0,1.2E1,2E0,3E0,5.4E1,5.9E1,5E0,4.8E1,6E0,3.8E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.4262974E-3,-8.005432E-2,1.4594802E-1,-2.888352E-1,-4.6757724E-2,-1.8107321E-2,2.4755892E-1,-4.6657827E-3,-3.4083346E-1,-1.0410578E-1,1.8106598E-2,2.52526E-2,-7.499621E-2,7.3558204E-2,3.0511466E-1,-1.0108946E-2,-1.9372413E-2,-1.6241044E-1,-4.8085254E-2,6.1593957E-2,-1.17093466E-1,8.193563E-2,-1.12674244E-1,-1.9586612E-3,1.11724325E-1,3.5032213E-1,1.3559058E-1,-6.2673695E-2,-2.1433787E-1,7.613574E-2,-8.7488696E-2,8.202132E-2,-6.9475058E-3,-5.7769872E-2,-1.4384934E-2,7.815317E-5,7.870827E-3,-1.6748264E-1,-8.807537E-3,7.7555277E-3,2.0889724E-5,3.6501428E-1,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,-3.6418545E-3,-1.605696E-1,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,-1.1916163E-1,1.1715029E-1,1.5941553E-2,1.3056229E-3,-9.284757E-2,-2.2564225E-1,-2.0749527E-3,-2.448091E-3,2.7077156E-3,3.8617578E-1,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,53,-1,55,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,-1,63,65,-1,-1,-1,-1,67,69,71,-1,73,75,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4140606E0,8.554564E-1,1.2713884E0,1.3789642E-1,4.0734783E-1,9.553926E-1,4.482491E-1,0E0,1.6353011E-2,1.8163341E-1,3.1294593E-1,0E0,1.7479275E-1,6.096188E-2,2.2073102E-1,0E0,0E0,1.3158071E-1,1.5839781E-1,1.786837E-1,1.20575935E-1,3.8199473E-2,1.2945855E-1,0E0,5.294875E-2,5.433297E-2,3.3570766E-2,4.5042265E-2,5.2512467E-2,7.294734E-2,5.8373034E-2,8.402106E-2,0E0,3.5133395E-2,0E0,0E0,0E0,9.929475E-2,2.6866937E-2,0E0,0E0,4.9077034E-2,0E0,0E0,0E0,0E0,2.0661239E-2,1.6366512E-2,0E0,0E0,0E0,0E0,5.8940127E-2,1.0018352E-1,5.5888433E-2,0E0,2.880384E-2,6.1261773E-2,0E0,0E0,0E0,1.6714334E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,37,37,38,38,41,41,46,46,47,47,52,52,53,53,54,54,56,56,57,57,61,61],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,54,-1,56,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,-1,64,66,-1,-1,-1,-1,68,70,72,-1,74,76,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.6770158E-4,1E0,6.051637E-1,3.3030225E2,6.948189E7,6.201765E5,-4.6657827E-3,1.57525425E1,2.0562818E4,9.959017E0,2.52526E-2,9.31E2,1.0363E4,5.03433E0,-1.0108946E-2,-1.9372413E-2,1.0363E4,2.0589474E1,6.579211E9,3.2776688E7,8.319244E8,1.5574268E7,-1.9586612E-3,3.4E1,5.2E1,1.5E1,5.3943863E0,3.62043E1,9.075745E7,5.948464E0,4.9212202E-1,-6.9475058E-3,2E1,-1.4384934E-2,7.815317E-5,7.870827E-3,5.840393E-1,6.804E3,7.7555277E-3,2.0889724E-5,1.0528413E10,5.4075024E-3,8.751887E-3,1.7192704E-3,-6.4621023E-3,6.948189E7,1.2421726E7,-1.5671642E-2,-7.613253E-4,8.3621675E-3,-4.5004938E-4,2E0,1.4265206E1,2.1150263E-1,1.3056229E-3,1.5996247E1,2.6023E4,-2.0749527E-3,-2.448091E-3,2.7077156E-3,1.4456249E7,7.658756E-3,3.0387286E-3,-2.0713676E-3,-4.929973E-3,-9.994446E-3,-7.126225E-3,2.953041E-4,2.5684154E-3,9.170282E-3,3.154728E-3,-2.9218486E-3,3.6448237E-4,-6.206989E-3,-2.1248057E-3,-1.2929704E-2,1.3357873E-2,2.152961E-2],"split_indices":[52,39,109,27,52,7,48,0,56,33,54,0,0,9,57,0,0,9,58,5,32,12,1,0,3,8,8,53,58,7,53,57,0,3,0,0,0,42,9,0,0,12,0,0,0,0,7,45,0,0,0,0,8,56,38,0,58,10,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.24E2,7.5E1,1.6E1,1.08E2,2.9E1,4.6E1,4E0,1.2E1,5.7E1,5.1E1,2E0,2.7E1,1.2E1,3.4E1,5E0,7E0,2.7E1,3E1,3.9E1,1.2E1,5E0,2.2E1,3E0,9E0,2.6E1,8E0,1E1,1.7E1,7E0,2.3E1,3.6E1,3E0,1E1,2E0,3E0,2E0,1.4E1,8E0,6E0,3E0,2.4E1,2E0,5E0,3E0,4E0,6E0,1.3E1,4E0,4E0,3E0,7E0,1.6E1,2.3E1,1.3E1,3E0,7E0,9E0,5E0,5E0,3E0,2.1E1,3E0,2E0,4E0,7E0,6E0,1.3E1,3E0,1.3E1,1E1,8E0,5E0,2E0,5E0,2E0,7E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.6450368E-2,-1.1724766E-1,1.2559639E-1,-7.511801E-2,-3.363559E-1,1.8126132E-1,-2.3499241E-1,-1.6364163E-1,-1.0940317E-2,-1.8125798E-1,-4.375745E-1,1.2667201E-1,3.6933804E-1,-4.659123E-2,-2.6914816E-2,-2.2536351E-1,-1.00311846E-1,7.6694086E-2,-4.540101E-2,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,-1.531599E-2,2.0203556E-1,4.1224214E-1,2.9349497E-3,1.9489948E-3,-3.9977366E-3,-2.5541717E-1,2.9264723E-3,1.9217292E-2,-1.2399441E-1,1.2778491E-1,-2.9145163E-2,-7.644804E-2,6.597971E-2,5.250523E-2,-1.4877042E-1,2.3447502E-1,7.5913966E-4,2.1494467E-2,8.123851E-3,-2.7855477E-1,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,-7.0292644E-2,7.10427E-2,8.864286E-3,-2.6523983E-3,1.1110839E-3,1.0315416E-2,-1.1217816E-1,-7.6833105E-4,1.1495934E-1,9.3826205E-2,-1.8227108E-2,-1.1465208E-2,-1.2050162E-3,2.9752174E-1,1.3666672E-1,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,-1,-1,65,-1,-1,-1,-1,67,69,-1,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.913867E0,1.1932194E0,1.6282953E0,6.345087E-1,2.5036097E-1,6.616366E-1,6.333728E-1,1.611085E-1,2.0279819E-1,5.5682838E-2,9.7411156E-2,5.916339E-1,1.740191E-1,2.7933039E-2,0E0,2.1270418E-1,7.315171E-2,1.06465176E-1,1.7102996E-1,0E0,0E0,0E0,0E0,1.8883207E-1,2.3487675E-1,2.0117044E-2,0E0,0E0,0E0,1.0595679E-1,0E0,2.1876162E-2,3.615102E-2,2.7157709E-2,1.046974E-2,1.1895977E-1,4.5869254E-2,4.262839E-2,6.952983E-2,1.5049982E-1,3.6906146E-2,0E0,0E0,4.3109298E-2,0E0,0E0,0E0,0E0,1.5443023E-2,1.32172E-2,0E0,0E0,0E0,9.080692E-2,1.4392808E-1,0E0,9.912826E-3,3.9224803E-2,1.2763529E-2,0E0,0E0,1.4601588E-2,1.00580364E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,48,48,49,49,53,53,54,54,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,-1,-1,66,-1,-1,-1,-1,68,70,-1,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0892118E3,4.352552E9,1.2055723E8,5.162122E2,3.4E1,5.0561914E3,1.5119754E0,1.822764E4,5.415E4,9.285558E4,1.2E1,1E0,1.02E2,5.4438036E9,-2.6914816E-2,1E0,2.1768606E5,6.0511904E0,2.2583E4,-1.0647467E-2,-1.010692E-3,-2.5244318E-2,-1.0717563E-2,3.20566E7,6.1108776E7,9.298568E7,2.9349497E-3,1.9489948E-3,-3.9977366E-3,6.1E1,2.9264723E-3,8.637E3,2.8774E4,2E0,4.411222E6,1.0092749E8,8.3154816E2,1.4364E4,5.840393E-1,8.383825E1,1.91E2,2.1494467E-2,8.123851E-3,3.5623631E0,-1.1553373E-3,3.7385558E-3,-2.1798273E-3,-8.099262E-3,5.6058385E6,6.314179E2,8.864286E-3,-2.6523983E-3,1.1110839E-3,9.075745E7,2.0053221E6,-7.6833105E-4,4.7E1,9.355911E-1,5.314E3,-1.1465208E-2,-1.2050162E-3,6.518218E6,2.6678162E9,4.1120616E-3,-3.0286296E-3,-1.6242817E-2,-8.766459E-3,-4.539726E-3,-3.2800512E-4,6.600441E-4,5.003798E-3,-2.7539064E-3,5.611085E-3,-9.167116E-3,-1.8819284E-3,6.8485094E-3,1.6400296E-3,1.9219483E-3,9.214018E-3,-3.1865286E-3,1.0593238E-3,1.7825328E-2,1.0729566E-2,2.1698026E-3,1.0822392E-2],"split_indices":[52,5,45,55,3,52,57,33,29,33,8,109,3,5,0,112,28,53,10,0,0,0,0,50,45,45,0,0,0,3,0,9,9,8,32,7,4,10,42,58,10,0,0,56,0,0,0,0,32,4,0,0,0,7,48,0,8,27,9,0,0,45,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.31E2,7.8E1,1.11E2,2E1,6.8E1,1E1,4.6E1,6.5E1,9E0,1.1E1,5.4E1,1.4E1,7E0,3E0,2.2E1,2.4E1,1.8E1,4.7E1,7E0,2E0,7E0,4E0,1.9E1,3.5E1,1.2E1,2E0,2E0,5E0,2E1,2E0,4E0,2E1,1.2E1,6E0,3.7E1,1E1,1.3E1,6E0,3E1,5E0,1E1,2E0,1.8E1,2E0,2E0,2E0,1E1,1E1,7E0,5E0,4E0,2E0,1.1E1,2.6E1,4E0,6E0,8E0,5E0,3E0,3E0,1.7E1,1.3E1,2E0,3E0,1E1,8E0,7E0,3E0,3E0,4E0,7E0,4E0,1.2E1,1.4E1,4E0,2E0,6E0,2E0,2E0,3E0,7E0,1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.8897833E-2,-6.780625E-2,2.0897096E-1,-1.5521011E-2,-2.9606915E-1,3.5466662E-1,-4.421246E-3,-9.93627E-2,9.932131E-2,-1.4433515E-1,-4.58412E-1,4.7411785E-1,2.1561413E-1,9.5952526E-2,-1.4718074E-1,-2.0804802E-1,-6.7736566E-2,1.9759446E-1,2.8317139E-2,-1.7917114E-1,-4.480132E-2,-5.3120285E-1,-6.104539E-3,5.165622E-1,9.185563E-3,2.6545724E-1,2.4120836E-3,7.911076E-3,-3.163457E-3,-1.2183908E-2,-3.5092747E-4,-2.297472E-1,-1.1178532E-3,1.8829303E-2,-9.4082914E-2,1.3437358E-1,1.6593615E-2,-1.20509595E-1,6.209947E-2,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,-2.9204038E-1,-6.37282E-3,-4.2878713E-2,1.0429371E-1,-4.7475312E-2,-1.8176384E-1,2.5304276E-2,1.6177177E-1,6.3902856E-4,-8.5941E-3,1.6473201E-1,1.7290728E-2,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,-1,-1,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,67,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3322675E0,2.048187E0,1.1519101E0,1.3674978E0,7.220886E-1,2.631929E-1,2.4339132E-1,2.6776582E-1,4.1307282E-1,5.4385334E-2,2.9693174E-1,3.5738707E-2,8.616072E-2,6.653001E-2,9.928414E-2,6.7011595E-2,1.5001667E-1,1.8078548E-1,1.8848129E-1,1.2413353E-2,1.4049292E-2,2.2869587E-2,0E0,3.3781767E-2,0E0,5.2916467E-2,0E0,0E0,1.2511282E-2,0E0,0E0,6.0228944E-2,0E0,8.88534E-2,1.9570062E-1,5.2684218E-2,0E0,5.4782815E-2,1.3511673E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1208367E-2,0E0,1.15634445E-2,1.6972624E-2,1.08784124E-1,5.8658242E-2,1.0622809E-2,1.1181682E-2,0E0,0E0,4.1976526E-2,3.8918935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,28,28,31,31,33,33,34,34,35,35,37,37,38,38,51,51,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,-1,-1,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,68,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.2829284E7,6.627576E7,5.3287085E2,8.67241E9,8.812601E-1,3.511154E7,1.056E3,1.26881E5,2.1845242E3,5.770668E6,3.2707875E8,1.7414924E7,5.1E1,1.4539318E-1,6.1E1,2.5126263E1,1.467574E3,2.035483E6,1.0892118E3,7.6336784E1,6.1826086E0,-6.104539E-3,4.5E1,9.185563E-3,4.747723E6,2.4120836E-3,7.911076E-3,7.358531E3,-1.2183908E-2,-3.5092747E-4,2.23695E5,-1.1178532E-3,6.21E2,9.348848E0,5.9734796E2,1.6593615E-2,9.31E2,2.7E1,-1.008885E-2,-4.511087E-3,1.3158219E-4,-4.536085E-3,-2.7798412E-2,-1.0745641E-2,2.7898407E-2,1.0321458E-2,3.608763E-3,1.506335E-2,-2.351347E-3,2.0947566E-3,2.2290532E3,-6.37282E-3,5.093772E-1,3.396881E4,1.1556807E0,7.688406E0,3.7E1,1.617623E5,6.3902856E-4,-8.5941E-3,1.38635E5,1.8454842E-1,-4.0594675E-3,-1.653353E-2,-6.209801E-4,-4.1301707E-3,6.610368E-3,8.257963E-4,-4.481925E-3,1.1117839E-3,-1.1097401E-2,-4.0438813E-3,-1.0723097E-3,3.124758E-3,8.4853675E-3,2.5943597E-3,1.1576316E-3,9.811342E-3,5.1839706E-3,-1.1701894E-4],"split_indices":[52,45,45,52,5,57,32,2,29,4,29,32,32,3,38,3,58,4,32,52,56,53,0,3,0,32,0,0,4,0,0,12,0,11,54,52,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,48,0,27,33,42,53,3,33,0,0,11,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.71E2,3.6E1,1.4E2,3.1E1,2.1E1,1.5E1,8.1E1,5.9E1,1.7E1,1.4E1,1E1,1.1E1,9E0,6E0,1.7E1,6.4E1,2.4E1,3.5E1,1.2E1,5E0,1.1E1,3E0,8E0,2E0,8E0,3E0,5E0,4E0,3E0,3E0,1.5E1,2E0,1.5E1,4.9E1,1.8E1,6E0,6E0,2.9E1,8E0,4E0,3E0,2E0,9E0,2E0,6E0,2E0,2E0,6E0,2E0,2E0,8E0,7E0,9E0,6E0,3.3E1,1.6E1,4E0,1.4E1,2E0,4E0,8E0,2.1E1,2E0,6E0,6E0,3E0,4E0,2E0,2E1,1.3E1,1E1,6E0,2E0,2E0,1.2E1,2E0,2E0,6E0,3E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[9.597382E-3,-7.689029E-2,1.3135731E-1,-5.166708E-2,-3.934461E-1,1.9286239E-1,-1.5746945E-1,-1.8152632E-1,-2.5963075E-2,-2.2153566E-2,-5.7512084E-3,1.2324249E-1,4.3318063E-1,-8.0194466E-2,-2.265891E-2,-2.6121805E-3,-2.3640567E-1,5.765824E-4,-9.947969E-2,-1.1233864E-2,1.7927498E-1,5.3216517E-1,2.2989117E-1,-1.9807076E-1,6.597765E-2,-4.3076635E-3,-1.2422029E-2,2.8069925E-2,-1.1729321E-1,6.657329E-2,-1.4218627E-1,9.798071E-2,-6.077645E-2,-1.3545295E-3,1.9635805E-1,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,7.5570874E-2,-6.374473E-2,-1.7920145E-1,-1.4495179E-3,2.901012E-4,4.642252E-3,-8.290076E-2,-1.9911025E-1,1.871269E-3,7.0417197E-3,-5.9855576E-3,-1.9459605E-2,2.7096123E-1,1.4672595E-1,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,63,65,-1,-1,-1,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2320848E0,9.697564E-1,1.5936292E0,3.824837E-1,9.9226475E-2,1.1679718E0,3.4362957E-1,1.2062222E-1,1.912657E-1,0E0,0E0,4.3904662E-1,1.9962025E-1,2.5429142E-1,0E0,0E0,1.3633668E-2,2.3985873E-1,1.9384623E-1,1.0334882E-1,1.5052485E-1,1.5721798E-2,2.7384043E-2,1.3692349E-2,4.27702E-2,0E0,0E0,2.664942E-1,7.050744E-2,9.9109635E-3,5.4363728E-2,1.1121236E-2,3.102351E-2,0E0,1.0094595E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2414542E-1,1.4501756E-1,3.8122535E-2,0E0,0E0,0E0,4.5270823E-2,2.6841223E-2,0E0,0E0,0E0,1.3282825E-2,1.0726178E-1,4.9301982E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,34,34,43,43,44,44,45,45,49,49,50,50,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,64,66,-1,-1,-1,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.2109607E2,8.260109E9,9.298568E7,1.295E3,9.4819355E0,5.08711E3,2.88001E10,5.7625353E-1,1.5989196E7,-2.2153566E-2,-5.7512084E-3,1E0,2.7E1,1.5574268E7,-2.265891E-2,-2.6121805E-3,2.9585883E2,1.193419E1,1.3729923E5,1E0,7.056125E7,9.886E3,1.917786E7,1.15E2,3.9E1,-4.3076635E-3,-1.2422029E-2,5.998024E-1,8.401064E0,2.68E2,1.09137096E8,1.9253E4,2.1845242E3,-1.3545295E-3,1.2951E4,3.194545E-2,1.6543105E-2,5.120687E-3,1.44582875E-2,-3.512597E-3,-1.1096234E-2,7.603609E-3,-6.690861E-5,4.1376923E2,8.05E2,2.6239506E5,-1.4495179E-3,2.901012E-4,4.642252E-3,3.1274893E2,1.0209414E1,1.871269E-3,7.0417197E-3,-5.9855576E-3,4E1,1.1865996E1,1.4456249E7,1.398154E-3,7.0097977E-3,-6.616069E-3,1.424601E-3,-1.1498456E-2,-3.0691253E-3,-7.603005E-3,-1.2998496E-3,-1.1062344E-2,-2.8003752E-3,9.4396167E-4,-2.6486153E-3,1.4941797E-2,1.7708328E-3,7.777994E-3,-1.2603356E-4],"split_indices":[52,5,45,2,53,52,5,27,45,0,0,109,8,1,0,0,33,54,28,8,7,9,1,10,3,0,0,57,53,0,7,2,4,0,9,0,0,0,0,0,0,0,0,52,0,28,0,0,0,52,54,0,0,0,3,54,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.23E2,8.7E1,1.15E2,8E0,7.2E1,1.5E1,1.8E1,9.7E1,6E0,2E0,5.7E1,1.5E1,1.3E1,2E0,6E0,1.2E1,7.2E1,2.5E1,1.7E1,4E1,9E0,6E0,7E0,6E0,2E0,1E1,5.9E1,1.3E1,5E0,2E1,5E0,1.2E1,3E0,3.7E1,4E0,5E0,3E0,3E0,2E0,5E0,2E0,4E0,3.9E1,2E1,7E0,6E0,2E0,3E0,1.1E1,9E0,3E0,2E0,4E0,8E0,1.3E1,2.4E1,2.4E1,1.5E1,1.1E1,9E0,4E0,3E0,4E0,7E0,7E0,2E0,4E0,4E0,1.1E1,2E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-1.772178E-2,-7.057884E-2,8.537469E-2,-5.372641E-2,-3.503544E-1,-7.3327735E-2,1.9524743E-1,-1.9434269E-1,-2.6594644E-2,-2.087923E-2,-8.001967E-3,1.7575795E-2,-2.1872297E-1,1.2774226E-1,3.8615E-1,-2.1584399E-1,-2.1231158E-3,2.6761827E-3,-1.002963E-1,-2.3103694E-2,1.2448591E-2,-3.2898784E-1,7.034172E-5,-6.594587E-2,1.6341075E-1,2.4855476E-2,1.1777236E-2,-2.5548145E-1,-6.297792E-3,6.468007E-2,-6.0867704E-2,3.847441E-3,-1.2086692E-1,1.0591442E-1,-6.7531325E-2,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,1.8735889E-1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,1.1528385E-1,-3.7465915E-2,-1.02739446E-1,5.237328E-2,-1.373252E-1,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,-2.5229553E-2,2.3754518E-1,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,-1,-1,41,-1,43,45,-1,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,55,57,59,61,63,-1,-1,-1,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1885403E0,6.6012216E-1,1.3046603E0,5.1750946E-1,4.6478212E-2,4.1356945E-1,5.176964E-1,6.1877012E-2,2.5051618E-1,0E0,0E0,1.9972178E-1,2.9179746E-1,2.4566746E-1,7.394695E-2,3.203535E-2,0E0,3.3489424E-1,1.3130611E-1,1.10328E-1,0E0,9.977138E-2,0E0,1.4139006E-2,1.0379839E-1,0E0,0E0,6.927937E-2,0E0,2.2658505E-1,2.045279E-1,0E0,7.257894E-2,1.1413369E-2,8.147253E-2,0E0,0E0,0E0,0E0,8.3029985E-2,0E0,0E0,0E0,9.5561266E-2,5.3016253E-2,1.7131087E-1,4.7250487E-2,5.080968E-2,0E0,0E0,0E0,0E0,2.8906034E-2,3.853917E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,23,23,24,24,27,27,29,29,30,30,32,32,33,33,34,34,39,39,43,43,44,44,45,45,46,46,47,47,52,52,53,53],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,-1,-1,42,-1,44,46,-1,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,56,58,60,62,64,-1,-1,-1,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.142E3,6.645232E2,8.307621E7,1.9253E4,1.3448806E8,1.6278354E7,-2.087923E-2,-8.001967E-3,1.9142364E7,1.4539318E-1,7.5773336E7,1.2117E4,2.23695E5,-2.1231158E-3,1.643E4,5.577E3,8.87E2,1.2448591E-2,2.2982106E0,7.034172E-5,1.33511E5,7.4308796E0,2.4855476E-2,1.1777236E-2,3.7790768E2,-6.297792E-3,1.0466548E1,6.3004294E5,3.847441E-3,9.568003E2,1.4783E4,2.9E1,-4.075592E-3,-1.9309761E-2,-6.2159507E-4,-5.5899085E-3,7.4009796E1,5.3474284E-4,-5.3322427E-3,-1.531741E-2,5.3055557E1,6.693824E5,4.966E3,1.175E1,1.30468176E11,9.94896E-4,7.1581546E-3,1.4326422E-3,-1.0425715E-2,7.7573473E9,4.60675E5,4.958644E-3,7.8121047E-3,1.8404423E-3,2.497017E-3,-3.6213708E-3,-6.6045774E-3,2.7866901E-3,-4.354339E-4,5.6056487E-3,-4.028666E-3,-8.860225E-3,-2.9659593E-3,1.7987367E-3,3.179355E-3,1.2605317E-2],"split_indices":[52,5,109,2,33,45,2,7,45,0,0,1,38,7,9,12,0,9,9,0,0,53,0,12,53,0,0,33,0,54,28,0,52,9,3,0,0,0,0,58,0,0,0,58,48,2,56,31,0,0,0,0,5,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.43E2,7.3E1,1.36E2,7E0,3E1,4.3E1,2.1E1,1.15E2,4E0,3E0,1.9E1,1.1E1,3.3E1,1E1,1.8E1,3E0,8.3E1,3.2E1,1.7E1,2E0,7E0,4E0,5E0,2.8E1,4E0,6E0,1.1E1,7E0,4.2E1,4.1E1,3E0,2.9E1,4E0,1.3E1,2E0,5E0,3E0,2E0,2.4E1,4E0,4E0,7E0,2.8E1,1.4E1,3E1,1.1E1,2.6E1,3E0,2E0,2E0,2E0,1.1E1,1.4E1,1E1,1.7E1,1.1E1,4E0,1E1,2.5E1,5E0,6E0,5E0,1.3E1,1.3E1,7E0,4E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.1619442E-2,-6.1717395E-2,1.3433132E-1,-2.3387735E-1,-4.0964365E-2,1.77865E-3,2.3689537E-1,1.3427171E-2,-3.110039E-1,-1.2818594E-1,-3.7928703E-3,5.2376132E-2,-8.1358645E-3,2.3620052E-2,1.8211213E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,-1.5684637E-1,1.68092E-2,7.7523957E-3,-1.8851958E-2,-1.6274573E-2,1.226543E-1,2.2858551E-1,6.3767E-2,-1.8596564E-1,-4.8825707E-2,8.553022E-2,-6.1434526E-3,-7.6919543E-3,1.647666E-1,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,1.4144917E-1,1.4252614E-2,4.003073E-4,4.388098E-3,-1.9882518E-1,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,-2.6464459E-2,1.329478E-1,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,-1,-1,-1,-1,27,29,31,-1,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,53,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5730232E0,5.620738E-1,7.473112E-1,3.2842952E-1,4.6638432E-1,2.2247311E-1,3.3355105E-1,3.4519482E-2,7.46156E-2,1.8132824E-1,4.54711E-1,9.848905E-2,0E0,0E0,1.3392115E-1,0E0,0E0,0E0,0E0,1.0415751E-1,8.9438625E-2,2.4376561E-1,0E0,1.8215474E-2,2.6880935E-2,6.499237E-2,1.245489E-2,5.9958518E-2,5.9645988E-2,1.10448785E-2,0E0,2.4641316E-1,2.660959E-1,0E0,0E0,0E0,0E0,1.1662543E-2,0E0,0E0,0E0,2.202785E-2,0E0,0E0,0E0,0E0,0E0,1.3542011E-1,6.190428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,37,37,41,41,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,-1,-1,-1,-1,28,30,32,-1,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,54,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,6.794276E-5,6.48045E6,1.7727281E2,2.802166E2,1.0430015E1,6.677E3,1.94122E5,3.2445145E2,1.01978E5,1.0590684E1,2.806E3,-8.1358645E-3,2.3620052E-2,8.269529E-1,-3.154838E-3,4.243926E-3,-2.0144468E-2,-9.486762E-3,2.1974122E5,4.87374E5,4.9192E4,-1.8851958E-2,1.5878E4,2.0837E4,1.980161E0,4.602015E0,6.1E1,3.7E1,3.52777E5,-6.1434526E-3,1E0,1.0670794E10,-2.3391573E-3,1.5323637E-3,7.236103E-3,6.008857E-4,5.649635E0,1.4252614E-2,4.003073E-4,4.388098E-3,1.8E2,-6.096782E-4,-4.642968E-3,3.705071E-3,7.7292946E-4,5.6639803E-3,3.2441288E5,3.9360254E2,1.5511897E-2,-1.0784101E-3,3.4808265E-3,8.572358E-3,-1.4133823E-2,-8.223176E-3,-4.0345252E-3,1.3375188E-4,-8.6961495E-4,8.197768E-3],"split_indices":[42,42,48,33,52,54,9,1,33,29,57,2,0,0,27,0,0,0,0,47,1,2,0,10,9,42,53,3,3,1,0,94,12,0,0,0,0,53,0,0,0,11,0,0,0,0,0,28,52,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.59E2,5.4E1,1.6E1,1.43E2,2.4E1,3E1,4E0,1.2E1,4.2E1,1.01E2,1.9E1,5E0,4E0,2.6E1,2E0,2E0,5E0,7E0,3.5E1,7E0,9.9E1,2E0,1E1,9E0,1.8E1,8E0,2.7E1,8E0,5E0,2E0,9.1E1,8E0,6E0,4E0,7E0,2E0,9E0,9E0,3E0,5E0,2.5E1,2E0,6E0,2E0,2E0,3E0,8.1E1,1E1,4E0,4E0,4E0,5E0,4E0,2.1E1,2.7E1,5.4E1,2E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.0766466E-3,-4.316265E-2,2.0560837E-1,-1.2818186E-2,-2.3354582E-1,2.703695E-1,-9.532744E-2,-5.3655125E-2,1.0286662E-1,-2.9023632E-1,-1.5101601E-2,2.0894068E-1,4.177894E-1,1.6216272E-2,-1.39497025E-2,-1.1859325E-1,-2.1754269E-2,2.239287E-1,4.9224846E-2,-7.534239E-3,-3.746514E-1,-5.9629036E-3,3.3697663E-3,7.089449E-2,2.384329E-1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,-1.6322732E-1,-5.6480993E-2,2.9111031E-2,-7.753057E-2,1.5182119E-2,4.083155E-3,8.881563E-2,-4.8623074E-2,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,2.5431147E-1,5.2253725E-3,6.213173E-4,-1.8242495E-1,-1.1897854E-2,-5.765586E-3,5.034584E-2,-1.3384877E-1,7.912595E-3,-1.0946924E-1,-9.919967E-3,1.289712E-1,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,37,-1,-1,39,41,-1,-1,-1,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,-1,55,-1,-1,57,59,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.055721E0,1.0079945E0,8.2482004E-1,7.235998E-1,2.9282618E-1,2.2060513E-1,1.7536506E-1,2.3150888E-1,2.487365E-1,1.5920115E-1,6.284737E-2,8.894992E-2,2.0777345E-2,1.1685234E-2,0E0,9.335673E-2,2.2071204E-1,1.2175274E-1,1.1666089E-1,0E0,1.730293E-1,0E0,0E0,3.1039445E-2,1.6396642E-2,0E0,0E0,0E0,0E0,7.328111E-2,4.5573883E-2,1.4861596E-1,1.0183704E-1,0E0,0E0,8.453986E-2,1.6937668E-2,0E0,0E0,0E0,0E0,1.6923666E-2,0E0,0E0,3.164649E-2,3.1010306E-2,0E0,1.0485908E-1,1.7051622E-2,8.5487984E-2,7.8597695E-2,2.3515034E-2,7.874721E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,20,20,23,23,24,24,29,29,30,30,31,31,32,32,35,35,36,36,41,41,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,38,-1,-1,40,42,-1,-1,-1,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,-1,56,-1,-1,58,60,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,7.0099896E9,1.2055723E8,8.297185E2,1.2728E4,1.4243088E6,2.374331E5,2.1015242E2,1.10597E5,5.033165E7,1.4792529E10,4.0492815E-1,1.4735E4,9.4908775E1,-1.39497025E-2,3.5E1,7.2929355E6,6.675932E1,8.7022424E-1,-7.534239E-3,8.260109E9,-5.9629036E-3,3.3697663E-3,8.601656E6,5.1E1,8.38461E-3,2.2548156E-2,-1.6415741E-3,2.414963E-3,7.36E2,3.5223506E5,1.996866E1,4.008916E10,1.5182119E-2,4.083155E-3,9.1014955E10,2.7577372E11,-5.800675E-3,-2.2169972E-2,-3.76564E-5,6.9505735E-3,1.6503E4,5.2253725E-3,6.213173E-4,1.13002876E-4,6.693824E5,-5.765586E-3,2.579E3,6.09E2,1.438E3,1.8272479E3,5.0895057E10,1E0,1.116267E-3,-3.5208017E-3,5.679224E-3,1.3337262E-2,-1.9185728E-3,-9.583869E-3,2.4025685E-3,-2.6260186E-3,-2.0070522E-4,4.972056E-3,-8.981337E-3,-1.8752366E-3,-5.4580304E-3,3.2584972E-3,-6.093815E-3,2.8344835E-3,2.9900987E-3,-2.4699366E-3,3.4145347E-4,8.403752E-3],"split_indices":[52,5,45,52,9,28,33,52,29,7,5,27,2,56,0,3,45,58,27,0,5,0,0,1,3,0,0,0,0,29,28,56,31,0,0,31,31,0,0,0,0,2,0,0,38,48,0,2,0,2,4,31,109,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.74E2,4E1,1.51E2,2.3E1,3.3E1,7E0,1.12E2,3.9E1,1.8E1,5E0,2.5E1,8E0,5E0,2E0,3.6E1,7.6E1,1.1E1,2.8E1,8E0,1E1,2E0,3E0,5E0,2E1,2E0,6E0,2E0,3E0,2E1,1.6E1,4E1,3.6E1,6E0,5E0,2E1,8E0,3E0,7E0,3E0,2E0,1.7E1,3E0,2E0,1.8E1,1E1,6E0,3.6E1,4E0,1E1,2.6E1,6E0,1.4E1,2E0,6E0,3E0,1.4E1,2E0,1.6E1,4E0,6E0,1.8E1,1.8E1,2E0,2E0,3E0,7E0,2.4E1,2E0,2E0,4E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[5.1350137E-3,-3.3741128E-2,2.475987E-1,-2.1715815E-1,-6.9968994E-3,3.2988298E-1,2.8091062E-2,-6.1420463E-2,-2.4671237E-1,-9.5820725E-2,3.2276653E-2,2.3344913E-3,3.5703343E-1,-5.512472E-2,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,-3.0382487E-1,-1.4058068E-1,-2.9446453E-2,-2.1209379E-1,5.59181E-2,4.091285E-1,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,-6.627251E-2,-2.2693902E-1,5.2225657E-2,-9.5052674E-2,1.0859418E-2,-3.6253917E-1,2.6486173E-2,1.5046827E-1,2.1380894E-2,7.393621E-3,2.4758913E-3,-8.528379E-2,-3.1659063E-3,-2.504589E-1,-1.3437112E-3,5.355821E-3,-7.7356305E-3,-4.3233983E-2,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,1.0883494E-2,1.4663106E-1,1.9825523E-1,8.6160064E-2,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,59,-1,-1,-1,-1,61,63,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9783993E0,8.8634837E-1,5.145322E-1,9.070897E-2,5.577526E-1,1.4360905E-1,6.3942134E-2,9.758426E-3,6.0073018E-2,1.4143506E-1,6.5313697E-1,0E0,1.07219934E-1,1.2388382E-2,0E0,0E0,0E0,0E0,3.4586906E-2,1.7105693E-1,1.1748636E-1,3.393597E-1,2.7985966E-1,4.202509E-2,0E0,0E0,0E0,0E0,0E0,4.2199805E-2,3.3204734E-2,4.9204696E-2,3.2862037E-2,3.151442E-2,7.372153E-2,1.4649613E-1,5.6672513E-2,0E0,0E0,0E0,3.002061E-2,0E0,3.334278E-2,0E0,0E0,0E0,2.2537217E-2,0E0,0E0,0E0,0E0,1.6933751E-1,1.3314232E-2,2.8548539E-2,3.267061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,18,18,19,19,20,20,21,21,22,22,23,23,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,42,42,46,46,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,60,-1,-1,-1,-1,62,64,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7162656E3,2.750917E-4,9.298568E7,2.56917E-2,2.802166E2,4.6111366E1,5.547E3,6.054E3,2.799504E11,1.8716404E10,3.791641E0,2.3344913E-3,4.5E1,4.8008E4,5.1434897E-3,-4.7476883E-3,-2.3417095E-4,-7.275425E-3,6.645232E2,8.1446184E7,4.57E2,8.640031E9,1.5085194E0,3.2707875E8,8.563964E-3,1.9127494E-4,-4.6624816E-3,-1.8404378E-2,-8.703427E-3,1.5E1,2.2057E4,2.3168E5,3.1E1,8.601996E-1,8.92851E6,7.6599895E9,2.5661135E0,2.1380894E-2,7.393621E-3,2.4758913E-3,3.07E2,-3.1659063E-3,2.0562818E4,-1.3437112E-3,5.355821E-3,-7.7356305E-3,2.7898628E5,-3.092113E-3,3.9729285E-3,-2.2000177E-2,-5.9534153E-3,6.7835927E0,1.743E3,2.6198992E-1,1.51E3,-8.76314E-4,-5.6999E-3,-1.5407574E-2,-6.9388226E-3,1.6985154E-3,-3.65465E-3,2.5752555E-3,-2.1928889E-3,2.1736892E-3,8.243293E-3,5.6419726E-3,1.2276442E-2,5.6594834E-3,-1.5452564E-4],"split_indices":[52,39,45,58,52,56,9,2,31,31,53,0,3,2,0,0,0,0,33,5,0,5,42,32,0,0,0,0,0,3,12,12,3,27,1,5,42,0,0,0,11,0,33,0,0,0,28,0,0,0,0,53,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.8E2,2.8E1,2.2E1,1.58E2,2E1,8E0,4E0,1.8E1,4.8E1,1.1E2,2E0,1.8E1,4E0,4E0,2E0,2E0,8E0,1E1,2.8E1,2E1,9E0,1.01E2,1.3E1,5E0,2E0,2E0,5E0,5E0,1.6E1,1.2E1,9E0,1.1E1,4E0,5E0,7.8E1,2.3E1,1.1E1,2E0,2E0,1.4E1,2E0,1E1,4E0,5E0,4E0,7E0,2E0,2E0,3E0,2E0,7E1,8E0,1.2E1,1.1E1,5E0,9E0,5E0,5E0,2E0,5E0,4E1,3E1,2E0,6E0,6E0,6E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-6.245155E-3,-2.833662E-2,3.4589776E-1,-7.005383E-2,5.9592E-2,4.2491645E-1,-1.5961174E-3,-4.3598708E-2,-2.8613925E-1,-8.835135E-2,1.3214482E-1,5.290952E-1,1.0459326E-2,-9.431229E-2,1.7313663E-2,-4.463223E-3,-3.3082795E-1,2.9486376E-3,-2.370824E-1,-2.7359384E-3,1.4807577E-1,3.203436E-2,8.785884E-3,-1.4088191E-1,-5.170084E-2,4.7399618E-2,-1.0460413E-1,-2.1237668E-2,-1.0130584E-2,6.996463E-3,-4.852486E-2,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,1.24671705E-1,-2.0670156E-1,-8.551871E-2,5.622916E-3,-6.533785E-2,6.141754E-2,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,-8.802993E-2,6.8448015E-2,1.8364535E-1,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,51,53,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5812309E0,7.0478874E-1,3.730235E-1,7.306783E-1,6.77165E-1,1.050607E-1,0E0,3.6305583E-1,9.1302514E-2,2.8586107E-1,1.3435823E-1,1.5126264E-1,0E0,1.1971778E-1,2.0303094E-1,0E0,5.1597834E-2,1.0845146E-1,1.1068231E-1,0E0,6.660807E-2,0E0,0E0,9.164274E-2,8.737236E-2,1.02572635E-1,6.1553992E-2,0E0,0E0,0E0,2.9702732E-2,0E0,0E0,0E0,1.00164235E-1,4.0271103E-2,6.0178608E-2,0E0,5.5570483E-2,7.76602E-2,0E0,0E0,0E0,0E0,1.725753E-2,3.47196E-2,2.8324187E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,30,30,34,34,35,35,36,36,38,38,39,39,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,52,54,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,1.0648859E3,1.2055723E8,4.934166E7,1E0,3.0291426E-1,-1.5961174E-3,3.3969492E2,8.7793425E-2,5E0,7.340097E7,1.04E3,1.0459326E-2,2.0562818E4,1.2670352E1,-4.463223E-3,3E0,8.78E2,9.875E3,-2.7359384E-3,3.835264E-2,3.203436E-2,8.785884E-3,1.4357987E10,4.914413E-3,1.7328871E6,1.018E3,-2.1237668E-2,-1.0130584E-2,6.996463E-3,2E0,-1.6071461E-2,-2.9865885E-3,1.2459558E-2,2.6673E4,7.3353534E2,5.571429E0,5.622916E-3,3.606519E-1,2.5356756E1,-5.262495E-3,-2.4963494E-3,-1.1176893E-2,8.0237375E-4,9.281264E-2,6.7835927E0,7.1847E4,-1.1261447E-2,-2.2988927E-3,-1.5598866E-4,-6.1518443E-3,-4.1247117E-3,9.193E-4,4.115068E-3,-9.1342727E-4,-6.4162794E-3,-1.0809026E-3,4.574522E-3,-1.2533266E-4,9.750748E-3,2.1624647E-3],"split_indices":[4,52,45,45,109,38,0,52,58,8,7,0,0,33,54,0,8,0,9,0,27,0,0,31,38,47,0,0,0,0,8,0,0,0,10,4,56,0,38,56,0,0,0,0,38,53,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.9E2,1.1E1,1.29E2,6.1E1,9E0,2E0,1.16E2,1.3E1,2E1,4.1E1,5E0,4E0,6.3E1,5.3E1,3E0,1E1,1.3E1,7E0,3E0,3.8E1,3E0,2E0,2.9E1,3.4E1,4.3E1,1E1,4E0,6E0,3E0,1E1,4E0,3E0,5E0,3.3E1,1.2E1,1.7E1,2E0,3.2E1,4E1,3E0,8E0,2E0,4E0,6E0,1.8E1,1.5E1,1E1,2E0,6E0,1.1E1,2.6E1,6E0,3.1E1,9E0,3E0,3E0,1.3E1,5E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.3435227E-3,-3.277699E-2,1.774612E-1,-6.964357E-3,-1.5355606E-1,1.9180963E-2,2.988968E-1,-6.0709182E-2,7.332652E-2,-6.922103E-2,-2.4452102E-1,1.5958549E-1,-3.233358E-2,3.533102E-1,1.318238E-1,-1.3005522E-1,-4.060321E-2,1.0830404E-1,-3.1816754E-2,-1.1131207E-2,-1.2607405E-1,-2.9532254E-1,2.7801134E-3,1.0396815E-3,1.1904416E-2,1.8165827E-2,-1.0879546E-2,3.7512168E-1,6.5092915E-3,2.031597E-3,9.1953445E-3,-1.5207392E-1,-1.9281486E-4,-4.925895E-3,-9.33342E-2,5.5923946E-2,1.5398896E-1,-1.0388468E-1,2.1812472E-2,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,-3.3411142E-1,-2.0121949E-2,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,4.4080522E-2,-6.593762E-2,-4.5851987E-2,-1.4824677E-1,-2.6830027E-3,8.220542E-2,1.7252632E-3,1.8331738E-1,9.3954586E-4,-7.641705E-3,4.75524E-3,-9.375982E-3,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,67,69,71,73,-1,75,-1,77,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4195844E0,5.663384E-1,7.4388015E-1,6.560496E-1,2.2856313E-1,1.353517E-1,1.5342748E-1,1.21597886E-1,2.2778258E-1,5.8043107E-2,2.467323E-1,5.353722E-2,1.3925183E-1,2.644825E-2,2.8130919E-2,5.492565E-2,1.3499385E-1,1.01465404E-1,6.410514E-2,3.194621E-2,4.1086197E-2,4.452479E-2,0E0,0E0,0E0,3.9501768E-2,0E0,4.0338874E-2,0E0,0E0,0E0,3.306678E-2,0E0,1.3446508E-1,6.8816334E-2,7.1661495E-2,7.693821E-2,4.8853584E-2,2.4587668E-2,0E0,0E0,0E0,0E0,0E0,4.43269E-2,1.6359001E-2,0E0,0E0,0E0,0E0,0E0,1.0982905E-1,8.9442335E-2,3.789381E-2,2.6375294E-2,0E0,2.339761E-2,0E0,3.7862718E-2,0E0,0E0,0E0,1.0178003E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,27,27,31,31,33,33,34,34,35,35,36,36,37,37,38,38,44,44,45,45,51,51,52,52,53,53,54,54,56,56,58,58,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,68,70,72,74,-1,76,-1,78,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.075104E7,1E0,5.470729E2,7.6599895E9,1E0,3.3326385E0,1.142E3,2.7883E4,1.0363E4,7.2418816E12,3.5912186E-1,8.129243E5,4.4E1,3.6E1,1.3448806E8,5.549974E8,7.8761675E5,1.6595541E1,1.1484E4,1.0731037E-1,8.67241E9,2.7801134E-3,1.0396815E-3,1.1904416E-2,2.613E3,-1.0879546E-2,2.608036E6,6.5092915E-3,2.031597E-3,9.1953445E-3,1.8E2,-1.9281486E-4,4.61E2,1.2845426E1,3.3997994E5,3.88E2,8.45E2,1.3050505E0,-2.1013466E-3,3.7978697E-3,1.070762E-5,-7.893186E-3,-6.0599316E-3,6.1826086E0,1.970492E6,4.855542E-3,6.6260127E-3,1.9641975E-2,-1.0790999E-2,-5.089104E-3,7.269755E0,3.6050353E5,9.5157854E-2,1E0,-2.6830027E-3,7E1,1.7252632E-3,7.3586698E0,9.3954586E-4,-7.641705E-3,4.75524E-3,1.82E9,-1.9896705E-2,-8.643983E-3,1.1088409E-3,-2.8715157E-3,4.1892715E-3,-2.7219784E-3,-5.353368E-3,1.5383086E-3,-3.528114E-3,1.5861554E-3,-1.3123517E-3,-8.168482E-3,4.589993E-3,-6.725394E-4,9.682982E-3,1.6319974E-3,1.4311767E-3,-1.8750144E-3],"split_indices":[52,45,109,52,5,8,57,2,9,9,31,27,28,3,3,7,5,28,56,2,27,5,0,0,0,0,0,1,0,0,0,11,0,0,56,47,0,0,57,0,0,0,0,0,53,29,0,0,0,0,0,53,28,38,64,0,8,0,53,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.81E2,3.8E1,1.5E2,3.1E1,1.7E1,2.1E1,9E1,6E1,1.7E1,1.4E1,4E0,1.3E1,1.5E1,6E0,1.9E1,7.1E1,4.5E1,1.5E1,9E0,8E0,1.2E1,2E0,2E0,2E0,1.1E1,2E0,1.3E1,2E0,3E0,3E0,1.6E1,3E0,4.3E1,2.8E1,2.2E1,2.3E1,6E0,9E0,7E0,2E0,2E0,6E0,3E0,9E0,8E0,3E0,2E0,1.1E1,5E0,1.1E1,2.4E1,1.9E1,1.6E1,1.2E1,4E0,1.8E1,5E0,1.8E1,2E0,4E0,2E0,7E0,5E0,4E0,4E0,4E0,1.7E1,7E0,1.3E1,6E0,1.2E1,4E0,2E0,1E1,1.6E1,2E0,1.6E1,2E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-1.0673481E-2,-2.922265E-2,3.2056907E-1,-9.558372E-2,6.192814E-3,7.489175E-3,1.9760614E-2,-6.519865E-2,-2.9253313E-1,-5.28204E-2,5.102089E-2,-1.4650565E-1,-1.8217016E-2,-8.463386E-3,-2.1037353E-2,2.8328342E-2,-8.429096E-2,-5.649434E-2,8.870596E-2,-1.687275E-1,-7.512345E-4,8.831442E-2,-5.69984E-2,7.728642E-2,-2.82216E-3,-1.0359726E-1,6.4482796E-4,4.802586E-3,-9.4108865E-2,1.5494351E-2,1.22702755E-1,-2.0045006E-1,-3.7551587E-3,5.5987155E-4,6.6966885E-3,-2.3729175E-3,-8.077933E-3,6.7342306E-3,6.569003E-4,-1.23539165E-1,-4.9365353E-2,-6.873523E-3,-9.76779E-4,4.8460853E-3,-3.942172E-2,7.549796E-2,1.7108953E-1,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,-1,39,-1,-1,41,43,45,47,-1,-1,-1,49,-1,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.158347E0,4.18929E-1,7.998216E-2,3.5195512E-1,3.1221244E-1,0E0,0E0,2.0742762E-1,5.744326E-2,1.3263142E-1,2.764765E-1,5.5647433E-2,1.537655E-1,0E0,0E0,6.787142E-2,7.104939E-2,1.143521E-1,1.2254682E-1,3.3913255E-2,0E0,3.627623E-2,1.603502E-1,3.7099797E-2,0E0,2.7427554E-2,0E0,0E0,4.950477E-2,8.2423344E-2,6.474209E-2,3.1638563E-2,0E0,0E0,0E0,9.902678E-2,0E0,0E0,0E0,5.4692864E-2,1.5442751E-2,0E0,0E0,0E0,3.445577E-2,8.0237605E-2,6.781274E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,28,28,29,29,30,30,31,31,35,35,39,39,40,40,44,44,45,45,46,46],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,-1,40,-1,-1,42,44,46,48,-1,-1,-1,50,-1,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3060927E0,1.04201E-2,5.9165E4,1.810504E10,5.470729E2,7.489175E-3,1.9760614E-2,5.162122E2,5.752E3,1.0275E4,3.8834286E0,1E0,9.8298E4,-8.463386E-3,-2.1037353E-2,2.8945708E6,4E1,1.4139E4,3.996671E5,3.5523141E-3,-7.512345E-4,2.7123368E10,5.347626E5,7.751E3,-2.82216E-3,7.2899837E0,6.4482796E-4,4.802586E-3,1.1334876E0,7.6420575E6,2.6673E4,3.0353E4,-3.7551587E-3,5.5987155E-4,6.6966885E-3,8.591E3,-8.077933E-3,6.7342306E-3,6.569003E-4,5.8910837E0,5.69434E-1,-6.873523E-3,-9.76779E-4,4.8460853E-3,5.754E3,7.6221466E8,7.6E0,-3.6969734E-3,-1.1363823E-2,-1.8219791E-3,6.7393794E-3,-2.960844E-3,-8.308281E-3,-3.964013E-3,-4.8056587E-5,-5.5220546E-3,1.2306932E-4,6.0909004E-3,-4.1008455E-4,2.9012219E-3,1.0528524E-2],"split_indices":[35,38,2,5,52,0,0,55,2,9,53,112,29,0,0,45,3,10,28,39,0,31,28,9,0,53,0,0,39,45,10,12,0,0,0,2,0,0,0,53,27,0,0,0,2,7,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.77E2,9E0,6.1E1,1.16E2,4E0,5E0,5.4E1,7E0,5E1,6.6E1,1.9E1,3.5E1,5E0,2E0,1.4E1,3.6E1,1.7E1,4.9E1,1.6E1,3E0,9E0,2.6E1,9E0,5E0,3E1,6E0,3E0,1.4E1,1.6E1,3.3E1,1.1E1,5E0,4E0,5E0,1.8E1,8E0,4E0,5E0,2.1E1,9E0,8E0,6E0,6E0,1E1,1.8E1,1.5E1,3E0,8E0,1.5E1,3E0,1E1,1.1E1,5E0,4E0,3E0,7E0,1.1E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[3.5595038E-4,-4.4311643E-2,2.0678306E-1,-1.9426705E-2,-1.8886262E-1,3.1928593E-1,5.56452E-2,-7.896872E-2,2.8734265E-2,-9.082003E-2,-3.8585064E-1,2.895736E-2,2.4027427E-1,1.074728E-1,-3.3747476E-2,-1.00550115E-1,1.675159E-2,7.017641E-2,-3.8212094E-2,-1.407528E-1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.6446205E-1,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,-1.5302381E-1,-7.644423E-2,7.8863114E-2,-4.7753017E-2,1.0884225E-1,-1.3557769E-2,-5.902727E-2,4.1850857E-3,-9.2872605E-3,-4.909929E-2,1.4215367E-2,5.618002E-3,-3.536953E-3,-2.0744017E-1,-4.741403E-2,-1.308574E-1,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.7041807E-2,1.5840885E-1,-8.000341E-2,2.9958343E-2,-1.2175934E-1,-2.2903696E-2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9178276E0,6.114423E-1,6.1136055E-1,4.238065E-1,4.477874E-1,3.14538E-1,8.238807E-2,1.3802564E-1,2.3018244E-1,2.9552594E-1,3.0219913E-2,0E0,4.197371E-2,6.660308E-2,4.066568E-2,5.66082E-2,5.5850185E-2,1.6659597E-1,8.780883E-2,6.407347E-2,0E0,0E0,0E0,0E0,2.8056264E-2,0E0,0E0,0E0,0E0,5.4878265E-2,5.5400893E-2,1.0032747E-2,1.9345354E-2,9.9761456E-2,5.1551525E-2,6.0662225E-2,0E0,0E0,1.02921E-2,0E0,0E0,0E0,4.0799797E-2,7.5311735E-2,1.8520892E-2,0E0,0E0,0E0,0E0,3.455124E-2,3.763908E-2,3.8690723E-2,1.8420529E-2,3.3416137E-2,2.2927333E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,6.6932364E7,5.9337012E7,3.3969492E2,1.4792529E10,5.0655737E0,1.2055723E8,2E0,2.2971932E7,2E1,1.8942408E7,2.895736E-2,2.819228E-1,2.1516852E5,9.4908775E1,1.142E3,1.06403265E1,9.607843E-1,1.4792529E10,5.9862564E1,1.0013678E-2,-2.1316389E-2,-7.441832E-3,4.355721E-3,2.5868E4,1.0314435E-2,2.034471E-3,-4.9490836E-3,2.0749366E-3,5.244755E-3,7.44E2,9.950143E7,1.2067E4,5.6426154E2,3.634E3,2.2502415E1,4.1850857E-3,-9.2872605E-3,1.2281632E3,1.4215367E-2,5.618002E-3,-3.536953E-3,5.322348E7,1.4300815E6,3.1001872E5,5.0161895E-3,5.94988E-4,-4.5470293E-3,4.8008256E-4,4.58881E5,2.94E2,5.4452E4,6.568221E0,2.88446E5,9.38E2,-1.7468637E-4,-4.006917E-3,-3.7704427E-3,-1.2629434E-2,-4.6083154E-3,5.248448E-4,-7.439676E-3,-2.097994E-3,4.1260887E-3,-2.9272644E-4,1.9273409E-3,8.672834E-3,1.2680544E-3,-6.2117064E-3,2.9835464E-3,-9.6943974E-4,-1.1834176E-3,-7.7888956E-3,3.0101623E-4,-3.0987978E-3],"split_indices":[52,45,45,52,5,54,45,8,45,8,32,0,57,33,56,2,54,57,5,56,0,0,0,0,9,0,0,0,0,57,0,12,9,52,2,56,0,0,52,0,0,0,5,32,28,0,0,0,0,1,0,29,53,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.7E2,3.6E1,1.46E2,2.4E1,2E1,1.6E1,6.5E1,8.1E1,1.7E1,7E0,3E0,1.7E1,1E1,6E0,5.3E1,1.2E1,5E1,3.1E1,1.5E1,2E0,5E0,2E0,3E0,1.4E1,3E0,7E0,3E0,3E0,1.5E1,3.8E1,6E0,6E0,3.4E1,1.6E1,2.7E1,4E0,9E0,6E0,1.1E1,3E0,7E0,8E0,2.6E1,1.2E1,4E0,2E0,3E0,3E0,1.6E1,1.8E1,6E0,1E1,9E0,1.8E1,3E0,3E0,3E0,5E0,1.4E1,1.2E1,9E0,3E0,9E0,7E0,3E0,1.5E1,2E0,4E0,6E0,4E0,3E0,6E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[2.6763344E-4,-3.562319E-2,1.7865828E-1,-1.8097268E-2,-2.4454215E-1,4.47172E-1,1.16943076E-1,-3.3486996E-2,1.414897E-1,-3.481418E-1,3.6788161E-3,2.790042E-2,6.3239834E-3,-1.0595338E-2,1.9547185E-1,-6.566554E-2,2.4707085E-2,1.8834342E-1,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,5.0848205E-2,2.2159144E-1,4.051122E-3,-5.639541E-2,-1.0586935E-2,-4.5325225E-3,4.210238E-2,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,-6.8116255E-2,7.0792675E-2,7.365901E-2,-2.802525E-2,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,37,-1,-1,-1,-1,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.299687E0,6.143675E-1,5.111331E-1,3.9033002E-1,4.5747256E-1,1.6556728E-1,2.925187E-1,2.7127695E-1,7.63599E-2,5.6399226E-2,0E0,0E0,0E0,1.5918176E-1,3.4319043E-2,1.1866668E-1,1.1051539E-1,3.6700696E-2,0E0,0E0,0E0,0E0,5.9108846E-2,4.338783E-2,0E0,1.3749906E-1,0E0,0E0,1.0386192E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3594824E-1,6.001374E-2,6.339806E-2,7.879637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,22,22,23,23,25,25,28,28,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,38,-1,-1,-1,-1,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,1.8000048E10,1.4164306E-3,7.918757E7,3.582157E12,8.50999E9,1E0,2.0053221E6,1.2117E4,6.1826086E0,3.6788161E-3,2.790042E-2,6.3239834E-3,5.9084116E3,7.1528664E0,4.187504E7,5.6865673E0,8.81832E5,9.482339E-4,-1.992977E-2,-7.4799582E-3,-1.0311221E-2,7.644773E-1,2.819228E-1,4.051122E-3,1E0,-1.0586935E-2,-4.5325225E-3,6.773399E0,4.1119955E-3,1.1850012E-2,2.200864E-5,8.189981E-3,3.8410614E-3,1.2328427E-2,1.2936847E4,6.7111494E8,2.7883E4,3.4288502E8,-6.325195E-3,-1.960313E-3,-6.21639E-4,7.6675043E-3,4.7318274E-3,-4.4807128E-4,-4.709818E-3,2.1525254E-3],"split_indices":[52,5,57,32,31,5,109,48,9,53,0,0,0,4,53,32,54,1,0,0,0,0,34,57,0,94,0,0,53,0,0,0,0,0,0,51,5,9,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.68E2,3.3E1,1.56E2,1.2E1,5E0,2.8E1,1.43E2,1.3E1,9E0,3E0,3E0,2E0,1.1E1,1.7E1,9.2E1,5.1E1,9E0,4E0,6E0,3E0,2E0,9E0,1.3E1,4E0,8.8E1,4E0,6E0,4.5E1,4E0,5E0,7E0,2E0,3E0,1E1,8.1E1,7E0,3.1E1,1.4E1,2.4E1,5.7E1,4E0,3E0,2.4E1,7E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[7.671295E-3,-4.4080827E-2,1.6683623E-1,-2.5097556E-2,-3.2062513E-1,2.2227801E-1,-1.2322681E-2,-1.1087152E-1,-5.909076E-3,-4.2326204E-3,-1.9872338E-2,1.3815795E-1,3.3456573E-1,-9.755224E-2,3.63379E-3,-1.355743E-1,1.2135234E-3,-9.3863435E-2,9.225092E-3,9.646266E-2,1.2028726E-2,3.6554006E-1,3.8201683E-3,-7.6256203E-3,-6.825402E-4,-9.017518E-2,-2.0942573E-1,-7.649883E-3,-5.741548E-2,5.0508834E-2,-2.6426286E-2,1.13049895E-1,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,-1.8938167E-2,9.144381E-2,1.4660759E-2,-8.3531834E-2,1.4068319E-3,1.457002E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,49,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7124388E0,8.161584E-1,5.0931644E-1,2.4293393E-1,1.6942883E-1,3.2215738E-1,1.0366744E-1,9.3960494E-2,1.6326205E-1,0E0,0E0,8.524135E-2,9.821105E-2,3.2380693E-2,0E0,5.822915E-2,0E0,3.252116E-2,1.558761E-1,4.3112665E-2,0E0,9.481287E-2,0E0,0E0,0E0,2.1951303E-2,2.0213217E-2,0E0,1.926352E-2,1.4102952E-1,1.3496363E-1,4.2491734E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.587293E-2,1.4206815E-1,5.812499E-2,9.924857E-2,0E0,3.1057805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,19,19,21,21,25,25,26,26,28,28,29,29,30,30,31,31,41,41,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,50,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8361508E3,1.8000048E10,9.298568E7,1.4862166E1,5.709406E7,5.8038013E3,9.4908775E1,3.068397E6,2.0293878E2,-4.2326204E-3,-1.9872338E-2,1.197355E6,5E1,6.97021E3,3.63379E-3,8.3949E4,1.2135234E-3,8.587809E4,7.9762895E6,1.127938E6,1.2028726E-2,6.677E3,3.8201683E-3,-7.6256203E-3,-6.825402E-4,1.3868161E5,1.0595825E3,-7.649883E-3,1.96897E5,2.495E3,4.497684E1,4.078932E5,-1.4447069E-3,2.8378414E-2,1.3659854E-2,2.5684392E-4,-5.070473E-3,-1.1850503E-2,-3.4776754E-3,-3.6095174E-3,1.1266653E-3,2.7309215E6,5.511358E1,5.8454223E0,1.6594656E3,1.4068319E-3,2.819228E-1,2.940453E-3,-2.9303525E-3,7.860316E-3,1.1795793E-3,4.343099E-3,-3.8837903E-4,-6.7108436E-3,-3.2844846E-4,1.988541E-3,8.570484E-3],"split_indices":[52,5,45,58,45,52,56,12,52,0,0,28,3,4,0,12,0,28,45,28,0,9,0,0,0,28,33,0,1,2,58,28,0,0,0,0,0,0,0,0,0,45,58,53,4,0,57,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.56E2,5E1,1.47E2,9E0,3.8E1,1.2E1,2.6E1,1.21E2,3E0,6E0,2.3E1,1.5E1,6E0,6E0,2.2E1,4E0,1.7E1,1.04E2,1.8E1,5E0,1.3E1,2E0,3E0,3E0,1.5E1,7E0,5E0,1.2E1,4.8E1,5.6E1,1.6E1,2E0,2E0,1.1E1,2E0,1.3E1,5E0,2E0,1E1,2E0,1.8E1,3E1,3.3E1,2.3E1,5E0,1.1E1,6E0,1.2E1,1.4E1,1.6E1,7E0,2.6E1,1.3E1,1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.0088654E-2,-4.6401702E-2,1.01957306E-1,-3.6083024E-2,-2.5793928E-1,3.6505505E-2,1.9830973E-1,-8.227722E-2,2.2556398E-2,-2.2164036E-3,-1.9751323E-2,1.0576863E-1,-6.8776435E-3,1.892742E-2,1.5552393E-1,-6.712239E-2,-1.4856316E-2,-5.168168E-2,6.346703E-2,6.68196E-3,1.6625825E-3,2.28483E-2,-3.8955654E-3,1.8000433E-1,1.2089454E-3,-7.764623E-2,3.5982527E-2,-1.81113E-2,-1.3062242E-1,-3.4493096E-2,1.2884454E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,-1.4065999E-1,-6.0010847E-2,-2.1526885E-4,3.771369E-3,-5.376633E-2,7.684118E-4,-8.152548E-3,-1.2452523E-3,-7.3128425E-2,5.791923E-3,1.6153444E-1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,-1,27,29,-1,-1,31,-1,33,-1,35,37,39,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.306833E-1,3.268422E-1,3.0808258E-1,4.0275842E-1,2.0142797E-1,9.3877874E-2,1.0262287E-1,2.599991E-1,2.0373048E-1,0E0,0E0,2.2404373E-2,4.5329027E-2,0E0,5.311328E-2,8.85137E-2,0E0,6.123595E-2,2.7983302E-1,0E0,0E0,9.2295585E-3,0E0,3.8849056E-2,0E0,7.1570426E-2,1.3761323E-2,2.2606054E-2,2.2841543E-2,1.1545663E-1,9.22758E-2,0E0,0E0,0E0,0E0,5.3088874E-2,9.419921E-2,0E0,0E0,2.1270107E-2,0E0,0E0,0E0,8.4302165E-2,0E0,8.3529234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,25,25,26,26,27,27,28,28,29,29,30,30,35,35,36,36,39,39,43,43,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,-1,28,30,-1,-1,32,-1,34,-1,36,38,40,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.537424E0,1.4146589E8,1.2236866E3,6.978001E2,1.5214152E5,1.1244E4,4.9695907E0,8.260109E9,3.41673E5,-2.2164036E-3,-1.9751323E-2,5.6352734E1,8.412314E0,1.892742E-2,4.8420677E-1,3.4741312E8,-1.4856316E-2,8.062193E-1,6.6217735E-2,6.68196E-3,1.6625825E-3,4.9852173E1,-3.8955654E-3,1.08E2,1.2089454E-3,8.8916E4,1.09524E5,4.737429E8,4.6E1,1E0,1.5107028E-1,2.8215116E-3,2.3378411E-4,9.7504305E-3,1.548986E-3,9.135018E0,5.580003E8,-2.1526885E-4,3.771369E-3,9.042926E3,7.684118E-4,-8.152548E-3,-1.2452523E-3,9.2312753E-1,5.791923E-3,1.234543E1,8.2920276E-4,-2.7251528E-3,-9.289472E-3,-5.610352E-4,-4.569528E-3,5.572257E-4,-4.2964076E-3,-1.2974755E-4,-7.459397E-3,9.182858E-3,2.793861E-4],"split_indices":[42,45,52,52,33,9,54,5,28,0,0,58,54,0,38,7,0,27,57,0,0,58,0,3,0,1,29,7,3,94,38,0,0,0,0,58,5,0,0,33,0,0,0,27,0,54,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.53E2,4.9E1,1.47E2,6E0,3E1,1.9E1,8.2E1,6.5E1,3E0,3E0,1.1E1,1.9E1,2E0,1.7E1,7.8E1,4E0,2.3E1,4.2E1,7E0,4E0,1.4E1,5E0,1.4E1,3E0,7.1E1,7E0,1.7E1,6E0,1.7E1,2.5E1,4E0,1E1,1.2E1,2E0,1.4E1,5.7E1,4E0,3E0,8E0,9E0,4E0,2E0,1.4E1,3E0,1.9E1,6E0,6E0,8E0,2.4E1,3.3E1,3E0,5E0,8E0,6E0,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.7253997E-2,-2.9623577E-2,9.3434505E-2,-2.2292453E-1,-1.1781458E-2,1.2750259E-1,-1.9930653E-1,-3.3444623E-3,-1.2307007E-2,1.7842796E-2,-6.838117E-2,8.674666E-2,3.2339618E-1,-8.01149E-2,-1.8067626E-2,-4.725675E-3,3.552434E-2,-1.2093892E-1,-3.2683358E-2,2.175943E-3,1.3742693E-1,2.7282907E-2,2.0839193E-1,-5.710558E-3,-1.8373676E-4,4.8699785E-2,-4.864452E-3,-1.5385593E-1,3.9161125E-4,1.3277833E-3,-5.8980435E-2,6.1291102E-2,-4.3464635E-2,4.9143904E-3,1.6467385E-1,1.2900099E-2,5.3549507E-3,1.10007E-1,1.44394925E-2,-1.135703E-2,-1.014236E-1,-6.185148E-3,-1.7182616E-3,1.0451842E-1,7.325834E-4,-7.334734E-2,9.2549727E-4,-2.225592E-3,2.6441324E-3,2.1629092E-1,7.352542E-2,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,35,-1,-1,37,-1,39,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,-1,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.848589E-1,4.077926E-1,7.5345045E-1,2.8820753E-2,1.8565977E-1,4.9887896E-1,1.3421485E-1,0E0,0E0,1.5181659E-1,6.734279E-2,2.3972067E-1,1.832602E-1,1.6607814E-2,0E0,0E0,1.1910778E-1,6.492545E-2,3.9455958E-2,6.2021095E-2,1.2556028E-1,0E0,2.1007508E-2,0E0,0E0,1.2233533E-1,0E0,2.6357263E-2,0E0,0E0,2.2924516E-2,1.8414091E-2,2.566748E-2,2.000419E-2,1.20536804E-1,0E0,0E0,4.2832226E-2,5.8858946E-2,0E0,2.2556297E-2,0E0,0E0,1.198972E-2,0E0,1.4668021E-2,0E0,0E0,0E0,3.9754987E-2,2.4631858E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,22,22,25,25,27,27,30,30,31,31,32,32,33,33,34,34,37,37,38,38,40,40,43,43,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,36,-1,-1,38,-1,40,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,-1,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,1.6770158E-4,1.2055723E8,1.9E1,7.183439E0,8.258924E3,3.312524E6,-3.3444623E-3,-1.2307007E-2,1.056E3,7.8206635E0,1E0,1.6232848E7,1.5574268E7,-1.8067626E-2,-4.725675E-3,4.759155E1,2.2235178E1,2.7E1,9.32784E5,1.1114967E-3,2.7282907E-2,1.5307794E2,-5.710558E-3,-1.8373676E-4,1.2029E4,-4.864452E-3,2.9239766E-2,3.9161125E-4,1.3277833E-3,1.1079511E1,8.78E2,3.0558723E-1,2.69336E3,8.637931E1,1.2900099E-2,5.3549507E-3,4E1,4.711463E5,-1.135703E-2,3.4996462E-1,-6.185148E-3,-1.7182616E-3,5.88E2,7.325834E-4,9.042926E3,9.2549727E-4,-2.225592E-3,2.6441324E-3,1.5213319E0,4.6379595E6,-2.2551669E-5,6.250936E-3,3.541708E-3,-5.390881E-4,-2.492868E-4,-6.239353E-3,1.3532604E-3,7.1243057E-3,7.096313E-5,-4.6198396E-3,1.2171428E-2,5.206903E-3,1.152214E-3,5.8121914E-3],"split_indices":[52,39,45,3,53,52,29,0,0,2,53,109,1,1,0,0,56,56,3,29,38,0,56,0,0,9,0,57,0,0,54,0,39,4,58,0,0,10,51,0,57,0,0,0,0,33,0,0,0,57,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.18E2,7.2E1,9E0,1.09E2,6.5E1,7E0,2E0,7E0,7.2E1,3.7E1,5.5E1,1E1,5E0,2E0,9E0,6.3E1,1.4E1,2.3E1,2.1E1,3.4E1,2E0,8E0,3E0,2E0,5.8E1,5E0,1.1E1,3E0,7E0,1.6E1,9E0,1.2E1,6E0,2.8E1,4E0,4E0,2E1,3.8E1,3E0,8E0,3E0,1.3E1,4E0,5E0,8E0,4E0,3E0,3E0,1.7E1,1.1E1,3E0,1.7E1,1.1E1,2.7E1,2E0,6E0,2E0,2E0,2E0,6E0,1.2E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.821953E-2,-5.8290947E-2,5.3941235E-2,-1.5808642E-2,-4.884159E-2,-6.909937E-2,1.1836419E-1,-1.4827322E-2,-8.093995E-2,-2.6398048E-2,-1.7481435E-2,4.550153E-2,1.5699527E-1,-6.594871E-2,3.347421E-2,-9.2051856E-2,-6.509846E-3,6.4362705E-2,-7.304391E-2,8.8149734E-2,-1.3496414E-2,1.2912967E-2,1.3428916E-1,-1.2285006E-1,-2.9534832E-2,9.6844934E-2,3.1477788E-3,-6.093492E-2,-1.3656648E-1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,-1.4490403E-1,5.8896925E-2,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.758437E-2,1.9397938E-1,-1.2398881E-3,-9.721017E-3,-4.254263E-2,2.629008E-3,6.676271E-3,9.927296E-5,-2.7347464E-2,5.9932888E-2,-1.4137745E-1,-3.3839725E-2,-1.0662791E-3,-1.5445867E-1,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,1.0476206E-1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,-1,-1,57,59,-1,-1,61,-1,-1,-1,63,65,67,69,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.496537E-1,2.9141405E-1,5.4719037E-1,0E0,1.2872437E-1,2.878313E-1,1.1952078E-1,1.4789282E-1,4.9851447E-2,9.7840816E-2,0E0,4.396437E-2,3.570825E-2,5.645959E-2,5.92712E-2,6.509176E-2,3.340755E-2,1.0004889E-2,1.5075365E-1,1.9013055E-2,1.2354755E-2,0E0,7.2952986E-2,7.7276245E-2,2.294885E-2,3.7963897E-2,3.9735876E-2,6.714282E-2,3.9696425E-2,0E0,0E0,0E0,0E0,4.4010997E-2,2.6443997E-2,0E0,0E0,0E0,0E0,4.359097E-2,9.496659E-3,0E0,0E0,1.2809919E-2,0E0,0E0,0E0,2.1849878E-2,1.7759107E-2,1.1468753E-2,4.3800417E-2,0E0,9.729177E-3,0E0,0E0,0E0,0E0,1.81911E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,39,39,40,40,43,43,47,47,48,48,49,49,50,50,52,52,57,57],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,-1,-1,58,60,-1,-1,62,-1,-1,-1,64,66,68,70,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,5.243E3,1E0,-1.5808642E-2,7.6420575E6,6.3639E4,4.0679638E6,2.802166E2,5.754E3,1.4364E4,-1.7481435E-2,1.9071735E6,7.573E3,1.822764E4,4.585366E-2,3E0,5.178571E0,7.406885E8,1.0807868E7,9.1797715E-1,3.508923E6,1.2912967E-2,1.56E4,1.17E2,3.65102E5,1.7027842E8,2.2583E4,1.8716404E10,7E1,-3.0996199E-3,2.52936E-3,1.4422777E-3,5.4682554E-3,2.6654E4,1.00152E5,5.6911367E-3,7.653452E-4,-2.207138E-3,1.4452778E-3,7.8114974E-1,4.2414474E0,-1.2398881E-3,-9.721017E-3,1.0437421E8,2.629008E-3,6.676271E-3,9.927296E-5,1.8465776E8,8.3154816E2,1.1272E4,2.7048333E10,-1.0662791E-3,7.64764E5,-3.624584E-3,-1.1286887E-2,-2.7536892E-4,6.0997666E-3,3.6E1,-1.1361084E-3,1.0606567E-2,4.456083E-3,-5.731937E-4,-3.3375588E-3,8.2396844E-4,-2.8590374E-3,-3.2639557E-5,4.692864E-3,-3.0147352E-3,-8.596752E-3,3.1731827E-3,-2.4135245E-3,-7.990612E-3,-2.4851307E-3,6.2205694E-3,1.2989099E-3],"split_indices":[52,9,109,0,45,10,32,52,2,10,0,32,9,33,57,8,54,7,1,27,32,0,2,10,1,32,10,31,10,0,0,0,0,2,2,0,0,0,0,27,57,0,0,7,0,0,0,5,4,29,31,0,9,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,1.21E2,6.7E1,3E0,1.18E2,2.3E1,4.4E1,5.8E1,6E1,2.1E1,2E0,1.6E1,2.8E1,2.8E1,3E1,5.2E1,8E0,7E0,1.4E1,9E0,7E0,3E0,2.5E1,1E1,1.8E1,9E0,2.1E1,3.2E1,2E1,4E0,4E0,5E0,2E0,9E0,5E0,6E0,3E0,4E0,3E0,1.4E1,1.1E1,5E0,5E0,1.6E1,2E0,6E0,3E0,1.4E1,7E0,7E0,2.5E1,3E0,1.7E1,6E0,3E0,3E0,2E0,1.1E1,3E0,8E0,3E0,8E0,8E0,6E0,8E0,3E0,4E0,3E0,4E0,3E0,2.2E1,1.5E1,2E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.81344E-3,-2.7871855E-2,1.0370427E-1,-1.6869712E-1,-1.2145221E-2,1.3553725E-1,-1.1319494E-2,-6.7867875E-2,-2.5078225E-1,1.1689531E-2,-7.812123E-2,2.2571503E-1,4.7493696E-2,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,-2.2323536E-2,5.4143388E-2,-9.7720034E-2,1.1783024E-4,1.8657329E-2,1.5811518E-1,9.230261E-2,-1.6328286E-3,-4.379925E-2,5.0750427E-2,1.1329648E-1,1.5480721E-2,-1.1520765E-1,-1.8632218E-3,8.1961945E-2,9.722441E-3,1.8049882E-3,8.147564E-3,-2.9605734E-3,1.3710238E-3,-1.09546825E-1,-2.015405E-2,-1.4587996E-3,6.54756E-2,1.231898E-1,7.869888E-4,7.951029E-2,-2.6912533E-2,-1.2801106E-1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,37,39,41,43,45,-1,47,-1,49,-1,-1,-1,51,53,-1,55,57,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8306116E-1,3.744514E-1,4.811502E-1,1.2368178E-1,2.4308157E-1,2.989232E-1,0E0,2.157684E-2,3.9408505E-2,1.6582306E-1,6.4956516E-2,1.4187676E-1,4.637814E-2,0E0,0E0,0E0,0E0,1.0265903E-1,1.14883915E-1,2.9888332E-2,0E0,0E0,3.1216264E-2,7.44717E-2,2.112299E-2,7.5522915E-2,1.9793212E-2,1.7225742E-2,8.900131E-2,4.644516E-2,0E0,3.160474E-2,0E0,1.4609658E-2,0E0,0E0,0E0,6.457342E-2,9.696657E-2,0E0,1.6802516E-2,1.4416695E-2,0E0,1.5758365E-2,1.763527E-2,4.027137E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,37,37,38,38,40,40,41,41,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,38,40,42,44,46,-1,48,-1,50,-1,-1,-1,52,54,-1,56,58,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0687964E3,5.178571E0,1.3578947E1,8.062193E-1,7.3808947E0,5.861021E7,-1.1319494E-2,3.662451E6,2.692487E6,5.716463E2,2.770862E8,6.661E3,2.1516852E5,-4.5884307E-3,7.9767354E-4,-6.215621E-3,-1.5752804E-2,3.9E1,1.26881E5,1.6259928E1,1.1783024E-4,1.8657329E-2,6.917027E3,4.078932E5,6.47008E1,1.4300815E6,2.1229417E-4,2.4025E4,3.6E1,4.2845196E1,-1.8632218E-3,5.59076E3,9.722441E-3,6.804E3,8.147564E-3,-2.9605734E-3,1.3710238E-3,2.07533E5,1.2670352E1,-1.4587996E-3,1.43E2,2.458E3,7.869888E-4,1E0,4.983287E-2,5.3E1,1.0392449E-3,6.0705324E-3,-8.103621E-4,-2.4922127E-3,2.000923E-3,-1.8509465E-3,-9.083123E-3,-2.53733E-4,-9.378999E-3,5.4745646E-3,1.5612462E-3,2.5095162E-3,6.8118707E-3,4.5920443E-3,-6.973587E-5,1.7163511E-3,-1.9447147E-3,2.480336E-4,-6.858595E-3],"split_indices":[52,54,54,27,53,45,0,1,1,52,7,9,33,0,0,0,0,3,29,54,0,0,4,28,56,32,38,9,3,56,0,4,0,9,0,0,0,1,54,0,10,2,0,89,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.69E2,4.1E1,1.6E1,1.53E2,3.8E1,3E0,8E0,8E0,1.13E2,4E1,1.8E1,2E1,6E0,2E0,4E0,4E0,6.3E1,5E1,3.2E1,8E0,4E0,1.4E1,1E1,1E1,4.9E1,1.4E1,1.9E1,3.1E1,2.4E1,8E0,6E0,8E0,5E0,5E0,3E0,7E0,1.2E1,3.7E1,2E0,1.2E1,1.7E1,2E0,1.2E1,1.9E1,2.2E1,2E0,4E0,2E0,2E0,3E0,7E0,5E0,3.5E1,2E0,4E0,8E0,4E0,1.3E1,1E1,2E0,3E0,1.6E1,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.1425839E-2,-5.037939E-2,8.030544E-2,-3.6732145E-2,-1.940263E-1,2.1132613E-2,5.40595E-2,-8.394264E-2,-1.2433366E-2,-2.411859E-3,-2.3667553E-1,-7.583577E-2,1.2632278E-1,-9.6114695E-2,3.007908E-3,3.00965E-2,-5.099469E-2,-1.4926101E-2,-5.8065807E-3,-2.905628E-2,-1.687885E-2,9.731201E-4,1.4231151E-1,-1.0521082E-1,6.9725106E-4,-3.51654E-2,6.765553E-2,-1.512059E-2,-1.1731359E-1,-7.190969E-2,1.2920251E-3,7.135396E-2,1.5594196E-1,-5.4826945E-2,-1.3231695E-1,1.7662038E-3,-8.1669055E-2,-1.3722117E-3,8.081185E-2,-6.1539885E-2,5.3886116E-2,-1.3972227E-1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,1.7186429E-1,7.422348E-2,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,33,-1,35,37,39,41,43,-1,45,47,49,51,-1,53,-1,55,57,59,61,-1,-1,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1869767E-1,2.692873E-1,5.338764E-1,1.4808676E-1,6.220767E-2,0E0,5.446193E-1,8.3999515E-2,1.4421546E-1,0E0,3.7372112E-2,2.5613624E-1,6.0055733E-2,4.269564E-2,0E0,1.05328426E-1,1.0766618E-1,0E0,0E0,4.71834E-2,0E0,0E0,1.9822717E-2,4.4642508E-2,0E0,5.613244E-2,3.6327116E-2,1.0261665E-1,3.450933E-2,6.169112E-2,0E0,1.8945247E-2,2.107352E-2,2.1910824E-2,4.9286067E-2,0E0,3.790599E-2,0E0,1.6026601E-2,3.391008E-2,3.6774833E-2,1.0349661E-2,0E0,0E0,0E0,0E0,0E0,1.815337E-2,3.8996745E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,19,19,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,38,38,39,39,40,40,41,41,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,34,-1,36,38,40,42,44,-1,46,48,50,52,-1,54,-1,56,58,60,62,-1,-1,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3453049E3,8.67241E9,1.4164306E-3,4.9054803E2,4.944487E7,2.1132613E-2,1E0,1.4864864E0,9.785595E6,-2.411859E-3,6.096518E0,4.4816768E8,1.6463401E3,9.741151E-1,3.007908E-3,2.4242616E5,8.78E2,-1.4926101E-2,-5.8065807E-3,3.0687964E3,-1.687885E-2,9.731201E-4,2.8626094E5,5.2646E4,6.9725106E-4,1.3585858E1,1.29E2,3.307175E-2,3.170081E6,7.6599895E9,1.2920251E-3,2.6295085E3,3.8014093E2,6.492337E0,2.0289855E-2,1.7662038E-3,1.1565725E0,-1.3722117E-3,1.2075471E7,1.2067E4,1.2117E4,4E1,-7.376369E-4,-1.4767719E-4,-7.492347E-3,-3.5948947E-4,5.0772545E-3,3.2014463E0,1.0807868E7,-4.9714665E-3,-1.0194686E-3,-8.58595E-3,-3.5773448E-3,-5.9391144E-3,4.5709414E-4,4.4098096E-3,4.4545234E-4,-4.22489E-3,3.7413978E-4,5.0192047E-3,-1.489486E-4,-7.460342E-3,-2.111153E-3,9.036839E-3,3.245346E-3,-1.344871E-3,6.4269425E-3],"split_indices":[52,5,57,55,45,0,109,57,45,0,53,44,52,27,0,28,0,0,0,52,0,0,28,1,0,56,0,57,1,5,0,4,58,53,57,0,42,0,48,9,9,3,0,0,0,0,0,35,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.4E2,5.9E1,1.29E2,1.1E1,3E0,5.6E1,4.3E1,8.6E1,3E0,8E0,2E1,3.6E1,4E1,3E0,4.1E1,4.5E1,4E0,4E0,1.8E1,2E0,5E0,3.1E1,3.7E1,3E0,1.5E1,2.6E1,3E1,1.5E1,1E1,8E0,6E0,2.5E1,1.4E1,2.3E1,6E0,9E0,3E0,2.3E1,1.8E1,1.2E1,1.2E1,3E0,6E0,4E0,2E0,4E0,2E1,5E0,5E0,9E0,1.2E1,1.1E1,6E0,3E0,2E1,3E0,1.3E1,5E0,6E0,6E0,1E1,2E0,1.7E1,3E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[4.6584974E-3,-1.20946085E-2,1.9950889E-1,-1.0169338E-1,3.8820638E-3,3.8749582E-1,8.888968E-2,-5.8970988E-2,-1.9482663E-1,-9.452532E-2,1.551003E-2,2.2913612E-2,7.1646026E-3,1.3861427E-1,-4.1111445E-4,-8.537462E-2,2.566781E-2,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,-1.250599E-1,-3.5212412E-2,4.8972625E-2,2.1056903E-3,7.941639E-3,-4.738872E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,-8.0461E-2,-1.16186915E-2,8.0707185E-2,-5.17336E-2,5.499706E-2,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,-4.0313277E-2,4.2702768E-2,1.3941039E-1,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1],"loss_changes":[7.009837E-1,2.8382987E-1,3.1906998E-1,1.07877135E-1,1.9484903E-1,5.082476E-2,5.925162E-2,5.135636E-2,2.9909045E-2,1.1930962E-1,2.5971475E-1,0E0,0E0,1.16812885E-2,0E0,1.3007693E-2,1.56768E-2,0E0,0E0,0E0,6.1911702E-2,1.2099819E-1,1.173628E-1,0E0,0E0,1.1479644E-2,0E0,0E0,0E0,1.0476619E-2,0E0,6.582728E-2,7.438664E-2,8.7449014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.335335E-2,8.5745096E-2,9.818435E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,20,20,21,21,22,22,25,25,29,29,31,31,32,32,33,33,42,42,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1.4862166E1,2E0,1E0,4.060294E0,9.886E3,6.627576E7,1.3448806E8,8.3949E4,7.56E2,4.2658337E2,2.2913612E-2,7.1646026E-3,1.625E3,-4.1111445E-4,8.351808E-1,9.282E3,-2.5829435E-3,-1.1083549E-2,4.8585758E-3,5.936149E1,2.124077E1,6.7948157E-1,2.1056903E-3,7.941639E-3,2.2970924E-2,-5.541865E-3,-9.5942366E-4,3.7775538E-3,1E0,-1.16186915E-2,1.2623029E7,1.38E2,1.6974416E6,-7.206752E-3,-3.8637866E-3,-2.8744648E-4,-4.379885E-4,-4.5082546E-3,6.56106E-3,-2.6481685E-3,-8.974547E-3,5.138372E8,3.810132E-2,1.2951E4,-1.5701434E-3,-7.706692E-3,8.290276E-3,1.6421576E-3,8.192997E-3,3.4559402E-3],"split_indices":[52,58,8,89,53,9,45,7,12,0,52,0,0,0,0,27,9,0,0,0,56,58,34,0,0,27,0,0,0,64,0,12,0,28,0,0,0,0,0,0,0,0,7,27,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.97E2,1.6E1,2.9E1,1.68E2,5E0,1.1E1,2.1E1,8E0,1.7E1,1.51E2,3E0,2E0,7E0,4E0,1.6E1,5E0,2E0,6E0,2E0,1.5E1,6E1,9.1E1,2E0,5E0,8E0,8E0,3E0,2E0,1.2E1,3E0,7E0,5.3E1,8.9E1,2E0,4E0,4E0,2E0,1E1,5E0,2E0,3E0,5E1,7.9E1,1E1,4.8E1,2E0,4E0,7.5E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-9.7492535E-4,2.4562754E-2,-1.2644652E-1,1.0812377E-2,3.050939E-1,-1.6741821E-1,3.276021E-2,-4.319679E-2,5.309041E-2,1.990497E-2,7.832562E-3,-1.8774702E-1,3.1424365E-3,3.0188295E-3,-1.7861315E-3,-2.2519317E-2,-9.010733E-2,8.511394E-2,-1.3702277E-2,-1.5522692E-1,-1.8752424E-2,-3.5709422E-2,8.45564E-2,-1.2079619E-1,5.7729695E-4,3.1312965E-2,1.5568885E-1,2.2016572E-2,-9.091352E-2,-1.9204946E-1,-1.8670933E-3,-5.38557E-2,2.1705188E-2,5.703837E-3,6.2500284E-4,-1.3545471E-1,-3.5225743E-4,8.462742E-2,-2.3649544E-2,1.15246095E-1,1.1550232E-2,-3.8249083E-2,4.199171E-2,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,27,29,-1,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,-1,53,55,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.63182E-1,6.601755E-1,2.3378861E-1,3.7905455E-1,5.4006636E-2,1.4417273E-1,1.8542241E-2,6.878023E-2,2.0008749E-1,0E0,0E0,1.0724431E-1,0E0,0E0,0E0,7.6461144E-2,7.027824E-2,2.343359E-1,8.749639E-2,9.590763E-2,0E0,5.013643E-2,1.2541946E-2,2.7317584E-2,0E0,1.1042236E-1,6.1801553E-2,2.8090071E-2,2.3553073E-2,3.9512277E-2,0E0,1.5470095E-2,4.2363483E-2,0E0,0E0,2.9321939E-2,0E0,1.5620723E-2,5.383783E-2,5.4485172E-2,0E0,1.0460219E-2,2.2912873E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,37,37,38,38,39,39,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,28,30,-1,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,-1,54,56,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,8.861832E3,1.4539318E-1,4.2658337E2,1.04E3,1E0,1.3507566E8,1.3839568E1,2.4934822E7,1.990497E-2,7.832562E-3,3.074E3,3.1424365E-3,3.0188295E-3,-1.7861315E-3,1.91126E5,7.407092E0,5.370622E5,1.06403265E1,8.66076E1,-1.8752424E-2,3.8E1,4.61E2,5.979E3,5.7729695E-4,9.6494156E-1,1.8907035E3,4.602015E0,1.27718E6,3.27E2,-1.8670933E-3,2.4397528E8,4.3E1,5.703837E-3,6.2500284E-4,1.7218965E2,-3.5225743E-4,8.047134E4,4.649E3,7.4245725E0,1.1550232E-2,8.062193E-1,3.706834E-4,-2.4384235E-3,-8.245662E-3,-3.7567283E-3,-1.0675064E-2,1.0927089E-3,-2.871555E-3,-2.5570283E-3,3.1826142E-3,-2.28771E-3,-7.949358E-3,2.1110226E-3,5.3865993E-3,-4.201074E-3,9.783958E-4,2.164214E-3,7.724535E-3,7.033023E-4,-3.3196472E-3,-6.131631E-4,3.2016633E-3],"split_indices":[45,4,38,52,0,94,45,56,45,0,0,0,0,0,0,29,53,28,54,56,0,3,0,2,0,57,4,53,28,0,0,31,10,0,0,52,0,33,2,54,0,27,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.71E2,3.4E1,1.64E2,7E0,2.7E1,7E0,7.2E1,9.2E1,3E0,4E0,2.5E1,2E0,5E0,2E0,5.1E1,2.1E1,6.2E1,3E1,2.3E1,2E0,4.6E1,5E0,1.6E1,5E0,3.6E1,2.6E1,2.1E1,9E0,1.7E1,6E0,3.5E1,1.1E1,3E0,2E0,1.4E1,2E0,1.8E1,1.8E1,1.9E1,7E0,5E0,1.6E1,7E0,2E0,4E0,1.3E1,2E0,3.3E1,4E0,7E0,4E0,1E1,8E0,1E1,7E0,1.1E1,8E0,1.1E1,2E0,3E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[6.475444E-3,-1.306737E-2,1.7832997E-1,2.339655E-3,-1.5614031E-1,2.8744474E-1,5.3003386E-2,-1.6489496E-2,6.7189105E-2,-2.4180317E-1,1.30160665E-2,3.55285E-1,3.888601E-3,9.1399415E-4,6.7161336E-3,-1.0044899E-2,-1.06382035E-2,1.3008821E-2,1.2666671E-1,-2.8520468E-1,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,-1.6523695E-2,1.5263823E-1,-4.6523552E-2,7.135857E-2,1.455868E-1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,-2.6452178E-2,7.834649E-2,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,-1,33,35,37,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9082713E-1,4.0816608E-1,2.7193904E-1,2.06246E-1,2.639749E-1,1.2648803E-1,2.9406311E-2,1.6842958E-1,1.2080774E-1,5.8472157E-2,2.9649243E-2,6.8154216E-2,0E0,0E0,0E0,1.3743296E-1,0E0,7.626754E-2,3.144613E-2,2.2196114E-2,0E0,0E0,0E0,0E0,0E0,1.1887014E-1,4.850761E-2,1.9805137E-2,2.4082582E-2,1.8560827E-2,0E0,0E0,0E0,7.060035E-2,5.3812273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,-1,34,36,38,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.993295E3,1.8000048E10,5.9337012E7,1.5085194E0,1.5461028E12,6.6737964E2,6.543368E6,1.3201905E0,7.7567725E2,4.577256E3,2.6023E4,1.04E3,3.888601E-3,9.1399415E-4,6.7161336E-3,2.3329E4,-1.06382035E-2,2.2906428E5,6.4487465E6,5.709406E7,-4.0733316E-3,-1.7463822E-3,4.388677E-3,2.1531254E-2,7.666843E-3,1E0,9.39E2,3E1,5.1E2,3.5E1,1.1704119E-3,-5.23084E-3,-1.5603794E-2,4.56567E7,3.2834557E-1,1.1368303E-2,1.0122965E-3,-4.5291707E-3,-3.228057E-4,6.00191E-3,1.1701281E-3,9.721572E-3,4.7948454E-3,-7.490489E-4,-4.086666E-3,-2.4904788E-3,5.322558E-3],"split_indices":[4,5,45,42,31,56,29,39,52,4,10,0,0,0,0,2,0,28,47,45,0,0,0,0,0,94,0,3,0,3,0,0,0,45,27,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.84E2,2E1,1.67E2,1.7E1,1E1,1E1,1.3E2,3.7E1,1.1E1,6E0,7E0,3E0,8E0,2E0,1.27E2,3E0,2E1,1.7E1,8E0,3E0,4E0,2E0,4E0,3E0,1.23E2,4E0,1E1,1E1,1.4E1,3E0,2E0,6E0,1.12E2,1.1E1,2E0,2E0,4E0,6E0,4E0,6E0,5E0,9E0,9.5E1,1.7E1,2E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0223907E-3,-2.7888758E-2,9.6585326E-2,-1.5584077E-2,-2.1336152E-1,1.575512E-1,-3.900375E-2,-3.563399E-2,2.183398E-2,-3.6633124E-3,-1.6371118E-2,1.8702927E-1,-5.3675026E-3,2.0629173E-2,-1.4183015E-1,-3.0176764E-2,-7.741039E-3,5.2294556E-2,-3.702995E-2,5.3999938E-2,2.235795E-1,4.8674736E-2,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,-4.079073E-2,3.0565633E-2,6.8358756E-3,7.473907E-2,2.9559305E-3,-5.9157714E-2,-3.6169306E-4,3.6848264E-3,1.9294271E-2,1.8000335E-1,4.2783604E-3,7.0099917E-4,-3.1242667E-2,-1.5811709E-1,-3.2913506E-2,1.07815154E-1,2.1910788E-3,-7.367315E-4,9.088779E-2,5.4238463E-4,-5.502338E-3,-4.669484E-3,8.802838E-2,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,-1,27,29,31,33,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8288884E-1,3.6638862E-1,4.1119415E-1,1.1556303E-1,1.445714E-1,2.890743E-1,1.0103525E-1,6.2558055E-2,9.8760046E-2,0E0,0E0,1.3865662E-1,0E0,3.1503566E-2,8.29455E-2,6.378441E-2,0E0,3.621982E-2,4.469773E-2,1.12352E-2,1.0943866E-1,1.083583E-2,0E0,0E0,0E0,8.803986E-2,7.710456E-2,1.1590129E-2,2.3634195E-2,0E0,4.6466663E-2,0E0,0E0,0E0,4.115665E-2,0E0,0E0,7.36302E-2,2.4497524E-2,2.882525E-2,2.4574421E-2,0E0,0E0,1.8441036E-2,0E0,0E0,1.4278462E-2,2.0401236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,25,25,26,26,27,27,28,28,30,30,34,34,37,37,38,38,39,39,40,40,43,43,46,46,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,-1,28,30,32,34,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0455479E3,1.775894E10,9.040637E7,2.0053221E6,6.0671224E7,1.3578947E1,5.3868E4,5.138372E8,3.8279E5,-3.6633124E-3,-1.6371118E-2,1.2737E4,-5.3675026E-3,9.123462E-1,2.6022625E5,2.0997734E8,-7.741039E-3,5.716463E2,8.75E2,5.22E2,1.6064256E-2,3.9E1,-3.4253343E-3,-8.741373E-4,-1.2639221E-2,5.1383884E7,4.016202E1,7.25E2,1.5562991E6,2.9559305E-3,2.7102E4,-3.6169306E-4,3.6848264E-3,1.9294271E-2,3.383436E5,4.2783604E-3,7.0099917E-4,2.47565E6,5.88E2,3.887674E6,1.631E4,2.1910788E-3,-7.367315E-4,7.3586698E0,5.4238463E-4,-5.502338E-3,1.617E3,9.851E3,1.0264958E-2,-1.112503E-3,-7.25428E-3,-2.043904E-3,-1.0009566E-2,1.2150757E-3,-4.098286E-3,7.0754387E-3,4.5055454E-4,5.084229E-3,6.7321456E-4,2.6064524E-3,-1.409226E-3,9.1176725E-4,6.585282E-3],"split_indices":[4,5,45,48,45,54,10,7,29,0,0,2,0,27,33,7,0,52,0,0,57,3,0,0,0,32,58,0,47,0,10,0,0,0,28,0,0,28,0,45,9,0,0,53,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.61E2,4.8E1,1.52E2,9E0,3.3E1,1.5E1,9.9E1,5.3E1,5E0,4E0,3E1,3E0,1E1,5E0,9.6E1,3E0,3.5E1,1.8E1,7E0,2.3E1,8E0,2E0,3E0,2E0,8.2E1,1.4E1,1.2E1,2.3E1,3E0,1.5E1,2E0,5E0,3E0,2E1,3E0,5E0,7.7E1,5E0,8E0,6E0,4E0,8E0,1.8E1,5E0,7E0,8E0,6E0,1.4E1,7.3E1,4E0,2E0,3E0,4E0,4E0,4E0,2E0,1.5E1,3E0,2E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[8.825459E-3,-2.0716727E-2,9.291196E-2,-9.286735E-3,-2.2815323E-1,4.1376296E-2,1.8566836E-1,-4.1899815E-2,3.8700562E-2,-1.1017629E-3,-1.425366E-2,-6.1172745E-3,7.967702E-2,2.2176522E-1,7.0497654E-2,-1.486095E-1,-2.7861884E-2,-2.4466898E-2,7.862809E-2,2.2698525E-2,-3.884307E-3,1.145321E-1,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,-1.2128689E-2,-1.3520806E-1,1.7996103E-2,-7.628645E-2,5.7908777E-2,9.716253E-3,-1.2009995E-3,3.937936E-2,2.097745E-3,6.499496E-3,-2.237607E-2,1.0090606E-1,-8.440877E-3,5.7535426E-4,5.9123063E-3,-9.376124E-3,-4.9414644E-3,-3.266944E-4,8.224998E-3,4.116531E-2,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,-1,49,-1,-1,51,53,-1,-1,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1613337E-1,3.6301482E-1,2.5045186E-1,2.3167856E-1,9.979531E-2,6.5972775E-2,6.335485E-2,1.2768199E-1,1.5325102E-1,0E0,0E0,3.7913833E-2,3.5779223E-2,8.4248126E-2,1.1696836E-2,5.5736482E-2,1.3178307E-1,5.418279E-2,8.121562E-2,1.124549E-2,0E0,1.054576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.3339594E-2,5.8268234E-2,4.0797673E-2,1.8643163E-2,5.442866E-2,0E0,0E0,9.034543E-3,0E0,0E0,8.4523104E-2,1.790364E-2,0E0,0E0,0E0,1.3487471E-2,0E0,0E0,0E0,5.1540002E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,29,29,30,30,31,31,32,32,33,33,36,36,39,39,40,40,44,44,48,48],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,-1,50,-1,-1,52,54,-1,-1,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5585917E0,1.4233672E8,1.6059814E3,1.2534044E3,8.667E3,5.433479E2,2.7304773E0,8.35E2,7.157224E0,-1.1017629E-3,-1.425366E-2,1.1350955E0,1.2224206E7,8E0,2.8352614E9,4.1749332E4,1.448907E6,7.794012E7,5.2086E4,2.9539914E4,-3.884307E-3,7.810231E4,1.1850562E-3,1.327848E-2,3.662109E-3,2.5773517E-4,4.9530035E-3,-3.5206839E-3,-1.1919544E-2,1E0,2.18E3,5.491E3,1.47245E7,6.489288E6,9.716253E-3,-1.2009995E-3,8.842206E7,2.097745E-3,6.499496E-3,1.5989196E7,2.94E2,-8.440877E-3,5.7535426E-4,5.9123063E-3,1.424506E6,-4.9414644E-3,-3.266944E-4,8.224998E-3,9.1487586E-1,3.3849503E-3,3.7329676E-4,7.924952E-5,-3.7214062E-3,7.4307784E-4,6.8088E-3,4.399627E-4,-3.2917524E-3,9.512646E-4,6.0532438E-3],"split_indices":[41,45,52,4,2,52,42,2,54,0,0,57,50,8,5,32,1,45,2,33,0,33,0,0,0,0,0,0,0,94,0,9,1,45,0,0,5,0,0,45,0,0,0,0,29,0,0,0,27,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.53E2,5.3E1,1.46E2,7E0,3.5E1,1.8E1,8.7E1,5.9E1,2E0,5E0,1.6E1,1.9E1,1.3E1,5E0,9E0,7.8E1,2.3E1,3.6E1,1.2E1,4E0,1.1E1,8E0,9E0,4E0,2E0,3E0,6E0,3E0,6.9E1,9E0,1.3E1,1E1,3.2E1,4E0,3E0,9E0,3E0,8E0,6.4E1,5E0,7E0,2E0,2E0,1.1E1,7E0,3E0,3E0,2.9E1,4E0,5E0,4.5E1,1.9E1,2E0,3E0,9E0,2E0,2.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.4462977E-3,-2.893987E-2,6.9793515E-2,-3.6714904E-2,1.3322261E-2,2.450053E-2,1.3456057E-1,-3.0940516E-2,-1.5242134E-2,6.13726E-2,-2.383746E-2,1.9235724E-1,5.9276864E-2,-1.44404005E-2,-1.02318145E-1,6.015054E-3,1.4756152E-3,-5.1880803E-2,1.3078791E-3,2.2946529E-1,2.957695E-3,4.6630064E-3,-1.3928295E-3,-7.5536236E-2,1.9160501E-3,-1.6728845E-1,-1.4484057E-2,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,-4.1924756E-2,-1.2142303E-2,9.4770074E-2,-1.28825195E-2,-1.1275523E-1,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,27,-1,29,-1,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9038587E-1,3.7482685E-1,1.5511182E-1,2.329789E-1,0E0,6.027959E-2,8.43229E-2,1.7518334E-1,0E0,3.29639E-2,2.2726193E-2,4.4647485E-2,3.807029E-2,1.2305999E-1,1.5736201E-1,0E0,0E0,1.2245912E-2,0E0,1.3593733E-2,0E0,0E0,0E0,8.423267E-2,1.2907745E-1,5.515635E-2,1.3991938E-2,0E0,0E0,0E0,0E0,0E0,5.6269135E-2,7.5751625E-2,1.7076403E-2,0E0,2.499877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,23,23,24,24,25,25,26,26,32,32,33,33,34,34,36,36],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,28,-1,30,-1,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.9028345E2,1.3453049E3,2.8159826E0,1.3322261E-2,2.4608E4,8.812601E-1,7.361366E7,-1.5242134E-2,7.614E3,2.449092E5,2.5501368E9,8.2731545E-1,6.974669E7,8.66076E1,6.015054E-3,1.4756152E-3,3.239E3,1.3078791E-3,3.6E1,2.957695E-3,4.6630064E-3,-1.3928295E-3,8.35E2,5.506676E11,1E0,2.313233E6,-3.9189835E-3,-3.4692822E-4,1.2538498E-2,4.2452826E-3,-8.78853E-3,3.54E2,8E0,3.3478114E-1,-1.28825195E-2,3.890129E5,-3.1040355E-3,3.2526033E-4,5.500743E-4,-4.4452474E-3,3.286314E-4,-2.84778E-3,1.0040635E-3,5.5941343E-3,-1.1847992E-3,-6.78995E-3],"split_indices":[42,56,52,35,0,9,57,45,0,10,28,7,27,7,56,0,0,2,0,3,0,0,0,2,31,64,1,0,0,0,0,0,0,8,27,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.54E2,5.3E1,1.51E2,3E0,3.2E1,2.1E1,1.49E2,2E0,1.8E1,1.4E1,1.1E1,1E1,1.22E2,2.7E1,5E0,1.3E1,9E0,5E0,8E0,3E0,7E0,3E0,2.5E1,9.7E1,1.5E1,1.2E1,5E0,4E0,6E0,2E0,5E0,2E1,8.5E1,1.2E1,4E0,1.1E1,3E0,9E0,1E1,1E1,6.1E1,2.4E1,3E0,9E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.4985325E-2,-3.9879307E-2,5.6348145E-2,-2.3429228E-2,-1.0598515E-1,-1.7923791E-2,9.930136E-2,-5.1214058E-2,3.0147022E-2,-3.333782E-2,-1.8125312E-1,3.6635974E-3,-3.763992E-2,1.5140602E-1,5.2652568E-2,-6.5377824E-2,1.8297803E-2,1.5232481E-2,8.540196E-3,-7.670071E-2,8.6073264E-2,-2.6167667E-1,-8.275727E-4,-9.86009E-3,-4.455716E-3,8.21394E-3,2.7782663E-3,8.663035E-2,1.0011745E-2,-1.3291118E-1,-5.3235017E-2,-3.8194293E-3,3.9224032E-2,7.2662863E-3,-3.2975117E-3,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,-1.8245183E-1,-3.2175884E-2,-1.0584831E-1,4.2874664E-3,-1.5806843E-3,9.24955E-3,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,-1,-1,-1,43,45,47,49,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.769355E-1,1.6824591E-1,1.7724186E-1,1.9059703E-1,1.6346598E-1,4.1423395E-2,7.709977E-2,8.389847E-2,9.2703044E-2,9.462805E-2,1.9369146E-1,0E0,2.6057394E-2,1.6584098E-2,2.810949E-2,5.03276E-2,3.3456847E-2,1.0235344E-1,0E0,3.1034954E-2,2.0551417E-2,4.1319788E-2,0E0,1.02055315E-2,0E0,0E0,0E0,1.1555165E-2,9.544139E-3,5.704157E-2,6.4181656E-2,0E0,4.842038E-2,0E0,6.882077E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.789392E-3,5.7711136E-2,5.315976E-2,0E0,0E0,3.5858765E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,27,27,28,28,29,29,30,30,32,32,34,34,48,48,49,49,50,50,53,53],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,-1,-1,-1,44,46,48,50,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,7.294459E7,3.239E3,1.2943141E3,1.3907746E10,7.751E3,6.4512783E-1,4.3E1,4.1983E4,1.753919E3,3.7284137E2,3.6635974E-3,8.724138E0,2.2151E4,2E0,8.35E2,3.99E2,2.73782E5,8.540196E-3,2.7864855E11,1.7837838E0,9.301866E7,-8.275727E-4,5.0900903E0,-4.455716E-3,8.21394E-3,2.7782663E-3,3.1E1,2.2621576E-1,4.569361E7,7E0,-3.8194293E-3,1.2623029E7,7.2662863E-3,1.2001309E0,-9.812847E-4,-5.950967E-3,2.4274121E-4,6.7387247E-3,-3.8657528E-3,-1.4468934E-2,-1.9695107E-3,6.973678E-4,1.9571974E-3,5.7721194E-3,2.795007E-3,-4.3908268E-4,-6.320425E-4,4.1749332E4,1E0,3.677609E-1,4.2874664E-3,-1.5806843E-3,1E1,-7.2308956E-3,-4.37504E-3,-1.1163702E-2,-2.2052925E-3,2.9578062E-3,1.6106452E-3,-6.1602243E-3,-4.6624234E-3,9.012807E-4],"split_indices":[42,45,2,55,5,9,57,3,2,52,58,0,54,9,8,2,2,29,0,31,57,7,0,56,0,0,0,3,38,5,8,0,12,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,32,79,27,0,0,3,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.56E2,5.4E1,1.26E2,3E1,2E1,3.4E1,8.3E1,4.3E1,1.6E1,1.4E1,3E0,1.7E1,1.5E1,1.9E1,6.9E1,1.4E1,4E1,3E0,1.2E1,4E0,9E0,5E0,1.2E1,5E0,1.2E1,3E0,1E1,9E0,9E0,6E1,2E0,1.2E1,4E0,3.6E1,6E0,6E0,2E0,2E0,2E0,7E0,5E0,7E0,5E0,5E0,2E0,7E0,3E0,6E0,4.4E1,1.6E1,7E0,5E0,3.4E1,2E0,3E0,3E0,3.9E1,5E0,2E0,1.4E1,2E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.3111593E-4,8.332628E-3,-1.9431697E-1,-3.4451315E-3,1.5696207E-1,-1.8588053E-3,-1.294587E-2,-3.1280745E-2,2.9869221E-2,2.6512837E-1,5.53925E-2,-1.5708588E-1,-2.058258E-2,1.0180442E-2,9.531762E-2,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,2.4204077E-3,-7.460408E-2,-4.4569806E-3,1.9710528E-2,1.22437425E-1,-5.1242317E-4,-1.3157908E-2,7.7513896E-2,-1.2132418E-1,-5.7141267E-2,9.711912E-2,6.6852947E-3,1.4493192E-3,1.5084128E-1,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5398465E-1,3.5764718E-1,9.0578735E-2,1.7712615E-1,1.5010995E-1,0E0,0E0,1.367387E-1,1.1113774E-1,3.353101E-2,1.3905274E-2,3.2876134E-2,1.20718196E-1,6.775201E-2,5.870013E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.17382E-2,1.6727448E-2,0E0,6.287142E-2,3.7616774E-2,0E0,7.575147E-2,3.0201033E-2,3.716676E-2,8.603498E-3,2.5892913E-2,4.6929307E-2,0E0,2.140811E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,34,34],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4233672E8,4.9192E4,8.901E3,4.2350553E5,8.812601E-1,-1.8588053E-3,-1.294587E-2,5.160874E0,9.855582E5,1.8136424E7,6.804E3,5.3012047E-2,7.501344E0,2.884462E0,3.9247762E6,1.6594345E-2,5.9856502E-3,-6.087151E-4,3.72685E-3,-4.1098916E-3,-1.216638E-2,1E0,2.6239506E5,-4.4569806E-3,1.5324128E6,2.8244882E6,-5.1242317E-4,1.6278354E7,3.57E2,4.888E3,1.3593E4,1.7417817E-1,1.5489E4,1.4493192E-3,1.02952E5,7.659061E-4,-2.8136016E-3,1.1194615E-3,6.238613E-3,-8.147945E-3,-1.9706316E-4,-3.0716902E-3,1.8188971E-4,-1.0515761E-4,6.121922E-3,1.7290254E-3,-1.089946E-3,2.6066836E-3,8.629353E-3],"split_indices":[45,2,2,47,57,0,0,54,28,1,9,57,53,53,47,0,0,0,0,0,0,79,28,0,32,32,0,45,0,2,10,38,9,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.03E2,8E0,1.89E2,1.4E1,3E0,5E0,1.03E2,8.6E1,6E0,8E0,7E0,9.6E1,6.7E1,1.9E1,3E0,3E0,2E0,6E0,5E0,2E0,6.8E1,2.8E1,5E0,6.2E1,1.5E1,4E0,5.7E1,1.1E1,6E0,2.2E1,8E0,5.4E1,4E0,1.1E1,3.5E1,2.2E1,6E0,5E0,4E0,2E0,2E1,2E0,2E0,6E0,2.7E1,2.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[7.821103E-3,-2.8981427E-3,1.7867728E-1,-5.675793E-2,1.8588716E-2,2.1517694E-1,-1.2397172E-4,-3.6930315E-2,-1.2052149E-2,-1.4890818E-2,4.163438E-2,2.7627808E-1,2.6501652E-3,-5.6514364E-2,6.0741283E-2,-4.3238487E-2,2.5888829E-2,-2.5325203E-2,7.3175E-2,1.5455141E-2,5.614201E-3,-7.2515686E-3,-4.0423278E-2,-8.0662005E-4,6.1275316E-3,1.2608571E-2,-6.090714E-2,-9.619795E-3,7.656288E-2,7.5304456E-2,-7.165898E-2,1.6621768E-1,5.4205015E-2,-1.64469E-2,-1.2827884E-1,2.7917859E-3,-9.4076915E-4,1.9582137E-3,-7.1012415E-2,3.3104515E-3,-3.469917E-2,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,-2.7355356E-2,-1.6309094E-1,1.9180217E-1,2.4875687E-3,5.2126553E-3,3.92498E-2,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,27,29,31,-1,-1,-1,33,-1,-1,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,55,-1,57,-1,-1,-1,-1,59,61,63,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7230206E-1,2.2325236E-1,7.992381E-2,1.9902351E-1,1.0709499E-1,8.315757E-2,0E0,1.0098582E-1,0E0,6.701185E-2,1.7503278E-1,9.8798275E-3,0E0,5.797288E-2,4.7433127E-2,3.4317195E-2,4.39213E-2,1.3128516E-1,8.98312E-2,0E0,0E0,0E0,7.7935606E-2,0E0,0E0,1.3974639E-2,2.9431812E-2,3.1568956E-2,1.9552685E-2,2.7546454E-2,7.250317E-2,1.6699761E-2,3.3773705E-2,4.245451E-2,5.6236163E-2,0E0,0E0,0E0,2.1543108E-2,0E0,1.2517829E-2,0E0,0E0,0E0,0E0,2.2041425E-2,2.2109985E-2,1.6221613E-2,0E0,0E0,3.1004522E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,38,38,40,40,45,45,46,46,47,47,50,50],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,28,30,32,-1,-1,-1,34,-1,-1,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,56,-1,58,-1,-1,-1,-1,60,62,64,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,4.039435E-3,4.6E1,2.0604518E8,5.470729E2,2.0867E4,-1.2397172E-4,1E0,-1.2052149E-2,3.5E1,1E0,9.299267E9,2.6501652E-3,9.342733E7,3.52777E5,6.027375E0,5.935E3,1.4364E4,7.573E3,1.5455141E-2,5.614201E-3,-7.2515686E-3,6.6932364E7,-8.0662005E-4,6.1275316E-3,1.625E3,6.558106E-3,6.693824E5,1.0275E4,9.123462E-1,5.4827127E0,1.81025E5,7.03482E5,1.3696512E7,9.64E2,2.7917859E-3,-9.4076915E-4,1.9582137E-3,4.053E3,3.3104515E-3,4E1,8.518124E-4,5.500228E-3,1.0277939E-3,6.70411E-3,2.3541917E3,3.800374E6,2.6807916E0,2.4875687E-3,5.2126553E-3,2.1225555E-1,2.304336E-3,-1.8019777E-3,-8.6027635E-3,5.578618E-4,-1.3813657E-3,-4.6085645E-3,-3.017669E-3,1.432435E-4,2.5471908E-3,-2.1899668E-3,-3.4273877E-3,-1.1304262E-2,1.1100004E-2,3.280095E-3,-1.4401524E-3,2.445197E-3],"split_indices":[2,38,3,7,52,9,0,79,0,3,109,12,0,5,1,53,10,10,9,0,0,0,45,0,0,0,38,48,9,27,57,29,47,45,0,0,0,0,11,0,3,0,0,0,0,4,1,39,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.91E2,1.1E1,5.4E1,1.37E2,9E0,2E0,5E1,4E0,5.6E1,8.1E1,6E0,3E0,4.2E1,8E0,3.3E1,2.3E1,2.6E1,5.5E1,4E0,2E0,5E0,3.7E1,4E0,4E0,8E0,2.5E1,1.4E1,9E0,8E0,1.8E1,8E0,4.7E1,3E1,7E0,3E0,5E0,2E0,2.3E1,3E0,1.1E1,4E0,5E0,5E0,3E0,1.3E1,5E0,6E0,2E0,9E0,3.8E1,7E0,2.3E1,5E0,2E0,9E0,1.4E1,6E0,5E0,2E0,1.1E1,3E0,2E0,4E0,2E0,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[1.6695334E-3,-2.430477E-2,6.987881E-2,-1.628185E-1,-1.665332E-2,3.3794872E-2,1.4406632E-1,-2.0858636E-3,-2.262213E-1,-1.3216189E-3,-8.5313104E-2,7.368082E-2,5.320639E-3,2.1326382E-1,6.0462188E-2,-1.3826551E-2,-4.5224247E-3,-3.0901669E-2,5.3741764E-2,-4.580613E-2,-1.4649896E-2,6.6577974E-3,3.5978813E-2,1.8743427E-2,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,-1.9552542E-2,-8.157249E-3,-1.2044725E-2,9.7219266E-2,-6.857402E-2,-3.44251E-4,5.0435625E-2,-1.1778983E-3,-4.252783E-4,5.2595764E-2,-4.5224298E-2,2.5154488E-2,-2.6128737E-3,4.6722338E-2,-8.63488E-4,1.1892799E-1,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,-1,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,-1,49,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7906808E-1,1.6156235E-1,1.5307292E-1,5.1149473E-2,1.556826E-1,4.617256E-2,9.702435E-2,0E0,1.2178838E-2,2.0037158E-1,2.1631023E-1,3.5536163E-2,2.652885E-2,6.27754E-2,2.7581971E-2,0E0,0E0,1.2084925E-1,1.2416096E-1,2.0679474E-2,0E0,0E0,1.1661252E-2,2.092692E-2,0E0,0E0,0E0,0E0,0E0,8.721569E-2,0E0,4.656056E-2,6.699793E-2,2.1412484E-2,0E0,8.723641E-3,0E0,0E0,1.359814E-2,4.531187E-2,9.016034E-2,0E0,1.4617754E-2,0E0,5.3301275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,22,22,23,23,29,29,31,31,32,32,33,33,35,35,38,38,39,39,40,40,42,42,44,44],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,-1,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,-1,50,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4726219E0,2.3125E0,1.3173E4,5.34E2,1.3845824E1,1.5489E4,8E0,-2.0858636E-3,4.233395E6,8.3764136E2,2.1024984E5,5.2560944E1,3.0794094E0,2.2047469E4,2.4711658E5,-1.3826551E-2,-4.5224247E-3,8.260109E9,7.157224E0,8.071663E5,-1.4649896E-2,6.6577974E-3,7.4809E4,1.5383295E6,-3.3986142E-3,1.2334748E-2,1.7033103E-3,5.5007753E-3,3.2015343E-4,3.7E1,-8.157249E-3,5E1,7.157903E7,1.17E2,-3.44251E-4,2.398624E8,-1.1778983E-3,-4.252783E-4,2.8314E4,1.4218562E5,7.47E2,-2.6128737E-3,5.5326223E9,-8.63488E-4,2.4039803E5,-1.4044909E-3,-5.345484E-3,-2.8586562E-4,3.175353E-3,1.1863445E-3,5.3685773E-3,1.2675842E-3,-2.8240036E-3,3.4695186E-3,-2.0587323E-3,-3.126678E-4,3.88823E-3,7.2713248E-3,1.570065E-3],"split_indices":[42,53,2,0,54,9,8,0,1,52,33,58,42,4,33,0,0,5,54,48,0,0,29,47,0,0,0,0,0,3,0,3,7,10,0,7,0,0,9,28,0,0,5,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.54E2,5.8E1,7E0,1.47E2,4E1,1.8E1,3E0,4E0,1.21E2,2.6E1,1.6E1,2.4E1,9E0,9E0,2E0,2E0,7.9E1,4.2E1,2.3E1,3E0,5E0,1.1E1,2.1E1,3E0,7E0,2E0,4E0,5E0,7.4E1,5E0,1.7E1,2.5E1,1.4E1,9E0,9E0,2E0,1.2E1,9E0,4.7E1,2.7E1,1E1,7E0,4E0,2.1E1,8E0,6E0,2E0,7E0,7E0,2E0,7E0,4E1,1.6E1,1.1E1,3E0,4E0,1.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[3.2811111E-3,-1.1759641E-2,9.609429E-2,-5.231051E-3,-1.926002E-1,4.223107E-2,1.7549825E-1,-2.3685513E-2,3.7570078E-2,-7.758012E-4,-1.34779E-2,7.0535643E-3,2.5339068E-3,1.425658E-2,4.615414E-3,-1.5669417E-2,-9.816772E-3,4.6593472E-2,-5.953628E-3,1.4607337E-3,-3.911037E-2,-1.08244635E-1,-6.703496E-3,1.3439576E-3,8.3413675E-2,-3.8560897E-3,3.702331E-5,-1.4499448E-1,1.2573712E-3,4.405412E-3,-4.881462E-2,2.0919163E-3,-1.956045E-2,9.7872205E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,23,-1,-1,25,27,29,31,33,-1,-1,35,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.80326E-1,2.0324215E-1,1.1234322E-1,1.3358834E-1,8.498618E-2,7.30211E-2,7.642406E-2,1.6523767E-1,7.928742E-2,0E0,0E0,0E0,1.7196054E-2,0E0,0E0,9.389689E-2,0E0,8.1526615E-2,0E0,0E0,9.659185E-3,5.3020775E-2,4.9334954E-2,2.086177E-2,4.4374347E-2,0E0,0E0,2.1213919E-2,0E0,4.3695264E-2,4.650379E-2,0E0,1.4095474E-2,3.8369343E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,17,17,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,32,32,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,24,-1,-1,26,28,30,32,34,-1,-1,36,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.918757E7,1.4623123E8,1.7227725E0,1.0892118E3,8.667E3,1.1944959E8,3.0291426E-1,6.6932364E7,1.8323593E1,-7.758012E-4,-1.34779E-2,7.0535643E-3,8.0756325E-1,1.425658E-2,4.615414E-3,8.35E2,-9.816772E-3,7.316901E0,-5.953628E-3,1.4607337E-3,1.467E3,6.1E1,1.1103951E1,4.8759037E1,2.3524794E7,-3.8560897E-3,3.702331E-5,1.0907971E6,1.2573712E-3,1.643E4,6.87E2,2.0919163E-3,6.879445E1,1.9652087E-2,-1.1772866E-3,-9.170651E-3,-2.6494178E-3,1.302618E-3,-9.048412E-4,-3.0659462E-4,-4.827822E-3,-2.0832724E-3,8.0024626E-4,1.4781184E-3,6.0287802E-3],"split_indices":[32,45,39,52,2,32,38,45,57,0,0,0,27,0,0,2,0,54,0,0,0,3,54,56,32,0,0,28,0,9,0,0,56,39,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.72E2,2.7E1,1.67E2,5E0,1.7E1,1E1,1.17E2,5E1,2E0,3E0,4E0,1.3E1,3E0,7E0,1.13E2,4E0,4.8E1,2E0,8E0,5E0,9E0,1.04E2,2.2E1,2.6E1,2E0,3E0,7E0,2E0,8.3E1,2.1E1,7E0,1.5E1,2.3E1,3E0,4E0,3E0,4.2E1,4.1E1,1.2E1,9E0,9E0,6E0,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.0387296E-3,-1.8574167E-2,8.5663736E-2,-8.023423E-3,-1.4978984E-1,1.4511184E-1,-3.3413577E-3,-2.7192928E-2,3.1242838E-2,-1.0438316E-2,-2.401442E-3,3.2174537E-1,9.6915334E-2,2.1752674E-2,-7.842911E-3,-2.143232E-2,-6.5535344E-3,9.6770816E-2,-2.4400586E-3,1.8892974E-2,6.1257724E-3,7.6597073E-4,1.1642241E-1,-6.7683046E-5,7.212856E-2,-7.260881E-2,-7.379258E-3,1.2818316E-1,1.6344215E-3,-7.859502E-2,1.6685827E-2,2.2511675E-3,1.3107267E-1,6.177627E-3,-1.2163549E-3,-2.2213712E-2,-1.08554E-1,2.886258E-2,-3.7549652E-2,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,4.6269923E-2,-4.119254E-2,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,-1,27,29,-1,-1,-1,31,-1,33,35,37,39,-1,41,43,-1,45,-1,-1,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1510934E-1,2.230747E-1,2.6504415E-1,1.14509515E-1,6.739807E-2,2.2421151E-1,8.667014E-2,6.0260966E-2,1.10594735E-1,0E0,0E0,2.9250383E-2,3.775531E-2,2.2251856E-2,0E0,6.994447E-2,0E0,2.9437363E-2,5.0833475E-2,0E0,0E0,0E0,1.449576E-2,0E0,3.511203E-2,3.5629295E-2,8.637136E-2,1.2054026E-2,0E0,1.4433451E-2,4.9849793E-2,0E0,1.9013852E-2,0E0,0E0,1.4445642E-2,3.5027638E-2,4.9891964E-2,7.3443145E-2,0E0,0E0,0E0,0E0,2.8110322E-2,1.6329382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,22,22,24,24,25,25,26,26,27,27,29,29,30,30,32,32,35,35,36,36,37,37,38,38,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,-1,28,30,-1,-1,-1,32,-1,34,36,38,40,-1,42,44,-1,46,-1,-1,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6503105E3,8.67241E9,5.9337012E7,6.90321E2,6.1826086E0,5.0655737E0,1.4623123E8,6.0073395E8,1.10597E5,-1.0438316E-2,-2.401442E-3,2E0,2.819228E-1,4.078932E5,-7.842911E-3,1.142E3,-6.5535344E-3,7.393122E1,1.6595541E1,1.8892974E-2,6.1257724E-3,7.6597073E-4,3.559158E6,-6.7683046E-5,2.1516852E5,5.12334E9,5.62E2,7.44905E0,1.6344215E-3,1E0,6.1093975E6,2.2511675E-3,6.797394E8,6.177627E-3,-1.2163549E-3,1.1086122E3,5.244755E-3,1.5989196E7,2.7816156E5,2.4467357E-3,7.3541827E-3,-2.1812765E-4,-5.2240393E-3,5.6E0,7.3376025E6,1.2208747E-3,7.0284875E-3,9.789177E-4,-2.6175354E-3,-2.5902812E-3,-8.534599E-3,2.538475E-3,-1.4030307E-3,-3.984272E-3,4.6120404E-5,-2.3583905E-3,2.932896E-3,-4.417455E-3,-3.395892E-4],"split_indices":[52,5,45,52,53,54,45,7,29,0,0,8,57,28,0,2,0,58,56,0,0,0,1,0,33,31,0,54,0,16,32,0,7,0,0,33,57,45,28,0,0,0,0,54,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.61E2,4.9E1,1.5E2,1.1E1,2.9E1,2E1,1.01E2,4.9E1,6E0,5E0,5E0,2.4E1,1.8E1,2E0,9.7E1,4E0,1.6E1,3.3E1,3E0,2E0,5E0,1.9E1,1.3E1,5E0,2E1,7.7E1,1E1,6E0,6E0,2.7E1,4E0,1.5E1,3E0,2E0,9E0,1.1E1,3.5E1,4.2E1,3E0,7E0,2E0,4E0,1.8E1,9E0,2E0,1.3E1,4E0,5E0,7E0,4E0,2.5E1,1E1,1.9E1,2.3E1,2E0,1.6E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.4796249E-3,-2.8328227E-2,4.2443145E-2,-1.2839314E-2,-1.4659679E-1,6.714284E-2,-6.827072E-2,-2.624263E-2,3.197302E-2,-5.994708E-2,-2.0733023E-1,1.3621603E-2,5.6769222E-2,-2.7255906E-2,-8.979017E-3,-2.0926751E-2,-6.972742E-3,6.93552E-2,-2.6764661E-2,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,3.61498E-2,1.4122447E-1,1.9069649E-3,-5.1106434E-2,-1.4165286E-2,-5.6748535E-3,8.369632E-2,-1.472121E-3,-3.9843167E-3,1.1822013E-3,8.772388E-2,2.698954E-3,1.7185566E-1,1.427893E-3,-4.393489E-3,-4.8403983E-4,-7.145611E-2,5.5866083E-4,1.9922133E-3,5.4703923E-3,1.7163315E-1,4.553083E-2,1.823748E-2,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,33,35,-1,37,39,-1,41,-1,-1,-1,43,45,47,-1,-1,-1,49,51,-1,-1,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6417843E-1,2.5274518E-1,2.370248E-1,7.531766E-2,7.177171E-2,1.3860276E-1,7.124599E-2,5.672677E-2,6.5532796E-2,1.3237113E-2,3.143558E-2,0E0,1.1331123E-1,2.2602845E-2,0E0,5.8513593E-2,0E0,2.8243646E-2,3.5750594E-2,0E0,0E0,0E0,0E0,9.637425E-2,3.9510548E-2,0E0,1.52250845E-2,7.427207E-2,0E0,1.5040562E-2,0E0,0E0,0E0,7.007395E-2,7.020194E-2,1.9569606E-2,0E0,0E0,0E0,2.5643677E-2,5.1912256E-2,0E0,0E0,1.0038182E-2,2.4570972E-2,2.6236098E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,18,18,23,23,24,24,26,26,27,27,29,29,33,33,34,34,35,35,39,39,40,40,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,34,36,-1,38,40,-1,42,-1,-1,-1,44,46,48,-1,-1,-1,50,52,-1,-1,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,4.56567E7,1.09495016E8,5.470729E2,7.683833E-1,1.7727281E2,1.4623123E8,5.138372E8,1.1424464E3,3.97E2,2.1E1,1.3621603E-2,4.2245E4,1E0,-8.979017E-3,9.85054E5,-6.972742E-3,2.4848485E0,1.2707859E3,3.475193E-4,-4.063641E-3,-2.8559028E-3,-1.1748931E-2,2.1637352E7,3.832274E2,1.9069649E-3,8.427037E6,1.056E3,-5.6748535E-3,6.685237E0,-1.472121E-3,-3.9843167E-3,1.1822013E-3,7.573E3,1.28885E1,4.759155E1,1.427893E-3,-4.393489E-3,-4.8403983E-4,7.8206635E0,4.570007E7,1.9922133E-3,5.4703923E-3,5.142232E-1,2.3033286E5,2.249985E5,-5.951127E-3,2.44071E-3,9.539238E-3,-4.6187406E-3,-4.2527405E-4,4.0088654E-3,-4.020263E-4,9.830915E-3,3.1049498E-3,2.7317004E-4,4.124331E-3,1.9737524E-3,-8.5179566E-4],"split_indices":[52,45,45,52,27,33,45,7,4,0,3,0,2,8,0,29,0,57,4,0,0,0,0,45,56,0,47,2,0,54,0,0,0,9,54,56,0,0,0,53,5,0,0,57,33,33,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.38E2,8.4E1,1.23E2,1.5E1,6.9E1,1.5E1,9.5E1,2.8E1,7E0,8E0,2E0,6.7E1,1.2E1,3E0,9.2E1,3E0,1.7E1,1.1E1,2E0,5E0,2E0,6E0,5.5E1,1.2E1,3E0,9E0,8.7E1,5E0,1.5E1,2E0,5E0,6E0,2.1E1,3.4E1,9E0,3E0,4E0,5E0,1.7E1,7E1,7E0,8E0,6E0,1.5E1,3.1E1,3E0,2E0,7E0,1.2E1,5E0,6E0,6.4E1,4E0,2E0,8E0,7E0,1.9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.5098556E-3,-3.430795E-2,4.6132654E-2,-7.989392E-2,-7.3299413E-3,6.539497E-2,-1.2817919E-1,-5.846227E-2,-1.0367309E-2,4.5335226E-2,-3.8782444E-2,4.477679E-2,1.7191277E-1,3.5083215E-4,-1.0261507E-2,1.7973488E-3,-6.9773324E-2,6.968256E-2,-1.4760661E-3,-5.74948E-2,8.712715E-3,1.1792199E-1,2.4033781E-2,2.3350367E-1,2.4967322E-3,2.0957861E-2,-8.140613E-2,-2.9711837E-3,8.504071E-2,-2.0592897E-3,-7.8736946E-2,-3.0431528E-2,3.5283656E-3,1.3522897E-3,1.7078307E-1,-1.0893706E-2,4.7292955E-2,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,-9.249218E-2,1.0750794E-3,7.799928E-4,1.05605684E-1,-2.1099192E-3,1.9261859E-3,-2.492124E-2,-1.10814184E-1,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,-3.874176E-2,3.5721097E-2,3.354856E-4,7.9686604E-2,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,29,31,33,35,37,-1,39,41,-1,43,45,47,49,-1,-1,51,53,55,-1,-1,-1,-1,57,-1,-1,59,-1,-1,61,63,-1,-1,-1,-1,65,67,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0736163E-1,1.5191185E-1,2.669213E-1,1.18796796E-1,1.3261785E-1,1.4603654E-1,9.106535E-2,4.6816275E-2,0E0,5.738108E-2,4.4938266E-2,8.8200994E-2,6.973371E-2,0E0,0E0,0E0,4.075806E-2,5.1372252E-2,0E0,4.223033E-2,3.9660476E-2,5.719474E-2,3.9509647E-2,2.0871669E-2,0E0,1.7594527E-2,3.99065E-2,0E0,2.8111845E-2,2.0630281E-2,4.2121127E-2,1.1639008E-2,0E0,0E0,1.40551925E-2,2.734653E-2,3.7402496E-2,0E0,0E0,0E0,0E0,5.2814364E-2,0E0,0E0,8.88367E-3,0E0,0E0,9.187069E-3,3.1335637E-2,0E0,0E0,0E0,0E0,2.286155E-2,2.1192856E-2,0E0,1.3821408E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,41,41,44,44,47,47,48,48,53,53,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,30,32,34,36,38,-1,40,42,-1,44,46,48,50,-1,-1,52,54,56,-1,-1,-1,-1,58,-1,-1,60,-1,-1,62,64,-1,-1,-1,-1,66,68,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,4.039435E-3,1.3917024E8,2.0604518E8,1.2903E4,5.4349E4,1.5214152E5,2.9270105E2,-1.0367309E-2,1.3845824E1,4.311809E5,1.20171E5,1.1406721E2,3.5083215E-4,-1.0261507E-2,1.7973488E-3,2.1013361E-1,9.33E2,-1.4760661E-3,5.948464E0,1.3012535E1,4.361E3,5.590909E0,1.671E3,2.4967322E-3,4E1,4.1211517E1,-2.9711837E-3,6.0775385E0,2.5848E4,1.2753862E1,5.8348556E0,3.5283656E-3,1.3522897E-3,1.57727E3,1.96E4,4.485647E5,1.3433593E-2,4.126197E-3,3.5126023E-3,-1.812695E-3,6.258335E5,1.0750794E-3,7.799928E-4,3.0597075E8,-2.1099192E-3,1.9261859E-3,2E0,1.7261968E7,-3.5149371E-3,-1.071705E-4,3.7613807E-3,1.0289142E-2,8.062193E-1,6.75741E5,3.354856E-4,1.184E3,-2.264212E-3,-6.587371E-3,5.8977976E-3,2.2915348E-3,-2.2845466E-3,5.301434E-4,-7.1209576E-3,-2.1539058E-3,5.8829127E-4,-3.505032E-3,4.2553237E-3,-6.2278885E-4,1.6649213E-3,5.0394568E-3],"split_indices":[52,38,45,7,9,2,33,33,0,54,28,29,56,0,0,0,27,2,0,53,56,2,53,0,0,3,58,0,54,9,56,54,0,0,4,2,28,0,0,0,0,32,0,0,7,0,0,8,45,0,0,0,0,27,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.23E2,7.6E1,4.5E1,7.8E1,6.9E1,7E0,4E1,5E0,2.9E1,4.9E1,5.9E1,1E1,3E0,4E0,4E0,3.6E1,2.2E1,7E0,3.5E1,1.4E1,1.2E1,4.7E1,6E0,4E0,4E0,3.2E1,2E0,2E1,1E1,2.5E1,9E0,5E0,5E0,7E0,1.9E1,2.8E1,4E0,2E0,2E0,2E0,2.9E1,3E0,5E0,1.5E1,5E0,5E0,1E1,1.5E1,3E0,6E0,3E0,4E0,1.2E1,7E0,1.3E1,1.5E1,1.5E1,1.4E1,1.1E1,4E0,6E0,4E0,9E0,6E0,5E0,7E0,3E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.6206973E-3,-1.9603977E-2,4.7420494E-2,-4.8373155E-2,9.0342695E-3,7.200922E-2,-6.0652774E-2,-2.3101505E-2,-7.6871574E-2,8.582439E-2,-5.1944912E-3,5.357398E-2,1.6620213E-1,1.5464444E-2,-1.2918839E-1,4.415707E-2,-5.9842244E-2,-5.267666E-4,-1.0224566E-1,5.1139877E-3,-2.7146392E-5,7.1116164E-2,-2.1878945E-2,1.2166778E-1,4.110889E-2,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,-3.735069E-2,-1.0302123E-2,5.6916097E-4,4.160516E-3,-7.803266E-2,1.388256E-3,-1.3427517E-1,-4.7179796E-2,5.5395655E-4,5.7911566E-3,-3.5666935E-2,2.1849289E-3,1.78514E-3,9.610851E-3,1.667288E-2,9.926257E-2,6.1937334E-4,-3.6489156E-3,-5.62302E-2,-5.528736E-3,-7.2530344E-2,-9.035339E-3,-3.0848335E-3,5.726469E-4,-9.4292425E-2,-1.1822653E-2,4.17147E-2,-1.5671453E-2,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,47,-1,49,51,-1,-1,53,-1,-1,-1,55,57,-1,-1,59,-1,61,-1,-1,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.23844E-1,1.1413506E-1,2.1164352E-1,4.791811E-2,7.69298E-2,1.0222992E-1,8.024894E-2,9.65454E-2,5.252196E-2,1.8357664E-2,7.7948034E-2,4.1682094E-2,4.1363686E-2,1.208929E-2,5.2706182E-2,1.7784836E-2,4.241229E-2,0E0,3.423077E-2,0E0,0E0,3.0106395E-2,4.7635924E-2,4.4303104E-2,6.655512E-2,0E0,0E0,0E0,0E0,1.0375681E-2,0E0,0E0,0E0,9.898692E-3,0E0,3.0951291E-2,1.0299623E-2,0E0,0E0,5.7596285E-2,0E0,0E0,0E0,2.9002367E-2,3.170462E-2,0E0,0E0,2.2735469E-2,0E0,9.1954805E-3,0E0,0E0,0E0,2.8296992E-2,2.1532139E-2,1.2805883E-2,1.26161575E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,29,29,33,33,35,35,36,36,39,39,43,43,44,44,47,47,49,49,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,48,-1,50,52,-1,-1,54,-1,-1,-1,56,58,-1,-1,60,-1,62,-1,-1,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,3.1749217E4,9.820072E7,1.4455165E7,8.637E3,1.6974416E6,1.3611247E5,1.1086122E3,6.9163686E-1,7.105731E0,3.9986582E4,6.518218E6,1.884E3,9.276886E1,2.1000721E0,1.9789175E2,9.946E3,-5.267666E-4,2.19E2,5.1139877E-3,-2.7146392E-5,3.067E3,1.3411796E3,5.62E2,1.6992E4,1.149749E-2,3.8995354E-3,-1.3426562E-3,2.278284E-3,2.012E3,-1.0302123E-2,5.6916097E-4,4.160516E-3,1.6126542E8,1.388256E-3,6.7E1,7.0229135E6,5.5395655E-4,5.7911566E-3,2.4861076E-1,2.1849289E-3,1.78514E-3,9.610851E-3,6.9089854E-1,7.438459E6,6.1937334E-4,-3.6489156E-3,7.774228E-2,-5.528736E-3,7.856E3,-9.035339E-3,-3.0848335E-3,5.726469E-4,1.0092749E8,1.4519928E1,1.8156171E3,1.521636E5,8.722917E-3,2.9762653E-3,-4.2856215E-3,-3.5389513E-4,-1.4286939E-3,-5.2737207E-3,-3.6758598E-4,-5.955131E-3,-1.4111706E-3,1.3209392E-3,9.315341E-4,3.5615226E-3,1.0936625E-3,-1.7058869E-3],"split_indices":[52,33,45,45,9,28,33,33,27,53,33,45,0,56,57,52,10,0,10,0,0,2,4,0,2,0,0,0,0,0,0,0,0,5,0,10,32,0,0,39,0,0,0,27,1,0,0,57,0,9,0,0,0,7,56,52,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.37E2,7.7E1,6.8E1,6.9E1,6.3E1,1.4E1,3.7E1,3.1E1,1E1,5.9E1,5.4E1,9E0,7E0,7E0,1.3E1,2.4E1,9E0,2.2E1,8E0,2E0,1E1,4.9E1,7E0,4.7E1,4E0,5E0,3E0,4E0,4E0,3E0,8E0,5E0,2E1,4E0,1.3E1,9E0,5E0,5E0,4.1E1,8E0,4E0,3E0,3.4E1,1.3E1,2E0,2E0,1.4E1,6E0,7E0,6E0,7E0,2E0,1.1E1,3E1,1.9E1,1.5E1,3E0,1E1,8E0,6E0,4E0,3E0,3E0,8E0,2.1E1,9E0,1.2E1,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[4.1845648E-4,-1.3225078E-2,1.3611369E-1,-1.417116E-1,-6.7631784E-3,1.953729E-2,8.1282414E-2,-1.8065992E-1,-1.7495391E-4,-5.01936E-2,7.976252E-3,7.00604E-4,1.2666328E-1,-1.9851862E-3,-1.1117795E-2,-3.9532438E-2,-9.23816E-3,1.1314319E-2,-6.9974E-3,7.236729E-3,1.2495602E-3,-2.5365911E-2,-6.0845776E-3,-5.2135065E-4,6.29387E-2,-5.9177596E-3,-7.433232E-2,-5.97098E-3,4.050646E-3,-1.3454848E-3,8.696749E-2,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,-1,23,-1,-1,-1,25,-1,27,29,31,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8503218E-1,1.5596202E-1,2.437979E-1,4.776524E-2,1.1626564E-1,0E0,4.97777E-2,3.7470296E-2,0E0,6.0973354E-2,7.052197E-2,0E0,1.8436834E-2,0E0,0E0,5.011916E-2,0E0,8.190803E-2,0E0,0E0,0E0,3.6700055E-2,0E0,6.1938964E-2,5.6821078E-2,3.1497225E-2,1.2199353E-2,0E0,4.7399078E-2,0E0,2.0637825E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,17,17,21,21,23,23,24,24,25,25,26,26,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,-1,24,-1,-1,-1,26,-1,28,30,32,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,5.314E3,1.4164306E-3,5.3E1,3.5523141E-3,1.953729E-2,4.1151314E0,9.34E2,-1.7495391E-4,4.0515396E-3,3.167407E7,7.00604E-4,1.7652562E4,-1.9851862E-3,-1.1117795E-2,2.47565E6,-9.23816E-3,9.855582E5,-6.9974E-3,7.236729E-3,1.2495602E-3,2.2345362E1,-6.0845776E-3,1E1,2.55E2,1.4740072E1,6.2174755E-1,-5.97098E-3,5.6E0,-1.3454848E-3,1.548321E-1,-1.6580995E-3,1.5258305E-3,-7.055104E-4,-4.6212934E-3,-3.1866936E-3,5.027597E-4,5.88934E-3,2.3413247E-3],"split_indices":[52,9,57,3,39,0,53,0,0,41,43,0,4,0,0,28,0,28,0,0,0,58,0,3,0,58,27,0,54,0,38,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.88E2,1.8E1,8E0,1.8E2,2E0,1.6E1,6E0,2E0,4.5E1,1.35E2,7E0,9E0,2E0,4E0,4.3E1,2E0,1.33E2,2E0,7E0,2E0,3.8E1,5E0,1.09E2,2.4E1,2.8E1,1E1,3E0,1.06E2,5E0,1.9E1,1.6E1,1.2E1,3E0,7E0,8E0,9.8E1,9E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.666469E-3,-1.7545465E-2,1.19480304E-1,-8.49554E-3,-8.6954616E-2,1.4809049E-2,7.858761E-2,-1.9953433E-2,4.7460496E-2,-1.262078E-2,-4.3817658E-2,1.2613867E-1,2.3892207E-2,-3.333415E-2,2.116914E-2,-2.6011164E-3,9.102063E-2,7.2902715E-4,-9.959214E-2,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,-2.6154688E-2,-6.2321755E-3,6.586735E-2,-6.205673E-3,-4.2967666E-3,1.9037895E-2,1.7304541E-1,1.7110182E-2,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,-4.9978763E-2,-6.258132E-3,5.289967E-3,1.691339E-3,8.535292E-3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,37,39,-1,41,43,45,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9837632E-1,1.14212275E-1,1.5746966E-1,1.0545254E-1,1.447895E-1,0E0,5.1603585E-2,7.553959E-2,6.1296083E-2,0E0,4.4794835E-2,2.756174E-2,1.8413085E-2,6.669822E-2,4.2011324E-2,2.7640013E-2,8.797648E-2,1.5424975E-2,1.9613385E-2,0E0,0E0,0E0,0E0,4.566688E-2,0E0,1.3546545E-2,2.2841673E-2,0E0,3.328212E-2,1.718907E-2,1.4277115E-2,0E0,0E0,0E0,0E0,2.4923556E-2,6.463696E-2,0E0,0E0,1.3547961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,25,25,26,26,28,28,29,29,30,30,35,35,36,36,39,39],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,38,40,-1,42,44,46,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,7.810581E11,1.4164306E-3,4.5E1,8.0522484E11,1.4809049E-2,1.0096877E8,1.5064244E0,6.787619E2,-1.262078E-2,7.294459E7,3.2635366E8,1.1038146E0,1.4114665E0,1.5489E4,8.27725E3,5.1270317E-2,6.324075E-5,7.5213313E-1,7.2923386E-3,6.316278E-4,-5.680382E-4,3.353301E-3,3.52777E5,-6.2321755E-3,3.0597075E8,4.187504E7,-4.2967666E-3,6.58542E5,8.63332E5,2.0323718E7,-2.435068E-3,1.2663178E-3,-1.2501667E-3,-6.7547397E-3,2.1034482E-1,4.95715E5,5.289967E-3,1.691339E-3,1.46E3,-3.63411E-3,3.1464316E-3,-1.8178234E-3,2.7417336E-3,1.0145066E-2,2.605178E-3,-1.106333E-3,-3.1847511E-3,-5.9238635E-4,3.0386809E-3,-1.1334497E-3,-1.8048598E-3,1.1277709E-3],"split_indices":[52,31,57,3,31,0,51,42,52,0,45,32,34,41,9,48,38,38,27,0,0,0,0,1,0,7,32,0,1,1,45,0,0,0,0,57,1,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.82E2,2.3E1,1.62E2,2E1,3E0,2E1,1.35E2,2.7E1,3E0,1.7E1,1E1,1E1,1.02E2,3.3E1,1.3E1,1.4E1,1E1,7E0,8E0,2E0,6E0,4E0,9.6E1,6E0,1.2E1,2.1E1,2E0,1.1E1,6E0,8E0,3E0,7E0,3E0,4E0,4.3E1,5.3E1,4E0,8E0,1.8E1,3E0,6E0,5E0,2E0,4E0,4E0,4E0,3E1,1.3E1,1E1,4.3E1,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-2.7402204E-3,-1.4488179E-2,8.47742E-2,-1.21823445E-1,-8.342185E-3,1.8740107E-1,-9.2469845E-3,-1.6784839E-3,-1.3934572E-1,1.2403458E-2,-4.2587325E-2,2.9215527E-1,4.7938665E-3,2.4776671E-2,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,4.4642862E-2,-1.8922342E-2,-3.3003904E-2,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,-3.79182E-2,5.552622E-2,-4.235295E-2,3.8699843E-2,1.6639564E-2,-5.707555E-2,-3.353905E-3,1.2455323E-4,7.502207E-2,-1.0837827E-2,-2.996247E-2,-1.0963698E-1,6.998143E-2,-2.54994E-4,7.42517E-2,-9.605715E-3,-8.033993E-2,-1.4847374E-3,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,-1,25,27,29,-1,-1,-1,-1,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1713255E-1,1.21684544E-1,2.4296036E-1,1.0496929E-2,1.2627566E-1,8.29688E-2,6.688262E-2,0E0,1.0810778E-2,1.1294642E-1,1.1641891E-1,3.8371354E-2,0E0,1.0111526E-2,0E0,0E0,0E0,5.1525243E-2,7.859604E-2,7.862471E-2,0E0,0E0,0E0,0E0,0E0,8.959427E-3,6.421043E-2,3.0895934E-2,2.3733113E-2,3.3978265E-2,5.6785285E-2,0E0,0E0,2.0940512E-2,1.2563176E-2,2.8267324E-2,2.118297E-2,8.720197E-3,0E0,1.4422674E-2,9.381125E-3,2.5498733E-2,8.510427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,-1,26,28,30,-1,-1,-1,-1,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,8.35E2,5.9337012E7,9.279E3,2.4934822E7,1.04E3,1.4233672E8,-1.6784839E-3,1.22E2,1.6672E4,6.3522E4,1.6232848E7,4.7938665E-3,9.5589536E-1,-6.9232536E-3,-7.949812E-3,-2.1842713E-3,5.277751E0,1.02942106E11,6.4348924E-1,-1.1486853E-2,1.8422948E-2,5.2739815E-3,1.8769933E-3,-1.4334748E-3,3.4E1,8.019512E0,4.305809E7,3.2395E4,2.7778377E11,3.9880952E-1,-3.353905E-3,1.2455323E-4,4.091697E7,6.2958473E4,2.237806E8,1.0519024E8,7.2023706E0,-2.54994E-4,7.951E3,4.83E9,4.944487E7,1.5107028E-1,4.351436E-3,1.5363606E-3,-2.408779E-3,7.1432913E-4,-2.910043E-4,-3.072794E-3,-7.4000456E-3,-8.045557E-4,1.8230815E-3,5.323976E-3,8.0183396E-5,5.0108824E-3,1.3392813E-3,-1.1813353E-3,-1.9343917E-3,-5.055712E-3,2.524649E-3,-5.2516176E-3],"split_indices":[52,2,45,9,45,0,45,0,10,9,10,1,0,27,0,0,0,54,31,27,0,0,0,0,0,3,53,32,9,31,57,0,0,32,33,7,7,54,0,9,5,45,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.85E2,2.4E1,9E0,1.76E2,1.1E1,1.3E1,2E0,7E0,1.1E2,6.6E1,4E0,7E0,1.1E1,2E0,5E0,2E0,5.4E1,5.6E1,6.4E1,2E0,2E0,2E0,9E0,2E0,6E0,4.8E1,4E1,1.6E1,2.1E1,4.3E1,3E0,3E0,3.7E1,1.1E1,3.5E1,5E0,9E0,7E0,6E0,1.5E1,3E1,1.3E1,2.7E1,1E1,4E0,7E0,2.1E1,1.4E1,3E0,2E0,6E0,3E0,2E0,4E0,4E0,1.1E1,1.2E1,1.8E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.9001366E-3,-1.343855E-2,1.1601315E-1,-1.0010185E-2,-1.111417E-2,2.2562835E-1,3.3268586E-2,-4.3225788E-2,2.3105354E-3,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,-5.3559467E-2,4.5828316E-3,4.5007914E-2,-1.1152626E-2,-4.4588957E-2,-8.577182E-3,-4.0048826E-3,5.9981905E-2,-3.185049E-2,2.3447553E-2,-2.0497924E-2,-8.2263514E-2,8.222998E-3,3.7578024E-2,-1.694235E-2,-1.1662346E-1,3.2336626E-2,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,23,-1,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.899919E-1,1.4066838E-1,1.6482484E-1,7.867399E-2,0E0,1.0884556E-1,1.5947495E-2,7.8872435E-2,8.1554644E-2,0E0,0E0,0E0,0E0,4.606816E-2,0E0,6.974669E-2,7.8090996E-2,4.0971868E-2,0E0,0E0,6.934259E-2,8.396302E-2,4.81951E-2,3.422363E-2,2.2953227E-2,0E0,6.309295E-2,6.78614E-2,3.393674E-2,2.4240296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,24,-1,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.5303E4,1.897887E6,3.5523141E-3,-1.111417E-2,3.0291426E-1,3.9637537E0,1E0,1.19855255E-1,1.5456219E-2,2.640784E-3,5.314707E-4,4.7046854E-3,4.0515396E-3,4.5828316E-3,5.178571E0,1.5064244E0,1.3431159E6,-8.577182E-3,-4.0048826E-3,3.849802E-1,1.4330416E0,5.46229E5,3.59E2,4.277001E3,8.222998E-3,6.448948E6,9.717001E5,4.3590187E1,1.3047E4,-4.863055E-3,7.1454706E-4,-2.540315E-3,-4.936126E-3,-5.9134926E-4,3.1115592E-3,-2.2807778E-3,-1.5878606E-3,2.6507506E-3,-1.7009196E-3,-8.041982E-3,3.2009885E-3,6.3695357E-4],"split_indices":[4,10,29,39,0,38,53,94,41,0,0,0,0,41,0,54,42,28,0,0,27,41,9,0,47,0,32,28,58,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.93E2,1.8E1,1.91E2,2E0,7E0,1.1E1,5.1E1,1.4E2,4E0,3E0,9E0,2E0,4.8E1,3E0,3.3E1,1.07E2,4.6E1,2E0,3E0,3E1,6.7E1,4E1,2.9E1,1.7E1,4E0,2.6E1,5.8E1,9E0,3.8E1,2E0,1.4E1,1.5E1,1.3E1,4E0,2E1,6E0,4.8E1,1E1,4E0,5E0,1.3E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.2056913E-3,-9.473315E-3,7.512071E-2,4.3374414E-4,-7.8392126E-2,1.5720385E-1,-1.8191284E-2,-2.5484065E-2,2.0686356E-2,-1.2111978E-2,-4.889973E-2,2.523382E-1,8.440286E-2,1.3624126E-2,-6.7835744E-3,1.453452E-2,-4.12981E-2,5.900995E-2,5.9917844E-3,-9.201298E-3,-1.0729721E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,2.635762E-2,-6.7390345E-2,-9.481888E-5,5.3193234E-3,3.617373E-2,-4.9073147E-3,6.533765E-2,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,1.4299699E-3,3.6974133E-3,-3.8031008E-2,-1.2230593E-1,7.3048964E-2,-2.9346012E-2,2.2003332E-3,-1.1724293E-3,-1.1925893E-2,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,-1,45,47,49,-1,-1,-1,-1,51,-1,53,55,57,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7272978E-1,1.1952097E-1,2.1732129E-1,8.136228E-2,9.84464E-2,8.259007E-2,5.5882495E-2,4.3634407E-2,4.864037E-2,0E0,4.537849E-2,8.354357E-2,1.8657587E-2,1.5502267E-2,0E0,1.9534279E-2,5.267284E-2,2.3609497E-2,4.1716885E-2,2.1403857E-2,2.0815924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2154378E-2,4.371713E-2,4.4934556E-2,0E0,1.089422E-2,2.8820923E-2,5.3309254E-2,0E0,0E0,0E0,0E0,1.3273055E-2,0E0,1.3264997E-2,2.0031199E-2,2.1552466E-2,2.3880532E-2,0E0,0E0,2.0580877E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,28,28,29,29,30,30,32,32,33,33,34,34,39,39,41,41,42,42,43,43,44,44,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,-1,46,48,50,-1,-1,-1,-1,52,-1,54,56,58,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.8236975E11,5.9337012E7,3.9360254E2,8.0522484E11,1.04E3,3.6030095E10,1.9689655E1,1.15062E5,-1.2111978E-2,7.294459E7,7.65E2,5.8199444E0,3.6377484E7,-6.7835744E-3,1.7081589E2,4.2060562E1,1E0,8.657441E7,1.6491606E3,5.534958E-1,2.6323516E-3,1.6446339E-2,1.1858551E-3,5.8577657E-3,-5.145168E-4,3.1467762E-3,-2.980767E-3,8.351808E-1,3.2022728E1,4.497684E1,5.3193234E-3,2.6921875E1,1E0,6.695E3,-2.3221816E-3,1.4905634E-3,-1.445699E-3,-7.1982783E-3,5.093772E-1,3.6974133E-3,5.1184835E0,4.8887E5,3.7E1,2.0133628E6,2.2003332E-3,-1.1724293E-3,5.0123274E-1,3.2929492E-3,-9.5332786E-4,6.094025E-3,1.3624355E-3,-1.6657992E-3,-4.082028E-3,-1.0857278E-3,-7.058917E-3,-1.01687E-3,5.6220116E-3,-3.8594857E-4,5.7275884E-4,-3.2505703E-3,4.4034544E-4,-1.504021E-3],"split_indices":[52,31,45,52,31,0,5,58,29,0,45,0,53,1,0,4,58,64,45,4,27,0,0,0,0,0,0,0,27,58,58,0,56,94,2,0,0,0,0,27,0,53,1,3,45,0,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.74E2,2.7E1,1.53E2,2.1E1,1.4E1,1.3E1,6.7E1,8.6E1,2E0,1.9E1,5E0,9E0,1.1E1,2E0,1.9E1,4.8E1,2.3E1,6.3E1,1.2E1,7E0,2E0,3E0,4E0,5E0,8E0,3E0,2E0,1.7E1,2.9E1,1.9E1,6E0,1.7E1,5.4E1,9E0,6E0,6E0,3E0,4E0,1.2E1,5E0,2E1,9E0,5E0,1.4E1,1.5E1,2E0,5E1,4E0,4E0,5E0,7E0,5E0,4E0,1.6E1,7E0,2E0,3E0,2E0,7E0,7E0,2.4E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-3.738608E-4,-7.6801833E-3,1.01941355E-1,7.692978E-3,-4.625046E-2,1.483836E-1,7.440007E-4,-9.925026E-3,6.1333936E-2,-8.378208E-2,-1.7553208E-2,1.8119766E-1,1.0880003E-3,-9.9033825E-2,-3.6509018E-3,1.2566844E-1,3.761748E-2,-1.050452E-1,-2.485187E-3,-5.857001E-3,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,1.0020773E-4,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.9409139E-2,5.540788E-3,-1.2027661E-1,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,1.2911957E-2,-6.143175E-2,2.4736589E-2,-1.5001127E-2,2.155236E-3,-5.720792E-4,-9.470102E-4,-1.3429011E-1,3.1634532E-2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,39,-1,41,-1,-1,-1,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5702043E-1,1.1657343E-1,5.407299E-2,1.3386032E-1,5.8844402E-2,3.31683E-2,0E0,5.945997E-2,4.867518E-2,4.122594E-2,4.3614455E-2,1.10411495E-2,0E0,1.9426428E-2,4.644964E-2,1.4112622E-2,3.5269197E-2,1.4692083E-2,8.812408E-3,3.3128686E-2,0E0,0E0,0E0,0E0,0E0,3.7203256E-2,0E0,0E0,0E0,1.8351674E-2,0E0,1.8578902E-2,0E0,0E0,0E0,1.6290672E-2,2.7178083E-2,4.342707E-2,2.3945123E-2,0E0,0E0,0E0,1.3021171E-2,2.1517647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,29,29,31,31,35,35,36,36,37,37,38,38,42,42,43,43],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,40,-1,42,-1,-1,-1,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9192E4,3.0143394E7,7.7434E4,7.7567725E2,2.3443186E3,2.0867E4,7.440007E-4,1.0135136E0,5.63905E6,1.07431E1,1.3844339E1,1.04E3,1.0880003E-3,5.045749E-1,1.3237829E7,1E0,1.3526E4,4.5E1,1.476E3,1.09495016E8,-6.4513674E-3,1.1147279E-2,4.2846506E-3,-1.3376054E-3,-7.096736E-3,6.0130353E0,-6.1427057E-3,1.5477632E-3,7.1998904E-3,1.7115049E6,5.540788E-3,2.95E2,-1.8659647E-3,-1.5995203E-3,1.8908035E-3,2.249985E5,2.884462E0,2.7496E4,9.0636E0,2.155236E-3,-5.720792E-4,-9.470102E-4,2.1E1,5.57E2,-1.061283E-3,-5.1928153E-3,5.1168946E-4,3.9574113E-3,2.1282327E-4,-2.0196738E-3,-2.0252832E-5,-7.2405227E-3,-1.7715811E-3,-1.2914025E-3,2.5908048E-3],"split_indices":[2,45,2,52,4,9,0,56,50,54,57,0,0,27,50,109,2,3,0,45,0,0,0,0,0,53,0,0,0,47,0,0,0,0,0,33,53,29,54,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.95E2,1.3E1,1.4E2,5.5E1,8E0,5E0,1.06E2,3.4E1,2.3E1,3.2E1,6E0,2E0,6E0,1E2,8E0,2.6E1,1.8E1,5E0,3E1,2E0,3E0,3E0,3E0,3E0,9.8E1,2E0,2E0,6E0,2.2E1,4E0,1.4E1,4E0,3E0,2E0,2.3E1,7E0,3.7E1,6.1E1,1.2E1,1E1,2E0,1.2E1,1.5E1,8E0,4E0,3E0,9E0,2.8E1,2.1E1,4E1,1E1,2E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.0470916E-4,-1.1145463E-2,9.843046E-2,-6.1118633E-2,4.996951E-3,1.2766055E-2,5.4078806E-2,-2.5317255E-2,-1.1859635E-1,-1.923276E-2,6.010194E-2,-2.131724E-4,3.8350024E-3,-5.8678132E-3,-2.225848E-3,-1.1466437E-2,-6.446723E-2,-1.1111286E-2,-6.9062617E-3,1.0727382E-1,4.0534303E-2,2.531128E-3,-8.6874526E-4,7.2951976E-4,-8.027679E-2,-5.5201775E-3,-5.2759657E-3,3.1679564E-3,8.534004E-3,2.4647668E-2,1.0526845E-1,-5.7665943E-3,-2.0202387E-3,-5.2320316E-2,9.702222E-3,3.7629638E-2,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.165946E-1,1.4873593E-1,1.3118732E-1,8.959766E-2,1.8808083E-1,0E0,2.4914969E-2,1.1248944E-2,9.248865E-2,9.6062645E-2,3.534253E-2,0E0,0E0,1.2025353E-2,0E0,0E0,1.753379E-2,4.9994655E-2,0E0,2.3044884E-2,3.0314423E-2,0E0,0E0,0E0,1.1376724E-2,6.391215E-2,0E0,0E0,0E0,1.0773087E-2,2.1219812E-2,0E0,0E0,2.3858346E-2,4.077713E-2,1.2596151E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,16,16,17,17,19,19,20,20,24,24,25,25,29,29,30,30,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.317626E3,1E0,1.4164306E-3,6.107924E11,8.3764136E2,1.2766055E-2,6.804E3,4.327473E5,8.0522484E11,4.462437E7,6.518218E6,-2.131724E-4,3.8350024E-3,8.6E1,-2.225848E-3,-1.1466437E-2,1E0,6.0073395E8,-6.9062617E-3,4.6008915E6,1.56E4,2.531128E-3,-8.6874526E-4,7.2951976E-4,3.7614656E5,1.292E3,-5.2759657E-3,3.1679564E-3,8.534004E-3,9.087618E6,9.56E2,-5.7665943E-3,-2.0202387E-3,2.8434807E3,7.269755E0,8.461558E10,-2.6753606E-4,8.11379E-3,1.599219E-3,6.0832634E-5,-3.5276993E-3,1.3863288E-3,-1.0578615E-3,-2.2297056E-4,2.5808953E-3],"split_indices":[52,109,57,31,52,0,9,28,31,45,45,0,0,10,0,0,8,7,0,45,2,0,0,0,28,2,0,0,0,32,0,0,0,47,53,31,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.83E2,1.9E1,4.4E1,1.39E2,3E0,1.6E1,2.8E1,1.6E1,9.7E1,4.2E1,5E0,1.1E1,1.5E1,1.3E1,4E0,1.2E1,9.2E1,5E0,1.1E1,3.1E1,2E0,1.3E1,2E0,1E1,8.8E1,4E0,8E0,3E0,2.6E1,5E0,4E0,6E0,2.1E1,6.7E1,1.8E1,8E0,2E0,3E0,6E0,1.5E1,4.2E1,2.5E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.050566E-3,4.2812275E-3,-1.5856923E-1,-1.1301997E-2,5.6136116E-2,-2.1941631E-4,-1.0672151E-2,-7.0011653E-3,-1.2052315E-1,8.916989E-3,1.02983706E-1,-5.7596188E-2,1.8009902E-3,-2.245222E-3,-8.737236E-3,-5.627105E-3,5.7158438E-3,1.7363462E-1,6.978703E-2,-1.5238659E-2,-8.90067E-2,1.7688595E-2,-2.4926206E-2,-3.5939007E-3,4.3866397E-3,1.0011509E-2,3.031237E-3,8.032361E-2,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,3.1169657E-2,-2.7009284E-2,-6.0450863E-2,-1.2030017E-3,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7956841E-1,1.679042E-1,6.4495996E-2,7.4314885E-2,1.0526259E-1,0E0,0E0,6.9132246E-2,1.8055126E-2,4.0123533E-2,4.4171542E-2,2.9244505E-2,5.6916848E-2,0E0,0E0,1.6036227E-2,0E0,1.2192935E-2,1.5846536E-2,1.4111153E-2,3.3006206E-2,5.1458158E-2,4.2065214E-2,0E0,8.906513E-3,0E0,0E0,9.820163E-3,0E0,0E0,0E0,0E0,0E0,2.6585214E-2,3.6565024E-2,2.0105965E-2,2.8099384E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,3.0455479E3,8.667E3,1.8600049E10,1E0,-2.1941631E-4,-1.0672151E-2,1.292E3,3.240192E6,8.129243E5,1.2117E4,8.137731E7,2.73782E5,-2.245222E-3,-8.737236E-3,2.3125E0,5.7158438E-3,1.884E3,7.317201E0,1.1086122E3,4.1749332E4,1.06403265E1,2.2009405E1,-3.5939007E-3,1.3174513E4,1.0011509E-2,3.031237E-3,3.2707875E8,-4.706238E-4,1.6980503E-3,-1.8687408E-3,-1.4091445E-3,-6.635329E-3,2.0589474E1,5.3153326E2,6.704386E6,4.462437E7,-3.9647566E-4,1.8044561E-3,4.504511E-3,9.965977E-4,3.995844E-3,1.0988942E-3,-3.2063897E-3,8.9813716E-4,5.605215E-4,-3.5929799E-3,9.870435E-4,-2.0493485E-3],"split_indices":[45,4,2,5,109,0,0,2,7,28,9,5,29,0,0,53,0,0,53,33,32,54,56,0,4,0,0,32,0,0,0,0,0,58,4,45,45,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.06E2,6E0,1.59E2,4.7E1,2E0,4E0,1.54E2,5E0,2.4E1,2.3E1,2.2E1,1.32E2,3E0,2E0,2.2E1,2E0,6E0,1.7E1,1E1,1.2E1,8.3E1,4.9E1,2E0,2E1,4E0,2E0,1.5E1,2E0,3E0,7E0,6E0,6E0,6.4E1,1.9E1,1.9E1,3E1,1.5E1,5E0,1.2E1,3E0,8E0,5.6E1,1E1,9E0,3E0,1.6E1,2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.4155842E-3,-1.3810372E-3,1.2642601E-2,-1.6525343E-2,4.4375908E-2,-1.2493689E-2,-8.009961E-3,1.661861E-2,9.891294E-2,-8.0245316E-2,-4.4310438E-3,4.9835492E-2,-6.8227405E-4,3.6446485E-3,9.054623E-3,-2.6625823E-2,-1.3053659E-1,-7.4558124E-2,1.1256639E-3,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,5.806915E-3,-7.995392E-2,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,9,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[2.0164502E-1,1.420966E-1,0E0,8.987266E-2,7.556003E-2,8.2129285E-2,0E0,2.0147853E-2,2.1653563E-2,3.9669834E-2,5.307982E-2,1.9236067E-2,1.3649627E-2,0E0,0E0,9.523076E-3,1.3058096E-2,3.7478834E-2,4.8667632E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2858007E-2,2.4820045E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,29,29,30,30],"right_children":[2,4,-1,6,8,10,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.8375797E2,1.6572577E0,1.2642601E-2,7.5656E4,1.9846272E9,4.994744E-5,-8.009961E-3,1.5489E4,9.246969E-1,8.7793425E-2,1.9E1,7.614E3,5E0,3.6446485E-3,9.054623E-3,2.8800612E9,3.2445145E2,2.75108E5,1.4655151E0,4.072302E-3,9.961356E-5,-8.1197097E-4,1.6258268E-3,1.2034691E-3,-2.1817805E-3,-7.57332E-3,-1.7940302E-3,-5.586186E-3,7.046286E-4,1E0,3.14608E6,-7.416806E-6,2.4997033E-3,-5.7289987E-3,4.691618E-4],"split_indices":[56,41,0,10,5,42,0,9,27,58,3,10,8,0,0,5,33,9,41,0,0,0,0,0,0,0,0,0,0,94,48,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,2.03E2,2E0,1.53E2,5E1,1.5E2,3E0,3.4E1,1.6E1,1.5E1,1.35E2,1.1E1,2.3E1,1.4E1,2E0,8E0,7E0,9E0,1.26E2,6E0,5E0,1.6E1,7E0,2E0,6E0,5E0,2E0,6E0,3E0,1.2E2,6E0,1.07E2,1.3E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.000657E-3,-1.0153504E-2,9.496438E-2,-3.1673333E-3,-7.777414E-2,1.5645174E-2,6.2693365E-2,5.9380285E-3,-5.026301E-2,-9.674218E-3,-2.5248412E-2,9.625793E-2,2.6888123E-2,-6.8429126E-3,4.4476364E-2,-2.1850644E-2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,8.983803E-3,-4.372294E-2,6.858251E-3,1.07610226E-1,-6.777975E-2,-4.1921283E-4,-1.2492558E-2,3.5249196E-2,-9.645313E-3,-8.684807E-2,-1.2344087E-3,1.4300514E-3,1.2848052E-1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,29,31,33,35,37,-1,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5137326E-1,8.573215E-2,1.699006E-1,7.148311E-2,1.03037156E-1,0E0,2.6254699E-2,6.926221E-2,8.881589E-2,0E0,1.071448E-2,2.5661737E-2,2.7722035E-2,6.226391E-2,8.238617E-2,1.3398929E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2746734E-2,4.618959E-2,1.7440343E-2,9.797484E-3,1.0284005E-2,0E0,3.8003672E-2,4.3830436E-2,3.493194E-2,1.7143585E-2,0E0,0E0,1.4258161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,30,32,34,36,38,-1,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.3983983E10,1.4164306E-3,1.2670352E1,5.535E3,1.5645174E-2,1.095857E10,6.695E3,2.0662694E-1,-9.674218E-3,2.1000721E0,1.7173885E8,3.6377484E7,1.6278354E7,4.485647E5,2.650027E2,-8.317451E-3,-2.1718242E-3,7.145327E-4,5.988639E-3,5.791561E-4,-6.829555E-4,3.7457973E-3,5.097052E8,7.149117E-1,4.602015E0,8.564948E6,2.3968115E2,-4.1921283E-4,1.5498047E1,6.034607E6,9.737113E0,5.8426323E0,-1.2344087E-3,1.4300514E-3,6.879445E1,2.3613751E-3,-3.8860086E-4,-5.1090647E-3,4.542698E-5,-3.8814347E-3,2.6935502E-3,-1.3007667E-3,-1.9402818E-3,2.3292864E-3,-5.540313E-3,-1.5515669E-3,7.346506E-3,1.6162232E-3],"split_indices":[52,5,57,54,9,0,12,2,38,0,57,32,1,45,28,4,0,0,0,0,0,0,0,5,27,53,32,4,0,56,48,54,53,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.81E2,2.5E1,1.65E2,1.6E1,2E0,2.3E1,1.39E2,2.6E1,4E0,1.2E1,1.1E1,1.2E1,1.05E2,3.4E1,2.2E1,4E0,8E0,4E0,8E0,3E0,7E0,5E0,7.4E1,3.1E1,2.2E1,1.2E1,4E0,1.8E1,4.1E1,3.3E1,1.8E1,1.3E1,9E0,1.3E1,8E0,4E0,2E0,2E0,3.5E1,6E0,2.5E1,8E0,1.2E1,6E0,8E0,5E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.156695E-3,-7.7511407E-3,6.4903505E-2,1.10561E-3,-6.161546E-2,8.889031E-2,9.5223775E-3,-1.1973289E-2,2.9857075E-2,-4.3537878E-2,-9.037242E-3,8.965046E-4,1.1017409E-1,-6.7902607E-4,2.1986503E-3,7.537986E-3,-3.928328E-2,6.545862E-2,1.1178152E-2,5.3028204E-2,-6.55025E-2,1.2060009E-3,6.3895895E-3,3.2280452E-4,6.727481E-2,-7.918549E-2,-8.271147E-3,3.5601782E-4,7.7395335E-2,2.8093105E-2,-2.6743395E-2,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,6.4077914E-2,-8.103772E-3,-7.31776E-5,4.6275426E-3,2.5301395E-5,-9.328173E-2,-5.3631708E-2,1.3650758E-2,2.564571E-3,6.1303745E-3,2.7586694E-3,-5.2228123E-3,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,-1,-1,-1,-1,49,51,-1,-1,-1,53,55,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5891957E-1,8.784768E-2,4.6976537E-2,6.014745E-2,5.0235853E-2,3.546658E-2,1.0674542E-2,5.890991E-2,3.251822E-2,5.437839E-2,0E0,0E0,3.1012043E-2,0E0,0E0,2.7994063E-2,5.6267872E-2,1.12324655E-2,2.2560306E-2,1.894341E-2,1.4103778E-2,0E0,0E0,3.2216277E-2,1.35331005E-2,2.251944E-2,2.7580641E-2,0E0,1.0156989E-2,2.3157822E-2,1.0649132E-2,0E0,0E0,0E0,0E0,1.371366E-2,2.1627188E-2,0E0,0E0,0E0,1.5584886E-2,1.2574337E-2,9.651603E-3,0E0,0E0,0E0,1.44525645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,35,35,36,36,40,40,41,41,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,-1,-1,-1,-1,50,52,-1,-1,-1,54,56,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,5.075104E7,5.1E1,6.787619E2,6.3538E4,6.19999E9,9.532E3,1.7086E4,1E0,5.577E3,-9.037242E-3,8.965046E-4,1.5214152E5,-6.7902607E-4,2.1986503E-3,1E0,2.5259212E5,2.695E3,7.6760286E-1,2.083122E6,5.534958E-1,1.2060009E-3,6.3895895E-3,4.570007E7,2.77E2,5.8910837E0,2.368447E6,3.5601782E-4,1.4854577E5,8.729897E0,2.3117E4,5.049313E-3,-7.4815657E-4,-7.712947E-4,-3.9418475E-3,1.297E3,2.0200117E-1,-7.31776E-5,4.6275426E-3,2.5301395E-5,1.2318777E3,2.187288E1,3.3E1,2.564571E-3,6.1303745E-3,2.7586694E-3,7.9206874E8,-2.9604407E-3,6.937693E-5,4.651002E-4,4.9921772E-3,2.1403502E-3,-7.6350395E-4,-5.0789355E-3,-3.3023278E-4,-2.809248E-5,-3.8963684E-3,1.5119945E-3,-6.901988E-4,-2.288945E-3,1.0493271E-3],"split_indices":[52,45,3,52,10,5,9,9,8,9,0,0,33,0,0,79,28,2,27,1,27,0,0,5,0,53,32,0,33,54,10,0,0,0,0,2,27,0,0,0,4,58,3,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.83E2,3.5E1,1.58E2,2.5E1,2.4E1,1.1E1,1.09E2,4.9E1,2.3E1,2E0,6E0,1.8E1,7E0,4E0,6.4E1,4.5E1,1.6E1,3.3E1,4E0,1.9E1,4E0,1.4E1,5.8E1,6E0,1.9E1,2.6E1,3E0,1.3E1,2.3E1,1E1,2E0,2E0,5E0,1.4E1,6E0,5.2E1,2E0,4E0,3E0,1.6E1,8E0,1.8E1,1E1,3E0,1.2E1,1.1E1,4E0,6E0,3E0,3E0,6E0,4.6E1,1.4E1,2E0,3E0,5E0,1.1E1,7E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-8.12835E-4,2.8342765E-3,-1.5601721E-1,-1.7404577E-2,2.4213504E-2,-2.3796102E-3,-1.0275062E-2,-1.0378984E-2,-1.0568353E-1,9.81486E-3,1.8756628E-2,-2.6070654E-2,1.303203E-2,-6.017882E-4,-7.747756E-3,5.4938868E-2,-4.513947E-3,-1.7527992E-2,-4.4134418E-3,-1.0665796E-2,3.9632928E-2,1.0806894E-1,3.8279194E-2,1.5463653E-3,-6.14344E-3,-6.364611E-2,-6.1187022E-3,2.6128873E-2,-4.657165E-2,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.8797043E-2,-8.169748E-4,-2.9431539E-2,2.5399424E-2,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,-1,-1,21,23,25,-1,27,29,31,33,35,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2327455E-1,9.2584625E-2,1.9196324E-2,6.712644E-2,9.692997E-2,0E0,0E0,3.8177617E-2,3.798466E-2,0E0,8.61301E-2,3.2634463E-2,2.6898822E-2,0E0,0E0,3.1268403E-2,4.6668507E-2,2.9076368E-2,0E0,3.1603176E-2,3.4378827E-2,1.0986567E-2,2.5717452E-2,4.5821525E-2,0E0,1.2384877E-2,2.5712674E-2,1.1580934E-2,1.3976639E-2,0E0,0E0,0E0,0E0,1.3670683E-2,1.1835324E-2,1.6557261E-2,5.3196885E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,-1,-1,22,24,26,-1,28,30,32,34,36,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,6.978001E2,2.9E1,4.462437E7,1.7727281E2,-2.3796102E-3,-1.0275062E-2,7.555544E2,9.214292E7,9.81486E-3,2.4611046E7,1.9284746E1,9.258696E0,-6.017882E-4,-7.747756E-3,7.573E3,1.3844339E1,2.547512E2,-4.4134418E-3,6.855402E6,1.2670352E1,1.884E3,6.5829754E-1,2.819228E-1,-6.14344E-3,3.7790768E2,1.3696512E7,6.735744E1,6.8539324E0,3.7548128E-3,-2.7270667E-4,6.575776E-3,1.9698313E-3,5.982359E-1,2.709E4,7.340202E1,1.3888E4,2.480503E-4,-3.868431E-3,2.629216E-4,-2.632077E-3,2.3529527E-3,-7.128859E-4,-2.4923E-4,-3.6717625E-3,2.0941987E-3,5.5986806E-3,9.4226614E-4,-2.2393765E-3,-2.4315931E-3,1.9385505E-5,5.031922E-3,2.7724257E-4],"split_indices":[45,52,3,45,33,0,0,4,7,0,45,56,54,0,0,9,57,55,0,45,54,0,27,57,0,33,45,58,54,0,0,0,0,27,9,56,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,2.12E2,4E0,1.09E2,1.03E2,2E0,2E0,1.02E2,7E0,2E0,1.01E2,6.1E1,4.1E1,3E0,4E0,3.9E1,6.2E1,5.5E1,6E0,2.2E1,1.9E1,8E0,3.1E1,6E1,2E0,1E1,4.5E1,1.1E1,1.1E1,1E1,9E0,5E0,3E0,2E1,1.1E1,2.6E1,3.4E1,2E0,8E0,3.7E1,8E0,7E0,4E0,5E0,6E0,1.7E1,3E0,8E0,3E0,1.5E1,1.1E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.6912411E-3,-1.9064195E-2,2.4100082E-2,-6.059589E-3,-7.711967E-2,4.095765E-2,-4.2764593E-2,-5.51559E-2,1.8715139E-3,-1.06207624E-1,-1.2241886E-3,1.0488696E-2,3.2134976E-2,-7.6291934E-3,-1.7113544E-2,5.660094E-4,-7.0297986E-2,5.156652E-3,-5.481648E-3,-6.6033505E-2,-8.217553E-3,7.82574E-3,2.3170885E-2,-4.49811E-2,1.0298288E-3,-4.236804E-3,9.818355E-4,2.7860066E-2,-4.866169E-3,-4.3798788E-3,-7.048719E-4,9.212592E-4,4.4673063E-2,-4.4644894E-3,-3.072789E-4,4.526607E-2,-1.4959747E-2,-3.7038807E-2,1.067402E-2,-6.5524145E-3,3.5458216E-3,2.2589561E-4,8.1912346E-2,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,25,27,-1,29,-1,-1,31,33,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,43,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.82962E-2,1.1127101E-1,9.417995E-2,4.7608547E-2,3.814037E-2,9.47225E-2,4.7029093E-2,1.8009417E-2,4.045555E-2,3.1635314E-2,0E0,0E0,7.009145E-2,0E0,1.6951097E-2,0E0,2.2984944E-2,2.3801006E-2,0E0,1.3520818E-2,0E0,0E0,2.9142186E-2,1.5691848E-2,0E0,0E0,0E0,2.4527468E-2,3.6893085E-2,0E0,0E0,1.7144382E-2,4.5832694E-2,0E0,0E0,1.8606756E-2,1.2692557E-2,1.1324577E-2,3.5423E-2,2.0130176E-2,0E0,0E0,1.3577841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,16,16,17,17,19,19,22,22,23,23,27,27,28,28,31,31,32,32,35,35,36,36,37,37,38,38,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,26,28,-1,30,-1,-1,32,34,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,44,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0523087E3,2.325945E9,1.09495016E8,1.142E3,7.888087E0,4.164E3,2.2970924E-2,4.569361E7,1.2224206E7,8.260109E9,-1.2241886E-3,1.0488696E-2,1.1307017E3,-7.6291934E-3,8.66076E1,5.660094E-4,5.9215684E0,2.6374866E1,-5.481648E-3,5.754E3,-8.217553E-3,7.82574E-3,7.6E0,2.884462E0,1.0298288E-3,-4.236804E-3,9.818355E-4,7.438972E0,4.2060562E1,-4.3798788E-3,-7.048719E-4,7.361366E7,2.6023E4,-4.4644894E-3,-3.072789E-4,6.8623977E0,7.444958E7,6.37E2,6.191606E0,8.062193E-1,3.5458216E-3,2.2589561E-4,7.3586698E0,1.1036837E-3,4.1260812E-3,1.16352E-3,-2.1829961E-3,-2.869491E-3,-6.7901827E-4,1.6735564E-3,-9.080979E-4,9.5369056E-4,-1.570197E-3,4.6753353E-3,7.5069134E-4],"split_indices":[52,5,45,2,53,12,27,5,50,5,0,0,52,0,56,0,56,58,0,2,0,0,54,53,0,0,0,53,58,0,0,45,10,0,0,53,7,0,53,27,0,0,53,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.47E2,8.1E1,1.21E2,2.6E1,6.5E1,1.6E1,1.6E1,1.05E2,1.6E1,1E1,2E0,6.3E1,2E0,1.4E1,3E0,1.3E1,1.03E2,2E0,1.1E1,5E0,3E0,6E1,8E0,6E0,1.1E1,2E0,3.1E1,7.2E1,7E0,4E0,3E1,3E1,3E0,5E0,2.2E1,9E0,2.3E1,4.9E1,2.8E1,2E0,1.5E1,1.5E1,1.5E1,7E0,4E0,5E0,1.1E1,1.2E1,2.7E1,2.2E1,1.4E1,1.4E1,1.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.5013376E-3,5.938352E-3,-1.24217264E-1,-2.7496833E-3,8.3399355E-2,-8.301269E-3,-2.700483E-4,-3.2061845E-2,9.8344935E-3,1.079024E-2,4.594788E-2,-1.9474141E-2,-7.3742555E-3,6.1866198E-2,-1.161049E-3,4.8218677E-3,7.569026E-4,-8.262385E-2,-3.5967962E-3,9.7572885E-2,-1.3912333E-3,3.422547E-2,-7.907227E-3,-1.2076516E-3,-6.5928353E-3,5.179689E-2,-2.5634378E-2,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,-5.0402116E-2,2.8263181E-3,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,29,31,-1,-1,33,35,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.14531964E-1,1.3348015E-1,3.7561677E-2,6.634294E-2,9.341748E-2,0E0,0E0,7.825357E-2,7.215855E-2,0E0,2.5913164E-2,4.983531E-2,0E0,7.3844664E-2,2.5336621E-2,0E0,0E0,2.7188063E-2,5.1378243E-2,3.849563E-2,0E0,1.8305652E-2,4.0737204E-2,0E0,0E0,2.3600813E-2,3.192762E-2,0E0,0E0,0E0,0E0,2.3702014E-2,4.1213304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,22,22,25,25,26,26,31,31,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,30,32,-1,-1,34,36,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,8.861832E3,6.45153E5,4.039435E-3,1.4164306E-3,-8.301269E-3,-2.700483E-4,2.0604518E8,7.573E3,1.079024E-2,3.0904205E2,7.5999E4,-7.3742555E-3,8.19162E5,1.2402962E-1,4.8218677E-3,7.569026E-4,4.1749332E4,5.4127317E10,1.946E3,-1.3912333E-3,7.317201E0,3.0738266E4,-1.2076516E-3,-6.5928353E-3,7.356E3,1.9990385E1,-1.662583E-4,5.8954954E-3,2.8480105E-3,-4.303386E-4,3.7E1,2.2759746E7,-1.65264E-3,3.5207167E-3,-1.9844588E-4,-3.673357E-3,-3.4019102E-3,6.949383E-4,8.460668E-4,-1.7469082E-3],"split_indices":[45,4,28,38,57,0,0,7,9,0,58,1,0,29,41,0,0,32,31,2,0,53,33,0,0,9,58,0,0,0,0,3,45,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.97E2,6E0,1.78E2,1.9E1,4E0,2E0,5.3E1,1.25E2,3E0,1.6E1,4.9E1,4E0,2.1E1,1.04E2,5E0,1.1E1,9E0,4E1,1.5E1,6E0,1.6E1,8.8E1,5E0,4E0,1.1E1,2.9E1,3E0,1.2E1,1E1,6E0,1.7E1,7.1E1,2E0,9E0,2.1E1,8E0,1.3E1,4E0,5.2E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.211318E-5,-7.6537346E-3,7.079598E-2,6.928152E-3,-3.6480606E-2,1.4365091E-2,3.16417E-2,-9.043619E-3,5.276066E-2,-6.891165E-2,-9.199198E-3,6.183768E-2,-4.7232076E-3,7.881159E-3,-3.276541E-2,7.939335E-3,8.772806E-2,-3.0282011E-2,-1.1089681E-1,-1.8124288E-2,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,1.8954586E-2,-4.7628164E-2,2.5371954E-2,2.6573949E-3,-1.2165993E-2,5.2440288E-3,4.916817E-4,-1.1942704E-2,-4.229069E-3,-3.970585E-5,-1.548476E-1,2.0617556E-3,-4.2224076E-2,6.205118E-2,4.711567E-4,-7.918416E-2,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,2.3531545E-2,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,-1,-1,45,-1,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1673426E-1,8.264138E-2,1.751531E-1,9.6372604E-2,5.776053E-2,0E0,7.98365E-2,3.9603867E-2,5.2591182E-2,4.529798E-2,3.7174046E-2,1.3060965E-2,0E0,4.1106842E-2,3.6635753E-2,1.5675733E-2,2.8447852E-2,1.6613591E-2,6.760767E-2,1.7113835E-2,0E0,0E0,0E0,0E0,3.9799407E-2,3.9343596E-2,1.4240889E-2,0E0,1.0493089E-2,0E0,0E0,1.1072869E-2,0E0,0E0,1.0195777E-2,1.9533398E-2,1.2358395E-2,2.6103884E-2,2.7533289E-2,3.5623252E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1920147E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,25,25,26,26,28,28,31,31,34,34,35,35,36,36,37,37,38,38,39,39,50,50],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,-1,-1,46,-1,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8038013E3,2.2971932E7,1.4164306E-3,7.7567725E2,9.875E3,1.4365091E-2,1.402511E8,1.7086E4,4.5470334E5,6.107924E11,3.5632E4,8.0426145E-1,-4.7232076E-3,5.421359E0,1.5071542E0,3.4706132E-3,4.493248E6,7.951E3,2.358119E-1,7.4500206E5,4.768983E-3,4.2687627E-3,1.2164821E-3,-2.6529813E-3,4.573327E5,4.4589956E5,1.69228E5,2.6573949E-3,1.8590495E9,5.2440288E-3,4.916817E-4,9.1487586E-1,-4.229069E-3,-3.970585E-5,1.96995E10,1.1036333E3,1.1192292E1,1.4214917E3,4.570007E7,5.6111975E0,-4.4534222E-4,-1.7505564E-3,2.3377908E-3,-2.049646E-3,7.4185274E-4,-1.4830381E-3,1.3386911E-3,-9.246738E-3,-3.9756782E-3,-2.0037123E-3,3.3478114E-1,-2.709715E-3,5.8699097E-4,-1.534671E-3,3.8385512E-3,2.7706537E-3,-6.189705E-4,7.5703E-4,-4.8262086E-3,-6.4742455E-4,2.2130334E-3],"split_indices":[52,45,57,52,9,0,45,9,28,31,2,27,0,54,57,27,1,9,27,48,0,0,0,0,48,28,29,0,5,0,0,27,0,0,5,33,54,48,5,53,0,0,0,0,0,0,0,0,0,0,27,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.95E2,2E1,1.3E2,6.5E1,2E0,1.8E1,9.7E1,3.3E1,2.9E1,3.6E1,1.5E1,3E0,5.7E1,4E1,1.5E1,1.8E1,1.6E1,1.3E1,3.4E1,2E0,8E0,7E0,8E0,4.9E1,3.2E1,8E0,4E0,1.1E1,1.4E1,4E0,1.3E1,3E0,4E0,9E0,1.9E1,1.5E1,1.4E1,3.5E1,1.7E1,1.5E1,2E0,6E0,5E0,6E0,9E0,4E0,5E0,4E0,6E0,1.3E1,1.2E1,3E0,2E0,1.2E1,6E0,2.9E1,3E0,1.4E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.1967006E-3,8.5786525E-3,-1.4402847E-1,8.315427E-4,8.315179E-2,-9.1837915E-3,-1.7736525E-3,-1.9747915E-2,1.3641278E-2,1.3899025E-2,4.2534754E-2,-1.29205575E-2,-7.1898274E-2,3.6052603E-2,-8.618837E-3,3.79517E-4,7.254509E-2,-1.8197084E-2,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,2.5952812E-2,-1.9911975E-2,1.2206658E-3,1.6528742E-4,4.4683916E-3,-2.9289007E-2,1.7770747E-2,5.043673E-2,-5.733453E-3,2.2968883E-3,-2.780274E-2,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.356853E-1,1.1663157E-1,2.196373E-2,4.877499E-2,1.4408445E-1,0E0,0E0,2.4350522E-2,5.7182897E-2,0E0,1.7155953E-2,2.955227E-2,1.8783536E-2,5.395387E-2,2.2573741E-2,0E0,1.1681624E-2,2.485658E-2,0E0,0E0,0E0,0E0,4.1406788E-2,2.4611853E-2,0E0,0E0,0E0,1.7023295E-2,2.698188E-2,2.68047E-2,2.1004716E-2,0E0,2.7273526E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,22,22,23,23,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,9.417025E3,1.6330187E8,8.1913336E2,1.4164306E-3,-9.1837915E-3,-1.7736525E-3,1.996866E1,4.91101E5,1.3899025E-2,8.476884E-1,1.6482397E1,2.25813E5,6.37E3,1.0807868E7,3.79517E-4,3.5766065E0,3.8E1,3.444446E-3,3.975245E-4,-4.86414E-3,6.613762E-3,6.995449E-1,2E1,1.2206658E-3,1.6528742E-4,4.4683916E-3,1.5662762E10,6.2734294E0,7.269755E0,1E0,2.2968883E-3,1.01618E5,-2.3744998E-3,-4.9519684E-4,-2.169697E-3,2.165552E-3,3.1829614E-3,-3.9339814E-4,-1.1482185E-3,1.9958988E-3,-1.3757133E-4,-2.6892552E-3],"split_indices":[45,4,45,55,57,0,0,56,29,0,34,56,1,9,1,0,35,3,0,0,0,0,27,3,0,0,0,31,54,53,108,0,11,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,2.01E2,5E0,1.83E2,1.8E1,3E0,2E0,7E1,1.13E2,2E0,1.6E1,6.3E1,7E0,5.6E1,5.7E1,8E0,8E0,6E1,3E0,2E0,5E0,4E0,5.2E1,4.3E1,1.4E1,2E0,6E0,4.6E1,1.4E1,2.9E1,2.3E1,4E0,3.9E1,2.2E1,2.4E1,4E0,1E1,2.3E1,6E0,1.7E1,6E0,2.1E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.9543333E-3,-2.559017E-2,1.9161552E-2,-2.0923022E-2,-5.902389E-3,2.6990097E-2,-7.9133324E-2,-8.834234E-3,-4.04624E-2,1.3379109E-2,7.079206E-2,3.3201702E-4,-8.117297E-3,8.160095E-3,-2.9381018E-2,-7.949769E-2,-1.22999055E-2,4.4339493E-2,-1.4131938E-2,1.0944012E-1,-7.471799E-4,7.50054E-2,-6.7173177E-3,-5.5795915E-2,-1.1298074E-3,-5.1226197E-3,1.4864717E-3,-1.065279E-3,-4.5908066E-3,-3.7182914E-3,6.852332E-2,-5.39476E-2,8.511279E-3,1.29912235E-2,5.2558865E-2,4.981872E-4,4.811432E-3,-3.245346E-3,1.7857874E-3,-2.4376418E-2,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,8.3817184E-2,-7.3740636E-3,-2.728049E-2,5.5922586E-2,-1.4394672E-2,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,-1,45,47,49,51,-1,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0851875E-1,4.7482766E-2,8.293395E-2,2.5152944E-2,0E0,5.747927E-2,6.141771E-2,2.3983913E-2,4.4094764E-2,6.53598E-2,7.825514E-2,0E0,0E0,3.8294274E-2,2.30708E-2,5.1408485E-2,2.2939531E-2,4.2094745E-2,3.7422903E-2,1.2931412E-1,0E0,9.805635E-3,1.6436392E-2,1.2211446E-2,2.4566848E-2,0E0,0E0,9.913108E-3,0E0,1.5312212E-2,2.8017767E-2,3.489702E-2,3.0127015E-2,0E0,2.5919788E-2,0E0,0E0,0E0,1.9405605E-2,2.0426273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.549913E-2,0E0,1.4559356E-2,1.8774323E-2,1.5052079E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,34,34,38,38,39,39,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,-1,46,48,50,52,-1,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,8.260109E9,1.4623123E8,1.7086E4,-5.902389E-3,9.276886E1,1.5214152E5,4.327473E5,2.5259212E5,2.4934822E7,5.2E1,3.3201702E-4,-8.117297E-3,4.570007E7,4.9054803E2,4.758E3,4.1969254E8,2.8626094E5,5.8816573E9,1.6064256E-2,-7.471799E-4,7.444958E7,8.8916E4,2.79E2,6.027375E0,-5.1226197E-3,1.4864717E-3,1.194E3,-4.5908066E-3,1E0,1E0,2.9E1,3.800374E6,1.29912235E-2,3.5935396E-1,4.981872E-4,4.811432E-3,-3.245346E-3,2.0558643E8,3.6E1,-4.0961746E-3,1.9105568E-3,-1.7960214E-3,-9.5954916E-4,1.0256129E-3,1.6912112E-3,-1.5624621E-3,-3.0936327E-4,1.5262272E3,-7.3740636E-3,4E0,5.57E2,2.5848E4,-5.7001284E-4,4.01135E-3,1.166477E-3,-1.3315215E-3,-3.2957904E-3,1.1604258E-3,1.3835307E-3,4.8853667E-3,-2.623845E-3,6.2189676E-4,-1.0046908E-3,3.929722E-3,-1.3374101E-3,2.0220869E-3],"split_indices":[52,5,45,9,0,56,33,28,28,45,3,0,0,5,55,2,7,28,5,57,0,7,1,0,53,0,0,10,0,8,109,3,1,0,57,0,0,0,7,10,0,0,0,0,0,0,0,0,4,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.11E2,1.04E2,1.07E2,4E0,9.7E1,7E0,6.7E1,4E1,7.5E1,2.2E1,4E0,3E0,3.7E1,3E1,1.6E1,2.4E1,3.5E1,4E1,1.5E1,7E0,6E0,3.1E1,1.5E1,1.5E1,1.3E1,3E0,2.2E1,2E0,1.2E1,2.3E1,1.4E1,2.6E1,3E0,1.2E1,2E0,4E0,3E0,2.8E1,8E0,7E0,7E0,8E0,1.2E1,1E1,5E0,7E0,4E0,1.9E1,2E0,1.2E1,8E0,1.8E1,4E0,8E0,1.6E1,1.2E1,4E0,4E0,5E0,1.4E1,7E0,5E0,2E0,6E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.1127363E-3,-5.290152E-3,6.0248755E-2,-5.542424E-2,-1.2838513E-3,6.840342E-3,8.6930476E-4,1.3661059E-4,-7.317434E-2,-5.0637187E-3,6.27141E-4,-3.2042078E-4,-4.331261E-3,-7.5224843E-3,3.989758E-2,6.3668867E-3,-2.6071595E-2,-3.1950418E-2,6.536836E-2,-2.7029186E-2,1.4747367E-2,-7.087237E-2,-1.15542095E-2,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,7.527232E-2,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,-1,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8252895E-2,3.5278834E-2,3.8336206E-2,1.3642367E-2,3.2272287E-2,0E0,0E0,0E0,9.951666E-3,0E0,5.21465E-2,0E0,0E0,3.4946784E-2,5.3329237E-2,2.2228872E-2,3.690375E-2,1.3936162E-2,1.901877E-2,1.1226752E-2,2.018778E-2,3.677965E-2,1.6500767E-2,0E0,0E0,0E0,1.2472153E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,-1,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8008E4,1.9388489E3,7.9684106E9,6.051637E-1,1E1,6.840342E-3,8.6930476E-4,1.3661059E-4,7.891632E1,-5.0637187E-3,9.198947E-1,-3.2042078E-4,-4.331261E-3,8.729897E0,7.5773336E7,4.817292E0,2.9E1,6.7995167E0,1.4740072E1,5.9145586E3,7.137324E0,1.6753285E-1,1.6144748E5,-3.1499017E-3,8.2755066E-4,-9.3177555E-4,1.2633E4,-2.3159643E-3,3.0185888E-4,1.0349049E-3,-1.5924309E-3,-8.527036E-5,-5.3084292E-3,7.1703445E-4,-1.2258718E-3,2.2928133E-3,5.260966E-3],"split_indices":[2,48,12,27,3,0,0,0,51,0,27,0,0,54,7,53,3,54,58,4,53,39,28,0,0,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.75E2,1.1E1,1.2E1,1.63E2,3E0,8E0,3E0,9E0,2E0,1.61E2,2E0,7E0,1.34E2,2.7E1,7.7E1,5.7E1,7E0,2E1,1.5E1,6.2E1,1.3E1,4.4E1,4E0,3E0,2E0,1.8E1,9E0,6E0,5.5E1,7E0,5E0,8E0,1.5E1,2.9E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-9.433804E-4,-1.2022093E-2,5.917883E-2,-6.3716695E-2,-6.556876E-3,1.185642E-2,3.261322E-2,-7.952249E-2,-4.8074138E-4,-1.3831026E-2,3.4149677E-2,4.693409E-2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,-4.5358995E-4,-3.4879703E-2,-6.250601E-4,2.4170529E-3,7.417909E-2,8.9431816E-4,1.8055465E-2,-2.7939798E-2,-7.63545E-2,-1.986685E-2,4.411847E-3,-8.005692E-4,-7.7093607E-3,6.321066E-2,1.0480897E-2,-5.4052684E-2,-4.8673116E-3,-6.69903E-4,-7.4792397E-3,-7.8966774E-2,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,27,29,31,33,-1,-1,35,37,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4125362E-1,5.015369E-2,1.5375312E-1,1.3680756E-2,4.8737537E-2,0E0,6.7347795E-2,8.856602E-3,0E0,3.9123423E-2,1.886667E-2,2.0422306E-2,0E0,0E0,0E0,4.4256426E-2,3.210645E-2,0E0,0E0,2.239582E-2,0E0,6.0984924E-2,3.5681155E-2,1.9354679E-2,2.9425774E-2,0E0,0E0,1.626718E-2,2.6215322E-2,1.7650409E-2,2.1931183E-2,0E0,0E0,1.2872141E-2,1.4290784E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,15,15,16,16,19,19,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,28,30,32,34,-1,-1,36,38,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5073215E3,4.060294E0,1.4164306E-3,4.668851E6,2.6493E4,1.185642E-2,1.18E2,7.7573473E9,-4.8074138E-4,1.7086E4,7.4353185E0,3.0904205E2,-5.4467157E-3,-2.4170883E-3,-5.7114405E-3,1.5989196E7,2.2741E4,-6.250601E-4,2.4170529E-3,1.2003011E8,8.9431816E-4,4.984018E8,7.149117E-1,4.4589956E5,1.3000701E7,4.411847E-3,-8.005692E-4,2.80373E5,7.11E2,2.2809816E5,5.36E3,-4.8673116E-3,-6.69903E-4,1.2318777E3,1.1075563E3,-1.2746986E-3,8.4917404E-4,4.1565746E-3,4.274588E-5,2.882671E-3,-6.147579E-4,-3.509966E-3,1.5297749E-4,-8.069514E-4,1.5231442E-3,-5.78054E-3,-9.4476645E-4],"split_indices":[52,53,57,29,10,0,3,5,0,9,54,58,0,0,0,45,9,0,0,32,0,5,27,28,50,0,0,1,0,28,2,0,0,4,52,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.78E2,3.2E1,1.6E1,1.62E2,3E0,2.9E1,1.2E1,4E0,1.38E2,2.4E1,2.7E1,2E0,8E0,4E0,8.5E1,5.3E1,6E0,1.8E1,1.3E1,1.4E1,5.1E1,3.4E1,1.3E1,4E1,1.1E1,2E0,3.3E1,1.8E1,1.4E1,2E1,9E0,4E0,3.4E1,6E0,1.9E1,1.4E1,1.3E1,5E0,4E0,1E1,1.5E1,5E0,2.8E1,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.7782936E-4,-1.382795E-2,2.3196248E-2,-7.2672334E-3,-1.0122567E-1,-7.0918067E-3,4.573724E-2,-4.658554E-3,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,2.595186E-2,-2.5460321E-2,5.915973E-3,3.5621542E-2,-5.366179E-2,-6.7920005E-4,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,1.8883888E-2,8.107459E-2,-4.6013836E-3,-9.936781E-5,2.8179128E-2,-8.45359E-3,7.830855E-4,5.3368773E-2,7.481408E-3,2.3286117E-3,-1.5134966E-3,4.5415547E-2,-1.37019865E-2,1.673101E-3,3.2098924E-3,-1.2978952E-2,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,-1,-1,31,33,35,37,-1,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.788673E-2,7.380529E-2,5.4942183E-2,3.1929776E-2,2.121576E-2,2.1904452E-2,3.0952565E-2,2.3363702E-2,0E0,0E0,0E0,9.656656E-3,1.7785855E-2,0E0,3.0421652E-2,1.8802509E-2,2.5366073E-2,0E0,0E0,0E0,0E0,1.993274E-2,1.9198395E-2,0E0,0E0,2.5934385E-2,2.0478453E-2,2.058842E-2,1.0298643E-2,0E0,0E0,0E0,1.0949407E-2,2.1243747E-2,0E0,0E0,1.2440793E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,15,15,16,16,21,21,22,22,25,25,26,26,27,27,28,28,32,32,33,33,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,-1,-1,32,34,36,38,-1,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0648859E3,1.4446067E10,1E0,1.2224206E7,3E0,1.256E3,5.0655737E0,1.1475E2,-5.3139855E-3,-7.1186107E-3,-1.7487173E-3,1.7837838E0,2.3894766E7,5.915973E-3,4.4E1,7.8206635E0,3.06123E5,-1.4610871E-4,2.4916518E-3,-3.0934727E-3,-2.3441718E-4,3.4170952E11,1.2117E4,-4.6013836E-3,-9.936781E-5,5.1184835E0,2.6493E4,2.95E2,3.5935396E-1,7.481408E-3,2.3286117E-3,-1.5134966E-3,7.856E3,2.325945E9,1.673101E-3,3.2098924E-3,3.5120192E-1,-3.839128E-4,3.3024102E-3,-3.730943E-4,2.7179017E-3,-3.615233E-4,-2.7678004E-3,-1.496436E-3,1.0646455E-3],"split_indices":[52,5,109,50,8,0,54,52,0,0,0,57,48,0,3,53,12,0,0,0,0,31,9,0,0,53,10,0,57,0,0,0,9,5,0,0,38,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.29E2,7.9E1,1.21E2,8E0,3.4E1,4.5E1,1.19E2,2E0,4E0,4E0,1.2E1,2.2E1,4E0,4.1E1,8E0,1.11E2,6E0,6E0,7E0,1.5E1,3.1E1,1E1,4E0,4E0,2.3E1,8.8E1,2.1E1,1E1,2E0,8E0,5E0,1.8E1,7.9E1,9E0,3E0,1.8E1,2E0,8E0,3E0,1.5E1,7E1,9E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.1277195E-4,4.217555E-3,-1.0916315E-1,-2.9917555E-3,5.3992495E-2,5.0713454E-4,-7.7418964E-3,3.2400577E-3,-3.849527E-2,1.5364976E-1,2.9980375E-2,-2.0002978E-2,1.4616362E-2,-1.2040921E-2,-9.3005456E-2,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,-2.5916894E-3,-7.337681E-2,2.9357852E-2,-1.9776661E-2,-3.767159E-2,6.012406E-4,-6.2919264E-3,-7.461606E-4,4.0725708E-2,-2.1920465E-2,-5.1689288E-3,-1.9261596E-3,3.854466E-2,-1.4079762E-2,-5.6224447E-3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7122716E-2,7.65951E-2,4.344309E-2,4.149871E-2,6.000404E-2,0E0,0E0,4.2599544E-2,3.9257012E-2,6.30681E-2,1.8668272E-2,4.906311E-2,5.533707E-2,1.2970728E-2,2.3380555E-2,0E0,0E0,0E0,0E0,3.5220888E-2,1.0222927E-2,3.087072E-2,1.9976046E-2,2.0381412E-2,0E0,0E0,0E0,1.7245183E-2,1.5136076E-2,0E0,0E0,2.189073E-2,2.3911959E-2,1.0170521E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,22,22,23,23,27,27,28,28,31,31,32,32,33,33],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,4.7162656E3,8.667E3,1E0,1.3277E4,5.0713454E-4,-7.7418964E-3,2.427E3,2.7577372E11,4.2E1,4.970186E7,1.3017557E7,8.40547E5,1.6391889E5,1.485E3,1.19463205E-2,5.1632506E-4,3.5616935E-3,5.074132E-4,6.522946E5,6.5657372E0,2.7883E4,1.7657008E7,2.1692245E1,6.012406E-4,-6.2919264E-3,-7.461606E-4,1E0,2.85898E10,-5.1689288E-3,-1.9261596E-3,1.2670352E1,1.5324128E6,2.016E3,-3.0808672E-3,-3.6385872E-3,6.9962005E-4,2.7713825E-3,-1.5731759E-3,-1.5861242E-3,1.1883401E-3,2.1978908E-3,-6.1306317E-4,2.5951513E-3,-1.8157793E-3,-1.1095738E-3,7.941144E-4],"split_indices":[45,52,2,89,10,0,0,2,31,3,51,45,29,28,0,0,0,0,0,48,54,9,32,56,0,0,0,68,31,0,0,54,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,2.12E2,6E0,1.86E2,2.6E1,2E0,4E0,1.59E2,2.7E1,4E0,2.2E1,5.2E1,1.07E2,1.9E1,8E0,2E0,2E0,6E0,1.6E1,4E1,1.2E1,7.5E1,3.2E1,9E0,1E1,5E0,3E0,1.2E1,2.8E1,5E0,7E0,6.2E1,1.3E1,2.5E1,7E0,5E0,4E0,1E1,2E0,2.3E1,5E0,5.5E1,7E0,3E0,1E1,1.4E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.3611307E-3,-9.2408955E-3,4.410103E-2,-6.9312286E-3,-7.04401E-3,1.3789503E-1,4.378376E-3,-2.0679065E-3,-5.6651037E-2,1.4739492E-3,1.1579285E-2,3.072128E-2,-8.0794595E-2,-4.1738063E-2,2.7708148E-3,-8.478865E-2,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,8.560338E-3,-6.760063E-2,-1.666495E-3,6.8934974E-3,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,-8.954448E-2,3.4993854E-3,4.3400084E-3,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,33,-1,35,-1,-1,-1,-1],"loss_changes":[6.6894464E-2,5.595534E-2,9.999141E-2,4.340747E-2,0E0,7.891193E-2,4.7356185E-2,3.178787E-2,3.738624E-2,0E0,0E0,2.251287E-2,2.4343904E-2,2.3993906E-2,2.2820415E-2,1.3125837E-2,0E0,0E0,0E0,0E0,0E0,1.6774707E-2,1.7332826E-2,0E0,2.1932926E-2,0E0,0E0,0E0,0E0,0E0,9.104654E-3,0E0,2.9089697E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,21,21,22,22,24,24,30,30,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,34,-1,36,-1,-1,-1,-1],"split_conditions":[5.2062725E3,7.4311E4,1.04E3,2.3361332E6,-7.04401E-3,7.65E2,3.093317E5,1.278E3,6.8125E0,1.4739492E-3,1.1579285E-2,5.8199444E0,9.532E3,2.8434807E3,1.0458333E1,1E0,1.0808542E-3,7.3618015E-5,3.8519548E-3,1.5215501E-4,-6.705464E-3,1.7081589E2,8.137731E7,-1.666495E-3,2.2413793E0,-1.4088308E-3,-5.309024E-3,-2.5179447E-3,2.0940083E-3,-6.457079E-5,2.1367E4,3.4993854E-3,5.3473053E0,-5.209139E-3,-9.1877626E-4,-2.63493E-3,3.9461057E-4],"split_indices":[52,10,0,28,0,0,33,2,53,0,0,53,9,47,58,89,0,0,0,0,0,4,5,0,56,0,0,0,0,0,9,0,56,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.81E2,2.6E1,1.79E2,2E0,7E0,1.9E1,1.64E2,1.5E1,4E0,3E0,1.5E1,4E0,1.7E1,1.47E2,1.1E1,4E0,1E1,5E0,2E0,2E0,6E0,1.1E1,1.4E1,1.33E2,4E0,7E0,2E0,4E0,3E0,8E0,4E0,1.29E2,6E0,2E0,7E0,1.22E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4610173E-3,9.7932376E-2,-4.4504614E-3,8.870057E-3,4.9648836E-4,-3.314924E-2,7.020033E-3,-2.3121499E-2,-1.00224335E-2,-5.438503E-3,5.8383252E-2,1.6109983E-2,-4.7826532E-2,7.8370846E-5,-3.111012E-3,7.136404E-3,4.1926984E-2,-1.6382972E-2,4.455269E-2,-6.4636454E-2,-4.9594564E-5,-5.14796E-2,5.3385887E-3,-4.3677186E-4,4.9276084E-2,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,-1.0062985E-1,-8.23799E-2,7.526664E-4,2.452092E-2,-9.486034E-3,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,-1,35,-1,-1,-1,-1,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.105056E-2,4.253041E-2,6.5741345E-2,0E0,0E0,9.4003044E-2,9.1824725E-2,5.4077312E-2,0E0,3.7433997E-2,3.4517996E-2,2.1062033E-2,2.668634E-2,2.929389E-2,0E0,0E0,9.794302E-3,1.1263341E-2,8.910602E-3,2.421417E-2,0E0,2.1960407E-2,2.8100949E-2,0E0,1.2502909E-2,0E0,0E0,0E0,0E0,0E0,1.0933511E-2,1.6032834E-2,0E0,1.9169066E-2,2.5302555E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,21,21,22,22,24,24,30,30,31,31,33,33,34,34],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,-1,36,-1,-1,-1,-1,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.813798E2,8.1356537E2,1E0,8.870057E-3,4.9648836E-4,4.4816768E8,2.7123457E3,2E0,-1.00224335E-2,4.462437E7,1.6064256E-2,6.822511E5,1.3088399E7,8.35E2,-3.111012E-3,7.136404E-3,5.4907064E0,8.901E3,5.1270317E-2,2.041E3,-4.9594564E-5,6.974669E7,1.6533424E6,-4.3677186E-4,7.317201E0,1.3931109E-4,-3.34137E-3,3.7710457E-3,8.9542795E-4,-1.6513594E-3,3.606E3,4.1749332E4,7.526664E-4,1.4653994E6,6.191606E0,2.8607487E-3,-3.990513E-4,-1.6642965E-3,-5.901635E-3,-9.575885E-4,-6.058401E-3,7.564508E-4,3.7614577E-3,2.7274885E-4,-1.9061548E-3],"split_indices":[33,51,109,0,0,44,4,8,0,45,57,32,1,2,0,0,54,2,38,0,0,7,50,0,53,0,0,0,0,0,10,32,0,50,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,5E0,1.98E2,2E0,3E0,5.6E1,1.42E2,5.4E1,2E0,1.15E2,2.7E1,2.1E1,3.3E1,1.06E2,9E0,3E0,2.4E1,1E1,1.1E1,2.4E1,9E0,9E0,9.7E1,3E0,2.1E1,8E0,2E0,4E0,7E0,1.4E1,1E1,6E0,3E0,4.2E1,5.5E1,1.8E1,3E0,3E0,7E0,3E0,3E0,3.7E1,5E0,3.7E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[4.282986E-4,4.1160197E-3,-9.1818266E-2,-1.1841053E-3,7.4616544E-2,-6.1210035E-3,3.2609148E-4,-1.09339515E-2,2.124421E-2,1.0531628E-1,7.8503106E-4,-7.0198895E-3,-9.130692E-2,-6.0027227E-4,4.235613E-2,6.323839E-3,6.2048313E-4,-3.122984E-2,2.2667213E-3,-1.7209452E-3,-6.5925727E-3,4.2787664E-2,-5.5588E-4,8.872538E-3,5.4689284E-2,3.1641412E-3,-7.729943E-2,-7.908227E-3,3.704432E-2,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,19,21,23,-1,-1,25,27,-1,-1,29,-1,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0795104E-2,7.4820675E-2,2.7605474E-2,4.112406E-2,2.3438096E-2,0E0,0E0,4.056964E-2,2.6312627E-2,1.8859334E-2,0E0,2.8399166E-2,1.00320205E-2,1.410548E-2,1.1490557E-2,0E0,0E0,5.5702664E-2,3.28481E-2,0E0,0E0,9.601074E-3,0E0,9.688978E-3,9.379521E-3,1.8371563E-2,2.2842146E-2,2.3135118E-2,2.1546472E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,20,22,24,-1,-1,26,28,-1,-1,30,-1,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4623123E8,5.4349E4,6.45153E5,1.1565725E0,2.0216698E7,-6.1210035E-3,3.2609148E-4,9.964687E-1,7.6126127E0,2.3274304E2,7.8503106E-4,6.126268E0,9.31E2,7.614E3,6.314179E2,6.323839E-3,6.2048313E-4,8.351808E-1,1.0685499E0,-1.7209452E-3,-6.5925727E-3,1.3559702E8,-5.5588E-4,2.6164518E7,1.5107028E-1,1.424506E6,9.313E3,6.4745765E0,6.6352534E-1,1.0122904E-4,4.029675E-3,1.3839344E-3,-1.9337956E-3,-3.476462E-4,2.9965858E-3,1.553697E-3,-1.2597209E-3,-5.336409E-3,-1.2000917E-3,3.2235107E-3,-5.8846874E-4,2.8890773E-4,3.439223E-3],"split_indices":[45,2,28,42,47,0,0,39,54,56,0,54,0,10,4,0,0,27,57,0,0,7,0,32,38,29,2,54,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.99E2,7E0,1.86E2,1.3E1,5E0,2E0,1.3E2,5.6E1,8E0,5E0,1.25E2,5E0,2.8E1,2.8E1,6E0,2E0,3.4E1,9.1E1,3E0,2E0,5E0,2.3E1,8E0,2E1,2E1,1.4E1,7.1E1,2E1,3E0,2E0,6E0,2E0,2E0,1.8E1,1E1,1E1,8E0,6E0,3E0,6.8E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.1866337E-3,-1.4964924E-3,7.432874E-2,-7.1397156E-2,2.0264261E-3,1.4760831E-1,2.5388267E-2,2.4828428E-4,-5.377491E-3,-2.0973684E-2,1.2397231E-2,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,-8.560193E-2,-1.29591115E-2,2.8582712E-4,3.782403E-2,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,6.9721565E-2,-6.839452E-3,-3.537567E-3,4.555346E-2,5.6208028E-5,4.49603E-3,-2.1360328E-2,1.4701819E-2,6.0154725E-2,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.437666E-2,4.619424E-2,6.0806543E-2,2.7483527E-2,4.295682E-2,4.940191E-2,1.3439575E-2,0E0,0E0,2.7896874E-2,3.8175512E-2,0E0,0E0,0E0,0E0,1.4757402E-2,1.4061235E-2,4.253052E-2,3.747972E-2,0E0,0E0,0E0,0E0,1.2324721E-2,2.471229E-2,0E0,2.2664323E-2,0E0,0E0,2.4652388E-2,2.1451397E-2,1.7983623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,23,23,24,24,26,26,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,5.314E3,1.256E3,6.035227E9,1.05190264E8,5.3696685E0,9.532E3,2.4828428E-4,-5.377491E-3,1.2338027E5,8.1350375E5,1.0787817E-2,1.7834331E-3,-8.8212016E-4,2.4046602E-3,1.4084E4,1.7669278E5,1.3448397E8,6.0599937E-6,-4.9006706E-4,-5.8813626E-3,2.1698806E-3,-8.509144E-4,2.8E1,1.1812092E0,-3.537567E-3,2.7991664E-1,5.6208028E-5,4.49603E-3,5.791575E-1,3.4E1,1.3839568E1,2.0660774E-4,-1.9377028E-4,-2.5206523E-3,-1.3422824E-3,1.4573362E-3,8.522566E-4,3.7150076E-3],"split_indices":[4,9,0,5,7,53,9,0,0,28,28,0,0,0,0,9,28,7,38,0,0,0,0,3,42,0,38,0,0,41,3,56,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.86E2,1.7E1,8E0,1.78E2,6E0,1.1E1,3E0,5E0,5.5E1,1.23E2,3E0,3E0,4E0,7E0,5E0,5E1,8.4E1,3.9E1,2E0,3E0,3E0,4.7E1,7E0,7.7E1,2E0,3.7E1,2E0,5E0,4.6E1,3.1E1,2.7E1,1E1,3E1,1.6E1,8E0,2.3E1,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4136843E-3,-1.7078152E-2,1.7051285E-2,-4.9061432E-2,-3.8076276E-3,2.0602716E-2,-4.516626E-3,-3.270243E-4,-7.544991E-2,6.5981923E-3,-3.537236E-2,-7.553134E-3,3.3188965E-2,-5.2146493E-3,-2.832102E-2,-1.7100578E-2,3.0886639E-2,1.3335078E-3,-4.885492E-2,4.1569836E-4,-4.885816E-2,2.7083127E-2,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,4.4731785E-2,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,4.3737285E-2,-3.549587E-3,6.7125736E-3,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6693766E-2,4.4803295E-2,3.7026227E-2,3.4130327E-2,2.5196122E-2,3.143581E-2,0E0,0E0,2.535192E-2,3.3913348E-2,1.7419232E-2,1.9031767E-2,2.084884E-2,0E0,1.2429329E-2,1.4870551E-2,2.5628617E-2,0E0,1.2944881E-2,0E0,1.3053814E-2,2.9234491E-2,0E0,0E0,0E0,0E0,0E0,2.8370071E-2,0E0,0E0,0E0,0E0,0E0,1.1751018E-2,1.2866924E-2,1.5719058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,27,27,33,33,34,34,35,35],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9640406E-1,5.511724E0,1.2083455E1,4.958582E-1,9.952575E5,3.225E3,-4.516626E-3,-3.270243E-4,1.0702152E-2,1.643614E6,5.916872E0,2.806E3,1.6974416E6,-5.2146493E-3,5.815166E0,1E0,5.2192265E1,1.3335078E-3,4.4589956E5,4.1569836E-4,1.8465776E8,8.062193E-1,4.679654E-3,-2.3958324E-3,1.4554259E-3,-1.1575769E-3,2.4818077E-3,1.01978E5,-1.442557E-3,-3.044713E-3,3.84936E-4,1.2072253E-4,-3.9009824E-3,1.0252E4,1.0266666E1,7.1706E4,3.7174122E-3,6.293774E-4,2.6604726E-3,-8.034792E-4,2.006297E-3,1.6411785E-3,-1.8418606E-3],"split_indices":[39,53,35,27,51,2,0,0,38,32,53,2,28,0,54,113,56,0,28,0,5,27,0,0,0,0,0,29,0,0,0,0,0,9,54,29,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.94E2,1.05E2,8.9E1,3E1,7.5E1,8.7E1,2E0,1.2E1,1.8E1,5.7E1,1.8E1,2.7E1,6E1,1E1,8E0,2.9E1,2.8E1,3E0,1.5E1,2E1,7E0,5.6E1,4E0,6E0,2E0,2.7E1,2E0,2.3E1,5E0,1.2E1,3E0,3E0,4E0,3.6E1,2E1,1.1E1,1.2E1,1E1,2.6E1,1.6E1,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.348882E-3,-1.4007735E-2,2.0349778E-2,-8.801855E-3,-7.60288E-2,3.1053293E-2,-1.0958469E-2,-5.7257973E-2,-5.0193043E-3,-4.4670764E-5,-4.9110665E-3,2.440679E-2,1.0895903E-1,-3.7627418E-2,2.2707816E-2,-4.087384E-3,3.10375E-4,3.1430066E-2,-1.0879586E-2,4.352828E-3,4.6011005E-2,6.7795296E-3,1.5659243E-3,-6.1382737E-2,-3.0169598E-4,4.8260357E-5,2.7864138E-3,6.0025465E-2,-8.0559475E-5,-1.663934E-2,1.3024335E-3,1.8343031E-2,-1.7778914E-2,-1.4250753E-4,6.0098227E-2,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,8.744232E-3,-2.3721121E-2,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,6.921383E-2,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,-1,41,43,-1,45,-1,-1,-1,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.254653E-2,3.32326E-2,3.619688E-2,1.76748E-2,1.4859837E-2,3.8764708E-2,2.5994083E-2,1.244238E-2,1.9993477E-2,0E0,0E0,3.2293484E-2,9.498283E-3,1.1564847E-2,9.696336E-3,0E0,0E0,1.2398537E-2,1.7790027E-2,1.2711223E-2,2.4995036E-2,0E0,0E0,1.1450622E-2,0E0,0E0,0E0,1.1509789E-2,0E0,1.27906855E-2,0E0,9.788271E-3,8.809424E-3,0E0,1.9199878E-2,0E0,0E0,0E0,0E0,8.899177E-3,1.8784469E-2,0E0,0E0,0E0,0E0,1.800523E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,27,27,29,29,31,31,32,32,34,34,39,39,40,40,45,45],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,-1,42,44,-1,46,-1,-1,-1,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.978001E2,3.6509445E1,1.3983983E10,1.1475E2,2.1434378E5,5.2086E4,1.5461028E12,8.842206E7,4.1740056E7,-4.4670764E-5,-4.9110665E-3,4.9754506E5,1.8136424E7,1.3088399E7,6.1921E4,-4.087384E-3,3.10375E-4,4.327473E5,2.6493E4,5.0123274E-1,1E0,6.7795296E-3,1.5659243E-3,1.775894E10,-3.0169598E-4,4.8260357E-5,2.7864138E-3,1.0380544E8,-8.0559475E-5,1.43E2,1.3024335E-3,1.1201525E2,2.513265E5,-1.4250753E-4,1.3578947E1,1.753593E-4,-3.915942E-3,4.7259862E-4,4.639547E-3,6.1579504E2,2.109E3,1.3840314E-3,-8.9389575E-4,-1.4391522E-3,1.2162981E-3,2.1525126E0,-5.790745E-4,1.4545477E-3,-7.8526477E-4,-2.3957968E-3,-4.8575958E-4,4.283088E-3,1.2663646E-3],"split_indices":[52,56,5,52,28,2,31,5,5,0,0,28,1,1,10,0,0,28,10,27,16,0,0,5,0,0,0,7,0,10,0,56,33,0,54,0,0,0,0,4,2,0,0,0,0,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.04E2,1.06E2,9.7E1,7E0,7.9E1,2.7E1,6E0,9.1E1,2E0,5E0,7.4E1,5E0,1.5E1,1.2E1,4E0,2E0,1.2E1,7.9E1,3.9E1,3.5E1,3E0,2E0,8E0,7E0,8E0,4E0,6E0,6E0,6.9E1,1E1,2.4E1,1.5E1,8E0,2.7E1,2E0,6E0,3E0,3E0,1.5E1,5.4E1,1.9E1,5E0,1.2E1,3E0,2.4E1,3E0,8E0,7E0,1.8E1,3.6E1,1.6E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.8318873E-3,-1.0389051E-2,6.847433E-2,-9.090252E-2,-7.818896E-3,1.4557077E-1,1.5369026E-2,-1.1227432E-3,-5.793796E-3,-3.6059227E-3,-5.7291586E-2,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,-1.0661409E-2,3.3012316E-2,-1.4323252E-3,-7.734146E-3,-6.9573727E-3,-4.916107E-2,4.209554E-3,2.0994669E-2,-5.233113E-2,-3.3860938E-3,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.4740467E-2,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[8.666744E-2,4.0054876E-2,5.917617E-2,8.709792E-3,3.9811417E-2,4.4877455E-2,9.926351E-3,0E0,0E0,4.6339944E-2,3.7794035E-2,0E0,0E0,0E0,0E0,2.1048589E-2,1.682442E-2,0E0,0E0,2.2184646E-2,1.6582798E-2,0E0,2.2704527E-2,1.6044017E-2,2.14594E-2,0E0,0E0,0E0,9.11136E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,19,19,20,20,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[9.417025E3,2.3125E0,2E0,5.34E2,7.8032995E6,6.677E3,6.881048E-1,-1.1227432E-3,-5.793796E-3,1.6503105E3,5.8199444E0,1.1405231E-2,2.0726947E-3,-2.2492031E-4,3.0929737E-3,9.5096946E-1,6.0775385E0,-1.4323252E-3,-7.734146E-3,1.2425E2,6.128552E0,4.209554E-3,5.590909E0,7.005162E5,1.2804077E7,-3.3503212E-3,6.001957E-4,-1.9922394E-3,3.842571E7,-4.138816E-3,-1.2696077E-4,3.9782308E-4,-8.603632E-4,2.6649933E-3,5.62697E-4],"split_indices":[4,53,8,0,47,9,34,0,0,52,53,0,0,0,0,27,54,0,0,52,53,0,53,28,45,0,0,0,45,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.96E2,1.4E1,5E0,1.91E2,5E0,9E0,2E0,3E0,1.77E2,1.4E1,2E0,3E0,7E0,2E0,1.49E2,2.8E1,1.2E1,2E0,1.37E2,1.2E1,4E0,2.4E1,9E0,1.28E2,9E0,3E0,4E0,2E1,5E0,4E0,7.1E1,5.7E1,1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6622468E-4,-4.8488043E-3,5.08311E-2,-7.834397E-4,-6.2016364E-2,7.5922403E-3,1.9395813E-2,2.0835144E-3,-6.411687E-2,-4.91167E-3,-1.6307954E-3,4.5530796E-2,-4.4596207E-4,-8.775512E-3,2.092946E-2,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,-4.312229E-3,-4.002928E-2,-6.387113E-4,3.425055E-2,-6.662406E-4,-2.56261E-3,-3.120549E-3,1.352972E-4,4.647988E-2,-1.3092252E-3,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,23,25,-1,27,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1110663E-2,4.5503184E-2,5.6485604E-2,3.354321E-2,1.1037149E-2,0E0,1.1613244E-2,3.640647E-2,3.848807E-2,0E0,0E0,9.618994E-3,0E0,1.5590253E-2,2.9935649E-2,0E0,0E0,0E0,0E0,1.7591223E-2,1.4617277E-2,0E0,2.0499848E-2,1.4878065E-2,0E0,0E0,0E0,2.9250316E-2,9.96951E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,22,22,23,23,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,24,26,-1,28,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[5.2062725E3,1.8600049E10,1.4164306E-3,7.8032995E6,2.884462E0,7.5922403E-3,5.861021E7,6.978001E2,7E0,-4.91167E-3,-1.6307954E-3,2.819228E-1,-4.4596207E-4,1E0,2.8626094E5,-4.1884492E-4,-7.483217E-3,-4.864919E-4,3.197675E-3,3.6509445E1,4.966E3,-6.387113E-4,5.2192265E1,4.454691E5,-2.56261E-3,-3.120549E-3,1.352972E-4,9.42191E-1,2.6673E4,9.1003685E-4,-4.2580368E-4,2.8414459E-3,-1.0121675E-3,-1.0272213E-3,1.6833383E-3],"split_indices":[52,5,57,47,53,0,45,52,8,0,0,57,0,89,28,0,0,0,0,56,2,0,56,48,0,0,0,27,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.95E2,1.7E1,1.83E2,1.2E1,3E0,1.4E1,1.76E2,7E0,4E0,8E0,7E0,7E0,1.12E2,6.4E1,5E0,2E0,2E0,5E0,9.9E1,1.3E1,1.8E1,4.6E1,9.3E1,6E0,8E0,5E0,3.4E1,1.2E1,2.7E1,6.6E1,2.9E1,5E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.3941145E-4,-2.0441902E-3,8.919721E-3,-1.1523694E-2,2.1440972E-2,-9.1706775E-3,-5.6796446E-3,5.424757E-3,1.2543296E-2,-1.505701E-2,4.4461366E-2,1.9426743E-2,-3.729582E-3,-7.865752E-3,-3.6841E-2,3.1923247E-4,3.1530529E-3,-4.811276E-3,5.0493825E-2,-1.1715324E-2,1.7492641E-3,-1.6764663E-2,-3.0829909E-3,-2.7253285E-2,1.2633276E-2,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0149213E-1,4.410473E-2,0E0,3.3826478E-2,4.4260666E-2,4.4551205E-2,0E0,0E0,3.4023255E-2,1.944866E-2,1.0361386E-2,3.789317E-2,0E0,1.6611358E-2,1.5435919E-2,0E0,0E0,1.1697531E-2,2.4396595E-2,1.604245E-2,0E0,1.6484369E-2,0E0,1.106496E-2,1.5908504E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19,21,21,23,23,24,24],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9028345E2,8.1350375E5,8.919721E-3,2.61E2,5.0655737E0,3.2432E4,-5.6796446E-3,5.424757E-3,7.251039E0,1.8747724E7,1.0252E4,7.6188426E0,-3.729582E-3,2.513265E5,6.995449E-1,3.1923247E-4,3.1530529E-3,1.2153278E-3,9.161318E0,4.8E1,1.7492641E-3,4.588941E-1,-3.0829909E-3,6.809655E0,2.0350546E-3,4.626764E-3,1.136513E-3,1.5251753E-3,-7.676115E-4,-7.086038E-5,-3.5919722E-3,-1.9680187E-3,1.4686801E-3,2.6364126E-3,-4.4721604E-4],"split_indices":[56,28,0,3,54,2,0,0,57,32,9,54,0,33,27,0,0,39,54,10,0,38,0,54,42,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.96E2,2E0,1.4E2,5.6E1,1.38E2,2E0,4E0,5.2E1,1.25E2,1.3E1,4.9E1,3E0,9.5E1,3E1,5E0,8E0,2.8E1,2.1E1,8.8E1,7E0,1.8E1,1.2E1,1.2E1,1.6E1,7E0,1.4E1,7E0,8.1E1,1.5E1,3E0,1E1,2E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.881544E-3,-6.655184E-3,2.188296E-2,-2.8302656E-3,-2.666419E-3,-2.7059872E-2,3.2757495E-2,-9.625506E-3,2.9928401E-2,-5.4137424E-2,7.742816E-7,2.5661085E-2,7.91566E-2,1.4555868E-3,-2.7334698E-2,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,3.091641E-2,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,-5.2867554E-2,1.0829677E-2,-4.0220562E-2,6.7792195E-3,6.600478E-2,1.1288791E-3,-7.709266E-4,-4.6325806E-3,-1.4827026E-2,2.3715133E-2,-2.952948E-4,-5.428135E-2,2.0235393E-3,-9.2693866E-4,8.701906E-2,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,35,37,-1,-1,-1,39,41,-1,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3877617E-2,2.2490527E-2,5.198053E-2,2.5887422E-2,0E0,1.3197487E-2,2.3980856E-2,1.8878557E-2,3.1391144E-2,9.128876E-3,0E0,1.9829843E-2,2.8112642E-2,3.112634E-2,1.6537398E-2,0E0,0E0,0E0,0E0,1.5805755E-2,0E0,0E0,0E0,1.2595711E-2,1.7542738E-2,1.2636464E-2,1.0678305E-2,1.2879096E-2,0E0,0E0,0E0,9.937875E-3,1.8333944E-2,0E0,1.3910651E-2,0E0,0E0,1.5306905E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,19,19,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,37,37],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,36,38,-1,-1,-1,40,42,-1,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2936785E3,3.0978662E6,3.673257E0,1.0907971E6,-2.666419E-3,4.8601307E-2,5.4349E4,2.2906403E-1,1E0,3.26E2,7.742816E-7,1.1778515E7,7.65E2,9.7043E4,4.4589956E5,-4.310812E-4,3.387207E-3,1.7881698E-4,-3.464076E-3,7.573E3,-1.2679292E-3,-9.0381445E-4,5.1542684E-3,4.0289572E2,3.1E1,1.0485419E8,1.2857332E6,1.1865996E1,1.1288791E-3,-7.709266E-4,-4.6325806E-3,1.5403727E-1,1.9540394E7,-2.952948E-4,1.6680729E0,2.0235393E-3,-9.2693866E-4,1.884E3,3.6378313E-4,-1.1705728E-3,1.913583E-3,1.6696395E-3,-1.1464225E-3,-3.4304948E-3,-3.6083275E-4,5.4672193E-3,3.5865017E-4],"split_indices":[4,28,53,28,0,38,2,57,16,0,0,47,0,1,28,0,0,0,0,9,0,0,0,4,3,7,48,54,0,0,0,57,45,0,39,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.22E2,9.5E1,1.14E2,8E0,1.7E1,7.8E1,9.5E1,1.9E1,8E0,9E0,6.9E1,9E0,5.9E1,3.6E1,1E1,9E0,2E0,6E0,6.3E1,6E0,2E0,7E0,8E0,5.1E1,2.6E1,1E1,1E1,5.3E1,5E0,3E0,1.7E1,3.4E1,8E0,1.8E1,4E0,6E0,7E0,3E0,1.5E1,2E0,2.8E1,6E0,1.3E1,5E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.403546E-3,-9.570334E-4,-4.4911288E-2,-7.611177E-2,1.5534413E-3,-2.1605039E-2,-8.161089E-3,-5.0543556E-3,-6.690376E-4,-5.41183E-3,1.55741805E-2,-2.0407424E-3,4.387262E-4,-1.5254784E-3,-6.8485916E-2,3.5775187E-3,7.775442E-3,5.1245918E-3,-2.5670715E-2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,1.9513896E-2,2.5408894E-2,-7.420587E-3,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4000924E-2,3.5022937E-2,7.077891E-2,8.954152E-3,1.7662771E-2,1.4694375E-2,0E0,0E0,0E0,2.955602E-2,2.6538467E-2,0E0,0E0,1.8596815E-2,1.0216929E-2,0E0,1.7291361E-2,2.3345338E-2,1.594018E-2,0E0,0E0,0E0,8.91481E-3,1.4992485E-2,2.6731981E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,13,13,14,14,16,16,17,17,18,18,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.361366E7,1.0135136E0,7.5656E4,1.38E2,1.500621E3,3.0687964E3,-8.161089E-3,-5.0543556E-3,-6.690376E-4,1.2473612E7,4.965E3,-2.0407424E-3,4.387262E-4,2.1067844E7,1E0,3.5775187E-3,5.1034484E0,1.2623029E7,8.103274E2,-2.580997E-4,-4.5112926E-3,-9.208097E-4,9.43017E5,2.1768606E5,1.1565725E0,-3.0443382E-3,-3.8210308E-4,-6.8026554E-4,1.2765435E-3,3.0987216E-3,6.603516E-4,-1.4007594E-3,6.9094513E-4],"split_indices":[45,56,10,0,4,52,0,0,0,50,2,0,0,45,8,0,53,12,4,0,0,0,48,28,42,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.84E2,2.5E1,5E0,1.79E2,2.2E1,3E0,3E0,2E0,1.2E2,5.9E1,1.3E1,9E0,1.14E2,6E0,6E0,5.3E1,9E1,2.4E1,2E0,4E0,1.6E1,3.7E1,3.4E1,5.6E1,7E0,1.7E1,6E0,3.1E1,7E0,2.7E1,2.8E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.779959E-6,1.8453373E-3,-5.9890207E-3,-1.285875E-3,6.8091586E-2,-5.2687E-3,3.303071E-2,8.70132E-3,5.314312E-4,-5.49993E-2,-3.4504228E-3,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,-8.831604E-3,1.3855618E-2,-1.8325018E-2,8.489807E-3,-3.5646225E-3,4.3126993E-2,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.611087E-2,4.1366342E-2,0E0,2.6279328E-2,5.5009544E-2,1.53631605E-2,2.091176E-2,0E0,0E0,1.1431506E-2,1.5674567E-2,0E0,0E0,0E0,0E0,2.1203808E-2,2.0550307E-2,1.3397509E-2,1.3573993E-2,1.2377562E-2,1.1399828E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6085916E7,2.0494681E2,-5.9890207E-3,1E0,9.900047E6,2.3125E0,1.04E3,8.70132E-3,5.314312E-4,3.26E2,2.6673E4,2.6435181E-3,-6.942157E-4,2.2105813E-4,-4.1807243E-3,3.9E1,5.8167644E0,5.540166E-3,7.31E3,3.7988422E5,2.3E1,1.632805E-4,-1.2518176E-3,1.7453751E-3,-1.0835632E-4,-1.4243656E-3,6.973504E-4,4.036547E-3,1.027257E-3],"split_indices":[29,56,0,79,1,53,0,0,0,0,10,0,0,0,0,3,53,57,9,28,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.98E2,2E0,1.9E2,8E0,1.71E2,1.9E1,2E0,6E0,5E0,1.66E2,1.3E1,6E0,2E0,3E0,1.27E2,3.9E1,8.2E1,4.5E1,2.5E1,1.4E1,2.1E1,6.1E1,1.2E1,3.3E1,1E1,1.5E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.3686826E-4,-3.519467E-3,5.055597E-2,-2.117268E-3,-4.640776E-3,7.0017064E-3,1.523785E-2,-4.871748E-2,3.465182E-4,-9.892045E-4,4.3544497E-2,6.1831385E-4,-3.1955007E-3,-5.2515082E-2,2.6555143E-3,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,-1.2239224E-3,2.5115369E-2,9.1643715E-3,-1.2135973E-2,5.6670893E-2,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,15,-1,-1,17,19,-1,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0816244E-2,2.5682297E-2,4.724356E-2,2.2864658E-2,0E0,0E0,1.2965487E-2,1.1241557E-2,2.333033E-2,0E0,8.747165E-3,0E0,0E0,2.0471422E-2,1.5975818E-2,0E0,0E0,0E0,0E0,1.790836E-2,3.341707E-2,2.1425955E-2,2.8729388E-2,1.4656175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,16,-1,-1,18,20,-1,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,6.7948157E-1,1.0038867E4,1.2425E2,-4.640776E-3,7.0017064E-3,6.881048E-1,3.7790768E2,2.3125E0,-9.892045E-4,2.7445236E-1,6.1831385E-4,-3.1955007E-3,8.591E3,4.4E1,3.6094838E-3,9.903872E-5,4.7422762E-4,-4.468538E-3,3.3E1,8.87E2,1.127938E6,4.570007E7,1.7E1,-6.437014E-4,-1.2700462E-4,1.535144E-3,2.18674E-3,-8.995893E-4,3.6758832E-3,1.8696976E-4],"split_indices":[4,34,4,52,0,0,34,33,53,0,38,0,0,2,3,0,0,0,0,3,0,28,5,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,2E2,1.4E1,1.98E2,2E0,3E0,1.1E1,9E0,1.89E2,5E0,6E0,2E0,7E0,7E0,1.82E2,3E0,3E0,3E0,4E0,1.56E2,2.6E1,8E1,7.6E1,1.4E1,1.2E1,5.3E1,2.7E1,7E0,6.9E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.8720286E-3,2.203888E-3,-3.8517896E-2,7.5318073E-3,-6.002819E-4,-2.1684619E-2,-8.622627E-3,1.0079312E-3,-4.2458405E-3,-2.1083932E-3,1.5030096E-3,-9.4883353E-4,3.8221332E-3,2.3723133E-3,-7.695643E-4,2.9969336E-3,-3.9027993E-2,-1.1395355E-2,1.3584136E-2,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,13,15,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0922936E-2,7.110839E-2,6.3847005E-2,0E0,2.2939872E-2,1.3612385E-2,0E0,2.4651024E-2,0E0,0E0,1.2223788E-2,2.4006953E-2,0E0,0E0,0E0,2.2249037E-2,1.5829908E-2,9.478996E-3,2.061506E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,10,10,11,11,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,14,16,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,3.7902024E0,7.5656E4,7.5318073E-3,1.3105492E1,4.6E1,-8.622627E-3,5.9165E4,-4.2458405E-3,-2.1083932E-3,1E0,2.72258E5,3.8221332E-3,2.3723133E-3,-7.695643E-4,7.6077003E0,5.1737704E7,3.1347964E-2,2.6673E4,-8.57854E-4,-4.5490144E-3,2.4675918E-4,-1.0014722E-3,2.034659E-4,1.9415156E-3],"split_indices":[45,54,10,0,35,3,0,2,0,0,8,9,0,0,0,54,32,57,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.65E2,2.8E1,2E0,1.63E2,2.6E1,2E0,1.61E2,2E0,1.3E1,1.3E1,1.58E2,3E0,3E0,1E1,1.44E2,1.4E1,6.1E1,8.3E1,1.1E1,3E0,2.2E1,3.9E1,6.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-6.800024E-4,1.8960047E-3,-6.0736863E-3,-3.6225263E-2,6.760957E-3,-4.00987E-3,-1.3761937E-2,4.2979596E-3,5.311085E-3,1.561032E-3,-1.3046184E-3,-4.1869044E-2,6.551281E-3,-3.4434297E-3,1.4532126E-3,2.6219185E-3,3.280929E-2,-3.7609767E-3,2.3818783E-2,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,11,-1,-1,-1,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.290568E-2,3.6038343E-2,0E0,2.1682842E-2,4.2899136E-2,0E0,1.0427506E-2,1.7855216E-2,0E0,0E0,0E0,1.8689083E-2,1.662869E-2,0E0,0E0,1.9301808E-2,1.2423562E-2,2.420176E-2,2.810772E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,11,11,12,12,15,15,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,-1,10,12,-1,-1,-1,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.02E2,3.8834286E0,-6.0736863E-3,2.3443186E3,5.2086E4,-4.00987E-3,1.2633E4,1.8E2,5.311085E-3,1.561032E-3,-1.3046184E-3,1.3448806E8,4.6E1,-3.4434297E-3,1.4532126E-3,1.1482285E6,1.10597E5,8.351808E-1,3.1E1,4.0768534E-3,9.568626E-4,1.8610083E-4,-1.5874148E-3,2.1685886E-3,-7.8801316E-4],"split_indices":[8,53,0,4,2,0,2,11,0,0,0,7,3,0,0,28,29,27,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.92E2,3E0,2.1E1,1.71E2,6E0,1.5E1,1.68E2,3E0,3E0,1.2E1,7E0,1.61E2,5E0,2E0,1.41E2,2E1,1.09E2,3.2E1,3E0,1.7E1,8.7E1,2.2E1,2.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.1066506E-4,4.845112E-3,-3.1581253E-2,6.452437E-2,-6.428997E-4,-5.0642762E-2,5.648247E-4,1.1988164E-1,-7.721919E-4,-4.0026516E-2,3.5905282E-3,-2.9436452E-2,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,-2.7368726E-2,7.2603608E-3,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,1.2873769E-2,-1.3507972E-2,3.7806993E-3,1.0231802E-2,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1],"loss_changes":[3.406493E-2,5.8340125E-2,2.528421E-2,6.86559E-2,2.7487058E-2,4.734615E-2,0E0,2.8751194E-2,0E0,2.1383194E-2,1.7127382E-2,1.0340074E-2,0E0,0E0,0E0,0E0,0E0,2.1022052E-2,1.5795767E-2,0E0,0E0,0E0,0E0,1.7355684E-2,1.4121132E-2,0E0,1.356209E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,17,17,18,18,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1],"split_conditions":[1.8000048E10,6.677E3,3.244E3,5.1E1,1.142E3,3.08E3,5.648247E-4,5.8229775E3,-7.721919E-4,5.244755E-3,1.13002876E-4,7.520736E6,-8.179296E-3,3.1937908E-3,1.0050461E-2,-1.4967358E-4,-3.7275134E-3,8.3949E4,7.384892E0,-1.8439381E-3,1.3760552E-3,3.8673286E-4,-3.0989514E-3,3.0404909E0,3.501498E7,3.7806993E-3,4.3E1,-2.187538E-4,-3.3441412E-3,1.7933325E-4,1.4848422E-3],"split_indices":[5,9,0,3,2,0,0,4,0,57,38,29,0,0,0,0,0,12,53,0,0,0,0,56,32,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.77E2,2.9E1,1.4E1,1.63E2,2E1,9E0,8E0,6E0,1.5E1,1.48E2,1.8E1,2E0,6E0,2E0,8E0,7E0,1.5E1,1.33E2,1.6E1,2E0,8E0,7E0,1.05E2,2.8E1,3E0,1.02E2,2.5E1,3E0,7.8E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-2.5543855E-3,-3.670569E-4,-9.478624E-2,-4.503718E-3,3.9570075E-2,-2.891256E-4,-7.399056E-3,7.92847E-3,-1.9427676E-2,1.2930362E-1,9.055213E-3,-2.102676E-2,1.859223E-2,-4.328215E-3,-3.512878E-2,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,6.881532E-3,-3.8464133E-2,3.4366657E-3,1.07829925E-2,-2.6562288E-3,8.551802E-5,-1.3136177E-2,-5.552462E-2,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,-1.9891918E-2,1.758985E-2,-5.6563243E-2,1.090067E-2,-8.1699066E-2,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,33,35,-1,-1,-1,-1,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3541346E-2,3.5201967E-2,2.452866E-2,3.5962764E-2,5.359327E-2,0E0,0E0,3.312072E-2,2.0710837E-2,1.6731478E-2,1.720418E-2,1.4377463E-2,3.0927734E-2,1.4162992E-2,1.8558595E-2,0E0,0E0,0E0,0E0,9.007291E-3,9.359658E-3,0E0,1.4757078E-2,0E0,0E0,2.358127E-2,2.1488503E-2,0E0,0E0,0E0,0E0,1.4124965E-2,1.671442E-2,8.771379E-3,1.1919808E-2,1.0015354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,25,25,26,26,31,31,32,32,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,34,36,-1,-1,-1,-1,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.25E2,5.2062725E3,2.133183E1,1.6278354E7,1.1823068E7,-2.891256E-4,-7.399056E-3,1.549E3,8.062193E-1,6.23E2,2.0090337E0,4.0431567E-3,1.26747945E5,1.775044E7,7.573E3,1.7648255E-3,8.723087E-3,-3.0351247E-4,3.7667116E-3,7.107292E6,8E-3,3.4366657E-3,3.104892E4,-2.6562288E-3,8.551802E-5,3.7614656E5,1.2067E4,1.6961485E-3,-8.797278E-4,-3.7114355E-3,-1.0812322E-3,4.9973932E-1,5.1821535E4,2.849513E3,9.8613055E4,9.377185E-1,-7.760048E-4,-1.8892186E-3,1.5768221E-3,2.3922722E-3,3.9321068E-4,-4.0106266E-3,-4.921803E-4,-2.7314315E-4,2.8087925E-3,-4.92643E-3,-1.4700346E-3],"split_indices":[8,52,54,45,1,0,0,2,27,0,34,42,47,45,9,0,0,0,0,45,57,0,33,0,0,28,9,0,0,0,0,27,33,4,33,27,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.11E2,4E0,1.92E2,1.9E1,2E0,2E0,1.05E2,8.7E1,4E0,1.5E1,2.8E1,7.7E1,4.5E1,4.2E1,2E0,2E0,1.3E1,2E0,1.1E1,1.7E1,9E0,6.8E1,4E0,4.1E1,2.1E1,2.1E1,5E0,6E0,4E0,1.3E1,1.2E1,5.6E1,7E0,1.4E1,1.2E1,9E0,9E0,3E0,1.2E1,4.4E1,4E0,3E0,1.1E1,3E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.7936366E-3,2.4298213E-3,-4.112654E-2,-4.0990007E-3,4.8309796E-2,-2.183913E-2,-8.530635E-3,2.3222237E-3,-3.93802E-2,1.3060242E-1,8.8548794E-4,-4.1647155E-2,3.9303087E-4,-8.698616E-3,2.1892874E-2,-5.8111317E-2,-5.4723965E-3,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,-4.9995827E-3,-3.3989844E-3,3.6737766E-2,-1.1948632E-2,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,-4.3598227E-2,7.2821265E-4,4.7861084E-2,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1747835E-2,5.5313785E-2,6.086246E-2,3.6792308E-2,5.4622788E-2,1.3965219E-2,0E0,2.9954968E-2,1.5560266E-2,4.6919495E-2,0E0,1.526756E-2,0E0,1.9780226E-2,2.5557641E-2,8.726589E-3,9.571268E-3,0E0,0E0,0E0,0E0,1.8823985E-2,0E0,1.4614824E-2,9.260894E-3,0E0,0E0,0E0,0E0,1.8716276E-2,2.0947833E-2,9.567626E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,21,21,23,23,24,24,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3849624E1,9.276886E1,1.019731E0,4.9386452E7,1.6064256E-2,1.7810251E1,-8.530635E-3,8.4182155E10,4.4E1,7.257E3,8.8548794E-4,2.6239506E5,3.9303087E-4,4.991491E8,1.7486866E7,8.3764136E2,9.39616E-1,1.7896533E-4,9.399709E-3,-4.2009302E-3,-7.370813E-4,6.974669E7,-3.3989844E-3,4.493248E6,2.6678162E9,-4.2009973E-3,-1.5842157E-3,8.446043E-4,-2.1438627E-3,1.0363E4,7.25E2,1.7747324E7,-3.4755936E-5,-1.7607809E-3,5.313654E-4,-2.466589E-4,-4.322573E-3,6.0066866E-4,-1.1190454E-3,3.6961967E-3,1.5591269E-3],"split_indices":[54,56,34,45,57,56,0,31,3,2,0,28,0,7,32,52,27,0,0,0,0,7,0,1,12,0,0,0,0,9,0,45,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.83E2,2.4E1,1.61E2,2.2E1,2.2E1,2E0,1.37E2,2.4E1,5E0,1.7E1,1.3E1,9E0,8.8E1,4.9E1,1.5E1,9E0,2E0,3E0,4E0,9E0,8.4E1,4E0,3.4E1,1.5E1,6E0,9E0,6E0,3E0,1E1,7.4E1,2.6E1,8E0,7E0,8E0,6E0,4E0,5E1,2.4E1,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.3284292E-3,-3.6234653E-3,6.495424E-2,-3.3305318E-4,-4.9157396E-2,9.216801E-3,3.1242464E-2,-1.132699E-2,1.368838E-2,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,-1.8146961E-3,-3.4096006E-2,5.2584518E-2,5.3565246E-3,-6.317367E-3,3.768521E-3,-8.287692E-2,-1.5646296E-2,4.7426983E-3,3.285665E-4,-1.2222636E-3,1.7330685E-3,-9.491318E-3,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,4.4616484E-3,-1.594493E-2,1.9072331E-2,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,27,29,-1,-1,31,-1,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5454505E-2,2.8899927E-2,5.700592E-2,2.8056335E-2,2.499102E-2,0E0,9.989295E-3,2.2029854E-2,2.564449E-2,0E0,0E0,0E0,0E0,2.65384E-2,2.5703099E-2,2.8132629E-2,1.3379129E-2,1.2622242E-2,0E0,1.1075061E-2,1.793515E-2,0E0,0E0,1.7036377E-2,0E0,1.3837478E-2,0E0,0E0,0E0,0E0,8.933199E-3,9.179806E-3,2.1313196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,28,30,-1,-1,32,-1,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.861832E3,7.8032995E6,1.4164306E-3,4.7132755E6,5.7592998E0,9.216801E-3,6.5772717E8,2.1557302E6,6.505428E6,-1.1004698E-3,-6.3316454E-3,2.1208725E-3,-1.1878744E-3,2.0041434E5,5.9083433E0,7.56E2,9.855582E5,1E0,3.768521E-3,1.1485E4,2.831E3,4.7426983E-3,3.285665E-4,3.6E1,1.7330685E-3,4.43E2,2.3567046E-3,-5.962461E-3,-1.6833517E-3,-2.6234328E-3,2.8037289E1,7.894435E-1,1.1156E4,2.835844E-4,-1.1117385E-3,2.2176453E-3,-4.151123E-4,-1.5468468E-4,-1.8503648E-3,-4.8530416E-4,2.3821113E-3],"split_indices":[4,47,57,32,53,0,32,32,32,0,0,0,0,33,53,0,28,113,0,9,2,0,0,3,0,0,0,0,0,0,58,27,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.92E2,1.4E1,1.8E2,1.2E1,2E0,1.2E1,1.01E2,7.9E1,1E1,2E0,1E1,2E0,7.2E1,2.9E1,1.3E1,6.6E1,6.9E1,3E0,7E0,2.2E1,6E0,7E0,5.5E1,1.1E1,6.6E1,3E0,3E0,4E0,7E0,1.5E1,3.2E1,2.3E1,3.1E1,3.5E1,3E0,1.2E1,2.1E1,1.1E1,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.613512E-4,9.168064E-3,-2.0914895E-2,2.484408E-3,9.1796964E-2,-1.50884595E-2,-5.1584444E-3,-3.610229E-2,6.524373E-3,2.7067963E-2,8.863301E-3,-3.4663703E-2,1.7349996E-2,-6.1991275E-4,-4.9615046E-3,2.1708814E-2,-6.561107E-3,2.5338226E-3,-1.149334E-3,-2.8247127E-2,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,3.0649362E-2,2.0936963E-3,-1.2641752E-2,-4.455137E-2,1.7206483E-3,3.736369E-3,4.129001E-2,-4.1820602E-3,-8.335901E-3,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,23,25,-1,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.960969E-2,7.377521E-2,3.0622425E-2,1.9721659E-2,5.3537726E-2,3.9970838E-2,0E0,1.7162973E-2,2.2793744E-2,1.0098371E-2,0E0,1.4024261E-2,1.5488645E-2,0E0,0E0,1.6034028E-2,1.923645E-2,0E0,0E0,1.8154921E-2,0E0,0E0,0E0,0E0,1.1873342E-2,0E0,1.6956246E-2,9.147577E-3,1.221883E-2,1.0724097E-2,1.0640074E-2,0E0,1.465915E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,19,19,24,24,26,26,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,24,26,-1,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7864855E11,1.3173E4,1.4146589E8,7.5999E4,5.4E9,1.0807868E7,-5.1584444E-3,5.6978E4,8.3333336E-2,7.438459E6,8.863301E-3,1.226189E0,3.093317E5,-6.1991275E-4,-4.9615046E-3,2.7E1,7.720203E4,2.5338226E-3,-1.149334E-3,8.305745E-1,-4.975992E-3,1.9929366E-4,3.255766E-3,-5.544163E-4,5.2682075E5,2.0936963E-3,9.484755E4,7.4074075E-2,3.7791205E6,3.63E2,7.501344E0,-4.1820602E-3,3.6360288E8,-1.569784E-3,-4.1280156E-3,1.5632148E-3,-1.2212717E-3,1.0212373E-3,-1.962127E-3,2.4679373E-3,1.5037836E-4,-1.5889367E-3,1.343493E-4],"split_indices":[31,2,45,1,5,1,0,1,57,1,0,39,33,0,0,3,28,0,0,57,0,0,0,0,32,0,28,57,47,0,53,0,5,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.33E2,6.4E1,1.24E2,9E0,6.1E1,3E0,1.1E1,1.13E2,6E0,3E0,3.8E1,2.3E1,9E0,2E0,5.2E1,6.1E1,4E0,2E0,3.6E1,2E0,1.9E1,4E0,1.1E1,4.1E1,6E0,5.5E1,2.3E1,1.3E1,1.2E1,2.9E1,2E0,5.3E1,1.9E1,4E0,6E0,7E0,9E0,3E0,2.3E1,6E0,1.6E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"3.6910433E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics index 560226124..4563466d8 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics @@ -47,63 +47,70 @@ 45,sqlOp_Subquery,0.00014946023,0.21474358974358973,0.411304017893851,0.0,0.0,0.0,0.0,1.0 46,sqlOp_WindowGroupLimit,0.00010408493,0.019230769230769232,0.13755578096385773,0.0,0.0,0.0,0.0,1.0 47,sqlOp_BroadcastNestedLoopJoin,5.774766e-05,0.038461538461538464,0.19261662052660916,0.0,0.0,0.0,0.0,1.0 -48,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_HashAggregate,0.0,0.9903846153846154,0.09774225976795364,0.0,1.0,1.0,1.0,1.0 -53,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -58,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -60,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -62,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_HashAggregate,0.0,0.9903846153846154,0.09774225976795364,0.0,1.0,1.0,1.0,1.0 +56,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +60,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_SortMergeJoin,0.0,0.5833333333333334,0.49379862765412535,0.0,0.0,1.0,1.0,1.0 +68,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 69,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_SortMergeJoin,0.0,0.5833333333333334,0.49379862765412535,0.0,0.0,1.0,1.0,1.0 -72,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -79,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -80,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -81,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -83,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -84,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -86,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +70,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +78,sqlOp_BroadcastHashJoin,0.0,0.9294871794871795,0.2564205620816261,0.0,1.0,1.0,1.0,1.0 +79,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +82,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +84,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +85,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 87,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -89,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -92,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -94,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -95,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,platform_databricks-aws,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -98,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 -101,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_BroadcastHashJoin,0.0,0.9294871794871795,0.2564205620816261,0.0,1.0,1.0,1.0,1.0 -105,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +89,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +90,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +92,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +93,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 +95,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +96,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +107,platform_databricks-aws,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +108,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg index f4badc4d1..b0ab129c4 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0479083359","gamma":"0.00511243986","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0479083359","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.00511243986","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.643989801"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-6.7159176E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0479083359","gamma":"0.00511243986","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0479083359","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.00511243986","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.643989801"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-6.7159176E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json index ca50953aa..9e19dad8c 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-5.0683957E-2,2.471375E-2,-7.1469086E-1,-4.211265E-2,4.9450392E-1,-3.8303677E-3,-9.356724E-1,-5.580517E-1,7.5147096E-3,4.142171E-2,7.9449415E-2,-2.2851696E-2,-5.449566E-2,-3.446924E-2,-1.196756E-2,4.0713497E-2,-2.8469712E-1,8.922781E-3,-1.9445838E-3,8.5911125E-2,-1.595332E-1,-2.2747863E-2,2.2966096E-3,-1.0979459E-1,1.2404155E-1,9.280069E-3,-2.6841623E-1,-1.1400769E-2,-8.741733E-4,1.1182567E-2,5.644807E-4,-1.9087E-2,-2.6889679E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[9.329678E0,5.2950816E0,2.5902376E0,3.793688E0,3.1571488E0,0E0,7.5346947E-1,4.6716928E-1,1.3309375E0,0E0,1.4857836E-1,0E0,0E0,0E0,0E0,1.1269388E0,9.081447E-1,0E0,0E0,7.673156E-1,9.366103E-1,0E0,0E0,1.9513953E-1,1.0409017E0,0E0,4.7435796E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,16,16,19,19,20,20,23,23,24,24,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,3.973E3,8.640031E9,3.9134244E-5,6.482745E7,-3.8303677E-3,2.6563678E5,1.097E3,9.4636065E-1,4.142171E-2,5.9220157E0,-2.2851696E-2,-5.449566E-2,-3.446924E-2,-1.196756E-2,2.8994E4,2.579497E8,8.922781E-3,-1.9445838E-3,9.189396E4,1.9024893E8,-2.2747863E-2,2.2966096E-3,1.4693E4,3.870239E5,9.280069E-3,1.594E3,-1.1400769E-2,-8.741733E-4,1.1182567E-2,5.644807E-4,-1.9087E-2,-2.6889679E-3],"split_indices":[45,2,5,42,45,0,33,2,27,0,56,0,0,0,0,9,7,0,0,32,7,0,0,9,33,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.85E2,1.67E2,1.8E1,1.47E2,2E1,5E0,1.3E1,1.2E1,1.35E2,1E1,1E1,5E0,8E0,7E0,5E0,1.22E2,1.3E1,5E0,5E0,1E2,2.2E1,8E0,5E0,1.6E1,8.4E1,5E0,1.7E1,6E0,1E1,4.2E1,4.2E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.230745E-2,1.0414932E-1,-5.010794E-1,3.4005634E-2,5.6172585E-1,-7.62362E-1,-7.547928E-2,-2.565942E-2,7.117637E-2,8.9466405E-1,1.4745463E-1,-9.130832E-1,-1.7198103E-2,-1.1619297E-2,4.9898042E-3,1.05104685E-1,-2.9100224E-1,5.056605E-2,2.6730007E-2,1.7217647E-2,-4.2664204E-3,-2.6734984E-2,-5.3462896E-2,-1.6213676E-1,1.3728411E-1,-2.3622975E-2,-3.2273104E-4,-1.42911635E-2,3.312105E-3,2.4434097E-1,2.3952302E-2,6.6767866E-3,1.8552445E-2,4.2158333E-3,-1.1787953E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[8.233663E0,5.3088512E0,2.8337073E0,3.1081257E0,2.9024653E0,6.4499474E-1,3.5534912E-1,0E0,1.7140567E0,6.0894012E-2,5.833674E-1,1.6950321E-1,0E0,0E0,0E0,1.1058182E0,6.860275E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6149853E-1,1.3651705E0,0E0,0E0,0E0,0E0,8.16133E-1,9.9003786E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,15,15,16,16,23,23,24,24,29,29,30,30],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[9.581563E11,3.973E3,2.1564245E0,8.960506E3,5.939257E7,3.7E1,8.0347336E4,-2.565942E-2,6.1061732E7,7.99E2,1.5E1,1E0,-1.7198103E-2,-1.1619297E-2,4.9898042E-3,5.94E2,1.2552425E6,5.056605E-2,2.6730007E-2,1.7217647E-2,-4.2664204E-3,-2.6734984E-2,-5.3462896E-2,1.6278E4,6.96E2,-2.3622975E-2,-3.2273104E-4,-1.42911635E-2,3.312105E-3,1.135E3,4.662977E5,6.6767866E-3,1.8552445E-2,4.2158333E-3,-1.1787953E-2],"split_indices":[31,2,54,33,45,8,33,0,45,0,3,75,0,0,0,2,32,0,0,0,0,0,0,9,0,0,0,0,0,2,33,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.65E2,2.5E1,1.44E2,2.1E1,1.5E1,1E1,8E0,1.36E2,1.1E1,1E1,1E1,5E0,5E0,5E0,1.25E2,1.1E1,6E0,5E0,5E0,5E0,5E0,5E0,1.3E1,1.12E2,6E0,5E0,8E0,5E0,5.7E1,5.5E1,3.4E1,2.3E1,4.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.0544179E-2,8.875575E-2,-7.398848E-1,-2.3204708E-3,6.6565776E-1,-4.3486863E-2,-1.613852E-2,7.2073705E-2,-2.4177684E-1,9.44416E-1,2.3850864E-1,-1.7552214E-2,1.0331505E-1,-3.366067E-1,4.113241E-3,2.5517184E-2,1.0831732E0,1.7368812E-2,2.5894907E-3,1.5693146E-1,-5.5736255E-2,-1.2722747E-1,-5.3803426E-1,3.2756265E-2,6.2380943E-2,1.0606914E-1,2.839202E-1,-1.8187533E-1,2.0160884E-1,-1.4508807E-2,2.9955446E-3,-1.3706747E-2,-3.3640802E-2,7.962405E-3,4.121306E-4,1.7426033E-2,-2.048489E-3,8.405352E-3,-1.565819E-2,1.4861025E-2,2.8466727E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,-1,23,-1,-1,25,27,29,31,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.57342E0,1.0404872E1,7.0878506E-1,3.081461E0,2.9896736E0,0E0,0E0,1.8459342E0,1.3027465E0,3.3729744E-1,2.5495237E-1,0E0,1.0661892E0,1.2811387E0,0E0,0E0,7.1621895E-2,0E0,0E0,5.705662E-1,1.0754284E0,5.854467E-1,4.6079874E-1,0E0,0E0,3.9710695E-1,6.934881E-1,1.1968935E0,1.5140972E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,12,12,13,13,16,16,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,-1,24,-1,-1,26,28,30,32,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,3.973E3,1E0,3.0396858E7,5.939257E7,-4.3486863E-2,-1.613852E-2,4.26E2,2.192884E0,5.666E3,2.0784883E0,-1.7552214E-2,1.9439217E0,4.3951338E11,4.113241E-3,2.5517184E-2,1.4781E4,1.7368812E-2,2.5894907E-3,3.0954462E2,4.6253732E2,1.895E3,2.5542571E8,3.2756265E-2,6.2380943E-2,8.182E3,1.898E3,3.58E2,1.7476E4,-1.4508807E-2,2.9955446E-3,-1.3706747E-2,-3.3640802E-2,7.962405E-3,4.121306E-4,1.7426033E-2,-2.048489E-3,8.405352E-3,-1.565819E-2,1.4861025E-2,2.8466727E-3],"split_indices":[45,2,104,45,45,0,0,2,54,2,54,0,56,31,0,0,9,0,0,4,4,2,7,0,0,10,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.97E2,1.4E1,1.71E2,2.6E1,9E0,5E0,1.31E2,4E1,1.5E1,1.1E1,8E0,1.23E2,3.1E1,9E0,5E0,1E1,6E0,5E0,9.2E1,3.1E1,1.6E1,1.5E1,5E0,5E0,6.7E1,2.5E1,2.1E1,1E1,8E0,8E0,7E0,8E0,4.1E1,2.6E1,2E1,5E0,6E0,1.5E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[7.14909E-5,-7.367273E-2,4.4007626E-1,-2.633183E-2,-7.943035E-1,8.512033E-1,7.56142E-2,5.1956424E-3,-2.5302224E-2,-1.8350258E-2,-5.1414963E-2,2.4242725E-2,5.015215E-2,1.9896E-2,-1.4741911E-1,7.1045555E-2,-1.7002335E-1,2.1733444E-3,-1.5121453E-2,1.4037002E-2,3.1658083E-1,4.5745276E-2,-2.7224022E-1,1.0740793E-1,-1.4456476E-1,2.524374E-2,2.047722E-1,-4.3178946E-3,7.010996E-3,-4.034405E-1,-6.963675E-2,-4.909596E-4,1.1806517E-2,1.3242443E-3,-1.1106085E-2,3.3911385E-3,1.2419809E-2,-1.479085E-2,-2.4345249E-2,5.763079E-3,-1.4912496E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,-1,-1,17,19,21,-1,-1,23,25,27,29,31,33,-1,35,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.781565E0,6.0857244E0,4.430844E0,2.6764405E0,8.5064554E-1,4.156742E-1,1.3491372E0,1.8581842E0,0E0,0E0,0E0,0E0,0E0,0E0,3.710366E-1,1.6332766E0,9.8288226E-1,0E0,0E0,1.4371717E0,4.0948606E-1,2.1821575E-1,7.693546E-1,1.0003574E0,5.47336E-1,0E0,9.842092E-2,0E0,0E0,2.1639347E-2,6.340637E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,-1,-1,18,20,22,-1,-1,24,26,28,30,32,34,-1,36,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.1855755E8,6.482745E7,4.476758E6,8.54374E5,1.4753E4,5.287008E5,2.67302E5,-2.5302224E-2,-1.8350258E-2,-5.1414963E-2,2.4242725E-2,5.015215E-2,1.9896E-2,5.9220157E0,4.6253732E2,3.0820766E-1,2.1733444E-3,-1.5121453E-2,5.657E4,1.0932244E9,2.697E3,1.6325E4,9.06E2,1.7111913E0,2.524374E-2,2.8352614E9,-4.3178946E-3,7.010996E-3,2.3005404E0,5.2726665E0,-4.909596E-4,1.1806517E-2,1.3242443E-3,-1.1106085E-2,3.3911385E-3,1.2419809E-2,-1.479085E-2,-2.4345249E-2,5.763079E-3,-1.4912496E-2],"split_indices":[2,45,45,28,29,9,32,29,0,0,0,0,0,0,56,4,27,0,0,29,5,2,9,2,54,0,5,0,0,56,56,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.78E2,2.9E1,1.68E2,1E1,1.3E1,1.6E1,1.59E2,9E0,5E0,5E0,6E0,7E0,6E0,1E1,1.16E2,4.3E1,5E0,5E0,9.5E1,2.1E1,1.4E1,2.9E1,6E1,3.5E1,6E0,1.5E1,6E0,8E0,1.7E1,1.2E1,3.3E1,2.7E1,1.2E1,2.3E1,5E0,1E1,1.1E1,6E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.3105734E-2,3.3917617E-2,-5.378899E-1,-1.463935E-2,4.987197E-1,-3.7297346E-3,-3.7451982E-2,4.624775E-2,-2.743497E-1,3.3244822E-2,8.047241E-3,-1.2655921E-1,1.00221716E-1,-3.6578873E-1,4.8535562E-4,-2.0073299E-1,4.8762416E-3,1.662886E-1,-4.8461746E-2,-2.7220068E-2,-1.9299005E-1,-2.48382E-1,-2.4237988E-5,3.215023E-1,8.2251705E-2,-2.4296811E-1,1.17883705E-1,-3.5932206E-3,-1.7125878E-2,-1.5706174E-2,-6.1416863E-3,8.9598E-3,1.953007E-2,-1.999043E-3,6.7944364E-3,2.1772254E-3,-1.8118775E-2,1.087723E-2,-1.3716562E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,19,-1,21,-1,23,25,-1,27,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1017284E0,4.2954826E0,1.8462272E0,2.748339E0,1.0455918E0,0E0,0E0,1.327802E0,8.6212254E-1,0E0,0E0,5.962887E-1,1.0689484E0,7.6462793E-1,0E0,2.4793053E-1,0E0,9.518316E-1,1.1287204E0,0E0,2.6429218E-1,1.5851068E-1,0E0,2.2499752E-1,3.7518588E-1,6.4121586E-1,3.1037417E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,20,-1,22,-1,24,26,-1,28,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,3.973E3,1.4689625E5,3.9712732E7,2.0118E4,-3.7297346E-3,-3.7451982E-2,7.38E2,9.6541266E2,3.3244822E-2,8.047241E-3,1.8149E4,2.126E4,1.1275881E5,4.8535562E-4,1.340973E-1,4.8762416E-3,5.5043226E8,1.7311496E0,-2.7220068E-2,3.2433453E5,1E0,-2.4237988E-5,1.131E3,1.9836358E5,1.4141012E0,4.7578287E-1,-3.5932206E-3,-1.7125878E-2,-1.5706174E-2,-6.1416863E-3,8.9598E-3,1.953007E-2,-1.999043E-3,6.7944364E-3,2.1772254E-3,-1.8118775E-2,1.087723E-2,-1.3716562E-3],"split_indices":[45,2,33,45,10,0,0,2,4,0,0,9,9,33,0,38,0,5,53,0,32,75,0,2,32,56,42,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.89E2,1.6E1,1.72E2,1.7E1,6E0,1E1,1.4E2,3.2E1,1E1,7E0,3.3E1,1.07E2,2.4E1,8E0,2.5E1,8E0,7.4E1,3.3E1,1E1,1.4E1,2E1,5E0,2.5E1,4.9E1,1.5E1,1.8E1,9E0,5E0,1.1E1,9E0,1.1E1,1.4E1,1.6E1,3.3E1,5E0,1E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.658149E-3,-8.5669294E-2,4.5224458E-1,5.4807942E-3,-5.227501E-1,8.260889E-1,9.481383E-2,2.4768563E-2,-2.0572197E-2,-6.902972E-1,-6.4858706E-3,2.0863598E-2,9.6404386E-1,-9.3566604E-2,1.9364879E-2,-1.3627768E-4,3.0552644E-1,-1.2061233E-2,-8.6026067E-1,2.7710496E-2,5.6963354E-2,3.4810742E-3,-1.3026494E-2,6.317388E-2,-1.2233644E-1,2.2833927E-2,4.001056E-3,-2.5003403E-2,-5.024592E-2,-7.2932154E-2,1.4571656E-1,-1.696894E-2,-3.1789118E-1,-6.5188236E-3,2.0904532E-3,1.6744744E-2,3.953099E-3,-6.344606E-3,5.567003E-3,-2.1772388E-2,-1.7867782E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,19,21,-1,23,25,-1,27,-1,-1,-1,-1,29,31,-1,-1,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9809074E0,6.6959777E0,4.331249E0,1.1849592E0,1.7887225E0,4.423294E-1,1.086925E0,9.439926E-1,0E0,1.2741423E0,0E0,0E0,2.6656723E-1,3.754934E-1,0E0,9.747925E-1,3.780203E-1,0E0,3.4571743E-1,0E0,0E0,0E0,0E0,9.430956E-1,8.806256E-1,0E0,0E0,0E0,0E0,2.4144478E-1,6.409569E-1,4.613E-1,5.5785644E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,12,12,13,13,15,15,16,16,18,18,23,23,24,24,29,29,30,30,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,20,22,-1,24,26,-1,28,-1,-1,-1,-1,30,32,-1,-1,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.013E3,7.6499896E9,5.6044944E7,4.476758E6,1.9191603E12,5.818E3,1.0083192E6,5.351875E2,-2.0572197E-2,8.0900437E-1,-6.4858706E-3,2.0863598E-2,1.4781E4,2.1030048E10,1.9364879E-2,2.1740167E0,2.674421E9,-1.2061233E-2,6.970116E7,2.7710496E-2,5.6963354E-2,3.4810742E-3,-1.3026494E-2,8.92E2,1.9577E4,2.2833927E-2,4.001056E-3,-2.5003403E-2,-5.024592E-2,1.8837452E0,3.08E2,9E0,1.2153666E6,-6.5188236E-3,2.0904532E-3,1.6744744E-2,3.953099E-3,-6.344606E-3,5.567003E-3,-2.1772388E-2,-1.7867782E-3],"split_indices":[2,5,45,28,31,2,28,52,0,27,0,0,9,5,0,56,5,0,45,0,0,0,0,2,9,0,0,0,0,53,0,3,47,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.67E2,3.2E1,1.39E2,2.8E1,1.5E1,1.7E1,1.34E2,5E0,1.9E1,9E0,5E0,1E1,1.1E1,6E0,1.24E2,1E1,6E0,1.3E1,5E0,5E0,6E0,5E0,8.2E1,4.2E1,5E0,5E0,6E0,7E0,3.1E1,5.1E1,2.8E1,1.4E1,2E1,1.1E1,1.1E1,4E1,1.5E1,1.3E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.6412618E-4,2.9746512E-2,-3.177863E-2,7.566148E-2,-4.213025E-1,1.741797E-2,4.8767343E-1,2.3707347E-3,-5.9297544E-1,2.453932E-1,-2.9302647E-2,3.3726763E-2,1.876982E-1,-1.4653362E-2,-4.0070232E-2,3.1074001E-3,2.916809E-1,3.335334E-2,-1.995446E-1,1.6215483E-2,2.7041623E-4,1.750941E-2,8.846719E-3,-2.6673706E-2,2.3808305E-1,-3.5163558E-1,-9.167898E-2,1.1175718E-3,-1.1573046E-2,1.9991705E-2,6.4005055E-3,-2.3686102E-2,-5.645601E-3,1.4264445E-3,-1.2099052E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,-1,19,-1,-1,-1,21,23,25,-1,-1,-1,-1,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1404233E0,4.256696E0,0E0,4.454952E0,1.5649507E0,1.7539783E0,1.3602915E0,0E0,7.226176E-1,2.1506047E-1,1.4679903E0,0E0,3.0002162E-1,0E0,0E0,0E0,1.0626316E-1,1.2475717E0,5.7476175E-1,0E0,0E0,0E0,0E0,8.544785E-1,3.7491035E-1,4.4241655E-1,4.5688963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,16,16,17,17,18,18,23,23,24,24,25,25,26,26],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,-1,20,-1,-1,-1,22,24,26,-1,-1,-1,-1,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,1.0388931E8,-3.177863E-2,4.774E3,5.29E2,2.0304577E-1,5.939257E7,2.3707347E-3,8.142301E-1,1.061E3,2.47538E5,3.3726763E-2,1E1,-1.4653362E-2,-4.0070232E-2,3.1074001E-3,2.65457E5,3.85024E2,3.0098013E2,1.6215483E-2,2.7041623E-4,1.750941E-2,8.846719E-3,2.054945E0,1.0932244E9,5.1412523E-2,5.5318156E7,1.1175718E-3,-1.1573046E-2,1.9991705E-2,6.4005055E-3,-2.3686102E-2,-5.645601E-3,1.4264445E-3,-1.2099052E-2],"split_indices":[28,45,0,2,0,27,45,0,27,2,29,0,3,0,0,0,1,4,52,0,0,0,0,56,5,39,45,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.03E2,8E0,1.85E2,1.8E1,1.63E2,2.2E1,5E0,1.3E1,2.7E1,1.36E2,1.2E1,1E1,7E0,6E0,6E0,2.1E1,1E2,3.6E1,5E0,5E0,1.1E1,1E1,7.8E1,2.2E1,1.4E1,2.2E1,6.4E1,1.4E1,7E0,1.5E1,8E0,6E0,1.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.7494125E-2,4.754708E-2,-3.5621274E-2,-1.0532961E-2,3.9438778E-1,7.360169E-2,-1.6759554E-1,6.925935E-1,2.5410628E-2,-2.1199968E-2,2.4291195E-1,-3.7742463E-1,-5.6034412E-2,4.7572453E-2,2.2314666E-2,-1.0107308E-2,1.398597E-2,-1.7981492E-1,6.339572E-2,3.3235803E-1,2.4692846E-2,-2.5587E-2,-7.2673513E-3,6.745743E-2,-3.2720587E-1,-1.7592138E-2,-2.9760548E-1,1.1502087E-2,-3.9779657E-4,3.899855E-1,2.2255937E-3,-6.638505E-3,8.835493E-3,-3.4677964E-3,1.3400528E-1,-6.2902095E-3,-2.557748E-2,-8.584453E-3,5.9132846E-3,-2.0052103E-2,-4.7711194E-3,-5.654876E-3,7.731384E-3,2.1913595E-2,1.1970141E-2,1.0740623E-2,2.6444825E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,-1,33,35,37,39,-1,41,43,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.815395E0,4.0733476E0,0E0,2.311642E0,3.1733027E0,1.8349516E0,1.4067862E0,6.2763596E-1,9.436025E-1,1.0043421E0,7.920656E-1,6.615987E-1,1.3950585E0,0E0,0E0,0E0,0E0,4.9158067E-1,5.524063E-1,4.596827E-1,3.6456668E-1,0E0,0E0,2.7957803E-1,4.5627367E-1,2.9557294E-1,3.1755733E-1,0E0,7.2319484E-1,1.1246085E-1,0E0,0E0,0E0,0E0,1.2976941E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,34,34],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,-1,34,36,38,40,-1,42,44,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,5.246E3,-3.5621274E-2,2.01351E5,6.482745E7,3.1546597E2,2.868989E2,1E0,8.039736E5,2.6727284E5,1.7476E4,2.7573213E8,9.149094E-1,4.7572453E-2,2.2314666E-2,-1.0107308E-2,1.398597E-2,1.8117242E8,4.898012E-1,3.706683E5,1.7775084E0,-2.5587E-2,-7.2673513E-3,1.7356589E0,1.4058E4,1.7440945E0,3.6327022E5,1.1502087E-2,1.4886476E0,1.5743027E7,2.2255937E-3,-6.638505E-3,8.835493E-3,-3.4677964E-3,1.7393E4,-6.2902095E-3,-2.557748E-2,-8.584453E-3,5.9132846E-3,-2.0052103E-2,-4.7711194E-3,-5.654876E-3,7.731384E-3,2.1913595E-2,1.1970141E-2,1.0740623E-2,2.6444825E-3],"split_indices":[45,2,0,29,45,4,52,75,28,33,9,7,27,0,0,0,0,7,27,33,53,0,0,53,10,54,32,0,53,45,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,2.01E2,7E0,1.73E2,2.8E1,1.13E2,6E1,1.5E1,1.3E1,7.3E1,4E1,2E1,4E1,5E0,1E1,7E0,6E0,2.5E1,4.8E1,2.8E1,1.2E1,1.1E1,9E0,2.8E1,1.2E1,1.1E1,1.4E1,1.2E1,3.6E1,2.3E1,5E0,6E0,6E0,9E0,1.9E1,7E0,5E0,5E0,6E0,8E0,6E0,2.1E1,1.5E1,1.4E1,9E0,8E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-3.846569E-3,4.2675298E-2,-6.763285E-1,1.9258281E-2,2.7171256E-2,-1.2538595E-2,-4.3248933E-2,1.0223437E-1,-1.0777823E-1,6.844068E-2,2.5130264E-2,-2.4064772E-2,-6.430109E-2,-8.152947E-4,1.6132517E-1,-1.752312E-1,6.6595204E-2,1.3770358E-1,-9.064334E-2,1.7832901E-2,1.0411781E-1,-2.746409E-1,-3.380403E-2,1.6328615E-1,-1.38092525E-2,6.875668E-5,1.1255376E-2,-1.1896266E-2,-6.8922905E-4,1.3330948E-2,2.1283384E-3,-1.746501E-2,-7.6540206E-3,4.820363E-3,-7.1638688E-3,-2.2508674E-3,1.1122382E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.285885E0,2.2980287E0,9.8408794E-1,1.9189385E0,0E0,0E0,0E0,1.533972E0,1.2027209E0,6.6271067E-1,0E0,0E0,9.7001195E-1,7.590547E-1,4.9211693E-1,4.9684834E-1,1.1106657E0,3.1765464E-1,4.4129378E-1,0E0,3.61857E-1,1.5865767E-1,2.6394644E-1,3.729565E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,9.772E3,8.950035E-1,2.00472E5,2.7171256E-2,-1.2538595E-2,-4.3248933E-2,2.297E3,6E0,1.059E3,2.5130264E-2,-2.4064772E-2,7.5614764E2,1.9024893E8,2.73E2,1.2137E4,9.799826E7,7.19E2,1.9606045E5,1.7832901E-2,1.3176E4,1.7142857E0,5.731563E2,1.3501E4,-1.38092525E-2,6.875668E-5,1.1255376E-2,-1.1896266E-2,-6.8922905E-4,1.3330948E-2,2.1283384E-3,-1.746501E-2,-7.6540206E-3,4.820363E-3,-7.1638688E-3,-2.2508674E-3,1.1122382E-2],"split_indices":[45,2,27,29,0,0,0,2,3,2,0,0,4,7,0,10,45,11,47,0,9,53,4,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.87E2,1.2E1,1.8E2,7E0,5E0,7E0,1.09E2,7.1E1,1.02E2,7E0,6E0,6.5E1,5.9E1,4.3E1,3.5E1,3E1,2.3E1,3.6E1,8E0,3.5E1,2E1,1.5E1,2.4E1,6E0,1E1,1.3E1,1.1E1,2.5E1,8E0,2.7E1,1E1,1E1,7E0,8E0,6E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.2346321E-2,7.2979055E-2,-3.959474E-1,1.723836E-2,4.9007657E-1,-5.9907186E-1,2.637014E-2,-2.2670643E-1,5.399271E-2,3.3464E-2,9.775284E-3,-7.985784E-1,-3.0883035E-1,-1.146209E-2,1.5899139E-2,-5.1012635E-3,-1.6054185E-2,9.682388E-2,-1.289906E-1,-4.915117E-2,-2.5306784E-2,-3.6793328E-3,-2.3445839E-2,-9.5871175E-3,1.2576337E-1,8.648533E-3,-2.488495E-1,1.6949372E-1,-4.426913E-2,-1.7645787E-2,-3.2769414E-3,3.0645367E-3,1.4927365E-2,-9.206193E-3,6.7746416E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,-1,27,-1,29,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[6.116844E0,3.5486958E0,2.9324207E0,1.2316209E0,9.3057585E-1,1.0851917E0,1.05315E0,1.9889551E-1,9.451388E-1,0E0,0E0,2.5712872E-1,4.2326367E-1,0E0,0E0,0E0,0E0,8.5870993E-1,8.981746E-1,0E0,0E0,0E0,0E0,0E0,6.692697E-1,0E0,3.4131098E-1,1.0446532E0,5.48923E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,17,17,18,18,24,24,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,-1,28,-1,30,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3365464E7,3.973E3,1.99607E3,5.7685807E-5,1.285E3,1.9006333E0,4.11591E5,2.5927516E11,2.5965E5,3.3464E-2,9.775284E-3,7.567149E4,8E0,-1.146209E-2,1.5899139E-2,-5.1012635E-3,-1.6054185E-2,4.75E2,1.8866123E8,-4.915117E-2,-2.5306784E-2,-3.6793328E-3,-2.3445839E-2,-9.5871175E-3,2.72126E5,8.648533E-3,9.120157E6,1.137E3,1.9476662E0,-1.7645787E-2,-3.2769414E-3,3.0645367E-3,1.4927365E-2,-9.206193E-3,6.7746416E-3],"split_indices":[45,2,4,38,0,53,28,31,9,0,0,32,3,0,0,0,0,2,7,0,0,0,0,0,29,0,50,2,54,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.85E2,1.52E2,3.3E1,1.35E2,1.7E1,2.2E1,1.1E1,1.7E1,1.18E2,9E0,8E0,1.2E1,1E1,6E0,5E0,9E0,8E0,9.6E1,2.2E1,5E0,7E0,5E0,5E0,8E0,8.8E1,6E0,1.6E1,7E1,1.8E1,9E0,7E0,4.1E1,2.9E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.9049242E-2,-4.8035493E-1,5.9438504E-2,-2.916009E-2,-8.932367E-3,2.7623249E-3,3.4090203E-1,-2.8239418E-2,3.0565694E-2,3.88462E-2,1.4218977E-1,5.048788E-2,-2.168624E-2,1.7575439E-2,-2.8442191E-2,1.1093366E-1,-4.3717675E-2,7.2735664E-3,-6.968673E-3,2.4444629E-2,2.189726E-1,-1.1726413E-1,8.794597E-2,-4.637495E-3,4.5833588E-3,1.347305E-2,4.942358E-4,7.290213E-3,-9.430183E-3,9.948807E-3,-3.0905432E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,-1,-1,7,9,-1,11,-1,13,15,-1,-1,17,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0424943E0,4.52183E-1,2.95023E0,0E0,0E0,2.5701554E0,2.7795877E0,0E0,1.4572673E0,0E0,8.921429E-1,8.24833E-1,0E0,0E0,3.173986E-1,8.169353E-1,5.6226635E-1,0E0,0E0,4.410391E-1,5.036887E-1,8.2323015E-1,3.966793E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,10,10,11,11,14,14,15,15,16,16,19,19,20,20,21,21,22,22],"right_children":[2,4,6,-1,-1,8,10,-1,12,-1,14,16,-1,-1,18,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.171708E-5,1.119E3,4.376E3,-2.916009E-2,-8.932367E-3,6E0,4.083812E7,-2.8239418E-2,1.443266E6,3.88462E-2,5.287008E5,1.33358E5,-2.168624E-2,1.7575439E-2,1.14416916E2,2.3103847E2,2.1296928E0,7.2735664E-3,-6.968673E-3,2.6727284E5,1.7315E4,1.4163916E0,5.509307E5,-4.637495E-3,4.5833588E-3,1.347305E-2,4.942358E-4,7.290213E-3,-9.430183E-3,9.948807E-3,-3.0905432E-3],"split_indices":[42,0,2,0,0,3,45,0,29,0,32,29,0,0,58,55,54,0,0,33,9,53,32,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.4E1,1.84E2,9E0,5E0,1.54E2,3E1,6E0,1.48E2,8E0,2.2E1,1.43E2,5E0,9E0,1.3E1,8.7E1,5.6E1,5E0,8E0,4.9E1,3.8E1,3.6E1,2E1,1.8E1,3.1E1,2.9E1,9E0,8E0,2.8E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.7406095E-2,-3.1569876E-2,3.435455E-1,1.0490399E-2,-5.0699943E-1,-8.800076E-2,5.437029E-1,4.9816135E-2,-1.6751517E-1,-9.425468E-3,-4.2537298E-2,4.3461984E-3,-1.2075476E-2,4.026478E-2,3.4918848E-1,2.0733293E-2,2.9138464E-1,-3.0736335E-2,-2.3788634E-1,2.2252733E-2,1.0574049E-2,4.6955746E-2,-2.3860836E-1,1.8803617E-2,9.179426E-3,4.9352003E-3,-7.6348325E-3,-1.7087992E-2,-3.1744582E-3,-1.0004239E-2,6.950604E-2,-5.8404566E-3,-1.511698E-2,6.420985E-3,-1.4672048E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,19,21,23,25,27,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1],"loss_changes":[3.7957053E0,3.4447508E0,2.8119166E0,1.1213769E0,1.479764E0,3.453808E-1,9.76418E-1,9.14299E-1,2.7182156E-1,0E0,0E0,0E0,0E0,0E0,1.196723E-1,8.1367177E-1,6.6036224E-2,2.0565866E-1,3.6374104E-1,0E0,0E0,6.3743985E-1,6.0289085E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.501559E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,14,14,15,15,16,16,17,17,18,18,21,21,22,22,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,20,22,24,26,28,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1],"split_conditions":[3.847E3,9.1131825E11,1.2047E4,3.00037E5,4.699E3,1.0388931E8,1E0,2.001E3,2.7234016E5,-9.425468E-3,-4.2537298E-2,4.3461984E-3,-1.2075476E-2,4.026478E-2,7.64E2,2.691667E7,6.505172E5,2.5927516E11,2.4229807E-1,2.2252733E-2,1.0574049E-2,4.75E2,2.0946825E6,1.8803617E-2,9.179426E-3,4.9352003E-3,-7.6348325E-3,-1.7087992E-2,-3.1744582E-3,-1.0004239E-2,7.134E3,-5.8404566E-3,-1.511698E-2,6.420985E-3,-1.4672048E-3],"split_indices":[2,31,9,29,10,45,64,2,32,0,0,0,0,0,0,45,32,31,41,0,0,2,28,0,0,0,0,0,0,0,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.71E2,3.1E1,1.58E2,1.3E1,1E1,2.1E1,1.3E2,2.8E1,8E0,5E0,5E0,5E0,7E0,1.4E1,1.17E2,1.3E1,1E1,1.8E1,6E0,8E0,1.07E2,1E1,5E0,8E0,5E0,5E0,1E1,8E0,8E0,9.9E1,5E0,5E0,6E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.7527538E-3,4.5091048E-2,-3.39698E-1,-5.119068E-3,3.6392906E-1,-3.4445386E-2,-1.943968E-1,2.0665696E-2,-2.4766281E-2,5.661521E-1,1.5015559E-1,1.2902547E-2,-2.1804703E-2,6.9899805E-2,-9.814537E-2,1.3225541E-2,3.4156375E-2,1.2338285E-2,-1.7221158E-3,8.632894E-3,-7.4996375E-3,2.454633E-1,3.405132E-2,-1.6444752E-1,1.0893984E-1,1.0092336E-3,3.4741086E-1,-3.4778193E-2,1.2956136E-1,2.390233E-4,-2.211112E-1,7.153902E-4,8.333883E-3,9.820453E-3,2.1089602E-2,-3.604042E-3,1.2413836E-2,-1.4417392E-3,8.804729E-3,-1.3778535E-2,-2.6911895E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,25,27,29,31,-1,33,35,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0394206E0,3.0595617E0,1.1277821E0,2.2014642E0,1.0389671E0,0E0,9.7878325E-1,9.369399E-1,0E0,3.5679817E-1,2.890378E-1,3.4002376E-1,0E0,7.02828E-1,6.63234E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.273399E-1,6.290273E-1,3.4673566E-1,7.074499E-2,0E0,8.158159E-2,6.852175E-1,3.5180014E-1,0E0,2.7994883E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,13,13,14,14,21,21,22,22,23,23,24,24,26,26,27,27,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,26,28,30,32,-1,34,36,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,4.376E3,6.1978314E2,4.476758E6,5.939257E7,-3.4445386E-2,2.01E3,2.493493E7,-2.4766281E-2,5.818E3,1.3E1,1.673295E6,-2.1804703E-2,1.3176E4,1.0141866E3,1.3225541E-2,3.4156375E-2,1.2338285E-2,-1.7221158E-3,8.632894E-3,-7.4996375E-3,2.0260053E5,1.7206225E0,6.50596E5,5.752585E5,1.0092336E-3,5.56E2,1.993E3,1.4176E4,2.390233E-4,3.6E1,7.153902E-4,8.333883E-3,9.820453E-3,2.1089602E-2,-3.604042E-3,1.2413836E-2,-1.4417392E-3,8.804729E-3,-1.3778535E-2,-2.6911895E-3],"split_indices":[45,2,52,28,45,0,0,45,0,2,3,32,0,9,4,0,0,0,0,0,0,47,53,28,28,0,0,2,29,0,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.9E2,2.2E1,1.65E2,2.5E1,5E0,1.7E1,1.58E2,7E0,1.2E1,1.3E1,1E1,7E0,1.12E2,4.6E1,5E0,7E0,8E0,5E0,5E0,5E0,1.8E1,9.4E1,3.5E1,1.1E1,6E0,1.2E1,5.5E1,3.9E1,9E0,2.6E1,5E0,6E0,6E0,6E0,4.9E1,6E0,1E1,2.9E1,1.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-9.676875E-3,-4.702332E-2,5.031941E-1,-1.8553495E-2,-3.351236E-2,3.3279005E-2,6.3316072E-3,-3.0306152E-1,1.5836349E-2,-3.9179197E-1,-1.4719908E-3,1.1068377E-1,-4.786979E-2,-1.2056768E-2,-2.4720086E-2,7.381449E-2,2.2671139E-2,-1.3672525E-1,1.00702934E-1,-8.401744E-3,1.087176E-1,-1.7106122E-1,7.7803046E-3,-5.187583E-3,1.432713E-1,4.5634908E-4,8.1301965E-3,9.229902E-4,-1.1185455E-2,9.712743E-3,-1.4957057E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,15,17,-1,-1,19,-1,21,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[3.950213E0,3.5536487E0,9.026501E-1,1.8148739E0,0E0,0E0,0E0,4.7125244E-1,1.0083307E0,1.3119459E-1,0E0,8.606195E-1,1.3339183E0,0E0,0E0,5.5830216E-1,0E0,6.7483747E-1,3.5356632E-1,0E0,3.3109432E-1,6.8161535E-1,0E0,0E0,3.3836383E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,11,11,12,12,15,15,17,17,18,18,20,20,21,21,24,24],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,16,18,-1,-1,20,-1,22,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3397336E3,1.2754595E8,1.9160621E0,2.1103053E-5,-3.351236E-2,3.3279005E-2,6.3316072E-3,3.4572864E0,1.8949672E0,2.1816352E6,-1.4719908E-3,2.902E3,1.340973E-1,-1.2056768E-2,-2.4720086E-2,4.75E2,2.2671139E-2,1.714168E6,2.2786127E5,-8.401744E-3,3.322259E-2,1.7141459E0,7.7803046E-3,-5.187583E-3,3.7490938E5,4.5634908E-4,8.1301965E-3,9.229902E-4,-1.1185455E-2,9.712743E-3,-1.4957057E-3],"split_indices":[52,45,54,38,0,0,0,56,58,28,0,2,38,0,0,2,0,28,28,0,57,54,0,0,33,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.91E2,1.3E1,1.84E2,7E0,8E0,5E0,1.9E1,1.65E2,1.4E1,5E0,6.6E1,9.9E1,8E0,6E0,6.1E1,5E0,6.2E1,3.7E1,7E0,5.4E1,5.6E1,6E0,6E0,3.1E1,2.1E1,3.3E1,1.4E1,4.2E1,2.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[6.5133916E-3,8.09256E-2,-1.5444407E-1,1.2799028E-1,-1.23670384E-1,2.7883213E-2,-3.172873E-1,9.7439356E-2,2.9092314E-2,1.2076002E-2,-2.1631126E-1,-1.794678E-1,1.2583877E-1,-3.8334322E-1,3.6833314E-3,-2.8249985E-4,1.6851407E-1,-1.3325632E-3,-3.1498834E-1,-1.0891992E-2,-4.8710145E-3,9.707346E-3,2.5164512E-1,-1.5811665E-1,-5.846109E-1,6.682559E-2,-1.09759785E-1,4.0213963E-1,9.1915585E-2,-2.193155E-2,-8.093075E-3,-3.1746507E-3,3.136715E-3,1.6024046E-2,6.0784356E-3,1.2088699E-3,-1.1677719E-2,-3.2342143E-2,-1.5393358E-2,-7.1081676E-2,1.5766606E-1,-1.4831737E-2,6.2818667E-3,1.3849405E-2,2.5984999E-2,-6.2484745E-2,1.4102556E-1,1.6948801E-3,-8.019203E-3,9.238251E-3,2.363558E-3,-3.0282736E-3,2.760255E-3,-8.085439E-3,4.2945826E-3,1.00351125E-2,8.41192E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,21,23,-1,25,27,-1,29,-1,-1,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5756192E0,1.4313273E0,2.0278187E0,1.6978472E0,1.0291605E0,6.9285315E-1,9.668145E-1,7.918408E-1,0E0,0E0,4.1904497E-1,1.7859489E-2,3.335888E-1,1.3128185E0,0E0,3.6732855E-1,1.1441311E0,0E0,2.3551333E-1,0E0,0E0,5.9415333E-2,7.12409E-2,2.573679E-1,1.6423845E-1,3.9956155E-1,4.425193E-1,9.690857E-2,3.955243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3920456E-1,6.286797E-2,0E0,5.0016046E-2,0E0,0E0,2.2532097E-1,3.2587248E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,12,12,13,13,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,39,39,40,40,42,42,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,22,24,-1,26,28,-1,30,-1,-1,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.595345E-1,2.5086E4,1.021246E0,1.0086E4,1.9024893E8,1.2399164E5,8.853438E8,1.137E3,2.9092314E-2,1.2076002E-2,2.5979605E0,5.94E2,3.2935953E2,4.3951338E11,3.6833314E-3,5.14E2,3.6004024E10,-1.3325632E-3,5.6216806E5,-1.0891992E-2,-4.8710145E-3,1.9107901E0,1.9173932E6,9.1300315E-1,1.9378502E0,7.23E2,1.3371641E6,1.594E3,1.3345E4,-2.193155E-2,-8.093075E-3,-3.1746507E-3,3.136715E-3,1.6024046E-2,6.0784356E-3,1.2088699E-3,-1.1677719E-2,-3.2342143E-2,-1.5393358E-2,5.9E1,1.289408E7,-1.4831737E-2,3.9232688E5,1.3849405E-2,2.5984999E-2,3.3487454E11,1.0016339E6,1.6948801E-3,-8.019203E-3,9.238251E-3,2.363558E-3,-3.0282736E-3,2.760255E-3,-8.085439E-3,4.2945826E-3,1.00351125E-2,8.41192E-4],"split_indices":[27,9,56,2,7,33,7,2,0,0,56,2,4,31,0,0,31,0,28,0,0,53,47,27,53,2,51,2,9,0,0,0,0,0,0,0,0,0,0,10,45,0,28,0,0,31,32,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.46E2,6.7E1,1.19E2,2.7E1,3.2E1,3.5E1,1.13E2,6E0,5E0,2.2E1,1E1,2.2E1,3E1,5E0,4.8E1,6.5E1,8E0,1.4E1,5E0,5E0,1.2E1,1E1,1.5E1,1.5E1,3E1,1.8E1,1.5E1,5E1,6E0,8E0,5E0,7E0,5E0,5E0,5E0,1E1,1E1,5E0,1.2E1,1.8E1,6E0,1.2E1,1E1,5E0,1.2E1,3.8E1,6E0,6E0,1.3E1,5E0,5E0,7E0,7E0,5E0,2.4E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.1821186E-2,3.3352602E-2,-4.40673E-1,-1.2966474E-2,3.7364328E-1,-2.2973595E-3,-6.4692664E-1,4.9243286E-2,-2.2766404E-1,2.5005925E-1,3.0705739E-2,-1.5479926E-2,-3.7201814E-2,-1.6238444E-2,1.5635999E-1,-3.0522484E-1,1.4523114E-3,1.6258791E-2,5.6687035E-3,-1.442857E-2,8.069018E-3,3.086278E-1,1.6109893E-2,-2.3043336E-1,-2.532384E-2,-1.5009213E-2,9.183369E-3,3.7204307E-1,3.7135356E-3,8.1505515E-2,-6.1527487E-2,-3.156817E-1,-1.1648669E-1,8.841852E-3,-1.6791756E-3,9.670926E-3,2.13919E-2,7.066632E-3,-1.488763E-3,-7.6162666E-3,2.9902852E-3,-1.9948233E-2,-8.828004E-3,-1.9563122E-3,-8.274941E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,33,-1,35,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.567297E0,2.763186E0,1.831975E0,2.0814347E0,5.3340197E-1,0E0,3.7855434E-1,8.512558E-1,7.0956826E-1,1.480962E-1,0E0,0E0,0E0,5.28804E-1,9.773848E-1,3.2925105E-1,0E0,0E0,0E0,0E0,3.037439E-1,2.9359746E-1,1.3194342E-1,1.7693007E-1,0E0,2.624322E-1,0E0,1.2532306E-1,0E0,1.0860463E-1,1.548607E-1,8.2075715E-2,3.9746493E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,13,13,14,14,15,15,20,20,21,21,22,22,23,23,25,25,27,27,29,29,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,34,-1,36,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,7.7186523E2,7.328832E-1,3.9882228E7,2.4450078E3,-2.2973595E-3,2.409E3,1.222E3,1.0141866E3,2.3734754E7,3.0705739E-2,-1.5479926E-2,-3.7201814E-2,8.53386E3,7.11E2,2.6654E4,1.4523114E-3,1.6258791E-2,5.6687035E-3,-1.442857E-2,2.0487762E2,4.9321375E5,3.829552E5,2.2243262E5,-2.532384E-2,1.3541759E9,9.183369E-3,3.1274893E2,3.7135356E-3,2.447064E7,1.875E3,9.190713E-1,8.2841175E5,8.841852E-3,-1.6791756E-3,9.670926E-3,2.13919E-2,7.066632E-3,-1.488763E-3,-7.6162666E-3,2.9902852E-3,-1.9948233E-2,-8.828004E-3,-1.9563122E-3,-8.274941E-3],"split_indices":[45,52,27,45,52,0,2,2,4,48,0,0,0,33,0,10,0,0,0,0,52,32,33,33,0,31,0,4,0,45,2,27,48,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.74E2,2.2E1,1.54E2,2E1,8E0,1.4E1,1.2E2,3.4E1,1.5E1,5E0,5E0,9E0,7.5E1,4.5E1,2.6E1,8E0,8E0,7E0,5E0,7E1,2.1E1,2.4E1,2.1E1,5E0,6.3E1,7E0,1.6E1,5E0,1.3E1,1.1E1,1.1E1,1E1,5E0,5.8E1,6E0,1E1,8E0,5E0,6E0,5E0,5E0,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[3.0091282E-2,6.145293E-2,-3.1156203E-2,1.6833354E-2,3.485918E-1,2.898522E-2,-1.708431E-2,3.07013E-2,1.610519E-1,6.2482465E-2,-8.49575E-2,2.7159026E-1,-5.926881E-3,3.4729928E-2,2.6450232E-1,-1.793216E-1,6.4361565E-2,6.7008883E-3,1.5772756E-2,9.4155E-2,-3.2206904E-2,1.6666839E-2,7.268907E-3,-2.6133847E-1,2.2213887E-4,-1.2882438E-3,8.1626475E-3,7.824576E-3,-1.01181235E-4,7.5823925E-3,-4.753267E-3,-3.687387E-3,-1.5633916E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,23,25,-1,-1,27,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.841265E0,2.7615275E0,0E0,8.634732E-1,1.4861982E0,7.0377684E-1,0E0,0E0,6.3167053E-1,7.86946E-1,6.0333645E-1,6.0240507E-2,0E0,5.0422597E-1,1.0708821E-1,3.9349157E-1,1.6904579E-1,0E0,0E0,4.4647658E-1,7.815173E-1,0E0,0E0,1.9056845E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,20,20,23,23],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,24,26,-1,-1,28,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,4.796E3,-3.1156203E-2,4.674595E6,4.083812E7,3.08891E5,-1.708431E-2,3.07013E-2,1.3E1,1.875E3,2.1564245E0,4.348354E-1,-5.926881E-3,2.459517E0,5E0,2.1996392E-1,1.0908286E3,6.7008883E-3,1.5772756E-2,6.805556E5,1.3257E4,1.6666839E-2,7.268907E-3,1.861058E0,2.2213887E-4,-1.2882438E-3,8.1626475E-3,7.824576E-3,-1.01181235E-4,7.5823925E-3,-4.753267E-3,-3.687387E-3,-1.5633916E-2],"split_indices":[45,2,0,28,45,29,0,0,3,2,54,27,0,58,8,39,4,0,0,28,9,0,0,56,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E2,2.15E2,9E0,1.87E2,2.8E1,1.82E2,5E0,1E1,1.8E1,1.41E2,4.1E1,1.3E1,5E0,1.25E2,1.6E1,2.5E1,1.6E1,5E0,8E0,6.6E1,5.9E1,8E0,8E0,1.7E1,8E0,9E0,7E0,3.8E1,2.8E1,1.5E1,4.4E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.2112274E-3,4.8081707E-2,-3.1884396E-1,-2.8697753E-3,4.2340034E-1,-3.0472731E-2,-6.496023E-2,-2.3495965E-1,2.0514237E-2,3.6657996E-2,2.1289678E-1,6.562134E-3,-1.5105893E-2,6.6261686E-4,-1.728236E-2,-3.4026064E-2,1.2290722E-1,1.5764516E-2,4.7379285E-3,7.438103E-2,-1.06945686E-1,9.727862E-2,1.4524853E-2,-2.9558819E-3,1.9057114E-1,-2.6383072E-1,-1.3303149E-2,1.8685614E-1,1.0976157E-2,5.744039E-3,-4.448314E-3,1.1160188E-2,3.8861774E-3,-4.7106883E-3,-1.4694652E-2,3.3241983E-3,-8.511366E-3,3.115662E-3,1.5035602E-2,-5.076677E-3,5.2904543E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9632285E0,3.5198545E0,1.9721012E0,8.894524E-1,1.4198577E0,0E0,8.014912E-1,4.7437644E-1,8.362791E-1,0E0,1.6609162E-1,0E0,0E0,0E0,0E0,7.8264517E-1,2.1075445E-1,0E0,0E0,3.599947E-1,8.6326575E-1,3.6129475E-1,0E0,2.872266E-1,5.5318475E-2,1.2601972E-1,5.287694E-1,3.353905E-1,3.0245772E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,16,16,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,4.774E3,9.1922754E2,3.0725318E-5,1E0,-3.0472731E-2,1.74E3,6.342E3,1.9107901E0,3.6657996E-2,1.203E3,6.562134E-3,-1.5105893E-2,6.6261686E-4,-1.728236E-2,1.4170854E0,5.8018835E2,1.5764516E-2,4.7379285E-3,2.3873117E2,8E0,4.720179E10,1.4524853E-2,5.3641737E-1,1E1,1.7046307E0,2.67302E5,2.6538E4,6E0,5.744039E-3,-4.448314E-3,1.1160188E-2,3.8861774E-3,-4.7106883E-3,-1.4694652E-2,3.3241983E-3,-8.511366E-3,3.115662E-3,1.5035602E-2,-5.076677E-3,5.2904543E-3],"split_indices":[45,2,52,42,75,0,0,12,53,0,0,0,0,0,0,53,52,0,0,4,3,31,0,27,3,54,29,29,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.83E2,2.4E1,1.62E2,2.1E1,1E1,1.4E1,1.4E1,1.48E2,7E0,1.4E1,8E0,6E0,5E0,9E0,9.7E1,5.1E1,6E0,8E0,3.9E1,5.8E1,4.6E1,5E0,2.4E1,1.5E1,2.1E1,3.7E1,2.2E1,2.4E1,1E1,1.4E1,1E1,5E0,5E0,1.6E1,2.5E1,1.2E1,1.2E1,1E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.7687019E-2,4.0660083E-2,-2.9475197E-1,-5.7623894E-3,4.4846636E-1,-4.96775E-1,-7.721567E-2,-1.931599E-2,1.2458072E-2,3.215538E-2,4.1282247E-3,-1.1024078E-2,-6.6979593E-1,5.9929606E-2,-1.8413667E-2,-2.3049511E-1,3.6664404E-2,-3.9873138E-2,-1.8956346E-2,8.688145E-3,-6.3329204E-3,-3.4587167E-3,-1.4871624E-2,1.4300562E-1,2.531077E-3,6.749068E-2,2.1455054E-1,-7.602184E-2,4.6893872E-2,7.560791E-3,-2.0779386E-4,6.0982113E-3,1.4219599E-2,-8.402338E-3,1.8109465E-3,-5.0916146E-3,3.880823E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,-1,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.439163E0,3.3345063E0,1.587693E0,1.1563815E0,1.3945937E0,7.2251654E-1,8.233579E-1,0E0,9.0824485E-1,0E0,0E0,0E0,1.6061783E-1,3.5054892E-1,0E0,1.5471596E-1,5.078072E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6921198E-1,3.7644148E-1,1.1746888E-1,8.424455E-2,4.4770265E-1,3.691272E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,12,12,13,13,15,15,16,16,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,-1,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8351054E11,3.965E3,2.1564245E0,3.0725318E-5,9E0,1.0463519E5,6.3018E4,-1.931599E-2,4.75E2,3.215538E-2,4.1282247E-3,-1.1024078E-2,3.1696565E-2,5.0797403E5,-1.8413667E-2,2.542029E-3,3.58E2,-3.9873138E-2,-1.8956346E-2,8.688145E-3,-6.3329204E-3,-3.4587167E-3,-1.4871624E-2,3.322259E-2,2.6477316E-1,1.5316E4,1.3723022E0,1.8270676E0,2.5256923E5,7.560791E-3,-2.0779386E-4,6.0982113E-3,1.4219599E-2,-8.402338E-3,1.8109465E-3,-5.0916146E-3,3.880823E-3],"split_indices":[31,2,54,42,3,33,10,0,2,0,0,0,38,32,0,39,0,0,0,0,0,0,0,57,41,9,56,53,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.75E2,3.6E1,1.58E2,1.7E1,1.8E1,1.8E1,6E0,1.52E2,1E1,7E0,8E0,1E1,1.3E1,5E0,1.3E1,1.39E2,5E0,5E0,8E0,5E0,5E0,8E0,3.3E1,1.06E2,1.7E1,1.6E1,3.8E1,6.8E1,7E0,1E1,9E0,7E0,2E1,1.8E1,1.2E1,5.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.328289E-2,2.0352677E-2,-4.3172076E-1,-5.887703E-3,4.40734E-1,-3.639438E-3,-3.5141233E-2,8.525475E-2,-7.068245E-2,2.8141689E-2,1.0568837E-2,-3.6925445E-3,1.7924613E-1,-1.1450928E-1,7.239069E-2,-1.4711688E-1,7.367797E-2,2.3025775E-1,-3.1131662E-3,-7.100218E-2,-2.7156484E-1,1.5453368E-1,-3.1752295E-3,-1.1038821E-2,1.12605645E-4,1.5254448E-1,5.672894E-3,2.56348E-1,3.9846315E-3,-1.12993615E-2,-2.1180889E-1,-2.006175E-2,-6.0679745E-3,2.860321E-3,9.843298E-3,3.167208E-3,9.813512E-3,5.1503284E-3,-2.6825285E-3,6.939849E-3,1.3602696E-2,-7.888598E-3,1.4590443E-3,-1.6870264E-2,1.2052925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,-1,-1,-1,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.83774E0,2.0653856E0,1.5547178E0,1.0513088E0,2.2557116E-1,0E0,0E0,6.193224E-1,6.604153E-1,0E0,0E0,4.436151E-1,4.6346724E-1,5.273608E-1,2.9500696E-1,1.7484736E-1,1.3858105E-1,9.380424E-2,0E0,5.356264E-1,3.0954695E-1,6.500396E-2,0E0,0E0,0E0,4.0699244E-2,1.0021178E-1,3.5754085E-2,0E0,2.995745E-1,6.406658E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,25,25,26,26,27,27,29,29,30,30],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,-1,-1,-1,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855755E8,8.015E3,8.950035E-1,8.0683605E6,1.0915004E7,-3.639438E-3,-3.5141233E-2,1.059E3,9.6541266E2,2.8141689E-2,1.0568837E-2,2.5934292E5,2.3701E4,5.5318156E7,4.1124488E9,5.514485E5,1E0,4.6385403E5,-3.1131662E-3,2.1282E4,1.4693E4,1.4041E4,-3.1752295E-3,-1.1038821E-2,1.12605645E-4,1.188049E8,2.585911E5,1.945E2,3.9846315E-3,7.7E1,1.808848E0,-2.006175E-2,-6.0679745E-3,2.860321E-3,9.843298E-3,3.167208E-3,9.813512E-3,5.1503284E-3,-2.6825285E-3,6.939849E-3,1.3602696E-2,-7.888598E-3,1.4590443E-3,-1.6870264E-2,1.2052925E-3],"split_indices":[45,2,27,45,47,0,0,2,4,0,0,32,9,45,12,28,104,33,0,9,9,9,0,0,0,5,28,4,0,10,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.86E2,1.4E1,1.76E2,1E1,7E0,7E0,7.3E1,1.03E2,5E0,5E0,3.8E1,3.5E1,7.9E1,2.4E1,1.3E1,2.5E1,2.9E1,6E0,6.3E1,1.6E1,1.5E1,9E0,8E0,5E0,1.1E1,1.4E1,2.4E1,5E0,4.5E1,1.8E1,7E0,9E0,6E0,9E0,5E0,6E0,5E0,9E0,6E0,1.8E1,9E0,3.6E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.983319E-3,-3.524799E-1,2.722492E-2,-6.366662E-3,-2.4592336E-2,6.5453954E-2,-1.6954966E-1,2.4516119E-2,2.7226755E-1,7.8222625E-2,-2.85729E-1,-1.2288755E-1,4.485502E-2,-2.0336143E-3,3.6420763E-1,1.0981665E-2,-4.1112197E-3,-4.3292966E-1,-9.917107E-2,-1.6136112E-3,-1.0493579E-2,-8.683938E-2,7.309682E-2,6.7098835E-3,4.2550126E-1,-1.2543922E-2,-2.4803964E-2,-8.54645E-3,-1.6393885E-4,-1.2391428E-2,-1.6016084E-1,1.6963375E-1,1.4635887E-2,2.4129942E-2,1.36239305E-2,-3.7315574E-3,3.1661785E-3,-2.759706E-3,-1.1307532E-2,5.099239E-3,1.0886122E-2,3.2962817E-3,-3.2184385E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,-1,-1,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,-1,-1,29,31,-1,33,-1,-1,-1,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5011991E0,3.2028794E-1,1.4779855E0,0E0,0E0,1.3818312E0,9.43503E-1,4.1924015E-1,7.967043E-1,2.921355E-1,5.612178E-1,1.3843027E-1,4.6010065E-1,0E0,2.2892523E-1,0E0,0E0,3.8566113E-2,8.282796E-2,0E0,0E0,1.1810528E-1,5.6752485E-1,0E0,3.327322E-2,0E0,0E0,0E0,0E0,6.683082E-2,7.198769E-2,1.1412513E-1,2.8825465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,17,17,18,18,21,21,22,22,24,24,29,29,30,30,31,31,32,32],"right_children":[2,4,6,-1,-1,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,-1,-1,30,32,-1,34,-1,-1,-1,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0287E4,1.5947007E0,2.68225E5,-6.366662E-3,-2.4592336E-2,3.625E3,6.5229E4,1.545823E-4,1.2047E4,1.065E3,8E0,7.017544E-3,7.53E2,-2.0336143E-3,7.381306E-2,1.0981665E-2,-4.1112197E-3,4.4818586E-1,1.1184356E-1,-1.6136112E-3,-1.0493579E-2,8.629298E5,5.34E2,6.7098835E-3,3.93672E6,-1.2543922E-2,-2.4803964E-2,-8.54645E-3,-1.6393885E-4,1.6498195E0,3.3438003E5,1.061E3,9E0,2.4129942E-2,1.36239305E-2,-3.7315574E-3,3.1661785E-3,-2.759706E-3,-1.1307532E-2,5.099239E-3,1.0886122E-2,3.2962817E-3,-3.2184385E-3],"split_indices":[9,53,9,0,0,2,29,42,9,2,3,57,2,0,27,0,0,27,38,0,0,28,0,0,1,0,0,0,0,53,32,2,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1E1,1.94E2,5E0,5E0,1.63E2,3.1E1,1.37E2,2.6E1,1E1,2.1E1,1.6E1,1.21E2,6E0,2E1,5E0,5E0,1.1E1,1E1,9E0,7E0,2.1E1,1E2,5E0,1.5E1,5E0,6E0,5E0,5E0,1.1E1,1E1,3.7E1,6.3E1,8E0,7E0,6E0,5E0,5E0,5E0,1.9E1,1.8E1,3.8E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-8.615122E-3,2.2319222E-2,-3.3891685E-2,4.699542E-2,-2.4777137E-2,8.946151E-3,2.6323947E-1,5.4896243E-2,-8.313295E-2,2.745765E-2,6.546693E-2,-1.0281701E-2,7.438199E-2,8.760412E-2,-1.4548934E-1,1.0630774E-2,-3.1727192E-3,1.6878971E-1,1.8071126E-2,7.0169987E-3,-7.3257356E-4,5.3886306E-3,-1.8931165E-1,4.5853024E-3,1.0483018E-2,-8.7115844E-4,6.634993E-3,-5.4690853E-4,-1.1434086E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,-1,17,19,21,-1,-1,23,25,-1,-1,-1,27,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0798435E0,2.433911E0,0E0,1.4293046E0,0E0,6.3498914E-1,1.5559572E0,5.411203E-1,5.450728E-1,0E0,3.6490712E-1,0E0,4.9667078E-1,8.703386E-2,4.3956816E-1,0E0,0E0,1.0700929E-1,2.6447305E-1,0E0,0E0,0E0,2.7823293E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,17,17,18,18,22,22],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,-1,18,20,22,-1,-1,24,26,-1,-1,-1,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,4.7379535E6,-3.3891685E-2,4.796E3,-2.4777137E-2,2.0946133E0,1.9824325E0,9.235376E5,2.0304577E-1,2.745765E-2,5.287008E5,-1.0281701E-2,6.7033327E-1,1.8495E4,3.9267014E-3,1.0630774E-2,-3.1727192E-3,1.046E3,2.1799042E0,7.0169987E-3,-7.3257356E-4,5.3886306E-3,1.4778827E0,4.5853024E-3,1.0483018E-2,-8.7115844E-4,6.634993E-3,-5.4690853E-4,-1.1434086E-2],"split_indices":[45,28,0,2,0,56,54,45,27,0,32,0,27,9,57,0,0,2,54,0,0,0,53,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.8E2,7E0,1.73E2,7E0,1.48E2,2.5E1,9.9E1,4.9E1,9E0,1.6E1,6E0,9.3E1,1.3E1,3.6E1,7E0,9E0,3.4E1,5.9E1,8E0,5E0,5E0,3.1E1,1.5E1,1.9E1,4.6E1,1.3E1,7E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.986548E-4,-3.1845292E-1,1.7985018E-2,-3.4269767E-3,-2.4543364E-2,-1.8185236E-2,1.9708934E-1,-9.742517E-4,-1.5913421E-2,3.985395E-1,2.945647E-2,-2.4388062E-2,1.5804447E-1,2.9602781E-2,9.965601E-3,1.2605089E-1,-6.723553E-3,-1.1332913E-3,-1.4393006E-1,2.3598392E-3,2.0446487E-1,9.273716E-3,2.4034798E-3,-1.676299E-1,1.9265613E-2,-2.5626883E-1,1.6446589E-3,1.2925185E-2,5.2665914E-3,-1.1401988E-2,-2.7594191E-3,8.560774E-3,-4.2228858E-5,-1.586716E-2,-5.4448466E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,15,17,19,-1,-1,21,-1,23,25,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1748582E0,4.89864E-1,1.1955819E0,0E0,0E0,8.3187425E-1,1.0351202E0,5.511698E-1,0E0,4.5659232E-1,3.1291878E-1,3.5844678E-1,8.587745E-2,0E0,0E0,5.1773027E-2,0E0,3.7343147E-1,4.293317E-1,0E0,5.062276E-2,0E0,0E0,7.9202086E-2,3.1492835E-1,1.0128021E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,17,17,18,18,20,20,23,23,24,24,25,25],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,16,18,20,-1,-1,22,-1,24,26,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0287E4,1.002E4,3.847E3,-3.4269767E-3,-2.4543364E-2,4.674595E6,8.652791E1,1E0,-1.5913421E-2,1.6659373E0,1.260129E6,3.10364E5,4.5071664E-1,2.9602781E-2,9.965601E-3,8E0,-6.723553E-3,8.960506E3,8E0,2.3598392E-3,8.421419E-1,9.273716E-3,2.4034798E-3,1.2241036E12,1.2414E4,3.8972758E8,1.6446589E-3,1.2925185E-2,5.2665914E-3,-1.1401988E-2,-2.7594191E-3,8.560774E-3,-4.2228858E-5,-1.586716E-2,-5.4448466E-3],"split_indices":[9,9,2,0,0,28,58,105,0,53,29,9,27,0,0,3,0,33,3,0,27,0,0,31,9,7,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1E1,1.83E2,5E0,5E0,1.53E2,3E1,1.46E2,7E0,1.3E1,1.7E1,1.28E2,1.8E1,5E0,8E0,1.1E1,6E0,1.08E2,2E1,6E0,1.2E1,5E0,6E0,1.1E1,9.7E1,1.2E1,8E0,6E0,6E0,6E0,5E0,1E1,8.7E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[8.045889E-4,4.4750556E-2,-2.1095945E-1,6.7334473E-3,3.2452223E-1,-5.1016223E-1,9.030463E-3,-1.0705848E-2,2.4121732E-2,4.9976826E-1,3.558373E-3,-3.199334E-2,-9.039044E-3,9.939486E-3,-1.4055084E-1,6.686401E-2,-6.288247E-2,1.1249147E-2,3.264647E-2,-9.149293E-5,-1.3360373E-2,7.7594034E-3,1.953505E-1,-1.2396897E-1,5.176696E-2,8.271019E-3,-2.3497034E-2,1.2856285E-2,5.8134798E-2,2.0341645E-3,-2.0098475E-1,6.791249E-3,-4.0330556E-3,1.0166594E-3,-7.795795E-3,1.0631097E-2,-5.525003E-3,-1.40098035E-2,-1.3061812E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7869202E0,1.6889563E0,2.1767197E0,5.697796E-1,7.96123E-1,5.835202E-1,6.228133E-1,0E0,4.9508065E-1,3.6948204E-1,0E0,0E0,0E0,0E0,2.2959414E-1,6.7436296E-1,3.1457287E-1,0E0,0E0,0E0,0E0,3.2358354E-1,2.6487327E-1,3.8024464E-1,2.0767505E-1,0E0,3.3342624E-1,0E0,3.38073E-1,0E0,3.0974334E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,14,14,15,15,16,16,21,21,22,22,23,23,24,24,26,26,28,28,30,30],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.780233E9,3.965E3,2.002992E6,1.2585019E2,1.3770975E2,1.9006333E0,5.0797403E5,-1.0705848E-2,1.49108E5,5.818E3,3.558373E-3,-3.199334E-2,-9.039044E-3,9.939486E-3,1.08725375E5,3.0332904E2,3.2970743E2,1.1249147E-2,3.264647E-2,-9.149293E-5,-1.3360373E-2,1.3257E4,2.8477445E0,1.8836116E5,1.9185424E0,8.271019E-3,7.3501956E-1,1.2856285E-2,1.2414E4,2.0341645E-3,2.1296928E0,6.791249E-3,-4.0330556E-3,1.0166594E-3,-7.795795E-3,1.0631097E-2,-5.525003E-3,-1.40098035E-2,-1.3061812E-3],"split_indices":[5,2,1,47,58,53,32,0,29,2,0,0,0,0,33,4,52,0,0,0,0,9,57,32,54,0,39,0,9,0,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.58E2,3.2E1,1.4E2,1.8E1,1.3E1,1.9E1,9E0,1.31E2,1E1,8E0,8E0,5E0,8E0,1.1E1,8.8E1,4.3E1,5E0,5E0,6E0,5E0,6.1E1,2.7E1,2.8E1,1.5E1,9E0,5.2E1,1.7E1,1E1,9E0,1.9E1,9E0,6E0,4E1,1.2E1,5E0,5E0,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.6608848E-3,4.782441E-2,-1.8848379E-1,6.24066E-3,4.3468302E-1,-3.436911E-1,-2.3852276E-2,-2.6888508E-1,2.7813107E-2,2.913417E-2,1.1125077E-2,-2.3869023E-2,-1.9915147E-1,1.4901195E-1,-1.0138252E-2,-5.924615E-3,-1.7692085E-2,-2.3946729E-2,1.0824097E-1,-1.45787485E-2,-3.859956E-3,2.5187046E-3,1.0569306E-2,2.7660474E-2,-2.51457E-1,-2.241764E-2,1.4921668E-1,5.440983E-2,-6.752757E-3,-2.8779164E-1,-6.8477327E-3,2.4939296E-3,-6.246875E-3,1.3651121E-2,9.18642E-2,-1.4470458E-3,6.452277E-3,-1.7658511E-2,-7.6187886E-3,6.20227E-4,7.478205E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,27,29,31,33,35,-1,37,-1,-1,-1,-1,39,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7614659E0,2.6516476E0,9.883306E-1,8.977488E-1,4.2063904E-1,3.4749722E-1,6.806921E-1,1.14720225E-1,5.854026E-1,0E0,0E0,0E0,1.2511396E-1,6.435989E-2,0E0,0E0,0E0,1.016696E0,2.9780567E-1,0E0,0E0,0E0,0E0,3.2789797E-1,2.1955013E-2,1.1936582E-1,3.0076098E-1,4.2517024E-1,0E0,5.5825472E-2,0E0,0E0,0E0,0E0,1.5511492E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,17,17,18,18,23,23,24,24,25,25,26,26,27,27,29,29,34,34],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,28,30,32,34,36,-1,38,-1,-1,-1,-1,40,-1,-1,-1,-1,-1,-1],"split_conditions":[6.6774924E7,3.973E3,1.0532789E3,2.3187147E-5,1E0,1.4693E4,2.17206E5,6.373709E1,1.7356589E0,2.913417E-2,1.1125077E-2,-2.3869023E-2,3.7448416E2,7.7919556E2,-1.0138252E-2,-5.924615E-3,-1.7692085E-2,2.1247E4,2.0752347E2,-1.45787485E-2,-3.859956E-3,2.5187046E-3,1.0569306E-2,1.875502E0,1.3E1,1.3697663E7,6.063759E8,2.4246153E2,-6.752757E-3,2.3407245E-1,-6.8477327E-3,2.4939296E-3,-6.246875E-3,1.3651121E-2,1.496E3,-1.4470458E-3,6.452277E-3,-1.7658511E-2,-7.6187886E-3,6.20227E-4,7.478205E-3],"split_indices":[45,2,4,39,75,9,11,47,53,0,0,0,52,52,0,0,0,9,4,0,0,0,0,54,8,45,5,4,0,42,0,0,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.64E2,3.8E1,1.49E2,1.5E1,1.9E1,1.9E1,1E1,1.39E2,7E0,8E0,8E0,1.1E1,1E1,9E0,5E0,5E0,8.5E1,5.4E1,5E0,6E0,5E0,5E0,7E1,1.5E1,1.3E1,4.1E1,6.1E1,9E0,1E1,5E0,8E0,5E0,1.1E1,3E1,3E1,3.1E1,5E0,5E0,1.4E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.8946908E-2,6.413307E-2,-1.7502837E-1,2.1907693E-2,3.9189348E-1,-4.74425E-1,3.186223E-2,-9.470217E-3,3.4124196E-2,3.2232758E-2,2.1049017E-1,-1.1822359E-2,-3.2157667E-2,1.957844E-1,-1.219015E-1,1.2668778E-1,-3.4172866E-3,1.272246E-2,4.8858933E-3,4.9799457E-3,1.2949473E-2,-1.3344702E-2,5.139391E-3,8.297384E-3,1.6461056E-1,2.8118532E-2,-1.6275041E-1,-3.7327516E-3,4.461527E-3,1.2927429E-2,1.2267642E-1,-1.2158008E-1,7.532387E-2,-1.4797753E-2,-1.4168705E-3,3.17265E-3,7.5824433E-3,-3.5839662E-4,-8.53225E-3,-1.3248391E-3,5.4862974E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,-1,-1,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,31,33,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8080497E0,2.3026621E0,2.4308677E0,4.064404E-1,8.2657456E-1,5.512285E-1,6.2879324E-1,0E0,4.9387926E-1,0E0,5.0595224E-2,0E0,0E0,5.401942E-2,4.9657404E-1,1.8271166E-1,5.170958E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.7701716E-2,1.0368699E-1,6.168832E-1,3.216926E-1,0E0,0E0,0E0,3.774342E-2,1.3408294E-1,2.6954037E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,13,13,14,14,15,15,16,16,23,23,24,24,25,25,26,26,30,30,31,31,32,32],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,-1,-1,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,32,34,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.6774924E7,3.973E3,1.0532789E3,4.26E2,1E0,8.639937E9,1.74E3,-9.470217E-3,4.06E2,3.2232758E-2,9.982716E0,-1.1822359E-2,-3.2157667E-2,2.1936802E3,4.0950096E10,7.2E2,4.8028188E5,1.272246E-2,4.8858933E-3,4.9799457E-3,1.2949473E-2,-1.3344702E-2,5.139391E-3,2.5823686E-2,2.0080225E5,2.2243262E5,4.876063E10,-3.7327516E-3,4.461527E-3,1.2927429E-2,8.364481E3,2.03E0,2.228762E2,-1.4797753E-2,-1.4168705E-3,3.17265E-3,7.5824433E-3,-3.5839662E-4,-8.53225E-3,-1.3248391E-3,5.4862974E-3],"split_indices":[45,2,4,2,75,5,0,0,0,0,57,0,0,4,5,2,33,0,0,0,0,0,0,57,28,33,31,0,0,0,47,54,4,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.66E2,3.8E1,1.48E2,1.8E1,1.5E1,2.3E1,7E0,1.41E2,6E0,1.2E1,8E0,7E0,1.1E1,1.2E1,4E1,1.01E2,7E0,5E0,6E0,5E0,7E0,5E0,1E1,3E1,8.5E1,1.6E1,5E0,5E0,7E0,2.3E1,2E1,6.5E1,7E0,9E0,1E1,1.3E1,7E0,1.3E1,1.8E1,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.347735E-3,2.99794E-2,-1.7060877E-1,2.6518682E-3,2.9625708E-1,-2.4153547E-1,6.0152872E-3,-1.410911E-2,1.640966E-2,1.9903222E-2,8.776839E-3,-1.537585E-1,-2.4937226E-2,-7.769096E-3,1.3858135E-1,-2.6932558E-1,-6.117016E-3,1.7459609E-1,-2.8316734E-2,2.2076572E-1,-1.5356402E-3,-5.4409946E-3,-1.6443912E-2,3.8554084E-3,-3.8070178E-3,2.0043582E-3,1.2089219E-2,-4.744251E-2,8.491665E-3,1.2947663E-2,7.030206E-3,3.9293463E-3,-3.2594209E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[1.2522455E0,1.2508088E0,8.2856035E-1,6.466775E-1,1.3948762E-1,6.791053E-1,0E0,0E0,4.4712457E-1,0E0,0E0,4.2491508E-1,0E0,4.8083508E-1,3.576647E-1,1.2218678E-1,8.261041E-2,1.2361914E-1,4.6054178E-1,2.26233E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8816044E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,27,27],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,5.251E3,3.0922055E12,2.3187147E-5,2.13281E5,1E0,6.0152872E-3,-1.410911E-2,2.0827537E0,1.9903222E-2,8.776839E-3,1.8663485E0,-2.4937226E-2,1.2414E4,3.329399E-1,1.083E3,3E0,1.117E3,5.8018835E2,6.063759E8,-1.5356402E-3,-5.4409946E-3,-1.6443912E-2,3.8554084E-3,-3.8070178E-3,2.0043582E-3,1.2089219E-2,2.0304577E-1,8.491665E-3,1.2947663E-2,7.030206E-3,3.9293463E-3,-3.2594209E-3],"split_indices":[45,2,31,39,29,101,0,0,53,0,0,53,0,9,41,0,8,2,52,5,0,0,0,0,0,0,0,27,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.71E2,3.7E1,1.56E2,1.5E1,3E1,7E0,6E0,1.5E2,6E0,9E0,2.4E1,6E0,1.26E2,2.4E1,1.3E1,1.1E1,1.2E1,1.14E2,1.6E1,8E0,5E0,8E0,5E0,6E0,5E0,7E0,1.05E2,9E0,8E0,8E0,1.4E1,9.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.3159277E-2,-1.2684307E-3,2.4888517E-1,-1.867153E-2,1.1005594E-2,2.7766839E-2,7.7967346E-2,-2.022436E-1,3.0439774E-2,-6.6097155E-3,9.568757E-3,-1.2905475E-2,-3.710245E-3,5.972386E-2,-6.9887124E-2,-7.167451E-2,9.143325E-2,-1.8482423E-1,4.290475E-2,2.6465082E-3,-1.1885702E-1,1.0909389E-1,-3.7460733E-2,-1.401572E-2,-7.0086926E-2,1.18056856E-1,-5.953316E-3,-1.6611579E-3,-8.843023E-3,6.7725587E-3,2.0228736E-3,4.9213176E-3,-1.1569566E-2,-5.2233515E-3,-9.3251956E-4,7.8016566E-3,2.089101E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,-1,7,-1,9,11,13,-1,-1,-1,-1,15,17,19,21,23,25,-1,27,29,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1858184E0,9.340575E-1,1.1417756E0,0E0,7.935776E-1,0E0,4.2194676E-1,1.1319184E-1,5.183208E-1,0E0,0E0,0E0,0E0,5.732057E-1,5.264887E-1,1.6824706E-1,2.544853E-1,2.2670567E-1,2.7849624E-1,0E0,1.042591E-1,2.0335376E-1,4.2523852E-1,0E0,1.9561965E-2,4.2917043E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,24,24,25,25],"right_children":[2,4,6,-1,8,-1,10,12,14,-1,-1,-1,-1,16,18,20,22,24,26,-1,28,30,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,6E0,3.0451374E9,-1.867153E-2,4.7486033E-2,2.7766839E-2,9.248443E8,3.0734711E-6,3.00037E5,-6.6097155E-3,9.568757E-3,-1.2905475E-2,-3.710245E-3,7.23E2,2.1296928E0,4.398424E9,4.8028188E5,4.048035E0,2.2909E4,2.6465082E-3,3.3438003E5,1.7476E4,3.413613E7,-1.401572E-2,4.57825E5,1.1E1,-5.953316E-3,-1.6611579E-3,-8.843023E-3,6.7725587E-3,2.0228736E-3,4.9213176E-3,-1.1569566E-2,-5.2233515E-3,-9.3251956E-4,7.8016566E-3,2.089101E-3],"split_indices":[2,3,12,0,58,0,7,38,29,0,0,0,0,2,54,31,33,57,9,0,32,9,12,0,29,3,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.94E2,2E1,5E0,1.89E2,6E0,1.4E1,1.5E1,1.74E2,5E0,9E0,9E0,6E0,1.35E2,3.9E1,2.6E1,1.09E2,1.9E1,2E1,7E0,1.9E1,9.6E1,1.3E1,9E0,1E1,1.4E1,6E0,9E0,1E1,6.4E1,3.2E1,8E0,5E0,5E0,5E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.9316217E-2,1.7384242E-3,-4.0611404E-1,1.2895812E-2,-1.4971867E-2,-2.9367857E-2,-6.3019297E-3,-1.9112745E-2,9.588359E-2,2.422868E-2,-1.11033335E-1,2.0667966E-1,-1.833689E-2,-8.566936E-2,5.3505994E-2,-1.5332667E-2,-7.7738605E-2,3.0302718E-1,1.0297666E-1,-7.6985806E-2,8.077721E-3,-3.622631E-2,-9.7368555E-3,1.8223117E-1,8.963458E-3,-1.0182656E-1,3.8204778E-3,1.7622638E-2,7.4402755E-3,2.7882059E-3,7.6865903E-3,2.0768017E-2,-1.7662764E-1,-4.8481966E-3,2.9590416E-3,5.16664E-3,1.46106975E-2,-5.7287784E-3,2.854076E-3,-6.3493457E-3,-4.1879606E-4,-2.248854E-3,4.6135853E-3,-3.981848E-3,-1.1531701E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,-1,35,37,39,-1,-1,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7353544E0,7.1578854E-1,5.4422545E-1,5.251274E-1,0E0,0E0,0E0,5.723978E-1,6.998247E-1,3.1990075E-1,2.9522884E-1,2.4855816E-1,3.199754E-1,1.1471072E-1,4.4559878E-1,0E0,1.6365258E-1,7.691753E-2,2.9261157E-2,2.1795633E-1,0E0,1.0811561E-1,0E0,1.4971662E-1,3.9094865E-1,1.0173422E-1,0E0,0E0,0E0,0E0,0E0,6.589077E-2,4.590586E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,31,31,32,32],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,-1,36,38,40,-1,-1,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,4.674595E6,4.44E3,7.5614764E2,-1.4971867E-2,-2.9367857E-2,-6.3019297E-3,1.3697663E7,6.5023036E7,4.1E1,1.3047E4,3.6025445E6,2.7545524E10,2.6855614E0,8.7678794E5,-1.5332667E-2,2.5522516E0,2.32018E5,2.185321E0,9.5E2,8.077721E-3,7.12E2,-9.7368555E-3,1.036E3,8E0,2.3361E4,3.8204778E-3,1.7622638E-2,7.4402755E-3,2.7882059E-3,7.6865903E-3,5.5053414E9,6.557827E-1,-4.8481966E-3,2.9590416E-3,5.16664E-3,1.46106975E-2,-5.7287784E-3,2.854076E-3,-6.3493457E-3,-4.1879606E-4,-2.248854E-3,4.6135853E-3,-3.981848E-3,-1.1531701E-2],"split_indices":[45,28,2,4,0,0,0,45,45,52,9,47,5,58,50,0,54,29,41,0,0,2,0,2,3,10,0,0,0,0,0,5,27,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.02E2,1E1,1.96E2,6E0,5E0,5E0,1.42E2,5.4E1,9.7E1,4.5E1,2.7E1,2.7E1,2E1,7.7E1,5E0,4E1,1.3E1,1.4E1,2.1E1,6E0,1.5E1,5E0,1.9E1,5.8E1,3.5E1,5E0,8E0,5E0,9E0,5E0,1.1E1,1E1,9E0,6E0,1.3E1,6E0,1.6E1,4.2E1,2.6E1,9E0,6E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.960221E-4,4.9734674E-2,-1.3045083E-1,2.8344585E-2,2.0658705E-2,-2.4736756E-1,1.8464018E-2,-1.8395035E-1,4.7144543E-2,-1.3834806E-1,-3.6169982E-1,1.2267338E-1,-1.1083173E-2,-5.1432513E-3,-1.162505E-2,1.0340959E-1,-7.357448E-3,-1.8207927E-1,-2.9187736E-3,-2.1794751E-2,-9.11054E-3,3.4559835E-4,1.894237E-1,4.018682E-2,2.0548746E-1,-5.9672784E-2,8.145917E-2,-1.1337451E-2,-4.6549262E-3,3.3939248E-3,1.4190368E-2,7.799603E-2,-4.1002747E-2,5.476589E-3,1.195926E-2,4.5717508E-2,-1.6714114E-1,3.7712115E-3,1.5885705E-1,5.3971703E-3,-5.530482E-4,4.198044E-3,-5.8543882E-3,6.207525E-3,-2.7565528E-3,-5.319216E-3,-1.1299442E-2,-4.5124264E-3,3.2893305E-3,1.2609243E-2,1.5246854E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,-1,-1,-1,-1,29,31,33,35,37,-1,-1,-1,-1,39,41,-1,-1,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3458968E0,1.2280759E0,9.9697375E-1,5.8738697E-1,0E0,3.4882176E-1,7.052126E-1,2.7904272E-2,4.1167948E-1,4.9852133E-2,1.5336025E-1,1.450659E-1,0E0,0E0,0E0,4.1768265E-1,3.254674E-1,2.79786E-2,0E0,0E0,0E0,0E0,1.3095483E-1,1.3212602E-1,7.207525E-2,5.059906E-1,1.5573584E-1,0E0,0E0,0E0,0E0,9.1301054E-2,1.5692843E-1,0E0,0E0,2.0659679E-1,5.8638692E-2,9.545635E-2,1.6392946E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,22,22,23,23,24,24,25,25,26,26,31,31,32,32,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,-1,-1,-1,-1,30,32,34,36,38,-1,-1,-1,-1,40,42,-1,-1,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3625324E7,5.666E3,1.0908286E3,4.75E2,2.0658705E-2,1E0,2.192772E10,4.398424E9,7.2591E4,4.177399E2,1E0,1E0,-1.1083173E-2,-5.1432513E-3,-1.162505E-2,2.4981093E2,4.506623E2,5.50569E5,-2.9187736E-3,-2.1794751E-2,-9.11054E-3,3.4559835E-4,1.2563624E3,8.869983E5,2.54E2,1.9439217E0,5.929222E-1,-1.1337451E-2,-4.6549262E-3,3.3939248E-3,1.4190368E-2,5.37E2,5.320276E5,5.476589E-3,1.195926E-2,6.717009E-1,1.3474992E6,3.0963843E8,4.1936096E8,5.3971703E-3,-5.530482E-4,4.198044E-3,-5.8543882E-3,6.207525E-3,-2.7565528E-3,-5.319216E-3,-1.1299442E-2,-4.5124264E-3,3.2893305E-3,1.2609243E-2,1.5246854E-3],"split_indices":[45,2,4,2,0,100,5,31,29,52,104,104,0,0,0,55,4,12,0,0,0,0,52,32,0,56,27,0,0,0,0,0,48,0,0,27,50,7,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.51E2,5.6E1,1.44E2,7E0,3.1E1,2.5E1,1.1E1,1.33E2,1.7E1,1.4E1,1.8E1,7E0,6E0,5E0,6.5E1,6.8E1,1E1,7E0,8E0,6E0,7E0,1.1E1,4.1E1,2.4E1,4.3E1,2.5E1,5E0,5E0,6E0,5E0,2.8E1,1.3E1,9E0,1.5E1,2.2E1,2.1E1,1.3E1,1.2E1,2E1,8E0,5E0,8E0,1.2E1,1E1,1.3E1,8E0,5E0,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.2828152E-3,-1.810118E-2,1.976273E-2,3.100423E-2,-1.662449E-2,8.899857E-4,1.4692181E-1,6.670543E-2,-4.2529665E-2,2.6706374E-1,-7.111895E-2,-5.2328603E-3,1.6651355E-1,-6.349596E-2,7.787039E-3,4.0701374E-1,1.3729946E-1,-1.12410365E-2,3.0643775E-3,-1.01899095E-1,3.9583586E-2,2.49493E-1,4.8913088E-2,-4.4254355E-2,-1.4979733E-2,9.9427225E-3,2.4905127E-2,8.9216195E-3,1.5751071E-3,-6.9077914E-3,-2.4478629E-3,4.531733E-3,-2.7733485E-3,1.3636827E-2,7.1539427E-3,7.207862E-3,-2.1610053E-3,-5.1050406E-4,-6.15707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,-1,3,5,-1,7,9,11,13,15,17,19,21,23,-1,25,27,-1,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0689161E0,0E0,8.3124214E-1,6.770787E-1,0E0,4.4577786E-1,1.0668268E0,4.4603288E-1,4.1447905E-1,4.1629052E-1,3.463327E-1,1.6432631E-1,2.4151552E-1,3.984984E-1,0E0,1.6221058E-1,7.050732E-2,0E0,0E0,1.841411E-2,1.4488176E-1,1.0310292E-2,1.21345885E-1,2.2795153E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,23,23],"right_children":[2,-1,4,6,-1,8,10,12,14,16,18,20,22,24,-1,26,28,-1,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01E4,-1.810118E-2,4.7379535E6,5.8018835E2,-1.662449E-2,6.769573E6,1.3E1,2.1231644E2,6.204429E6,1.9160621E0,3.6E1,2.6727284E5,5.3246627E0,5.08616E5,7.787039E-3,1.3841E4,1.244201E6,-1.12410365E-2,3.0643775E-3,5.5812734E3,5.34E2,1.0955342E-1,7.0918E4,9.0763354E-1,-1.4979733E-2,9.9427225E-3,2.4905127E-2,8.9216195E-3,1.5751071E-3,-6.9077914E-3,-2.4478629E-3,4.531733E-3,-2.7733485E-3,1.3636827E-2,7.1539427E-3,7.207862E-3,-2.1610053E-3,-5.1050406E-4,-6.15707E-3],"split_indices":[9,0,28,52,0,45,3,4,32,54,3,33,57,33,0,9,29,0,0,48,0,38,29,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,6E0,1.98E2,1.93E2,5E0,1.54E2,3.9E1,6.1E1,9.3E1,2.5E1,1.4E1,3.6E1,2.5E1,8.5E1,8E0,1.1E1,1.4E1,6E0,8E0,1.1E1,2.5E1,1.4E1,1.1E1,8E1,5E0,5E0,6E0,9E0,5E0,5E0,6E0,1.6E1,9E0,9E0,5E0,5E0,6E0,5.8E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.483439E-3,-1.839502E-2,1.8489068E-2,2.684116E-3,2.5665677E-1,5.8165703E-2,-4.762403E-2,3.0995454E-3,2.097586E-2,1.413173E-1,-1.5608356E-2,-1.3283266E-2,-2.6447117E-2,1.4328361E-2,1.9337034E-1,-1.5175442E-1,4.091086E-2,-8.34186E-2,5.9744753E-2,6.473251E-3,-5.1984196E-3,1.3063373E-1,1.3613368E-2,-1.1930123E-2,9.311462E-4,8.138736E-2,-3.8598268E-3,-3.4057397E-2,-2.533447E-1,-2.3098495E-2,1.772652E-1,3.534431E-3,8.356614E-3,1.0678837E-3,7.1132667E-3,-3.6367425E-3,3.2752915E-3,-6.0690586E-3,-1.5604789E-2,5.179016E-3,-3.787409E-3,2.265727E-3,1.1379285E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,31,-1,-1,-1,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2441623E0,0E0,7.258073E-1,5.1076406E-1,3.8901556E-1,5.3590107E-1,4.5952183E-1,0E0,0E0,2.68246E-1,3.6786726E-1,0E0,4.4201645E-1,2.0754541E-1,1.2804914E-1,2.3795244E-1,1.73735E-1,4.443106E-1,3.545825E-1,0E0,0E0,3.444457E-2,0E0,0E0,0E0,9.97749E-2,0E0,1.8951869E-1,6.8748295E-2,1.7006123E-1,1.0624263E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,25,25,27,27,28,28,29,29,30,30],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,32,-1,-1,-1,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,-1.839502E-2,1.4056875E3,6.781768E-1,1.3841E4,1.8949672E0,1.1507483E2,3.0995454E-3,2.097586E-2,3.2258064E-2,1.0419601E2,-1.3283266E-2,1.546644E6,3.4246575E-3,1E0,1.514E3,2.5086E4,3.3487454E11,3.200388E2,6.473251E-3,-5.1984196E-3,2.0495868E0,1.3613368E-2,-1.1930123E-2,9.311462E-4,8.9625574E8,-3.8598268E-3,1E0,2.0037713E0,1E0,1.3161152E-4,3.534431E-3,8.356614E-3,1.0678837E-3,7.1132667E-3,-3.6367425E-3,3.2752915E-3,-6.0690586E-3,-1.5604789E-2,5.179016E-3,-3.787409E-3,2.265727E-3,1.1379285E-2],"split_indices":[3,0,52,27,9,58,47,0,0,57,52,0,28,57,75,2,9,31,4,0,0,54,0,0,0,12,0,16,56,104,38,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,7E0,1.92E2,1.81E2,1.1E1,8.6E1,9.5E1,6E0,5E0,4E1,4.6E1,7E0,8.8E1,1.2E1,2.8E1,1.3E1,3.3E1,5.3E1,3.5E1,6E0,6E0,1.8E1,1E1,8E0,5E0,2.5E1,8E0,4.2E1,1.1E1,2.1E1,1.4E1,9E0,9E0,1.4E1,1.1E1,3E1,1.2E1,5E0,6E0,6E0,1.5E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.7281901E-2,2.5003694E-4,3.3157447E-1,-1.6468253E-2,1.2410216E-2,2.0462748E-2,8.660084E-3,-1.1171214E-2,2.397807E-2,1.2280472E-2,1.2356259E-2,-7.322393E-2,2.9728822E-2,-1.20522976E-1,1.8280072E-3,6.524474E-2,-4.8581608E-2,-8.009211E-3,-8.6138054E-4,-4.609298E-3,4.1734027E-3,1.3997809E-3,-6.113033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,-1,7,-1,-1,-1,9,-1,11,13,15,17,-1,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1454566E0,8.5760653E-1,8.129895E-2,0E0,5.6809705E-1,0E0,0E0,0E0,5.099051E-1,0E0,2.727649E-1,1.679123E-1,4.2592233E-1,1.0224521E-1,0E0,3.7909704E-1,2.9880708E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,10,10,11,11,12,12,13,13,15,15,16,16],"right_children":[2,4,6,-1,8,-1,-1,-1,10,-1,12,14,16,18,-1,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,1.01E4,8.142301E-1,-1.6468253E-2,6E0,2.0462748E-2,8.660084E-3,-1.1171214E-2,2.0318625E-2,1.2280472E-2,7.38E2,1.8837452E0,1.8733E4,1E0,1.8280072E-3,1.2245E4,1.9439217E0,-8.009211E-3,-8.6138054E-4,-4.609298E-3,4.1734027E-3,1.3997809E-3,-6.113033E-3],"split_indices":[2,9,27,0,3,0,0,0,27,0,2,53,9,101,0,9,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,2.03E2,1E1,6E0,1.97E2,5E0,5E0,8E0,1.89E2,8E0,1.81E2,3E1,1.51E2,2.1E1,9E0,1.04E2,4.7E1,1.4E1,7E0,1.2E1,9.2E1,2.4E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.1349075E-2,6.594623E-2,-1.0745421E-1,4.3561485E-2,1.8716639E-2,-1.850474E-1,2.7927723E-2,-4.09549E-2,8.3716765E-2,-1.4021446E-1,-1.8843077E-2,-1.0789985E-2,1.1563168E-1,-1.1761252E-1,8.9538135E-2,6.697921E-2,1.092667E-2,-2.1173939E-1,-4.5036923E-2,1.7170806E-1,5.2664854E-4,5.9023726E-4,-1.9146965E-1,1.0155692E-3,7.886318E-3,8.171284E-2,-2.3240119E-3,-2.9050133E-3,-2.8223756E-1,-5.945428E-3,1.028148E-3,1.01647675E-2,4.916686E-3,-3.4245248E-3,-2.395118E-1,1.2981564E-1,3.539849E-2,-7.7928035E-3,-1.6500223E-2,-1.3993454E-2,-7.0433337E-3,2.8547198E-3,9.405362E-3,-4.2669126E-3,3.0766996E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,25,-1,27,29,31,-1,-1,33,-1,-1,35,-1,-1,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1940831E0,8.9887476E-1,6.144416E-1,4.0728825E-1,0E0,2.9033005E-1,5.144908E-1,4.0065897E-1,1.804356E-1,2.0830607E-1,0E0,0E0,9.7864494E-2,2.422702E-1,7.3469326E-2,1.2987995E-1,0E0,1.7462999E-1,8.152598E-2,8.973122E-3,0E0,0E0,7.5111866E-2,0E0,0E0,1.4244276E-1,0E0,0E0,3.3193648E-2,0E0,0E0,0E0,0E0,0E0,1.0551572E-2,1.3775784E-1,1.3127412E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,22,22,25,25,28,28,34,34,35,35,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,26,-1,28,30,32,-1,-1,34,-1,-1,36,-1,-1,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.083812E7,5.818E3,9.1922754E2,8E0,1.8716639E-2,3.4231E4,4.11591E5,1.8270676E0,5.5403314E2,1.5626E4,-1.8843077E-2,-1.0789985E-2,8.294758E-1,1.5182E4,2.0752347E2,3.237835E-1,1.092667E-2,1.8271865E0,2E0,2.035294E0,5.2664854E-4,5.9023726E-4,4.7655502E-1,1.0155692E-3,7.886318E-3,9.402985E-1,-2.3240119E-3,-2.9050133E-3,1.2699157E0,-5.945428E-3,1.028148E-3,1.01647675E-2,4.916686E-3,-3.4245248E-3,1.0177765E7,2.1548604E2,3.322259E-2,-7.7928035E-3,-1.6500223E-2,-1.3993454E-2,-7.0433337E-3,2.8547198E-3,9.405362E-3,-4.2669126E-3,3.0766996E-3],"split_indices":[45,2,52,3,0,10,28,53,52,9,0,0,27,9,4,38,0,54,8,54,0,0,56,0,0,56,0,0,56,0,0,0,0,0,45,4,57,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.82E2,1.25E2,5.7E1,1.18E2,7E0,3.6E1,2.1E1,3.8E1,8E1,3.1E1,5E0,5E0,1.6E1,2.4E1,1.4E1,7.3E1,7E0,1.7E1,1.4E1,1E1,6E0,9E0,1.5E1,8E0,6E0,6.5E1,8E0,6E0,1.1E1,6E0,8E0,5E0,5E0,5E0,1E1,3.1E1,3.4E1,5E0,6E0,5E0,5E0,1.6E1,1.5E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.0029137E-3,3.7795924E-2,-1.6071907E-1,-6.395732E-3,1.6147704E-1,-1.9704046E-2,-6.877452E-2,-1.1048368E-2,4.963173E-3,2.1178237E-1,-4.3773707E-3,7.7737286E-4,-1.3772557E-2,-2.2910938E-2,9.225907E-2,2.8975055E-1,1.0301094E-1,1.017838E-1,-1.1866453E-1,1.1112798E-2,-1.7166796E-1,1.0512692E-2,1.3776052E-2,6.77288E-3,3.5384205E-1,2.0704875E-2,1.0426444E-2,2.3069135E-4,1.0257712E-2,-1.9801257E-3,-8.442411E-3,-7.7763036E-2,5.4461487E-2,-1.6342095E-3,-1.2673987E-2,7.619628E-2,-3.908185E-3,2.1486767E-2,1.086505E-2,3.4320725E-3,-1.6135231E-3,-1.6360213E-4,-6.5124435E-3,5.1919664E-3,-2.8463904E-4,-1.1635876E-3,7.255253E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,17,-1,19,21,23,25,27,29,31,33,-1,35,-1,37,39,-1,-1,-1,-1,-1,41,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2199233E0,9.138001E-1,8.471694E-1,3.1695798E-1,5.790101E-1,0E0,4.4134796E-1,0E0,2.9167593E-1,2.83939E-1,0E0,2.8955933E-1,0E0,4.624833E-1,2.864269E-1,1.4968312E-1,1.5587376E-1,1.4056791E-1,4.1676924E-2,2.9318386E-1,2.0816946E-1,0E0,1.193022E-1,0E0,6.6654444E-2,3.2882486E-2,0E0,0E0,0E0,0E0,0E0,1.0766117E-1,1.6672039E-1,0E0,0E0,9.440806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,25,25,31,31,32,32,35,35],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,18,-1,20,22,24,26,28,30,32,34,-1,36,-1,38,40,-1,-1,-1,-1,-1,42,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.970116E7,2.081E3,7.567149E4,1.5768223E4,2.2909E4,-1.9704046E-2,5.578E4,-1.1048368E-2,1.9323944E0,1.285E3,-4.3773707E-3,2.0600267E5,-1.3772557E-2,1.9577E4,4.50847E5,1E0,1.2978232E6,1.3182058E3,4.1314016E8,7.99E2,2.0787193E2,1.0512692E-2,6.063759E8,6.77288E-3,1E0,5.428733E5,1.0426444E-2,2.3069135E-4,1.0257712E-2,-1.9801257E-3,-8.442411E-3,1.3579175E0,7.07E2,-1.6342095E-3,-1.2673987E-2,4.6002E4,-3.908185E-3,2.1486767E-2,1.086505E-2,3.4320725E-3,-1.6135231E-3,-1.6360213E-4,-6.5124435E-3,5.1919664E-3,-2.8463904E-4,-1.1635876E-3,7.255253E-3],"split_indices":[45,2,32,33,9,0,10,0,53,0,0,33,0,9,32,8,28,4,7,2,4,0,5,0,75,28,0,0,0,0,0,57,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.66E2,3.7E1,1.23E2,4.3E1,9E0,2.8E1,5E0,1.18E2,3.6E1,7E0,2.2E1,6E0,9E1,2.8E1,2E1,1.6E1,1.2E1,1E1,7.4E1,1.6E1,1E1,1.8E1,7E0,1.3E1,1E1,6E0,7E0,5E0,5E0,5E0,2.4E1,5E1,7E0,9E0,1.1E1,7E0,6E0,7E0,5E0,5E0,1.1E1,1.3E1,2.6E1,2.4E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-4.663613E-3,2.3061233E-2,-2.3680216E-1,7.205894E-3,1.9394493E-2,-7.225073E-2,-2.3722623E-2,-1.4616153E-1,2.4089273E-2,5.8870264E-3,-9.309104E-3,-1.3324203E-2,1.7752482E-3,-1.1407759E-2,8.3039664E-2,5.0689846E-2,-8.735919E-2,1.2802789E-1,9.012896E-3,9.0964185E-3,9.366502E-3,-1.6602299E-1,-8.317334E-3,1.531839E-1,-1.1154417E-3,-4.8415534E-2,4.240434E-3,-2.7362527E-3,2.6150653E-3,-2.979243E-3,-1.185115E-2,9.552315E-3,-3.6932605E-3,1.3648756E-3,8.717929E-3,-6.1697145E-3,1.3393618E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,-1,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2669181E0,1.0624099E0,8.5702384E-1,4.4371215E-1,0E0,3.573261E-1,0E0,4.2131042E-1,3.2320216E-1,0E0,0E0,0E0,0E0,4.6191922E-1,1.920054E-1,3.2349145E-1,2.7219936E-1,1.4059609E-1,1.0929125E-1,1.3262574E-1,0E0,1.7044955E-1,3.4391278E-1,1.0609341E-1,0E0,8.952874E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,-1,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,1.3904022E3,2.6563678E5,3.9134244E-5,1.9394493E-2,1.3649979E10,-2.3722623E-2,5.980258E2,1.9121015E0,5.8870264E-3,-9.309104E-3,-1.3324203E-2,1.7752482E-3,8.3860955E6,8.5913794E5,1.436E3,2.6259702E2,2.9119748E-1,6.16E2,2.1696895E5,9.366502E-3,1.2166988E-2,2.493493E7,8.041484E-5,-1.1154417E-3,7.6696295E-1,4.240434E-3,-2.7362527E-3,2.6150653E-3,-2.979243E-3,-1.185115E-2,9.552315E-3,-3.6932605E-3,1.3648756E-3,8.717929E-3,-6.1697145E-3,1.3393618E-3],"split_indices":[45,52,33,42,0,5,0,4,53,0,0,0,0,45,32,2,52,39,10,32,0,38,45,38,0,27,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.75E2,2E1,1.69E2,6E0,1.3E1,7E0,1.6E1,1.53E2,5E0,8E0,9E0,7E0,9.6E1,5.7E1,5.3E1,4.3E1,3.5E1,2.2E1,4.2E1,1.1E1,2.1E1,2.2E1,3E1,5E0,1.3E1,9E0,1.7E1,2.5E1,1E1,1.1E1,5E0,1.7E1,6E0,2.4E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.500134E-2,1.9240899E-3,-1.8297764E-2,-9.238321E-3,1.4594265E-2,-1.1058382E-2,1.76996E-3,-2.5740376E-2,6.921475E-2,-4.8717983E-2,7.7773966E-2,1.1893928E-1,-1.8777702E-2,1.9638969E-2,-8.5130066E-2,1.4256452E-1,-1.3025928E-3,-2.2204304E-3,1.5601315E-1,4.089425E-3,-5.9180677E-2,-1.4725901E-3,4.739682E-3,3.1695236E-3,-5.78709E-3,8.529443E-3,2.8593335E-3,1.0406741E-2,2.6972203E-3,-5.493269E-3,9.872431E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,-1,5,-1,-1,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2676712E0,6.650818E-1,0E0,4.6387067E-1,0E0,0E0,3.3946964E-1,3.129955E-1,2.3412871E-1,2.6799694E-1,1.6709125E-1,2.1557665E-1,8.9200564E-2,1.5483312E-1,3.856898E-1,3.347382E-2,0E0,0E0,1.5866417E-1,0E0,6.876373E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,20,20],"right_children":[2,4,-1,6,-1,-1,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4089414E8,9.772E3,-1.8297764E-2,4.305949E-2,1.4594265E-2,-1.1058382E-2,1.9483395E0,1.0710328E0,8.5913794E5,1.7039711E0,3.6188666E5,9.814481E3,6.76E2,6.930967E8,2.42E2,4.870977E8,-1.3025928E-3,-2.2204304E-3,3.2693877E0,4.089425E-3,2.9656984E7,-1.4725901E-3,4.739682E-3,3.1695236E-3,-5.78709E-3,8.529443E-3,2.8593335E-3,1.0406741E-2,2.6972203E-3,-5.493269E-3,9.872431E-4],"split_indices":[45,2,0,58,0,0,53,39,32,54,33,51,2,5,10,7,0,0,57,0,12,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.95E2,8E0,1.89E2,6E0,8E0,1.81E2,1.29E2,5.2E1,1.06E2,2.3E1,3.3E1,1.9E1,3.7E1,6.9E1,1.4E1,9E0,6E0,2.7E1,5E0,1.4E1,2.3E1,1.4E1,1.3E1,5.6E1,9E0,5E0,1.6E1,1.1E1,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.4863362E-2,3.979328E-3,-2.1793346E-1,1.5606407E-2,-1.4001171E-2,2.744275E-4,-1.9995995E-2,-1.3591747E-2,1.3966893E-1,1.0195989E-2,-1.1335464E-1,1.5294917E-2,8.1285685E-2,-1.2808241E-2,1.05770335E-1,-1.9811264E-1,-3.7404105E-2,8.165068E-3,1.8690068E-1,6.1503053E-3,-2.5355093E-2,1.4248666E-1,-1.0543197E-3,-1.2489826E-2,-5.136775E-3,1.957013E-3,-5.7855765E-3,-3.7619716E-3,6.2588796E-2,5.538094E-3,1.0877756E-2,-2.7490235E-3,2.41031E-3,3.6977201E-3,1.2467169E-2,8.755685E-4,4.621727E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.6614463E-1,6.3830703E-1,7.6073694E-1,6.46685E-1,0E0,0E0,0E0,3.4526005E-1,3.3053058E-1,2.607912E-1,1.7283744E-1,0E0,2.0698503E-1,1.7331173E-1,1.1104053E-1,4.6002984E-2,1.0930329E-1,8.5120104E-2,5.246341E-3,0E0,2.1855736E-1,1.1831695E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.4752131E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,28,28],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,4.7379535E6,2.0204625E5,3.625E3,-1.4001171E-2,2.744275E-4,-1.9995995E-2,3.70592E5,2.13281E5,2.573537E2,2.868989E2,1.5294917E-2,2.41E3,1.2414E4,1.6235546E1,5.354141E2,8.091679E-1,5.682408E5,2.6823762E5,6.1503053E-3,2.195572E0,2.2614942E0,-1.0543197E-3,-1.2489826E-2,-5.136775E-3,1.957013E-3,-5.7855765E-3,-3.7619716E-3,6.0004652E7,5.538094E-3,1.0877756E-2,-2.7490235E-3,2.41031E-3,3.6977201E-3,1.2467169E-2,8.755685E-4,4.621727E-3],"split_indices":[45,28,33,2,0,0,0,29,29,52,52,0,0,9,57,4,27,28,33,0,54,57,0,0,0,0,0,0,45,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.83E2,1.6E1,1.77E2,6E0,8E0,8E0,1.44E2,3.3E1,1.17E2,2.7E1,7E0,2.6E1,9.5E1,2.2E1,1.2E1,1.5E1,1.6E1,1E1,7E0,8.8E1,1.7E1,5E0,6E0,6E0,8E0,7E0,6E0,1E1,5E0,5E0,6.2E1,2.6E1,1.2E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[9.258E-3,2.2918187E-1,-5.7043983E-3,3.7155168E-3,1.7624674E-2,6.5003214E-3,-1.6989365E-2,-1.5778549E-2,1.3813122E-1,-1.1845693E-2,-2.5330898E-3,1.421478E-2,5.9978753E-2,-1.8269397E-2,9.358504E-2,-1.38391815E-2,7.3769772E-3,-6.1244193E-2,3.347312E-2,1.5494566E-1,6.5755754E-5,2.9625369E-3,-4.1780593E-3,-1.2168624E-3,-8.806099E-3,-4.9815625E-3,2.9706287E-3,9.4350055E-3,3.8388711E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,-1,-1,7,-1,9,11,-1,13,-1,15,17,19,21,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.19938E-1,2.238397E-1,7.5604993E-1,0E0,0E0,5.065846E-1,0E0,4.5314822E-1,2.9203278E-1,0E0,2.1497849E-1,0E0,1.2703401E-1,2.7333218E-1,1.12951666E-1,6.6469505E-2,0E0,2.8968546E-1,2.2608146E-1,2.1920502E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,10,10,12,12,13,13,14,14,15,15,17,17,18,18,19,19],"right_children":[2,4,6,-1,-1,8,-1,10,12,-1,14,-1,16,18,20,22,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6604617E-2,1.594E3,1.485318E8,3.7155168E-3,1.7624674E-2,3.847E3,-1.6989365E-2,6E0,2.13281E5,-1.1845693E-2,1.1256753E0,1.421478E-2,9.6600875E5,1.7206225E0,2E0,1E0,7.3769772E-3,1.8271865E0,1.4827905E2,1.537E4,6.5755754E-5,2.9625369E-3,-4.1780593E-3,-1.2168624E-3,-8.806099E-3,-4.9815625E-3,2.9706287E-3,9.4350055E-3,3.8388711E-3],"split_indices":[27,2,45,0,0,2,0,3,29,0,39,0,28,53,8,16,0,54,47,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.1E1,1.76E2,6E0,5E0,1.71E2,5E0,1.47E2,2.4E1,7E0,1.4E2,7E0,1.7E1,1.21E2,1.9E1,1E1,7E0,6.6E1,5.5E1,1.1E1,8E0,5E0,5E0,5.2E1,1.4E1,9E0,4.6E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.1622097E-2,-5.0481926E-3,2.2760473E-1,1.0897221E-2,-2.0660275E-1,-3.7855402E-4,3.4592232E-1,-9.52579E-2,2.6778463E-2,-1.330758E-4,-1.9640733E-2,2.1852504E-2,8.170833E-3,-2.3192376E-2,-8.659962E-3,3.1985189E-3,9.022446E-2,3.0776726E-3,-5.812834E-3,5.588069E-2,-5.0624076E-2,4.0167157E-2,1.5028565E-1,-3.0274395E-2,1.1432656E-1,-1.2019506E-1,4.0221903E-2,1.2888435E-2,5.0886264E-3,9.697122E-3,1.08191915E-1,2.445736E-3,-5.2146735E-3,6.964426E-3,-1.6324653E-3,-8.110077E-3,1.0754179E-4,5.797402E-3,-1.7815169E-3,3.2207412E-3,-1.7651679E-3,1.5030974E-3,7.999609E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,-1,19,21,-1,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0064808E0,5.261123E-1,5.878887E-1,2.588039E-1,4.9626458E-1,0E0,1.9279122E-1,1.1904432E-1,1.9868648E-1,0E0,0E0,0E0,0E0,1.1076291E-1,0E0,2.8070706E-1,1.021975E-1,0E0,0E0,2.5523165E-1,3.1452644E-1,3.646726E-2,1.3206542E-2,1.3974428E-1,1.4315921E-1,1.6866964E-1,1.420803E-1,4.5715913E-2,0E0,0E0,4.4434443E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,30,30],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,-1,20,22,-1,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,1.00068504E8,1.3841E4,1.4150975E4,5.29E2,-3.7855402E-4,1.442E3,2.0778067E0,3.2970743E2,-1.330758E-4,-1.9640733E-2,2.1852504E-2,8.170833E-3,4.5198887E3,-8.659962E-3,6.769573E6,1.2978232E6,3.0776726E-3,-5.812834E-3,1.6407117E2,1.8270676E0,2.7034E4,1.1421988E0,4.3243725E5,4.6385403E5,1.7E1,3.64E2,3.23153E5,5.0886264E-3,9.697122E-3,2.798E3,2.445736E-3,-5.2146735E-3,6.964426E-3,-1.6324653E-3,-8.110077E-3,1.0754179E-4,5.797402E-3,-1.7815169E-3,3.2207412E-3,-1.7651679E-3,1.5030974E-3,7.999609E-3],"split_indices":[2,45,9,51,0,0,0,54,52,0,0,0,0,51,0,45,28,0,0,4,53,10,56,28,33,8,0,29,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.82E2,1.62E2,2E1,1.51E2,1.1E1,7E0,1.3E1,1.9E1,1.32E2,6E0,5E0,7E0,6E0,1.1E1,8E0,9.7E1,3.5E1,6E0,5E0,4.9E1,4.8E1,2E1,1.5E1,2E1,2.9E1,2.7E1,2.1E1,1.5E1,5E0,5E0,1E1,1E1,1E1,2.4E1,5E0,1.9E1,8E0,1E1,1.1E1,7E0,8E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.8618332E-2,3.137787E-2,-1.6613502E-2,9.732362E-3,2.2172322E-2,-6.0146726E-3,3.0211026E-2,6.0974315E-2,-1.4080745E-2,5.558436E-3,1.11343086E-1,3.127791E-2,-1.0604917E-1,-3.7302032E-2,8.196357E-2,1.7635198E-1,3.4987167E-2,1.0753848E-1,-2.1391442E-2,2.0714686E-3,-1.7826769E-1,-6.5118684E-3,1.7432083E-4,4.897972E-4,5.1192315E-3,6.1948556E-3,1.38725275E-2,6.18817E-3,-1.8125791E-3,8.9236805E-3,2.6287385E-3,-8.155214E-3,6.600618E-4,4.9080336E-3,-4.7260923E-3,-1.2074854E-2,-4.842628E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,-1,-1,5,-1,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.666473E-1,3.1015754E-1,0E0,0E0,2.2941011E-1,0E0,2.4748202E-1,2.9804265E-1,3.1618428E-1,1.7334493E-1,2.7180356E-1,2.0714672E-1,1.9557711E-1,1.3777393E-1,3.2835647E-2,1.223411E-1,1.900612E-1,7.68546E-2,1.6461006E-1,1.21313654E-1,5.9937805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,-1,6,-1,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,1.03932914E5,-1.6613502E-2,9.732362E-3,1.68128E8,-6.0146726E-3,2.1358025E0,1.059E3,6.872321E-1,1E0,6.5411394E5,5.81964E-2,6.82E2,8.7722336E4,9.608108E1,2.342E3,1.39778E5,1.4416069E9,2.7573213E8,1.5316E4,6.567405E2,-6.5118684E-3,1.7432083E-4,4.897972E-4,5.1192315E-3,6.1948556E-3,1.38725275E-2,6.18817E-3,-1.8125791E-3,8.9236805E-3,2.6287385E-3,-8.155214E-3,6.600618E-4,4.9080336E-3,-4.7260923E-3,-1.2074854E-2,-4.842628E-3],"split_indices":[45,28,0,0,7,0,56,2,27,16,32,41,0,32,4,2,29,5,7,9,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.98E2,6E0,9E0,1.89E2,9E0,1.8E2,1.06E2,7.4E1,5.1E1,5.5E1,5E1,2.4E1,3.3E1,1.8E1,2.9E1,2.6E1,2E1,3E1,1E1,1.4E1,9E0,2.4E1,5E0,1.3E1,2.2E1,7E0,1.1E1,1.5E1,7E0,1.3E1,5E0,2.5E1,5E0,5E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.5588157E-2,-1.0718863E-2,2.830293E-2,6.3396767E-3,1.5279642E-1,-8.056633E-3,1.4230662E-2,-1.3687786E-3,2.1461266E-1,4.795031E-2,-2.4838245E-2,9.5527455E-2,3.1418762E-1,7.034596E-2,-9.041948E-2,6.6778235E-2,-6.1359547E-2,4.5723846E-4,7.933124E-3,1.8646631E-2,8.9490805E-3,4.904829E-3,1.22576244E-1,-1.0204915E-2,1.3210474E-3,-2.3568382E-3,1.0609446E-1,-1.4450073E-1,-1.6848097E-2,1.4263155E-3,-4.3937294E-3,9.405544E-3,3.125113E-3,1.4751039E-3,6.588575E-3,-3.833653E-3,-1.0305003E-2,4.575123E-3,-2.5713649E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,-1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.027534E-1,0E0,5.1034206E-1,2.227187E-1,3.2005197E-1,0E0,2.0406903E-1,0E0,2.1900243E-1,2.635678E-1,2.4500613E-1,6.468565E-2,3.2432556E-2,2.4549541E-1,1.7945173E-1,1.00971304E-1,1.9000363E-1,0E0,0E0,0E0,0E0,8.201433E-2,1.563561E-1,0E0,0E0,0E0,3.3634797E-2,6.472629E-2,1.49923E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,21,21,22,22,26,26,27,27,28,28],"right_children":[2,-1,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0287E4,-1.0718863E-2,3.625E3,8.819892E2,1.2047E4,-8.056633E-3,2.9420671E0,-1.3687786E-3,5.818E3,2.8233623E11,1.3469E4,1.8308E4,6.135771E9,2.3114078E2,3.494881E2,2.6089528E5,1.0419601E2,4.5723846E-4,7.933124E-3,1.8646631E-2,8.9490805E-3,3.0875E4,8.5913794E5,-1.0204915E-2,1.3210474E-3,-2.3568382E-3,1.062E3,1E0,1.4507979E0,1.4263155E-3,-4.3937294E-3,9.405544E-3,3.125113E-3,1.4751039E-3,6.588575E-3,-3.833653E-3,-1.0305003E-2,4.575123E-3,-2.5713649E-3],"split_indices":[9,0,2,48,9,0,57,0,2,31,9,10,5,4,52,28,52,0,0,0,0,29,32,0,0,0,2,75,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,9E0,1.86E2,1.59E2,2.7E1,6E0,1.53E2,7E0,2E1,8.2E1,7.1E1,1E1,1E1,7.1E1,1.1E1,2E1,5.1E1,5E0,5E0,5E0,5E0,3.2E1,3.9E1,5E0,6E0,5E0,1.5E1,1.7E1,3.4E1,2.6E1,6E0,1.6E1,2.3E1,5E0,1E1,1E1,7E0,8E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.486105E-3,4.1557796E-2,-9.416759E-2,2.7200432E-2,1.4971977E-2,-3.525441E-2,-2.6240265E-1,8.793478E-2,-2.7163435E-2,-4.185474E-3,-1.144165E-2,-5.082664E-3,-1.9495143E-2,6.7650564E-2,1.1662892E-2,-1.7416552E-1,2.200309E-3,-4.0547837E-2,1.052504E-1,8.922179E-2,-2.8604285E-3,-4.0774103E-3,-1.0809043E-2,3.4158327E-2,-1.4616871E-1,-1.284168E-1,3.2511782E-3,4.4347206E-4,8.263947E-3,5.300417E-2,1.6287145E-1,7.574886E-2,-1.5787225E-2,-9.569575E-3,-3.2687078E-3,-2.394403E-3,-8.494344E-3,4.3361983E-3,-3.5711277E-2,1.1014144E-3,6.024018E-3,9.776303E-3,2.8845135E-3,6.003184E-3,-3.683954E-5,-2.5358978E-3,5.0049843E-3,-3.8350436E-3,2.1749863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,27,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.6168945E-1,5.400181E-1,6.8453765E-1,4.481254E-1,0E0,3.3420867E-1,3.6345947E-1,1.8540865E-1,3.111735E-1,1.9129701E-1,0E0,0E0,0E0,1.6466716E-1,0E0,3.5785586E-2,2.9422924E-1,1.3896619E-1,7.586396E-2,1.2473792E-1,0E0,0E0,0E0,1.0713622E-1,3.2307073E-2,3.715396E-2,8.822799E-2,0E0,0E0,7.351835E-2,5.5374265E-2,1.0627167E-1,1.1273695E-1,0E0,0E0,0E0,0E0,0E0,6.836739E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,18,18,19,19,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,28,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.837056E-1,1.3904022E3,2.0856102E0,2.0218372E0,1.4971977E-2,4.674595E6,9.309742E-1,2.902E3,1.4065148E6,9.9472994E-1,-1.144165E-2,-5.082664E-3,-1.9495143E-2,2.72126E5,1.1662892E-2,1.6999166E5,2.5086E4,1.6827956E0,1.8656379E6,1.135E3,-2.8604285E-3,-4.0774103E-3,-1.0809043E-2,3.5293162E5,9.60419E5,4.2905E4,1.9182949E0,4.4347206E-4,8.263947E-3,2.3357933E0,6.748347E5,1E1,2.9E1,-9.569575E-3,-3.2687078E-3,-2.394403E-3,-8.494344E-3,4.3361983E-3,2.20347E0,1.1014144E-3,6.024018E-3,9.776303E-3,2.8845135E-3,6.003184E-3,-3.683954E-5,-2.5358978E-3,5.0049843E-3,-3.8350436E-3,2.1749863E-3],"split_indices":[27,52,56,58,0,28,27,2,50,39,0,0,0,29,0,47,9,53,28,2,0,0,0,33,9,29,53,0,0,54,32,3,8,0,0,0,0,0,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.4E2,6.9E1,1.34E2,6E0,5.2E1,1.7E1,6.3E1,7.1E1,4.6E1,6E0,9E0,8E0,5.7E1,6E0,1.1E1,6E1,3.5E1,1.1E1,4.9E1,8E0,5E0,6E0,5E1,1E1,1.1E1,2.4E1,5E0,6E0,3.4E1,1.5E1,2.7E1,2.3E1,5E0,5E0,5E0,6E0,7E0,1.7E1,2.5E1,9E0,1E1,5E0,1.6E1,1.1E1,1.8E1,5E0,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-6.508105E-3,1.6223421E-2,-1.4915955E-1,2.065409E-3,1.2026902E-2,-2.4486842E-2,-4.825025E-2,-7.387953E-3,1.1943197E-2,-6.3995714E-3,5.6979783E-2,-8.320494E-2,2.2505622E-2,-2.7172603E-3,7.7219056E-3,2.3895928E-3,-1.4823179E-1,9.554526E-2,5.457966E-3,-3.134557E-3,-9.88493E-3,4.9054846E-2,8.128773E-3,-6.606881E-3,1.8681938E-2,4.279873E-3,7.330425E-4,5.8659716E-3,-9.807984E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,9,-1,11,-1,13,15,17,-1,-1,-1,19,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,-1],"loss_changes":[6.606849E-1,5.845995E-1,9.9803925E-1,2.596325E-1,0E0,0E0,2.1413289E-1,0E0,1.6066182E-1,0E0,1.3946344E-1,1.4585742E-1,1.7753564E-1,0E0,0E0,0E0,3.94184E-2,8.4725484E-2,2.2439085E-1,0E0,0E0,2.3105893E-2,0E0,0E0,2.3254266E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,10,10,11,11,12,12,16,16,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,-1,-1,10,-1,12,-1,14,16,18,-1,-1,-1,20,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,-1],"split_conditions":[7.18771E7,5.818E3,7.567149E4,1.4827905E2,1.2026902E-2,-2.4486842E-2,1.118307E6,-7.387953E-3,5.94E2,-6.3995714E-3,9.508291E2,1E0,2.42E2,-2.7172603E-3,7.7219056E-3,2.3895928E-3,3.8E1,1.0215677E2,3.1743118E-1,-3.134557E-3,-9.88493E-3,2.2673786E5,8.128773E-3,-6.606881E-3,6.4266656E5,4.279873E-3,7.330425E-4,5.8659716E-3,-9.807984E-5],"split_indices":[45,2,32,47,0,0,28,0,2,0,4,104,10,0,0,0,10,52,58,0,0,32,0,0,51,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.75E2,2.7E1,1.66E2,9E0,5E0,2.2E1,9E0,1.57E2,1.2E1,1E1,1.5E1,1.42E2,5E0,5E0,5E0,1E1,2.6E1,1.16E2,5E0,5E0,1.7E1,9E0,9E0,1.07E2,7E0,1E1,1.7E1,9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[9.90877E-3,2.3706255E-2,-2.119755E-2,9.805705E-2,-3.1209667E-3,6.32914E-2,1.2318536E-2,-3.8262174E-2,5.3411704E-2,-2.467048E-3,8.2770735E-2,-1.938913E-2,-8.973596E-3,1.1987318E-2,1.4019722E-1,-3.2636008E-4,1.03218876E-1,2.5436036E-2,-6.83146E-2,-8.005571E-2,4.6486836E-2,1.6948196E-1,2.5573832E-3,1.4141361E-1,3.9455898E-2,-4.5608737E-2,6.9225855E-2,3.7034915E-3,-9.200269E-2,-1.9178271E-3,-5.1136226E-3,1.06466666E-1,-2.1146114E-3,1.0752613E-2,4.3266416E-3,4.2873137E-3,9.062497E-3,2.9874174E-3,1.1235389E-4,1.1143949E-3,-5.24169E-3,-5.564356E-4,6.9525344E-3,-2.6034045E-3,-8.477653E-3,6.8726144E-3,1.8879018E-3,2.157627E-3,-4.6022916E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,15,17,-1,19,21,-1,23,25,27,29,31,33,-1,35,37,39,41,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2650425E0,3.912526E-1,0E0,2.7129525E-1,2.9010448E-1,1.0264477E-1,0E0,2.495329E-1,1.9964147E-1,0E0,7.0311725E-2,1.7937812E-1,0E0,1.2756371E-1,3.6716074E-2,0E0,6.9918126E-2,1.3693194E-1,1.4163953E-1,7.4744076E-3,8.476323E-2,3.629613E-2,0E0,3.0179203E-2,1.0902125E-2,7.7243544E-2,1.6734256E-1,0E0,9.916505E-2,0E0,0E0,2.659136E-2,7.963385E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,28,28,31,31,32,32],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,16,18,-1,20,22,-1,24,26,28,30,32,34,-1,36,38,40,42,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4089414E8,3.77E2,-2.119755E-2,1.564E3,9.835643E5,8.041484E-5,1.2318536E-2,4.9462875E5,5.8018835E2,-2.467048E-3,3.2503597E1,6.2654173E-1,-8.973596E-3,2.0618556E-2,2.304414E-1,-3.2636008E-4,2.1848371E8,3.1622803E5,3.08E3,3.0734711E-6,8.563462E-1,1.7446084E0,2.5573832E-3,8E0,4.283054E6,1.872148E-1,3.6907338E-2,3.7034915E-3,1.1E1,-1.9178271E-3,-5.1136226E-3,1.4320541E1,4.1847788E6,1.0752613E-2,4.3266416E-3,4.2873137E-3,9.062497E-3,2.9874174E-3,1.1235389E-4,1.1143949E-3,-5.24169E-3,-5.564356E-4,6.9525344E-3,-2.6034045E-3,-8.477653E-3,6.8726144E-3,1.8879018E-3,2.157627E-3,-4.6022916E-3],"split_indices":[45,0,0,2,28,38,0,33,52,0,52,27,0,57,38,0,7,28,11,38,27,53,0,3,45,39,38,0,3,0,0,58,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.95E2,5E0,5.1E1,1.44E2,4.3E1,8E0,8.9E1,5.5E1,6E0,3.7E1,8E1,9E0,3.8E1,1.7E1,7E0,3E1,4.2E1,3.8E1,1E1,2.8E1,1.2E1,5E0,1.8E1,1.2E1,1.6E1,2.6E1,5E0,3.3E1,5E0,5E0,1.2E1,1.6E1,6E0,6E0,1E1,8E0,7E0,5E0,8E0,8E0,1.3E1,1.3E1,2.4E1,9E0,7E0,5E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[8.647224E-3,1.9496158E-2,-1.08353505E-2,-1.1126499E-1,3.187505E-2,-1.597631E-1,2.0482688E-4,1.06668904E-1,9.422713E-3,-1.0229649E-2,-3.3733766E-3,7.389277E-2,1.3680107E-2,-3.797559E-2,5.737139E-2,1.1217321E-1,-2.4628686E-3,-6.7563714E-3,-1.3490868E-1,1.3849469E-1,-2.0838702E-2,1.9041004E-3,1.2914152E-1,-4.1387137E-3,2.9122985E-3,7.712396E-2,-5.449495E-2,-1.1433886E-2,5.5795244E-4,1.8086307E-1,6.5049626E-2,-8.939601E-2,3.847694E-2,7.132906E-3,3.5463409E-3,1.354723E-3,7.035777E-3,-6.3518547E-3,5.1264965E-4,3.5332786E-3,1.0481925E-2,8.578658E-3,-8.713419E-4,-7.4946424E-3,-1.2850695E-3,-3.2911068E-3,3.7742646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,-1,15,-1,17,19,21,23,25,27,29,31,-1,33,-1,-1,35,37,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1900417E-1,3.1757018E-1,0E0,9.594238E-2,2.9790974E-1,4.2608976E-2,0E0,2.1928102E-1,3.1580937E-1,0E0,0E0,1.0550165E-1,0E0,2.1092077E-1,4.4066757E-1,2.43631E-2,7.426105E-2,2.2043659E-1,2.612734E-1,9.4034374E-2,1.4983442E-1,0E0,9.6566975E-3,0E0,0E0,6.397368E-2,1.7960474E-1,0E0,0E0,6.9177985E-2,1.3612026E-1,6.711765E-2,9.180185E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,29,29,30,30,31,31,32,32],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,-1,16,-1,18,20,22,24,26,28,30,32,-1,34,-1,-1,36,38,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.3086E4,1.4827905E2,-1.08353505E-2,3.9925885E2,3.06E2,1.7284092E-6,2.0482688E-4,1.564E3,1.6720915E2,-1.0229649E-2,-3.3733766E-3,6.805556E5,1.3680107E-2,2.5492957E0,4.17149E5,1.4781E4,1.5227739E6,2.2193549E0,1.307E3,7.4456794E5,1.3298E4,1.9041004E-3,2.201776E6,-4.1387137E-3,2.9122985E-3,1.5E0,1.8297239E6,-1.1433886E-2,5.5795244E-4,3.1274893E2,1.60756E5,1.8E1,3.6129813E2,7.132906E-3,3.5463409E-3,1.354723E-3,7.035777E-3,-6.3518547E-3,5.1264965E-4,3.5332786E-3,1.0481925E-2,8.578658E-3,-8.713419E-4,-7.4946424E-3,-1.2850695E-3,-3.2911068E-3,3.7742646E-3],"split_indices":[10,47,0,52,0,38,0,2,52,0,0,28,0,56,29,9,28,58,2,32,9,0,32,0,0,58,50,0,0,4,29,3,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.93E2,8E0,1.6E1,1.77E2,1.1E1,5E0,4E1,1.37E2,6E0,5E0,3.5E1,5E0,6.9E1,6.8E1,2.3E1,1.2E1,5.3E1,1.6E1,3.3E1,3.5E1,5E0,1.8E1,5E0,7E0,1.9E1,3.4E1,9E0,7E0,2E1,1.3E1,1.6E1,1.9E1,1.2E1,6E0,1.2E1,7E0,1.5E1,1.9E1,6E0,1.4E1,5E0,8E0,7E0,9E0,5E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-5.417669E-4,9.957788E-3,-1.3962981E-2,-9.563695E-3,1.4863345E-1,7.3793177E-3,-1.2464952E-1,1.5305095E-2,2.483602E-2,-5.8863517E-3,1.5582397E-2,-5.1496316E-2,-1.4906296E-2,4.7171833E-3,-1.0455315E-3,8.380311E-2,-1.3908334E-2,-7.929708E-3,2.5744634E-2,4.8982598E-2,1.3950132E-1,1.436248E-2,-6.041236E-2,3.9720554E-3,-1.7108541E-3,4.962254E-3,1.0523302E-3,9.880532E-3,3.0986848E-3,-7.108199E-4,5.635014E-3,4.7363034E-3,-4.4729947E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,-1,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.137879E-1,5.266094E-1,0E0,3.3435282E-1,4.9963355E-1,1.6217765E-1,2.8411284E-1,0E0,5.367968E-2,0E0,2.8685236E-1,1.5398309E-1,0E0,0E0,0E0,7.656276E-2,1.3233833E-1,0E0,4.16049E-2,3.9025195E-2,6.6554755E-2,1.916962E-1,2.086776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,10,10,11,11,15,15,16,16,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,5.251E3,-1.3962981E-2,9.4801545E-1,1.9824325E0,9.541066E5,1.7999E4,1.5305095E-2,1.14416916E2,-5.8863517E-3,5.0341E4,8E0,-1.4906296E-2,4.7171833E-3,-1.0455315E-3,2.5828317E2,7.0248324E-1,-7.929708E-3,3.830986E-1,4.4358948E-1,2.5537605E8,6.2654173E-1,1.453034E0,3.9720554E-3,-1.7108541E-3,4.962254E-3,1.0523302E-3,9.880532E-3,3.0986848E-3,-7.108199E-4,5.635014E-3,4.7363034E-3,-4.4729947E-3],"split_indices":[45,2,0,27,54,45,10,0,58,0,29,3,0,0,0,4,27,0,56,27,7,27,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.93E2,6E0,1.7E2,2.3E1,1.49E2,2.1E1,9E0,1.4E1,8E0,1.41E2,1.6E1,5E0,5E0,9E0,4.2E1,9.9E1,6E0,1E1,2.7E1,1.5E1,6.2E1,3.7E1,5E0,5E0,8E0,1.9E1,7E0,8E0,4.9E1,1.3E1,6E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[9.526137E-3,1.7923448E-2,-1.3029722E-2,-9.830976E-3,2.9511666E-2,2.1925574E-2,1.0816995E-2,8.412871E-3,1.08070366E-1,-8.296361E-3,6.4635605E-2,1.1868334E-2,6.0768947E-2,9.757413E-3,-1.0356654E-1,1.4077666E-1,2.7007591E-2,1.0870775E-1,-4.7436738E-4,-1.9907411E-3,1.928068E-3,-7.215144E-3,-1.40581E-3,3.1957072E-3,9.760426E-3,3.6754857E-3,-4.2432707E-4,7.925963E-3,2.1343308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,-1,-1,5,7,-1,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8945904E-1,5.231855E-1,0E0,0E0,2.794388E-1,2.1443993E-1,0E0,1.5178588E-1,1.5009823E-1,2.1600886E-1,1.0220432E-1,0E0,6.8833575E-2,1.674914E-1,6.5404415E-2,4.2368308E-2,4.703456E-2,3.6307275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17],"right_children":[2,4,-1,-1,6,8,-1,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,6E0,-1.3029722E-2,-9.830976E-3,1.3717E4,1.2221828E0,1.0816995E-2,2.078125E0,1E0,2.1159E4,6.063759E8,1.1868334E-2,1.8933852E0,1.085E3,1.3E1,3.8372688E8,2.4178083E0,6.6828716E-1,-4.7436738E-4,-1.9907411E-3,1.928068E-3,-7.215144E-3,-1.40581E-3,3.1957072E-3,9.760426E-3,3.6754857E-3,-4.2432707E-4,7.925963E-3,2.1343308E-3],"split_indices":[28,3,0,0,2,39,0,53,64,9,5,0,54,2,8,5,53,27,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.99E2,5E0,9E0,1.9E2,1.84E2,6E0,1.6E2,2.4E1,1.24E2,3.6E1,5E0,1.9E1,1.05E2,1.9E1,1.1E1,2.5E1,1.1E1,8E0,3.9E1,6.6E1,1.1E1,8E0,6E0,5E0,1E1,1.5E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.2389604E-2,2.8968551E-3,-1.3933234E-1,-8.578134E-3,1.5661155E-1,-2.6708233E-1,-4.9959205E-2,6.243325E-3,-1.0250659E-1,2.4541242E-3,1.2142852E-2,-2.93747E-3,-1.9041086E-2,5.1775025E-3,-1.4144672E-1,-7.181487E-2,2.8184786E-2,-1.5944836E-1,3.6368573E-3,-9.700135E-3,-3.0057589E-3,1.1973319E-3,-1.12553746E-1,1.3296191E-1,1.0241429E-2,-2.0866373E-1,-1.6500998E-3,-1.0029227E-2,-6.317428E-2,3.4055253E-3,7.8081065E-3,-7.927909E-2,2.5400525E-2,-1.1923528E-2,-6.082681E-3,5.4569397E-4,-7.697447E-3,-1.0972073E-3,-6.342307E-3,5.209385E-3,3.1830545E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,19,21,23,25,-1,-1,-1,-1,27,29,31,33,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1547375E-1,3.0472982E-1,4.091012E-1,2.2477722E-1,1.0949436E-1,3.7800765E-1,3.4781548E-1,2.4180362E-1,2.356135E-1,0E0,0E0,0E0,0E0,0E0,5.934453E-2,1.2507102E-1,2.0544416E-1,9.740031E-2,0E0,0E0,0E0,0E0,9.192073E-2,1.8134028E-2,1.309335E-1,7.830203E-3,0E0,0E0,1.181462E-1,0E0,0E0,3.8882725E-2,1.2745085E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,14,14,15,15,16,16,17,17,22,22,23,23,24,24,25,25,28,28,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,20,22,24,26,-1,-1,-1,-1,28,30,32,34,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.18771E7,5.666E3,3.4092497E5,3.10364E5,2E0,8.640031E9,1.4689625E5,7.38E2,4.427746E0,2.4541242E-3,1.2142852E-2,-2.93747E-3,-1.9041086E-2,5.1775025E-3,1.1484E4,4.5154606E5,2.86E2,1.569E3,3.6368573E-3,-9.700135E-3,-3.0057589E-3,1.1973319E-3,4.01054E1,1.5596E4,2.0033424E4,6E0,-1.6500998E-3,-1.0029227E-2,9.77E1,3.4055253E-3,7.8081065E-3,1.061E3,6.4266656E5,-1.1923528E-2,-6.082681E-3,5.4569397E-4,-7.697447E-3,-1.0972073E-3,-6.342307E-3,5.209385E-3,3.1830545E-4],"split_indices":[45,2,32,9,8,5,33,2,53,0,0,0,0,0,9,28,0,0,0,0,0,0,52,9,51,8,0,0,52,0,0,2,51,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.71E2,3.6E1,1.6E2,1.1E1,1.4E1,2.2E1,1.39E2,2.1E1,6E0,5E0,6E0,8E0,8E0,1.4E1,3E1,1.09E2,1.6E1,5E0,7E0,7E0,9E0,2.1E1,1.5E1,9.4E1,1.1E1,5E0,6E0,1.5E1,6E0,9E0,1.3E1,8.1E1,6E0,5E0,9E0,6E0,7E0,6E0,1.4E1,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.1634502E-2,-9.842955E-3,-1.6948089E-2,-1.8391602E-2,9.41662E-3,5.338623E-3,-8.9693494E-2,-1.685271E-2,5.196848E-2,-1.3748512E-2,-4.9011827E-2,6.348203E-2,-3.4281645E-2,7.837441E-2,-3.268857E-3,-8.423813E-2,5.1178765E-3,6.7976667E-3,-6.4352644E-4,-6.315026E-2,3.4064792E-2,-1.7868509E-3,1.0053585E-1,-3.812431E-2,-1.4739184E-1,-5.78529E-4,-5.452369E-3,5.844637E-3,-3.30967E-4,7.553949E-3,2.1156387E-3,-2.9762862E-3,1.5441101E-3,-1.0527569E-2,-2.091088E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9935586E-1,3.5473505E-1,0E0,3.2534474E-1,0E0,1.5091874E-1,3.71171E-1,1.4079101E-1,1.5452522E-1,0E0,2.3407266E-1,1.104573E-1,1.6394264E-1,1.03724316E-1,0E0,9.33976E-2,0E0,0E0,0E0,1.4869128E-1,9.091345E-2,0E0,9.933871E-2,3.7461564E-2,9.549427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,1.0086E4,-1.6948089E-2,4.3625324E7,9.41662E-3,2.170343E0,5.8866205E2,1.3257E4,1.472353E0,-1.3748512E-2,2.1907706E12,6.886894E0,4.1698615E2,1.7311496E0,-3.268857E-3,2.5831E4,5.1178765E-3,6.7976667E-3,-6.4352644E-4,1.7111913E0,1.4416069E9,-1.7868509E-3,6.063759E8,9.083174E2,2.7608912E1,-5.78529E-4,-5.452369E-3,5.844637E-3,-3.30967E-4,7.553949E-3,2.1156387E-3,-2.9762862E-3,1.5441101E-3,-1.0527569E-2,-2.091088E-3],"split_indices":[45,2,0,45,0,54,4,9,41,0,31,57,4,53,0,10,0,0,0,54,5,0,5,52,57,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.98E2,6E0,1.91E2,7E0,1.44E2,4.7E1,9.8E1,4.6E1,7E0,4E1,1.7E1,8.1E1,3.8E1,8E0,3.3E1,7E0,8E0,9E0,5.7E1,2.4E1,6E0,3.2E1,2E1,1.3E1,2.9E1,2.8E1,7E0,1.7E1,1.5E1,1.7E1,1.5E1,5E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.276032E-2,-1.2688185E-2,2.1324357E-2,-8.462374E-3,8.292337E-2,4.3239994E-3,-1.356209E-1,1.7443113E-1,2.2138298E-2,-8.407238E-2,1.9729929E-2,-1.0491965E-2,-2.145239E-3,9.395743E-2,1.7627442E-2,5.078603E-2,-6.893461E-3,-1.2056291E-1,4.312951E-4,5.2116897E-2,-5.0548993E-2,7.556235E-3,9.963198E-4,-3.8994014E-2,1.1558176E-1,-2.3126125E-3,-8.414187E-3,7.531314E-2,9.07289E-3,-1.2205438E-1,-3.8766612E-3,-3.7436762E-3,8.101443E-4,3.968125E-2,1.0754098E-2,4.6975417E-3,-2.852415E-4,2.057832E-3,-2.36818E-3,-8.739845E-3,-2.1196534E-3,-4.122912E-3,2.077363E-3,-4.8944075E-4,3.4563073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,-1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,25,-1,27,29,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.763738E-1,0E0,3.5482454E-1,2.1357886E-1,3.4656367E-1,1.6507816E-1,8.1087336E-2,3.5474062E-1,1.9280294E-1,6.2219694E-2,2.3707494E-1,0E0,0E0,8.535756E-2,0E0,2.0229733E-1,0E0,4.3274224E-2,0E0,7.020731E-2,1.096585E-1,0E0,0E0,3.414993E-2,1.5638971E-1,0E0,0E0,8.559072E-2,5.365955E-2,5.2999213E-2,8.5255414E-2,0E0,0E0,2.1794658E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,17,17,19,19,20,20,23,23,24,24,27,27,28,28,29,29,30,30,33,33],"right_children":[2,-1,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,26,-1,28,30,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01E4,-1.2688185E-2,1.996E3,4.939386E7,4.083812E7,4.1E1,9.064853E-1,5.666E3,5.69815E5,2.377416E6,1.00473E5,-1.0491965E-2,-2.145239E-3,7.4456794E5,1.7627442E-2,1.3841E4,-6.893461E-3,3.0847954E1,4.312951E-4,3.5540915E0,3.9232688E5,7.556235E-3,9.963198E-4,2.0907634E3,7.7186523E2,-2.3126125E-3,-8.414187E-3,1.3697663E7,1.5798E4,3.5000316E5,1E0,-3.7436762E-3,8.101443E-4,1.4E1,1.0754098E-2,4.6975417E-3,-2.852415E-4,2.057832E-3,-2.36818E-3,-8.739845E-3,-2.1196534E-3,-4.122912E-3,2.077363E-3,-4.8944075E-4,3.4563073E-3],"split_indices":[9,0,2,45,45,52,27,2,9,45,29,0,0,32,0,9,0,52,0,57,28,0,0,4,52,0,0,45,9,32,64,0,0,8,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,5E0,1.92E2,1.3E2,6.2E1,1.19E2,1.1E1,2.4E1,3.8E1,1.7E1,1.02E2,5E0,6E0,1.8E1,6E0,3.3E1,5E0,1.2E1,5E0,7E1,3.2E1,9E0,9E0,1.4E1,1.9E1,6E0,6E0,4.5E1,2.5E1,1.2E1,2E1,8E0,6E0,1.2E1,7E0,3.5E1,1E1,1.6E1,9E0,6E0,6E0,7E0,1.3E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-3.4794533E-3,2.6454803E-2,-5.5916402E-2,1.7488146E-2,1.0363684E-2,-2.531659E-2,-1.7099589E-1,-4.225516E-3,6.0024522E-2,-4.1870818E-2,4.3911966E-3,-2.3966893E-3,-1.2433394E-2,3.2045785E-2,-4.9688805E-2,8.825244E-3,3.1551097E-2,-2.5482023E-2,-7.2155152E-3,7.593517E-2,-3.0089356E-2,-1.01271614E-1,6.1266944E-2,6.1673205E-2,-4.5253262E-2,-5.709456E-2,1.9386811E-2,4.601767E-2,8.764797E-3,3.3287395E-2,-5.667091E-3,-1.4231472E-1,1.7402736E-3,5.7093985E-3,-2.5831757E-4,3.7106775E-2,5.5276696E-3,4.4736642E-4,-4.422049E-3,-6.281703E-3,-3.2082945E-2,5.201976E-3,-5.655795E-2,3.4515057E-3,-2.1112477E-3,-9.780844E-4,4.759391E-3,-1.2670521E-3,-8.473561E-3,-1.102822E-3,2.8114272E-3,4.369223E-4,-3.7301746E-3,-2.8927648E-4,-4.397073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,21,-1,23,25,-1,27,29,31,33,35,37,39,41,43,-1,45,-1,47,-1,-1,-1,49,-1,-1,-1,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3267772E-1,2.2605234E-1,2.6681963E-1,1.20324306E-1,0E0,1.2389642E-1,1.583651E-1,1.4505641E-1,1.4977263E-1,9.417862E-2,0E0,0E0,0E0,1.3591142E-1,2.2941756E-1,0E0,8.839075E-2,7.060933E-2,0E0,8.442737E-2,1.2132525E-1,1.5764928E-1,5.082228E-2,3.1342067E-2,2.9115021E-2,4.94861E-2,1.4839298E-1,5.9445433E-2,0E0,4.814477E-2,0E0,7.940328E-2,0E0,0E0,0E0,2.7297348E-2,0E0,0E0,0E0,0E0,4.409211E-2,0E0,2.0799331E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,14,14,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,31,31,35,35,40,40,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,22,-1,24,26,-1,28,30,32,34,36,38,40,42,44,-1,46,-1,48,-1,-1,-1,50,-1,-1,-1,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.228668E-1,1.0086E4,2.1492538E0,1.993E3,1.0363684E-2,3.4349198E6,7.6499896E9,6.063759E8,2.01351E5,2.201776E6,4.3911966E-3,-2.3966893E-3,-1.2433394E-2,2.3378747E0,4.1108723E8,8.825244E-3,2.418059E-1,1.8044023E5,-7.2155152E-3,1.43259E5,1.3184165E0,1.12160355E-1,3.5089097E5,1.3640963E3,7.45226E5,2.3135895E6,3.3897146E8,8.869983E5,8.764797E-3,1.127E3,-5.667091E-3,9.189396E4,1.7402736E-3,5.7093985E-3,-2.5831757E-4,2.5185E4,5.5276696E-3,4.4736642E-4,-4.422049E-3,-6.281703E-3,9.857292E-1,5.201976E-3,3.5089097E5,3.4515057E-3,-2.1112477E-3,-9.780844E-4,4.759391E-3,-1.2670521E-3,-8.473561E-3,-1.102822E-3,2.8114272E-3,4.369223E-4,-3.7301746E-3,-2.8927648E-4,-4.397073E-3],"split_indices":[27,2,56,2,0,32,5,5,29,32,0,0,0,58,7,0,42,47,0,1,53,38,33,4,29,45,7,32,0,2,0,32,0,0,0,11,0,0,0,0,56,0,33,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.34E2,7.6E1,1.29E2,5E0,6.1E1,1.5E1,8.6E1,4.3E1,5.4E1,7E0,7E0,8E0,4.8E1,3.8E1,7E0,3.6E1,4.8E1,6E0,2.8E1,2E1,2.6E1,1.2E1,2.6E1,1E1,2.8E1,2E1,2.3E1,5E0,1.2E1,8E0,2E1,6E0,6E0,6E0,1.9E1,7E0,5E0,5E0,6E0,2.2E1,9E0,1.1E1,1.8E1,5E0,7E0,5E0,5E0,1.5E1,5E0,1.4E1,1.2E1,1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-6.8071927E-3,3.3252921E-3,-1.5401018E-2,-8.267497E-2,1.6495116E-2,-1.1915976E-1,4.6711383E-4,1.0675588E-1,-4.6671294E-3,-1.6336638E-1,-1.2625835E-3,-8.955416E-4,1.4412394E-1,-5.1809225E-2,2.8212395E-2,-3.8822824E-3,-1.0089378E-2,6.7288384E-2,9.303808E-3,-1.0931614E-1,4.650273E-2,1.5048489E-2,6.8586054E-3,1.4825326E-3,4.427537E-3,3.3466555E-3,-1.4150657E-1,8.981178E-2,-2.5365364E-3,6.7976825E-2,-1.6345285E-2,-5.0717853E-3,-1.2764283E-2,6.4589083E-4,6.668862E-3,1.3675679E-3,6.3313507E-3,-2.1935876E-3,1.3367885E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,5,7,9,-1,11,13,15,-1,-1,17,19,21,-1,-1,23,-1,25,27,29,-1,-1,-1,-1,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.132226E-1,2.1316153E-1,0E0,8.54646E-2,3.118479E-1,6.954181E-2,0E0,1.4801514E-1,2.0761296E-1,2.9597044E-2,0E0,0E0,7.927585E-2,3.1603244E-1,1.17386304E-1,0E0,0E0,7.1858726E-3,0E0,2.1175134E-1,9.520243E-2,1.2074777E-1,0E0,0E0,0E0,0E0,1.0537225E-1,5.4670863E-2,0E0,6.455509E-2,6.1204318E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,17,17,19,19,20,20,21,21,26,26,27,27,29,29,30,30],"right_children":[2,4,-1,6,8,10,-1,12,14,16,-1,-1,18,20,22,-1,-1,24,-1,26,28,30,-1,-1,-1,-1,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,7.23E2,-1.5401018E-2,3.0050538E5,3.77E2,1.9107901E0,4.6711383E-4,5.033057E-4,3.2198023E-2,2.888788E-3,-1.2625835E-3,-8.955416E-4,1.511E4,3.4942383E2,2.363013E6,-3.8822824E-3,-1.0089378E-2,1.097E3,9.303808E-3,7.518072E-1,5.287008E5,5.4024344E5,6.8586054E-3,1.4825326E-3,4.427537E-3,3.3466555E-3,2.7E1,2.149E3,-2.5365364E-3,3.5519625E5,1.631769E0,-5.0717853E-3,-1.2764283E-2,6.4589083E-4,6.668862E-3,1.3675679E-3,6.3313507E-3,-2.1935876E-3,1.3367885E-3],"split_indices":[45,2,0,33,0,53,0,39,38,39,0,0,9,52,28,0,0,2,0,56,32,32,0,0,0,0,8,2,0,32,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.91E2,1.86E2,5E0,2.4E1,1.62E2,1.7E1,7E0,3E1,1.32E2,1.1E1,6E0,7E0,2.3E1,5.4E1,7.8E1,5E0,6E0,1E1,1.3E1,3.4E1,2E1,7.1E1,7E0,5E0,5E0,5E0,2.9E1,1.4E1,6E0,2.6E1,4.5E1,2.4E1,5E0,6E0,8E0,1.7E1,9E0,2.7E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.8308635E-3,1.46187525E-2,-1.3871151E-2,6.1980085E-3,1.3591889E-1,2.6856175E-2,-3.4569602E-2,1.2774899E-2,1.000254E-3,1.8641052E-3,7.6795936E-2,-7.93177E-2,2.9689558E-2,2.390677E-2,-7.59634E-2,9.5986865E-2,-1.5748015E-3,-9.646353E-3,-5.4726403E-2,4.495078E-3,-2.2537764E-3,5.889842E-3,1.0599353E-1,-7.1861115E-3,2.714796E-4,1.3231534E-1,2.153531E-2,2.993991E-3,-8.3090015E-2,4.436289E-3,-4.2744204E-2,-1.7108666E-3,2.1967562E-3,2.8913823E-4,8.457265E-3,4.871581E-3,9.929357E-3,-2.4663352E-3,5.5238428E-3,-1.3903013E-3,-6.6634226E-3,-1.7853742E-4,-4.5104027E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,-1,-1,35,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3893536E-1,2.0446648E-1,0E0,1.601012E-1,1.8994874E-1,1.571172E-1,1.863853E-1,0E0,0E0,1.476357E-1,9.154895E-2,1.03016466E-1,5.5527773E-2,9.846544E-2,1.15674116E-1,9.452015E-2,0E0,0E0,1.14911E-1,0E0,7.693046E-2,9.473883E-2,8.354321E-2,0E0,0E0,3.4016132E-2,9.493763E-2,0E0,7.771298E-2,0E0,2.7727442E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,20,20,21,21,22,22,25,25,26,26,28,28,30,30],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,-1,-1,36,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,6.146E3,-1.3871151E-2,2.1799106E7,2.9551638E6,1.08770256E2,2.1577182E0,1.2774899E-2,1.000254E-3,8.524E3,2.126E4,3.74449E2,2.5979605E0,9E0,6.109438E-1,3.7861453E5,-1.5748015E-3,-9.646353E-3,8.606702E-2,4.495078E-3,1.1890752E-1,3.0823356E5,1.131E3,-7.1861115E-3,2.714796E-4,1.0016339E6,4.3691156E5,2.993991E-3,7.7198017E-1,4.436289E-3,7.5644026E0,-1.7108666E-3,2.1967562E-3,2.8913823E-4,8.457265E-3,4.871581E-3,9.929357E-3,-2.4663352E-3,5.5238428E-3,-1.3903013E-3,-6.6634226E-3,-1.7853742E-4,-4.5104027E-3],"split_indices":[45,2,0,45,47,52,54,0,0,10,9,4,56,3,27,33,0,0,58,0,27,33,2,0,0,32,33,0,27,0,56,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,2E2,5E0,1.88E2,1.2E1,1.25E2,6.3E1,5E0,7E0,8.4E1,4.1E1,3.7E1,2.6E1,6.6E1,1.8E1,3.5E1,6E0,5E0,3.2E1,8E0,1.8E1,5.5E1,1.1E1,9E0,9E0,2.3E1,1.2E1,6E0,2.6E1,5E0,1.3E1,2.7E1,2.8E1,5E0,6E0,1.8E1,5E0,7E0,5E0,1.4E1,1.2E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.153934E-3,-1.0263918E-2,4.7890167E-3,-2.3694092E-3,1.3645916E-1,-1.1245125E-1,5.1354775E-3,2.8760004E-4,1.16978735E-2,-8.3417995E-3,-2.498098E-3,5.1209807E-2,-1.6877936E-2,7.125255E-2,-2.8810976E-3,-9.97591E-2,6.8517854E-3,4.7775857E-2,1.5654758E-1,-1.6337195E-1,3.9734445E-3,9.836672E-2,-1.2608458E-2,3.5357121E-3,-3.6180114E-5,2.9950335E-3,1.0754845E-2,-2.9826458E-3,-1.0000441E-2,1.9558158E-3,-1.9010629E-3,6.1042355E-3,9.119149E-4,-4.158144E-3,3.456223E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,-1,3,5,7,9,11,-1,-1,-1,-1,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.698895E-1,0E0,2.0108666E-1,1.682863E-1,1.5310387E-1,3.9269328E-2,1.9458671E-1,0E0,0E0,0E0,0E0,1.4265096E-1,2.5636062E-1,9.8650396E-2,0E0,1.9201213E-1,1.8289237E-1,5.4367177E-2,5.6239963E-2,6.9579124E-2,2.0924417E-2,3.905487E-2,1.2562314E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,-1,4,6,8,10,12,-1,-1,-1,-1,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,-1.0263918E-2,1.0086E4,4.75E2,1.3841E4,2.3135895E6,3.6628513E10,2.8760004E-4,1.16978735E-2,-8.3417995E-3,-2.498098E-3,8.801E3,1.4793115E2,1.43259E5,-2.8810976E-3,1.252E3,9.362688E-1,2.7661E4,1.6747E4,2.0261486E8,4.2342335E-2,4.330339E2,2.776338E2,3.5357121E-3,-3.6180114E-5,2.9950335E-3,1.0754845E-2,-2.9826458E-3,-1.0000441E-2,1.9558158E-3,-1.9010629E-3,6.1042355E-3,9.119149E-4,-4.158144E-3,3.456223E-4],"split_indices":[3,0,2,2,9,45,31,0,0,0,0,10,52,1,0,0,56,29,9,7,38,4,52,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,7E0,2.12E2,2.02E2,1E1,1.2E1,1.9E2,5E0,5E0,5E0,7E0,6.1E1,1.29E2,5.2E1,9E0,2.8E1,1.01E2,4.2E1,1E1,1.7E1,1.1E1,1.7E1,8.4E1,2.7E1,1.5E1,5E0,5E0,6E0,1.1E1,6E0,5E0,1.2E1,5E0,1.7E1,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.3912643E-3,1.2482385E-2,-1.2885857E-2,-2.2994645E-3,1.4408407E-1,6.4485692E-3,-6.8467176E-3,1.432078E-2,2.8852832E-2,-2.9761005E-2,3.392924E-2,-1.9287659E-3,5.0148093E-3,-1.3594067E-2,-9.578921E-3,1.2756962E-1,9.427939E-3,6.475554E-2,-4.049915E-2,2.6658962E-3,1.5648478E-1,1.2046006E-1,-9.803019E-3,1.4699479E-3,5.2243117E-3,-2.4634995E-4,-3.8443764E-3,9.624417E-3,4.9641915E-3,1.04865896E-4,1.0475376E-2,-3.639463E-3,5.749806E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,-1,-1,17,-1,19,21,23,25,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.345933E-1,3.6605078E-1,0E0,2.1003702E-1,3.3037305E-1,1.6121496E-1,0E0,0E0,6.709722E-2,1.8987414E-1,2.098739E-1,0E0,0E0,1.3964272E-1,0E0,3.0806392E-2,1.5918967E-1,2.2310205E-2,6.844584E-2,0E0,9.703428E-3,1.2727079E-1,9.324394E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,13,13,15,15,16,16,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,-1,-1,18,-1,20,22,24,26,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3798E4,1.00857574E3,-1.2885857E-2,1.00068504E8,1.3770975E2,2.8374527E2,-6.8467176E-3,1.432078E-2,1.0083192E6,5.3159845E6,1.021246E0,-1.9287659E-3,5.0148093E-3,2.77E2,-9.578921E-3,9.835643E5,1.9687062E8,1.7244057E2,1.9198604E2,2.6658962E-3,9.235021E-1,1.8094341E8,1.436E3,1.4699479E-3,5.2243117E-3,-2.4634995E-4,-3.8443764E-3,9.624417E-3,4.9641915E-3,1.04865896E-4,1.0475376E-2,-3.639463E-3,5.749806E-4],"split_indices":[10,52,0,45,58,4,0,0,28,28,56,0,0,0,0,28,7,4,55,0,27,7,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.87E2,6E0,1.69E2,1.8E1,1.6E2,9E0,7E0,1.1E1,6.9E1,9.1E1,6E0,5E0,6.4E1,5E0,1.8E1,7.3E1,1.6E1,4.8E1,6E0,1.2E1,1E1,6.3E1,1E1,6E0,2.6E1,2.2E1,5E0,7E0,5E0,5E0,1.5E1,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-2.0789208E-4,6.5857256E-3,-1.1020066E-2,-2.9858344E-3,1.3601506E-1,-1.0459375E-1,4.4517424E-3,1.2444372E-2,5.1462144E-4,2.241757E-5,-1.0886867E-2,-9.2982784E-2,1.2691934E-2,2.17591E-3,-8.380063E-3,-7.56861E-3,5.1213115E-2,6.1149478E-2,-2.5648465E-2,2.512527E-2,1.295454E-1,4.439571E-3,5.226114E-4,5.3619494E-4,-3.3160201E-3,2.216708E-3,-2.9820278E-3,4.3095765E-3,8.017013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2626164E-1,2.5189075E-1,0E0,1.4405179E-1,2.1422595E-1,1.6762184E-1,1.4409262E-1,0E0,0E0,0E0,0E0,1.667046E-1,1.2916353E-1,0E0,0E0,1.3699141E-1,1.1322582E-1,3.54101E-2,1.4041404E-1,8.416517E-2,5.8957636E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,1.4056875E3,-1.1020066E-2,2.2495176E4,1E0,4.8206668E2,1.2106E4,1.2444372E-2,5.1462144E-4,2.241757E-5,-1.0886867E-2,1.8035231E0,1.631E3,2.17591E-3,-8.380063E-3,1.37E2,1.5819151E6,6.805556E5,6.769573E6,7.0208377E-1,7.7329254E2,4.439571E-3,5.226114E-4,5.3619494E-4,-3.3160201E-3,2.216708E-3,-2.9820278E-3,4.3095765E-3,8.017013E-3],"split_indices":[45,52,0,33,64,48,9,0,0,0,0,54,2,0,0,10,28,28,45,27,52,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,2.02E2,5E0,1.89E2,1.3E1,1.2E1,1.77E2,6E0,7E0,7E0,5E0,1.3E1,1.64E2,5E0,8E0,1.08E2,5.6E1,2.2E1,8.6E1,4.3E1,1.3E1,1.3E1,9E0,4.7E1,3.9E1,3.5E1,8E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.387268E-3,4.6030553E-3,-1.1797118E-2,1.276042E-2,-8.010899E-3,3.2018356E-3,1.4192265E-1,-7.401338E-3,9.638315E-3,1.0102898E-2,3.4716537E-3,4.733129E-2,-1.1162673E-2,2.228997E-2,8.279184E-3,9.022479E-2,-2.4363425E-2,4.599769E-3,4.1625256E-4,1.0714047E-3,6.9561317E-3,4.6151816E-3,-1.8687742E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,-1,13,15,17,-1,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[5.415336E-1,2.7824238E-1,0E0,2.3288654E-1,0E0,1.8114002E-1,4.6983927E-2,0E0,1.3460109E-1,0E0,0E0,1.8782389E-1,1.5075955E-1,5.1307492E-2,0E0,4.5248806E-2,1.7873728E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,11,11,12,12,13,13,15,15,16,16],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,-1,14,16,18,-1,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,4.7379535E6,-1.1797118E-2,9.248443E8,-8.010899E-3,9.235376E5,1E0,-7.401338E-3,3.9423492E10,1.0102898E-2,3.4716537E-3,3.843097E2,1.4170854E0,1.1280869E9,8.279184E-3,1.6497E4,3.4246575E-3,4.599769E-3,4.1625256E-4,1.0714047E-3,6.9561317E-3,4.6151816E-3,-1.8687742E-3],"split_indices":[45,28,0,7,0,45,75,0,31,0,0,4,53,31,0,9,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.96E2,8E0,1.88E2,8E0,1.76E2,1.2E1,6E0,1.7E2,5E0,7E0,6E1,1.1E2,5.1E1,9E0,1.2E1,9.8E1,7E0,4.4E1,6E0,6E0,1E1,8.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.7694116E-2,-9.864165E-3,-1.3022849E-2,-3.1359454E-3,-9.200282E-3,-8.599557E-3,2.5700845E-3,1.4138242E-2,-6.607715E-2,-2.258918E-3,6.1785094E-2,-1.1990333E-1,2.311579E-2,3.58232E-2,-1.9006291E-2,1.5279996E-2,1.0985145E-1,-7.192368E-3,-1.8909773E-3,3.2761593E-3,-1.2458544E-3,-2.9154273E-3,3.3090992E-3,2.3607996E-3,-1.8385185E-3,3.1253954E-3,-1.5228739E-3,3.000021E-3,7.225577E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,-1,5,-1,-1,7,9,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9927018E-1,2.4086349E-1,0E0,1.9267225E-1,0E0,0E0,1.4864092E-1,1.2514326E-1,1.323964E-1,7.721392E-2,8.997001E-2,3.540173E-2,2.6237965E-2,1.2313458E-1,1.1299104E-1,5.3803127E-2,2.9256493E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,-1,6,-1,-1,8,10,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,4.476758E6,-1.3022849E-2,1.0058E4,-9.200282E-3,-8.599557E-3,3.10364E5,7.5614764E2,6.069098E5,1E0,5E0,1.4435129E6,2.1704407E11,8.960506E3,2.0304577E-1,1.8933852E0,7.7186523E2,-7.192368E-3,-1.8909773E-3,3.2761593E-3,-1.2458544E-3,-2.9154273E-3,3.3090992E-3,2.3607996E-3,-1.8385185E-3,3.1253954E-3,-1.5228739E-3,3.000021E-3,7.225577E-3],"split_indices":[45,28,0,9,0,0,9,4,28,104,8,32,31,33,27,54,52,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.96E2,5E0,1.9E2,6E0,5E0,1.85E2,1.59E2,2.6E1,1.19E2,4E1,1.6E1,1E1,3.6E1,8.3E1,2.1E1,1.9E1,1.1E1,5E0,5E0,5E0,9E0,2.7E1,1.8E1,6.5E1,1E1,1.1E1,1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.749581E-4,-1.2903669E-2,1.6795477E-1,-2.0537914E-4,-1.752457E-1,7.171186E-4,1.345378E-2,9.025887E-3,-9.572364E-3,-1.3354766E-2,-1.5288711E-3,1.9184079E-2,-5.8057286E-2,3.1810097E-2,-3.2801203E-2,-1.2144796E-1,1.0171495E-2,-1.4284042E-2,4.9781516E-2,3.7225805E-2,-9.640145E-2,-7.645547E-3,-2.7169352E-3,2.1551186E-3,-1.5397061E-3,2.0880892E-3,-2.4174824E-3,4.0970114E-3,6.579349E-4,-1.5400983E-6,3.8236082E-3,-1.3475748E-3,-7.7208444E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2696482E-1,3.97827E-1,2.3753917E-1,3.3346507E-1,1.9881603E-1,0E0,0E0,1.1900355E-1,0E0,0E0,0E0,1.0015486E-1,1.0071187E-1,1.017731E-1,1.3720067E-1,2.057764E-2,1.9120872E-2,7.5472E-2,1.1218977E-1,2.3802074E-2,6.6205904E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4056875E3,1.00068504E8,1.376E4,4.674595E6,6.0268195E2,7.171186E-4,1.345378E-2,3.10364E5,-9.572364E-3,-1.3354766E-2,-1.5288711E-3,9.235021E-1,1.569E3,8.92E2,2.493493E7,4.7861097E5,3.328384E5,1.5139E4,1.8352579E0,1.024E3,9.4636065E-1,-7.645547E-3,-2.7169352E-3,2.1551186E-3,-1.5397061E-3,2.0880892E-3,-2.4174824E-3,4.0970114E-3,6.579349E-4,-1.5400983E-6,3.8236082E-3,-1.3475748E-3,-7.7208444E-3],"split_indices":[52,45,9,28,52,0,0,9,0,0,0,27,0,2,45,28,33,9,56,2,27,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.92E2,1.3E1,1.79E2,1.3E1,6E0,7E0,1.72E2,7E0,7E0,6E0,1.5E2,2.2E1,1.21E2,2.9E1,1.1E1,1.1E1,3.4E1,8.7E1,1.4E1,1.5E1,6E0,5E0,6E0,5E0,1.3E1,2.1E1,4.3E1,4.4E1,8E0,6E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.8531388E-3,1.2359911E-1,-6.852457E-3,1.7630536E-3,1.1172085E-2,1.1536126E-2,-7.254988E-2,-8.177492E-3,5.7998043E-2,-1.0339157E-1,3.6078074E-3,7.435079E-2,-2.2137865E-2,9.937891E-3,3.0396534E-2,-2.4012681E-2,-1.8143643E-1,-9.1053837E-4,6.336613E-3,-6.334781E-2,2.2240942E-2,4.3656616E-3,5.7285605E-3,-6.363419E-3,2.4336314E-2,-2.4333574E-1,-2.3668564E-3,8.905296E-3,-8.702193E-2,5.7012364E-2,-8.102912E-2,4.8900854E-2,-2.1290908E-2,-4.552355E-4,3.4033153E-3,-6.398044E-3,-1.5851235E-2,3.8093245E-3,-2.1637066E-3,-1.9615754E-3,-6.390239E-3,5.7167215E-3,1.9165799E-3,-5.9253643E-3,-8.5101504E-4,6.113963E-4,3.972221E-3,2.8642567E-4,-3.1787762E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,-1,-1,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,-1,27,29,31,-1,-1,33,35,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2236003E-1,1.2459944E-1,2.3762111E-1,0E0,0E0,1.4152789E-1,2.0387211E-1,1.2722766E-1,1.8343085E-1,2.1866739E-1,0E0,8.902116E-2,1.7332064E-1,0E0,9.2332E-2,1.0319072E-1,1.3508731E-1,0E0,0E0,8.459088E-2,1.6979861E-1,3.760499E-2,0E0,0E0,2.279948E-2,7.113087E-2,0E0,5.3220633E-2,7.358116E-2,3.2318838E-2,3.018403E-2,1.3691863E-2,2.6200928E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,19,19,20,20,21,21,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32],"right_children":[2,4,6,-1,-1,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,-1,28,30,32,-1,-1,34,36,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.390852E-2,4.8444237E8,8.50999E9,1.7630536E-3,1.1172085E-2,1.996E3,2.783671E12,1.3252E4,2.3068698E8,6.792816E7,3.6078074E-3,1.9599061E5,1.7206225E0,9.937891E-3,2.1936802E3,1.5023475E-2,1E0,-9.1053837E-4,6.336613E-3,4.45E2,2.1845032E7,2.46604E5,5.7285605E-3,-6.363419E-3,2.3E1,1E0,-2.3668564E-3,1.7829868E2,1.6844329E0,8.9281055E4,5.233944E-5,2.539E3,8.091679E-1,-4.552355E-4,3.4033153E-3,-6.398044E-3,-1.5851235E-2,3.8093245E-3,-2.1637066E-3,-1.9615754E-3,-6.390239E-3,5.7167215E-3,1.9165799E-3,-5.9253643E-3,-8.5101504E-4,6.113963E-4,3.972221E-3,2.8642567E-4,-3.1787762E-3],"split_indices":[27,7,5,0,0,2,31,9,7,45,0,32,53,0,4,57,16,0,0,10,45,29,0,0,8,101,0,4,54,33,38,2,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.3E1,1.95E2,8E0,5E0,1.53E2,4.2E1,1.08E2,4.5E1,3.5E1,7E0,1.5E1,9.3E1,6E0,3.9E1,1.8E1,1.7E1,6E0,9E0,4.8E1,4.5E1,3.1E1,8E0,5E0,1.3E1,1.1E1,6E0,1.2E1,3.6E1,3.4E1,1.1E1,1.1E1,2E1,8E0,5E0,6E0,5E0,5E0,7E0,1.9E1,1.7E1,6E0,2.8E1,6E0,5E0,6E0,5E0,1.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[9.312813E-3,2.0384718E-2,-1.00693405E-1,4.210924E-2,-8.657957E-3,-2.2129597E-1,3.7235224E-3,-4.8295617E-2,5.7040647E-2,-5.578887E-3,4.4489536E-4,-4.60687E-3,-1.4829988E-2,-4.066287E-3,2.7002086E-4,-5.723057E-2,7.242452E-2,4.271354E-3,-1.2037168E-2,6.5350096E-4,-5.680173E-3,1.464994E-1,4.6086866E-2,8.071656E-3,-6.0088154E-2,9.2303686E-2,1.0360417E-2,8.00706E-3,1.0976622E-1,-4.6853465E-3,2.3367928E-2,3.6239537E-4,-9.189426E-2,1.5637908E-3,6.64871E-3,1.4179124E-3,-3.10671E-3,7.700402E-3,2.1169956E-3,3.0918987E-3,-1.1060754E-3,-2.1713246E-3,-6.643653E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,-1,-1,19,21,-1,23,-1,-1,25,27,29,31,33,-1,35,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3955838E-1,1.1267159E-1,4.045115E-1,1.4044172E-1,7.560481E-2,9.1713846E-2,0E0,3.0071411E-2,1.5884316E-1,0E0,8.079935E-2,0E0,0E0,0E0,0E0,4.943132E-2,1.4542308E-1,0E0,6.2324934E-2,0E0,0E0,5.564791E-2,1.4219007E-1,7.677835E-2,4.1578606E-2,3.151743E-2,0E0,6.16882E-2,6.642634E-2,0E0,7.990484E-2,0E0,2.2037104E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,30,30,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,-1,-1,20,22,-1,24,-1,-1,26,28,30,32,34,-1,36,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.472477E7,4.9538516E5,4.8314605E0,1.2904155E3,4.910112E4,4E0,3.7235224E-3,9.123214E-1,5.94E2,-5.578887E-3,5.732505E6,-4.60687E-3,-1.4829988E-2,-4.066287E-3,2.7002086E-4,1.5605E4,1.151349E7,4.271354E-3,3.73802E5,6.5350096E-4,-5.680173E-3,4.0480963E2,4.3935942E8,9.06E2,1.6810659E6,1.5064E4,1.0360417E-2,2.136E4,6.234292E-1,-4.6853465E-3,4.530014E8,3.6239537E-4,4.5360103E5,1.5637908E-3,6.64871E-3,1.4179124E-3,-3.10671E-3,7.700402E-3,2.1169956E-3,3.0918987E-3,-1.1060754E-3,-2.1713246E-3,-6.643653E-3],"split_indices":[45,32,56,48,48,8,0,27,2,0,12,0,0,0,0,1,12,0,33,0,0,4,7,2,50,9,0,9,27,0,7,0,33,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.94E2,1.77E2,1.7E1,1.01E2,7.6E1,1E1,7E0,1.4E1,8.7E1,5E0,7.1E1,5E0,5E0,8E0,6E0,1E1,7.7E1,8E0,6.3E1,5E0,5E0,1.9E1,5.8E1,4.5E1,1.8E1,1.2E1,7E0,3.7E1,2.1E1,5E0,4E1,6E0,1.2E1,6E0,6E0,2.9E1,8E0,1.1E1,1E1,2.1E1,1.9E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[6.8093715E-3,1.4430599E-2,-8.796921E-3,1.1017902E-3,1.5706244E-1,3.1792246E-2,-3.096493E-2,1.4386229E-2,5.164142E-2,7.031274E-2,-6.7262264E-4,-5.159023E-2,6.0637258E-2,-1.4836404E-4,4.68413E-3,4.551819E-2,7.070452E-3,4.0215356E-3,-2.0334868E-2,2.3187757E-2,-9.021548E-2,6.7608804E-3,3.907748E-3,8.123324E-2,-3.6543574E-2,-4.7605936E-3,-2.2069146E-3,-3.0156644E-3,5.2175377E-2,-9.720595E-3,-6.977348E-2,1.9852205E-3,-1.6419952E-3,2.2047295E-3,8.585354E-3,-4.591934E-3,1.3822409E-3,-3.860378E-3,6.243871E-4,-2.2294065E-4,3.682378E-3,-5.7999506E-3,-4.923137E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,19,21,-1,-1,23,-1,-1,25,27,29,-1,31,33,35,-1,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0161846E-1,3.7814838E-1,0E0,1.8108249E-1,2.4117297E-1,1.1760017E-1,1.7316732E-1,0E0,2.8079426E-2,7.56651E-2,8.822035E-2,2.1582748E-1,7.706735E-2,0E0,0E0,1.02617115E-1,0E0,0E0,6.1387807E-2,6.82018E-2,9.87142E-2,0E0,1.718273E-2,7.4550316E-2,4.5426212E-2,0E0,4.4047542E-2,0E0,2.8417062E-2,0E0,1.2997551E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,15,15,18,18,19,19,20,20,22,22,23,23,24,24,26,26,28,28,30,30],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,20,22,-1,-1,24,-1,-1,26,28,30,-1,32,34,36,-1,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6030095E10,1.3397336E3,-8.796921E-3,6.781768E-1,6.234292E-1,1.933162E0,2.225675E0,1.4386229E-2,4E0,5.49E2,1.2414E4,1.453034E0,2.6486957E0,-1.4836404E-4,4.68413E-3,1.8556E5,7.070452E-3,4.0215356E-3,7.308952E1,1E0,1.2106E4,6.7608804E-3,1.01E3,1.33807E5,2.34958E5,-4.7605936E-3,2.7573213E8,-3.0156644E-3,7.794276E-1,-9.720595E-3,8E0,1.9852205E-3,-1.6419952E-3,2.2047295E-3,8.585354E-3,-4.591934E-3,1.3822409E-3,-3.860378E-3,6.243871E-4,-2.2294065E-4,3.682378E-3,-5.7999506E-3,-4.923137E-4],"split_indices":[5,52,0,27,27,58,54,0,8,0,9,53,54,0,0,1,0,0,52,100,9,0,2,1,1,0,7,0,27,0,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.98E2,7E0,1.82E2,1.6E1,9.3E1,8.9E1,6E0,1E1,4.2E1,5.1E1,7.3E1,1.6E1,5E0,5E0,3.3E1,9E0,9E0,4.2E1,2.5E1,4.8E1,6E0,1E1,2.3E1,1E1,7E0,3.5E1,6E0,1.9E1,6E0,4.2E1,5E0,5E0,1.8E1,5E0,5E0,5E0,5E0,3E1,6E0,1.3E1,2.2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[7.7712657E-3,1.4626645E-2,-8.783076E-3,2.0215094E-2,-7.84965E-3,1.4291962E-2,8.24861E-3,1.9547911E-2,-5.7801497E-3,-7.612793E-2,2.6967537E-2,-1.3829741E-3,-4.8894086E-3,3.528419E-2,-3.8796455E-2,5.525904E-4,3.2854828E-3,-3.721949E-3,-4.6160037E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,-1,7,-1,9,-1,11,13,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[2.6469702E-1,1.9680634E-1,0E0,1.6765395E-1,0E0,1.3215105E-1,0E0,1.2770154E-1,0E0,1.2984835E-2,9.1613404E-2,0E0,0E0,1.1544715E-1,2.0800486E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,9,9,10,10,13,13,14,14],"right_children":[2,4,-1,6,-1,8,-1,10,-1,12,14,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[4.7379535E6,1.0953058E10,-8.783076E-3,4.056359E6,-7.84965E-3,6.3086E4,8.24861E-3,8.960506E3,-5.7801497E-3,8.601996E-1,5.327338E6,-1.3829741E-3,-4.8894086E-3,3.7597818E2,1.5947007E0,5.525904E-4,3.2854828E-3,-3.721949E-3,-4.6160037E-4],"split_indices":[28,12,0,29,0,10,0,33,0,27,47,0,0,55,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.93E2,6E0,1.88E2,5E0,1.82E2,6E0,1.76E2,6E0,1.2E1,1.64E2,5E0,7E0,1.46E2,1.8E1,8.6E1,6E1,7E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-5.510306E-3,5.0003547E-3,-1.1848575E-1,-5.0059482E-3,9.7427145E-2,1.6208075E-3,-1.2878792E-2,1.2181632E-2,-4.7371227E-2,2.1482715E-2,1.0757752E-2,4.1712616E-3,-4.028903E-3,-4.9234265E-3,6.475918E-2,-1.0385798E-2,-1.2762773E-1,-1.0091404E-3,3.575456E-3,-3.2539263E-2,2.0081514E-2,1.03368156E-1,3.3491964E-3,-4.701882E-2,4.2907633E-3,-9.834811E-3,-7.3667295E-2,-1.1070125E-2,-5.7284604E-3,1.0911348E-1,-2.0046068E-2,2.3026217E-3,1.316242E-1,1.8622361E-3,-2.13533E-3,-9.007259E-2,2.3793258E-2,-1.5322553E-3,-4.938087E-3,1.8956792E-3,-2.0245223E-3,3.3456804E-3,6.6909995E-3,1.8176496E-3,-3.2736745E-3,7.6239016E-3,3.717241E-3,-6.6060983E-3,-9.1223663E-4,-8.047626E-4,2.8945708E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,-1,19,21,23,25,-1,-1,27,29,31,33,35,-1,-1,37,39,-1,41,43,-1,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.511437E-1,1.7990871E-1,3.2544938E-1,1.2873262E-1,1.786053E-1,8.7888554E-2,0E0,1.1363593E-1,1.5005887E-1,3.096883E-2,0E0,0E0,0E0,6.694455E-2,7.315409E-2,1.3602105E-1,5.1923066E-2,0E0,0E0,8.4627435E-2,1.8439181E-1,2.336508E-2,2.386123E-2,8.445874E-2,0E0,0E0,1.0186981E-2,6.1686862E-2,0E0,9.445384E-3,1.033897E-1,0E0,5.490765E-3,0E0,0E0,5.5115893E-2,1.7368406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,26,26,27,27,29,29,30,30,32,32,35,35,36,36],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,-1,20,22,24,26,-1,-1,28,30,32,34,36,-1,-1,38,40,-1,42,44,-1,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,5.666E3,1.0129378E12,8.5913794E5,2.605017E5,9.846896E4,-1.2878792E-2,2.225675E0,8.19E2,1.3841E4,1.0757752E-2,4.1712616E-3,-4.028903E-3,2.8003845E2,5.9351373E0,7.32E2,5E0,-1.0091404E-3,3.575456E-3,2.010204E0,1.28831E5,4E0,3.6907338E-2,7.830236E-1,4.2907633E-3,-9.834811E-3,6.894317E-1,7.3328E4,-5.7284604E-3,1.117E3,5.3707652E7,2.3026217E-3,5.606407E5,1.8622361E-3,-2.13533E-3,3.0050538E5,9.8340225E1,-1.5322553E-3,-4.938087E-3,1.8956792E-3,-2.0245223E-3,3.3456804E-3,6.6909995E-3,1.8176496E-3,-3.2736745E-3,7.6239016E-3,3.717241E-3,-6.6060983E-3,-9.1223663E-4,-8.047626E-4,2.8945708E-3],"split_indices":[45,2,31,32,33,33,0,54,0,9,0,0,0,4,57,10,8,0,0,54,29,8,38,27,0,0,27,1,0,2,12,0,28,0,0,33,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.93E2,1.7E1,1.75E2,1.8E1,1E1,7E0,1.25E2,5E1,1.2E1,6E0,5E0,5E0,9.5E1,3E1,3.5E1,1.5E1,7E0,5E0,4.5E1,5E1,1.8E1,1.2E1,2.6E1,9E0,5E0,1E1,3.7E1,8E0,1.5E1,3.5E1,7E0,1.1E1,7E0,5E0,1.6E1,1E1,5E0,5E0,1.4E1,2.3E1,8E0,7E0,1.6E1,1.9E1,6E0,5E0,9E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-1.1704756E-3,1.423016E-2,-7.418005E-2,3.464207E-3,1.4067067E-1,-2.1076834E-2,-1.2013915E-2,-7.474051E-2,1.3713993E-2,1.3681324E-2,6.9036474E-4,2.6308002E-3,-5.9962284E-2,-1.4676923E-2,-7.0897327E-3,2.129113E-2,-4.6191015E-3,-1.0574755E-1,1.274341E-4,-2.3660313E-3,1.0769286E-3,-1.025916E-3,8.3677754E-2,-6.6958177E-3,-2.5921953E-3,-1.2871689E-2,6.927588E-2,1.0762742E-1,-1.1533889E-3,8.907732E-4,-2.6094443E-3,6.658708E-3,-1.8308598E-5,1.5578158E-3,7.3930137E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,17,19,-1,21,-1,23,-1,-1,-1,25,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2593437E-1,2.2548062E-1,3.229246E-1,1.2446856E-1,2.337256E-1,8.603012E-2,0E0,7.701631E-2,1.16374545E-1,0E0,0E0,0E0,5.4757625E-2,1.5296519E-2,0E0,1.7969061E-1,0E0,1.1760026E-2,0E0,0E0,0E0,8.0838494E-2,9.033291E-2,0E0,0E0,1.09643884E-1,6.803774E-2,9.31727E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,12,12,13,13,15,15,17,17,21,21,22,22,25,25,26,26,27,27],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,18,20,-1,22,-1,24,-1,-1,-1,26,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.10364E5,1.4056875E3,5.8018835E2,5.643341E-2,3.6007138E2,6.5229E4,-1.2013915E-2,2.1600779E8,1.6565532E3,1.3681324E-2,6.9036474E-4,2.6308002E-3,2.3361E4,5.4166875E3,-7.0897327E-3,2.001E3,-4.6191015E-3,4.7647306E5,1.274341E-4,-2.3660313E-3,1.0769286E-3,7.9201184E8,6.1037578E-2,-6.6958177E-3,-2.5921953E-3,5.14E2,1.2226295E6,4.2378342E-1,-1.1533889E-3,8.907732E-4,-2.6094443E-3,6.658708E-3,-1.8308598E-5,1.5578158E-3,7.3930137E-3],"split_indices":[9,52,52,58,58,29,0,7,4,0,0,0,10,33,0,2,0,32,0,0,0,12,38,0,0,0,47,27,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.65E2,3.4E1,1.53E2,1.2E1,2.7E1,7E0,1.7E1,1.36E2,5E0,7E0,9E0,1.8E1,1E1,7E0,1.28E2,8E0,1E1,8E0,5E0,5E0,9.5E1,3.3E1,5E0,5E0,8.2E1,1.3E1,2.7E1,6E0,4.7E1,3.5E1,6E0,7E0,1.1E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.06563E-3,1.08985186E-1,-2.0713164E-3,6.860201E-4,1.0398121E-2,4.7928747E-3,-7.609318E-3,-2.401882E-4,7.3447437E-3,5.3998614E-3,-5.114108E-3,-1.753003E-2,3.2034673E-2,-3.0969437E-2,7.46468E-2,7.3104566E-3,2.1826807E-2,1.3890924E-3,-2.866995E-3,2.3294083E-4,5.930972E-3,2.7797022E-3,-3.416525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,-1,-1,7,-1,9,-1,11,-1,13,15,17,19,-1,21,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5004274E-1,1.2987271E-1,2.0520635E-1,0E0,0E0,1.3682699E-1,0E0,1.0750245E-1,0E0,1.0438623E-1,0E0,1.1628154E-1,9.3889885E-2,1.4197868E-1,4.0558286E-2,0E0,7.797508E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,9,9,11,11,12,12,13,13,14,14,16,16],"right_children":[2,4,6,-1,-1,8,-1,10,-1,12,-1,14,16,18,20,-1,22,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6604617E-2,5.7669523E8,7.5875E4,6.860201E-4,1.0398121E-2,1.1195705E7,-7.609318E-3,4.674595E6,7.3447437E-3,1.7311496E0,-5.114108E-3,1.5819151E6,1.9557953E0,1.3614458E0,6.8915665E-1,7.3104566E-3,1.3549481E2,1.3890924E-3,-2.866995E-3,2.3294083E-4,5.930972E-3,2.7797022E-3,-3.416525E-4],"split_indices":[27,7,10,0,0,1,0,28,0,53,0,28,54,53,56,0,52,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.2E1,1.89E2,7E0,5E0,1.82E2,7E0,1.77E2,5E0,1.69E2,8E0,9.1E1,7.8E1,8E1,1.1E1,5E0,7.3E1,2.6E1,5.4E1,5E0,6E0,3.2E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-7.1378117E-3,-5.48326E-4,-1.1386778E-2,4.746688E-3,-6.491706E-3,-2.0689298E-3,7.984019E-2,-1.803695E-2,2.9150937E-2,4.3914295E-4,6.785877E-3,2.5732996E-2,-4.0495634E-2,5.5968378E-2,-1.7876811E-2,1.0987985E-2,5.5012773E-3,-8.769598E-2,-1.7427126E-2,7.829422E-2,1.2050634E-2,-5.6626904E-3,3.0384941E-2,4.73473E-3,-2.546839E-4,3.833993E-4,-5.529559E-3,1.0059337E-3,-3.4079712E-3,1.9049768E-4,4.5514796E-3,-6.503391E-4,2.527213E-3,3.9520497E-3,-2.675112E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,17,19,21,23,-1,25,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.280048E-1,1.5142064E-1,0E0,1.04574144E-1,0E0,9.424745E-2,7.295491E-2,1.23918995E-1,8.171502E-2,0E0,0E0,5.5228427E-2,8.8845566E-2,3.9257035E-2,1.19792186E-1,5.4947596E-2,0E0,7.25577E-2,1.19123966E-1,3.315164E-2,1.636046E-2,0E0,8.0984175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,18,20,22,24,-1,26,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,4.7379535E6,-1.1386778E-2,8.853438E8,-6.491706E-3,2.173763E0,1.3841E4,5.2139695E6,8.3268556E5,4.3914295E-4,6.785877E-3,1.9234526E1,1.4693E4,5.964497E0,2.9307398E-1,4.0748124E7,5.5012773E-3,7.492958E-1,1.7476E4,9.090909E-2,2.5831E4,-5.6626904E-3,6.5229E4,4.73473E-3,-2.546839E-4,3.833993E-4,-5.529559E-3,1.0059337E-3,-3.4079712E-3,1.9049768E-4,4.5514796E-3,-6.503391E-4,2.527213E-3,3.9520497E-3,-2.675112E-3],"split_indices":[45,28,0,7,0,54,9,45,32,0,0,58,9,57,27,5,0,56,9,58,10,0,29,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.1E2,5E0,2.03E2,7E0,1.87E2,1.6E1,1.24E2,6.3E1,8E0,8E0,4.2E1,8.2E1,4E1,2.3E1,3.7E1,5E0,2.6E1,5.6E1,2.6E1,1.4E1,7E0,1.6E1,5E0,3.2E1,6E0,2E1,3.3E1,2.3E1,5E0,2.1E1,9E0,5E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.3128191E-3,-6.524628E-3,7.5661866E-3,1.8436778E-2,-4.6637483E-2,6.9907054E-2,-6.0014296E-3,-3.093164E-2,-9.547033E-3,3.8563076E-2,9.124745E-3,-6.3883595E-2,2.0588739E-2,-6.1520715E-3,-1.65147E-2,5.6605645E-2,-1.7505082E-3,-9.986308E-2,-1.7454684E-2,-7.609727E-3,6.835933E-2,-3.8818546E-2,4.1401498E-2,3.7190385E-2,5.670462E-3,-5.752283E-3,-2.3735466E-3,1.0435028E-3,-2.1414924E-3,3.732331E-3,-2.2927383E-2,5.953668E-3,3.9625045E-2,1.0686785E-2,-7.73106E-2,-1.8302911E-3,8.438793E-2,-2.0623507E-3,3.033656E-3,-5.1741805E-3,4.1280012E-4,-6.118685E-7,5.0634416E-3,2.5249694E-3,-1.9547925E-3,-5.023662E-3,1.532921E-3,1.6936888E-3,5.478927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,7,9,11,13,-1,15,-1,17,19,-1,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5757432E-1,2.011145E-1,0E0,1.5639186E-1,1.7843074E-1,1.4380273E-1,1.321385E-1,9.766185E-2,0E0,4.6784256E-2,0E0,4.3360107E-2,7.999657E-2,0E0,8.2987644E-2,2.7970836E-2,0E0,9.302333E-3,1.4870444E-2,5.1562198E-2,3.042119E-2,8.8482045E-2,6.481919E-2,4.8854634E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.948839E-2,0E0,4.1900203E-2,4.7583066E-2,8.165015E-2,0E0,1.3594806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,30,30,32,32,33,33,34,34,36,36],"right_children":[2,4,-1,6,8,10,12,14,-1,16,-1,18,20,-1,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,8.111963E-1,7.5661866E-3,3.74E2,2.68225E5,1.594E3,1E0,1.1334E4,-9.547033E-3,1.8267011E5,9.124745E-3,1.4065148E6,5.247412E5,-6.1520715E-3,2.192884E0,6.1895325E5,-1.7505082E-3,1.18107956E2,7.476915E5,5.7573294E5,8.8317425E5,1E0,1.0860942E6,1.763677E8,5.670462E-3,-5.752283E-3,-2.3735466E-3,1.0435028E-3,-2.1414924E-3,3.732331E-3,1.2086619E6,5.953668E-3,1.13728695E1,2E0,1.875E3,-1.8302911E-3,1.4768839E5,-2.0623507E-3,3.033656E-3,-5.1741805E-3,4.1280012E-4,-6.118685E-7,5.0634416E-3,2.5249694E-3,-1.9547925E-3,-5.023662E-3,1.532921E-3,1.6936888E-3,5.478927E-3],"split_indices":[2,27,0,0,9,2,8,9,0,47,0,50,28,0,54,28,0,52,47,51,28,104,28,7,0,0,0,0,0,0,48,0,57,8,2,0,32,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.99E2,9E0,1.23E2,7.6E1,3.9E1,8.4E1,7E1,6E0,3.2E1,7E0,2.6E1,5.8E1,8E0,6.2E1,2.6E1,6E0,1.4E1,1.2E1,3.7E1,2.1E1,4.5E1,1.7E1,2.1E1,5E0,9E0,5E0,5E0,7E0,5E0,3.2E1,6E0,1.5E1,2E1,2.5E1,6E0,1.1E1,5E0,1.6E1,8E0,2.4E1,1E1,5E0,1.1E1,9E0,2E1,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.1399253E-3,-1.0319589E-2,1.1769322E-1,-9.930938E-2,-1.8346643E-3,1.2467294E-2,2.9245282E-2,-2.3859425E-4,-9.2503885E-3,-1.0813295E-1,4.6715345E-3,-1.5202757E-3,4.0889457E-3,-2.3818857E-3,-7.115642E-3,-6.5294735E-2,1.3825411E-2,-9.5653646E-2,1.2717176E-3,8.832767E-2,4.3508234E-3,-1.9361881E-3,-7.7085476E-3,1.2049634E-1,9.851455E-4,-4.324805E-2,1.7911656E-2,1.8467655E-3,9.390998E-3,1.6584299E-3,-3.6037588E-3,1.2947698E-3,-2.8885487E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,-1,-1,-1,17,19,21,-1,23,25,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4338289E-1,1.5611765E-1,1.9410592E-1,1.582626E-1,1.3190873E-1,0E0,4.0340826E-2,0E0,0E0,1.8571094E-2,1.1618599E-1,0E0,0E0,0E0,0E0,6.1090223E-2,1.125562E-1,5.116552E-2,0E0,3.676033E-2,9.307833E-2,0E0,0E0,6.671253E-2,0E0,8.2272455E-2,8.146141E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,15,15,16,16,17,17,19,19,20,20,23,23,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,-1,-1,-1,18,20,22,-1,24,26,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3397336E3,7E0,1.3770975E2,3.10364E5,7.999278E1,1.2467294E-2,7.304467E7,-2.3859425E-4,-9.2503885E-3,1.4596E4,8.186529E-2,-1.5202757E-3,4.0889457E-3,-2.3818857E-3,-7.115642E-3,1.4E1,1.2345679E-2,2.0713173E8,1.2717176E-3,9.0763354E-1,2.3184108E5,-1.9361881E-3,-7.7085476E-3,4.5701938E5,9.851455E-4,3.08E2,3.6777365E6,1.8467655E-3,9.390998E-3,1.6584299E-3,-3.6037588E-3,1.2947698E-3,-2.8885487E-3],"split_indices":[52,3,58,9,55,0,48,0,0,9,58,0,0,0,0,8,57,7,0,27,33,0,0,28,0,0,47,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.21E2,2.06E2,1.5E1,1.7E1,1.89E2,5E0,1E1,9E0,8E0,1E1,1.79E2,5E0,5E0,5E0,5E0,2E1,1.59E2,1.5E1,5E0,1.7E1,1.42E2,9E0,6E0,1.1E1,6E0,3.1E1,1.11E2,6E0,5E0,9E0,2.2E1,1E2,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.6466057E-3,2.6532562E-2,-5.2822184E-2,1.8290184E-2,7.134139E-3,-9.416979E-3,-1.4906937E-1,3.1907268E-2,-2.2711808E-2,-2.3482192E-2,3.8488703E-3,-2.5267503E-1,-2.2493258E-2,4.329997E-3,8.108087E-2,2.6188273E-2,-6.452295E-2,-1.0754311E-2,-4.901009E-3,-1.8350847E-2,-2.801159E-3,-4.9708397E-3,2.995213E-3,-1.4986211E-2,6.886798E-2,9.780521E-3,4.6170875E-2,-9.432969E-4,2.447933E-3,-1.0898385E-1,1.6841067E-3,4.725242E-3,-2.9741282E-2,2.839944E-3,-3.1740006E-2,9.657583E-4,4.9288245E-3,6.7162156E-2,-1.844102E-3,-2.725091E-3,-7.263197E-3,-4.4702613E-3,-6.593305E-3,-2.9105484E-3,1.8492938E-3,5.987247E-4,4.6713767E-3,2.226526E-3,-1.3697336E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,-1,37,-1,-1,39,-1,-1,41,-1,43,-1,-1,45,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8837648E-1,1.2739864E-1,2.9520786E-1,6.884062E-2,0E0,6.500782E-2,2.828245E-1,1.24407694E-1,6.507135E-2,4.303976E-2,0E0,2.8141755E-1,8.2480304E-2,7.582982E-2,1.3235432E-1,1.8146899E-2,7.916945E-2,8.3932266E-2,0E0,0E0,0E0,0E0,0E0,6.0359813E-2,2.1120228E-2,0E0,5.0569315E-2,0E0,0E0,1.802516E-2,0E0,0E0,4.9414266E-2,0E0,8.19345E-2,0E0,0E0,3.5274424E-2,0E0,0E0,0E0,0E0,3.1680077E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,23,23,24,24,26,26,29,29,32,32,34,34,37,37,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,-1,38,-1,-1,40,-1,-1,42,-1,44,-1,-1,46,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1],"split_conditions":[8.601996E-1,2.1374558E3,7.625922E11,4.7564573E0,7.134139E-3,7.7247575E6,1.880597E0,1.281E3,2.4204762E-1,3.384024E9,3.8488703E-3,1E0,1.5566E4,1E0,1.1875076E-2,2.6119404E-2,2.014E3,1.3155E4,-4.901009E-3,-1.8350847E-2,-2.801159E-3,-4.9708397E-3,2.995213E-3,8.9281055E4,3.98E2,9.780521E-3,6.872321E-1,-9.432969E-4,2.447933E-3,8.5913794E5,1.6841067E-3,4.725242E-3,6.2271062E-2,2.839944E-3,4.4010544E5,9.657583E-4,4.9288245E-3,4.066361E5,-1.844102E-3,-2.725091E-3,-7.263197E-3,-4.4702613E-3,1.0326397E6,-2.9105484E-3,1.8492938E-3,5.987247E-4,4.6713767E-3,2.226526E-3,-1.3697336E-3],"split_indices":[27,4,31,56,0,32,53,2,27,12,0,16,9,16,38,57,0,9,0,0,0,0,0,33,0,0,27,0,0,32,0,0,57,0,33,0,0,32,0,0,0,0,48,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.28E2,7E1,1.21E2,7E0,4.9E1,2.1E1,9.1E1,3E1,4.3E1,6E0,1.1E1,1E1,5.9E1,3.2E1,1.4E1,1.6E1,3.8E1,5E0,6E0,5E0,5E0,5E0,4.6E1,1.3E1,6E0,2.6E1,5E0,9E0,1.1E1,5E0,5E0,3.3E1,8E0,3.8E1,6E0,7E0,2.1E1,5E0,6E0,5E0,8E0,2.5E1,2.7E1,1.1E1,8E0,1.3E1,7E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-5.3319368E-3,1.9454253E-3,-9.157576E-3,-5.625844E-3,1.1420001E-1,-1.2629728E-2,8.482593E-2,2.1487193E-3,8.989161E-3,7.5695324E-3,-3.7411164E-2,7.0219827E-3,-1.0506121E-3,2.3961361E-2,-4.2564407E-2,-7.4835666E-2,-4.2397473E-3,-9.30783E-3,7.5284384E-2,-9.6776895E-4,-9.541761E-2,-4.784752E-2,-8.983735E-3,4.827979E-3,-2.0287205E-2,5.2395364E-4,-2.7694847E-3,-4.41446E-4,4.637192E-3,-1.9841243E-3,2.451665E-3,-1.759204E-3,-6.6215103E-3,-3.968716E-3,4.6314477E-4,-1.6684765E-3,3.1371799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8704882E-1,1.7487396E-1,0E0,1.2333948E-1,5.7787046E-2,9.038518E-2,9.549738E-2,0E0,0E0,8.319861E-2,1.0023805E-1,0E0,0E0,1.3046888E-1,5.487055E-2,1.0659027E-1,7.619435E-2,4.6779525E-2,5.541545E-2,3.375428E-2,2.2555992E-2,6.520321E-2,0E0,0E0,5.0914794E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.674595E6,1.3904022E3,-9.157576E-3,6.204429E6,1.376E4,1.8352579E0,1.25077E5,2.1487193E-3,8.989161E-3,8.869983E5,6.275614E2,7.0219827E-3,-1.0506121E-3,2.5930756E2,2E0,5.0229483E2,1.65E9,6.791045E-1,3.6482175E5,3.7186194E5,3.3396155E-1,3.936351E2,-8.983735E-3,4.827979E-3,3.906E3,5.2395364E-4,-2.7694847E-3,-4.41446E-4,4.637192E-3,-1.9841243E-3,2.451665E-3,-1.759204E-3,-6.6215103E-3,-3.968716E-3,4.6314477E-4,-1.6684765E-3,3.1371799E-3],"split_indices":[28,52,0,32,9,56,29,0,0,32,4,0,0,4,8,55,5,56,28,47,39,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.04E2,7E0,1.92E2,1.2E1,1.79E2,1.3E1,7E0,5E0,9.9E1,8E1,8E0,5E0,7.5E1,2.4E1,3.7E1,4.3E1,4.6E1,2.9E1,1.4E1,1E1,3.1E1,6E0,5E0,3.8E1,3.3E1,1.3E1,6E0,2.3E1,8E0,6E0,5E0,5E0,1.9E1,1.2E1,3.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[9.215593E-3,1.5134277E-2,-7.6826103E-3,-1.0834546E-1,2.2240853E-2,-6.600661E-3,-2.9155319E-3,1.7129209E-2,1.03399344E-1,-4.026407E-2,2.4601722E-2,7.1439277E-3,1.9378386E-3,2.859518E-2,-5.6948047E-3,1.6462695E-2,8.744284E-2,-5.471778E-4,2.8174908E-3,1.0249649E-2,4.6686637E-3,1.2684178E-3,5.2150637E-3,2.4350159E-4,5.6901E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,-1,-1,9,11,13,15,-1,-1,17,-1,19,21,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[2.0817885E-1,1.7703581E-1,0E0,6.9897324E-3,7.663613E-2,0E0,0E0,7.7758655E-2,2.5625534E-2,1.1706494E-1,7.988347E-2,0E0,0E0,1.5180811E-2,0E0,7.0452355E-2,2.0615816E-2,0E0,0E0,7.3973E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19],"right_children":[2,4,-1,6,8,-1,-1,10,12,14,16,-1,-1,18,-1,20,22,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,4.75E2,-7.6826103E-3,2.3135895E6,3.26989E6,-6.600661E-3,-2.9155319E-3,7E0,1.06748E6,2.7546022E5,1E0,7.1439277E-3,1.9378386E-3,2.0037713E0,-5.6948047E-3,2.446727E12,1.6262975E-1,-5.471778E-4,2.8174908E-3,1.3717E4,4.6686637E-3,1.2684178E-3,5.2150637E-3,2.4350159E-4,5.6901E-3],"split_indices":[45,2,0,45,28,0,0,3,1,33,105,0,0,56,0,31,58,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.98E2,6E0,1E1,1.88E2,5E0,5E0,1.78E2,1E1,2E1,1.58E2,5E0,5E0,1.1E1,9E0,1.41E2,1.7E1,5E0,6E0,1.32E2,9E0,5E0,1.2E1,1.27E2,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[3.4546372E-4,2.372358E-2,-4.9853805E-2,1.2959531E-2,1.4284217E-1,-3.1912476E-2,-9.660273E-3,7.776619E-2,4.2106467E-3,1.1041931E-2,2.2669116E-3,-6.034005E-2,6.118326E-2,-1.3269203E-4,6.053922E-3,-4.3374207E-3,1.1786608E-2,-3.7096877E-2,-1.0389376E-2,-4.9713074E-4,4.6950607E-3,5.130763E-2,-3.972149E-3,3.17068E-2,-6.79687E-2,-9.028315E-4,7.170416E-2,-7.34203E-2,9.169853E-3,4.9376395E-3,-9.288397E-4,-9.671152E-2,-2.8211791E-2,7.2704774E-4,4.075919E-3,-7.784588E-4,-5.132003E-3,-3.663453E-5,4.660076E-3,-2.2579727E-3,-5.434838E-3,1.3324361E-4,-2.8554383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,19,-1,-1,-1,21,23,-1,-1,-1,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5115192E-1,1.8534979E-1,1.7874677E-1,7.6215655E-2,8.955331E-2,1.6777395E-1,0E0,6.2972784E-2,8.7202586E-2,0E0,0E0,1.6657959E-1,4.0536977E-2,0E0,0E0,0E0,6.995764E-2,9.349113E-2,0E0,0E0,0E0,4.7139093E-2,7.4600965E-2,5.304201E-2,3.1553358E-2,0E0,1.7608136E-2,2.3306921E-2,6.0465973E-2,0E0,0E0,8.755282E-3,1.3786099E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,12,12,16,16,17,17,21,21,22,22,23,23,24,24,26,26,27,27,28,28,31,31,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,20,-1,-1,-1,22,24,-1,-1,-1,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.083812E7,5.108104E2,7.1656E4,1.3176E4,7.7198017E-1,7.5601786E8,-9.660273E-3,7.335273E-1,9.235376E5,1.1041931E-2,2.2669116E-3,2.3177125E6,1.2047E4,-1.3269203E-4,6.053922E-3,-4.3374207E-3,5.1373015E6,2.6786362E8,-1.0389376E-2,-4.9713074E-4,4.6950607E-3,2.3135895E6,1.4693E4,6.50596E5,2.126386E0,-9.028315E-4,3.0662747E5,7.492958E-1,2.5831E4,4.9376395E-3,-9.288397E-4,1.8313253E0,4E0,7.2704774E-4,4.075919E-3,-7.784588E-4,-5.132003E-3,-3.663453E-5,4.660076E-3,-2.2579727E-3,-5.434838E-3,1.3324361E-4,-2.8554383E-3],"split_indices":[45,52,10,9,27,7,0,42,45,0,0,28,9,0,0,0,45,7,0,0,0,45,9,28,54,0,48,56,10,0,0,54,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.45E2,6.7E1,1.34E2,1.1E1,6.1E1,6E0,1.5E1,1.19E2,5E0,6E0,4.7E1,1.4E1,6E0,9E0,8E0,1.11E2,4.2E1,5E0,5E0,9E0,3.1E1,8E1,1.3E1,2.9E1,7E0,2.4E1,1.2E1,6.8E1,5E0,8E0,1.6E1,1.3E1,5E0,1.9E1,5E0,7E0,6.2E1,6E0,5E0,1.1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.364336E-3,-1.0574123E-1,1.7093858E-2,-4.32481E-2,-8.923564E-3,6.1308825E-3,1.4396141E-1,5.683063E-4,-4.039024E-3,4.5742624E-2,-1.04307765E-2,1.2526486E-2,1.8126821E-3,3.0005876E-2,8.651818E-2,-2.5241837E-2,4.2459678E-2,4.409337E-2,-2.0641144E-3,1.1095878E-3,6.15454E-3,-4.675296E-3,-8.04788E-2,8.221066E-2,-1.5774528E-3,2.6091704E-2,3.890061E-3,-3.8204283E-2,2.3239443E-2,-1.2638317E-1,2.3813143E-4,1.6691883E-3,1.005748E-1,2.7227003E-4,2.804804E-3,3.8902808E-3,-3.184187E-3,3.1590352E-3,-1.2384685E-3,-8.974933E-3,-2.3778311E-3,6.7931544E-3,2.0405264E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5689712E-1,8.75539E-2,2.2721612E-1,2.8153611E-2,0E0,1.0019461E-1,1.5222168E-1,0E0,0E0,2.62793E-2,8.5675046E-2,0E0,0E0,3.6908273E-2,2.8916374E-2,9.618656E-2,7.462707E-2,1.725182E-2,0E0,0E0,0E0,5.985288E-2,9.0845674E-2,1.04142055E-2,0E0,1.3576198E-2,0E0,1.0325533E-1,7.506945E-2,6.0873598E-2,0E0,0E0,2.025108E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,13,13,14,14,15,15,16,16,17,17,21,21,22,22,23,23,25,25,27,27,28,28,29,29,32,32],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2106E4,1.3907746E10,5.818E3,4.791778E-2,-8.923564E-3,6.8915665E-1,5.199796E9,5.683063E-4,-4.039024E-3,3.0954462E2,1E0,1.2526486E-2,1.8126821E-3,9E0,6.16E2,7.751252E-1,3.2004956E5,6.584533E5,-2.0641144E-3,1.1095878E-3,6.15454E-3,1.436E3,1.9E1,2.2E2,-1.5774528E-3,8.7371956E1,3.890061E-3,8.04E2,5.591566E0,2.095716E0,2.3813143E-4,1.6691883E-3,6.4675875E5,2.7227003E-4,2.804804E-3,3.8902808E-3,-3.184187E-3,3.1590352E-3,-1.2384685E-3,-8.974933E-3,-2.3778311E-3,6.7931544E-3,2.0405264E-3],"split_indices":[9,5,2,38,0,56,12,0,0,4,16,0,0,3,10,27,47,32,0,0,0,2,8,10,0,52,0,2,56,54,0,0,48,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.81E2,1.8E1,1.63E2,1.1E1,7E0,1.51E2,1.2E1,5E0,6E0,4.4E1,1.07E2,5E0,7E0,3.3E1,1.1E1,8.4E1,2.3E1,2.8E1,5E0,5E0,6E0,6.2E1,2.2E1,1.5E1,8E0,2E1,8E0,2.8E1,3.4E1,1.4E1,8E0,5E0,1E1,1.3E1,7E0,5E0,2.3E1,1.8E1,1.6E1,7E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.98988E-3,4.298993E-3,-1.3263792E-2,-6.2565473E-3,8.90277E-2,-2.8164687E-2,3.5866547E-2,3.0801646E-2,8.962361E-3,-1.3428184E-2,-1.6895024E-1,2.1984953E-2,9.696608E-2,-4.7206893E-4,3.769065E-3,8.018422E-2,-2.6096953E-2,-1.2021441E-2,-2.8177882E-3,6.0870703E-2,-1.2130867E-2,1.1111092E-3,7.405611E-3,6.073899E-3,2.2470218E-4,-1.1540919E-1,-1.1793869E-2,9.140091E-4,9.476595E-2,-4.8236985E-2,2.7420748E-2,-8.923705E-3,-1.3445544E-3,3.7949048E-3,-1.01653E-3,5.5873278E-3,2.387068E-3,-3.703658E-3,-1.2442356E-3,-4.41121E-4,2.684925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,19,21,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.603289E-1,1.7763977E-1,0E0,1.6435263E-1,1.1983137E-1,2.4046963E-1,4.9605906E-2,2.996817E-2,0E0,1.2898964E-1,8.4552854E-2,6.8395995E-2,4.316824E-2,0E0,0E0,4.5137107E-2,1.1999571E-1,0E0,0E0,3.2189578E-2,4.1308243E-2,0E0,0E0,0E0,0E0,7.522862E-2,7.286735E-2,0E0,6.9598258E-3,7.701315E-3,1.5193404E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,19,19,20,20,25,25,26,26,28,28,29,29,30,30],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,20,22,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,1.0482935E3,-1.3263792E-2,2.1799042E0,1.4545E4,7.8236975E11,2.0459295E6,8.096533E-1,8.962361E-3,1.2414E4,1.03232314E12,6.3236547E10,2.33948E8,-4.7206893E-4,3.769065E-3,1.3261E4,3.4646995E-4,-1.2021441E-2,-2.8177882E-3,3.4585872E8,3.2E1,1.1111092E-3,7.405611E-3,6.073899E-3,2.2470218E-4,1.45592E5,1.1280869E9,9.140091E-4,1.083E3,2.523484E7,4.6466772E5,-8.923705E-3,-1.3445544E-3,3.7949048E-3,-1.01653E-3,5.5873278E-3,2.387068E-3,-3.703658E-3,-1.2442356E-3,-4.41121E-4,2.684925E-3],"split_indices":[45,52,0,54,9,31,28,27,0,9,31,31,7,0,0,10,38,0,0,5,8,0,0,0,0,1,31,0,2,45,32,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.97E2,5E0,1.76E2,2.1E1,1.16E2,6E1,1.4E1,7E0,1.06E2,1E1,5E1,1E1,8E0,6E0,1.2E1,9.4E1,5E0,5E0,2.3E1,2.7E1,5E0,5E0,7E0,5E0,1.2E1,8.2E1,1.1E1,1.2E1,1.4E1,1.3E1,6E0,6E0,7E0,7.5E1,7E0,5E0,5E0,9E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-9.557284E-4,4.2302134E-3,-8.444042E-3,-4.5208056E-3,1.22691885E-1,2.2237676E-3,-1.04750626E-1,6.1699655E-4,1.2789755E-2,-4.894829E-3,7.066396E-3,-1.2004817E-3,-8.636295E-3,5.6956097E-2,-4.7437935E-3,7.72804E-2,-2.2700573E-3,1.3601924E-3,-4.4910153E-3,1.9858927E-3,4.8656194E-3,-4.1936638E-4,3.3675423E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,-1,-1,15,17,19,-1,21,-1,-1,-1,-1,-1],"loss_changes":[1.8983576E-1,2.109967E-1,0E0,1.2867433E-1,2.183621E-1,9.12143E-2,6.770022E-2,0E0,0E0,0E0,1.0161376E-1,0E0,0E0,7.36425E-2,7.618149E-2,2.0148352E-2,0E0,9.263907E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10,13,13,14,14,15,15,17,17],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,-1,-1,16,18,20,-1,22,-1,-1,-1,-1,-1],"split_conditions":[3.6030095E10,1.4056875E3,-8.444042E-3,2.289485E1,1.3841E4,1.68128E8,2.4272E4,6.1699655E-4,1.2789755E-2,-4.894829E-3,1.9413766E8,-1.2004817E-3,-8.636295E-3,1.4688152E9,1.0388931E8,8E0,-2.2700573E-3,9.4010997E-1,-4.4910153E-3,1.9858927E-3,4.8656194E-3,-4.1936638E-4,3.3675423E-3],"split_indices":[5,52,0,57,9,7,10,0,0,0,7,0,0,5,45,3,0,27,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,2.02E2,5E0,1.89E2,1.3E1,1.78E2,1.1E1,8E0,5E0,7E0,1.71E2,6E0,5E0,3.2E1,1.39E2,2.7E1,5E0,1.31E2,8E0,1.2E1,1.5E1,1.15E2,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.3546964E-3,-9.8290674E-2,7.936172E-3,2.122498E-3,-1.4562026E-1,1.5572541E-2,-1.2190936E-1,-1.0809848E-2,-1.791769E-3,8.984492E-3,1.1956564E-1,6.0504995E-4,-1.1312587E-2,-2.8677914E-2,3.129741E-2,8.9405396E-4,9.60763E-3,4.2789616E-3,-4.1965857E-2,5.867157E-3,2.3027342E-2,-1.0168736E-1,-1.7937874E-2,8.3036914E-2,8.465212E-3,-5.9975026E-4,-6.60166E-3,1.8371216E-3,-1.9279694E-3,-1.726462E-4,5.7694097E-3,-2.9594365E-3,1.2408379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,-1,7,9,11,-1,-1,13,15,-1,-1,17,19,-1,-1,-1,21,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.140292E-1,1.481786E-1,1.9624195E-1,0E0,1.3011056E-1,1.2635654E-1,1.7201993E-1,0E0,0E0,1.4880061E-1,8.6136565E-2,0E0,0E0,1.0722902E-1,8.087097E-2,0E0,0E0,0E0,8.427517E-2,0E0,8.93054E-2,5.301349E-2,5.67819E-2,6.522225E-2,1.04466684E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,9,9,10,10,13,13,14,14,18,18,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,-1,8,10,12,-1,-1,14,16,-1,-1,18,20,-1,-1,-1,22,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7E0,1.09315805E5,6.1723E4,2.122498E-3,1E0,1.1510987E1,1E1,-1.0809848E-2,-1.791769E-3,2.5324342E5,1.6773E4,6.0504995E-4,-1.1312587E-2,1.92E2,2.808836E9,8.9405396E-4,9.60763E-3,4.2789616E-3,1.18107956E2,5.867157E-3,9.0996725E5,2.53E2,6.50596E5,1.5697E4,1.4859048E6,-5.9975026E-4,-6.60166E-3,1.8371216E-3,-1.9279694E-3,-1.726462E-4,5.7694097E-3,-2.9594365E-3,1.2408379E-3],"split_indices":[3,33,10,0,16,56,3,0,0,33,9,0,0,11,31,0,0,0,52,0,50,11,28,9,50,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2E1,1.95E2,5E0,1.5E1,1.85E2,1E1,8E0,7E0,1.75E2,1E1,5E0,5E0,6.5E1,1.1E2,5E0,5E0,6E0,5.9E1,8E0,1.02E2,1.6E1,4.3E1,1.9E1,8.3E1,5E0,1.1E1,1.2E1,3.1E1,6E0,1.3E1,1.6E1,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[7.1131624E-3,-2.2026269E-2,2.8357007E-2,-1.2215349E-2,-7.3030414E-3,3.8369182E-2,-7.343246E-2,-4.783255E-3,-2.7056546E-3,7.721029E-2,1.286863E-2,1.7460152E-3,-8.195732E-3,3.722718E-2,-4.0521275E-2,1.2180363E-1,1.4471154E-2,-4.5522254E-2,4.242689E-2,-6.861623E-4,4.766653E-2,-2.4685629E-2,-5.9168264E-3,1.5069374E-3,1.4281815E-1,-1.9210699E-3,4.9645238E-2,2.2518933E-3,-8.094361E-2,-1.6356172E-2,6.95017E-2,2.131153E-2,7.814678E-2,-5.358374E-2,2.6117768E-2,9.241918E-3,3.5182505E-3,-3.9588238E-4,4.4166204E-3,-1.1776626E-3,-1.02379315E-1,1.2706688E-3,-3.3128536E-3,3.6682256E-2,1.2583867E-1,2.0858315E-3,1.5375883E-4,5.662791E-3,2.0486228E-3,-6.920204E-4,-3.8120132E-3,2.8781423E-3,-1.2531123E-4,-2.6468302E-3,-6.139556E-3,2.8727932E-3,2.4315815E-4,7.790612E-3,3.2620435E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,9,11,-1,13,15,17,-1,-1,19,21,23,25,27,29,-1,31,33,-1,-1,35,-1,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,53,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2752648E-1,1.09450705E-1,1.2409449E-1,6.8060525E-2,0E0,1.0696761E-1,1.242457E-1,0E0,1.1476487E-1,1.18540525E-1,1.1756887E-1,0E0,0E0,2.0594038E-2,4.8199996E-2,4.2973727E-2,3.8460277E-2,7.918517E-2,7.275609E-2,0E0,2.323708E-2,5.1315654E-2,0E0,0E0,5.5339545E-2,0E0,3.0325547E-2,0E0,1.802209E-2,3.579022E-2,5.2701354E-2,7.1024783E-3,1.4788084E-2,2.12254E-2,1.2841713E-2,0E0,0E0,0E0,0E0,0E0,7.4964166E-3,0E0,0E0,1.5148977E-2,1.2289554E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,24,24,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,40,40,43,43,44,44],"right_children":[2,4,6,8,-1,10,12,-1,14,16,18,-1,-1,20,22,24,26,28,30,-1,32,34,-1,-1,36,-1,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,54,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1546597E2,3.6543632E6,1.4089414E8,9.235376E5,-7.3030414E-3,2.1358025E0,1.4689625E5,-4.783255E-3,5.2139695E6,9.235021E-1,1.631E3,1.7460152E-3,-8.195732E-3,1.0963781E0,4.8028188E5,1.92E2,1.3843815E6,4.720179E10,1.4157E4,-6.861623E-4,4.17266E5,1.64935E5,-5.9168264E-3,1.5069374E-3,1.306E3,-1.9210699E-3,1.514E3,2.2518933E-3,1.08989E5,2.4010162E5,4.633548E-1,5.9168223E-2,6.6803556E5,1.3986014E-2,8.193631E7,9.241918E-3,3.5182505E-3,-3.9588238E-4,4.4166204E-3,-1.1776626E-3,1.541336E0,1.2706688E-3,-3.3128536E-3,8.509488E2,1.6705686E0,2.0858315E-3,1.5375883E-4,5.662791E-3,2.0486228E-3,-6.920204E-4,-3.8120132E-3,2.8781423E-3,-1.2531123E-4,-2.6468302E-3,-6.139556E-3,2.8727932E-3,2.4315815E-4,7.790612E-3,3.2620435E-3],"split_indices":[4,28,45,45,0,56,33,0,45,27,2,0,0,53,33,0,28,31,9,0,32,1,0,0,0,0,2,0,12,33,27,38,32,57,12,0,0,0,0,0,53,0,0,4,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,8.6E1,1.18E2,8.1E1,5E0,1.08E2,1E1,7E0,7.4E1,4.2E1,6.6E1,5E0,5E0,3.6E1,3.8E1,2.4E1,1.8E1,2.2E1,4.4E1,6E0,3E1,3.3E1,5E0,5E0,1.9E1,7E0,1.1E1,6E0,1.6E1,1.4E1,3E1,1.7E1,1.3E1,2.1E1,1.2E1,1E1,9E0,5E0,6E0,5E0,1.1E1,8E0,6E0,2E1,1E1,7E0,1E1,5E0,8E0,9E0,1.2E1,5E0,7E0,5E0,6E0,1.1E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.095566E-3,-1.2488788E-2,1.1695275E-1,4.6495297E-3,-4.796547E-2,1.064835E-2,1.1186311E-3,-7.3224274E-3,6.2121253E-2,-9.3917735E-2,-1.7680738E-2,1.3521488E-3,-4.3468946E-3,6.463305E-3,1.9159265E-2,-1.5515956E-1,-3.0162381E-2,9.640252E-3,-5.5849615E-2,-1.6811745E-2,4.056598E-2,-9.3058153E-4,2.6516225E-3,-9.063187E-3,-4.293154E-3,-3.5266269E-3,2.9679737E-4,-2.6606468E-2,3.8771576E-3,-8.85595E-2,6.4325426E-4,2.519898E-3,-2.848914E-2,7.713755E-2,-3.5862029E-3,1.072817E-3,-4.4461307E-3,-5.5913585E-3,-2.186995E-3,-5.858892E-3,-6.331601E-4,7.232601E-3,1.1087124E-3,2.2731214E-3,-1.6215863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,-1,37,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0364168E-1,1.1392102E-1,1.2295887E-1,8.783711E-2,8.323024E-2,0E0,0E0,7.681966E-2,6.621235E-2,8.902785E-2,4.016524E-2,6.977274E-2,0E0,0E0,2.2019297E-2,9.805173E-3,2.0992368E-2,6.158456E-2,3.7445426E-2,5.580391E-2,5.0332785E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.3977367E-2,0E0,7.959329E-3,0E0,0E0,8.1479475E-2,6.4276084E-2,2.4771227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,27,27,29,29,32,32,33,33,34,34],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,-1,38,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3904022E3,8.617448E5,4.607055E5,4E1,1.6155814E6,1.064835E-2,1.1186311E-3,6.6774924E7,5.3246627E0,3.0050538E5,2.67302E5,1.436E3,-4.3468946E-3,6.463305E-3,2.1100497E-1,2.0983605E0,1.4936014E0,3.14016E5,3.0229E4,1.4E0,2.9807162E0,-9.3058153E-4,2.6516225E-3,-9.063187E-3,-4.293154E-3,-3.5266269E-3,2.9679737E-4,7.9716E4,3.8771576E-3,1.714995E0,6.4325426E-4,2.519898E-3,5.94E2,3.701903E8,3.81938E5,1.072817E-3,-4.4461307E-3,-5.5913585E-3,-2.186995E-3,-5.858892E-3,-6.331601E-4,7.232601E-3,1.1087124E-3,2.2731214E-3,-1.6215863E-3],"split_indices":[52,32,28,8,32,0,0,45,57,33,29,2,0,0,39,56,53,1,10,54,56,0,0,0,0,0,0,1,0,53,0,0,2,7,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.86E2,1.2E1,1.26E2,6E1,5E0,7E0,1.05E2,2.1E1,2.3E1,3.7E1,9.6E1,9E0,7E0,1.4E1,1.1E1,1.2E1,2.2E1,1.5E1,6.6E1,3E1,7E0,7E0,6E0,5E0,5E0,7E0,1.5E1,7E0,1E1,5E0,9E0,5.7E1,1.6E1,1.4E1,9E0,6E0,5E0,5E0,7E0,5E1,6E0,1E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.381206E-5,6.741054E-3,-1.08200945E-1,1.0291582E-3,5.851233E-3,-1.1417227E-2,8.998055E-4,-9.020174E-2,6.8240184E-3,-6.8379045E-3,-1.0846909E-3,-3.7622836E-3,1.0896399E-2,5.4145367E-3,5.68964E-3,1.19317835E-2,-7.1770936E-2,-3.908729E-5,2.4048458E-3,-3.3393555E-4,-5.608214E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,13,-1,15,17,19,-1,-1,-1,-1],"loss_changes":[1.4916813E-1,1.257647E-1,2.0274137E-1,9.736221E-2,0E0,0E0,0E0,3.5805017E-2,6.099009E-2,0E0,0E0,0E0,8.762349E-2,0E0,7.7736035E-2,7.223382E-2,3.4402214E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,12,12,14,14,15,15,16,16],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,14,-1,16,18,20,-1,-1,-1,-1],"split_conditions":[2.835E3,1.1510987E1,3.299E3,2.3950832E-5,5.851233E-3,-1.1417227E-2,8.998055E-4,2.12293E5,9.541066E5,-6.8379045E-3,-1.0846909E-3,-3.7622836E-3,1.1280869E9,5.4145367E-3,5.08616E5,2.1E1,1.3723022E0,-3.908729E-5,2.4048458E-3,-3.3393555E-4,-5.608214E-3],"split_indices":[0,56,0,42,0,0,0,29,45,0,0,0,31,0,33,8,56,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.9E2,1.1E1,1.82E2,8E0,5E0,6E0,1E1,1.72E2,5E0,5E0,7E0,1.65E2,7E0,1.58E2,1.47E2,1.1E1,1.11E2,3.6E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.8882776E-3,1.00166835E-2,-7.371662E-2,4.7211065E-3,4.8804465E-3,-4.1055635E-2,-1.0245122E-2,1.7040744E-2,-2.1460107E-2,-2.0179208E-3,-8.808312E-3,2.4134438E-2,-3.576226E-3,3.66077E-2,-4.2288773E-2,-3.3715464E-2,2.9525624E-3,-5.267561E-3,4.6923134E-2,-6.3393277E-4,3.150777E-3,-1.0187121E-1,9.985219E-4,8.436718E-4,-4.179634E-3,-4.012018E-2,2.6900155E-2,7.190755E-2,3.906673E-3,-1.7125866E-3,-6.164742E-3,3.5533864E-2,-3.362003E-2,-4.214546E-3,-1.0811037E-5,3.1787308E-3,-5.716725E-5,2.431797E-3,8.105542E-3,-1.9674457E-3,1.1181055E-3,-2.0289488E-4,3.0922557E-3,4.8777572E-4,-3.1792584E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,17,-1,19,21,23,-1,25,27,-1,-1,29,31,-1,-1,33,35,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9298853E-1,8.02646E-2,1.4225613E-1,5.1327497E-2,0E0,1.5571955E-1,0E0,7.2045505E-2,6.323132E-2,4.847953E-2,0E0,6.792052E-2,0E0,2.1216236E-2,9.8109886E-2,4.5493107E-2,0E0,5.1530052E-2,6.0979337E-2,0E0,0E0,2.3761153E-2,2.8692609E-2,0E0,0E0,4.197786E-2,2.6678098E-2,6.547545E-2,2.018457E-2,0E0,0E0,1.4118374E-2,1.788503E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,17,17,18,18,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,18,-1,20,22,24,-1,26,28,-1,-1,30,32,-1,-1,34,36,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8351054E11,6.811E3,2.363013E6,2.2760513E0,4.8804465E-3,1.9E1,-1.0245122E-2,3.9556694E-1,3.1E2,5.683493E6,-8.808312E-3,8.92E2,-3.576226E-3,1.11258715E-1,2.7573213E8,7.822023E7,2.9525624E-3,8E0,1.007E3,-6.3393277E-4,3.150777E-3,1.9303116E5,1.6389892E0,8.436718E-4,-4.179634E-3,4.3779754E-4,1.6827E4,1.996E3,7.280497E0,-1.7125866E-3,-6.164742E-3,1.6E1,4.2736053E0,-4.214546E-3,-1.0811037E-5,3.1787308E-3,-5.716725E-5,2.431797E-3,8.105542E-3,-1.9674457E-3,1.1181055E-3,-2.0289488E-4,3.0922557E-3,4.8777572E-4,-3.1792584E-3],"split_indices":[31,2,28,56,0,3,0,38,10,1,0,2,0,58,7,45,0,3,0,0,0,32,53,0,0,38,29,2,57,0,0,8,57,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.65E2,3.2E1,1.57E2,8E0,2.7E1,5E0,1.07E2,5E1,2.2E1,5E0,1E2,7E0,1.3E1,3.7E1,1.5E1,7E0,4.4E1,5.6E1,5E0,8E0,1.5E1,2.2E1,8E0,7E0,2.1E1,2.3E1,3.5E1,2.1E1,5E0,1E1,1.1E1,1.1E1,9E0,1.2E1,9E0,1.4E1,3E1,5E0,6E0,1.5E1,5E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-9.95141E-3,-4.0595704E-3,-9.053523E-3,-1.3823979E-2,1.15495674E-1,-2.445644E-2,3.1380143E-2,1.2050496E-4,1.2730055E-2,-6.7949397E-3,-6.0015667E-2,4.132615E-3,1.3286835E-2,-2.6767712E-2,3.093557E-2,-9.7700045E-2,1.0016516E-2,-3.060755E-3,3.4704007E-2,2.8235086E-3,-3.8746968E-2,6.0488484E-3,-1.0071418E-2,-5.927876E-2,-1.03229545E-2,2.7865102E-3,-1.8800803E-3,6.552377E-2,3.4940982E-5,9.767149E-4,-2.5867554E-3,-3.526529E-3,5.0416094E-4,-3.6588984E-3,-3.2836446E-4,1.3759182E-3,4.3791626E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9858697E-1,2.1639566E-1,0E0,8.295535E-2,2.3694031E-1,8.688826E-2,3.1622004E-2,0E0,0E0,7.166487E-2,1.2237811E-1,0E0,4.5191552E-2,6.593666E-2,1.3026589E-1,1.2297669E-1,4.260149E-2,0E0,2.1941137E-2,0E0,5.068937E-2,0E0,3.2180242E-2,2.0984277E-2,0E0,0E0,0E0,7.853508E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,20,20,22,22,23,23,27,27],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841177E10,1.3904022E3,-9.053523E-3,2.078125E0,1.4545E4,8.601996E-1,3.77E2,1.2050496E-4,1.2730055E-2,1.6921558E2,1.3642281E-1,4.132615E-3,6.9146306E-4,8.66E3,1.854111E7,7.7296586E11,7.4456794E5,-3.060755E-3,1.497692E8,2.8235086E-3,1.3345E4,6.0488484E-3,1.895E3,2.1577182E0,-1.03229545E-2,2.7865102E-3,-1.8800803E-3,3.101519E5,3.4940982E-5,9.767149E-4,-2.5867554E-3,-3.526529E-3,5.0416094E-4,-3.6588984E-3,-3.2836446E-4,1.3759182E-3,4.3791626E-3],"split_indices":[5,52,0,53,9,27,0,0,0,52,42,0,38,29,45,31,32,0,12,0,9,0,2,54,0,0,0,32,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,1.83E2,5E0,1.7E2,1.3E1,1.38E2,3.2E1,8E0,5E0,9.3E1,4.5E1,7E0,2.5E1,6.1E1,3.2E1,2.9E1,1.6E1,5E0,2E1,7E0,5.4E1,9E0,2.3E1,2.3E1,6E0,8E0,8E0,1E1,1E1,1.1E1,4.3E1,5E0,1.8E1,1.7E1,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[6.7305095E-3,1.7460078E-2,-7.1220785E-2,-4.888752E-3,2.2854166E-2,1.7001651E-3,-1.23524144E-1,2.404821E-4,4.0388856E-2,-9.314982E-3,-9.316741E-4,-9.192367E-3,3.9643114E-3,6.6240965E-3,3.134043E-2,-2.4730662E-2,4.943009E-2,4.4192925E-2,-1.3605638E-2,-3.580801E-2,2.775148E-3,3.6156115E-3,1.1347134E-3,2.7026076E-2,8.432041E-3,-5.0528117E-2,2.4553718E-2,-2.5033941E-3,1.1523056E-4,-1.0646E-3,1.976691E-3,-7.2936976E-4,-3.7086133E-3,-6.730798E-4,2.8296807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,-1,7,-1,9,11,13,-1,-1,15,-1,-1,17,19,21,23,25,27,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7270218E-1,1.1900742E-1,1.4391415E-1,0E0,6.898545E-2,0E0,1.21679455E-1,6.068817E-2,8.2458824E-2,0E0,0E0,6.498495E-2,0E0,0E0,5.3169154E-2,5.334888E-2,8.268211E-3,1.5590033E-1,3.082835E-2,3.154809E-2,0E0,0E0,0E0,4.5779627E-2,0E0,9.261088E-3,1.5484236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,11,11,14,14,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26],"right_children":[2,4,6,-1,8,-1,10,12,14,-1,-1,16,-1,-1,18,20,22,24,26,28,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5965E5,1.01E4,6.5229E4,-4.888752E-3,1.222E3,1.7001651E-3,1E0,3.1515045E6,1.5789307E8,-9.314982E-3,-9.316741E-4,4.5929728E5,3.9643114E-3,6.6240965E-3,2.943697E7,2.9372938E0,1.1924399E0,1.2199858E7,7.203918E9,4.982441E0,2.775148E-3,3.6156115E-3,1.1347134E-3,1.3328E4,8.432041E-3,2.1287676E7,1.856461E7,-2.5033941E-3,1.1523056E-4,-1.0646E-3,1.976691E-3,-7.2936976E-4,-3.7086133E-3,-6.730798E-4,2.8296807E-3],"split_indices":[9,9,29,0,2,0,16,32,5,0,0,33,0,0,50,56,58,48,12,57,0,0,0,9,0,45,47,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.8E2,2.4E1,7E0,1.73E2,8E0,1.6E1,7.6E1,9.7E1,9E0,7E0,6.9E1,7E0,7E0,9E1,5.5E1,1.4E1,7E1,2E1,4.9E1,6E0,6E0,8E0,6.3E1,7E0,1E1,1E1,3.4E1,1.5E1,1.4E1,4.9E1,5E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.734199E-3,1.3744242E-2,-9.240938E-3,2.3152918E-2,-3.7991587E-2,-8.5648766E-4,4.9247783E-2,-6.1307233E-3,4.633573E-3,1.0621249E-2,-6.555168E-3,7.024371E-3,3.5039723E-2,-3.1629456E-3,2.929784E-2,-4.920238E-3,1.920218E-2,-1.7098352E-2,5.024164E-2,2.8506932E-3,1.055836E-4,8.000154E-2,6.9356076E-3,-2.0492584E-3,8.2087173E-4,-7.4160867E-3,6.4500555E-2,1.524072E-3,5.08515E-3,2.57247E-3,-3.2032706E-4,-2.0306283E-3,1.1314659E-3,4.420035E-3,4.033807E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,-1,17,-1,19,-1,21,23,25,-1,-1,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4862354E-1,9.774597E-2,0E0,1.05924435E-1,1.1948568E-1,1.4034997E-1,1.0692477E-1,0E0,4.027229E-2,8.268222E-2,0E0,0E0,5.7926156E-2,0E0,1.378132E-2,0E0,5.7617523E-2,1.5675277E-2,4.6517998E-2,0E0,0E0,1.3000146E-2,4.2398307E-2,0E0,0E0,1.4023744E-2,6.927845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,12,12,14,14,16,16,17,17,18,18,21,21,22,22,25,25,26,26],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,-1,18,-1,20,-1,22,24,26,-1,-1,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4299594E8,2.5965E5,-9.240938E-3,1.6720915E2,7.05571E5,2.0866848E7,1.0932244E9,-6.1307233E-3,3.1622803E5,1.074074E0,-6.555168E-3,7.024371E-3,5.269822E5,-3.1629456E-3,7.756314E7,-4.920238E-3,1.704313E4,1.5697E4,2.2903395E8,2.8506932E-3,1.055836E-4,7.12E2,6.2085975E9,-2.0492584E-3,8.2087173E-4,1.8567E4,8.992676E1,1.524072E-3,5.08515E-3,2.57247E-3,-3.2032706E-4,-2.0306283E-3,1.1314659E-3,4.420035E-3,4.033807E-4],"split_indices":[45,9,0,52,9,45,5,0,28,53,0,0,28,0,12,0,48,9,7,0,0,2,31,0,0,12,58,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.98E2,5E0,1.68E2,3E1,8.8E1,8E1,9E0,2.1E1,8.2E1,6E0,9E0,7.1E1,5E0,1.6E1,5E0,7.7E1,1.6E1,5.5E1,7E0,9E0,1.2E1,6.5E1,9E0,7E0,1.1E1,4.4E1,5E0,7E0,1.4E1,5.1E1,5E0,6E0,2.9E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.6418557E-4,-6.4944443E-3,4.5279595E-3,-3.6861983E-4,6.6873482E-3,-5.0364896E-3,4.702346E-3,8.246746E-3,-4.173349E-3,1.877944E-2,-1.951385E-2,2.9930364E-2,-5.2886378E-2,-4.9523227E-3,-7.8992825E-3,9.2911575E-4,4.0745535E-3,-4.378941E-3,1.8866103E-3,7.835974E-4,-2.165033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,-1,3,5,-1,-1,7,9,-1,11,13,15,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5095755E-1,0E0,1.3231623E-1,1.0355283E-1,0E0,0E0,6.124718E-2,5.299736E-2,0E0,1.0627027E-1,4.788155E-2,6.455845E-2,6.756645E-2,0E0,4.144308E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,9,9,10,10,11,11,12,12,14,14],"right_children":[2,-1,4,6,-1,-1,8,10,-1,12,14,16,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01E4,-6.4944443E-3,1.2144419E1,6E0,6.6873482E-3,-5.0364896E-3,4.966421E6,2.5605617E0,-4.173349E-3,3.34231E-1,1.3016E4,2.409E3,9.621332E-1,-4.9523227E-3,1.8738E4,9.2911575E-4,4.0745535E-3,-4.378941E-3,1.8866103E-3,7.835974E-4,-2.165033E-3],"split_indices":[9,0,56,3,0,0,28,56,0,38,9,2,27,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,7E0,1.99E2,1.93E2,6E0,8E0,1.85E2,1.79E2,6E0,1.3E2,4.9E1,1.13E2,1.7E1,5E0,4.4E1,9.6E1,1.7E1,1.2E1,5E0,2.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.3359891E-3,7.598631E-3,-5.795484E-3,-1.3054161E-3,9.671589E-2,-1.30533045E-2,3.0901961E-2,6.017363E-4,8.150426E-3,-5.933671E-4,-1.1269043E-1,8.863279E-2,2.8749483E-3,-1.2696118E-2,4.0392615E-2,-8.385759E-3,-2.1103797E-3,7.4418155E-3,1.9042853E-3,2.6180213E-2,-2.854947E-3,-4.3699328E-2,5.8003473E-3,4.338851E-3,8.50301E-3,-6.0795285E-3,4.381679E-3,-2.9421232E-3,2.2153168E-3,1.6627694E-3,-1.7814175E-3,-1.1049599E-3,3.3641206E-3,-1.5691422E-3,2.3130474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,-1,27,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2680905E-1,1.4851156E-1,0E0,6.510442E-2,1.0288632E-1,1.5635866E-1,7.417423E-2,0E0,0E0,5.6565702E-2,5.2202687E-2,4.273562E-2,4.81459E-2,5.071706E-2,4.08299E-2,0E0,0E0,0E0,0E0,5.1097564E-2,0E0,5.5409856E-2,7.087297E-2,0E0,3.4739327E-2,2.623093E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,19,19,21,21,22,22,24,24,25,25],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,-1,28,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4299594E8,5.818E3,-5.795484E-3,3.494881E2,1.4781E4,4.520813E7,3.2875034E5,6.017363E-4,8.150426E-3,3.7597818E2,2.081E3,5.5318156E7,1.3182058E3,2.6727284E5,1.45819E5,-8.385759E-3,-2.1103797E-3,7.4418155E-3,1.9042853E-3,8.4342186E1,-2.854947E-3,9E0,6.71234E6,4.338851E-3,3.649047E8,1.5452969E6,4.381679E-3,-2.9421232E-3,2.2153168E-3,1.6627694E-3,-1.7814175E-3,-1.1049599E-3,3.3641206E-3,-1.5691422E-3,2.3130474E-3],"split_indices":[45,2,0,52,9,45,32,0,0,55,2,45,4,33,29,0,0,0,0,58,0,8,45,0,7,28,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.86E2,7E0,1.7E2,1.6E1,1.25E2,4.5E1,8E0,8E0,1.12E2,1.3E1,1.4E1,3.1E1,8.7E1,2.5E1,6E0,7E0,5E0,9E0,2.3E1,8E0,3.2E1,5.5E1,9E0,1.6E1,1.6E1,7E0,2.7E1,5E0,3.3E1,2.2E1,1.1E1,5E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.7254089E-3,7.6746326E-3,-4.719512E-3,1.1724859E-2,-5.173395E-3,4.4106925E-3,1.0753885E-1,4.4548288E-2,-4.2752638E-3,8.823091E-3,-1.2132875E-3,2.0563329E-2,7.1408995E-3,-9.569353E-3,3.904215E-3,-2.2979942E-3,3.7546612E-2,-4.2280434E-3,-6.8416536E-2,1.0763042E-3,3.666471E-3,-2.0393976E-3,1.3758341E-4,1.3414687E-3,-6.768783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,-1,17,-1,-1,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0747822E-1,9.682966E-2,0E0,1.3828397E-1,0E0,6.468763E-2,1.4719811E-1,7.965112E-2,7.025212E-2,0E0,0E0,3.4389094E-2,0E0,4.528729E-2,0E0,0E0,1.1802625E-2,3.6598813E-2,8.8266775E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,13,13,16,16,17,17,18,18],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,-1,18,-1,-1,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,3.6030095E10,-4.719512E-3,1.3904022E3,-5.173395E-3,1.9356645E8,2.193351E6,1.10472E5,2.7319932E6,8.823091E-3,-1.2132875E-3,1.68128E8,7.1408995E-3,5.08616E5,3.904215E-3,-2.2979942E-3,2.0895522E0,9.06E2,9.5606E4,1.0763042E-3,3.666471E-3,-2.0393976E-3,1.3758341E-4,1.3414687E-3,-6.768783E-3],"split_indices":[28,5,0,52,0,7,32,29,28,0,0,7,0,33,0,0,56,2,1,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.03E2,9E0,1.97E2,6E0,1.84E2,1.3E1,3.2E1,1.52E2,8E0,5E0,2.7E1,5E0,1.44E2,8E0,5E0,2.2E1,1.33E2,1.1E1,1.7E1,5E0,2E1,1.13E2,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.831833E-3,-7.791919E-2,1.5585499E-2,3.901734E-4,-8.420296E-3,1.007099E-2,6.597637E-3,-2.6131146E-2,2.9313901E-2,-1.8034883E-2,-4.792742E-3,3.8269266E-2,-5.298094E-2,1.5172308E-3,-3.1689696E-2,2.2402702E-2,7.82467E-2,1.2109441E-3,-5.3892015E-3,-5.4035876E-3,-1.904842E-2,5.0052438E-2,-6.53043E-3,-1.1575788E-3,1.1046521E-1,8.6281466E-4,-2.3395296E-3,-1.3551611E-3,3.1000965E-3,-1.6205095E-3,1.887408E-3,3.1010793E-3,7.0423125E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,-1,15,17,-1,19,21,23,-1,-1,-1,25,27,29,-1,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3618603E-1,1.4457783E-1,1.2455337E-1,0E0,0E0,1.2616022E-1,0E0,3.5893995E-2,8.899123E-2,4.0368393E-2,0E0,6.611766E-2,5.872734E-2,0E0,4.5011334E-2,6.272635E-2,1.01653144E-1,0E0,0E0,0E0,4.613435E-2,4.7757335E-2,5.0221354E-2,0E0,2.8912157E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,20,20,21,21,22,22,24,24],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,-1,16,18,-1,20,22,24,-1,-1,-1,26,28,30,-1,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7E0,1.3473596E6,1.1510987E1,3.901734E-4,-8.420296E-3,2.5324342E5,6.597637E-3,3.241538E6,3.237835E-1,3.58E2,-4.792742E-3,1.0220919E6,8.194808E5,1.5172308E-3,6.71E2,3.2144122E5,2.5176196E2,1.2109441E-3,-5.3892015E-3,-5.4035876E-3,4.6318403E5,8.359578E4,5.0778572E5,-1.1575788E-3,1.766129E0,8.6281466E-4,-2.3395296E-3,-1.3551611E-3,3.1000965E-3,-1.6205095E-3,1.887408E-3,3.1010793E-3,7.0423125E-3],"split_indices":[3,28,56,0,0,33,0,28,38,0,0,28,48,0,2,47,4,0,0,0,32,32,28,0,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.6E1,1.86E2,9E0,7E0,1.79E2,7E0,6.2E1,1.17E2,5.7E1,5E0,1.06E2,1.1E1,1.2E1,4.5E1,7.7E1,2.9E1,5E0,6E0,5E0,4E1,3.9E1,3.8E1,7E0,2.2E1,1.8E1,2.2E1,6E0,3.3E1,2.4E1,1.4E1,1.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-2.1946263E-3,2.646767E-3,-7.5484533E-3,-4.838696E-3,7.674689E-3,-5.268686E-3,1.1553685E-2,1.5994856E-2,-4.814388E-3,6.35635E-3,4.8124943E-2,2.6919441E-2,-2.1407055E-2,7.85664E-2,2.317886E-3,8.7501877E-4,4.3714005E-3,-4.068125E-3,-1.4343046E-4,4.502313E-3,7.986167E-4,-2.5087707E-3,2.1957948E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,-1,5,-1,7,9,-1,11,13,15,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4934996E-1,1.013519E-1,0E0,0E0,8.56972E-2,0E0,9.117762E-2,5.357947E-2,0E0,7.764796E-2,5.559524E-2,4.175908E-2,6.7569844E-2,2.1327168E-2,4.2853165E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14],"right_children":[2,4,-1,-1,6,-1,8,10,-1,12,14,16,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,9.235376E5,-7.5484533E-3,-4.838696E-3,1.0057E4,-5.268686E-3,9.6883655E-1,1.86611E5,-4.814388E-3,3.1617646E0,5.9919E4,4.51E3,2.0767594E-2,2.6486957E0,7.3240237E8,8.7501877E-4,4.3714005E-3,-4.068125E-3,-1.4343046E-4,4.502313E-3,7.986167E-4,-2.5087707E-3,2.1957948E-3],"split_indices":[45,45,0,0,9,0,27,11,0,57,10,10,38,54,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.92E2,5E0,8E0,1.84E2,5E0,1.79E2,1.73E2,6E0,1.34E2,3.9E1,7.7E1,5.7E1,2.3E1,1.6E1,6.9E1,8E0,1.2E1,4.5E1,1.8E1,5E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-6.016137E-3,-1.0773682E-1,-4.194025E-4,-3.5203335E-5,-9.427532E-3,-6.4156796E-3,4.3749237E-3,9.354751E-3,-6.261878E-3,2.8013228E-3,7.5348266E-3,-7.2546857E-3,4.1099217E-2,1.8817674E-2,-2.38712E-2,5.4259285E-2,-1.8267948E-3,3.8214512E-3,3.7954942E-5,-2.5428159E-3,2.809707E-4,3.16686E-3,1.1768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,-1,-1,-1,7,9,-1,11,-1,13,15,17,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.200779E-1,1.04664356E-1,1.2923089E-1,0E0,0E0,0E0,1.3239466E-1,1.8277425E-1,0E0,7.039071E-2,0E0,6.326781E-2,4.1690648E-2,6.255804E-2,7.760361E-2,9.982131E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15],"right_children":[2,4,6,-1,-1,-1,8,10,-1,12,-1,14,16,18,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,1.3335946E6,1.01E4,-3.5203335E-5,-9.427532E-3,-6.4156796E-3,1.1430797E9,2.6825117E3,-6.261878E-3,2.9E1,7.5348266E-3,1.537E4,1.3E1,8.0224875E4,2.6727284E5,6.4E1,-1.8267948E-3,3.8214512E-3,3.7954942E-5,-2.5428159E-3,2.809707E-4,3.16686E-3,1.1768E-3],"split_indices":[3,28,9,0,0,0,7,4,0,8,0,9,3,33,33,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1E1,2E2,5E0,5E0,6E0,1.94E2,1.88E2,6E0,1.81E2,7E0,1.44E2,3.7E1,5.6E1,8.8E1,3.2E1,5E0,1.2E1,4.4E1,4.4E1,4.4E1,2.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[7.302973E-3,1.5426283E-3,7.593345E-3,9.934854E-3,-7.230181E-2,-3.5403352E-3,4.441492E-2,1.2183896E-3,-1.1936586E-1,5.9988275E-3,-5.2711107E-3,9.856841E-2,7.403497E-3,-8.7363655E-3,-1.8839287E-3,-1.1002768E-2,2.039365E-2,9.268261E-3,1.1970541E-2,-8.413091E-3,3.32656E-3,3.488051E-2,-3.236424E-2,1.0666802E-2,5.1634945E-3,-1.1845741E-3,2.23597E-3,6.4151622E-3,-1.6212009E-3,-5.816462E-4,2.6581686E-3,-3.972148E-3,1.7107339E-4,1.2173996E-3,-2.8600174E-3,1.1328069E-3,-1.25724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,19,-1,-1,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6610347E-1,1.1536636E-1,0E0,7.7749684E-2,9.043061E-2,1.23568654E-1,9.3472235E-2,0E0,5.8373302E-2,2.7634699E-2,0E0,1.5381205E-1,2.9037274E-2,0E0,0E0,5.2095868E-2,5.0699145E-2,0E0,1.5161534E-2,9.26584E-3,0E0,1.7143657E-2,6.568709E-2,5.9615273E-2,0E0,0E0,0E0,9.62368E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,15,15,16,16,18,18,19,19,21,21,22,22,23,23,27,27],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,20,-1,-1,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3717E4,9.826537E7,7.593345E-3,2.081E3,1.4457831E-1,5.0826204E7,1.06748E6,1.2183896E-3,3.595572E6,3.0823356E5,-5.2711107E-3,2.9807162E0,2.1030048E10,-8.7363655E-3,-1.8839287E-3,1.2345679E-2,2.5302054E2,9.268261E-3,2.8352614E9,8.091679E-1,3.32656E-3,2.5549E4,5.01E2,4.1847788E6,5.1634945E-3,-1.1845741E-3,2.23597E-3,6.055513E2,-1.6212009E-3,-5.816462E-4,2.6581686E-3,-3.972148E-3,1.7107339E-4,1.2173996E-3,-2.8600174E-3,1.1328069E-3,-1.25724E-3],"split_indices":[2,45,0,2,57,45,1,0,1,33,0,56,5,0,0,57,52,0,5,27,0,1,0,50,0,0,0,52,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.84E2,6E0,1.66E2,1.8E1,1.2E2,4.6E1,6E0,1.2E1,1.11E2,9E0,1.8E1,2.8E1,6E0,6E0,5.1E1,6E1,8E0,1E1,2.3E1,5E0,1.6E1,3.5E1,5.5E1,5E0,5E0,5E0,1.5E1,8E0,5E0,1.1E1,1.4E1,2.1E1,4.6E1,9E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-6.7159176E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-5.0683957E-2,2.471375E-2,-7.1469086E-1,-4.211265E-2,4.9450392E-1,-3.8303677E-3,-9.356724E-1,-5.580517E-1,7.5147096E-3,4.142171E-2,7.9449415E-2,-2.2851696E-2,-5.449566E-2,-3.446924E-2,-1.196756E-2,4.0713497E-2,-2.8469712E-1,8.922781E-3,-1.9445838E-3,8.5911125E-2,-1.595332E-1,-2.2747863E-2,2.2966096E-3,-1.0979459E-1,1.2404155E-1,9.280069E-3,-2.6841623E-1,-1.1400769E-2,-8.741733E-4,1.1182567E-2,5.644807E-4,-1.9087E-2,-2.6889679E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[9.329678E0,5.2950816E0,2.5902376E0,3.793688E0,3.1571488E0,0E0,7.5346947E-1,4.6716928E-1,1.3309375E0,0E0,1.4857836E-1,0E0,0E0,0E0,0E0,1.1269388E0,9.081447E-1,0E0,0E0,7.673156E-1,9.366103E-1,0E0,0E0,1.9513953E-1,1.0409017E0,0E0,4.7435796E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,16,16,19,19,20,20,23,23,24,24,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,3.973E3,8.640031E9,3.9134244E-5,6.482745E7,-3.8303677E-3,2.6563678E5,1.097E3,9.4636065E-1,4.142171E-2,5.9220157E0,-2.2851696E-2,-5.449566E-2,-3.446924E-2,-1.196756E-2,2.8994E4,2.579497E8,8.922781E-3,-1.9445838E-3,9.189396E4,1.9024893E8,-2.2747863E-2,2.2966096E-3,1.4693E4,3.870239E5,9.280069E-3,1.594E3,-1.1400769E-2,-8.741733E-4,1.1182567E-2,5.644807E-4,-1.9087E-2,-2.6889679E-3],"split_indices":[44,2,5,41,44,0,32,2,26,0,55,0,0,0,0,9,7,0,0,31,7,0,0,9,32,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.85E2,1.67E2,1.8E1,1.47E2,2E1,5E0,1.3E1,1.2E1,1.35E2,1E1,1E1,5E0,8E0,7E0,5E0,1.22E2,1.3E1,5E0,5E0,1E2,2.2E1,8E0,5E0,1.6E1,8.4E1,5E0,1.7E1,6E0,1E1,4.2E1,4.2E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.230745E-2,1.0414932E-1,-5.010794E-1,3.4005634E-2,5.6172585E-1,-7.62362E-1,-7.547928E-2,-2.565942E-2,7.117637E-2,8.9466405E-1,1.4745463E-1,-9.130832E-1,-1.7198103E-2,-1.1619297E-2,4.9898042E-3,1.05104685E-1,-2.9100224E-1,5.056605E-2,2.6730007E-2,1.7217647E-2,-4.2664204E-3,-2.6734984E-2,-5.3462896E-2,-1.6213676E-1,1.3728411E-1,-2.3622975E-2,-3.2273104E-4,-1.42911635E-2,3.312105E-3,2.4434097E-1,2.3952302E-2,6.6767866E-3,1.8552445E-2,4.2158333E-3,-1.1787953E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[8.233663E0,5.3088512E0,2.8337073E0,3.1081257E0,2.9024653E0,6.4499474E-1,3.5534912E-1,0E0,1.7140567E0,6.0894012E-2,5.833674E-1,1.6950321E-1,0E0,0E0,0E0,1.1058182E0,6.860275E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6149853E-1,1.3651705E0,0E0,0E0,0E0,0E0,8.16133E-1,9.9003786E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,15,15,16,16,23,23,24,24,29,29,30,30],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[9.581563E11,3.973E3,2.1564245E0,8.960506E3,5.939257E7,3.7E1,8.0347336E4,-2.565942E-2,6.1061732E7,7.99E2,1.5E1,1E0,-1.7198103E-2,-1.1619297E-2,4.9898042E-3,5.94E2,1.2552425E6,5.056605E-2,2.6730007E-2,1.7217647E-2,-4.2664204E-3,-2.6734984E-2,-5.3462896E-2,1.6278E4,6.96E2,-2.3622975E-2,-3.2273104E-4,-1.42911635E-2,3.312105E-3,1.135E3,4.662977E5,6.6767866E-3,1.8552445E-2,4.2158333E-3,-1.1787953E-2],"split_indices":[30,2,53,32,44,8,32,0,44,0,3,80,0,0,0,2,31,0,0,0,0,0,0,9,0,0,0,0,0,2,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.65E2,2.5E1,1.44E2,2.1E1,1.5E1,1E1,8E0,1.36E2,1.1E1,1E1,1E1,5E0,5E0,5E0,1.25E2,1.1E1,6E0,5E0,5E0,5E0,5E0,5E0,1.3E1,1.12E2,6E0,5E0,8E0,5E0,5.7E1,5.5E1,3.4E1,2.3E1,4.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.0544179E-2,8.875575E-2,-7.398848E-1,-2.3204708E-3,6.6565776E-1,-4.3486863E-2,-1.613852E-2,7.2073705E-2,-2.4177684E-1,9.44416E-1,2.3850864E-1,-1.7552214E-2,1.0331505E-1,-3.366067E-1,4.113241E-3,2.5517184E-2,1.0831732E0,1.7368812E-2,2.5894907E-3,1.5693146E-1,-5.5736255E-2,-1.2722747E-1,-5.3803426E-1,3.2756265E-2,6.2380943E-2,1.0606914E-1,2.839202E-1,-1.8187533E-1,2.0160884E-1,-1.4508807E-2,2.9955446E-3,-1.3706747E-2,-3.3640802E-2,7.962405E-3,4.121306E-4,1.7426033E-2,-2.048489E-3,8.405352E-3,-1.565819E-2,1.4861025E-2,2.8466727E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,-1,23,-1,-1,25,27,29,31,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.57342E0,1.0404872E1,7.0878506E-1,3.081461E0,2.9896736E0,0E0,0E0,1.8459342E0,1.3027465E0,3.3729744E-1,2.5495237E-1,0E0,1.0661892E0,1.2811387E0,0E0,0E0,7.1621895E-2,0E0,0E0,5.705662E-1,1.0754284E0,5.854467E-1,4.6079874E-1,0E0,0E0,3.9710695E-1,6.934881E-1,1.1968935E0,1.5140972E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,12,12,13,13,16,16,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,-1,24,-1,-1,26,28,30,32,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,3.973E3,1E0,3.0396858E7,5.939257E7,-4.3486863E-2,-1.613852E-2,4.26E2,2.192884E0,5.666E3,2.0784883E0,-1.7552214E-2,1.9439217E0,4.3951338E11,4.113241E-3,2.5517184E-2,1.4781E4,1.7368812E-2,2.5894907E-3,3.0954462E2,4.6253732E2,1.895E3,2.5542571E8,3.2756265E-2,6.2380943E-2,8.182E3,1.898E3,3.58E2,1.7476E4,-1.4508807E-2,2.9955446E-3,-1.3706747E-2,-3.3640802E-2,7.962405E-3,4.121306E-4,1.7426033E-2,-2.048489E-3,8.405352E-3,-1.565819E-2,1.4861025E-2,2.8466727E-3],"split_indices":[44,2,111,44,44,0,0,2,53,2,53,0,55,30,0,0,9,0,0,4,4,2,7,0,0,10,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.97E2,1.4E1,1.71E2,2.6E1,9E0,5E0,1.31E2,4E1,1.5E1,1.1E1,8E0,1.23E2,3.1E1,9E0,5E0,1E1,6E0,5E0,9.2E1,3.1E1,1.6E1,1.5E1,5E0,5E0,6.7E1,2.5E1,2.1E1,1E1,8E0,8E0,7E0,8E0,4.1E1,2.6E1,2E1,5E0,6E0,1.5E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[7.14909E-5,-7.367273E-2,4.4007626E-1,-2.633183E-2,-7.943035E-1,8.512033E-1,7.56142E-2,5.1956424E-3,-2.5302224E-2,-1.8350258E-2,-5.1414963E-2,2.4242725E-2,5.015215E-2,1.9896E-2,-1.4741911E-1,7.1045555E-2,-1.7002335E-1,2.1733444E-3,-1.5121453E-2,1.4037002E-2,3.1658083E-1,4.5745276E-2,-2.7224022E-1,1.0740793E-1,-1.4456476E-1,2.524374E-2,2.047722E-1,-4.3178946E-3,7.010996E-3,-4.034405E-1,-6.963675E-2,-4.909596E-4,1.1806517E-2,1.3242443E-3,-1.1106085E-2,3.3911385E-3,1.2419809E-2,-1.479085E-2,-2.4345249E-2,5.763079E-3,-1.4912496E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,-1,-1,17,19,21,-1,-1,23,25,27,29,31,33,-1,35,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.781565E0,6.0857244E0,4.430844E0,2.6764405E0,8.5064554E-1,4.156742E-1,1.3491372E0,1.8581842E0,0E0,0E0,0E0,0E0,0E0,0E0,3.710366E-1,1.6332766E0,9.8288226E-1,0E0,0E0,1.4371717E0,4.0948606E-1,2.1821575E-1,7.693546E-1,1.0003574E0,5.47336E-1,0E0,9.842092E-2,0E0,0E0,2.1639347E-2,6.340637E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,-1,-1,18,20,22,-1,-1,24,26,28,30,32,34,-1,36,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.1855755E8,6.482745E7,4.476758E6,8.54374E5,1.4753E4,5.287008E5,2.67302E5,-2.5302224E-2,-1.8350258E-2,-5.1414963E-2,2.4242725E-2,5.015215E-2,1.9896E-2,5.9220157E0,4.6253732E2,3.0820766E-1,2.1733444E-3,-1.5121453E-2,5.657E4,1.0932244E9,2.697E3,1.6325E4,9.06E2,1.7111913E0,2.524374E-2,2.8352614E9,-4.3178946E-3,7.010996E-3,2.3005404E0,5.2726665E0,-4.909596E-4,1.1806517E-2,1.3242443E-3,-1.1106085E-2,3.3911385E-3,1.2419809E-2,-1.479085E-2,-2.4345249E-2,5.763079E-3,-1.4912496E-2],"split_indices":[2,44,44,27,28,9,31,28,0,0,0,0,0,0,55,4,26,0,0,28,5,2,9,2,53,0,5,0,0,55,55,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.78E2,2.9E1,1.68E2,1E1,1.3E1,1.6E1,1.59E2,9E0,5E0,5E0,6E0,7E0,6E0,1E1,1.16E2,4.3E1,5E0,5E0,9.5E1,2.1E1,1.4E1,2.9E1,6E1,3.5E1,6E0,1.5E1,6E0,8E0,1.7E1,1.2E1,3.3E1,2.7E1,1.2E1,2.3E1,5E0,1E1,1.1E1,6E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.3105734E-2,3.3917617E-2,-5.378899E-1,-1.463935E-2,4.987197E-1,-3.7297346E-3,-3.7451982E-2,4.624775E-2,-2.743497E-1,3.3244822E-2,8.047241E-3,-1.2655921E-1,1.00221716E-1,-3.6578873E-1,4.8535562E-4,-2.0073299E-1,4.8762416E-3,1.662886E-1,-4.8461746E-2,-2.7220068E-2,-1.9299005E-1,-2.48382E-1,-2.4237988E-5,3.215023E-1,8.2251705E-2,-2.4296811E-1,1.17883705E-1,-3.5932206E-3,-1.7125878E-2,-1.5706174E-2,-6.1416863E-3,8.9598E-3,1.953007E-2,-1.999043E-3,6.7944364E-3,2.1772254E-3,-1.8118775E-2,1.087723E-2,-1.3716562E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,19,-1,21,-1,23,25,-1,27,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1017284E0,4.2954826E0,1.8462272E0,2.748339E0,1.0455918E0,0E0,0E0,1.327802E0,8.6212254E-1,0E0,0E0,5.962887E-1,1.0689484E0,7.6462793E-1,0E0,2.4793053E-1,0E0,9.518316E-1,1.1287204E0,0E0,2.6429218E-1,1.5851068E-1,0E0,2.2499752E-1,3.7518588E-1,6.4121586E-1,3.1037417E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,20,-1,22,-1,24,26,-1,28,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,3.973E3,1.4689625E5,3.9712732E7,2.0118E4,-3.7297346E-3,-3.7451982E-2,7.38E2,9.6541266E2,3.3244822E-2,8.047241E-3,1.8149E4,2.126E4,1.1275881E5,4.8535562E-4,1.340973E-1,4.8762416E-3,5.5043226E8,1.7311496E0,-2.7220068E-2,3.2433453E5,1E0,-2.4237988E-5,1.131E3,1.9836358E5,1.4141012E0,4.7578287E-1,-3.5932206E-3,-1.7125878E-2,-1.5706174E-2,-6.1416863E-3,8.9598E-3,1.953007E-2,-1.999043E-3,6.7944364E-3,2.1772254E-3,-1.8118775E-2,1.087723E-2,-1.3716562E-3],"split_indices":[44,2,32,44,10,0,0,2,4,0,0,9,9,32,0,37,0,5,52,0,31,80,0,2,31,55,41,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.89E2,1.6E1,1.72E2,1.7E1,6E0,1E1,1.4E2,3.2E1,1E1,7E0,3.3E1,1.07E2,2.4E1,8E0,2.5E1,8E0,7.4E1,3.3E1,1E1,1.4E1,2E1,5E0,2.5E1,4.9E1,1.5E1,1.8E1,9E0,5E0,1.1E1,9E0,1.1E1,1.4E1,1.6E1,3.3E1,5E0,1E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.658149E-3,-8.5669294E-2,4.5224458E-1,5.4807942E-3,-5.227501E-1,8.260889E-1,9.481383E-2,2.4768563E-2,-2.0572197E-2,-6.902972E-1,-6.4858706E-3,2.0863598E-2,9.6404386E-1,-9.3566604E-2,1.9364879E-2,-1.3627768E-4,3.0552644E-1,-1.2061233E-2,-8.6026067E-1,2.7710496E-2,5.6963354E-2,3.4810742E-3,-1.3026494E-2,6.317388E-2,-1.2233644E-1,2.2833927E-2,4.001056E-3,-2.5003403E-2,-5.024592E-2,-7.2932154E-2,1.4571656E-1,-1.696894E-2,-3.1789118E-1,-6.5188236E-3,2.0904532E-3,1.6744744E-2,3.953099E-3,-6.344606E-3,5.567003E-3,-2.1772388E-2,-1.7867782E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,19,21,-1,23,25,-1,27,-1,-1,-1,-1,29,31,-1,-1,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9809074E0,6.6959777E0,4.331249E0,1.1849592E0,1.7887225E0,4.423294E-1,1.086925E0,9.439926E-1,0E0,1.2741423E0,0E0,0E0,2.6656723E-1,3.754934E-1,0E0,9.747925E-1,3.780203E-1,0E0,3.4571743E-1,0E0,0E0,0E0,0E0,9.430956E-1,8.806256E-1,0E0,0E0,0E0,0E0,2.4144478E-1,6.409569E-1,4.613E-1,5.5785644E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,12,12,13,13,15,15,16,16,18,18,23,23,24,24,29,29,30,30,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,20,22,-1,24,26,-1,28,-1,-1,-1,-1,30,32,-1,-1,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.013E3,7.6499896E9,5.6044944E7,4.476758E6,1.9191603E12,5.818E3,1.0083192E6,5.351875E2,-2.0572197E-2,8.0900437E-1,-6.4858706E-3,2.0863598E-2,1.4781E4,2.1030048E10,1.9364879E-2,2.1740167E0,2.674421E9,-1.2061233E-2,6.970116E7,2.7710496E-2,5.6963354E-2,3.4810742E-3,-1.3026494E-2,8.92E2,1.9577E4,2.2833927E-2,4.001056E-3,-2.5003403E-2,-5.024592E-2,1.8837452E0,3.08E2,9E0,1.2153666E6,-6.5188236E-3,2.0904532E-3,1.6744744E-2,3.953099E-3,-6.344606E-3,5.567003E-3,-2.1772388E-2,-1.7867782E-3],"split_indices":[2,5,44,27,30,2,27,51,0,26,0,0,9,5,0,55,5,0,44,0,0,0,0,2,9,0,0,0,0,52,0,3,46,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.67E2,3.2E1,1.39E2,2.8E1,1.5E1,1.7E1,1.34E2,5E0,1.9E1,9E0,5E0,1E1,1.1E1,6E0,1.24E2,1E1,6E0,1.3E1,5E0,5E0,6E0,5E0,8.2E1,4.2E1,5E0,5E0,6E0,7E0,3.1E1,5.1E1,2.8E1,1.4E1,2E1,1.1E1,1.1E1,4E1,1.5E1,1.3E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.6412618E-4,2.9746512E-2,-3.177863E-2,7.566148E-2,-4.213025E-1,1.741797E-2,4.8767343E-1,2.3707347E-3,-5.9297544E-1,2.453932E-1,-2.9302647E-2,3.3726763E-2,1.876982E-1,-1.4653362E-2,-4.0070232E-2,3.1074001E-3,2.916809E-1,3.335334E-2,-1.995446E-1,1.6215483E-2,2.7041623E-4,1.750941E-2,8.846719E-3,-2.6673706E-2,2.3808305E-1,-3.5163558E-1,-9.167898E-2,1.1175718E-3,-1.1573046E-2,1.9991705E-2,6.4005055E-3,-2.3686102E-2,-5.645601E-3,1.4264445E-3,-1.2099052E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,-1,19,-1,-1,-1,21,23,25,-1,-1,-1,-1,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1404233E0,4.256696E0,0E0,4.454952E0,1.5649507E0,1.7539783E0,1.3602915E0,0E0,7.226176E-1,2.1506047E-1,1.4679903E0,0E0,3.0002162E-1,0E0,0E0,0E0,1.0626316E-1,1.2475717E0,5.7476175E-1,0E0,0E0,0E0,0E0,8.544785E-1,3.7491035E-1,4.4241655E-1,4.5688963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,16,16,17,17,18,18,23,23,24,24,25,25,26,26],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,-1,20,-1,-1,-1,22,24,26,-1,-1,-1,-1,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,1.0388931E8,-3.177863E-2,4.774E3,5.29E2,2.0304577E-1,5.939257E7,2.3707347E-3,8.142301E-1,1.061E3,2.47538E5,3.3726763E-2,1E1,-1.4653362E-2,-4.0070232E-2,3.1074001E-3,2.65457E5,3.85024E2,3.0098013E2,1.6215483E-2,2.7041623E-4,1.750941E-2,8.846719E-3,2.054945E0,1.0932244E9,5.1412523E-2,5.5318156E7,1.1175718E-3,-1.1573046E-2,1.9991705E-2,6.4005055E-3,-2.3686102E-2,-5.645601E-3,1.4264445E-3,-1.2099052E-2],"split_indices":[27,44,0,2,0,26,44,0,26,2,28,0,3,0,0,0,1,4,51,0,0,0,0,55,5,38,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.03E2,8E0,1.85E2,1.8E1,1.63E2,2.2E1,5E0,1.3E1,2.7E1,1.36E2,1.2E1,1E1,7E0,6E0,6E0,2.1E1,1E2,3.6E1,5E0,5E0,1.1E1,1E1,7.8E1,2.2E1,1.4E1,2.2E1,6.4E1,1.4E1,7E0,1.5E1,8E0,6E0,1.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.7494125E-2,4.754708E-2,-3.5621274E-2,-1.0532961E-2,3.9438778E-1,7.360169E-2,-1.6759554E-1,6.925935E-1,2.5410628E-2,-2.1199968E-2,2.4291195E-1,-3.7742463E-1,-5.6034412E-2,4.7572453E-2,2.2314666E-2,-1.0107308E-2,1.398597E-2,-1.7981492E-1,6.339572E-2,3.3235803E-1,2.4692846E-2,-2.5587E-2,-7.2673513E-3,6.745743E-2,-3.2720587E-1,-1.7592138E-2,-2.9760548E-1,1.1502087E-2,-3.9779657E-4,3.899855E-1,2.2255937E-3,-6.638505E-3,8.835493E-3,-3.4677964E-3,1.3400528E-1,-6.2902095E-3,-2.557748E-2,-8.584453E-3,5.9132846E-3,-2.0052103E-2,-4.7711194E-3,-5.654876E-3,7.731384E-3,2.1913595E-2,1.1970141E-2,1.0740623E-2,2.6444825E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,-1,33,35,37,39,-1,41,43,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.815395E0,4.0733476E0,0E0,2.311642E0,3.1733027E0,1.8349516E0,1.4067862E0,6.2763596E-1,9.436025E-1,1.0043421E0,7.920656E-1,6.615987E-1,1.3950585E0,0E0,0E0,0E0,0E0,4.9158067E-1,5.524063E-1,4.596827E-1,3.6456668E-1,0E0,0E0,2.7957803E-1,4.5627367E-1,2.9557294E-1,3.1755733E-1,0E0,7.2319484E-1,1.1246085E-1,0E0,0E0,0E0,0E0,1.2976941E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,34,34],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,-1,34,36,38,40,-1,42,44,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,5.246E3,-3.5621274E-2,2.01351E5,6.482745E7,3.1546597E2,2.868989E2,1E0,8.039736E5,2.6727284E5,1.7476E4,2.7573213E8,9.149094E-1,4.7572453E-2,2.2314666E-2,-1.0107308E-2,1.398597E-2,1.8117242E8,4.898012E-1,3.706683E5,1.7775084E0,-2.5587E-2,-7.2673513E-3,1.7356589E0,1.4058E4,1.7440945E0,3.6327022E5,1.1502087E-2,1.4886476E0,1.5743027E7,2.2255937E-3,-6.638505E-3,8.835493E-3,-3.4677964E-3,1.7393E4,-6.2902095E-3,-2.557748E-2,-8.584453E-3,5.9132846E-3,-2.0052103E-2,-4.7711194E-3,-5.654876E-3,7.731384E-3,2.1913595E-2,1.1970141E-2,1.0740623E-2,2.6444825E-3],"split_indices":[44,2,0,28,44,4,51,80,27,32,9,7,26,0,0,0,0,7,26,32,52,0,0,52,10,53,31,0,52,44,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,2.01E2,7E0,1.73E2,2.8E1,1.13E2,6E1,1.5E1,1.3E1,7.3E1,4E1,2E1,4E1,5E0,1E1,7E0,6E0,2.5E1,4.8E1,2.8E1,1.2E1,1.1E1,9E0,2.8E1,1.2E1,1.1E1,1.4E1,1.2E1,3.6E1,2.3E1,5E0,6E0,6E0,9E0,1.9E1,7E0,5E0,5E0,6E0,8E0,6E0,2.1E1,1.5E1,1.4E1,9E0,8E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-3.846569E-3,4.2675298E-2,-6.763285E-1,1.9258281E-2,2.7171256E-2,-1.2538595E-2,-4.3248933E-2,1.0223437E-1,-1.0777823E-1,6.844068E-2,2.5130264E-2,-2.4064772E-2,-6.430109E-2,-8.152947E-4,1.6132517E-1,-1.752312E-1,6.6595204E-2,1.3770358E-1,-9.064334E-2,1.7832901E-2,1.0411781E-1,-2.746409E-1,-3.380403E-2,1.6328615E-1,-1.38092525E-2,6.875668E-5,1.1255376E-2,-1.1896266E-2,-6.8922905E-4,1.3330948E-2,2.1283384E-3,-1.746501E-2,-7.6540206E-3,4.820363E-3,-7.1638688E-3,-2.2508674E-3,1.1122382E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.285885E0,2.2980287E0,9.8408794E-1,1.9189385E0,0E0,0E0,0E0,1.533972E0,1.2027209E0,6.6271067E-1,0E0,0E0,9.7001195E-1,7.590547E-1,4.9211693E-1,4.9684834E-1,1.1106657E0,3.1765464E-1,4.4129378E-1,0E0,3.61857E-1,1.5865767E-1,2.6394644E-1,3.729565E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,9.772E3,8.950035E-1,2.00472E5,2.7171256E-2,-1.2538595E-2,-4.3248933E-2,2.297E3,6E0,1.059E3,2.5130264E-2,-2.4064772E-2,7.5614764E2,1.9024893E8,2.73E2,1.2137E4,9.799826E7,7.19E2,1.9606045E5,1.7832901E-2,1.3176E4,1.7142857E0,5.731563E2,1.3501E4,-1.38092525E-2,6.875668E-5,1.1255376E-2,-1.1896266E-2,-6.8922905E-4,1.3330948E-2,2.1283384E-3,-1.746501E-2,-7.6540206E-3,4.820363E-3,-7.1638688E-3,-2.2508674E-3,1.1122382E-2],"split_indices":[44,2,26,28,0,0,0,2,3,2,0,0,4,7,0,10,44,11,46,0,9,52,4,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.87E2,1.2E1,1.8E2,7E0,5E0,7E0,1.09E2,7.1E1,1.02E2,7E0,6E0,6.5E1,5.9E1,4.3E1,3.5E1,3E1,2.3E1,3.6E1,8E0,3.5E1,2E1,1.5E1,2.4E1,6E0,1E1,1.3E1,1.1E1,2.5E1,8E0,2.7E1,1E1,1E1,7E0,8E0,6E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.2346321E-2,7.2979055E-2,-3.959474E-1,1.723836E-2,4.9007657E-1,-5.9907186E-1,2.637014E-2,-2.2670643E-1,5.399271E-2,3.3464E-2,9.775284E-3,-7.985784E-1,-3.0883035E-1,-1.146209E-2,1.5899139E-2,-5.1012635E-3,-1.6054185E-2,9.682388E-2,-1.289906E-1,-4.915117E-2,-2.5306784E-2,-3.6793328E-3,-2.3445839E-2,-9.5871175E-3,1.2576337E-1,8.648533E-3,-2.488495E-1,1.6949372E-1,-4.426913E-2,-1.7645787E-2,-3.2769414E-3,3.0645367E-3,1.4927365E-2,-9.206193E-3,6.7746416E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,-1,27,-1,29,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[6.116844E0,3.5486958E0,2.9324207E0,1.2316209E0,9.3057585E-1,1.0851917E0,1.05315E0,1.9889551E-1,9.451388E-1,0E0,0E0,2.5712872E-1,4.2326367E-1,0E0,0E0,0E0,0E0,8.5870993E-1,8.981746E-1,0E0,0E0,0E0,0E0,0E0,6.692697E-1,0E0,3.4131098E-1,1.0446532E0,5.48923E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,17,17,18,18,24,24,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,-1,28,-1,30,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3365464E7,3.973E3,1.99607E3,5.7685807E-5,1.285E3,1.9006333E0,4.11591E5,2.5927516E11,2.5965E5,3.3464E-2,9.775284E-3,7.567149E4,8E0,-1.146209E-2,1.5899139E-2,-5.1012635E-3,-1.6054185E-2,4.75E2,1.8866123E8,-4.915117E-2,-2.5306784E-2,-3.6793328E-3,-2.3445839E-2,-9.5871175E-3,2.72126E5,8.648533E-3,9.120157E6,1.137E3,1.9476662E0,-1.7645787E-2,-3.2769414E-3,3.0645367E-3,1.4927365E-2,-9.206193E-3,6.7746416E-3],"split_indices":[44,2,4,37,0,52,27,30,9,0,0,31,3,0,0,0,0,2,7,0,0,0,0,0,28,0,49,2,53,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.85E2,1.52E2,3.3E1,1.35E2,1.7E1,2.2E1,1.1E1,1.7E1,1.18E2,9E0,8E0,1.2E1,1E1,6E0,5E0,9E0,8E0,9.6E1,2.2E1,5E0,7E0,5E0,5E0,8E0,8.8E1,6E0,1.6E1,7E1,1.8E1,9E0,7E0,4.1E1,2.9E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.9049242E-2,-4.8035493E-1,5.9438504E-2,-2.916009E-2,-8.932367E-3,2.7623249E-3,3.4090203E-1,-2.8239418E-2,3.0565694E-2,3.88462E-2,1.4218977E-1,5.048788E-2,-2.168624E-2,1.7575439E-2,-2.8442191E-2,1.1093366E-1,-4.3717675E-2,7.2735664E-3,-6.968673E-3,2.4444629E-2,2.189726E-1,-1.1726413E-1,8.794597E-2,-4.637495E-3,4.5833588E-3,1.347305E-2,4.942358E-4,7.290213E-3,-9.430183E-3,9.948807E-3,-3.0905432E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,-1,-1,7,9,-1,11,-1,13,15,-1,-1,17,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0424943E0,4.52183E-1,2.95023E0,0E0,0E0,2.5701554E0,2.7795877E0,0E0,1.4572673E0,0E0,8.921429E-1,8.24833E-1,0E0,0E0,3.173986E-1,8.169353E-1,5.6226635E-1,0E0,0E0,4.410391E-1,5.036887E-1,8.2323015E-1,3.966793E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,10,10,11,11,14,14,15,15,16,16,19,19,20,20,21,21,22,22],"right_children":[2,4,6,-1,-1,8,10,-1,12,-1,14,16,-1,-1,18,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.171708E-5,1.119E3,4.376E3,-2.916009E-2,-8.932367E-3,6E0,4.083812E7,-2.8239418E-2,1.443266E6,3.88462E-2,5.287008E5,1.33358E5,-2.168624E-2,1.7575439E-2,1.14416916E2,2.3103847E2,2.1296928E0,7.2735664E-3,-6.968673E-3,2.6727284E5,1.7315E4,1.4163916E0,5.509307E5,-4.637495E-3,4.5833588E-3,1.347305E-2,4.942358E-4,7.290213E-3,-9.430183E-3,9.948807E-3,-3.0905432E-3],"split_indices":[41,0,2,0,0,3,44,0,28,0,31,28,0,0,57,54,53,0,0,32,9,52,31,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.4E1,1.84E2,9E0,5E0,1.54E2,3E1,6E0,1.48E2,8E0,2.2E1,1.43E2,5E0,9E0,1.3E1,8.7E1,5.6E1,5E0,8E0,4.9E1,3.8E1,3.6E1,2E1,1.8E1,3.1E1,2.9E1,9E0,8E0,2.8E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.7406095E-2,-3.1569876E-2,3.435455E-1,1.0490399E-2,-5.0699943E-1,-8.800076E-2,5.437029E-1,4.9816135E-2,-1.6751517E-1,-9.425468E-3,-4.2537298E-2,4.3461984E-3,-1.2075476E-2,4.026478E-2,3.4918848E-1,2.0733293E-2,2.9138464E-1,-3.0736335E-2,-2.3788634E-1,2.2252733E-2,1.0574049E-2,4.6955746E-2,-2.3860836E-1,1.8803617E-2,9.179426E-3,4.9352003E-3,-7.6348325E-3,-1.7087992E-2,-3.1744582E-3,-1.0004239E-2,6.950604E-2,-5.8404566E-3,-1.511698E-2,6.420985E-3,-1.4672048E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,19,21,23,25,27,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1],"loss_changes":[3.7957053E0,3.4447508E0,2.8119166E0,1.1213769E0,1.479764E0,3.453808E-1,9.76418E-1,9.14299E-1,2.7182156E-1,0E0,0E0,0E0,0E0,0E0,1.196723E-1,8.1367177E-1,6.6036224E-2,2.0565866E-1,3.6374104E-1,0E0,0E0,6.3743985E-1,6.0289085E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.501559E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,14,14,15,15,16,16,17,17,18,18,21,21,22,22,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,20,22,24,26,28,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1],"split_conditions":[3.847E3,9.1131825E11,1.2047E4,3.00037E5,4.699E3,1.0388931E8,1E0,2.001E3,2.7234016E5,-9.425468E-3,-4.2537298E-2,4.3461984E-3,-1.2075476E-2,4.026478E-2,7.64E2,2.691667E7,6.505172E5,2.5927516E11,2.4229807E-1,2.2252733E-2,1.0574049E-2,4.75E2,2.0946825E6,1.8803617E-2,9.179426E-3,4.9352003E-3,-7.6348325E-3,-1.7087992E-2,-3.1744582E-3,-1.0004239E-2,7.134E3,-5.8404566E-3,-1.511698E-2,6.420985E-3,-1.4672048E-3],"split_indices":[2,30,9,28,10,44,63,2,31,0,0,0,0,0,0,44,31,30,40,0,0,2,27,0,0,0,0,0,0,0,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.71E2,3.1E1,1.58E2,1.3E1,1E1,2.1E1,1.3E2,2.8E1,8E0,5E0,5E0,5E0,7E0,1.4E1,1.17E2,1.3E1,1E1,1.8E1,6E0,8E0,1.07E2,1E1,5E0,8E0,5E0,5E0,1E1,8E0,8E0,9.9E1,5E0,5E0,6E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.7527538E-3,4.5091048E-2,-3.39698E-1,-5.119068E-3,3.6392906E-1,-3.4445386E-2,-1.943968E-1,2.0665696E-2,-2.4766281E-2,5.661521E-1,1.5015559E-1,1.2902547E-2,-2.1804703E-2,6.9899805E-2,-9.814537E-2,1.3225541E-2,3.4156375E-2,1.2338285E-2,-1.7221158E-3,8.632894E-3,-7.4996375E-3,2.454633E-1,3.405132E-2,-1.6444752E-1,1.0893984E-1,1.0092336E-3,3.4741086E-1,-3.4778193E-2,1.2956136E-1,2.390233E-4,-2.211112E-1,7.153902E-4,8.333883E-3,9.820453E-3,2.1089602E-2,-3.604042E-3,1.2413836E-2,-1.4417392E-3,8.804729E-3,-1.3778535E-2,-2.6911895E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,25,27,29,31,-1,33,35,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0394206E0,3.0595617E0,1.1277821E0,2.2014642E0,1.0389671E0,0E0,9.7878325E-1,9.369399E-1,0E0,3.5679817E-1,2.890378E-1,3.4002376E-1,0E0,7.02828E-1,6.63234E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.273399E-1,6.290273E-1,3.4673566E-1,7.074499E-2,0E0,8.158159E-2,6.852175E-1,3.5180014E-1,0E0,2.7994883E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,13,13,14,14,21,21,22,22,23,23,24,24,26,26,27,27,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,26,28,30,32,-1,34,36,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,4.376E3,6.1978314E2,4.476758E6,5.939257E7,-3.4445386E-2,2.01E3,2.493493E7,-2.4766281E-2,5.818E3,1.3E1,1.673295E6,-2.1804703E-2,1.3176E4,1.0141866E3,1.3225541E-2,3.4156375E-2,1.2338285E-2,-1.7221158E-3,8.632894E-3,-7.4996375E-3,2.0260053E5,1.7206225E0,6.50596E5,5.752585E5,1.0092336E-3,5.56E2,1.993E3,1.4176E4,2.390233E-4,3.6E1,7.153902E-4,8.333883E-3,9.820453E-3,2.1089602E-2,-3.604042E-3,1.2413836E-2,-1.4417392E-3,8.804729E-3,-1.3778535E-2,-2.6911895E-3],"split_indices":[44,2,51,27,44,0,0,44,0,2,3,31,0,9,4,0,0,0,0,0,0,46,52,27,27,0,0,2,28,0,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.9E2,2.2E1,1.65E2,2.5E1,5E0,1.7E1,1.58E2,7E0,1.2E1,1.3E1,1E1,7E0,1.12E2,4.6E1,5E0,7E0,8E0,5E0,5E0,5E0,1.8E1,9.4E1,3.5E1,1.1E1,6E0,1.2E1,5.5E1,3.9E1,9E0,2.6E1,5E0,6E0,6E0,6E0,4.9E1,6E0,1E1,2.9E1,1.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-9.676875E-3,-4.702332E-2,5.031941E-1,-1.8553495E-2,-3.351236E-2,3.3279005E-2,6.3316072E-3,-3.0306152E-1,1.5836349E-2,-3.9179197E-1,-1.4719908E-3,1.1068377E-1,-4.786979E-2,-1.2056768E-2,-2.4720086E-2,7.381449E-2,2.2671139E-2,-1.3672525E-1,1.00702934E-1,-8.401744E-3,1.087176E-1,-1.7106122E-1,7.7803046E-3,-5.187583E-3,1.432713E-1,4.5634908E-4,8.1301965E-3,9.229902E-4,-1.1185455E-2,9.712743E-3,-1.4957057E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,15,17,-1,-1,19,-1,21,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[3.950213E0,3.5536487E0,9.026501E-1,1.8148739E0,0E0,0E0,0E0,4.7125244E-1,1.0083307E0,1.3119459E-1,0E0,8.606195E-1,1.3339183E0,0E0,0E0,5.5830216E-1,0E0,6.7483747E-1,3.5356632E-1,0E0,3.3109432E-1,6.8161535E-1,0E0,0E0,3.3836383E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,11,11,12,12,15,15,17,17,18,18,20,20,21,21,24,24],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,16,18,-1,-1,20,-1,22,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3397336E3,1.2754595E8,1.9160621E0,2.1103053E-5,-3.351236E-2,3.3279005E-2,6.3316072E-3,3.4572864E0,1.8949672E0,2.1816352E6,-1.4719908E-3,2.902E3,1.340973E-1,-1.2056768E-2,-2.4720086E-2,4.75E2,2.2671139E-2,1.714168E6,2.2786127E5,-8.401744E-3,3.322259E-2,1.7141459E0,7.7803046E-3,-5.187583E-3,3.7490938E5,4.5634908E-4,8.1301965E-3,9.229902E-4,-1.1185455E-2,9.712743E-3,-1.4957057E-3],"split_indices":[51,44,53,37,0,0,0,55,57,27,0,2,37,0,0,2,0,27,27,0,56,53,0,0,32,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.91E2,1.3E1,1.84E2,7E0,8E0,5E0,1.9E1,1.65E2,1.4E1,5E0,6.6E1,9.9E1,8E0,6E0,6.1E1,5E0,6.2E1,3.7E1,7E0,5.4E1,5.6E1,6E0,6E0,3.1E1,2.1E1,3.3E1,1.4E1,4.2E1,2.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[6.5133916E-3,8.09256E-2,-1.5444407E-1,1.2799028E-1,-1.23670384E-1,2.7883213E-2,-3.172873E-1,9.7439356E-2,2.9092314E-2,1.2076002E-2,-2.1631126E-1,-1.794678E-1,1.2583877E-1,-3.8334322E-1,3.6833314E-3,-2.8249985E-4,1.6851407E-1,-1.3325632E-3,-3.1498834E-1,-1.0891992E-2,-4.8710145E-3,9.707346E-3,2.5164512E-1,-1.5811665E-1,-5.846109E-1,6.682559E-2,-1.09759785E-1,4.0213963E-1,9.1915585E-2,-2.193155E-2,-8.093075E-3,-3.1746507E-3,3.136715E-3,1.6024046E-2,6.0784356E-3,1.2088699E-3,-1.1677719E-2,-3.2342143E-2,-1.5393358E-2,-7.1081676E-2,1.5766606E-1,-1.4831737E-2,6.2818667E-3,1.3849405E-2,2.5984999E-2,-6.2484745E-2,1.4102556E-1,1.6948801E-3,-8.019203E-3,9.238251E-3,2.363558E-3,-3.0282736E-3,2.760255E-3,-8.085439E-3,4.2945826E-3,1.00351125E-2,8.41192E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,21,23,-1,25,27,-1,29,-1,-1,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5756192E0,1.4313273E0,2.0278187E0,1.6978472E0,1.0291605E0,6.9285315E-1,9.668145E-1,7.918408E-1,0E0,0E0,4.1904497E-1,1.7859489E-2,3.335888E-1,1.3128185E0,0E0,3.6732855E-1,1.1441311E0,0E0,2.3551333E-1,0E0,0E0,5.9415333E-2,7.12409E-2,2.573679E-1,1.6423845E-1,3.9956155E-1,4.425193E-1,9.690857E-2,3.955243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3920456E-1,6.286797E-2,0E0,5.0016046E-2,0E0,0E0,2.2532097E-1,3.2587248E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,12,12,13,13,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,39,39,40,40,42,42,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,22,24,-1,26,28,-1,30,-1,-1,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.595345E-1,2.5086E4,1.021246E0,1.0086E4,1.9024893E8,1.2399164E5,8.853438E8,1.137E3,2.9092314E-2,1.2076002E-2,2.5979605E0,5.94E2,3.2935953E2,4.3951338E11,3.6833314E-3,5.14E2,3.6004024E10,-1.3325632E-3,5.6216806E5,-1.0891992E-2,-4.8710145E-3,1.9107901E0,1.9173932E6,9.1300315E-1,1.9378502E0,7.23E2,1.3371641E6,1.594E3,1.3345E4,-2.193155E-2,-8.093075E-3,-3.1746507E-3,3.136715E-3,1.6024046E-2,6.0784356E-3,1.2088699E-3,-1.1677719E-2,-3.2342143E-2,-1.5393358E-2,5.9E1,1.289408E7,-1.4831737E-2,3.9232688E5,1.3849405E-2,2.5984999E-2,3.3487454E11,1.0016339E6,1.6948801E-3,-8.019203E-3,9.238251E-3,2.363558E-3,-3.0282736E-3,2.760255E-3,-8.085439E-3,4.2945826E-3,1.00351125E-2,8.41192E-4],"split_indices":[26,9,55,2,7,32,7,2,0,0,55,2,4,30,0,0,30,0,27,0,0,52,46,26,52,2,50,2,9,0,0,0,0,0,0,0,0,0,0,10,44,0,27,0,0,30,31,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.46E2,6.7E1,1.19E2,2.7E1,3.2E1,3.5E1,1.13E2,6E0,5E0,2.2E1,1E1,2.2E1,3E1,5E0,4.8E1,6.5E1,8E0,1.4E1,5E0,5E0,1.2E1,1E1,1.5E1,1.5E1,3E1,1.8E1,1.5E1,5E1,6E0,8E0,5E0,7E0,5E0,5E0,5E0,1E1,1E1,5E0,1.2E1,1.8E1,6E0,1.2E1,1E1,5E0,1.2E1,3.8E1,6E0,6E0,1.3E1,5E0,5E0,7E0,7E0,5E0,2.4E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.1821186E-2,3.3352602E-2,-4.40673E-1,-1.2966474E-2,3.7364328E-1,-2.2973595E-3,-6.4692664E-1,4.9243286E-2,-2.2766404E-1,2.5005925E-1,3.0705739E-2,-1.5479926E-2,-3.7201814E-2,-1.6238444E-2,1.5635999E-1,-3.0522484E-1,1.4523114E-3,1.6258791E-2,5.6687035E-3,-1.442857E-2,8.069018E-3,3.086278E-1,1.6109893E-2,-2.3043336E-1,-2.532384E-2,-1.5009213E-2,9.183369E-3,3.7204307E-1,3.7135356E-3,8.1505515E-2,-6.1527487E-2,-3.156817E-1,-1.1648669E-1,8.841852E-3,-1.6791756E-3,9.670926E-3,2.13919E-2,7.066632E-3,-1.488763E-3,-7.6162666E-3,2.9902852E-3,-1.9948233E-2,-8.828004E-3,-1.9563122E-3,-8.274941E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,33,-1,35,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.567297E0,2.763186E0,1.831975E0,2.0814347E0,5.3340197E-1,0E0,3.7855434E-1,8.512558E-1,7.0956826E-1,1.480962E-1,0E0,0E0,0E0,5.28804E-1,9.773848E-1,3.2925105E-1,0E0,0E0,0E0,0E0,3.037439E-1,2.9359746E-1,1.3194342E-1,1.7693007E-1,0E0,2.624322E-1,0E0,1.2532306E-1,0E0,1.0860463E-1,1.548607E-1,8.2075715E-2,3.9746493E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,13,13,14,14,15,15,20,20,21,21,22,22,23,23,25,25,27,27,29,29,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,34,-1,36,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,7.7186523E2,7.328832E-1,3.9882228E7,2.4450078E3,-2.2973595E-3,2.409E3,1.222E3,1.0141866E3,2.3734754E7,3.0705739E-2,-1.5479926E-2,-3.7201814E-2,8.53386E3,7.11E2,2.6654E4,1.4523114E-3,1.6258791E-2,5.6687035E-3,-1.442857E-2,2.0487762E2,4.9321375E5,3.829552E5,2.2243262E5,-2.532384E-2,1.3541759E9,9.183369E-3,3.1274893E2,3.7135356E-3,2.447064E7,1.875E3,9.190713E-1,8.2841175E5,8.841852E-3,-1.6791756E-3,9.670926E-3,2.13919E-2,7.066632E-3,-1.488763E-3,-7.6162666E-3,2.9902852E-3,-1.9948233E-2,-8.828004E-3,-1.9563122E-3,-8.274941E-3],"split_indices":[44,51,26,44,51,0,2,2,4,47,0,0,0,32,0,10,0,0,0,0,51,31,32,32,0,30,0,4,0,44,2,26,47,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.74E2,2.2E1,1.54E2,2E1,8E0,1.4E1,1.2E2,3.4E1,1.5E1,5E0,5E0,9E0,7.5E1,4.5E1,2.6E1,8E0,8E0,7E0,5E0,7E1,2.1E1,2.4E1,2.1E1,5E0,6.3E1,7E0,1.6E1,5E0,1.3E1,1.1E1,1.1E1,1E1,5E0,5.8E1,6E0,1E1,8E0,5E0,6E0,5E0,5E0,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[3.0091282E-2,6.145293E-2,-3.1156203E-2,1.6833354E-2,3.485918E-1,2.898522E-2,-1.708431E-2,3.07013E-2,1.610519E-1,6.2482465E-2,-8.49575E-2,2.7159026E-1,-5.926881E-3,3.4729928E-2,2.6450232E-1,-1.793216E-1,6.4361565E-2,6.7008883E-3,1.5772756E-2,9.4155E-2,-3.2206904E-2,1.6666839E-2,7.268907E-3,-2.6133847E-1,2.2213887E-4,-1.2882438E-3,8.1626475E-3,7.824576E-3,-1.01181235E-4,7.5823925E-3,-4.753267E-3,-3.687387E-3,-1.5633916E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,23,25,-1,-1,27,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.841265E0,2.7615275E0,0E0,8.634732E-1,1.4861982E0,7.0377684E-1,0E0,0E0,6.3167053E-1,7.86946E-1,6.0333645E-1,6.0240507E-2,0E0,5.0422597E-1,1.0708821E-1,3.9349157E-1,1.6904579E-1,0E0,0E0,4.4647658E-1,7.815173E-1,0E0,0E0,1.9056845E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,20,20,23,23],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,24,26,-1,-1,28,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,4.796E3,-3.1156203E-2,4.674595E6,4.083812E7,3.08891E5,-1.708431E-2,3.07013E-2,1.3E1,1.875E3,2.1564245E0,4.348354E-1,-5.926881E-3,2.459517E0,5E0,2.1996392E-1,1.0908286E3,6.7008883E-3,1.5772756E-2,6.805556E5,1.3257E4,1.6666839E-2,7.268907E-3,1.861058E0,2.2213887E-4,-1.2882438E-3,8.1626475E-3,7.824576E-3,-1.01181235E-4,7.5823925E-3,-4.753267E-3,-3.687387E-3,-1.5633916E-2],"split_indices":[44,2,0,27,44,28,0,0,3,2,53,26,0,57,8,38,4,0,0,27,9,0,0,55,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E2,2.15E2,9E0,1.87E2,2.8E1,1.82E2,5E0,1E1,1.8E1,1.41E2,4.1E1,1.3E1,5E0,1.25E2,1.6E1,2.5E1,1.6E1,5E0,8E0,6.6E1,5.9E1,8E0,8E0,1.7E1,8E0,9E0,7E0,3.8E1,2.8E1,1.5E1,4.4E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.2112274E-3,4.8081707E-2,-3.1884396E-1,-2.8697753E-3,4.2340034E-1,-3.0472731E-2,-6.496023E-2,-2.3495965E-1,2.0514237E-2,3.6657996E-2,2.1289678E-1,6.562134E-3,-1.5105893E-2,6.6261686E-4,-1.728236E-2,-3.4026064E-2,1.2290722E-1,1.5764516E-2,4.7379285E-3,7.438103E-2,-1.06945686E-1,9.727862E-2,1.4524853E-2,-2.9558819E-3,1.9057114E-1,-2.6383072E-1,-1.3303149E-2,1.8685614E-1,1.0976157E-2,5.744039E-3,-4.448314E-3,1.1160188E-2,3.8861774E-3,-4.7106883E-3,-1.4694652E-2,3.3241983E-3,-8.511366E-3,3.115662E-3,1.5035602E-2,-5.076677E-3,5.2904543E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9632285E0,3.5198545E0,1.9721012E0,8.894524E-1,1.4198577E0,0E0,8.014912E-1,4.7437644E-1,8.362791E-1,0E0,1.6609162E-1,0E0,0E0,0E0,0E0,7.8264517E-1,2.1075445E-1,0E0,0E0,3.599947E-1,8.6326575E-1,3.6129475E-1,0E0,2.872266E-1,5.5318475E-2,1.2601972E-1,5.287694E-1,3.353905E-1,3.0245772E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,16,16,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.826537E7,4.774E3,9.1922754E2,3.0725318E-5,1E0,-3.0472731E-2,1.74E3,6.342E3,1.9107901E0,3.6657996E-2,1.203E3,6.562134E-3,-1.5105893E-2,6.6261686E-4,-1.728236E-2,1.4170854E0,5.8018835E2,1.5764516E-2,4.7379285E-3,2.3873117E2,8E0,4.720179E10,1.4524853E-2,5.3641737E-1,1E1,1.7046307E0,2.67302E5,2.6538E4,6E0,5.744039E-3,-4.448314E-3,1.1160188E-2,3.8861774E-3,-4.7106883E-3,-1.4694652E-2,3.3241983E-3,-8.511366E-3,3.115662E-3,1.5035602E-2,-5.076677E-3,5.2904543E-3],"split_indices":[44,2,51,41,80,0,0,12,52,0,0,0,0,0,0,52,51,0,0,4,3,30,0,26,3,53,28,28,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.83E2,2.4E1,1.62E2,2.1E1,1E1,1.4E1,1.4E1,1.48E2,7E0,1.4E1,8E0,6E0,5E0,9E0,9.7E1,5.1E1,6E0,8E0,3.9E1,5.8E1,4.6E1,5E0,2.4E1,1.5E1,2.1E1,3.7E1,2.2E1,2.4E1,1E1,1.4E1,1E1,5E0,5E0,1.6E1,2.5E1,1.2E1,1.2E1,1E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.7687019E-2,4.0660083E-2,-2.9475197E-1,-5.7623894E-3,4.4846636E-1,-4.96775E-1,-7.721567E-2,-1.931599E-2,1.2458072E-2,3.215538E-2,4.1282247E-3,-1.1024078E-2,-6.6979593E-1,5.9929606E-2,-1.8413667E-2,-2.3049511E-1,3.6664404E-2,-3.9873138E-2,-1.8956346E-2,8.688145E-3,-6.3329204E-3,-3.4587167E-3,-1.4871624E-2,1.4300562E-1,2.531077E-3,6.749068E-2,2.1455054E-1,-7.602184E-2,4.6893872E-2,7.560791E-3,-2.0779386E-4,6.0982113E-3,1.4219599E-2,-8.402338E-3,1.8109465E-3,-5.0916146E-3,3.880823E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,-1,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.439163E0,3.3345063E0,1.587693E0,1.1563815E0,1.3945937E0,7.2251654E-1,8.233579E-1,0E0,9.0824485E-1,0E0,0E0,0E0,1.6061783E-1,3.5054892E-1,0E0,1.5471596E-1,5.078072E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6921198E-1,3.7644148E-1,1.1746888E-1,8.424455E-2,4.4770265E-1,3.691272E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,12,12,13,13,15,15,16,16,23,23,24,24,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,-1,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8351054E11,3.965E3,2.1564245E0,3.0725318E-5,9E0,1.0463519E5,6.3018E4,-1.931599E-2,4.75E2,3.215538E-2,4.1282247E-3,-1.1024078E-2,3.1696565E-2,5.0797403E5,-1.8413667E-2,2.542029E-3,3.58E2,-3.9873138E-2,-1.8956346E-2,8.688145E-3,-6.3329204E-3,-3.4587167E-3,-1.4871624E-2,3.322259E-2,2.6477316E-1,1.5316E4,1.3723022E0,1.8270676E0,2.5256923E5,7.560791E-3,-2.0779386E-4,6.0982113E-3,1.4219599E-2,-8.402338E-3,1.8109465E-3,-5.0916146E-3,3.880823E-3],"split_indices":[30,2,53,41,3,32,10,0,2,0,0,0,37,31,0,38,0,0,0,0,0,0,0,56,40,9,55,52,27,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.75E2,3.6E1,1.58E2,1.7E1,1.8E1,1.8E1,6E0,1.52E2,1E1,7E0,8E0,1E1,1.3E1,5E0,1.3E1,1.39E2,5E0,5E0,8E0,5E0,5E0,8E0,3.3E1,1.06E2,1.7E1,1.6E1,3.8E1,6.8E1,7E0,1E1,9E0,7E0,2E1,1.8E1,1.2E1,5.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.328289E-2,2.0352677E-2,-4.3172076E-1,-5.887703E-3,4.40734E-1,-3.639438E-3,-3.5141233E-2,8.525475E-2,-7.068245E-2,2.8141689E-2,1.0568837E-2,-3.6925445E-3,1.7924613E-1,-1.1450928E-1,7.239069E-2,-1.4711688E-1,7.367797E-2,2.3025775E-1,-3.1131662E-3,-7.100218E-2,-2.7156484E-1,1.5453368E-1,-3.1752295E-3,-1.1038821E-2,1.12605645E-4,1.5254448E-1,5.672894E-3,2.56348E-1,3.9846315E-3,-1.12993615E-2,-2.1180889E-1,-2.006175E-2,-6.0679745E-3,2.860321E-3,9.843298E-3,3.167208E-3,9.813512E-3,5.1503284E-3,-2.6825285E-3,6.939849E-3,1.3602696E-2,-7.888598E-3,1.4590443E-3,-1.6870264E-2,1.2052925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,-1,-1,-1,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.83774E0,2.0653856E0,1.5547178E0,1.0513088E0,2.2557116E-1,0E0,0E0,6.193224E-1,6.604153E-1,0E0,0E0,4.436151E-1,4.6346724E-1,5.273608E-1,2.9500696E-1,1.7484736E-1,1.3858105E-1,9.380424E-2,0E0,5.356264E-1,3.0954695E-1,6.500396E-2,0E0,0E0,0E0,4.0699244E-2,1.0021178E-1,3.5754085E-2,0E0,2.995745E-1,6.406658E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,25,25,26,26,27,27,29,29,30,30],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,-1,-1,-1,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855755E8,8.015E3,8.950035E-1,8.0683605E6,1.0915004E7,-3.639438E-3,-3.5141233E-2,1.059E3,9.6541266E2,2.8141689E-2,1.0568837E-2,2.5934292E5,2.3701E4,5.5318156E7,4.1124488E9,5.514485E5,1E0,4.6385403E5,-3.1131662E-3,2.1282E4,1.4693E4,1.4041E4,-3.1752295E-3,-1.1038821E-2,1.12605645E-4,1.188049E8,2.585911E5,1.945E2,3.9846315E-3,7.7E1,1.808848E0,-2.006175E-2,-6.0679745E-3,2.860321E-3,9.843298E-3,3.167208E-3,9.813512E-3,5.1503284E-3,-2.6825285E-3,6.939849E-3,1.3602696E-2,-7.888598E-3,1.4590443E-3,-1.6870264E-2,1.2052925E-3],"split_indices":[44,2,26,44,46,0,0,2,4,0,0,31,9,44,12,27,111,32,0,9,9,9,0,0,0,5,27,4,0,10,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.86E2,1.4E1,1.76E2,1E1,7E0,7E0,7.3E1,1.03E2,5E0,5E0,3.8E1,3.5E1,7.9E1,2.4E1,1.3E1,2.5E1,2.9E1,6E0,6.3E1,1.6E1,1.5E1,9E0,8E0,5E0,1.1E1,1.4E1,2.4E1,5E0,4.5E1,1.8E1,7E0,9E0,6E0,9E0,5E0,6E0,5E0,9E0,6E0,1.8E1,9E0,3.6E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.983319E-3,-3.524799E-1,2.722492E-2,-6.366662E-3,-2.4592336E-2,6.5453954E-2,-1.6954966E-1,2.4516119E-2,2.7226755E-1,7.8222625E-2,-2.85729E-1,-1.2288755E-1,4.485502E-2,-2.0336143E-3,3.6420763E-1,1.0981665E-2,-4.1112197E-3,-4.3292966E-1,-9.917107E-2,-1.6136112E-3,-1.0493579E-2,-8.683938E-2,7.309682E-2,6.7098835E-3,4.2550126E-1,-1.2543922E-2,-2.4803964E-2,-8.54645E-3,-1.6393885E-4,-1.2391428E-2,-1.6016084E-1,1.6963375E-1,1.4635887E-2,2.4129942E-2,1.36239305E-2,-3.7315574E-3,3.1661785E-3,-2.759706E-3,-1.1307532E-2,5.099239E-3,1.0886122E-2,3.2962817E-3,-3.2184385E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,-1,-1,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,-1,-1,29,31,-1,33,-1,-1,-1,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5011991E0,3.2028794E-1,1.4779855E0,0E0,0E0,1.3818312E0,9.43503E-1,4.1924015E-1,7.967043E-1,2.921355E-1,5.612178E-1,1.3843027E-1,4.6010065E-1,0E0,2.2892523E-1,0E0,0E0,3.8566113E-2,8.282796E-2,0E0,0E0,1.1810528E-1,5.6752485E-1,0E0,3.327322E-2,0E0,0E0,0E0,0E0,6.683082E-2,7.198769E-2,1.1412513E-1,2.8825465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,17,17,18,18,21,21,22,22,24,24,29,29,30,30,31,31,32,32],"right_children":[2,4,6,-1,-1,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,-1,-1,30,32,-1,34,-1,-1,-1,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0287E4,1.5947007E0,2.68225E5,-6.366662E-3,-2.4592336E-2,3.625E3,6.5229E4,1.545823E-4,1.2047E4,1.065E3,8E0,7.017544E-3,7.53E2,-2.0336143E-3,7.381306E-2,1.0981665E-2,-4.1112197E-3,4.4818586E-1,1.1184356E-1,-1.6136112E-3,-1.0493579E-2,8.629298E5,5.34E2,6.7098835E-3,3.93672E6,-1.2543922E-2,-2.4803964E-2,-8.54645E-3,-1.6393885E-4,1.6498195E0,3.3438003E5,1.061E3,9E0,2.4129942E-2,1.36239305E-2,-3.7315574E-3,3.1661785E-3,-2.759706E-3,-1.1307532E-2,5.099239E-3,1.0886122E-2,3.2962817E-3,-3.2184385E-3],"split_indices":[9,52,9,0,0,2,28,41,9,2,3,56,2,0,26,0,0,26,37,0,0,27,0,0,1,0,0,0,0,52,31,2,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1E1,1.94E2,5E0,5E0,1.63E2,3.1E1,1.37E2,2.6E1,1E1,2.1E1,1.6E1,1.21E2,6E0,2E1,5E0,5E0,1.1E1,1E1,9E0,7E0,2.1E1,1E2,5E0,1.5E1,5E0,6E0,5E0,5E0,1.1E1,1E1,3.7E1,6.3E1,8E0,7E0,6E0,5E0,5E0,5E0,1.9E1,1.8E1,3.8E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-8.615122E-3,2.2319222E-2,-3.3891685E-2,4.699542E-2,-2.4777137E-2,8.946151E-3,2.6323947E-1,5.4896243E-2,-8.313295E-2,2.745765E-2,6.546693E-2,-1.0281701E-2,7.438199E-2,8.760412E-2,-1.4548934E-1,1.0630774E-2,-3.1727192E-3,1.6878971E-1,1.8071126E-2,7.0169987E-3,-7.3257356E-4,5.3886306E-3,-1.8931165E-1,4.5853024E-3,1.0483018E-2,-8.7115844E-4,6.634993E-3,-5.4690853E-4,-1.1434086E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,-1,17,19,21,-1,-1,23,25,-1,-1,-1,27,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0798435E0,2.433911E0,0E0,1.4293046E0,0E0,6.3498914E-1,1.5559572E0,5.411203E-1,5.450728E-1,0E0,3.6490712E-1,0E0,4.9667078E-1,8.703386E-2,4.3956816E-1,0E0,0E0,1.0700929E-1,2.6447305E-1,0E0,0E0,0E0,2.7823293E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,17,17,18,18,22,22],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,-1,18,20,22,-1,-1,24,26,-1,-1,-1,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,4.7379535E6,-3.3891685E-2,4.796E3,-2.4777137E-2,2.0946133E0,1.9824325E0,9.235376E5,2.0304577E-1,2.745765E-2,5.287008E5,-1.0281701E-2,6.7033327E-1,1.8495E4,3.9267014E-3,1.0630774E-2,-3.1727192E-3,1.046E3,2.1799042E0,7.0169987E-3,-7.3257356E-4,5.3886306E-3,1.4778827E0,4.5853024E-3,1.0483018E-2,-8.7115844E-4,6.634993E-3,-5.4690853E-4,-1.1434086E-2],"split_indices":[44,27,0,2,0,55,53,44,26,0,31,0,26,9,56,0,0,2,53,0,0,0,52,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.8E2,7E0,1.73E2,7E0,1.48E2,2.5E1,9.9E1,4.9E1,9E0,1.6E1,6E0,9.3E1,1.3E1,3.6E1,7E0,9E0,3.4E1,5.9E1,8E0,5E0,5E0,3.1E1,1.5E1,1.9E1,4.6E1,1.3E1,7E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.986548E-4,-3.1845292E-1,1.7985018E-2,-3.4269767E-3,-2.4543364E-2,-1.8185236E-2,1.9708934E-1,-9.742517E-4,-1.5913421E-2,3.985395E-1,2.945647E-2,-2.4388062E-2,1.5804447E-1,2.9602781E-2,9.965601E-3,1.2605089E-1,-6.723553E-3,-1.1332913E-3,-1.4393006E-1,2.3598392E-3,2.0446487E-1,9.273716E-3,2.4034798E-3,-1.676299E-1,1.9265613E-2,-2.5626883E-1,1.6446589E-3,1.2925185E-2,5.2665914E-3,-1.1401988E-2,-2.7594191E-3,8.560774E-3,-4.2228858E-5,-1.586716E-2,-5.4448466E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,-1,-1,7,9,11,-1,13,15,17,19,-1,-1,21,-1,23,25,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1748582E0,4.89864E-1,1.1955819E0,0E0,0E0,8.3187425E-1,1.0351202E0,5.511698E-1,0E0,4.5659232E-1,3.1291878E-1,3.5844678E-1,8.587745E-2,0E0,0E0,5.1773027E-2,0E0,3.7343147E-1,4.293317E-1,0E0,5.062276E-2,0E0,0E0,7.9202086E-2,3.1492835E-1,1.0128021E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,17,17,18,18,20,20,23,23,24,24,25,25],"right_children":[2,4,6,-1,-1,8,10,12,-1,14,16,18,20,-1,-1,22,-1,24,26,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0287E4,1.002E4,3.847E3,-3.4269767E-3,-2.4543364E-2,4.674595E6,8.652791E1,1E0,-1.5913421E-2,1.6659373E0,1.260129E6,3.10364E5,4.5071664E-1,2.9602781E-2,9.965601E-3,8E0,-6.723553E-3,8.960506E3,8E0,2.3598392E-3,8.421419E-1,9.273716E-3,2.4034798E-3,1.2241036E12,1.2414E4,3.8972758E8,1.6446589E-3,1.2925185E-2,5.2665914E-3,-1.1401988E-2,-2.7594191E-3,8.560774E-3,-4.2228858E-5,-1.586716E-2,-5.4448466E-3],"split_indices":[9,9,2,0,0,27,57,112,0,52,28,9,26,0,0,3,0,32,3,0,26,0,0,30,9,7,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1E1,1.83E2,5E0,5E0,1.53E2,3E1,1.46E2,7E0,1.3E1,1.7E1,1.28E2,1.8E1,5E0,8E0,1.1E1,6E0,1.08E2,2E1,6E0,1.2E1,5E0,6E0,1.1E1,9.7E1,1.2E1,8E0,6E0,6E0,6E0,5E0,1E1,8.7E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[8.045889E-4,4.4750556E-2,-2.1095945E-1,6.7334473E-3,3.2452223E-1,-5.1016223E-1,9.030463E-3,-1.0705848E-2,2.4121732E-2,4.9976826E-1,3.558373E-3,-3.199334E-2,-9.039044E-3,9.939486E-3,-1.4055084E-1,6.686401E-2,-6.288247E-2,1.1249147E-2,3.264647E-2,-9.149293E-5,-1.3360373E-2,7.7594034E-3,1.953505E-1,-1.2396897E-1,5.176696E-2,8.271019E-3,-2.3497034E-2,1.2856285E-2,5.8134798E-2,2.0341645E-3,-2.0098475E-1,6.791249E-3,-4.0330556E-3,1.0166594E-3,-7.795795E-3,1.0631097E-2,-5.525003E-3,-1.40098035E-2,-1.3061812E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,21,23,-1,-1,-1,-1,25,27,29,31,-1,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7869202E0,1.6889563E0,2.1767197E0,5.697796E-1,7.96123E-1,5.835202E-1,6.228133E-1,0E0,4.9508065E-1,3.6948204E-1,0E0,0E0,0E0,0E0,2.2959414E-1,6.7436296E-1,3.1457287E-1,0E0,0E0,0E0,0E0,3.2358354E-1,2.6487327E-1,3.8024464E-1,2.0767505E-1,0E0,3.3342624E-1,0E0,3.38073E-1,0E0,3.0974334E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,14,14,15,15,16,16,21,21,22,22,23,23,24,24,26,26,28,28,30,30],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,22,24,-1,-1,-1,-1,26,28,30,32,-1,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.780233E9,3.965E3,2.002992E6,1.2585019E2,1.3770975E2,1.9006333E0,5.0797403E5,-1.0705848E-2,1.49108E5,5.818E3,3.558373E-3,-3.199334E-2,-9.039044E-3,9.939486E-3,1.08725375E5,3.0332904E2,3.2970743E2,1.1249147E-2,3.264647E-2,-9.149293E-5,-1.3360373E-2,1.3257E4,2.8477445E0,1.8836116E5,1.9185424E0,8.271019E-3,7.3501956E-1,1.2856285E-2,1.2414E4,2.0341645E-3,2.1296928E0,6.791249E-3,-4.0330556E-3,1.0166594E-3,-7.795795E-3,1.0631097E-2,-5.525003E-3,-1.40098035E-2,-1.3061812E-3],"split_indices":[5,2,1,46,57,52,31,0,28,2,0,0,0,0,32,4,51,0,0,0,0,9,56,31,53,0,38,0,9,0,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.58E2,3.2E1,1.4E2,1.8E1,1.3E1,1.9E1,9E0,1.31E2,1E1,8E0,8E0,5E0,8E0,1.1E1,8.8E1,4.3E1,5E0,5E0,6E0,5E0,6.1E1,2.7E1,2.8E1,1.5E1,9E0,5.2E1,1.7E1,1E1,9E0,1.9E1,9E0,6E0,4E1,1.2E1,5E0,5E0,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.6608848E-3,4.782441E-2,-1.8848379E-1,6.24066E-3,4.3468302E-1,-3.436911E-1,-2.3852276E-2,-2.6888508E-1,2.7813107E-2,2.913417E-2,1.1125077E-2,-2.3869023E-2,-1.9915147E-1,1.4901195E-1,-1.0138252E-2,-5.924615E-3,-1.7692085E-2,-2.3946729E-2,1.0824097E-1,-1.45787485E-2,-3.859956E-3,2.5187046E-3,1.0569306E-2,2.7660474E-2,-2.51457E-1,-2.241764E-2,1.4921668E-1,5.440983E-2,-6.752757E-3,-2.8779164E-1,-6.8477327E-3,2.4939296E-3,-6.246875E-3,1.3651121E-2,9.18642E-2,-1.4470458E-3,6.452277E-3,-1.7658511E-2,-7.6187886E-3,6.20227E-4,7.478205E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,27,29,31,33,35,-1,37,-1,-1,-1,-1,39,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7614659E0,2.6516476E0,9.883306E-1,8.977488E-1,4.2063904E-1,3.4749722E-1,6.806921E-1,1.14720225E-1,5.854026E-1,0E0,0E0,0E0,1.2511396E-1,6.435989E-2,0E0,0E0,0E0,1.016696E0,2.9780567E-1,0E0,0E0,0E0,0E0,3.2789797E-1,2.1955013E-2,1.1936582E-1,3.0076098E-1,4.2517024E-1,0E0,5.5825472E-2,0E0,0E0,0E0,0E0,1.5511492E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,17,17,18,18,23,23,24,24,25,25,26,26,27,27,29,29,34,34],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,28,30,32,34,36,-1,38,-1,-1,-1,-1,40,-1,-1,-1,-1,-1,-1],"split_conditions":[6.6774924E7,3.973E3,1.0532789E3,2.3187147E-5,1E0,1.4693E4,2.17206E5,6.373709E1,1.7356589E0,2.913417E-2,1.1125077E-2,-2.3869023E-2,3.7448416E2,7.7919556E2,-1.0138252E-2,-5.924615E-3,-1.7692085E-2,2.1247E4,2.0752347E2,-1.45787485E-2,-3.859956E-3,2.5187046E-3,1.0569306E-2,1.875502E0,1.3E1,1.3697663E7,6.063759E8,2.4246153E2,-6.752757E-3,2.3407245E-1,-6.8477327E-3,2.4939296E-3,-6.246875E-3,1.3651121E-2,1.496E3,-1.4470458E-3,6.452277E-3,-1.7658511E-2,-7.6187886E-3,6.20227E-4,7.478205E-3],"split_indices":[44,2,4,38,80,9,11,46,52,0,0,0,51,51,0,0,0,9,4,0,0,0,0,53,8,44,5,4,0,41,0,0,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.64E2,3.8E1,1.49E2,1.5E1,1.9E1,1.9E1,1E1,1.39E2,7E0,8E0,8E0,1.1E1,1E1,9E0,5E0,5E0,8.5E1,5.4E1,5E0,6E0,5E0,5E0,7E1,1.5E1,1.3E1,4.1E1,6.1E1,9E0,1E1,5E0,8E0,5E0,1.1E1,3E1,3E1,3.1E1,5E0,5E0,1.4E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.8946908E-2,6.413307E-2,-1.7502837E-1,2.1907693E-2,3.9189348E-1,-4.74425E-1,3.186223E-2,-9.470217E-3,3.4124196E-2,3.2232758E-2,2.1049017E-1,-1.1822359E-2,-3.2157667E-2,1.957844E-1,-1.219015E-1,1.2668778E-1,-3.4172866E-3,1.272246E-2,4.8858933E-3,4.9799457E-3,1.2949473E-2,-1.3344702E-2,5.139391E-3,8.297384E-3,1.6461056E-1,2.8118532E-2,-1.6275041E-1,-3.7327516E-3,4.461527E-3,1.2927429E-2,1.2267642E-1,-1.2158008E-1,7.532387E-2,-1.4797753E-2,-1.4168705E-3,3.17265E-3,7.5824433E-3,-3.5839662E-4,-8.53225E-3,-1.3248391E-3,5.4862974E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,-1,-1,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,31,33,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8080497E0,2.3026621E0,2.4308677E0,4.064404E-1,8.2657456E-1,5.512285E-1,6.2879324E-1,0E0,4.9387926E-1,0E0,5.0595224E-2,0E0,0E0,5.401942E-2,4.9657404E-1,1.8271166E-1,5.170958E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.7701716E-2,1.0368699E-1,6.168832E-1,3.216926E-1,0E0,0E0,0E0,3.774342E-2,1.3408294E-1,2.6954037E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,13,13,14,14,15,15,16,16,23,23,24,24,25,25,26,26,30,30,31,31,32,32],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,-1,-1,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,32,34,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.6774924E7,3.973E3,1.0532789E3,4.26E2,1E0,8.639937E9,1.74E3,-9.470217E-3,4.06E2,3.2232758E-2,9.982716E0,-1.1822359E-2,-3.2157667E-2,2.1936802E3,4.0950096E10,7.2E2,4.8028188E5,1.272246E-2,4.8858933E-3,4.9799457E-3,1.2949473E-2,-1.3344702E-2,5.139391E-3,2.5823686E-2,2.0080225E5,2.2243262E5,4.876063E10,-3.7327516E-3,4.461527E-3,1.2927429E-2,8.364481E3,2.03E0,2.228762E2,-1.4797753E-2,-1.4168705E-3,3.17265E-3,7.5824433E-3,-3.5839662E-4,-8.53225E-3,-1.3248391E-3,5.4862974E-3],"split_indices":[44,2,4,2,80,5,0,0,0,0,56,0,0,4,5,2,32,0,0,0,0,0,0,56,27,32,30,0,0,0,46,53,4,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.66E2,3.8E1,1.48E2,1.8E1,1.5E1,2.3E1,7E0,1.41E2,6E0,1.2E1,8E0,7E0,1.1E1,1.2E1,4E1,1.01E2,7E0,5E0,6E0,5E0,7E0,5E0,1E1,3E1,8.5E1,1.6E1,5E0,5E0,7E0,2.3E1,2E1,6.5E1,7E0,9E0,1E1,1.3E1,7E0,1.3E1,1.8E1,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.347735E-3,2.99794E-2,-1.7060877E-1,2.6518682E-3,2.9625708E-1,-2.4153547E-1,6.0152872E-3,-1.410911E-2,1.640966E-2,1.9903222E-2,8.776839E-3,-1.537585E-1,-2.4937226E-2,-7.769096E-3,1.3858135E-1,-2.6932558E-1,-6.117016E-3,1.7459609E-1,-2.8316734E-2,2.2076572E-1,-1.5356402E-3,-5.4409946E-3,-1.6443912E-2,3.8554084E-3,-3.8070178E-3,2.0043582E-3,1.2089219E-2,-4.744251E-2,8.491665E-3,1.2947663E-2,7.030206E-3,3.9293463E-3,-3.2594209E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,15,-1,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[1.2522455E0,1.2508088E0,8.2856035E-1,6.466775E-1,1.3948762E-1,6.791053E-1,0E0,0E0,4.4712457E-1,0E0,0E0,4.2491508E-1,0E0,4.8083508E-1,3.576647E-1,1.2218678E-1,8.261041E-2,1.2361914E-1,4.6054178E-1,2.26233E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8816044E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,27,27],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,16,-1,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,5.251E3,3.0922055E12,2.3187147E-5,2.13281E5,1E0,6.0152872E-3,-1.410911E-2,2.0827537E0,1.9903222E-2,8.776839E-3,1.8663485E0,-2.4937226E-2,1.2414E4,3.329399E-1,1.083E3,3E0,1.117E3,5.8018835E2,6.063759E8,-1.5356402E-3,-5.4409946E-3,-1.6443912E-2,3.8554084E-3,-3.8070178E-3,2.0043582E-3,1.2089219E-2,2.0304577E-1,8.491665E-3,1.2947663E-2,7.030206E-3,3.9293463E-3,-3.2594209E-3],"split_indices":[44,2,30,38,28,108,0,0,52,0,0,52,0,9,40,0,8,2,51,5,0,0,0,0,0,0,0,26,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.71E2,3.7E1,1.56E2,1.5E1,3E1,7E0,6E0,1.5E2,6E0,9E0,2.4E1,6E0,1.26E2,2.4E1,1.3E1,1.1E1,1.2E1,1.14E2,1.6E1,8E0,5E0,8E0,5E0,6E0,5E0,7E0,1.05E2,9E0,8E0,8E0,1.4E1,9.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.3159277E-2,-1.2684307E-3,2.4888517E-1,-1.867153E-2,1.1005594E-2,2.7766839E-2,7.7967346E-2,-2.022436E-1,3.0439774E-2,-6.6097155E-3,9.568757E-3,-1.2905475E-2,-3.710245E-3,5.972386E-2,-6.9887124E-2,-7.167451E-2,9.143325E-2,-1.8482423E-1,4.290475E-2,2.6465082E-3,-1.1885702E-1,1.0909389E-1,-3.7460733E-2,-1.401572E-2,-7.0086926E-2,1.18056856E-1,-5.953316E-3,-1.6611579E-3,-8.843023E-3,6.7725587E-3,2.0228736E-3,4.9213176E-3,-1.1569566E-2,-5.2233515E-3,-9.3251956E-4,7.8016566E-3,2.089101E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,-1,7,-1,9,11,13,-1,-1,-1,-1,15,17,19,21,23,25,-1,27,29,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1858184E0,9.340575E-1,1.1417756E0,0E0,7.935776E-1,0E0,4.2194676E-1,1.1319184E-1,5.183208E-1,0E0,0E0,0E0,0E0,5.732057E-1,5.264887E-1,1.6824706E-1,2.544853E-1,2.2670567E-1,2.7849624E-1,0E0,1.042591E-1,2.0335376E-1,4.2523852E-1,0E0,1.9561965E-2,4.2917043E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,24,24,25,25],"right_children":[2,4,6,-1,8,-1,10,12,14,-1,-1,-1,-1,16,18,20,22,24,26,-1,28,30,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,6E0,3.0451374E9,-1.867153E-2,4.7486033E-2,2.7766839E-2,9.248443E8,3.0734711E-6,3.00037E5,-6.6097155E-3,9.568757E-3,-1.2905475E-2,-3.710245E-3,7.23E2,2.1296928E0,4.398424E9,4.8028188E5,4.048035E0,2.2909E4,2.6465082E-3,3.3438003E5,1.7476E4,3.413613E7,-1.401572E-2,4.57825E5,1.1E1,-5.953316E-3,-1.6611579E-3,-8.843023E-3,6.7725587E-3,2.0228736E-3,4.9213176E-3,-1.1569566E-2,-5.2233515E-3,-9.3251956E-4,7.8016566E-3,2.089101E-3],"split_indices":[2,3,12,0,57,0,7,37,28,0,0,0,0,2,53,30,32,56,9,0,31,9,12,0,28,3,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.94E2,2E1,5E0,1.89E2,6E0,1.4E1,1.5E1,1.74E2,5E0,9E0,9E0,6E0,1.35E2,3.9E1,2.6E1,1.09E2,1.9E1,2E1,7E0,1.9E1,9.6E1,1.3E1,9E0,1E1,1.4E1,6E0,9E0,1E1,6.4E1,3.2E1,8E0,5E0,5E0,5E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.9316217E-2,1.7384242E-3,-4.0611404E-1,1.2895812E-2,-1.4971867E-2,-2.9367857E-2,-6.3019297E-3,-1.9112745E-2,9.588359E-2,2.422868E-2,-1.11033335E-1,2.0667966E-1,-1.833689E-2,-8.566936E-2,5.3505994E-2,-1.5332667E-2,-7.7738605E-2,3.0302718E-1,1.0297666E-1,-7.6985806E-2,8.077721E-3,-3.622631E-2,-9.7368555E-3,1.8223117E-1,8.963458E-3,-1.0182656E-1,3.8204778E-3,1.7622638E-2,7.4402755E-3,2.7882059E-3,7.6865903E-3,2.0768017E-2,-1.7662764E-1,-4.8481966E-3,2.9590416E-3,5.16664E-3,1.46106975E-2,-5.7287784E-3,2.854076E-3,-6.3493457E-3,-4.1879606E-4,-2.248854E-3,4.6135853E-3,-3.981848E-3,-1.1531701E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,-1,35,37,39,-1,-1,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7353544E0,7.1578854E-1,5.4422545E-1,5.251274E-1,0E0,0E0,0E0,5.723978E-1,6.998247E-1,3.1990075E-1,2.9522884E-1,2.4855816E-1,3.199754E-1,1.1471072E-1,4.4559878E-1,0E0,1.6365258E-1,7.691753E-2,2.9261157E-2,2.1795633E-1,0E0,1.0811561E-1,0E0,1.4971662E-1,3.9094865E-1,1.0173422E-1,0E0,0E0,0E0,0E0,0E0,6.589077E-2,4.590586E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,31,31,32,32],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,-1,36,38,40,-1,-1,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,4.674595E6,4.44E3,7.5614764E2,-1.4971867E-2,-2.9367857E-2,-6.3019297E-3,1.3697663E7,6.5023036E7,4.1E1,1.3047E4,3.6025445E6,2.7545524E10,2.6855614E0,8.7678794E5,-1.5332667E-2,2.5522516E0,2.32018E5,2.185321E0,9.5E2,8.077721E-3,7.12E2,-9.7368555E-3,1.036E3,8E0,2.3361E4,3.8204778E-3,1.7622638E-2,7.4402755E-3,2.7882059E-3,7.6865903E-3,5.5053414E9,6.557827E-1,-4.8481966E-3,2.9590416E-3,5.16664E-3,1.46106975E-2,-5.7287784E-3,2.854076E-3,-6.3493457E-3,-4.1879606E-4,-2.248854E-3,4.6135853E-3,-3.981848E-3,-1.1531701E-2],"split_indices":[44,27,2,4,0,0,0,44,44,51,9,46,5,57,49,0,53,28,40,0,0,2,0,2,3,10,0,0,0,0,0,5,26,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.02E2,1E1,1.96E2,6E0,5E0,5E0,1.42E2,5.4E1,9.7E1,4.5E1,2.7E1,2.7E1,2E1,7.7E1,5E0,4E1,1.3E1,1.4E1,2.1E1,6E0,1.5E1,5E0,1.9E1,5.8E1,3.5E1,5E0,8E0,5E0,9E0,5E0,1.1E1,1E1,9E0,6E0,1.3E1,6E0,1.6E1,4.2E1,2.6E1,9E0,6E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.960221E-4,4.9734674E-2,-1.3045083E-1,2.8344585E-2,2.0658705E-2,-2.4736756E-1,1.8464018E-2,-1.8395035E-1,4.7144543E-2,-1.3834806E-1,-3.6169982E-1,1.2267338E-1,-1.1083173E-2,-5.1432513E-3,-1.162505E-2,1.0340959E-1,-7.357448E-3,-1.8207927E-1,-2.9187736E-3,-2.1794751E-2,-9.11054E-3,3.4559835E-4,1.894237E-1,4.018682E-2,2.0548746E-1,-5.9672784E-2,8.145917E-2,-1.1337451E-2,-4.6549262E-3,3.3939248E-3,1.4190368E-2,7.799603E-2,-4.1002747E-2,5.476589E-3,1.195926E-2,4.5717508E-2,-1.6714114E-1,3.7712115E-3,1.5885705E-1,5.3971703E-3,-5.530482E-4,4.198044E-3,-5.8543882E-3,6.207525E-3,-2.7565528E-3,-5.319216E-3,-1.1299442E-2,-4.5124264E-3,3.2893305E-3,1.2609243E-2,1.5246854E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,-1,-1,-1,-1,29,31,33,35,37,-1,-1,-1,-1,39,41,-1,-1,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3458968E0,1.2280759E0,9.9697375E-1,5.8738697E-1,0E0,3.4882176E-1,7.052126E-1,2.7904272E-2,4.1167948E-1,4.9852133E-2,1.5336025E-1,1.450659E-1,0E0,0E0,0E0,4.1768265E-1,3.254674E-1,2.79786E-2,0E0,0E0,0E0,0E0,1.3095483E-1,1.3212602E-1,7.207525E-2,5.059906E-1,1.5573584E-1,0E0,0E0,0E0,0E0,9.1301054E-2,1.5692843E-1,0E0,0E0,2.0659679E-1,5.8638692E-2,9.545635E-2,1.6392946E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,22,22,23,23,24,24,25,25,26,26,31,31,32,32,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,-1,-1,-1,-1,30,32,34,36,38,-1,-1,-1,-1,40,42,-1,-1,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3625324E7,5.666E3,1.0908286E3,4.75E2,2.0658705E-2,1E0,2.192772E10,4.398424E9,7.2591E4,4.177399E2,1E0,1E0,-1.1083173E-2,-5.1432513E-3,-1.162505E-2,2.4981093E2,4.506623E2,5.50569E5,-2.9187736E-3,-2.1794751E-2,-9.11054E-3,3.4559835E-4,1.2563624E3,8.869983E5,2.54E2,1.9439217E0,5.929222E-1,-1.1337451E-2,-4.6549262E-3,3.3939248E-3,1.4190368E-2,5.37E2,5.320276E5,5.476589E-3,1.195926E-2,6.717009E-1,1.3474992E6,3.0963843E8,4.1936096E8,5.3971703E-3,-5.530482E-4,4.198044E-3,-5.8543882E-3,6.207525E-3,-2.7565528E-3,-5.319216E-3,-1.1299442E-2,-4.5124264E-3,3.2893305E-3,1.2609243E-2,1.5246854E-3],"split_indices":[44,2,4,2,0,107,5,30,28,51,111,111,0,0,0,54,4,12,0,0,0,0,51,31,0,55,26,0,0,0,0,0,47,0,0,26,49,7,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.51E2,5.6E1,1.44E2,7E0,3.1E1,2.5E1,1.1E1,1.33E2,1.7E1,1.4E1,1.8E1,7E0,6E0,5E0,6.5E1,6.8E1,1E1,7E0,8E0,6E0,7E0,1.1E1,4.1E1,2.4E1,4.3E1,2.5E1,5E0,5E0,6E0,5E0,2.8E1,1.3E1,9E0,1.5E1,2.2E1,2.1E1,1.3E1,1.2E1,2E1,8E0,5E0,8E0,1.2E1,1E1,1.3E1,8E0,5E0,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.2828152E-3,-1.810118E-2,1.976273E-2,3.100423E-2,-1.662449E-2,8.899857E-4,1.4692181E-1,6.670543E-2,-4.2529665E-2,2.6706374E-1,-7.111895E-2,-5.2328603E-3,1.6651355E-1,-6.349596E-2,7.787039E-3,4.0701374E-1,1.3729946E-1,-1.12410365E-2,3.0643775E-3,-1.01899095E-1,3.9583586E-2,2.49493E-1,4.8913088E-2,-4.4254355E-2,-1.4979733E-2,9.9427225E-3,2.4905127E-2,8.9216195E-3,1.5751071E-3,-6.9077914E-3,-2.4478629E-3,4.531733E-3,-2.7733485E-3,1.3636827E-2,7.1539427E-3,7.207862E-3,-2.1610053E-3,-5.1050406E-4,-6.15707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,-1,3,5,-1,7,9,11,13,15,17,19,21,23,-1,25,27,-1,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0689161E0,0E0,8.3124214E-1,6.770787E-1,0E0,4.4577786E-1,1.0668268E0,4.4603288E-1,4.1447905E-1,4.1629052E-1,3.463327E-1,1.6432631E-1,2.4151552E-1,3.984984E-1,0E0,1.6221058E-1,7.050732E-2,0E0,0E0,1.841411E-2,1.4488176E-1,1.0310292E-2,1.21345885E-1,2.2795153E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,23,23],"right_children":[2,-1,4,6,-1,8,10,12,14,16,18,20,22,24,-1,26,28,-1,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01E4,-1.810118E-2,4.7379535E6,5.8018835E2,-1.662449E-2,6.769573E6,1.3E1,2.1231644E2,6.204429E6,1.9160621E0,3.6E1,2.6727284E5,5.3246627E0,5.08616E5,7.787039E-3,1.3841E4,1.244201E6,-1.12410365E-2,3.0643775E-3,5.5812734E3,5.34E2,1.0955342E-1,7.0918E4,9.0763354E-1,-1.4979733E-2,9.9427225E-3,2.4905127E-2,8.9216195E-3,1.5751071E-3,-6.9077914E-3,-2.4478629E-3,4.531733E-3,-2.7733485E-3,1.3636827E-2,7.1539427E-3,7.207862E-3,-2.1610053E-3,-5.1050406E-4,-6.15707E-3],"split_indices":[9,0,27,51,0,44,3,4,31,53,3,32,56,32,0,9,28,0,0,47,0,37,28,26,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,6E0,1.98E2,1.93E2,5E0,1.54E2,3.9E1,6.1E1,9.3E1,2.5E1,1.4E1,3.6E1,2.5E1,8.5E1,8E0,1.1E1,1.4E1,6E0,8E0,1.1E1,2.5E1,1.4E1,1.1E1,8E1,5E0,5E0,6E0,9E0,5E0,5E0,6E0,1.6E1,9E0,9E0,5E0,5E0,6E0,5.8E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.483439E-3,-1.839502E-2,1.8489068E-2,2.684116E-3,2.5665677E-1,5.8165703E-2,-4.762403E-2,3.0995454E-3,2.097586E-2,1.413173E-1,-1.5608356E-2,-1.3283266E-2,-2.6447117E-2,1.4328361E-2,1.9337034E-1,-1.5175442E-1,4.091086E-2,-8.34186E-2,5.9744753E-2,6.473251E-3,-5.1984196E-3,1.3063373E-1,1.3613368E-2,-1.1930123E-2,9.311462E-4,8.138736E-2,-3.8598268E-3,-3.4057397E-2,-2.533447E-1,-2.3098495E-2,1.772652E-1,3.534431E-3,8.356614E-3,1.0678837E-3,7.1132667E-3,-3.6367425E-3,3.2752915E-3,-6.0690586E-3,-1.5604789E-2,5.179016E-3,-3.787409E-3,2.265727E-3,1.1379285E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,-1,17,19,21,23,25,27,29,-1,-1,31,-1,-1,-1,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2441623E0,0E0,7.258073E-1,5.1076406E-1,3.8901556E-1,5.3590107E-1,4.5952183E-1,0E0,0E0,2.68246E-1,3.6786726E-1,0E0,4.4201645E-1,2.0754541E-1,1.2804914E-1,2.3795244E-1,1.73735E-1,4.443106E-1,3.545825E-1,0E0,0E0,3.444457E-2,0E0,0E0,0E0,9.97749E-2,0E0,1.8951869E-1,6.8748295E-2,1.7006123E-1,1.0624263E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,25,25,27,27,28,28,29,29,30,30],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,-1,18,20,22,24,26,28,30,-1,-1,32,-1,-1,-1,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,-1.839502E-2,1.4056875E3,6.781768E-1,1.3841E4,1.8949672E0,1.1507483E2,3.0995454E-3,2.097586E-2,3.2258064E-2,1.0419601E2,-1.3283266E-2,1.546644E6,3.4246575E-3,1E0,1.514E3,2.5086E4,3.3487454E11,3.200388E2,6.473251E-3,-5.1984196E-3,2.0495868E0,1.3613368E-2,-1.1930123E-2,9.311462E-4,8.9625574E8,-3.8598268E-3,1E0,2.0037713E0,1E0,1.3161152E-4,3.534431E-3,8.356614E-3,1.0678837E-3,7.1132667E-3,-3.6367425E-3,3.2752915E-3,-6.0690586E-3,-1.5604789E-2,5.179016E-3,-3.787409E-3,2.265727E-3,1.1379285E-2],"split_indices":[3,0,51,26,9,57,46,0,0,56,51,0,27,56,80,2,9,30,4,0,0,53,0,0,0,12,0,15,55,111,37,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,7E0,1.92E2,1.81E2,1.1E1,8.6E1,9.5E1,6E0,5E0,4E1,4.6E1,7E0,8.8E1,1.2E1,2.8E1,1.3E1,3.3E1,5.3E1,3.5E1,6E0,6E0,1.8E1,1E1,8E0,5E0,2.5E1,8E0,4.2E1,1.1E1,2.1E1,1.4E1,9E0,9E0,1.4E1,1.1E1,3E1,1.2E1,5E0,6E0,6E0,1.5E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.7281901E-2,2.5003694E-4,3.3157447E-1,-1.6468253E-2,1.2410216E-2,2.0462748E-2,8.660084E-3,-1.1171214E-2,2.397807E-2,1.2280472E-2,1.2356259E-2,-7.322393E-2,2.9728822E-2,-1.20522976E-1,1.8280072E-3,6.524474E-2,-4.8581608E-2,-8.009211E-3,-8.6138054E-4,-4.609298E-3,4.1734027E-3,1.3997809E-3,-6.113033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,-1,7,-1,-1,-1,9,-1,11,13,15,17,-1,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1454566E0,8.5760653E-1,8.129895E-2,0E0,5.6809705E-1,0E0,0E0,0E0,5.099051E-1,0E0,2.727649E-1,1.679123E-1,4.2592233E-1,1.0224521E-1,0E0,3.7909704E-1,2.9880708E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,10,10,11,11,12,12,13,13,15,15,16,16],"right_children":[2,4,6,-1,8,-1,-1,-1,10,-1,12,14,16,18,-1,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,1.01E4,8.142301E-1,-1.6468253E-2,6E0,2.0462748E-2,8.660084E-3,-1.1171214E-2,2.0318625E-2,1.2280472E-2,7.38E2,1.8837452E0,1.8733E4,1E0,1.8280072E-3,1.2245E4,1.9439217E0,-8.009211E-3,-8.6138054E-4,-4.609298E-3,4.1734027E-3,1.3997809E-3,-6.113033E-3],"split_indices":[2,9,26,0,3,0,0,0,26,0,2,52,9,108,0,9,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,2.03E2,1E1,6E0,1.97E2,5E0,5E0,8E0,1.89E2,8E0,1.81E2,3E1,1.51E2,2.1E1,9E0,1.04E2,4.7E1,1.4E1,7E0,1.2E1,9.2E1,2.4E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.1349075E-2,6.594623E-2,-1.0745421E-1,4.3561485E-2,1.8716639E-2,-1.850474E-1,2.7927723E-2,-4.09549E-2,8.3716765E-2,-1.4021446E-1,-1.8843077E-2,-1.0789985E-2,1.1563168E-1,-1.1761252E-1,8.9538135E-2,6.697921E-2,1.092667E-2,-2.1173939E-1,-4.5036923E-2,1.7170806E-1,5.2664854E-4,5.9023726E-4,-1.9146965E-1,1.0155692E-3,7.886318E-3,8.171284E-2,-2.3240119E-3,-2.9050133E-3,-2.8223756E-1,-5.945428E-3,1.028148E-3,1.01647675E-2,4.916686E-3,-3.4245248E-3,-2.395118E-1,1.2981564E-1,3.539849E-2,-7.7928035E-3,-1.6500223E-2,-1.3993454E-2,-7.0433337E-3,2.8547198E-3,9.405362E-3,-4.2669126E-3,3.0766996E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,25,-1,27,29,31,-1,-1,33,-1,-1,35,-1,-1,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1940831E0,8.9887476E-1,6.144416E-1,4.0728825E-1,0E0,2.9033005E-1,5.144908E-1,4.0065897E-1,1.804356E-1,2.0830607E-1,0E0,0E0,9.7864494E-2,2.422702E-1,7.3469326E-2,1.2987995E-1,0E0,1.7462999E-1,8.152598E-2,8.973122E-3,0E0,0E0,7.5111866E-2,0E0,0E0,1.4244276E-1,0E0,0E0,3.3193648E-2,0E0,0E0,0E0,0E0,0E0,1.0551572E-2,1.3775784E-1,1.3127412E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,22,22,25,25,28,28,34,34,35,35,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,26,-1,28,30,32,-1,-1,34,-1,-1,36,-1,-1,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.083812E7,5.818E3,9.1922754E2,8E0,1.8716639E-2,3.4231E4,4.11591E5,1.8270676E0,5.5403314E2,1.5626E4,-1.8843077E-2,-1.0789985E-2,8.294758E-1,1.5182E4,2.0752347E2,3.237835E-1,1.092667E-2,1.8271865E0,2E0,2.035294E0,5.2664854E-4,5.9023726E-4,4.7655502E-1,1.0155692E-3,7.886318E-3,9.402985E-1,-2.3240119E-3,-2.9050133E-3,1.2699157E0,-5.945428E-3,1.028148E-3,1.01647675E-2,4.916686E-3,-3.4245248E-3,1.0177765E7,2.1548604E2,3.322259E-2,-7.7928035E-3,-1.6500223E-2,-1.3993454E-2,-7.0433337E-3,2.8547198E-3,9.405362E-3,-4.2669126E-3,3.0766996E-3],"split_indices":[44,2,51,3,0,10,27,52,51,9,0,0,26,9,4,37,0,53,8,53,0,0,55,0,0,55,0,0,55,0,0,0,0,0,44,4,56,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.82E2,1.25E2,5.7E1,1.18E2,7E0,3.6E1,2.1E1,3.8E1,8E1,3.1E1,5E0,5E0,1.6E1,2.4E1,1.4E1,7.3E1,7E0,1.7E1,1.4E1,1E1,6E0,9E0,1.5E1,8E0,6E0,6.5E1,8E0,6E0,1.1E1,6E0,8E0,5E0,5E0,5E0,1E1,3.1E1,3.4E1,5E0,6E0,5E0,5E0,1.6E1,1.5E1,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.0029137E-3,3.7795924E-2,-1.6071907E-1,-6.395732E-3,1.6147704E-1,-1.9704046E-2,-6.877452E-2,-1.1048368E-2,4.963173E-3,2.1178237E-1,-4.3773707E-3,7.7737286E-4,-1.3772557E-2,-2.2910938E-2,9.225907E-2,2.8975055E-1,1.0301094E-1,1.017838E-1,-1.1866453E-1,1.1112798E-2,-1.7166796E-1,1.0512692E-2,1.3776052E-2,6.77288E-3,3.5384205E-1,2.0704875E-2,1.0426444E-2,2.3069135E-4,1.0257712E-2,-1.9801257E-3,-8.442411E-3,-7.7763036E-2,5.4461487E-2,-1.6342095E-3,-1.2673987E-2,7.619628E-2,-3.908185E-3,2.1486767E-2,1.086505E-2,3.4320725E-3,-1.6135231E-3,-1.6360213E-4,-6.5124435E-3,5.1919664E-3,-2.8463904E-4,-1.1635876E-3,7.255253E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,17,-1,19,21,23,25,27,29,31,33,-1,35,-1,37,39,-1,-1,-1,-1,-1,41,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2199233E0,9.138001E-1,8.471694E-1,3.1695798E-1,5.790101E-1,0E0,4.4134796E-1,0E0,2.9167593E-1,2.83939E-1,0E0,2.8955933E-1,0E0,4.624833E-1,2.864269E-1,1.4968312E-1,1.5587376E-1,1.4056791E-1,4.1676924E-2,2.9318386E-1,2.0816946E-1,0E0,1.193022E-1,0E0,6.6654444E-2,3.2882486E-2,0E0,0E0,0E0,0E0,0E0,1.0766117E-1,1.6672039E-1,0E0,0E0,9.440806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,25,25,31,31,32,32,35,35],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,18,-1,20,22,24,26,28,30,32,34,-1,36,-1,38,40,-1,-1,-1,-1,-1,42,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.970116E7,2.081E3,7.567149E4,1.5768223E4,2.2909E4,-1.9704046E-2,5.578E4,-1.1048368E-2,1.9323944E0,1.285E3,-4.3773707E-3,2.0600267E5,-1.3772557E-2,1.9577E4,4.50847E5,1E0,1.2978232E6,1.3182058E3,4.1314016E8,7.99E2,2.0787193E2,1.0512692E-2,6.063759E8,6.77288E-3,1E0,5.428733E5,1.0426444E-2,2.3069135E-4,1.0257712E-2,-1.9801257E-3,-8.442411E-3,1.3579175E0,7.07E2,-1.6342095E-3,-1.2673987E-2,4.6002E4,-3.908185E-3,2.1486767E-2,1.086505E-2,3.4320725E-3,-1.6135231E-3,-1.6360213E-4,-6.5124435E-3,5.1919664E-3,-2.8463904E-4,-1.1635876E-3,7.255253E-3],"split_indices":[44,2,31,32,9,0,10,0,52,0,0,32,0,9,31,8,27,4,7,2,4,0,5,0,80,27,0,0,0,0,0,56,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.66E2,3.7E1,1.23E2,4.3E1,9E0,2.8E1,5E0,1.18E2,3.6E1,7E0,2.2E1,6E0,9E1,2.8E1,2E1,1.6E1,1.2E1,1E1,7.4E1,1.6E1,1E1,1.8E1,7E0,1.3E1,1E1,6E0,7E0,5E0,5E0,5E0,2.4E1,5E1,7E0,9E0,1.1E1,7E0,6E0,7E0,5E0,5E0,1.1E1,1.3E1,2.6E1,2.4E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-4.663613E-3,2.3061233E-2,-2.3680216E-1,7.205894E-3,1.9394493E-2,-7.225073E-2,-2.3722623E-2,-1.4616153E-1,2.4089273E-2,5.8870264E-3,-9.309104E-3,-1.3324203E-2,1.7752482E-3,-1.1407759E-2,8.3039664E-2,5.0689846E-2,-8.735919E-2,1.2802789E-1,9.012896E-3,9.0964185E-3,9.366502E-3,-1.6602299E-1,-8.317334E-3,1.531839E-1,-1.1154417E-3,-4.8415534E-2,4.240434E-3,-2.7362527E-3,2.6150653E-3,-2.979243E-3,-1.185115E-2,9.552315E-3,-3.6932605E-3,1.3648756E-3,8.717929E-3,-6.1697145E-3,1.3393618E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,-1,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2669181E0,1.0624099E0,8.5702384E-1,4.4371215E-1,0E0,3.573261E-1,0E0,4.2131042E-1,3.2320216E-1,0E0,0E0,0E0,0E0,4.6191922E-1,1.920054E-1,3.2349145E-1,2.7219936E-1,1.4059609E-1,1.0929125E-1,1.3262574E-1,0E0,1.7044955E-1,3.4391278E-1,1.0609341E-1,0E0,8.952874E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,-1,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,1.3904022E3,2.6563678E5,3.9134244E-5,1.9394493E-2,1.3649979E10,-2.3722623E-2,5.980258E2,1.9121015E0,5.8870264E-3,-9.309104E-3,-1.3324203E-2,1.7752482E-3,8.3860955E6,8.5913794E5,1.436E3,2.6259702E2,2.9119748E-1,6.16E2,2.1696895E5,9.366502E-3,1.2166988E-2,2.493493E7,8.041484E-5,-1.1154417E-3,7.6696295E-1,4.240434E-3,-2.7362527E-3,2.6150653E-3,-2.979243E-3,-1.185115E-2,9.552315E-3,-3.6932605E-3,1.3648756E-3,8.717929E-3,-6.1697145E-3,1.3393618E-3],"split_indices":[44,51,32,41,0,5,0,4,52,0,0,0,0,44,31,2,51,38,10,31,0,37,44,37,0,26,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.75E2,2E1,1.69E2,6E0,1.3E1,7E0,1.6E1,1.53E2,5E0,8E0,9E0,7E0,9.6E1,5.7E1,5.3E1,4.3E1,3.5E1,2.2E1,4.2E1,1.1E1,2.1E1,2.2E1,3E1,5E0,1.3E1,9E0,1.7E1,2.5E1,1E1,1.1E1,5E0,1.7E1,6E0,2.4E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.500134E-2,1.9240899E-3,-1.8297764E-2,-9.238321E-3,1.4594265E-2,-1.1058382E-2,1.76996E-3,-2.5740376E-2,6.921475E-2,-4.8717983E-2,7.7773966E-2,1.1893928E-1,-1.8777702E-2,1.9638969E-2,-8.5130066E-2,1.4256452E-1,-1.3025928E-3,-2.2204304E-3,1.5601315E-1,4.089425E-3,-5.9180677E-2,-1.4725901E-3,4.739682E-3,3.1695236E-3,-5.78709E-3,8.529443E-3,2.8593335E-3,1.0406741E-2,2.6972203E-3,-5.493269E-3,9.872431E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,-1,5,-1,-1,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2676712E0,6.650818E-1,0E0,4.6387067E-1,0E0,0E0,3.3946964E-1,3.129955E-1,2.3412871E-1,2.6799694E-1,1.6709125E-1,2.1557665E-1,8.9200564E-2,1.5483312E-1,3.856898E-1,3.347382E-2,0E0,0E0,1.5866417E-1,0E0,6.876373E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,20,20],"right_children":[2,4,-1,6,-1,-1,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4089414E8,9.772E3,-1.8297764E-2,4.305949E-2,1.4594265E-2,-1.1058382E-2,1.9483395E0,1.0710328E0,8.5913794E5,1.7039711E0,3.6188666E5,9.814481E3,6.76E2,6.930967E8,2.42E2,4.870977E8,-1.3025928E-3,-2.2204304E-3,3.2693877E0,4.089425E-3,2.9656984E7,-1.4725901E-3,4.739682E-3,3.1695236E-3,-5.78709E-3,8.529443E-3,2.8593335E-3,1.0406741E-2,2.6972203E-3,-5.493269E-3,9.872431E-4],"split_indices":[44,2,0,57,0,0,52,38,31,53,32,50,2,5,10,7,0,0,56,0,12,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.95E2,8E0,1.89E2,6E0,8E0,1.81E2,1.29E2,5.2E1,1.06E2,2.3E1,3.3E1,1.9E1,3.7E1,6.9E1,1.4E1,9E0,6E0,2.7E1,5E0,1.4E1,2.3E1,1.4E1,1.3E1,5.6E1,9E0,5E0,1.6E1,1.1E1,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.4863362E-2,3.979328E-3,-2.1793346E-1,1.5606407E-2,-1.4001171E-2,2.744275E-4,-1.9995995E-2,-1.3591747E-2,1.3966893E-1,1.0195989E-2,-1.1335464E-1,1.5294917E-2,8.1285685E-2,-1.2808241E-2,1.05770335E-1,-1.9811264E-1,-3.7404105E-2,8.165068E-3,1.8690068E-1,6.1503053E-3,-2.5355093E-2,1.4248666E-1,-1.0543197E-3,-1.2489826E-2,-5.136775E-3,1.957013E-3,-5.7855765E-3,-3.7619716E-3,6.2588796E-2,5.538094E-3,1.0877756E-2,-2.7490235E-3,2.41031E-3,3.6977201E-3,1.2467169E-2,8.755685E-4,4.621727E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.6614463E-1,6.3830703E-1,7.6073694E-1,6.46685E-1,0E0,0E0,0E0,3.4526005E-1,3.3053058E-1,2.607912E-1,1.7283744E-1,0E0,2.0698503E-1,1.7331173E-1,1.1104053E-1,4.6002984E-2,1.0930329E-1,8.5120104E-2,5.246341E-3,0E0,2.1855736E-1,1.1831695E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.4752131E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,28,28],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,4.7379535E6,2.0204625E5,3.625E3,-1.4001171E-2,2.744275E-4,-1.9995995E-2,3.70592E5,2.13281E5,2.573537E2,2.868989E2,1.5294917E-2,2.41E3,1.2414E4,1.6235546E1,5.354141E2,8.091679E-1,5.682408E5,2.6823762E5,6.1503053E-3,2.195572E0,2.2614942E0,-1.0543197E-3,-1.2489826E-2,-5.136775E-3,1.957013E-3,-5.7855765E-3,-3.7619716E-3,6.0004652E7,5.538094E-3,1.0877756E-2,-2.7490235E-3,2.41031E-3,3.6977201E-3,1.2467169E-2,8.755685E-4,4.621727E-3],"split_indices":[44,27,32,2,0,0,0,28,28,51,51,0,0,9,56,4,26,27,32,0,53,56,0,0,0,0,0,0,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.83E2,1.6E1,1.77E2,6E0,8E0,8E0,1.44E2,3.3E1,1.17E2,2.7E1,7E0,2.6E1,9.5E1,2.2E1,1.2E1,1.5E1,1.6E1,1E1,7E0,8.8E1,1.7E1,5E0,6E0,6E0,8E0,7E0,6E0,1E1,5E0,5E0,6.2E1,2.6E1,1.2E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[9.258E-3,2.2918187E-1,-5.7043983E-3,3.7155168E-3,1.7624674E-2,6.5003214E-3,-1.6989365E-2,-1.5778549E-2,1.3813122E-1,-1.1845693E-2,-2.5330898E-3,1.421478E-2,5.9978753E-2,-1.8269397E-2,9.358504E-2,-1.38391815E-2,7.3769772E-3,-6.1244193E-2,3.347312E-2,1.5494566E-1,6.5755754E-5,2.9625369E-3,-4.1780593E-3,-1.2168624E-3,-8.806099E-3,-4.9815625E-3,2.9706287E-3,9.4350055E-3,3.8388711E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,-1,-1,7,-1,9,11,-1,13,-1,15,17,19,21,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.19938E-1,2.238397E-1,7.5604993E-1,0E0,0E0,5.065846E-1,0E0,4.5314822E-1,2.9203278E-1,0E0,2.1497849E-1,0E0,1.2703401E-1,2.7333218E-1,1.12951666E-1,6.6469505E-2,0E0,2.8968546E-1,2.2608146E-1,2.1920502E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,10,10,12,12,13,13,14,14,15,15,17,17,18,18,19,19],"right_children":[2,4,6,-1,-1,8,-1,10,12,-1,14,-1,16,18,20,22,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6604617E-2,1.594E3,1.485318E8,3.7155168E-3,1.7624674E-2,3.847E3,-1.6989365E-2,6E0,2.13281E5,-1.1845693E-2,1.1256753E0,1.421478E-2,9.6600875E5,1.7206225E0,2E0,1E0,7.3769772E-3,1.8271865E0,1.4827905E2,1.537E4,6.5755754E-5,2.9625369E-3,-4.1780593E-3,-1.2168624E-3,-8.806099E-3,-4.9815625E-3,2.9706287E-3,9.4350055E-3,3.8388711E-3],"split_indices":[26,2,44,0,0,2,0,3,28,0,38,0,27,52,8,15,0,53,46,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.1E1,1.76E2,6E0,5E0,1.71E2,5E0,1.47E2,2.4E1,7E0,1.4E2,7E0,1.7E1,1.21E2,1.9E1,1E1,7E0,6.6E1,5.5E1,1.1E1,8E0,5E0,5E0,5.2E1,1.4E1,9E0,4.6E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.1622097E-2,-5.0481926E-3,2.2760473E-1,1.0897221E-2,-2.0660275E-1,-3.7855402E-4,3.4592232E-1,-9.52579E-2,2.6778463E-2,-1.330758E-4,-1.9640733E-2,2.1852504E-2,8.170833E-3,-2.3192376E-2,-8.659962E-3,3.1985189E-3,9.022446E-2,3.0776726E-3,-5.812834E-3,5.588069E-2,-5.0624076E-2,4.0167157E-2,1.5028565E-1,-3.0274395E-2,1.1432656E-1,-1.2019506E-1,4.0221903E-2,1.2888435E-2,5.0886264E-3,9.697122E-3,1.08191915E-1,2.445736E-3,-5.2146735E-3,6.964426E-3,-1.6324653E-3,-8.110077E-3,1.0754179E-4,5.797402E-3,-1.7815169E-3,3.2207412E-3,-1.7651679E-3,1.5030974E-3,7.999609E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,-1,19,21,-1,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0064808E0,5.261123E-1,5.878887E-1,2.588039E-1,4.9626458E-1,0E0,1.9279122E-1,1.1904432E-1,1.9868648E-1,0E0,0E0,0E0,0E0,1.1076291E-1,0E0,2.8070706E-1,1.021975E-1,0E0,0E0,2.5523165E-1,3.1452644E-1,3.646726E-2,1.3206542E-2,1.3974428E-1,1.4315921E-1,1.6866964E-1,1.420803E-1,4.5715913E-2,0E0,0E0,4.4434443E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,30,30],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,-1,20,22,-1,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,1.00068504E8,1.3841E4,1.4150975E4,5.29E2,-3.7855402E-4,1.442E3,2.0778067E0,3.2970743E2,-1.330758E-4,-1.9640733E-2,2.1852504E-2,8.170833E-3,4.5198887E3,-8.659962E-3,6.769573E6,1.2978232E6,3.0776726E-3,-5.812834E-3,1.6407117E2,1.8270676E0,2.7034E4,1.1421988E0,4.3243725E5,4.6385403E5,1.7E1,3.64E2,3.23153E5,5.0886264E-3,9.697122E-3,2.798E3,2.445736E-3,-5.2146735E-3,6.964426E-3,-1.6324653E-3,-8.110077E-3,1.0754179E-4,5.797402E-3,-1.7815169E-3,3.2207412E-3,-1.7651679E-3,1.5030974E-3,7.999609E-3],"split_indices":[2,44,9,50,0,0,0,53,51,0,0,0,0,50,0,44,27,0,0,4,52,10,55,27,32,8,0,28,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.82E2,1.62E2,2E1,1.51E2,1.1E1,7E0,1.3E1,1.9E1,1.32E2,6E0,5E0,7E0,6E0,1.1E1,8E0,9.7E1,3.5E1,6E0,5E0,4.9E1,4.8E1,2E1,1.5E1,2E1,2.9E1,2.7E1,2.1E1,1.5E1,5E0,5E0,1E1,1E1,1E1,2.4E1,5E0,1.9E1,8E0,1E1,1.1E1,7E0,8E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.8618332E-2,3.137787E-2,-1.6613502E-2,9.732362E-3,2.2172322E-2,-6.0146726E-3,3.0211026E-2,6.0974315E-2,-1.4080745E-2,5.558436E-3,1.11343086E-1,3.127791E-2,-1.0604917E-1,-3.7302032E-2,8.196357E-2,1.7635198E-1,3.4987167E-2,1.0753848E-1,-2.1391442E-2,2.0714686E-3,-1.7826769E-1,-6.5118684E-3,1.7432083E-4,4.897972E-4,5.1192315E-3,6.1948556E-3,1.38725275E-2,6.18817E-3,-1.8125791E-3,8.9236805E-3,2.6287385E-3,-8.155214E-3,6.600618E-4,4.9080336E-3,-4.7260923E-3,-1.2074854E-2,-4.842628E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,-1,-1,5,-1,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.666473E-1,3.1015754E-1,0E0,0E0,2.2941011E-1,0E0,2.4748202E-1,2.9804265E-1,3.1618428E-1,1.7334493E-1,2.7180356E-1,2.0714672E-1,1.9557711E-1,1.3777393E-1,3.2835647E-2,1.223411E-1,1.900612E-1,7.68546E-2,1.6461006E-1,1.21313654E-1,5.9937805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,-1,6,-1,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,1.03932914E5,-1.6613502E-2,9.732362E-3,1.68128E8,-6.0146726E-3,2.1358025E0,1.059E3,6.872321E-1,1E0,6.5411394E5,5.81964E-2,6.82E2,8.7722336E4,9.608108E1,2.342E3,1.39778E5,1.4416069E9,2.7573213E8,1.5316E4,6.567405E2,-6.5118684E-3,1.7432083E-4,4.897972E-4,5.1192315E-3,6.1948556E-3,1.38725275E-2,6.18817E-3,-1.8125791E-3,8.9236805E-3,2.6287385E-3,-8.155214E-3,6.600618E-4,4.9080336E-3,-4.7260923E-3,-1.2074854E-2,-4.842628E-3],"split_indices":[44,27,0,0,7,0,55,2,26,15,31,40,0,31,4,2,28,5,7,9,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.98E2,6E0,9E0,1.89E2,9E0,1.8E2,1.06E2,7.4E1,5.1E1,5.5E1,5E1,2.4E1,3.3E1,1.8E1,2.9E1,2.6E1,2E1,3E1,1E1,1.4E1,9E0,2.4E1,5E0,1.3E1,2.2E1,7E0,1.1E1,1.5E1,7E0,1.3E1,5E0,2.5E1,5E0,5E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.5588157E-2,-1.0718863E-2,2.830293E-2,6.3396767E-3,1.5279642E-1,-8.056633E-3,1.4230662E-2,-1.3687786E-3,2.1461266E-1,4.795031E-2,-2.4838245E-2,9.5527455E-2,3.1418762E-1,7.034596E-2,-9.041948E-2,6.6778235E-2,-6.1359547E-2,4.5723846E-4,7.933124E-3,1.8646631E-2,8.9490805E-3,4.904829E-3,1.22576244E-1,-1.0204915E-2,1.3210474E-3,-2.3568382E-3,1.0609446E-1,-1.4450073E-1,-1.6848097E-2,1.4263155E-3,-4.3937294E-3,9.405544E-3,3.125113E-3,1.4751039E-3,6.588575E-3,-3.833653E-3,-1.0305003E-2,4.575123E-3,-2.5713649E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,-1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.027534E-1,0E0,5.1034206E-1,2.227187E-1,3.2005197E-1,0E0,2.0406903E-1,0E0,2.1900243E-1,2.635678E-1,2.4500613E-1,6.468565E-2,3.2432556E-2,2.4549541E-1,1.7945173E-1,1.00971304E-1,1.9000363E-1,0E0,0E0,0E0,0E0,8.201433E-2,1.563561E-1,0E0,0E0,0E0,3.3634797E-2,6.472629E-2,1.49923E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,21,21,22,22,26,26,27,27,28,28],"right_children":[2,-1,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0287E4,-1.0718863E-2,3.625E3,8.819892E2,1.2047E4,-8.056633E-3,2.9420671E0,-1.3687786E-3,5.818E3,2.8233623E11,1.3469E4,1.8308E4,6.135771E9,2.3114078E2,3.494881E2,2.6089528E5,1.0419601E2,4.5723846E-4,7.933124E-3,1.8646631E-2,8.9490805E-3,3.0875E4,8.5913794E5,-1.0204915E-2,1.3210474E-3,-2.3568382E-3,1.062E3,1E0,1.4507979E0,1.4263155E-3,-4.3937294E-3,9.405544E-3,3.125113E-3,1.4751039E-3,6.588575E-3,-3.833653E-3,-1.0305003E-2,4.575123E-3,-2.5713649E-3],"split_indices":[9,0,2,47,9,0,56,0,2,30,9,10,5,4,51,27,51,0,0,0,0,28,31,0,0,0,2,80,52,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,9E0,1.86E2,1.59E2,2.7E1,6E0,1.53E2,7E0,2E1,8.2E1,7.1E1,1E1,1E1,7.1E1,1.1E1,2E1,5.1E1,5E0,5E0,5E0,5E0,3.2E1,3.9E1,5E0,6E0,5E0,1.5E1,1.7E1,3.4E1,2.6E1,6E0,1.6E1,2.3E1,5E0,1E1,1E1,7E0,8E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.486105E-3,4.1557796E-2,-9.416759E-2,2.7200432E-2,1.4971977E-2,-3.525441E-2,-2.6240265E-1,8.793478E-2,-2.7163435E-2,-4.185474E-3,-1.144165E-2,-5.082664E-3,-1.9495143E-2,6.7650564E-2,1.1662892E-2,-1.7416552E-1,2.200309E-3,-4.0547837E-2,1.052504E-1,8.922179E-2,-2.8604285E-3,-4.0774103E-3,-1.0809043E-2,3.4158327E-2,-1.4616871E-1,-1.284168E-1,3.2511782E-3,4.4347206E-4,8.263947E-3,5.300417E-2,1.6287145E-1,7.574886E-2,-1.5787225E-2,-9.569575E-3,-3.2687078E-3,-2.394403E-3,-8.494344E-3,4.3361983E-3,-3.5711277E-2,1.1014144E-3,6.024018E-3,9.776303E-3,2.8845135E-3,6.003184E-3,-3.683954E-5,-2.5358978E-3,5.0049843E-3,-3.8350436E-3,2.1749863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,27,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.6168945E-1,5.400181E-1,6.8453765E-1,4.481254E-1,0E0,3.3420867E-1,3.6345947E-1,1.8540865E-1,3.111735E-1,1.9129701E-1,0E0,0E0,0E0,1.6466716E-1,0E0,3.5785586E-2,2.9422924E-1,1.3896619E-1,7.586396E-2,1.2473792E-1,0E0,0E0,0E0,1.0713622E-1,3.2307073E-2,3.715396E-2,8.822799E-2,0E0,0E0,7.351835E-2,5.5374265E-2,1.0627167E-1,1.1273695E-1,0E0,0E0,0E0,0E0,0E0,6.836739E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,18,18,19,19,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,28,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.837056E-1,1.3904022E3,2.0856102E0,2.0218372E0,1.4971977E-2,4.674595E6,9.309742E-1,2.902E3,1.4065148E6,9.9472994E-1,-1.144165E-2,-5.082664E-3,-1.9495143E-2,2.72126E5,1.1662892E-2,1.6999166E5,2.5086E4,1.6827956E0,1.8656379E6,1.135E3,-2.8604285E-3,-4.0774103E-3,-1.0809043E-2,3.5293162E5,9.60419E5,4.2905E4,1.9182949E0,4.4347206E-4,8.263947E-3,2.3357933E0,6.748347E5,1E1,2.9E1,-9.569575E-3,-3.2687078E-3,-2.394403E-3,-8.494344E-3,4.3361983E-3,2.20347E0,1.1014144E-3,6.024018E-3,9.776303E-3,2.8845135E-3,6.003184E-3,-3.683954E-5,-2.5358978E-3,5.0049843E-3,-3.8350436E-3,2.1749863E-3],"split_indices":[26,51,55,57,0,27,26,2,49,38,0,0,0,28,0,46,9,52,27,2,0,0,0,32,9,28,52,0,0,53,31,3,8,0,0,0,0,0,52,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.4E2,6.9E1,1.34E2,6E0,5.2E1,1.7E1,6.3E1,7.1E1,4.6E1,6E0,9E0,8E0,5.7E1,6E0,1.1E1,6E1,3.5E1,1.1E1,4.9E1,8E0,5E0,6E0,5E1,1E1,1.1E1,2.4E1,5E0,6E0,3.4E1,1.5E1,2.7E1,2.3E1,5E0,5E0,5E0,6E0,7E0,1.7E1,2.5E1,9E0,1E1,5E0,1.6E1,1.1E1,1.8E1,5E0,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-6.508105E-3,1.6223421E-2,-1.4915955E-1,2.065409E-3,1.2026902E-2,-2.4486842E-2,-4.825025E-2,-7.387953E-3,1.1943197E-2,-6.3995714E-3,5.6979783E-2,-8.320494E-2,2.2505622E-2,-2.7172603E-3,7.7219056E-3,2.3895928E-3,-1.4823179E-1,9.554526E-2,5.457966E-3,-3.134557E-3,-9.88493E-3,4.9054846E-2,8.128773E-3,-6.606881E-3,1.8681938E-2,4.279873E-3,7.330425E-4,5.8659716E-3,-9.807984E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,9,-1,11,-1,13,15,17,-1,-1,-1,19,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,-1],"loss_changes":[6.606849E-1,5.845995E-1,9.9803925E-1,2.596325E-1,0E0,0E0,2.1413289E-1,0E0,1.6066182E-1,0E0,1.3946344E-1,1.4585742E-1,1.7753564E-1,0E0,0E0,0E0,3.94184E-2,8.4725484E-2,2.2439085E-1,0E0,0E0,2.3105893E-2,0E0,0E0,2.3254266E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,10,10,11,11,12,12,16,16,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,-1,-1,10,-1,12,-1,14,16,18,-1,-1,-1,20,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,-1],"split_conditions":[7.18771E7,5.818E3,7.567149E4,1.4827905E2,1.2026902E-2,-2.4486842E-2,1.118307E6,-7.387953E-3,5.94E2,-6.3995714E-3,9.508291E2,1E0,2.42E2,-2.7172603E-3,7.7219056E-3,2.3895928E-3,3.8E1,1.0215677E2,3.1743118E-1,-3.134557E-3,-9.88493E-3,2.2673786E5,8.128773E-3,-6.606881E-3,6.4266656E5,4.279873E-3,7.330425E-4,5.8659716E-3,-9.807984E-5],"split_indices":[44,2,31,46,0,0,27,0,2,0,4,111,10,0,0,0,10,51,57,0,0,31,0,0,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.75E2,2.7E1,1.66E2,9E0,5E0,2.2E1,9E0,1.57E2,1.2E1,1E1,1.5E1,1.42E2,5E0,5E0,5E0,1E1,2.6E1,1.16E2,5E0,5E0,1.7E1,9E0,9E0,1.07E2,7E0,1E1,1.7E1,9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[9.90877E-3,2.3706255E-2,-2.119755E-2,9.805705E-2,-3.1209667E-3,6.32914E-2,1.2318536E-2,-3.8262174E-2,5.3411704E-2,-2.467048E-3,8.2770735E-2,-1.938913E-2,-8.973596E-3,1.1987318E-2,1.4019722E-1,-3.2636008E-4,1.03218876E-1,2.5436036E-2,-6.83146E-2,-8.005571E-2,4.6486836E-2,1.6948196E-1,2.5573832E-3,1.4141361E-1,3.9455898E-2,-4.5608737E-2,6.9225855E-2,3.7034915E-3,-9.200269E-2,-1.9178271E-3,-5.1136226E-3,1.06466666E-1,-2.1146114E-3,1.0752613E-2,4.3266416E-3,4.2873137E-3,9.062497E-3,2.9874174E-3,1.1235389E-4,1.1143949E-3,-5.24169E-3,-5.564356E-4,6.9525344E-3,-2.6034045E-3,-8.477653E-3,6.8726144E-3,1.8879018E-3,2.157627E-3,-4.6022916E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,15,17,-1,19,21,-1,23,25,27,29,31,33,-1,35,37,39,41,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2650425E0,3.912526E-1,0E0,2.7129525E-1,2.9010448E-1,1.0264477E-1,0E0,2.495329E-1,1.9964147E-1,0E0,7.0311725E-2,1.7937812E-1,0E0,1.2756371E-1,3.6716074E-2,0E0,6.9918126E-2,1.3693194E-1,1.4163953E-1,7.4744076E-3,8.476323E-2,3.629613E-2,0E0,3.0179203E-2,1.0902125E-2,7.7243544E-2,1.6734256E-1,0E0,9.916505E-2,0E0,0E0,2.659136E-2,7.963385E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,28,28,31,31,32,32],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,16,18,-1,20,22,-1,24,26,28,30,32,34,-1,36,38,40,42,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4089414E8,3.77E2,-2.119755E-2,1.564E3,9.835643E5,8.041484E-5,1.2318536E-2,4.9462875E5,5.8018835E2,-2.467048E-3,3.2503597E1,6.2654173E-1,-8.973596E-3,2.0618556E-2,2.304414E-1,-3.2636008E-4,2.1848371E8,3.1622803E5,3.08E3,3.0734711E-6,8.563462E-1,1.7446084E0,2.5573832E-3,8E0,4.283054E6,1.872148E-1,3.6907338E-2,3.7034915E-3,1.1E1,-1.9178271E-3,-5.1136226E-3,1.4320541E1,4.1847788E6,1.0752613E-2,4.3266416E-3,4.2873137E-3,9.062497E-3,2.9874174E-3,1.1235389E-4,1.1143949E-3,-5.24169E-3,-5.564356E-4,6.9525344E-3,-2.6034045E-3,-8.477653E-3,6.8726144E-3,1.8879018E-3,2.157627E-3,-4.6022916E-3],"split_indices":[44,0,0,2,27,37,0,32,51,0,51,26,0,56,37,0,7,27,11,37,26,52,0,3,44,38,37,0,3,0,0,57,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.95E2,5E0,5.1E1,1.44E2,4.3E1,8E0,8.9E1,5.5E1,6E0,3.7E1,8E1,9E0,3.8E1,1.7E1,7E0,3E1,4.2E1,3.8E1,1E1,2.8E1,1.2E1,5E0,1.8E1,1.2E1,1.6E1,2.6E1,5E0,3.3E1,5E0,5E0,1.2E1,1.6E1,6E0,6E0,1E1,8E0,7E0,5E0,8E0,8E0,1.3E1,1.3E1,2.4E1,9E0,7E0,5E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[8.647224E-3,1.9496158E-2,-1.08353505E-2,-1.1126499E-1,3.187505E-2,-1.597631E-1,2.0482688E-4,1.06668904E-1,9.422713E-3,-1.0229649E-2,-3.3733766E-3,7.389277E-2,1.3680107E-2,-3.797559E-2,5.737139E-2,1.1217321E-1,-2.4628686E-3,-6.7563714E-3,-1.3490868E-1,1.3849469E-1,-2.0838702E-2,1.9041004E-3,1.2914152E-1,-4.1387137E-3,2.9122985E-3,7.712396E-2,-5.449495E-2,-1.1433886E-2,5.5795244E-4,1.8086307E-1,6.5049626E-2,-8.939601E-2,3.847694E-2,7.132906E-3,3.5463409E-3,1.354723E-3,7.035777E-3,-6.3518547E-3,5.1264965E-4,3.5332786E-3,1.0481925E-2,8.578658E-3,-8.713419E-4,-7.4946424E-3,-1.2850695E-3,-3.2911068E-3,3.7742646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,-1,15,-1,17,19,21,23,25,27,29,31,-1,33,-1,-1,35,37,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1900417E-1,3.1757018E-1,0E0,9.594238E-2,2.9790974E-1,4.2608976E-2,0E0,2.1928102E-1,3.1580937E-1,0E0,0E0,1.0550165E-1,0E0,2.1092077E-1,4.4066757E-1,2.43631E-2,7.426105E-2,2.2043659E-1,2.612734E-1,9.4034374E-2,1.4983442E-1,0E0,9.6566975E-3,0E0,0E0,6.397368E-2,1.7960474E-1,0E0,0E0,6.9177985E-2,1.3612026E-1,6.711765E-2,9.180185E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,29,29,30,30,31,31,32,32],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,-1,16,-1,18,20,22,24,26,28,30,32,-1,34,-1,-1,36,38,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.3086E4,1.4827905E2,-1.08353505E-2,3.9925885E2,3.06E2,1.7284092E-6,2.0482688E-4,1.564E3,1.6720915E2,-1.0229649E-2,-3.3733766E-3,6.805556E5,1.3680107E-2,2.5492957E0,4.17149E5,1.4781E4,1.5227739E6,2.2193549E0,1.307E3,7.4456794E5,1.3298E4,1.9041004E-3,2.201776E6,-4.1387137E-3,2.9122985E-3,1.5E0,1.8297239E6,-1.1433886E-2,5.5795244E-4,3.1274893E2,1.60756E5,1.8E1,3.6129813E2,7.132906E-3,3.5463409E-3,1.354723E-3,7.035777E-3,-6.3518547E-3,5.1264965E-4,3.5332786E-3,1.0481925E-2,8.578658E-3,-8.713419E-4,-7.4946424E-3,-1.2850695E-3,-3.2911068E-3,3.7742646E-3],"split_indices":[10,46,0,51,0,37,0,2,51,0,0,27,0,55,28,9,27,57,2,31,9,0,31,0,0,57,49,0,0,4,28,3,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.93E2,8E0,1.6E1,1.77E2,1.1E1,5E0,4E1,1.37E2,6E0,5E0,3.5E1,5E0,6.9E1,6.8E1,2.3E1,1.2E1,5.3E1,1.6E1,3.3E1,3.5E1,5E0,1.8E1,5E0,7E0,1.9E1,3.4E1,9E0,7E0,2E1,1.3E1,1.6E1,1.9E1,1.2E1,6E0,1.2E1,7E0,1.5E1,1.9E1,6E0,1.4E1,5E0,8E0,7E0,9E0,5E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-5.417669E-4,9.957788E-3,-1.3962981E-2,-9.563695E-3,1.4863345E-1,7.3793177E-3,-1.2464952E-1,1.5305095E-2,2.483602E-2,-5.8863517E-3,1.5582397E-2,-5.1496316E-2,-1.4906296E-2,4.7171833E-3,-1.0455315E-3,8.380311E-2,-1.3908334E-2,-7.929708E-3,2.5744634E-2,4.8982598E-2,1.3950132E-1,1.436248E-2,-6.041236E-2,3.9720554E-3,-1.7108541E-3,4.962254E-3,1.0523302E-3,9.880532E-3,3.0986848E-3,-7.108199E-4,5.635014E-3,4.7363034E-3,-4.4729947E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,-1,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.137879E-1,5.266094E-1,0E0,3.3435282E-1,4.9963355E-1,1.6217765E-1,2.8411284E-1,0E0,5.367968E-2,0E0,2.8685236E-1,1.5398309E-1,0E0,0E0,0E0,7.656276E-2,1.3233833E-1,0E0,4.16049E-2,3.9025195E-2,6.6554755E-2,1.916962E-1,2.086776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,10,10,11,11,15,15,16,16,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,5.251E3,-1.3962981E-2,9.4801545E-1,1.9824325E0,9.541066E5,1.7999E4,1.5305095E-2,1.14416916E2,-5.8863517E-3,5.0341E4,8E0,-1.4906296E-2,4.7171833E-3,-1.0455315E-3,2.5828317E2,7.0248324E-1,-7.929708E-3,3.830986E-1,4.4358948E-1,2.5537605E8,6.2654173E-1,1.453034E0,3.9720554E-3,-1.7108541E-3,4.962254E-3,1.0523302E-3,9.880532E-3,3.0986848E-3,-7.108199E-4,5.635014E-3,4.7363034E-3,-4.4729947E-3],"split_indices":[44,2,0,26,53,44,10,0,57,0,28,3,0,0,0,4,26,0,55,26,7,26,52,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.93E2,6E0,1.7E2,2.3E1,1.49E2,2.1E1,9E0,1.4E1,8E0,1.41E2,1.6E1,5E0,5E0,9E0,4.2E1,9.9E1,6E0,1E1,2.7E1,1.5E1,6.2E1,3.7E1,5E0,5E0,8E0,1.9E1,7E0,8E0,4.9E1,1.3E1,6E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[9.526137E-3,1.7923448E-2,-1.3029722E-2,-9.830976E-3,2.9511666E-2,2.1925574E-2,1.0816995E-2,8.412871E-3,1.08070366E-1,-8.296361E-3,6.4635605E-2,1.1868334E-2,6.0768947E-2,9.757413E-3,-1.0356654E-1,1.4077666E-1,2.7007591E-2,1.0870775E-1,-4.7436738E-4,-1.9907411E-3,1.928068E-3,-7.215144E-3,-1.40581E-3,3.1957072E-3,9.760426E-3,3.6754857E-3,-4.2432707E-4,7.925963E-3,2.1343308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,-1,-1,5,7,-1,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8945904E-1,5.231855E-1,0E0,0E0,2.794388E-1,2.1443993E-1,0E0,1.5178588E-1,1.5009823E-1,2.1600886E-1,1.0220432E-1,0E0,6.8833575E-2,1.674914E-1,6.5404415E-2,4.2368308E-2,4.703456E-2,3.6307275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17],"right_children":[2,4,-1,-1,6,8,-1,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,6E0,-1.3029722E-2,-9.830976E-3,1.3717E4,1.2221828E0,1.0816995E-2,2.078125E0,1E0,2.1159E4,6.063759E8,1.1868334E-2,1.8933852E0,1.085E3,1.3E1,3.8372688E8,2.4178083E0,6.6828716E-1,-4.7436738E-4,-1.9907411E-3,1.928068E-3,-7.215144E-3,-1.40581E-3,3.1957072E-3,9.760426E-3,3.6754857E-3,-4.2432707E-4,7.925963E-3,2.1343308E-3],"split_indices":[27,3,0,0,2,38,0,52,63,9,5,0,53,2,8,5,52,26,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.99E2,5E0,9E0,1.9E2,1.84E2,6E0,1.6E2,2.4E1,1.24E2,3.6E1,5E0,1.9E1,1.05E2,1.9E1,1.1E1,2.5E1,1.1E1,8E0,3.9E1,6.6E1,1.1E1,8E0,6E0,5E0,1E1,1.5E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.2389604E-2,2.8968551E-3,-1.3933234E-1,-8.578134E-3,1.5661155E-1,-2.6708233E-1,-4.9959205E-2,6.243325E-3,-1.0250659E-1,2.4541242E-3,1.2142852E-2,-2.93747E-3,-1.9041086E-2,5.1775025E-3,-1.4144672E-1,-7.181487E-2,2.8184786E-2,-1.5944836E-1,3.6368573E-3,-9.700135E-3,-3.0057589E-3,1.1973319E-3,-1.12553746E-1,1.3296191E-1,1.0241429E-2,-2.0866373E-1,-1.6500998E-3,-1.0029227E-2,-6.317428E-2,3.4055253E-3,7.8081065E-3,-7.927909E-2,2.5400525E-2,-1.1923528E-2,-6.082681E-3,5.4569397E-4,-7.697447E-3,-1.0972073E-3,-6.342307E-3,5.209385E-3,3.1830545E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,19,21,23,25,-1,-1,-1,-1,27,29,31,33,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1547375E-1,3.0472982E-1,4.091012E-1,2.2477722E-1,1.0949436E-1,3.7800765E-1,3.4781548E-1,2.4180362E-1,2.356135E-1,0E0,0E0,0E0,0E0,0E0,5.934453E-2,1.2507102E-1,2.0544416E-1,9.740031E-2,0E0,0E0,0E0,0E0,9.192073E-2,1.8134028E-2,1.309335E-1,7.830203E-3,0E0,0E0,1.181462E-1,0E0,0E0,3.8882725E-2,1.2745085E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,14,14,15,15,16,16,17,17,22,22,23,23,24,24,25,25,28,28,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,20,22,24,26,-1,-1,-1,-1,28,30,32,34,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.18771E7,5.666E3,3.4092497E5,3.10364E5,2E0,8.640031E9,1.4689625E5,7.38E2,4.427746E0,2.4541242E-3,1.2142852E-2,-2.93747E-3,-1.9041086E-2,5.1775025E-3,1.1484E4,4.5154606E5,2.86E2,1.569E3,3.6368573E-3,-9.700135E-3,-3.0057589E-3,1.1973319E-3,4.01054E1,1.5596E4,2.0033424E4,6E0,-1.6500998E-3,-1.0029227E-2,9.77E1,3.4055253E-3,7.8081065E-3,1.061E3,6.4266656E5,-1.1923528E-2,-6.082681E-3,5.4569397E-4,-7.697447E-3,-1.0972073E-3,-6.342307E-3,5.209385E-3,3.1830545E-4],"split_indices":[44,2,31,9,8,5,32,2,52,0,0,0,0,0,9,27,0,0,0,0,0,0,51,9,50,8,0,0,51,0,0,2,50,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.71E2,3.6E1,1.6E2,1.1E1,1.4E1,2.2E1,1.39E2,2.1E1,6E0,5E0,6E0,8E0,8E0,1.4E1,3E1,1.09E2,1.6E1,5E0,7E0,7E0,9E0,2.1E1,1.5E1,9.4E1,1.1E1,5E0,6E0,1.5E1,6E0,9E0,1.3E1,8.1E1,6E0,5E0,9E0,6E0,7E0,6E0,1.4E1,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.1634502E-2,-9.842955E-3,-1.6948089E-2,-1.8391602E-2,9.41662E-3,5.338623E-3,-8.9693494E-2,-1.685271E-2,5.196848E-2,-1.3748512E-2,-4.9011827E-2,6.348203E-2,-3.4281645E-2,7.837441E-2,-3.268857E-3,-8.423813E-2,5.1178765E-3,6.7976667E-3,-6.4352644E-4,-6.315026E-2,3.4064792E-2,-1.7868509E-3,1.0053585E-1,-3.812431E-2,-1.4739184E-1,-5.78529E-4,-5.452369E-3,5.844637E-3,-3.30967E-4,7.553949E-3,2.1156387E-3,-2.9762862E-3,1.5441101E-3,-1.0527569E-2,-2.091088E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9935586E-1,3.5473505E-1,0E0,3.2534474E-1,0E0,1.5091874E-1,3.71171E-1,1.4079101E-1,1.5452522E-1,0E0,2.3407266E-1,1.104573E-1,1.6394264E-1,1.03724316E-1,0E0,9.33976E-2,0E0,0E0,0E0,1.4869128E-1,9.091345E-2,0E0,9.933871E-2,3.7461564E-2,9.549427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,1.0086E4,-1.6948089E-2,4.3625324E7,9.41662E-3,2.170343E0,5.8866205E2,1.3257E4,1.472353E0,-1.3748512E-2,2.1907706E12,6.886894E0,4.1698615E2,1.7311496E0,-3.268857E-3,2.5831E4,5.1178765E-3,6.7976667E-3,-6.4352644E-4,1.7111913E0,1.4416069E9,-1.7868509E-3,6.063759E8,9.083174E2,2.7608912E1,-5.78529E-4,-5.452369E-3,5.844637E-3,-3.30967E-4,7.553949E-3,2.1156387E-3,-2.9762862E-3,1.5441101E-3,-1.0527569E-2,-2.091088E-3],"split_indices":[44,2,0,44,0,53,4,9,40,0,30,56,4,52,0,10,0,0,0,53,5,0,5,51,56,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.98E2,6E0,1.91E2,7E0,1.44E2,4.7E1,9.8E1,4.6E1,7E0,4E1,1.7E1,8.1E1,3.8E1,8E0,3.3E1,7E0,8E0,9E0,5.7E1,2.4E1,6E0,3.2E1,2E1,1.3E1,2.9E1,2.8E1,7E0,1.7E1,1.5E1,1.7E1,1.5E1,5E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.276032E-2,-1.2688185E-2,2.1324357E-2,-8.462374E-3,8.292337E-2,4.3239994E-3,-1.356209E-1,1.7443113E-1,2.2138298E-2,-8.407238E-2,1.9729929E-2,-1.0491965E-2,-2.145239E-3,9.395743E-2,1.7627442E-2,5.078603E-2,-6.893461E-3,-1.2056291E-1,4.312951E-4,5.2116897E-2,-5.0548993E-2,7.556235E-3,9.963198E-4,-3.8994014E-2,1.1558176E-1,-2.3126125E-3,-8.414187E-3,7.531314E-2,9.07289E-3,-1.2205438E-1,-3.8766612E-3,-3.7436762E-3,8.101443E-4,3.968125E-2,1.0754098E-2,4.6975417E-3,-2.852415E-4,2.057832E-3,-2.36818E-3,-8.739845E-3,-2.1196534E-3,-4.122912E-3,2.077363E-3,-4.8944075E-4,3.4563073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,-1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,25,-1,27,29,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.763738E-1,0E0,3.5482454E-1,2.1357886E-1,3.4656367E-1,1.6507816E-1,8.1087336E-2,3.5474062E-1,1.9280294E-1,6.2219694E-2,2.3707494E-1,0E0,0E0,8.535756E-2,0E0,2.0229733E-1,0E0,4.3274224E-2,0E0,7.020731E-2,1.096585E-1,0E0,0E0,3.414993E-2,1.5638971E-1,0E0,0E0,8.559072E-2,5.365955E-2,5.2999213E-2,8.5255414E-2,0E0,0E0,2.1794658E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,17,17,19,19,20,20,23,23,24,24,27,27,28,28,29,29,30,30,33,33],"right_children":[2,-1,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,26,-1,28,30,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01E4,-1.2688185E-2,1.996E3,4.939386E7,4.083812E7,4.1E1,9.064853E-1,5.666E3,5.69815E5,2.377416E6,1.00473E5,-1.0491965E-2,-2.145239E-3,7.4456794E5,1.7627442E-2,1.3841E4,-6.893461E-3,3.0847954E1,4.312951E-4,3.5540915E0,3.9232688E5,7.556235E-3,9.963198E-4,2.0907634E3,7.7186523E2,-2.3126125E-3,-8.414187E-3,1.3697663E7,1.5798E4,3.5000316E5,1E0,-3.7436762E-3,8.101443E-4,1.4E1,1.0754098E-2,4.6975417E-3,-2.852415E-4,2.057832E-3,-2.36818E-3,-8.739845E-3,-2.1196534E-3,-4.122912E-3,2.077363E-3,-4.8944075E-4,3.4563073E-3],"split_indices":[9,0,2,44,44,51,26,2,9,44,28,0,0,31,0,9,0,51,0,56,27,0,0,4,51,0,0,44,9,31,63,0,0,8,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,5E0,1.92E2,1.3E2,6.2E1,1.19E2,1.1E1,2.4E1,3.8E1,1.7E1,1.02E2,5E0,6E0,1.8E1,6E0,3.3E1,5E0,1.2E1,5E0,7E1,3.2E1,9E0,9E0,1.4E1,1.9E1,6E0,6E0,4.5E1,2.5E1,1.2E1,2E1,8E0,6E0,1.2E1,7E0,3.5E1,1E1,1.6E1,9E0,6E0,6E0,7E0,1.3E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-3.4794533E-3,2.6454803E-2,-5.5916402E-2,1.7488146E-2,1.0363684E-2,-2.531659E-2,-1.7099589E-1,-4.225516E-3,6.0024522E-2,-4.1870818E-2,4.3911966E-3,-2.3966893E-3,-1.2433394E-2,3.2045785E-2,-4.9688805E-2,8.825244E-3,3.1551097E-2,-2.5482023E-2,-7.2155152E-3,7.593517E-2,-3.0089356E-2,-1.01271614E-1,6.1266944E-2,6.1673205E-2,-4.5253262E-2,-5.709456E-2,1.9386811E-2,4.601767E-2,8.764797E-3,3.3287395E-2,-5.667091E-3,-1.4231472E-1,1.7402736E-3,5.7093985E-3,-2.5831757E-4,3.7106775E-2,5.5276696E-3,4.4736642E-4,-4.422049E-3,-6.281703E-3,-3.2082945E-2,5.201976E-3,-5.655795E-2,3.4515057E-3,-2.1112477E-3,-9.780844E-4,4.759391E-3,-1.2670521E-3,-8.473561E-3,-1.102822E-3,2.8114272E-3,4.369223E-4,-3.7301746E-3,-2.8927648E-4,-4.397073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,21,-1,23,25,-1,27,29,31,33,35,37,39,41,43,-1,45,-1,47,-1,-1,-1,49,-1,-1,-1,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3267772E-1,2.2605234E-1,2.6681963E-1,1.20324306E-1,0E0,1.2389642E-1,1.583651E-1,1.4505641E-1,1.4977263E-1,9.417862E-2,0E0,0E0,0E0,1.3591142E-1,2.2941756E-1,0E0,8.839075E-2,7.060933E-2,0E0,8.442737E-2,1.2132525E-1,1.5764928E-1,5.082228E-2,3.1342067E-2,2.9115021E-2,4.94861E-2,1.4839298E-1,5.9445433E-2,0E0,4.814477E-2,0E0,7.940328E-2,0E0,0E0,0E0,2.7297348E-2,0E0,0E0,0E0,0E0,4.409211E-2,0E0,2.0799331E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,14,14,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,31,31,35,35,40,40,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,22,-1,24,26,-1,28,30,32,34,36,38,40,42,44,-1,46,-1,48,-1,-1,-1,50,-1,-1,-1,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.228668E-1,1.0086E4,2.1492538E0,1.993E3,1.0363684E-2,3.4349198E6,7.6499896E9,6.063759E8,2.01351E5,2.201776E6,4.3911966E-3,-2.3966893E-3,-1.2433394E-2,2.3378747E0,4.1108723E8,8.825244E-3,2.418059E-1,1.8044023E5,-7.2155152E-3,1.43259E5,1.3184165E0,1.12160355E-1,3.5089097E5,1.3640963E3,7.45226E5,2.3135895E6,3.3897146E8,8.869983E5,8.764797E-3,1.127E3,-5.667091E-3,9.189396E4,1.7402736E-3,5.7093985E-3,-2.5831757E-4,2.5185E4,5.5276696E-3,4.4736642E-4,-4.422049E-3,-6.281703E-3,9.857292E-1,5.201976E-3,3.5089097E5,3.4515057E-3,-2.1112477E-3,-9.780844E-4,4.759391E-3,-1.2670521E-3,-8.473561E-3,-1.102822E-3,2.8114272E-3,4.369223E-4,-3.7301746E-3,-2.8927648E-4,-4.397073E-3],"split_indices":[26,2,55,2,0,31,5,5,28,31,0,0,0,57,7,0,41,46,0,1,52,37,32,4,28,44,7,31,0,2,0,31,0,0,0,11,0,0,0,0,55,0,32,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.34E2,7.6E1,1.29E2,5E0,6.1E1,1.5E1,8.6E1,4.3E1,5.4E1,7E0,7E0,8E0,4.8E1,3.8E1,7E0,3.6E1,4.8E1,6E0,2.8E1,2E1,2.6E1,1.2E1,2.6E1,1E1,2.8E1,2E1,2.3E1,5E0,1.2E1,8E0,2E1,6E0,6E0,6E0,1.9E1,7E0,5E0,5E0,6E0,2.2E1,9E0,1.1E1,1.8E1,5E0,7E0,5E0,5E0,1.5E1,5E0,1.4E1,1.2E1,1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-6.8071927E-3,3.3252921E-3,-1.5401018E-2,-8.267497E-2,1.6495116E-2,-1.1915976E-1,4.6711383E-4,1.0675588E-1,-4.6671294E-3,-1.6336638E-1,-1.2625835E-3,-8.955416E-4,1.4412394E-1,-5.1809225E-2,2.8212395E-2,-3.8822824E-3,-1.0089378E-2,6.7288384E-2,9.303808E-3,-1.0931614E-1,4.650273E-2,1.5048489E-2,6.8586054E-3,1.4825326E-3,4.427537E-3,3.3466555E-3,-1.4150657E-1,8.981178E-2,-2.5365364E-3,6.7976825E-2,-1.6345285E-2,-5.0717853E-3,-1.2764283E-2,6.4589083E-4,6.668862E-3,1.3675679E-3,6.3313507E-3,-2.1935876E-3,1.3367885E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,5,7,9,-1,11,13,15,-1,-1,17,19,21,-1,-1,23,-1,25,27,29,-1,-1,-1,-1,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.132226E-1,2.1316153E-1,0E0,8.54646E-2,3.118479E-1,6.954181E-2,0E0,1.4801514E-1,2.0761296E-1,2.9597044E-2,0E0,0E0,7.927585E-2,3.1603244E-1,1.17386304E-1,0E0,0E0,7.1858726E-3,0E0,2.1175134E-1,9.520243E-2,1.2074777E-1,0E0,0E0,0E0,0E0,1.0537225E-1,5.4670863E-2,0E0,6.455509E-2,6.1204318E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,17,17,19,19,20,20,21,21,26,26,27,27,29,29,30,30],"right_children":[2,4,-1,6,8,10,-1,12,14,16,-1,-1,18,20,22,-1,-1,24,-1,26,28,30,-1,-1,-1,-1,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,7.23E2,-1.5401018E-2,3.0050538E5,3.77E2,1.9107901E0,4.6711383E-4,5.033057E-4,3.2198023E-2,2.888788E-3,-1.2625835E-3,-8.955416E-4,1.511E4,3.4942383E2,2.363013E6,-3.8822824E-3,-1.0089378E-2,1.097E3,9.303808E-3,7.518072E-1,5.287008E5,5.4024344E5,6.8586054E-3,1.4825326E-3,4.427537E-3,3.3466555E-3,2.7E1,2.149E3,-2.5365364E-3,3.5519625E5,1.631769E0,-5.0717853E-3,-1.2764283E-2,6.4589083E-4,6.668862E-3,1.3675679E-3,6.3313507E-3,-2.1935876E-3,1.3367885E-3],"split_indices":[44,2,0,32,0,52,0,38,37,38,0,0,9,51,27,0,0,2,0,55,31,31,0,0,0,0,8,2,0,31,52,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.91E2,1.86E2,5E0,2.4E1,1.62E2,1.7E1,7E0,3E1,1.32E2,1.1E1,6E0,7E0,2.3E1,5.4E1,7.8E1,5E0,6E0,1E1,1.3E1,3.4E1,2E1,7.1E1,7E0,5E0,5E0,5E0,2.9E1,1.4E1,6E0,2.6E1,4.5E1,2.4E1,5E0,6E0,8E0,1.7E1,9E0,2.7E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.8308635E-3,1.46187525E-2,-1.3871151E-2,6.1980085E-3,1.3591889E-1,2.6856175E-2,-3.4569602E-2,1.2774899E-2,1.000254E-3,1.8641052E-3,7.6795936E-2,-7.93177E-2,2.9689558E-2,2.390677E-2,-7.59634E-2,9.5986865E-2,-1.5748015E-3,-9.646353E-3,-5.4726403E-2,4.495078E-3,-2.2537764E-3,5.889842E-3,1.0599353E-1,-7.1861115E-3,2.714796E-4,1.3231534E-1,2.153531E-2,2.993991E-3,-8.3090015E-2,4.436289E-3,-4.2744204E-2,-1.7108666E-3,2.1967562E-3,2.8913823E-4,8.457265E-3,4.871581E-3,9.929357E-3,-2.4663352E-3,5.5238428E-3,-1.3903013E-3,-6.6634226E-3,-1.7853742E-4,-4.5104027E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,-1,-1,35,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3893536E-1,2.0446648E-1,0E0,1.601012E-1,1.8994874E-1,1.571172E-1,1.863853E-1,0E0,0E0,1.476357E-1,9.154895E-2,1.03016466E-1,5.5527773E-2,9.846544E-2,1.15674116E-1,9.452015E-2,0E0,0E0,1.14911E-1,0E0,7.693046E-2,9.473883E-2,8.354321E-2,0E0,0E0,3.4016132E-2,9.493763E-2,0E0,7.771298E-2,0E0,2.7727442E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,20,20,21,21,22,22,25,25,26,26,28,28,30,30],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,-1,-1,36,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,6.146E3,-1.3871151E-2,2.1799106E7,2.9551638E6,1.08770256E2,2.1577182E0,1.2774899E-2,1.000254E-3,8.524E3,2.126E4,3.74449E2,2.5979605E0,9E0,6.109438E-1,3.7861453E5,-1.5748015E-3,-9.646353E-3,8.606702E-2,4.495078E-3,1.1890752E-1,3.0823356E5,1.131E3,-7.1861115E-3,2.714796E-4,1.0016339E6,4.3691156E5,2.993991E-3,7.7198017E-1,4.436289E-3,7.5644026E0,-1.7108666E-3,2.1967562E-3,2.8913823E-4,8.457265E-3,4.871581E-3,9.929357E-3,-2.4663352E-3,5.5238428E-3,-1.3903013E-3,-6.6634226E-3,-1.7853742E-4,-4.5104027E-3],"split_indices":[44,2,0,44,46,51,53,0,0,10,9,4,55,3,26,32,0,0,57,0,26,32,2,0,0,31,32,0,26,0,55,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,2E2,5E0,1.88E2,1.2E1,1.25E2,6.3E1,5E0,7E0,8.4E1,4.1E1,3.7E1,2.6E1,6.6E1,1.8E1,3.5E1,6E0,5E0,3.2E1,8E0,1.8E1,5.5E1,1.1E1,9E0,9E0,2.3E1,1.2E1,6E0,2.6E1,5E0,1.3E1,2.7E1,2.8E1,5E0,6E0,1.8E1,5E0,7E0,5E0,1.4E1,1.2E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.153934E-3,-1.0263918E-2,4.7890167E-3,-2.3694092E-3,1.3645916E-1,-1.1245125E-1,5.1354775E-3,2.8760004E-4,1.16978735E-2,-8.3417995E-3,-2.498098E-3,5.1209807E-2,-1.6877936E-2,7.125255E-2,-2.8810976E-3,-9.97591E-2,6.8517854E-3,4.7775857E-2,1.5654758E-1,-1.6337195E-1,3.9734445E-3,9.836672E-2,-1.2608458E-2,3.5357121E-3,-3.6180114E-5,2.9950335E-3,1.0754845E-2,-2.9826458E-3,-1.0000441E-2,1.9558158E-3,-1.9010629E-3,6.1042355E-3,9.119149E-4,-4.158144E-3,3.456223E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,-1,3,5,7,9,11,-1,-1,-1,-1,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.698895E-1,0E0,2.0108666E-1,1.682863E-1,1.5310387E-1,3.9269328E-2,1.9458671E-1,0E0,0E0,0E0,0E0,1.4265096E-1,2.5636062E-1,9.8650396E-2,0E0,1.9201213E-1,1.8289237E-1,5.4367177E-2,5.6239963E-2,6.9579124E-2,2.0924417E-2,3.905487E-2,1.2562314E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,-1,4,6,8,10,12,-1,-1,-1,-1,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,-1.0263918E-2,1.0086E4,4.75E2,1.3841E4,2.3135895E6,3.6628513E10,2.8760004E-4,1.16978735E-2,-8.3417995E-3,-2.498098E-3,8.801E3,1.4793115E2,1.43259E5,-2.8810976E-3,1.252E3,9.362688E-1,2.7661E4,1.6747E4,2.0261486E8,4.2342335E-2,4.330339E2,2.776338E2,3.5357121E-3,-3.6180114E-5,2.9950335E-3,1.0754845E-2,-2.9826458E-3,-1.0000441E-2,1.9558158E-3,-1.9010629E-3,6.1042355E-3,9.119149E-4,-4.158144E-3,3.456223E-4],"split_indices":[3,0,2,2,9,44,30,0,0,0,0,10,51,1,0,0,55,28,9,7,37,4,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,7E0,2.12E2,2.02E2,1E1,1.2E1,1.9E2,5E0,5E0,5E0,7E0,6.1E1,1.29E2,5.2E1,9E0,2.8E1,1.01E2,4.2E1,1E1,1.7E1,1.1E1,1.7E1,8.4E1,2.7E1,1.5E1,5E0,5E0,6E0,1.1E1,6E0,5E0,1.2E1,5E0,1.7E1,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.3912643E-3,1.2482385E-2,-1.2885857E-2,-2.2994645E-3,1.4408407E-1,6.4485692E-3,-6.8467176E-3,1.432078E-2,2.8852832E-2,-2.9761005E-2,3.392924E-2,-1.9287659E-3,5.0148093E-3,-1.3594067E-2,-9.578921E-3,1.2756962E-1,9.427939E-3,6.475554E-2,-4.049915E-2,2.6658962E-3,1.5648478E-1,1.2046006E-1,-9.803019E-3,1.4699479E-3,5.2243117E-3,-2.4634995E-4,-3.8443764E-3,9.624417E-3,4.9641915E-3,1.04865896E-4,1.0475376E-2,-3.639463E-3,5.749806E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,-1,-1,17,-1,19,21,23,25,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.345933E-1,3.6605078E-1,0E0,2.1003702E-1,3.3037305E-1,1.6121496E-1,0E0,0E0,6.709722E-2,1.8987414E-1,2.098739E-1,0E0,0E0,1.3964272E-1,0E0,3.0806392E-2,1.5918967E-1,2.2310205E-2,6.844584E-2,0E0,9.703428E-3,1.2727079E-1,9.324394E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,13,13,15,15,16,16,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,-1,-1,18,-1,20,22,24,26,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3798E4,1.00857574E3,-1.2885857E-2,1.00068504E8,1.3770975E2,2.8374527E2,-6.8467176E-3,1.432078E-2,1.0083192E6,5.3159845E6,1.021246E0,-1.9287659E-3,5.0148093E-3,2.77E2,-9.578921E-3,9.835643E5,1.9687062E8,1.7244057E2,1.9198604E2,2.6658962E-3,9.235021E-1,1.8094341E8,1.436E3,1.4699479E-3,5.2243117E-3,-2.4634995E-4,-3.8443764E-3,9.624417E-3,4.9641915E-3,1.04865896E-4,1.0475376E-2,-3.639463E-3,5.749806E-4],"split_indices":[10,51,0,44,57,4,0,0,27,27,55,0,0,0,0,27,7,4,54,0,26,7,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.87E2,6E0,1.69E2,1.8E1,1.6E2,9E0,7E0,1.1E1,6.9E1,9.1E1,6E0,5E0,6.4E1,5E0,1.8E1,7.3E1,1.6E1,4.8E1,6E0,1.2E1,1E1,6.3E1,1E1,6E0,2.6E1,2.2E1,5E0,7E0,5E0,5E0,1.5E1,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-2.0789208E-4,6.5857256E-3,-1.1020066E-2,-2.9858344E-3,1.3601506E-1,-1.0459375E-1,4.4517424E-3,1.2444372E-2,5.1462144E-4,2.241757E-5,-1.0886867E-2,-9.2982784E-2,1.2691934E-2,2.17591E-3,-8.380063E-3,-7.56861E-3,5.1213115E-2,6.1149478E-2,-2.5648465E-2,2.512527E-2,1.295454E-1,4.439571E-3,5.226114E-4,5.3619494E-4,-3.3160201E-3,2.216708E-3,-2.9820278E-3,4.3095765E-3,8.017013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2626164E-1,2.5189075E-1,0E0,1.4405179E-1,2.1422595E-1,1.6762184E-1,1.4409262E-1,0E0,0E0,0E0,0E0,1.667046E-1,1.2916353E-1,0E0,0E0,1.3699141E-1,1.1322582E-1,3.54101E-2,1.4041404E-1,8.416517E-2,5.8957636E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,1.4056875E3,-1.1020066E-2,2.2495176E4,1E0,4.8206668E2,1.2106E4,1.2444372E-2,5.1462144E-4,2.241757E-5,-1.0886867E-2,1.8035231E0,1.631E3,2.17591E-3,-8.380063E-3,1.37E2,1.5819151E6,6.805556E5,6.769573E6,7.0208377E-1,7.7329254E2,4.439571E-3,5.226114E-4,5.3619494E-4,-3.3160201E-3,2.216708E-3,-2.9820278E-3,4.3095765E-3,8.017013E-3],"split_indices":[44,51,0,32,63,47,9,0,0,0,0,53,2,0,0,10,27,27,44,26,51,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,2.02E2,5E0,1.89E2,1.3E1,1.2E1,1.77E2,6E0,7E0,7E0,5E0,1.3E1,1.64E2,5E0,8E0,1.08E2,5.6E1,2.2E1,8.6E1,4.3E1,1.3E1,1.3E1,9E0,4.7E1,3.9E1,3.5E1,8E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.387268E-3,4.6030553E-3,-1.1797118E-2,1.276042E-2,-8.010899E-3,3.2018356E-3,1.4192265E-1,-7.401338E-3,9.638315E-3,1.0102898E-2,3.4716537E-3,4.733129E-2,-1.1162673E-2,2.228997E-2,8.279184E-3,9.022479E-2,-2.4363425E-2,4.599769E-3,4.1625256E-4,1.0714047E-3,6.9561317E-3,4.6151816E-3,-1.8687742E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,-1,13,15,17,-1,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[5.415336E-1,2.7824238E-1,0E0,2.3288654E-1,0E0,1.8114002E-1,4.6983927E-2,0E0,1.3460109E-1,0E0,0E0,1.8782389E-1,1.5075955E-1,5.1307492E-2,0E0,4.5248806E-2,1.7873728E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,11,11,12,12,13,13,15,15,16,16],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,-1,14,16,18,-1,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,4.7379535E6,-1.1797118E-2,9.248443E8,-8.010899E-3,9.235376E5,1E0,-7.401338E-3,3.9423492E10,1.0102898E-2,3.4716537E-3,3.843097E2,1.4170854E0,1.1280869E9,8.279184E-3,1.6497E4,3.4246575E-3,4.599769E-3,4.1625256E-4,1.0714047E-3,6.9561317E-3,4.6151816E-3,-1.8687742E-3],"split_indices":[44,27,0,7,0,44,80,0,30,0,0,4,52,30,0,9,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.96E2,8E0,1.88E2,8E0,1.76E2,1.2E1,6E0,1.7E2,5E0,7E0,6E1,1.1E2,5.1E1,9E0,1.2E1,9.8E1,7E0,4.4E1,6E0,6E0,1E1,8.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.7694116E-2,-9.864165E-3,-1.3022849E-2,-3.1359454E-3,-9.200282E-3,-8.599557E-3,2.5700845E-3,1.4138242E-2,-6.607715E-2,-2.258918E-3,6.1785094E-2,-1.1990333E-1,2.311579E-2,3.58232E-2,-1.9006291E-2,1.5279996E-2,1.0985145E-1,-7.192368E-3,-1.8909773E-3,3.2761593E-3,-1.2458544E-3,-2.9154273E-3,3.3090992E-3,2.3607996E-3,-1.8385185E-3,3.1253954E-3,-1.5228739E-3,3.000021E-3,7.225577E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,-1,5,-1,-1,7,9,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9927018E-1,2.4086349E-1,0E0,1.9267225E-1,0E0,0E0,1.4864092E-1,1.2514326E-1,1.323964E-1,7.721392E-2,8.997001E-2,3.540173E-2,2.6237965E-2,1.2313458E-1,1.1299104E-1,5.3803127E-2,2.9256493E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,-1,6,-1,-1,8,10,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,4.476758E6,-1.3022849E-2,1.0058E4,-9.200282E-3,-8.599557E-3,3.10364E5,7.5614764E2,6.069098E5,1E0,5E0,1.4435129E6,2.1704407E11,8.960506E3,2.0304577E-1,1.8933852E0,7.7186523E2,-7.192368E-3,-1.8909773E-3,3.2761593E-3,-1.2458544E-3,-2.9154273E-3,3.3090992E-3,2.3607996E-3,-1.8385185E-3,3.1253954E-3,-1.5228739E-3,3.000021E-3,7.225577E-3],"split_indices":[44,27,0,9,0,0,9,4,27,111,8,31,30,32,26,53,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.96E2,5E0,1.9E2,6E0,5E0,1.85E2,1.59E2,2.6E1,1.19E2,4E1,1.6E1,1E1,3.6E1,8.3E1,2.1E1,1.9E1,1.1E1,5E0,5E0,5E0,9E0,2.7E1,1.8E1,6.5E1,1E1,1.1E1,1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.749581E-4,-1.2903669E-2,1.6795477E-1,-2.0537914E-4,-1.752457E-1,7.171186E-4,1.345378E-2,9.025887E-3,-9.572364E-3,-1.3354766E-2,-1.5288711E-3,1.9184079E-2,-5.8057286E-2,3.1810097E-2,-3.2801203E-2,-1.2144796E-1,1.0171495E-2,-1.4284042E-2,4.9781516E-2,3.7225805E-2,-9.640145E-2,-7.645547E-3,-2.7169352E-3,2.1551186E-3,-1.5397061E-3,2.0880892E-3,-2.4174824E-3,4.0970114E-3,6.579349E-4,-1.5400983E-6,3.8236082E-3,-1.3475748E-3,-7.7208444E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2696482E-1,3.97827E-1,2.3753917E-1,3.3346507E-1,1.9881603E-1,0E0,0E0,1.1900355E-1,0E0,0E0,0E0,1.0015486E-1,1.0071187E-1,1.017731E-1,1.3720067E-1,2.057764E-2,1.9120872E-2,7.5472E-2,1.1218977E-1,2.3802074E-2,6.6205904E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4056875E3,1.00068504E8,1.376E4,4.674595E6,6.0268195E2,7.171186E-4,1.345378E-2,3.10364E5,-9.572364E-3,-1.3354766E-2,-1.5288711E-3,9.235021E-1,1.569E3,8.92E2,2.493493E7,4.7861097E5,3.328384E5,1.5139E4,1.8352579E0,1.024E3,9.4636065E-1,-7.645547E-3,-2.7169352E-3,2.1551186E-3,-1.5397061E-3,2.0880892E-3,-2.4174824E-3,4.0970114E-3,6.579349E-4,-1.5400983E-6,3.8236082E-3,-1.3475748E-3,-7.7208444E-3],"split_indices":[51,44,9,27,51,0,0,9,0,0,0,26,0,2,44,27,32,9,55,2,26,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.92E2,1.3E1,1.79E2,1.3E1,6E0,7E0,1.72E2,7E0,7E0,6E0,1.5E2,2.2E1,1.21E2,2.9E1,1.1E1,1.1E1,3.4E1,8.7E1,1.4E1,1.5E1,6E0,5E0,6E0,5E0,1.3E1,2.1E1,4.3E1,4.4E1,8E0,6E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.8531388E-3,1.2359911E-1,-6.852457E-3,1.7630536E-3,1.1172085E-2,1.1536126E-2,-7.254988E-2,-8.177492E-3,5.7998043E-2,-1.0339157E-1,3.6078074E-3,7.435079E-2,-2.2137865E-2,9.937891E-3,3.0396534E-2,-2.4012681E-2,-1.8143643E-1,-9.1053837E-4,6.336613E-3,-6.334781E-2,2.2240942E-2,4.3656616E-3,5.7285605E-3,-6.363419E-3,2.4336314E-2,-2.4333574E-1,-2.3668564E-3,8.905296E-3,-8.702193E-2,5.7012364E-2,-8.102912E-2,4.8900854E-2,-2.1290908E-2,-4.552355E-4,3.4033153E-3,-6.398044E-3,-1.5851235E-2,3.8093245E-3,-2.1637066E-3,-1.9615754E-3,-6.390239E-3,5.7167215E-3,1.9165799E-3,-5.9253643E-3,-8.5101504E-4,6.113963E-4,3.972221E-3,2.8642567E-4,-3.1787762E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,-1,-1,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,-1,27,29,31,-1,-1,33,35,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2236003E-1,1.2459944E-1,2.3762111E-1,0E0,0E0,1.4152789E-1,2.0387211E-1,1.2722766E-1,1.8343085E-1,2.1866739E-1,0E0,8.902116E-2,1.7332064E-1,0E0,9.2332E-2,1.0319072E-1,1.3508731E-1,0E0,0E0,8.459088E-2,1.6979861E-1,3.760499E-2,0E0,0E0,2.279948E-2,7.113087E-2,0E0,5.3220633E-2,7.358116E-2,3.2318838E-2,3.018403E-2,1.3691863E-2,2.6200928E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,19,19,20,20,21,21,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32],"right_children":[2,4,6,-1,-1,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,-1,28,30,32,-1,-1,34,36,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.390852E-2,4.8444237E8,8.50999E9,1.7630536E-3,1.1172085E-2,1.996E3,2.783671E12,1.3252E4,2.3068698E8,6.792816E7,3.6078074E-3,1.9599061E5,1.7206225E0,9.937891E-3,2.1936802E3,1.5023475E-2,1E0,-9.1053837E-4,6.336613E-3,4.45E2,2.1845032E7,2.46604E5,5.7285605E-3,-6.363419E-3,2.3E1,1E0,-2.3668564E-3,1.7829868E2,1.6844329E0,8.9281055E4,5.233944E-5,2.539E3,8.091679E-1,-4.552355E-4,3.4033153E-3,-6.398044E-3,-1.5851235E-2,3.8093245E-3,-2.1637066E-3,-1.9615754E-3,-6.390239E-3,5.7167215E-3,1.9165799E-3,-5.9253643E-3,-8.5101504E-4,6.113963E-4,3.972221E-3,2.8642567E-4,-3.1787762E-3],"split_indices":[26,7,5,0,0,2,30,9,7,44,0,31,52,0,4,56,15,0,0,10,44,28,0,0,8,108,0,4,53,32,37,2,26,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.3E1,1.95E2,8E0,5E0,1.53E2,4.2E1,1.08E2,4.5E1,3.5E1,7E0,1.5E1,9.3E1,6E0,3.9E1,1.8E1,1.7E1,6E0,9E0,4.8E1,4.5E1,3.1E1,8E0,5E0,1.3E1,1.1E1,6E0,1.2E1,3.6E1,3.4E1,1.1E1,1.1E1,2E1,8E0,5E0,6E0,5E0,5E0,7E0,1.9E1,1.7E1,6E0,2.8E1,6E0,5E0,6E0,5E0,1.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[9.312813E-3,2.0384718E-2,-1.00693405E-1,4.210924E-2,-8.657957E-3,-2.2129597E-1,3.7235224E-3,-4.8295617E-2,5.7040647E-2,-5.578887E-3,4.4489536E-4,-4.60687E-3,-1.4829988E-2,-4.066287E-3,2.7002086E-4,-5.723057E-2,7.242452E-2,4.271354E-3,-1.2037168E-2,6.5350096E-4,-5.680173E-3,1.464994E-1,4.6086866E-2,8.071656E-3,-6.0088154E-2,9.2303686E-2,1.0360417E-2,8.00706E-3,1.0976622E-1,-4.6853465E-3,2.3367928E-2,3.6239537E-4,-9.189426E-2,1.5637908E-3,6.64871E-3,1.4179124E-3,-3.10671E-3,7.700402E-3,2.1169956E-3,3.0918987E-3,-1.1060754E-3,-2.1713246E-3,-6.643653E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,-1,-1,19,21,-1,23,-1,-1,25,27,29,31,33,-1,35,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3955838E-1,1.1267159E-1,4.045115E-1,1.4044172E-1,7.560481E-2,9.1713846E-2,0E0,3.0071411E-2,1.5884316E-1,0E0,8.079935E-2,0E0,0E0,0E0,0E0,4.943132E-2,1.4542308E-1,0E0,6.2324934E-2,0E0,0E0,5.564791E-2,1.4219007E-1,7.677835E-2,4.1578606E-2,3.151743E-2,0E0,6.16882E-2,6.642634E-2,0E0,7.990484E-2,0E0,2.2037104E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,15,15,16,16,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,30,30,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,-1,-1,20,22,-1,24,-1,-1,26,28,30,32,34,-1,36,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.472477E7,4.9538516E5,4.8314605E0,1.2904155E3,4.910112E4,4E0,3.7235224E-3,9.123214E-1,5.94E2,-5.578887E-3,5.732505E6,-4.60687E-3,-1.4829988E-2,-4.066287E-3,2.7002086E-4,1.5605E4,1.151349E7,4.271354E-3,3.73802E5,6.5350096E-4,-5.680173E-3,4.0480963E2,4.3935942E8,9.06E2,1.6810659E6,1.5064E4,1.0360417E-2,2.136E4,6.234292E-1,-4.6853465E-3,4.530014E8,3.6239537E-4,4.5360103E5,1.5637908E-3,6.64871E-3,1.4179124E-3,-3.10671E-3,7.700402E-3,2.1169956E-3,3.0918987E-3,-1.1060754E-3,-2.1713246E-3,-6.643653E-3],"split_indices":[44,31,55,47,47,8,0,26,2,0,12,0,0,0,0,1,12,0,32,0,0,4,7,2,49,9,0,9,26,0,7,0,32,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.94E2,1.77E2,1.7E1,1.01E2,7.6E1,1E1,7E0,1.4E1,8.7E1,5E0,7.1E1,5E0,5E0,8E0,6E0,1E1,7.7E1,8E0,6.3E1,5E0,5E0,1.9E1,5.8E1,4.5E1,1.8E1,1.2E1,7E0,3.7E1,2.1E1,5E0,4E1,6E0,1.2E1,6E0,6E0,2.9E1,8E0,1.1E1,1E1,2.1E1,1.9E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[6.8093715E-3,1.4430599E-2,-8.796921E-3,1.1017902E-3,1.5706244E-1,3.1792246E-2,-3.096493E-2,1.4386229E-2,5.164142E-2,7.031274E-2,-6.7262264E-4,-5.159023E-2,6.0637258E-2,-1.4836404E-4,4.68413E-3,4.551819E-2,7.070452E-3,4.0215356E-3,-2.0334868E-2,2.3187757E-2,-9.021548E-2,6.7608804E-3,3.907748E-3,8.123324E-2,-3.6543574E-2,-4.7605936E-3,-2.2069146E-3,-3.0156644E-3,5.2175377E-2,-9.720595E-3,-6.977348E-2,1.9852205E-3,-1.6419952E-3,2.2047295E-3,8.585354E-3,-4.591934E-3,1.3822409E-3,-3.860378E-3,6.243871E-4,-2.2294065E-4,3.682378E-3,-5.7999506E-3,-4.923137E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,19,21,-1,-1,23,-1,-1,25,27,29,-1,31,33,35,-1,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0161846E-1,3.7814838E-1,0E0,1.8108249E-1,2.4117297E-1,1.1760017E-1,1.7316732E-1,0E0,2.8079426E-2,7.56651E-2,8.822035E-2,2.1582748E-1,7.706735E-2,0E0,0E0,1.02617115E-1,0E0,0E0,6.1387807E-2,6.82018E-2,9.87142E-2,0E0,1.718273E-2,7.4550316E-2,4.5426212E-2,0E0,4.4047542E-2,0E0,2.8417062E-2,0E0,1.2997551E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,15,15,18,18,19,19,20,20,22,22,23,23,24,24,26,26,28,28,30,30],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,20,22,-1,-1,24,-1,-1,26,28,30,-1,32,34,36,-1,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6030095E10,1.3397336E3,-8.796921E-3,6.781768E-1,6.234292E-1,1.933162E0,2.225675E0,1.4386229E-2,4E0,5.49E2,1.2414E4,1.453034E0,2.6486957E0,-1.4836404E-4,4.68413E-3,1.8556E5,7.070452E-3,4.0215356E-3,7.308952E1,1E0,1.2106E4,6.7608804E-3,1.01E3,1.33807E5,2.34958E5,-4.7605936E-3,2.7573213E8,-3.0156644E-3,7.794276E-1,-9.720595E-3,8E0,1.9852205E-3,-1.6419952E-3,2.2047295E-3,8.585354E-3,-4.591934E-3,1.3822409E-3,-3.860378E-3,6.243871E-4,-2.2294065E-4,3.682378E-3,-5.7999506E-3,-4.923137E-4],"split_indices":[5,51,0,26,26,57,53,0,8,0,9,52,53,0,0,1,0,0,51,107,9,0,2,1,1,0,7,0,26,0,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.98E2,7E0,1.82E2,1.6E1,9.3E1,8.9E1,6E0,1E1,4.2E1,5.1E1,7.3E1,1.6E1,5E0,5E0,3.3E1,9E0,9E0,4.2E1,2.5E1,4.8E1,6E0,1E1,2.3E1,1E1,7E0,3.5E1,6E0,1.9E1,6E0,4.2E1,5E0,5E0,1.8E1,5E0,5E0,5E0,5E0,3E1,6E0,1.3E1,2.2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[7.7712657E-3,1.4626645E-2,-8.783076E-3,2.0215094E-2,-7.84965E-3,1.4291962E-2,8.24861E-3,1.9547911E-2,-5.7801497E-3,-7.612793E-2,2.6967537E-2,-1.3829741E-3,-4.8894086E-3,3.528419E-2,-3.8796455E-2,5.525904E-4,3.2854828E-3,-3.721949E-3,-4.6160037E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,-1,7,-1,9,-1,11,13,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[2.6469702E-1,1.9680634E-1,0E0,1.6765395E-1,0E0,1.3215105E-1,0E0,1.2770154E-1,0E0,1.2984835E-2,9.1613404E-2,0E0,0E0,1.1544715E-1,2.0800486E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,9,9,10,10,13,13,14,14],"right_children":[2,4,-1,6,-1,8,-1,10,-1,12,14,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[4.7379535E6,1.0953058E10,-8.783076E-3,4.056359E6,-7.84965E-3,6.3086E4,8.24861E-3,8.960506E3,-5.7801497E-3,8.601996E-1,5.327338E6,-1.3829741E-3,-4.8894086E-3,3.7597818E2,1.5947007E0,5.525904E-4,3.2854828E-3,-3.721949E-3,-4.6160037E-4],"split_indices":[27,12,0,28,0,10,0,32,0,26,46,0,0,54,52,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.93E2,6E0,1.88E2,5E0,1.82E2,6E0,1.76E2,6E0,1.2E1,1.64E2,5E0,7E0,1.46E2,1.8E1,8.6E1,6E1,7E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-5.510306E-3,5.0003547E-3,-1.1848575E-1,-5.0059482E-3,9.7427145E-2,1.6208075E-3,-1.2878792E-2,1.2181632E-2,-4.7371227E-2,2.1482715E-2,1.0757752E-2,4.1712616E-3,-4.028903E-3,-4.9234265E-3,6.475918E-2,-1.0385798E-2,-1.2762773E-1,-1.0091404E-3,3.575456E-3,-3.2539263E-2,2.0081514E-2,1.03368156E-1,3.3491964E-3,-4.701882E-2,4.2907633E-3,-9.834811E-3,-7.3667295E-2,-1.1070125E-2,-5.7284604E-3,1.0911348E-1,-2.0046068E-2,2.3026217E-3,1.316242E-1,1.8622361E-3,-2.13533E-3,-9.007259E-2,2.3793258E-2,-1.5322553E-3,-4.938087E-3,1.8956792E-3,-2.0245223E-3,3.3456804E-3,6.6909995E-3,1.8176496E-3,-3.2736745E-3,7.6239016E-3,3.717241E-3,-6.6060983E-3,-9.1223663E-4,-8.047626E-4,2.8945708E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,-1,19,21,23,25,-1,-1,27,29,31,33,35,-1,-1,37,39,-1,41,43,-1,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.511437E-1,1.7990871E-1,3.2544938E-1,1.2873262E-1,1.786053E-1,8.7888554E-2,0E0,1.1363593E-1,1.5005887E-1,3.096883E-2,0E0,0E0,0E0,6.694455E-2,7.315409E-2,1.3602105E-1,5.1923066E-2,0E0,0E0,8.4627435E-2,1.8439181E-1,2.336508E-2,2.386123E-2,8.445874E-2,0E0,0E0,1.0186981E-2,6.1686862E-2,0E0,9.445384E-3,1.033897E-1,0E0,5.490765E-3,0E0,0E0,5.5115893E-2,1.7368406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,26,26,27,27,29,29,30,30,32,32,35,35,36,36],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,-1,20,22,24,26,-1,-1,28,30,32,34,36,-1,-1,38,40,-1,42,44,-1,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,5.666E3,1.0129378E12,8.5913794E5,2.605017E5,9.846896E4,-1.2878792E-2,2.225675E0,8.19E2,1.3841E4,1.0757752E-2,4.1712616E-3,-4.028903E-3,2.8003845E2,5.9351373E0,7.32E2,5E0,-1.0091404E-3,3.575456E-3,2.010204E0,1.28831E5,4E0,3.6907338E-2,7.830236E-1,4.2907633E-3,-9.834811E-3,6.894317E-1,7.3328E4,-5.7284604E-3,1.117E3,5.3707652E7,2.3026217E-3,5.606407E5,1.8622361E-3,-2.13533E-3,3.0050538E5,9.8340225E1,-1.5322553E-3,-4.938087E-3,1.8956792E-3,-2.0245223E-3,3.3456804E-3,6.6909995E-3,1.8176496E-3,-3.2736745E-3,7.6239016E-3,3.717241E-3,-6.6060983E-3,-9.1223663E-4,-8.047626E-4,2.8945708E-3],"split_indices":[44,2,30,31,32,32,0,53,0,9,0,0,0,4,56,10,8,0,0,53,28,8,37,26,0,0,26,1,0,2,12,0,27,0,0,32,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.93E2,1.7E1,1.75E2,1.8E1,1E1,7E0,1.25E2,5E1,1.2E1,6E0,5E0,5E0,9.5E1,3E1,3.5E1,1.5E1,7E0,5E0,4.5E1,5E1,1.8E1,1.2E1,2.6E1,9E0,5E0,1E1,3.7E1,8E0,1.5E1,3.5E1,7E0,1.1E1,7E0,5E0,1.6E1,1E1,5E0,5E0,1.4E1,2.3E1,8E0,7E0,1.6E1,1.9E1,6E0,5E0,9E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-1.1704756E-3,1.423016E-2,-7.418005E-2,3.464207E-3,1.4067067E-1,-2.1076834E-2,-1.2013915E-2,-7.474051E-2,1.3713993E-2,1.3681324E-2,6.9036474E-4,2.6308002E-3,-5.9962284E-2,-1.4676923E-2,-7.0897327E-3,2.129113E-2,-4.6191015E-3,-1.0574755E-1,1.274341E-4,-2.3660313E-3,1.0769286E-3,-1.025916E-3,8.3677754E-2,-6.6958177E-3,-2.5921953E-3,-1.2871689E-2,6.927588E-2,1.0762742E-1,-1.1533889E-3,8.907732E-4,-2.6094443E-3,6.658708E-3,-1.8308598E-5,1.5578158E-3,7.3930137E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,17,19,-1,21,-1,23,-1,-1,-1,25,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2593437E-1,2.2548062E-1,3.229246E-1,1.2446856E-1,2.337256E-1,8.603012E-2,0E0,7.701631E-2,1.16374545E-1,0E0,0E0,0E0,5.4757625E-2,1.5296519E-2,0E0,1.7969061E-1,0E0,1.1760026E-2,0E0,0E0,0E0,8.0838494E-2,9.033291E-2,0E0,0E0,1.09643884E-1,6.803774E-2,9.31727E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,12,12,13,13,15,15,17,17,21,21,22,22,25,25,26,26,27,27],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,18,20,-1,22,-1,24,-1,-1,-1,26,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.10364E5,1.4056875E3,5.8018835E2,5.643341E-2,3.6007138E2,6.5229E4,-1.2013915E-2,2.1600779E8,1.6565532E3,1.3681324E-2,6.9036474E-4,2.6308002E-3,2.3361E4,5.4166875E3,-7.0897327E-3,2.001E3,-4.6191015E-3,4.7647306E5,1.274341E-4,-2.3660313E-3,1.0769286E-3,7.9201184E8,6.1037578E-2,-6.6958177E-3,-2.5921953E-3,5.14E2,1.2226295E6,4.2378342E-1,-1.1533889E-3,8.907732E-4,-2.6094443E-3,6.658708E-3,-1.8308598E-5,1.5578158E-3,7.3930137E-3],"split_indices":[9,51,51,57,57,28,0,7,4,0,0,0,10,32,0,2,0,31,0,0,0,12,37,0,0,0,46,26,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.65E2,3.4E1,1.53E2,1.2E1,2.7E1,7E0,1.7E1,1.36E2,5E0,7E0,9E0,1.8E1,1E1,7E0,1.28E2,8E0,1E1,8E0,5E0,5E0,9.5E1,3.3E1,5E0,5E0,8.2E1,1.3E1,2.7E1,6E0,4.7E1,3.5E1,6E0,7E0,1.1E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.06563E-3,1.08985186E-1,-2.0713164E-3,6.860201E-4,1.0398121E-2,4.7928747E-3,-7.609318E-3,-2.401882E-4,7.3447437E-3,5.3998614E-3,-5.114108E-3,-1.753003E-2,3.2034673E-2,-3.0969437E-2,7.46468E-2,7.3104566E-3,2.1826807E-2,1.3890924E-3,-2.866995E-3,2.3294083E-4,5.930972E-3,2.7797022E-3,-3.416525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,-1,-1,7,-1,9,-1,11,-1,13,15,17,19,-1,21,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5004274E-1,1.2987271E-1,2.0520635E-1,0E0,0E0,1.3682699E-1,0E0,1.0750245E-1,0E0,1.0438623E-1,0E0,1.1628154E-1,9.3889885E-2,1.4197868E-1,4.0558286E-2,0E0,7.797508E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,9,9,11,11,12,12,13,13,14,14,16,16],"right_children":[2,4,6,-1,-1,8,-1,10,-1,12,-1,14,16,18,20,-1,22,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6604617E-2,5.7669523E8,7.5875E4,6.860201E-4,1.0398121E-2,1.1195705E7,-7.609318E-3,4.674595E6,7.3447437E-3,1.7311496E0,-5.114108E-3,1.5819151E6,1.9557953E0,1.3614458E0,6.8915665E-1,7.3104566E-3,1.3549481E2,1.3890924E-3,-2.866995E-3,2.3294083E-4,5.930972E-3,2.7797022E-3,-3.416525E-4],"split_indices":[26,7,10,0,0,1,0,27,0,52,0,27,53,52,55,0,51,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.2E1,1.89E2,7E0,5E0,1.82E2,7E0,1.77E2,5E0,1.69E2,8E0,9.1E1,7.8E1,8E1,1.1E1,5E0,7.3E1,2.6E1,5.4E1,5E0,6E0,3.2E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-7.1378117E-3,-5.48326E-4,-1.1386778E-2,4.746688E-3,-6.491706E-3,-2.0689298E-3,7.984019E-2,-1.803695E-2,2.9150937E-2,4.3914295E-4,6.785877E-3,2.5732996E-2,-4.0495634E-2,5.5968378E-2,-1.7876811E-2,1.0987985E-2,5.5012773E-3,-8.769598E-2,-1.7427126E-2,7.829422E-2,1.2050634E-2,-5.6626904E-3,3.0384941E-2,4.73473E-3,-2.546839E-4,3.833993E-4,-5.529559E-3,1.0059337E-3,-3.4079712E-3,1.9049768E-4,4.5514796E-3,-6.503391E-4,2.527213E-3,3.9520497E-3,-2.675112E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,17,19,21,23,-1,25,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.280048E-1,1.5142064E-1,0E0,1.04574144E-1,0E0,9.424745E-2,7.295491E-2,1.23918995E-1,8.171502E-2,0E0,0E0,5.5228427E-2,8.8845566E-2,3.9257035E-2,1.19792186E-1,5.4947596E-2,0E0,7.25577E-2,1.19123966E-1,3.315164E-2,1.636046E-2,0E0,8.0984175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,18,20,22,24,-1,26,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,4.7379535E6,-1.1386778E-2,8.853438E8,-6.491706E-3,2.173763E0,1.3841E4,5.2139695E6,8.3268556E5,4.3914295E-4,6.785877E-3,1.9234526E1,1.4693E4,5.964497E0,2.9307398E-1,4.0748124E7,5.5012773E-3,7.492958E-1,1.7476E4,9.090909E-2,2.5831E4,-5.6626904E-3,6.5229E4,4.73473E-3,-2.546839E-4,3.833993E-4,-5.529559E-3,1.0059337E-3,-3.4079712E-3,1.9049768E-4,4.5514796E-3,-6.503391E-4,2.527213E-3,3.9520497E-3,-2.675112E-3],"split_indices":[44,27,0,7,0,53,9,44,31,0,0,57,9,56,26,5,0,55,9,57,10,0,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.1E2,5E0,2.03E2,7E0,1.87E2,1.6E1,1.24E2,6.3E1,8E0,8E0,4.2E1,8.2E1,4E1,2.3E1,3.7E1,5E0,2.6E1,5.6E1,2.6E1,1.4E1,7E0,1.6E1,5E0,3.2E1,6E0,2E1,3.3E1,2.3E1,5E0,2.1E1,9E0,5E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.3128191E-3,-6.524628E-3,7.5661866E-3,1.8436778E-2,-4.6637483E-2,6.9907054E-2,-6.0014296E-3,-3.093164E-2,-9.547033E-3,3.8563076E-2,9.124745E-3,-6.3883595E-2,2.0588739E-2,-6.1520715E-3,-1.65147E-2,5.6605645E-2,-1.7505082E-3,-9.986308E-2,-1.7454684E-2,-7.609727E-3,6.835933E-2,-3.8818546E-2,4.1401498E-2,3.7190385E-2,5.670462E-3,-5.752283E-3,-2.3735466E-3,1.0435028E-3,-2.1414924E-3,3.732331E-3,-2.2927383E-2,5.953668E-3,3.9625045E-2,1.0686785E-2,-7.73106E-2,-1.8302911E-3,8.438793E-2,-2.0623507E-3,3.033656E-3,-5.1741805E-3,4.1280012E-4,-6.118685E-7,5.0634416E-3,2.5249694E-3,-1.9547925E-3,-5.023662E-3,1.532921E-3,1.6936888E-3,5.478927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,7,9,11,13,-1,15,-1,17,19,-1,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5757432E-1,2.011145E-1,0E0,1.5639186E-1,1.7843074E-1,1.4380273E-1,1.321385E-1,9.766185E-2,0E0,4.6784256E-2,0E0,4.3360107E-2,7.999657E-2,0E0,8.2987644E-2,2.7970836E-2,0E0,9.302333E-3,1.4870444E-2,5.1562198E-2,3.042119E-2,8.8482045E-2,6.481919E-2,4.8854634E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.948839E-2,0E0,4.1900203E-2,4.7583066E-2,8.165015E-2,0E0,1.3594806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,30,30,32,32,33,33,34,34,36,36],"right_children":[2,4,-1,6,8,10,12,14,-1,16,-1,18,20,-1,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,8.111963E-1,7.5661866E-3,3.74E2,2.68225E5,1.594E3,1E0,1.1334E4,-9.547033E-3,1.8267011E5,9.124745E-3,1.4065148E6,5.247412E5,-6.1520715E-3,2.192884E0,6.1895325E5,-1.7505082E-3,1.18107956E2,7.476915E5,5.7573294E5,8.8317425E5,1E0,1.0860942E6,1.763677E8,5.670462E-3,-5.752283E-3,-2.3735466E-3,1.0435028E-3,-2.1414924E-3,3.732331E-3,1.2086619E6,5.953668E-3,1.13728695E1,2E0,1.875E3,-1.8302911E-3,1.4768839E5,-2.0623507E-3,3.033656E-3,-5.1741805E-3,4.1280012E-4,-6.118685E-7,5.0634416E-3,2.5249694E-3,-1.9547925E-3,-5.023662E-3,1.532921E-3,1.6936888E-3,5.478927E-3],"split_indices":[2,26,0,0,9,2,8,9,0,46,0,49,27,0,53,27,0,51,46,50,27,111,27,7,0,0,0,0,0,0,47,0,56,8,2,0,31,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.99E2,9E0,1.23E2,7.6E1,3.9E1,8.4E1,7E1,6E0,3.2E1,7E0,2.6E1,5.8E1,8E0,6.2E1,2.6E1,6E0,1.4E1,1.2E1,3.7E1,2.1E1,4.5E1,1.7E1,2.1E1,5E0,9E0,5E0,5E0,7E0,5E0,3.2E1,6E0,1.5E1,2E1,2.5E1,6E0,1.1E1,5E0,1.6E1,8E0,2.4E1,1E1,5E0,1.1E1,9E0,2E1,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.1399253E-3,-1.0319589E-2,1.1769322E-1,-9.930938E-2,-1.8346643E-3,1.2467294E-2,2.9245282E-2,-2.3859425E-4,-9.2503885E-3,-1.0813295E-1,4.6715345E-3,-1.5202757E-3,4.0889457E-3,-2.3818857E-3,-7.115642E-3,-6.5294735E-2,1.3825411E-2,-9.5653646E-2,1.2717176E-3,8.832767E-2,4.3508234E-3,-1.9361881E-3,-7.7085476E-3,1.2049634E-1,9.851455E-4,-4.324805E-2,1.7911656E-2,1.8467655E-3,9.390998E-3,1.6584299E-3,-3.6037588E-3,1.2947698E-3,-2.8885487E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,-1,-1,-1,17,19,21,-1,23,25,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4338289E-1,1.5611765E-1,1.9410592E-1,1.582626E-1,1.3190873E-1,0E0,4.0340826E-2,0E0,0E0,1.8571094E-2,1.1618599E-1,0E0,0E0,0E0,0E0,6.1090223E-2,1.125562E-1,5.116552E-2,0E0,3.676033E-2,9.307833E-2,0E0,0E0,6.671253E-2,0E0,8.2272455E-2,8.146141E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,15,15,16,16,17,17,19,19,20,20,23,23,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,-1,-1,-1,18,20,22,-1,24,26,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3397336E3,7E0,1.3770975E2,3.10364E5,7.999278E1,1.2467294E-2,7.304467E7,-2.3859425E-4,-9.2503885E-3,1.4596E4,8.186529E-2,-1.5202757E-3,4.0889457E-3,-2.3818857E-3,-7.115642E-3,1.4E1,1.2345679E-2,2.0713173E8,1.2717176E-3,9.0763354E-1,2.3184108E5,-1.9361881E-3,-7.7085476E-3,4.5701938E5,9.851455E-4,3.08E2,3.6777365E6,1.8467655E-3,9.390998E-3,1.6584299E-3,-3.6037588E-3,1.2947698E-3,-2.8885487E-3],"split_indices":[51,3,57,9,54,0,47,0,0,9,57,0,0,0,0,8,56,7,0,26,32,0,0,27,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.21E2,2.06E2,1.5E1,1.7E1,1.89E2,5E0,1E1,9E0,8E0,1E1,1.79E2,5E0,5E0,5E0,5E0,2E1,1.59E2,1.5E1,5E0,1.7E1,1.42E2,9E0,6E0,1.1E1,6E0,3.1E1,1.11E2,6E0,5E0,9E0,2.2E1,1E2,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.6466057E-3,2.6532562E-2,-5.2822184E-2,1.8290184E-2,7.134139E-3,-9.416979E-3,-1.4906937E-1,3.1907268E-2,-2.2711808E-2,-2.3482192E-2,3.8488703E-3,-2.5267503E-1,-2.2493258E-2,4.329997E-3,8.108087E-2,2.6188273E-2,-6.452295E-2,-1.0754311E-2,-4.901009E-3,-1.8350847E-2,-2.801159E-3,-4.9708397E-3,2.995213E-3,-1.4986211E-2,6.886798E-2,9.780521E-3,4.6170875E-2,-9.432969E-4,2.447933E-3,-1.0898385E-1,1.6841067E-3,4.725242E-3,-2.9741282E-2,2.839944E-3,-3.1740006E-2,9.657583E-4,4.9288245E-3,6.7162156E-2,-1.844102E-3,-2.725091E-3,-7.263197E-3,-4.4702613E-3,-6.593305E-3,-2.9105484E-3,1.8492938E-3,5.987247E-4,4.6713767E-3,2.226526E-3,-1.3697336E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,-1,37,-1,-1,39,-1,-1,41,-1,43,-1,-1,45,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8837648E-1,1.2739864E-1,2.9520786E-1,6.884062E-2,0E0,6.500782E-2,2.828245E-1,1.24407694E-1,6.507135E-2,4.303976E-2,0E0,2.8141755E-1,8.2480304E-2,7.582982E-2,1.3235432E-1,1.8146899E-2,7.916945E-2,8.3932266E-2,0E0,0E0,0E0,0E0,0E0,6.0359813E-2,2.1120228E-2,0E0,5.0569315E-2,0E0,0E0,1.802516E-2,0E0,0E0,4.9414266E-2,0E0,8.19345E-2,0E0,0E0,3.5274424E-2,0E0,0E0,0E0,0E0,3.1680077E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,23,23,24,24,26,26,29,29,32,32,34,34,37,37,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,-1,38,-1,-1,40,-1,-1,42,-1,44,-1,-1,46,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1],"split_conditions":[8.601996E-1,2.1374558E3,7.625922E11,4.7564573E0,7.134139E-3,7.7247575E6,1.880597E0,1.281E3,2.4204762E-1,3.384024E9,3.8488703E-3,1E0,1.5566E4,1E0,1.1875076E-2,2.6119404E-2,2.014E3,1.3155E4,-4.901009E-3,-1.8350847E-2,-2.801159E-3,-4.9708397E-3,2.995213E-3,8.9281055E4,3.98E2,9.780521E-3,6.872321E-1,-9.432969E-4,2.447933E-3,8.5913794E5,1.6841067E-3,4.725242E-3,6.2271062E-2,2.839944E-3,4.4010544E5,9.657583E-4,4.9288245E-3,4.066361E5,-1.844102E-3,-2.725091E-3,-7.263197E-3,-4.4702613E-3,1.0326397E6,-2.9105484E-3,1.8492938E-3,5.987247E-4,4.6713767E-3,2.226526E-3,-1.3697336E-3],"split_indices":[26,4,30,55,0,31,52,2,26,12,0,15,9,15,37,56,0,9,0,0,0,0,0,32,0,0,26,0,0,31,0,0,56,0,32,0,0,31,0,0,0,0,47,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.28E2,7E1,1.21E2,7E0,4.9E1,2.1E1,9.1E1,3E1,4.3E1,6E0,1.1E1,1E1,5.9E1,3.2E1,1.4E1,1.6E1,3.8E1,5E0,6E0,5E0,5E0,5E0,4.6E1,1.3E1,6E0,2.6E1,5E0,9E0,1.1E1,5E0,5E0,3.3E1,8E0,3.8E1,6E0,7E0,2.1E1,5E0,6E0,5E0,8E0,2.5E1,2.7E1,1.1E1,8E0,1.3E1,7E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-5.3319368E-3,1.9454253E-3,-9.157576E-3,-5.625844E-3,1.1420001E-1,-1.2629728E-2,8.482593E-2,2.1487193E-3,8.989161E-3,7.5695324E-3,-3.7411164E-2,7.0219827E-3,-1.0506121E-3,2.3961361E-2,-4.2564407E-2,-7.4835666E-2,-4.2397473E-3,-9.30783E-3,7.5284384E-2,-9.6776895E-4,-9.541761E-2,-4.784752E-2,-8.983735E-3,4.827979E-3,-2.0287205E-2,5.2395364E-4,-2.7694847E-3,-4.41446E-4,4.637192E-3,-1.9841243E-3,2.451665E-3,-1.759204E-3,-6.6215103E-3,-3.968716E-3,4.6314477E-4,-1.6684765E-3,3.1371799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8704882E-1,1.7487396E-1,0E0,1.2333948E-1,5.7787046E-2,9.038518E-2,9.549738E-2,0E0,0E0,8.319861E-2,1.0023805E-1,0E0,0E0,1.3046888E-1,5.487055E-2,1.0659027E-1,7.619435E-2,4.6779525E-2,5.541545E-2,3.375428E-2,2.2555992E-2,6.520321E-2,0E0,0E0,5.0914794E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.674595E6,1.3904022E3,-9.157576E-3,6.204429E6,1.376E4,1.8352579E0,1.25077E5,2.1487193E-3,8.989161E-3,8.869983E5,6.275614E2,7.0219827E-3,-1.0506121E-3,2.5930756E2,2E0,5.0229483E2,1.65E9,6.791045E-1,3.6482175E5,3.7186194E5,3.3396155E-1,3.936351E2,-8.983735E-3,4.827979E-3,3.906E3,5.2395364E-4,-2.7694847E-3,-4.41446E-4,4.637192E-3,-1.9841243E-3,2.451665E-3,-1.759204E-3,-6.6215103E-3,-3.968716E-3,4.6314477E-4,-1.6684765E-3,3.1371799E-3],"split_indices":[27,51,0,31,9,55,28,0,0,31,4,0,0,4,8,54,5,55,27,46,38,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.04E2,7E0,1.92E2,1.2E1,1.79E2,1.3E1,7E0,5E0,9.9E1,8E1,8E0,5E0,7.5E1,2.4E1,3.7E1,4.3E1,4.6E1,2.9E1,1.4E1,1E1,3.1E1,6E0,5E0,3.8E1,3.3E1,1.3E1,6E0,2.3E1,8E0,6E0,5E0,5E0,1.9E1,1.2E1,3.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[9.215593E-3,1.5134277E-2,-7.6826103E-3,-1.0834546E-1,2.2240853E-2,-6.600661E-3,-2.9155319E-3,1.7129209E-2,1.03399344E-1,-4.026407E-2,2.4601722E-2,7.1439277E-3,1.9378386E-3,2.859518E-2,-5.6948047E-3,1.6462695E-2,8.744284E-2,-5.471778E-4,2.8174908E-3,1.0249649E-2,4.6686637E-3,1.2684178E-3,5.2150637E-3,2.4350159E-4,5.6901E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,-1,-1,9,11,13,15,-1,-1,17,-1,19,21,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[2.0817885E-1,1.7703581E-1,0E0,6.9897324E-3,7.663613E-2,0E0,0E0,7.7758655E-2,2.5625534E-2,1.1706494E-1,7.988347E-2,0E0,0E0,1.5180811E-2,0E0,7.0452355E-2,2.0615816E-2,0E0,0E0,7.3973E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19],"right_children":[2,4,-1,6,8,-1,-1,10,12,14,16,-1,-1,18,-1,20,22,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,4.75E2,-7.6826103E-3,2.3135895E6,3.26989E6,-6.600661E-3,-2.9155319E-3,7E0,1.06748E6,2.7546022E5,1E0,7.1439277E-3,1.9378386E-3,2.0037713E0,-5.6948047E-3,2.446727E12,1.6262975E-1,-5.471778E-4,2.8174908E-3,1.3717E4,4.6686637E-3,1.2684178E-3,5.2150637E-3,2.4350159E-4,5.6901E-3],"split_indices":[44,2,0,44,27,0,0,3,1,32,112,0,0,55,0,30,57,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.98E2,6E0,1E1,1.88E2,5E0,5E0,1.78E2,1E1,2E1,1.58E2,5E0,5E0,1.1E1,9E0,1.41E2,1.7E1,5E0,6E0,1.32E2,9E0,5E0,1.2E1,1.27E2,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[3.4546372E-4,2.372358E-2,-4.9853805E-2,1.2959531E-2,1.4284217E-1,-3.1912476E-2,-9.660273E-3,7.776619E-2,4.2106467E-3,1.1041931E-2,2.2669116E-3,-6.034005E-2,6.118326E-2,-1.3269203E-4,6.053922E-3,-4.3374207E-3,1.1786608E-2,-3.7096877E-2,-1.0389376E-2,-4.9713074E-4,4.6950607E-3,5.130763E-2,-3.972149E-3,3.17068E-2,-6.79687E-2,-9.028315E-4,7.170416E-2,-7.34203E-2,9.169853E-3,4.9376395E-3,-9.288397E-4,-9.671152E-2,-2.8211791E-2,7.2704774E-4,4.075919E-3,-7.784588E-4,-5.132003E-3,-3.663453E-5,4.660076E-3,-2.2579727E-3,-5.434838E-3,1.3324361E-4,-2.8554383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,19,-1,-1,-1,21,23,-1,-1,-1,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5115192E-1,1.8534979E-1,1.7874677E-1,7.6215655E-2,8.955331E-2,1.6777395E-1,0E0,6.2972784E-2,8.7202586E-2,0E0,0E0,1.6657959E-1,4.0536977E-2,0E0,0E0,0E0,6.995764E-2,9.349113E-2,0E0,0E0,0E0,4.7139093E-2,7.4600965E-2,5.304201E-2,3.1553358E-2,0E0,1.7608136E-2,2.3306921E-2,6.0465973E-2,0E0,0E0,8.755282E-3,1.3786099E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,12,12,16,16,17,17,21,21,22,22,23,23,24,24,26,26,27,27,28,28,31,31,32,32],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,20,-1,-1,-1,22,24,-1,-1,-1,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.083812E7,5.108104E2,7.1656E4,1.3176E4,7.7198017E-1,7.5601786E8,-9.660273E-3,7.335273E-1,9.235376E5,1.1041931E-2,2.2669116E-3,2.3177125E6,1.2047E4,-1.3269203E-4,6.053922E-3,-4.3374207E-3,5.1373015E6,2.6786362E8,-1.0389376E-2,-4.9713074E-4,4.6950607E-3,2.3135895E6,1.4693E4,6.50596E5,2.126386E0,-9.028315E-4,3.0662747E5,7.492958E-1,2.5831E4,4.9376395E-3,-9.288397E-4,1.8313253E0,4E0,7.2704774E-4,4.075919E-3,-7.784588E-4,-5.132003E-3,-3.663453E-5,4.660076E-3,-2.2579727E-3,-5.434838E-3,1.3324361E-4,-2.8554383E-3],"split_indices":[44,51,10,9,26,7,0,41,44,0,0,27,9,0,0,0,44,7,0,0,0,44,9,27,53,0,47,55,10,0,0,53,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.45E2,6.7E1,1.34E2,1.1E1,6.1E1,6E0,1.5E1,1.19E2,5E0,6E0,4.7E1,1.4E1,6E0,9E0,8E0,1.11E2,4.2E1,5E0,5E0,9E0,3.1E1,8E1,1.3E1,2.9E1,7E0,2.4E1,1.2E1,6.8E1,5E0,8E0,1.6E1,1.3E1,5E0,1.9E1,5E0,7E0,6.2E1,6E0,5E0,1.1E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.364336E-3,-1.0574123E-1,1.7093858E-2,-4.32481E-2,-8.923564E-3,6.1308825E-3,1.4396141E-1,5.683063E-4,-4.039024E-3,4.5742624E-2,-1.04307765E-2,1.2526486E-2,1.8126821E-3,3.0005876E-2,8.651818E-2,-2.5241837E-2,4.2459678E-2,4.409337E-2,-2.0641144E-3,1.1095878E-3,6.15454E-3,-4.675296E-3,-8.04788E-2,8.221066E-2,-1.5774528E-3,2.6091704E-2,3.890061E-3,-3.8204283E-2,2.3239443E-2,-1.2638317E-1,2.3813143E-4,1.6691883E-3,1.005748E-1,2.7227003E-4,2.804804E-3,3.8902808E-3,-3.184187E-3,3.1590352E-3,-1.2384685E-3,-8.974933E-3,-2.3778311E-3,6.7931544E-3,2.0405264E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5689712E-1,8.75539E-2,2.2721612E-1,2.8153611E-2,0E0,1.0019461E-1,1.5222168E-1,0E0,0E0,2.62793E-2,8.5675046E-2,0E0,0E0,3.6908273E-2,2.8916374E-2,9.618656E-2,7.462707E-2,1.725182E-2,0E0,0E0,0E0,5.985288E-2,9.0845674E-2,1.04142055E-2,0E0,1.3576198E-2,0E0,1.0325533E-1,7.506945E-2,6.0873598E-2,0E0,0E0,2.025108E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,13,13,14,14,15,15,16,16,17,17,21,21,22,22,23,23,25,25,27,27,28,28,29,29,32,32],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2106E4,1.3907746E10,5.818E3,4.791778E-2,-8.923564E-3,6.8915665E-1,5.199796E9,5.683063E-4,-4.039024E-3,3.0954462E2,1E0,1.2526486E-2,1.8126821E-3,9E0,6.16E2,7.751252E-1,3.2004956E5,6.584533E5,-2.0641144E-3,1.1095878E-3,6.15454E-3,1.436E3,1.9E1,2.2E2,-1.5774528E-3,8.7371956E1,3.890061E-3,8.04E2,5.591566E0,2.095716E0,2.3813143E-4,1.6691883E-3,6.4675875E5,2.7227003E-4,2.804804E-3,3.8902808E-3,-3.184187E-3,3.1590352E-3,-1.2384685E-3,-8.974933E-3,-2.3778311E-3,6.7931544E-3,2.0405264E-3],"split_indices":[9,5,2,37,0,55,12,0,0,4,15,0,0,3,10,26,46,31,0,0,0,2,8,10,0,51,0,2,55,53,0,0,47,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.81E2,1.8E1,1.63E2,1.1E1,7E0,1.51E2,1.2E1,5E0,6E0,4.4E1,1.07E2,5E0,7E0,3.3E1,1.1E1,8.4E1,2.3E1,2.8E1,5E0,5E0,6E0,6.2E1,2.2E1,1.5E1,8E0,2E1,8E0,2.8E1,3.4E1,1.4E1,8E0,5E0,1E1,1.3E1,7E0,5E0,2.3E1,1.8E1,1.6E1,7E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.98988E-3,4.298993E-3,-1.3263792E-2,-6.2565473E-3,8.90277E-2,-2.8164687E-2,3.5866547E-2,3.0801646E-2,8.962361E-3,-1.3428184E-2,-1.6895024E-1,2.1984953E-2,9.696608E-2,-4.7206893E-4,3.769065E-3,8.018422E-2,-2.6096953E-2,-1.2021441E-2,-2.8177882E-3,6.0870703E-2,-1.2130867E-2,1.1111092E-3,7.405611E-3,6.073899E-3,2.2470218E-4,-1.1540919E-1,-1.1793869E-2,9.140091E-4,9.476595E-2,-4.8236985E-2,2.7420748E-2,-8.923705E-3,-1.3445544E-3,3.7949048E-3,-1.01653E-3,5.5873278E-3,2.387068E-3,-3.703658E-3,-1.2442356E-3,-4.41121E-4,2.684925E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,19,21,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.603289E-1,1.7763977E-1,0E0,1.6435263E-1,1.1983137E-1,2.4046963E-1,4.9605906E-2,2.996817E-2,0E0,1.2898964E-1,8.4552854E-2,6.8395995E-2,4.316824E-2,0E0,0E0,4.5137107E-2,1.1999571E-1,0E0,0E0,3.2189578E-2,4.1308243E-2,0E0,0E0,0E0,0E0,7.522862E-2,7.286735E-2,0E0,6.9598258E-3,7.701315E-3,1.5193404E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,19,19,20,20,25,25,26,26,28,28,29,29,30,30],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,20,22,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.485318E8,1.0482935E3,-1.3263792E-2,2.1799042E0,1.4545E4,7.8236975E11,2.0459295E6,8.096533E-1,8.962361E-3,1.2414E4,1.03232314E12,6.3236547E10,2.33948E8,-4.7206893E-4,3.769065E-3,1.3261E4,3.4646995E-4,-1.2021441E-2,-2.8177882E-3,3.4585872E8,3.2E1,1.1111092E-3,7.405611E-3,6.073899E-3,2.2470218E-4,1.45592E5,1.1280869E9,9.140091E-4,1.083E3,2.523484E7,4.6466772E5,-8.923705E-3,-1.3445544E-3,3.7949048E-3,-1.01653E-3,5.5873278E-3,2.387068E-3,-3.703658E-3,-1.2442356E-3,-4.41121E-4,2.684925E-3],"split_indices":[44,51,0,53,9,30,27,26,0,9,30,30,7,0,0,10,37,0,0,5,8,0,0,0,0,1,30,0,2,44,31,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.97E2,5E0,1.76E2,2.1E1,1.16E2,6E1,1.4E1,7E0,1.06E2,1E1,5E1,1E1,8E0,6E0,1.2E1,9.4E1,5E0,5E0,2.3E1,2.7E1,5E0,5E0,7E0,5E0,1.2E1,8.2E1,1.1E1,1.2E1,1.4E1,1.3E1,6E0,6E0,7E0,7.5E1,7E0,5E0,5E0,9E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-9.557284E-4,4.2302134E-3,-8.444042E-3,-4.5208056E-3,1.22691885E-1,2.2237676E-3,-1.04750626E-1,6.1699655E-4,1.2789755E-2,-4.894829E-3,7.066396E-3,-1.2004817E-3,-8.636295E-3,5.6956097E-2,-4.7437935E-3,7.72804E-2,-2.2700573E-3,1.3601924E-3,-4.4910153E-3,1.9858927E-3,4.8656194E-3,-4.1936638E-4,3.3675423E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,-1,-1,15,17,19,-1,21,-1,-1,-1,-1,-1],"loss_changes":[1.8983576E-1,2.109967E-1,0E0,1.2867433E-1,2.183621E-1,9.12143E-2,6.770022E-2,0E0,0E0,0E0,1.0161376E-1,0E0,0E0,7.36425E-2,7.618149E-2,2.0148352E-2,0E0,9.263907E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10,13,13,14,14,15,15,17,17],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,-1,-1,16,18,20,-1,22,-1,-1,-1,-1,-1],"split_conditions":[3.6030095E10,1.4056875E3,-8.444042E-3,2.289485E1,1.3841E4,1.68128E8,2.4272E4,6.1699655E-4,1.2789755E-2,-4.894829E-3,1.9413766E8,-1.2004817E-3,-8.636295E-3,1.4688152E9,1.0388931E8,8E0,-2.2700573E-3,9.4010997E-1,-4.4910153E-3,1.9858927E-3,4.8656194E-3,-4.1936638E-4,3.3675423E-3],"split_indices":[5,51,0,56,9,7,10,0,0,0,7,0,0,5,44,3,0,26,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,2.02E2,5E0,1.89E2,1.3E1,1.78E2,1.1E1,8E0,5E0,7E0,1.71E2,6E0,5E0,3.2E1,1.39E2,2.7E1,5E0,1.31E2,8E0,1.2E1,1.5E1,1.15E2,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.3546964E-3,-9.8290674E-2,7.936172E-3,2.122498E-3,-1.4562026E-1,1.5572541E-2,-1.2190936E-1,-1.0809848E-2,-1.791769E-3,8.984492E-3,1.1956564E-1,6.0504995E-4,-1.1312587E-2,-2.8677914E-2,3.129741E-2,8.9405396E-4,9.60763E-3,4.2789616E-3,-4.1965857E-2,5.867157E-3,2.3027342E-2,-1.0168736E-1,-1.7937874E-2,8.3036914E-2,8.465212E-3,-5.9975026E-4,-6.60166E-3,1.8371216E-3,-1.9279694E-3,-1.726462E-4,5.7694097E-3,-2.9594365E-3,1.2408379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,-1,7,9,11,-1,-1,13,15,-1,-1,17,19,-1,-1,-1,21,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.140292E-1,1.481786E-1,1.9624195E-1,0E0,1.3011056E-1,1.2635654E-1,1.7201993E-1,0E0,0E0,1.4880061E-1,8.6136565E-2,0E0,0E0,1.0722902E-1,8.087097E-2,0E0,0E0,0E0,8.427517E-2,0E0,8.93054E-2,5.301349E-2,5.67819E-2,6.522225E-2,1.04466684E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,9,9,10,10,13,13,14,14,18,18,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,-1,8,10,12,-1,-1,14,16,-1,-1,18,20,-1,-1,-1,22,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7E0,1.09315805E5,6.1723E4,2.122498E-3,1E0,1.1510987E1,1E1,-1.0809848E-2,-1.791769E-3,2.5324342E5,1.6773E4,6.0504995E-4,-1.1312587E-2,1.92E2,2.808836E9,8.9405396E-4,9.60763E-3,4.2789616E-3,1.18107956E2,5.867157E-3,9.0996725E5,2.53E2,6.50596E5,1.5697E4,1.4859048E6,-5.9975026E-4,-6.60166E-3,1.8371216E-3,-1.9279694E-3,-1.726462E-4,5.7694097E-3,-2.9594365E-3,1.2408379E-3],"split_indices":[3,32,10,0,15,55,3,0,0,32,9,0,0,11,30,0,0,0,51,0,49,11,27,9,49,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2E1,1.95E2,5E0,1.5E1,1.85E2,1E1,8E0,7E0,1.75E2,1E1,5E0,5E0,6.5E1,1.1E2,5E0,5E0,6E0,5.9E1,8E0,1.02E2,1.6E1,4.3E1,1.9E1,8.3E1,5E0,1.1E1,1.2E1,3.1E1,6E0,1.3E1,1.6E1,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[7.1131624E-3,-2.2026269E-2,2.8357007E-2,-1.2215349E-2,-7.3030414E-3,3.8369182E-2,-7.343246E-2,-4.783255E-3,-2.7056546E-3,7.721029E-2,1.286863E-2,1.7460152E-3,-8.195732E-3,3.722718E-2,-4.0521275E-2,1.2180363E-1,1.4471154E-2,-4.5522254E-2,4.242689E-2,-6.861623E-4,4.766653E-2,-2.4685629E-2,-5.9168264E-3,1.5069374E-3,1.4281815E-1,-1.9210699E-3,4.9645238E-2,2.2518933E-3,-8.094361E-2,-1.6356172E-2,6.95017E-2,2.131153E-2,7.814678E-2,-5.358374E-2,2.6117768E-2,9.241918E-3,3.5182505E-3,-3.9588238E-4,4.4166204E-3,-1.1776626E-3,-1.02379315E-1,1.2706688E-3,-3.3128536E-3,3.6682256E-2,1.2583867E-1,2.0858315E-3,1.5375883E-4,5.662791E-3,2.0486228E-3,-6.920204E-4,-3.8120132E-3,2.8781423E-3,-1.2531123E-4,-2.6468302E-3,-6.139556E-3,2.8727932E-3,2.4315815E-4,7.790612E-3,3.2620435E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,9,11,-1,13,15,17,-1,-1,19,21,23,25,27,29,-1,31,33,-1,-1,35,-1,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,53,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2752648E-1,1.09450705E-1,1.2409449E-1,6.8060525E-2,0E0,1.0696761E-1,1.242457E-1,0E0,1.1476487E-1,1.18540525E-1,1.1756887E-1,0E0,0E0,2.0594038E-2,4.8199996E-2,4.2973727E-2,3.8460277E-2,7.918517E-2,7.275609E-2,0E0,2.323708E-2,5.1315654E-2,0E0,0E0,5.5339545E-2,0E0,3.0325547E-2,0E0,1.802209E-2,3.579022E-2,5.2701354E-2,7.1024783E-3,1.4788084E-2,2.12254E-2,1.2841713E-2,0E0,0E0,0E0,0E0,0E0,7.4964166E-3,0E0,0E0,1.5148977E-2,1.2289554E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,24,24,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,40,40,43,43,44,44],"right_children":[2,4,6,8,-1,10,12,-1,14,16,18,-1,-1,20,22,24,26,28,30,-1,32,34,-1,-1,36,-1,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,54,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1546597E2,3.6543632E6,1.4089414E8,9.235376E5,-7.3030414E-3,2.1358025E0,1.4689625E5,-4.783255E-3,5.2139695E6,9.235021E-1,1.631E3,1.7460152E-3,-8.195732E-3,1.0963781E0,4.8028188E5,1.92E2,1.3843815E6,4.720179E10,1.4157E4,-6.861623E-4,4.17266E5,1.64935E5,-5.9168264E-3,1.5069374E-3,1.306E3,-1.9210699E-3,1.514E3,2.2518933E-3,1.08989E5,2.4010162E5,4.633548E-1,5.9168223E-2,6.6803556E5,1.3986014E-2,8.193631E7,9.241918E-3,3.5182505E-3,-3.9588238E-4,4.4166204E-3,-1.1776626E-3,1.541336E0,1.2706688E-3,-3.3128536E-3,8.509488E2,1.6705686E0,2.0858315E-3,1.5375883E-4,5.662791E-3,2.0486228E-3,-6.920204E-4,-3.8120132E-3,2.8781423E-3,-1.2531123E-4,-2.6468302E-3,-6.139556E-3,2.8727932E-3,2.4315815E-4,7.790612E-3,3.2620435E-3],"split_indices":[4,27,44,44,0,55,32,0,44,26,2,0,0,52,32,0,27,30,9,0,31,1,0,0,0,0,2,0,12,32,26,37,31,56,12,0,0,0,0,0,52,0,0,4,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,8.6E1,1.18E2,8.1E1,5E0,1.08E2,1E1,7E0,7.4E1,4.2E1,6.6E1,5E0,5E0,3.6E1,3.8E1,2.4E1,1.8E1,2.2E1,4.4E1,6E0,3E1,3.3E1,5E0,5E0,1.9E1,7E0,1.1E1,6E0,1.6E1,1.4E1,3E1,1.7E1,1.3E1,2.1E1,1.2E1,1E1,9E0,5E0,6E0,5E0,1.1E1,8E0,6E0,2E1,1E1,7E0,1E1,5E0,8E0,9E0,1.2E1,5E0,7E0,5E0,6E0,1.1E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.095566E-3,-1.2488788E-2,1.1695275E-1,4.6495297E-3,-4.796547E-2,1.064835E-2,1.1186311E-3,-7.3224274E-3,6.2121253E-2,-9.3917735E-2,-1.7680738E-2,1.3521488E-3,-4.3468946E-3,6.463305E-3,1.9159265E-2,-1.5515956E-1,-3.0162381E-2,9.640252E-3,-5.5849615E-2,-1.6811745E-2,4.056598E-2,-9.3058153E-4,2.6516225E-3,-9.063187E-3,-4.293154E-3,-3.5266269E-3,2.9679737E-4,-2.6606468E-2,3.8771576E-3,-8.85595E-2,6.4325426E-4,2.519898E-3,-2.848914E-2,7.713755E-2,-3.5862029E-3,1.072817E-3,-4.4461307E-3,-5.5913585E-3,-2.186995E-3,-5.858892E-3,-6.331601E-4,7.232601E-3,1.1087124E-3,2.2731214E-3,-1.6215863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,-1,37,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0364168E-1,1.1392102E-1,1.2295887E-1,8.783711E-2,8.323024E-2,0E0,0E0,7.681966E-2,6.621235E-2,8.902785E-2,4.016524E-2,6.977274E-2,0E0,0E0,2.2019297E-2,9.805173E-3,2.0992368E-2,6.158456E-2,3.7445426E-2,5.580391E-2,5.0332785E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.3977367E-2,0E0,7.959329E-3,0E0,0E0,8.1479475E-2,6.4276084E-2,2.4771227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,27,27,29,29,32,32,33,33,34,34],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,-1,38,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3904022E3,8.617448E5,4.607055E5,4E1,1.6155814E6,1.064835E-2,1.1186311E-3,6.6774924E7,5.3246627E0,3.0050538E5,2.67302E5,1.436E3,-4.3468946E-3,6.463305E-3,2.1100497E-1,2.0983605E0,1.4936014E0,3.14016E5,3.0229E4,1.4E0,2.9807162E0,-9.3058153E-4,2.6516225E-3,-9.063187E-3,-4.293154E-3,-3.5266269E-3,2.9679737E-4,7.9716E4,3.8771576E-3,1.714995E0,6.4325426E-4,2.519898E-3,5.94E2,3.701903E8,3.81938E5,1.072817E-3,-4.4461307E-3,-5.5913585E-3,-2.186995E-3,-5.858892E-3,-6.331601E-4,7.232601E-3,1.1087124E-3,2.2731214E-3,-1.6215863E-3],"split_indices":[51,31,27,8,31,0,0,44,56,32,28,2,0,0,38,55,52,1,10,53,55,0,0,0,0,0,0,1,0,52,0,0,2,7,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.98E2,1.86E2,1.2E1,1.26E2,6E1,5E0,7E0,1.05E2,2.1E1,2.3E1,3.7E1,9.6E1,9E0,7E0,1.4E1,1.1E1,1.2E1,2.2E1,1.5E1,6.6E1,3E1,7E0,7E0,6E0,5E0,5E0,7E0,1.5E1,7E0,1E1,5E0,9E0,5.7E1,1.6E1,1.4E1,9E0,6E0,5E0,5E0,7E0,5E1,6E0,1E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.381206E-5,6.741054E-3,-1.08200945E-1,1.0291582E-3,5.851233E-3,-1.1417227E-2,8.998055E-4,-9.020174E-2,6.8240184E-3,-6.8379045E-3,-1.0846909E-3,-3.7622836E-3,1.0896399E-2,5.4145367E-3,5.68964E-3,1.19317835E-2,-7.1770936E-2,-3.908729E-5,2.4048458E-3,-3.3393555E-4,-5.608214E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,13,-1,15,17,19,-1,-1,-1,-1],"loss_changes":[1.4916813E-1,1.257647E-1,2.0274137E-1,9.736221E-2,0E0,0E0,0E0,3.5805017E-2,6.099009E-2,0E0,0E0,0E0,8.762349E-2,0E0,7.7736035E-2,7.223382E-2,3.4402214E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,12,12,14,14,15,15,16,16],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,14,-1,16,18,20,-1,-1,-1,-1],"split_conditions":[2.835E3,1.1510987E1,3.299E3,2.3950832E-5,5.851233E-3,-1.1417227E-2,8.998055E-4,2.12293E5,9.541066E5,-6.8379045E-3,-1.0846909E-3,-3.7622836E-3,1.1280869E9,5.4145367E-3,5.08616E5,2.1E1,1.3723022E0,-3.908729E-5,2.4048458E-3,-3.3393555E-4,-5.608214E-3],"split_indices":[0,55,0,41,0,0,0,28,44,0,0,0,30,0,32,8,55,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.9E2,1.1E1,1.82E2,8E0,5E0,6E0,1E1,1.72E2,5E0,5E0,7E0,1.65E2,7E0,1.58E2,1.47E2,1.1E1,1.11E2,3.6E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.8882776E-3,1.00166835E-2,-7.371662E-2,4.7211065E-3,4.8804465E-3,-4.1055635E-2,-1.0245122E-2,1.7040744E-2,-2.1460107E-2,-2.0179208E-3,-8.808312E-3,2.4134438E-2,-3.576226E-3,3.66077E-2,-4.2288773E-2,-3.3715464E-2,2.9525624E-3,-5.267561E-3,4.6923134E-2,-6.3393277E-4,3.150777E-3,-1.0187121E-1,9.985219E-4,8.436718E-4,-4.179634E-3,-4.012018E-2,2.6900155E-2,7.190755E-2,3.906673E-3,-1.7125866E-3,-6.164742E-3,3.5533864E-2,-3.362003E-2,-4.214546E-3,-1.0811037E-5,3.1787308E-3,-5.716725E-5,2.431797E-3,8.105542E-3,-1.9674457E-3,1.1181055E-3,-2.0289488E-4,3.0922557E-3,4.8777572E-4,-3.1792584E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,17,-1,19,21,23,-1,25,27,-1,-1,29,31,-1,-1,33,35,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9298853E-1,8.02646E-2,1.4225613E-1,5.1327497E-2,0E0,1.5571955E-1,0E0,7.2045505E-2,6.323132E-2,4.847953E-2,0E0,6.792052E-2,0E0,2.1216236E-2,9.8109886E-2,4.5493107E-2,0E0,5.1530052E-2,6.0979337E-2,0E0,0E0,2.3761153E-2,2.8692609E-2,0E0,0E0,4.197786E-2,2.6678098E-2,6.547545E-2,2.018457E-2,0E0,0E0,1.4118374E-2,1.788503E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,17,17,18,18,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,18,-1,20,22,24,-1,26,28,-1,-1,30,32,-1,-1,34,36,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8351054E11,6.811E3,2.363013E6,2.2760513E0,4.8804465E-3,1.9E1,-1.0245122E-2,3.9556694E-1,3.1E2,5.683493E6,-8.808312E-3,8.92E2,-3.576226E-3,1.11258715E-1,2.7573213E8,7.822023E7,2.9525624E-3,8E0,1.007E3,-6.3393277E-4,3.150777E-3,1.9303116E5,1.6389892E0,8.436718E-4,-4.179634E-3,4.3779754E-4,1.6827E4,1.996E3,7.280497E0,-1.7125866E-3,-6.164742E-3,1.6E1,4.2736053E0,-4.214546E-3,-1.0811037E-5,3.1787308E-3,-5.716725E-5,2.431797E-3,8.105542E-3,-1.9674457E-3,1.1181055E-3,-2.0289488E-4,3.0922557E-3,4.8777572E-4,-3.1792584E-3],"split_indices":[30,2,27,55,0,3,0,37,10,1,0,2,0,57,7,44,0,3,0,0,0,31,52,0,0,37,28,2,56,0,0,8,56,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.65E2,3.2E1,1.57E2,8E0,2.7E1,5E0,1.07E2,5E1,2.2E1,5E0,1E2,7E0,1.3E1,3.7E1,1.5E1,7E0,4.4E1,5.6E1,5E0,8E0,1.5E1,2.2E1,8E0,7E0,2.1E1,2.3E1,3.5E1,2.1E1,5E0,1E1,1.1E1,1.1E1,9E0,1.2E1,9E0,1.4E1,3E1,5E0,6E0,1.5E1,5E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-9.95141E-3,-4.0595704E-3,-9.053523E-3,-1.3823979E-2,1.15495674E-1,-2.445644E-2,3.1380143E-2,1.2050496E-4,1.2730055E-2,-6.7949397E-3,-6.0015667E-2,4.132615E-3,1.3286835E-2,-2.6767712E-2,3.093557E-2,-9.7700045E-2,1.0016516E-2,-3.060755E-3,3.4704007E-2,2.8235086E-3,-3.8746968E-2,6.0488484E-3,-1.0071418E-2,-5.927876E-2,-1.03229545E-2,2.7865102E-3,-1.8800803E-3,6.552377E-2,3.4940982E-5,9.767149E-4,-2.5867554E-3,-3.526529E-3,5.0416094E-4,-3.6588984E-3,-3.2836446E-4,1.3759182E-3,4.3791626E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,17,19,21,23,25,-1,27,-1,29,-1,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9858697E-1,2.1639566E-1,0E0,8.295535E-2,2.3694031E-1,8.688826E-2,3.1622004E-2,0E0,0E0,7.166487E-2,1.2237811E-1,0E0,4.5191552E-2,6.593666E-2,1.3026589E-1,1.2297669E-1,4.260149E-2,0E0,2.1941137E-2,0E0,5.068937E-2,0E0,3.2180242E-2,2.0984277E-2,0E0,0E0,0E0,7.853508E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,20,20,22,22,23,23,27,27],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,18,20,22,24,26,-1,28,-1,30,-1,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841177E10,1.3904022E3,-9.053523E-3,2.078125E0,1.4545E4,8.601996E-1,3.77E2,1.2050496E-4,1.2730055E-2,1.6921558E2,1.3642281E-1,4.132615E-3,6.9146306E-4,8.66E3,1.854111E7,7.7296586E11,7.4456794E5,-3.060755E-3,1.497692E8,2.8235086E-3,1.3345E4,6.0488484E-3,1.895E3,2.1577182E0,-1.03229545E-2,2.7865102E-3,-1.8800803E-3,3.101519E5,3.4940982E-5,9.767149E-4,-2.5867554E-3,-3.526529E-3,5.0416094E-4,-3.6588984E-3,-3.2836446E-4,1.3759182E-3,4.3791626E-3],"split_indices":[5,51,0,52,9,26,0,0,0,51,41,0,37,28,44,30,31,0,12,0,9,0,2,53,0,0,0,31,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,1.83E2,5E0,1.7E2,1.3E1,1.38E2,3.2E1,8E0,5E0,9.3E1,4.5E1,7E0,2.5E1,6.1E1,3.2E1,2.9E1,1.6E1,5E0,2E1,7E0,5.4E1,9E0,2.3E1,2.3E1,6E0,8E0,8E0,1E1,1E1,1.1E1,4.3E1,5E0,1.8E1,1.7E1,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[6.7305095E-3,1.7460078E-2,-7.1220785E-2,-4.888752E-3,2.2854166E-2,1.7001651E-3,-1.23524144E-1,2.404821E-4,4.0388856E-2,-9.314982E-3,-9.316741E-4,-9.192367E-3,3.9643114E-3,6.6240965E-3,3.134043E-2,-2.4730662E-2,4.943009E-2,4.4192925E-2,-1.3605638E-2,-3.580801E-2,2.775148E-3,3.6156115E-3,1.1347134E-3,2.7026076E-2,8.432041E-3,-5.0528117E-2,2.4553718E-2,-2.5033941E-3,1.1523056E-4,-1.0646E-3,1.976691E-3,-7.2936976E-4,-3.7086133E-3,-6.730798E-4,2.8296807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,-1,7,-1,9,11,13,-1,-1,15,-1,-1,17,19,21,23,25,27,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7270218E-1,1.1900742E-1,1.4391415E-1,0E0,6.898545E-2,0E0,1.21679455E-1,6.068817E-2,8.2458824E-2,0E0,0E0,6.498495E-2,0E0,0E0,5.3169154E-2,5.334888E-2,8.268211E-3,1.5590033E-1,3.082835E-2,3.154809E-2,0E0,0E0,0E0,4.5779627E-2,0E0,9.261088E-3,1.5484236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,11,11,14,14,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26],"right_children":[2,4,6,-1,8,-1,10,12,14,-1,-1,16,-1,-1,18,20,22,24,26,28,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5965E5,1.01E4,6.5229E4,-4.888752E-3,1.222E3,1.7001651E-3,1E0,3.1515045E6,1.5789307E8,-9.314982E-3,-9.316741E-4,4.5929728E5,3.9643114E-3,6.6240965E-3,2.943697E7,2.9372938E0,1.1924399E0,1.2199858E7,7.203918E9,4.982441E0,2.775148E-3,3.6156115E-3,1.1347134E-3,1.3328E4,8.432041E-3,2.1287676E7,1.856461E7,-2.5033941E-3,1.1523056E-4,-1.0646E-3,1.976691E-3,-7.2936976E-4,-3.7086133E-3,-6.730798E-4,2.8296807E-3],"split_indices":[9,9,28,0,2,0,15,31,5,0,0,32,0,0,49,55,57,47,12,56,0,0,0,9,0,44,46,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.8E2,2.4E1,7E0,1.73E2,8E0,1.6E1,7.6E1,9.7E1,9E0,7E0,6.9E1,7E0,7E0,9E1,5.5E1,1.4E1,7E1,2E1,4.9E1,6E0,6E0,8E0,6.3E1,7E0,1E1,1E1,3.4E1,1.5E1,1.4E1,4.9E1,5E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.734199E-3,1.3744242E-2,-9.240938E-3,2.3152918E-2,-3.7991587E-2,-8.5648766E-4,4.9247783E-2,-6.1307233E-3,4.633573E-3,1.0621249E-2,-6.555168E-3,7.024371E-3,3.5039723E-2,-3.1629456E-3,2.929784E-2,-4.920238E-3,1.920218E-2,-1.7098352E-2,5.024164E-2,2.8506932E-3,1.055836E-4,8.000154E-2,6.9356076E-3,-2.0492584E-3,8.2087173E-4,-7.4160867E-3,6.4500555E-2,1.524072E-3,5.08515E-3,2.57247E-3,-3.2032706E-4,-2.0306283E-3,1.1314659E-3,4.420035E-3,4.033807E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,-1,17,-1,19,-1,21,23,25,-1,-1,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4862354E-1,9.774597E-2,0E0,1.05924435E-1,1.1948568E-1,1.4034997E-1,1.0692477E-1,0E0,4.027229E-2,8.268222E-2,0E0,0E0,5.7926156E-2,0E0,1.378132E-2,0E0,5.7617523E-2,1.5675277E-2,4.6517998E-2,0E0,0E0,1.3000146E-2,4.2398307E-2,0E0,0E0,1.4023744E-2,6.927845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,12,12,14,14,16,16,17,17,18,18,21,21,22,22,25,25,26,26],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,-1,18,-1,20,-1,22,24,26,-1,-1,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4299594E8,2.5965E5,-9.240938E-3,1.6720915E2,7.05571E5,2.0866848E7,1.0932244E9,-6.1307233E-3,3.1622803E5,1.074074E0,-6.555168E-3,7.024371E-3,5.269822E5,-3.1629456E-3,7.756314E7,-4.920238E-3,1.704313E4,1.5697E4,2.2903395E8,2.8506932E-3,1.055836E-4,7.12E2,6.2085975E9,-2.0492584E-3,8.2087173E-4,1.8567E4,8.992676E1,1.524072E-3,5.08515E-3,2.57247E-3,-3.2032706E-4,-2.0306283E-3,1.1314659E-3,4.420035E-3,4.033807E-4],"split_indices":[44,9,0,51,9,44,5,0,27,52,0,0,27,0,12,0,47,9,7,0,0,2,30,0,0,12,57,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.98E2,5E0,1.68E2,3E1,8.8E1,8E1,9E0,2.1E1,8.2E1,6E0,9E0,7.1E1,5E0,1.6E1,5E0,7.7E1,1.6E1,5.5E1,7E0,9E0,1.2E1,6.5E1,9E0,7E0,1.1E1,4.4E1,5E0,7E0,1.4E1,5.1E1,5E0,6E0,2.9E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.6418557E-4,-6.4944443E-3,4.5279595E-3,-3.6861983E-4,6.6873482E-3,-5.0364896E-3,4.702346E-3,8.246746E-3,-4.173349E-3,1.877944E-2,-1.951385E-2,2.9930364E-2,-5.2886378E-2,-4.9523227E-3,-7.8992825E-3,9.2911575E-4,4.0745535E-3,-4.378941E-3,1.8866103E-3,7.835974E-4,-2.165033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,-1,3,5,-1,-1,7,9,-1,11,13,15,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5095755E-1,0E0,1.3231623E-1,1.0355283E-1,0E0,0E0,6.124718E-2,5.299736E-2,0E0,1.0627027E-1,4.788155E-2,6.455845E-2,6.756645E-2,0E0,4.144308E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,9,9,10,10,11,11,12,12,14,14],"right_children":[2,-1,4,6,-1,-1,8,10,-1,12,14,16,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01E4,-6.4944443E-3,1.2144419E1,6E0,6.6873482E-3,-5.0364896E-3,4.966421E6,2.5605617E0,-4.173349E-3,3.34231E-1,1.3016E4,2.409E3,9.621332E-1,-4.9523227E-3,1.8738E4,9.2911575E-4,4.0745535E-3,-4.378941E-3,1.8866103E-3,7.835974E-4,-2.165033E-3],"split_indices":[9,0,55,3,0,0,27,55,0,37,9,2,26,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,7E0,1.99E2,1.93E2,6E0,8E0,1.85E2,1.79E2,6E0,1.3E2,4.9E1,1.13E2,1.7E1,5E0,4.4E1,9.6E1,1.7E1,1.2E1,5E0,2.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.3359891E-3,7.598631E-3,-5.795484E-3,-1.3054161E-3,9.671589E-2,-1.30533045E-2,3.0901961E-2,6.017363E-4,8.150426E-3,-5.933671E-4,-1.1269043E-1,8.863279E-2,2.8749483E-3,-1.2696118E-2,4.0392615E-2,-8.385759E-3,-2.1103797E-3,7.4418155E-3,1.9042853E-3,2.6180213E-2,-2.854947E-3,-4.3699328E-2,5.8003473E-3,4.338851E-3,8.50301E-3,-6.0795285E-3,4.381679E-3,-2.9421232E-3,2.2153168E-3,1.6627694E-3,-1.7814175E-3,-1.1049599E-3,3.3641206E-3,-1.5691422E-3,2.3130474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,-1,27,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2680905E-1,1.4851156E-1,0E0,6.510442E-2,1.0288632E-1,1.5635866E-1,7.417423E-2,0E0,0E0,5.6565702E-2,5.2202687E-2,4.273562E-2,4.81459E-2,5.071706E-2,4.08299E-2,0E0,0E0,0E0,0E0,5.1097564E-2,0E0,5.5409856E-2,7.087297E-2,0E0,3.4739327E-2,2.623093E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,19,19,21,21,22,22,24,24,25,25],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,-1,28,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4299594E8,5.818E3,-5.795484E-3,3.494881E2,1.4781E4,4.520813E7,3.2875034E5,6.017363E-4,8.150426E-3,3.7597818E2,2.081E3,5.5318156E7,1.3182058E3,2.6727284E5,1.45819E5,-8.385759E-3,-2.1103797E-3,7.4418155E-3,1.9042853E-3,8.4342186E1,-2.854947E-3,9E0,6.71234E6,4.338851E-3,3.649047E8,1.5452969E6,4.381679E-3,-2.9421232E-3,2.2153168E-3,1.6627694E-3,-1.7814175E-3,-1.1049599E-3,3.3641206E-3,-1.5691422E-3,2.3130474E-3],"split_indices":[44,2,0,51,9,44,31,0,0,54,2,44,4,32,28,0,0,0,0,57,0,8,44,0,7,27,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.86E2,7E0,1.7E2,1.6E1,1.25E2,4.5E1,8E0,8E0,1.12E2,1.3E1,1.4E1,3.1E1,8.7E1,2.5E1,6E0,7E0,5E0,9E0,2.3E1,8E0,3.2E1,5.5E1,9E0,1.6E1,1.6E1,7E0,2.7E1,5E0,3.3E1,2.2E1,1.1E1,5E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.7254089E-3,7.6746326E-3,-4.719512E-3,1.1724859E-2,-5.173395E-3,4.4106925E-3,1.0753885E-1,4.4548288E-2,-4.2752638E-3,8.823091E-3,-1.2132875E-3,2.0563329E-2,7.1408995E-3,-9.569353E-3,3.904215E-3,-2.2979942E-3,3.7546612E-2,-4.2280434E-3,-6.8416536E-2,1.0763042E-3,3.666471E-3,-2.0393976E-3,1.3758341E-4,1.3414687E-3,-6.768783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,-1,17,-1,-1,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0747822E-1,9.682966E-2,0E0,1.3828397E-1,0E0,6.468763E-2,1.4719811E-1,7.965112E-2,7.025212E-2,0E0,0E0,3.4389094E-2,0E0,4.528729E-2,0E0,0E0,1.1802625E-2,3.6598813E-2,8.8266775E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,13,13,16,16,17,17,18,18],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,-1,18,-1,-1,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,3.6030095E10,-4.719512E-3,1.3904022E3,-5.173395E-3,1.9356645E8,2.193351E6,1.10472E5,2.7319932E6,8.823091E-3,-1.2132875E-3,1.68128E8,7.1408995E-3,5.08616E5,3.904215E-3,-2.2979942E-3,2.0895522E0,9.06E2,9.5606E4,1.0763042E-3,3.666471E-3,-2.0393976E-3,1.3758341E-4,1.3414687E-3,-6.768783E-3],"split_indices":[27,5,0,51,0,7,31,28,27,0,0,7,0,32,0,0,55,2,1,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2.03E2,9E0,1.97E2,6E0,1.84E2,1.3E1,3.2E1,1.52E2,8E0,5E0,2.7E1,5E0,1.44E2,8E0,5E0,2.2E1,1.33E2,1.1E1,1.7E1,5E0,2E1,1.13E2,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.831833E-3,-7.791919E-2,1.5585499E-2,3.901734E-4,-8.420296E-3,1.007099E-2,6.597637E-3,-2.6131146E-2,2.9313901E-2,-1.8034883E-2,-4.792742E-3,3.8269266E-2,-5.298094E-2,1.5172308E-3,-3.1689696E-2,2.2402702E-2,7.82467E-2,1.2109441E-3,-5.3892015E-3,-5.4035876E-3,-1.904842E-2,5.0052438E-2,-6.53043E-3,-1.1575788E-3,1.1046521E-1,8.6281466E-4,-2.3395296E-3,-1.3551611E-3,3.1000965E-3,-1.6205095E-3,1.887408E-3,3.1010793E-3,7.0423125E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,-1,15,17,-1,19,21,23,-1,-1,-1,25,27,29,-1,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3618603E-1,1.4457783E-1,1.2455337E-1,0E0,0E0,1.2616022E-1,0E0,3.5893995E-2,8.899123E-2,4.0368393E-2,0E0,6.611766E-2,5.872734E-2,0E0,4.5011334E-2,6.272635E-2,1.01653144E-1,0E0,0E0,0E0,4.613435E-2,4.7757335E-2,5.0221354E-2,0E0,2.8912157E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,20,20,21,21,22,22,24,24],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,-1,16,18,-1,20,22,24,-1,-1,-1,26,28,30,-1,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7E0,1.3473596E6,1.1510987E1,3.901734E-4,-8.420296E-3,2.5324342E5,6.597637E-3,3.241538E6,3.237835E-1,3.58E2,-4.792742E-3,1.0220919E6,8.194808E5,1.5172308E-3,6.71E2,3.2144122E5,2.5176196E2,1.2109441E-3,-5.3892015E-3,-5.4035876E-3,4.6318403E5,8.359578E4,5.0778572E5,-1.1575788E-3,1.766129E0,8.6281466E-4,-2.3395296E-3,-1.3551611E-3,3.1000965E-3,-1.6205095E-3,1.887408E-3,3.1010793E-3,7.0423125E-3],"split_indices":[3,27,55,0,0,32,0,27,37,0,0,27,47,0,2,46,4,0,0,0,31,31,27,0,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.6E1,1.86E2,9E0,7E0,1.79E2,7E0,6.2E1,1.17E2,5.7E1,5E0,1.06E2,1.1E1,1.2E1,4.5E1,7.7E1,2.9E1,5E0,6E0,5E0,4E1,3.9E1,3.8E1,7E0,2.2E1,1.8E1,2.2E1,6E0,3.3E1,2.4E1,1.4E1,1.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-2.1946263E-3,2.646767E-3,-7.5484533E-3,-4.838696E-3,7.674689E-3,-5.268686E-3,1.1553685E-2,1.5994856E-2,-4.814388E-3,6.35635E-3,4.8124943E-2,2.6919441E-2,-2.1407055E-2,7.85664E-2,2.317886E-3,8.7501877E-4,4.3714005E-3,-4.068125E-3,-1.4343046E-4,4.502313E-3,7.986167E-4,-2.5087707E-3,2.1957948E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,-1,5,-1,7,9,-1,11,13,15,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4934996E-1,1.013519E-1,0E0,0E0,8.56972E-2,0E0,9.117762E-2,5.357947E-2,0E0,7.764796E-2,5.559524E-2,4.175908E-2,6.7569844E-2,2.1327168E-2,4.2853165E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14],"right_children":[2,4,-1,-1,6,-1,8,10,-1,12,14,16,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.566851E8,9.235376E5,-7.5484533E-3,-4.838696E-3,1.0057E4,-5.268686E-3,9.6883655E-1,1.86611E5,-4.814388E-3,3.1617646E0,5.9919E4,4.51E3,2.0767594E-2,2.6486957E0,7.3240237E8,8.7501877E-4,4.3714005E-3,-4.068125E-3,-1.4343046E-4,4.502313E-3,7.986167E-4,-2.5087707E-3,2.1957948E-3],"split_indices":[44,44,0,0,9,0,26,11,0,56,10,10,37,53,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.92E2,5E0,8E0,1.84E2,5E0,1.79E2,1.73E2,6E0,1.34E2,3.9E1,7.7E1,5.7E1,2.3E1,1.6E1,6.9E1,8E0,1.2E1,4.5E1,1.8E1,5E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-6.016137E-3,-1.0773682E-1,-4.194025E-4,-3.5203335E-5,-9.427532E-3,-6.4156796E-3,4.3749237E-3,9.354751E-3,-6.261878E-3,2.8013228E-3,7.5348266E-3,-7.2546857E-3,4.1099217E-2,1.8817674E-2,-2.38712E-2,5.4259285E-2,-1.8267948E-3,3.8214512E-3,3.7954942E-5,-2.5428159E-3,2.809707E-4,3.16686E-3,1.1768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,-1,-1,-1,7,9,-1,11,-1,13,15,17,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.200779E-1,1.04664356E-1,1.2923089E-1,0E0,0E0,0E0,1.3239466E-1,1.8277425E-1,0E0,7.039071E-2,0E0,6.326781E-2,4.1690648E-2,6.255804E-2,7.760361E-2,9.982131E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15],"right_children":[2,4,6,-1,-1,-1,8,10,-1,12,-1,14,16,18,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6E0,1.3335946E6,1.01E4,-3.5203335E-5,-9.427532E-3,-6.4156796E-3,1.1430797E9,2.6825117E3,-6.261878E-3,2.9E1,7.5348266E-3,1.537E4,1.3E1,8.0224875E4,2.6727284E5,6.4E1,-1.8267948E-3,3.8214512E-3,3.7954942E-5,-2.5428159E-3,2.809707E-4,3.16686E-3,1.1768E-3],"split_indices":[3,27,9,0,0,0,7,4,0,8,0,9,3,32,32,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1E1,2E2,5E0,5E0,6E0,1.94E2,1.88E2,6E0,1.81E2,7E0,1.44E2,3.7E1,5.6E1,8.8E1,3.2E1,5E0,1.2E1,4.4E1,4.4E1,4.4E1,2.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[7.302973E-3,1.5426283E-3,7.593345E-3,9.934854E-3,-7.230181E-2,-3.5403352E-3,4.441492E-2,1.2183896E-3,-1.1936586E-1,5.9988275E-3,-5.2711107E-3,9.856841E-2,7.403497E-3,-8.7363655E-3,-1.8839287E-3,-1.1002768E-2,2.039365E-2,9.268261E-3,1.1970541E-2,-8.413091E-3,3.32656E-3,3.488051E-2,-3.236424E-2,1.0666802E-2,5.1634945E-3,-1.1845741E-3,2.23597E-3,6.4151622E-3,-1.6212009E-3,-5.816462E-4,2.6581686E-3,-3.972148E-3,1.7107339E-4,1.2173996E-3,-2.8600174E-3,1.1328069E-3,-1.25724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,19,-1,-1,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6610347E-1,1.1536636E-1,0E0,7.7749684E-2,9.043061E-2,1.23568654E-1,9.3472235E-2,0E0,5.8373302E-2,2.7634699E-2,0E0,1.5381205E-1,2.9037274E-2,0E0,0E0,5.2095868E-2,5.0699145E-2,0E0,1.5161534E-2,9.26584E-3,0E0,1.7143657E-2,6.568709E-2,5.9615273E-2,0E0,0E0,0E0,9.62368E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,15,15,16,16,18,18,19,19,21,21,22,22,23,23,27,27],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,20,-1,-1,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3717E4,9.826537E7,7.593345E-3,2.081E3,1.4457831E-1,5.0826204E7,1.06748E6,1.2183896E-3,3.595572E6,3.0823356E5,-5.2711107E-3,2.9807162E0,2.1030048E10,-8.7363655E-3,-1.8839287E-3,1.2345679E-2,2.5302054E2,9.268261E-3,2.8352614E9,8.091679E-1,3.32656E-3,2.5549E4,5.01E2,4.1847788E6,5.1634945E-3,-1.1845741E-3,2.23597E-3,6.055513E2,-1.6212009E-3,-5.816462E-4,2.6581686E-3,-3.972148E-3,1.7107339E-4,1.2173996E-3,-2.8600174E-3,1.1328069E-3,-1.25724E-3],"split_indices":[2,44,0,2,56,44,1,0,1,32,0,55,5,0,0,56,51,0,5,26,0,1,0,49,0,0,0,51,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.84E2,6E0,1.66E2,1.8E1,1.2E2,4.6E1,6E0,1.2E1,1.11E2,9E0,1.8E1,2.8E1,6E0,6E0,5.1E1,6E1,8E0,1E1,2.3E1,5E0,1.6E1,3.5E1,5.5E1,5E0,5E0,5E0,1.5E1,8E0,5E0,1.1E1,1.4E1,2.1E1,4.6E1,9E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-6.7159176E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics index 170529e7f..d3305e570 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics @@ -40,70 +40,77 @@ 38,sr_remoteBlocksFetched_sum,0.0012814552,117736.82903225806,172191.79281083873,3.0,1697.75,52133.5,165851.75,999894.0 39,sqlOp_Subquery,0.00110424,0.25806451612903225,0.4382771459540444,0.0,0.0,0.0,1.0,1.0 40,sqlOp_SortMergeJoin,0.00069089007,0.535483870967742,0.4995456752050157,0.0,0.0,1.0,1.0,1.0 -41,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -42,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_HashAggregate,0.0,0.9903225806451613,0.09805499648850532,0.0,1.0,1.0,1.0,1.0 -47,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -52,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -71,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -75,diskBytesSpilledRatio,0.0,0.008646349817617971,0.11400204273633267,0.0,0.0,0.0,0.0,1.8994929430148348 -76,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -78,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -81,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -83,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -84,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -85,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -86,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +41,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +42,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +43,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +44,sqlOp_HashAggregate,0.0,0.9903225806451613,0.09805499648850532,0.0,1.0,1.0,1.0,1.0 +45,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +48,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +56,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_BroadcastHashJoin,0.0,0.9290322580645162,0.2571861028762713,0.0,1.0,1.0,1.0,1.0 +76,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +81,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,diskBytesSpilledRatio,0.0,0.008646349817617971,0.11400204273633267,0.0,0.0,0.0,0.0,1.8994929430148348 +83,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +84,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +85,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +86,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 87,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -88,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -89,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -92,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,sqlOp_BroadcastNestedLoopJoin,0.0,0.03870967741935484,0.1932140342025925,0.0,0.0,0.0,0.0,1.0 -95,sqlOp_BroadcastHashJoin,0.0,0.9290322580645162,0.2571861028762713,0.0,1.0,1.0,1.0,1.0 -96,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,diskBytesSpilled_mean,0.0,508837.0986981265,6334276.584243264,0.0,0.0,0.0,0.0,105076448.0334672 -107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +90,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +91,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 +93,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +94,diskBytesSpilled_mean,0.0,508837.0986981265,6334276.584243264,0.0,0.0,0.0,0.0,105076448.0334672 +95,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +98,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +103,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_BroadcastNestedLoopJoin,0.0,0.03870967741935484,0.1932140342025925,0.0,0.0,0.0,0.0,1.0 +105,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg index 707eaec55..c07432a16 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.036102999","gamma":"0.00912368763","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.036102999","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00912368763","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.603316069"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.036102999","gamma":"0.00912368763","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.036102999","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00912368763","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.603316069"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json index b569c8da7..a66b14f3e 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-azure","platform_databricks-aws","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.6600735E-2,-4.183997E-1,4.016022E-1,-5.4472244E-1,2.4702572E-3,-3.0444106E-1,5.4333526E-1,-2.095471E-1,-6.9834214E-1,1.3250048E-2,-7.2323172E-3,-6.7226007E-4,-1.9111963E-2,3.2672933E-1,8.139915E-1,-1.4351901E-2,3.8755343E-3,-2.7703207E-2,-1.1876688E-2,-2.0970795E-3,4.2583746E-1,3.311289E-2,1.6418E-2,2.9950497E-1,2.2299686E-2,5.3400216E-3,1.2468227E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,19,21,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,-1],"loss_changes":[1.7139414E1,2.7659492E0,5.1688957E0,1.9132452E0,1.0345602E0,5.687549E-1,2.1963263E0,8.766462E-1,4.3567657E-1,0E0,0E0,0E0,0E0,9.781935E-1,2.5014305E-1,0E0,0E0,0E0,0E0,0E0,2.9969501E-1,0E0,0E0,4.6219945E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,13,13,14,14,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,20,22,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,-1],"split_conditions":[3.3433047E-1,2.561146E3,2.80369E5,4.828443E-1,2.5927516E11,6.313997E-1,1.5364555E0,1.3564575E5,1.1653038E-2,1.3250048E-2,-7.2323172E-3,-6.7226007E-4,-1.9111963E-2,8.184232E5,2.326E4,-1.4351901E-2,3.8755343E-3,-2.7703207E-2,-1.1876688E-2,-2.0970795E-3,1.5791555E9,3.311289E-2,1.6418E-2,1.8589372E5,2.2299686E-2,5.3400216E-3,1.2468227E-2],"split_indices":[40,53,1,28,32,39,43,29,39,0,0,0,0,33,9,0,0,0,0,0,7,0,0,34,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.1E1,4.9E1,3.9E1,1.2E1,8E0,4.1E1,1.3E1,2.6E1,4E0,8E0,4E0,4E0,2.4E1,1.7E1,8E0,5E0,2.1E1,5E0,5E0,1.9E1,1.2E1,5E0,1.3E1,6E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[3.9494306E-2,-3.070603E-1,6.038862E-1,-4.283072E-1,2.940214E-1,8.1463903E-1,2.2642255E-1,-5.962956E-1,-1.721278E-1,1.8948603E-2,3.146083E-3,1.4225878E-2,9.0458536E-1,3.1656706E-1,-6.202453E-4,-6.8261397E-1,-4.4358503E-3,-1.1051375E-2,-5.5826362E-2,1.6590422E-2,3.583697E-2,4.103856E-3,1.5028562E-2,-7.388364E-3,-7.60215E-1,1.989614E-3,-7.0197335E-3,-1.7825114E-2,-3.1670492E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[1.9549099E1,4.644448E0,2.8385057E0,2.135189E0,4.7955102E-1,5.5152035E-1,3.3482993E-1,1.1829653E0,3.2571435E-1,0E0,0E0,0E0,2.883768E-1,1.7520332E-1,0E0,8.087826E-1,0E0,0E0,2.1061364E-1,0E0,0E0,0E0,0E0,0E0,2.5288773E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,18,18,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,3.5324219E3,1.4046429E1,1.1816701E-2,6.15786E5,2.1146134E5,1.2707124E1,2.2789746E3,1.2239595E3,1.8948603E-2,3.146083E-3,1.4225878E-2,3.8902016E8,7.765E3,-6.202453E-4,4.029937E-1,-4.4358503E-3,-1.1051375E-2,4.5843E7,1.6590422E-2,3.583697E-2,4.103856E-3,1.5028562E-2,-7.388364E-3,4.307821E7,1.989614E-3,-7.0197335E-3,-1.7825114E-2,-3.1670492E-2],"split_indices":[40,53,55,39,30,29,54,53,4,0,0,0,7,2,0,28,0,0,46,0,0,0,0,0,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.1E1,3.7E1,5.1E1,1E1,2.3E1,1.4E1,3E1,2.1E1,4E0,6E0,5E0,1.8E1,1E1,4E0,2.5E1,5E0,9E0,1.2E1,4E0,1.4E1,4E0,6E0,4E0,2.1E1,7E0,5E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.488806E-2,-3.7544534E-1,4.4622344E-1,-4.5515296E-1,1.4603574E-2,-3.0572563E-2,6.548415E-1,-2.9288656E-1,-7.6341426E-1,-1.2825903E-2,1.6137537E-1,4.3922216E-1,8.377188E-1,-2.4863131E-2,-1.7758082E-1,-1.2843414E-2,-3.2234408E-2,1.3643051E-2,2.1229606E-4,8.309624E-3,1.9282134E-2,2.190433E-2,3.800347E-2,-3.5284942E-1,-9.7808175E-2,-5.4097087E-3,-1.7520351E-2,4.9898024E-3,-1.7535536E-1,-4.5112697E-3,-9.795604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1],"loss_changes":[1.9396439E1,3.851984E0,5.5902433E0,2.4775124E0,0E0,1.1791408E0,1.169899E0,1.5662498E0,6.3215256E-1,0E0,4.0178597E-1,2.7371645E-1,3.542862E-1,0E0,3.8493752E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6925931E-1,4.2689222E-1,0E0,0E0,0E0,4.8322022E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,23,23,24,24,28,28],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1],"split_conditions":[1.5363757E8,7.621429E1,5.1707494E-1,5.070136E7,1.4603574E-2,3.0962205E-2,7.3845965E8,1.497E3,5.3323513E-1,-1.2825903E-2,2E0,1.218666E0,2.5172096E-1,-2.4863131E-2,1E0,-1.2843414E-2,-3.2234408E-2,1.3643051E-2,2.1229606E-4,8.309624E-3,1.9282134E-2,2.190433E-2,3.800347E-2,6.9533944E7,2.9000288E-2,-5.4097087E-3,-1.7520351E-2,4.9898024E-3,1.0128521E6,-4.5112697E-3,-9.795604E-3],"split_indices":[12,57,40,46,0,39,7,2,28,0,8,40,39,0,17,0,0,0,0,0,0,0,0,7,39,0,0,0,48,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.8E1,5.5E1,5.3E1,5E0,1.7E1,3.8E1,3.6E1,1.7E1,6E0,1.1E1,1.9E1,1.9E1,7E0,2.9E1,5E0,1.2E1,4E0,7E0,7E0,1.2E1,1.1E1,8E0,8E0,2.1E1,4E0,4E0,5E0,1.6E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-4.2027324E-2,-3.7183625E-1,5.0865483E-1,-1.9803534E-1,-6.767564E-1,1.4365476E-1,8.339433E-1,-5.00647E-1,7.6184355E-2,-7.7971935E-1,-1.0852783E-2,1.6833028E-2,1.3703063E-2,1.730201E-2,3.6189623E-2,-7.088462E-1,-1.7992336E-1,-1.0535246E-1,3.168087E-1,-3.1054046E-2,-1.6682882E-2,-6.622137E-3,1.5032132E-1,-1.8500404E-2,-3.3237103E-2,-2.6283695E-3,-9.837503E-3,6.741438E-2,-1.1279693E-2,1.9201145E-2,4.2584944E-3,1.3669189E-3,7.905985E-3,5.1801493E-3,-7.9919887E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0348944E1,3.575656E0,4.8668747E0,3.855317E0,7.338495E-1,8.739692E-1,8.3414936E-1,1.3414993E0,1.125431E0,8.793354E-2,0E0,0E0,4.5846978E-1,0E0,0E0,6.897545E-2,7.9315215E-2,5.577067E-1,4.0763426E-1,0E0,0E0,0E0,6.892693E-2,0E0,0E0,0E0,0E0,6.448403E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,22,22,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2310118E7,5.9633E7,1.2480776E0,1.3486356E3,3.5904072E3,1.5306258E5,4.867052E6,1.977643E6,9E0,3.1346153E1,-1.0852783E-2,1.6833028E-2,6.919127E5,1.730201E-2,3.6189623E-2,7.472597E7,1.1954737E1,3.863112E7,4.3650795E-2,-3.1054046E-2,-1.6682882E-2,-6.622137E-3,4.340646E-1,-1.8500404E-2,-3.3237103E-2,-2.6283695E-3,-9.837503E-3,5.526E3,-1.1279693E-2,1.9201145E-2,4.2584944E-3,1.3669189E-3,7.905985E-3,5.1801493E-3,-7.9919887E-4],"split_indices":[51,46,40,4,4,29,1,49,8,59,0,0,33,0,0,7,55,46,58,0,0,0,40,0,0,0,0,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,6.9E1,4.1E1,4.5E1,2.4E1,2E1,2.1E1,2.1E1,2.4E1,1.8E1,6E0,5E0,1.5E1,8E0,1.3E1,1.2E1,9E0,1.4E1,1E1,1.3E1,5E0,6E0,9E0,8E0,4E0,5E0,4E0,8E0,6E0,4E0,6E0,4E0,5E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.0581495E-2,-3.905329E-1,5.229766E-1,-2.1395065E-1,-6.927597E-1,2.7906787E-1,7.874607E-1,-3.7685356E-1,1.7368104E-1,-1.1198943E-2,-2.8782893E-2,4.6702987E-1,5.5180494E-2,1.8140873E-2,8.869554E-1,-6.437481E-1,-1.6036002E-1,-6.7222877E-3,3.9430216E-1,1.015999E-2,2.2110943E-2,-5.6470808E-2,8.849449E-3,2.1888694E-2,3.8152087E-2,-2.8269665E-2,-1.5936205E-2,-2.3753755E-1,2.1222164E-4,2.22954E-2,3.328482E-3,-4.3679755E-3,6.981977E-4,-3.8479902E-3,-1.024952E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.33296E1,3.5341682E0,2.812214E0,2.907037E0,7.5199986E-1,1.0683694E0,1.7812729E-1,1.745758E0,1.1849732E0,0E0,0E0,2.0551062E-1,2.8952807E-1,0E0,7.4178696E-2,4.890871E-2,2.4516359E-1,0E0,5.5007005E-1,0E0,0E0,4.6358094E-2,0E0,0E0,0E0,0E0,0E0,4.866296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,18,18,21,21,27,27],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,5.4875424E7,1.3536367E0,1.0664373E3,5.3323513E-1,9.48E2,3.1E1,2.667836E6,1E0,-1.1198943E-2,-2.8782893E-2,4.903324E1,6.9015497E-1,1.8140873E-2,1.9088202E0,2.3E1,2.3514317E8,-6.7222877E-3,9.070543E0,1.015999E-2,2.2110943E-2,2.1E1,8.849449E-3,2.1888694E-2,3.8152087E-2,-2.8269665E-2,-1.5936205E-2,2.5445784E1,2.1222164E-4,2.22954E-2,3.328482E-3,-4.3679755E-3,6.981977E-4,-3.8479902E-3,-1.024952E-2],"split_indices":[12,46,40,53,28,0,3,33,17,0,0,57,40,0,43,3,7,0,54,0,0,8,0,0,0,0,0,59,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6.8E1,4.6E1,4.4E1,2.4E1,2.5E1,2.1E1,3.1E1,1.3E1,6E0,1.8E1,1.3E1,1.2E1,7E0,1.4E1,1.3E1,1.8E1,5E0,8E0,7E0,6E0,8E0,4E0,7E0,7E0,6E0,7E0,1.2E1,6E0,4E0,4E0,4E0,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.979678E-3,-3.4272724E-1,4.9574625E-1,-5.276246E-1,-1.013813E-1,2.4871044E-1,7.319686E-1,-7.0078516E-1,-2.0545907E-1,1.7967814E-1,-2.8312197E-1,-2.1515675E-2,3.8516822E-1,2.9084628E-2,1.6195044E-2,-1.3636926E-2,-2.7782705E-2,-1.0596163E-2,-3.7093132E-3,-3.6961345E-3,2.1482423E-2,-3.5835654E-1,-2.632875E-3,-6.17006E-3,4.7718557E-3,1.934534E-2,2.4118295E-1,-6.990057E-3,-1.480449E-2,4.890936E-3,1.0436601E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.8815786E1,2.8024602E0,2.5022974E0,1.8643255E0,1.5321889E0,8.9327407E-1,1.1856079E-1,2.2020626E-1,9.644824E-2,1.466751E0,2.5851727E-1,2.2921947E-1,2.1787167E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.1342387E-2,0E0,0E0,0E0,0E0,1.1469007E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,21,21,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[1.8147125E5,3.3041668E1,1.3536367E0,2.9282052E6,6.15786E5,4.7308203E3,2.1685E4,8.977316E0,2.207872E-1,3.09E2,2.7938022E3,4.226681E-1,6.19999E9,2.9084628E-2,1.6195044E-2,-1.3636926E-2,-2.7782705E-2,-1.0596163E-2,-3.7093132E-3,-3.6961345E-3,2.1482423E-2,2.9673992E9,-2.632875E-3,-6.17006E-3,4.7718557E-3,1.934534E-2,5.5918517E0,-6.990057E-3,-1.480449E-2,4.890936E-3,1.0436601E-2],"split_indices":[34,57,40,33,30,4,9,55,40,0,53,40,5,0,0,0,0,0,0,0,0,5,0,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,6.3E1,4.5E1,3.5E1,2.8E1,2.3E1,2.2E1,2.2E1,1.3E1,1.1E1,1.7E1,8E0,1.5E1,1.6E1,6E0,5E0,1.7E1,6E0,7E0,7E0,4E0,1.2E1,5E0,4E0,4E0,6E0,9E0,4E0,8E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.1256974E-2,-2.8215232E-1,5.441066E-1,-4.1236064E-1,5.7476006E-2,1.9641325E-4,6.3900536E-1,-2.2134565E-1,-6.4540946E-1,1.152904E-2,-1.0809073E-1,3.4943733E-1,8.0162185E-1,-3.7772104E-1,-1.19162746E-1,-1.0983238E-2,-2.6278079E-2,-6.874574E-3,7.108995E-4,1.5086514E-2,5.151032E-3,2.1407075E-2,3.850762E-2,-1.6439706E-2,-8.56121E-3,2.1845386E-3,-1.7110898E-1,-8.0796555E-3,-7.5740006E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.7286366E1,3.075047E0,2.041273E0,2.0829325E0,9.016283E-1,0E0,1.3208942E0,4.1837752E-1,3.9748955E-1,0E0,1.4050426E-1,1.384809E-1,5.2877617E-1,1.2079835E-2,1.8768522E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1395046E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[5.1707494E-1,2.5323425E3,1.1462246E3,4.851587E7,6.15786E5,1.9641325E-4,1.5083523E0,7.8714124E2,5.3323513E-1,1.152904E-2,1.7802E4,5.5E1,1.5605432E9,1.44E2,1.77E2,-1.0983238E-2,-2.6278079E-2,-6.874574E-3,7.108995E-4,1.5086514E-2,5.151032E-3,2.1407075E-2,3.850762E-2,-1.6439706E-2,-8.56121E-3,2.1845386E-3,4.01E2,-8.0796555E-3,-7.5740006E-4],"split_indices":[40,53,53,46,30,0,43,4,28,0,10,3,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.8E1,3.9E1,4.9E1,1.9E1,6E0,3.3E1,2.8E1,2.1E1,7E0,1.2E1,1.3E1,2E1,1E1,1.8E1,5E0,1.6E1,7E0,5E0,9E0,4E0,1.3E1,7E0,5E0,5E0,4E0,1.4E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.717804E-2,-3.7835562E-1,4.055E-1,-4.9538177E-1,5.748244E-3,1.4678942E-1,2.4617178E-2,-6.4040095E-1,-2.3421851E-1,1.187477E-2,-1.8872583E-1,3.5689092E-1,1.943298E-2,-1.0111054E-2,-6.868529E-1,-1.2533823E-2,-8.269428E-2,-1.2363759E-2,1.2093743E-3,5.453824E-3,1.7738875E-2,-1.2941217E-1,6.659366E-3,-1.4572546E-2,-2.7964214E-2,6.718205E-4,-6.0457406E-3,-1.0731786E-2,2.3218838E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,23,-1,25,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5594854E1,2.6998606E0,3.0457854E0,1.5921955E0,1.0047839E0,6.352512E-1,0E0,2.9305458E-1,2.7935505E-1,0E0,3.531009E-1,1.7483854E-1,4.1687348E-1,0E0,3.0840778E-1,0E0,8.039741E-2,0E0,0E0,0E0,0E0,3.117548E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,14,14,16,16,21,21],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,24,-1,26,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,2.667836E6,6.15786E5,1.1E1,2.4617178E-2,2.3452184E5,1.6757976E-1,1.187477E-2,2.4502628E6,1.0262E4,2.3342845E5,-1.0111054E-2,4.4E1,-1.2533823E-2,1.675502E8,-1.2363759E-2,1.2093743E-3,5.453824E-3,1.7738875E-2,1.4634906E-1,6.659366E-3,-1.4572546E-2,-2.7964214E-2,6.718205E-4,-6.0457406E-3,-1.0731786E-2,2.3218838E-3],"split_indices":[12,2,40,33,30,8,0,29,40,0,33,2,34,0,10,0,5,0,0,0,0,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.9E1,4.3E1,4.5E1,1.4E1,2.3E1,2E1,2.8E1,1.7E1,5E0,9E0,8E0,1.5E1,4E0,2.4E1,9E0,8E0,5E0,4E0,4E0,4E0,8E0,7E0,7E0,1.7E1,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.0237521E-2,-3.393916E-1,3.3153063E-1,-4.1472912E-1,1.794064E-2,1.8513696E-1,5.837542E-1,-5.59573E-1,-1.6224243E-1,-9.584433E-3,2.518441E-1,1.3587898E-2,2.431059E-2,-6.446323E-1,-5.9100497E-3,-2.562468E-1,3.6034028E-3,3.3394757E-1,1.4041496E-2,-1.2825363E-2,-7.3255205E-1,-1.2800422E-2,-1.513147E-1,3.880833E-1,2.2182493E-3,-4.391941E-3,5.3044334E-3,-3.0928945E-2,-1.80312E-2,-7.1180547E-3,-2.7151913E-3,2.967635E-1,2.109953E-2,1.3289075E-2,7.5208065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1],"loss_changes":[1.2149417E1,3.6714392E0,1.8488941E0,1.7743425E0,0E0,1.1189239E0,9.96809E-2,9.720001E-1,5.1825535E-1,0E0,6.005678E-1,0E0,0E0,4.012823E-1,0E0,1.010828E-1,0E0,3.1545758E-1,1.8015428E-1,0E0,1.219511E-1,0E0,1.657471E-2,1.6723347E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7949343E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,15,15,17,17,18,18,20,20,22,22,23,23,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1],"split_conditions":[2.0020565E8,1.0269E4,1.3536367E0,2.9282052E6,1.794064E-2,3.0962205E-2,4.1372877E8,3.159013E3,2.1816934E8,-9.584433E-3,1.8634315E1,1.3587898E-2,2.431059E-2,4.4E1,-5.9100497E-3,2.1225102E-1,3.6034028E-3,2.3152558E9,1.5959719E-1,-1.2825363E-2,1.2599278E1,-1.2800422E-2,2.862E3,1.62739E9,2.2182493E-3,-4.391941E-3,5.3044334E-3,-3.0928945E-2,-1.80312E-2,-7.1180547E-3,-2.7151913E-3,2.0246488E5,2.109953E-2,1.3289075E-2,7.5208065E-3],"split_indices":[12,2,40,33,0,39,7,4,7,0,55,0,0,10,0,40,0,7,39,0,54,0,2,7,0,0,0,0,0,0,0,29,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,5.4E1,5.2E1,5E1,4E0,3.4E1,1.8E1,3.1E1,1.9E1,4E0,3E1,7E0,1.1E1,2.5E1,6E0,1.4E1,5E0,2.2E1,8E0,7E0,1.8E1,6E0,8E0,1.8E1,4E0,4E0,4E0,1E1,8E0,4E0,4E0,1.4E1,4E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-6.7869793E-3,-3.906748E-1,4.028442E-1,-5.969478E-1,-7.2045274E-2,2.2578067E-1,7.419192E-1,-3.3789346E-1,-2.5150344E-2,-2.10561E-1,7.7721947E-3,4.3038604E-1,5.4099772E-2,1.792945E-2,3.229338E-2,-5.669667E-3,-1.56068895E-2,-8.948621E-3,-3.3880319E-3,4.431246E-3,5.4117095E-1,-1.1790051E-1,1.0247393E-2,2.496625E-2,9.116291E-3,-6.723284E-3,4.8160528E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1,-1],"loss_changes":[1.4624603E1,3.1117768E0,2.5341916E0,5.134649E-1,8.413062E-1,1.0656481E0,1.1712742E-1,1.0282707E-1,0E0,3.76938E-2,0E0,4.1073418E-1,7.4473435E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5941038E-1,8.986153E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,4.852701E6,1.3536367E0,6.072427E-1,9.63264E2,8.69374E5,1.5605432E9,4.80378E5,-2.5150344E-2,3.670858E1,7.7721947E-3,1.2288E4,2.3342845E5,1.792945E-2,3.229338E-2,-5.669667E-3,-1.56068895E-2,-8.948621E-3,-3.3880319E-3,4.431246E-3,1.9045439E5,1.8154767E5,1.0247393E-2,2.496625E-2,9.116291E-3,-6.723284E-3,4.8160528E-5],"split_indices":[2,33,40,28,53,30,7,12,0,59,0,2,34,0,0,0,0,0,0,0,34,29,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.1E1,4.7E1,4.4E1,2.8E1,1.9E1,3E1,1.4E1,9E0,1.9E1,1.3E1,6E0,1.3E1,1.7E1,7E0,7E0,4E0,5E0,9E0,4E0,4E0,9E0,1E1,7E0,5E0,4E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.1054016E-2,-3.1460762E-1,4.859552E-1,-4.4477397E-1,6.504634E-2,-9.317041E-3,5.7835984E-1,-5.9096265E-1,-2.7023336E-1,1.4754604E-2,-8.687175E-2,4.267692E-1,8.703059E-1,-3.6451545E-1,-2.4515454E-2,-3.1843606E-1,-2.0392803E-3,-9.375171E-3,8.218775E-3,9.218639E-3,1.83128E-2,1.7255593E-2,3.704989E-2,-7.613364E-3,-1.5589032E-2,-9.2998715E-3,-1.5438079E-2,-2.0880434E-3,3.099097E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6132267E1,3.5376215E0,2.830555E0,1.1926422E0,1.0273844E0,0E0,1.1775494E0,3.1094837E-1,2.4989724E-1,0E0,2.3218612E-1,2.1534109E-1,1.8244553E-1,1.2307167E-2,0E0,2.9345512E-2,0E0,0E0,5.6237247E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,9.066E3,2.80369E5,3.572374E6,6.52678E5,-9.317041E-3,1.62739E9,4.4E1,3.365958E8,1.4754604E-2,5.8163033E1,2.0681982E-1,2.0105577E-1,5.189068E-1,-2.4515454E-2,7.959073E-1,-2.0392803E-3,-9.375171E-3,1.662E3,9.218639E-3,1.83128E-2,1.7255593E-2,3.704989E-2,-7.613364E-3,-1.5589032E-2,-9.2998715E-3,-1.5438079E-2,-2.0880434E-3,3.099097E-3],"split_indices":[43,2,1,12,30,0,7,10,7,0,57,39,39,28,0,28,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7E1,3.8E1,5.2E1,1.8E1,4E0,3.4E1,2.7E1,2.5E1,5E0,1.3E1,2.4E1,1E1,9E0,1.8E1,2E1,5E0,4E0,9E0,9E0,1.5E1,4E0,6E0,4E0,5E0,1.5E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.0797309E-2,-3.6156452E-1,3.3161274E-1,-4.2126018E-1,5.285972E-3,1.06708296E-1,6.242183E-1,-5.598041E-1,-2.0221046E-1,-8.68231E-3,1.8969616E-1,4.364018E-1,3.2840192E-2,-3.724325E-1,-2.7545957E-2,-1.3955579E-2,-1.2728357E-1,2.8850928E-1,1.075685E-2,1.87067E-2,8.866908E-3,-4.663343E-1,-4.9896906E-3,6.6139596E-4,-7.289815E-3,3.6883238E-1,5.126675E-3,3.4905286E-3,-4.03236E-3,-2.0380057E-2,-1.04860645E-2,1.781412E-2,4.355333E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,-1,-1,23,25,27,-1,-1,29,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2902058E1,1.5778832E0,3.7502704E0,1.2524624E0,0E0,1.025262E0,9.4314003E-1,7.437258E-1,2.13225E-1,0E0,4.9198008E-1,1.3789701E-1,0E0,2.8745627E-1,0E0,0E0,1.667025E-1,1.5944874E-1,1.2653352E-1,0E0,0E0,2.5963306E-2,0E0,0E0,0E0,2.8063214E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,21,21,25,25],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,-1,-1,24,26,28,-1,-1,30,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4350427E8,2.7938022E3,1.3536367E0,2.667836E6,5.285972E-3,4.739085E-2,1.0861106E4,4.6268812E5,1E0,-8.68231E-3,5E1,1.2271712E1,3.2840192E-2,1.9121604E-4,-2.7545957E-2,-1.3955579E-2,2.1923357E1,1E0,3.7E1,1.87067E-2,8.866908E-3,3.8E1,-4.9896906E-3,6.6139596E-4,-7.289815E-3,1.4273524E-1,5.126675E-3,3.4905286E-3,-4.03236E-3,-2.0380057E-2,-1.04860645E-2,1.781412E-2,4.355333E-3],"split_indices":[12,53,40,33,0,39,4,33,17,0,3,55,0,39,0,0,59,101,8,0,0,3,0,0,0,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,4.9E1,5.7E1,4.4E1,5E0,3.3E1,2.4E1,2.6E1,1.8E1,6E0,2.7E1,1.6E1,8E0,1.5E1,1.1E1,4E0,1.4E1,1.7E1,1E1,1E1,6E0,1E1,5E0,5E0,9E0,1E1,7E0,6E0,4E0,5E0,5E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-3.439152E-2,-2.8745982E-1,4.0628415E-1,-4.0453276E-1,1.5489583E-1,1.832985E-2,5.115344E-1,-2.7192265E-1,-5.81067E-1,-2.951975E-2,1.6866421E-2,4.405638E-3,-2.5684275E-3,3.12952E-1,6.413871E-1,-1.4641315E-1,-3.4862357E-1,-6.277753E-1,-1.0037368E-2,2.4273365E-3,-3.408564E-3,1.3045562E-2,6.50366E-3,1.4951235E-2,3.0611156E-2,-8.457483E-3,2.7486004E-3,-8.842935E-3,-1.5185564E-2,-1.171555E-2,-2.4669606E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.283019E1,3.843176E0,1.7055788E0,1.1997871E0,9.352175E-1,1.0146322E-1,6.5023136E-1,2.858913E-1,1.6518116E-1,7.541191E-2,0E0,0E0,0E0,3.1309128E-2,5.405083E-1,3.1108278E-1,5.3738117E-2,1.1511612E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,2.7938022E3,1.118632E3,8.0073607E-1,6.082581E3,9.5032835E0,1.5941123E0,2.8399336E7,2.7240606E5,1.8345214E-1,1.6866421E-2,4.405638E-3,-2.5684275E-3,1.5366939E7,1.286E4,1.2108435E3,2.1227852E5,2.07E2,-1.0037368E-2,2.4273365E-3,-3.408564E-3,1.3045562E-2,6.50366E-3,1.4951235E-2,3.0611156E-2,-8.457483E-3,2.7486004E-3,-8.842935E-3,-1.5185564E-2,-1.171555E-2,-2.4669606E-2],"split_indices":[43,53,53,28,53,54,40,46,48,28,0,0,0,48,2,4,29,11,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,7.2E1,4.1E1,5.7E1,1.5E1,9E0,3.2E1,3.4E1,2.3E1,1E1,5E0,4E0,5E0,1.4E1,1.8E1,1.4E1,2E1,1.9E1,4E0,4E0,6E0,9E0,5E0,1E1,8E0,1E1,4E0,1E1,1E1,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.472108E-2,-2.642644E-1,4.434464E-1,-3.4340566E-1,6.654095E-2,2.0786707E-1,6.330183E-1,-5.238764E-1,-1.6358441E-1,1.0051563E-2,-3.6348896E-3,-3.4342918E-3,3.025779E-1,4.346444E-1,3.3842564E-2,-3.2459053E-1,-2.2928808E-2,-2.596711E-1,2.5730023E-2,2.8112214E-3,3.8971868E-1,1.8355465E-2,1.0401099E-2,-6.572296E-3,-1.4911974E-2,-1.3982144E-2,-8.9025676E-2,3.8719538E-3,-2.7884827E-3,1.7398123E-2,9.207346E-3,-5.392798E-3,-3.92571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3315104E1,1.6722131E0,1.9227409E0,1.5693779E0,4.8861796E-1,6.601082E-1,1.1020508E0,3.479581E-1,4.9782735E-1,0E0,0E0,0E0,3.025396E-1,3.7325144E-2,0E0,6.349945E-2,0E0,3.5750234E-1,9.22194E-2,0E0,2.609384E-2,0E0,0E0,0E0,0E0,0E0,4.082188E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,17,17,18,18,20,20,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[9.095874E6,2.5323425E3,1.5364555E0,2.667836E6,6.15786E5,4.1451712E5,1.5605432E9,1.08E2,2.0237808E8,1.0051563E-2,-3.6348896E-3,-3.4342918E-3,3.486353E6,2.326E4,3.3842564E-2,8.317631E-1,-2.2928808E-2,2.6326587E-2,9.242E3,2.8112214E-3,7.118011E-1,1.8355465E-2,1.0401099E-2,-6.572296E-3,-1.4911974E-2,-1.3982144E-2,2.68E2,3.8719538E-3,-2.7884827E-3,1.7398123E-2,9.207346E-3,-5.392798E-3,-3.92571E-4],"split_indices":[49,53,43,33,30,33,7,10,7,0,0,0,1,9,0,28,0,39,9,0,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.2E1,4.5E1,5E1,1.2E1,2.1E1,2.4E1,2.4E1,2.6E1,5E0,7E0,5E0,1.6E1,1.6E1,8E0,1E1,1.4E1,1.7E1,9E0,5E0,1.1E1,9E0,7E0,5E0,5E0,9E0,8E0,5E0,4E0,5E0,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.6608386E-3,-2.7908042E-1,3.9974752E-1,-3.936006E-1,3.6976933E-2,1.7811656E-1,6.047223E-1,-5.568649E-1,-1.6049762E-1,2.739653E-1,-1.1665881E-2,3.156085E-1,8.964504E-2,4.4029915E-1,3.1717613E-2,-6.084411E-1,-6.773321E-3,-2.2856256E-1,-4.6261054E-4,1.5954038E-3,1.7920634E-2,1.399342E-2,6.5165246E-3,-3.280731E-2,9.089672E-3,1.8156307E-2,1.0073523E-2,-1.3630825E-2,-7.147181E-1,-1.0780131E-2,-2.6625989E-3,-3.5189209E-3,9.583637E-4,-2.8507112E-2,-1.4154024E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2806435E1,2.4758105E0,2.0274415E0,1.8134947E0,1.7099975E0,2.6364613E-1,6.9367695E-1,4.3814564E-1,2.1821755E-1,5.913157E-1,0E0,1.7577112E-2,3.2590234E-1,3.132105E-2,0E0,3.4360504E-1,0E0,1.4060438E-1,0E0,0E0,0E0,0E0,0E0,4.096568E-2,0E0,0E0,0E0,0E0,7.706261E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.5364555E0,2.667836E6,6.1358623E9,1.7644464E5,1.5605432E9,3.0387878E3,3.167669E5,1.1653E4,-1.1665881E-2,1.0550945E2,3.9219662E5,2.0837E4,3.1717613E-2,2.63E2,-6.773321E-3,2.207872E-1,-4.6261054E-4,1.5954038E-3,1.7920634E-2,1.399342E-2,6.5165246E-3,4.152984E-1,9.089672E-3,1.8156307E-2,1.0073523E-2,-1.3630825E-2,1.2599278E1,-1.0780131E-2,-2.6625989E-3,-3.5189209E-3,9.583637E-4,-2.8507112E-2,-1.4154024E-2],"split_indices":[51,2,43,33,5,29,7,4,29,2,0,58,29,9,0,11,0,40,0,0,0,0,0,40,0,0,0,0,54,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,6.7E1,4.6E1,4.9E1,1.8E1,2.3E1,2.3E1,2.8E1,2.1E1,1.1E1,7E0,8E0,1.5E1,1.6E1,7E0,2.4E1,4E0,1.4E1,7E0,6E0,5E0,4E0,4E0,9E0,6E0,1E1,6E0,9E0,1.5E1,9E0,5E0,4E0,5E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.1170119E-2,-2.286945E-1,2.9556155E-1,-2.757291E-1,9.817417E-3,7.455043E-2,5.5917764E-1,-4.1882515E-1,-1.4190349E-1,1.7954913E-1,-1.20397314E-1,2.1995092E-2,1.0729085E-2,-5.0299E-1,-2.1702932E-1,-2.541205E-1,-2.7940229E-2,4.6226624E-4,2.7831575E-1,-6.8809628E-3,-9.431044E-4,-8.674969E-3,-6.057556E-1,-2.6501452E-3,-1.0850561E-2,-1.1013974E-2,-4.2730784E-3,-1.1090766E-1,2.861581E-3,1.2885317E-2,4.6336856E-3,-2.5775805E-2,-1.3016385E-2,-1.4181143E-3,-6.306461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9691954E0,1.581959E0,2.4053206E0,1.0704117E0,0E0,5.128795E-1,6.811571E-2,3.6639166E-1,4.021356E-1,2.5993794E-1,5.457996E-2,0E0,0E0,3.6739302E-1,9.788659E-2,7.456696E-2,1.5999314E-1,0E0,7.2051585E-2,0E0,0E0,0E0,8.751106E-2,0E0,0E0,0E0,0E0,3.8817257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,22,22,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.379017E6,1.2424786E2,1.3589408E0,2.667836E6,9.817417E-3,5.6E1,2.1383E4,2.6631096E3,2.834008E-1,1.1015E4,1.0448E4,2.1995092E-2,1.0729085E-2,4.4E1,8.661788E7,2.942787E6,1.3448806E8,4.6226624E-4,2.581938E2,-6.8809628E-3,-9.431044E-4,-8.674969E-3,1.2599278E1,-2.6501452E-3,-1.0850561E-2,-1.1013974E-2,-4.2730784E-3,2.4E1,2.861581E-3,1.2885317E-2,4.6336856E-3,-2.5775805E-2,-1.3016385E-2,-1.4181143E-3,-6.306461E-3],"split_indices":[52,57,43,33,0,3,9,56,58,2,9,0,0,10,46,52,7,0,57,0,0,0,54,0,0,0,0,3,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,6.3E1,4.1E1,5.8E1,5E0,2.3E1,1.8E1,2.7E1,3.1E1,1.5E1,8E0,1.4E1,4E0,1.8E1,9E0,1.5E1,1.6E1,6E0,9E0,4E0,4E0,6E0,1.2E1,4E0,5E0,1E1,5E0,9E0,7E0,5E0,4E0,7E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6760997E-2,-1.9793408E-1,2.6402497E-1,-2.781832E-1,2.7125448E-1,-4.521447E-3,3.2001278E-1,-1.6120237E-1,-4.5164436E-1,1.738997E-2,1.8301925E-3,1.1050596E-3,3.8652018E-1,-2.1479665E-1,-2.4742452E-2,-5.228169E-1,-3.794221E-3,2.1083939E-1,4.408205E-1,-2.7492607E-1,-7.7352576E-2,3.2140955E-3,-4.8517697E-3,-1.2521984E-2,-2.1779887E-2,4.294831E-3,9.40665E-3,4.914743E-1,9.4613675E-3,-1.109944E-2,-4.8886845E-3,9.531097E-4,-5.979898E-3,1.0108562E-2,2.0295251E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,21,23,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4445705E0,2.5228367E0,9.554236E-1,1.0635252E0,4.3969083E-1,0E0,7.003269E-1,2.506569E-1,4.876814E-1,0E0,0E0,0E0,1.932435E-1,1.8534458E-1,1.4917842E-1,4.5605183E-2,0E0,1.0111302E-2,7.040262E-2,3.548169E-2,8.680768E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.1743956E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,22,24,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,5.188755E3,3.727E3,8.317631E-1,8.69374E5,-4.521447E-3,1E0,7.287E3,3.271557E8,1.738997E-2,1.8301925E-3,1.1050596E-3,1.2756285E3,2.834008E-1,4.5843E7,4.4815472E5,-3.794221E-3,1.2958418E5,1.0007143E1,3.7712722E6,3.39E2,3.2140955E-3,-4.8517697E-3,-1.2521984E-2,-2.1779887E-2,4.294831E-3,9.40665E-3,4.0383283E8,9.4613675E-3,-1.109944E-2,-4.8886845E-3,9.531097E-4,-5.979898E-3,1.0108562E-2,2.0295251E-2],"split_indices":[43,4,2,28,30,0,102,2,7,0,0,0,53,58,46,29,0,34,54,51,0,0,0,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.4E1,4.1E1,5.5E1,9E0,5E0,3.6E1,3.4E1,2.1E1,4E0,5E0,7E0,2.9E1,2.4E1,1E1,1.7E1,4E0,8E0,2.1E1,1.6E1,8E0,5E0,5E0,7E0,1E1,4E0,4E0,1.5E1,6E0,1.2E1,4E0,4E0,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7335935E-2,-2.5816494E-1,2.9193032E-1,-3.4373537E-1,1.05630405E-1,1.6434705E-1,6.829365E-1,-4.440817E-1,-1.3462842E-1,-3.7841632E-3,1.2042006E-2,-2.8410375E-3,2.1557806E-1,1.4404345E-2,2.845641E-2,-4.9313763E-1,-2.9081749E-3,-9.198718E-3,-2.5035532E-2,2.7649057E-1,-2.6771717E-3,-5.622826E-1,-1.1280322E-2,2.1476792E-3,-4.3849307E-3,3.290193E-3,3.435109E-1,-1.0248506E-2,-2.2297278E-2,1.6219167E-2,7.725943E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[7.8213167E0,1.8729858E0,2.18251E0,9.4740343E-1,6.105282E-1,4.652062E-1,1.4320374E-2,5.309415E-1,2.174935E-1,0E0,0E0,0E0,5.4995143E-1,0E0,0E0,1.7414093E-1,0E0,0E0,8.87228E-2,2.7926195E-1,0E0,1.17313385E-1,0E0,0E0,0E0,0E0,1.5035939E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,18,18,19,19,21,21,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[2.3724032E8,2.561146E3,1.8614753E0,3.2936362E6,1E0,2.2843675E-1,7.4684826E8,7.33E3,2.9544958E-1,-3.7841632E-3,1.2042006E-2,-2.8410375E-3,2.1972223E2,1.4404345E-2,2.845641E-2,2.9282052E6,-2.9081749E-3,-9.198718E-3,3.51387E5,8.784E3,-2.6771717E-3,2.63E2,-1.1280322E-2,2.1476792E-3,-4.3849307E-3,3.290193E-3,1.5625807E1,-1.0248506E-2,-2.2297278E-2,1.6219167E-2,7.725943E-3],"split_indices":[7,53,40,51,17,40,7,2,42,0,0,0,58,0,0,33,0,0,9,2,0,11,0,0,0,0,55,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,5.8E1,4.5E1,4.7E1,1.1E1,3.5E1,1E1,3.1E1,1.6E1,6E0,5E0,6E0,2.9E1,4E0,6E0,2.7E1,4E0,7E0,9E0,2.4E1,5E0,1.8E1,9E0,5E0,4E0,7E0,1.7E1,4E0,1.4E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5175937E-2,-1.7119141E-1,4.021599E-1,-2.3383586E-1,1.771783E-1,-9.246658E-3,4.8470944E-1,-9.42045E-2,-3.9910802E-1,1.671502E-2,-8.5188553E-4,1.0748329E-2,5.9519696E-1,-3.2642293E-1,7.6650004E-3,-1.8785229E-2,-2.3206866E-1,3.7581056E-1,2.8844167E-2,-1.50354775E-2,-6.5700733E-3,1.6668053E-1,-7.22686E-2,-1.0507577E-2,-4.74809E-3,1.5626846E-2,7.5599775E-3,9.283002E-3,1.5487989E-3,-1.13182046E-1,2.048584E-3,-4.8094895E-3,-1.8224122E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,21,-1,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[8.433935E0,1.6781878E0,2.2434268E0,1.4393415E0,6.995077E-1,0E0,5.459299E-1,8.5411704E-1,4.6643257E-1,0E0,0E0,0E0,6.3842154E-1,6.726909E-2,3.4252357E-1,0E0,4.47464E-2,2.3247719E-2,0E0,0E0,0E0,8.972788E-2,1.0143166E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0862216E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,14,14,16,16,17,17,21,21,22,22,29,29],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,22,-1,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[1.0230335E0,3.5324219E3,1.4960615E3,5.070136E7,5.4875424E7,-9.246658E-3,1.7908138E0,7.998101E2,3.0595828E3,1.671502E-2,-8.5188553E-4,1.0748329E-2,2.463E4,1.54E2,5.83E2,-1.8785229E-2,3.4707712E8,1.2257767E1,2.8844167E-2,-1.50354775E-2,-6.5700733E-3,9.20242E0,4.7992307E-1,-1.0507577E-2,-4.74809E-3,1.5626846E-2,7.5599775E-3,9.283002E-3,1.5487989E-3,4.01E2,2.048584E-3,-4.8094895E-3,-1.8224122E-3],"split_indices":[40,53,4,46,46,0,43,4,4,0,0,0,2,0,10,0,7,55,0,0,0,54,40,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.4E1,3.8E1,6.3E1,1.1E1,4E0,3.4E1,3.5E1,2.8E1,4E0,7E0,1.4E1,2E1,1E1,2.5E1,1.5E1,1.3E1,1.1E1,9E0,5E0,5E0,8E0,1.7E1,7E0,6E0,7E0,4E0,4E0,4E0,1.3E1,4E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.2068253E-2,-1.648847E-1,3.3613706E-1,-3.5189933E-1,5.7612408E-2,4.1618764E-1,2.6709302E-3,-4.3274614E-1,-1.9394387E-1,2.1607071E-1,-2.1128578E-1,2.1955945E-2,3.239555E-1,-2.0425077E-1,-1.9531492E-2,-9.119429E-3,-1.3666184E-3,-1.1566876E-3,2.8010044E-1,-1.1187661E-2,-4.449788E-3,6.5825083E-3,1.3277747E-2,-1.1341023E-2,-1.9322936E-3,1.3200576E-2,4.2562406E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.057305E0,2.7665477E0,5.777972E-1,3.7579823E-1,1.3665363E0,1.9249415E-1,0E0,4.58385E-1,1.1860639E-1,3.267014E-1,6.596291E-2,0E0,3.84475E-2,1.3224477E-1,0E0,0E0,0E0,0E0,1.7888987E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,6.363496E7,3.143873E6,2.9282052E6,2.7070812E7,1.9718067E5,2.6709302E-3,4.5E1,4E1,8.873786E0,1.9452468E-2,2.1955945E-2,3.9E1,1.7E2,-1.9531492E-2,-9.119429E-3,-1.3666184E-3,-1.1566876E-3,2.26182E5,-1.1187661E-2,-4.449788E-3,6.5825083E-3,1.3277747E-2,-1.1341023E-2,-1.9322936E-3,1.3200576E-2,4.2562406E-3],"split_indices":[53,12,30,33,46,34,0,10,3,57,39,0,3,0,0,0,0,0,30,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.3E1,6.5E1,2.8E1,3.5E1,3E1,2.1E1,7E0,2.2E1,1.3E1,1.9E1,1.1E1,5E0,1.6E1,8E0,1.4E1,9E0,4E0,4E0,1.5E1,4E0,7E0,5E0,1.1E1,4E0,4E0,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.045447E-3,-1.3045764E-1,5.493323E-1,-2.3233856E-1,1.6364071E-1,1.0528478E-2,6.968363E-1,-4.1000903E-1,-1.4423028E-1,-1.806817E-3,2.225081E-1,2.9008435E-2,1.39654195E-2,-4.578551E-1,-5.6234505E-3,-7.763605E-2,-3.319938E-1,2.806637E-1,1.795295E-3,-8.949189E-3,-5.195474E-1,-1.8612137E-1,-1.3987516E-4,-1.6001105E-2,-7.978266E-3,5.450663E-3,1.146059E-2,-1.0650814E-2,-2.1176642E-2,-8.391662E-3,-3.6097334E-3,6.374891E-3,-7.7443935E-2,-4.1714245E-3,1.0643514E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,19,-1,21,23,25,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,33,-1,-1],"loss_changes":[7.5769463E0,2.764654E0,5.5704784E-1,1.0053525E0,3.1303632E-1,0E0,8.5967064E-2,1.9630885E-1,5.6191164E-1,0E0,1.7695892E-1,0E0,0E0,1.0438442E-1,0E0,3.0263305E-1,5.020356E-2,1.8849254E-2,0E0,0E0,2.2569895E-2,3.660202E-2,3.1421196E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.858496E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,13,13,15,15,16,16,17,17,20,20,21,21,22,22,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,20,-1,22,24,26,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,34,-1,-1],"split_conditions":[6.0880933E3,6.402099E-1,1.3127055E4,1.6755434E1,8.873786E0,1.0528478E-2,1.890618E6,1.8839418E3,7.752181E7,-1.806817E-3,4.5E1,2.9008435E-2,1.39654195E-2,1.16E2,-5.6234505E-3,2.8866995E1,3.0595828E3,1.601247E1,1.795295E-3,-8.949189E-3,5.432393E-1,1.3101191E1,3.9709E5,-1.6001105E-2,-7.978266E-3,5.450663E-3,1.146059E-2,-1.0650814E-2,-2.1176642E-2,-8.391662E-3,-3.6097334E-3,6.374891E-3,4.6961885E3,-4.1714245E-3,1.0643514E-3],"split_indices":[53,43,4,59,57,0,30,53,46,0,3,0,0,0,0,57,4,57,0,0,28,57,30,0,0,0,0,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,6.7E1,2.3E1,8E0,1.1E1,2.1E1,4.6E1,5E0,1.8E1,7E0,4E0,1.7E1,4E0,3.5E1,1.1E1,1.3E1,5E0,5E0,1.2E1,1.4E1,2.1E1,4E0,7E0,4E0,9E0,4E0,8E0,8E0,6E0,6E0,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.197816E-2,-7.497011E-2,5.014067E-1,-2.3085347E-1,1.19796686E-1,5.8881146E-1,5.0151106E-3,-3.131732E-1,-2.3126926E-2,3.297368E-1,-9.704131E-3,1.1902153E-2,2.6172534E-2,-2.1767734E-1,-1.5210481E-2,-5.2955006E-3,4.8081987E-2,1.9302368E-2,4.8680156E-3,-1.1230028E-1,1.03101134E-1,-4.1702559E-4,-9.9552395E-3,3.2475363E-3,-1.2290914E-4,1.4748385E-3,-7.003339E-3,2.650139E-4,7.817953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.034423E0,2.3047233E0,5.2020025E-1,7.109568E-1,9.275719E-1,2.9015398E-1,0E0,2.1808338E-1,1.2142535E-1,4.4709122E-1,2.6619276E-1,0E0,0E0,2.1226358E-1,0E0,0E0,1.9707847E-2,0E0,0E0,1.580405E-1,1.17909305E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3536367E0,2.561146E3,2.326E4,2.6326587E-2,2.5927516E11,2.5172096E-1,5.0151106E-3,8.4395015E-1,1.5938355E8,8.012376E0,9.250217E5,1.1902153E-2,2.6172534E-2,8.1837E4,-1.5210481E-2,-5.2955006E-3,4.9457936E7,1.9302368E-2,4.8680156E-3,1.5457461E5,4.1822475E-1,-4.1702559E-4,-9.9552395E-3,3.2475363E-3,-1.2290914E-4,1.4748385E-3,-7.003339E-3,2.650139E-4,7.817953E-3],"split_indices":[40,53,9,39,32,39,0,28,7,54,33,0,0,12,0,0,33,0,0,29,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.2E1,7.4E1,1.8E1,4.1E1,3.3E1,1.4E1,4E0,2.9E1,1.2E1,1.2E1,2.1E1,6E0,8E0,1.7E1,1.2E1,4E0,8E0,5E0,7E0,1.1E1,1E1,4E0,1.3E1,4E0,4E0,4E0,7E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.737566E-4,-2.4148624E-1,2.8224903E-1,-3.1347293E-1,1.0660252E-1,5.9629556E-2,3.9899176E-1,-1.6329855E-1,-4.773247E-1,1.1325116E-2,-3.0231455E-3,-2.9234872E-3,1.6615129E-1,4.9787334E-1,2.0917995E-1,-3.1328753E-1,-5.934927E-2,-1.926247E-2,-1.0421838E-2,1.2407742E-3,8.963621E-3,2.0304255E-2,7.3911734E-3,3.2640689E-3,9.287992E-3,-6.30179E-3,-1.5059091E-2,4.346179E-3,-5.487278E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.8135777E0,1.3865364E0,1.1717119E0,1.0293384E0,4.2043418E-1,2.6807314E-1,4.5892143E-1,3.7118775E-1,6.87027E-2,0E0,0E0,0E0,9.9698305E-2,2.4425411E-1,4.5271814E-2,7.123941E-2,2.8470373E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6225724E-1,2.7938022E3,1.6492815E3,8.297822E-1,1.0803E4,2.0978074E8,1.4046429E1,1.0930736E1,1.0034782E-3,1.1325116E-2,-3.0231455E-3,-2.9234872E-3,1.8589372E5,2.57281E6,9.257902E-2,6.948189E7,2.083117E1,-1.926247E-2,-1.0421838E-2,1.2407742E-3,8.963621E-3,2.0304255E-2,7.3911734E-3,3.2640689E-3,9.287992E-3,-6.30179E-3,-1.5059091E-2,4.346179E-3,-5.487278E-3],"split_indices":[43,53,53,28,9,7,55,59,40,0,0,0,34,30,39,7,59,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,5.3E1,4.5E1,4.4E1,9E0,1.6E1,2.9E1,2.4E1,2E1,4E0,5E0,7E0,9E0,1.8E1,1.1E1,9E0,1.5E1,1.4E1,6E0,4E0,5E0,1.4E1,4E0,4E0,7E0,5E0,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.6093224E-2,-2.1192521E-1,2.9062074E-1,-4.335574E-1,-5.6446034E-2,1.5187746E-1,5.564244E-1,-4.9036043E-3,-4.83801E-1,-1.8762808E-1,1.0892272E-1,3.4369522E-1,1.8826129E-2,1.2425799E-2,2.488158E-2,-1.0767954E-2,-2.0046884E-2,-9.543987E-3,-1.3038243E-1,2.0427169E-1,-3.6077367E-3,3.526767E-3,1.9222017E-2,-9.245615E-2,1.2801662E-1,-2.9906249E-3,-5.9965374E-3,2.5470222E-3,9.470343E-3,1.3927397E-3,-7.4010394E-3,-6.437534E-4,8.962964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0079656E0,2.0775373E0,1.3537521E0,3.0827093E-1,8.230931E-1,6.8261296E-1,1.2911081E-1,0E0,1.1285877E-1,5.7770073E-2,3.5894716E-1,4.5870197E-1,2.1840233E-1,0E0,0E0,0E0,0E0,0E0,1.001364E-2,7.4634016E-2,0E0,0E0,0E0,1.4062831E-1,1.6226287E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2789746E3,2.667836E6,1.6957201E0,2.3452184E5,2.3514317E8,6.19999E9,2.463E4,-4.9036043E-3,2.9488824E2,1.7301265E6,2.63382E5,7.21E2,8.184232E5,1.2425799E-2,2.488158E-2,-1.0767954E-2,-2.0046884E-2,-9.543987E-3,9E0,9.489487E2,-3.6077367E-3,3.526767E-3,1.9222017E-2,5.6075206E0,4.4432E4,-2.9906249E-3,-5.9965374E-3,2.5470222E-3,9.470343E-3,1.3927397E-3,-7.4010394E-3,-6.437534E-4,8.962964E-3],"split_indices":[53,33,40,29,7,5,2,0,53,49,30,0,33,0,0,0,0,0,18,53,0,0,0,54,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6E1,3.8E1,2.4E1,3.6E1,2.6E1,1.2E1,4E0,2E1,2E1,1.6E1,1E1,1.6E1,6E0,6E0,7E0,1.3E1,7E0,1.3E1,1.1E1,5E0,5E0,5E0,8E0,8E0,7E0,6E0,4E0,7E0,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[5.2020777E-2,-1.21101394E-1,3.6666822E-1,-2.0498253E-1,1.4658998E-1,2.7944025E-2,2.8636226E-1,-1.4501555E-1,-3.592246E-1,2.809931E-1,1.0968173E-2,1.7762247E-2,2.072143E-1,-3.1797555E-1,-7.3385365E-2,-1.6689165E-2,-4.607437E-3,5.7943943E-3,1.24660535E-2,-2.0134482E-3,3.2081057E-3,1.1230931E-1,1.1426109E-2,-1.6463945E-2,-6.9298414E-3,-1.2628634E-1,6.995995E-2,4.787221E-2,6.1650337E-3,7.958925E-4,-1.5867427E-1,4.6231477E-3,-7.6772274E-5,-1.0806797E-3,4.1916743E-3,-6.5995953E-3,-1.969959E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[6.145319E0,1.6692088E0,1.0868273E0,4.6587515E-1,3.2502186E-1,0E0,4.6767354E-1,4.9700975E-1,2.9897773E-1,1.4311731E-2,5.693878E-2,0E0,2.4445832E-1,1.2124944E-1,2.4391133E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.208996E-2,0E0,0E0,0E0,1.1399335E-1,3.796269E-2,5.125404E-2,0E0,0E0,3.774631E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,21,21,25,25,26,26,27,27,30,30],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,2.8938947E8,1.6451234E-2,5.501383E9,5.46E2,2.7944025E-2,2.8129198E7,2.253E3,3.5904072E3,2.2659668E-1,3.9219662E5,1.7762247E-2,8.640353E3,4.0748124E7,2.3514317E8,-1.6689165E-2,-4.607437E-3,5.7943943E-3,1.24660535E-2,-2.0134482E-3,3.2081057E-3,1E0,1.1426109E-2,-1.6463945E-2,-6.9298414E-3,5.2287582E-2,7.0590184E7,8.18961E3,6.1650337E-3,7.958925E-4,3.5983E4,4.6231477E-3,-7.6772274E-5,-1.0806797E-3,4.1916743E-3,-6.5995953E-3,-1.969959E-3],"split_indices":[2,12,58,5,0,0,46,2,4,39,29,0,53,5,7,0,0,0,0,0,0,102,0,0,0,58,12,4,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,7.2E1,3.9E1,5.5E1,1.7E1,5E0,3.4E1,4.1E1,1.4E1,8E0,9E0,8E0,2.6E1,1.1E1,3E1,9E0,5E0,4E0,4E0,5E0,4E0,1.5E1,1.1E1,4E0,7E0,2.2E1,8E0,8E0,7E0,4E0,1.8E1,4E0,4E0,4E0,4E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.251468E-2,-1.2000842E-1,3.6634618E-1,-2.7589586E-1,8.500792E-2,5.2591234E-1,8.943176E-2,-3.3621106E-1,-8.631847E-2,-3.040177E-2,2.2731982E-1,7.803743E-3,6.5605694E-1,1.6560884E-1,-2.917438E-3,-3.7966388E-1,-3.578619E-3,-6.0078017E-3,1.5549397E-3,6.212109E-3,-1.00540236E-1,1.0412533E-2,2.1543296E-3,1.3267694E-2,2.6267806E-2,2.5978214E-3,7.8001083E-3,-2.2031178E-1,-1.7659152E-2,-4.9550044E-3,-2.5344206E-4,-9.553114E-3,-4.7625317E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[5.517495E0,2.2606163E0,1.4994822E0,4.2835498E-1,5.1606023E-1,7.4509144E-1,1.9494057E-1,2.7143145E-1,1.2115493E-1,2.7291396E-1,1.2617743E-1,0E0,4.2197704E-2,3.169495E-2,0E0,3.2834005E-1,0E0,0E0,0E0,0E0,4.709524E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2081087E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,20,20,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,5.6614784E7,1.4046429E1,3.0595828E3,9.2954946E-1,1.7059127E5,1.782E3,3.2936362E6,7.056912E7,2.7186182E-1,4.4E1,7.803743E-3,3E0,4.1E1,-2.917438E-3,2.9180167E2,-3.578619E-3,-6.0078017E-3,1.5549397E-3,6.212109E-3,1.2836887E1,1.0412533E-2,2.1543296E-3,1.3267694E-2,2.6267806E-2,2.5978214E-3,7.8001083E-3,9.7E1,-1.7659152E-2,-4.9550044E-3,-2.5344206E-4,-9.553114E-3,-4.7625317E-3],"split_indices":[2,12,55,4,40,29,0,51,7,28,3,0,8,3,0,53,0,0,0,0,55,0,0,0,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,3.9E1,3E1,2.1E1,1.3E1,2.9E1,1E1,1.7E1,1.3E1,7E0,1.4E1,9E0,4E0,2.4E1,5E0,6E0,4E0,4E0,1.3E1,9E0,4E0,4E0,1E1,4E0,5E0,1.1E1,1.3E1,9E0,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[9.404197E-3,-8.191902E-2,3.9164418E-1,-1.7726035E-1,1.2871456E-1,2.3037602E-1,2.0978406E-2,-3.552201E-1,-1.11507826E-1,-4.5729848E-3,1.7655551E-1,9.793322E-3,3.9969105E-3,-1.6186135E-2,-6.1901193E-3,-1.7534252E-1,-2.716451E-2,2.3359561E-1,5.5533648E-2,-2.0777021E-1,8.693186E-5,2.9419977E-3,-8.7311916E-2,2.4158028E-3,1.1802541E-2,-1.4702341E-4,3.7558996E-3,-4.640943E-3,-2.8998297E-1,-6.154108E-3,-2.2178702E-2,-1.2990197E-2,-5.3502736E-3,-2.6759414E-3,1.2346494E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,-1,-1,19,21,23,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[3.8020809E0,1.7900395E0,5.076542E-1,6.741699E-1,3.644562E-1,3.3572435E-2,0E0,1.968987E-1,2.4290222E-1,0E0,1.5270191E-1,0E0,0E0,0E0,0E0,1.5036523E-1,1.4435473E-1,2.2699332E-1,2.6441261E-2,1.06012404E-1,0E0,0E0,7.203901E-2,0E0,0E0,0E0,0E0,0E0,4.568374E-2,0E0,2.8889056E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,15,15,16,16,17,17,18,18,19,19,22,22,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,-1,-1,20,22,24,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[6.1596104E3,6.402099E-1,1.2768678E4,1.983671E4,3.568E3,7.118011E-1,2.0978406E-2,2.8374892E-2,7.388E3,-4.5729848E-3,1.4163358E7,9.793322E-3,3.9969105E-3,-1.6186135E-2,-6.1901193E-3,2.3514317E8,4.171707E9,1.09484E3,4.0389168E-1,6.681133E-1,8.693186E-5,2.9419977E-3,2.074635E-1,2.4158028E-3,1.1802541E-2,-1.4702341E-4,3.7558996E-3,-4.640943E-3,1.2163482E8,-6.154108E-3,3.3487454E11,-1.2990197E-2,-5.3502736E-3,-2.6759414E-3,1.2346494E-3],"split_indices":[53,43,53,49,2,28,0,58,2,0,46,0,0,0,0,7,5,53,28,28,0,0,40,0,0,0,0,0,7,0,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,8.7E1,2E1,6E1,2.7E1,1.2E1,8E0,1.5E1,4.5E1,4E0,2.3E1,8E0,4E0,9E0,6E0,2.5E1,2E1,1.5E1,8E0,2.1E1,4E0,7E0,1.3E1,6E0,9E0,4E0,4E0,1.2E1,9E0,5E0,8E0,5E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.6260551E-2,-1.8317385E-1,2.6741064E-1,-2.540629E-1,9.2660606E-2,1.4874665E-1,4.6568242E-1,-3.7536886E-1,-8.0223374E-2,-3.7698024E-3,9.9825505E-3,-2.4102514E-3,2.0709147E-1,9.956253E-3,2.3324372E-2,-1.4566985E-3,-4.2952442E-1,4.1348813E-3,-1.327848E-1,2.665169E-1,9.6425675E-2,-6.5715313E-3,-4.8507664E-1,-6.243079E-3,-7.78916E-2,1.7643574E-1,1.2484552E-2,4.8931944E-3,8.0829137E-4,-2.0885479E-2,-1.2388691E-2,-9.059528E-4,-4.155864E-3,2.2343034E-3,1.0058555E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.655051E0,1.1967599E0,1.1541727E0,9.8209953E-1,4.998774E-1,4.4133198E-1,4.7265387E-1,4.9067974E-1,2.301748E-1,0E0,0E0,0E0,1.5186274E-1,0E0,0E0,0E0,2.416892E-1,0E0,2.3988068E-2,6.040609E-2,2.8816007E-2,0E0,5.3410053E-2,0E0,1.4797598E-2,9.979406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,16,16,18,18,19,19,20,20,22,22,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.770862E8,2.561146E3,1.7618678E0,2.667836E6,2.9844797E-1,3.5849205E-1,9.563013E8,4.029937E-1,3.42E2,-3.7698024E-3,9.9825505E-3,-2.4102514E-3,1.6135693E1,9.956253E-3,2.3324372E-2,-1.4566985E-3,2.63E2,4.1348813E-3,9.320373E4,1.7302E4,1E0,-6.5715313E-3,1.4613986E3,-6.243079E-3,1.3560078E5,1.1803382E4,1.2484552E-2,4.8931944E-3,8.0829137E-4,-2.0885479E-2,-1.2388691E-2,-9.059528E-4,-4.155864E-3,2.2343034E-3,1.0058555E-2],"split_indices":[7,53,40,33,28,40,7,28,10,0,0,0,55,0,0,0,11,0,34,9,84,0,53,0,34,4,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.9E1,5.1E1,4.7E1,1.2E1,3.3E1,1.8E1,2.7E1,2E1,6E0,6E0,7E0,2.6E1,1E1,8E0,4E0,2.3E1,4E0,1.6E1,1.6E1,1E1,5E0,1.8E1,8E0,8E0,9E0,7E0,6E0,4E0,9E0,9E0,4E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0040615E-2,-1.6906129E-1,2.188069E-1,-2.1587087E-1,6.489098E-3,2.9824182E-2,3.124698E-1,-1.4519845E-1,-3.1849194E-1,8.985236E-3,-6.618626E-2,2.0420541E-1,3.985903E-1,-1.4569583E-2,-9.884405E-2,-3.6497048E-1,-3.1579603E-3,2.670431E-3,-1.2964426E-1,2.4638239E-3,2.6649892E-1,1.101048E-2,1.8611364E-2,-3.096264E-2,-6.4645973E-3,-2.5481278E-1,-1.6131384E-2,-6.710294E-3,-1.3087403E-3,1.1467256E-2,5.8643576E-3,-9.1229245E-2,5.1979795E-3,-1.1566119E-2,-4.5196684E-3,-4.2870683E-3,-6.480817E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,31,-1,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[4.3742824E0,1.0674121E0,9.46517E-1,3.5989428E-1,0E0,4.1606957E-1,2.5426173E-1,3.7927127E-1,2.0430112E-1,0E0,1.3410327E-1,1.3426387E-1,5.074501E-2,0E0,1.6325581E-1,4.8431396E-2,0E0,0E0,4.5769468E-2,0E0,1.2253225E-2,0E0,0E0,2.0290853E-1,0E0,4.486531E-2,0E0,0E0,0E0,0E0,0E0,2.609723E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,18,18,20,20,23,23,25,25,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,32,-1,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,3.7249353E3,6.2371206E-1,8.4947616E-1,6.489098E-3,3.5925132E-1,2.6167098E5,7.110474E7,2.4502628E6,8.985236E-3,9.069479E-1,2.0783027E5,1.5605432E9,-1.4569583E-2,3.925844E7,3.91028E5,-3.1579603E-3,2.670431E-3,1.939024E7,2.4638239E-3,2.548476E5,1.101048E-2,1.8611364E-2,8E0,-6.4645973E-3,7.434519E5,-1.6131384E-2,-6.710294E-3,-1.3087403E-3,1.1467256E-2,5.8643576E-3,1.4198953E1,5.1979795E-3,-1.1566119E-2,-4.5196684E-3,-4.2870683E-3,-6.480817E-4],"split_indices":[7,53,40,28,0,28,29,5,33,0,58,34,7,0,46,12,0,0,49,0,34,0,0,8,0,29,0,0,0,0,0,57,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,6.2E1,5.3E1,5.5E1,7E0,1.8E1,3.5E1,3.4E1,2.1E1,5E0,1.3E1,1.7E1,1.8E1,4E0,3E1,1.7E1,4E0,4E0,9E0,6E0,1.1E1,1.2E1,6E0,1.7E1,1.3E1,9E0,8E0,5E0,4E0,6E0,5E0,1.3E1,4E0,5E0,4E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-7.882657E-3,-1.962244E-1,2.4056515E-1,-9.932849E-2,-4.060304E-1,-6.703214E-2,3.3300647E-1,-1.7431891E-1,8.36856E-3,-6.773431E-3,-1.7170759E-2,-4.2780684E-3,6.651603E-4,4.1061303E-1,5.650482E-3,-2.618067E-1,-4.604724E-2,6.484681E-3,4.815757E-1,-3.1523302E-1,-3.0545453E-3,-4.3786177E-3,4.645628E-3,1.9725902E-2,7.826164E-3,-7.88878E-3,-1.6787257E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,-1,17,-1,19,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7733073E0,1.1387713E0,1.2734487E0,1.0584866E0,1.8430138E-1,5.0560266E-2,4.014423E-1,3.6949956E-1,0E0,0E0,0E0,0E0,0E0,2.9052138E-1,0E0,1.626724E-1,2.1278434E-1,0E0,1.7327881E-1,1.1566639E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,13,13,15,15,16,16,18,18,19,19],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,-1,18,-1,20,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,2.5927516E11,5.12544E-1,1E1,6.948189E7,5.3534385E3,1.4046429E1,1.386854E8,8.36856E-3,-6.773431E-3,-1.7170759E-2,-4.2780684E-3,6.651603E-4,5.403511E8,5.650482E-3,1.977643E6,3.167669E5,6.484681E-3,1.746E3,2E0,-3.0545453E-3,-4.3786177E-3,4.645628E-3,1.9725902E-2,7.826164E-3,-7.88878E-3,-1.6787257E-2],"split_indices":[12,32,40,8,7,4,55,7,0,0,0,0,0,7,0,49,29,0,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.7E1,4.3E1,4E1,1.7E1,1E1,3.3E1,3.3E1,7E0,5E0,1.2E1,6E0,4E0,2.2E1,1.1E1,1.9E1,1.4E1,6E0,1.6E1,1.4E1,5E0,1E1,4E0,1.2E1,4E0,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.234922E-2,-1.1959446E-1,2.692677E-1,-1.5458244E-1,1.2714416E-2,5.4225646E-4,3.244151E-1,-7.8147516E-2,-2.969515E-1,4.1584706E-1,5.732994E-3,-1.2855452E-1,1.2656279E-1,-1.4236198E-1,-1.4189349E-2,4.6046913E-1,6.855493E-3,-1.9554819E-1,-4.6117224E-2,8.02689E-3,1.9784305E-4,-7.7642547E-3,-9.622835E-4,8.53168E-3,1.8610887E-2,-8.071357E-3,-2.6134562E-3,1.9264045E-3,-8.0829665E-2,-3.943927E-3,-4.8823442E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[3.747346E0,1.1768546E0,5.4195094E-1,6.7324007E-1,0E0,0E0,4.1853905E-1,4.5998007E-1,2.707882E-1,1.1421871E-1,0E0,1.8427902E-1,1.0314682E-1,7.8383625E-2,0E0,7.552695E-2,0E0,4.937589E-2,6.3014776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3382068E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,28,28],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[5.1707494E-1,6.082581E3,1.6511278E1,8.106617E-1,1.2714416E-2,5.4225646E-4,1.343964E1,4.899102E5,3.3012803E9,4.698217E7,5.732994E-3,2.834008E-1,4.828443E-1,2.45E2,-1.4189349E-2,5.403511E8,6.855493E-3,4.3E1,9.069479E-1,8.02689E-3,1.9784305E-4,-7.7642547E-3,-9.622835E-4,8.53168E-3,1.8610887E-2,-8.071357E-3,-2.6134562E-3,1.9264045E-3,5.691229E6,-3.943927E-3,-4.8823442E-4],"split_indices":[40,53,57,28,0,0,55,29,5,46,0,58,28,11,0,7,0,3,58,0,0,0,0,0,0,0,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.7E1,3.8E1,6.3E1,4E0,7E0,3.1E1,4.2E1,2.1E1,1.9E1,1.2E1,3.4E1,8E0,9E0,1.2E1,1.5E1,4E0,1.8E1,1.6E1,4E0,4E0,5E0,4E0,4E0,1.1E1,1.4E1,4E0,4E0,1.2E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.3009777E-2,-1.8076913E-1,1.8460925E-1,-2.2224353E-1,4.2094267E-3,-4.635656E-3,2.9711828E-1,-1.4615564E-1,-3.4048074E-1,-9.690105E-2,5.266527E-3,2.193756E-1,4.255929E-1,-1.5337352E-2,-9.483938E-2,-3.8632032E-1,-4.5380723E-3,-2.348625E-2,-7.5696246E-3,2.442181E-1,4.058913E-3,1.9231215E-2,6.8796948E-3,-3.5304856E-2,-1.6997646E-1,-7.313136E-3,-1.554929E-2,-2.6376923E-3,1.1114292E-3,1.0115554E-2,5.5168583E-3,4.8919874E-3,-9.378512E-2,-7.996792E-3,-3.9744787E-3,-8.3479716E-4,-4.3228953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,-1,23,25,-1,27,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[3.6801403E0,7.6589215E-1,1.0872574E0,4.270475E-1,0E0,2.9187408E-1,2.2624993E-1,4.4591415E-1,1.4835119E-1,1.0269912E-1,0E0,3.0995369E-2,1.7534697E-1,0E0,1.2819651E-1,4.3216944E-2,0E0,2.6463097E-2,0E0,1.3077617E-2,0E0,0E0,0E0,1.9250572E-1,1.5729427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2905715E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,17,17,19,19,23,23,24,24,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,-1,24,26,-1,28,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[1.5363757E8,2.711196E3,9.745363E-1,8.297822E-1,4.2094267E-3,3.9219662E5,1.5435694E9,9.2433E4,3.1346153E1,2.2492808E5,5.266527E-3,1.5723504E1,4.4E1,-1.5337352E-2,3.925844E7,2.08E2,-4.5380723E-3,5.3990013E1,-7.5696246E-3,2.326E4,4.058913E-3,1.9231215E-2,6.8796948E-3,2.083117E1,1.7E2,-7.313136E-3,-1.554929E-2,-2.6376923E-3,1.1114292E-3,1.0115554E-2,5.5168583E-3,4.8919874E-3,3.2785356E7,-7.996792E-3,-3.9744787E-3,-8.3479716E-4,-4.3228953E-3],"split_indices":[12,53,43,28,0,29,7,1,59,34,0,55,3,0,46,11,0,57,0,9,0,0,0,59,10,0,0,0,0,0,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.9E1,5E1,5.2E1,7E0,1.9E1,3.1E1,3.3E1,1.9E1,1.2E1,7E0,2.1E1,1E1,4E0,2.9E1,1.5E1,4E0,8E0,4E0,1.6E1,5E0,6E0,4E0,1.7E1,1.2E1,4E0,1.1E1,4E0,4E0,1E1,6E0,4E0,1.3E1,5E0,7E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.9804446E-2,-1.6584219E-1,2.2534335E-1,-1.9925624E-1,8.962442E-3,1.1894395E-1,4.0559512E-1,-6.371361E-2,-3.373642E-1,-4.79808E-2,1.6835034E-1,2.5866732E-1,2.008889E-2,-1.2507846E-2,-6.890907E-3,-3.8854584E-1,-1.861423E-1,4.3842144E-4,-3.556473E-3,2.6246664E-1,6.928512E-2,1.119961E-2,5.6111203E-3,7.1935244E-2,-5.912861E-3,-6.931863E-3,-1.5930815E-2,-3.332643E-3,-8.763888E-3,1.2654458E-2,3.128588E-3,3.9051168E-3,-2.550271E-4,-1.4070388E-3,1.3063706E-1,8.102995E-3,-2.0714968E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1],"loss_changes":[4.4254956E0,8.93484E-1,1.0065844E0,1.0455177E0,0E0,3.049749E-1,2.9931355E-1,4.7958618E-1,1.4438891E-1,2.8538108E-2,2.428773E-1,1.5650094E-2,0E0,0E0,3.3331746E-1,1.2249017E-1,2.539453E-2,0E0,0E0,1.8632936E-1,4.5291804E-2,0E0,0E0,1.2228049E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.47826E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,19,19,20,20,23,23,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1],"split_conditions":[3.4707712E8,3.7249353E3,1.7618678E0,6.4987373E-1,8.962442E-3,3.5849205E-1,1.2750684E9,1.497E3,3.1371258E1,1.0824E4,1.2320755E1,2.3559767E5,2.008889E-2,-1.2507846E-2,3.925844E7,4.5E1,3.25E2,4.3842144E-4,-3.556473E-3,7.118011E-1,2.5E1,1.119961E-2,5.6111203E-3,3.22115E5,-5.912861E-3,-6.931863E-3,-1.5930815E-2,-3.332643E-3,-8.763888E-3,1.2654458E-2,3.128588E-3,3.9051168E-3,-2.550271E-4,-1.4070388E-3,1.0428423E9,8.102995E-3,-2.0714968E-5],"split_indices":[7,53,40,28,0,40,7,2,59,10,55,34,0,0,46,10,0,0,0,28,8,0,0,1,0,0,0,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6E1,5.4E1,5.6E1,4E0,3.5E1,1.9E1,2.9E1,2.7E1,8E0,2.7E1,1.1E1,8E0,4E0,2.5E1,1.9E1,8E0,4E0,4E0,1.3E1,1.4E1,6E0,5E0,1.7E1,8E0,5E0,1.4E1,4E0,4E0,8E0,5E0,9E0,5E0,6E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[8.9094596E-4,-8.418677E-2,2.9088813E-1,-1.6697854E-1,3.8710527E-2,4.689037E-1,1.03466496E-1,-2.052833E-1,6.5171765E-3,-4.9591303E-2,7.356622E-3,9.249497E-3,2.178669E-2,-5.2451534E-4,5.9309984E-3,-1.4593045E-1,-3.2189476E-1,8.833714E-4,-1.0717486E-1,-1.0503395E-2,-8.657066E-2,-6.019513E-3,-1.3442244E-2,-5.490938E-3,-5.008993E-4,-1.1700313E-1,1.4549642E-3,-6.5351226E-3,-2.1594516E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,19,21,-1,23,-1,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[2.3930295E0,7.6958007E-1,6.854913E-1,6.3603854E-1,4.619252E-1,1.602242E-1,8.849475E-2,2.367959E-1,0E0,9.2179015E-2,0E0,0E0,0E0,0E0,0E0,2.2441888E-1,3.9655447E-2,0E0,4.817827E-2,0E0,8.965723E-2,0E0,0E0,0E0,0E0,5.506572E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,15,15,16,16,18,18,20,20,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,20,22,-1,24,-1,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[4.492037E3,2.3514317E8,2.5927516E11,1E0,1.1329926E0,7.845511E-1,1.5438823E7,5.5309926E9,6.5171765E-3,9.513889E-1,7.356622E-3,9.249497E-3,2.178669E-2,-5.2451534E-4,5.9309984E-3,9.2433E4,5.145631E-1,8.833714E-4,1.447E3,-1.0503395E-2,6.939E3,-6.019513E-3,-1.3442244E-2,-5.490938E-3,-5.008993E-4,9.320373E4,1.4549642E-3,-6.5351226E-3,-2.1594516E-3],"split_indices":[53,7,32,89,40,28,1,5,0,58,0,0,0,0,0,1,59,0,0,0,2,0,0,0,0,34,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,7.4E1,2.1E1,4.4E1,3E1,1E1,1.1E1,4E1,4E0,2E1,1E1,5E0,5E0,4E0,7E0,2.8E1,1.2E1,9E0,1.1E1,7E0,2.1E1,4E0,8E0,7E0,4E0,1.7E1,4E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.5014396E-2,-1.1989209E-1,2.198597E-1,-1.8675263E-1,4.2297527E-2,-2.6039736E-4,2.7899745E-1,-3.1083202E-1,-1.0462264E-1,2.1596667E-1,-1.07906945E-1,3.4681892E-1,1.4881904E-1,-4.906404E-3,-3.5700446E-1,-1.3887258E-1,1.7085096E-3,1.4346364E-2,2.0050958E-3,-1.8250199E-1,1.7309767E-3,1.8190123E-2,9.37768E-3,1.8626243E-3,7.112547E-3,-1.4488746E-2,-6.4715906E-3,-9.498444E-2,-7.522882E-3,-9.739513E-3,-2.1204518E-3,-5.0669527E-3,-4.923456E-2,-2.7675268E-3,-3.599623E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[2.7113624E0,8.320497E-1,4.5858276E-1,5.177746E-1,6.2327963E-1,0E0,1.8676686E-1,1.21100664E-1,1.8351245E-1,2.9806107E-1,1.598858E-1,1.0155201E-1,4.1373044E-2,0E0,5.4087877E-2,6.56144E-2,0E0,0E0,0E0,8.136505E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2521486E-2,0E0,0E0,0E0,0E0,1.4473958E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,19,19,27,27,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.0230335E0,2.4722983E3,1.6511278E1,3.572374E6,5.4875424E7,-2.6039736E-4,2.8129198E7,3.842849E5,3.4707712E8,1.0007143E1,9.250217E5,8.892E3,3.8737573E11,-4.906404E-3,2.1883162E1,6.784095E7,1.7085096E-3,1.4346364E-2,2.0050958E-3,2.1227852E5,1.7309767E-3,1.8190123E-2,9.37768E-3,1.8626243E-3,7.112547E-3,-1.4488746E-2,-6.4715906E-3,1.3101191E1,-7.522882E-3,-9.739513E-3,-2.1204518E-3,-5.0669527E-3,3.1851864E7,-2.7675268E-3,-3.599623E-5],"split_indices":[40,53,57,12,46,0,46,29,7,54,33,9,32,0,59,46,0,0,0,29,0,0,0,0,0,0,0,57,0,0,0,0,33,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7.5E1,3.3E1,5.3E1,2.2E1,7E0,2.6E1,2E1,3.3E1,1E1,1.2E1,1.6E1,1E1,5E0,1.5E1,2.7E1,6E0,4E0,6E0,8E0,4E0,4E0,1.2E1,4E0,6E0,1.1E1,4E0,1.8E1,9E0,4E0,4E0,8E0,1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[8.900368E-3,-1.15445875E-1,2.4601895E-1,-2.604195E-1,3.313542E-2,2.0173583E-2,1.5582131E-1,-1.14240386E-1,-3.0438584E-1,-9.80291E-2,1.2803899E-1,2.1317573E-1,-5.024573E-3,-6.8414137E-3,-5.8254326E-4,-3.6944482E-3,-3.521724E-1,-6.209623E-3,-7.600827E-4,7.6640663E-3,-5.1744486E-4,2.620522E-1,3.2050724E-3,-4.1836056E-1,-7.5407824E-3,1.2138781E-2,4.283731E-3,-6.7753308E-3,-1.8421523E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,-1,-1,-1,-1,-1],"loss_changes":[2.8294582E0,1.3656723E0,8.438661E-1,1.7088962E-1,4.1088697E-1,0E0,4.8648798E-1,6.3389756E-2,1.9569445E-1,7.609184E-2,2.3092851E-1,1.1658132E-1,0E0,0E0,0E0,0E0,9.2832804E-2,0E0,0E0,0E0,0E0,1.3029003E-1,0E0,1.5862107E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,16,16,21,21,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,-1,-1,-1,-1,-1],"split_conditions":[3.7249353E3,3.2316522E7,6.4417176E-2,2.4232728E5,3.8757455E-1,2.0173583E-2,2.2950048E10,8.6E1,1.68E2,1.3045E4,9.1197605E0,1.5625807E1,-5.024573E-3,-6.8414137E-3,-5.8254326E-4,-3.6944482E-3,1.365445E1,-6.209623E-3,-7.600827E-4,7.6640663E-3,-5.1744486E-4,6.19999E9,3.2050724E-3,2E0,-7.5407824E-3,1.2138781E-2,4.283731E-3,-6.7753308E-3,-1.8421523E-2],"split_indices":[53,12,58,29,42,0,5,10,0,9,54,55,0,0,0,0,55,0,0,0,0,5,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6.2E1,3.2E1,3.1E1,3.1E1,6E0,2.6E1,8E0,2.3E1,1.3E1,1.8E1,2.2E1,4E0,4E0,4E0,5E0,1.8E1,6E0,7E0,1.1E1,7E0,1.5E1,7E0,1.1E1,7E0,9E0,6E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.832938E-3,-1.3853852E-1,1.2620504E-1,-1.905101E-1,9.6116275E-2,-2.5492298E-4,2.4364126E-1,-3.3862132E-1,-1.3584775E-1,9.738527E-3,-1.5030988E-3,-1.2742741E-1,7.0410624E-2,2.8838646E-1,4.847233E-3,-7.497833E-3,-1.7344054E-2,-5.534093E-2,-2.5840554E-1,-1.5488224E-3,-6.376834E-3,7.605512E-3,-1.7269943E-2,6.6758594E-3,1.47138005E-2,-9.5443234E-2,2.678462E-3,-5.5975732E-3,-1.0780397E-2,-2.1615345E-3,1.9620631E-3,-5.338603E-4,-4.5246924E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9434575E0,7.134478E-1,8.02015E-1,3.3451617E-1,2.743179E-1,2.516135E-1,9.831953E-2,1.2301552E-1,3.409021E-1,0E0,0E0,3.401135E-2,2.2498912E-1,1.5891886E-1,0E0,0E0,0E0,1.2655362E-1,1.1868715E-2,0E0,0E0,0E0,3.986518E-2,0E0,0E0,4.15328E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,17,17,18,18,22,22,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,2.7938022E3,1.0230335E0,1.73E2,9E0,1.937363E6,2.8129198E7,2.16268E5,7.332772E-1,9.738527E-3,-1.5030988E-3,8.44629E7,9E0,4.2864155E1,4.847233E-3,-7.497833E-3,-1.7344054E-2,1E1,8.07967E5,-1.5488224E-3,-6.376834E-3,7.605512E-3,5.759438E5,6.6758594E-3,1.47138005E-2,4.3786978E2,2.678462E-3,-5.5975732E-3,-1.0780397E-2,-2.1615345E-3,1.9620631E-3,-5.338603E-4,-4.5246924E-3],"split_indices":[7,53,40,0,18,33,46,12,28,0,0,46,8,57,0,0,0,8,1,0,0,0,29,0,0,53,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,4.6E1,1E1,2.6E1,2.7E1,1.1E1,3.5E1,4E0,6E0,9E0,1.7E1,1.8E1,9E0,7E0,4E0,2.2E1,1.3E1,4E0,5E0,6E0,1.1E1,1.1E1,7E0,1.7E1,5E0,5E0,8E0,7E0,4E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-7.356629E-3,-8.673777E-2,2.8287694E-1,-2.2338766E-1,2.0403562E-2,4.1299033E-1,3.459055E-2,-1.732699E-1,-1.6370945E-2,-5.2707877E-2,1.347688E-1,6.4738477E-3,1.8480975E-2,5.8937715E-3,-3.645891E-3,-1.9200978E-1,-1.2156821E-3,3.48537E-3,-8.6418964E-2,1.3644533E-3,1.8818837E-1,-1.4850397E-1,-9.223227E-3,-1.0390652E-1,1.5569644E-4,4.4298754E-3,9.2181945E-3,-2.1377632E-3,-1.683674E-1,-4.756301E-3,-4.4620893E-4,-8.739493E-3,-4.31646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,-1,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,31,-1,-1,-1,-1],"loss_changes":[2.4666777E0,1.2343894E0,7.2873807E-1,3.4805036E-1,4.0845093E-1,2.54884E-1,1.7347302E-1,7.724774E-2,0E0,1.5928172E-1,9.131506E-2,0E0,0E0,0E0,0E0,3.9366364E-2,0E0,0E0,4.011491E-2,0E0,2.1434158E-2,2.3729384E-2,0E0,5.188629E-2,0E0,0E0,0E0,0E0,2.5017768E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,15,15,18,18,20,20,21,21,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,-1,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,32,-1,-1,-1,-1],"split_conditions":[6.1596104E3,3.851071E7,2.692756E6,1E0,9.2954946E-1,1.1681136E4,1.063036E8,2.6666668E-1,-1.6370945E-2,1.283561E6,6E0,6.4738477E-3,1.8480975E-2,5.8937715E-3,-3.645891E-3,8.4395015E-1,-1.2156821E-3,3.48537E-3,1.1855E4,1.3644533E-3,5.506832E-1,1.728E3,-9.223227E-3,1E0,1.5569644E-4,4.4298754E-3,9.2181945E-3,-2.1377632E-3,5.127381E1,-4.756301E-3,-4.4620893E-4,-8.739493E-3,-4.31646E-3],"split_indices":[53,12,30,105,40,4,52,58,0,49,8,0,0,0,0,28,0,0,2,0,28,2,0,101,0,0,0,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,3.6E1,4.7E1,1.4E1,8E0,3.1E1,5E0,2.9E1,1.8E1,5E0,9E0,4E0,4E0,2.7E1,4E0,5E0,2.4E1,7E0,1.1E1,1.8E1,9E0,2E1,4E0,7E0,4E0,4E0,1.4E1,1.5E1,5E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.9744236E-2,-9.327785E-2,2.0905109E-1,-1.2495501E-1,1.01146875E-2,1.2056951E-1,3.5438117E-1,-4.6363864E-2,-2.357004E-1,2.1097495E-3,1.7033628E-1,1.5840996E-2,3.53248E-3,-1.7231452E-1,2.1365339E-2,-1.1647715E-2,-6.4637825E-2,1.5850387E-3,-1.4479358E-3,7.3449826E-3,2.335599E-3,-7.999368E-3,-3.1095452E-3,7.366086E-3,-5.5106208E-2,-3.941915E-3,-2.5859976E-4,-9.458616E-3,-3.8008306E-3,-1.6233418E-3,7.8364415E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[2.2253702E0,7.4477845E-1,4.844929E-1,4.764741E-1,0E0,1.5881532E-1,2.818097E-1,2.9371983E-1,3.2114422E-1,1.7639747E-2,5.329585E-2,0E0,0E0,3.185776E-2,3.325218E-1,0E0,2.2261191E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.7937537E-2,0E0,0E0,1.2041082E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,24,24,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[7.307441E-1,7.3153823E3,1.5605432E9,3.3012803E9,1.01146875E-2,1.5832817E5,2.9548872E9,1.1301075E1,4.0916511E3,2.4E1,7.639958E-1,1.5840996E-2,3.53248E-3,4.1100917E0,1.53E2,-1.1647715E-2,1.4E1,1.5850387E-3,-1.4479358E-3,7.3449826E-3,2.335599E-3,-7.999368E-3,-3.1095452E-3,7.366086E-3,3.6449524E7,-3.941915E-3,-2.5859976E-4,9.9987485E-2,-3.8008306E-3,-1.6233418E-3,7.8364415E-4],"split_indices":[40,4,7,5,0,34,7,57,4,3,28,0,0,57,10,0,8,0,0,0,0,0,0,0,46,0,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,5.9E1,4E1,5.5E1,4E0,2.6E1,1.4E1,3.3E1,2.2E1,8E0,1.8E1,1E1,4E0,1.1E1,2.2E1,1.4E1,8E0,4E0,4E0,1.3E1,5E0,6E0,5E0,6E0,1.6E1,4E0,4E0,9E0,7E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.354157E-3,-9.587389E-2,2.811731E-1,-2.1141864E-1,4.2217784E-2,2.0473529E-2,1.9797917E-1,-2.5688913E-1,-6.405464E-2,-5.279064E-2,1.8850118E-1,2.5317988E-1,2.01925E-3,-2.9090002E-1,-2.407845E-3,3.9753322E-3,-6.473553E-3,-2.2955444E-2,-3.6873005E-3,9.01752E-3,4.0192218E-3,2.1807535E-3,3.078492E-1,-3.9866236E-3,-3.1869465E-1,-2.6209368E-3,6.1207065E-5,1.2906996E-2,5.6645344E-3,-1.3646177E-2,-6.0544596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.973064E0,1.2056081E0,5.775304E-1,2.582742E-1,4.9638656E-1,0E0,1.7402846E-1,1.8112898E-1,2.4054828E-1,3.0042421E-2,3.8393497E-2,1.605668E-1,0E0,9.523201E-2,0E0,0E0,0E0,1.8478293E-2,0E0,0E0,0E0,0E0,4.134941E-2,0E0,1.3354897E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,22,22,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[3.944201E3,3.2936362E6,6.4417176E-2,7.33E3,9.2954946E-1,2.0473529E-2,1.0090909E1,3.0174E4,6.25379E5,1.5832817E5,7.6363635E-1,1.286E4,2.01925E-3,9.122283E0,-2.407845E-3,3.9753322E-3,-6.473553E-3,1E0,-3.6873005E-3,9.01752E-3,4.0192218E-3,2.1807535E-3,8.5E1,-3.9866236E-3,2.9282052E6,-2.6209368E-3,6.1207065E-5,1.2906996E-2,5.6645344E-3,-1.3646177E-2,-6.0544596E-3],"split_indices":[53,51,58,2,40,0,54,9,30,34,58,2,0,54,0,0,0,8,0,0,0,0,3,0,33,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.4E1,2.8E1,4E1,3.4E1,5E0,2.3E1,3E1,1E1,2.1E1,1.3E1,1.6E1,7E0,2.5E1,5E0,4E0,6E0,1.4E1,7E0,6E0,7E0,4E0,1.2E1,4E0,2.1E1,4E0,1E1,8E0,4E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.742977E-2,-1.3676295E-1,1.1548542E-1,-2.2288944E-1,-1.5563322E-2,1.9800499E-1,-6.0553983E-2,-1.2472117E-1,-3.194958E-1,-9.593062E-2,6.399417E-3,1.9342003E-2,1.1535384E-1,3.6078976E-3,-1.1354564E-1,-7.6137953E-3,7.265229E-4,-3.8447765E-1,-3.1281011E-3,-4.4159214E-3,-6.635637E-4,1.6669326E-2,1.7680356E-1,-1.704069E-1,8.2625286E-4,-1.0050992E-2,-1.7998286E-2,3.225355E-3,-1.4410706E-3,7.2806743E-3,2.4139055E-3,-1.8518103E-3,-1.0082227E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,-1,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8081089E0,6.22841E-1,7.964315E-1,2.9208076E-1,4.199064E-1,9.7104037E-1,1.6442794E-1,2.4138746E-1,2.23917E-1,3.6627695E-2,0E0,0E0,1.8504104E-1,0E0,1.1250734E-1,0E0,0E0,1.8489003E-2,0E0,0E0,0E0,5.702096E-2,3.844726E-2,1.1533919E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,17,17,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,-1,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9512938E3,3.2316522E7,7.096748E7,7.7E2,3.167669E5,8.892E3,1.6872E5,3.13E2,2.4636926E3,7.287E3,6.399417E-3,1.9342003E-2,4.6947882E-1,3.6078976E-3,2.3342845E5,-7.6137953E-3,7.265229E-4,1.4731E4,-3.1281011E-3,-4.4159214E-3,-6.635637E-4,4.171707E9,1.0488685E6,5.7194314E0,8.2625286E-4,-1.0050992E-2,-1.7998286E-2,3.225355E-3,-1.4410706E-3,7.2806743E-3,2.4139055E-3,-1.8518103E-3,-1.0082227E-2],"split_indices":[53,12,46,11,29,9,29,11,4,2,0,0,40,0,34,0,0,9,0,0,0,5,29,54,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5.9E1,5.3E1,3.4E1,2.5E1,3.6E1,1.7E1,1.8E1,1.6E1,1.8E1,7E0,6E0,3E1,4E0,1.3E1,1.1E1,7E0,1.2E1,4E0,1.3E1,5E0,1.2E1,1.8E1,9E0,4E0,8E0,4E0,5E0,7E0,1.4E1,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.02846E-2,-1.2583281E-1,1.609192E-1,-2.4537455E-1,-3.224422E-2,2.2265982E-2,2.2434208E-1,-3.0606943E-1,-1.10208705E-1,-8.0833375E-2,6.9238697E-3,6.414566E-3,-3.1021109E-2,1.4450054E-1,3.30083E-1,-7.0478697E-3,-1.5188813E-2,-1.463285E-3,-5.6986716E-3,2.9271317E-3,-1.4155595E-1,-2.425322E-3,1.2423156E-3,6.8439497E-3,9.876944E-2,2.1005973E-1,1.5302213E-2,-7.6886537E-3,-7.620329E-2,1.3894588E-3,4.297493E-3,3.7717826E-3,9.879033E-3,-4.072505E-3,-8.679343E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3012059E0,6.074141E-1,4.9625063E-1,1.631614E-1,3.6382857E-1,1.6285528E-1,2.8156102E-1,1.2120068E-1,2.347505E-2,2.76931E-1,0E0,0E0,3.779842E-2,3.0739427E-2,9.103167E-2,0E0,0E0,0E0,0E0,0E0,7.710606E-2,0E0,0E0,0E0,1.2521699E-2,3.1826735E-2,0E0,0E0,3.2140665E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,20,20,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,3.6E1,1.0539878E-1,1.977643E6,7.621429E1,5.28E2,1.5605432E9,1E0,1.453125E1,4.828443E-1,6.9238697E-3,6.414566E-3,7.331134E-2,1.8831E5,2.5172096E-1,-7.0478697E-3,-1.5188813E-2,-1.463285E-3,-5.6986716E-3,2.9271317E-3,9.5137325E-4,-2.425322E-3,1.2423156E-3,6.8439497E-3,2.0492467E5,2.4190365E6,1.5302213E-2,-7.6886537E-3,1.3890547E1,1.3894588E-3,4.297493E-3,3.7717826E-3,9.879033E-3,-4.072505E-3,-8.679343E-5],"split_indices":[7,3,39,49,57,0,7,17,55,28,0,0,39,30,39,0,0,0,0,0,40,0,0,0,34,33,0,0,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.4E1,5.6E1,2.3E1,3.1E1,1.8E1,3.8E1,1.5E1,8E0,2.6E1,5E0,4E0,1.4E1,2.3E1,1.5E1,9E0,6E0,4E0,4E0,7E0,1.9E1,9E0,5E0,1E1,1.3E1,8E0,7E0,8E0,1.1E1,4E0,9E0,4E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7413821E-3,-7.304192E-2,1.538762E-1,-4.7649376E-2,-1.2434536E-2,9.16067E-2,2.8793538E-1,-7.807313E-2,1.2463469E-2,-1.2068142E-3,4.2704353E-3,3.3709877E-3,1.4516395E-2,-1.6121142E-1,-2.4195444E-3,-2.2930211E-1,-9.5653705E-2,1.843168E-1,-6.696599E-2,-4.3924944E-3,-1.0643932E-2,-4.8598866E-3,-6.4819455E-5,8.599346E-3,3.378555E-3,-9.3958385E-2,7.120131E-4,-2.051382E-3,-5.5891774E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,-1,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.1651835E0,4.7973612E-1,2.4910057E-1,8.2777345E-1,0E0,8.400969E-2,1.8454671E-1,3.9586163E-1,0E0,0E0,0E0,0E0,0E0,1.11982524E-1,4.2215064E-1,6.286186E-2,6.1919376E-2,2.1703422E-2,6.269066E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.64033E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,13,13,14,14,15,15,16,16,17,17,18,18,25,25],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,-1,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[1.0230335E0,1.5081978E8,1.5605432E9,8.517173E3,-1.2434536E-2,3.22115E5,2.5172096E-1,3.0870352E1,1.2463469E-2,-1.2068142E-3,4.2704353E-3,3.3709877E-3,1.4516395E-2,2.1883162E1,3.24395E5,3.9024E4,1.3101191E1,1.1247059E1,3.5324219E3,-4.3924944E-3,-1.0643932E-2,-4.8598866E-3,-6.4819455E-5,8.599346E-3,3.378555E-3,5.4470426E-1,7.120131E-4,-2.051382E-3,-5.5891774E-3],"split_indices":[40,46,7,4,0,1,39,57,0,0,0,0,0,59,30,12,57,55,53,0,0,0,0,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.1E1,3.2E1,6.6E1,5E0,2.3E1,9E0,6.2E1,4E0,4E0,1.9E1,4E0,5E0,2.9E1,3.3E1,1.3E1,1.6E1,8E0,2.5E1,6E0,7E0,1.1E1,5E0,4E0,4E0,1.9E1,6E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.636178E-2,-7.384814E-2,1.9631414E-1,9.506933E-3,-9.827288E-2,1.5571484E-1,1.4757711E-2,-5.056059E-2,-2.8890294E-1,1.1577707E-1,1.2283535E-2,-1.6273664E-1,9.539735E-4,-4.91648E-3,-1.3261816E-2,7.256067E-5,1.4154191E-1,-1.8801776E-1,-2.3271416E-3,-6.0944006E-2,1.1141794E-1,6.11137E-3,2.0914278E-3,-8.3101755E-3,-2.690497E-3,-8.331574E-2,1.6320741E-3,6.5980474E-3,1.2661217E-3,-3.5396528E-3,-5.985618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,17,19,-1,-1,-1,21,23,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[1.7027802E0,6.023588E-1,2.3823917E-1,0E0,5.8568907E-1,1.8905854E-1,0E0,3.1220505E-1,1.0569215E-1,7.885781E-2,0E0,3.0118614E-2,2.6661515E-1,0E0,0E0,0E0,4.1209966E-2,4.5053452E-2,0E0,6.313539E-2,6.984249E-2,0E0,0E0,0E0,0E0,1.9014508E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,11,11,12,12,16,16,17,17,19,19,20,20,25,25],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,18,20,-1,-1,-1,22,24,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,6.371407E0,1.1276298E9,9.506933E-3,7.526629E7,6.2767E4,1.4757711E-2,2.782713E6,8.588E3,3.22115E5,1.2283535E-2,1.8873918E3,3.9219662E5,-4.91648E-3,-1.3261816E-2,7.256067E-5,1.9445117E2,1.9121604E-4,-2.3271416E-3,7.64755E5,8E0,6.11137E-3,2.0914278E-3,-8.3101755E-3,-2.690497E-3,3.8E1,1.6320741E-3,6.5980474E-3,1.2661217E-3,-3.5396528E-3,-5.985618E-4],"split_indices":[40,55,33,0,46,2,0,12,9,1,0,53,29,0,0,0,57,39,0,9,8,0,0,0,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,4E0,6.5E1,3E1,4E0,5.3E1,1.2E1,2.6E1,4E0,1.6E1,3.7E1,5E0,7E0,5E0,2.1E1,1.2E1,4E0,2.4E1,1.3E1,1.5E1,6E0,8E0,4E0,2E1,4E0,6E0,7E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.2470782E-2,-6.1596636E-2,1.5408692E-1,1.3200057E-2,-1.5370871E-1,6.761617E-2,2.7063823E-1,-4.1043464E-2,1.1494473E-2,-2.795465E-1,-4.729551E-2,1.0111448E-1,-2.4008842E-3,3.305493E-1,3.058147E-3,-1.1177263E-1,1.4055576E-2,-5.4431413E-3,-1.1407882E-2,1.5595459E-2,-6.3424553E-3,2.8277634E-4,5.310851E-3,6.1746673E-3,1.434267E-2,-6.1946614E-3,-5.2834257E-2,-5.1676974E-2,4.3003736E-3,-1.9323725E-3,5.226226E-2,-7.302295E-5,-3.3604323E-3,5.067166E-4,-2.9764003E-3,-2.6324703E-4,3.6595308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[1.100362E0,4.7741678E-1,3.4451842E-1,6.5867484E-1,4.0189344E-1,1.0556354E-1,1.3909137E-1,1.3407274E-1,0E0,1.8049717E-2,1.4807205E-1,7.6065645E-2,0E0,4.9132705E-2,0E0,4.3811426E-2,1.4427063E-1,0E0,0E0,3.574428E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.821586E-2,2.7438339E-2,0E0,0E0,2.705666E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,19,19,26,26,27,27,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[9.745363E-1,2.5927516E11,4.8E1,8.784E3,3.5904072E3,4.5E1,1.31E3,1.56967745E1,1.1494473E-2,3.290356E5,7.752181E7,9.1094507E2,-2.4008842E-3,1.5605432E9,3.058147E-3,9.7E1,3.1228534E5,-5.4431413E-3,-1.1407882E-2,4.6562446E3,-6.3424553E-3,2.8277634E-4,5.310851E-3,6.1746673E-3,1.434267E-2,-6.1946614E-3,1.7076684E7,1.4783E4,4.3003736E-3,-1.9323725E-3,1.8302105E6,-7.302295E-5,-3.3604323E-3,5.067166E-4,-2.9764003E-3,-2.6324703E-4,3.6595308E-3],"split_indices":[43,32,57,2,4,3,0,57,0,29,46,53,0,7,0,10,29,0,0,56,0,0,0,0,0,0,46,9,0,0,33,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.8E1,3.5E1,3.8E1,3E1,2.1E1,1.4E1,3.3E1,5E0,1.3E1,1.7E1,1.7E1,4E0,1E1,4E0,1.4E1,1.9E1,4E0,9E0,1.2E1,5E0,6E0,1.1E1,4E0,6E0,6E0,8E0,1.2E1,7E0,4E0,8E0,4E0,4E0,4E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-5.6217406E-3,-1.4812388E-1,1.00196324E-1,-2.0029053E-1,4.5021903E-3,4.8456933E-2,2.4591851E-1,-1.6120425E-1,-1.4022321E-2,-2.4330225E-2,1.2546659E-1,2.9834944E-1,2.5597236E-3,-2.0845999E-1,6.655365E-4,-6.092472E-2,4.7496065E-2,1.6492017E-1,6.743358E-2,6.5516746E-3,1.2849381E-2,-1.1372252E-1,-9.620884E-3,-3.717309E-3,-1.2116025E-2,4.403759E-3,-1.317208E-3,7.523454E-3,3.534184E-3,1.4049079E-5,4.449301E-3,-2.2817212E-3,-6.2029837E-3,1.0321692E-3,-1.819534E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,21,-1,23,25,27,29,-1,-1,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6283265E0,6.796731E-1,4.5553726E-1,2.2413254E-1,0E0,2.6650238E-1,1.2566847E-1,2.924673E-1,0E0,6.860485E-2,4.1539848E-2,1.6141653E-2,0E0,1.181159E-1,0E0,3.3634484E-2,6.0745515E-2,1.7477602E-2,4.1107986E-2,0E0,0E0,2.4359941E-2,0E0,0E0,1.5465637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,22,-1,24,26,28,30,-1,-1,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.363496E7,2.4722983E3,1.5791555E9,1E0,4.5021903E-3,1.1058695E0,2.797345E6,2.834008E-1,-1.4022321E-2,3.9219662E5,4.4E1,8.443332E-1,2.5597236E-3,3.842849E5,6.655365E-4,4.1E1,9.513889E-1,2.1987225E5,5.1E1,6.5516746E-3,1.2849381E-2,1.3578177E4,-9.620884E-3,-3.717309E-3,2.014695E6,4.403759E-3,-1.317208E-3,7.523454E-3,3.534184E-3,1.4049079E-5,4.449301E-3,-2.2817212E-3,-6.2029837E-3,1.0321692E-3,-1.819534E-3],"split_indices":[12,53,7,105,0,42,30,58,0,29,3,43,0,29,0,3,58,34,3,0,0,34,0,0,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,4.5E1,6.1E1,3.8E1,7E0,4.6E1,1.5E1,3.3E1,5E0,2.4E1,2.2E1,1.1E1,4E0,2.6E1,7E0,1.6E1,8E0,1.2E1,1E1,5E0,6E0,1.1E1,1.5E1,8E0,8E0,4E0,4E0,6E0,6E0,5E0,5E0,7E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7194271E-2,-9.72937E-2,1.8896739E-1,-2.0284453E-1,1.6372813E-2,1.5365627E-2,9.84426E-2,-2.4842085E-1,-6.0105067E-2,6.6005394E-2,-1.114531E-1,1.2767667E-1,4.831433E-4,-1.961372E-1,-1.6990492E-2,-5.9313225E-3,8.2670106E-4,-2.2887293E-2,1.4457154E-1,-1.8221717E-3,-5.0215493E-3,5.5879564E-3,1.1436479E-3,-1.0958742E-1,-1.0396929E-2,1.460551E-3,-6.257253E-2,7.2778035E-3,1.827394E-3,-6.322765E-3,-1.0955501E-3,-3.8567085E-3,-2.0959246E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7688091E0,9.2713606E-1,6.007501E-1,2.4529016E-1,2.4844103E-1,0E0,5.5484504E-2,2.56204E-1,9.888579E-2,1.988596E-1,1.1518404E-2,3.932637E-2,0E0,1.7511368E-1,0E0,0E0,0E0,3.7724E-2,7.014665E-2,0E0,0E0,0E0,0E0,7.259332E-2,0E0,0E0,2.198869E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,18,18,23,23,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,3.2936362E6,6.4417176E-2,5.127381E1,2.7260774E7,1.5365627E-2,8.442486E-1,1E0,6.444E3,1.1462246E3,5.661451E-1,2.1897722E10,4.831433E-4,4.178728E7,-1.6990492E-2,-5.9313225E-3,8.2670106E-4,2.4E1,1.4922581E1,-1.8221717E-3,-5.0215493E-3,5.5879564E-3,1.1436479E-3,9.7E1,-1.0396929E-2,1.460551E-3,2.83E2,7.2778035E-3,1.827394E-3,-6.322765E-3,-1.0955501E-3,-3.8567085E-3,-2.0959246E-4],"split_indices":[53,51,58,57,46,0,28,105,2,53,28,5,0,46,0,0,0,3,55,0,0,0,0,10,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.6E1,2.9E1,3.9E1,3.7E1,7E0,2.2E1,2.9E1,1E1,2.7E1,1E1,1.6E1,6E0,2.5E1,4E0,4E0,6E0,1.3E1,1.4E1,4E0,6E0,1.2E1,4E0,1.4E1,1.1E1,5E0,8E0,8E0,6E0,7E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.814685E-2,-7.426212E-2,1.5305519E-1,-1.5168595E-1,5.018264E-3,2.13539E-1,-5.187447E-3,-1.9152962E-1,1.1419349E-3,4.6743922E-2,-7.5811096E-2,1.53488E-2,1.2629546E-1,-1.550316E-1,-1.3174422E-2,-3.4991976E-2,1.2514096E-1,-3.5485455E-3,-1.0635336E-3,3.955607E-2,1.9900203E-1,-8.6807534E-2,-2.323222E-1,1.5362607E-3,-7.0264846E-2,6.1947E-3,2.609006E-4,2.6120916E-3,-4.1524476E-5,3.763078E-3,9.169147E-3,-5.7708244E-3,-1.9698124E-2,-1.0033611E-2,-3.7470725E-3,-3.7190807E-3,-6.106465E-4,9.7564876E-4,-2.5931012E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,21,-1,23,25,-1,-1,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0104992E0,4.8567823E-1,4.9732918E-1,2.9821712E-1,1.3845004E-1,3.5244358E-1,0E0,1.5225494E-1,0E0,1.7739096E-1,1.1691667E-2,0E0,9.933916E-2,1.32752E-1,0E0,4.128278E-2,7.5428426E-2,0E0,0E0,1.20978765E-2,2.0413637E-2,8.017501E-2,4.7340453E-2,0E0,1.5729092E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6295848E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,24,24,32,32],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,22,-1,24,26,-1,-1,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[4.492037E3,3.851071E7,9.302862E7,1.2599278E1,2.7260774E7,6.691E3,-5.187447E-3,8.8945635E-2,1.1419349E-3,1.1462246E3,4.1E1,1.53488E-2,3.3244818E7,4.8861527E9,-1.3174422E-2,2.4E1,1.5E1,-3.5485455E-3,-1.0635336E-3,2.2087644E9,1.5821141E0,1.66E2,3.159013E3,1.5362607E-3,3.670858E1,6.1947E-3,2.609006E-4,2.6120916E-3,-4.1524476E-5,3.763078E-3,9.169147E-3,-5.7708244E-3,1.13474E5,-1.0033611E-2,-3.7470725E-3,-3.7190807E-3,-6.106465E-4,9.7564876E-4,-2.5931012E-3],"split_indices":[53,12,46,54,46,9,0,39,0,53,3,0,52,5,0,3,8,0,0,12,40,0,4,0,59,0,0,0,0,0,0,0,30,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.8E1,2.5E1,3.9E1,3.9E1,2.1E1,4E0,3.2E1,7E0,2.6E1,1.3E1,5E0,1.6E1,2.8E1,4E0,1.3E1,1.3E1,8E0,5E0,8E0,8E0,1.6E1,1.2E1,4E0,9E0,9E0,4E0,4E0,4E0,4E0,4E0,7E0,9E0,8E0,4E0,5E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.737481E-3,-8.9211E-2,1.6433759E-1,-1.941062E-1,-9.074778E-3,2.2123896E-1,-4.01216E-3,-7.731144E-2,-2.4368058E-1,5.9842452E-2,-9.1321714E-2,1.3956001E-1,3.5705885E-1,-4.9284315E-3,-1.8872228E-4,-1.8668921E-1,-1.4445293E-2,1.2894379E-1,9.785402E-3,-5.6381626E-3,-4.4391062E-2,1.086708E-2,7.029671E-2,1.6602146E-2,5.1169656E-3,-7.6647694E-3,-2.8027901E-3,1.7613143E-3,6.291E-3,2.3403992E-3,-1.2315066E-3,4.2009E-4,-2.5483228E-3,6.3044066E-3,4.8197852E-3,2.5986042E-3,-2.18891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,25,-1,27,29,-1,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1],"loss_changes":[1.585721E0,6.202755E-1,6.0454416E-1,1.6644156E-1,2.4919885E-1,2.9407132E-1,0E0,4.6226688E-2,1.2144017E-1,8.175443E-2,5.3933844E-2,2.1366283E-1,1.7830575E-1,0E0,0E0,3.379804E-2,0E0,2.7816713E-2,3.845221E-2,0E0,1.9899528E-2,0E0,6.387175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3925844E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,20,20,22,22,33,33],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,26,-1,28,30,-1,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1],"split_conditions":[1.1855E4,1.04856936E8,1.650269E10,2.5107703E-1,2.8060134E7,1.1811385E4,-4.01216E-3,1.7593515E1,9.180692E7,8.66623E0,9.095E3,2.70482E5,3.813527E2,-4.9284315E-3,-1.8872228E-4,9.869215E-4,-1.4445293E-2,2.0195505E-1,1.2597537E7,-5.6381626E-3,1.1894881E6,1.086708E-2,1.5603712E2,1.6602146E-2,5.1169656E-3,-7.6647694E-3,-2.8027901E-3,1.7613143E-3,6.291E-3,2.3403992E-3,-1.2315066E-3,4.2009E-4,-2.5483228E-3,4.653145E9,4.8197852E-3,2.5986042E-3,-2.18891E-3],"split_indices":[2,7,5,28,46,53,0,59,7,54,9,30,57,0,0,40,0,39,33,0,49,0,59,0,0,0,0,0,0,0,0,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,7.3E1,3.6E1,3.1E1,4.2E1,3E1,6E0,1E1,2.1E1,2.3E1,1.9E1,2E1,1E1,5E0,5E0,1.7E1,4E0,9E0,1.4E1,7E0,1.2E1,5E0,1.5E1,6E0,4E0,1.3E1,4E0,4E0,5E0,6E0,8E0,4E0,8E0,8E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[5.060023E-3,-3.9900444E-2,1.6904347E-1,-1.4403671E-1,-2.902061E-3,1.5705138E-2,7.09969E-2,-1.0699426E-1,-8.122603E-3,5.0977856E-2,-7.2072595E-2,-4.707141E-3,1.3784634E-1,-1.8178518E-3,-5.1125013E-3,-9.907756E-3,1.2770411E-1,-1.289613E-1,3.4883092E-3,8.555458E-3,1.2231402E-3,-3.3133812E-3,1.47538865E-2,5.727273E-3,1.0081484E-3,-8.083487E-2,-7.0653395E-3,-1.5714758E-3,1.8053614E-3,3.6307182E-2,-1.1123942E-3,-4.3220604E-3,-3.695545E-4,8.1295395E-5,3.0158428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[7.8825873E-1,3.2322335E-1,5.688874E-1,4.1894823E-2,2.3846897E-1,0E0,2.6028833E-1,2.6200786E-2,0E0,1.6944015E-1,1.2080957E-1,0E0,1.3625526E-1,0E0,0E0,4.3753054E-2,4.4953912E-2,3.557062E-2,3.0608458E-2,0E0,0E0,0E0,1.7814184E-2,0E0,0E0,2.8968133E-2,0E0,0E0,0E0,1.9112017E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,16,16,17,17,18,18,22,22,25,25,29,29],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[4.3972437E3,7.5773336E7,6.4417176E-2,7.156922E7,2.727403E7,1.5705138E-2,9.040125E5,6.072427E-1,-8.122603E-3,1.6987854E1,5.127381E1,-4.707141E-3,3.077632E5,-1.8178518E-3,-5.1125013E-3,2.3616585E6,1.8634315E1,4.95E9,3.1676855E3,8.555458E-3,1.2231402E-3,-3.3133812E-3,1.499169E1,5.727273E-3,1.0081484E-3,2E0,-7.0653395E-3,-1.5714758E-3,1.8053614E-3,2.0195505E-1,-1.1123942E-3,-4.3220604E-3,-3.695545E-4,8.1295395E-5,3.0158428E-3],"split_indices":[53,7,58,7,46,0,33,28,0,57,57,0,29,0,0,49,55,5,53,0,0,0,55,0,0,8,0,0,0,39,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,2.1E1,6.2E1,5E0,1.7E1,1.6E1,5E0,3.5E1,2.7E1,4E0,1.3E1,7E0,9E0,2E1,1.5E1,1.5E1,1.2E1,6E0,7E0,4E0,1.6E1,1.1E1,4E0,1E1,5E0,6E0,6E0,1.1E1,5E0,6E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[9.323929E-3,-7.813383E-2,1.8397304E-1,-1.0277315E-1,9.485412E-3,2.4387056E-1,7.670746E-2,-1.3511458E-1,-2.7036062E-2,4.4523873E-3,2.813675E-1,1.5282334E-3,4.697767E-3,-1.0695489E-1,-1.1012138E-2,2.3927214E-3,-5.2890096E-2,1.204059E-2,3.6088598E-3,-7.6536015E-2,-6.6053187E-3,-3.0771093E-3,-7.8444736E-4,-1.20134674E-1,-2.815386E-2,-1.2017918E-3,-5.6531634E-3,6.4621645E-4,-1.6245084E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,-1,21,-1,-1,23,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[1.6029142E0,6.1491096E-1,2.0616627E-1,1.5801042E-1,0E0,6.474292E-2,2.0199351E-2,1.8746293E-1,5.416703E-2,0E0,1.1799741E-1,0E0,0E0,8.078575E-2,0E0,0E0,1.5281063E-2,0E0,0E0,6.2009066E-2,0E0,0E0,0E0,4.543622E-2,1.3215896E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,16,16,19,19,23,23,24,24],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,-1,22,-1,-1,24,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[7.90671E-1,6.082581E3,1.33556795E1,2.561146E3,9.485412E-3,3.840785E1,5.72143E5,1.01949844E6,3.09E2,4.4523873E-3,2.034139E6,1.5282334E-3,4.697767E-3,5.501383E9,-1.1012138E-2,2.3927214E-3,1.4E1,1.204059E-2,3.6088598E-3,2.91E2,-6.6053187E-3,-3.0771093E-3,-7.8444736E-4,9.122283E0,3.717E3,-1.2017918E-3,-5.6531634E-3,6.4621645E-4,-1.6245084E-3],"split_indices":[40,53,55,53,0,57,30,29,0,0,30,0,0,5,0,0,8,0,0,0,0,0,0,54,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,6.5E1,4E0,2.1E1,1.3E1,4.5E1,2E1,6E0,1.5E1,9E0,4E0,4E1,5E0,4E0,1.6E1,1.1E1,4E0,3E1,1E1,7E0,9E0,1.5E1,1.5E1,5E0,1E1,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.867907E-3,-6.967474E-2,1.558265E-1,-1.2791926E-1,3.3604927E-2,7.122294E-2,2.1767244E-1,1.9862214E-3,-1.8571019E-1,9.777716E-3,-1.3307461E-2,3.3243027E-2,3.966981E-3,1.3160989E-1,1.2062451E-2,4.2340755E-3,-5.5825025E-2,-1.08377844E-1,-2.2470771E-1,2.86648E-2,-7.051779E-3,2.6680941E-3,-5.077828E-4,6.3804924E-3,1.1948403E-3,-3.6459872E-3,-3.299267E-4,-1.6376357E-1,8.6896884E-4,-8.955007E-3,-4.9078837E-3,-3.211083E-2,8.198706E-2,-7.876448E-3,-3.2902192E-3,-2.5604675E-3,7.1524755E-6,4.625124E-3,1.5492659E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1663843E0,4.915521E-1,1.5193307E-1,3.9075166E-1,3.3746555E-1,1.8369898E-2,1.3640732E-1,1.1985446E-1,8.421445E-2,0E0,2.0422962E-1,1.8350948E-2,0E0,4.7491208E-2,0E0,0E0,2.4379428E-2,1.06641084E-1,1.3918042E-2,7.392795E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9629747E-2,0E0,0E0,0E0,1.3807188E-2,1.612898E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,19,19,27,27,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,2.561146E3,2.4221443E-1,5.1056176E-1,6.169213E0,1.0188E4,1.286E4,1.0526E4,4.178728E7,9.777716E-3,1.0990962E8,5E0,3.966981E-3,1.1180212E1,1.2062451E-2,4.2340755E-3,2.83E2,5.677647E2,1.1309524E-1,1E0,-7.051779E-3,2.6680941E-3,-5.077828E-4,6.3804924E-3,1.1948403E-3,-3.6459872E-3,-3.299267E-4,4.2E1,8.6896884E-4,-8.955007E-3,-4.9078837E-3,1.0837745E1,1.18631315E1,-7.876448E-3,-3.2902192E-3,-2.5604675E-3,7.1524755E-6,4.625124E-3,1.5492659E-3],"split_indices":[40,53,39,28,55,9,2,9,46,0,46,8,0,55,0,0,0,53,58,102,0,0,0,0,0,0,0,3,0,0,0,55,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,8E1,3.1E1,5.1E1,2.9E1,1.4E1,1.7E1,1.6E1,3.5E1,4E0,2.5E1,8E0,6E0,1.1E1,6E0,5E0,1.1E1,1.3E1,2.2E1,2.1E1,4E0,4E0,4E0,7E0,4E0,5E0,6E0,9E0,4E0,1.6E1,6E0,1E1,1.1E1,4E0,5E0,4E0,6E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.1503411E-2,-6.022531E-2,1.0210416E-1,1.1743105E-2,-1.8415894E-1,1.3324112E-1,-6.2689167E-3,-1.1934079E-1,5.8139388E-2,-2.0429802E-3,-2.1084705E-1,8.170475E-2,2.848334E-1,-1.7364284E-3,-6.5334076E-3,1.0161169E-1,-3.0271929E-2,-3.4372571E-3,-9.57698E-3,5.4357923E-3,4.3561555E-2,1.4179162E-2,3.5516846E-3,1.3515343E-1,2.3150533E-4,2.9081703E-4,-2.06386E-3,6.5976917E-3,3.8783143E-3,7.5871915E-2,8.184036E-3,1.1731047E-3,-1.8662412E-3,4.8724907E-3,5.049307E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,23,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1,-1],"loss_changes":[6.625155E-1,5.105631E-1,4.0949824E-1,2.319652E-1,5.956614E-2,2.9704094E-1,0E0,3.5200447E-2,1.1069242E-1,0E0,8.173025E-2,7.7488944E-2,1.6256648E-1,0E0,0E0,5.9803218E-2,1.0768166E-2,0E0,0E0,0E0,5.1224194E-2,0E0,0E0,5.876735E-2,0E0,0E0,0E0,2.432139E-2,0E0,3.4680277E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,15,15,16,16,20,20,23,23,27,27,29,29],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,24,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1,-1],"split_conditions":[1.9552084E3,4.307821E7,2.192772E10,1.92E2,5.3323513E-1,1.2768678E4,-6.2689167E-3,8.5959184E-1,2.70482E5,-2.0429802E-3,2.6078947E1,1.2187778E11,8.260109E9,-1.7364284E-3,-6.5334076E-3,4.3952936E7,2.1151997E5,-3.4372571E-3,-9.57698E-3,5.4357923E-3,2.3E1,1.4179162E-2,3.5516846E-3,1.5150703E3,2.3150533E-4,2.9081703E-4,-2.06386E-3,3.200296E9,3.8783143E-3,1.2410928E8,8.184036E-3,1.1731047E-3,-1.8662412E-3,4.8724907E-3,5.049307E-4],"split_indices":[53,46,5,0,28,53,0,28,30,0,57,32,5,0,0,33,29,0,0,0,8,0,0,4,0,0,0,12,0,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.6E1,4.4E1,3.6E1,2E1,4E1,4E0,9E0,2.7E1,4E0,1.6E1,3.1E1,9E0,5E0,4E0,1.8E1,9E0,6E0,1E1,1E1,2.1E1,5E0,4E0,1.3E1,5E0,4E0,5E0,1.4E1,7E0,9E0,4E0,1E1,4E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.697377E-3,-9.7170286E-2,8.702731E-2,-1.2952103E-1,5.0018826E-3,1.0664505E-2,1.9197288E-1,-1.9993305E-1,-4.4293467E-2,4.369449E-2,-5.8928872E-3,4.7823135E-3,9.264073E-3,-1.4409603E-1,-1.4644435E-2,-1.0872006E-1,4.8893443E-3,5.649267E-3,9.390396E-3,-8.207968E-2,-8.107118E-3,-5.162672E-3,-1.6550336E-3,2.0104758E-3,-1.4502563E-3,-3.0271173E-2,4.100922E-3,-4.444497E-3,5.9169147E-4,-4.6842344E-2,5.6154124E-4,-3.8674063E-4,-2.9311008E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,-1,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1],"loss_changes":[8.807678E-1,4.2972445E-1,3.9864957E-1,2.7919453E-1,0E0,1.7943832E-1,4.6776295E-2,2.4017239E-1,7.3411226E-2,9.912152E-2,0E0,0E0,0E0,9.134397E-2,0E0,1.499825E-2,3.4281798E-2,0E0,9.0225205E-2,5.8574945E-2,0E0,0E0,0E0,0E0,0E0,1.28785735E-2,0E0,0E0,0E0,1.402094E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,18,18,19,19,25,25,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,-1,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1],"split_conditions":[2.1915238E8,2.561146E3,1.3536367E0,2.667836E6,5.0018826E-3,8.637973E6,1.3127055E4,1.5755641E6,2.9544958E-1,1.2192353E7,-5.8928872E-3,4.7823135E-3,9.264073E-3,2.8225484E11,-1.4644435E-2,6.114E3,3.7E1,5.649267E-3,5.528107E3,3.845592E5,-8.107118E-3,-5.162672E-3,-1.6550336E-3,2.0104758E-3,-1.4502563E-3,4.599408E6,4.100922E-3,-4.444497E-3,5.9169147E-4,3.494164E7,5.6154124E-4,-3.8674063E-4,-2.9311008E-3],"split_indices":[7,53,40,33,0,48,4,33,42,46,0,0,0,32,0,2,3,0,53,33,0,0,0,0,0,1,0,0,0,46,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.3E1,4.9E1,4.7E1,6E0,2.9E1,2E1,2.5E1,2.2E1,2.5E1,4E0,1.2E1,8E0,2.1E1,4E0,9E0,1.3E1,5E0,2E1,1.3E1,8E0,5E0,4E0,6E0,7E0,1.5E1,5E0,9E0,4E0,1.1E1,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-4.237777E-3,-8.269796E-2,7.865978E-2,-1.0649972E-1,4.9593593E-3,2.8124793E-2,1.8699329E-1,-7.389247E-2,-9.9861985E-3,5.3282514E-2,-5.363919E-3,2.2256964E-1,2.0613405E-3,-1.5181887E-2,-1.7935653E-1,1.2531739E-2,1.1060278E-1,9.59934E-3,3.6133083E-3,-4.053344E-2,4.320899E-3,-3.6861477E-3,-7.78019E-3,5.7807863E-2,-3.3883635E-2,1.5514398E-1,9.154708E-4,-9.04947E-2,-3.6517316E-3,-5.268551E-4,3.655955E-3,-2.0437986E-3,1.7005613E-4,7.0945136E-3,2.9875794E-3,-4.632834E-3,-1.356904E-3,1.3300382E-3,-1.4166357E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.219622E-1,3.1319165E-1,2.90371E-1,2.6798433E-1,0E0,1.7683795E-1,6.5853715E-2,2.7591056E-1,0E0,7.8032695E-2,0E0,4.2365193E-2,0E0,1.0742183E-1,2.2673666E-2,4.609041E-2,4.9223155E-2,0E0,0E0,4.7578614E-2,0E0,0E0,0E0,3.608715E-2,9.914843E-3,1.0688394E-2,0E0,1.7193533E-2,2.4501283E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,19,19,23,23,24,24,25,25,27,27,28,28],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4707712E8,5.188755E3,1.7618678E0,1E0,4.9593593E-3,1.5939707E7,2.393487E5,4.352552E9,-9.9861985E-3,1.9649E4,-5.363919E-3,1.8613E4,2.0613405E-3,5.980516E5,7.056912E7,1.5E1,2.3152558E9,9.59934E-3,3.6133083E-3,2.83E2,4.320899E-3,-3.6861477E-3,-7.78019E-3,2.951724E1,4.1E1,1.2271712E1,9.154708E-4,1.85E2,1.0521739E1,-5.268551E-4,3.655955E-3,-2.0437986E-3,1.7005613E-4,7.0945136E-3,2.9875794E-3,-4.632834E-3,-1.356904E-3,1.3300382E-3,-1.4166357E-3],"split_indices":[7,4,40,105,0,48,34,5,0,2,0,9,0,29,7,8,7,0,0,0,0,0,0,57,3,55,0,10,54,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,5.1E1,5E0,3.7E1,1.6E1,4.4E1,7E0,3.3E1,4E0,1.2E1,4E0,2.9E1,1.5E1,2E1,1.3E1,8E0,4E0,2.5E1,4E0,6E0,9E0,1E1,1E1,8E0,5E0,1E1,1.5E1,4E0,6E0,6E0,4E0,4E0,4E0,5E0,5E0,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-5.985119E-3,-3.7334602E-2,2.798937E-1,-1.1924549E-1,3.852818E-2,1.2968256E-2,4.6480955E-3,-1.0174001E-2,-8.61623E-2,1.4390624E-1,-4.7238437E-3,-6.5295007E-3,-1.8414815E-1,6.496868E-3,1.5535147E-3,-8.2686484E-2,3.44543E-2,-6.916707E-2,9.163222E-2,-2.148222E-3,-7.867692E-3,-7.92647E-4,-3.9824406E-3,1.4682037E-2,3.2970735E-3,7.660684E-4,-3.8790284E-3,5.910601E-3,4.4155237E-5,-1.5095895E-3,5.1122166E-2,4.8930116E-4,2.7445937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1],"loss_changes":[9.06665E-1,5.700673E-1,7.3627055E-2,2.1235025E-1,2.1945485E-1,0E0,0E0,0E0,2.953073E-1,4.3165177E-2,1.0975401E-1,1.4160739E-1,5.8602095E-2,0E0,0E0,1.7707884E-2,2.5645847E-2,5.0714858E-2,5.845225E-2,0E0,0E0,0E0,0E0,4.125292E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0194946E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,23,23,30,30],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1],"split_conditions":[2.8937003E2,2.942787E6,2.95279E9,9.2433E4,9.366E4,1.2968256E-2,4.6480955E-3,-1.0174001E-2,2.4126838E11,7.6363635E-1,2.207872E-1,2.4557575E1,2.0161922E-1,6.496868E-3,1.5535147E-3,1.3919356E5,8.517173E3,5.9775025E-1,6.8E1,-2.148222E-3,-7.867692E-3,-7.92647E-4,-3.9824406E-3,1E0,3.2970735E-3,7.660684E-4,-3.8790284E-3,5.910601E-3,4.4155237E-5,-1.5095895E-3,5.3278846E-1,4.8930116E-4,2.7445937E-3],"split_indices":[57,52,7,1,30,0,0,0,32,58,40,57,28,0,0,34,4,28,10,0,0,0,0,102,0,0,0,0,0,0,28,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9E1,9E0,4.3E1,4.7E1,5E0,4E0,6E0,3.7E1,1.3E1,3.4E1,2.1E1,1.6E1,9E0,4E0,1.1E1,2.3E1,1.3E1,8E0,4E0,1.2E1,4E0,7E0,1.8E1,5E0,4E0,9E0,4E0,4E0,7E0,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.1812337E-2,-4.2948756E-2,1.2545536E-1,-1.0358797E-1,3.6788717E-2,1.3685177E-2,7.398414E-2,-9.595848E-3,-8.0722906E-2,1.05620585E-1,-5.7439685E-2,1.160214E-1,-3.3475934E-3,-4.8523424E-3,-4.3138888E-2,5.4669747E-4,1.5317461E-1,7.3793606E-4,-3.1632595E-3,5.3820442E-2,5.7059852E-3,1.9388814E-2,-7.952252E-2,6.6943145E-3,3.05605E-3,-1.1515613E-3,3.6343448E-3,-7.71638E-4,2.3259541E-3,-4.2981757E-3,-1.5848794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,-1,-1,21,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0861155E-1,3.761226E-1,4.5492786E-1,1.4173093E-1,2.26588E-1,0E0,2.3501164E-1,0E0,7.490218E-2,8.373228E-2,3.704451E-2,5.945933E-2,0E0,0E0,5.841647E-2,0E0,1.3032317E-2,0E0,0E0,5.1396437E-2,0E0,1.9734863E-2,1.730533E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,14,14,16,16,19,19,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,-1,-1,22,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,2.3514317E8,1.6451234E-2,4.570007E7,1.5046106E7,1.3685177E-2,1.650269E10,-9.595848E-3,1.9121604E-4,1.118632E3,3.9343938E-1,1E0,-3.3475934E-3,-4.8523424E-3,3.25E2,5.4669747E-4,2.326E4,7.3793606E-4,-3.1632595E-3,4.4E1,5.7059852E-3,1.33125E1,1.0613095E1,6.6943145E-3,3.05605E-3,-1.1515613E-3,3.6343448E-3,-7.71638E-4,2.3259541E-3,-4.2981757E-3,-1.5848794E-3],"split_indices":[2,7,58,5,46,0,5,0,39,53,28,102,0,0,0,0,9,0,0,3,0,55,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.6E1,3.6E1,4.3E1,3.3E1,5E0,3.1E1,4E0,3.9E1,1.9E1,1.4E1,2.5E1,6E0,1.5E1,2.4E1,7E0,1.2E1,4E0,1E1,1.1E1,1.4E1,9E0,1.5E1,7E0,5E0,4E0,7E0,5E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.9333752E-2,-1.7338527E-2,1.7851615E-1,3.232561E-2,-1.274609E-1,3.1809425E-1,-1.1914452E-2,-4.1906007E-2,1.1910658E-1,-1.8609008E-1,-5.4517087E-2,6.789022E-3,1.4265265E-2,3.2805654E-3,-4.054831E-3,-1.5610947E-2,-5.5327644E-3,1.4730425E-1,-1.2428112E-3,-2.8834061E-3,-7.964071E-3,-3.2274926E-3,3.8824248E-5,4.5374773E-2,-5.4154005E-2,1.8340005E-1,2.047027E-3,1.3910569E-4,3.409216E-3,-2.6057994E-3,1.2766357E-4,3.193395E-3,7.889118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,25,-1,-1,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9269094E-1,4.5805824E-1,5.088089E-1,3.78621E-1,1.01047456E-1,3.5895586E-2,1.0307721E-1,9.129667E-2,1.2195939E-1,3.3692688E-2,2.530377E-2,0E0,0E0,0E0,0E0,6.592278E-2,0E0,6.482148E-2,0E0,0E0,0E0,0E0,0E0,2.2041727E-2,1.7930765E-2,3.4020007E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,15,15,17,17,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,26,-1,-1,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.5081426E3,3.494164E7,8.50999E9,1.6987854E1,5.127381E1,2.2059325E5,7.096748E7,7.925698E5,2.8399336E7,2.6078947E1,9.74026E0,6.789022E-3,1.4265265E-2,3.2805654E-3,-4.054831E-3,3.1E1,-5.5327644E-3,1.4922581E1,-1.2428112E-3,-2.8834061E-3,-7.964071E-3,-3.2274926E-3,3.8824248E-5,1.5566606E1,3.95875E5,3.1346828E5,2.047027E-3,1.3910569E-4,3.409216E-3,-2.6057994E-3,1.2766357E-4,3.193395E-3,7.889118E-3],"split_indices":[53,46,5,57,57,29,46,29,46,57,54,0,0,0,0,3,0,55,0,0,0,0,0,55,1,29,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,8.2E1,1.8E1,5.7E1,2.5E1,1E1,8E0,3.1E1,2.6E1,1.3E1,1.2E1,5E0,5E0,4E0,4E0,2.6E1,5E0,2.2E1,4E0,4E0,9E0,7E0,5E0,1E1,1.6E1,1.5E1,7E0,6E0,4E0,1.2E1,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.561637E-3,-5.404985E-2,1.2403449E-1,-1.3783206E-1,1.10711185E-2,1.651183E-1,-5.31708E-3,-1.145633E-2,-1.00857444E-1,9.28129E-2,-4.8225813E-2,1.0642485E-1,1.1308259E-2,-1.2210609E-1,7.6926546E-4,5.9804716E-3,5.837162E-3,-4.2109964E-3,-2.0482969E-2,8.411137E-3,4.805522E-2,-1.4793777E-1,-1.3805073E-3,1.3471049E-3,-9.584615E-4,2.6229895E-3,-2.1830723E-3,-1.8132571E-3,9.3190506E-2,-2.3490298E-3,-1.7480707E-1,-7.038803E-4,1.5131987E-3,4.841804E-3,1.0801419E-3,-3.381597E-3,-7.7192737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,-1,21,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,31,-1,-1,33,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[7.356404E-1,3.998124E-1,4.032969E-1,1.8036765E-1,2.0805155E-1,2.2744715E-1,0E0,0E0,7.5285286E-2,1.06672496E-1,4.505991E-2,1.586996E-1,0E0,4.633802E-2,0E0,1.0163315E-2,0E0,0E0,1.7712517E-2,0E0,8.0090515E-2,2.8707564E-2,0E0,0E0,0E0,1.2115218E-2,0E0,0E0,2.705706E-2,0E0,2.1117866E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,15,15,18,18,20,20,21,21,25,25,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,-1,22,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,32,-1,-1,34,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,3.2334878E7,3.6181125E9,2.7571955E6,1.5046106E7,1.1811385E4,-5.31708E-3,-1.145633E-2,2.834008E-1,2.710413E5,7.898E3,2.70482E5,1.1308259E-2,3.159013E3,7.6926546E-4,3.8826916E-1,5.837162E-3,-4.2109964E-3,1.1787824E7,8.411137E-3,1.9649E4,6.059339E2,-1.3805073E-3,1.3471049E-3,-9.584615E-4,4.4178323E8,-2.1830723E-3,-1.8132571E-3,7.118011E-1,-2.3490298E-3,2.77E2,-7.038803E-4,1.5131987E-3,4.841804E-3,1.0801419E-3,-3.381597E-3,-7.7192737E-3],"split_indices":[2,12,7,46,46,53,0,0,58,29,9,30,0,4,0,28,0,0,51,0,2,34,0,0,0,12,0,0,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.2E1,3.3E1,3.1E1,4.1E1,2.9E1,4E0,4E0,2.7E1,1.7E1,2.4E1,2.2E1,7E0,2.3E1,4E0,8E0,9E0,6E0,1.8E1,6E0,1.6E1,1.7E1,6E0,4E0,4E0,1.2E1,6E0,5E0,1.1E1,5E0,1.2E1,8E0,4E0,6E0,5E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.1684482E-2,-8.92959E-2,4.733844E-2,-1.14849634E-1,5.3549916E-3,-5.167489E-2,8.901832E-2,-8.102203E-2,-1.0009397E-2,-7.7019674E-3,-6.255158E-3,5.3949416E-2,7.921108E-3,-1.3699213E-1,-2.659785E-2,-1.7146335E-3,2.9380014E-3,2.9502569E-2,4.477201E-3,-4.9719807E-2,-7.2869565E-3,3.1689203E-3,-6.0676824E-2,3.0852994E-3,1.0357463E-2,-3.169842E-3,-6.121934E-5,-3.8326145E-3,-2.2772465E-2,-1.8357523E-3,3.135E-2,-1.7745014E-3,4.850933E-4,1.9359319E-3,-6.650146E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,21,-1,-1,23,-1,25,-1,-1,27,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[4.9895924E-1,3.0242434E-1,2.604656E-1,2.1375608E-1,0E0,1.00247085E-1,1.8935111E-1,1.0906297E-1,0E0,5.7033602E-2,0E0,5.800236E-2,0E0,9.183115E-2,8.327968E-2,0E0,0E0,2.9110642E-2,0E0,1.6310014E-2,0E0,0E0,2.516535E-2,0E0,2.9860783E-2,0E0,0E0,0E0,1.0211804E-2,0E0,1.9832205E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,14,14,17,17,19,19,22,22,24,24,28,28,30,30],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,22,-1,-1,24,-1,26,-1,-1,28,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[2.870791E6,1.0098112E2,1E0,1E0,5.3549916E-3,5.9E1,1.5435694E9,1.4780533E-4,-1.0009397E-2,8.517173E3,-6.255158E-3,2.4624854E8,7.921108E-3,8.8824E4,5.739445E4,-1.7146335E-3,2.9380014E-3,3.162046E6,4.477201E-3,7.366E3,-7.2869565E-3,3.1689203E-3,1.94E2,3.0852994E-3,1.091E3,-3.169842E-3,-6.121934E-5,-3.8326145E-3,1.5285803E5,-1.8357523E-3,2.727403E7,-1.7745014E-3,4.850933E-4,1.9359319E-3,-6.650146E-4],"split_indices":[49,57,102,105,0,3,7,39,0,4,0,33,0,12,52,0,0,33,0,9,0,0,10,0,10,0,0,0,29,0,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,4.6E1,6.1E1,4.2E1,4E0,1.8E1,4.3E1,3.6E1,6E0,1.4E1,4E0,3.5E1,8E0,1.7E1,1.9E1,1E1,4E0,2.7E1,8E0,8E0,9E0,4E0,1.5E1,6E0,2.1E1,4E0,4E0,6E0,9E0,5E0,1.6E1,5E0,4E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[6.662455E-4,-4.9558103E-2,1.4785257E-1,-2.4057561E-1,-2.6587768E-2,2.1840347E-1,1.255075E-2,-3.0996343E-3,-1.2534267E-2,4.097304E-2,-9.346684E-2,1.2528654E-2,1.323453E-1,-2.1566772E-3,2.252529E-3,-3.8970392E-2,1.1886782E-1,-1.62801E-1,-5.253818E-2,1.8877957E-3,5.8528706E-3,-2.51236E-3,2.2326184E-4,1.6014557E-1,6.9791457E-4,-2.4208599E-3,-7.0182164E-3,-2.220887E-2,-4.260263E-3,2.6866517E-3,6.7511033E-3,-2.7233032E-3,-2.309643E-3,1.3026467E-3,-7.8897056E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1,33,-1,-1,-1],"loss_changes":[8.426613E-1,3.6656123E-1,2.7408314E-1,1.18638515E-1,3.5174674E-1,1.6438305E-1,4.335879E-2,0E0,0E0,2.474922E-1,1.0211915E-1,0E0,2.2506237E-2,0E0,0E0,2.808135E-2,7.9077154E-2,2.9314399E-2,4.900237E-2,0E0,0E0,0E0,0E0,1.831019E-2,0E0,0E0,0E0,1.5288202E-2,0E0,0E0,0E0,1.1181185E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,15,15,16,16,17,17,18,18,23,23,27,27,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1,34,-1,-1,-1],"split_conditions":[1.3137E4,5.7578644E2,2.797345E6,1.44E2,2.8399336E7,6.4417176E-2,1.7313588E7,-3.0996343E-3,-1.2534267E-2,1.6987854E1,1.0951729E3,1.2528654E-2,1.2943E4,-2.1566772E-3,2.252529E-3,1.0289962E3,1.8831E5,3.5E1,8.661788E7,1.8877957E-3,5.8528706E-3,-2.51236E-3,2.2326184E-4,1.7694729E3,6.9791457E-4,-2.4208599E-3,-7.0182164E-3,1.3163732E7,-4.260263E-3,2.6866517E-3,6.7511033E-3,9.985778E0,-2.309643E-3,1.3026467E-3,-7.8897056E-4],"split_indices":[2,4,30,0,46,58,1,0,0,57,34,0,9,0,0,4,30,3,46,0,0,0,0,4,0,0,0,51,0,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,8.4E1,2.8E1,8E0,7.6E1,1.8E1,1E1,4E0,4E0,3.8E1,3.8E1,6E0,1.2E1,4E0,6E0,1.9E1,1.9E1,1.3E1,2.5E1,4E0,8E0,1.1E1,8E0,1.3E1,6E0,4E0,9E0,1.8E1,7E0,4E0,9E0,1.3E1,5E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.530839E-3,-4.5932192E-2,1.1182062E-1,-1.1521912E-1,-8.350247E-3,2.2884932E-1,-2.427885E-2,-4.7911853E-2,-1.6587082E-1,3.1141045E-2,-6.1822448E-2,1.0794275E-2,2.0518857E-3,1.2534768E-3,-4.1092574E-3,-3.2648654E-3,5.245126E-5,-2.0179935E-1,-1.9370661E-3,-1.724491E-2,9.574152E-2,-5.9825747E-4,-3.811307E-3,-3.4863604E-3,-8.958482E-3,-5.2409623E-2,2.2318675E-3,2.1035306E-3,5.2006897E-3,-2.9059898E-3,-2.249932E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,-1,-1,23,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[4.5526692E-1,1.9954106E-1,3.8781723E-1,8.4101945E-2,1.0962121E-1,1.3984585E-1,6.734538E-2,2.7418276E-2,4.9649537E-2,9.542371E-2,4.18005E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.9676288E-2,0E0,5.3284883E-2,1.5143432E-2,0E0,0E0,0E0,0E0,1.6356282E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,17,17,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,-1,-1,24,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,3.572374E6,6.1358623E9,8.8824E4,2.7260774E7,1.343964E1,1.063036E8,2.69E2,2.1272728E1,1.118632E3,5.4357555E6,1.0794275E-2,2.0518857E-3,1.2534768E-3,-4.1092574E-3,-3.2648654E-3,5.245126E-5,5.432393E-1,-1.9370661E-3,1.8116838E1,3.6742297E5,-5.9825747E-4,-3.811307E-3,-3.4863604E-3,-8.958482E-3,2.1562665E10,2.2318675E-3,2.1035306E-3,5.2006897E-3,-2.9059898E-3,-2.249932E-4],"split_indices":[2,12,5,12,46,55,52,0,59,53,51,0,0,0,0,0,0,28,0,55,29,0,0,0,0,32,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,7.6E1,2.3E1,2.6E1,5E1,1.2E1,1.1E1,1.2E1,1.4E1,2.9E1,2.1E1,8E0,4E0,7E0,4E0,6E0,6E0,1E1,4E0,1.7E1,1.2E1,1.1E1,1E1,4E0,6E0,1.2E1,5E0,8E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.5239592E-2,-6.590716E-2,1.024054E-1,-1.0023185E-1,2.5494773E-2,1.6332898E-1,-3.6385134E-3,-7.927773E-2,-1.0460003E-2,9.016657E-2,-3.0522956E-2,7.439078E-2,2.6976025E-1,-2.9142814E-2,-1.471239E-1,1.1949413E-3,5.076637E-3,5.7299284E-4,-2.0110758E-3,1.0745387E-1,-4.775046E-4,1.2036726E-2,4.5747217E-3,-5.735839E-2,4.455354E-3,-3.456769E-3,-7.4006557E-3,5.036613E-3,1.4834374E-3,-9.322939E-2,-7.7397805E-3,-1.268817E-3,-4.0130746E-3,1.0774358E-3,-2.3945274E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[6.326317E-1,2.3471883E-1,4.1258442E-1,1.9144881E-1,7.8830265E-2,2.1657747E-1,0E0,1.6578552E-1,0E0,2.4136253E-2,1.4902991E-2,4.4874497E-2,5.7891726E-2,1.3620666E-1,4.2802185E-2,0E0,0E0,0E0,0E0,1.7666847E-2,0E0,0E0,0E0,4.5555018E-2,0E0,0E0,0E0,0E0,0E0,1.171273E-2,2.8401129E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,19,19,23,23,29,29,30,30],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.7249353E3,1.4112683E8,9.302862E7,8.674699E-1,1.8831E5,1.1681136E4,-3.6385134E-3,3.3012803E9,-1.0460003E-2,3.4E1,1.7108389E5,7.2266235E6,8.838004E0,6.444E3,7.444956E7,1.1949413E-3,5.076637E-3,5.7299284E-4,-2.0110758E-3,1.0007143E1,-4.775046E-4,1.2036726E-2,4.5747217E-3,1.2337662E-1,4.455354E-3,-3.456769E-3,-7.4006557E-3,5.036613E-3,1.4834374E-3,5.1612902E-2,3.727E3,-1.268817E-3,-4.0130746E-3,1.0774358E-3,-2.3945274E-3],"split_indices":[53,12,46,58,30,4,0,5,0,3,29,48,54,2,7,0,0,0,0,54,0,0,0,58,0,0,0,0,0,58,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,7.3E1,3.1E1,5.3E1,2E1,2.4E1,7E0,4.9E1,4E0,9E0,1.1E1,1.4E1,1E1,2.9E1,2E1,5E0,4E0,4E0,7E0,1E1,4E0,6E0,4E0,2.5E1,4E0,1.2E1,8E0,6E0,4E0,1.4E1,1.1E1,4E0,1E1,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8842183E-2,-4.7217514E-3,1.8946552E-1,-4.280184E-2,6.544878E-2,2.4351479E-1,1.4723232E-3,-6.7809536E-3,-1.3468155E-1,1.1246225E-1,3.279852E-4,4.6755387E-3,1.1651745E-2,-4.3688595E-2,1.0120317E-1,-7.6954566E-2,-6.5992386E-3,5.445598E-3,3.8187767E-4,1.6872303E-3,-1.9045807E-3,-9.187959E-2,-1.407047E-2,5.62055E-3,1.6945435E-4,-4.080795E-3,-9.201279E-4,-4.730798E-3,-1.1149936E-3,8.1509864E-4,-1.5682246E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6157434E-1,2.3527406E-1,1.3472217E-1,1.8773326E-1,9.487128E-2,7.561958E-2,0E0,1.7180988E-1,3.0367315E-2,6.87775E-2,3.6951836E-2,0E0,0E0,4.4381678E-2,5.7103172E-2,1.3830885E-2,0E0,0E0,0E0,0E0,0E0,2.461379E-2,2.35815E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,14,14,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.621782E3,6.402099E-1,9.299267E9,8.5154843E-1,2.5588E4,1.3772013E4,1.4723232E-3,4.3316594E5,9.067368E5,1.4922581E1,2.4E1,4.6755387E-3,1.1651745E-2,1E0,2.0477528E1,7.158023E7,-6.5992386E-3,5.445598E-3,3.8187767E-4,1.6872303E-3,-1.9045807E-3,3.0387878E3,3.3166495E6,5.62055E-3,1.6945435E-4,-4.080795E-3,-9.201279E-4,-4.730798E-3,-1.1149936E-3,8.1509864E-4,-1.5682246E-3],"split_indices":[53,43,12,28,9,4,0,29,33,55,3,0,0,17,59,7,0,0,0,0,0,4,33,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,8.6E1,1.7E1,5.6E1,3E1,1.2E1,5E0,4.1E1,1.5E1,1.7E1,1.3E1,6E0,6E0,3.1E1,1E1,8E0,7E0,1.2E1,5E0,7E0,6E0,1.1E1,2E1,6E0,4E0,4E0,4E0,6E0,5E0,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.5199473E-3,-2.00504E-2,1.7427467E-1,-1.5459003E-1,8.918726E-3,1.0275923E-2,-2.137881E-3,-2.296439E-3,-6.865453E-3,3.375763E-2,-5.6560256E-2,-3.2112934E-4,9.341152E-2,-1.1457382E-1,6.5951557E-3,-3.560842E-2,5.145497E-2,1.3627262E-1,-2.0416873E-4,-6.3188877E-3,-1.2646198E-3,1.8932971E-3,-1.4567714E-3,-3.0581884E-3,-7.04844E-3,8.38519E-4,3.728294E-3,2.2793598E-4,6.773822E-3,8.1962877E-4,-1.2967597E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2892936E-1,3.5215333E-1,3.5181886E-1,3.968811E-2,1.2389132E-1,0E0,0E0,0E0,0E0,1.1184118E-1,7.769693E-2,6.76056E-2,8.569278E-2,4.676324E-2,2.579118E-2,3.0252742E-2,1.9541252E-2,9.224725E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3699039E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3509795E3,4.1964554E-4,3.9858868E1,3.3109E4,8.63558E-1,1.0275923E-2,-2.137881E-3,-2.296439E-3,-6.865453E-3,3.1982497E5,1.020707E1,2.0699982E8,2.8399336E7,1.06993E-1,1.6E1,1E0,3.2736E4,1.688E3,-2.0416873E-4,-6.3188877E-3,-1.2646198E-3,1.8932971E-3,-1.4567714E-3,-3.0581884E-3,1.6872E5,8.38519E-4,3.728294E-3,2.2793598E-4,6.773822E-3,8.1962877E-4,-1.2967597E-3],"split_indices":[53,43,58,33,28,0,0,0,0,29,55,12,46,39,8,8,9,2,0,0,0,0,0,0,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,8.9E1,1.2E1,1.5E1,7.4E1,8E0,4E0,5E0,1E1,5.4E1,2E1,3.5E1,1.9E1,1E1,1E1,2.1E1,1.4E1,1.3E1,6E0,5E0,5E0,5E0,5E0,7E0,1.4E1,1E1,4E0,4E0,9E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.084436E-4,-3.0529765E-2,1.429765E-1,-1.0845123E-2,-1.2839371E-2,9.87389E-2,8.60141E-3,4.3266765E-3,-9.543071E-3,2.2865399E-3,4.8879883E-3,-1.3061572E-2,8.913715E-3,-1.0353696E-1,5.561586E-3,-1.5558228E-3,-4.6346034E-3,-1.9769596E-2,6.4358704E-2,2.68207E-3,-1.3542122E-3,4.7533456E-3,3.4518127E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,-1,13,-1,15,17,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[4.4434074E-1,3.876115E-1,5.233702E-2,0E0,3.3787614E-1,9.59219E-3,0E0,3.1534722E-1,0E0,0E0,0E0,1.1852147E-1,0E0,1.2480959E-2,8.914697E-2,0E0,0E0,7.286841E-2,6.512507E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,11,11,13,13,14,14,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,-1,14,-1,16,18,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.7123758E0,5.279518E7,9.683623E6,-1.0845123E-2,1.5081978E8,3.1346828E5,8.60141E-3,1.3127055E4,-9.543071E-3,2.2865399E-3,4.8879883E-3,6.7281544E-5,8.913715E-3,1.6278E4,1.1392E4,-1.5558228E-3,-4.6346034E-3,2.0477528E1,2.483304E0,2.68207E-3,-1.3542122E-3,4.7533456E-3,3.4518127E-4],"split_indices":[40,5,1,0,46,29,0,4,0,0,0,39,0,12,2,0,0,59,58,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,8.1E1,1.7E1,4E0,7.7E1,1.3E1,4E0,7.3E1,4E0,8E0,5E0,6.9E1,4E0,1.1E1,5.8E1,4E0,7E0,4.1E1,1.7E1,6E0,3.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.2409871E-2,-5.640977E-2,1.13069564E-1,-1.3070142E-1,9.130545E-3,5.9516463E-2,2.403144E-1,-9.858008E-3,-9.282424E-2,5.1835133E-3,-1.718563E-2,1.0855935E-1,-4.8438333E-2,1.14710005E-2,4.641702E-3,-1.2477392E-3,-1.167992E-1,-4.7894478E-2,5.2092396E-2,1.3644609E-1,2.4571328E-4,-3.8370572E-3,-7.3106407E-6,-4.8090126E-3,-1.7152494E-3,-6.056952E-2,2.640781E-4,3.30507E-3,-2.0470054E-4,5.9417128E-3,2.913899E-3,-6.979562E-4,-2.8729707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,19,21,-1,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.683883E-1,3.2255393E-1,2.9234338E-1,1.4180148E-1,1.2983838E-1,1.7997411E-1,7.1611345E-2,0E0,3.228198E-2,0E0,6.844584E-2,6.4335525E-2,3.0669386E-2,0E0,0E0,0E0,1.4129981E-2,1.5838135E-2,2.330812E-2,1.4938563E-2,0E0,0E0,0E0,0E0,0E0,1.2570299E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,11,11,12,12,16,16,17,17,18,18,19,19,25,25],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,20,22,-1,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,1.04856936E8,1.1803382E4,5.279518E7,1.8955729E1,6.19999E9,1.0889231E1,-9.858008E-3,3.566468E5,5.1835133E-3,4.664004E8,7.5305285E6,2.074635E-1,1.14710005E-2,4.641702E-3,-1.2477392E-3,8.0168776E-2,7.879123E7,8.63558E-1,1.343964E1,2.4571328E-4,-3.8370572E-3,-7.3106407E-6,-4.8090126E-3,-1.7152494E-3,2.1821466E5,2.640781E-4,3.30507E-3,-2.0470054E-4,5.9417128E-3,2.913899E-3,-6.979562E-4,-2.8729707E-3],"split_indices":[2,7,4,5,59,5,55,0,29,0,12,48,40,0,0,0,58,33,28,55,0,0,0,0,0,29,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,6.5E1,4.4E1,3E1,3.5E1,3.2E1,1.2E1,5E0,2.5E1,5E0,3E1,2.2E1,1E1,6E0,6E0,8E0,1.7E1,2.1E1,9E0,1.7E1,5E0,4E0,6E0,1.3E1,4E0,1.7E1,4E0,5E0,4E0,1E1,7E0,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.3187193E-2,-3.4000084E-2,1.1714888E-1,-5.2852556E-2,8.351912E-2,1.09519195E-2,7.9743795E-2,-7.7567585E-3,-4.0347572E-2,4.412856E-3,1.761286E-4,1.5121083E-1,1.9125398E-2,-9.946081E-3,-1.1433393E-1,2.8071261E-3,7.4288077E-3,-2.3928876E-3,5.4361384E-2,-2.4444059E-2,4.5282017E-3,-5.2205385E-3,-1.3526303E-3,2.845917E-3,3.1054555E-4,1.2113955E-2,-5.061873E-2,-8.5555745E-4,1.8158134E-3,-3.2026474E-3,-1.18605E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,-1,15,17,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1],"loss_changes":[5.9756905E-1,1.905185E-1,2.404775E-1,1.3921364E-1,3.5934314E-2,0E0,1.4007121E-1,0E0,1.5406421E-1,0E0,0E0,4.411444E-2,6.078022E-2,1.0119589E-1,4.111421E-2,0E0,0E0,0E0,1.499597E-2,4.463032E-2,0E0,0E0,0E0,0E0,0E0,2.853866E-2,1.535771E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,11,11,12,12,13,13,14,14,18,18,19,19,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,-1,16,18,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.8319664E0,1.6451234E-2,5.1114212E7,9.5032835E0,1.09519195E-2,1.5E1,-7.7567585E-3,5.501383E9,4.412856E-3,1.761286E-4,8.220836E8,1.9649E4,1E0,2.711196E3,2.8071261E-3,7.4288077E-3,-2.3928876E-3,8.106617E-1,1.4163358E7,4.5282017E-3,-5.2205385E-3,-1.3526303E-3,2.845917E-3,3.1054555E-4,1.4243386E1,7.48E2,-8.5555745E-4,1.8158134E-3,-3.2026474E-3,-1.18605E-3],"split_indices":[2,40,58,5,54,0,8,0,5,0,0,7,2,89,53,0,0,0,28,46,0,0,0,0,0,57,11,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.2E2,8.3E1,3.7E1,7.2E1,1.1E1,5E0,3.2E1,4E0,6.8E1,7E0,4E0,1.4E1,1.8E1,4.9E1,1.9E1,7E0,7E0,5E0,1.3E1,4.5E1,4E0,1.3E1,6E0,8E0,5E0,1.9E1,2.6E1,1E1,9E0,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.138847E-2,-1.6537586E-2,1.378826E-1,1.3055443E-3,-1.792534E-1,2.6409472E-3,7.540905E-3,-3.0678686E-2,1.1850852E-1,-1.8756153E-3,-9.773239E-3,7.5826794E-3,-6.916983E-2,8.814884E-3,5.13998E-2,-2.657893E-2,5.8283474E-2,-9.322858E-2,-4.540672E-4,5.1580014E-4,3.3329213E-3,-2.841676E-3,-4.7905644E-4,3.9274837E-3,2.4581605E-4,-4.93551E-3,-6.0926672E-2,1.3738719E-3,-8.502662E-4,-5.129459E-4,-3.2671015E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[3.9085332E-1,2.6443976E-1,7.662693E-2,3.1477338E-1,9.0713054E-2,0E0,0E0,9.7724244E-2,1.3923103E-1,0E0,0E0,6.0437817E-2,4.3758735E-2,0E0,1.842231E-2,2.8535325E-2,3.5653424E-2,2.3972481E-2,0E0,0E0,0E0,0E0,1.4231859E-2,0E0,0E0,0E0,1.9434478E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[2.0935743E0,1.1384717E8,4.4993465E1,1.1855E4,9.546808E0,2.6409472E-3,7.540905E-3,2.8399336E7,1.4994E4,-1.8756153E-3,-9.773239E-3,2.4662777E1,2.5E-1,8.814884E-3,4.3E1,1.92E2,1.4922581E1,2.1750195E3,-4.540672E-4,5.1580014E-4,3.3329213E-3,-2.841676E-3,3.1E1,3.9274837E-3,2.4581605E-4,-4.93551E-3,4.8861527E9,1.3738719E-3,-8.502662E-4,-5.129459E-4,-3.2671015E-3],"split_indices":[43,46,57,2,54,0,0,46,10,0,0,57,58,0,8,0,55,4,0,0,0,0,3,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,8.2E1,8E0,1.1E1,8E0,6.5E1,1.7E1,4E0,4E0,3.3E1,3.2E1,5E0,1.2E1,2E1,1.3E1,2.2E1,1E1,7E0,5E0,6E0,1.4E1,6E0,7E0,8E0,1.4E1,5E0,9E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-4.325153E-3,-6.465679E-2,5.7140812E-2,3.4588529E-3,-8.501403E-2,7.3060505E-2,-4.4419733E-3,-6.85497E-2,-7.2139436E-3,4.150803E-2,1.5799184E-1,-1.1530733E-1,-3.8210128E-2,-1.877802E-3,7.709088E-2,2.2758744E-3,7.3556066E-3,-1.5388815E-1,-1.1804346E-3,6.839154E-4,-5.6428324E-2,5.9329014E-4,-2.0374889E-3,1.4138842E-4,9.313434E-2,-2.7908136E-3,-7.1321437E-3,-9.8254375E-2,-1.6696407E-2,4.0959646E-3,1.2126976E-3,-4.4327853E-3,-1.5981115E-3,8.26462E-4,-1.5993807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,15,17,19,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2644554E-1,1.9760542E-1,1.7248571E-1,0E0,8.458957E-2,1.3547117E-1,0E0,6.22686E-2,0E0,6.189643E-2,4.797116E-2,5.2338034E-2,3.2305997E-2,1.9908579E-2,2.5462687E-2,0E0,0E0,2.4856001E-2,0E0,0E0,3.6581583E-2,0E0,0E0,0E0,1.7965809E-2,0E0,0E0,9.131268E-3,1.5220466E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,20,20,24,24,27,27,28,28],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,16,18,20,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3724032E8,8.977316E0,2.4692087E1,3.4588529E-3,6.2640357E-1,1.5791555E9,-4.4419733E-3,5.048584E-4,-7.2139436E-3,9.8309404E-1,2.4190365E6,8.670369E-1,1.0465239E-3,2.2087644E9,1.601247E1,2.2758744E-3,7.3556066E-3,3.08E2,-1.1804346E-3,6.839154E-4,2.8900872E6,5.9329014E-4,-2.0374889E-3,1.4138842E-4,2.3559767E5,-2.7908136E-3,-7.1321437E-3,4E0,5.661451E-1,4.0959646E-3,1.2126976E-3,-4.4327853E-3,-1.5981115E-3,8.26462E-4,-1.5993807E-3],"split_indices":[7,55,55,0,39,7,0,43,0,40,33,28,39,12,57,0,0,0,0,0,51,0,0,0,34,0,0,8,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.7E1,5.6E1,6E0,5.1E1,5.2E1,4E0,4.6E1,5E0,3.9E1,1.3E1,1.7E1,2.9E1,1.8E1,2.1E1,5E0,8E0,1.1E1,6E0,7E0,2.2E1,1.4E1,4E0,4E0,1.7E1,5E0,6E0,1E1,1.2E1,1.2E1,5E0,6E0,4E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8995876E-3,1.7307267E-2,-8.49541E-3,-1.9665277E-2,1.0299962E-1,-5.0792895E-2,5.0558295E-2,9.836733E-3,6.644898E-2,-6.829946E-2,2.3107193E-3,7.184726E-2,1.5602639E-4,-6.355511E-4,9.254501E-2,-4.8872404E-2,-5.313775E-3,6.652992E-4,3.8057142E-3,1.2834902E-1,1.5851419E-3,-3.2241517E-3,-1.1584569E-3,5.7846876E-3,2.095763E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[3.61006E-1,3.156837E-1,0E0,1.5558049E-1,1.6771504E-1,1.0284625E-1,2.134474E-2,0E0,5.865871E-2,5.8703825E-2,0E0,2.4953179E-2,0E0,0E0,2.9194072E-2,2.1565579E-2,0E0,0E0,0E0,1.5350193E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,14,14,15,15,19,19],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.4783066E1,-8.49541E-3,7.307441E-1,1.0822511E-2,1E0,4.3E1,9.836733E-3,1.9956842E5,1E0,2.3107193E-3,5.3278846E-1,1.5602639E-4,-6.355511E-4,5.8639234E11,1E0,-5.313775E-3,6.652992E-4,3.8057142E-3,1.2271712E1,1.5851419E-3,-3.2241517E-3,-1.1584569E-3,5.7846876E-3,2.095763E-3],"split_indices":[46,57,0,40,58,89,3,0,34,84,0,28,0,0,32,8,0,0,0,55,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.8E1,5E0,6.9E1,2.9E1,4.8E1,2.1E1,4E0,2.5E1,4.2E1,6E0,1.4E1,7E0,6E0,1.9E1,3.5E1,7E0,6E0,8E0,1E1,9E0,9E0,2.6E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.299195E-3,-1.4052154E-2,1.7097886E-1,-7.453978E-2,3.1034399E-2,9.727058E-3,2.7355868E-3,-3.8559325E-2,-1.0469606E-2,8.58175E-2,-1.615867E-2,7.1115796E-3,-1.2693503E-1,2.869411E-3,1.344417E-1,-1.118793E-3,2.5316318E-3,4.2062886E-3,-2.4483189E-2,-1.6516506E-3,-5.701923E-3,2.0075378E-3,-1.2711645E-3,5.727348E-3,2.787558E-3,-6.625224E-2,3.2847042E-3,-3.2566807E-3,1.6193952E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[3.9381814E-1,2.7503112E-1,1.0143152E-1,3.221581E-1,1.513389E-1,0E0,0E0,1.5427823E-1,0E0,1.0851292E-1,4.3119304E-2,9.2717715E-2,2.5492847E-2,2.404739E-2,1.1293352E-2,0E0,0E0,0E0,1.0730756E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.751474E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,25,25],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[1.0825377E4,3.2316522E7,1.3309696E7,2.915E4,1.5046106E7,9.727058E-3,2.7355868E-3,4.352552E9,-1.0469606E-2,9.489487E2,3.9887E4,5.2287582E-2,3.240192E6,1.0875838E7,7.6363635E-1,-1.118793E-3,2.5316318E-3,4.2062886E-3,2.6666668E-1,-1.6516506E-3,-5.701923E-3,2.0075378E-3,-1.2711645E-3,5.727348E-3,2.787558E-3,1.1674918E3,3.2847042E-3,-3.2566807E-3,1.6193952E-4],"split_indices":[53,12,48,9,46,0,0,5,0,53,10,58,7,33,58,0,0,0,58,0,0,0,0,0,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,9.9E1,1.2E1,4.2E1,5.7E1,5E0,7E0,3.7E1,5E0,2.6E1,3.1E1,2.5E1,1.2E1,1E1,1.6E1,2.7E1,4E0,5E0,2E1,4E0,8E0,4E0,6E0,1E1,6E0,1.5E1,5E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.852963E-4,1.2467966E-2,-8.251737E-3,-7.0391726E-3,1.7803189E-1,-4.2995278E-2,5.208442E-2,9.643615E-3,2.1401076E-3,-5.983899E-2,3.2737432E-3,-4.583001E-4,7.522155E-2,-1.2944703E-2,-9.8201685E-2,-1.5455259E-3,2.433131E-3,9.761275E-2,1.33563E-4,-2.6628922E-3,1.014157E-2,-7.3017746E-2,-7.3823784E-3,4.3414882E-3,3.6716516E-4,2.971794E-3,-7.7672035E-4,-3.8003572E-3,-1.4499791E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2955632E-1,3.371398E-1,0E0,2.0206968E-1,1.00533515E-1,1.3826767E-1,4.379922E-2,0E0,0E0,9.407537E-2,0E0,3.7368078E-2,3.9675504E-2,3.584695E-2,6.268868E-2,0E0,0E0,3.6391586E-2,0E0,0E0,4.5179658E-2,2.225922E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,17,17,20,20,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.1730811E4,-8.251737E-3,6.402099E-1,8.50999E9,1E0,1.6511278E1,9.643615E-3,2.1401076E-3,7.059086E-1,3.2737432E-3,1.7618678E0,2.2606992E5,1.2179555E-1,1.4342E4,-1.5455259E-3,2.433131E-3,1.3021952E1,1.33563E-4,-2.6628922E-3,7.0843E5,8.0168776E-2,-7.3823784E-3,4.3414882E-3,3.6716516E-4,2.971794E-3,-7.7672035E-4,-3.8003572E-3,-1.4499791E-3],"split_indices":[46,4,0,43,5,89,57,0,0,28,0,40,34,28,9,0,0,54,0,0,1,58,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,1.03E2,5E0,9.3E1,1E1,5.8E1,3.5E1,5E0,5E0,5.2E1,6E0,1.1E1,2.4E1,2.4E1,2.8E1,7E0,4E0,1.8E1,6E0,6E0,1.8E1,2.4E1,4E0,1.4E1,4E0,5E0,1.3E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[6.055449E-4,-1.9225232E-2,2.0281893E-1,-6.364392E-2,4.574536E-2,9.911284E-3,2.7512023E-3,-4.881323E-2,-7.942117E-3,9.781974E-2,-1.525634E-3,-1.08574174E-1,-1.0936733E-2,2.1899366E-3,5.270318E-3,-1.218257E-3,3.6689226E-2,-4.635602E-2,-6.0139713E-3,4.6248944E-3,-3.2319885E-2,2.4503567E-3,-2.1939763E-5,-2.736184E-3,3.612752E-4,1.9101058E-3,-5.0062187E-2,-3.055991E-3,-2.2767717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[4.4901463E-1,2.9727405E-1,6.987399E-2,1.3069914E-1,1.0353677E-1,0E0,0E0,1.2783408E-1,0E0,2.6939541E-2,2.9556045E-2,7.167393E-2,1.1117197E-1,0E0,0E0,0E0,1.283419E-2,2.0665012E-2,0E0,0E0,5.103659E-2,0E0,0E0,0E0,0E0,0E0,4.032433E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,16,16,17,17,20,20,26,26],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[1.0825377E4,3.4707712E8,2.9548872E9,6.2640357E-1,1.5046106E7,9.911284E-3,2.7512023E-3,1.4616883E1,-7.942117E-3,8.5731603E8,1.8489E4,3.9024E4,1.7593515E1,2.1899366E-3,5.270318E-3,-1.218257E-3,8.106617E-1,5.348407E0,-6.0139713E-3,4.6248944E-3,4.5479352E2,2.4503567E-3,-2.1939763E-5,-2.736184E-3,3.612752E-4,1.9101058E-3,3.7E1,-3.055991E-3,-2.2767717E-4],"split_indices":[53,7,7,39,46,0,0,59,0,5,10,12,59,0,0,0,28,59,0,0,53,0,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.01E2,9E0,6E1,4.1E1,5E0,4E0,5.6E1,4E0,1.9E1,2.2E1,2.1E1,3.5E1,1.2E1,7E0,1.2E1,1E1,1.1E1,1E1,4E0,3.1E1,5E0,5E0,7E0,4E0,5E0,2.6E1,1.4E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.9022836E-3,1.2229664E-2,-7.96542E-3,-5.760372E-3,1.8077672E-1,-1.9990418E-2,1.02964014E-1,9.545216E-3,2.2026303E-3,-4.9231723E-2,1.21620875E-2,1.1601846E-3,6.0423976E-3,-3.2194313E-2,-4.5111333E-3,7.429765E-2,-7.5407457E-3,-4.606831E-2,2.075508E-3,4.518162E-3,3.1010047E-4,-2.3640944E-2,1.6765204E-3,-2.268631E-3,-4.11228E-4,5.4941524E-4,-1.2737332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,-1,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1],"loss_changes":[3.0562162E-1,2.8303292E-1,0E0,1.3356629E-1,7.39952E-2,7.205254E-2,4.023634E-2,0E0,0E0,4.7580205E-2,4.585738E-2,0E0,0E0,4.4953216E-2,0E0,2.8995778E-2,2.6300408E-2,1.6730115E-2,0E0,0E0,0E0,1.09398505E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,15,15,16,16,17,17,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,-1,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825377E4,-7.96542E-3,2.1729739E0,6.691E3,7.33E3,3.6742297E5,9.545216E-3,2.2026303E-3,7.925698E5,1.216644E11,1.1601846E-3,6.0423976E-3,1E0,-4.5111333E-3,8.155485E-2,5.2E1,1.4731E4,2.075508E-3,4.518162E-3,3.1010047E-4,6.983E3,1.6765204E-3,-2.268631E-3,-4.11228E-4,5.4941524E-4,-1.2737332E-3],"split_indices":[46,53,0,40,9,2,29,0,0,29,32,0,0,101,0,39,8,9,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,9.2E1,5E0,8.4E1,8E0,7.5E1,9E0,4E0,4E0,3.9E1,3.6E1,5E0,4E0,3.3E1,6E0,8E0,2.8E1,2.9E1,4E0,4E0,4E0,2.2E1,6E0,1.9E1,1E1,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-5.688046E-3,4.129763E-3,-7.962236E-3,-1.5068855E-2,1.6217422E-1,-1.3067493E-1,8.668473E-4,8.916432E-3,3.2096936E-3,-5.2778964E-4,-1.0478153E-2,-1.00261845E-1,1.9732645E-2,-4.993728E-3,-1.965056E-3,5.618472E-3,5.6606433E-3,-3.927957E-2,2.614031E-2,-2.6951663E-3,-7.7076635E-4,2.9834446E-3,2.5671237E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,-1,-1,-1,17,19,21,-1,-1,-1,-1],"loss_changes":[2.4138209E-1,3.3620965E-1,0E0,1.8249744E-1,5.2601337E-2,2.179641E-1,1.6987129E-1,0E0,0E0,0E0,0E0,1.6891032E-2,1.425382E-1,0E0,0E0,0E0,6.4531654E-2,1.2359679E-2,5.10181E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,-1,-1,-1,18,20,22,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.669409E3,-7.962236E-3,3.0894287E2,7.7854166E0,1.1485E4,4.1964554E-4,8.916432E-3,3.2096936E-3,-5.2778964E-4,-1.0478153E-2,5.4432504E7,1.82E2,-4.993728E-3,-1.965056E-3,5.618472E-3,1.1674918E3,8.834262E4,1.5046106E7,-2.6951663E-3,-7.7076635E-4,2.9834446E-3,2.5671237E-4],"split_indices":[46,53,0,53,55,9,43,0,0,0,0,46,10,0,0,0,53,34,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,1.09E2,4E0,9.8E1,1.1E1,1.1E1,8.7E1,4E0,7E0,7E0,4E0,1.3E1,7.4E1,6E0,7E0,6E0,6.8E1,2.1E1,4.7E1,6E0,1.5E1,1.1E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.5438551E-3,1.9281605E-2,-1.6711915E-1,4.5982033E-2,-6.162308E-2,-9.372038E-3,-8.2055957E-4,2.5376797E-2,9.690412E-3,-3.176471E-2,-6.0917297E-3,-6.6602975E-2,4.268207E-2,-6.465919E-2,9.26382E-4,6.842585E-4,-5.092631E-3,8.374747E-2,1.2439377E-2,-3.4085168E-3,-9.2678075E-4,3.679606E-2,5.263811E-3,3.8492754E-3,-3.6719176E-3,-4.4132138E-4,2.5573298E-3,-1.368757E-3,1.0346461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,-1],"loss_changes":[3.1435934E-1,2.080516E-1,1.276218E-1,3.2317793E-1,7.139461E-2,0E0,0E0,1.0948901E-1,0E0,3.943786E-2,0E0,7.274461E-2,6.974814E-2,1.2656186E-2,0E0,0E0,0E0,6.4557835E-2,5.1981732E-2,0E0,0E0,2.5894187E-2,0E0,0E0,3.4296595E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,13,13,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,-1],"split_conditions":[1E0,6.784095E7,3E0,1.0825377E4,1.13808104E8,-9.372038E-3,-8.2055957E-4,2.0676967E4,9.690412E-3,1.1855E4,-6.0917297E-3,3.9024E4,1.5046106E7,8.3722024E7,9.26382E-4,6.842585E-4,-5.092631E-3,3.0765435E10,4.795186E10,-3.4085168E-3,-9.2678075E-4,2.028302E-1,5.263811E-3,3.8492754E-3,7.94E2,-4.4132138E-4,2.5573298E-3,-1.368757E-3,1.0346461E-3],"split_indices":[105,46,8,53,46,0,0,49,0,2,0,12,46,46,0,0,0,32,32,0,0,58,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.4E1,9E0,7.1E1,2.3E1,5E0,4E0,6.6E1,5E0,1.9E1,4E0,1E1,5.6E1,1.2E1,7E0,5E0,5E0,2.3E1,3.3E1,6E0,6E0,1.4E1,9E0,4E0,2.9E1,6E0,8E0,1.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.706806E-3,6.6160626E-4,-8.590499E-3,-2.2892008E-2,1.1591553E-1,-6.606376E-2,1.8957838E-2,8.783608E-3,1.9151013E-3,-4.0266957E-2,-9.731352E-3,-3.6894733E-3,7.422198E-2,1.895479E-2,-8.547305E-2,-2.3264242E-3,6.2277764E-3,4.1628326E-3,-8.226836E-4,-1.7624825E-2,4.5093615E-3,-1.0041023E-1,-1.2582879E-3,1.7482936E-3,-1.6172178E-2,-1.864533E-3,1.5167823E-3,-1.5103664E-3,-4.3222914E-3,-1.6289857E-3,5.5052835E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,23,-1,-1,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7230027E-1,3.039793E-1,0E0,1.6927843E-1,1.3925076E-1,2.3060688E-1,6.034989E-2,0E0,0E0,1.14061855E-1,0E0,2.1487534E-2,5.8422536E-2,7.583651E-2,1.4647439E-2,0E0,3.0084949E-2,0E0,0E0,3.260273E-2,0E0,1.5351996E-2,0E0,0E0,1.0827113E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,16,16,19,19,21,21,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,24,-1,-1,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.8027E4,-8.590499E-3,3.3209075E6,1.6451234E-2,7.650722E4,3.9219662E5,8.783608E-3,1.9151013E-3,3.925844E7,-9.731352E-3,4.475326E6,2.1212247E5,3E0,1.5225641E1,-2.3264242E-3,1.3710804E7,4.1628326E-3,-8.226836E-4,7.3434E4,4.5093615E-3,2.1129233E-1,-1.2582879E-3,1.7482936E-3,1.4342E4,-1.864533E-3,1.5167823E-3,-1.5103664E-3,-4.3222914E-3,-1.6289857E-3,5.5052835E-5],"split_indices":[5,2,0,51,58,34,29,0,0,46,0,46,34,8,55,0,46,0,0,30,0,28,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,1.1E2,4E0,9.2E1,1.8E1,4.5E1,4.7E1,5E0,1.3E1,4.1E1,4E0,3.4E1,1.3E1,1.8E1,2.3E1,4E0,3E1,9E0,4E0,1.4E1,4E0,1.7E1,6E0,1E1,2E1,9E0,5E0,5E0,1.2E1,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.736818E-3,4.113295E-3,-8.342385E-3,-1.0323528E-2,6.449642E-3,-7.903802E-2,1.0886561E-2,-2.5692797E-4,-9.862084E-2,-2.0650776E-2,5.631539E-2,-1.4796215E-3,-4.247032E-3,-4.948225E-2,1.4655777E-2,9.341702E-2,4.5907656E-3,-1.7335454E-4,-2.2867348E-3,-5.0949603E-3,2.342427E-3,6.286833E-4,1.2996759E-1,3.3477068E-2,-1.9531718E-3,-1.018831E-3,1.3773413E-3,6.3288775E-3,2.6190532E-3,2.1420955E-3,-1.5326959E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6382893E-1,2.6389223E-1,0E0,1.4223216E-1,0E0,3.169197E-2,1.0861577E-1,0E0,1.4907181E-2,4.6532497E-2,5.9062257E-2,0E0,0E0,1.5174277E-2,2.1162583E-2,4.774022E-2,2.5546132E-2,0E0,0E0,1.7077478E-2,0E0,0E0,1.8521264E-2,1.00053325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,19,19,22,22,23,23],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.1811385E4,-8.342385E-3,1.9121604E-4,6.449642E-3,1.56E2,3.290356E5,-2.5692797E-4,4.04E3,4.0291533E8,9.580646E0,-1.4796215E-3,-4.247032E-3,2.9777256E-3,2.3E1,1.6984678E3,1.6558E4,-1.7335454E-4,-2.2867348E-3,1.218666E0,2.342427E-3,6.286833E-4,1.4117E4,8.5154843E-1,-1.9531718E-3,-1.018831E-3,1.3773413E-3,6.3288775E-3,2.6190532E-3,2.1420955E-3,-1.5326959E-4],"split_indices":[5,53,0,39,0,0,29,0,2,7,54,0,0,39,8,4,9,0,0,40,0,0,9,28,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,1.03E2,4E0,9.6E1,7E0,2.2E1,7.4E1,5E0,1.7E1,4.4E1,3E1,5E0,1.2E1,2.4E1,2E1,1.7E1,1.3E1,6E0,1.8E1,1.5E1,5E0,6E0,1.1E1,9E0,4E0,1E1,5E0,5E0,6E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5940958E-3,-4.6554036E-2,6.830869E-2,-7.2021233E-3,-2.624325E-2,1.6035995E-1,3.6518317E-2,6.065297E-5,-1.11229755E-1,1.3220954E-3,8.153393E-3,-1.1043287E-2,7.03636E-2,-4.967899E-2,2.3270464E-2,-1.2433941E-3,-5.1097213E-3,1.3591817E-3,-1.4135528E-3,9.0586595E-2,2.9167102E-4,-3.290976E-4,-2.5705846E-3,7.671628E-2,-2.2112587E-2,4.0585003E-3,8.8264357E-4,4.894953E-3,1.4300863E-3,-2.1808026E-3,1.5399962E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,7,9,11,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3724707E-1,1.8424381E-1,1.22903034E-1,0E0,1.2295798E-1,8.0853134E-2,5.5508073E-2,5.0797306E-2,2.5375023E-2,0E0,0E0,2.2004453E-2,2.4459846E-2,1.15733E-2,7.446243E-2,0E0,0E0,0E0,0E0,1.8906705E-2,0E0,0E0,0E0,2.5208868E-2,1.7429084E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,19,19,23,23,24,24],"right_children":[2,4,6,-1,8,10,12,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.108166E3,9.687E3,1.6451234E-2,-7.2021233E-3,4.95E9,7.618855E1,1E0,1.1301075E1,2.7782312E1,1.3220954E-3,8.153393E-3,2.1486234E5,9.780346E9,1.433458E10,9.3152986E8,-1.2433941E-3,-5.1097213E-3,1.3591817E-3,-1.4135528E-3,1.0155914E1,2.9167102E-4,-3.290976E-4,-2.5705846E-3,5.83E2,2E0,4.0585003E-3,8.8264357E-4,4.894953E-3,1.4300863E-3,-2.1808026E-3,1.5399962E-4],"split_indices":[53,30,58,0,5,57,102,57,57,0,0,29,5,32,5,0,0,0,0,54,0,0,0,10,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6E1,4.3E1,6E0,5.4E1,1E1,3.3E1,4.2E1,1.2E1,4E0,6E0,1.4E1,1.9E1,1.3E1,2.9E1,4E0,8E0,5E0,9E0,1.4E1,5E0,5E0,8E0,1.3E1,1.6E1,1E1,4E0,4E0,9E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5897438E-4,-1.3053351E-2,6.461985E-3,-7.5818904E-2,2.809792E-3,-4.5627163E-4,-1.0400714E-1,-3.512621E-2,2.3239039E-2,-1.1999704E-3,-4.757195E-3,-4.7030836E-2,9.625173E-4,7.5231716E-2,-5.824817E-3,-8.361244E-4,-3.548405E-3,4.533259E-3,3.952262E-4,4.6399564E-2,-3.7876822E-2,-8.6584355E-4,2.9007045E-3,-1.1877371E-2,-3.6426443E-3,-2.1551596E-3,5.5181986E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,7,-1,9,11,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1],"loss_changes":[2.4026561E-1,9.3804985E-2,0E0,3.252372E-2,5.97166E-2,0E0,2.1159917E-2,2.1113586E-2,7.59936E-2,0E0,0E0,2.5864813E-2,0E0,5.6866713E-2,5.699604E-2,0E0,0E0,0E0,0E0,3.298611E-2,3.320931E-2,0E0,0E0,2.1693436E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,23,23],"right_children":[2,4,-1,6,8,-1,10,12,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1],"split_conditions":[1.0825377E4,4.1964554E-4,6.461985E-3,4.8861527E9,1.6511278E1,-4.5627163E-4,2.9916666E1,1.1789883E0,2.0752128E7,-1.1999704E-3,-4.757195E-3,2.6666668E-1,9.625173E-4,1.2320755E1,2.0086452E5,-8.361244E-4,-3.548405E-3,4.533259E-3,3.952262E-4,1.1653E4,7.331148E7,-8.6584355E-4,2.9007045E-3,1E0,-3.6426443E-3,-2.1551596E-3,5.5181986E-4],"split_indices":[53,43,0,5,57,0,57,58,46,0,0,58,0,55,29,0,0,0,0,2,46,0,0,102,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9.3E1,6E0,1.8E1,7.5E1,6E0,1.2E1,2.6E1,4.9E1,4E0,8E0,2.2E1,4E0,1.7E1,3.2E1,1.6E1,6E0,9E0,8E0,1.2E1,2E1,4E0,8E0,1.5E1,5E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-7.199455E-3,1.0465599E-2,-1.6016547E-1,3.932468E-2,-5.875473E-2,-8.316888E-3,-1.584643E-3,1.6268708E-2,7.876037E-3,-1.6845651E-2,-6.1411713E-3,3.0123955E-2,-4.315039E-2,-4.513322E-2,1.5212361E-3,3.5156552E-3,1.1132896E-2,-2.4072235E-4,-3.0902775E-3,-2.699677E-3,-4.8931065E-4,-2.5096811E-2,4.3182988E-2,-1.6545515E-3,8.6665107E-4,3.6377704E-3,8.1199093E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,19,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[2.610829E-1,1.7610475E-1,7.151261E-2,2.52083E-1,1.1846192E-1,0E0,0E0,4.7402777E-2,0E0,3.5270683E-2,0E0,5.7668686E-2,1.6463183E-2,1.2092693E-2,0E0,0E0,4.4046875E-2,0E0,0E0,0E0,0E0,1.942357E-2,2.1561876E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,13,13,16,16,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,20,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[1E0,6.9754904E7,3.9E1,8.974092E3,1.1245455E1,-8.316888E-3,-1.584643E-3,4.5843E7,7.876037E-3,1.1855E4,-6.1411713E-3,1.048576E8,6.25379E5,7.396E3,1.5212361E-3,3.5156552E-3,4.034396E8,-2.4072235E-4,-3.0902775E-3,-2.699677E-3,-4.8931065E-4,2.31308E5,1.3047E4,-1.6545515E-3,8.6665107E-4,3.6377704E-3,8.1199093E-4],"split_indices":[105,46,3,4,54,0,0,46,0,2,0,7,30,9,0,0,7,0,0,0,0,30,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,8.6E1,9E0,6.1E1,2.5E1,5E0,4E0,5.5E1,6E0,1.9E1,6E0,4.5E1,1E1,1.3E1,6E0,9E0,3.6E1,6E0,4E0,6E0,7E0,1.7E1,1.9E1,1.2E1,5E0,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.273087E-3,-7.470429E-3,7.443775E-3,3.6390252E-2,-5.0697304E-2,1.3581093E-2,1.6413313E-1,-2.5516236E-2,-6.191642E-3,5.562921E-2,-1.4437392E-2,8.375183E-3,2.2910745E-3,-5.694799E-2,3.046937E-2,-1.1643596E-2,1.1784481E-1,-2.5296456E-3,1.0377308E-2,-6.73674E-4,-2.9562493E-3,-6.682096E-4,2.390234E-3,9.113448E-4,-1.6820207E-3,1.8230348E-3,5.383556E-3,2.548729E-3,-1.5029071E-2,-1.8590839E-3,3.444982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,-1,3,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[2.9814023E-1,0E0,1.5163006E-1,1.7259336E-1,8.6881734E-2,6.206562E-2,4.6752453E-2,4.7488037E-2,0E0,8.926592E-2,4.490131E-2,0E0,0E0,1.4702745E-2,1.8728366E-2,1.5355412E-2,1.5637785E-2,0E0,3.624961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5723072E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,18,18,28,28],"right_children":[2,-1,4,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[9.7888E4,-7.470429E-3,1.0504551E1,1.0825377E4,2.192772E10,2.0752128E7,8E0,7.287E3,-6.191642E-3,1.6511278E1,6.604E3,8.375183E-3,2.2910745E-3,1.09141096E8,9.877E3,3.1E1,3.3052456E5,-2.5296456E-3,1.7994973E11,-6.73674E-4,-2.9562493E-3,-6.682096E-4,2.390234E-3,9.113448E-4,-1.6820207E-3,1.8230348E-3,5.383556E-3,2.548729E-3,1.6226676E0,-1.8590839E-3,3.444982E-4],"split_indices":[1,0,54,53,5,46,55,2,0,57,2,0,0,7,9,3,29,0,32,0,0,0,0,0,0,0,0,0,58,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6E0,8.8E1,5.9E1,2.9E1,5.1E1,8E0,2.5E1,4E0,2E1,3.1E1,4E0,4E0,1.6E1,9E0,1E1,1E1,9E0,2.2E1,7E0,9E0,4E0,5E0,5E0,5E0,4E0,6E0,6E0,1.6E1,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5544318E-2,8.2856574E-4,1.7569044E-1,2.3314485E-2,-5.16662E-2,8.75562E-3,2.6616931E-3,-1.423079E-2,5.7185426E-2,-9.980586E-2,-1.4028323E-2,3.732155E-2,-3.7265997E-2,8.990336E-2,1.340531E-2,-5.706309E-4,-4.8877364E-3,1.2460187E-3,-1.486541E-3,2.8164522E-3,-3.461824E-4,-3.1046993E-3,-5.865025E-4,1.3043243E-3,1.2722433E-1,3.3343535E-2,-1.48438E-3,5.527806E-3,2.1791419E-3,1.1838021E-4,2.4503965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1],"loss_changes":[2.5201818E-1,1.1806349E-1,4.3447137E-2,8.9832366E-2,5.2956127E-2,0E0,0E0,4.176647E-2,5.1793434E-2,3.6710635E-2,2.5140256E-2,2.1744628E-2,2.3186956E-2,3.7549675E-2,1.9850623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1530235E-2,1.327269E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,24,24,25,25],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1],"split_conditions":[1.0825377E4,4.3201213E9,1.055E3,1.1462246E3,3.0595828E3,8.75562E-3,2.6616931E-3,3.640909E6,2.0752128E7,2.4759493E1,5.9633E7,1.5662651E0,2.5975E4,3.6742297E5,1.4664377E-1,-5.706309E-4,-4.8877364E-3,1.2460187E-3,-1.486541E-3,2.8164522E-3,-3.461824E-4,-3.1046993E-3,-5.865025E-4,1.3043243E-3,7.408377E-1,2.7723257E3,-1.48438E-3,5.527806E-3,2.1791419E-3,1.1838021E-4,2.4503965E-3],"split_indices":[53,5,0,53,4,0,0,46,46,57,46,57,30,29,39,0,0,0,0,0,0,0,0,0,58,53,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,9.8E1,8E0,6.9E1,2.9E1,4E0,4E0,3.3E1,3.6E1,1.2E1,1.7E1,1E1,2.3E1,2E1,1.6E1,4E0,8E0,6E0,1.1E1,5E0,5E0,6E0,1.7E1,9E0,1.1E1,1.2E1,4E0,7E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.8123404E-3,-1.4526056E-1,5.97127E-3,1.5809156E-4,-9.597907E-3,1.5154353E-2,-6.6571943E-3,-6.059623E-4,1.2431084E-1,-9.94634E-2,1.5669389E-2,7.1533322E-3,1.181514E-3,-1.0294307E-3,-5.639434E-3,3.3276565E-2,-1.7589279E-2,-2.202198E-2,6.3199975E-2,-1.8199122E-3,1.361609E-3,-1.4930501E-3,1.4579241E-3,3.245211E-3,4.756501E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8983582E-1,1.6356577E-1,1.8984586E-1,0E0,0E0,1.7727041E-1,0E0,1.4797205E-1,8.141294E-2,4.7880545E-2,4.6847023E-2,0E0,0E0,0E0,0E0,8.743729E-2,5.2722573E-2,2.4593193E-2,4.4207484E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,10,10,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.477E3,2.16268E5,1.4974915E8,1.5809156E-4,-9.597907E-3,1.0825377E4,-6.6571943E-3,1.7145766E4,6.5566176E7,2.2367E4,3.925844E7,7.1533322E-3,1.181514E-3,-1.0294307E-3,-5.639434E-3,1.5493506E1,4.7308203E3,1.7618678E0,7.6363635E-1,-1.8199122E-3,1.361609E-3,-1.4930501E-3,1.4579241E-3,3.245211E-3,4.756501E-4],"split_indices":[2,12,46,0,0,53,0,52,46,33,46,0,0,0,0,57,4,40,58,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,8E0,1.06E2,4E0,4E0,1.02E2,4E0,9E1,1.2E1,1.2E1,7.8E1,6E0,6E0,6E0,6E0,5.1E1,2.7E1,1.8E1,3.3E1,1.7E1,1E1,1.4E1,4E0,2.1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.3294464E-2,-1.6678155E-4,-6.107358E-3,-6.838803E-3,1.0120793E-2,-2.4004694E-2,4.6296407E-2,-3.7772857E-2,3.6809193E-3,3.556834E-2,4.5131375E-3,-9.956454E-3,-9.088747E-2,5.878632E-2,1.9912338E-2,-3.7122454E-2,5.7555597E-2,-1.6427932E-3,-4.919216E-3,3.9683143E-3,6.1795994E-4,1.964694E-3,-1.5570889E-3,-2.056355E-3,-2.7524898E-4,3.552975E-5,3.7047437E-3,-1.0763321E-3,7.6549914E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,-1,-1,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1073212E-1,1.8882969E-1,0E0,0E0,1.146863E-1,8.7095074E-2,3.3552386E-2,6.410319E-2,0E0,1.3732959E-2,0E0,5.715758E-2,2.468478E-2,3.220789E-2,1.93466E-2,1.2438357E-2,2.2841193E-2,0E0,0E0,0E0,0E0,0E0,1.1564974E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,22,22],"right_children":[2,4,-1,-1,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2511909E8,2.7803582E6,-6.107358E-3,-6.838803E-3,3.4288502E8,3.7249353E3,1.2668315E6,2.8225484E11,3.6809193E-3,4.760274E-1,4.5131375E-3,4.899102E5,7.444956E7,7E0,2.2059325E5,4.757E3,1.68E2,-1.6427932E-3,-4.919216E-3,3.9683143E-3,6.1795994E-4,1.964694E-3,9.904E3,-2.056355E-3,-2.7524898E-4,3.552975E-5,3.7047437E-3,-1.0763321E-3,7.6549914E-4],"split_indices":[46,46,0,0,7,53,29,32,0,58,0,29,7,8,29,2,0,0,0,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.5E1,7E0,4E0,9.1E1,4.7E1,4.4E1,4.3E1,4E0,4E1,4E0,2.9E1,1.4E1,1.5E1,2.5E1,2.1E1,8E0,8E0,6E0,6E0,9E0,9E0,1.6E1,1.2E1,9E0,4E0,4E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.3286694E-2,-6.1373953E-3,-1.2186766E-3,1.1916811E-2,-1.10121354E-1,-1.8787432E-3,1.2555566E-1,-7.114069E-3,-6.9779414E-4,-7.865443E-2,1.7031323E-2,7.1104984E-3,1.0487871E-3,-1.0030704E-3,-3.9526024E-3,3.926016E-3,6.900742E-3,-3.933339E-3,3.5323286E-3,9.4712945E-4,-5.797085E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,-1,17,19,-1,-1,-1],"loss_changes":[1.8512852E-1,0E0,1.3291734E-1,1.303559E-1,7.511633E-2,1.1022709E-1,5.6288615E-2,0E0,0E0,2.0481043E-2,5.5925384E-2,0E0,0E0,0E0,0E0,0E0,5.555865E-2,1.907408E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,16,16,17,17],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,-1,18,20,-1,-1,-1],"split_conditions":[2.7571955E6,-6.1373953E-3,1.1384717E8,1.0861106E4,2.7140412E5,2.0676967E4,2.5669595E5,-7.114069E-3,-6.9779414E-4,5.895365E5,4.80378E5,7.1104984E-3,1.0487871E-3,-1.0030704E-3,-3.9526024E-3,3.926016E-3,2.4241872E0,2.8327732E6,3.5323286E-3,9.4712945E-4,-5.797085E-4],"split_indices":[46,0,46,4,33,49,34,0,0,29,12,0,0,0,0,0,40,33,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,6E0,9.1E1,8.2E1,9E0,7.4E1,8E0,4E0,5E0,1.4E1,6E1,4E0,4E0,6E0,8E0,5E0,5.5E1,5E1,5E0,1.4E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.8605918E-3,-2.1736171E-2,8.437174E-2,-6.7620915E-3,-1.111673E-2,6.9409097E-3,5.3182837E-2,1.4328168E-2,-4.2614054E-2,7.405703E-2,-8.7553845E-4,-1.8179422E-2,5.976905E-2,-6.996603E-2,-7.9285465E-3,4.372196E-3,1.5041559E-3,9.2035875E-4,-2.9263632E-2,8.864982E-2,3.2408815E-4,-3.7530188E-3,-1.5765788E-3,7.9274306E-4,-1.3334267E-3,-2.1656025E-3,-3.945525E-4,4.22959E-3,1.466527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,-1,7,-1,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1016249E-1,1.4560623E-1,7.0528194E-2,0E0,6.472628E-2,0E0,3.4123387E-2,6.7559384E-2,3.3599094E-2,1.933889E-2,0E0,1.3815888E-2,2.7075693E-2,1.333572E-2,1.5547715E-2,0E0,0E0,0E0,1.1736214E-2,1.1669204E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,18,18,19,19],"right_children":[2,4,6,-1,8,-1,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,1.369606E6,1.6451234E-2,-6.7620915E-3,2.8399336E7,6.9409097E-3,9.3393946E-1,1.6987854E1,1.2163482E8,9.275501E-1,-8.7553845E-4,6.910683E0,9.3977806E1,8.49491E5,5.4357555E6,4.372196E-3,1.5041559E-3,9.2035875E-4,2.5975E4,8.70707E0,3.2408815E-4,-3.7530188E-3,-1.5765788E-3,7.9274306E-4,-1.3334267E-3,-2.1656025E-3,-3.945525E-4,4.22959E-3,1.466527E-3],"split_indices":[2,46,58,0,46,0,28,57,7,58,0,59,59,1,51,0,0,0,30,54,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,8.3E1,2.3E1,4E0,7.9E1,4E0,1.9E1,4.4E1,3.5E1,1.5E1,4E0,2.6E1,1.8E1,1.9E1,1.6E1,5E0,1E1,5E0,2.1E1,1.1E1,7E0,7E0,1.2E1,8E0,8E0,7E0,1.4E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-8.72379E-3,-3.2157023E-2,6.567417E-2,-5.4606968E-3,-1.8477479E-2,1.06017515E-1,-4.220538E-3,1.6758474E-2,-5.2261718E-2,1.3683248E-1,2.2292566E-4,3.2988226E-3,4.576324E-3,-7.992488E-2,-1.09585095E-2,1.6422248E-1,1.5787827E-3,2.2053518E-3,-1.0813563E-2,-3.8940748E-3,-9.960631E-4,2.36622E-4,-1.833925E-3,3.5757283E-3,7.6861647E-3,-1.7838805E-3,2.6325219E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.816805E-1,1.259106E-1,1.9653776E-1,0E0,8.658606E-2,6.3764766E-2,0E0,3.23044E-2,4.1399516E-2,3.361982E-2,0E0,0E0,2.8489737E-2,2.918826E-2,1.149574E-2,1.6984403E-2,0E0,0E0,1.1619508E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8355938E3,9.7888E4,1.2707124E1,-5.4606968E-3,2.8399336E7,9.299267E9,-4.220538E-3,1.82E2,1.1309524E-1,1.343964E1,2.2292566E-4,3.2988226E-3,4.3650795E-2,9.08958E5,2.4835809E-1,1.1681136E4,1.5787827E-3,2.2053518E-3,6.4637297E9,-3.8940748E-3,-9.960631E-4,2.36622E-4,-1.833925E-3,3.5757283E-3,7.6861647E-3,-1.7838805E-3,2.6325219E-5],"split_indices":[53,1,54,0,46,12,0,10,58,55,0,0,58,30,42,4,0,0,32,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.8E1,2.4E1,7E0,7.1E1,2E1,4E0,3.5E1,3.6E1,1.5E1,5E0,4E0,3.1E1,2.1E1,1.5E1,1.1E1,4E0,6E0,2.5E1,1.3E1,8E0,1.1E1,4E0,6E0,5E0,5E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.9045455E-4,1.1739794E-2,-5.0104256E-3,2.1063041E-2,-6.0562724E-3,7.0360657E-3,1.2753847E-1,-2.1995079E-2,4.3521825E-2,7.092836E-3,1.348787E-3,2.283017E-2,-4.3062016E-2,1.8608687E-2,7.661435E-2,-1.0269417E-2,3.5273766E-3,-3.9992332E-3,-2.990283E-2,2.1972363E-3,3.4910275E-3,3.8514629E-3,3.1500537E-4,-1.3301228E-3,1.2383802E-3,-2.3676392E-4,-2.4372146E-3,1.796641E-3,-4.7961052E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6770124E-1,1.6920248E-1,0E0,1.4098781E-1,0E0,9.099125E-2,6.102948E-2,4.631099E-2,2.9902972E-2,0E0,0E0,4.065559E-2,2.609123E-2,1.4478652E-2,3.1727165E-2,1.5476296E-2,0E0,0E0,2.5015475E-2,0E0,1.4633759E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,18,18,20,20],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.5081978E8,-5.0104256E-3,1.1335518E4,-6.0562724E-3,5.8585525E-1,1.3477259E7,4.795186E10,3.1346828E5,7.092836E-3,1.348787E-3,9.144343E0,3.411305E2,1.6988876E-1,1.343964E1,1.0210356E1,3.5273766E-3,-3.9992332E-3,2.119303E6,2.1972363E-3,1.2606002E-1,3.8514629E-3,3.1500537E-4,-1.3301228E-3,1.2383802E-3,-2.3676392E-4,-2.4372146E-3,1.796641E-3,-4.7961052E-4],"split_indices":[105,46,0,4,0,43,48,32,29,0,0,57,34,39,55,54,0,0,33,0,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.8E1,7E0,9.4E1,4E0,8.4E1,1E1,4.7E1,3.7E1,5E0,5E0,1.5E1,3.2E1,2.2E1,1.5E1,1.1E1,4E0,4E0,2.8E1,5E0,1.7E1,1E1,5E0,7E0,4E0,1.8E1,1E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.4349855E-4,8.81773E-3,-5.9871273E-3,-2.0220012E-3,1.1308135E-1,-5.3416163E-2,1.1181911E-2,6.2670093E-3,6.447404E-4,7.6067676E-3,-4.159057E-3,7.13886E-2,2.2942477E-3,1.6861771E-3,-7.7285536E-4,-1.6962165E-4,4.4369227E-3,-2.8251627E-2,2.142086E-2,6.010141E-4,-3.92126E-2,5.6421056E-2,4.3734084E-3,-8.103488E-4,-2.5970933E-3,3.7789282E-3,6.493335E-4,5.017618E-4,-1.6618022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,-1,-1,-1,-1,19,21,-1,23,25,27,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7321569E-1,1.2002874E-1,0E0,6.642188E-2,5.4515243E-2,7.628062E-2,4.1573767E-2,0E0,0E0,1.3477924E-2,0E0,3.976794E-2,4.0917672E-2,0E0,0E0,0E0,0E0,1.3940422E-2,2.5409807E-2,0E0,1.0456149E-2,2.408043E-2,1.5041824E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,-1,-1,-1,-1,20,22,-1,24,26,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.3127055E4,-5.9871273E-3,1.1112019E3,8.50999E9,6.8917E4,1.53E2,6.2670093E-3,6.447404E-4,5.625738E2,-4.159057E-3,6.277045E0,2.2781514E8,1.6861771E-3,-7.7285536E-4,-1.6962165E-4,4.4369227E-3,8.1980075E5,4.760274E-1,6.010141E-4,8.63558E-1,7.84E3,9.014471E-1,-8.103488E-4,-2.5970933E-3,3.7789282E-3,6.493335E-4,5.017618E-4,-1.6618022E-3],"split_indices":[46,4,0,34,5,12,10,0,0,34,0,59,7,0,0,0,0,33,58,0,28,9,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.05E2,5E0,9.6E1,9E0,1.9E1,7.7E1,5E0,4E0,1E1,9E0,9E0,6.8E1,4E0,6E0,4E0,5E0,2.6E1,4.2E1,5E0,2.1E1,1.3E1,2.9E1,1.5E1,6E0,5E0,8E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.2598174E-3,1.4670459E-2,-1.386452E-1,2.5006693E-2,-4.663763E-3,-9.339003E-3,3.2908953E-4,1.2198944E-2,1.4745949E-1,-3.9475158E-2,2.7000148E-2,7.674119E-3,1.9085946E-3,-6.622676E-2,7.1005063E-4,5.454945E-3,6.2422793E-2,-7.4367505E-4,-3.2235263E-3,2.7724443E-2,-3.154511E-2,1.01069845E-1,-9.519478E-3,-7.5294264E-4,1.41495E-3,-1.7789672E-3,2.9685492E-5,4.9565737E-3,1.0727721E-3,-1.783428E-3,9.133871E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,-1,17,-1,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9439337E-1,1.5489066E-1,1.619811E-1,1.4876224E-1,0E0,0E0,0E0,6.837165E-2,4.4187605E-2,3.294565E-2,5.2319463E-2,0E0,0E0,1.2891479E-2,0E0,3.7129365E-2,7.325141E-2,0E0,0E0,1.6415996E-2,9.79618E-3,3.9857015E-2,1.5135152E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,-1,18,-1,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.8841052E10,2.667836E6,1.2768678E4,-4.663763E-3,-9.339003E-3,3.2908953E-4,1E0,9.9182825E0,4.7E1,3.290356E5,7.674119E-3,1.9085946E-3,1.5285803E5,7.1005063E-4,2.3365998E5,2.0752128E7,-7.4367505E-4,-3.2235263E-3,2.0511957E-1,7.22E2,9.275501E-1,3.598686E1,-7.5294264E-4,1.41495E-3,-1.7789672E-3,2.9685492E-5,4.9565737E-3,1.0727721E-3,-1.783428E-3,9.133871E-4],"split_indices":[105,5,33,53,0,0,0,17,55,3,29,0,0,29,0,29,46,0,0,40,0,58,57,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,1.01E2,8E0,9.5E1,6E0,4E0,4E0,8.7E1,8E0,1.9E1,6.8E1,4E0,4E0,1.3E1,6E0,4.3E1,2.5E1,5E0,8E0,2.7E1,1.6E1,1.6E1,9E0,5E0,2.2E1,1E1,6E0,1E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.5288215E-3,-6.140324E-3,8.652144E-3,1.9167883E-2,-4.613466E-3,4.772793E-3,1.3385132E-1,-6.674228E-2,1.9426296E-2,7.069505E-3,2.162803E-3,-3.366233E-3,-5.7304214E-4,7.623323E-2,1.0581485E-2,7.9137104E-4,4.1626757E-3,-2.1845449E-2,2.6141416E-2,-1.498703E-3,7.44017E-4,2.4439825E-3,1.5997738E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,15,17,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[2.0840193E-1,0E0,1.4018543E-1,1.4794022E-1,0E0,8.618021E-2,3.40883E-2,1.7390944E-2,3.3359714E-2,0E0,0E0,0E0,0E0,1.6569298E-2,3.0844722E-2,0E0,0E0,1.7553702E-2,3.6687523E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,17,17,18,18],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,16,18,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.477E3,-6.140324E-3,1.12782936E8,1.0825377E4,-4.613466E-3,5.981675E-4,4.2123712E5,2.2196926E-2,2.1923357E1,7.069505E-3,2.162803E-3,-3.366233E-3,-5.7304214E-4,3.2334878E7,1.1674918E3,7.9137104E-4,4.1626757E-3,5.407E3,9.275501E-1,-1.498703E-3,7.44017E-4,2.4439825E-3,1.5997738E-4],"split_indices":[2,0,46,53,0,43,29,58,59,0,0,0,0,12,53,0,0,2,58,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,6E0,9.5E1,8.9E1,6E0,8E1,9E0,1.3E1,6.7E1,4E0,5E0,8E0,5E0,8E0,5.9E1,4E0,4E0,1.9E1,4E1,1.3E1,6E0,1.3E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.739652E-3,6.9263284E-3,-4.405061E-3,-3.8693522E-3,4.7010025E-3,1.3269627E-3,-3.413561E-3,3.2038175E-2,-1.4311399E-2,2.8845035E-3,6.994032E-2,-2.5837254E-2,3.2984633E-2,-9.994091E-4,1.6085008E-3,4.070153E-3,3.83931E-2,-8.252682E-3,-1.6695434E-3,2.4891414E-3,-1.1246001E-3,-3.9452192E-4,2.8895128E-3,-9.2690036E-4,1.0629499E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,-1,5,-1,7,-1,9,11,13,15,17,19,-1,-1,-1,21,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2305258E-1,1.3226694E-1,0E0,4.3476637E-2,0E0,4.272254E-2,0E0,3.294786E-2,3.3014737E-2,2.4159394E-2,1.3223283E-2,1.6842786E-2,2.9823627E-2,0E0,0E0,0E0,1.9359013E-2,1.8486392E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,16,16,17,17],"right_children":[2,4,-1,6,-1,8,-1,10,12,14,16,18,20,-1,-1,-1,22,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8799975E10,1.0825377E4,-4.405061E-3,1.5088414E6,4.7010025E-3,1.5046106E7,-3.413561E-3,3.6742297E5,1.8027E4,2.0699982E8,1.82E2,1.7994973E11,2.4494735E9,-9.994091E-4,1.6085008E-3,4.070153E-3,3.97E2,5.3433334E1,-1.6695434E-3,2.4891414E-3,-1.1246001E-3,-3.9452192E-4,2.8895128E-3,-9.2690036E-4,1.0629499E-3],"split_indices":[5,53,0,29,0,46,0,29,2,12,10,32,7,0,0,0,0,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.8E1,7E0,9.1E1,7E0,8.7E1,4E0,2.9E1,5.8E1,1.7E1,1.2E1,4.7E1,1.1E1,1E1,7E0,4E0,8E0,2.6E1,2.1E1,7E0,4E0,4E0,4E0,1.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.6483202E-3,4.0761293E-3,-5.454882E-3,-1.9531498E-2,5.437144E-2,-9.483714E-3,-5.0616013E-3,6.2591457E-3,3.4693863E-2,1.9309424E-2,-5.747635E-2,6.914881E-2,-6.5053966E-5,-9.307056E-3,4.9574755E-2,-1.7867592E-4,-6.602468E-2,9.292712E-2,3.6346307E-4,1.7277653E-3,-2.987749E-2,4.009477E-3,2.6326647E-2,-1.4866267E-3,-3.7136124E-3,1.150766E-3,4.313193E-3,-1.6008106E-3,7.0349715E-4,1.5688151E-3,-5.4768653E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3728663E-1,1.3760753E-1,0E0,9.445317E-2,8.0623865E-2,1.03367366E-1,0E0,0E0,4.1620765E-2,4.1167498E-2,1.2245826E-2,2.2947013E-2,0E0,3.0826839E-2,2.9950999E-2,0E0,1.7731592E-2,1.5637465E-2,0E0,0E0,1.4428532E-2,0E0,1.3452238E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,20,20,22,22],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,4.818997E6,-5.454882E-3,6.2640357E-1,1.4440433E-2,4.178728E7,-5.0616013E-3,6.2591457E-3,7.639958E-1,1.6987854E1,2.4557575E1,9.780346E9,-6.5053966E-5,1.2410928E8,2.85E2,-1.7867592E-4,4.10362E5,7.4684826E8,3.6346307E-4,1.7277653E-3,9E0,4.009477E-3,4.7E1,-1.4866267E-3,-3.7136124E-3,1.150766E-3,4.313193E-3,-1.6008106E-3,7.0349715E-4,1.5688151E-3,-5.4768653E-4],"split_indices":[5,1,0,39,58,46,0,0,28,57,57,5,0,5,0,0,29,7,0,0,8,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,1.14E2,5E0,7.8E1,3.6E1,7.3E1,5E0,4E0,3.2E1,4.6E1,2.7E1,1.6E1,1.6E1,2.4E1,2.2E1,4E0,2.3E1,1.1E1,5E0,6E0,1.8E1,5E0,1.7E1,1.5E1,8E0,4E0,7E0,1.4E1,4E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.1603726E-3,1.5485994E-2,-6.3458327E-3,4.0161577E-3,1.2863575E-1,3.0635163E-2,-2.789822E-2,6.486663E-3,1.5042772E-3,7.0876986E-2,1.6491812E-2,-4.182692E-2,1.5052001E-2,1.00328155E-1,-7.9453544E-5,5.359031E-3,3.530967E-3,-2.1188335E-2,-7.1271345E-2,1.8600136E-3,-6.6271616E-4,1.1476396E-3,5.0805463E-3,-6.4179447E-4,2.1220494E-2,-2.0126458E-3,-4.8533773E-5,-7.362675E-4,-3.5117695E-3,1.4871164E-3,-1.5560671E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7482221E-1,1.4094457E-1,0E0,8.580757E-2,3.6898896E-2,3.0134596E-2,2.8148647E-2,0E0,0E0,3.0351937E-2,3.749491E-2,1.9759078E-2,1.5565556E-2,2.3213856E-2,0E0,1.4321999E-2,0E0,1.5012425E-2,1.6535476E-2,0E0,0E0,0E0,0E0,0E0,1.1904246E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[6.313997E-1,1.1811385E4,-6.3458327E-3,3.925844E7,3.0376984E1,2.083117E1,1.9649E4,6.486663E-3,1.5042772E-3,1.82E2,1.2028319E6,7.752181E7,1.6944988E9,1.16E2,-7.9453544E-5,2.3514317E8,3.530967E-3,9.371747E0,7.056912E7,1.8600136E-3,-6.6271616E-4,1.1476396E-3,5.0805463E-3,-6.4179447E-4,1.5046106E7,-2.0126458E-3,-4.8533773E-5,-7.362675E-4,-3.5117695E-3,1.4871164E-3,-1.5560671E-4],"split_indices":[39,53,0,46,58,59,2,0,0,10,29,46,7,0,0,7,0,54,7,0,0,0,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,1.08E2,4E0,9.9E1,9E0,5.4E1,4.5E1,5E0,4E0,1.3E1,4.1E1,3.4E1,1.1E1,9E0,4E0,3.7E1,4E0,2.1E1,1.3E1,5E0,6E0,4E0,5E0,1.5E1,2.2E1,7E0,1.4E1,5E0,8E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-8.785858E-3,-1.0762257E-3,-5.7649054E-3,1.0205673E-2,-1.1502917E-1,-5.704144E-4,8.119578E-2,-8.205172E-3,7.299546E-4,2.520764E-2,-2.697866E-2,5.2220975E-3,5.4919295E-4,-9.181918E-3,6.1775774E-2,-3.428799E-2,1.1207894E-3,1.398447E-3,-1.2320307E-3,3.1725105E-3,9.7081985E-4,-4.029546E-2,3.9776257E-4,-2.159745E-3,-6.148451E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[1.1965152E-1,1.2844892E-1,0E0,6.966092E-2,1.4121953E-1,5.514033E-2,4.803879E-2,0E0,0E0,5.2127264E-2,1.8028772E-2,0E0,0E0,2.761657E-2,1.5845947E-2,1.0242112E-2,0E0,0E0,0E0,0E0,0E0,1.35182515E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.4974915E8,1E0,-5.7649054E-3,1.0825377E4,1.04856936E8,2.727403E7,5.4875424E7,-8.205172E-3,7.299546E-4,1.1462246E3,3.8E1,5.2220975E-3,5.4919295E-4,2.8327732E6,9.366E4,2.022832E1,1.1207894E-3,1.398447E-3,-1.2320307E-3,3.1725105E-3,9.7081985E-4,9.08958E5,3.9776257E-4,-2.159745E-3,-6.148451E-4],"split_indices":[46,105,0,53,7,46,46,0,0,53,8,0,0,33,30,55,0,0,0,0,0,30,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.8E1,4E0,9E1,8E0,7.9E1,1.1E1,4E0,4E0,4E1,3.9E1,5E0,6E0,2.1E1,1.9E1,3.5E1,4E0,7E0,1.4E1,1E1,9E0,3.1E1,4E0,1.6E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.8353552E-3,4.223751E-3,-5.5867573E-3,1.25166355E-2,-5.0125904E-3,9.2093274E-4,7.112307E-2,2.7651932E-2,-2.9744381E-2,6.007968E-3,2.9002529E-2,1.5225844E-2,3.1166228E-3,-5.4415897E-2,4.8034373E-3,1.8822147E-3,-6.655829E-4,5.8158778E-2,-3.0645356E-3,-7.410822E-2,-2.082319E-4,-4.501191E-4,1.8846401E-3,4.3006588E-4,4.35119E-3,-4.699623E-4,1.7081059E-3,-6.844289E-4,-3.4481182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.20756365E-1,1.2987149E-1,0E0,6.993152E-2,0E0,7.213765E-2,6.3405104E-2,3.2952726E-2,3.520761E-2,0E0,1.5226593E-2,3.1588614E-2,0E0,2.256447E-2,1.5385991E-2,0E0,0E0,3.317311E-2,1.5156018E-2,1.8252775E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,6.313997E-1,-5.5867573E-3,1.3400168E7,-5.0125904E-3,3.925844E7,1.3453537E-1,2.4241872E0,3.4442917E3,6.007968E-3,2.5669595E5,8.01112E5,3.1166228E-3,1.1309524E-1,1E0,1.8822147E-3,-6.655829E-4,3.1259478E10,6.866558E5,2.4759493E1,-2.082319E-4,-4.501191E-4,1.8846401E-3,4.3006588E-4,4.35119E-3,-4.699623E-4,1.7081059E-3,-6.844289E-4,-3.4481182E-3],"split_indices":[46,39,0,1,0,46,58,40,4,0,34,33,0,58,89,0,0,32,29,57,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,1.07E2,4E0,1.02E2,5E0,8.6E1,1.6E1,4.6E1,4E1,4E0,1.2E1,3.9E1,7E0,2.3E1,1.7E1,8E0,4E0,1.1E1,2.8E1,1.6E1,7E0,1.3E1,4E0,7E0,4E0,2.4E1,4E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.20608555E-2,-8.557722E-3,5.3557336E-2,2.1239823E-2,-4.6016023E-2,1.1540856E-1,2.1647736E-2,3.8935628E-3,7.878925E-3,-3.1252787E-2,-3.6052517E-3,2.3051652E-3,5.939278E-3,-5.275031E-4,4.6000607E-2,-1.2518006E-3,2.4584552E-2,-7.370518E-4,-2.5459786E-3,3.3509156E-3,3.4946052E-4,2.684215E-3,8.426045E-3,7.491929E-4,-1.1805954E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[8.516258E-2,7.576045E-2,6.263205E-2,4.3121457E-2,2.072651E-2,1.7343491E-2,2.0981252E-2,0E0,2.5021566E-2,9.199277E-3,0E0,0E0,0E0,0E0,2.2906693E-2,0E0,1.947644E-2,0E0,0E0,0E0,0E0,0E0,1.0816756E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,14,14,16,16,22,22],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[3.062708E3,2.7260774E7,1.3453537E-1,4.4493478E2,7.705493E5,2.8129198E7,1.2532358E6,3.8935628E-3,4.7586722E2,1.2903092E7,-3.6052517E-3,2.3051652E-3,5.939278E-3,-5.275031E-4,1.2320755E1,-1.2518006E-3,3.854803E-2,-7.370518E-4,-2.5459786E-3,3.3509156E-3,3.4946052E-4,2.684215E-3,4.7E1,7.491929E-4,-1.1805954E-3],"split_indices":[53,46,58,4,29,46,33,0,53,33,0,0,0,0,55,0,39,0,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.6E1,3.2E1,3.7E1,2.9E1,1E1,2.2E1,4E0,3.3E1,2.4E1,5E0,6E0,4E0,9E0,1.3E1,9E0,2.4E1,2E1,4E0,5E0,8E0,5E0,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-azure","platform_databricks-aws","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.6600735E-2,-4.183997E-1,4.016022E-1,-5.4472244E-1,2.4702572E-3,-3.0444106E-1,5.4333526E-1,-2.095471E-1,-6.9834214E-1,1.3250048E-2,-7.2323172E-3,-6.7226007E-4,-1.9111963E-2,3.2672933E-1,8.139915E-1,-1.4351901E-2,3.8755343E-3,-2.7703207E-2,-1.1876688E-2,-2.0970795E-3,4.2583746E-1,3.311289E-2,1.6418E-2,2.9950497E-1,2.2299686E-2,5.3400216E-3,1.2468227E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,19,21,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,-1],"loss_changes":[1.7139414E1,2.7659492E0,5.1688957E0,1.9132452E0,1.0345602E0,5.687549E-1,2.1963263E0,8.766462E-1,4.3567657E-1,0E0,0E0,0E0,0E0,9.781935E-1,2.5014305E-1,0E0,0E0,0E0,0E0,0E0,2.9969501E-1,0E0,0E0,4.6219945E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,13,13,14,14,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,20,22,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,-1],"split_conditions":[3.3433047E-1,2.561146E3,2.80369E5,4.828443E-1,2.5927516E11,6.313997E-1,1.5364555E0,1.3564575E5,1.1653038E-2,1.3250048E-2,-7.2323172E-3,-6.7226007E-4,-1.9111963E-2,8.184232E5,2.326E4,-1.4351901E-2,3.8755343E-3,-2.7703207E-2,-1.1876688E-2,-2.0970795E-3,1.5791555E9,3.311289E-2,1.6418E-2,1.8589372E5,2.2299686E-2,5.3400216E-3,1.2468227E-2],"split_indices":[39,52,1,27,31,38,42,28,38,0,0,0,0,32,9,0,0,0,0,0,7,0,0,33,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.1E1,4.9E1,3.9E1,1.2E1,8E0,4.1E1,1.3E1,2.6E1,4E0,8E0,4E0,4E0,2.4E1,1.7E1,8E0,5E0,2.1E1,5E0,5E0,1.9E1,1.2E1,5E0,1.3E1,6E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[3.9494306E-2,-3.070603E-1,6.038862E-1,-4.283072E-1,2.940214E-1,8.1463903E-1,2.2642255E-1,-5.962956E-1,-1.721278E-1,1.8948603E-2,3.146083E-3,1.4225878E-2,9.0458536E-1,3.1656706E-1,-6.202453E-4,-6.8261397E-1,-4.4358503E-3,-1.1051375E-2,-5.5826362E-2,1.6590422E-2,3.583697E-2,4.103856E-3,1.5028562E-2,-7.388364E-3,-7.60215E-1,1.989614E-3,-7.0197335E-3,-1.7825114E-2,-3.1670492E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[1.9549099E1,4.644448E0,2.8385057E0,2.135189E0,4.7955102E-1,5.5152035E-1,3.3482993E-1,1.1829653E0,3.2571435E-1,0E0,0E0,0E0,2.883768E-1,1.7520332E-1,0E0,8.087826E-1,0E0,0E0,2.1061364E-1,0E0,0E0,0E0,0E0,0E0,2.5288773E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,18,18,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,3.5324219E3,1.4046429E1,1.1816701E-2,6.15786E5,2.1146134E5,1.2707124E1,2.2789746E3,1.2239595E3,1.8948603E-2,3.146083E-3,1.4225878E-2,3.8902016E8,7.765E3,-6.202453E-4,4.029937E-1,-4.4358503E-3,-1.1051375E-2,4.5843E7,1.6590422E-2,3.583697E-2,4.103856E-3,1.5028562E-2,-7.388364E-3,4.307821E7,1.989614E-3,-7.0197335E-3,-1.7825114E-2,-3.1670492E-2],"split_indices":[39,52,54,38,29,28,53,52,4,0,0,0,7,2,0,27,0,0,45,0,0,0,0,0,45,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.1E1,3.7E1,5.1E1,1E1,2.3E1,1.4E1,3E1,2.1E1,4E0,6E0,5E0,1.8E1,1E1,4E0,2.5E1,5E0,9E0,1.2E1,4E0,1.4E1,4E0,6E0,4E0,2.1E1,7E0,5E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.488806E-2,-3.7544534E-1,4.4622344E-1,-4.5515296E-1,1.4603574E-2,-3.0572563E-2,6.548415E-1,-2.9288656E-1,-7.6341426E-1,-1.2825903E-2,1.6137537E-1,4.3922216E-1,8.377188E-1,-2.4863131E-2,-1.7758082E-1,-1.2843414E-2,-3.2234408E-2,1.3643051E-2,2.1229606E-4,8.309624E-3,1.9282134E-2,2.190433E-2,3.800347E-2,-3.5284942E-1,-9.7808175E-2,-5.4097087E-3,-1.7520351E-2,4.9898024E-3,-1.7535536E-1,-4.5112697E-3,-9.795604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1],"loss_changes":[1.9396439E1,3.851984E0,5.5902433E0,2.4775124E0,0E0,1.1791408E0,1.169899E0,1.5662498E0,6.3215256E-1,0E0,4.0178597E-1,2.7371645E-1,3.542862E-1,0E0,3.8493752E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6925931E-1,4.2689222E-1,0E0,0E0,0E0,4.8322022E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,23,23,24,24,28,28],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1],"split_conditions":[1.5363757E8,7.621429E1,5.1707494E-1,5.070136E7,1.4603574E-2,3.0962205E-2,7.3845965E8,1.497E3,5.3323513E-1,-1.2825903E-2,2E0,1.218666E0,2.5172096E-1,-2.4863131E-2,1E0,-1.2843414E-2,-3.2234408E-2,1.3643051E-2,2.1229606E-4,8.309624E-3,1.9282134E-2,2.190433E-2,3.800347E-2,6.9533944E7,2.9000288E-2,-5.4097087E-3,-1.7520351E-2,4.9898024E-3,1.0128521E6,-4.5112697E-3,-9.795604E-3],"split_indices":[12,56,39,45,0,38,7,2,27,0,8,39,38,0,16,0,0,0,0,0,0,0,0,7,38,0,0,0,47,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.8E1,5.5E1,5.3E1,5E0,1.7E1,3.8E1,3.6E1,1.7E1,6E0,1.1E1,1.9E1,1.9E1,7E0,2.9E1,5E0,1.2E1,4E0,7E0,7E0,1.2E1,1.1E1,8E0,8E0,2.1E1,4E0,4E0,5E0,1.6E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-4.2027324E-2,-3.7183625E-1,5.0865483E-1,-1.9803534E-1,-6.767564E-1,1.4365476E-1,8.339433E-1,-5.00647E-1,7.6184355E-2,-7.7971935E-1,-1.0852783E-2,1.6833028E-2,1.3703063E-2,1.730201E-2,3.6189623E-2,-7.088462E-1,-1.7992336E-1,-1.0535246E-1,3.168087E-1,-3.1054046E-2,-1.6682882E-2,-6.622137E-3,1.5032132E-1,-1.8500404E-2,-3.3237103E-2,-2.6283695E-3,-9.837503E-3,6.741438E-2,-1.1279693E-2,1.9201145E-2,4.2584944E-3,1.3669189E-3,7.905985E-3,5.1801493E-3,-7.9919887E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,-1,23,25,27,29,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0348944E1,3.575656E0,4.8668747E0,3.855317E0,7.338495E-1,8.739692E-1,8.3414936E-1,1.3414993E0,1.125431E0,8.793354E-2,0E0,0E0,4.5846978E-1,0E0,0E0,6.897545E-2,7.9315215E-2,5.577067E-1,4.0763426E-1,0E0,0E0,0E0,6.892693E-2,0E0,0E0,0E0,0E0,6.448403E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15,16,16,17,17,18,18,22,22,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,-1,24,26,28,30,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2310118E7,5.9633E7,1.2480776E0,1.3486356E3,3.5904072E3,1.5306258E5,4.867052E6,1.977643E6,9E0,3.1346153E1,-1.0852783E-2,1.6833028E-2,6.919127E5,1.730201E-2,3.6189623E-2,7.472597E7,1.1954737E1,3.863112E7,4.3650795E-2,-3.1054046E-2,-1.6682882E-2,-6.622137E-3,4.340646E-1,-1.8500404E-2,-3.3237103E-2,-2.6283695E-3,-9.837503E-3,5.526E3,-1.1279693E-2,1.9201145E-2,4.2584944E-3,1.3669189E-3,7.905985E-3,5.1801493E-3,-7.9919887E-4],"split_indices":[50,45,39,4,4,28,1,48,8,58,0,0,32,0,0,7,54,45,57,0,0,0,39,0,0,0,0,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,6.9E1,4.1E1,4.5E1,2.4E1,2E1,2.1E1,2.1E1,2.4E1,1.8E1,6E0,5E0,1.5E1,8E0,1.3E1,1.2E1,9E0,1.4E1,1E1,1.3E1,5E0,6E0,9E0,8E0,4E0,5E0,4E0,8E0,6E0,4E0,6E0,4E0,5E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.0581495E-2,-3.905329E-1,5.229766E-1,-2.1395065E-1,-6.927597E-1,2.7906787E-1,7.874607E-1,-3.7685356E-1,1.7368104E-1,-1.1198943E-2,-2.8782893E-2,4.6702987E-1,5.5180494E-2,1.8140873E-2,8.869554E-1,-6.437481E-1,-1.6036002E-1,-6.7222877E-3,3.9430216E-1,1.015999E-2,2.2110943E-2,-5.6470808E-2,8.849449E-3,2.1888694E-2,3.8152087E-2,-2.8269665E-2,-1.5936205E-2,-2.3753755E-1,2.1222164E-4,2.22954E-2,3.328482E-3,-4.3679755E-3,6.981977E-4,-3.8479902E-3,-1.024952E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.33296E1,3.5341682E0,2.812214E0,2.907037E0,7.5199986E-1,1.0683694E0,1.7812729E-1,1.745758E0,1.1849732E0,0E0,0E0,2.0551062E-1,2.8952807E-1,0E0,7.4178696E-2,4.890871E-2,2.4516359E-1,0E0,5.5007005E-1,0E0,0E0,4.6358094E-2,0E0,0E0,0E0,0E0,0E0,4.866296E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,18,18,21,21,27,27],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,5.4875424E7,1.3536367E0,1.0664373E3,5.3323513E-1,9.48E2,3.1E1,2.667836E6,1E0,-1.1198943E-2,-2.8782893E-2,4.903324E1,6.9015497E-1,1.8140873E-2,1.9088202E0,2.3E1,2.3514317E8,-6.7222877E-3,9.070543E0,1.015999E-2,2.2110943E-2,2.1E1,8.849449E-3,2.1888694E-2,3.8152087E-2,-2.8269665E-2,-1.5936205E-2,2.5445784E1,2.1222164E-4,2.22954E-2,3.328482E-3,-4.3679755E-3,6.981977E-4,-3.8479902E-3,-1.024952E-2],"split_indices":[12,45,39,52,27,0,3,32,16,0,0,56,39,0,42,3,7,0,53,0,0,8,0,0,0,0,0,58,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6.8E1,4.6E1,4.4E1,2.4E1,2.5E1,2.1E1,3.1E1,1.3E1,6E0,1.8E1,1.3E1,1.2E1,7E0,1.4E1,1.3E1,1.8E1,5E0,8E0,7E0,6E0,8E0,4E0,7E0,7E0,6E0,7E0,1.2E1,6E0,4E0,4E0,4E0,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.979678E-3,-3.4272724E-1,4.9574625E-1,-5.276246E-1,-1.013813E-1,2.4871044E-1,7.319686E-1,-7.0078516E-1,-2.0545907E-1,1.7967814E-1,-2.8312197E-1,-2.1515675E-2,3.8516822E-1,2.9084628E-2,1.6195044E-2,-1.3636926E-2,-2.7782705E-2,-1.0596163E-2,-3.7093132E-3,-3.6961345E-3,2.1482423E-2,-3.5835654E-1,-2.632875E-3,-6.17006E-3,4.7718557E-3,1.934534E-2,2.4118295E-1,-6.990057E-3,-1.480449E-2,4.890936E-3,1.0436601E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.8815786E1,2.8024602E0,2.5022974E0,1.8643255E0,1.5321889E0,8.9327407E-1,1.1856079E-1,2.2020626E-1,9.644824E-2,1.466751E0,2.5851727E-1,2.2921947E-1,2.1787167E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.1342387E-2,0E0,0E0,0E0,0E0,1.1469007E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,21,21,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[1.8147125E5,3.3041668E1,1.3536367E0,2.9282052E6,6.15786E5,4.7308203E3,2.1685E4,8.977316E0,2.207872E-1,3.09E2,2.7938022E3,4.226681E-1,6.19999E9,2.9084628E-2,1.6195044E-2,-1.3636926E-2,-2.7782705E-2,-1.0596163E-2,-3.7093132E-3,-3.6961345E-3,2.1482423E-2,2.9673992E9,-2.632875E-3,-6.17006E-3,4.7718557E-3,1.934534E-2,5.5918517E0,-6.990057E-3,-1.480449E-2,4.890936E-3,1.0436601E-2],"split_indices":[33,56,39,32,29,4,9,54,39,0,52,39,5,0,0,0,0,0,0,0,0,5,0,0,0,0,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,6.3E1,4.5E1,3.5E1,2.8E1,2.3E1,2.2E1,2.2E1,1.3E1,1.1E1,1.7E1,8E0,1.5E1,1.6E1,6E0,5E0,1.7E1,6E0,7E0,7E0,4E0,1.2E1,5E0,4E0,4E0,6E0,9E0,4E0,8E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.1256974E-2,-2.8215232E-1,5.441066E-1,-4.1236064E-1,5.7476006E-2,1.9641325E-4,6.3900536E-1,-2.2134565E-1,-6.4540946E-1,1.152904E-2,-1.0809073E-1,3.4943733E-1,8.0162185E-1,-3.7772104E-1,-1.19162746E-1,-1.0983238E-2,-2.6278079E-2,-6.874574E-3,7.108995E-4,1.5086514E-2,5.151032E-3,2.1407075E-2,3.850762E-2,-1.6439706E-2,-8.56121E-3,2.1845386E-3,-1.7110898E-1,-8.0796555E-3,-7.5740006E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.7286366E1,3.075047E0,2.041273E0,2.0829325E0,9.016283E-1,0E0,1.3208942E0,4.1837752E-1,3.9748955E-1,0E0,1.4050426E-1,1.384809E-1,5.2877617E-1,1.2079835E-2,1.8768522E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1395046E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[5.1707494E-1,2.5323425E3,1.1462246E3,4.851587E7,6.15786E5,1.9641325E-4,1.5083523E0,7.8714124E2,5.3323513E-1,1.152904E-2,1.7802E4,5.5E1,1.5605432E9,1.44E2,1.77E2,-1.0983238E-2,-2.6278079E-2,-6.874574E-3,7.108995E-4,1.5086514E-2,5.151032E-3,2.1407075E-2,3.850762E-2,-1.6439706E-2,-8.56121E-3,2.1845386E-3,4.01E2,-8.0796555E-3,-7.5740006E-4],"split_indices":[39,52,52,45,29,0,42,4,27,0,10,3,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.8E1,3.9E1,4.9E1,1.9E1,6E0,3.3E1,2.8E1,2.1E1,7E0,1.2E1,1.3E1,2E1,1E1,1.8E1,5E0,1.6E1,7E0,5E0,9E0,4E0,1.3E1,7E0,5E0,5E0,4E0,1.4E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.717804E-2,-3.7835562E-1,4.055E-1,-4.9538177E-1,5.748244E-3,1.4678942E-1,2.4617178E-2,-6.4040095E-1,-2.3421851E-1,1.187477E-2,-1.8872583E-1,3.5689092E-1,1.943298E-2,-1.0111054E-2,-6.868529E-1,-1.2533823E-2,-8.269428E-2,-1.2363759E-2,1.2093743E-3,5.453824E-3,1.7738875E-2,-1.2941217E-1,6.659366E-3,-1.4572546E-2,-2.7964214E-2,6.718205E-4,-6.0457406E-3,-1.0731786E-2,2.3218838E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,23,-1,25,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5594854E1,2.6998606E0,3.0457854E0,1.5921955E0,1.0047839E0,6.352512E-1,0E0,2.9305458E-1,2.7935505E-1,0E0,3.531009E-1,1.7483854E-1,4.1687348E-1,0E0,3.0840778E-1,0E0,8.039741E-2,0E0,0E0,0E0,0E0,3.117548E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,14,14,16,16,21,21],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,24,-1,26,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,2.667836E6,6.15786E5,1.1E1,2.4617178E-2,2.3452184E5,1.6757976E-1,1.187477E-2,2.4502628E6,1.0262E4,2.3342845E5,-1.0111054E-2,4.4E1,-1.2533823E-2,1.675502E8,-1.2363759E-2,1.2093743E-3,5.453824E-3,1.7738875E-2,1.4634906E-1,6.659366E-3,-1.4572546E-2,-2.7964214E-2,6.718205E-4,-6.0457406E-3,-1.0731786E-2,2.3218838E-3],"split_indices":[12,2,39,32,29,8,0,28,39,0,32,2,33,0,10,0,5,0,0,0,0,38,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.9E1,4.3E1,4.5E1,1.4E1,2.3E1,2E1,2.8E1,1.7E1,5E0,9E0,8E0,1.5E1,4E0,2.4E1,9E0,8E0,5E0,4E0,4E0,4E0,8E0,7E0,7E0,1.7E1,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.0237521E-2,-3.393916E-1,3.3153063E-1,-4.1472912E-1,1.794064E-2,1.8513696E-1,5.837542E-1,-5.59573E-1,-1.6224243E-1,-9.584433E-3,2.518441E-1,1.3587898E-2,2.431059E-2,-6.446323E-1,-5.9100497E-3,-2.562468E-1,3.6034028E-3,3.3394757E-1,1.4041496E-2,-1.2825363E-2,-7.3255205E-1,-1.2800422E-2,-1.513147E-1,3.880833E-1,2.2182493E-3,-4.391941E-3,5.3044334E-3,-3.0928945E-2,-1.80312E-2,-7.1180547E-3,-2.7151913E-3,2.967635E-1,2.109953E-2,1.3289075E-2,7.5208065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,21,-1,23,25,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1],"loss_changes":[1.2149417E1,3.6714392E0,1.8488941E0,1.7743425E0,0E0,1.1189239E0,9.96809E-2,9.720001E-1,5.1825535E-1,0E0,6.005678E-1,0E0,0E0,4.012823E-1,0E0,1.010828E-1,0E0,3.1545758E-1,1.8015428E-1,0E0,1.219511E-1,0E0,1.657471E-2,1.6723347E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7949343E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,15,15,17,17,18,18,20,20,22,22,23,23,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,22,-1,24,26,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1],"split_conditions":[2.0020565E8,1.0269E4,1.3536367E0,2.9282052E6,1.794064E-2,3.0962205E-2,4.1372877E8,3.159013E3,2.1816934E8,-9.584433E-3,1.8634315E1,1.3587898E-2,2.431059E-2,4.4E1,-5.9100497E-3,2.1225102E-1,3.6034028E-3,2.3152558E9,1.5959719E-1,-1.2825363E-2,1.2599278E1,-1.2800422E-2,2.862E3,1.62739E9,2.2182493E-3,-4.391941E-3,5.3044334E-3,-3.0928945E-2,-1.80312E-2,-7.1180547E-3,-2.7151913E-3,2.0246488E5,2.109953E-2,1.3289075E-2,7.5208065E-3],"split_indices":[12,2,39,32,0,38,7,4,7,0,54,0,0,10,0,39,0,7,38,0,53,0,2,7,0,0,0,0,0,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,5.4E1,5.2E1,5E1,4E0,3.4E1,1.8E1,3.1E1,1.9E1,4E0,3E1,7E0,1.1E1,2.5E1,6E0,1.4E1,5E0,2.2E1,8E0,7E0,1.8E1,6E0,8E0,1.8E1,4E0,4E0,4E0,1E1,8E0,4E0,4E0,1.4E1,4E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-6.7869793E-3,-3.906748E-1,4.028442E-1,-5.969478E-1,-7.2045274E-2,2.2578067E-1,7.419192E-1,-3.3789346E-1,-2.5150344E-2,-2.10561E-1,7.7721947E-3,4.3038604E-1,5.4099772E-2,1.792945E-2,3.229338E-2,-5.669667E-3,-1.56068895E-2,-8.948621E-3,-3.3880319E-3,4.431246E-3,5.4117095E-1,-1.1790051E-1,1.0247393E-2,2.496625E-2,9.116291E-3,-6.723284E-3,4.8160528E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1,-1],"loss_changes":[1.4624603E1,3.1117768E0,2.5341916E0,5.134649E-1,8.413062E-1,1.0656481E0,1.1712742E-1,1.0282707E-1,0E0,3.76938E-2,0E0,4.1073418E-1,7.4473435E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5941038E-1,8.986153E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,4.852701E6,1.3536367E0,6.072427E-1,9.63264E2,8.69374E5,1.5605432E9,4.80378E5,-2.5150344E-2,3.670858E1,7.7721947E-3,1.2288E4,2.3342845E5,1.792945E-2,3.229338E-2,-5.669667E-3,-1.56068895E-2,-8.948621E-3,-3.3880319E-3,4.431246E-3,1.9045439E5,1.8154767E5,1.0247393E-2,2.496625E-2,9.116291E-3,-6.723284E-3,4.8160528E-5],"split_indices":[2,32,39,27,52,29,7,12,0,58,0,2,33,0,0,0,0,0,0,0,33,28,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.1E1,4.7E1,4.4E1,2.8E1,1.9E1,3E1,1.4E1,9E0,1.9E1,1.3E1,6E0,1.3E1,1.7E1,7E0,7E0,4E0,5E0,9E0,4E0,4E0,9E0,1E1,7E0,5E0,4E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.1054016E-2,-3.1460762E-1,4.859552E-1,-4.4477397E-1,6.504634E-2,-9.317041E-3,5.7835984E-1,-5.9096265E-1,-2.7023336E-1,1.4754604E-2,-8.687175E-2,4.267692E-1,8.703059E-1,-3.6451545E-1,-2.4515454E-2,-3.1843606E-1,-2.0392803E-3,-9.375171E-3,8.218775E-3,9.218639E-3,1.83128E-2,1.7255593E-2,3.704989E-2,-7.613364E-3,-1.5589032E-2,-9.2998715E-3,-1.5438079E-2,-2.0880434E-3,3.099097E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6132267E1,3.5376215E0,2.830555E0,1.1926422E0,1.0273844E0,0E0,1.1775494E0,3.1094837E-1,2.4989724E-1,0E0,2.3218612E-1,2.1534109E-1,1.8244553E-1,1.2307167E-2,0E0,2.9345512E-2,0E0,0E0,5.6237247E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,9.066E3,2.80369E5,3.572374E6,6.52678E5,-9.317041E-3,1.62739E9,4.4E1,3.365958E8,1.4754604E-2,5.8163033E1,2.0681982E-1,2.0105577E-1,5.189068E-1,-2.4515454E-2,7.959073E-1,-2.0392803E-3,-9.375171E-3,1.662E3,9.218639E-3,1.83128E-2,1.7255593E-2,3.704989E-2,-7.613364E-3,-1.5589032E-2,-9.2998715E-3,-1.5438079E-2,-2.0880434E-3,3.099097E-3],"split_indices":[42,2,1,12,29,0,7,10,7,0,56,38,38,27,0,27,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7E1,3.8E1,5.2E1,1.8E1,4E0,3.4E1,2.7E1,2.5E1,5E0,1.3E1,2.4E1,1E1,9E0,1.8E1,2E1,5E0,4E0,9E0,9E0,1.5E1,4E0,6E0,4E0,5E0,1.5E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.0797309E-2,-3.6156452E-1,3.3161274E-1,-4.2126018E-1,5.285972E-3,1.06708296E-1,6.242183E-1,-5.598041E-1,-2.0221046E-1,-8.68231E-3,1.8969616E-1,4.364018E-1,3.2840192E-2,-3.724325E-1,-2.7545957E-2,-1.3955579E-2,-1.2728357E-1,2.8850928E-1,1.075685E-2,1.87067E-2,8.866908E-3,-4.663343E-1,-4.9896906E-3,6.6139596E-4,-7.289815E-3,3.6883238E-1,5.126675E-3,3.4905286E-3,-4.03236E-3,-2.0380057E-2,-1.04860645E-2,1.781412E-2,4.355333E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,-1,-1,23,25,27,-1,-1,29,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2902058E1,1.5778832E0,3.7502704E0,1.2524624E0,0E0,1.025262E0,9.4314003E-1,7.437258E-1,2.13225E-1,0E0,4.9198008E-1,1.3789701E-1,0E0,2.8745627E-1,0E0,0E0,1.667025E-1,1.5944874E-1,1.2653352E-1,0E0,0E0,2.5963306E-2,0E0,0E0,0E0,2.8063214E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,21,21,25,25],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,-1,-1,24,26,28,-1,-1,30,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4350427E8,2.7938022E3,1.3536367E0,2.667836E6,5.285972E-3,4.739085E-2,1.0861106E4,4.6268812E5,1E0,-8.68231E-3,5E1,1.2271712E1,3.2840192E-2,1.9121604E-4,-2.7545957E-2,-1.3955579E-2,2.1923357E1,1E0,3.7E1,1.87067E-2,8.866908E-3,3.8E1,-4.9896906E-3,6.6139596E-4,-7.289815E-3,1.4273524E-1,5.126675E-3,3.4905286E-3,-4.03236E-3,-2.0380057E-2,-1.04860645E-2,1.781412E-2,4.355333E-3],"split_indices":[12,52,39,32,0,38,4,32,16,0,3,54,0,38,0,0,58,108,8,0,0,3,0,0,0,38,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,4.9E1,5.7E1,4.4E1,5E0,3.3E1,2.4E1,2.6E1,1.8E1,6E0,2.7E1,1.6E1,8E0,1.5E1,1.1E1,4E0,1.4E1,1.7E1,1E1,1E1,6E0,1E1,5E0,5E0,9E0,1E1,7E0,6E0,4E0,5E0,5E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-3.439152E-2,-2.8745982E-1,4.0628415E-1,-4.0453276E-1,1.5489583E-1,1.832985E-2,5.115344E-1,-2.7192265E-1,-5.81067E-1,-2.951975E-2,1.6866421E-2,4.405638E-3,-2.5684275E-3,3.12952E-1,6.413871E-1,-1.4641315E-1,-3.4862357E-1,-6.277753E-1,-1.0037368E-2,2.4273365E-3,-3.408564E-3,1.3045562E-2,6.50366E-3,1.4951235E-2,3.0611156E-2,-8.457483E-3,2.7486004E-3,-8.842935E-3,-1.5185564E-2,-1.171555E-2,-2.4669606E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.283019E1,3.843176E0,1.7055788E0,1.1997871E0,9.352175E-1,1.0146322E-1,6.5023136E-1,2.858913E-1,1.6518116E-1,7.541191E-2,0E0,0E0,0E0,3.1309128E-2,5.405083E-1,3.1108278E-1,5.3738117E-2,1.1511612E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,2.7938022E3,1.118632E3,8.0073607E-1,6.082581E3,9.5032835E0,1.5941123E0,2.8399336E7,2.7240606E5,1.8345214E-1,1.6866421E-2,4.405638E-3,-2.5684275E-3,1.5366939E7,1.286E4,1.2108435E3,2.1227852E5,2.07E2,-1.0037368E-2,2.4273365E-3,-3.408564E-3,1.3045562E-2,6.50366E-3,1.4951235E-2,3.0611156E-2,-8.457483E-3,2.7486004E-3,-8.842935E-3,-1.5185564E-2,-1.171555E-2,-2.4669606E-2],"split_indices":[42,52,52,27,52,53,39,45,47,27,0,0,0,47,2,4,28,11,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,7.2E1,4.1E1,5.7E1,1.5E1,9E0,3.2E1,3.4E1,2.3E1,1E1,5E0,4E0,5E0,1.4E1,1.8E1,1.4E1,2E1,1.9E1,4E0,4E0,6E0,9E0,5E0,1E1,8E0,1E1,4E0,1E1,1E1,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.472108E-2,-2.642644E-1,4.434464E-1,-3.4340566E-1,6.654095E-2,2.0786707E-1,6.330183E-1,-5.238764E-1,-1.6358441E-1,1.0051563E-2,-3.6348896E-3,-3.4342918E-3,3.025779E-1,4.346444E-1,3.3842564E-2,-3.2459053E-1,-2.2928808E-2,-2.596711E-1,2.5730023E-2,2.8112214E-3,3.8971868E-1,1.8355465E-2,1.0401099E-2,-6.572296E-3,-1.4911974E-2,-1.3982144E-2,-8.9025676E-2,3.8719538E-3,-2.7884827E-3,1.7398123E-2,9.207346E-3,-5.392798E-3,-3.92571E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3315104E1,1.6722131E0,1.9227409E0,1.5693779E0,4.8861796E-1,6.601082E-1,1.1020508E0,3.479581E-1,4.9782735E-1,0E0,0E0,0E0,3.025396E-1,3.7325144E-2,0E0,6.349945E-2,0E0,3.5750234E-1,9.22194E-2,0E0,2.609384E-2,0E0,0E0,0E0,0E0,0E0,4.082188E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,17,17,18,18,20,20,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[9.095874E6,2.5323425E3,1.5364555E0,2.667836E6,6.15786E5,4.1451712E5,1.5605432E9,1.08E2,2.0237808E8,1.0051563E-2,-3.6348896E-3,-3.4342918E-3,3.486353E6,2.326E4,3.3842564E-2,8.317631E-1,-2.2928808E-2,2.6326587E-2,9.242E3,2.8112214E-3,7.118011E-1,1.8355465E-2,1.0401099E-2,-6.572296E-3,-1.4911974E-2,-1.3982144E-2,2.68E2,3.8719538E-3,-2.7884827E-3,1.7398123E-2,9.207346E-3,-5.392798E-3,-3.92571E-4],"split_indices":[48,52,42,32,29,32,7,10,7,0,0,0,1,9,0,27,0,38,9,0,27,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,6.2E1,4.5E1,5E1,1.2E1,2.1E1,2.4E1,2.4E1,2.6E1,5E0,7E0,5E0,1.6E1,1.6E1,8E0,1E1,1.4E1,1.7E1,9E0,5E0,1.1E1,9E0,7E0,5E0,5E0,9E0,8E0,5E0,4E0,5E0,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.6608386E-3,-2.7908042E-1,3.9974752E-1,-3.936006E-1,3.6976933E-2,1.7811656E-1,6.047223E-1,-5.568649E-1,-1.6049762E-1,2.739653E-1,-1.1665881E-2,3.156085E-1,8.964504E-2,4.4029915E-1,3.1717613E-2,-6.084411E-1,-6.773321E-3,-2.2856256E-1,-4.6261054E-4,1.5954038E-3,1.7920634E-2,1.399342E-2,6.5165246E-3,-3.280731E-2,9.089672E-3,1.8156307E-2,1.0073523E-2,-1.3630825E-2,-7.147181E-1,-1.0780131E-2,-2.6625989E-3,-3.5189209E-3,9.583637E-4,-2.8507112E-2,-1.4154024E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2806435E1,2.4758105E0,2.0274415E0,1.8134947E0,1.7099975E0,2.6364613E-1,6.9367695E-1,4.3814564E-1,2.1821755E-1,5.913157E-1,0E0,1.7577112E-2,3.2590234E-1,3.132105E-2,0E0,3.4360504E-1,0E0,1.4060438E-1,0E0,0E0,0E0,0E0,0E0,4.096568E-2,0E0,0E0,0E0,0E0,7.706261E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.5364555E0,2.667836E6,6.1358623E9,1.7644464E5,1.5605432E9,3.0387878E3,3.167669E5,1.1653E4,-1.1665881E-2,1.0550945E2,3.9219662E5,2.0837E4,3.1717613E-2,2.63E2,-6.773321E-3,2.207872E-1,-4.6261054E-4,1.5954038E-3,1.7920634E-2,1.399342E-2,6.5165246E-3,4.152984E-1,9.089672E-3,1.8156307E-2,1.0073523E-2,-1.3630825E-2,1.2599278E1,-1.0780131E-2,-2.6625989E-3,-3.5189209E-3,9.583637E-4,-2.8507112E-2,-1.4154024E-2],"split_indices":[50,2,42,32,5,28,7,4,28,2,0,57,28,9,0,11,0,39,0,0,0,0,0,39,0,0,0,0,53,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,6.7E1,4.6E1,4.9E1,1.8E1,2.3E1,2.3E1,2.8E1,2.1E1,1.1E1,7E0,8E0,1.5E1,1.6E1,7E0,2.4E1,4E0,1.4E1,7E0,6E0,5E0,4E0,4E0,9E0,6E0,1E1,6E0,9E0,1.5E1,9E0,5E0,4E0,5E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.1170119E-2,-2.286945E-1,2.9556155E-1,-2.757291E-1,9.817417E-3,7.455043E-2,5.5917764E-1,-4.1882515E-1,-1.4190349E-1,1.7954913E-1,-1.20397314E-1,2.1995092E-2,1.0729085E-2,-5.0299E-1,-2.1702932E-1,-2.541205E-1,-2.7940229E-2,4.6226624E-4,2.7831575E-1,-6.8809628E-3,-9.431044E-4,-8.674969E-3,-6.057556E-1,-2.6501452E-3,-1.0850561E-2,-1.1013974E-2,-4.2730784E-3,-1.1090766E-1,2.861581E-3,1.2885317E-2,4.6336856E-3,-2.5775805E-2,-1.3016385E-2,-1.4181143E-3,-6.306461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9691954E0,1.581959E0,2.4053206E0,1.0704117E0,0E0,5.128795E-1,6.811571E-2,3.6639166E-1,4.021356E-1,2.5993794E-1,5.457996E-2,0E0,0E0,3.6739302E-1,9.788659E-2,7.456696E-2,1.5999314E-1,0E0,7.2051585E-2,0E0,0E0,0E0,8.751106E-2,0E0,0E0,0E0,0E0,3.8817257E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,22,22,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.379017E6,1.2424786E2,1.3589408E0,2.667836E6,9.817417E-3,5.6E1,2.1383E4,2.6631096E3,2.834008E-1,1.1015E4,1.0448E4,2.1995092E-2,1.0729085E-2,4.4E1,8.661788E7,2.942787E6,1.3448806E8,4.6226624E-4,2.581938E2,-6.8809628E-3,-9.431044E-4,-8.674969E-3,1.2599278E1,-2.6501452E-3,-1.0850561E-2,-1.1013974E-2,-4.2730784E-3,2.4E1,2.861581E-3,1.2885317E-2,4.6336856E-3,-2.5775805E-2,-1.3016385E-2,-1.4181143E-3,-6.306461E-3],"split_indices":[51,56,42,32,0,3,9,55,57,2,9,0,0,10,45,51,7,0,56,0,0,0,53,0,0,0,0,3,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,6.3E1,4.1E1,5.8E1,5E0,2.3E1,1.8E1,2.7E1,3.1E1,1.5E1,8E0,1.4E1,4E0,1.8E1,9E0,1.5E1,1.6E1,6E0,9E0,4E0,4E0,6E0,1.2E1,4E0,5E0,1E1,5E0,9E0,7E0,5E0,4E0,7E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.6760997E-2,-1.9793408E-1,2.6402497E-1,-2.781832E-1,2.7125448E-1,-4.521447E-3,3.2001278E-1,-1.6120237E-1,-4.5164436E-1,1.738997E-2,1.8301925E-3,1.1050596E-3,3.8652018E-1,-2.1479665E-1,-2.4742452E-2,-5.228169E-1,-3.794221E-3,2.1083939E-1,4.408205E-1,-2.7492607E-1,-7.7352576E-2,3.2140955E-3,-4.8517697E-3,-1.2521984E-2,-2.1779887E-2,4.294831E-3,9.40665E-3,4.914743E-1,9.4613675E-3,-1.109944E-2,-4.8886845E-3,9.531097E-4,-5.979898E-3,1.0108562E-2,2.0295251E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,21,23,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4445705E0,2.5228367E0,9.554236E-1,1.0635252E0,4.3969083E-1,0E0,7.003269E-1,2.506569E-1,4.876814E-1,0E0,0E0,0E0,1.932435E-1,1.8534458E-1,1.4917842E-1,4.5605183E-2,0E0,1.0111302E-2,7.040262E-2,3.548169E-2,8.680768E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.1743956E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,22,24,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,5.188755E3,3.727E3,8.317631E-1,8.69374E5,-4.521447E-3,1E0,7.287E3,3.271557E8,1.738997E-2,1.8301925E-3,1.1050596E-3,1.2756285E3,2.834008E-1,4.5843E7,4.4815472E5,-3.794221E-3,1.2958418E5,1.0007143E1,3.7712722E6,3.39E2,3.2140955E-3,-4.8517697E-3,-1.2521984E-2,-2.1779887E-2,4.294831E-3,9.40665E-3,4.0383283E8,9.4613675E-3,-1.109944E-2,-4.8886845E-3,9.531097E-4,-5.979898E-3,1.0108562E-2,2.0295251E-2],"split_indices":[42,4,2,27,29,0,109,2,7,0,0,0,52,57,45,28,0,33,53,50,0,0,0,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.4E1,4.1E1,5.5E1,9E0,5E0,3.6E1,3.4E1,2.1E1,4E0,5E0,7E0,2.9E1,2.4E1,1E1,1.7E1,4E0,8E0,2.1E1,1.6E1,8E0,5E0,5E0,7E0,1E1,4E0,4E0,1.5E1,6E0,1.2E1,4E0,4E0,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7335935E-2,-2.5816494E-1,2.9193032E-1,-3.4373537E-1,1.05630405E-1,1.6434705E-1,6.829365E-1,-4.440817E-1,-1.3462842E-1,-3.7841632E-3,1.2042006E-2,-2.8410375E-3,2.1557806E-1,1.4404345E-2,2.845641E-2,-4.9313763E-1,-2.9081749E-3,-9.198718E-3,-2.5035532E-2,2.7649057E-1,-2.6771717E-3,-5.622826E-1,-1.1280322E-2,2.1476792E-3,-4.3849307E-3,3.290193E-3,3.435109E-1,-1.0248506E-2,-2.2297278E-2,1.6219167E-2,7.725943E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[7.8213167E0,1.8729858E0,2.18251E0,9.4740343E-1,6.105282E-1,4.652062E-1,1.4320374E-2,5.309415E-1,2.174935E-1,0E0,0E0,0E0,5.4995143E-1,0E0,0E0,1.7414093E-1,0E0,0E0,8.87228E-2,2.7926195E-1,0E0,1.17313385E-1,0E0,0E0,0E0,0E0,1.5035939E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,18,18,19,19,21,21,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[2.3724032E8,2.561146E3,1.8614753E0,3.2936362E6,1E0,2.2843675E-1,7.4684826E8,7.33E3,2.9544958E-1,-3.7841632E-3,1.2042006E-2,-2.8410375E-3,2.1972223E2,1.4404345E-2,2.845641E-2,2.9282052E6,-2.9081749E-3,-9.198718E-3,3.51387E5,8.784E3,-2.6771717E-3,2.63E2,-1.1280322E-2,2.1476792E-3,-4.3849307E-3,3.290193E-3,1.5625807E1,-1.0248506E-2,-2.2297278E-2,1.6219167E-2,7.725943E-3],"split_indices":[7,52,39,50,16,39,7,2,41,0,0,0,57,0,0,32,0,0,9,2,0,11,0,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,5.8E1,4.5E1,4.7E1,1.1E1,3.5E1,1E1,3.1E1,1.6E1,6E0,5E0,6E0,2.9E1,4E0,6E0,2.7E1,4E0,7E0,9E0,2.4E1,5E0,1.8E1,9E0,5E0,4E0,7E0,1.7E1,4E0,1.4E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5175937E-2,-1.7119141E-1,4.021599E-1,-2.3383586E-1,1.771783E-1,-9.246658E-3,4.8470944E-1,-9.42045E-2,-3.9910802E-1,1.671502E-2,-8.5188553E-4,1.0748329E-2,5.9519696E-1,-3.2642293E-1,7.6650004E-3,-1.8785229E-2,-2.3206866E-1,3.7581056E-1,2.8844167E-2,-1.50354775E-2,-6.5700733E-3,1.6668053E-1,-7.22686E-2,-1.0507577E-2,-4.74809E-3,1.5626846E-2,7.5599775E-3,9.283002E-3,1.5487989E-3,-1.13182046E-1,2.048584E-3,-4.8094895E-3,-1.8224122E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,21,-1,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[8.433935E0,1.6781878E0,2.2434268E0,1.4393415E0,6.995077E-1,0E0,5.459299E-1,8.5411704E-1,4.6643257E-1,0E0,0E0,0E0,6.3842154E-1,6.726909E-2,3.4252357E-1,0E0,4.47464E-2,2.3247719E-2,0E0,0E0,0E0,8.972788E-2,1.0143166E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0862216E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,14,14,16,16,17,17,21,21,22,22,29,29],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,22,-1,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[1.0230335E0,3.5324219E3,1.4960615E3,5.070136E7,5.4875424E7,-9.246658E-3,1.7908138E0,7.998101E2,3.0595828E3,1.671502E-2,-8.5188553E-4,1.0748329E-2,2.463E4,1.54E2,5.83E2,-1.8785229E-2,3.4707712E8,1.2257767E1,2.8844167E-2,-1.50354775E-2,-6.5700733E-3,9.20242E0,4.7992307E-1,-1.0507577E-2,-4.74809E-3,1.5626846E-2,7.5599775E-3,9.283002E-3,1.5487989E-3,4.01E2,2.048584E-3,-4.8094895E-3,-1.8224122E-3],"split_indices":[39,52,4,45,45,0,42,4,4,0,0,0,2,0,10,0,7,54,0,0,0,53,39,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.4E1,3.8E1,6.3E1,1.1E1,4E0,3.4E1,3.5E1,2.8E1,4E0,7E0,1.4E1,2E1,1E1,2.5E1,1.5E1,1.3E1,1.1E1,9E0,5E0,5E0,8E0,1.7E1,7E0,6E0,7E0,4E0,4E0,4E0,1.3E1,4E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.2068253E-2,-1.648847E-1,3.3613706E-1,-3.5189933E-1,5.7612408E-2,4.1618764E-1,2.6709302E-3,-4.3274614E-1,-1.9394387E-1,2.1607071E-1,-2.1128578E-1,2.1955945E-2,3.239555E-1,-2.0425077E-1,-1.9531492E-2,-9.119429E-3,-1.3666184E-3,-1.1566876E-3,2.8010044E-1,-1.1187661E-2,-4.449788E-3,6.5825083E-3,1.3277747E-2,-1.1341023E-2,-1.9322936E-3,1.3200576E-2,4.2562406E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.057305E0,2.7665477E0,5.777972E-1,3.7579823E-1,1.3665363E0,1.9249415E-1,0E0,4.58385E-1,1.1860639E-1,3.267014E-1,6.596291E-2,0E0,3.84475E-2,1.3224477E-1,0E0,0E0,0E0,0E0,1.7888987E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,6.363496E7,3.143873E6,2.9282052E6,2.7070812E7,1.9718067E5,2.6709302E-3,4.5E1,4E1,8.873786E0,1.9452468E-2,2.1955945E-2,3.9E1,1.7E2,-1.9531492E-2,-9.119429E-3,-1.3666184E-3,-1.1566876E-3,2.26182E5,-1.1187661E-2,-4.449788E-3,6.5825083E-3,1.3277747E-2,-1.1341023E-2,-1.9322936E-3,1.3200576E-2,4.2562406E-3],"split_indices":[52,12,29,32,45,33,0,10,3,56,38,0,3,0,0,0,0,0,29,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.3E1,6.5E1,2.8E1,3.5E1,3E1,2.1E1,7E0,2.2E1,1.3E1,1.9E1,1.1E1,5E0,1.6E1,8E0,1.4E1,9E0,4E0,4E0,1.5E1,4E0,7E0,5E0,1.1E1,4E0,4E0,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.045447E-3,-1.3045764E-1,5.493323E-1,-2.3233856E-1,1.6364071E-1,1.0528478E-2,6.968363E-1,-4.1000903E-1,-1.4423028E-1,-1.806817E-3,2.225081E-1,2.9008435E-2,1.39654195E-2,-4.578551E-1,-5.6234505E-3,-7.763605E-2,-3.319938E-1,2.806637E-1,1.795295E-3,-8.949189E-3,-5.195474E-1,-1.8612137E-1,-1.3987516E-4,-1.6001105E-2,-7.978266E-3,5.450663E-3,1.146059E-2,-1.0650814E-2,-2.1176642E-2,-8.391662E-3,-3.6097334E-3,6.374891E-3,-7.7443935E-2,-4.1714245E-3,1.0643514E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,19,-1,21,23,25,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,33,-1,-1],"loss_changes":[7.5769463E0,2.764654E0,5.5704784E-1,1.0053525E0,3.1303632E-1,0E0,8.5967064E-2,1.9630885E-1,5.6191164E-1,0E0,1.7695892E-1,0E0,0E0,1.0438442E-1,0E0,3.0263305E-1,5.020356E-2,1.8849254E-2,0E0,0E0,2.2569895E-2,3.660202E-2,3.1421196E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.858496E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,13,13,15,15,16,16,17,17,20,20,21,21,22,22,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,20,-1,22,24,26,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,34,-1,-1],"split_conditions":[6.0880933E3,6.402099E-1,1.3127055E4,1.6755434E1,8.873786E0,1.0528478E-2,1.890618E6,1.8839418E3,7.752181E7,-1.806817E-3,4.5E1,2.9008435E-2,1.39654195E-2,1.16E2,-5.6234505E-3,2.8866995E1,3.0595828E3,1.601247E1,1.795295E-3,-8.949189E-3,5.432393E-1,1.3101191E1,3.9709E5,-1.6001105E-2,-7.978266E-3,5.450663E-3,1.146059E-2,-1.0650814E-2,-2.1176642E-2,-8.391662E-3,-3.6097334E-3,6.374891E-3,4.6961885E3,-4.1714245E-3,1.0643514E-3],"split_indices":[52,42,4,58,56,0,29,52,45,0,3,0,0,0,0,56,4,56,0,0,27,56,29,0,0,0,0,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,6.7E1,2.3E1,8E0,1.1E1,2.1E1,4.6E1,5E0,1.8E1,7E0,4E0,1.7E1,4E0,3.5E1,1.1E1,1.3E1,5E0,5E0,1.2E1,1.4E1,2.1E1,4E0,7E0,4E0,9E0,4E0,8E0,8E0,6E0,6E0,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.197816E-2,-7.497011E-2,5.014067E-1,-2.3085347E-1,1.19796686E-1,5.8881146E-1,5.0151106E-3,-3.131732E-1,-2.3126926E-2,3.297368E-1,-9.704131E-3,1.1902153E-2,2.6172534E-2,-2.1767734E-1,-1.5210481E-2,-5.2955006E-3,4.8081987E-2,1.9302368E-2,4.8680156E-3,-1.1230028E-1,1.03101134E-1,-4.1702559E-4,-9.9552395E-3,3.2475363E-3,-1.2290914E-4,1.4748385E-3,-7.003339E-3,2.650139E-4,7.817953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.034423E0,2.3047233E0,5.2020025E-1,7.109568E-1,9.275719E-1,2.9015398E-1,0E0,2.1808338E-1,1.2142535E-1,4.4709122E-1,2.6619276E-1,0E0,0E0,2.1226358E-1,0E0,0E0,1.9707847E-2,0E0,0E0,1.580405E-1,1.17909305E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3536367E0,2.561146E3,2.326E4,2.6326587E-2,2.5927516E11,2.5172096E-1,5.0151106E-3,8.4395015E-1,1.5938355E8,8.012376E0,9.250217E5,1.1902153E-2,2.6172534E-2,8.1837E4,-1.5210481E-2,-5.2955006E-3,4.9457936E7,1.9302368E-2,4.8680156E-3,1.5457461E5,4.1822475E-1,-4.1702559E-4,-9.9552395E-3,3.2475363E-3,-1.2290914E-4,1.4748385E-3,-7.003339E-3,2.650139E-4,7.817953E-3],"split_indices":[39,52,9,38,31,38,0,27,7,53,32,0,0,12,0,0,32,0,0,28,27,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.2E1,7.4E1,1.8E1,4.1E1,3.3E1,1.4E1,4E0,2.9E1,1.2E1,1.2E1,2.1E1,6E0,8E0,1.7E1,1.2E1,4E0,8E0,5E0,7E0,1.1E1,1E1,4E0,1.3E1,4E0,4E0,4E0,7E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.737566E-4,-2.4148624E-1,2.8224903E-1,-3.1347293E-1,1.0660252E-1,5.9629556E-2,3.9899176E-1,-1.6329855E-1,-4.773247E-1,1.1325116E-2,-3.0231455E-3,-2.9234872E-3,1.6615129E-1,4.9787334E-1,2.0917995E-1,-3.1328753E-1,-5.934927E-2,-1.926247E-2,-1.0421838E-2,1.2407742E-3,8.963621E-3,2.0304255E-2,7.3911734E-3,3.2640689E-3,9.287992E-3,-6.30179E-3,-1.5059091E-2,4.346179E-3,-5.487278E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.8135777E0,1.3865364E0,1.1717119E0,1.0293384E0,4.2043418E-1,2.6807314E-1,4.5892143E-1,3.7118775E-1,6.87027E-2,0E0,0E0,0E0,9.9698305E-2,2.4425411E-1,4.5271814E-2,7.123941E-2,2.8470373E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6225724E-1,2.7938022E3,1.6492815E3,8.297822E-1,1.0803E4,2.0978074E8,1.4046429E1,1.0930736E1,1.0034782E-3,1.1325116E-2,-3.0231455E-3,-2.9234872E-3,1.8589372E5,2.57281E6,9.257902E-2,6.948189E7,2.083117E1,-1.926247E-2,-1.0421838E-2,1.2407742E-3,8.963621E-3,2.0304255E-2,7.3911734E-3,3.2640689E-3,9.287992E-3,-6.30179E-3,-1.5059091E-2,4.346179E-3,-5.487278E-3],"split_indices":[42,52,52,27,9,7,54,58,39,0,0,0,33,29,38,7,58,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,5.3E1,4.5E1,4.4E1,9E0,1.6E1,2.9E1,2.4E1,2E1,4E0,5E0,7E0,9E0,1.8E1,1.1E1,9E0,1.5E1,1.4E1,6E0,4E0,5E0,1.4E1,4E0,4E0,7E0,5E0,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.6093224E-2,-2.1192521E-1,2.9062074E-1,-4.335574E-1,-5.6446034E-2,1.5187746E-1,5.564244E-1,-4.9036043E-3,-4.83801E-1,-1.8762808E-1,1.0892272E-1,3.4369522E-1,1.8826129E-2,1.2425799E-2,2.488158E-2,-1.0767954E-2,-2.0046884E-2,-9.543987E-3,-1.3038243E-1,2.0427169E-1,-3.6077367E-3,3.526767E-3,1.9222017E-2,-9.245615E-2,1.2801662E-1,-2.9906249E-3,-5.9965374E-3,2.5470222E-3,9.470343E-3,1.3927397E-3,-7.4010394E-3,-6.437534E-4,8.962964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0079656E0,2.0775373E0,1.3537521E0,3.0827093E-1,8.230931E-1,6.8261296E-1,1.2911081E-1,0E0,1.1285877E-1,5.7770073E-2,3.5894716E-1,4.5870197E-1,2.1840233E-1,0E0,0E0,0E0,0E0,0E0,1.001364E-2,7.4634016E-2,0E0,0E0,0E0,1.4062831E-1,1.6226287E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2789746E3,2.667836E6,1.6957201E0,2.3452184E5,2.3514317E8,6.19999E9,2.463E4,-4.9036043E-3,2.9488824E2,1.7301265E6,2.63382E5,7.21E2,8.184232E5,1.2425799E-2,2.488158E-2,-1.0767954E-2,-2.0046884E-2,-9.543987E-3,9E0,9.489487E2,-3.6077367E-3,3.526767E-3,1.9222017E-2,5.6075206E0,4.4432E4,-2.9906249E-3,-5.9965374E-3,2.5470222E-3,9.470343E-3,1.3927397E-3,-7.4010394E-3,-6.437534E-4,8.962964E-3],"split_indices":[52,32,39,28,7,5,2,0,52,48,29,0,32,0,0,0,0,0,17,52,0,0,0,53,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6E1,3.8E1,2.4E1,3.6E1,2.6E1,1.2E1,4E0,2E1,2E1,1.6E1,1E1,1.6E1,6E0,6E0,7E0,1.3E1,7E0,1.3E1,1.1E1,5E0,5E0,5E0,8E0,8E0,7E0,6E0,4E0,7E0,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[5.2020777E-2,-1.21101394E-1,3.6666822E-1,-2.0498253E-1,1.4658998E-1,2.7944025E-2,2.8636226E-1,-1.4501555E-1,-3.592246E-1,2.809931E-1,1.0968173E-2,1.7762247E-2,2.072143E-1,-3.1797555E-1,-7.3385365E-2,-1.6689165E-2,-4.607437E-3,5.7943943E-3,1.24660535E-2,-2.0134482E-3,3.2081057E-3,1.1230931E-1,1.1426109E-2,-1.6463945E-2,-6.9298414E-3,-1.2628634E-1,6.995995E-2,4.787221E-2,6.1650337E-3,7.958925E-4,-1.5867427E-1,4.6231477E-3,-7.6772274E-5,-1.0806797E-3,4.1916743E-3,-6.5995953E-3,-1.969959E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[6.145319E0,1.6692088E0,1.0868273E0,4.6587515E-1,3.2502186E-1,0E0,4.6767354E-1,4.9700975E-1,2.9897773E-1,1.4311731E-2,5.693878E-2,0E0,2.4445832E-1,1.2124944E-1,2.4391133E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.208996E-2,0E0,0E0,0E0,1.1399335E-1,3.796269E-2,5.125404E-2,0E0,0E0,3.774631E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,21,21,25,25,26,26,27,27,30,30],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,2.8938947E8,1.6451234E-2,5.501383E9,5.46E2,2.7944025E-2,2.8129198E7,2.253E3,3.5904072E3,2.2659668E-1,3.9219662E5,1.7762247E-2,8.640353E3,4.0748124E7,2.3514317E8,-1.6689165E-2,-4.607437E-3,5.7943943E-3,1.24660535E-2,-2.0134482E-3,3.2081057E-3,1E0,1.1426109E-2,-1.6463945E-2,-6.9298414E-3,5.2287582E-2,7.0590184E7,8.18961E3,6.1650337E-3,7.958925E-4,3.5983E4,4.6231477E-3,-7.6772274E-5,-1.0806797E-3,4.1916743E-3,-6.5995953E-3,-1.969959E-3],"split_indices":[2,12,57,5,0,0,45,2,4,38,28,0,52,5,7,0,0,0,0,0,0,109,0,0,0,57,12,4,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,7.2E1,3.9E1,5.5E1,1.7E1,5E0,3.4E1,4.1E1,1.4E1,8E0,9E0,8E0,2.6E1,1.1E1,3E1,9E0,5E0,4E0,4E0,5E0,4E0,1.5E1,1.1E1,4E0,7E0,2.2E1,8E0,8E0,7E0,4E0,1.8E1,4E0,4E0,4E0,4E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.251468E-2,-1.2000842E-1,3.6634618E-1,-2.7589586E-1,8.500792E-2,5.2591234E-1,8.943176E-2,-3.3621106E-1,-8.631847E-2,-3.040177E-2,2.2731982E-1,7.803743E-3,6.5605694E-1,1.6560884E-1,-2.917438E-3,-3.7966388E-1,-3.578619E-3,-6.0078017E-3,1.5549397E-3,6.212109E-3,-1.00540236E-1,1.0412533E-2,2.1543296E-3,1.3267694E-2,2.6267806E-2,2.5978214E-3,7.8001083E-3,-2.2031178E-1,-1.7659152E-2,-4.9550044E-3,-2.5344206E-4,-9.553114E-3,-4.7625317E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[5.517495E0,2.2606163E0,1.4994822E0,4.2835498E-1,5.1606023E-1,7.4509144E-1,1.9494057E-1,2.7143145E-1,1.2115493E-1,2.7291396E-1,1.2617743E-1,0E0,4.2197704E-2,3.169495E-2,0E0,3.2834005E-1,0E0,0E0,0E0,0E0,4.709524E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2081087E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,20,20,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,5.6614784E7,1.4046429E1,3.0595828E3,9.2954946E-1,1.7059127E5,1.782E3,3.2936362E6,7.056912E7,2.7186182E-1,4.4E1,7.803743E-3,3E0,4.1E1,-2.917438E-3,2.9180167E2,-3.578619E-3,-6.0078017E-3,1.5549397E-3,6.212109E-3,1.2836887E1,1.0412533E-2,2.1543296E-3,1.3267694E-2,2.6267806E-2,2.5978214E-3,7.8001083E-3,9.7E1,-1.7659152E-2,-4.9550044E-3,-2.5344206E-4,-9.553114E-3,-4.7625317E-3],"split_indices":[2,12,54,4,39,28,0,50,7,27,3,0,8,3,0,52,0,0,0,0,54,0,0,0,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,3.9E1,3E1,2.1E1,1.3E1,2.9E1,1E1,1.7E1,1.3E1,7E0,1.4E1,9E0,4E0,2.4E1,5E0,6E0,4E0,4E0,1.3E1,9E0,4E0,4E0,1E1,4E0,5E0,1.1E1,1.3E1,9E0,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[9.404197E-3,-8.191902E-2,3.9164418E-1,-1.7726035E-1,1.2871456E-1,2.3037602E-1,2.0978406E-2,-3.552201E-1,-1.11507826E-1,-4.5729848E-3,1.7655551E-1,9.793322E-3,3.9969105E-3,-1.6186135E-2,-6.1901193E-3,-1.7534252E-1,-2.716451E-2,2.3359561E-1,5.5533648E-2,-2.0777021E-1,8.693186E-5,2.9419977E-3,-8.7311916E-2,2.4158028E-3,1.1802541E-2,-1.4702341E-4,3.7558996E-3,-4.640943E-3,-2.8998297E-1,-6.154108E-3,-2.2178702E-2,-1.2990197E-2,-5.3502736E-3,-2.6759414E-3,1.2346494E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,-1,-1,19,21,23,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[3.8020809E0,1.7900395E0,5.076542E-1,6.741699E-1,3.644562E-1,3.3572435E-2,0E0,1.968987E-1,2.4290222E-1,0E0,1.5270191E-1,0E0,0E0,0E0,0E0,1.5036523E-1,1.4435473E-1,2.2699332E-1,2.6441261E-2,1.06012404E-1,0E0,0E0,7.203901E-2,0E0,0E0,0E0,0E0,0E0,4.568374E-2,0E0,2.8889056E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,15,15,16,16,17,17,18,18,19,19,22,22,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,-1,-1,20,22,24,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[6.1596104E3,6.402099E-1,1.2768678E4,1.983671E4,3.568E3,7.118011E-1,2.0978406E-2,2.8374892E-2,7.388E3,-4.5729848E-3,1.4163358E7,9.793322E-3,3.9969105E-3,-1.6186135E-2,-6.1901193E-3,2.3514317E8,4.171707E9,1.09484E3,4.0389168E-1,6.681133E-1,8.693186E-5,2.9419977E-3,2.074635E-1,2.4158028E-3,1.1802541E-2,-1.4702341E-4,3.7558996E-3,-4.640943E-3,1.2163482E8,-6.154108E-3,3.3487454E11,-1.2990197E-2,-5.3502736E-3,-2.6759414E-3,1.2346494E-3],"split_indices":[52,42,52,48,2,27,0,57,2,0,45,0,0,0,0,7,5,52,27,27,0,0,39,0,0,0,0,0,7,0,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,8.7E1,2E1,6E1,2.7E1,1.2E1,8E0,1.5E1,4.5E1,4E0,2.3E1,8E0,4E0,9E0,6E0,2.5E1,2E1,1.5E1,8E0,2.1E1,4E0,7E0,1.3E1,6E0,9E0,4E0,4E0,1.2E1,9E0,5E0,8E0,5E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.6260551E-2,-1.8317385E-1,2.6741064E-1,-2.540629E-1,9.2660606E-2,1.4874665E-1,4.6568242E-1,-3.7536886E-1,-8.0223374E-2,-3.7698024E-3,9.9825505E-3,-2.4102514E-3,2.0709147E-1,9.956253E-3,2.3324372E-2,-1.4566985E-3,-4.2952442E-1,4.1348813E-3,-1.327848E-1,2.665169E-1,9.6425675E-2,-6.5715313E-3,-4.8507664E-1,-6.243079E-3,-7.78916E-2,1.7643574E-1,1.2484552E-2,4.8931944E-3,8.0829137E-4,-2.0885479E-2,-1.2388691E-2,-9.059528E-4,-4.155864E-3,2.2343034E-3,1.0058555E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.655051E0,1.1967599E0,1.1541727E0,9.8209953E-1,4.998774E-1,4.4133198E-1,4.7265387E-1,4.9067974E-1,2.301748E-1,0E0,0E0,0E0,1.5186274E-1,0E0,0E0,0E0,2.416892E-1,0E0,2.3988068E-2,6.040609E-2,2.8816007E-2,0E0,5.3410053E-2,0E0,1.4797598E-2,9.979406E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,16,16,18,18,19,19,20,20,22,22,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.770862E8,2.561146E3,1.7618678E0,2.667836E6,2.9844797E-1,3.5849205E-1,9.563013E8,4.029937E-1,3.42E2,-3.7698024E-3,9.9825505E-3,-2.4102514E-3,1.6135693E1,9.956253E-3,2.3324372E-2,-1.4566985E-3,2.63E2,4.1348813E-3,9.320373E4,1.7302E4,1E0,-6.5715313E-3,1.4613986E3,-6.243079E-3,1.3560078E5,1.1803382E4,1.2484552E-2,4.8931944E-3,8.0829137E-4,-2.0885479E-2,-1.2388691E-2,-9.059528E-4,-4.155864E-3,2.2343034E-3,1.0058555E-2],"split_indices":[7,52,39,32,27,39,7,27,10,0,0,0,54,0,0,0,11,0,33,9,89,0,52,0,33,4,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.9E1,5.1E1,4.7E1,1.2E1,3.3E1,1.8E1,2.7E1,2E1,6E0,6E0,7E0,2.6E1,1E1,8E0,4E0,2.3E1,4E0,1.6E1,1.6E1,1E1,5E0,1.8E1,8E0,8E0,9E0,7E0,6E0,4E0,9E0,9E0,4E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0040615E-2,-1.6906129E-1,2.188069E-1,-2.1587087E-1,6.489098E-3,2.9824182E-2,3.124698E-1,-1.4519845E-1,-3.1849194E-1,8.985236E-3,-6.618626E-2,2.0420541E-1,3.985903E-1,-1.4569583E-2,-9.884405E-2,-3.6497048E-1,-3.1579603E-3,2.670431E-3,-1.2964426E-1,2.4638239E-3,2.6649892E-1,1.101048E-2,1.8611364E-2,-3.096264E-2,-6.4645973E-3,-2.5481278E-1,-1.6131384E-2,-6.710294E-3,-1.3087403E-3,1.1467256E-2,5.8643576E-3,-9.1229245E-2,5.1979795E-3,-1.1566119E-2,-4.5196684E-3,-4.2870683E-3,-6.480817E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,31,-1,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[4.3742824E0,1.0674121E0,9.46517E-1,3.5989428E-1,0E0,4.1606957E-1,2.5426173E-1,3.7927127E-1,2.0430112E-1,0E0,1.3410327E-1,1.3426387E-1,5.074501E-2,0E0,1.6325581E-1,4.8431396E-2,0E0,0E0,4.5769468E-2,0E0,1.2253225E-2,0E0,0E0,2.0290853E-1,0E0,4.486531E-2,0E0,0E0,0E0,0E0,0E0,2.609723E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,18,18,20,20,23,23,25,25,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,32,-1,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,3.7249353E3,6.2371206E-1,8.4947616E-1,6.489098E-3,3.5925132E-1,2.6167098E5,7.110474E7,2.4502628E6,8.985236E-3,9.069479E-1,2.0783027E5,1.5605432E9,-1.4569583E-2,3.925844E7,3.91028E5,-3.1579603E-3,2.670431E-3,1.939024E7,2.4638239E-3,2.548476E5,1.101048E-2,1.8611364E-2,8E0,-6.4645973E-3,7.434519E5,-1.6131384E-2,-6.710294E-3,-1.3087403E-3,1.1467256E-2,5.8643576E-3,1.4198953E1,5.1979795E-3,-1.1566119E-2,-4.5196684E-3,-4.2870683E-3,-6.480817E-4],"split_indices":[7,52,39,27,0,27,28,5,32,0,57,33,7,0,45,12,0,0,48,0,33,0,0,8,0,28,0,0,0,0,0,56,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,6.2E1,5.3E1,5.5E1,7E0,1.8E1,3.5E1,3.4E1,2.1E1,5E0,1.3E1,1.7E1,1.8E1,4E0,3E1,1.7E1,4E0,4E0,9E0,6E0,1.1E1,1.2E1,6E0,1.7E1,1.3E1,9E0,8E0,5E0,4E0,6E0,5E0,1.3E1,4E0,5E0,4E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-7.882657E-3,-1.962244E-1,2.4056515E-1,-9.932849E-2,-4.060304E-1,-6.703214E-2,3.3300647E-1,-1.7431891E-1,8.36856E-3,-6.773431E-3,-1.7170759E-2,-4.2780684E-3,6.651603E-4,4.1061303E-1,5.650482E-3,-2.618067E-1,-4.604724E-2,6.484681E-3,4.815757E-1,-3.1523302E-1,-3.0545453E-3,-4.3786177E-3,4.645628E-3,1.9725902E-2,7.826164E-3,-7.88878E-3,-1.6787257E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,-1,17,-1,19,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7733073E0,1.1387713E0,1.2734487E0,1.0584866E0,1.8430138E-1,5.0560266E-2,4.014423E-1,3.6949956E-1,0E0,0E0,0E0,0E0,0E0,2.9052138E-1,0E0,1.626724E-1,2.1278434E-1,0E0,1.7327881E-1,1.1566639E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,13,13,15,15,16,16,18,18,19,19],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,-1,18,-1,20,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,2.5927516E11,5.12544E-1,1E1,6.948189E7,5.3534385E3,1.4046429E1,1.386854E8,8.36856E-3,-6.773431E-3,-1.7170759E-2,-4.2780684E-3,6.651603E-4,5.403511E8,5.650482E-3,1.977643E6,3.167669E5,6.484681E-3,1.746E3,2E0,-3.0545453E-3,-4.3786177E-3,4.645628E-3,1.9725902E-2,7.826164E-3,-7.88878E-3,-1.6787257E-2],"split_indices":[12,31,39,8,7,4,54,7,0,0,0,0,0,7,0,48,28,0,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.7E1,4.3E1,4E1,1.7E1,1E1,3.3E1,3.3E1,7E0,5E0,1.2E1,6E0,4E0,2.2E1,1.1E1,1.9E1,1.4E1,6E0,1.6E1,1.4E1,5E0,1E1,4E0,1.2E1,4E0,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.234922E-2,-1.1959446E-1,2.692677E-1,-1.5458244E-1,1.2714416E-2,5.4225646E-4,3.244151E-1,-7.8147516E-2,-2.969515E-1,4.1584706E-1,5.732994E-3,-1.2855452E-1,1.2656279E-1,-1.4236198E-1,-1.4189349E-2,4.6046913E-1,6.855493E-3,-1.9554819E-1,-4.6117224E-2,8.02689E-3,1.9784305E-4,-7.7642547E-3,-9.622835E-4,8.53168E-3,1.8610887E-2,-8.071357E-3,-2.6134562E-3,1.9264045E-3,-8.0829665E-2,-3.943927E-3,-4.8823442E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[3.747346E0,1.1768546E0,5.4195094E-1,6.7324007E-1,0E0,0E0,4.1853905E-1,4.5998007E-1,2.707882E-1,1.1421871E-1,0E0,1.8427902E-1,1.0314682E-1,7.8383625E-2,0E0,7.552695E-2,0E0,4.937589E-2,6.3014776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3382068E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,28,28],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[5.1707494E-1,6.082581E3,1.6511278E1,8.106617E-1,1.2714416E-2,5.4225646E-4,1.343964E1,4.899102E5,3.3012803E9,4.698217E7,5.732994E-3,2.834008E-1,4.828443E-1,2.45E2,-1.4189349E-2,5.403511E8,6.855493E-3,4.3E1,9.069479E-1,8.02689E-3,1.9784305E-4,-7.7642547E-3,-9.622835E-4,8.53168E-3,1.8610887E-2,-8.071357E-3,-2.6134562E-3,1.9264045E-3,5.691229E6,-3.943927E-3,-4.8823442E-4],"split_indices":[39,52,56,27,0,0,54,28,5,45,0,57,27,11,0,7,0,3,57,0,0,0,0,0,0,0,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.7E1,3.8E1,6.3E1,4E0,7E0,3.1E1,4.2E1,2.1E1,1.9E1,1.2E1,3.4E1,8E0,9E0,1.2E1,1.5E1,4E0,1.8E1,1.6E1,4E0,4E0,5E0,4E0,4E0,1.1E1,1.4E1,4E0,4E0,1.2E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.3009777E-2,-1.8076913E-1,1.8460925E-1,-2.2224353E-1,4.2094267E-3,-4.635656E-3,2.9711828E-1,-1.4615564E-1,-3.4048074E-1,-9.690105E-2,5.266527E-3,2.193756E-1,4.255929E-1,-1.5337352E-2,-9.483938E-2,-3.8632032E-1,-4.5380723E-3,-2.348625E-2,-7.5696246E-3,2.442181E-1,4.058913E-3,1.9231215E-2,6.8796948E-3,-3.5304856E-2,-1.6997646E-1,-7.313136E-3,-1.554929E-2,-2.6376923E-3,1.1114292E-3,1.0115554E-2,5.5168583E-3,4.8919874E-3,-9.378512E-2,-7.996792E-3,-3.9744787E-3,-8.3479716E-4,-4.3228953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,-1,23,25,-1,27,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[3.6801403E0,7.6589215E-1,1.0872574E0,4.270475E-1,0E0,2.9187408E-1,2.2624993E-1,4.4591415E-1,1.4835119E-1,1.0269912E-1,0E0,3.0995369E-2,1.7534697E-1,0E0,1.2819651E-1,4.3216944E-2,0E0,2.6463097E-2,0E0,1.3077617E-2,0E0,0E0,0E0,1.9250572E-1,1.5729427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2905715E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,17,17,19,19,23,23,24,24,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,-1,24,26,-1,28,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[1.5363757E8,2.711196E3,9.745363E-1,8.297822E-1,4.2094267E-3,3.9219662E5,1.5435694E9,9.2433E4,3.1346153E1,2.2492808E5,5.266527E-3,1.5723504E1,4.4E1,-1.5337352E-2,3.925844E7,2.08E2,-4.5380723E-3,5.3990013E1,-7.5696246E-3,2.326E4,4.058913E-3,1.9231215E-2,6.8796948E-3,2.083117E1,1.7E2,-7.313136E-3,-1.554929E-2,-2.6376923E-3,1.1114292E-3,1.0115554E-2,5.5168583E-3,4.8919874E-3,3.2785356E7,-7.996792E-3,-3.9744787E-3,-8.3479716E-4,-4.3228953E-3],"split_indices":[12,52,42,27,0,28,7,1,58,33,0,54,3,0,45,11,0,56,0,9,0,0,0,58,10,0,0,0,0,0,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.9E1,5E1,5.2E1,7E0,1.9E1,3.1E1,3.3E1,1.9E1,1.2E1,7E0,2.1E1,1E1,4E0,2.9E1,1.5E1,4E0,8E0,4E0,1.6E1,5E0,6E0,4E0,1.7E1,1.2E1,4E0,1.1E1,4E0,4E0,1E1,6E0,4E0,1.3E1,5E0,7E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.9804446E-2,-1.6584219E-1,2.2534335E-1,-1.9925624E-1,8.962442E-3,1.1894395E-1,4.0559512E-1,-6.371361E-2,-3.373642E-1,-4.79808E-2,1.6835034E-1,2.5866732E-1,2.008889E-2,-1.2507846E-2,-6.890907E-3,-3.8854584E-1,-1.861423E-1,4.3842144E-4,-3.556473E-3,2.6246664E-1,6.928512E-2,1.119961E-2,5.6111203E-3,7.1935244E-2,-5.912861E-3,-6.931863E-3,-1.5930815E-2,-3.332643E-3,-8.763888E-3,1.2654458E-2,3.128588E-3,3.9051168E-3,-2.550271E-4,-1.4070388E-3,1.3063706E-1,8.102995E-3,-2.0714968E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1],"loss_changes":[4.4254956E0,8.93484E-1,1.0065844E0,1.0455177E0,0E0,3.049749E-1,2.9931355E-1,4.7958618E-1,1.4438891E-1,2.8538108E-2,2.428773E-1,1.5650094E-2,0E0,0E0,3.3331746E-1,1.2249017E-1,2.539453E-2,0E0,0E0,1.8632936E-1,4.5291804E-2,0E0,0E0,1.2228049E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.47826E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,19,19,20,20,23,23,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1],"split_conditions":[3.4707712E8,3.7249353E3,1.7618678E0,6.4987373E-1,8.962442E-3,3.5849205E-1,1.2750684E9,1.497E3,3.1371258E1,1.0824E4,1.2320755E1,2.3559767E5,2.008889E-2,-1.2507846E-2,3.925844E7,4.5E1,3.25E2,4.3842144E-4,-3.556473E-3,7.118011E-1,2.5E1,1.119961E-2,5.6111203E-3,3.22115E5,-5.912861E-3,-6.931863E-3,-1.5930815E-2,-3.332643E-3,-8.763888E-3,1.2654458E-2,3.128588E-3,3.9051168E-3,-2.550271E-4,-1.4070388E-3,1.0428423E9,8.102995E-3,-2.0714968E-5],"split_indices":[7,52,39,27,0,39,7,2,58,10,54,33,0,0,45,10,0,0,0,27,8,0,0,1,0,0,0,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6E1,5.4E1,5.6E1,4E0,3.5E1,1.9E1,2.9E1,2.7E1,8E0,2.7E1,1.1E1,8E0,4E0,2.5E1,1.9E1,8E0,4E0,4E0,1.3E1,1.4E1,6E0,5E0,1.7E1,8E0,5E0,1.4E1,4E0,4E0,8E0,5E0,9E0,5E0,6E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[8.9094596E-4,-8.418677E-2,2.9088813E-1,-1.6697854E-1,3.8710527E-2,4.689037E-1,1.03466496E-1,-2.052833E-1,6.5171765E-3,-4.9591303E-2,7.356622E-3,9.249497E-3,2.178669E-2,-5.2451534E-4,5.9309984E-3,-1.4593045E-1,-3.2189476E-1,8.833714E-4,-1.0717486E-1,-1.0503395E-2,-8.657066E-2,-6.019513E-3,-1.3442244E-2,-5.490938E-3,-5.008993E-4,-1.1700313E-1,1.4549642E-3,-6.5351226E-3,-2.1594516E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,19,21,-1,23,-1,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[2.3930295E0,7.6958007E-1,6.854913E-1,6.3603854E-1,4.619252E-1,1.602242E-1,8.849475E-2,2.367959E-1,0E0,9.2179015E-2,0E0,0E0,0E0,0E0,0E0,2.2441888E-1,3.9655447E-2,0E0,4.817827E-2,0E0,8.965723E-2,0E0,0E0,0E0,0E0,5.506572E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,15,15,16,16,18,18,20,20,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,20,22,-1,24,-1,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[4.492037E3,2.3514317E8,2.5927516E11,1E0,1.1329926E0,7.845511E-1,1.5438823E7,5.5309926E9,6.5171765E-3,9.513889E-1,7.356622E-3,9.249497E-3,2.178669E-2,-5.2451534E-4,5.9309984E-3,9.2433E4,5.145631E-1,8.833714E-4,1.447E3,-1.0503395E-2,6.939E3,-6.019513E-3,-1.3442244E-2,-5.490938E-3,-5.008993E-4,9.320373E4,1.4549642E-3,-6.5351226E-3,-2.1594516E-3],"split_indices":[52,7,31,94,39,27,1,5,0,57,0,0,0,0,0,1,58,0,0,0,2,0,0,0,0,33,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,7.4E1,2.1E1,4.4E1,3E1,1E1,1.1E1,4E1,4E0,2E1,1E1,5E0,5E0,4E0,7E0,2.8E1,1.2E1,9E0,1.1E1,7E0,2.1E1,4E0,8E0,7E0,4E0,1.7E1,4E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.5014396E-2,-1.1989209E-1,2.198597E-1,-1.8675263E-1,4.2297527E-2,-2.6039736E-4,2.7899745E-1,-3.1083202E-1,-1.0462264E-1,2.1596667E-1,-1.07906945E-1,3.4681892E-1,1.4881904E-1,-4.906404E-3,-3.5700446E-1,-1.3887258E-1,1.7085096E-3,1.4346364E-2,2.0050958E-3,-1.8250199E-1,1.7309767E-3,1.8190123E-2,9.37768E-3,1.8626243E-3,7.112547E-3,-1.4488746E-2,-6.4715906E-3,-9.498444E-2,-7.522882E-3,-9.739513E-3,-2.1204518E-3,-5.0669527E-3,-4.923456E-2,-2.7675268E-3,-3.599623E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[2.7113624E0,8.320497E-1,4.5858276E-1,5.177746E-1,6.2327963E-1,0E0,1.8676686E-1,1.21100664E-1,1.8351245E-1,2.9806107E-1,1.598858E-1,1.0155201E-1,4.1373044E-2,0E0,5.4087877E-2,6.56144E-2,0E0,0E0,0E0,8.136505E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2521486E-2,0E0,0E0,0E0,0E0,1.4473958E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,19,19,27,27,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.0230335E0,2.4722983E3,1.6511278E1,3.572374E6,5.4875424E7,-2.6039736E-4,2.8129198E7,3.842849E5,3.4707712E8,1.0007143E1,9.250217E5,8.892E3,3.8737573E11,-4.906404E-3,2.1883162E1,6.784095E7,1.7085096E-3,1.4346364E-2,2.0050958E-3,2.1227852E5,1.7309767E-3,1.8190123E-2,9.37768E-3,1.8626243E-3,7.112547E-3,-1.4488746E-2,-6.4715906E-3,1.3101191E1,-7.522882E-3,-9.739513E-3,-2.1204518E-3,-5.0669527E-3,3.1851864E7,-2.7675268E-3,-3.599623E-5],"split_indices":[39,52,56,12,45,0,45,28,7,53,32,9,31,0,58,45,0,0,0,28,0,0,0,0,0,0,0,56,0,0,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,7.5E1,3.3E1,5.3E1,2.2E1,7E0,2.6E1,2E1,3.3E1,1E1,1.2E1,1.6E1,1E1,5E0,1.5E1,2.7E1,6E0,4E0,6E0,8E0,4E0,4E0,1.2E1,4E0,6E0,1.1E1,4E0,1.8E1,9E0,4E0,4E0,8E0,1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[8.900368E-3,-1.15445875E-1,2.4601895E-1,-2.604195E-1,3.313542E-2,2.0173583E-2,1.5582131E-1,-1.14240386E-1,-3.0438584E-1,-9.80291E-2,1.2803899E-1,2.1317573E-1,-5.024573E-3,-6.8414137E-3,-5.8254326E-4,-3.6944482E-3,-3.521724E-1,-6.209623E-3,-7.600827E-4,7.6640663E-3,-5.1744486E-4,2.620522E-1,3.2050724E-3,-4.1836056E-1,-7.5407824E-3,1.2138781E-2,4.283731E-3,-6.7753308E-3,-1.8421523E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,-1,-1,-1,-1,-1],"loss_changes":[2.8294582E0,1.3656723E0,8.438661E-1,1.7088962E-1,4.1088697E-1,0E0,4.8648798E-1,6.3389756E-2,1.9569445E-1,7.609184E-2,2.3092851E-1,1.1658132E-1,0E0,0E0,0E0,0E0,9.2832804E-2,0E0,0E0,0E0,0E0,1.3029003E-1,0E0,1.5862107E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,16,16,21,21,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,-1,-1,-1,-1,-1],"split_conditions":[3.7249353E3,3.2316522E7,6.4417176E-2,2.4232728E5,3.8757455E-1,2.0173583E-2,2.2950048E10,8.6E1,1.68E2,1.3045E4,9.1197605E0,1.5625807E1,-5.024573E-3,-6.8414137E-3,-5.8254326E-4,-3.6944482E-3,1.365445E1,-6.209623E-3,-7.600827E-4,7.6640663E-3,-5.1744486E-4,6.19999E9,3.2050724E-3,2E0,-7.5407824E-3,1.2138781E-2,4.283731E-3,-6.7753308E-3,-1.8421523E-2],"split_indices":[52,12,57,28,41,0,5,10,0,9,53,54,0,0,0,0,54,0,0,0,0,5,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6.2E1,3.2E1,3.1E1,3.1E1,6E0,2.6E1,8E0,2.3E1,1.3E1,1.8E1,2.2E1,4E0,4E0,4E0,5E0,1.8E1,6E0,7E0,1.1E1,7E0,1.5E1,7E0,1.1E1,7E0,9E0,6E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.832938E-3,-1.3853852E-1,1.2620504E-1,-1.905101E-1,9.6116275E-2,-2.5492298E-4,2.4364126E-1,-3.3862132E-1,-1.3584775E-1,9.738527E-3,-1.5030988E-3,-1.2742741E-1,7.0410624E-2,2.8838646E-1,4.847233E-3,-7.497833E-3,-1.7344054E-2,-5.534093E-2,-2.5840554E-1,-1.5488224E-3,-6.376834E-3,7.605512E-3,-1.7269943E-2,6.6758594E-3,1.47138005E-2,-9.5443234E-2,2.678462E-3,-5.5975732E-3,-1.0780397E-2,-2.1615345E-3,1.9620631E-3,-5.338603E-4,-4.5246924E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9434575E0,7.134478E-1,8.02015E-1,3.3451617E-1,2.743179E-1,2.516135E-1,9.831953E-2,1.2301552E-1,3.409021E-1,0E0,0E0,3.401135E-2,2.2498912E-1,1.5891886E-1,0E0,0E0,0E0,1.2655362E-1,1.1868715E-2,0E0,0E0,0E0,3.986518E-2,0E0,0E0,4.15328E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,17,17,18,18,22,22,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,2.7938022E3,1.0230335E0,1.73E2,9E0,1.937363E6,2.8129198E7,2.16268E5,7.332772E-1,9.738527E-3,-1.5030988E-3,8.44629E7,9E0,4.2864155E1,4.847233E-3,-7.497833E-3,-1.7344054E-2,1E1,8.07967E5,-1.5488224E-3,-6.376834E-3,7.605512E-3,5.759438E5,6.6758594E-3,1.47138005E-2,4.3786978E2,2.678462E-3,-5.5975732E-3,-1.0780397E-2,-2.1615345E-3,1.9620631E-3,-5.338603E-4,-4.5246924E-3],"split_indices":[7,52,39,0,17,32,45,12,27,0,0,45,8,56,0,0,0,8,1,0,0,0,28,0,0,52,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,4.6E1,1E1,2.6E1,2.7E1,1.1E1,3.5E1,4E0,6E0,9E0,1.7E1,1.8E1,9E0,7E0,4E0,2.2E1,1.3E1,4E0,5E0,6E0,1.1E1,1.1E1,7E0,1.7E1,5E0,5E0,8E0,7E0,4E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-7.356629E-3,-8.673777E-2,2.8287694E-1,-2.2338766E-1,2.0403562E-2,4.1299033E-1,3.459055E-2,-1.732699E-1,-1.6370945E-2,-5.2707877E-2,1.347688E-1,6.4738477E-3,1.8480975E-2,5.8937715E-3,-3.645891E-3,-1.9200978E-1,-1.2156821E-3,3.48537E-3,-8.6418964E-2,1.3644533E-3,1.8818837E-1,-1.4850397E-1,-9.223227E-3,-1.0390652E-1,1.5569644E-4,4.4298754E-3,9.2181945E-3,-2.1377632E-3,-1.683674E-1,-4.756301E-3,-4.4620893E-4,-8.739493E-3,-4.31646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,-1,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,31,-1,-1,-1,-1],"loss_changes":[2.4666777E0,1.2343894E0,7.2873807E-1,3.4805036E-1,4.0845093E-1,2.54884E-1,1.7347302E-1,7.724774E-2,0E0,1.5928172E-1,9.131506E-2,0E0,0E0,0E0,0E0,3.9366364E-2,0E0,0E0,4.011491E-2,0E0,2.1434158E-2,2.3729384E-2,0E0,5.188629E-2,0E0,0E0,0E0,0E0,2.5017768E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,15,15,18,18,20,20,21,21,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,-1,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,32,-1,-1,-1,-1],"split_conditions":[6.1596104E3,3.851071E7,2.692756E6,1E0,9.2954946E-1,1.1681136E4,1.063036E8,2.6666668E-1,-1.6370945E-2,1.283561E6,6E0,6.4738477E-3,1.8480975E-2,5.8937715E-3,-3.645891E-3,8.4395015E-1,-1.2156821E-3,3.48537E-3,1.1855E4,1.3644533E-3,5.506832E-1,1.728E3,-9.223227E-3,1E0,1.5569644E-4,4.4298754E-3,9.2181945E-3,-2.1377632E-3,5.127381E1,-4.756301E-3,-4.4620893E-4,-8.739493E-3,-4.31646E-3],"split_indices":[52,12,29,112,39,4,51,57,0,48,8,0,0,0,0,27,0,0,2,0,27,2,0,108,0,0,0,0,56,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,3.6E1,4.7E1,1.4E1,8E0,3.1E1,5E0,2.9E1,1.8E1,5E0,9E0,4E0,4E0,2.7E1,4E0,5E0,2.4E1,7E0,1.1E1,1.8E1,9E0,2E1,4E0,7E0,4E0,4E0,1.4E1,1.5E1,5E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.9744236E-2,-9.327785E-2,2.0905109E-1,-1.2495501E-1,1.01146875E-2,1.2056951E-1,3.5438117E-1,-4.6363864E-2,-2.357004E-1,2.1097495E-3,1.7033628E-1,1.5840996E-2,3.53248E-3,-1.7231452E-1,2.1365339E-2,-1.1647715E-2,-6.4637825E-2,1.5850387E-3,-1.4479358E-3,7.3449826E-3,2.335599E-3,-7.999368E-3,-3.1095452E-3,7.366086E-3,-5.5106208E-2,-3.941915E-3,-2.5859976E-4,-9.458616E-3,-3.8008306E-3,-1.6233418E-3,7.8364415E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[2.2253702E0,7.4477845E-1,4.844929E-1,4.764741E-1,0E0,1.5881532E-1,2.818097E-1,2.9371983E-1,3.2114422E-1,1.7639747E-2,5.329585E-2,0E0,0E0,3.185776E-2,3.325218E-1,0E0,2.2261191E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.7937537E-2,0E0,0E0,1.2041082E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,24,24,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[7.307441E-1,7.3153823E3,1.5605432E9,3.3012803E9,1.01146875E-2,1.5832817E5,2.9548872E9,1.1301075E1,4.0916511E3,2.4E1,7.639958E-1,1.5840996E-2,3.53248E-3,4.1100917E0,1.53E2,-1.1647715E-2,1.4E1,1.5850387E-3,-1.4479358E-3,7.3449826E-3,2.335599E-3,-7.999368E-3,-3.1095452E-3,7.366086E-3,3.6449524E7,-3.941915E-3,-2.5859976E-4,9.9987485E-2,-3.8008306E-3,-1.6233418E-3,7.8364415E-4],"split_indices":[39,4,7,5,0,33,7,56,4,3,27,0,0,56,10,0,8,0,0,0,0,0,0,0,45,0,0,39,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,5.9E1,4E1,5.5E1,4E0,2.6E1,1.4E1,3.3E1,2.2E1,8E0,1.8E1,1E1,4E0,1.1E1,2.2E1,1.4E1,8E0,4E0,4E0,1.3E1,5E0,6E0,5E0,6E0,1.6E1,4E0,4E0,9E0,7E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.354157E-3,-9.587389E-2,2.811731E-1,-2.1141864E-1,4.2217784E-2,2.0473529E-2,1.9797917E-1,-2.5688913E-1,-6.405464E-2,-5.279064E-2,1.8850118E-1,2.5317988E-1,2.01925E-3,-2.9090002E-1,-2.407845E-3,3.9753322E-3,-6.473553E-3,-2.2955444E-2,-3.6873005E-3,9.01752E-3,4.0192218E-3,2.1807535E-3,3.078492E-1,-3.9866236E-3,-3.1869465E-1,-2.6209368E-3,6.1207065E-5,1.2906996E-2,5.6645344E-3,-1.3646177E-2,-6.0544596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.973064E0,1.2056081E0,5.775304E-1,2.582742E-1,4.9638656E-1,0E0,1.7402846E-1,1.8112898E-1,2.4054828E-1,3.0042421E-2,3.8393497E-2,1.605668E-1,0E0,9.523201E-2,0E0,0E0,0E0,1.8478293E-2,0E0,0E0,0E0,0E0,4.134941E-2,0E0,1.3354897E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,22,22,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[3.944201E3,3.2936362E6,6.4417176E-2,7.33E3,9.2954946E-1,2.0473529E-2,1.0090909E1,3.0174E4,6.25379E5,1.5832817E5,7.6363635E-1,1.286E4,2.01925E-3,9.122283E0,-2.407845E-3,3.9753322E-3,-6.473553E-3,1E0,-3.6873005E-3,9.01752E-3,4.0192218E-3,2.1807535E-3,8.5E1,-3.9866236E-3,2.9282052E6,-2.6209368E-3,6.1207065E-5,1.2906996E-2,5.6645344E-3,-1.3646177E-2,-6.0544596E-3],"split_indices":[52,50,57,2,39,0,53,9,29,33,57,2,0,53,0,0,0,8,0,0,0,0,3,0,32,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.4E1,2.8E1,4E1,3.4E1,5E0,2.3E1,3E1,1E1,2.1E1,1.3E1,1.6E1,7E0,2.5E1,5E0,4E0,6E0,1.4E1,7E0,6E0,7E0,4E0,1.2E1,4E0,2.1E1,4E0,1E1,8E0,4E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.742977E-2,-1.3676295E-1,1.1548542E-1,-2.2288944E-1,-1.5563322E-2,1.9800499E-1,-6.0553983E-2,-1.2472117E-1,-3.194958E-1,-9.593062E-2,6.399417E-3,1.9342003E-2,1.1535384E-1,3.6078976E-3,-1.1354564E-1,-7.6137953E-3,7.265229E-4,-3.8447765E-1,-3.1281011E-3,-4.4159214E-3,-6.635637E-4,1.6669326E-2,1.7680356E-1,-1.704069E-1,8.2625286E-4,-1.0050992E-2,-1.7998286E-2,3.225355E-3,-1.4410706E-3,7.2806743E-3,2.4139055E-3,-1.8518103E-3,-1.0082227E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,-1,-1,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8081089E0,6.22841E-1,7.964315E-1,2.9208076E-1,4.199064E-1,9.7104037E-1,1.6442794E-1,2.4138746E-1,2.23917E-1,3.6627695E-2,0E0,0E0,1.8504104E-1,0E0,1.1250734E-1,0E0,0E0,1.8489003E-2,0E0,0E0,0E0,5.702096E-2,3.844726E-2,1.1533919E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,17,17,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,-1,-1,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9512938E3,3.2316522E7,7.096748E7,7.7E2,3.167669E5,8.892E3,1.6872E5,3.13E2,2.4636926E3,7.287E3,6.399417E-3,1.9342003E-2,4.6947882E-1,3.6078976E-3,2.3342845E5,-7.6137953E-3,7.265229E-4,1.4731E4,-3.1281011E-3,-4.4159214E-3,-6.635637E-4,4.171707E9,1.0488685E6,5.7194314E0,8.2625286E-4,-1.0050992E-2,-1.7998286E-2,3.225355E-3,-1.4410706E-3,7.2806743E-3,2.4139055E-3,-1.8518103E-3,-1.0082227E-2],"split_indices":[52,12,45,11,28,9,28,11,4,2,0,0,39,0,33,0,0,9,0,0,0,5,28,53,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5.9E1,5.3E1,3.4E1,2.5E1,3.6E1,1.7E1,1.8E1,1.6E1,1.8E1,7E0,6E0,3E1,4E0,1.3E1,1.1E1,7E0,1.2E1,4E0,1.3E1,5E0,1.2E1,1.8E1,9E0,4E0,8E0,4E0,5E0,7E0,1.4E1,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.02846E-2,-1.2583281E-1,1.609192E-1,-2.4537455E-1,-3.224422E-2,2.2265982E-2,2.2434208E-1,-3.0606943E-1,-1.10208705E-1,-8.0833375E-2,6.9238697E-3,6.414566E-3,-3.1021109E-2,1.4450054E-1,3.30083E-1,-7.0478697E-3,-1.5188813E-2,-1.463285E-3,-5.6986716E-3,2.9271317E-3,-1.4155595E-1,-2.425322E-3,1.2423156E-3,6.8439497E-3,9.876944E-2,2.1005973E-1,1.5302213E-2,-7.6886537E-3,-7.620329E-2,1.3894588E-3,4.297493E-3,3.7717826E-3,9.879033E-3,-4.072505E-3,-8.679343E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3012059E0,6.074141E-1,4.9625063E-1,1.631614E-1,3.6382857E-1,1.6285528E-1,2.8156102E-1,1.2120068E-1,2.347505E-2,2.76931E-1,0E0,0E0,3.779842E-2,3.0739427E-2,9.103167E-2,0E0,0E0,0E0,0E0,0E0,7.710606E-2,0E0,0E0,0E0,1.2521699E-2,3.1826735E-2,0E0,0E0,3.2140665E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,20,20,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,3.6E1,1.0539878E-1,1.977643E6,7.621429E1,5.28E2,1.5605432E9,1E0,1.453125E1,4.828443E-1,6.9238697E-3,6.414566E-3,7.331134E-2,1.8831E5,2.5172096E-1,-7.0478697E-3,-1.5188813E-2,-1.463285E-3,-5.6986716E-3,2.9271317E-3,9.5137325E-4,-2.425322E-3,1.2423156E-3,6.8439497E-3,2.0492467E5,2.4190365E6,1.5302213E-2,-7.6886537E-3,1.3890547E1,1.3894588E-3,4.297493E-3,3.7717826E-3,9.879033E-3,-4.072505E-3,-8.679343E-5],"split_indices":[7,3,38,48,56,0,7,16,54,27,0,0,38,29,38,0,0,0,0,0,39,0,0,0,33,32,0,0,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.4E1,5.6E1,2.3E1,3.1E1,1.8E1,3.8E1,1.5E1,8E0,2.6E1,5E0,4E0,1.4E1,2.3E1,1.5E1,9E0,6E0,4E0,4E0,7E0,1.9E1,9E0,5E0,1E1,1.3E1,8E0,7E0,8E0,1.1E1,4E0,9E0,4E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7413821E-3,-7.304192E-2,1.538762E-1,-4.7649376E-2,-1.2434536E-2,9.16067E-2,2.8793538E-1,-7.807313E-2,1.2463469E-2,-1.2068142E-3,4.2704353E-3,3.3709877E-3,1.4516395E-2,-1.6121142E-1,-2.4195444E-3,-2.2930211E-1,-9.5653705E-2,1.843168E-1,-6.696599E-2,-4.3924944E-3,-1.0643932E-2,-4.8598866E-3,-6.4819455E-5,8.599346E-3,3.378555E-3,-9.3958385E-2,7.120131E-4,-2.051382E-3,-5.5891774E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,-1,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.1651835E0,4.7973612E-1,2.4910057E-1,8.2777345E-1,0E0,8.400969E-2,1.8454671E-1,3.9586163E-1,0E0,0E0,0E0,0E0,0E0,1.11982524E-1,4.2215064E-1,6.286186E-2,6.1919376E-2,2.1703422E-2,6.269066E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.64033E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,13,13,14,14,15,15,16,16,17,17,18,18,25,25],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,-1,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[1.0230335E0,1.5081978E8,1.5605432E9,8.517173E3,-1.2434536E-2,3.22115E5,2.5172096E-1,3.0870352E1,1.2463469E-2,-1.2068142E-3,4.2704353E-3,3.3709877E-3,1.4516395E-2,2.1883162E1,3.24395E5,3.9024E4,1.3101191E1,1.1247059E1,3.5324219E3,-4.3924944E-3,-1.0643932E-2,-4.8598866E-3,-6.4819455E-5,8.599346E-3,3.378555E-3,5.4470426E-1,7.120131E-4,-2.051382E-3,-5.5891774E-3],"split_indices":[39,45,7,4,0,1,38,56,0,0,0,0,0,58,29,12,56,54,52,0,0,0,0,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.1E1,3.2E1,6.6E1,5E0,2.3E1,9E0,6.2E1,4E0,4E0,1.9E1,4E0,5E0,2.9E1,3.3E1,1.3E1,1.6E1,8E0,2.5E1,6E0,7E0,1.1E1,5E0,4E0,4E0,1.9E1,6E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.636178E-2,-7.384814E-2,1.9631414E-1,9.506933E-3,-9.827288E-2,1.5571484E-1,1.4757711E-2,-5.056059E-2,-2.8890294E-1,1.1577707E-1,1.2283535E-2,-1.6273664E-1,9.539735E-4,-4.91648E-3,-1.3261816E-2,7.256067E-5,1.4154191E-1,-1.8801776E-1,-2.3271416E-3,-6.0944006E-2,1.1141794E-1,6.11137E-3,2.0914278E-3,-8.3101755E-3,-2.690497E-3,-8.331574E-2,1.6320741E-3,6.5980474E-3,1.2661217E-3,-3.5396528E-3,-5.985618E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,17,19,-1,-1,-1,21,23,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[1.7027802E0,6.023588E-1,2.3823917E-1,0E0,5.8568907E-1,1.8905854E-1,0E0,3.1220505E-1,1.0569215E-1,7.885781E-2,0E0,3.0118614E-2,2.6661515E-1,0E0,0E0,0E0,4.1209966E-2,4.5053452E-2,0E0,6.313539E-2,6.984249E-2,0E0,0E0,0E0,0E0,1.9014508E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,11,11,12,12,16,16,17,17,19,19,20,20,25,25],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,18,20,-1,-1,-1,22,24,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,6.371407E0,1.1276298E9,9.506933E-3,7.526629E7,6.2767E4,1.4757711E-2,2.782713E6,8.588E3,3.22115E5,1.2283535E-2,1.8873918E3,3.9219662E5,-4.91648E-3,-1.3261816E-2,7.256067E-5,1.9445117E2,1.9121604E-4,-2.3271416E-3,7.64755E5,8E0,6.11137E-3,2.0914278E-3,-8.3101755E-3,-2.690497E-3,3.8E1,1.6320741E-3,6.5980474E-3,1.2661217E-3,-3.5396528E-3,-5.985618E-4],"split_indices":[39,54,32,0,45,2,0,12,9,1,0,52,28,0,0,0,56,38,0,9,8,0,0,0,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,4E0,6.5E1,3E1,4E0,5.3E1,1.2E1,2.6E1,4E0,1.6E1,3.7E1,5E0,7E0,5E0,2.1E1,1.2E1,4E0,2.4E1,1.3E1,1.5E1,6E0,8E0,4E0,2E1,4E0,6E0,7E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.2470782E-2,-6.1596636E-2,1.5408692E-1,1.3200057E-2,-1.5370871E-1,6.761617E-2,2.7063823E-1,-4.1043464E-2,1.1494473E-2,-2.795465E-1,-4.729551E-2,1.0111448E-1,-2.4008842E-3,3.305493E-1,3.058147E-3,-1.1177263E-1,1.4055576E-2,-5.4431413E-3,-1.1407882E-2,1.5595459E-2,-6.3424553E-3,2.8277634E-4,5.310851E-3,6.1746673E-3,1.434267E-2,-6.1946614E-3,-5.2834257E-2,-5.1676974E-2,4.3003736E-3,-1.9323725E-3,5.226226E-2,-7.302295E-5,-3.3604323E-3,5.067166E-4,-2.9764003E-3,-2.6324703E-4,3.6595308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[1.100362E0,4.7741678E-1,3.4451842E-1,6.5867484E-1,4.0189344E-1,1.0556354E-1,1.3909137E-1,1.3407274E-1,0E0,1.8049717E-2,1.4807205E-1,7.6065645E-2,0E0,4.9132705E-2,0E0,4.3811426E-2,1.4427063E-1,0E0,0E0,3.574428E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.821586E-2,2.7438339E-2,0E0,0E0,2.705666E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,19,19,26,26,27,27,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[9.745363E-1,2.5927516E11,4.8E1,8.784E3,3.5904072E3,4.5E1,1.31E3,1.56967745E1,1.1494473E-2,3.290356E5,7.752181E7,9.1094507E2,-2.4008842E-3,1.5605432E9,3.058147E-3,9.7E1,3.1228534E5,-5.4431413E-3,-1.1407882E-2,4.6562446E3,-6.3424553E-3,2.8277634E-4,5.310851E-3,6.1746673E-3,1.434267E-2,-6.1946614E-3,1.7076684E7,1.4783E4,4.3003736E-3,-1.9323725E-3,1.8302105E6,-7.302295E-5,-3.3604323E-3,5.067166E-4,-2.9764003E-3,-2.6324703E-4,3.6595308E-3],"split_indices":[42,31,56,2,4,3,0,56,0,28,45,52,0,7,0,10,28,0,0,55,0,0,0,0,0,0,45,9,0,0,32,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.8E1,3.5E1,3.8E1,3E1,2.1E1,1.4E1,3.3E1,5E0,1.3E1,1.7E1,1.7E1,4E0,1E1,4E0,1.4E1,1.9E1,4E0,9E0,1.2E1,5E0,6E0,1.1E1,4E0,6E0,6E0,8E0,1.2E1,7E0,4E0,8E0,4E0,4E0,4E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-5.6217406E-3,-1.4812388E-1,1.00196324E-1,-2.0029053E-1,4.5021903E-3,4.8456933E-2,2.4591851E-1,-1.6120425E-1,-1.4022321E-2,-2.4330225E-2,1.2546659E-1,2.9834944E-1,2.5597236E-3,-2.0845999E-1,6.655365E-4,-6.092472E-2,4.7496065E-2,1.6492017E-1,6.743358E-2,6.5516746E-3,1.2849381E-2,-1.1372252E-1,-9.620884E-3,-3.717309E-3,-1.2116025E-2,4.403759E-3,-1.317208E-3,7.523454E-3,3.534184E-3,1.4049079E-5,4.449301E-3,-2.2817212E-3,-6.2029837E-3,1.0321692E-3,-1.819534E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,21,-1,23,25,27,29,-1,-1,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6283265E0,6.796731E-1,4.5553726E-1,2.2413254E-1,0E0,2.6650238E-1,1.2566847E-1,2.924673E-1,0E0,6.860485E-2,4.1539848E-2,1.6141653E-2,0E0,1.181159E-1,0E0,3.3634484E-2,6.0745515E-2,1.7477602E-2,4.1107986E-2,0E0,0E0,2.4359941E-2,0E0,0E0,1.5465637E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,22,-1,24,26,28,30,-1,-1,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.363496E7,2.4722983E3,1.5791555E9,1E0,4.5021903E-3,1.1058695E0,2.797345E6,2.834008E-1,-1.4022321E-2,3.9219662E5,4.4E1,8.443332E-1,2.5597236E-3,3.842849E5,6.655365E-4,4.1E1,9.513889E-1,2.1987225E5,5.1E1,6.5516746E-3,1.2849381E-2,1.3578177E4,-9.620884E-3,-3.717309E-3,2.014695E6,4.403759E-3,-1.317208E-3,7.523454E-3,3.534184E-3,1.4049079E-5,4.449301E-3,-2.2817212E-3,-6.2029837E-3,1.0321692E-3,-1.819534E-3],"split_indices":[12,52,7,112,0,41,29,57,0,28,3,42,0,28,0,3,57,33,3,0,0,33,0,0,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,4.5E1,6.1E1,3.8E1,7E0,4.6E1,1.5E1,3.3E1,5E0,2.4E1,2.2E1,1.1E1,4E0,2.6E1,7E0,1.6E1,8E0,1.2E1,1E1,5E0,6E0,1.1E1,1.5E1,8E0,8E0,4E0,4E0,6E0,6E0,5E0,5E0,7E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7194271E-2,-9.72937E-2,1.8896739E-1,-2.0284453E-1,1.6372813E-2,1.5365627E-2,9.84426E-2,-2.4842085E-1,-6.0105067E-2,6.6005394E-2,-1.114531E-1,1.2767667E-1,4.831433E-4,-1.961372E-1,-1.6990492E-2,-5.9313225E-3,8.2670106E-4,-2.2887293E-2,1.4457154E-1,-1.8221717E-3,-5.0215493E-3,5.5879564E-3,1.1436479E-3,-1.0958742E-1,-1.0396929E-2,1.460551E-3,-6.257253E-2,7.2778035E-3,1.827394E-3,-6.322765E-3,-1.0955501E-3,-3.8567085E-3,-2.0959246E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7688091E0,9.2713606E-1,6.007501E-1,2.4529016E-1,2.4844103E-1,0E0,5.5484504E-2,2.56204E-1,9.888579E-2,1.988596E-1,1.1518404E-2,3.932637E-2,0E0,1.7511368E-1,0E0,0E0,0E0,3.7724E-2,7.014665E-2,0E0,0E0,0E0,0E0,7.259332E-2,0E0,0E0,2.198869E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,18,18,23,23,26,26],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5324219E3,3.2936362E6,6.4417176E-2,5.127381E1,2.7260774E7,1.5365627E-2,8.442486E-1,1E0,6.444E3,1.1462246E3,5.661451E-1,2.1897722E10,4.831433E-4,4.178728E7,-1.6990492E-2,-5.9313225E-3,8.2670106E-4,2.4E1,1.4922581E1,-1.8221717E-3,-5.0215493E-3,5.5879564E-3,1.1436479E-3,9.7E1,-1.0396929E-2,1.460551E-3,2.83E2,7.2778035E-3,1.827394E-3,-6.322765E-3,-1.0955501E-3,-3.8567085E-3,-2.0959246E-4],"split_indices":[52,50,57,56,45,0,27,112,2,52,27,5,0,45,0,0,0,3,54,0,0,0,0,10,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.6E1,2.9E1,3.9E1,3.7E1,7E0,2.2E1,2.9E1,1E1,2.7E1,1E1,1.6E1,6E0,2.5E1,4E0,4E0,6E0,1.3E1,1.4E1,4E0,6E0,1.2E1,4E0,1.4E1,1.1E1,5E0,8E0,8E0,6E0,7E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.814685E-2,-7.426212E-2,1.5305519E-1,-1.5168595E-1,5.018264E-3,2.13539E-1,-5.187447E-3,-1.9152962E-1,1.1419349E-3,4.6743922E-2,-7.5811096E-2,1.53488E-2,1.2629546E-1,-1.550316E-1,-1.3174422E-2,-3.4991976E-2,1.2514096E-1,-3.5485455E-3,-1.0635336E-3,3.955607E-2,1.9900203E-1,-8.6807534E-2,-2.323222E-1,1.5362607E-3,-7.0264846E-2,6.1947E-3,2.609006E-4,2.6120916E-3,-4.1524476E-5,3.763078E-3,9.169147E-3,-5.7708244E-3,-1.9698124E-2,-1.0033611E-2,-3.7470725E-3,-3.7190807E-3,-6.106465E-4,9.7564876E-4,-2.5931012E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,21,-1,23,25,-1,-1,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0104992E0,4.8567823E-1,4.9732918E-1,2.9821712E-1,1.3845004E-1,3.5244358E-1,0E0,1.5225494E-1,0E0,1.7739096E-1,1.1691667E-2,0E0,9.933916E-2,1.32752E-1,0E0,4.128278E-2,7.5428426E-2,0E0,0E0,1.20978765E-2,2.0413637E-2,8.017501E-2,4.7340453E-2,0E0,1.5729092E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6295848E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,24,24,32,32],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,22,-1,24,26,-1,-1,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[4.492037E3,3.851071E7,9.302862E7,1.2599278E1,2.7260774E7,6.691E3,-5.187447E-3,8.8945635E-2,1.1419349E-3,1.1462246E3,4.1E1,1.53488E-2,3.3244818E7,4.8861527E9,-1.3174422E-2,2.4E1,1.5E1,-3.5485455E-3,-1.0635336E-3,2.2087644E9,1.5821141E0,1.66E2,3.159013E3,1.5362607E-3,3.670858E1,6.1947E-3,2.609006E-4,2.6120916E-3,-4.1524476E-5,3.763078E-3,9.169147E-3,-5.7708244E-3,1.13474E5,-1.0033611E-2,-3.7470725E-3,-3.7190807E-3,-6.106465E-4,9.7564876E-4,-2.5931012E-3],"split_indices":[52,12,45,53,45,9,0,38,0,52,3,0,51,5,0,3,8,0,0,12,39,0,4,0,58,0,0,0,0,0,0,0,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7.8E1,2.5E1,3.9E1,3.9E1,2.1E1,4E0,3.2E1,7E0,2.6E1,1.3E1,5E0,1.6E1,2.8E1,4E0,1.3E1,1.3E1,8E0,5E0,8E0,8E0,1.6E1,1.2E1,4E0,9E0,9E0,4E0,4E0,4E0,4E0,4E0,7E0,9E0,8E0,4E0,5E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.737481E-3,-8.9211E-2,1.6433759E-1,-1.941062E-1,-9.074778E-3,2.2123896E-1,-4.01216E-3,-7.731144E-2,-2.4368058E-1,5.9842452E-2,-9.1321714E-2,1.3956001E-1,3.5705885E-1,-4.9284315E-3,-1.8872228E-4,-1.8668921E-1,-1.4445293E-2,1.2894379E-1,9.785402E-3,-5.6381626E-3,-4.4391062E-2,1.086708E-2,7.029671E-2,1.6602146E-2,5.1169656E-3,-7.6647694E-3,-2.8027901E-3,1.7613143E-3,6.291E-3,2.3403992E-3,-1.2315066E-3,4.2009E-4,-2.5483228E-3,6.3044066E-3,4.8197852E-3,2.5986042E-3,-2.18891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,25,-1,27,29,-1,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1],"loss_changes":[1.585721E0,6.202755E-1,6.0454416E-1,1.6644156E-1,2.4919885E-1,2.9407132E-1,0E0,4.6226688E-2,1.2144017E-1,8.175443E-2,5.3933844E-2,2.1366283E-1,1.7830575E-1,0E0,0E0,3.379804E-2,0E0,2.7816713E-2,3.845221E-2,0E0,1.9899528E-2,0E0,6.387175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3925844E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,20,20,22,22,33,33],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,26,-1,28,30,-1,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1],"split_conditions":[1.1855E4,1.04856936E8,1.650269E10,2.5107703E-1,2.8060134E7,1.1811385E4,-4.01216E-3,1.7593515E1,9.180692E7,8.66623E0,9.095E3,2.70482E5,3.813527E2,-4.9284315E-3,-1.8872228E-4,9.869215E-4,-1.4445293E-2,2.0195505E-1,1.2597537E7,-5.6381626E-3,1.1894881E6,1.086708E-2,1.5603712E2,1.6602146E-2,5.1169656E-3,-7.6647694E-3,-2.8027901E-3,1.7613143E-3,6.291E-3,2.3403992E-3,-1.2315066E-3,4.2009E-4,-2.5483228E-3,4.653145E9,4.8197852E-3,2.5986042E-3,-2.18891E-3],"split_indices":[2,7,5,27,45,52,0,58,7,53,9,29,56,0,0,39,0,38,32,0,48,0,58,0,0,0,0,0,0,0,0,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,7.3E1,3.6E1,3.1E1,4.2E1,3E1,6E0,1E1,2.1E1,2.3E1,1.9E1,2E1,1E1,5E0,5E0,1.7E1,4E0,9E0,1.4E1,7E0,1.2E1,5E0,1.5E1,6E0,4E0,1.3E1,4E0,4E0,5E0,6E0,8E0,4E0,8E0,8E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[5.060023E-3,-3.9900444E-2,1.6904347E-1,-1.4403671E-1,-2.902061E-3,1.5705138E-2,7.09969E-2,-1.0699426E-1,-8.122603E-3,5.0977856E-2,-7.2072595E-2,-4.707141E-3,1.3784634E-1,-1.8178518E-3,-5.1125013E-3,-9.907756E-3,1.2770411E-1,-1.289613E-1,3.4883092E-3,8.555458E-3,1.2231402E-3,-3.3133812E-3,1.47538865E-2,5.727273E-3,1.0081484E-3,-8.083487E-2,-7.0653395E-3,-1.5714758E-3,1.8053614E-3,3.6307182E-2,-1.1123942E-3,-4.3220604E-3,-3.695545E-4,8.1295395E-5,3.0158428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[7.8825873E-1,3.2322335E-1,5.688874E-1,4.1894823E-2,2.3846897E-1,0E0,2.6028833E-1,2.6200786E-2,0E0,1.6944015E-1,1.2080957E-1,0E0,1.3625526E-1,0E0,0E0,4.3753054E-2,4.4953912E-2,3.557062E-2,3.0608458E-2,0E0,0E0,0E0,1.7814184E-2,0E0,0E0,2.8968133E-2,0E0,0E0,0E0,1.9112017E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,15,15,16,16,17,17,18,18,22,22,25,25,29,29],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[4.3972437E3,7.5773336E7,6.4417176E-2,7.156922E7,2.727403E7,1.5705138E-2,9.040125E5,6.072427E-1,-8.122603E-3,1.6987854E1,5.127381E1,-4.707141E-3,3.077632E5,-1.8178518E-3,-5.1125013E-3,2.3616585E6,1.8634315E1,4.95E9,3.1676855E3,8.555458E-3,1.2231402E-3,-3.3133812E-3,1.499169E1,5.727273E-3,1.0081484E-3,2E0,-7.0653395E-3,-1.5714758E-3,1.8053614E-3,2.0195505E-1,-1.1123942E-3,-4.3220604E-3,-3.695545E-4,8.1295395E-5,3.0158428E-3],"split_indices":[52,7,57,7,45,0,32,27,0,56,56,0,28,0,0,48,54,5,52,0,0,0,54,0,0,8,0,0,0,38,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8.3E1,2.2E1,2.1E1,6.2E1,5E0,1.7E1,1.6E1,5E0,3.5E1,2.7E1,4E0,1.3E1,7E0,9E0,2E1,1.5E1,1.5E1,1.2E1,6E0,7E0,4E0,1.6E1,1.1E1,4E0,1E1,5E0,6E0,6E0,1.1E1,5E0,6E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[9.323929E-3,-7.813383E-2,1.8397304E-1,-1.0277315E-1,9.485412E-3,2.4387056E-1,7.670746E-2,-1.3511458E-1,-2.7036062E-2,4.4523873E-3,2.813675E-1,1.5282334E-3,4.697767E-3,-1.0695489E-1,-1.1012138E-2,2.3927214E-3,-5.2890096E-2,1.204059E-2,3.6088598E-3,-7.6536015E-2,-6.6053187E-3,-3.0771093E-3,-7.8444736E-4,-1.20134674E-1,-2.815386E-2,-1.2017918E-3,-5.6531634E-3,6.4621645E-4,-1.6245084E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,-1,21,-1,-1,23,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[1.6029142E0,6.1491096E-1,2.0616627E-1,1.5801042E-1,0E0,6.474292E-2,2.0199351E-2,1.8746293E-1,5.416703E-2,0E0,1.1799741E-1,0E0,0E0,8.078575E-2,0E0,0E0,1.5281063E-2,0E0,0E0,6.2009066E-2,0E0,0E0,0E0,4.543622E-2,1.3215896E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,16,16,19,19,23,23,24,24],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,-1,22,-1,-1,24,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[7.90671E-1,6.082581E3,1.33556795E1,2.561146E3,9.485412E-3,3.840785E1,5.72143E5,1.01949844E6,3.09E2,4.4523873E-3,2.034139E6,1.5282334E-3,4.697767E-3,5.501383E9,-1.1012138E-2,2.3927214E-3,1.4E1,1.204059E-2,3.6088598E-3,2.91E2,-6.6053187E-3,-3.0771093E-3,-7.8444736E-4,9.122283E0,3.717E3,-1.2017918E-3,-5.6531634E-3,6.4621645E-4,-1.6245084E-3],"split_indices":[39,52,54,52,0,56,29,28,0,0,29,0,0,5,0,0,8,0,0,0,0,0,0,53,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6.9E1,3.4E1,6.5E1,4E0,2.1E1,1.3E1,4.5E1,2E1,6E0,1.5E1,9E0,4E0,4E1,5E0,4E0,1.6E1,1.1E1,4E0,3E1,1E1,7E0,9E0,1.5E1,1.5E1,5E0,1E1,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.867907E-3,-6.967474E-2,1.558265E-1,-1.2791926E-1,3.3604927E-2,7.122294E-2,2.1767244E-1,1.9862214E-3,-1.8571019E-1,9.777716E-3,-1.3307461E-2,3.3243027E-2,3.966981E-3,1.3160989E-1,1.2062451E-2,4.2340755E-3,-5.5825025E-2,-1.08377844E-1,-2.2470771E-1,2.86648E-2,-7.051779E-3,2.6680941E-3,-5.077828E-4,6.3804924E-3,1.1948403E-3,-3.6459872E-3,-3.299267E-4,-1.6376357E-1,8.6896884E-4,-8.955007E-3,-4.9078837E-3,-3.211083E-2,8.198706E-2,-7.876448E-3,-3.2902192E-3,-2.5604675E-3,7.1524755E-6,4.625124E-3,1.5492659E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1663843E0,4.915521E-1,1.5193307E-1,3.9075166E-1,3.3746555E-1,1.8369898E-2,1.3640732E-1,1.1985446E-1,8.421445E-2,0E0,2.0422962E-1,1.8350948E-2,0E0,4.7491208E-2,0E0,0E0,2.4379428E-2,1.06641084E-1,1.3918042E-2,7.392795E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9629747E-2,0E0,0E0,0E0,1.3807188E-2,1.612898E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,19,19,27,27,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,2.561146E3,2.4221443E-1,5.1056176E-1,6.169213E0,1.0188E4,1.286E4,1.0526E4,4.178728E7,9.777716E-3,1.0990962E8,5E0,3.966981E-3,1.1180212E1,1.2062451E-2,4.2340755E-3,2.83E2,5.677647E2,1.1309524E-1,1E0,-7.051779E-3,2.6680941E-3,-5.077828E-4,6.3804924E-3,1.1948403E-3,-3.6459872E-3,-3.299267E-4,4.2E1,8.6896884E-4,-8.955007E-3,-4.9078837E-3,1.0837745E1,1.18631315E1,-7.876448E-3,-3.2902192E-3,-2.5604675E-3,7.1524755E-6,4.625124E-3,1.5492659E-3],"split_indices":[39,52,38,27,54,9,2,9,45,0,45,8,0,54,0,0,0,52,57,109,0,0,0,0,0,0,0,3,0,0,0,54,54,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,8E1,3.1E1,5.1E1,2.9E1,1.4E1,1.7E1,1.6E1,3.5E1,4E0,2.5E1,8E0,6E0,1.1E1,6E0,5E0,1.1E1,1.3E1,2.2E1,2.1E1,4E0,4E0,4E0,7E0,4E0,5E0,6E0,9E0,4E0,1.6E1,6E0,1E1,1.1E1,4E0,5E0,4E0,6E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.1503411E-2,-6.022531E-2,1.0210416E-1,1.1743105E-2,-1.8415894E-1,1.3324112E-1,-6.2689167E-3,-1.1934079E-1,5.8139388E-2,-2.0429802E-3,-2.1084705E-1,8.170475E-2,2.848334E-1,-1.7364284E-3,-6.5334076E-3,1.0161169E-1,-3.0271929E-2,-3.4372571E-3,-9.57698E-3,5.4357923E-3,4.3561555E-2,1.4179162E-2,3.5516846E-3,1.3515343E-1,2.3150533E-4,2.9081703E-4,-2.06386E-3,6.5976917E-3,3.8783143E-3,7.5871915E-2,8.184036E-3,1.1731047E-3,-1.8662412E-3,4.8724907E-3,5.049307E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,23,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1,-1],"loss_changes":[6.625155E-1,5.105631E-1,4.0949824E-1,2.319652E-1,5.956614E-2,2.9704094E-1,0E0,3.5200447E-2,1.1069242E-1,0E0,8.173025E-2,7.7488944E-2,1.6256648E-1,0E0,0E0,5.9803218E-2,1.0768166E-2,0E0,0E0,0E0,5.1224194E-2,0E0,0E0,5.876735E-2,0E0,0E0,0E0,2.432139E-2,0E0,3.4680277E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,15,15,16,16,20,20,23,23,27,27,29,29],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,24,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1,-1],"split_conditions":[1.9552084E3,4.307821E7,2.192772E10,1.92E2,5.3323513E-1,1.2768678E4,-6.2689167E-3,8.5959184E-1,2.70482E5,-2.0429802E-3,2.6078947E1,1.2187778E11,8.260109E9,-1.7364284E-3,-6.5334076E-3,4.3952936E7,2.1151997E5,-3.4372571E-3,-9.57698E-3,5.4357923E-3,2.3E1,1.4179162E-2,3.5516846E-3,1.5150703E3,2.3150533E-4,2.9081703E-4,-2.06386E-3,3.200296E9,3.8783143E-3,1.2410928E8,8.184036E-3,1.1731047E-3,-1.8662412E-3,4.8724907E-3,5.049307E-4],"split_indices":[52,45,5,0,27,52,0,27,29,0,56,31,5,0,0,32,28,0,0,0,8,0,0,4,0,0,0,12,0,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.6E1,4.4E1,3.6E1,2E1,4E1,4E0,9E0,2.7E1,4E0,1.6E1,3.1E1,9E0,5E0,4E0,1.8E1,9E0,6E0,1E1,1E1,2.1E1,5E0,4E0,1.3E1,5E0,4E0,5E0,1.4E1,7E0,9E0,4E0,1E1,4E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.697377E-3,-9.7170286E-2,8.702731E-2,-1.2952103E-1,5.0018826E-3,1.0664505E-2,1.9197288E-1,-1.9993305E-1,-4.4293467E-2,4.369449E-2,-5.8928872E-3,4.7823135E-3,9.264073E-3,-1.4409603E-1,-1.4644435E-2,-1.0872006E-1,4.8893443E-3,5.649267E-3,9.390396E-3,-8.207968E-2,-8.107118E-3,-5.162672E-3,-1.6550336E-3,2.0104758E-3,-1.4502563E-3,-3.0271173E-2,4.100922E-3,-4.444497E-3,5.9169147E-4,-4.6842344E-2,5.6154124E-4,-3.8674063E-4,-2.9311008E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,-1,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1],"loss_changes":[8.807678E-1,4.2972445E-1,3.9864957E-1,2.7919453E-1,0E0,1.7943832E-1,4.6776295E-2,2.4017239E-1,7.3411226E-2,9.912152E-2,0E0,0E0,0E0,9.134397E-2,0E0,1.499825E-2,3.4281798E-2,0E0,9.0225205E-2,5.8574945E-2,0E0,0E0,0E0,0E0,0E0,1.28785735E-2,0E0,0E0,0E0,1.402094E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,18,18,19,19,25,25,29,29],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,-1,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1],"split_conditions":[2.1915238E8,2.561146E3,1.3536367E0,2.667836E6,5.0018826E-3,8.637973E6,1.3127055E4,1.5755641E6,2.9544958E-1,1.2192353E7,-5.8928872E-3,4.7823135E-3,9.264073E-3,2.8225484E11,-1.4644435E-2,6.114E3,3.7E1,5.649267E-3,5.528107E3,3.845592E5,-8.107118E-3,-5.162672E-3,-1.6550336E-3,2.0104758E-3,-1.4502563E-3,4.599408E6,4.100922E-3,-4.444497E-3,5.9169147E-4,3.494164E7,5.6154124E-4,-3.8674063E-4,-2.9311008E-3],"split_indices":[7,52,39,32,0,47,4,32,41,45,0,0,0,31,0,2,3,0,52,32,0,0,0,0,0,1,0,0,0,45,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.3E1,4.9E1,4.7E1,6E0,2.9E1,2E1,2.5E1,2.2E1,2.5E1,4E0,1.2E1,8E0,2.1E1,4E0,9E0,1.3E1,5E0,2E1,1.3E1,8E0,5E0,4E0,6E0,7E0,1.5E1,5E0,9E0,4E0,1.1E1,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-4.237777E-3,-8.269796E-2,7.865978E-2,-1.0649972E-1,4.9593593E-3,2.8124793E-2,1.8699329E-1,-7.389247E-2,-9.9861985E-3,5.3282514E-2,-5.363919E-3,2.2256964E-1,2.0613405E-3,-1.5181887E-2,-1.7935653E-1,1.2531739E-2,1.1060278E-1,9.59934E-3,3.6133083E-3,-4.053344E-2,4.320899E-3,-3.6861477E-3,-7.78019E-3,5.7807863E-2,-3.3883635E-2,1.5514398E-1,9.154708E-4,-9.04947E-2,-3.6517316E-3,-5.268551E-4,3.655955E-3,-2.0437986E-3,1.7005613E-4,7.0945136E-3,2.9875794E-3,-4.632834E-3,-1.356904E-3,1.3300382E-3,-1.4166357E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.219622E-1,3.1319165E-1,2.90371E-1,2.6798433E-1,0E0,1.7683795E-1,6.5853715E-2,2.7591056E-1,0E0,7.8032695E-2,0E0,4.2365193E-2,0E0,1.0742183E-1,2.2673666E-2,4.609041E-2,4.9223155E-2,0E0,0E0,4.7578614E-2,0E0,0E0,0E0,3.608715E-2,9.914843E-3,1.0688394E-2,0E0,1.7193533E-2,2.4501283E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,19,19,23,23,24,24,25,25,27,27,28,28],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4707712E8,5.188755E3,1.7618678E0,1E0,4.9593593E-3,1.5939707E7,2.393487E5,4.352552E9,-9.9861985E-3,1.9649E4,-5.363919E-3,1.8613E4,2.0613405E-3,5.980516E5,7.056912E7,1.5E1,2.3152558E9,9.59934E-3,3.6133083E-3,2.83E2,4.320899E-3,-3.6861477E-3,-7.78019E-3,2.951724E1,4.1E1,1.2271712E1,9.154708E-4,1.85E2,1.0521739E1,-5.268551E-4,3.655955E-3,-2.0437986E-3,1.7005613E-4,7.0945136E-3,2.9875794E-3,-4.632834E-3,-1.356904E-3,1.3300382E-3,-1.4166357E-3],"split_indices":[7,4,39,112,0,47,33,5,0,2,0,9,0,28,7,8,7,0,0,0,0,0,0,56,3,54,0,10,53,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,5.6E1,5.3E1,5.1E1,5E0,3.7E1,1.6E1,4.4E1,7E0,3.3E1,4E0,1.2E1,4E0,2.9E1,1.5E1,2E1,1.3E1,8E0,4E0,2.5E1,4E0,6E0,9E0,1E1,1E1,8E0,5E0,1E1,1.5E1,4E0,6E0,6E0,4E0,4E0,4E0,5E0,5E0,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-5.985119E-3,-3.7334602E-2,2.798937E-1,-1.1924549E-1,3.852818E-2,1.2968256E-2,4.6480955E-3,-1.0174001E-2,-8.61623E-2,1.4390624E-1,-4.7238437E-3,-6.5295007E-3,-1.8414815E-1,6.496868E-3,1.5535147E-3,-8.2686484E-2,3.44543E-2,-6.916707E-2,9.163222E-2,-2.148222E-3,-7.867692E-3,-7.92647E-4,-3.9824406E-3,1.4682037E-2,3.2970735E-3,7.660684E-4,-3.8790284E-3,5.910601E-3,4.4155237E-5,-1.5095895E-3,5.1122166E-2,4.8930116E-4,2.7445937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1],"loss_changes":[9.06665E-1,5.700673E-1,7.3627055E-2,2.1235025E-1,2.1945485E-1,0E0,0E0,0E0,2.953073E-1,4.3165177E-2,1.0975401E-1,1.4160739E-1,5.8602095E-2,0E0,0E0,1.7707884E-2,2.5645847E-2,5.0714858E-2,5.845225E-2,0E0,0E0,0E0,0E0,4.125292E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0194946E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,23,23,30,30],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1],"split_conditions":[2.8937003E2,2.942787E6,2.95279E9,9.2433E4,9.366E4,1.2968256E-2,4.6480955E-3,-1.0174001E-2,2.4126838E11,7.6363635E-1,2.207872E-1,2.4557575E1,2.0161922E-1,6.496868E-3,1.5535147E-3,1.3919356E5,8.517173E3,5.9775025E-1,6.8E1,-2.148222E-3,-7.867692E-3,-7.92647E-4,-3.9824406E-3,1E0,3.2970735E-3,7.660684E-4,-3.8790284E-3,5.910601E-3,4.4155237E-5,-1.5095895E-3,5.3278846E-1,4.8930116E-4,2.7445937E-3],"split_indices":[56,51,7,1,29,0,0,0,31,57,39,56,27,0,0,33,4,27,10,0,0,0,0,109,0,0,0,0,0,0,27,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9E1,9E0,4.3E1,4.7E1,5E0,4E0,6E0,3.7E1,1.3E1,3.4E1,2.1E1,1.6E1,9E0,4E0,1.1E1,2.3E1,1.3E1,8E0,4E0,1.2E1,4E0,7E0,1.8E1,5E0,4E0,9E0,4E0,4E0,7E0,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.1812337E-2,-4.2948756E-2,1.2545536E-1,-1.0358797E-1,3.6788717E-2,1.3685177E-2,7.398414E-2,-9.595848E-3,-8.0722906E-2,1.05620585E-1,-5.7439685E-2,1.160214E-1,-3.3475934E-3,-4.8523424E-3,-4.3138888E-2,5.4669747E-4,1.5317461E-1,7.3793606E-4,-3.1632595E-3,5.3820442E-2,5.7059852E-3,1.9388814E-2,-7.952252E-2,6.6943145E-3,3.05605E-3,-1.1515613E-3,3.6343448E-3,-7.71638E-4,2.3259541E-3,-4.2981757E-3,-1.5848794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,-1,-1,21,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0861155E-1,3.761226E-1,4.5492786E-1,1.4173093E-1,2.26588E-1,0E0,2.3501164E-1,0E0,7.490218E-2,8.373228E-2,3.704451E-2,5.945933E-2,0E0,0E0,5.841647E-2,0E0,1.3032317E-2,0E0,0E0,5.1396437E-2,0E0,1.9734863E-2,1.730533E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,14,14,16,16,19,19,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,-1,-1,22,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,2.3514317E8,1.6451234E-2,4.570007E7,1.5046106E7,1.3685177E-2,1.650269E10,-9.595848E-3,1.9121604E-4,1.118632E3,3.9343938E-1,1E0,-3.3475934E-3,-4.8523424E-3,3.25E2,5.4669747E-4,2.326E4,7.3793606E-4,-3.1632595E-3,4.4E1,5.7059852E-3,1.33125E1,1.0613095E1,6.6943145E-3,3.05605E-3,-1.1515613E-3,3.6343448E-3,-7.71638E-4,2.3259541E-3,-4.2981757E-3,-1.5848794E-3],"split_indices":[2,7,57,5,45,0,5,0,38,52,27,109,0,0,0,0,9,0,0,3,0,54,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.6E1,3.6E1,4.3E1,3.3E1,5E0,3.1E1,4E0,3.9E1,1.9E1,1.4E1,2.5E1,6E0,1.5E1,2.4E1,7E0,1.2E1,4E0,1E1,1.1E1,1.4E1,9E0,1.5E1,7E0,5E0,4E0,7E0,5E0,4E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.9333752E-2,-1.7338527E-2,1.7851615E-1,3.232561E-2,-1.274609E-1,3.1809425E-1,-1.1914452E-2,-4.1906007E-2,1.1910658E-1,-1.8609008E-1,-5.4517087E-2,6.789022E-3,1.4265265E-2,3.2805654E-3,-4.054831E-3,-1.5610947E-2,-5.5327644E-3,1.4730425E-1,-1.2428112E-3,-2.8834061E-3,-7.964071E-3,-3.2274926E-3,3.8824248E-5,4.5374773E-2,-5.4154005E-2,1.8340005E-1,2.047027E-3,1.3910569E-4,3.409216E-3,-2.6057994E-3,1.2766357E-4,3.193395E-3,7.889118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,25,-1,-1,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9269094E-1,4.5805824E-1,5.088089E-1,3.78621E-1,1.01047456E-1,3.5895586E-2,1.0307721E-1,9.129667E-2,1.2195939E-1,3.3692688E-2,2.530377E-2,0E0,0E0,0E0,0E0,6.592278E-2,0E0,6.482148E-2,0E0,0E0,0E0,0E0,0E0,2.2041727E-2,1.7930765E-2,3.4020007E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,15,15,17,17,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,26,-1,-1,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.5081426E3,3.494164E7,8.50999E9,1.6987854E1,5.127381E1,2.2059325E5,7.096748E7,7.925698E5,2.8399336E7,2.6078947E1,9.74026E0,6.789022E-3,1.4265265E-2,3.2805654E-3,-4.054831E-3,3.1E1,-5.5327644E-3,1.4922581E1,-1.2428112E-3,-2.8834061E-3,-7.964071E-3,-3.2274926E-3,3.8824248E-5,1.5566606E1,3.95875E5,3.1346828E5,2.047027E-3,1.3910569E-4,3.409216E-3,-2.6057994E-3,1.2766357E-4,3.193395E-3,7.889118E-3],"split_indices":[52,45,5,56,56,28,45,28,45,56,53,0,0,0,0,3,0,54,0,0,0,0,0,54,1,28,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,8.2E1,1.8E1,5.7E1,2.5E1,1E1,8E0,3.1E1,2.6E1,1.3E1,1.2E1,5E0,5E0,4E0,4E0,2.6E1,5E0,2.2E1,4E0,4E0,9E0,7E0,5E0,1E1,1.6E1,1.5E1,7E0,6E0,4E0,1.2E1,4E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.561637E-3,-5.404985E-2,1.2403449E-1,-1.3783206E-1,1.10711185E-2,1.651183E-1,-5.31708E-3,-1.145633E-2,-1.00857444E-1,9.28129E-2,-4.8225813E-2,1.0642485E-1,1.1308259E-2,-1.2210609E-1,7.6926546E-4,5.9804716E-3,5.837162E-3,-4.2109964E-3,-2.0482969E-2,8.411137E-3,4.805522E-2,-1.4793777E-1,-1.3805073E-3,1.3471049E-3,-9.584615E-4,2.6229895E-3,-2.1830723E-3,-1.8132571E-3,9.3190506E-2,-2.3490298E-3,-1.7480707E-1,-7.038803E-4,1.5131987E-3,4.841804E-3,1.0801419E-3,-3.381597E-3,-7.7192737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,-1,21,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,31,-1,-1,33,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[7.356404E-1,3.998124E-1,4.032969E-1,1.8036765E-1,2.0805155E-1,2.2744715E-1,0E0,0E0,7.5285286E-2,1.06672496E-1,4.505991E-2,1.586996E-1,0E0,4.633802E-2,0E0,1.0163315E-2,0E0,0E0,1.7712517E-2,0E0,8.0090515E-2,2.8707564E-2,0E0,0E0,0E0,1.2115218E-2,0E0,0E0,2.705706E-2,0E0,2.1117866E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,15,15,18,18,20,20,21,21,25,25,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,-1,22,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,32,-1,-1,34,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,3.2334878E7,3.6181125E9,2.7571955E6,1.5046106E7,1.1811385E4,-5.31708E-3,-1.145633E-2,2.834008E-1,2.710413E5,7.898E3,2.70482E5,1.1308259E-2,3.159013E3,7.6926546E-4,3.8826916E-1,5.837162E-3,-4.2109964E-3,1.1787824E7,8.411137E-3,1.9649E4,6.059339E2,-1.3805073E-3,1.3471049E-3,-9.584615E-4,4.4178323E8,-2.1830723E-3,-1.8132571E-3,7.118011E-1,-2.3490298E-3,2.77E2,-7.038803E-4,1.5131987E-3,4.841804E-3,1.0801419E-3,-3.381597E-3,-7.7192737E-3],"split_indices":[2,12,7,45,45,52,0,0,57,28,9,29,0,4,0,27,0,0,50,0,2,33,0,0,0,12,0,0,27,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,7.2E1,3.3E1,3.1E1,4.1E1,2.9E1,4E0,4E0,2.7E1,1.7E1,2.4E1,2.2E1,7E0,2.3E1,4E0,8E0,9E0,6E0,1.8E1,6E0,1.6E1,1.7E1,6E0,4E0,4E0,1.2E1,6E0,5E0,1.1E1,5E0,1.2E1,8E0,4E0,6E0,5E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.1684482E-2,-8.92959E-2,4.733844E-2,-1.14849634E-1,5.3549916E-3,-5.167489E-2,8.901832E-2,-8.102203E-2,-1.0009397E-2,-7.7019674E-3,-6.255158E-3,5.3949416E-2,7.921108E-3,-1.3699213E-1,-2.659785E-2,-1.7146335E-3,2.9380014E-3,2.9502569E-2,4.477201E-3,-4.9719807E-2,-7.2869565E-3,3.1689203E-3,-6.0676824E-2,3.0852994E-3,1.0357463E-2,-3.169842E-3,-6.121934E-5,-3.8326145E-3,-2.2772465E-2,-1.8357523E-3,3.135E-2,-1.7745014E-3,4.850933E-4,1.9359319E-3,-6.650146E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,21,-1,-1,23,-1,25,-1,-1,27,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[4.9895924E-1,3.0242434E-1,2.604656E-1,2.1375608E-1,0E0,1.00247085E-1,1.8935111E-1,1.0906297E-1,0E0,5.7033602E-2,0E0,5.800236E-2,0E0,9.183115E-2,8.327968E-2,0E0,0E0,2.9110642E-2,0E0,1.6310014E-2,0E0,0E0,2.516535E-2,0E0,2.9860783E-2,0E0,0E0,0E0,1.0211804E-2,0E0,1.9832205E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,14,14,17,17,19,19,22,22,24,24,28,28,30,30],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,22,-1,-1,24,-1,26,-1,-1,28,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[2.870791E6,1.0098112E2,1E0,1E0,5.3549916E-3,5.9E1,1.5435694E9,1.4780533E-4,-1.0009397E-2,8.517173E3,-6.255158E-3,2.4624854E8,7.921108E-3,8.8824E4,5.739445E4,-1.7146335E-3,2.9380014E-3,3.162046E6,4.477201E-3,7.366E3,-7.2869565E-3,3.1689203E-3,1.94E2,3.0852994E-3,1.091E3,-3.169842E-3,-6.121934E-5,-3.8326145E-3,1.5285803E5,-1.8357523E-3,2.727403E7,-1.7745014E-3,4.850933E-4,1.9359319E-3,-6.650146E-4],"split_indices":[48,56,109,112,0,3,7,38,0,4,0,32,0,12,51,0,0,32,0,9,0,0,10,0,10,0,0,0,28,0,45,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,4.6E1,6.1E1,4.2E1,4E0,1.8E1,4.3E1,3.6E1,6E0,1.4E1,4E0,3.5E1,8E0,1.7E1,1.9E1,1E1,4E0,2.7E1,8E0,8E0,9E0,4E0,1.5E1,6E0,2.1E1,4E0,4E0,6E0,9E0,5E0,1.6E1,5E0,4E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[6.662455E-4,-4.9558103E-2,1.4785257E-1,-2.4057561E-1,-2.6587768E-2,2.1840347E-1,1.255075E-2,-3.0996343E-3,-1.2534267E-2,4.097304E-2,-9.346684E-2,1.2528654E-2,1.323453E-1,-2.1566772E-3,2.252529E-3,-3.8970392E-2,1.1886782E-1,-1.62801E-1,-5.253818E-2,1.8877957E-3,5.8528706E-3,-2.51236E-3,2.2326184E-4,1.6014557E-1,6.9791457E-4,-2.4208599E-3,-7.0182164E-3,-2.220887E-2,-4.260263E-3,2.6866517E-3,6.7511033E-3,-2.7233032E-3,-2.309643E-3,1.3026467E-3,-7.8897056E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1,33,-1,-1,-1],"loss_changes":[8.426613E-1,3.6656123E-1,2.7408314E-1,1.18638515E-1,3.5174674E-1,1.6438305E-1,4.335879E-2,0E0,0E0,2.474922E-1,1.0211915E-1,0E0,2.2506237E-2,0E0,0E0,2.808135E-2,7.9077154E-2,2.9314399E-2,4.900237E-2,0E0,0E0,0E0,0E0,1.831019E-2,0E0,0E0,0E0,1.5288202E-2,0E0,0E0,0E0,1.1181185E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,15,15,16,16,17,17,18,18,23,23,27,27,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1,34,-1,-1,-1],"split_conditions":[1.3137E4,5.7578644E2,2.797345E6,1.44E2,2.8399336E7,6.4417176E-2,1.7313588E7,-3.0996343E-3,-1.2534267E-2,1.6987854E1,1.0951729E3,1.2528654E-2,1.2943E4,-2.1566772E-3,2.252529E-3,1.0289962E3,1.8831E5,3.5E1,8.661788E7,1.8877957E-3,5.8528706E-3,-2.51236E-3,2.2326184E-4,1.7694729E3,6.9791457E-4,-2.4208599E-3,-7.0182164E-3,1.3163732E7,-4.260263E-3,2.6866517E-3,6.7511033E-3,9.985778E0,-2.309643E-3,1.3026467E-3,-7.8897056E-4],"split_indices":[2,4,29,0,45,57,1,0,0,56,33,0,9,0,0,4,29,3,45,0,0,0,0,4,0,0,0,50,0,0,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,8.4E1,2.8E1,8E0,7.6E1,1.8E1,1E1,4E0,4E0,3.8E1,3.8E1,6E0,1.2E1,4E0,6E0,1.9E1,1.9E1,1.3E1,2.5E1,4E0,8E0,1.1E1,8E0,1.3E1,6E0,4E0,9E0,1.8E1,7E0,4E0,9E0,1.3E1,5E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.530839E-3,-4.5932192E-2,1.1182062E-1,-1.1521912E-1,-8.350247E-3,2.2884932E-1,-2.427885E-2,-4.7911853E-2,-1.6587082E-1,3.1141045E-2,-6.1822448E-2,1.0794275E-2,2.0518857E-3,1.2534768E-3,-4.1092574E-3,-3.2648654E-3,5.245126E-5,-2.0179935E-1,-1.9370661E-3,-1.724491E-2,9.574152E-2,-5.9825747E-4,-3.811307E-3,-3.4863604E-3,-8.958482E-3,-5.2409623E-2,2.2318675E-3,2.1035306E-3,5.2006897E-3,-2.9059898E-3,-2.249932E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,-1,-1,23,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[4.5526692E-1,1.9954106E-1,3.8781723E-1,8.4101945E-2,1.0962121E-1,1.3984585E-1,6.734538E-2,2.7418276E-2,4.9649537E-2,9.542371E-2,4.18005E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.9676288E-2,0E0,5.3284883E-2,1.5143432E-2,0E0,0E0,0E0,0E0,1.6356282E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,17,17,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,-1,-1,24,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,3.572374E6,6.1358623E9,8.8824E4,2.7260774E7,1.343964E1,1.063036E8,2.69E2,2.1272728E1,1.118632E3,5.4357555E6,1.0794275E-2,2.0518857E-3,1.2534768E-3,-4.1092574E-3,-3.2648654E-3,5.245126E-5,5.432393E-1,-1.9370661E-3,1.8116838E1,3.6742297E5,-5.9825747E-4,-3.811307E-3,-3.4863604E-3,-8.958482E-3,2.1562665E10,2.2318675E-3,2.1035306E-3,5.2006897E-3,-2.9059898E-3,-2.249932E-4],"split_indices":[2,12,5,12,45,54,51,0,58,52,50,0,0,0,0,0,0,27,0,54,28,0,0,0,0,31,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,7.6E1,2.3E1,2.6E1,5E1,1.2E1,1.1E1,1.2E1,1.4E1,2.9E1,2.1E1,8E0,4E0,7E0,4E0,6E0,6E0,1E1,4E0,1.7E1,1.2E1,1.1E1,1E1,4E0,6E0,1.2E1,5E0,8E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.5239592E-2,-6.590716E-2,1.024054E-1,-1.0023185E-1,2.5494773E-2,1.6332898E-1,-3.6385134E-3,-7.927773E-2,-1.0460003E-2,9.016657E-2,-3.0522956E-2,7.439078E-2,2.6976025E-1,-2.9142814E-2,-1.471239E-1,1.1949413E-3,5.076637E-3,5.7299284E-4,-2.0110758E-3,1.0745387E-1,-4.775046E-4,1.2036726E-2,4.5747217E-3,-5.735839E-2,4.455354E-3,-3.456769E-3,-7.4006557E-3,5.036613E-3,1.4834374E-3,-9.322939E-2,-7.7397805E-3,-1.268817E-3,-4.0130746E-3,1.0774358E-3,-2.3945274E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,-1,-1],"loss_changes":[6.326317E-1,2.3471883E-1,4.1258442E-1,1.9144881E-1,7.8830265E-2,2.1657747E-1,0E0,1.6578552E-1,0E0,2.4136253E-2,1.4902991E-2,4.4874497E-2,5.7891726E-2,1.3620666E-1,4.2802185E-2,0E0,0E0,0E0,0E0,1.7666847E-2,0E0,0E0,0E0,4.5555018E-2,0E0,0E0,0E0,0E0,0E0,1.171273E-2,2.8401129E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,19,19,23,23,29,29,30,30],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,-1,-1],"split_conditions":[3.7249353E3,1.4112683E8,9.302862E7,8.674699E-1,1.8831E5,1.1681136E4,-3.6385134E-3,3.3012803E9,-1.0460003E-2,3.4E1,1.7108389E5,7.2266235E6,8.838004E0,6.444E3,7.444956E7,1.1949413E-3,5.076637E-3,5.7299284E-4,-2.0110758E-3,1.0007143E1,-4.775046E-4,1.2036726E-2,4.5747217E-3,1.2337662E-1,4.455354E-3,-3.456769E-3,-7.4006557E-3,5.036613E-3,1.4834374E-3,5.1612902E-2,3.727E3,-1.268817E-3,-4.0130746E-3,1.0774358E-3,-2.3945274E-3],"split_indices":[52,12,45,57,29,4,0,5,0,3,28,47,53,2,7,0,0,0,0,53,0,0,0,57,0,0,0,0,0,57,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,7.3E1,3.1E1,5.3E1,2E1,2.4E1,7E0,4.9E1,4E0,9E0,1.1E1,1.4E1,1E1,2.9E1,2E1,5E0,4E0,4E0,7E0,1E1,4E0,6E0,4E0,2.5E1,4E0,1.2E1,8E0,6E0,4E0,1.4E1,1.1E1,4E0,1E1,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8842183E-2,-4.7217514E-3,1.8946552E-1,-4.280184E-2,6.544878E-2,2.4351479E-1,1.4723232E-3,-6.7809536E-3,-1.3468155E-1,1.1246225E-1,3.279852E-4,4.6755387E-3,1.1651745E-2,-4.3688595E-2,1.0120317E-1,-7.6954566E-2,-6.5992386E-3,5.445598E-3,3.8187767E-4,1.6872303E-3,-1.9045807E-3,-9.187959E-2,-1.407047E-2,5.62055E-3,1.6945435E-4,-4.080795E-3,-9.201279E-4,-4.730798E-3,-1.1149936E-3,8.1509864E-4,-1.5682246E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6157434E-1,2.3527406E-1,1.3472217E-1,1.8773326E-1,9.487128E-2,7.561958E-2,0E0,1.7180988E-1,3.0367315E-2,6.87775E-2,3.6951836E-2,0E0,0E0,4.4381678E-2,5.7103172E-2,1.3830885E-2,0E0,0E0,0E0,0E0,0E0,2.461379E-2,2.35815E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,14,14,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.621782E3,6.402099E-1,9.299267E9,8.5154843E-1,2.5588E4,1.3772013E4,1.4723232E-3,4.3316594E5,9.067368E5,1.4922581E1,2.4E1,4.6755387E-3,1.1651745E-2,1E0,2.0477528E1,7.158023E7,-6.5992386E-3,5.445598E-3,3.8187767E-4,1.6872303E-3,-1.9045807E-3,3.0387878E3,3.3166495E6,5.62055E-3,1.6945435E-4,-4.080795E-3,-9.201279E-4,-4.730798E-3,-1.1149936E-3,8.1509864E-4,-1.5682246E-3],"split_indices":[52,42,12,27,9,4,0,28,32,54,3,0,0,16,58,7,0,0,0,0,0,4,32,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,8.6E1,1.7E1,5.6E1,3E1,1.2E1,5E0,4.1E1,1.5E1,1.7E1,1.3E1,6E0,6E0,3.1E1,1E1,8E0,7E0,1.2E1,5E0,7E0,6E0,1.1E1,2E1,6E0,4E0,4E0,4E0,6E0,5E0,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.5199473E-3,-2.00504E-2,1.7427467E-1,-1.5459003E-1,8.918726E-3,1.0275923E-2,-2.137881E-3,-2.296439E-3,-6.865453E-3,3.375763E-2,-5.6560256E-2,-3.2112934E-4,9.341152E-2,-1.1457382E-1,6.5951557E-3,-3.560842E-2,5.145497E-2,1.3627262E-1,-2.0416873E-4,-6.3188877E-3,-1.2646198E-3,1.8932971E-3,-1.4567714E-3,-3.0581884E-3,-7.04844E-3,8.38519E-4,3.728294E-3,2.2793598E-4,6.773822E-3,8.1962877E-4,-1.2967597E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2892936E-1,3.5215333E-1,3.5181886E-1,3.968811E-2,1.2389132E-1,0E0,0E0,0E0,0E0,1.1184118E-1,7.769693E-2,6.76056E-2,8.569278E-2,4.676324E-2,2.579118E-2,3.0252742E-2,1.9541252E-2,9.224725E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3699039E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,24,24],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3509795E3,4.1964554E-4,3.9858868E1,3.3109E4,8.63558E-1,1.0275923E-2,-2.137881E-3,-2.296439E-3,-6.865453E-3,3.1982497E5,1.020707E1,2.0699982E8,2.8399336E7,1.06993E-1,1.6E1,1E0,3.2736E4,1.688E3,-2.0416873E-4,-6.3188877E-3,-1.2646198E-3,1.8932971E-3,-1.4567714E-3,-3.0581884E-3,1.6872E5,8.38519E-4,3.728294E-3,2.2793598E-4,6.773822E-3,8.1962877E-4,-1.2967597E-3],"split_indices":[52,42,57,32,27,0,0,0,0,28,54,12,45,38,8,8,9,2,0,0,0,0,0,0,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,8.9E1,1.2E1,1.5E1,7.4E1,8E0,4E0,5E0,1E1,5.4E1,2E1,3.5E1,1.9E1,1E1,1E1,2.1E1,1.4E1,1.3E1,6E0,5E0,5E0,5E0,5E0,7E0,1.4E1,1E1,4E0,4E0,9E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.084436E-4,-3.0529765E-2,1.429765E-1,-1.0845123E-2,-1.2839371E-2,9.87389E-2,8.60141E-3,4.3266765E-3,-9.543071E-3,2.2865399E-3,4.8879883E-3,-1.3061572E-2,8.913715E-3,-1.0353696E-1,5.561586E-3,-1.5558228E-3,-4.6346034E-3,-1.9769596E-2,6.4358704E-2,2.68207E-3,-1.3542122E-3,4.7533456E-3,3.4518127E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,-1,13,-1,15,17,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[4.4434074E-1,3.876115E-1,5.233702E-2,0E0,3.3787614E-1,9.59219E-3,0E0,3.1534722E-1,0E0,0E0,0E0,1.1852147E-1,0E0,1.2480959E-2,8.914697E-2,0E0,0E0,7.286841E-2,6.512507E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,11,11,13,13,14,14,17,17,18,18],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,-1,14,-1,16,18,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.7123758E0,5.279518E7,9.683623E6,-1.0845123E-2,1.5081978E8,3.1346828E5,8.60141E-3,1.3127055E4,-9.543071E-3,2.2865399E-3,4.8879883E-3,6.7281544E-5,8.913715E-3,1.6278E4,1.1392E4,-1.5558228E-3,-4.6346034E-3,2.0477528E1,2.483304E0,2.68207E-3,-1.3542122E-3,4.7533456E-3,3.4518127E-4],"split_indices":[39,5,1,0,45,28,0,4,0,0,0,38,0,12,2,0,0,58,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,8.1E1,1.7E1,4E0,7.7E1,1.3E1,4E0,7.3E1,4E0,8E0,5E0,6.9E1,4E0,1.1E1,5.8E1,4E0,7E0,4.1E1,1.7E1,6E0,3.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.2409871E-2,-5.640977E-2,1.13069564E-1,-1.3070142E-1,9.130545E-3,5.9516463E-2,2.403144E-1,-9.858008E-3,-9.282424E-2,5.1835133E-3,-1.718563E-2,1.0855935E-1,-4.8438333E-2,1.14710005E-2,4.641702E-3,-1.2477392E-3,-1.167992E-1,-4.7894478E-2,5.2092396E-2,1.3644609E-1,2.4571328E-4,-3.8370572E-3,-7.3106407E-6,-4.8090126E-3,-1.7152494E-3,-6.056952E-2,2.640781E-4,3.30507E-3,-2.0470054E-4,5.9417128E-3,2.913899E-3,-6.979562E-4,-2.8729707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,19,21,-1,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.683883E-1,3.2255393E-1,2.9234338E-1,1.4180148E-1,1.2983838E-1,1.7997411E-1,7.1611345E-2,0E0,3.228198E-2,0E0,6.844584E-2,6.4335525E-2,3.0669386E-2,0E0,0E0,0E0,1.4129981E-2,1.5838135E-2,2.330812E-2,1.4938563E-2,0E0,0E0,0E0,0E0,0E0,1.2570299E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,11,11,12,12,16,16,17,17,18,18,19,19,25,25],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,20,22,-1,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,1.04856936E8,1.1803382E4,5.279518E7,1.8955729E1,6.19999E9,1.0889231E1,-9.858008E-3,3.566468E5,5.1835133E-3,4.664004E8,7.5305285E6,2.074635E-1,1.14710005E-2,4.641702E-3,-1.2477392E-3,8.0168776E-2,7.879123E7,8.63558E-1,1.343964E1,2.4571328E-4,-3.8370572E-3,-7.3106407E-6,-4.8090126E-3,-1.7152494E-3,2.1821466E5,2.640781E-4,3.30507E-3,-2.0470054E-4,5.9417128E-3,2.913899E-3,-6.979562E-4,-2.8729707E-3],"split_indices":[2,7,4,5,58,5,54,0,28,0,12,47,39,0,0,0,57,32,27,54,0,0,0,0,0,28,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,6.5E1,4.4E1,3E1,3.5E1,3.2E1,1.2E1,5E0,2.5E1,5E0,3E1,2.2E1,1E1,6E0,6E0,8E0,1.7E1,2.1E1,9E0,1.7E1,5E0,4E0,6E0,1.3E1,4E0,1.7E1,4E0,5E0,4E0,1E1,7E0,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.3187193E-2,-3.4000084E-2,1.1714888E-1,-5.2852556E-2,8.351912E-2,1.09519195E-2,7.9743795E-2,-7.7567585E-3,-4.0347572E-2,4.412856E-3,1.761286E-4,1.5121083E-1,1.9125398E-2,-9.946081E-3,-1.1433393E-1,2.8071261E-3,7.4288077E-3,-2.3928876E-3,5.4361384E-2,-2.4444059E-2,4.5282017E-3,-5.2205385E-3,-1.3526303E-3,2.845917E-3,3.1054555E-4,1.2113955E-2,-5.061873E-2,-8.5555745E-4,1.8158134E-3,-3.2026474E-3,-1.18605E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,-1,15,17,19,21,-1,-1,-1,23,25,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1],"loss_changes":[5.9756905E-1,1.905185E-1,2.404775E-1,1.3921364E-1,3.5934314E-2,0E0,1.4007121E-1,0E0,1.5406421E-1,0E0,0E0,4.411444E-2,6.078022E-2,1.0119589E-1,4.111421E-2,0E0,0E0,0E0,1.499597E-2,4.463032E-2,0E0,0E0,0E0,0E0,0E0,2.853866E-2,1.535771E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,11,11,12,12,13,13,14,14,18,18,19,19,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,-1,16,18,20,22,-1,-1,-1,24,26,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.8319664E0,1.6451234E-2,5.1114212E7,9.5032835E0,1.09519195E-2,1.5E1,-7.7567585E-3,5.501383E9,4.412856E-3,1.761286E-4,8.220836E8,1.9649E4,1E0,2.711196E3,2.8071261E-3,7.4288077E-3,-2.3928876E-3,8.106617E-1,1.4163358E7,4.5282017E-3,-5.2205385E-3,-1.3526303E-3,2.845917E-3,3.1054555E-4,1.4243386E1,7.48E2,-8.5555745E-4,1.8158134E-3,-3.2026474E-3,-1.18605E-3],"split_indices":[2,39,57,5,53,0,8,0,5,0,0,7,2,94,52,0,0,0,27,45,0,0,0,0,0,56,11,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.2E2,8.3E1,3.7E1,7.2E1,1.1E1,5E0,3.2E1,4E0,6.8E1,7E0,4E0,1.4E1,1.8E1,4.9E1,1.9E1,7E0,7E0,5E0,1.3E1,4.5E1,4E0,1.3E1,6E0,8E0,5E0,1.9E1,2.6E1,1E1,9E0,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.138847E-2,-1.6537586E-2,1.378826E-1,1.3055443E-3,-1.792534E-1,2.6409472E-3,7.540905E-3,-3.0678686E-2,1.1850852E-1,-1.8756153E-3,-9.773239E-3,7.5826794E-3,-6.916983E-2,8.814884E-3,5.13998E-2,-2.657893E-2,5.8283474E-2,-9.322858E-2,-4.540672E-4,5.1580014E-4,3.3329213E-3,-2.841676E-3,-4.7905644E-4,3.9274837E-3,2.4581605E-4,-4.93551E-3,-6.0926672E-2,1.3738719E-3,-8.502662E-4,-5.129459E-4,-3.2671015E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[3.9085332E-1,2.6443976E-1,7.662693E-2,3.1477338E-1,9.0713054E-2,0E0,0E0,9.7724244E-2,1.3923103E-1,0E0,0E0,6.0437817E-2,4.3758735E-2,0E0,1.842231E-2,2.8535325E-2,3.5653424E-2,2.3972481E-2,0E0,0E0,0E0,0E0,1.4231859E-2,0E0,0E0,0E0,1.9434478E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,22,22,26,26],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[2.0935743E0,1.1384717E8,4.4993465E1,1.1855E4,9.546808E0,2.6409472E-3,7.540905E-3,2.8399336E7,1.4994E4,-1.8756153E-3,-9.773239E-3,2.4662777E1,2.5E-1,8.814884E-3,4.3E1,1.92E2,1.4922581E1,2.1750195E3,-4.540672E-4,5.1580014E-4,3.3329213E-3,-2.841676E-3,3.1E1,3.9274837E-3,2.4581605E-4,-4.93551E-3,4.8861527E9,1.3738719E-3,-8.502662E-4,-5.129459E-4,-3.2671015E-3],"split_indices":[42,45,56,2,53,0,0,45,10,0,0,56,57,0,8,0,54,4,0,0,0,0,3,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,9E1,1.9E1,8.2E1,8E0,1.1E1,8E0,6.5E1,1.7E1,4E0,4E0,3.3E1,3.2E1,5E0,1.2E1,2E1,1.3E1,2.2E1,1E1,7E0,5E0,6E0,1.4E1,6E0,7E0,8E0,1.4E1,5E0,9E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-4.325153E-3,-6.465679E-2,5.7140812E-2,3.4588529E-3,-8.501403E-2,7.3060505E-2,-4.4419733E-3,-6.85497E-2,-7.2139436E-3,4.150803E-2,1.5799184E-1,-1.1530733E-1,-3.8210128E-2,-1.877802E-3,7.709088E-2,2.2758744E-3,7.3556066E-3,-1.5388815E-1,-1.1804346E-3,6.839154E-4,-5.6428324E-2,5.9329014E-4,-2.0374889E-3,1.4138842E-4,9.313434E-2,-2.7908136E-3,-7.1321437E-3,-9.8254375E-2,-1.6696407E-2,4.0959646E-3,1.2126976E-3,-4.4327853E-3,-1.5981115E-3,8.26462E-4,-1.5993807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,15,17,19,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2644554E-1,1.9760542E-1,1.7248571E-1,0E0,8.458957E-2,1.3547117E-1,0E0,6.22686E-2,0E0,6.189643E-2,4.797116E-2,5.2338034E-2,3.2305997E-2,1.9908579E-2,2.5462687E-2,0E0,0E0,2.4856001E-2,0E0,0E0,3.6581583E-2,0E0,0E0,0E0,1.7965809E-2,0E0,0E0,9.131268E-3,1.5220466E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,20,20,24,24,27,27,28,28],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,16,18,20,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3724032E8,8.977316E0,2.4692087E1,3.4588529E-3,6.2640357E-1,1.5791555E9,-4.4419733E-3,5.048584E-4,-7.2139436E-3,9.8309404E-1,2.4190365E6,8.670369E-1,1.0465239E-3,2.2087644E9,1.601247E1,2.2758744E-3,7.3556066E-3,3.08E2,-1.1804346E-3,6.839154E-4,2.8900872E6,5.9329014E-4,-2.0374889E-3,1.4138842E-4,2.3559767E5,-2.7908136E-3,-7.1321437E-3,4E0,5.661451E-1,4.0959646E-3,1.2126976E-3,-4.4327853E-3,-1.5981115E-3,8.26462E-4,-1.5993807E-3],"split_indices":[7,54,54,0,38,7,0,42,0,39,32,27,38,12,56,0,0,0,0,0,50,0,0,0,33,0,0,8,27,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.7E1,5.6E1,6E0,5.1E1,5.2E1,4E0,4.6E1,5E0,3.9E1,1.3E1,1.7E1,2.9E1,1.8E1,2.1E1,5E0,8E0,1.1E1,6E0,7E0,2.2E1,1.4E1,4E0,4E0,1.7E1,5E0,6E0,1E1,1.2E1,1.2E1,5E0,6E0,4E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8995876E-3,1.7307267E-2,-8.49541E-3,-1.9665277E-2,1.0299962E-1,-5.0792895E-2,5.0558295E-2,9.836733E-3,6.644898E-2,-6.829946E-2,2.3107193E-3,7.184726E-2,1.5602639E-4,-6.355511E-4,9.254501E-2,-4.8872404E-2,-5.313775E-3,6.652992E-4,3.8057142E-3,1.2834902E-1,1.5851419E-3,-3.2241517E-3,-1.1584569E-3,5.7846876E-3,2.095763E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[3.61006E-1,3.156837E-1,0E0,1.5558049E-1,1.6771504E-1,1.0284625E-1,2.134474E-2,0E0,5.865871E-2,5.8703825E-2,0E0,2.4953179E-2,0E0,0E0,2.9194072E-2,2.1565579E-2,0E0,0E0,0E0,1.5350193E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,14,14,15,15,19,19],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.4783066E1,-8.49541E-3,7.307441E-1,1.0822511E-2,1E0,4.3E1,9.836733E-3,1.9956842E5,1E0,2.3107193E-3,5.3278846E-1,1.5602639E-4,-6.355511E-4,5.8639234E11,1E0,-5.313775E-3,6.652992E-4,3.8057142E-3,1.2271712E1,1.5851419E-3,-3.2241517E-3,-1.1584569E-3,5.7846876E-3,2.095763E-3],"split_indices":[45,56,0,39,57,94,3,0,33,89,0,27,0,0,31,8,0,0,0,54,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.8E1,5E0,6.9E1,2.9E1,4.8E1,2.1E1,4E0,2.5E1,4.2E1,6E0,1.4E1,7E0,6E0,1.9E1,3.5E1,7E0,6E0,8E0,1E1,9E0,9E0,2.6E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.299195E-3,-1.4052154E-2,1.7097886E-1,-7.453978E-2,3.1034399E-2,9.727058E-3,2.7355868E-3,-3.8559325E-2,-1.0469606E-2,8.58175E-2,-1.615867E-2,7.1115796E-3,-1.2693503E-1,2.869411E-3,1.344417E-1,-1.118793E-3,2.5316318E-3,4.2062886E-3,-2.4483189E-2,-1.6516506E-3,-5.701923E-3,2.0075378E-3,-1.2711645E-3,5.727348E-3,2.787558E-3,-6.625224E-2,3.2847042E-3,-3.2566807E-3,1.6193952E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[3.9381814E-1,2.7503112E-1,1.0143152E-1,3.221581E-1,1.513389E-1,0E0,0E0,1.5427823E-1,0E0,1.0851292E-1,4.3119304E-2,9.2717715E-2,2.5492847E-2,2.404739E-2,1.1293352E-2,0E0,0E0,0E0,1.0730756E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.751474E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,25,25],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[1.0825377E4,3.2316522E7,1.3309696E7,2.915E4,1.5046106E7,9.727058E-3,2.7355868E-3,4.352552E9,-1.0469606E-2,9.489487E2,3.9887E4,5.2287582E-2,3.240192E6,1.0875838E7,7.6363635E-1,-1.118793E-3,2.5316318E-3,4.2062886E-3,2.6666668E-1,-1.6516506E-3,-5.701923E-3,2.0075378E-3,-1.2711645E-3,5.727348E-3,2.787558E-3,1.1674918E3,3.2847042E-3,-3.2566807E-3,1.6193952E-4],"split_indices":[52,12,47,9,45,0,0,5,0,52,10,57,7,32,57,0,0,0,57,0,0,0,0,0,0,52,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,9.9E1,1.2E1,4.2E1,5.7E1,5E0,7E0,3.7E1,5E0,2.6E1,3.1E1,2.5E1,1.2E1,1E1,1.6E1,2.7E1,4E0,5E0,2E1,4E0,8E0,4E0,6E0,1E1,6E0,1.5E1,5E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.852963E-4,1.2467966E-2,-8.251737E-3,-7.0391726E-3,1.7803189E-1,-4.2995278E-2,5.208442E-2,9.643615E-3,2.1401076E-3,-5.983899E-2,3.2737432E-3,-4.583001E-4,7.522155E-2,-1.2944703E-2,-9.8201685E-2,-1.5455259E-3,2.433131E-3,9.761275E-2,1.33563E-4,-2.6628922E-3,1.014157E-2,-7.3017746E-2,-7.3823784E-3,4.3414882E-3,3.6716516E-4,2.971794E-3,-7.7672035E-4,-3.8003572E-3,-1.4499791E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2955632E-1,3.371398E-1,0E0,2.0206968E-1,1.00533515E-1,1.3826767E-1,4.379922E-2,0E0,0E0,9.407537E-2,0E0,3.7368078E-2,3.9675504E-2,3.584695E-2,6.268868E-2,0E0,0E0,3.6391586E-2,0E0,0E0,4.5179658E-2,2.225922E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,17,17,20,20,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.1730811E4,-8.251737E-3,6.402099E-1,8.50999E9,1E0,1.6511278E1,9.643615E-3,2.1401076E-3,7.059086E-1,3.2737432E-3,1.7618678E0,2.2606992E5,1.2179555E-1,1.4342E4,-1.5455259E-3,2.433131E-3,1.3021952E1,1.33563E-4,-2.6628922E-3,7.0843E5,8.0168776E-2,-7.3823784E-3,4.3414882E-3,3.6716516E-4,2.971794E-3,-7.7672035E-4,-3.8003572E-3,-1.4499791E-3],"split_indices":[45,4,0,42,5,94,56,0,0,27,0,39,33,27,9,0,0,53,0,0,1,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,1.03E2,5E0,9.3E1,1E1,5.8E1,3.5E1,5E0,5E0,5.2E1,6E0,1.1E1,2.4E1,2.4E1,2.8E1,7E0,4E0,1.8E1,6E0,6E0,1.8E1,2.4E1,4E0,1.4E1,4E0,5E0,1.3E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[6.055449E-4,-1.9225232E-2,2.0281893E-1,-6.364392E-2,4.574536E-2,9.911284E-3,2.7512023E-3,-4.881323E-2,-7.942117E-3,9.781974E-2,-1.525634E-3,-1.08574174E-1,-1.0936733E-2,2.1899366E-3,5.270318E-3,-1.218257E-3,3.6689226E-2,-4.635602E-2,-6.0139713E-3,4.6248944E-3,-3.2319885E-2,2.4503567E-3,-2.1939763E-5,-2.736184E-3,3.612752E-4,1.9101058E-3,-5.0062187E-2,-3.055991E-3,-2.2767717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[4.4901463E-1,2.9727405E-1,6.987399E-2,1.3069914E-1,1.0353677E-1,0E0,0E0,1.2783408E-1,0E0,2.6939541E-2,2.9556045E-2,7.167393E-2,1.1117197E-1,0E0,0E0,0E0,1.283419E-2,2.0665012E-2,0E0,0E0,5.103659E-2,0E0,0E0,0E0,0E0,0E0,4.032433E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,16,16,17,17,20,20,26,26],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[1.0825377E4,3.4707712E8,2.9548872E9,6.2640357E-1,1.5046106E7,9.911284E-3,2.7512023E-3,1.4616883E1,-7.942117E-3,8.5731603E8,1.8489E4,3.9024E4,1.7593515E1,2.1899366E-3,5.270318E-3,-1.218257E-3,8.106617E-1,5.348407E0,-6.0139713E-3,4.6248944E-3,4.5479352E2,2.4503567E-3,-2.1939763E-5,-2.736184E-3,3.612752E-4,1.9101058E-3,3.7E1,-3.055991E-3,-2.2767717E-4],"split_indices":[52,7,7,38,45,0,0,58,0,5,10,12,58,0,0,0,27,58,0,0,52,0,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.01E2,9E0,6E1,4.1E1,5E0,4E0,5.6E1,4E0,1.9E1,2.2E1,2.1E1,3.5E1,1.2E1,7E0,1.2E1,1E1,1.1E1,1E1,4E0,3.1E1,5E0,5E0,7E0,4E0,5E0,2.6E1,1.4E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.9022836E-3,1.2229664E-2,-7.96542E-3,-5.760372E-3,1.8077672E-1,-1.9990418E-2,1.02964014E-1,9.545216E-3,2.2026303E-3,-4.9231723E-2,1.21620875E-2,1.1601846E-3,6.0423976E-3,-3.2194313E-2,-4.5111333E-3,7.429765E-2,-7.5407457E-3,-4.606831E-2,2.075508E-3,4.518162E-3,3.1010047E-4,-2.3640944E-2,1.6765204E-3,-2.268631E-3,-4.11228E-4,5.4941524E-4,-1.2737332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,-1,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1],"loss_changes":[3.0562162E-1,2.8303292E-1,0E0,1.3356629E-1,7.39952E-2,7.205254E-2,4.023634E-2,0E0,0E0,4.7580205E-2,4.585738E-2,0E0,0E0,4.4953216E-2,0E0,2.8995778E-2,2.6300408E-2,1.6730115E-2,0E0,0E0,0E0,1.09398505E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,15,15,16,16,17,17,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,-1,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825377E4,-7.96542E-3,2.1729739E0,6.691E3,7.33E3,3.6742297E5,9.545216E-3,2.2026303E-3,7.925698E5,1.216644E11,1.1601846E-3,6.0423976E-3,1E0,-4.5111333E-3,8.155485E-2,5.2E1,1.4731E4,2.075508E-3,4.518162E-3,3.1010047E-4,6.983E3,1.6765204E-3,-2.268631E-3,-4.11228E-4,5.4941524E-4,-1.2737332E-3],"split_indices":[45,52,0,39,9,2,28,0,0,28,31,0,0,108,0,38,8,9,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,9.2E1,5E0,8.4E1,8E0,7.5E1,9E0,4E0,4E0,3.9E1,3.6E1,5E0,4E0,3.3E1,6E0,8E0,2.8E1,2.9E1,4E0,4E0,4E0,2.2E1,6E0,1.9E1,1E1,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-5.688046E-3,4.129763E-3,-7.962236E-3,-1.5068855E-2,1.6217422E-1,-1.3067493E-1,8.668473E-4,8.916432E-3,3.2096936E-3,-5.2778964E-4,-1.0478153E-2,-1.00261845E-1,1.9732645E-2,-4.993728E-3,-1.965056E-3,5.618472E-3,5.6606433E-3,-3.927957E-2,2.614031E-2,-2.6951663E-3,-7.7076635E-4,2.9834446E-3,2.5671237E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,-1,-1,-1,17,19,21,-1,-1,-1,-1],"loss_changes":[2.4138209E-1,3.3620965E-1,0E0,1.8249744E-1,5.2601337E-2,2.179641E-1,1.6987129E-1,0E0,0E0,0E0,0E0,1.6891032E-2,1.425382E-1,0E0,0E0,0E0,6.4531654E-2,1.2359679E-2,5.10181E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,16,16,17,17,18,18],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,-1,-1,-1,18,20,22,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.669409E3,-7.962236E-3,3.0894287E2,7.7854166E0,1.1485E4,4.1964554E-4,8.916432E-3,3.2096936E-3,-5.2778964E-4,-1.0478153E-2,5.4432504E7,1.82E2,-4.993728E-3,-1.965056E-3,5.618472E-3,1.1674918E3,8.834262E4,1.5046106E7,-2.6951663E-3,-7.7076635E-4,2.9834446E-3,2.5671237E-4],"split_indices":[45,52,0,52,54,9,42,0,0,0,0,45,10,0,0,0,52,33,45,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,1.09E2,4E0,9.8E1,1.1E1,1.1E1,8.7E1,4E0,7E0,7E0,4E0,1.3E1,7.4E1,6E0,7E0,6E0,6.8E1,2.1E1,4.7E1,6E0,1.5E1,1.1E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.5438551E-3,1.9281605E-2,-1.6711915E-1,4.5982033E-2,-6.162308E-2,-9.372038E-3,-8.2055957E-4,2.5376797E-2,9.690412E-3,-3.176471E-2,-6.0917297E-3,-6.6602975E-2,4.268207E-2,-6.465919E-2,9.26382E-4,6.842585E-4,-5.092631E-3,8.374747E-2,1.2439377E-2,-3.4085168E-3,-9.2678075E-4,3.679606E-2,5.263811E-3,3.8492754E-3,-3.6719176E-3,-4.4132138E-4,2.5573298E-3,-1.368757E-3,1.0346461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,27,-1,-1,-1,-1],"loss_changes":[3.1435934E-1,2.080516E-1,1.276218E-1,3.2317793E-1,7.139461E-2,0E0,0E0,1.0948901E-1,0E0,3.943786E-2,0E0,7.274461E-2,6.974814E-2,1.2656186E-2,0E0,0E0,0E0,6.4557835E-2,5.1981732E-2,0E0,0E0,2.5894187E-2,0E0,0E0,3.4296595E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,13,13,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,28,-1,-1,-1,-1],"split_conditions":[1E0,6.784095E7,3E0,1.0825377E4,1.13808104E8,-9.372038E-3,-8.2055957E-4,2.0676967E4,9.690412E-3,1.1855E4,-6.0917297E-3,3.9024E4,1.5046106E7,8.3722024E7,9.26382E-4,6.842585E-4,-5.092631E-3,3.0765435E10,4.795186E10,-3.4085168E-3,-9.2678075E-4,2.028302E-1,5.263811E-3,3.8492754E-3,7.94E2,-4.4132138E-4,2.5573298E-3,-1.368757E-3,1.0346461E-3],"split_indices":[112,45,8,52,45,0,0,48,0,2,0,12,45,45,0,0,0,31,31,0,0,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.4E1,9E0,7.1E1,2.3E1,5E0,4E0,6.6E1,5E0,1.9E1,4E0,1E1,5.6E1,1.2E1,7E0,5E0,5E0,2.3E1,3.3E1,6E0,6E0,1.4E1,9E0,4E0,2.9E1,6E0,8E0,1.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-9.706806E-3,6.6160626E-4,-8.590499E-3,-2.2892008E-2,1.1591553E-1,-6.606376E-2,1.8957838E-2,8.783608E-3,1.9151013E-3,-4.0266957E-2,-9.731352E-3,-3.6894733E-3,7.422198E-2,1.895479E-2,-8.547305E-2,-2.3264242E-3,6.2277764E-3,4.1628326E-3,-8.226836E-4,-1.7624825E-2,4.5093615E-3,-1.0041023E-1,-1.2582879E-3,1.7482936E-3,-1.6172178E-2,-1.864533E-3,1.5167823E-3,-1.5103664E-3,-4.3222914E-3,-1.6289857E-3,5.5052835E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,23,-1,-1,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7230027E-1,3.039793E-1,0E0,1.6927843E-1,1.3925076E-1,2.3060688E-1,6.034989E-2,0E0,0E0,1.14061855E-1,0E0,2.1487534E-2,5.8422536E-2,7.583651E-2,1.4647439E-2,0E0,3.0084949E-2,0E0,0E0,3.260273E-2,0E0,1.5351996E-2,0E0,0E0,1.0827113E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,16,16,19,19,21,21,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,24,-1,-1,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.8027E4,-8.590499E-3,3.3209075E6,1.6451234E-2,7.650722E4,3.9219662E5,8.783608E-3,1.9151013E-3,3.925844E7,-9.731352E-3,4.475326E6,2.1212247E5,3E0,1.5225641E1,-2.3264242E-3,1.3710804E7,4.1628326E-3,-8.226836E-4,7.3434E4,4.5093615E-3,2.1129233E-1,-1.2582879E-3,1.7482936E-3,1.4342E4,-1.864533E-3,1.5167823E-3,-1.5103664E-3,-4.3222914E-3,-1.6289857E-3,5.5052835E-5],"split_indices":[5,2,0,50,57,33,28,0,0,45,0,45,33,8,54,0,45,0,0,29,0,27,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,1.1E2,4E0,9.2E1,1.8E1,4.5E1,4.7E1,5E0,1.3E1,4.1E1,4E0,3.4E1,1.3E1,1.8E1,2.3E1,4E0,3E1,9E0,4E0,1.4E1,4E0,1.7E1,6E0,1E1,2E1,9E0,5E0,5E0,1.2E1,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.736818E-3,4.113295E-3,-8.342385E-3,-1.0323528E-2,6.449642E-3,-7.903802E-2,1.0886561E-2,-2.5692797E-4,-9.862084E-2,-2.0650776E-2,5.631539E-2,-1.4796215E-3,-4.247032E-3,-4.948225E-2,1.4655777E-2,9.341702E-2,4.5907656E-3,-1.7335454E-4,-2.2867348E-3,-5.0949603E-3,2.342427E-3,6.286833E-4,1.2996759E-1,3.3477068E-2,-1.9531718E-3,-1.018831E-3,1.3773413E-3,6.3288775E-3,2.6190532E-3,2.1420955E-3,-1.5326959E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6382893E-1,2.6389223E-1,0E0,1.4223216E-1,0E0,3.169197E-2,1.0861577E-1,0E0,1.4907181E-2,4.6532497E-2,5.9062257E-2,0E0,0E0,1.5174277E-2,2.1162583E-2,4.774022E-2,2.5546132E-2,0E0,0E0,1.7077478E-2,0E0,0E0,1.8521264E-2,1.00053325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,19,19,22,22,23,23],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.1811385E4,-8.342385E-3,1.9121604E-4,6.449642E-3,1.56E2,3.290356E5,-2.5692797E-4,4.04E3,4.0291533E8,9.580646E0,-1.4796215E-3,-4.247032E-3,2.9777256E-3,2.3E1,1.6984678E3,1.6558E4,-1.7335454E-4,-2.2867348E-3,1.218666E0,2.342427E-3,6.286833E-4,1.4117E4,8.5154843E-1,-1.9531718E-3,-1.018831E-3,1.3773413E-3,6.3288775E-3,2.6190532E-3,2.1420955E-3,-1.5326959E-4],"split_indices":[5,52,0,38,0,0,28,0,2,7,53,0,0,38,8,4,9,0,0,39,0,0,9,27,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,1.03E2,4E0,9.6E1,7E0,2.2E1,7.4E1,5E0,1.7E1,4.4E1,3E1,5E0,1.2E1,2.4E1,2E1,1.7E1,1.3E1,6E0,1.8E1,1.5E1,5E0,6E0,1.1E1,9E0,4E0,1E1,5E0,5E0,6E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5940958E-3,-4.6554036E-2,6.830869E-2,-7.2021233E-3,-2.624325E-2,1.6035995E-1,3.6518317E-2,6.065297E-5,-1.11229755E-1,1.3220954E-3,8.153393E-3,-1.1043287E-2,7.03636E-2,-4.967899E-2,2.3270464E-2,-1.2433941E-3,-5.1097213E-3,1.3591817E-3,-1.4135528E-3,9.0586595E-2,2.9167102E-4,-3.290976E-4,-2.5705846E-3,7.671628E-2,-2.2112587E-2,4.0585003E-3,8.8264357E-4,4.894953E-3,1.4300863E-3,-2.1808026E-3,1.5399962E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,7,9,11,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3724707E-1,1.8424381E-1,1.22903034E-1,0E0,1.2295798E-1,8.0853134E-2,5.5508073E-2,5.0797306E-2,2.5375023E-2,0E0,0E0,2.2004453E-2,2.4459846E-2,1.15733E-2,7.446243E-2,0E0,0E0,0E0,0E0,1.8906705E-2,0E0,0E0,0E0,2.5208868E-2,1.7429084E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,19,19,23,23,24,24],"right_children":[2,4,6,-1,8,10,12,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.108166E3,9.687E3,1.6451234E-2,-7.2021233E-3,4.95E9,7.618855E1,1E0,1.1301075E1,2.7782312E1,1.3220954E-3,8.153393E-3,2.1486234E5,9.780346E9,1.433458E10,9.3152986E8,-1.2433941E-3,-5.1097213E-3,1.3591817E-3,-1.4135528E-3,1.0155914E1,2.9167102E-4,-3.290976E-4,-2.5705846E-3,5.83E2,2E0,4.0585003E-3,8.8264357E-4,4.894953E-3,1.4300863E-3,-2.1808026E-3,1.5399962E-4],"split_indices":[52,29,57,0,5,56,109,56,56,0,0,28,5,31,5,0,0,0,0,53,0,0,0,10,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,6E1,4.3E1,6E0,5.4E1,1E1,3.3E1,4.2E1,1.2E1,4E0,6E0,1.4E1,1.9E1,1.3E1,2.9E1,4E0,8E0,5E0,9E0,1.4E1,5E0,5E0,8E0,1.3E1,1.6E1,1E1,4E0,4E0,9E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5897438E-4,-1.3053351E-2,6.461985E-3,-7.5818904E-2,2.809792E-3,-4.5627163E-4,-1.0400714E-1,-3.512621E-2,2.3239039E-2,-1.1999704E-3,-4.757195E-3,-4.7030836E-2,9.625173E-4,7.5231716E-2,-5.824817E-3,-8.361244E-4,-3.548405E-3,4.533259E-3,3.952262E-4,4.6399564E-2,-3.7876822E-2,-8.6584355E-4,2.9007045E-3,-1.1877371E-2,-3.6426443E-3,-2.1551596E-3,5.5181986E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,7,-1,9,11,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1],"loss_changes":[2.4026561E-1,9.3804985E-2,0E0,3.252372E-2,5.97166E-2,0E0,2.1159917E-2,2.1113586E-2,7.59936E-2,0E0,0E0,2.5864813E-2,0E0,5.6866713E-2,5.699604E-2,0E0,0E0,0E0,0E0,3.298611E-2,3.320931E-2,0E0,0E0,2.1693436E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,19,19,20,20,23,23],"right_children":[2,4,-1,6,8,-1,10,12,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1],"split_conditions":[1.0825377E4,4.1964554E-4,6.461985E-3,4.8861527E9,1.6511278E1,-4.5627163E-4,2.9916666E1,1.1789883E0,2.0752128E7,-1.1999704E-3,-4.757195E-3,2.6666668E-1,9.625173E-4,1.2320755E1,2.0086452E5,-8.361244E-4,-3.548405E-3,4.533259E-3,3.952262E-4,1.1653E4,7.331148E7,-8.6584355E-4,2.9007045E-3,1E0,-3.6426443E-3,-2.1551596E-3,5.5181986E-4],"split_indices":[52,42,0,5,56,0,56,57,45,0,0,57,0,54,28,0,0,0,0,2,45,0,0,109,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,9.3E1,6E0,1.8E1,7.5E1,6E0,1.2E1,2.6E1,4.9E1,4E0,8E0,2.2E1,4E0,1.7E1,3.2E1,1.6E1,6E0,9E0,8E0,1.2E1,2E1,4E0,8E0,1.5E1,5E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-7.199455E-3,1.0465599E-2,-1.6016547E-1,3.932468E-2,-5.875473E-2,-8.316888E-3,-1.584643E-3,1.6268708E-2,7.876037E-3,-1.6845651E-2,-6.1411713E-3,3.0123955E-2,-4.315039E-2,-4.513322E-2,1.5212361E-3,3.5156552E-3,1.1132896E-2,-2.4072235E-4,-3.0902775E-3,-2.699677E-3,-4.8931065E-4,-2.5096811E-2,4.3182988E-2,-1.6545515E-3,8.6665107E-4,3.6377704E-3,8.1199093E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,19,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[2.610829E-1,1.7610475E-1,7.151261E-2,2.52083E-1,1.1846192E-1,0E0,0E0,4.7402777E-2,0E0,3.5270683E-2,0E0,5.7668686E-2,1.6463183E-2,1.2092693E-2,0E0,0E0,4.4046875E-2,0E0,0E0,0E0,0E0,1.942357E-2,2.1561876E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,13,13,16,16,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,20,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[1E0,6.9754904E7,3.9E1,8.974092E3,1.1245455E1,-8.316888E-3,-1.584643E-3,4.5843E7,7.876037E-3,1.1855E4,-6.1411713E-3,1.048576E8,6.25379E5,7.396E3,1.5212361E-3,3.5156552E-3,4.034396E8,-2.4072235E-4,-3.0902775E-3,-2.699677E-3,-4.8931065E-4,2.31308E5,1.3047E4,-1.6545515E-3,8.6665107E-4,3.6377704E-3,8.1199093E-4],"split_indices":[112,45,3,4,53,0,0,45,0,2,0,7,29,9,0,0,7,0,0,0,0,29,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E1,8.6E1,9E0,6.1E1,2.5E1,5E0,4E0,5.5E1,6E0,1.9E1,6E0,4.5E1,1E1,1.3E1,6E0,9E0,3.6E1,6E0,4E0,6E0,7E0,1.7E1,1.9E1,1.2E1,5E0,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.273087E-3,-7.470429E-3,7.443775E-3,3.6390252E-2,-5.0697304E-2,1.3581093E-2,1.6413313E-1,-2.5516236E-2,-6.191642E-3,5.562921E-2,-1.4437392E-2,8.375183E-3,2.2910745E-3,-5.694799E-2,3.046937E-2,-1.1643596E-2,1.1784481E-1,-2.5296456E-3,1.0377308E-2,-6.73674E-4,-2.9562493E-3,-6.682096E-4,2.390234E-3,9.113448E-4,-1.6820207E-3,1.8230348E-3,5.383556E-3,2.548729E-3,-1.5029071E-2,-1.8590839E-3,3.444982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,-1,3,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,29,-1,-1],"loss_changes":[2.9814023E-1,0E0,1.5163006E-1,1.7259336E-1,8.6881734E-2,6.206562E-2,4.6752453E-2,4.7488037E-2,0E0,8.926592E-2,4.490131E-2,0E0,0E0,1.4702745E-2,1.8728366E-2,1.5355412E-2,1.5637785E-2,0E0,3.624961E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5723072E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,18,18,28,28],"right_children":[2,-1,4,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,30,-1,-1],"split_conditions":[9.7888E4,-7.470429E-3,1.0504551E1,1.0825377E4,2.192772E10,2.0752128E7,8E0,7.287E3,-6.191642E-3,1.6511278E1,6.604E3,8.375183E-3,2.2910745E-3,1.09141096E8,9.877E3,3.1E1,3.3052456E5,-2.5296456E-3,1.7994973E11,-6.73674E-4,-2.9562493E-3,-6.682096E-4,2.390234E-3,9.113448E-4,-1.6820207E-3,1.8230348E-3,5.383556E-3,2.548729E-3,1.6226676E0,-1.8590839E-3,3.444982E-4],"split_indices":[1,0,53,52,5,45,54,2,0,56,2,0,0,7,9,3,28,0,31,0,0,0,0,0,0,0,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,6E0,8.8E1,5.9E1,2.9E1,5.1E1,8E0,2.5E1,4E0,2E1,3.1E1,4E0,4E0,1.6E1,9E0,1E1,1E1,9E0,2.2E1,7E0,9E0,4E0,5E0,5E0,5E0,4E0,6E0,6E0,1.6E1,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5544318E-2,8.2856574E-4,1.7569044E-1,2.3314485E-2,-5.16662E-2,8.75562E-3,2.6616931E-3,-1.423079E-2,5.7185426E-2,-9.980586E-2,-1.4028323E-2,3.732155E-2,-3.7265997E-2,8.990336E-2,1.340531E-2,-5.706309E-4,-4.8877364E-3,1.2460187E-3,-1.486541E-3,2.8164522E-3,-3.461824E-4,-3.1046993E-3,-5.865025E-4,1.3043243E-3,1.2722433E-1,3.3343535E-2,-1.48438E-3,5.527806E-3,2.1791419E-3,1.1838021E-4,2.4503965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,-1],"loss_changes":[2.5201818E-1,1.1806349E-1,4.3447137E-2,8.9832366E-2,5.2956127E-2,0E0,0E0,4.176647E-2,5.1793434E-2,3.6710635E-2,2.5140256E-2,2.1744628E-2,2.3186956E-2,3.7549675E-2,1.9850623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1530235E-2,1.327269E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,24,24,25,25],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,-1],"split_conditions":[1.0825377E4,4.3201213E9,1.055E3,1.1462246E3,3.0595828E3,8.75562E-3,2.6616931E-3,3.640909E6,2.0752128E7,2.4759493E1,5.9633E7,1.5662651E0,2.5975E4,3.6742297E5,1.4664377E-1,-5.706309E-4,-4.8877364E-3,1.2460187E-3,-1.486541E-3,2.8164522E-3,-3.461824E-4,-3.1046993E-3,-5.865025E-4,1.3043243E-3,7.408377E-1,2.7723257E3,-1.48438E-3,5.527806E-3,2.1791419E-3,1.1838021E-4,2.4503965E-3],"split_indices":[52,5,0,52,4,0,0,45,45,56,45,56,29,28,38,0,0,0,0,0,0,0,0,0,57,52,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,9.8E1,8E0,6.9E1,2.9E1,4E0,4E0,3.3E1,3.6E1,1.2E1,1.7E1,1E1,2.3E1,2E1,1.6E1,4E0,8E0,6E0,1.1E1,5E0,5E0,6E0,1.7E1,9E0,1.1E1,1.2E1,4E0,7E0,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.8123404E-3,-1.4526056E-1,5.97127E-3,1.5809156E-4,-9.597907E-3,1.5154353E-2,-6.6571943E-3,-6.059623E-4,1.2431084E-1,-9.94634E-2,1.5669389E-2,7.1533322E-3,1.181514E-3,-1.0294307E-3,-5.639434E-3,3.3276565E-2,-1.7589279E-2,-2.202198E-2,6.3199975E-2,-1.8199122E-3,1.361609E-3,-1.4930501E-3,1.4579241E-3,3.245211E-3,4.756501E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8983582E-1,1.6356577E-1,1.8984586E-1,0E0,0E0,1.7727041E-1,0E0,1.4797205E-1,8.141294E-2,4.7880545E-2,4.6847023E-2,0E0,0E0,0E0,0E0,8.743729E-2,5.2722573E-2,2.4593193E-2,4.4207484E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,10,10,15,15,16,16,17,17,18,18],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.477E3,2.16268E5,1.4974915E8,1.5809156E-4,-9.597907E-3,1.0825377E4,-6.6571943E-3,1.7145766E4,6.5566176E7,2.2367E4,3.925844E7,7.1533322E-3,1.181514E-3,-1.0294307E-3,-5.639434E-3,1.5493506E1,4.7308203E3,1.7618678E0,7.6363635E-1,-1.8199122E-3,1.361609E-3,-1.4930501E-3,1.4579241E-3,3.245211E-3,4.756501E-4],"split_indices":[2,12,45,0,0,52,0,51,45,32,45,0,0,0,0,56,4,39,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,8E0,1.06E2,4E0,4E0,1.02E2,4E0,9E1,1.2E1,1.2E1,7.8E1,6E0,6E0,6E0,6E0,5.1E1,2.7E1,1.8E1,3.3E1,1.7E1,1E1,1.4E1,4E0,2.1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.3294464E-2,-1.6678155E-4,-6.107358E-3,-6.838803E-3,1.0120793E-2,-2.4004694E-2,4.6296407E-2,-3.7772857E-2,3.6809193E-3,3.556834E-2,4.5131375E-3,-9.956454E-3,-9.088747E-2,5.878632E-2,1.9912338E-2,-3.7122454E-2,5.7555597E-2,-1.6427932E-3,-4.919216E-3,3.9683143E-3,6.1795994E-4,1.964694E-3,-1.5570889E-3,-2.056355E-3,-2.7524898E-4,3.552975E-5,3.7047437E-3,-1.0763321E-3,7.6549914E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,-1,-1,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1073212E-1,1.8882969E-1,0E0,0E0,1.146863E-1,8.7095074E-2,3.3552386E-2,6.410319E-2,0E0,1.3732959E-2,0E0,5.715758E-2,2.468478E-2,3.220789E-2,1.93466E-2,1.2438357E-2,2.2841193E-2,0E0,0E0,0E0,0E0,0E0,1.1564974E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,22,22],"right_children":[2,4,-1,-1,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2511909E8,2.7803582E6,-6.107358E-3,-6.838803E-3,3.4288502E8,3.7249353E3,1.2668315E6,2.8225484E11,3.6809193E-3,4.760274E-1,4.5131375E-3,4.899102E5,7.444956E7,7E0,2.2059325E5,4.757E3,1.68E2,-1.6427932E-3,-4.919216E-3,3.9683143E-3,6.1795994E-4,1.964694E-3,9.904E3,-2.056355E-3,-2.7524898E-4,3.552975E-5,3.7047437E-3,-1.0763321E-3,7.6549914E-4],"split_indices":[45,45,0,0,7,52,28,31,0,57,0,28,7,8,28,2,0,0,0,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.5E1,7E0,4E0,9.1E1,4.7E1,4.4E1,4.3E1,4E0,4E1,4E0,2.9E1,1.4E1,1.5E1,2.5E1,2.1E1,8E0,8E0,6E0,6E0,9E0,9E0,1.6E1,1.2E1,9E0,4E0,4E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.3286694E-2,-6.1373953E-3,-1.2186766E-3,1.1916811E-2,-1.10121354E-1,-1.8787432E-3,1.2555566E-1,-7.114069E-3,-6.9779414E-4,-7.865443E-2,1.7031323E-2,7.1104984E-3,1.0487871E-3,-1.0030704E-3,-3.9526024E-3,3.926016E-3,6.900742E-3,-3.933339E-3,3.5323286E-3,9.4712945E-4,-5.797085E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,-1,17,19,-1,-1,-1],"loss_changes":[1.8512852E-1,0E0,1.3291734E-1,1.303559E-1,7.511633E-2,1.1022709E-1,5.6288615E-2,0E0,0E0,2.0481043E-2,5.5925384E-2,0E0,0E0,0E0,0E0,0E0,5.555865E-2,1.907408E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,16,16,17,17],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,-1,18,20,-1,-1,-1],"split_conditions":[2.7571955E6,-6.1373953E-3,1.1384717E8,1.0861106E4,2.7140412E5,2.0676967E4,2.5669595E5,-7.114069E-3,-6.9779414E-4,5.895365E5,4.80378E5,7.1104984E-3,1.0487871E-3,-1.0030704E-3,-3.9526024E-3,3.926016E-3,2.4241872E0,2.8327732E6,3.5323286E-3,9.4712945E-4,-5.797085E-4],"split_indices":[45,0,45,4,32,48,33,0,0,28,12,0,0,0,0,0,39,32,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,6E0,9.1E1,8.2E1,9E0,7.4E1,8E0,4E0,5E0,1.4E1,6E1,4E0,4E0,6E0,8E0,5E0,5.5E1,5E1,5E0,1.4E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.8605918E-3,-2.1736171E-2,8.437174E-2,-6.7620915E-3,-1.111673E-2,6.9409097E-3,5.3182837E-2,1.4328168E-2,-4.2614054E-2,7.405703E-2,-8.7553845E-4,-1.8179422E-2,5.976905E-2,-6.996603E-2,-7.9285465E-3,4.372196E-3,1.5041559E-3,9.2035875E-4,-2.9263632E-2,8.864982E-2,3.2408815E-4,-3.7530188E-3,-1.5765788E-3,7.9274306E-4,-1.3334267E-3,-2.1656025E-3,-3.945525E-4,4.22959E-3,1.466527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,-1,7,-1,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1016249E-1,1.4560623E-1,7.0528194E-2,0E0,6.472628E-2,0E0,3.4123387E-2,6.7559384E-2,3.3599094E-2,1.933889E-2,0E0,1.3815888E-2,2.7075693E-2,1.333572E-2,1.5547715E-2,0E0,0E0,0E0,1.1736214E-2,1.1669204E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,18,18,19,19],"right_children":[2,4,6,-1,8,-1,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3137E4,1.369606E6,1.6451234E-2,-6.7620915E-3,2.8399336E7,6.9409097E-3,9.3393946E-1,1.6987854E1,1.2163482E8,9.275501E-1,-8.7553845E-4,6.910683E0,9.3977806E1,8.49491E5,5.4357555E6,4.372196E-3,1.5041559E-3,9.2035875E-4,2.5975E4,8.70707E0,3.2408815E-4,-3.7530188E-3,-1.5765788E-3,7.9274306E-4,-1.3334267E-3,-2.1656025E-3,-3.945525E-4,4.22959E-3,1.466527E-3],"split_indices":[2,45,57,0,45,0,27,56,7,57,0,58,58,1,50,0,0,0,29,53,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,8.3E1,2.3E1,4E0,7.9E1,4E0,1.9E1,4.4E1,3.5E1,1.5E1,4E0,2.6E1,1.8E1,1.9E1,1.6E1,5E0,1E1,5E0,2.1E1,1.1E1,7E0,7E0,1.2E1,8E0,8E0,7E0,1.4E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-8.72379E-3,-3.2157023E-2,6.567417E-2,-5.4606968E-3,-1.8477479E-2,1.06017515E-1,-4.220538E-3,1.6758474E-2,-5.2261718E-2,1.3683248E-1,2.2292566E-4,3.2988226E-3,4.576324E-3,-7.992488E-2,-1.09585095E-2,1.6422248E-1,1.5787827E-3,2.2053518E-3,-1.0813563E-2,-3.8940748E-3,-9.960631E-4,2.36622E-4,-1.833925E-3,3.5757283E-3,7.6861647E-3,-1.7838805E-3,2.6325219E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.816805E-1,1.259106E-1,1.9653776E-1,0E0,8.658606E-2,6.3764766E-2,0E0,3.23044E-2,4.1399516E-2,3.361982E-2,0E0,0E0,2.8489737E-2,2.918826E-2,1.149574E-2,1.6984403E-2,0E0,0E0,1.1619508E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8355938E3,9.7888E4,1.2707124E1,-5.4606968E-3,2.8399336E7,9.299267E9,-4.220538E-3,1.82E2,1.1309524E-1,1.343964E1,2.2292566E-4,3.2988226E-3,4.3650795E-2,9.08958E5,2.4835809E-1,1.1681136E4,1.5787827E-3,2.2053518E-3,6.4637297E9,-3.8940748E-3,-9.960631E-4,2.36622E-4,-1.833925E-3,3.5757283E-3,7.6861647E-3,-1.7838805E-3,2.6325219E-5],"split_indices":[52,1,53,0,45,12,0,10,57,54,0,0,57,29,41,4,0,0,31,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,7.8E1,2.4E1,7E0,7.1E1,2E1,4E0,3.5E1,3.6E1,1.5E1,5E0,4E0,3.1E1,2.1E1,1.5E1,1.1E1,4E0,6E0,2.5E1,1.3E1,8E0,1.1E1,4E0,6E0,5E0,5E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.9045455E-4,1.1739794E-2,-5.0104256E-3,2.1063041E-2,-6.0562724E-3,7.0360657E-3,1.2753847E-1,-2.1995079E-2,4.3521825E-2,7.092836E-3,1.348787E-3,2.283017E-2,-4.3062016E-2,1.8608687E-2,7.661435E-2,-1.0269417E-2,3.5273766E-3,-3.9992332E-3,-2.990283E-2,2.1972363E-3,3.4910275E-3,3.8514629E-3,3.1500537E-4,-1.3301228E-3,1.2383802E-3,-2.3676392E-4,-2.4372146E-3,1.796641E-3,-4.7961052E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6770124E-1,1.6920248E-1,0E0,1.4098781E-1,0E0,9.099125E-2,6.102948E-2,4.631099E-2,2.9902972E-2,0E0,0E0,4.065559E-2,2.609123E-2,1.4478652E-2,3.1727165E-2,1.5476296E-2,0E0,0E0,2.5015475E-2,0E0,1.4633759E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,18,18,20,20],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.5081978E8,-5.0104256E-3,1.1335518E4,-6.0562724E-3,5.8585525E-1,1.3477259E7,4.795186E10,3.1346828E5,7.092836E-3,1.348787E-3,9.144343E0,3.411305E2,1.6988876E-1,1.343964E1,1.0210356E1,3.5273766E-3,-3.9992332E-3,2.119303E6,2.1972363E-3,1.2606002E-1,3.8514629E-3,3.1500537E-4,-1.3301228E-3,1.2383802E-3,-2.3676392E-4,-2.4372146E-3,1.796641E-3,-4.7961052E-4],"split_indices":[112,45,0,4,0,42,47,31,28,0,0,56,33,38,54,53,0,0,32,0,27,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.8E1,7E0,9.4E1,4E0,8.4E1,1E1,4.7E1,3.7E1,5E0,5E0,1.5E1,3.2E1,2.2E1,1.5E1,1.1E1,4E0,4E0,2.8E1,5E0,1.7E1,1E1,5E0,7E0,4E0,1.8E1,1E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.4349855E-4,8.81773E-3,-5.9871273E-3,-2.0220012E-3,1.1308135E-1,-5.3416163E-2,1.1181911E-2,6.2670093E-3,6.447404E-4,7.6067676E-3,-4.159057E-3,7.13886E-2,2.2942477E-3,1.6861771E-3,-7.7285536E-4,-1.6962165E-4,4.4369227E-3,-2.8251627E-2,2.142086E-2,6.010141E-4,-3.92126E-2,5.6421056E-2,4.3734084E-3,-8.103488E-4,-2.5970933E-3,3.7789282E-3,6.493335E-4,5.017618E-4,-1.6618022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,-1,-1,-1,-1,19,21,-1,23,25,27,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7321569E-1,1.2002874E-1,0E0,6.642188E-2,5.4515243E-2,7.628062E-2,4.1573767E-2,0E0,0E0,1.3477924E-2,0E0,3.976794E-2,4.0917672E-2,0E0,0E0,0E0,0E0,1.3940422E-2,2.5409807E-2,0E0,1.0456149E-2,2.408043E-2,1.5041824E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,-1,-1,-1,-1,20,22,-1,24,26,28,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.3127055E4,-5.9871273E-3,1.1112019E3,8.50999E9,6.8917E4,1.53E2,6.2670093E-3,6.447404E-4,5.625738E2,-4.159057E-3,6.277045E0,2.2781514E8,1.6861771E-3,-7.7285536E-4,-1.6962165E-4,4.4369227E-3,8.1980075E5,4.760274E-1,6.010141E-4,8.63558E-1,7.84E3,9.014471E-1,-8.103488E-4,-2.5970933E-3,3.7789282E-3,6.493335E-4,5.017618E-4,-1.6618022E-3],"split_indices":[45,4,0,33,5,12,10,0,0,33,0,58,7,0,0,0,0,32,57,0,27,9,27,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.05E2,5E0,9.6E1,9E0,1.9E1,7.7E1,5E0,4E0,1E1,9E0,9E0,6.8E1,4E0,6E0,4E0,5E0,2.6E1,4.2E1,5E0,2.1E1,1.3E1,2.9E1,1.5E1,6E0,5E0,8E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.2598174E-3,1.4670459E-2,-1.386452E-1,2.5006693E-2,-4.663763E-3,-9.339003E-3,3.2908953E-4,1.2198944E-2,1.4745949E-1,-3.9475158E-2,2.7000148E-2,7.674119E-3,1.9085946E-3,-6.622676E-2,7.1005063E-4,5.454945E-3,6.2422793E-2,-7.4367505E-4,-3.2235263E-3,2.7724443E-2,-3.154511E-2,1.01069845E-1,-9.519478E-3,-7.5294264E-4,1.41495E-3,-1.7789672E-3,2.9685492E-5,4.9565737E-3,1.0727721E-3,-1.783428E-3,9.133871E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,-1,17,-1,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9439337E-1,1.5489066E-1,1.619811E-1,1.4876224E-1,0E0,0E0,0E0,6.837165E-2,4.4187605E-2,3.294565E-2,5.2319463E-2,0E0,0E0,1.2891479E-2,0E0,3.7129365E-2,7.325141E-2,0E0,0E0,1.6415996E-2,9.79618E-3,3.9857015E-2,1.5135152E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,-1,18,-1,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,2.8841052E10,2.667836E6,1.2768678E4,-4.663763E-3,-9.339003E-3,3.2908953E-4,1E0,9.9182825E0,4.7E1,3.290356E5,7.674119E-3,1.9085946E-3,1.5285803E5,7.1005063E-4,2.3365998E5,2.0752128E7,-7.4367505E-4,-3.2235263E-3,2.0511957E-1,7.22E2,9.275501E-1,3.598686E1,-7.5294264E-4,1.41495E-3,-1.7789672E-3,2.9685492E-5,4.9565737E-3,1.0727721E-3,-1.783428E-3,9.133871E-4],"split_indices":[112,5,32,52,0,0,0,16,54,3,28,0,0,28,0,28,45,0,0,39,0,57,56,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,1.01E2,8E0,9.5E1,6E0,4E0,4E0,8.7E1,8E0,1.9E1,6.8E1,4E0,4E0,1.3E1,6E0,4.3E1,2.5E1,5E0,8E0,2.7E1,1.6E1,1.6E1,9E0,5E0,2.2E1,1E1,6E0,1E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.5288215E-3,-6.140324E-3,8.652144E-3,1.9167883E-2,-4.613466E-3,4.772793E-3,1.3385132E-1,-6.674228E-2,1.9426296E-2,7.069505E-3,2.162803E-3,-3.366233E-3,-5.7304214E-4,7.623323E-2,1.0581485E-2,7.9137104E-4,4.1626757E-3,-2.1845449E-2,2.6141416E-2,-1.498703E-3,7.44017E-4,2.4439825E-3,1.5997738E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,15,17,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[2.0840193E-1,0E0,1.4018543E-1,1.4794022E-1,0E0,8.618021E-2,3.40883E-2,1.7390944E-2,3.3359714E-2,0E0,0E0,0E0,0E0,1.6569298E-2,3.0844722E-2,0E0,0E0,1.7553702E-2,3.6687523E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,17,17,18,18],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,16,18,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.477E3,-6.140324E-3,1.12782936E8,1.0825377E4,-4.613466E-3,5.981675E-4,4.2123712E5,2.2196926E-2,2.1923357E1,7.069505E-3,2.162803E-3,-3.366233E-3,-5.7304214E-4,3.2334878E7,1.1674918E3,7.9137104E-4,4.1626757E-3,5.407E3,9.275501E-1,-1.498703E-3,7.44017E-4,2.4439825E-3,1.5997738E-4],"split_indices":[2,0,45,52,0,42,28,57,58,0,0,0,0,12,52,0,0,2,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,6E0,9.5E1,8.9E1,6E0,8E1,9E0,1.3E1,6.7E1,4E0,5E0,8E0,5E0,8E0,5.9E1,4E0,4E0,1.9E1,4E1,1.3E1,6E0,1.3E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.739652E-3,6.9263284E-3,-4.405061E-3,-3.8693522E-3,4.7010025E-3,1.3269627E-3,-3.413561E-3,3.2038175E-2,-1.4311399E-2,2.8845035E-3,6.994032E-2,-2.5837254E-2,3.2984633E-2,-9.994091E-4,1.6085008E-3,4.070153E-3,3.83931E-2,-8.252682E-3,-1.6695434E-3,2.4891414E-3,-1.1246001E-3,-3.9452192E-4,2.8895128E-3,-9.2690036E-4,1.0629499E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,-1,5,-1,7,-1,9,11,13,15,17,19,-1,-1,-1,21,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2305258E-1,1.3226694E-1,0E0,4.3476637E-2,0E0,4.272254E-2,0E0,3.294786E-2,3.3014737E-2,2.4159394E-2,1.3223283E-2,1.6842786E-2,2.9823627E-2,0E0,0E0,0E0,1.9359013E-2,1.8486392E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,16,16,17,17],"right_children":[2,4,-1,6,-1,8,-1,10,12,14,16,18,20,-1,-1,-1,22,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8799975E10,1.0825377E4,-4.405061E-3,1.5088414E6,4.7010025E-3,1.5046106E7,-3.413561E-3,3.6742297E5,1.8027E4,2.0699982E8,1.82E2,1.7994973E11,2.4494735E9,-9.994091E-4,1.6085008E-3,4.070153E-3,3.97E2,5.3433334E1,-1.6695434E-3,2.4891414E-3,-1.1246001E-3,-3.9452192E-4,2.8895128E-3,-9.2690036E-4,1.0629499E-3],"split_indices":[5,52,0,28,0,45,0,28,2,12,10,31,7,0,0,0,0,56,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,9.8E1,7E0,9.1E1,7E0,8.7E1,4E0,2.9E1,5.8E1,1.7E1,1.2E1,4.7E1,1.1E1,1E1,7E0,4E0,8E0,2.6E1,2.1E1,7E0,4E0,4E0,4E0,1.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.6483202E-3,4.0761293E-3,-5.454882E-3,-1.9531498E-2,5.437144E-2,-9.483714E-3,-5.0616013E-3,6.2591457E-3,3.4693863E-2,1.9309424E-2,-5.747635E-2,6.914881E-2,-6.5053966E-5,-9.307056E-3,4.9574755E-2,-1.7867592E-4,-6.602468E-2,9.292712E-2,3.6346307E-4,1.7277653E-3,-2.987749E-2,4.009477E-3,2.6326647E-2,-1.4866267E-3,-3.7136124E-3,1.150766E-3,4.313193E-3,-1.6008106E-3,7.0349715E-4,1.5688151E-3,-5.4768653E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,-1,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3728663E-1,1.3760753E-1,0E0,9.445317E-2,8.0623865E-2,1.03367366E-1,0E0,0E0,4.1620765E-2,4.1167498E-2,1.2245826E-2,2.2947013E-2,0E0,3.0826839E-2,2.9950999E-2,0E0,1.7731592E-2,1.5637465E-2,0E0,0E0,1.4428532E-2,0E0,1.3452238E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,20,20,22,22],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,-1,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,4.818997E6,-5.454882E-3,6.2640357E-1,1.4440433E-2,4.178728E7,-5.0616013E-3,6.2591457E-3,7.639958E-1,1.6987854E1,2.4557575E1,9.780346E9,-6.5053966E-5,1.2410928E8,2.85E2,-1.7867592E-4,4.10362E5,7.4684826E8,3.6346307E-4,1.7277653E-3,9E0,4.009477E-3,4.7E1,-1.4866267E-3,-3.7136124E-3,1.150766E-3,4.313193E-3,-1.6008106E-3,7.0349715E-4,1.5688151E-3,-5.4768653E-4],"split_indices":[5,1,0,38,57,45,0,0,27,56,56,5,0,5,0,0,28,7,0,0,8,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,1.14E2,5E0,7.8E1,3.6E1,7.3E1,5E0,4E0,3.2E1,4.6E1,2.7E1,1.6E1,1.6E1,2.4E1,2.2E1,4E0,2.3E1,1.1E1,5E0,6E0,1.8E1,5E0,1.7E1,1.5E1,8E0,4E0,7E0,1.4E1,4E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.1603726E-3,1.5485994E-2,-6.3458327E-3,4.0161577E-3,1.2863575E-1,3.0635163E-2,-2.789822E-2,6.486663E-3,1.5042772E-3,7.0876986E-2,1.6491812E-2,-4.182692E-2,1.5052001E-2,1.00328155E-1,-7.9453544E-5,5.359031E-3,3.530967E-3,-2.1188335E-2,-7.1271345E-2,1.8600136E-3,-6.6271616E-4,1.1476396E-3,5.0805463E-3,-6.4179447E-4,2.1220494E-2,-2.0126458E-3,-4.8533773E-5,-7.362675E-4,-3.5117695E-3,1.4871164E-3,-1.5560671E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7482221E-1,1.4094457E-1,0E0,8.580757E-2,3.6898896E-2,3.0134596E-2,2.8148647E-2,0E0,0E0,3.0351937E-2,3.749491E-2,1.9759078E-2,1.5565556E-2,2.3213856E-2,0E0,1.4321999E-2,0E0,1.5012425E-2,1.6535476E-2,0E0,0E0,0E0,0E0,0E0,1.1904246E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[6.313997E-1,1.1811385E4,-6.3458327E-3,3.925844E7,3.0376984E1,2.083117E1,1.9649E4,6.486663E-3,1.5042772E-3,1.82E2,1.2028319E6,7.752181E7,1.6944988E9,1.16E2,-7.9453544E-5,2.3514317E8,3.530967E-3,9.371747E0,7.056912E7,1.8600136E-3,-6.6271616E-4,1.1476396E-3,5.0805463E-3,-6.4179447E-4,1.5046106E7,-2.0126458E-3,-4.8533773E-5,-7.362675E-4,-3.5117695E-3,1.4871164E-3,-1.5560671E-4],"split_indices":[38,52,0,45,57,58,2,0,0,10,28,45,7,0,0,7,0,53,7,0,0,0,0,0,45,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,1.08E2,4E0,9.9E1,9E0,5.4E1,4.5E1,5E0,4E0,1.3E1,4.1E1,3.4E1,1.1E1,9E0,4E0,3.7E1,4E0,2.1E1,1.3E1,5E0,6E0,4E0,5E0,1.5E1,2.2E1,7E0,1.4E1,5E0,8E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-8.785858E-3,-1.0762257E-3,-5.7649054E-3,1.0205673E-2,-1.1502917E-1,-5.704144E-4,8.119578E-2,-8.205172E-3,7.299546E-4,2.520764E-2,-2.697866E-2,5.2220975E-3,5.4919295E-4,-9.181918E-3,6.1775774E-2,-3.428799E-2,1.1207894E-3,1.398447E-3,-1.2320307E-3,3.1725105E-3,9.7081985E-4,-4.029546E-2,3.9776257E-4,-2.159745E-3,-6.148451E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[1.1965152E-1,1.2844892E-1,0E0,6.966092E-2,1.4121953E-1,5.514033E-2,4.803879E-2,0E0,0E0,5.2127264E-2,1.8028772E-2,0E0,0E0,2.761657E-2,1.5845947E-2,1.0242112E-2,0E0,0E0,0E0,0E0,0E0,1.35182515E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,21,21],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.4974915E8,1E0,-5.7649054E-3,1.0825377E4,1.04856936E8,2.727403E7,5.4875424E7,-8.205172E-3,7.299546E-4,1.1462246E3,3.8E1,5.2220975E-3,5.4919295E-4,2.8327732E6,9.366E4,2.022832E1,1.1207894E-3,1.398447E-3,-1.2320307E-3,3.1725105E-3,9.7081985E-4,9.08958E5,3.9776257E-4,-2.159745E-3,-6.148451E-4],"split_indices":[45,112,0,52,7,45,45,0,0,52,8,0,0,32,29,54,0,0,0,0,0,29,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.8E1,4E0,9E1,8E0,7.9E1,1.1E1,4E0,4E0,4E1,3.9E1,5E0,6E0,2.1E1,1.9E1,3.5E1,4E0,7E0,1.4E1,1E1,9E0,3.1E1,4E0,1.6E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.8353552E-3,4.223751E-3,-5.5867573E-3,1.25166355E-2,-5.0125904E-3,9.2093274E-4,7.112307E-2,2.7651932E-2,-2.9744381E-2,6.007968E-3,2.9002529E-2,1.5225844E-2,3.1166228E-3,-5.4415897E-2,4.8034373E-3,1.8822147E-3,-6.655829E-4,5.8158778E-2,-3.0645356E-3,-7.410822E-2,-2.082319E-4,-4.501191E-4,1.8846401E-3,4.3006588E-4,4.35119E-3,-4.699623E-4,1.7081059E-3,-6.844289E-4,-3.4481182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,-1,19,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.20756365E-1,1.2987149E-1,0E0,6.993152E-2,0E0,7.213765E-2,6.3405104E-2,3.2952726E-2,3.520761E-2,0E0,1.5226593E-2,3.1588614E-2,0E0,2.256447E-2,1.5385991E-2,0E0,0E0,3.317311E-2,1.5156018E-2,1.8252775E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,19,19],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,-1,20,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,6.313997E-1,-5.5867573E-3,1.3400168E7,-5.0125904E-3,3.925844E7,1.3453537E-1,2.4241872E0,3.4442917E3,6.007968E-3,2.5669595E5,8.01112E5,3.1166228E-3,1.1309524E-1,1E0,1.8822147E-3,-6.655829E-4,3.1259478E10,6.866558E5,2.4759493E1,-2.082319E-4,-4.501191E-4,1.8846401E-3,4.3006588E-4,4.35119E-3,-4.699623E-4,1.7081059E-3,-6.844289E-4,-3.4481182E-3],"split_indices":[45,38,0,1,0,45,57,39,4,0,33,32,0,57,94,0,0,31,28,56,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,1.07E2,4E0,1.02E2,5E0,8.6E1,1.6E1,4.6E1,4E1,4E0,1.2E1,3.9E1,7E0,2.3E1,1.7E1,8E0,4E0,1.1E1,2.8E1,1.6E1,7E0,1.3E1,4E0,7E0,4E0,2.4E1,4E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.20608555E-2,-8.557722E-3,5.3557336E-2,2.1239823E-2,-4.6016023E-2,1.1540856E-1,2.1647736E-2,3.8935628E-3,7.878925E-3,-3.1252787E-2,-3.6052517E-3,2.3051652E-3,5.939278E-3,-5.275031E-4,4.6000607E-2,-1.2518006E-3,2.4584552E-2,-7.370518E-4,-2.5459786E-3,3.3509156E-3,3.4946052E-4,2.684215E-3,8.426045E-3,7.491929E-4,-1.1805954E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[8.516258E-2,7.576045E-2,6.263205E-2,4.3121457E-2,2.072651E-2,1.7343491E-2,2.0981252E-2,0E0,2.5021566E-2,9.199277E-3,0E0,0E0,0E0,0E0,2.2906693E-2,0E0,1.947644E-2,0E0,0E0,0E0,0E0,0E0,1.0816756E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,14,14,16,16,22,22],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[3.062708E3,2.7260774E7,1.3453537E-1,4.4493478E2,7.705493E5,2.8129198E7,1.2532358E6,3.8935628E-3,4.7586722E2,1.2903092E7,-3.6052517E-3,2.3051652E-3,5.939278E-3,-5.275031E-4,1.2320755E1,-1.2518006E-3,3.854803E-2,-7.370518E-4,-2.5459786E-3,3.3509156E-3,3.4946052E-4,2.684215E-3,4.7E1,7.491929E-4,-1.1805954E-3],"split_indices":[52,45,57,4,28,45,32,0,52,32,0,0,0,0,54,0,38,0,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.8E1,6.6E1,3.2E1,3.7E1,2.9E1,1E1,2.2E1,4E0,3.3E1,2.4E1,5E0,6E0,4E0,9E0,1.3E1,9E0,2.4E1,2E1,4E0,5E0,8E0,5E0,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics index 27abdd04d..5c1a96086 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics @@ -40,70 +40,77 @@ 38,numExecutors,0.000671226,8.239700374531836,0.42770227167428976,8.0,8.0,8.0,8.0,9.0 39,sqlOp_ObjectHashAggregate,0.0005553033,0.12734082397003746,0.33398039193838686,0.0,0.0,0.0,0.0,1.0 40,sqlOp_Subquery,0.00046715353,0.2247191011235955,0.4181810501732485,0.0,0.0,0.0,0.0,1.0 -41,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -42,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_HashAggregate,0.0,0.9887640449438202,0.10560054067615239,0.0,1.0,1.0,1.0,1.0 -48,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -52,sqlOp_Expand,0.0,0.10861423220973783,0.3117389214311049,0.0,0.0,0.0,0.0,1.0 -53,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -56,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_WindowGroupLimit,0.0,0.0149812734082397,0.1217058407722143,0.0,0.0,0.0,0.0,1.0 -58,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_SortMergeJoin,0.0,0.5617977528089888,0.4970981094018272,0.0,0.0,1.0,1.0,1.0 -61,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_BroadcastHashJoin,0.0,0.947565543071161,0.22332010811379224,0.0,1.0,1.0,1.0,1.0 -73,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,memoryBytesSpilled_mean,0.0,11069065.0864877,79810143.77222423,0.0,0.0,0.0,0.0,874196883.2566372 -76,diskBytesSpilled_mean,0.0,2880697.370667337,23636965.460199267,0.0,0.0,0.0,0.0,314481379.9144543 -77,maxMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 -78,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,maxOnHeapMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 -80,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -82,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,memoryBytesSpilledRatio,0.0,0.13140823253187295,0.9393631585625732,0.0,0.0,0.0,0.0,9.641198498202698 -84,diskBytesSpilledRatio,0.0,0.03193087515173895,0.2585787901798907,0.0,0.0,0.0,0.0,3.4683004089982026 +41,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +42,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +43,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +44,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_HashAggregate,0.0,0.9887640449438202,0.10560054067615239,0.0,1.0,1.0,1.0,1.0 +50,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,sqlOp_Expand,0.0,0.10861423220973783,0.3117389214311049,0.0,0.0,0.0,0.0,1.0 +55,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_WindowGroupLimit,0.0,0.0149812734082397,0.1217058407722143,0.0,0.0,0.0,0.0,1.0 +61,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_SortMergeJoin,0.0,0.5617977528089888,0.4970981094018272,0.0,0.0,1.0,1.0,1.0 +64,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +79,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,maxMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 +81,memoryBytesSpilled_mean,0.0,11069065.0864877,79810143.77222423,0.0,0.0,0.0,0.0,874196883.2566372 +82,diskBytesSpilled_mean,0.0,2880697.370667337,23636965.460199267,0.0,0.0,0.0,0.0,314481379.9144543 +83,maxOnHeapMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 +84,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 85,executorMemory,0.0,47016.0,0.0,47016.0,47016.0,47016.0,47016.0,47016.0 -86,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -89,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -90,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_BroadcastNestedLoopJoin,0.0,0.04119850187265917,0.19912228909086188,0.0,0.0,0.0,0.0,1.0 -94,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -95,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_AQEShuffleRead,0.0,0.9176029962546817,0.2754849853147098,0.0,1.0,1.0,1.0,1.0 -98,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -101,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,platform_databricks-azure,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -103,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -105,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -106,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +87,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +89,memoryBytesSpilledRatio,0.0,0.13140823253187295,0.9393631585625732,0.0,0.0,0.0,0.0,9.641198498202698 +90,diskBytesSpilledRatio,0.0,0.03193087515173895,0.2585787901798907,0.0,0.0,0.0,0.0,3.4683004089982026 +91,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +94,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +96,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_BroadcastNestedLoopJoin,0.0,0.04119850187265917,0.19912228909086188,0.0,0.0,0.0,0.0,1.0 +104,sqlOp_BroadcastHashJoin,0.0,0.947565543071161,0.22332010811379224,0.0,1.0,1.0,1.0,1.0 +105,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +107,sqlOp_AQEShuffleRead,0.0,0.9176029962546817,0.2754849853147098,0.0,1.0,1.0,1.0,1.0 +108,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,platform_databricks-azure,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +113,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg index 24c179963..4369310c6 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"100"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0427632518","gamma":"0.023735268","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0427632518","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.023735268","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.634272933"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"100"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0427632518","gamma":"0.023735268","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0427632518","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.023735268","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.634272933"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json index 1aa1b696d..bf868a28e 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"100"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-9.1459654E-2,-5.54109E-1,2.0924011E-1,-1.2843856E-1,-7.506931E-1,4.6708934E-2,4.3444833E-1,-3.00433E-1,5.9030973E-3,-4.0931627E-2,-2.0354155E-2,1.2529472E-1,-2.1817419E-1,1.044215E-2,5.1811427E-1,-1.7521013E-2,-4.962099E-3,1.7057521E-2,4.6703424E-2,-1.376186E-2,-2.5653557E-3,1.03572775E-2,2.624943E-2,1.5643975E-1,-6.770325E-2,2.220804E-3,1.0714781E-2,-2.6944376E-4,-4.600815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[1.0967188E1,2.4976635E0,1.7366686E0,5.456359E-1,7.3559284E-1,6.310014E-1,1.9185519E-1,9.352398E-2,0E0,0E0,0E0,4.759083E-1,9.545508E-2,0E0,1.946559E-1,0E0,0E0,0E0,2.4912825E-1,0E0,0E0,0E0,0E0,8.535066E-2,2.3812339E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,18,18,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[1.1576994E-2,5.8871865E-1,5.517496E-1,4.2332153E0,1.4235585E3,2.5042E4,4.650193E-1,1.264233E6,5.9030973E-3,-4.0931627E-2,-2.0354155E-2,2.9862975E5,2.9816154E1,1.044215E-2,2.4786325E0,-1.7521013E-2,-4.962099E-3,1.7057521E-2,7.75024E0,-1.376186E-2,-2.5653557E-3,1.03572775E-2,2.624943E-2,2.3797054E0,6.5957415E-1,2.220804E-3,1.0714781E-2,-2.6944376E-4,-4.600815E-3],"split_indices":[38,27,42,54,4,9,27,1,0,0,0,32,56,0,56,0,0,0,57,0,0,0,0,57,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3E1,4.7E1,1E1,2E1,2.8E1,1.9E1,6E0,4E0,1E1,1E1,2.2E1,6E0,7E0,1.2E1,3E0,3E0,4E0,1.8E1,3E0,3E0,4E0,8E0,9E0,9E0,5E0,4E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.1174776E-2,-4.830118E-1,2.093845E-1,-5.955108E-1,2.832456E-3,2.718551E-1,-1.1238484E-2,-4.2921847E-1,-3.8282927E-2,1.8877497E-1,2.3755293E-2,-2.0353394E-2,-6.2712687E-3,-7.041739E-3,2.28291E-1,1.6740719E-2,1.4373985E-1,-1.1000476E-3,2.088966E-1,1.0797673E-2,2.6714997E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1,15,-1,17,-1,19,-1,-1],"loss_changes":[8.775161E0,1.8931375E0,1.4578552E0,8.6177444E-1,0E0,8.8489985E-1,0E0,1.6791415E-1,0E0,5.124587E-1,0E0,0E0,0E0,0E0,3.7144732E-1,0E0,2.4829155E-1,0E0,8.6377144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,14,14,16,16,18,18],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1,16,-1,18,-1,20,-1,-1],"split_conditions":[1.0420895E-2,1.962943E3,1.4696082E12,1.2427474E8,2.832456E-3,1.1997641E0,-1.1238484E-2,1.137E3,-3.8282927E-2,3.1183178E8,2.3755293E-2,-2.0353394E-2,-6.2712687E-3,-7.041739E-3,2.325535E7,1.6740719E-2,9.9583336E1,-1.1000476E-3,1.3312784E7,1.0797673E-2,2.6714997E-3],"split_indices":[38,4,31,45,0,39,0,0,0,7,0,0,0,0,45,0,58,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.9E1,4.6E1,2.4E1,5E0,4.1E1,5E0,1.7E1,7E0,3.3E1,8E0,1.4E1,3E0,3E0,3E1,9E0,2.1E1,6E0,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[4.9319595E-2,-1.9302101E-1,3.6433876E-1,7.7906296E-2,-4.79115E-1,9.435853E-2,4.2625263E-1,1.1187466E-2,1.46483015E-2,-6.209801E-1,-2.6790038E-1,-5.8295415E-4,8.798848E-3,2.8855914E-1,2.4525782E-2,-7.1154446E-3,6.2758505E-2,-3.1027814E-2,-8.73463E-3,-2.221211E-3,-1.4641795E-2,3.4274822E-1,1.1442407E-3,9.597006E-3,1.2086243E-2,1.618916E-2,6.7794104E-3,-3.4221742E-2,4.374038E-3,-3.5373755E-3,1.3179845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,-1,23,-1,-1,-1,-1,25,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[6.3303638E0,3.6803284E0,5.615382E-1,4.37576E-1,5.2281046E-1,9.9045046E-2,4.2257786E-1,2.0275913E-1,0E0,4.1841602E-1,1.5917337E-1,0E0,0E0,2.3200428E-1,0E0,0E0,1.3654894E-1,0E0,0E0,0E0,0E0,3.2384157E-2,0E0,0E0,6.1977267E-2,0E0,0E0,3.409381E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,16,16,21,21,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,-1,24,-1,-1,-1,-1,26,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[2.1091123E-1,5.81609E-1,2.1267605E0,7.347893E5,9.2989355E2,6.11E2,1.1328785E-1,1.58966E5,1.46483015E-2,7.940699E-3,1.5326E4,-5.8295415E-4,8.798848E-3,1.6129E4,2.4525782E-2,-7.1154446E-3,4.5E1,-3.1027814E-2,-8.73463E-3,-2.221211E-3,-1.4641795E-2,5.405855E6,1.1442407E-3,9.597006E-3,1.04776725E6,1.618916E-2,6.7794104E-3,3.7582534E5,4.374038E-3,-3.5373755E-3,1.3179845E-3],"split_indices":[39,27,56,28,4,0,38,1,0,38,9,0,0,9,0,0,8,0,0,0,0,32,0,0,32,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,4.6E1,3.5E1,2.4E1,2.2E1,7E0,2.8E1,2E1,4E0,1.2E1,1E1,4E0,3E0,1.6E1,1.2E1,4E0,1.6E1,9E0,3E0,3E0,7E0,1.3E1,3E0,3E0,1.3E1,1E1,3E0,9E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.1746E-5,-3.7669906E-1,2.8866062E-1,-3.884967E-2,-6.1617005E-1,2.2242902E-1,4.041311E-2,-1.8333656E-1,1.349988E-2,-1.0974476E-2,-7.124857E-1,1.21058464E-1,4.1034156E-1,-1.5099317E-2,-5.4323003E-2,-1.7728277E-2,-3.464742E-2,-6.61903E-3,1.6944882E-1,2.1158814E-2,6.093224E-3,2.6566937E-4,-4.978143E-3,1.9736634E-1,-1.4299648E-3,2.4857002E-1,1.0594456E-1,1.2756806E-2,5.6673386E-3,-2.7505087E-4,6.629363E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,17,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,29,-1,-1,-1,-1],"loss_changes":[9.024752E0,2.8886461E0,1.8330379E0,8.774959E-1,5.4436207E-1,7.884717E-1,0E0,2.4362645E-1,0E0,0E0,1.2995434E-1,4.2667007E-1,2.687955E-1,0E0,3.0791815E-2,0E0,0E0,0E0,1.5386707E-1,0E0,0E0,0E0,0E0,8.133137E-2,0E0,4.12758E-2,5.6151725E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,11,11,12,12,14,14,18,18,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,18,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,30,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,7.6312915E3,8.231683E2,5.0415697E5,9.306648E-1,4.041311E-2,1.6361E4,1.349988E-2,-1.0974476E-2,6.1358623E9,4.9538263E2,3.3934937E0,-1.5099317E-2,8.62E2,-1.7728277E-2,-3.464742E-2,-6.61903E-3,8.325696E8,2.1158814E-2,6.093224E-3,2.6566937E-4,-4.978143E-3,5.405855E6,-1.4299648E-3,3.1167011E0,1.1343E4,1.2756806E-2,5.6673386E-3,-2.7505087E-4,6.629363E-3],"split_indices":[42,27,52,52,28,39,0,9,0,0,5,4,54,0,0,0,0,0,7,0,0,0,0,32,0,54,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3.5E1,4.6E1,1.5E1,2E1,4.3E1,3E0,1.1E1,4E0,5E0,1.5E1,2.9E1,1.4E1,4E0,7E0,5E0,1E1,4E0,2.5E1,1E1,4E0,4E0,3E0,2.2E1,3E0,1.3E1,9E0,8E0,5E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.464384E-4,-4.6920645E-1,1.9333163E-1,-5.952402E-1,-5.518288E-4,8.82653E-2,7.322041E-1,-1.7733527E-2,-3.505385E-2,1.2868324E-1,-1.661365E-2,3.751995E-2,1.4170783E-2,1.1289165E-2,2.404824E-1,-1.4591739E-1,7.5357206E-2,3.0781708E-3,2.721677E-1,-2.24413E-3,-8.675699E-3,1.2053321E-2,1.097327E-2,1.6005205E-2,1.8141574E-1,2.3323535E-3,-3.3507994E-3,9.380849E-3,3.6096657E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,-1,-1,-1,15,17,19,21,-1,23,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[7.1566606E0,1.3150363E0,3.1059434E0,3.8960123E-1,0E0,9.7495353E-1,2.3299408E-1,0E0,0E0,5.878903E-1,0E0,0E0,0E0,2.4265E-1,1.0320592E-1,2.660957E-2,2.2293174E-1,0E0,1.0816252E-1,0E0,0E0,0E0,5.876059E-2,0E0,2.5660902E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,13,13,14,14,15,15,16,16,18,18,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,-1,-1,-1,16,18,20,22,-1,24,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[4.823969E-3,1.962943E3,4.0230347E3,9.1569895E-1,-5.518288E-4,1.6597747E8,8.795191E-1,-1.7733527E-2,-3.505385E-2,3.839604E-1,-1.661365E-2,3.751995E-2,1.4170783E-2,6.8177136E2,2.048724E0,4.191214E5,2.325535E7,3.0781708E-3,2.4696612E0,-2.24413E-3,-8.675699E-3,1.2053321E-2,2.2968803E1,1.6005205E-2,6.3315526E8,2.3323535E-3,-3.3507994E-3,9.380849E-3,3.6096657E-3],"split_indices":[39,4,52,27,0,45,27,0,0,42,0,0,0,4,53,28,45,0,53,0,0,0,57,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.2E1,5.5E1,1.7E1,5E0,4.7E1,8E0,1.1E1,6E0,4.4E1,3E0,5E0,3E0,2.2E1,2.2E1,6E0,1.6E1,4E0,1.8E1,3E0,3E0,3E0,1.3E1,7E0,1.1E1,9E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.242935E-2,-4.2519015E-1,1.5809536E-1,-1.7596853E-1,-5.4585034E-1,-1.1979334E-2,3.1758392E-1,-1.2096516E-2,7.0671114E-5,-2.8898947E-2,-1.2043131E-2,-1.1141433E-1,1.699006E-1,3.909627E-1,1.486259E-1,1.0207987E-1,-2.1020326E-1,1.4478304E-2,6.432891E-2,6.3625416E-3,4.2305976E-1,1.0282089E-2,2.6432208E-3,7.859005E-3,-2.1978715E-4,-1.2100349E-1,-1.4820124E-2,5.283804E-3,-1.1029282E-3,2.0101778E-2,1.07203815E-2,-2.3454928E-3,-7.534415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,19,21,23,25,-1,27,-1,29,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1031775E0,5.053699E-1,1.6313026E0,1.5237364E-1,2.5601578E-1,5.614863E-1,3.2622886E-1,0E0,0E0,0E0,0E0,4.432748E-1,1.7409271E-1,1.0029149E-1,7.281983E-2,6.226304E-2,1.283493E-1,0E0,4.5889825E-2,0E0,2.8016567E-2,0E0,0E0,0E0,0E0,2.6844382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,13,13,14,14,15,15,16,16,18,18,20,20,25,25],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,20,22,24,26,-1,28,-1,30,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9956966E-1,5.0415697E5,3.8151306E-1,1.3E1,1E0,1.1808436E3,3.3027112E0,-1.2096516E-2,7.0671114E-5,-2.8898947E-2,-1.2043131E-2,2.2519132E-1,5.77E2,1.9522085E0,5.643973E8,2.3526582E3,7.1974045E-1,1.4478304E-2,1.96E2,6.3625416E-3,1.5199E4,1.0282089E-2,2.6432208E-3,7.859005E-3,-2.1978715E-4,8.07E2,-1.4820124E-2,5.283804E-3,-1.1029282E-3,2.0101778E-2,1.07203815E-2,-2.3454928E-3,-7.534415E-3],"split_indices":[57,28,42,3,16,52,54,0,0,0,0,27,0,53,7,4,27,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,1.9E1,5.9E1,7E0,1.2E1,2.9E1,3E1,4E0,3E0,7E0,5E0,1.9E1,1E1,2E1,1E1,6E0,1.3E1,3E0,7E0,3E0,1.7E1,4E0,6E0,3E0,3E0,9E0,4E0,4E0,3E0,1.2E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.513954E-4,-3.7165117E-1,1.9000456E-1,-4.7413486E-1,2.825124E-3,3.6059546E-1,1.2376453E-2,-3.4155813E-1,-3.2959625E-2,8.805712E-2,4.5158103E-1,6.926323E-2,-1.0766846E-2,-1.8985951E-2,-2.011872E-1,6.119726E-4,6.7662527E-3,5.690954E-1,2.4471958E-1,-9.67671E-3,1.2490984E-1,-1.9245803E-3,-1.1622075E-2,2.7504956E-2,1.1122954E-2,3.9141355E-3,1.3088083E-2,2.4368338E-1,2.5420533E-2,4.2327684E-3,1.2809196E-2,-5.441798E-3,3.8679964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,-1,-1,29,31,-1,-1,-1,-1],"loss_changes":[5.6427393E0,1.2425084E0,1.6015294E0,6.018686E-1,0E0,6.2975526E-1,4.2316514E-1,1.550827E-1,0E0,3.913322E-2,3.4691477E-1,4.0653005E-1,0E0,0E0,8.6992264E-2,0E0,0E0,1.07441425E-1,5.655533E-2,0E0,2.301394E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.309106E-2,1.3065305E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,14,14,17,17,18,18,20,20,27,27,28,28],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,-1,-1,30,32,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.962943E3,1.6129E4,1.2427474E8,2.825124E-3,3.3379373E-1,1.4181119E12,1.648E3,-3.2959625E-2,1.3652755E6,7.8345644E-1,2.074E3,-1.0766846E-2,-1.8985951E-2,2.2858976E8,6.119726E-4,6.7662527E-3,2.5492783E0,4.932816E6,-9.67671E-3,1.2E1,-1.9245803E-3,-1.1622075E-2,2.7504956E-2,1.1122954E-2,3.9141355E-3,1.3088083E-2,2.5630938E5,6.8175425E6,4.2327684E-3,1.2809196E-2,-5.441798E-3,3.8679964E-3],"split_indices":[42,4,9,45,0,39,31,11,0,47,27,2,0,0,7,0,0,53,1,0,3,0,0,0,0,0,0,28,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.6E1,5.2E1,2.1E1,5E0,2.6E1,2.6E1,1.6E1,5E0,7E0,1.9E1,2.2E1,4E0,8E0,8E0,4E0,3E0,1.1E1,8E0,3E0,1.9E1,3E0,5E0,8E0,3E0,3E0,5E0,8E0,1.1E1,3E0,5E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[6.091429E-2,-4.1792938E-1,1.8350436E-1,-5.0144905E-1,-1.7964066E-3,8.22498E-2,6.454801E-1,-4.900659E-3,-2.5916405E-2,-2.5496872E-2,2.559193E-1,3.7555333E-2,1.6550634E-2,2.6243908E-2,-1.904673E-1,3.7283357E-5,3.196267E-1,1.13695696E-1,-1.2504852E-1,-1.4939752E-3,-1.1836821E-2,1.552213E-2,6.485372E-3,1.1388198E-2,2.7914882E-2,-1.0534041E-2,-1.619807E-3,-1.9044313E-3,5.9120446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[4.5967064E0,4.8128414E-1,2.8470929E0,4.565587E-1,0E0,9.795654E-1,3.2176065E-1,0E0,0E0,2.8667668E-1,3.246895E-1,0E0,0E0,3.582181E-1,9.774792E-2,0E0,6.1459064E-2,2.1511662E-1,9.639382E-2,0E0,0E0,0E0,0E0,0E0,1.0208167E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,13,13,14,14,16,16,17,17,18,18,24,24],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[1.6403629E4,4.593E3,4.0230347E3,8.0159146E-1,-1.7964066E-3,5.517496E-1,1.3946067E7,-4.900659E-3,-2.5916405E-2,1.2932927E1,7.97E2,3.7555333E-2,1.6550634E-2,5.484294E-1,5.303548E9,3.7283357E-5,1.984018E7,1.3E1,2.8506322E-2,-1.4939752E-3,-1.1836821E-2,1.552213E-2,6.485372E-3,1.1388198E-2,8.8E1,-1.0534041E-2,-1.619807E-3,-1.9044313E-3,5.9120446E-3],"split_indices":[47,2,52,27,0,42,47,0,0,57,10,0,0,27,12,0,48,3,39,0,0,0,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,1.5E1,6.1E1,1.2E1,3E0,5.1E1,1E1,3E0,9E0,3.2E1,1.9E1,4E0,6E0,2.5E1,7E0,4E0,1.5E1,1.6E1,9E0,3E0,4E0,1.1E1,4E0,5E0,1.1E1,3E0,6E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.6044117E-2,-2.813553E-1,1.1194357E-1,-3.8589445E-1,6.365203E-3,1.5455592E-1,-8.159058E-3,-2.0304742E-1,-2.2294467E-2,-7.5167357E-3,1.8609025E-1,-1.2465446E-2,1.0527307E-3,8.595018E-2,2.8701866E-1,1.0672706E-2,2.9445041E-2,1.3768066E-2,3.6020828E-3,-3.5456778E-3,1.2604819E-1,6.465945E-4,7.6542827E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,-1,-1,15,17,-1,19,-1,-1,-1,21,-1,-1],"loss_changes":[2.6090114E0,1.2717111E0,6.1000854E-1,4.3901563E-1,0E0,4.4938153E-1,0E0,2.2868678E-1,0E0,0E0,3.492887E-1,0E0,0E0,1.7756584E-1,1.00421906E-1,0E0,1.8411915E-1,0E0,0E0,0E0,5.0150126E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,13,13,14,14,16,16,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,-1,-1,16,18,-1,20,-1,-1,-1,22,-1,-1],"split_conditions":[9.254704E-3,1.962943E3,6.2012E4,5.0415697E5,6.365203E-3,2.074E3,-8.159058E-3,1.3E1,-2.2294467E-2,-7.5167357E-3,4.828561E-1,-1.2465446E-2,1.0527307E-3,3.4941578E6,8.0933E4,1.0672706E-2,1.077646E3,1.3768066E-2,3.6020828E-3,-3.5456778E-3,9.57E2,6.465945E-4,7.6542827E-3],"split_indices":[38,4,10,28,0,2,0,3,0,0,39,0,0,51,9,0,52,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,2.6E1,4.4E1,2.1E1,5E0,3.9E1,5E0,1E1,1.1E1,3E0,3.6E1,7E0,3E0,1.9E1,1.7E1,4E0,1.5E1,1.4E1,3E0,7E0,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.375215E-2,-1.7956609E-1,3.1396008E-1,-4.475621E-1,-7.0033245E-2,2.2233696E-1,3.357808E-2,-5.835884E-1,-1.26632E-3,1.17291234E-1,-1.402892E-1,1.2604502E-2,1.09961465E-1,-1.2833614E-2,-2.8878465E-2,8.959712E-3,1.5504903E-3,-8.761031E-2,-1.3359717E-2,3.247988E-4,6.303277E-3,-1.3335772E-1,-8.037925E-3,-1.5199379E-3,-8.473441E-3,2.137595E-3,-3.3594493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[4.002802E0,1.201539E0,9.912052E-1,6.6494465E-1,4.2443517E-1,1.6288972E-1,0E0,3.1345367E-2,0E0,5.966536E-2,1.8117502E-1,0E0,4.1036606E-2,0E0,0E0,0E0,0E0,6.8092436E-2,0E0,0E0,0E0,6.900616E-2,3.6662724E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,17,17,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[4.7985215E-2,5.382431E0,7.6312915E3,7.940699E-3,2.805454E-1,2.4832625E6,3.357808E-2,2.202021E8,-1.26632E-3,4.5E1,2.0536139E1,1.2604502E-2,5.316E3,-1.2833614E-2,-2.8878465E-2,8.959712E-3,1.5504903E-3,1.7058623E3,-1.3359717E-2,3.247988E-4,6.303277E-3,6.433619E2,1.0519099E6,-1.5199379E-3,-8.473441E-3,2.137595E-3,-3.3594493E-3],"split_indices":[38,56,52,38,27,32,0,7,0,8,56,0,2,0,0,0,0,4,0,0,0,4,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,4.1E1,2.6E1,1.1E1,3E1,2.3E1,3E0,8E0,3E0,8E0,2.2E1,1.3E1,1E1,3E0,5E0,3E0,5E0,1.8E1,4E0,3E0,7E0,1.1E1,7E0,5E0,6E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.3291592E-2,-3.5239986E-1,1.72509E-1,-5.0849515E-1,5.0978072E-2,1.1366999E-1,5.429267E-1,-3.4126726E-1,-2.9658329E-2,-1.7515961E-3,5.6755748E-3,1.6279958E-1,-1.9032758E-1,2.9985784E-2,1.0644514E-2,-1.7278347E-2,-6.842589E-3,-3.87651E-3,2.166911E-1,-1.2132395E-2,-2.1109018E-3,7.0085307E-3,-8.727592E-2,1.4054082E-2,1.5595174E-1,-3.872923E-4,-5.6616897E-3,1.5735772E-3,8.481439E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[5.16432E0,1.8519189E0,1.0934212E0,4.131775E-1,7.307397E-2,7.329432E-1,1.5120387E-1,8.3277225E-2,0E0,0E0,0E0,3.6911952E-1,7.814169E-2,0E0,0E0,0E0,0E0,1.68213E-1,1.5955377E-1,0E0,0E0,0E0,2.7035102E-2,0E0,1.0380441E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,17,17,18,18,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,4.0230347E3,8.616169E-1,8.231683E2,1.1269586E12,2.1050402E7,1.5341808E3,-2.9658329E-2,-1.7515961E-3,5.6755748E-3,3.6732382E2,2.8312179E12,2.9985784E-2,1.0644514E-2,-1.7278347E-2,-6.842589E-3,1.7970878E6,2.325535E7,-1.2132395E-2,-2.1109018E-3,7.0085307E-3,1.2E1,1.4054082E-2,9.9583336E1,-3.872923E-4,-5.6616897E-3,1.5735772E-3,8.481439E-3],"split_indices":[42,4,52,27,52,31,47,4,0,0,0,52,31,0,0,0,0,48,45,0,0,0,3,0,58,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.8E1,5.2E1,2E1,8E0,4.6E1,6E0,1.2E1,8E0,4E0,4E0,4E1,6E0,3E0,3E0,8E0,4E0,1E1,3E1,3E0,3E0,3E0,7E0,9E0,2.1E1,3E0,4E0,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.11815E-2,-3.3522725E-1,1.8761614E-1,6.0145766E-2,-4.4398713E-1,1.0632117E-1,4.696459E-1,-1.2920295E-3,6.759094E-3,-5.660932E-1,-2.2601584E-1,2.910224E-2,1.8621325E-1,2.3904392E-2,1.1481897E-2,-2.8437966E-1,-3.473485E-2,-3.3279383E-3,-1.1625643E-2,-6.4782416E-3,8.3718695E-2,9.543453E-3,3.2117711E-3,-1.5034129E-2,-4.811055E-3,5.5344934E-3,-1.8265166E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,-1,23,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[5.2518044E0,1.4457278E0,1.0047364E0,7.55541E-2,5.640602E-1,2.2284049E-1,2.96309E-2,0E0,0E0,8.7858295E-1,5.3575993E-2,2.099497E-1,5.7148635E-2,0E0,0E0,6.437707E-2,0E0,0E0,0E0,0E0,9.798034E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,15,15,20,20],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,-1,24,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4865639E-2,3.459181E-1,1.0272479E0,8.231683E2,1.500766E3,4.7565946E-1,1.383E4,-1.2920295E-3,6.759094E-3,1.0370839E6,5.0017652E4,4.57563E5,9.19322E0,2.3904392E-2,1.1481897E-2,2.23E2,-3.473485E-2,-3.3279383E-3,-1.1625643E-2,-6.4782416E-3,3.64E2,9.543453E-3,3.2117711E-3,-1.5034129E-2,-4.811055E-3,5.5344934E-3,-1.8265166E-3],"split_indices":[38,27,39,52,4,42,10,0,0,28,33,1,56,0,0,10,0,0,0,0,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3.2E1,4.5E1,7E0,2.5E1,3.6E1,9E0,4E0,3E0,1.5E1,1E1,1.9E1,1.7E1,5E0,4E0,8E0,7E0,3E0,7E0,4E0,1.5E1,1.2E1,5E0,5E0,3E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.4895428E-3,-3.0832598E-1,1.476936E-1,7.337417E-2,-5.032524E-1,5.1306404E-2,3.429725E-1,-6.620061E-2,1.2798463E-2,-5.4586494E-1,-9.307089E-3,8.52972E-2,-1.10731E-2,4.080719E-1,3.9384817E-3,-6.1457483E-3,1.1915804E-3,-1.4732508E-2,-2.903554E-2,1.234163E-1,-7.188532E-3,2.0559082E-2,9.879167E-3,1.8509729E-1,2.3773596E-2,1.8284154E-4,2.4937104E-1,-3.5463482E-2,7.0954026E-3,1.3132916E-2,6.3187866E-3,-3.965284E-3,7.7684724E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1,25,27,-1,29,31,-1,-1,-1,-1,-1],"loss_changes":[3.7659874E0,2.0458183E0,1.0176347E0,3.351296E-1,1.0026741E-1,4.2281622E-1,2.5638676E-1,5.504486E-2,0E0,1.6813612E-1,0E0,3.5882187E-1,0E0,6.912041E-2,0E0,0E0,0E0,0E0,0E0,1.8612745E-1,0E0,0E0,0E0,2.1975332E-1,1.1535082E-1,0E0,3.675276E-2,3.2394454E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,19,19,23,23,24,24,26,26,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1,26,28,-1,30,32,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,9.306648E-1,8.231683E2,1.3513911E1,1.1696494E7,3.3934937E0,1.3E1,1.2798463E-2,2.8225487E11,-9.307089E-3,1E0,-1.10731E-2,1.3996E4,3.9384817E-3,-6.1457483E-3,1.1915804E-3,-1.4732508E-2,-2.903554E-2,5.6039695E9,-7.188532E-3,2.0559082E-2,9.879167E-3,8.1729946E10,2.0901184E3,1.8284154E-4,1.181508E6,1.742E3,7.0954026E-3,1.3132916E-2,6.3187866E-3,-3.965284E-3,7.7684724E-4],"split_indices":[42,27,39,52,56,47,54,3,0,31,0,105,0,9,0,0,0,0,0,5,0,0,0,31,52,0,32,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.6E1,5.4E1,9E0,1.7E1,3.7E1,1.7E1,6E0,3E0,1.4E1,3E0,3.4E1,3E0,1.3E1,4E0,3E0,3E0,7E0,7E0,3E1,4E0,8E0,5E0,1.8E1,1.2E1,5E0,1.3E1,9E0,3E0,7E0,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.9342616E-2,-1.177099E-1,2.5990668E-1,-3.7666702E-1,-3.965099E-2,1.0666826E-2,3.361856E-1,-6.8167797E-3,-2.0442668E-2,7.8358404E-2,-1.4977737E-1,-3.909633E-3,6.890782E-3,1.4904882E-3,3.7397426E-1,1.1430879E-2,1.0151933E-2,-8.376353E-2,-1.1135905E-2,1.761733E-2,9.761365E-3,5.960206E-3,-3.3769023E-2,-7.8526065E-3,-6.7802267E-3,1.5333147E-3,-5.919079E-3,-2.4406018E-3,2.3791464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,19,-1,21,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1,-1],"loss_changes":[2.8364453E0,9.5533246E-1,6.428132E-1,1.660701E-1,5.0558233E-1,1.5298896E-1,2.830248E-1,0E0,0E0,2.4214801E-1,1.2425065E-1,0E0,0E0,0E0,3.3308268E-2,0E0,8.984181E-2,7.82373E-2,0E0,0E0,0E0,0E0,9.239468E-2,2.8174143E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,14,14,16,16,17,17,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,20,-1,22,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1,-1],"split_conditions":[3.839604E-1,1.346E3,2.1262457E0,2.62518E5,5.81609E-1,1.6E1,3.2414E-1,-6.8167797E-3,-2.0442668E-2,4.48E2,3.6979167E0,-3.909633E-3,6.890782E-3,1.4904882E-3,1.5199E4,1.1430879E-2,1.6439099E-2,1.5326E4,-1.1135905E-2,1.761733E-2,9.761365E-3,5.960206E-3,2.1446E4,1.27426E6,-6.7802267E-3,1.5333147E-3,-5.919079E-3,-2.4406018E-3,2.3791464E-3],"split_indices":[42,11,56,29,27,8,27,0,0,0,54,0,0,0,9,0,27,9,0,0,0,0,9,1,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,4.7E1,3.3E1,1E1,3.7E1,8E0,2.5E1,4E0,6E0,1.8E1,1.9E1,5E0,3E0,3E0,2.2E1,4E0,1.4E1,1.3E1,6E0,1.6E1,6E0,3E0,1.1E1,7E0,6E0,7E0,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.996961E-2,-2.8205118E-1,1.5501395E-1,-1.55176E-1,-2.9040772E-2,6.840957E-2,5.63874E-1,-2.587841E-1,1.1721904E-2,2.6414385E-1,1.1448369E-2,3.3335984E-2,1.2408488E-2,-1.2926956E-2,-3.6485724E-3,-2.4606911E-3,2.392361E-3,4.0180907E-3,1.3522474E-2,-1.0006361E-1,6.47153E-2,-7.220365E-4,-7.224762E-3,1.0255815E-1,-3.6275077E-3,2.6295839E-2,7.0096995E-3,4.39064E-3,-2.3290648E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1,21,23,-1,-1,25,-1,27,-1,-1,-1],"loss_changes":[3.428976E0,1.2044048E0,2.0151515E0,3.6616492E-1,0E0,5.4328966E-1,3.6411428E-1,7.2318256E-2,3.0786386E-2,6.776947E-2,2.3813179E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.1633935E-2,1.6149634E-1,0E0,0E0,9.8638654E-2,0E0,7.344261E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,19,19,20,20,23,23,25,25],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1,22,24,-1,-1,26,-1,28,-1,-1,-1],"split_conditions":[4.856813E0,7.916317E11,3.9509753E3,1.6E1,-2.9040772E-2,1.7329262E7,8.982776E9,1.0370839E6,1.938E3,1.0843103E5,1.4343789E6,3.3335984E-2,1.2408488E-2,-1.2926956E-2,-3.6485724E-3,-2.4606911E-3,2.392361E-3,4.0180907E-3,1.3522474E-2,1.8523E4,2.3297022E7,-7.220365E-4,-7.224762E-3,2.1202636E-1,-3.6275077E-3,1.3397689E-1,7.0096995E-3,4.39064E-3,-2.3290648E-3],"split_indices":[58,31,52,8,0,45,5,28,2,33,47,0,0,0,0,0,0,0,0,9,32,0,0,39,0,39,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.5E1,5.7E1,2E1,5E0,4.8E1,9E0,1.2E1,8E0,1E1,3.8E1,4E0,5E0,9E0,3E0,3E0,5E0,3E0,7E0,1.2E1,2.6E1,6E0,6E0,2.1E1,5E0,1E1,1.1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.7959079E-2,3.3515338E-2,-5.439698E-1,-9.869692E-2,1.8315943E-1,-3.189193E-2,-4.5014187E-3,3.8901392E-2,-2.3159537E-1,1.0944469E-1,1.3747038E-2,-1.9519506E-2,1.2489866E-2,-1.5762134E-2,-1.3676985E-1,-6.1160773E-3,1.64186E-1,2.8373315E-3,-4.879285E-2,1.6439244E-3,-9.010175E-3,-3.0944736E-3,4.0532374E-3,2.1961888E-3,8.336786E-3,3.047169E-4,-4.52133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,-1,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.682834E0,1.5217208E0,7.183106E-1,7.5876E-1,3.3185184E-1,0E0,0E0,3.1629825E-1,2.2239149E-1,1.5915287E-1,0E0,4.8483647E-2,0E0,0E0,1.8944594E-1,6.701711E-2,4.9000174E-2,0E0,4.532664E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,14,14,15,15,16,16,18,18],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,-1,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5414109E8,3.2711282E-2,7.493E3,5.81609E-1,2.4273067E1,-3.189193E-2,-4.5014187E-3,7.347893E5,3.507E3,2.6880343E0,1.3747038E-2,1.204507E-1,1.2489866E-2,-1.5762134E-2,5.7229916E7,1.6E1,1.2E1,2.8373315E-3,2.1416E4,1.6439244E-3,-9.010175E-3,-3.0944736E-3,4.0532374E-3,2.1961888E-3,8.336786E-3,3.047169E-4,-4.52133E-3],"split_indices":[45,38,2,27,57,0,0,28,2,54,0,27,0,0,45,3,3,0,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.5E1,8E0,4E1,3.5E1,5E0,3E0,2E1,2E1,2.4E1,1.1E1,1.7E1,3E0,7E0,1.3E1,8E0,1.6E1,4E0,1.3E1,4E0,9E0,5E0,3E0,4E0,1.2E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.9505762E-2,-3.0160806E-1,1.1446385E-1,-4.101308E-1,6.7047074E-2,-8.379094E-3,2.612527E-1,-2.6891392E-1,-5.6854063E-1,5.741581E-3,-7.2406675E-4,2.5123904E-2,-1.0655792E-2,1.7354633E-1,3.8422826E-1,-3.8618457E-3,-1.3404787E-2,-1.2552934E-2,-2.8100347E-2,-4.6093785E-3,7.231362E-2,2.4235931E-4,8.942462E-3,9.910835E-3,2.09687E-2,7.163702E-3,5.9442273E-3,-1.8904549E-3,4.4926545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[3.1713533E0,1.1076999E0,1.0300117E0,3.1688452E-1,4.1787274E-2,2.644232E-1,2.1374798E-1,7.513434E-2,2.6323795E-2,0E0,0E0,1.8968515E-1,0E0,1.0032731E-1,4.8149586E-2,0E0,0E0,0E0,0E0,0E0,1.3801764E-1,0E0,0E0,0E0,0E0,0E0,7.423505E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,20,20,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,5.451147E-1,9.29295E5,1.6867E4,3.6030095E10,1.1997641E0,3.3159972E5,8.28645E-1,5.741581E-3,-7.2406675E-4,3.7818575E0,-1.0655792E-2,1.0966E4,1.3E1,-3.8618457E-3,-1.3404787E-2,-1.2552934E-2,-2.8100347E-2,-4.6093785E-3,2.491358E0,2.4235931E-4,8.942462E-3,9.910835E-3,2.09687E-2,7.163702E-3,3.6616542E0,-1.8904549E-3,4.4926545E-3],"split_indices":[42,4,42,28,9,5,39,28,27,0,0,56,0,9,3,0,0,0,0,0,53,0,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.6E1,5.6E1,2E1,6E0,3.1E1,2.5E1,1.2E1,8E0,3E0,3E0,2.8E1,3E0,1.6E1,9E0,3E0,9E0,3E0,5E0,7E0,2.1E1,3E0,1.3E1,5E0,4E0,8E0,1.3E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.449704E-2,-3.1889904E-1,1.4883669E-1,-4.2908838E-1,5.913231E-3,9.372266E-2,2.1409126E-2,-1.1719601E-2,-2.513279E-2,1.2944044E-1,-8.096517E-3,1.7205084E-2,1.8951772E-1,6.2284645E-2,-4.7494997E-3,1.2188283E-2,1.3342847E-1,-1.9648627E-3,4.6447357E-3,8.554429E-2,8.850229E-3,5.8247205E-3,5.207967E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,13,15,17,-1,-1,19,-1,-1,21,-1,-1,-1],"loss_changes":[3.721609E0,1.4194877E0,8.449938E-1,3.7996912E-1,0E0,4.390232E-1,0E0,0E0,0E0,2.4622273E-1,0E0,8.7870665E-2,9.0527E-2,6.0149476E-2,0E0,0E0,4.215181E-2,0E0,0E0,4.2945392E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,11,11,12,12,13,13,16,16,19,19],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,14,16,18,-1,-1,20,-1,-1,22,-1,-1,-1],"split_conditions":[1.7489342E-2,1.9315491E3,6.7780703E3,8.616169E-1,5.913231E-3,2.9461394E5,2.1409126E-2,-1.1719601E-2,-2.513279E-2,2.847953E-1,-8.096517E-3,5.303548E9,9.693049E2,4.57563E5,-4.7494997E-3,1.2188283E-2,4.946992E6,-1.9648627E-3,4.6447357E-3,5.99187E0,8.850229E-3,5.8247205E-3,5.207967E-4],"split_indices":[38,4,4,27,0,33,0,0,0,42,0,12,4,1,0,0,1,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.6E1,4.5E1,2.1E1,5E0,4E1,5E0,1.2E1,9E0,3.6E1,4E0,1.3E1,2.3E1,1E1,3E0,7E0,1.6E1,3E0,7E0,1.1E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.07692E-3,-2.7644157E-1,1.2962286E-1,-8.736741E-2,-4.1278672E-1,8.3613865E-2,4.5945916E-1,5.1238113E-3,-1.6150486E-1,-5.141799E-1,-3.715523E-3,6.8261735E-3,2.6075876E-1,2.6412223E-2,7.971721E-3,-9.983092E-3,-1.8332362E-3,-9.530817E-3,-2.7025245E-2,5.4569438E-2,-1.7903225E-1,1.6170103E-2,6.5223556E-3,1.2187874E-1,5.6436267E-3,-1.6137054E-2,6.6004094E-4,6.8556686E-4,7.955213E-3,6.314655E-2,-2.7909344E-3,-2.401067E-3,4.913517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[3.130982E0,6.8575764E-1,8.569336E-1,2.1903388E-1,4.8403072E-1,7.1915495E-1,1.8719184E-1,0E0,7.534486E-2,2.7091742E-1,0E0,3.4696224E-1,1.4558935E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.01400204E-1,3.1436688E-1,0E0,0E0,8.528942E-2,8.159262E-2,0E0,0E0,0E0,0E0,7.43649E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,19,19,20,20,23,23,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[7.2921924E-2,5.0415697E5,4.4839956E3,8.205673E-4,1.0735684E3,8.363552E-1,5.902777E-1,5.1238113E-3,6.386E3,9.29295E5,-3.715523E-3,1.0743855E-1,2.4696612E0,2.6412223E-2,7.971721E-3,-9.983092E-3,-1.8332362E-3,-9.530817E-3,-2.7025245E-2,4.1508866E5,2.8808794E5,1.6170103E-2,6.5223556E-3,1.3397689E-1,6.0487356E7,-1.6137054E-2,6.6004094E-4,6.8556686E-4,7.955213E-3,1.1171711E9,-2.7909344E-3,-2.401067E-3,4.913517E-3],"split_indices":[42,28,52,42,52,41,27,0,10,28,0,38,53,0,0,0,0,0,0,28,28,0,0,39,45,0,0,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,2.7E1,5.8E1,1.2E1,1.5E1,5.2E1,6E0,3E0,9E0,1.1E1,4E0,3.7E1,1.5E1,3E0,3E0,5E0,4E0,4E0,7E0,3E1,7E0,6E0,9E0,1.2E1,1.8E1,3E0,4E0,5E0,7E0,1E1,8E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.484174E-2,-2.0651723E-1,1.23060025E-1,9.060144E-2,-3.7930906E-1,8.6913936E-2,1.6235925E-2,-8.894108E-4,1.12425E-2,-2.322102E-1,-2.28896E-2,9.985029E-3,6.3343115E-2,-1.4884285E-2,-4.1465354E-3,1.2130751E-3,1.28519E-1,-4.1467164E-2,5.214476E-3,7.1523585E-3,2.1171435E-3,-3.2622847E-3,3.881595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,15,-1,-1,17,19,21,-1,-1,-1,-1,-1],"loss_changes":[1.8220457E0,1.3178582E0,4.4004917E-1,1.9740945E-1,2.4275732E-1,1.4414191E-1,0E0,0E0,0E0,1.2293494E-1,0E0,0E0,1.6586871E-1,0E0,0E0,1.1841317E-1,5.2930146E-2,8.520052E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,16,-1,-1,18,20,22,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,5.8871865E-1,6.5317163E3,8.231683E2,1.1686677E8,3.5175372E5,1.6235925E-2,-8.894108E-4,1.12425E-2,1.93E2,-2.28896E-2,9.985029E-3,5.517496E-1,-1.4884285E-2,-4.1465354E-3,3.327103E0,6.7584877E8,1.1812E4,5.214476E-3,7.1523585E-3,2.1171435E-3,-3.2622847E-3,3.881595E-3],"split_indices":[38,27,4,52,45,32,0,0,0,10,0,0,41,0,0,53,7,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.4E1,5E1,9E0,1.5E1,4.5E1,5E0,6E0,3E0,9E0,6E0,5E0,4E1,4E0,5E0,2.1E1,1.9E1,1.6E1,5E0,1.2E1,7E0,1.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-3.6807183E-2,-2.6082924E-1,9.745226E-2,-3.38609E-2,-4.4575256E-1,5.3367496E-2,2.569166E-1,3.8533923E-3,-1.0485729E-1,-2.6165378E-1,-2.6519928E-2,-7.0941593E-3,1.42699E-1,1.3604495E-2,5.124206E-3,-2.7395165E-4,-6.2492257E-3,-2.6505715E-3,-1.4469911E-2,5.6039942E-3,-3.8167406E-2,7.6184836E-3,2.8479039E-3,-6.874908E-2,2.3452176E-3,3.404119E-4,-1.1131427E-1,-8.842438E-4,-6.103098E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,27,-1,-1],"loss_changes":[2.4917388E0,1.2860303E0,3.518464E-1,1.4147085E-1,3.8356328E-1,2.2785982E-1,5.4195166E-2,0E0,3.9702587E-2,1.3221234E-1,0E0,1.1660655E-1,3.400594E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6346675E-2,0E0,0E0,5.6393713E-2,0E0,0E0,2.8358817E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,20,20,23,23,26,26],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,28,-1,-1],"split_conditions":[1.7489342E-2,5.0415697E5,4.108022E1,1.8144448E-4,2.5036643E8,6.3104886E-1,4.668526E6,3.8533923E-3,4.4092423E-1,5.86E2,-2.6519928E-2,2.7252597E-1,2.6529046E7,1.3604495E-2,5.124206E-3,-2.7395165E-4,-6.2492257E-3,-2.6505715E-3,-1.4469911E-2,5.6039942E-3,2.6597537E3,7.6184836E-3,2.8479039E-3,3.9785632E8,2.3452176E-3,3.404119E-4,1.0192E4,-8.842438E-4,-6.103098E-3],"split_indices":[38,28,57,38,7,41,29,0,27,0,0,27,45,0,0,0,0,0,0,0,4,0,0,7,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3E1,5.1E1,1.4E1,1.6E1,4.1E1,1E1,5E0,9E0,9E0,7E0,2.5E1,1.6E1,6E0,4E0,3E0,6E0,3E0,6E0,4E0,2.1E1,1E1,6E0,1.6E1,5E0,6E0,1E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.750396E-3,-2.637187E-1,1.3615979E-1,-1.4564972E-1,-5.4651755E-1,5.4761996E-3,2.2307935E-1,-1.2620885E-2,-9.917557E-2,-3.0120319E-2,-1.07787E-2,9.194145E-2,-6.576594E-2,3.183191E-1,1.2566628E-1,7.8938337E-4,-1.3992551E-1,9.1104873E-4,6.1660395E-3,-4.332866E-4,-6.867432E-3,6.8999347E-3,1.6945135E-2,1.6493298E-3,7.0712506E-3,-4.079417E-4,-8.720541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5698216E0,7.5514436E-1,5.099317E-1,1.02725804E-1,1.4779592E-1,1.2308604E-1,2.1049309E-1,0E0,7.928079E-2,0E0,0E0,3.0144311E-2,5.6403577E-2,9.509659E-2,4.5478165E-2,0E0,9.819192E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,13,13,14,14,16,16],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,1.5135763E8,2.5222173E-1,5.4508613E3,9.7646296E7,4.4605106E5,1.5312917E5,-1.2620885E-2,5.7777777E-2,-3.0120319E-2,-1.07787E-2,2.9795604E9,2.1446E4,3.9976162E-1,1.7325155E5,7.8938337E-4,3.027892E5,9.1104873E-4,6.1660395E-3,-4.332866E-4,-6.867432E-3,6.8999347E-3,1.6945135E-2,1.6493298E-3,7.0712506E-3,-4.079417E-4,-8.720541E-3],"split_indices":[38,45,39,33,12,28,33,0,57,0,0,5,9,27,33,0,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,2.4E1,4.4E1,1.8E1,6E0,1.8E1,2.6E1,3E0,1.5E1,3E0,3E0,8E0,1E1,1.2E1,1.4E1,4E0,1.1E1,4E0,4E0,7E0,3E0,5E0,7E0,5E0,9E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.408408E-3,-2.5382924E-1,1.1687296E-1,-1.6837038E-1,-2.9154288E-2,5.9414465E-2,2.300275E-2,-2.20062E-1,-1.8780673E-2,2.3231715E-1,4.9326853E-3,-1.4188947E-1,-1.6220024E-2,2.9747307E-3,-4.3801954E-3,3.8770789E-3,1.28422305E-2,-1.8587545E-1,5.7399318E-2,-8.522451E-3,-2.7460514E-3,-2.3119294E-3,-1.0868278E-2,6.147688E-3,4.379439E-3,-5.93595E-2,8.179598E-2,1.3064143E-3,-4.1435147E-3,6.122379E-5,6.2349276E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,-1,25,27,29,-1,-1,-1,-1],"loss_changes":[2.3664117E0,8.3604467E-1,1.2018381E0,1.7371541E-1,0E0,4.3217617E-1,0E0,1.5779579E-1,5.8853693E-2,7.8718185E-2,3.710673E-1,4.5167238E-2,0E0,0E0,0E0,0E0,0E0,5.8254957E-2,1.3215809E-1,0E0,0E0,0E0,0E0,0E0,9.860988E-2,4.0082153E-2,4.6084777E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,17,17,18,18,24,24,25,25,26,26],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,-1,26,28,30,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.5135763E8,3.4409692E3,5.36E3,-2.9154288E-2,1.7329262E7,2.300275E-2,1.02E3,2.2858976E8,6.498673E2,4.492E3,3E0,-1.6220024E-2,2.9747307E-3,-4.3801954E-3,3.8770789E-3,1.28422305E-2,4.841492E5,5.090285E7,-8.522451E-3,-2.7460514E-3,-2.3119294E-3,-1.0868278E-2,6.147688E-3,1.204781E3,4.1485094E-2,1.7495675E5,1.3064143E-3,-4.1435147E-3,6.122379E-5,6.2349276E-3],"split_indices":[58,45,52,2,0,45,0,0,7,4,2,8,0,0,0,0,0,28,45,0,0,0,0,0,52,39,33,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.5E1,5E1,2.2E1,3E0,4.5E1,5E0,1.6E1,6E0,1E1,3.5E1,1.2E1,4E0,3E0,3E0,4E0,6E0,7E0,2.8E1,6E0,6E0,3E0,4E0,1E1,1.8E1,1E1,8E0,3E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.0632061E-2,-2.4092656E-1,9.551605E-2,-4.0311107E-1,-8.516071E-2,1.254183E-1,-1.1961503E-2,-7.877372E-3,-2.3726251E-2,-1.6082093E-1,1.2791782E-3,4.8489507E-2,1.8568814E-1,-1.0854732E-2,-2.319776E-3,1.9796067E-1,-1.993472E-2,-2.5485766E-3,2.2027308E-1,3.1508906E-3,1.1663632E-2,-2.6950627E-3,3.7402357E-3,2.500118E-1,5.6986962E-3,5.1142857E-3,2.8351766E-1,1.466244E-2,7.1420944E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,17,-1,-1,19,21,-1,23,-1,-1,-1,-1,25,-1,-1,27,-1,-1],"loss_changes":[1.7830759E0,5.4671514E-1,5.960934E-1,2.6311874E-1,1.1799519E-1,2.0891088E-1,0E0,0E0,0E0,6.5531775E-2,0E0,2.2895017E-1,2.3369044E-1,0E0,0E0,4.4965267E-2,7.9553865E-2,0E0,2.6198149E-2,0E0,0E0,0E0,0E0,3.600812E-2,0E0,0E0,2.5718153E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,12,12,15,15,16,16,18,18,23,23,26,26],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,18,-1,-1,20,22,-1,24,-1,-1,-1,-1,26,-1,-1,28,-1,-1],"split_conditions":[2.0113895E-2,3.554517E0,6.2012E4,1.0370839E6,1.7493458E3,3.3597556E-1,-1.1961503E-2,-7.877372E-3,-2.3726251E-2,9.4596675E-5,1.2791782E-3,2.5143658E6,1.4212261E0,-1.0854732E-2,-2.319776E-3,3.7700243E-2,3.6616542E0,-2.5485766E-3,3.2979845E6,3.1508906E-3,1.1663632E-2,-2.6950627E-3,3.7402357E-3,3.1823853E-1,5.6986962E-3,5.1142857E-3,6.585265E-1,1.466244E-2,7.1420944E-3],"split_indices":[41,54,10,28,4,39,0,0,0,38,0,51,56,0,0,27,53,0,32,0,0,0,0,27,0,0,27,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.2E1,4.9E1,1E1,1.2E1,4.6E1,3E0,5E0,5E0,7E0,5E0,2.1E1,2.5E1,3E0,4E0,6E0,1.5E1,3E0,2.2E1,3E0,3E0,1.1E1,4E0,1.5E1,7E0,4E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.2602936E-2,-1.7656966E-1,6.665869E-2,-2.5975782E-1,8.466752E-2,1.0225561E-1,-1.767866E-1,-1.4000464E-3,-2.921736E-1,8.9720427E-4,6.119811E-3,1.7116562E-2,1.6708885E-1,-1.2227611E-2,-1.0023373E-3,-3.4930915E-1,-4.303655E-3,-3.5181323E-3,6.2355846E-2,2.0918262E-1,4.6230074E-2,-1.6783658E-2,-5.6650047E-3,7.1607847E-3,-6.9373236E-3,9.782477E-3,3.3604016E-3,-1.9280773E-3,4.705579E-3,1.6120562E-3,-3.5627545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,-1,-1,21,-1,-1,23,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1249788E0,6.739482E-1,4.687044E-1,1.5969086E-1,2.6773125E-2,2.5105268E-1,1.1046371E-1,0E0,1.8371439E-1,0E0,0E0,9.954972E-2,1.2260193E-1,0E0,0E0,8.8416815E-2,0E0,0E0,1.1039816E-1,3.0600965E-2,5.1575392E-2,0E0,0E0,0E0,3.7230834E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,15,15,18,18,19,19,20,20,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,-1,-1,22,-1,-1,24,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.962943E3,1.4696082E12,2.5346E3,2.4399805E3,3.3597556E-1,3.05E3,-1.4000464E-3,2.2867646E0,8.9720427E-4,6.119811E-3,7.095772E2,5.7588155E6,-1.2227611E-2,-1.0023373E-3,1.650581E3,-4.303655E-3,-3.5181323E-3,4.8178736E7,1.6129E4,6.249143E-1,-1.6783658E-2,-5.6650047E-3,7.1607847E-3,7.844101E0,9.782477E-3,3.3604016E-3,-1.9280773E-3,4.705579E-3,1.6120562E-3,-3.5627545E-3],"split_indices":[42,4,31,32,4,39,0,0,57,0,0,4,32,0,0,4,0,0,45,9,42,0,0,0,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.9E1,5.1E1,2.2E1,7E0,4.5E1,6E0,3E0,1.9E1,4E0,3E0,2E1,2.5E1,3E0,3E0,1.4E1,5E0,6E0,1.4E1,1.8E1,7E0,1.1E1,3E0,5E0,9E0,1.5E1,3E0,3E0,4E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.6865763E-3,-1.8272516E-1,8.7893896E-2,-2.5947955E-1,3.8822438E-3,1.21314E-1,-1.1284622E-2,-1.7682245E-1,-2.2460883E-2,2.035009E-1,7.4624578E-3,-1.0745997E-2,-7.0943184E-2,2.0190727E-2,2.6090908E-1,-4.7269452E-2,3.9244615E-3,3.7158409E-3,-7.826691E-3,-1.5020958E-3,3.6044621E-3,1.7683446E-1,3.441152E-1,1.9708078E-3,-8.922047E-2,1.8667543E-3,9.08392E-3,1.899623E-2,8.695617E-3,-5.6091757E-3,-8.99704E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2168006E0,5.273904E-1,6.668652E-1,3.2449567E-1,0E0,4.6633655E-1,0E0,1.08326495E-1,0E0,2.9991376E-1,1.0585508E-1,0E0,1.5742677E-1,3.1326193E-2,9.305024E-2,5.894113E-2,0E0,0E0,0E0,0E0,0E0,5.2344352E-2,5.0588608E-2,0E0,2.584079E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,12,12,13,13,14,14,15,15,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,1.5414109E8,1.2427474E8,3.8822438E-3,3.3934937E0,-1.1284622E-2,5E0,-2.2460883E-2,2.0753424E0,5.2518907E0,-1.0745997E-2,5.0415697E5,1.6E1,2.9168832E0,7.843361E5,3.9244615E-3,3.7158409E-3,-7.826691E-3,-1.5020958E-3,3.6044621E-3,1.0737101E0,7.33E2,1.9708078E-3,9.21919E5,1.8667543E-3,9.08392E-3,1.899623E-2,8.695617E-3,-5.6091757E-3,-8.99704E-4],"split_indices":[58,4,45,45,0,54,0,8,0,53,54,0,28,3,54,32,0,0,0,0,0,57,0,0,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.3E1,5.3E1,1.8E1,5E0,4.9E1,4E0,1.5E1,3E0,2.8E1,2.1E1,8E0,7E0,7E0,2.1E1,1.3E1,8E0,3E0,4E0,4E0,3E0,1.2E1,9E0,4E0,9E0,3E0,9E0,4E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.621743E-2,3.558231E-2,-2.938916E-1,-1.8531272E-1,7.990702E-2,6.033404E-4,-3.8770264E-1,-9.306622E-3,-2.8338241E-3,3.3018474E-2,1.9380707E-1,-1.9141065E-2,-7.951701E-3,9.9072486E-2,-1.2703481E-1,2.4355294E-1,2.651927E-3,8.730972E-3,3.930368E-2,-7.339577E-3,-9.130585E-4,6.346931E-3,1.3425479E-2,-2.515763E-3,7.3647335E-2,4.856358E-3,1.2821703E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,23,-1,-1,-1,-1,-1,25,-1,-1],"loss_changes":[1.1899004E0,7.001853E-1,3.8109004E-1,3.1746447E-2,3.0869663E-1,0E0,3.7626266E-2,0E0,0E0,4.6718863E-1,9.635085E-2,0E0,0E0,1.8670347E-1,5.760783E-2,3.3767164E-2,0E0,0E0,7.510835E-2,0E0,0E0,0E0,0E0,0E0,2.448184E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,13,13,14,14,15,15,18,18,24,24],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,24,-1,-1,-1,-1,-1,26,-1,-1],"split_conditions":[1.0232127E12,1.885E3,2.7307262E5,1.4324325E1,3.029697E3,6.033404E-4,2.8312179E12,-9.306622E-3,-2.8338241E-3,6.98494E7,5.405855E6,-1.9141065E-2,-7.951701E-3,1.7329262E7,1.962943E3,3.3072104E3,2.651927E-3,8.730972E-3,4.57563E5,-7.339577E-3,-9.130585E-4,6.346931E-3,1.3425479E-2,-2.515763E-3,1.7231327E3,4.856358E-3,1.2821703E-3],"split_indices":[31,2,28,56,4,0,31,0,0,45,32,0,0,45,4,52,0,0,1,0,0,0,0,0,55,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,6.9E1,1.2E1,1.1E1,5.8E1,3E0,9E0,8E0,3E0,4.2E1,1.6E1,6E0,3E0,3E1,1.2E1,1.1E1,5E0,1E1,2E1,8E0,4E0,6E0,5E0,5E0,1.5E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.759476E-2,2.2812529E-2,-5.061224E-1,-1.1263222E-1,1.0611612E-1,-9.673372E-3,-2.8202647E-2,-1.7281608E-1,9.435549E-4,2.3669061E-1,5.7915013E-2,-2.286358E-1,-6.0411368E-2,-2.3361205E-3,2.8840434E-3,3.760488E-3,1.2037039E-2,8.164562E-3,1.3035301E-2,-3.5303698E-3,-2.7309614E-1,3.2271762E-4,-4.843644E-3,4.0815067E-3,-1.801196E-2,-5.5391397E-3,-1.4254254E-2,3.493249E-2,-2.957302E-3,-2.0820445E-3,3.3237783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,-1,-1,-1,23,-1,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[1.7723994E0,7.6791537E-1,1.513561E-1,1.7788422E-1,2.5063318E-1,0E0,0E0,9.285253E-2,4.063935E-2,4.8537076E-2,1.8853325E-1,4.889655E-2,2.5998276E-2,0E0,0E0,0E0,0E0,0E0,6.568345E-2,0E0,2.6003301E-2,0E0,0E0,0E0,5.3863045E-2,0E0,0E0,3.9567515E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,18,18,20,20,24,24,27,27],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,-1,-1,-1,24,-1,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.5414109E8,2.1647314E8,1.2096E4,3.5671377E0,2.325535E7,-9.673372E-3,-2.8202647E-2,3.6512393E2,9.18853E0,3.8198416E8,3.5010372E5,2.68E2,1.401E4,-2.3361205E-3,2.8840434E-3,3.760488E-3,1.2037039E-2,8.164562E-3,1.3E1,-3.5303698E-3,4.5859104E-1,3.2271762E-4,-4.843644E-3,4.0815067E-3,5.327983E8,-5.5391397E-3,-1.4254254E-2,7.84E3,-2.957302E-3,-2.0820445E-3,3.3237783E-3],"split_indices":[45,12,9,53,45,0,0,52,54,7,32,11,9,0,0,0,0,0,8,0,27,0,0,0,7,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.6E1,6E0,2.5E1,4.1E1,3E0,3E0,1.6E1,9E0,1E1,3.1E1,1E1,6E0,5E0,4E0,3E0,7E0,7E0,2.4E1,3E0,7E0,3E0,3E0,6E0,1.8E1,3E0,4E0,9E0,9E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.2696033E-2,-2.0930885E-1,1.13604575E-1,-2.9838702E-1,-2.7691113E-2,6.8088025E-2,3.6902377E-1,-3.4380555E-1,-3.2577056E-3,1.9661505E-3,-4.097642E-3,-5.626787E-3,8.707275E-2,2.508905E-2,6.9449507E-3,-8.780094E-3,-2.014807E-2,1.8381791E-1,5.423027E-2,3.3000312E-3,9.728457E-3,9.0375565E-2,-4.2705968E-2,-1.5911086E-4,1.0181595E-1,-4.5049815E-3,-2.5859752E-4,5.1225866E-3,1.6457686E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,-1,-1,-1,-1],"loss_changes":[1.7954413E0,3.8001347E-1,6.8404776E-1,1.352861E-1,4.9577374E-2,2.1236113E-1,3.0949605E-1,1.3253212E-1,0E0,0E0,0E0,0E0,1.4572173E-1,0E0,0E0,0E0,0E0,3.834212E-2,1.3700354E-1,0E0,0E0,3.0520767E-2,2.4622736E-2,0E0,2.3846567E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,17,17,18,18,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,4.0614333E0,3.9509753E3,1E0,9.256755E7,2.3688402E0,5.99999E9,8.2220456E5,-3.2577056E-3,1.9661505E-3,-4.097642E-3,-5.626787E-3,1.7329262E7,2.508905E-2,6.9449507E-3,-8.780094E-3,-2.014807E-2,6.498673E2,5.3920375E5,3.3000312E-3,9.728457E-3,3.786E3,1.0403953E3,-1.5911086E-4,7.53991E8,-4.5049815E-3,-2.5859752E-4,5.1225866E-3,1.6457686E-3],"split_indices":[41,54,52,16,45,54,5,28,0,0,0,0,45,0,0,0,0,4,28,0,0,2,52,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.3E1,6E1,1.5E1,8E0,5.2E1,8E0,1.2E1,3E0,4E0,4E0,4E0,4.8E1,3E0,5E0,7E0,5E0,1.1E1,3.7E1,4E0,7E0,2.7E1,1E1,3E0,2.4E1,3E0,7E0,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.4678428E-3,-2.3422952E-1,4.73607E-2,-4.2342767E-3,-1.3100968E-2,9.732043E-2,-1.4843525E-1,-3.4495117E-3,1.2342452E-1,-8.683877E-3,-1.9701239E-2,1.055637E-2,8.4631935E-2,-3.048812E-3,4.407044E-3,4.3589123E-2,1.1791568E-2,1.4780079E-2,4.264745E-3,-9.921147E-4,4.0220926E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,-1,7,9,-1,11,13,-1,-1,15,-1,-1,17,-1,19,-1,-1,-1],"loss_changes":[8.521338E-1,9.645444E-2,6.2092435E-1,0E0,0E0,2.4226391E-1,5.633207E-1,0E0,1.8979019E-1,7.7309705E-2,0E0,0E0,2.6250815E-1,0E0,0E0,4.704625E-2,0E0,6.5410495E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,9,9,12,12,15,15,17,17],"right_children":[2,4,6,-1,-1,8,10,-1,12,14,-1,-1,16,-1,-1,18,-1,20,-1,-1,-1],"split_conditions":[1.528E3,5.295928E-1,1.0232127E12,-4.2342767E-3,-1.3100968E-2,1.885E3,1.0202749E8,-3.4495117E-3,1.6256282E7,1.5E1,-1.9701239E-2,1.055637E-2,1.0011831E8,-3.048812E-3,4.407044E-3,1.2772205E3,1.1791568E-2,5.2518907E0,4.264745E-3,-9.921147E-4,4.0220926E-3],"split_indices":[11,27,31,0,0,2,45,0,45,3,0,0,45,0,0,52,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,1.2E1,6.1E1,5E0,7E0,4.9E1,1.2E1,6E0,4.3E1,9E0,3E0,9E0,3.4E1,6E0,3E0,2.9E1,5E0,2E1,9E0,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-6.2962943E-3,3.5496358E-2,-3.399379E-1,-1.1311451E-2,2.488019E-1,-2.0067945E-2,-2.6059947E-3,-6.1576482E-2,1.3076591E-1,1.859947E-2,6.391664E-3,-6.850429E-3,-1.2964153E-1,9.274452E-3,3.074991E-3,1.8079594E-2,-4.925148E-3,-7.928694E-3,-9.756059E-4,-3.9710565E-3,5.617174E-2,6.8172435E-3,5.1917305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.1275679E0,7.216886E-1,2.961781E-1,4.3654352E-1,1.7536354E-1,0E0,0E0,1.6673383E-1,6.5500766E-2,0E0,0E0,7.22945E-2,1.1492002E-1,0E0,0E0,9.818778E-2,0E0,0E0,0E0,0E0,7.534757E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,15,15,20,20],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.2159766E6,3.3795098E3,2.277961E6,5.517496E-1,7.33E2,-2.0067945E-2,-2.6059947E-3,5.6660336E-1,7.843361E5,1.859947E-2,6.391664E-3,1.6813238E-1,9.014711E-2,9.274452E-3,3.074991E-3,2.23E2,-4.925148E-3,-7.928694E-3,-9.756059E-4,-3.9710565E-3,2.325535E7,6.8172435E-3,5.1917305E-4],"split_indices":[28,4,1,41,0,0,0,27,32,0,0,42,39,0,0,10,0,0,0,0,45,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,5.9E1,1.2E1,5E0,3E0,4.4E1,1.5E1,3E0,9E0,2.5E1,1.9E1,5E0,1E1,2.1E1,4E0,1.2E1,7E0,5E0,1.6E1,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.8961756E-2,-1.8900508E-1,5.249203E-2,-2.7111888E-1,3.7989654E-3,1.823027E-2,1.40801435E-2,-1.6137661E-1,-1.659747E-2,1.5119879E-1,-4.879017E-2,-1.6829027E-3,-9.642337E-3,7.2035886E-4,2.0577563E-1,-1.814221E-2,-1.1722019E-2,4.2255507E-3,1.0558483E-2,-7.085719E-2,5.093006E-2,-1.0153317E-3,-4.9320213E-3,3.5038202E-3,-2.0127334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,-1,-1,17,19,-1,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[9.4548965E-1,5.48823E-1,5.1553446E-1,1.6849434E-1,0E0,4.5295742E-1,0E0,7.717165E-2,0E0,1.2118772E-1,2.2982162E-1,0E0,0E0,0E0,2.8395116E-2,1.16484195E-1,0E0,0E0,0E0,3.4979656E-2,4.6394322E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,14,14,15,15,19,19,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,-1,-1,18,20,-1,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,4.0230347E3,4.1E1,3.7989654E-3,2.325535E7,1.40801435E-2,8.918406E-2,-1.659747E-2,4.4939005E2,4.927835E0,-1.6829027E-3,-9.642337E-3,7.2035886E-4,1.7752522E-1,1.204781E3,-1.1722019E-2,4.2255507E-3,1.0558483E-2,5.2127117E-1,2.969772E0,-1.0153317E-3,-4.9320213E-3,3.5038202E-3,-2.0127334E-3],"split_indices":[58,4,52,8,0,45,0,57,0,4,54,0,0,0,27,52,0,0,0,27,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.2E1,5.4E1,1.7E1,5E0,4.9E1,5E0,1E1,7E0,1.6E1,3.3E1,4E0,6E0,5E0,1.1E1,3E1,3E0,4E0,7E0,1.7E1,1.3E1,9E0,8E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.6817848E-3,-2.2616802E-1,9.206123E-2,-1.0064887E-1,-4.0438184E-1,5.8719475E-2,2.2913849E-2,-7.919475E-3,-3.140276E-2,-2.117215E-2,-6.1804466E-3,-6.9123055E-3,7.877879E-2,1.6335608E-3,-4.646041E-3,1.4994971E-1,4.8611876E-2,5.5734406E-4,7.82964E-3,1.5576696E-2,1.7054671E-1,-1.7116611E-3,4.5251427E-3,1.0051072E-2,3.6281538E-3,-1.3453253E-3,1.6290755E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,-1,15,-1,-1,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1],"loss_changes":[1.7782433E0,5.185307E-1,8.415028E-1,8.7854594E-2,1.6418362E-1,2.6026985E-1,0E0,0E0,5.791353E-2,0E0,0E0,0E0,1.0435501E-1,0E0,0E0,6.568107E-2,1.5041274E-1,0E0,0E0,4.8545223E-2,2.4277136E-2,3.31714E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,12,12,15,15,16,16,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,-1,16,-1,-1,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.0789844E6,7.6312915E3,5E0,1.4235585E3,1.9956966E-1,2.2913849E-2,-7.919475E-3,9.502012E2,-2.117215E-2,-6.1804466E-3,-6.9123055E-3,2.6529046E7,1.6335608E-3,-4.646041E-3,1.0843103E5,9.599149E7,5.5734406E-4,7.82964E-3,5.2518907E0,3.882652E6,4.531609E6,4.5251427E-3,1.0051072E-2,3.6281538E-3,-1.3453253E-3,1.6290755E-3],"split_indices":[58,28,52,8,4,57,0,0,4,0,0,0,45,0,0,33,45,0,0,54,29,32,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.4E1,5.8E1,1.5E1,9E0,5.5E1,3E0,6E0,9E0,6E0,3E0,4E0,5.1E1,5E0,4E0,1.4E1,3.7E1,3E0,1.1E1,3E1,7E0,2.6E1,4E0,3E0,4E0,1.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.3843824E-2,-2.2078529E-1,8.560404E-2,-7.2355405E-4,-3.3810332E-1,4.831773E-2,2.659959E-1,-3.418276E-3,3.3625811E-3,-9.524407E-3,-2.0271981E-2,-2.8013967E-2,7.723982E-2,1.5543894E-2,5.7646437E-3,-2.8248949E-3,2.762108E-3,7.633473E-3,3.8658738E-2,1.4537007E-2,4.472216E-3,3.2292134E-3,-6.296606E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1,-1,19,21,-1,-1,-1],"loss_changes":[1.4594924E0,5.9355116E-1,3.1000277E-1,6.285858E-2,1.2970114E-1,9.267834E-2,5.332136E-2,0E0,0E0,0E0,0E0,4.654448E-2,1.1030732E-1,0E0,0E0,0E0,0E0,0E0,3.5053264E-2,3.32286E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,18,18,19,19],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1,-1,20,22,-1,-1,-1],"split_conditions":[1.0420895E-2,3.6588228E5,1.1997641E0,1.2E1,3.1457312E8,2.6277744E8,9.07E2,-3.418276E-3,3.3625811E-3,-9.524407E-3,-2.0271981E-2,4.57563E5,2.325535E7,1.5543894E-2,5.7646437E-3,-2.8248949E-3,2.762108E-3,7.633473E-3,3.3017554E2,4.4200982E8,4.472216E-3,3.2292134E-3,-6.296606E-4],"split_indices":[38,28,39,3,7,12,0,0,0,0,0,1,45,0,0,0,0,0,57,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.9E1,2.2E1,4.7E1,8E0,1.4E1,4E1,7E0,4E0,4E0,9E0,5E0,1.1E1,2.9E1,3E0,4E0,8E0,3E0,7E0,2.2E1,1.7E1,5E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-7.5184467E-4,-1.3201915E-1,1.3373545E-1,8.0386795E-2,-2.009081E-1,8.526345E-2,2.3981012E-2,-1.3384914E-3,1.4867929E-1,-2.0083936E-2,-1.5691182E-1,1.7738946E-1,4.602597E-2,8.84728E-3,2.2792364E-3,-1.8371418E-1,-1.0070925E-3,3.6515344E-3,1.0255674E-2,3.3308978E-3,5.2403803E-3,-1.181424E-2,-1.4481658E-1,1.7038058E-3,-2.2174134E-3,-2.304634E-3,-8.811093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,15,17,19,-1,-1,21,-1,-1,-1,23,-1,-1,25,-1,-1,-1,-1],"loss_changes":[1.4652659E0,6.3071203E-1,8.008842E-1,8.855475E-2,3.046682E-1,1.2919757E-1,0E0,0E0,2.7837962E-2,0E0,9.933102E-2,4.2705685E-2,9.1066234E-2,0E0,0E0,4.639697E-2,0E0,0E0,0E0,4.0348608E-2,0E0,0E0,9.466842E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,10,10,11,11,12,12,15,15,19,19,22,22],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,16,18,20,-1,-1,22,-1,-1,-1,24,-1,-1,26,-1,-1,-1,-1],"split_conditions":[4.283039E1,3.459181E-1,7.6312915E3,2.84E2,7E0,2.156825E7,2.3981012E-2,-1.3384914E-3,4.31E2,-2.0083936E-2,1.517E3,7.285458E-1,3.981664E7,8.84728E-3,2.2792364E-3,7.446808E-2,-1.0070925E-3,3.6515344E-3,1.0255674E-2,7.640048E7,5.2403803E-3,-1.181424E-2,1.5368E4,1.7038058E-3,-2.2174134E-3,-2.304634E-3,-8.811093E-3],"split_indices":[58,27,52,0,3,45,0,0,10,0,0,39,48,0,0,58,0,0,0,45,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,4.1E1,4E1,1E1,3.1E1,3.7E1,3E0,4E0,6E0,3E0,2.8E1,1E1,2.7E1,3E0,3E0,2.3E1,5E0,5E0,5E0,1.8E1,9E0,5E0,1.8E1,1.1E1,7E0,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-5.1782955E-3,-3.2325856E-2,1.3359509E-2,5.334816E-2,-1.058763E-1,-2.037616E-2,1.0874377E-1,-1.9263065E-1,-1.8165844E-2,-7.343333E-2,3.0553695E-3,9.943217E-3,7.100378E-2,-1.1755063E-2,-4.6106526E-3,3.9332747E-2,-1.1032636E-2,-5.766646E-3,-1.1908424E-3,5.5161244E-3,4.94738E-4,4.2416505E-3,-5.339266E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,-1,5,7,9,11,13,15,17,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5672225E-1,4.4674364E-1,0E0,1.369879E-1,2.856527E-1,7.832603E-2,7.2243735E-2,1.0291326E-1,2.8594252E-1,2.4242405E-2,0E0,0E0,5.3651884E-2,0E0,0E0,5.3966694E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,18,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,5.460315E9,1.3359509E-2,7.767353E2,2.667437E6,3.109375E0,3.1841638E5,1.2439503E3,9.71518E5,1.5039519E5,3.0553695E-3,9.943217E-3,1.0191781E1,-1.1755063E-2,-4.6106526E-3,3.309969E0,-1.1032636E-2,-5.766646E-3,-1.1908424E-3,5.5161244E-3,4.94738E-4,4.2416505E-3,-5.339266E-4],"split_indices":[52,5,0,4,1,53,32,4,9,33,0,0,56,0,0,54,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,3.2E1,3.7E1,1.4E1,1.8E1,1.8E1,1.9E1,9E0,5E0,3E0,1.5E1,8E0,1E1,1.6E1,3E0,3E0,6E0,7E0,8E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.7208477E-2,-3.595266E-2,3.2588106E-1,-1.5382302E-1,4.6290033E-2,2.2359533E-2,4.7245156E-3,-1.9070506E-2,-1.029633E-1,-5.3705927E-3,6.273173E-2,3.1597181E-3,-1.292435E-1,1.0151825E-1,1.1840872E-2,-1.8279533E-1,-2.5630025E-2,9.364669E-6,1.3129927E-1,-2.2221901E-3,2.9114087E-3,-1.1613733E-2,-4.7571445E-3,1.0529151E-3,-4.045425E-3,8.268006E-3,8.933545E-2,3.8189502E-4,4.9239676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,15,17,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.3408766E0,6.8569875E-1,4.5140564E-1,3.8495845E-1,1.2656963E-1,0E0,0E0,0E0,1.303899E-1,0E0,7.5779244E-2,0E0,1.2293485E-1,6.634076E-2,6.813111E-2,6.50807E-2,3.352238E-2,0E0,2.6989102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3874164E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,12,12,13,13,14,14,15,15,16,16,18,18,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,16,18,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[4.9891987E3,1.1576994E-2,8.982776E9,7E0,7.97E2,2.2359533E-2,4.7245156E-3,-1.9070506E-2,2.5346E3,-5.3705927E-3,5.090285E7,3.1597181E-3,1.7742582E3,2.727E3,2.6597537E3,5.9072212E-5,9.635135E7,9.364669E-6,1.6256282E7,-2.2221901E-3,2.9114087E-3,-1.1613733E-2,-4.7571445E-3,1.0529151E-3,-4.045425E-3,8.268006E-3,1.8E9,3.8189502E-4,4.9239676E-3],"split_indices":[4,38,5,3,10,0,0,0,32,0,45,0,4,2,4,38,45,0,45,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.9E1,1.1E1,2.8E1,4.1E1,5E0,6E0,3E0,2.5E1,3E0,3.8E1,3E0,2.2E1,2.1E1,1.7E1,1.4E1,8E0,5E0,1.6E1,8E0,9E0,5E0,9E0,5E0,3E0,5E0,1.1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.5003342E-2,-1.2385285E-1,3.199236E-2,-1.8356408E-1,3.4853884E-3,1.691761E-1,-7.5208065E-3,-9.232601E-2,-1.5646424E-2,9.289425E-3,1.3160805E-3,1.6775912E-2,-9.5922705E-3,-6.052781E-3,-3.775368E-4,-5.8194143E-3,3.479608E-2,9.331157E-2,1.233866E-2,4.0741113E-5,5.9582987E-3,4.076601E-2,-2.2735174E-3,-1.9654573E-3,2.943318E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,15,-1,-1,-1,-1,17,19,21,-1,-1,23,-1,-1,-1],"loss_changes":[4.3214613E-1,3.5830086E-1,2.6688004E-1,3.3218026E-1,0E0,6.647059E-2,2.0918739E-1,6.4500496E-2,0E0,0E0,0E0,1.0389972E-1,0E0,0E0,0E0,0E0,4.2214192E-2,3.8120978E-2,4.871987E-2,0E0,0E0,4.6971135E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,11,11,16,16,17,17,18,18,21,21],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,16,-1,-1,-1,-1,18,20,22,-1,-1,24,-1,-1,-1],"split_conditions":[7.2921924E-2,1.0735684E3,3.5175372E5,9.29295E5,3.4853884E-3,2.2980049E10,2.292109E7,1.348668E6,-1.5646424E-2,9.289425E-3,1.3160805E-3,2.074E3,-9.5922705E-3,-6.052781E-3,-3.775368E-4,-5.8194143E-3,1.7329262E7,4.5243898E-1,5.451104E5,4.0741113E-5,5.9582987E-3,2.721693E0,-2.2735174E-3,-1.9654573E-3,2.943318E-3],"split_indices":[42,52,32,28,0,5,47,1,0,0,0,2,0,0,0,0,45,27,28,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.7E1,4.8E1,2.1E1,6E0,1E1,3.8E1,1.5E1,6E0,7E0,3E0,3.5E1,3E0,9E0,6E0,3E0,3.2E1,8E0,2.4E1,3E0,5E0,1.7E1,7E0,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.6820382E-2,-7.763189E-3,-1.58089E-2,-1.0773235E-1,4.673664E-2,-1.5367822E-1,-8.975431E-3,1.9179458E-2,9.261775E-2,-2.6514006E-3,-1.9804698E-1,3.894548E-3,-3.1720933E-3,3.398798E-2,-2.6368145E-3,5.70798E-3,1.2813766E-3,-1.15429815E-2,-5.0636507E-3,2.8144785E-3,-2.9021094E-3,-2.6330808E-3,2.4026027E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,-1,5,7,9,11,13,15,-1,17,-1,-1,19,-1,-1,-1,-1,-1,-1,21,-1,-1],"loss_changes":[7.2253686E-1,3.8641617E-1,0E0,1.1205751E-1,5.638393E-2,5.6869358E-2,6.5465905E-2,3.800945E-2,3.9521575E-2,0E0,3.1004459E-2,0E0,0E0,3.0717986E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.852589E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,20,20],"right_children":[2,4,-1,6,8,10,12,14,16,-1,18,-1,-1,20,-1,-1,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.5414109E8,1.4007953E1,-1.58089E-2,4.593E3,1.0379711E0,1.762456E3,2.2858976E8,2.5922572E7,7.7591565E8,-2.6514006E-3,1.6633E4,3.894548E-3,-3.1720933E-3,9.343137E0,-2.6368145E-3,5.70798E-3,1.2813766E-3,-1.15429815E-2,-5.0636507E-3,2.8144785E-3,2.916545E3,-2.6330808E-3,2.4026027E-3],"split_indices":[45,58,0,2,42,47,7,32,7,0,9,0,0,57,0,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,2.4E1,4.5E1,1.6E1,8E0,2.9E1,1.6E1,6E0,1E1,3E0,5E0,2.5E1,4E0,9E0,7E0,4E0,6E0,1.3E1,1.2E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[4.8719868E-2,5.610019E-3,3.0250698E-1,-8.749326E-2,5.7985973E-2,2.1753406E-2,3.4324657E-3,-5.387793E-3,-2.0549142E-1,1.3562806E-1,2.2639867E-2,-5.5633806E-2,4.1130465E-3,-1.1024251E-2,-3.2354558E-3,9.438401E-4,9.133674E-3,-6.299642E-3,3.610687E-3,1.8283814E-4,-4.117503E-3,3.986649E-2,-5.6452423E-2,3.4658436E-3,-1.1015756E-3,-5.1110643E-3,-5.280906E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[7.5321287E-1,2.9765582E-1,4.1739923E-1,2.1203533E-1,1.03958875E-1,0E0,0E0,7.67995E-2,4.1613758E-2,1.0151906E-1,5.0604362E-2,2.4766162E-2,0E0,0E0,0E0,0E0,0E0,4.855776E-2,0E0,0E0,0E0,3.1815737E-2,2.633889E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,17,17,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[6.5317163E3,1.7489342E-2,1.3946067E7,9.635135E7,1.7329262E7,2.1753406E-2,3.4324657E-3,5.7630615E2,9.29342E5,4.4690585E2,5.0175633E0,5.7269626E0,4.1130465E-3,-1.1024251E-2,-3.2354558E-3,9.438401E-4,9.133674E-3,6.650418E-2,3.610687E-3,1.8283814E-4,-4.117503E-3,3.309969E0,4.828561E-1,3.4658436E-3,-1.1015756E-3,-5.1110643E-3,-5.280906E-4],"split_indices":[4,38,47,45,45,0,0,52,1,52,54,56,0,0,0,0,0,38,0,0,0,54,39,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,5.9E1,9E0,2.1E1,3.8E1,4E0,5E0,1.3E1,8E0,1.1E1,2.7E1,9E0,4E0,5E0,3E0,5E0,6E0,1.9E1,8E0,4E0,5E0,1E1,9E0,6E0,4E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[5.1178336E-3,-3.5064798E-2,3.4833393E-1,7.9573144E-4,-2.754099E-1,2.4909087E-2,5.7377797E-3,-9.032387E-2,5.5872798E-2,4.037428E-3,-1.9131992E-2,-1.1880068E-1,2.2681134E-3,1.655477E-1,1.7621385E-2,-7.609299E-2,-1.0638993E-2,2.5833421E-3,9.279461E-3,-3.336372E-3,3.7486546E-2,7.7561266E-3,-1.1866389E-1,3.4736774E-3,4.4034184E-3,3.5901912E-3,-3.009756E-3,-8.278113E-3,-3.0867772E-3,-2.1095695E-3,2.4652558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.184477E0,6.638746E-1,3.7315917E-1,3.4631148E-1,6.7827475E-1,0E0,0E0,1.1244637E-1,1.7747727E-1,0E0,0E0,1.0320121E-1,0E0,4.6391904E-2,6.562217E-2,6.5171964E-2,0E0,0E0,0E0,0E0,4.0163472E-2,4.7587115E-2,2.7812913E-2,0E0,5.1483214E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,14,14,15,15,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.4899646E8,1.523967E6,5.666351E7,2.7307262E5,2.4909087E-2,5.7377797E-3,1.7493458E3,1.7329262E7,4.037428E-3,-1.9131992E-2,1.0789844E6,2.2681134E-3,4.189857E8,4.57563E5,2.5722395E3,-1.0638993E-2,2.5833421E-3,9.279461E-3,-3.336372E-3,5.090285E7,5.0415697E5,2.6812036E2,3.4736774E-3,2.916545E3,3.5901912E-3,-3.009756E-3,-8.278113E-3,-3.0867772E-3,-2.1095695E-3,2.4652558E-3],"split_indices":[52,45,29,12,28,0,0,4,45,0,0,28,0,7,1,47,0,0,0,0,45,28,4,0,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,7.6E1,8E0,6.7E1,9E0,3E0,5E0,2.5E1,4.2E1,3E0,6E0,2.1E1,4E0,1E1,3.2E1,1.7E1,4E0,4E0,6E0,5E0,2.7E1,6E0,1.1E1,1.1E1,1.6E1,3E0,3E0,3E0,8E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5637407E-2,-2.5006428E-2,3.3905062E-1,-1.5941723E-1,4.133777E-2,2.185099E-2,4.2470414E-3,-7.091135E-2,-1.4007748E-2,5.673808E-2,-6.247722E-3,1.7179417E-3,-1.20544404E-1,1.5288186E-1,1.4230473E-2,-1.2896355E-3,-6.2999893E-3,2.6257324E-3,9.143504E-3,4.1992106E-2,-2.3015393E-3,6.445971E-2,-9.3140965E-4,-7.411087E-4,3.5054372E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,-1,15,17,19,-1,-1,-1,-1,21,-1,23,-1,-1,-1],"loss_changes":[1.0600588E0,6.486411E-1,3.2020152E-1,3.339417E-1,1.4973304E-1,0E0,0E0,9.857167E-2,0E0,1.858192E-1,0E0,0E0,2.4601787E-2,6.491217E-2,6.4603835E-2,0E0,0E0,0E0,0E0,3.5791658E-2,0E0,2.720429E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,12,12,13,13,14,14,19,19,21,21],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,-1,16,18,20,-1,-1,-1,-1,22,-1,24,-1,-1,-1],"split_conditions":[4.9891987E3,7.940699E-3,8.982776E9,1.0370839E6,2.953641E5,2.185099E-2,4.2470414E-3,4.4092423E-1,-1.4007748E-2,1.6256282E7,-6.247722E-3,1.7179417E-3,7.4338E4,4.189857E8,4.800481E5,-1.2896355E-3,-6.2999893E-3,2.6257324E-3,9.143504E-3,6.3315526E8,-2.3015393E-3,3.786E3,-9.3140965E-4,-7.411087E-4,3.5054372E-3],"split_indices":[4,38,5,28,33,0,0,27,0,45,0,0,12,7,28,0,0,0,0,7,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,2.3E1,4.8E1,4E0,4E0,1.6E1,7E0,4.5E1,3E0,5E0,1.1E1,1.3E1,3.2E1,3E0,8E0,6E0,7E0,2.3E1,9E0,1.7E1,6E0,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.2074906E-2,-1.9096171E-2,3.4432575E-1,-5.4967962E-2,8.952426E-2,2.3409376E-2,2.3584794E-3,3.024535E-2,-8.083202E-2,-4.1524414E-4,1.2565877E-1,-5.338709E-4,3.1969172E-3,-1.3285823E-2,-5.6436E-2,7.607575E-3,1.1269858E-3,-1.4143957E-1,-1.6384881E-2,-1.0491334E-3,-7.4432874E-3,3.075574E-3,-4.2548187E-2,-4.939354E-3,1.7371443E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,-1,-1,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1],"loss_changes":[8.4546125E-1,2.9715168E-1,3.8091153E-1,1.2807064E-1,6.935024E-2,0E0,0E0,2.756178E-2,2.2919422E-1,0E0,6.794058E-2,0E0,0E0,0E0,1.3726655E-1,0E0,0E0,4.53026E-2,7.0060916E-2,0E0,0E0,0E0,7.868144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,14,14,17,17,18,18,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,-1,-1,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1],"split_conditions":[4.0230347E3,7.114597E-1,1.904071E-1,2.7307262E5,1.9222778E5,2.3409376E-2,2.3584794E-3,3.015899E-2,2.8808794E5,-4.1524414E-4,2.156825E7,-5.338709E-4,3.1969172E-3,-1.3285823E-2,1.648E3,7.607575E-3,1.1269858E-3,1.1E1,1.774E3,-1.0491334E-3,-7.4432874E-3,3.075574E-3,4.8561827E8,-4.939354E-3,1.7371443E-4],"split_indices":[52,42,38,28,28,0,0,38,28,0,45,0,0,0,11,0,0,3,10,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.4E1,6E0,5.6E1,1.8E1,3E0,3E0,1.3E1,4.3E1,5E0,1.3E1,7E0,6E0,3E0,4E1,8E0,5E0,1.2E1,2.8E1,3E0,9E0,6E0,2.2E1,8E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-5.9960326E-3,2.7045945E-2,-2.5865164E-1,5.777442E-3,1.587435E-2,3.6735178E-3,-1.904019E-2,8.117006E-2,-2.6427446E-2,-1.9208613E-3,5.3964723E-3,-4.7913524E-3,-3.201398E-3,-1.8006846E-2,4.1957563E-3,3.6515899E-3,-3.8778543E-2,-2.1457253E-3,1.9245644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,13,15,-1,-1,17,-1,-1],"loss_changes":[6.4981E-1,5.029323E-1,6.1687857E-1,1.6239382E-1,0E0,0E0,0E0,1.192089E-1,9.3102075E-2,0E0,0E0,0E0,5.8768597E-2,7.633468E-2,0E0,0E0,2.9952977E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,12,12,13,13,16,16],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,14,16,-1,-1,18,-1,-1],"split_conditions":[1.5414109E8,2.1769184E7,2.7307262E5,2.5511668E7,1.587435E-2,3.6735178E-3,-1.904019E-2,1.51E3,4.593E3,-1.9208613E-3,5.3964723E-3,-4.7913524E-3,1.6132394E2,2.217E3,4.1957563E-3,3.6515899E-3,1.143417E6,-2.1457253E-3,1.9245644E-3],"split_indices":[45,1,28,45,0,0,0,2,2,0,0,0,57,10,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.8E1,8E0,6.5E1,3E0,3E0,5E0,1.9E1,4.6E1,5E0,1.4E1,9E0,3.7E1,3.3E1,4E0,5E0,2.8E1,2.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-5.711259E-3,-5.9429295E-2,9.795722E-2,-6.701965E-3,-1.8904324E-1,1.8801034E-4,1.24479495E-1,2.6698522E-2,-1.5843248E-1,-8.706835E-2,-2.6738772E-1,1.4957869E-1,6.092195E-4,-4.602374E-3,5.181901E-3,-1.15465885E-2,-3.0981613E-4,-3.0473905E-4,-5.7135304E-3,-1.41528435E-2,-5.1776837E-3,9.188629E-3,3.5230974E-3,2.6963288E-2,-3.8325759E-3,-2.293203E-3,5.869189E-3,-1.458305E-3,2.412053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,25,-1,27,-1,-1,-1],"loss_changes":[4.4019318E-1,3.541113E-1,6.645164E-2,1.9680908E-1,9.6556544E-2,0E0,5.5971086E-2,9.518956E-2,1.1612977E-1,2.8811987E-2,3.433603E-2,5.6880474E-2,0E0,6.950994E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1619308E-2,0E0,3.1797156E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,23,23,25,25],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,26,-1,28,-1,-1,-1],"split_conditions":[5.517496E-1,8.28645E-1,1.9222778E5,1.0129377E12,9.29295E5,1.8801034E-4,9.0161455E-1,2.8303965E3,4.077E3,1.2015E4,7.2767865E2,7.33E2,6.092195E-4,3.4941578E6,5.181901E-3,-1.15465885E-2,-3.0981613E-4,-3.0473905E-4,-5.7135304E-3,-1.41528435E-2,-5.1776837E-3,9.188629E-3,3.5230974E-3,3.241969E8,-3.8325759E-3,4.0614333E0,5.869189E-3,-1.458305E-3,2.412053E-3],"split_indices":[42,27,28,31,28,0,27,4,0,9,52,0,0,51,0,0,0,0,0,0,0,0,0,7,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,5.1E1,2.6E1,3.7E1,1.4E1,6E0,2E1,3.1E1,6E0,7E0,7E0,1.6E1,4E0,2.4E1,7E0,3E0,3E0,3E0,4E0,4E0,3E0,7E0,9E0,1.8E1,6E0,1.5E1,3E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.9165456E-4,-3.8781587E-2,3.4417918E-1,5.3044194E-3,-1.817248E-1,1.99703E-2,4.473567E-3,-8.6738095E-2,2.9731214E-2,-8.603092E-2,-1.6856587E-2,-2.9449616E-2,-8.294046E-3,7.4777864E-3,4.1837315E-3,-7.3961904E-3,5.6383194E-4,-3.3043532E-3,2.122967E-3,4.2452343E-2,-3.60083E-3,2.4153173E-3,-2.0879728E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,-1,-1,-1],"loss_changes":[1.0559442E0,4.546932E-1,1.8653047E-1,1.2848383E-1,3.1171566E-1,0E0,0E0,6.799406E-2,6.7467555E-2,1.14397846E-1,0E0,3.7877567E-2,0E0,1.15803E-1,0E0,0E0,0E0,0E0,0E0,3.605155E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,13,13,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,-1,-1,-1],"split_conditions":[4.0230347E3,1.0104842E12,8.982776E9,2.98E2,3.5715E4,1.99703E-2,4.473567E-3,1E0,1.204781E3,5.034889E6,-1.6856587E-2,1.58966E5,-8.294046E-3,6.98494E7,4.1837315E-3,-7.3961904E-3,5.6383194E-4,-3.3043532E-3,2.122967E-3,5.557455E8,-3.60083E-3,2.4153173E-3,-2.0879728E-3],"split_indices":[52,31,5,0,9,0,0,105,52,29,0,1,0,45,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.1E1,7E0,5.5E1,1.6E1,4E0,3E0,1.1E1,4.4E1,1.2E1,4E0,8E0,3E0,3.4E1,1E1,6E0,6E0,5E0,3E0,2.5E1,9E0,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.0148763E-2,5.6554414E-3,-2.7899873E-1,-9.1230854E-2,5.1017262E-2,-2.023174E-2,-6.473222E-4,-1.2074006E-1,3.3835531E-3,1.2188841E-1,1.9186279E-2,-6.4867088E-3,-9.940204E-4,8.8833953E-4,7.2855134E-3,-1.8867245E-2,7.7614225E-2,-1.7115562E-3,2.611574E-3,5.4111253E-3,1.0735798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,-1,-1,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[5.1589304E-1,3.1669497E-1,3.5136843E-1,1.251744E-1,1.0820043E-1,0E0,0E0,5.6821316E-2,0E0,6.99919E-2,7.928299E-2,0E0,0E0,0E0,0E0,3.9651208E-2,3.278754E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,15,15,16,16],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,-1,-1,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[1.5414109E8,4.279412E0,1.6E1,1.0735684E3,1.6256282E7,-2.023174E-2,-6.473222E-4,1.6825E4,3.3835531E-3,4.4939005E2,3.029697E3,-6.4867088E-3,-9.940204E-4,8.8833953E-4,7.2855134E-3,5.0175633E0,1.0129377E12,-1.7115562E-3,2.611574E-3,5.4111253E-3,1.0735798E-3],"split_indices":[45,58,3,52,45,0,0,9,0,4,4,0,0,0,0,54,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,2.2E1,4.8E1,3E0,3E0,1.9E1,3E0,1.4E1,3.4E1,1.4E1,5E0,5E0,9E0,2.1E1,1.3E1,1.7E1,4E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.2194844E-4,-2.5736807E-2,1.3198032E-2,1.7811785E-2,-1.6511537E-1,-6.985885E-2,5.2434742E-2,-6.9658205E-2,-1.958286E-2,-1.434458E-1,3.2796732E-3,2.7613118E-2,4.6206024E-3,-7.3789353E-3,2.8558375E-4,-7.75484E-3,-1.3593952E-3,3.210601E-3,-1.9622368E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,-1,5,7,9,11,13,-1,15,-1,17,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.18537E-1,4.3389258E-1,0E0,1.7061093E-1,4.4328308E-1,1.835511E-1,5.223558E-2,1.1111727E-1,0E0,4.0782303E-2,0E0,1.0461639E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,11,11],"right_children":[2,4,-1,6,8,10,12,14,-1,16,-1,18,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.660861E3,1.0232127E12,1.3198032E-2,5.02409E5,1.5135763E8,5.596155E-1,1.7830602E3,4.375793E6,-1.958286E-2,1.3513911E1,3.2796732E-3,5.7229916E7,4.6206024E-3,-7.3789353E-3,2.8558375E-4,-7.75484E-3,-1.3593952E-3,3.210601E-3,-1.9622368E-3],"split_indices":[52,31,0,1,45,39,52,1,0,56,0,45,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7E1,5E0,5.4E1,1.6E1,1.5E1,3.9E1,1.3E1,3E0,1E1,5E0,2.8E1,1.1E1,5E0,8E0,7E0,3E0,1.7E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.6400775E-2,-2.2976097E-2,2.3160684E-1,4.491022E-3,-2.2267409E-1,1.7653229E-2,1.8473951E-3,-1.3448575E-1,2.57857E-2,-1.4549338E-2,-8.578628E-4,-1.8978072E-3,-7.6834317E-3,3.6859646E-2,-3.6640125E-3,1.03758745E-1,1.8760912E-2,8.905121E-4,5.829795E-3,1.7850982E-3,2.6431878E-3,1.0591097E-3,-2.6358312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,15,-1,17,19,-1,-1,21,-1,-1,-1],"loss_changes":[6.578303E-1,3.6301848E-1,3.9185107E-1,1.7807628E-1,1.8372184E-1,0E0,0E0,2.4599954E-2,6.734595E-2,0E0,0E0,0E0,0E0,5.617126E-2,0E0,2.4171598E-2,2.8390463E-2,0E0,0E0,4.3796003E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,13,13,15,15,16,16,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,16,-1,18,20,-1,-1,22,-1,-1,-1],"split_conditions":[2.6949062E3,1.5135763E8,1.3946067E7,2.1E2,3.435678E8,1.7653229E-2,1.8473951E-3,1E0,1E0,-1.4549338E-2,-8.578628E-4,-1.8978072E-3,-7.6834317E-3,1.7329262E7,-3.6640125E-3,8.747313E6,3.1672727E3,8.905121E-4,5.829795E-3,7.76264E7,2.6431878E-3,1.0591097E-3,-2.6358312E-3],"split_indices":[52,45,47,10,12,0,0,104,105,0,0,0,0,45,0,45,4,0,0,45,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.5E1,1.1E1,5.8E1,7E0,5E0,6E0,7E0,5.1E1,4E0,3E0,3E0,4E0,4.7E1,4E0,9E0,3.8E1,3E0,6E0,2.8E1,1E1,2.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.6840015E-2,3.5289768E-2,-7.284617E-2,-4.283189E-3,5.815482E-2,-1.2934384E-1,2.505884E-4,-2.6282337E-3,7.1713045E-2,-1.9054475E-1,-2.6570966E-2,1.3743454E-1,4.287798E-2,-4.481184E-3,-1.0651398E-2,2.2508444E-3,-3.204954E-3,1.8109176E-3,7.608443E-3,-5.149703E-5,3.8947552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,-1,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.186203E-1,1.2661085E-1,1.6083947E-1,0E0,5.954753E-2,1.2702271E-1,0E0,0E0,5.2854672E-2,3.9152443E-2,3.842946E-2,2.711241E-2,4.897837E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,8,8,9,9,10,10,11,11,12,12],"right_children":[2,4,6,-1,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7376137E-1,3.7605792E2,1.6595918E-1,-4.283189E-3,2.23E2,1.4235585E3,2.505884E-4,-2.6282337E-3,6.1577463E0,9.29295E5,1.5368E4,2.1262457E0,1.0403953E3,-4.481184E-3,-1.0651398E-2,2.2508444E-3,-3.204954E-3,1.8109176E-3,7.608443E-3,-5.149703E-5,3.8947552E-3],"split_indices":[27,4,39,0,10,4,0,0,56,28,9,56,52,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3.8E1,3.5E1,5E0,3.3E1,2E1,1.5E1,3E0,3E1,1.2E1,8E0,8E0,2.2E1,6E0,6E0,3E0,5E0,3E0,5E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.2130767E-2,-8.4802E-3,2.1888897E-1,-7.729291E-2,4.2447068E-2,1.9493025E-2,5.2220885E-2,-2.2486696E-2,-5.272389E-3,4.797697E-3,7.241265E-4,-1.3039544E-3,5.2119405E-3,-5.7250157E-2,3.3669428E-3,-4.7865384E-3,-4.2821982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,15,-1,-1,-1],"loss_changes":[4.4755512E-1,2.310697E-1,3.7111074E-1,6.781736E-2,6.571936E-2,0E0,4.4047922E-2,5.3770743E-2,0E0,0E0,0E0,0E0,0E0,2.7291413E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,13,13],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,16,-1,-1,-1],"split_conditions":[6.7780703E3,6.0199605E6,9.192915E0,5.8871865E-1,1.7329262E7,1.9493025E-2,5.2E1,4.2332153E0,-5.272389E-3,4.797697E-3,7.241265E-4,-1.3039544E-3,5.2119405E-3,2.0435429E8,3.3669428E-3,-4.7865384E-3,-4.2821982E-4],"split_indices":[4,51,57,27,45,0,8,54,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.4E1,9E0,2.7E1,3.7E1,3E0,6E0,1.3E1,1.4E1,9E0,2.8E1,3E0,3E0,1E1,3E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.708013E-3,-1.3678592E-2,2.6891476E-1,4.5294645E-3,-1.7356527E-1,2.0133464E-2,-9.042819E-6,-5.253121E-2,4.2906906E-2,3.5952816E-3,-1.475177E-2,-7.14166E-2,2.2408804E-3,1.0652952E-1,2.2932407E-2,6.7398176E-3,-4.3697716E-3,-8.8201766E-4,7.0119365E-3,5.8309175E-3,2.9843901E-3,-1.8544922E-3,2.8945482E-3,1.4451584E-3,-1.5126783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[5.128834E-1,2.2786173E-1,3.8045102E-1,1.5773278E-1,3.822744E-1,0E0,0E0,6.1211832E-2,5.220271E-2,0E0,0E0,6.0807794E-2,0E0,7.64219E-2,2.6837666E-2,2.7366389E-2,0E0,0E0,0E0,3.115299E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,14,14,15,15,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,1.1408508E9,3.1457312E8,2.7307262E5,2.0133464E-2,-9.042819E-6,4.7985487E0,1.6256282E7,3.5952816E-3,-1.475177E-2,1.5368E4,2.2408804E-3,1.9222778E5,5.0175633E0,1.5023E4,-4.3697716E-3,-8.8201766E-4,7.0119365E-3,3.28464E0,2.9843901E-3,-1.8544922E-3,2.8945482E-3,1.4451584E-3,-1.5126783E-3],"split_indices":[52,45,7,7,28,0,0,54,45,0,0,9,0,28,54,9,0,0,0,54,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.7E1,6E0,7E1,7E0,3E0,3E0,2.8E1,4.2E1,3E0,4E0,2.4E1,4E0,9E0,3.3E1,7E0,1.7E1,3E0,6E0,2.5E1,8E0,4E0,3E0,1.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.4178472E-3,-1.9839479E-2,1.3543539E-2,2.2233801E-3,-5.4569315E-2,-9.610083E-2,1.768151E-2,-6.870575E-2,-8.305915E-3,2.7290715E-3,-1.0951506E-3,-4.373685E-3,-8.121649E-3,2.6205312E-3,-2.762623E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,-1,5,7,9,11,-1,-1,-1,-1,13,-1,-1],"loss_changes":[4.2823428E-1,1.7497307E-1,0E0,0E0,1.4273147E-1,6.730312E-2,3.758589E-2,5.0016E-2,0E0,0E0,0E0,0E0,4.315765E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,-1,-1,6,8,10,12,-1,-1,-1,-1,14,-1,-1],"split_conditions":[2.1769184E7,3.6130127E-1,1.3543539E-2,2.2233801E-3,4.283039E1,2.0333653E5,5.4734116E9,5.4111224E2,-8.305915E-3,2.7290715E-3,-1.0951506E-3,-4.373685E-3,1.5326E4,2.6205312E-3,-2.762623E-3],"split_indices":[1,27,0,0,58,33,5,52,0,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.8E1,3E0,2.2E1,4.6E1,2.9E1,1.7E1,2.4E1,5E0,8E0,9E0,1.5E1,9E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.3094041E-3,-2.1720128E-2,1.2349375E-2,-1.1930148E-1,6.5481183E-3,-8.695391E-4,-6.0114884E-3,2.242943E-2,-8.788545E-3,-4.956161E-3,3.6805633E-3,-1.9529995E-2,3.2511167E-3,2.3623288E-2,-2.6314133E-3,3.5430347E-3,-4.166359E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,7,-1,-1,9,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[4.5034984E-1,1.966314E-1,0E0,3.0829445E-2,1.9320983E-1,0E0,0E0,9.279443E-2,0E0,4.6333138E-2,0E0,6.127045E-2,0E0,2.484786E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,9,9,11,11,13,13],"right_children":[2,4,-1,6,8,-1,-1,10,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[7.6312915E3,3.731E3,1.2349375E-2,4.0246898E-1,2.5805172E7,-8.695391E-4,-6.0114884E-3,7.9276E-2,-8.788545E-3,7.0826065E1,3.6805633E-3,6.98494E7,3.2511167E-3,2.3209375E6,-2.6314133E-3,3.5430347E-3,-4.166359E-5],"split_indices":[52,11,0,27,47,0,0,38,0,57,0,45,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7E1,4E0,1.5E1,5.5E1,3E0,1.2E1,5.2E1,3E0,3.7E1,1.5E1,3.2E1,5E0,1.6E1,1.6E1,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[6.4163175E-3,-1.9764232E-2,2.2387463E-1,6.819803E-3,-2.43258E-1,3.4413047E-3,1.637866E-2,3.8325094E-2,-3.4364402E-2,-3.923652E-3,-1.3505083E-2,-2.6299434E-3,5.8338765E-2,-5.6182365E-3,-1.8718965E-2,-4.3122875E-4,7.7148154E-2,1.7720513E-3,-1.6198772E-3,9.911954E-4,4.2697773E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,15,-1,17,-1,19,-1,-1,-1,-1],"loss_changes":[4.769585E-1,4.472614E-1,1.7455706E-1,8.954841E-2,5.895999E-2,0E0,0E0,8.15047E-2,4.3076247E-2,0E0,0E0,0E0,4.324872E-2,0E0,3.2692313E-2,0E0,2.9536694E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,12,12,14,14,16,16],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,16,-1,18,-1,20,-1,-1,-1,-1],"split_conditions":[4.9891987E3,1.15266744E8,1.3996E4,5.090285E7,9.3570206E5,3.4413047E-3,1.637866E-2,2.0435429E8,7.801514E2,-3.923652E-3,-1.3505083E-2,-2.6299434E-3,2.5630938E5,-5.6182365E-3,1.120848E5,-4.3122875E-4,6.8177136E2,1.7720513E-3,-1.6198772E-3,9.911954E-4,4.2697773E-3],"split_indices":[4,45,9,45,28,0,0,7,4,0,0,0,28,0,32,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,6.7E1,7E0,5E0,3E0,3.8E1,2.9E1,3E0,4E0,6E0,3.2E1,3E0,2.6E1,7E0,2.5E1,6E0,2E1,8E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.0528256E-3,-2.9798124E-2,2.4247636E-1,3.5122198E-3,-1.5853176E-1,1.5186589E-2,5.5053935E-4,-5.8366675E-2,2.030846E-2,-1.118973E-2,9.6615084E-4,-7.1998354E-4,-4.8383037E-3,4.579047E-3,3.7899753E-3,2.2516385E-2,-2.525277E-3,2.4012732E-3,-2.783299E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,15,17,-1,-1,-1],"loss_changes":[5.892119E-1,2.9215178E-1,2.2822142E-1,5.834864E-2,2.6743665E-1,0E0,0E0,2.5392842E-2,6.2660165E-2,0E0,0E0,0E0,0E0,0E0,4.604858E-2,2.965433E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,14,14,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,16,18,-1,-1,-1],"split_conditions":[3.3072104E3,1.0789844E6,8.982776E9,2.566528E0,1.650581E3,1.5186589E-2,5.5053935E-4,1.9227438E5,1.3024019E7,-1.118973E-2,9.6615084E-4,-7.1998354E-4,-4.8383037E-3,4.579047E-3,5.451104E5,1E0,-2.525277E-3,2.4012732E-3,-2.783299E-4],"split_indices":[52,28,5,54,4,0,0,33,45,0,0,0,0,0,28,104,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,5.4E1,1.3E1,5E0,3E0,1.1E1,4.3E1,8E0,5E0,7E0,4E0,6E0,3.7E1,2.9E1,8E0,1.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.8821817E-2,6.513124E-4,1.5755745E-2,1.5206267E-2,-1.16494335E-2,-8.7076865E-2,3.607094E-2,3.1107303E-3,-6.085089E-3,9.0867594E-2,1.5483406E-2,1.9333474E-3,6.3988105E-3,2.9734147E-3,4.296543E-3,-4.5213415E-4,2.0260846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,13,-1,-1,15,-1,-1,-1],"loss_changes":[5.1433367E-1,3.1415313E-1,0E0,1.6319874E-1,0E0,1.2508008E-1,6.966513E-2,0E0,0E0,3.8847312E-2,4.9672425E-2,0E0,0E0,2.5765456E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9,10,10,13,13],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,14,-1,-1,16,-1,-1,-1],"split_conditions":[7.6312915E3,1.5277152E8,1.5755745E-2,1.7363004E4,-1.16494335E-2,1.2015E4,2.156825E7,3.1107303E-3,-6.085089E-3,4.1108544E8,3.4409692E3,1.9333474E-3,6.3988105E-3,4.9153895E0,4.296543E-3,-4.5213415E-4,2.0260846E-3],"split_indices":[52,45,0,51,0,9,45,0,0,7,52,0,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,7.4E1,3E0,1.2E1,6.2E1,3E0,9E0,1.6E1,4.6E1,1E1,6E0,4.1E1,5E0,3.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.4485215E-2,2.5037797E-3,-1.8230584E-1,-3.8953675E-3,2.7529225E-2,-2.2674387E-3,-1.13755455E-2,8.252429E-3,9.065614E-2,-3.3313275E-4,4.0266677E-3,7.7218297E-3,-5.221486E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,-1,7,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[2.1714033E-1,1.6646607E-1,6.1647728E-2,0E0,6.7397416E-2,0E0,0E0,6.137479E-2,1.2244543E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,8,8],"right_children":[2,4,6,-1,8,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.648E3,4.66E2,-3.8953675E-3,1.891862E3,-2.2674387E-3,-1.13755455E-2,1.0379711E0,1.3946067E7,-3.3313275E-4,4.0266677E-3,7.7218297E-3,-5.221486E-4],"split_indices":[45,11,0,0,52,0,0,42,47,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,1.4E1,5.5E1,3E0,3E0,4.3E1,1.2E1,3.7E1,6E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-5.363816E-3,-1.0751262E-1,2.6641496E-2,-2.35924E-2,-9.854531E-3,-6.254303E-2,3.694581E-2,-2.992021E-3,4.2011673E-3,8.7164465E-4,-5.552096E-3,5.8769984E-3,2.695744E-2,6.0710985E-2,-4.393295E-5,-6.2641635E-4,3.477225E-3,-1.0101526E-3,3.0231779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,-1,-1,-1,-1,-1,13,15,17,-1,-1,-1,-1],"loss_changes":[2.742166E-1,2.0089172E-1,6.1125763E-2,8.032447E-2,0E0,4.170724E-2,5.4508865E-2,0E0,0E0,0E0,0E0,0E0,4.921784E-2,3.8453184E-2,5.3374946E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,12,12,13,13,14,14],"right_children":[2,4,6,8,-1,10,12,-1,-1,-1,-1,-1,14,16,18,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,9.29295E5,2.074E3,1E0,-9.854531E-3,1.297959E7,2.2235682E6,-2.992021E-3,4.2011673E-3,8.7164465E-4,-5.552096E-3,5.8769984E-3,5.090285E7,2.5630938E5,4.9767124E1,-6.2641635E-4,3.477225E-3,-1.0101526E-3,3.0231779E-3],"split_indices":[41,28,2,16,0,12,51,0,0,0,0,0,45,28,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,1.9E1,6.3E1,1.2E1,7E0,6E0,5.7E1,9E0,3E0,3E0,3E0,4E0,5.3E1,2.3E1,3E1,5E0,1.8E1,2.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.693405E-2,4.13869E-3,1.9617298E-1,-5.844114E-2,3.7422627E-2,1.499067E-2,2.589713E-3,-1.5262753E-1,-1.8087281E-3,4.163394E-3,8.438932E-4,-1.6765323E-3,-1.0071796E-2,-1.434124E-3,2.4229684E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9238567E-1,1.4382444E-1,1.6719195E-1,1.2773997E-1,4.578729E-2,0E0,0E0,7.538779E-2,3.1581394E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9891987E3,2.8506322E-2,3.6274084E9,3.0810812E0,3.5175372E5,1.499067E-2,2.589713E-3,9.29295E5,9.402116E2,4.163394E-3,8.438932E-4,-1.6765323E-3,-1.0071796E-2,-1.434124E-3,2.4229684E-3],"split_indices":[4,39,12,53,32,0,0,28,52,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,2.3E1,4.4E1,3E0,5E0,8E0,1.5E1,9E0,3.5E1,4E0,4E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.064792E-2,2.0973708E-2,-1.3646895E-1,-4.197483E-2,4.283103E-2,-5.771324E-2,-1.2751524E-2,-4.0191924E-3,2.9526069E-3,2.0472784E-2,4.0910444E-3,-4.918086E-3,8.017551E-4,3.567889E-3,-8.048857E-3,-1.6376469E-3,1.1935147E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1,15,-1,-1],"loss_changes":[3.1717682E-1,8.985457E-2,1.8657237E-1,1.04654364E-1,5.474507E-2,5.472601E-2,0E0,0E0,0E0,6.352257E-2,0E0,0E0,0E0,0E0,2.8259704E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,14,14],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.0232127E12,2.4786325E0,1.3946397E5,1E0,1.7830602E3,7.575E3,-1.2751524E-2,-4.0191924E-3,2.9526069E-3,1.6939252E7,4.0910444E-3,-4.918086E-3,8.017551E-4,3.567889E-3,3.0546596E0,-1.6376469E-3,1.1935147E-3],"split_indices":[31,56,33,101,52,2,0,0,0,45,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,6.3E1,1.5E1,1.6E1,4.7E1,1.1E1,4E0,1.1E1,5E0,3.4E1,1.3E1,6E0,5E0,1E1,2.4E1,1.3E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-2.2003432E-3,1.3855694E-2,-1.1434812E-1,-1.1150522E-2,6.817658E-2,3.4854615E-3,-9.6584875E-3,-2.1333362E-3,2.6386695E-2,3.7836058E-3,2.591604E-4,4.293631E-3,1.0609669E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,-1,-1,-1],"loss_changes":[1.2943046E-1,8.633688E-2,2.1496859E-1,6.5295205E-2,2.4684705E-2,0E0,0E0,0E0,4.447379E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.1808436E3,2.7307262E5,1.9129465E0,4.815659E6,3.4854615E-3,-9.6584875E-3,-2.1333362E-3,9.9948E4,3.7836058E-3,2.591604E-4,4.293631E-3,1.0609669E-5],"split_indices":[45,52,28,57,29,0,0,0,29,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.2E1,8E0,4.3E1,1.9E1,3E0,5E0,2.1E1,2.2E1,1.4E1,5E0,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.0043612E-3,1.9909082E-2,-1.1606994E-2,7.1236794E-3,1.0069891E-2,-4.2050085E-3,1.6898869E-2,-5.4443326E-2,2.8062813E-2,1.2662858E-4,-4.317341E-3,7.0129715E-2,1.371399E-2,-1.040023E-3,4.195067E-3,-3.697128E-4,3.686076E-3,1.6813657E-3,-1.3889614E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,-1,-1,7,9,11,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[3.968184E-1,1.9676611E-1,0E0,7.27905E-2,0E0,0E0,5.250129E-2,2.4330903E-2,3.2840434E-2,0E0,0E0,3.9370775E-2,4.3927696E-2,0E0,0E0,4.842964E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,11,11,12,12,15,15],"right_children":[2,4,-1,6,-1,-1,8,10,12,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[2.9851845E10,2.1747866E7,-1.1606994E-2,1.09E2,1.0069891E-2,-4.2050085E-3,1.1E1,5.86E2,1.6939252E7,1.2662858E-4,-4.317341E-3,1.9222778E5,2.001E3,-1.040023E-3,4.195067E-3,4.0526348E-1,3.686076E-3,1.6813657E-3,-1.3889614E-3],"split_indices":[5,1,0,10,0,0,3,0,45,0,0,28,0,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.1E1,4E0,6.8E1,3E0,5E0,6.3E1,8E0,5.5E1,4E0,4E0,1.3E1,4.2E1,3E0,1E1,3.6E1,6E0,1.6E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.200589E-3,-2.1732036E-2,1.0016172E-2,-2.6431836E-3,-1.3602486E-1,-3.1311153E-3,1.9123519E-2,3.4403803E-3,-2.2726008E-1,2.9781288E-3,-7.403925E-3,-1.426959E-2,-1.5997718E-3,-1.9696243E-3,8.791814E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1],"loss_changes":[3.075554E-1,1.6763279E-1,0E0,1.0432693E-1,2.3553672E-1,0E0,7.020253E-2,0E0,1.4915979E-1,0E0,3.877893E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,10,10],"right_children":[2,4,-1,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.0016172E-2,6.942296E2,2.7307262E5,-3.1311153E-3,5.7229916E7,3.4403803E-3,1.835052E6,2.9781288E-3,1.077646E3,-1.426959E-2,-1.5997718E-3,-1.9696243E-3,8.791814E-4],"split_indices":[52,45,0,4,28,0,45,0,32,0,52,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.6E1,4E0,6.6E1,1E1,1.5E1,5.1E1,3E0,7E0,1.7E1,3.4E1,4E0,3E0,1.4E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.6793303E-2,-9.393666E-3,-5.1272856E-3,5.90596E-3,-8.185342E-3,-7.9633825E-2,2.5657946E-2,-1.709952E-3,-6.7926743E-3,4.0292032E-2,-1.619516E-3,1.4328537E-2,3.7665039E-3,2.9905946E-3,-1.9751016E-2,-2.4464256E-3,1.8638973E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,-1,3,5,-1,7,9,-1,-1,11,-1,13,-1,-1,15,-1,-1],"loss_changes":[1.740114E-1,0E0,1.4705747E-1,1.1693473E-1,0E0,3.447441E-2,5.358901E-2,0E0,0E0,5.5809796E-2,0E0,6.0126282E-2,0E0,0E0,4.9942862E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,9,9,11,11,14,14],"right_children":[2,-1,4,6,-1,8,10,-1,-1,12,-1,14,-1,-1,16,-1,-1],"split_conditions":[7E0,-9.393666E-3,2.292109E7,1.5278552E4,-8.185342E-3,9.1569895E-1,6.349519E6,-1.709952E-3,-6.7926743E-3,1.1808436E3,-1.619516E-3,1.1795429E9,3.7665039E-3,2.9905946E-3,3.5671377E0,-2.4464256E-3,1.8638973E-3],"split_indices":[3,0,47,51,0,27,32,0,0,52,0,5,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3E0,7E1,6.7E1,3E0,1.2E1,5.5E1,9E0,3E0,4.5E1,1E1,3E1,1.5E1,1.1E1,1.9E1,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.1337942E-3,-1.0757713E-2,8.935454E-3,2.2692056E-3,-1.2832415E-1,-5.536229E-3,1.3209011E-2,-9.376748E-3,-2.2647734E-4,-3.0485902E-3,2.391797E-2,1.5370602E-2,4.47338E-3,-2.4051809E-4,4.9921643E-2,3.2920255E-3,-3.1927234E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,-1,9,-1,-1,-1,11,13,-1,-1,15,-1,-1],"loss_changes":[1.8265042E-1,1.0749818E-1,0E0,9.394156E-2,7.716199E-2,0E0,5.599886E-2,0E0,0E0,0E0,3.563524E-2,3.655076E-2,0E0,0E0,3.0081086E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,10,10,11,11,14,14],"right_children":[2,4,-1,6,8,-1,10,-1,-1,-1,12,14,-1,-1,16,-1,-1],"split_conditions":[6.5915E4,3.698723E10,8.935454E-3,7.446808E-2,3.132E3,-5.536229E-3,1.296E3,-9.376748E-3,-2.2647734E-4,-3.0485902E-3,1.7823958E0,1.2544873E3,4.47338E-3,-2.4051809E-4,6.7584877E8,3.2920255E-3,-3.1927234E-4],"split_indices":[2,5,0,58,0,0,2,0,0,0,41,52,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.9E1,3E0,6.3E1,6E0,4E0,5.9E1,3E0,3E0,6E0,5.3E1,4.9E1,4E0,3.1E1,1.8E1,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-7.395596E-3,-8.771179E-3,3.8474181E-3,1.6381849E-2,-7.050952E-3,3.8326194E-3,9.11174E-3,-3.9967686E-2,2.9378345E-2,6.267657E-3,-4.503994E-3,1.7474805E-3,-1.9000736E-3,-3.121586E-3,1.3755604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,-1,3,5,-1,7,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[1.6528434E-1,0E0,1.5286219E-1,1.6456185E-1,0E0,7.278447E-2,0E0,7.318258E-2,3.657425E-2,3.3141453E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,9,9],"right_children":[2,-1,4,6,-1,8,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[5E0,-8.771179E-3,1.3866357E8,3.9509753E3,-7.050952E-3,2.881172E8,9.11174E-3,8.0159146E-1,2.5922572E7,1.001E3,-4.503994E-3,1.7474805E-3,-1.9000736E-3,-3.121586E-3,1.3755604E-3],"split_indices":[3,0,48,52,0,7,0,27,32,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3E0,7E1,6.6E1,4E0,6.3E1,3E0,2.3E1,4E1,1.4E1,9E0,3.5E1,5E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.0723137E-3,-8.5909795E-3,1.0697194E-2,9.614017E-4,-8.649597E-3,8.117288E-3,-6.012843E-3,-3.850339E-3,1.4552777E-2,6.908579E-3,3.7556298E-3,5.6384385E-2,-6.7104334E-3,3.755906E-4,4.1453713E-3,-3.2477104E-3,2.124768E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,-1,13,15,-1,-1,-1,-1],"loss_changes":[2.557413E-1,1.5745021E-1,0E0,8.4211655E-2,0E0,5.121307E-2,0E0,0E0,4.1400604E-2,4.6871368E-2,0E0,2.8104484E-2,4.4851087E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,11,11,12,12],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,-1,14,16,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,1.0697194E-2,2.2306405E6,-8.649597E-3,5.734767E-2,-6.012843E-3,-3.850339E-3,4.9891987E3,1.7329262E7,3.7556298E-3,2.9187737E0,2.35764E5,3.755906E-4,4.1453713E-3,-3.2477104E-3,2.124768E-4],"split_indices":[1,2,0,28,0,58,0,0,4,45,0,54,29,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.4E1,3E0,8.1E1,3E0,7.8E1,3E0,4E0,7.4E1,6.8E1,6E0,1.4E1,5.4E1,7E0,7E0,7E0,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.8342175E-2,4.210557E-3,1.2976714E-2,1.2470733E-2,-6.84129E-3,-1.9795425E-2,2.2920885E-3,2.4096826E-3,-8.463627E-3,-2.6712036E-2,4.958187E-2,-4.322486E-2,1.5195858E-3,4.8055174E-3,2.8589224E-5,-2.848112E-3,1.1254408E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,-1,7,-1,9,-1,11,13,15,-1,-1,-1,-1,-1],"loss_changes":[3.4119606E-1,1.1320739E-1,0E0,1.0589931E-1,0E0,1.7846255E-1,0E0,5.764757E-2,0E0,2.8260658E-2,4.906754E-2,3.589257E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,9,9,10,10,11,11],"right_children":[2,4,-1,6,-1,8,-1,10,-1,12,14,16,-1,-1,-1,-1,-1],"split_conditions":[9.55721E3,2.5805172E7,1.2976714E-2,1.7807747E3,-6.84129E-3,1.052686E6,2.2920885E-3,4.3652397E8,-8.463627E-3,2.1E1,2.325535E7,2.68219E5,1.5195858E-3,4.8055174E-3,2.8589224E-5,-2.848112E-3,1.1254408E-3],"split_indices":[4,47,0,4,0,28,0,7,0,3,45,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8.1E1,3E0,7.8E1,3E0,4.4E1,3.4E1,4E1,4E0,2.5E1,1.5E1,2E1,5E0,6E0,9E0,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.5442309E-2,5.7833386E-4,-1.7371845E-1,-2.5272978E-2,4.9490523E-2,-9.4285025E-4,-1.3678969E-2,-4.053253E-2,1.8985323E-3,1.0105279E-1,4.503017E-3,-5.2775275E-2,9.0473064E-4,2.198966E-3,8.546961E-3,-2.745345E-3,1.4006452E-3,-3.7324363E-3,-1.9081002E-2,-1.4987033E-3,2.28966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,-1,-1,-1,-1,-1,-1,19,-1,-1],"loss_changes":[2.207051E-1,1.02406316E-1,1.7029025E-1,5.8144998E-2,6.449538E-2,0E0,0E0,3.434737E-2,0E0,5.347757E-2,3.3156414E-2,4.1352957E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.506599E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,18,18],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,-1,-1,-1,-1,-1,-1,20,-1,-1],"split_conditions":[1.5135763E8,1.204781E3,1.6867E4,1.062872E6,1.4302E4,-9.4285025E-4,-1.3678969E-2,2.1E1,1.8985323E-3,9.698E3,1.2204E4,2.289789E6,9.0473064E-4,2.198966E-3,8.546961E-3,-2.745345E-3,1.4006452E-3,-3.7324363E-3,5.0175633E0,-1.4987033E-3,2.28966E-3],"split_indices":[45,52,9,9,10,0,0,3,0,2,2,51,0,0,0,0,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,5.2E1,2.7E1,4E0,3E0,4.3E1,9E0,1.2E1,1.5E1,3.6E1,7E0,9E0,3E0,4E0,1.1E1,1.7E1,1.9E1,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.1001062E-2,2.2680014E-2,-5.52554E-2,-3.5939754E-3,1.5027825E-3,-8.571232E-2,6.935895E-4,-6.4120725E-2,-7.5964686E-3,1.452593E-3,-8.235601E-2,-4.3654586E-3,-4.5523487E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,-1,7,-1,9,-1,-1,11,-1,-1],"loss_changes":[1.17514804E-1,6.280247E-2,7.540477E-2,0E0,0E0,3.6246404E-2,0E0,4.035978E-2,0E0,0E0,2.4378404E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,10,10],"right_children":[2,4,6,-1,-1,8,-1,10,-1,-1,12,-1,-1],"split_conditions":[7.8345644E-1,3.235021E2,1.6925E4,-3.5939754E-3,1.5027825E-3,1.58206E6,6.935895E-4,2.202021E8,-7.5964686E-3,1.452593E-3,7.0333635E6,-4.3654586E-3,-4.5523487E-4],"split_indices":[27,4,10,0,0,28,0,7,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,4.4E1,3.3E1,4E0,4E1,2.3E1,1E1,2E1,3E0,3E0,1.7E1,1.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.6991522E-2,5.2764895E-3,1.0940148E-2,1.7171655E-2,-7.856569E-3,6.267542E-2,1.7175402E-3,-4.366747E-4,4.047201E-3,-1.8609276E-3,1.6430588E-2,4.1084653E-3,8.903775E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[2.3927875E-1,1.8975322E-1,0E0,5.544676E-2,0E0,4.7565363E-2,4.064727E-2,0E0,0E0,0E0,5.236745E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,10,10],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.0940148E-2,1.7329262E7,-7.856569E-3,1.63254E5,5.2886304E2,-4.366747E-4,4.047201E-3,-1.8609276E-3,2.5036643E8,4.1084653E-3,8.903775E-5],"split_indices":[52,45,0,45,0,1,52,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8.2E1,3E0,7.8E1,4E0,1.9E1,5.9E1,6E0,1.3E1,1.4E1,4.5E1,6E0,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1138222E-2,-8.2536833E-4,1.058927E-2,-4.8880085E-2,1.2394956E-3,-1.3220548E-2,-1.345789E-1,-2.3657857E-3,1.2821914E-3,-8.202078E-3,-6.4570416E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,-1],"loss_changes":[2.3503102E-1,1.16019346E-1,0E0,9.295617E-2,0E0,4.3495543E-2,5.8636725E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,-1],"split_conditions":[7.6312915E3,1.03593215E-1,1.058927E-2,1.052622E6,1.2394956E-3,1.3E1,1.4235585E3,-2.3657857E-3,1.2821914E-3,-8.202078E-3,-6.4570416E-4],"split_indices":[52,42,0,28,0,3,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,3E1,4.9E1,2.2E1,8E0,1.1E1,1.1E1,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-4.331802E-3,5.3937766E-3,-7.135269E-3,-5.879663E-3,1.42568E-2,-1.999665E-3,3.19095E-3,4.591095E-2,-2.3248835E-2,-3.5294245E-3,3.557072E-3,-3.568722E-2,1.6566316E-3,-1.7328085E-2,-3.385792E-3,-3.899823E-3,-5.204307E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,-1,5,7,-1,9,11,-1,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.3999574E-1,1.08338825E-1,0E0,0E0,7.865125E-2,6.6223264E-2,0E0,1.02607176E-1,3.6079135E-2,0E0,0E0,2.8967366E-2,0E0,3.3211436E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,8,8,11,11,13,13],"right_children":[2,4,-1,-1,6,8,-1,10,12,-1,-1,14,-1,16,-1,-1,-1],"split_conditions":[1.3866357E8,7E0,-7.135269E-3,-5.879663E-3,3.029697E3,1.7329262E7,3.19095E-3,8.47E2,5.0175633E0,-3.5294245E-3,3.557072E-3,6.9E1,1.6566316E-3,2.045738E0,-3.385792E-3,-3.899823E-3,-5.204307E-5],"split_indices":[48,3,0,0,4,45,0,2,54,0,0,8,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.3E1,4E0,4E0,7.9E1,6.3E1,1.6E1,1.9E1,4.4E1,4E0,1.5E1,3.7E1,7E0,2.7E1,1E1,4E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.3860375E-3,-4.46792E-3,8.615266E-3,1.099775E-2,-1.1051276E-1,-3.062494E-2,2.1599693E-2,-8.0041E-3,-5.024926E-4,-3.2655252E-3,8.333703E-4,2.990377E-3,3.322378E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9781733E-1,1.1601921E-1,0E0,2.8020937E-2,6.594029E-2,3.128498E-2,3.2877292E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,8.615266E-3,3.7997437E2,9.1686124E-1,1.5039519E5,9.693049E2,-8.0041E-3,-5.024926E-4,-3.2655252E-3,8.333703E-4,2.990377E-3,3.322378E-4],"split_indices":[52,45,0,52,27,33,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.9E1,4E0,6.1E1,8E0,1.2E1,4.9E1,4E0,4E0,6E0,6E0,1E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-6.192797E-3,7.832392E-3,-1.456706E-1,-1.7304857E-3,2.0539446E-2,-1.2593521E-4,-1.2301277E-2,5.294579E-3,9.689084E-3,6.0695782E-2,-7.5284606E-3,6.346825E-4,4.7458704E-3,-2.6058521E-2,2.1714324E-2,-3.308736E-3,-3.6548026E-4,3.087236E-3,-4.0802493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,13,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[1.7133056E-1,4.993025E-2,1.6127712E-1,0E0,7.051483E-2,0E0,0E0,0E0,5.2271336E-2,3.293893E-2,2.4950016E-2,0E0,0E0,2.450012E-2,2.9088743E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9,10,10,13,13,14,14],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,14,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[1.5414109E8,1.528E3,1.0232127E12,-1.7304857E-3,7.446808E-2,-1.2593521E-4,-1.2301277E-2,5.294579E-3,1.7329262E7,4.189857E8,1E0,6.346825E-4,4.7458704E-3,1.1017415E1,5.81609E-1,-3.308736E-3,-3.6548026E-4,3.087236E-3,-4.0802493E-4],"split_indices":[45,11,31,0,57,0,0,0,45,7,16,0,0,56,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,1.6E1,6.3E1,4E0,3E0,5E0,5.8E1,1.4E1,4.4E1,8E0,6E0,2.7E1,1.7E1,6E0,2.1E1,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.83628E-3,-5.039373E-2,1.7211659E-2,-1.6240288E-2,-5.71947E-3,2.496904E-2,-3.4287497E-3,-1.5405546E-3,3.286436E-3,1.2113629E-2,4.829595E-3,3.5186373E-3,2.8071045E-3,-9.174293E-4,1.3813911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,13,-1,-1],"loss_changes":[8.256978E-2,7.753863E-2,4.0532652E-2,4.1446827E-2,0E0,5.2913994E-2,0E0,0E0,0E0,2.7848965E-2,0E0,0E0,2.8601397E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,12,12],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,14,-1,-1],"split_conditions":[6.6504064E0,1.0789844E6,1.8199778E0,6.817E3,-5.71947E-3,1.1997641E0,-3.4287497E-3,-1.5405546E-3,3.286436E-3,1.7329262E7,4.829595E-3,3.5186373E-3,1.381E3,-9.174293E-4,1.3813911E-3],"split_indices":[58,28,39,2,0,39,0,0,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.7E1,5E1,2E1,7E0,4.7E1,3E0,1.7E1,3E0,4.2E1,5E0,4E0,3.8E1,2.1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-9.158938E-3,-3.5582323E-2,3.7559703E-2,-2.010544E-2,-6.2207812E-3,2.1761567E-3,-1.3214346E-3,-4.500451E-3,-1.081828E-2,3.6963965E-3,-2.3378816E-3,2.4388537E-2,-1.2092583E-3,-3.7122457E-4,2.3701799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,-1,13,-1,-1,-1],"loss_changes":[1.1243827E-1,9.571718E-2,3.168499E-2,3.9976224E-2,0E0,0E0,0E0,0E0,3.0754011E-2,2.5169533E-2,0E0,2.408788E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9,11,11],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,-1,14,-1,-1,-1],"split_conditions":[1.204781E3,9.8788344E5,2.3297022E7,3.235021E2,-6.2207812E-3,2.1761567E-3,-1.3214346E-3,-4.500451E-3,6.237206E5,1.3793921E3,-2.3378816E-3,2.912E3,-1.2092583E-3,-3.7122457E-4,2.3701799E-3],"split_indices":[52,28,32,4,0,0,0,0,28,4,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,5.7E1,3.2E1,5.1E1,6E0,2.7E1,5E0,4E0,4.7E1,3.6E1,1.1E1,2.2E1,1.4E1,1.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.0962158E-2,9.0959285E-5,9.184452E-3,1.080748E-2,-8.360569E-3,-3.0612098E-3,9.848634E-2,-2.9778656E-2,2.138885E-2,8.029899E-3,1.4280527E-3,-3.3120395E-4,-6.106778E-3,2.3738118E-3,-2.2544239E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7501944E-1,1.6141982E-1,0E0,8.9068815E-2,0E0,4.244449E-2,5.184953E-2,7.609519E-2,3.122276E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,9.184452E-3,4.9891987E3,-8.360569E-3,7.2767865E2,7.33E2,1.0789844E6,1.9188015E9,8.029899E-3,1.4280527E-3,-3.3120395E-4,-6.106778E-3,2.3738118E-3,-2.2544239E-4],"split_indices":[1,2,0,4,0,52,0,28,12,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.5E1,3E0,7.2E1,3E0,6.3E1,9E0,3E1,3.3E1,3E0,6E0,2.6E1,4E0,1.4E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.4084175E-2,2.4047477E-2,-8.06955E-3,9.160869E-3,1.3025806E-2,-2.2201473E-2,1.2218094E-3,1.5059721E-2,-3.0826356E-3,4.5753196E-3,-8.0603943E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,-1,5,-1,7,-1,9,-1,-1,-1],"loss_changes":[1.722334E-1,3.3133096E-1,0E0,4.7509026E-2,0E0,5.684841E-2,0E0,5.8127902E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7],"right_children":[2,4,-1,6,-1,8,-1,10,-1,-1,-1],"split_conditions":[1.3866357E8,7.467988E3,-8.06955E-3,3.826E3,1.3025806E-2,1.4E1,1.2218094E-3,1.5414852E5,-3.0826356E-3,4.5753196E-3,-8.0603943E-4],"split_indices":[48,4,0,2,0,3,0,32,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,7.6E1,3E0,2.9E1,4.7E1,1.7E1,1.2E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.8934305E-3,-1.2732357E-2,1.1318758E-2,-5.003322E-4,-9.431507E-3,-4.851714E-2,1.7398095E-2,-4.1011097E-3,1.5908298E-4,-3.52695E-3,2.0294997E-3,1.7986002E-3,-1.4767022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,-1,-1,-1],"loss_changes":[2.9163423E-1,1.8291733E-1,0E0,6.014445E-2,0E0,4.738724E-2,3.2247663E-2,0E0,0E0,4.5268156E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,-1,-1,-1],"split_conditions":[9.55721E3,3.698723E10,1.1318758E-2,3.2716873E0,-9.431507E-3,4.283039E1,1.1773262E3,-4.1011097E-3,1.5908298E-4,1.0197425E1,2.0294997E-3,1.7986002E-3,-1.4767022E-3],"split_indices":[4,5,0,56,0,58,52,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7.1E1,3E0,6.8E1,3E0,1.8E1,5E1,9E0,9E0,3E1,2E1,1.2E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1080338E-2,-5.3662215E-3,1.4537941E-1,1.5275009E-2,-3.3924855E-2,1.3167339E-2,-5.868377E-4,-1.960574E-3,1.1878536E-3,-3.9606597E-3,-5.337057E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6201943E-1,3.9424237E-2,2.1110065E-1,3.1076372E-2,5.358601E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4],"right_children":[2,4,6,8,10,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7780703E3,6.401022E7,9.192915E0,1.018E3,7.553E3,1.3167339E-2,-5.868377E-4,-1.960574E-3,1.1878536E-3,-3.9606597E-3,-5.337057E-5],"split_indices":[4,45,57,11,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.5E1,7E0,3.8E1,2.7E1,3E0,4E0,6E0,3.2E1,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-6.9997166E-3,4.0016603E-3,-1.12870276E-1,-1.5674913E-2,5.9529636E-2,6.331049E-4,-9.07983E-3,5.378683E-3,-2.8464128E-3,6.647169E-3,8.990292E-4,-4.099305E-3,7.325761E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[8.6641096E-2,7.522994E-2,9.203134E-2,5.499731E-2,6.320955E-2,0E0,0E0,4.5664474E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[3.698723E10,1.1879E4,1.2845453E5,6.8989144E7,1.383E4,6.331049E-4,-9.07983E-3,4.8595375E6,-2.8464128E-3,6.647169E-3,8.990292E-4,-4.099305E-3,7.325761E-4],"split_indices":[5,2,33,45,10,0,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.7E1,6E0,5E1,1.7E1,3E0,3E0,3.6E1,1.4E1,4E0,1.3E1,3E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.0791813E-2,-1.8801097E-3,-6.238033E-3,1.1297966E-2,-1.02703124E-1,1.8365833E-3,5.646304E-3,-6.797323E-3,-1.1081329E-3,-1.8478564E-3,1.7427966E-2,3.1305603E-3,2.73192E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[9.72343E-2,1.0346665E-1,0E0,7.857981E-2,3.4755453E-2,4.6406426E-2,0E0,0E0,0E0,0E0,2.9705036E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,10,10],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[1.3866357E8,1.3237324E6,-6.238033E-3,5.639591E3,1.4235585E3,6E0,5.646304E-3,-6.797323E-3,-1.1081329E-3,-1.8478564E-3,1.7329262E7,3.1305603E-3,2.73192E-4],"split_indices":[48,28,0,4,4,8,0,0,0,0,45,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.6E1,4E0,6.8E1,8E0,6.4E1,4E0,4E0,4E0,1.6E1,4.8E1,7E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.8591696E-3,9.815839E-3,-3.6795665E-2,-3.6681313E-2,2.1475587E-2,9.056731E-4,-3.727448E-3,3.821858E-4,-3.2953229E-3,3.7346368E-3,4.598712E-4,-1.3272683E-3,3.4440418E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.421554E-2,3.4942888E-2,3.9957553E-2,2.4634743E-2,3.5072025E-2,3.556776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.533027E-1,7.095772E2,4.25174E0,1.2817016E7,4.07E2,3.1023192E0,-3.727448E-3,3.821858E-4,-3.2953229E-3,3.7346368E-3,4.598712E-4,-1.3272683E-3,3.4440418E-3],"split_indices":[27,4,56,12,0,56,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,6.2E1,2E1,1.2E1,5E1,1.2E1,8E0,6E0,6E0,6E0,4.4E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.418808E-3,1.1304898E-2,-1.4341435E-1,4.081586E-3,3.8006776E-3,-1.2294907E-3,-1.0728865E-2,2.30106E-3,-4.7141784E-3,-1.7767029E-2,1.9549048E-3,-3.3403442E-3,-1.2859463E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[1.7209224E-1,3.942059E-2,9.137453E-2,2.9121613E-2,0E0,0E0,0E0,0E0,3.8331848E-2,4.073698E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.5092398E8,3.4409692E3,1.2341693E10,5.7578983E10,3.8006776E-3,-1.2294907E-3,-1.0728865E-2,2.30106E-3,5.0175633E0,3.068E3,1.9549048E-3,-3.3403442E-3,-1.2859463E-4],"split_indices":[45,52,5,31,0,0,0,0,54,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7E1,7E0,6.5E1,5E0,4E0,3E0,9E0,5.6E1,4.5E1,1.1E1,8E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.9704896E-4,9.198559E-3,-6.9760242E-3,-4.2951675E-3,1.4474942E-1,-3.2552388E-2,1.972492E-2,2.8636443E-5,1.0803786E-2,-2.906538E-4,-9.1110535E-2,3.215161E-3,1.2774585E-4,-6.389038E-3,-1.1694393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1],"loss_changes":[1.1292363E-1,1.4146137E-1,0E0,4.88327E-2,1.086466E-1,4.9151298E-2,3.597805E-2,0E0,0E0,0E0,3.3084884E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2415501E9,2.6949062E3,-6.9760242E-3,8.6646006E5,1.4099E4,1.0112447E6,2.156825E7,2.8636443E-5,1.0803786E-2,-2.906538E-4,1.9598669E6,3.215161E-3,1.2774585E-4,-6.389038E-3,-1.1694393E-3],"split_indices":[7,52,0,47,9,32,45,0,0,0,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.6E1,3E0,7E1,6E0,3.2E1,3.8E1,3E0,3E0,2.3E1,9E0,8E0,3E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.299149E-3,-6.6466117E-3,1.030528E-2,1.0409964E-3,-5.9662475E-3,-2.8077487E-3,1.1958189E-2,-2.7244217E-3,1.8385906E-2,1.0324459E-3,-2.7340848E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[2.3250343E-1,7.4712105E-2,0E0,5.1756743E-2,0E0,0E0,3.0839918E-2,0E0,2.855911E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[8.791079E3,5.673536E2,1.030528E-2,2.1113522E8,-5.9662475E-3,-2.8077487E-3,7E0,-2.7244217E-3,4.2E1,1.0324459E-3,-2.7340848E-3],"split_indices":[4,58,0,7,0,0,3,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,6.9E1,3E0,9E0,6E1,4E0,5.6E1,5.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.937586E-2,9.897601E-3,8.018543E-3,-1.009328E-2,4.4329964E-2,-4.3930426E-2,1.2423929E-2,5.9737567E-2,-2.531505E-3,1.4012291E-4,-3.0678778E-3,4.245226E-3,-1.4152724E-3,4.926671E-3,1.4901694E-3,-1.4873481E-3,1.0849742E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,-1,-1,15,-1,-1,-1,-1],"loss_changes":[1.1925925E-1,5.0695766E-2,0E0,3.6355942E-2,4.6604592E-2,2.5179353E-2,3.499613E-2,2.9121995E-2,0E0,0E0,0E0,0E0,2.4120333E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,-1,-1,16,-1,-1,-1,-1],"split_conditions":[2.1769184E7,1.077646E3,8.018543E-3,2.8586518E8,2.2167318E-1,1.0793079E5,2.526E3,7.33E2,-2.531505E-3,1.4012291E-4,-3.0678778E-3,4.245226E-3,3.3392856E0,4.926671E-3,1.4901694E-3,-1.4873481E-3,1.0849742E-3],"split_indices":[1,52,0,7,38,32,2,0,0,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,4.6E1,2.6E1,1.8E1,2.8E1,2.3E1,3E0,7E0,1.1E1,3E0,2.5E1,6E0,1.7E1,1.1E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[5.880934E-4,9.789349E-3,-7.6539125E-3,-1.275904E-2,4.629207E-2,7.9654885E-4,-5.6536244E-3,8.215862E-2,9.067132E-4,-4.1097035E-3,1.0006172E-2,5.5095456E-3,1.3004519E-3,4.3614373E-2,-1.0981136E-3,8.3502494E-2,6.353207E-4,3.145271E-4,5.1465724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,13,-1,-1,15,-1,17,-1,-1,-1],"loss_changes":[1.3558705E-1,6.479069E-2,0E0,7.9445854E-2,2.5253631E-2,4.112026E-2,0E0,2.5069006E-2,0E0,0E0,5.149105E-2,0E0,0E0,2.3995396E-2,0E0,2.436763E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,10,10,13,13,15,15],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,14,-1,-1,16,-1,18,-1,-1,-1],"split_conditions":[5.139E4,1.077646E3,-7.6539125E-3,1.0789844E6,1.4E1,8.192308E1,-5.6536244E-3,7.33E2,9.067132E-4,-4.1097035E-3,6.1577463E0,5.5095456E-3,1.3004519E-3,3.5175372E5,-1.0981136E-3,2.37E2,6.353207E-4,3.145271E-4,5.1465724E-3],"split_indices":[2,52,0,28,3,52,0,0,0,0,56,0,0,32,0,10,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,4.8E1,2.9E1,4.4E1,4E0,1.1E1,1.8E1,3E0,4.1E1,5E0,6E0,2.1E1,2E1,8E0,1.3E1,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.808404E-3,-4.0421966E-3,8.400543E-3,2.2803247E-3,-5.1991097E-3,-1.4997546E-3,1.8446412E-2,2.4496089E-3,4.447638E-6],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[1.5290117E-1,5.8241256E-2,0E0,4.593393E-2,0E0,0E0,3.7748966E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[7.6312915E3,2.5805172E7,8.400543E-3,2.881172E8,-5.1991097E-3,-1.4997546E-3,4.5503766E5,2.4496089E-3,4.447638E-6],"split_indices":[52,47,0,7,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,7.4E1,3E0,2.2E1,5.2E1,1.6E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.4626985E-2,-6.5052616E-3,-5.2099507E-3,-3.3596627E-2,6.805525E-4,-4.6231814E-2,1.6844646E-3,-1.09878994E-4,-6.898021E-2,-3.9237575E-3,-4.4140316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[7.4087046E-2,4.973244E-2,0E0,3.5942703E-2,0E0,3.1606704E-2,0E0,0E0,2.7107805E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[1.278211E8,8.6646006E5,-5.2099507E-3,2.477407E6,6.805525E-4,1.5326E4,1.6844646E-3,-1.09878994E-4,1.9598669E6,-3.9237575E-3,-4.4140316E-4],"split_indices":[48,47,0,1,0,9,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8E1,5E0,3.6E1,4.4E1,3.1E1,5E0,1.1E1,2E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.0076585E-2,6.9905276E-4,7.84219E-3,1.0549464E-2,-7.270946E-2,-3.8446547E-3,1.6663069E-2,-6.822399E-4,-5.97255E-3,9.212863E-3,4.3931943E-3,-2.595745E-3,2.2281876E-3,-1.6221173E-2,2.1725013E-3,-1.4562735E-3,1.3798906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,-1,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.2663867E-1,5.4999854E-2,0E0,4.2645644E-2,3.197294E-2,0E0,4.009278E-2,0E0,0E0,3.0525176E-2,0E0,3.5747956E-2,0E0,3.4822833E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,9,9,11,11,13,13],"right_children":[2,4,-1,6,8,-1,10,-1,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[2.1769184E7,9.167896E-1,7.84219E-3,3.235021E2,6.33068E5,-3.8446547E-3,1.1997641E0,-6.822399E-4,-5.97255E-3,9.29295E5,4.3931943E-3,5.0175633E0,2.2281876E-3,3.029697E3,2.1725013E-3,-1.4562735E-3,1.3798906E-3],"split_indices":[1,27,0,4,29,0,39,0,0,28,0,54,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.4E1,3E0,6.6E1,8E0,3E0,6.3E1,5E0,3E0,5.9E1,4E0,4.7E1,1.2E1,3.8E1,9E0,2.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[7.1125855E-3,-5.903616E-3,1.3625851E-1,6.023849E-3,-5.7360013E-3,2.0349465E-4,9.993506E-3,-1.0403292E-3,2.3224136E-2,-2.6063612E-3,1.336854E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,-1,-1,-1,-1,9,-1,-1],"loss_changes":[1.2855962E-1,1.07870355E-1,8.857702E-2,3.4498528E-2,0E0,0E0,0E0,0E0,3.0320372E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8],"right_children":[2,4,6,8,-1,-1,-1,-1,10,-1,-1],"split_conditions":[4.0230347E3,3.698723E10,2.2620792E5,3.826E3,-5.7360013E-3,2.0349465E-4,9.993506E-3,-1.0403292E-3,5E0,-2.6063612E-3,1.336854E-3],"split_indices":[52,5,28,2,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,6.4E1,5E0,3E0,3E0,2.3E1,4.1E1,3E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[6.4143366E-3,-1.3931212E-3,7.4674455E-3,-5.4263496E-3,4.5996397E-3,4.8249832E-4,-5.2348804E-2,-4.3446505E-3,1.4801146E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[1.18438624E-1,6.601547E-2,0E0,0E0,3.25617E-2,0E0,4.206098E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[2.1698572E7,5E0,7.4674455E-3,-5.4263496E-3,1.8003757E-1,4.8249832E-4,1.4E1,-4.3446505E-3,1.4801146E-3],"split_indices":[1,3,0,0,38,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,8.6E1,3E0,3E0,8.3E1,7.5E1,8E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.14453E-3,1.5539227E-2,-6.0570575E-3,2.9807705E-3,1.0567889E-2,-3.317125E-3,8.611344E-3,1.3842303E-2,-2.7028853E-3,3.1798948E-2,-4.6114207E-4,-1.2050027E-2,2.02419E-3,-1.8756003E-3,2.3341302E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,-1,-1,7,9,-1,11,-1,13,-1,-1,-1],"loss_changes":[1.1605032E-1,2.2611591E-1,0E0,3.4683798E-2,0E0,0E0,2.7547667E-2,2.9970221E-2,0E0,2.71384E-2,0E0,2.5709478E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,9,9,11,11],"right_children":[2,4,-1,6,-1,-1,8,10,-1,12,-1,14,-1,-1,-1],"split_conditions":[1.278211E8,7.467988E3,-6.0570575E-3,1.09E2,1.0567889E-2,-3.317125E-3,3.698723E10,4.5090426E8,-2.7028853E-3,1.648E3,-4.6114207E-4,1.2293572E6,2.02419E-3,-1.8756003E-3,2.3341302E-3],"split_indices":[48,4,0,10,0,0,5,7,0,11,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,7.7E1,4E0,7.4E1,3E0,4E0,7E1,6.6E1,4E0,3.8E1,2.8E1,1E1,2.8E1,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.0882834E-2,-1.1136126E-4,-1.1858166E-1,7.435513E-4,-2.6054291E-2,7.699214E-4,-9.644061E-3,-2.0954113E-3,6.8199955E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,-1,7,-1,-1,-1,-1],"loss_changes":[8.931257E-2,3.2685246E-2,1.0630635E-1,0E0,2.873138E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4],"right_children":[2,4,6,-1,8,-1,-1,-1,-1],"split_conditions":[1.5414109E8,7.455661E-1,1.6867E4,7.435513E-4,1.419E3,7.699214E-4,-9.644061E-3,-2.0954113E-3,6.8199955E-4],"split_indices":[45,27,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,4.2E1,2.8E1,3E0,3E0,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.1758199E-2,-8.113677E-3,8.1760526E-2,6.226069E-5,-5.188308E-3,1.22552544E-1,-2.0872424E-3,2.5562535E-3,1.1312912E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[1.13340646E-1,6.951435E-2,1.0185341E-1,0E0,0E0,1.0897979E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.3305548E3,1.5092398E8,1.06102504E8,6.226069E-5,-5.188308E-3,3.9509753E3,-2.0872424E-3,2.5562535E-3,1.1312912E-2],"split_indices":[52,45,51,0,0,52,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.3E1,1.7E1,5.9E1,4E0,1.3E1,4E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[8.700694E-3,-1.2264316E-3,7.6019564E-3,8.359954E-3,-5.371242E-3,5.6372937E-2,-3.6212875E-4,8.92916E-4,4.451379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,-1,5,-1,7,-1,-1,-1],"loss_changes":[1.21059716E-1,8.3321005E-2,0E0,5.2995488E-2,0E0,2.661978E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5],"right_children":[2,4,-1,6,-1,8,-1,-1,-1],"split_conditions":[2.1747866E7,1.5277152E8,7.6019564E-3,1.7329262E7,-5.371242E-3,2.9719496E0,-3.6212875E-4,8.92916E-4,4.451379E-3],"split_indices":[1,45,0,45,0,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.8E1,3E0,6.4E1,4E0,1.6E1,4.8E1,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[5.437385E-3,-1.752151E-3,6.630797E-3,-1.9731363E-2,2.8080674E-2,-7.1678157E-3,-4.330542E-3,4.5535218E-2,-1.7431164E-3,1.08149936E-4,-3.1581158E-3,9.993475E-4,4.377305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":99,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,-1,-1,-1],"loss_changes":[9.3857996E-2,4.5609206E-2,0E0,5.3566657E-2,4.0307872E-2,3.0577855E-2,0E0,3.035758E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,-1,-1,-1],"split_conditions":[2.1769184E7,1.204781E3,6.630797E-3,1.3237324E6,1.06102504E8,1E0,-4.330542E-3,4.146482E3,-1.7431164E-3,1.08149936E-4,-3.1581158E-3,9.993475E-4,4.377305E-3],"split_indices":[1,52,0,28,51,105,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,8.3E1,3E0,5.2E1,3.1E1,4.6E1,6E0,2.5E1,6E0,4.1E1,5E0,1.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"100"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-9.1459654E-2,-5.54109E-1,2.0924011E-1,-1.2843856E-1,-7.506931E-1,4.6708934E-2,4.3444833E-1,-3.00433E-1,5.9030973E-3,-4.0931627E-2,-2.0354155E-2,1.2529472E-1,-2.1817419E-1,1.044215E-2,5.1811427E-1,-1.7521013E-2,-4.962099E-3,1.7057521E-2,4.6703424E-2,-1.376186E-2,-2.5653557E-3,1.03572775E-2,2.624943E-2,1.5643975E-1,-6.770325E-2,2.220804E-3,1.0714781E-2,-2.6944376E-4,-4.600815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[1.0967188E1,2.4976635E0,1.7366686E0,5.456359E-1,7.3559284E-1,6.310014E-1,1.9185519E-1,9.352398E-2,0E0,0E0,0E0,4.759083E-1,9.545508E-2,0E0,1.946559E-1,0E0,0E0,0E0,2.4912825E-1,0E0,0E0,0E0,0E0,8.535066E-2,2.3812339E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,14,14,18,18,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[1.1576994E-2,5.8871865E-1,5.517496E-1,4.2332153E0,1.4235585E3,2.5042E4,4.650193E-1,1.264233E6,5.9030973E-3,-4.0931627E-2,-2.0354155E-2,2.9862975E5,2.9816154E1,1.044215E-2,2.4786325E0,-1.7521013E-2,-4.962099E-3,1.7057521E-2,7.75024E0,-1.376186E-2,-2.5653557E-3,1.03572775E-2,2.624943E-2,2.3797054E0,6.5957415E-1,2.220804E-3,1.0714781E-2,-2.6944376E-4,-4.600815E-3],"split_indices":[37,26,41,53,4,9,26,1,0,0,0,31,55,0,55,0,0,0,56,0,0,0,0,56,26,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3E1,4.7E1,1E1,2E1,2.8E1,1.9E1,6E0,4E0,1E1,1E1,2.2E1,6E0,7E0,1.2E1,3E0,3E0,4E0,1.8E1,3E0,3E0,4E0,8E0,9E0,9E0,5E0,4E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.1174776E-2,-4.830118E-1,2.093845E-1,-5.955108E-1,2.832456E-3,2.718551E-1,-1.1238484E-2,-4.2921847E-1,-3.8282927E-2,1.8877497E-1,2.3755293E-2,-2.0353394E-2,-6.2712687E-3,-7.041739E-3,2.28291E-1,1.6740719E-2,1.4373985E-1,-1.1000476E-3,2.088966E-1,1.0797673E-2,2.6714997E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1,15,-1,17,-1,19,-1,-1],"loss_changes":[8.775161E0,1.8931375E0,1.4578552E0,8.6177444E-1,0E0,8.8489985E-1,0E0,1.6791415E-1,0E0,5.124587E-1,0E0,0E0,0E0,0E0,3.7144732E-1,0E0,2.4829155E-1,0E0,8.6377144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,14,14,16,16,18,18],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1,16,-1,18,-1,20,-1,-1],"split_conditions":[1.0420895E-2,1.962943E3,1.4696082E12,1.2427474E8,2.832456E-3,1.1997641E0,-1.1238484E-2,1.137E3,-3.8282927E-2,3.1183178E8,2.3755293E-2,-2.0353394E-2,-6.2712687E-3,-7.041739E-3,2.325535E7,1.6740719E-2,9.9583336E1,-1.1000476E-3,1.3312784E7,1.0797673E-2,2.6714997E-3],"split_indices":[37,4,30,44,0,38,0,0,0,7,0,0,0,0,44,0,57,0,31,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.9E1,4.6E1,2.4E1,5E0,4.1E1,5E0,1.7E1,7E0,3.3E1,8E0,1.4E1,3E0,3E0,3E1,9E0,2.1E1,6E0,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[4.9319595E-2,-1.9302101E-1,3.6433876E-1,7.7906296E-2,-4.79115E-1,9.435853E-2,4.2625263E-1,1.1187466E-2,1.46483015E-2,-6.209801E-1,-2.6790038E-1,-5.8295415E-4,8.798848E-3,2.8855914E-1,2.4525782E-2,-7.1154446E-3,6.2758505E-2,-3.1027814E-2,-8.73463E-3,-2.221211E-3,-1.4641795E-2,3.4274822E-1,1.1442407E-3,9.597006E-3,1.2086243E-2,1.618916E-2,6.7794104E-3,-3.4221742E-2,4.374038E-3,-3.5373755E-3,1.3179845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,-1,23,-1,-1,-1,-1,25,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[6.3303638E0,3.6803284E0,5.615382E-1,4.37576E-1,5.2281046E-1,9.9045046E-2,4.2257786E-1,2.0275913E-1,0E0,4.1841602E-1,1.5917337E-1,0E0,0E0,2.3200428E-1,0E0,0E0,1.3654894E-1,0E0,0E0,0E0,0E0,3.2384157E-2,0E0,0E0,6.1977267E-2,0E0,0E0,3.409381E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,16,16,21,21,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,-1,24,-1,-1,-1,-1,26,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[2.1091123E-1,5.81609E-1,2.1267605E0,7.347893E5,9.2989355E2,6.11E2,1.1328785E-1,1.58966E5,1.46483015E-2,7.940699E-3,1.5326E4,-5.8295415E-4,8.798848E-3,1.6129E4,2.4525782E-2,-7.1154446E-3,4.5E1,-3.1027814E-2,-8.73463E-3,-2.221211E-3,-1.4641795E-2,5.405855E6,1.1442407E-3,9.597006E-3,1.04776725E6,1.618916E-2,6.7794104E-3,3.7582534E5,4.374038E-3,-3.5373755E-3,1.3179845E-3],"split_indices":[38,26,55,27,4,0,37,1,0,37,9,0,0,9,0,0,8,0,0,0,0,31,0,0,31,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,4.6E1,3.5E1,2.4E1,2.2E1,7E0,2.8E1,2E1,4E0,1.2E1,1E1,4E0,3E0,1.6E1,1.2E1,4E0,1.6E1,9E0,3E0,3E0,7E0,1.3E1,3E0,3E0,1.3E1,1E1,3E0,9E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.1746E-5,-3.7669906E-1,2.8866062E-1,-3.884967E-2,-6.1617005E-1,2.2242902E-1,4.041311E-2,-1.8333656E-1,1.349988E-2,-1.0974476E-2,-7.124857E-1,1.21058464E-1,4.1034156E-1,-1.5099317E-2,-5.4323003E-2,-1.7728277E-2,-3.464742E-2,-6.61903E-3,1.6944882E-1,2.1158814E-2,6.093224E-3,2.6566937E-4,-4.978143E-3,1.9736634E-1,-1.4299648E-3,2.4857002E-1,1.0594456E-1,1.2756806E-2,5.6673386E-3,-2.7505087E-4,6.629363E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,17,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,25,-1,27,29,-1,-1,-1,-1],"loss_changes":[9.024752E0,2.8886461E0,1.8330379E0,8.774959E-1,5.4436207E-1,7.884717E-1,0E0,2.4362645E-1,0E0,0E0,1.2995434E-1,4.2667007E-1,2.687955E-1,0E0,3.0791815E-2,0E0,0E0,0E0,1.5386707E-1,0E0,0E0,0E0,0E0,8.133137E-2,0E0,4.12758E-2,5.6151725E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,11,11,12,12,14,14,18,18,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,18,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,26,-1,28,30,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,7.6312915E3,8.231683E2,5.0415697E5,9.306648E-1,4.041311E-2,1.6361E4,1.349988E-2,-1.0974476E-2,6.1358623E9,4.9538263E2,3.3934937E0,-1.5099317E-2,8.62E2,-1.7728277E-2,-3.464742E-2,-6.61903E-3,8.325696E8,2.1158814E-2,6.093224E-3,2.6566937E-4,-4.978143E-3,5.405855E6,-1.4299648E-3,3.1167011E0,1.1343E4,1.2756806E-2,5.6673386E-3,-2.7505087E-4,6.629363E-3],"split_indices":[41,26,51,51,27,38,0,9,0,0,5,4,53,0,0,0,0,0,7,0,0,0,0,31,0,53,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3.5E1,4.6E1,1.5E1,2E1,4.3E1,3E0,1.1E1,4E0,5E0,1.5E1,2.9E1,1.4E1,4E0,7E0,5E0,1E1,4E0,2.5E1,1E1,4E0,4E0,3E0,2.2E1,3E0,1.3E1,9E0,8E0,5E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.464384E-4,-4.6920645E-1,1.9333163E-1,-5.952402E-1,-5.518288E-4,8.82653E-2,7.322041E-1,-1.7733527E-2,-3.505385E-2,1.2868324E-1,-1.661365E-2,3.751995E-2,1.4170783E-2,1.1289165E-2,2.404824E-1,-1.4591739E-1,7.5357206E-2,3.0781708E-3,2.721677E-1,-2.24413E-3,-8.675699E-3,1.2053321E-2,1.097327E-2,1.6005205E-2,1.8141574E-1,2.3323535E-3,-3.3507994E-3,9.380849E-3,3.6096657E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,-1,-1,-1,15,17,19,21,-1,23,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[7.1566606E0,1.3150363E0,3.1059434E0,3.8960123E-1,0E0,9.7495353E-1,2.3299408E-1,0E0,0E0,5.878903E-1,0E0,0E0,0E0,2.4265E-1,1.0320592E-1,2.660957E-2,2.2293174E-1,0E0,1.0816252E-1,0E0,0E0,0E0,5.876059E-2,0E0,2.5660902E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,13,13,14,14,15,15,16,16,18,18,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,-1,-1,-1,16,18,20,22,-1,24,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[4.823969E-3,1.962943E3,4.0230347E3,9.1569895E-1,-5.518288E-4,1.6597747E8,8.795191E-1,-1.7733527E-2,-3.505385E-2,3.839604E-1,-1.661365E-2,3.751995E-2,1.4170783E-2,6.8177136E2,2.048724E0,4.191214E5,2.325535E7,3.0781708E-3,2.4696612E0,-2.24413E-3,-8.675699E-3,1.2053321E-2,2.2968803E1,1.6005205E-2,6.3315526E8,2.3323535E-3,-3.3507994E-3,9.380849E-3,3.6096657E-3],"split_indices":[38,4,51,26,0,44,26,0,0,41,0,0,0,4,52,27,44,0,52,0,0,0,56,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.2E1,5.5E1,1.7E1,5E0,4.7E1,8E0,1.1E1,6E0,4.4E1,3E0,5E0,3E0,2.2E1,2.2E1,6E0,1.6E1,4E0,1.8E1,3E0,3E0,3E0,1.3E1,7E0,1.1E1,9E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.242935E-2,-4.2519015E-1,1.5809536E-1,-1.7596853E-1,-5.4585034E-1,-1.1979334E-2,3.1758392E-1,-1.2096516E-2,7.0671114E-5,-2.8898947E-2,-1.2043131E-2,-1.1141433E-1,1.699006E-1,3.909627E-1,1.486259E-1,1.0207987E-1,-2.1020326E-1,1.4478304E-2,6.432891E-2,6.3625416E-3,4.2305976E-1,1.0282089E-2,2.6432208E-3,7.859005E-3,-2.1978715E-4,-1.2100349E-1,-1.4820124E-2,5.283804E-3,-1.1029282E-3,2.0101778E-2,1.07203815E-2,-2.3454928E-3,-7.534415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,19,21,23,25,-1,27,-1,29,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1031775E0,5.053699E-1,1.6313026E0,1.5237364E-1,2.5601578E-1,5.614863E-1,3.2622886E-1,0E0,0E0,0E0,0E0,4.432748E-1,1.7409271E-1,1.0029149E-1,7.281983E-2,6.226304E-2,1.283493E-1,0E0,4.5889825E-2,0E0,2.8016567E-2,0E0,0E0,0E0,0E0,2.6844382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,13,13,14,14,15,15,16,16,18,18,20,20,25,25],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,20,22,24,26,-1,28,-1,30,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9956966E-1,5.0415697E5,3.8151306E-1,1.3E1,1E0,1.1808436E3,3.3027112E0,-1.2096516E-2,7.0671114E-5,-2.8898947E-2,-1.2043131E-2,2.2519132E-1,5.77E2,1.9522085E0,5.643973E8,2.3526582E3,7.1974045E-1,1.4478304E-2,1.96E2,6.3625416E-3,1.5199E4,1.0282089E-2,2.6432208E-3,7.859005E-3,-2.1978715E-4,8.07E2,-1.4820124E-2,5.283804E-3,-1.1029282E-3,2.0101778E-2,1.07203815E-2,-2.3454928E-3,-7.534415E-3],"split_indices":[56,27,41,3,15,51,53,0,0,0,0,26,0,52,7,4,26,0,8,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,1.9E1,5.9E1,7E0,1.2E1,2.9E1,3E1,4E0,3E0,7E0,5E0,1.9E1,1E1,2E1,1E1,6E0,1.3E1,3E0,7E0,3E0,1.7E1,4E0,6E0,3E0,3E0,9E0,4E0,4E0,3E0,1.2E1,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.513954E-4,-3.7165117E-1,1.9000456E-1,-4.7413486E-1,2.825124E-3,3.6059546E-1,1.2376453E-2,-3.4155813E-1,-3.2959625E-2,8.805712E-2,4.5158103E-1,6.926323E-2,-1.0766846E-2,-1.8985951E-2,-2.011872E-1,6.119726E-4,6.7662527E-3,5.690954E-1,2.4471958E-1,-9.67671E-3,1.2490984E-1,-1.9245803E-3,-1.1622075E-2,2.7504956E-2,1.1122954E-2,3.9141355E-3,1.3088083E-2,2.4368338E-1,2.5420533E-2,4.2327684E-3,1.2809196E-2,-5.441798E-3,3.8679964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,-1,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,-1,-1,29,31,-1,-1,-1,-1],"loss_changes":[5.6427393E0,1.2425084E0,1.6015294E0,6.018686E-1,0E0,6.2975526E-1,4.2316514E-1,1.550827E-1,0E0,3.913322E-2,3.4691477E-1,4.0653005E-1,0E0,0E0,8.6992264E-2,0E0,0E0,1.07441425E-1,5.655533E-2,0E0,2.301394E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.309106E-2,1.3065305E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,14,14,17,17,18,18,20,20,27,27,28,28],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,-1,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,-1,-1,30,32,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.962943E3,1.6129E4,1.2427474E8,2.825124E-3,3.3379373E-1,1.4181119E12,1.648E3,-3.2959625E-2,1.3652755E6,7.8345644E-1,2.074E3,-1.0766846E-2,-1.8985951E-2,2.2858976E8,6.119726E-4,6.7662527E-3,2.5492783E0,4.932816E6,-9.67671E-3,1.2E1,-1.9245803E-3,-1.1622075E-2,2.7504956E-2,1.1122954E-2,3.9141355E-3,1.3088083E-2,2.5630938E5,6.8175425E6,4.2327684E-3,1.2809196E-2,-5.441798E-3,3.8679964E-3],"split_indices":[41,4,9,44,0,38,30,11,0,46,26,2,0,0,7,0,0,52,1,0,3,0,0,0,0,0,0,27,49,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.6E1,5.2E1,2.1E1,5E0,2.6E1,2.6E1,1.6E1,5E0,7E0,1.9E1,2.2E1,4E0,8E0,8E0,4E0,3E0,1.1E1,8E0,3E0,1.9E1,3E0,5E0,8E0,3E0,3E0,5E0,8E0,1.1E1,3E0,5E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[6.091429E-2,-4.1792938E-1,1.8350436E-1,-5.0144905E-1,-1.7964066E-3,8.22498E-2,6.454801E-1,-4.900659E-3,-2.5916405E-2,-2.5496872E-2,2.559193E-1,3.7555333E-2,1.6550634E-2,2.6243908E-2,-1.904673E-1,3.7283357E-5,3.196267E-1,1.13695696E-1,-1.2504852E-1,-1.4939752E-3,-1.1836821E-2,1.552213E-2,6.485372E-3,1.1388198E-2,2.7914882E-2,-1.0534041E-2,-1.619807E-3,-1.9044313E-3,5.9120446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,17,19,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[4.5967064E0,4.8128414E-1,2.8470929E0,4.565587E-1,0E0,9.795654E-1,3.2176065E-1,0E0,0E0,2.8667668E-1,3.246895E-1,0E0,0E0,3.582181E-1,9.774792E-2,0E0,6.1459064E-2,2.1511662E-1,9.639382E-2,0E0,0E0,0E0,0E0,0E0,1.0208167E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,13,13,14,14,16,16,17,17,18,18,24,24],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,18,20,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[1.6403629E4,4.593E3,4.0230347E3,8.0159146E-1,-1.7964066E-3,5.517496E-1,1.3946067E7,-4.900659E-3,-2.5916405E-2,1.2932927E1,7.97E2,3.7555333E-2,1.6550634E-2,5.484294E-1,5.303548E9,3.7283357E-5,1.984018E7,1.3E1,2.8506322E-2,-1.4939752E-3,-1.1836821E-2,1.552213E-2,6.485372E-3,1.1388198E-2,8.8E1,-1.0534041E-2,-1.619807E-3,-1.9044313E-3,5.9120446E-3],"split_indices":[46,2,51,26,0,41,46,0,0,56,10,0,0,26,12,0,47,3,38,0,0,0,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,1.5E1,6.1E1,1.2E1,3E0,5.1E1,1E1,3E0,9E0,3.2E1,1.9E1,4E0,6E0,2.5E1,7E0,4E0,1.5E1,1.6E1,9E0,3E0,4E0,1.1E1,4E0,5E0,1.1E1,3E0,6E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.6044117E-2,-2.813553E-1,1.1194357E-1,-3.8589445E-1,6.365203E-3,1.5455592E-1,-8.159058E-3,-2.0304742E-1,-2.2294467E-2,-7.5167357E-3,1.8609025E-1,-1.2465446E-2,1.0527307E-3,8.595018E-2,2.8701866E-1,1.0672706E-2,2.9445041E-2,1.3768066E-2,3.6020828E-3,-3.5456778E-3,1.2604819E-1,6.465945E-4,7.6542827E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,-1,-1,15,17,-1,19,-1,-1,-1,21,-1,-1],"loss_changes":[2.6090114E0,1.2717111E0,6.1000854E-1,4.3901563E-1,0E0,4.4938153E-1,0E0,2.2868678E-1,0E0,0E0,3.492887E-1,0E0,0E0,1.7756584E-1,1.00421906E-1,0E0,1.8411915E-1,0E0,0E0,0E0,5.0150126E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,13,13,14,14,16,16,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,-1,-1,16,18,-1,20,-1,-1,-1,22,-1,-1],"split_conditions":[9.254704E-3,1.962943E3,6.2012E4,5.0415697E5,6.365203E-3,2.074E3,-8.159058E-3,1.3E1,-2.2294467E-2,-7.5167357E-3,4.828561E-1,-1.2465446E-2,1.0527307E-3,3.4941578E6,8.0933E4,1.0672706E-2,1.077646E3,1.3768066E-2,3.6020828E-3,-3.5456778E-3,9.57E2,6.465945E-4,7.6542827E-3],"split_indices":[37,4,10,27,0,2,0,3,0,0,38,0,0,50,9,0,51,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,2.6E1,4.4E1,2.1E1,5E0,3.9E1,5E0,1E1,1.1E1,3E0,3.6E1,7E0,3E0,1.9E1,1.7E1,4E0,1.5E1,1.4E1,3E0,7E0,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.375215E-2,-1.7956609E-1,3.1396008E-1,-4.475621E-1,-7.0033245E-2,2.2233696E-1,3.357808E-2,-5.835884E-1,-1.26632E-3,1.17291234E-1,-1.402892E-1,1.2604502E-2,1.09961465E-1,-1.2833614E-2,-2.8878465E-2,8.959712E-3,1.5504903E-3,-8.761031E-2,-1.3359717E-2,3.247988E-4,6.303277E-3,-1.3335772E-1,-8.037925E-3,-1.5199379E-3,-8.473441E-3,2.137595E-3,-3.3594493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[4.002802E0,1.201539E0,9.912052E-1,6.6494465E-1,4.2443517E-1,1.6288972E-1,0E0,3.1345367E-2,0E0,5.966536E-2,1.8117502E-1,0E0,4.1036606E-2,0E0,0E0,0E0,0E0,6.8092436E-2,0E0,0E0,0E0,6.900616E-2,3.6662724E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,17,17,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[4.7985215E-2,5.382431E0,7.6312915E3,7.940699E-3,2.805454E-1,2.4832625E6,3.357808E-2,2.202021E8,-1.26632E-3,4.5E1,2.0536139E1,1.2604502E-2,5.316E3,-1.2833614E-2,-2.8878465E-2,8.959712E-3,1.5504903E-3,1.7058623E3,-1.3359717E-2,3.247988E-4,6.303277E-3,6.433619E2,1.0519099E6,-1.5199379E-3,-8.473441E-3,2.137595E-3,-3.3594493E-3],"split_indices":[37,55,51,37,26,31,0,7,0,8,55,0,2,0,0,0,0,4,0,0,0,4,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,4.1E1,2.6E1,1.1E1,3E1,2.3E1,3E0,8E0,3E0,8E0,2.2E1,1.3E1,1E1,3E0,5E0,3E0,5E0,1.8E1,4E0,3E0,7E0,1.1E1,7E0,5E0,6E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.3291592E-2,-3.5239986E-1,1.72509E-1,-5.0849515E-1,5.0978072E-2,1.1366999E-1,5.429267E-1,-3.4126726E-1,-2.9658329E-2,-1.7515961E-3,5.6755748E-3,1.6279958E-1,-1.9032758E-1,2.9985784E-2,1.0644514E-2,-1.7278347E-2,-6.842589E-3,-3.87651E-3,2.166911E-1,-1.2132395E-2,-2.1109018E-3,7.0085307E-3,-8.727592E-2,1.4054082E-2,1.5595174E-1,-3.872923E-4,-5.6616897E-3,1.5735772E-3,8.481439E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,-1,25,-1,27,-1,-1,-1,-1],"loss_changes":[5.16432E0,1.8519189E0,1.0934212E0,4.131775E-1,7.307397E-2,7.329432E-1,1.5120387E-1,8.3277225E-2,0E0,0E0,0E0,3.6911952E-1,7.814169E-2,0E0,0E0,0E0,0E0,1.68213E-1,1.5955377E-1,0E0,0E0,0E0,2.7035102E-2,0E0,1.0380441E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,17,17,18,18,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,-1,26,-1,28,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,4.0230347E3,8.616169E-1,8.231683E2,1.1269586E12,2.1050402E7,1.5341808E3,-2.9658329E-2,-1.7515961E-3,5.6755748E-3,3.6732382E2,2.8312179E12,2.9985784E-2,1.0644514E-2,-1.7278347E-2,-6.842589E-3,1.7970878E6,2.325535E7,-1.2132395E-2,-2.1109018E-3,7.0085307E-3,1.2E1,1.4054082E-2,9.9583336E1,-3.872923E-4,-5.6616897E-3,1.5735772E-3,8.481439E-3],"split_indices":[41,4,51,26,51,30,46,4,0,0,0,51,30,0,0,0,0,47,44,0,0,0,3,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.8E1,5.2E1,2E1,8E0,4.6E1,6E0,1.2E1,8E0,4E0,4E0,4E1,6E0,3E0,3E0,8E0,4E0,1E1,3E1,3E0,3E0,3E0,7E0,9E0,2.1E1,3E0,4E0,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.11815E-2,-3.3522725E-1,1.8761614E-1,6.0145766E-2,-4.4398713E-1,1.0632117E-1,4.696459E-1,-1.2920295E-3,6.759094E-3,-5.660932E-1,-2.2601584E-1,2.910224E-2,1.8621325E-1,2.3904392E-2,1.1481897E-2,-2.8437966E-1,-3.473485E-2,-3.3279383E-3,-1.1625643E-2,-6.4782416E-3,8.3718695E-2,9.543453E-3,3.2117711E-3,-1.5034129E-2,-4.811055E-3,5.5344934E-3,-1.8265166E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,-1,23,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[5.2518044E0,1.4457278E0,1.0047364E0,7.55541E-2,5.640602E-1,2.2284049E-1,2.96309E-2,0E0,0E0,8.7858295E-1,5.3575993E-2,2.099497E-1,5.7148635E-2,0E0,0E0,6.437707E-2,0E0,0E0,0E0,0E0,9.798034E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,15,15,20,20],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,-1,24,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4865639E-2,3.459181E-1,1.0272479E0,8.231683E2,1.500766E3,4.7565946E-1,1.383E4,-1.2920295E-3,6.759094E-3,1.0370839E6,5.0017652E4,4.57563E5,9.19322E0,2.3904392E-2,1.1481897E-2,2.23E2,-3.473485E-2,-3.3279383E-3,-1.1625643E-2,-6.4782416E-3,3.64E2,9.543453E-3,3.2117711E-3,-1.5034129E-2,-4.811055E-3,5.5344934E-3,-1.8265166E-3],"split_indices":[37,26,38,51,4,41,10,0,0,27,32,1,55,0,0,10,0,0,0,0,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3.2E1,4.5E1,7E0,2.5E1,3.6E1,9E0,4E0,3E0,1.5E1,1E1,1.9E1,1.7E1,5E0,4E0,8E0,7E0,3E0,7E0,4E0,1.5E1,1.2E1,5E0,5E0,3E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.4895428E-3,-3.0832598E-1,1.476936E-1,7.337417E-2,-5.032524E-1,5.1306404E-2,3.429725E-1,-6.620061E-2,1.2798463E-2,-5.4586494E-1,-9.307089E-3,8.52972E-2,-1.10731E-2,4.080719E-1,3.9384817E-3,-6.1457483E-3,1.1915804E-3,-1.4732508E-2,-2.903554E-2,1.234163E-1,-7.188532E-3,2.0559082E-2,9.879167E-3,1.8509729E-1,2.3773596E-2,1.8284154E-4,2.4937104E-1,-3.5463482E-2,7.0954026E-3,1.3132916E-2,6.3187866E-3,-3.965284E-3,7.7684724E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1,25,27,-1,29,31,-1,-1,-1,-1,-1],"loss_changes":[3.7659874E0,2.0458183E0,1.0176347E0,3.351296E-1,1.0026741E-1,4.2281622E-1,2.5638676E-1,5.504486E-2,0E0,1.6813612E-1,0E0,3.5882187E-1,0E0,6.912041E-2,0E0,0E0,0E0,0E0,0E0,1.8612745E-1,0E0,0E0,0E0,2.1975332E-1,1.1535082E-1,0E0,3.675276E-2,3.2394454E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,19,19,23,23,24,24,26,26,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1,26,28,-1,30,32,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,9.306648E-1,8.231683E2,1.3513911E1,1.1696494E7,3.3934937E0,1.3E1,1.2798463E-2,2.8225487E11,-9.307089E-3,1E0,-1.10731E-2,1.3996E4,3.9384817E-3,-6.1457483E-3,1.1915804E-3,-1.4732508E-2,-2.903554E-2,5.6039695E9,-7.188532E-3,2.0559082E-2,9.879167E-3,8.1729946E10,2.0901184E3,1.8284154E-4,1.181508E6,1.742E3,7.0954026E-3,1.3132916E-2,6.3187866E-3,-3.965284E-3,7.7684724E-4],"split_indices":[41,26,38,51,55,46,53,3,0,30,0,112,0,9,0,0,0,0,0,5,0,0,0,30,51,0,31,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.6E1,5.4E1,9E0,1.7E1,3.7E1,1.7E1,6E0,3E0,1.4E1,3E0,3.4E1,3E0,1.3E1,4E0,3E0,3E0,7E0,7E0,3E1,4E0,8E0,5E0,1.8E1,1.2E1,5E0,1.3E1,9E0,3E0,7E0,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.9342616E-2,-1.177099E-1,2.5990668E-1,-3.7666702E-1,-3.965099E-2,1.0666826E-2,3.361856E-1,-6.8167797E-3,-2.0442668E-2,7.8358404E-2,-1.4977737E-1,-3.909633E-3,6.890782E-3,1.4904882E-3,3.7397426E-1,1.1430879E-2,1.0151933E-2,-8.376353E-2,-1.1135905E-2,1.761733E-2,9.761365E-3,5.960206E-3,-3.3769023E-2,-7.8526065E-3,-6.7802267E-3,1.5333147E-3,-5.919079E-3,-2.4406018E-3,2.3791464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,-1,19,-1,21,23,-1,-1,-1,-1,25,27,-1,-1,-1,-1,-1],"loss_changes":[2.8364453E0,9.5533246E-1,6.428132E-1,1.660701E-1,5.0558233E-1,1.5298896E-1,2.830248E-1,0E0,0E0,2.4214801E-1,1.2425065E-1,0E0,0E0,0E0,3.3308268E-2,0E0,8.984181E-2,7.82373E-2,0E0,0E0,0E0,0E0,9.239468E-2,2.8174143E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,14,14,16,16,17,17,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,-1,20,-1,22,24,-1,-1,-1,-1,26,28,-1,-1,-1,-1,-1],"split_conditions":[3.839604E-1,1.346E3,2.1262457E0,2.62518E5,5.81609E-1,1.6E1,3.2414E-1,-6.8167797E-3,-2.0442668E-2,4.48E2,3.6979167E0,-3.909633E-3,6.890782E-3,1.4904882E-3,1.5199E4,1.1430879E-2,1.6439099E-2,1.5326E4,-1.1135905E-2,1.761733E-2,9.761365E-3,5.960206E-3,2.1446E4,1.27426E6,-6.7802267E-3,1.5333147E-3,-5.919079E-3,-2.4406018E-3,2.3791464E-3],"split_indices":[41,11,55,28,26,8,26,0,0,0,53,0,0,0,9,0,26,9,0,0,0,0,9,1,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,4.7E1,3.3E1,1E1,3.7E1,8E0,2.5E1,4E0,6E0,1.8E1,1.9E1,5E0,3E0,3E0,2.2E1,4E0,1.4E1,1.3E1,6E0,1.6E1,6E0,3E0,1.1E1,7E0,6E0,7E0,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.996961E-2,-2.8205118E-1,1.5501395E-1,-1.55176E-1,-2.9040772E-2,6.840957E-2,5.63874E-1,-2.587841E-1,1.1721904E-2,2.6414385E-1,1.1448369E-2,3.3335984E-2,1.2408488E-2,-1.2926956E-2,-3.6485724E-3,-2.4606911E-3,2.392361E-3,4.0180907E-3,1.3522474E-2,-1.0006361E-1,6.47153E-2,-7.220365E-4,-7.224762E-3,1.0255815E-1,-3.6275077E-3,2.6295839E-2,7.0096995E-3,4.39064E-3,-2.3290648E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1,21,23,-1,-1,25,-1,27,-1,-1,-1],"loss_changes":[3.428976E0,1.2044048E0,2.0151515E0,3.6616492E-1,0E0,5.4328966E-1,3.6411428E-1,7.2318256E-2,3.0786386E-2,6.776947E-2,2.3813179E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.1633935E-2,1.6149634E-1,0E0,0E0,9.8638654E-2,0E0,7.344261E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,19,19,20,20,23,23,25,25],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1,22,24,-1,-1,26,-1,28,-1,-1,-1],"split_conditions":[4.856813E0,7.916317E11,3.9509753E3,1.6E1,-2.9040772E-2,1.7329262E7,8.982776E9,1.0370839E6,1.938E3,1.0843103E5,1.4343789E6,3.3335984E-2,1.2408488E-2,-1.2926956E-2,-3.6485724E-3,-2.4606911E-3,2.392361E-3,4.0180907E-3,1.3522474E-2,1.8523E4,2.3297022E7,-7.220365E-4,-7.224762E-3,2.1202636E-1,-3.6275077E-3,1.3397689E-1,7.0096995E-3,4.39064E-3,-2.3290648E-3],"split_indices":[57,30,51,8,0,44,5,27,2,32,46,0,0,0,0,0,0,0,0,9,31,0,0,38,0,38,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.5E1,5.7E1,2E1,5E0,4.8E1,9E0,1.2E1,8E0,1E1,3.8E1,4E0,5E0,9E0,3E0,3E0,5E0,3E0,7E0,1.2E1,2.6E1,6E0,6E0,2.1E1,5E0,1E1,1.1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.7959079E-2,3.3515338E-2,-5.439698E-1,-9.869692E-2,1.8315943E-1,-3.189193E-2,-4.5014187E-3,3.8901392E-2,-2.3159537E-1,1.0944469E-1,1.3747038E-2,-1.9519506E-2,1.2489866E-2,-1.5762134E-2,-1.3676985E-1,-6.1160773E-3,1.64186E-1,2.8373315E-3,-4.879285E-2,1.6439244E-3,-9.010175E-3,-3.0944736E-3,4.0532374E-3,2.1961888E-3,8.336786E-3,3.047169E-4,-4.52133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,-1,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.682834E0,1.5217208E0,7.183106E-1,7.5876E-1,3.3185184E-1,0E0,0E0,3.1629825E-1,2.2239149E-1,1.5915287E-1,0E0,4.8483647E-2,0E0,0E0,1.8944594E-1,6.701711E-2,4.9000174E-2,0E0,4.532664E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,14,14,15,15,16,16,18,18],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,-1,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5414109E8,3.2711282E-2,7.493E3,5.81609E-1,2.4273067E1,-3.189193E-2,-4.5014187E-3,7.347893E5,3.507E3,2.6880343E0,1.3747038E-2,1.204507E-1,1.2489866E-2,-1.5762134E-2,5.7229916E7,1.6E1,1.2E1,2.8373315E-3,2.1416E4,1.6439244E-3,-9.010175E-3,-3.0944736E-3,4.0532374E-3,2.1961888E-3,8.336786E-3,3.047169E-4,-4.52133E-3],"split_indices":[44,37,2,26,56,0,0,27,2,53,0,26,0,0,44,3,3,0,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.5E1,8E0,4E1,3.5E1,5E0,3E0,2E1,2E1,2.4E1,1.1E1,1.7E1,3E0,7E0,1.3E1,8E0,1.6E1,4E0,1.3E1,4E0,9E0,5E0,3E0,4E0,1.2E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.9505762E-2,-3.0160806E-1,1.1446385E-1,-4.101308E-1,6.7047074E-2,-8.379094E-3,2.612527E-1,-2.6891392E-1,-5.6854063E-1,5.741581E-3,-7.2406675E-4,2.5123904E-2,-1.0655792E-2,1.7354633E-1,3.8422826E-1,-3.8618457E-3,-1.3404787E-2,-1.2552934E-2,-2.8100347E-2,-4.6093785E-3,7.231362E-2,2.4235931E-4,8.942462E-3,9.910835E-3,2.09687E-2,7.163702E-3,5.9442273E-3,-1.8904549E-3,4.4926545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[3.1713533E0,1.1076999E0,1.0300117E0,3.1688452E-1,4.1787274E-2,2.644232E-1,2.1374798E-1,7.513434E-2,2.6323795E-2,0E0,0E0,1.8968515E-1,0E0,1.0032731E-1,4.8149586E-2,0E0,0E0,0E0,0E0,0E0,1.3801764E-1,0E0,0E0,0E0,0E0,0E0,7.423505E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,20,20,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,5.451147E-1,9.29295E5,1.6867E4,3.6030095E10,1.1997641E0,3.3159972E5,8.28645E-1,5.741581E-3,-7.2406675E-4,3.7818575E0,-1.0655792E-2,1.0966E4,1.3E1,-3.8618457E-3,-1.3404787E-2,-1.2552934E-2,-2.8100347E-2,-4.6093785E-3,2.491358E0,2.4235931E-4,8.942462E-3,9.910835E-3,2.09687E-2,7.163702E-3,3.6616542E0,-1.8904549E-3,4.4926545E-3],"split_indices":[41,4,41,27,9,5,38,27,26,0,0,55,0,9,3,0,0,0,0,0,52,0,0,0,0,0,52,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.6E1,5.6E1,2E1,6E0,3.1E1,2.5E1,1.2E1,8E0,3E0,3E0,2.8E1,3E0,1.6E1,9E0,3E0,9E0,3E0,5E0,7E0,2.1E1,3E0,1.3E1,5E0,4E0,8E0,1.3E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.449704E-2,-3.1889904E-1,1.4883669E-1,-4.2908838E-1,5.913231E-3,9.372266E-2,2.1409126E-2,-1.1719601E-2,-2.513279E-2,1.2944044E-1,-8.096517E-3,1.7205084E-2,1.8951772E-1,6.2284645E-2,-4.7494997E-3,1.2188283E-2,1.3342847E-1,-1.9648627E-3,4.6447357E-3,8.554429E-2,8.850229E-3,5.8247205E-3,5.207967E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,13,15,17,-1,-1,19,-1,-1,21,-1,-1,-1],"loss_changes":[3.721609E0,1.4194877E0,8.449938E-1,3.7996912E-1,0E0,4.390232E-1,0E0,0E0,0E0,2.4622273E-1,0E0,8.7870665E-2,9.0527E-2,6.0149476E-2,0E0,0E0,4.215181E-2,0E0,0E0,4.2945392E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,11,11,12,12,13,13,16,16,19,19],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,14,16,18,-1,-1,20,-1,-1,22,-1,-1,-1],"split_conditions":[1.7489342E-2,1.9315491E3,6.7780703E3,8.616169E-1,5.913231E-3,2.9461394E5,2.1409126E-2,-1.1719601E-2,-2.513279E-2,2.847953E-1,-8.096517E-3,5.303548E9,9.693049E2,4.57563E5,-4.7494997E-3,1.2188283E-2,4.946992E6,-1.9648627E-3,4.6447357E-3,5.99187E0,8.850229E-3,5.8247205E-3,5.207967E-4],"split_indices":[37,4,4,26,0,32,0,0,0,41,0,12,4,1,0,0,1,0,0,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.6E1,4.5E1,2.1E1,5E0,4E1,5E0,1.2E1,9E0,3.6E1,4E0,1.3E1,2.3E1,1E1,3E0,7E0,1.6E1,3E0,7E0,1.1E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.07692E-3,-2.7644157E-1,1.2962286E-1,-8.736741E-2,-4.1278672E-1,8.3613865E-2,4.5945916E-1,5.1238113E-3,-1.6150486E-1,-5.141799E-1,-3.715523E-3,6.8261735E-3,2.6075876E-1,2.6412223E-2,7.971721E-3,-9.983092E-3,-1.8332362E-3,-9.530817E-3,-2.7025245E-2,5.4569438E-2,-1.7903225E-1,1.6170103E-2,6.5223556E-3,1.2187874E-1,5.6436267E-3,-1.6137054E-2,6.6004094E-4,6.8556686E-4,7.955213E-3,6.314655E-2,-2.7909344E-3,-2.401067E-3,4.913517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[3.130982E0,6.8575764E-1,8.569336E-1,2.1903388E-1,4.8403072E-1,7.1915495E-1,1.8719184E-1,0E0,7.534486E-2,2.7091742E-1,0E0,3.4696224E-1,1.4558935E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.01400204E-1,3.1436688E-1,0E0,0E0,8.528942E-2,8.159262E-2,0E0,0E0,0E0,0E0,7.43649E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,19,19,20,20,23,23,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[7.2921924E-2,5.0415697E5,4.4839956E3,8.205673E-4,1.0735684E3,8.363552E-1,5.902777E-1,5.1238113E-3,6.386E3,9.29295E5,-3.715523E-3,1.0743855E-1,2.4696612E0,2.6412223E-2,7.971721E-3,-9.983092E-3,-1.8332362E-3,-9.530817E-3,-2.7025245E-2,4.1508866E5,2.8808794E5,1.6170103E-2,6.5223556E-3,1.3397689E-1,6.0487356E7,-1.6137054E-2,6.6004094E-4,6.8556686E-4,7.955213E-3,1.1171711E9,-2.7909344E-3,-2.401067E-3,4.913517E-3],"split_indices":[41,27,51,41,51,40,26,0,10,27,0,37,52,0,0,0,0,0,0,27,27,0,0,38,44,0,0,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,2.7E1,5.8E1,1.2E1,1.5E1,5.2E1,6E0,3E0,9E0,1.1E1,4E0,3.7E1,1.5E1,3E0,3E0,5E0,4E0,4E0,7E0,3E1,7E0,6E0,9E0,1.2E1,1.8E1,3E0,4E0,5E0,7E0,1E1,8E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.484174E-2,-2.0651723E-1,1.23060025E-1,9.060144E-2,-3.7930906E-1,8.6913936E-2,1.6235925E-2,-8.894108E-4,1.12425E-2,-2.322102E-1,-2.28896E-2,9.985029E-3,6.3343115E-2,-1.4884285E-2,-4.1465354E-3,1.2130751E-3,1.28519E-1,-4.1467164E-2,5.214476E-3,7.1523585E-3,2.1171435E-3,-3.2622847E-3,3.881595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,15,-1,-1,17,19,21,-1,-1,-1,-1,-1],"loss_changes":[1.8220457E0,1.3178582E0,4.4004917E-1,1.9740945E-1,2.4275732E-1,1.4414191E-1,0E0,0E0,0E0,1.2293494E-1,0E0,0E0,1.6586871E-1,0E0,0E0,1.1841317E-1,5.2930146E-2,8.520052E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,16,-1,-1,18,20,22,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,5.8871865E-1,6.5317163E3,8.231683E2,1.1686677E8,3.5175372E5,1.6235925E-2,-8.894108E-4,1.12425E-2,1.93E2,-2.28896E-2,9.985029E-3,5.517496E-1,-1.4884285E-2,-4.1465354E-3,3.327103E0,6.7584877E8,1.1812E4,5.214476E-3,7.1523585E-3,2.1171435E-3,-3.2622847E-3,3.881595E-3],"split_indices":[37,26,4,51,44,31,0,0,0,10,0,0,40,0,0,52,7,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.4E1,5E1,9E0,1.5E1,4.5E1,5E0,6E0,3E0,9E0,6E0,5E0,4E1,4E0,5E0,2.1E1,1.9E1,1.6E1,5E0,1.2E1,7E0,1.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-3.6807183E-2,-2.6082924E-1,9.745226E-2,-3.38609E-2,-4.4575256E-1,5.3367496E-2,2.569166E-1,3.8533923E-3,-1.0485729E-1,-2.6165378E-1,-2.6519928E-2,-7.0941593E-3,1.42699E-1,1.3604495E-2,5.124206E-3,-2.7395165E-4,-6.2492257E-3,-2.6505715E-3,-1.4469911E-2,5.6039942E-3,-3.8167406E-2,7.6184836E-3,2.8479039E-3,-6.874908E-2,2.3452176E-3,3.404119E-4,-1.1131427E-1,-8.842438E-4,-6.103098E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,-1,-1,25,-1,-1,27,-1,-1],"loss_changes":[2.4917388E0,1.2860303E0,3.518464E-1,1.4147085E-1,3.8356328E-1,2.2785982E-1,5.4195166E-2,0E0,3.9702587E-2,1.3221234E-1,0E0,1.1660655E-1,3.400594E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6346675E-2,0E0,0E0,5.6393713E-2,0E0,0E0,2.8358817E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,20,20,23,23,26,26],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,-1,-1,26,-1,-1,28,-1,-1],"split_conditions":[1.7489342E-2,5.0415697E5,4.108022E1,1.8144448E-4,2.5036643E8,6.3104886E-1,4.668526E6,3.8533923E-3,4.4092423E-1,5.86E2,-2.6519928E-2,2.7252597E-1,2.6529046E7,1.3604495E-2,5.124206E-3,-2.7395165E-4,-6.2492257E-3,-2.6505715E-3,-1.4469911E-2,5.6039942E-3,2.6597537E3,7.6184836E-3,2.8479039E-3,3.9785632E8,2.3452176E-3,3.404119E-4,1.0192E4,-8.842438E-4,-6.103098E-3],"split_indices":[37,27,56,37,7,40,28,0,26,0,0,26,44,0,0,0,0,0,0,0,4,0,0,7,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3E1,5.1E1,1.4E1,1.6E1,4.1E1,1E1,5E0,9E0,9E0,7E0,2.5E1,1.6E1,6E0,4E0,3E0,6E0,3E0,6E0,4E0,2.1E1,1E1,6E0,1.6E1,5E0,6E0,1E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.750396E-3,-2.637187E-1,1.3615979E-1,-1.4564972E-1,-5.4651755E-1,5.4761996E-3,2.2307935E-1,-1.2620885E-2,-9.917557E-2,-3.0120319E-2,-1.07787E-2,9.194145E-2,-6.576594E-2,3.183191E-1,1.2566628E-1,7.8938337E-4,-1.3992551E-1,9.1104873E-4,6.1660395E-3,-4.332866E-4,-6.867432E-3,6.8999347E-3,1.6945135E-2,1.6493298E-3,7.0712506E-3,-4.079417E-4,-8.720541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5698216E0,7.5514436E-1,5.099317E-1,1.02725804E-1,1.4779592E-1,1.2308604E-1,2.1049309E-1,0E0,7.928079E-2,0E0,0E0,3.0144311E-2,5.6403577E-2,9.509659E-2,4.5478165E-2,0E0,9.819192E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,13,13,14,14,16,16],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,1.5135763E8,2.5222173E-1,5.4508613E3,9.7646296E7,4.4605106E5,1.5312917E5,-1.2620885E-2,5.7777777E-2,-3.0120319E-2,-1.07787E-2,2.9795604E9,2.1446E4,3.9976162E-1,1.7325155E5,7.8938337E-4,3.027892E5,9.1104873E-4,6.1660395E-3,-4.332866E-4,-6.867432E-3,6.8999347E-3,1.6945135E-2,1.6493298E-3,7.0712506E-3,-4.079417E-4,-8.720541E-3],"split_indices":[37,44,38,32,12,27,32,0,56,0,0,5,9,26,32,0,27,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,2.4E1,4.4E1,1.8E1,6E0,1.8E1,2.6E1,3E0,1.5E1,3E0,3E0,8E0,1E1,1.2E1,1.4E1,4E0,1.1E1,4E0,4E0,7E0,3E0,5E0,7E0,5E0,9E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.408408E-3,-2.5382924E-1,1.1687296E-1,-1.6837038E-1,-2.9154288E-2,5.9414465E-2,2.300275E-2,-2.20062E-1,-1.8780673E-2,2.3231715E-1,4.9326853E-3,-1.4188947E-1,-1.6220024E-2,2.9747307E-3,-4.3801954E-3,3.8770789E-3,1.28422305E-2,-1.8587545E-1,5.7399318E-2,-8.522451E-3,-2.7460514E-3,-2.3119294E-3,-1.0868278E-2,6.147688E-3,4.379439E-3,-5.93595E-2,8.179598E-2,1.3064143E-3,-4.1435147E-3,6.122379E-5,6.2349276E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1,-1,-1,25,27,29,-1,-1,-1,-1],"loss_changes":[2.3664117E0,8.3604467E-1,1.2018381E0,1.7371541E-1,0E0,4.3217617E-1,0E0,1.5779579E-1,5.8853693E-2,7.8718185E-2,3.710673E-1,4.5167238E-2,0E0,0E0,0E0,0E0,0E0,5.8254957E-2,1.3215809E-1,0E0,0E0,0E0,0E0,0E0,9.860988E-2,4.0082153E-2,4.6084777E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,17,17,18,18,24,24,25,25,26,26],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1,-1,-1,26,28,30,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.5135763E8,3.4409692E3,5.36E3,-2.9154288E-2,1.7329262E7,2.300275E-2,1.02E3,2.2858976E8,6.498673E2,4.492E3,3E0,-1.6220024E-2,2.9747307E-3,-4.3801954E-3,3.8770789E-3,1.28422305E-2,4.841492E5,5.090285E7,-8.522451E-3,-2.7460514E-3,-2.3119294E-3,-1.0868278E-2,6.147688E-3,1.204781E3,4.1485094E-2,1.7495675E5,1.3064143E-3,-4.1435147E-3,6.122379E-5,6.2349276E-3],"split_indices":[57,44,51,2,0,44,0,0,7,4,2,8,0,0,0,0,0,27,44,0,0,0,0,0,51,38,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.5E1,5E1,2.2E1,3E0,4.5E1,5E0,1.6E1,6E0,1E1,3.5E1,1.2E1,4E0,3E0,3E0,4E0,6E0,7E0,2.8E1,6E0,6E0,3E0,4E0,1E1,1.8E1,1E1,8E0,3E0,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.0632061E-2,-2.4092656E-1,9.551605E-2,-4.0311107E-1,-8.516071E-2,1.254183E-1,-1.1961503E-2,-7.877372E-3,-2.3726251E-2,-1.6082093E-1,1.2791782E-3,4.8489507E-2,1.8568814E-1,-1.0854732E-2,-2.319776E-3,1.9796067E-1,-1.993472E-2,-2.5485766E-3,2.2027308E-1,3.1508906E-3,1.1663632E-2,-2.6950627E-3,3.7402357E-3,2.500118E-1,5.6986962E-3,5.1142857E-3,2.8351766E-1,1.466244E-2,7.1420944E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,17,-1,-1,19,21,-1,23,-1,-1,-1,-1,25,-1,-1,27,-1,-1],"loss_changes":[1.7830759E0,5.4671514E-1,5.960934E-1,2.6311874E-1,1.1799519E-1,2.0891088E-1,0E0,0E0,0E0,6.5531775E-2,0E0,2.2895017E-1,2.3369044E-1,0E0,0E0,4.4965267E-2,7.9553865E-2,0E0,2.6198149E-2,0E0,0E0,0E0,0E0,3.600812E-2,0E0,0E0,2.5718153E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,12,12,15,15,16,16,18,18,23,23,26,26],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,18,-1,-1,20,22,-1,24,-1,-1,-1,-1,26,-1,-1,28,-1,-1],"split_conditions":[2.0113895E-2,3.554517E0,6.2012E4,1.0370839E6,1.7493458E3,3.3597556E-1,-1.1961503E-2,-7.877372E-3,-2.3726251E-2,9.4596675E-5,1.2791782E-3,2.5143658E6,1.4212261E0,-1.0854732E-2,-2.319776E-3,3.7700243E-2,3.6616542E0,-2.5485766E-3,3.2979845E6,3.1508906E-3,1.1663632E-2,-2.6950627E-3,3.7402357E-3,3.1823853E-1,5.6986962E-3,5.1142857E-3,6.585265E-1,1.466244E-2,7.1420944E-3],"split_indices":[40,53,10,27,4,38,0,0,0,37,0,50,55,0,0,26,52,0,31,0,0,0,0,26,0,0,26,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.2E1,4.9E1,1E1,1.2E1,4.6E1,3E0,5E0,5E0,7E0,5E0,2.1E1,2.5E1,3E0,4E0,6E0,1.5E1,3E0,2.2E1,3E0,3E0,1.1E1,4E0,1.5E1,7E0,4E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.2602936E-2,-1.7656966E-1,6.665869E-2,-2.5975782E-1,8.466752E-2,1.0225561E-1,-1.767866E-1,-1.4000464E-3,-2.921736E-1,8.9720427E-4,6.119811E-3,1.7116562E-2,1.6708885E-1,-1.2227611E-2,-1.0023373E-3,-3.4930915E-1,-4.303655E-3,-3.5181323E-3,6.2355846E-2,2.0918262E-1,4.6230074E-2,-1.6783658E-2,-5.6650047E-3,7.1607847E-3,-6.9373236E-3,9.782477E-3,3.3604016E-3,-1.9280773E-3,4.705579E-3,1.6120562E-3,-3.5627545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,-1,-1,21,-1,-1,23,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1249788E0,6.739482E-1,4.687044E-1,1.5969086E-1,2.6773125E-2,2.5105268E-1,1.1046371E-1,0E0,1.8371439E-1,0E0,0E0,9.954972E-2,1.2260193E-1,0E0,0E0,8.8416815E-2,0E0,0E0,1.1039816E-1,3.0600965E-2,5.1575392E-2,0E0,0E0,0E0,3.7230834E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,15,15,18,18,19,19,20,20,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,-1,-1,22,-1,-1,24,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.962943E3,1.4696082E12,2.5346E3,2.4399805E3,3.3597556E-1,3.05E3,-1.4000464E-3,2.2867646E0,8.9720427E-4,6.119811E-3,7.095772E2,5.7588155E6,-1.2227611E-2,-1.0023373E-3,1.650581E3,-4.303655E-3,-3.5181323E-3,4.8178736E7,1.6129E4,6.249143E-1,-1.6783658E-2,-5.6650047E-3,7.1607847E-3,7.844101E0,9.782477E-3,3.3604016E-3,-1.9280773E-3,4.705579E-3,1.6120562E-3,-3.5627545E-3],"split_indices":[41,4,30,31,4,38,0,0,56,0,0,4,31,0,0,4,0,0,44,9,41,0,0,0,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.9E1,5.1E1,2.2E1,7E0,4.5E1,6E0,3E0,1.9E1,4E0,3E0,2E1,2.5E1,3E0,3E0,1.4E1,5E0,6E0,1.4E1,1.8E1,7E0,1.1E1,3E0,5E0,9E0,1.5E1,3E0,3E0,4E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.6865763E-3,-1.8272516E-1,8.7893896E-2,-2.5947955E-1,3.8822438E-3,1.21314E-1,-1.1284622E-2,-1.7682245E-1,-2.2460883E-2,2.035009E-1,7.4624578E-3,-1.0745997E-2,-7.0943184E-2,2.0190727E-2,2.6090908E-1,-4.7269452E-2,3.9244615E-3,3.7158409E-3,-7.826691E-3,-1.5020958E-3,3.6044621E-3,1.7683446E-1,3.441152E-1,1.9708078E-3,-8.922047E-2,1.8667543E-3,9.08392E-3,1.899623E-2,8.695617E-3,-5.6091757E-3,-8.99704E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2168006E0,5.273904E-1,6.668652E-1,3.2449567E-1,0E0,4.6633655E-1,0E0,1.08326495E-1,0E0,2.9991376E-1,1.0585508E-1,0E0,1.5742677E-1,3.1326193E-2,9.305024E-2,5.894113E-2,0E0,0E0,0E0,0E0,0E0,5.2344352E-2,5.0588608E-2,0E0,2.584079E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,12,12,13,13,14,14,15,15,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,1.5414109E8,1.2427474E8,3.8822438E-3,3.3934937E0,-1.1284622E-2,5E0,-2.2460883E-2,2.0753424E0,5.2518907E0,-1.0745997E-2,5.0415697E5,1.6E1,2.9168832E0,7.843361E5,3.9244615E-3,3.7158409E-3,-7.826691E-3,-1.5020958E-3,3.6044621E-3,1.0737101E0,7.33E2,1.9708078E-3,9.21919E5,1.8667543E-3,9.08392E-3,1.899623E-2,8.695617E-3,-5.6091757E-3,-8.99704E-4],"split_indices":[57,4,44,44,0,53,0,8,0,52,53,0,27,3,53,31,0,0,0,0,0,56,0,0,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.3E1,5.3E1,1.8E1,5E0,4.9E1,4E0,1.5E1,3E0,2.8E1,2.1E1,8E0,7E0,7E0,2.1E1,1.3E1,8E0,3E0,4E0,4E0,3E0,1.2E1,9E0,4E0,9E0,3E0,9E0,4E0,5E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.621743E-2,3.558231E-2,-2.938916E-1,-1.8531272E-1,7.990702E-2,6.033404E-4,-3.8770264E-1,-9.306622E-3,-2.8338241E-3,3.3018474E-2,1.9380707E-1,-1.9141065E-2,-7.951701E-3,9.9072486E-2,-1.2703481E-1,2.4355294E-1,2.651927E-3,8.730972E-3,3.930368E-2,-7.339577E-3,-9.130585E-4,6.346931E-3,1.3425479E-2,-2.515763E-3,7.3647335E-2,4.856358E-3,1.2821703E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,-1,17,19,21,-1,-1,23,-1,-1,-1,-1,-1,25,-1,-1],"loss_changes":[1.1899004E0,7.001853E-1,3.8109004E-1,3.1746447E-2,3.0869663E-1,0E0,3.7626266E-2,0E0,0E0,4.6718863E-1,9.635085E-2,0E0,0E0,1.8670347E-1,5.760783E-2,3.3767164E-2,0E0,0E0,7.510835E-2,0E0,0E0,0E0,0E0,0E0,2.448184E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,13,13,14,14,15,15,18,18,24,24],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,-1,18,20,22,-1,-1,24,-1,-1,-1,-1,-1,26,-1,-1],"split_conditions":[1.0232127E12,1.885E3,2.7307262E5,1.4324325E1,3.029697E3,6.033404E-4,2.8312179E12,-9.306622E-3,-2.8338241E-3,6.98494E7,5.405855E6,-1.9141065E-2,-7.951701E-3,1.7329262E7,1.962943E3,3.3072104E3,2.651927E-3,8.730972E-3,4.57563E5,-7.339577E-3,-9.130585E-4,6.346931E-3,1.3425479E-2,-2.515763E-3,1.7231327E3,4.856358E-3,1.2821703E-3],"split_indices":[30,2,27,55,4,0,30,0,0,44,31,0,0,44,4,51,0,0,1,0,0,0,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,6.9E1,1.2E1,1.1E1,5.8E1,3E0,9E0,8E0,3E0,4.2E1,1.6E1,6E0,3E0,3E1,1.2E1,1.1E1,5E0,1E1,2E1,8E0,4E0,6E0,5E0,5E0,1.5E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.759476E-2,2.2812529E-2,-5.061224E-1,-1.1263222E-1,1.0611612E-1,-9.673372E-3,-2.8202647E-2,-1.7281608E-1,9.435549E-4,2.3669061E-1,5.7915013E-2,-2.286358E-1,-6.0411368E-2,-2.3361205E-3,2.8840434E-3,3.760488E-3,1.2037039E-2,8.164562E-3,1.3035301E-2,-3.5303698E-3,-2.7309614E-1,3.2271762E-4,-4.843644E-3,4.0815067E-3,-1.801196E-2,-5.5391397E-3,-1.4254254E-2,3.493249E-2,-2.957302E-3,-2.0820445E-3,3.3237783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,-1,-1,-1,-1,-1,23,-1,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[1.7723994E0,7.6791537E-1,1.513561E-1,1.7788422E-1,2.5063318E-1,0E0,0E0,9.285253E-2,4.063935E-2,4.8537076E-2,1.8853325E-1,4.889655E-2,2.5998276E-2,0E0,0E0,0E0,0E0,0E0,6.568345E-2,0E0,2.6003301E-2,0E0,0E0,0E0,5.3863045E-2,0E0,0E0,3.9567515E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,18,18,20,20,24,24,27,27],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,-1,-1,-1,-1,-1,24,-1,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.5414109E8,2.1647314E8,1.2096E4,3.5671377E0,2.325535E7,-9.673372E-3,-2.8202647E-2,3.6512393E2,9.18853E0,3.8198416E8,3.5010372E5,2.68E2,1.401E4,-2.3361205E-3,2.8840434E-3,3.760488E-3,1.2037039E-2,8.164562E-3,1.3E1,-3.5303698E-3,4.5859104E-1,3.2271762E-4,-4.843644E-3,4.0815067E-3,5.327983E8,-5.5391397E-3,-1.4254254E-2,7.84E3,-2.957302E-3,-2.0820445E-3,3.3237783E-3],"split_indices":[44,12,9,52,44,0,0,51,53,7,31,11,9,0,0,0,0,0,8,0,26,0,0,0,7,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.6E1,6E0,2.5E1,4.1E1,3E0,3E0,1.6E1,9E0,1E1,3.1E1,1E1,6E0,5E0,4E0,3E0,7E0,7E0,2.4E1,3E0,7E0,3E0,3E0,6E0,1.8E1,3E0,4E0,9E0,9E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.2696033E-2,-2.0930885E-1,1.13604575E-1,-2.9838702E-1,-2.7691113E-2,6.8088025E-2,3.6902377E-1,-3.4380555E-1,-3.2577056E-3,1.9661505E-3,-4.097642E-3,-5.626787E-3,8.707275E-2,2.508905E-2,6.9449507E-3,-8.780094E-3,-2.014807E-2,1.8381791E-1,5.423027E-2,3.3000312E-3,9.728457E-3,9.0375565E-2,-4.2705968E-2,-1.5911086E-4,1.0181595E-1,-4.5049815E-3,-2.5859752E-4,5.1225866E-3,1.6457686E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,27,-1,-1,-1,-1],"loss_changes":[1.7954413E0,3.8001347E-1,6.8404776E-1,1.352861E-1,4.9577374E-2,2.1236113E-1,3.0949605E-1,1.3253212E-1,0E0,0E0,0E0,0E0,1.4572173E-1,0E0,0E0,0E0,0E0,3.834212E-2,1.3700354E-1,0E0,0E0,3.0520767E-2,2.4622736E-2,0E0,2.3846567E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,17,17,18,18,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,28,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,4.0614333E0,3.9509753E3,1E0,9.256755E7,2.3688402E0,5.99999E9,8.2220456E5,-3.2577056E-3,1.9661505E-3,-4.097642E-3,-5.626787E-3,1.7329262E7,2.508905E-2,6.9449507E-3,-8.780094E-3,-2.014807E-2,6.498673E2,5.3920375E5,3.3000312E-3,9.728457E-3,3.786E3,1.0403953E3,-1.5911086E-4,7.53991E8,-4.5049815E-3,-2.5859752E-4,5.1225866E-3,1.6457686E-3],"split_indices":[40,53,51,15,44,53,5,27,0,0,0,0,44,0,0,0,0,4,27,0,0,2,51,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,2.3E1,6E1,1.5E1,8E0,5.2E1,8E0,1.2E1,3E0,4E0,4E0,4E0,4.8E1,3E0,5E0,7E0,5E0,1.1E1,3.7E1,4E0,7E0,2.7E1,1E1,3E0,2.4E1,3E0,7E0,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.4678428E-3,-2.3422952E-1,4.73607E-2,-4.2342767E-3,-1.3100968E-2,9.732043E-2,-1.4843525E-1,-3.4495117E-3,1.2342452E-1,-8.683877E-3,-1.9701239E-2,1.055637E-2,8.4631935E-2,-3.048812E-3,4.407044E-3,4.3589123E-2,1.1791568E-2,1.4780079E-2,4.264745E-3,-9.921147E-4,4.0220926E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,-1,7,9,-1,11,13,-1,-1,15,-1,-1,17,-1,19,-1,-1,-1],"loss_changes":[8.521338E-1,9.645444E-2,6.2092435E-1,0E0,0E0,2.4226391E-1,5.633207E-1,0E0,1.8979019E-1,7.7309705E-2,0E0,0E0,2.6250815E-1,0E0,0E0,4.704625E-2,0E0,6.5410495E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,9,9,12,12,15,15,17,17],"right_children":[2,4,6,-1,-1,8,10,-1,12,14,-1,-1,16,-1,-1,18,-1,20,-1,-1,-1],"split_conditions":[1.528E3,5.295928E-1,1.0232127E12,-4.2342767E-3,-1.3100968E-2,1.885E3,1.0202749E8,-3.4495117E-3,1.6256282E7,1.5E1,-1.9701239E-2,1.055637E-2,1.0011831E8,-3.048812E-3,4.407044E-3,1.2772205E3,1.1791568E-2,5.2518907E0,4.264745E-3,-9.921147E-4,4.0220926E-3],"split_indices":[11,26,30,0,0,2,44,0,44,3,0,0,44,0,0,51,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,1.2E1,6.1E1,5E0,7E0,4.9E1,1.2E1,6E0,4.3E1,9E0,3E0,9E0,3.4E1,6E0,3E0,2.9E1,5E0,2E1,9E0,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-6.2962943E-3,3.5496358E-2,-3.399379E-1,-1.1311451E-2,2.488019E-1,-2.0067945E-2,-2.6059947E-3,-6.1576482E-2,1.3076591E-1,1.859947E-2,6.391664E-3,-6.850429E-3,-1.2964153E-1,9.274452E-3,3.074991E-3,1.8079594E-2,-4.925148E-3,-7.928694E-3,-9.756059E-4,-3.9710565E-3,5.617174E-2,6.8172435E-3,5.1917305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.1275679E0,7.216886E-1,2.961781E-1,4.3654352E-1,1.7536354E-1,0E0,0E0,1.6673383E-1,6.5500766E-2,0E0,0E0,7.22945E-2,1.1492002E-1,0E0,0E0,9.818778E-2,0E0,0E0,0E0,0E0,7.534757E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,15,15,20,20],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.2159766E6,3.3795098E3,2.277961E6,5.517496E-1,7.33E2,-2.0067945E-2,-2.6059947E-3,5.6660336E-1,7.843361E5,1.859947E-2,6.391664E-3,1.6813238E-1,9.014711E-2,9.274452E-3,3.074991E-3,2.23E2,-4.925148E-3,-7.928694E-3,-9.756059E-4,-3.9710565E-3,2.325535E7,6.8172435E-3,5.1917305E-4],"split_indices":[27,4,1,40,0,0,0,26,31,0,0,41,38,0,0,10,0,0,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,5.9E1,1.2E1,5E0,3E0,4.4E1,1.5E1,3E0,9E0,2.5E1,1.9E1,5E0,1E1,2.1E1,4E0,1.2E1,7E0,5E0,1.6E1,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.8961756E-2,-1.8900508E-1,5.249203E-2,-2.7111888E-1,3.7989654E-3,1.823027E-2,1.40801435E-2,-1.6137661E-1,-1.659747E-2,1.5119879E-1,-4.879017E-2,-1.6829027E-3,-9.642337E-3,7.2035886E-4,2.0577563E-1,-1.814221E-2,-1.1722019E-2,4.2255507E-3,1.0558483E-2,-7.085719E-2,5.093006E-2,-1.0153317E-3,-4.9320213E-3,3.5038202E-3,-2.0127334E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,-1,-1,17,19,-1,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[9.4548965E-1,5.48823E-1,5.1553446E-1,1.6849434E-1,0E0,4.5295742E-1,0E0,7.717165E-2,0E0,1.2118772E-1,2.2982162E-1,0E0,0E0,0E0,2.8395116E-2,1.16484195E-1,0E0,0E0,0E0,3.4979656E-2,4.6394322E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,14,14,15,15,19,19,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,-1,-1,18,20,-1,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[4.279412E0,1.962943E3,4.0230347E3,4.1E1,3.7989654E-3,2.325535E7,1.40801435E-2,8.918406E-2,-1.659747E-2,4.4939005E2,4.927835E0,-1.6829027E-3,-9.642337E-3,7.2035886E-4,1.7752522E-1,1.204781E3,-1.1722019E-2,4.2255507E-3,1.0558483E-2,5.2127117E-1,2.969772E0,-1.0153317E-3,-4.9320213E-3,3.5038202E-3,-2.0127334E-3],"split_indices":[57,4,51,8,0,44,0,56,0,4,53,0,0,0,26,51,0,0,0,26,52,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.2E1,5.4E1,1.7E1,5E0,4.9E1,5E0,1E1,7E0,1.6E1,3.3E1,4E0,6E0,5E0,1.1E1,3E1,3E0,4E0,7E0,1.7E1,1.3E1,9E0,8E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.6817848E-3,-2.2616802E-1,9.206123E-2,-1.0064887E-1,-4.0438184E-1,5.8719475E-2,2.2913849E-2,-7.919475E-3,-3.140276E-2,-2.117215E-2,-6.1804466E-3,-6.9123055E-3,7.877879E-2,1.6335608E-3,-4.646041E-3,1.4994971E-1,4.8611876E-2,5.5734406E-4,7.82964E-3,1.5576696E-2,1.7054671E-1,-1.7116611E-3,4.5251427E-3,1.0051072E-2,3.6281538E-3,-1.3453253E-3,1.6290755E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,-1,-1,15,-1,-1,17,19,-1,-1,21,23,25,-1,-1,-1,-1,-1],"loss_changes":[1.7782433E0,5.185307E-1,8.415028E-1,8.7854594E-2,1.6418362E-1,2.6026985E-1,0E0,0E0,5.791353E-2,0E0,0E0,0E0,1.0435501E-1,0E0,0E0,6.568107E-2,1.5041274E-1,0E0,0E0,4.8545223E-2,2.4277136E-2,3.31714E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,12,12,15,15,16,16,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,-1,-1,16,-1,-1,18,20,-1,-1,22,24,26,-1,-1,-1,-1,-1],"split_conditions":[4.856813E0,1.0789844E6,7.6312915E3,5E0,1.4235585E3,1.9956966E-1,2.2913849E-2,-7.919475E-3,9.502012E2,-2.117215E-2,-6.1804466E-3,-6.9123055E-3,2.6529046E7,1.6335608E-3,-4.646041E-3,1.0843103E5,9.599149E7,5.5734406E-4,7.82964E-3,5.2518907E0,3.882652E6,4.531609E6,4.5251427E-3,1.0051072E-2,3.6281538E-3,-1.3453253E-3,1.6290755E-3],"split_indices":[57,27,51,8,4,56,0,0,4,0,0,0,44,0,0,32,44,0,0,53,28,31,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.4E1,5.8E1,1.5E1,9E0,5.5E1,3E0,6E0,9E0,6E0,3E0,4E0,5.1E1,5E0,4E0,1.4E1,3.7E1,3E0,1.1E1,3E1,7E0,2.6E1,4E0,3E0,4E0,1.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.3843824E-2,-2.2078529E-1,8.560404E-2,-7.2355405E-4,-3.3810332E-1,4.831773E-2,2.659959E-1,-3.418276E-3,3.3625811E-3,-9.524407E-3,-2.0271981E-2,-2.8013967E-2,7.723982E-2,1.5543894E-2,5.7646437E-3,-2.8248949E-3,2.762108E-3,7.633473E-3,3.8658738E-2,1.4537007E-2,4.472216E-3,3.2292134E-3,-6.296606E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1,-1,19,21,-1,-1,-1],"loss_changes":[1.4594924E0,5.9355116E-1,3.1000277E-1,6.285858E-2,1.2970114E-1,9.267834E-2,5.332136E-2,0E0,0E0,0E0,0E0,4.654448E-2,1.1030732E-1,0E0,0E0,0E0,0E0,0E0,3.5053264E-2,3.32286E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,12,12,18,18,19,19],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1,-1,20,22,-1,-1,-1],"split_conditions":[1.0420895E-2,3.6588228E5,1.1997641E0,1.2E1,3.1457312E8,2.6277744E8,9.07E2,-3.418276E-3,3.3625811E-3,-9.524407E-3,-2.0271981E-2,4.57563E5,2.325535E7,1.5543894E-2,5.7646437E-3,-2.8248949E-3,2.762108E-3,7.633473E-3,3.3017554E2,4.4200982E8,4.472216E-3,3.2292134E-3,-6.296606E-4],"split_indices":[37,27,38,3,7,12,0,0,0,0,0,1,44,0,0,0,0,0,56,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.9E1,2.2E1,4.7E1,8E0,1.4E1,4E1,7E0,4E0,4E0,9E0,5E0,1.1E1,2.9E1,3E0,4E0,8E0,3E0,7E0,2.2E1,1.7E1,5E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-7.5184467E-4,-1.3201915E-1,1.3373545E-1,8.0386795E-2,-2.009081E-1,8.526345E-2,2.3981012E-2,-1.3384914E-3,1.4867929E-1,-2.0083936E-2,-1.5691182E-1,1.7738946E-1,4.602597E-2,8.84728E-3,2.2792364E-3,-1.8371418E-1,-1.0070925E-3,3.6515344E-3,1.0255674E-2,3.3308978E-3,5.2403803E-3,-1.181424E-2,-1.4481658E-1,1.7038058E-3,-2.2174134E-3,-2.304634E-3,-8.811093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,15,17,19,-1,-1,21,-1,-1,-1,23,-1,-1,25,-1,-1,-1,-1],"loss_changes":[1.4652659E0,6.3071203E-1,8.008842E-1,8.855475E-2,3.046682E-1,1.2919757E-1,0E0,0E0,2.7837962E-2,0E0,9.933102E-2,4.2705685E-2,9.1066234E-2,0E0,0E0,4.639697E-2,0E0,0E0,0E0,4.0348608E-2,0E0,0E0,9.466842E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,10,10,11,11,12,12,15,15,19,19,22,22],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,16,18,20,-1,-1,22,-1,-1,-1,24,-1,-1,26,-1,-1,-1,-1],"split_conditions":[4.283039E1,3.459181E-1,7.6312915E3,2.84E2,7E0,2.156825E7,2.3981012E-2,-1.3384914E-3,4.31E2,-2.0083936E-2,1.517E3,7.285458E-1,3.981664E7,8.84728E-3,2.2792364E-3,7.446808E-2,-1.0070925E-3,3.6515344E-3,1.0255674E-2,7.640048E7,5.2403803E-3,-1.181424E-2,1.5368E4,1.7038058E-3,-2.2174134E-3,-2.304634E-3,-8.811093E-3],"split_indices":[57,26,51,0,3,44,0,0,10,0,0,38,47,0,0,57,0,0,0,44,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,4.1E1,4E1,1E1,3.1E1,3.7E1,3E0,4E0,6E0,3E0,2.8E1,1E1,2.7E1,3E0,3E0,2.3E1,5E0,5E0,5E0,1.8E1,9E0,5E0,1.8E1,1.1E1,7E0,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-5.1782955E-3,-3.2325856E-2,1.3359509E-2,5.334816E-2,-1.058763E-1,-2.037616E-2,1.0874377E-1,-1.9263065E-1,-1.8165844E-2,-7.343333E-2,3.0553695E-3,9.943217E-3,7.100378E-2,-1.1755063E-2,-4.6106526E-3,3.9332747E-2,-1.1032636E-2,-5.766646E-3,-1.1908424E-3,5.5161244E-3,4.94738E-4,4.2416505E-3,-5.339266E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,-1,5,7,9,11,13,15,17,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5672225E-1,4.4674364E-1,0E0,1.369879E-1,2.856527E-1,7.832603E-2,7.2243735E-2,1.0291326E-1,2.8594252E-1,2.4242405E-2,0E0,0E0,5.3651884E-2,0E0,0E0,5.3966694E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,18,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,5.460315E9,1.3359509E-2,7.767353E2,2.667437E6,3.109375E0,3.1841638E5,1.2439503E3,9.71518E5,1.5039519E5,3.0553695E-3,9.943217E-3,1.0191781E1,-1.1755063E-2,-4.6106526E-3,3.309969E0,-1.1032636E-2,-5.766646E-3,-1.1908424E-3,5.5161244E-3,4.94738E-4,4.2416505E-3,-5.339266E-4],"split_indices":[51,5,0,4,1,52,31,4,9,32,0,0,55,0,0,53,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,3.2E1,3.7E1,1.4E1,1.8E1,1.8E1,1.9E1,9E0,5E0,3E0,1.5E1,8E0,1E1,1.6E1,3E0,3E0,6E0,7E0,8E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.7208477E-2,-3.595266E-2,3.2588106E-1,-1.5382302E-1,4.6290033E-2,2.2359533E-2,4.7245156E-3,-1.9070506E-2,-1.029633E-1,-5.3705927E-3,6.273173E-2,3.1597181E-3,-1.292435E-1,1.0151825E-1,1.1840872E-2,-1.8279533E-1,-2.5630025E-2,9.364669E-6,1.3129927E-1,-2.2221901E-3,2.9114087E-3,-1.1613733E-2,-4.7571445E-3,1.0529151E-3,-4.045425E-3,8.268006E-3,8.933545E-2,3.8189502E-4,4.9239676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,13,-1,15,17,19,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.3408766E0,6.8569875E-1,4.5140564E-1,3.8495845E-1,1.2656963E-1,0E0,0E0,0E0,1.303899E-1,0E0,7.5779244E-2,0E0,1.2293485E-1,6.634076E-2,6.813111E-2,6.50807E-2,3.352238E-2,0E0,2.6989102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3874164E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,10,10,12,12,13,13,14,14,15,15,16,16,18,18,26,26],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,14,-1,16,18,20,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[4.9891987E3,1.1576994E-2,8.982776E9,7E0,7.97E2,2.2359533E-2,4.7245156E-3,-1.9070506E-2,2.5346E3,-5.3705927E-3,5.090285E7,3.1597181E-3,1.7742582E3,2.727E3,2.6597537E3,5.9072212E-5,9.635135E7,9.364669E-6,1.6256282E7,-2.2221901E-3,2.9114087E-3,-1.1613733E-2,-4.7571445E-3,1.0529151E-3,-4.045425E-3,8.268006E-3,1.8E9,3.8189502E-4,4.9239676E-3],"split_indices":[4,37,5,3,10,0,0,0,31,0,44,0,4,2,4,37,44,0,44,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.9E1,1.1E1,2.8E1,4.1E1,5E0,6E0,3E0,2.5E1,3E0,3.8E1,3E0,2.2E1,2.1E1,1.7E1,1.4E1,8E0,5E0,1.6E1,8E0,9E0,5E0,9E0,5E0,3E0,5E0,1.1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.5003342E-2,-1.2385285E-1,3.199236E-2,-1.8356408E-1,3.4853884E-3,1.691761E-1,-7.5208065E-3,-9.232601E-2,-1.5646424E-2,9.289425E-3,1.3160805E-3,1.6775912E-2,-9.5922705E-3,-6.052781E-3,-3.775368E-4,-5.8194143E-3,3.479608E-2,9.331157E-2,1.233866E-2,4.0741113E-5,5.9582987E-3,4.076601E-2,-2.2735174E-3,-1.9654573E-3,2.943318E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,15,-1,-1,-1,-1,17,19,21,-1,-1,23,-1,-1,-1],"loss_changes":[4.3214613E-1,3.5830086E-1,2.6688004E-1,3.3218026E-1,0E0,6.647059E-2,2.0918739E-1,6.4500496E-2,0E0,0E0,0E0,1.0389972E-1,0E0,0E0,0E0,0E0,4.2214192E-2,3.8120978E-2,4.871987E-2,0E0,0E0,4.6971135E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,11,11,16,16,17,17,18,18,21,21],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,16,-1,-1,-1,-1,18,20,22,-1,-1,24,-1,-1,-1],"split_conditions":[7.2921924E-2,1.0735684E3,3.5175372E5,9.29295E5,3.4853884E-3,2.2980049E10,2.292109E7,1.348668E6,-1.5646424E-2,9.289425E-3,1.3160805E-3,2.074E3,-9.5922705E-3,-6.052781E-3,-3.775368E-4,-5.8194143E-3,1.7329262E7,4.5243898E-1,5.451104E5,4.0741113E-5,5.9582987E-3,2.721693E0,-2.2735174E-3,-1.9654573E-3,2.943318E-3],"split_indices":[41,51,31,27,0,5,46,1,0,0,0,2,0,0,0,0,44,26,27,0,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.7E1,4.8E1,2.1E1,6E0,1E1,3.8E1,1.5E1,6E0,7E0,3E0,3.5E1,3E0,9E0,6E0,3E0,3.2E1,8E0,2.4E1,3E0,5E0,1.7E1,7E0,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.6820382E-2,-7.763189E-3,-1.58089E-2,-1.0773235E-1,4.673664E-2,-1.5367822E-1,-8.975431E-3,1.9179458E-2,9.261775E-2,-2.6514006E-3,-1.9804698E-1,3.894548E-3,-3.1720933E-3,3.398798E-2,-2.6368145E-3,5.70798E-3,1.2813766E-3,-1.15429815E-2,-5.0636507E-3,2.8144785E-3,-2.9021094E-3,-2.6330808E-3,2.4026027E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,-1,5,7,9,11,13,15,-1,17,-1,-1,19,-1,-1,-1,-1,-1,-1,21,-1,-1],"loss_changes":[7.2253686E-1,3.8641617E-1,0E0,1.1205751E-1,5.638393E-2,5.6869358E-2,6.5465905E-2,3.800945E-2,3.9521575E-2,0E0,3.1004459E-2,0E0,0E0,3.0717986E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.852589E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,20,20],"right_children":[2,4,-1,6,8,10,12,14,16,-1,18,-1,-1,20,-1,-1,-1,-1,-1,-1,22,-1,-1],"split_conditions":[1.5414109E8,1.4007953E1,-1.58089E-2,4.593E3,1.0379711E0,1.762456E3,2.2858976E8,2.5922572E7,7.7591565E8,-2.6514006E-3,1.6633E4,3.894548E-3,-3.1720933E-3,9.343137E0,-2.6368145E-3,5.70798E-3,1.2813766E-3,-1.15429815E-2,-5.0636507E-3,2.8144785E-3,2.916545E3,-2.6330808E-3,2.4026027E-3],"split_indices":[44,57,0,2,41,46,7,31,7,0,9,0,0,56,0,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,2.4E1,4.5E1,1.6E1,8E0,2.9E1,1.6E1,6E0,1E1,3E0,5E0,2.5E1,4E0,9E0,7E0,4E0,6E0,1.3E1,1.2E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[4.8719868E-2,5.610019E-3,3.0250698E-1,-8.749326E-2,5.7985973E-2,2.1753406E-2,3.4324657E-3,-5.387793E-3,-2.0549142E-1,1.3562806E-1,2.2639867E-2,-5.5633806E-2,4.1130465E-3,-1.1024251E-2,-3.2354558E-3,9.438401E-4,9.133674E-3,-6.299642E-3,3.610687E-3,1.8283814E-4,-4.117503E-3,3.986649E-2,-5.6452423E-2,3.4658436E-3,-1.1015756E-3,-5.1110643E-3,-5.280906E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[7.5321287E-1,2.9765582E-1,4.1739923E-1,2.1203533E-1,1.03958875E-1,0E0,0E0,7.67995E-2,4.1613758E-2,1.0151906E-1,5.0604362E-2,2.4766162E-2,0E0,0E0,0E0,0E0,0E0,4.855776E-2,0E0,0E0,0E0,3.1815737E-2,2.633889E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,17,17,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[6.5317163E3,1.7489342E-2,1.3946067E7,9.635135E7,1.7329262E7,2.1753406E-2,3.4324657E-3,5.7630615E2,9.29342E5,4.4690585E2,5.0175633E0,5.7269626E0,4.1130465E-3,-1.1024251E-2,-3.2354558E-3,9.438401E-4,9.133674E-3,6.650418E-2,3.610687E-3,1.8283814E-4,-4.117503E-3,3.309969E0,4.828561E-1,3.4658436E-3,-1.1015756E-3,-5.1110643E-3,-5.280906E-4],"split_indices":[4,37,46,44,44,0,0,51,1,51,53,55,0,0,0,0,0,37,0,0,0,53,38,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,5.9E1,9E0,2.1E1,3.8E1,4E0,5E0,1.3E1,8E0,1.1E1,2.7E1,9E0,4E0,5E0,3E0,5E0,6E0,1.9E1,8E0,4E0,5E0,1E1,9E0,6E0,4E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[5.1178336E-3,-3.5064798E-2,3.4833393E-1,7.9573144E-4,-2.754099E-1,2.4909087E-2,5.7377797E-3,-9.032387E-2,5.5872798E-2,4.037428E-3,-1.9131992E-2,-1.1880068E-1,2.2681134E-3,1.655477E-1,1.7621385E-2,-7.609299E-2,-1.0638993E-2,2.5833421E-3,9.279461E-3,-3.336372E-3,3.7486546E-2,7.7561266E-3,-1.1866389E-1,3.4736774E-3,4.4034184E-3,3.5901912E-3,-3.009756E-3,-8.278113E-3,-3.0867772E-3,-2.1095695E-3,2.4652558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.184477E0,6.638746E-1,3.7315917E-1,3.4631148E-1,6.7827475E-1,0E0,0E0,1.1244637E-1,1.7747727E-1,0E0,0E0,1.0320121E-1,0E0,4.6391904E-2,6.562217E-2,6.5171964E-2,0E0,0E0,0E0,0E0,4.0163472E-2,4.7587115E-2,2.7812913E-2,0E0,5.1483214E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,14,14,15,15,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.4899646E8,1.523967E6,5.666351E7,2.7307262E5,2.4909087E-2,5.7377797E-3,1.7493458E3,1.7329262E7,4.037428E-3,-1.9131992E-2,1.0789844E6,2.2681134E-3,4.189857E8,4.57563E5,2.5722395E3,-1.0638993E-2,2.5833421E-3,9.279461E-3,-3.336372E-3,5.090285E7,5.0415697E5,2.6812036E2,3.4736774E-3,2.916545E3,3.5901912E-3,-3.009756E-3,-8.278113E-3,-3.0867772E-3,-2.1095695E-3,2.4652558E-3],"split_indices":[51,44,28,12,27,0,0,4,44,0,0,27,0,7,1,46,0,0,0,0,44,27,4,0,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,7.6E1,8E0,6.7E1,9E0,3E0,5E0,2.5E1,4.2E1,3E0,6E0,2.1E1,4E0,1E1,3.2E1,1.7E1,4E0,4E0,6E0,5E0,2.7E1,6E0,1.1E1,1.1E1,1.6E1,3E0,3E0,3E0,8E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.5637407E-2,-2.5006428E-2,3.3905062E-1,-1.5941723E-1,4.133777E-2,2.185099E-2,4.2470414E-3,-7.091135E-2,-1.4007748E-2,5.673808E-2,-6.247722E-3,1.7179417E-3,-1.20544404E-1,1.5288186E-1,1.4230473E-2,-1.2896355E-3,-6.2999893E-3,2.6257324E-3,9.143504E-3,4.1992106E-2,-2.3015393E-3,6.445971E-2,-9.3140965E-4,-7.411087E-4,3.5054372E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,-1,15,17,19,-1,-1,-1,-1,21,-1,23,-1,-1,-1],"loss_changes":[1.0600588E0,6.486411E-1,3.2020152E-1,3.339417E-1,1.4973304E-1,0E0,0E0,9.857167E-2,0E0,1.858192E-1,0E0,0E0,2.4601787E-2,6.491217E-2,6.4603835E-2,0E0,0E0,0E0,0E0,3.5791658E-2,0E0,2.720429E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,12,12,13,13,14,14,19,19,21,21],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,-1,16,18,20,-1,-1,-1,-1,22,-1,24,-1,-1,-1],"split_conditions":[4.9891987E3,7.940699E-3,8.982776E9,1.0370839E6,2.953641E5,2.185099E-2,4.2470414E-3,4.4092423E-1,-1.4007748E-2,1.6256282E7,-6.247722E-3,1.7179417E-3,7.4338E4,4.189857E8,4.800481E5,-1.2896355E-3,-6.2999893E-3,2.6257324E-3,9.143504E-3,6.3315526E8,-2.3015393E-3,3.786E3,-9.3140965E-4,-7.411087E-4,3.5054372E-3],"split_indices":[4,37,5,27,32,0,0,26,0,44,0,0,12,7,27,0,0,0,0,7,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,2.3E1,4.8E1,4E0,4E0,1.6E1,7E0,4.5E1,3E0,5E0,1.1E1,1.3E1,3.2E1,3E0,8E0,6E0,7E0,2.3E1,9E0,1.7E1,6E0,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.2074906E-2,-1.9096171E-2,3.4432575E-1,-5.4967962E-2,8.952426E-2,2.3409376E-2,2.3584794E-3,3.024535E-2,-8.083202E-2,-4.1524414E-4,1.2565877E-1,-5.338709E-4,3.1969172E-3,-1.3285823E-2,-5.6436E-2,7.607575E-3,1.1269858E-3,-1.4143957E-1,-1.6384881E-2,-1.0491334E-3,-7.4432874E-3,3.075574E-3,-4.2548187E-2,-4.939354E-3,1.7371443E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,-1,-1,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1],"loss_changes":[8.4546125E-1,2.9715168E-1,3.8091153E-1,1.2807064E-1,6.935024E-2,0E0,0E0,2.756178E-2,2.2919422E-1,0E0,6.794058E-2,0E0,0E0,0E0,1.3726655E-1,0E0,0E0,4.53026E-2,7.0060916E-2,0E0,0E0,0E0,7.868144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,14,14,17,17,18,18,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,-1,-1,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1],"split_conditions":[4.0230347E3,7.114597E-1,1.904071E-1,2.7307262E5,1.9222778E5,2.3409376E-2,2.3584794E-3,3.015899E-2,2.8808794E5,-4.1524414E-4,2.156825E7,-5.338709E-4,3.1969172E-3,-1.3285823E-2,1.648E3,7.607575E-3,1.1269858E-3,1.1E1,1.774E3,-1.0491334E-3,-7.4432874E-3,3.075574E-3,4.8561827E8,-4.939354E-3,1.7371443E-4],"split_indices":[51,41,37,27,27,0,0,37,27,0,44,0,0,0,11,0,0,3,10,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.4E1,6E0,5.6E1,1.8E1,3E0,3E0,1.3E1,4.3E1,5E0,1.3E1,7E0,6E0,3E0,4E1,8E0,5E0,1.2E1,2.8E1,3E0,9E0,6E0,2.2E1,8E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-5.9960326E-3,2.7045945E-2,-2.5865164E-1,5.777442E-3,1.587435E-2,3.6735178E-3,-1.904019E-2,8.117006E-2,-2.6427446E-2,-1.9208613E-3,5.3964723E-3,-4.7913524E-3,-3.201398E-3,-1.8006846E-2,4.1957563E-3,3.6515899E-3,-3.8778543E-2,-2.1457253E-3,1.9245644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,-1,13,15,-1,-1,17,-1,-1],"loss_changes":[6.4981E-1,5.029323E-1,6.1687857E-1,1.6239382E-1,0E0,0E0,0E0,1.192089E-1,9.3102075E-2,0E0,0E0,0E0,5.8768597E-2,7.633468E-2,0E0,0E0,2.9952977E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,12,12,13,13,16,16],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,-1,14,16,-1,-1,18,-1,-1],"split_conditions":[1.5414109E8,2.1769184E7,2.7307262E5,2.5511668E7,1.587435E-2,3.6735178E-3,-1.904019E-2,1.51E3,4.593E3,-1.9208613E-3,5.3964723E-3,-4.7913524E-3,1.6132394E2,2.217E3,4.1957563E-3,3.6515899E-3,1.143417E6,-2.1457253E-3,1.9245644E-3],"split_indices":[44,1,27,44,0,0,0,2,2,0,0,0,56,10,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.8E1,8E0,6.5E1,3E0,3E0,5E0,1.9E1,4.6E1,5E0,1.4E1,9E0,3.7E1,3.3E1,4E0,5E0,2.8E1,2.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-5.711259E-3,-5.9429295E-2,9.795722E-2,-6.701965E-3,-1.8904324E-1,1.8801034E-4,1.24479495E-1,2.6698522E-2,-1.5843248E-1,-8.706835E-2,-2.6738772E-1,1.4957869E-1,6.092195E-4,-4.602374E-3,5.181901E-3,-1.15465885E-2,-3.0981613E-4,-3.0473905E-4,-5.7135304E-3,-1.41528435E-2,-5.1776837E-3,9.188629E-3,3.5230974E-3,2.6963288E-2,-3.8325759E-3,-2.293203E-3,5.869189E-3,-1.458305E-3,2.412053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,25,-1,27,-1,-1,-1],"loss_changes":[4.4019318E-1,3.541113E-1,6.645164E-2,1.9680908E-1,9.6556544E-2,0E0,5.5971086E-2,9.518956E-2,1.1612977E-1,2.8811987E-2,3.433603E-2,5.6880474E-2,0E0,6.950994E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1619308E-2,0E0,3.1797156E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,23,23,25,25],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,26,-1,28,-1,-1,-1],"split_conditions":[5.517496E-1,8.28645E-1,1.9222778E5,1.0129377E12,9.29295E5,1.8801034E-4,9.0161455E-1,2.8303965E3,4.077E3,1.2015E4,7.2767865E2,7.33E2,6.092195E-4,3.4941578E6,5.181901E-3,-1.15465885E-2,-3.0981613E-4,-3.0473905E-4,-5.7135304E-3,-1.41528435E-2,-5.1776837E-3,9.188629E-3,3.5230974E-3,3.241969E8,-3.8325759E-3,4.0614333E0,5.869189E-3,-1.458305E-3,2.412053E-3],"split_indices":[41,26,27,30,27,0,26,4,0,9,51,0,0,50,0,0,0,0,0,0,0,0,0,7,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,5.1E1,2.6E1,3.7E1,1.4E1,6E0,2E1,3.1E1,6E0,7E0,7E0,1.6E1,4E0,2.4E1,7E0,3E0,3E0,3E0,4E0,4E0,3E0,7E0,9E0,1.8E1,6E0,1.5E1,3E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.9165456E-4,-3.8781587E-2,3.4417918E-1,5.3044194E-3,-1.817248E-1,1.99703E-2,4.473567E-3,-8.6738095E-2,2.9731214E-2,-8.603092E-2,-1.6856587E-2,-2.9449616E-2,-8.294046E-3,7.4777864E-3,4.1837315E-3,-7.3961904E-3,5.6383194E-4,-3.3043532E-3,2.122967E-3,4.2452343E-2,-3.60083E-3,2.4153173E-3,-2.0879728E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,-1,-1,-1],"loss_changes":[1.0559442E0,4.546932E-1,1.8653047E-1,1.2848383E-1,3.1171566E-1,0E0,0E0,6.799406E-2,6.7467555E-2,1.14397846E-1,0E0,3.7877567E-2,0E0,1.15803E-1,0E0,0E0,0E0,0E0,0E0,3.605155E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,13,13,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,-1,-1,-1],"split_conditions":[4.0230347E3,1.0104842E12,8.982776E9,2.98E2,3.5715E4,1.99703E-2,4.473567E-3,1E0,1.204781E3,5.034889E6,-1.6856587E-2,1.58966E5,-8.294046E-3,6.98494E7,4.1837315E-3,-7.3961904E-3,5.6383194E-4,-3.3043532E-3,2.122967E-3,5.557455E8,-3.60083E-3,2.4153173E-3,-2.0879728E-3],"split_indices":[51,30,5,0,9,0,0,112,51,28,0,1,0,44,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.1E1,7E0,5.5E1,1.6E1,4E0,3E0,1.1E1,4.4E1,1.2E1,4E0,8E0,3E0,3.4E1,1E1,6E0,6E0,5E0,3E0,2.5E1,9E0,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.0148763E-2,5.6554414E-3,-2.7899873E-1,-9.1230854E-2,5.1017262E-2,-2.023174E-2,-6.473222E-4,-1.2074006E-1,3.3835531E-3,1.2188841E-1,1.9186279E-2,-6.4867088E-3,-9.940204E-4,8.8833953E-4,7.2855134E-3,-1.8867245E-2,7.7614225E-2,-1.7115562E-3,2.611574E-3,5.4111253E-3,1.0735798E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,-1,-1,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[5.1589304E-1,3.1669497E-1,3.5136843E-1,1.251744E-1,1.0820043E-1,0E0,0E0,5.6821316E-2,0E0,6.99919E-2,7.928299E-2,0E0,0E0,0E0,0E0,3.9651208E-2,3.278754E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,15,15,16,16],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,-1,-1,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[1.5414109E8,4.279412E0,1.6E1,1.0735684E3,1.6256282E7,-2.023174E-2,-6.473222E-4,1.6825E4,3.3835531E-3,4.4939005E2,3.029697E3,-6.4867088E-3,-9.940204E-4,8.8833953E-4,7.2855134E-3,5.0175633E0,1.0129377E12,-1.7115562E-3,2.611574E-3,5.4111253E-3,1.0735798E-3],"split_indices":[44,57,3,51,44,0,0,9,0,4,4,0,0,0,0,53,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,2.2E1,4.8E1,3E0,3E0,1.9E1,3E0,1.4E1,3.4E1,1.4E1,5E0,5E0,9E0,2.1E1,1.3E1,1.7E1,4E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.2194844E-4,-2.5736807E-2,1.3198032E-2,1.7811785E-2,-1.6511537E-1,-6.985885E-2,5.2434742E-2,-6.9658205E-2,-1.958286E-2,-1.434458E-1,3.2796732E-3,2.7613118E-2,4.6206024E-3,-7.3789353E-3,2.8558375E-4,-7.75484E-3,-1.3593952E-3,3.210601E-3,-1.9622368E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,-1,5,7,9,11,13,-1,15,-1,17,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.18537E-1,4.3389258E-1,0E0,1.7061093E-1,4.4328308E-1,1.835511E-1,5.223558E-2,1.1111727E-1,0E0,4.0782303E-2,0E0,1.0461639E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,11,11],"right_children":[2,4,-1,6,8,10,12,14,-1,16,-1,18,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.660861E3,1.0232127E12,1.3198032E-2,5.02409E5,1.5135763E8,5.596155E-1,1.7830602E3,4.375793E6,-1.958286E-2,1.3513911E1,3.2796732E-3,5.7229916E7,4.6206024E-3,-7.3789353E-3,2.8558375E-4,-7.75484E-3,-1.3593952E-3,3.210601E-3,-1.9622368E-3],"split_indices":[51,30,0,1,44,38,51,1,0,55,0,44,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7E1,5E0,5.4E1,1.6E1,1.5E1,3.9E1,1.3E1,3E0,1E1,5E0,2.8E1,1.1E1,5E0,8E0,7E0,3E0,1.7E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.6400775E-2,-2.2976097E-2,2.3160684E-1,4.491022E-3,-2.2267409E-1,1.7653229E-2,1.8473951E-3,-1.3448575E-1,2.57857E-2,-1.4549338E-2,-8.578628E-4,-1.8978072E-3,-7.6834317E-3,3.6859646E-2,-3.6640125E-3,1.03758745E-1,1.8760912E-2,8.905121E-4,5.829795E-3,1.7850982E-3,2.6431878E-3,1.0591097E-3,-2.6358312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,15,-1,17,19,-1,-1,21,-1,-1,-1],"loss_changes":[6.578303E-1,3.6301848E-1,3.9185107E-1,1.7807628E-1,1.8372184E-1,0E0,0E0,2.4599954E-2,6.734595E-2,0E0,0E0,0E0,0E0,5.617126E-2,0E0,2.4171598E-2,2.8390463E-2,0E0,0E0,4.3796003E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,13,13,15,15,16,16,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,16,-1,18,20,-1,-1,22,-1,-1,-1],"split_conditions":[2.6949062E3,1.5135763E8,1.3946067E7,2.1E2,3.435678E8,1.7653229E-2,1.8473951E-3,1E0,1E0,-1.4549338E-2,-8.578628E-4,-1.8978072E-3,-7.6834317E-3,1.7329262E7,-3.6640125E-3,8.747313E6,3.1672727E3,8.905121E-4,5.829795E-3,7.76264E7,2.6431878E-3,1.0591097E-3,-2.6358312E-3],"split_indices":[51,44,46,10,12,0,0,111,112,0,0,0,0,44,0,44,4,0,0,44,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.5E1,1.1E1,5.8E1,7E0,5E0,6E0,7E0,5.1E1,4E0,3E0,3E0,4E0,4.7E1,4E0,9E0,3.8E1,3E0,6E0,2.8E1,1E1,2.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.6840015E-2,3.5289768E-2,-7.284617E-2,-4.283189E-3,5.815482E-2,-1.2934384E-1,2.505884E-4,-2.6282337E-3,7.1713045E-2,-1.9054475E-1,-2.6570966E-2,1.3743454E-1,4.287798E-2,-4.481184E-3,-1.0651398E-2,2.2508444E-3,-3.204954E-3,1.8109176E-3,7.608443E-3,-5.149703E-5,3.8947552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,-1,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.186203E-1,1.2661085E-1,1.6083947E-1,0E0,5.954753E-2,1.2702271E-1,0E0,0E0,5.2854672E-2,3.9152443E-2,3.842946E-2,2.711241E-2,4.897837E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,8,8,9,9,10,10,11,11,12,12],"right_children":[2,4,6,-1,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7376137E-1,3.7605792E2,1.6595918E-1,-4.283189E-3,2.23E2,1.4235585E3,2.505884E-4,-2.6282337E-3,6.1577463E0,9.29295E5,1.5368E4,2.1262457E0,1.0403953E3,-4.481184E-3,-1.0651398E-2,2.2508444E-3,-3.204954E-3,1.8109176E-3,7.608443E-3,-5.149703E-5,3.8947552E-3],"split_indices":[26,4,38,0,10,4,0,0,55,27,9,55,51,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3.8E1,3.5E1,5E0,3.3E1,2E1,1.5E1,3E0,3E1,1.2E1,8E0,8E0,2.2E1,6E0,6E0,3E0,5E0,3E0,5E0,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.2130767E-2,-8.4802E-3,2.1888897E-1,-7.729291E-2,4.2447068E-2,1.9493025E-2,5.2220885E-2,-2.2486696E-2,-5.272389E-3,4.797697E-3,7.241265E-4,-1.3039544E-3,5.2119405E-3,-5.7250157E-2,3.3669428E-3,-4.7865384E-3,-4.2821982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,15,-1,-1,-1],"loss_changes":[4.4755512E-1,2.310697E-1,3.7111074E-1,6.781736E-2,6.571936E-2,0E0,4.4047922E-2,5.3770743E-2,0E0,0E0,0E0,0E0,0E0,2.7291413E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,13,13],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,16,-1,-1,-1],"split_conditions":[6.7780703E3,6.0199605E6,9.192915E0,5.8871865E-1,1.7329262E7,1.9493025E-2,5.2E1,4.2332153E0,-5.272389E-3,4.797697E-3,7.241265E-4,-1.3039544E-3,5.2119405E-3,2.0435429E8,3.3669428E-3,-4.7865384E-3,-4.2821982E-4],"split_indices":[4,50,56,26,44,0,8,53,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.4E1,9E0,2.7E1,3.7E1,3E0,6E0,1.3E1,1.4E1,9E0,2.8E1,3E0,3E0,1E1,3E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.708013E-3,-1.3678592E-2,2.6891476E-1,4.5294645E-3,-1.7356527E-1,2.0133464E-2,-9.042819E-6,-5.253121E-2,4.2906906E-2,3.5952816E-3,-1.475177E-2,-7.14166E-2,2.2408804E-3,1.0652952E-1,2.2932407E-2,6.7398176E-3,-4.3697716E-3,-8.8201766E-4,7.0119365E-3,5.8309175E-3,2.9843901E-3,-1.8544922E-3,2.8945482E-3,1.4451584E-3,-1.5126783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[5.128834E-1,2.2786173E-1,3.8045102E-1,1.5773278E-1,3.822744E-1,0E0,0E0,6.1211832E-2,5.220271E-2,0E0,0E0,6.0807794E-2,0E0,7.64219E-2,2.6837666E-2,2.7366389E-2,0E0,0E0,0E0,3.115299E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,13,13,14,14,15,15,19,19],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,1.1408508E9,3.1457312E8,2.7307262E5,2.0133464E-2,-9.042819E-6,4.7985487E0,1.6256282E7,3.5952816E-3,-1.475177E-2,1.5368E4,2.2408804E-3,1.9222778E5,5.0175633E0,1.5023E4,-4.3697716E-3,-8.8201766E-4,7.0119365E-3,3.28464E0,2.9843901E-3,-1.8544922E-3,2.8945482E-3,1.4451584E-3,-1.5126783E-3],"split_indices":[51,44,7,7,27,0,0,53,44,0,0,9,0,27,53,9,0,0,0,53,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.7E1,6E0,7E1,7E0,3E0,3E0,2.8E1,4.2E1,3E0,4E0,2.4E1,4E0,9E0,3.3E1,7E0,1.7E1,3E0,6E0,2.5E1,8E0,4E0,3E0,1.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.4178472E-3,-1.9839479E-2,1.3543539E-2,2.2233801E-3,-5.4569315E-2,-9.610083E-2,1.768151E-2,-6.870575E-2,-8.305915E-3,2.7290715E-3,-1.0951506E-3,-4.373685E-3,-8.121649E-3,2.6205312E-3,-2.762623E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,-1,5,7,9,11,-1,-1,-1,-1,13,-1,-1],"loss_changes":[4.2823428E-1,1.7497307E-1,0E0,0E0,1.4273147E-1,6.730312E-2,3.758589E-2,5.0016E-2,0E0,0E0,0E0,0E0,4.315765E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,-1,-1,6,8,10,12,-1,-1,-1,-1,14,-1,-1],"split_conditions":[2.1769184E7,3.6130127E-1,1.3543539E-2,2.2233801E-3,4.283039E1,2.0333653E5,5.4734116E9,5.4111224E2,-8.305915E-3,2.7290715E-3,-1.0951506E-3,-4.373685E-3,1.5326E4,2.6205312E-3,-2.762623E-3],"split_indices":[1,26,0,0,57,32,5,51,0,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.8E1,3E0,2.2E1,4.6E1,2.9E1,1.7E1,2.4E1,5E0,8E0,9E0,1.5E1,9E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.3094041E-3,-2.1720128E-2,1.2349375E-2,-1.1930148E-1,6.5481183E-3,-8.695391E-4,-6.0114884E-3,2.242943E-2,-8.788545E-3,-4.956161E-3,3.6805633E-3,-1.9529995E-2,3.2511167E-3,2.3623288E-2,-2.6314133E-3,3.5430347E-3,-4.166359E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,7,-1,-1,9,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[4.5034984E-1,1.966314E-1,0E0,3.0829445E-2,1.9320983E-1,0E0,0E0,9.279443E-2,0E0,4.6333138E-2,0E0,6.127045E-2,0E0,2.484786E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,9,9,11,11,13,13],"right_children":[2,4,-1,6,8,-1,-1,10,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[7.6312915E3,3.731E3,1.2349375E-2,4.0246898E-1,2.5805172E7,-8.695391E-4,-6.0114884E-3,7.9276E-2,-8.788545E-3,7.0826065E1,3.6805633E-3,6.98494E7,3.2511167E-3,2.3209375E6,-2.6314133E-3,3.5430347E-3,-4.166359E-5],"split_indices":[51,11,0,26,46,0,0,37,0,56,0,44,0,50,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7E1,4E0,1.5E1,5.5E1,3E0,1.2E1,5.2E1,3E0,3.7E1,1.5E1,3.2E1,5E0,1.6E1,1.6E1,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[6.4163175E-3,-1.9764232E-2,2.2387463E-1,6.819803E-3,-2.43258E-1,3.4413047E-3,1.637866E-2,3.8325094E-2,-3.4364402E-2,-3.923652E-3,-1.3505083E-2,-2.6299434E-3,5.8338765E-2,-5.6182365E-3,-1.8718965E-2,-4.3122875E-4,7.7148154E-2,1.7720513E-3,-1.6198772E-3,9.911954E-4,4.2697773E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,15,-1,17,-1,19,-1,-1,-1,-1],"loss_changes":[4.769585E-1,4.472614E-1,1.7455706E-1,8.954841E-2,5.895999E-2,0E0,0E0,8.15047E-2,4.3076247E-2,0E0,0E0,0E0,4.324872E-2,0E0,3.2692313E-2,0E0,2.9536694E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,12,12,14,14,16,16],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,16,-1,18,-1,20,-1,-1,-1,-1],"split_conditions":[4.9891987E3,1.15266744E8,1.3996E4,5.090285E7,9.3570206E5,3.4413047E-3,1.637866E-2,2.0435429E8,7.801514E2,-3.923652E-3,-1.3505083E-2,-2.6299434E-3,2.5630938E5,-5.6182365E-3,1.120848E5,-4.3122875E-4,6.8177136E2,1.7720513E-3,-1.6198772E-3,9.911954E-4,4.2697773E-3],"split_indices":[4,44,9,44,27,0,0,7,4,0,0,0,27,0,31,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,6.7E1,7E0,5E0,3E0,3.8E1,2.9E1,3E0,4E0,6E0,3.2E1,3E0,2.6E1,7E0,2.5E1,6E0,2E1,8E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.0528256E-3,-2.9798124E-2,2.4247636E-1,3.5122198E-3,-1.5853176E-1,1.5186589E-2,5.5053935E-4,-5.8366675E-2,2.030846E-2,-1.118973E-2,9.6615084E-4,-7.1998354E-4,-4.8383037E-3,4.579047E-3,3.7899753E-3,2.2516385E-2,-2.525277E-3,2.4012732E-3,-2.783299E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,15,17,-1,-1,-1],"loss_changes":[5.892119E-1,2.9215178E-1,2.2822142E-1,5.834864E-2,2.6743665E-1,0E0,0E0,2.5392842E-2,6.2660165E-2,0E0,0E0,0E0,0E0,0E0,4.604858E-2,2.965433E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,14,14,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,16,18,-1,-1,-1],"split_conditions":[3.3072104E3,1.0789844E6,8.982776E9,2.566528E0,1.650581E3,1.5186589E-2,5.5053935E-4,1.9227438E5,1.3024019E7,-1.118973E-2,9.6615084E-4,-7.1998354E-4,-4.8383037E-3,4.579047E-3,5.451104E5,1E0,-2.525277E-3,2.4012732E-3,-2.783299E-4],"split_indices":[51,27,5,53,4,0,0,32,44,0,0,0,0,0,27,111,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,5.4E1,1.3E1,5E0,3E0,1.1E1,4.3E1,8E0,5E0,7E0,4E0,6E0,3.7E1,2.9E1,8E0,1.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.8821817E-2,6.513124E-4,1.5755745E-2,1.5206267E-2,-1.16494335E-2,-8.7076865E-2,3.607094E-2,3.1107303E-3,-6.085089E-3,9.0867594E-2,1.5483406E-2,1.9333474E-3,6.3988105E-3,2.9734147E-3,4.296543E-3,-4.5213415E-4,2.0260846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,13,-1,-1,15,-1,-1,-1],"loss_changes":[5.1433367E-1,3.1415313E-1,0E0,1.6319874E-1,0E0,1.2508008E-1,6.966513E-2,0E0,0E0,3.8847312E-2,4.9672425E-2,0E0,0E0,2.5765456E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9,10,10,13,13],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,14,-1,-1,16,-1,-1,-1],"split_conditions":[7.6312915E3,1.5277152E8,1.5755745E-2,1.7363004E4,-1.16494335E-2,1.2015E4,2.156825E7,3.1107303E-3,-6.085089E-3,4.1108544E8,3.4409692E3,1.9333474E-3,6.3988105E-3,4.9153895E0,4.296543E-3,-4.5213415E-4,2.0260846E-3],"split_indices":[51,44,0,50,0,9,44,0,0,7,51,0,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,7.4E1,3E0,1.2E1,6.2E1,3E0,9E0,1.6E1,4.6E1,1E1,6E0,4.1E1,5E0,3.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.4485215E-2,2.5037797E-3,-1.8230584E-1,-3.8953675E-3,2.7529225E-2,-2.2674387E-3,-1.13755455E-2,8.252429E-3,9.065614E-2,-3.3313275E-4,4.0266677E-3,7.7218297E-3,-5.221486E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,-1,7,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[2.1714033E-1,1.6646607E-1,6.1647728E-2,0E0,6.7397416E-2,0E0,0E0,6.137479E-2,1.2244543E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,8,8],"right_children":[2,4,6,-1,8,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.648E3,4.66E2,-3.8953675E-3,1.891862E3,-2.2674387E-3,-1.13755455E-2,1.0379711E0,1.3946067E7,-3.3313275E-4,4.0266677E-3,7.7218297E-3,-5.221486E-4],"split_indices":[44,11,0,0,51,0,0,41,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,1.4E1,5.5E1,3E0,3E0,4.3E1,1.2E1,3.7E1,6E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-5.363816E-3,-1.0751262E-1,2.6641496E-2,-2.35924E-2,-9.854531E-3,-6.254303E-2,3.694581E-2,-2.992021E-3,4.2011673E-3,8.7164465E-4,-5.552096E-3,5.8769984E-3,2.695744E-2,6.0710985E-2,-4.393295E-5,-6.2641635E-4,3.477225E-3,-1.0101526E-3,3.0231779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,-1,-1,-1,-1,-1,13,15,17,-1,-1,-1,-1],"loss_changes":[2.742166E-1,2.0089172E-1,6.1125763E-2,8.032447E-2,0E0,4.170724E-2,5.4508865E-2,0E0,0E0,0E0,0E0,0E0,4.921784E-2,3.8453184E-2,5.3374946E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,12,12,13,13,14,14],"right_children":[2,4,6,8,-1,10,12,-1,-1,-1,-1,-1,14,16,18,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,9.29295E5,2.074E3,1E0,-9.854531E-3,1.297959E7,2.2235682E6,-2.992021E-3,4.2011673E-3,8.7164465E-4,-5.552096E-3,5.8769984E-3,5.090285E7,2.5630938E5,4.9767124E1,-6.2641635E-4,3.477225E-3,-1.0101526E-3,3.0231779E-3],"split_indices":[40,27,2,15,0,12,50,0,0,0,0,0,44,27,56,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,1.9E1,6.3E1,1.2E1,7E0,6E0,5.7E1,9E0,3E0,3E0,3E0,4E0,5.3E1,2.3E1,3E1,5E0,1.8E1,2.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.693405E-2,4.13869E-3,1.9617298E-1,-5.844114E-2,3.7422627E-2,1.499067E-2,2.589713E-3,-1.5262753E-1,-1.8087281E-3,4.163394E-3,8.438932E-4,-1.6765323E-3,-1.0071796E-2,-1.434124E-3,2.4229684E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9238567E-1,1.4382444E-1,1.6719195E-1,1.2773997E-1,4.578729E-2,0E0,0E0,7.538779E-2,3.1581394E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9891987E3,2.8506322E-2,3.6274084E9,3.0810812E0,3.5175372E5,1.499067E-2,2.589713E-3,9.29295E5,9.402116E2,4.163394E-3,8.438932E-4,-1.6765323E-3,-1.0071796E-2,-1.434124E-3,2.4229684E-3],"split_indices":[4,38,12,52,31,0,0,27,51,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,2.3E1,4.4E1,3E0,5E0,8E0,1.5E1,9E0,3.5E1,4E0,4E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.064792E-2,2.0973708E-2,-1.3646895E-1,-4.197483E-2,4.283103E-2,-5.771324E-2,-1.2751524E-2,-4.0191924E-3,2.9526069E-3,2.0472784E-2,4.0910444E-3,-4.918086E-3,8.017551E-4,3.567889E-3,-8.048857E-3,-1.6376469E-3,1.1935147E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1,15,-1,-1],"loss_changes":[3.1717682E-1,8.985457E-2,1.8657237E-1,1.04654364E-1,5.474507E-2,5.472601E-2,0E0,0E0,0E0,6.352257E-2,0E0,0E0,0E0,0E0,2.8259704E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,14,14],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.0232127E12,2.4786325E0,1.3946397E5,1E0,1.7830602E3,7.575E3,-1.2751524E-2,-4.0191924E-3,2.9526069E-3,1.6939252E7,4.0910444E-3,-4.918086E-3,8.017551E-4,3.567889E-3,3.0546596E0,-1.6376469E-3,1.1935147E-3],"split_indices":[30,55,32,108,51,2,0,0,0,44,0,0,0,0,52,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,6.3E1,1.5E1,1.6E1,4.7E1,1.1E1,4E0,1.1E1,5E0,3.4E1,1.3E1,6E0,5E0,1E1,2.4E1,1.3E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-2.2003432E-3,1.3855694E-2,-1.1434812E-1,-1.1150522E-2,6.817658E-2,3.4854615E-3,-9.6584875E-3,-2.1333362E-3,2.6386695E-2,3.7836058E-3,2.591604E-4,4.293631E-3,1.0609669E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,-1,-1,-1,11,-1,-1,-1,-1],"loss_changes":[1.2943046E-1,8.633688E-2,2.1496859E-1,6.5295205E-2,2.4684705E-2,0E0,0E0,0E0,4.447379E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8],"right_children":[2,4,6,8,10,-1,-1,-1,12,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.1808436E3,2.7307262E5,1.9129465E0,4.815659E6,3.4854615E-3,-9.6584875E-3,-2.1333362E-3,9.9948E4,3.7836058E-3,2.591604E-4,4.293631E-3,1.0609669E-5],"split_indices":[44,51,27,56,28,0,0,0,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.2E1,8E0,4.3E1,1.9E1,3E0,5E0,2.1E1,2.2E1,1.4E1,5E0,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.0043612E-3,1.9909082E-2,-1.1606994E-2,7.1236794E-3,1.0069891E-2,-4.2050085E-3,1.6898869E-2,-5.4443326E-2,2.8062813E-2,1.2662858E-4,-4.317341E-3,7.0129715E-2,1.371399E-2,-1.040023E-3,4.195067E-3,-3.697128E-4,3.686076E-3,1.6813657E-3,-1.3889614E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,-1,-1,7,9,11,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[3.968184E-1,1.9676611E-1,0E0,7.27905E-2,0E0,0E0,5.250129E-2,2.4330903E-2,3.2840434E-2,0E0,0E0,3.9370775E-2,4.3927696E-2,0E0,0E0,4.842964E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,11,11,12,12,15,15],"right_children":[2,4,-1,6,-1,-1,8,10,12,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[2.9851845E10,2.1747866E7,-1.1606994E-2,1.09E2,1.0069891E-2,-4.2050085E-3,1.1E1,5.86E2,1.6939252E7,1.2662858E-4,-4.317341E-3,1.9222778E5,2.001E3,-1.040023E-3,4.195067E-3,4.0526348E-1,3.686076E-3,1.6813657E-3,-1.3889614E-3],"split_indices":[5,1,0,10,0,0,3,0,44,0,0,27,0,0,0,26,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.1E1,4E0,6.8E1,3E0,5E0,6.3E1,8E0,5.5E1,4E0,4E0,1.3E1,4.2E1,3E0,1E1,3.6E1,6E0,1.6E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.200589E-3,-2.1732036E-2,1.0016172E-2,-2.6431836E-3,-1.3602486E-1,-3.1311153E-3,1.9123519E-2,3.4403803E-3,-2.2726008E-1,2.9781288E-3,-7.403925E-3,-1.426959E-2,-1.5997718E-3,-1.9696243E-3,8.791814E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1],"loss_changes":[3.075554E-1,1.6763279E-1,0E0,1.0432693E-1,2.3553672E-1,0E0,7.020253E-2,0E0,1.4915979E-1,0E0,3.877893E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,10,10],"right_children":[2,4,-1,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.0016172E-2,6.942296E2,2.7307262E5,-3.1311153E-3,5.7229916E7,3.4403803E-3,1.835052E6,2.9781288E-3,1.077646E3,-1.426959E-2,-1.5997718E-3,-1.9696243E-3,8.791814E-4],"split_indices":[51,44,0,4,27,0,44,0,31,0,51,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.6E1,4E0,6.6E1,1E1,1.5E1,5.1E1,3E0,7E0,1.7E1,3.4E1,4E0,3E0,1.4E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.6793303E-2,-9.393666E-3,-5.1272856E-3,5.90596E-3,-8.185342E-3,-7.9633825E-2,2.5657946E-2,-1.709952E-3,-6.7926743E-3,4.0292032E-2,-1.619516E-3,1.4328537E-2,3.7665039E-3,2.9905946E-3,-1.9751016E-2,-2.4464256E-3,1.8638973E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,-1,3,5,-1,7,9,-1,-1,11,-1,13,-1,-1,15,-1,-1],"loss_changes":[1.740114E-1,0E0,1.4705747E-1,1.1693473E-1,0E0,3.447441E-2,5.358901E-2,0E0,0E0,5.5809796E-2,0E0,6.0126282E-2,0E0,0E0,4.9942862E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,9,9,11,11,14,14],"right_children":[2,-1,4,6,-1,8,10,-1,-1,12,-1,14,-1,-1,16,-1,-1],"split_conditions":[7E0,-9.393666E-3,2.292109E7,1.5278552E4,-8.185342E-3,9.1569895E-1,6.349519E6,-1.709952E-3,-6.7926743E-3,1.1808436E3,-1.619516E-3,1.1795429E9,3.7665039E-3,2.9905946E-3,3.5671377E0,-2.4464256E-3,1.8638973E-3],"split_indices":[3,0,46,50,0,26,31,0,0,51,0,5,0,0,52,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3E0,7E1,6.7E1,3E0,1.2E1,5.5E1,9E0,3E0,4.5E1,1E1,3E1,1.5E1,1.1E1,1.9E1,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.1337942E-3,-1.0757713E-2,8.935454E-3,2.2692056E-3,-1.2832415E-1,-5.536229E-3,1.3209011E-2,-9.376748E-3,-2.2647734E-4,-3.0485902E-3,2.391797E-2,1.5370602E-2,4.47338E-3,-2.4051809E-4,4.9921643E-2,3.2920255E-3,-3.1927234E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,-1,9,-1,-1,-1,11,13,-1,-1,15,-1,-1],"loss_changes":[1.8265042E-1,1.0749818E-1,0E0,9.394156E-2,7.716199E-2,0E0,5.599886E-2,0E0,0E0,0E0,3.563524E-2,3.655076E-2,0E0,0E0,3.0081086E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,10,10,11,11,14,14],"right_children":[2,4,-1,6,8,-1,10,-1,-1,-1,12,14,-1,-1,16,-1,-1],"split_conditions":[6.5915E4,3.698723E10,8.935454E-3,7.446808E-2,3.132E3,-5.536229E-3,1.296E3,-9.376748E-3,-2.2647734E-4,-3.0485902E-3,1.7823958E0,1.2544873E3,4.47338E-3,-2.4051809E-4,6.7584877E8,3.2920255E-3,-3.1927234E-4],"split_indices":[2,5,0,57,0,0,2,0,0,0,40,51,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.9E1,3E0,6.3E1,6E0,4E0,5.9E1,3E0,3E0,6E0,5.3E1,4.9E1,4E0,3.1E1,1.8E1,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-7.395596E-3,-8.771179E-3,3.8474181E-3,1.6381849E-2,-7.050952E-3,3.8326194E-3,9.11174E-3,-3.9967686E-2,2.9378345E-2,6.267657E-3,-4.503994E-3,1.7474805E-3,-1.9000736E-3,-3.121586E-3,1.3755604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,-1,3,5,-1,7,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[1.6528434E-1,0E0,1.5286219E-1,1.6456185E-1,0E0,7.278447E-2,0E0,7.318258E-2,3.657425E-2,3.3141453E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,9,9],"right_children":[2,-1,4,6,-1,8,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[5E0,-8.771179E-3,1.3866357E8,3.9509753E3,-7.050952E-3,2.881172E8,9.11174E-3,8.0159146E-1,2.5922572E7,1.001E3,-4.503994E-3,1.7474805E-3,-1.9000736E-3,-3.121586E-3,1.3755604E-3],"split_indices":[3,0,47,51,0,7,0,26,31,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3E0,7E1,6.6E1,4E0,6.3E1,3E0,2.3E1,4E1,1.4E1,9E0,3.5E1,5E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.0723137E-3,-8.5909795E-3,1.0697194E-2,9.614017E-4,-8.649597E-3,8.117288E-3,-6.012843E-3,-3.850339E-3,1.4552777E-2,6.908579E-3,3.7556298E-3,5.6384385E-2,-6.7104334E-3,3.755906E-4,4.1453713E-3,-3.2477104E-3,2.124768E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,-1,13,15,-1,-1,-1,-1],"loss_changes":[2.557413E-1,1.5745021E-1,0E0,8.4211655E-2,0E0,5.121307E-2,0E0,0E0,4.1400604E-2,4.6871368E-2,0E0,2.8104484E-2,4.4851087E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,11,11,12,12],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,-1,14,16,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,1.0697194E-2,2.2306405E6,-8.649597E-3,5.734767E-2,-6.012843E-3,-3.850339E-3,4.9891987E3,1.7329262E7,3.7556298E-3,2.9187737E0,2.35764E5,3.755906E-4,4.1453713E-3,-3.2477104E-3,2.124768E-4],"split_indices":[1,2,0,27,0,57,0,0,4,44,0,53,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.4E1,3E0,8.1E1,3E0,7.8E1,3E0,4E0,7.4E1,6.8E1,6E0,1.4E1,5.4E1,7E0,7E0,7E0,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.8342175E-2,4.210557E-3,1.2976714E-2,1.2470733E-2,-6.84129E-3,-1.9795425E-2,2.2920885E-3,2.4096826E-3,-8.463627E-3,-2.6712036E-2,4.958187E-2,-4.322486E-2,1.5195858E-3,4.8055174E-3,2.8589224E-5,-2.848112E-3,1.1254408E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,-1,7,-1,9,-1,11,13,15,-1,-1,-1,-1,-1],"loss_changes":[3.4119606E-1,1.1320739E-1,0E0,1.0589931E-1,0E0,1.7846255E-1,0E0,5.764757E-2,0E0,2.8260658E-2,4.906754E-2,3.589257E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,9,9,10,10,11,11],"right_children":[2,4,-1,6,-1,8,-1,10,-1,12,14,16,-1,-1,-1,-1,-1],"split_conditions":[9.55721E3,2.5805172E7,1.2976714E-2,1.7807747E3,-6.84129E-3,1.052686E6,2.2920885E-3,4.3652397E8,-8.463627E-3,2.1E1,2.325535E7,2.68219E5,1.5195858E-3,4.8055174E-3,2.8589224E-5,-2.848112E-3,1.1254408E-3],"split_indices":[4,46,0,4,0,27,0,7,0,3,44,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,8.1E1,3E0,7.8E1,3E0,4.4E1,3.4E1,4E1,4E0,2.5E1,1.5E1,2E1,5E0,6E0,9E0,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.5442309E-2,5.7833386E-4,-1.7371845E-1,-2.5272978E-2,4.9490523E-2,-9.4285025E-4,-1.3678969E-2,-4.053253E-2,1.8985323E-3,1.0105279E-1,4.503017E-3,-5.2775275E-2,9.0473064E-4,2.198966E-3,8.546961E-3,-2.745345E-3,1.4006452E-3,-3.7324363E-3,-1.9081002E-2,-1.4987033E-3,2.28966E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,-1,-1,-1,-1,-1,-1,19,-1,-1],"loss_changes":[2.207051E-1,1.02406316E-1,1.7029025E-1,5.8144998E-2,6.449538E-2,0E0,0E0,3.434737E-2,0E0,5.347757E-2,3.3156414E-2,4.1352957E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.506599E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,18,18],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,-1,-1,-1,-1,-1,-1,20,-1,-1],"split_conditions":[1.5135763E8,1.204781E3,1.6867E4,1.062872E6,1.4302E4,-9.4285025E-4,-1.3678969E-2,2.1E1,1.8985323E-3,9.698E3,1.2204E4,2.289789E6,9.0473064E-4,2.198966E-3,8.546961E-3,-2.745345E-3,1.4006452E-3,-3.7324363E-3,5.0175633E0,-1.4987033E-3,2.28966E-3],"split_indices":[44,51,9,9,10,0,0,3,0,2,2,50,0,0,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,5.2E1,2.7E1,4E0,3E0,4.3E1,9E0,1.2E1,1.5E1,3.6E1,7E0,9E0,3E0,4E0,1.1E1,1.7E1,1.9E1,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.1001062E-2,2.2680014E-2,-5.52554E-2,-3.5939754E-3,1.5027825E-3,-8.571232E-2,6.935895E-4,-6.4120725E-2,-7.5964686E-3,1.452593E-3,-8.235601E-2,-4.3654586E-3,-4.5523487E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,-1,7,-1,9,-1,-1,11,-1,-1],"loss_changes":[1.17514804E-1,6.280247E-2,7.540477E-2,0E0,0E0,3.6246404E-2,0E0,4.035978E-2,0E0,0E0,2.4378404E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,10,10],"right_children":[2,4,6,-1,-1,8,-1,10,-1,-1,12,-1,-1],"split_conditions":[7.8345644E-1,3.235021E2,1.6925E4,-3.5939754E-3,1.5027825E-3,1.58206E6,6.935895E-4,2.202021E8,-7.5964686E-3,1.452593E-3,7.0333635E6,-4.3654586E-3,-4.5523487E-4],"split_indices":[26,4,10,0,0,27,0,7,0,0,31,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,4.4E1,3.3E1,4E0,4E1,2.3E1,1E1,2E1,3E0,3E0,1.7E1,1.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.6991522E-2,5.2764895E-3,1.0940148E-2,1.7171655E-2,-7.856569E-3,6.267542E-2,1.7175402E-3,-4.366747E-4,4.047201E-3,-1.8609276E-3,1.6430588E-2,4.1084653E-3,8.903775E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[2.3927875E-1,1.8975322E-1,0E0,5.544676E-2,0E0,4.7565363E-2,4.064727E-2,0E0,0E0,0E0,5.236745E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,10,10],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[7.6312915E3,1.5414109E8,1.0940148E-2,1.7329262E7,-7.856569E-3,1.63254E5,5.2886304E2,-4.366747E-4,4.047201E-3,-1.8609276E-3,2.5036643E8,4.1084653E-3,8.903775E-5],"split_indices":[51,44,0,44,0,1,51,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8.2E1,3E0,7.8E1,4E0,1.9E1,5.9E1,6E0,1.3E1,1.4E1,4.5E1,6E0,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1138222E-2,-8.2536833E-4,1.058927E-2,-4.8880085E-2,1.2394956E-3,-1.3220548E-2,-1.345789E-1,-2.3657857E-3,1.2821914E-3,-8.202078E-3,-6.4570416E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,-1],"loss_changes":[2.3503102E-1,1.16019346E-1,0E0,9.295617E-2,0E0,4.3495543E-2,5.8636725E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,-1],"split_conditions":[7.6312915E3,1.03593215E-1,1.058927E-2,1.052622E6,1.2394956E-3,1.3E1,1.4235585E3,-2.3657857E-3,1.2821914E-3,-8.202078E-3,-6.4570416E-4],"split_indices":[51,41,0,27,0,3,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,3E1,4.9E1,2.2E1,8E0,1.1E1,1.1E1,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-4.331802E-3,5.3937766E-3,-7.135269E-3,-5.879663E-3,1.42568E-2,-1.999665E-3,3.19095E-3,4.591095E-2,-2.3248835E-2,-3.5294245E-3,3.557072E-3,-3.568722E-2,1.6566316E-3,-1.7328085E-2,-3.385792E-3,-3.899823E-3,-5.204307E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,-1,5,7,-1,9,11,-1,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.3999574E-1,1.08338825E-1,0E0,0E0,7.865125E-2,6.6223264E-2,0E0,1.02607176E-1,3.6079135E-2,0E0,0E0,2.8967366E-2,0E0,3.3211436E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,8,8,11,11,13,13],"right_children":[2,4,-1,-1,6,8,-1,10,12,-1,-1,14,-1,16,-1,-1,-1],"split_conditions":[1.3866357E8,7E0,-7.135269E-3,-5.879663E-3,3.029697E3,1.7329262E7,3.19095E-3,8.47E2,5.0175633E0,-3.5294245E-3,3.557072E-3,6.9E1,1.6566316E-3,2.045738E0,-3.385792E-3,-3.899823E-3,-5.204307E-5],"split_indices":[47,3,0,0,4,44,0,2,53,0,0,8,0,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E1,8.3E1,4E0,4E0,7.9E1,6.3E1,1.6E1,1.9E1,4.4E1,4E0,1.5E1,3.7E1,7E0,2.7E1,1E1,4E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.3860375E-3,-4.46792E-3,8.615266E-3,1.099775E-2,-1.1051276E-1,-3.062494E-2,2.1599693E-2,-8.0041E-3,-5.024926E-4,-3.2655252E-3,8.333703E-4,2.990377E-3,3.322378E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9781733E-1,1.1601921E-1,0E0,2.8020937E-2,6.594029E-2,3.128498E-2,3.2877292E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0230347E3,1.5092398E8,8.615266E-3,3.7997437E2,9.1686124E-1,1.5039519E5,9.693049E2,-8.0041E-3,-5.024926E-4,-3.2655252E-3,8.333703E-4,2.990377E-3,3.322378E-4],"split_indices":[51,44,0,51,26,32,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.9E1,4E0,6.1E1,8E0,1.2E1,4.9E1,4E0,4E0,6E0,6E0,1E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-6.192797E-3,7.832392E-3,-1.456706E-1,-1.7304857E-3,2.0539446E-2,-1.2593521E-4,-1.2301277E-2,5.294579E-3,9.689084E-3,6.0695782E-2,-7.5284606E-3,6.346825E-4,4.7458704E-3,-2.6058521E-2,2.1714324E-2,-3.308736E-3,-3.6548026E-4,3.087236E-3,-4.0802493E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,13,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[1.7133056E-1,4.993025E-2,1.6127712E-1,0E0,7.051483E-2,0E0,0E0,0E0,5.2271336E-2,3.293893E-2,2.4950016E-2,0E0,0E0,2.450012E-2,2.9088743E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9,10,10,13,13,14,14],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,14,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[1.5414109E8,1.528E3,1.0232127E12,-1.7304857E-3,7.446808E-2,-1.2593521E-4,-1.2301277E-2,5.294579E-3,1.7329262E7,4.189857E8,1E0,6.346825E-4,4.7458704E-3,1.1017415E1,5.81609E-1,-3.308736E-3,-3.6548026E-4,3.087236E-3,-4.0802493E-4],"split_indices":[44,11,30,0,56,0,0,0,44,7,15,0,0,55,26,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,1.6E1,6.3E1,4E0,3E0,5E0,5.8E1,1.4E1,4.4E1,8E0,6E0,2.7E1,1.7E1,6E0,2.1E1,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.83628E-3,-5.039373E-2,1.7211659E-2,-1.6240288E-2,-5.71947E-3,2.496904E-2,-3.4287497E-3,-1.5405546E-3,3.286436E-3,1.2113629E-2,4.829595E-3,3.5186373E-3,2.8071045E-3,-9.174293E-4,1.3813911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,13,-1,-1],"loss_changes":[8.256978E-2,7.753863E-2,4.0532652E-2,4.1446827E-2,0E0,5.2913994E-2,0E0,0E0,0E0,2.7848965E-2,0E0,0E0,2.8601397E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,12,12],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,14,-1,-1],"split_conditions":[6.6504064E0,1.0789844E6,1.8199778E0,6.817E3,-5.71947E-3,1.1997641E0,-3.4287497E-3,-1.5405546E-3,3.286436E-3,1.7329262E7,4.829595E-3,3.5186373E-3,1.381E3,-9.174293E-4,1.3813911E-3],"split_indices":[57,27,38,2,0,38,0,0,0,44,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.7E1,5E1,2E1,7E0,4.7E1,3E0,1.7E1,3E0,4.2E1,5E0,4E0,3.8E1,2.1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-9.158938E-3,-3.5582323E-2,3.7559703E-2,-2.010544E-2,-6.2207812E-3,2.1761567E-3,-1.3214346E-3,-4.500451E-3,-1.081828E-2,3.6963965E-3,-2.3378816E-3,2.4388537E-2,-1.2092583E-3,-3.7122457E-4,2.3701799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,-1,13,-1,-1,-1],"loss_changes":[1.1243827E-1,9.571718E-2,3.168499E-2,3.9976224E-2,0E0,0E0,0E0,0E0,3.0754011E-2,2.5169533E-2,0E0,2.408788E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9,11,11],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,-1,14,-1,-1,-1],"split_conditions":[1.204781E3,9.8788344E5,2.3297022E7,3.235021E2,-6.2207812E-3,2.1761567E-3,-1.3214346E-3,-4.500451E-3,6.237206E5,1.3793921E3,-2.3378816E-3,2.912E3,-1.2092583E-3,-3.7122457E-4,2.3701799E-3],"split_indices":[51,27,31,4,0,0,0,0,27,4,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,5.7E1,3.2E1,5.1E1,6E0,2.7E1,5E0,4E0,4.7E1,3.6E1,1.1E1,2.2E1,1.4E1,1.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.0962158E-2,9.0959285E-5,9.184452E-3,1.080748E-2,-8.360569E-3,-3.0612098E-3,9.848634E-2,-2.9778656E-2,2.138885E-2,8.029899E-3,1.4280527E-3,-3.3120395E-4,-6.106778E-3,2.3738118E-3,-2.2544239E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7501944E-1,1.6141982E-1,0E0,8.9068815E-2,0E0,4.244449E-2,5.184953E-2,7.609519E-2,3.122276E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3650324E7,5.139E4,9.184452E-3,4.9891987E3,-8.360569E-3,7.2767865E2,7.33E2,1.0789844E6,1.9188015E9,8.029899E-3,1.4280527E-3,-3.3120395E-4,-6.106778E-3,2.3738118E-3,-2.2544239E-4],"split_indices":[1,2,0,4,0,51,0,27,12,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.5E1,3E0,7.2E1,3E0,6.3E1,9E0,3E1,3.3E1,3E0,6E0,2.6E1,4E0,1.4E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.4084175E-2,2.4047477E-2,-8.06955E-3,9.160869E-3,1.3025806E-2,-2.2201473E-2,1.2218094E-3,1.5059721E-2,-3.0826356E-3,4.5753196E-3,-8.0603943E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,-1,5,-1,7,-1,9,-1,-1,-1],"loss_changes":[1.722334E-1,3.3133096E-1,0E0,4.7509026E-2,0E0,5.684841E-2,0E0,5.8127902E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7],"right_children":[2,4,-1,6,-1,8,-1,10,-1,-1,-1],"split_conditions":[1.3866357E8,7.467988E3,-8.06955E-3,3.826E3,1.3025806E-2,1.4E1,1.2218094E-3,1.5414852E5,-3.0826356E-3,4.5753196E-3,-8.0603943E-4],"split_indices":[47,4,0,2,0,3,0,31,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.9E1,3E0,7.6E1,3E0,2.9E1,4.7E1,1.7E1,1.2E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.8934305E-3,-1.2732357E-2,1.1318758E-2,-5.003322E-4,-9.431507E-3,-4.851714E-2,1.7398095E-2,-4.1011097E-3,1.5908298E-4,-3.52695E-3,2.0294997E-3,1.7986002E-3,-1.4767022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,-1,-1,-1],"loss_changes":[2.9163423E-1,1.8291733E-1,0E0,6.014445E-2,0E0,4.738724E-2,3.2247663E-2,0E0,0E0,4.5268156E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,-1,-1,-1],"split_conditions":[9.55721E3,3.698723E10,1.1318758E-2,3.2716873E0,-9.431507E-3,4.283039E1,1.1773262E3,-4.1011097E-3,1.5908298E-4,1.0197425E1,2.0294997E-3,1.7986002E-3,-1.4767022E-3],"split_indices":[4,5,0,55,0,57,51,0,0,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,7.1E1,3E0,6.8E1,3E0,1.8E1,5E1,9E0,9E0,3E1,2E1,1.2E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1080338E-2,-5.3662215E-3,1.4537941E-1,1.5275009E-2,-3.3924855E-2,1.3167339E-2,-5.868377E-4,-1.960574E-3,1.1878536E-3,-3.9606597E-3,-5.337057E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6201943E-1,3.9424237E-2,2.1110065E-1,3.1076372E-2,5.358601E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4],"right_children":[2,4,6,8,10,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7780703E3,6.401022E7,9.192915E0,1.018E3,7.553E3,1.3167339E-2,-5.868377E-4,-1.960574E-3,1.1878536E-3,-3.9606597E-3,-5.337057E-5],"split_indices":[4,44,56,11,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.5E1,7E0,3.8E1,2.7E1,3E0,4E0,6E0,3.2E1,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-6.9997166E-3,4.0016603E-3,-1.12870276E-1,-1.5674913E-2,5.9529636E-2,6.331049E-4,-9.07983E-3,5.378683E-3,-2.8464128E-3,6.647169E-3,8.990292E-4,-4.099305E-3,7.325761E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[8.6641096E-2,7.522994E-2,9.203134E-2,5.499731E-2,6.320955E-2,0E0,0E0,4.5664474E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[3.698723E10,1.1879E4,1.2845453E5,6.8989144E7,1.383E4,6.331049E-4,-9.07983E-3,4.8595375E6,-2.8464128E-3,6.647169E-3,8.990292E-4,-4.099305E-3,7.325761E-4],"split_indices":[5,2,32,44,10,0,0,44,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.7E1,6E0,5E1,1.7E1,3E0,3E0,3.6E1,1.4E1,4E0,1.3E1,3E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.0791813E-2,-1.8801097E-3,-6.238033E-3,1.1297966E-2,-1.02703124E-1,1.8365833E-3,5.646304E-3,-6.797323E-3,-1.1081329E-3,-1.8478564E-3,1.7427966E-2,3.1305603E-3,2.73192E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[9.72343E-2,1.0346665E-1,0E0,7.857981E-2,3.4755453E-2,4.6406426E-2,0E0,0E0,0E0,0E0,2.9705036E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,10,10],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[1.3866357E8,1.3237324E6,-6.238033E-3,5.639591E3,1.4235585E3,6E0,5.646304E-3,-6.797323E-3,-1.1081329E-3,-1.8478564E-3,1.7329262E7,3.1305603E-3,2.73192E-4],"split_indices":[47,27,0,4,4,8,0,0,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.6E1,4E0,6.8E1,8E0,6.4E1,4E0,4E0,4E0,1.6E1,4.8E1,7E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.8591696E-3,9.815839E-3,-3.6795665E-2,-3.6681313E-2,2.1475587E-2,9.056731E-4,-3.727448E-3,3.821858E-4,-3.2953229E-3,3.7346368E-3,4.598712E-4,-1.3272683E-3,3.4440418E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.421554E-2,3.4942888E-2,3.9957553E-2,2.4634743E-2,3.5072025E-2,3.556776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.533027E-1,7.095772E2,4.25174E0,1.2817016E7,4.07E2,3.1023192E0,-3.727448E-3,3.821858E-4,-3.2953229E-3,3.7346368E-3,4.598712E-4,-1.3272683E-3,3.4440418E-3],"split_indices":[26,4,55,12,0,55,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,6.2E1,2E1,1.2E1,5E1,1.2E1,8E0,6E0,6E0,6E0,4.4E1,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.418808E-3,1.1304898E-2,-1.4341435E-1,4.081586E-3,3.8006776E-3,-1.2294907E-3,-1.0728865E-2,2.30106E-3,-4.7141784E-3,-1.7767029E-2,1.9549048E-3,-3.3403442E-3,-1.2859463E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[1.7209224E-1,3.942059E-2,9.137453E-2,2.9121613E-2,0E0,0E0,0E0,0E0,3.8331848E-2,4.073698E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.5092398E8,3.4409692E3,1.2341693E10,5.7578983E10,3.8006776E-3,-1.2294907E-3,-1.0728865E-2,2.30106E-3,5.0175633E0,3.068E3,1.9549048E-3,-3.3403442E-3,-1.2859463E-4],"split_indices":[44,51,5,30,0,0,0,0,53,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7E1,7E0,6.5E1,5E0,4E0,3E0,9E0,5.6E1,4.5E1,1.1E1,8E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.9704896E-4,9.198559E-3,-6.9760242E-3,-4.2951675E-3,1.4474942E-1,-3.2552388E-2,1.972492E-2,2.8636443E-5,1.0803786E-2,-2.906538E-4,-9.1110535E-2,3.215161E-3,1.2774585E-4,-6.389038E-3,-1.1694393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1],"loss_changes":[1.1292363E-1,1.4146137E-1,0E0,4.88327E-2,1.086466E-1,4.9151298E-2,3.597805E-2,0E0,0E0,0E0,3.3084884E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2415501E9,2.6949062E3,-6.9760242E-3,8.6646006E5,1.4099E4,1.0112447E6,2.156825E7,2.8636443E-5,1.0803786E-2,-2.906538E-4,1.9598669E6,3.215161E-3,1.2774585E-4,-6.389038E-3,-1.1694393E-3],"split_indices":[7,51,0,46,9,31,44,0,0,0,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.6E1,3E0,7E1,6E0,3.2E1,3.8E1,3E0,3E0,2.3E1,9E0,8E0,3E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.299149E-3,-6.6466117E-3,1.030528E-2,1.0409964E-3,-5.9662475E-3,-2.8077487E-3,1.1958189E-2,-2.7244217E-3,1.8385906E-2,1.0324459E-3,-2.7340848E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[2.3250343E-1,7.4712105E-2,0E0,5.1756743E-2,0E0,0E0,3.0839918E-2,0E0,2.855911E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[8.791079E3,5.673536E2,1.030528E-2,2.1113522E8,-5.9662475E-3,-2.8077487E-3,7E0,-2.7244217E-3,4.2E1,1.0324459E-3,-2.7340848E-3],"split_indices":[4,57,0,7,0,0,3,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,6.9E1,3E0,9E0,6E1,4E0,5.6E1,5.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.937586E-2,9.897601E-3,8.018543E-3,-1.009328E-2,4.4329964E-2,-4.3930426E-2,1.2423929E-2,5.9737567E-2,-2.531505E-3,1.4012291E-4,-3.0678778E-3,4.245226E-3,-1.4152724E-3,4.926671E-3,1.4901694E-3,-1.4873481E-3,1.0849742E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,-1,-1,15,-1,-1,-1,-1],"loss_changes":[1.1925925E-1,5.0695766E-2,0E0,3.6355942E-2,4.6604592E-2,2.5179353E-2,3.499613E-2,2.9121995E-2,0E0,0E0,0E0,0E0,2.4120333E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,-1,-1,16,-1,-1,-1,-1],"split_conditions":[2.1769184E7,1.077646E3,8.018543E-3,2.8586518E8,2.2167318E-1,1.0793079E5,2.526E3,7.33E2,-2.531505E-3,1.4012291E-4,-3.0678778E-3,4.245226E-3,3.3392856E0,4.926671E-3,1.4901694E-3,-1.4873481E-3,1.0849742E-3],"split_indices":[1,51,0,7,37,31,2,0,0,0,0,0,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,7.2E1,3E0,4.6E1,2.6E1,1.8E1,2.8E1,2.3E1,3E0,7E0,1.1E1,3E0,2.5E1,6E0,1.7E1,1.1E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[5.880934E-4,9.789349E-3,-7.6539125E-3,-1.275904E-2,4.629207E-2,7.9654885E-4,-5.6536244E-3,8.215862E-2,9.067132E-4,-4.1097035E-3,1.0006172E-2,5.5095456E-3,1.3004519E-3,4.3614373E-2,-1.0981136E-3,8.3502494E-2,6.353207E-4,3.145271E-4,5.1465724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,13,-1,-1,15,-1,17,-1,-1,-1],"loss_changes":[1.3558705E-1,6.479069E-2,0E0,7.9445854E-2,2.5253631E-2,4.112026E-2,0E0,2.5069006E-2,0E0,0E0,5.149105E-2,0E0,0E0,2.3995396E-2,0E0,2.436763E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,10,10,13,13,15,15],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,14,-1,-1,16,-1,18,-1,-1,-1],"split_conditions":[5.139E4,1.077646E3,-7.6539125E-3,1.0789844E6,1.4E1,8.192308E1,-5.6536244E-3,7.33E2,9.067132E-4,-4.1097035E-3,6.1577463E0,5.5095456E-3,1.3004519E-3,3.5175372E5,-1.0981136E-3,2.37E2,6.353207E-4,3.145271E-4,5.1465724E-3],"split_indices":[2,51,0,27,3,51,0,0,0,0,55,0,0,31,0,10,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,4.8E1,2.9E1,4.4E1,4E0,1.1E1,1.8E1,3E0,4.1E1,5E0,6E0,2.1E1,2E1,8E0,1.3E1,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.808404E-3,-4.0421966E-3,8.400543E-3,2.2803247E-3,-5.1991097E-3,-1.4997546E-3,1.8446412E-2,2.4496089E-3,4.447638E-6],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[1.5290117E-1,5.8241256E-2,0E0,4.593393E-2,0E0,0E0,3.7748966E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[7.6312915E3,2.5805172E7,8.400543E-3,2.881172E8,-5.1991097E-3,-1.4997546E-3,4.5503766E5,2.4496089E-3,4.447638E-6],"split_indices":[51,46,0,7,0,0,31,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.7E1,3E0,7.4E1,3E0,2.2E1,5.2E1,1.6E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.4626985E-2,-6.5052616E-3,-5.2099507E-3,-3.3596627E-2,6.805525E-4,-4.6231814E-2,1.6844646E-3,-1.09878994E-4,-6.898021E-2,-3.9237575E-3,-4.4140316E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[7.4087046E-2,4.973244E-2,0E0,3.5942703E-2,0E0,3.1606704E-2,0E0,0E0,2.7107805E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[1.278211E8,8.6646006E5,-5.2099507E-3,2.477407E6,6.805525E-4,1.5326E4,1.6844646E-3,-1.09878994E-4,1.9598669E6,-3.9237575E-3,-4.4140316E-4],"split_indices":[47,46,0,1,0,9,0,0,31,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8E1,5E0,3.6E1,4.4E1,3.1E1,5E0,1.1E1,2E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.0076585E-2,6.9905276E-4,7.84219E-3,1.0549464E-2,-7.270946E-2,-3.8446547E-3,1.6663069E-2,-6.822399E-4,-5.97255E-3,9.212863E-3,4.3931943E-3,-2.595745E-3,2.2281876E-3,-1.6221173E-2,2.1725013E-3,-1.4562735E-3,1.3798906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,-1,9,-1,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.2663867E-1,5.4999854E-2,0E0,4.2645644E-2,3.197294E-2,0E0,4.009278E-2,0E0,0E0,3.0525176E-2,0E0,3.5747956E-2,0E0,3.4822833E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,9,9,11,11,13,13],"right_children":[2,4,-1,6,8,-1,10,-1,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[2.1769184E7,9.167896E-1,7.84219E-3,3.235021E2,6.33068E5,-3.8446547E-3,1.1997641E0,-6.822399E-4,-5.97255E-3,9.29295E5,4.3931943E-3,5.0175633E0,2.2281876E-3,3.029697E3,2.1725013E-3,-1.4562735E-3,1.3798906E-3],"split_indices":[1,26,0,4,28,0,38,0,0,27,0,53,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.4E1,3E0,6.6E1,8E0,3E0,6.3E1,5E0,3E0,5.9E1,4E0,4.7E1,1.2E1,3.8E1,9E0,2.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[7.1125855E-3,-5.903616E-3,1.3625851E-1,6.023849E-3,-5.7360013E-3,2.0349465E-4,9.993506E-3,-1.0403292E-3,2.3224136E-2,-2.6063612E-3,1.336854E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,-1,-1,-1,-1,9,-1,-1],"loss_changes":[1.2855962E-1,1.07870355E-1,8.857702E-2,3.4498528E-2,0E0,0E0,0E0,0E0,3.0320372E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8],"right_children":[2,4,6,8,-1,-1,-1,-1,10,-1,-1],"split_conditions":[4.0230347E3,3.698723E10,2.2620792E5,3.826E3,-5.7360013E-3,2.0349465E-4,9.993506E-3,-1.0403292E-3,5E0,-2.6063612E-3,1.336854E-3],"split_indices":[51,5,27,2,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,6.4E1,5E0,3E0,3E0,2.3E1,4.1E1,3E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[6.4143366E-3,-1.3931212E-3,7.4674455E-3,-5.4263496E-3,4.5996397E-3,4.8249832E-4,-5.2348804E-2,-4.3446505E-3,1.4801146E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[1.18438624E-1,6.601547E-2,0E0,0E0,3.25617E-2,0E0,4.206098E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[2.1698572E7,5E0,7.4674455E-3,-5.4263496E-3,1.8003757E-1,4.8249832E-4,1.4E1,-4.3446505E-3,1.4801146E-3],"split_indices":[1,3,0,0,37,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,8.6E1,3E0,3E0,8.3E1,7.5E1,8E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.14453E-3,1.5539227E-2,-6.0570575E-3,2.9807705E-3,1.0567889E-2,-3.317125E-3,8.611344E-3,1.3842303E-2,-2.7028853E-3,3.1798948E-2,-4.6114207E-4,-1.2050027E-2,2.02419E-3,-1.8756003E-3,2.3341302E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,-1,-1,7,9,-1,11,-1,13,-1,-1,-1],"loss_changes":[1.1605032E-1,2.2611591E-1,0E0,3.4683798E-2,0E0,0E0,2.7547667E-2,2.9970221E-2,0E0,2.71384E-2,0E0,2.5709478E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,9,9,11,11],"right_children":[2,4,-1,6,-1,-1,8,10,-1,12,-1,14,-1,-1,-1],"split_conditions":[1.278211E8,7.467988E3,-6.0570575E-3,1.09E2,1.0567889E-2,-3.317125E-3,3.698723E10,4.5090426E8,-2.7028853E-3,1.648E3,-4.6114207E-4,1.2293572E6,2.02419E-3,-1.8756003E-3,2.3341302E-3],"split_indices":[47,4,0,10,0,0,5,7,0,11,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,7.7E1,4E0,7.4E1,3E0,4E0,7E1,6.6E1,4E0,3.8E1,2.8E1,1E1,2.8E1,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.0882834E-2,-1.1136126E-4,-1.1858166E-1,7.435513E-4,-2.6054291E-2,7.699214E-4,-9.644061E-3,-2.0954113E-3,6.8199955E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,-1,7,-1,-1,-1,-1],"loss_changes":[8.931257E-2,3.2685246E-2,1.0630635E-1,0E0,2.873138E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4],"right_children":[2,4,6,-1,8,-1,-1,-1,-1],"split_conditions":[1.5414109E8,7.455661E-1,1.6867E4,7.435513E-4,1.419E3,7.699214E-4,-9.644061E-3,-2.0954113E-3,6.8199955E-4],"split_indices":[44,26,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,4.2E1,2.8E1,3E0,3E0,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.1758199E-2,-8.113677E-3,8.1760526E-2,6.226069E-5,-5.188308E-3,1.22552544E-1,-2.0872424E-3,2.5562535E-3,1.1312912E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[1.13340646E-1,6.951435E-2,1.0185341E-1,0E0,0E0,1.0897979E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.3305548E3,1.5092398E8,1.06102504E8,6.226069E-5,-5.188308E-3,3.9509753E3,-2.0872424E-3,2.5562535E-3,1.1312912E-2],"split_indices":[51,44,50,0,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.3E1,1.7E1,5.9E1,4E0,1.3E1,4E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[8.700694E-3,-1.2264316E-3,7.6019564E-3,8.359954E-3,-5.371242E-3,5.6372937E-2,-3.6212875E-4,8.92916E-4,4.451379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,-1,5,-1,7,-1,-1,-1],"loss_changes":[1.21059716E-1,8.3321005E-2,0E0,5.2995488E-2,0E0,2.661978E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5],"right_children":[2,4,-1,6,-1,8,-1,-1,-1],"split_conditions":[2.1747866E7,1.5277152E8,7.6019564E-3,1.7329262E7,-5.371242E-3,2.9719496E0,-3.6212875E-4,8.92916E-4,4.451379E-3],"split_indices":[1,44,0,44,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.8E1,3E0,6.4E1,4E0,1.6E1,4.8E1,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[5.437385E-3,-1.752151E-3,6.630797E-3,-1.9731363E-2,2.8080674E-2,-7.1678157E-3,-4.330542E-3,4.5535218E-2,-1.7431164E-3,1.08149936E-4,-3.1581158E-3,9.993475E-4,4.377305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":99,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,-1,-1,-1],"loss_changes":[9.3857996E-2,4.5609206E-2,0E0,5.3566657E-2,4.0307872E-2,3.0577855E-2,0E0,3.035758E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,-1,-1,-1],"split_conditions":[2.1769184E7,1.204781E3,6.630797E-3,1.3237324E6,1.06102504E8,1E0,-4.330542E-3,4.146482E3,-1.7431164E-3,1.08149936E-4,-3.1581158E-3,9.993475E-4,4.377305E-3],"split_indices":[1,51,0,27,50,112,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,8.3E1,3E0,5.2E1,3.1E1,4.6E1,6E0,2.5E1,6E0,4.1E1,5E0,1.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics index 02fd5b915..4d97da9d7 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics @@ -37,73 +37,80 @@ 35,sqlOp_Subquery,0.00058086135,0.2818181818181818,0.45091116645433477,0.0,0.0,0.0,1.0,1.0 36,sr_totalBytesRead_mean,0.00057853124,17488097.096494444,36315663.17124838,58.75728155339806,345292.2404184723,2879392.1611024104,15671497.58876525,243101856.34140846 37,executorRunTime_mean,0.00026640674,1820.9901176973162,2822.0779471701867,41.0,433.6194879832811,944.1120382732532,2108.383181217503,23591.60349127182 -38,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -39,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -40,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -41,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -42,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -46,sqlOp_HashAggregate,0.0,0.990909090909091,0.09512832433347063,0.0,1.0,1.0,1.0,1.0 +38,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +39,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +40,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +41,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +42,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +43,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +44,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 47,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 48,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 49,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 50,sqlOp_Filter,0.0,0.4727272727272727,0.5003941991110044,0.0,0.0,0.0,1.0,1.0 51,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_SortMergeJoin,0.0,0.5590909090909091,0.49762824776768727,0.0,0.0,1.0,1.0,1.0 -60,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_BroadcastHashJoin,0.0,0.9045454545454545,0.2945118594681246,0.0,1.0,1.0,1.0,1.0 -72,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -74,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -75,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,diskBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 -78,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -79,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -81,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -82,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -84,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,diskBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -88,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -89,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -91,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,sqlOp_BroadcastNestedLoopJoin,0.0,0.03636363636363636,0.18762017042954973,0.0,0.0,0.0,0.0,1.0 -95,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -96,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,sqlOp_HashAggregate,0.0,0.990909090909091,0.09512832433347063,0.0,1.0,1.0,1.0,1.0 +55,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_SortMergeJoin,0.0,0.5590909090909091,0.49762824776768727,0.0,0.0,1.0,1.0,1.0 +62,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 +78,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +80,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,diskBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +83,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +84,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +89,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,diskBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +92,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +93,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +95,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +96,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 97,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sqlOp_AQEShuffleRead,0.0,0.9090909090909091,0.28813538628606516,0.0,1.0,1.0,1.0,1.0 -99,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_Scan parquet ,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -101,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -106,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +101,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_BroadcastNestedLoopJoin,0.0,0.03636363636363636,0.18762017042954973,0.0,0.0,0.0,0.0,1.0 +103,sqlOp_BroadcastHashJoin,0.0,0.9045454545454545,0.2945118594681246,0.0,1.0,1.0,1.0,1.0 +104,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_AQEShuffleRead,0.0,0.9090909090909091,0.28813538628606516,0.0,1.0,1.0,1.0,1.0 +106,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +107,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg index 4eaa3245f..c62483032 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0499760844","gamma":"0.037331894","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0499760844","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"1","min_split_loss":"0.037331894","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.735622287"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"91"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0482159182","gamma":"0.0364067033","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0482159182","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"8","max_leaves":"0","min_child_weight":"1","min_split_loss":"0.0364067033","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.723129928"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json index 4077c5635..7af4f010b 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan csv ","sqlOp_Scan orc ","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortAggregate","sqlOp_SortMergeJoin","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_dataproc","platform_databricks-aws","platform_databricks-azure","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan json ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[1.4857235E-3,-2.965608E-1,5.363805E-1,-6.081042E-1,-1.9329263E-1,6.194652E-1,-5.773755E-1,-6.254226E-1,-1.5078232E-3,-2.2354639E-1,3.1191745E-1,3.4459683E-1,9.4478184E-1,-9.608562E-1,-9.766551E-2,-3.2782823E-2,-1.8114489E-2,-3.2691827E-1,-1.3094649E-1,1.8283451E-2,1.4088118E-3,2.9292017E-1,5.199973E-2,7.9174286E-1,1.3544148E0,-5.6336094E-2,-2.0924874E-2,-1.0740333E-2,4.5585646E-3,-3.7175104E-1,-7.4982226E-2,-2.3960607E-2,-2.4165808E-1,3.1552166E-2,3.5883486E-1,8.375097E-1,-9.4547E-3,5.445212E-1,7.4935496E-2,-2.2782652E-1,-4.2883322E-1,4.2892373E-3,-1.2233638E-1,8.3871454E-2,-1.7294605E-1,-1.07227445E-1,-3.0571964E-1,1.5722747E-1,-2.76379E-1,4.087299E-1,1.69263E-1,8.51175E-1,5.2201133E-3,4.017866E-2,-5.8419164E-3,-2.6189747E-1,-2.2977677E-3,-1.0398948E-2,-4.520926E-1,-7.7472785E-3,1.3686883E-3,-7.307728E-2,1.8169455E-1,-1.2570597E-3,-2.5009203E-1,-8.530849E-3,-2.478975E-2,-3.241502E-1,-3.6085164E-4,2.5900844E-1,-5.5516716E-3,-1.648653E-2,-1.5577908E-3,4.6767187E-1,2.689842E-1,-1.61118E-3,1.2648332E-2,1.4754654E-2,4.3465957E-2,-1.4071626E-2,-2.8276553E-3,-4.6481928E-1,-9.79002E-3,9.4813167E-4,-1.5468666E-1,2.328049E-1,-9.094357E-4,-8.83379E-4,-2.8448862E-1,-5.1299143E-3,1.5616007E-3,-3.3635673E-1,1.355652E-3,1.464775E-2,-3.4198043E-4,3.648782E-1,2.7636437E-2,-1.1774825E-3,1.561549E-2,-2.371319E-2,-8.558666E-3,-1.0140763E-2,3.3650717E-3,1.4689181E-2,6.7440243E-3,-1.5088612E-2,-1.8829007E-3,-1.7929971E-2,-7.3988643E-3,1.9438792E-2,6.1387527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,-1,35,37,-1,-1,-1,-1,39,41,43,45,47,49,51,-1,53,-1,55,57,-1,59,61,63,65,67,69,71,73,75,77,-1,-1,-1,79,-1,-1,81,-1,-1,83,85,-1,87,-1,89,91,-1,93,-1,-1,-1,95,97,-1,-1,-1,-1,-1,-1,99,-1,-1,101,103,-1,-1,105,-1,-1,107,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.7276375E1,8.601212E0,1.4330605E1,6.5172577E-1,3.2030501E0,1.2282547E1,1.9297128E0,2.843647E-1,0E0,1.8142033E0,1.7329252E-1,2.5002213E0,2.9397964E0,6.932497E-2,1.5247346E-1,0E0,0E0,9.9997044E-1,1.2260457E0,0E0,0E0,1.2635498E0,0E0,2.3582478E0,1.062643E0,0E0,0E0,0E0,0E0,5.3489494E-1,1.24723546E-1,8.820373E-1,4.0642214E-1,6.626438E-1,4.8887968E-1,3.8191223E-1,0E0,7.803595E-1,0E0,1.3665736E-1,1.8319893E-1,0E0,6.296696E-2,5.045805E-1,2.5594664E-1,8.995195E-2,1.847105E-1,3.5648343E-1,5.5321872E-2,2.5310087E-1,2.4463114E-1,2.178688E-1,0E0,0E0,0E0,6.483948E-2,0E0,0E0,6.545162E-2,0E0,0E0,1.0059145E-1,2.0856738E-1,0E0,1.14886224E-1,0E0,4.2473387E-2,1.463356E-1,0E0,8.3563745E-2,0E0,0E0,0E0,7.011032E-2,2.1085656E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6504974E-2,0E0,0E0,8.861235E-2,5.6034446E-2,0E0,0E0,4.4543862E-2,0E0,0E0,7.772994E-2,0E0,0E0,0E0,5.132079E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,55,55,58,58,61,61,62,62,64,64,66,66,67,67,69,69,73,73,74,74,81,81,84,84,85,85,88,88,91,91,95,95],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,-1,36,38,-1,-1,-1,-1,40,42,44,46,48,50,52,-1,54,-1,56,58,-1,60,62,64,66,68,70,72,74,76,78,-1,-1,-1,80,-1,-1,82,-1,-1,84,86,-1,88,-1,90,92,-1,94,-1,-1,-1,96,98,-1,-1,-1,-1,-1,-1,100,-1,-1,102,104,-1,-1,106,-1,-1,108,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,1.0429407E3,1.2730462E7,1.2678202E9,6.0891E4,1.700398E4,2.9778808E7,-1.5078232E-3,7.377246E2,1.5E2,7.1853364E3,8.4987135E0,3.1E1,1.6130411E5,-3.2782823E-2,-1.8114489E-2,7.164074E8,2.1692945E1,1.8283451E-2,1.4088118E-3,2.3890688E3,5.199973E-2,1.4656219E8,6.069097E4,-5.6336094E-2,-2.0924874E-2,-1.0740333E-2,4.5585646E-3,7.37365E6,4.45E3,1.6777562E7,8.237054E6,1.3508742E7,2.47E2,7.2125336E2,-9.4547E-3,1.207E3,7.4935496E-2,6.1614815E6,5.1547173E1,4.2892373E-3,6.46E2,2.753337E6,2.58E2,1.849533E6,5.9E2,2.9743198E1,1.2587E4,6.0796064E8,1.3428E4,3.0811954E-2,5.2201133E-3,4.017866E-2,-5.8419164E-3,3.01E2,-2.2977677E-3,-1.0398948E-2,2.779416E8,-7.7472785E-3,1.3686883E-3,6.59701E5,7.753E3,-1.2570597E-3,2.4401142E9,-8.530849E-3,3.4458816E8,3.15584E6,-3.6085164E-4,1.14E2,-5.5516716E-3,-1.648653E-2,-1.5577908E-3,2.0663544E9,6.168645E8,-1.61118E-3,1.2648332E-2,1.4754654E-2,4.3465957E-2,-1.4071626E-2,-2.8276553E-3,7.177E3,-9.79002E-3,9.4813167E-4,1.852E3,9.663487E0,-9.094357E-4,-8.83379E-4,9E1,-5.1299143E-3,1.5616007E-3,9.072834E8,1.355652E-3,1.464775E-2,-3.4198043E-4,1.9412996E9,2.7636437E-2,-1.1774825E-3,1.561549E-2,-2.371319E-2,-8.558666E-3,-1.0140763E-2,3.3650717E-3,1.4689181E-2,6.7440243E-3,-1.5088612E-2,-1.8829007E-3,-1.7929971E-2,-7.3988643E-3,1.9438792E-2,6.1387527E-3],"split_indices":[2,1,70,9,7,2,45,12,0,64,3,4,65,3,40,0,0,7,65,0,0,4,0,1,40,0,0,0,0,57,2,57,57,57,3,70,0,0,0,57,60,0,8,1,3,1,8,65,2,44,2,50,0,0,0,8,0,0,12,0,0,41,2,0,5,0,7,1,0,3,0,0,0,12,44,0,0,0,0,0,0,2,0,0,0,65,0,0,8,0,0,7,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,2.7E2,1.5E2,6.6E1,2.04E2,1.4E2,1E1,6.4E1,2E0,1.93E2,1.1E1,7.7E1,6.3E1,5E0,5E0,5.6E1,8E0,9E1,1.03E2,9E0,2E0,7.3E1,4E0,4.8E1,1.5E1,3E0,2E0,3E0,2E0,7.6E1,1.4E1,5.3E1,5E1,1.5E1,5.8E1,4.6E1,2E0,3E0,1.2E1,2.3E1,5.3E1,3E0,1.1E1,3.1E1,2.2E1,1.7E1,3.3E1,1.1E1,4E0,4.5E1,1.3E1,4.5E1,1E0,2E0,1E0,1.9E1,4E0,6E0,4.7E1,9E0,2E0,1.2E1,1.9E1,8E0,1.4E1,9E0,8E0,3.1E1,2E0,8E0,3E0,3E0,1E0,3E1,1.5E1,4E0,9E0,2E0,4.3E1,1.7E1,2E0,4.4E1,3E0,6E0,6E0,1.5E1,4E0,2E0,1.2E1,3E0,5E0,3E1,1E0,7E0,1E0,1.6E1,1.4E1,2E0,1.3E1,4.2E1,2E0,5E0,1E0,8E0,7E0,1.1E1,1E0,2.6E1,4E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[1.6865663E-2,-2.8437036E-1,5.739436E-1,-5.9718937E-1,-1.8265463E-1,4.3719468E-1,1.0980139E0,-6.337835E-1,-3.1790537E-1,-2.1990728E-1,3.2870778E-1,4.833704E-1,-5.284346E-2,-2.8204769E-2,1.2196875E0,-3.2210924E-2,-1.0735854E-2,-1.8561725E-2,-3.4609516E-3,-2.7191135E-1,3.081771E-3,3.9041665E-1,-2.4121741E-3,3.2801256E-1,6.610112E-1,-9.139693E-3,1.0890412E-2,6.019764E-1,1.3969256E0,-3.0097374E-1,-9.944806E-2,-1.1908863E-1,1.2509808E-1,2.7008452E-2,1.1449842E-2,3.710176E-1,5.0082535E-2,2.0846318E-2,7.108115E-1,3.6009606E-2,-5.6959386E-3,7.134287E-2,1.9606447E-2,-3.162087E-1,8.943549E-2,-1.7669444E-1,-1.039947E-3,-1.5393186E-1,2.9369553E-3,-1.32860895E-2,2.3952933E-1,-2.3339844E-4,3.847639E-1,-9.213355E-2,2.1072534E-1,-1.4842666E-2,1.6579028E-2,2.6544219E-2,3.923254E-2,-3.6207968E-1,-2.43738E-1,-6.1127082E-3,7.8086457E-3,7.921228E-4,-9.631631E-3,-5.9219208E-3,2.5626682E-3,1.4226036E-3,-8.351544E-3,-3.3814607E-3,2.729779E-3,4.7222944E-3,1.5438092E-2,2.4136802E-2,3.3209166E-1,7.6020934E-4,-7.476868E-3,-2.1661585E-4,1.3272341E-2,-2.533375E-1,-4.097456E-1,-2.9037826E-2,-3.16342E-1,3.4128216E-1,-4.2847602E-4,-4.3180785E-3,-1.4360616E-2,-2.2047773E-2,-1.3829033E-2,5.3282483E-3,-6.562854E-3,-4.335553E-3,-1.693279E-2,1.0201701E-2,1.8831342E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,33,-1,35,37,-1,-1,39,41,43,45,47,49,-1,-1,51,53,55,57,-1,-1,-1,-1,59,61,63,65,67,-1,69,71,-1,73,75,77,-1,-1,-1,-1,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,83,-1,-1,-1,-1,85,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3161415E1,8.895588E0,1.0484856E1,5.03706E-1,4.167951E0,8.99968E0,4.282364E0,1.3160515E-1,1.07325315E-1,2.3317013E0,3.6778414E-1,3.0949268E0,0E0,1.9212641E-1,2.2245827E0,0E0,0E0,0E0,0E0,7.870569E-1,5.962622E-1,1.9076216E-1,0E0,7.70277E-1,1.7386017E0,0E0,0E0,7.6118803E-1,8.5819244E-2,8.550329E-1,1.89858E-1,1.3298753E-1,3.1989187E-1,0E0,0E0,2.961626E-1,2.4787478E-1,5.9259874E-1,2.6309776E-1,0E0,0E0,0E0,0E0,3.5720348E-1,1.03995174E-1,4.6266437E-2,7.981616E-2,4.5621693E-2,0E0,4.0620986E-2,8.1501424E-2,0E0,1.2410259E-1,3.929394E-2,6.0129657E-2,0E0,0E0,0E0,0E0,3.1085968E-1,8.481598E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.25031E-1,0E0,0E0,0E0,0E0,1.2847352E-1,1.0364342E-1,2.2212505E-1,1.8217516E-1,1.0822773E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,19,19,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,43,43,44,44,45,45,46,46,47,47,49,49,50,50,52,52,53,53,54,54,59,59,60,60,74,74,79,79,80,80,81,81,82,82,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,34,-1,36,38,-1,-1,40,42,44,46,48,50,-1,-1,52,54,56,58,-1,-1,-1,-1,60,62,64,66,68,-1,70,72,-1,74,76,78,-1,-1,-1,-1,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,84,-1,-1,-1,-1,86,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.6512886E3,3.5002026E8,1.4777102E9,4.673502E7,5.0314346E1,2.9778808E7,4.48149E5,4.1619555E8,1.87E2,4.1331047E-1,-5.284346E-2,3.677E3,6.069097E4,-3.2210924E-2,-1.0735854E-2,-1.8561725E-2,-3.4609516E-3,7.10711E8,1.347372E3,6.088569E6,-2.4121741E-3,2.7246006E7,2.1853803E4,-9.139693E-3,1.0890412E-2,3.14E2,2.3322949E2,3.3616E4,2.6452028E7,1E0,7.959718E4,2.7008452E-2,1.1449842E-2,2.56E2,8.5E1,3.11E3,1E0,3.6009606E-2,-5.6959386E-3,7.134287E-2,1.9606447E-2,7.8936383E2,2.094E3,4.45E3,1E2,6.7370984E7,2.9369553E-3,1.6602323E5,8.307654E8,-2.3339844E-4,1.4040298E5,6.27427E5,1.1459E4,-1.4842666E-2,1.6579028E-2,2.6544219E-2,3.923254E-2,7.517043E6,4.0757E1,-6.1127082E-3,7.8086457E-3,7.921228E-4,-9.631631E-3,-5.9219208E-3,2.5626682E-3,1.4226036E-3,-8.351544E-3,-3.3814607E-3,2.729779E-3,4.7222944E-3,1.5438092E-2,2.4136802E-2,3.3400793E0,7.6020934E-4,-7.476868E-3,-2.1661585E-4,1.3272341E-2,7.140485E-2,6.057E3,2.58E2,6.9140625E-1,1.2592834E-1,-4.2847602E-4,-4.3180785E-3,-1.4360616E-2,-2.2047773E-2,-1.3829033E-2,5.3282483E-3,-6.562854E-3,-4.335553E-3,-1.693279E-2,1.0201701E-2,1.8831342E-2],"split_indices":[2,1,64,7,7,60,68,12,1,12,3,47,0,0,40,0,0,0,0,7,64,57,0,57,45,0,0,3,69,10,12,101,40,0,0,0,8,0,102,0,0,0,0,64,0,2,8,7,0,45,12,0,45,9,2,0,0,0,0,57,68,0,0,0,0,0,0,0,0,0,0,0,0,0,53,0,0,0,0,70,2,3,70,50,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,2.82E2,1.52E2,6.8E1,2.14E2,1.22E2,3E1,5.9E1,9E0,2E2,1.4E1,1.19E2,3E0,3E0,2.7E1,5.7E1,2E0,7E0,2E0,1.62E2,3.8E1,1.2E1,2E0,6.5E1,5.4E1,2E0,1E0,7E0,2E1,1.38E2,2.4E1,1.9E1,1.9E1,5E0,7E0,5.6E1,9E0,4E0,5E1,6E0,1E0,1.9E1,1E0,1.33E2,5E0,1.3E1,1.1E1,1.6E1,3E0,9E0,1E1,2E0,5.4E1,5E0,4E0,2E0,2E0,1.7E1,3.3E1,7.9E1,5.4E1,1E0,4E0,1E0,1.2E1,3E0,8E0,1E0,1.5E1,5E0,4E0,4E0,6E0,1.6E1,3.8E1,2E0,3E0,1E0,3E0,2.6E1,5.3E1,1.4E1,4E1,3.7E1,1E0,5E0,2.1E1,4.1E1,1.2E1,6E0,8E0,4E0,3.6E1,9E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-2.027892E-3,-2.295055E-1,5.5601895E-1,-5.4557E-1,-1.2273176E-1,6.451821E-1,-5.60624E-1,-5.726306E-1,-1.526629E-3,-1.7274493E-1,2.7080524E-1,5.3345466E-1,1.1411738E0,9.074902E-2,-1.0099975E0,-2.9976021E-2,-1.6792016E-2,-2.1824294E-1,7.790328E-2,3.311019E-1,2.2905087E-4,4.099725E-1,6.668152E-1,7.477354E-2,4.4949643E-2,-9.704379E-3,1.7263182E-2,-5.5229854E-2,-1.3352525E-2,-2.2830525E-1,3.9050918E-2,1.8723865E-1,-4.9967963E-2,3.5946295E-1,-6.6088433E-3,4.5154583E-1,9.0904115E-3,7.0917875E-1,-8.835182E-3,-9.611593E-2,-2.8868943E-1,-1.8732274E-3,2.54122E-1,-1.6863608E-1,3.0518796E-3,2.36246E-2,1.1750798E-2,3.0921918E-1,5.269149E-1,9.723287E-1,5.0747955E-1,-1.2200937E-1,1.3867086E-2,-3.3915642E-1,-1.7801484E-1,-5.604069E-4,1.4956795E-2,8.461375E-4,-9.873492E-3,1.9507347E-2,7.947918E-3,2.8286302E-2,1.4457171E-2,-5.5536083E-3,5.2104972E-2,5.3694E-1,4.638572E-3,-1.7135441E-1,-2.209195E-2,-3.5352802E-1,-2.3552878E-3,4.9724903E-2,-2.6611355E-1,1.0436168E-2,2.7806325E-2,-1.0598646E-2,-1.2809286E-3,-7.6905536E-3,1.7975719E-3,-1.8079907E-2,-5.526689E-3,6.23212E-3,-2.3006948E-3,-1.4206145E-2,-7.047072E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,49,-1,51,53,-1,55,57,-1,-1,-1,59,61,63,65,67,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,73,-1,75,77,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6109547E1,1.052972E1,1.3105816E1,1.0835152E0,4.69174E0,5.9857254E0,3.0187538E0,2.8077698E-1,0E0,2.411841E0,4.3189478E-1,1.3569202E0,4.5661354E-1,4.299045E-1,1.2871313E-1,0E0,0E0,1.968832E0,4.7072396E-1,3.3663464E-1,0E0,4.298916E-1,1.7693233E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3850918E0,0E0,2.8006697E-1,2.2111863E-1,1.8456244E-1,0E0,3.197832E-1,0E0,1.8419552E0,0E0,5.852461E-1,6.154604E-1,0E0,1.7110074E-1,4.6289936E-2,0E0,0E0,0E0,1.5720212E-1,9.630823E-2,1.4862652E0,2.800007E-1,2.6245767E-1,0E0,3.2748508E-1,8.157753E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3380737E-2,0E0,2.0818305E-1,1.5092234E-1,1.1619663E-1,0E0,9.189924E-2,1.2860465E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,29,29,31,31,32,32,33,33,35,35,37,37,39,39,40,40,42,42,43,43,47,47,48,48,49,49,50,50,51,51,53,53,54,54,65,65,67,67,68,68,69,69,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,50,-1,52,54,-1,56,58,-1,-1,-1,60,62,64,66,68,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,74,-1,76,78,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3664E4,7.41718E5,1.0429407E3,1.2852292E7,1.4777102E9,4.026531E9,2.909019E3,2.9778808E7,-1.526629E-3,5.9170656E8,1.436E3,2.1360708E7,2.231E3,2.5212732E4,3.1E1,-2.9976021E-2,-1.6792016E-2,5.2191963E3,1.4674221E6,2.47E2,2.2905087E-4,2.237815E7,1.4656219E8,7.477354E-2,4.4949643E-2,-9.704379E-3,1.7263182E-2,-5.5229854E-2,-1.3352525E-2,7.556039E6,3.9050918E-2,5.0363316E5,1.685028E6,6.3349745E6,-6.6088433E-3,2.1959E4,9.0904115E-3,6.363344E6,-8.835182E-3,3.7069206E5,1.0864745E3,-1.8732274E-3,4.4626824E4,6.691311E8,3.0518796E-3,2.36246E-2,1.1750798E-2,1.8053012E5,1.1448067E-1,8.713209E10,6.2185767E10,7.164074E8,1.3867086E-2,1.6459569E2,4.98884E1,-5.604069E-4,1.4956795E-2,8.461375E-4,-9.873492E-3,1.9507347E-2,7.947918E-3,2.8286302E-2,1.4457171E-2,-5.5536083E-3,5.2104972E-2,6.1707415E-2,4.638572E-3,2.612106E6,1.7597401E3,2.3176976E7,-2.3552878E-3,7.062E3,2.34E2,1.0436168E-2,2.7806325E-2,-1.0598646E-2,-1.2809286E-3,-7.6905536E-3,1.7975719E-3,-1.8079907E-2,-5.526689E-3,6.23212E-3,-2.3006948E-3,-1.4206145E-2,-7.047072E-4],"split_indices":[2,1,70,9,7,7,44,12,0,12,0,1,0,4,3,0,0,4,59,3,0,57,1,0,0,0,0,0,0,57,0,59,9,57,0,2,0,41,0,40,64,0,40,5,0,0,0,45,69,43,5,7,0,66,68,0,0,0,0,0,0,0,0,0,0,50,0,1,45,9,0,2,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.13E2,1.27E2,7.8E1,2.35E2,1.18E2,9E0,7.4E1,4E0,2.09E2,2.6E1,9.8E1,2E1,4E0,5E0,6.5E1,9E0,1.77E2,3.2E1,2.1E1,5E0,5.3E1,4.5E1,6E0,1.4E1,2E0,2E0,4E0,1E0,1.76E2,1E0,1.7E1,1.5E1,2E1,1E0,4.4E1,9E0,4.3E1,2E0,5.6E1,1.2E2,4E0,1.3E1,7E0,8E0,9E0,1.1E1,1.7E1,2.7E1,1.7E1,2.6E1,5.3E1,3E0,8.1E1,3.9E1,2E0,1.1E1,1E0,6E0,1E1,7E0,2.2E1,5E0,1E0,1.6E1,2.4E1,2E0,3.5E1,1.8E1,7.7E1,4E0,1.1E1,2.8E1,2E0,2.2E1,2.7E1,8E0,5E0,1.3E1,7.4E1,3E0,6E0,5E0,2.6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[9.293402E-3,-2.641622E-1,5.168235E-1,-5.402759E-1,-1.6958784E-1,3.7590942E-1,9.6435225E-1,-5.751695E-1,-2.664181E-1,-1.9889978E-1,2.9666507E-1,4.459382E-1,-2.2401074E-1,8.2886785E-2,1.0419737E0,-2.9203515E-2,-9.834308E-3,-1.7775215E-2,-2.1797032E-3,-2.5721452E-1,-1.2755776E-2,2.2628702E-2,7.120982E-3,2.866922E-1,6.4741796E-1,-3.5113607E-2,-1.0170695E-2,1.0945568E-2,-8.133638E-3,5.770292E-1,1.1676749E0,-1.4419201E-1,-2.9330724E-1,-5.1734265E-2,1.1419823E-2,3.4406483E-1,1.1795918E-1,5.0118923E-1,7.5020254E-1,2.0626307E-2,-1.674847E-1,3.3796076E-2,-5.4148347E-3,1.2286386E0,1.5545595E-2,-1.9770765E-1,-7.306183E-3,-3.328452E-1,-1.7018813E-1,-2.1674609E-1,-1.3627906E-2,6.7440574E-3,3.6117536E-1,-1.1420029E-2,2.0852469E-1,5.3781307E-1,4.496206E-3,9.695658E-3,3.8246386E-2,-1.2232458E-2,2.144305E-3,6.874553E-2,4.234175E-2,-5.278183E-3,-2.2926433E-1,-2.7646052E-3,7.7548134E-3,-3.5174114E-1,-1.5406208E-1,4.200838E-3,-2.5122172E-1,-2.0730558E-3,-1.34063E-2,-5.1586296E-2,6.658773E-2,4.2835382E-1,2.7679804E-1,1.611665E-3,1.3783914E-2,9.56936E-3,2.826932E-2,-1.22741135E-2,-2.7402095E-3,-9.011662E-3,-3.6630732E-1,-8.935276E-3,1.7116477E-3,-1.0756968E-3,-2.7207273E-1,4.720956E-3,-8.390376E-2,-1.8053015E-3,8.761774E-3,2.2050852E-2,4.269671E-3,-4.7856346E-3,1.500351E-2,-2.0779597E-2,-1.5393791E-2,-6.7824116E-3,-1.5302828E-2,1.5795637E-3,-5.243059E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,35,37,-1,39,-1,-1,41,43,45,47,49,-1,51,53,55,57,-1,59,-1,-1,61,-1,63,65,67,69,71,73,-1,75,-1,77,79,-1,-1,-1,-1,-1,-1,-1,-1,81,-1,-1,83,85,-1,87,-1,-1,89,91,93,95,-1,-1,-1,-1,-1,-1,-1,97,-1,-1,-1,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0647636E1,7.324545E0,9.286148E0,5.3479004E-1,2.9700303E0,5.057274E0,2.3767815E0,1.0385513E-1,1.833533E-1,2.1785617E0,2.4840605E-1,3.2153835E0,1.3233168E0,1.6939883E-1,1.2548676E0,0E0,0E0,0E0,0E0,5.8183956E-1,4.7261807E-1,0E0,0E0,5.499344E-1,3.1538963E-1,0E0,7.343967E-1,0E0,0E0,6.8527246E-1,9.233818E-1,2.8425115E-1,5.031595E-1,2.6743016E-1,0E0,1.1227703E-1,5.8107847E-1,2.829461E-1,6.137085E-2,0E0,1.4057702E-1,0E0,0E0,1.2345505E-1,0E0,5.7159066E-2,1.021945E-1,2.3634243E-1,6.3918877E-1,6.10933E-2,1.1304985E-1,0E0,1.1383438E-1,0E0,1.5715694E-1,8.455467E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3522835E-2,0E0,0E0,1.2621403E-1,5.2690968E-2,0E0,1.0430062E-1,0E0,0E0,1.2592553E-1,1.4034833E-1,5.5015802E-2,2.1194541E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.381439E-2,0E0,0E0,0E0,5.6177855E-2,0E0,5.4274276E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,23,23,24,24,26,26,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,40,40,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,54,54,55,55,64,64,67,67,68,68,70,70,73,73,74,74,75,75,76,76,84,84,88,88,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,36,38,-1,40,-1,-1,42,44,46,48,50,-1,52,54,56,58,-1,60,-1,-1,62,-1,64,66,68,70,72,74,-1,76,-1,78,80,-1,-1,-1,-1,-1,-1,-1,-1,82,-1,-1,84,86,-1,88,-1,-1,90,92,94,96,-1,-1,-1,-1,-1,-1,-1,98,-1,-1,-1,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.0495757E3,3.5002026E8,1.4096699E9,3.5301748E7,5.3140095E1,2.9778808E7,1.0321884E3,9.7467445E4,6.088569E6,2.7371603E-1,2.2782404E0,1.0014992E8,6.069097E4,-2.9203515E-2,-9.834308E-3,-1.7775215E-2,-2.1797032E-3,7.556039E6,1.4998456E5,2.2628702E-2,7.120982E-3,1.7458724E7,1.2384782E5,-3.5113607E-2,1.1E1,1.0945568E-2,-8.133638E-3,2.96E2,2.3322949E2,7.192302E2,1.1883916E3,1.6359736E3,1.1419823E-2,2.1E1,1.2584659E5,1.4485321E9,2.4768E4,2.0626307E-2,9.704588E6,3.3796076E-2,-5.4148347E-3,6.0976E4,1.5545595E-2,1.6459E4,4.52737E5,6.4065235E8,4.98884E1,1.4590734E8,1.207E3,6.7440574E-3,1.15E2,-1.1420029E-2,4.2220557E3,2E0,4.496206E-3,9.695658E-3,3.8246386E-2,-1.2232458E-2,2.144305E-3,6.874553E-2,4.234175E-2,-5.278183E-3,2.8246716E7,-2.7646052E-3,7.7548134E-3,5.125401E1,7.792E3,4.200838E-3,4.041298E-1,-2.0730558E-3,-1.34063E-2,3.44094E1,1.133457E6,1.8956E4,2.4903E4,1.611665E-3,1.3783914E-2,9.56936E-3,2.826932E-2,-1.22741135E-2,-2.7402095E-3,-9.011662E-3,2.72E2,-8.935276E-3,1.7116477E-3,-1.0756968E-3,6.053216E1,4.720956E-3,4.1286648E1,-1.8053015E-3,8.761774E-3,2.2050852E-2,4.269671E-3,-4.7856346E-3,1.500351E-2,-2.0779597E-2,-1.5393791E-2,-6.7824116E-3,-1.5302828E-2,1.5795637E-3,-5.243059E-3],"split_indices":[2,1,64,7,7,57,68,12,4,45,57,47,66,1,40,0,0,0,0,57,40,0,0,57,40,0,8,0,0,3,69,64,64,67,0,8,45,44,10,0,9,0,0,10,0,9,41,7,68,5,0,0,3,0,4,8,0,0,0,0,0,0,0,0,44,0,0,60,2,0,70,0,0,70,9,0,9,0,0,0,0,0,0,0,3,0,0,0,68,0,68,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,2.83E2,1.52E2,7.1E1,2.12E2,1.17E2,3.5E1,6.2E1,9E0,2E2,1.2E1,1.05E2,1.2E1,3E0,3.2E1,6E1,2E0,6E0,3E0,1.52E2,4.8E1,5E0,7E0,6E1,4.5E1,3E0,9E0,2E0,1E0,8E0,2.4E1,3.8E1,1.14E2,4.2E1,6E0,4.4E1,1.6E1,2.1E1,2.4E1,2E0,7E0,7E0,1E0,2.2E1,2E0,2.7E1,1.1E1,8.5E1,2.9E1,7E0,3.5E1,4E0,4E1,3E0,1.3E1,1.9E1,2E0,1E0,2.3E1,5E0,2E0,1.4E1,8E0,8E0,1.9E1,9E0,2E0,7.6E1,9E0,7E0,2.2E1,2E0,5E0,2.4E1,1.1E1,2E1,2E1,4E0,9E0,2E0,1.7E1,1.7E1,2E0,7E0,6.9E1,8E0,1E0,2E0,2E1,4E0,2E1,6E0,5E0,1.9E1,1E0,1E0,1.9E1,3.4E1,3.5E1,5E0,1.5E1,3E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.7218053E-2,-2.2466917E-1,4.7799495E-1,-4.9538025E-1,-1.4453138E-1,3.4568125E-1,8.8003534E-1,-5.154521E-1,-1.554426E-1,-1.746414E-1,2.766076E-1,4.1629118E-1,-2.8330997E-1,2.774607E-2,9.857045E-1,-2.6679318E-2,-1.4990479E-2,2.295397E-3,-1.08582135E-2,-2.2530545E-1,-4.2338395E-3,-3.3439083E-3,3.3460343E-1,2.861624E-1,6.0291785E-1,-5.9500265E-1,2.432807E-2,1.0580891E-2,-8.269824E-3,6.563999E-2,7.770379E-1,-2.3631316E-1,1.587932E-1,-7.746222E-2,1.2211691E-1,2.1618191E-2,7.4071134E-3,-1.7245399E-3,3.030525E-1,3.5442153E-1,3.5264783E-2,-4.1049374E-3,-3.404111E-2,-7.2548673E-3,1.4928363E-2,4.2733055E-2,-2.1891594E-3,-2.5941142E-1,-9.347482E-2,1.3056653E-2,-6.27365E-3,-1.202256E-2,-1.8007313E-1,-4.2224615E-3,1.8718232E-1,6.770307E-2,3.1893787E-1,3.9174935E-1,4.3464038E-3,-5.2795284E-3,5.11542E-3,-2.929884E-1,-1.5798415E-1,-1.4242786E-1,-1.5003218E-3,-1.01551205E-1,2.8049478E-3,-9.976153E-3,8.727457E-4,1.3414694E-2,6.759633E-2,-3.9295065E-3,9.568731E-3,2.674296E-1,3.840444E-1,3.458414E-3,2.0729125E-2,-1.796125E-1,-3.197564E-1,-2.7858168E-1,2.6367754E-3,3.8412923E-3,-9.099513E-3,3.027823E-3,-3.1627875E-3,-7.4748807E-3,-2.0672473E-4,8.963537E-3,-6.4864237E-4,1.8115351E-2,2.062105E-1,2.0484848E-2,4.1832784E-3,-1.1119628E-2,-3.1012138E-3,-1.3562522E-2,-1.9813031E-2,-1.51024815E-2,-5.426561E-3,4.810513E-3,-4.0417435E-3,-1.9018073E-3,1.094692E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,-1,65,67,-1,69,71,73,75,-1,-1,-1,77,79,81,83,85,-1,-1,-1,-1,87,-1,-1,89,91,-1,-1,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7922035E1,6.0461645E0,7.568714E0,3.565588E-1,2.8253374E0,5.136613E0,3.2148342E0,7.764816E-2,7.222898E-2,1.7689204E0,3.2122743E-1,2.3045864E0,1.1651378E0,2.1277224E-1,1.167736E0,0E0,0E0,0E0,0E0,7.055578E-1,4.5357752E-1,0E0,1.7326915E-1,3.4625864E-1,7.964096E-1,2.0913696E-1,3.689072E-1,0E0,0E0,0E0,1.3395767E0,4.838457E-1,1.784637E-1,2.0599455E-1,2.5778794E-1,0E0,0E0,0E0,1.989851E-1,1.05697274E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.3259766E-2,4.032545E-1,1.0334225E-1,0E0,0E0,1.2772425E-1,4.9817175E-2,0E0,1.1786789E-1,1.0560635E-1,8.173895E-2,6.044185E-2,0E0,0E0,0E0,2.399826E-1,6.7870164E-1,1.444163E-1,3.835832E-2,4.1480213E-2,0E0,0E0,0E0,0E0,6.536373E-2,0E0,0E0,1.23096466E-1,1.3648319E-1,0E0,0E0,9.2068195E-2,1.8180275E-1,4.7468662E-2,1.3287576E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.0430806E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,23,23,24,24,25,25,26,26,30,30,31,31,32,32,33,33,34,34,38,38,39,39,46,46,47,47,48,48,51,51,52,52,54,54,55,55,56,56,57,57,61,61,62,62,63,63,64,64,65,65,70,70,73,73,74,74,77,77,78,78,79,79,80,80,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,-1,66,68,-1,70,72,74,76,-1,-1,-1,78,80,82,84,86,-1,-1,-1,-1,88,-1,-1,90,92,-1,-1,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,5.932393E3,6.932264E8,1.4777102E9,3.3503532E7,5.3140095E1,2.9778808E7,2.3E2,9.7467445E4,1E0,4.1331047E-1,3.5226266E0,1.0014992E8,1.417988E6,-2.6679318E-2,-1.4990479E-2,2.295397E-3,-1.08582135E-2,3.7069206E5,3.23643E6,-3.3439083E-3,6.1614815E6,1E0,1E0,1.6799316E7,4.579283E6,1.0580891E-2,-8.269824E-3,6.563999E-2,2.0531621E4,7.164074E8,1.0767E4,7.03764E5,4.3E1,2.1618191E-2,7.4071134E-3,-1.7245399E-3,2.0772269E8,1.5631589E9,3.5264783E-2,-4.1049374E-3,-3.404111E-2,-7.2548673E-3,1.4928363E-2,4.2733055E-2,1.379E3,1.0864745E3,3.0811954E-2,1.3056653E-2,-6.27365E-3,1.2103396E5,1.7997152E7,-4.2224615E-3,2.9194368E1,3.2604893E3,1.74384E5,1.872503E4,4.3464038E-3,-5.2795284E-3,5.11542E-3,6.1453125E1,7.203055E10,4.45E3,9.481747E8,6.2581446E8,2.8049478E-3,-9.976153E-3,8.727457E-4,1.3414694E-2,7.9E1,-3.9295065E-3,9.568731E-3,8.57253E8,7.378667E12,3.458414E-3,2.0729125E-2,1.0038E4,9E1,1.6505348E9,7.062E3,3.8412923E-3,-9.099513E-3,3.027823E-3,-3.1627875E-3,-7.4748807E-3,-2.0672473E-4,8.963537E-3,-6.4864237E-4,1.8115351E-2,8.15E2,2.0484848E-2,4.1832784E-3,-1.1119628E-2,-3.1012138E-3,-1.3562522E-2,-1.9813031E-2,-1.51024815E-2,-5.426561E-3,4.810513E-3,-4.0417435E-3,-1.9018073E-3,1.094692E-2],"split_indices":[2,1,64,7,7,57,68,12,0,45,13,47,66,1,41,0,0,0,0,40,1,0,57,8,102,1,41,0,0,0,4,7,0,41,8,0,0,0,5,44,0,0,0,0,0,0,0,64,50,0,0,45,9,0,66,4,11,45,0,0,0,60,43,2,7,7,0,0,0,0,3,0,0,5,43,0,0,0,8,5,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.81E2,1.47E2,6.3E1,2.18E2,1.12E2,3.5E1,5.9E1,4E0,2.04E2,1.4E1,1.01E2,1.1E1,4E0,3.1E1,5.3E1,6E0,1E0,3E0,1.57E2,4.7E1,2E0,1.2E1,6.1E1,4E1,5E0,6E0,2E0,2E0,1E1,2.1E1,1.53E2,4E0,3E1,1.7E1,7E0,5E0,3E0,5.8E1,1.3E1,2.7E1,1E0,4E0,4E0,2E0,1.9E1,2E0,1.31E2,2.2E1,3E0,1E0,1.9E1,1.1E1,4E0,1.3E1,4E0,5.4E1,1.1E1,2E0,1E0,1E0,9.7E1,3.4E1,1.4E1,8E0,8E0,1.1E1,1E1,1E0,7E0,6E0,2E0,2E0,3.3E1,2.1E1,1E0,1E1,2E1,7.7E1,1.9E1,1.5E1,2E0,1.2E1,4E0,4E0,5E0,3E0,2E0,4E0,1.1E1,2.2E1,1.9E1,2E0,1.4E1,6E0,5E1,2.7E1,1.6E1,3E0,7E0,8E0,1E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.3297128E-2,-2.032274E-1,5.2392906E-1,-4.5092753E-1,-1.2879252E-1,5.9602433E-1,-4.0802637E-1,-4.7635454E-1,-1.0005311E-3,-1.707354E-1,2.3706791E-1,4.570334E-1,1.0259942E0,-7.347379E-1,3.2800676E-3,-2.5173776E-2,-1.4556293E-2,-2.8419396E-1,-9.4364844E-2,3.1133983E-1,4.032263E-2,3.0342624E-1,5.9791887E-1,1.1265488E0,3.1802255E-1,-4.3545235E-2,-1.53783355E-2,-1.7457744E-1,-3.2878485E-1,-1.4484467E-1,2.4969776E-1,1.7776366E-2,8.163177E-3,6.0333996E-3,-3.511415E-3,3.9762172E-1,2.3124896E-1,4.5573443E-1,3.501827E-2,6.958406E-2,4.2805854E-2,3.2766066E-2,-6.276865E-3,-1.4203016E-3,-1.9876055E-1,-3.5112506E-1,-6.9808033E-3,-2.0976418E-1,-1.4620871E-2,-6.740831E-3,4.2987886E-1,2.2030275E-2,6.268879E-3,1.4800412E-2,1.5093143E-1,4.8712742E-1,4.201592E-3,-3.595816E-3,-2.4974293E-1,-2.1475814E-2,-2.9456446E-1,-7.715504E-2,-2.4666588E-1,-4.6119086E-2,1.3595942E-1,-3.2261293E-3,3.358901E-3,1.2899818E-2,3.1134473E-2,8.287592E-3,-1.4409218E-3,8.559894E-3,2.5555762E-2,-4.5632706E-3,-1.4740203E-2,-2.343262E-1,-3.854741E-1,-1.03635825E-2,2.456147E-2,-2.8377247E-1,-7.676894E-3,-9.007862E-2,6.2178336E-3,-1.7946715E-3,8.525397E-3,-1.2430013E-2,2.0570315E-3,-1.0125012E-2,-2.2568597E-2,3.976036E-3,-7.4273245E-3,-1.5749516E-2,-5.7464684E-3,-7.602002E-3,-1.8378153E-3,4.099022E-3,-3.0911432E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,57,59,-1,61,63,65,67,-1,-1,-1,69,71,-1,-1,73,-1,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,87,-1,89,91,-1,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8644222E1,5.639121E0,9.0450325E0,7.6838684E-1,3.7079883E0,6.8220406E0,1.5957217E0,1.9259834E-1,0E0,1.8361335E0,3.527621E-1,1.8528328E0,1.7062359E0,8.182907E-2,0E0,0E0,0E0,3.6296082E-1,2.2940357E0,6.006694E-2,7.961315E-2,2.2759247E-1,3.821373E-1,6.016998E-1,8.31204E-1,0E0,0E0,8.9784205E-2,2.0442867E-1,9.6069646E-1,7.883892E-1,0E0,0E0,0E0,0E0,1.6808629E-1,1.0646939E-1,2.2740078E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3072723E-1,1.0902929E-1,0E0,3.5287476E-1,1.9124773E-1,3.8541064E-2,1.7087603E-1,0E0,0E0,0E0,4.4957638E-2,7.3143005E-2,0E0,0E0,8.7653756E-2,0E0,9.586549E-2,2.4350512E-1,1.6013265E-1,7.6483406E-2,4.7789097E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0143435E-1,5.3597927E-2,0E0,1.1659201E-1,1.808443E-1,0E0,5.3927094E-2,8.7630756E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,35,35,36,36,37,37,44,44,45,45,47,47,48,48,49,49,50,50,54,54,55,55,58,58,60,60,61,61,62,62,63,63,64,64,75,75,76,76,78,78,79,79,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,58,60,-1,62,64,66,68,-1,-1,-1,70,72,-1,-1,74,-1,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,88,-1,90,92,-1,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.32424E5,1.0429407E3,1.2852292E7,1.4777102E9,7.7101436E3,1.31805E5,2.4434292E7,-1.0005311E-3,7.377246E2,5.4629724E10,1.8618166E-1,1.92E2,2.8E1,3.2800676E-3,-2.5173776E-2,-1.4556293E-2,7.37365E6,2.3171705E5,1.2155118E7,2.341E4,1.8887723E-2,1.30817086E5,3.3616E4,2.235E5,-4.3545235E-2,-1.53783355E-2,8.87682E5,3.312442E6,2.8884522E8,3.724162E6,1.7776366E-2,8.163177E-3,6.0333996E-3,-3.511415E-3,6.2185767E10,1.8053012E5,1.7092716E9,3.501827E-2,6.958406E-2,4.2805854E-2,3.2766066E-2,-6.276865E-3,-1.4203016E-3,1.6459E4,1.934371E6,-6.9808033E-3,4.2393894E1,2.1145713E3,1.214E3,1.901216E3,2.2030275E-2,6.268879E-3,1.4800412E-2,3.007E3,2E0,4.201592E-3,-3.595816E-3,6.1E1,-2.1475814E-2,6.9E1,8.8543115E2,1.4849418E3,1.07E3,8.42E2,-3.2261293E-3,3.358901E-3,1.2899818E-2,3.1134473E-2,8.287592E-3,-1.4409218E-3,8.559894E-3,2.5555762E-2,-4.5632706E-3,-1.4740203E-2,4.61E2,2.362677E6,-1.03635825E-2,3E2,2.3961474E7,-7.676894E-3,9.704588E6,1.4040858E-2,-1.7946715E-3,8.525397E-3,-1.2430013E-2,2.0570315E-3,-1.0125012E-2,-2.2568597E-2,3.976036E-3,-7.4273245E-3,-1.5749516E-2,-5.7464684E-3,-7.602002E-3,-1.8378153E-3,4.099022E-3,-3.0911432E-3],"split_indices":[2,1,70,9,7,64,40,12,0,64,43,47,8,3,0,0,0,57,40,57,11,69,40,10,2,0,0,1,9,12,1,0,0,0,0,5,45,44,0,0,0,0,0,0,9,1,0,68,64,0,4,0,0,0,0,8,0,0,3,0,8,64,64,0,0,0,0,0,0,0,0,0,0,0,0,3,1,0,3,9,0,9,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,3.08E2,1.3E2,7E1,2.38E2,1.21E2,9E0,6.6E1,4E0,2.14E2,2.4E1,9.3E1,2.8E1,5E0,4E0,5.6E1,1E1,8.5E1,1.29E2,1.7E1,7E0,4.6E1,4.7E1,2.4E1,4E0,3E0,2E0,2.6E1,5.9E1,1.13E2,1.6E1,1.2E1,5E0,4E0,3E0,1.8E1,2.8E1,2.2E1,2.5E1,1E1,1.4E1,2E0,2E0,4E0,2.2E1,5.2E1,7E0,7.5E1,3.8E1,7E0,9E0,1.5E1,3E0,1.4E1,1.4E1,2E1,2E0,7E0,1.5E1,1.9E1,3.3E1,1.7E1,5.8E1,3.2E1,6E0,4E0,3E0,6E0,3E0,1.3E1,1E0,2E0,1.8E1,4E0,1.1E1,2.2E1,1.1E1,7E0,1E1,4E1,1.8E1,1.7E1,1.5E1,1E0,5E0,2.1E1,1E0,4E0,7E0,8E0,2E0,3.3E1,7E0,7E0,1E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-2.354039E-3,-2.3067957E-1,3.907276E-1,-4.6439278E-1,-1.4581376E-1,2.7093747E-1,8.92801E-1,-4.8461908E-1,-1.2094458E-2,-1.7238756E-1,2.8708923E-1,3.5039607E-1,-2.7316603E-1,5.706212E-2,9.8354745E-1,-2.4448477E-2,-4.778079E-3,-2.0829849E-1,-2.8006803E-2,3.665053E-1,-3.6646174E-3,2.234144E-1,5.5973613E-1,-5.977377E-2,-4.756461E-2,9.664576E-3,-8.793381E-3,6.1766434E-2,7.623919E-1,-2.2451666E-1,4.4451267E-2,-1.0253909E-1,1.289454E-1,2.2316683E-2,9.852981E-3,1.0819259E-1,2.8618777E-1,3.7053436E-2,4.3133864E-1,5.180612E-2,-1.757534E-2,4.1106544E-2,-4.990757E-3,-2.3061898E-1,2.2585997E-3,-3.0591192E-3,1.09072095E-2,-1.2478681E-1,5.8491156E-3,1.5086938E-3,2.391306E-1,1.4920136E-1,-1.3474122E-2,2.900256E-3,3.189602E-1,4.5609322E-1,4.061606E-3,-7.066511E-2,2.0087942E-2,-2.537372E-1,-1.446657E-1,-1.5621501E-1,3.8563634E-5,1.0988088E-3,1.3901449E-2,4.854119E-2,1.2704159E-2,3.7386715E-1,1.0656553E-2,1.4259201E-2,2.6703238E-2,-8.866142E-3,5.353189E-3,-7.450575E-3,-2.688514E-1,4.618591E-2,-2.1020733E-1,-1.0092341E-2,-3.117481E-3,-4.3383013E-3,8.041336E-3,2.0440955E-2,7.570073E-3,-1.4229931E-2,-7.883766E-3,4.1355332E-3,-6.155274E-3,-6.494236E-4,-1.1371812E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,39,-1,-1,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,57,-1,-1,-1,59,-1,-1,-1,61,-1,-1,63,65,-1,-1,67,69,-1,71,-1,73,75,77,-1,-1,-1,79,-1,81,-1,-1,-1,-1,-1,-1,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8593132E1,5.3272047E0,9.260775E0,2.0658112E-1,2.3617702E0,5.672863E0,2.2190876E0,6.914997E-2,0E0,9.7928715E-1,3.7034935E-1,2.8787518E0,2.4047785E0,1.6108583E-1,5.647106E-1,0E0,0E0,6.330495E-1,4.694442E-1,4.7513485E-2,0E0,4.8980236E-1,6.5997887E-1,4.7420326E-1,0E0,0E0,0E0,0E0,9.635792E-1,2.4577951E-1,1.9699869E-1,1.4650244E-1,1.3424145E-1,0E0,0E0,4.585448E-1,3.2544065E-1,0E0,1.9690847E-1,5.0011355E-1,0E0,0E0,0E0,2.4148607E-1,0E0,0E0,0E0,9.877244E-2,0E0,0E0,4.4736743E-2,2.5190222E-1,0E0,0E0,1.5248537E-1,1.8152428E-1,0E0,1.897934E-1,0E0,1.3233137E-1,4.099878E-1,7.322192E-2,0E0,0E0,0E0,2.2688137E-1,0E0,1.3345933E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.20960236E-1,6.592151E-2,7.910454E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,23,23,28,28,29,29,30,30,31,31,32,32,35,35,36,36,38,38,39,39,43,43,47,47,50,50,51,51,54,54,55,55,57,57,59,59,60,60,61,61,65,65,67,67,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,40,-1,-1,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,58,-1,-1,-1,60,-1,-1,-1,62,-1,-1,64,66,-1,-1,68,70,-1,72,-1,74,76,78,-1,-1,-1,80,-1,82,-1,-1,-1,-1,-1,-1,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.203E3,7.41718E5,6.6512886E3,2.7299898E8,1.4096699E9,3.5301748E7,5.3140095E1,3.6851662E6,-1.2094458E-2,4.3159333E-1,1.5E2,4.1331047E-1,1.6244629E-1,1.0014992E8,1.37893E6,-2.4448477E-2,-4.778079E-3,2.9063344E5,4.819212E10,6.088569E6,-3.6646174E-3,1.3428E4,7.7688866E1,2.5503985E9,-4.756461E-2,9.664576E-3,-8.793381E-3,6.1766434E-2,5.01E2,7.8E2,4.037E3,5.4210108E7,5.792806E4,2.2316683E-2,9.852981E-3,1.9051096E7,3.8952081E0,3.7053436E-2,1.5631589E9,8.01E3,-1.757534E-2,4.1106544E-2,-4.990757E-3,1.1883916E3,2.2585997E-3,-3.0591192E-3,1.09072095E-2,1.8931546E8,5.8491156E-3,1.5086938E-3,1.094E3,3.44E2,-1.3474122E-2,2.900256E-3,1.955799E5,8.4E1,4.061606E-3,9.704588E6,2.0087942E-2,5.1698097E1,4.0757E1,7.164074E8,3.8563634E-5,1.0988088E-3,1.3901449E-2,9.65325E8,1.2704159E-2,2.5111986E-3,1.0656553E-2,1.4259201E-2,2.6703238E-2,-8.866142E-3,5.353189E-3,-7.450575E-3,6.4065235E8,8.8E1,4.041298E-1,-1.0092341E-2,-3.117481E-3,-4.3383013E-3,8.041336E-3,2.0440955E-2,7.570073E-3,-1.4229931E-2,-7.883766E-3,4.1355332E-3,-6.155274E-3,-6.494236E-4,-1.1371812E-2],"split_indices":[2,1,64,7,7,57,68,59,0,51,3,47,50,1,41,0,0,40,43,57,0,2,70,7,0,0,0,0,3,8,10,9,40,0,0,57,65,0,44,0,0,0,0,64,0,0,0,44,0,0,0,8,0,0,45,3,0,9,0,60,68,7,0,0,0,5,0,46,0,0,0,0,0,0,7,8,70,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.71E2,1.57E2,7.1E1,2E2,1.28E2,2.9E1,6.4E1,7E0,1.89E2,1.1E1,1.12E2,1.6E1,3E0,2.6E1,6.3E1,1E0,1.51E2,3.8E1,9E0,2E0,7.1E1,4.1E1,1.3E1,3E0,2E0,1E0,1E1,1.6E1,1.42E2,9E0,2.6E1,1.2E1,5E0,4E0,2.6E1,4.5E1,1.5E1,2.6E1,1E1,3E0,1.5E1,1E0,1.39E2,3E0,6E0,3E0,2.4E1,2E0,7E0,5E0,2.4E1,2E0,6E0,3.9E1,2.4E1,2E0,8E0,2E0,1.08E2,3.1E1,1.9E1,5E0,1E0,4E0,1.3E1,1.1E1,2.4E1,1.5E1,9E0,1.5E1,5E0,3E0,1.5E1,9.3E1,8E0,2.3E1,1.2E1,7E0,6E0,7E0,2E1,4E0,8E1,1.3E1,7E0,1E0,2E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.4103465E-2,-1.7913064E-1,5.0167763E-1,-4.2415485E-1,-1.00428194E-1,5.528132E-1,-2.2450832E-1,-4.4663623E-1,-5.5450725E-4,-1.3513464E-1,2.359129E-1,3.3365214E-1,7.0905435E-1,-5.05738E-1,2.5346333E-1,-4.7088405E-1,-1.2333427E-2,-1.4963098E-1,4.0684015E-1,-2.4871837E-2,2.92124E-1,2.905069E-1,3.360827E-2,5.975827E-1,1.0083172E0,-3.0478288E-3,-2.9110637E-2,1.8522026E-2,4.5414562E-3,-2.3746148E-2,-5.0504724E-3,-1.9482227E-1,-1.4623186E-2,3.1687373E-3,2.8121863E-2,-5.4569016E-3,3.38524E-3,1.8675435E-2,7.995031E-3,-6.147868E-3,3.116194E-1,6.3146263E-1,-8.309849E-3,6.152765E-1,6.1066683E-2,-2.2247672E-1,-4.2675026E-2,-4.5777075E-2,1.02722375E-2,2.2516921E-1,1.7794747E-2,8.878683E-3,6.466699E-1,-4.8660478E-3,3.653214E-2,-2.4695952E-1,-7.664429E-2,1.7438428E-1,-1.1651203E-1,-1.0656836E-2,-1.745129E-2,1.495144E-2,6.7992397E-3,6.5721726E-1,3.771792E-3,-2.918693E-1,-1.6392966E-1,4.7109234E-3,-9.959582E-2,3.4508284E-4,1.1994012E-2,-7.044373E-2,-1.2025656E-2,1.228225E-2,-1.7212126E-1,6.7097795E-1,1.1581204E-2,-5.0471444E-3,-1.5204595E-2,-4.990555E-3,-1.2362542E-2,-6.07733E-3,2.9146126E-3,-8.564569E-3,-1.697673E-3,-2.041634E-3,3.5302208E-3,1.2756065E-3,-1.2808088E-2,3.398795E-2,6.754233E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,49,51,-1,53,-1,55,57,59,-1,61,-1,-1,63,-1,-1,65,67,69,71,-1,73,-1,-1,75,-1,77,79,-1,81,-1,-1,83,-1,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1827534E1,6.089716E0,4.802986E0,7.1007824E-1,2.856795E0,3.8039742E0,1.3379647E0,2.3754406E-1,0E0,1.7893395E0,3.4442258E-1,5.502124E-1,1.6658821E0,1.6929364E-1,4.2513937E-2,5.295849E-2,0E0,1.3089042E0,2.855013E-1,4.6439487E-2,1.4500725E-1,4.48668E-1,0E0,1.4507294E0,6.753998E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.6792107E-1,3.8665605E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.2731E-2,2.301712E-1,0E0,7.308891E-1,0E0,4.666438E-1,4.3702012E-1,2.2870937E-1,0E0,7.800895E-2,0E0,0E0,2.3958015E-1,0E0,0E0,3.8470125E-1,9.17628E-2,7.5261936E-2,9.74057E-2,0E0,1.9986612E-1,0E0,0E0,1.1969185E-1,0E0,1.4397383E-1,2.0517743E-1,0E0,7.312772E-2,0E0,0E0,5.307944E-2,0E0,1.1761209E-1,1.22982755E-1,1.2766266E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,31,31,32,32,40,40,41,41,43,43,45,45,46,46,47,47,49,49,52,52,55,55,56,56,57,57,58,58,60,60,63,63,65,65,66,66,68,68,71,71,73,73,74,74,75,75],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,50,52,-1,54,-1,56,58,60,-1,62,-1,-1,64,-1,-1,66,68,70,72,-1,74,-1,-1,76,-1,78,80,-1,82,-1,-1,84,-1,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0429407E3,1.2852292E7,1.5522031E9,5.4276E4,3.236383E1,2.9778808E7,-5.5450725E-4,3.058642E3,2.0532622E3,7.1853364E3,8.4987135E0,3.087E3,2.9E1,3.84142E5,-1.2333427E-2,9.7467445E4,1.563E3,2.255E3,6.3349745E6,3.0811954E-2,3.360827E-2,2.0531621E4,1.7552137E0,-3.0478288E-3,-2.9110637E-2,1.8522026E-2,4.5414562E-3,-2.3746148E-2,-5.0504724E-3,2.1898598E5,1.4998456E5,3.1687373E-3,2.8121863E-2,-5.4569016E-3,3.38524E-3,1.8675435E-2,7.995031E-3,-6.147868E-3,1.9412996E9,3.0811954E-2,-8.309849E-3,1.379E3,6.1066683E-2,7.164074E8,1.36E2,1.6826648E3,1.02722375E-2,5.05991E5,1.7794747E-2,8.878683E-3,7.2125336E2,-4.8660478E-3,3.653214E-2,2.735322E6,2.0673077E0,2.251E3,1.0088743E7,-1.0656836E-2,1E0,1.495144E-2,6.7992397E-3,7.6807027E0,3.771792E-3,6.453125E0,2.7394568E7,4.7109234E-3,7.87768E8,3.4508284E-4,1.1994012E-2,6.7777224E0,-1.2025656E-2,1.3E2,6.691311E8,6.029086E10,1.1581204E-2,-5.0471444E-3,-1.5204595E-2,-4.990555E-3,-1.2362542E-2,-6.07733E-3,2.9146126E-3,-8.564569E-3,-1.697673E-3,-2.041634E-3,3.5302208E-3,1.2756065E-3,-1.2808088E-2,3.398795E-2,6.754233E-3],"split_indices":[2,1,70,9,7,2,68,12,0,64,4,4,65,0,3,41,0,45,0,0,57,50,0,4,54,0,0,0,0,0,0,40,40,0,0,0,0,0,0,0,12,50,0,0,0,7,3,4,0,41,0,0,70,0,0,1,70,0,57,0,19,0,0,65,0,59,57,0,44,0,0,59,0,8,5,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,3.17E2,1.25E2,7.6E1,2.41E2,1.17E2,8E0,7.2E1,4E0,2.19E2,2.2E1,5E1,6.7E1,5E0,3E0,6.3E1,9E0,2.14E2,5E0,4E0,1.8E1,4.6E1,4E0,5.1E1,1.6E1,1E0,4E0,1E0,2E0,6.2E1,1E0,1.6E2,5.4E1,2E0,3E0,2E0,2E0,1E1,8E0,2E0,4.4E1,4.9E1,2E0,7E0,9E0,1.35E2,2.5E1,4.8E1,6E0,1.7E1,2.7E1,2E0,4.7E1,1E0,6E0,1.15E2,2E1,6E0,1.9E1,6E0,4.2E1,8E0,9E0,4.6E1,1E0,7.3E1,4.2E1,2E0,1.8E1,2E0,4E0,1.5E1,4E0,3.6E1,6E0,4.4E1,2E0,5E0,6.8E1,2.5E1,1.7E1,1.6E1,2E0,3E0,1.2E1,1.9E1,1.7E1,2E0,4E0,4.3E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.3329867E-2,-1.6674933E-1,4.3204945E-1,-2.3522067E-1,4.8191648E-2,5.0440174E-1,-3.3350694E-1,-3.6315867E-1,-1.3965711E-1,-3.516428E-2,2.1631008E-1,3.927016E-1,8.613807E-1,-7.248316E-1,4.955804E-2,-3.8404727E-1,-1.1160061E-1,-2.3415881E-1,-6.970452E-2,-5.857936E-2,1.2724296E-2,2.4812916E-1,-4.98459E-3,2.1842732E-1,4.7097975E-1,-8.3660055E-3,9.056837E-1,-4.3623816E-2,-1.4283402E-2,-6.263455E-2,1.8059198E-2,-4.010596E-1,-7.866838E-3,-8.145454E-3,-3.3088212E-4,-1.3212469E-1,-2.5843182E-1,-1.286527E-1,1.2321885E-1,-3.561487E-2,-1.5107022E-2,1.680368E-2,9.996888E-2,3.3530596E-1,1.02450415E-1,4.9058023E-1,9.982469E-3,5.121193E-1,5.2197088E-2,-7.329156E-3,3.5117492E-3,-4.6025065E-1,-3.1343177E-1,-1.0500927E-2,-2.854301E-3,-1.785379E-1,-3.0994898E-1,-1.702927E-1,-1.0861361E-2,1.1460323E-2,-1.0068486E-3,-8.894478E-3,-9.149907E-3,6.4834408E-3,-4.1952175E-3,1.7659241E-2,2.0649028E-3,-2.2416927E-3,6.8577123E-3,1.1034693E-2,5.1147103E-1,2.9979045E-2,-4.7444543E-3,-2.3456901E-2,-8.351913E-3,-9.650872E-3,-1.725224E-2,-9.606703E-3,2.0374204E-3,-2.1968503E-2,-2.5815198E-1,-1.3319312E-1,-1.3636259E-2,-1.4113423E-1,9.444307E-2,6.1928765E-3,-2.7898012E-2,1.3559164E-2,2.6509313E-2,-1.6290975E-1,-1.7323082E-2,-1.5785855E-1,-5.9872353E-4,-9.142012E-3,-5.248717E-4,7.806926E-4,8.502957E-3,-5.643887E-3,1.7076257E-3,-9.33372E-3,2.485914E-3,-8.573968E-3,-1.4655588E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,-1,47,-1,-1,49,-1,51,-1,-1,-1,53,55,57,59,61,-1,-1,63,65,67,69,-1,71,-1,-1,-1,73,75,-1,-1,77,79,81,83,-1,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,89,91,-1,93,95,-1,97,-1,-1,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.309842E1,4.529546E0,7.4796677E0,2.797989E0,1.0586315E0,4.486561E0,1.8347542E0,4.773569E-1,8.7611437E-1,3.6095148E-1,2.7615857E-1,1.1794615E0,1.4277973E0,1.405425E-1,2.6750398E-1,2.877264E-1,4.7471732E-2,1.07025385E-1,9.142786E-1,2.578698E-1,0E0,2.6662207E-1,0E0,3.858596E-1,2.4225521E-1,0E0,9.3959427E-1,0E0,0E0,7.730324E-2,0E0,2.9143238E-1,0E0,0E0,0E0,6.405541E-2,1.298778E-1,2.9734504E-1,2.936231E-1,1.6791116E-1,0E0,0E0,6.5626465E-2,6.498814E-2,9.322466E-2,2.1958828E-1,0E0,5.3636074E-1,0E0,0E0,0E0,4.9768448E-2,7.481003E-2,0E0,0E0,6.787425E-2,6.1099052E-2,1.3488448E-1,2.4654041E-1,0E0,0E0,0E0,1.0075223E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.02083206E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.204896E-1,1.0275072E-1,0E0,4.175529E-2,5.700825E-2,0E0,1.8858099E-1,0E0,0E0,7.0161134E-2,0E0,4.604596E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,26,26,29,29,31,31,35,35,36,36,37,37,38,38,39,39,42,42,43,43,44,44,45,45,47,47,51,51,52,52,55,55,56,56,57,57,58,58,62,62,70,70,80,80,81,81,83,83,84,84,86,86,89,89,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,-1,48,-1,-1,50,-1,52,-1,-1,-1,54,56,58,60,62,-1,-1,64,66,68,70,-1,72,-1,-1,-1,74,76,-1,-1,78,80,82,84,-1,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,90,92,-1,94,96,-1,98,-1,-1,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.164074E8,1.0429407E3,1.204977E6,1.5522031E9,7.7101436E3,3.5226266E0,1.73E2,7.3123126E2,2.478066E5,7.4433655E-2,2.5096E4,5.0314346E1,3.1E1,2.5212732E4,1.7212875E5,1.9299634E3,7.37365E6,2.103E3,1E0,1.2724296E-2,7.200105E8,-4.98459E-3,1.6087094E5,1.773296E6,-8.3660055E-3,6.274818E4,-4.3623816E-2,-1.4283402E-2,1.6130411E5,1.8059198E-2,1.425966E6,-7.866838E-3,-8.145454E-3,-3.3088212E-4,2.652E3,1E0,5.3008755E8,1.4074289E7,1.384081E3,-1.5107022E-2,1.680368E-2,4.264618E6,1.6197167E4,2.3133965E3,3E0,9.982469E-3,5.37E2,5.2197088E-2,-7.329156E-3,3.5117492E-3,1.5650286E6,2.68269E5,-1.0500927E-2,-2.854301E-3,4.75E2,9.3E4,6.54925E6,1.656968E6,1.1460323E-2,-1.0068486E-3,-8.894478E-3,2.409091E0,6.4834408E-3,-4.1952175E-3,1.7659241E-2,2.0649028E-3,-2.2416927E-3,6.8577123E-3,1.1034693E-2,2.160242E-1,2.9979045E-2,-4.7444543E-3,-2.3456901E-2,-8.351913E-3,-9.650872E-3,-1.725224E-2,-9.606703E-3,2.0374204E-3,-2.1968503E-2,1.902843E-2,1.2472753E5,-1.3636259E-2,2.029E3,2.972948E6,6.1928765E-3,1.8756017E5,1.3559164E-2,2.6509313E-2,4.61E2,-1.7323082E-2,3.061705E6,-5.9872353E-4,-9.142012E-3,-5.248717E-4,7.806926E-4,8.502957E-3,-5.643887E-3,1.7076257E-3,-9.33372E-3,2.485914E-3,-8.573968E-3,-1.4655588E-3],"split_indices":[2,7,70,1,7,64,66,8,64,40,69,2,68,3,4,40,4,57,0,19,0,5,0,45,11,0,40,0,0,40,0,9,0,0,0,10,25,12,57,4,0,0,1,55,4,8,0,3,0,0,0,59,12,0,0,3,12,9,9,0,0,0,70,0,0,0,0,0,0,0,54,0,0,0,0,0,0,0,0,0,69,40,0,0,1,0,59,0,0,3,0,9,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.06E2,1.31E2,2.32E2,7.4E1,1.2E2,1.1E1,9.8E1,1.34E2,5E1,2.4E1,9.3E1,2.7E1,5E0,6E0,9E1,8E0,5.6E1,7.8E1,4.7E1,3E0,2.2E1,2E0,3E1,6.3E1,1E0,2.6E1,3E0,2E0,5E0,1E0,8.3E1,7E0,5E0,3E0,1.2E1,4.4E1,6E1,1.8E1,4.4E1,3E0,1.3E1,9E0,1.4E1,1.6E1,5.8E1,5E0,8E0,1.8E1,3E0,2E0,4.7E1,3.6E1,5E0,7E0,1.9E1,2.5E1,4.4E1,1.6E1,1E1,8E0,6E0,3.8E1,8E0,1E0,1.3E1,1E0,3E0,1.3E1,5E0,5.3E1,7E0,1E0,4.5E1,2E0,9E0,2.7E1,1.8E1,1E0,5E0,2E1,3.4E1,1E1,7E0,9E0,4E0,3.4E1,5E0,4.8E1,1.1E1,9E0,2.8E1,6E0,5E0,2E0,5E0,4E0,1.4E1,2E1,1E1,1E0,2.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-6.6460073E-3,-1.5758432E-1,4.0623355E-1,-2.5586525E-1,9.642101E-2,1.2914006E-1,5.046144E-1,-4.0369043E-1,-1.9301175E-1,-3.891419E-2,3.3136103E-1,1.791976E-1,-2.0714182E-2,4.4297755E-1,4.236941E-2,-4.1853684E-1,1.4768438E-2,-1.7371638E-1,-4.60478E-2,-7.4190535E-2,9.537275E-3,2.5830218E-1,3.9804053E-2,-9.411167E-3,2.1217386E-1,4.5955458E-1,-5.4002153E-3,-4.4349834E-1,-1.33521985E-2,-2.3844753E-1,-1.18670434E-1,-3.2388043E-2,-1.823551E-1,1.15950875E-1,1.7329775E-2,-3.1637207E-3,2.4776347E-1,3.0998316E-1,5.4573274E-1,-2.2388462E-2,-4.9180016E-3,-1.4762458E-1,-2.799195E-1,-1.5843473E-1,5.9062764E-2,-5.225001E-2,5.454937E-3,-5.486195E-4,-2.2809644E-1,7.4985437E-3,8.664679E-5,6.4550093E-3,2.7841857E-1,5.818711E-3,3.362945E-1,5.6420326E-1,1.0762791E-2,-2.0894608E-1,-7.494735E-2,-3.347672E-1,-1.9084792E-1,-4.9186133E-2,-1.9873257E-1,6.2205414E-3,-4.0778597E-3,-9.443754E-2,7.4833673E-3,-1.5939359E-2,-6.5280357E-3,3.353264E-1,1.779292E-1,1.7393623E-2,1.3600695E-3,5.1582444E-1,4.472091E-2,-1.2316352E-2,-3.1466186E-3,-7.008333E-3,-8.3030964E-4,-3.5992387E-1,-7.098125E-3,-9.257362E-2,-1.6940491E-2,4.869057E-2,-4.1388073E-3,-3.0724942E-3,-2.1174897E-1,-9.030734E-3,-1.703577E-3,2.230602E-3,-3.7853566E-3,6.95425E-3,1.956554E-2,1.0056348E-2,-2.1047056E-4,1.1845644E-2,2.6610944E-2,-8.612565E-3,-1.9363182E-2,-6.1325957E-3,2.9402035E-3,-3.9653103E-3,4.506161E-3,-1.123102E-2,-4.1701035E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,-1,29,-1,31,-1,33,-1,-1,35,37,-1,39,-1,41,43,45,47,49,-1,-1,51,53,55,-1,-1,57,59,61,63,65,-1,-1,67,-1,-1,-1,69,-1,71,73,-1,75,77,79,81,83,85,-1,-1,87,89,-1,-1,91,93,-1,-1,95,-1,-1,-1,-1,-1,97,-1,99,-1,101,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7235247E1,8.013113E0,3.1243382E0,2.0683813E0,2.8745036E0,9.4507E-1,1.4435616E0,8.2956505E-1,2.144867E0,4.8423514E-1,9.156034E-1,4.0352947E-1,0E0,7.34745E-1,0E0,1.3465595E-1,0E0,5.453768E-1,0E0,2.2469285E-1,0E0,3.3756113E-1,0E0,0E0,2.9020154E-1,7.7178764E-1,0E0,4.259205E-2,0E0,2.3363924E-1,6.383594E-1,1.12530366E-1,1.07242525E-1,5.0360084E-2,0E0,0E0,5.4932356E-2,1.0853195E-1,1.5309143E-1,0E0,0E0,9.575701E-2,1.7549372E-1,3.116064E-1,1.6655913E-1,8.638684E-2,0E0,0E0,5.5741787E-2,0E0,0E0,0E0,4.8989654E-2,0E0,8.127713E-2,2.1025372E-1,0E0,5.564797E-2,4.7178887E-2,8.94897E-2,2.7431262E-1,7.3155046E-2,8.1668615E-2,0E0,0E0,9.7959876E-2,4.975881E-2,0E0,0E0,6.673658E-2,3.903249E-2,0E0,0E0,5.4818153E-2,0E0,0E0,0E0,0E0,0E0,6.2315226E-2,0E0,7.1100146E-2,0E0,3.9016206E-2,0E0,0E0,6.1182022E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,17,17,19,19,21,21,24,24,25,25,27,27,29,29,30,30,31,31,32,32,33,33,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,48,48,52,52,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,69,69,70,70,73,73,79,79,81,81,83,83,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,-1,30,-1,32,-1,34,-1,-1,36,38,-1,40,-1,42,44,46,48,50,-1,-1,52,54,56,-1,-1,58,60,62,64,66,-1,-1,68,-1,-1,-1,70,-1,72,74,-1,76,78,80,82,84,86,-1,-1,88,90,-1,-1,92,94,-1,-1,96,-1,-1,-1,-1,-1,98,-1,100,-1,102,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,9.7467445E4,1.00493375E5,7.41718E5,2.081964E3,8.494E3,1.1275989E1,8.588E3,5.825042E6,1.5271514E5,8.596084E3,3.0972284E1,-2.0714182E-2,5.22E2,4.236941E-2,2.7299898E8,1.4768438E-2,6.505339E2,-4.60478E-2,1.716E3,9.537275E-3,2.0663544E9,3.9804053E-2,-9.411167E-3,1.484798E9,5.6503E4,-5.4002153E-3,8.95654E8,-1.33521985E-2,7.2123305E6,2.103E3,1.207E3,1.0843525E5,1.556838E0,1.7329775E-2,-3.1637207E-3,1.7532333E9,1.6772E4,3.0113987E10,-2.2388462E-2,-4.9180016E-3,1.523794E6,2.72E2,9.174246E6,2.7394568E7,7.10711E8,5.454937E-3,-5.486195E-4,1.1762006E9,7.4985437E-3,8.664679E-5,6.4550093E-3,1.59E2,5.818711E-3,1.1367206E8,3.63796E5,1.0762791E-2,2.96E2,5.8955963E1,3.312442E6,9.3E1,5.6E1,8.824219E0,6.2205414E-3,-4.0778597E-3,8.7998315E2,5.192606E6,-1.5939359E-2,-6.5280357E-3,2.4339888E3,1.31E3,1.7393623E-2,1.3600695E-3,1.5200746E-1,4.472091E-2,-1.2316352E-2,-3.1466186E-3,-7.008333E-3,-8.3030964E-4,5.381665E1,-7.098125E-3,4.75E2,-1.6940491E-2,3.84E2,-4.1388073E-3,-3.0724942E-3,2.3176976E7,-9.030734E-3,-1.703577E-3,2.230602E-3,-3.7853566E-3,6.95425E-3,1.956554E-2,1.0056348E-2,-2.1047056E-4,1.1845644E-2,2.6610944E-2,-8.612565E-3,-1.9363182E-2,-6.1325957E-3,2.9402035E-3,-3.9653103E-3,4.506161E-3,-1.123102E-2,-4.1701035E-3],"split_indices":[7,45,45,1,64,0,47,2,59,40,4,68,0,3,0,7,0,64,0,0,0,12,0,0,7,2,0,5,0,57,0,0,45,53,0,0,7,9,12,0,0,1,3,57,57,7,0,0,12,0,0,0,3,0,1,2,0,8,62,9,8,3,59,0,0,64,62,0,0,4,0,0,0,53,0,0,0,0,0,60,0,3,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,3.19E2,1.16E2,2.3E2,8.9E1,3.1E1,8.5E1,6.7E1,1.63E2,5.7E1,3.2E1,2.9E1,2E0,7.4E1,1.1E1,6.6E1,1E0,1.6E2,3E0,5E1,7E0,2.9E1,3E0,2E0,2.7E1,7.2E1,2E0,5.5E1,1.1E1,7.2E1,8.8E1,3.7E1,1.3E1,1.2E1,1.7E1,3E0,2.4E1,2.8E1,4.4E1,5.4E1,1E0,2.4E1,4.8E1,7.2E1,1.6E1,3.3E1,4E0,3E0,1E1,9E0,3E0,6E0,1.8E1,4E0,2.4E1,4.1E1,3E0,1.2E1,1.2E1,2.8E1,2E1,2E1,5.2E1,1.1E1,5E0,1.9E1,1.4E1,4E0,6E0,1E1,8E0,2.3E1,1E0,3.8E1,3E0,9E0,3E0,5E0,7E0,2.4E1,4E0,1.3E1,7E0,5E0,1.5E1,5E0,4.7E1,7E0,1.2E1,1E1,4E0,3E0,7E0,7E0,1E0,3E0,3.5E1,4E0,2E1,1.1E1,2E0,1E0,4E0,4.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.2736447E-3,-1.6982651E-1,3.8341367E-1,-2.4999872E-1,5.2451666E-2,7.9158686E-2,4.6317568E-1,-2.3756562E-1,-4.432183E-2,-4.259399E-2,2.5116095E-1,2.6578328E-1,-1.0030152E-1,3.7390417E-1,7.377287E-1,-3.777308E-1,-1.7950164E-1,-8.4150046E-2,9.41491E-3,1.5766539E-1,1.863215E-2,6.3292584E-3,1.5431302E-2,-1.9954007E-2,7.7134767E-3,3.9701095E-1,-7.9262275E-3,5.6576836E-1,4.8050314E-2,-1.9852467E-2,-1.1968977E-1,-1.8862449E-1,3.198856E-2,-2.0385182E-1,-3.526186E-2,9.027301E-3,1.892313E-3,-2.9471847E-3,7.7876337E-3,2.8026792E-1,5.124896E-1,2.936646E-2,5.95064E-3,2.7580117E-3,-8.2811555E-3,-2.402078E-1,-1.2543026E-1,-2.7877465E-1,-5.6379447E-3,-6.458512E-2,5.9884038E-2,1.678117E-1,3.2819825E-1,4.841647E-3,5.3347075E-1,-1.3233073E-1,-2.7690846E-1,-1.6364545E-1,6.647529E-2,-1.6228428E-2,-7.694407E-5,-1.0366019E-1,5.2272785E-3,-6.1576103E-4,7.6573733E-3,1.0369907E-2,2.742588E-3,6.8039894E-3,3.465322E-1,7.0908684E-1,2.225975E-2,-9.676703E-2,-1.2400035E-2,-8.1062075E-3,-2.976551E-1,-6.869177E-2,-2.0092897E-1,-3.6976798E-3,5.797899E-3,-6.796912E-3,3.8144336E-4,-1.3955958E-3,5.144663E-3,2.7737576E-3,1.7830098E-2,9.661771E-3,3.722781E-2,-7.429623E-3,-2.487845E-3,-2.073147E-2,-1.2454204E-2,3.0667293E-3,-5.9733097E-3,-1.0729715E-2,-4.7849392E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,47,49,-1,-1,-1,-1,51,53,-1,-1,-1,-1,55,57,59,-1,61,63,65,67,-1,69,71,73,75,77,-1,-1,79,81,-1,-1,-1,-1,-1,83,85,-1,87,-1,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7881329E1,5.5374775E0,3.0939064E0,1.594656E0,1.5782737E0,9.6442586E-1,2.2241573E0,1.7633686E0,0E0,5.629796E-1,2.4833608E-1,4.4637322E-2,4.8747778E-1,1.0350189E0,3.7940884E-1,2.79665E-1,1.3245153E0,2.7955994E-1,0E0,3.993553E-2,0E0,0E0,0E0,0E0,1.2771367E-1,8.809452E-1,0E0,8.614159E-2,0E0,0E0,5.7423674E-2,4.9143314E-1,0E0,6.4043224E-2,1.04307175E-1,0E0,0E0,0E0,0E0,1.6052699E-1,2.709589E-1,0E0,0E0,0E0,0E0,3.0780363E-1,5.5357265E-1,8.867121E-2,0E0,7.692374E-2,6.254238E-2,5.135885E-2,4.78909E-2,0E0,2.2544956E-1,7.481918E-2,1.0402584E-1,2.0569074E-1,9.904221E-2,0E0,0E0,6.582996E-2,3.8509298E-2,0E0,0E0,0E0,0E0,0E0,5.172515E-2,4.0761948E-2,0E0,4.1367948E-2,0E0,0E0,1.9751692E-1,1.3294229E-1,1.1386573E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,24,24,25,25,27,27,30,30,31,31,33,33,34,34,39,39,40,40,45,45,46,46,47,47,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,61,61,62,62,68,68,69,69,71,71,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,48,50,-1,-1,-1,-1,52,54,-1,-1,-1,-1,56,58,60,-1,62,64,66,68,-1,70,72,74,76,78,-1,-1,80,82,-1,-1,-1,-1,-1,84,86,-1,88,-1,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.7467445E4,6.028226E4,5.825042E6,2.1695537E3,8.169258E3,4.026531E9,7.41718E5,-4.432183E-2,1.5271514E5,9.825568E6,3.149493E3,2.412E4,3.768034E4,7.443074E0,3.5002026E8,9.256843E3,8.689165E-2,9.41491E-3,1.2678202E9,1.863215E-2,6.3292584E-3,1.5431302E-2,-1.9954007E-2,3.087E3,6.4945E4,-7.9262275E-3,6.029086E10,4.8050314E-2,-1.9852467E-2,2.3E2,7.8936383E2,3.198856E-2,6.426437E5,3.53E2,9.027301E-3,1.892313E-3,-2.9471847E-3,7.7876337E-3,7.996302E-1,2.4126951E-1,2.936646E-2,5.95064E-3,2.7580117E-3,-8.2811555E-3,7.2123305E6,2.103E3,3.672717E3,-5.6379447E-3,2.681357E6,2.1530056E3,1.6255458E5,5E1,4.841647E-3,5.339615E-1,1.376072E8,5.381665E1,1.423795E7,3.7730695E-7,-1.6228428E-2,-7.694407E-5,6.2581446E8,6.212394E4,-6.1576103E-4,7.6573733E-3,1.0369907E-2,2.742588E-3,6.8039894E-3,1.01319025E3,9.904738E6,2.225975E-2,5.92E2,-1.2400035E-2,-8.1062075E-3,4.94E4,2.1692945E1,6.4065235E8,-3.6976798E-3,5.797899E-3,-6.796912E-3,3.8144336E-4,-1.3955958E-3,5.144663E-3,2.7737576E-3,1.7830098E-2,9.661771E-3,3.722781E-2,-7.429623E-3,-2.487845E-3,-2.073147E-2,-1.2454204E-2,3.0667293E-3,-5.9733097E-3,-1.0729715E-2,-4.7849392E-4],"split_indices":[7,45,45,59,64,45,7,1,0,40,1,4,9,4,65,7,4,50,0,7,0,0,0,0,0,2,0,5,0,0,0,64,0,59,8,0,0,0,0,54,53,0,0,0,0,57,0,4,0,1,4,45,3,0,46,12,60,57,50,0,0,7,40,0,0,0,0,0,4,1,0,10,0,0,9,65,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.09E2,1.28E2,2.27E2,8.2E1,2.7E1,1.01E2,2.24E2,3E0,5.6E1,2.6E1,1.3E1,1.4E1,7.8E1,2.3E1,6.4E1,1.6E2,4.8E1,8E0,1.6E1,1E1,4E0,9E0,3E0,1.1E1,7.5E1,3E0,1.5E1,8E0,5.9E1,5E0,1.59E2,1E0,1.3E1,3.5E1,1.3E1,3E0,8E0,3E0,3.9E1,3.6E1,1.4E1,1E0,1E0,4E0,8.6E1,7.3E1,6E0,7E0,2.7E1,8E0,1.3E1,2.6E1,2E0,3.4E1,2.3E1,6.3E1,6.1E1,1.2E1,5E0,1E0,1.7E1,1E1,5E0,3E0,9E0,4E0,3E0,2.3E1,9E0,2.5E1,1.9E1,4E0,1.1E1,5.2E1,1.8E1,4.3E1,3E0,9E0,1.3E1,4E0,8E0,2E0,1E0,2.2E1,1E0,8E0,8E0,1.1E1,1.3E1,3.9E1,5E0,1.3E1,4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[7.4815685E-3,-1.7263705E-1,3.3803168E-1,-3.5948348E-1,-1.0460054E-1,3.77988E-1,-5.7788205E-1,-3.8422588E-1,-1.7609951E-1,-1.2210213E-1,1.1961301E-1,2.9822886E-1,7.530429E-1,-3.9222434E-2,-1.1318062E-2,-3.9243245E-1,-7.839435E-3,-1.1320916E-2,-1.5602678E-3,-1.443582E-1,4.6196647E-2,2.5564477E-1,-4.8374355E-2,1.947771E-1,4.344733E-1,-7.3102503E-3,8.340808E-1,-1.9808382E-2,-3.6275962E-3,-1.5324715E-1,9.510105E-3,-1.07010804E-1,9.2146546E-2,6.691333E-4,1.6139673E-2,2.5584572E-4,-6.95852E-3,2.2025315E-1,-4.2931974E-2,2.8291345E-1,5.1530516E-1,5.475735E-2,6.040819E-1,-1.7119913E-1,-4.5855258E-2,-8.251245E-3,3.0569904E-4,9.584191E-3,3.141196E-2,1.6033211E-1,2.8078142E-1,-7.7728657E-3,5.424938E-3,-6.874888E-5,3.1291103E-1,2.6609106E-2,8.036138E-3,3.4810822E-2,1.0133701E-4,-2.2007543E-1,-1.21343546E-1,-1.5836855E-3,-6.4620757E-3,-6.479751E-4,6.7219394E-3,1.061145E-1,1.2990009E-2,3.4018382E-3,3.004389E-1,3.4222263E-1,2.1243975E-3,-1.9926394E-3,-2.3273934E-1,-1.66619E-1,-4.3781817E-2,4.8799803E-3,-2.7939439E-2,1.7360361E-1,4.0687077E-2,1.4515755E-3,1.5447318E-2,1.7816847E-2,2.8402624E-3,-1.1971159E-2,-2.7959378E-4,-6.3724034E-3,-1.2071872E-2,-5.748621E-3,4.160958E-3,-3.833612E-3,1.2155423E-3,1.0632084E-2,-7.3616445E-4,-6.4967843E-3,3.9964737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,35,37,39,-1,41,-1,-1,43,-1,45,47,-1,-1,-1,-1,49,51,53,55,-1,57,59,61,-1,-1,-1,63,65,67,-1,-1,-1,69,-1,-1,-1,-1,71,73,75,-1,-1,-1,77,-1,-1,79,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6671944E1,3.6489878E0,6.0006905E0,2.8171444E-1,8.5436106E-1,4.322357E0,3.3131528E-1,6.345558E-2,7.329211E-2,7.520573E-1,3.779925E-1,1.7054043E0,2.0168562E0,0E0,0E0,5.156994E-2,0E0,0E0,0E0,5.527084E-1,1.7881748E-1,1.4241725E-1,3.959745E-2,4.575913E-1,5.157604E-1,0E0,7.8209686E-1,0E0,0E0,3.237481E-1,0E0,4.044155E-2,1.0894355E-1,0E0,0E0,0E0,0E0,1.9798541E-1,1.5333833E-1,1.7951798E-1,1.20874405E-1,0E0,8.336463E-1,3.3149862E-1,9.584908E-2,0E0,0E0,0E0,6.121821E-2,1.6680604E-1,1.1335349E-1,0E0,0E0,0E0,1.3604558E-1,0E0,0E0,0E0,0E0,1.5731215E-1,2.5607836E-1,4.9803246E-2,0E0,0E0,0E0,1.0332754E-1,0E0,0E0,5.9143066E-2,4.9043655E-2,0E0,0E0,1.0366702E-1,1.06940985E-1,2.7205902E-1,0E0,4.0097557E-2,9.159008E-2,1.04260236E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,31,31,32,32,37,37,38,38,39,39,40,40,42,42,43,43,44,44,48,48,49,49,50,50,54,54,59,59,60,60,61,61,65,65,68,68,69,69,72,72,73,73,74,74,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,36,38,40,-1,42,-1,-1,44,-1,46,48,-1,-1,-1,-1,50,52,54,56,-1,58,60,62,-1,-1,-1,64,66,68,-1,-1,-1,70,-1,-1,-1,-1,72,74,76,-1,-1,-1,78,-1,-1,80,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,2.3028242E3,3.5002026E8,1.1576422E9,7.3068325E3,3.1E1,2.9778808E7,2.033552E6,5.9170656E8,1.24E2,4.1331047E-1,5.0314346E1,-3.9222434E-2,-1.1318062E-2,3.84142E5,-7.839435E-3,-1.1320916E-2,-1.5602678E-3,3.346087E5,8.7998315E2,1.8505337E3,2.08E2,3.271774E7,1E0,-7.3102503E-3,1.417988E6,-1.9808382E-2,-3.6275962E-3,7.164074E8,9.510105E-3,1.6776951E3,2.299087E3,6.691333E-4,1.6139673E-2,2.5584572E-4,-6.95852E-3,2.0663544E9,1.2538845E9,2.1853803E4,5.848343E7,5.475735E-2,4.42384E5,1.947899E6,8.9846605E8,-8.251245E-3,3.0569904E-4,9.584191E-3,3.274075E-1,2.1597655E9,3.8952081E0,-7.7728657E-3,5.424938E-3,-6.874888E-5,1.5631589E9,2.6609106E-2,8.036138E-3,3.4810822E-2,1.0133701E-4,1.8460208E8,1.0623282E3,1.2504131E1,-6.4620757E-3,-6.479751E-4,6.7219394E-3,1.2254369E7,1.2990009E-2,3.4018382E-3,4.264618E6,9.677921E7,2.1243975E-3,-1.9926394E-3,2.9063344E5,8.593763E7,8.26042E4,4.8799803E-3,1.4555919E-1,2.0569E4,1.2009271E3,1.4515755E-3,1.5447318E-2,1.7816847E-2,2.8402624E-3,-1.1971159E-2,-2.7959378E-4,-6.3724034E-3,-1.2071872E-2,-5.748621E-3,4.160958E-3,-3.833612E-3,1.2155423E-3,1.0632084E-2,-7.3616445E-4,-6.4967843E-3,3.9964737E-3],"split_indices":[2,1,70,7,7,64,3,12,9,12,3,47,68,0,0,41,0,0,0,40,64,4,3,57,102,0,41,0,0,7,0,4,4,0,0,0,0,12,7,45,1,0,2,1,7,0,0,0,50,7,65,0,0,0,44,0,0,0,0,5,64,70,0,0,0,9,0,0,1,1,0,0,40,7,40,0,50,10,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,2.89E2,1.57E2,7.6E1,2.13E2,1.51E2,6E0,6.6E1,1E1,1.98E2,1.5E1,1.26E2,2.5E1,3E0,3E0,6.3E1,3E0,7E0,3E0,1.75E2,2.3E1,8E0,7E0,7.3E1,5.3E1,2E0,2.3E1,6.2E1,1E0,1.71E2,4E0,5E0,1.8E1,2E0,6E0,5E0,2E0,6.6E1,7E0,2E1,3.3E1,9E0,1.4E1,1.46E2,2.5E1,3E0,2E0,6E0,1.2E1,3.5E1,3.1E1,4E0,3E0,2E0,1.8E1,3.1E1,2E0,1.2E1,2E0,7.2E1,7.4E1,1.7E1,8E0,9E0,3E0,2.4E1,1.1E1,3E0,2.8E1,1.6E1,2E0,5E0,6.7E1,4.6E1,2.8E1,3E0,1.4E1,1.1E1,1.3E1,1E0,2.7E1,1.5E1,1E0,6.5E1,2E0,3.2E1,1.4E1,1.8E1,1E1,7E0,7E0,9E0,2E0,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-1.754247E-2,-1.6004801E-1,2.8222972E-1,-3.3112457E-1,-1.02329634E-1,-4.8885766E-1,3.304729E-1,-3.444614E-1,-8.054545E-5,-1.4688173E-1,3.0040538E-2,1.7713524E-2,-6.385749E-1,2.2134902E-1,4.6459076E-1,-1.806396E-2,-1.0709468E-2,-2.3818153E-1,-8.4017605E-2,-7.640849E-3,2.1544342E-1,-4.0565256E-2,-1.3120374E-2,1.1967729E-4,2.3007527E-1,-4.569553E-2,5.10968E-1,-1.2915762E-1,-2.7920794E-1,-3.0383179E-2,-1.6907312E-1,9.016252E-3,-1.357388E-2,-3.3601238E-3,1.2710049E-2,2.4681231E-1,9.660319E-2,-6.633865E-3,4.2777867E-3,4.466933E-1,3.8079903E-2,-2.3045359E-4,-1.5819138E-1,-7.373073E-3,-3.084386E-1,3.5484686E-2,-1.0841185E-1,-1.8635686E-1,-2.0026613E-3,1.0175195E-2,-8.480554E-3,4.438103E-3,2.5292692E-1,8.693154E-3,-7.7608577E-4,5.065033E-1,1.7474592E-1,-2.5452941E-3,-1.05983E-2,-3.3202383E-1,-7.4873776E-3,-4.040507E-2,7.877535E-3,-1.345567E-1,9.292383E-4,-4.4534383E-3,-2.0748568E-1,-8.699064E-2,5.780714E-2,3.1351006E-1,2.2172675E-1,5.911461E-1,1.8971637E-2,-7.957809E-3,1.3900575E-2,-1.9531962E-3,-3.397165E-1,-9.066579E-3,-5.8458456E-5,-3.6012467E-3,-2.0754786E-1,-7.1972203E-3,-1.2638661E-2,-7.6336204E-3,-2.4945727E-2,-1.9616764E-3,7.816381E-2,3.3376533E-1,4.7090854E-3,3.976867E-3,2.336543E-1,3.0415181E-2,6.0514547E-3,-1.7951231E-2,-7.935686E-3,-1.1895051E-2,-2.347086E-3,-2.733295E-3,4.1242247E-3,5.297672E-3,-4.5500082E-4,1.7379833E-2,2.743761E-3,8.867864E-3,1.39303375E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,-1,27,29,31,33,-1,-1,-1,35,37,39,41,43,45,47,49,-1,-1,-1,51,53,-1,-1,55,-1,-1,57,-1,59,61,63,65,-1,-1,67,-1,69,-1,-1,71,73,-1,-1,75,-1,77,-1,79,-1,-1,81,83,85,87,89,91,-1,-1,-1,-1,93,-1,-1,-1,95,-1,-1,-1,97,-1,99,101,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9354248E1,2.9975142E0,5.5927105E0,3.3783436E-1,1.3667042E0,1.3626423E0,1.9168558E0,8.6206436E-2,0E0,9.729192E-1,4.138342E-1,0E0,3.0770206E-1,1.484797E-1,1.4669561E0,0E0,0E0,2.770381E-1,4.69301E-1,2.2229485E-1,1.2700221E-1,0E0,0E0,0E0,1.4370179E-1,7.993237E-2,5.630617E-1,7.520756E-2,1.4694381E-1,3.379198E-1,8.1716776E-2,1.6514826E-1,0E0,0E0,0E0,4.1305065E-2,8.9427836E-2,0E0,0E0,6.7595387E-1,0E0,0E0,8.743903E-2,0E0,9.989953E-2,3.4006307E-1,1.02118045E-1,5.3424716E-2,0E0,0E0,2.3922004E-1,0E0,4.956913E-2,0E0,0E0,1.8372154E-1,3.896185E-1,0E0,0E0,6.3342094E-2,0E0,1.2704E-1,0E0,9.908804E-2,0E0,0E0,4.4281125E-2,8.1864476E-2,5.24633E-2,5.294144E-2,5.7705164E-2,9.8546505E-2,0E0,0E0,0E0,0E0,5.9269667E-2,0E0,0E0,0E0,4.2639375E-2,0E0,0E0,0E0,4.2875357E-2,0E0,5.1986173E-2,4.7265768E-2,0E0,0E0,5.4448843E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,35,35,36,36,39,39,42,42,44,44,45,45,46,46,47,47,50,50,52,52,55,55,56,56,59,59,61,61,63,63,66,66,67,67,68,68,69,69,70,70,71,71,76,76,80,80,84,84,86,86,87,87,90,90],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,-1,28,30,32,34,-1,-1,-1,36,38,40,42,44,46,48,50,-1,-1,-1,52,54,-1,-1,56,-1,-1,58,-1,60,62,64,66,-1,-1,68,-1,70,-1,-1,72,74,-1,-1,76,-1,78,-1,80,-1,-1,82,84,86,88,90,92,-1,-1,-1,-1,94,-1,-1,-1,96,-1,-1,-1,98,-1,100,102,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.700398E4,1.2852292E7,7.10711E8,1.3812E4,6.8963E4,1.7367111E6,-8.054545E-5,7.283496E2,3.4491902E3,1.7713524E-2,3.1E1,1.4E1,2.943086E4,-1.806396E-2,-1.0709468E-2,7.37365E6,2.474199E1,1E0,8.287E3,-4.0565256E-2,-1.3120374E-2,1.1967729E-4,2.5916522E7,1.6130411E5,1.3861362E8,8.840007E7,5.381665E1,1.6777562E7,1.4205709E0,2.829932E0,-1.357388E-2,-3.3601238E-3,1.2710049E-2,4.841781E-2,5.5580516E4,-6.633865E-3,4.2777867E-3,4.42384E5,3.8079903E-2,-2.3045359E-4,1.5112E4,-7.373073E-3,2.0964778E5,2.327793E9,6.0070217E1,3.26021E5,-2.0026613E-3,1.0175195E-2,1.9252769E5,4.438103E-3,1.7045455E-2,8.693154E-3,-7.7608577E-4,5.3699964E7,3.087E3,-2.5452941E-3,-1.05983E-2,5.39E2,-7.4873776E-3,1.6845247E3,7.877535E-3,1.656968E6,9.292383E-4,-4.4534383E-3,2.9480488E1,2.8688732E7,5.898222E2,1.5953E4,8.879629E4,1.5953E4,1.8971637E-2,-7.957809E-3,1.3900575E-2,-1.9531962E-3,7.177E3,-9.066579E-3,-5.8458456E-5,-3.6012467E-3,1.16985195E5,-7.1972203E-3,-1.2638661E-2,-7.6336204E-3,1.5522031E9,-1.9616764E-3,6.843157E-2,6.7304E4,4.7090854E-3,3.976867E-3,1.3210104E9,3.0415181E-2,6.0514547E-3,-1.7951231E-2,-7.935686E-3,-1.1895051E-2,-2.347086E-3,-2.733295E-3,4.1242247E-3,5.297672E-3,-4.5500082E-4,1.7379833E-2,2.743761E-3,8.867864E-3,1.39303375E-2],"split_indices":[2,1,45,9,7,9,2,63,0,64,4,0,3,3,45,0,0,57,65,19,2,0,0,0,57,40,56,5,60,57,51,70,0,0,0,51,40,0,0,2,0,0,9,0,40,5,68,41,0,0,59,0,69,0,0,56,0,0,0,0,0,4,0,9,0,0,65,9,64,0,45,0,0,0,0,0,2,0,0,0,40,0,0,0,7,0,69,2,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.51E2,3.06E2,1.45E2,7.6E1,2.3E2,8E0,1.37E2,7.3E1,3E0,1.72E2,5.8E1,1E0,7E0,7.7E1,6E1,6.3E1,1E1,6.9E1,1.03E2,4.9E1,9E0,4E0,3E0,3E0,7.4E1,5E0,5.5E1,2E1,4.9E1,6.4E1,3.9E1,4.7E1,2E0,1E0,8E0,6.5E1,9E0,3E0,2E0,4.6E1,9E0,4E0,1.6E1,1E1,3.9E1,3.5E1,2.9E1,3.4E1,5E0,3E0,4.4E1,3E0,6.2E1,5E0,4E0,3.7E1,9E0,6E0,1E1,3.3E1,6E0,2.2E1,1.3E1,2.4E1,5E0,7E0,2.7E1,2E1,2.4E1,1.8E1,4.4E1,2E1,1.7E1,2E0,7E0,1E0,3.2E1,4E0,1.8E1,1.4E1,1E1,1.3E1,1.4E1,9E0,1.1E1,4E0,2E1,1.6E1,2E0,4E0,4E1,1.9E1,1E0,2.8E1,4E0,8E0,2E0,9E0,2E0,1.5E1,5E0,1.5E1,1E0,2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[6.4767827E-3,-1.29907E-1,3.4626526E-1,-3.3372983E-1,-7.41637E-2,4.1594756E-1,-3.9470613E-1,-3.4834534E-1,-1.187278E-3,-1.03079624E-1,1.5776859E-1,3.2167333E-1,7.869933E-1,-6.677378E-1,6.647938E-2,-2.24362E-2,-1.5617291E-2,-1.152558E-1,4.289358E-1,1.442738E-2,7.6271966E-2,2.0965776E-1,4.0933296E-1,4.892368E-2,2.292708E-2,-3.8943425E-2,-1.2959774E-2,2.607892E-1,-7.495924E-3,-1.4931358E-1,2.7999345E-3,4.933318E-3,2.4329005E-2,1.3995814E-1,-4.380032E-2,2.4859093E-1,1.308544E-1,5.0667703E-1,3.2754767E-1,2.278936E-3,1.7270897E-2,-1.6080798E-1,-2.5181806E-2,-8.115722E-2,6.0604874E-2,8.441808E-3,-1.6112167E-4,3.1464421E-3,-4.952421E-3,8.9546805E-4,2.6632807E-1,4.0654964E-3,1.1169499E-2,5.924501E-3,5.4129356E-1,3.4719107E-1,1.7505038E-3,-1.7070049E-1,1.2042357E-2,-4.946436E-3,2.5867706E-3,-1.2088463E-1,3.1086083E-3,-1.718546E-2,1.451792E-1,1.4649181E-2,6.627871E-3,2.794906E-2,5.4499237E-3,5.897614E-3,1.8005468E-2,-1.9438305E-1,-1.1756515E-1,-2.8376507E-3,3.9209444E-3,-1.7186281E-1,-2.4200217E-3,-3.271167E-3,1.9623388E-3,9.656817E-3,-1.1620384E-3,-1.015751E-2,3.7971328E-4,1.7207018E-3,-7.0578856E-3,-9.858656E-3,7.839862E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,33,35,37,-1,-1,-1,-1,39,-1,41,43,-1,-1,45,47,49,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,-1,-1,65,-1,-1,-1,67,69,-1,71,73,-1,-1,75,-1,77,79,-1,-1,-1,-1,-1,-1,81,83,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9740177E1,3.429224E0,6.463355E0,2.86201E-1,1.6258318E0,3.66461E0,1.4294915E0,8.329773E-2,0E0,1.4357109E0,2.6605248E-1,8.077316E-1,9.3698025E-1,1.167078E-1,2.4942662E-1,0E0,0E0,8.4476876E-1,4.750651E-2,0E0,1.437752E-1,9.632945E-2,2.3210526E-1,0E0,0E0,0E0,0E0,3.8981393E-2,0E0,2.2852206E-1,2.3786917E-1,0E0,0E0,5.030121E-2,4.7562126E-2,1.0569787E-1,5.519989E-2,2.1798134E-1,1.6022205E-1,0E0,0E0,2.568345E-1,9.029114E-2,1.21425E-1,1.9328783E-1,0E0,0E0,0E0,0E0,0E0,5.069542E-2,0E0,0E0,0E0,8.647251E-2,4.1487455E-2,0E0,1.5457869E-1,4.5591813E-2,0E0,0E0,5.0780684E-2,0E0,4.6167627E-2,1.1779547E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7342615E-1,1.7028922E-1,0E0,0E0,4.59764E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,20,20,21,21,22,22,27,27,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,50,50,54,54,55,55,57,57,58,58,61,61,63,63,64,64,71,71,72,72,75,75],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,34,36,38,-1,-1,-1,-1,40,-1,42,44,-1,-1,46,48,50,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,-1,-1,66,-1,-1,-1,68,70,-1,72,74,-1,-1,76,-1,78,80,-1,-1,-1,-1,-1,-1,82,84,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0429407E3,1.2852292E7,1.4096699E9,7.7101436E3,3.5226266E0,1.44E2,-1.187278E-3,3.058642E3,3.907527E8,4.1331047E-1,5.192243E6,3.1E1,1.872503E4,-2.24362E-2,-1.5617291E-2,2.8884522E8,1.563E3,1.442738E-2,5.78E2,1.921711E5,9.170229E6,4.892368E-2,2.292708E-2,-3.8943425E-2,-1.2959774E-2,3.498E3,-7.495924E-3,3.94E2,1.656968E6,4.933318E-3,2.4329005E-2,1.24E2,2.1933604E3,3.7860696E0,1.55592E5,3.0811954E-2,1.7092716E9,2.278936E-3,1.7270897E-2,2.889654E5,2.6452028E7,1.4998456E5,3.15584E6,8.441808E-3,-1.6112167E-4,3.1464421E-3,-4.952421E-3,8.9546805E-4,2.5111986E-3,4.0654964E-3,1.1169499E-2,5.924501E-3,5.848343E7,5.0491E4,1.7505038E-3,2.1721223E3,6.471207E6,-4.946436E-3,2.5867706E-3,8.9E1,3.1086083E-3,1.2022638E0,1.6130304E7,1.4649181E-2,6.627871E-3,2.794906E-2,5.4499237E-3,5.897614E-3,1.8005468E-2,1.1340564E3,1.64E2,-2.8376507E-3,3.9209444E-3,8.01E3,-2.4200217E-3,-3.271167E-3,1.9623388E-3,9.656817E-3,-1.1620384E-3,-1.015751E-2,3.7971328E-4,1.7207018E-3,-7.0578856E-3,-9.858656E-3,7.839862E-4],"split_indices":[2,1,70,9,7,64,66,0,0,64,5,47,41,3,45,0,0,12,0,0,8,45,60,0,0,0,0,0,0,8,9,0,0,3,44,65,11,50,44,0,0,40,12,40,1,0,0,0,0,0,46,0,0,0,1,2,0,67,57,0,0,8,0,53,57,0,0,0,0,0,0,64,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,3.03E2,1.21E2,6.4E1,2.39E2,1.11E2,1E1,6.1E1,3E0,2.13E2,2.6E1,9E1,2.1E1,6E0,4E0,1.3E1,4.8E1,2.09E2,4E0,9E0,1.7E1,4.1E1,4.9E1,1.2E1,9E0,4E0,2E0,2E0,2E0,1.62E2,4.7E1,1E0,3E0,1.1E1,6E0,2.6E1,1.5E1,2E1,2.9E1,1E0,1E0,1.48E2,1.4E1,1.9E1,2.8E1,9E0,2E0,2E0,4E0,2E0,2.4E1,1.1E1,4E0,2E0,1.8E1,2.7E1,2E0,1.4E2,8E0,7E0,7E0,1.5E1,4E0,1.5E1,1.3E1,1.9E1,5E0,1.7E1,1E0,2E0,2.5E1,9.5E1,4.5E1,4E0,4E0,8E0,7E0,8E0,7E0,1E1,3E0,9.1E1,4E0,6E0,3.9E1,7E0,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-3.3745257E-4,-1.5864608E-1,2.55752E-1,-3.185605E-1,-9.708536E-2,1.8780926E-1,6.818292E-1,-3.2428056E-1,2.7597616E-3,-1.1393065E-1,2.3436834E-1,2.3159711E-1,-3.7860292E-1,7.793327E-1,2.3854628E-2,-3.7428176E-1,-1.3420012E-2,-1.8471307E-1,-7.594994E-2,1.6746635E-2,5.5658254E-3,1.1959833E-1,3.418626E-1,-6.417816E-1,6.5567784E-2,4.7681917E-2,1.988688E-2,-6.5651285E-3,1.2232014E-2,-3.81463E-1,-2.892548E-3,-1.9490217E-1,1.0966551E-3,-9.427757E-2,1.2422776E-1,-2.7191767E-2,1.5043752E-1,2.6100993E-1,4.174165E-1,-3.7386432E-2,-1.2527987E-2,1.2707546E-2,-7.246178E-3,-1.9439433E-2,-2.962436E-3,-5.2711805E-3,-2.1830052E-1,-5.6959458E-2,-1.6649641E-1,-1.1975896E-4,1.1501857E-2,-1.0810249E-1,6.191332E-3,2.2629377E-1,9.412359E-2,2.7282703E-1,2.0030153E-3,2.5350976E-1,4.5809898E-1,-2.3912685E-1,-6.2278295E-3,-8.255702E-2,1.6450921E-2,-1.9840728E-1,-2.7044136E-3,-7.547312E-3,1.4027756E-3,2.4949484E-1,-9.634742E-4,1.125155E-1,-3.825301E-3,2.813431E-1,-2.0499254E-4,1.494005E-2,3.1481597E-3,4.9241576E-1,1.30757615E-2,-2.9563266E-1,-1.5677936E-1,-3.358885E-2,-1.2129401E-1,6.104189E-3,-3.3968404E-2,-1.1637037E-2,-3.7270912E-3,1.5680652E-2,7.831384E-3,4.9984753E-2,1.4753471E-1,4.4852975E-3,1.486843E-2,1.49526475E-2,2.7474714E-2,-1.5658757E-2,-3.955088E-3,-1.1362235E-2,-6.695873E-4,-3.594884E-3,3.5750872E-4,-3.7430048E-3,-1.0148317E-2,1.0090659E-3,-4.192081E-3,-2.727012E-3,4.33826E-3,9.037825E-3,1.9827937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,-1,35,37,39,41,-1,-1,-1,-1,43,-1,45,-1,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,59,61,63,-1,-1,65,-1,67,69,71,-1,73,75,77,-1,79,81,83,-1,-1,-1,85,-1,87,-1,89,-1,-1,-1,91,-1,93,95,97,99,-1,101,-1,-1,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8040854E1,2.6762958E0,4.793252E0,1.8039942E-1,1.1433887E0,3.7583761E0,1.4569693E0,1.08569145E-1,0E0,4.9944925E-1,8.657056E-2,1.6633582E0,1.3279387E0,9.5344067E-1,1.6930676E-1,6.1991215E-2,0E0,1.4324951E-1,4.7397858E-1,0E0,0E0,3.2097268E-1,3.125534E-1,1.0350251E-1,2.3553678E-1,0E0,0E0,0E0,0E0,6.4068794E-2,0E0,1.0880041E-1,0E0,3.0314457E-1,1.48083E-1,1.7230181E-1,2.2646272E-1,8.3052635E-2,1.1058378E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.580496E-2,1.4793006E-1,1.2912583E-1,0E0,0E0,5.683428E-2,0E0,1.4155102E-1,1.1847299E-1,8.133769E-2,0E0,5.9072256E-2,4.645157E-2,1.351223E-1,0E0,1.0594487E-1,1.16055526E-1,1.1059332E-1,0E0,0E0,0E0,8.203149E-2,0E0,6.2698394E-2,0E0,7.552171E-2,0E0,0E0,0E0,6.8184376E-2,0E0,5.9469342E-2,1.7909583E-1,4.269389E-2,1.06197625E-1,0E0,4.2242784E-2,0E0,0E0,0E0,0E0,5.478381E-2,6.4677656E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,21,21,22,22,23,23,24,24,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,46,46,47,47,48,48,51,51,53,53,54,54,55,55,57,57,58,58,59,59,61,61,62,62,63,63,67,67,69,69,71,71,75,75,77,77,78,78,79,79,80,80,82,82,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,-1,36,38,40,42,-1,-1,-1,-1,44,-1,46,-1,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,60,62,64,-1,-1,66,-1,68,70,72,-1,74,76,78,-1,80,82,84,-1,-1,-1,86,-1,88,-1,90,-1,-1,-1,92,-1,94,96,98,100,-1,102,-1,-1,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,7.7101436E3,2.91E2,1.4096699E9,9.05929E2,1.92E2,4.90403E5,2.7597616E-3,6.505339E2,3.907527E8,2.5096E4,3.5226266E0,5.192243E6,3.677E3,1.6395413E7,-1.3420012E-2,3.01E2,2.4082131E5,1.6746635E-2,5.5658254E-3,4.0712055E1,1.30817086E5,3.1E1,1.872503E4,4.7681917E-2,1.988688E-2,-6.5651285E-3,1.2232014E-2,8.95654E8,-2.892548E-3,5.1698097E1,1.0966551E-3,2.515304E7,2.251E3,1.5271514E5,1.12E2,1.0014992E8,5.3515047E-1,-3.7386432E-2,-1.2527987E-2,1.2707546E-2,-7.246178E-3,-1.9439433E-2,-2.962436E-3,-5.2711805E-3,9.457892E6,2.753337E6,5.502256E9,-1.1975896E-4,1.1501857E-2,1.153215E9,6.191332E-3,6.88E2,4.2663252E7,5.0772004E9,2.0030153E-3,2.724043E6,1.7176836E7,2.72E2,-6.2278295E-3,7.477264E6,1.3474197E8,1.530655E3,-2.7044136E-3,-7.547312E-3,1.4027756E-3,9.1747506E5,-9.634742E-4,1.79E2,-3.825301E-3,2E0,-2.0499254E-4,1.494005E-2,3.1481597E-3,5.6503E4,1.30757615E-2,3.8147795E-1,2.662171E6,6.2581446E8,1.7045455E-2,6.104189E-3,1.0586429E11,-1.1637037E-2,-3.7270912E-3,1.5680652E-2,7.831384E-3,8.42E2,3.27E2,4.4852975E-3,1.486843E-2,1.49526475E-2,2.7474714E-2,-1.5658757E-2,-3.955088E-3,-1.1362235E-2,-6.695873E-4,-3.594884E-3,3.5750872E-4,-3.7430048E-3,-1.0148317E-2,1.0090659E-3,-4.192081E-3,-2.727012E-3,4.33826E-3,9.037825E-3,1.9827937E-3],"split_indices":[2,1,64,8,7,70,8,9,0,64,5,2,66,41,0,12,0,8,40,0,0,68,40,3,45,0,0,0,0,5,0,60,0,57,0,40,3,1,47,0,0,0,0,0,0,0,44,1,5,0,0,7,0,8,57,44,0,41,55,3,0,57,7,64,0,0,0,59,0,3,0,8,0,0,0,2,0,50,1,7,69,0,43,0,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,2.74E2,1.69E2,7.5E1,1.99E2,1.47E2,2.2E1,7.4E1,1E0,1.9E2,9E0,1.37E2,1E1,1.9E1,3E0,3.6E1,3.8E1,6.5E1,1.25E2,4E0,5E0,6.9E1,6.8E1,6E0,4E0,1.2E1,7E0,2E0,1E0,3.5E1,1E0,6.2E1,3E0,1.15E2,1E1,1.2E1,5.7E1,3.5E1,3.3E1,4E0,2E0,2E0,2E0,3.4E1,1E0,1.4E1,4.8E1,7.7E1,3.8E1,5E0,5E0,8E0,4E0,2.3E1,3.4E1,3.3E1,2E0,8E0,2.5E1,3.8E1,1E1,5.7E1,2E1,2.9E1,9E0,6E0,2E0,2.1E1,2E0,3.1E1,3E0,3.2E1,1E0,6E0,2E0,2E1,5E0,2.1E1,1.7E1,2.6E1,3.1E1,6E0,1.4E1,2.2E1,7E0,1.1E1,1E1,1.2E1,1.9E1,3E0,2.9E1,6E0,1.4E1,1.9E1,2E0,1.1E1,6E0,1.3E1,1.3E1,2.1E1,1E1,7E0,7E0,3E0,9E0,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[6.914658E-3,-1.375686E-1,2.5927174E-1,-3.0013898E-1,-8.7718986E-2,3.0763754E-1,-2.7399808E-1,-3.1545487E-1,8.05666E-4,-1.08775854E-1,1.676461E-1,2.3644063E-1,6.353598E-1,-2.4508651E-2,-3.589169E-2,-1.764556E-2,-2.4414134E-1,-1.4596921E-1,-1.4592769E-3,-2.9842185E-3,2.206311E-1,1.1860321E-1,3.3492017E-1,7.305997E-1,-7.1109384E-2,-1.6415203E-1,1.1294316E-2,-1.2647287E-2,-7.4791734E-4,-1.7872418E-1,-1.03842266E-1,-8.606982E-3,1.9916467E-2,1.4995705E-2,6.1747446E-3,1.5337913E-1,-2.74861E-2,2.673003E-1,3.9268947E-1,4.745792E-2,2.7511101E-2,-9.480651E-3,7.113438E-3,-9.918533E-4,-1.3364579E-2,-2.2289921E-1,-9.7433835E-2,-6.406284E-2,-2.107816E-1,-3.143761E-2,1.07253134E-1,2.1023472E-1,5.8526974E-2,-1.1436814E-2,5.0875377E-3,8.097888E-3,3.0440927E-1,2.3801678E-1,4.3885097E-1,-2.4345683E-1,-6.1107804E-3,-1.165747E-2,-1.2676226E-1,-1.1431789E-1,1.9548368E-2,-2.142738E-3,-2.556923E-1,-6.991408E-2,2.0277679E-2,9.76633E-3,4.6884656E-2,2.512935E-1,1.3112697E-1,-4.8032343E-2,5.5674855E-3,7.729273E-3,1.6775828E-2,3.0432711E-3,1.3727707E-2,7.474615E-3,4.575036E-1,-1.5563093E-2,-1.8704803E-1,4.182032E-3,-2.5413497E-3,-1.4250593E-1,1.2842668E-3,-1.3079995E-1,4.428318E-3,7.899326E-2,-4.779585E-2,-2.711135E-1,-1.3797609E-3,1.2413735E-3,-5.410832E-3,-2.8242762E-3,2.8308968E-3,-3.0703156E-4,6.114718E-3,1.5607449E-2,6.7967544E-3,-3.347285E-4,7.964274E-3,-5.4446757E-3,3.4733661E-3,2.3871105E-2,8.194765E-3,-1.1245429E-2,-4.5142686E-3,-8.430691E-3,-1.2475826E-3,-3.5433506E-3,-1.0178906E-2,6.9158785E-3,3.2170716E-4,3.415501E-3,-4.266625E-3,-5.418153E-3,-1.5446687E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,27,29,31,-1,33,35,37,39,41,43,-1,-1,-1,45,47,-1,49,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1,59,61,63,65,67,69,71,73,-1,-1,-1,75,77,79,81,-1,83,85,87,89,-1,91,93,95,-1,97,99,101,103,-1,-1,-1,-1,-1,-1,105,-1,107,-1,-1,109,-1,111,-1,113,115,117,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6224516E1,2.2671237E0,4.2626E0,3.2477713E-1,1.1904632E0,3.3263636E0,1.5338507E0,8.283901E-2,0E0,8.060622E-1,2.1796706E-1,1.3992448E0,1.8013382E0,3.8690832E-1,0E0,0E0,4.735005E-2,1.8842101E-1,1.9688964E-1,0E0,7.087362E-2,2.988786E-1,1.5675974E-1,3.845415E-1,1.282559E-1,9.900704E-2,0E0,0E0,0E0,2.7633548E-1,2.7576762E-1,0E0,2.1865629E-1,0E0,0E0,2.4116051E-1,3.3569106E-1,8.3488226E-2,1.3830328E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.986784E-2,7.650846E-2,2.1653089E-1,1.24771476E-1,6.310217E-2,8.664283E-2,6.124258E-2,1.12405166E-1,0E0,0E0,0E0,6.0461044E-2,4.8221648E-2,6.6473484E-2,1.0657883E-1,0E0,3.788529E-2,5.8868527E-2,1.18618816E-1,8.3840944E-2,0E0,4.1757524E-2,6.810502E-2,4.229836E-2,0E0,4.877504E-2,8.412957E-2,4.7764674E-2,5.7686985E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.769348E-2,0E0,7.515341E-2,0E0,0E0,6.0432345E-2,0E0,1.16514504E-1,0E0,4.650972E-2,4.947658E-2,4.67965E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,29,29,30,30,32,32,35,35,36,36,37,37,38,38,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,66,66,67,67,68,68,70,70,71,71,72,72,73,73,80,80,82,82,85,85,87,87,89,89,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,28,30,32,-1,34,36,38,40,42,44,-1,-1,-1,46,48,-1,50,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1,60,62,64,66,68,70,72,74,-1,-1,-1,76,78,80,82,-1,84,86,88,90,-1,92,94,96,-1,98,100,102,104,-1,-1,-1,-1,-1,-1,106,-1,108,-1,-1,110,-1,112,-1,114,116,118,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,9.05929E2,1.2852292E7,1.4777102E9,7.7101436E3,7.339209E6,1.425966E6,8.05666E-4,9.7467445E4,2.0532622E3,2.5096E4,1.3936486E8,3.236383E1,-3.589169E-2,-1.764556E-2,8.32E3,2.09299E6,1.5802073E3,-2.9842185E-3,6.088569E6,2.4722598E7,1.3047821E5,2.3222E4,3.677E3,2.2201815E9,1.1294316E-2,-1.2647287E-2,-7.4791734E-4,6.82045E6,2.8248588E-2,-8.606982E-3,1.2E3,1.4995705E-2,6.1747446E-3,5.1063637E1,6.426437E5,3.3810287E0,5.3515047E-1,4.745792E-2,2.7511101E-2,-9.480651E-3,7.113438E-3,-9.918533E-4,-1.3364579E-2,5.7192594E5,2.50089E5,7.330576E2,1.339596E7,6.2581446E8,1.473681E6,1.14E2,7.10711E8,-1.1436814E-2,5.0875377E-3,8.097888E-3,1.1274355E7,6.45254E0,4.1322E4,2.55E2,-6.1107804E-3,4.138E3,7.8E2,4.61E2,2.58E2,-2.142738E-3,1.0707373E-1,1.4301278E1,1.0368186E-1,9.76633E-3,2.024909E6,2.488E3,1.66E2,1.0336E4,5.5674855E-3,7.729273E-3,1.6775828E-2,3.0432711E-3,1.3727707E-2,7.474615E-3,5.825042E6,-1.5563093E-2,1E0,4.182032E-3,-2.5413497E-3,1.0643513E3,1.2842668E-3,1.7377544E7,4.428318E-3,6.1E1,9.06E2,3.7382904E7,-1.3797609E-3,1.2413735E-3,-5.410832E-3,-2.8242762E-3,2.8308968E-3,-3.0703156E-4,6.114718E-3,1.5607449E-2,6.7967544E-3,-3.347285E-4,7.964274E-3,-5.4446757E-3,3.4733661E-3,2.3871105E-2,8.194765E-3,-1.1245429E-2,-4.5142686E-3,-8.430691E-3,-1.2475826E-3,-3.5433506E-3,-1.0178906E-2,6.9158785E-3,3.2170716E-4,3.415501E-3,-4.266625E-3,-5.418153E-3,-1.5446687E-2],"split_indices":[2,1,70,9,7,64,59,9,0,45,4,2,1,68,0,0,2,1,4,0,57,57,40,10,0,7,0,0,0,9,69,0,0,0,0,70,59,65,47,0,0,0,0,0,0,63,41,64,57,7,41,3,7,0,0,0,1,66,2,3,0,2,8,3,3,0,51,65,50,0,9,0,3,10,0,0,0,0,0,0,59,0,26,0,0,64,0,57,0,8,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,2.82E2,1.61E2,6.5E1,2.17E2,1.48E2,1.3E1,6.2E1,3E0,2.01E2,1.6E1,1.23E2,2.5E1,9E0,4E0,3.8E1,2.4E1,1.49E2,5.2E1,3E0,1.3E1,5.7E1,6.6E1,2.2E1,3E0,6E0,3E0,2.3E1,1E0,8.2E1,6.7E1,5E0,4.7E1,6E0,7E0,4.6E1,1.1E1,3.3E1,3.3E1,8E0,1.4E1,2E0,1E0,3E0,3E0,5.2E1,3E1,5E1,1.7E1,3E1,1.7E1,2.8E1,1.8E1,4E0,7E0,1E1,2.3E1,9E0,2.4E1,4.2E1,1E1,8E0,2.2E1,3.1E1,1.9E1,4E0,1.3E1,1.7E1,1.3E1,6E0,1.1E1,1.7E1,1.1E1,6E0,1.2E1,5E0,1.8E1,2E0,7E0,2E0,2.2E1,1.7E1,2.5E1,2E0,6E0,2E1,2E0,2.9E1,2E0,1E1,9E0,1.2E1,1E0,5E0,1.2E1,4E0,9E0,7E0,4E0,1E1,7E0,2E0,9E0,4E0,2E0,2E1,2E0,1.7E1,8E0,1.6E1,4E0,1.7E1,1.2E1,5E0,5E0,2E0,7E0,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-4.5158667E-3,-1.1468655E-1,2.8693977E-1,-1.7676862E-1,4.648886E-2,2.1168965E-1,6.262045E-1,-1.6690476E-1,-3.5463657E-2,-3.8676865E-2,2.3283775E-1,2.4176304E-1,-1.5732015E-2,1.8295826E-2,4.0603448E-2,-3.2807168E-1,-1.3398427E-1,-6.376014E-2,8.729884E-3,1.8902464E-1,2.7464014E-2,1.00318655E-1,3.026897E-1,-1.7497705E-2,-5.402815E-3,-1.4107442E-1,7.760317E-2,-8.407632E-3,-3.849008E-2,4.8027337E-3,1.2841712E-2,1.3443221E-1,-1.0395823E-1,2.2458532E-1,3.7418863E-1,-1.5926063E-1,-4.630308E-2,2.3180418E-4,8.663032E-3,-1.6562516E-2,-1.6455689E-1,1.00598615E-2,4.6943597E-2,-1.0924818E-2,2.2657758E-3,1.275023E-1,2.76157E-1,4.8426003E-3,3.884912E-1,-1.635734E-1,1.3047633E-2,-9.789412E-2,-3.6889873E-5,-2.1150315E-3,5.1360115E-2,-3.608747E-3,-1.3174543E-2,-1.6283733E-3,7.254288E-3,1.0035674E-2,3.2216536E-3,4.0086736E-3,2.9174092E-1,2.0434916E-2,7.1690106E-3,-1.8277587E-1,-7.1852885E-2,-1.0371063E-2,-3.9164472E-2,7.680068E-3,9.006588E-6,6.9954344E-3,1.6039012E-2,-9.884866E-3,-5.4080896E-3,7.098563E-4,-7.129677E-3,2.2595797E-3,-3.8210596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,35,37,-1,39,-1,-1,41,43,45,47,49,51,-1,-1,53,55,-1,57,-1,-1,59,61,-1,63,65,-1,67,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,71,-1,-1,73,75,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4514458E1,3.2902198E0,3.033124E0,1.1272106E0,1.469095E0,1.7069097E0,7.683964E-1,1.197576E0,0E0,3.5364774E-1,2.9852247E-1,8.1417274E-1,0E0,0E0,0E0,1.5130568E-1,3.0498433E-1,1.4664695E-1,0E0,1.4574534E-1,0E0,2.300022E-1,2.9570103E-1,0E0,0E0,3.2009673E-1,4.8095778E-2,0E0,1.2996274E-1,0E0,0E0,1.4848989E-1,9.54889E-2,1.4002502E-1,9.719801E-2,3.3094358E-1,7.515147E-2,0E0,0E0,7.565577E-2,4.4999927E-2,0E0,1.06265314E-1,0E0,0E0,5.175139E-2,4.3777943E-2,0E0,1.013298E-1,2.6514578E-1,0E0,8.844736E-2,0E0,0E0,6.2809795E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.368639E-2,0E0,0E0,1.2414837E-1,1.7874038E-1,0E0,3.8070843E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,17,17,19,19,21,21,22,22,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,42,42,45,45,46,46,48,48,49,49,51,51,54,54,62,62,65,65,66,66,68,68],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,36,38,-1,40,-1,-1,42,44,46,48,50,52,-1,-1,54,56,-1,58,-1,-1,60,62,-1,64,66,-1,68,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,72,-1,-1,74,76,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.7467445E4,1.9900316E0,2.2233E4,2.1695537E3,2.0531621E4,6.0495757E3,4.26944E5,-3.5463657E-2,1.5271514E5,3.3056E4,9.396226E4,-1.5732015E-2,1.8295826E-2,4.0603448E-2,7.154E3,2.9063344E5,1.6826648E3,8.729884E-3,1.757E3,2.7464014E-2,1.1863593E-1,6.7304E4,-1.7497705E-2,-5.402815E-3,6.4065235E8,2.8182E4,-8.407632E-3,1E0,4.8027337E-3,1.2841712E-2,7.801943E3,2.4469028E7,7.996302E-1,4.2464733E-2,1.3664E4,3.738719E4,2.3180418E-4,8.663032E-3,3.15584E6,2.865605E8,1.00598615E-2,8.255324E4,-1.0924818E-2,2.2657758E-3,1.5191719E5,1.0721749E3,4.8426003E-3,3.0113987E10,2.822E3,1.3047633E-2,3.923606E1,-3.6889873E-5,-2.1150315E-3,7.25E3,-3.608747E-3,-1.3174543E-2,-1.6283733E-3,7.254288E-3,1.0035674E-2,3.2216536E-3,4.0086736E-3,1.375746E9,2.0434916E-2,7.1690106E-3,1.347372E3,1.4399977E10,-1.0371063E-2,4.422E3,7.680068E-3,9.006588E-6,6.9954344E-3,1.6039012E-2,-9.884866E-3,-5.4080896E-3,7.098563E-4,-7.129677E-3,2.2595797E-3,-3.8210596E-3],"split_indices":[7,45,51,2,64,4,64,1,0,40,2,45,0,0,0,2,40,4,0,0,0,50,2,0,0,7,9,0,19,0,0,44,9,54,50,2,40,0,0,1,7,0,45,0,0,45,4,0,12,0,0,65,0,0,2,0,0,0,0,0,0,0,7,0,0,64,5,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.5E2,3.27E2,1.23E2,2.36E2,9.1E1,1.02E2,2.1E1,2.33E2,3E0,6.3E1,2.8E1,9.7E1,5E0,1E1,1.1E1,3.8E1,1.95E2,5.7E1,6E0,2.6E1,2E0,3E1,6.7E1,3.4E1,4E0,1.89E2,6E0,1E1,4.7E1,1.2E1,1.4E1,2.6E1,4E0,3.4E1,3.3E1,1.58E2,3.1E1,4E0,2E0,4.1E1,6E0,1.4E1,1.2E1,2E0,2E0,1.3E1,2.1E1,2E0,3.1E1,1.57E2,1E0,1.4E1,1.7E1,3E1,1.1E1,4E0,2E0,7E0,5E0,5E0,8E0,2E0,1.9E1,2.8E1,3E0,1.29E2,2.8E1,4E0,1E1,3E0,8E0,4E0,1.5E1,1.06E2,2.3E1,1.3E1,1.5E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.29316E-3,-1.2366014E-1,2.6226074E-1,-2.6201954E-1,-7.987835E-2,3.1039494E-1,-3.2606393E-1,-2.737112E-1,8.425078E-4,-1.10770546E-1,6.2356103E-2,2.3377381E-1,5.665446E-1,8.241814E-2,-5.712563E-1,-1.7760327E-2,-1.2409545E-2,-1.2958758E-1,5.5793077E-2,1.2357809E-2,1.686126E-1,1.6638435E-1,3.3238313E-1,-9.579106E-3,6.2980795E-1,-6.1339363E-3,1.2998829E-2,-3.3236064E-2,-1.1221248E-2,-1.4182223E-1,3.8490176E-2,-7.1630003E-3,9.6942715E-2,-9.020776E-2,6.5494165E-2,1.3357296E-2,6.1207533E-2,-4.7589215E-3,1.8130043E-1,2.7086937E-1,4.1684806E-1,2.1011192E-2,8.229569E-1,-3.7125804E-2,-1.5571955E-1,-1.7791794E-3,8.203853E-3,1.2700339E-1,-2.0000655E-3,-9.409242E-3,-3.634753E-4,-1.1811532E-4,1.074345E-1,5.9928386E-3,-1.3732224E-3,2.0163079E-1,8.487063E-2,5.1353686E-3,2.967646E-1,2.1663504E-2,4.5987293E-3,9.541472E-3,4.2965267E-2,-3.8816715E-3,6.7940517E-3,-1.8447061E-1,-9.001048E-2,8.107463E-4,7.6636653E-3,6.5866155E-3,-8.845845E-4,1.3973573E-2,1.7089231E-1,-8.119495E-4,6.869721E-3,1.6269138E-2,6.688087E-3,4.384359E-3,-1.7253119E-3,-2.1550064E-1,-1.3866964E-1,-1.7942494E-2,-1.6443175E-1,6.8859295E-3,1.1825405E-2,-8.143927E-3,-1.358049E-2,-2.5548001E-3,-9.105189E-3,2.407192E-3,-2.9055818E-3,-9.103464E-3,7.222216E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,35,37,39,-1,41,-1,-1,-1,-1,43,45,-1,47,49,51,-1,53,-1,55,57,59,-1,61,63,65,-1,-1,67,-1,-1,-1,-1,69,-1,-1,71,73,-1,75,-1,-1,-1,-1,-1,77,79,81,-1,-1,-1,-1,-1,83,-1,-1,-1,-1,-1,-1,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4405259E1,1.9053941E0,4.037485E0,2.5241566E-1,1.074633E0,2.3993711E0,1.1488053E0,6.318998E-2,0E0,6.3407683E-1,2.3167266E-1,6.1586475E-1,1.5117731E0,2.1418726E-1,7.829523E-2,0E0,0E0,3.756063E-1,1.859641E-1,1.7485687E-1,1.3199472E-1,2.5919867E-1,9.491873E-2,0E0,7.4594593E-1,0E0,0E0,0E0,0E0,2.3865843E-1,1.2688246E-1,0E0,7.91924E-2,8.809562E-2,6.0019284E-2,0E0,4.4946156E-2,0E0,9.855425E-2,8.269191E-2,4.1137695E-2,0E0,1.3789654E-1,3.7922837E-2,2.6299787E-1,0E0,0E0,4.128723E-2,0E0,0E0,0E0,0E0,4.196185E-2,0E0,0E0,6.7264795E-2,6.604587E-2,0E0,5.9578896E-2,0E0,0E0,0E0,0E0,0E0,3.9812423E-2,1.1507964E-1,2.4912912E-1,0E0,0E0,0E0,0E0,0E0,4.7914505E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3006568E-1,1.5833747E-1,6.923527E-2,7.555705E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,29,29,30,30,32,32,33,33,34,34,36,36,38,38,39,39,40,40,42,42,43,43,44,44,47,47,52,52,55,55,56,56,58,58,64,64,65,65,66,66,72,72,79,79,80,80,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,36,38,40,-1,42,-1,-1,-1,-1,44,46,-1,48,50,52,-1,54,-1,56,58,60,-1,62,64,66,-1,-1,68,-1,-1,-1,-1,70,-1,-1,72,74,-1,76,-1,-1,-1,-1,-1,78,80,82,-1,-1,-1,-1,-1,84,-1,-1,-1,-1,-1,-1,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.41718E5,1.0429407E3,1.2852292E7,3.44E2,6.6512886E3,2.909019E3,1.44E2,8.425078E-4,8.464347E8,2.9222983E3,6.0891E4,5.0314346E1,2.5212732E4,3.1E1,-1.7760327E-2,-1.2409545E-2,2.889654E5,4.6288285E0,1.570662E5,1.5E2,4.56E2,1.619807E5,-9.579106E-3,2.2876814E1,-6.1339363E-3,1.2998829E-2,-3.3236064E-2,-1.1221248E-2,2.8101E5,4.392E3,-7.1630003E-3,1.6804138E1,2.4469028E7,3.6828573E8,1.3357296E-2,3.61E2,-4.7589215E-3,2.237815E7,3E0,9.2E1,2.1011192E-2,4.9E1,6.2581446E8,1.0864745E3,-1.7791794E-3,8.203853E-3,1.9937408E-1,-2.0000655E-3,-9.409242E-3,-3.634753E-4,-1.1811532E-4,1.03816045E9,5.9928386E-3,-1.3732224E-3,3.4E1,3.7860696E0,5.1353686E-3,1.0038E4,2.1663504E-2,4.5987293E-3,9.541472E-3,4.2965267E-2,-3.8816715E-3,4.45E3,2.73E2,2.695258E1,8.107463E-4,7.6636653E-3,6.5866155E-3,-8.845845E-4,1.3973573E-2,1.6283882E0,-8.119495E-4,6.869721E-3,1.6269138E-2,6.688087E-3,4.384359E-3,-1.7253119E-3,1.36E2,2.7434757E1,1.4040858E-2,1.8449864E6,6.8859295E-3,1.1825405E-2,-8.143927E-3,-1.358049E-2,-2.5548001E-3,-9.105189E-3,2.407192E-3,-2.9055818E-3,-9.103464E-3,7.222216E-4],"split_indices":[2,1,70,9,8,64,44,0,0,12,4,2,68,4,3,0,0,40,66,59,3,10,40,0,66,0,0,0,0,41,10,0,65,9,5,0,0,0,57,8,8,0,3,7,64,0,0,53,0,0,0,0,44,0,0,8,65,0,0,0,0,0,0,0,2,3,65,0,0,0,0,0,51,0,0,0,0,0,0,3,66,69,59,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.55E2,3.17E2,1.38E2,7.5E1,2.42E2,1.28E2,1E1,7.2E1,3E0,1.99E2,4.3E1,1E2,2.8E1,4E0,6E0,1.4E1,5.8E1,1.79E2,2E1,3E1,1.3E1,6.1E1,3.9E1,2E0,2.6E1,2E0,2E0,4E0,2E0,1.67E2,1.2E1,3E0,1.7E1,1E1,2E1,6E0,7E0,3E0,5.8E1,2.5E1,1.4E1,1.4E1,1.2E1,2E1,1.47E2,8E0,4E0,1.4E1,3E0,4E0,6E0,8E0,1.2E1,4E0,3E0,4.7E1,1.1E1,4E0,2.1E1,1.3E1,1E0,1E0,1.1E1,1E1,1E1,1.01E2,4.6E1,3E0,1.1E1,1E1,2E0,1.1E1,3.6E1,4E0,7E0,1.7E1,4E0,3E0,7E0,5.8E1,4.3E1,2.4E1,2.2E1,2.6E1,1E1,3.2E1,2.6E1,1.5E1,2.8E1,9E0,1.5E1,2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.0118642E-2,-1.1064205E-1,2.322225E-1,-1.482631E-1,4.0452495E-2,1.8081708E-1,6.41901E-1,-2.2425067E-1,-8.486963E-2,-1.196619E-2,1.6757792E-1,2.0487635E-1,-3.055771E-1,1.5708769E-2,3.44657E-2,-2.8851342E-1,-1.8747228E-1,-4.037244E-2,-1.4056477E-1,-3.4082413E-2,7.921726E-3,1.308292E-2,3.770976E-3,1.5031286E-1,3.1306732E-1,2.013823E-1,-2.7418746E-2,-1.6460095E-2,-8.2830265E-3,-5.230244E-3,-1.1324873E-2,-1.012855E-1,3.0685853E-2,-1.6689155E-1,-4.06461E-2,4.707947E-3,-6.107999E-2,7.198736E-2,2.287792E-1,2.9271256E-2,2.456573E-1,1.6307868E-2,-1.2114196E-3,-1.0273271E-2,-7.205077E-2,-6.565904E-2,1.1020335E-1,-1.9082592E-1,-5.261052E-2,-7.1322955E-3,-4.554945E-4,-4.349437E-3,7.450287E-4,-2.6296644E-2,1.0643607E-1,2.65676E-1,-1.0786434E-2,4.7467453E-3,2.6104298E-1,-1.8523576E-2,-6.2180366E-3,7.382024E-4,-5.1755747E-3,-6.4423727E-4,7.1909525E-3,-1.325472E-1,-2.4816407E-1,-8.191436E-3,1.5170552E-3,-4.2720055E-3,4.272681E-3,1.0401469E-2,6.859637E-2,3.1505397E-1,1.3129017E-1,6.9745984E-3,2.9433557E-1,-3.5113832E-3,1.2353559E-3,-1.5905786E-1,6.6267577E-4,-1.5944216E-2,-7.482292E-3,1.0424624E-3,6.803541E-3,6.2172273E-3,3.3959997E-1,-7.021423E-3,8.278387E-3,1.660147E-2,5.95235E-3,-2.9454117E-3,-1.0410603E-2,2.3336867E-3,1.7464057E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,-1,-1,-1,37,39,41,-1,-1,-1,-1,-1,43,45,47,49,-1,51,53,55,-1,57,-1,-1,-1,59,61,63,65,67,-1,-1,-1,-1,69,71,73,-1,-1,75,77,-1,-1,-1,-1,-1,79,81,-1,-1,-1,-1,-1,83,85,87,-1,89,-1,-1,91,-1,-1,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1880759E1,1.637402E0,3.1692448E0,1.0889049E0,3.8850266E-1,1.7009263E0,4.9096584E-2,1.9171381E-1,3.090663E-1,1.6375487E-1,1.2778053E-1,7.4398136E-1,9.730376E-1,0E0,0E0,1.2594533E-1,1.9990945E-1,3.1475127E-1,1.4052653E-1,1.3736403E-1,0E0,0E0,0E0,5.4280925E-1,6.6477203E-1,9.247157E-2,0E0,0E0,0E0,0E0,0E0,1.04082316E-1,2.677134E-1,1.1031127E-1,4.05384E-2,0E0,6.439856E-2,1.6192947E-1,7.955539E-1,0E0,6.0982227E-2,0E0,0E0,0E0,8.739738E-2,5.0305113E-2,8.063592E-2,8.740401E-2,8.808014E-2,0E0,0E0,0E0,0E0,9.331989E-2,1.20404035E-1,2.3733592E-1,0E0,0E0,9.2392206E-2,3.9770342E-2,0E0,0E0,0E0,0E0,0E0,7.9417884E-2,7.0846796E-2,0E0,0E0,0E0,0E0,0E0,8.421224E-2,9.81431E-2,1.4466262E-1,0E0,1.262598E-1,0E0,0E0,7.155603E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.8681965E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,23,23,24,24,25,25,31,31,32,32,33,33,34,34,36,36,37,37,38,38,40,40,44,44,45,45,46,46,47,47,48,48,53,53,54,54,55,55,58,58,59,59,65,65,66,66,72,72,73,73,74,74,76,76,79,79,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,-1,-1,-1,38,40,42,-1,-1,-1,-1,-1,44,46,48,50,-1,52,54,56,-1,58,-1,-1,-1,60,62,64,66,68,-1,-1,-1,-1,70,72,74,-1,-1,76,78,-1,-1,-1,-1,-1,80,82,-1,-1,-1,-1,-1,84,86,88,-1,90,-1,-1,92,-1,-1,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,8.625562E0,1.252737E6,1.5522031E9,2.3028242E3,4.6033485E1,1.86005E5,1.8071064E7,1E0,6.1614815E6,1.5271514E5,2.909019E3,1.5708769E-2,3.44657E-2,1.10388414E5,1.2921541E-2,2.753337E6,1.530655E3,1.4193897E-3,7.921726E-3,1.308292E-2,3.770976E-3,2.5096E4,5.695737E0,8.494E3,-2.7418746E-2,-1.6460095E-2,-8.2830265E-3,-5.230244E-3,-1.1324873E-2,4.5765094E2,5.16152E2,6.842986E6,6.844E3,4.707947E-3,6.426437E5,7.10711E8,7.79557E5,2.9271256E-2,1E0,1.6307868E-2,-1.2114196E-3,-1.0273271E-2,1.8518519E-2,7.012954E2,6.5610016E4,5.989E3,1.7895421E0,-7.1322955E-3,-4.554945E-4,-4.349437E-3,7.450287E-4,2.7385738E5,5.4629724E10,2.024909E6,-1.0786434E-2,4.7467453E-3,1.8805E4,1.533115E6,-6.2180366E-3,7.382024E-4,-5.1755747E-3,-6.4423727E-4,7.1909525E-3,1E0,9.3E1,-8.191436E-3,1.5170552E-3,-4.2720055E-3,4.272681E-3,1.0401469E-2,7.228959E6,5.0150156E0,1.379E3,6.9745984E-3,1.18138E5,-3.5113832E-3,1.2353559E-3,7.94203E5,6.6267577E-4,-1.5944216E-2,-7.482292E-3,1.0424624E-3,6.803541E-3,6.2172273E-3,1.1274355E7,-7.021423E-3,8.278387E-3,1.660147E-2,5.95235E-3,-2.9454117E-3,-1.0410603E-2,2.3336867E-3,1.7464057E-2],"split_indices":[2,7,47,1,7,70,68,9,57,101,57,40,44,0,0,40,51,1,64,51,0,0,0,2,66,0,0,0,0,0,0,64,64,41,2,0,59,7,2,0,102,0,0,0,69,67,40,2,65,0,0,0,0,45,43,9,0,0,2,1,0,0,0,0,0,24,8,0,0,0,0,0,1,66,0,0,10,0,0,41,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,2.86E2,1.55E2,2.29E2,5.7E1,1.39E2,1.6E1,1.03E2,1.26E2,4.1E1,1.6E1,1.33E2,6E0,3E0,1.3E1,3.5E1,6.8E1,7.1E1,5.5E1,3.7E1,4E0,7E0,9E0,9E1,4.3E1,2E0,4E0,2.5E1,1E1,2.3E1,4.5E1,3.8E1,3.3E1,4.3E1,1.2E1,6E0,3.1E1,4.6E1,4.4E1,7E0,3.6E1,1E0,1E0,7E0,3.1E1,1.5E1,1.8E1,3.5E1,8E0,2E0,1E1,2.3E1,8E0,1.2E1,3.4E1,4.1E1,3E0,4E0,3.2E1,1.6E1,1.5E1,5E0,1E1,4E0,1.4E1,1.9E1,1.6E1,3E0,5E0,8E0,4E0,8E0,2.6E1,2.9E1,1.2E1,8E0,2.4E1,7E0,9E0,1.6E1,3E0,8E0,8E0,1.6E1,1E1,4E0,2.5E1,1E0,1.1E1,1.9E1,5E0,6E0,1E1,1E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[1.647935E-2,-1.0800233E-1,2.5379223E-1,-2.305319E-1,-6.834447E-2,1.7691189E-1,4.700406E-1,-2.3805451E-1,8.408377E-4,-9.220398E-2,5.97228E-2,7.9566844E-2,2.8957498E-1,7.856261E-1,3.3412743E-1,-1.675159E-2,-2.134281E-1,-1.43772E-1,-4.6933446E-2,7.622511E-3,2.0938781E-1,1.0626647E-1,-2.4035897E-2,3.0569378E-1,-1.33056985E-2,1.8956881E-2,4.450752E-2,4.0694165E-1,-9.6769236E-2,-1.3231741E-3,-1.0981993E-2,-1.5539342E-1,4.789232E-4,5.154256E-3,-9.707956E-2,-3.586917E-2,4.927209E-3,1.5041862E-3,1.2809556E-2,4.5523564E-3,1.3712397E-1,2.086619E-1,3.5229716E-1,6.351443E-3,4.5419493E-1,-1.009842E-2,8.106471E-3,-3.5865575E-3,-1.705486E-1,-2.0502027E-2,8.810765E-2,-7.420468E-4,-1.1677388E-1,-4.1350573E-3,7.29116E-4,-1.1129401E-2,5.6643765E-2,1.5328002E-1,-3.5400363E-3,2.393879E-1,1.678819E-3,6.7270654E-3,3.7768197E-1,2.3491438E-2,3.4239031E-3,-1.9676779E-1,-1.1183504E-1,-3.1494997E-2,6.0926825E-3,8.684901E-3,1.1768502E-3,-3.648602E-3,-7.559087E-3,6.416758E-3,-2.4222042E-3,5.6535593E-4,1.6364478E-1,1.3668489E-2,5.4793707E-3,4.1697806E-1,1.1115056E-2,-2.1669492E-1,-2.8803984E-3,-2.3783368E-1,-4.184603E-2,-9.468447E-2,7.942482E-4,2.3164696E-1,1.2622789E-1,4.683328E-3,2.1385228E-2,-1.2016297E-2,-5.5386415E-3,-1.3395162E-2,-6.609183E-4,-3.8383205E-3,1.4601192E-3,-1.0110219E-3,-6.863419E-3,3.8866235E-3,1.3495201E-2,3.6749647E-3,8.345667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,41,-1,-1,-1,43,45,-1,-1,47,-1,49,51,53,-1,-1,-1,55,57,59,61,-1,63,-1,-1,-1,65,67,69,-1,71,-1,-1,-1,73,75,-1,77,-1,-1,79,-1,-1,81,83,85,-1,-1,-1,-1,-1,-1,-1,-1,87,-1,-1,89,-1,91,-1,93,95,97,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2966414E1,1.3837118E0,2.4272566E0,1.3425922E-1,6.746881E-1,1.2162526E0,1.4102974E0,9.112668E-2,0E0,4.2513692E-1,2.713196E-1,9.7897196E-1,5.472603E-1,1.312499E-1,9.492667E-1,0E0,6.1496258E-2,1.5475857E-1,2.6169991E-1,1.1035886E-1,6.800538E-2,1.8769431E-1,0E0,1.5711212E-1,0E0,0E0,0E0,2.7287483E-1,1.6912194E-1,0E0,0E0,8.9152455E-2,0E0,1.08220175E-1,8.0860406E-2,4.6143394E-2,0E0,0E0,0E0,1.9017844E-1,1.6540784E-1,9.416282E-2,1.315217E-1,0E0,9.6241E-2,0E0,0E0,0E0,8.113575E-2,6.489619E-2,6.228009E-2,0E0,5.0830126E-2,0E0,0E0,0E0,1.04269184E-1,6.1421514E-2,0E0,4.0830314E-2,0E0,0E0,8.420944E-2,0E0,0E0,1.12273335E-1,1.9287357E-1,1.1229738E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.2536945E-2,0E0,0E0,3.7641525E-2,0E0,7.13979E-2,0E0,5.0716847E-2,4.1990094E-2,4.8028886E-2,0E0,5.5787623E-2,4.7866046E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,23,23,27,27,28,28,31,31,33,33,34,34,35,35,39,39,40,40,41,41,42,42,44,44,48,48,49,49,50,50,52,52,56,56,57,57,59,59,62,62,65,65,66,66,67,67,76,76,79,79,81,81,83,83,84,84,85,85,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,42,-1,-1,-1,44,46,-1,-1,48,-1,50,52,54,-1,-1,-1,56,58,60,62,-1,64,-1,-1,-1,66,68,70,-1,72,-1,-1,-1,74,76,-1,78,-1,-1,80,-1,-1,82,84,86,-1,-1,-1,-1,-1,-1,-1,-1,88,-1,-1,90,-1,92,-1,94,96,98,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.32424E5,5.932393E3,1.2852292E7,2.76E2,1.0693399E-1,1.417988E6,1.44E2,8.408377E-4,7.621583E2,1.2103396E5,3.764913E7,2.8756912E8,9.256843E3,1.5096262E7,-1.675159E-2,2.586E4,1.348197E2,2.1692945E1,1.4096699E9,3.636015E8,4.7051765E1,-2.4035897E-2,5.2060513E9,-1.33056985E-2,1.8956881E-2,4.450752E-2,2.3193505E0,3.677E3,-1.3231741E-3,-1.0981993E-2,5.1698097E1,4.789232E-4,2.473E3,1.32E2,1.8374788E7,4.927209E-3,1.5041862E-3,1.2809556E-2,2.8884522E8,4.2663252E7,2.237815E7,7.815434E4,6.351443E-3,2.1030048E10,-1.009842E-2,8.106471E-3,-3.5865575E-3,1.646359E5,2.2860639E9,7.494142E6,-7.420468E-4,1.70386E7,-4.1350573E-3,7.29116E-4,-1.1129401E-2,4.3603E6,2.8593372E3,-3.5400363E-3,5.31972E7,1.678819E-3,6.7270654E-3,1.5508488E0,2.3491438E-2,3.4239031E-3,1.958583E3,1.763213E-5,1.1016051E3,6.0926825E-3,8.684901E-3,1.1768502E-3,-3.648602E-3,-7.559087E-3,6.416758E-3,-2.4222042E-3,5.6535593E-4,7.2123305E6,1.3668489E-2,5.4793707E-3,2.640416E-2,1.1115056E-2,6.623914E4,-2.8803984E-3,2.1933604E3,2.135E3,7.1303144E7,7.942482E-4,3.44E2,4.729687E3,4.683328E-3,2.1385228E-2,-1.2016297E-2,-5.5386415E-3,-1.3395162E-2,-6.609183E-4,-3.8383205E-3,1.4601192E-3,-1.0110219E-3,-6.863419E-3,3.8866235E-3,1.3495201E-2,3.6749647E-3,8.345667E-3],"split_indices":[2,1,64,9,8,47,41,0,0,64,45,60,1,4,9,0,41,66,65,7,5,68,0,12,0,0,0,65,0,0,0,60,0,0,3,12,0,0,0,12,57,57,40,0,5,0,0,0,40,12,1,0,57,0,0,0,60,67,0,56,0,0,46,0,0,4,50,64,0,0,0,0,0,0,0,0,57,0,0,50,0,45,0,44,0,7,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.87E2,1.5E2,6.9E1,2.18E2,1.12E2,3.8E1,6.7E1,2E0,1.84E2,3.4E1,6.1E1,5.1E1,1E1,2.8E1,1.1E1,5.6E1,8.5E1,9.9E1,2.6E1,8E0,5.9E1,2E0,5E1,1E0,3E0,7E0,2.4E1,4E0,2E0,5.4E1,7.9E1,6E0,4.9E1,5E1,1.8E1,8E0,2E0,6E0,1.4E1,4.5E1,1.8E1,3.2E1,4E0,2E1,3E0,1E0,1.3E1,6.6E1,3.8E1,1.1E1,1E1,4E1,9E0,9E0,2E0,1.2E1,4.2E1,3E0,1.5E1,3E0,4E0,2.8E1,1.9E1,1E0,4.4E1,2.2E1,3.6E1,2E0,4E0,7E0,1.9E1,2.1E1,7E0,5E0,3E0,3.9E1,1.1E1,4E0,2.1E1,7E0,3.8E1,6E0,7E0,1.5E1,1.5E1,2.1E1,1.2E1,2.7E1,1E0,2E1,3E1,8E0,6E0,1E0,1E1,5E0,6E0,9E0,3E0,9E0,1.3E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[8.508928E-3,-1.1184809E-1,2.2897916E-1,-2.291963E-1,-6.3077055E-2,1.6880473E-1,4.8460573E-1,-2.4334742E-1,-2.038602E-3,-6.2371246E-3,-1.1985523E-1,1.8931116E-1,-2.7629325E-1,3.8017595E-1,4.1058216E-2,-2.5463197E-1,-4.7040437E-3,-3.608007E-2,1.2602837E-1,-2.3302774E-1,-1.0343384E-1,2.854015E-1,1.2944797E-1,4.267786E-3,-2.3912922E-2,4.0902588E-1,-7.080921E-3,-4.823212E-3,-1.31253E-2,-7.333689E-2,6.592007E-2,-4.903948E-3,1.7419016E-1,-2.8294215E-1,-3.1216764E-3,-1.1915602E-1,-2.3271507E-2,1.4631501E-2,3.256512E-1,1.944744E-1,3.1927224E-2,6.954831E-3,2.1394776E-2,-1.19564235E-1,-2.1494145E-3,-4.018017E-3,1.22291364E-1,2.1949092E-1,1.5307837E-3,-1.5577815E-2,-1.3202757E-3,-1.5962383E-1,-5.9789155E-2,-3.3478504E-3,2.124315E-3,-3.935328E-3,3.3849124E-3,3.3129446E-2,2.8038985E-1,1.4749454E-1,2.9132158E-1,-1.124878E-2,7.6681696E-2,-1.3365248E-1,1.1796481E-3,1.0378074E-2,2.8410386E-3,1.3134291E-2,3.3557345E-3,-1.3175184E-1,-1.4399387E-2,-9.465883E-2,-7.717996E-5,2.959239E-1,-6.967557E-3,9.332888E-2,1.1177012E-2,1.6353047E-2,-4.3810206E-3,1.05758764E-1,-1.0039201E-2,-1.2337245E-3,-8.208533E-3,2.2882062E-3,-1.4288688E-1,-2.4354638E-4,-6.636338E-3,9.600813E-3,1.6983235E-2,6.6534174E-3,1.2726691E-3,-2.7975924E-3,1.2928881E-1,-3.0124153E-3,-8.612954E-3,4.538085E-3,1.0209081E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,-1,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,43,45,-1,47,49,-1,51,53,55,57,59,61,-1,-1,63,-1,-1,65,67,-1,-1,-1,69,71,-1,-1,-1,-1,-1,73,75,77,-1,79,81,-1,-1,-1,-1,-1,83,-1,85,-1,87,-1,89,-1,-1,-1,91,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,95,-1,-1,-1,-1],"loss_changes":[1.1648003E1,1.61093E0,2.302435E0,2.1136284E-1,6.511606E-1,1.2114811E0,7.081089E-1,1.0569286E-1,0E0,4.0716067E-1,1.6360056E-1,6.648326E-1,4.7965032E-1,4.192896E-1,0E0,6.446266E-2,0E0,3.2405126E-1,2.2221032E-1,8.448988E-2,1.1065638E-1,4.9656224E-1,4.820807E-1,0E0,0E0,5.8816195E-2,0E0,0E0,0E0,8.17807E-2,1.9954029E-1,0E0,9.733081E-2,5.817139E-2,0E0,1.7063987E-1,4.885801E-2,4.0040597E-2,4.2471218E-1,1.6632533E-1,3.8533562E-1,0E0,0E0,5.1562577E-2,0E0,0E0,8.3171636E-2,6.154865E-2,0E0,0E0,0E0,1.19629264E-1,6.484511E-2,0E0,0E0,0E0,0E0,0E0,2.825458E-1,1.1524761E-1,2.1914017E-1,0E0,2.4722399E-1,6.929353E-2,0E0,0E0,0E0,0E0,0E0,8.1664145E-2,0E0,6.782594E-2,0E0,9.879923E-2,0E0,5.333437E-2,0E0,0E0,0E0,1.0618526E-1,0E0,0E0,0E0,0E0,7.720834E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.782632E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,43,43,46,46,47,47,51,51,52,52,58,58,59,59,60,60,62,62,63,63,69,69,71,71,73,73,75,75,79,79,84,84,92,92],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,-1,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,44,46,-1,48,50,-1,52,54,56,58,60,62,-1,-1,64,-1,-1,66,68,-1,-1,-1,70,72,-1,-1,-1,-1,-1,74,76,78,-1,80,82,-1,-1,-1,-1,-1,84,-1,86,-1,88,-1,90,-1,-1,-1,92,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,96,-1,-1,-1,-1],"split_conditions":[1.0262E4,9.04782E5,4.026531E9,1.2852292E7,1.4074289E7,9.533205E6,5.01E2,2.4752695E6,-2.038602E-3,9.481747E8,1.2808544E3,1.30569E5,1.618E3,2.617838E8,4.1058216E-2,7.547656E1,-4.7040437E-3,2.972948E6,1.5560601E6,4.241E3,4.723148E11,5.274031E6,1.6285607E7,4.267786E-3,-2.3912922E-2,3.9541063E0,-7.080921E-3,-4.823212E-3,-1.31253E-2,4.827E3,3E1,-4.903948E-3,1.5E2,2.4927816E1,-3.1216764E-3,7.194E3,5.767668E0,1.153215E9,4.7431055E-2,5.6503E4,2.3808822E-2,6.954831E-3,2.1394776E-2,3.53E2,-2.1494145E-3,-4.018017E-3,1.03E2,5.88824E7,1.5307837E-3,-1.5577815E-2,-1.3202757E-3,6.746E3,1.0821084E-2,-3.3478504E-3,2.124315E-3,-3.935328E-3,3.3849124E-3,3.3129446E-2,1.505649E6,5.841568E1,1.2622761E8,-1.124878E-2,4.31E2,5.428232E9,1.1796481E-3,1.0378074E-2,2.8410386E-3,1.3134291E-2,3.3557345E-3,3.904E3,-1.4399387E-2,1.83918E5,-7.717996E-5,5.0491E4,-6.967557E-3,7.0217915E-2,1.1177012E-2,1.6353047E-2,-4.3810206E-3,1.5165479E0,-1.0039201E-2,-1.2337245E-3,-8.208533E-3,2.2882062E-3,7.058968E0,-2.4354638E-4,-6.636338E-3,9.600813E-3,1.6983235E-2,6.6534174E-3,1.2726691E-3,-2.7975924E-3,3.5940964E7,-3.0124153E-3,-8.612954E-3,4.538085E-3,1.0209081E-2],"split_indices":[2,1,7,9,57,59,3,62,0,7,4,9,0,1,0,66,0,1,57,2,43,1,57,0,0,66,0,0,0,2,8,0,3,70,0,2,66,7,50,2,50,0,0,3,0,0,3,9,0,0,0,2,50,0,0,0,0,0,2,68,1,0,3,43,0,0,0,0,0,2,0,12,0,2,0,69,0,0,0,65,0,0,0,0,70,0,0,0,0,0,0,0,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.83E2,1.54E2,8.2E1,2.01E2,1.26E2,2.8E1,7.6E1,6E0,1.01E2,1E2,1.21E2,5E0,2.3E1,5E0,7E1,6E0,8.3E1,1.8E1,1.1E1,8.9E1,4.5E1,7.6E1,2E0,3E0,2.2E1,1E0,4E0,6.6E1,6.1E1,2.2E1,3E0,1.5E1,8E0,3E0,7.4E1,1.5E1,6E0,3.9E1,4.5E1,3.1E1,2E0,2E1,2.3E1,3.8E1,6E0,1.6E1,1.1E1,4E0,7E0,1E0,4.3E1,3.1E1,9E0,6E0,2E0,4E0,3E0,3.6E1,3.2E1,1.3E1,4E0,2.7E1,2.1E1,2E0,6E0,1E1,8E0,3E0,3.7E1,6E0,1.9E1,1.2E1,3.5E1,1E0,2E1,1.2E1,1.2E1,1E0,2.5E1,2E0,5E0,1.6E1,2E0,3.5E1,6E0,1.3E1,1.2E1,2.3E1,1.2E1,8E0,3E0,2.2E1,1E1,2.5E1,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[1.2828724E-3,-9.232781E-2,2.5336355E-1,-1.9655019E-1,-3.796487E-2,1.7874552E-1,5.5727774E-1,-2.3793441E-1,-9.008674E-2,-6.471515E-2,1.24768384E-1,1.9975714E-1,-2.5897043E-2,2.9595125E-1,7.916996E-1,-2.5218174E-1,-3.1737727E-3,-1.3411176E-1,-9.4018725E-4,-9.093448E-2,5.529416E-2,-4.184684E-3,1.5204684E-1,2.1040517E-1,-9.131623E-3,1.6911719E-2,-4.271548E-3,1.642885E-2,4.2882215E-2,-2.665827E-1,-5.4499703E-3,-7.0190924E-4,-7.7835997E-3,-1.3793243E-1,-5.1611952E-2,-3.6615107E-2,9.873542E-2,1.0650064E-2,7.4597925E-2,1.4914721E-1,2.8171748E-1,-1.348688E-2,-1.6557266E-3,-2.364135E-1,-1.1788345E-1,8.2582384E-2,-7.196232E-2,2.727557E-3,-3.879863E-3,7.5293533E-3,1.0675424E-3,4.7237566E-3,-4.109812E-3,9.992521E-2,2.0725133E-1,1.7489443E-2,2.1606466E-1,-1.3125584E-2,-9.973045E-6,-1.3885228E-1,-9.734801E-4,-4.5183612E-4,6.8082525E-3,-4.1797828E-2,-1.1078838E-1,-1.6454906E-3,1.1652052E-1,3.5269302E-3,1.125051E-2,4.897515E-3,2.7250084E-1,-4.904379E-4,-1.5090805E-1,-8.380902E-2,6.5043746E-3,-8.0856174E-2,-1.9758618E-1,3.1698558E-3,7.871149E-3,8.35037E-4,1.4471986E-2,-5.5293133E-3,-1.0795636E-2,1.1080916E-3,-5.0830627E-3,2.8360258E-3,-3.0590051E-3,-4.7098952E-3,2.2122338E-3,-1.24630565E-2,-2.2689353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,-1,33,35,-1,37,39,-1,-1,-1,-1,-1,41,-1,-1,-1,43,45,47,49,-1,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,-1,-1,-1,65,67,-1,69,-1,-1,71,-1,-1,-1,73,75,-1,77,-1,-1,-1,79,-1,81,83,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.934319E0,1.7319491E0,2.485621E0,4.4118547E-1,8.9176595E-1,1.4653776E0,1.1134295E0,1.7037058E-1,9.475002E-2,5.556767E-1,1.7767233E-1,4.090538E-1,0E0,2.2319734E-1,5.585003E-2,1.13152504E-1,0E0,4.716721E-2,0E0,2.5900435E-1,1.3138016E-1,0E0,1.0705048E-1,3.467741E-1,0E0,0E0,0E0,0E0,0E0,3.8591385E-2,0E0,0E0,0E0,1.0046971E-1,2.2441514E-1,4.5384172E-2,8.516815E-2,0E0,4.8391476E-2,1.2229049E-1,1.03589535E-1,0E0,0E0,6.1891854E-2,1.1284727E-1,5.5301048E-2,7.735106E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.7772776E-2,3.7836492E-2,0E0,1.2655127E-1,0E0,0E0,7.0079505E-2,0E0,0E0,0E0,8.374367E-2,6.1965466E-2,0E0,4.6576828E-2,0E0,0E0,0E0,6.0685396E-2,0E0,8.621687E-2,4.3992713E-2,7.1516715E-2,4.4371396E-2,4.3854415E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,17,17,19,19,20,20,22,22,23,23,29,29,33,33,34,34,35,35,36,36,38,38,39,39,40,40,43,43,44,44,45,45,46,46,53,53,54,54,56,56,59,59,63,63,64,64,66,66,70,70,72,72,73,73,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,-1,34,36,-1,38,40,-1,-1,-1,-1,-1,42,-1,-1,-1,44,46,48,50,-1,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,-1,-1,-1,66,68,-1,70,-1,-1,72,-1,-1,-1,74,76,-1,78,-1,-1,-1,80,-1,82,84,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,1.074952E6,8.10288E3,3.006846E6,1.2678202E9,5.0328052E7,7.4065337E0,1.7212875E5,7.66951E5,5.3008755E8,1.7102936E6,1.505649E6,-2.5897043E-2,1.354316E6,1E1,1.7626338E6,-3.1737727E-3,2E1,-9.4018725E-4,7.283496E2,1.0643513E3,-4.184684E-3,6.3349745E6,6.0891E4,-9.131623E-3,1.6911719E-2,-4.271548E-3,1.642885E-2,4.2882215E-2,8.332E3,-5.4499703E-3,-7.0190924E-4,-7.7835997E-3,6.506511E0,2.823537E1,1.4010049E6,1.03E2,1.0650064E-2,4.264618E6,1.5271514E5,1.423795E7,-1.348688E-2,-1.6557266E-3,1.154375E6,2.1592189E5,5.0563E4,2.6402641E-2,2.727557E-3,-3.879863E-3,7.5293533E-3,1.0675424E-3,4.7237566E-3,-4.109812E-3,3.7860696E0,8.447218E4,1.7489443E-2,7.6559805E3,-1.3125584E-2,-9.973045E-6,1.4590734E8,-9.734801E-4,-4.5183612E-4,6.8082525E-3,6.746E3,1E0,-1.6454906E-3,2.0663544E9,3.5269302E-3,1.125051E-2,4.897515E-3,1.4817759E9,-4.904379E-4,9E1,4.3712845E0,2.9991518E7,3.53E2,2.55E2,3.1698558E-3,7.871149E-3,8.35037E-4,1.4471986E-2,-5.5293133E-3,-1.0795636E-2,1.1080916E-3,-5.0830627E-3,2.8360258E-3,-3.0590051E-3,-4.7098952E-3,2.2122338E-3,-1.24630565E-2,-2.2689353E-3],"split_indices":[2,1,64,9,7,57,65,40,1,12,57,2,0,2,8,62,0,10,0,64,64,0,57,2,0,0,0,0,0,2,0,0,0,68,68,59,3,0,1,40,57,0,0,9,40,12,69,0,0,0,0,0,0,65,45,0,4,0,0,5,0,0,0,2,19,0,12,0,0,0,7,0,8,70,57,8,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,3.06E2,1.13E2,1.04E2,2.02E2,9.2E1,2.1E1,7.4E1,3E1,1.74E2,2.8E1,9E1,2E0,1.1E1,1E1,6.8E1,6E0,1.8E1,1.2E1,1.43E2,3.1E1,3E0,2.5E1,8.8E1,2E0,1E1,1E0,2E0,8E0,6.1E1,7E0,3E0,1.5E1,6.4E1,7.9E1,1E1,2.1E1,1.3E1,1.2E1,4.9E1,3.9E1,6E1,1E0,9E0,5.5E1,1E1,6.9E1,3E0,7E0,1.2E1,9E0,1.1E1,1E0,2.8E1,2.1E1,1.7E1,2.2E1,8E0,1E0,4.5E1,1E1,4E0,6E0,4E1,2.9E1,3E0,2.5E1,3E0,1.8E1,8E0,1.4E1,4E0,4.1E1,2.1E1,1.9E1,2.3E1,6E0,1.2E1,1.3E1,1E0,1.3E1,2.7E1,1.4E1,3E0,1.8E1,1.1E1,8E0,2.1E1,2E0,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.2343949E-2,-1.0123858E-1,2.2944875E-1,-1.2859523E-1,4.2503256E-2,1.5653016E-1,4.9960825E-1,-1.9947056E-1,-8.256784E-2,-2.3097374E-2,9.853168E-2,1.8036947E-1,-4.1831186E-1,3.912851E-2,3.274031E-1,-3.314055E-3,-2.1294802E-1,-1.3085209E-1,-5.0173968E-2,1.7185119E-3,-5.494903E-3,1.735962E-1,1.3720532E-4,1.1557563E-1,2.477742E-1,3.7989735E-3,-2.803147E-2,3.901973E-1,-8.967108E-3,-2.298753E-1,-5.9850146E-3,-9.936132E-2,-1.8160433E-1,-7.034077E-2,8.5091375E-2,6.073988E-4,1.0820875E-2,1.4323488E-1,-3.704415E-2,-1.433982E-2,2.6317194E-1,1.3379053E-2,2.4815617E-2,-2.4041462E-1,-2.2379875E-3,-1.4083368E-1,-1.3184419E-3,-2.5154254E-1,-9.515663E-2,-3.1342376E-2,-1.0660568E-1,6.3773454E-3,-1.5914246E-3,1.7366499E-1,8.738497E-2,-5.3142677E-3,1.3430204E-3,5.240277E-3,-9.293712E-3,1.6014807E-1,3.0863643E-1,-1.3157342E-2,-1.6483335E-1,-1.5932426E-1,-1.9343072E-3,-1.3826277E-2,-9.724315E-6,-7.7204104E-3,3.463515E-4,-5.388501E-4,-4.9315607E-3,-1.333763E-1,-3.302618E-2,2.0928809E-1,5.272545E-3,1.1697024E-3,7.447644E-3,-1.957076E-6,9.004388E-3,1.8096011E-2,8.795927E-3,-9.159422E-3,-5.761027E-4,-9.882032E-3,-3.8477043E-3,-2.6160264E-3,-1.5677947E-1,-6.510743E-3,-2.7400258E-5,1.1700692E-2,4.395383E-3,-3.464542E-3,-9.643906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,31,33,-1,-1,35,-1,37,39,-1,-1,41,-1,43,-1,45,47,49,51,-1,-1,53,55,57,59,-1,-1,61,-1,63,-1,65,67,69,71,-1,-1,73,75,-1,-1,-1,-1,77,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,85,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0626734E1,1.1183054E0,2.8373446E0,7.599356E-1,1.7134881E-1,1.6925025E0,1.2562141E0,1.5277934E-1,2.2179526E-1,1.13619775E-1,1.7927718E-1,4.6924853E-1,3.9505786E-1,0E0,7.3835945E-1,0E0,1.0497713E-1,7.465273E-2,2.4876572E-1,0E0,0E0,9.4386935E-2,0E0,2.58587E-1,2.2575164E-1,0E0,0E0,1.1458111E-1,0E0,1.2578773E-1,0E0,1.1129719E-1,1.03030026E-1,1.0697201E-1,6.3723646E-2,0E0,0E0,7.1499705E-2,4.7147293E-2,1.01326235E-1,1.9413662E-1,0E0,0E0,7.0628405E-2,0E0,3.9155602E-2,0E0,6.938571E-2,6.769054E-2,5.2929338E-2,7.438594E-2,0E0,0E0,5.562228E-2,7.538524E-2,0E0,0E0,0E0,0E0,5.7749033E-2,1.844275E-1,0E0,4.8780113E-2,4.8026264E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6788156E-2,3.783068E-2,3.9684117E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.6254685E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,17,17,18,18,21,21,23,23,24,24,27,27,29,29,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,43,43,45,45,47,47,48,48,49,49,50,50,53,53,54,54,59,59,60,60,62,62,63,63,71,71,72,72,73,73,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,32,34,-1,-1,36,-1,38,40,-1,-1,42,-1,44,-1,46,48,50,52,-1,-1,54,56,58,60,-1,-1,62,-1,64,-1,66,68,70,72,-1,-1,74,76,-1,-1,-1,-1,78,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,86,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.164074E8,7.7101436E3,1.129877E6,2.1530056E3,9.533205E6,3.0418E4,3.5454545E0,7.283496E2,9.481747E8,1.5E2,4.1331047E-1,1.563E3,3.912851E-2,1.852084E7,-3.314055E-3,1.6395413E7,1.7377544E7,7.1999524E9,1.7185119E-3,-5.494903E-3,1.01685606E9,1.3720532E-4,2.5916522E7,2.2782404E0,3.7989735E-3,-2.803147E-2,1.2307312E5,-8.967108E-3,1.7212875E5,-5.9850146E-3,3.345E3,2.56445E7,1.70386E7,1.6991062E3,6.073988E-4,1.0820875E-2,1.921711E5,1.2538845E9,2.203418E7,1.6441814E9,1.3379053E-2,2.4815617E-2,3.006846E6,-2.2379875E-3,5.79E3,-1.3184419E-3,1.5592E4,1E0,9.9378884E-2,1.530655E3,6.3773454E-3,-1.5914246E-3,1.14E2,4.3018994E3,-5.3142677E-3,1.3430204E-3,5.240277E-3,-9.293712E-3,2.8372517E3,9.83854E-1,-1.3157342E-2,2.19546E5,1.926778E6,-1.9343072E-3,-1.3826277E-2,-9.724315E-6,-7.7204104E-3,3.463515E-4,-5.388501E-4,-4.9315607E-3,1.89285E3,6.844E3,4.316273E7,5.272545E-3,1.1697024E-3,7.447644E-3,-1.957076E-6,9.004388E-3,1.8096011E-2,8.795927E-3,-9.159422E-3,-5.761027E-4,-9.882032E-3,-3.8477043E-3,-2.6160264E-3,4.90403E5,-6.510743E-3,-2.7400258E-5,1.1700692E-2,4.395383E-3,-3.464542E-3,-9.643906E-3],"split_indices":[2,7,64,1,4,59,10,70,64,7,3,47,0,0,9,0,12,57,5,0,0,7,0,57,66,0,0,40,0,40,0,10,57,57,45,0,0,45,7,1,7,0,0,9,0,2,0,0,13,69,64,0,0,3,4,0,0,0,0,64,46,0,41,41,0,0,0,0,0,0,0,4,2,1,0,0,0,0,0,0,0,0,0,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,2.82E2,1.47E2,2.37E2,4.5E1,1.17E2,3E1,9.2E1,1.45E2,2.1E1,2.4E1,1.13E2,4E0,1E1,2E1,9E0,8.3E1,5.7E1,8.8E1,1.3E1,8E0,1.3E1,1.1E1,5.9E1,5.4E1,1E0,3E0,1.8E1,2E0,6.9E1,1.4E1,3.7E1,2E1,7.7E1,1.1E1,3E0,1E1,5E1,9E0,3E0,5.1E1,1E1,8E0,6.5E1,4E0,2.3E1,1.4E1,1E1,1E1,3.8E1,3.9E1,8E0,3E0,3.1E1,1.9E1,4E0,5E0,2E0,1E0,1.7E1,3.4E1,4.8E1,1.7E1,1.9E1,4E0,9E0,1E0,6E0,4E0,3E1,8E0,2.8E1,1.1E1,1.9E1,1.2E1,1E1,9E0,2E0,1.5E1,2.3E1,1.1E1,1.5E1,2E0,1.2E1,7E0,7E0,2.1E1,2E0,9E0,1.5E1,4E0,7E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[6.8671815E-4,-1.1392014E-1,1.752277E-1,-1.0517078E-1,-2.7379524E-2,8.527372E-2,3.4911638E-1,-1.38489E-1,1.3830686E-2,4.7940973E-2,9.530251E-3,3.7010703E-2,2.8217924E-1,-1.4519805E-1,3.5941693E-1,-7.2818124E-3,3.163048E-2,9.156724E-3,1.4936809E-1,3.0387342E-1,-8.668348E-3,-2.0726171E-1,-1.0365096E-1,2.316739E-2,3.7759875E-3,-3.2205682E-2,9.4295144E-2,-2.772771E-2,5.2453587E-3,7.396025E-2,1.0369311E-2,2.5295886E-1,3.8033488E-1,-2.2931817E-1,-1.2437948E-1,-1.16077416E-1,1.6704737E-3,-5.790598E-2,3.3682056E-3,1.0766968E-1,-6.3330387E-3,-6.5123383E-3,-1.2413044E-2,6.0656457E-3,3.1590098E-4,2.1158643E-3,2.6518163E-1,3.9501175E-1,3.2691362E-3,-2.4025877E-1,-2.8258248E-3,-2.2461326E-3,-7.875466E-3,-1.4267032E-1,-3.4130048E-2,2.4808235E-3,-5.064492E-3,1.2568059E-2,7.797764E-2,2.381576E-4,-5.8111264E-3,5.4056407E-4,2.732999E-1,3.967589E-3,2.0479014E-2,-1.5862577E-2,-1.0293394E-2,-1.02868654E-1,-1.828753E-1,2.0434083E-2,-5.140428E-3,7.79772E-3,1.2967946E-3,1.4398391E-2,5.2493038E-3,-5.712038E-3,5.6995533E-4,-9.605883E-3,-2.8544913E-5,-3.4411408E-3,3.7680592E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,29,31,-1,33,35,-1,-1,37,39,41,-1,43,-1,45,47,49,51,53,-1,55,-1,57,-1,-1,59,-1,-1,-1,61,63,-1,65,-1,-1,-1,67,69,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,-1,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.781535E0,9.484887E-1,2.6919336E0,1.0367582E0,0E0,4.4740057E-1,1.3369851E0,7.336445E-1,1.6931248E-1,3.4087068E-1,0E0,0E0,5.6670046E-1,4.9938726E-1,5.3669184E-2,0E0,2.1505125E-1,2.2905762E-1,8.983445E-2,9.84416E-2,0E0,1.1772442E-1,2.13395E-1,0E0,0E0,7.301731E-2,9.3455866E-2,7.221945E-2,0E0,3.7714683E-2,0E0,7.372713E-2,5.735755E-2,1.0361099E-1,4.3033153E-2,2.4138224E-1,0E0,1.0779208E-1,0E0,9.141177E-2,0E0,0E0,7.549237E-2,0E0,0E0,0E0,6.105876E-2,4.668808E-2,0E0,8.8063E-2,0E0,0E0,0E0,1.1720693E-1,1.1085376E-1,0E0,0E0,0E0,8.935343E-2,0E0,0E0,0E0,4.447651E-2,0E0,0E0,0E0,0E0,6.0063362E-2,6.9658995E-2,8.862161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,39,39,42,42,46,46,47,47,49,49,53,53,54,54,58,58,62,62,67,67,68,68,69,69],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,30,32,-1,34,36,-1,-1,38,40,42,-1,44,-1,46,48,50,52,54,-1,56,-1,58,-1,-1,60,-1,-1,-1,62,64,-1,66,-1,-1,-1,68,70,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,-1,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,6.0891E4,7.164074E8,-2.7379524E-2,1.5271514E5,3.0418E4,5.2191963E3,2.9648642E1,1.29186E9,9.530251E-3,3.7010703E-2,5.22E2,9.50639E5,1.906E3,-7.2818124E-3,1.577058E9,2.143564E3,1.0339844E-1,1.6846506E5,-8.668348E-3,3.006846E6,2.889654E5,2.316739E-2,3.7759875E-3,3.0251338E5,6.6053805E6,1.7895421E0,5.2453587E-3,1.6255458E5,1.0369311E-2,1.7648785E-1,5.88824E7,1.6120845E5,1.7062635E3,1.0623282E3,1.6704737E-3,8.110162E0,3.3682056E-3,2.412E4,-6.3330387E-3,-6.5123383E-3,6.855442E1,6.0656457E-3,3.1590098E-4,2.1158643E-3,1.926778E6,1.1100566E3,3.2691362E-3,3.586E3,-2.8258248E-3,-2.2461326E-3,-7.875466E-3,1.7377544E7,6.5E1,2.4808235E-3,-5.064492E-3,1.2568059E-2,3.5546432E6,2.381576E-4,-5.8111264E-3,5.4056407E-4,4.242988E2,3.967589E-3,2.0479014E-2,-1.5862577E-2,-1.0293394E-2,3.8E2,2.5252392E6,6.12E2,-5.140428E-3,7.79772E-3,1.2967946E-3,1.4398391E-2,5.2493038E-3,-5.712038E-3,5.6995533E-4,-9.605883E-3,-2.8544913E-5,-3.4411408E-3,3.7680592E-3],"split_indices":[45,59,2,7,0,40,10,4,68,7,0,0,3,1,0,0,7,64,50,40,0,9,40,0,0,59,59,65,0,45,0,53,9,40,59,64,0,70,0,9,0,0,68,0,0,0,41,4,0,2,0,0,0,57,8,0,0,0,57,0,0,0,69,0,0,0,0,3,60,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.64E2,1.73E2,2.6E2,4E0,1.15E2,5.8E1,2.03E2,5.7E1,8.6E1,2.9E1,7E0,5.1E1,2.01E2,2E0,5E0,5.2E1,6.3E1,2.3E1,4.9E1,2E0,7.9E1,1.22E2,1E0,1E0,2.6E1,2.6E1,4.6E1,1.7E1,1.1E1,1.2E1,3.2E1,1.7E1,6.1E1,1.8E1,1.12E2,1E1,2.1E1,5E0,2.5E1,1E0,5E0,4.1E1,6E0,5E0,2E0,3E1,1.6E1,1E0,5.7E1,4E0,6E0,1.2E1,8.4E1,2.8E1,6E0,1.5E1,3E0,2.2E1,3.6E1,5E0,1E0,2.9E1,1E0,1.5E1,1.5E1,4.2E1,4.4E1,4E1,1.6E1,1.2E1,8E0,1.4E1,2.6E1,3E0,4E1,4E0,3.8E1,2E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[6.4035733E-3,-8.937738E-2,1.9856532E-1,-1.9758534E-1,-5.7276003E-2,1.3598657E-1,4.528227E-1,-1.0094103E-2,2.418048E-3,-9.565375E-3,-1.0250067E-1,1.559042E-1,-2.1065827E-1,3.6591727E-2,2.905005E-1,-3.331173E-2,1.0370807E-1,-1.303187E-1,-4.6400063E-2,1.0148708E-1,2.4140713E-1,1.7749779E-1,-2.0061411E-2,3.507235E-1,-8.37954E-3,-6.774013E-2,2.248793E-2,-7.109631E-4,1.3408074E-1,-1.06043324E-1,-1.8153128E-1,-8.55397E-2,8.363892E-3,1.3143325E-1,1.7055387E-2,2.5595507E-1,1.5961862E-3,1.3813749E-2,-5.077823E-4,3.2630486E-3,1.910694E-2,-4.1275444E-3,6.6394237E-4,4.782669E-2,-4.6447343E-3,-5.407716E-4,7.7739717E-3,-1.2945457E-1,-9.414223E-3,-2.2072856E-1,-4.5816624E-3,7.847245E-4,-5.647731E-3,-3.024223E-3,1.974557E-3,5.9647363E-2,1.5820011E-1,-9.319103E-3,6.394075E-2,1.7609752E-3,2.6637048E-1,1.8049946E-2,6.555792E-3,-1.1079848E-1,-1.2821042E-2,-4.312158E-3,2.8895882E-3,-1.1715736E-2,-3.811696E-4,-1.7549363E-3,5.293153E-3,2.0451404E-3,1.7079389E-1,-2.3967717E-3,4.1009304E-3,1.5632087E-2,9.737114E-3,1.4893975E-3,-6.303284E-3,-1.2120445E-2,-9.000306E-2,1.1207155E-2,1.2216185E-1,-3.0160283E-3,-9.493145E-3,3.0382609E-3,9.516032E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,39,-1,41,43,-1,45,47,49,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,63,65,67,-1,-1,-1,-1,-1,69,71,-1,73,-1,75,77,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,81,-1,-1,-1,-1,-1,-1,-1,83,-1,85,-1,-1,-1,-1],"loss_changes":[8.244977E0,1.027297E0,2.309177E0,8.325529E-2,4.9960518E-1,8.68268E-1,1.1023536E0,0E0,0E0,3.100955E-1,1.7966568E-1,5.105748E-1,5.895691E-1,0E0,6.1065555E-1,1.8402717E-1,7.3546946E-2,7.896292E-2,8.859652E-2,1.7980355E-1,1.2590742E-1,5.8491938E-2,0E0,1.3738108E-1,0E0,7.208881E-2,1.126618E-1,0E0,5.1466852E-2,1.2546498E-1,6.4293206E-2,6.8518594E-2,4.100561E-2,9.24024E-2,2.0277151E-1,8.4867954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.4572615E-2,0E0,0E0,0E0,7.826954E-2,6.700808E-2,4.491806E-2,0E0,0E0,0E0,0E0,0E0,7.386856E-2,4.9135685E-2,0E0,3.839927E-2,0E0,6.856489E-2,4.4424273E-2,0E0,9.0483546E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.0994864E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.820071E-2,0E0,7.219058E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,43,43,47,47,48,48,49,49,55,55,56,56,58,58,60,60,61,61,63,63,72,72,80,80,82,82],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,40,-1,42,44,-1,46,48,50,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,64,66,68,-1,-1,-1,-1,-1,70,72,-1,74,-1,76,78,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,82,-1,-1,-1,-1,-1,-1,-1,84,-1,86,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.41718E5,7.7101436E3,2.91E2,1.3026638E7,2.3028242E3,1.681178E6,-1.0094103E-2,2.418048E-3,3.23643E6,7.198E3,6.050605E9,2.909019E3,3.6591727E-2,1.5096262E7,7.164074E8,3.3E1,2.7075084E5,1.4253456E-2,1.9196308E7,9.677921E7,8.494E3,-2.0061411E-2,1.7895421E0,-8.37954E-3,3.4015296E8,2.39E2,-7.109631E-4,5.66936E6,4.5474697E3,2.946708E6,1.572445E3,1.1172539E9,1.5704E4,1.2587E4,8.946923E2,1.5961862E-3,1.3813749E-2,-5.077823E-4,3.2630486E-3,1.910694E-2,-4.1275444E-3,6.6394237E-4,1.5522031E9,-4.6447343E-3,-5.407716E-4,7.7739717E-3,3.9034148E7,8.7182234E4,6.4065235E8,-4.5816624E-3,7.847245E-4,-5.647731E-3,-3.024223E-3,1.974557E-3,3.253389E8,1.1E1,-9.319103E-3,2.1853803E4,1.7609752E-3,8.080874E-1,1.007452E6,6.555792E-3,9.687575E2,-1.2821042E-2,-4.312158E-3,2.8895882E-3,-1.1715736E-2,-3.811696E-4,-1.7549363E-3,5.293153E-3,2.0451404E-3,7.8E1,-2.3967717E-3,4.1009304E-3,1.5632087E-2,9.737114E-3,1.4893975E-3,-6.303284E-3,-1.2120445E-2,1.5362013E3,1.1207155E-2,2.5096E4,-3.0160283E-3,-9.493145E-3,3.0382609E-3,9.516032E-3],"split_indices":[2,1,64,8,57,70,41,0,0,1,2,12,44,0,9,7,8,60,50,57,1,0,0,65,0,44,3,0,57,59,1,59,5,2,2,64,0,0,0,0,0,0,0,7,0,0,0,57,40,7,0,0,0,0,0,7,8,0,45,0,46,41,0,4,0,0,0,0,0,0,0,0,3,0,0,0,0,0,0,0,59,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,2.98E2,1.48E2,6.7E1,2.31E2,1.2E2,2.8E1,6.6E1,1E0,1.13E2,1.18E2,1.14E2,6E0,9E0,1.9E1,9.4E1,1.9E1,7.8E1,4E1,7.1E1,4.3E1,2E0,4E0,1.7E1,2E0,5.8E1,3.6E1,4E0,1.5E1,5.5E1,2.3E1,2.3E1,1.7E1,5.2E1,1.9E1,4E1,3E0,1E0,1E0,2E0,1.5E1,4.9E1,9E0,3E1,6E0,2E0,1.3E1,4.4E1,1.1E1,1.5E1,8E0,5E0,1.8E1,5E0,1.2E1,1.5E1,3.7E1,3E0,1.6E1,2E0,3.8E1,2.3E1,7E0,4E1,4E0,5E0,6E0,1.4E1,1E0,5E0,1E1,4E0,3.3E1,2E0,1.4E1,2.1E1,1.7E1,2.2E1,1E0,4E0,3.6E1,1.4E1,1.9E1,2.9E1,7E0,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[1.4494569E-3,-5.165316E-2,2.869512E-1,-1.11480385E-1,6.841784E-2,5.767569E-1,2.2805607E-1,-1.03099346E-1,-2.4897022E-2,5.59849E-3,1.825076E-1,8.92731E-3,3.516941E-2,2.8314754E-1,3.0751565E-2,-1.2607013E-1,-6.675537E-3,-2.1473894E-2,8.9871725E-3,5.681992E-3,2.5939894E-1,-3.2622525E-3,3.0046853E-1,6.0253954E-3,-1.1190888E-2,-1.063556E-2,-1.04680754E-1,4.7284206E-3,-3.247596E-2,-5.905905E-2,3.4996297E-2,3.007514E-1,6.9785137E-3,7.0875264E-3,3.4295923E-1,-1.0830692E-1,1.1791333E-2,-7.7164926E-2,1.8626368E-2,-3.819994E-2,-1.0492394E-2,5.9445333E-3,1.2550305E-2,3.9323866E-3,1.6552342E-2,3.6996064E-1,2.3967717E-3,-1.2640475E-1,-2.6272746E-2,-6.0842456E-3,-2.0991419E-4,-2.9134548E-3,3.4161853E-3,-5.1620933E-3,-6.093524E-4,2.153887E-3,-2.1565587E-3,3.4100902E-3,1.887815E-2,7.6157926E-4,-1.3458966E-1,-1.1289586E-2,4.274695E-3,-9.945303E-3,-5.384774E-3,-1.2942656E-3,7.107238E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,25,27,29,-1,-1,31,-1,33,-1,-1,-1,35,-1,37,39,41,43,-1,-1,45,47,-1,49,51,53,-1,-1,55,-1,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,-1,-1,-1,-1],"loss_changes":[6.670453E0,2.6727111E0,1.0461688E0,7.523892E-1,8.8769186E-1,4.3033338E-1,6.326108E-1,5.396011E-1,0E0,3.859263E-1,2.033186E-1,0E0,0E0,2.9722548E-1,3.4722677E-1,3.4323192E-1,1.2851128E-1,1.5276015E-1,0E0,0E0,5.705273E-2,0E0,2.4797297E-1,0E0,0E0,0E0,2.2317696E-1,0E0,9.0501994E-2,1.2731887E-1,5.315389E-2,6.861782E-2,0E0,0E0,2.5308228E-1,2.3273659E-1,0E0,6.779395E-2,7.666758E-2,6.476112E-2,0E0,0E0,4.2697962E-2,0E0,0E0,4.7023773E-2,0E0,1.5276551E-1,1.8390852E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9122243E-1,0E0,1.16054885E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,17,17,20,20,22,22,26,26,28,28,29,29,30,30,31,31,34,34,35,35,37,37,38,38,39,39,42,42,45,45,47,47,48,48,60,60,62,62],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,26,28,30,-1,-1,32,-1,34,-1,-1,-1,36,-1,38,40,42,44,-1,-1,46,48,-1,50,52,54,-1,-1,56,-1,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,-1,-1,-1,-1],"split_conditions":[3.2991206E3,8.393928E4,1.064816E6,2.7148398E7,1.484798E9,9.256843E3,1.5096262E7,7.164074E8,-2.4897022E-2,1.7670108E5,4.9014646E9,8.92731E-3,3.516941E-2,2.4950776E0,1.92144E5,5.46223E5,2.0718214E-4,1.7657828E3,8.9871725E-3,5.681992E-3,1.5029658E-1,-3.2622525E-3,2.1360708E7,6.0253954E-3,-1.1190888E-2,-1.063556E-2,1.3428E4,4.7284206E-3,2.943086E4,1E0,7.9E1,1.5200746E-1,6.9785137E-3,7.0875264E-3,3.020675E5,2.0964778E5,1.1791333E-2,2.579534E7,1.9252769E5,4.39E3,-1.0492394E-2,5.9445333E-3,5.792948E8,3.9323866E-3,1.6552342E-2,3.1095595E0,2.3967717E-3,1.0640107E8,2.4094029E2,-6.0842456E-3,-2.0991419E-4,-2.9134548E-3,3.4161853E-3,-5.1620933E-3,-6.093524E-4,2.153887E-3,-2.1565587E-3,3.4100902E-3,1.887815E-2,7.6157926E-4,4.822E3,-1.1289586E-2,4.735985E6,-9.945303E-3,-5.384774E-3,-1.2942656E-3,7.107238E-3],"split_indices":[64,45,41,60,7,4,9,7,0,40,12,0,0,66,2,1,50,64,0,0,50,0,1,0,0,0,2,0,45,19,3,53,0,0,45,40,0,9,59,2,0,0,44,0,0,66,0,5,64,0,0,0,0,0,0,0,0,0,0,0,2,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,3.7E2,6.8E1,2.47E2,1.23E2,1E1,5.8E1,2.43E2,4E0,8E1,4.3E1,3E0,7E0,4.5E1,1.3E1,1.96E2,4.7E1,7E1,1E1,2.4E1,1.9E1,2E0,4.3E1,1E1,3E0,3.7E1,1.59E2,9E0,3.8E1,4.2E1,2.8E1,1.3E1,6E0,1E1,3.3E1,1.58E2,1E0,2E1,1.8E1,3.8E1,4E0,5E0,2.3E1,2E0,1.1E1,3E1,3E0,1.29E2,2.9E1,1.2E1,8E0,7E0,1.1E1,1E1,2.8E1,1.5E1,8E0,1E0,2.9E1,7E0,1.22E2,3E0,2.6E1,3.4E1,8.8E1,2.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[3.603184E-3,-7.994202E-2,1.6792168E-1,-1.54378E-1,-4.2881798E-2,1.9374241E-1,-1.806661E-1,-1.7485082E-1,-2.8073438E-2,3.1336157E-3,-8.669388E-2,1.6879725E-1,2.529483E-2,1.1698272E-2,-2.5998974E-2,-1.1904041E-2,-1.5551198E-1,3.546691E-3,-2.935981E-3,-1.1669727E-2,6.7992634E-3,-1.3551107E-1,-5.8441192E-2,1.0471378E-1,2.3036137E-1,-8.466878E-3,7.7089164E-3,-9.7586575E-4,-1.6370595E-1,-4.056159E-2,5.0501492E-2,-7.6681904E-2,-1.7213634E-1,-8.078895E-2,3.132184E-3,6.619255E-2,1.7097296E-1,3.0366208E-2,1.967641E-1,-8.513913E-3,-2.5547482E-3,-6.208802E-2,7.748312E-3,-5.214544E-4,1.00663766E-1,-9.193193E-2,3.7998955E-3,-7.33133E-4,-1.8714951E-1,9.985294E-4,-8.877114E-2,1.4587475E-1,3.5211366E-2,7.131661E-5,9.311612E-3,2.2148854E-1,-1.343371E-1,-1.08217E-1,-1.0525064E-3,3.125872E-3,-2.1376465E-3,7.421694E-3,1.7121563E-3,-5.639626E-3,1.4663892E-3,-2.1417004E-1,-3.1611836E-3,-3.2876777E-3,-1.37815E-1,1.1822629E-2,2.3638883E-3,-3.8596846E-3,6.305887E-2,2.1970807E-2,2.019521E-1,4.758754E-3,-1.077143E-2,-7.684168E-3,-1.0338827E-3,-1.2652668E-2,-7.772546E-2,-1.6754599E-1,2.3422774E-3,-1.552597E-3,8.0779284E-2,4.228538E-3,2.1434079E-1,1.9372143E-3,-8.411238E-3,-1.0653637E-2,-3.557226E-3,6.441184E-3,2.0314215E-3,6.4416993E-3,1.2038158E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,-1,27,-1,-1,29,-1,31,33,35,37,-1,-1,-1,39,41,43,45,47,49,-1,51,53,-1,55,-1,-1,57,59,-1,61,63,-1,-1,65,-1,67,69,71,-1,-1,73,75,77,-1,-1,-1,-1,-1,-1,-1,79,-1,-1,81,-1,-1,-1,83,-1,85,-1,-1,-1,-1,-1,87,89,-1,-1,91,-1,93,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.191034E0,8.182055E-1,1.4031229E0,2.510867E-1,4.054939E-1,1.0211396E0,7.246021E-1,6.708264E-2,4.9739458E-2,1.9638124E-1,1.3413996E-1,5.005951E-1,0E0,2.3268442E-1,0E0,0E0,7.2573066E-2,0E0,0E0,1.6371024E-1,0E0,6.6521704E-2,1.8640852E-1,1.596927E-1,7.296479E-1,0E0,0E0,0E0,3.9235353E-2,6.495734E-2,8.969367E-2,4.4252798E-2,4.9264193E-2,4.7622174E-2,0E0,1.0348414E-1,6.2188327E-2,0E0,5.454905E-1,0E0,0E0,7.91015E-2,5.8046184E-2,0E0,4.574199E-2,4.135695E-2,0E0,0E0,5.340737E-2,0E0,5.0682217E-2,9.608799E-2,1.0820064E-1,0E0,0E0,1.6265297E-1,1.13717295E-1,7.654081E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.547296E-2,0E0,0E0,9.570548E-2,0E0,0E0,0E0,4.7544077E-2,0E0,6.327152E-2,0E0,0E0,0E0,0E0,0E0,5.9281416E-2,4.6386242E-2,0E0,0E0,3.916405E-2,0E0,8.232713E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,16,16,19,19,21,21,22,22,23,23,24,24,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,38,38,41,41,42,42,44,44,45,45,48,48,50,50,51,51,52,52,55,55,56,56,57,57,65,65,68,68,72,72,74,74,80,80,81,81,84,84,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,-1,28,-1,-1,30,-1,32,34,36,38,-1,-1,-1,40,42,44,46,48,50,-1,52,54,-1,56,-1,-1,58,60,-1,62,64,-1,-1,66,-1,68,70,72,-1,-1,74,76,78,-1,-1,-1,-1,-1,-1,-1,80,-1,-1,82,-1,-1,-1,84,-1,86,-1,-1,-1,-1,-1,88,90,-1,-1,92,-1,94,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,9.50639E5,1.0429407E3,1.2254369E7,1.3508742E7,1.6828056E1,6.6053805E6,1.44E2,2.409091E0,1.577058E9,7.121916E2,1.2511627E7,2.529483E-2,1.4414646E-1,-2.5998974E-2,-1.1904041E-2,2E1,3.546691E-3,-2.935981E-3,2.455358E6,6.7992634E-3,1.7377544E7,7.1999524E9,1.5271514E5,1.064816E6,-8.466878E-3,7.7089164E-3,-9.7586575E-4,1.6120845E5,2.2582366E3,1E0,4.94E2,2E1,7.0934258E0,3.132184E-3,5.65E2,1.9937408E-1,3.0366208E-2,5.22E2,-8.513913E-3,-2.5547482E-3,1E0,1.52E2,-5.214544E-4,1E0,6.24E3,3.7998955E-3,-7.33133E-4,3.7075245E1,9.985294E-4,7.313879E6,1.93E2,7.10711E8,7.131661E-5,9.311612E-3,3.7137297E3,1.3590209E7,6.927258E4,-1.0525064E-3,3.125872E-3,-2.1376465E-3,7.421694E-3,1.7121563E-3,-5.639626E-3,1.4663892E-3,6.842986E6,-3.1611836E-3,-3.2876777E-3,6.33E2,1.1822629E-2,2.3638883E-3,-3.8596846E-3,2.8529238E1,2.1970807E-2,2.4547362E0,4.758754E-3,-1.077143E-2,-7.684168E-3,-1.0338827E-3,-1.2652668E-2,2.2967976E7,7.567335E7,2.3422774E-3,-1.552597E-3,2.5871767E-2,4.228538E-3,6.7304E4,1.9372143E-3,-8.411238E-3,-1.0653637E-2,-3.557226E-3,6.441184E-3,2.0314215E-3,6.4416993E-3,1.2038158E-2],"split_indices":[2,1,70,9,57,47,59,0,70,7,64,1,0,50,0,0,10,0,0,1,0,57,5,40,41,0,0,0,40,4,13,3,3,59,0,0,53,0,3,0,0,25,3,0,26,2,0,0,68,0,9,3,7,0,0,44,1,45,0,0,0,0,0,0,0,41,0,0,8,0,0,0,68,0,65,0,0,0,0,0,41,44,0,0,69,0,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.49E2,2.98E2,1.51E2,9.8E1,2E2,1.41E2,1E1,8.4E1,1.4E1,9.8E1,1.02E2,1.32E2,9E0,7E0,3E0,1.7E1,6.7E1,3E0,1.1E1,8.9E1,9E0,3.6E1,6.6E1,6.6E1,6.6E1,3E0,4E0,4E0,6.3E1,6.1E1,2.8E1,1.5E1,2.1E1,5.6E1,1E1,4.3E1,2.3E1,4E0,6.2E1,5.9E1,4E0,4.2E1,1.9E1,1.3E1,1.5E1,1.4E1,1E0,2E0,1.9E1,4E0,5.2E1,1.1E1,3.2E1,2E0,2.1E1,5.8E1,4E0,1.9E1,2.3E1,9E0,1E1,8E0,7E0,1.2E1,2E0,1.5E1,4E0,3.7E1,1.5E1,5E0,6E0,6E0,2.6E1,3E0,5.5E1,1E0,3E0,1.2E1,7E0,1.1E1,4E0,1.3E1,2E0,4E0,2.2E1,6E0,4.9E1,2E0,2E0,8E0,5E0,9E0,1.3E1,1.3E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[4.52243E-3,-7.515865E-2,1.6385403E-1,-9.711443E-2,1.4682604E-2,-2.3352416E-1,1.8822938E-1,-1.4995912E-1,-6.338092E-2,-8.357022E-3,6.4175525E-3,5.008996E-3,-4.171048E-1,1.4523996E-1,4.005999E-1,-1.8921623E-1,-5.498939E-3,-2.3654286E-2,-1.1900119E-1,-2.5664924E-2,1.22845076E-1,-2.502448E-2,-8.324558E-3,9.591124E-2,2.1771716E-1,3.0360255E-2,2.4975577E-1,-1.08294785E-2,-5.6421114E-3,-4.4695266E-2,5.400266E-2,-2.0094751E-1,-7.714576E-2,2.068826E-3,-4.128347E-2,9.1205565E-3,1.17889256E-4,7.062934E-2,7.901855E-3,1.8485922E-2,1.9495882E-1,1.4528658E-2,-8.086992E-3,-6.92315E-2,4.194826E-3,-1.9041717E-3,5.5426136E-3,-1.11552946E-1,-1.2556382E-2,-6.7922415E-4,-1.0516718E-1,-3.6953972E-3,-1.3499704E-4,3.5845558E-3,9.460479E-2,2.4139805E-1,7.2006206E-3,-8.885009E-2,-3.1191792E-4,3.4691186E-3,-1.3365628E-3,-7.902519E-3,1.7142725E-3,-3.662599E-3,-9.739957E-3,-3.2665918E-3,2.2286682E-3,1.593681E-1,4.748738E-2,1.7795531E-3,1.3053055E-2,-4.7815223E-3,3.5792359E-3,2.0177453E-3,1.0484275E-2,1.4844714E-4,4.9890312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,29,31,33,35,-1,-1,37,39,-1,41,-1,-1,43,45,47,49,-1,51,-1,-1,53,-1,-1,55,-1,-1,57,59,-1,-1,61,-1,-1,63,-1,-1,65,67,69,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.636516E0,5.8529615E-1,1.4775188E0,4.1476274E-1,1.5567048E-1,5.9323925E-1,1.2199726E0,1.2338543E-1,3.2199717E-1,1.1669446E-1,0E0,0E0,4.2297482E-2,4.0099287E-1,5.660815E-1,6.9903135E-2,0E0,1.4456978E-1,1.9372165E-1,4.8841946E-2,4.2693913E-2,0E0,0E0,1.02057815E-1,9.0661764E-2,0E0,2.9989076E-1,0E0,0E0,8.306135E-2,1.03804305E-1,6.1904013E-2,7.3368266E-2,0E0,4.6423584E-2,0E0,0E0,8.48546E-2,0E0,0E0,6.6815734E-2,0E0,0E0,5.6291252E-2,5.0285008E-2,0E0,0E0,6.6560164E-2,0E0,0E0,6.8670034E-2,0E0,0E0,4.5328137E-2,1.11439705E-1,7.621539E-2,0E0,4.5190543E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.752051E-2,5.5625033E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,23,23,24,24,26,26,29,29,30,30,31,31,32,32,34,34,37,37,40,40,43,43,44,44,47,47,50,50,53,53,54,54,55,55,57,57,67,67,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,30,32,34,36,-1,-1,38,40,-1,42,-1,-1,44,46,48,50,-1,52,-1,-1,54,-1,-1,56,-1,-1,58,60,-1,-1,62,-1,-1,64,-1,-1,66,68,70,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,2.4950776E0,1.129877E6,1.6219076E9,2.9632012E7,7.7101436E3,4.90403E5,1.7377544E7,1E0,6.4175525E-3,5.008996E-3,3.1E1,6.0891E4,3.3616E4,1.10388414E5,-5.498939E-3,1.272E4,7.121916E2,1.5124741E1,1.21E2,-2.502448E-2,-8.324558E-3,1.5271514E5,1.3812E4,3.0360255E-2,7.79557E5,-1.08294785E-2,-5.6421114E-3,1.0220021E3,1.8E9,3.8E1,2.0839167E3,2.068826E-3,3.0251338E5,9.1205565E-3,1.17889256E-4,5.2798065E1,7.901855E-3,1.8485922E-2,4.917967E7,1.4528658E-2,-8.086992E-3,2.1592189E5,3.7755436E1,-1.9041717E-3,5.5426136E-3,1.5E1,-1.2556382E-2,-6.7922415E-4,6.54925E6,-3.6953972E-3,-1.3499704E-4,5.795E3,1.16E2,8.825362E-2,7.2006206E-3,2.1603052E9,-3.1191792E-4,3.4691186E-3,-1.3365628E-3,-7.902519E-3,1.7142725E-3,-3.662599E-3,-9.739957E-3,-3.2665918E-3,2.2286682E-3,4.1E1,2.6196917E3,1.7795531E-3,1.3053055E-2,-4.7815223E-3,3.5792359E-3,2.0177453E-3,1.0484275E-2,1.4844714E-4,4.9890312E-3],"split_indices":[2,7,66,1,7,1,64,9,57,101,0,0,3,2,10,40,0,10,64,70,3,0,0,40,9,0,2,0,0,64,5,8,4,0,59,0,0,68,0,0,56,0,0,40,70,0,0,8,0,0,9,0,0,10,3,51,0,12,0,0,0,0,0,0,0,0,0,3,64,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,2.95E2,1.47E2,2.37E2,5.8E1,8E0,1.39E2,9.1E1,1.46E2,4.9E1,9E0,3E0,5E0,1.17E2,2.2E1,4.4E1,4.7E1,8.6E1,6E1,4.4E1,5E0,3E0,2E0,7.1E1,4.6E1,8E0,1.4E1,3.1E1,1.3E1,6.8E1,1.8E1,1.9E1,4.1E1,8E0,3.6E1,3E0,2E0,5.2E1,1.9E1,4E0,4.2E1,1.3E1,1E0,4.5E1,2.3E1,7E0,1.1E1,8E0,1.1E1,1.3E1,2.8E1,1.9E1,1.7E1,1.4E1,3.8E1,2E1,2.2E1,3.4E1,1.1E1,7E0,1.6E1,6E0,2E0,2.2E1,6E0,5E0,9E0,1.5E1,2.3E1,2E0,1.8E1,3.3E1,1E0,5E0,1E1,1.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[1.7336373E-4,-6.1993726E-2,1.7657764E-1,-9.623901E-2,2.3195093E-2,1.431533E-1,3.4052044E-2,-9.99158E-2,1.9277878E-2,-2.4719685E-2,1.07409984E-1,2.079739E-2,1.8223439E-1,-9.505005E-2,-2.229628E-2,-1.03549E-1,-1.6925395E-3,7.5254175E-3,6.750316E-2,5.845359E-2,-1.6411185E-1,1.2184009E-1,2.1892405E-1,-1.7511977E-1,-7.558591E-2,-3.3454841E-3,-1.076794E-2,-9.022027E-4,4.2184717E-3,4.648295E-3,-3.7559227E-4,6.9758315E-3,-2.093189E-2,-1.0690592E-2,8.770143E-4,3.5932206E-3,9.010977E-3,2.7140152E-1,1.6975486E-1,-1.1493872E-2,-6.098388E-3,-8.864319E-2,-2.99231E-3,-8.7407485E-2,3.5578418E-3,1.4016579E-2,1.7988716E-3,6.4493595E-3,1.1703963E-2,-9.193334E-2,1.0939286E-2,7.260179E-2,-5.057846E-2,-7.7335215E-3,9.303342E-4,-1.0526128E-1,-3.0670233E-2,5.2850293E-3,-1.0062566E-3,-7.892783E-3,-1.4322475E-3,-4.3181903E-4,-5.6560463E-3,-4.7149705E-3,2.9406662E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,-1,25,27,-1,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,-1,-1,45,47,-1,-1,49,51,53,-1,-1,-1,-1,-1,55,-1,57,59,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.923957E0,9.7078085E-1,1.8765128E0,4.5853996E-1,3.9002836E-1,5.269902E-1,0E0,3.5516834E-1,0E0,1.1236802E-1,5.013314E-2,2.0455636E-1,1.594572E-1,3.5144973E-1,0E0,4.2932212E-2,7.054754E-2,0E0,3.8969353E-2,1.5749443E-1,4.8989475E-2,8.631122E-2,8.5522175E-2,1.0302615E-1,1.7983091E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.583353E-2,0E0,0E0,0E0,0E0,3.8981557E-2,4.5843363E-2,0E0,0E0,1.8302822E-1,1.1158905E-1,5.999483E-2,0E0,0E0,0E0,0E0,0E0,1.2741339E-1,0E0,3.9019175E-2,4.018377E-2,0E0,0E0,9.91894E-2,7.038238E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,32,32,37,37,38,38,41,41,42,42,43,43,49,49,51,51,52,52,55,55,56,56],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,-1,26,28,-1,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,-1,-1,46,48,-1,-1,50,52,54,-1,-1,-1,-1,-1,56,-1,58,60,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.484798E9,9.7467445E4,1.5324995E4,3.723672E3,2.1695537E3,8.447218E4,3.4052044E-2,5.38168E6,1.9277878E-2,3.0192542E-1,1.3058374E7,1.505649E6,5.3515047E-1,4.66085E5,-2.229628E-2,1.153215E9,1.5271514E5,7.5254175E-3,1.2753316E9,3.798313E3,2.412E4,7.3036104E-1,1.7176836E7,3.606E3,4.723148E11,-3.3454841E-3,-1.076794E-2,-9.022027E-4,4.2184717E-3,4.648295E-3,-3.7559227E-4,6.9758315E-3,1.173433E5,-1.0690592E-2,8.770143E-4,3.5932206E-3,9.010977E-3,6.520156E7,7.309061E1,-1.1493872E-2,-6.098388E-3,1.3664E4,1.4399977E10,1.6167939E7,3.5578418E-3,1.4016579E-2,1.7988716E-3,6.4493595E-3,1.1703963E-2,6.4065235E8,1.0939286E-2,2.58E2,4.9256845E2,-7.7335215E-3,9.303342E-4,1.0640107E8,3.738719E4,5.2850293E-3,-1.0062566E-3,-7.892783E-3,-1.4322475E-3,-4.3181903E-4,-5.6560463E-3,-4.7149705E-3,2.9406662E-4],"split_indices":[7,45,64,64,64,45,0,59,0,54,57,2,47,1,0,7,40,0,7,44,9,51,55,2,43,0,0,0,0,0,0,0,40,0,0,0,0,1,68,0,0,2,5,44,0,0,0,0,0,7,0,3,45,0,0,5,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.31E2,1.16E2,2.36E2,9.5E1,1.1E2,6E0,2.35E2,1E0,6.1E1,3.4E1,2.7E1,8.3E1,2.33E2,2E0,1.3E1,4.8E1,1.5E1,1.9E1,2.3E1,4E0,3.3E1,5E1,4.4E1,1.89E2,1.1E1,2E0,4.1E1,7E0,1.4E1,5E0,1.1E1,1.2E1,3E0,1E0,1.9E1,1.4E1,2.2E1,2.8E1,2E1,2.4E1,1.6E2,2.9E1,7E0,5E0,2.1E1,1E0,1.9E1,9E0,1.59E2,1E0,1.1E1,1.8E1,4E0,3E0,1.3E2,2.9E1,8E0,3E0,2E0,1.6E1,1E1,1.2E2,1E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[2.2962114E-3,-6.9581464E-2,1.4898972E-1,-9.317597E-2,1.3157424E-2,1.7996335E-1,-1.8173766E-1,-1.293594E-1,-4.57916E-2,2.6108235E-2,-9.862527E-3,1.3907112E-1,3.9338005E-1,-1.3150192E-2,-2.2431644E-2,-1.061326E-2,-1.16794944E-1,-5.2521453E-4,-1.0311735E-1,6.413465E-3,1.3320901E-2,1.4907606E-1,-1.08851746E-1,2.834243E-2,1.3293716E-2,1.21679725E-2,-7.566294E-2,-1.2260261E-1,-1.1527503E-3,-5.417974E-2,2.4973813E-2,-1.2551725E-1,1.8197875E-3,2.739252E-4,5.414325E-3,1.2665074E-1,3.866658E-1,1.0042598E-2,-1.1821278E-2,-8.105767E-3,2.5695337E-3,-1.4652708E-1,-9.0317E-2,-6.1172387E-4,-6.213389E-3,-1.3162934E-3,3.2007399E-3,-9.241534E-2,-1.9229527E-1,-3.6289403E-3,1.3225609E-2,9.441477E-2,1.7845827E-1,2.6187172E-2,8.596104E-3,-1.1555315E-3,-1.5478146E-1,-5.310119E-2,-5.930876E-3,-1.6134007E-1,-3.832639E-2,-2.2248366E-1,-1.3777835E-3,-1.178233E-3,1.8196992E-3,-3.312013E-3,1.0325883E-1,1.1742057E-1,2.2752118E-1,-8.546352E-3,-4.3331007E-3,-1.5702773E-3,-7.4079037E-3,-1.1643921E-2,-7.2127517E-4,8.322883E-4,-3.922016E-3,4.372218E-4,-1.2451451E-2,-1.5942693E-3,1.0829583E-1,-4.895034E-4,1.3296324E-1,1.9619276E-3,1.2307032E-2,9.80109E-3,4.7430457E-3,3.1993613E-3,8.73698E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,-1,33,35,37,-1,-1,-1,39,41,-1,43,45,47,-1,49,-1,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,59,61,-1,63,65,67,-1,-1,-1,69,71,-1,73,75,77,-1,-1,-1,-1,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,-1,87,-1,-1,-1,-1,-1,-1],"loss_changes":[4.681354E0,5.8297634E-1,1.5283046E0,3.9184713E-1,1.8780613E-1,1.1147218E0,5.7950556E-1,1.1182237E-1,2.6462865E-1,8.194828E-2,0E0,2.9899883E-1,3.1389022E-1,1.6280374E-1,0E0,0E0,6.0316324E-2,8.070509E-2,1.4521667E-1,0E0,7.187234E-2,5.371115E-1,2.4531896E-1,0E0,0E0,0E0,9.6307784E-2,7.095945E-2,0E0,5.4375138E-2,8.188102E-2,6.8437636E-2,0E0,5.0049167E-2,0E0,1.534977E-1,1.7518234E-1,0E0,0E0,0E0,0E0,5.8557868E-2,4.540986E-2,0E0,0E0,0E0,0E0,9.820227E-2,5.354029E-2,0E0,3.9255716E-2,9.921622E-2,8.7082386E-2,0E0,0E0,0E0,4.8142433E-2,4.2778425E-2,0E0,1.2294823E-1,3.8834997E-2,6.383672E-2,0E0,0E0,0E0,0E0,4.566413E-2,3.8870394E-2,6.0883522E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.193925E-2,0E0,3.7871063E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,16,16,17,17,18,18,20,20,21,21,22,22,26,26,27,27,29,29,30,30,31,31,33,33,35,35,36,36,41,41,42,42,47,47,48,48,50,50,51,51,52,52,56,56,57,57,59,59,60,60,61,61,66,66,67,67,68,68,80,80,82,82],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,-1,34,36,38,-1,-1,-1,40,42,-1,44,46,48,-1,50,-1,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,60,62,-1,64,66,68,-1,-1,-1,70,72,-1,74,76,78,-1,-1,-1,-1,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,-1,88,-1,-1,-1,-1,-1,-1],"split_conditions":[1.227E4,7.164074E8,1.0429407E3,1.732734E6,1E0,4.966055E9,7.339209E6,1.44E2,1.8071064E7,2.829932E0,-9.862527E-3,3.768034E4,6.0976E4,1.3812E4,-2.2431644E-2,-1.061326E-2,2.1592189E5,9.286926E4,1.7076351E3,6.413465E-3,1E0,8.10288E3,1.92144E5,2.834243E-2,1.3293716E-2,1.21679725E-2,1.173433E5,4.822E3,-1.1527503E-3,1E0,5.878966E2,3.069012E6,1.8197875E-3,1.4338295E6,5.414325E-3,5.9161E4,1.6441814E9,1.0042598E-2,-1.1821278E-2,-8.105767E-3,2.5695337E-3,2E1,1E0,-6.1172387E-4,-6.213389E-3,-1.3162934E-3,3.2007399E-3,2.47771E7,4.058817E6,-3.6289403E-3,3.24E2,4.56E2,1.9333144E7,2.6187172E-2,8.596104E-3,-1.1555315E-3,6.1859414E4,6.17296E5,-5.930876E-3,2.206001E3,4.084E4,1.24E2,-1.3777835E-3,-1.178233E-3,1.8196992E-3,-3.312013E-3,2.0772269E8,7.335845E1,1E1,-8.546352E-3,-4.3331007E-3,-1.5702773E-3,-7.4079037E-3,-1.1643921E-2,-7.2127517E-4,8.322883E-4,-3.922016E-3,4.372218E-4,-1.2451451E-2,-1.5942693E-3,3.457E3,-4.895034E-4,7.769761E0,1.9619276E-3,1.2307032E-2,9.80109E-3,4.7430457E-3,3.1993613E-3,8.73698E-3],"split_indices":[2,7,70,1,19,7,59,0,57,70,0,4,10,9,0,0,40,40,64,0,101,64,2,0,0,0,40,2,0,25,64,9,0,57,0,2,7,0,0,0,0,10,24,0,0,0,0,57,41,0,8,10,44,0,0,0,45,41,0,4,11,3,0,0,0,0,5,70,8,0,0,0,0,0,0,0,0,0,0,0,10,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,2.97E2,1.45E2,2.31E2,6.6E1,1.33E2,1.2E1,1.3E2,1.01E2,6.3E1,3E0,1.13E2,2E1,8E0,4E0,1.5E1,1.15E2,5.7E1,4.4E1,6E0,5.7E1,1.09E2,4E0,7E0,1.3E1,1E0,7E0,1.08E2,7E0,1.8E1,3.9E1,3.8E1,6E0,5.1E1,6E0,1.01E2,8E0,1E0,3E0,4E0,3E0,6E1,4.8E1,1.2E1,6E0,1.7E1,2.2E1,2.7E1,1.1E1,7E0,4.4E1,6.4E1,3.7E1,4E0,4E0,4E0,5.6E1,2.2E1,2.6E1,1.1E1,1.6E1,9E0,2E0,1.7E1,2.7E1,3E0,6.1E1,1.8E1,1.9E1,4.4E1,1.2E1,1.9E1,3E0,7E0,4E0,7E0,9E0,1E0,8E0,2E0,5.9E1,2E0,1.6E1,2E0,1.7E1,6E0,5.3E1,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.069079E-2,-6.958858E-2,1.2871398E-1,-7.284245E-2,1.8750688E-2,9.591172E-2,3.6001065E-1,-8.402459E-2,3.205247E-2,-1.9426567E-2,1.24432825E-1,1.3232165E-2,4.8168057E-1,-1.6705692E-1,-7.251025E-2,4.102122E-3,-1.400537E-1,-4.3530106E-3,1.497196E-5,1.3128097E-1,-9.543978E-3,2.7499074E-2,9.195203E-3,-9.530848E-3,-1.2418949E-3,-6.845348E-2,-1.5931956E-2,-9.225682E-4,-9.807078E-3,1.1844186E-1,2.368891E-2,-7.2468616E-2,1.026306E-1,6.788819E-2,1.6373886E-1,-9.440486E-2,-3.80727E-2,9.0794E-3,-5.309415E-4,4.1678756E-2,8.131126E-3,1.7375877E-1,2.1454657E-3,-1.2868752E-1,-2.6756634E-3,-8.0464475E-2,2.1458392E-3,-1.6143502E-4,4.1470123E-3,1.9491435E-3,1.8065067E-1,-7.896157E-3,-4.6172184E-3,-6.6227913E-3,-7.628198E-4,2.3792102E-3,-2.6486826E-3,2.797376E-3,9.479764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,33,-1,35,37,39,41,43,45,-1,-1,47,-1,49,-1,51,-1,53,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0262837E0,3.9348733E-1,1.2704439E0,2.9827678E-1,0E0,5.024328E-1,1.1581206E-1,2.0586884E-1,2.2675432E-1,4.1410267E-2,2.8858256E-1,0E0,6.2045455E-2,8.607215E-2,1.7999732E-1,0E0,3.736546E-2,0E0,0E0,4.718907E-1,0E0,0E0,0E0,0E0,0E0,1.4482278E-1,0E0,0E0,0E0,2.568059E-1,0E0,1.4429104E-1,4.6690717E-2,1.3486227E-1,6.638098E-2,1.5982854E-1,1.3315761E-1,0E0,0E0,8.5588604E-2,0E0,4.496324E-2,0E0,5.349648E-2,0E0,1.2694961E-1,1.05307944E-1,0E0,0E0,0E0,4.908669E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,19,19,25,25,29,29,31,31,32,32,33,33,34,34,35,35,36,36,39,39,41,41,43,43,45,45,46,46,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,34,-1,36,38,40,42,44,46,-1,-1,48,-1,50,-1,52,-1,54,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,7.765071E0,1.577058E9,1.8750688E-2,1.1016051E3,1.8114515E1,4.26944E5,3.3503532E7,4E3,3.768034E4,1.3232165E-2,3.964E3,7.154E3,5.825042E6,4.102122E-3,3.087E3,-4.3530106E-3,1.497196E-5,1.4609149E4,-9.543978E-3,2.7499074E-2,9.195203E-3,-9.530848E-3,-1.2418949E-3,1.3664E4,-1.5931956E-2,-9.225682E-4,-9.807078E-3,1.3210104E9,2.368891E-2,2.09299E6,4.6976192E8,1.7670108E5,1.773296E6,2.551424E6,6.210482E2,9.0794E-3,-5.309415E-4,2.1695537E3,8.131126E-3,2E0,2.1454657E-3,2.88E2,-2.6756634E-3,2.72E2,2.6957834E7,-1.6143502E-4,4.1470123E-3,1.9491435E-3,3.534225E4,-7.896157E-3,-4.6172184E-3,-6.6227913E-3,-7.628198E-4,2.3792102E-3,-2.6486826E-3,2.797376E-3,9.479764E-3],"split_indices":[45,64,47,7,0,64,47,1,57,2,4,0,0,2,59,0,0,0,0,4,0,0,0,0,0,2,0,0,0,7,0,1,7,40,11,9,64,0,0,64,0,8,0,3,0,3,57,0,0,0,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.23E2,2.52E2,1.71E2,2.51E2,1E0,1.51E2,2E1,2.27E2,2.4E1,3E1,1.21E2,1.3E1,7E0,2.6E1,2.01E2,1.9E1,5E0,6E0,2.4E1,1.19E2,2E0,5E0,2E0,2.2E1,4E0,1.99E2,2E0,2E0,3E0,1.16E2,3E0,1.95E2,4E0,5.6E1,6E1,1.18E2,7.7E1,2E0,2E0,4.5E1,1.1E1,5.5E1,5E0,6.3E1,5.5E1,3.7E1,4E1,2.2E1,2.3E1,3E0,5.2E1,3.3E1,3E1,2E1,1.7E1,2.2E1,1.8E1,4E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.7791603E-3,-3.876945E-2,2.1839918E-1,-6.4392515E-2,9.912304E-2,4.197145E-1,1.509777E-1,-8.868081E-2,9.594918E-3,1.2764938E-1,6.935815E-3,9.723304E-3,2.7880026E-2,2.0848687E-1,2.437677E-2,-8.23837E-2,-2.1812852E-2,-1.0962109E-2,7.623083E-3,1.5458927E-1,3.0327065E-2,-6.2850025E-3,2.5282819E-3,2.4070145E-1,9.635124E-5,4.4006975E-3,-1.1020465E-2,-1.5392084E-1,-6.90951E-2,-7.135608E-2,1.8205049E-2,4.7456933E-4,8.390736E-3,3.1535435E-3,-3.9033282E-3,9.669534E-3,3.2523322E-1,-8.683164E-3,-1.2911523E-3,-7.2703786E-2,6.2923315E-3,-1.1397471E-3,-1.2224671E-1,4.4683544E-3,-5.193903E-4,2.0063888E-2,8.4311E-3,-7.515616E-2,1.019415E-2,9.348538E-4,-7.778677E-3,6.872145E-4,-3.3843438E-3,-8.581904E-2,-1.3294231E-2,-5.45577E-3,-2.634318E-3,4.594236E-3,-1.4188427E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,-1,35,-1,-1,-1,37,39,41,43,-1,-1,-1,-1,-1,45,-1,-1,47,-1,-1,49,-1,51,-1,-1,53,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1],"loss_changes":[3.89962E0,1.3278099E0,8.867409E-1,5.68712E-1,1.5427071E-1,4.2026138E-1,3.880763E-1,4.852717E-1,2.3380691E-1,1.1329508E-1,9.3252905E-2,0E0,0E0,2.457416E-1,3.0011904E-1,2.1276271E-1,0E0,1.2460548E-1,0E0,6.5975785E-2,4.4019315E-2,0E0,0E0,6.479895E-2,0E0,0E0,0E0,8.692825E-2,1.4989442E-1,5.403936E-2,6.40434E-2,0E0,0E0,0E0,0E0,0E0,5.160451E-2,0E0,0E0,1.4900148E-1,0E0,0E0,5.469957E-2,0E0,3.8330622E-2,0E0,0E0,1.2814581E-1,0E0,0E0,0E0,0E0,0E0,1.2333858E-1,5.0263893E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,19,19,20,20,23,23,27,27,28,28,29,29,30,30,36,36,39,39,42,42,44,44,47,47,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,-1,36,-1,-1,-1,38,40,42,44,-1,-1,-1,-1,-1,46,-1,-1,48,-1,-1,50,-1,52,-1,-1,54,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.5522031E9,1.417988E6,9.7467445E4,1.3185959E7,9.256843E3,1.5096262E7,2.2233E4,1.7670108E5,1.59E2,2.1853803E4,9.723304E-3,2.7880026E-2,2.2576077E10,1.92144E5,4.15408E5,-2.1812852E-2,2.0926301E3,7.623083E-3,1.504E4,7.4433655E-2,-6.2850025E-3,2.5282819E-3,4.321724E7,9.635124E-5,4.4006975E-3,-1.1020465E-2,7.154E3,3.346087E5,6.3349745E6,7.9E1,4.7456933E-4,8.390736E-3,3.1535435E-3,-3.9033282E-3,9.669534E-3,8.3E1,-8.683164E-3,-1.2911523E-3,1.3428E4,6.2923315E-3,-1.1397471E-3,7.5091325E6,4.4683544E-3,7.1366974E1,2.0063888E-2,8.4311E-3,7.164074E8,1.019415E-2,9.348538E-4,-7.778677E-3,6.872145E-4,-3.3843438E-3,1.732734E6,2.409091E0,-5.45577E-3,-2.634318E-3,4.594236E-3,-1.4188427E-3],"split_indices":[64,7,41,45,57,4,9,2,40,3,45,0,0,12,2,1,0,4,0,9,69,0,0,1,0,0,0,2,40,57,3,0,0,0,0,0,8,0,0,2,0,0,44,0,68,0,0,7,0,0,0,0,0,1,70,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,3.73E2,6.9E1,3.15E2,5.8E1,1.6E1,5.3E1,2.37E2,7.8E1,4.4E1,1.4E1,7E0,9E0,3.6E1,1.7E1,2.34E2,3E0,6.9E1,9E0,3.4E1,1E1,3E0,1.1E1,3.1E1,5E0,1.4E1,3E0,3.5E1,1.99E2,2.2E1,4.7E1,3E0,3.1E1,8E0,2E0,2.2E1,9E0,3E1,5E0,1.96E2,3E0,1.2E1,1E1,9E0,3.8E1,5E0,4E0,1.95E2,1E0,2E0,8E0,3.2E1,6E0,1.66E2,2.9E1,9.6E1,7E1,3E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.3981937E-3,-5.366389E-2,1.298958E-1,-7.399241E-2,2.1687092E-2,1.422995E-1,-1.6672356E-2,-1.1124854E-1,-2.7512455E-2,3.4535494E-2,-9.3004005E-3,1.18673764E-1,1.4719158E-2,-1.3351567E-1,-7.239193E-2,-3.4283455E-2,1.7585479E-2,-6.245318E-5,7.394082E-2,1.2818965E-1,-6.478702E-3,-9.742109E-3,-5.8070845E-3,2.5505287E-5,-8.802599E-2,9.407555E-3,-8.515318E-2,9.161102E-3,1.2293936E-1,1.0740006E-1,2.3086174E-1,-6.2922677E-3,-2.5617743E-3,-7.8645666E-5,6.989779E-3,-6.3888365E-3,-1.2553515E-1,3.1099168E-3,-3.5965915E-3,2.1766622E-3,8.703518E-3,7.817381E-2,1.8352386E-1,2.5169834E-1,1.7908249E-3,-3.1298962E-3,1.3231375E-2,-3.3689619E-3,1.9276812E-3,-1.145071E-2,-1.0334624E-1,1.2354696E-1,4.9479727E-2,1.4677053E-2,7.2534727E-3,1.6497146E-2,8.18641E-3,2.2560188E-3,-1.2535286E-3,2.3066152E-3,-1.16129786E-1,1.5912277E-1,2.178869E-3,-2.7824363E-3,5.791474E-2,-3.8113971E-3,-9.364853E-3,3.011504E-4,8.834757E-3,-1.2868326E-3,3.4633968E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,25,-1,-1,27,29,-1,-1,-1,-1,31,33,35,37,39,41,43,-1,-1,45,-1,47,49,-1,-1,-1,-1,51,53,55,-1,-1,57,-1,-1,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,65,67,-1,-1,69,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2713912E0,4.7185922E-1,8.6385083E-1,4.1600168E-1,1.8262675E-1,4.649496E-1,0E0,1.0627484E-1,2.9206848E-1,8.893033E-2,0E0,3.0489063E-1,0E0,6.669903E-2,5.8172643E-2,2.4083647E-1,0E0,0E0,9.409709E-2,2.270776E-1,0E0,0E0,0E0,0E0,5.203864E-2,7.302477E-2,1.5823093E-1,6.475091E-2,6.1531126E-2,2.0813453E-1,6.819165E-2,0E0,0E0,4.7450423E-2,0E0,5.1985607E-2,5.2172065E-2,0E0,0E0,0E0,0E0,9.0230584E-2,7.154226E-2,6.307316E-2,0E0,0E0,5.859545E-2,0E0,0E0,0E0,6.0783178E-2,7.2701246E-2,4.4912703E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.2407494E-2,5.029595E-2,0E0,0E0,4.3699875E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,15,15,18,18,19,19,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,36,36,41,41,42,42,43,43,46,46,50,50,51,51,52,52,60,60,61,61,64,64],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,26,-1,-1,28,30,-1,-1,-1,-1,32,34,36,38,40,42,44,-1,-1,46,-1,48,50,-1,-1,-1,-1,52,54,56,-1,-1,58,-1,-1,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,66,68,-1,-1,70,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.164074E8,2.3028242E3,1.732734E6,1E0,6.001104E9,-1.6672356E-2,7.6E1,4.6454727E3,1.1576422E9,-9.3004005E-3,1.2622761E8,1.4719158E-2,3.133E3,8.9E1,1.7239808E7,1.7585479E-2,-6.245318E-5,2.9222983E3,6.204522E3,-6.478702E-3,-9.742109E-3,-5.8070845E-3,2.5505287E-5,3.006846E6,5.0543875E-1,1.4E4,5.78E2,1E0,7.2473733E9,3.5301748E7,-6.2922677E-3,-2.5617743E-3,2.2737327E0,6.989779E-3,1.534E3,6.7440504E7,3.1099168E-3,-3.5965915E-3,2.1766622E-3,8.703518E-3,8.16997E6,9.151661E6,3.69814E6,1.7908249E-3,-3.1298962E-3,2.1813516E-2,-3.3689619E-3,1.9276812E-3,-1.145071E-2,9.97639E-6,1.52E2,2.1853803E4,1.4677053E-2,7.2534727E-3,1.6497146E-2,8.18641E-3,2.2560188E-3,-1.2535286E-3,2.3066152E-3,3.069012E6,6.287E3,2.178869E-3,-2.7824363E-3,5.767668E0,-3.8113971E-3,-9.364853E-3,3.011504E-4,8.834757E-3,-1.2868326E-3,3.4633968E-3],"split_indices":[2,7,70,1,19,7,0,8,4,7,0,1,0,2,8,57,0,0,4,64,0,0,0,0,9,69,11,8,26,12,57,0,0,65,0,11,7,0,0,0,0,57,63,59,0,0,69,0,0,0,50,3,45,0,0,0,0,0,0,0,9,11,0,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.06E2,1.41E2,2.41E2,6.5E1,1.38E2,3E0,1.33E2,1.08E2,6.2E1,3E0,1.21E2,1.7E1,8.3E1,5E1,1.07E2,1E0,3.3E1,2.9E1,1.17E2,4E0,1.6E1,6.7E1,9E0,4.1E1,5.8E1,4.9E1,1.3E1,1.6E1,9.9E1,1.8E1,1.9E1,2.2E1,5.5E1,3E0,1.7E1,3.2E1,8E0,5E0,7E0,9E0,7.3E1,2.6E1,1.6E1,2E0,9E0,4.6E1,7E0,1E1,4E0,2.8E1,2.7E1,4.6E1,5E0,2.1E1,7E0,9E0,2.5E1,2.1E1,2E0,2.6E1,1.8E1,9E0,3E0,4.3E1,1.8E1,8E0,2E0,1.6E1,5E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[7.979123E-3,-6.958848E-2,1.0561463E-1,-6.238112E-2,-1.963937E-2,5.514976E-2,2.5690955E-1,-1.09892555E-1,-3.4030605E-2,-3.2504003E-2,8.385704E-2,2.6674887E-2,1.8643518E-1,-1.1569285E-1,9.718655E-3,-3.8098093E-2,1.4090513E-2,-1.3982342E-2,-8.854848E-3,4.6036143E-2,1.1815436E-1,2.1883409E-1,-2.8231194E-2,-1.4651564E-1,-4.291283E-2,-5.804345E-2,2.9771794E-3,-4.869472E-3,6.1138254E-4,7.2105194E-4,5.603442E-3,1.2988494E-1,8.738465E-4,1.1519298E-2,1.0741537E-3,-6.9388556E-3,3.0878147E-3,-2.00759E-3,-1.5545295E-1,2.4806203E-3,-3.0568568E-3,-7.613656E-2,2.7605334E-2,3.2494925E-2,-7.313825E-2,9.2955925E-2,1.6309573E-1,-1.0220464E-2,-6.3000624E-3,-4.8556764E-2,-1.15616955E-1,5.0903847E-3,-1.8008241E-2,3.170746E-3,-2.9129567E-4,-1.3175979E-3,-7.599262E-3,5.2687745E-3,-1.8885626E-3,9.3981E-3,2.6350247E-3,-9.8031096E-2,-2.7811804E-3,-2.5810536E-3,-1.4525501E-1,-4.437872E-3,1.4236979E-3,-7.470877E-4,-6.9285785E-3,-1.59672E-3,2.9538753E-3,-9.2616575E-3,-2.5974023E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,-1,25,-1,27,-1,29,31,33,35,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,51,53,55,57,59,-1,-1,61,63,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,67,69,-1,71,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2335978E0,5.302081E-1,1.4288611E0,3.1145215E-1,0E0,3.625567E-1,8.1747746E-1,1.7608535E-1,2.0095526E-1,9.399077E-2,1.3465345E-1,0E0,2.7742112E-1,1.866349E-1,0E0,1.2060416E-1,0E0,7.338969E-2,0E0,1.09092414E-1,6.38597E-2,7.3293924E-2,6.832053E-2,5.078149E-2,4.8648488E-2,1.555174E-1,1.1246768E-1,0E0,0E0,0E0,0E0,4.6161354E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.3516865E-2,0E0,0E0,8.367261E-2,6.2797695E-2,4.3070458E-2,4.7670186E-2,4.63742E-2,5.89453E-2,0E0,0E0,1.1296316E-1,5.5065572E-2,0E0,4.20279E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.889636E-2,5.062745E-2,0E0,7.259363E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,38,38,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,52,52,61,61,62,62,64,64],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,-1,26,-1,28,-1,30,32,34,36,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,52,54,56,58,60,-1,-1,62,64,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,68,70,-1,72,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.393928E4,7.339209E6,9.3058E4,9.50639E5,-1.963937E-2,9.9800183E2,1.681178E6,1.0043E4,8.861901E3,1E0,2.325945E9,2.6674887E-2,1.5096262E7,3.006846E6,9.718655E-3,1.0975164E3,1.4090513E-2,3.957E3,-8.854848E-3,1.29186E9,2.192772E10,3.0113987E10,3.677E3,6.738547E7,2.409091E0,3.94E2,3.653602E7,-4.869472E-3,6.1138254E-4,7.2105194E-4,5.603442E-3,1.334086E5,8.738465E-4,1.1519298E-2,1.0741537E-3,-6.9388556E-3,3.0878147E-3,-2.00759E-3,3.606E3,2.4806203E-3,-3.0568568E-3,8.593763E7,4.16606E5,7.2E1,7.494142E6,4.17E2,2.5519034E7,-1.0220464E-2,-6.3000624E-3,2.09299E6,1.1004841E9,5.0903847E-3,2.6490103E1,3.170746E-3,-2.9129567E-4,-1.3175979E-3,-7.599262E-3,5.2687745E-3,-1.8885626E-3,9.3981E-3,2.6350247E-3,5.91294E5,2.53383E5,-2.5810536E-3,1.16985195E5,-4.437872E-3,1.4236979E-3,-7.470877E-4,-6.9285785E-3,-1.59672E-3,2.9538753E-3,-9.2616575E-3,-2.5974023E-3],"split_indices":[45,59,2,1,0,64,41,2,64,19,5,0,9,9,0,64,0,2,0,7,5,12,0,7,70,8,57,0,0,0,0,40,0,0,0,0,0,0,2,0,0,7,41,8,1,3,9,0,0,1,5,0,70,0,0,0,0,0,0,0,0,41,9,0,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,2.37E2,1.88E2,2.33E2,4E0,1.42E2,4.6E1,8.6E1,1.47E2,3.5E1,1.07E2,8E0,3.8E1,8.5E1,1E0,1.46E2,1E0,3.2E1,3E0,5.2E1,5.5E1,3.3E1,5E0,5.9E1,2.6E1,9.8E1,4.8E1,7E0,2.5E1,3.6E1,1.6E1,4.9E1,6E0,3.1E1,2E0,2E0,3E0,5E0,5.4E1,4E0,2.2E1,8.1E1,1.7E1,3.5E1,1.3E1,2.5E1,2.4E1,1.8E1,3.6E1,4.9E1,3.2E1,6E0,1.1E1,1.9E1,1.6E1,9E0,4E0,2.3E1,2E0,1.9E1,5E0,2.3E1,2.6E1,1.1E1,2.1E1,4E0,7E0,8E0,1.5E1,1.8E1,8E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[9.305403E-3,-5.4016035E-2,1.21552825E-1,-1.1848685E-1,-3.3374026E-2,8.2017794E-2,3.4051996E-1,-1.318432E-1,-1.8636568E-3,-3.9393086E-2,4.6955533E-3,9.62635E-2,-1.5516191E-2,2.7030058E-2,2.0654626E-1,-7.6234695E-3,-3.3970748E-3,-6.5418184E-2,-1.3239449E-2,5.9112474E-2,1.5052421E-1,3.4680734E-3,1.2318133E-2,-1.0822117E-1,-3.166306E-2,1.3496502E-2,-5.4650705E-2,-7.814791E-3,6.9538705E-2,-1.3629463E-2,1.6064602E-1,-1.5090683E-1,-6.0830254E-2,-4.2783473E-2,1.7427024E-3,3.2116822E-3,5.61637E-3,-8.648985E-3,-3.7172068E-2,1.0156484E-1,4.5747947E-2,4.8482283E-3,-4.1403486E-3,1.9462213E-1,4.4694473E-3,-3.0355821E-3,-9.323449E-3,-1.91301E-3,-9.91604E-3,-4.170164E-6,-3.2050181E-3,3.1411946E-3,-1.335345E-2,-7.001184E-3,-2.045191E-2,1.22105524E-1,1.2333898E-3,-4.300899E-3,5.6945164E-2,1.0117019E-2,-1.3858333E-3,-4.274658E-2,2.8464606E-2,3.0407312E-3,-4.890026E-2,3.9898884E-4,1.3730472E-1,2.0047787E-3,5.8006984E-3,-4.17755E-3,9.334796E-4,8.714481E-5,5.411335E-3,-4.036655E-3,3.49039E-4,3.4255232E-3,8.580945E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,-1,21,-1,-1,23,25,27,29,-1,-1,31,33,35,37,-1,39,41,43,45,47,49,-1,51,-1,-1,53,55,57,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,63,65,-1,-1,67,-1,-1,69,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1339397E0,3.7133318E-1,1.3485141E0,6.8828166E-2,1.6847876E-1,7.903035E-1,5.32454E-1,6.630921E-2,0E0,1.39348E-1,0E0,2.571931E-1,0E0,0E0,7.052845E-2,0E0,0E0,1.4538988E-1,1.1588321E-1,1.9060796E-1,9.025073E-2,0E0,0E0,8.18513E-2,4.5144998E-2,6.472775E-2,7.842364E-2,0E0,5.4021925E-2,3.866988E-2,1.0357392E-1,6.260294E-2,6.3769154E-2,4.6483256E-2,0E0,5.9130266E-2,0E0,0E0,6.0805045E-2,4.752329E-2,7.27908E-2,0E0,0E0,6.2957525E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.8987454E-2,0E0,7.929526E-2,4.226783E-2,0E0,0E0,3.8132593E-2,0E0,0E0,7.1809754E-2,4.2465128E-2,0E0,4.5092065E-2,0E0,4.026419E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,35,35,38,38,39,39,40,40,43,43,52,52,54,54,55,55,58,58,61,61,62,62,64,64,66,66],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,-1,22,-1,-1,24,26,28,30,-1,-1,32,34,36,38,-1,40,42,44,46,48,50,-1,52,-1,-1,54,56,58,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,64,66,-1,-1,68,-1,-1,70,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,8.10288E3,2.7299898E8,1.577058E9,4.673502E7,3.0418E4,8.1672195E4,-1.8636568E-3,2.025313E6,4.6955533E-3,4.1331047E-1,-1.5516191E-2,2.7030058E-2,2.9169512E5,-7.6234695E-3,-3.3970748E-3,2.551424E6,1.8071064E7,1.293844E0,2.2782404E0,3.4680734E-3,1.2318133E-2,1E0,6.03E2,5.0543875E-1,1.9120918E7,-7.814791E-3,8.16997E6,2.722E3,9.705148E-1,4.52737E5,1.09014104E8,1.039769E6,1.7427024E-3,1.6176678E3,5.61637E-3,-8.648985E-3,7.0151276E1,7.4433655E-2,2.6248154E3,4.8482283E-3,-4.1403486E-3,3.020675E5,4.4694473E-3,-3.0355821E-3,-9.323449E-3,-1.91301E-3,-9.91604E-3,-4.170164E-6,-3.2050181E-3,3.1411946E-3,1.656968E6,-7.001184E-3,1.53042E5,3.44E2,1.2333898E-3,-4.300899E-3,4.23168E3,1.0117019E-2,-1.3858333E-3,1.152E4,1.1762006E9,3.0407312E-3,2.058478E7,3.9898884E-4,4.884611E1,2.0047787E-3,5.8006984E-3,-4.17755E-3,9.334796E-4,8.714481E-5,5.411335E-3,-4.036655E-3,3.49039E-4,3.4255232E-3,8.580945E-3],"split_indices":[2,1,64,7,7,60,10,40,0,1,0,47,0,0,44,0,0,9,57,65,66,0,0,26,8,69,57,0,57,0,46,41,44,1,0,45,0,0,60,69,45,0,0,45,0,0,0,0,0,0,0,0,9,0,12,0,0,0,64,0,0,10,12,0,44,0,68,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,2.81E2,1.58E2,6.7E1,2.14E2,1.35E2,2.3E1,5.7E1,1E1,2.05E2,9E0,1.31E2,4E0,8E0,1.5E1,4.2E1,1.5E1,1.02E2,1.03E2,7.9E1,5.2E1,4E0,1.1E1,4.4E1,5.8E1,6.3E1,4E1,3E0,7.6E1,3E0,4.9E1,2.2E1,2.2E1,5E1,8E0,5.8E1,5E0,4E0,3.6E1,3.1E1,4.5E1,1E0,2E0,3.2E1,1.7E1,7E0,1.5E1,2E1,2E0,1.7E1,3.3E1,1.2E1,4.6E1,4E0,3.2E1,2.4E1,7E0,3E0,4.2E1,3.1E1,1E0,2.7E1,1.9E1,8E0,2.4E1,3E0,2.1E1,3.4E1,8E0,1.6E1,1.1E1,1.5E1,4E0,1.5E1,9E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[3.5049938E-4,-4.9051628E-2,1.1725362E-1,-9.582418E-2,-1.980691E-2,1.0032905E-1,2.0126745E-2,-1.1352916E-1,-1.6989084E-2,-2.5552236E-2,1.2824435E-2,1.13377996E-1,-1.3583568E-1,-1.4107162E-1,-5.0339695E-2,-6.161619E-5,-6.0780537E-3,-4.1804936E-2,4.2345475E-2,8.793043E-2,1.9294219E-1,-9.290382E-3,-3.5594657E-4,-1.4938375E-1,-4.0602504E-4,3.4829488E-4,-4.8949197E-3,-5.220551E-2,2.9969169E-2,1.7358916E-2,7.344029E-3,1.2719208E-1,4.9290765E-2,-6.5178975E-2,2.3867817E-1,-7.7102073E-3,-3.699678E-6,-6.1885342E-2,-2.7991377E-3,-2.6278704E-4,5.64944E-3,7.1505643E-3,-6.1028833E-5,9.154261E-3,8.666187E-2,1.2838417E-3,5.2361996E-3,1.6890946E-3,-7.1180784E-3,1.9115858E-2,9.030614E-3,-4.8920758E-2,-1.424616E-1,2.589322E-3,-2.857444E-3,3.0204253E-3,7.831318E-3,-4.0444978E-2,-1.1946312E-1,-8.47109E-3,-1.2855014E-3,-4.467584E-3,-1.200106E-3,-8.759733E-3,4.037847E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,37,39,41,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5468671E0,4.2213553E-1,5.925242E-1,1.6416824E-1,3.1147474E-1,4.006132E-1,0E0,1.617167E-1,3.7767176E-2,2.103357E-1,0E0,2.2397554E-1,4.378049E-2,7.2826385E-2,8.525749E-2,0E0,0E0,1.1577785E-1,9.415706E-2,1.3329977E-1,3.4610868E-1,0E0,0E0,4.6025753E-2,0E0,0E0,0E0,6.391168E-2,5.912386E-2,7.25877E-2,0E0,8.622694E-2,6.0164362E-2,4.304408E-2,1.3101208E-1,0E0,0E0,1.10030025E-1,7.126208E-2,0E0,0E0,0E0,0E0,0E0,4.1958302E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.3762645E-2,4.2991698E-2,0E0,0E0,0E0,0E0,6.943664E-2,7.26046E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,27,27,28,28,29,29,31,31,32,32,33,33,34,34,37,37,38,38,44,44,51,51,52,52,57,57,58,58],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,38,40,42,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,4.844E3,1.0871896E10,2.0964778E5,3.058642E3,9.05929E2,2.0126745E-2,3.006846E6,9.30419E5,3.53E2,1.2824435E-2,6.204522E3,1.31805E5,1.3421E4,1.19E2,-6.161619E-5,-6.0780537E-3,1.4998456E5,1.3738772E5,1.1291153E7,5.3140095E1,-9.290382E-3,-3.5594657E-4,3.5111064E7,-4.0602504E-4,3.4829488E-4,-4.8949197E-3,1.7985837E3,1.3412494E6,1.12608824E8,7.344029E-3,1.50301E6,1.2393866E2,1.0014992E8,2.1172E4,-7.7102073E-3,-3.699678E-6,1E0,6.7211235E1,-2.6278704E-4,5.64944E-3,7.1505643E-3,-6.1028833E-5,9.154261E-3,2.5501363E9,1.2838417E-3,5.2361996E-3,1.6890946E-3,-7.1180784E-3,1.9115858E-2,9.030614E-3,3.9034148E7,3.77E2,2.589322E-3,-2.857444E-3,3.0204253E-3,7.831318E-3,3.34E2,1.4171124E3,-8.47109E-3,-1.2855014E-3,-4.467584E-3,-1.200106E-3,-8.759733E-3,4.037847E-4],"split_indices":[2,2,7,40,64,70,0,9,9,8,0,64,40,10,3,0,0,40,45,57,68,0,0,57,0,0,0,64,60,5,0,9,70,1,10,0,0,19,68,0,0,0,0,0,7,0,0,0,0,0,0,57,3,0,0,0,0,0,64,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.09E2,1.3E2,1.18E2,1.91E2,1.24E2,6E0,9.6E1,2.2E1,1.88E2,3E0,1.18E2,6E0,6.6E1,3E1,2E1,2E0,1.52E2,3.6E1,9.1E1,2.7E1,4E0,2E0,6.2E1,4E0,1.4E1,1.6E1,1.33E2,1.9E1,3E1,6E0,4.4E1,4.7E1,4E0,2.3E1,6E1,2E0,1.11E2,2.2E1,1.4E1,5E0,3E0,2.7E1,1.7E1,2.7E1,3.4E1,1.3E1,2E0,2E0,5E0,1.8E1,9.7E1,1.4E1,1.1E1,1.1E1,2.1E1,6E0,8.8E1,9E0,1.1E1,3E0,2.1E1,6.7E1,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-5.343586E-3,-6.0238432E-2,7.3978655E-2,-5.5175144E-2,-1.8515898E-2,2.5159422E-2,1.4045288E-1,-1.0111012E-1,-3.0238695E-2,-3.4809798E-2,4.961124E-2,9.180722E-2,2.1612665E-1,-1.1652162E-1,-1.628455E-2,-3.3435658E-2,1.2585321E-2,-1.9319358E-4,-7.014219E-3,1.112606E-2,8.883586E-2,1.08493365E-1,-9.727819E-3,1.6305342E-1,2.5341421E-2,-7.6062037E-3,-8.060897E-2,-2.6466285E-3,2.3764523E-3,-4.3950394E-2,3.4306634E-2,-2.309749E-2,3.086823E-3,7.2743637E-3,1.9335019E-3,8.555502E-3,4.314024E-3,-1.5590776E-3,8.599449E-3,7.415862E-4,-1.0493463E-1,-2.7480425E-2,-8.9603186E-2,2.9606454E-3,-4.9447864E-3,-2.6438022E-3,1.6219004E-3,-6.5997257E-3,-1.6021406E-3,-1.0580803E-2,-8.618824E-2,-5.627741E-3,-1.4804305E-4,1.3065879E-2,-4.6435118E-2,-2.4611312E-3,-9.43582E-3,-2.557373E-3,1.576671E-3,-3.992351E-3,-6.1288744E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,31,33,35,-1,37,-1,-1,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,47,49,51,-1,-1,-1,-1,-1,-1,53,55,-1,-1,57,59,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9160302E0,3.8799703E-1,5.806061E-1,2.9132456E-1,0E0,1.556944E-1,2.599293E-1,1.1575854E-1,1.5991443E-1,1.00700065E-1,1.1222908E-1,2.5055143E-1,3.6511958E-1,8.51022E-2,3.763463E-2,1.2046063E-1,0E0,0E0,0E0,6.901671E-2,9.96117E-2,4.916829E-2,0E0,5.091268E-2,0E0,0E0,9.443033E-2,0E0,0E0,1.0656318E-1,8.2279675E-2,4.145207E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.90187E-2,1.0623918E-1,7.54087E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.267274E-2,8.16845E-2,0E0,0E0,6.335315E-2,3.78837E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,23,23,26,26,29,29,30,30,31,31,40,40,41,41,42,42,49,49,50,50,53,53,54,54],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,32,34,36,-1,38,-1,-1,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,48,50,52,-1,-1,-1,-1,-1,-1,54,56,-1,-1,58,60,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,2.5096E4,4.407E3,-1.8515898E-2,2.0231652E3,1.5452943E0,2.0964778E5,8.861901E3,1.1291153E7,9.2849056E8,4.52E2,3.305542E4,2.53383E5,4.174811E2,1.577058E9,1.2585321E-2,-1.9319358E-4,-7.014219E-3,1.07E3,7.625084E6,1.0548451E7,-9.727819E-3,1.4797951E-2,2.5341421E-2,-7.6062037E-3,2.1016268E-2,-2.6466285E-3,2.3764523E-3,4.6263345E-2,1.4241472E3,9.6E1,3.086823E-3,7.2743637E-3,1.9335019E-3,8.555502E-3,4.314024E-3,-1.5590776E-3,8.599449E-3,7.415862E-4,1.6776951E3,3.0840238E7,1.6120845E5,2.9606454E-3,-4.9447864E-3,-2.6438022E-3,1.6219004E-3,-6.5997257E-3,-1.6021406E-3,1.0062742E2,6.54925E6,-5.627741E-3,-1.4804305E-4,4.5765094E2,3.069055E6,-2.4611312E-3,-9.43582E-3,-2.557373E-3,1.576671E-3,-3.992351E-3,-6.1288744E-4],"split_indices":[45,59,2,2,0,67,51,40,64,57,7,3,4,9,45,7,0,0,0,0,60,57,0,69,0,0,51,0,0,69,70,8,0,0,0,0,0,0,0,0,4,57,40,0,0,0,0,0,0,66,9,0,0,64,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,2.59E2,1.79E2,2.56E2,3E0,1.04E2,7.5E1,8.9E1,1.67E2,3E1,7.4E1,4.7E1,2.8E1,7.5E1,1.4E1,1.66E2,1E0,2.4E1,6E0,3.8E1,3.6E1,4.5E1,2E0,2.5E1,3E0,3.6E1,3.9E1,9E0,5E0,1.44E2,2.2E1,2.3E1,1.5E1,1.6E1,2E1,1E1,3.5E1,1E0,2.4E1,8E0,3.1E1,1.07E2,3.7E1,1.9E1,3E0,1.5E1,8E0,2.2E1,9E0,8.4E1,2.3E1,2.9E1,8E0,5.1E1,3.3E1,1.8E1,5E0,1.1E1,4E1,1.6E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[2.81754E-3,-3.865204E-2,1.1897957E-1,-9.380661E-2,-1.3721979E-2,8.6210854E-2,2.477309E-1,-1.2186428E-1,-4.198079E-2,-2.6082521E-2,4.7791287E-2,1.01563305E-1,-2.0737547E-1,1.5459806E-1,2.4280427E-2,-1.3208808E-1,-9.137162E-4,9.5844455E-4,-3.428025E-3,-6.613803E-2,-3.3564316E-3,5.962589E-3,1.0194378E-2,2.2037597E-1,8.04591E-2,-1.3263213E-2,6.16893E-4,8.561089E-3,-3.2334009E-3,-6.9519817E-3,-1.4228976E-3,-2.2631954E-2,-8.787153E-2,-1.3570071E-2,5.5659353E-3,-3.5176654E-3,2.4947412E-3,1.9352641E-2,1.03188604E-1,2.1141209E-3,1.2159726E-1,-4.4853375E-3,5.139614E-4,-2.2214537E-3,-1.1517269E-1,2.8771344E-3,-6.074232E-2,-1.3534238E-3,7.2104186E-3,-4.5213387E-3,1.4296997E-1,-1.1080875E-2,-4.348017E-3,-1.5337454E-3,1.778965E-3,-2.0411878E-3,-7.4009066E-3,1.7334595E-1,4.026411E-3,1.9516768E-1,3.004489E-3,4.1581163E-5,1.0322573E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,41,43,45,-1,-1,-1,-1,47,-1,49,-1,-1,-1,51,53,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,59,-1,61,-1,-1,-1],"loss_changes":[2.1241624E0,4.4559002E-1,4.680488E-1,1.4174938E-1,1.7232242E-1,4.4474512E-1,4.3493474E-1,6.641966E-2,6.1536357E-2,1.7091537E-1,1.0095959E-1,2.0793903E-1,6.70118E-2,7.702616E-2,0E0,3.894341E-2,0E0,0E0,0E0,6.2307954E-2,1.4329846E-1,0E0,8.674066E-2,2.1424508E-1,1.1729169E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.3945087E-2,4.822573E-2,8.7070145E-2,0E0,0E0,0E0,0E0,5.208098E-2,0E0,1.8063521E-1,0E0,0E0,0E0,6.1753273E-2,9.3347624E-2,4.4356823E-2,0E0,0E0,0E0,5.0483048E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.262483E-2,0E0,3.9644778E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,22,22,23,23,24,24,31,31,32,32,33,33,38,38,40,40,44,44,45,45,46,46,50,50,57,57,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,42,44,46,-1,-1,-1,-1,48,-1,50,-1,-1,-1,52,54,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,60,-1,62,-1,-1,-1],"split_conditions":[1.7267E4,9.26787E5,2.1809912E0,3.006846E6,3.53E2,3.768034E4,1.2123712E1,1.7626338E6,1.19E2,7.283496E2,4.16606E5,7.801943E3,9.652E3,1.3936486E8,2.4280427E-2,1.4921534E5,-9.137162E-4,9.5844455E-4,-3.428025E-3,4.1E1,2.5653894E5,5.962589E-3,1.5292238E5,4.8E1,6.0891E4,-1.3263213E-2,6.16893E-4,8.561089E-3,-3.2334009E-3,-6.9519817E-3,-1.4228976E-3,1E0,6.17296E5,2.7582266E7,5.5659353E-3,-3.5176654E-3,2.4947412E-3,1.9352641E-2,3E0,2.1141209E-3,1.5165479E0,-4.4853375E-3,5.139614E-4,-2.2214537E-3,1.874E3,1.1639729E9,6.54925E6,-1.3534238E-3,7.2104186E-3,-4.5213387E-3,4.128514E-1,-1.1080875E-2,-4.348017E-3,-1.5337454E-3,1.778965E-3,-2.0411878E-3,-7.4009066E-3,1.3590306E10,4.026411E-3,1E0,3.004489E-3,4.1581163E-5,1.0322573E-2],"split_indices":[2,1,54,9,8,4,65,62,3,64,41,44,0,1,0,40,0,0,0,8,40,0,59,3,2,0,0,0,0,0,0,14,41,57,0,0,0,0,8,0,65,0,0,0,11,5,9,0,0,0,69,0,0,0,0,0,0,12,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.24E2,1.15E2,1E2,2.24E2,9.3E1,2.2E1,6.4E1,3.6E1,1.87E2,3.7E1,8.9E1,4E0,1.7E1,5E0,5.8E1,6E0,1.1E1,2.5E1,6.7E1,1.2E2,1.2E1,2.5E1,1.2E1,7.7E1,3E0,1E0,1.6E1,1E0,5.4E1,4E0,2.3E1,4.4E1,1.11E2,9E0,8E0,1.7E1,4E0,8E0,4.1E1,3.6E1,7E0,1.6E1,1.8E1,2.6E1,8.3E1,2.8E1,2E0,6E0,3E0,3.3E1,4E0,2.2E1,4.1E1,4.2E1,2.4E1,4E0,2.1E1,1.2E1,1.7E1,4E0,1E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[5.544589E-3,-3.512823E-2,1.0414052E-1,-9.243788E-2,-1.8505335E-2,-1.5029754E-1,1.24820516E-1,-1.0577108E-1,-1.3426739E-3,-3.0529955E-2,3.7294213E-2,1.4248106E-3,-1.6732337E-2,9.522637E-2,2.3110168E-1,-5.9993328E-3,-1.757704E-3,-5.89731E-2,-1.3808691E-2,5.7297708E-3,2.1651333E-2,5.371469E-2,1.2917319E-1,4.169747E-1,1.19104095E-1,-6.796784E-3,-4.311335E-2,-4.7851186E-2,9.217308E-3,-2.7165072E-2,2.4600944E-3,4.726046E-3,1.4654844E-3,1.0412661E-2,9.660243E-2,2.2237627E-2,4.8239273E-3,1.6246264E-1,-6.8662115E-3,-1.2421554E-3,-4.930933E-3,-7.008409E-5,-7.777294E-2,4.2907475E-3,-3.6954654E-3,5.427002E-4,-4.8866915E-3,1.1576109E-3,5.7679447E-3,9.287843E-3,3.489136E-4,-6.5392083E-3,-2.1009946E-3,1.0022652E-2,-5.5364378E-2,4.7005154E-2,-6.010395E-4,-3.894747E-3,2.6668124E-3,5.3358534E-3,6.879782E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,-1,-1,25,27,-1,29,31,33,35,37,-1,39,41,43,45,-1,-1,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1],"loss_changes":[1.7440737E0,2.9170623E-1,6.870061E-1,5.7483554E-2,1.62171E-1,3.3946165E-1,3.4913838E-1,5.3215504E-2,0E0,9.335552E-2,4.9550723E-2,0E0,0E0,1.255253E-1,4.7077E-1,0E0,0E0,8.331618E-2,9.920773E-2,0E0,5.1141746E-2,4.1068897E-2,1.1139864E-1,3.777623E-2,2.1138123E-1,0E0,6.043408E-2,7.077228E-2,7.552826E-2,3.8653273E-2,0E0,0E0,0E0,0E0,4.982406E-2,0E0,0E0,5.323693E-2,0E0,0E0,0E0,0E0,5.329019E-2,0E0,4.7335714E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3783333E-2,3.8510602E-2,3.8259458E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,17,17,18,18,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,34,34,37,37,42,42,44,44,53,53,54,54,55,55],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,-1,-1,26,28,-1,30,32,34,36,38,-1,40,42,44,46,-1,-1,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.41718E5,2.4950776E0,9.703059E6,3.53E2,6.27427E5,9.571874E3,3.91E2,-1.3426739E-3,1.732734E6,2.25161E5,1.4248106E-3,-1.6732337E-2,4.1331047E-1,3.3616E4,-5.9993328E-3,-1.757704E-3,2.8E1,1E0,5.7297708E-3,1.570662E5,1.2394286E9,2.1207E4,2.1592189E5,5.22E2,-6.796784E-3,6.03257E5,1.7377544E7,9.639932E0,3.4129692E-3,2.4600944E-3,4.726046E-3,1.4654844E-3,1.0412661E-2,2.9651852E0,2.2237627E-2,4.8239273E-3,1.0038E4,-6.8662115E-3,-1.2421554E-3,-4.930933E-3,-7.008409E-5,7.121916E2,4.2907475E-3,2.3177492E6,5.427002E-4,-4.8866915E-3,1.1576109E-3,5.7679447E-3,9.287843E-3,3.489136E-4,-6.5392083E-3,-2.1009946E-3,1.0270569E7,4.735985E6,1.3482595E8,-6.010395E-4,-3.894747E-3,2.6668124E-3,5.3358534E-3,6.879782E-4],"split_indices":[2,1,66,9,8,9,4,3,0,1,41,0,0,47,10,0,0,8,13,0,59,7,9,40,3,0,41,57,70,69,0,0,0,0,65,0,0,0,0,0,0,0,64,0,59,0,0,0,0,0,0,0,0,57,1,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.07E2,1.26E2,6.8E1,2.39E2,9E0,1.17E2,5.6E1,1.2E1,1.97E2,4.2E1,5E0,4E0,9.3E1,2.4E1,4.6E1,1E1,7.2E1,1.25E2,6E0,3.6E1,4.3E1,5E1,8E0,1.6E1,1.1E1,6.1E1,5E1,7.5E1,1.3E1,2.3E1,1.5E1,2.8E1,1.3E1,3.7E1,7E0,1E0,1.4E1,2E0,4.7E1,1.4E1,2E1,3E1,1E1,6.5E1,9E0,4E0,8E0,2.9E1,1.2E1,2E0,1.1E1,1.9E1,5.2E1,1.3E1,1.9E1,3.3E1,1.1E1,2E0,6E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[1.9057722E-4,-4.3841608E-2,9.4081335E-2,-1.073289E-1,-3.112049E-2,1.1287331E-1,-1.5398487E-1,-7.144994E-3,-3.2850541E-3,-6.172148E-2,-1.30347125E-2,9.499429E-2,1.46319885E-2,-2.5451863E-1,8.267714E-3,-4.3367893E-2,-1.19960986E-1,-2.7721755E-2,1.5484613E-3,1.03856444E-1,-1.2428917E-1,-1.5503317E-2,-2.9129505E-3,-5.136403E-2,1.7958932E-3,1.8471581E-3,-7.949092E-3,-4.6263598E-2,1.3117957E-2,9.361801E-2,2.2392258E-2,6.915568E-3,-1.1222142E-2,-7.501377E-2,-5.7321985E-4,-1.6360085E-2,-7.888432E-2,-1.2085952E-3,5.2499683E-3,2.5931022E-3,1.20249115E-1,-7.0243017E-3,-4.8482165E-2,-3.6388896E-2,1.9197523E-3,-2.2173068E-3,-5.735838E-3,1.4044355E-3,-3.1177495E-3,1.2897697E-2,9.986786E-2,-6.7056715E-4,-4.377773E-3,-8.7089435E-4,-5.078734E-3,5.425433E-3,1.7690935E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,21,-1,23,25,27,-1,29,31,-1,-1,33,-1,-1,-1,35,37,39,-1,-1,-1,41,-1,43,45,47,-1,-1,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7611661E0,2.308724E-1,6.5136576E-1,6.1760783E-2,1.3368206E-1,3.8068974E-1,3.6322904E-1,0E0,0E0,9.099886E-2,1.0042076E-1,2.4027705E-1,0E0,6.935328E-2,0E0,4.623398E-2,1.3471735E-1,8.8447124E-2,0E0,3.5627663E-1,1.6274893E-1,0E0,0E0,5.952272E-2,0E0,0E0,0E0,7.67474E-2,4.8858315E-2,1.1712009E-1,0E0,0E0,0E0,6.2740415E-2,0E0,4.8570897E-2,4.1979298E-2,6.0926147E-2,0E0,0E0,1.6243702E-1,0E0,3.9971158E-2,3.781368E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.98569E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,13,13,15,15,16,16,17,17,19,19,20,20,23,23,27,27,28,28,29,29,33,33,35,35,36,36,37,37,40,40,42,42,43,43,50,50],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,22,-1,24,26,28,-1,30,32,-1,-1,34,-1,-1,-1,36,38,40,-1,-1,-1,42,-1,44,46,48,-1,-1,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,5.53564E5,1.0429407E3,3.606E3,6.505339E2,1.6941456E1,3.236383E1,-7.144994E-3,-3.2850541E-3,1.7377544E7,1.2472753E5,3.7826266E4,1.46319885E-2,3.2991206E3,8.267714E-3,1.6285607E7,6.465659E4,7.10711E8,1.5484613E-3,3.305542E4,1.92144E5,-1.5503317E-2,-2.9129505E-3,3.006846E6,1.7958932E-3,1.8471581E-3,-7.949092E-3,2.6366013E1,9.78696E4,1.2511627E7,2.2392258E-2,6.915568E-3,-1.1222142E-2,4.9256845E2,-5.7321985E-4,6.0070217E1,1E0,2.1035208E7,5.2499683E-3,2.5931022E-3,1.6459E4,-7.0243017E-3,4.8E1,4.086885E7,1.9197523E-3,-2.2173068E-3,-5.735838E-3,1.4044355E-3,-3.1177495E-3,1.2897697E-2,6.029086E10,-6.7056715E-4,-4.377773E-3,-8.7089435E-4,-5.078734E-3,5.425433E-3,1.7690935E-4],"split_indices":[2,1,70,2,64,47,68,0,0,57,40,64,0,64,0,57,40,7,0,4,2,0,0,9,0,0,0,65,40,1,0,0,0,45,0,68,26,9,0,0,9,0,8,57,0,0,0,0,0,0,5,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,2.89E2,1.35E2,4.7E1,2.42E2,1.26E2,9E0,2.4E1,2.3E1,8.9E1,1.53E2,1.16E2,1E1,7E0,2E0,6.9E1,2E1,1.15E2,3.8E1,1.12E2,4E0,5E0,2E0,6.3E1,6E0,4E0,1.6E1,7.9E1,3.6E1,1.1E2,2E0,1E0,3E0,3.9E1,2.4E1,4.2E1,3.7E1,3.2E1,4E0,4.4E1,6.6E1,1E1,2.9E1,3.1E1,1.1E1,2E1,1.7E1,2.2E1,1E1,7E0,5.9E1,1.6E1,1.3E1,2.5E1,6E0,5.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-5.0757453E-3,-4.1375384E-2,7.95706E-2,-4.409644E-2,1.2582895E-2,9.641743E-2,-1.1604129E-1,-5.9519663E-2,7.2841127E-3,7.9281606E-2,1.40994545E-2,1.0094637E-2,-1.6957028E-2,-8.08756E-2,-3.0951012E-2,1.7707981E-2,-8.713262E-3,8.757089E-2,-1.18939005E-1,1.1394761E-2,-5.3607352E-2,-1.22296095E-1,-6.735318E-2,-1.5308324E-3,-6.597227E-2,5.1729707E-3,4.758531E-3,6.392052E-2,1.307E-1,6.7427615E-3,-1.0801513E-2,-5.2492185E-3,2.497498E-3,-9.160124E-4,-1.3649392E-1,-4.2416914E-3,-1.97244E-3,-1.0990009E-3,2.0263288E-3,2.0269063E-4,-1.022466E-1,-3.4250531E-4,4.1252333E-3,9.712118E-2,2.709981E-2,1.6429149E-2,1.1056611E-1,-7.962517E-3,-2.2606745E-3,-1.6814854E-3,-1.287812E-1,5.3988313E-3,1.7634178E-4,-9.655979E-4,2.5545456E-3,6.2915757E-3,6.474933E-4,-9.946406E-3,-3.7195652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,21,23,25,-1,27,29,-1,31,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,49,-1,-1,51,53,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3613955E0,2.5838578E-1,4.494819E-1,2.4450928E-1,0E0,3.6904764E-1,3.1319946E-1,1.42093E-1,1.394061E-1,1.9772983E-1,0E0,1.23272926E-1,0E0,6.737715E-2,1.0602317E-1,7.617162E-2,0E0,1.038478E-1,1.5252924E-1,0E0,4.253727E-2,4.4733644E-2,4.7463417E-2,4.9961727E-2,1.1980499E-1,0E0,5.566335E-2,8.660847E-2,1.1517072E-1,0E0,0E0,0E0,0E0,0E0,5.1135838E-2,0E0,0E0,0E0,0E0,0E0,5.2096248E-2,0E0,0E0,3.8410246E-2,4.1121352E-2,0E0,5.222258E-2,0E0,0E0,0E0,6.9182396E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,17,17,18,18,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,34,34,40,40,43,43,44,44,46,46,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,22,24,26,-1,28,30,-1,32,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,50,-1,-1,52,54,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,3.2991206E3,1.0429407E3,7.164074E8,1.2582895E-2,1.146155E1,6.6053805E6,1.732734E6,1E0,3.305542E4,1.40994545E-2,1.3812E4,-1.6957028E-2,4.94E4,1.8071064E7,1.7608592E-4,-8.713262E-3,5.8145845E3,1.92144E5,1.1394761E-2,1.6130411E5,1.4297844E2,7.37642E4,1.1669E4,6.465659E4,5.1729707E-3,1E0,1.1532659E7,3.7137297E3,6.7427615E-3,-1.0801513E-2,-5.2492185E-3,2.497498E-3,-9.160124E-4,2.0454705E-3,-4.2416914E-3,-1.97244E-3,-1.0990009E-3,2.0263288E-3,2.0269063E-4,2.7E1,-3.4250531E-4,4.1252333E-3,2.4585695E0,2.326757E9,1.6429149E-2,3.0038656E5,-7.962517E-3,-2.2606745E-3,-1.6814854E-3,2.73E2,5.3988313E-3,1.7634178E-4,-9.655979E-4,2.5545456E-3,6.2915757E-3,6.474933E-4,-9.946406E-3,-3.7195652E-3],"split_indices":[2,64,70,7,0,47,59,1,19,4,0,9,0,9,57,50,0,4,2,0,40,45,40,10,40,0,101,57,44,0,0,0,0,0,51,0,0,0,0,0,8,0,0,54,12,0,45,0,0,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,3.09E2,1.32E2,3.07E2,2E0,1.22E2,1E1,2.36E2,7.1E1,1.13E2,9E0,7E0,3E0,1.34E2,1.02E2,6.8E1,3E0,1.09E2,4E0,1E0,6E0,3.1E1,1.03E2,5.6E1,4.6E1,8E0,6E1,7.2E1,3.7E1,1E0,3E0,4E0,2E0,4E0,2.7E1,6.2E1,4.1E1,3.8E1,1.8E1,1.6E1,3E1,5.3E1,7E0,3.7E1,3.5E1,2E0,3.5E1,2.1E1,6E0,9E0,2.1E1,3.3E1,4E0,1.2E1,2.3E1,3E1,5E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.5044258E-3,-3.7345733E-2,7.6977715E-2,-4.0150125E-2,1.2163665E-2,-1.7551139E-1,9.039395E-2,-7.624012E-2,-2.4069777E-2,-1.5107096E-3,-1.3839191E-2,8.135891E-2,1.2870588E-2,-9.3174994E-2,5.7902485E-3,3.3054322E-2,-3.3561226E-2,9.200262E-2,-1.379793E-1,-1.0300372E-1,-1.0471808E-3,-1.4885436E-3,3.5006606E-3,-1.6533234E-4,5.121249E-3,-1.5047007E-2,-6.8765625E-2,8.211675E-2,2.183945E-2,6.574274E-3,-1.0904508E-2,-4.6008118E-4,-1.10796645E-1,-3.1999074E-2,2.2571333E-2,-7.723607E-2,1.9072073E-3,1.3323898E-2,7.812477E-2,-1.1685783E-1,8.791054E-5,-8.757582E-4,-6.648194E-2,1.6622155E-3,-3.5610355E-3,-3.2211542E-3,-9.603749E-3,-2.853789E-4,8.6319536E-2,-4.731293E-3,-8.105512E-3,-6.27397E-3,-1.3301288E-3,6.0308803E-2,5.5308673E-3,8.134608E-3,2.0366893E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,37,-1,-1,-1,-1,39,41,43,45,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,55,-1,-1,-1],"loss_changes":[1.1842937E0,2.3729962E-1,4.6300906E-1,1.6472238E-1,0E0,9.475586E-2,1.6932309E-1,1.2278688E-1,1.0954743E-1,0E0,0E0,2.951777E-1,0E0,4.9660265E-2,3.866151E-2,7.354301E-2,1.11071765E-1,3.5974216E-1,1.584245E-1,4.5761347E-2,0E0,0E0,0E0,0E0,0E0,7.342104E-2,5.6385487E-2,6.312978E-2,0E0,0E0,0E0,0E0,4.0456533E-2,3.826846E-2,3.8474225E-2,6.84126E-2,0E0,0E0,7.812494E-2,4.0965974E-2,0E0,0E0,5.0810494E-2,0E0,0E0,0E0,0E0,0E0,5.856377E-2,0E0,0E0,0E0,0E0,9.755808E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,27,27,32,32,33,33,34,34,35,35,38,38,39,39,42,42,48,48,53,53],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,38,-1,-1,-1,-1,40,42,44,46,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,56,-1,-1,-1],"split_conditions":[1.2934E4,3.2991206E3,2.4950776E0,1.5031561E3,1.2163665E-2,6.27427E5,1.6828056E1,2.1592189E5,4.3712845E0,-1.5107096E-3,-1.3839191E-2,3.768034E4,1.2870588E-2,6.209E3,4.18E3,8.6665695E1,1.8457164E7,3.305542E4,1.92144E5,5.895129E1,-1.0471808E-3,-1.4885436E-3,3.5006606E-3,-1.6533234E-4,5.121249E-3,9.69E2,9.407814E6,1.2662238E1,2.183945E-2,6.574274E-3,-1.0904508E-2,-4.6008118E-4,1.3421E4,5.559E3,2.0136225E5,2.231E3,1.9072073E-3,1.3323898E-2,6E0,4.02959E5,8.791054E-5,-8.757582E-4,5.787671E8,1.6622155E-3,-3.5610355E-3,-3.2211542E-3,-9.603749E-3,-2.853789E-4,1.648428E-1,-4.731293E-3,-8.105512E-3,-6.27397E-3,-1.3301288E-3,1.8008743E3,5.5308673E-3,8.134608E-3,2.0366893E-3],"split_indices":[2,64,66,4,0,9,47,40,70,0,0,4,0,2,2,59,57,4,2,62,0,0,0,0,0,0,1,68,0,0,0,0,10,10,45,0,0,0,8,41,0,0,5,0,0,0,0,0,50,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,2.88E2,1.31E2,2.86E2,2E0,6E0,1.25E2,8.7E1,1.99E2,3E0,3E0,1.2E2,5E0,7.2E1,1.5E1,2.8E1,1.71E2,1.15E2,5E0,6.3E1,9E0,1E1,5E0,1.9E1,9E0,1.13E2,5.8E1,1.13E2,2E0,1E0,4E0,5E0,5.8E1,7.8E1,3.5E1,5.4E1,4E0,1E0,1.12E2,5.5E1,3E0,5.6E1,2.2E1,3.2E1,3E0,5E1,4E0,1E1,1.02E2,3.9E1,1.6E1,8E0,1.4E1,5.1E1,5.1E1,7E0,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-3.5769674E-3,-4.46986E-2,8.625172E-2,-5.7014383E-2,7.03394E-3,4.8121482E-2,1.949098E-1,-6.6504985E-2,-3.1469143E-3,5.966327E-3,-3.4990686E-3,5.664029E-2,-4.8786397E-3,3.587558E-1,1.0486555E-1,-7.352405E-2,-1.3307479E-2,-4.930942E-3,1.142104E-2,-7.999162E-4,3.5191367E-3,-1.6422821E-4,6.595374E-2,7.388784E-3,2.062171E-2,1.5791379E-1,-9.987926E-3,-9.800828E-2,-5.667645E-2,-1.16811685E-1,9.200175E-4,-5.3346355E-4,3.1638825E-3,7.9098895E-2,4.7444645E-4,5.802352E-3,1.3418038E-2,-1.2714735E-1,-5.7611093E-2,-2.8463665E-2,-4.5299237E-3,-3.1593223E-5,-8.732991E-3,5.3880274E-2,5.208167E-3,-1.4484613E-3,-7.153948E-3,-4.0786555E-3,1.9873406E-3,-5.9464197E-3,-5.179148E-3,5.455523E-3,1.420283E-3,-1.1146478E-3,2.5538218E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,-1,31,-1,-1,-1,33,-1,-1,35,-1,37,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1],"loss_changes":[1.599621E0,1.8962765E-1,5.5409014E-1,1.2248701E-1,6.90538E-2,1.3216622E-1,4.787476E-1,7.51946E-2,5.261303E-2,0E0,5.1047616E-2,5.435452E-2,0E0,7.5348735E-2,4.19516E-1,6.869531E-2,8.4220536E-2,0E0,3.8509037E-2,0E0,0E0,0E0,6.159213E-2,0E0,0E0,6.5916836E-2,0E0,7.742876E-2,1.0187012E-1,4.02724E-2,0E0,0E0,0E0,3.745264E-2,0E0,0E0,0E0,6.0247183E-2,7.803415E-2,1.0263209E-1,0E0,0E0,0E0,4.7569335E-2,0E0,0E0,0E0,0E0,0E0,4.593019E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,18,18,22,22,25,25,27,27,28,28,29,29,33,33,37,37,38,38,39,39,43,43,49,49],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,-1,32,-1,-1,-1,34,-1,-1,36,-1,38,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1],"split_conditions":[1.1153E4,7.10711E8,6.204522E3,3.4863858E6,2.829932E0,9.05929E2,1.417988E6,2.1592189E5,7.6807027E0,5.966327E-3,1E0,1E1,-4.8786397E-3,9.256843E3,1.5096262E7,3.88672E5,6.52487E2,-4.930942E-3,6.8480647E10,-7.999162E-4,3.5191367E-3,-1.6422821E-4,7.789796E1,7.388784E-3,2.062171E-2,1.6828056E1,-9.987926E-3,7.121916E2,3.5389444E5,1.0029164E-6,9.200175E-4,-5.3346355E-4,3.1638825E-3,1.93536E5,4.7444645E-4,5.802352E-3,1.3418038E-2,5.8955963E1,3.685938E6,3.271774E7,-4.5299237E-3,-3.1593223E-5,-8.732991E-3,4.41341E5,5.208167E-3,-1.4484613E-3,-7.153948E-3,-4.0786555E-3,1.9873406E-3,2.143469E6,-5.179148E-3,5.455523E-3,1.420283E-3,-1.1146478E-3,2.5538218E-3],"split_indices":[2,7,64,62,70,70,41,40,65,0,101,8,0,4,9,9,4,0,43,0,0,0,68,0,0,47,0,64,60,50,0,0,0,11,0,0,0,62,1,57,0,0,0,41,0,0,0,0,0,1,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.96E2,1.35E2,2.39E2,5.7E1,1.01E2,3.4E1,2.03E2,3.6E1,4E0,5.3E1,9.6E1,5E0,1.1E1,2.3E1,1.79E2,2.4E1,4E0,3.2E1,4.6E1,7E0,1.3E1,8.3E1,3E0,8E0,2E1,3E0,7.1E1,1.08E2,5E0,1.9E1,2.3E1,9E0,6.7E1,1.6E1,1.6E1,4E0,4E1,3.1E1,6E1,4.8E1,2E0,3E0,3.5E1,3.2E1,6E0,3.4E1,2.5E1,6E0,4.7E1,1.3E1,1E1,2.5E1,3.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.1722329E-3,-2.2786593E-2,1.247586E-1,-4.923096E-2,2.8184226E-2,3.0928555E-1,8.410662E-2,-5.501502E-2,3.0558933E-2,3.7491437E-2,-1.3246593E-2,3.7163594E-3,1.9395031E-2,1.4031406E-2,7.088948E-2,-6.759944E-2,-1.2662115E-2,-3.952504E-3,2.5213454E-3,1.6902057E-2,3.9876844E-3,1.4128439E-2,1.17010035E-1,-8.941428E-2,-2.311165E-3,-2.9152814E-2,1.5521493E-3,7.671637E-3,5.86423E-3,4.380074E-2,-7.25569E-3,1.4798077E-3,6.7329067E-3,6.1095477E-4,-9.728847E-2,-6.7934254E-4,-4.914642E-3,6.539371E-4,-6.598049E-3,4.6469066E-3,-4.1690386E-5,-1.05115004E-1,-1.2362063E-3,-9.370906E-2,-8.659715E-3,-5.4381443E-3,-2.125715E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,-1,27,-1,29,31,33,-1,35,-1,37,-1,39,-1,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,43,-1,45,-1,-1,-1],"loss_changes":[1.1705441E0,4.9606353E-1,4.4657457E-1,1.13366485E-1,3.538259E-1,1.747737E-1,1.1785588E-1,1.19598925E-1,4.106889E-2,1.05776906E-1,0E0,0E0,0E0,0E0,1.3185722E-1,7.692027E-2,3.911697E-2,0E0,0E0,7.697545E-2,0E0,1.19810596E-1,3.9356798E-2,6.9217026E-2,0E0,4.0647257E-2,0E0,6.071301E-2,0E0,4.61774E-2,0E0,0E0,0E0,0E0,4.22588E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.896159E-2,0E0,4.4072747E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,19,19,21,21,22,22,23,23,25,25,27,27,29,29,34,34,41,41,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,-1,28,-1,30,32,34,-1,36,-1,38,-1,40,-1,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,44,-1,46,-1,-1,-1],"split_conditions":[3.2991206E3,7.10711E8,1.37893E6,2.889654E5,1.190724E4,1.082556E7,8.169258E3,3.447954E5,5.012955E6,1.9795151E9,-1.3246593E-2,3.7163594E-3,1.9395031E-2,1.4031406E-2,1.9078612E-1,2.73E2,1.7657828E3,-3.952504E-3,2.5213454E-3,1E0,3.9876844E-3,2.5212732E4,3.1063303E5,5.8601086E1,-2.311165E-3,4.0266477E8,1.5521493E-3,1E0,5.86423E-3,3.7057E5,-7.25569E-3,1.4798077E-3,6.7329067E-3,6.1095477E-4,1.3421E4,-6.7934254E-4,-4.914642E-3,6.539371E-4,-6.598049E-3,4.6469066E-3,-4.1690386E-5,4.816E3,-1.2362063E-3,1.6395413E7,-8.659715E-3,-5.4381443E-3,-2.125715E-3],"split_indices":[64,7,41,40,4,1,45,59,57,7,0,0,0,0,50,3,64,0,0,101,0,4,44,62,0,7,0,19,0,9,0,0,0,0,10,0,0,0,0,0,0,10,0,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,3.66E2,6.2E1,2.41E2,1.25E2,1E1,5.2E1,2.25E2,1.6E1,1.22E2,3E0,3E0,7E0,2E0,5E1,1.73E2,5.2E1,2E0,1.4E1,8.3E1,3.9E1,2.3E1,2.7E1,8.4E1,8.9E1,3.8E1,1.4E1,7.7E1,6E0,2E1,3E0,5E0,2.2E1,6E0,7.8E1,3.2E1,6E0,7.5E1,2E0,9E0,1.1E1,7E1,8E0,6.2E1,8E0,4.7E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.9097693E-3,-2.9472075E-2,7.402421E-2,-6.247058E-2,-4.235935E-3,-7.856033E-3,8.60728E-2,-7.396903E-2,-2.7621519E-3,4.452734E-2,-2.8582273E-2,1.4437816E-2,7.145314E-2,-9.398713E-2,-4.2333867E-2,-6.7318873E-3,8.1050984E-4,1.3003242E-1,2.2678599E-2,-8.1427254E-2,-1.4522827E-2,4.0605575E-2,1.2538108E-1,-1.2326426E-3,-1.0397442E-1,-5.4636825E-4,-3.957893E-3,3.7241462E-3,1.1116206E-2,3.3514205E-2,-7.7282395E-3,-5.827089E-3,-1.3026411E-3,-6.6318964E-3,-4.639639E-4,5.3676233E-2,-8.929544E-3,9.8143436E-2,3.255558E-1,-1.4418462E-1,-3.4475864E-3,-1.2340529E-3,2.5980375E-3,1.3109803E-1,4.027494E-2,6.953521E-3,2.1654468E-3,3.2724515E-3,1.9511703E-2,-7.735769E-3,-3.87298E-4,4.3679236E-5,8.4049795E-3,1.2731963E-3,5.287566E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,-1,41,-1,-1,-1,-1,-1,43,-1,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.958566E-1,2.6122037E-1,3.3112007E-1,9.3167126E-2,2.1378279E-1,0E0,3.0256265E-1,6.8214595E-2,5.978201E-2,1.0914175E-1,8.7974E-2,0E0,1.6625404E-1,4.5207202E-2,5.2379638E-2,0E0,0E0,4.4977278E-2,9.932829E-2,4.5638263E-2,5.820647E-2,2.0162332E-1,1.6978347E-1,0E0,7.447004E-2,0E0,0E0,0E0,0E0,5.1816512E-2,0E0,0E0,0E0,0E0,0E0,6.1123386E-2,0E0,7.03696E-2,4.1914314E-2,3.7865818E-2,0E0,0E0,0E0,4.3313727E-2,5.1609047E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,29,29,35,35,37,37,38,38,39,39,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,-1,42,-1,-1,-1,-1,-1,44,-1,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5704E4,1.6861708E3,1.4144695E0,2.1592189E5,1.24E2,-7.856033E-3,9.957E3,1.656968E6,3.516E3,3.34031E5,9.04782E5,1.4437816E-2,2.1383806E-1,5.8981678E1,1.33E2,-6.7318873E-3,8.1050984E-4,7.09E3,1E0,6.2E1,1.32E2,1.4609149E4,9.920844E0,-1.2326426E-3,2.7276272E-2,-5.4636825E-4,-3.957893E-3,3.7241462E-3,1.1116206E-2,1.1327554E6,-7.7282395E-3,-5.827089E-3,-1.3026411E-3,-6.6318964E-3,-4.639639E-4,1.8297544E3,-8.929544E-3,2.27901E5,1.281E3,4.61E2,-3.4475864E-3,-1.2340529E-3,2.5980375E-3,1.504E4,8.0200186E3,6.953521E-3,2.1654468E-3,3.2724515E-3,1.9511703E-2,-7.735769E-3,-3.87298E-4,4.3679236E-5,8.4049795E-3,1.2731963E-3,5.287566E-3],"split_indices":[2,67,65,40,3,0,9,9,12,12,1,0,50,62,3,0,0,2,19,8,3,4,65,0,54,0,0,0,0,60,0,0,0,0,0,4,0,9,0,3,0,0,0,9,4,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,3.13E2,1.13E2,1.35E2,1.78E2,5E0,1.08E2,1.13E2,2.2E1,5.9E1,1.19E2,6E0,1.02E2,6.8E1,4.5E1,2E0,2E1,1.1E1,4.8E1,2.4E1,9.5E1,6.6E1,3.6E1,9E0,5.9E1,2.5E1,2E1,8E0,3E0,4.6E1,2E0,1.4E1,1E1,3E0,9.2E1,6.3E1,3E0,3.3E1,3E0,2.6E1,3.3E1,1.1E1,3.5E1,8E0,5.5E1,1.8E1,1.5E1,1E0,2E0,2.4E1,2E0,2E0,6E0,4.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.6762093E-3,-4.329192E-2,6.305308E-2,-4.6974506E-2,1.5428075E-2,2.5035758E-2,1.2997788E-1,-4.245986E-2,-1.41069135E-2,-3.7218716E-2,4.466814E-2,9.261448E-2,2.9632443E-1,-8.666192E-2,-3.0281452E-2,-8.339047E-3,-9.34087E-4,6.776784E-4,7.241056E-2,5.717102E-3,-1.529093E-3,7.1113985E-3,1.8755725E-2,-5.008331E-3,-2.6358923E-2,-4.7315866E-2,6.988253E-3,4.225571E-3,-6.24984E-4,-6.507599E-3,3.715007E-4,-2.5326205E-2,-8.062196E-2,-7.988225E-4,2.3558754E-3,-4.8004776E-2,4.282462E-2,-9.167577E-3,-3.5822098E-3,-6.590091E-2,2.682642E-4,-1.2775036E-3,7.4341536E-2,-4.3635294E-3,-1.2616031E-3,8.261517E-3,2.0306706E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,-1,27,-1,-1,-1,-1,-1,29,31,33,-1,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,43,-1,-1,45,-1,-1,-1,-1],"loss_changes":[1.1378195E0,3.7356943E-1,3.9874554E-1,2.801445E-1,0E0,1.2632728E-1,3.2682663E-1,1.4325795E-1,0E0,5.693157E-2,6.6333175E-2,1.3232324E-1,6.8232715E-2,4.475984E-2,1.3619228E-1,0E0,0E0,0E0,4.332812E-2,0E0,0E0,0E0,0E0,0E0,4.3082323E-2,1.0561943E-1,6.3475415E-2,0E0,0E0,0E0,0E0,1.4115648E-1,4.0184945E-2,0E0,0E0,6.530486E-2,5.1473275E-2,0E0,0E0,4.2192414E-2,0E0,0E0,4.2345524E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,24,24,25,25,26,26,31,31,32,32,35,35,36,36,39,39,42,42],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,-1,28,-1,-1,-1,-1,-1,30,32,34,-1,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,44,-1,-1,46,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,5.6503E4,7.339209E6,1.5428075E-2,1.0643513E3,1.2123712E1,3.836E3,-1.41069135E-2,1E0,7.959718E4,4.42384E5,1.92144E5,1.7212875E5,1.2028214E5,-8.339047E-3,-9.34087E-4,6.776784E-4,2.237815E7,5.717102E-3,-1.529093E-3,7.1113985E-3,1.8755725E-2,-5.008331E-3,3.516E3,3.4129692E-3,2.735322E6,4.225571E-3,-6.24984E-4,-6.507599E-3,3.715007E-4,7.198E3,7.07812E7,-7.988225E-4,2.3558754E-3,1.54E2,5.625619E1,-9.167577E-3,-3.5822098E-3,1.0599062E3,2.682642E-4,-1.2775036E-3,4.9832974E2,-4.3635294E-3,-1.2616031E-3,8.261517E-3,2.0306706E-3],"split_indices":[45,64,2,59,0,64,65,2,0,13,40,2,2,40,40,0,0,0,57,0,0,0,0,0,12,69,1,0,0,0,0,2,7,0,0,8,68,0,0,45,0,0,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.76E2,1.57E2,2.74E2,2E0,1.01E2,5.6E1,2.7E2,4E0,2.4E1,7.7E1,4.7E1,9E0,5.7E1,2.13E2,2E0,2.2E1,3.7E1,4E1,4E1,7E0,4E0,5E0,4.6E1,1.1E1,1.46E2,6.7E1,3.5E1,5E0,2E0,9E0,8.9E1,5.7E1,4.3E1,2.4E1,6.7E1,2.2E1,3E0,5.4E1,5E1,1.7E1,7E0,1.5E1,3.2E1,1.8E1,3E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.83888E-3,-3.1505685E-2,9.495612E-2,-4.551759E-2,1.0406145E-2,2.4448071E-1,7.442078E-2,-6.255245E-2,-2.5327334E-2,1.8394146E-2,-7.7937418E-3,1.703688E-2,4.56002E-3,-1.2077044E-1,8.944308E-2,-3.6171337E-3,-8.0137077E-4,-5.765748E-2,-9.046284E-3,-8.121633E-4,3.5697255E-2,-2.9794502E-4,-1.358657E-2,7.414078E-2,1.0601171E-2,-2.0557244E-3,-8.085591E-3,-2.095319E-2,2.38548E-3,6.705194E-2,5.530207E-4,1.8580599E-1,6.0514547E-2,6.6048084E-3,-2.1355385E-3,6.4281086E-3,1.7058217E-3,3.2917245E-3,1.4457056E-2,7.4992E-2,-4.1262205E-3,1.7764501E-3,-2.3296846E-3,-3.9353026E-3,3.9293445E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,-1,-1,25,27,-1,29,-1,-1,31,-1,-1,-1,33,-1,35,-1,37,39,41,-1,-1,-1,-1,-1,43,-1,-1,-1,-1,-1],"loss_changes":[1.2607517E0,1.9779453E-1,2.980711E-1,8.508819E-2,1.1582068E-1,1.4619565E-1,2.8056633E-1,6.0994923E-2,6.1063565E-2,4.973661E-2,0E0,0E0,0E0,1.1980515E-1,1.4572191E-1,0E0,0E0,6.0448393E-2,5.4408792E-2,0E0,4.1735314E-2,0E0,0E0,1.0743901E-1,0E0,0E0,0E0,3.98927E-2,0E0,4.4255123E-2,0E0,8.023122E-2,1.5326184E-1,4.7865223E-2,0E0,0E0,0E0,0E0,0E0,4.2490035E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,17,17,18,18,20,20,23,23,27,27,29,29,31,31,32,32,33,33,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,-1,-1,26,28,-1,30,-1,-1,32,-1,-1,-1,34,-1,36,-1,38,40,42,-1,-1,-1,-1,-1,44,-1,-1,-1,-1,-1],"split_conditions":[2.1722E4,7.164074E8,9.73307E5,1.732734E6,1E0,2.3879834E9,1.9813E4,3.0946184E5,7.121916E2,2.0069241E3,-7.7937418E-3,1.703688E-2,4.56002E-3,7.600717E6,1.6941456E1,-3.6171337E-3,-8.0137077E-4,6.97E3,7.1999524E9,-8.121633E-4,1.5E2,-2.9794502E-4,-1.358657E-2,1.3812E4,1.0601171E-2,-2.0557244E-3,-8.085591E-3,1.4040858E-2,2.38548E-3,6.72E2,5.530207E-4,2.9632012E7,1.4656219E8,2.515304E7,-2.1355385E-3,6.4281086E-3,1.7058217E-3,3.2917245E-3,1.4457056E-2,1.700398E4,-4.1262205E-3,1.7764501E-3,-2.3296846E-3,-3.9353026E-3,3.9293445E-3],"split_indices":[2,7,41,1,19,5,10,59,64,67,0,0,0,1,47,0,0,2,5,0,3,0,0,9,0,0,0,69,0,0,0,1,1,57,0,0,0,0,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.35E2,1.02E2,2.51E2,8.4E1,1.1E1,9.1E1,1.35E2,1.16E2,8.1E1,3E0,6E0,5E0,6E0,8.5E1,1.11E2,2.4E1,3.8E1,7.8E1,2.7E1,5.4E1,4E0,2E0,7.7E1,8E0,3.4E1,4E0,6.5E1,1.3E1,2.3E1,3.1E1,7E0,7E1,2.9E1,3.6E1,7E0,1.6E1,4E0,3E0,6.4E1,6E0,1.9E1,1E1,1E0,6.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.6657364E-3,-1.6230375E-2,1.4032856E-1,-3.0203806E-2,6.1110664E-2,8.405176E-2,2.2616886E-1,-2.7168322E-2,-1.3988816E-2,6.905196E-2,-5.901448E-3,1.297636E-2,6.4346895E-2,6.0318024E-3,1.707545E-2,-3.9163496E-2,1.7857725E-2,1.10801635E-2,5.996534E-2,3.6801326E-3,-5.356998E-3,-5.2230526E-3,-3.404043E-2,4.5894124E-2,-1.1456036E-3,1.0188965E-1,3.341668E-2,-3.984543E-2,4.926508E-4,1.2162745E-3,5.727184E-3,1.1536736E-3,7.260139E-3,5.3720153E-4,3.985338E-3,-4.1311376E-2,7.0631974E-3,-4.5591306E-2,1.2454672E-3,2.9451582E-3,-2.3959118E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,19,-1,-1,21,23,-1,25,-1,-1,-1,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,37,-1,39,-1,-1,-1],"loss_changes":[1.12889E0,4.373759E-1,2.3219264E-1,2.5162446E-1,9.638305E-2,9.7477704E-2,2.0241714E-1,1.837917E-1,0E0,6.993353E-2,0E0,0E0,5.7299808E-2,0E0,0E0,8.61634E-2,8.337292E-2,0E0,6.1151147E-2,0E0,0E0,0E0,6.409994E-2,6.0896553E-2,0E0,7.185948E-2,3.799819E-2,6.453535E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.349981E-2,0E0,5.3667307E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,12,12,15,15,16,16,18,18,22,22,23,23,25,25,26,26,27,27,35,35,37,37],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,20,-1,-1,22,24,-1,26,-1,-1,-1,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,38,-1,40,-1,-1,-1],"split_conditions":[4.23168E3,1.5522031E9,2.2876814E1,3.764913E7,1.190724E4,8.169258E3,2.235E5,1.5135906E3,-1.3988816E-2,1.3E1,-5.901448E-3,1.297636E-2,5.22E2,6.0318024E-3,1.707545E-2,1.44E2,6.878971E1,1.10801635E-2,6.3504875E6,3.6801326E-3,-5.356998E-3,-5.2230526E-3,2.0964778E5,1.6130411E5,-1.1456036E-3,2.867292E3,9.9179E4,1.9485E4,4.926508E-4,1.2162745E-3,5.727184E-3,1.1536736E-3,7.260139E-3,5.3720153E-4,3.985338E-3,4.79E2,7.0631974E-3,5.949655E0,1.2454672E-3,2.9451582E-3,-2.3959118E-3],"split_indices":[64,7,66,60,4,45,2,64,0,3,0,0,3,0,0,0,68,0,57,0,0,0,40,40,0,4,2,2,0,0,0,0,0,0,0,8,0,68,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.53E2,4.02E2,5.1E1,3.41E2,6.1E1,3.2E1,1.9E1,3.38E2,3E0,5.9E1,2E0,2E0,3E1,1.1E1,8E0,2.67E2,7.1E1,2E0,5.7E1,2.9E1,1E0,1.8E1,2.49E2,4.2E1,2.9E1,2.1E1,3.6E1,2.2E2,2.9E1,3.3E1,9E0,8E0,1.3E1,2.5E1,1.1E1,2.19E2,1E0,2.06E2,1.3E1,4E0,2.02E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-3.8273662E-4,-1.9588554E-2,9.514597E-2,-3.0770771E-2,4.2636964E-2,2.3074453E-1,6.6115856E-2,-2.767215E-2,-1.3464486E-2,8.73124E-3,2.6881015E-2,2.0919892E-3,1.4154424E-2,8.468875E-2,-1.2753686E-1,-3.6554012E-2,1.363874E-2,1.8004556E-3,-4.7055413E-3,6.1234064E-2,1.3903362E-1,5.206573E-3,-9.731181E-3,-3.7750006E-2,6.8867034E-3,3.7153366E-3,-1.6284496E-2,8.0076605E-2,5.792999E-4,9.16635E-3,3.3073185E-3,-3.4658718E-3,-3.0303154E-2,-1.645871E-3,2.1424915E-3,2.4728107E-3,6.9657536E-3,-1.7256556E-2,-5.5613063E-2,-1.2454471E-2,-7.313641E-3,-1.4665913E-3,-3.9152782E-3,-1.9834144E-3,3.5979378E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,-1,-1,19,21,23,25,-1,-1,27,29,-1,-1,31,-1,-1,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1],"loss_changes":[8.3298606E-1,2.6433164E-1,2.8391373E-1,2.2991759E-1,1.1599646E-1,1.1700761E-1,2.4098915E-1,1.17178395E-1,0E0,0E0,6.205896E-2,0E0,0E0,6.6690266E-2,1.1368664E-1,5.9835732E-2,1.0474809E-1,0E0,0E0,3.9293602E-2,4.282996E-2,0E0,0E0,5.9318304E-2,0E0,0E0,3.9820995E-2,4.935424E-2,0E0,0E0,0E0,0E0,6.9374666E-2,0E0,0E0,0E0,0E0,8.62008E-2,4.0688753E-2,7.414715E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,13,13,14,14,15,15,16,16,19,19,20,20,23,23,26,26,27,27,32,32,37,37,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,-1,-1,20,22,24,26,-1,-1,28,30,-1,-1,32,-1,-1,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.6053742E9,1E0,3.372042E7,1.5E1,5.993877E3,3.768034E4,1.4720299E3,-1.3464486E-2,8.73124E-3,2.4070468E7,2.0919892E-3,1.4154424E-2,6.477944E0,1.92144E5,1.7267E4,1.03E2,1.8004556E-3,-4.7055413E-3,1.0392101E0,4.1923336E7,5.206573E-3,-9.731181E-3,5.87114E5,6.8867034E-3,3.7153366E-3,1.2472753E5,8.51E4,5.792999E-4,9.16635E-3,3.3073185E-3,-3.4658718E-3,1.902843E-2,-1.645871E-3,2.1424915E-3,2.4728107E-3,6.9657536E-3,4.086885E7,7.9879355E6,1.59551E5,-7.313641E-3,-1.4665913E-3,-3.9152782E-3,-1.9834144E-3,3.5979378E-4],"split_indices":[64,7,28,60,3,4,4,64,0,0,60,0,0,47,2,2,3,0,0,46,55,0,0,1,0,0,40,2,0,0,0,0,69,0,0,0,0,57,57,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.52E2,3.77E2,7.5E1,3.2E2,5.7E1,1.2E1,6.3E1,3.17E2,3E0,5E0,5.2E1,3E0,9E0,5.8E1,5E0,2.61E2,5.6E1,4.9E1,3E0,4.2E1,1.6E1,1E0,4E0,2.6E2,1E0,1.8E1,3.8E1,3E1,1.2E1,9E0,7E0,4.8E1,2.12E2,3E1,8E0,2.1E1,9E0,1.41E2,7.1E1,1.37E2,4E0,3.4E1,3.7E1,5.7E1,8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.704983E-3,-9.6624885E-3,1.4588639E-1,-3.3298146E-2,3.2194354E-2,2.9020366E-1,7.9218075E-2,-2.974744E-2,-1.295981E-2,5.000202E-3,7.571256E-2,4.355774E-3,1.627426E-2,1.2057051E-1,-1.1857627E-1,-5.752256E-3,-2.414346E-2,-1.0916167E-2,3.6387136E-3,1.05796E-2,6.4446464E-2,4.164135E-3,1.0295639E-2,-1.1102328E-2,1.2256985E-3,-5.749331E-3,-2.218985E-2,-3.7259907E-3,-1.228004E-4,8.3263926E-2,1.6945582E-4,-3.1893063E-2,-1.0061341E-3,-6.431286E-4,9.372181E-2,7.042771E-3,-4.219229E-2,-4.8525423E-2,6.5131055E-4,6.127938E-3,2.539507E-3,1.8619348E-3,-1.8861094E-3,-8.623703E-3,-1.9433085E-3,4.0130914E-4,-4.260224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,-1,-1,33,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.018076E-1,4.1361597E-1,3.233807E-1,2.0656356E-1,1.7832097E-1,5.076772E-2,2.2695808E-1,1.23496264E-1,0E0,1.0224632E-1,7.5211436E-2,0E0,0E0,5.28495E-2,7.955851E-2,0E0,4.166457E-2,4.1267738E-2,0E0,0E0,6.290795E-2,0E0,0E0,0E0,0E0,0E0,5.0327696E-2,0E0,0E0,4.3430537E-2,0E0,6.766665E-2,5.2666456E-2,0E0,4.0882975E-2,5.011632E-2,5.0637454E-2,3.8065236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,16,16,17,17,20,20,26,26,29,29,31,31,32,32,34,34,35,35,36,36,37,37],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,-1,-1,34,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.204522E3,9.7467445E4,1.417988E6,7.339209E6,1.484798E9,9.256843E3,1.5096262E7,1.44E2,-1.295981E-2,1.5271514E5,1.4E1,4.355774E-3,1.627426E-2,1.2472753E5,3.087E3,-5.752256E-3,2.398E3,3.6340196E-2,3.6387136E-3,1.05796E-2,3.2006908E7,4.164135E-3,1.0295639E-2,-1.1102328E-2,1.2256985E-3,-5.749331E-3,1.2028214E5,-3.7259907E-3,-1.228004E-4,8.946923E2,1.6945582E-4,3.2081733E3,4.5765094E2,-6.431286E-4,1.006467E7,3.0840238E7,1.5988636E1,3.251E3,6.5131055E-4,6.127938E-3,2.539507E-3,1.8619348E-3,-1.8861094E-3,-8.623703E-3,-1.9433085E-3,4.0130914E-4,-4.260224E-3],"split_indices":[64,45,41,59,7,4,9,0,0,40,3,0,0,40,0,0,2,50,0,0,1,0,0,0,0,0,40,0,0,64,0,60,64,0,60,57,68,2,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.51E2,4.16E2,3.5E1,2.66E2,1.5E2,1E1,2.5E1,2.63E2,3E0,9.3E1,5.7E1,2E0,8E0,2.1E1,4E0,1.5E1,2.48E2,7.6E1,1.7E1,3E0,5.4E1,1.6E1,5E0,2E0,2E0,4E0,2.44E2,8E0,6.8E1,4.1E1,1.3E1,1.67E2,7.7E1,4E0,3.7E1,3.5E1,1.32E2,1.7E1,6E1,2.1E1,1.6E1,2.1E1,1.4E1,2E0,1.3E2,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.2027072E-3,-2.1871498E-2,7.529484E-2,-1.9424658E-2,-1.4261402E-2,2.046406E-1,6.0886916E-2,-3.410473E-2,2.724388E-2,4.6764086E-3,1.3148219E-2,7.102984E-2,-3.8367254E-3,-2.2410464E-2,-6.263292E-2,-2.9200143E-3,3.0315376E-3,6.2274534E-2,1.0278284E-2,-2.4408007E-3,-2.81583E-3,-2.1828445E-2,-7.761898E-2,-1.4639634E-3,1.9559993E-3,8.757298E-2,3.0189885E-2,1.5609784E-2,-3.195849E-2,1.2093167E-3,-3.7420704E-3,1.4959046E-3,-8.467071E-2,1.431511E-1,3.104124E-3,-1.6171422E-2,2.6983346E-3,6.146431E-6,3.1682611E-3,-4.1960245E-3,-6.234612E-4,-6.8117365E-2,-8.1491545E-3,4.979199E-3,1.771413E-2,-3.4728453E-3,1.624417E-3,-4.4827728E-4,5.2710976E-3,-5.688526E-3,-3.385374E-2,-3.8308282E-3,4.151397E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,19,21,23,-1,25,-1,-1,27,29,31,-1,-1,33,35,37,39,-1,-1,-1,41,43,-1,45,-1,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,51,-1,-1],"loss_changes":[7.4874943E-1,2.0979507E-1,1.8135482E-1,2.2838153E-1,0E0,4.030052E-2,1.3809735E-1,8.3001286E-2,8.081339E-2,0E0,0E0,9.210235E-2,0E0,9.359599E-2,4.2945087E-2,4.884321E-2,0E0,6.634998E-2,0E0,0E0,5.688087E-2,5.3086355E-2,4.2730838E-2,0E0,0E0,5.761653E-2,4.391046E-2,4.846566E-2,4.04927E-2,0E0,0E0,0E0,5.377382E-2,9.3425244E-2,0E0,3.859297E-2,0E0,4.827923E-2,0E0,0E0,0E0,6.348924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.7552392E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,17,17,20,20,21,21,22,22,25,25,26,26,27,27,28,28,32,32,33,33,35,35,37,37,41,41,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,20,22,24,-1,26,-1,-1,28,30,32,-1,-1,34,36,38,40,-1,-1,-1,42,44,-1,46,-1,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,52,-1,-1],"split_conditions":[1.615586E9,3.2875448E7,1.3812E4,1.4720299E3,-1.4261402E-2,2.094E3,1.0429407E3,1.8071064E7,9.5430945E4,4.6764086E-3,1.3148219E-2,1.0871896E10,-3.8367254E-3,4.39E3,6.465659E4,8.5716705E1,3.0315376E-3,1E0,1.0278284E-2,-2.4408007E-3,2.1813516E-2,1.3404826E-3,7.236662E0,-1.4639634E-3,1.9559993E-3,1.06904055E5,5.2610065E1,2.2331E4,1.19279E6,1.2093167E-3,-3.7420704E-3,1.4959046E-3,6.588828E6,4.52E2,3.104124E-3,1.1796855E7,2.6983346E-3,8.32E3,3.1682611E-3,-4.1960245E-3,-6.234612E-4,6.603817E2,-8.1491545E-3,4.979199E-3,1.771413E-2,-3.4728453E-3,1.624417E-3,-4.4827728E-4,5.2710976E-3,-5.688526E-3,1.6507361E9,-3.8308282E-3,4.151397E-4],"split_indices":[7,60,9,64,0,0,70,57,40,0,0,7,0,2,40,70,0,24,0,0,69,69,59,0,0,40,68,11,1,0,0,0,9,3,0,44,0,2,0,0,0,64,0,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.36E2,3.33E2,1.03E2,3.31E2,2E0,9E0,9.4E1,2.52E2,7.9E1,4E0,5E0,8.8E1,6E0,1.8E2,7.2E1,4.2E1,3.7E1,8.4E1,4E0,7.6E1,1.04E2,2E1,5.2E1,2.6E1,1.6E1,4.6E1,3.8E1,6.4E1,4E1,1.1E1,9E0,3E0,4.9E1,1.3E1,3.3E1,1.3E1,2.5E1,4.9E1,1.5E1,1E1,3E1,4.2E1,7E0,1.2E1,1E0,6E0,7E0,4.6E1,3E0,1.7E1,2.5E1,1.2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-5.8905076E-4,-2.2802964E-2,6.124458E-2,-3.0460376E-2,6.1969485E-2,3.2799777E-2,1.3222851E-1,-2.749425E-2,-1.2407606E-2,1.718641E-3,9.0936795E-3,4.1141883E-2,-1.0719525E-2,6.3347906E-2,2.532306E-1,-3.424972E-2,1.3202839E-2,8.490057E-3,3.4360126E-2,5.821894E-3,1.0425934E-3,4.3374733E-3,1.49462E-2,-2.9194035E-2,-8.080763E-2,-3.1669595E-5,6.0598273E-3,3.0099688E-2,9.481855E-3,-2.646238E-2,-6.4945067E-3,-1.8529937E-3,-6.6514835E-3,3.6367524E-2,-4.9800174E-3,-4.6864793E-2,-1.1972643E-2,1.6971994E-2,3.4260624E-3,-2.5660405E-3,3.4495862E-3,3.9442282E-2,-1.9097282E-2,2.9542698E-3,-2.2979111E-4,3.478441E-4,5.797586E-3,-2.5193265E-3,-3.3893393E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,31,-1,-1,33,-1,35,-1,-1,-1,37,-1,39,41,43,-1,-1,-1,45,47,-1,-1,-1,-1,-1,-1],"loss_changes":[6.2362236E-1,2.1869954E-1,2.3795778E-1,1.915124E-1,8.64052E-2,1.8830067E-1,2.673381E-1,8.40303E-2,0E0,0E0,0E0,6.837517E-2,0E0,4.950262E-2,6.559169E-2,5.9075624E-2,6.585952E-2,0E0,4.856789E-2,0E0,0E0,0E0,0E0,6.1792225E-2,5.2226588E-2,0E0,0E0,6.949617E-2,0E0,6.8024054E-2,0E0,0E0,0E0,4.752992E-2,0E0,5.339223E-2,5.093808E-2,4.5721635E-2,0E0,0E0,0E0,4.1471347E-2,4.631998E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,13,13,14,14,15,15,16,16,18,18,23,23,24,24,27,27,29,29,33,33,35,35,36,36,37,37,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,32,-1,-1,34,-1,36,-1,-1,-1,38,-1,40,42,44,-1,-1,-1,46,48,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4817759E9,2.9034685E3,1.486066E0,4.836269E3,6.0495757E3,3.768034E4,2.2876814E1,1.5135906E3,-1.2407606E-2,1.718641E-3,9.0936795E-3,1.5988636E1,-1.0719525E-2,1.0647493E2,1.8636872E0,1E0,1.7670108E5,8.490057E-3,3.305542E4,5.821894E-3,1.0425934E-3,4.3374733E-3,1.49462E-2,4.086885E7,1.3485402E8,-3.1669595E-5,6.0598273E-3,1.4609149E4,9.481855E-3,4.407E3,-6.4945067E-3,-1.8529937E-3,-6.6514835E-3,2.5065362E9,-4.9800174E-3,7.10711E8,5.7844446E2,5.473125E3,3.4260624E-3,-2.5660405E-3,3.4495862E-3,1.303394E3,7.1032936E7,2.9542698E-3,-2.2979111E-4,3.478441E-4,5.797586E-3,-2.5193265E-3,-3.3893393E-4],"split_indices":[7,64,51,4,64,4,66,64,0,0,0,68,0,70,51,19,40,0,4,0,0,0,0,57,7,0,0,4,0,2,0,0,0,7,0,7,59,44,0,0,0,60,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.52E2,3.33E2,1.19E2,3.06E2,2.7E1,8.6E1,3.3E1,3.03E2,3E0,2.3E1,4E0,8.4E1,2E0,2.2E1,1.1E1,2.6E2,4.3E1,3E0,8.1E1,9E0,1.3E1,3E0,8E0,2.36E2,2.4E1,3.9E1,4E0,8E1,1E0,2.31E2,5E0,1.4E1,1E1,7.7E1,3E0,9.5E1,1.36E2,4.9E1,2.8E1,9.2E1,3E0,1.6E1,1.2E2,1.6E1,3.3E1,1.2E1,4E0,3.3E1,8.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-7.499905E-4,-2.3788698E-2,4.984043E-2,-5.3966142E-2,-1.2720708E-2,3.0126855E-2,1.3774073E-1,-8.1066124E-2,-3.0484688E-2,-1.6807934E-2,4.9496304E-2,7.1392097E-3,7.973511E-2,3.9393133E-3,1.1680582E-2,-5.866339E-3,-4.43277E-2,-7.157993E-5,-3.0381735E-3,-2.2309765E-2,2.1548534E-3,4.408565E-3,-1.8909608E-3,-3.943479E-2,4.5082953E-2,9.267941E-3,3.0348385E-3,-3.791648E-3,1.2537962E-3,-2.7007198E-2,1.967339E-2,-6.917565E-3,-9.4978325E-4,5.9343453E-2,-1.1235789E-1,1.69204E-3,-3.0532258E-2,2.9136832E-3,-1.700024E-3,8.866031E-3,2.2666254E-3,4.3668235E-3,-1.0398159E-2,-9.822416E-3,-2.1126342E-3,-2.003821E-3,5.4463476E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,-1,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,45,-1,-1,-1],"loss_changes":[5.1402235E-1,1.0051662E-1,2.3498073E-1,4.9072847E-2,5.753971E-2,1.2943372E-1,1.22951776E-1,4.4162884E-2,3.9536156E-2,7.019909E-2,5.067612E-2,1.41386E-1,6.1081216E-2,0E0,0E0,0E0,4.569292E-2,0E0,0E0,3.8593434E-2,0E0,0E0,0E0,7.0169345E-2,1.0545585E-1,0E0,0E0,0E0,0E0,3.836541E-2,4.3462254E-2,0E0,0E0,5.966948E-2,9.464298E-2,0E0,3.9704025E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.870043E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,19,19,23,23,24,24,29,29,30,30,33,33,34,34,36,36,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,-1,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,46,-1,-1,-1],"split_conditions":[1.0863E4,4.069E3,9.908038E1,1.65602E5,2.4082131E5,1.646359E5,1E0,2.39E2,3.5389444E5,1.2678202E9,2.823537E1,4.98884E1,5.210789E-2,3.9393133E-3,1.1680582E-2,-5.866339E-3,3.5254697E2,-7.157993E-5,-3.0381735E-3,1.7985837E3,2.1548534E-3,4.408565E-3,-1.8909608E-3,1.2799757E9,3.768034E4,9.267941E-3,3.0348385E-3,-3.791648E-3,1.2537962E-3,5.0442883E1,6.7211235E1,-6.917565E-3,-9.4978325E-4,2.909019E3,1.92144E5,1.69204E-3,1.18E2,2.9136832E-3,-1.700024E-3,8.866031E-3,2.2666254E-3,4.3668235E-3,-1.0398159E-2,6.606756E1,-2.1126342E-3,-2.003821E-3,5.4463476E-4],"split_indices":[2,2,68,9,40,40,13,3,60,7,68,68,50,0,0,0,45,0,0,64,0,0,0,7,4,0,0,0,0,60,68,0,0,44,2,0,3,0,0,0,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.02E2,1.37E2,8E1,2.22E2,1.13E2,2.4E1,3.6E1,4.4E1,2.09E2,1.3E1,7.8E1,3.5E1,1.6E1,8E0,1.7E1,1.9E1,2.3E1,2.1E1,1.92E2,1.7E1,9E0,4E0,3.5E1,4.3E1,4E0,3.1E1,1.3E1,6E0,1.73E2,1.9E1,5E0,3E1,4E1,3E0,9E0,1.64E2,1.1E1,8E0,3E0,3.7E1,1E0,2E0,6E1,1.04E2,2.4E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.3846618E-3,-2.8292919E-2,4.0907655E-2,-3.1308882E-2,1.2773425E-2,2.8989794E-2,1.3182159E-1,-2.7768802E-2,-1.077002E-2,3.3040933E-2,-9.996281E-2,4.056247E-3,9.231509E-3,-5.6596645E-3,-2.2985715E-2,2.6974764E-2,1.625901E-2,4.2577037E-3,-8.373539E-3,-2.4491964E-2,9.023375E-3,-1.5710019E-2,3.690004E-2,1.6839002E-3,-2.905333E-2,1.1015086E-4,-6.4081387E-3,3.694021E-3,2.1758558E-2,-7.132073E-3,-2.4973746E-2,7.930922E-3,2.9136955E-3,-4.1895144E-2,3.0376622E-4,-2.8510723E-4,3.4299132E-3,9.094786E-4,-2.339734E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,-1,-1,-1,19,21,-1,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,31,-1,33,35,-1,37,-1,-1,-1,-1,-1],"loss_changes":[5.121251E-1,2.4345255E-1,1.8579093E-1,1.7202917E-1,0E0,8.479802E-2,3.8600415E-2,1.0737808E-1,0E0,2.6073694E-1,7.6846756E-2,0E0,0E0,0E0,8.29069E-2,6.412846E-2,0E0,0E0,0E0,6.8657964E-2,0E0,5.873466E-2,6.6769406E-2,0E0,1.06325954E-1,0E0,0E0,0E0,4.4004593E-2,0E0,1.2110652E-1,5.4191716E-2,0E0,4.492253E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,14,14,15,15,19,19,21,21,22,22,24,24,28,28,30,30,31,31,33,33],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,-1,-1,-1,20,22,-1,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,32,-1,34,36,-1,38,-1,-1,-1,-1,-1],"split_conditions":[1.0136646E5,6.0495757E3,7.765071E0,7.339209E6,1.2773425E-2,3.7826266E4,2.833843E8,1.44E2,-1.077002E-2,1.4609149E4,1.92144E5,4.056247E-3,9.231509E-3,-5.6596645E-3,3.305542E4,9.9800183E2,1.625901E-2,4.2577037E-3,-8.373539E-3,1.27E2,9.023375E-3,1E0,6.527467E6,1.6839002E-3,2.2744698E2,1.1015086E-4,-6.4081387E-3,3.694021E-3,1.8822492E0,-7.132073E-3,1.173433E5,2.6691768E7,2.9136955E-3,6.281509E1,3.0376622E-4,-2.8510723E-4,3.4299132E-3,9.094786E-4,-2.339734E-3],"split_indices":[45,64,47,59,0,64,44,0,0,4,2,0,0,0,4,64,0,0,0,11,0,19,62,0,64,0,0,0,53,0,40,60,0,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.48E2,2.74E2,1.74E2,2.72E2,2E0,1.55E2,1.9E1,2.68E2,4E0,1.51E2,4E0,1.1E1,8E0,1.3E1,2.55E2,1.49E2,2E0,1E0,3E0,2.54E2,1E0,2.8E1,1.21E2,1.8E1,2.36E2,2.5E1,3E0,3.4E1,8.7E1,7E0,2.29E2,6.4E1,2.3E1,1.48E2,8.1E1,5.3E1,1.1E1,1.1E1,1.37E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.9832794E-4,-1.3666869E-2,7.710252E-2,-3.2414395E-2,1.7542977E-2,1.9919232E-1,5.0046735E-2,-2.859398E-2,-1.0078713E-2,1.4275513E-3,7.239154E-2,3.1677037E-3,1.4350286E-2,1.02148E-2,3.1275023E-2,-4.2730268E-2,-8.055775E-3,-2.6946248E-2,1.6776865E-2,9.223771E-3,2.7198612E-3,2.701284E-3,-5.8533017E-2,-5.1055796E-2,4.0533545E-4,4.526924E-2,-8.526822E-4,-9.3447656E-4,-8.504971E-3,1.7791375E-3,-8.942937E-4,-1.12102956E-1,3.1604932E-3,-3.392925E-3,-3.0157525E-2,1.837462E-4,5.9671695E-3,-9.23903E-3,-1.5727225E-3,1.9886768E-3,-1.9855648E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,-1,31,33,-1,35,-1,-1,-1,-1,-1,37,-1,-1,39,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7406402E-1,2.1939884E-1,2.1465111E-1,1.4554481E-1,1.24527685E-1,1.2513253E-1,1.5875697E-1,6.6432595E-2,0E0,4.8359092E-2,5.4452732E-2,0E0,0E0,0E0,1.0953051E-1,5.7979763E-2,4.639581E-2,4.289117E-2,4.7625307E-2,0E0,0E0,0E0,7.8846574E-2,3.9116204E-2,0E0,4.272733E-2,0E0,0E0,0E0,0E0,0E0,4.1121297E-2,0E0,0E0,3.7740257E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,15,15,16,16,17,17,18,18,22,22,23,23,25,25,31,31,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,-1,32,34,-1,36,-1,-1,-1,-1,-1,38,-1,-1,40,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,7.222818E4,1.064816E6,5.0350455E6,1.646359E5,1.082556E7,1.3812E4,5.855E3,-1.0078713E-2,2.0231652E3,1.4E1,3.1677037E-3,1.4350286E-2,1.02148E-2,4.42384E5,2.1592189E5,4.9832974E2,2.0192318E3,1.1106066E7,9.223771E-3,2.7198612E-3,2.701284E-3,2.785032E9,1.6776951E3,4.0533545E-4,6.651711E1,-8.526822E-4,-9.3447656E-4,-8.504971E-3,1.7791375E-3,-8.942937E-4,9.820362E4,3.1604932E-3,-3.392925E-3,4.2969458E2,1.837462E-4,5.9671695E-3,-9.23903E-3,-1.5727225E-3,1.9886768E-3,-1.9855648E-3],"split_indices":[64,45,41,59,40,1,9,2,0,67,3,0,0,0,2,40,64,67,60,0,0,0,7,4,0,66,0,0,0,0,0,40,0,0,64,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.73E2,6.7E1,2.33E2,1.4E2,1.1E1,5.6E1,2.29E2,4E0,1.09E2,3.1E1,5E0,6E0,5E0,5.1E1,1.35E2,9.4E1,3.8E1,7.1E1,3E0,2.8E1,4.1E1,1E1,1.16E2,1.9E1,1.3E1,8.1E1,3.7E1,1E0,4.6E1,2.5E1,7E0,3E0,6.3E1,5.3E1,9E0,4E0,3E0,4E0,6E0,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[6.5604253E-3,-1.9894673E-2,4.1327238E-2,-2.2994876E-2,1.1938625E-2,1.635464E-2,8.0101505E-2,-1.9575879E-2,-1.11567015E-2,8.897198E-3,3.4742989E-3,9.3693696E-2,1.5992568E-4,-2.8514485E-3,-1.293088E-2,-2.5995204E-4,2.0667922E-3,7.978188E-2,1.5805694E-1,2.215752E-4,-3.689661E-2,4.7045415E-3,4.3210274E-4,1.07525345E-2,9.3073264E-4,-4.0708864E-3,3.9321505E-3,-6.328488E-3,-1.1946433E-3,-3.0273305E-2,1.0499983E-2,1.0152361E-3,-2.650745E-3,-3.8711175E-2,2.4723245E-2,5.5133854E-4,-4.7580833E-3,2.063624E-3,-5.976653E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,-1,19,-1,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,31,33,-1,-1,35,37,-1,-1,-1,-1],"loss_changes":[3.9815828E-1,2.0285283E-1,1.7976102E-1,1.6268216E-1,0E0,4.4975728E-2,7.600415E-2,5.928497E-2,0E0,4.6892606E-2,0E0,4.290527E-2,0E0,0E0,6.494105E-2,0E0,0E0,5.330333E-2,7.5604886E-2,4.5449167E-2,8.207991E-2,0E0,0E0,0E0,0E0,4.9187247E-2,0E0,0E0,0E0,5.4057036E-2,5.9052348E-2,0E0,0E0,5.4445926E-2,4.000004E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,14,14,17,17,18,18,19,19,20,20,25,25,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,-1,20,-1,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,32,34,-1,-1,36,38,-1,-1,-1,-1],"split_conditions":[8.393928E4,8.861901E3,1.3590209E7,7.339209E6,1.1938625E-2,2.4251762E5,1.773296E6,4.26944E5,-1.11567015E-2,2.34E2,3.4742989E-3,3.63796E5,1.5992568E-4,-2.8514485E-3,1.8071064E7,-2.5995204E-4,2.0667922E-3,4.307733E1,7.9E1,6.46E2,1.9281628E7,4.7045415E-3,4.3210274E-4,1.07525345E-2,9.3073264E-4,7.959718E4,3.9321505E-3,-6.328488E-3,-1.1946433E-3,1.1347253E1,3.9560037E2,1.0152361E-3,-2.650745E-3,1.0547217E-5,6.427212E4,5.5133854E-4,-4.7580833E-3,2.063624E-3,-5.976653E-4],"split_indices":[45,64,1,59,0,40,11,1,0,8,0,2,0,0,57,0,0,69,8,8,57,0,0,0,0,40,0,0,0,70,64,0,0,51,59,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.45E2,1.86E2,2.43E2,2E0,1.14E2,7.2E1,2.4E2,3E0,1.01E2,1.3E1,6.1E1,1.1E1,3.5E1,2.05E2,7.1E1,3E1,5.2E1,9E0,1.33E2,7.2E1,4.3E1,9E0,6E0,3E0,1.27E2,6E0,8E0,6.4E1,4.5E1,8.2E1,1.4E1,3.1E1,1.8E1,6.4E1,1E1,8E0,4.4E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.5774983E-3,-1.5281041E-2,6.605846E-2,-1.2914552E-2,-2.3331916E-1,4.5070697E-2,8.722418E-3,-1.5305939E-2,8.3514685E-3,-1.5380709E-2,-2.4969102E-4,8.035125E-2,-1.1198101E-2,-4.5491386E-2,-3.6910432E-3,2.945361E-3,8.181789E-3,-2.710587E-3,1.4689042E-3,-4.8944904E-3,-3.74993E-2,1.8921254E-2,-1.5357561E-2,-2.2661048E-3,1.5280386E-3,-4.9741205E-2,1.3020115E-3,-2.2505603E-3,1.13149064E-4,1.7212956E-3,-6.195863E-3,6.982532E-3,-2.794035E-3,9.487465E-4,-2.7128158E-2,1.973539E-3,-3.0366655E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,17,19,21,-1,-1,-1,-1,-1,23,25,27,-1,-1,29,-1,-1,31,-1,-1,33,-1,-1,35,-1,-1],"loss_changes":[3.907576E-1,1.9202907E-1,1.5149659E-1,1.639978E-1,6.644985E-2,1.2001145E-1,0E0,1.29905E-1,0E0,0E0,0E0,5.6406975E-2,4.3521855E-2,3.949958E-2,7.125461E-2,0E0,0E0,0E0,0E0,0E0,4.99814E-2,4.63448E-2,8.172422E-2,0E0,0E0,6.0514644E-2,0E0,0E0,4.5799945E-2,0E0,0E0,4.390914E-2,0E0,0E0,6.513734E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,12,12,13,13,14,14,20,20,21,21,22,22,25,25,28,28,31,31,34,34],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,18,20,22,-1,-1,-1,-1,-1,24,26,28,-1,-1,30,-1,-1,32,-1,-1,34,-1,-1,36,-1,-1],"split_conditions":[3.5257432E6,1.69595E5,1.6828056E1,6.6512886E3,8.75E3,2.3619232E7,8.722418E-3,4.407E3,8.3514685E-3,-1.5380709E-2,-2.4969102E-4,2.36316E5,5.0314346E1,6.506511E0,7.37365E6,2.945361E-3,8.181789E-3,-2.710587E-3,1.4689042E-3,-4.8944904E-3,2.681357E6,1.366276E6,1.1767653E9,-2.2661048E-3,1.5280386E-3,1.2E3,1.3020115E-3,-2.2505603E-3,4.2663252E7,1.7212956E-3,-6.195863E-3,1.73E2,-2.794035E-3,9.487465E-4,1.0717E4,1.973539E-3,-3.0366655E-3],"split_indices":[56,2,47,64,0,55,0,2,0,0,0,2,68,68,57,0,0,0,0,0,1,57,5,0,0,11,0,0,57,0,0,8,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,3.77E2,6.9E1,3.74E2,3E0,5.9E1,1E1,3.7E2,4E0,2E0,1E0,3.6E1,2.3E1,1.02E2,2.68E2,3E1,6E0,1.1E1,1.2E1,1.2E1,9E1,9.1E1,1.77E2,8.1E1,9E0,8E0,8.3E1,6E1,1.17E2,4E0,4E0,1.05E2,1.2E1,7.8E1,2.7E1,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.6588347E-3,-9.5964605E-3,1.378394E-1,-2.8568476E-2,2.2045098E-2,1.2562832E-2,4.6895545E-2,-2.5289115E-2,-9.303901E-3,8.972755E-3,3.5376062E-3,8.821252E-3,5.2895217E-4,-3.574927E-2,6.160709E-3,-1.3929022E-3,1.0439812E-3,-5.4965876E-2,-2.149367E-2,2.4303705E-3,-9.7329856E-4,-2.3981698E-2,-8.7765075E-2,-4.824239E-3,-1.2086107E-2,-2.8381802E-3,1.1453906E-3,-5.9536034E-3,-2.5053585E-3,-2.3366518E-2,4.7186404E-2,-8.572188E-3,-1.946792E-2,-7.9979276E-4,4.332962E-3,8.783337E-5,-2.2654224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,-1,17,19,-1,-1,21,23,-1,-1,25,27,-1,29,-1,-1,-1,-1,31,33,-1,35,-1,-1,-1,-1],"loss_changes":[4.730261E-1,2.4315268E-1,2.2569886E-1,1.260482E-1,9.621281E-2,0E0,6.4022355E-2,8.213383E-2,0E0,5.370925E-2,0E0,0E0,0E0,5.004616E-2,6.953896E-2,0E0,0E0,7.817949E-2,7.5778365E-2,0E0,0E0,6.592953E-2,3.8881212E-2,0E0,6.6627175E-2,0E0,0E0,0E0,0E0,4.4602677E-2,4.1338008E-2,0E0,4.5092463E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,13,13,14,14,17,17,18,18,21,21,22,22,24,24,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,-1,18,20,-1,-1,22,24,-1,-1,26,28,-1,30,-1,-1,-1,-1,32,34,-1,36,-1,-1,-1,-1],"split_conditions":[8.10288E3,9.7467445E4,3.0418E4,7.339209E6,1.7670108E5,1.2562832E-2,9.812318E3,7.198E3,-9.303901E-3,3.6308475E1,3.5376062E-3,8.821252E-3,5.2895217E-4,7.683712E4,2.1933604E3,-1.3929022E-3,1.0439812E-3,3.5546432E6,9.284744E2,2.4303705E-3,-9.7329856E-4,1.3202006E3,2.55E2,-4.824239E-3,7.2008003E9,-2.8381802E-3,1.1453906E-3,-5.9536034E-3,-2.5053585E-3,2.90694E5,5.0563E4,-8.572188E-3,1.7197124E7,-7.9979276E-4,4.332962E-3,8.783337E-5,-2.2654224E-3],"split_indices":[64,45,10,59,40,0,4,2,0,68,0,0,0,40,44,0,0,57,4,0,0,4,3,0,5,0,0,0,0,1,12,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,4.03E2,2.2E1,2.52E2,1.51E2,9E0,1.3E1,2.48E2,4E0,1.2E2,3.1E1,2E0,1.1E1,1.86E2,6.2E1,2.9E1,9.1E1,7.8E1,1.08E2,2.3E1,3.9E1,4.1E1,3.7E1,1.1E1,9.7E1,2.4E1,1.7E1,1.9E1,1.8E1,8.2E1,1.5E1,1E0,8.1E1,6E0,9E0,4.5E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.5892266E-4,-1.09341815E-2,9.025434E-2,-9.21702E-3,-1.1463298E-2,1.9932939E-1,5.441878E-2,-1.7021887E-2,2.9119173E-2,1.6513603E-2,5.44054E-3,-1.0448571E-2,6.792054E-2,-2.365136E-2,2.7895581E-2,4.4368234E-4,5.5452664E-3,-8.3438033E-4,8.6003035E-2,-4.3994483E-2,-1.34084765E-2,4.4338023E-3,1.0974945E-2,1.00451045E-1,-1.4697844E-3,-6.001143E-2,-1.2568874E-2,-8.357987E-3,-1.1700653E-2,-1.7247086E-3,1.8668042E-3,3.1580643E-3,8.186527E-3,-4.6693064E-2,-6.2269014E-3,-1.5862783E-3,3.947845E-3,1.2214053E-3,-2.878561E-2,-2.699951E-3,4.6733543E-3,-4.8144758E-4,2.0868229E-3,-1.0428665E-3,-5.4729963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,17,19,21,-1,-1,-1,23,25,27,-1,29,31,-1,33,35,-1,37,-1,-1,-1,-1,39,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2130667E-1,1.4417459E-1,1.6895592E-1,1.16588704E-1,0E0,9.4490886E-2,1.4227319E-1,9.6842445E-2,1.0834597E-1,0E0,0E0,0E0,5.498837E-2,5.817108E-2,4.2251132E-2,0E0,0E0,0E0,5.131127E-2,4.65599E-2,4.7696646E-2,0E0,4.208454E-2,5.1026434E-2,0E0,4.7513485E-2,6.0437035E-2,0E0,4.1540448E-2,0E0,0E0,0E0,0E0,5.953349E-2,0E0,0E0,0E0,4.792294E-2,4.9491443E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,12,12,13,13,14,14,18,18,19,19,20,20,22,22,23,23,25,25,26,26,28,28,33,33,37,37,38,38],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,18,20,22,-1,-1,-1,24,26,28,-1,30,32,-1,34,36,-1,38,-1,-1,-1,-1,40,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,3.372042E7,1.417988E6,1.646359E5,-1.1463298E-2,4.542969E1,1.461379E6,3.73E2,1.1895911E-1,1.6513603E-2,5.44054E-3,-1.0448571E-2,3.6308475E1,1.6858337E3,2.96147E5,4.4368234E-4,5.5452664E-3,-8.3438033E-4,2.3423826E10,2.033552E6,2.90694E5,4.4338023E-3,1.570662E5,2.20229E5,-1.4697844E-3,6.8E1,1.875E3,-8.357987E-3,1.8071064E7,-1.7247086E-3,1.8668042E-3,3.1580643E-3,8.186527E-3,5.4630665E1,-6.2269014E-3,-1.5862783E-3,3.947845E-3,1.963372E6,2.11E2,-2.699951E-3,4.6733543E-3,-4.8144758E-4,2.0868229E-3,-1.0428665E-3,-5.4729963E-3],"split_indices":[64,60,41,40,0,66,41,8,69,0,0,0,68,4,41,0,0,0,12,9,1,0,59,2,0,8,0,0,57,0,0,0,0,68,0,0,0,41,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,3.89E2,4.5E1,3.87E2,2E0,1E1,3.5E1,3.22E2,6.5E1,3E0,7E0,1E0,3.4E1,2.81E2,4.1E1,5.3E1,1.2E1,6E0,2.8E1,9.3E1,1.88E2,8E0,3.3E1,2.5E1,3E0,6.1E1,3.2E1,1E0,1.87E2,1.2E1,2.1E1,1.7E1,8E0,5.2E1,9E0,2.7E1,5E0,1.07E2,8E1,5E1,2E0,8.5E1,2.2E1,7.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[4.1947532E-3,-6.4652883E-3,7.40463E-2,-4.352768E-3,-8.345571E-3,1.5560552E-1,4.270553E-2,-2.5049772E-2,1.4910038E-2,2.3859008E-3,1.0772358E-2,3.0541616E-3,-7.828351E-3,-3.276617E-2,8.7902957E-4,7.6503777E-3,7.0129596E-2,-1.8673334E-3,-2.063619E-3,-3.9048255E-3,1.008547E-2,6.597789E-3,1.0451834E-3,-1.6532236E-3,2.0967615E-3,8.6663617E-4,-1.0038671E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,23,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3183482E-1,1.3051042E-1,1.4416638E-1,1.5306884E-1,0E0,9.3164116E-2,1.7102532E-1,6.1444618E-2,7.923764E-2,0E0,0E0,0E0,0E0,4.1283324E-2,0E0,3.7762E-2,6.729565E-2,4.9262088E-2,0E0,0E0,3.8221046E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,24,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,4.2064876E7,1.417988E6,6.24E3,-8.345571E-3,1.2273383E7,2.3767492E4,2.1592189E5,7.049895E9,2.3859008E-3,1.0772358E-2,3.0541616E-3,-7.828351E-3,4.3712845E0,8.7902957E-4,6.7440504E7,1.3590306E10,5.24228E5,-2.063619E-3,-3.9048255E-3,2.4722598E7,6.597789E-3,1.0451834E-3,-1.6532236E-3,2.0967615E-3,8.6663617E-4,-1.0038671E-3],"split_indices":[64,62,41,2,0,57,64,40,12,0,0,0,0,70,0,7,12,9,0,0,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.44E2,3.86E2,5.8E1,3.82E2,4E0,1.5E1,4.3E1,1.84E2,1.98E2,6E0,9E0,4E1,3E0,1.56E2,2.8E1,1.76E2,2.2E1,3.4E1,1.22E2,4E0,1.72E2,9E0,1.3E1,2E1,1.4E1,1.39E2,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[5.0503097E-4,-1.5768487E-2,4.367275E-2,-4.909478E-2,-8.844461E-3,1.4040245E-1,2.8660033E-2,-2.8972556E-3,1.6527841E-3,-7.8774395E-4,2.387284E-2,2.1866704E-3,2.1071161E-1,3.8257696E-2,-7.903229E-2,-3.8938315E-3,1.6284013E-3,5.468569E-3,1.5490864E-2,1.6072996E-2,6.870782E-2,-6.49673E-3,3.3098636E-3,1.0861254E-3,-4.744543E-3,1.4262411E-1,2.0121906E-3,-1.181502E-3,8.076875E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[3.1048703E-1,7.3637225E-2,1.7168932E-1,4.203847E-2,6.080825E-2,9.950438E-2,1.125846E-1,0E0,0E0,0E0,4.4210855E-2,0E0,5.656144E-2,6.509729E-2,7.523803E-2,0E0,0E0,0E0,0E0,3.8504772E-2,8.046125E-2,0E0,0E0,0E0,0E0,3.8553223E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,14,14,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[9.407814E6,3.807E3,1.417988E6,4.049296E-2,1.5271514E5,9.256843E3,1.5291998E4,-2.8972556E-3,1.6527841E-3,-7.8774395E-4,4.4593253E2,2.1866704E-3,1.281E3,7.2038723E9,1.2408761E0,-3.8938315E-3,1.6284013E-3,5.468569E-3,1.5490864E-2,8.6206274E2,2.1207E4,-6.49673E-3,3.3098636E-3,1.0861254E-3,-4.744543E-3,1.2511627E7,2.0121906E-3,-1.181502E-3,8.076875E-3],"split_indices":[1,2,41,69,40,4,4,0,0,0,64,0,0,12,46,0,0,0,0,70,9,0,0,0,0,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.2E2,1.2E2,5.4E1,2.66E2,1.5E1,1.05E2,4.9E1,5E0,2.2E2,4.6E1,7E0,8E0,9.7E1,8E0,3E0,4.3E1,5E0,3E0,5.7E1,4E1,6E0,2E0,5.5E1,2E0,1E1,3E1,1E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.267155E-3,-1.8551797E-2,3.064989E-2,-1.6366217E-2,-8.723522E-3,2.5247024E-2,7.6668016E-3,-4.7053327E-3,-1.32957725E-2,3.0088557E-2,-8.950915E-2,-2.1004928E-2,1.5835155E-2,1.1270756E-2,2.7374548E-3,-6.3549865E-5,-8.8619E-3,-3.553209E-3,-1.2933667E-2,1.8717118E-3,-2.0568362E-2,1.6429644E-3,-7.720068E-4,-2.0668117E-2,1.4318541E-3,-2.4964176E-3,2.0375475E-3,-2.9045392E-2,1.2611569E-3,-3.100027E-2,6.0641654E-3,4.9184648E-5,-1.9896654E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,-1,-1,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,25,-1,-1,27,-1,-1,-1,29,-1,31,-1,-1,-1],"loss_changes":[2.3803532E-1,9.8406844E-2,9.362508E-2,6.8965964E-2,0E0,8.15878E-2,0E0,0E0,6.411187E-2,6.302603E-2,4.6265297E-2,9.005148E-2,4.7986403E-2,4.61089E-2,0E0,0E0,0E0,0E0,6.331056E-2,0E0,4.3491147E-2,0E0,0E0,6.418285E-2,0E0,0E0,0E0,4.4919915E-2,0E0,3.921625E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,9,9,10,10,11,11,12,12,13,13,18,18,20,20,23,23,27,27,29,29],"right_children":[2,4,6,8,-1,10,-1,-1,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,26,-1,-1,28,-1,-1,-1,30,-1,32,-1,-1,-1],"split_conditions":[1.279709E5,7.339209E6,1.0871896E10,2.398E3,-8.723522E-3,5.37E2,7.6668016E-3,-4.7053327E-3,2.332E3,1.5333999E7,1.4912397E7,4.516183E1,2.2980049E10,2.0421524E6,2.7374548E-3,-6.3549865E-5,-8.8619E-3,-3.553209E-3,1.643477E9,1.8717118E-3,3.7137297E3,1.6429644E-3,-7.720068E-4,3.94E2,1.4318541E-3,-2.4964176E-3,2.0375475E-3,1.9381E4,1.2611569E-3,1.1347253E1,6.0641654E-3,4.9184648E-5,-1.9896654E-3],"split_indices":[45,59,7,2,0,3,0,0,0,1,57,66,5,59,0,0,0,0,5,0,44,0,0,8,0,0,0,2,0,70,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,2.96E2,1.46E2,2.93E2,3E0,1.41E2,5E0,1E1,2.83E2,1.36E2,5E0,2.24E2,5.9E1,7.8E1,5.8E1,3E0,2E0,3E1,1.94E2,3.7E1,2.2E1,4.3E1,3.5E1,1.64E2,3E1,1.5E1,7E0,1.39E2,2.5E1,1.38E2,1E0,3E1,1.08E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.3851766E-3,-8.555901E-3,8.765691E-2,-1.754065E-2,2.780584E-2,7.671462E-3,2.473388E-3,-1.5805518E-2,-1.032541E-2,3.6449753E-2,-2.941969E-3,-2.2944536E-3,-8.304134E-3,9.346795E-3,1.5996163E-3,-1.5579082E-2,2.1913178E-2,8.616356E-4,-2.9919378E-2,3.382069E-3,1.033959E-4,7.201604E-4,-2.7057605E-2,-4.991514E-2,-6.471156E-3,-4.8421015E-4,-6.289924E-3,-3.8345456E-2,-5.7508796E-3,1.3765211E-4,-4.1116285E-3,-4.3159006E-3,-9.0862904E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,-1,15,-1,-1,17,19,21,23,-1,-1,-1,25,27,29,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.9126576E-1,1.3835563E-1,7.784611E-2,1.0794522E-1,6.5078266E-2,0E0,0E0,7.572448E-2,0E0,4.5517102E-2,0E0,0E0,5.991514E-2,0E0,0E0,5.165826E-2,4.7988504E-2,3.9350975E-2,5.4534204E-2,0E0,0E0,0E0,5.7126373E-2,4.3080136E-2,3.8688175E-2,0E0,0E0,5.1678687E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,12,12,15,15,16,16,17,17,18,18,22,22,23,23,24,24,27,27],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,-1,16,-1,-1,18,20,22,24,-1,-1,-1,26,28,30,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[6.204522E3,1.5522031E9,1.681178E6,3.764913E7,2.785563E7,7.671462E-3,2.473388E-3,1.0780007E3,-1.032541E-2,1.2662238E1,-2.941969E-3,-2.2944536E-3,1.5271514E5,9.346795E-3,1.5996163E-3,2.7977833E1,2.5159248E1,2.9991518E7,9.606877E-2,3.382069E-3,1.033959E-4,7.201604E-4,6.54925E6,1E0,1.3958886E-1,-4.8421015E-4,-6.289924E-3,7.1032936E7,-5.7508796E-3,1.3765211E-4,-4.1116285E-3,-4.3159006E-3,-9.0862904E-4],"split_indices":[64,7,41,60,55,0,0,67,0,68,0,0,40,0,0,70,68,57,50,0,0,0,9,19,69,0,0,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.54E2,4.21E2,3.3E1,3.38E2,8.3E1,1.1E1,2.2E1,3.36E2,2E0,7.6E1,7E0,6.6E1,2.7E2,1E0,7.5E1,2.18E2,5.2E1,1.02E2,1.16E2,1.5E1,3.7E1,6.9E1,3.3E1,6.2E1,5.4E1,2.9E1,4E0,5.4E1,8E0,4.9E1,5E0,1.5E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.3344603E-3,-2.2538431E-2,2.0072022E-2,-1.1229228E-2,-2.1540523E-3,1.0385378E-1,1.2850213E-2,-2.3204312E-3,3.520242E-3,1.2997606E-2,1.7881122E-3,1.7302338E-2,-6.6960733E-3,5.0529605E-4,-3.2470908E-3,6.968949E-3,2.7376635E-3,1.0510577E-2,-8.735646E-3,-1.4848099E-3,8.410847E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,-1,15,-1,-1,-1,17,-1,19,-1,-1,-1],"loss_changes":[1.9699216E-1,5.021271E-2,1.2935753E-1,7.389055E-2,0E0,1.7148578E-1,1.3306448E-1,0E0,4.624884E-2,0E0,0E0,7.5135075E-2,0E0,0E0,0E0,1.00974046E-1,0E0,3.913282E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,11,11,15,15,17,17],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,-1,16,-1,-1,-1,18,-1,20,-1,-1,-1],"split_conditions":[1.0975164E3,1.3058374E7,1.3812E4,5.42218E5,-2.1540523E-3,3.7137297E3,4.242889E3,-2.3204312E-3,1E0,1.2997606E-2,1.7881122E-3,1.8770729E0,-6.6960733E-3,5.0529605E-4,-3.2470908E-3,2.5212732E4,2.7376635E-3,4.59E2,-8.735646E-3,-1.4848099E-3,8.410847E-4],"split_indices":[64,57,9,1,0,44,70,0,19,0,0,53,0,0,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,2.17E2,2.15E2,1.41E2,7.6E1,1.6E1,1.99E2,4.1E1,1E2,4E0,1.2E1,1.94E2,5E0,9.2E1,8E0,1.53E2,4.1E1,1.51E2,2E0,2E1,1.31E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.2136965E-3,-1.4582835E-2,3.895704E-2,-1.2849405E-2,-7.7585457E-3,4.847287E-2,-2.1642826E-3,-8.7156694E-4,1.3359499E-3,2.6864478E-2,8.2278736E-2,3.8729828E-2,-5.2115633E-3,2.9061933E-3,1.4551783E-1,1.1539502E-3,1.1221862E-2,9.146057E-3,4.7643282E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,15,-1,-1,17,-1,-1,-1,-1],"loss_changes":[2.225996E-1,7.993151E-2,8.116244E-2,6.0968477E-2,0E0,6.426427E-2,0E0,0E0,0E0,9.273586E-2,4.4939965E-2,1.488387E-1,0E0,0E0,4.4138953E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,11,11,14,14],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,16,-1,-1,18,-1,-1,-1,-1],"split_conditions":[1.6053742E9,3.764913E7,2.00036E5,2.2860639E9,-7.7585457E-3,7.2038723E9,-2.1642826E-3,-8.7156694E-4,1.3359499E-3,5.358657E7,3.63796E5,4.5446856E7,-5.2115633E-3,2.9061933E-3,2.2576077E10,1.1539502E-3,1.1221862E-2,9.146057E-3,4.7643282E-4],"split_indices":[7,60,10,12,0,12,0,0,0,1,2,1,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,3.35E2,1E2,3.32E2,3E0,9E1,1E1,2.98E2,3.4E1,5.6E1,3.4E1,5.2E1,4E0,2.6E1,8E0,4.9E1,3E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.0018868E-4,-8.872676E-3,6.43786E-2,-2.2091264E-2,1.2409362E-2,1.3756613E-1,3.2702204E-2,-1.0198755E-3,-1.0440747E-2,3.2443993E-2,-8.910657E-3,9.02812E-4,1.7589448E-1,2.2559494E-3,-3.0438865E-3,-1.6471515E-4,2.4489632E-3,7.7277753E-3,-7.8923354E-4,1.2278668E-2,4.5448863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5386387E-1,1.077378E-1,1.20690405E-1,6.998452E-2,6.307849E-2,7.010704E-2,4.815881E-2,0E0,0E0,4.5138776E-2,8.347163E-2,0E0,4.8809767E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,8.393928E4,3.0418E4,1.1770536E8,1.0270569E7,1.3590209E7,3.768034E4,-1.0198755E-3,-1.0440747E-2,1.0220021E3,1.2662238E1,9.02812E-4,3.5432243E8,2.2559494E-3,-3.0438865E-3,-1.6471515E-4,2.4489632E-3,7.7277753E-3,-7.8923354E-4,1.2278668E-2,4.5448863E-3],"split_indices":[64,45,10,1,57,1,4,0,0,64,68,0,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,3.81E2,5.3E1,2.35E2,1.46E2,1.5E1,3.8E1,2.34E2,1E0,7.5E1,7.1E1,4E0,1.1E1,3.4E1,4E0,2.4E1,5.1E1,2E0,6.9E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.1175605E-3,-2.2947702E-3,1.6213124E-1,-1.8514453E-2,1.3484989E-2,1.968001E-1,-2.879115E-3,-2.6227355E-2,3.6798354E-4,4.3082857E-3,9.906706E-3,1.1368203E-2,-4.4751805E-4,-1.8123848E-3,6.76526E-4,-2.0128433E-2,9.302147E-4,1.3849856E-3,-1.7244085E-3,-3.286672E-4,-4.82912E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,-1,-1,-1,17,19,-1,-1,-1,-1,-1],"loss_changes":[2.3827605E-1,1.09025314E-1,7.99012E-2,4.210738E-2,5.5353425E-2,5.252549E-2,0E0,4.3773085E-2,0E0,0E0,5.450529E-2,0E0,0E0,0E0,4.375908E-2,4.8090644E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,14,14,15,15],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,-1,-1,-1,18,20,-1,-1,-1,-1,-1],"split_conditions":[3.305542E4,6.97E3,2.6279484E5,9.704588E6,8.547769E7,8.494E3,-2.879115E-3,2.103E3,3.6798354E-4,4.3082857E-3,1.1016051E3,1.1368203E-2,-4.4751805E-4,-1.8123848E-3,1.4399977E10,1E0,9.302147E-4,1.3849856E-3,-1.7244085E-3,-3.286672E-4,-4.82912E-3],"split_indices":[4,2,45,9,7,0,0,0,0,0,64,0,0,0,5,19,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,4.24E2,8E0,2.09E2,2.15E2,7E0,1E0,1.61E2,4.8E1,9E0,2.06E2,6E0,1E0,1.17E2,4.4E1,4.6E1,1.6E2,2.5E1,1.9E1,4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.0222289E-3,-6.2944572E-3,4.8310764E-2,-4.540565E-3,-8.387627E-3,1.052603E-2,3.562798E-2,7.7409536E-4,-1.5621538E-2,1.0020816E-1,1.5275183E-2,-1.0354397E-3,1.1849942E-3,1.2609476E-3,9.169429E-3,-9.848899E-3,1.250565E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,-1,-1,9,-1,11,13,15,-1,-1,-1,-1,-1,-1],"loss_changes":[1.497741E-1,1.0548328E-1,1.1189191E-1,8.2807586E-2,0E0,0E0,7.07606E-2,0E0,4.8607793E-2,7.653242E-2,9.394058E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,9,9,10,10],"right_children":[2,4,6,8,-1,-1,10,-1,12,14,16,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,7.78623E6,8.169258E3,1.04E2,-8.387627E-3,1.052603E-2,1.417988E6,7.7409536E-4,1.484798E9,1.752E5,1.461379E6,-1.0354397E-3,1.1849942E-3,1.2609476E-3,9.169429E-3,-9.848899E-3,1.250565E-3],"split_indices":[64,59,45,3,0,0,41,0,7,2,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,3.74E2,5.7E1,3.71E2,3E0,3E0,5.4E1,1.32E2,2.39E2,1.2E1,4.2E1,2.12E2,2.7E1,7E0,5E0,1E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-2.3966386E-4,-9.587779E-3,3.3721242E-2,-1.1249492E-2,5.4631056E-3,1.6092595E-2,8.192009E-2,-9.733436E-3,-7.0322864E-3,2.5347946E-2,-7.491376E-3,1.3625825E-2,3.167741E-3,-1.8143073E-2,7.150901E-4,7.7934284E-3,1.921363E-2,-7.2814217E-3,-1.9394836E-3,1.3743006E-3,-2.8445057E-3,-1.6929297E-2,1.840906E-3,-2.6223448E-4,-2.3662644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1],"loss_changes":[1.4413743E-1,7.156868E-2,8.195037E-2,6.771883E-2,0E0,1.15950465E-1,7.4631155E-2,7.082224E-2,0E0,5.3081635E-2,0E0,0E0,0E0,5.7851993E-2,0E0,0E0,4.4458613E-2,7.34075E-2,0E0,0E0,0E0,4.9856395E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13,16,16,17,17,21,21],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.6219076E9,8.10288E3,2.388522E-1,4.242889E3,5.4631056E-3,3.768034E4,1.6441814E9,1.4720299E3,-7.0322864E-3,1.5988636E1,-7.491376E-3,1.3625825E-2,3.167741E-3,8.227176E-3,7.150901E-4,7.7934284E-3,1.5953E4,6.2581446E8,-1.9394836E-3,1.3743006E-3,-2.8445057E-3,4.2726562E1,1.840906E-3,-2.6223448E-4,-2.3662644E-3],"split_indices":[7,64,50,70,0,4,7,64,0,68,0,0,0,69,0,0,0,7,0,0,0,65,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.52E2,3.55E2,9.7E1,3.51E2,4E0,7.2E1,2.5E1,3.48E2,3E0,6.9E1,3E0,1E0,2.4E1,2.58E2,9E1,2E0,6.7E1,1.7E2,8.8E1,6.1E1,6E0,1.4E2,3E1,1.02E2,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.4567467E-3,-1.439525E-2,2.21131E-2,-1.6210277E-2,8.379526E-3,1.0068247E-2,5.741454E-2,-1.4920245E-2,-9.817784E-3,-1.277596E-3,2.412581E-2,3.7584007E-3,-2.625312E-4,-2.8482422E-2,-3.3966477E-3,5.66768E-4,7.6855E-2,-5.102506E-2,-1.4886961E-2,-4.1062804E-3,1.8772137E-3,2.420301E-3,1.3285341E-2,-1.1120378E-3,-3.9435457E-3,3.108493E-3,-1.0759338E-3,-6.651892E-3,3.1172158E-2,5.345242E-4,-3.4645557E-2,2.294754E-3,-3.1750596E-3,-4.0618945E-3,-1.5225825E-2,1.3052122E-3,-2.352041E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,15,-1,-1,17,19,-1,21,23,25,-1,27,-1,-1,-1,-1,-1,-1,29,31,-1,33,-1,-1,-1,35,-1,-1],"loss_changes":[1.3937941E-1,1.0020207E-1,6.830444E-2,6.519563E-2,0E0,6.172713E-2,4.6102807E-2,4.5538493E-2,0E0,0E0,5.8453083E-2,0E0,0E0,4.030838E-2,6.620489E-2,0E0,7.844783E-2,3.8359836E-2,4.4859946E-2,0E0,3.7686307E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.7188086E-2,5.0394636E-2,0E0,3.9512943E-2,0E0,0E0,0E0,4.4665188E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,10,10,13,13,14,14,16,16,17,17,18,18,20,20,27,27,28,28,30,30,34,34],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,16,-1,-1,18,20,-1,22,24,26,-1,28,-1,-1,-1,-1,-1,-1,30,32,-1,34,-1,-1,-1,36,-1,-1],"split_conditions":[1.05381016E5,8.861901E3,1.6846506E5,1.3180642E7,8.379526E-3,4.359929E1,2.84E2,7.805608E4,-9.817784E-3,-1.277596E-3,2.0901188E7,3.7584007E-3,-2.625312E-4,1.6776951E3,2.2744698E2,5.66768E-4,2.9632012E7,1.6225006E4,4.2969458E2,-4.1062804E-3,4.723148E11,2.420301E-3,1.3285341E-2,-1.1120378E-3,-3.9435457E-3,3.108493E-3,-1.0759338E-3,1.7239808E7,1.929271E6,5.345242E-4,6.603817E2,2.294754E-3,-3.1750596E-3,-4.0618945E-3,2.0353708E3,1.3052122E-3,-2.352041E-3],"split_indices":[45,64,40,59,0,68,8,40,0,0,9,0,0,4,64,0,1,40,64,0,43,0,0,0,0,0,0,57,2,0,64,0,0,0,67,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.55E2,2.94E2,1.61E2,2.92E2,2E0,1.21E2,4E1,2.91E2,1E0,3.4E1,8.7E1,3.1E1,9E0,1.33E2,1.58E2,7.1E1,1.6E1,4.9E1,8.4E1,9E0,1.49E2,1.5E1,1E0,2.5E1,2.4E1,6E0,7.8E1,1.16E2,3.3E1,7.2E1,4.4E1,2.9E1,4E0,1.2E1,3.2E1,1.4E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.3349347E-3,-1.1454055E-3,4.8283143E-3,-8.797994E-3,2.1234712E-2,-7.5815814E-3,-9.42976E-3,1.6176079E-3,-2.0999634E-3,-1.8322743E-3,-1.5505636E-3,9.0180174E-4,-6.366056E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[1.3824834E-1,6.9892004E-2,0E0,6.508992E-2,7.459718E-2,5.3086534E-2,0E0,0E0,0E0,0E0,5.5474404E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,10,10],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[6.0538214E9,5.4779566E9,4.8283143E-3,2.47815E5,2.2967976E7,1.3202006E3,-9.42976E-3,1.6176079E-3,-2.0999634E-3,-1.8322743E-3,7.477264E6,9.0180174E-4,-6.366056E-4],"split_indices":[7,5,0,2,41,4,0,0,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,4.06E2,1.4E1,3.03E2,1.03E2,3.02E2,1E0,8.8E1,1.5E1,5.1E1,2.51E2,9.1E1,1.6E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.6614007E-4,-2.7712164E-3,6.070747E-3,-8.236409E-3,2.979659E-2,-5.7226825E-3,-9.349261E-3,5.120406E-2,-5.237731E-4,-2.4833063E-2,4.2195097E-3,-2.297449E-3,6.327826E-2,-2.6930284E-2,5.7729334E-3,5.1385295E-3,1.6513156E-3,9.543727E-3,2.5063292E-3,-3.9867926E-2,-1.1164478E-4,9.439829E-3,-2.6175365E-2,-5.48291E-2,1.1051044E-2,2.5118568E-3,2.1855347E-4,-3.8490375E-4,-6.999857E-2,-1.9224414E-3,-5.4063764E-3,2.4131944E-3,-2.3081077E-3,-8.077932E-3,-1.0985597E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,7,9,-1,11,-1,13,15,-1,17,19,-1,-1,21,-1,-1,23,-1,25,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.21200666E-1,7.55349E-2,0E0,1.6211629E-1,5.3132676E-2,6.8314455E-2,0E0,4.9841367E-2,0E0,3.9314464E-2,5.9844453E-2,0E0,5.076769E-2,3.8891412E-2,0E0,0E0,5.0528094E-2,0E0,0E0,6.1550498E-2,0E0,3.7422933E-2,4.0749084E-2,4.995896E-2,4.2723082E-2,0E0,0E0,0E0,6.244906E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,16,16,19,19,21,21,22,22,23,23,24,24,28,28],"right_children":[2,4,-1,6,8,10,-1,12,-1,14,16,-1,18,20,-1,-1,22,-1,-1,24,-1,26,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0871896E10,6.0784E4,6.070747E-3,2.235E5,1.40368E5,1.7066003E3,-9.349261E-3,1.1804314E0,-5.237731E-4,1.3989E4,5.738E3,-2.297449E-3,9.957E3,2.55E2,5.7729334E-3,5.1385295E-3,2.4722598E7,9.543727E-3,2.5063292E-3,2.613714E6,-1.1164478E-4,1.0106809E1,6.54925E6,1.1160287E7,1E0,2.5118568E-3,2.1855347E-4,-3.8490375E-4,4.3290512E1,-1.9224414E-3,-5.4063764E-3,2.4131944E-3,-2.3081077E-3,-8.077932E-3,-1.0985597E-3],"split_indices":[7,10,0,2,10,4,0,65,0,2,9,0,9,3,0,0,57,0,0,9,0,70,9,57,26,0,0,0,68,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,4.22E2,7E0,3.62E2,6E1,3.58E2,4E0,3.9E1,2.1E1,1.22E2,2.36E2,4E0,3.5E1,1.21E2,1E0,5E0,2.31E2,2E0,3.3E1,7.9E1,4.2E1,1.81E2,5E1,6.1E1,1.8E1,1.9E1,1.62E2,3.6E1,1.4E1,4.8E1,1.3E1,1.1E1,7E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.5935398E-3,-3.1889186E-3,4.9055573E-2,-1.6836694E-3,-7.5105275E-3,9.519404E-3,3.8077895E-2,-4.8786888E-4,1.3653868E-2,4.9860578E-2,-5.152383E-2,4.125789E-2,3.3038415E-3,1.0143504E-3,1.0129096E-1,-8.960509E-3,-1.7283037E-5,8.447715E-4,5.2636988E-3,6.54983E-4,-1.9577527E-3,9.060968E-4,8.296339E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,-1,-1,9,-1,11,13,15,17,19,-1,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4014478E-1,8.742629E-2,8.434464E-2,4.882694E-2,0E0,0E0,6.1690934E-2,0E0,3.8719222E-2,7.396816E-2,4.571184E-2,5.59507E-2,4.215036E-2,0E0,9.386155E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,9,9,10,10,11,11,12,12,14,14],"right_children":[2,4,6,8,-1,-1,10,-1,12,14,16,18,20,-1,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,1.3674345E4,2.909019E3,7.10711E8,-7.5105275E-3,9.519404E-3,1.93E2,-4.8786888E-4,5.473125E3,2.0928232E1,1.379E3,2.5573445E6,2.237815E7,1.0143504E-3,1.69595E5,-8.960509E-3,-1.7283037E-5,8.447715E-4,5.2636988E-3,6.54983E-4,-1.9577527E-3,9.060968E-4,8.296339E-3],"split_indices":[64,4,44,7,0,0,8,0,44,66,0,62,57,0,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.53E2,3.95E2,5.8E1,3.92E2,3E0,3E0,5.5E1,2.57E2,1.35E2,4.9E1,6E0,3.6E1,9.9E1,3.2E1,1.7E1,1E0,5E0,2.7E1,9E0,8.1E1,1.8E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.770981E-3,-6.8052746E-3,2.4614258E-2,-6.1822747E-4,1.1728033E-3,9.7862E-2,1.2499398E-2,2.2103896E-3,8.218987E-3,1.9971963E-2,-7.842133E-2,6.36417E-3,8.0613495E-4,-1.1952125E-2,-8.342836E-3,6.9880723E-3,-2.4730975E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,-1,-1,7,9,-1,-1,11,13,-1,-1,-1,15,-1,-1],"loss_changes":[9.760367E-2,4.982125E-2,1.3099732E-1,0E0,0E0,6.773332E-2,8.921026E-2,0E0,0E0,4.7443036E-2,1.1064582E-1,0E0,0E0,0E0,5.5688884E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,9,9,10,10,14,14],"right_children":[2,4,6,-1,-1,8,10,-1,-1,12,14,-1,-1,-1,16,-1,-1],"split_conditions":[5.837049E6,3.53E2,1.417988E6,-6.1822747E-4,1.1728033E-3,9.256843E3,1.4609149E4,2.2103896E-3,8.218987E-3,1.4E1,1.3936486E8,6.36417E-3,8.0613495E-4,-1.1952125E-2,1.3812E4,6.9880723E-3,-2.4730975E-3],"split_indices":[1,8,41,0,0,4,4,0,0,3,1,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,2.93E2,1.48E2,2.48E2,4.5E1,2E1,1.28E2,1.2E1,8E0,1.19E2,9E0,3E0,1.16E2,2E0,7E0,1E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[4.4245797E-4,-5.1608505E-3,4.6877827E-2,-3.5756961E-3,-6.052757E-3,1.139685E-3,4.452132E-3,-1.1354662E-2,2.466775E-2,-1.0225511E-2,-8.437944E-3,6.399685E-3,9.483774E-4,-8.891559E-3,-6.7183687E-3,-2.942466E-2,-7.439917E-5,-7.2903914E-4,-3.819273E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,-1,-1,15,-1,17,-1,-1,-1],"loss_changes":[1.0874187E-1,6.812514E-2,4.3691523E-2,8.135116E-2,0E0,0E0,0E0,4.9842574E-2,4.4641092E-2,4.668765E-2,0E0,0E0,0E0,4.358127E-2,0E0,5.18796E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,13,13,15,15],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,-1,-1,16,-1,18,-1,-1,-1],"split_conditions":[2.6422684E9,1.0067637E7,2.2876814E1,5.5292703E9,-6.052757E-3,1.139685E-3,4.452132E-3,3.2006908E7,3.0969632E2,5.5051494E9,-8.437944E-3,6.399685E-3,9.483774E-4,1.5031561E3,-6.7183687E-3,4.70436E5,-7.439917E-5,-7.2903914E-4,-3.819273E-3],"split_indices":[7,59,66,5,0,0,0,1,45,5,0,0,0,4,0,41,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.16E2,3.72E2,4.4E1,3.68E2,4E0,2.9E1,1.5E1,2.89E2,7.9E1,2.88E2,1E0,3E0,7.6E1,2.86E2,2E0,7.5E1,2.11E2,5.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.4875304E-3,-3.4001325E-3,5.80789E-3,8.639358E-3,-1.4305864E-2,7.00403E-2,4.117209E-3,-1.7585488E-2,2.0381366E-3,1.642436E-3,7.734491E-3,-1.5564246E-2,7.638626E-4,-7.506525E-4,-5.198058E-3,-2.3548726E-3,5.34564E-3,4.3585626E-3,-3.8588815E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,-1,15,-1,-1,-1,-1,17,-1,-1],"loss_changes":[9.85453E-2,5.6582358E-2,0E0,5.6632794E-2,4.1548207E-2,3.900916E-2,4.2437535E-2,4.5882203E-2,0E0,0E0,0E0,4.643978E-2,0E0,0E0,0E0,0E0,4.6673454E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,11,11,16,16],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,-1,16,-1,-1,-1,-1,18,-1,-1],"split_conditions":[1.0871896E10,1.33E2,5.80789E-3,3.7577084E2,9.2242E4,4.230839E6,6.927258E4,1.99814E5,2.0381366E-3,1.642436E-3,7.734491E-3,5.9376766E1,7.638626E-4,-7.506525E-4,-5.198058E-3,-2.3548726E-3,5.9073865E2,4.3585626E-3,-3.8588815E-4],"split_indices":[7,3,0,45,10,1,45,2,0,0,0,66,0,0,0,0,59,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,4.29E2,6E0,2.04E2,2.25E2,1.3E1,1.91E2,2.13E2,1.2E1,1E1,3E0,6.9E1,1.22E2,2.08E2,5E0,2.7E1,4.2E1,5E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.514629E-3,-7.583127E-3,2.5424462E-2,-1.6222168E-2,6.31014E-4,2.9532779E-2,-8.11846E-3,-1.4181763E-3,2.2991034E-4,3.723783E-2,-6.2971143E-4,1.4901076E-3,4.5423433E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,-1],"loss_changes":[1.0059344E-1,5.2931074E-2,1.074516E-1,5.3787116E-2,0E0,4.2996667E-2,0E0,0E0,0E0,4.148771E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,-1],"split_conditions":[5.4779566E9,1.4171124E3,5.374925E7,9.704588E6,6.31014E-4,1.5953E4,-8.11846E-3,-1.4181763E-3,2.2991034E-4,1.00973885E2,-6.2971143E-4,1.4901076E-3,4.5423433E-3],"split_indices":[5,64,57,9,0,0,0,0,0,68,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.01E2,1.32E2,2.11E2,9E1,1.3E2,2E0,1.33E2,7.8E1,1.1E2,2E1,9.8E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.608586E-4,-5.1743905E-3,4.697952E-2,7.6955543E-3,-1.6441539E-2,5.7546433E-2,-4.4442015E-3,2.6359228E-2,-6.467867E-4,-1.4422469E-2,-5.888559E-3,3.5687543E-2,8.8635655E-3,1.9271031E-3,-1.0102665E-3,-4.2104147E-2,-3.9102763E-4,4.6697054E-3,8.2585687E-4,4.999236E-4,-5.9806727E-2,2.0934251E-4,-4.1614207E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,17,-1,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.20788395E-1,5.7694536E-2,7.692279E-2,7.198298E-2,4.169818E-2,1.16759226E-1,0E0,5.6469478E-2,0E0,3.7844118E-2,0E0,4.382893E-2,0E0,0E0,0E0,3.7495673E-2,0E0,0E0,0E0,0E0,4.5391828E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,11,11,15,15,20,20],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,18,-1,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[4.23168E3,9.362063E6,1.93E2,1.33E2,3.764913E7,3.305542E4,-4.4442015E-3,2.236341E0,-6.467867E-4,1.6858337E3,-5.888559E-3,9.957E3,8.8635655E-3,1.9271031E-3,-1.0102665E-3,3.543E3,-3.9102763E-4,4.6697054E-3,8.2585687E-4,4.999236E-4,8.553271E-5,2.0934251E-4,-4.1614207E-3],"split_indices":[64,57,8,3,60,4,0,53,0,4,0,9,0,0,0,2,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.45E2,3.96E2,4.9E1,1.85E2,2.11E2,4.6E1,3E0,9.7E1,8.8E1,2.08E2,3E0,4E1,6E0,7.7E1,2E1,3.9E1,1.69E2,9E0,3.1E1,1E1,2.9E1,8E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.8960129E-3,-1.7645143E-4,4.2399704E-2,6.449769E-3,3.0174833E-2,4.335445E-5,3.1038662E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":78,"left_children":[1,-1,3,-1,5,-1,-1],"loss_changes":[9.4884634E-2,0E0,5.013389E-2,0E0,4.2995106E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4],"right_children":[2,-1,4,-1,6,-1,-1],"split_conditions":[7.2038723E9,-1.7645143E-4,1.3812E4,6.449769E-3,4.09591E0,4.335445E-5,3.1038662E-3],"split_indices":[12,0,9,0,65,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.3E2,3.8E2,5E1,5E0,4.5E1,2.4E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[9.320372E-5,-8.848887E-3,2.8536681E-2,-1.2093044E-2,1.2837705E-3,3.5204917E-2,-5.590775E-3,-5.0592783E-4,-3.796509E-3,2.8014757E-2,1.3031024E-2,4.9126926E-3,1.079404E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,-1,-1],"loss_changes":[1.1165526E-1,3.77327E-2,1.02245435E-1,3.772823E-2,0E0,1.5648402E-1,0E0,0E0,0E0,4.2523876E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,-1,-1],"split_conditions":[2.1959E4,4.61E2,3.768034E4,4.14E2,1.2837705E-3,3.305542E4,-5.590775E-3,-5.0592783E-4,-3.796509E-3,9.957E3,1.3031024E-2,4.9126926E-3,1.079404E-3],"split_indices":[2,3,4,3,0,4,0,0,0,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.33E2,1.04E2,3.05E2,2.8E1,1E2,4E0,2.97E2,8E0,9.8E1,2E0,7E0,9.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[5.997633E-3,1.3719666E-3,8.272882E-2,3.110966E-3,-4.3831966E-3,8.617526E-3,1.517872E-3,-1.3718063E-4,1.5506065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[1.5605696E-1,6.471389E-2,1.1217858E-1,6.6945665E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[8.10288E3,5.30113E5,1.681178E6,1.6219076E9,-4.3831966E-3,8.617526E-3,1.517872E-3,-1.3718063E-4,1.5506065E-3],"split_indices":[64,2,41,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,4.15E2,2.4E1,4.08E2,7E0,8E0,1.6E1,3.38E2,7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.5147931E-3,-9.452077E-3,1.9095305E-2,-5.821288E-3,-4.4933233E-2,3.4971032E-3,5.0503353E-4,-8.86664E-3,1.8711337E-3,-4.824376E-3,-8.439133E-4,-3.8430656E-4,-8.073147E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[7.117777E-2,4.018935E-2,5.463865E-2,3.801179E-2,4.0339645E-2,0E0,0E0,4.69096E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[5.5015055E9,2.1730006E11,2.064164E6,2.3528348E9,1.0517595E3,3.4971032E-3,5.0503353E-4,2.2860639E9,1.8711337E-3,-4.824376E-3,-8.439133E-4,-3.8430656E-4,-8.073147E-3],"split_indices":[5,43,41,12,64,0,0,12,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.13E2,1.2E2,2.85E2,2.8E1,1.7E1,1.03E2,2.67E2,1.8E1,9E0,1.9E1,2.66E2,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[7.317351E-4,-4.651968E-3,2.5586313E-2,1.4756955E-4,-2.4314823E-2,9.582981E-2,1.0918317E-2,-1.6035828E-4,-4.910184E-2,3.058532E-3,1.2778599E-2,-5.316873E-3,8.205246E-4,-1.661268E-2,-7.683766E-2,-8.272678E-5,-8.263973E-3,-7.1197334E-3,-2.070778E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,-1,7,9,11,-1,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[5.7121944E-2,5.2557357E-2,7.7258885E-2,0E0,5.1111914E-2,5.6320697E-2,4.3038875E-2,0E0,3.9273627E-2,0E0,0E0,0E0,0E0,4.867285E-2,4.985571E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,13,13,14,14],"right_children":[2,4,6,-1,8,10,12,-1,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[3.2784192E3,1.7239808E7,1.064816E6,1.4756955E-4,1.4935554E11,4.916536E7,1.379E3,-1.6035828E-4,2.4E1,3.058532E-3,1.2778599E-2,-5.316873E-3,8.205246E-4,1.1367206E8,7.121916E2,-8.272678E-5,-8.263973E-3,-7.1197334E-3,-2.070778E-3],"split_indices":[64,57,41,0,43,1,0,0,8,0,0,0,0,1,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,3.5E2,7.5E1,2.53E2,9.7E1,1.2E1,6.3E1,5.3E1,4.4E1,1.1E1,1E0,2E0,6.1E1,2.1E1,2.3E1,2E1,1E0,7E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[4.5351475E-3,-2.2029572E-3,7.411274E-3,6.1844926E-2,4.542152E-3,4.6532806E-3,-2.9977407E-5,6.77689E-3,-4.965478E-2,4.9521375E-5,2.7254438E-2,-4.780898E-5,-7.826223E-3,4.0257527E-3,7.801361E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,-1,3,5,7,-1,-1,9,11,-1,13,-1,-1,-1,-1],"loss_changes":[6.254835E-2,0E0,6.546952E-2,4.1055188E-2,4.8903994E-2,0E0,0E0,4.5707382E-2,8.317824E-2,0E0,5.149597E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,7,7,8,8,10,10],"right_children":[2,-1,4,6,8,-1,-1,10,12,-1,14,-1,-1,-1,-1],"split_conditions":[9.63428E2,-2.2029572E-3,1.0097E4,2.8184534E7,4.5859156E7,4.6532806E-3,-2.9977407E-5,2.940448E6,2.003E3,4.9521375E-5,8.169258E3,-4.780898E-5,-7.826223E-3,4.0257527E-3,7.801361E-4],"split_indices":[4,0,9,57,57,0,0,41,0,0,45,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.44E2,2.4E1,4.2E2,2E1,4E2,1.3E1,7E0,3.85E2,1.5E1,3.01E2,8.4E1,1.1E1,4E0,1.4E1,7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.6320193E-3,-4.3623816E-4,1.968047E-2,3.3609013E-3,1.5325875E-2,6.047856E-4,9.064974E-2,-3.8844817E-3,9.385133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,-1,3,-1,5,-1,7,-1,-1],"loss_changes":[8.9767E-2,0E0,4.9098887E-2,0E0,5.306354E-2,0E0,1.6180554E-1,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,6,6],"right_children":[2,-1,4,-1,6,-1,8,-1,-1],"split_conditions":[6.97E3,-4.3623816E-4,6.798766E-4,3.3609013E-3,3.305542E4,6.047856E-4,1.5435187E2,-3.8844817E-3,9.385133E-3],"split_indices":[2,0,51,0,4,0,70,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,2.05E2,2.41E2,1.9E1,2.22E2,2.14E2,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.5456805E-3,-4.2547244E-3,2.5850266E-2,-5.9326465E-4,5.8211706E-4,8.0401026E-2,3.7743204E-4,6.585361E-4,6.293344E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[6.439524E-2,4.4830967E-2,8.72318E-2,0E0,0E0,6.5670446E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.8163872E7,6.211238E5,2.1207E4,-5.9326465E-4,5.8211706E-4,2.565595E0,3.7743204E-4,6.585361E-4,6.293344E-3],"split_indices":[1,59,9,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.55E2,3.68E2,8.7E1,2.49E2,1.19E2,2.1E1,6.6E1,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.4880665E-3,-3.502576E-4,3.5587521E-3,-2.2103162E-3,1.7347862E-3,5.5235E-4,-8.830971E-3,-1.7932688E-3,1.457947E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[5.488164E-2,3.8317848E-2,0E0,0E0,3.947618E-2,0E0,5.9852753E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[1.6941456E1,2.849E3,3.5587521E-3,-2.2103162E-3,1.8887723E-2,5.5235E-4,1.156185E3,-1.7932688E-3,1.457947E-4],"split_indices":[47,2,0,0,69,0,64,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.27E2,4.17E2,1E1,1.8E1,3.99E2,2.12E2,1.87E2,5.6E1,1.31E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-5.086955E-4,-9.392726E-3,2.560235E-2,-1.2143699E-4,-1.4153268E-3,1.8314151E-2,9.9568196E-2,6.176781E-3,6.4573664E-4,1.1929295E-2,3.4947716E-2,4.51772E-3,-2.1540504E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,-1,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[1.0416363E-1,4.4080127E-2,5.9631355E-2,0E0,0E0,5.802239E-2,8.1565656E-2,0E0,0E0,0E0,3.8518928E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,10,10],"right_children":[2,4,6,-1,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[2.143564E3,1.8071064E7,3.305542E4,-1.2143699E-4,-1.4153268E-3,8.534E3,3.768034E4,6.176781E-3,6.4573664E-4,1.1929295E-2,2.1358348E7,4.51772E-3,-2.1540504E-3],"split_indices":[64,57,4,0,0,9,4,0,0,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.34E2,1.13E2,2.45E2,8.9E1,1.04E2,9E0,4E0,1E2,2E0,7E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.640311E-5,-2.8585454E-3,2.3645193E-3,-3.8125531E-3,-1.4292703E-3,-3.9724482E-4,1.1720464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[6.1631534E-2,4.397007E-2,0E0,0E0,6.7180224E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[7.0484486E0,2.1066585E2,2.3645193E-3,-3.8125531E-3,1.5271514E5,-3.9724482E-4,1.1720464E-3],"split_indices":[47,64,0,0,40,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.44E2,4.19E2,2.5E1,7E0,4.12E2,3.27E2,8.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-2.8323252E-3,-1.5177465E-3,-4.2039575E-3,-3.9357827E-3,3.8295478E-2,2.5520467E-3,-1.8138263E-2,5.1070424E-3,6.3669233E-4,-7.589453E-4,8.236626E-4,-6.1916307E-2,-4.2772095E-4,-6.468739E-3,-1.5228546E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,-1,-1,-1],"loss_changes":[4.6996817E-2,4.202809E-2,0E0,3.7917174E-2,3.9519414E-2,7.017312E-2,5.357046E-2,0E0,0E0,0E0,0E0,4.4888385E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,-1,-1,-1],"split_conditions":[4.8537783E3,9.430977E3,-4.2039575E-3,1.7239808E7,3.2946018E6,7.796878E4,1.960369E7,5.1070424E-3,6.3669233E-4,-7.589453E-4,8.236626E-4,1.2309012E1,-4.2772095E-4,-6.468739E-3,-1.5228546E-3],"split_indices":[69,4,0,57,59,40,57,0,0,0,0,70,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,4.34E2,6E0,4.1E2,2.4E1,2.82E2,1.28E2,6E0,1.8E1,1.24E2,1.58E2,2.2E1,1.06E2,6E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.2353722E-3,-3.2569517E-4,1.4113727E-2,-3.2355892E-4,2.7614553E-2,2.4603676E-2,1.4085202E-1,3.5940714E-2,-2.3008192E-4,1.1906386E-2,-1.3475377E-3,2.058871E-3,-2.9763326E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,-1,3,-1,5,7,9,11,-1,-1,-1,-1,-1],"loss_changes":[4.6028942E-2,0E0,5.7159565E-2,0E0,3.8811587E-2,4.0562935E-2,5.5454392E-2,4.662647E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,6,6,7,7],"right_children":[2,-1,4,-1,6,8,10,12,-1,-1,-1,-1,-1],"split_conditions":[1.0975164E3,-3.2569517E-4,7.959718E4,-3.2355892E-4,1.5324995E4,1.2730462E7,1.492E3,1.929271E6,-2.3008192E-4,1.1906386E-2,-1.3475377E-3,2.058871E-3,-2.9763326E-3],"split_indices":[64,0,40,0,64,9,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,2.28E2,2.04E2,8.1E1,1.23E2,1.21E2,2E0,8.7E1,3.4E1,1E0,1E0,8.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.0338613E-3,-1.44663155E-2,6.743359E-3,-6.092748E-4,-5.0124833E-3,3.3679686E-3,9.95236E-3,5.6080264E-3,-6.2268856E-3,-7.425854E-3,6.9477805E-3,3.0603059E-2,-6.04302E-6,9.2832965E-4,4.719382E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,-1,-1,7,-1,9,-1,-1,11,13,-1,-1,-1],"loss_changes":[4.827371E-2,3.7635047E-2,1.5064463E-1,0E0,0E0,6.662816E-2,0E0,4.7927126E-2,0E0,0E0,3.7793618E-2,3.816717E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,10,10,11,11],"right_children":[2,4,6,-1,-1,8,-1,10,-1,-1,12,14,-1,-1,-1],"split_conditions":[8.361085E4,4.42384E5,3.305542E4,-6.092748E-4,-5.0124833E-3,4.352718E7,9.95236E-3,1.44E2,-6.2268856E-3,-7.425854E-3,1.2452798E3,6.840166E6,-6.04302E-6,9.2832965E-4,4.719382E-3],"split_indices":[40,2,4,0,0,57,0,0,0,0,45,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,1.98E2,2.32E2,1.94E2,4E0,2.29E2,3E0,2.26E2,3E0,1E0,2.25E2,5.1E1,1.74E2,4.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[3.6258632E-3,-2.8895526E-3,2.8265058E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":92,"left_children":[1,-1,-1],"loss_changes":[5.4581713E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[6.744371E2,-2.8895526E-3,2.8265058E-4],"split_indices":[67,0,0],"split_type":[0,0,0],"sum_hessian":[4.34E2,1.3E1,4.21E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-2.3945232E-3,-1.2231922E-2,9.752088E-3,-5.041748E-4,-9.0135485E-3,-3.2860443E-2,1.6713258E-2,-2.6360992E-3,1.7159383E-3,4.002852E-3,1.2672672E-2,4.163746E-3,2.2764516E-3,2.1918885E-2,-1.42113E-2,7.079407E-4,8.996505E-3,-4.786922E-4,-7.6484005E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,-1,-1,7,9,-1,-1,-1,11,13,-1,15,17,-1,-1,-1,-1],"loss_changes":[5.257586E-2,8.575614E-2,5.898592E-2,0E0,0E0,3.92276E-2,4.2522103E-2,0E0,0E0,0E0,4.4851333E-2,4.2397942E-2,0E0,7.835111E-2,3.969773E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,10,10,11,11,13,13,14,14],"right_children":[2,4,6,-1,-1,8,10,-1,-1,-1,12,14,-1,16,18,-1,-1,-1,-1],"split_conditions":[5.3257984E1,4.352718E7,6.944307E7,-5.041748E-4,-9.0135485E-3,9.309699E8,4.9832974E2,-2.6360992E-3,1.7159383E-3,4.002852E-3,2.3781708E5,7.555878E1,2.2764516E-3,6.204522E3,2.3767492E4,7.079407E-4,8.996505E-3,-4.786922E-4,-7.6484005E-3],"split_indices":[68,57,7,0,0,5,64,0,0,0,45,68,0,64,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,2.42E2,1.96E2,2.4E2,2E0,2.7E1,1.69E2,2.1E1,6E0,9E0,1.6E2,1.28E2,3.2E1,6.5E1,6.3E1,6.3E1,2E0,6.2E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-2.8932016E-4,-4.2714477E-3,2.2612168E-2,-1.6022487E-4,-7.0464606E-3,4.7503058E-2,-5.247355E-4,6.8740016E-3,1.5100779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[4.186465E-2,5.650465E-2,5.654332E-2,0E0,0E0,5.6550294E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[3.33069E3,1.1770536E8,1E0,-1.6022487E-4,-7.0464606E-3,2.3528348E9,-5.247355E-4,6.8740016E-3,1.5100779E-3],"split_indices":[64,1,24,0,0,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.57E2,3.9E2,6.7E1,3.88E2,2E0,3.8E1,2.9E1,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.7021253E-4,-3.6585496E-3,2.655077E-2,-1.2234553E-2,5.130406E-4,-3.9796607E-4,2.488876E-3,5.8548115E-3,-2.2336284E-2,1.3355107E-2,-3.3986196E-3,-7.438595E-3,-9.814933E-4,-2.9382936E-4,1.9871562E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[4.383225E-2,4.538373E-2,4.453231E-2,4.3073244E-2,0E0,0E0,0E0,4.799565E-2,4.8596814E-2,3.9852034E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[2.3624256E5,1.0368186E-1,4.95E9,1.0878975E4,5.130406E-4,-3.9796607E-4,2.488876E-3,3.9034148E7,1.3436805E4,2.613714E6,-3.3986196E-3,-7.438595E-3,-9.814933E-4,-2.9382936E-4,1.9871562E-3],"split_indices":[40,50,5,60,0,0,0,57,62,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,3.78E2,5.4E1,2.34E2,1.44E2,2.2E1,3.2E1,8.4E1,1.5E2,7.7E1,7E0,2E0,1.48E2,4.5E1,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[4.3620815E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0],"id":96,"left_children":[-1],"loss_changes":[0E0],"parents":[2147483647],"right_children":[-1],"split_conditions":[2.1799975E-5],"split_indices":[0],"split_type":[0],"sum_hessian":[4.39E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"1","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortAggregate","sqlOp_SortMergeJoin","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_dataproc","platform_databricks-aws","platform_databricks-azure","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"91"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[7.4319993E-3,-2.986165E-1,5.363805E-1,-6.081042E-1,-1.9074835E-1,6.194652E-1,-5.773755E-1,-6.254226E-1,-1.4547175E-3,-2.2249421E-1,3.1191745E-1,3.4459683E-1,9.4478184E-1,-9.608562E-1,-9.766551E-2,-3.1628206E-2,-1.7476493E-2,-2.7656183E-1,-2.7171573E-2,1.7639505E-2,1.3591931E-3,2.9292017E-1,5.016829E-2,7.9174286E-1,1.3544148E0,-5.4351926E-2,-2.0187896E-2,-1.0362057E-2,4.398011E-3,-3.0325854E-1,7.379495E-2,-8.177539E-2,9.6727066E-2,3.1552166E-2,3.5883486E-1,8.375097E-1,-9.121704E-3,5.445212E-1,7.2296254E-2,-3.3149186E-1,-1.4445761E-1,-3.2934556E-3,1.2438638E-2,-1.5279225E-1,-2.2658065E-2,1.1858878E-2,-6.631761E-2,1.5722747E-1,-2.76379E-1,4.087299E-1,1.69263E-1,8.51175E-1,5.03626E-3,3.876356E-2,-5.6361635E-3,-1.9878548E-1,-3.7923712E-1,-2.6507935E-1,5.9227068E-2,-8.643E-3,-2.3274795E-4,-1.0098926E-1,2.038217E-3,1.5455564E-4,-7.7185766E-3,2.5900844E-1,-5.3561404E-3,-1.590587E-2,-1.5029252E-3,4.6767187E-1,2.689842E-1,-1.554434E-3,1.2202856E-2,1.4234993E-2,4.193508E-2,-1.1583127E-2,-4.029607E-3,-1.9122614E-2,-8.1494795E-3,-7.288551E-4,-1.36520695E-2,-4.7538094E-3,4.401101E-3,-7.345976E-3,1.7071784E-3,1.4131854E-2,-3.299358E-4,1.7592939E-2,2.6663076E-2,-1.1360114E-3,1.506551E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,-1,35,37,-1,-1,-1,-1,39,41,43,45,47,49,51,-1,53,-1,55,57,-1,-1,59,61,-1,63,65,67,69,71,73,-1,-1,-1,75,77,79,81,-1,-1,83,-1,-1,-1,85,-1,-1,-1,87,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.669446E1,8.597195E0,1.4330605E1,6.5172577E-1,3.1811247E0,1.2282547E1,1.9297128E0,2.843647E-1,0E0,1.9356413E0,1.7329252E-1,2.5002213E0,2.9397964E0,6.932497E-2,1.5247346E-1,0E0,0E0,1.3692656E0,2.8528878E-1,0E0,0E0,1.2635498E0,0E0,2.3582478E0,1.062643E0,0E0,0E0,0E0,0E0,5.528984E-1,3.0552053E-1,1.182895E-1,3.3260876E-1,6.626438E-1,4.8887968E-1,3.8191223E-1,0E0,7.803595E-1,0E0,6.405926E-1,5.562135E-1,0E0,0E0,5.0039202E-2,9.073286E-2,0E0,4.6145454E-2,3.5648343E-1,5.5321872E-2,2.5310087E-1,2.4463114E-1,2.178688E-1,0E0,0E0,0E0,1.3273513E-1,2.3232746E-1,5.8938444E-2,5.4525863E-2,0E0,0E0,6.144408E-2,0E0,0E0,0E0,8.3563745E-2,0E0,0E0,0E0,7.011032E-2,2.1085656E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,21,21,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,55,55,56,56,57,57,58,58,61,61,65,65,69,69,70,70],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,-1,36,38,-1,-1,-1,-1,40,42,44,46,48,50,52,-1,54,-1,56,58,-1,-1,60,62,-1,64,66,68,70,72,74,-1,-1,-1,76,78,80,82,-1,-1,84,-1,-1,-1,86,-1,-1,-1,88,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,1.0429407E3,1.2730462E7,1.2678202E9,6.0891E4,1.700398E4,2.9778808E7,-1.4547175E-3,9.7467445E4,1.5E2,7.1853364E3,8.4987135E0,3.1E1,1.6130411E5,-3.1628206E-2,-1.7476493E-2,2.9063344E5,1.207E3,1.7639505E-2,1.3591931E-3,2.3890688E3,5.016829E-2,1.4656219E8,6.069097E4,-5.4351926E-2,-2.0187896E-2,-1.0362057E-2,4.398011E-3,1.2120859E3,4.037E3,4.6976192E8,2.021977E6,1.3508742E7,2.47E2,7.2125336E2,-9.121704E-3,1.207E3,7.2296254E-2,7.37365E6,8.459549E4,-3.2934556E-3,1.2438638E-2,1.07E3,1.5219347E-1,1.1858878E-2,1.475931E9,2.9743198E1,1.2587E4,6.0796064E8,1.3428E4,3.0811954E-2,5.03626E-3,3.876356E-2,-5.6361635E-3,7.164074E8,6.4065235E8,1.7249636E-7,4.66E2,-8.643E-3,-2.3274795E-4,3.01E2,2.038217E-3,1.5455564E-4,-7.7185766E-3,1.14E2,-5.3561404E-3,-1.590587E-2,-1.5029252E-3,2.0663544E9,6.168645E8,-1.554434E-3,1.2202856E-2,1.4234993E-2,4.193508E-2,-1.1583127E-2,-4.029607E-3,-1.9122614E-2,-8.1494795E-3,-7.288551E-4,-1.36520695E-2,-4.7538094E-3,4.401101E-3,-7.345976E-3,1.7071784E-3,1.4131854E-2,-3.299358E-4,1.7592939E-2,2.6663076E-2,-1.1360114E-3,1.506551E-2],"split_indices":[2,1,67,9,7,2,42,12,0,42,3,4,62,3,37,0,0,37,0,0,0,4,0,1,37,0,0,0,0,61,10,7,38,54,3,67,0,0,0,54,37,0,0,0,47,0,12,62,2,41,2,47,0,0,0,7,7,47,0,0,0,8,0,0,0,3,0,0,0,12,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.1E2,2.6E2,1.5E2,6.6E1,1.94E2,1.4E2,1E1,6.4E1,2E0,1.83E2,1.1E1,7.7E1,6.3E1,5E0,5E0,5.6E1,8E0,1.43E2,4E1,9E0,2E0,7.3E1,4E0,4.8E1,1.5E1,3E0,2E0,3E0,2E0,1.33E2,1E1,2.8E1,1.2E1,1.5E1,5.8E1,4.6E1,2E0,3E0,1.2E1,1.12E2,2.1E1,6E0,4E0,1.2E1,1.6E1,6E0,6E0,1.1E1,4E0,4.5E1,1.3E1,4.5E1,1E0,2E0,1E0,3.1E1,8.1E1,1.3E1,8E0,1E1,2E0,7E0,9E0,4E0,2E0,8E0,3E0,3E0,1E0,3E1,1.5E1,4E0,9E0,2E0,4.3E1,2.2E1,9E0,7.4E1,7E0,1E0,1.2E1,1E0,7E0,5E0,2E0,7E0,1E0,1.6E1,1.4E1,2E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.7302966E-2,-2.8948772E-1,5.851634E-1,-5.9922904E-1,-1.8633845E-1,4.5383883E-1,1.0865788E0,-6.366671E-1,-3.1842166E-1,-2.2112006E-1,3.0894497E-1,5.0208855E-1,-5.105406E-2,-2.7988553E-2,1.2114563E0,-3.1228371E-2,-1.03782425E-2,-1.7938454E-2,-3.3407633E-3,-2.8451058E-1,-3.2865163E-2,2.0680368E-2,7.750416E-2,3.3640644E-1,6.860864E-1,-8.828496E-3,1.0543756E-2,6.0285133E-1,1.3943254E0,-3.0053923E-1,7.441699E-2,-1.2950003E-1,1.4612462E-1,7.0575345E-3,-6.433035E-3,-2.7355435E-3,3.6402175E-1,4.909799E-1,3.7917692E-2,3.4790974E-2,-5.5002873E-3,6.876919E-2,1.897953E-2,-3.2629547E-1,-1.4873125E-1,1.065684E-2,-1.3707876E-3,-2.0893127E-1,-6.742317E-2,1.4010787E-2,4.9679447E-2,-2.2700206E-4,3.7704223E-1,2.6317466E-2,1.2143095E-1,-3.540253E-1,-2.1972358E-1,-1.8510579E-1,2.9522902E-3,-1.1822998E-2,-2.7451029E-3,4.2446833E-3,-8.958712E-2,-8.456569E-4,7.1018673E-3,3.879621E-1,2.4669014E-3,-6.3722404E-3,1.20546995E-2,-1.0592453E-2,-1.872176E-2,2.9536393E-3,-1.5483591E-2,-3.041635E-3,-1.0291179E-2,-5.8088214E-3,-3.7279562E-4,1.9138958E-2,4.820802E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,-1,33,35,37,-1,-1,39,41,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,57,-1,-1,59,61,-1,63,-1,65,-1,67,69,71,73,-1,-1,-1,-1,75,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.351362E1,8.642527E0,9.252396E0,5.121479E-1,3.6342506E0,9.217152E0,4.2094345E0,1.3435555E-1,1.0781407E-1,2.3057623E0,3.5547018E-1,3.3054733E0,0E0,1.9308713E-1,2.1643944E0,0E0,0E0,0E0,0E0,8.565693E-1,8.837559E-1,0E0,1.0668725E-1,7.046261E-1,6.709175E-1,0E0,0E0,7.6319265E-1,7.801056E-2,4.9495983E-1,1.1183003E-1,1.4863098E-1,2.363478E-1,0E0,0E0,0E0,2.8503704E-1,3.593483E-1,0E0,0E0,0E0,0E0,0E0,2.7092743E-1,1.7935398E-1,0E0,0E0,6.3238144E-2,7.679807E-2,0E0,8.1320494E-2,0E0,1.8298817E-1,0E0,1.6347332E-1,3.3817768E-1,7.8899586E-1,5.2224815E-2,0E0,0E0,0E0,0E0,4.4578865E-2,0E0,0E0,1.02555275E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,19,19,20,20,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,43,43,44,44,47,47,48,48,50,50,52,52,54,54,55,55,56,56,57,57,62,62,65,65],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,-1,34,36,38,-1,-1,40,42,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,58,-1,-1,60,62,-1,64,-1,66,-1,68,70,72,74,-1,-1,-1,-1,76,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.6512886E3,3.5002026E8,1.4096699E9,4.673502E7,5.0314346E1,2.9778808E7,4.48149E5,2.4242829E-1,7.12E2,4.1331047E-1,-5.105406E-2,3.677E3,6.069097E4,-3.1228371E-2,-1.03782425E-2,-1.7938454E-2,-3.3407633E-3,3.5921E4,7.788E3,2.0680368E-2,2.08E2,1E0,1E0,-8.828496E-3,1.0543756E-2,3.14E2,2.3322949E2,6.4065235E8,4.185E3,7.860814E-1,4.4329906E1,7.0575345E-3,-6.433035E-3,-2.7355435E-3,2.56E2,7.51881E5,3.7917692E-2,3.4790974E-2,-5.5002873E-3,6.876919E-2,1.897953E-2,1.1496869E3,3.6340196E-2,1.065684E-2,-1.3707876E-3,3.01E2,1.1433524E7,1.4010787E-2,1.7684188E-1,-2.2700206E-4,4.2663252E7,2.6317466E-2,3.087E3,7.517043E6,3.8499493E1,8.110162E0,2.9522902E-3,-1.1822998E-2,-2.7451029E-3,4.2446833E-3,4.0246515E8,-8.456569E-4,7.1018673E-3,6.2185767E10,2.4669014E-3,-6.3722404E-3,1.20546995E-2,-1.0592453E-2,-1.872176E-2,2.9536393E-3,-1.5483591E-2,-3.041635E-3,-1.0291179E-2,-5.8088214E-3,-3.7279562E-4,1.9138958E-2,4.820802E-3],"split_indices":[2,1,61,7,7,57,65,12,1,48,0,44,0,0,37,0,0,0,0,10,10,0,3,8,109,0,0,3,66,7,2,48,67,0,0,0,0,9,0,0,0,0,0,61,47,0,0,8,41,0,47,0,54,0,0,54,65,67,0,0,0,0,41,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,2.73E2,1.47E2,6.7E1,2.06E2,1.18E2,2.9E1,5.8E1,9E0,1.93E2,1.3E1,1.15E2,3E0,3E0,2.6E1,5.6E1,2E0,7E0,2E0,1.44E2,4.9E1,8E0,5E0,6.2E1,5.3E1,2E0,1E0,7E0,1.9E1,1.38E2,6E0,3.2E1,1.7E1,4E0,1E0,4E0,5.8E1,2E1,3.3E1,6E0,1E0,1.8E1,1E0,1.17E2,2.1E1,2E0,4E0,1.3E1,1.9E1,6E0,1.1E1,2E0,5.6E1,1.7E1,3E0,9.1E1,2.6E1,1.8E1,3E0,1E1,3E0,2E0,1.7E1,7E0,4E0,5.4E1,2E0,1E0,2E0,2E1,7.1E1,7E0,1.9E1,4E0,1.4E1,1.2E1,5E0,5.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-1.8002357E-3,-2.310228E-1,5.548974E-1,-5.460405E-1,-1.22715384E-1,6.458128E-1,-5.643019E-1,-5.734775E-1,-1.4833343E-3,-1.7458607E-1,2.718203E-1,5.311061E-1,1.1448956E0,8.7507755E-2,-1.0134263E0,-2.8979078E-2,-1.6251488E-2,-2.2213458E-1,7.709214E-2,2.014936E-2,1.452729E-1,3.7821496E-1,6.5089846E-1,7.235545E-2,4.3517187E-2,-9.447308E-3,1.647942E-2,-5.3458676E-2,-1.2943152E-2,-2.3266001E-1,3.7781738E-2,1.870301E-1,-5.140632E-2,1.7538875E-2,1.4802132E-2,4.324233E-1,7.0030484E-3,6.8684846E-1,-8.544712E-3,-9.3470685E-2,-2.9841602E-1,-1.7534051E-3,2.53455E-1,-1.679295E-1,2.790817E-3,-6.0114484E-2,5.169476E-3,2.1389024E-2,9.864364E-4,7.77396E-1,3.6237118E-1,-1.1965176E-1,1.3345984E-2,-3.5280895E-1,-1.9163835E-1,-8.444915E-4,1.446845E-2,8.768821E-4,-9.504109E-3,8.4381155E-4,-6.922032E-3,-5.3676856E-3,8.037131E-1,1.9376297E-2,4.6658413E-3,-8.147365E-3,-1.0845024E-3,-1.7731138E-2,-3.3829643E-3,2.6243676E-3,-1.3677773E-2,6.3523054E-2,3.396917E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,33,35,37,-1,-1,-1,-1,-1,-1,39,-1,41,43,45,-1,47,-1,49,-1,51,53,-1,55,57,-1,59,-1,-1,-1,61,63,65,-1,67,69,-1,-1,-1,-1,-1,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5127098E1,1.0373329E1,1.3185234E1,1.0850945E0,4.733903E0,6.0899734E0,3.016119E0,2.794056E-1,0E0,2.4464111E0,4.3840754E-1,1.5303936E0,4.5615005E-1,4.2733502E-1,1.2839031E-1,0E0,0E0,1.9924726E0,4.758008E-1,0E0,3.2513788E-1,5.0549793E-1,1.6995735E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5280371E0,0E0,2.7631992E-1,2.1347335E-1,7.608181E-2,0E0,1.5681887E-1,0E0,1.2167053E0,0E0,5.759823E-1,6.003027E-1,0E0,1.8211693E-1,4.704085E-2,0E0,4.1373774E-2,0E0,0E0,0E0,1.0016308E0,9.747696E-2,2.5019467E-1,0E0,2.9080772E-1,9.440037E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0083008E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,20,20,21,21,22,22,29,29,31,31,32,32,33,33,35,35,37,37,39,39,40,40,42,42,43,43,45,45,49,49,50,50,51,51,53,53,54,54,62,62],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,34,36,38,-1,-1,-1,-1,-1,-1,40,-1,42,44,46,-1,48,-1,50,-1,52,54,-1,56,58,-1,60,-1,-1,-1,62,64,66,-1,68,70,-1,-1,-1,-1,-1,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3664E4,7.41718E5,1.0429407E3,1.2852292E7,1.4777102E9,4.026531E9,2.909019E3,2.9778808E7,-1.4833343E-3,5.9170656E8,3.79634E5,1.3590209E7,2.231E3,2.5212732E4,3.1E1,-2.8979078E-2,-1.6251488E-2,5.2191963E3,1.4674221E6,2.014936E-2,1.2347272E3,2.2586242E5,1.4656219E8,7.235545E-2,4.3517187E-2,-9.447308E-3,1.647942E-2,-5.3458676E-2,-1.2943152E-2,7.556039E6,3.7781738E-2,5.0363316E5,1.685028E6,2.6490103E1,1.4802132E-2,3.28496E5,7.0030484E-3,1.227647E12,-8.544712E-3,3.7069206E5,1.0517595E3,-1.7534051E-3,4.4626824E4,6.691311E8,2.790817E-3,1E0,5.169476E-3,2.1389024E-2,9.864364E-4,8.713209E10,6.2185767E10,7.164074E8,1.3345984E-2,2.5645154E3,4.98884E1,-8.444915E-4,1.446845E-2,8.768821E-4,-9.504109E-3,8.4381155E-4,-6.922032E-3,-5.3676856E-3,2.3222E4,1.9376297E-2,4.6658413E-3,-8.147365E-3,-1.0845024E-3,-1.7731138E-2,-3.3829643E-3,2.6243676E-3,-1.3677773E-2,6.3523054E-2,3.396917E-2],"split_indices":[2,1,67,9,7,7,41,12,0,12,38,1,0,4,3,0,0,4,56,0,61,42,1,0,0,0,0,0,0,54,0,56,9,67,0,11,0,40,0,37,61,0,37,5,0,23,0,0,0,40,5,7,0,64,65,0,0,0,0,0,0,0,10,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,3.05E2,1.25E2,7.7E1,2.28E2,1.16E2,9E0,7.3E1,4E0,2.02E2,2.6E1,9.6E1,2E1,4E0,5E0,6.4E1,9E0,1.7E2,3.2E1,1.1E1,1.5E1,4.4E1,5.2E1,6E0,1.4E1,2E0,2E0,4E0,1E0,1.69E2,1E0,1.7E1,1.5E1,9E0,6E0,3.5E1,9E0,5E1,2E0,5.5E1,1.14E2,4E0,1.3E1,7E0,8E0,5E0,4E0,3.4E1,1E0,3.8E1,1.2E1,5.2E1,3E0,7.4E1,4E1,2E0,1.1E1,1E0,6E0,3E0,2E0,1E0,3.7E1,1E1,2E0,3.4E1,1.8E1,7E1,4E0,1.1E1,2.9E1,4E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[1.4289183E-2,-2.6126218E-1,5.217898E-1,-5.3903335E-1,-1.6962315E-1,3.7970018E-1,9.690869E-1,-5.541783E-1,-1.0182787E-3,-1.9945547E-1,3.0238333E-1,4.5152277E-1,-2.296129E-1,8.651437E-2,1.0466992E0,-2.7983269E-2,-1.3907298E-2,-2.5768682E-1,-1.4748883E-2,2.2413587E-2,6.881806E-3,2.8877404E-1,6.5398794E-1,-3.431716E-2,-1.3801095E-2,1.0808296E-2,-7.869704E-3,5.784021E-1,1.173418E0,-1.468106E-1,-2.9335776E-1,-5.4195415E-2,1.10738715E-2,3.4956577E-1,1.1444409E-1,5.025463E-1,7.610971E-1,1.9653235E-2,-1.7010468E-1,3.2683786E-2,-5.2382816E-3,5.8489166E-2,5.347906E-3,-2.0133136E-1,-7.361278E-3,-3.3380723E-1,-1.6328833E-1,-2.2253737E-1,-1.5280664E-2,6.4926483E-3,3.6769012E-1,-1.0864891E-2,2.0335019E-1,5.3875685E-1,4.5143375E-3,9.549387E-3,3.7430253E-2,-1.2066416E-2,2.26149E-3,-5.032496E-3,-2.3489554E-1,-3.2062712E-3,6.1493185E-3,-3.52791E-1,-1.5425149E-1,4.2837304E-3,-2.4755089E-1,-1.9528551E-3,-1.3330031E-2,-5.4359328E-2,6.74066E-2,4.456909E-1,2.758946E-1,1.5074061E-3,1.29729E-2,9.00519E-3,2.7362086E-2,-1.2126731E-2,-2.744309E-3,-8.277583E-3,-1.7765148E-2,-8.697831E-3,1.9533562E-3,-6.6926107E-3,-1.5051042E-2,4.778148E-3,-4.257874E-3,-1.704043E-3,8.488193E-3,2.2171825E-2,4.2616148E-3,-4.634405E-3,1.4431078E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,35,-1,37,-1,-1,39,41,43,45,47,-1,49,51,53,55,-1,57,-1,-1,-1,-1,59,61,63,65,67,69,-1,71,-1,73,75,-1,-1,-1,-1,-1,-1,77,-1,-1,79,81,-1,83,-1,-1,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.040844E1,7.035877E0,9.292591E0,5.295181E-1,3.0454931E0,5.223881E0,2.375389E0,2.990036E-1,0E0,2.1473422E0,2.7086997E-1,3.2709942E0,1.3428085E0,1.7409039E-1,1.2795792E0,0E0,0E0,5.5810165E-1,4.8488444E-1,0E0,0E0,5.9586334E-1,3.6366844E-1,0E0,7.28015E-1,0E0,0E0,6.8865013E-1,9.1864395E-1,2.9503042E-1,5.4527473E-1,2.7829188E-1,0E0,1.21870995E-1,5.593709E-1,2.7531528E-1,6.0316086E-2,0E0,1.5088919E-1,0E0,0E0,0E0,0E0,6.6603065E-2,1.04210675E-1,2.3872471E-1,6.3810945E-1,6.733683E-2,1.1999126E-1,0E0,1.6344118E-1,0E0,1.4989334E-1,9.6291065E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.4821143E-2,0E0,0E0,1.5511894E-1,5.8223218E-2,0E0,1.1124027E-1,0E0,0E0,1.3899577E-1,1.4017154E-1,6.049919E-2,2.1162581E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,21,21,22,22,24,24,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,38,38,43,43,44,44,45,45,46,46,47,47,48,48,50,50,52,52,53,53,60,60,63,63,64,64,66,66,69,69,70,70,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,36,-1,38,-1,-1,40,42,44,46,48,-1,50,52,54,56,-1,58,-1,-1,-1,-1,60,62,64,66,68,70,-1,72,-1,74,76,-1,-1,-1,-1,-1,-1,78,-1,-1,80,82,-1,84,-1,-1,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.0495757E3,1.2730462E7,1.4096699E9,3.5301748E7,5.3140095E1,2.9778808E7,-1.0182787E-3,9.7467445E4,6.088569E6,2.7371603E-1,2.2782404E0,1.0014992E8,6.069097E4,-2.7983269E-2,-1.3907298E-2,7.556039E6,1.4998456E5,2.2413587E-2,6.881806E-3,1.7458724E7,1.2384782E5,-3.431716E-2,1.1E1,1.0808296E-2,-7.869704E-3,2.96E2,1.1986E4,7.192302E2,1.1883916E3,1.6359736E3,1.10738715E-2,2.1E1,1.2584659E5,1.4485321E9,2.4768E4,1.9653235E-2,9.704588E6,3.2683786E-2,-5.2382816E-3,5.8489166E-2,5.347906E-3,1.6459E4,1.12E3,6.4065235E8,4.98884E1,1.4590734E8,1.207E3,6.4926483E-3,1.15E2,-1.0864891E-2,4.2220557E3,2E0,4.5143375E-3,9.549387E-3,3.7430253E-2,-1.2066416E-2,2.26149E-3,-5.032496E-3,2.8246716E7,-3.2062712E-3,6.1493185E-3,5.125401E1,7.792E3,4.2837304E-3,1.1785101E4,-1.9528551E-3,-1.3330031E-2,3.44094E1,1.133457E6,1.8956E4,2.4903E4,1.5074061E-3,1.29729E-2,9.00519E-3,2.7362086E-2,-1.2126731E-2,-2.744309E-3,-8.277583E-3,-1.7765148E-2,-8.697831E-3,1.9533562E-3,-6.6926107E-3,-1.5051042E-2,4.778148E-3,-4.257874E-3,-1.704043E-3,8.488193E-3,2.2171825E-2,4.2616148E-3,-4.634405E-3,1.4431078E-2],"split_indices":[2,1,61,9,7,54,65,12,0,42,54,44,63,1,37,0,0,54,37,0,0,54,37,0,8,0,0,3,0,61,61,64,0,8,42,41,10,0,9,0,0,0,0,9,0,7,65,5,0,0,3,0,4,8,0,0,0,0,0,0,41,0,0,57,2,0,57,0,0,67,9,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,2.79E2,1.51E2,6.8E1,2.11E2,1.16E2,3.5E1,6.6E1,2E0,1.99E2,1.2E1,1.04E2,1.2E1,3E0,3.2E1,5.9E1,7E0,1.51E2,4.8E1,5E0,7E0,5.9E1,4.5E1,3E0,9E0,2E0,1E0,8E0,2.4E1,3.8E1,1.13E2,4.2E1,6E0,4.3E1,1.6E1,2.1E1,2.4E1,2E0,7E0,7E0,1E0,2.3E1,1E0,2.7E1,1.1E1,8.5E1,2.8E1,7E0,3.5E1,4E0,3.9E1,3E0,1.3E1,1.9E1,2E0,1E0,2.3E1,5E0,2E0,8E0,1.9E1,8E0,3E0,7.6E1,9E0,7E0,2.1E1,2E0,5E0,2.4E1,1.1E1,1.9E1,2E1,4E0,9E0,2E0,1.7E1,1.7E1,2E0,7E0,6.9E1,8E0,1E0,9E0,1.2E1,4E0,2E1,6E0,5E0,1.8E1,1E0,1E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.991209E-2,-2.2447844E-1,4.8195735E-1,-4.9887118E-1,-1.4175464E-1,3.4816936E-1,8.8479894E-1,-5.193955E-1,-1.5280545E-1,-1.7272626E-1,2.8271523E-1,4.2048353E-1,-2.8948897E-1,3.0410528E-2,9.906472E-1,-2.5935208E-2,-1.4583776E-2,-1.2126548E-2,-1.5287577E-4,-2.2396152E-1,-8.3882755E-5,-3.1206394E-3,3.4114578E-1,2.8690192E-1,6.089712E-1,-6.0382354E-1,2.1296194E-2,1.0460874E-2,-8.017088E-3,6.409416E-2,7.762837E-1,-2.3514861E-1,1.5868093E-1,-7.3845714E-2,1.2285716E-1,2.1334577E-2,7.192654E-3,4.9890332E-2,3.1185064E-1,3.5640252E-1,3.4402374E-2,-4.116771E-3,-3.3289976E-2,-7.2364057E-3,1.4456579E-2,4.1161455E-2,1.4792085E-3,-2.5885102E-1,-8.440026E-2,1.2640998E-2,-6.154628E-3,9.3115633E-4,-1.860886E-1,-3.931253E-3,1.8707861E-1,8.772103E-3,-1.8955506E-3,7.0269965E-2,3.2942027E-1,6.109285E-3,4.0752798E-1,-5.111997E-3,5.218979E-3,-2.9226086E-1,-1.5981688E-1,-1.4517765E-2,-1.5296295E-1,-6.2620267E-3,2.7446568E-3,-9.859624E-3,3.9333533E-4,1.2981948E-2,6.644738E-2,-3.7864812E-3,9.433366E-3,3.7948382E-1,2.4617818E-1,2.0786127E-2,4.140709E-3,-8.407143E-3,-1.54769225E-2,-1.3795064E-2,3.8757388E-4,4.339099E-3,-3.7619176E-3,-8.601582E-3,2.4440358E-3,8.554191E-3,-6.47165E-4,1.9149937E-2,4.23887E-3,4.7520925E-3,1.4241806E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,45,47,49,51,53,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,61,63,65,-1,-1,67,69,-1,71,-1,-1,73,75,-1,77,-1,-1,79,81,83,85,-1,-1,-1,-1,-1,87,-1,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7986046E1,6.2395515E0,7.614559E0,3.752098E-1,2.8753333E0,5.2862387E0,3.2255516E0,7.811546E-2,7.304707E-2,1.7779021E0,3.26594E-1,2.3683128E0,1.1851454E0,2.195314E-1,1.291214E0,0E0,0E0,0E0,0E0,7.008152E-1,4.352852E-1,0E0,1.8688309E-1,3.4514284E-1,8.2820034E-1,2.1046853E-1,3.7914446E-1,0E0,0E0,0E0,1.3181849E0,5.1769066E-1,1.8163186E-1,2.5196844E-1,2.5152704E-1,0E0,0E0,8.960423E-2,2.1029854E-1,1.12769604E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.590778E-2,3.8342476E-1,1.0318862E-1,0E0,0E0,1.4655337E-1,4.4558704E-2,0E0,1.2087688E-1,0E0,0E0,1.0864722E-1,1.1911583E-1,0E0,4.6396613E-2,0E0,0E0,2.6921082E-1,7.4426425E-1,8.666443E-2,6.601989E-2,0E0,0E0,0E0,0E0,0E0,6.442146E-2,0E0,0E0,1.1978912E-1,1.3066316E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,22,22,23,23,24,24,25,25,26,26,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,46,46,47,47,48,48,51,51,52,52,54,54,57,57,58,58,60,60,63,63,64,64,65,65,66,66,72,72,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,46,48,50,52,54,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,62,64,66,-1,-1,68,70,-1,72,-1,-1,74,76,-1,78,-1,-1,80,82,84,86,-1,-1,-1,-1,-1,88,-1,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,5.932393E3,6.932264E8,1.4777102E9,3.3503532E7,5.3140095E1,2.9778808E7,6.7E1,9.7467445E4,1E0,4.1331047E-1,3.5226266E0,1.0014992E8,1.417988E6,-2.5935208E-2,-1.4583776E-2,-1.2126548E-2,-1.5287577E-4,3.7069206E5,3.23643E6,-3.1206394E-3,6.1614815E6,2.8529238E1,1E0,1.6799316E7,4.579283E6,1.0460874E-2,-8.017088E-3,6.409416E-2,2.0531621E4,7.164074E8,1.0767E4,7.03764E5,4.3E1,2.1334577E-2,7.192654E-3,1.6485538E3,2.0772269E8,1.150124E0,3.4402374E-2,-4.116771E-3,-3.3289976E-2,-7.2364057E-3,1.4456579E-2,4.1161455E-2,1.379E3,1.0864745E3,5.477955E6,1.2640998E-2,-6.154628E-3,1.1244955E5,1.7997152E7,-3.931253E-3,2.9194368E1,8.772103E-3,-1.8955506E-3,3.2604893E3,1.15E2,6.109285E-3,1.5631589E9,-5.111997E-3,5.218979E-3,6.1453125E1,7.203055E10,5.6E1,7.8E2,-6.2620267E-3,2.7446568E-3,-9.859624E-3,3.9333533E-4,1.2981948E-2,7.9E1,-3.7864812E-3,9.433366E-3,7.378667E12,6.9E1,2.0786127E-2,4.140709E-3,-8.407143E-3,-1.54769225E-2,-1.3795064E-2,3.8757388E-4,4.339099E-3,-3.7619176E-3,-8.601582E-3,2.4440358E-3,8.554191E-3,-6.47165E-4,1.9149937E-2,4.23887E-3,4.7520925E-3,1.4241806E-2],"split_indices":[2,1,61,7,7,54,65,12,3,42,13,44,63,1,38,0,0,0,0,37,1,0,54,65,109,1,38,0,0,0,4,7,0,38,8,0,0,4,5,44,0,0,0,0,0,0,0,61,54,0,0,42,9,0,63,0,0,4,3,0,41,0,0,57,40,3,8,0,0,0,0,0,3,0,0,40,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.23E2,2.77E2,1.46E2,6.3E1,2.14E2,1.11E2,3.5E1,5.9E1,4E0,2E2,1.4E1,1E2,1.1E1,4E0,3.1E1,5.3E1,6E0,2E0,2E0,1.54E2,4.6E1,2E0,1.2E1,6E1,4E1,5E0,6E0,2E0,2E0,1E1,2.1E1,1.5E2,4E0,2.9E1,1.7E1,7E0,5E0,6E0,5.4E1,1.3E1,2.7E1,1E0,4E0,4E0,2E0,1.9E1,2E0,1.29E2,2.1E1,3E0,1E0,1.8E1,1.1E1,4E0,1.3E1,2E0,4E0,4E0,5E1,3E0,1E1,1E0,1E0,9.5E1,3.4E1,1.1E1,1E1,5E0,1.3E1,1E1,1E0,7E0,6E0,2E0,2E0,2.9E1,2.1E1,9E0,1E0,2E1,7.5E1,1.9E1,1.5E1,4E0,7E0,9E0,1E0,2E0,4E0,2.7E1,2E0,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.4917978E-2,-2.0547587E-1,5.318811E-1,-4.5584297E-1,-1.3034378E-1,6.053006E-1,-4.0968323E-1,-4.8190233E-1,-9.976368E-4,-1.7334475E-1,2.3951818E-1,4.6595705E-1,1.0312886E0,-7.391404E-1,3.259496E-3,-2.4621041E-2,-1.406869E-2,-2.888304E-1,-9.377694E-2,3.148984E-1,3.99729E-2,3.0885956E-1,6.0680133E-1,1.1332271E0,3.1533855E-1,-4.2274687E-2,-1.4910418E-2,-1.799261E-1,-3.330235E-1,-1.4646667E-1,2.5577226E-1,1.9089093E-2,1.0149424E-2,5.673467E-3,-3.2371737E-3,4.0124384E-1,2.3513955E-1,4.5861363E-1,3.4452777E-2,6.7862645E-2,4.130004E-2,3.143465E-2,-6.0940892E-3,-2.0133379E-1,-3.133276E-4,-3.5606384E-1,-6.690111E-3,-2.0990397E-1,-1.39189875E-2,-3.4758523E-3,4.3759352E-1,2.1399785E-2,6.2959725E-3,2.9831824E-1,1.5065244E-1,4.898338E-1,4.2044907E-3,-1.7391391E-3,-2.3258309E-1,-2.1117693E-2,-2.9740438E-1,-6.8365425E-2,-2.502107E-1,-4.7185298E-2,1.3539167E-1,-3.1191285E-3,3.5637275E-3,1.2850891E-2,3.025837E-2,6.112967E-3,1.5941953E-2,2.174943E-4,8.4201135E-3,8.283464E-3,2.4795955E-2,-1.36728315E-2,-6.6900193E-3,-1.1306686E-2,-1.8957788E-2,-9.543026E-3,1.5464342E-3,-1.8747413E-3,-1.2784191E-2,-4.4449666E-3,6.7727425E-4,-1.6609002E-3,8.169672E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,57,-1,59,-1,61,63,65,67,-1,-1,69,71,73,-1,-1,75,-1,77,79,81,83,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9557293E1,5.6778183E0,9.234776E0,7.8381443E-1,3.7949624E0,6.7018623E0,1.6224183E0,2.1161175E-1,0E0,1.9209695E0,3.6345696E-1,1.870327E0,1.7591553E0,8.3884954E-2,0E0,0E0,0E0,3.5398245E-1,2.3765035E0,6.889415E-2,7.4476674E-2,2.1893787E-1,4.3872833E-1,6.9125366E-1,8.25872E-1,0E0,0E0,9.893417E-2,2.1915865E-1,9.3041825E-1,8.0284715E-1,0E0,0E0,0E0,0E0,1.6106224E-1,1.0451543E-1,2.2398376E-1,0E0,0E0,0E0,0E0,0E0,1.1555606E-1,0E0,1.2442112E-1,0E0,4.1076565E-1,1.9016804E-1,4.2679716E-2,1.5770435E-1,0E0,0E0,4.1237235E-2,4.827702E-2,7.4845314E-2,0E0,0E0,5.6346893E-2,0E0,1.1264491E-1,2.4057381E-1,1.7273617E-1,8.6720176E-2,4.6315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,35,35,36,36,37,37,43,43,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,58,58,60,60,61,61,62,62,63,63,64,64],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,58,-1,60,-1,62,64,66,68,-1,-1,70,72,74,-1,-1,76,-1,78,80,82,84,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.32424E5,1.0429407E3,1.2852292E7,1.4777102E9,7.7101436E3,1.31805E5,2.4434292E7,-9.976368E-4,7.377246E2,5.4629724E10,1.8618166E-1,1.92E2,2.8E1,3.259496E-3,-2.4621041E-2,-1.406869E-2,7.37365E6,2.3171705E5,6.3349745E6,2.341E4,1.8887723E-2,1.30817086E5,3.3616E4,2.235E5,-4.2274687E-2,-1.4910418E-2,6.1614815E6,3.312442E6,2.8884522E8,3.724162E6,1.9089093E-2,1.0149424E-2,5.673467E-3,-3.2371737E-3,6.2185767E10,1.8053012E5,1.7092716E9,3.4452777E-2,6.7862645E-2,4.130004E-2,3.143465E-2,-6.0940892E-3,8.87682E5,-3.133276E-4,1.934371E6,-6.690111E-3,4.2393894E1,2.1145713E3,1.214E3,1.901216E3,2.1399785E-2,6.2959725E-3,8.879629E4,3.076578E3,2E0,4.2044907E-3,-1.7391391E-3,4.759E3,-2.1117693E-2,6.9E1,8.8543115E2,7.3938284E0,1.07E3,8.42E2,-3.1191285E-3,3.5637275E-3,1.2850891E-2,3.025837E-2,6.112967E-3,1.5941953E-2,2.174943E-4,8.4201135E-3,8.283464E-3,2.4795955E-2,-1.36728315E-2,-6.6900193E-3,-1.1306686E-2,-1.8957788E-2,-9.543026E-3,1.5464342E-3,-1.8747413E-3,-1.2784191E-2,-4.4449666E-3,6.7727425E-4,-1.6609002E-3,8.169672E-3],"split_indices":[2,1,67,9,7,61,37,12,0,61,40,44,8,3,0,0,0,54,37,54,11,66,37,10,2,0,0,54,9,12,1,0,0,0,0,5,42,41,0,0,0,0,0,1,0,1,0,65,61,0,4,0,0,42,4,8,0,0,2,0,8,61,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.04E2,1.29E2,6.9E1,2.35E2,1.2E2,9E0,6.5E1,4E0,2.11E2,2.4E1,9.2E1,2.8E1,5E0,4E0,5.5E1,1E1,8.5E1,1.26E2,1.7E1,7E0,4.5E1,4.7E1,2.4E1,4E0,3E0,2E0,2.6E1,5.9E1,1.1E2,1.6E1,8E0,9E0,4E0,3E0,1.8E1,2.7E1,2.2E1,2.5E1,1E1,1.4E1,2E0,2E0,2.3E1,3E0,5.2E1,7E0,7.4E1,3.6E1,7E0,9E0,1.5E1,3E0,1.4E1,1.3E1,2E1,2E0,4E0,1.9E1,1.9E1,3.3E1,1.7E1,5.7E1,3E1,6E0,4E0,3E0,6E0,3E0,3E0,1.1E1,2E0,1.1E1,2E0,1.8E1,1.1E1,8E0,2.2E1,1.1E1,7E0,1E1,4E0,5.3E1,1.7E1,1.3E1,1E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-1.7011873E-3,-2.2863716E-1,3.862971E-1,-4.6781018E-1,-1.4370531E-1,2.749962E-1,8.81305E-1,-4.893151E-1,-1.1700603E-2,-1.7105514E-1,2.9219243E-1,3.5668412E-1,-2.793593E-1,6.1222464E-2,9.7726595E-1,-2.38297E-2,-4.5701964E-3,-2.0681953E-1,-3.09478E-2,3.723473E-1,-3.4902496E-3,2.2516745E-1,5.789406E-1,-6.547504E-2,-4.619637E-2,9.590774E-3,-8.482366E-3,4.91489E-1,1.1048014E0,-2.2348484E-1,4.551022E-2,-1.0562274E-1,1.265269E-1,2.2070192E-2,9.421905E-3,1.0911027E-1,2.8839302E-1,3.6736473E-2,4.5584542E-1,4.778463E-2,-1.7385231E-2,2.9261138E-2,-4.841841E-3,5.4594748E-2,1.4702893E-2,-2.2981358E-1,2.2933218E-3,-2.8918968E-3,1.0546612E-2,-1.2864591E-1,5.8555603E-3,1.1685172E-3,2.4182817E-1,1.4991865E-1,-1.2889544E-2,2.9992247E-3,3.2076627E-1,4.6992472E-1,2.1644896E-3,-7.409299E-2,1.9165311E-2,-2.7724597E-1,-1.7212018E-1,-1.5976903E-1,-1.6688407E-4,1.3628257E-3,1.344683E-2,4.6405293E-2,1.2448917E-2,3.798749E-1,1.0046324E-2,3.233497E-3,4.8335668E-1,-8.822753E-3,5.196091E-3,-1.661367E-2,-1.1438736E-2,-1.0380183E-2,-2.4035044E-3,-1.0099525E-2,-2.8467993E-3,-4.3054563E-3,7.6828534E-3,2.0043418E-2,7.397729E-3,2.3841169E-2,5.2245473E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,39,-1,-1,-1,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,-1,-1,-1,-1,-1,61,-1,-1,-1,63,-1,-1,65,67,-1,-1,69,71,-1,73,-1,75,77,79,-1,-1,-1,81,-1,83,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6981594E1,5.3158274E0,8.297344E0,2.1532631E-1,2.3985615E0,5.896014E0,2.1436558E0,7.335091E-2,0E0,9.256182E-1,3.776281E-1,3.1434488E0,2.4052434E0,1.6560534E-1,1.005846E0,0E0,0E0,6.3253355E-1,4.7198126E-1,6.1638236E-2,0E0,4.968536E-1,5.7614136E-1,4.8514333E-1,0E0,0E0,0E0,5.390376E-1,7.2582245E-2,2.4937105E-1,1.958535E-1,1.567742E-1,1.4746615E-1,0E0,0E0,4.5484903E-1,3.168807E-1,0E0,1.3512516E-1,4.9828407E-1,0E0,0E0,0E0,0E0,0E0,3.244486E-1,0E0,0E0,0E0,9.6850485E-2,0E0,0E0,3.9604634E-2,2.6820946E-1,0E0,0E0,1.8661356E-1,1.08276844E-1,0E0,1.9794644E-1,0E0,1.1486578E-1,3.2705224E-1,8.774549E-2,0E0,0E0,0E0,2.2878797E-1,0E0,1.3903737E-1,0E0,0E0,5.0578117E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,23,23,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,38,38,39,39,45,45,49,49,52,52,53,53,56,56,57,57,59,59,61,61,62,62,63,63,67,67,69,69,72,72],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,40,-1,-1,-1,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,-1,-1,-1,-1,-1,62,-1,-1,-1,64,-1,-1,66,68,-1,-1,70,72,-1,74,-1,76,78,80,-1,-1,-1,82,-1,84,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.203E3,7.41718E5,6.6512886E3,2.7299898E8,1.4096699E9,3.5301748E7,5.3140095E1,3.6851662E6,-1.1700603E-2,4.3159333E-1,1.5E2,4.1331047E-1,1.6244629E-1,1.0014992E8,6.274818E4,-2.38297E-2,-4.5701964E-3,2.9063344E5,4.819212E10,6.088569E6,-3.4902496E-3,1.3428E4,1.6611694E6,2.5503985E9,-4.619637E-2,9.590774E-3,-8.482366E-3,4.42384E5,2.3322949E2,7.8E2,4.037E3,5.4210108E7,5.792806E4,2.2070192E-2,9.421905E-3,1.9051096E7,3.8952081E0,3.6736473E-2,1.5631589E9,8.01E3,-1.7385231E-2,2.9261138E-2,-4.841841E-3,5.4594748E-2,1.4702893E-2,2.025313E6,2.2933218E-3,-2.8918968E-3,1.0546612E-2,1.8931546E8,5.8555603E-3,1.1685172E-3,1.094E3,3.44E2,-1.2889544E-2,2.9992247E-3,1.955799E5,1.872503E4,2.1644896E-3,9.704588E6,1.9165311E-2,4.822E3,1.156185E3,7.164074E8,-1.6688407E-4,1.3628257E-3,1.344683E-2,9.65325E8,1.2448917E-2,2.5111986E-3,1.0046324E-2,3.233497E-3,9.677921E7,-8.822753E-3,5.196091E-3,-1.661367E-2,-1.1438736E-2,-1.0380183E-2,-2.4035044E-3,-1.0099525E-2,-2.8467993E-3,-4.3054563E-3,7.6828534E-3,2.0043418E-2,7.397729E-3,2.3841169E-2,5.2245473E-3],"split_indices":[2,1,61,7,7,54,65,56,0,48,3,44,47,1,37,0,0,37,40,54,0,2,56,7,0,0,0,2,66,8,10,9,37,0,0,54,62,0,41,0,0,0,0,0,0,1,0,0,0,41,0,0,0,8,0,0,42,42,0,9,0,2,61,7,0,0,0,5,0,43,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.18E2,2.64E2,1.54E2,6.8E1,1.96E2,1.27E2,2.7E1,6.1E1,7E0,1.85E2,1.1E1,1.11E2,1.6E1,3E0,2.4E1,6E1,1E0,1.47E2,3.8E1,9E0,2E0,7.1E1,4E1,1.3E1,3E0,2E0,1E0,6E0,1.8E1,1.38E2,9E0,2.6E1,1.2E1,5E0,4E0,2.6E1,4.5E1,1.4E1,2.6E1,1E1,3E0,5E0,1E0,1.7E1,1E0,1.35E2,3E0,6E0,3E0,2.4E1,2E0,7E0,5E0,2.4E1,2E0,6E0,3.9E1,2.5E1,1E0,8E0,2E0,7.2E1,6.3E1,1.9E1,5E0,1E0,4E0,1.3E1,1.1E1,2.4E1,1.5E1,1E0,2.4E1,5E0,3E0,2.4E1,4.8E1,4.6E1,1.7E1,1.2E1,7E0,6E0,7E0,2E1,4E0,2.3E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[1.3451445E-2,-2.1121365E-1,4.4921204E-1,-4.3484473E-1,-1.3155079E-1,3.1517997E-1,8.099208E-1,-4.524948E-1,-1.9383975E-4,-1.6849951E-1,9.43851E-2,3.7594262E-1,-1.7179556E-1,1.0290357E-1,8.856198E-1,-4.7715017E-1,-1.2027502E-2,-1.9172114E-1,1.3048907E-1,2.6635118E-2,1.5567267E-2,2.549085E-1,6.102863E-1,-5.3840035E-1,5.0962158E-2,-8.052561E-3,1.6321879E-2,1.181492E0,7.0817643E-1,-2.3211928E-2,-5.023752E-3,-2.1995755E-1,-6.043081E-2,2.5765356E-1,-3.9539877E-3,-3.32162E-2,1.8528368E-1,8.234978E-4,2.8934675E-1,8.551573E-3,6.3249576E-1,-3.8211537E-3,-3.0538756E-2,-3.41921E-2,1.7651757E-2,5.9366792E-2,1.5282953E-2,-4.7251135E-3,7.458504E-1,-2.4727733E-1,-6.7856066E-2,-1.0691359E-2,-2.9467497E-2,1.5897109E-3,1.357845E-2,-1.21408015E-1,2.0231588E-3,1.1948627E-2,9.307408E-4,-1.1599798E-2,7.792757E-3,2.0285854E-1,3.5598186E-1,6.870725E-1,1.6741635E-2,-9.551076E-3,1.792279E-1,3.7120193E-2,4.6601235E-3,-2.6040578E-1,-9.351332E-2,1.1328396E-2,-1.07883036E-1,-6.379748E-2,1.4169553E-3,1.8851543E-3,-7.056812E-3,1.02419525E-1,3.1087127E-1,3.688856E-1,5.221002E-3,3.3950817E-2,6.688106E-3,-1.7159295E-4,1.1636579E-2,-1.45349605E-2,-8.579748E-3,5.4873074E-3,-6.057175E-3,-8.519776E-3,-1.4107025E-3,1.925453E-3,-4.6425886E-3,-1.6406847E-3,8.07783E-3,1.7260093E-2,5.563836E-3,1.8808052E-2,8.915079E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,-1,37,39,41,43,-1,-1,45,47,-1,-1,49,51,53,-1,55,57,59,61,-1,63,-1,-1,65,-1,-1,-1,-1,67,69,71,-1,73,-1,-1,75,-1,-1,-1,-1,-1,77,79,81,-1,-1,83,-1,-1,85,87,-1,89,91,-1,-1,-1,93,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3170967E1,5.1269884E0,6.99671E0,5.7608986E-1,1.8190811E0,3.3490677E0,2.0496902E0,2.4642181E-1,0E0,1.3164158E0,4.7126675E-1,2.6524267E0,1.0890708E0,3.7451178E-1,1.0533142E0,5.166912E-2,0E0,6.2804747E-1,3.9943448E-1,2.4353756E-1,0E0,5.860276E-1,2.0511818E-1,1.6784227E-1,2.540342E-1,0E0,0E0,1.2613297E-1,8.323889E-1,0E0,0E0,5.732851E-1,1.5329444E-1,3.9173663E-2,0E0,1.3106357E-1,6.786893E-2,3.8823947E-1,2.6781225E-1,0E0,2.4408054E-1,0E0,0E0,3.153354E-1,0E0,0E0,0E0,0E0,2.9990005E-1,2.1789026E-1,3.0411792E-1,0E0,5.8448818E-2,0E0,0E0,4.176481E-2,0E0,0E0,0E0,0E0,0E0,2.7192414E-1,6.2335014E-2,1.3632393E-1,0E0,0E0,4.6274245E-2,0E0,0E0,3.1871128E-1,8.7530926E-2,0E0,1.0845633E-1,6.450933E-2,0E0,0E0,0E0,1.4785714E-1,7.839191E-2,4.4857502E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,31,31,32,32,33,33,35,35,36,36,37,37,38,38,40,40,43,43,48,48,49,49,50,50,52,52,55,55,61,61,62,62,63,63,66,66,69,69,70,70,72,72,73,73,77,77,78,78,79,79],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,-1,38,40,42,44,-1,-1,46,48,-1,-1,50,52,54,-1,56,58,60,62,-1,64,-1,-1,66,-1,-1,-1,-1,68,70,72,-1,74,-1,-1,76,-1,-1,-1,-1,-1,78,80,82,-1,-1,84,-1,-1,86,88,-1,90,92,-1,-1,-1,94,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,5.7638027E3,1.2852292E7,3.53E2,3.5301748E7,5.0314346E1,2.9778808E7,-1.9383975E-4,2.48374E5,1.1316252E3,6.0891E4,2.9648642E1,3.677E3,1.417988E6,3.84142E5,-1.2027502E-2,9.7467445E4,1.4074289E7,1.577058E9,1.5567267E-2,2.0690203E3,3.0811954E-2,1.6799316E7,3.305542E4,-8.052561E-3,1.6321879E-2,2.231E3,1.379E3,-2.3211928E-2,-5.023752E-3,2.1898598E5,1.6826648E3,2.473E3,-3.9539877E-3,1.570662E5,1.845378E6,2.722E3,1.9412996E9,8.551573E-3,6.482967E7,-3.8211537E-3,-3.0538756E-2,2.1853803E4,1.7651757E-2,5.9366792E-2,1.5282953E-2,-4.7251135E-3,2.617838E8,7.164074E8,6.4E1,-1.0691359E-2,7.788E3,1.5897109E-3,1.357845E-2,3.73E2,2.0231588E-3,1.1948627E-2,9.307408E-4,-1.1599798E-2,7.792757E-3,1.613448E9,1.9658574E2,6.029086E10,1.6741635E-2,-9.551076E-3,1.1459E4,3.7120193E-2,4.6601235E-3,2.735322E6,4.138E3,1.1328396E-2,4.069E3,1.8505337E3,1.4169553E-3,1.8851543E-3,-7.056812E-3,2.0236879E3,7.50044E5,9.920844E0,5.221002E-3,3.3950817E-2,6.688106E-3,-1.7159295E-4,1.1636579E-2,-1.45349605E-2,-8.579748E-3,5.4873074E-3,-6.057175E-3,-8.519776E-3,-1.4107025E-3,1.925453E-3,-4.6425886E-3,-1.6406847E-3,8.07783E-3,1.7260093E-2,5.563836E-3,1.8808052E-2,8.915079E-3],"split_indices":[2,1,61,9,8,54,65,12,0,11,61,2,65,0,38,38,0,42,54,7,0,4,47,1,4,0,0,0,0,0,0,37,4,0,0,56,1,0,12,0,53,0,0,42,0,0,0,0,1,7,3,0,10,0,0,8,0,0,0,0,0,7,67,5,0,0,2,0,0,1,2,0,2,4,0,0,0,61,38,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,2.9E2,1.49E2,7.5E1,2.15E2,1.1E2,3.9E1,7.2E1,3E0,1.85E2,3E1,9.8E1,1.2E1,4E0,3.5E1,6.3E1,9E0,1.72E2,1.3E1,2.4E1,6E0,6.6E1,3.2E1,4E0,8E0,2E0,2E0,1.1E1,2.4E1,6.2E1,1E0,1.41E2,3.1E1,8E0,5E0,1.8E1,6E0,8E0,5.8E1,2E0,3E1,1E0,3E0,7E0,1E0,1E1,1E0,1E0,2.3E1,1.19E2,2.2E1,4E0,2.7E1,1E0,7E0,8E0,1E1,4E0,2E0,3E0,5E0,2.7E1,3.1E1,2.4E1,6E0,4E0,3E0,2.2E1,1E0,1.09E2,1E1,2E0,2E1,1.7E1,1E1,1E0,7E0,1.5E1,1.2E1,2.9E1,2E0,2.3E1,1E0,1E0,2E0,7.1E1,3.8E1,1E0,9E0,1E1,1E1,4E0,1.3E1,5E0,1E1,9E0,3E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[1.3379573E-2,-1.7200804E-1,4.3641403E-1,-2.4046129E-1,4.7149602E-2,5.1138306E-1,-3.4964392E-1,-3.6591083E-1,-1.4307694E-1,-4.2926572E-2,2.2909056E-1,4.0001094E-1,8.630339E-1,-7.3529744E-1,3.0865358E-2,-3.890949E-1,-1.2066325E-1,-2.408678E-1,-7.042465E-2,-1.9282982E-2,-1.4662279E-2,2.630201E-1,-4.6330485E-3,2.670497E-1,5.190986E-1,9.5850897E-1,2.321469E-1,-4.219915E-2,-1.4640545E-2,-6.018872E-3,1.3503923E-2,-4.0615314E-1,-7.882271E-3,-1.9877638E-4,-8.197684E-3,-1.5410715E-1,-2.789106E-1,-1.3441631E-1,1.2862487E-1,-6.917702E-2,6.989962E-2,1.6764969E-2,1.0206189E-1,-2.7806014E-3,2.8312033E-1,1.6672589E-2,5.6726277E-1,1.0010651E0,1.5763123E-2,2.7118588E-2,-8.463296E-3,-4.5650473E-1,-3.2405654E-1,-1.0270924E-2,-3.8468705E-3,-1.7586412E-1,-3.339883E-1,-1.7928152E-1,-1.566604E-2,1.12121E-2,-1.26734115E-2,6.2422194E-3,-1.0471653E-1,1.04607E-3,8.73915E-3,6.5069613E-3,-3.8833101E-3,3.3323506E-1,1.9221935E-1,2.8496841E-2,1.106091E-2,5.082206E-2,2.40652E-2,-2.255799E-2,-1.0215369E-2,-4.437316E-3,-1.6168242E-2,-9.397355E-3,2.185614E-3,-4.526874E-3,-3.523842E-1,-1.396611E-1,-1.341572E-2,-1.5206295E-1,9.501809E-2,2.473041E-3,-3.5729492E-3,-1.6648301E-1,-2.3299016E-3,2.6227366E-3,1.7037883E-2,4.57737E-3,1.1836875E-2,-3.1369342E-3,-1.7526362E-2,-5.4552522E-3,-1.2547474E-2,-9.352482E-3,-8.466469E-4,6.1902596E-4,8.419938E-3,-1.1067818E-2,-4.2740954E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,43,45,47,49,-1,-1,-1,-1,51,-1,-1,-1,53,55,57,59,61,63,-1,65,-1,67,-1,69,71,-1,-1,-1,73,75,-1,-1,77,79,81,83,-1,85,-1,87,-1,-1,-1,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,93,95,-1,97,99,-1,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3719543E1,4.496951E0,7.89863E0,2.7330818E0,1.1898093E0,4.3544674E0,1.783632E0,5.159483E-1,9.0665174E-1,2.9671055E-1,2.9007065E-1,1.3339405E0,1.4640331E0,9.661245E-2,3.065668E-1,2.7997875E-1,5.682005E-2,1.3855195E-1,9.8534E-1,2.0952073E-1,0E0,2.6439238E-1,0E0,2.563765E-1,2.0451546E-1,3.1778717E-1,7.7198875E-1,0E0,0E0,0E0,0E0,1.9516373E-1,0E0,0E0,0E0,6.619638E-2,1.5125918E-1,3.062011E-1,2.81923E-1,2.2534503E-1,8.731616E-2,0E0,6.492572E-2,0E0,1.3638687E-1,0E0,1.5344238E-1,5.9036255E-2,0E0,0E0,0E0,3.691864E-2,6.0690165E-2,0E0,0E0,7.19991E-2,6.850338E-2,1.2581706E-1,2.7109724E-1,0E0,3.916463E-2,0E0,8.2520366E-2,0E0,0E0,0E0,0E0,1.3234043E-1,6.998986E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.3405533E-2,7.2876394E-2,0E0,4.1688025E-2,6.3182496E-2,0E0,0E0,3.85572E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,26,26,31,31,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,46,46,47,47,51,51,52,52,55,55,56,56,57,57,58,58,60,60,62,62,67,67,68,68,80,80,81,81,83,83,84,84,87,87],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,44,46,48,50,-1,-1,-1,-1,52,-1,-1,-1,54,56,58,60,62,64,-1,66,-1,68,-1,70,72,-1,-1,-1,74,76,-1,-1,78,80,82,84,-1,86,-1,88,-1,-1,-1,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,94,96,-1,98,100,-1,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.164074E8,1.0429407E3,1.276292E6,1.5522031E9,7.7101436E3,3.5226266E0,1.73E2,7.3123126E2,1E0,7.4433655E-2,4.1331047E-1,1.62E2,3.1E1,3.0972284E1,1.7212875E5,1.5755866E4,1.2155118E7,2.103E3,1.02470586E5,-1.4662279E-2,7.200105E8,-4.6330485E-3,1.4E1,1E0,3.931053E7,2.235E5,-4.219915E-2,-1.4640545E-2,-6.018872E-3,1.3503923E-2,1.425966E6,-7.882271E-3,-1.9877638E-4,-8.197684E-3,2.652E3,1E0,5.3008755E8,1.4074289E7,2.409091E0,9.302345E4,1.6764969E-2,4.264618E6,-2.7806014E-3,1.955799E5,1.6672589E-2,5.848343E7,7.549299E6,1.5763123E-2,2.7118588E-2,-8.463296E-3,1.5650286E6,4.5E1,-1.0270924E-2,-3.8468705E-3,4.75E2,3.4416797E2,6.54925E6,1.656968E6,1.12121E-2,1.6991062E3,6.2422194E-3,7.33E2,1.04607E-3,8.73915E-3,6.5069613E-3,-3.8833101E-3,3.8952081E0,3.2352256E3,2.8496841E-2,1.106091E-2,5.082206E-2,2.40652E-2,-2.255799E-2,-1.0215369E-2,-4.437316E-3,-1.6168242E-2,-9.397355E-3,2.185614E-3,-4.526874E-3,5.24E0,1.6866E4,-1.341572E-2,2.029E3,2.972948E6,2.473041E-3,-3.5729492E-3,9.3E1,-2.3299016E-3,2.6227366E-3,1.7037883E-2,4.57737E-3,1.1836875E-2,-3.1369342E-3,-1.7526362E-2,-5.4552522E-3,-1.2547474E-2,-9.352482E-3,-8.466469E-4,6.1902596E-4,8.419938E-3,-1.1067818E-2,-4.2740954E-3],"split_indices":[2,7,67,1,7,61,63,8,61,18,66,44,8,3,65,37,37,54,0,42,0,5,0,3,109,54,2,0,0,0,0,9,0,0,0,10,22,12,54,67,37,0,1,0,42,0,1,38,0,0,0,56,3,0,0,3,42,9,9,0,42,0,10,0,0,0,0,62,61,0,0,0,0,0,0,0,0,0,0,0,65,10,0,0,1,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.98E2,1.3E2,2.27E2,7.1E1,1.19E2,1.1E1,9.8E1,1.29E2,4.8E1,2.3E1,9.2E1,2.7E1,5E0,6E0,8.9E1,9E0,5.4E1,7.5E1,4.5E1,3E0,2.1E1,2E0,4.5E1,4.7E1,2.3E1,4E0,3E0,2E0,4E0,2E0,8.2E1,7E0,3E0,6E0,1.8E1,3.6E1,5.7E1,1.8E1,2.9E1,1.6E1,1.3E1,8E0,2E0,4.3E1,1.2E1,3.5E1,2.1E1,2E0,2E0,2E0,4.8E1,3.4E1,9E0,9E0,1.4E1,2.2E1,4.1E1,1.6E1,1E1,8E0,4E0,2.5E1,1.2E1,4E0,7E0,1E0,2.6E1,1.7E1,3.2E1,3E0,1.8E1,3E0,4.5E1,3E0,2E0,3.2E1,1.3E1,1E0,2E0,2E1,3.1E1,1E1,7E0,9E0,4E0,4E0,1.1E1,1.4E1,2E0,2.4E1,7E0,1E1,1E0,1.9E1,2.7E1,4E0,5E0,2E0,5E0,4E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-5.58562E-3,-1.5888067E-1,4.153524E-1,-2.569112E-1,9.468497E-2,1.3238333E-1,5.194447E-1,-4.0791467E-1,-1.934134E-1,-4.5743205E-2,3.4186375E-1,1.8252933E-1,-1.992303E-2,3.1241807E-1,6.136822E-1,-4.2356533E-1,1.4690833E-2,-1.73565E-1,-4.4796072E-2,-7.681732E-2,9.025473E-3,2.472045E-1,3.4274437E-2,-9.648483E-3,2.1700749E-1,7.761726E-3,3.552323E-1,9.516186E-1,5.540974E-1,-4.4975019E-1,-1.3039337E-2,-2.3917848E-1,-1.20008714E-1,-1.0795607E-2,-4.4245847E-2,1.2004377E-1,1.643826E-2,-3.1751571E-3,2.5358486E-1,2.7991529E-3,1.7704329E-2,2.4004757E-2,5.629078E-2,5.8393395E-1,-5.2066473E-3,-2.1909047E-2,-4.9084527E-3,-2.8781936E-1,-1.5105338E-1,-1.6042197E-1,6.058992E-2,-1.2478954E-2,-1.6286755E-1,7.3812846E-3,3.578557E-4,6.5216874E-3,2.8383163E-1,6.013265E-1,1.1681939E-2,-4.1978485E-3,-3.2344714E-1,-7.6361164E-2,-2.9195324E-1,-4.8190635E-2,-2.0186417E-1,6.397285E-3,-4.517274E-3,-3.8368344E-2,6.6575766E-2,-3.8388942E-3,-1.4601006E-2,3.431447E-1,1.79801E-1,2.9367901E-2,6.260212E-3,-1.6482918E-2,-6.6179694E-3,-6.753017E-3,-8.3431747E-4,-7.042112E-3,-1.8296117E-2,2.5856418E-3,-4.0192828E-3,-1.8374359E-3,-1.0340109E-2,-3.3546842E-3,6.43689E-4,6.07015E-4,8.213666E-3,6.7841806E-3,1.9357337E-2,9.766869E-3,-5.576214E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,-1,33,-1,35,-1,-1,37,-1,39,41,43,45,-1,47,49,-1,51,53,-1,-1,55,-1,-1,-1,-1,57,-1,-1,-1,59,61,63,65,67,69,-1,-1,-1,71,73,-1,-1,75,77,79,81,83,-1,-1,85,87,-1,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7554798E1,7.804619E0,3.289093E0,2.0880795E0,3.0701942E0,9.509112E-1,1.4275761E0,8.6513233E-1,2.1896138E0,4.26954E-1,9.419787E-1,4.3920636E-1,0E0,1.243608E-1,6.003189E-1,1.3899994E-1,0E0,5.2651024E-1,0E0,2.3442116E-1,0E0,2.808653E-1,0E0,0E0,3.0639505E-1,0E0,5.3297997E-2,1.0438299E-1,1.0507708E0,4.100418E-2,0E0,2.565775E-1,6.5929353E-1,0E0,1.6000202E-1,4.724349E-2,0E0,0E0,5.108273E-2,0E0,0E0,0E0,0E0,1.3953209E-1,0E0,0E0,0E0,2.8654885E-1,2.618574E-1,3.2980323E-1,2.0150338E-1,7.418814E-2,8.075127E-2,0E0,0E0,0E0,5.553937E-2,8.566475E-2,0E0,0E0,7.989955E-2,6.90499E-2,5.9485853E-2,7.966803E-2,1.01097584E-1,0E0,0E0,4.4507578E-2,4.827765E-2,0E0,0E0,7.33968E-2,3.7308544E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,17,17,19,19,21,21,24,24,26,26,27,27,28,28,29,29,31,31,32,32,34,34,35,35,38,38,43,43,47,47,48,48,49,49,50,50,51,51,52,52,56,56,57,57,60,60,61,61,62,62,63,63,64,64,67,67,68,68,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,-1,34,-1,36,-1,-1,38,-1,40,42,44,46,-1,48,50,-1,52,54,-1,-1,56,-1,-1,-1,-1,58,-1,-1,-1,60,62,64,66,68,70,-1,-1,-1,72,74,-1,-1,76,78,80,82,84,-1,-1,86,88,-1,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,9.7467445E4,1.00493375E5,7.41718E5,2.081964E3,8.494E3,5.6503E4,8.588E3,5.825042E6,1.5271514E5,3.3056E4,3.0972284E1,-1.992303E-2,9.992596E-2,1.417988E6,2.7299898E8,1.4690833E-2,6.505339E2,-4.4796072E-2,1.6826648E3,9.025473E-3,2.0663544E9,3.4274437E-2,-9.648483E-3,1.484798E9,7.761726E-3,1.01319025E3,2.560372E7,5.22E2,8.95654E8,-1.3039337E-2,2.72E2,2.103E3,-1.0795607E-2,8.557E3,1.556838E0,1.643826E-2,-3.1751571E-3,1.7532333E9,2.7991529E-3,1.7704329E-2,2.4004757E-2,5.629078E-2,3.0113987E10,-5.2066473E-3,-2.1909047E-2,-4.9084527E-3,2.0185115E4,9.3E1,9.174246E6,2.7394568E7,3.53E2,1.91E2,7.3812846E-3,3.578557E-4,6.5216874E-3,1.59E2,6.029086E10,1.1681939E-2,-4.1978485E-3,1.3421E4,5.802715E2,3.64E2,5.6E1,7.0934258E0,6.397285E-3,-4.517274E-3,1.754916E8,7.4378266E4,-3.8388942E-3,-1.4601006E-2,2.4339888E3,1.31E3,2.9367901E-2,6.260212E-3,-1.6482918E-2,-6.6179694E-3,-6.753017E-3,-8.3431747E-4,-7.042112E-3,-1.8296117E-2,2.5856418E-3,-4.0192828E-3,-1.8374359E-3,-1.0340109E-2,-3.3546842E-3,6.43689E-4,6.07015E-4,8.213666E-3,6.7841806E-3,1.9357337E-2,9.766869E-3,-5.576214E-5],"split_indices":[7,42,42,1,61,0,2,2,56,37,2,65,0,47,38,7,0,61,0,4,0,12,0,0,7,0,4,1,3,5,0,3,0,0,2,50,0,0,7,0,0,0,0,12,0,0,0,37,8,54,54,8,8,0,0,0,3,5,0,0,10,42,3,3,56,0,0,41,37,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,3.12E2,1.13E2,2.25E2,8.7E1,3.1E1,8.2E1,6.5E1,1.6E2,5.6E1,3.1E1,2.9E1,2E0,2.7E1,5.5E1,6.4E1,1E0,1.57E2,3E0,5E1,6E0,2.6E1,5E0,2E0,2.7E1,7E0,2E1,6E0,4.9E1,5.3E1,1.1E1,6.9E1,8.8E1,8E0,4.2E1,1.2E1,1.4E1,3E0,2.4E1,1E0,1.9E1,3E0,3E0,4.7E1,2E0,5.2E1,1E0,4.3E1,2.6E1,7.2E1,1.6E1,3.4E1,8E0,9E0,3E0,6E0,1.8E1,4.4E1,3E0,7E0,3.6E1,1.8E1,8E0,2E1,5.2E1,1.1E1,5E0,2.6E1,8E0,6E0,2E0,1E1,8E0,4.3E1,1E0,3.2E1,4E0,8E0,1E1,4E0,4E0,5E0,1.5E1,4E0,4.8E1,1.6E1,1E1,6E0,2E0,3E0,7E0,7E0,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-4.3986808E-3,-1.725971E-1,3.907061E-1,-2.5592425E-1,5.135353E-2,8.2149796E-2,4.7157738E-1,-2.4301541E-1,-4.3176156E-2,-4.5700498E-2,2.543434E-1,2.7169514E-1,-1.0023585E-1,3.818121E-1,7.4740577E-1,-3.8364676E-1,-1.8338044E-1,-8.8848114E-2,9.367974E-3,1.5891434E-1,1.8259488E-2,6.2849447E-3,1.5197571E-2,-1.951503E-2,9.619345E-3,4.0524137E-1,-7.656558E-3,5.728854E-1,4.6992153E-2,-1.947604E-2,-1.20216206E-1,-1.9287252E-1,3.0819604E-2,-2.1076639E-1,-3.8967445E-2,8.788774E-3,1.8036485E-3,-2.8756866E-3,7.861712E-3,2.952099E-1,5.3966993E-1,2.8649101E-2,6.109292E-3,2.6475564E-3,-8.014624E-3,-2.4533491E-1,-1.2808783E-1,-2.870228E-1,-5.6748325E-3,-7.7960536E-2,2.1945692E-3,2.1232684E-1,1.8517481E-2,7.0332193E-1,2.1648724E-2,-1.3511151E-1,-2.846796E-1,-1.703479E-1,7.3560804E-2,-1.6021172E-2,-3.7322237E-4,-7.008279E-3,2.102419E-2,4.1629244E-3,2.4756825E-1,3.797352E-3,3.6835376E-2,-7.6138843E-3,8.857409E-4,-7.609898E-3,-1.4919451E-2,-3.2617336E-3,-9.827445E-3,-3.383431E-3,5.9642144E-3,-1.6719104E-3,8.732634E-3,1.5639432E-2,7.0403554E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,47,49,-1,-1,-1,-1,51,53,-1,-1,-1,-1,55,57,59,-1,61,-1,63,-1,65,-1,67,69,71,73,-1,-1,-1,75,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.870997E1,5.6674767E0,3.1802406E0,1.6069307E0,1.646805E0,9.9520355E-1,2.2401314E0,1.7579136E0,0E0,6.075037E-1,2.6024044E-1,4.4999123E-2,5.0566787E-1,1.0649729E0,3.9333534E-1,2.9347134E-1,1.3335233E0,2.8977382E-1,0E0,4.144469E-2,0E0,0E0,0E0,0E0,1.3724807E-1,9.6482754E-1,0E0,7.675648E-2,0E0,0E0,5.7470143E-2,4.9198437E-1,0E0,6.5579236E-2,1.2214139E-1,0E0,0E0,0E0,0E0,2.4245405E-1,1.7390156E-1,0E0,0E0,0E0,0E0,3.2581997E-1,6.1757076E-1,8.590418E-2,0E0,1.6982396E-1,0E0,8.962476E-2,0E0,3.1860113E-1,0E0,8.688891E-2,1.4267778E-1,1.8903792E-1,1.0236327E-1,0E0,0E0,0E0,1.0436742E-1,0E0,1.00809574E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,24,24,25,25,27,27,30,30,31,31,33,33,34,34,39,39,40,40,45,45,46,46,47,47,49,49,51,51,53,53,55,55,56,56,57,57,58,58,62,62,64,64],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,48,50,-1,-1,-1,-1,52,54,-1,-1,-1,-1,56,58,60,-1,62,-1,64,-1,66,-1,68,70,72,74,-1,-1,-1,76,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.7467445E4,6.028226E4,5.825042E6,2.1695537E3,8.169258E3,4.026531E9,7.41718E5,-4.3176156E-2,1.5271514E5,9.825568E6,3.149493E3,2.412E4,3.768034E4,7.443074E0,3.5002026E8,9.256843E3,8.689165E-2,9.367974E-3,1.2678202E9,1.8259488E-2,6.2849447E-3,1.5197571E-2,-1.951503E-2,3.087E3,6.8963E4,-7.656558E-3,6.029086E10,4.6992153E-2,-1.947604E-2,2.3E2,7.8936383E2,3.0819604E-2,6.426437E5,7.10711E8,8.788774E-3,1.8036485E-3,-2.8756866E-3,7.861712E-3,1.2019813E0,5.339615E-1,2.8649101E-2,6.109292E-3,2.6475564E-3,-8.014624E-3,7.2123305E6,2.103E3,3.672717E3,-5.6748325E-3,2.681357E6,2.1945692E-3,2.4390244E-3,1.8517481E-2,7.532E3,2.1648724E-2,5.9531575E6,5.381665E1,1.423795E7,3.7730695E-7,-1.6021172E-2,-3.7322237E-4,-7.008279E-3,6.212394E4,4.1629244E-3,1.7657828E3,3.797352E-3,3.6835376E-2,-7.6138843E-3,8.857409E-4,-7.609898E-3,-1.4919451E-2,-3.2617336E-3,-9.827445E-3,-3.383431E-3,5.9642144E-3,-1.6719104E-3,8.732634E-3,1.5639432E-2,7.0403554E-3],"split_indices":[7,42,42,56,61,42,7,1,0,37,1,4,9,4,62,7,4,47,0,7,0,0,0,0,0,2,0,5,0,0,0,61,0,56,7,0,0,0,0,51,43,0,0,0,0,54,0,4,0,1,0,66,0,9,0,54,57,54,47,0,0,0,37,0,61,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,3.02E2,1.28E2,2.2E2,8.2E1,2.7E1,1.01E2,2.17E2,3E0,5.6E1,2.6E1,1.3E1,1.4E1,7.8E1,2.3E1,6.3E1,1.54E2,4.8E1,8E0,1.6E1,1E1,4E0,9E0,3E0,1.1E1,7.5E1,3E0,1.5E1,8E0,5.8E1,5E0,1.53E2,1E0,1.3E1,3.5E1,1.3E1,3E0,8E0,3E0,4.3E1,3.2E1,1.4E1,1E0,1E0,4E0,8.3E1,7E1,6E0,7E0,2.4E1,1.1E1,2.4E1,1.9E1,9E0,2.3E1,2.3E1,6E1,5.8E1,1.2E1,5E0,1E0,1.4E1,1E1,6E0,1.8E1,1E0,8E0,2E1,3E0,1.1E1,4.9E1,1.5E1,4.3E1,3E0,9E0,8E0,2E0,9E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[8.079336E-3,-1.7830636E-1,3.4637132E-1,-3.6806935E-1,-1.1000999E-1,3.8785776E-1,-5.9803206E-1,-3.9438555E-1,-1.7910372E-1,-1.2416616E-1,1.5866278E-1,3.0766535E-1,7.6156497E-1,-3.807693E-2,-1.238373E-2,-4.0071005E-1,-6.2733386E-3,-1.1011452E-2,-1.7251339E-3,-1.4740914E-1,3.7946574E-2,1.6202858E-2,9.293028E-3,1.9770692E-1,4.5538718E-1,-7.108201E-3,8.4345675E-1,-1.9518834E-2,-3.515145E-3,-1.6080947E-1,9.1117315E-2,-3.4344825E-3,9.042737E-2,3.7415938E-3,-5.190489E-3,2.232489E-1,-4.0920496E-2,3.1082076E-1,5.270325E-1,5.3693984E-2,6.071207E-1,-1.8277577E-1,-4.5234803E-2,-3.841609E-4,9.247603E-3,-5.0521134E-3,1.2137723E-1,1.6241202E-1,2.8471386E-1,-7.8059975E-3,5.8114585E-3,3.3742338E-1,2.2949835E-3,2.626179E-2,7.869617E-3,3.3608027E-2,1.5132527E-2,-2.3600444E-1,-1.2772249E-1,-9.5449E-2,1.430641E-3,9.428857E-3,1.0971656E-3,1.0909238E-1,1.2534224E-2,3.3128697E-3,3.0468988E-1,2.3573693E-3,3.5151982E-1,-4.0970617E-3,5.1915045E-3,-2.4435795E-3,-1.2028931E-2,-8.344234E-3,-2.2868793E-3,5.325753E-3,-5.5555156E-3,1.2630569E-2,3.680793E-3,9.6623146E-4,1.5146561E-2,2.0126672E-2,1.0857969E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,-1,33,35,37,-1,39,-1,-1,41,43,-1,45,-1,-1,47,49,51,53,-1,55,57,59,-1,-1,-1,61,63,65,-1,-1,67,-1,-1,-1,-1,69,71,73,75,-1,-1,-1,77,-1,-1,79,-1,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7868038E1,3.6531448E0,6.3831997E0,3.023281E-1,8.2221365E-1,4.2909355E0,2.5498104E-1,5.6506157E-2,6.951296E-2,7.6295805E-1,2.8833273E-1,1.9566183E0,2.0597553E0,0E0,0E0,5.5392265E-2,0E0,0E0,0E0,5.7720757E-1,1.5541452E-1,0E0,6.427106E-2,4.6016645E-1,3.851347E-1,0E0,8.562565E-1,0E0,0E0,4.1570282E-1,1.01285085E-1,0E0,1.2146816E-1,0E0,0E0,2.0429444E-1,1.7576662E-1,1.23987675E-1,1.2928009E-1,0E0,7.878461E-1,3.8011456E-1,1.0815028E-1,0E0,0E0,0E0,1.1259878E-1,1.5888494E-1,1.1714268E-1,0E0,0E0,5.6024313E-2,0E0,0E0,0E0,0E0,3.6940437E-2,1.621778E-1,2.5024903E-1,8.193876E-2,0E0,0E0,0E0,1.05171174E-1,0E0,0E0,7.171607E-2,0E0,4.753685E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,19,19,20,20,22,22,23,23,24,24,26,26,29,29,30,30,32,32,35,35,36,36,37,37,38,38,40,40,41,41,42,42,46,46,47,47,48,48,51,51,56,56,57,57,58,58,59,59,63,63,66,66,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,-1,34,36,38,-1,40,-1,-1,42,44,-1,46,-1,-1,48,50,52,54,-1,56,58,60,-1,-1,-1,62,64,66,-1,-1,68,-1,-1,-1,-1,70,72,74,76,-1,-1,-1,78,-1,-1,80,-1,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,2.3028242E3,3.5002026E8,1.4777102E9,7.3068325E3,3.1E1,2.9778808E7,2.033552E6,5.9170656E8,3.08031E5,4.1331047E-1,5.0314346E1,-3.807693E-2,-1.238373E-2,3.84142E5,-6.2733386E-3,-1.1011452E-2,-1.7251339E-3,2.9063344E5,3.5212005E6,1.6202858E-2,6.544E3,3.271774E7,1E0,-7.108201E-3,1.417988E6,-1.9518834E-2,-3.515145E-3,7.164074E8,4.037E3,-3.4344825E-3,9E0,3.7415938E-3,-5.190489E-3,2.0663544E9,1.2538845E9,1.5631589E9,5.848343E7,5.3693984E-2,4.42384E5,1.947899E6,9.470162E-2,-3.841609E-4,9.247603E-3,-5.0521134E-3,7.753E3,2.1597655E9,3.8952081E0,-7.8059975E-3,5.8114585E-3,1.872503E4,2.2949835E-3,2.626179E-2,7.869617E-3,3.3608027E-2,1.379E3,1.8460208E8,1.0623282E3,4.138E3,1.430641E-3,9.428857E-3,1.0971656E-3,5.3888105E6,1.2534224E-2,3.3128697E-3,4.264618E6,2.3573693E-3,4.5709072E1,-4.0970617E-3,5.1915045E-3,-2.4435795E-3,-1.2028931E-2,-8.344234E-3,-2.2868793E-3,5.325753E-3,-5.5555156E-3,1.2630569E-2,3.680793E-3,9.6623146E-4,1.5146561E-2,2.0126672E-2,1.0857969E-2],"split_indices":[2,1,67,7,7,61,3,12,9,12,38,44,65,0,0,38,0,0,0,37,57,0,10,54,109,0,38,0,0,7,10,0,8,0,0,12,7,41,1,0,2,1,47,0,0,0,2,7,62,0,0,42,0,0,0,0,0,5,61,2,0,0,0,41,0,0,1,0,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,2.84E2,1.56E2,7.4E1,2.1E2,1.5E2,6E0,6.4E1,1E1,2E2,1E1,1.25E2,2.5E1,3E0,3E0,6.2E1,2E0,7E0,3E0,1.75E2,2.5E1,4E0,6E0,7.3E1,5.2E1,2E0,2.3E1,6.1E1,1E0,1.66E2,9E0,8E0,1.7E1,4E0,2E0,6.6E1,7E0,1.9E1,3.3E1,9E0,1.4E1,1.39E2,2.7E1,5E0,4E0,2E0,1.5E1,3.5E1,3.1E1,4E0,3E0,1.7E1,2E0,3.1E1,2E0,1.2E1,2E0,6.9E1,7E1,1.6E1,1.1E1,8E0,7E0,2.4E1,1.1E1,3E0,2.8E1,1E0,1.6E1,1E0,1E0,5E0,6.4E1,4.4E1,2.6E1,1E0,1.5E1,3E0,2.1E1,1E0,2.7E1,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[-2.0808099E-2,-1.633379E-1,2.8346893E-1,-3.352381E-1,-1.05321445E-1,-5.000978E-1,3.3398113E-1,-3.4900728E-1,-6.8168733E-6,-1.5051822E-1,3.618505E-2,1.6985567E-2,-6.5068036E-1,2.2412908E-1,4.6982473E-1,-1.7642738E-2,-1.0630001E-2,-2.44318E-1,-8.5936464E-2,-4.3584774E-3,2.2312112E-1,-3.9514847E-2,-1.3352743E-2,1.07650466E-1,2.4428679E-1,-6.796905E-2,5.2087915E-1,-3.6844842E-3,-2.743265E-1,-2.7837746E-2,-1.7819846E-1,-6.2321182E-2,8.4340304E-2,-3.1726568E-3,1.2658357E-2,-1.7571494E-3,1.1396587E-2,2.808123E-1,1.872964E-1,-7.7701723E-3,3.805849E-3,4.5390648E-1,3.7134364E-2,-1.9044292E-1,-3.3040434E-1,4.0451333E-2,-1.08856715E-1,-1.9666292E-1,-1.966699E-3,-1.0691607E-1,2.3460966E-3,-1.4520946E-3,1.5361649E-1,3.1155348E-1,8.035462E-3,7.247461E-3,1.3344938E-2,4.860099E-1,3.3439547E-3,-2.3288459E-1,-4.785324E-3,-1.2815478E-1,-3.6085582E-1,1.4609458E-1,-6.1760753E-2,-1.3532403E-1,9.434185E-4,-1.0159545E-2,-3.3868027E-3,-6.0783032E-2,-1.1686824E-2,9.108796E-3,1.7254888E-3,1.540779E-2,2.3005381E-3,5.880839E-1,1.8134926E-2,-7.5976863E-3,1.1718993E-2,-1.27460975E-2,-5.0378744E-3,-8.500472E-3,2.7138772E-3,-9.163612E-3,-1.8732881E-2,1.9040679E-3,8.874547E-3,-6.929997E-3,5.26436E-4,-3.3665206E-3,-1.0238318E-2,4.907575E-3,-4.960883E-3,2.987657E-2,9.773961E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,-1,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,49,51,-1,-1,-1,-1,53,55,-1,-1,57,-1,59,61,63,65,67,-1,69,-1,-1,71,73,-1,-1,-1,75,77,79,-1,81,83,85,87,89,-1,-1,-1,91,-1,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9301937E1,2.9865236E0,5.7873363E0,3.506136E-1,1.4636519E0,1.3844035E0,1.8943405E0,7.794571E-2,0E0,1.0269952E0,4.2539912E-1,0E0,2.7792025E-1,1.5214586E-1,1.6553516E0,0E0,0E0,3.3173037E-1,5.525108E-1,2.4689358E-1,1.3114995E-1,0E0,0E0,2.4972576E-1,8.062768E-2,9.4854735E-2,5.5197525E-1,0E0,2.1461391E-1,3.6402962E-1,9.3465686E-2,1.4872651E-1,1.5527984E-1,0E0,0E0,0E0,0E0,7.792187E-2,5.1056027E-2,0E0,0E0,6.492815E-1,0E0,7.7519E-2,1.6315365E-1,3.977515E-1,1.0461837E-1,4.7350764E-2,0E0,1.1650872E-1,0E0,0E0,4.315242E-2,4.3887854E-2,0E0,0E0,0E0,2.4992466E-1,1.9259503E-1,4.107082E-2,0E0,6.320297E-2,6.319165E-2,6.5579355E-2,1.1475857E-1,1.1129689E-1,0E0,0E0,0E0,1.2683731E-1,0E0,0E0,0E0,0E0,0E0,1.1761618E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,37,37,38,38,41,41,43,43,44,44,45,45,46,46,47,47,49,49,52,52,53,53,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,69,69,75,75],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,-1,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,50,52,-1,-1,-1,-1,54,56,-1,-1,58,-1,60,62,64,66,68,-1,70,-1,-1,72,74,-1,-1,-1,76,78,80,-1,82,84,86,88,90,-1,-1,-1,92,-1,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.700398E4,1.2852292E7,7.10711E8,1.3812E4,6.8963E4,2.9778808E7,-6.8168733E-6,7.283496E2,3.4491902E3,1.6985567E-2,3.1E1,2.8529238E1,2.943086E4,-1.7642738E-2,-1.0630001E-2,5.1698097E1,2.474199E1,3.2318832E7,8.287E3,-3.9514847E-2,-1.3352743E-2,1.5271514E5,1.16E2,1.6130411E5,1.3861362E8,-3.6844842E-3,1.0428788E7,1.6777562E7,1.4205709E0,6.136946E5,9.072834E8,-3.1726568E-3,1.2658357E-2,-1.7571494E-3,1.1396587E-2,7.201163E1,2.5096E4,-7.7701723E-3,3.805849E-3,5.37E2,3.7134364E-2,3.183E3,6.465659E4,4.7232704E1,6.0070217E1,1.2598097E5,-1.966699E-3,1.00999994E9,2.3460966E-3,-1.4520946E-3,1.123E3,5.062077E-1,8.035462E-3,7.247461E-3,1.3344938E-2,5.3699964E7,3.087E3,1.576081E6,-4.785324E-3,1.249E3,3.640932E2,2.681357E6,1.402368E3,1.656968E6,9.434185E-4,-1.0159545E-2,-3.3868027E-3,2.2893274E1,-1.1686824E-2,9.108796E-3,1.7254888E-3,1.540779E-2,2.3005381E-3,8.65124E5,1.8134926E-2,-7.5976863E-3,1.1718993E-2,-1.27460975E-2,-5.0378744E-3,-8.500472E-3,2.7138772E-3,-9.163612E-3,-1.8732881E-2,1.9040679E-3,8.874547E-3,-6.929997E-3,5.26436E-4,-3.3665206E-3,-1.0238318E-2,4.907575E-3,-4.960883E-3,2.987657E-2,9.773961E-3],"split_indices":[2,1,42,9,7,9,2,12,0,61,4,0,3,65,42,0,0,57,62,9,2,0,0,37,3,37,53,0,54,54,48,56,7,0,0,0,0,65,2,0,0,3,0,10,37,67,65,37,0,5,0,0,0,66,0,0,0,53,0,1,0,0,42,1,61,9,0,0,0,62,0,0,0,0,0,11,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,3.02E2,1.41E2,7.5E1,2.27E2,8E0,1.33E2,7.2E1,3E0,1.72E2,5.5E1,1E0,7E0,7.5E1,5.8E1,6.2E1,1E1,6.9E1,1.03E2,4.6E1,9E0,4E0,3E0,1.2E1,6.3E1,5E0,5.3E1,1.1E1,5.8E1,6.4E1,3.9E1,2.8E1,1.8E1,1E0,8E0,6E0,6E0,3.6E1,2.7E1,3E0,2E0,4.4E1,9E0,2.5E1,3.3E1,3.5E1,2.9E1,3.4E1,5E0,2E1,8E0,7E0,1.1E1,2.7E1,9E0,2.1E1,6E0,4.1E1,3E0,1.6E1,9E0,5E0,2.8E1,1.7E1,1.8E1,2.4E1,5E0,3E1,4E0,1.6E1,4E0,8E0,3E0,2.6E1,1E0,1.9E1,2.2E1,2E0,1E0,1.2E1,4E0,4E0,1E0,5E0,2.3E1,5E0,1.2E1,8E0,1E1,1.4E1,1E1,3E0,1.3E1,1.7E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[5.127347E-3,-1.3262297E-1,3.5060835E-1,-3.41737E-1,-7.4711435E-2,4.2321807E-1,-4.0736234E-1,-3.5678047E-1,-1.1151939E-3,-1.0535834E-1,1.6755556E-1,3.278886E-1,8.1161964E-1,-6.8112516E-1,5.737806E-2,-2.204295E-2,-1.5468438E-2,-1.1789662E-1,4.3479794E-1,2.352313E-1,2.9253768E-4,2.3601766E-1,4.4739574E-1,4.7542334E-2,2.263804E-2,-3.7990656E-2,-1.3311415E-2,2.6023433E-1,-7.936544E-3,-1.5221311E-1,5.520072E-3,4.533485E-3,2.3938484E-2,2.6330996E-1,-6.5135914E-3,1.9846775E-1,2.0367732E-2,9.0833735E-3,4.6421373E-1,2.2450762E-3,1.6576078E-2,-1.6538376E-1,-1.2115101E-2,-8.490566E-2,7.229765E-2,1.5482409E-2,7.277082E-3,-8.662911E-4,2.1398567E-1,5.22826E-1,1.5660834E-2,-1.7605734E-1,1.902565E-2,-4.1658515E-3,3.0705878E-3,-1.292716E-1,3.5702167E-3,-7.2515947E-3,1.4668278E-1,1.1407883E-2,4.121174E-3,2.5851047E-2,5.5366606E-3,-1.02597205E-2,-6.817143E-3,-2.4498033E-3,4.1010138E-3,-8.7760445E-3,-2.592846E-3,-3.1478745E-3,2.1425162E-3,9.3932925E-3,-1.0779971E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,-1,-1,-1,-1,39,-1,41,43,-1,-1,45,-1,47,-1,-1,49,-1,-1,51,53,55,57,-1,-1,-1,59,61,-1,63,65,-1,-1,67,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.003435E1,3.6196122E0,6.7767267E0,3.0329084E-1,1.7773154E0,3.8067513E0,1.4385965E0,7.718372E-2,0E0,1.4802856E0,2.9365152E-1,8.8253784E-1,7.900467E-1,8.528924E-2,2.6798815E-1,0E0,0E0,8.775325E-1,5.842483E-2,2.3313642E-1,0E0,2.8714442E-1,7.807398E-2,0E0,0E0,0E0,0E0,3.7551597E-2,0E0,2.9691577E-1,2.8390545E-1,0E0,0E0,6.8451285E-2,0E0,1.5834105E-1,0E0,0E0,1.1616707E-1,0E0,0E0,2.9465914E-1,8.996326E-2,1.5061325E-1,1.6083024E-1,0E0,0E0,0E0,1.04744196E-1,6.348038E-2,0E0,1.481905E-1,4.5821924E-2,0E0,0E0,5.392316E-2,0E0,4.489695E-2,1.18269116E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,27,27,29,29,30,30,33,33,35,35,38,38,41,41,42,42,43,43,44,44,48,48,49,49,51,51,52,52,55,55,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,-1,-1,-1,-1,40,-1,42,44,-1,-1,46,-1,48,-1,-1,50,-1,-1,52,54,56,58,-1,-1,-1,60,62,-1,64,66,-1,-1,68,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0429407E3,1.2852292E7,1.4096699E9,7.7101436E3,3.5226266E0,1.44E2,-1.1151939E-3,3.058642E3,5.063299E10,6.050605E9,5.192243E6,3.1E1,1.872503E4,-2.204295E-2,-1.5468438E-2,2.8884522E8,1.563E3,2.42E2,2.9253768E-4,2.6421343E9,8.825362E-2,4.7542334E-2,2.263804E-2,-3.7990656E-2,-1.3311415E-2,3.498E3,-7.936544E-3,3.94E2,1.656968E6,4.533485E-3,2.3938484E-2,6.3349745E6,-6.5135914E-3,2.8529238E1,2.0367732E-2,9.0833735E-3,8.080874E-1,2.2450762E-3,1.6576078E-2,2.889654E5,2.6452028E7,1.4998456E5,2.972948E6,1.5482409E-2,7.277082E-3,-8.662911E-4,1.962918E7,6.029086E10,1.5660834E-2,6.8984265E2,2.455358E6,-4.1658515E-3,3.0705878E-3,8.9E1,3.5702167E-3,1.2022638E0,1.6130304E7,1.1407883E-2,4.121174E-3,2.5851047E-2,5.5366606E-3,-1.02597205E-2,-6.817143E-3,-2.4498033E-3,4.1010138E-3,-8.7760445E-3,-2.592846E-3,-3.1478745E-3,2.1425162E-3,9.3932925E-3,-1.0779971E-3],"split_indices":[2,1,67,9,7,61,63,0,0,61,40,12,38,3,42,0,0,12,0,3,0,7,48,0,0,0,0,0,0,8,9,0,0,54,0,65,0,0,43,0,0,37,12,37,1,0,0,0,54,5,0,61,1,0,0,8,0,50,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,3E2,1.19E2,6.4E1,2.36E2,1.09E2,1E1,6.1E1,3E0,2.1E2,2.6E1,8.9E1,2E1,6E0,4E0,1.3E1,4.8E1,2.06E2,4E0,1.8E1,8E0,5.2E1,3.7E1,1.2E1,8E0,4E0,2E0,2E0,2E0,1.61E2,4.5E1,1E0,3E0,1.7E1,1E0,4.5E1,7E0,3E0,3.4E1,1E0,1E0,1.47E2,1.4E1,1.9E1,2.6E1,1E1,7E0,3E0,4.2E1,2.2E1,1.2E1,1.39E2,8E0,7E0,7E0,1.5E1,4E0,1.3E1,1.3E1,3.5E1,7E0,2.1E1,1E0,6.5E1,7.4E1,4E0,4E0,8E0,7E0,6E0,7E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[2.7156705E-3,-1.5303582E-1,2.79257E-1,-3.24133E-1,-9.022098E-2,2.1166614E-1,6.8132555E-1,-3.300803E-1,2.6523673E-3,-1.07351236E-1,1.9280909E-1,2.5429624E-1,-3.466397E-1,7.83818E-1,2.415502E-2,-3.8032964E-1,-1.3201006E-2,-1.8991637E-1,-6.2947966E-2,1.2695651E-2,3.7813426E-3,1.3389002E-1,3.550267E-1,-6.2505656E-1,5.6788452E-2,9.618983E-1,2.0188987E-2,-6.2075183E-3,1.1640591E-2,-3.879795E-1,-2.7141573E-3,-2.0028579E-1,1.1317935E-3,-9.453636E-2,1.2158267E-1,6.2124956E-2,1.8483125E-1,2.6675686E-1,4.354507E-1,-3.5543863E-2,-1.2883535E-2,1.22449435E-2,-7.6814317E-3,1.3005922E-2,4.823035E-2,-1.9077472E-2,-3.0517902E-3,-1.19385704E-1,-2.3701873E-1,-5.782788E-2,-1.81059E-1,2.0727752E-1,-1.0683408E-3,-5.4542787E-2,5.206282E-3,2.081751E-1,-6.0646064E-3,2.7910313E-1,2.0264718E-3,4.4573966E-1,2.3123368E-3,-7.938872E-3,-9.4441575E-4,-2.6489583E-1,-6.3568004E-3,-1.7186787E-2,-9.600725E-2,-2.1381916E-1,-3.547771E-3,5.1349434E-3,1.2731321E-2,4.123198E-3,-5.5680396E-3,1.276306E-2,1.05190486E-1,4.155514E-3,2.9552203E-1,6.5734778E-3,2.2242168E-2,-1.398144E-2,-6.2833675E-3,-1.074692E-2,-2.0982136E-6,-7.479795E-3,-1.4043702E-3,-1.1682065E-2,-4.244744E-3,-2.3283674E-3,5.8825715E-3,1.4708724E-2,4.565006E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,-1,35,37,39,41,43,-1,-1,-1,45,-1,47,-1,49,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,61,63,65,67,69,-1,71,-1,73,-1,75,-1,77,-1,-1,-1,79,-1,81,83,85,-1,-1,-1,-1,-1,-1,87,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8649723E1,2.9477978E0,4.0952673E0,1.8501902E-1,1.0097675E0,3.3012352E0,1.4628763E0,1.0235548E-1,0E0,6.956146E-1,7.6117635E-2,1.4806652E0,1.1587081E0,8.32551E-1,1.6396461E-1,6.7399025E-2,0E0,1.529541E-1,7.517803E-1,0E0,0E0,2.0420969E-1,3.6657333E-1,4.3765068E-2,2.5350544E-1,4.9129486E-2,0E0,0E0,0E0,6.2322617E-2,0E0,1.6189218E-1,0E0,3.3572614E-1,2.3863062E-1,1.4497976E-1,2.7271366E-1,8.537841E-2,1.14177704E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.616122E-2,9.115505E-2,1.19378746E-1,9.693909E-2,4.0534317E-2,0E0,7.815494E-2,0E0,1.5845466E-1,0E0,7.905793E-2,0E0,9.602022E-2,0E0,0E0,0E0,7.359266E-2,0E0,1.3752192E-1,1.5365142E-1,6.4607024E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.944038E-2,0E0,7.895875E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,21,21,22,22,23,23,24,24,25,25,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,47,47,48,48,49,49,50,50,51,51,53,53,55,55,57,57,59,59,63,63,65,65,66,66,67,67,74,74,76,76],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,-1,36,38,40,42,44,-1,-1,-1,46,-1,48,-1,50,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,62,64,66,68,70,-1,72,-1,74,-1,76,-1,78,-1,-1,-1,80,-1,82,84,86,-1,-1,-1,-1,-1,-1,88,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,7.7101436E3,2.91E2,1.4777102E9,9.05929E2,1.92E2,4.90403E5,2.6523673E-3,6.505339E2,8.749E3,2.5096E4,3.5226266E0,5.192243E6,3.677E3,1.6395413E7,-1.3201006E-2,3.01E2,2.251E3,1.2695651E-2,3.7813426E-3,5.5785873E1,1.3047821E5,3.1E1,1.872503E4,5.3412586E4,2.0188987E-2,-6.2075183E-3,1.1640591E-2,8.95654E8,-2.7141573E-3,6.1453125E1,1.1317935E-3,2.515304E7,1.6777562E7,1.2009271E3,4.2663252E7,1.0014992E8,8.759649E7,-3.5543863E-2,-1.2883535E-2,1.22449435E-2,-7.6814317E-3,1.3005922E-2,4.823035E-2,-1.9077472E-2,-3.0517902E-3,1.0038E4,9.457892E6,7.581957E1,1.530655E3,2.1592189E5,-1.0683408E-3,1.0721749E3,5.206282E-3,1.6225381E7,-6.0646064E-3,2E0,2.0264718E-3,8.946923E2,2.3123368E-3,-7.938872E-3,-9.4441575E-4,4.52E2,-6.3568004E-3,2.9E1,6.794917E8,5.4779566E9,-3.547771E-3,5.1349434E-3,1.2731321E-2,4.123198E-3,-5.5680396E-3,1.276306E-2,8.42E2,4.155514E-3,5.0772004E9,6.5734778E-3,2.2242168E-2,-1.398144E-2,-6.2833675E-3,-1.074692E-2,-2.0982136E-6,-7.479795E-3,-1.4043702E-3,-1.1682065E-2,-4.244744E-3,-2.3283674E-3,5.8825715E-3,1.4708724E-2,4.565006E-4],"split_indices":[2,1,61,8,7,67,8,9,0,61,2,2,63,38,0,12,0,8,0,0,0,65,37,3,42,37,0,0,0,5,0,57,0,54,54,61,54,1,1,0,0,0,0,0,0,0,0,0,41,63,61,37,0,4,0,57,0,8,0,61,0,0,0,3,0,3,7,5,0,0,0,0,0,0,0,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.76E2,1.55E2,7.3E1,2.03E2,1.34E2,2.1E1,7.2E1,1E0,1.92E2,1.1E1,1.25E2,9E0,1.8E1,3E0,3.5E1,3.7E1,6.6E1,1.26E2,6E0,5E0,5.8E1,6.7E1,5E0,4E0,1.1E1,7E0,2E0,1E0,3.4E1,1E0,6.3E1,3E0,1.08E2,1.8E1,2.5E1,3.3E1,3.4E1,3.3E1,3E0,2E0,2E0,2E0,1E0,1E1,3.3E1,1E0,2.1E1,4.2E1,7.7E1,3.1E1,1.1E1,7E0,7E0,1.8E1,3.1E1,2E0,3.2E1,2E0,3.2E1,1E0,1.4E1,7E0,3.2E1,1E1,3.8E1,3.9E1,2.3E1,8E0,5E0,6E0,2E0,5E0,1.9E1,1.2E1,3E0,2.9E1,2E0,3E1,2.6E1,6E0,2E0,3.6E1,2E1,1.9E1,1.8E1,5E0,1E0,1.1E1,2.8E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[6.7140316E-3,-1.4423268E-1,2.5948542E-1,-3.0860072E-1,-9.242332E-2,3.090736E-1,-2.9035118E-1,-3.24369E-1,6.2477996E-4,-1.144356E-1,1.8025205E-1,2.3783712E-1,6.5574247E-1,-4.1941617E-2,-3.5154227E-2,-1.758601E-2,-2.5003865E-1,-1.4948994E-1,-1.616904E-3,-2.9473177E-3,2.4065712E-1,1.2050409E-1,3.426777E-1,-9.14954E-3,7.375151E-1,-1.9375388E-1,1.1292903E-2,-1.2497269E-2,-7.3080073E-4,-1.7942767E-1,-9.244255E-2,-5.1041044E-2,9.733243E-2,1.53384E-2,6.210964E-3,1.5653327E-1,-4.20928E-4,2.770242E-1,3.999365E-1,4.577568E-2,2.5723033E-2,-1.033015E-3,-1.245902E-2,-2.066956E-1,-4.9783543E-2,-1.5593386E-2,-1.4647765E-1,-1.009935E-1,2.9735126E-2,9.398403E-3,1.928562E-2,2.0656343E-1,5.6093946E-2,-1.1052396E-2,1.0362016E-1,4.005242E-3,2.9311445E-1,2.4475837E-1,4.4793004E-1,-2.3587155E-1,-1.5117916E-1,4.8018303E-3,-7.4679635E-2,-7.939338E-2,1.1395577E-1,-7.4456335E-4,-2.060619E-1,-1.2229343E-1,7.1261043E-4,-2.9088093E-3,3.6869145E-3,-1.5070557E-3,7.118394E-3,1.3539673E-1,2.5043955E-1,-4.9861033E-2,5.709748E-3,6.8050385E-3,1.2930937E-4,3.0226895E-1,4.4549504E-4,1.4349647E-2,4.3912427E-3,7.5147026E-3,4.67398E-1,-1.18012065E-2,-7.2675955E-4,-9.368607E-3,-2.1936514E-3,-5.2254153E-3,1.0186047E-3,2.8103893E-4,-5.7213297E-3,8.622252E-3,2.111485E-4,4.883904E-3,-4.1297534E-3,-1.7565036E-3,-1.1655427E-2,-6.875562E-3,9.643673E-4,-1.82583E-3,7.31135E-3,1.2677498E-2,9.1782364E-4,-5.3487523E-3,3.3050312E-3,1.593002E-2,8.854355E-3,2.3636386E-2,7.688266E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,27,29,31,-1,33,35,37,-1,39,41,-1,-1,-1,43,45,47,49,-1,-1,51,53,55,57,-1,-1,-1,-1,59,61,63,65,67,69,-1,71,73,75,-1,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,-1,-1,-1,101,103,105,-1,-1,-1,107,-1,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6634855E1,2.2962918E0,4.5339985E0,3.3378172E-1,1.2750883E0,3.5483618E0,1.4952724E0,9.482622E-2,0E0,7.669866E-1,2.4799961E-1,1.5086656E0,1.8684292E0,4.6462005E-1,0E0,0E0,4.9831152E-2,2.3617768E-1,2.3482075E-1,0E0,7.1646154E-2,2.6583236E-1,1.3731766E-1,0E0,4.8770905E-1,7.244739E-2,0E0,0E0,0E0,3.2908225E-1,2.1780124E-1,1.3212201E-1,1.1810723E-1,0E0,0E0,2.2458792E-1,3.8083124E-1,8.175802E-2,1.3613462E-1,0E0,0E0,0E0,0E0,9.230113E-2,7.4376136E-2,1.9914775E-1,2.6903182E-1,5.1126614E-2,5.932799E-2,0E0,6.9485314E-2,6.288934E-2,1.18169114E-1,0E0,4.1276984E-2,0E0,7.776809E-2,5.4122925E-2,6.361961E-2,9.441066E-2,1.2789047E-1,0E0,5.394125E-2,5.423464E-2,5.6082785E-2,9.531198E-2,1.2431842E-1,5.197242E-2,0E0,0E0,0E0,0E0,0E0,4.5137912E-2,4.7028422E-2,5.8224015E-2,0E0,0E0,0E0,3.9403915E-2,0E0,0E0,0E0,0E0,7.644272E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,20,20,21,21,22,22,24,24,25,25,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52,54,54,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,73,73,74,74,75,75,79,79,84,84],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,28,30,32,-1,34,36,38,-1,40,42,-1,-1,-1,44,46,48,50,-1,-1,52,54,56,58,-1,-1,-1,-1,60,62,64,66,68,70,-1,72,74,76,-1,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,-1,-1,-1,102,104,106,-1,-1,-1,108,-1,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.508E3,7.41718E5,9.05929E2,1.2852292E7,1.4777102E9,7.7101436E3,7.339209E6,1.425966E6,6.2477996E-4,9.7467445E4,2.0532622E3,2.5096E4,5.0314346E1,3.236383E1,-3.5154227E-2,-1.758601E-2,8.32E3,2.735322E6,3.23643E6,-2.9473177E-3,6.088569E6,1.99218E7,1.3047821E5,-9.14954E-3,1.681178E6,3.7137984E7,1.1292903E-2,-1.2497269E-2,-7.3080073E-4,7.164074E8,1.7377544E7,6.2581446E8,1.473681E6,1.53384E-2,6.210964E-3,5.1063637E1,6.426437E5,2E0,5.3515047E-1,4.577568E-2,2.5723033E-2,-1.033015E-3,-1.245902E-2,1.3219183E5,4.138E3,5.16152E2,2.7E1,9.216E3,1.0368186E-1,9.398403E-3,1.685028E6,1.227E4,7.10711E8,-1.1052396E-2,2.1172E4,4.005242E-3,5.0772004E9,2.724043E6,4.1322E4,2.76E2,1.6845247E3,4.8018303E-3,1.6161358E5,1.72232E5,7.234083E7,1.6991062E3,1.792E3,1.2009812E9,7.1261043E-4,-2.9088093E-3,3.6869145E-3,-1.5070557E-3,7.118394E-3,1.7102936E6,2.314E4,1.0336E4,5.709748E-3,6.8050385E-3,1.2930937E-4,1.1356496E12,4.4549504E-4,1.4349647E-2,4.3912427E-3,7.5147026E-3,5.825042E6,-1.18012065E-2,-7.2675955E-4,-9.368607E-3,-2.1936514E-3,-5.2254153E-3,1.0186047E-3,2.8103893E-4,-5.7213297E-3,8.622252E-3,2.111485E-4,4.883904E-3,-4.1297534E-3,-1.7565036E-3,-1.1655427E-2,-6.875562E-3,9.643673E-4,-1.82583E-3,7.31135E-3,1.2677498E-2,9.1782364E-4,-5.3487523E-3,3.3050312E-3,1.593002E-2,8.854355E-3,2.3636386E-2,7.688266E-3],"split_indices":[2,1,67,9,7,61,56,9,0,42,4,2,65,65,0,0,2,1,1,0,54,54,37,0,38,1,0,0,0,7,54,7,38,0,0,67,56,8,44,0,0,0,0,37,2,61,8,10,47,0,9,2,7,0,10,0,41,38,2,8,4,0,56,7,7,42,11,41,0,0,0,0,0,54,2,10,0,0,0,40,0,0,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,2.72E2,1.62E2,6.4E1,2.08E2,1.49E2,1.3E1,6.1E1,3E0,1.93E2,1.5E1,1.25E2,2.4E1,9E0,4E0,3.7E1,2.4E1,1.47E2,4.6E1,3E0,1.2E1,6E1,6.5E1,2E0,2.2E1,6E0,3E0,2.3E1,1E0,9.5E1,5.2E1,3.1E1,1.5E1,6E0,6E0,4.6E1,1.4E1,3.3E1,3.2E1,9E0,1.3E1,2E0,4E0,7.8E1,1.7E1,2.2E1,3E1,1.9E1,1.2E1,6E0,9E0,3E1,1.6E1,4E0,1E1,3E0,3E1,9E0,2.3E1,4.9E1,2.9E1,2E0,1.5E1,1.5E1,7E0,9E0,2.1E1,1.6E1,3E0,4E0,8E0,7E0,2E0,1.3E1,1.7E1,6E0,1E1,7E0,3E0,2.9E1,1E0,6E0,3E0,2E0,2.1E1,4.7E1,2E0,2E1,9E0,1.1E1,4E0,5E0,1E1,4E0,3E0,4E0,5E0,4E0,1.7E1,1.4E1,2E0,1E0,1.2E1,1.6E1,1E0,4E0,2E0,2.2E1,7E0,1.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-9.600974E-3,-1.1764853E-1,2.8247464E-1,-1.7945826E-1,4.4621326E-2,2.0807458E-1,6.477912E-1,-1.693935E-1,-3.4878563E-2,-4.0463634E-2,2.3750763E-1,2.4546397E-1,-1.612432E-2,2.126134E-2,3.611041E-2,-3.251256E-1,-1.3263115E-1,-6.630062E-2,8.649684E-3,1.8763308E-1,2.8475571E-2,1.7914078E-1,3.6113742E-1,-1.7462926E-2,-1.6291659E-1,-1.3622014E-1,1.3241906E-2,-8.3619505E-3,-4.036952E-2,4.7633685E-3,1.237826E-2,-4.2503774E-3,1.9397469E-1,3.7842733E-1,6.598647E-3,-1.0284011E-2,1.2401202E-3,-1.4370148E-1,7.989248E-2,-1.8196758E-2,-1.6763903E-1,1.4230289E-1,1.3344125E-2,3.4535483E-3,1.8624192E-2,-1.6268824E-1,-4.081319E-2,5.9326174E-4,7.99944E-3,-4.4641092E-2,5.1634163E-2,-3.6332298E-3,-1.2804646E-2,1.1435127E-2,9.656546E-2,-8.840533E-3,-3.2102605E-3,-4.9597113E-3,3.022714E-4,-5.551609E-4,-4.3212837E-3,7.3965765E-3,3.208414E-5,2.3040625E-3,8.65906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,27,-1,29,-1,31,33,-1,35,37,-1,-1,39,-1,-1,-1,41,43,-1,-1,-1,45,47,49,51,53,-1,-1,-1,55,57,-1,-1,59,61,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4139684E1,3.2877712E0,3.1539392E0,1.1749129E0,1.5030773E0,2.1510053E0,8.812618E-2,1.2965465E0,0E0,3.7544462E-1,3.8225424E-1,6.718011E-1,0E0,0E0,0E0,2.0547152E-1,3.1659222E-1,1.5411755E-1,0E0,1.3423121E-1,0E0,2.668953E-1,8.005476E-2,0E0,1.0051012E-1,3.1867552E-1,0E0,0E0,1.324015E-1,0E0,0E0,0E0,2.1726894E-1,4.6889305E-2,0E0,0E0,0E0,3.5280657E-1,3.8654383E-2,7.9863556E-2,4.325117E-2,1.4630914E-1,0E0,0E0,0E0,2.9562807E-1,8.829059E-2,0E0,0E0,4.5162313E-2,6.2143665E-2,0E0,0E0,0E0,1.0481325E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,17,17,19,19,21,21,22,22,24,24,25,25,28,28,32,32,33,33,37,37,38,38,39,39,40,40,41,41,45,45,46,46,49,49,50,50,54,54],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,28,-1,30,-1,32,34,-1,36,38,-1,-1,40,-1,-1,-1,42,44,-1,-1,-1,46,48,50,52,54,-1,-1,-1,56,58,-1,-1,60,62,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.7467445E4,7.765071E0,2.2233E4,2.1695537E3,1.4609149E4,1.6828056E1,4.66085E5,-3.4878563E-2,1.5271514E5,3.3056E4,6.117756E9,-1.612432E-2,2.126134E-2,3.611041E-2,2.1915232E8,1.3664E4,1.6826648E3,8.649684E-3,1.757E3,2.8475571E-2,1.5560601E6,5.30113E5,-1.7462926E-2,1E2,2.9063344E5,1.3241906E-2,-8.3619505E-3,1E0,4.7633685E-3,1.237826E-2,-4.2503774E-3,6.929085E-1,4.585483E6,6.598647E-3,-1.0284011E-2,1.2401202E-3,6.4065235E8,2.8182E4,3.15584E6,2.865605E8,1.648393E1,1.3344125E-2,3.4535483E-3,1.8624192E-2,2.822E3,3.738719E4,5.9326174E-4,7.99944E-3,7.03764E5,7.25E3,-3.6332298E-3,-1.2804646E-2,1.1435127E-2,4.850336E6,-8.840533E-3,-3.2102605E-3,-4.9597113E-3,3.022714E-4,-5.551609E-4,-4.3212837E-3,7.3965765E-3,3.208414E-5,2.3040625E-3,8.65906E-3],"split_indices":[7,42,44,2,61,4,44,1,0,37,2,12,0,0,0,7,2,4,0,0,0,54,2,0,3,37,0,0,18,0,0,0,48,1,0,0,0,7,9,1,7,67,0,0,0,0,37,0,0,38,2,0,0,0,38,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,3.26E2,1.2E2,2.36E2,9E1,1.01E2,1.9E1,2.33E2,3E0,6.3E1,2.7E1,9.5E1,6E0,8E0,1.1E1,4.3E1,1.9E2,5.7E1,6E0,2.5E1,2E0,6.2E1,3.3E1,3.4E1,9E0,1.89E2,1E0,1E1,4.7E1,1.2E1,1.3E1,3E0,5.9E1,3E1,3E0,7E0,2E0,1.83E2,6E0,4.1E1,6E0,3.8E1,2.1E1,1E0,2.9E1,1.54E2,2.9E1,4E0,2E0,3E1,1.1E1,4E0,2E0,1.1E1,2.7E1,1.26E2,2.8E1,1.2E1,1.7E1,1.8E1,1.2E1,3E0,8E0,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-7.0573245E-3,-1.2877052E-1,2.6929203E-1,-2.720917E-1,-8.287238E-2,3.1972867E-1,-3.3750328E-1,-2.8429714E-1,7.535864E-4,-1.14770964E-1,6.7960806E-2,2.3768832E-1,5.8877236E-1,7.805366E-2,-5.8611494E-1,-2.9932433E-1,-8.882214E-3,-1.34742E-1,5.807063E-2,1.676142E-2,1.808855E-1,1.19794965E-1,3.0028945E-1,-8.337139E-3,6.515977E-1,-7.075428E-3,1.334781E-2,-3.263372E-2,-1.1550625E-2,-1.4594696E-2,-2.3411268E-3,-1.4701973E-1,4.5305297E-2,-7.3922356E-3,1.0181912E-1,-8.8996366E-2,7.409013E-2,1.3459464E-2,2.737719E-3,2.2416912E-1,5.8267504E-2,3.191979E-1,4.856363E-3,2.3451509E-2,4.4743396E-2,-2.9742712E-2,-1.620781E-1,-4.818664E-3,1.0482451E-1,1.3349421E-1,-2.0451862E-3,-9.331088E-3,-7.800238E-5,-5.439661E-4,1.2092751E-1,1.3993818E-2,6.079149E-3,-4.4446755E-3,8.415005E-2,1.135859E-2,3.6375535E-1,3.4413957E-3,-2.8680263E-3,-1.9364978E-1,-9.1455825E-2,4.904808E-4,7.254328E-3,1.082668E-3,7.6847933E-3,7.0538446E-3,-5.980293E-4,1.9199243E-3,6.2165717E-3,3.8520262E-1,4.622775E-3,-1.0922399E-2,-6.981419E-3,-5.228882E-4,-8.44262E-3,1.9954874E-3,1.9028584E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,45,47,-1,49,51,53,-1,-1,55,57,59,-1,-1,-1,61,63,-1,65,67,-1,-1,-1,-1,69,-1,-1,-1,71,-1,73,-1,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5069684E1,2.0232635E0,4.3007574E0,2.678256E-1,1.1480935E0,2.6632557E0,1.1821835E0,5.3141117E-2,0E0,6.843822E-1,2.3979846E-1,6.946726E-1,1.5004082E0,2.6405144E-1,5.791068E-2,3.6856174E-2,0E0,3.9572167E-1,2.0981449E-1,1.8848248E-1,1.4268607E-1,2.1812582E-1,2.0935583E-1,0E0,7.8151226E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.8656936E-1,1.14214204E-1,0E0,8.789906E-2,1.0015816E-1,8.133631E-2,0E0,0E0,4.7646523E-2,1.01214014E-1,1.3040972E-1,0E0,0E0,0E0,6.4390734E-2,3.0782056E-1,0E0,3.734064E-2,3.9541215E-2,0E0,0E0,0E0,0E0,4.578747E-2,0E0,0E0,0E0,3.655553E-2,0E0,1.6989803E-1,0E0,0E0,1.2784004E-1,3.1500652E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.089613E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,24,24,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,45,45,46,46,48,48,49,49,54,54,58,58,60,60,63,63,64,64,73,73],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,46,48,-1,50,52,54,-1,-1,56,58,60,-1,-1,-1,62,64,-1,66,68,-1,-1,-1,-1,70,-1,-1,-1,72,-1,74,-1,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.41718E5,1.0429407E3,1.2852292E7,3.44E2,6.6512886E3,2.909019E3,1.7367111E6,7.535864E-4,8.464347E8,2.9222983E3,2.5096E4,5.0314346E1,2.5212732E4,3.1E1,9.3139405E8,-8.882214E-3,2.889654E5,4.6288285E0,1.570662E5,1.5E2,5.52801E5,1.773296E6,-8.337139E-3,1.4609149E4,-7.075428E-3,1.334781E-2,-3.263372E-2,-1.1550625E-2,-1.4594696E-2,-2.3411268E-3,2.8101E5,3E1,-7.3922356E-3,1.6804138E1,2.4469028E7,3.6828573E8,1.3459464E-2,2.737719E-3,1.14E2,4.56E2,1E0,4.856363E-3,2.3451509E-2,4.4743396E-2,4.138E3,1.0864745E3,-4.818664E-3,8.2E2,1.9937408E-1,-2.0451862E-3,-9.331088E-3,-7.800238E-5,-5.439661E-4,1.03816045E9,1.3993818E-2,6.079149E-3,-4.4446755E-3,2.0663544E9,1.135859E-2,2.0942988E9,3.4413957E-3,-2.8680263E-3,2.73E2,2.695258E1,4.904808E-4,7.254328E-3,1.082668E-3,7.6847933E-3,7.0538446E-3,-5.980293E-4,1.9199243E-3,6.2165717E-3,3.0811954E-2,4.622775E-3,-1.0922399E-2,-6.981419E-3,-5.228882E-4,-8.44262E-3,1.9954874E-3,1.9028584E-2],"split_indices":[2,1,67,9,8,61,41,60,0,12,4,2,65,4,3,5,0,37,63,56,3,38,11,0,4,0,0,0,0,0,0,38,8,0,62,9,5,0,0,3,10,22,0,0,0,2,61,0,10,50,0,0,0,0,41,0,0,0,12,0,41,0,0,3,62,0,0,0,0,0,0,0,0,47,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,3.1E2,1.36E2,7.4E1,2.36E2,1.26E2,1E1,7.1E1,3E0,1.95E2,4.1E1,9.8E1,2.8E1,4E0,6E0,6E1,1.1E1,1.75E2,2E1,2.9E1,1.2E1,3.5E1,6.3E1,2E0,2.6E1,2E0,2E0,4E0,2E0,5.9E1,1E0,1.64E2,1.1E1,3E0,1.7E1,1E1,1.9E1,6E0,6E0,1.2E1,2.3E1,5.7E1,6E0,1.8E1,8E0,1.9E1,1.45E2,3E0,8E0,1.4E1,3E0,4E0,6E0,7E0,1.2E1,6E0,6E0,3E0,2E1,2.2E1,3.5E1,4E0,1.5E1,9.9E1,4.6E1,3E0,5E0,3E0,1.1E1,1E1,2E0,1.1E1,9E0,3.2E1,3E0,5.7E1,4.2E1,2.4E1,2.2E1,1E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[1.0332783E-2,-1.151309E-1,2.3919618E-1,-1.5494882E-1,4.4064928E-2,1.8575595E-1,3.1771142E-2,-2.242329E-1,-7.781484E-2,-1.2577285E-2,1.7808056E-1,2.173784E-1,-2.4074903E-1,-2.6217318E-1,-1.591487E-1,-4.910108E-4,-1.3303834E-1,-3.0791886E-2,7.888402E-3,1.3520963E-2,3.7799703E-3,1.4568672E-1,3.2073554E-1,-2.704143E-2,6.611855E-2,-2.996104E-1,-1.8729459E-1,-9.045948E-2,-2.0850733E-1,-6.811804E-2,5.904042E-2,-1.1221767E-1,-1.5049722E-2,5.0197984E-3,-5.9339605E-2,9.178292E-2,2.5629148E-1,4.1899428E-1,7.34662E-2,2.0569861E-1,-6.8479963E-3,-1.481326E-2,-4.1216104E-3,-2.1442394E-1,-2.7292382E-3,-6.5863593E-3,-1.255702E-3,-1.0650966E-2,-1.921501E-3,-6.3552205E-3,-2.230924E-2,-2.7502356E-3,1.0735169E-1,-1.679026E-1,-5.4648317E-2,-4.3071504E-3,1.3128861E-3,-1.7485145E-2,1.2685536E-1,3.2323839E-3,1.3010263E-2,3.3812454E-1,2.7936418E-2,7.2379657E-3,-9.354428E-3,2.7732959E-3,1.567595E-2,-1.1679441E-2,-4.8166225E-3,3.4733457E-3,-3.2425977E-3,-1.2432081E-3,7.0989374E-3,-6.390032E-2,-2.0447965E-1,1.2682405E-3,-9.059443E-2,-3.5534883E-3,4.7464017E-3,1.1589393E-2,8.2485825E-2,7.2388523E-3,1.8033978E-2,-5.8764354E-3,1.8823745E-3,-1.3263095E-2,-5.5589643E-3,2.3978374E-3,-5.628962E-3,4.3127645E-4,5.6450944E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,37,-1,39,41,43,45,47,49,51,53,-1,-1,55,57,59,61,63,65,-1,-1,-1,67,-1,-1,-1,-1,-1,-1,69,-1,71,73,75,-1,-1,77,79,-1,-1,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,83,85,-1,87,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2489435E1,1.7880433E0,3.3319607E0,1.1849866E0,4.3492344E-1,1.9135566E0,0E0,2.496543E-1,4.611047E-1,1.3661098E-1,1.5145153E-1,9.0914345E-1,1.0193384E0,1.485219E-1,1.3709724E-1,1.8921468E-1,1.9989073E-1,1.5252215E-1,0E0,0E0,0E0,4.415139E-1,1.2326298E0,0E0,2.035333E-1,5.039692E-2,8.359212E-2,5.886036E-2,4.554522E-2,6.1245337E-2,1.4632314E-1,1.767577E-1,0E0,0E0,8.551324E-2,2.0916218E-1,4.598117E-2,2.8406E-1,3.5715616E-1,5.208318E-2,0E0,0E0,0E0,4.5866966E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.420682E-2,0E0,9.936781E-2,9.725237E-2,8.967988E-2,0E0,0E0,9.848862E-2,1.8666345E-1,0E0,0E0,1.260693E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.847213E-2,1.0050863E-1,0E0,8.2867056E-2,0E0,0E0,0E0,7.781853E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,38,38,39,39,43,43,50,50,52,52,53,53,54,54,57,57,58,58,61,61,73,73,74,74,76,76,80,80],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,38,-1,40,42,44,46,48,50,52,54,-1,-1,56,58,60,62,64,66,-1,-1,-1,68,-1,-1,-1,-1,-1,-1,70,-1,72,74,76,-1,-1,78,80,-1,-1,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,84,86,-1,88,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,8.625562E0,1.533115E6,1.5522031E9,7.2125336E2,3.1771142E-2,2.613714E6,1.4074289E7,1E0,6.1614815E6,1.6799316E7,3.5226266E0,1E0,2.5914587E-2,2.753337E6,6.82045E6,1.4193897E-3,7.888402E-3,1.3520963E-2,3.7799703E-3,2.5096E4,4.949877E6,-2.704143E-2,8.494E3,7.619011E7,1.6120845E5,8.2E1,7.55E2,4.39E3,4.3E1,7.3123126E2,-1.5049722E-2,5.0197984E-3,6.426437E5,7.10711E8,2.137514E-1,1.30817086E5,1.92144E5,4.166E3,-6.8479963E-3,-1.481326E-2,-4.1216104E-3,3.59494E4,-2.7292382E-3,-6.5863593E-3,-1.255702E-3,-1.0650966E-2,-1.921501E-3,-6.3552205E-3,4.6734717E-3,-2.7502356E-3,6.5610016E4,6.530506E2,2.7E1,-4.3071504E-3,1.3128861E-3,2.6079211E3,4.1124658E6,3.2323839E-3,1.3010263E-2,5.0150156E0,2.7936418E-2,7.2379657E-3,-9.354428E-3,2.7732959E-3,1.567595E-2,-1.1679441E-2,-4.8166225E-3,3.4733457E-3,-3.2425977E-3,-1.2432081E-3,7.0989374E-3,7.325293E7,2.55E2,1.2682405E-3,1.8071064E7,-3.5534883E-3,4.7464017E-3,1.1589393E-2,3.1843357E3,7.2388523E-3,1.8033978E-2,-5.8764354E-3,1.8823745E-3,-1.3263095E-2,-5.5589643E-3,2.3978374E-3,-5.628962E-3,4.3127645E-4,5.6450944E-3],"split_indices":[2,7,44,1,7,67,0,9,54,108,54,1,63,23,48,1,9,48,0,0,0,2,9,0,0,41,37,8,0,2,8,61,0,0,56,7,50,37,2,0,0,0,0,42,0,0,0,0,0,0,66,0,37,42,8,0,0,61,54,0,0,63,0,0,0,0,0,0,0,0,0,0,0,7,3,0,54,0,0,0,4,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.8E2,1.53E2,2.24E2,5.6E1,1.37E2,1.6E1,1.17E2,1.07E2,4E1,1.6E1,1.28E2,9E0,7.2E1,4.5E1,4.5E1,6.2E1,3.7E1,3E0,7E0,9E0,7.7E1,5.1E1,4E0,5E0,4.6E1,2.6E1,2E1,2.5E1,2.1E1,2.4E1,5.7E1,5E0,6E0,3.1E1,5.3E1,2.4E1,3.6E1,1.5E1,3E0,2E0,4.4E1,2E0,2.1E1,5E0,1.1E1,9E0,2.3E1,2E0,8E0,1.3E1,7E0,1.7E1,2.8E1,2.9E1,2.3E1,8E0,1.3E1,4E1,2E0,2.2E1,2.6E1,1E1,1.2E1,3E0,2E0,1E0,1.6E1,5E0,4E0,9E0,4E0,1.3E1,8E0,2E1,9E0,2E1,9E0,4E0,1E1,3E1,5E0,2.1E1,5E0,3E0,1E1,1E1,3E0,1.7E1,1E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.6736414E-2,-1.08971454E-1,2.5784653E-1,-2.3816262E-1,-6.719756E-2,1.8118238E-1,4.8294216E-1,-1.719464E-2,-2.1204115E-1,-9.224286E-2,6.4957954E-2,-2.5342238E-1,2.0817342E-1,8.1764114E-1,3.2869658E-1,-2.2699656E-3,-2.2644357E-1,-1.4662458E-1,-4.516905E-2,9.335337E-3,2.2460827E-1,2.9604465E-3,-2.4343682E-2,1.11364834E-1,3.2283038E-1,1.9730028E-2,4.434205E-2,4.0386176E-1,-8.280251E-2,-1.6019816E-3,-1.1245693E-2,-1.5964094E-1,1.9361112E-2,1.2012634E-2,-1.03762865E-1,-3.5796843E-2,4.994066E-3,1.7592486E-3,1.31699275E-2,9.910794E-3,1.4270808E-1,1.9494493E-1,3.6476085E-1,7.893403E-3,4.4580385E-1,-9.016154E-3,8.051311E-3,-3.1397662E-3,-1.7754352E-1,-1.9622848E-3,5.4486734E-3,-1.7775262E-2,9.200745E-2,-1.0100138E-2,-1.2458447E-1,-3.584486E-3,1.2775318E-3,-1.0264808E-1,7.3873303E-3,1.0535506E-1,2.0382571E-1,1.15140965E-2,3.2902632E-3,7.205441E-3,3.956661E-1,2.213496E-2,4.9863504E-3,-2.10781E-1,-1.02998406E-1,7.6582156E-2,-4.1552167E-2,1.8238161E-3,7.615465E-3,-3.620411E-3,2.2053635E-3,-2.433189E-3,-1.4153747E-1,-7.735146E-3,3.2008593E-3,1.2057195E-1,-1.1426058E-3,3.0498038E-4,1.116624E-2,4.3260804E-1,1.1571808E-2,-1.1545605E-2,-4.96558E-3,-1.0272438E-2,-1.8050721E-3,-1.042085E-3,6.741634E-3,-5.593017E-3,5.579721E-4,-7.7711777E-3,-3.0125063E-3,-1.6461054E-3,6.4760987E-3,6.6826246E-3,2.1890465E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,29,31,33,35,37,-1,-1,39,41,-1,-1,43,45,-1,-1,47,49,51,53,55,-1,-1,-1,57,59,61,63,-1,65,-1,-1,-1,67,-1,-1,69,71,73,75,-1,-1,77,-1,79,81,-1,-1,-1,83,-1,-1,85,87,89,91,-1,-1,-1,-1,-1,93,-1,-1,95,-1,-1,-1,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3091466E1,1.5166922E0,2.4665012E0,1.3788342E-1,7.209249E-1,1.3665645E0,1.6414003E0,0E0,1.2951446E-1,4.5928895E-1,3.0870914E-1,5.8517075E-1,1.1406245E0,8.201027E-2,8.685615E-1,0E0,6.312251E-2,1.8457508E-1,3.329453E-1,1.1854784E-1,7.221064E-2,0E0,0E0,1.8619967E-1,1.8628168E-1,0E0,0E0,1.5868473E-1,1.6135547E-1,0E0,0E0,1.204294E-1,4.3968476E-2,1.2316221E-1,9.430188E-2,4.7590926E-2,0E0,0E0,0E0,2.5767696E-1,8.12577E-2,6.150204E-2,1.7377901E-1,0E0,3.888917E-2,0E0,0E0,0E0,1.3933969E-1,0E0,0E0,8.84362E-2,4.404188E-2,3.9723024E-2,4.311967E-2,0E0,0E0,9.854989E-2,0E0,6.1770707E-2,8.626497E-2,0E0,0E0,0E0,6.869316E-2,0E0,0E0,1.1134541E-1,1.5075745E-1,5.270047E-2,1.2394659E-1,0E0,0E0,0E0,0E0,0E0,3.9608836E-2,0E0,0E0,6.1991513E-2,0E0,0E0,0E0,8.179426E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,23,23,24,24,27,27,28,28,31,31,32,32,33,33,34,34,35,35,39,39,40,40,41,41,42,42,44,44,48,48,51,51,52,52,53,53,54,54,57,57,59,59,60,60,64,64,67,67,68,68,69,69,70,70,76,76,79,79,83,83],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,30,32,34,36,38,-1,-1,40,42,-1,-1,44,46,-1,-1,48,50,52,54,56,-1,-1,-1,58,60,62,64,-1,66,-1,-1,-1,68,-1,-1,70,72,74,76,-1,-1,78,-1,80,82,-1,-1,-1,84,-1,-1,86,88,90,92,-1,-1,-1,-1,-1,94,-1,-1,96,-1,-1,-1,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.32424E5,5.932393E3,1.44E2,2.76E2,3.5226266E0,1.417988E6,-1.719464E-2,2E1,7.621583E2,1.2103396E5,5.5835126E9,1.0693399E-1,9.256843E3,1.5096262E7,-2.2699656E-3,2.586E4,1.348197E2,6.726204E1,1.4096699E9,3.636015E8,2.9604465E-3,-2.4343682E-2,7.822336E5,3.116E3,1.9730028E-2,4.434205E-2,2.3193505E0,3.677E3,-1.6019816E-3,-1.1245693E-2,5.1698097E1,1.3E2,2.103E3,1.32E2,5.4893E5,4.994066E-3,1.7592486E-3,1.31699275E-2,3.44E2,2.2331157E-1,3.12066E5,8.048354E4,7.893403E-3,2.1030048E10,-9.016154E-3,8.051311E-3,-3.1397662E-3,1.646359E5,-1.9622848E-3,5.4486734E-3,1E0,2.1592189E5,2.3133965E3,7.477264E6,-3.584486E-3,1.2775318E-3,1.7267E4,7.3873303E-3,2.0097266E-1,3.3555597E8,1.15140965E-2,3.2902632E-3,7.205441E-3,1.5508488E0,2.213496E-2,4.9863504E-3,6.623914E4,2.9307446E-5,1.860068E6,3.2235294E1,1.8238161E-3,7.615465E-3,-3.620411E-3,2.2053635E-3,-2.433189E-3,1.643477E9,-7.735146E-3,3.2008593E-3,6.028226E4,-1.1426058E-3,3.0498038E-4,1.116624E-2,3.00394E-2,1.1571808E-2,-1.1545605E-2,-4.96558E-3,-1.0272438E-2,-1.8050721E-3,-1.042085E-3,6.741634E-3,-5.593017E-3,5.579721E-4,-7.7711777E-3,-3.0125063E-3,-1.6461054E-3,6.4760987E-3,6.6826246E-3,2.1890465E-2],"split_indices":[2,1,61,0,8,63,38,0,10,61,42,12,44,4,9,0,38,63,63,7,5,0,0,56,0,0,0,62,0,0,0,57,8,0,3,38,0,0,0,8,47,11,37,0,5,0,0,0,37,0,0,109,37,4,54,0,0,2,0,47,7,0,0,0,43,0,0,42,47,1,63,0,0,0,0,0,5,0,0,42,0,0,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,2.83E2,1.47E2,6.8E1,2.15E2,1.11E2,3.6E1,1E1,5.8E1,1.81E2,3.4E1,6E0,1.05E2,1E1,2.6E1,5E0,5.3E1,8.3E1,9.8E1,2.6E1,8E0,3E0,3E0,5.8E1,4.7E1,3E0,7E0,2.2E1,4E0,2E0,5.1E1,7.7E1,6E0,5E1,4.8E1,1.8E1,8E0,2E0,6E0,1.4E1,4.4E1,1.3E1,3.4E1,4E0,1.8E1,3E0,1E0,1.3E1,6.4E1,4E0,2E0,3.7E1,1.3E1,9E0,3.9E1,1.1E1,7E0,8E0,6E0,2.9E1,1.5E1,9E0,4E0,5E0,2.9E1,1.7E1,1E0,4.3E1,2.1E1,7E0,3E1,8E0,5E0,4E0,5E0,8E0,3.1E1,6E0,2E0,2.6E1,3E0,2E0,1.3E1,2.2E1,7E0,3.3E1,1E1,7E0,1.4E1,3E0,4E0,1.2E1,1.8E1,2.4E1,7E0,2E0,2.4E1,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[1.0360244E-2,-1.1587383E-1,2.3470867E-1,-2.3735072E-1,-6.402226E-2,1.5961574E-1,4.6582118E-1,-2.5217342E-1,-2.1518169E-3,-1.3231471E-3,-1.2802742E-1,1.9584389E-1,-2.968825E-1,4.165462E-2,3.4413454E-1,-2.6392874E-1,-4.8348443E-3,-3.0930037E-2,1.5234528E-1,-2.4795322E-1,-1.0937103E-1,8.944114E-2,2.9134393E-1,-5.8294497E-3,-2.3463368E-2,4.0907294E-1,-8.690112E-3,-4.816474E-3,-1.313818E-2,-7.247271E-2,8.059313E-2,2.545395E-1,-1.5410031E-2,-3.0238307E-1,-3.061601E-3,-1.3019237E-1,-4.9133957E-3,1.475843E-1,-1.38743045E-2,2.0049937E-1,3.2813117E-1,2.2720862E-2,9.765763E-3,-1.03209525E-1,-3.499727E-2,-2.1638868E-3,1.275174E-1,4.6340744E-3,1.4884607E-2,1.2983587E-3,-6.4956066E-3,-1.6064605E-2,-1.3501332E-3,-1.5819882E-1,-4.0496014E-2,-3.1987748E-3,4.739031E-3,2.0784827E-1,7.5093985E-2,-1.0798883E-2,5.264751E-2,6.1926963E-3,1.21751E-2,3.9093396E-1,2.3314272E-1,-3.3228414E-3,-1.7519337E-1,-6.816442E-3,-3.3486192E-4,9.332668E-3,2.986092E-3,3.1244024E-3,-1.6821447E-1,-3.5112046E-3,9.983587E-4,1.3420669E-2,2.762953E-3,4.7939294E-3,-3.4522004E-3,1.7017916E-5,5.0577507E-3,6.4678183E-3,1.9763991E-2,4.543405E-3,1.2830314E-2,1.7697144E-4,-9.425004E-3,-6.982221E-3,-1.540987E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,43,45,47,49,51,-1,53,55,57,59,61,63,-1,-1,65,67,-1,69,-1,-1,-1,-1,-1,-1,71,73,-1,-1,75,77,-1,79,-1,-1,81,83,-1,85,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2035052E1,1.6980693E0,2.5809488E0,2.2013521E-1,7.703394E-1,1.993099E0,1.4950914E0,1.0745621E-1,0E0,4.5057717E-1,1.8543243E-1,1.081542E0,2.4169171E-1,0E0,1.0952768E0,6.963873E-2,0E0,3.9037737E-1,2.7822033E-1,1.0127795E-1,1.8205607E-1,3.2042328E-1,1.24699116E-1,0E0,0E0,2.5117254E-1,0E0,0E0,0E0,6.665236E-2,1.4113979E-1,5.6147218E-2,3.8986914E-2,6.672251E-2,0E0,1.6943884E-1,1.0161425E-1,1.3292271E-1,2.9130936E-1,3.878343E-2,1.4151287E-1,0E0,0E0,6.938985E-2,8.555657E-2,0E0,6.1647087E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2927496E-1,3.7119914E-2,0E0,0E0,1.750825E-1,6.779807E-2,0E0,4.3681588E-2,0E0,0E0,5.2194357E-2,5.7335258E-2,0E0,3.699386E-2,0E0,0E0,0E0,0E0,0E0,1.3440573E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,46,46,53,53,54,54,57,57,58,58,60,60,63,63,64,64,66,66,72,72],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,44,46,48,50,52,-1,54,56,58,60,62,64,-1,-1,66,68,-1,70,-1,-1,-1,-1,-1,-1,72,74,-1,-1,76,78,-1,80,-1,-1,82,84,-1,86,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,9.04782E5,6.6512886E3,1.2852292E7,1.4074289E7,9.05929E2,1E0,2.4752695E6,-2.1518169E-3,1.1576422E9,1.2808544E3,2.5096E4,7.339209E6,4.165462E-2,5.37E2,7.547656E1,-4.8348443E-3,2.972948E6,1.5E2,4.241E3,4.723148E11,1.3508742E7,5.6503E4,-5.8294497E-3,-2.3463368E-2,1E0,-8.690112E-3,-4.816474E-3,-1.313818E-2,1.604679E6,3.3E1,2.4237612E3,7.49E2,2.4927816E1,-3.061601E-3,6.4065235E8,5.767668E0,6.7344745E6,2.3808822E-2,8E1,6.0495055E-1,2.2720862E-2,9.765763E-3,6.17296E5,8.8E1,-2.1638868E-3,1.03E2,4.6340744E-3,1.4884607E-2,1.2983587E-3,-6.4956066E-3,-1.6064605E-2,-1.3501332E-3,3.904E3,7.792E3,-3.1987748E-3,4.739031E-3,7.4433655E-2,2.0863011E5,-1.0798883E-2,9.3E1,6.1926963E-3,1.21751E-2,3.00394E-2,3.4606943E0,-3.3228414E-3,4.9E1,-6.816442E-3,-3.3486192E-4,9.332668E-3,2.986092E-3,3.1244024E-3,2.5233118E11,-3.5112046E-3,9.983587E-4,1.3420669E-2,2.762953E-3,4.7939294E-3,-3.4522004E-3,1.7017916E-5,5.0577507E-3,6.4678183E-3,1.9763991E-2,4.543405E-3,1.2830314E-2,1.7697144E-4,-9.425004E-3,-6.982221E-3,-1.540987E-2],"split_indices":[2,1,61,9,54,67,25,59,0,7,4,2,56,0,3,63,0,1,3,2,40,54,2,0,0,13,0,0,0,1,8,4,0,67,0,7,63,60,47,3,43,0,0,38,3,0,3,0,0,0,0,0,0,2,2,0,0,66,42,0,8,0,0,47,44,0,3,0,0,0,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.23E2,2.71E2,1.52E2,8E1,1.91E2,1.16E2,3.6E1,7.4E1,6E0,9.7E1,9.4E1,1.08E2,8E0,7E0,2.9E1,6.8E1,6E0,8.2E1,1.5E1,1.1E1,8.3E1,5.2E1,5.6E1,5E0,3E0,2.6E1,3E0,4E0,6.4E1,6E1,2.2E1,9E0,6E0,8E0,3E0,6.9E1,1.4E1,3.3E1,1.9E1,1.8E1,3.8E1,1.9E1,7E0,3.2E1,2.8E1,6E0,1.6E1,3E0,6E0,5E0,1E0,7E0,1E0,5.2E1,1.7E1,9E0,5E0,1.7E1,1.6E1,4E0,1.5E1,9E0,9E0,2.1E1,1.7E1,2.3E1,9E0,5E0,2.3E1,7E0,9E0,2E0,5E1,1.1E1,6E0,1.1E1,6E0,1.4E1,2E0,8E0,7E0,2E0,1.9E1,4E0,1.3E1,1E0,8E0,4.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[7.6630467E-4,-9.550667E-2,2.6269126E-1,-2.0206894E-1,-4.0149566E-2,1.7392924E-1,5.07676E-1,-2.4534772E-1,-8.857899E-2,-6.6349E-2,1.3017826E-1,2.0315427E-1,-2.1892605E-2,8.041404E-1,3.3406E-1,-5.403103E-3,-2.693378E-1,-1.3788097E-1,-1.3500943E-2,-1.4535369E-1,-1.9061238E-2,-4.4121505E-3,1.515735E-1,1.7336762E-1,1.723709E-2,1.5613823E-2,4.2610276E-2,1.7242925E-2,-2.7376816E-3,-2.7834532E-1,-3.3337683E-3,-8.581462E-4,-7.74883E-3,3.5287049E-3,-2.2577303E-3,-2.4464324E-1,-1.250278E-1,-5.2465342E-2,1.0957452E-1,9.6175745E-3,5.9783548E-2,1.8082005E-1,-8.037514E-3,-1.369362E-2,-3.7611725E-3,-1.298183E-2,-5.6025386E-4,-1.4653718E-1,-1.1433616E-3,-8.032827E-2,2.0962365E-2,8.703976E-3,7.6024863E-4,4.0967297E-3,-4.022691E-3,1.2596552E-1,2.4610358E-1,-7.2495436E-4,-1.5870315E-1,-5.1544297E-2,-2.1432883E-1,9.9826306E-2,-3.8268678E-2,1.7895983E-1,4.117033E-3,1.404201E-2,5.2287877E-3,-5.390998E-3,-1.1362456E-2,-4.5473096E-3,-9.2549017E-4,-4.583657E-3,-1.2965199E-2,6.142507E-3,-7.7905867E-4,-5.4650907E-3,5.1131524E-4,1.018805E-2,-1.5244297E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,29,31,33,35,37,-1,39,41,-1,-1,-1,-1,-1,43,-1,-1,-1,-1,-1,45,47,49,51,-1,53,55,-1,-1,-1,-1,-1,57,-1,59,61,-1,-1,-1,-1,63,65,-1,67,69,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0414202E1,1.7733784E0,2.3254337E0,4.8417377E-1,9.011356E-1,1.6155257E0,1.2240257E0,2.0643806E-1,1.0918337E-1,6.472855E-1,1.4193213E-1,3.0581188E-1,0E0,9.613609E-2,2.0446062E-1,0E0,9.255695E-2,4.6486706E-2,4.0981065E-2,1.00592375E-1,4.7841468E-1,0E0,9.798288E-2,2.0501018E-1,0E0,0E0,0E0,0E0,0E0,4.4756413E-2,0E0,0E0,0E0,0E0,0E0,5.419618E-2,1.1856085E-1,1.8172404E-1,1.5022448E-1,0E0,4.3154165E-2,2.1251392E-1,0E0,0E0,0E0,0E0,0E0,7.120669E-2,0E0,2.3580495E-1,1.20600045E-1,0E0,0E0,0E0,0E0,6.858498E-2,1.5478563E-1,0E0,1.2521577E-1,7.28986E-2,4.6027362E-2,3.7232302E-2,5.6241382E-2,1.1564374E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23,29,29,35,35,36,36,37,37,38,38,40,40,41,41,47,47,49,49,50,50,55,55,56,56,58,58,59,59,60,60,61,61,62,62,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,30,32,34,36,38,-1,40,42,-1,-1,-1,-1,-1,44,-1,-1,-1,-1,-1,46,48,50,52,-1,54,56,-1,-1,-1,-1,-1,58,-1,60,62,-1,-1,-1,-1,64,66,-1,68,70,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,1.074952E6,6.0495757E3,3.006846E6,1.2678202E9,4.673502E7,1.681178E6,7.581957E1,7.66951E5,7.283496E2,1.5560601E6,4.3431563E0,-2.1892605E-2,1.082556E7,1.354316E6,-5.403103E-3,1.7212875E5,2E1,4.138E3,6.506511E0,1.5271514E5,-4.4121505E-3,7.200105E8,2.8756912E8,1.723709E-2,1.5613823E-2,4.2610276E-2,1.7242925E-2,-2.7376816E-3,1.5650286E6,-3.3337683E-3,-8.581462E-4,-7.74883E-3,3.5287049E-3,-2.2577303E-3,1.154375E6,2.1592189E5,1.7657828E3,1.1547264E1,9.6175745E-3,3.23643E6,7.757358E9,-8.037514E-3,-1.369362E-2,-3.7611725E-3,-1.298183E-2,-5.6025386E-4,1.4590734E8,-1.1433616E-3,1E0,1.36E2,8.703976E-3,7.6024863E-4,4.0967297E-3,-4.022691E-3,1.30569E5,1.5953E4,-7.2495436E-4,9E1,2.025313E6,4.264897E6,7.309061E1,3.0167932E-5,2.237815E7,4.117033E-3,1.404201E-2,5.2287877E-3,-5.390998E-3,-1.1362456E-2,-4.5473096E-3,-9.2549017E-4,-4.583657E-3,-1.2965199E-2,6.142507E-3,-7.7905867E-4,-5.4650907E-3,5.1131524E-4,1.018805E-2,-1.5244297E-3],"split_indices":[2,1,61,9,7,57,38,63,1,61,54,44,0,1,2,0,37,10,2,65,37,0,5,1,0,0,0,0,0,56,0,0,0,0,0,9,37,61,62,0,1,5,0,0,0,0,0,5,0,18,3,0,0,0,0,9,0,0,8,1,9,65,47,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.11E2,3.01E2,1.1E2,1.02E2,1.99E2,8.2E1,2.8E1,7.3E1,2.9E1,1.73E2,2.6E1,7.9E1,3E0,9E0,1.9E1,1.2E1,6.1E1,1.7E1,1.2E1,6.4E1,1.09E2,2E0,2.4E1,6.8E1,1.1E1,2E0,7E0,1.8E1,1E0,5.8E1,3E0,3E0,1.4E1,3E0,9E0,9E0,5.5E1,8.7E1,2.2E1,1.5E1,9E0,6.7E1,1E0,5.6E1,2E0,8E0,1E0,4.5E1,1E1,6.3E1,2.4E1,1.2E1,1E1,8E0,1E0,3.8E1,2.9E1,4E0,4.1E1,5.3E1,1E1,1E1,1.4E1,1.5E1,2.3E1,2.1E1,8E0,2.7E1,1.4E1,2.2E1,3.1E1,4E0,6E0,8E0,2E0,5E0,9E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[1.546744E-2,-1.0390022E-1,2.3958056E-1,-1.319064E-1,4.4040836E-2,1.6462655E-1,5.1716095E-1,-2.082038E-1,-8.189755E-2,-1.6179278E-2,1.0234613E-1,1.8940137E-1,-4.3316472E-1,3.9206713E-2,3.3749297E-1,-3.192788E-3,-2.2280197E-1,-1.3540867E-1,-4.589719E-2,2.1711164E-3,-1.00253E-1,1.8837179E-1,3.7568054E-4,1.1776624E-1,2.6410514E-1,3.199979E-3,-2.7706783E-2,3.9735788E-1,-7.4324403E-3,-2.3997337E-1,-6.2179263E-3,-9.920137E-2,-1.9275445E-1,-6.8896115E-2,9.031946E-2,-6.7808386E-3,-2.1800918E-4,1.01538E-3,1.1658745E-2,1.4724818E-1,-4.436826E-2,1.1622444E-2,2.7844793E-1,1.3388451E-2,2.4083063E-2,-2.508232E-1,-2.4556848E-3,-1.1988134E-1,2.7544163E-3,-2.697475E-1,-9.8238245E-2,-9.2852116E-2,-1.0697664E-2,7.037407E-3,-1.3447588E-3,9.20633E-2,1.8593797E-1,-5.6828777E-3,1.1703041E-3,6.1290455E-3,-8.072794E-3,1.7348944E-1,3.2447094E-1,-1.3281964E-2,-1.7116794E-1,-1.4777277E-1,-1.9746225E-3,-1.4197386E-2,-5.4674805E-4,-7.7765505E-3,4.665463E-4,-4.855281E-2,-1.3371246E-1,1.116896E-3,-2.7499893E-3,2.2323185E-3,1.5810893E-1,1.2005555E-2,6.4401818E-3,4.5815273E-3,1.1639319E-2,1.8123094E-2,9.384081E-3,-9.215674E-3,-3.6785155E-4,-9.123783E-3,-4.3659024E-3,-3.5742475E-3,2.5709916E-3,-3.2542816E-3,-7.862669E-3,1.0088847E-3,9.369093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,31,33,-1,35,37,-1,39,41,-1,-1,43,-1,45,-1,47,49,51,53,-1,-1,-1,-1,55,57,59,61,-1,-1,63,-1,65,-1,67,69,71,73,-1,-1,75,77,-1,-1,-1,-1,79,81,-1,83,85,-1,-1,-1,-1,-1,87,89,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1394431E1,1.1576369E0,2.994114E0,8.757858E-1,1.5965772E-1,1.8296425E0,1.3741341E0,1.7593694E-1,2.6926303E-1,1.22872554E-1,1.8561693E-1,5.789585E-1,3.914942E-1,0E0,6.7932653E-1,0E0,1.0279894E-1,9.922969E-2,2.7877945E-1,0E0,3.8022257E-2,1.02184206E-1,0E0,2.9333544E-1,1.9593573E-1,0E0,0E0,9.3532324E-2,0E0,1.2874985E-1,0E0,1.2646428E-1,1.2631941E-1,1.0357186E-1,8.956909E-2,0E0,0E0,0E0,0E0,9.17716E-2,5.3307958E-2,1.0400126E-1,1.9489384E-1,0E0,0E0,8.045149E-2,0E0,6.659448E-2,0E0,6.688988E-2,7.640196E-2,8.944237E-2,3.7409957E-2,0E0,0E0,6.219898E-2,6.250644E-2,0E0,0E0,0E0,0E0,7.298082E-2,1.6044831E-1,0E0,5.7314157E-2,3.979844E-2,0E0,0E0,0E0,0E0,0E0,7.4306324E-2,4.1485697E-2,0E0,0E0,0E0,4.0636778E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,17,17,18,18,20,20,21,21,23,23,24,24,27,27,29,29,31,31,32,32,33,33,34,34,39,39,40,40,41,41,42,42,45,45,47,47,49,49,50,50,51,51,52,52,55,55,56,56,61,61,62,62,64,64,65,65,71,71,72,72,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,32,34,-1,36,38,-1,40,42,-1,-1,44,-1,46,-1,48,50,52,54,-1,-1,-1,-1,56,58,60,62,-1,-1,64,-1,66,-1,68,70,72,74,-1,-1,76,78,-1,-1,-1,-1,80,82,-1,84,86,-1,-1,-1,-1,-1,88,90,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.164074E8,7.7101436E3,1.129877E6,2.299087E3,9.533205E6,3.0418E4,3.5454545E0,7.283496E2,9.481747E8,1.21E2,4.1331047E-1,1.563E3,3.9206713E-2,1.852084E7,-3.192788E-3,1.6395413E7,1.7377544E7,3.978477E11,2.1711164E-3,2.8688732E7,1.319525E7,3.7568054E-4,2.5916522E7,2.2782404E0,3.199979E-3,-2.7706783E-2,1.2307312E5,-7.4324403E-3,1.7212875E5,-6.2179263E-3,1.2833E4,2.56445E7,1.4849418E3,1.6991062E3,-6.7808386E-3,-2.1800918E-4,1.01538E-3,1.1658745E-2,5.589612E1,1.2538845E9,2.203418E7,1.6441814E9,1.3388451E-2,2.4083063E-2,3.006846E6,-2.4556848E-3,4.5765094E2,2.7544163E-3,1.5592E4,1E0,1.5372442E7,6.7211235E1,7.037407E-3,-1.3447588E-3,2.1583E4,6.3349745E6,-5.6828777E-3,1.1703041E-3,6.1290455E-3,-8.072794E-3,1.3299019E0,9.83854E-1,-1.3281964E-2,2.19546E5,4.653159E9,-1.9746225E-3,-1.4197386E-2,-5.4674805E-4,-7.7765505E-3,4.665463E-4,2.327793E9,5.399E3,1.116896E-3,-2.7499893E-3,2.2323185E-3,2.1E1,1.2005555E-2,6.4401818E-3,4.5815273E-3,1.1639319E-2,1.8123094E-2,9.384081E-3,-9.215674E-3,-3.6785155E-4,-9.123783E-3,-4.3659024E-3,-3.5742475E-3,2.5709916E-3,-3.2542816E-3,-7.862669E-3,1.0088847E-3,9.369093E-3],"split_indices":[2,7,61,1,4,56,10,67,61,7,3,44,0,0,9,0,12,54,40,0,9,9,0,54,63,0,0,37,0,37,0,10,54,61,42,0,0,0,0,65,7,1,7,0,0,9,0,61,0,0,13,54,65,0,0,10,54,0,0,0,0,48,43,0,38,5,0,0,0,0,0,5,2,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,2.77E2,1.47E2,2.33E2,4.4E1,1.17E2,3E1,9.1E1,1.42E2,2.2E1,2.2E1,1.13E2,4E0,1E1,2E1,9E0,8.2E1,5.6E1,8.6E1,1.3E1,9E0,1.1E1,1.1E1,5.9E1,5.4E1,1E0,3E0,1.8E1,2E0,6.8E1,1.4E1,3.6E1,2E1,7.4E1,1.2E1,6E0,3E0,3E0,8E0,5E1,9E0,3E0,5.1E1,1E1,8E0,6.4E1,4E0,3.2E1,4E0,1E1,1E1,5.2E1,2.2E1,8E0,4E0,2.2E1,2.8E1,4E0,5E0,2E0,1E0,1.7E1,3.4E1,4.7E1,1.7E1,2.3E1,9E0,9E0,1E0,6E0,4E0,2.6E1,2.6E1,1.3E1,9E0,1.4E1,8E0,1.1E1,1.7E1,9E0,8E0,2.3E1,1.1E1,1.5E1,2E0,1.2E1,1.1E1,2.1E1,5E0,9E0,1.7E1,2E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.7597096E-3,-1.168915E-1,1.8317546E-1,-1.412023E-1,4.5510303E-2,8.922614E-2,3.7115696E-1,-1.3266256E-1,-2.8370062E-2,7.1954235E-2,-1.2562321E-2,5.0185658E-2,9.617597E-3,3.8300313E-2,2.9602295E-1,-2.026586E-1,-8.739414E-2,-8.795061E-3,9.424901E-2,1.0220941E-2,1.5466717E-1,3.1768188E-1,-7.1935314E-3,-2.1958934E-1,-4.863981E-2,-9.471234E-2,2.2240015E-2,1.6732134E-1,1.52861215E-2,-2.7431672E-2,5.205789E-3,1.2349384E-2,4.7166464E-3,2.599969E-1,4.075782E-1,-1.5318218E-2,-1.9727542E-1,4.9696416E-3,-3.710157E-3,-1.2021609E-1,-1.9112382E-2,4.351779E-3,2.3310462E-1,-3.8349116E-3,3.8636932E-3,-6.642287E-3,-1.1017163E-2,1.7503998E-3,2.7369967E-1,4.24947E-1,3.1262636E-3,-2.0705023E-1,-3.0563888E-3,-7.56127E-2,-1.8582089E-1,2.6031913E-2,-1.0098551E-1,1.39218895E-2,2.1278302E-3,3.2682024E-4,-5.73381E-3,6.462005E-4,1.3607173E-2,4.021685E-3,2.1318933E-2,-6.6902046E-3,-1.0891703E-2,-4.537935E-3,2.2116732E-3,-1.4993844E-2,-7.901927E-3,-1.7015742E-3,4.9041547E-3,2.2911462E-3,-5.656726E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,-1,21,23,25,-1,27,29,31,33,-1,35,37,39,-1,41,43,45,-1,-1,-1,47,49,-1,51,-1,-1,53,55,-1,57,-1,-1,-1,59,-1,61,63,-1,65,-1,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.363349E0,1.0447812E0,3.004529E0,7.9647255E-1,3.0201095E-1,4.8924744E-1,1.5772376E0,6.9839454E-1,0E0,2.0433466E-1,0E0,3.6169407E-1,0E0,0E0,5.2954245E-1,2.1879268E-1,6.0179305E-1,0E0,1.7631239E-1,2.3851007E-1,1.13361E-1,1.4294338E-1,0E0,1.10275745E-1,5.087883E-2,2.6377773E-1,0E0,6.0074598E-2,1.0475663E-1,8.359957E-2,0E0,0E0,0E0,8.815074E-2,7.364726E-2,0E0,7.4558735E-2,0E0,0E0,2.873479E-1,1.3568863E-1,0E0,7.1367085E-2,0E0,0E0,0E0,8.145309E-2,0E0,6.270027E-2,5.715704E-2,0E0,4.844737E-2,0E0,1.4593795E-1,6.1587453E-2,1.14971675E-1,3.711085E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,27,27,28,28,29,29,33,33,34,34,36,36,39,39,40,40,42,42,46,46,48,48,49,49,51,51,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,-1,22,24,26,-1,28,30,32,34,-1,36,38,40,-1,42,44,46,-1,-1,-1,48,50,-1,52,-1,-1,54,56,-1,58,-1,-1,-1,60,-1,62,64,-1,66,-1,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,1.4777102E9,6.0891E4,5.825042E6,2.325001E6,1.5271514E5,3.0418E4,9.50639E5,-2.8370062E-2,2.2782404E0,-1.2562321E-2,1.29186E9,9.617597E-3,3.8300313E-2,5.22E2,1.1482625E7,3.723672E3,-8.795061E-3,7.801943E3,2.143564E3,1.1797827E9,1.6846506E5,-7.1935314E-3,1.44E2,2.0673077E0,1.0643513E3,2.2240015E-2,1E0,1.6167939E7,1.5716302E3,5.205789E-3,1.2349384E-2,4.7166464E-3,1.7648785E-1,5.88824E7,-1.5318218E-2,1.6120845E5,4.9696416E-3,-3.710157E-3,1.7377544E7,3.271774E7,4.351779E-3,1.30817086E5,-3.8349116E-3,3.8636932E-3,-6.642287E-3,6.855442E1,1.7503998E-3,1.926778E6,1.1100566E3,3.1262636E-3,1.4388227E4,-3.0563888E-3,2.889654E5,1.01319025E3,2.455358E6,6.453125E0,1.39218895E-2,2.1278302E-3,3.2682024E-4,-5.73381E-3,6.462005E-4,1.3607173E-2,4.021685E-3,2.1318933E-2,-6.6902046E-3,-1.0891703E-2,-4.537935E-3,2.2116732E-3,-1.4993844E-2,-7.901927E-3,-1.7015742E-3,4.9041547E-3,2.2911462E-3,-5.656726E-3],"split_indices":[42,7,2,56,2,37,10,1,0,63,0,7,0,0,3,9,61,0,41,61,5,37,0,0,67,61,0,23,41,4,0,0,0,50,9,0,37,0,0,54,54,0,37,0,0,0,65,0,38,4,0,37,0,37,4,1,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.62E2,1.71E2,2.28E2,3.4E1,1.15E2,5.6E1,2.25E2,3E0,3.2E1,2E0,8.6E1,2.9E1,7E0,4.9E1,8.7E1,1.38E2,2E0,3E1,6.3E1,2.3E1,4.7E1,2E0,7.8E1,9E0,1.37E2,1E0,1.5E1,1.5E1,4.6E1,1.7E1,7E0,1.6E1,3.1E1,1.6E1,1.2E1,6.6E1,1E0,8E0,1.02E2,3.5E1,8E0,7E0,6E0,9E0,5E0,4.1E1,2E0,2.9E1,1.5E1,1E0,6.1E1,5E0,6.2E1,4E1,2.3E1,1.2E1,5E0,2E0,3.6E1,5E0,1E0,2.8E1,1E0,1.4E1,1.5E1,4.6E1,5.4E1,8E0,4E0,3.6E1,1.3E1,1E1,1E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[7.082897E-3,-9.3978085E-2,2.0779131E-1,-2.0580742E-1,-6.070377E-2,1.4287964E-1,4.6922612E-1,-2.1054009E-1,2.5686724E-3,-1.3680133E-2,-1.0519901E-1,1.6371809E-1,-2.169007E-1,3.7526373E-2,3.1442246E-1,-2.6027006E-1,-8.8360645E-3,-3.993859E-2,1.0910569E-1,-1.383444E-1,-4.4940777E-2,1.0864145E-1,2.5213203E-1,1.3799772E-2,-3.4845343E-1,3.703193E-1,-6.962304E-3,-1.3469471E-2,-2.5374705E-3,-7.5318806E-2,1.8172272E-2,-3.4949984E-4,1.3864732E-1,-1.5750937E-1,9.540835E-5,3.9329953E-3,-7.1806036E-2,2.5962574E-2,1.4217663E-1,2.65883E-1,2.4874273E-3,-1.9401709E-2,-1.898735E-3,9.788069E-3,2.082569E-2,-9.1332E-2,5.148375E-4,4.5123175E-2,-4.818103E-3,1.03584565E-2,3.4450933E-3,-2.1050055E-1,-9.813938E-2,-7.5839334E-3,-4.5747884E-2,7.499257E-2,-7.5090467E-3,1.655308E-1,2.0349093E-3,1.1411193E-3,2.780084E-1,-8.3916765E-3,-7.7977404E-2,1.0298809E-1,6.2020123E-3,1.0118803E-3,-2.2562432E-1,-2.4531811E-2,-1.5108539E-1,1.1765874E-3,-9.670122E-2,-1.854045E-3,5.8622546E-3,2.3396684E-1,1.3590734E-1,3.7227538E-1,1.13749085E-2,-1.9803843E-3,-5.1600426E-3,-1.342175E-3,6.3614505E-3,-3.324105E-3,2.2269548E-3,-1.243824E-2,-3.5319484E-3,-3.1102833E-3,4.89526E-3,-3.2702945E-3,-1.0570675E-2,-7.6257545E-3,-1.8257112E-3,1.2282894E-2,6.3062116E-4,4.3004947E-3,9.760827E-3,1.9319447E-2,2.810489E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,35,37,39,-1,41,43,-1,-1,-1,45,47,-1,49,51,-1,-1,53,55,57,59,-1,-1,-1,-1,-1,61,-1,63,-1,-1,-1,65,67,-1,69,71,-1,73,-1,-1,75,-1,77,79,81,-1,83,85,87,-1,89,-1,-1,91,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.964679E0,1.0814936E0,2.4445424E0,9.336829E-2,4.756117E-1,9.3518186E-1,1.1428056E0,4.1840076E-2,0E0,3.6546415E-1,2.2746491E-1,5.2773595E-1,5.6302714E-1,0E0,5.9205294E-1,6.8528295E-2,0E0,1.9257434E-1,6.975092E-2,2.0200813E-1,1.5200382E-1,1.9593954E-1,1.0486245E-1,0E0,8.418012E-2,1.0816169E-1,0E0,0E0,0E0,8.084586E-2,1.19093426E-1,0E0,6.656158E-2,1.8698335E-1,0E0,0E0,7.299721E-2,2.0767222E-1,1.1208749E-1,1.1089897E-1,0E0,0E0,0E0,0E0,0E0,4.0553004E-2,0E0,6.692635E-2,0E0,0E0,0E0,1.23767495E-1,1.2490213E-1,0E0,1.0601719E-1,9.981461E-2,0E0,5.1321983E-2,0E0,0E0,6.295538E-2,0E0,4.276237E-2,4.9119115E-2,6.0272485E-2,0E0,1.3881493E-1,8.213895E-2,9.2939734E-2,0E0,5.5481404E-2,0E0,0E0,4.7160804E-2,7.82938E-2,6.584966E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,29,29,30,30,32,32,33,33,36,36,37,37,38,38,39,39,45,45,47,47,51,51,52,52,54,54,55,55,57,57,60,60,62,62,63,63,64,64,66,66,67,67,68,68,70,70,73,73,74,74,75,75],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,36,38,40,-1,42,44,-1,-1,-1,46,48,-1,50,52,-1,-1,54,56,58,60,-1,-1,-1,-1,-1,62,-1,64,-1,-1,-1,66,68,-1,70,72,-1,74,-1,-1,76,-1,78,80,82,-1,84,86,88,-1,90,-1,-1,92,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.41718E5,7.7101436E3,2.91E2,1.3026638E7,2.3028242E3,3.0418E4,3.171E3,2.5686724E-3,3.23643E6,1.0864745E3,6.050605E9,1.3812E4,3.7526373E-2,1.852084E7,4.14E2,-8.8360645E-3,7.164074E8,3.3E1,6.7204064E8,1.423795E7,1.5704E4,9.677921E7,1.3799772E-2,9.652E3,3.221225E9,-6.962304E-3,-1.3469471E-2,-2.5374705E-3,3.4015296E8,2.39E2,-3.4949984E-4,9.5E1,2.73E2,9.540835E-5,3.9329953E-3,5.91294E5,1.799389E6,2.237815E7,8.946923E2,2.4874273E-3,-1.9401709E-2,-1.898735E-3,9.788069E-3,2.082569E-2,2.6E1,5.148375E-4,2.3466667E1,-4.818103E-3,1.03584565E-2,3.4450933E-3,6.453125E0,2.7434757E1,-7.5839334E-3,4.462272E-3,3.253389E8,-7.5090467E-3,2.1830985E-2,2.0349093E-3,1.1411193E-3,5.5203705E1,-8.3916765E-3,1E0,2.94E2,1.8756017E5,1.0118803E-3,2.3883E4,4.17E2,2.412E3,1.1765874E-3,3.5317596E1,-1.854045E-3,5.8622546E-3,5.4276E4,1.4773067E5,5.848343E7,1.13749085E-2,-1.9803843E-3,-5.1600426E-3,-1.342175E-3,6.3614505E-3,-3.324105E-3,2.2269548E-3,-1.243824E-2,-3.5319484E-3,-3.1102833E-3,4.89526E-3,-3.2702945E-3,-1.0570675E-2,-7.6257545E-3,-1.8257112E-3,1.2282894E-2,6.3062116E-4,4.3004947E-3,9.760827E-3,1.9319447E-2,2.810489E-3],"split_indices":[2,1,61,8,54,67,10,2,0,1,61,12,9,0,9,0,0,7,8,7,54,2,1,0,0,7,0,0,0,41,3,0,3,3,0,0,38,38,54,61,0,0,0,0,0,8,0,67,0,0,0,56,63,0,48,7,0,66,0,0,67,0,23,0,56,0,10,3,10,0,67,0,0,2,37,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,2.93E2,1.47E2,6.6E1,2.27E2,1.19E2,2.8E1,6.5E1,1E0,1.11E2,1.16E2,1.13E2,6E0,8E0,2E1,2E1,4.5E1,9.2E1,1.9E1,7.4E1,4.2E1,7.1E1,4.2E1,1E0,5E0,1.8E1,2E0,1.8E1,2E0,5.7E1,3.5E1,4E0,1.5E1,6.5E1,9E0,7E0,3.5E1,2.1E1,5E1,3.9E1,3E0,4E0,1E0,6E0,1.2E1,4.8E1,9E0,2.9E1,6E0,6E0,9E0,3.3E1,3.2E1,7E0,2.8E1,1.7E1,4E0,4E1,1E1,2E0,3.7E1,5E0,4.3E1,1.1E1,1.8E1,2E0,3.1E1,1.4E1,1.8E1,1.2E1,1.6E1,5E0,1.2E1,1E1,3E1,9E0,2.8E1,2E1,2.3E1,2E0,9E0,6E0,1.2E1,2.5E1,6E0,1.1E1,3E0,9E0,9E0,7E0,9E0,9E0,1E0,1.9E1,1.1E1,8E0,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[2.003553E-3,-5.2732266E-2,3.0342913E-1,-1.119531E-1,8.242248E-2,6.008736E-1,2.4070425E-1,-1.0457353E-1,-2.6642542E-2,1.9049972E-2,1.9760107E-1,9.249757E-3,3.521117E-2,2.957336E-1,9.6997665E-3,-1.3200247E-1,-2.930618E-3,-1.5144641E-2,8.036632E-3,1.2688367E-1,2.6908433E-1,-1.9506584E-3,3.1193444E-1,5.03713E-3,-9.930492E-3,-9.163199E-2,-1.8132257E-1,-7.248193E-3,1.818593E-2,-5.6156933E-2,3.8580354E-2,4.5896373E-3,1.0395679E-2,2.968744E-1,4.370916E-3,6.81605E-3,3.5794407E-1,-1.3240927E-1,-3.6501832E-2,-2.0511392E-1,-1.1909381E-1,-4.0397175E-2,9.191961E-2,-1.4058717E-2,-1.1135064E-1,5.9696054E-3,6.548828E-4,1.6946608E-2,7.1020764E-3,1.8498797E-2,3.3324452E-3,-1.4231193E-1,1.1848042E-2,6.3378096E-2,-7.4330084E-2,-2.1719001E-1,-2.612946E-3,-6.419736E-3,-2.415539E-4,-6.58884E-2,3.0883285E-3,6.620703E-3,1.641392E-3,-5.2588433E-3,3.7780224E-4,1.2687654E-3,-1.4795971E-1,-1.5656314E-3,-7.831419E-3,5.0909584E-3,-5.0616096E-4,-2.2921336E-3,-6.7132553E-3,-1.1126449E-2,-5.67272E-3,2.6964778E-3,-3.86464E-3,-9.84731E-3,-2.7794978E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,25,27,29,-1,31,33,-1,35,-1,-1,37,39,-1,41,43,45,-1,-1,47,-1,-1,49,51,53,55,57,59,61,63,65,-1,-1,-1,-1,-1,-1,67,-1,69,71,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.192968E0,2.9626946E0,1.1053977E0,7.778628E-1,8.2104045E-1,4.4215393E-1,7.2171664E-1,7.077539E-1,0E0,3.7608773E-1,1.6806197E-1,0E0,0E0,2.6316214E-1,2.667734E-1,3.8160038E-1,1.7392245E-1,1.3657305E-1,0E0,4.1341156E-2,6.7301154E-2,0E0,2.9470396E-1,0E0,0E0,2.4694926E-1,1.0733724E-1,0E0,2.1537113E-1,7.9759024E-2,5.584382E-2,0E0,0E0,7.889676E-2,0E0,0E0,2.260747E-1,2.7462256E-1,1.8984677E-1,1.0304999E-1,4.261616E-2,7.877138E-2,5.312553E-2,4.4538397E-2,7.9490244E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3390028E-1,0E0,4.476319E-2,5.5978835E-2,5.120969E-2,0E0,0E0,0E0,4.444719E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.921359E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,25,25,26,26,28,28,29,29,30,30,33,33,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,51,51,53,53,54,54,55,55,59,59,66,66],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,26,28,30,-1,32,34,-1,36,-1,-1,38,40,-1,42,44,46,-1,-1,48,-1,-1,50,52,54,56,58,60,62,64,66,-1,-1,-1,-1,-1,-1,68,-1,70,72,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,9.7467445E4,1.064816E6,5.38168E6,1.484798E9,9.256843E3,1.5096262E7,7.164074E8,-2.6642542E-2,1.5271514E5,4.9014646E9,9.249757E-3,3.521117E-2,2.4950776E0,1.92144E5,2.9743198E1,3.0972284E1,1.7657828E3,8.036632E-3,2.4472732E7,5.848343E7,-1.9506584E-3,2.1360708E7,5.03713E-3,-9.930492E-3,7.192302E2,7.805608E4,-7.248193E-3,1.577058E9,4.2378E4,7.9E1,4.5896373E-3,1.0395679E-2,1.5029658E-1,4.370916E-3,6.81605E-3,3.020675E5,8.287E3,3.0664597E1,2.91E2,5.863807E10,2.7753216E5,5.473125E3,4.17E2,4.565957E6,5.9696054E-3,6.548828E-4,1.6946608E-2,7.1020764E-3,1.8498797E-2,3.3324452E-3,5.1698097E1,1.1848042E-2,1.6991062E3,5.3403894E5,7.154E3,-2.612946E-3,-6.419736E-3,-2.415539E-4,4.069E3,3.0883285E-3,6.620703E-3,1.641392E-3,-5.2588433E-3,3.7780224E-4,1.2687654E-3,1.1762006E9,-1.5656314E-3,-7.831419E-3,5.0909584E-3,-5.0616096E-4,-2.2921336E-3,-6.7132553E-3,-1.1126449E-2,-5.67272E-3,2.6964778E-3,-3.86464E-3,-9.84731E-3,-2.7794978E-3],"split_indices":[61,42,38,56,7,4,9,7,0,37,12,0,0,63,2,62,65,61,0,9,1,0,1,0,0,61,37,0,7,11,3,0,0,47,0,0,42,2,65,8,40,56,41,0,59,0,0,0,0,0,0,57,0,42,57,2,0,0,0,2,0,0,0,0,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,3.68E2,6.6E1,2.56E2,1.12E2,1E1,5.6E1,2.53E2,3E0,7.3E1,3.9E1,3E0,7E0,4.5E1,1.1E1,1.99E2,5.4E1,6E1,1.3E1,2.1E1,1.8E1,2E0,4.3E1,8E0,3E0,1.11E2,8.8E1,6E0,4.8E1,3.4E1,2.6E1,1.7E1,4E0,1.5E1,3E0,1E1,3.3E1,6.3E1,4.8E1,6.2E1,2.6E1,2.7E1,2.1E1,2E1,1.4E1,5E0,2.1E1,1E1,5E0,3E1,3E0,6.2E1,1E0,1.3E1,3.5E1,5.7E1,5E0,2.3E1,3E0,2.2E1,5E0,1.1E1,1E1,3E0,1.7E1,3E0,1.1E1,1E1,5.2E1,8E0,5E0,2.6E1,9E0,4.9E1,8E0,2E0,2E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[5.1406133E-3,-8.329783E-2,1.7729796E-1,-1.62368E-1,-4.414073E-2,2.0477778E-1,-1.9106117E-1,-1.838515E-1,-2.1869311E-2,1.4158789E-3,-8.7907135E-2,1.5968002E-1,3.8107654E-1,3.5836063E-3,-2.5679097E-2,-2.2192629E-1,-1.5067194E-1,3.9023454E-3,-2.7610606E-3,-1.4224315E-2,6.772987E-3,-1.051854E-1,2.6978148E-2,8.067765E-2,2.1799447E-1,2.8783487E-2,2.2797035E-1,-8.4655E-3,1.461936E-1,-2.4171616E-1,-4.946066E-3,-2.6638838E-3,-8.11837E-3,-4.274031E-2,4.942626E-2,-1.1350785E-1,2.6841285E-3,4.0157833E-3,-2.9885303E-3,1.4880937E-1,4.722665E-2,2.3059192E-1,2.6850242E-3,3.3191025E-1,-1.08655415E-1,1.3021119E-2,2.300513E-3,-1.1996344E-2,-7.006335E-4,-5.4613374E-2,2.813396E-3,-6.679992E-4,9.77526E-2,-8.8024974E-2,-1.6404486E-1,3.4577295E-3,1.14603415E-2,-2.8634728E-3,6.697547E-2,1.5647367E-1,2.6112297E-1,7.6124407E-3,1.9819152E-2,-9.5713865E-3,6.045472E-3,-1.0182977E-1,-1.3250802E-2,6.948446E-3,1.6094607E-3,-9.457089E-3,-7.1778186E-2,-9.379711E-2,-2.225751E-1,6.6230264E-3,1.45944E-3,8.449794E-3,-4.0195606E-4,1.7254863E-2,2.1837129E-1,-7.6989E-3,-2.3512274E-3,1.9064444E-3,-1.9417513E-3,-1.1991619E-3,-5.213254E-3,2.1676552E-3,-6.544165E-3,-1.16096595E-2,3.411483E-4,1.3733777E-3,1.1113035E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,-1,35,37,39,41,-1,43,-1,45,47,-1,-1,-1,49,51,53,-1,-1,-1,55,57,59,-1,61,63,-1,-1,-1,-1,65,-1,-1,67,69,71,-1,-1,-1,73,75,77,-1,-1,-1,-1,79,81,-1,-1,-1,83,85,87,-1,-1,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.7748146E0,9.0310335E-1,1.5676126E0,2.8876448E-1,3.9490655E-1,1.060195E0,7.331436E-1,7.267952E-2,5.5577483E-2,2.1513487E-1,2.0332444E-1,5.0760865E-1,7.8944016E-1,2.3006608E-1,0E0,6.259155E-2,7.178402E-2,0E0,0E0,1.636528E-1,0E0,1.24124825E-1,7.529256E-2,1.06925935E-1,1.200614E-1,0E0,6.6586304E-1,0E0,4.8106387E-2,4.630673E-2,0E0,0E0,0E0,7.760224E-2,8.698163E-2,9.373748E-2,0E0,0E0,0E0,9.258938E-2,7.767089E-2,9.491563E-2,0E0,1.2791765E-1,1.3003832E-1,0E0,0E0,0E0,0E0,1.08018205E-1,0E0,0E0,4.2936742E-2,8.7800056E-2,9.013677E-2,0E0,0E0,0E0,7.335159E-2,6.33485E-2,9.1426134E-2,0E0,0E0,0E0,0E0,7.2023004E-2,4.5812704E-2,0E0,0E0,0E0,8.596307E-2,8.755176E-2,6.0249746E-2,0E0,0E0,0E0,0E0,0E0,6.4741015E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,23,23,24,24,26,26,28,28,29,29,33,33,34,34,35,35,39,39,40,40,41,41,43,43,44,44,49,49,52,52,53,53,54,54,58,58,59,59,60,60,65,65,66,66,70,70,71,71,72,72,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,-1,36,38,40,42,-1,44,-1,46,48,-1,-1,-1,50,52,54,-1,-1,-1,56,58,60,-1,62,64,-1,-1,-1,-1,66,-1,-1,68,70,72,-1,-1,-1,74,76,78,-1,-1,-1,-1,80,82,-1,-1,-1,84,86,88,-1,-1,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,9.50639E5,1.0429407E3,1.2254369E7,1.3508742E7,6.6512886E3,6.6053805E6,1.86005E5,2.409091E0,1.577058E9,1.7076351E3,2.5096E4,1.417988E6,1.4414646E-1,-2.5679097E-2,7.6E1,1.43479E5,3.9023454E-3,-2.7610606E-3,2.455358E6,6.772987E-3,6.46E2,6.935338E1,4.19719E5,1.773296E6,2.8783487E-2,1.27E2,-8.4655E-3,1.3812E4,1.6395413E7,-4.946066E-3,-2.6638838E-3,-8.11837E-3,4.4536823E1,1E0,3.069012E6,2.6841285E-3,4.0157833E-3,-2.9885303E-3,2.6979439E1,8.393928E4,4.9014646E9,2.6850242E-3,6.274818E4,3.677E3,1.3021119E-2,2.300513E-3,-1.1996344E-2,-7.006335E-4,1E0,2.813396E-3,-6.679992E-4,2.5159248E1,3.2471478E2,2.279073E7,3.4577295E-3,1.14603415E-2,-2.8634728E-3,1.5191719E5,2.4851996E7,4.5E1,7.6124407E-3,1.9819152E-2,-9.5713865E-3,6.045472E-3,4.5593824E4,1E0,6.948446E-3,1.6094607E-3,-9.457089E-3,6.573499E3,1.014E3,5.0328052E7,6.6230264E-3,1.45944E-3,8.449794E-3,-4.0195606E-4,1.7254863E-2,1.7648785E-1,-7.6989E-3,-2.3512274E-3,1.9064444E-3,-1.9417513E-3,-1.1991619E-3,-5.213254E-3,2.1676552E-3,-6.544165E-3,-1.16096595E-2,3.411483E-4,1.3733777E-3,1.1113035E-2],"split_indices":[2,1,67,9,54,61,56,9,67,7,61,2,38,47,0,8,12,0,0,1,0,8,63,38,11,0,8,0,9,12,0,0,0,62,13,9,0,0,0,67,42,12,0,37,0,0,0,0,0,22,0,0,65,61,54,0,0,0,42,54,3,0,0,0,0,42,23,0,0,0,42,10,54,0,0,0,0,0,50,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,2.93E2,1.5E2,9.6E1,1.97E2,1.4E2,1E1,8.3E1,1.3E1,9.7E1,1E2,1.13E2,2.7E1,7E0,3E0,3.6E1,4.7E1,3E0,1E1,8.8E1,9E0,8.7E1,1.3E1,4.9E1,6.4E1,1E1,1.7E1,3E0,4E0,3E1,6E0,8E0,3.9E1,6.1E1,2.7E1,8.3E1,4E0,8E0,5E0,1.5E1,3.4E1,5.9E1,5E0,1.3E1,4E0,1E0,3E0,2.9E1,1E0,5.5E1,6E0,1.2E1,1.5E1,5.7E1,2.6E1,9E0,6E0,5E0,2.9E1,1.9E1,4E1,5E0,8E0,3E0,1E0,2.5E1,3E1,8E0,7E0,6E0,5.1E1,1.3E1,1.3E1,9E0,2E1,1.7E1,2E0,1E1,3E1,1.1E1,1.4E1,1E1,2E1,2.3E1,2.8E1,3E0,1E1,1.2E1,1E0,2E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[7.838265E-3,-7.870466E-2,1.8087007E-1,-1.0357306E-1,2.2554666E-2,1.2787387E-1,3.797597E-1,-1.2305615E-1,-3.5571423E-4,-9.530737E-3,1.3435265E-1,-1.8022212E-1,1.4631748E-1,5.821971E-1,2.4943112E-1,-1.4175273E-1,-3.3001788E-2,-4.0554643E-2,1.0200117E-1,-2.2201559E-2,4.76423E-3,9.880461E-3,2.1499926E-3,4.9963794E-3,-1.5163233E-2,9.833416E-2,2.383296E-1,1.3986736E-2,3.160449E-2,1.3447568E-2,-7.4863797E-3,-1.9656373E-3,-1.5488529E-1,-3.7833187E-3,2.5891308E-2,-7.059665E-2,9.923144E-4,8.492066E-3,1.6626504E-3,1.4386227E-2,-3.045676E-3,7.744659E-2,7.898408E-3,1.8103166E-2,1.9967654E-1,-1.6523597E-1,-6.84487E-2,-3.6320838E-3,2.6540235E-3,-6.8554883E-3,-1.6814882E-3,4.7046244E-3,-8.262485E-3,1.5356647E-2,1.0281665E-1,3.0480078E-3,1.0775661E-2,-1.4110039E-2,-1.5761566E-1,2.4499227E-3,-5.258077E-3,-3.2601273E-3,1.5426447E-3,-3.958923E-3,4.9068622E-2,1.6007411E-1,5.558952E-2,2.4592914E-3,-7.787726E-3,1.3614587E-4,5.316188E-3,2.294423E-3,1.0238768E-2,4.6448936E-4,4.8574028E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,-1,-1,-1,41,43,-1,-1,-1,-1,-1,45,-1,47,49,-1,-1,-1,51,-1,53,-1,-1,55,57,59,-1,-1,-1,-1,-1,61,63,65,-1,-1,-1,67,-1,-1,-1,-1,-1,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.513246E0,7.326014E-1,1.4797955E0,4.6851754E-1,2.0933132E-1,6.8552864E-1,6.462841E-1,3.2532883E-1,1.6049282E-1,6.5341175E-2,7.32089E-2,2.9936168E-1,4.6221924E-1,4.5324326E-2,2.8184986E-1,2.0852947E-1,9.574543E-2,5.2878466E-2,4.897856E-2,6.3860476E-2,0E0,0E0,0E0,0E0,0E0,8.852303E-2,1.21148586E-1,0E0,0E0,0E0,0E0,0E0,1.1662841E-1,0E0,5.1361144E-2,4.36292E-2,0E0,0E0,0E0,4.4140458E-2,0E0,8.9784145E-2,0E0,0E0,8.659959E-2,7.717633E-2,8.786267E-2,0E0,0E0,0E0,0E0,0E0,4.7561172E-2,6.317213E-2,1.0196993E-1,0E0,0E0,0E0,1.0755348E-1,0E0,0E0,0E0,0E0,0E0,3.9304428E-2,9.5746875E-2,4.8830926E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,25,25,26,26,32,32,34,34,35,35,39,39,41,41,44,44,45,45,46,46,52,52,53,53,54,54,58,58,64,64,65,65,66,66],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,-1,-1,-1,42,44,-1,-1,-1,-1,-1,46,-1,48,50,-1,-1,-1,52,-1,54,-1,-1,56,58,60,-1,-1,-1,-1,-1,62,64,66,-1,-1,-1,68,-1,-1,-1,-1,-1,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.164074E8,6.6512886E3,1.646359E5,1.5522031E9,2.4950776E0,1.681178E6,3.823456E6,1.1669E4,1E0,6.088569E6,2.203418E7,6.8963E4,9.256843E3,7.79557E5,7.117012E1,1.012892E6,5.9827844E2,1.3482595E8,9.481747E8,4.76423E-3,9.880461E-3,2.1499926E-3,4.9963794E-3,-1.5163233E-2,1.7278508E5,2.1207E4,1.3986736E-2,3.160449E-2,1.3447568E-2,-7.4863797E-3,-1.9656373E-3,1.347372E3,-3.7833187E-3,2.7E1,2.1592189E5,9.923144E-4,8.492066E-3,1.6626504E-3,2.7809315E-3,-3.045676E-3,5.2798065E1,7.898408E-3,1.8103166E-2,1E1,6.506511E0,3.016494E3,-3.6320838E-3,2.6540235E-3,-6.8554883E-3,-1.6814882E-3,4.7046244E-3,2.192E3,1.2009271E3,1.16E2,3.0480078E-3,1.0775661E-2,-1.4110039E-2,2.669E3,2.4499227E-3,-5.258077E-3,-3.2601273E-3,1.5426447E-3,-3.958923E-3,1.6195753E9,4.1E1,2.6196917E3,2.4592914E-3,-7.787726E-3,1.3614587E-4,5.316188E-3,2.294423E-3,1.0238768E-2,4.6448936E-4,4.8574028E-3],"split_indices":[2,7,61,37,7,63,38,59,10,108,54,1,2,4,2,59,38,61,7,7,0,0,0,0,0,37,9,0,0,0,0,0,61,0,8,37,0,0,0,48,0,65,0,0,8,65,4,0,0,0,0,0,10,61,3,0,0,0,9,0,0,0,0,0,7,3,61,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.89E2,1.44E2,2.32E2,5.7E1,1.15E2,2.9E1,1.95E2,3.7E1,4.5E1,1.2E1,6E0,1.09E2,1E1,1.9E1,1.61E2,3.4E1,2.7E1,1E1,4.1E1,4E0,6E0,6E0,2E0,4E0,7.3E1,3.6E1,3E0,7E0,1.8E1,1E0,1.9E1,1.42E2,1.9E1,1.5E1,1.8E1,9E0,4E0,6E0,2.2E1,1.9E1,5.7E1,1.6E1,6E0,3E1,1.26E2,1.6E1,3E0,1.2E1,5E0,1.3E1,4E0,1.8E1,1.7E1,4E1,5E0,2.5E1,5E0,1.21E2,4E0,1.2E1,7E0,1.1E1,4E0,1.3E1,1.7E1,2.3E1,2E0,1.19E2,8E0,5E0,6E0,1.1E1,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[2.3300913E-3,-7.6258816E-2,1.622018E-1,-1.4439322E-1,-3.6712144E-2,1.298466E-1,3.375373E-2,-1.6334549E-1,-2.2670983E-3,-5.098105E-2,9.2889205E-2,1.5716018E-1,-1.7004254E-1,-1.7295861E-1,-2.0825E-3,-2.2906568E-2,-9.951552E-2,-5.1017934E-3,1.22489944E-1,1.01189405E-1,2.2200145E-1,-2.9357769E-2,-2.141137E-2,-3.944438E-3,-9.026607E-3,-8.365091E-2,6.954281E-3,-1.2366608E-2,-8.124664E-2,9.769951E-3,2.910159E-3,1.968917E-2,8.614994E-2,1.3315377E-1,2.4724495E-1,-1.2160351E-1,9.434323E-3,-1.0403149E-1,7.543168E-4,-1.2504661E-1,2.044034E-2,-1.0742652E-1,-7.4465517E-3,1.0935618E-1,2.9004732E-2,3.233815E-3,9.599661E-3,1.4336792E-1,1.3011546E-2,1.982325E-3,-7.501204E-3,-6.6465326E-3,-2.3724323E-3,-9.968075E-3,2.545238E-3,-2.631491E-3,3.13523E-2,-9.054415E-3,-7.6444864E-2,3.7703945E-3,-3.2137437E-3,6.24414E-3,4.5150977E-2,-2.016932E-3,4.128134E-3,1.4108729E-3,9.068953E-3,7.6329324E-4,4.4416194E-3,-5.2488716E-3,-1.7947921E-3,-7.147823E-4,4.7577745E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,-1,25,27,-1,29,31,33,35,-1,-1,-1,37,39,-1,41,-1,-1,-1,43,45,47,49,-1,51,-1,53,55,57,59,61,63,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,67,-1,69,-1,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.528028E0,7.895732E-1,2.432434E0,1.9162822E-1,3.5239816E-1,1.157088E0,0E0,9.64849E-2,0E0,2.29359E-1,1.2471287E-1,4.3844628E-1,4.495854E-1,9.252024E-2,0E0,1.9829437E-1,1.5695506E-1,0E0,7.2418034E-2,2.881918E-1,9.716225E-2,2.1061315E-1,0E0,0E0,0E0,7.448071E-2,1.342887E-1,0E0,1.0932842E-1,0E0,0E0,0E0,8.700472E-2,5.2014202E-2,6.467438E-2,4.5090713E-2,0E0,4.8841327E-2,0E0,1.1260678E-1,5.7375167E-2,9.02144E-2,8.634387E-2,5.6244433E-2,8.779637E-2,0E0,0E0,4.55274E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.513368E-2,0E0,3.6633104E-2,0E0,0E0,0E0,4.3195702E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,25,25,26,26,28,28,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,42,42,43,43,44,44,47,47,56,56,58,58,62,62],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,-1,26,28,-1,30,32,34,36,-1,-1,-1,38,40,-1,42,-1,-1,-1,44,46,48,50,-1,52,-1,54,56,58,60,62,64,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,68,-1,70,-1,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,1.074952E6,1.5324995E4,1.1482625E7,1.4096699E9,9.05929E2,3.375373E-2,2.812953E5,-2.2670983E-3,1.7377544E7,1.7102936E6,5.3515047E-1,7.339209E6,7.581957E1,-2.0825E-3,1.732734E6,1.9120918E7,-5.1017934E-3,6.088569E6,8.504733E7,1.6441814E9,3.305542E4,-2.141137E-2,-3.944438E-3,-9.026607E-3,7.164074E8,2.9E1,-1.2366608E-2,2.231E3,9.769951E-3,2.910159E-3,1.968917E-2,1.8322262E7,1.0273642E5,7.796878E4,4.6976192E8,9.434323E-3,4.759E3,7.543168E-4,6.746E3,2.5320764E2,3.8499493E1,3.0840238E7,1.921711E5,2.0273019E6,3.233815E-3,9.599661E-3,7.1853364E3,1.3011546E-2,1.982325E-3,-7.501204E-3,-6.6465326E-3,-2.3724323E-3,-9.968075E-3,2.545238E-3,-2.631491E-3,5.837049E6,-9.054415E-3,2.7E2,3.7703945E-3,-3.2137437E-3,6.24414E-3,3.9560684E3,-2.016932E-3,4.128134E-3,1.4108729E-3,9.068953E-3,7.6329324E-4,4.4416194E-3,-5.2488716E-3,-1.7947921E-3,-7.147823E-4,4.7577745E-3],"split_indices":[2,1,61,9,7,67,0,56,0,54,54,44,56,63,0,1,54,0,54,7,7,4,0,0,0,7,3,0,0,0,0,0,54,37,37,7,0,2,0,2,61,65,54,42,56,0,0,4,0,0,0,0,0,0,0,0,1,0,3,0,0,0,4,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,2.94E2,1.44E2,1.07E2,1.87E2,1.37E2,7E0,8.9E1,1.8E1,1.69E2,1.8E1,1.26E2,1.1E1,8.2E1,7E0,1.08E2,6.1E1,2E0,1.6E1,6.9E1,5.7E1,8E0,3E0,1.2E1,7E1,3.5E1,7.3E1,5E0,5.6E1,6E0,1E1,2E0,6.7E1,1.4E1,4.3E1,6E0,2E0,2.9E1,6E0,6E0,6.7E1,4.1E1,1.5E1,4.7E1,2E1,8E0,6E0,9E0,3.4E1,1E0,5E0,1.7E1,1.2E1,4E0,2E0,8E0,5.9E1,1E1,3.1E1,6E0,9E0,3.5E1,1.2E1,9E0,1.1E1,3E0,6E0,4.8E1,1.1E1,1.6E1,1.5E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[1.9300999E-3,-7.331161E-2,1.5396765E-1,-9.963632E-2,1.794134E-2,1.8647277E-1,-1.8792798E-1,-1.374043E-1,-4.9798567E-2,3.205413E-2,-1.006847E-2,1.426715E-1,3.682354E-1,-1.3204873E-2,-2.2412878E-2,-1.074198E-2,-1.2409844E-1,-9.1634E-2,1.976973E-2,1.0853977E-1,1.3787175E-2,1.5306465E-1,-1.0204066E-1,1.4551526E-2,2.765763E-2,-1.20415635E-1,1.509125E-1,-7.619929E-2,-1.3753729E-1,-3.3532005E-2,-1.416772E-1,-4.1243877E-2,3.4293907E-3,7.967105E-3,1.171733E-3,-1.595565E-3,1.3198744E-1,1.3003056E-1,4.536756E-1,8.9927325E-3,-1.0646346E-2,1.934535E-3,-7.740955E-3,1.2043335E-2,1.6729562E-3,-1.1838634E-1,-1.0443783E-3,-1.5302579E-1,-3.5704244E-3,-8.8420714E-4,-7.915164E-3,-1.5964466E-1,-4.402332E-4,3.701702E-3,-2.931257E-3,-1.9760812E-3,1.2823512E-3,9.137881E-3,5.439503E-4,1.6923034E-1,7.8075595E-2,2.864329E-2,9.636887E-3,-7.2064167E-3,2.1385211E-4,-1.6515625E-1,-2.5942752E-3,-2.2475228E-1,-1.0046563E-1,2.319988E-1,1.3504449E-1,-1.4889236E-3,9.9523485E-2,-8.557603E-3,-4.483181E-3,-1.179601E-2,3.379123E-4,-1.0946319E-3,-1.0043968E-2,-8.610183E-5,1.1854166E-2,3.8058755E-3,9.305877E-3,5.643791E-3,3.7018282E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,39,-1,-1,41,43,45,47,49,51,53,-1,-1,-1,55,57,59,61,-1,-1,-1,-1,-1,-1,63,-1,65,-1,-1,-1,67,-1,-1,-1,-1,-1,-1,-1,69,71,-1,-1,-1,-1,73,-1,75,77,79,81,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.976059E0,7.0083E-1,1.6353664E0,4.185505E-1,2.1790951E-1,9.9837303E-1,6.2284875E-1,1.177752E-1,2.8994313E-1,8.65246E-2,0E0,2.9029894E-1,1.6800666E-1,1.7652856E-1,0E0,0E0,6.225896E-2,1.7552114E-1,1.2200317E-1,5.329822E-2,9.475928E-2,6.611357E-1,2.1253069E-1,0E0,0E0,4.5097865E-2,3.7292674E-2,5.9558183E-2,7.4216485E-2,5.653087E-2,7.713282E-2,4.6198763E-2,0E0,0E0,0E0,5.327985E-2,3.952904E-2,1.8637633E-1,1.3068306E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.791317E-2,0E0,7.416129E-2,0E0,0E0,0E0,8.915728E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.66884E-2,1.0589582E-1,0E0,0E0,0E0,0E0,3.777671E-2,0E0,6.166327E-2,1.3784075E-1,5.8751404E-2,1.0834712E-1,0E0,5.8669925E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,35,35,36,36,37,37,38,38,45,45,47,47,51,51,59,59,60,60,65,65,67,67,68,68,69,69,70,70,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,40,-1,-1,42,44,46,48,50,52,54,-1,-1,-1,56,58,60,62,-1,-1,-1,-1,-1,-1,64,-1,66,-1,-1,-1,68,-1,-1,-1,-1,-1,-1,-1,70,72,-1,-1,-1,-1,74,-1,76,78,80,82,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.227E4,7.164074E8,1.0429407E3,1.732734E6,1E0,3.9596844E9,7.339209E6,1.44E2,1.0623282E3,1.136E1,-1.006847E-2,3.768034E4,5.22E2,3.236383E1,-2.2412878E-2,-1.074198E-2,1.22028E5,1.7377544E7,8.26042E4,2.25161E5,1E0,8.10288E3,1.92144E5,1.4551526E-2,2.765763E-2,4.6976192E8,2.9E1,2.2526654E-4,7.159259E4,2.865605E8,7.1015224E7,7.4E2,3.4293907E-3,7.967105E-3,1.171733E-3,4.31768E5,1.5E2,1.50301E6,2.0453656E5,8.9927325E-3,-1.0646346E-2,1.934535E-3,-7.740955E-3,1.2043335E-2,1.6729562E-3,3.019898E8,-1.0443783E-3,1.6415557E9,-3.5704244E-3,-8.8420714E-4,-7.915164E-3,2.47771E7,-4.402332E-4,3.701702E-3,-2.931257E-3,-1.9760812E-3,1.2823512E-3,9.137881E-3,5.439503E-4,1.1532659E7,8.879629E4,2.864329E-2,9.636887E-3,-7.2064167E-3,2.1385211E-4,1.0865863E7,-2.5942752E-3,2.4521964E7,2.748728E6,1E1,3.029061E6,-1.4889236E-3,8.689085E1,-8.557603E-3,-4.483181E-3,-1.179601E-2,3.379123E-4,-1.0946319E-3,-1.0043968E-2,-8.610183E-5,1.1854166E-2,3.8058755E-3,9.305877E-3,5.643791E-3,3.7018282E-4],"split_indices":[2,7,67,1,18,7,56,0,61,67,0,4,3,65,0,0,12,54,37,38,108,61,2,0,0,7,3,50,42,7,12,10,0,0,0,38,3,9,42,0,0,0,0,0,0,7,0,5,0,0,0,54,0,0,0,0,0,0,0,54,42,0,0,0,0,9,0,1,9,8,56,0,65,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.9E2,1.43E2,2.25E2,6.5E1,1.31E2,1.2E1,1.27E2,9.8E1,6.2E1,3E0,1.07E2,2.4E1,8E0,4E0,1.5E1,1.12E2,6.1E1,3.7E1,1.1E1,5.1E1,1.03E2,4E0,2E1,4E0,5E0,3E0,2.6E1,8.6E1,2.9E1,3.2E1,1.7E1,2E1,6E0,5E0,4.6E1,5E0,9.7E1,6E0,1E0,3E0,1E0,4E0,1E0,2E0,1.4E1,1.2E1,6.8E1,1.8E1,2.7E1,2E0,2.8E1,4E0,2E0,1.5E1,1.9E1,2.7E1,3E0,2E0,5.4E1,4.3E1,3E0,3E0,1.1E1,3E0,6E1,8E0,1.2E1,1.6E1,1.7E1,3.7E1,7E0,3.6E1,5E1,1E1,1.1E1,1E0,1E1,6E0,1E0,1.6E1,2E1,1.7E1,3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.0176436E-2,-7.4507125E-2,1.3613306E-1,-7.779444E-2,1.7937126E-2,4.8686538E-2,2.3640873E-1,-8.973379E-2,3.7875865E-2,-1.9444754E-2,8.086803E-2,1.9837126E-1,4.7522628E-1,-1.7825517E-1,-7.733473E-2,4.481E-3,-1.427942E-1,-4.294992E-3,1.6752258E-3,1.2334795E-1,3.4409136E-2,2.1213052E-1,-8.439825E-3,2.768787E-2,1.26015E-2,-9.349344E-3,5.585717E-4,-7.303022E-2,-1.6183874E-2,-8.213011E-4,-9.711455E-3,9.944917E-4,-4.736446E-3,2.8596658E-3,9.256048E-3,-3.166927E-3,5.1670153E-2,2.7053734E-2,1.9276088E-1,-9.603236E-2,-3.7901662E-2,5.514791E-3,2.4003914E-2,2.3687275E-3,2.0391154E-1,-1.0045947E-1,1.0278842E-2,-8.56274E-2,5.156005E-3,-2.5212534E-3,2.2346051E-3,2.142106E-1,3.6411858E-3,-6.533959E-3,-2.855761E-3,-6.8693897E-3,-7.016528E-4,3.1192042E-3,-2.1539258E-3,1.1227724E-2,5.7102065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,33,35,37,-1,-1,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,41,-1,43,45,47,-1,49,-1,51,53,-1,55,57,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4580555E0,3.9590633E-1,1.4616411E0,3.4727478E-1,0E0,2.0109598E-1,6.1428213E-1,2.3426831E-1,2.5201645E-1,4.4269003E-2,1.18117064E-1,3.9164615E-1,6.1694384E-2,8.2466245E-2,1.9787467E-1,0E0,4.0802874E-2,0E0,4.746627E-2,1.2923819E-1,5.6951825E-2,3.4471726E-1,0E0,0E0,0E0,0E0,0E0,1.5635073E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.4095777E-2,0E0,9.4086885E-2,1.843195E-1,1.6483821E-1,0E0,3.6514852E-2,0E0,6.3162565E-2,1.6252506E-1,0E0,1.5145409E-1,1.3045421E-1,0E0,0E0,7.0684195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,21,21,27,27,36,36,38,38,39,39,40,40,42,42,44,44,45,45,47,47,48,48,51,51],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,34,36,38,-1,-1,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,42,-1,44,46,48,-1,50,-1,52,54,-1,56,58,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,2.5096E4,1.577058E9,1.7937126E-2,1.1016051E3,1.6941456E1,4.26944E5,3.3503532E7,4E3,9.756896E6,3.768034E4,3.964E3,8.332E3,5.825042E6,4.481E-3,3.087E3,-4.294992E-3,1E0,7.7444346E8,3.6340196E-2,1.01267644E11,-8.439825E-3,2.768787E-2,1.26015E-2,-9.349344E-3,5.585717E-4,2.09299E6,-1.6183874E-2,-8.213011E-4,-9.711455E-3,9.944917E-4,-4.736446E-3,2.8596658E-3,9.256048E-3,-3.166927E-3,7.625084E6,2.7053734E-2,3E0,1.2934E4,6.210482E2,5.514791E-3,3.107511E6,2.3687275E-3,3.0038656E5,2.551424E6,1.0278842E-2,2.72E2,2.41E2,-2.5212534E-3,2.2346051E-3,4.949877E6,3.6411858E-3,-6.533959E-3,-2.855761E-3,-6.8693897E-3,-7.016528E-4,3.1192042E-3,-2.1539258E-3,1.1227724E-2,5.7102065E-3],"split_indices":[42,61,2,7,0,61,44,1,54,2,54,4,0,2,56,0,0,0,18,7,47,40,0,0,0,0,0,1,0,0,0,0,0,0,0,0,57,0,8,2,61,0,1,0,42,9,0,3,3,0,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.16E2,2.49E2,1.67E2,2.48E2,1E0,9E1,7.7E1,2.25E2,2.3E1,2.9E1,6.1E1,6.8E1,9E0,2.6E1,1.99E2,1.8E1,5E0,6E0,2.3E1,3.1E1,3E1,6.6E1,2E0,5E0,4E0,2.4E1,2E0,1.97E2,2E0,2E0,3E0,2E1,3E0,1.7E1,1.4E1,4E0,2.6E1,2E0,6.4E1,1.18E2,7.9E1,7E0,1.9E1,5E0,5.9E1,1.17E2,1E0,3.7E1,4.2E1,4E0,1.5E1,5.4E1,5E0,6.2E1,5.5E1,2E1,1.7E1,1.9E1,2.3E1,4.4E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.9216714E-4,-3.9535843E-2,2.2272989E-1,-6.6878766E-2,1.0665421E-1,4.277139E-1,1.605131E-1,-9.227924E-2,1.1157016E-2,1.4368413E-1,3.280557E-2,1.11736255E-2,5.7022214E-1,2.1920699E-1,3.420795E-2,-8.5651234E-2,-2.203215E-2,-1.4970543E-2,6.8922914E-3,1.76376E-1,3.124016E-2,-4.2118E-3,3.192692E-3,6.7345966E-3,2.9497296E-2,2.517958E-1,6.893434E-4,4.6342155E-3,-9.956154E-3,-1.6474402E-1,-7.0912585E-2,-8.416858E-2,1.919229E-2,9.4996137E-4,9.009787E-3,-3.247179E-3,4.0073493E-3,6.399306E-3,1.3548461E-2,-1.8497705E-1,-1.5790408E-3,-7.3372185E-2,1.0031041E-2,-1.1257307E-3,-1.3290702E-1,3.590025E-3,5.7863286E-3,-9.368564E-3,-1.5985451E-3,-9.546795E-2,-3.6646515E-2,-6.498647E-4,-8.255483E-3,9.1457856E-4,-2.9071548E-3,-6.466885E-3,-2.2576787E-3,-4.745434E-3,8.914094E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,-1,31,-1,33,35,-1,-1,-1,-1,37,-1,-1,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,47,-1,49,-1,-1,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9051373E0,1.4941146E0,7.858274E-1,6.234232E-1,1.5662831E-1,2.867713E-1,3.854084E-1,5.336964E-1,2.707201E-1,1.3785416E-1,9.009107E-2,0E0,5.7673454E-2,2.3679984E-1,2.880588E-1,2.6109636E-1,0E0,1.5763947E-1,0E0,4.5611024E-2,5.6736704E-2,0E0,0E0,0E0,0E0,7.049072E-2,0E0,0E0,0E0,9.008551E-2,1.5180302E-1,6.2111855E-2,7.035254E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.7457347E-2,0E0,1.571747E-1,0E0,0E0,5.2598998E-2,4.0441286E-2,0E0,0E0,0E0,2.2500193E-1,2.606657E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,19,19,20,20,25,25,29,29,30,30,31,31,32,32,39,39,41,41,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,-1,32,-1,34,36,-1,-1,-1,-1,38,-1,-1,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,48,-1,50,-1,-1,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.5522031E9,1.417988E6,9.7467445E4,1.1532659E7,1.2636069E7,1.5096262E7,2.2233E4,1.5271514E5,1.59E2,2.1853803E4,1.11736255E-2,1.0468137E7,2.2576077E10,1.92144E5,4.15408E5,-2.203215E-2,2.0926301E3,6.8922914E-3,1.5428992E3,2.1597655E9,-4.2118E-3,3.192692E-3,6.7345966E-3,2.9497296E-2,2.097068E7,6.893434E-4,4.6342155E-3,-9.956154E-3,7.154E3,1.3428E4,9.302345E4,2.169762E0,9.4996137E-4,9.009787E-3,-3.247179E-3,4.0073493E-3,6.399306E-3,1.3548461E-2,3.6653217E2,-1.5790408E-3,2.025313E6,1.0031041E-2,-1.1257307E-3,1.4E1,7.0764465E1,5.7863286E-3,-9.368564E-3,-1.5985451E-3,3.006846E6,7.121916E2,-6.498647E-4,-8.255483E-3,9.1457856E-4,-2.9071548E-3,-6.466885E-3,-2.2576787E-3,-4.745434E-3,8.914094E-4],"split_indices":[61,7,38,42,54,54,9,2,37,3,42,0,1,12,2,1,0,4,0,4,7,0,0,0,0,1,0,0,0,2,2,37,50,0,0,0,0,0,0,63,0,1,0,0,3,65,0,0,0,9,61,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.71E2,6.6E1,3.13E2,5.8E1,1.4E1,5.2E1,2.36E2,7.7E1,3.8E1,2E1,7E0,7E0,3.5E1,1.7E1,2.33E2,3E0,6.5E1,1.2E1,2.9E1,9E0,4E0,1.6E1,1E0,6E0,3E1,5E0,1.4E1,3E0,3.5E1,1.98E2,2.1E1,4.4E1,2E0,2.7E1,3E0,6E0,7E0,2.3E1,3E1,5E0,1.97E2,1E0,1E1,1.1E1,3.9E1,5E0,2.8E1,2E0,1.22E2,7.5E1,3E0,8E0,3.2E1,7E0,6.7E1,5.5E1,3.5E1,4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[5.219767E-3,-5.6080297E-2,1.3728033E-1,-7.917224E-2,2.9089969E-2,1.4985004E-1,-3.3383703E-1,-1.1847408E-1,-3.0490423E-2,4.2870395E-2,-9.412915E-3,1.25115E-1,3.0917355E-1,-1.9449655E-2,-3.0180363E-3,-9.339219E-3,-1.06531836E-1,-3.722273E-2,1.6793577E-2,2.613165E-4,8.388548E-2,1.3460782E-1,-1.2393823E-1,2.2974385E-2,1.17620025E-2,-1.9236532E-3,-1.1593999E-1,9.956284E-3,-9.214951E-2,1.16944544E-1,-1.0436333E-3,9.063335E-2,1.886786E-1,-9.875729E-3,-8.3931074E-5,-6.236656E-3,-2.9557482E-3,-5.85564E-3,4.746902E-3,-4.019866E-3,-1.3742782E-1,4.937531E-2,1.6512278E-1,1.577636E-2,7.832042E-2,7.0276216E-4,2.0288357E-1,2.3220377E-2,-2.18122E-3,-6.843366E-2,2.0825404E-3,-1.9845867E-1,-9.789155E-2,3.8337316E-3,-2.7720497E-3,2.8308728E-3,9.927304E-3,5.2504092E-2,6.0941568E-3,2.4936022E-1,1.3125359E-1,-1.1136805E-3,3.7514516E-3,-5.3522675E-3,1.9056236E-3,-2.2884125E-3,-1.1741417E-2,-3.433952E-3,-9.648346E-3,4.4202895E-4,3.8307754E-3,1.5659632E-2,9.132874E-3,1.1274556E-3,7.485969E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,25,27,-1,-1,29,31,33,-1,-1,-1,35,37,39,41,-1,43,45,-1,-1,-1,-1,47,-1,49,51,53,55,-1,57,-1,59,61,-1,63,-1,65,67,-1,-1,-1,-1,69,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6263072E0,6.0412765E-1,8.9092207E-1,4.5748937E-1,2.1238492E-1,5.09099E-1,5.020961E-2,9.884119E-2,2.9092884E-1,9.631606E-2,0E0,3.0511725E-1,8.021307E-2,0E0,0E0,0E0,6.872332E-2,2.8078777E-1,0E0,0E0,1.0719341E-1,2.6347208E-1,4.9063027E-2,0E0,0E0,0E0,6.67969E-2,8.313346E-2,1.9896561E-1,6.6720486E-2,0E0,1.6953474E-1,1.2564564E-1,0E0,0E0,0E0,0E0,5.9450515E-2,0E0,5.7695463E-2,6.019938E-2,3.9998047E-2,4.4309914E-2,0E0,7.4032634E-2,0E0,1.270684E-1,7.764525E-2,0E0,4.1155104E-2,0E0,7.0087075E-2,4.5725048E-2,0E0,0E0,0E0,0E0,5.065293E-2,0E0,6.507051E-2,5.0749004E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,16,16,17,17,20,20,21,21,22,22,26,26,27,27,28,28,29,29,31,31,32,32,37,37,39,39,40,40,41,41,42,42,44,44,46,46,47,47,49,49,51,51,52,52,57,57,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,26,28,-1,-1,30,32,34,-1,-1,-1,36,38,40,42,-1,44,46,-1,-1,-1,-1,48,-1,50,52,54,56,-1,58,-1,60,62,-1,64,-1,66,68,-1,-1,-1,-1,70,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.164074E8,2.3028242E3,1.732734E6,1E0,6.001104E9,1.3936486E8,1.44E2,4.6454727E3,1.1576422E9,-9.412915E-3,1.2622761E8,2.043E3,-1.9449655E-2,-3.0180363E-3,-9.339219E-3,1.4297844E2,1.7239808E7,1.6793577E-2,2.613165E-4,1.93E2,5.6503E4,1.08935805E5,2.2974385E-2,1.17620025E-2,-1.9236532E-3,1.8523391E6,3.0224E4,1.4E4,2.9222983E3,-1.0436333E-3,8.504733E7,4.0235848E4,-9.875729E-3,-8.3931074E-5,-6.236656E-3,-2.9557482E-3,1.2202955E-2,4.746902E-3,1.534E3,6.2651404E2,2.6835732E3,1E0,1.577636E-2,1.5271514E5,7.0276216E-4,5.993377E6,4.8404854E-5,-2.18122E-3,9.1909076E2,2.0825404E-3,5.802715E2,6.54925E6,3.8337316E-3,-2.7720497E-3,2.8308728E-3,9.927304E-3,1.2592834E-1,6.0941568E-3,2.1207E4,6.45254E0,-1.1136805E-3,3.7514516E-3,-5.3522675E-3,1.9056236E-3,-2.2884125E-3,-1.1741417E-2,-3.433952E-3,-9.648346E-3,4.4202895E-4,3.8307754E-3,1.5659632E-2,9.132874E-3,1.1274556E-3,7.485969E-3],"split_indices":[2,7,67,1,18,7,1,0,4,7,0,1,0,0,0,0,42,54,0,0,3,2,37,0,0,0,59,10,11,4,0,7,42,0,0,0,0,66,0,11,61,4,23,0,37,0,38,47,0,61,0,42,9,0,0,0,0,47,0,9,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,3.05E2,1.41E2,2.4E2,6.5E1,1.38E2,3E0,1.32E2,1.08E2,6.2E1,3E0,1.21E2,1.7E1,2E0,1E0,1.6E1,1.16E2,1.07E2,1E0,3.3E1,2.9E1,1.17E2,4E0,3E0,1.4E1,1.5E1,1.01E2,5.8E1,4.9E1,2.2E1,7E0,6.6E1,5.1E1,2E0,2E0,8E1,2.1E1,5E1,8E0,1.7E1,3.2E1,1E1,1.2E1,2E0,6.4E1,4E0,4.7E1,2.9E1,2.1E1,7E0,1E1,1.1E1,2.1E1,8E0,2E0,4E0,8E0,4.3E1,2.1E1,2.7E1,2E1,1.6E1,1.3E1,5E0,2E0,3E0,8E0,1.8E1,3E0,1.7E1,2.6E1,1E1,1.7E1,4E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.0870942E-2,-7.338359E-2,1.14998214E-1,-6.56137E-2,-1.9764004E-2,6.0589567E-2,2.7461976E-1,-1.622227E-1,-4.9423434E-2,-3.134483E-2,9.0373665E-2,2.0141712E-1,2.706446E-2,-1.0347641E-2,-5.2822395E-3,-5.2563623E-2,1.349499E-2,-1.1131266E-2,-8.930414E-3,6.1830945E-2,1.4650801E-1,2.2216284E-1,-5.86237E-3,-6.40116E-2,1.7592685E-2,-4.89954E-3,8.8087143E-4,2.9704785E-2,1.0336216E-1,8.94223E-3,3.60797E-3,1.5520549E-2,1.8355013E-1,-8.567361E-2,-2.8700864E-2,-2.9753098E-2,4.0115556E-3,-1.1464598E-3,6.353156E-2,-1.7305921E-3,5.764897E-3,4.232447E-3,1.0126888E-2,-9.033983E-2,9.408877E-3,4.551983E-3,-9.859026E-2,-3.2317261E-3,2.398829E-3,3.877605E-3,-1.1326533E-3,-5.7459697E-3,-2.1473975E-3,-3.976342E-3,2.0482906E-3,-3.0650278E-3,-1.116388E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,-1,23,-1,25,-1,27,29,31,-1,33,35,-1,-1,37,39,-1,-1,-1,41,43,45,47,-1,-1,49,-1,-1,-1,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6493149E0,5.7449603E-1,1.5987325E0,3.4547478E-1,0E0,3.8617837E-1,8.733325E-1,6.478304E-2,2.1636051E-1,1.0679905E-1,1.6077733E-1,2.8834796E-1,0E0,0E0,0E0,1.5693694E-1,0E0,8.698678E-2,0E0,9.299484E-2,8.384824E-2,8.340287E-2,0E0,1.252796E-1,8.944965E-2,0E0,0E0,7.749439E-2,7.423389E-2,0E0,0E0,0E0,5.511135E-2,1.5259403E-1,1.5151016E-1,5.371228E-2,0E0,0E0,4.0104695E-2,0E0,0E0,0E0,0E0,1.2939107E-1,0E0,1.5203461E-1,8.3058804E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,19,19,20,20,21,21,23,23,24,24,27,27,28,28,32,32,33,33,34,34,35,35,38,38,43,43,45,45,46,46],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,-1,24,-1,26,-1,28,30,32,-1,34,36,-1,-1,38,40,-1,-1,-1,42,44,46,48,-1,-1,50,-1,-1,-1,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.393928E4,7.339209E6,9.3058E4,4.15408E5,-1.9764004E-2,9.9800183E2,9.207803E0,5.6E1,8.861901E3,1E0,1.4773067E5,2.0531621E4,2.706446E-2,-1.0347641E-2,-5.2822395E-3,1.1576422E9,1.349499E-2,3.957E3,-8.930414E-3,1.29186E9,2.3961474E7,2.1207E4,-5.86237E-3,2.09299E6,6.541155E1,-4.89954E-3,8.8087143E-4,3.724162E6,3.7860696E0,8.94223E-3,3.60797E-3,1.5520549E-2,3.706875E5,1.0043E4,2.7582266E7,1.173433E5,4.0115556E-3,-1.1464598E-3,1.2753316E9,-1.7305921E-3,5.764897E-3,4.232447E-3,1.0126888E-2,2.613714E6,9.408877E-3,3.9560037E2,6.54925E6,-3.2317261E-3,2.398829E-3,3.877605E-3,-1.1326533E-3,-5.7459697E-3,-2.1473975E-3,-3.976342E-3,2.0482906E-3,-3.0650278E-3,-1.116388E-2],"split_indices":[42,56,2,1,0,61,62,10,61,18,37,4,0,0,0,7,0,2,0,7,9,9,0,1,65,0,0,1,62,0,0,0,41,2,54,37,0,0,7,0,0,0,0,9,0,61,9,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.14E2,2.29E2,1.85E2,2.25E2,4E0,1.39E2,4.6E1,3.1E1,1.94E2,3.4E1,1.05E2,3.8E1,8E0,1.4E1,1.7E1,1.93E2,1E0,3.1E1,3E0,7.1E1,3.4E1,3.6E1,2E0,1.66E2,2.7E1,7E0,2.4E1,4.1E1,3E1,2.1E1,1.3E1,8E0,2.8E1,1.02E2,6.4E1,1.6E1,1.1E1,1.6E1,2.5E1,3E0,2.7E1,7E0,2.1E1,1.01E2,1E0,4.4E1,2E1,1.1E1,5E0,2.1E1,4E0,6.1E1,4E1,1.3E1,3.1E1,1.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[9.5061725E-3,-5.7985652E-2,1.2660201E-1,-1.1215921E-1,-2.984194E-2,8.378763E-2,3.6394173E-1,-1.196764E-1,4.1898093E-3,-3.690837E-2,5.3460198E-3,1.03856735E-1,-2.1094935E-1,2.7549397E-2,2.2451386E-1,-1.4006384E-1,-2.9212696E-2,-4.565681E-2,5.2882805E-2,4.6533667E-2,1.4000571E-1,1.1811161E-2,-2.98559E-1,1.3474539E-2,5.4938407E-3,-2.8787935E-3,-1.531468E-1,-1.2421516E-1,2.8639799E-2,-5.994508E-2,3.4621623E-3,3.8586603E-3,-4.4081854E-3,7.080843E-2,-1.0992911E-2,1.508509E-1,1.2658513E-3,-1.9001652E-2,-5.0385273E-3,-8.252552E-3,-4.754544E-3,7.997162E-4,-8.8646365E-3,-1.2511939E-3,4.2215143E-3,-9.969407E-2,-4.255124E-2,-2.728988E-3,1.6661321E-3,4.1612587E-3,-1.0152749E-3,-2.830017E-3,2.1237026E-3,1.8611057E-1,5.957702E-3,-1.8262179E-1,-7.0520155E-2,-3.0647438E-2,-5.934981E-3,9.5894635E-3,2.7263039E-3,-3.8235781E-3,-1.2025921E-2,9.832374E-4,-4.1408725E-3,7.4482895E-4,-2.364129E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,23,25,27,29,31,33,35,-1,37,-1,-1,-1,39,41,43,45,47,-1,-1,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4372482E0,4.1745508E-1,1.5851932E0,1.5105557E-1,1.860635E-1,8.26363E-1,5.6584764E-1,1.6374183E-1,0E0,1.398827E-1,0E0,2.5872266E-1,4.3261784E-1,0E0,6.536925E-2,6.8653464E-2,1.02487825E-1,1.12499535E-1,6.999509E-2,7.199705E-2,9.226537E-2,0E0,1.0713494E-1,0E0,0E0,0E0,4.5006752E-2,6.1828926E-2,4.08655E-2,8.148998E-2,7.1054086E-2,0E0,0E0,5.3065658E-2,4.4592235E-2,4.5317173E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.663658E-2,8.059651E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.6756635E-2,0E0,4.2333752E-2,4.22519E-2,6.685825E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,45,45,46,46,53,53,55,55,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,24,26,28,30,32,34,36,-1,38,-1,-1,-1,40,42,44,46,48,-1,-1,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,4.426E3,8.10288E3,7.10711E8,1.577058E9,2.3028242E3,3.0418E4,2.681357E6,4.1898093E-3,8.464347E8,5.3460198E-3,2.1959E4,1.3812E4,2.7549397E-2,6.755346E1,1.4953075E2,2.4094029E2,1.2307312E5,1E0,1.3508742E7,1.773296E6,1.1811161E-2,3.1E1,1.3474539E-2,5.4938407E-3,-2.8787935E-3,3.006846E6,2.946708E6,2.4E1,2.53383E5,7.46E2,3.8586603E-3,-4.4081854E-3,2.0863011E5,2.6079211E3,1.0632783E7,1.2658513E-3,-1.9001652E-2,-5.0385273E-3,-8.252552E-3,-4.754544E-3,7.997162E-4,-8.8646365E-3,-1.2511939E-3,4.2215143E-3,5.8522424E2,3.4867884E7,-2.728988E-3,1.6661321E-3,4.1612587E-3,-1.0152749E-3,-2.830017E-3,2.1237026E-3,2.8673116E7,5.957702E-3,8.09994E5,7.058968E0,2.884095E-3,-5.934981E-3,9.5894635E-3,2.7263039E-3,-3.8235781E-3,-1.2025921E-2,9.832374E-4,-4.1408725E-3,7.4482895E-4,-2.364129E-3],"split_indices":[2,2,61,7,7,67,10,1,0,12,0,2,9,0,66,42,61,37,18,54,11,0,3,0,0,0,9,1,8,9,0,0,0,42,61,54,0,0,0,0,0,0,0,0,0,61,54,0,0,0,0,0,0,9,0,1,67,48,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.75E2,1.58E2,9.3E1,1.82E2,1.35E2,2.3E1,9E1,3E0,1.74E2,8E0,1.27E2,8E0,8E0,1.5E1,7.3E1,1.7E1,1.59E2,1.5E1,5E1,7.7E1,1E0,7E0,9E0,6E0,1.1E1,6.2E1,6E0,1.1E1,1.23E2,3.6E1,1.3E1,2E0,3.5E1,1.5E1,7E1,7E0,4E0,3E0,4.5E1,1.7E1,2E0,4E0,6E0,5E0,3.6E1,8.7E1,1.2E1,2.4E1,3E1,5E0,8E0,7E0,2.8E1,4.2E1,8E0,2.8E1,7.7E1,1E1,2.5E1,3E0,4E0,4E0,4E0,2.4E1,2.2E1,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[8.067029E-4,-5.1621657E-2,1.2420879E-1,-1.0225879E-1,-1.973381E-2,9.0327695E-2,2.7790222E-1,-1.20855466E-1,-1.5616212E-2,-2.586163E-2,1.3025207E-2,1.0460117E-1,-8.184886E-3,4.5004284E-1,1.350945E-1,-1.2972738E-1,3.2947548E-3,-4.61026E-3,6.527448E-4,-4.2871717E-2,4.620462E-2,9.119106E-2,1.0616955E-2,6.649069E-3,5.108403E-1,9.061953E-3,-5.850524E-3,-1.4897244E-1,-7.716643E-2,-5.6896538E-2,1.8091183E-2,2.0307364E-2,7.260941E-3,6.453852E-2,1.3233852E-1,2.6604949E-2,5.405212E-3,-1.5800187E-1,-2.607092E-4,-2.963384E-4,-5.146869E-3,-6.55326E-2,2.8448137E-2,6.0960123E-5,5.927811E-3,6.143149E-3,-6.209359E-3,2.6883185E-2,9.76826E-2,1.0175824E-2,1.0606246E-1,-1.6314088E-1,-4.0570743E-5,-4.69812E-2,-1.1044295E-1,3.9512287E-3,-1.8664609E-3,-4.5380864E-2,1.929031E-3,7.4874237E-3,5.602761E-4,5.475882E-3,-1.8297895E-3,2.6508432E-3,6.696594E-3,-3.8437315E-3,-8.400597E-3,-3.809898E-3,-3.2538292E-4,-3.45307E-3,-1.0509589E-2,3.1221937E-4,-5.573849E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,-1,-1,-1,29,31,33,-1,-1,35,-1,-1,37,39,41,43,45,-1,47,49,-1,-1,51,-1,-1,-1,53,55,-1,-1,-1,57,59,61,-1,63,65,-1,67,69,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8207614E0,4.9208295E-1,6.518469E-1,1.8824196E-1,3.4270984E-1,4.186828E-1,5.0461066E-1,1.7167616E-1,5.2606273E-2,2.2998957E-1,0E0,1.3958454E-1,0E0,1.1932826E-1,2.1254402E-1,8.257735E-2,0E0,0E0,0E0,1.3013336E-1,9.426252E-2,9.454733E-2,0E0,0E0,6.875992E-2,0E0,0E0,8.5998535E-2,5.6064725E-2,9.2520475E-2,5.1010076E-2,8.599078E-2,0E0,7.1008995E-2,5.2069247E-2,0E0,0E0,5.0744653E-2,0E0,0E0,0E0,8.812407E-2,4.6288498E-2,0E0,0E0,0E0,4.7075868E-2,5.503174E-2,7.111028E-2,0E0,4.099238E-2,4.2533398E-2,0E0,1.0386811E-1,1.17984354E-1,0E0,0E0,5.172818E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,19,19,20,20,21,21,24,24,27,27,28,28,29,29,30,30,31,31,33,33,34,34,37,37,41,41,42,42,46,46,47,47,48,48,50,50,51,51,53,53,54,54,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,-1,-1,-1,30,32,34,-1,-1,36,-1,-1,38,40,42,44,46,-1,48,50,-1,-1,52,-1,-1,-1,54,56,-1,-1,-1,58,60,62,-1,64,66,-1,68,70,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,4.844E3,8.10288E3,2.0964778E5,3.058642E3,9.05929E2,3.3616E4,7.164074E8,6.52487E2,3.53E2,1.3025207E-2,6.8545566E0,-8.184886E-3,6.7211235E1,5.01E2,3.006846E6,3.2947548E-3,-4.61026E-3,6.527448E-4,5.9170656E8,1.3738772E5,1.5271514E5,1.0616955E-2,6.649069E-3,2.1592189E5,9.061953E-3,-5.850524E-3,1.3421E4,2.37E2,4.723148E11,3.1519476E-1,4.55E3,7.260941E-3,5.589612E1,2.8E1,2.6604949E-2,5.405212E-3,3.5111064E7,-2.607092E-4,-2.963384E-4,-5.146869E-3,5.559E3,2.58E2,6.0960123E-5,5.927811E-3,6.143149E-3,1.570662E5,5.473125E3,4.2663252E7,1.0175824E-2,4.7232704E1,1.2919344E2,-4.0570743E-5,2.78E2,1.91E2,3.9512287E-3,-1.8664609E-3,3.4129692E-3,1.929031E-3,7.4874237E-3,5.602761E-4,5.475882E-3,-1.8297895E-3,2.6508432E-3,6.696594E-3,-3.8437315E-3,-8.400597E-3,-3.809898E-3,-3.2538292E-4,-3.45307E-3,-1.0509589E-2,3.1221937E-4,-5.573849E-3],"split_indices":[2,2,61,37,61,67,10,7,4,8,0,44,0,65,3,9,0,0,0,12,42,37,0,0,37,0,0,10,10,40,66,12,0,65,3,0,0,54,0,0,0,10,3,0,0,0,56,41,54,0,67,42,0,10,8,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,3.05E2,1.29E2,1.17E2,1.88E2,1.07E2,2.2E1,9.6E1,2.1E1,1.85E2,3E0,1.02E2,5E0,9E0,1.3E1,9.2E1,4E0,5E0,1.6E1,1.5E2,3.5E1,9.3E1,9E0,2E0,7E0,1.1E1,2E0,6.6E1,2.6E1,1.22E2,2.8E1,2.9E1,6E0,5.8E1,3.5E1,6E0,1E0,6.2E1,4E0,8E0,1.8E1,1.11E2,1.1E1,2.5E1,3E0,5E0,2.4E1,2.8E1,3E1,7E0,2.8E1,6E1,2E0,8E1,3.1E1,6E0,5E0,1.3E1,1.1E1,2E0,2.6E1,2.7E1,3E0,1.2E1,1.6E1,8E0,5.2E1,4.4E1,3.6E1,2.4E1,7E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-4.1973814E-3,-6.2179834E-2,7.9542436E-2,-5.6815293E-2,-1.8673219E-2,2.6421305E-2,1.5214318E-1,-1.0664655E-1,-2.9756274E-2,-2.1946229E-2,6.0876228E-2,9.878971E-2,2.3336628E-1,-1.2290533E-1,-1.8242896E-2,3.4819506E-2,-5.0532866E-2,1.727478E-3,-1.1420259E-1,1.2369924E-1,2.2265827E-2,1.16214015E-1,-9.426448E-3,1.7721008E-1,2.6038526E-2,-1.6061133E-1,-8.3981544E-2,-3.0060543E-3,2.8110957E-3,1.4102624E-1,-3.9899666E-3,-6.817293E-2,-4.5390045E-3,3.0139957E-3,-2.405481E-2,-7.877539E-3,-5.7305204E-4,7.396521E-3,1.6339152E-3,5.9819873E-3,3.6460985E-3,8.925443E-3,9.1239326E-2,-1.4948007E-3,9.005705E-3,-8.364532E-3,-2.0998584E-3,1.2110722E-3,-1.0863193E-1,-1.3131966E-4,9.415261E-3,2.9319994E-2,-5.9174072E-2,-4.546503E-2,-1.0670607E-1,-4.077768E-2,2.3584168E-3,-2.6755203E-3,1.5328401E-3,-3.7313828E-3,1.3269727E-3,5.0462103E-3,4.3087793E-4,-6.195663E-3,-9.31714E-4,-2.0025484E-3,3.0208318E-3,-4.8206975E-3,8.655987E-4,-7.761519E-2,-6.7128404E-4,-1.2136137E-1,3.8717715E-3,-3.4328848E-3,1.6673185E-3,-2.5317392E-3,-7.1851914E-3,-4.6793654E-4,-6.411654E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,-1,45,47,-1,-1,49,51,53,55,-1,57,-1,-1,-1,-1,-1,59,-1,61,-1,-1,-1,-1,-1,63,-1,-1,65,67,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,75,-1,77,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.112207E0,4.2621237E-1,6.824608E-1,3.393163E-1,0E0,1.7465198E-1,3.0196095E-1,1.2568069E-1,2.2447589E-1,9.6295476E-2,1.4520864E-1,2.6373076E-1,4.0372682E-1,9.658611E-2,5.4272696E-2,1.6955762E-1,1.0191876E-1,5.7920083E-2,4.3344E-2,5.616206E-2,7.347228E-2,6.054437E-2,0E0,5.7590246E-2,0E0,4.818493E-2,1.0761276E-1,0E0,0E0,8.630924E-2,5.8718573E-2,7.4774235E-2,7.172851E-2,0E0,4.7408424E-2,0E0,0E0,0E0,0E0,0E0,6.791113E-2,0E0,3.7720323E-2,0E0,0E0,0E0,0E0,0E0,5.4292202E-2,0E0,0E0,4.983577E-2,3.9562948E-2,5.8751233E-2,1.001865E-1,5.2895404E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.4165656E-2,0E0,3.8918436E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,26,26,29,29,30,30,31,31,32,32,34,34,40,40,42,42,48,48,51,51,52,52,53,53,54,54,55,55,69,69,71,71],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,-1,46,48,-1,-1,50,52,54,56,-1,58,-1,-1,-1,-1,-1,60,-1,62,-1,-1,-1,-1,-1,64,-1,-1,66,68,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,76,-1,78,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,2.5096E4,4.407E3,-1.8673219E-2,2.5353972E3,1.486066E0,2.0964778E5,1.031356E1,1.3026638E7,8.237054E6,4.52E2,3.305542E4,2.53383E5,4.174811E2,9.7E1,1.2028214E5,1.0832388E1,2.0690203E3,1.93E2,4.194301E7,1.0548451E7,-9.426448E-3,1.4797951E-2,2.6038526E-2,2.560923E3,2.184632E4,-3.0060543E-3,2.8110957E-3,9.56503E5,3.0840238E7,4.2056076E-2,2.025313E6,3.0139957E-3,6.2581446E8,-7.877539E-3,-5.7305204E-4,7.396521E-3,1.6339152E-3,5.9819873E-3,3.107511E6,8.925443E-3,1.5953E4,-1.4948007E-3,9.005705E-3,-8.364532E-3,-2.0998584E-3,1.2110722E-3,1.56439E3,-1.3131966E-4,9.415261E-3,4.198242E2,6.97E3,6.178E3,9.590705E6,1.8756017E5,2.3584168E-3,-2.6755203E-3,1.5328401E-3,-3.7313828E-3,1.3269727E-3,5.0462103E-3,4.3087793E-4,-6.195663E-3,-9.31714E-4,-2.0025484E-3,3.0208318E-3,-4.8206975E-3,8.655987E-4,9.1381116E2,-6.7128404E-4,2.7075084E5,3.8717715E-3,-3.4328848E-3,1.6673185E-3,-2.5317392E-3,-7.1851914E-3,-4.6793654E-4,-6.411654E-3],"split_indices":[42,56,2,2,0,4,48,37,67,54,54,3,4,9,42,3,37,62,4,3,41,54,0,66,0,4,56,0,0,1,54,66,1,0,7,0,0,0,0,0,1,0,0,0,0,0,0,0,64,0,0,61,2,2,38,56,0,0,0,0,0,0,0,0,0,0,0,0,0,61,0,57,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,2.56E2,1.77E2,2.53E2,3E0,1.03E2,7.4E1,8.8E1,1.65E2,4.3E1,6E1,4.6E1,2.8E1,7.4E1,1.4E1,4E1,1.25E2,3.5E1,8E0,2.2E1,3.8E1,4.4E1,2E0,2.5E1,3E0,3.6E1,3.8E1,9E0,5E0,1E1,3E1,9E1,3.5E1,1E1,2.5E1,5E0,3E0,1.6E1,6E0,5E0,3.3E1,1E1,3.4E1,1E0,2.4E1,3.2E1,4E0,7E0,3.1E1,3E0,7E0,1.9E1,1.1E1,5.8E1,3.2E1,2.1E1,1.4E1,1.6E1,9E0,7E0,2.6E1,2.9E1,5E0,2.5E1,6E0,6E0,1.3E1,7E0,4E0,2.8E1,3E1,3E1,2E0,1.5E1,6E0,2.2E1,6E0,3E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[5.33186E-3,-3.9137416E-2,1.268097E-1,-6.065864E-2,3.235334E-2,9.082176E-2,2.6837736E-1,-8.884292E-2,1.6221129E-3,1.9067204E-2,6.0843765E-3,1.06238425E-1,-2.0484257E-1,1.6989323E-1,2.5028892E-2,-1.3281189E-1,-7.066645E-2,3.1212479E-2,-7.377255E-2,1.03686884E-1,3.8871325E-3,2.2137332E-1,8.568724E-2,-1.2838757E-2,9.858314E-4,8.919293E-3,-2.0899652E-3,-8.616487E-3,-5.1367055E-3,-3.7296847E-2,-8.793514E-2,-6.969571E-3,4.0399007E-2,-2.2816295E-3,-1.0873887E-2,6.6593997E-3,-1.0938725E-3,-1.1996875E-3,5.4389734E-2,1.886416E-2,1.02404006E-1,4.189284E-2,1.3308394E-1,-6.15366E-2,1.6371814E-4,-1.04058154E-1,-2.1289664E-3,6.683426E-2,-2.0851933E-4,4.2482843E-3,-1.7358312E-5,-1.8965383E-3,7.1610208E-3,4.840231E-3,1.2139351E-3,-4.5456802E-3,1.5591812E-1,5.407888E-4,-4.2218245E-3,-1.17747836E-1,-8.661227E-4,1.8739505E-2,6.0995542E-3,8.965264E-3,4.1168276E-3,-8.082659E-3,-4.1469005E-3,1.9694082E-3,-3.8522724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,49,-1,51,53,55,57,-1,59,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,-1,65,-1,67,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3386407E0,4.8968878E-1,5.6661403E-1,4.284467E-1,8.985189E-2,4.50229E-1,4.797294E-1,1.2488043E-1,1.7408621E-1,8.437613E-2,0E0,1.9398534E-1,7.4645996E-2,6.595081E-2,0E0,3.884667E-2,6.644648E-2,9.6259445E-2,7.7637486E-2,4.66431E-2,8.416827E-2,2.2265631E-1,1.5633011E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.2622987E-2,5.1169038E-2,0E0,6.413231E-2,0E0,0E0,0E0,0E0,0E0,3.8800932E-2,0E0,6.4669356E-2,3.8538836E-2,2.0678937E-1,5.2101925E-2,0E0,6.506562E-2,0E0,9.5204234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.6121707E-2,0E0,0E0,6.3171566E-2,0E0,4.6872452E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,29,29,30,30,32,32,38,38,40,40,41,41,42,42,43,43,45,45,47,47,56,56,59,59,61,61],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,50,-1,52,54,56,58,-1,60,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,-1,66,-1,68,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7267E4,7.164074E8,2.1809912E0,2.735322E6,1E0,3.768034E4,1.2123712E1,5.63321E5,2.7394568E7,4.3712845E0,6.0843765E-3,7.801943E3,9.652E3,1.3936486E8,2.5028892E-2,3.133E3,7.477264E6,5.349E3,6.54925E6,1.24E2,7.770229E5,4.8E1,6.0891E4,-1.2838757E-2,9.858314E-4,8.919293E-3,-2.0899652E-3,-8.616487E-3,-5.1367055E-3,8.037592E2,3.135717E6,-6.969571E-3,2.028994E8,-2.2816295E-3,-1.0873887E-2,6.6593997E-3,-1.0938725E-3,-1.1996875E-3,1.152E4,1.886416E-2,3E0,1.1613266E9,1.5165479E0,1.0640107E8,1.6371814E-4,2.0964778E5,-2.1289664E-3,3.88672E5,-2.0851933E-4,4.2482843E-3,-1.7358312E-5,-1.8965383E-3,7.1610208E-3,4.840231E-3,1.2139351E-3,-4.5456802E-3,1.0850484E0,5.407888E-4,-4.2218245E-3,5.7410406E2,-8.661227E-4,3.3219E4,6.0995542E-3,8.965264E-3,4.1168276E-3,-8.082659E-3,-4.1469005E-3,1.9694082E-3,-3.8522724E-3],"split_indices":[2,7,51,1,108,4,62,1,54,67,0,41,0,1,0,2,54,9,9,3,56,3,2,0,0,0,0,0,0,61,9,0,7,0,0,0,0,0,10,0,8,7,62,5,0,37,0,9,0,0,0,0,0,0,0,0,66,0,0,61,0,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,3.16E2,1.15E2,2.43E2,7.3E1,9.3E1,2.2E1,1.67E2,7.6E1,6.5E1,8E0,8.9E1,4E0,1.7E1,5E0,4.7E1,1.2E2,5.5E1,2.1E1,9E0,5.6E1,1.2E1,7.7E1,3E0,1E0,1.6E1,1E0,1.5E1,3.2E1,4.2E1,7.8E1,2E0,5.3E1,1.9E1,2E0,7E0,2E0,3.6E1,2E1,4E0,8E0,4.1E1,3.6E1,2.6E1,1.6E1,5.6E1,2.2E1,3.3E1,2E1,1.2E1,8E0,2E0,6E0,8E0,3.3E1,3E0,3.3E1,7E0,1.9E1,4.8E1,8E0,1.9E1,1.4E1,2.2E1,1.1E1,1.7E1,3.1E1,1.6E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[6.200259E-3,-5.013612E-2,8.637296E-2,-4.474484E-2,-1.7994625E-2,2.618247E-2,1.6601844E-1,-1.017802E-1,-2.609986E-2,-2.4929245E-3,3.9160933E-2,1.3237011E-1,4.0180373E-1,-1.0932826E-1,1.5965193E-3,-4.4650447E-2,1.6568717E-2,7.036314E-2,1.4039993E-2,1.41745E-1,-5.689518E-3,1.0132869E-2,2.4739137E-2,-5.9431414E-3,-2.2009485E-3,-6.825185E-2,2.0182505E-2,-4.5049624E-3,2.6447242E-2,3.9240667E-3,-4.2927135E-3,-1.4616427E-3,1.7263378E-3,1.5547284E-1,1.0985534E-3,-5.3383134E-2,-1.13769315E-1,-2.3437176E-2,1.0937496E-1,-4.8661153E-3,3.5918523E-2,2.1152851E-1,1.2969565E-1,-4.4026304E-2,-1.5364897E-1,-1.5919342E-3,-7.4559273E-3,-3.5165022E-3,1.3433388E-3,1.0221393E-2,2.4886446E-3,5.327736E-3,2.3519168E-2,6.9912863E-3,1.4914711E-2,-6.5019523E-4,6.606014E-3,-3.1136659E-3,3.748537E-5,-1.142715E-2,4.195481E-4,-3.3482048E-4,2.5535128E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,-1,25,27,29,31,33,-1,-1,-1,-1,-1,35,37,-1,39,-1,-1,-1,-1,41,-1,43,45,47,49,-1,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.937452E0,4.222287E-1,8.4417343E-1,2.6213586E-1,0E0,1.0513027E-1,5.497806E-1,6.572878E-2,1.5033846E-1,0E0,6.798904E-2,1.7633569E-1,8.412182E-2,4.583347E-2,0E0,2.033571E-1,6.549706E-2,7.576361E-2,4.9352687E-2,1.04240894E-1,0E0,0E0,0E0,0E0,0E0,5.9574902E-2,1.4262311E-1,0E0,6.876865E-2,0E0,0E0,0E0,0E0,5.7818174E-2,0E0,6.3598186E-2,7.079685E-2,6.550719E-2,6.018457E-2,0E0,4.456245E-2,6.5761685E-2,4.6874106E-2,6.4504325E-2,8.325422E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0726695E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,25,25,26,26,28,28,33,33,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,52,52],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,-1,26,28,30,32,34,-1,-1,-1,-1,-1,36,38,-1,40,-1,-1,-1,-1,42,-1,44,46,48,50,-1,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,3.3056E4,7.41718E5,-1.7994625E-2,1.6861708E3,1.2123712E1,1.2852292E7,6.4065235E8,-2.4929245E-3,7.27886E6,5.01E2,1.92144E5,7.37642E4,1.5965193E-3,5.5292703E9,1.4338295E6,7.789796E1,2.9624592E3,1.5953E4,-5.689518E-3,1.0132869E-2,2.4739137E-2,-5.9431414E-3,-2.2009485E-3,9.76E3,7.330576E2,-4.5049624E-3,4E0,3.9240667E-3,-4.2927135E-3,-1.4616427E-3,1.7263378E-3,5.1E1,1.0985534E-3,2.7174674E11,1.339596E7,1.32E3,6.333721E2,-4.8661153E-3,1.6E1,7.893653E9,3E0,1.688021E6,3.77E2,-1.5919342E-3,-7.4559273E-3,-3.5165022E-3,1.3433388E-3,1.0221393E-2,2.4886446E-3,5.327736E-3,1.8756017E5,6.9912863E-3,1.4914711E-2,-6.5019523E-4,6.606014E-3,-3.1136659E-3,3.748537E-5,-1.142715E-2,4.195481E-4,-3.3482048E-4,2.5535128E-3],"split_indices":[42,56,2,1,0,64,62,9,7,0,59,3,2,37,0,5,54,65,4,0,0,0,0,0,0,10,61,0,8,0,0,0,0,3,0,40,54,10,42,0,10,12,8,1,3,0,0,0,0,0,0,0,56,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.27E2,2.51E2,1.76E2,2.48E2,3E0,1.01E2,7.5E1,6E1,1.88E2,1.4E1,8.7E1,6.7E1,8E0,5.7E1,3E0,1.31E2,5.7E1,3.8E1,4.9E1,6.5E1,2E0,4E0,4E0,4.6E1,1.1E1,9.6E1,3.5E1,4E0,5.3E1,3.6E1,2E0,1.6E1,3.3E1,5.8E1,7E0,7.4E1,2.2E1,2.4E1,1.1E1,3E0,5E1,1.6E1,4.2E1,6.9E1,5E0,8E0,1.4E1,1.2E1,1.2E1,3E0,8E0,6E0,4.4E1,1.1E1,5E0,2E0,4E1,4.7E1,2.2E1,3E0,2E0,2.2E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[2.7235828E-3,-5.6438107E-2,8.314159E-2,-5.112623E-2,-1.7343903E-2,3.3051584E-2,1.6971265E-1,-7.270181E-2,5.346936E-3,7.7559734E-3,6.527484E-2,2.007142E-2,1.3758217E-1,-7.987045E-2,1.9350799E-3,1.1504489E-2,-1.3250365E-2,1.9703187E-2,-3.8491166E-3,3.501997E-3,-1.8073824E-3,-8.957451E-3,1.5218739E-1,-1.1706332E-1,-5.8210928E-2,-3.5162235E-3,-7.519267E-3,-9.155581E-4,4.7573607E-2,1.3783646E-2,1.2116558E-1,-5.2065076E-4,-1.3116056E-1,-1.05580045E-2,-8.508038E-2,-1.5676413E-2,3.7359893E-3,1.3141527E-3,5.3134067E-3,1.2935445E-1,-6.357535E-3,-7.841832E-3,-9.5989324E-2,-3.3730045E-2,3.3958426E-3,-4.5455755E-3,2.304389E-3,7.822884E-4,-2.0557726E-3,3.5685864E-3,1.5553047E-1,-3.4312278E-3,-9.208705E-3,-7.267278E-4,-7.5169234E-3,7.8336485E-3,-1.2695421E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,-1,-1,25,27,-1,-1,-1,-1,29,31,33,35,-1,-1,37,-1,39,-1,41,43,45,47,-1,-1,-1,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9981538E0,3.7146157E-1,7.6626015E-1,2.9159623E-1,0E0,9.241594E-2,4.5768285E-1,1.4314121E-1,2.9380378E-1,6.9590114E-2,4.052137E-2,0E0,3.0691504E-1,1.2449503E-1,0E0,0E0,8.696226E-2,6.3448206E-2,0E0,0E0,0E0,0E0,2.0175135E-1,8.69683E-2,1.335445E-1,6.031378E-2,0E0,0E0,4.0259734E-2,0E0,1.16509914E-1,0E0,4.1961014E-2,7.556515E-2,8.6364776E-2,4.6275273E-2,0E0,0E0,0E0,5.6574285E-2,0E0,0E0,5.008574E-2,6.83126E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.3401778E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,16,16,17,17,22,22,23,23,24,24,25,25,28,28,30,30,32,32,33,33,34,34,35,35,39,39,42,42,43,43,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,-1,-1,26,28,-1,-1,-1,-1,30,32,34,36,-1,-1,38,-1,40,-1,42,44,46,48,-1,-1,-1,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,7.339209E6,5.9161E4,6.979E3,-1.7343903E-2,1.3428E4,9.73307E5,2.9063344E5,1.3812E4,7.309061E1,5.4276E4,2.007142E-2,1.3879E5,1.65602E5,1.9350799E-3,1.1504489E-2,2.4180542E7,1.3996752E5,-3.8491166E-3,3.501997E-3,-1.8073824E-3,-8.957451E-3,2.1207E4,5.2077852E-2,2.6931593E-3,2.1597655E9,-7.519267E-3,-9.155581E-4,1E0,1.3783646E-2,6.95E2,-5.2065076E-4,2.55E2,1.0599062E3,3.94E2,1.3404826E-3,3.7359893E-3,1.3141527E-3,5.3134067E-3,1.4015416E6,-6.357535E-3,-7.841832E-3,4.14E2,2.652E3,3.3958426E-3,-4.5455755E-3,2.304389E-3,7.822884E-4,-2.0557726E-3,3.5685864E-3,3.020675E5,-3.4312278E-3,-9.208705E-3,-7.267278E-4,-7.5169234E-3,7.8336485E-3,-1.2695421E-3],"split_indices":[42,56,2,2,0,2,38,37,9,65,2,0,11,9,0,0,52,42,0,0,0,0,9,67,48,7,0,0,108,0,3,0,3,42,8,66,0,0,0,41,0,0,3,10,0,0,0,0,0,0,42,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.18E2,2.41E2,1.77E2,2.38E2,3E0,1.13E2,6.4E1,1.72E2,6.6E1,6.4E1,4.9E1,6E0,5.8E1,1.62E2,1E1,4E0,6.2E1,5.7E1,7E0,4.6E1,3E0,2E0,5.6E1,5.8E1,1.04E2,5.9E1,3E0,2.4E1,3.3E1,9E0,4.7E1,7E0,5.1E1,3.8E1,6.6E1,5.2E1,7E0,2.6E1,7E0,4.6E1,1E0,2.5E1,2.6E1,3E1,8E0,6.2E1,4E0,2.4E1,2.8E1,1.6E1,3E1,2.2E1,4E0,2.7E1,3E0,2.9E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-5.1678247E-3,-4.4332895E-2,8.5959785E-2,-6.2151395E-2,1.3635322E-2,1.0439873E-1,-1.2635939E-1,-6.496775E-2,1.4389468E-2,2.4137987E-2,-8.368843E-3,8.643733E-2,1.43164E-2,-2.8094178E-1,4.9282912E-2,-8.770066E-2,-3.416401E-2,5.376075E-3,1.106708E-2,9.4822526E-2,-1.13647506E-1,-1.3840034E-3,-1.5701437E-2,8.786841E-3,-3.0257998E-3,-1.2949483E-1,-7.38646E-2,-1.5764704E-3,-7.309311E-2,-3.5753564E-3,4.7103106E-3,6.962631E-2,1.400674E-1,6.674133E-3,-1.018659E-2,-1.5816809E-1,-3.3040137E-3,-4.4745365E-3,-4.2964015E-2,-1.1410795E-3,4.2685624E-2,4.3740165E-3,-1.1455471E-1,-3.2190336E-3,3.4985656E-4,1.03158295E-1,3.1230832E-2,1.6379856E-2,1.1953902E-1,-8.333266E-3,-6.0619356E-4,-1.4031002E-3,-5.736986E-3,3.757395E-3,-7.4804196E-4,3.8681105E-3,-1.6352843E-3,-3.3753582E-3,-1.7554542E-1,-7.056369E-4,1.13673E-1,-1.0903602E-3,2.7411722E-3,-7.0667104E-4,1.2786394E-1,-1.2146002E-2,-4.0990002E-3,8.888516E-3,4.1419645E-3,3.6513708E-3,7.5473064E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,23,25,27,-1,29,31,33,-1,-1,-1,-1,35,37,39,41,43,-1,45,47,-1,-1,49,-1,-1,51,-1,53,55,57,-1,-1,59,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,65,-1,67,-1,-1,-1,69,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5670414E0,3.1794977E-1,5.299639E-1,2.5804287E-1,1.4771901E-1,3.962109E-1,3.1250894E-1,1.6086435E-1,0E0,7.869907E-2,0E0,2.0036232E-1,0E0,5.8312625E-2,1.0081367E-1,6.792259E-2,1.2801373E-1,0E0,7.89901E-2,1.14507616E-1,1.5228291E-1,0E0,0E0,0E0,0E0,4.510373E-2,5.6286156E-2,5.57631E-2,1.4824896E-1,3.744252E-2,0E0,8.9476585E-2,1.1513406E-1,0E0,0E0,4.26628E-2,0E0,0E0,3.9748766E-2,0E0,4.018064E-2,5.209426E-2,6.922439E-2,0E0,0E0,4.8728913E-2,4.957065E-2,0E0,4.2092025E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.154406E-2,0E0,5.285266E-2,0E0,0E0,0E0,3.8960993E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,18,18,19,19,20,20,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,38,38,40,40,41,41,42,42,45,45,46,46,48,48,58,58,60,60,64,64],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,24,26,28,-1,30,32,34,-1,-1,-1,-1,36,38,40,42,44,-1,46,48,-1,-1,50,-1,-1,52,-1,54,56,58,-1,-1,60,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,66,-1,68,-1,-1,-1,70,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,7.164074E8,1.0429407E3,5.2191963E3,1E0,1.146155E1,1.0095158E0,1.732734E6,1.4389468E-2,1.7608592E-4,-8.368843E-3,3.305542E4,1.43164E-2,3.087E3,8.494E3,4.94E4,1.8071064E7,5.376075E-3,1E0,5.8145845E3,1.92144E5,-1.3840034E-3,-1.5701437E-2,8.786841E-3,-3.0257998E-3,1.7481459E3,7.37642E4,1.1669E4,6.465659E4,1.3857716E6,4.7103106E-3,1.1532659E7,3.7137297E3,6.674133E-3,-1.018659E-2,1.6013113E7,-3.3040137E-3,-4.4745365E-3,1.475021E6,-1.1410795E-3,1.0846253E1,5.5219E4,6E1,-3.2190336E-3,3.4985656E-4,6.287E3,2.326757E9,1.6379856E-2,7.410559E1,-8.333266E-3,-6.0619356E-4,-1.4031002E-3,-5.736986E-3,3.757395E-3,-7.4804196E-4,3.8681105E-3,-1.6352843E-3,-3.3753582E-3,5.730192E6,-7.056369E-4,1.8297544E3,-1.0903602E-3,2.7411722E-3,-7.0667104E-4,1.062363E1,-1.2146002E-2,-4.0990002E-3,8.888516E-3,4.1419645E-3,3.6513708E-3,7.5473064E-3],"split_indices":[2,7,67,4,18,44,48,1,0,47,0,4,0,0,0,9,54,0,108,4,2,0,0,0,0,64,37,10,37,54,0,54,41,0,0,54,0,0,1,0,62,12,8,0,0,11,12,0,67,0,0,0,0,0,0,0,0,0,41,0,4,0,0,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.06E2,1.31E2,2.34E2,7.2E1,1.21E2,1E1,2.33E2,1E0,6.9E1,3E0,1.12E2,9E0,5E0,5E0,1.33E2,1E2,8E0,6.1E1,1.08E2,4E0,1E0,4E0,2E0,3E0,3.1E1,1.02E2,5.5E1,4.5E1,5.3E1,8E0,7.1E1,3.7E1,1E0,3E0,2E1,1.1E1,6.2E1,4E1,3.7E1,1.8E1,1.6E1,2.9E1,7E0,4.6E1,3.7E1,3.4E1,2E0,3.5E1,1.8E1,2E0,3.5E1,5E0,1.1E1,7E0,5E0,1.1E1,1.8E1,1.1E1,3E0,3.4E1,1.1E1,2.3E1,2E0,3.3E1,5E0,6E0,8E0,2.6E1,1.3E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-8.068479E-4,-3.9410036E-2,8.293859E-2,-4.2288955E-2,1.1688771E-2,-1.7907192E-1,9.706064E-2,-8.009502E-2,-2.5677705E-2,-1.3640828E-3,-1.3745622E-2,8.7483846E-2,1.3058012E-2,-9.9485114E-2,8.208468E-4,8.122609E-3,-4.3295875E-2,9.810779E-2,-1.2895183E-1,-1.1098605E-1,-3.8209473E-4,4.566631E-3,-1.4066037E-2,-6.488058E-2,-1.112006E-2,8.7432414E-2,2.2369904E-2,6.513235E-3,-1.006633E-2,-8.178425E-4,-1.20585725E-1,-1.6821921E-4,-7.3775453E-3,-8.072163E-3,-5.6968763E-2,2.0387121E-3,-3.3312913E-2,6.727937E-2,1.2810886E-1,-1.291318E-1,-1.2739715E-3,-2.3019235E-3,2.957064E-2,-3.0639851E-3,1.3057502E-3,1.4452769E-2,-6.177843E-2,5.4487106E-3,4.5710303E-2,1.2150607E-2,1.0474683E-1,-4.9260003E-3,-8.408489E-3,1.9730805E-4,4.1086734E-3,-1.7616393E-3,3.0682397E-3,-9.987716E-3,-1.8875337E-3,9.823327E-4,4.5926534E-3,1.5378199E-4,5.9515624E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,19,-1,21,23,25,27,29,-1,-1,31,33,35,37,-1,-1,-1,-1,39,41,-1,-1,43,-1,45,47,49,51,-1,-1,53,-1,-1,55,57,-1,59,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3319663E0,2.390615E-1,4.9839735E-1,1.7378497E-1,0E0,1.0382749E-1,1.8265486E-1,1.6176176E-1,1.1706962E-1,0E0,0E0,2.8627986E-1,0E0,7.388568E-2,0E0,1.3198087E-1,8.8869736E-2,4.046662E-1,1.5466163E-1,5.481839E-2,0E0,0E0,1.0618068E-1,5.2928627E-2,6.422235E-2,8.319342E-2,0E0,0E0,0E0,0E0,4.245752E-2,7.356602E-2,0E0,0E0,4.208377E-2,0E0,5.2560516E-2,7.077879E-2,7.7803314E-2,4.5649767E-2,0E0,0E0,4.5017958E-2,0E0,0E0,3.994179E-2,7.084573E-2,0E0,6.584112E-2,0E0,6.0341835E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,13,13,15,15,16,16,17,17,18,18,19,19,22,22,23,23,24,24,25,25,30,30,31,31,34,34,36,36,37,37,38,38,39,39,42,42,45,45,46,46,48,48,50,50],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,20,-1,22,24,26,28,30,-1,-1,32,34,36,38,-1,-1,-1,-1,40,42,-1,-1,44,-1,46,48,50,52,-1,-1,54,-1,-1,56,58,-1,60,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,3.2991206E3,2.4950776E0,1.5031561E3,1.1688771E-2,6.27427E5,1.6828056E1,2.1592189E5,1.24E2,-1.3640828E-3,-1.3745622E-2,3.768034E4,1.3058012E-2,6.209E3,8.208468E-4,2.0718214E-4,1.436034E9,3.305542E4,1.92144E5,5.8955963E1,-3.8209473E-4,4.566631E-3,1E0,1.7506002E3,1.8071064E7,9.3058E4,2.2369904E-2,6.513235E-3,-1.006633E-2,-8.178425E-4,5.54E2,5.9694834E-2,-7.3775453E-3,-8.072163E-3,1.577058E9,2.0387121E-3,3.9E1,8.16997E6,2.1207E4,4.02959E5,-1.2739715E-3,-2.3019235E-3,1.123E3,-3.0639851E-3,1.3057502E-3,7.07812E7,7.9004836E0,5.4487106E-3,4.0292856E3,1.2150607E-2,2.6049583E0,-4.9260003E-3,-8.408489E-3,1.9730805E-4,4.1086734E-3,-1.7616393E-3,3.0682397E-3,-9.987716E-3,-1.8875337E-3,9.823327E-4,4.5926534E-3,1.5378199E-4,5.9515624E-3],"split_indices":[2,61,63,4,0,9,44,37,3,0,0,4,0,2,0,47,5,4,2,59,0,0,18,4,54,2,0,0,0,0,3,48,0,0,7,0,8,54,9,38,0,0,0,0,0,7,62,0,61,0,62,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.1E2,2.81E2,1.29E2,2.79E2,2E0,6E0,1.23E2,8.4E1,1.95E2,3E0,3E0,1.18E2,5E0,7E1,1.4E1,6.7E1,1.28E2,1.13E2,5E0,6.2E1,8E0,1.3E1,5.4E1,7.6E1,5.2E1,1.11E2,2E0,1E0,4E0,6E0,5.6E1,5E1,4E0,4E0,7.2E1,1.5E1,3.7E1,7.6E1,3.5E1,5.1E1,5E0,1.9E1,3.1E1,6.7E1,5E0,1.4E1,2.3E1,2.3E1,5.3E1,4E0,3.1E1,3.4E1,1.7E1,2.2E1,9E0,7E0,7E0,2E0,2.1E1,3.6E1,1.7E1,5E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-3.0188558E-3,-4.9524106E-2,8.904375E-2,-6.1421823E-2,7.623601E-3,4.9896702E-2,2.0953321E-1,-7.0444085E-2,-1.7035292E-3,5.9391167E-3,-5.1023364E-3,6.321364E-2,-6.412609E-2,3.7052968E-1,1.2030443E-1,-7.90439E-2,-8.197472E-3,4.1567024E-2,-2.580257E-3,-1.6197995E-3,1.0807815E-3,7.2704133E-4,7.162963E-2,1.9225504E-3,-5.8394414E-3,7.7621397E-3,2.0370731E-2,1.7483947E-1,-9.454206E-3,-7.399006E-2,-8.604264E-3,-7.6637752E-3,8.860276E-4,-1.048977E-3,3.6181917E-3,5.292559E-2,4.9132123E-3,6.187881E-3,1.4367396E-2,-1.0533839E-1,-5.6412738E-2,6.2265364E-4,7.608208E-2,-1.3697223E-1,-6.043014E-2,1.5081667E-3,-6.8125516E-2,5.821042E-3,2.2295883E-3,-2.002357E-3,-7.6832003E-3,-1.1833028E-3,-5.323055E-3,-1.9163066E-3,-4.3257074E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,-1,-1,-1,37,-1,39,-1,-1,-1,-1,-1,41,-1,-1,-1,43,45,-1,47,49,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8368468E0,1.9424748E-1,6.6876507E-1,1.2703544E-1,7.415508E-2,1.7108145E-1,4.582169E-1,1.0909045E-1,7.396507E-2,0E0,3.68192E-2,3.902501E-2,7.594524E-2,6.263876E-2,4.4838026E-1,7.722771E-2,1.0707621E-1,3.97871E-2,0E0,0E0,0E0,0E0,4.2166144E-2,0E0,0E0,0E0,0E0,8.201182E-2,0E0,8.938396E-2,0E0,0E0,0E0,0E0,0E0,4.9050197E-2,0E0,0E0,0E0,7.8380585E-2,1.18195385E-1,0E0,3.971438E-2,6.81352E-2,4.5711048E-2,0E0,5.770889E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,22,22,27,27,29,29,35,35,39,39,40,40,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,-1,-1,-1,38,-1,40,-1,-1,-1,-1,-1,42,-1,-1,-1,44,46,-1,48,50,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.10711E8,6.204522E3,1.46705E3,2.829932E0,3.5301748E7,1.417988E6,2.1592189E5,6.7211235E1,5.9391167E-3,4.5593824E4,1.1E1,1.6012479E3,9.256843E3,1.5096262E7,3.703296E7,2.1066585E2,3.345434E1,-2.580257E-3,-1.6197995E-3,1.0807815E-3,7.2704133E-4,1.8618166E-1,1.9225504E-3,-5.8394414E-3,7.7621397E-3,2.0370731E-2,1.6828056E1,-9.454206E-3,1.65602E5,-8.604264E-3,-7.6637752E-3,8.860276E-4,-1.048977E-3,3.6181917E-3,5.589612E1,4.9132123E-3,6.187881E-3,1.4367396E-2,7.121916E2,8.9208E4,6.2265364E-4,6.992308E1,5.897327E1,1.0220021E3,1.5081667E-3,1.18E2,5.821042E-3,2.2295883E-3,-2.002357E-3,-7.6832003E-3,-1.1833028E-3,-5.323055E-3,-1.9163066E-3,-4.3257074E-3],"split_indices":[2,7,61,61,67,54,38,37,65,0,42,8,61,4,9,54,61,63,0,0,0,0,44,0,0,0,0,44,0,9,0,0,0,0,0,65,0,0,0,61,12,0,65,59,61,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.27E2,2.84E2,1.43E2,2.35E2,4.9E1,1.09E2,3.4E1,2.04E2,3.1E1,4E0,4.5E1,9.8E1,1.1E1,1.1E1,2.3E1,1.79E2,2.5E1,1.7E1,1.4E1,2.2E1,2.3E1,1.5E1,8.3E1,4E0,7E0,3E0,8E0,2E1,3E0,1.72E2,7E0,3E0,2.2E1,6E0,1.1E1,5.3E1,3E1,1.6E1,4E0,6E1,1.12E2,2E1,3.3E1,3.4E1,2.6E1,1.3E1,9.9E1,1.2E1,2.1E1,7E0,2.7E1,1.6E1,1E1,4.4E1,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-3.740929E-4,-2.4554553E-2,1.4042461E-1,-4.517057E-2,4.842384E-2,3.2563648E-1,9.854472E-2,-5.4666307E-2,1.9184066E-2,8.952364E-2,-1.7603487E-3,1.7255944E-3,1.8614743E-2,8.487421E-3,1.22499414E-1,-5.1616482E-2,-1.3717255E-2,-5.4951916E-3,3.8777664E-2,1.7032584E-2,1.22785054E-1,-6.1476976E-3,1.3084737E-2,5.40506E-3,-2.7643193E-3,1.4965567E-1,1.6794496E-3,-6.911305E-2,-9.684618E-3,4.5111296E-3,5.251172E-4,4.7400272E-3,-1.034683E-3,6.274149E-3,-2.1722491E-3,2.761459E-2,-7.052154E-3,1.4242334E-2,6.240081E-3,-9.315615E-2,-4.455191E-2,-1.9282624E-2,2.9272072E-3,2.7684472E-4,3.995837E-3,-1.5103087E-3,-1.0180724E-1,2.1246013E-3,-5.285663E-2,-3.64976E-2,1.1724503E-3,-6.706522E-3,-3.6950123E-3,-3.2267298E-3,3.6593454E-4,2.3183477E-4,-2.8960663E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,-1,29,31,33,-1,35,-1,-1,37,-1,39,41,-1,-1,-1,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4401798E0,5.453205E-1,4.4882274E-1,1.7337734E-1,1.6516243E-1,1.7886627E-1,1.11139596E-1,1.5774679E-1,1.0095164E-1,1.03999466E-1,7.073505E-2,0E0,0E0,8.8264644E-2,9.2396915E-2,1.7834467E-1,0E0,0E0,4.939602E-2,4.9036566E-2,4.282555E-2,0E0,8.275861E-2,0E0,0E0,5.3191185E-2,0E0,9.74201E-2,5.0493233E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.8201764E-2,0E0,0E0,0E0,4.2813838E-2,6.635529E-2,4.9673937E-2,0E0,0E0,0E0,0E0,5.973488E-2,0E0,6.852606E-2,4.604107E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,18,18,19,19,20,20,22,22,25,25,27,27,28,28,35,35,39,39,40,40,41,41,46,46,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,-1,30,32,34,-1,36,-1,-1,38,-1,40,42,-1,-1,-1,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2991206E3,1.1576422E9,1.37893E6,2.1592189E5,1.1532659E7,8.511305E6,5.0314346E1,1.5609541E0,5.8955963E1,1.5403295E6,2.0690203E3,1.7255944E-3,1.8614743E-2,5.462141E3,1.5096262E7,3.447954E5,-1.3717255E-2,-5.4951916E-3,5.666993E0,1.244306E-4,2.5307438E0,-6.1476976E-3,1.190724E4,5.40506E-3,-2.7643193E-3,1E0,1.6794496E-3,1.8008743E3,4.79E2,4.5111296E-3,5.251172E-4,4.7400272E-3,-1.034683E-3,6.274149E-3,-2.1722491E-3,2.203418E7,-7.052154E-3,1.4242334E-2,6.240081E-3,5.8981678E1,1E0,1.7657828E3,2.9272072E-3,2.7684472E-4,3.995837E-3,-1.5103087E-3,2.53383E5,2.1246013E-3,6.067857E8,3.9150736E7,1.1724503E-3,-6.706522E-3,-3.6950123E-3,-3.2267298E-3,3.6593454E-4,2.3183477E-4,-2.8960663E-3],"split_indices":[61,7,38,37,54,1,65,66,59,59,4,0,0,4,9,56,0,0,62,47,43,0,4,0,0,25,0,4,8,0,0,0,0,0,0,1,0,0,0,59,109,61,0,0,0,0,9,0,7,41,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.21E2,3.6E2,6.1E1,2.81E2,7.9E1,1E1,5.1E1,2.45E2,3.6E1,4.3E1,3.6E1,2E0,8E0,1.1E1,4E1,2.43E2,2E0,4E0,3.2E1,1.4E1,2.9E1,3E0,3.3E1,4E0,7E0,3E1,1E1,1.71E2,7.2E1,1E1,2.2E1,4E0,1E1,2.8E1,1E0,3.1E1,2E0,2E0,2.8E1,8.5E1,8.6E1,6.4E1,8E0,2.3E1,8E0,1.1E1,7.4E1,7E0,7.9E1,4.6E1,1.8E1,2.8E1,4.6E1,6.4E1,1.5E1,1.7E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-5.079172E-5,-3.540679E-2,6.991461E-2,-6.8019964E-2,-7.2810063E-3,2.870401E-1,5.48098E-2,-7.927486E-2,-2.154307E-3,5.161256E-2,-3.4134436E-2,-1.6301366E-3,1.5977426E-2,-5.768703E-3,6.6835724E-2,-1.02645814E-1,-4.1354418E-2,-6.9484697E-3,1.0426651E-3,6.64037E-2,-9.5759606E-4,-8.816023E-2,-1.7905734E-2,4.1246187E-2,1.2082209E-1,-1.2783159E-3,-1.1381003E-1,-3.5966796E-4,-3.9370856E-3,4.2158607E-3,4.8405476E-4,-1.6367373E-3,-6.000952E-3,-6.7765564E-3,-5.530098E-4,5.094055E-2,-1.1917828E-2,9.509898E-2,2.8186402E-1,-1.5441298E-1,-7.7140644E-2,1.25212325E-2,7.1668886E-2,7.067916E-3,2.2457836E-3,2.1525532E-2,1.1250208E-3,-8.038833E-3,-1.620588E-5,-1.0529811E-3,-4.832941E-3,-4.9758866E-2,2.1479176E-3,1.3889784E-1,5.5495877E-2,-1.6347563E-4,-8.3065275E-3,1.5580938E-3,8.676928E-3,3.2524734E-3,-4.700467E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,-1,-1,29,-1,31,33,35,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,41,-1,43,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,55,-1,57,59,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0463755E0,2.5725794E-1,4.499761E-1,9.619963E-2,2.421489E-1,1.3921928E-1,2.891228E-1,9.412742E-2,7.0629045E-2,5.206293E-2,9.048214E-2,0E0,0E0,0E0,1.6908383E-1,5.478072E-2,5.949463E-2,0E0,0E0,4.6543613E-2,0E0,4.300618E-2,6.31146E-2,2.5584954E-1,1.3889492E-1,0E0,7.593274E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.679395E-2,0E0,8.3113104E-2,2.023237E-1,5.116576E-2,3.995791E-2,6.404998E-2,5.209291E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.190689E-2,0E0,3.7949666E-2,3.6554277E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,16,16,19,19,21,21,22,22,23,23,24,24,26,26,35,35,37,37,38,38,39,39,40,40,41,41,42,42,51,51,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,-1,-1,30,-1,32,34,36,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,42,-1,44,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,56,-1,58,60,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0409E4,1.6781406E3,9.957E3,2.1592189E5,1.24E2,5.274031E6,1.5165479E0,1.656968E6,3.516E3,2.3177492E6,9.04782E5,-1.6301366E-3,1.5977426E-2,-5.768703E-3,1.486066E0,5.8981678E1,4.080184E1,-6.9484697E-3,1.0426651E-3,1.2885352E7,-9.5759606E-4,1.2151747E3,1.32E2,1.4609149E4,5.01E2,-1.2783159E-3,2.7276272E-2,-3.5966796E-4,-3.9370856E-3,4.2158607E-3,4.8405476E-4,-1.6367373E-3,-6.000952E-3,-6.7765564E-3,-5.530098E-4,2.1959E4,-1.1917828E-2,2.27901E5,1.06904055E5,4.61E2,2.4999522E5,2.9015378E3,1.8297544E3,7.067916E-3,2.2457836E-3,2.1525532E-2,1.1250208E-3,-8.038833E-3,-1.620588E-5,-1.0529811E-3,-4.832941E-3,4.585483E6,2.1479176E-3,1.504E4,1.5953E4,-1.6347563E-4,-8.3065275E-3,1.5580938E-3,8.676928E-3,3.2524734E-3,-4.700467E-4],"split_indices":[2,64,9,37,3,1,62,9,12,56,1,0,0,0,48,59,62,0,0,54,0,56,3,4,3,0,51,0,0,0,0,0,0,0,0,2,0,9,37,3,57,4,4,0,0,0,0,0,0,0,0,1,0,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.21E2,2.8E2,1.41E2,1.29E2,1.51E2,8E0,1.33E2,1.1E2,1.9E1,4.7E1,1.04E2,1E0,7E0,8E0,1.25E2,6.7E1,4.3E1,2E0,1.7E1,3.9E1,8E0,2.3E1,8.1E1,8.6E1,3.9E1,9E0,5.8E1,2.4E1,1.9E1,2.8E1,1.1E1,1E1,1.3E1,3E0,7.8E1,8.4E1,2E0,3.5E1,4E0,2.6E1,3.2E1,3E1,5.4E1,1.6E1,1.9E1,2E0,2E0,2.4E1,2E0,1E1,2.2E1,1E1,2E1,9E0,4.5E1,8E0,2E0,3E0,6E0,3.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-4.745195E-3,-4.6117246E-2,6.926275E-2,-4.9821753E-2,1.4473992E-2,2.6701031E-2,1.4098886E-1,-4.5017608E-2,-1.4152161E-2,-3.7873086E-2,4.641054E-2,1.0163995E-1,3.1182984E-1,-8.982378E-2,-3.252256E-2,-8.2318885E-3,-8.2401454E-4,1.606645E-2,7.473477E-2,1.2018911E-1,-1.0378538E-3,6.911959E-3,1.9298637E-2,-1.1667622E-1,-5.3319875E-2,-4.956259E-2,8.403355E-3,-6.117694E-4,2.848871E-3,9.818748E-2,1.0646334E-3,1.0512048E-2,1.0372032E-1,-1.2595691E-1,7.243393E-4,3.1594635E-4,-4.3948716E-3,-6.712826E-2,8.020485E-4,-7.5320946E-4,4.9820513E-2,5.862771E-3,1.4620663E-3,7.949279E-3,3.7693973E-3,-1.1332212E-3,-6.7895283E-3,-4.7043707E-2,-9.361268E-2,3.736313E-3,-3.61521E-2,-7.732676E-4,3.5683606E-3,-3.8094022E-3,-1.5073134E-4,-8.615115E-3,-3.8033146E-3,-7.623459E-4,-5.9059844E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,27,29,31,-1,-1,-1,33,35,37,39,-1,-1,41,-1,-1,43,45,-1,-1,-1,47,49,-1,51,-1,-1,-1,-1,-1,-1,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2891756E0,3.6133963E-1,4.5720613E-1,3.0054617E-1,0E0,1.2393924E-1,3.4476042E-1,1.4488053E-1,0E0,6.0588833E-2,6.240028E-2,1.09963715E-1,9.159559E-2,4.9651265E-2,1.454713E-1,0E0,0E0,4.6518683E-2,4.4757545E-2,4.3444455E-2,0E0,0E0,0E0,4.100454E-2,5.9486687E-2,1.3010308E-1,6.2471077E-2,0E0,0E0,3.7613124E-2,0E0,0E0,4.4901192E-2,4.2358696E-2,0E0,0E0,0E0,5.357623E-2,1.1332689E-1,0E0,3.782449E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.9593366E-2,4.3551147E-2,0E0,4.54809E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,23,23,24,24,25,25,26,26,29,29,32,32,33,33,37,37,38,38,40,40,47,47,48,48,50,50],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,28,30,32,-1,-1,-1,34,36,38,40,-1,-1,42,-1,-1,44,46,-1,-1,-1,48,50,-1,52,-1,-1,-1,-1,-1,-1,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.7467445E4,8.861901E3,5.6503E4,7.339209E6,1.4473992E-2,1.0643513E3,1.2123712E1,3.836E3,-1.4152161E-2,1E0,9.222873E4,5.30113E5,1.92144E5,2.39E2,1.2128688E5,-8.2318885E-3,-8.2401454E-4,1.29186E9,1.535373E6,2.0159504E1,-1.0378538E-3,6.911959E-3,1.9298637E-2,1.26E2,4.198242E2,1.2852292E7,2.735322E6,-6.117694E-4,2.848871E-3,1.3307285E7,1.0646334E-3,1.0512048E-2,9.174246E6,4.5E1,7.243393E-4,3.1594635E-4,-4.3948716E-3,3.4129692E-3,1.0893103E-3,-7.5320946E-4,5.555794E0,5.862771E-3,1.4620663E-3,7.949279E-3,3.7693973E-3,-1.1332212E-3,-6.7895283E-3,1.874E3,3.8346853E5,3.736313E-3,9.274153E-2,-7.732676E-4,3.5683606E-3,-3.8094022E-3,-1.5073134E-4,-8.615115E-3,-3.8033146E-3,-7.623459E-4,-5.9059844E-3],"split_indices":[42,61,2,56,0,61,62,2,0,13,37,2,2,3,37,0,0,7,38,65,0,0,0,8,61,9,1,0,0,57,0,0,54,3,0,0,0,66,51,0,67,0,0,0,0,0,0,11,60,0,47,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,2.69E2,1.5E2,2.67E2,2E0,9.5E1,5.5E1,2.63E2,4E0,2.2E1,7.3E1,4.6E1,9E0,5.6E1,2.07E2,2E0,2E1,3.6E1,3.7E1,4E1,6E0,4E0,5E0,3.1E1,2.5E1,1.46E2,6.1E1,2.2E1,1.4E1,2.5E1,1.2E1,4E0,3.6E1,2.9E1,2E0,1E1,1.5E1,1.08E2,3.8E1,3.9E1,2.2E1,1.8E1,7E0,9E0,2.7E1,4E0,2.5E1,6.3E1,4.5E1,1.2E1,2.6E1,6E0,1.6E1,3.6E1,2.7E1,5E0,4E1,2.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.077489E-3,-3.2607842E-2,1.0156128E-1,-4.8269983E-2,1.4207412E-2,6.876931E-2,1.9980974E-1,-6.709187E-2,-2.5754988E-2,2.2487892E-2,-7.571064E-3,8.040109E-2,-1.6651882E-1,8.8895224E-2,3.1255922E-1,-3.625406E-3,1.3087684E-4,-1.8498434E-2,-5.2635246E-3,4.9184255E-2,3.7817823E-5,9.6382566E-2,-4.750557E-2,7.3410716E-4,-1.1194549E-2,-5.7124937E-5,6.0291933E-3,5.322418E-3,1.9278983E-2,-7.936017E-3,-8.275219E-2,1.2158036E-3,6.4087575E-3,-2.625093E-3,8.273737E-4,6.259212E-2,1.3138486E-1,-1.0781152E-1,1.0485564E-2,1.5102117E-3,-2.1282485E-2,-6.7928256E-3,-6.8837346E-4,7.809215E-2,-6.1611825E-4,7.012903E-3,-1.6481055E-4,-7.177047E-3,4.9214356E-4,-4.3822404E-2,5.831914E-4,5.7632467E-3,1.8984171E-3,-2.6560714E-3,2.4374702E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,-1,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,-1,-1,-1,-1,43,45,47,-1,-1,49,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1],"loss_changes":[1.4046019E0,2.441172E-1,3.1487882E-1,1.0379189E-1,1.22633025E-1,2.2686014E-1,2.8485376E-1,7.652962E-2,6.758717E-2,4.8544303E-2,0E0,1.5788901E-1,5.1265754E-2,4.573825E-2,1.6762757E-1,0E0,0E0,7.196149E-2,0E0,7.090605E-2,4.2912226E-2,7.102013E-2,1.6726284E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.9394317E-2,5.769696E-2,0E0,0E0,0E0,0E0,4.305467E-2,6.1160743E-2,4.0267885E-2,0E0,0E0,5.3193506E-2,0E0,0E0,4.3426514E-2,0E0,0E0,0E0,0E0,0E0,4.7435276E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,19,19,20,20,21,21,22,22,29,29,30,30,35,35,36,36,37,37,40,40,43,43,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,-1,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,-1,-1,-1,-1,44,46,48,-1,-1,50,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1],"split_conditions":[2.1722E4,7.164074E8,1.9900316E0,1.732734E6,1E0,5.37E2,2.2876814E1,8.067265E5,1.93E2,4.16606E5,-7.571064E-3,7.2125336E2,1.3590209E7,3.3406117E11,6.7304E4,-3.625406E-3,1.3087684E-4,3.931053E7,-5.2635246E-3,4.2681945E6,4.5593824E4,9.3058E4,3.305542E4,7.3410716E-4,-1.1194549E-2,-5.7124937E-5,6.0291933E-3,5.322418E-3,1.9278983E-2,7.7815704E2,1.4171124E3,1.2158036E-3,6.4087575E-3,-2.625093E-3,8.273737E-4,6.791E4,2.9005406E5,1.19573875E5,1.0485564E-2,1.5102117E-3,1.2833E4,-6.7928256E-3,-6.8837346E-4,2.6402641E-2,-6.1611825E-4,7.012903E-3,-1.6481055E-4,-7.177047E-3,4.9214356E-4,9.355285E8,5.831914E-4,5.7632467E-3,1.8984171E-3,-2.6560714E-3,2.4374702E-3],"split_indices":[2,7,48,1,18,3,63,56,8,38,0,67,1,40,2,0,0,54,0,57,42,2,4,0,0,0,0,0,0,42,61,0,0,0,0,10,42,37,0,0,10,0,0,66,0,0,0,0,0,41,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,3.31E2,1.01E2,2.48E2,8.3E1,7.7E1,2.4E1,1.34E2,1.14E2,8E1,3E0,7.4E1,3E0,1.3E1,1.1E1,1.2E2,1.4E1,1.06E2,8E0,3.6E1,4.4E1,6.6E1,8E0,1E0,2E0,4E0,9E0,4E0,7E0,9.2E1,1.4E1,2.9E1,7E0,1E1,3.4E1,3.5E1,3.1E1,7E0,1E0,2.3E1,6.9E1,7E0,7E0,2.9E1,6E0,2.8E1,3E0,5E0,2E0,4.1E1,2.8E1,1.3E1,1.6E1,3.7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.7790763E-3,-2.9075405E-2,9.5716335E-2,-2.5761636E-2,-1.4432695E-2,7.595189E-2,2.8774202E-1,-4.3039355E-2,3.5758965E-2,2.1484078E-1,6.345649E-2,3.4201688E-1,-4.8392345E-3,-4.9671624E-2,6.828063E-4,9.9574765E-3,1.0040284E-1,4.969834E-3,1.4505199E-2,6.89613E-2,-6.18826E-3,1.8401371E-2,6.0247426E-4,-9.599164E-2,-3.9677817E-2,4.0029578E-2,-1.5580928E-3,7.5600827E-3,2.5861978E-3,2.2100324E-2,8.149718E-2,1.5237789E-3,-1.0689659E-1,-3.5148747E-2,-6.8057356E-3,5.445658E-3,9.093233E-4,3.4621595E-3,-2.5145265E-3,4.1808034E-3,-1.451392E-3,-1.1924802E-1,-7.36504E-5,-1.6806884E-2,-6.317801E-2,-6.5387883E-3,-1.7717317E-3,-1.3483303E-3,2.602607E-3,-2.1266646E-3,-6.0041887E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,-1,-1,31,33,35,-1,-1,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,45,-1,47,49,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3012596E0,2.9499352E-1,3.994217E-1,3.5762253E-1,0E0,1.633851E-1,2.4497187E-1,1.0009858E-1,1.2242544E-1,4.588422E-2,1.0915458E-1,1.1275649E-1,0E0,1.0497588E-1,0E0,6.99394E-2,4.6971932E-2,0E0,0E0,5.2831143E-2,0E0,0E0,0E0,6.0426295E-2,8.4656835E-2,4.7264952E-2,0E0,0E0,0E0,8.1249036E-2,4.507777E-2,0E0,4.9273282E-2,9.5590636E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0926546E-2,0E0,9.233497E-2,8.108619E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,19,19,23,23,24,24,25,25,29,29,30,30,32,32,33,33,41,41,43,43,44,44],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,-1,-1,32,34,36,-1,-1,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,46,-1,48,50,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,3.764913E7,2.5212732E4,1.5135906E3,-1.4432695E-2,9.957E3,2.6398756E5,2.0964778E5,7.709139E6,2.094E3,6.5880767E3,8.494E3,-4.8392345E-3,3.586E3,6.828063E-4,6.878971E1,2.326757E9,4.969834E-3,1.4505199E-2,1.3125501E6,-6.18826E-3,1.8401371E-2,6.0247426E-4,7.6889176E0,4.086885E7,7.37E3,-1.5580928E-3,7.5600827E-3,2.5861978E-3,2.0918E4,6.029086E10,1.5237789E-3,4.641776E-2,8.227176E-3,-6.8057356E-3,5.445658E-3,9.093233E-4,3.4621595E-3,-2.5145265E-3,4.1808034E-3,-1.451392E-3,7.954694E4,-7.36504E-5,6.783E3,1.799E3,-6.5387883E-3,-1.7717317E-3,-1.3483303E-3,2.602607E-3,-2.1266646E-3,-6.0041887E-3],"split_indices":[7,57,4,61,0,9,42,37,1,0,66,0,0,2,0,65,12,0,0,57,0,0,0,56,54,2,0,0,0,11,5,0,66,66,0,0,0,0,0,0,0,42,0,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.37E2,1.1E2,3.34E2,3E0,1.01E2,9E0,2.61E2,7.3E1,7E0,9.4E1,8E0,1E0,2.34E2,2.7E1,5.3E1,2E1,4E0,3E0,9.2E1,2E0,7E0,1E0,4E1,1.94E2,3.1E1,2.2E1,8E0,1.2E1,2E1,7.2E1,3E0,3.7E1,1.87E2,7E0,6E0,2.5E1,1.2E1,8E0,6.9E1,3E0,3.3E1,4E0,1.14E2,7.3E1,2.7E1,6E0,9.9E1,1.5E1,5.7E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-2.4648476E-3,-2.1911282E-2,9.5589004E-2,-1.9044286E-2,-1.7799359E-2,7.3832564E-2,1.2818228E-2,-3.68883E-2,2.150659E-2,1.644273E-1,4.2186625E-2,-5.4520853E-2,9.4564405E-4,-2.1731975E-3,5.655874E-2,7.4963766E-4,1.9521956E-1,6.0892243E-2,-6.8506864E-3,-5.7375133E-2,6.2295473E-3,2.2779979E-2,-9.9555336E-2,5.155032E-3,-7.961055E-3,-3.2697916E-3,6.6813126E-2,1.0530829E-2,-2.561558E-3,5.4047484E-4,4.27541E-3,-3.359185E-3,-2.876385E-2,3.8795777E-2,-5.6528047E-2,-2.3772067E-3,-8.038369E-3,3.5212059E-3,-5.286141E-3,4.331911E-3,6.11807E-4,-6.4047255E-2,4.2872044E-4,1.6221214E-2,1.0858575E-1,-6.42691E-3,-7.150892E-5,-2.4725243E-3,4.9000385E-4,6.6502305E-4,-3.9329072E-3,4.2411285E-3,3.3326454E-5,1.0512484E-4,6.625215E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,-1,27,29,-1,31,-1,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,-1,-1,-1,47,-1,-1,49,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.5252625E-1,3.6431313E-1,2.544886E-1,2.692454E-1,0E0,1.8514937E-1,0E0,1.7216566E-1,9.474604E-2,7.489869E-2,1.8444298E-1,9.970629E-2,1.8440075E-1,8.324111E-2,6.319858E-2,0E0,9.2232406E-2,6.387466E-2,0E0,5.9202433E-2,0E0,8.983556E-2,4.240474E-2,4.7868524E-2,0E0,0E0,5.2460656E-2,0E0,0E0,0E0,0E0,0E0,7.231472E-2,8.961634E-2,5.0509486E-2,0E0,0E0,0E0,4.2443927E-2,0E0,0E0,3.9313175E-2,0E0,5.0074738E-2,4.2635053E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,19,19,21,21,22,22,23,23,26,26,32,32,33,33,34,34,38,38,41,41,43,43,44,44],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,-1,28,30,-1,32,-1,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,-1,-1,-1,48,-1,-1,50,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.1331047E-1,1.92144E5,1.0871896E10,7.164074E8,-1.7799359E-2,2.1207E4,1.2818228E-2,2.735322E6,3.082244E3,2.4390244E-3,2.0531621E4,1.5704E4,3.566061E7,1E0,3.7860696E0,7.4963766E-4,1.505649E6,3.3810287E0,-6.8506864E-3,1.2472753E5,6.2295473E-3,1.0965E4,3.019898E8,1.3398149E1,-7.961055E-3,-3.2697916E-3,1.921711E5,1.0530829E-2,-2.561558E-3,5.4047484E-4,4.27541E-3,-3.359185E-3,4.407E3,1.5271514E5,2.4521964E7,-2.3772067E-3,-8.038369E-3,3.5212059E-3,1.8756017E5,4.331911E-3,6.11807E-4,1.3590152E5,4.2872044E-4,7.7815704E2,1.058E3,-6.42691E-3,-7.150892E-5,-2.4725243E-3,4.9000385E-4,6.6502305E-4,-3.9329072E-3,4.2411285E-3,3.3326454E-5,1.0512484E-4,6.625215E-3],"split_indices":[44,2,7,7,0,9,0,1,4,66,4,2,54,18,62,0,2,62,0,37,0,0,7,67,0,0,42,0,0,0,0,0,2,37,1,0,0,0,56,0,0,37,0,42,10,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.45E2,3.72E2,7.3E1,3.7E2,2E0,6.6E1,7E0,2.57E2,1.13E2,1.6E1,5E1,1.75E2,8.2E1,6.8E1,4.5E1,3E0,1.3E1,4.6E1,4E0,1.73E2,2E0,6.8E1,1.4E1,6.6E1,2E0,3E0,4.2E1,1.2E1,1E0,1.7E1,2.9E1,1.2E2,5.3E1,5.7E1,1.1E1,9E0,5E0,8E0,5.8E1,2.9E1,1.3E1,2.7E1,2.6E1,4.4E1,1.3E1,4E0,7E0,1.4E1,4.4E1,5E0,2.2E1,7E0,3.7E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[2.332284E-3,-1.0111674E-2,1.5122518E-1,-3.5899274E-2,3.5259187E-2,6.264397E-2,2.4696381E-1,-3.194116E-2,-1.355992E-2,6.576089E-3,8.364013E-2,4.9457913E-3,-1.1274478E-1,3.6653793E-1,1.5929031E-1,-5.813868E-3,-2.5948763E-2,-8.4147705E-3,4.476327E-3,1.0420965E-2,7.184005E-2,5.1125744E-3,-1.0656507E-2,3.4225993E-3,1.9838516E-2,8.816432E-5,1.9407864E-1,-2.3787909E-3,-1.726262E-2,-2.3217829E-2,1.4428205E-3,5.637783E-3,2.147102E-3,2.092558E-3,1.1134492E-2,-3.0390156E-2,1.8924247E-2,1.0444181E-3,-4.5076936E-2,-1.6096551E-2,-8.0109164E-2,-6.596814E-4,2.2680874E-3,-4.517091E-3,-1.1286876E-3,4.0332586E-4,-1.9406206E-3,-7.0472453E-3,-2.3137315E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,-1,27,29,-1,-1,31,-1,-1,-1,-1,-1,33,-1,35,37,-1,-1,-1,-1,-1,39,41,-1,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.1676865E-1,4.77458E-1,2.7287114E-1,2.4349263E-1,2.0483188E-1,1.4463353E-1,1.0934973E-1,1.3337874E-1,0E0,1.2233809E-1,7.046047E-2,0E0,1.18186004E-1,5.0439358E-2,5.9901297E-2,0E0,4.8309848E-2,4.6664976E-2,0E0,0E0,5.8820963E-2,0E0,0E0,0E0,0E0,0E0,3.995195E-2,0E0,8.5133E-2,5.885643E-2,0E0,0E0,0E0,0E0,0E0,9.1807485E-2,4.464765E-2,0E0,3.993602E-2,6.1188582E-2,5.451578E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,16,16,17,17,20,20,26,26,28,28,29,29,35,35,36,36,38,38,39,39,40,40],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,-1,28,30,-1,-1,32,-1,-1,-1,-1,-1,34,-1,36,38,-1,-1,-1,-1,-1,40,42,-1,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.204522E3,9.7467445E4,1.0078588E5,7.339209E6,1.4931864E9,1.5096262E7,2.1172E4,1.44E2,-1.355992E-2,1.7670108E5,1.4E1,4.9457913E-3,8.8E2,1.0468137E7,6.219584E5,-5.813868E-3,1.192104E3,2.4345527E3,4.476327E-3,1.0420965E-2,6.7728635E6,5.1125744E-3,-1.0656507E-2,3.4225993E-3,1.9838516E-2,8.816432E-5,3.4536388E7,-2.3787909E-3,1.2028214E5,1.4040858E-2,1.4428205E-3,5.637783E-3,2.147102E-3,2.092558E-3,1.1134492E-2,6.6864E4,9.0442514E10,1.0444181E-3,3.2078102E-2,7.330576E2,1.3421E4,-6.596814E-4,2.2680874E-3,-4.517091E-3,-1.1286876E-3,4.0332586E-4,-1.9406206E-3,-7.0472453E-3,-2.3137315E-3],"split_indices":[61,42,37,56,7,9,10,0,0,37,3,0,0,1,41,0,64,61,0,0,54,0,0,0,0,0,1,0,37,66,0,0,0,0,0,11,40,0,66,61,10,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,4.06E2,3.3E1,2.59E2,1.47E2,1.8E1,1.5E1,2.56E2,3E0,9.3E1,5.4E1,1.5E1,3E0,5E0,1E1,1.5E1,2.41E2,8E1,1.3E1,3E0,5.1E1,1E0,2E0,1E0,4E0,2E0,8E0,6.4E1,1.77E2,5.8E1,2.2E1,1.8E1,3.3E1,2E0,6E0,1.3E2,4.7E1,1.9E1,3.9E1,1.02E2,2.8E1,2.2E1,2.5E1,1.1E1,2.8E1,5.1E1,5.1E1,8E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.6051404E-3,-2.6649602E-2,7.7533774E-2,-2.3485838E-2,-1.2671437E-2,2.0491458E-1,6.546822E-2,-3.160151E-2,6.9323026E-2,4.0797484E-3,1.3067117E-2,3.5560835E-2,1.1263215E-1,-5.006758E-2,-1.3172718E-2,9.483306E-3,2.0740277E-3,4.878506E-2,-7.04488E-3,8.423029E-2,2.7373856E-1,-3.8383022E-2,-9.202101E-2,-2.4981238E-2,5.2909657E-2,2.1568153E-2,3.8922979E-3,6.5935985E-3,2.9213415E-3,1.6433045E-2,-1.4869435E-3,-8.102251E-2,-2.3205308E-2,-1.3211794E-1,-2.078888E-3,-7.390784E-3,-7.02921E-2,3.3794367E-3,-2.021341E-3,3.8627523E-3,-4.6953725E-4,-5.6682103E-3,-9.3857344E-4,-1.787941E-2,-7.764921E-3,-7.882124E-3,-1.4675044E-3,1.1363275E-3,-1.4403057E-3,-7.9471625E-2,2.4615955E-3,-1.6958214E-3,1.6405246E-3,-2.4084316E-3,-7.8008412E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,-1,-1,25,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,-1,41,43,45,-1,47,49,-1,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1],"loss_changes":[9.446339E-1,2.2544223E-1,1.7255193E-1,2.3342508E-1,0E0,5.6584626E-2,1.5397623E-1,9.5647484E-2,7.53037E-2,0E0,0E0,1.7530096E-1,1.736989E-1,6.558719E-2,1.12909675E-1,0E0,0E0,5.563563E-2,0E0,3.889835E-2,1.331034E-1,7.060288E-2,5.313152E-2,9.67192E-2,3.8781155E-2,6.859549E-2,0E0,0E0,0E0,0E0,0E0,6.293407E-2,5.8786597E-2,4.6040446E-2,0E0,6.264616E-2,4.192978E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.479356E-2,0E0,0E0,0E0,0E0,0E0,6.968117E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,31,31,32,32,33,33,35,35,36,36,43,43,49,49],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,-1,-1,26,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,-1,42,44,46,-1,48,50,-1,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1],"split_conditions":[1.3210104E9,4.4533052E0,1.3812E4,1.7267E4,-1.2671437E-2,2.094E3,1.3458943E0,7.377246E2,6.124889E6,4.0797484E-3,1.3067117E-2,1.4609149E4,2.3767492E4,4.592E3,2.473E3,9.483306E-3,2.0740277E-3,1E0,-7.04488E-3,8.641592E1,2.043E3,4.94E4,1.3421E4,2.415804E7,5.099706E6,5.0293133E4,3.8922979E-3,6.5935985E-3,2.9213415E-3,1.6433045E-2,-1.4869435E-3,1.964127E6,3.9034148E7,9.606877E-2,-2.078888E-3,7.075361E1,1.7997152E7,3.3794367E-3,-2.021341E-3,3.8627523E-3,-4.6953725E-4,-5.6682103E-3,-9.3857344E-4,7.76E2,-7.764921E-3,-7.882124E-3,-1.4675044E-3,1.1363275E-3,-1.4403057E-3,6.54925E6,2.4615955E-3,-1.6958214E-3,1.6405246E-3,-2.4084316E-3,-7.8008412E-3],"split_indices":[7,66,9,2,0,0,48,61,57,0,0,4,61,10,0,0,0,22,0,67,0,9,10,54,60,37,0,0,0,0,0,1,54,47,0,63,9,0,0,0,0,0,0,0,0,0,0,0,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,3.09E2,1.2E2,3.06E2,3E0,9E0,1.11E2,2.82E2,2.4E1,4E0,5E0,6.9E1,4.2E1,1.4E2,1.42E2,3E0,2.1E1,6.5E1,4E0,3.7E1,5E0,1.11E2,2.9E1,1.21E2,2.1E1,3.6E1,2.9E1,1E1,2.7E1,4E0,1E0,2.8E1,8.3E1,1.5E1,1.4E1,8.8E1,3.3E1,1.8E1,3E0,1.2E1,2.4E1,1.7E1,1.1E1,8.1E1,2E0,1.1E1,4E0,3.7E1,5.1E1,3.1E1,2E0,6.1E1,2E1,2.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.1644588E-5,-2.4696752E-2,6.9402136E-2,-2.791085E-2,9.120471E-3,3.8387597E-2,1.4398505E-1,-2.4921698E-2,-1.2611343E-2,4.726442E-2,-1.0464851E-2,7.315778E-2,2.6773855E-1,-3.3457287E-2,3.609063E-2,8.794251E-3,3.9824177E-2,5.486011E-3,1.2747805E-3,4.042716E-3,1.5415584E-2,-4.060517E-2,6.29155E-3,3.6217048E-4,3.5831544E-3,4.4928018E-2,-5.0361105E-3,-6.4521946E-2,-2.891122E-2,-4.7758933E-2,1.4705128E-3,2.324661E-2,8.001288E-2,-5.6322105E-2,-8.3401585E-3,-5.336574E-3,-6.1767906E-2,-5.3469287E-3,-1.7422335E-5,3.2255175E-3,2.02136E-3,5.2373094E-4,5.103112E-3,-3.5044302E-3,-9.2625374E-4,9.2556336E-5,-6.6433465E-3,-3.4802891E-3,2.8197112E-4,-2.6665442E-3,1.0504845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,19,21,23,-1,25,-1,-1,-1,-1,27,29,-1,-1,31,-1,33,35,37,-1,39,41,43,-1,45,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.642155E-1,2.3141868E-1,2.651118E-1,2.2046527E-1,0E0,2.0071977E-1,2.78427E-1,1.688974E-1,0E0,7.5178325E-2,0E0,4.0641733E-2,8.790034E-2,8.0661416E-2,4.3138392E-2,0E0,6.28642E-2,0E0,0E0,0E0,0E0,6.5258175E-2,5.9026048E-2,0E0,0E0,5.671303E-2,0E0,5.9623033E-2,1.2592769E-1,4.185523E-2,0E0,4.526318E-2,5.064076E-2,4.3532237E-2,0E0,9.2526175E-2,4.8301876E-2,0E0,0E0,0E0,3.9729707E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,16,16,21,21,22,22,25,25,27,27,28,28,29,29,31,31,32,32,33,33,35,35,36,36,40,40],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,20,22,24,-1,26,-1,-1,-1,-1,28,30,-1,-1,32,-1,34,36,38,-1,40,42,44,-1,46,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4817759E9,6.0495757E3,1.486066E0,3.372042E7,9.120471E-3,3.768034E4,2.2876814E1,2.143564E3,-1.2611343E-2,1.5988636E1,-1.0464851E-2,1.0772926E2,1.8636872E0,1.646359E5,9.799349E4,8.794251E-3,1.505649E6,5.486011E-3,1.2747805E-3,4.042716E-3,1.5415584E-2,2.53383E5,4.4593253E2,3.6217048E-4,3.5831544E-3,2.5065362E9,-5.0361105E-3,1.4690361E5,2.6548672E-2,1.964127E6,1.4705128E-3,5.473125E3,2.6475494E1,1.436034E9,-8.3401585E-3,3.566061E7,3.44E2,-5.3469287E-3,-1.7422335E-5,3.2255175E-3,1.904981E7,5.2373094E-4,5.103112E-3,-3.5044302E-3,-9.2625374E-4,9.2556336E-5,-6.6433465E-3,-3.4802891E-3,2.8197112E-4,-2.6665442E-3,1.0504845E-3],"split_indices":[7,61,48,57,0,4,63,61,0,65,0,67,48,37,37,0,2,0,0,0,0,9,61,0,0,7,0,37,66,1,0,41,65,5,0,54,8,0,0,0,41,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.44E2,3.28E2,1.16E2,3.24E2,4E0,8.3E1,3.3E1,3.21E2,3E0,8.1E1,2E0,2.2E1,1.1E1,2.82E2,3.9E1,3E0,7.8E1,1.1E1,1.1E1,3E0,8E0,2.39E2,4.3E1,2.3E1,1.6E1,7.6E1,2E0,7.7E1,1.62E2,1.3E1,3E1,4.8E1,2.8E1,7.3E1,4E0,9.5E1,6.7E1,5E0,8E0,1.5E1,3.3E1,8E0,2E1,5E1,2.3E1,9.1E1,4E0,5.8E1,9E0,8E0,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-7.425536E-4,-2.6380187E-2,5.6513984E-2,-5.513925E-2,-1.12791415E-2,3.5662983E-2,1.451638E-1,-8.915347E-2,-3.9728418E-2,-1.4360557E-2,5.270148E-3,7.650782E-2,3.4659596E-3,4.2631947E-3,1.1389537E-2,4.3772144E-4,-5.016508E-3,-3.2413047E-2,-6.3293916E-3,-2.0289047E-2,1.9617968E-3,5.1459417E-2,7.5257053E-3,1.5569745E-2,-8.482692E-3,-3.142624E-3,-7.8480225E-3,-2.824458E-2,1.8838996E-2,3.2767726E-3,-4.4655343E-4,-6.439841E-3,2.884414E-2,9.019182E-4,-3.1068996E-3,-1.1576326E-3,-5.222232E-3,-9.080228E-4,2.7852366E-3,2.5860188E-4,3.9062114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,25,-1,27,-1,29,-1,31,-1,-1,33,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.312294E-1,1.2836277E-1,2.4066523E-1,4.9925208E-2,7.439621E-2,1.4307877E-1,1.0828745E-1,4.62846E-2,4.396163E-2,6.3485324E-2,0E0,8.764365E-2,1.3736518E-1,0E0,0E0,0E0,0E0,5.4220006E-2,0E0,5.46956E-2,0E0,3.8703687E-2,0E0,1.20651335E-1,0E0,0E0,6.131177E-2,4.6568036E-2,4.508085E-2,0E0,0E0,0E0,6.694098E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,17,17,19,19,21,21,23,23,26,26,27,27,28,28,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,26,-1,28,-1,30,-1,32,-1,-1,34,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,4.407E3,9.908038E1,9.357879E2,2.5653894E5,1.8607925E6,1E0,5.5347366E0,2.0340272E8,1.2678202E9,5.270148E-3,9.243444E-1,3.768034E4,4.2631947E-3,1.1389537E-2,4.3772144E-4,-5.016508E-3,1.65602E5,-6.3293916E-3,1.5626266E3,1.9617968E-3,1.1532659E7,7.5257053E-3,2.4950776E0,-8.482692E-3,-3.142624E-3,1.6911902E-1,3.9608764E7,1.1172539E9,3.2767726E-3,-4.4655343E-4,-6.439841E-3,2.826907E7,9.019182E-4,-3.1068996E-3,-1.1576326E-3,-5.222232E-3,-9.080228E-4,2.7852366E-3,2.5860188E-4,3.9062114E-3],"split_indices":[2,2,65,64,37,56,13,56,41,7,0,44,4,0,0,0,0,9,0,61,0,54,0,63,0,0,48,54,5,0,0,0,57,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.96E2,1.32E2,1.01E2,1.95E2,1.08E2,2.4E1,3E1,7.1E1,1.91E2,4E0,4.7E1,6.1E1,1.6E1,8E0,4E0,2.6E1,6.7E1,4E0,1.73E2,1.8E1,3.7E1,1E1,5.8E1,3E0,2.8E1,3.9E1,1.44E2,2.9E1,2.9E1,8E0,4E0,5.4E1,2.7E1,1.2E1,1.38E2,6E0,1.5E1,1.4E1,3.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.1608982E-3,-2.5617968E-2,5.6591142E-2,-2.8835727E-2,1.1291447E-2,3.5150476E-2,1.4714272E-1,-2.5942847E-2,-1.1922415E-2,4.3894418E-2,-9.37422E-3,3.2064815E-3,2.7415386E-1,-3.9646525E-2,2.3812214E-2,-3.8118593E-3,4.8464984E-2,3.6824434E-3,1.5417127E-2,-8.25635E-2,-3.185909E-2,5.5136794E-4,5.5549056E-3,3.217715E-2,8.2629755E-2,-6.004239E-3,-2.6347095E-3,-2.5528047E-2,-8.451258E-2,7.154748E-4,4.0172916E-3,7.7434597E-3,2.5649364E-3,-2.1816252E-2,-6.484229E-3,-2.1375704E-3,-6.6356384E-3,-1.6716645E-3,2.0839907E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,17,19,21,-1,23,-1,-1,25,27,-1,-1,29,31,-1,-1,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1],"loss_changes":[6.1334866E-1,2.7259558E-1,2.413966E-1,1.926361E-1,0E0,2.1730003E-1,2.275809E-1,2.1367423E-1,0E0,6.058982E-2,0E0,0E0,5.6744456E-2,7.9251826E-2,7.5604215E-2,0E0,5.1875263E-2,0E0,0E0,3.6545694E-2,6.751299E-2,0E0,0E0,5.9059426E-2,6.2597826E-2,0E0,0E0,7.309689E-2,4.0845722E-2,0E0,0E0,0E0,0E0,5.2350216E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,12,12,13,13,14,14,16,16,19,19,20,20,23,23,24,24,27,27,28,28,33,33],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,18,20,22,-1,24,-1,-1,26,28,-1,-1,30,32,-1,-1,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,6.942396E3,1.9900316E0,3.764913E7,1.1291447E-2,3.768034E4,2.2876814E1,1.4720299E3,-1.1922415E-2,2.2782404E0,-9.37422E-3,3.2064815E-3,1.082556E7,4.26944E5,2.4013862E5,-3.8118593E-3,8.51E4,3.6824434E-3,1.5417127E-2,5.6E1,1E0,5.5136794E-4,5.5549056E-3,1.0816122E2,2.1207E4,-6.004239E-3,-2.6347095E-3,4.086885E7,1.3485402E8,7.154748E-4,4.0172916E-3,7.7434597E-3,2.5649364E-3,3.312442E6,-6.484229E-3,-2.1375704E-3,-6.6356384E-3,-1.6716645E-3,2.0839907E-5],"split_indices":[7,61,48,57,0,4,63,61,0,63,0,0,1,1,37,0,2,0,0,10,18,0,0,63,9,0,0,54,7,0,0,0,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,3.17E2,1.26E2,3.14E2,3E0,1.03E2,2.3E1,3.11E2,3E0,1E2,3E0,1.5E1,8E0,2.44E2,6.7E1,3E0,9.7E1,2E0,6E0,3.6E1,2.08E2,6E1,7E0,6.7E1,3E1,1.3E1,2.3E1,1.87E2,2.1E1,5.1E1,1.6E1,7E0,2.3E1,1.82E2,5E0,1.3E1,8E0,1.15E2,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.7882137E-4,-1.5600791E-2,8.467687E-2,-3.5127643E-2,1.7680854E-2,2.0761408E-1,5.5789396E-2,-3.140863E-2,-1.1491278E-2,2.220131E-3,7.4649036E-2,3.0156274E-3,1.4575697E-2,1.0422491E-2,3.5013273E-2,-4.7223568E-2,-8.770623E-3,-2.822796E-2,1.9152353E-2,1.0054979E-2,2.7486647E-3,-1.4768976E-1,4.5930203E-2,-6.516512E-2,-1.7677957E-2,5.117715E-2,-9.112158E-4,-5.8489703E-3,-8.245713E-4,1.8094627E-3,-6.4555556E-4,-1.0883824E-2,2.0232779E-4,-4.438754E-4,3.474583E-3,-2.1992107E-3,-4.503815E-3,-2.0400214E-3,6.1824813E-4,2.9837617E-4,6.3123973E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.550414E-1,2.3847927E-1,2.1925479E-1,1.6708362E-1,1.1961889E-1,1.4592665E-1,1.723561E-1,8.1214085E-2,0E0,5.6220442E-2,5.6611538E-2,0E0,0E0,0E0,1.045167E-1,6.9880605E-2,5.8646683E-2,3.8315274E-2,4.2360794E-2,0E0,0E0,3.6507323E-2,6.838393E-2,4.1577816E-2,3.9609708E-2,4.9414933E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.33069E3,7.222818E4,1.064816E6,5.38168E6,1.646359E5,1.082556E7,1.3812E4,5.855E3,-1.1491278E-2,2.0231652E3,1.4E1,3.0156274E-3,1.4575697E-2,1.0422491E-2,8.713209E10,1.16985195E5,4.9832974E2,1E0,1.1106066E7,1.0054979E-2,2.7486647E-3,1.379E3,1.4015416E6,1.6645125E5,6.603817E2,6.651711E1,-9.112158E-4,-5.8489703E-3,-8.245713E-4,1.8094627E-3,-6.4555556E-4,-1.0883824E-2,2.0232779E-4,-4.438754E-4,3.474583E-3,-2.1992107E-3,-4.503815E-3,-2.0400214E-3,6.1824813E-4,2.9837617E-4,6.3123973E-3],"split_indices":[61,42,38,56,37,1,9,2,0,64,3,0,0,0,40,37,61,13,57,0,0,0,41,57,61,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,3.65E2,6.4E1,2.3E2,1.35E2,1.1E1,5.3E1,2.27E2,3E0,1.07E2,2.8E1,5E0,6E0,5E0,4.8E1,1.33E2,9.4E1,3.8E1,6.9E1,2E0,2.6E1,2E0,4.6E1,8.2E1,5.1E1,1.3E1,8.1E1,3E0,3.5E1,4.4E1,2.5E1,1E0,1E0,1.5E1,3.1E1,5E1,3.2E1,2.8E1,2.3E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[7.2130472E-3,-2.1580344E-2,4.50928E-2,-2.473751E-2,1.1433633E-2,2.0849803E-2,9.093898E-2,-2.1055685E-2,-1.1368388E-2,-2.4087618E-3,2.7678939E-2,7.301811E-3,3.480223E-3,-3.0350555E-3,-1.3447979E-2,3.5205458E-3,8.4484613E-4,2.2282766E-3,-1.9568963E-2,-3.3183384E-3,-1.0603148E-2,-5.6553134E-3,-4.803756E-3,-6.42433E-4,1.7365646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,15,-1,-1,-1,17,-1,-1,-1,19,-1,21,-1,23,-1,-1],"loss_changes":[4.623335E-1,2.027159E-1,2.014991E-1,1.8074432E-1,0E0,5.9892904E-2,6.057012E-2,7.436624E-2,0E0,0E0,4.9834087E-2,0E0,0E0,0E0,7.430823E-2,0E0,0E0,0E0,8.008267E-2,0E0,9.595797E-2,0E0,5.2302275E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,10,10,14,14,18,18,20,20,22,22],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,16,-1,-1,-1,18,-1,-1,-1,20,-1,22,-1,24,-1,-1],"split_conditions":[8.393928E4,8.861901E3,6.0891E4,7.339209E6,1.1433633E-2,1.5802073E3,1.799389E6,4.26944E5,-1.1368388E-2,-2.4087618E-3,2.171486E7,7.301811E-3,3.480223E-3,-3.0350555E-3,1.12608824E8,3.5205458E-3,8.4484613E-4,2.2282766E-3,3.9739098E4,-3.3183384E-3,2.2744698E2,-5.6553134E-3,7.2008003E9,-6.42433E-4,1.7365646E-3],"split_indices":[42,61,2,56,0,4,38,1,0,0,41,0,0,0,5,0,0,0,37,0,61,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.22E2,2.4E2,1.82E2,2.38E2,2E0,1.2E2,6.2E1,2.35E2,3E0,1E1,1.1E2,1.3E1,4.9E1,3.5E1,2E2,1.9E1,9.1E1,1.8E1,1.82E2,2.7E1,1.55E2,7E0,1.48E2,1.23E2,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.3705716E-3,-1.695825E-2,7.0130505E-2,-1.44198015E-2,-2.4752513E-1,4.7887288E-2,8.826383E-3,-1.6980806E-2,8.482616E-3,-1.5520502E-2,-5.885488E-4,8.2766965E-2,-6.4275335E-3,-4.7499113E-2,-5.1700603E-3,5.9994895E-2,8.1262E-3,-5.477538E-2,1.8657475E-3,-2.5259051E-3,1.2693045E-3,-1.23497E-2,1.2888594E-3,-1.6161288E-3,3.6485353E-3,9.180501E-4,-4.535349E-3,-5.2504176E-3,-5.4776024E-2,-2.3996872E-3,9.871076E-4,-5.7508405E-3,-7.5644406E-5,2.5938303E-4,-3.6522304E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,17,19,21,23,-1,25,-1,-1,-1,27,-1,-1,-1,-1,-1,29,31,-1,33,-1,-1,-1,-1],"loss_changes":[4.416199E-1,2.1474093E-1,1.6455519E-1,1.8335742E-1,6.607401E-2,1.12306505E-1,0E0,1.3167691E-1,0E0,0E0,0E0,6.020297E-2,5.447821E-2,3.818819E-2,6.1025206E-2,4.6513505E-2,0E0,3.659184E-2,0E0,0E0,0E0,6.507203E-2,0E0,0E0,0E0,0E0,0E0,5.1977064E-2,1.0619502E-1,0E0,5.602186E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,12,12,13,13,14,14,15,15,17,17,21,21,27,27,28,28,30,30],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,18,20,22,24,-1,26,-1,-1,-1,28,-1,-1,-1,-1,-1,30,32,-1,34,-1,-1,-1,-1],"split_conditions":[3.5257432E6,1.69595E5,1.6828056E1,6.6512886E3,8.75E3,2.3619232E7,8.826383E-3,4.407E3,8.482616E-3,-1.5520502E-2,-5.885488E-4,2.36316E5,5.0314346E1,1.3421E4,1.5271514E5,2.4054918E0,8.1262E-3,1.619002E9,1.8657475E-3,-2.5259051E-3,1.2693045E-3,1E0,1.2888594E-3,-1.6161288E-3,3.6485353E-3,9.180501E-4,-4.535349E-3,3.1E2,1.3395E3,-2.3996872E-3,4.2663252E7,-5.7508405E-3,-7.5644406E-5,2.5938303E-4,-3.6522304E-3],"split_indices":[53,2,44,61,0,52,0,2,0,0,0,2,65,10,37,62,0,7,0,0,0,18,0,0,0,0,0,0,61,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.72E2,6.8E1,3.69E2,3E0,5.8E1,1E1,3.65E2,4E0,2E0,1E0,3.5E1,2.3E1,1.01E2,2.64E2,2.9E1,6E0,1.1E1,1.2E1,9.5E1,6E0,2.16E2,4.8E1,4E0,2.5E1,4E0,7E0,1.86E2,3E1,2.2E1,1.64E2,1.3E1,1.7E1,1.56E2,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.4583319E-3,-1.1534438E-2,1.5549387E-1,-3.122146E-2,2.1839146E-2,1.2893193E-2,6.4450406E-2,-2.7664341E-2,-9.711179E-3,7.3557645E-3,3.8112127E-3,6.9922693E-3,1.7243133E-4,-3.9504915E-2,7.732658E-3,-1.2378537E-3,2.1257935E-2,-6.0336195E-2,-2.42357E-2,5.352549E-2,-1.1476724E-3,-2.2031E-3,1.3429144E-3,-2.809954E-2,-9.353005E-2,-9.949643E-2,-1.4772796E-2,7.039903E-3,1.6930732E-3,-3.036208E-3,9.62241E-4,3.5662823E-3,-4.824291E-3,-6.0375687E-3,9.3598187E-4,-2.4851859E-2,2.2891334E-3,-7.903724E-3,-1.0243363E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,-1,17,19,-1,21,23,25,27,-1,-1,-1,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1],"loss_changes":[6.067689E-1,2.6349247E-1,2.1710825E-1,1.4698428E-1,1.2286951E-1,0E0,6.8145834E-2,1.0424742E-1,0E0,5.5646565E-2,0E0,0E0,0E0,5.7700098E-2,9.2252135E-2,0E0,3.8798522E-2,8.0836415E-2,7.615831E-2,3.9143242E-2,0E0,0E0,0E0,6.99646E-2,4.8936963E-2,3.9135024E-2,6.266672E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.915718E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,13,13,14,14,16,16,17,17,18,18,19,19,23,23,24,24,25,25,26,26,35,35],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,-1,18,20,-1,22,24,26,28,-1,-1,-1,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1],"split_conditions":[8.10288E3,9.7467445E4,3.0418E4,7.339209E6,1.7670108E5,1.2893193E-2,2.6293862E7,7.198E3,-9.711179E-3,7.576111E-1,3.8112127E-3,6.9922693E-3,1.7243133E-4,7.683712E4,1.3404826E-3,-1.2378537E-3,1.6861708E3,3.5546432E6,9.284744E2,4.9832974E2,-1.1476724E-3,-2.2031E-3,1.3429144E-3,1.3202006E3,1.198E5,2.9293E4,2.3450147E5,7.039903E-3,1.6930732E-3,-3.036208E-3,9.62241E-4,3.5662823E-3,-4.824291E-3,-6.0375687E-3,9.3598187E-4,2.90694E5,2.2891334E-3,-7.903724E-3,-1.0243363E-3],"split_indices":[61,42,10,56,37,0,60,2,0,51,0,0,0,37,66,0,64,54,4,61,0,0,0,4,7,0,37,0,0,0,0,0,0,0,0,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.21E2,3.99E2,2.2E1,2.51E2,1.48E2,9E0,1.3E1,2.47E2,4E0,1.19E2,2.9E1,5E0,8E0,1.85E2,6.2E1,3.5E1,8.4E1,7.7E1,1.08E2,2.5E1,3.7E1,7E0,7.7E1,4E1,3.7E1,1.1E1,9.7E1,3E0,2.2E1,2.3E1,1.7E1,1E0,3.6E1,9E0,2E0,8.4E1,1.3E1,1E0,8.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[6.5627357E-4,-7.5833257E-3,1.3932885E-1,-1.8905044E-2,3.7797164E-2,-9.130998E-3,1.8932274E-1,-1.6764231E-2,-1.1686765E-2,4.4661473E-2,-5.087076E-3,3.0644077E-3,-1.0734132E-2,2.2437888E-1,2.5699567E-3,-2.1590445E-2,2.2620638E-3,1.6471706E-3,9.21482E-3,1.4216163E-2,3.9192354E-3,-3.0430735E-3,-1.5327132E-2,-1.5872812E-3,-4.156788E-2,-1.0583995E-2,1.7118916E-3,-8.069014E-3,-3.5306927E-2,-3.3884388E-3,-2.1402098E-4,-2.1350353E-3,2.284759E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,19,-1,21,-1,-1,-1,-1,-1,-1,23,25,27,29,-1,-1,31,-1,-1,-1,-1],"loss_changes":[4.8894954E-1,2.0828374E-1,1.7985877E-1,1.509468E-1,8.439002E-2,1.22777455E-1,7.387155E-2,9.997353E-2,0E0,1.134067E-1,0E0,0E0,0E0,1.171968E-1,0E0,7.705416E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.439653E-2,5.8106907E-2,6.496872E-2,5.104299E-2,0E0,0E0,6.6658266E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,13,13,15,15,22,22,23,23,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,20,-1,22,-1,-1,-1,-1,-1,-1,24,26,28,30,-1,-1,32,-1,-1,-1,-1],"split_conditions":[8.10288E3,1.484798E9,6.274818E4,3.372042E7,1.505649E6,4.42384E5,8.639938E9,2.1603052E9,-1.1686765E-2,1.4646042E3,-5.087076E-3,3.0644077E-3,-1.0734132E-2,5.014586E8,2.5699567E-3,4.26944E5,2.2620638E-3,1.6471706E-3,9.21482E-3,1.4216163E-2,3.9192354E-3,-3.0430735E-3,1.8071064E7,2.068E3,9.284744E2,2.9331678E1,1.7118916E-3,-8.069014E-3,1.0328767E7,-3.3884388E-3,-2.1402098E-4,-2.1350353E-3,2.284759E-3],"split_indices":[61,7,37,57,2,2,12,12,0,66,0,0,0,41,0,1,0,0,0,0,0,0,54,0,4,65,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,4.03E2,2.3E1,3.23E2,8E1,6E0,1.7E1,3.21E2,2E0,7.7E1,3E0,5E0,1E0,1.3E1,4E0,2.99E2,2.2E1,7.3E1,4E0,8E0,5E0,3.8E1,2.61E2,1.72E2,8.9E1,1.39E2,3.3E1,3E0,8.6E1,1.2E1,1.27E2,7.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.1396204E-3,-9.026371E-3,8.2793705E-2,-6.6730184E-3,-8.881044E-3,1.6568163E-1,5.0026257E-2,-2.8848872E-2,1.4058664E-2,2.6581176E-3,2.2649997E-1,3.3879343E-3,-7.949678E-3,-3.6493592E-2,1.6389104E-2,6.6579976E-3,6.911331E-2,2.7903984E-3,1.2604731E-2,1.1418057E-3,-2.0092828E-3,1.4502521E-3,-4.9734935E-3,9.129682E-4,-1.801079E-2,6.014098E-3,1.6734404E-2,-5.911734E-3,-4.8583007E-4,-1.3404309E-3,4.5106243E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1],"loss_changes":[4.2423132E-1,1.5541798E-1,1.4924106E-1,1.7326885E-1,0E0,9.509078E-2,1.9861609E-1,6.353946E-2,7.8990854E-2,0E0,4.3759704E-2,0E0,0E0,4.9603507E-2,4.7285352E-2,5.2735575E-2,6.491808E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.690198E-2,0E0,4.7073822E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,24,24,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1],"split_conditions":[4.0292856E3,4.2064876E7,1.417988E6,6.24E3,-8.881044E-3,1.2273383E7,2.3767492E4,2.1592189E5,7.049895E9,2.6581176E-3,8.596084E3,3.3879343E-3,-7.949678E-3,3.4206028E-6,1.099906E6,1.7458724E7,1.4399966E10,2.7903984E-3,1.2604731E-2,1.1418057E-3,-2.0092828E-3,1.4502521E-3,-4.9734935E-3,9.129682E-4,6.79499E7,6.014098E-3,1.13E2,-5.911734E-3,-4.8583007E-4,-1.3404309E-3,4.5106243E-3],"split_indices":[61,59,38,2,0,54,61,37,12,0,4,0,0,50,9,54,12,0,0,0,0,0,0,0,7,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.36E2,3.79E2,5.7E1,3.75E2,4E0,1.5E1,4.2E1,1.81E2,1.94E2,6E0,9E0,3.9E1,3E0,1.55E2,2.6E1,1.72E2,2.2E1,2E0,7E0,1.2E1,1.43E2,2.4E1,2E0,1.15E2,5.7E1,1E1,1.2E1,3E0,5.4E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[6.174467E-4,-1.7673716E-2,4.9083885E-2,-5.153561E-2,-1.0552209E-2,1.484252E-1,3.3478692E-2,-6.0106937E-2,1.3732556E-3,-1.7875794E-2,1.1834028E-3,2.2349858E-3,2.2266375E-1,4.329124E-2,-4.3833028E-3,-7.384599E-2,1.07707965E-4,2.5161989E-2,-2.4316736E-2,5.473277E-3,1.5945943E-2,3.3506885E-2,7.989435E-3,-2.820843E-3,-7.405381E-3,1.9542696E-3,-2.7347393E-3,-8.895128E-2,-1.9872673E-2,4.063512E-2,-9.23379E-3,-8.171374E-3,-2.0867689E-3,-3.5897724E-2,7.5545236E-6,9.99893E-4,6.731727E-2,-1.9251014E-3,3.4126656E-3,6.6402E-3,2.2650785E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,19,21,-1,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,33,35,-1,-1,-1,37,-1,-1,39,-1,-1,-1,-1],"loss_changes":[3.8827127E-1,7.614116E-2,1.8105972E-1,3.9433762E-2,6.8294324E-2,1.10920966E-1,1.3209534E-1,4.2990148E-2,0E0,6.1319508E-2,0E0,0E0,6.860441E-2,1.1182277E-1,0E0,4.117252E-2,0E0,3.8794674E-2,5.309514E-2,0E0,0E0,1.5534703E-1,0E0,0E0,0E0,0E0,0E0,3.679675E-2,5.7780474E-2,4.638523E-2,0E0,0E0,0E0,4.580544E-2,0E0,0E0,4.7942996E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,12,12,13,13,15,15,17,17,18,18,21,21,27,27,28,28,29,29,33,33,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,20,22,-1,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,34,36,-1,-1,-1,38,-1,-1,40,-1,-1,-1,-1],"split_conditions":[9.407814E6,3.807E3,1.417988E6,4.049296E-2,1.5271514E5,9.256843E3,1.0429407E3,1.6013113E7,1.3732556E-3,4.3712845E0,1.1834028E-3,2.2349858E-3,1.281E3,1.6941456E1,-4.3833028E-3,3.2859375E1,1.07707965E-4,2.7174674E11,2.7515392E1,5.473277E-3,1.5945943E-2,1.4609149E4,7.989435E-3,-2.820843E-3,-7.405381E-3,1.9542696E-3,-2.7347393E-3,5.497718E7,3.447954E5,9.3058E4,-9.23379E-3,-8.171374E-3,-2.0867689E-3,2.1072E4,7.5545236E-6,9.99893E-4,2.1207E4,-1.9251014E-3,3.4126656E-3,6.6402E-3,2.2650785E-3],"split_indices":[1,2,38,66,37,4,67,54,0,67,0,0,0,44,0,67,0,40,65,0,0,4,0,0,0,0,0,41,56,2,0,0,0,2,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.36E2,3.17E2,1.19E2,5.4E1,2.63E2,1.5E1,1.04E2,4.9E1,5E0,2.18E2,4.5E1,7E0,8E0,9.7E1,7E0,4E1,9E0,2.8E1,1.9E2,5E0,3E0,9.1E1,6E0,3.5E1,5E0,2.4E1,4E0,1.1E1,1.79E2,8.9E1,2E0,3E0,8E0,9.9E1,8E1,5.2E1,3.7E1,9.6E1,3E0,7E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.4278618E-3,-2.4258455E-2,2.77665E-2,-2.1517849E-2,-9.136042E-3,1.07649295E-2,7.277538E-2,-2.6888081E-3,-1.4532491E-2,-4.8823893E-4,3.2823458E-3,9.018398E-3,3.0914873E-3,1.3577322E-2,-1.1653621E-3,4.5367274E-3,-1.0993485E-1,3.8303647E-2,-4.7541678E-2,-6.946851E-3,2.092543E-3,-8.936979E-3,1.0262958E-4,-2.3062988E-3,2.387385E-3,-8.615528E-4,-6.369016E-3,-8.840807E-4,1.7581577E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,-1,13,15,-1,-1,-1,17,-1,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8740856E-1,1.1048435E-1,1.3925214E-1,5.8998443E-2,0E0,8.645396E-2,3.7472248E-2,0E0,5.6946013E-2,6.2644854E-2,0E0,0E0,0E0,8.342843E-2,0E0,4.8946317E-2,4.2652767E-2,4.0029086E-2,3.778229E-2,4.2693444E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,13,13,15,15,16,16,17,17,18,18,19,19],"right_children":[2,4,6,8,-1,10,12,-1,14,16,-1,-1,-1,18,-1,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.393928E4,7.339209E6,5.9209933E9,3.606E3,-9.136042E-3,1.7670108E5,9.957E3,-2.6888081E-3,1.0106809E1,2.36316E5,3.2823458E-3,9.018398E-3,3.0914873E-3,2.8184534E7,-1.1653621E-3,1.0284004E2,1.354316E6,9.72597E2,5.309569E6,1.3590209E7,2.092543E-3,-8.936979E-3,1.0262958E-4,-2.3062988E-3,2.387385E-3,-8.615528E-4,-6.369016E-3,-8.840807E-4,1.7581577E-3],"split_indices":[42,56,12,2,0,37,9,0,67,2,0,0,0,54,0,63,2,4,9,1,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,2.52E2,1.82E2,2.49E2,3E0,1.33E2,4.9E1,4.1E1,2.08E2,1.12E2,2.1E1,2E0,4.7E1,5.3E1,1.55E2,1.08E2,4E0,3.8E1,1.5E1,8.4E1,2.4E1,2E0,2E0,4E0,3.4E1,1.2E1,3E0,6.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.3218154E-3,-9.106642E-3,9.967895E-2,-4.105247E-2,3.1288313E-3,1.3410749E-1,1.0491855E-3,-2.9379938E-2,-3.599534E-3,5.5978633E-3,-8.755831E-3,7.851731E-3,2.375265E-3,-1.9080386E-3,1.2823094E-3,-2.7784887E-3,3.6267314E-2,-1.1318671E-2,3.072844E-2,5.9064813E-3,1.0492147E-3,-1.7772287E-2,1.3414749E-3,-4.19067E-3,1.9528773E-3,7.476601E-4,-3.2971207E-2,1.1479123E-3,-1.4591436E-3,-1.0181712E-3,-4.3415837E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,-1,-1,-1,-1,17,19,21,23,-1,-1,25,-1,-1,-1,27,29,-1,-1,-1,-1],"loss_changes":[3.5158566E-1,1.6234046E-1,8.2926005E-2,4.2912006E-2,1.3830031E-1,4.5618862E-2,0E0,5.0689943E-2,0E0,7.665641E-2,0E0,0E0,0E0,0E0,0E0,6.759429E-2,7.739504E-2,4.798335E-2,5.871452E-2,0E0,0E0,4.562493E-2,0E0,0E0,0E0,5.363799E-2,5.8312833E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,15,15,16,16,17,17,18,18,21,21,25,25,26,26],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,-1,-1,-1,-1,18,20,22,24,-1,-1,26,-1,-1,-1,28,30,-1,-1,-1,-1],"split_conditions":[6.6512886E3,4.822E3,2.8184534E7,1.2273383E7,5.183582E7,8.639938E9,1.0491855E-3,2.455358E6,-3.599534E-3,1.5271514E5,-8.755831E-3,7.851731E-3,2.375265E-3,-1.9080386E-3,1.2823094E-3,3.53E2,2.7E1,6.425793E8,3.2E1,5.9064813E-3,1.0492147E-3,2.7977833E1,1.3414749E-3,-4.19067E-3,1.9528773E-3,2.7394568E7,1E0,1.1479123E-3,-1.4591436E-3,-1.0181712E-3,-4.3415837E-3],"split_indices":[61,2,54,54,59,12,0,1,0,37,0,0,0,0,0,8,3,41,3,0,0,67,0,0,0,54,18,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.45E2,4.14E2,3.1E1,1.14E2,3E2,2.1E1,1E1,8.6E1,2.8E1,2.97E2,3E0,1.5E1,6E0,7.3E1,1.3E1,2.34E2,6.3E1,1.87E2,4.7E1,8E0,5.5E1,1.61E2,2.6E1,3E0,4.4E1,7.3E1,8.8E1,4.2E1,3.1E1,7.4E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.2385537E-4,-2.430536E-2,2.4058228E-2,-1.1823806E-2,-4.6359245E-2,-1.058663E-1,2.8315023E-2,-2.47558E-3,4.830646E-3,-7.2627135E-2,-1.0964356E-3,-9.6044317E-4,-1.0309623E-2,1.1526391E-1,2.1446217E-2,1.3003441E-2,-3.7335535E-3,-5.733626E-3,-4.4528075E-2,1.2548227E-2,1.8747448E-3,1.00893825E-2,3.0041854E-3,3.273948E-2,-7.1674585E-4,-4.24653E-3,-3.4394677E-4,1.4681425E-2,-7.938006E-3,3.1765422E-3,3.420459E-4,-1.5650715E-3,2.2183586E-2,5.9562456E-2,1.1806965E-2,3.5176522E-3,-1.8823156E-3,8.4920415E-5,2.4565628E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,19,21,23,-1,-1,25,-1,-1,27,-1,29,-1,-1,-1,31,-1,-1,-1,-1,33,35,37,-1,-1,-1,-1],"loss_changes":[2.502769E-1,5.8358938E-2,1.21352896E-1,9.1076985E-2,4.6121165E-2,6.0689755E-2,1.217529E-1,0E0,6.6895485E-2,4.152876E-2,0E0,0E0,0E0,1.5599579E-1,8.939672E-2,4.9988415E-2,0E0,0E0,3.8452204E-2,0E0,0E0,1.2517501E-1,0E0,4.4579677E-2,0E0,0E0,0E0,5.376905E-2,0E0,0E0,0E0,0E0,4.971938E-2,3.9826386E-2,4.0411074E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,13,13,14,14,15,15,18,18,21,21,23,23,27,27,32,32,33,33,34,34],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,20,22,24,-1,-1,26,-1,-1,28,-1,30,-1,-1,-1,32,-1,-1,-1,-1,34,36,38,-1,-1,-1,-1],"split_conditions":[1.0975164E3,1.3058374E7,2.4950776E0,5.42218E5,2.72E2,4.352718E7,1.3812E4,-2.47558E-3,1E0,1.9281628E7,-1.0964356E-3,-9.6044317E-4,-1.0309623E-2,3.7137297E3,1.8770729E0,1.33E2,-3.7335535E-3,-5.733626E-3,1E0,1.2548227E-2,1.8747448E-3,2.5212732E4,3.0041854E-3,1E0,-7.1674585E-4,-4.24653E-3,-3.4394677E-4,4.59E2,-7.938006E-3,3.1765422E-3,3.420459E-4,-1.5650715E-3,9.090909E-3,5.374925E7,2.5501363E9,3.5176522E-3,-1.8823156E-3,8.4920415E-5,2.4565628E-3],"split_indices":[61,54,63,1,3,54,9,0,18,54,0,0,0,41,50,3,0,0,13,0,0,4,0,23,0,0,0,0,0,0,0,0,66,54,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,2.13E2,2.13E2,1.37E2,7.6E1,6E0,2.07E2,4E1,9.7E1,3.5E1,4.1E1,4E0,2E0,1.4E1,1.93E2,8.9E1,8E0,1.2E1,2.3E1,4E0,1E1,1.52E2,4.1E1,5.2E1,3.7E1,1E1,1.3E1,1.49E2,3E0,2.2E1,3E1,2E1,1.29E2,2.7E1,1.02E2,2.4E1,3E0,8.2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.4728844E-3,-1.5937453E-2,4.127473E-2,-1.39959855E-2,-8.228916E-3,5.9424777E-2,-9.744388E-3,-2.1718781E-2,1.4155486E-2,3.9528202E-2,2.5843468E-1,-8.5376045E-3,1.2504392E-2,-3.0208956E-2,-4.4858893E-3,-1.3921475E-3,1.263094E-3,1.117412E-2,3.3917595E-3,1.304868E-3,1.4430851E-2,8.300236E-3,-1.2425112E-4,-8.168185E-2,-2.3283029E-2,1.6136144E-3,-1.1756049E-3,-1.2419504E-3,2.1906309E-3,2.6990497E-4,-5.359116E-3,-3.8764328E-2,-2.6595037E-5,-4.8456784E-2,4.675791E-4,-1.0479054E-3,-3.2496506E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1,33,-1,35,-1,-1,-1],"loss_changes":[2.3547466E-1,9.674475E-2,8.577101E-2,7.139986E-2,0E0,2.590384E-1,9.528406E-2,3.746622E-2,3.7542474E-2,5.42626E-2,4.862559E-2,0E0,5.581915E-2,5.9415773E-2,6.5755166E-2,0E0,0E0,4.4189975E-2,0E0,0E0,0E0,0E0,0E0,5.2058563E-2,5.38212E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.3213993E-2,0E0,3.700386E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,23,23,24,24,31,31,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1,34,-1,36,-1,-1,-1],"split_conditions":[1.613448E9,3.764913E7,5.358657E7,2.231E3,-8.228916E-3,3.63796E5,3.087E3,7.177E3,4.5765094E2,3.3487457E11,7.34E2,-8.5376045E-3,8.169258E3,4.0842392E1,1.2283395E-2,-1.3921475E-3,1.263094E-3,5.52488E1,3.3917595E-3,1.304868E-3,1.4430851E-2,8.300236E-3,-1.2425112E-4,3.572932E2,8.1672195E4,1.6136144E-3,-1.1756049E-3,-1.2419504E-3,2.1906309E-3,2.6990497E-4,-5.359116E-3,1.93E2,-2.6595037E-5,4.750965E2,4.675791E-4,-1.0479054E-3,-3.2496506E-3],"split_indices":[7,57,1,0,0,2,0,2,61,40,0,0,42,63,66,0,0,65,0,0,0,0,0,42,37,0,0,0,0,0,0,8,0,61,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,3.29E2,9.1E1,3.26E2,3E0,6.7E1,2.4E1,2.56E2,7E1,6.2E1,5E0,2E0,2.2E1,1.71E2,8.5E1,1.5E1,5.5E1,3.3E1,2.9E1,1E0,4E0,1E0,2.1E1,1.9E1,1.52E2,2.9E1,5.6E1,1.6E1,1.7E1,5E0,1.4E1,9E1,6.2E1,7.5E1,1.5E1,3.2E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.3525167E-4,-8.102568E-3,1.0309668E-1,-2.4368891E-2,1.8807622E-2,1.9930716E-1,4.502608E-2,-2.1625508E-2,-7.3420736E-3,6.402053E-3,1.5556304E-2,1.4198481E-2,4.9593076E-3,3.080649E-3,-2.2378895E-3,-2.3109408E-2,8.100135E-3,3.8374685E-2,-2.3281675E-4,-2.7043223E-2,1.1705592E-3,-1.6683136E-4,2.7362285E-3,-4.610841E-2,-1.30496025E-2,-1.3827949E-3,-7.218367E-2,2.7842136E-4,-3.8823783E-2,-4.73967E-3,-4.6195305E-4,-2.3648087E-3,1.8456845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,-1,-1,-1,-1,19,-1,21,-1,23,-1,-1,-1,25,27,-1,29,-1,31,-1,-1,-1,-1],"loss_changes":[3.4471107E-1,1.729239E-1,1.58636E-1,8.3027214E-2,5.3147387E-2,7.0682615E-2,3.746893E-2,7.197648E-2,0E0,0E0,6.809737E-2,0E0,0E0,0E0,0E0,4.5581624E-2,0E0,5.323705E-2,0E0,5.8892682E-2,0E0,0E0,0E0,4.0650055E-2,6.329694E-2,0E0,5.9553668E-2,0E0,4.5227617E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,15,15,17,17,19,19,23,23,24,24,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,-1,-1,-1,-1,20,-1,22,-1,24,-1,-1,-1,26,28,-1,30,-1,32,-1,-1,-1,-1],"split_conditions":[6.942396E3,9.7467445E4,3.0418E4,7.339209E6,1.2662238E1,4.542969E1,4.52E2,3.305542E4,-7.3420736E-3,6.402053E-3,1.0361716E7,1.4198481E-2,4.9593076E-3,3.080649E-3,-2.2378895E-3,4.13E2,8.100135E-3,1.0220021E3,-2.3281675E-4,7.683712E4,1.1705592E-3,-1.6683136E-4,2.7362285E-3,2.7473627E1,1.7377544E7,-1.3827949E-3,5.6961117E1,2.7842136E-4,6.927258E4,-4.73967E-3,-4.6195305E-4,-2.3648087E-3,1.8456845E-3],"split_indices":[61,42,10,56,65,63,3,4,0,0,54,0,0,0,0,8,0,61,0,37,0,0,0,67,54,0,67,0,42,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.22E2,3.93E2,2.9E1,2.45E2,1.48E2,1E1,1.9E1,2.41E2,4E0,3E0,1.45E2,4E0,6E0,1.6E1,3E0,2.4E2,1E0,6.8E1,7.7E1,2.22E2,1.8E1,2.1E1,4.7E1,9.3E1,1.29E2,5.7E1,3.6E1,7.5E1,5.4E1,2.5E1,1.1E1,4.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.2990474E-3,-2.7667282E-3,1.9134265E-1,-2.0932969E-2,1.5071351E-2,2.2927593E-1,-2.70307E-3,-2.9313713E-2,3.3900456E-4,4.369016E-3,1.1294423E-2,1.2467863E-2,5.814747E-4,-1.8867478E-3,-2.3187236E-3,-2.2120198E-2,2.1162732E-2,1.1812474E-3,-2.0451585E-3,-3.6821156E-4,-5.004425E-3,-3.062725E-3,1.1738332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,-1,-1,-1,17,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[3.320068E-1,1.367409E-1,9.731737E-2,4.918705E-2,5.9749216E-2,4.7810942E-2,0E0,4.257804E-2,0E0,0E0,6.773627E-2,0E0,0E0,0E0,4.8354145E-2,5.4803252E-2,4.4129916E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,-1,-1,-1,18,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[3.305542E4,6.97E3,2.6279484E5,9.704588E6,8.547769E7,8.494E3,-2.70307E-3,2.103E3,3.3900456E-4,4.369016E-3,1.1016051E3,1.2467863E-2,5.814747E-4,-1.8867478E-3,1.4445602E10,1E0,1.293844E0,1.1812474E-3,-2.0451585E-3,-3.6821156E-4,-5.004425E-3,-3.062725E-3,1.1738332E-3],"split_indices":[4,2,42,9,7,0,0,0,0,0,61,0,0,0,5,18,62,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,4.2E2,8E0,2.08E2,2.12E2,7E0,1E0,1.6E2,4.8E1,9E0,2.03E2,6E0,1E0,1.17E2,4.3E1,4.6E1,1.57E2,2.6E1,1.7E1,4E1,6E0,5E0,1.52E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.8819119E-3,-6.966758E-3,5.8297876E-2,-5.4188077E-3,-9.37927E-3,1.3114409E-1,3.4339104E-2,1.606882E-2,-1.8683443E-2,4.401581E-3,1.5652351E-2,-1.0280605E-2,4.481136E-2,6.705042E-3,2.507284E-3,-1.9051544E-3,-1.0134774E-2,8.106993E-3,1.6844368E-3,8.530201E-5,4.5531876E-3,2.6425773E-3,-6.888113E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,17,19,-1,-1,21,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1356723E-1,1.06369115E-1,9.672439E-2,1.0518962E-1,0E0,7.832472E-2,1.262174E-1,4.7092546E-2,4.019831E-2,0E0,0E0,0E0,4.771787E-2,4.87637E-2,0E0,0E0,4.4725627E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,12,12,13,13,16,16],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,18,20,-1,-1,22,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0292856E3,3.764913E7,1.417988E6,1.09E2,-9.37927E-3,1.9840434E7,1.461379E6,1.889447E5,9.50639E5,4.401581E-3,1.5652351E-2,-1.0280605E-2,8.169258E3,1.5775E2,2.507284E-3,-1.9051544E-3,1.87904E5,8.106993E-3,1.6844368E-3,8.530201E-5,4.5531876E-3,2.6425773E-3,-6.888113E-4],"split_indices":[61,57,38,3,0,9,38,37,1,0,0,0,42,63,0,0,38,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,3.69E2,5.7E1,3.67E2,2E0,1.3E1,4.4E1,1.4E2,2.27E2,1.2E1,1E0,1E0,4.3E1,1.12E2,2.8E1,6.5E1,1.62E2,2E0,4.1E1,1.07E2,5E0,9E0,1.53E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.107303E-4,-1.0894414E-2,4.039315E-2,-9.268121E-3,-7.2159106E-3,2.2232188E-2,1.0129497E-1,-1.9881973E-2,1.9207528E-2,3.1715393E-2,-7.540699E-3,1.5275004E-2,7.594672E-2,-9.671431E-3,-2.196603E-3,7.403636E-3,6.3441053E-4,4.1313626E-2,-2.479959E-3,-7.6014694E-4,4.747023E-3,-3.2766253E-2,7.414113E-3,-2.5822043E-3,4.8297975E-2,-3.0830692E-4,-5.8974255E-2,1.0915254E-2,-5.6137876E-3,1.0497506E-2,1.8814802E-3,-4.6289302E-3,-2.5996385E-2,-9.124705E-4,2.3846248E-2,-2.9105195E-3,1.0261084E-3,1.4815815E-5,2.4511907E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1,25,27,-1,29,-1,31,33,-1,-1,-1,-1,35,-1,37,-1,-1,-1,-1],"loss_changes":[1.9990557E-1,7.782334E-2,1.0503344E-1,1.0556013E-1,0E0,1.3370053E-1,9.612098E-2,6.614863E-2,7.5191736E-2,6.0385622E-2,0E0,0E0,4.4899136E-2,7.245246E-2,0E0,0E0,0E0,4.6317913E-2,0E0,0E0,0E0,5.3533085E-2,4.723208E-2,0E0,8.894007E-2,0E0,4.512979E-2,4.0580712E-2,0E0,0E0,0E0,0E0,3.703118E-2,0E0,4.636515E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,17,17,21,21,22,22,24,24,26,26,27,27,32,32,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1,26,28,-1,30,-1,32,34,-1,-1,-1,-1,36,-1,38,-1,-1,-1,-1],"split_conditions":[1.6219076E9,4.242889E3,2.5139713E-1,1.4720299E3,-7.2159106E-3,3.768034E4,1.6441814E9,1.902843E-2,7.532E3,1.5953E4,-7.540699E-3,1.5275004E-2,4.7746178E1,2.0011E5,-2.196603E-3,7.403636E-3,6.3441053E-4,2.2782404E0,-2.479959E-3,-7.6014694E-4,4.747023E-3,7.072893E-5,4.086885E7,-2.5822043E-3,1E0,-3.0830692E-4,1.8243062E-3,1.2681007E3,-5.6137876E-3,1.0497506E-2,1.8814802E-3,-4.6289302E-3,2.0629E8,-9.124705E-4,7.805608E4,-2.9105195E-3,1.0261084E-3,1.4815815E-5,2.4511907E-3],"split_indices":[7,67,47,61,0,4,7,66,9,0,0,0,65,9,0,0,0,63,0,0,0,47,54,0,8,0,48,64,0,0,0,0,5,0,37,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,3.5E2,9.6E1,3.47E2,3E0,7.5E1,2.1E1,2.53E2,9.4E1,7.2E1,3E0,1E0,2E1,1.82E2,7.1E1,3E0,9.1E1,6.5E1,7E0,4E0,1.6E1,7.7E1,1.05E2,4E0,6.1E1,3.9E1,3.8E1,1.03E2,2E0,2E0,5.9E1,1.7E1,2.1E1,3.1E1,7.2E1,1.2E1,9E0,3.9E1,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.0647596E-3,-1.636384E-2,2.721623E-2,-1.810259E-2,7.5780908E-3,1.4326885E-2,6.2474255E-2,-1.6766632E-2,-9.744304E-3,-1.1195122E-3,2.8734867E-2,3.988749E-3,-5.114069E-4,-4.1442808E-2,-8.838909E-3,7.599463E-4,7.970193E-2,-2.313507E-3,1.7506233E-3,2.611241E-3,-1.2413621E-2,2.2197387E-3,1.4906755E-2,-2.242476E-2,6.469856E-4,2.739539E-2,-3.3493627E-2,-2.8047175E-4,4.732718E-3,-6.578786E-3,-1.169243E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,15,-1,-1,17,19,-1,21,-1,-1,-1,23,-1,-1,25,-1,27,29,-1,-1,-1,-1],"loss_changes":[1.947156E-1,9.095084E-2,7.090848E-2,6.7896165E-2,0E0,6.395909E-2,6.304054E-2,5.616959E-2,0E0,0E0,5.494781E-2,0E0,0E0,3.733328E-2,5.0214775E-2,0E0,1.1708847E-1,0E0,0E0,0E0,5.4350186E-2,0E0,0E0,8.418692E-2,0E0,6.637529E-2,1.14692345E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,10,10,13,13,14,14,16,16,20,20,23,23,25,25,26,26],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,16,-1,-1,18,20,-1,22,-1,-1,-1,24,-1,-1,26,-1,28,30,-1,-1,-1,-1],"split_conditions":[1.05381016E5,8.861901E3,1.6768953E5,1.3180642E7,7.5780908E-3,4.359929E1,2.84E2,1.1802526E3,-9.744304E-3,-1.1195122E-3,2.0901188E7,3.988749E-3,-5.114069E-4,1.2833E4,1.3812E4,7.599463E-4,2.9632012E7,-2.313507E-3,1.7506233E-3,2.611241E-3,1.2028214E5,2.2197387E-3,1.4906755E-2,5.190585E2,6.469856E-4,5.855E3,2.9648642E1,-2.8047175E-4,4.732718E-3,-6.578786E-3,-1.169243E-3],"split_indices":[42,61,37,56,0,65,8,64,0,0,9,0,0,10,9,0,1,0,0,0,37,0,0,61,0,2,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.48E2,2.91E2,1.57E2,2.89E2,2E0,1.16E2,4.1E1,2.88E2,1E0,3.2E1,8.4E1,3.2E1,9E0,6.9E1,2.19E2,6.8E1,1.6E1,6.4E1,5E0,1.1E1,2.08E2,1.5E1,1E0,1.5E2,5.8E1,2.7E1,1.23E2,1.9E1,8E0,9E0,1.14E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.8383546E-3,-1.0857119E-3,5.198356E-3,-3.1945404E-2,5.2927444E-3,-4.1162275E-2,2.0963566E-3,-2.0036073E-3,4.0785603E-2,-2.4872466E-3,1.7185763E-3,2.1866014E-2,-1.0141778E-2,8.8994345E-3,1.4165489E-3,4.823109E-3,1.4428761E-2,-2.2655336E-2,2.0385245E-2,3.0973859E-3,9.988595E-4,-1.9141495E-2,-6.9608456E-3,2.7129626E-2,-3.9780354E-3,-1.2331227E-3,1.6477588E-3,-5.5240904E-4,-7.2723486E-2,7.974773E-3,3.7084161E-3,-5.221053E-3,7.9299224E-4,2.3061975E-3,-7.2732806E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,-1,15,17,-1,-1,-1,19,21,23,-1,25,27,-1,29,-1,-1,-1,-1,31,33,-1,-1,-1,-1,-1],"loss_changes":[1.714807E-1,7.9297625E-2,0E0,4.9756587E-2,8.657717E-2,5.1473983E-2,0E0,5.4224662E-2,8.806326E-2,0E0,0E0,3.983155E-2,7.991818E-2,0E0,0E0,0E0,4.4087112E-2,6.053315E-2,4.4567958E-2,0E0,4.75992E-2,5.887076E-2,0E0,5.4694705E-2,0E0,0E0,0E0,0E0,5.8859147E-2,4.0241066E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,11,11,12,12,16,16,17,17,18,18,20,20,21,21,23,23,28,28,29,29],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,-1,16,18,-1,-1,-1,20,22,24,-1,26,28,-1,30,-1,-1,-1,-1,32,34,-1,-1,-1,-1,-1],"split_conditions":[6.0538214E9,1.2595157E3,5.198356E-3,2.889654E5,1.5271514E5,3.312442E6,2.0963566E-3,6.088569E6,8.534E3,-2.4872466E-3,1.7185763E-3,2.7E1,3.17008E6,8.8994345E-3,1.4165489E-3,4.823109E-3,1.136E1,1.92144E5,1.505649E6,3.0973859E-3,9.327769E2,1E0,-6.9608456E-3,3.638653E6,-3.9780354E-3,-1.2331227E-3,1.6477588E-3,-5.5240904E-4,3.5432583E3,2.649945E3,3.7084161E-3,-5.221053E-3,7.9299224E-4,2.3061975E-3,-7.2732806E-4],"split_indices":[7,4,0,37,37,9,0,54,9,0,0,8,38,0,0,0,67,2,2,0,61,18,0,56,0,0,0,0,4,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.15E2,4.01E2,1.4E1,6.8E1,3.33E2,6.1E1,7E0,2.77E2,5.6E1,5.4E1,7E0,7E1,2.07E2,3E0,5.3E1,5E0,6.5E1,1.47E2,6E1,1.3E1,5.2E1,1.44E2,3E0,5.7E1,3E0,2.9E1,2.3E1,1.27E2,1.7E1,4.2E1,1.5E1,1.2E1,5E0,1.5E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-9.5425453E-4,-3.5566702E-3,6.5316437E-3,-6.5014376E-3,3.0090297E-3,-4.2752847E-3,-7.849513E-2,-8.262944E-3,4.3984983E-2,7.003415E-3,-1.1204034E-1,-1.5455346E-3,-1.9339649E-3,9.173146E-3,9.974334E-4,-1.0952357E-2,-1.9517471E-3,1.6654918E-2,-1.0930981E-2,3.0209308E-3,2.3799133E-4,-9.435323E-4,6.28422E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,-1,17,-1,-1,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[1.5172186E-1,8.1796E-2,0E0,6.409253E-2,0E0,7.560965E-2,1.0634134E-1,5.4446496E-2,9.829652E-2,0E0,8.141789E-2,0E0,4.818122E-2,0E0,0E0,0E0,0E0,5.036279E-2,4.0413655E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,12,12,17,17,18,18],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,-1,18,-1,-1,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[1.0871896E10,7.0484486E0,6.5316437E-3,1.4609149E4,3.0090297E-3,5.8145845E3,1.3812E4,1.3063165E3,3.7137297E3,7.003415E-3,9.5430945E4,-1.5455346E-3,7.477264E6,9.173146E-3,9.974334E-4,-1.0952357E-2,-1.9517471E-3,6.2E1,1.2128688E5,3.0209308E-3,2.3799133E-4,-9.435323E-4,6.28422E-4],"split_indices":[7,44,0,4,0,4,9,4,41,0,37,0,54,0,0,0,0,3,37,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,4.18E2,7E0,4.01E2,1.7E1,3.9E2,1.1E1,3.61E2,2.9E1,1E0,1E1,7.5E1,2.86E2,3E0,2.6E1,3E0,7E0,9.3E1,1.93E2,1.8E1,7.5E1,1.42E2,5.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.8844878E-3,-4.181966E-3,5.7139277E-2,-2.7302764E-3,-9.187052E-3,7.088285E-2,-5.5081956E-2,-7.4883327E-3,3.0340528E-2,1.4071889E-1,2.1537163E-3,-6.504209E-3,1.8565117E-3,-1.3943094E-2,1.8535757E-2,1.8777588E-3,-2.5634444E-3,5.03569E-3,1.476225E-2,-1.8949455E-2,7.9946563E-4,4.126952E-3,3.3056518E-4,-8.277964E-4,-5.6656483E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,-1],"loss_changes":[1.9268957E-1,1.04987934E-1,9.490128E-2,6.1280068E-2,0E0,9.0743065E-2,5.748151E-2,5.7371408E-2,3.7277594E-2,5.2055746E-2,0E0,0E0,0E0,4.203251E-2,5.267203E-2,0E0,0E0,0E0,0E0,3.923013E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,14,14,19,19],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,-1],"split_conditions":[4.23168E3,1.1770536E8,1.93E2,3.73E2,-9.187052E-3,3.0418E4,5.743529E6,1.646359E5,2.165E3,1.62E2,2.1537163E-3,-6.504209E-3,1.8565117E-3,3.774E3,2.8E1,1.8777588E-3,-2.5634444E-3,5.03569E-3,1.476225E-2,3.454E3,7.9946563E-4,4.126952E-3,3.3056518E-4,-8.277964E-4,-5.6656483E-3],"split_indices":[61,1,8,8,0,10,38,37,0,8,0,0,0,0,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.47E2,3.89E2,5.8E1,3.87E2,2E0,5.2E1,6E0,3.39E2,4.8E1,1.3E1,3.9E1,3E0,3E0,2.72E2,6.7E1,4.4E1,4E0,1.2E1,1E0,2.34E2,3.8E1,9E0,5.8E1,2.31E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[4.6564364E-3,-3.0664438E-3,3.9399333E-2,-1.174872E-2,2.608834E-2,1.4623775E-3,6.9763996E-3,-4.8012525E-4,-8.235505E-3,3.290568E-2,-5.5485037E-3,2.355554E-2,1.2628907E-2,1.07226186E-1,6.5846904E-4,7.08569E-4,7.705159E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,-1,13,-1,15,-1,-1,-1],"loss_changes":[1.1656964E-1,9.0420745E-2,7.1052454E-2,7.663399E-2,8.27007E-2,0E0,0E0,0E0,0E0,1.6299716E-1,0E0,6.2310483E-2,0E0,3.6572903E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,11,11,13,13],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,-1,14,-1,16,-1,-1,-1],"split_conditions":[1.7670108E5,1.484798E9,9.70625E1,3.764913E7,3.7826266E4,1.4623775E-3,6.9763996E-3,-4.8012525E-4,-8.235505E-3,2.3767492E4,-5.5485037E-3,1.3812E4,1.2628907E-2,1.4166278E0,6.5846904E-4,7.08569E-4,7.705159E-3],"split_indices":[37,7,65,57,61,0,0,0,0,61,0,9,0,44,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.55E2,7.8E1,2.74E2,8.1E1,7.3E1,5E0,2.72E2,2E0,7.8E1,3E0,7.6E1,2E0,7E0,6.9E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.5729057E-3,-4.5989202E-3,2.834E-3,-3.6119344E-3,-8.9147305E-3,-1.0035433E-2,1.4026632E-3,-3.97121E-3,-7.349969E-3,-5.0880974E-3,-4.8842034E-3,-9.229334E-3,3.307249E-2,-1.2941261E-2,1.5683454E-3,2.3283136E-4,4.1587367E-3,-5.6801527E-4,-7.1390467E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,-1,13,15,17,-1,-1,-1,-1,-1],"loss_changes":[1.45023E-1,6.535036E-2,0E0,7.801751E-2,0E0,5.960694E-2,0E0,0E0,6.323446E-2,4.6766646E-2,0E0,4.165054E-2,4.314116E-2,3.667756E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,11,11,12,12,13,13],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,-1,14,16,18,-1,-1,-1,-1,-1],"split_conditions":[2.785032E9,1.3180642E7,2.834E-3,3.774E3,-8.9147305E-3,9.306416E0,1.4026632E-3,-3.97121E-3,2.9632012E7,1.8460338E5,-4.8842034E-3,4.8384636E7,1.556E3,3.498E3,1.5683454E-3,2.3283136E-4,4.1587367E-3,-5.6801527E-4,-7.1390467E-3],"split_indices":[7,56,0,0,0,65,0,0,1,37,0,9,61,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.09E2,3.7E2,3.9E1,3.69E2,1E0,3.09E2,6E1,1E1,2.99E2,2.93E2,6E0,2.65E2,2.8E1,2.44E2,2.1E1,1.9E1,9E0,2.43E2,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.8376033E-3,-4.1071884E-3,6.521128E-3,-1.563402E-2,9.137336E-3,-1.755075E-2,6.397921E-3,5.481504E-4,-7.881924E-2,-7.484976E-3,-3.7448287E-2,-7.446629E-3,1.1127252E-3,-1.5196647E-3,5.213986E-3,-5.4353722E-3,-1.2910734E-3,2.5968255E-2,-8.37917E-4,7.987098E-4,5.3540524E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,-1,-1,-1,17,-1,-1,19,-1,-1,-1],"loss_changes":[1.3371846E-1,6.45685E-2,0E0,6.6581346E-2,3.9559215E-2,4.458335E-2,0E0,0E0,4.2093676E-2,4.5943324E-2,5.723326E-2,0E0,0E0,0E0,4.6871312E-2,0E0,0E0,3.97584E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,14,14,17,17],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,-1,-1,-1,18,-1,-1,20,-1,-1,-1],"split_conditions":[1.0871896E10,7.222818E4,6.521128E-3,6.0495757E3,2.0531621E4,1.9982805E-2,6.397921E-3,5.481504E-4,1E0,1.17032E5,2.7276272E-2,-7.446629E-3,1.1127252E-3,-1.5196647E-3,1.7239808E7,-5.4353722E-3,-1.2910734E-3,5.146E3,-8.37917E-4,7.987098E-4,5.3540524E-3],"split_indices":[7,42,0,61,4,48,0,0,22,9,51,0,0,0,54,0,0,10,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.27E2,4.21E2,6E0,2.25E2,1.96E2,2.23E2,2E0,1.92E2,4E0,1.49E2,7.4E1,2E0,2E0,5.1E1,9.8E1,8E0,6.6E1,5.1E1,4.7E1,4.7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.5752533E-3,-6.871921E-3,2.7054312E-2,-1.8540328E-2,1.2292036E-2,3.124557E-2,-7.887404E-3,-1.2746057E-3,8.980597E-4,2.0742684E-3,2.237338E-3,2.5478525E-2,5.108446E-3,2.9688329E-4,-5.576947E-3,7.145241E-2,1.7354636E-2,6.239901E-3,1.0993979E-3,-7.7166397E-4,1.617874E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,-1,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[1.0516995E-1,6.664091E-2,1.1058185E-1,5.4776587E-2,3.9287932E-2,5.330719E-2,0E0,0E0,0E0,4.3017846E-2,0E0,4.4325344E-2,0E0,0E0,0E0,4.780852E-2,5.728111E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,15,15,16,16],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,-1,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[5.4779566E9,6.4065235E8,5.374925E7,1.46705E3,4.101113E1,1.6828056E1,-7.887404E-3,-1.2746057E-3,8.980597E-4,2.47815E5,2.237338E-3,2.8E1,5.108446E-3,2.9688329E-4,-5.576947E-3,4.439757E0,6.45254E0,6.239901E-3,1.0993979E-3,-7.7166397E-4,1.617874E-3],"split_indices":[5,7,54,61,62,44,0,0,0,2,0,3,0,0,0,63,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.27E2,2.96E2,1.31E2,1.84E2,1.12E2,1.29E2,2E0,1.52E2,3.2E1,8.7E1,2.5E1,1.21E2,8E0,8.5E1,2E0,1.7E1,1.04E2,7E0,1E1,3.4E1,7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[4.4479186E-4,-5.9184977E-3,5.1082306E-2,-4.410277E-3,-5.842295E-3,1.2419796E-1,2.4012418E-2,-9.482359E-3,1.1237237E-3,4.0994654E-3,1.432728E-2,-8.715153E-3,3.472124E-2,3.861176E-4,-1.0695717E-3,1.0215516E-1,8.596554E-4,-2.207354E-2,7.023988E-4,-7.101753E-4,7.321812E-3,-4.8216432E-3,-1.3474052E-2,-1.0261569E-3,3.2862937E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,9,11,13,-1,-1,-1,-1,15,17,-1,19,-1,21,-1,-1,-1,-1,23,-1,-1],"loss_changes":[1.414352E-1,6.72569E-2,9.40001E-2,5.4484714E-2,0E0,6.281461E-2,8.740927E-2,4.0993296E-2,0E0,0E0,0E0,0E0,3.918591E-2,5.9246883E-2,0E0,4.2900294E-2,0E0,4.6902437E-2,0E0,0E0,0E0,0E0,4.3520384E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,12,12,13,13,15,15,17,17,22,22],"right_children":[2,4,6,8,-1,10,12,14,-1,-1,-1,-1,16,18,-1,20,-1,22,-1,-1,-1,-1,24,-1,-1],"split_conditions":[4.23168E3,1.3674345E4,1.417988E6,1.6219076E9,-5.842295E-3,1.62E2,1.461379E6,7.287871E-3,1.1237237E-3,4.0994654E-3,1.432728E-2,-8.715153E-3,1.3812E4,4.9956406E2,-1.0695717E-3,2.043E3,8.596554E-4,5.074412E0,7.023988E-4,-7.101753E-4,7.321812E-3,-4.8216432E-3,1.799E3,-1.0261569E-3,3.2862937E-3],"split_indices":[61,4,38,7,0,8,38,66,0,0,0,0,9,42,0,0,0,62,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.89E2,4.8E1,3.85E2,4E0,1.2E1,3.6E1,3.26E2,5.9E1,1.1E1,1E0,1E0,3.5E1,1.84E2,1.42E2,6E0,2.9E1,7.1E1,1.13E2,2E0,4E0,6E0,6.5E1,6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.2563269E-3,-4.697219E-3,4.5007765E-2,6.5785395E-3,-6.551613E-4,6.2552514E-3,3.297807E-2,3.046369E-3,-7.32546E-4,1.14094975E-4,3.195034E-3,-2.5563694E-3,7.3795905E-3,1.4934578E-2,-1.9305266E-3,2.402996E-3,2.4454135E-3,-3.1084274E-3,9.319519E-3,3.363544E-3,1.4183009E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,-1,-1,9,-1,11,-1,-1,-1,13,15,-1,17,-1,-1,19,-1,-1],"loss_changes":[1.109138E-1,3.768355E-2,4.77025E-2,6.874229E-2,0E0,0E0,4.6716034E-2,0E0,6.316642E-2,0E0,0E0,0E0,4.681201E-2,5.010144E-2,0E0,3.9452583E-2,0E0,0E0,4.1982785E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,12,12,13,13,15,15,18,18],"right_children":[2,4,6,8,-1,-1,10,-1,12,-1,-1,-1,14,16,-1,18,-1,-1,20,-1,-1],"split_conditions":[7.2038723E9,1.26E2,1.3812E4,1.3436554E3,-6.551613E-4,6.2552514E-3,4.09591E0,3.046369E-3,4.7749156E5,1.14094975E-4,3.195034E-3,-2.5563694E-3,2.2931854E7,1E2,-1.9305266E-3,4.18E3,2.4454135E-3,-3.1084274E-3,5.3642064E-2,3.363544E-3,1.4183009E-5],"split_indices":[12,3,9,56,0,0,62,0,60,0,0,0,54,3,0,2,0,0,47,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,3.74E2,5E1,1.65E2,2.09E2,5E0,4.5E1,1.8E1,1.47E2,2.4E1,2.1E1,1.9E1,1.28E2,1.11E2,1.7E1,8.3E1,2.8E1,7E0,7.6E1,9E0,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[7.9031393E-4,-8.231624E-3,2.9794289E-2,-5.545266E-4,2.5872348E-2,3.6419958E-2,-5.1122233E-3,2.2543808E-3,-1.6775082E-3,2.8414672E-2,1.3522112E-2,1.5697355E-3,2.1453903E-3,4.798238E-3,-4.821342E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,7,9,-1,-1,-1,11,-1,13,-1,-1,-1],"loss_changes":[1.12502575E-1,3.6868043E-2,9.565253E-2,0E0,3.8366266E-2,1.8347639E-1,0E0,0E0,0E0,4.137221E-2,0E0,4.2921934E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,9,9,11,11],"right_children":[2,4,6,-1,8,10,-1,-1,-1,12,-1,14,-1,-1,-1],"split_conditions":[2.1959E4,4.61E2,3.768034E4,-5.545266E-4,1.8E2,3.305542E4,-5.1122233E-3,2.2543808E-3,-1.6775082E-3,3.8952081E0,1.3522112E-2,1.2662238E1,2.1453903E-3,4.798238E-3,-4.821342E-4],"split_indices":[2,3,4,0,8,4,0,0,0,62,0,65,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,3.27E2,1.01E2,2.99E2,2.8E1,9.7E1,4E0,2.1E1,7E0,9.5E1,2E0,3.6E1,5.9E1,3E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.2487056E-3,6.3563953E-4,9.827756E-2,2.2281997E-3,-5.170363E-3,9.291656E-3,2.0493332E-3,-1.7978303E-4,1.4886481E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[2.2464289E-1,7.084387E-2,1.2348087E-1,6.939661E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[8.10288E3,1.1770536E8,1.681178E6,1.6441814E9,-5.170363E-3,9.291656E-3,2.0493332E-3,-1.7978303E-4,1.4886481E-3],"split_indices":[61,1,38,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,4.1E2,2.4E1,4.05E2,5E0,8E0,1.6E1,3.36E2,6.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.1863469E-3,-2.880961E-3,5.72632E-3,1.1469375E-2,-1.5150764E-2,3.0291625E-3,4.278605E-3,-2.9315531E-2,-3.3389654E-3,4.525503E-2,-5.3419676E-3,-1.0541637E-3,-4.3554585E-3,1.2676406E-4,-7.014265E-3,9.2605967E-4,7.67946E-3,1.1027713E-2,-2.210304E-2,-4.316684E-2,4.2755576E-4,-2.1608188E-3,1.0008423E-3,-6.8155095E-2,-6.045682E-3,-9.550754E-4,-6.6291364E-3,-5.2836817E-3,-7.015635E-4,-9.265666E-4,2.5319532E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,-1,5,7,-1,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1],"loss_changes":[8.7544866E-2,7.482991E-2,0E0,7.211128E-2,3.821642E-2,0E0,6.844124E-2,4.5314446E-2,6.208716E-2,9.495031E-2,3.8930394E-2,0E0,0E0,4.730673E-2,0E0,0E0,0E0,4.005093E-2,5.1351167E-2,4.3544143E-2,0E0,0E0,0E0,3.8379334E-2,4.1887723E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,17,17,18,18,19,19,23,23,24,24],"right_children":[2,4,-1,6,8,-1,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0871896E10,1.33E2,5.72632E-3,9.1417163E2,2.72E2,3.0291625E-3,2.1207E4,2.53E2,2.5212732E4,1.32737E5,9.565983E6,-1.0541637E-3,-4.3554585E-3,3.723868E2,-7.014265E-3,9.2605967E-4,7.67946E-3,1.0321884E3,2.0231652E3,4.05E2,4.2755576E-4,-2.1608188E-3,1.0008423E-3,2.9673992E9,9.355285E8,-9.550754E-4,-6.6291364E-3,-5.2836817E-3,-7.015635E-4,-9.265666E-4,2.5319532E-3],"split_indices":[7,3,0,56,3,0,9,3,4,2,54,0,0,42,0,0,0,4,64,3,0,0,0,5,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,4.23E2,5E0,1.95E2,2.28E2,2.3E1,1.72E2,1.03E2,1.25E2,3.2E1,1.4E2,9.3E1,1E1,1.23E2,2E0,2.7E1,5E0,7.1E1,6.9E1,2E1,1.03E2,1E1,6.1E1,1.7E1,5.2E1,1.7E1,3E0,9E0,8E0,4.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.6787375E-3,-1.7386043E-3,5.7710152E-2,-2.229923E-3,1.2735327E-3,7.7279382E-3,-3.591001E-3,-4.178769E-3,2.4293492E-2,-8.420911E-3,9.381018E-4,-4.2598415E-3,-2.4526613E-3,6.2464834E-3,8.5556926E-4,1.1890418E-2,-1.1169901E-2,2.590114E-3,2.7616457E-3,2.2211073E-3,-1.4724059E-2,5.9190305E-4,-2.5860704E-3,-6.3407742E-3,-1.19061675E-2,-8.8161865E-4,1.0988626E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,-1,7,-1,9,11,13,-1,-1,-1,15,-1,-1,17,19,21,-1,-1,23,-1,-1,-1,25,-1,-1],"loss_changes":[7.994686E-2,5.2882664E-2,1.5147585E-1,0E0,4.6540864E-2,0E0,6.647713E-2,4.3369517E-2,4.664394E-2,0E0,0E0,0E0,3.7017275E-2,0E0,0E0,4.7127478E-2,3.8108446E-2,5.185056E-2,0E0,0E0,5.56949E-2,0E0,0E0,0E0,3.826196E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,12,12,15,15,16,16,17,17,20,20,24,24],"right_children":[2,4,6,-1,8,-1,10,12,14,-1,-1,-1,16,-1,-1,18,20,22,-1,-1,24,-1,-1,-1,26,-1,-1],"split_conditions":[8.861901E3,3.9541063E0,1.417988E6,-2.229923E-3,2.1959E4,7.7279382E-3,1.379E3,2.9E1,1.8117047E4,-8.420911E-3,9.381018E-4,-4.2598415E-3,1.26E2,6.2464834E-3,8.5556926E-4,2.9049585E3,5.8601086E1,9.304E3,2.7616457E-3,2.2211073E-3,6.52487E2,5.9190305E-4,-2.5860704E-3,-6.3407742E-3,4.61E2,-8.8161865E-4,1.0988626E-3],"split_indices":[61,63,38,0,2,0,0,3,42,0,0,0,3,0,0,4,59,2,0,0,4,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.16E2,3.93E2,2.3E1,2.4E1,3.69E2,8E0,1.5E1,2.99E2,7E1,1E0,1.4E1,5E0,2.94E2,3E0,6.7E1,1.11E2,1.83E2,9.3E1,1.8E1,1E1,1.73E2,8E1,1.3E1,3E0,1.7E2,1.44E2,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.5647286E-3,1.0947653E-3,5.7344735E-2,2.5765046E-3,-4.8022578E-3,1.3178156E-1,5.8065127E-3,-2.0098465E-3,5.4588635E-3,8.557968E-3,3.5747775E-4,-8.2179E-3,1.3247018E-3,3.9259915E-4,3.1186039E-2,3.0647952E-4,-9.120734E-4,1.7489619E-3,-3.985292E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[8.01546E-2,6.1727814E-2,1.02816015E-1,5.215079E-2,0E0,6.1218664E-2,6.9603585E-2,0E0,4.9908236E-2,0E0,0E0,0E0,0E0,3.708506E-2,3.948569E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,13,13,14,14],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[8.10288E3,1.0067637E7,4.0601485E6,9.63428E2,-4.8022578E-3,2.4231377E5,1.7223539E9,-2.0098465E-3,9.779828E9,8.557968E-3,3.5747775E-4,-8.2179E-3,1.3247018E-3,6.3973755E-2,1.929271E6,3.0647952E-4,-9.120734E-4,1.7489619E-3,-3.985292E-3],"split_indices":[61,56,56,4,0,42,7,0,5,0,0,0,0,66,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,4.11E2,2.6E1,4.06E2,5E0,1E1,1.6E1,2.4E1,3.82E2,7E0,3E0,1E0,1.5E1,3.2E2,6.2E1,2.45E2,7.5E1,6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.3603803E-3,-3.9014255E-4,1.6834868E-2,3.214626E-3,1.2120603E-2,3.369909E-4,2.7395936E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":84,"left_children":[1,-1,3,-1,5,-1,-1],"loss_changes":[6.6971615E-2,0E0,5.4168552E-2,0E0,4.891672E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4],"right_children":[2,-1,4,-1,6,-1,-1],"split_conditions":[6.97E3,-3.9014255E-4,6.798766E-4,3.214626E-3,4.966055E9,3.369909E-4,2.7395936E-3],"split_indices":[2,0,48,0,7,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.32E2,1.99E2,2.33E2,1.9E1,2.14E2,1.93E2,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[8.083121E-4,-9.473695E-4,4.6899505E-3,3.5173517E-2,-4.5480877E-3,1.9668102E-4,5.1777656E-3,-6.1028963E-4,7.2480803E-3,5.8905716E-4,-4.2925004E-2,-4.1045906E-4,-6.5298793E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,-1,5,7,-1,-1,-1,9,-1,11,-1,-1],"loss_changes":[7.579409E-2,5.7386875E-2,0E0,8.937925E-2,3.8451068E-2,0E0,0E0,0E0,4.141117E-2,0E0,4.6596393E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,8,8,10,10],"right_children":[2,4,-1,6,8,-1,-1,-1,10,-1,12,-1,-1],"split_conditions":[1.0871896E10,1.3812E4,4.6899505E-3,4.3018994E3,6.211238E5,1.9668102E-4,5.1777656E-3,-6.1028963E-4,2.62797E5,5.8905716E-4,1.4266666E1,-4.1045906E-4,-6.5298793E-3],"split_indices":[7,9,0,4,56,0,0,0,2,0,63,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.46E2,4.39E2,7E0,3.9E1,4E2,2.8E1,1.1E1,2.37E2,1.63E2,1.49E2,1.4E1,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.3969525E-3,-9.9818884E-5,2.7193879E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":86,"left_children":[1,-1,-1],"loss_changes":[8.0404535E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[8.10288E3,-9.9818884E-5,2.7193879E-3],"split_indices":[61,0,0],"split_type":[0,0,0],"sum_hessian":[4.2E2,3.96E2,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[1.549403E-4,-2.5319469E-3,1.17019035E-1,-1.216721E-2,1.3941526E-2,1.25890765E-2,2.3318212E-3,-3.9508124E-3,-1.744118E-3,5.293718E-3,5.2073994E-4,-3.4184735E-3,-5.9766713E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[1.3971257E-1,6.921555E-2,8.629298E-2,5.4098558E-2,4.729142E-2,0E0,0E0,3.7303984E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[3.305542E4,1.4171124E3,3.768034E4,1.8071064E7,8.534E3,1.25890765E-2,2.3318212E-3,2.317E3,-1.744118E-3,5.293718E-3,5.2073994E-4,-3.4184735E-3,-5.9766713E-5],"split_indices":[4,61,4,54,9,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.43E2,4.34E2,9E0,2.74E2,1.6E2,2E0,7E0,2.05E2,6.9E1,4E0,1.56E2,7E0,1.98E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[5.4313074E-4,-1.6637576E-3,3.7837243E-3,-3.6641129E-3,-2.3377089E-4,-6.2292144E-3,2.2251798E-2,-4.454214E-3,-6.9042645E-2,3.3872875E-3,6.265651E-4,-1.7118491E-2,2.715326E-4,5.2722194E-3,-5.0631287E-3,-3.7957726E-3,-1.980575E-3,-3.972686E-5,-6.6828392E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,-1,5,7,9,11,13,-1,-1,15,-1,-1,-1,17,-1,-1,-1],"loss_changes":[7.494449E-2,4.5049634E-2,0E0,0E0,5.635478E-2,3.6485437E-2,3.81884E-2,4.1225307E-2,6.922706E-2,0E0,0E0,4.548899E-2,0E0,0E0,0E0,3.714374E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,8,8,11,11,15,15],"right_children":[2,4,-1,-1,6,8,10,12,14,-1,-1,16,-1,-1,-1,18,-1,-1,-1],"split_conditions":[1.6941456E1,2.1066585E2,3.7837243E-3,-3.6641129E-3,1.5271514E5,8.6206274E2,3.1E1,2.0814514E3,1.3812E4,3.3872875E-3,6.265651E-4,8.227176E-3,2.715326E-4,5.2722194E-3,-5.0631287E-3,1.4690361E5,-1.980575E-3,-3.972686E-5,-6.6828392E-3],"split_indices":[44,61,0,0,37,67,3,64,9,0,0,66,0,0,0,37,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.34E2,4.23E2,1.1E1,7E0,4.16E2,3.29E2,8.7E1,3.21E2,8E0,1.3E1,7.4E1,1.42E2,1.79E2,1E0,7E0,9.2E1,5E1,9.1E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-2.4566138E-3,-5.010864E-3,1.9903947E-3,-1.6308035E-4,-4.1670036E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":89,"left_children":[1,3,-1,-1,-1],"loss_changes":[4.858008E-2,5.4068297E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[6.935545E0,5.30113E5,1.9903947E-3,-1.6308035E-4,-4.1670036E-3],"split_indices":[44,2,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[4.32E2,4.09E2,2.3E1,4.02E2,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[3.1135285E-3,-1.2265798E-4,2.531741E-2,1.5824541E-1,8.198957E-4,1.0152658E-3,1.1701315E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":90,"left_children":[1,-1,3,5,-1,-1,-1],"loss_changes":[5.3828716E-2,0E0,9.344375E-2,5.281116E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3],"right_children":[2,-1,4,6,-1,-1,-1],"split_conditions":[1.6799316E7,-1.2265798E-4,1.906E3,8.8E2,8.198957E-4,1.0152658E-3,1.1701315E-2],"split_indices":[1,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.27E2,3.41E2,8.6E1,4E0,8.2E1,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics index 5f5bdac0d..0a36afbae 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics @@ -1,109 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,duration_max,0.21813749,82414.25463743677,311681.84687962633,1504.0,4702.0,7190.0,20850.0,3183955.0 -1,peakExecutionMemory_max,0.082618274,1233353538.7048905,2601007219.6365647,0.0,85937632.0,471303176.0,1493186488.0,25501360736.0 -2,executorCPUTime_mean,0.071007006,2722.177687148267,7012.88983905512,23.339985149433822,585.2242380261248,1063.8777813608513,2163.7598204264873,68323.01515151515 -3,duration_sum,0.06950031,16930121.96627319,45073353.38047576,3197.0,1082442.0,2678990.0,10368109.0,515693115.0 -4,shuffle_write_bw,0.053588066,88801.55237700409,93971.75983640963,0.0,1351.9543033361367,52744.59381627414,166023.2400627615,326131.0503035477 -5,input_bytesRead_mean,0.034031473,15367291.833589979,13710603.535320966,21863.21726572529,5113159.273237375,11711469.0760095,21549125.812765956,92298335.07638542 -6,scan_bw,0.028711345,110505.86766836004,89612.1377495093,0.0,45187.54901663616,92722.52201161183,143609.24321367222,465063.5526726399 -7,memoryBytesSpilledRatio,0.020162081,1.1100054272857585,3.9835905644793064,0.0,0.0,0.0,0.0,43.971493342592005 -8,sw_writeTime_mean,0.015406664,131.22221226581013,570.380294897338,0.0,16.074844074844076,33.17843866171004,70.21488764044943,6475.538414708389 -9,sw_recordsWritten_sum,0.014074505,2556345618.8684654,5824955615.583718,0.0,319167.0,53631567.0,2066354478.0,44204763473.0 -10,resultSize_max,0.013840832,10957325.65767285,22245624.20798865,2645.0,40208.0,1477382.0,12756524.0,263794922.0 -11,scan_time,0.01295209,5138075.225969646,16908006.48547024,0.0,322677.0,911898.0,2940448.0,217849366.0 -12,duration_mean,0.011668744,4389.021479495678,8051.0668064364545,245.4699965313909,1580.207299270073,2302.596153846154,3826.246829791034,72233.70872350912 -13,duration_min,0.010455646,217.46880269814503,213.6000053492964,5.0,77.0,148.0,295.0,1779.0 -14,jvmGCTime_mean,0.0098547805,63.70179049174891,111.16738988131927,0.0,31.140514631685168,49.11107416029265,70.23493975903614,1390.436668309512 -15,sr_localBytesRead_mean,0.008638238,1550809.257874584,3239093.401866476,0.0,2585.3312883435583,284627.3147440794,1918199.7854019508,29629639.667323805 -16,resultSerializationTime_sum,0.008087278,142.54300168634063,201.73330481784959,0.0,27.0,68.0,152.0,1135.0 -17,executorDeserializeCPUTime_mean,0.006968015,26.95913143088412,29.063846586016425,1.0653743823641202,6.2544901065449015,19.212783171521036,36.59782608695652,245.76923076923077 -18,numTasks_sum,0.006788302,3448.059021922428,7298.431350220087,4.0,527.0,1156.0,3101.0,86122.0 -19,sr_remoteBytesRead_mean,0.006599492,8621230.876874011,20763598.714249212,0.0,14815.464675201518,1045794.9007633588,7696222.868940513,207400140.1710202 -20,sr_fetchWaitTime_mean,0.005970912,282.57444030933357,2063.740776556858,0.0,0.0,0.014450867052023121,0.0744336569579288,26216.59561667383 -21,input_recordsRead_sum,0.005626504,7537500441.369308,19524836240.42767,30000.0,410551846.0,1658123725.0,5999989709.0,230400923204.0 -22,sr_localBytesReadRatio,0.0047795097,0.13775142996534326,0.21834409881656594,0.0,0.0003435677540369934,0.052422868005860185,0.18397368378307108,1.6954497523177325 -23,executorDeserializeTime_mean,0.0046688416,77.619961219705,78.66729885547291,1.2399579021224347,16.74,57.15866922584773,111.59245283018868,438.3862068965517 -24,sr_localBlocksFetched_sum,0.00451675,34560.099494097805,81564.66670257768,0.0,329.0,5582.0,30418.0,870491.0 -25,sr_remoteBytesReadRatio,0.0034871036,0.6856243468838018,1.1473671005608985,0.0,0.0015316802906526895,0.13301082808430215,1.0519672879672015,11.86796722526039 -26,shuffle_read_bw,0.0034601516,276816955.4836904,847565126.6737152,0.0,0.0,4930489.4375,137218859.4920635,10530883927.666666 -27,sqlOp_Expand,0.0031522522,0.11129848229342328,0.3147669133570336,0.0,0.0,0.0,0.0,1.0 -28,executorRunTime_mean,0.0027702448,4297.896908257712,8062.7718056424565,226.90115532734274,1490.5288200108755,2236.8281325012003,3741.984555984556,72194.51503203549 -29,sw_bytesWrittenRatio,0.0026040608,0.8014652434613514,1.2780756184305504,0.0,0.0023701803469738034,0.22863716255153424,1.206131247096677,13.563416977578123 -30,data_size,0.0019577835,650411028780.6053,2082773012482.3335,0.0,21721183788.0,89736852644.0,333792838797.0,21374406089219.0 -31,sr_totalBytesRead_mean,0.0018648852,10172040.134748595,23817088.532298744,0.0,19575.48201438849,1695974.662708024,9072792.773504274,237029779.838344 -32,sqlOp_SubqueryBroadcast,0.0018241289,0.7976391231028668,0.40209900029018336,0.0,1.0,1.0,1.0,1.0 -33,diskBytesSpilledRatio,0.0013953688,0.24474497918684088,0.8105072165034726,0.0,0.0,0.0,0.0,6.729163387759685 -34,sr_remoteBlocksFetched_sum,0.0013747218,224234.64755480608,509217.62116016017,0.0,2164.0,37041.0,193536.0,3942472.0 -35,memoryBytesSpilled_mean,0.0013567632,18194132.33421661,71075266.1814507,0.0,0.0,0.0,0.0,724082083.4348422 -36,sr_totalBytesReadRatio,0.001344917,0.823375776849145,1.319042835980761,0.0,0.002400770847119191,0.22866221128988315,1.2670990125430908,13.563416977578123 -37,sqlOp_AQEShuffleRead,0.0013331274,0.7065767284991569,0.45571511734167724,0.0,0.0,1.0,1.0,1.0 -38,sqlOp_Subquery,0.0009835393,0.09106239460370995,0.28794070420987344,0.0,0.0,0.0,0.0,1.0 -39,diskBytesSpilled_mean,0.0009471796,4546045.185000375,15069027.640347425,0.0,0.0,0.0,0.0,111631484.38187319 -40,sqlOp_Scan orc ,0.0006244791,0.45868465430016864,0.498710794080452,0.0,0.0,0.0,1.0,1.0 -41,sw_bytesWritten_mean,0.00060904864,9961926.690653043,23574771.301399454,0.0,19575.48201438849,1687125.7618110236,9963399.731213873,237029779.838344 -42,sqlOp_SortMergeJoin,0.00042181378,0.41652613827993257,0.49339906934430416,0.0,0.0,0.0,1.0,1.0 -43,sqlOp_Sort,0.00041253344,0.6053962900505903,0.4891780403788745,0.0,0.0,1.0,1.0,1.0 -44,sqlOp_Scan parquet ,0.00033403642,0.5311973018549747,0.49944707588925713,0.0,0.0,1.0,1.0,1.0 -45,sqlOp_BroadcastExchange,9.627697e-05,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 -46,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Window,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_BroadcastNestedLoopJoin,0.0,0.04047217537942664,0.1972302630779745,0.0,0.0,0.0,0.0,1.0 -75,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -76,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,maxOnHeapMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 -79,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,executorMemory,0.0,40960.0,0.0,40960.0,40960.0,40960.0,40960.0,40960.0 -81,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -84,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,duration_max,0.22266534,82414.25463743677,311681.84687962633,1504.0,4702.0,7190.0,20850.0,3183955.0 +1,peakExecutionMemory_max,0.08085616,1233353538.7048905,2601007219.6365647,0.0,85937632.0,471303176.0,1493186488.0,25501360736.0 +2,executorCPUTime_mean,0.06561653,2722.177687148267,7012.88983905512,23.339985149433822,585.2242380261248,1063.8777813608513,2163.7598204264873,68323.01515151515 +3,duration_sum,0.06497773,16930121.96627319,45073353.38047576,3197.0,1082442.0,2678990.0,10368109.0,515693115.0 +4,shuffle_write_bw,0.058292877,88801.55237700409,93971.75983640963,0.0,1351.9543033361367,52744.59381627414,166023.2400627615,326131.0503035477 +5,input_bytesRead_mean,0.031057531,15367291.833589979,13710603.535320966,21863.21726572529,5113159.273237375,11711469.0760095,21549125.812765956,92298335.07638542 +6,scan_bw,0.029582083,110505.86766836004,89612.1377495093,0.0,45187.54901663616,92722.52201161183,143609.24321367222,465063.5526726399 +7,memoryBytesSpilledRatio,0.021674993,1.1100054272857585,3.9835905644793064,0.0,0.0,0.0,0.0,43.971493342592005 +8,sw_writeTime_mean,0.015679812,131.22221226581013,570.380294897338,0.0,16.074844074844076,33.17843866171004,70.21488764044943,6475.538414708389 +9,resultSize_max,0.014430633,10957325.65767285,22245624.20798865,2645.0,40208.0,1477382.0,12756524.0,263794922.0 +10,duration_mean,0.012332905,4389.021479495678,8051.0668064364545,245.4699965313909,1580.207299270073,2302.596153846154,3826.246829791034,72233.70872350912 +11,sw_recordsWritten_sum,0.012244041,2556345618.8684654,5824955615.583718,0.0,319167.0,53631567.0,2066354478.0,44204763473.0 +12,scan_time,0.0121625895,5138075.225969646,16908006.48547024,0.0,322677.0,911898.0,2940448.0,217849366.0 +13,jvmGCTime_mean,0.010460112,63.70179049174891,111.16738988131927,0.0,31.140514631685168,49.11107416029265,70.23493975903614,1390.436668309512 +14,duration_min,0.009937596,217.46880269814503,213.6000053492964,5.0,77.0,148.0,295.0,1779.0 +15,sr_localBytesRead_mean,0.009483001,1550809.257874584,3239093.401866476,0.0,2585.3312883435583,284627.3147440794,1918199.7854019508,29629639.667323805 +16,numTasks_sum,0.008629044,3448.059021922428,7298.431350220087,4.0,527.0,1156.0,3101.0,86122.0 +17,resultSerializationTime_sum,0.008303617,142.54300168634063,201.73330481784959,0.0,27.0,68.0,152.0,1135.0 +18,sr_fetchWaitTime_mean,0.00601879,282.57444030933357,2063.740776556858,0.0,0.0,0.014450867052023121,0.0744336569579288,26216.59561667383 +19,executorDeserializeCPUTime_mean,0.005568955,26.95913143088412,29.063846586016425,1.0653743823641202,6.2544901065449015,19.212783171521036,36.59782608695652,245.76923076923077 +20,executorDeserializeTime_mean,0.0055408347,77.619961219705,78.66729885547291,1.2399579021224347,16.74,57.15866922584773,111.59245283018868,438.3862068965517 +21,sr_localBlocksFetched_sum,0.0052517653,34560.099494097805,81564.66670257768,0.0,329.0,5582.0,30418.0,870491.0 +22,sr_remoteBytesReadRatio,0.0052471147,0.6856243468838018,1.1473671005608985,0.0,0.0015316802906526895,0.13301082808430215,1.0519672879672015,11.86796722526039 +23,sr_remoteBytesRead_mean,0.004776956,8621230.876874011,20763598.714249212,0.0,14815.464675201518,1045794.9007633588,7696222.868940513,207400140.1710202 +24,input_recordsRead_sum,0.0038761208,7537500441.369308,19524836240.42767,30000.0,410551846.0,1658123725.0,5999989709.0,230400923204.0 +25,executorRunTime_mean,0.0037833047,4297.896908257712,8062.7718056424565,226.90115532734274,1490.5288200108755,2236.8281325012003,3741.984555984556,72194.51503203549 +26,sr_localBytesReadRatio,0.0032253778,0.13775142996534326,0.21834409881656594,0.0,0.0003435677540369934,0.052422868005860185,0.18397368378307108,1.6954497523177325 +27,sqlOp_Expand,0.0032066684,0.11129848229342328,0.3147669133570336,0.0,0.0,0.0,0.0,1.0 +28,shuffle_read_bw,0.0029248933,276816955.4836904,847565126.6737152,0.0,0.0,4930489.4375,137218859.4920635,10530883927.666666 +29,data_size,0.0025719197,650411028780.6053,2082773012482.3335,0.0,21721183788.0,89736852644.0,333792838797.0,21374406089219.0 +30,sr_totalBytesRead_mean,0.0022292563,10172040.134748595,23817088.532298744,0.0,19575.48201438849,1695974.662708024,9072792.773504274,237029779.838344 +31,sqlOp_SubqueryBroadcast,0.0020551344,0.7976391231028668,0.40209900029018336,0.0,1.0,1.0,1.0,1.0 +32,sr_remoteBlocksFetched_sum,0.0016197824,224234.64755480608,509217.62116016017,0.0,2164.0,37041.0,193536.0,3942472.0 +33,diskBytesSpilledRatio,0.0016090691,0.24474497918684088,0.8105072165034726,0.0,0.0,0.0,0.0,6.729163387759685 +34,sw_bytesWrittenRatio,0.0014864905,0.8014652434613514,1.2780756184305504,0.0,0.0023701803469738034,0.22863716255153424,1.206131247096677,13.563416977578123 +35,memoryBytesSpilled_mean,0.0013712087,18194132.33421661,71075266.1814507,0.0,0.0,0.0,0.0,724082083.4348422 +36,sr_totalBytesReadRatio,0.0010026896,0.823375776849145,1.319042835980761,0.0,0.002400770847119191,0.22866221128988315,1.2670990125430908,13.563416977578123 +37,sqlOp_SortMergeJoin,0.0009628031,0.41652613827993257,0.49339906934430416,0.0,0.0,0.0,1.0,1.0 +38,sqlOp_AQEShuffleRead,0.00090924423,0.7065767284991569,0.45571511734167724,0.0,0.0,1.0,1.0,1.0 +39,sqlOp_Subquery,0.0008563004,0.09106239460370995,0.28794070420987344,0.0,0.0,0.0,0.0,1.0 +40,sqlOp_Scan parquet,0.00080813543,0.5311973018549747,0.49944707588925713,0.0,0.0,1.0,1.0,1.0 +41,sw_bytesWritten_mean,0.0007380446,9961926.690653043,23574771.301399454,0.0,19575.48201438849,1687125.7618110236,9963399.731213873,237029779.838344 +42,sqlOp_Sort,0.00046764748,0.6053962900505903,0.4891780403788745,0.0,0.0,1.0,1.0,1.0 +43,diskBytesSpilled_mean,0.0004394822,4546045.185000375,15069027.640347425,0.0,0.0,0.0,0.0,111631484.38187319 +44,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_Window,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +82,sqlOp_HashAggregate,0.0,0.9797639123102867,0.14092579360637603,0.0,1.0,1.0,1.0,1.0 +83,sqlOp_Project,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +84,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 85,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,numExecutors,0.0,8.062394603709949,0.3504133651237026,8.0,8.0,8.0,8.0,14.0 -87,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -89,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -90,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_Project,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -94,sqlOp_HashAggregate,0.0,0.9797639123102867,0.14092579360637603,0.0,1.0,1.0,1.0,1.0 -95,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,sqlOp_Filter,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -97,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sqlOp_Exchange,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -100,sqlOp_ColumnarToRow,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -101,sqlOp_BroadcastHashJoin,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 -102,maxMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 -103,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,platform_dataproc,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -105,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +88,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,executorMemory,0.0,40960.0,0.0,40960.0,40960.0,40960.0,40960.0,40960.0 +91,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,maxOnHeapMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 +93,maxMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 +94,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +95,numExecutors,0.0,8.062394603709949,0.3504133651237026,8.0,8.0,8.0,8.0,14.0 +96,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_Filter,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +98,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_BroadcastNestedLoopJoin,0.0,0.04047217537942664,0.1972302630779745,0.0,0.0,0.0,0.0,1.0 +103,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,platform_dataproc,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +107,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_BroadcastExchange,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 +110,sqlOp_BroadcastHashJoin,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 +111,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_ColumnarToRow,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +113,sqlOp_Exchange,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.0-parquet.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.0-parquet.json deleted file mode 100644 index b463d09da..000000000 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.0-parquet.json +++ /dev/null @@ -1 +0,0 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","diskBytesSpilled_sum","duration_sum","duration_max","duration_min","duration_mean","executorCPUTime_sum","executorDeserializeCPUTime_sum","executorDeserializeTime_sum","executorRunTime_sum","input_bytesRead_sum","input_recordsRead_sum","jvmGCTime_sum","memoryBytesSpilled_sum","output_bytesWritten_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_fetchWaitTime_sum","sr_localBlocksFetched_sum","sr_localBytesRead_sum","sr_remoteBlocksFetched_sum","sr_remoteBytesRead_sum","sr_remoteBytesReadToDisk_sum","sr_totalBytesRead_sum","sw_bytesWritten_sum","sw_recordsWritten_sum","sw_writeTime_sum","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CustomShuffleReader","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_Project","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","resourceProfileId","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","hasSqlID","cache_hit_ratio","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilledBool","memoryBytesSpilledBool","sqlOp_AQEShuffleRead","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_RunningWindowFunction","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit"],"feature_types":["int","int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","i","float","float","float","float","float","float","float","float","float","float","i","i","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"137"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130,131,132,133,134,135,136,137],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-4.2948145E-3,-5.097333E-1,9.67318E-1,-8.680958E-1,-2.3944771E-1,5.372626E-1,1.5688226E0,-1.0284226E0,4.5961246E-2,-3.447184E-1,3.3476815E-1,1.3921557E-1,7.324191E-1,4.1080326E-2,6.0292955E-2,-3.4476873E-2,-1.7194442E-2,-1.091018E-2,1.5687432E-2,-4.715714E-1,-2.6515059E-3,-6.809994E-2,8.6247E-1,2.4114221E-2,-2.1967944E-2,5.15364E-1,1.0019386E0,4.5176274E-3,-5.226616E-1,-1.0903647E-1,2.2374797E-1,4.551139E-2,-8.043853E-3,3.451231E-2,1.407121E-2,1.0132033E-2,-1.6044366E-1,3.2318872E-1,2.4368348E-2,3.6686942E-2,1.6042808E-2,-4.0881447E-3,4.342626E-3,-5.844236E-1,-1.8011394E-1,-1.7774177E-1,2.9422184E-3,-1.3276102E-3,1.2585631E-2,-1.1254252E-3,3.6891128E-3,-1.8416885E-2,-9.637405E-3,4.116481E-3,1.3438293E-2,-6.690872E-1,-3.5920227E-1,-7.6510655E-3,-1.1048367E-3,-7.553959E-3,-1.2783123E-3,1.3104709E-3,-2.3479066E-3,-3.0117825E-2,-5.6133157E-1,-4.4573867E-3,-1.2967139E-2,-1.9129088E-2,-7.4541564E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,31,33,-1,35,37,39,41,43,45,47,49,-1,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,67,-1,-1,-1,-1],"loss_changes":[1.2915878E2,1.6548954E1,2.2438354E1,1.0891815E1,6.160209E0,4.0928345E0,9.036026E-1,7.5603485E-1,2.2039075E0,3.687088E0,3.4602418E0,1.8843604E0,1.5896568E0,0E0,0E0,0E0,0E0,0E0,0E0,1.510417E0,6.0244864E-1,2.1791258E-1,1.5566492E-1,0E0,7.9657704E-1,7.7710724E-1,4.7277164E-1,1.3718107E-1,1.0794964E0,2.5035957E-1,3.7067348E-1,4.293065E-2,0E0,0E0,0E0,0E0,2.3131385E-1,2.0443058E-1,0E0,0E0,0E0,0E0,0E0,6.558609E-1,7.578522E-2,9.284088E-2,0E0,0E0,0E0,0E0,0E0,2.5537306E-2,0E0,0E0,0E0,4.226923E-1,8.871019E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4482498E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,36,36,37,37,43,43,44,44,45,45,51,51,55,55,56,56,64,64],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,32,34,-1,36,38,40,42,44,46,48,50,-1,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,68,-1,-1,-1,-1],"split_conditions":[2.88414E5,8.03052E5,4.148E4,1.83E2,7.58587E8,9.577105E8,2.12632E5,2.8681312E8,1.64108E8,1.098767E6,1.83E2,7.62802E5,1.83E2,4.1080326E-2,6.0292955E-2,-3.4476873E-2,-1.7194442E-2,-1.091018E-2,1.5687432E-2,3.2416E4,4.635E8,1.276E3,9.490297E8,2.4114221E-2,7.51885E5,6.0958E5,2.239676E10,3.5590376E7,7.529E3,5.601505E6,7.2903E4,1.685836E6,-8.043853E-3,3.451231E-2,1.407121E-2,1.0132033E-2,1.17E2,3.35E2,2.4368348E-2,3.6686942E-2,1.6042808E-2,-4.0881447E-3,4.342626E-3,1.46379E5,1E3,1.521867E6,2.9422184E-3,-1.3276102E-3,1.2585631E-2,-1.1254252E-3,3.6891128E-3,9.5E1,-9.637405E-3,4.116481E-3,1.3438293E-2,3.7E1,2.6E1,-7.6510655E-3,-1.1048367E-3,-7.553959E-3,-1.2783123E-3,1.3104709E-3,-2.3479066E-3,-3.0117825E-2,2.338E3,-4.4573867E-3,-1.2967139E-2,-1.9129088E-2,-7.4541564E-3],"split_indices":[12,9,3,47,16,23,3,16,23,6,47,6,47,0,0,0,0,0,0,12,27,17,10,0,18,12,10,16,3,2,22,2,0,0,0,0,4,17,0,0,0,0,0,12,0,6,0,0,0,0,0,4,0,0,0,4,17,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,1.72E2,8.9E1,7.3E1,9.9E1,5.3E1,3.6E1,6.2E1,1.1E1,8.4E1,1.5E1,1.8E1,3.5E1,2.1E1,1.5E1,5.6E1,6E0,6E0,5E0,6.1E1,2.3E1,9E0,6E0,3E0,1.5E1,2.1E1,1.4E1,6E0,5.5E1,1.6E1,7E0,6E0,3E0,3E0,3E0,4E0,1.1E1,1.3E1,8E0,1E1,4E0,3E0,3E0,4.6E1,9E0,1.2E1,4E0,3E0,4E0,3E0,3E0,6E0,5E0,5E0,8E0,3.2E1,1.4E1,6E0,3E0,8E0,4E0,3E0,3E0,7E0,2.5E1,3E0,1.1E1,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-5.9768695E-2,-4.672948E-1,9.421753E-1,-8.439249E-1,-1.4857344E-1,4.6850228E-1,1.3107026E0,-9.484451E-1,2.4444E-3,-3.170696E-1,6.4219184E-2,8.0388594E-1,3.1451258E-1,2.3595441E-2,4.3924607E-2,-3.295703E-2,-3.754458E-1,9.930676E-3,-9.786423E-3,-3.92838E-1,6.304481E-2,5.010258E-1,-6.0629327E-2,1.0426705E-2,3.321504E-2,1.629767E-1,1.4973287E-2,-4.303361E-3,-1.5917214E-2,-2.3486143E-1,-5.6077576E-1,9.892991E-3,-6.698395E-2,4.53857E-3,2.5028512E-2,5.1342387E-2,-3.9048153E-1,-4.1002776E-2,3.035189E-1,-1.1689476E-1,-1.5158228E-2,-6.0331535E-1,-8.963407E-3,-6.1730803E-3,3.404063E-3,-4.2481147E-2,1.177113E-2,-1.6588023E-2,-5.232761E-3,-3.91489E-3,1.6051753E-3,5.049856E-3,1.1288365E-2,-2.2909176E-1,1.8211408E-2,-2.436501E-2,-1.4883216E-2,-8.5477896E-2,6.24054E-3,-1.2696257E-2,-7.826638E-2,4.6974365E-3,-2.9375197E-3,-1.6426139E-1,3.781802E-2,-3.7983828E-3,-6.104155E-4,-1.9739264E-1,-1.2994E-3,3.3316396E-3,-2.2584836E-3,-2.8096372E-3,-7.4714413E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,-1,-1,29,31,33,35,-1,-1,37,-1,-1,-1,39,41,-1,43,-1,-1,45,47,49,51,53,-1,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,59,61,-1,-1,63,-1,-1,65,-1,-1,67,69,-1,-1,71,-1,-1,-1,-1,-1],"loss_changes":[1.143611E2,2.3762585E1,1.35254135E1,8.052429E0,3.9285285E0,1.6580572E0,4.004364E-1,3.1158752E0,1.1256188E0,1.7816072E0,2.7025747E0,8.6945343E-1,5.4059196E-1,0E0,0E0,0E0,2.504741E-1,0E0,0E0,1.2213368E0,3.70008E-1,9.85487E-1,1.4604788E0,0E0,0E0,4.424609E-1,0E0,0E0,0E0,7.2464955E-1,1.2022734E-1,0E0,1.9273035E-1,0E0,0E0,9.0032655E-1,2.0078337E-1,5.734615E-2,7.237315E-3,3.0734223E-1,0E0,8.305931E-2,0E0,0E0,0E0,2.6046985E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4343693E-1,1.5313466E-1,0E0,0E0,2.101994E-1,0E0,0E0,1.4257584E-2,0E0,0E0,4.5393407E-2,7.10964E-2,0E0,0E0,1.7968267E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,35,35,36,36,37,37,38,38,39,39,41,41,45,45,53,53,54,54,57,57,60,60,63,63,64,64,67,67],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,-1,-1,30,32,34,36,-1,-1,38,-1,-1,-1,40,42,-1,44,-1,-1,46,48,50,52,54,-1,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,60,62,-1,-1,64,-1,-1,66,-1,-1,68,70,-1,-1,72,-1,-1,-1,-1,-1],"split_conditions":[1.1026E4,8.81041E5,7.469149E6,1.83E2,2.07812E5,1.1541E4,3.191E4,3.7372906E8,6.7427456E8,1.203673E6,1.0382E4,7.58587E8,2.9138386E7,2.3595441E-2,4.3924607E-2,-3.295703E-2,1.41048E5,9.930676E-3,-9.786423E-3,1.5264E4,5.6035635E9,2.917845E8,7.192367E10,1.0426705E-2,3.321504E-2,1.5551E4,1.4973287E-2,-4.303361E-3,-1.5917214E-2,2.25E2,3.65997E8,9.892991E-3,1.492234E6,4.53857E-3,2.5028512E-2,2.926E3,4.73E2,5.5E1,5.2068E4,4.939E3,-1.5158228E-2,1.27001E5,-8.963407E-3,-6.1730803E-3,3.404063E-3,1.0886205E9,1.177113E-2,-1.6588023E-2,-5.232761E-3,-3.91489E-3,1.6051753E-3,5.049856E-3,1.1288365E-2,1.440233E6,1.0822827E3,-2.436501E-2,-1.4883216E-2,2.85218E5,6.24054E-3,-1.2696257E-2,3.66846E6,4.6974365E-3,-2.9375197E-3,9.448E3,3.23E2,-3.7983828E-3,-6.104155E-4,6.103E3,-1.2994E-3,3.3316396E-3,-2.2584836E-3,-2.8096372E-3,-7.4714413E-3],"split_indices":[3,9,6,47,12,7,3,16,10,6,7,16,18,0,0,0,6,0,0,20,10,16,10,0,0,3,0,0,0,4,16,0,6,0,0,0,17,19,8,3,0,12,0,0,0,16,0,0,0,0,0,0,0,2,5,0,0,12,0,0,21,0,0,3,17,0,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,1.98E2,8E1,9E1,1.08E2,3.6E1,4.4E1,8E1,1E1,6E1,4.8E1,1E1,2.6E1,5E0,3.9E1,7E1,1E1,5E0,5E0,5E1,1E1,1E1,3.8E1,4E0,6E0,1.4E1,1.2E1,4E0,6E0,2.7E1,2.3E1,3E0,7E0,5E0,5E0,2.9E1,9E0,6E0,8E0,1.9E1,8E0,1.9E1,4E0,4E0,3E0,2.3E1,6E0,5E0,4E0,3E0,3E0,3E0,5E0,1E1,9E0,7E0,1.2E1,2E1,3E0,4E0,6E0,4E0,5E0,1.2E1,8E0,3E0,3E0,9E0,3E0,5E0,3E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-1.04451515E-2,-5.182287E-1,9.181728E-1,-8.225506E-1,-2.0533854E-1,5.0036126E-1,1.3507504E0,-9.562876E-1,1.15156814E-1,-2.9513332E-1,2.2930388E-1,1.052168E0,3.3849666E-1,2.9134523E-2,4.6695687E-2,-1.002868E0,-1.23152295E-2,-8.775733E-3,4.8649442E-1,-3.9996946E-1,-3.385029E-2,1.9079294E-2,2.8483225E-2,3.9998986E-2,1.7285287E-2,-2.7749162E-2,5.1033634E-1,-3.4980215E-2,-2.6514793E-2,1.916828E-2,8.236282E-3,7.9865735E-3,-4.5356295E-1,-1.4245005E-1,2.0115548E-1,-9.963475E-2,9.966599E-3,9.120458E-3,-2.1162845E-1,5.95364E-1,2.4578695E-1,-1.2290665E-1,-5.184038E-1,-2.2511207E-1,-6.207542E-2,-1.242925E-3,1.1354317E-2,5.564892E-4,-6.3293395E-3,-1.1669552E-2,3.7926267E-4,1.2456523E-2,6.6681045E-1,1.0504883E-2,2.0438354E-3,1.6237264E-3,-6.579611E-3,-5.704506E-1,-1.6180661E-1,-8.574041E-3,-3.7746963E-3,-5.38343E-4,-3.3502749E-3,2.4432605E-2,1.0640073E-2,-6.3359785E-1,-1.2884443E-2,-1.7665859E-3,-7.3480904E-3,-2.1496492E-2,-8.961733E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,29,31,33,-1,35,-1,-1,37,39,-1,-1,-1,-1,-1,41,43,45,47,-1,-1,49,51,53,55,57,59,61,-1,-1,-1,-1,-1,-1,-1,63,-1,-1,-1,-1,65,67,-1,-1,-1,-1,-1,-1,69,-1,-1,-1,-1,-1],"loss_changes":[1.4005075E2,1.8113213E1,1.8301407E1,1.2274521E1,3.8144236E0,4.5563326E0,9.580231E-1,1.8877716E0,2.0046403E0,2.177813E0,1.223829E0,5.1017E-1,2.7833266E0,0E0,0E0,2.7645874E-1,0E0,0E0,2.3590565E-2,2.0923233E0,6.4117223E-1,0E0,4.630859E-1,0E0,0E0,9.180091E-1,5.284867E-1,0E0,0E0,0E0,0E0,0E0,1.072608E0,9.897286E-2,3.0438152E-1,1.0546893E-1,0E0,0E0,3.41408E-1,6.9497585E-2,1.11453176E-1,1.5158997E-1,7.243242E-1,4.3563545E-3,1.7309453E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.1428108E-1,0E0,0E0,0E0,0E0,1.5473843E-1,3.7225112E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3012695E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,18,18,19,19,20,20,22,22,25,25,26,26,32,32,33,33,34,34,35,35,38,38,39,39,40,40,41,41,42,42,43,43,44,44,52,52,57,57,58,58,65,65],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,30,32,34,-1,36,-1,-1,38,40,-1,-1,-1,-1,-1,42,44,46,48,-1,-1,50,52,54,56,58,60,62,-1,-1,-1,-1,-1,-1,-1,64,-1,-1,-1,-1,66,68,-1,-1,-1,-1,-1,-1,70,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,8.99622E5,7.762619E6,1.83E2,7.164074E8,9.099E3,4.6908E4,3.1361875E8,1.64108E8,1.22661E6,2.4E2,5.776113E9,1.0126E4,2.9134523E-2,4.6695687E-2,3.8E3,-1.23152295E-2,-8.775733E-3,3.342E3,2.9169E4,6.5232E4,1.9079294E-2,1.1156849E9,3.9998986E-2,1.7285287E-2,7.6E1,1.9002575E10,-3.4980215E-2,-2.6514793E-2,1.916828E-2,8.236282E-3,7.9865735E-3,2.036E3,2.854041E6,6.463662E9,1.02E2,9.966599E-3,9.120458E-3,1.801545E6,3.969357E6,4.6368E4,3.185029E2,2.6421262E3,7.404E3,1E2,-1.242925E-3,1.1354317E-2,5.564892E-4,-6.3293395E-3,-1.1669552E-2,3.7926267E-4,1.2456523E-2,6.402839E-1,1.0504883E-2,2.0438354E-3,1.6237264E-3,-6.579611E-3,1.37951E5,6.437E3,-8.574041E-3,-3.7746963E-3,-5.38343E-4,-3.3502749E-3,2.4432605E-2,1.0640073E-2,5.4730977E9,-1.2884443E-2,-1.7665859E-3,-7.3480904E-3,-2.1496492E-2,-8.961733E-3],"split_indices":[12,9,6,47,16,7,3,16,23,6,17,10,3,0,0,3,0,0,7,12,28,0,16,0,0,4,10,0,0,0,0,0,22,2,23,4,0,0,6,2,20,5,5,3,19,0,0,0,0,0,0,0,63,0,0,0,0,12,3,0,0,0,0,0,0,11,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.95E2,1.91E2,1.04E2,9.6E1,9.5E1,5.4E1,5E1,8.4E1,1.2E1,7.9E1,1.6E1,1.1E1,4.3E1,1.1E1,3.9E1,7.7E1,7E0,6E0,6E0,5.6E1,2.3E1,5E0,1.1E1,7E0,4E0,1.4E1,2.9E1,4.9E1,2.8E1,3E0,3E0,4E0,5.2E1,1.6E1,7E0,8E0,3E0,5E0,9E0,2.1E1,8E0,9E0,4.3E1,7E0,9E0,3E0,4E0,4E0,4E0,5E0,4E0,7E0,1.4E1,5E0,3E0,3E0,6E0,3.7E1,6E0,4E0,3E0,5E0,4E0,1E1,4E0,2.5E1,1.2E1,3E0,3E0,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[1.5012548E-2,-4.4777912E-1,8.4255326E-1,-5.6950563E-1,1.558269E-1,4.7040752E-1,1.3393337E0,-8.97149E-1,-2.3456092E-1,-5.7694167E-2,8.0360186E-1,9.1054434E-1,3.1992966E-1,3.5900768E-2,1.5753971E0,-9.5903E-1,-3.8045558E-1,-4.020763E-1,-1.2148722E-1,-3.8720822E-1,1.8779576E-1,3.1263083E-2,1.26553625E-2,-2.1285478E-3,3.734636E-2,9.3239434E-2,5.1568747E-1,5.9595205E-2,3.6754545E-2,-3.164266E-2,-1.748531E-2,-2.2911301E-3,-1.6185738E-2,-1.7617552E-1,-1.723648E-2,-2.1246824E-1,5.6059104E-3,-2.0045802E-2,-2.8045522E-3,3.056521E-1,-6.8473085E-3,-1.0061287E-2,1.6973925E-1,6.339364E-1,2.3502985E-1,-2.537851E-4,-2.5009564E-1,-2.8699988E-1,9.735806E-3,1.7922495E-1,-7.579004E-2,4.1871658E-3,1.3277213E-2,2.940622E-1,-5.3502955E-2,8.264683E-3,2.2417076E-2,1.9346676E-3,1.0058271E-2,-1.0605165E-2,-2.205469E-3,-3.0967607E-3,-1.0501116E-2,2.9888477E-3,-1.8896614E-3,6.8206637E-6,1.00890435E-2,-1.2384835E-1,2.811139E-3,4.3218876E-3,1.0576469E-2,-4.258278E-3,1.8784384E-3,-6.641082E-3,-5.682532E-2,4.4894958E-4,-3.2857964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1,45,-1,47,49,-1,-1,51,-1,-1,53,55,57,-1,59,61,63,65,67,-1,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,75,-1,-1],"loss_changes":[1.110582E2,1.3755066E1,1.859694E1,1.6799637E1,4.4723983E0,3.7487717E0,5.6980133E-1,2.1290703E0,1.4285965E0,2.0950255E0,1.6858482E-1,3.7345247E0,2.0376077E0,0E0,1.5974045E-1,1.545639E-1,3.427025E-1,8.3986425E-1,5.562181E-1,7.2325015E-1,7.73073E-1,0E0,0E0,0E0,0E0,7.6707447E-1,6.806927E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5974957E-1,0E0,4.6650863E-1,3.103531E-1,0E0,0E0,1.7140973E-1,0E0,0E0,5.708158E-1,2.2184801E-1,1.0314524E-1,0E0,1.0713428E-1,1.2581086E-1,5.096012E-2,1.68107E-1,1.2840681E-1,0E0,0E0,2.757752E-2,7.8224674E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.460243E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7045138E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,26,26,33,33,35,35,36,36,39,39,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,53,53,54,54,67,67,74,74],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1,46,-1,48,50,-1,-1,52,-1,-1,54,56,58,-1,60,62,64,66,68,-1,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,76,-1,-1],"split_conditions":[2.57951E5,1.83E2,1.2065335E7,8.99622E5,1.2211046E7,9.458E3,1.99916E5,3.7372906E8,1.33368E5,1.2806644E-1,5.13905E5,2.917845E8,9.2171E4,3.5900768E-2,1.7754914E0,6.103E3,1.41048E5,2.433E3,6.887E3,6.7452E4,1.0618027E9,3.1263083E-2,1.26553625E-2,-2.1285478E-3,3.734636E-2,4.995E3,2.7179317E10,5.9595205E-2,3.6754545E-2,-3.164266E-2,-1.748531E-2,-2.2911301E-3,-1.6185738E-2,1.008E3,-1.723648E-2,1.443451E6,1.82951E5,-2.0045802E-2,-2.8045522E-3,2.253E3,-6.8473085E-3,-1.0061287E-2,5.909232E8,5.007532E6,3.07E2,-2.537851E-4,4.344E3,1E0,1.82E2,7.6802E4,1.113E3,4.1871658E-3,1.3277213E-2,3.04326E5,1.038E3,8.264683E-3,2.2417076E-2,1.9346676E-3,1.0058271E-2,-1.0605165E-2,-2.205469E-3,-3.0967607E-3,-1.0501116E-2,2.9888477E-3,-1.8896614E-3,6.8206637E-6,1.00890435E-2,9.35E2,2.811139E-3,4.3218876E-3,1.0576469E-2,-4.258278E-3,1.8784384E-3,-6.641082E-3,7.0571E4,4.4894958E-4,-3.2857964E-3],"split_indices":[12,47,6,9,18,7,3,16,12,67,6,16,22,0,59,3,6,20,3,18,21,0,0,0,0,3,10,0,0,0,0,0,0,22,0,6,12,0,0,3,0,0,27,2,17,0,22,45,17,8,17,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.88E2,1.85E2,1.03E2,1.54E2,3.1E1,6E1,4.3E1,7.7E1,7.7E1,2.4E1,7E0,1.4E1,4.6E1,2.5E1,1.8E1,6.8E1,9E0,3E1,4.7E1,1E1,1.4E1,4E0,3E0,3E0,1.1E1,2.2E1,2.4E1,9E0,9E0,6.3E1,5E0,3E0,6E0,1.2E1,1.8E1,2.7E1,2E1,5E0,5E0,1.1E1,3E0,3E0,1.9E1,1.6E1,8E0,4E0,8E0,2E1,7E0,6E0,1.4E1,5E0,6E0,1.2E1,7E0,3E0,1.3E1,3E0,5E0,5E0,3E0,4E0,1.6E1,3E0,4E0,3E0,3E0,1.1E1,3E0,3E0,9E0,4E0,3E0,4E0,7E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[2.731469E-2,-3.9841607E-1,9.283052E-1,-7.2929764E-1,-7.6604225E-2,5.557672E-1,1.3963234E0,-8.406583E-1,2.2788762E-1,-1.5054666E-1,2.54331E-1,-4.3561798E-2,6.609453E-1,3.5140567E-2,5.3485088E-2,-9.231012E-1,-4.42038E-1,-2.0379871E-1,2.5322119E-2,-2.1690284E-1,2.656657E-1,2.6864935E-2,9.336781E-2,1.0552923E-2,-1.3076899E-2,3.208736E-1,7.5865716E-1,-3.0934246E-2,-1.9052641E-2,-2.3740782E-1,-1.7126841E-2,-9.883307E-3,-1.5968133E-3,4.079604E-3,-2.4403445E-1,3.0000135E-3,1.2088278E-2,-1.002181E-1,2.5506997E-1,3.5322185E-3,1.2201574E-2,3.5383385E-2,6.342529E-1,-1.0598673E-2,-2.774671E-3,-3.229575E-1,-6.433369E-2,-4.9204443E-4,-5.8367774E-3,1.00392625E-2,3.4145901E-3,2.4237697E-1,7.533365E-1,-3.904865E-1,-6.645458E-2,8.575674E-2,-1.9655545E-1,1.0977455E-2,1.8819049E-3,1.4494048E-2,2.8091362E-2,-4.1679642E-1,-2.22847E-3,4.274363E-3,-1.5776992E-1,1.6557643E-1,-1.3887017E-3,-9.805078E-3,-4.078075E-3,-4.516182E-1,-5.7626283E-3,-7.0656925E-3,-2.472965E-3,2.2893124E-3,7.03772E-3,-6.5342346E-3,-4.7249782E-1,-1.7078277E-2,-9.781173E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,-1,33,35,-1,37,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,45,-1,-1,47,49,-1,-1,-1,51,-1,-1,53,55,-1,-1,-1,-1,57,59,61,63,65,67,-1,-1,-1,-1,69,-1,-1,71,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,77,-1,-1],"loss_changes":[1.1237503E2,2.113397E1,1.5613312E1,1.0637669E1,2.5342445E0,3.4327507E0,1.4992371E0,2.4839325E0,2.8137329E0,2.377558E0,1.696715E0,1.3455386E0,1.2851028E0,0E0,0E0,3.354149E-1,1.8688393E-1,9.620112E-2,0E0,7.115488E-1,1.9239634E-1,0E0,5.2641475E-1,0E0,0E0,1.0583174E-1,7.8575134E-1,0E0,0E0,6.884381E-2,0E0,0E0,0E0,0E0,9.43635E-1,0E0,0E0,5.233916E-2,4.309976E-2,0E0,0E0,0E0,1.1240568E0,0E0,0E0,7.881551E-1,4.534508E-1,0E0,0E0,0E0,0E0,1.252127E-1,3.6670113E-1,2.8387642E-1,2.210849E-1,1.2031871E-1,5.1997513E-2,0E0,0E0,0E0,0E0,2.0065689E-1,0E0,0E0,2.3113579E-2,1.9533724E-2,0E0,0E0,0E0,5.462742E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.3360424E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,22,22,25,25,26,26,29,29,34,34,37,37,38,38,42,42,45,45,46,46,51,51,52,52,53,53,54,54,55,55,56,56,61,61,64,64,65,65,69,69,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,-1,34,36,-1,38,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,46,-1,-1,48,50,-1,-1,-1,52,-1,-1,54,56,-1,-1,-1,-1,58,60,62,64,66,68,-1,-1,-1,-1,70,-1,-1,72,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,78,-1,-1],"split_conditions":[3.47989E5,9.98004E5,6.6569E4,1.83E2,5.101357E7,5.7678764E-5,2.12632E5,2.6876314E8,3.676E3,2.594684E6,7.284E3,6.806435E-6,4.618673E6,3.5140567E-2,5.3485088E-2,1.763598E8,4.369E3,3.15859E5,2.5322119E-2,2.9169E4,3.718604E6,2.6864935E-2,7.826E3,1.0552923E-2,-1.3076899E-2,2.9138386E7,4.1294E4,-3.0934246E-2,-1.9052641E-2,2E0,-1.7126841E-2,-9.883307E-3,-1.5968133E-3,4.079604E-3,3.044519E6,3.0000135E-3,1.2088278E-2,1.207E3,9.89E3,3.5322185E-3,1.2201574E-2,3.5383385E-2,2.0585947E9,-1.0598673E-2,-2.774671E-3,9.21E3,1.706089E10,-4.9204443E-4,-5.8367774E-3,1.00392625E-2,3.4145901E-3,2.239676E10,3.0274E5,2.1504E4,1.7283224E7,1.9812842E3,2.259E3,1.0977455E-2,1.8819049E-3,1.4494048E-2,2.8091362E-2,1E0,-2.22847E-3,4.274363E-3,2.8561795E8,4.7E1,-1.3887017E-3,-9.805078E-3,-4.078075E-3,5.87E2,-5.7626283E-3,-7.0656925E-3,-2.472965E-3,2.2893124E-3,7.03772E-3,-6.5342346E-3,1.69E2,-1.7078277E-2,-9.781173E-3],"split_indices":[12,9,3,47,18,63,3,16,3,6,7,64,2,0,0,21,20,21,0,12,6,0,3,0,0,18,20,0,0,19,0,0,0,0,2,0,0,0,20,0,0,0,27,0,0,3,10,0,0,0,0,10,22,7,27,5,0,0,0,0,0,34,0,0,21,4,0,0,0,28,0,0,0,0,0,0,19,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,1.98E2,9.3E1,9.7E1,1.01E2,5.3E1,4E1,8.7E1,1E1,8.3E1,1.8E1,8E0,4.5E1,2.1E1,1.9E1,7.1E1,1.6E1,6E0,4E0,7.2E1,1.1E1,3E0,1.5E1,4E0,4E0,1.1E1,3.4E1,6.2E1,9E0,6E0,1E1,3E0,3E0,5E0,6.7E1,5E0,6E0,7E0,8E0,3E0,8E0,7E0,2.7E1,3E0,3E0,4.6E1,2.1E1,4E0,3E0,5E0,3E0,7E0,2E1,3.6E1,1E1,1E1,1.1E1,4E0,3E0,7E0,1.3E1,3.3E1,3E0,3E0,7E0,6E0,4E0,3E0,8E0,2.8E1,5E0,3E0,4E0,3E0,3E0,3E0,2.5E1,1.7E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-1.6545374E-2,-3.7674236E-1,9.411498E-1,-7.935405E-1,-1.4412029E-1,6.124253E-1,1.316781E0,-8.627E-1,1.5373571E-3,-2.4617739E-1,1.9525832E-1,2.0540261E-1,7.4187523E-1,1.3955498E0,1.768156E-2,-9.053927E-1,-3.5522264E-1,-6.3509964E-3,6.437597E-3,-3.407789E-1,8.730174E-2,-5.871338E-2,8.5082084E-1,-7.312535E-4,1.0374578E-2,1.1033022E-2,8.361469E-1,5.1780824E-2,3.4825463E-2,-2.9693663E-2,-1.360752E-2,-6.1942074E-3,-1.3815725E-2,9.726382E-2,-3.7664098E-1,9.351397E-3,3.4240037E-3,-1.2418054E-1,6.9173407E-3,3.2131847E-2,1.3423062E-2,5.3145146E-1,9.6619433E-1,-3.7917523E-3,9.270689E-3,-4.856896E-1,-2.485711E-1,-9.284641E-2,5.4067746E-3,-2.4082352E-1,-2.6349923E-2,9.982757E-3,2.0192485E-2,3.342443E-2,1.5192855E-2,-2.0286975E-2,-3.9274573E-1,-1.5351483E-1,-5.2724457E-1,-4.1193073E-3,3.0255388E-4,-3.8251507E-3,-9.077688E-3,-3.0601488E-3,7.1193394E-3,-2.1336779E-1,-1.4319568E-2,-2.2659957E-1,-1.3321639E-2,-2.3515314E-2,-9.7803045E-3,-9.132369E-4,1.3257323E-3,-1.9603346E-3,-9.4206715E-3,-1.3889873E-1,-1.0616752E-2,-8.6137295E-2,4.366113E-3,-1.7065549E-3,-1.9266915E-1,-4.539784E-3,1.2939057E-4,-3.3494076E-3,-7.5037763E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,35,37,39,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,43,45,-1,47,49,-1,-1,-1,51,53,-1,-1,55,57,59,-1,61,63,-1,-1,-1,-1,-1,65,67,69,-1,-1,-1,-1,-1,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,79,-1,81,-1,-1,83,-1,-1,-1,-1],"loss_changes":[9.9702736E1,2.0246489E1,8.948425E0,4.125366E0,4.758564E0,2.1659355E0,1.4097252E0,1.1733055E0,3.1568858E-1,3.3422503E0,5.377773E0,3.2732815E-1,9.550419E-1,3.9839935E-1,0E0,1.2960052E-1,1.7139316E-3,0E0,0E0,1.3247843E0,4.080882E-1,4.5658746E-1,1.5924644E-1,0E0,0E0,0E0,5.167618E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2107776E-1,9.4067574E-1,0E0,3.0073154E-1,2.2187948E-1,0E0,0E0,0E0,1.7609835E-2,1.1637592E-1,0E0,0E0,2.8989983E-1,8.9917755E-1,4.4300973E-2,0E0,1.1708677E-2,2.8276257E-2,0E0,0E0,0E0,0E0,0E0,1.5747929E-1,2.9411483E-1,1.8678546E-1,0E0,0E0,0E0,0E0,0E0,1.2049844E-2,7.890132E-2,0E0,1.2619531E-1,1.3099785E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.520023E-2,0E0,4.016255E-2,0E0,0E0,8.327365E-4,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,26,26,33,33,34,34,36,36,37,37,41,41,42,42,45,45,46,46,47,47,49,49,50,50,56,56,57,57,58,58,64,64,65,65,67,67,68,68,75,75,77,77,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,36,38,40,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,44,46,-1,48,50,-1,-1,-1,52,54,-1,-1,56,58,60,-1,62,64,-1,-1,-1,-1,-1,66,68,70,-1,-1,-1,-1,-1,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,80,-1,82,-1,-1,84,-1,-1,-1,-1],"split_conditions":[3.47989E5,7.97369E5,4.6908E4,1.83E2,7.164074E8,8.511E3,2.3177024E7,2.8681312E8,4.027E3,1.521867E6,1.83E2,4.4748E5,4.689138E6,2.054214E0,1.768156E-2,6.591E3,3.0178509E0,-6.3509964E-3,6.437597E-3,2.9169E4,1.73E2,1.3925089E9,3.28451E5,-7.312535E-4,1.0374578E-2,1.1033022E-2,7.502168E8,5.1780824E-2,3.4825463E-2,-2.9693663E-2,-1.360752E-2,-6.1942074E-3,-1.3815725E-2,2.90721E5,1.835475E6,9.351397E-3,2.402988E6,2.83326E9,6.9173407E-3,3.2131847E-2,1.3423062E-2,3.1E1,8.501854E10,-3.7917523E-3,9.270689E-3,1.86009E5,1.13472E5,9.310384E6,5.4067746E-3,7.721E3,8.656044E6,9.982757E-3,2.0192485E-2,3.342443E-2,1.5192855E-2,-2.0286975E-2,7.2318E4,2.34E3,1.6E2,-4.1193073E-3,3.0255388E-4,-3.8251507E-3,-9.077688E-3,-3.0601488E-3,1.3818E4,3.32387E-1,-1.4319568E-2,1.4828954E10,1.83E2,-2.3515314E-2,-9.7803045E-3,-9.132369E-4,1.3257323E-3,-1.9603346E-3,-9.4206715E-3,4.5839E4,-1.0616752E-2,4.890295E6,4.366113E-3,-1.7065549E-3,9.39E2,-4.539784E-3,1.2939057E-4,-3.3494076E-3,-7.5037763E-3],"split_indices":[12,9,3,47,16,3,18,16,28,6,47,12,9,59,0,3,67,0,0,12,17,16,6,0,0,0,16,0,0,0,0,0,0,6,9,0,6,10,0,0,0,4,13,0,0,18,22,18,0,3,21,0,0,0,0,0,12,0,17,0,0,0,0,0,7,63,0,10,47,0,0,0,0,0,0,22,0,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.87E2,2.09E2,7.8E1,7.4E1,1.35E2,4.3E1,3.5E1,6.8E1,6E0,1.04E2,3.1E1,1.1E1,3.2E1,3.1E1,4E0,6.2E1,6E0,3E0,3E0,8.1E1,2.3E1,2.3E1,8E0,4E0,7E0,7E0,2.5E1,1.6E1,1.5E1,5.9E1,3E0,3E0,3E0,6E0,7.5E1,6E0,1.7E1,1.9E1,4E0,5E0,3E0,9E0,1.6E1,3E0,3E0,3.9E1,3.6E1,1.1E1,6E0,8E0,1.1E1,4E0,5E0,1.3E1,3E0,1.3E1,2.6E1,2.8E1,8E0,8E0,3E0,3E0,5E0,3E0,8E0,7E0,1.9E1,1.8E1,1E1,3E0,5E0,4E0,4E0,3E0,4E0,1.1E1,7E0,7E0,3E0,5E0,6E0,4E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-6.5471E-2,-4.24095E-1,9.337454E-1,-7.2517115E-1,-1.747601E-1,4.7130606E-1,1.3366703E0,-8.0688936E-1,-1.0633264E-1,-2.6990956E-1,8.870164E-2,8.8269174E-1,2.791311E-1,3.4056965E-2,1.602968E0,-3.045858E-2,-6.025417E-1,-9.949606E-3,4.762399E-3,-3.6404744E-1,2.9839626E-2,3.0334327E-1,-1.3148382E-1,3.2464832E-2,1.32058915E-2,1.2041326E-4,3.445221E-1,5.82543E-2,3.2815725E-2,-2.1512404E-2,-3.3096066E-1,1.0499822E-1,-4.234916E-1,2.357029E-1,-1.7873634E-1,1.4749965E-2,1.9128263E-1,-1.494119E-2,6.1935138E-2,3.88969E-1,4.007468E-3,-4.787035E-3,-1.2788562E-2,-1.8298492E-3,9.046879E-3,-4.9381936E-1,-1.474731E-1,3.6559302E-1,2.1618618E-4,-9.514781E-3,-1.033004E-3,5.4670614E-4,9.285078E-3,-7.5936936E-2,7.808044E-3,4.772548E-3,1.3839887E-2,-2.6487893E-1,-5.361611E-1,-2.3302066E-1,1.1590326E-3,5.589818E-3,1.5004287E-2,-3.2019366E-3,-1.0756424E-3,-2.624222E-3,-1.1039783E-2,-5.771913E-1,-1.0148463E-2,-9.22448E-3,-3.039838E-3,-1.4177053E-2,-6.8458825E-1,-1.102236E-2,-2.3871109E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,-1,-1,31,33,35,37,-1,-1,-1,39,-1,-1,-1,41,43,45,47,49,-1,51,-1,53,55,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,-1,-1,63,-1,-1,-1,65,67,69,-1,-1,-1,-1,-1,-1,-1,71,-1,-1,-1,-1,73,-1,-1],"loss_changes":[1.0037369E2,1.53374405E1,1.315831E1,4.6172447E0,2.8817208E0,2.5997028E0,1.1978302E0,1.7649307E0,6.644574E-1,2.381154E0,1.5049747E0,2.4038029E-1,4.6692014E-1,0E0,3.26931E-1,0E0,4.218855E-1,0E0,0E0,1.8289156E0,9.4382954E-1,1.9005668E-1,1.0583175E0,0E0,0E0,0E0,1.5694094E-1,0E0,0E0,0E0,4.9719274E-2,2.4392933E-1,1.0336456E0,3.2471895E-1,1.7901161E-1,0E0,1.8140835E-1,0E0,2.92368E-1,1.0397434E-1,0E0,0E0,0E0,0E0,0E0,2.942009E-1,2.1244213E-1,5.4132342E-2,0E0,0E0,0E0,0E0,0E0,3.681343E-3,0E0,0E0,0E0,1.0090047E-1,1.5335083E-1,3.9729774E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3911057E-1,0E0,0E0,0E0,0E0,3.863144E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,19,19,20,20,21,21,22,22,26,26,30,30,31,31,32,32,33,33,34,34,36,36,38,38,39,39,45,45,46,46,47,47,53,53,57,57,58,58,59,59,67,67,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,-1,-1,32,34,36,38,-1,-1,-1,40,-1,-1,-1,42,44,46,48,50,-1,52,-1,54,56,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,-1,-1,64,-1,-1,-1,66,68,70,-1,-1,-1,-1,-1,-1,-1,72,-1,-1,-1,-1,74,-1,-1],"split_conditions":[1.0259E4,8.81041E5,1.4066751E7,1.83E2,3.987307E6,4.94E4,2.12632E5,3.958E3,1.0241836E-1,3.904915E8,7.192367E10,2.9663E4,5.1012262E8,3.4056965E-2,1.3887953E0,-3.045858E-2,5.0373805E8,-9.949606E-3,4.762399E-3,3.2416E4,1.05E2,1.0382E4,1.33E2,3.2464832E-2,1.32058915E-2,1.2041326E-4,2.7179317E10,5.82543E-2,3.2815725E-2,-2.1512404E-2,1.41048E5,3.24654E5,1.203673E6,1.1234E4,2.83326E9,1.4749965E-2,5.018E6,-1.494119E-2,2.4866E4,1.3323E4,4.007468E-3,-4.787035E-3,-1.2788562E-2,-1.8298492E-3,9.046879E-3,1E0,6.9252394E-2,1.199E3,2.1618618E-4,-9.514781E-3,-1.033004E-3,5.4670614E-4,9.285078E-3,7.069832E6,7.808044E-3,4.772548E-3,1.3839887E-2,4.283E3,1.59577E5,5.5293235E9,1.1590326E-3,5.589818E-3,1.5004287E-2,-3.2019366E-3,-1.0756424E-3,-2.624222E-3,-1.1039783E-2,8.504848E7,-1.0148463E-2,-9.22448E-3,-3.039838E-3,-1.4177053E-2,9.75341E5,-1.102236E-2,-2.3871109E-2],"split_indices":[3,9,2,47,2,8,3,3,67,16,10,3,16,0,59,0,16,0,0,12,4,7,4,0,0,0,10,0,0,0,6,6,6,20,10,0,2,0,0,7,0,0,0,0,0,19,63,0,0,0,0,0,0,2,0,0,0,3,12,11,0,0,0,0,0,0,0,16,0,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,2.05E2,7.3E1,9.2E1,1.13E2,3.5E1,3.8E1,8.1E1,1.1E1,8.3E1,3E1,1E1,2.5E1,2.1E1,1.7E1,4.6E1,3.5E1,6E0,5E0,6.3E1,2E1,1.5E1,1.5E1,7E0,3E0,5E0,2E1,1.1E1,6E0,2.7E1,8E0,7E0,5.6E1,1E1,1E1,5E0,1E1,5E0,1E1,1.6E1,4E0,3E0,5E0,4E0,3E0,4.4E1,1.2E1,6E0,4E0,5E0,5E0,4E0,6E0,6E0,4E0,3E0,1.3E1,8E0,3.6E1,8E0,4E0,3E0,3E0,3E0,3E0,3E0,5E0,2.9E1,7E0,5E0,3E0,1.5E1,1.4E1,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.9427562E-2,-3.8867658E-1,8.0867213E-1,-7.1420634E-1,-1.8350904E-1,5.193958E-1,1.2480946E0,-8.177964E-1,1.1342196E-1,2.1029133E-1,-2.9155424E-1,3.6898378E-1,3.1432744E-2,5.3567797E-2,3.3956856E-2,-8.455719E-1,-8.721133E-3,-1.6744566E-3,9.311217E-3,-5.4281168E-2,8.0426145E-1,-4.4295993E-1,-1.05685964E-1,6.485523E-1,2.733105E-1,-2.9013157E-2,-2.1733467E-2,-2.7593586E-1,2.1713257E-1,1.139899E-2,3.1233182E-2,-5.044542E-1,-1.3994268E-1,-6.267783E-2,-1.8125797E-2,2.4483752E-2,8.442109E-3,-3.1626378E-3,3.3640942E-1,-1.33017525E-2,-1.1882753E-1,-5.6426594E-4,1.0307111E-2,-5.465505E-1,-4.046284E-3,-6.094399E-3,-1.9887679E-4,-1.4913614E-1,2.5483987E-1,1.3493203E-1,4.006219E-1,-1.00958656E-4,-6.5926765E-3,-4.8468085E-3,-5.7314944E-1,-2.0333298E-1,1.2108931E-2,9.880407E-3,3.2168631E-3,-2.1324917E-3,7.936643E-3,3.0497086E-1,5.155268E-1,-5.112857E-1,-2.8698562E-2,-2.8384286E-1,-7.904204E-2,2.5069602E-3,-2.2288044E-3,3.6409225E-3,1.1057227E-2,7.801109E-3,1.9690668E-2,-1.714963E-2,-8.876582E-3,-3.4447014E-1,-2.6256165E-3,2.1926371E-5,-1.0569213E-1,-1.2272637E-2,-5.650973E-3,-1.5595505E-3,-4.564265E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,-1,-1,27,29,31,33,35,37,-1,-1,39,41,-1,-1,43,45,47,-1,-1,-1,-1,49,-1,51,-1,-1,53,-1,-1,-1,55,57,59,61,-1,-1,-1,63,65,67,-1,-1,-1,-1,69,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,79,-1,-1,81,-1,-1,-1,-1],"loss_changes":[9.418723E1,1.40948105E1,1.1427631E1,7.1229515E0,5.677679E0,3.69623E0,1.2182541E0,8.797264E-1,3.0596906E-1,4.6009526E0,2.8799028E0,1.0531168E0,0E0,0E0,0E0,4.7073364E-2,0E0,0E0,0E0,1.3232774E0,3.290739E-1,9.9153423E-1,9.089993E-1,2.7672815E-1,8.770802E-1,0E0,0E0,2.0975453E-1,2.4748874E-1,0E0,0E0,6.8066406E-1,7.1501955E-2,1.2733479E0,0E0,0E0,0E0,0E0,3.4700155E-1,0E0,6.899088E-2,0E0,0E0,3.5606575E-1,0E0,0E0,0E0,3.1706524E-1,5.0044775E-2,2.184592E-1,9.5567465E-2,0E0,0E0,0E0,3.123932E-1,2.4774396E-1,5.8899708E-2,0E0,0E0,0E0,0E0,7.205677E-2,8.8246584E-2,3.0259132E-2,0E0,1.6811514E-1,2.5567569E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.6549053E-3,0E0,0E0,1.173E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,31,31,32,32,33,33,38,38,40,40,43,43,47,47,48,48,49,49,50,50,54,54,55,55,56,56,61,61,62,62,63,63,65,65,66,66,75,75,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,-1,-1,28,30,32,34,36,38,-1,-1,40,42,-1,-1,44,46,48,-1,-1,-1,-1,50,-1,52,-1,-1,54,-1,-1,-1,56,58,60,62,-1,-1,-1,64,66,68,-1,-1,-1,-1,70,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,80,-1,-1,82,-1,-1,-1,-1],"split_conditions":[1.0126E4,8.03052E5,6.6569E4,1.83E2,9.603E3,4.0663186E3,5.5409517E0,4.3199274E8,1.932E3,2.4472736E7,1.46379E5,5.1889E4,3.1432744E-2,5.3567797E-2,3.3956856E-2,3.958E3,-8.721133E-3,-1.6744566E-3,9.311217E-3,1E0,2.5735368E7,1.080557E6,9.45839E5,1.1988E4,1.566327E3,-2.9013157E-2,-2.1733467E-2,1.0500989E10,3.676E3,1.139899E-2,3.1233182E-2,6.451668E8,1.492234E6,2.594684E6,-1.8125797E-2,2.4483752E-2,8.442109E-3,-3.1626378E-3,1.37E4,-1.33017525E-2,2.25E2,-5.6426594E-4,1.0307111E-2,2.036E3,-4.046284E-3,-6.094399E-3,-1.9887679E-4,7.164074E8,1.83E2,1.5551E4,1.023E3,-1.00958656E-4,-6.5926765E-3,-4.8468085E-3,4.718E3,1.8723524E3,2.9911E4,9.880407E-3,3.2168631E-3,-2.1324917E-3,7.936643E-3,3.0174216E7,1.5559E4,8.17992E5,-2.8698562E-2,1.460109E6,2.963893E9,2.5069602E-3,-2.2288044E-3,3.6409225E-3,1.1057227E-2,7.801109E-3,1.9690668E-2,-1.714963E-2,-8.876582E-3,4.579412E8,-2.6256165E-3,2.1926371E-5,7.8E1,-1.2272637E-2,-5.650973E-3,-1.5595505E-3,-4.564265E-3],"split_indices":[3,9,3,47,7,5,60,16,3,18,12,8,0,0,0,3,0,0,0,19,18,6,12,7,5,0,0,10,3,0,0,16,6,6,0,0,0,0,7,0,4,0,0,22,0,0,0,16,17,3,17,0,0,0,0,5,28,0,0,0,0,27,7,6,0,6,10,0,0,0,0,0,0,0,0,16,0,0,19,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[3.06E2,2.12E2,9.4E1,8.1E1,1.31E2,5.8E1,3.6E1,7.2E1,9E0,2.8E1,1.03E2,4.5E1,1.3E1,9E0,2.7E1,6.8E1,4E0,5E0,4E0,2E1,8E0,5.6E1,4.7E1,1E1,3.5E1,4.8E1,2E1,1.1E1,9E0,3E0,5E0,4.6E1,1E1,4.4E1,3E0,7E0,3E0,5E0,3E1,5E0,6E0,3E0,6E0,4.1E1,5E0,7E0,3E0,3.5E1,9E0,8E0,2.2E1,3E0,3E0,3E0,3.8E1,2.6E1,9E0,6E0,3E0,3E0,5E0,1.4E1,8E0,3.4E1,4E0,1.5E1,1.1E1,5E0,4E0,3E0,1.1E1,3E0,5E0,3E1,4E0,1.1E1,4E0,3E0,8E0,8E0,3E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[2.4894243E-2,-2.9520097E-1,1.137211E0,-7.1533906E-1,-6.2342007E-2,9.3051255E-1,1.5062977E0,-8.063898E-1,7.231549E-2,-1.9233125E-1,2.984794E-1,1.4361169E-2,3.2584902E-2,5.606283E-2,3.4545537E-2,-8.8389635E-1,-5.684286E-1,-6.8544005E-3,8.061242E-3,-3.4677008E-1,2.1293188E-2,2.8632002E-2,8.438764E-2,-2.9039795E-2,-1.4281674E-2,-8.624442E-3,-1.9496748E-2,9.687732E-2,-3.9692938E-1,-6.9562696E-2,3.4624505E-1,-9.524597E-2,3.1280178E-1,-3.1224748E-3,8.57964E-3,-4.4576383E-1,-1.8479232E-1,-1.7670779E-2,-1.5459725E-2,1.857662E-2,1.6485776E-1,8.116562E-2,-2.2685064E-1,1.1166075E-2,4.8082927E-3,-4.548527E-3,-4.6459466E-1,-7.3245284E-3,-2.5566078E-3,-2.1152118E-1,1.5562241E-1,2.517457E-3,6.7690928E-3,4.9820314E-3,-1.0022656E-3,-8.887209E-3,-2.7025992E-3,-1.6293844E-2,-8.383412E-3,-3.2707387E-1,-1.6228592E-2,1.1280536E-2,7.4926175E-2,-3.5524787E-3,-4.041837E-1,1.8769819E-3,-2.7911495E-3,-2.9993579E-3,1.7867163E-1,-1.5924223E-2,-6.843718E-3,7.4868333E-3,1.7100718E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,23,25,-1,-1,27,29,-1,31,-1,-1,-1,-1,33,35,37,39,41,43,-1,-1,45,47,49,-1,-1,51,53,55,-1,-1,-1,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,63,65,-1,67,-1,69,-1,-1,-1,71,-1,-1,-1,-1],"loss_changes":[1.0038499E2,2.1380814E1,3.4046173E0,5.6522026E0,6.7176194E0,1.2518082E0,2.7156067E-1,7.886162E-1,5.106185E-1,3.471534E0,4.733158E0,0E0,0E0,0E0,0E0,9.899139E-3,6.1694622E-2,0E0,0E0,1.3958898E0,1.3526562E0,0E0,1.2196877E0,0E0,0E0,0E0,0E0,2.5611696E-1,4.8740482E-1,7.5877845E-1,3.2362115E-1,4.1309446E-1,2.0606756E-2,0E0,0E0,1.8660927E-1,3.5987467E-2,1.1414849E0,0E0,0E0,1.1111379E-2,7.095097E-2,4.718423E-2,0E0,0E0,0E0,2.1635532E-1,0E0,0E0,3.5575742E-1,2.5673184E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.22498155E-1,4.1832753E-2,0E0,2.608117E-1,0E0,1.6216874E-2,0E0,0E0,0E0,4.856643E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,15,15,16,16,19,19,20,20,22,22,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,40,40,41,41,42,42,46,46,49,49,50,50,59,59,60,60,62,62,64,64,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,24,26,-1,-1,28,30,-1,32,-1,-1,-1,-1,34,36,38,40,42,44,-1,-1,46,48,50,-1,-1,52,54,56,-1,-1,-1,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,64,66,-1,68,-1,70,-1,-1,-1,72,-1,-1,-1,-1],"split_conditions":[1.9383872E9,7.97369E5,2.12632E5,1.83E2,7.164074E8,1.0431399E7,1.7754914E0,4.058E3,1.766E3,2.597379E6,9.603E3,1.4361169E-2,3.2584902E-2,5.606283E-2,3.4545537E-2,7.03651E5,1.18399E5,-6.8544005E-3,8.061242E-3,2.9169E4,2.8893403E3,2.8632002E-2,1.2084881E9,-2.9039795E-2,-1.4281674E-2,-8.624442E-3,-1.9496748E-2,3.24654E5,1.77451E5,1.33097E5,5.019054E8,7.1E1,3.6E1,-3.1224748E-3,8.57964E-3,4.3E1,4.579412E8,2.466E3,-1.5459725E-2,1.857662E-2,3.7742935E3,6.5878E4,2.83326E9,1.1166075E-2,4.8082927E-3,-4.548527E-3,8.17992E5,-7.3245284E-3,-2.5566078E-3,1.578606E6,3.51E3,2.517457E-3,6.7690928E-3,4.9820314E-3,-1.0022656E-3,-8.887209E-3,-2.7025992E-3,-1.6293844E-2,-8.383412E-3,5.8E1,3.23E2,1.1280536E-2,1.914442E6,-3.5524787E-3,1.63E2,1.8769819E-3,-2.7911495E-3,-2.9993579E-3,3.3831064E7,-1.5924223E-2,-6.843718E-3,7.4868333E-3,1.7100718E-3],"split_indices":[27,9,3,47,16,2,59,3,3,9,7,0,0,0,0,2,6,0,0,12,5,0,16,0,0,0,0,6,12,8,26,4,19,0,0,22,16,0,0,0,5,8,10,0,0,0,6,0,0,6,0,0,0,0,0,0,0,0,0,4,17,0,6,0,17,0,0,0,16,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.18E2,6.2E1,7.7E1,1.41E2,4.2E1,2E1,6.9E1,8E0,1.04E2,3.7E1,7E0,3.5E1,1.1E1,9E0,5E1,1.9E1,3E0,5E0,6E1,4.4E1,9E0,2.8E1,4.7E1,3E0,3E0,1.6E1,6E0,5.4E1,3.5E1,9E0,1.6E1,1.2E1,3E0,3E0,4.3E1,1.1E1,3.2E1,3E0,3E0,6E0,7E0,9E0,9E0,3E0,3E0,4E1,7E0,4E0,1.5E1,1.7E1,3E0,3E0,4E0,3E0,6E0,3E0,3.2E1,8E0,9E0,6E0,4E0,1.3E1,3E0,6E0,3E0,3E0,5E0,8E0,3E0,3E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-9.678331E-3,-3.9076748E-1,7.612711E-1,-7.760011E-1,-2.5848183E-1,1.3036053E-1,9.4491595E-1,-2.75185E-2,-3.6539638E-1,-3.298985E-1,8.2590364E-2,-9.738542E-2,1.1145605E-2,5.258759E-1,1.1085602E0,-1.3548808E-2,-6.140679E-3,-4.5248884E-1,-1.9211818E-1,2.4616615E-1,-1.3310522E-1,-6.608648E-3,-6.385718E-3,5.3344755E-3,1.9341351E-2,1.1531873E0,1.0601928E-2,2.202822E-2,-5.640147E-1,-2.1748346E-1,5.7720826E-3,1.4281644E-2,1.2279191E-1,-1.0651782E-2,-1.75626E-2,-1.441136E-3,1.0814242E-3,8.902001E-1,4.039739E-2,-8.23782E-3,1.431135E-1,-2.0533394E-2,-4.2155465E-1,-2.6040018E-1,-1.5484966E-2,6.0341037E-3,-6.7832234E-4,-2.1419176E-3,1.5467532E-3,3.7425004E-2,6.1045396E-1,1.4029278E-3,5.974726E-3,-1.4530777E-2,-6.331572E-3,-5.9771836E-3,-3.0263796E-1,-3.0072972E-3,4.016658E-3,2.2166274E-2,1.0333714E-2,-2.8826233E-3,2.5459246E-3,-3.6105055E-1,-8.72835E-2,-2.6632443E-1,-2.1373218E-2,1.2252887E-3,-5.0312066E-3,-1.0217237E-2,-2.21033E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,25,-1,-1,27,29,31,33,-1,35,-1,-1,37,-1,39,41,43,-1,-1,45,-1,47,-1,-1,49,-1,-1,51,-1,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,61,63,-1,-1,-1,-1,-1,-1,65,67,69,-1,-1,-1,-1,-1],"loss_changes":[7.81546E1,8.870455E0,1.0072811E1,1.145544E0,3.2912502E0,1.0662737E0,4.0800285E0,0E0,6.956935E-3,1.7879181E0,8.795478E-1,1.0671903E-1,0E0,4.6761084E-1,1.2892914E0,0E0,0E0,3.0818386E0,5.477654E-1,2.8667563E-1,2.455973E-1,0E0,1.224717E-2,0E0,0E0,8.778E-2,0E0,4.404919E-1,2.258091E-1,4.3809247E-1,0E0,0E0,9.69834E-2,0E0,2.8907714E-2,0E0,0E0,5.097866E-1,0E0,0E0,2.9980734E-2,0E0,3.880191E-2,4.4953418E-1,1.2098544E-1,0E0,0E0,0E0,0E0,0E0,5.1841736E-3,0E0,0E0,0E0,0E0,5.685187E-2,4.2134476E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.266644E-1,8.3812594E-2,2.1046448E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19,20,20,22,22,25,25,27,27,28,28,29,29,32,32,34,34,37,37,40,40,42,42,43,43,44,44,50,50,55,55,56,56,63,63,64,64,65,65],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,26,-1,-1,28,30,32,34,-1,36,-1,-1,38,-1,40,42,44,-1,-1,46,-1,48,-1,-1,50,-1,-1,52,-1,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,62,64,-1,-1,-1,-1,-1,-1,66,68,70,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,4.96E2,4.764156E6,6.78647E5,6.490568E8,1.833321E7,8.220982E8,-2.75185E-2,5.86E3,1.530129E6,1.05E2,2.654028E6,1.1145605E-2,2.0830981E3,5.11E2,-1.3548808E-2,-6.140679E-3,2.4095E4,1.51405E5,9.6671375E2,1.7140013E3,-6.608648E-3,1.333505E6,5.3344755E-3,1.9341351E-2,4.148E4,1.0601928E-2,3.443128E7,1.1374049E3,3.1784865E-1,5.7720826E-3,1.4281644E-2,3.633747E9,-1.0651782E-2,8.3344527E9,-1.441136E-3,1.0814242E-3,3.4555535E9,4.039739E-2,-8.23782E-3,3.3E1,-2.0533394E-2,2.4283556E7,5.798156E2,2.965665E6,6.0341037E-3,-6.7832234E-4,-2.1419176E-3,1.5467532E-3,3.7425004E-2,4.5323792E8,1.4029278E-3,5.974726E-3,-1.4530777E-2,-6.331572E-3,3.764E3,2.2317708E3,-3.0072972E-3,4.016658E-3,2.2166274E-2,1.0333714E-2,-2.8826233E-3,2.5459246E-3,1.83E2,2.249942E3,2.1504E4,-2.1373218E-2,1.2252887E-3,-5.0312066E-3,-1.0217237E-2,-2.21033E-3],"split_indices":[3,0,2,2,16,18,16,0,3,9,4,2,0,5,4,0,0,12,20,5,5,0,6,0,0,3,0,16,5,63,0,0,23,0,10,0,0,10,0,0,4,0,18,5,2,0,0,0,0,0,1,0,0,0,0,3,5,0,0,0,0,0,0,47,5,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,1.77E2,8.7E1,4.4E1,1.33E2,2E1,6.7E1,3.6E1,8E0,1.1E2,2.3E1,1E1,1E1,2E1,4.7E1,5E0,3E0,5.7E1,5.3E1,1.3E1,1E1,4E0,6E0,4E0,1.6E1,4.4E1,3E0,1.1E1,4.6E1,5E1,3E0,4E0,9E0,3E0,7E0,3E0,3E0,1.5E1,2.9E1,3E0,8E0,2.8E1,1.8E1,4.1E1,9E0,6E0,3E0,4E0,3E0,6E0,9E0,3E0,5E0,1.5E1,3E0,6E0,3.5E1,6E0,3E0,6E0,3E0,3E0,3E0,2.7E1,8E0,2.2E1,5E0,3E0,5E0,1.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[3.5845062E-3,-3.3584887E-1,7.7824914E-1,-6.0576546E-1,-1.451604E-1,5.2345705E-1,1.1450415E0,-7.3513144E-1,1.8856938E-1,-2.0250371E-1,2.8299516E-1,-5.6975614E-2,6.1501265E-1,1.4372507E-2,1.1982385E0,-7.8015643E-1,-2.942965E-1,-7.0650056E-2,1.4792882E-2,-2.9591E-1,3.1535067E-2,4.4401947E-1,5.853952E-2,-1.3692103E-2,8.019307E-3,6.6259074E-1,-1.233176E-3,4.098708E-2,2.16679E-2,-2.5638707E-2,-1.3985048E-2,-2.7535069E-3,-1.2954152E-2,2.585949E-3,-6.5657143E-3,-3.606404E-1,-6.146629E-2,-4.3387894E-2,1.8571977E-1,1.858257E-2,8.001984E-3,-2.9459784E-3,6.2435493E-3,2.7252576E-1,7.6071376E-1,8.519385E-4,-3.9035544E-1,6.839641E-3,-1.3534737E-1,-1.343774E-1,7.0138305E-2,1.9430926E-3,8.3443E-3,1.0700821E-2,3.204364E-3,4.4855046E-1,2.9540878E-2,-4.6726942E-1,-2.9251128E-1,-1.9471398E-1,-7.64152E-4,-8.2905457E-4,-1.677228E-1,1.3286532E-1,-1.840196E-3,1.7195439E-2,9.618683E-3,-1.589353E-2,-5.9518567E-3,-1.99906E-1,-1.3541546E-2,-7.964791E-3,-2.1550045E-3,-7.056037E-3,-3.3942095E-3,5.7161856E-3,1.768208E-3,-8.482758E-3,-1.2901616E-1,-1.2729442E-3,-5.205312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,49,51,-1,-1,-1,-1,53,55,-1,57,-1,59,61,63,-1,-1,-1,-1,65,-1,67,69,71,-1,-1,73,75,-1,-1,-1,-1,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,79,-1,-1],"loss_changes":[7.336065E1,9.8972435E0,7.344368E0,8.359413E0,2.8807712E0,2.79842E0,7.2842026E-1,1.1399574E0,8.754326E-1,2.2391224E0,4.800067E-1,1.008111E0,1.4241581E0,0E0,3.6515045E-1,4.3640137E-3,1.4617938E-1,1.5729758E-1,0E0,1.0856538E0,3.546172E-1,6.485307E-2,1.6000709E-1,0E0,0E0,1.4002094E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6599655E-1,3.7283486E-1,2.2634855E-1,7.649833E-2,0E0,0E0,0E0,0E0,7.054019E-2,1.2058697E0,0E0,2.6217556E-1,0E0,8.813846E-2,3.91445E-2,8.7451495E-2,0E0,0E0,0E0,0E0,7.0929527E-3,0E0,1.1812067E-1,2.245202E-1,4.406178E-2,0E0,0E0,5.7415366E-3,1.4639527E-2,0E0,0E0,0E0,0E0,0E0,4.2675495E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.285719E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,35,35,36,36,37,37,38,38,43,43,44,44,46,46,48,48,49,49,50,50,55,55,57,57,58,58,59,59,62,62,63,63,69,69,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,50,52,-1,-1,-1,-1,54,56,-1,58,-1,60,62,64,-1,-1,-1,-1,66,-1,68,70,72,-1,-1,74,76,-1,-1,-1,-1,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,80,-1,-1],"split_conditions":[3.47989E5,8.03052E5,6.117756E9,1.83E2,1.0737418E9,2.917845E8,1.6918336E7,3.1361875E8,5.876475E7,2.989734E6,9.5E1,6.162E3,2.62E2,1.4372507E-2,1.47626E5,5.58E3,6.7E2,3.185029E2,1.4792882E-2,6.490568E8,7.0507E4,9.603E3,1.340499E6,-1.3692103E-2,8.019307E-3,4.112304E6,-1.233176E-3,4.098708E-2,2.16679E-2,-2.5638707E-2,-1.3985048E-2,-2.7535069E-3,-1.2954152E-2,2.585949E-3,-6.5657143E-3,2.4095E4,5.728E3,5.601505E6,8E1,1.858257E-2,8.001984E-3,-2.9459784E-3,6.2435493E-3,2.318509E6,7.502168E8,8.519385E-4,1.486961E6,6.839641E-3,8.054554E8,1.296E3,1E0,1.9430926E-3,8.3443E-3,1.0700821E-2,3.204364E-3,5.397065E8,2.9540878E-2,6.91032E5,8.256E4,2.83326E9,-7.64152E-4,-8.2905457E-4,2.259E3,6.224E3,-1.840196E-3,1.7195439E-2,9.618683E-3,-1.589353E-2,-5.9518567E-3,1.2126E4,-1.3541546E-2,-7.964791E-3,-2.1550045E-3,-7.056037E-3,-3.3942095E-3,5.7161856E-3,1.768208E-3,-8.482758E-3,6.24933E5,-1.2729442E-3,-5.205312E-3],"split_indices":[12,9,27,47,16,16,2,16,16,9,4,20,4,0,19,3,0,5,0,16,28,7,2,0,0,2,0,0,0,0,0,0,0,0,0,12,3,2,4,0,0,0,0,1,16,0,9,0,23,0,44,0,0,0,0,16,0,6,22,10,0,0,0,0,0,0,0,0,0,28,0,0,0,0,0,0,0,0,18,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,1.93E2,8.4E1,7.9E1,1.14E2,5.1E1,3.3E1,6.8E1,1.1E1,1.01E2,1.3E1,7E0,4.4E1,3E0,3E1,6.1E1,7E0,6E0,5E0,7.2E1,2.9E1,7E0,6E0,3E0,4E0,4.1E1,3E0,2.5E1,5E0,5.7E1,4E0,3E0,4E0,3E0,3E0,5.6E1,1.6E1,2E1,9E0,3E0,4E0,3E0,3E0,9E0,3.2E1,4E0,5.2E1,3E0,1.3E1,1.1E1,9E0,4E0,5E0,6E0,3E0,1.2E1,2E1,2.7E1,2.5E1,8E0,5E0,3E0,8E0,6E0,3E0,6E0,6E0,2.4E1,3E0,1.6E1,9E0,5E0,3E0,3E0,5E0,3E0,3E0,7E0,9E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-2.3879929E-2,-3.3753425E-1,7.0824546E-1,-6.56427E-1,-1.197293E-1,4.131385E-1,1.1198003E0,-7.317648E-1,-1.821749E-2,4.0484697E-1,-1.9426526E-1,9.051118E-2,5.3490514E-1,1.4857712E0,8.324367E-1,-7.735544E-1,-4.4442418E-1,-9.090898E-3,1.353595E-1,2.0927134E-1,2.5534183E-2,-4.0472662E-1,-4.4636644E-2,-2.7085317E-2,9.367617E-3,3.265639E-1,6.9793266E-1,2.0245947E-2,5.303314E-2,2.9206758E-2,8.039018E-3,-7.894896E-1,-1.3662051E-2,-1.598426E-2,-6.9520716E-3,1.7474714E-4,7.4501447E-3,4.1794401E-4,3.0579197E-1,-5.070546E-1,-2.1738003E-1,-1.03389084E-1,2.5852457E-1,-3.860986E-3,3.4873148E-3,1.9014896E-3,3.7302646E-1,9.042805E-3,7.79526E-1,-2.5875943E-2,-1.2344788E-2,2.2236558E-3,1.3970076E-2,-1.7939776E-2,-1.0732221E-2,-2.5595817E-1,-5.91215E-4,-1.4643659E-1,1.3768531E-1,3.4845348E-3,3.289297E-1,2.5814056E-1,4.556448E-1,8.601557E-1,1.3325701E-2,-1.0691036E-2,-1.4778796E-1,4.1439976E-3,-1.6764256E-1,1.1146093E-3,6.8628956E-3,1.2985634E-2,5.5431984E-3,2.7173087E-3,1.06523745E-2,1.7614773E-2,7.314997E-3,3.0417932E-2,1.3939602E-2,-6.4489497E-3,-1.4530945E-3,-1.1126743E-1,-2.5395057E-1,-3.2538816E-2,-1.5099098E-1,-9.498002E-3,-3.0981696E-3,-5.8452018E-2,8.8269485E-4,-1.6989191E-1,-1.2115772E-3,-1.2842957E-4,-2.9076715E-3,-6.216713E-3,-3.1857418E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,-1,39,41,43,-1,45,47,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,51,53,55,57,59,-1,-1,-1,61,-1,63,-1,-1,-1,-1,-1,-1,65,-1,67,69,-1,71,73,75,77,-1,-1,79,-1,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,83,85,87,89,-1,-1,91,-1,93,-1,-1,-1,-1,-1],"loss_changes":[6.889971E1,1.4507257E1,1.0468109E1,4.073578E0,5.0052214E0,2.0729885E0,2.6543617E0,5.688133E-1,4.4398722E-1,1.0494163E0,3.4774394E0,3.851477E-1,1.0967302E0,1.0429993E0,7.7741814E-1,2.6527405E-2,3.5718203E-2,0E0,8.6139545E-2,2.2337878E-1,0E0,7.716336E-1,1.2022849E0,1.5132865E-1,0E0,2.141025E-1,4.9552822E-1,0E0,0E0,0E0,0E0,4.798889E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1280813E-1,6.7331314E-2,1.3349473E-1,6.013107E-1,8.0771685E-2,0E0,0E0,0E0,3.4246445E-2,0E0,1.4495945E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.4513376E-2,0E0,3.0168045E-1,6.2665015E-2,0E0,1.224643E-2,8.587444E-2,4.298091E-2,6.1742783E-2,0E0,0E0,3.4115046E-2,0E0,1.9069672E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.578092E-2,8.183801E-2,1.8694513E-2,3.431186E-2,0E0,0E0,1.3529321E-2,0E0,7.257253E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,21,21,22,22,23,23,25,25,26,26,31,31,38,38,39,39,40,40,41,41,42,42,46,46,48,48,55,55,57,57,58,58,60,60,61,61,62,62,63,63,66,66,68,68,81,81,82,82,83,83,84,84,87,87,89,89],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,-1,40,42,44,-1,46,48,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,52,54,56,58,60,-1,-1,-1,62,-1,64,-1,-1,-1,-1,-1,-1,66,-1,68,70,-1,72,74,76,78,-1,-1,80,-1,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,84,86,88,90,-1,-1,92,-1,94,-1,-1,-1,-1,-1],"split_conditions":[1.0773E4,7.97369E5,7.8908E4,1.83E2,2.0547E4,1.315E3,2.054214E0,5.58E3,3.443128E7,1.6942586E7,1.835475E6,1.0886205E9,7.9382246E8,2.8358776E7,2.51E2,5.101357E7,9.310384E6,-9.090898E-3,1.932E3,4.7E1,2.5534183E-2,2.6599069E8,2.5735368E7,7.2903E4,9.367617E-3,1.6671127E3,4.425011E6,2.0245947E-2,5.303314E-2,2.9206758E-2,8.039018E-3,2.73158E5,-1.3662051E-2,-1.598426E-2,-6.9520716E-3,1.7474714E-4,7.4501447E-3,4.1794401E-4,4.283E3,2.84E2,1.138E3,2.402988E6,2.124614E6,-3.860986E-3,3.4873148E-3,1.9014896E-3,5.96025E5,9.042805E-3,7.59577E5,-2.5875943E-2,-1.2344788E-2,2.2236558E-3,1.3970076E-2,-1.7939776E-2,-1.0732221E-2,1.089441E6,-5.91215E-4,2.2689E4,4.365884E6,3.4845348E-3,7.816172E7,1.9037409E9,7.385252E8,8.501854E10,1.3325701E-2,-1.0691036E-2,2.177237E9,4.1439976E-3,2.6304E4,1.1146093E-3,6.8628956E-3,1.2985634E-2,5.5431984E-3,2.7173087E-3,1.06523745E-2,1.7614773E-2,7.314997E-3,3.0417932E-2,1.3939602E-2,-6.4489497E-3,-1.4530945E-3,1.17E2,5.5293235E9,4.3406326E8,9.268E3,-9.498002E-3,-3.0981696E-3,1.59037E5,8.8269485E-4,2.414E3,-1.2115772E-3,-1.2842957E-4,-2.9076715E-3,-6.216713E-3,-3.1857418E-3],"split_indices":[3,9,3,47,8,0,59,3,16,18,9,16,16,2,4,18,18,0,3,4,0,16,18,22,0,5,2,0,0,0,0,6,0,0,0,0,0,0,3,4,17,6,2,0,0,0,12,0,28,0,0,0,0,0,0,2,0,21,6,0,18,11,16,13,0,0,10,0,20,0,0,0,0,0,0,0,0,0,0,0,0,4,11,16,3,0,0,12,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.98E2,2.09E2,8.9E1,8.4E1,1.25E2,5.3E1,3.6E1,7.5E1,9E0,1.5E1,1.1E2,1.5E1,3.8E1,1.4E1,2.2E1,6.4E1,1.1E1,3E0,6E0,1.1E1,4E0,4.5E1,6.5E1,1E1,5E0,1.8E1,2E1,3E0,1.1E1,1.9E1,3E0,6E1,4E0,8E0,3E0,3E0,3E0,4E0,7E0,2.8E1,1.7E1,5.5E1,1E1,6E0,4E0,3E0,1.5E1,4E0,1.6E1,5.7E1,3E0,3E0,4E0,2E1,8E0,1.4E1,3E0,4.7E1,8E0,4E0,6E0,8E0,7E0,1.2E1,4E0,7E0,7E0,3E0,4.4E1,4E0,4E0,3E0,3E0,3E0,5E0,4E0,3E0,9E0,3E0,4E0,3E0,2.8E1,1.6E1,1E1,1.8E1,1.2E1,4E0,7E0,3E0,1.5E1,3E0,3E0,4E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-9.957617E-3,-3.264823E-1,7.349234E-1,-5.2157855E-1,-6.1801407E-2,3.5852566E-1,9.788254E-1,-6.0899067E-1,1.8149355E-2,-1.487488E-1,2.3256241E-1,8.0902576E-1,2.3742132E-1,1.0129744E0,1.0439893E-2,-7.3892874E-1,-4.5553708E-1,-1.9240138E-1,8.982943E-3,-2.652291E-1,3.7512213E-3,2.126743E-2,-4.879984E-2,1.3216344E-2,3.2356627E-2,-4.737047E-3,3.0484146E-1,9.242147E-1,4.7647893E-2,-7.561448E-1,-1.1050462E-2,-5.1491773E-1,-1.7839882E-1,-8.738099E-3,-3.040251E-3,2.4336076E-3,-2.9660347E-1,8.1203505E-2,-1.6737921E-2,-1.1323191E-1,2.47655E-3,2.0877488E-1,1.1131924E-2,3.0646406E-2,1.3343191E-2,-1.114979E-2,-2.4957327E-2,-1.7766507E-2,-1.06278015E-2,-1.0195672E-2,7.498613E-4,-1.773039E-2,-2.460326E-1,4.6118796E-3,2.6253214E-1,-5.5209184E-3,-1.7866142E-3,9.981935E-3,-6.6777057E-4,-2.755012E-1,7.002964E-4,1.6941968E-1,-1.1937829E-1,1.0152193E-2,4.2110314E-3,-1.0161742E-1,-3.1830508E-1,9.085568E-3,7.306303E-4,-4.80046E-3,-7.2695396E-4,-1.713251E-3,-4.0109274E-3,-1.0990379E-2,-4.148198E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,37,-1,39,-1,-1,-1,41,43,-1,45,-1,47,49,-1,-1,-1,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,59,61,63,-1,-1,-1,-1,65,-1,67,69,-1,-1,71,73,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.719954E1,1.0295416E1,7.4524193E0,5.4366074E0,2.2356918E0,1.717433E0,8.113251E-1,1.6374016E0,9.876196E-1,1.1911328E0,2.4415097E0,1.3444471E-1,7.9679656E-1,6.227608E-1,0E0,9.459305E-2,6.980572E-1,5.180177E-2,0E0,4.2877388E-1,1.2591724E0,0E0,1.1403185E-1,0E0,0E0,0E0,2.6241064E-2,2.1600723E-1,0E0,1.16802216E-1,0E0,8.4548E-2,2.86414E-1,0E0,0E0,0E0,3.1444192E-1,3.7377214E-1,0E0,2.0761736E-2,0E0,2.3900473E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5062573E-1,4.2891598E-1,1.443857E-2,0E0,0E0,0E0,0E0,1.7605567E-1,0E0,1.4259458E-1,3.1193867E-2,0E0,0E0,1.1551306E-3,5.2387714E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,22,22,26,26,27,27,29,29,31,31,32,32,36,36,37,37,39,39,41,41,52,52,53,53,54,54,59,59,61,61,62,62,65,65,66,66],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,38,-1,40,-1,-1,-1,42,44,-1,46,-1,48,50,-1,-1,-1,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,60,62,64,-1,-1,-1,-1,66,-1,68,70,-1,-1,72,74,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1026E4,1.447831E6,9.683002E6,1.83E2,2.317698E7,5.345E3,5.3408066E4,4.222E3,1.64108E8,3.068292E6,5.4066E4,1.8943199E9,3.3587363E8,1.2738551E4,1.0439893E-2,7.7999923E9,6.910113E8,7.3508E4,8.982943E-3,3.342E3,9.45839E5,2.126743E-2,5.737332E9,1.3216344E-2,3.2356627E-2,-4.737047E-3,1.3366E4,1.6321E4,4.7647893E-2,1.5E1,-1.1050462E-2,5.26565E5,1.31E2,-8.738099E-3,-3.040251E-3,2.4336076E-3,3.967E3,2.594684E6,-1.6737921E-2,1.746735E6,2.47655E-3,9.096352E9,1.1131924E-2,3.0646406E-2,1.3343191E-2,-1.114979E-2,-2.4957327E-2,-1.7766507E-2,-1.06278015E-2,-1.0195672E-2,7.498613E-4,-1.773039E-2,2.414E3,1.6010206E10,1.1086E4,-5.5209184E-3,-1.7866142E-3,9.981935E-3,-6.6777057E-4,1.138E3,7.002964E-4,1.1087451E9,5.617E3,1.0152193E-2,4.2110314E-3,9.25E2,2.0408E4,9.085568E-3,7.306303E-4,-4.80046E-3,-7.2695396E-4,-1.713251E-3,-4.0109274E-3,-1.0990379E-2,-4.148198E-3],"split_indices":[3,9,2,47,18,7,5,3,23,2,8,16,16,5,0,10,16,27,0,7,12,0,10,0,0,0,3,0,0,28,0,6,4,0,0,0,7,6,0,2,0,10,0,0,0,0,0,0,0,0,0,0,0,10,0,0,0,0,0,0,0,21,3,0,0,0,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.99E2,8.4E1,1.14E2,8.5E1,3.4E1,5E1,9.8E1,1.6E1,6.6E1,1.9E1,6E0,2.8E1,4.7E1,3E0,5.1E1,4.7E1,9E0,7E0,3.7E1,2.9E1,7E0,1.2E1,3E0,3E0,4E0,2.4E1,4.2E1,5E0,4.8E1,3E0,3.8E1,9E0,4E0,5E0,3E0,3.4E1,2.6E1,3E0,8E0,4E0,9E0,1.5E1,3.9E1,3E0,3E0,4.5E1,3E1,8E0,5E0,4E0,4E0,3E1,1.9E1,7E0,3E0,5E0,6E0,3E0,2.7E1,3E0,8E0,1.1E1,4E0,3E0,6E0,2.1E1,4E0,4E0,8E0,3E0,3E0,3E0,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.4737593E-2,-3.3374155E-1,5.901943E-1,-4.2947096E-1,1.4640711E-1,3.5968474E-1,9.6438557E-1,-6.5913886E-1,-2.2001956E-1,-7.723147E-2,6.2377125E-1,8.4914106E-1,2.4706309E-1,1.0292754E0,1.1466129E-2,-6.9911045E-1,-1.8810159E-1,-3.5804477E-1,-1.1766796E-1,6.714639E-2,-1.5239224E-2,3.0771127E-2,3.4484306E-1,3.0918524E-2,1.3328609E-2,-1.1855602E-2,3.0727193E-1,2.3861451E-2,3.9015554E-2,-2.4533264E-2,-1.8193718E-2,-1.1442942E-3,-9.451596E-3,-1.3785134E-1,-1.3899204E-2,-1.4873855E-1,4.0112576E-3,-1.1344529E-1,1.0987411E-2,1.3535666E-2,5.889578E-3,1.6418745E-1,5.2867544E-1,-7.079403E-4,-5.93445E-3,-1.8644787E-1,-5.7308678E-2,-1.0020525E-2,-3.3568963E-4,3.9207708E-2,2.7109423E-1,8.091598E-3,1.8989801E-2,-2.1915615E-1,-8.577789E-2,-6.9646086E-3,5.74037E-3,-1.7345351E-3,2.146558E-3,7.7851177E-3,-8.63025E-2,1.288914E-1,1.105777E-2,-8.329164E-3,-4.2876313E-3,-1.2989044E-3,-3.6710775E-3,-1.6297619E-3,1.9685188E-3,-5.5116317E-3,-3.6952617E-5,5.35908E-3,1.9014591E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,-1,-1,39,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,43,-1,45,-1,47,-1,-1,-1,49,51,-1,-1,53,55,-1,57,59,61,-1,-1,63,65,-1,67,-1,-1,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3651234E1,8.428385E0,7.9335747E0,7.166828E0,3.3576431E0,3.1527348E0,1.0540657E0,1.224123E0,1.102165E0,1.29024E0,5.96894E-1,1.3535643E-1,2.0027766E0,7.861099E-1,0E0,1.2641525E-1,1.0225037E-1,4.9265242E-1,3.7987465E-1,8.8051695E-1,0E0,0E0,8.796096E-3,0E0,0E0,0E0,1.4021297E0,0E0,0E0,0E0,0E0,0E0,0E0,4.9832672E-2,0E0,1.3756847E-1,0E0,2.4607137E-1,0E0,0E0,0E0,3.9020646E-1,1.5753174E-1,0E0,0E0,7.998359E-2,1.4164016E-1,0E0,3.230579E-2,4.0239072E-1,1.2052417E-1,0E0,0E0,4.0420413E-2,6.955415E-3,0E0,3.745824E-2,0E0,0E0,0E0,7.2120816E-2,8.540191E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,22,22,26,26,33,33,35,35,37,37,41,41,42,42,45,45,46,46,48,48,49,49,50,50,53,53,54,54,56,56,60,60,61,61],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,-1,-1,40,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,44,-1,46,-1,48,-1,-1,-1,50,52,-1,-1,54,56,-1,58,60,62,-1,-1,64,66,-1,68,-1,-1,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,1.83E2,4.148E4,8.99622E5,1.1458619E7,9.099E3,2.3177024E7,4.3199274E8,1.33368E5,9.9914E4,9.02E2,5.776113E9,6.162E3,3.1376936E7,1.1466129E-2,6.38524E5,1.06095E5,1.484E3,2.29221E6,1.9254597E8,-1.5239224E-2,3.0771127E-2,1.4933E4,3.0918524E-2,1.3328609E-2,-1.1855602E-2,6.0958E5,2.3861451E-2,3.9015554E-2,-2.4533264E-2,-1.8193718E-2,-1.1442942E-3,-9.451596E-3,4.6585E5,-1.3899204E-2,6.490568E8,4.0112576E-3,2.1159E4,1.0987411E-2,1.3535666E-2,5.889578E-3,2.5420546E7,2.2256E4,-7.079403E-4,-5.93445E-3,1.333505E6,2.8699438E7,-1.0020525E-2,3.676E3,1E0,6.4614874E8,8.091598E-3,1.8989801E-2,2.0497083E3,1.757482E6,-6.9646086E-3,1.7452324E3,-1.7345351E-3,2.146558E-3,7.7851177E-3,2.8673542E8,1.0498E4,1.105777E-2,-8.329164E-3,-4.2876313E-3,-1.2989044E-3,-3.6710775E-3,-1.6297619E-3,1.9685188E-3,-5.5116317E-3,-3.6952617E-5,5.35908E-3,1.9014591E-3],"split_indices":[12,47,3,9,18,7,18,16,12,12,20,10,20,2,0,2,6,20,6,23,0,0,8,0,0,0,12,0,0,0,0,0,0,6,0,16,0,18,0,0,0,18,18,0,0,6,18,0,3,45,21,0,0,5,6,0,5,0,0,0,27,3,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.81E2,9.5E1,1.51E2,3E1,6E1,3.5E1,7.1E1,8E1,2.1E1,9E0,1E1,5E1,3.1E1,4E0,6.5E1,6E0,3.3E1,4.7E1,1.6E1,5E0,3E0,6E0,7E0,3E0,4E0,4.6E1,1.4E1,1.7E1,4.1E1,2.4E1,3E0,3E0,9E0,2.4E1,4.2E1,5E0,1E1,6E0,3E0,3E0,2.9E1,1.7E1,3E0,6E0,2.9E1,1.3E1,3E0,7E0,1.4E1,1.5E1,4E0,1.3E1,2.1E1,8E0,3E0,1E1,4E0,3E0,5E0,9E0,6E0,9E0,1.3E1,8E0,4E0,4E0,5E0,5E0,4E0,5E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[2.1012286E-2,-3.1084636E-1,6.6792655E-1,-5.456579E-1,-1.0854557E-1,3.9529082E-1,1.0669978E0,-6.2340933E-1,-2.000182E-1,1.5294099E-1,-2.0901377E-1,2.565468E-1,7.124823E-1,4.514249E-2,7.849477E-1,-6.7312694E-1,-3.8629282E-1,4.0233824E-3,-2.7335322E-1,9.519048E-5,2.1088116E-2,-3.1275696E-1,4.8155352E-2,8.788099E-2,4.7385016E-1,2.6559504E-2,8.196015E-3,2.7631864E-2,6.53262E-3,-7.051065E-1,-1.5744098E-2,-1.3834288E-2,-6.8676565E-3,-3.2075828E-1,-2.0214578E-3,-2.0597422E-1,1.9030578E-1,-3.713444E-1,-1.3176756E-1,-9.988196E-2,2.5059205E-1,-3.149474E-2,9.078304E-3,1.163882E-2,1.9785352E-2,-1.1024206E-2,-2.3229824E-2,-1.1585209E-2,-4.9079075E-3,-2.97291E-3,-8.81767E-3,6.8962004E-4,9.4371205E-3,-5.479854E-1,-2.8830984E-1,-1.9550455E-1,-1.15171286E-4,-1.0945771E-2,3.0188933E-2,4.0277843E-3,9.221691E-3,-1.3748115E-1,1.0492511E-1,-7.459956E-3,-2.1796145E-2,-1.708268E-3,-3.2245946E-1,-2.346672E-3,-7.6491702E-3,6.5176147E-3,-2.3361365E-3,-6.8691177E-3,-1.6513602E-3,1.0324236E-3,4.5779496E-3,-1.1189767E-2,-3.1264618E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,33,35,-1,37,39,41,43,-1,-1,-1,-1,45,-1,-1,-1,47,-1,49,51,53,55,57,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,65,67,-1,-1,69,-1,-1,71,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.332474E1,9.203772E0,1.045834E1,2.2939644E0,2.8167014E0,2.501727E0,2.683422E0,6.0056305E-1,4.631912E-1,2.3026853E0,2.069386E0,1.5710304E0,7.181463E-1,0E0,8.5214424E-1,2.3155212E-3,2.1122456E-2,0E0,1.2957597E-1,9.799161E-1,0E0,5.342727E-1,7.1429795E-1,6.114813E-1,7.8128815E-2,0E0,0E0,0E0,0E0,4.1152954E-2,0E0,0E0,0E0,2.419293E-2,0E0,6.7356706E-2,2.1957272E-1,4.4336987E-1,1.2185678E-1,4.4760507E-1,9.18591E-3,2.6022226E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3314872E-1,2.238872E-1,3.4326375E-2,0E0,0E0,1.9174923E-1,0E0,0E0,5.447817E-2,1.7174907E-2,0E0,0E0,0E0,1.1368513E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,21,21,22,22,23,23,24,24,29,29,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,53,53,54,54,55,55,58,58,61,61,62,62,66,66],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,34,36,-1,38,40,42,44,-1,-1,-1,-1,46,-1,-1,-1,48,-1,50,52,54,56,58,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,66,68,-1,-1,70,-1,-1,72,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,8.64E2,6.6569E4,2.8681312E8,9.068E3,1.83E2,4.331342E-1,5.013E3,7.514E3,1.2534363E7,4.112304E6,1.378469E9,4.3740602E10,4.514249E-2,2.31E2,7.03651E5,1.122053E6,4.0233824E-3,1.763598E8,1E0,2.1088116E-2,1.080557E6,1.06454085E-4,4.2524388E7,3.419E3,2.6559504E-2,8.196015E-3,2.7631864E-2,6.53262E-3,4.41E3,-1.5744098E-2,-1.3834288E-2,-6.8676565E-3,1.31E2,-2.0214578E-3,3.342E3,3.676E3,3.4713E4,1.521867E6,1.28E2,2.2213E4,1.5551E4,9.078304E-3,1.163882E-2,1.9785352E-2,-1.1024206E-2,-2.3229824E-2,-1.1585209E-2,-4.9079075E-3,-2.97291E-3,-8.81767E-3,6.8962004E-4,9.4371205E-3,1.7979E4,9E0,6.9E1,-1.15171286E-4,-1.0945771E-2,2.2605868E-6,4.0277843E-3,9.221691E-3,2.486228E3,4.320722E6,-7.459956E-3,-2.1796145E-2,-1.708268E-3,6.490568E8,-2.346672E-3,-7.6491702E-3,6.5176147E-3,-2.3361365E-3,-6.8691177E-3,-1.6513602E-3,1.0324236E-3,4.5779496E-3,-1.1189767E-2,-3.1264618E-3],"split_indices":[3,0,3,16,7,47,63,3,7,18,2,27,26,0,4,2,9,0,21,19,0,6,63,18,0,0,0,0,0,7,0,0,0,4,0,7,3,18,6,4,18,3,0,0,0,0,0,0,0,0,0,0,0,18,19,19,0,0,63,0,0,5,2,0,0,0,16,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.93E2,1.94E2,9.9E1,8.9E1,1.05E2,6E1,3.9E1,7.2E1,1.7E1,2.9E1,7.6E1,4.3E1,1.7E1,1.6E1,2.3E1,5.8E1,1.4E1,3E0,1.4E1,2.3E1,6E0,5.4E1,2.2E1,2.5E1,1.8E1,1.3E1,4E0,2E1,3E0,4.7E1,1.1E1,1E1,4E0,1.1E1,3E0,1.1E1,1.2E1,4E1,1.4E1,1.3E1,9E0,1.6E1,9E0,1.2E1,6E0,3E0,4.4E1,8E0,3E0,5E0,6E0,5E0,7E0,1.1E1,2.9E1,9E0,5E0,4E0,9E0,3E0,6E0,9E0,7E0,4E0,7E0,4E0,2.5E1,3E0,6E0,3E0,6E0,4E0,5E0,3E0,4E0,2.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[7.8056445E-3,-3.1351525E-1,6.537132E-1,-5.641316E-1,-1.4590006E-1,4.1859275E-1,1.0015657E0,2.0434974E-1,-6.3195187E-1,2.3893863E-2,-1.7578515E-1,1.569812E-1,5.8137286E-1,1.0606976E0,1.2848417E-2,2.4637377E-3,9.047423E-3,-6.5167075E-1,-7.7681164E-3,-1.44938E-1,-2.2753647E-2,3.852505E-1,5.948863E-2,3.1269174E-2,4.871249E-1,2.8153362E-2,4.410662E-2,-6.8298596E-1,-1.5098391E-2,-2.2178148E-1,2.3867657E-2,1.8102614E-2,3.5988097E-3,-7.531394E-2,2.3740609E-1,5.851077E-3,5.388552E-1,-2.2524891E-2,-1.2076771E-2,1.2145464E-1,-2.589007E-1,1.0835922E-2,-2.734857E-2,-4.1707302E-3,5.0561206E-4,3.7967898E-3,9.189536E-3,8.285725E-3,1.861574E-2,-2.274781E-3,8.075022E-3,-3.4812585E-1,-9.150845E-2,-2.3357187E-1,3.1487897E-2,-1.4137079E-2,-2.6134923E-1,-1.2631503E-1,3.95296E-3,-2.0786843E-3,-1.10785775E-2,-4.645483E-2,2.0292304E-1,-1.981869E-3,-2.8579324E-1,-2.028875E-1,-4.5597337E-2,3.1014876E-3,-1.0879247E-1,8.70509E-3,2.182401E-3,-4.551214E-3,-3.0115205E-1,-2.6559012E-3,-7.651814E-3,-3.8203376E-3,1.5215987E-3,-5.5938135E-3,-7.1918E-2,-4.925223E-3,-1.0280048E-2,-8.342743E-4,-2.8303657E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,27,-1,29,-1,31,33,-1,35,-1,-1,37,-1,39,41,-1,-1,43,45,-1,47,-1,-1,49,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,61,-1,63,65,-1,-1,-1,67,69,-1,71,73,75,-1,77,-1,-1,-1,79,-1,-1,-1,-1,-1,81,-1,-1,-1,-1],"loss_changes":[5.99775E1,8.025066E0,7.432331E0,4.142273E0,3.2361262E0,2.421259E0,9.30233E-1,4.712802E-2,3.8961983E-1,0E0,1.765171E0,5.1119155E-1,8.5021496E-1,7.598343E-1,0E0,0E0,0E0,6.2791824E-2,0E0,1.419199E0,0E0,2.7618456E-1,4.4958717E-1,0E0,3.9172506E-1,0E0,0E0,2.1608353E-2,0E0,9.869945E-1,5.698627E-1,0E0,0E0,5.635863E-2,1.2273312E-2,0E0,1.3997173E-1,0E0,0E0,2.1662843E-1,9.837642E-1,0E0,3.8349167E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3291779E-1,2.0199987E-1,1.0861114E-1,3.434798E-1,0E0,1.0448909E-1,1.2672547E-1,0E0,0E0,0E0,1.7072335E-1,5.4646283E-2,0E0,6.2128305E-3,2.6503831E-2,8.7060206E-2,0E0,1.8655255E-2,0E0,0E0,0E0,2.399683E-3,0E0,0E0,0E0,0E0,0E0,5.0864145E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,17,17,19,19,21,21,22,22,24,24,27,27,29,29,30,30,33,33,34,34,36,36,39,39,40,40,42,42,51,51,52,52,53,53,54,54,56,56,57,57,61,61,62,62,64,64,65,65,66,66,68,68,72,72,78,78],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,28,-1,30,-1,32,34,-1,36,-1,-1,38,-1,40,42,-1,-1,44,46,-1,48,-1,-1,50,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,62,-1,64,66,-1,-1,-1,68,70,-1,72,74,76,-1,78,-1,-1,-1,80,-1,-1,-1,-1,-1,82,-1,-1,-1,-1],"split_conditions":[9.91E3,8.03052E5,6.6569E4,3.4727E2,3.068E3,4.425011E6,2.3177024E7,5.876475E7,3.7372906E8,2.3893863E-2,1.17442E5,5.71365E5,1.0927762E9,1.0778861E4,1.2848417E-2,2.4637377E-3,9.047423E-3,7.19175E5,-7.7681164E-3,2.702002E6,-2.2753647E-2,2.0599214E3,1.3925089E9,3.1269174E-2,1.24673E5,2.8153362E-2,4.410662E-2,6.103E3,-1.5098391E-2,3.2416E4,7.62802E5,1.8102614E-2,3.5988097E-3,2.962E4,3.88957E5,5.851077E-3,8.657312E6,-2.2524891E-2,-1.2076771E-2,3.5888664E7,3.1633773E8,1.0835922E-2,1.203673E6,-4.1707302E-3,5.0561206E-4,3.7967898E-3,9.189536E-3,8.285725E-3,1.861574E-2,-2.274781E-3,8.075022E-3,1.358313E6,1.1408653E9,1.5491053E3,8.1188E4,-1.4137079E-2,8.808534E8,2.3912651E8,3.95296E-3,-2.0786843E-3,-1.10785775E-2,5.1E1,1.008599E8,-1.981869E-3,1.542305E6,2.433E3,6.378634E8,3.1014876E-3,7.7E1,8.70509E-3,2.182401E-3,-4.551214E-3,4.64E2,-2.6559012E-3,-7.651814E-3,-3.8203376E-3,1.5215987E-3,-5.5938135E-3,1.9329154E-5,-4.925223E-3,-1.0280048E-2,-8.342743E-4,-2.8303657E-3],"split_indices":[3,9,3,5,7,2,18,16,16,0,8,6,11,5,0,0,0,2,0,2,0,5,16,0,28,0,0,3,0,12,6,0,0,28,12,0,9,0,0,16,16,0,6,0,0,0,0,0,0,0,0,9,16,5,28,0,11,21,0,0,0,17,16,0,2,20,16,0,4,0,0,0,28,0,0,0,0,0,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.87E2,1.92E2,9.5E1,7.6E1,1.16E2,5.8E1,3.7E1,6E0,7E1,3E0,1.13E2,2.3E1,3.5E1,3.3E1,4E0,3E0,3E0,6.6E1,4E0,1.08E2,5E0,6E0,1.7E1,5E0,3E1,2.3E1,1E1,5.4E1,1.2E1,7.4E1,3.4E1,3E0,3E0,1E1,7E0,5E0,2.5E1,5E1,4E0,7E0,6.7E1,4E0,3E1,6E0,4E0,3E0,4E0,4E0,2.1E1,3E0,4E0,4.3E1,2.4E1,6E0,2.4E1,1.9E1,2.4E1,2.1E1,3E0,3E0,3E0,1.7E1,7E0,3E0,2.1E1,1E1,1.1E1,5E0,1.2E1,4E0,3E0,3E0,1.8E1,3E0,7E0,6E0,5E0,3E0,9E0,3E0,1.5E1,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[1.8465197E-2,-2.9814774E-1,6.389691E-1,-4.887915E-1,-3.359799E-2,3.709431E-1,9.777542E-1,-5.5016166E-1,-4.297969E-2,-9.3357585E-2,2.999627E-1,1.24915525E-1,5.2904934E-1,1.770746E-2,1.0575055E0,-6.100958E-1,-1.9363058E-1,-2.3061329E-1,8.913324E-3,-1.643865E-1,1.6551778E-1,2.2712333E-2,1.0771394E-1,3.9211318E-1,-8.172526E-3,4.1001534E-1,3.334919E-2,3.958928E-2,2.699121E-2,-6.744788E-1,-4.6029195E-1,-8.51165E-2,-9.383244E-3,-1.2469128E-2,-1.0809165E-1,-7.2139695E-2,-2.795768E-1,2.8004217E-1,4.532746E-2,1.20322744E-4,5.678397E-3,1.6587803E-2,4.5086783E-3,-1.273104E-1,1.4177138E-1,1.249393E-1,5.080003E-1,-6.948547E-1,-1.2298735E-2,-1.5888346E-2,-9.177328E-3,1.3716905E-3,-5.024178E-3,-6.7494423E-3,6.605662E-4,-1.790101E-1,8.5685134E-2,-2.015864E-1,-1.7743237E-2,5.2648787E-3,1.1447433E-2,3.27951E-3,-1.6364167E-3,-7.2531514E-3,-5.311176E-2,1.5365279E-3,6.072335E-3,5.798006E-3,3.5173952E-4,2.7520823E-1,5.780968E-1,-2.2945985E-2,-1.2215822E-2,-1.8194126E-3,-2.0609921E-1,-5.388001E-3,1.7752798E-1,-2.5738937E-1,-7.873291E-2,-7.2321604E-4,-2.2686056E-3,4.093486E-3,1.1409181E-2,2.035989E-2,1.0682936E-2,-3.8062744E-3,-7.950199E-3,-3.7821257E-3,2.748206E-3,1.1849736E-3,8.815299E-3,-5.247793E-3,-9.9070985E-3,-3.1074798E-3,-1.1843108E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,-1,39,41,43,45,-1,-1,-1,47,49,51,-1,-1,53,55,57,59,61,-1,-1,-1,-1,63,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,73,75,77,-1,-1,-1,-1,-1,-1,79,-1,-1,-1,-1,81,83,-1,-1,-1,85,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8735462E1,9.969662E0,8.808434E0,3.1226997E0,1.7024137E0,2.1900268E0,9.188843E-1,2.0400715E0,9.641785E-1,1.3511186E0,9.377649E-1,8.5666406E-1,1.6874428E0,0E0,3.277626E-1,5.189972E-1,1.4512032E-1,1.5019256E-1,0E0,5.855099E-1,2.0754197E-1,0E0,7.076609E-2,1.7625892E-1,3.2173696E-1,7.7452564E-1,0E0,0E0,0E0,8.006859E-2,2.8468132E-2,8.8233404E-2,0E0,0E0,9.57639E-2,5.71943E-1,4.1852283E-1,1.2270391E-2,5.4127686E-2,0E0,0E0,0E0,0E0,6.076184E-2,2.6258916E-2,5.4657236E-2,1.9989729E-1,3.0281067E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.4712117E-2,1.1805833E-1,1.2370235E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1416545E-3,0E0,0E0,0E0,0E0,3.7034273E-2,4.444599E-3,0E0,0E0,0E0,2.826482E-2,9.143691E-2,8.480674E-2,1.6388655E-2,2.4223663E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,22,22,23,23,24,24,25,25,29,29,30,30,31,31,34,34,35,35,36,36,37,37,38,38,43,43,44,44,45,45,46,46,47,47,55,55,56,56,57,57,64,64,69,69,70,70,74,74,75,75,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,-1,40,42,44,46,-1,-1,-1,48,50,52,-1,-1,54,56,58,60,62,-1,-1,-1,-1,64,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,74,76,78,-1,-1,-1,-1,-1,-1,80,-1,-1,-1,-1,82,84,-1,-1,-1,86,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,1.486961E6,4.6908E4,1.83E2,6.910113E8,4.425011E6,8.220982E8,3.5655382E8,2.09153E5,2.113977E6,3.28451E5,5.1410074E8,1.5860209E9,1.770746E-2,6.4401817E0,7.19175E5,2E1,2.8E1,8.913324E-3,1.6010206E10,1.58E2,2.2712333E-2,2.5735368E7,5.71365E5,1.0886205E9,1.39104E5,3.334919E-2,3.958928E-2,2.699121E-2,2.1811952E7,5.26565E5,1.18443E5,-9.383244E-3,-1.2469128E-2,4.57473E5,2.564429E6,1.83E2,1.05213176E8,3.1584E4,1.20322744E-4,5.678397E-3,1.6587803E-2,4.5086783E-3,1.5597291E3,6.549943E7,8.5867E4,2.7797654E3,5.244E3,-1.2298735E-2,-1.5888346E-2,-9.177328E-3,1.3716905E-3,-5.024178E-3,-6.7494423E-3,6.605662E-4,3.7388639E9,2.466E3,5.5293235E9,-1.7743237E-2,5.2648787E-3,1.1447433E-2,3.27951E-3,-1.6364167E-3,-7.2531514E-3,1.0773E4,1.5365279E-3,6.072335E-3,5.798006E-3,3.5173952E-4,5.1E1,6.41E2,-2.2945985E-2,-1.2215822E-2,-1.8194126E-3,1.105E3,2.2E1,1.0446692E3,2.7653658E9,2.53387E5,-7.2321604E-4,-2.2686056E-3,4.093486E-3,1.1409181E-2,2.035989E-2,1.0682936E-2,-3.8062744E-3,-7.950199E-3,-3.7821257E-3,2.748206E-3,1.1849736E-3,8.815299E-3,-5.247793E-3,-9.9070985E-3,-3.1074798E-3,-1.1843108E-3],"split_indices":[3,9,3,47,16,2,16,16,18,6,6,11,16,0,60,2,19,4,0,10,4,0,18,6,16,22,0,0,0,18,6,12,0,0,21,2,47,23,20,0,0,0,0,5,27,8,5,3,0,0,0,0,0,0,0,10,0,11,0,0,0,0,0,0,3,0,0,0,0,4,19,0,0,0,0,19,5,11,18,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.97E2,1.97E2,1E2,1.14E2,8.3E1,5.7E1,4.3E1,1E2,1.4E1,7.1E1,1.2E1,2.3E1,3.4E1,8E0,3.5E1,8.5E1,1.5E1,9E0,5E0,5.6E1,1.5E1,3E0,9E0,7E0,1.6E1,2.9E1,5E0,1.7E1,1.8E1,5.7E1,2.8E1,8E0,7E0,3E0,6E0,3.2E1,2.4E1,7E0,8E0,4E0,5E0,4E0,3E0,9E0,7E0,8E0,2.1E1,5.2E1,5E0,2.2E1,6E0,3E0,5E0,3E0,3E0,1.9E1,1.3E1,2E1,4E0,4E0,3E0,5E0,3E0,3E0,6E0,3E0,4E0,5E0,3E0,6E0,1.5E1,4.8E1,4E0,4E0,1.5E1,7E0,6E0,1.3E1,7E0,3E0,3E0,3E0,3E0,1.1E1,4E0,6E0,9E0,3E0,4E0,3E0,3E0,6E0,7E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[3.585037E-3,-2.3175979E-1,7.982482E-1,-5.442919E-1,-4.8587322E-2,4.2888325E-1,9.300292E-1,-6.127004E-1,3.6882784E-2,-8.426464E-2,2.6067264E-2,2.00343E-1,5.9765774E-1,1.103143E0,6.96525E-1,-6.7828816E-1,-4.579596E-1,-8.688198E-3,9.218863E-3,-1.6072363E-1,1.09870486E-1,1.4932224E-3,9.503703E-3,8.424807E-3,2.2239821E-2,2.1570936E-2,4.127968E-2,1.013734E-2,7.8036875E-1,-2.2628156E-2,-1.19688315E-2,-1.5560759E-2,-6.3437102E-3,-2.1312551E-1,6.740043E-2,4.1622728E-1,1.0700535E-2,1.231435E-2,8.3104676E-1,-2.853735E-1,-3.1205282E-2,1.2341833E-2,9.159349E-3,-1.2505404E-3,1.9854484E-2,-1.1731588E-2,6.2077828E-2,1.3714318E-2,2.870394E-2,6.63079E-2,-3.3307308E-1,1.40670715E-2,-1.1491789E-1,6.0856044E-3,-3.9030273E-2,7.456962E-3,7.7661867E-3,-3.2482485E-3,6.0136104E-3,-1.859925E-1,-3.7622446E-1,-2.034762E-1,-7.861746E-3,-3.0481662E-3,3.0163467E-2,7.616208E-2,-8.166064E-3,-3.3785158E-3,-7.0606195E-3,-4.432561E-1,-2.788076E-1,-1.4884996E-3,-8.07136E-3,-7.433229E-2,2.7930154E-3,-3.4472414E-5,1.733601E-3,-3.5717306E-3,1.3345909E-1,-1.1012763E-2,-1.8587941E-2,-5.11823E-3,-9.661657E-3,-3.2258031E-3,-9.6138526E-4,1.6679871E-3,4.813891E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,39,41,43,45,-1,47,49,51,53,-1,-1,-1,-1,55,-1,-1,57,59,-1,61,-1,63,-1,65,-1,-1,67,69,71,73,-1,75,77,-1,-1,-1,79,81,-1,-1,83,-1,-1,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6477978E1,1.3312171E1,2.9597206E0,3.4417248E0,4.600839E0,6.516988E-1,1.2527733E0,4.708538E-1,8.4280396E-1,2.1402612E0,0E0,1.3241231E-1,1.6377044E-1,1.2365456E0,5.220976E-1,1.3207436E-1,7.741976E-2,0E0,0E0,1.2456403E0,1.2411829E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.510925E-2,0E0,0E0,0E0,0E0,1.0951383E0,2.3560649E-1,9.3677115E-1,6.394163E-1,0E0,1.3305664E-2,1.0285888E0,1.0301204E0,1.6171046E-1,0E0,0E0,0E0,0E0,2.6530248E-1,0E0,0E0,1.800715E-1,2.664013E-1,0E0,2.0697597E-1,0E0,5.678045E-2,0E0,4.3062213E-1,0E0,0E0,1.4819503E-2,1.3761759E-1,7.759991E-2,7.5641766E-2,0E0,5.2380534E-3,2.1851945E-1,0E0,0E0,0E0,1.0101795E-1,8.6402893E-4,0E0,0E0,5.0629564E-3,0E0,0E0,0E0,0E0,1.1956722E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,28,28,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,46,46,49,49,50,50,52,52,54,54,56,56,59,59,60,60,61,61,62,62,64,64,65,65,69,69,70,70,73,73,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,40,42,44,46,-1,48,50,52,54,-1,-1,-1,-1,56,-1,-1,58,60,-1,62,-1,64,-1,66,-1,-1,68,70,72,74,-1,76,78,-1,-1,-1,80,82,-1,-1,84,-1,-1,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4880835E9,8.03052E5,1.4066751E7,1.83E2,1.7833586E9,7.502168E8,4.331342E-1,4.014E3,6.7452E4,2.57951E5,2.6067264E-2,6.449162E10,4.618673E6,6.6569E4,3.298233E11,4.0200136E7,2.9105232E7,-8.688198E-3,9.218863E-3,2.989734E6,1.0382E4,1.4932224E-3,9.503703E-3,8.424807E-3,2.2239821E-2,2.1570936E-2,4.127968E-2,1.013734E-2,2.345689E7,-2.2628156E-2,-1.19688315E-2,-1.5560759E-2,-6.3437102E-3,4.3199274E8,4.157189E8,1.3708E4,6.162E3,1.231435E-2,2.2491941E0,3.2416E4,9.1416E4,2.964093E-6,9.159349E-3,-1.2505404E-3,1.9854484E-2,-1.1731588E-2,1.5268964E3,1.3714318E-2,2.870394E-2,4.57473E5,1.723E3,1.40670715E-2,1.7140013E3,6.0856044E-3,2.748775E6,7.456962E-3,5.215832E10,-3.2482485E-3,6.0136104E-3,4.45E2,1.27001E5,5.024E3,8.405997E8,-3.0481662E-3,1.11505E5,1.1988E4,-8.166064E-3,-3.3785158E-3,-7.0606195E-3,1.352E3,1.7739E4,-1.4884996E-3,-8.07136E-3,2.1E1,2.7930154E-3,-3.4472414E-5,1.733601E-3,-3.5717306E-3,1.52E2,-1.1012763E-2,-1.8587941E-2,-5.11823E-3,-9.661657E-3,-3.2258031E-3,-9.6138526E-4,1.6679871E-3,4.813891E-3],"split_indices":[27,9,2,47,16,16,63,3,18,12,0,13,2,3,13,18,18,0,0,9,7,0,0,0,0,0,0,0,2,0,0,0,0,16,27,8,20,0,59,12,12,63,0,0,0,0,5,0,0,21,20,0,5,0,18,0,10,0,0,20,12,20,16,0,22,7,0,0,0,0,28,0,0,19,0,0,0,0,17,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[3E2,2.32E2,6.8E1,8.5E1,1.47E2,1.9E1,4.9E1,7.6E1,9E0,1.42E2,5E0,9E0,1E1,2.6E1,2.3E1,5.1E1,2.5E1,4E0,5E0,1.02E2,4E1,4E0,5E0,3E0,7E0,9E0,1.7E1,5E0,1.8E1,4.6E1,5E0,2.2E1,3E0,8.3E1,1.9E1,9E0,3.1E1,3E0,1.5E1,5.9E1,2.4E1,1.6E1,3E0,3E0,6E0,3E0,2.8E1,3E0,1.2E1,7E0,5.2E1,3E0,2.1E1,3E0,1.3E1,6E0,2.2E1,3E0,4E0,1.3E1,3.9E1,1.1E1,1E1,7E0,6E0,1.8E1,4E0,5E0,8E0,2.1E1,1.8E1,3E0,8E0,6E0,4E0,3E0,3E0,4E0,1.4E1,1.4E1,7E0,4E0,1.4E1,3E0,3E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[2.7615003E-2,-2.7866903E-1,5.99685E-1,-5.5470645E-1,-1.1723503E-1,2.3670952E-1,8.977811E-1,-2.119634E-3,-5.76256E-1,-2.250991E-1,1.5587325E-1,1.01153955E-1,5.3932554E-1,1.0598148E0,6.674294E-1,-5.8813983E-1,-7.982608E-3,1.22651145E-1,-2.578799E-1,3.8881353E-1,2.7562099E-2,3.0067667E-1,1.3804733E-2,1.101448E-2,2.3985712E-2,1.6593633E-2,1.1237273E0,5.9865937E-3,2.5058843E-2,-1.9776294E-2,-1.3217506E-2,-1.7568188E-4,8.115628E-3,-3.422348E-1,-1.2592493E-1,5.877699E-1,-2.3431839E-4,-6.447553E-2,6.7014867E-3,4.5838384E-3,1.1207049E-2,7.389735E-2,-9.821904E-3,1.7256565E-2,3.7888505E-2,-4.049148E-1,-2.3737107E-1,-1.6375563E-1,5.7885037E-3,2.2748947E-2,9.403221E-3,1.9721871E-2,-1.7554674E-1,-6.871898E-2,1.6694732E-1,-1.5483998E-2,-9.477573E-3,-1.0240395E-2,-1.8003465E-1,-1.4179248E-2,-2.107036E-1,2.8721332E-3,-2.1769067E-3,-7.111534E-3,-2.7771327E-3,-3.9038728E-3,-7.771326E-5,1.9755222E-1,1.5675462E-3,-6.431786E-3,-2.5955369E-3,-2.9190998E-3,2.7360443E-3,-1.354627E-1,-2.874464E-1,3.013956E-3,7.3620253E-3,-1.6054125E-1,-1.2521717E-3,-1.3635631E-2,-5.4262117E-3,-2.0695995E-3,-6.1997375E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,-1,43,-1,-1,-1,-1,-1,-1,45,47,49,-1,51,-1,-1,-1,53,-1,-1,-1,55,57,59,-1,-1,-1,61,63,65,67,-1,-1,-1,69,71,73,-1,-1,-1,-1,-1,-1,75,-1,-1,-1,-1,-1,77,79,-1,-1,81,-1,-1,-1,-1,-1],"loss_changes":[4.975322E1,8.178906E0,1.0499435E1,6.784096E-1,3.5111985E0,1.8324242E0,1.3081856E0,0E0,1.077137E-1,1.0005898E0,1.0055038E0,5.709783E-1,3.1310892E-1,4.6435547E-1,1.1919804E0,3.6535263E-2,0E0,1.3406974E-1,8.251257E-1,9.499415E-1,3.957927E-1,2.5579989E-2,4.8252735E-1,0E0,0E0,0E0,1.8388367E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1280861E-1,3.9958805E-1,7.490778E-2,0E0,1.5309262E-1,0E0,0E0,0E0,2.901518E-1,0E0,0E0,0E0,9.383631E-2,3.5337806E-2,2.006594E-1,0E0,0E0,0E0,6.674791E-2,9.299308E-3,3.1072509E-2,3.742653E-2,0E0,0E0,0E0,1.1412352E-2,6.841131E-2,8.809394E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0966599E-2,0E0,0E0,0E0,0E0,0E0,2.52361E-2,9.045398E-2,0E0,0E0,1.847595E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,26,26,33,33,34,34,35,35,37,37,41,41,45,45,46,46,47,47,51,51,52,52,53,53,54,54,58,58,59,59,60,60,67,67,73,73,74,74,77,77],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,-1,44,-1,-1,-1,-1,-1,-1,46,48,50,-1,52,-1,-1,-1,54,-1,-1,-1,56,58,60,-1,-1,-1,62,64,66,68,-1,-1,-1,70,72,74,-1,-1,-1,-1,-1,-1,76,-1,-1,-1,-1,-1,78,80,-1,-1,82,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,8.03052E5,7.502168E8,3.185029E2,7.164074E8,1.4012293E0,6.4401817E0,-2.119634E-3,6.591E3,3.2416E4,1.1326E4,2.239676E10,1.313587E7,4.7914E4,8.798E5,7.19175E5,-7.982608E-3,3.24654E5,2.004803E6,1.06E2,1.0886205E9,1.6505978E9,1.83E2,1.101448E-2,2.3985712E-2,1.6593633E-2,1.9E1,5.9865937E-3,2.5058843E-2,-1.9776294E-2,-1.3217506E-2,-1.7568188E-4,8.115628E-3,1.7013242E8,3.05031E5,4.9E1,-2.3431839E-4,2.934742E8,6.7014867E-3,4.5838384E-3,1.1207049E-2,3.718604E6,-9.821904E-3,1.7256565E-2,3.7888505E-2,2.63E2,4.951E3,4.105292E9,5.7885037E-3,2.2748947E-2,9.403221E-3,2.9911E4,1.7140013E3,1.914442E6,1.0456E5,-1.5483998E-2,-9.477573E-3,-1.0240395E-2,1.82951E5,2.965665E6,4.6545E4,2.8721332E-3,-2.1769067E-3,-7.111534E-3,-2.7771327E-3,-3.9038728E-3,-7.771326E-5,3.419E3,1.5675462E-3,-6.431786E-3,-2.5955369E-3,-2.9190998E-3,2.7360443E-3,1.260274E6,6.3226E4,3.013956E-3,7.3620253E-3,1.4441514E9,-1.2521717E-3,-1.3635631E-2,-5.4262117E-3,-2.0695995E-3,-6.1997375E-3],"split_indices":[2,9,16,5,16,64,60,0,3,12,28,10,2,18,12,2,0,6,9,4,16,11,47,0,0,0,4,0,0,0,0,0,0,16,12,4,0,27,0,0,0,6,0,0,0,4,3,10,0,0,0,28,5,6,8,0,0,0,12,2,28,0,0,0,0,0,0,0,0,0,0,0,0,6,8,0,0,11,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.84E2,9.8E1,6.7E1,1.17E2,4.5E1,5.3E1,3E0,6.4E1,8.4E1,3.3E1,3.2E1,1.3E1,2.9E1,2.4E1,6.1E1,3E0,7E0,7.7E1,1.1E1,2.2E1,9E0,2.3E1,8E0,5E0,4E0,2.5E1,5E0,1.9E1,5.1E1,1E1,4E0,3E0,4.6E1,3.1E1,7E0,4E0,1.5E1,7E0,3E0,6E0,2E1,3E0,3E0,2.2E1,2.7E1,1.9E1,2.8E1,3E0,4E0,3E0,9E0,6E0,8E0,1.2E1,1.4E1,1.3E1,6E0,1.3E1,7E0,2.1E1,5E0,4E0,3E0,3E0,4E0,4E0,9E0,3E0,1E1,3E0,4E0,3E0,1.2E1,9E0,3E0,6E0,9E0,3E0,3E0,6E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[6.9011925E-3,-2.571306E-1,5.2353436E-1,-4.76839E-1,-8.39869E-2,1.15709275E-1,6.886187E-1,-5.472473E-1,1.4562682E-2,-1.7116198E-1,3.6797002E-2,-3.7946023E-2,2.1736541E-1,3.4192908E-1,8.5419244E-1,-5.732526E-1,-5.611005E-3,-1.1659199E-1,8.795033E-3,-2.53181E-1,5.5351164E-3,2.0633225E-1,-1.00303076E-1,-5.7924413E-3,2.9383603E-2,3.331894E-1,1.0296969E-1,4.0475377E-1,-1.8565492E-3,7.314094E-1,3.1896424E-2,-5.836263E-1,-8.663604E-3,-6.363615E-3,4.4856934E-4,1.7364722E-3,-2.7884752E-1,1.7824207E-1,-8.121373E-2,8.226529E-3,3.3771718E-1,-4.1427E-3,-1.9445654E-2,-6.2495586E-4,3.065546E-3,1.2805343E-2,5.443361E-3,-1.5428961E-3,5.616637E-3,4.370468E-3,4.8536155E-1,7.7378434E-1,9.47345E-3,-1.9113857E-2,-9.258593E-3,-1.3404715E-2,-2.2827537E-1,1.6710663E-3,8.36943E-3,-6.569503E-3,-1.1653321E-2,-1.5263127E-3,2.8852765E-3,1.9778672E-3,4.2298046E-1,-1.241736E-1,7.1277004E-3,1.7745277E-2,7.830243E-3,2.0500151E-2,3.1159116E-2,-1.6550866E-3,-2.4602185E-1,1.4402393E-3,-1.3588628E-3,7.163185E-3,1.5647495E-2,-6.3283525E-2,-7.5095883E-3,-2.6828682E-1,-3.971474E-3,-3.1469688E-3,-7.571177E-5,-2.847003E-1,-3.967324E-3,-9.817797E-3,-4.258794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,39,41,-1,43,45,47,49,-1,51,-1,53,-1,-1,-1,-1,55,57,59,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,67,69,-1,-1,-1,-1,71,-1,-1,-1,73,-1,-1,-1,75,77,-1,-1,-1,-1,-1,-1,79,-1,-1,-1,-1,81,-1,83,-1,-1,-1,85,-1,-1,-1],"loss_changes":[3.7510296E1,6.883438E0,6.181595E0,2.7758694E0,1.090821E0,4.456083E-1,3.4965076E0,5.789509E-1,4.0503824E-1,8.709527E-1,1.0434008E0,1.2002148E-1,1.9093764E-1,6.007619E-1,1.38549805E-2,4.7969818E-2,0E0,8.744498E-2,0E0,3.382411E-1,3.141183E-1,5.1778376E-1,1.2086525E0,0E0,3.0770682E-2,1.7561793E-2,1.162887E-1,3.677349E-1,0E0,2.7236843E-1,0E0,1.5937805E-2,0E0,0E0,0E0,0E0,1.6935134E-1,5.8808193E-2,1.1728593E-1,4.5019466E-2,2.5667965E-1,6.2316954E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.05297565E-1,2.171421E-2,0E0,0E0,0E0,0E0,8.150804E-2,0E0,0E0,0E0,1.9124694E-2,0E0,0E0,0E0,5.715251E-3,8.4904134E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.0625334E-2,0E0,0E0,0E0,0E0,2.2881418E-2,0E0,1.7282486E-2,0E0,0E0,0E0,1.843834E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,29,29,31,31,36,36,37,37,38,38,39,39,40,40,41,41,50,50,51,51,56,56,60,60,64,64,65,65,72,72,77,77,79,79,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,40,42,-1,44,46,48,50,-1,52,-1,54,-1,-1,-1,-1,56,58,60,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,68,70,-1,-1,-1,-1,72,-1,-1,-1,74,-1,-1,-1,76,78,-1,-1,-1,-1,-1,-1,80,-1,-1,-1,-1,82,-1,84,-1,-1,-1,86,-1,-1,-1],"split_conditions":[9.896E3,8.99622E5,4.689138E6,1.83E2,2.597379E6,2.317698E7,9.854431E8,3.7372906E8,8.4677495E-2,4.579412E8,1.9694447E10,1.0821E5,1.06E2,3.4877544E7,1.8458214E9,7.7999923E9,-5.611005E-3,6.52028E5,8.795033E-3,2.5961E4,5E1,1.83E2,1.83E2,-5.7924413E-3,3.615948E6,1.6121E4,3.1043533E8,4.93153E5,-1.8565492E-3,8.81E2,3.1896424E-2,5.101357E7,-8.663604E-3,-6.363615E-3,4.4856934E-4,1.7364722E-3,3.9E1,1.1E1,2.83326E9,3.751062E6,4.856E3,3.718604E6,-1.9445654E-2,-6.2495586E-4,3.065546E-3,1.2805343E-2,5.443361E-3,-1.5428961E-3,5.616637E-3,4.370468E-3,6.196538E-1,6.8344595E3,9.47345E-3,-1.9113857E-2,-9.258593E-3,-1.3404715E-2,1.2257E4,1.6710663E-3,8.36943E-3,-6.569503E-3,7.09E2,-1.5263127E-3,2.8852765E-3,1.9778672E-3,7.143424E7,4.223761E7,7.1277004E-3,1.7745277E-2,7.830243E-3,2.0500151E-2,3.1159116E-2,-1.6550866E-3,1.77451E5,1.4402393E-3,-1.3588628E-3,7.163185E-3,1.5647495E-2,1.610428E6,-7.5095883E-3,1.8468E4,-3.971474E-3,-3.1469688E-3,-7.571177E-5,3.069059E6,-3.967324E-3,-9.817797E-3,-4.258794E-3],"split_indices":[3,9,9,47,9,18,16,16,63,16,10,12,4,2,16,10,0,21,0,12,4,47,47,0,2,7,21,12,0,17,0,18,0,0,0,0,4,4,10,2,3,6,0,0,0,0,0,0,0,0,63,5,0,0,0,0,8,0,0,0,17,0,0,0,16,21,0,0,0,0,0,0,12,0,0,0,0,6,0,7,0,0,0,18,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.81E2,9.2E1,7.9E1,1.02E2,2.7E1,6.5E1,6.9E1,1E1,5.9E1,4.3E1,1.1E1,1.6E1,2.2E1,4.3E1,6.4E1,5E0,7E0,3E0,4E1,1.9E1,1.9E1,2.4E1,3E0,8E0,7E0,9E0,1.9E1,3E0,2.6E1,1.7E1,6.1E1,3E0,4E0,3E0,3E0,3.7E1,6E0,1.3E1,8E0,1.1E1,2.1E1,3E0,5E0,3E0,4E0,3E0,3E0,6E0,5E0,1.4E1,2.3E1,3E0,5.8E1,3E0,8E0,2.9E1,3E0,3E0,4E0,9E0,5E0,3E0,3E0,8E0,1.4E1,7E0,1E1,4E0,1.6E1,7E0,3E0,2.6E1,3E0,6E0,3E0,5E0,1E1,4E0,2.1E1,5E0,6E0,4E0,1.8E1,3E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-1.4078515E-2,-2.9519936E-1,5.5718887E-1,-4.9864766E-1,-1.3046089E-1,2.597489E-1,8.1968987E-1,-5.5135095E-1,-4.296753E-2,-2.3282275E-1,2.3418881E-3,6.846518E-1,1.5227853E-1,6.539929E-1,1.0173669E0,-5.862284E-1,-2.5904948E-1,-7.56337E-3,3.9976747E-3,-2.7384496E-1,-2.1463111E-2,-6.871231E-2,3.662019E-1,5.7682376E-3,2.921185E-2,6.9057815E-2,3.2683524E-1,1.5736744E-2,7.679058E-1,3.524897E-2,1.3306547E-2,-2.1630712E-2,-5.1713705E-1,-1.0055833E-2,-3.2486243E-3,-3.2085472E-1,-1.609516E-1,-1.5460555E-1,5.447303E-3,-1.7915912E-1,1.1505828E-1,1.5182144E-2,6.7145377E-3,-1.050905E-1,2.3827343E-1,5.393875E-3,1.1630038E-2,1.2332693E-2,2.6627587E-2,-1.7257541E-2,-8.669945E-3,-4.38609E-4,-3.492316E-1,-1.5763175E-3,-6.1687008E-3,-7.3196213E-3,-1.3894143E-3,5.2874917E-3,-2.4906626E-1,-4.0099572E-2,2.1216294E-1,-1.7514011E-1,5.632744E-4,8.524628E-3,3.401479E-3,-1.2169977E-2,-6.1864955E-3,-1.2235396E-1,-3.8888562E-1,1.2894602E-3,-3.5482964E-3,1.2095584E-1,1.0259736E-2,-2.717496E-3,-7.430144E-3,-6.547307E-3,-6.373224E-2,-1.6323837E-2,-5.447045E-3,5.603521E-3,1.2100035E-3,-7.2630827E-4,-2.5930062E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,49,-1,-1,51,53,55,-1,57,59,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,65,-1,-1,-1,-1,-1,67,69,71,73,-1,-1,-1,-1,-1,75,77,-1,-1,79,-1,-1,-1,-1,81,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6255295E1,6.4034863E0,7.204525E0,2.0416718E0,1.4686904E0,1.9963086E0,9.7237015E-1,6.33131E-1,3.501322E-1,5.2301955E-1,1.266144E0,8.511791E-1,5.2944547E-1,1.8881321E-1,5.327854E-1,5.542183E-2,5.2832603E-2,0E0,0E0,2.1901965E-1,3.0544496E-1,8.527855E-1,3.4574866E-2,0E0,0E0,8.206928E-1,5.3349733E-3,0E0,4.8265457E-2,0E0,0E0,0E0,5.3284645E-2,0E0,0E0,3.0037808E-1,4.9036473E-2,4.696718E-2,0E0,6.65114E-1,2.4957243E-1,0E0,0E0,1.232876E-1,2.131635E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.2301626E-2,0E0,0E0,0E0,0E0,0E0,3.421886E-1,4.376864E-2,5.864817E-2,2.5979936E-2,0E0,0E0,0E0,0E0,0E0,4.8802197E-2,1.7391467E-1,0E0,0E0,2.4458304E-2,0E0,0E0,0E0,0E0,4.4157207E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,26,26,28,28,32,32,35,35,36,36,37,37,39,39,40,40,43,43,44,44,52,52,58,58,59,59,60,60,61,61,67,67,68,68,71,71,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,50,-1,-1,52,54,56,-1,58,60,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,66,-1,-1,-1,-1,-1,68,70,72,74,-1,-1,-1,-1,-1,76,78,-1,-1,80,-1,-1,-1,-1,82,-1,-1,-1,-1,-1,-1],"split_conditions":[9.568E3,8.81041E5,7.651454E6,1.83E2,2.597379E6,5.345E3,1.99916E5,6.591E3,1.0241836E-1,6.490568E8,1.6942586E7,7.58587E8,3.636759E6,2.0697207E0,4.885431E11,1.17036E5,6.4573E4,-7.56337E-3,3.9976747E-3,6.39277E5,8.58E2,1.914442E6,6.09603E8,5.7682376E-3,2.921185E-2,2.9138386E7,4.93153E5,1.5736744E-2,5.36109E5,3.524897E-2,1.3306547E-2,-2.1630712E-2,8.33591E5,-1.0055833E-2,-3.2486243E-3,2.6437E4,1.02E3,9.1455296E8,5.447303E-3,4.0061934E9,4.667E3,1.5182144E-2,6.7145377E-3,2.511056E3,1.01E2,5.393875E-3,1.1630038E-2,1.2332693E-2,2.6627587E-2,-1.7257541E-2,-8.669945E-3,-4.38609E-4,2.7496557E8,-1.5763175E-3,-6.1687008E-3,-7.3196213E-3,-1.3894143E-3,5.2874917E-3,2.925808E6,2.0868E4,8.833334E6,1.3323E4,5.632744E-4,8.524628E-3,3.401479E-3,-1.2169977E-2,-6.1864955E-3,2.207E3,1.63E2,1.2894602E-3,-3.5482964E-3,3.3831064E7,1.0259736E-2,-2.717496E-3,-7.430144E-3,-6.547307E-3,2.466E3,-1.6323837E-2,-5.447045E-3,5.603521E-3,1.2100035E-3,-7.2630827E-4,-2.5930062E-3],"split_indices":[3,9,6,47,9,7,3,3,67,16,18,16,6,67,1,18,12,0,0,6,0,6,11,0,0,18,12,0,12,0,0,0,2,0,0,12,0,10,0,10,3,0,0,5,19,0,0,0,0,0,0,0,16,0,0,0,0,0,27,7,21,7,0,0,0,0,0,0,17,0,0,16,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.92E2,9.4E1,8.5E1,1.07E2,4.5E1,4.9E1,7.6E1,9E0,6E1,4.7E1,8E0,3.7E1,2.9E1,2E1,6.7E1,9E0,4E0,5E0,5E1,1E1,4E1,7E0,3E0,5E0,2.6E1,1.1E1,1.4E1,1.5E1,1.7E1,3E0,2.6E1,4.1E1,6E0,3E0,3.4E1,1.6E1,6E0,4E0,2.5E1,1.5E1,3E0,4E0,1.3E1,1.3E1,3E0,8E0,3E0,1.2E1,3.7E1,4E0,3E0,3.1E1,4E0,1.2E1,3E0,3E0,4E0,2.1E1,6E0,9E0,8E0,5E0,1E1,3E0,2.5E1,6E0,1.2E1,9E0,3E0,3E0,6E0,3E0,4E0,4E0,4E0,8E0,5E0,4E0,3E0,3E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[1.8320658E-3,-2.4947843E-1,4.9872065E-1,-3.949573E-1,-5.2180234E-2,2.7345237E-1,7.3893225E-1,-4.7012106E-1,-1.6294804E-1,-1.3973361E-1,4.191195E-1,1.9732615E-1,2.4488088E-2,5.327863E-1,8.860621E-1,-5.321484E-1,-1.0013182E-1,-5.38436E-2,-2.6791066E-1,-2.598378E-1,-6.602059E-2,2.2173865E-2,1.2803441E-1,2.1066656E-2,3.566045E-1,2.018188E-2,-2.3810448E-5,2.4711108E-2,4.044261E-2,-2.3146216E-3,-5.536044E-1,-2.746243E-1,3.8669037E-3,-5.336085E-3,2.6759129E-2,-1.1712583E-2,-3.0680485E-3,-2.7638423E-1,-3.4767159E-3,-1.0167567E-1,1.21686295E-1,5.1888294E-4,6.6933823E-3,1.0518808E-2,-4.80407E-2,2.0779097E-2,2.565347E-1,-5.755086E-1,-1.0833211E-2,-1.1341576E-2,-4.128196E-3,3.1823062E-3,-2.8354253E-3,-1.026441E-2,-5.3906506E-3,-1.6571157E-1,-5.6303922E-2,1.023135E-3,5.8315364E-3,-1.1003309E-1,4.3502133E-3,-7.3272495E-5,3.0844864E-1,-1.8976076E-2,-8.14285E-3,-3.0050962E-3,-7.2337706E-3,-1.216404E-1,-1.7140255E-3,-6.1130445E-3,-2.6385589E-2,1.11616105E-2,5.23038E-3,-5.2023004E-3,-6.8466924E-4,2.8500488E-2,-1.7947708E-3,1.1372224E-3,-3.1201774E-3,-2.536636E-4,2.4446477E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,47,49,-1,-1,51,-1,-1,53,-1,55,57,-1,-1,-1,59,-1,61,63,-1,-1,-1,-1,-1,-1,-1,65,67,-1,-1,69,-1,-1,71,-1,-1,-1,-1,73,75,-1,77,-1,-1,-1,-1,79,-1,-1,-1,-1,-1],"loss_changes":[3.2716354E1,4.9736633E0,4.5597973E0,1.667264E0,3.1533566E0,1.5934372E0,8.0178833E-1,1.6679249E0,2.8332382E-1,5.494529E-1,8.5403013E-1,1.1716089E0,0E0,1.0055757E0,4.377556E-2,5.921879E-1,4.942022E-1,1.3074222E-1,1.8062705E-1,3.0441046E-2,2.8677332E-1,0E0,5.924642E-2,4.593712E-1,4.7048855E-1,0E0,0E0,0E0,0E0,0E0,9.633064E-2,3.444588E-2,0E0,0E0,8.323643E-2,0E0,0E0,4.3794274E-2,0E0,9.235948E-2,3.1672418E-2,0E0,0E0,0E0,2.1928194E-1,0E0,2.4254847E-1,9.4926834E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.8800836E-2,7.825915E-2,0E0,0E0,8.853157E-2,0E0,0E0,2.751851E-2,0E0,0E0,0E0,0E0,3.6687434E-2,2.2816716E-2,0E0,3.8255446E-2,0E0,0E0,0E0,0E0,1.6133761E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,30,30,31,31,34,34,37,37,39,39,40,40,44,44,46,46,47,47,55,55,56,56,59,59,62,62,67,67,68,68,70,70,75,75],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,48,50,-1,-1,52,-1,-1,54,-1,56,58,-1,-1,-1,60,-1,62,64,-1,-1,-1,-1,-1,-1,-1,66,68,-1,-1,70,-1,-1,72,-1,-1,-1,-1,74,76,-1,78,-1,-1,-1,-1,80,-1,-1,-1,-1,-1],"split_conditions":[9.896E3,5.013E3,1.4066751E7,1.717383E6,1.83E2,2.0259231E9,1.1962493E9,1.83E2,1.82E2,1.37586E5,6.09603E8,7.5937E4,2.4488088E-2,2.33E2,1.2738551E4,3.7E1,6.909286E6,5.76E3,5.8480744E7,1.138E3,2.466E3,2.2173865E-2,2.0547E4,9.1416E4,3.178409E9,2.018188E-2,-2.3810448E-5,2.4711108E-2,4.044261E-2,-2.3146216E-3,1.164162E9,1.932E3,3.8669037E-3,-5.336085E-3,4.283E3,-1.1712583E-2,-3.0680485E-3,5.92953E5,-3.4767159E-3,1.9761268E-2,4.365884E6,5.1888294E-4,6.6933823E-3,1.0518808E-2,4.2524388E7,2.0779097E-2,2.2052716E9,2.8681312E8,-1.0833211E-2,-1.1341576E-2,-4.128196E-3,3.1823062E-3,-2.8354253E-3,-1.026441E-2,-5.3906506E-3,1.83687E5,4.411652E8,1.023135E-3,5.8315364E-3,2.486228E3,4.3502133E-3,-7.3272495E-5,3.389E3,-1.8976076E-2,-8.14285E-3,-3.0050962E-3,-7.2337706E-3,2.846864E6,1.068557E6,-6.1130445E-3,1.3366E4,1.11616105E-2,5.23038E-3,-5.2023004E-3,-6.8466924E-4,4.6E1,-1.7947708E-3,1.1372224E-3,-3.1201774E-3,-2.536636E-4,2.4446477E-3],"split_indices":[3,3,2,9,47,16,16,47,17,12,11,28,0,4,5,20,26,22,16,17,0,0,8,12,21,0,0,0,0,0,11,3,0,0,3,0,0,6,0,63,6,0,0,0,18,0,11,16,0,0,0,0,0,0,0,12,16,0,0,5,0,0,0,0,0,0,0,2,6,0,3,0,0,0,0,19,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,1.73E2,8.7E1,9.9E1,7.4E1,4.6E1,4.1E1,7.4E1,2.5E1,6.3E1,1.1E1,4.1E1,5E0,1.9E1,2.2E1,6.3E1,1.1E1,1.3E1,1.2E1,2.3E1,4E1,5E0,6E0,2E1,2.1E1,1.6E1,3E0,1.9E1,3E0,3E0,6E1,6E0,5E0,5E0,8E0,7E0,5E0,2E1,3E0,3.4E1,6E0,3E0,3E0,3E0,1.7E1,4E0,1.7E1,5.3E1,7E0,3E0,3E0,5E0,3E0,1.3E1,7E0,1.3E1,2.1E1,3E0,3E0,1.3E1,4E0,3E0,1.4E1,5E1,3E0,7E0,6E0,9E0,1.2E1,6E0,7E0,1E1,4E0,6E0,3E0,8E0,4E0,4E0,3E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-2.3305079E-2,-2.5792035E-1,5.041634E-1,-4.3200937E-1,-1.0178377E-1,2.4764395E-1,7.608942E-1,-5.3101724E-1,-2.687471E-1,-1.3047628E-1,1.5481165E-2,5.1628035E-1,1.0507096E-1,6.131044E-1,3.3586513E-2,1.6897662E-3,-5.6569093E-1,7.043997E-4,-2.9583114E-1,-1.0506692E-1,-1.873646E-2,2.4872476E-2,2.594767E-1,-4.042774E-3,1.4538862E-1,6.380511E-1,9.381087E-3,-2.0094754E-2,-1.4724124E-2,-3.385827E-1,-8.867077E-2,-1.5515974E-1,5.6313522E-2,1.0307961E-2,3.3306996E-3,3.6497496E-2,2.2280894E-1,2.212653E-2,1.2839186E-2,-1.1595372E-2,-6.527444E-3,-3.8248696E-4,-4.612398E-3,-8.2153596E-2,-2.6694962E-1,9.548801E-3,-9.896345E-3,-2.6698632E-3,8.5527085E-2,9.911043E-3,1.3186897E-1,-1.6086805E-1,8.5999854E-2,-3.4861946E-1,-5.5800032E-2,-1.1415819E-1,6.0436245E-2,4.0895338E-4,4.5464966E-3,1.1606559E-3,5.593317E-3,-1.1714521E-2,-1.0782042E-1,-5.8700703E-2,2.1995044E-1,-1.4681727E-2,-7.691534E-3,-3.087237E-3,5.8952934E-4,-5.1985965E-3,-1.7205362E-3,-1.84096E-3,4.9129026E-3,2.505274E-2,-1.48935E-1,-4.4607297E-3,5.453604E-4,8.691864E-3,3.295121E-3,1.8108403E-3,-3.9960252E-4,-7.429136E-3,-7.512678E-2,-4.0028957E-3,-2.2197772E-2,-2.2461712E-3,9.957558E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,-1,27,-1,29,31,-1,-1,33,-1,35,37,-1,-1,-1,39,41,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,51,53,-1,55,-1,57,-1,59,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,79,81,-1,-1,-1,-1,-1,-1,-1,83,-1,85,-1,-1],"loss_changes":[3.4161877E1,5.147725E0,5.4093685E0,1.3120747E0,1.768192E0,1.6309669E0,1.113945E0,1.1424847E0,2.8984594E-1,1.0633825E0,0E0,7.872529E-1,2.9725748E-1,5.4782867E-2,0E0,0E0,4.424858E-2,0E0,2.6223946E-1,7.686337E-1,0E0,0E0,5.2167773E-2,0E0,2.1105814E-1,4.4478416E-2,0E0,0E0,0E0,6.3154697E-3,2.7656827E-2,5.65686E-1,3.6892402E-1,0E0,0E0,7.736759E-2,7.547253E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.094595E-1,4.633944E-1,0E0,1.462264E-1,0E0,3.4722418E-2,0E0,2.986157E-2,2.9469126E-1,3.0365208E-1,1.2678385E-1,2.8511062E-2,1.4360569E-2,1.3886017E-1,0E0,0E0,0E0,0E0,0E0,1.457699E-1,5.068615E-2,1.9462228E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.882184E-3,1.03503674E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8507516E-2,0E0,1.9856075E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,16,16,18,18,19,19,22,22,24,24,25,25,29,29,30,30,31,31,32,32,35,35,36,36,43,43,44,44,46,46,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,62,62,63,63,64,64,73,73,74,74,82,82,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,-1,28,-1,30,32,-1,-1,34,-1,36,38,-1,-1,-1,40,42,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,52,54,-1,56,-1,58,-1,60,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,80,82,-1,-1,-1,-1,-1,-1,-1,84,-1,86,-1,-1],"split_conditions":[9.91E3,7.83E2,7.762619E6,4.445E3,1.1156849E9,4.94E4,2.12632E5,1.71E3,1.6028E4,1.33097E5,1.5481165E-2,3.4555535E9,1.6671127E3,1.2101E4,3.3586513E-2,1.6897662E-3,9.56E3,7.043997E-4,1.51731E5,4.112304E6,-1.873646E-2,2.4872476E-2,1.9002575E10,-4.042774E-3,3.47989E5,8.35E2,9.381087E-3,-2.0094754E-2,-1.4724124E-2,1.122053E6,2.17E2,1.6506441E9,2.160042E9,1.0307961E-2,3.3306996E-3,3.3587363E8,1.5688E4,2.212653E-2,1.2839186E-2,-1.1595372E-2,-6.527444E-3,-3.8248696E-4,-4.612398E-3,1.83E2,1.460109E6,9.548801E-3,2.785E3,-2.6698632E-3,1.3323E4,9.911043E-3,1.6121E4,1.0005E6,3.719E3,2.63E2,5.64E2,1E0,2.379217E6,4.0895338E-4,4.5464966E-3,1.1606559E-3,5.593317E-3,-1.1714521E-2,2.1E1,3.443128E7,8.2E1,-1.4681727E-2,-7.691534E-3,-3.087237E-3,5.8952934E-4,-5.1985965E-3,-1.7205362E-3,-1.84096E-3,4.9129026E-3,5.275E3,3.4814704E8,-4.4607297E-3,5.453604E-4,8.691864E-3,3.295121E-3,1.8108403E-3,-3.9960252E-4,-7.429136E-3,4E2,-4.0028957E-3,1.4603E4,-2.2461712E-3,9.957558E-4],"split_indices":[3,0,6,3,16,8,3,3,12,8,0,10,5,0,0,0,22,0,12,2,0,0,10,0,12,17,0,0,0,9,4,11,11,0,0,16,7,0,0,0,0,0,0,47,6,0,20,0,7,0,7,2,7,4,19,44,6,0,0,0,0,0,19,16,17,0,0,0,0,0,0,0,0,3,21,0,0,0,0,0,0,0,17,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.9E2,8.4E1,8.9E1,1.01E2,4.3E1,4.1E1,5.4E1,3.5E1,9.7E1,4E0,1.4E1,2.9E1,2.9E1,1.2E1,3E0,5.1E1,3E0,3.2E1,9.3E1,4E0,6E0,8E0,4E0,2.5E1,2.6E1,3E0,3E1,2.1E1,2.6E1,6E0,7.1E1,2.2E1,5E0,3E0,1.1E1,1.4E1,2E1,6E0,2.1E1,5E0,3E0,3E0,4.4E1,2.7E1,4E0,1.8E1,3E0,8E0,6E0,8E0,3E1,1.4E1,1.9E1,8E0,7E0,1.1E1,4E0,4E0,3E0,5E0,5E0,2.5E1,7E0,7E0,8E0,1.1E1,5E0,3E0,3E0,4E0,5E0,6E0,6E0,1.9E1,3E0,4E0,4E0,3E0,3E0,3E0,8E0,1.1E1,5E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-1.6856318E-2,-1.9231147E-1,6.5802914E-1,-3.5722032E-1,2.5582707E-2,5.440039E-1,3.7047878E-2,-4.1103187E-1,1.462834E-1,1.9277046E-2,-1.2083755E-2,3.5164472E-1,6.645397E-1,-5.40369E-1,-2.834667E-1,-7.285957E-2,1.646414E-2,-7.871604E-2,1.0943203E-1,1.8845424E-1,4.3757316E-1,2.3705948E-2,1.4621545E-2,-1.8014638E-2,-1.0475339E-2,-3.1183732E-1,2.9284726E-3,-9.210703E-3,2.6225578E-3,-1.520042E-1,3.7549708E-2,4.1096956E-2,1.5575885E-2,8.418842E-3,1.0205648E-3,1.7331777E-2,9.889333E-3,-3.699944E-1,-2.1230783E-1,-3.2314292E-1,-1.0845661E-1,1.8910044E-1,-1.0523172E-1,-1.602412E-2,1.6981314E-1,-1.3191074E-2,-7.515413E-3,-1.8349972E-4,-2.4301386E-1,-1.4800305E-2,-3.4025558E-3,1.0631272E-3,-1.3693543E-1,1.1412087E-3,7.6087248E-3,-1.1381347E-2,4.630442E-3,-7.3281534E-2,4.484984E-3,8.357694E-3,1.4813001E-3,-3.3329986E-3,-2.714096E-1,-1.667937E-1,-3.0929927E-4,4.278349E-3,-6.933569E-2,-3.0490123E-2,-6.2457277E-3,-9.639605E-3,-4.8214034E-3,-8.168432E-2,-2.0351382E-1,-2.8022397E-3,-1.1034866E-3,1.0025033E-3,-5.2094888E-2,-7.999879E-4,-3.5669429E-3,-7.1321046E-3,-3.3148555E-3,-7.0180625E-2,-2.388784E-4,-1.1446967E-3,-2.8086256E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,-1,37,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,45,47,49,51,53,55,57,59,-1,-1,-1,61,-1,-1,-1,63,-1,-1,-1,65,67,-1,-1,-1,-1,69,71,-1,-1,73,75,-1,-1,-1,77,79,-1,-1,-1,81,-1,-1,-1,-1,83,-1,-1,-1],"loss_changes":[3.328194E1,8.022126E0,2.633791E0,3.5011063E0,2.1018307E0,9.024401E-1,0E0,1.7441978E0,1.0776715E0,0E0,7.5352645E-1,2.1201396E-1,1.0410023E-1,4.9978256E-2,6.657629E-1,3.195666E-1,0E0,5.169619E-1,8.2453585E-1,9.481877E-2,5.4929256E-3,0E0,0E0,0E0,0E0,2.455945E-1,0E0,0E0,0E0,2.4070203E-1,5.3922653E-1,2.1594091E-1,0E0,0E0,0E0,0E0,0E0,8.9086056E-2,1.4452863E-1,1.5938908E-1,1.2943113E-1,7.8786105E-2,3.5632905E-1,1.9701281E-1,8.813715E-2,0E0,0E0,0E0,5.1103473E-2,0E0,0E0,0E0,9.724286E-2,0E0,0E0,0E0,1.041017E-1,7.675807E-2,0E0,0E0,0E0,0E0,9.74834E-3,4.9005866E-2,0E0,0E0,1.3637692E-3,1.8933231E-2,0E0,0E0,0E0,1.0489222E-2,2.595663E-3,0E0,0E0,0E0,7.5587574E-3,0E0,0E0,0E0,0E0,1.0345802E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,25,25,29,29,30,30,31,31,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,48,48,52,52,56,56,57,57,62,62,63,63,66,66,67,67,71,71,72,72,76,76,81,81],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,-1,38,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,46,48,50,52,54,56,58,60,-1,-1,-1,62,-1,-1,-1,64,-1,-1,-1,66,68,-1,-1,-1,-1,70,72,-1,-1,74,76,-1,-1,-1,78,80,-1,-1,-1,82,-1,-1,-1,-1,84,-1,-1,-1],"split_conditions":[2.3807E4,1.530129E6,1.2738551E4,1.83E2,3.28451E5,6.6569E4,3.7047878E-2,4.258E3,4.115E3,1.9277046E-2,4.3406326E8,1.04E2,3.91955E10,2.1811952E7,1.1408653E9,6.909286E6,1.646414E-2,4.580147E6,1.83E2,2.7179317E10,8.583224E10,2.3705948E-2,1.4621545E-2,-1.8014638E-2,-1.0475339E-2,7.31384E6,2.9284726E-3,-9.210703E-3,2.6225578E-3,3.757E3,2.01E2,3.6395932E9,1.5575885E-2,8.418842E-3,1.0205648E-3,1.7331777E-2,9.889333E-3,5.04569E5,3.4851824E8,1.2081E4,4.0061934E9,1.8E1,1.491777E6,1.3925089E9,1.08E2,-1.3191074E-2,-7.515413E-3,-1.8349972E-4,5.5659E4,-1.4800305E-2,-3.4025558E-3,1.0631272E-3,1.576081E6,1.1412087E-3,7.6087248E-3,-1.1381347E-2,2.421091E10,1.247E3,4.484984E-3,8.357694E-3,1.4813001E-3,-3.3329986E-3,2.3912651E8,1.02E3,-3.0929927E-4,4.278349E-3,3.0379E4,4.79E2,-6.2457277E-3,-9.639605E-3,-4.8214034E-3,7.514E3,1.9669E4,-2.8022397E-3,-1.1034866E-3,1.0025033E-3,1.1502255E10,-7.999879E-4,-3.5669429E-3,-7.1321046E-3,-3.3148555E-3,2.318509E6,-2.388784E-4,-1.1446967E-3,-2.8086256E-3],"split_indices":[3,9,5,47,6,3,0,3,3,0,16,19,21,18,16,26,0,2,47,10,13,0,0,0,0,18,0,0,0,3,17,21,0,0,0,0,0,6,10,7,10,17,21,16,4,0,0,0,8,0,0,0,6,0,0,0,10,0,0,0,0,0,21,0,0,0,7,17,0,0,0,7,7,0,0,0,10,0,0,0,0,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,2.22E2,5.7E1,1.26E2,9.6E1,4.8E1,9E0,1.14E2,1.2E1,5E0,9.1E1,2E1,2.8E1,5.5E1,5.9E1,8E0,4E0,5.9E1,3.2E1,8E0,1.2E1,1.9E1,9E0,4.9E1,6E0,5.5E1,4E0,3E0,5E0,3.6E1,2.3E1,2.8E1,4E0,5E0,3E0,5E0,7E0,3.3E1,2.2E1,6E0,3E1,1.1E1,1.2E1,2E1,8E0,2.4E1,9E0,3E0,1.9E1,3E0,3E0,5E0,2.5E1,3E0,8E0,3E0,9E0,1.5E1,5E0,4E0,4E0,4E0,1.5E1,2E1,5E0,3E0,6E0,1.2E1,3E0,1.1E1,4E0,7E0,1.3E1,3E0,3E0,3E0,9E0,3E0,4E0,1E1,3E0,6E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[-1.2414109E-2,-2.6702875E-1,4.058284E-1,-3.9342895E-1,-9.0546496E-2,2.5785366E-1,7.6983404E-1,-4.5288342E-1,-6.2521756E-2,1.4945705E-2,-1.2782072E-1,9.042226E-2,4.0319985E-1,6.008805E-1,3.290989E-2,-4.9314034E-1,-8.956533E-2,7.8798965E-2,-1.7430766E-2,-1.731229E-1,1.3913894E-1,-6.39812E-2,3.5968396E-1,4.7137713E-1,-2.7176074E-3,1.0815646E-2,6.626538E-1,-5.77063E-1,-4.2039007E-1,-4.164435E-3,-1.3346623E-3,-3.6153108E-2,9.42954E-3,-2.6132804E-1,-5.668352E-2,9.370171E-3,-1.1591524E-3,-1.3373774E-1,7.78305E-3,5.276335E-1,5.400265E-3,5.460233E-3,5.1322407E-1,2.4024129E-2,1.28361285E-2,-1.9518388E-2,-1.0919793E-2,-4.4682574E-1,-7.6921145E-3,-5.581895E-3,2.7120279E-3,-3.2271895E-1,-1.991489E-1,-8.7909E-2,4.9135354E-3,-1.1232772E-2,-4.4669475E-2,7.223253E-3,2.24987E-2,6.6775286E-1,3.8711917E-1,-6.425988E-3,-1.4867428E-2,-4.7279145E-3,-1.2530937E-2,-6.884558E-3,-2.7926066E-3,-1.4270654E-1,-2.6349224E-2,-1.3719779E-1,4.2621624E-3,3.7769753E-1,2.776024E-2,7.989539E-3,1.4254074E-2,-1.8840547E-1,-1.0270583E-3,-3.3749528E-3,3.581862E-2,-2.3509727E-3,-5.7454933E-3,6.0186475E-3,1.5257311E-2,-3.021274E-3,-7.0826816E-3,-3.2448876E-5,1.8707018E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,29,31,-1,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,-1,51,53,-1,-1,55,-1,57,-1,-1,59,-1,-1,-1,-1,61,-1,-1,-1,63,65,67,-1,-1,69,-1,-1,71,73,-1,-1,-1,-1,-1,-1,75,77,79,-1,81,-1,-1,-1,83,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0671186E1,3.9573984E0,5.6726303E0,2.0177708E0,1.6311729E0,1.8860478E0,7.084732E-1,1.2429256E0,1.1934263E0,0E0,8.9484584E-1,1.6028607E0,1.4360895E0,1.20952606E-1,0E0,2.3973465E-1,1.3784751E-2,3.552195E-1,0E0,6.2196136E-1,3.0325606E-1,5.655798E-1,3.6273205E-1,3.8014698E-1,0E0,0E0,5.5734634E-2,3.126335E-2,1.138401E-1,0E0,0E0,1.7987683E-1,0E0,6.933403E-2,1.9643918E-1,0E0,0E0,3.8697743E-1,0E0,2.0680821E-1,0E0,0E0,3.6507893E-1,0E0,0E0,0E0,0E0,5.2921772E-2,0E0,0E0,0E0,1.3012171E-1,1.4556646E-2,8.05729E-2,0E0,0E0,2.8032532E-1,0E0,0E0,4.082842E-1,4.4851065E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.9812874E-2,6.7196935E-2,1.6108304E-2,0E0,3.9929926E-2,0E0,0E0,0E0,6.260693E-3,0E0,0E0,6.627771E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,26,26,27,27,28,28,31,31,33,33,34,34,37,37,39,39,42,42,47,47,51,51,52,52,53,53,56,56,59,59,60,60,67,67,68,68,69,69,71,71,75,75,78,78],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,30,32,-1,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,-1,52,54,-1,-1,56,-1,58,-1,-1,60,-1,-1,-1,-1,62,-1,-1,-1,64,66,68,-1,-1,70,-1,-1,72,74,-1,-1,-1,-1,-1,-1,76,78,80,-1,82,-1,-1,-1,84,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,5.013E3,6.117756E9,1.83E2,2.5961E4,2.943003E3,3.391351E6,7.11868E5,1.64354E9,1.4945705E-2,2.328E3,2.9138386E7,2.62E2,7.4399194E8,3.290989E-2,1.65606E5,2.346E3,2.748775E6,-1.7430766E-2,1.267716E7,2.01E2,1.0886205E9,1.5688E4,4.01145E5,-2.7176074E-3,1.0815646E-2,2.0685E4,1.164162E9,2.207745E9,-4.164435E-3,-1.3346623E-3,1.852E3,9.42954E-3,1.05604E5,1.1408653E9,9.370171E-3,-1.1591524E-3,4.8749E4,7.78305E-3,1.607E3,5.400265E-3,5.460233E-3,1.619E3,2.4024129E-2,1.28361285E-2,-1.9518388E-2,-1.0919793E-2,3.7E1,-7.6921145E-3,-5.581895E-3,2.7120279E-3,2.351E3,3.0854016E8,8.74E2,4.9135354E-3,-1.1232772E-2,4.365884E6,7.223253E-3,2.24987E-2,1.062E3,8.23838E5,-6.425988E-3,-1.4867428E-2,-4.7279145E-3,-1.2530937E-2,-6.884558E-3,-2.7926066E-3,2.7022E4,2.8E2,1.4828954E10,4.2621624E-3,5.4251177E3,2.776024E-2,7.989539E-3,1.4254074E-2,3.71E2,-1.0270583E-3,-3.3749528E-3,1.4603E4,-2.3509727E-3,-5.7454933E-3,6.0186475E-3,1.5257311E-2,-3.021274E-3,-7.0826816E-3,-3.2448876E-5,1.8707018E-3],"split_indices":[12,3,27,47,12,5,12,6,11,0,0,18,4,16,0,18,0,18,0,18,17,16,7,12,0,0,7,11,10,0,0,3,0,12,16,0,0,22,0,0,0,0,0,0,0,0,0,20,0,0,0,20,27,0,0,0,6,0,0,0,12,0,0,0,0,0,0,28,17,10,0,5,0,0,0,17,0,0,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.78E2,1.08E2,1.03E2,7.5E1,7.8E1,3E1,8.7E1,1.6E1,4E0,7.1E1,3.7E1,4.1E1,2E1,1E1,7.8E1,9E0,1.3E1,3E0,6.1E1,1E1,2.4E1,1.3E1,3.6E1,5E0,5E0,1.5E1,3.3E1,4.5E1,4E0,5E0,9E0,4E0,3.4E1,2.7E1,5E0,5E0,2E1,4E0,6E0,7E0,5E0,3.1E1,1E1,5E0,2.8E1,5E0,3.8E1,7E0,4E0,5E0,1.5E1,1.9E1,2.4E1,3E0,5E0,1.5E1,3E0,3E0,1.2E1,1.9E1,3E0,3.5E1,5E0,1E1,1.6E1,3E0,1.2E1,1.2E1,1E1,5E0,6E0,6E0,7E0,1.2E1,8E0,4E0,5E0,7E0,5E0,5E0,3E0,3E0,3E0,5E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-3.4024164E-2,-2.5033942E-1,4.3022996E-1,-4.5414832E-1,-1.2800682E-1,2.0764732E-1,6.2066376E-1,-4.92254E-1,1.5146972E-3,-1.7722169E-1,2.6430944E-1,5.4657392E-2,3.8085288E-1,4.912E-1,8.4889126E-1,-1.7948933E-2,-3.7477583E-1,-2.4510151E-1,-2.0277904E-2,2.4114596E-2,-4.545465E-3,-7.086317E-3,7.7597788E-3,7.732941E-4,4.6855873E-1,5.481809E-1,2.7834195E-1,1.0124943E0,3.378502E-3,-4.46904E-3,-1.2826759E-2,5.4697176E-3,-2.7051863E-1,1.0182582E-2,-7.5704575E-2,-4.3307734E-3,7.0387736E-2,-8.9410186E-2,9.431911E-2,2.199863E-2,3.0876455E-1,9.69732E-3,1.869751E-2,4.388896E-3,1.0513377E-2,4.0901385E-2,2.2284565E-2,-2.285124E-1,-4.8268834E-1,-3.462241E-2,-1.0753787E-2,3.8114851E-3,1.5350379E-4,-4.4478015E-3,9.810453E-4,4.8550408E-4,4.979357E-3,1.1469834E-2,5.807613E-3,-2.6666176E-1,-4.7334976E-2,-1.8576378E-2,-5.574641E-3,-9.676176E-2,1.5674171E-1,-3.394407E-1,-1.7739402E-1,2.0935964E-3,-4.4065057E-3,-1.3192473E-1,-7.589438E-3,1.3250746E-3,7.504292E-3,-3.8942516E-1,-7.870459E-3,-2.3350761E-4,-2.0026515E-1,-1.518976E-1,-1.2561729E-3,1.3391968E-3,-1.766715E-3,-1.3793428E-2,-7.008382E-3,-7.4883346E-3,-1.145353E-1,-2.0413508E-3,-5.611967E-3,-1.8545579E-3,-4.4151894E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,29,31,33,-1,35,37,-1,-1,39,41,43,45,-1,-1,-1,-1,47,-1,49,-1,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,65,67,-1,-1,69,71,73,75,-1,-1,77,79,-1,-1,81,-1,-1,83,85,-1,-1,-1,-1,-1,-1,87,-1,-1,-1,-1],"loss_changes":[2.8427622E1,4.7538347E0,3.6859741E0,1.3982754E0,2.4023924E0,1.1186281E0,1.0012894E0,3.0545044E-1,0E0,1.1562407E0,1.8283578E0,2.7794302E-1,6.1466384E-1,2.4767017E-1,1.8410864E0,0E0,9.569907E-2,8.476758E-1,6.5830266E-1,0E0,1.0688083E-1,1.6704674E-1,0E0,0E0,3.3832526E-1,2.6778221E-2,1.7156363E-2,1.9493866E-1,0E0,0E0,0E0,0E0,5.3139544E-1,0E0,3.068757E-1,0E0,2.1493368E-2,6.852452E-2,4.071986E-2,0E0,2.8588772E-3,0E0,0E0,0E0,0E0,0E0,0E0,4.159093E-1,2.214961E-1,3.3623016E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7112293E-1,1.2967624E-1,0E0,0E0,6.484559E-2,5.2205667E-2,3.2901525E-2,9.582949E-2,0E0,0E0,2.190432E-2,1.857023E-2,0E0,0E0,1.4086008E-2,0E0,0E0,3.4409523E-2,1.2776643E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.4019182E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,21,21,24,24,25,25,26,26,27,27,32,32,34,34,36,36,37,37,38,38,40,40,47,47,48,48,49,49,59,59,60,60,63,63,64,64,65,65,66,66,69,69,70,70,73,73,76,76,77,77,84,84],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,30,32,34,-1,36,38,-1,-1,40,42,44,46,-1,-1,-1,-1,48,-1,50,-1,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,66,68,-1,-1,70,72,74,76,-1,-1,78,80,-1,-1,82,-1,-1,84,86,-1,-1,-1,-1,-1,-1,88,-1,-1,-1,-1],"split_conditions":[1.0126E4,6.97149E5,2.6865E4,1.83E2,7.164074E8,4.29906E5,1.99916E5,3.561E3,1.5146972E-3,1.22661E6,2.4E1,1.7833586E9,2.486228E3,1.09E2,2.3177024E7,-1.7948933E-2,8.27E4,2.9169E4,1.0382E4,2.4114596E-2,9.1455296E8,1.4947E4,7.7597788E-3,7.732941E-4,7.149075E-1,1.111798E9,1.306E4,3.7822556E11,3.378502E-3,-4.46904E-3,-1.2826759E-2,5.4697176E-3,1.12776E5,1.0182582E-2,1.1148E5,-4.3307734E-3,4.3016243E9,2.4292E4,1.1443E4,2.199863E-2,2.144E4,9.69732E-3,1.869751E-2,4.388896E-3,1.0513377E-2,4.0901385E-2,2.2284565E-2,2.4552E6,2.0201286E9,3.718604E6,-1.0753787E-2,3.8114851E-3,1.5350379E-4,-4.4478015E-3,9.810453E-4,4.8550408E-4,4.979357E-3,1.1469834E-2,5.807613E-3,5.55386E5,3.388763E7,-1.8576378E-2,-5.574641E-3,1.914442E6,4.365884E6,6.00237E5,7.514E3,2.0935964E-3,-4.4065057E-3,7.9E3,1.958E3,1.3250746E-3,7.504292E-3,9.0123E4,-7.870459E-3,-2.3350761E-4,5.701E3,9.6806134E2,-1.2561729E-3,1.3391968E-3,-1.766715E-3,-1.3793428E-2,-7.008382E-3,-7.4883346E-3,7.9031E4,-2.0413508E-3,-5.611967E-3,-1.8545579E-3,-4.4151894E-3],"split_indices":[3,9,3,47,16,12,3,3,0,6,4,16,5,4,18,0,6,12,7,0,10,7,0,0,66,16,7,13,0,0,0,0,22,0,8,0,10,20,3,0,3,0,0,0,0,0,0,2,11,6,0,0,0,0,0,0,0,0,0,18,16,0,0,6,6,6,7,0,0,3,20,0,0,12,0,0,3,5,0,0,0,0,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.92E2,8.9E1,7.1E1,1.21E2,4.2E1,4.7E1,6.6E1,5E0,1.08E2,1.3E1,2.3E1,1.9E1,3.2E1,1.5E1,4E1,2.6E1,7.5E1,3.3E1,4E0,9E0,1.8E1,5E0,4E0,1.5E1,2.4E1,8E0,1.2E1,3E0,3E0,2.3E1,4E0,7.1E1,4E0,2.9E1,3E0,6E0,1E1,8E0,5E0,1E1,4E0,2E1,3E0,5E0,5E0,7E0,6.1E1,1E1,2.6E1,3E0,3E0,3E0,7E0,3E0,4E0,4E0,6E0,4E0,5E1,1.1E1,7E0,3E0,2E1,6E0,2.6E1,2.4E1,5E0,6E0,1.4E1,6E0,3E0,3E0,1.5E1,1.1E1,3E0,2.1E1,1.1E1,3E0,3E0,3E0,1.1E1,4E0,1.4E1,7E0,3E0,8E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.4144151E-2,-2.000922E-1,4.7471538E-1,-4.2058647E-1,-7.277834E-2,2.3858188E-1,6.335243E-1,-4.4598565E-1,4.5192335E-4,-1.0706111E-1,3.924118E-1,2.1199401E-2,1.2612359E-1,3.7622795E-2,5.8016527E-1,-5.074381E-1,-3.0961898E-1,-1.7686686E-1,5.1236963E-3,2.0030264E-3,2.0733364E-2,1.6922715E-1,-2.8604097E-3,4.0356705E-1,7.171221E-1,-1.6830428E-2,-8.896326E-3,-2.7961533E-3,-1.0857471E-2,-2.1941642E-1,1.5481527E-1,1.4879511E-1,-1.16641484E-1,-2.0906883E-3,2.1827467E-1,4.6192E-1,-5.8796233E-4,2.4068473E-2,1.11559415E-2,-1.7679043E-1,-5.063562E-1,8.938537E-3,1.5159893E-3,3.8580787E-1,3.5802327E-2,-2.482609E-1,-5.508376E-2,-3.1743916E-3,3.0566216E-3,9.050575E-3,1.476722E-1,5.66893E-3,5.156483E-1,7.0773795E-2,-2.0615192E-1,-6.3808872E-3,-2.0973817E-2,5.6330874E-3,1.6099732E-2,5.7262983E-3,-4.495826E-2,-1.1777939E-2,-3.3636645E-3,-4.291297E-3,-8.721603E-3,5.9979793E-2,7.2543607E-3,1.7881276E-2,7.637035E-3,-1.0537613E-3,5.0404975E-3,-2.2551543E-1,-5.4351625E-4,-7.211202E-3,5.0195985E-2,-2.68515E-3,2.2029255E-2,6.8721117E-4,2.6914913E-3,-8.776532E-2,-2.4689312E-1,-1.976554E-3,4.1664536E-3,2.4342504E-3,-1.5029028E-3,-8.249697E-4,-3.860143E-3,-3.2411104E-1,-2.0376463E-1,-4.129309E-3,-1.1910913E-2,-2.3211427E-1,-1.3627303E-1,-2.733251E-3,-8.189114E-3,-5.2336506E-3,-1.595537E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,-1,35,37,-1,-1,-1,-1,39,41,43,45,47,49,51,-1,-1,-1,53,55,-1,-1,57,59,61,63,-1,-1,-1,65,-1,67,69,71,-1,-1,-1,-1,-1,73,-1,-1,-1,75,77,-1,-1,-1,-1,-1,79,-1,-1,81,-1,83,-1,-1,85,87,-1,-1,-1,-1,-1,-1,89,91,-1,-1,93,95,-1,-1,-1,-1],"loss_changes":[2.8906948E1,5.583785E0,3.3363628E0,8.121195E-1,2.083357E0,1.7590487E0,8.928261E-1,4.2111206E-1,0E0,9.406434E-1,7.078928E-1,0E0,3.115722E-1,0E0,9.682808E-1,1.9443512E-2,1.1870623E-1,1.0783262E0,8.395819E-1,0E0,0E0,2.2732967E-1,0E0,6.2381744E-1,5.8363914E-2,0E0,0E0,0E0,0E0,7.177336E-1,1.0604447E-1,5.753633E-1,1.9698182E-1,7.4939586E-2,5.173099E-2,2.4701643E-1,0E0,0E0,0E0,4.4345248E-1,2.288208E-1,0E0,0E0,8.121967E-2,1.9160748E-1,9.706196E-2,6.775038E-2,0E0,0E0,0E0,6.824243E-2,0E0,6.794882E-2,6.730707E-2,1.904304E-1,0E0,0E0,0E0,0E0,0E0,1.9867612E-1,0E0,0E0,0E0,3.16985E-2,4.606249E-3,0E0,0E0,0E0,0E0,0E0,1.19680405E-1,0E0,0E0,7.869521E-2,0E0,4.0360626E-2,0E0,0E0,1.2910992E-2,7.04999E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.943238E-2,2.385652E-2,0E0,0E0,4.0748E-2,1.704213E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,21,21,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,43,43,44,44,45,45,46,46,50,50,52,52,53,53,54,54,60,60,64,64,65,65,71,71,74,74,76,76,79,79,80,80,87,87,88,88,91,91,92,92],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,-1,36,38,-1,-1,-1,-1,40,42,44,46,48,50,52,-1,-1,-1,54,56,-1,-1,58,60,62,64,-1,-1,-1,66,-1,68,70,72,-1,-1,-1,-1,-1,74,-1,-1,-1,76,78,-1,-1,-1,-1,-1,80,-1,-1,82,-1,84,-1,-1,86,88,-1,-1,-1,-1,-1,-1,90,92,-1,-1,94,96,-1,-1,-1,-1],"split_conditions":[1.1026E4,8.03052E5,7.469149E6,1.83E2,1.0886205E9,5.345E3,4.06635E5,3.969E3,4.5192335E-4,1.98168E5,1.83E2,2.1199401E-2,1E0,3.7622795E-2,1.1962493E9,2.252272E8,9.5398E4,1.333505E6,1.99E2,2.0030264E-3,2.0733364E-2,5.1012262E8,-2.8604097E-3,2.264E5,4.66328E5,-1.6830428E-2,-8.896326E-3,-2.7961533E-3,-1.0857471E-2,1.13472E5,1.5914E4,1.1268727E3,2.0037E4,2.536742E6,8.4017466E8,2.93968E5,-5.8796233E-4,2.4068473E-2,1.11559415E-2,3.2416E4,1.0105651E8,8.938537E-3,1.5159893E-3,7.848431E2,1.619794E6,6.6E1,4.73E2,-3.1743916E-3,3.0566216E-3,9.050575E-3,1.7833586E9,5.66893E-3,7.3512E5,3.5888664E7,2.866171E6,-6.3808872E-3,-2.0973817E-2,5.6330874E-3,1.6099732E-2,5.7262983E-3,5.7312E4,-1.1777939E-2,-3.3636645E-3,-4.291297E-3,1.5013646E3,1.4468E4,7.2543607E-3,1.7881276E-2,7.637035E-3,-1.0537613E-3,5.0404975E-3,4.64E2,-5.4351625E-4,-7.211202E-3,3.28616E6,-2.68515E-3,1.9812842E3,6.8721117E-4,2.6914913E-3,2.25E2,1.49015E5,-1.976554E-3,4.1664536E-3,2.4342504E-3,-1.5029028E-3,-8.249697E-4,-3.860143E-3,1.1728E4,7.0243E4,-4.129309E-3,-1.1910913E-2,6.4573E4,6.00251E8,-2.733251E-3,-8.189114E-3,-5.2336506E-3,-1.595537E-3],"split_indices":[3,9,6,47,16,7,12,3,0,12,47,0,36,0,16,16,6,6,17,0,0,16,0,3,19,0,0,0,0,22,7,5,28,6,16,28,0,0,0,12,16,0,0,5,21,4,17,0,0,0,16,0,28,16,2,0,0,0,0,0,22,0,0,0,5,3,0,0,0,0,0,28,0,0,2,0,5,0,0,4,18,0,0,0,0,0,0,18,8,0,0,12,16,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,1.99E2,9.2E1,7.2E1,1.27E2,3.8E1,5.4E1,6.8E1,4E0,1.19E2,8E0,7E0,3.1E1,3E0,5.1E1,4.5E1,2.3E1,7.3E1,4.6E1,4E0,4E0,2.6E1,5E0,2.4E1,2.7E1,4.1E1,4E0,3E0,2E1,6.5E1,8E0,2.1E1,2.5E1,6E0,2E1,2.1E1,3E0,2.4E1,3E0,5.8E1,7E0,3E0,5E0,6E0,1.5E1,7E0,1.8E1,3E0,3E0,9E0,1.1E1,4E0,1.7E1,6E0,5.2E1,3E0,4E0,3E0,3E0,5E0,1E1,3E0,4E0,6E0,1.2E1,6E0,5E0,1.4E1,3E0,3E0,3E0,4.7E1,5E0,3E0,7E0,3E0,9E0,3E0,3E0,7E0,4E1,3E0,4E0,5E0,4E0,3E0,4E0,1.2E1,2.8E1,3E0,9E0,1.8E1,1E1,3E0,1.5E1,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-1.09050805E-2,-2.0468327E-1,4.5521516E-1,-3.8511145E-1,-7.2979204E-2,2.6130146E-1,6.7226166E-1,-4.27238E-1,-6.287498E-2,-1.3272478E-1,1.9376169E-1,3.4025785E-1,-1.0110347E-1,1.5974859E-2,8.708046E-1,-4.515432E-1,-2.332763E-3,-7.0607313E-3,5.432487E-3,-1.7884022E-1,5.3761657E-2,-1.9797372E-2,5.699647E-1,2.5590777E-1,1.77312E-2,3.7256232E-3,-7.365372E-3,1.265238E-2,9.875653E-1,-1.714327E-2,-3.9155233E-1,-1.5724519E-1,-1.7351035E-2,1.6037507E-1,-9.597254E-3,2.4510575E-3,-7.53836E-2,2.3618924E-2,7.1694893E-3,4.4636163E-1,1.4788547E-1,1.6057577E-2,3.490228E-2,-4.0763077E-1,-7.1389913E-3,-1.7346732E-1,5.6029554E-3,2.2897242E-1,-1.1695398E-3,-4.5960057E-3,-2.1419516E-4,1.7585041E-2,5.1459675E-3,-4.926623E-3,2.0999542E-1,-1.3510554E-2,-6.336809E-3,-2.2129159E-1,-1.0307744E-1,2.68373E-1,2.6741936E-3,2.8116235E-1,-5.4842705E-4,6.317115E-4,-2.5437832E-1,-1.4712021E-2,-1.4445564E-1,4.842947E-3,1.070662E-2,1.3233876E-2,2.0105872E-1,-2.8592157E-3,-2.7175355E-1,-6.61694E-2,2.957546E-3,-1.6450056E-1,-5.8389426E-4,7.7117397E-3,3.6896537E-3,-3.069399E-1,-2.0207638E-1,-7.562708E-4,-3.3145102E-3,-3.473985E-3,-6.7302203E-3,-5.1995255E-3,-1.0725244E-2,-3.092443E-3,-7.1540927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,-1,31,33,35,37,39,-1,-1,-1,-1,41,-1,43,45,-1,47,-1,-1,49,-1,-1,51,53,-1,-1,55,-1,57,-1,59,-1,-1,-1,-1,-1,-1,61,-1,-1,63,65,67,-1,69,-1,-1,71,73,75,-1,-1,-1,77,-1,79,81,-1,83,-1,-1,-1,85,87,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5835499E1,4.772045E0,3.3597527E0,1.1269884E0,1.9062345E0,1.3506536E0,9.304619E-1,6.1397743E-1,4.357373E-1,8.4380543E-1,1.7787983E0,5.3403664E-1,2.7573043E-1,0E0,5.6010246E-1,1.3177872E-1,0E0,0E0,0E0,5.2195835E-1,7.981942E-1,8.573704E-2,2.9156613E-1,5.306144E-1,0E0,0E0,0E0,0E0,7.363415E-2,0E0,2.9696465E-2,4.3328583E-1,0E0,2.242184E-1,0E0,0E0,4.5372356E-2,0E0,0E0,1.9900012E-1,4.0596622E-1,0E0,0E0,1.6796589E-2,0E0,2.1956658E-1,0E0,4.6684027E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7946365E-1,0E0,0E0,3.397801E-1,1.1122322E-1,1.8137813E-2,0E0,5.268097E-2,0E0,0E0,8.229208E-2,6.6414535E-2,5.0187975E-2,0E0,0E0,0E0,5.83148E-3,0E0,1.8880367E-2,1.0144506E-2,0E0,2.283752E-2,0E0,0E0,0E0,2.2536397E-2,8.591294E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,19,19,20,20,21,21,22,22,23,23,28,28,30,30,31,31,33,33,36,36,39,39,40,40,43,43,45,45,47,47,54,54,57,57,58,58,59,59,61,61,64,64,65,65,66,66,70,70,72,72,73,73,75,75,79,79,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,-1,32,34,36,38,40,-1,-1,-1,-1,42,-1,44,46,-1,48,-1,-1,50,-1,-1,52,54,-1,-1,56,-1,58,-1,60,-1,-1,-1,-1,-1,-1,62,-1,-1,64,66,68,-1,70,-1,-1,72,74,76,-1,-1,-1,78,-1,80,82,-1,84,-1,-1,-1,86,88,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0773E4,8.81041E5,1.021304E6,1.83E2,7.164074E8,2.51E2,2.18449E5,4.3199274E8,2.09153E5,4.580147E6,1.83E2,3.7868384E3,6.5891E4,1.5974859E-2,1.29E2,1.34028E5,-2.332763E-3,-7.0607313E-3,5.432487E-3,1.16215E5,1E0,2.2368E4,3.0257034E8,1.55E2,1.77312E-2,3.7256232E-3,-7.365372E-3,1.265238E-2,2.531174E6,-1.714327E-2,2.83098E5,7.4198804E9,-1.7351035E-2,3.23E2,-9.597254E-3,2.4510575E-3,8.172629E8,2.3618924E-2,7.1694893E-3,5.660314E6,6.8001E4,1.6057577E-2,3.490228E-2,5.244E3,-7.1389913E-3,1.859752E6,5.6029554E-3,8.597307E-2,-1.1695398E-3,-4.5960057E-3,-2.1419516E-4,1.7585041E-2,5.1459675E-3,-4.926623E-3,1.7867039E10,-1.3510554E-2,-6.336809E-3,3.2416E4,6.2E1,9.6671375E2,2.6741936E-3,1.1828E4,-5.4842705E-4,6.317115E-4,8.95E3,7.13E2,2.0988E4,4.842947E-3,1.070662E-2,1.3233876E-2,1.4291E4,-2.8592157E-3,5.097E3,3.28616E6,2.957546E-3,1.6010206E10,-5.8389426E-4,7.7117397E-3,3.6896537E-3,1.245E4,5.324E3,-7.562708E-4,-3.3145102E-3,-3.473985E-3,-6.7302203E-3,-5.1995255E-3,-1.0725244E-2,-3.092443E-3,-7.1540927E-3],"split_indices":[3,9,12,47,16,4,3,16,18,2,47,5,8,0,17,18,0,0,0,28,44,28,11,17,0,0,0,0,12,0,6,23,0,17,0,0,23,0,0,6,8,0,0,3,0,9,0,63,0,0,0,0,0,0,10,0,0,12,4,5,0,3,0,0,28,19,7,0,0,0,7,0,3,2,0,10,0,0,0,7,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.01E2,8.3E1,8.4E1,1.17E2,4.5E1,3.8E1,7.4E1,1E1,9.6E1,2.1E1,3.7E1,8E0,2.2E1,1.6E1,6.9E1,5E0,6E0,4E0,7.7E1,1.9E1,1.4E1,7E0,2.8E1,9E0,3E0,5E0,4E0,1.2E1,2.6E1,4.3E1,7.4E1,3E0,1.5E1,4E0,5E0,9E0,4E0,3E0,9E0,1.9E1,3E0,9E0,3.8E1,5E0,7.1E1,3E0,1.1E1,4E0,4E0,5E0,6E0,3E0,3E0,1.6E1,3.5E1,3E0,4.1E1,3E1,8E0,3E0,1.2E1,4E0,5E0,3.6E1,1E1,2E1,4E0,4E0,3E0,9E0,4E0,3.2E1,7E0,3E0,1.7E1,3E0,5E0,4E0,1.9E1,1.3E1,4E0,3E0,9E0,8E0,4E0,1.5E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-8.3573785E-4,-2.2178674E-1,4.2374495E-1,-2.934358E-1,-6.7027155E-3,1.11993544E-1,5.5208594E-1,-3.371973E-1,-5.5108245E-2,4.0349734E-1,-7.1809486E-2,-2.721815E-3,2.93853E-1,4.7410497E-1,7.953876E-1,-4.153568E-1,-1.8564917E-1,-1.305246E-1,1.1414356E-2,5.678624E-3,1.7050656E-2,-1.2761316E-1,5.130407E-3,1.3919418E-1,-8.984537E-2,3.803605E-1,3.11172E-3,4.1042827E-3,4.9785763E-1,9.449656E-1,3.5540517E-3,-4.3062437E-3,-4.2824677E-1,-2.231146E-1,-7.914586E-2,1.2318902E-3,-1.4810821E-2,-1.9008721E-1,-3.7640143E-2,1.931198E-3,5.623867E-3,1.2960661E-2,-8.270217E-3,1.4648841E-2,6.1757667E-3,2.0207064E-2,4.247154E-1,3.61815E-2,2.0308074E-2,-1.6356679E-2,-3.7221995E-1,-3.4145159E-3,-2.51642E-1,-3.5475104E-3,-7.8983605E-4,-6.970683E-3,9.187381E-2,-2.1370898E-1,-1.167412E-3,-6.3085E-2,4.086818E-3,3.710367E-3,-1.8477928E-3,8.920555E-3,1.5872695E-2,-1.2253879E-2,-5.8522113E-3,-9.09962E-3,-5.690841E-3,-7.139731E-4,4.882217E-3,-7.7735805E-3,-1.8048867E-3,-7.684428E-4,-2.4618048E-3,-7.8108086E-4,1.0112941E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,-1,37,-1,39,41,43,-1,-1,45,47,-1,-1,49,51,53,55,-1,57,59,-1,-1,61,-1,-1,-1,-1,63,-1,-1,-1,65,-1,67,-1,-1,-1,69,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7956003E1,3.018505E0,4.0192394E0,1.5145655E0,1.3643149E0,6.475242E-1,9.889488E-1,1.3916607E0,7.87862E-1,1.07168674E-1,5.880034E-1,2.5979042E-1,1.6792369E-1,4.057808E-1,1.5349808E0,2.366991E-1,1.5790987E-1,9.1252303E-1,0E0,0E0,0E0,1.9520032E-1,0E0,1.2022942E-2,2.2663112E-1,2.5377631E-2,0E0,0E0,2.3696518E-1,2.1951675E-2,0E0,0E0,1.5554714E-1,8.014703E-2,1.934769E-2,3.357458E-1,0E0,6.855267E-2,1.7245602E-2,0E0,0E0,7.140702E-2,0E0,0E0,0E0,0E0,2.368207E-1,0E0,0E0,0E0,9.776115E-3,0E0,8.124709E-3,0E0,0E0,0E0,8.52095E-2,6.531036E-2,0E0,3.426589E-3,6.1865593E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,23,23,24,24,25,25,28,28,29,29,32,32,33,33,34,34,35,35,37,37,38,38,41,41,46,46,50,50,52,52,56,56,57,57,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,-1,38,-1,40,42,44,-1,-1,46,48,-1,-1,50,52,54,56,-1,58,60,-1,-1,62,-1,-1,-1,-1,64,-1,-1,-1,66,-1,68,-1,-1,-1,70,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.618673E6,4.0317974E8,2.7797654E3,1.83E2,7.284E3,5.1559043E8,1.1741724E4,4.54205E5,1.02E2,1.595E3,1.1613266E9,2.01E2,1.0667833E0,4.01145E5,2.3177024E7,1.5E1,1.080557E6,4.3683886E9,1.1414356E-2,5.678624E-3,1.7050656E-2,3.4814704E8,5.130407E-3,3.3076E4,3.35495E5,1E2,3.11172E-3,4.1042827E-3,2.9568602E10,1.3887953E0,3.5540517E-3,-4.3062437E-3,1.04184E5,1.723E3,1.521867E6,1.852E3,-1.4810821E-2,8.5731603E8,8.5591583E9,1.931198E-3,5.623867E-3,7.6E1,-8.270217E-3,1.4648841E-2,6.1757667E-3,2.0207064E-2,6.6569E4,3.61815E-2,2.0308074E-2,-1.6356679E-2,7.166E3,-3.4145159E-3,5.15E2,-3.5475104E-3,-7.8983605E-4,-6.970683E-3,1.645E3,1.484E3,-1.167412E-3,6.832E3,2.783874E6,3.710367E-3,-1.8477928E-3,8.920555E-3,1.5872695E-2,-1.2253879E-2,-5.8522113E-3,-9.09962E-3,-5.690841E-3,-7.139731E-4,4.882217E-3,-7.7735805E-3,-1.8048867E-3,-7.684428E-4,-2.4618048E-3,-7.8108086E-4,1.0112941E-3],"split_indices":[2,16,5,47,7,27,5,6,4,0,16,17,64,12,18,28,6,10,0,0,0,21,0,7,12,4,0,0,23,59,0,0,6,20,6,3,0,11,10,0,0,4,0,0,0,0,3,0,0,0,3,0,17,0,0,0,20,17,0,3,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.96E2,1.95E2,1.01E2,1.46E2,4.9E1,3E1,7.1E1,1.23E2,2.3E1,6E0,4.3E1,1.9E1,1.1E1,5.6E1,1.5E1,8E1,4.3E1,2E1,3E0,3E0,3E0,3.5E1,8E0,7E0,1.2E1,7E0,4E0,4E0,5.2E1,1.2E1,3E0,4E0,7.6E1,3.1E1,1.2E1,1.5E1,5E0,2E1,1.5E1,3E0,4E0,8E0,4E0,4E0,3E0,1.6E1,3.6E1,6E0,6E0,2.8E1,4.8E1,7E0,2.4E1,7E0,5E0,4E0,1.1E1,1.7E1,3E0,9E0,6E0,3E0,5E0,1.3E1,2.3E1,4.5E1,3E0,1.5E1,9E0,4E0,7E0,1.4E1,3E0,3E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[1.2216616E-3,-1.7132634E-1,5.388535E-1,-2.6285332E-1,1.4410907E-1,3.7909058E-1,7.5925314E-1,-3.8768828E-1,-1.04553275E-1,6.6720457E-3,4.8722506E-1,2.4688074E-1,4.5911804E-1,3.4592822E-2,5.243118E-1,-4.1753504E-1,-6.272593E-2,5.5325065E-2,-1.3289683E-1,-8.22922E-2,1.7040049E-1,2.1669758E-2,4.0896516E-3,1.09939456E-1,1.1961628E-2,4.857646E-1,6.366408E-3,1.505155E-3,1.9851085E-2,-5.0905657E-1,-3.399058E-1,2.1151064E-3,-4.4386946E-3,-4.3099437E-2,5.1802597E-3,-1.0904729E-2,-1.0970784E-1,-7.092669E-3,-1.4537698E-1,7.2762733E-3,3.4559437E-3,5.819405E-3,1.0485568E-3,6.408163E-3,1.6756054E-2,-1.6861757E-2,-8.245572E-3,-3.560397E-1,-4.0789754E-3,-3.00326E-3,5.754394E-4,-1.3698114E-1,1.1246253E-1,4.9124755E-2,-3.0779731E-3,-6.556022E-3,-6.631281E-2,-1.2410655E-2,-2.5372538E-1,-1.6636467E-1,2.5156673E-2,5.4574534E-3,8.776372E-4,2.9181289E-3,-4.8511144E-4,-2.6679924E-4,-3.4686464E-3,-4.0353606E-3,-9.003159E-3,-1.79588E-1,-7.937938E-4,3.6075476E-3,-3.5893454E-3,-2.0991333E-1,-4.005206E-3,-1.9170133E-3,-2.277097E-1,-7.843476E-3,-3.0570475E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,-1,41,-1,43,-1,-1,-1,45,47,-1,-1,49,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,-1,59,61,63,-1,-1,65,-1,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,73,-1,-1,-1,75,-1,-1,77,-1,-1],"loss_changes":[2.6345465E1,6.245114E0,2.1417809E0,3.259615E0,2.307434E0,3.310299E-1,1.5115223E0,8.75824E-1,3.4721655E-1,5.383978E-1,8.522582E-1,2.6658773E-1,7.8017235E-2,0E0,7.6536703E-1,4.30871E-1,9.594993E-2,1.3166879E-1,2.683729E-1,1.1282359E-1,2.1332592E-2,0E0,0E0,4.8923E-2,0E0,1.1577988E-1,0E0,0E0,0E0,4.1131973E-3,1.2341976E-1,0E0,0E0,2.3095664E-2,0E0,0E0,3.7290502E-1,6.442011E-2,4.6412498E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.8073063E-2,0E0,0E0,0E0,2.566868E-1,2.941984E-2,2.269531E-2,0E0,0E0,1.5940988E-2,0E0,8.274496E-3,7.98924E-2,1.1940452E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.246354E-2,0E0,0E0,0E0,5.3334594E-2,0E0,0E0,2.3465276E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,25,25,29,29,30,30,33,33,36,36,37,37,38,38,47,47,51,51,52,52,53,53,56,56,58,58,59,59,60,60,69,69,73,73,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,-1,42,-1,44,-1,-1,-1,46,48,-1,-1,50,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,-1,60,62,64,-1,-1,66,-1,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,74,-1,-1,-1,76,-1,-1,78,-1,-1],"split_conditions":[1.378469E9,6.490568E8,1.14669E5,1.111431E6,1.83E2,7.502168E8,1.7754914E0,1.83E2,5.798156E2,1.3925089E9,6.044738E8,1.1328217E10,1E0,3.4592822E-2,1.5096E4,1.13161E5,4.37356E5,2.785E3,2.87112E5,7.221E3,2.231867E9,2.1669758E-2,4.0896516E-3,1.7867039E10,1.1961628E-2,4.618673E6,6.366408E-3,1.505155E-3,1.9851085E-2,1.3953088E8,7.482E3,2.1151064E-3,-4.4386946E-3,1.4064E4,5.1802597E-3,-1.0904729E-2,2.594684E6,7.571E4,1.0917633E9,7.2762733E-3,3.4559437E-3,5.819405E-3,1.0485568E-3,6.408163E-3,1.6756054E-2,-1.6861757E-2,-8.245572E-3,8.6257E4,-4.0789754E-3,-3.00326E-3,5.754394E-4,1.83E2,7.4E1,7.09E2,-3.0779731E-3,-6.556022E-3,2.2599797E3,-1.2410655E-2,1.04345E5,1.9669E4,1E0,5.4574534E-3,8.776372E-4,2.9181289E-3,-4.8511144E-4,-2.6679924E-4,-3.4686464E-3,-4.0353606E-3,-9.003159E-3,3.228882E6,-7.937938E-4,3.6075476E-3,-3.5893454E-3,4.5E1,-4.005206E-3,-1.9170133E-3,7.0507E4,-7.843476E-3,-3.0570475E-3],"split_indices":[27,16,3,9,47,16,59,47,5,16,11,21,34,0,7,6,2,20,16,3,10,0,0,10,0,2,0,0,0,16,3,0,0,7,0,0,6,8,10,0,0,0,0,0,0,0,0,12,0,0,0,47,4,17,0,0,5,0,12,7,34,0,0,0,0,0,0,0,0,18,0,0,0,17,0,0,28,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.14E2,6.8E1,1.66E2,4.8E1,4.1E1,2.7E1,9.2E1,7.4E1,3.5E1,1.3E1,1.7E1,2.4E1,1E1,1.7E1,8.4E1,8E0,1.1E1,6.3E1,2.3E1,1.2E1,8E0,5E0,9E0,8E0,2.1E1,3E0,3E0,1.4E1,3.6E1,4.8E1,3E0,5E0,6E0,5E0,5E0,5.8E1,1.1E1,1.2E1,5E0,7E0,4E0,5E0,3E0,1.8E1,3.3E1,3E0,4.4E1,4E0,3E0,3E0,5.2E1,6E0,7E0,4E0,6E0,6E0,3.2E1,1.2E1,4.4E1,8E0,3E0,3E0,4E0,3E0,3E0,3E0,3E0,9E0,4E1,4E0,5E0,3E0,2.4E1,1.6E1,3E0,2.1E1,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[1.886632E-2,-1.884953E-1,4.469081E-1,-2.9699528E-1,6.3556306E-2,3.0015472E-1,7.001897E-1,-3.581915E-1,-1.5094061E-1,-2.3274306E-2,2.4301253E-1,1.5241918E-1,5.154072E-1,9.155265E-1,4.4366527E-1,-3.918764E-1,-1.5690489E-1,-1.9286716E-1,4.048581E-3,8.638295E-2,-9.997843E-2,5.58225E-4,3.0203518E-1,6.249177E-2,3.1363195E-1,2.4054883E-2,3.2645005E-1,1.4147483E-2,3.3096153E-2,-1.7297564E-3,1.783558E-2,-4.190246E-1,-1.8595801E-1,-2.9890941E-2,-1.6053295E-2,-2.3938422E-1,-1.1178434E-1,8.459664E-3,-9.030455E-3,-1.0485094E-2,-3.8540184E-2,6.40514E-3,1.3630261E-2,-2.6395934E-2,5.7303174E-3,3.8411068E-3,1.1587689E-2,3.8890192E-1,1.8502832E-3,-4.492887E-1,-8.62786E-3,-2.4648309E-3,-7.645656E-3,-4.0564667E-3,5.67759E-2,-9.167102E-3,-4.001888E-3,-4.305226E-3,-5.7765463E-4,4.689306E-2,-4.26828E-3,-1.1369428E-1,2.1037064E-2,-3.8139469E-3,3.149781E-2,8.337644E-3,1.5309446E-2,-4.6940237E-1,-7.7715553E-3,2.8727538E-4,2.9109495E-3,2.807725E-5,2.6889124E-3,-2.074384E-3,-5.12275E-3,-1.3330157E-3,7.155971E-2,5.13634E-3,-4.6185054E-2,-1.5634404E-2,-8.655072E-3,3.227124E-3,8.0388336E-4,-1.3368513E-4,-2.4679503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,43,45,-1,47,-1,-1,-1,-1,49,51,53,-1,55,57,-1,59,-1,61,-1,-1,63,-1,-1,-1,65,-1,67,-1,-1,-1,-1,69,-1,-1,-1,-1,71,-1,73,75,-1,77,-1,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,81,-1,83,-1,-1,-1,-1,-1,-1],"loss_changes":[2.502564E1,5.2149186E0,3.209467E0,1.1346874E0,9.0942895E-1,1.8294692E0,1.4167061E0,5.690832E-1,4.9992967E-1,3.45083E-1,2.4783576E-1,5.1680607E-1,8.076806E-1,4.597435E-1,9.635103E-1,3.6371326E-1,6.363231E-1,1.09023094E-1,0E0,2.885471E-1,3.203292E-1,0E0,1.0340917E-1,2.6142767E-1,7.41384E-2,0E0,2.2960877E-1,0E0,0E0,0E0,0E0,1.9486713E-1,4.3857634E-2,1.071298E-1,0E0,7.953012E-2,2.8516099E-2,0E0,8.914387E-2,0E0,9.194128E-2,0E0,0E0,8.3703786E-2,0E0,0E0,0E0,1.19485855E-2,0E0,1.20788574E-1,0E0,0E0,0E0,0E0,1.046671E-2,0E0,0E0,0E0,0E0,1.5103858E-2,0E0,9.890683E-3,4.0824678E-2,0E0,1.06858544E-1,0E0,0E0,3.28722E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.8539865E-3,0E0,8.65117E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,23,23,24,24,26,26,31,31,32,32,33,33,35,35,36,36,38,38,40,40,43,43,47,47,49,49,54,54,59,59,61,61,62,62,64,64,67,67,76,76,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,44,46,-1,48,-1,-1,-1,-1,50,52,54,-1,56,58,-1,60,-1,62,-1,-1,64,-1,-1,-1,66,-1,68,-1,-1,-1,-1,70,-1,-1,-1,-1,72,-1,74,76,-1,78,-1,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,82,-1,84,-1,-1,-1,-1,-1,-1],"split_conditions":[7.164074E8,1.098767E6,7.8908E4,4.951E3,4.365884E6,1.83E2,4.331342E-1,1.83E2,1.83E2,9.3E1,1.9102706E-6,9.7321754E8,1.895E3,3.060244E7,2.2818398E0,1.374403E6,1.6525185E9,1.845383E6,4.048581E-3,6.521E3,1.29E2,5.58225E-4,1.1123341E7,1.2084881E9,1.4145E4,2.4054883E-2,4.3740602E10,1.4147483E-2,3.3096153E-2,-1.7297564E-3,1.783558E-2,1.80773E8,1.26E2,6.52028E5,-1.6053295E-2,1.507738E6,7.2149E4,8.459664E-3,1.3321924E9,-1.0485094E-2,3.615948E6,6.40514E-3,1.3630261E-2,1.1E2,5.7303174E-3,3.8411068E-3,1.1587689E-2,3E1,1.8502832E-3,1.164162E9,-8.62786E-3,-2.4648309E-3,-7.645656E-3,-4.0564667E-3,1.71E3,-9.167102E-3,-4.001888E-3,-4.305226E-3,-5.7765463E-4,8.6E1,-4.26828E-3,2.62E2,6.348E3,-3.8139469E-3,6.23E2,8.337644E-3,1.5309446E-2,1.6871828E7,-7.7715553E-3,2.8727538E-4,2.9109495E-3,2.807725E-5,2.6889124E-3,-2.074384E-3,-5.12275E-3,-1.3330157E-3,3.35495E5,5.13634E-3,1.3818E4,-1.5634404E-2,-8.655072E-3,3.227124E-3,8.0388336E-4,-1.3368513E-4,-2.4679503E-3],"split_indices":[16,6,3,3,6,47,63,47,47,4,63,27,0,2,59,9,11,18,0,7,17,0,6,16,7,0,26,0,0,0,0,16,4,21,0,2,8,0,21,0,2,0,0,4,0,0,0,19,0,11,0,0,0,0,3,0,0,0,0,19,0,4,3,0,17,0,0,27,0,0,0,0,0,0,0,0,12,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.89E2,9.1E1,1.32E2,5.7E1,5.9E1,3.2E1,9.2E1,4E1,3.9E1,1.8E1,3.6E1,2.3E1,1.6E1,1.6E1,7.8E1,1.4E1,3.5E1,5E0,1.6E1,2.3E1,4E0,1.4E1,2.4E1,1.2E1,9E0,1.4E1,4E0,1.2E1,3E0,1.3E1,6.8E1,1E1,1.1E1,3E0,2.1E1,1.4E1,5E0,1.1E1,4E0,1.9E1,9E0,5E0,1.4E1,1E1,3E0,9E0,1.1E1,3E0,5.5E1,1.3E1,4E0,6E0,5E0,6E0,1.4E1,7E0,1.1E1,3E0,8E0,3E0,8E0,1.1E1,5E0,9E0,6E0,5E0,4.9E1,6E0,3E0,3E0,4E0,4E0,5E0,3E0,5E0,6E0,3E0,6E0,4.4E1,5E0,3E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.305396E-2,-1.4882736E-1,5.607591E-1,-2.285431E-1,1.19809955E-1,4.6035948E-1,3.108899E-2,-3.280764E-1,-1.0973797E-1,-2.7264308E-3,4.4385087E-1,3.669005E-1,6.132102E-1,-3.700728E-1,-4.4400282E-2,-1.6534369E-1,4.1846998E-2,-9.109918E-2,1.552768E-1,2.6672646E-2,2.6358214E-1,2.4290239E-3,4.1147256E-1,2.1649031E-2,1.0059585E-2,-4.0694904E-1,-2.3026235E-1,-5.7998328E-3,6.298514E-2,-1.3708451E-1,-1.649914E-2,1.4998926E-1,-7.993065E-2,-1.9884793E-1,-1.806251E-2,1.7468438E-1,2.0306967E-3,6.1703584E-4,1.2891943E-2,1.4688085E-2,8.745323E-3,-4.1747522E-1,-4.8267418E-3,-8.293206E-3,-3.2856222E-3,5.294724E-3,-9.918973E-4,-1.6482048E-1,7.665082E-2,6.011281E-4,1.9168569E-1,-8.954025E-3,2.91807E-2,-3.0802395E-3,-7.383732E-3,5.685728E-2,-6.870011E-2,3.6469752E-3,7.2646337E-3,-4.2815584E-1,-6.968702E-3,2.7980106E-3,-1.9540343E-1,5.1296013E-3,-8.118084E-4,2.7255253E-3,8.380768E-3,-1.5941783E-3,2.7782167E-3,3.5086405E-3,-3.058311E-4,-2.082293E-5,-9.777336E-2,-1.432276E-2,-8.813943E-3,-2.9636058E-1,-1.3296488E-1,-1.6440937E-3,-3.863546E-3,-2.408481E-3,-3.41729E-1,-1.33833615E-2,-1.6357817E-1,-1.3236949E-2,-6.183852E-3,2.4079937E-3,-3.1618874E-3,-6.4800847E-3,-6.7659445E-2,-2.8861833E-3,-1.1756726E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,-1,39,-1,-1,41,43,-1,45,47,-1,49,51,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,61,63,-1,65,-1,67,-1,-1,69,71,-1,-1,73,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,77,-1,-1,79,81,-1,-1,-1,83,85,87,-1,-1,-1,-1,-1,89,-1,-1],"loss_changes":[2.5263815E1,4.738599E0,2.198843E0,1.9742584E0,2.0262573E0,5.464182E-1,0E0,1.0795994E0,6.728095E-1,5.4474646E-1,7.5267243E-1,4.3659115E-1,1.2889385E-1,3.1874943E-1,2.0089975E-1,4.9885178E-1,3.0171356E-1,1.9314855E-1,1.4028341E-2,0E0,3.604542E-1,0E0,7.0314884E-2,0E0,0E0,1.05122566E-1,4.038596E-2,0E0,8.123703E-2,3.386823E-1,0E0,6.2124312E-2,2.4604982E-1,9.552211E-3,6.4606234E-2,8.8689625E-3,0E0,0E0,0E0,0E0,0E0,3.395748E-2,0E0,0E0,0E0,0E0,0E0,3.9424157E-1,6.299797E-2,0E0,4.4097394E-2,0E0,4.0245794E-2,0E0,0E0,2.525702E-2,1.9722037E-2,0E0,0E0,2.0793915E-2,0E0,0E0,2.3795366E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1437312E-3,0E0,0E0,1.3524342E-1,1.0397351E-1,0E0,0E0,0E0,5.61738E-2,5.972688E-2,6.962913E-2,0E0,0E0,0E0,0E0,0E0,2.2060089E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,22,22,25,25,26,26,28,28,29,29,31,31,32,32,33,33,34,34,35,35,41,41,47,47,48,48,50,50,52,52,55,55,56,56,59,59,62,62,72,72,75,75,76,76,80,80,81,81,82,82,88,88],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,-1,40,-1,-1,42,44,-1,46,48,-1,50,52,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,62,64,-1,66,-1,68,-1,-1,70,72,-1,-1,74,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,78,-1,-1,80,82,-1,-1,-1,84,86,88,-1,-1,-1,-1,-1,90,-1,-1],"split_conditions":[2.4995E4,6.490568E8,1.2738551E4,1.122053E6,1.83E2,1.9367423E9,3.108899E-2,1.83E2,4.112304E6,1.1408653E9,2.59E3,2.0585947E9,2.2799654E0,1.7013242E8,3.4713E4,1.06056E5,3.23E2,1.0699521E9,2.0396E4,2.6672646E-2,4.796E3,2.4290239E-3,6.72444E5,2.1649031E-2,1.0059585E-2,7.482E3,5.0373805E8,-5.7998328E-3,2.302E3,7.0507E4,-1.649914E-2,4.764156E6,6.162E3,7.482E3,6.5878E4,3.323062E3,2.0306967E-3,6.1703584E-4,1.2891943E-2,1.4688085E-2,8.745323E-3,7.7999923E9,-4.8267418E-3,-8.293206E-3,-3.2856222E-3,5.294724E-3,-9.918973E-4,3.2416E4,1.5509368E9,6.011281E-4,5.1E1,-8.954025E-3,2.1547517E3,-3.0802395E-3,-7.383732E-3,7.1E1,8.24E2,3.6469752E-3,7.2646337E-3,8.487963E7,-6.968702E-3,2.7980106E-3,4.618E3,5.1296013E-3,-8.118084E-4,2.7255253E-3,8.380768E-3,-1.5941783E-3,2.7782167E-3,3.5086405E-3,-3.058311E-4,-2.082293E-5,4.8E1,-1.432276E-2,-8.813943E-3,5.87E2,3.388763E7,-1.6440937E-3,-3.863546E-3,-2.408481E-3,2.2723E4,1.619794E6,3.1784865E-1,-1.3236949E-2,-6.183852E-3,2.4079937E-3,-3.1618874E-3,-6.4800847E-3,1.098767E6,-2.8861833E-3,-1.1756726E-3],"split_indices":[3,16,5,9,47,16,0,47,2,16,20,27,67,16,18,28,17,10,3,0,7,0,28,0,0,3,16,0,7,28,0,2,20,3,8,5,0,0,0,0,0,10,0,0,0,0,0,12,21,0,17,0,5,0,0,4,0,0,0,21,0,0,3,0,0,0,0,0,0,0,0,0,19,0,0,28,16,0,0,0,28,21,63,0,0,0,0,0,6,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.19E2,6.4E1,1.69E2,5E1,5.3E1,1.1E1,9.1E1,7.8E1,3.7E1,1.3E1,3.5E1,1.8E1,7.9E1,1.2E1,5.7E1,2.1E1,2.4E1,1.3E1,3E0,1E1,5E0,3E1,1.4E1,4E0,6.1E1,1.8E1,5E0,7E0,5.4E1,3E0,1.1E1,1E1,9E0,1.5E1,1E1,3E0,4E0,6E0,2.1E1,9E0,5.8E1,3E0,1.4E1,4E0,3E0,4E0,4.8E1,6E0,3E0,8E0,3E0,7E0,3E0,6E0,6E0,9E0,6E0,4E0,5.4E1,4E0,5E0,4.3E1,3E0,3E0,4E0,4E0,3E0,4E0,3E0,3E0,3E0,6E0,4.7E1,7E0,1.5E1,2.8E1,3E0,3E0,3E0,1.2E1,6E0,2.2E1,7E0,5E0,3E0,3E0,1.5E1,7E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[6.322361E-3,-1.9551156E-1,4.120058E-1,-3.664265E-1,-9.900715E-2,2.56217E-1,6.4714587E-1,4.060352E-3,-3.981903E-1,-1.306202E-1,3.2647574E-1,1.2947546E-1,3.2618645E-1,7.9715264E-1,4.8775214E-1,-1.4691655E-2,-1.0366749E-2,-1.7328739E-1,2.7020924E-2,1.9611286E-2,2.6891797E-3,8.106053E-4,2.1213068E-1,4.64897E-1,2.4428554E-1,8.5587484E-1,1.2156361E-2,7.6610222E-3,5.1615494E-1,5.1948316E-2,-1.9222458E-1,-2.0914802E-2,4.425681E-3,-3.7967218E-3,3.2074226E-3,1.16673276E-1,1.1607961E-2,5.372482E-3,1.7304923E-2,1.1028404E-1,3.914303E-1,3.3068467E-2,1.6518516E-2,1.1751467E-2,1.9329762E-2,-2.6733517E-3,6.6860137E-3,-1.527908E-1,-3.1471652E-1,2.8399066E-3,-4.6892855E-2,5.5697015E-3,2.182444E-3,-5.0988114E-3,6.0442E-3,1.3965616E-2,6.3765645E-3,-1.7070925E-1,5.539792E-3,-4.0937725E-1,-5.7656253E-3,-6.1505932E-2,3.966602E-4,3.3343157E-3,-3.6215354E-3,-2.1220371E-1,-1.0978053E-1,-7.5236554E-3,-1.6634962E-2,-9.5622946E-4,-2.5000225E-3,-2.488012E-1,-1.1522554E-1,-1.7137797E-1,-6.603872E-2,-9.588488E-3,-1.6671765E-1,-1.0784705E-3,-1.3874322E-1,-6.767531E-3,-3.3460138E-3,1.8885338E-3,-1.0015972E-1,-2.2724357E-3,-6.145319E-3,-5.2942843E-3,-2.1070738E-3,-1.1017778E-3,1.2081601E-3,-4.2944676E-3,-1.4378623E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,-1,-1,29,31,-1,-1,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,51,-1,-1,-1,53,55,-1,-1,-1,-1,-1,-1,57,59,-1,61,-1,-1,63,-1,-1,-1,65,-1,67,-1,69,-1,-1,-1,71,73,-1,-1,-1,-1,75,77,79,81,-1,83,-1,85,-1,-1,87,89,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3907831E1,3.18006E0,3.3875122E0,1.1454363E0,1.7374461E0,4.8984933E-1,5.197716E-1,0E0,1.4558506E-1,7.9836273E-1,5.6736654E-1,2.4442893E-1,3.181324E-1,1.2018108E-1,2.6120663E-2,0E0,0E0,4.066615E-1,1.405572E-1,0E0,0E0,1.2912872E-1,1.5636605E-1,1.9193244E-1,4.6948957E-1,3.5413933E-1,0E0,0E0,1.01594925E-2,1.8547511E-1,3.6732197E-1,5.8007274E-2,0E0,0E0,0E0,1.5811786E-2,0E0,0E0,0E0,1.3506994E-1,1.2497187E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.194255E-1,1.8338978E-1,0E0,1.4603116E-2,0E0,0E0,9.337075E-2,0E0,0E0,0E0,1.3850927E-1,0E0,8.673966E-2,0E0,4.3736063E-3,0E0,0E0,0E0,1.0265064E-1,6.412518E-2,0E0,0E0,0E0,0E0,5.5083275E-2,1.8414214E-2,6.9728196E-3,4.1908592E-2,0E0,1.4431268E-2,0E0,6.2051117E-3,0E0,0E0,1.0296392E-2,1.6184129E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,35,35,39,39,40,40,47,47,48,48,50,50,53,53,57,57,59,59,61,61,65,65,66,66,71,71,72,72,73,73,74,74,76,76,78,78,81,81,82,82],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,-1,-1,30,32,-1,-1,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,52,-1,-1,-1,54,56,-1,-1,-1,-1,-1,-1,58,60,-1,62,-1,-1,64,-1,-1,-1,66,-1,68,-1,70,-1,-1,-1,72,74,-1,-1,-1,-1,76,78,80,82,-1,84,-1,86,-1,-1,88,90,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.13108E5,8.33591E5,6.6569E4,2.798795E2,1.0397685E9,9.2171E4,6.60017E0,4.060352E-3,1.13161E5,2.989734E6,3.3E1,1.3880545E7,4.8242E4,2.054214E0,1.5111E4,-1.4691655E-2,-1.0366749E-2,3.2416E4,3.04649E6,1.9611286E-2,2.6891797E-3,9.5E1,3.6815E6,4.236E4,1.6728869E7,1E0,1.2156361E-2,7.6610222E-3,1.4864444E1,6.52028E5,7.9635E4,3.4311706E9,4.425681E-3,-3.7967218E-3,3.2074226E-3,8.58E2,1.1607961E-2,5.372482E-3,1.7304923E-2,1.14E2,6.38E2,3.3068467E-2,1.6518516E-2,1.1751467E-2,1.9329762E-2,-2.6733517E-3,6.6860137E-3,3.008E3,1.164162E9,2.8399066E-3,2.402988E6,5.5697015E-3,2.182444E-3,1.2081E4,6.0442E-3,1.3965616E-2,6.3765645E-3,1.37586E5,5.539792E-3,2.83326E9,-5.7656253E-3,3.969357E6,3.966602E-4,3.3343157E-3,-3.6215354E-3,6.00237E5,6.44E4,-7.5236554E-3,-1.6634962E-2,-9.5622946E-4,-2.5000225E-3,5.656E3,4.283E3,1.7140013E3,1.9073843E3,-9.588488E-3,6.8E2,-1.0784705E-3,1.08298E5,-6.767531E-3,-3.3460138E-3,5.3E1,3.357931E9,-2.2724357E-3,-6.145319E-3,-5.2942843E-3,-2.1070738E-3,-1.1017778E-3,1.2081601E-3,-4.2944676E-3,-1.4378623E-3],"split_indices":[12,2,3,5,16,22,60,0,6,9,4,18,20,59,7,0,0,12,6,0,0,4,2,28,2,36,0,0,60,21,22,10,0,0,0,0,0,0,0,19,17,0,0,0,0,0,0,0,11,0,6,0,0,7,0,0,0,12,0,10,0,2,0,0,0,6,8,0,0,0,0,3,3,5,5,0,20,0,12,0,0,4,10,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.9E2,1.94E2,9.6E1,6.9E1,1.25E2,5.9E1,3.7E1,4E0,6.5E1,1.17E2,8E0,2.2E1,3.7E1,1.7E1,2E1,3.4E1,3.1E1,9.2E1,2.5E1,3E0,5E0,9E0,1.3E1,1.2E1,2.5E1,1.4E1,3E0,3E0,1.7E1,7E0,8.5E1,1.8E1,7E0,4E0,5E0,9E0,4E0,3E0,9E0,1.4E1,1.1E1,8E0,6E0,8E0,9E0,4E0,3E0,6.6E1,1.9E1,3E0,1.5E1,3E0,6E0,6E0,8E0,8E0,3E0,6.3E1,3E0,1E1,9E0,1.2E1,3E0,3E0,3E0,3.6E1,2.7E1,5E0,5E0,5E0,7E0,2.5E1,1.1E1,1E1,1.7E1,1.4E1,1.1E1,3E0,8E0,5E0,5E0,6E0,1.1E1,3E0,8E0,5E0,3E0,3E0,3E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[2.7093343E-2,-1.7215988E-1,4.1216448E-1,-2.2731338E-1,1.2748417E-1,1.7326108E-1,5.961424E-1,-3.2851958E-1,-4.3181777E-2,2.5472847E-1,-2.9063848E-1,8.997096E-2,4.2246336E-1,4.7168636E-1,7.80927E-1,-3.764815E-1,-2.0472883E-1,-8.997601E-2,1.4792079E-1,1.5490164E-1,2.141286E-2,-2.0115466E-3,-1.4360316E-2,2.013188E-1,3.1898743E-3,5.864402E-3,1.6075565E-2,5.1236546E-1,6.5419576E-3,1.3680133E-2,2.8251894E-2,-4.4469935E-1,-3.1435287E-1,-2.3704699E-1,-1.0657214E-1,-1.23161145E-1,-3.5199642E-2,5.8787544E-3,2.2448585E-3,3.937789E-2,2.0519121E-1,4.620607E-3,8.903465E-3,1.06863126E-1,-1.0080236E-1,9.412351E-3,5.7175994E-1,-1.485009E-2,-7.119959E-3,-1.0777487E-2,-4.6990663E-3,-8.306154E-3,-4.556292E-3,-8.6571125E-4,-4.5685284E-3,-1.4323214E-1,-1.9964803E-4,-1.3588389E-2,-3.458444E-3,2.530092E-3,-3.1190866E-4,1.9023215E-3,2.4310915E-1,5.855077E-4,4.5794426E-3,-5.081959E-3,-1.1729061E-3,1.9691167E-2,8.012624E-3,-3.030183E-3,-1.6806898E-1,-2.7968541E-3,1.1977385E-2,9.239698E-3,3.394105E-3,-6.046596E-3,-3.0547106E-3,-4.8029906E-4,1.331428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,-1,41,43,-1,-1,45,-1,-1,-1,47,49,51,53,55,57,-1,-1,59,61,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,69,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,75,-1,77,-1,-1,-1,-1,-1,-1],"loss_changes":[2.048025E1,2.9376297E0,3.896429E0,2.762467E0,1.5637381E0,8.1299007E-1,8.0452347E-1,4.9289894E-1,4.9620423E-1,7.9848146E-1,2.204377E-1,3.0857018E-1,9.390414E-2,2.5623894E-1,2.8136063E-1,1.4840984E-1,6.6732526E-2,7.564628E-2,1.7115012E-2,1.0230315E-1,0E0,0E0,0E0,2.118361E-2,2.1561512E-1,0E0,0E0,1.9860935E-1,0E0,0E0,0E0,9.541512E-3,8.551335E-2,5.582452E-3,2.1536775E-2,6.2493503E-2,2.664247E-2,0E0,0E0,1.4233868E-2,5.7646513E-2,0E0,0E0,2.8806455E-2,3.0984081E-2,0E0,1.18747234E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.222375E-2,0E0,2.9150356E-2,0E0,0E0,0E0,0E0,3.0218542E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4334192E-3,0E0,1.0102337E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,27,27,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,43,43,44,44,46,46,55,55,57,57,62,62,70,70,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,-1,42,44,-1,-1,46,-1,-1,-1,48,50,52,54,56,58,-1,-1,60,62,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,70,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,76,-1,78,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,1.83E2,7.502168E8,1.530129E6,1.1768184E9,1.0720187E10,1.99916E5,4.754E3,1.0886205E9,7.031E3,9.19E2,1.73E2,3.389E3,6.6E2,7.5E1,1.65606E5,4.2524388E7,3.1043533E8,2.9E1,1E0,2.141286E-2,-2.0115466E-3,-1.4360316E-2,1.83E2,5.215832E10,5.864402E-3,1.6075565E-2,1.111798E9,6.5419576E-3,1.3680133E-2,2.8251894E-2,1.0444653E10,3.9182508E9,3.69E2,1.18443E5,2.1504E4,2.2799654E0,5.8787544E-3,2.2448585E-3,2.6E1,5.876475E7,4.620607E-3,8.903465E-3,3.16483E5,1.28E2,9.412351E-3,1E0,-1.485009E-2,-7.119959E-3,-1.0777487E-2,-4.6990663E-3,-8.306154E-3,-4.556292E-3,-8.6571125E-4,-4.5685284E-3,1.8277E5,-1.9964803E-4,1.1055E4,-3.458444E-3,2.530092E-3,-3.1190866E-4,1.9023215E-3,1.1048E4,5.855077E-4,4.5794426E-3,-5.081959E-3,-1.1729061E-3,1.9691167E-2,8.012624E-3,-3.030183E-3,7.7183E4,-2.7968541E-3,4.4598663E-1,9.239698E-3,3.394105E-3,-6.046596E-3,-3.0547106E-3,-4.8029906E-4,1.331428E-3],"split_indices":[2,47,16,9,11,21,3,3,16,3,0,17,0,17,4,18,18,21,19,42,0,0,0,47,10,0,0,16,0,0,0,10,10,4,12,7,67,0,0,17,16,0,0,18,4,0,34,0,0,0,0,0,0,0,0,27,0,7,0,0,0,0,20,0,0,0,0,0,0,0,8,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.75E2,9E1,1.48E2,2.7E1,4E1,5E1,9.5E1,5.3E1,2.1E1,6E0,3.1E1,9E0,3.2E1,1.8E1,6.7E1,2.8E1,4.3E1,1E1,1.8E1,3E0,3E0,3E0,1.3E1,1.8E1,3E0,6E0,2.7E1,5E0,5E0,1.3E1,2.9E1,3.8E1,2E1,8E0,2.6E1,1.7E1,6E0,4E0,6E0,1.2E1,9E0,4E0,9E0,9E0,7E0,2E1,2.6E1,3E0,3.3E1,5E0,1.5E1,5E0,3E0,5E0,2.2E1,4E0,1.4E1,3E0,3E0,3E0,3E0,9E0,3E0,6E0,4E0,5E0,1.7E1,3E0,9E0,1.3E1,3E0,1.1E1,6E0,3E0,9E0,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-7.059858E-4,-1.8247148E-1,3.286385E-1,-2.4366139E-1,1.0592256E-2,1.5942685E-1,5.4422116E-1,-3.161389E-1,-1.2722051E-1,-8.793196E-2,3.642202E-1,7.828822E-2,3.6752272E-1,6.63954E-1,3.3438727E-1,-3.4886253E-1,-1.0806461E-1,-1.0487504E-1,-1.2421926E-2,2.0718963E-3,-1.08980745E-1,1.5794413E-2,6.377772E-3,-2.1645887E-2,1.4446229E-1,4.850175E-1,-3.4189832E-3,4.7424185E-1,2.8866613E-2,4.3100607E-1,-3.5900357E-3,-4.2399588E-1,-3.0582127E-1,-2.4867019E-1,4.7978777E-2,-1.457375E-1,4.444835E-3,-1.527364E-1,-2.3326632E-2,-6.987459E-2,2.78297E-3,1.6577817E-1,-4.5771617E-4,2.4565253E-1,2.3283388E-2,4.7339355E-3,1.7183946E-2,1.5202891E-2,6.417465E-3,-1.4469956E-2,-7.6305144E-3,-4.261807E-3,-3.140737E-1,-9.981351E-3,-4.0264097E-3,5.5861725E-3,-2.8834953E-3,-2.3302804E-1,-8.954176E-2,-6.7911013E-3,-9.147528E-2,-3.4935782E-3,1.7134903E-2,-1.0141822E-1,2.8289542E-5,8.31828E-2,6.1170734E-3,4.046059E-4,1.2327994E-2,-3.2219836E-1,-4.705517E-3,-1.4811226E-2,-1.5149583E-1,3.509424E-3,-1.2567359E-1,-3.947537E-3,-1.2835278E-3,-1.5914171E-3,1.8882768E-3,-1.6507211E-3,-4.2254636E-3,4.202889E-3,4.8285455E-4,-1.0756159E-2,-6.140653E-3,-7.562901E-2,-6.403746E-3,-1.6534613E-1,-4.9630173E-2,-3.2380228E-3,-1.0222106E-3,-3.3795582E-3,-6.1500208E-3,-1.4759117E-4,-6.828016E-2,-2.7427608E-3,-1.103507E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,-1,-1,39,41,43,-1,45,-1,47,-1,49,51,53,55,57,-1,59,61,63,-1,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,69,-1,-1,-1,-1,71,73,-1,75,-1,77,79,-1,81,-1,-1,-1,83,-1,-1,85,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,89,-1,91,93,-1,-1,-1,-1,-1,95,-1,-1],"loss_changes":[1.7001328E1,2.1644607E0,3.5976248E0,1.1328497E0,1.5998678E0,9.505352E-1,8.8343906E-1,5.410614E-1,2.77453E-1,1.2241998E-1,1.1280644E-1,2.8748798E-1,9.42106E-1,7.6732635E-1,8.235822E-1,1.064024E-1,2.971583E-1,5.36379E-1,0E0,0E0,1.1636838E-1,0E0,0E0,9.988733E-2,9.030098E-2,5.6389284E-1,0E0,2.5300527E-1,0E0,4.9320936E-2,0E0,8.005619E-3,2.9973984E-2,1.4342368E-2,1.364536E-1,1.9987524E-1,0E0,5.562538E-2,4.3230888E-2,2.9102795E-2,0E0,3.0275345E-2,0E0,2.5127074E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7010212E-2,0E0,0E0,0E0,0E0,2.45071E-1,2.2176518E-1,0E0,1.44050345E-2,0E0,2.7782306E-2,6.7382604E-3,0E0,2.0657767E-2,0E0,0E0,0E0,5.4073334E-3,0E0,0E0,3.5348773E-2,0E0,6.721473E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.4724643E-3,0E0,4.77165E-3,8.087818E-3,0E0,0E0,0E0,0E0,0E0,1.107499E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,23,23,24,24,25,25,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,41,41,43,43,52,52,57,57,58,58,60,60,62,62,63,63,65,65,69,69,72,72,74,74,85,85,87,87,88,88,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,-1,-1,40,42,44,-1,46,-1,48,-1,50,52,54,56,58,-1,60,62,64,-1,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,70,-1,-1,-1,-1,72,74,-1,76,-1,78,80,-1,82,-1,-1,-1,84,-1,-1,86,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,90,-1,92,94,-1,-1,-1,-1,-1,96,-1,-1],"split_conditions":[9.91E3,2.917845E8,1.4066751E7,4.695E5,1.83E2,1.83E2,3.3046817E10,1.83E2,9.45839E5,1E0,2.59E3,3.47989E5,1.7867039E10,1.99916E5,2.31E2,3.95265E5,3.4E1,2.594684E6,-1.2421926E-2,2.0718963E-3,5.581883E7,1.5794413E-2,6.377772E-3,1.116E3,2.9424374E10,2.01E2,-3.4189832E-3,4.142927E8,2.8866613E-2,8.6939E4,-3.5900357E-3,1.8149016E7,2.908E3,2.7552E4,1.2361E4,4.635E3,4.444835E-3,2.83326E9,5.2E2,2.564429E6,2.78297E-3,1.225E3,-4.5771617E-4,1.2842149E8,2.3283388E-2,4.7339355E-3,1.7183946E-2,1.5202891E-2,6.417465E-3,-1.4469956E-2,-7.6305144E-3,-4.261807E-3,7.208E3,-9.981351E-3,-4.0264097E-3,5.5861725E-3,-2.8834953E-3,2.7708E4,9.554299E2,-6.7911013E-3,1.34E2,-3.4935782E-3,7.0345754E8,1.8E1,2.8289542E-5,2.9E1,6.1170734E-3,4.046059E-4,1.2327994E-2,1.8635109E8,-4.705517E-3,-1.4811226E-2,5.2304E4,3.509424E-3,1.203673E6,-3.947537E-3,-1.2835278E-3,-1.5914171E-3,1.8882768E-3,-1.6507211E-3,-4.2254636E-3,4.202889E-3,4.8285455E-4,-1.0756159E-2,-6.140653E-3,2.280142E6,-6.403746E-3,3.0988458E-4,1.9796E4,-3.2380228E-3,-1.0222106E-3,-3.3795582E-3,-6.1500208E-3,-1.4759117E-4,2.1E2,-2.7427608E-3,-1.103507E-3],"split_indices":[3,16,2,6,47,47,21,47,12,45,20,12,10,3,4,2,4,6,0,0,18,0,0,17,10,17,0,16,0,8,0,18,21,18,12,3,0,10,0,2,0,0,0,1,0,0,0,0,0,0,0,0,3,0,0,0,0,8,5,0,4,0,16,19,0,19,0,0,0,21,0,0,8,0,6,0,0,0,0,0,0,0,0,0,0,2,0,64,20,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.82E2,1E2,1.38E2,4.4E1,5.7E1,4.3E1,8.4E1,5.4E1,3.5E1,9E0,4.2E1,1.5E1,2.6E1,1.7E1,7.2E1,1.2E1,5.1E1,3E0,4E0,3.1E1,4E0,5E0,1.7E1,2.5E1,1.2E1,3E0,1.6E1,1E1,1.4E1,3E0,2.3E1,4.9E1,6E0,6E0,4.4E1,7E0,2E1,1.1E1,1.2E1,5E0,2.2E1,3E0,7E0,5E0,3E0,1.3E1,1.1E1,3E0,1.9E1,4E0,3E0,4.6E1,3E0,3E0,3E0,3E0,1.6E1,2.8E1,9E0,1.1E1,3E0,8E0,8E0,4E0,6E0,1.6E1,3E0,4E0,4.3E1,3E0,3E0,1.3E1,4E0,2.4E1,6E0,5E0,3E0,5E0,4E0,4E0,3E0,3E0,3.8E1,5E0,6E0,7E0,1.5E1,9E0,3E0,3E0,6E0,9E0,3E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[3.496392E-2,-1.3831912E-1,4.047622E-1,-1.8966779E-1,1.6187572E-1,2.7735507E-1,5.810904E-1,-3.3879265E-1,-8.516876E-2,5.0170366E-2,1.8125612E-2,1.8523522E-2,2.3431349E-1,3.0225163E-2,4.9307433E-1,-3.570691E-1,-3.1067657E-3,-1.1329773E-1,8.767938E-2,-5.2185632E-2,2.9158202E-1,8.497697E-2,2.9961866E-1,2.0009145E-2,1.3562423E-2,-1.4244452E-2,-3.045895E-1,-1.3452458E-1,1.0251958E-1,1.1968532E-2,4.6019414E-3,-1.36143E-1,3.7570134E-2,1.1814069E-2,4.6109455E-3,-3.3129598E-3,1.3625541E-1,2.3559944E-1,1.5136178E-2,-4.037782E-3,-3.1543183E-1,-2.0632963E-1,-9.528035E-2,5.142205E-3,1.1662338E-3,1.06121E-3,-3.8701438E-4,-7.3202658E-3,-5.6787627E-4,3.4965076E-3,-2.5237384E-3,9.434876E-2,6.6617695E-3,6.799974E-4,2.6265585E-1,-1.0539537E-2,-5.4971534E-3,-3.2094745E-3,-7.119662E-3,-1.1640416E-1,-1.6927619E-2,4.1183736E-3,1.9851986E-3,3.191819E-3,2.918574E-1,-1.3991623E-1,-6.27449E-2,6.4714433E-4,-4.625124E-2,5.346561E-3,3.1431964E-1,-1.6288993E-1,-1.0416321E-3,-3.8928788E-3,-2.73737E-2,-2.242724E-3,-3.6263975E-4,1.2144525E-2,6.826016E-3,-6.2886877E-3,-3.359872E-3,-1.7258228E-3,8.173628E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,-1,27,29,31,33,35,37,-1,-1,-1,39,41,43,45,-1,47,49,-1,-1,-1,51,53,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,63,-1,-1,-1,-1,65,67,-1,-1,-1,69,71,73,-1,75,-1,77,79,-1,-1,81,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7677923E1,2.9286485E0,1.815793E0,2.4800735E0,1.2266791E0,5.4520416E-1,6.7104435E-1,2.577815E-1,4.7668976E-1,5.835448E-1,0E0,0E0,4.42657E-1,0E0,2.3943424E-2,1.3878107E-1,0E0,3.940246E-1,5.6890525E-2,1.3322109E-1,2.5759041E-2,1.6818659E-1,2.496512E-1,0E0,0E0,0E0,4.0070534E-2,1.945039E-1,2.4562255E-2,3.9231004E-3,0E0,9.333083E-2,8.2680635E-2,0E0,0E0,0E0,1.8991664E-2,1.4532685E-1,0E0,0E0,1.0019064E-2,1.9123673E-2,8.244011E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.0865893E-3,0E0,0E0,8.087015E-2,0E0,0E0,0E0,0E0,4.1686535E-2,1.3960865E-2,0E0,0E0,0E0,1.4631748E-3,6.1444998E-2,2.5506668E-2,0E0,4.9511464E-3,0E0,1.6578794E-2,2.1413982E-2,0E0,0E0,1.5208205E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,26,26,27,27,28,28,29,29,31,31,32,32,36,36,37,37,40,40,41,41,42,42,51,51,54,54,59,59,60,60,64,64,65,65,66,66,68,68,70,70,71,71,74,74],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,-1,28,30,32,34,36,38,-1,-1,-1,40,42,44,46,-1,48,50,-1,-1,-1,52,54,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,64,-1,-1,-1,-1,66,68,-1,-1,-1,70,72,74,-1,76,-1,78,80,-1,-1,82,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.47989E5,1.83E2,2.3176046E11,8.99622E5,2.9124946E7,5.345E3,4.4166436E0,4.3199274E8,2.379217E6,7.281275E-1,1.8125612E-2,1.8523522E-2,6.3381E4,3.0225163E-2,1.38306E5,1.17036E5,-3.1067657E-3,1.1408653E9,3.718604E6,1.3557681E-3,4.833E3,7.562E3,8.9969E4,2.0009145E-2,1.3562423E-2,-1.4244452E-2,3.4702344E7,1.530129E6,2.231867E9,7.7245E4,4.6019414E-3,3.618E3,1.542305E6,1.1814069E-2,4.6109455E-3,-3.3129598E-3,1.7833586E9,1.9037409E9,1.5136178E-2,-4.037782E-3,4.821E3,1.723E3,2.6059178E-1,5.142205E-3,1.1662338E-3,1.06121E-3,-3.8701438E-4,-7.3202658E-3,-5.6787627E-4,3.4965076E-3,-2.5237384E-3,1.2128E4,6.6617695E-3,6.799974E-4,6E1,-1.0539537E-2,-5.4971534E-3,-3.2094745E-3,-7.119662E-3,3.228882E6,2.902042E9,4.1183736E-3,1.9851986E-3,3.191819E-3,9.617851E6,4.718E3,6.7932027E9,6.4714433E-4,2.22426E6,5.346561E-3,1.9762979E10,7.7183E4,-1.0416321E-3,-3.8928788E-3,8.405997E8,-2.242724E-3,-3.6263975E-4,1.2144525E-2,6.826016E-3,-6.2886877E-3,-3.359872E-3,-1.7258228E-3,8.173628E-4],"split_indices":[12,47,13,9,18,7,60,16,6,64,0,0,28,0,20,18,0,16,6,66,7,20,8,0,0,0,16,9,10,8,0,7,2,0,0,0,16,11,0,0,3,20,63,0,0,0,0,0,0,0,0,3,0,0,19,0,0,0,0,18,10,0,0,0,2,0,10,0,2,0,10,8,0,0,16,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.87E2,8.7E1,1.6E2,2.7E1,5.2E1,3.5E1,6.5E1,9.5E1,2.2E1,5E0,5E0,4.7E1,5E0,3E1,6E1,5E0,8.2E1,1.3E1,1.6E1,6E0,1.5E1,3.2E1,8E0,2.2E1,2E1,4E1,7.5E1,7E0,6E0,7E0,8E0,8E0,3E0,3E0,3E0,1.2E1,2.5E1,7E0,3E0,3.7E1,2.5E1,5E1,3E0,4E0,3E0,3E0,4E0,4E0,5E0,3E0,9E0,3E0,3E0,2.2E1,3.3E1,4E0,4E0,2.1E1,3.9E1,1.1E1,3E0,6E0,4E0,1.8E1,2.6E1,1.3E1,5E0,6E0,4E0,1.4E1,2.1E1,5E0,4E0,9E0,3E0,3E0,7E0,7E0,1.2E1,9E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[1.9403175E-2,-1.1644433E-1,4.934833E-1,-1.922386E-1,1.0367332E-1,3.9947134E-1,9.015878E-1,-2.9705054E-1,-6.995695E-2,-8.124471E-3,4.4763258E-1,2.433919E-1,4.6641466E-1,3.5654034E-2,1.3930933E-2,-3.290066E-1,-4.344817E-2,-1.2851489E-1,6.471327E-2,-6.417397E-2,1.3046847E-1,2.1156134E-2,2.581519E-1,2.787767E-3,9.561152E-3,6.8402714E-3,4.9151278E-1,-3.7675002E-1,-2.6284072E-1,2.8672514E-3,-1.3190114E-1,-2.3126382E-2,-1.8056673E-1,1.6422611E-1,-6.7822464E-2,-9.071527E-2,3.1317326E-3,1.7180717E-1,1.7255036E-3,1.1866275E-2,8.9727994E-4,5.4335594E-1,9.613291E-3,-3.9054573E-1,-4.654696E-3,-7.3876632E-3,-1.2593819E-2,-6.960762E-3,-4.693184E-4,4.259935E-3,-1.0506091E-1,-1.1832423E-2,-1.5242521E-1,7.967656E-2,9.672139E-3,1.3266671E-2,-6.8577006E-3,-1.7533241E-1,-4.810179E-2,2.6814207E-3,6.703334E-3,2.110158E-2,4.3233395E-1,-1.2946682E-2,-5.955798E-3,-4.3280624E-2,-7.507959E-3,-1.7124651E-1,-1.2614303E-3,7.998905E-4,4.169543E-3,2.7057284E-3,-1.481318E-3,-2.8552879E-3,-6.5621263E-3,-1.0030559E-1,1.4132562E-2,1.51023455E-2,6.879386E-3,-2.8372942E-3,6.184368E-4,-1.6519009E-3,-1.8909617E-1,4.179272E-4,-4.6484703E-3,2.481076E-3,-2.3855468E-2,-2.0038466E-1,-2.6574412E-3,7.1842645E-5,-1.4156374E-3,-3.7770392E-3,-7.049463E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,41,43,45,-1,47,49,51,53,55,57,-1,59,-1,-1,-1,61,-1,63,-1,-1,-1,-1,-1,-1,65,-1,67,69,-1,71,-1,73,75,-1,-1,-1,77,-1,-1,79,-1,81,-1,-1,-1,-1,-1,-1,-1,83,85,-1,-1,-1,-1,-1,87,-1,-1,-1,89,91,-1,-1,-1,-1,-1],"loss_changes":[1.8218996E1,3.690335E0,2.0569448E0,2.0811377E0,2.1955082E0,4.4024658E-1,5.8323956E-1,6.988702E-1,6.155394E-1,3.501675E-1,3.8638353E-1,1.2565506E-1,1.24969006E-1,0E0,0E0,1.473751E-1,1.4069253E-1,2.9205006E-1,3.276744E-1,1.4596255E-1,3.209664E-2,0E0,2.1871805E-1,0E0,0E0,0E0,1.576786E-1,8.1237316E-2,6.428361E-2,0E0,6.6115804E-2,2.559295E-1,1.3344455E-1,1.3734168E-1,1.3236217E-1,9.2529446E-2,0E0,8.454815E-3,0E0,0E0,0E0,1.5534878E-2,0E0,2.1460056E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.285632E-2,0E0,6.3620985E-2,2.4115935E-2,0E0,3.7443664E-2,0E0,4.1603446E-3,6.6394955E-2,0E0,0E0,0E0,2.054739E-2,0E0,0E0,2.9730711E-2,0E0,5.0234497E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.6839496E-2,2.5750656E-2,0E0,0E0,0E0,0E0,0E0,1.23205185E-2,0E0,0E0,0E0,3.7729423E-3,6.7709684E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,37,37,41,41,43,43,50,50,52,52,53,53,55,55,57,57,58,58,62,62,65,65,67,67,75,75,76,76,82,82,86,86,87,87],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,42,44,46,-1,48,50,52,54,56,58,-1,60,-1,-1,-1,62,-1,64,-1,-1,-1,-1,-1,-1,66,-1,68,70,-1,72,-1,74,76,-1,-1,-1,78,-1,-1,80,-1,82,-1,-1,-1,-1,-1,-1,-1,84,86,-1,-1,-1,-1,-1,88,-1,-1,-1,90,92,-1,-1,-1,-1,-1],"split_conditions":[1.6907827E9,4.3199274E8,1.2738551E4,1.122053E6,1.83E2,8.391885E8,1.7754914E0,1.83E2,4.580147E6,1.0886205E9,2.59E3,9.035389E6,2.318509E6,3.5654034E-2,1.3930933E-2,6.38524E5,2.798795E2,4.3683886E9,5.215832E10,3.15888E3,1.0428423E9,2.1156134E-2,2.8699438E7,2.787767E-3,9.561152E-3,6.8402714E-3,4.8778326E10,2.1811952E7,1.606E3,2.8672514E-3,1.852E3,9.068E3,2.4E1,2.04E2,8.353684E2,1.3141365E7,3.1317326E-3,7.6613E4,1.7255036E-3,1.1866275E-2,8.9727994E-4,2.8125507E-1,9.613291E-3,6.32E2,-4.654696E-3,-7.3876632E-3,-1.2593819E-2,-6.960762E-3,-4.693184E-4,4.259935E-3,8.622E4,-1.1832423E-2,1.460109E6,3.016E3,9.672139E-3,1E0,-6.8577006E-3,7.404E3,2.3912651E8,2.6814207E-3,6.703334E-3,2.110158E-2,1E0,-1.2946682E-2,-5.955798E-3,1.4809E4,-7.507959E-3,4.8E1,-1.2614303E-3,7.998905E-4,4.169543E-3,2.7057284E-3,-1.481318E-3,-2.8552879E-3,-6.5621263E-3,2.433E3,1.8484124E3,1.51023455E-2,6.879386E-3,-2.8372942E-3,6.184368E-4,-1.6519009E-3,6.3765E4,4.179272E-4,-4.6484703E-3,2.481076E-3,8.24E2,1.7739E4,-2.6574412E-3,7.1842645E-5,-1.4156374E-3,-3.7770392E-3,-7.049463E-3],"split_indices":[27,16,5,9,47,16,59,47,2,16,20,2,1,0,0,2,5,10,10,5,11,0,18,0,0,0,21,18,19,0,3,7,17,4,5,18,0,8,0,0,0,63,0,4,0,0,0,0,0,0,8,0,6,0,0,19,0,3,21,0,0,0,34,0,0,7,0,17,0,0,0,0,0,0,0,20,5,0,0,0,0,0,28,0,0,0,0,28,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,2.19E2,6.2E1,1.63E2,5.6E1,5.2E1,1E1,8.7E1,7.6E1,4.3E1,1.3E1,1.7E1,3.5E1,6E0,4E0,7.7E1,1E1,5.3E1,2.3E1,3.1E1,1.2E1,5E0,8E0,5E0,1.2E1,4E0,3.1E1,4.2E1,3.5E1,4E0,6E0,1.8E1,3.5E1,1.3E1,1E1,2.7E1,4E0,7E0,5E0,5E0,3E0,2.3E1,8E0,3.9E1,3E0,3E1,5E0,3E0,3E0,6E0,1.2E1,3E0,3.2E1,9E0,4E0,7E0,3E0,8E0,1.9E1,3E0,4E0,9E0,1.4E1,3.6E1,3E0,9E0,3E0,2.7E1,5E0,5E0,4E0,3E0,4E0,3E0,5E0,1E1,9E0,1.1E1,3E0,5E0,4E0,4E0,2.3E1,3E0,7E0,3E0,6E0,2E1,3E0,3E0,3E0,5E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.8832276E-3,-1.0019486E-1,4.7020456E-1,-1.7996775E-1,9.719142E-2,3.5207853E-1,6.857462E-1,-2.827368E-1,-7.988598E-2,5.472894E-3,3.7202507E-1,1.5626296E-1,4.3531635E-1,8.775698E-1,6.0453415E-3,-3.0786827E-1,-7.100356E-2,1.2622463E-2,-1.0167157E-1,-3.539714E-2,1.4465609E-1,4.92437E-1,-4.973507E-4,8.6007995E-4,6.3243317E-3,4.8224267E-1,9.051002E-3,3.1530675E-2,1.3800155E-2,-3.1962198E-1,-8.1151316E-4,-6.1206263E-3,2.7736896E-3,-1.5116926E-1,3.4025427E-2,-7.357109E-2,5.167274E-3,5.352677E-3,1.9254658E-3,1.8354582E-2,5.62941E-3,1.7079148E-2,6.1952677E-3,-3.3844933E-1,-6.4512803E-3,-1.2577072E-1,-1.5503998E-2,1.556347E-1,-1.3963011E-1,-3.329984E-2,-1.5840839E-1,-1.2769758E-2,-2.9581156E-1,-1.3766193E-1,2.2267094E-3,5.485965E-4,1.9188307E-1,-3.060539E-2,-9.512347E-3,2.87863E-3,-5.410057E-2,-7.1406397E-3,-2.2075295E-3,-9.7965235E-3,-4.738154E-3,-1.5895148E-1,-6.396033E-2,2.4893403E-3,7.2480403E-3,-2.1967622E-5,-1.7020549E-3,-9.472045E-2,-1.2357294E-2,-1.8959567E-1,-8.412054E-2,-9.776121E-5,-3.137429E-3,-3.7931092E-3,-7.984018E-4,-1.6999505E-3,1.4471311E-2,-2.085951E-1,-6.019474E-4,-4.2195725E-3,-5.2224983E-2,-2.9378932E-4,1.2988695E-3,-1.977419E-3,-2.251194E-1,-3.382178E-4,-2.208258E-3,-8.894597E-3,-5.2942843E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,35,37,39,-1,-1,-1,41,-1,-1,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1,-1,51,-1,53,-1,55,57,59,61,-1,63,65,-1,-1,67,69,-1,-1,71,-1,-1,-1,-1,73,75,-1,-1,-1,-1,77,79,81,83,-1,-1,-1,-1,-1,85,87,-1,-1,89,-1,-1,-1,91,-1,-1,-1,-1],"loss_changes":[1.3623831E1,3.7020726E0,1.0570793E0,1.6950598E0,1.7120631E0,5.0245047E-1,1.4589634E0,4.1447592E-1,9.242316E-1,3.0091834E-1,8.007734E-1,5.72609E-2,3.799057E-2,1.2932205E-1,0E0,2.3446798E-1,2.0364514E-1,0E0,5.626288E-1,3.1846127E-1,1.2076825E-2,2.2135115E-1,0E0,0E0,0E0,1.3604999E-1,0E0,0E0,0E0,8.926821E-2,0E0,0E0,0E0,4.514482E-1,5.074481E-1,1.14302695E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.921629E-2,0E0,1.4397597E-1,0E0,6.706187E-2,1.609096E-1,6.8659455E-2,4.7458887E-2,0E0,3.3876896E-3,7.613242E-2,0E0,0E0,2.4530768E-2,4.6289307E-3,0E0,0E0,3.6132865E-2,0E0,0E0,0E0,0E0,8.036268E-2,2.828445E-2,0E0,0E0,0E0,0E0,1.485718E-2,1.3788246E-2,9.025955E-2,1.4126509E-2,0E0,0E0,0E0,0E0,0E0,5.2541033E-3,4.9396753E-2,0E0,0E0,6.1116926E-3,0E0,0E0,0E0,2.59732E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,25,25,29,29,33,33,34,34,35,35,43,43,45,45,47,47,48,48,49,49,50,50,52,52,53,53,56,56,57,57,60,60,65,65,66,66,71,71,72,72,73,73,74,74,80,80,81,81,84,84,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,36,38,40,-1,-1,-1,42,-1,-1,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1,-1,52,-1,54,-1,56,58,60,62,-1,64,66,-1,-1,68,70,-1,-1,72,-1,-1,-1,-1,74,76,-1,-1,-1,-1,78,80,82,84,-1,-1,-1,-1,-1,86,88,-1,-1,90,-1,-1,-1,92,-1,-1,-1,-1],"split_conditions":[2.6865E4,4.579412E8,2.12632E5,4.54205E5,1.83E2,1.111798E9,1.7754914E0,1.83E2,3.308E3,1.5102771E9,4.72323E8,3.0371344E-1,1.78E2,1E0,6.0453415E-3,1.447831E6,1.782131E6,1.2622463E-2,1.914442E6,5.710497E6,1.01E2,6.259021E7,-4.973507E-4,8.6007995E-4,6.3243317E-3,7.57418E5,9.051002E-3,3.1530675E-2,1.3800155E-2,7.000598E7,-8.1151316E-4,-6.1206263E-3,2.7736896E-3,1.11386E5,3.14E2,8.4E1,5.167274E-3,5.352677E-3,1.9254658E-3,1.8354582E-2,5.62941E-3,1.7079148E-2,6.1952677E-3,1.17036E5,-6.4512803E-3,3.0831E4,-1.5503998E-2,4.070204E6,3.88957E5,1E0,7.3504E4,-1.2769758E-2,5.737332E9,2.2317708E3,2.2267094E-3,5.485965E-4,1.0922E4,5.133E3,-9.512347E-3,2.87863E-3,2.3912651E8,-7.1406397E-3,-2.2075295E-3,-9.7965235E-3,-4.738154E-3,6E2,2.500539E3,2.4893403E-3,7.2480403E-3,-2.1967622E-5,-1.7020549E-3,7.7643576E7,1.5323213E7,2.1321451E8,1.0821E5,-9.776121E-5,-3.137429E-3,-3.7931092E-3,-7.984018E-4,-1.6999505E-3,2.2105154E-1,1.4556772E-5,-6.019474E-4,-4.2195725E-3,4.821E3,-2.9378932E-4,1.2988695E-3,-1.977419E-3,1.952E4,-3.382178E-4,-2.208258E-3,-8.894597E-3,-5.2942843E-3],"split_indices":[3,16,3,6,47,16,59,47,7,16,27,63,17,36,0,9,27,0,6,2,19,18,0,0,0,22,0,0,0,21,0,0,0,28,17,19,0,0,0,0,0,0,0,18,0,7,0,2,12,45,8,0,10,5,0,0,20,3,0,0,21,0,0,0,0,17,5,0,0,0,0,18,18,27,12,0,0,0,0,0,63,64,0,0,3,0,0,0,20,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.33E2,5E1,1.66E2,6.7E1,3.4E1,1.6E1,8.1E1,8.5E1,5.1E1,1.6E1,1.1E1,2.3E1,1.1E1,5E0,7.2E1,9E0,3E0,8.2E1,4E1,1.1E1,1.2E1,4E0,3E0,8E0,1.6E1,7E0,8E0,3E0,6.9E1,3E0,5E0,4E0,6E1,2.2E1,3.4E1,6E0,8E0,3E0,9E0,3E0,1.3E1,3E0,5.8E1,1.1E1,5.7E1,3E0,1.3E1,9E0,2.4E1,1E1,2.1E1,3.7E1,5.4E1,3E0,3E0,1E1,6E0,3E0,3E0,2.1E1,5E0,5E0,3.4E1,3E0,4.1E1,1.3E1,3E0,7E0,3E0,3E0,1E1,1.1E1,2.8E1,1.3E1,5E0,8E0,7E0,3E0,4E0,7E0,2.5E1,3E0,4E0,9E0,4E0,3E0,3E0,2.2E1,3E0,6E0,1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[9.6403565E-3,-1.3837782E-1,3.2530534E-1,-2.067246E-1,5.4634362E-2,2.0483917E-1,5.4797196E-1,9.464351E-3,-2.2336969E-1,-9.04179E-3,1.2811976E-2,3.3140418E-1,-4.5991298E-2,3.7373847E-1,2.1020805E-2,-3.1755522E-1,-1.3001776E-1,-5.853313E-2,1.4475343E-1,5.5199915E-1,2.3947254E-1,-8.052148E-3,1.6761482E-2,5.9758252E-3,1.3138204E-2,-3.4340042E-1,-8.0791235E-2,-1.10709794E-1,-2.90732E-1,-7.650912E-2,3.0712802E-3,5.4672696E-3,1.8789816E-3,2.4911009E-2,1.01280045E-2,1.5441641E-1,3.541887E-1,9.5301434E-2,-1.0905295E-1,-3.891447E-1,-8.432313E-3,-6.71653E-3,1.2122865E-3,-4.2231336E-2,-1.5825419E-1,-1.1552561E-2,-4.82457E-3,-1.2991641E-1,-2.4884503E-2,1.070821E-3,1.9530796E-1,1.5156406E-2,5.119458E-3,-5.074822E-4,4.4717654E-3,-6.3369856E-3,1.9395915E-4,-1.3136102E-2,-4.210137E-3,-1.0520175E-1,2.8253492E-2,-3.3623693E-4,-1.8033287E-1,-5.6794636E-2,-6.0128537E-3,-3.986437E-2,1.086897E-3,3.6329217E-3,7.3574563E-3,8.124473E-4,-1.399155E-1,-4.573491E-2,8.9461654E-2,-1.5510911E-1,-1.0442995E-2,3.6624182E-4,-4.1141226E-3,-5.328333E-2,2.2544741E-4,-2.2030184E-3,-5.378715E-3,-2.6198307E-3,4.355224E-5,3.8357673E-3,9.646462E-4,-1.6954963E-1,-2.685857E-3,-5.176802E-4,-2.0994705E-3,-5.7749166E-3,-2.4274443E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,23,-1,25,27,29,31,33,35,-1,37,-1,-1,39,41,43,45,47,-1,-1,-1,-1,-1,49,51,53,55,57,-1,-1,-1,59,61,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,69,71,-1,73,75,-1,77,-1,-1,-1,-1,79,81,83,85,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,89,-1,-1,-1,-1,-1],"loss_changes":[1.3174501E1,2.5441127E0,2.3018951E0,1.2492485E0,1.1187507E0,1.9200237E0,3.2345104E-1,0E0,1.1720638E0,3.4338024E-1,0E0,6.789975E-1,2.732382E-1,1.4863491E-2,0E0,3.8836288E-1,1.8813062E-1,1.0139008E-1,1.3930827E-2,3.358631E-1,2.3802686E-1,0E0,1.783777E-1,0E0,0E0,1.2530136E-1,1.2902918E-1,2.0494163E-1,1.3415515E-2,8.223815E-2,0E0,0E0,0E0,0E0,0E0,8.762869E-2,1.9832003E-1,5.5483565E-2,7.1925916E-2,1.2775326E-1,0E0,0E0,0E0,1.2724915E-1,1.221261E-1,0E0,0E0,5.178076E-2,1.6281903E-2,0E0,1.259619E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.145326E-2,6.749323E-2,0E0,6.962073E-2,4.018546E-2,0E0,9.178115E-3,0E0,0E0,0E0,0E0,1.1878416E-2,1.1862298E-2,1.0565646E-2,1.6900718E-2,0E0,0E0,0E0,3.8369875E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.740723E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,35,35,36,36,37,37,38,38,39,39,43,43,44,44,47,47,48,48,50,50,59,59,60,60,62,62,63,63,65,65,70,70,71,71,72,72,73,73,77,77,85,85],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,24,-1,26,28,30,32,34,36,-1,38,-1,-1,40,42,44,46,48,-1,-1,-1,-1,-1,50,52,54,56,58,-1,-1,-1,60,62,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,70,72,-1,74,76,-1,78,-1,-1,-1,-1,80,82,84,86,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,90,-1,-1,-1,-1,-1],"split_conditions":[4.764156E6,4.3199274E8,6.112126E9,2.302E3,1.83E2,3.6060664E10,4.7016313E3,9.464351E-3,8.75602E5,1.5102771E9,1.2811976E-2,1.619E3,1.6174E4,5.315E3,2.1020805E-2,1.83E2,3.916E3,3.99E2,1.01E2,4.4748E5,1.83E2,-8.052148E-3,1.11301E5,5.9758252E-3,1.3138204E-2,1.916746E6,1.852E3,9.3E1,4.695E5,8.3552723E8,3.0712802E-3,5.4672696E-3,1.8789816E-3,2.4911009E-2,1.01280045E-2,4.365884E6,1.3432413E10,3.718604E6,3E0,1.05213176E8,-8.432313E-3,-6.71653E-3,1.2122865E-3,1.496E3,9.532393E-6,-1.1552561E-2,-4.82457E-3,2E1,3.068292E6,1.070821E-3,3.0676925E9,1.5156406E-2,5.119458E-3,-5.074822E-4,4.4717654E-3,-6.3369856E-3,1.9395915E-4,-1.3136102E-2,-4.210137E-3,3.682E3,2.0170048E-2,-3.3623693E-4,1.14282E5,7.5469E4,-6.0128537E-3,2.9138386E7,1.086897E-3,3.6329217E-3,7.3574563E-3,8.124473E-4,2.496E4,5.3E1,6.2941E4,9.57317E5,-1.0442995E-2,3.6624182E-4,-4.1141226E-3,6.1E1,2.2544741E-4,-2.2030184E-3,-5.378715E-3,-2.6198307E-3,4.355224E-5,3.8357673E-3,9.646462E-4,7.503E3,-2.685857E-3,-5.176802E-4,-2.0994705E-3,-5.7749166E-3,-2.4274443E-3],"split_indices":[2,16,27,7,47,10,5,0,9,16,0,0,7,0,0,47,19,4,19,12,47,0,8,0,0,18,3,4,6,23,0,0,0,0,0,6,10,6,19,23,0,0,0,0,64,0,0,19,2,0,11,0,0,0,0,0,0,0,0,3,64,0,22,8,0,18,0,0,0,0,22,4,8,6,0,0,0,4,0,0,0,0,0,0,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.91E2,8.9E1,1.41E2,5E1,5.9E1,3E1,4E0,1.37E2,4.3E1,7E0,3.9E1,2E1,1.3E1,1.7E1,6.7E1,7E1,3.3E1,1E1,1E1,2.9E1,4E0,1.6E1,3E0,1E1,6E1,7E0,6.4E1,6E0,3E1,3E0,7E0,3E0,4E0,6E0,1.8E1,1.1E1,1E1,6E0,3.6E1,2.4E1,3E0,4E0,2.7E1,3.7E1,3E0,3E0,1.4E1,1.6E1,5E0,1.3E1,6E0,5E0,3E0,7E0,3E0,3E0,3.3E1,3E0,1.4E1,1.3E1,5E0,3.2E1,7E0,7E0,1.3E1,3E0,5E0,8E0,3E0,1.1E1,6E0,7E0,2.9E1,3E0,4E0,3E0,1E1,3E0,4E0,7E0,3E0,3E0,4E0,3E0,2.3E1,6E0,3E0,7E0,2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-1.3995446E-4,-1.6086783E-1,3.271978E-1,-2.5258976E-1,-4.4987727E-2,7.39544E-2,3.922929E-1,-2.748874E-1,1.8683877E-2,-1.14599645E-1,1.8181665E-1,2.697064E-1,-4.639712E-2,2.3668526E-1,4.7187126E-1,-3.493821E-1,-1.8048397E-1,5.4193484E-3,-3.3732152E-3,-1.196498E-2,-8.867086E-2,3.4904E-1,1.3133985E-2,1.2617346E-2,2.5753987E-3,-3.6997984E-3,1.3294814E-3,1.9087965E-2,1.3577299E-1,5.467343E-1,3.524721E-1,-4.098577E-1,-2.5499055E-1,-2.2240789E-1,-8.673898E-2,-6.5635316E-2,-1.1594663E-2,1.4435124E-2,3.626591E-3,2.1940153E-3,-1.1237304E-3,4.7093432E-2,1.540943E-2,9.684971E-3,6.1197066E-1,5.821491E-3,1.40830465E-2,-1.3777873E-2,-6.3847904E-3,-8.9443745E-3,-2.8416843E-3,-8.492905E-3,-1.6508424E-1,2.0304504E-3,-1.3333292E-1,-1.1799117E-1,6.788482E-2,1.65629E-1,-6.367833E-3,2.179813E-2,1.1866814E-2,-6.575482E-3,-3.5457602E-3,-1.4627276E-3,-1.6061181E-1,-5.9069794E-2,-2.2222528E-1,1.455984E-1,-4.1730467E-2,2.3269244E-3,6.54281E-3,-2.632819E-3,-6.165628E-3,-1.05222985E-1,-3.0191988E-4,-1.1906371E-2,-1.409369E-1,1.1474488E-3,6.265025E-3,-4.7870795E-5,-2.3028345E-3,-1.8937796E-3,-4.006176E-3,4.395754E-2,-2.28724E-3,-1.7869618E-3,-5.4597533E-3,2.4787006E-3,2.8095514E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,35,37,39,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,61,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,71,73,75,77,79,-1,-1,-1,-1,81,83,-1,85,-1,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,-1],"loss_changes":[1.3100486E1,1.7636366E0,1.322751E0,5.676718E-1,1.2049577E0,4.3657646E-1,6.9320965E-1,5.417423E-1,1.6549738E-1,3.5370153E-1,5.0315386E-1,1.310001E-1,7.688141E-2,7.824372E-1,1.827116E-1,1.6378355E-1,1.379149E-1,0E0,0E0,0E0,3.1412575E-1,1.609559E-1,2.8816672E-2,0E0,0E0,0E0,0E0,0E0,5.8569956E-1,2.008009E-1,1.9816732E-1,1.887989E-2,5.557096E-2,2.057743E-2,1.0613912E-1,3.6452845E-1,0E0,0E0,0E0,0E0,0E0,5.263053E-1,0E0,0E0,3.599024E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.6381564E-3,0E0,1.9074738E-2,2.14113E-1,1.3385482E-1,2.7458578E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.837958E-3,6.7776345E-2,1.0391343E-1,4.1584313E-2,8.291361E-3,0E0,0E0,0E0,0E0,5.170092E-3,4.0702276E-2,0E0,1.50818825E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.641685E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,20,20,21,21,22,22,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,41,41,44,44,52,52,54,54,55,55,56,56,57,57,64,64,65,65,66,66,67,67,68,68,73,73,74,74,76,76,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,36,38,40,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,62,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,72,74,76,78,80,-1,-1,-1,-1,82,84,-1,86,-1,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,1.374403E6,3.969357E6,1.83E2,1.3141365E7,1.3588E4,9.0629734E8,4.258E3,9.11E2,3.967E3,7.091E3,2.0599214E3,2.654028E6,1.619E3,2.2799654E0,2.315676E6,5.0373805E8,5.4193484E-3,-3.3732152E-3,-1.196498E-2,4.23859E5,6.4762153E9,2.1916846E3,1.2617346E-2,2.5753987E-3,-3.6997984E-3,1.3294814E-3,1.9087965E-2,3.298233E11,3.2842822E3,2.6731534E0,1.7645E4,7.0319E4,4.951E3,5.275E3,4.719012E6,-1.1594663E-2,1.4435124E-2,3.626591E-3,2.1940153E-3,-1.1237304E-3,5.4730977E9,1.540943E-2,9.684971E-3,3.954342E-1,5.821491E-3,1.40830465E-2,-1.3777873E-2,-6.3847904E-3,-8.9443745E-3,-2.8416843E-3,-8.492905E-3,8.3E1,2.0304504E-3,5.3E1,2.5459167E10,2.01E2,2.1602268E9,-6.367833E-3,2.179813E-2,1.1866814E-2,-6.575482E-3,-3.5457602E-3,-1.4627276E-3,7.482E3,1.619E3,3.757E3,1.9884E4,5.133E3,2.3269244E-3,6.54281E-3,-2.632819E-3,-6.165628E-3,9.35E2,3.3012803E9,-1.1906371E-2,2.57E2,1.1474488E-3,6.265025E-3,-4.7870795E-5,-2.3028345E-3,-1.8937796E-3,-4.006176E-3,4.98945E8,-2.28724E-3,-1.7869618E-3,-5.4597533E-3,2.4787006E-3,2.8095514E-4],"split_indices":[3,9,2,47,18,20,16,3,0,7,7,5,2,0,67,18,16,0,0,0,12,10,5,0,0,0,0,0,13,5,67,28,8,3,3,9,0,0,0,0,0,11,0,0,63,0,0,0,0,0,0,0,17,0,4,10,17,27,0,0,0,0,0,0,3,0,3,18,3,0,0,0,0,0,11,0,4,0,0,0,0,0,0,11,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.47E2,1.66E2,8.1E1,9.2E1,7.4E1,1.7E1,6.4E1,8.5E1,7E0,5.7E1,1.7E1,6E0,1.1E1,2.3E1,4.1E1,4.6E1,3.9E1,3E0,4E0,4E0,5.3E1,8E0,9E0,3E0,3E0,6E0,5E0,4E0,1.9E1,2.3E1,1.8E1,2.6E1,2E1,2.6E1,1.3E1,5E1,3E0,5E0,3E0,4E0,5E0,1.6E1,3E0,6E0,1.7E1,7E0,1.1E1,2.3E1,3E0,1.7E1,3E0,1.3E1,1.3E1,3E0,1E1,3.6E1,1.4E1,1.1E1,5E0,1.2E1,5E0,6E0,7E0,3E0,7E0,2.4E1,1.2E1,8E0,6E0,4E0,7E0,3E0,4E0,1.3E1,1.1E1,3E0,9E0,3E0,5E0,3E0,3E0,5E0,8E0,7E0,4E0,3E0,6E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[2.3155231E-2,-1.3746329E-1,3.1235737E-1,-1.9299243E-1,3.588083E-2,1.6439836E-1,4.85691E-1,-2.9513794E-1,-1.0823901E-1,-6.018671E-2,3.382884E-1,8.008524E-2,3.0110663E-1,7.374192E-1,3.5251558E-1,-3.3027342E-1,-1.8270437E-1,-6.2675975E-2,-2.0624588E-1,-1.321966E-1,-5.8850436E-3,4.1648358E-1,3.1328201E-3,9.686506E-3,2.0389174E-1,2.136258E-2,1.7676061E-1,4.264037E-1,2.8885879E-2,4.050689E-1,-3.9548107E-4,-1.1726207E-2,-6.551334E-3,-4.843819E-4,-7.002897E-3,-3.513688E-4,-1.16391964E-1,-1.4716035E-1,-1.2918679E-2,-5.741532E-3,-6.9499016E-2,-8.500521E-2,3.1430036E-2,1.5930265E-2,6.8995133E-3,-5.5639025E-2,1.24744505E-1,3.5526168E-3,8.000515E-3,-1.0528769E-3,2.667655E-1,1.7446486E-2,6.573124E-3,2.584327E-1,4.8474923E-1,-2.5688231E-2,4.681029E-3,-1.4900355E-1,-1.21611E-2,-2.147847E-1,-5.8057554E-2,-2.7742968E-3,-1.0063307E-3,-9.841517E-4,-3.8042492E-3,3.0776656E-3,-6.010709E-4,-8.007014E-2,1.2126176E-3,5.2195666E-3,1.2052904E-3,3.7063468E-1,1.5200177E-3,9.80089E-3,2.9040363E-3,2.0611092E-2,1.00372E-2,5.5745738E-3,-3.7080161E-3,-1.6775046E-1,-1.6646183E-3,-1.3589043E-3,9.1572414E-4,-8.1573045E-3,-3.7899038E-3,2.188179E-4,-3.2970353E-3,-1.6100389E-3,2.2114892E-3,-3.294044E-2,-1.1576125E-1,5.6831595E-3,1.4542024E-2,-3.451189E-2,2.9452732E-3,-2.1826117E-3,-1.832189E-1,-1.6892549E-3,-1.6630237E-4,-4.6541085E-3,-1.8668011E-3,-3.671752E-3,2.9469686E-2,-6.3887103E-3,-2.9499645E-3,2.5650873E-3,-1.3091621E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,-1,49,51,-1,53,-1,-1,-1,-1,-1,55,57,59,-1,-1,61,63,65,-1,-1,67,69,-1,-1,-1,71,-1,-1,73,75,77,-1,79,81,83,85,-1,-1,-1,-1,-1,87,89,-1,-1,-1,91,-1,-1,-1,-1,-1,93,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,97,99,-1,-1,101,-1,-1,103,-1,-1,-1,-1,-1,105,-1,-1,-1,-1],"loss_changes":[1.3142181E1,1.7587929E0,2.5104265E0,1.162746E0,1.3276801E0,6.2135863E-1,1.2821531E0,1.8458319E-1,3.3091623E-1,1.36081E-1,1.6672862E-1,3.1179455E-1,8.294188E-1,3.636284E-1,6.184664E-1,1.24200344E-1,9.605837E-2,1.807418E-1,2.1759093E-1,3.1027555E-2,6.467159E-2,2.0727277E-2,0E0,1.8733007E-1,2.6862562E-2,0E0,3.2923204E-1,6.9122195E-2,0E0,2.0187807E-1,0E0,0E0,0E0,0E0,0E0,9.9766046E-2,9.6762866E-2,1.0804045E-1,0E0,0E0,2.4103746E-3,9.028621E-3,3.089498E-2,0E0,0E0,3.9491408E-2,2.3320973E-2,0E0,0E0,0E0,2.5614393E-1,0E0,0E0,6.5479815E-2,2.576661E-1,6.5001704E-2,0E0,3.145027E-2,1.0965258E-2,1.140511E-2,2.947265E-2,0E0,0E0,0E0,0E0,0E0,4.110963E-2,1.8054217E-2,0E0,0E0,0E0,4.6211123E-2,0E0,0E0,0E0,0E0,0E0,6.515749E-2,0E0,1.5404284E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.5275794E-3,3.2707825E-3,0E0,0E0,6.953375E-2,0E0,0E0,2.7668476E-3,0E0,0E0,0E0,0E0,0E0,3.1420093E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,27,27,29,29,35,35,36,36,37,37,40,40,41,41,42,42,45,45,46,46,50,50,53,53,54,54,55,55,57,57,58,58,59,59,60,60,66,66,67,67,71,71,77,77,79,79,89,89,90,90,93,93,96,96,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,-1,50,52,-1,54,-1,-1,-1,-1,-1,56,58,60,-1,-1,62,64,66,-1,-1,68,70,-1,-1,-1,72,-1,-1,74,76,78,-1,80,82,84,86,-1,-1,-1,-1,-1,88,90,-1,-1,-1,92,-1,-1,-1,-1,-1,94,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,98,100,-1,-1,102,-1,-1,104,-1,-1,-1,-1,-1,106,-1,-1,-1,-1],"split_conditions":[9.91E3,3.1633773E8,4.148E4,7.31E2,1.83E2,1.83E2,4.4166436E0,6.549254E6,1.0112496E8,1.7140013E3,5.2E1,2.6542E9,3.1884794E8,5.7577495E3,4.40538E5,5.8688E5,6.089041E7,1.34E2,5.315E3,1.4812E4,6.039798E8,2E0,3.1328201E-3,5.421011E7,6.2699E4,2.136258E-2,7.992919E8,5.2203887E3,2.8885879E-2,2.345689E7,-3.9548107E-4,-1.1726207E-2,-6.551334E-3,-4.843819E-4,-7.002897E-3,3.183066E6,5.5293235E9,1.19021E6,-1.2918679E-2,-5.741532E-3,1.5323213E7,1.1E2,2.5E2,1.5930265E-2,6.8995133E-3,2.144E4,1.4728E4,3.5526168E-3,8.000515E-3,-1.0528769E-3,4.3740602E10,1.7446486E-2,6.573124E-3,2.8065149E10,6.541211E10,1.6506441E9,4.681029E-3,1.9313741E3,2.92E2,2.0197088E8,2.0201286E9,-2.7742968E-3,-1.0063307E-3,-9.841517E-4,-3.8042492E-3,3.0776656E-3,1.5022E4,1.0773E4,1.2126176E-3,5.2195666E-3,1.2052904E-3,3E1,1.5200177E-3,9.80089E-3,2.9040363E-3,2.0611092E-2,1.00372E-2,1.7183E4,-3.7080161E-3,8.74E2,-1.6646183E-3,-1.3589043E-3,9.1572414E-4,-8.1573045E-3,-3.7899038E-3,2.188179E-4,-3.2970353E-3,-1.6100389E-3,2.2114892E-3,1.878288E6,1.4468E4,5.6831595E-3,1.4542024E-2,6.7452E4,2.9452732E-3,-2.1826117E-3,1.8468E4,-1.6892549E-3,-1.6630237E-4,-4.6541085E-3,-1.8668011E-3,-3.671752E-3,4.125617E2,-6.3887103E-3,-2.9499645E-3,2.5650873E-3,-1.3091621E-3],"split_indices":[3,16,3,0,47,47,60,18,16,5,4,21,11,5,3,2,16,4,0,7,16,19,0,18,8,0,23,5,0,2,0,0,0,0,0,2,11,2,0,0,18,4,17,0,0,3,7,0,0,0,26,0,0,10,10,11,0,5,4,16,11,0,0,0,0,0,7,3,0,0,0,19,0,0,0,0,0,22,0,0,0,0,0,0,0,0,0,0,0,2,3,0,0,18,0,0,7,0,0,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.81E2,1E2,1.37E2,4.4E1,5.5E1,4.5E1,6.1E1,7.6E1,3.4E1,1E1,3.5E1,2E1,1.4E1,3.1E1,4.5E1,1.6E1,5.3E1,2.3E1,1.4E1,2E1,7E0,3E0,2.3E1,1.2E1,4E0,1.6E1,6E0,8E0,2.7E1,4E0,3.4E1,1.1E1,3E0,1.3E1,2.5E1,2.8E1,1.9E1,4E0,7E0,7E0,6E0,1.4E1,4E0,3E0,1.5E1,8E0,5E0,7E0,5E0,1.1E1,3E0,3E0,1.1E1,1.6E1,2.2E1,3E0,2.1E1,7E0,1E1,9E0,4E0,3E0,3E0,3E0,4E0,1E1,1.2E1,3E0,5E0,3E0,7E0,4E0,8E0,3E0,7E0,9E0,1.7E1,5E0,1.7E1,4E0,4E0,3E0,6E0,4E0,4E0,5E0,6E0,4E0,6E0,6E0,3E0,4E0,1.2E1,5E0,3E0,1.4E1,3E0,3E0,3E0,3E0,5E0,7E0,1.1E1,3E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[6.4829714E-3,-1.318115E-1,3.204165E-1,-1.7552851E-1,9.362496E-2,3.499123E-2,4.0205422E-1,-3.0334035E-1,-9.533184E-2,7.013413E-3,1.802339E-2,1.11075364E-1,-9.563794E-3,3.4980392E-1,2.1234265E-2,-1.2663601E-2,-2.6345775E-1,-1.5863736E-1,-3.7294865E-2,8.430352E-2,-1.9674979E-1,2.2417365E-1,-4.418192E-2,4.6639556E-1,2.8859258E-1,-2.9648656E-1,-1.896741E-1,-2.556632E-1,-1.1919367E-1,-5.787513E-2,7.203703E-2,-3.675442E-2,2.211427E-1,-1.1201166E-2,1.3351145E-3,2.1729718E-3,9.066753E-3,1.02405E-3,-4.1244025E-3,5.14561E-1,7.6691224E-3,1.8177865E-1,3.4540978E-1,-9.987353E-3,-4.162319E-3,-3.0992606E-3,-6.5282113E-3,-1.0000392E-2,-5.283024E-3,-4.5375112E-2,-1.3866414E-1,-7.224985E-2,3.5768976E-3,6.3874386E-4,4.2591775E-3,-4.7297883E-3,6.293947E-2,2.676836E-3,1.1024504E-2,1.8405931E-2,9.5650535E-3,1.3026981E-1,9.777E-3,6.182384E-3,1.2694471E-2,-2.948422E-3,1.1363329E-3,-1.6980673E-3,-1.4686334E-1,-4.1535888E-2,-1.16780855E-1,3.3512062E-3,1.9421708E-4,4.8866873E-3,1.4113436E-3,-4.992999E-3,-2.0846187E-3,-6.3095555E-2,2.9076802E-2,-1.40663E-1,-1.286953E-4,-3.3916596E-2,-8.215339E-2,-1.479271E-4,1.785843E-3,-1.9432585E-3,-5.0560096E-3,-1.9254014E-3,-1.4271947E-2,-3.5850185E-3,-1.7260116E-3,-8.6063E-4,5.668154E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,-1,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,-1,-1,-1,-1,-1,-1,59,-1,61,63,-1,-1,-1,-1,-1,-1,65,67,69,-1,-1,-1,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,75,77,79,-1,-1,-1,-1,-1,-1,81,83,85,-1,87,89,-1,-1,-1,-1,-1,91,-1,-1,-1,-1],"loss_changes":[1.2068264E1,1.9185278E0,1.9665565E0,1.6315956E0,1.2884549E0,5.38361E-1,6.576538E-1,1.1908007E-1,3.651638E-1,4.5755538E-1,0E0,3.0841312E-1,0E0,2.7219534E-1,0E0,0E0,5.5865526E-2,1.532253E-1,1.2567344E-1,3.5600275E-1,3.026536E-1,7.106727E-2,5.511441E-2,7.519889E-2,1.7996645E-1,2.9020786E-2,1.3263822E-3,1.4476657E-2,4.545027E-2,1.1977564E-1,2.5690772E-2,1.4106405E-1,1.3896108E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.3303967E-3,0E0,6.0943663E-2,1.1196232E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.67952E-2,1.1975765E-2,5.39615E-2,0E0,0E0,0E0,0E0,1.5771901E-2,0E0,0E0,0E0,0E0,1.7547265E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.8918467E-3,4.2939033E-2,4.522699E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.891478E-3,6.4783706E-3,8.963674E-3,0E0,4.23408E-3,4.032761E-3,0E0,0E0,0E0,0E0,0E0,1.4460111E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,39,39,41,41,42,42,49,49,50,50,51,51,56,56,61,61,68,68,69,69,70,70,77,77,78,78,79,79,81,81,82,82,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,-1,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,-1,-1,-1,-1,-1,-1,60,-1,62,64,-1,-1,-1,-1,-1,-1,66,68,70,-1,-1,-1,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,76,78,80,-1,-1,-1,-1,-1,-1,82,84,86,-1,88,90,-1,-1,-1,-1,-1,92,-1,-1,-1,-1],"split_conditions":[3.47989E5,1.83E2,2.7732336E3,6.97149E5,2.9124946E7,1.33097E5,1.1170185E4,1.2083252E7,1.969156E6,9.9914E4,1.802339E-2,8.3E1,-9.563794E-3,1.7655054E-1,2.1234265E-2,-1.2663601E-2,1.1750605E8,6.2935E4,2.402988E6,8.3171934E-2,3.044519E6,1.595E3,1.7259076E10,2.317698E7,3.937E3,1.6744E4,1.41E2,1.0816216E3,2.433E3,1.3925089E9,3.718604E6,6.7452E4,2.8E1,-1.1201166E-2,1.3351145E-3,2.1729718E-3,9.066753E-3,1.02405E-3,-4.1244025E-3,4.93E2,7.6691224E-3,8.6022E4,2.029183E0,-9.987353E-3,-4.162319E-3,-3.0992606E-3,-6.5282113E-3,-1.0000392E-2,-5.283024E-3,1.0939E4,1.1E1,1.14E2,3.5768976E-3,6.3874386E-4,4.2591775E-3,-4.7297883E-3,3.649951E9,2.676836E-3,1.1024504E-2,1.8405931E-2,9.5650535E-3,6.7795E4,9.777E-3,6.182384E-3,1.2694471E-2,-2.948422E-3,1.1363329E-3,-1.6980673E-3,8.998854E7,8.1214E4,1.521867E6,3.3512062E-3,1.9421708E-4,4.8866873E-3,1.4113436E-3,-4.992999E-3,-2.0846187E-3,1.25E2,4.821E3,2.6304E4,-1.286953E-4,5.9418E4,5.4026153E9,-1.479271E-4,1.785843E-3,-1.9432585E-3,-5.0560096E-3,-1.9254014E-3,1.92E4,-3.5850185E-3,-1.7260116E-3,-8.6063E-4,5.668154E-5],"split_indices":[12,47,5,9,18,8,5,11,2,12,0,19,0,63,0,0,16,18,6,64,2,0,10,18,0,7,20,5,20,16,6,18,17,0,0,0,0,0,0,19,0,3,67,0,0,0,0,0,0,7,19,19,0,0,0,0,10,0,0,0,0,8,0,0,0,0,0,0,18,8,6,0,0,0,0,0,0,4,3,20,0,8,10,0,0,0,0,0,20,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.92E2,8.4E1,1.61E2,3.1E1,1.9E1,6.5E1,6.1E1,1E2,2.7E1,4E0,1.6E1,3E0,5.6E1,9E0,1.6E1,4.5E1,4.7E1,5.3E1,2E1,7E0,9E0,7E0,1.7E1,3.9E1,2.9E1,1.6E1,1.2E1,3.5E1,4.5E1,8E0,1.1E1,9E0,4E0,3E0,3E0,6E0,4E0,3E0,1.3E1,4E0,1.5E1,2.4E1,2.6E1,3E0,3E0,1.3E1,6E0,6E0,8E0,2.7E1,4.2E1,3E0,5E0,3E0,5E0,6E0,5E0,4E0,9E0,4E0,1.2E1,3E0,7E0,1.7E1,5E0,3E0,3E0,2.4E1,2.6E1,1.6E1,3E0,3E0,9E0,3E0,2.1E1,3E0,2E1,6E0,1.3E1,3E0,9E0,1.1E1,3E0,3E0,3E0,1E1,3E0,6E0,4E0,7E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.4617624E-3,-1.529497E-1,2.8057614E-1,-1.978553E-1,2.3685884E-2,1.9164558E-1,4.7468096E-1,-2.679137E-1,-9.952817E-2,2.029332E-1,-4.5121234E-2,1.2473906E-1,4.047059E-1,3.640575E-1,2.0237688E-2,-2.9652202E-1,-4.3299623E-2,-1.4586525E-1,2.298477E-3,2.7399692E-1,3.2422197E-4,-1.0739829E-1,2.0163901E-2,1.5117548E-1,-4.0289783E-3,1.6340257E-3,4.7692475E-1,4.901845E-3,1.2582439E-2,-3.3210716E-1,-2.2238706E-1,-1.2902366E-1,2.7480982E-3,-1.105999E-1,-1.5156778E-2,-2.4784375E-3,2.5485327E-2,1.05086155E-2,4.5035817E-3,-1.3788554E-1,-1.2452774E-3,3.445259E-3,-2.1154827E-2,9.8153576E-2,4.3052676E-1,1.8512495E-2,8.051918E-3,-1.3230065E-2,-2.8654733E-1,-2.35621E-1,-2.7063882E-3,-5.3585838E-3,-1.9094063E-3,3.1546543E-3,-1.3580047E-1,6.532427E-2,-8.934361E-3,-1.8268717E-3,-5.8749276E-3,-6.267806E-2,1.3264641E-3,4.3461543E-2,1.5858915E-1,1.8825566E-2,7.1126805E-3,-9.581298E-3,-5.1615797E-3,-8.225075E-3,-4.761921E-3,-1.5503033E-1,-5.8748648E-2,2.9182914E-3,5.575764E-4,-3.2175954E-2,1.0935239E-3,-3.352253E-3,-1.7844478E-4,-2.7763875E-2,1.2293607E-1,9.160247E-2,5.77074E-3,-1.6460302E-1,-1.7398193E-3,-2.2737333E-3,-8.4428006E-4,-3.5658857E-4,-1.4559048E-3,1.264083E-3,-6.936871E-2,5.3314995E-3,1.6942855E-3,3.918541E-3,1.2414884E-3,-5.6539765E-3,-1.9292339E-3,-7.8371225E-4,-2.8268853E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,-1,39,41,43,-1,-1,45,-1,-1,47,49,51,-1,53,-1,-1,55,-1,-1,57,-1,-1,59,61,63,-1,-1,-1,65,67,-1,-1,-1,-1,69,71,73,-1,-1,75,-1,77,79,-1,-1,-1,-1,-1,-1,81,83,-1,-1,85,-1,-1,-1,87,89,91,-1,93,-1,-1,-1,-1,-1,-1,95,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2584566E1,1.4958882E0,1.7042618E0,1.0066314E0,4.9016276E-1,9.901414E-1,3.1638527E-1,5.462074E-1,3.023097E-1,1.4799958E-1,1.20072775E-1,3.9549875E-1,4.1033173E-1,5.981779E-2,0E0,1.2927723E-1,1.3234983E-1,4.5222193E-1,4.05729E-2,1.0607362E-2,0E0,2.6088223E-2,5.6103636E-2,7.279526E-1,0E0,0E0,1.6789603E-1,0E0,0E0,6.1971188E-2,3.143978E-2,8.397765E-3,0E0,2.2884369E-1,0E0,0E0,2.389478E-2,0E0,0E0,2.5848776E-2,0E0,0E0,3.1067755E-2,1.4184794E-1,1.2948132E-1,0E0,0E0,0E0,1.3427734E-3,5.829811E-3,0E0,0E0,0E0,0E0,4.5714915E-2,8.159433E-3,1.1065208E-2,0E0,0E0,1.600616E-2,0E0,1.4492835E-1,1.2679815E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.9136071E-2,1.6267449E-3,0E0,0E0,1.4268323E-3,0E0,0E0,0E0,4.1769635E-2,2.7299836E-2,6.4912923E-3,0E0,1.953429E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.338762E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,26,26,29,29,30,30,31,31,33,33,36,36,39,39,42,42,43,43,44,44,48,48,49,49,54,54,55,55,56,56,59,59,61,61,62,62,69,69,70,70,73,73,77,77,78,78,79,79,81,81,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,-1,40,42,44,-1,-1,46,-1,-1,48,50,52,-1,54,-1,-1,56,-1,-1,58,-1,-1,60,62,64,-1,-1,-1,66,68,-1,-1,-1,-1,70,72,74,-1,-1,76,-1,78,80,-1,-1,-1,-1,-1,-1,82,84,-1,-1,86,-1,-1,-1,88,90,92,-1,94,-1,-1,-1,-1,-1,-1,96,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,4.3199274E8,1.021304E6,1.042754E6,7.386E3,1.83E2,1.09E2,1.83E2,1.22661E6,4.9E1,2.8699438E7,5.11E2,7.143424E7,2.5E1,2.0237688E-2,2.315676E6,6.52028E5,8.1188E4,2.846864E6,5.71365E5,3.2422197E-4,4.760905E9,1.37507E5,7.8908E4,-4.0289783E-3,1.6340257E-3,5.15297E5,4.901845E-3,1.2582439E-2,1.2590113E7,1.6802073E3,4.1E1,2.7480982E-3,3.2416E4,-1.5156778E-2,-2.4784375E-3,1.34E2,1.05086155E-2,4.5035817E-3,7.031E3,-1.2452774E-3,3.445259E-3,1.19E2,3.636759E6,5.4E1,1.8512495E-2,8.051918E-3,-1.3230065E-2,2.2728746E-1,4.0200136E7,-2.7063882E-3,-5.3585838E-3,-1.9094063E-3,3.1546543E-3,2.281119E6,1.2465643E3,5.601505E6,-1.8268717E-3,-5.8749276E-3,8.027122E8,1.3264641E-3,2.9138386E7,1.4947E4,1.8825566E-2,7.1126805E-3,-9.581298E-3,-5.1615797E-3,-8.225075E-3,-4.761921E-3,2.94E2,1.369E3,2.9182914E-3,5.575764E-4,3.751062E6,1.0935239E-3,-3.352253E-3,-1.7844478E-4,3.43E2,1.4728E4,1.9002575E10,5.77074E-3,2.7496557E8,-1.7398193E-3,-2.2737333E-3,-8.4428006E-4,-3.5658857E-4,-1.4559048E-3,1.264083E-3,3.08131E5,5.3314995E-3,1.6942855E-3,3.918541E-3,1.2414884E-3,-5.6539765E-3,-1.9292339E-3,-7.8371225E-4,-2.8268853E-3],"split_indices":[12,16,12,9,7,47,4,47,6,4,18,4,16,17,0,18,21,28,2,6,0,10,12,3,0,0,12,0,0,11,5,17,0,12,0,0,4,0,0,3,0,0,4,6,19,0,0,0,63,18,0,0,0,0,9,5,2,0,0,16,0,18,7,0,0,0,0,0,0,19,0,0,0,2,0,0,0,17,7,10,0,16,0,0,0,0,0,0,12,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.9E2,1.87E2,1.03E2,1.49E2,3.8E1,7.2E1,3.1E1,8.6E1,6.3E1,1E1,2.8E1,5.6E1,1.6E1,2E1,1.1E1,7.6E1,1E1,4.3E1,2E1,7E0,3E0,1.4E1,1.4E1,5.1E1,5E0,3E0,1.3E1,3E0,1.7E1,4.9E1,2.7E1,6E0,4E0,4E1,3E0,4E0,1.6E1,4E0,3E0,9E0,5E0,4E0,1E1,4.4E1,7E0,8E0,5E0,1.5E1,3.4E1,2.4E1,3E0,3E0,3E0,4E0,3.6E1,7E0,9E0,4E0,5E0,6E0,4E0,2.4E1,2E1,3E0,4E0,3E1,4E0,1.8E1,6E0,2.8E1,8E0,4E0,3E0,6E0,3E0,3E0,3E0,1.3E1,1.1E1,6E0,1.4E1,2.5E1,3E0,5E0,3E0,3E0,3E0,5E0,8E0,6E0,5E0,3E0,3E0,2.2E1,3E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[5.129172E-3,-1.3390855E-1,3.2403773E-1,-2.2909184E-1,-6.4268745E-2,1.9948438E-1,4.8802742E-1,-2.7393922E-1,-7.6368E-2,1.1871911E-2,-7.929652E-2,3.7061435E-1,1.4743653E-1,6.2644255E-1,2.2577208E-1,-1.0811839E-2,-1.9915852E-1,4.358832E-2,-1.1701678E-2,-1.1342498E-1,7.236948E-2,1.585175E-2,1.1031517E-3,8.100389E-3,2.2215213E-1,4.6258256E-1,3.0093342E-2,-4.869998E-3,1.1020287E-2,-1.6590615E-3,-2.2556409E-1,-3.481776E-3,1.3028255E-1,2.7164482E-3,-1.2590718E-1,1.6042326E-1,-1.2665571E-1,-5.5578094E-2,4.7180317E-3,6.5555625E-2,2.6585332E-1,1.6595649E-2,5.1979492E-3,-7.606355E-3,-3.4578354E-3,1.6192707E-3,6.4441944E-3,-1.959208E-1,-7.861717E-2,4.735815E-2,2.30538E-1,8.5667416E-4,-7.991277E-3,1.2507202E-3,-9.5847666E-2,-2.2531112E-4,3.918104E-3,3.1458148E-1,4.198016E-3,-2.1415064E-1,-1.5243915E-3,-8.6175255E-2,1.1431322E-3,3.4915917E-3,-8.238748E-4,9.20539E-3,4.152011E-3,-4.320098E-3,-7.703501E-4,1.2875673E-2,2.0051518E-1,-2.4122249E-1,-1.24180496E-1,-1.03497736E-1,-2.8911445E-2,2.979266E-3,9.184668E-3,-5.4796026E-3,-2.8619492E-1,-4.6815043E-3,-1.971536E-3,-5.0967205E-3,-7.825718E-2,-6.4039215E-2,1.2908039E-3,-1.05351405E-2,-5.2533345E-3,-5.3874604E-2,-5.1077716E-3,-2.9157645E-3,-1.1481906E-3,-6.642816E-2,7.19804E-4,-7.912553E-2,1.3334914E-4,-7.229708E-4,-2.8861845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,-1,29,31,-1,33,35,-1,-1,37,39,41,-1,-1,-1,-1,43,-1,45,-1,47,49,51,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,63,65,-1,-1,-1,67,-1,-1,69,-1,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,75,77,79,81,83,-1,-1,-1,85,-1,-1,-1,87,89,-1,-1,-1,91,-1,-1,-1,93,-1,95,-1,-1,-1],"loss_changes":[1.241434E1,1.2778432E0,1.6388102E0,5.407195E-1,7.711983E-1,3.90705E-1,1.16537E0,2.2495079E-1,7.047855E-1,0E0,5.827584E-1,4.3392146E-1,4.1462374E-1,7.3002815E-1,6.672685E-1,0E0,1.0920942E-1,2.114366E-1,0E0,2.3532033E-1,3.883419E-1,0E0,0E0,1.4041209E-1,1.6050172E-1,1.8796921E-1,0E0,0E0,0E0,0E0,7.6025724E-3,0E0,4.5846254E-2,0E0,2.693385E-1,1.0799509E-1,1.3707642E-1,4.555505E-2,0E0,2.9378166E-2,9.096336E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.1939936E-2,4.8778474E-2,3.3985294E-2,1.3782471E-2,0E0,0E0,0E0,1.8859163E-2,0E0,0E0,5.6205153E-2,0E0,4.3114662E-2,0E0,4.660529E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6848327E-2,2.1517515E-2,3.1321049E-3,4.020807E-2,3.4083236E-2,0E0,0E0,0E0,7.2383285E-3,0E0,0E0,0E0,4.635735E-2,3.545981E-3,0E0,0E0,0E0,2.3497336E-2,0E0,0E0,0E0,1.824908E-2,0E0,8.137763E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,19,19,20,20,23,23,24,24,25,25,30,30,32,32,34,34,35,35,36,36,37,37,39,39,40,40,47,47,48,48,49,49,50,50,54,54,57,57,59,59,61,61,70,70,71,71,72,72,73,73,74,74,78,78,82,82,83,83,87,87,91,91,93,93],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,-1,30,32,-1,34,36,-1,-1,38,40,42,-1,-1,-1,-1,44,-1,46,-1,48,50,52,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,64,66,-1,-1,-1,68,-1,-1,70,-1,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,76,78,80,82,84,-1,-1,-1,86,-1,-1,-1,88,90,-1,-1,-1,92,-1,-1,-1,94,-1,96,-1,-1,-1],"split_conditions":[1.0259E4,4.258E3,6.2316E4,1.83E2,2.5961E4,9.099E3,2.5297756E0,3.89319E5,1.542305E6,1.1871911E-2,2.466E3,2.6504964E-1,2.7732336E3,1.0778861E4,2.0432923E0,-1.0811839E-2,1.009E3,1.1564E5,-1.1701678E-2,3.453E3,6.344E3,1.585175E-2,1.1031517E-3,2.2105154E-1,3.51825E5,3.3046817E10,3.0093342E-2,-4.869998E-3,1.1020287E-2,-1.6590615E-3,1.0128E5,-3.481776E-3,5.876475E7,2.7164482E-3,1.20944E5,2.7552E4,1.11301E5,2.01E2,4.7180317E-3,1.3334E4,8.501854E10,1.6595649E-2,5.1979492E-3,-7.606355E-3,-3.4578354E-3,1.6192707E-3,6.4441944E-3,2.9105232E7,4.23859E5,1.73E2,2.699E3,8.5667416E-4,-7.991277E-3,1.2507202E-3,9.854431E8,-2.2531112E-4,3.918104E-3,1.5369331E10,4.198016E-3,1.7387058E3,-1.5243915E-3,3.409479E-1,1.1431322E-3,3.4915917E-3,-8.238748E-4,9.20539E-3,4.152011E-3,-4.320098E-3,-7.703501E-4,1.2875673E-2,2.4717906E9,9.86712E2,1.1750605E8,5.737332E9,7.164074E8,2.979266E-3,9.184668E-3,-5.4796026E-3,1.05E3,-4.6815043E-3,-1.971536E-3,-5.0967205E-3,2.810908E10,5.888E3,1.2908039E-3,-1.05351405E-2,-5.2533345E-3,8.405997E8,-5.1077716E-3,-2.9157645E-3,-1.1481906E-3,1.412488E6,7.19804E-4,8.67E2,1.3334914E-4,-7.229708E-4,-2.8861845E-3],"split_indices":[3,3,3,47,12,7,66,9,2,0,0,63,5,5,64,0,20,27,0,7,0,0,0,63,12,21,0,0,0,0,22,0,16,0,12,18,8,17,0,7,13,0,0,0,0,0,0,18,12,17,0,0,0,0,16,0,0,10,0,5,0,63,0,0,0,0,0,0,0,0,27,5,16,10,16,0,0,0,17,0,0,0,10,3,0,0,0,16,0,0,0,6,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,1.94E2,8.4E1,8.1E1,1.13E2,4.9E1,3.5E1,6.2E1,1.9E1,3E0,1.1E2,1E1,3.9E1,2.2E1,1.3E1,3.2E1,3E1,1.4E1,5E0,9E1,2E1,7E0,3E0,1.4E1,2.5E1,1.6E1,6E0,3E0,1E1,5E0,2.5E1,5E0,9E0,5E0,8.5E1,1.4E1,6E0,1E1,4E0,6E0,1.9E1,1.3E1,3E0,2.2E1,3E0,5E0,4E0,3.3E1,5.2E1,6E0,8E0,3E0,3E0,3E0,7E0,3E0,3E0,1.3E1,6E0,2.9E1,4E0,4.9E1,3E0,3E0,3E0,4E0,4E0,4E0,3E0,6E0,7E0,2.1E1,8E0,3.7E1,1.2E1,4E0,3E0,1E1,1.1E1,5E0,3E0,1E1,2.7E1,8E0,4E0,7E0,4E0,2.2E1,5E0,3E0,5E0,1.9E1,3E0,1.6E1,3E0,3E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[2.3426262E-3,-1.2277652E-1,2.3430999E-1,-2.4306393E-1,-4.620737E-2,1.3974068E-1,3.4289777E-1,-2.9461682E-1,-1.617973E-1,1.6030582E-2,-7.144231E-2,1.6644718E-2,1.0675759E-1,3.855117E-1,-5.7817576E-3,-9.857346E-3,-4.8195883E-3,-1.0647838E-1,-1.5480343E-2,-9.871442E-2,7.7141896E-2,1.4354175E-1,-1.3970825E-1,2.5825295E-1,4.419328E-1,-2.4579424E-2,-1.7492181E-1,-1.390952E-1,1.1839416E-2,8.060054E-3,4.5709116E-3,1.7336008E-1,8.319401E-3,-6.294119E-4,-7.240448E-3,1.690554E-1,1.0816851E-2,1.9959789E-2,1.1636482E-2,-7.197361E-2,1.3747708E-3,-6.6209384E-3,-2.5236662E-3,-6.4956583E-3,-1.0534963E-1,7.7430405E-2,-6.567002E-2,-4.662031E-3,6.3875474E-2,2.083664E-1,6.150699E-2,-2.566992E-3,2.1130175E-3,2.32446E-3,6.6129356E-3,-3.123196E-3,-7.295158E-4,-8.177319E-3,-8.8680215E-2,1.0472827E-1,-5.470941E-4,-5.24617E-3,-1.5124293E-2,3.807601E-3,-4.569512E-4,1.2396887E-1,3.1985652E-1,-1.5396605E-3,4.582794E-3,-1.0976005E-1,-1.0248726E-2,5.0672158E-3,1.1131235E-3,-2.372811E-3,8.516211E-4,1.9271715E-3,4.689522E-3,3.6598968E-3,1.2326844E-2,-7.7260494E-2,-1.3946205E-1,2.1033895E-3,-5.198107E-2,-2.933002E-3,-7.405598E-4,-2.1785318E-3,-4.854776E-3,-2.2488611E-3,-6.792679E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,-1,-1,25,-1,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,-1,-1,53,-1,-1,-1,55,-1,-1,-1,-1,57,59,61,-1,63,65,67,-1,-1,-1,-1,-1,-1,-1,69,71,-1,-1,73,-1,-1,75,77,-1,-1,79,81,-1,-1,-1,-1,-1,-1,-1,-1,83,85,-1,87,-1,-1,-1,-1,-1,-1],"loss_changes":[7.952199E0,1.6282027E0,9.379144E-1,2.414093E-1,1.5462556E0,6.0444903E-1,1.0489707E0,2.3157835E-2,4.6074593E-1,0E0,4.3714857E-1,0E0,4.73356E-1,1.8025255E-1,0E0,0E0,0E0,1.4144209E-1,0E0,4.034294E-1,2.1260281E-1,1.759702E-1,6.728503E-2,4.7269344E-2,1.6269493E-1,4.4532225E-2,2.5449783E-2,1.1839211E-1,1.3218288E-1,0E0,1.24435805E-1,1.2814438E-1,5.067074E-2,0E0,0E0,1.6877651E-2,0E0,0E0,0E0,7.684078E-3,0E0,0E0,0E0,0E0,8.900857E-2,3.786707E-2,5.655853E-2,0E0,4.416064E-2,2.1440387E-1,9.522642E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.831464E-2,3.521499E-2,0E0,0E0,2.3876756E-2,0E0,0E0,1.494354E-2,9.954214E-2,0E0,0E0,2.1080077E-2,3.4943763E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1436753E-2,2.8386712E-3,0E0,2.3911577E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,35,35,39,39,44,44,45,45,46,46,48,48,49,49,50,50,58,58,59,59,62,62,65,65,66,66,69,69,70,70,79,79,80,80,82,82],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,-1,-1,26,-1,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,-1,-1,54,-1,-1,-1,56,-1,-1,-1,-1,58,60,62,-1,64,66,68,-1,-1,-1,-1,-1,-1,-1,70,72,-1,-1,74,-1,-1,76,78,-1,-1,80,82,-1,-1,-1,-1,-1,-1,-1,-1,84,86,-1,88,-1,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,4.258E3,7.762619E6,8.64E2,3.453E3,3.28451E5,1.0353053E8,7.03651E5,5.1041E4,1.6030582E-2,2.346E3,1.6644718E-2,7.192367E10,7.4399194E8,-5.7817576E-3,-9.857346E-3,-4.8195883E-3,6.3645E4,-1.5480343E-2,6.490568E8,2.66E3,1.05E3,1.11393E5,4.07729E5,1.153969E6,2.09153E5,1.835E3,3.75429E5,2.1886E4,8.060054E-3,1.9669E4,1.7867039E10,1.1662E4,-6.294119E-4,-7.240448E-3,7.7433E4,1.0816851E-2,1.9959789E-2,1.1636482E-2,2.905E3,1.3747708E-3,-6.6209384E-3,-2.5236662E-3,-6.4956583E-3,6.388E3,4.886616E9,1.08298E5,-4.662031E-3,3.3831064E7,1.83E2,2.2052716E9,-2.566992E-3,2.1130175E-3,2.32446E-3,6.6129356E-3,-3.123196E-3,-7.295158E-4,-8.177319E-3,1.619E3,6E0,-5.470941E-4,-5.24617E-3,5.8E1,3.807601E-3,-4.569512E-4,1.0773E4,7.62802E5,-1.5396605E-3,4.582794E-3,7.0243E4,1.719E3,5.0672158E-3,1.1131235E-3,-2.372811E-3,8.516211E-4,1.9271715E-3,4.689522E-3,3.6598968E-3,1.2326844E-2,1.77451E5,1.501E4,2.1033895E-3,2.654028E6,-2.933002E-3,-7.405598E-4,-2.1785318E-3,-4.854776E-3,-2.2488611E-3,-6.792679E-4],"split_indices":[12,3,6,0,7,6,2,2,20,0,0,0,10,16,0,0,0,12,0,16,0,17,8,18,12,18,0,6,28,0,7,10,3,0,0,8,0,0,0,3,0,0,0,0,7,10,12,0,16,47,11,0,0,0,0,0,0,0,0,19,0,0,4,0,0,3,6,0,0,8,0,0,0,0,0,0,0,0,0,12,18,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.77E2,9.5E1,6.8E1,1.09E2,5.2E1,4.3E1,4E1,2.8E1,4E0,1.05E2,3E0,4.9E1,4E1,3E0,3.6E1,4E0,2.5E1,3E0,8.9E1,1.6E1,4.3E1,6E0,1.4E1,2.6E1,1.2E1,1.3E1,6.5E1,2.4E1,4E0,1.2E1,3.5E1,8E0,3E0,3E0,8E0,6E0,6E0,2E1,7E0,5E0,9E0,4E0,2.1E1,4.4E1,1.3E1,1.1E1,3E0,9E0,2.6E1,9E0,3E0,5E0,3E0,5E0,4E0,3E0,3E0,4.1E1,1E1,3E0,3E0,8E0,5E0,4E0,1.6E1,1E1,4E0,5E0,3.2E1,9E0,5E0,5E0,3E0,5E0,5E0,1.1E1,3E0,7E0,1.7E1,1.5E1,3E0,6E0,1.3E1,4E0,3E0,1.2E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.5027356E-2,-1.3221163E-1,2.693333E-1,-1.663246E-1,5.6476586E-2,1.8368316E-1,4.5362943E-1,-2.420585E-1,-7.986575E-2,-7.891603E-2,1.9750606E-1,1.3531767E-1,4.3959016E-1,3.1586E-1,7.251362E-1,1.2172112E-3,-2.5345373E-1,-9.374597E-2,1.55661795E-2,-2.0859693E-1,3.0715335E-2,3.0774903E-1,4.3298934E-2,-8.086752E-2,1.7355417E-1,1.693356E-2,7.4229054E-3,4.6074706E-1,2.2367522E-1,2.8350612E-2,8.739879E-3,-2.7253357E-1,-1.481279E-1,-1.1621946E-1,-6.787364E-3,1.3208949E-3,-2.6564725E-4,-8.53877E-3,-3.2116368E-3,-2.768479E-3,3.3286947E-3,4.204836E-3,1.2485925E-2,-1.4565194E-3,3.8955775E-3,-1.6060583E-1,2.5394324E-3,1.9994695E-1,-6.4295577E-3,1.9140385E-2,5.7364553E-3,2.908546E-3,2.4748996E-1,-2.812046E-1,-3.668712E-3,-1.9008197E-3,-5.377262E-3,5.2242086E-4,-1.2501898E-1,1.9836575E-3,-3.10477E-2,-6.9525586E-3,-2.0944774E-3,-3.0684002E-2,2.382051E-1,4.175834E-3,9.187179E-3,-2.8865755E-1,-3.3701512E-3,-1.4964698E-1,-8.084893E-2,4.963948E-4,-5.0401226E-2,-2.2642317E-3,8.5491745E-4,1.8562934E-1,3.9314222E-1,-2.9903436E-1,-4.8481594E-3,-1.2789439E-1,-7.3545524E-3,-9.056743E-2,-7.0051826E-4,-7.609055E-4,-1.9870563E-3,1.0848982E-1,3.1801096E-1,3.1373908E-3,1.6285546E-2,-1.1212378E-2,-8.455773E-3,-2.4044986E-3,-4.4246344E-3,-3.1967496E-3,-1.271649E-3,2.6535766E-2,1.5042959E-1,1.1324207E-2,5.2298596E-3,-2.5359247E-4,1.4502996E-3,5.380578E-3,2.0164112E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,63,-1,-1,-1,-1,65,67,-1,-1,-1,-1,69,-1,71,-1,-1,73,75,-1,-1,77,-1,79,81,-1,83,-1,-1,85,87,89,-1,91,-1,93,-1,-1,-1,95,97,-1,-1,-1,-1,-1,-1,-1,-1,99,101,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0519858E1,1.1551285E0,1.5425301E0,9.6939516E-1,5.502282E-1,8.3166456E-1,9.6792936E-1,2.6472187E-1,9.683162E-2,2.1966353E-1,2.2467834E-1,5.2653706E-1,7.748461E-2,2.04139E-1,4.667778E-1,0E0,1.1436939E-1,1.2232971E-1,6.405211E-3,1.6704023E-2,8.526109E-2,6.289208E-2,5.3651776E-2,1.400595E-1,5.621147E-1,0E0,0E0,1.9663405E-1,2.8435647E-2,0E0,0E0,5.594158E-2,1.373601E-2,6.030357E-2,2.514936E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2986054E-2,0E0,4.4848478E-1,0E0,0E0,0E0,0E0,2.103132E-2,4.870701E-2,0E0,0E0,0E0,0E0,3.9028823E-2,0E0,1.0669975E-2,0E0,0E0,2.0029426E-2,2.7729678E-1,0E0,0E0,3.660965E-2,0E0,2.0508826E-2,8.9393705E-3,0E0,9.664614E-4,0E0,0E0,3.1270218E-1,2.841891E-1,1.6565323E-3,0E0,2.4505258E-3,0E0,3.2207966E-3,0E0,0E0,0E0,7.5062364E-2,5.914092E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.0911025E-3,1.1554152E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,31,31,32,32,33,33,34,34,45,45,47,47,52,52,53,53,58,58,60,60,63,63,64,64,67,67,69,69,70,70,72,72,75,75,76,76,77,77,79,79,81,81,85,85,86,86,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,64,-1,-1,-1,-1,66,68,-1,-1,-1,-1,70,-1,72,-1,-1,74,76,-1,-1,78,-1,80,82,-1,84,-1,-1,86,88,90,-1,92,-1,94,-1,-1,-1,96,98,-1,-1,-1,-1,-1,-1,-1,-1,100,102,-1,-1,-1,-1,-1,-1],"split_conditions":[2.72074E5,1.83E2,6.45689E5,8.99622E5,3.533E3,1.9367423E9,1.2738551E4,1.5E1,7.164074E8,1.85948E5,7.899E3,6.0383285E-5,9.3E1,2.7804902E-1,1.5860209E9,1.2172112E-3,2.252272E8,2.863047E6,1.4607E4,2.7552E4,2.9962454E-3,8.388608E7,4.873E3,1E0,4.04E2,1.693356E-2,7.4229054E-3,1.0353053E8,9.458E3,2.8350612E-2,8.739879E-3,6.591E3,1.41048E5,2.8156E5,3.3831064E7,1.3208949E-3,-2.6564725E-4,-8.53877E-3,-3.2116368E-3,-2.768479E-3,3.3286947E-3,4.204836E-3,1.2485925E-2,-1.4565194E-3,3.8955775E-3,2.1035875E8,2.5394324E-3,1.138E3,-6.4295577E-3,1.9140385E-2,5.7364553E-3,2.908546E-3,1.7994546E0,7.3831117E9,-3.668712E-3,-1.9008197E-3,-5.377262E-3,5.2242086E-4,7.76055E5,1.9836575E-3,1.958E3,-6.9525586E-3,-2.0944774E-3,2.4552E6,1.83E2,4.175834E-3,9.187179E-3,8.487963E7,-3.3701512E-3,2.147E3,9.7237E4,4.963948E-4,5.275E3,-2.2642317E-3,8.5491745E-4,1.346289E7,1.0022458E-2,6.7452E4,-4.8481594E-3,1.089441E6,-7.3545524E-3,3.3587363E8,-7.0051826E-4,-7.609055E-4,-1.9870563E-3,3.69126E5,7.144995E10,3.1373908E-3,1.6285546E-2,-1.1212378E-2,-8.455773E-3,-2.4044986E-3,-4.4246344E-3,-3.1967496E-3,-1.271649E-3,1.4E2,4.6908E4,1.1324207E-2,5.2298596E-3,-2.5359247E-4,1.4502996E-3,5.380578E-3,2.0164112E-3],"split_indices":[12,47,28,9,3,16,5,28,16,18,20,63,4,63,16,0,16,2,7,18,63,16,7,31,4,0,0,2,7,0,0,3,6,16,16,0,0,0,0,0,0,0,0,0,0,16,0,0,0,0,0,0,64,10,0,0,0,0,6,0,20,0,0,2,47,0,0,21,0,19,8,0,3,0,0,2,63,18,0,2,0,16,0,0,0,12,26,0,0,0,0,0,0,0,0,19,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.77E2,1.02E2,1.5E2,2.7E1,7.1E1,3.1E1,7.9E1,7.1E1,1.4E1,1.3E1,6.1E1,1E1,2.2E1,9E0,3E0,7.6E1,6.2E1,9E0,6E0,8E0,7E0,6E0,9E0,5.2E1,6E0,4E0,7E0,1.5E1,6E0,3E0,6.3E1,1.3E1,4.9E1,1.3E1,4E0,5E0,3E0,3E0,3E0,5E0,3E0,4E0,3E0,3E0,6E0,3E0,4.9E1,3E0,4E0,3E0,3E0,1.2E1,5.9E1,4E0,3E0,1E1,3E0,4.6E1,3E0,1E1,3E0,3E0,7E0,4.2E1,4E0,8E0,5.6E1,3E0,2.8E1,1.8E1,3E0,7E0,4E0,3E0,3.3E1,9E0,5.1E1,5E0,2.4E1,4E0,1.5E1,3E0,3E0,4E0,2.2E1,1.1E1,3E0,6E0,1.8E1,3.3E1,5E0,1.9E1,1.2E1,3E0,8E0,1.4E1,8E0,3E0,3E0,5E0,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.5857957E-2,-1.0993813E-1,2.636322E-1,-2.2424996E-1,-4.262552E-2,1.1764458E-1,4.160262E-1,-2.578948E-1,-5.053163E-3,1.5334828E-1,-8.215143E-2,5.302293E-2,3.1737262E-1,2.7477242E-2,3.6993226E-1,-2.8307426E-1,-1.3540213E-1,-5.1709646E-3,3.4379438E-3,-2.0980554E-2,4.610463E-1,-6.472309E-2,-1.3784297E-2,-2.273006E-2,1.10784754E-1,1.6340446E-2,-2.8514755E-3,1.8563391E-1,4.5052108E-1,-1.14257885E-2,-2.453573E-1,5.907183E-4,-5.902277E-3,7.214759E-3,-1.0093825E-1,1.7407227E-2,7.922168E-3,-9.683977E-2,3.5351567E-2,-5.5841833E-2,2.612175E-3,1.3911194E-1,-7.944531E-4,2.9257444E-1,-6.705966E-3,5.6739163E-1,3.4069428E-1,-8.128256E-3,-3.9068316E-3,-1.0043556E-2,4.0906263E-3,-1.0973226E-1,2.5206385E-3,-7.490633E-3,8.5958354E-2,-7.773805E-2,1.1260369E-4,1.8887881E-3,4.82366E-3,1.179763E-2,4.6747695E-3,2.0212177E-2,8.339628E-3,7.2891577E-3,1.354737E-2,3.937797E-3,-2.4276883E-3,-1.1710345E-1,4.9605785E-4,-6.0246453E-2,2.6293285E-3,4.614342E-3,3.9342716E-2,-1.2422753E-3,-3.0448572E-3,-1.2826923E-1,-1.4535049E-2,-5.2931096E-4,-7.6669864E-2,2.7010306E-3,-8.434862E-4,-1.4882192E-1,-7.663981E-2,1.6512915E-3,-2.9998505E-3,-3.0617185E-3,-1.2571469E-3,-1.4959795E-3,-1.5789476E-1,-4.726224E-2,-3.6952407E-3,-9.923302E-2,-1.7172451E-1,-1.2517226E-4,-5.9885442E-2,-3.726417E-3,-1.4642893E-3,-4.1902335E-3,-1.9449969E-1,-7.3901727E-4,-2.5969024E-3,-3.1754842E-3,-6.6660955E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,35,37,-1,39,41,-1,-1,43,45,-1,47,-1,-1,-1,49,-1,-1,51,53,55,-1,57,-1,59,-1,61,63,-1,-1,-1,65,67,-1,69,71,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,75,-1,77,-1,-1,79,-1,-1,81,83,-1,85,-1,-1,87,89,-1,-1,-1,-1,-1,91,93,-1,95,97,-1,99,-1,-1,-1,101,-1,-1,-1,-1],"loss_changes":[9.535537E0,1.5524507E0,2.2425637E0,5.5181026E-1,1.0118372E0,6.8013686E-1,6.927452E-1,1.621499E-1,2.0860292E-1,1.189767E0,6.236948E-1,1.8629463E-1,1.0491043E0,0E0,6.144304E-1,5.6607246E-2,9.922993E-2,0E0,0E0,3.1661043E-1,4.0085316E-3,3.3768627E-1,0E0,6.9885835E-2,9.553006E-2,0E0,0E0,7.350457E-1,2.1742964E-1,0E0,3.083706E-3,0E0,0E0,0E0,2.6731083E-1,0E0,0E0,1.8698168E-1,5.701464E-2,1.9761775E-2,0E0,8.488089E-3,0E0,8.839822E-2,0E0,9.858656E-2,7.8728676E-2,0E0,0E0,0E0,9.384059E-2,7.0066035E-2,0E0,7.548852E-2,2.646529E-2,3.607072E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.8303576E-2,0E0,5.933132E-3,0E0,0E0,2.5569726E-2,0E0,0E0,5.5445194E-2,4.6209514E-2,0E0,1.1423677E-3,0E0,0E0,3.3519268E-2,1.6994499E-2,0E0,0E0,0E0,0E0,0E0,1.5410304E-2,6.8847984E-3,0E0,3.6194623E-3,4.0152073E-3,0E0,5.8253333E-3,0E0,0E0,0E0,1.2962818E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,20,20,21,21,23,23,24,24,27,27,28,28,30,30,34,34,37,37,38,38,39,39,41,41,43,43,45,45,46,46,50,50,51,51,53,53,54,54,55,55,67,67,69,69,72,72,75,75,76,76,78,78,81,81,82,82,88,88,89,89,91,91,92,92,94,94,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,36,38,-1,40,42,-1,-1,44,46,-1,48,-1,-1,-1,50,-1,-1,52,54,56,-1,58,-1,60,-1,62,64,-1,-1,-1,66,68,-1,70,72,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,76,-1,78,-1,-1,80,-1,-1,82,84,-1,86,-1,-1,88,90,-1,-1,-1,-1,-1,92,94,-1,96,98,-1,100,-1,-1,-1,102,-1,-1,-1,-1],"split_conditions":[9.896E3,6.78404E5,1.4066751E7,1.83E2,7.284E3,1.83E2,4.06635E5,1.7013242E8,3.15859E5,1.6942586E7,1.24389E5,3.47989E5,1.5369331E10,2.7477242E-2,6.117756E9,2.7E2,1E0,-5.1709646E-3,3.4379438E-3,3.009E4,4.41E3,1.9557E4,-1.3784297E-2,1.5102771E9,2.9188708E10,1.6340446E-2,-2.8514755E-3,4.534996E9,3.954342E-1,-1.14257885E-2,2.23334E5,5.907183E-4,-5.902277E-3,7.214759E-3,1.5E1,1.7407227E-2,7.922168E-3,1.83E2,6.93652E5,1.105E3,2.612175E-3,3.88957E5,-7.944531E-4,2.5617332E10,-6.705966E-3,1.312E3,7.8667163E3,-8.128256E-3,-3.9068316E-3,-1.0043556E-2,3.342E3,2.113977E6,2.5206385E-3,3.0831E4,2.748732E6,1.8E1,1.1260369E-4,1.8887881E-3,4.82366E-3,1.179763E-2,4.6747695E-3,2.0212177E-2,8.339628E-3,7.2891577E-3,1.354737E-2,3.937797E-3,-2.4276883E-3,5.581883E7,4.9605785E-4,8.5101E4,2.6293285E-3,4.614342E-3,2.0397697E10,-1.2422753E-3,-3.0448572E-3,1.9313741E3,9.8E2,-5.2931096E-4,3.5877696E7,2.7010306E-3,-8.434862E-4,1.1728E4,2.59E2,1.6512915E-3,-2.9998505E-3,-3.0617185E-3,-1.2571469E-3,-1.4959795E-3,5.7E1,9.902779E8,-3.6952407E-3,1.05E2,1.1960812E8,-1.2517226E-4,2.3595012E3,-3.726417E-3,-1.4642893E-3,-4.1902335E-3,1.0471593E3,-7.3901727E-4,-2.5969024E-3,-3.1754842E-3,-6.6660955E-3],"split_indices":[3,9,2,47,7,47,12,16,21,18,8,12,10,0,27,0,19,0,0,18,7,7,0,16,10,0,0,27,63,0,6,0,0,0,17,0,0,47,18,0,0,12,0,10,0,17,5,0,0,0,7,6,0,7,18,19,0,0,0,0,0,0,0,0,0,0,0,18,0,8,0,0,10,0,0,5,0,0,16,0,0,18,4,0,0,0,0,0,4,10,0,19,26,0,5,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[3.04E2,2.02E2,1.02E2,7.4E1,1.28E2,5.3E1,4.9E1,6.4E1,1E1,2.1E1,1.07E2,4.1E1,1.2E1,3E0,4.6E1,5.2E1,1.2E1,4E0,6E0,1.4E1,7E0,1.03E2,4E0,1.8E1,2.3E1,8E0,4E0,1.5E1,3.1E1,1.5E1,3.7E1,3E0,9E0,3E0,1.1E1,4E0,3E0,7.8E1,2.5E1,1.4E1,4E0,1.9E1,4E0,1.2E1,3E0,1.3E1,1.8E1,3.4E1,3E0,3E0,8E0,7.3E1,5E0,1.4E1,1.1E1,1E1,4E0,3E0,1.6E1,7E0,5E0,1E1,3E0,9E0,9E0,3E0,5E0,6.9E1,4E0,9E0,5E0,4E0,7E0,4E0,6E0,6.2E1,7E0,3E0,6E0,4E0,3E0,4.3E1,1.9E1,4E0,3E0,3E0,3E0,4E0,3.9E1,1.2E1,7E0,9E0,3E1,3E0,9E0,6E0,3E0,1.3E1,1.7E1,4E0,5E0,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.2786053E-2,-1.1436636E-1,2.6567736E-1,-1.6607302E-1,2.5635956E-2,2.1126758E-1,1.9614913E-2,6.856583E-2,-1.8222351E-1,2.8830716E-1,-2.6313536E-2,1.6741261E-1,4.7141013E-1,-3.3431307E-3,8.425882E-3,-2.4158064E-1,-1.3002938E-1,3.3042543E-3,1.3400299E-2,-7.9196826E-2,1.1402073E-1,1.4061984E-2,2.1136004E-1,1.6824806E-2,7.6667857E-3,-2.5930968E-1,3.9890385E-4,-1.0523995E-2,-1.1710419E-1,-1.2076619E-1,-1.8491218E-2,7.9865154E-4,4.4518104E-3,6.438267E-2,-5.7346597E-3,1.14514224E-1,2.4177305E-1,-3.327127E-1,-1.8464535E-1,-1.6529533E-1,-7.33244E-2,-4.907381E-3,-5.4175448E-2,-2.2278498E-3,6.2445267E-3,1.4496289E-1,-2.4245374E-2,1.7232718E-1,6.259373E-2,2.7559638E-1,1.3991769E-1,-5.431191E-3,-3.5301307E-1,-2.1897787E-1,-7.639059E-2,-1.8146412E-1,-2.0051056E-3,2.746772E-3,-9.623731E-2,-3.252869E-3,-1.8786675E-4,2.285138E-3,-2.6909456E-2,2.1335345E-3,5.7590995E-3,8.6068164E-4,-2.6367188E-3,2.6548617E-3,7.052446E-3,2.775607E-3,1.7975543E-4,2.2919172E-1,1.18687255E-2,2.1470198E-4,5.7690656E-3,-4.244798E-1,-8.743984E-3,-7.5997435E-3,-3.5448552E-3,1.9933438E-4,-3.8212896E-3,-6.4567844E-3,-4.0873056E-3,2.3018494E-3,-1.7708482E-3,-6.975162E-2,-4.5783496E-3,-1.3181763E-3,-3.3135878E-4,1.0228828E-3,2.5588053E-1,-1.52469305E-2,-7.080939E-3,-4.568989E-2,-2.815642E-3,1.1902798E-2,2.0732923E-1,-1.8231962E-3,-6.830874E-4,3.4714276E-3,7.3919976E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,25,27,-1,-1,29,31,33,35,-1,-1,37,-1,-1,39,41,43,-1,-1,45,-1,47,49,51,53,55,57,-1,59,-1,61,63,65,67,69,71,73,-1,75,77,79,81,-1,83,85,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,89,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,93,-1,-1,-1,-1,95,-1,-1,97,-1,-1,99,-1,-1,-1,-1],"loss_changes":[9.516329E0,1.4303508E0,1.7225718E0,5.5818677E-1,7.444507E-1,9.136236E-1,0E0,3.6030775E-1,3.8682985E-1,1.7113078E-1,3.504116E-1,5.092199E-1,1.9021988E-2,0E0,0E0,2.823813E-1,1.5005553E-1,0E0,0E0,8.356568E-2,2.4728537E-2,1.85576E-1,1.4609051E-1,0E0,0E0,2.5644064E-1,0E0,0E0,1.3767546E-1,3.3944845E-2,1.9251294E-2,0E0,0E0,1.1063969E-1,0E0,3.7240207E-2,1.1302304E-1,3.3646822E-2,9.848845E-2,4.365599E-2,6.824182E-2,0E0,1.7539283E-2,0E0,2.5523106E-2,9.512618E-3,2.5711466E-2,1.1344269E-2,1.299252E-2,5.377674E-2,6.68934E-2,0E0,2.4254084E-2,1.6860604E-2,3.2192208E-2,3.7078261E-3,0E0,4.3652665E-2,2.662623E-2,0E0,0E0,0E0,1.4449009E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.18318915E-1,0E0,0E0,0E0,6.45411E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.9575865E-3,0E0,0E0,0E0,0E0,3.946221E-2,0E0,0E0,1.7454233E-3,0E0,0E0,1.297313E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,19,19,20,20,21,21,22,22,25,25,28,28,29,29,30,30,33,33,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,55,55,57,57,58,58,62,62,71,71,75,75,85,85,90,90,93,93,96,96],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,26,28,-1,-1,30,32,34,36,-1,-1,38,-1,-1,40,42,44,-1,-1,46,-1,48,50,52,54,56,58,-1,60,-1,62,64,66,68,70,72,74,-1,76,78,80,82,-1,84,86,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,90,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,94,-1,-1,-1,-1,96,-1,-1,98,-1,-1,100,-1,-1,-1,-1],"split_conditions":[4.320722E6,4.3199274E8,1.1741724E4,1.0916E4,7.386E3,1.8458214E9,1.9614913E-2,3.6080938E-4,4.014E3,1.6942586E7,1.1408653E9,1.9073843E3,2.7804902E-1,-3.3431307E-3,8.425882E-3,4.231009E8,4.41E3,3.3042543E-3,1.3400299E-2,8.054554E8,4.5E2,9.57554E5,1.63192E5,1.6824806E-2,7.6667857E-3,1.65606E5,3.9890385E-4,-1.0523995E-2,1.358313E6,7.7643576E7,1.5778465E9,7.9865154E-4,4.4518104E-3,3.14E2,-5.7346597E-3,2.54E2,6.72444E5,1.5497559E-5,7.52952E5,7.529E3,8.6257E4,-4.907381E-3,7.078E3,-2.2278498E-3,8.183E3,3.04326E5,3.36E5,2.2213E4,7.0526E4,1.05055E5,3.3276E4,-5.431191E-3,6.6362006E2,4.0200136E7,2.4095E4,4.44688E5,-2.0051056E-3,3.7388639E9,4.4E2,-3.252869E-3,-1.8786675E-4,2.285138E-3,2.3E1,2.1335345E-3,5.7590995E-3,8.6068164E-4,-2.6367188E-3,2.6548617E-3,7.052446E-3,2.775607E-3,1.7975543E-4,6.3E1,1.18687255E-2,2.1470198E-4,5.7690656E-3,2.8895E4,-8.743984E-3,-7.5997435E-3,-3.5448552E-3,1.9933438E-4,-3.8212896E-3,-6.4567844E-3,-4.0873056E-3,2.3018494E-3,-1.7708482E-3,5.395E3,-4.5783496E-3,-1.3181763E-3,-3.3135878E-4,1.0228828E-3,4.8242E4,-1.52469305E-2,-7.080939E-3,1.42894E5,-2.815642E-3,1.1902798E-2,3.1E1,-1.8231962E-3,-6.830874E-4,3.4714276E-3,7.3919976E-3],"split_indices":[2,16,5,8,7,16,0,63,3,18,16,5,63,0,0,21,7,0,0,23,17,12,22,0,0,18,0,0,9,18,10,0,0,17,0,17,28,63,2,3,12,0,3,0,3,12,12,18,8,3,3,0,5,18,12,6,0,10,4,0,0,0,19,0,0,0,0,0,0,0,0,19,0,0,0,18,0,0,0,0,0,0,0,0,0,3,0,0,0,0,20,0,0,12,0,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.94E2,1.96E2,9.8E1,1.43E2,5.3E1,8.6E1,1.2E1,9E0,1.34E2,8E0,4.5E1,7.5E1,1.1E1,5E0,4E0,6.1E1,7.3E1,4E0,4E0,3.3E1,1.2E1,1.7E1,5.8E1,8E0,3E0,5.7E1,4E0,3E0,7E1,1.9E1,1.4E1,3E0,9E0,1.4E1,3E0,1.5E1,4.3E1,2.7E1,3E1,3.2E1,3.8E1,1.2E1,7E0,4E0,1E1,7E0,7E0,6E0,9E0,3.1E1,1.2E1,4E0,2.3E1,2.2E1,8E0,2.7E1,5E0,9E0,2.9E1,3E0,4E0,3E0,7E0,3E0,4E0,4E0,3E0,3E0,3E0,6E0,3E0,2.3E1,8E0,3E0,9E0,1E1,1.3E1,1.8E1,4E0,3E0,5E0,1.8E1,9E0,4E0,5E0,2E1,9E0,3E0,4E0,3E0,2E1,7E0,3E0,1E1,1E1,4E0,1.6E1,6E0,4E0,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[6.0809157E-3,-1.20095246E-1,2.6609626E-1,-1.6176912E-1,9.8286895E-3,1.9662452E-1,6.0984993E-1,-2.5201732E-1,-1.2419921E-1,-2.1686148E-2,4.4438997E-3,1.0900586E-1,3.026097E-1,1.2187664E-2,2.7458617E-2,-2.656189E-1,-2.9871764E-3,2.5536543E-2,-1.537729E-1,7.3491044E-2,-6.360399E-2,1.4074716E-1,-4.2280485E-3,2.3596686E-1,3.753305E-1,-9.943135E-3,-2.1175127E-1,-6.366382E-2,1.3172181E-1,-1.8779378E-1,-9.021397E-2,8.453531E-3,-9.510473E-3,-3.568843E-2,-6.13074E-3,1.1298821E-1,1.2109813E-2,2.5854707E-1,2.244186E-3,5.258269E-3,4.5683184E-1,-3.4370185E-3,-7.1882806E-3,-5.27377E-3,1.6983896E-2,3.52506E-4,6.1249784E-3,-1.7587931E-3,-1.9911101E-1,-1.0091955E-1,-2.5550503E-4,-5.6788187E-2,2.4335864E-3,-1.00618124E-1,7.1231546E-3,-1.2867033E-3,1.2684539E-1,9.700094E-3,3.7205922E-3,1.695746E-2,8.610429E-3,-1.1737056E-3,2.13042E-3,-2.2559807E-1,-4.1263746E-3,-1.1157685E-1,-1.2777161E-3,-4.129147E-4,-2.7860024E-3,-4.8515038E-3,-6.557813E-4,2.3144565E-3,-2.8858421E-2,1.3955288E-1,6.4855494E-4,-2.0284748E-1,-1.1854106E-2,-1.2123363E-1,-9.840273E-4,-2.0122712E-3,-6.958566E-4,6.5552494E-3,1.0561748E-1,-2.2732362E-1,-1.5019015E-3,-1.2981816E-1,-1.6142348E-3,-5.5591564E-4,5.1671756E-4,1.2921514E-2,1.3757361E-1,-8.144554E-3,-5.3977338E-3,-2.1728124E-3,-4.5687635E-3,1.2852383E-3,-5.573606E-4,1.2808809E-3,5.011318E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,25,-1,27,29,31,33,35,-1,37,39,-1,41,43,45,47,49,-1,51,53,-1,55,-1,57,-1,-1,59,-1,-1,-1,61,-1,-1,-1,63,65,-1,67,-1,69,71,-1,73,-1,-1,-1,-1,-1,-1,75,-1,77,-1,-1,-1,-1,-1,-1,79,81,-1,83,-1,85,-1,-1,87,-1,89,91,-1,93,-1,-1,-1,95,97,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.841471E0,1.0953057E0,2.2204442E0,4.8746395E-1,2.050481E-1,7.3777485E-1,5.767207E-1,7.0636034E-2,4.9103332E-1,1.6824883E-1,0E0,3.7706137E-1,9.074831E-2,0E0,0E0,2.784586E-2,0E0,1.8834944E-1,1.8199253E-1,2.0657378E-1,9.589995E-2,2.320134E-1,0E0,6.456041E-2,2.0179272E-1,0E0,1.6473532E-3,9.164947E-2,6.156732E-2,7.802987E-2,2.9146671E-2,0E0,4.4533137E-2,7.02103E-2,0E0,8.7735444E-2,0E0,8.142817E-2,0E0,0E0,4.822731E-3,0E0,0E0,0E0,2.082074E-2,0E0,0E0,0E0,7.251549E-2,1.51459575E-2,0E0,8.048568E-3,0E0,3.713339E-2,3.9368607E-2,0E0,4.599893E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.8637867E-2,0E0,1.5850425E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0382702E-2,4.8095226E-2,0E0,1.23194695E-1,0E0,6.9148242E-3,0E0,0E0,2.220423E-3,0E0,6.635305E-2,3.5482645E-3,0E0,4.7664642E-3,0E0,0E0,0E0,6.4074714E-3,2.3910016E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,27,27,28,28,29,29,30,30,32,32,33,33,35,35,37,37,40,40,44,44,48,48,49,49,51,51,53,53,54,54,56,56,63,63,65,65,72,72,73,73,75,75,77,77,80,80,82,82,83,83,85,85,89,89,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,26,-1,28,30,32,34,36,-1,38,40,-1,42,44,46,48,50,-1,52,54,-1,56,-1,58,-1,-1,60,-1,-1,-1,62,-1,-1,-1,64,66,-1,68,-1,70,72,-1,74,-1,-1,-1,-1,-1,-1,76,-1,78,-1,-1,-1,-1,-1,-1,80,82,-1,84,-1,86,-1,-1,88,-1,90,92,-1,94,-1,-1,-1,96,98,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,4.3199274E8,1.0218795E4,4.39E2,1.5102771E9,8.912913E8,2.77873E5,6.38524E5,3.2416E4,9E0,4.4438997E-3,1E0,1.8943199E9,1.2187664E-2,2.7458617E-2,1.20791E5,-2.9871764E-3,5.9625816E7,1.20944E5,8.2E1,1.275E3,1.044E3,-4.2280485E-3,4.9E2,4.3E1,-9.943135E-3,7.45575E2,6.5442E4,5.876475E7,2.341E3,2.1504E4,8.453531E-3,1E0,2.3912651E8,-6.13074E-3,1.116E3,1.2109813E-2,6.45689E5,2.244186E-3,5.258269E-3,5.105882E3,-3.4370185E-3,-7.1882806E-3,-5.27377E-3,6.52028E5,3.52506E-4,6.1249784E-3,-1.7587931E-3,2.81E2,3.409479E-1,-2.5550503E-4,7.869E3,2.4335864E-3,7.09E2,8.183E3,-1.2867033E-3,1E0,9.700094E-3,3.7205922E-3,1.695746E-2,8.610429E-3,-1.1737056E-3,2.13042E-3,1.13472E5,-4.1263746E-3,1.032E3,-1.2777161E-3,-4.129147E-4,-2.7860024E-3,-4.8515038E-3,-6.557813E-4,2.3144565E-3,1.1E2,5.4924E4,6.4855494E-4,6.7919E4,-1.1854106E-2,9.2921E4,-9.840273E-4,-2.0122712E-3,7.385252E8,6.5552494E-3,5.710497E6,4.014E3,-1.5019015E-3,4E1,-1.6142348E-3,-5.5591564E-4,5.1671756E-4,5.125581E6,1.6174E4,-8.144554E-3,-5.3977338E-3,-2.1728124E-3,-4.5687635E-3,1.2852383E-3,-5.573606E-4,1.2808809E-3,5.011318E-3],"split_indices":[2,16,5,0,16,16,3,2,12,19,0,44,16,0,0,18,0,23,12,17,0,17,0,17,4,0,5,6,16,22,7,0,19,21,0,28,0,28,0,0,5,0,0,0,21,0,0,0,4,63,0,7,0,17,3,0,36,0,0,0,0,0,0,22,0,17,0,0,0,0,0,0,4,8,0,22,0,28,0,0,16,0,2,3,0,17,0,0,0,2,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.98E2,2.01E2,9.7E1,1.52E2,4.9E1,8.2E1,1.5E1,4.3E1,1.09E2,4E1,9E0,4.6E1,3.6E1,9E0,6E0,3.9E1,4E0,1.8E1,9.1E1,1.2E1,2.8E1,4.1E1,5E0,2.1E1,1.5E1,1.9E1,2E1,1E1,8E0,5.8E1,3.3E1,3E0,9E0,2.4E1,4E0,3.8E1,3E0,1.8E1,3E0,5E0,1E1,3E0,1.7E1,4E0,6E0,3E0,5E0,5E0,5.3E1,2.9E1,4E0,6E0,3E0,9E0,1.5E1,3E0,3.5E1,1.3E1,5E0,6E0,4E0,3E0,3E0,3.7E1,1.6E1,2.4E1,5E0,3E0,3E0,5E0,4E0,5E0,1E1,3.1E1,4E0,3.4E1,3E0,2.1E1,3E0,4E0,6E0,9E0,2.2E1,2.9E1,5E0,1.8E1,3E0,3E0,3E0,6E0,1.6E1,1.8E1,1.1E1,4E0,1.4E1,3E0,3E0,3E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[1.46321235E-2,-1.1562929E-1,2.8331387E-1,-1.654131E-1,3.1056317E-2,1.0296707E-1,3.5075578E-1,6.9776356E-2,-1.8715784E-1,-5.6743294E-2,1.2112566E-1,1.8684976E-1,1.1729581E-2,3.7347576E-1,-3.4163054E-3,-5.5180755E-3,2.4195725E-1,-2.3892862E-1,-1.2932181E-1,-1.4310022E-2,-6.30268E-3,-3.0840575E-3,1.8015039E-1,6.740761E-3,2.513656E-3,1.0704454E-1,-7.313721E-2,3.2095516E-1,2.1393066E-2,3.0865897E-3,1.1718467E-2,-2.1889853E-1,-1.4534404E-2,-1.7366126E-1,-9.107408E-2,-3.6550716E-2,2.8761032E-3,1.2047491E-2,1.2387715E-1,1.429567E-3,4.6003247E-3,-3.9890874E-3,2.7793774E-4,4.7136036E-1,2.3945121E-1,-2.519763E-1,-1.1853016E-1,-2.0791888E-1,-1.0780306E-1,-7.149025E-2,-7.556941E-3,-5.8341578E-2,1.0227667E-3,1.4362872E-1,7.6536916E-4,3.0552462E-1,1.8160792E-2,6.0631774E-2,2.7012652E-1,-2.7551752E-1,-1.5839288E-1,-1.4981663E-1,9.1502466E-4,-7.954414E-3,-4.547147E-3,-4.012144E-3,-1.3828812E-3,-8.8390574E-2,1.423066E-3,-4.9802416E-4,-2.5892467E-3,9.758067E-2,5.6567807E-3,1.2862798E-2,4.839338E-3,8.2495715E-4,2.5904719E-3,2.8007177E-1,4.120361E-3,-9.135207E-3,-4.385497E-3,-1.7460933E-3,-6.0219606E-3,-5.8043427E-3,-1.6977713E-3,-6.0531233E-2,-4.2170244E-3,1.2347358E-3,4.0378436E-3,5.924669E-3,9.698227E-3,-3.3659157E-3,-4.2986732E-2,-5.0078567E-2,-2.9586756E-4,-1.8106762E-3,-7.122347E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,-1,-1,39,41,43,-1,-1,-1,45,-1,47,49,51,-1,-1,53,-1,-1,-1,-1,55,57,59,61,63,65,67,-1,69,-1,71,-1,73,-1,75,77,79,81,83,-1,-1,-1,-1,-1,85,-1,-1,-1,87,-1,-1,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,93,95,-1,-1,-1],"loss_changes":[1.0077434E1,1.4218352E0,1.1071992E0,7.546084E-1,4.022757E-1,2.0444599E-1,7.4551105E-1,5.813773E-1,3.6733198E-1,1.5061726E-1,3.3737615E-1,1.800105E-2,1.2107599E-1,7.8057194E-1,0E0,0E0,1.07764244E-1,1.9499636E-1,9.441638E-2,5.1681004E-2,0E0,0E0,1.721152E-1,0E0,0E0,9.380065E-3,3.429555E-2,5.858736E-1,0E0,0E0,0E0,1.8593073E-1,0E0,4.301393E-2,8.7337285E-2,2.8653052E-2,0E0,0E0,3.0196011E-2,0E0,0E0,0E0,0E0,1.2013006E-1,1.975503E-1,6.2107325E-2,8.701849E-2,1.2119651E-2,7.7486783E-3,7.0390224E-2,0E0,1.2017772E-2,0E0,3.5506487E-3,0E0,7.131714E-2,0E0,2.8000195E-3,1.0792255E-2,4.0125847E-3,2.5706112E-2,3.490618E-2,0E0,0E0,0E0,0E0,0E0,2.768904E-2,0E0,0E0,0E0,8.38808E-3,0E0,0E0,0E0,0E0,0E0,5.6626797E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0956839E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.3744127E-3,9.98415E-4,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,22,22,25,25,26,26,27,27,31,31,33,33,34,34,35,35,38,38,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,53,53,55,55,57,57,58,58,59,59,60,60,61,61,67,67,71,71,77,77,85,85,92,92,93,93],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,-1,-1,40,42,44,-1,-1,-1,46,-1,48,50,52,-1,-1,54,-1,-1,-1,-1,56,58,60,62,64,66,68,-1,70,-1,72,-1,74,-1,76,78,80,82,84,-1,-1,-1,-1,-1,86,-1,-1,-1,88,-1,-1,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,94,96,-1,-1,-1],"split_conditions":[4.320722E6,4.3199274E8,2.7797654E3,3.719E3,4.2524388E7,3.6060664E10,6.25E2,6.52028E5,4.014E3,8.89E2,1.064273E6,5.7138336E-1,1.83E2,1.4276579E4,-3.4163054E-3,-5.5180755E-3,5.598581E7,5.814E3,1.374403E6,3.43745E5,-6.30268E-3,-3.0840575E-3,5.202E3,6.740761E-3,2.513656E-3,5.660314E6,7.395E3,2.8125507E-1,2.1393066E-2,3.0865897E-3,1.1718467E-2,8.64E2,-1.4534404E-2,5.244E3,8.618333E9,2.943003E3,2.8761032E-3,1.2047491E-2,1.3798331E10,1.429567E-3,4.6003247E-3,-3.9890874E-3,2.7793774E-4,1.3925089E9,1.1751934E7,3.89319E5,1.64108E8,5.283369E6,1.6868966E8,1.83E2,-7.556941E-3,9.89E2,1.0227667E-3,2.124614E6,7.6536916E-4,1.5581422E10,1.8160792E-2,3.1561361E10,3.9743017E9,1.9006344E7,8.48986E5,2.63E2,9.1502466E-4,-7.954414E-3,-4.547147E-3,-4.012144E-3,-1.3828812E-3,4.4E2,1.423066E-3,-4.9802416E-4,-2.5892467E-3,7.5162E4,5.6567807E-3,1.2862798E-2,4.839338E-3,8.2495715E-4,2.5904719E-3,6.378634E8,4.120361E-3,-9.135207E-3,-4.385497E-3,-1.7460933E-3,-6.0219606E-3,-5.8043427E-3,-1.6977713E-3,1.783892E6,-4.2170244E-3,1.2347358E-3,4.0378436E-3,5.924669E-3,9.698227E-3,-3.3659157E-3,2.1114904E8,9.6E2,-2.9586756E-4,-1.8106762E-3,-7.122347E-4],"split_indices":[2,16,5,7,18,10,4,21,3,19,2,63,17,5,0,0,16,0,9,12,0,0,3,0,0,6,0,63,0,0,0,0,0,3,25,5,0,0,10,0,0,0,0,16,2,9,23,27,16,47,0,0,0,2,0,10,0,23,16,18,21,4,0,0,0,0,0,4,0,0,0,8,0,0,0,0,0,16,0,0,0,0,0,0,0,2,0,0,0,0,0,0,21,19,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.93E2,9.3E1,1.44E2,4.9E1,2.6E1,6.7E1,1.2E1,1.32E2,2.5E1,2.4E1,1.3E1,1.3E1,6.4E1,3E0,5E0,7E0,6.8E1,6.4E1,2E1,5E0,5E0,1.9E1,1E1,3E0,6E0,7E0,5.6E1,8E0,4E0,3E0,6.4E1,4E0,2.8E1,3.6E1,1.7E1,3E0,3E0,1.6E1,3E0,3E0,4E0,3E0,1.8E1,3.8E1,4.7E1,1.7E1,1.7E1,1.1E1,3.3E1,3E0,1.3E1,4E0,1.3E1,3E0,8E0,1E1,6E0,3.2E1,3.6E1,1.1E1,1.4E1,3E0,9E0,8E0,8E0,3E0,2.9E1,4E0,5E0,8E0,7E0,6E0,4E0,4E0,3E0,3E0,2.9E1,3E0,3.3E1,3E0,3E0,8E0,1E1,4E0,1.9E1,1E1,3E0,4E0,7E0,2.2E1,4E0,1.5E1,1.2E1,3E0,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[5.677875E-3,-1.1829775E-1,2.5357E-1,-2.1299475E-1,-5.841644E-2,2.049451E-1,5.768654E-1,-2.4025358E-1,1.0569352E-2,2.332647E-1,-8.4401846E-2,1.3734809E-1,3.5305572E-1,2.4149839E-2,1.1132188E-2,-2.7881047E-1,-1.7796886E-1,3.0296328E-3,-3.7789624E-3,-7.234731E-4,1.5885264E-2,-6.86048E-2,-1.2953449E-2,-1.193608E-1,1.7913717E-1,4.4992802E-1,2.2157356E-1,-1.1269863E-2,-2.2969182E-1,-6.104017E-3,-2.7810603E-3,-9.4711505E-2,7.364277E-2,-1.7231738E-3,-6.059964E-3,2.6476076E-1,1.1259342E-1,1.918824E-2,9.736508E-3,4.1994867E-3,8.440246E-3,-7.72602E-3,-3.6333983E-3,-7.9157196E-2,-1.3176225E-2,1.3160077E-1,-8.010071E-3,3.069845E-1,4.198289E-3,5.050538E-2,6.6531585E-3,-9.918079E-2,4.2452555E-2,6.3331064E-4,5.689668E-3,7.404926E-4,-1.4412875E-3,1.2578649E-2,2.1763253E-1,-2.479748E-2,1.0277374E-1,-1.1201526E-1,2.230988E-2,1.1426358E-1,-3.542312E-2,2.268227E-3,8.711535E-3,-4.238572E-3,1.6284072E-3,2.531826E-4,4.298381E-3,-1.3885365E-1,-8.1092946E-2,1.5224636E-3,-4.6681592E-4,5.468019E-3,9.6852635E-4,-1.9831192E-3,-1.2288946E-5,-6.0033747E-3,-1.1233328E-1,-2.5247281E-2,-1.06704034E-1,-4.557752E-3,-7.922917E-2,-5.1722474E-2,7.133872E-4,-1.1852171E-1,-6.7402486E-4,-9.368229E-2,-6.333435E-4,-8.697693E-4,-2.2425733E-3,-1.2915339E-1,-1.406873E-3,-1.4303921E-3,-3.4311651E-3,-1.7901917E-3,-4.5366823E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,-1,-1,-1,31,-1,33,35,37,39,-1,41,-1,-1,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,51,-1,53,55,57,-1,59,-1,61,63,-1,-1,-1,-1,-1,65,67,69,71,73,75,77,-1,-1,-1,-1,-1,-1,79,81,-1,-1,-1,-1,-1,-1,-1,83,85,87,-1,89,91,-1,93,-1,95,-1,-1,-1,97,-1,-1,-1,-1,-1],"loss_changes":[8.850246E0,1.0763094E0,1.390914E0,4.5350218E-1,9.2164314E-1,8.025141E-1,2.212174E-1,1.08879805E-1,1.0727673E-1,6.766199E-1,5.250028E-1,6.6503966E-1,2.314694E-1,0E0,0E0,5.6175947E-2,1.3517201E-2,0E0,0E0,0E0,0E0,4.006181E-1,0E0,3.0742332E-2,2.664709E-1,1.21453285E-1,1.3921499E-2,0E0,4.278183E-3,0E0,0E0,4.0804237E-1,8.1505515E-2,0E0,0E0,8.4747195E-2,1.706728E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1606106E-1,0E0,4.7065407E-2,1.0152257E-2,3.47445E-2,0E0,8.126767E-2,0E0,1.1944419E-1,7.674788E-2,0E0,0E0,0E0,0E0,0E0,5.893463E-2,7.917744E-2,3.398463E-2,4.5173526E-2,8.044789E-3,2.7654827E-2,6.3995207E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.327913E-2,4.5935124E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2997627E-2,1.6208487E-2,2.0829529E-2,0E0,1.0206975E-2,1.663981E-3,0E0,1.0263294E-2,0E0,2.257675E-3,0E0,0E0,0E0,6.8956017E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,21,21,23,23,24,24,25,25,26,26,28,28,31,31,32,32,35,35,36,36,43,43,45,45,46,46,47,47,49,49,51,51,52,52,58,58,59,59,60,60,61,61,62,62,63,63,64,64,71,71,72,72,80,80,81,81,82,82,84,84,85,85,87,87,89,89,93,93],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,-1,-1,-1,32,-1,34,36,38,40,-1,42,-1,-1,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,52,-1,54,56,58,-1,60,-1,62,64,-1,-1,-1,-1,-1,66,68,70,72,74,76,78,-1,-1,-1,-1,-1,-1,80,82,-1,-1,-1,-1,-1,-1,-1,84,86,88,-1,90,92,-1,94,-1,96,-1,-1,-1,98,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,8.03052E5,1.2738551E4,1.83E2,3.719E3,1.9367423E9,1.3887953E0,1.916746E6,3.185029E2,5.8111176E7,1.33097E5,9.577105E8,2.7804902E-1,2.4149839E-2,1.1132188E-2,1.2590113E7,4.982E3,3.0296328E-3,-3.7789624E-3,-7.234731E-4,1.5885264E-2,2.113977E6,-1.2953449E-2,9.35E2,1.78E2,9.068E3,5.669117E3,-1.1269863E-2,7.19175E5,-6.104017E-3,-2.7810603E-3,1.11386E5,1.83E2,-1.7231738E-3,-6.059964E-3,1E0,9.31278E5,1.918824E-2,9.736508E-3,4.1994867E-3,8.440246E-3,-7.72602E-3,-3.6333983E-3,1.83E2,-1.3176225E-2,4.618673E6,5.133E3,4.1E1,4.198289E-3,1.2E2,6.6531585E-3,2.0988E4,3.7388639E9,6.3331064E-4,5.689668E-3,7.404926E-4,-1.4412875E-3,1.2578649E-2,3.054E3,1.14E2,9.19E2,1.530129E6,1.29E2,1.835E3,2.0547E4,2.268227E-3,8.711535E-3,-4.238572E-3,1.6284072E-3,2.531826E-4,4.298381E-3,6.2935E4,1.207E3,1.5224636E-3,-4.6681592E-4,5.468019E-3,9.6852635E-4,-1.9831192E-3,-1.2288946E-5,-6.0033747E-3,3.75429E5,1.3818E4,7.396363E9,-4.557752E-3,4.48546E7,2.42E2,7.133872E-4,7.29E2,-6.7402486E-4,1E0,-6.333435E-4,-8.697693E-4,-2.2425733E-3,1.6035E4,-1.406873E-3,-1.4303921E-3,-3.4311651E-3,-1.7901917E-3,-4.5366823E-3],"split_indices":[3,9,5,47,7,16,59,18,5,16,8,23,63,0,0,11,3,0,0,0,0,6,0,0,17,7,5,0,2,0,0,28,17,0,0,34,12,0,0,0,0,0,0,47,0,2,3,17,0,4,0,7,10,0,0,0,0,0,0,19,0,9,4,0,8,0,0,0,0,0,0,18,0,0,0,0,0,0,0,0,6,7,11,0,18,4,0,17,0,42,0,0,0,18,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.91E2,9.5E1,7.3E1,1.18E2,8.4E1,1.1E1,6.5E1,8E0,9E0,1.09E2,5.9E1,2.5E1,5E0,6E0,3.8E1,2.7E1,5E0,3E0,5E0,4E0,1.05E2,4E0,8E0,5.1E1,1.3E1,1.2E1,1.3E1,2.5E1,2.3E1,4E0,8.9E1,1.6E1,5E0,3E0,2.1E1,3E1,5E0,8E0,5E0,7E0,2.2E1,3E0,8.6E1,3E0,9E0,7E0,1.5E1,6E0,1.9E1,1.1E1,7.4E1,1.2E1,3E0,6E0,4E0,3E0,6E0,9E0,8E0,1.1E1,6.7E1,7E0,6E0,6E0,3E0,6E0,3E0,5E0,3E0,8E0,3.4E1,3.3E1,4E0,3E0,3E0,3E0,3E0,3E0,1E1,2.4E1,1.1E1,2.2E1,1.1E1,1.3E1,7E0,4E0,1.9E1,3E0,1E1,3E0,4E0,3E0,1.6E1,3E0,3E0,7E0,3E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[2.652937E-2,-8.3240844E-2,2.2595395E-1,-1.2660164E-1,1.0696817E-1,1.23385824E-1,3.363112E-1,-1.8829909E-1,-3.253556E-2,-2.2233807E-2,3.6722493E-1,8.6517155E-2,1.3397212E-2,5.2734107E-1,2.7537018E-1,-8.686552E-3,-1.611393E-1,-6.330692E-2,9.068559E-2,-1.4106116E-1,9.830384E-2,1.848262E-2,1.9196787E-1,1.12075046E-1,-7.7835103E-3,6.3329196E-1,5.9100226E-3,-5.81281E-4,2.9964975E-1,-5.0427664E-2,-1.7211738E-1,-8.0221064E-2,-1.2875183E-2,3.8442553E-3,1.1989882E-3,-7.121587E-3,-4.0727567E-2,2.1204079E-2,5.194017E-3,9.2925485E-3,7.42807E-4,1.6426322E-1,-2.6400743E-2,1.0926402E-2,2.3874894E-2,3.32176E-1,1.4022005E-1,-4.8754603E-4,-2.2071057E-3,-5.9986617E-3,-1.05913356E-1,-8.899816E-2,-1.446113E-4,1.5880017E-2,-2.8562287E-3,1.4736996E-3,-3.767911E-3,-4.3421896E-4,1.6286591E-3,9.302655E-2,2.0732449E-1,3.4790006E-2,-4.6191835E-3,6.13536E-3,3.5962704E-1,6.3120304E-3,2.1720356E-3,-1.7695482E-3,-3.8881628E-3,-3.063954E-3,-1.4245192E-3,4.135206E-2,-7.389572E-4,2.8850002E-2,1.4557478E-1,2.3240182E-1,1.3727109E-3,-1.1219745E-2,3.431651E-3,1.3042038E-2,8.219392E-3,1.2796959E-4,2.20142E-3,3.8496163E-3,-1.5938506E-3,6.025635E-3,2.6769326E-3,1.6290645E-1,3.183365E-1,2.4302476E-4,-8.750406E-4,6.055522E-3,2.106444E-3,1.3041802E-2,4.8903082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,29,31,33,35,37,-1,39,41,-1,43,-1,-1,45,47,49,51,53,-1,-1,-1,55,57,-1,-1,-1,59,61,-1,-1,63,65,-1,-1,-1,67,69,-1,71,-1,-1,-1,-1,-1,73,75,77,-1,-1,79,-1,-1,-1,-1,-1,-1,81,-1,83,85,87,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,91,93,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4766183E0,1.5888367E0,1.1470132E0,8.99241E-1,1.2186897E0,5.685592E-1,4.3686342E-1,1.6474462E-1,2.4461703E-1,3.719461E-1,3.250413E-1,4.550552E-1,0E0,2.8433657E-1,2.9038405E-1,0E0,8.004475E-2,4.2470604E-2,1.5518203E-2,9.5573604E-2,5.9779704E-2,0E0,1.2401897E-1,3.5122305E-1,0E0,3.096819E-3,0E0,0E0,1.4529157E-1,4.081335E-3,4.248488E-2,2.4857879E-2,3.194745E-2,0E0,0E0,0E0,5.1581547E-2,7.82075E-3,0E0,0E0,0E0,8.807719E-2,1.0530972E-1,0E0,0E0,4.804206E-2,1.9282773E-2,0E0,0E0,0E0,3.9702505E-3,5.2016377E-3,0E0,1.1831112E-2,0E0,0E0,0E0,0E0,0E0,4.638563E-2,7.68131E-2,3.4240134E-2,0E0,0E0,1.0523081E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.802329E-3,0E0,6.28115E-2,5.2133054E-3,5.5637956E-2,0E0,2.3028147E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.718375E-2,3.9588988E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23,25,25,28,28,29,29,30,30,31,31,32,32,36,36,37,37,41,41,42,42,45,45,46,46,50,50,51,51,53,53,59,59,60,60,61,61,64,64,71,71,73,73,74,74,75,75,77,77,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,30,32,34,36,38,-1,40,42,-1,44,-1,-1,46,48,50,52,54,-1,-1,-1,56,58,-1,-1,-1,60,62,-1,-1,64,66,-1,-1,-1,68,70,-1,72,-1,-1,-1,-1,-1,74,76,78,-1,-1,80,-1,-1,-1,-1,-1,-1,82,-1,84,86,88,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,92,94,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,1.83E2,9.0629734E8,1.20944E5,1.267716E7,1.763175E6,2.3880364E9,3.95265E5,1.3925089E9,1.0016946E7,1.1541341E9,4.56E2,1.3397212E-2,5.5666803E10,8.763841E6,-8.686552E-3,5.46E2,1.3222029E0,1.16E2,6.7452E4,2.1E1,1.848262E-2,1.5268964E3,7.192367E10,-7.7835103E-3,1.5102771E9,5.9100226E-3,-5.81281E-4,1E0,3.7E1,4.695E5,1.914442E6,2.4064786E0,3.8442553E-3,1.1989882E-3,-7.121587E-3,2.1509E4,1.71E3,5.194017E-3,9.2925485E-3,7.42807E-4,9.2E1,7.6565686E2,1.0926402E-2,2.3874894E-2,2.3787892E0,1.6E1,-4.8754603E-4,-2.2071057E-3,-5.9986617E-3,1.219769E6,7.7643576E7,-1.446113E-4,8.922E3,-2.8562287E-3,1.4736996E-3,-3.767911E-3,-4.3421896E-4,1.6286591E-3,2.7729868E3,1.06E3,2.4866E4,-4.6191835E-3,6.13536E-3,4.93E2,6.3120304E-3,2.1720356E-3,-1.7695482E-3,-3.8881628E-3,-3.063954E-3,-1.4245192E-3,6.832E3,-7.389572E-4,1.8484124E3,4.0318163E8,7.83E2,1.3727109E-3,4.445E3,3.431651E-3,1.3042038E-2,8.219392E-3,1.2796959E-4,2.20142E-3,3.8496163E-3,-1.5938506E-3,6.025635E-3,2.6769326E-3,2.748732E6,1.828682E0,2.4302476E-4,-8.750406E-4,6.055522E-3,2.106444E-3,1.3041802E-2,4.8903082E-3],"split_indices":[2,47,16,12,18,12,11,2,16,26,10,4,0,13,2,0,28,64,4,18,19,0,5,10,0,16,0,0,34,20,6,6,67,0,0,0,12,3,0,0,0,4,5,0,0,60,17,0,0,0,2,18,0,3,0,0,0,0,0,5,17,0,0,0,19,0,0,0,0,0,0,3,0,5,16,17,0,3,0,0,0,0,0,0,0,0,0,18,64,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.94E2,1.9E2,1.04E2,1.55E2,3.5E1,5.5E1,4.9E1,9.3E1,6.2E1,2.4E1,1.1E1,5E1,5E0,1E1,3.9E1,2.1E1,7.2E1,5E1,1.2E1,1.2E1,1.2E1,4E0,7E0,4.7E1,3E0,7E0,3E0,3E0,3.6E1,7E0,6.5E1,3.7E1,1.3E1,7E0,5E0,6E0,6E0,6E0,6E0,4E0,3E0,3.4E1,1.3E1,3E0,4E0,2.9E1,7E0,3E0,4E0,5.2E1,1.3E1,3.3E1,4E0,1E1,3E0,3E0,3E0,3E0,3E0,1.4E1,2E1,9E0,4E0,6E0,2.3E1,3E0,4E0,4E0,9E0,2.8E1,5E0,6E0,4E0,7E0,7E0,1.7E1,3E0,6E0,3E0,1.4E1,9E0,3E0,3E0,3E0,4E0,3E0,4E0,1.1E1,6E0,3E0,3E0,8E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-5.3288587E-3,-9.817604E-2,2.0741794E-1,-1.6387928E-1,-1.8241324E-2,1.0529056E-1,3.169409E-1,-3.1416774E-1,-1.4359741E-1,-6.303225E-2,1.5245277E-1,3.633572E-1,5.7496637E-2,3.4750572E-1,-1.6950751E-3,-1.1474968E-2,-5.9106094E-3,5.005609E-3,-1.6872329E-1,-1.4913638E-1,-2.9350832E-2,2.8048944E-2,1.0300528E-2,5.065624E-3,1.5402532E-2,8.1650786E-2,-2.9931015E-3,2.7002636E-1,4.7573322E-1,-1.2283333E-1,1.3221884E-1,-1.9069889E-1,-7.5467383E-3,-1.7166819E-1,-1.5195552E-3,-4.433342E-2,3.9704666E-3,-4.7344035E-3,5.6058313E-3,6.152744E-3,5.7030346E-2,3.0954373E-1,1.4241934E-1,6.7380224E-3,1.7212156E-2,-6.270494E-3,-1.3098081E-3,5.9278687E-3,1.1021384E-3,-1.7544529E-1,-1.3196254E-2,-1.7394013E-3,1.2940449E-3,-3.318834E-3,-6.2272837E-3,-5.635975E-2,1.6496377E-3,4.173E-3,9.72052E-2,4.9063703E-3,1.0605527E-2,2.020062E-3,5.718879E-3,1.024655E-3,-1.852015E-1,-6.856517E-2,-8.0048125E-3,-2.5550243E-2,1.5470776E-3,1.2406336E-1,-6.791765E-4,-2.0941915E-1,-1.2614635E-1,-7.99068E-2,-2.692191E-2,-2.05168E-3,2.498663E-2,1.6932399E-4,-1.6497083E-3,1.4697692E-3,4.548137E-3,-7.25913E-3,-3.3724022E-3,-1.4482333E-1,-6.543519E-2,-9.016654E-2,-9.325131E-4,3.2073763E-4,-1.5137075E-3,-5.302267E-4,1.9377405E-3,-5.0162375E-3,-2.3440765E-3,-3.1579772E-3,-5.280317E-4,-5.8578365E-2,-3.7477417E-3,-7.8393385E-4,-2.3384062E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,-1,29,31,33,35,37,-1,-1,-1,39,-1,41,43,45,47,49,51,53,-1,55,-1,-1,-1,-1,57,59,61,-1,-1,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,65,-1,67,69,-1,-1,-1,-1,-1,71,73,75,77,-1,79,-1,81,83,85,87,-1,89,-1,-1,-1,-1,-1,-1,91,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,97,-1,-1,-1],"loss_changes":[5.5906925E0,1.0316485E0,9.285641E-1,2.8408456E-1,6.977086E-1,5.4647297E-1,4.814682E-1,8.3708763E-4,3.6302114E-1,2.0443058E-1,3.8705564E-1,9.072673E-2,1.5298292E-1,2.4906874E-1,0E0,0E0,0E0,2.6018295E-1,2.9254293E-1,3.7828922E-2,1.2672369E-1,3.3266398E-1,0E0,0E0,0E0,8.3451435E-2,0E0,8.702493E-2,9.236407E-2,3.9366923E-2,3.4406483E-2,1.7222643E-1,2.65906E-2,3.9595366E-3,0E0,5.921386E-2,0E0,0E0,0E0,0E0,6.3300446E-2,9.286523E-3,1.1313081E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.474526E-1,0E0,0E0,0E0,0E0,0E0,2.5419593E-2,0E0,1.9491486E-2,5.6634367E-2,0E0,0E0,0E0,0E0,0E0,6.940436E-2,1.4379352E-2,1.9980004E-2,7.3962165E-3,0E0,1.2462422E-2,0E0,5.386257E-2,1.5469283E-2,1.1498034E-2,7.142522E-3,0E0,1.1210666E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3397038E-3,9.6043255E-3,1.14308745E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.008308E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,40,40,41,41,42,42,49,49,55,55,57,57,58,58,64,64,65,65,66,66,67,67,69,69,71,71,72,72,73,73,74,74,76,76,83,83,84,84,85,85,95,95],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,-1,30,32,34,36,38,-1,-1,-1,40,-1,42,44,46,48,50,52,54,-1,56,-1,-1,-1,-1,58,60,62,-1,-1,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,66,-1,68,70,-1,-1,-1,-1,-1,72,74,76,78,-1,80,-1,82,84,86,88,-1,90,-1,-1,-1,-1,-1,-1,92,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,98,-1,-1,-1],"split_conditions":[1.1026E4,5.013E3,1.4066751E7,1.0727E4,1.83E2,5.345E3,2.3960164E7,1.00292633E3,2.2155E4,1.13606E5,7.31384E6,4.9E1,1E0,1.0218795E4,-1.6950751E-3,-1.1474968E-2,-5.9106094E-3,6.52028E5,1.125733E6,2.748732E6,4.365884E6,8.6E1,1.0300528E-2,5.065624E-3,1.5402532E-2,9E1,-2.9931015E-3,8.28337E5,5.2068E4,9E0,1.213102E6,1.56669E5,4.4078424E7,7.318324E-4,-1.5195552E-3,6.2498252E7,3.9704666E-3,-4.7344035E-3,5.6058313E-3,6.152744E-3,4.29906E5,3.6386406E-1,5.669117E3,6.7380224E-3,1.7212156E-2,-6.270494E-3,-1.3098081E-3,5.9278687E-3,1.1021384E-3,1E0,-1.3196254E-2,-1.7394013E-3,1.2940449E-3,-3.318834E-3,-6.2272837E-3,4.0828186E-1,1.6496377E-3,1.346326E9,2.239676E10,4.9063703E-3,1.0605527E-2,2.020062E-3,5.718879E-3,1.024655E-3,2.47796E5,1.578606E6,6.3226E4,7.5469E4,1.5470776E-3,2.4272145E-1,-6.791765E-4,2.4472736E7,1.835E3,8.183E3,2.235E3,-2.05168E-3,2.210589E6,1.6932399E-4,-1.6497083E-3,1.4697692E-3,4.548137E-3,-7.25913E-3,-3.3724022E-3,1.9276653E8,2.953E3,9.675527E7,-9.325131E-4,3.2073763E-4,-1.5137075E-3,-5.302267E-4,1.9377405E-3,-5.0162375E-3,-2.3440765E-3,-3.1579772E-3,-5.280317E-4,5.395E3,-3.7477417E-3,-7.8393385E-4,-2.3384062E-3],"split_indices":[3,3,2,18,47,7,18,5,8,12,18,4,36,5,0,0,0,21,6,18,6,19,0,0,0,4,0,28,8,17,2,22,16,63,0,18,0,0,0,0,12,67,5,0,0,0,0,0,0,38,0,0,0,0,0,63,0,16,10,0,0,0,0,0,6,6,8,8,0,63,0,18,0,3,0,0,2,0,0,0,0,0,0,21,3,21,0,0,0,0,0,0,0,0,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.96E2,8.5E1,1.07E2,8.9E1,4.5E1,4E1,1.1E1,9.6E1,7.1E1,1.8E1,6E0,3.9E1,3.7E1,3E0,7E0,4E0,1.4E1,8.2E1,1.9E1,5.2E1,1.1E1,7E0,3E0,3E0,3.4E1,5E0,2.5E1,1.2E1,7E0,7E0,7.2E1,1E1,1.5E1,4E0,4.8E1,4E0,5E0,6E0,5E0,2.9E1,1.8E1,7E0,3E0,9E0,3E0,4E0,4E0,3E0,6.9E1,3E0,5E0,5E0,5E0,1E1,4.3E1,5E0,1.3E1,1.6E1,3E0,1.5E1,3E0,4E0,3E0,6.6E1,3.4E1,9E0,8E0,5E0,1.3E1,3E0,4.5E1,2.1E1,2.6E1,8E0,3E0,6E0,4E0,4E0,3E0,1E1,3.8E1,7E0,1.5E1,6E0,2.1E1,5E0,3E0,5E0,3E0,3E0,1.2E1,3E0,3E0,3E0,1.1E1,1E1,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[-3.8726124E-4,-9.229344E-2,2.1580555E-1,-1.2585959E-1,3.491428E-2,1.6993396E-1,4.6737748E-1,-1.5266202E-1,-1.0528796E-2,-3.145606E-2,2.6371905E-1,1.043963E-1,2.3272435E-1,6.416324E-1,-2.7394127E-3,2.2467883E-2,-1.7021596E-1,-6.909781E-3,2.629978E-2,-4.560599E-2,2.8843696E-3,1.1167462E-2,1.6790462E-3,1.067803E-2,6.120818E-2,1.3524753E-2,2.0465301E-1,2.4961473E-2,1.135319E-2,-5.1714466E-3,9.3471855E-2,-3.16743E-1,-1.526838E-1,-4.276427E-2,1.0927863E-1,-1.0002303E-1,-7.95462E-3,8.934034E-4,1.2900648E-1,1.5978183E-1,7.814344E-3,2.2564564E-2,6.2508276E-3,-4.0591037E-1,-1.1887581E-3,-1.7566659E-1,-6.7515634E-2,-8.46676E-3,-3.7508325E-3,9.0010243E-4,4.212794E-3,-7.16679E-4,-3.8988567E-3,5.6076884E-2,-4.6746727E-2,1.210577E-1,-6.387917E-2,5.9918673E-3,8.6111225E-2,6.9614407E-3,3.5024567E-3,-1.3454405E-3,2.6165177E-3,-1.5164849E-2,-6.65085E-3,-2.2185792E-1,-1.4559768E-1,-8.163972E-2,8.705754E-4,1.1445532E-3,-3.4923464E-2,2.796004E-3,6.512874E-4,-2.151242E-2,-3.1592115E-3,1.998565E-3,4.8206975E-3,-1.1173947E-1,-6.893273E-3,6.485588E-4,3.486988E-3,-2.3703393E-1,-1.814407E-3,-4.9578648E-2,-1.563433E-1,-8.924809E-4,-9.208013E-2,-1.4359838E-3,-4.533203E-4,2.3724667E-4,-1.3518096E-3,-1.5859257E-3,-4.708435E-3,-6.7842007E-4,2.901173E-4,-8.063246E-3,-3.937047E-3,-2.407052E-3,-3.8574668E-4,-1.8908484E-1,-1.2503411E-1,-4.9782814E-3,-2.2292146E-3,-6.400895E-3,-3.0711957E-3,-1.3318254E-1,-1.960786E-3,-2.0841605E-3,-4.502454E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,35,-1,-1,-1,-1,37,-1,39,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,-1,61,-1,63,-1,65,67,69,-1,-1,-1,-1,-1,71,73,75,77,-1,79,-1,-1,-1,-1,-1,-1,81,83,85,-1,-1,87,-1,-1,89,-1,-1,-1,91,93,-1,-1,95,-1,97,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,103,105,-1,-1,-1,-1,107,-1,-1,-1],"loss_changes":[5.861538E0,8.8825786E-1,9.3609715E-1,5.053408E-1,6.7647296E-1,2.8845453E-1,1.3061497E0,4.1269565E-1,2.4622078E-1,6.404304E-2,1.5795124E-1,3.626012E-1,1.13959074E-1,1.13174915E-1,0E0,1.840527E-1,2.6248026E-1,0E0,1.6513716E-1,6.47684E-2,0E0,0E0,0E0,0E0,1.3891886E-1,0E0,2.1064878E-2,0E0,0E0,0E0,6.703583E-2,2.8441107E-1,2.0345378E-1,3.949059E-2,1.9171953E-2,1.863332E-2,5.2299753E-2,1.5561673E-1,2.318737E-2,3.1173646E-2,0E0,2.9854013E-2,0E0,1.9618511E-2,0E0,8.714557E-2,3.5597876E-2,1.5218572E-2,0E0,0E0,0E0,0E0,0E0,7.070156E-3,1.4749911E-2,2.5503412E-3,3.4685463E-2,0E0,1.3938114E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.700659E-2,4.8996806E-2,9.830266E-3,0E0,0E0,9.868834E-4,0E0,0E0,6.22579E-3,0E0,0E0,0E0,7.895179E-3,1.7691324E-3,0E0,0E0,1.4153719E-2,0E0,5.7356358E-3,2.658844E-2,0E0,1.5000716E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1589527E-3,6.516367E-3,0E0,0E0,0E0,0E0,1.9364953E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,24,24,26,26,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,43,43,45,45,46,46,47,47,53,53,54,54,55,55,56,56,58,58,65,65,66,66,67,67,70,70,73,73,77,77,78,78,81,81,83,83,84,84,86,86,99,99,100,100,105,105],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,36,-1,-1,-1,-1,38,-1,40,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,-1,62,-1,64,-1,66,68,70,-1,-1,-1,-1,-1,72,74,76,78,-1,80,-1,-1,-1,-1,-1,-1,82,84,86,-1,-1,88,-1,-1,90,-1,-1,-1,92,94,-1,-1,96,-1,98,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,104,106,-1,-1,-1,-1,108,-1,-1,-1],"split_conditions":[1.0655E4,4.3199274E8,1.0778861E4,1.203673E6,1.806E3,2.55085E5,2.3177024E7,1.3213E4,6.162E3,3.99E2,5.776113E9,5.345E3,3.7439016E-1,1.7754914E0,-2.7394127E-3,4.57473E5,2.7708E4,-6.909781E-3,2.402988E6,2.47796E5,2.8843696E-3,1.1167462E-2,1.6790462E-3,1.067803E-2,1.90063E7,1.3524753E-2,7.51842E5,2.4961473E-2,1.135319E-2,-5.1714466E-3,1.1117731E-2,2.748775E6,1.447831E6,4.4E2,5.710497E6,5.46E2,9.12E3,4.2E1,1.08E2,3.8248E4,7.814344E-3,8.109E3,6.2508276E-3,1.644793E3,-1.1887581E-3,1.49015E5,6.344E3,3.4311706E9,-3.7508325E-3,9.0010243E-4,4.212794E-3,-7.16679E-4,-3.8988567E-3,5E1,9.9E1,2.9645083E10,2.8673542E8,5.9918673E-3,1.2639E4,6.9614407E-3,3.5024567E-3,-1.3454405E-3,2.6165177E-3,-1.5164849E-2,-6.65085E-3,5.26565E5,1.2140554E-4,6.3645E4,8.705754E-4,1.1445532E-3,2.1504E4,2.796004E-3,6.512874E-4,7.221E3,-3.1592115E-3,1.998565E-3,4.8206975E-3,6.71104E5,6.12383E8,6.485588E-4,3.486988E-3,1.7696283E3,-1.814407E-3,6.7488E4,4.436827E7,-8.924809E-4,4.591E3,-1.4359838E-3,-4.533203E-4,2.3724667E-4,-1.3518096E-3,-1.5859257E-3,-4.708435E-3,-6.7842007E-4,2.901173E-4,-8.063246E-3,-3.937047E-3,-2.407052E-3,-3.8574668E-4,2.0230963E8,6.641E3,-4.9782814E-3,-2.2292146E-3,-6.400895E-3,-3.0711957E-3,1.2178E4,-1.960786E-3,-2.0841605E-3,-4.502454E-3],"split_indices":[3,16,5,6,0,22,18,8,20,4,10,7,64,59,0,21,8,0,6,6,0,0,0,0,18,0,18,0,0,0,63,18,9,4,2,28,20,4,4,18,0,20,0,5,0,18,0,10,0,0,0,0,0,4,19,1,27,0,3,0,0,0,0,0,0,6,63,12,0,0,7,0,0,3,0,0,0,6,16,0,0,5,0,8,11,0,3,0,0,0,0,0,0,0,0,0,0,0,0,16,3,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.93E2,2.06E2,8.7E1,1.63E2,4.3E1,7.5E1,1.2E1,1.32E2,3.1E1,3.4E1,9E0,3.8E1,3.7E1,9E0,3E0,1.2E1,1.2E2,4E0,2.7E1,3.1E1,3E0,6E0,3E0,5E0,3.3E1,3E0,3.4E1,5E0,4E0,3E0,9E0,1.1E1,1.09E2,1.5E1,1.2E1,1.2E1,1.9E1,1.8E1,1.5E1,1.8E1,1.6E1,6E0,3E0,8E0,3E0,8.5E1,2.4E1,1.1E1,4E0,3E0,9E0,3E0,9E0,7E0,1.2E1,6E0,1.2E1,5E0,1E1,7E0,1.1E1,3E0,3E0,5E0,3E0,3.1E1,5.4E1,2.1E1,3E0,4E0,7E0,3E0,4E0,9E0,3E0,3E0,3E0,6E0,6E0,3E0,7E0,2.8E1,3E0,6E0,4.8E1,4E0,1.7E1,4E0,3E0,4E0,5E0,3E0,3E0,3E0,3E0,2.4E1,4E0,3E0,3E0,2.1E1,2.7E1,3E0,1.4E1,1.8E1,3E0,2.3E1,4E0,3E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-5.130161E-4,-8.851197E-2,1.896838E-1,-1.4073077E-1,9.082016E-3,6.0334906E-2,3.2053298E-1,-1.3170178E-1,-1.2697756E-2,-3.5858124E-2,1.9461387E-1,8.552273E-2,-7.664575E-3,2.3854199E-1,5.1031464E-1,-1.9580139E-1,-9.697137E-2,-5.890188E-2,8.064209E-2,3.0922568E-1,3.582698E-2,1.3507098E-1,-1.0140321E-1,3.3523017E-1,1.7772457E-1,6.0592E-1,4.626278E-3,-2.0700414E-1,-1.0307814E-3,-1.1822115E-1,-2.199855E-2,2.4239533E-3,-6.929147E-2,4.227056E-3,8.0376136E-4,1.240708E-2,6.0001807E-3,-2.5253296E-3,4.543488E-3,9.4899654E-2,2.6380813E-1,1.5307982E-3,-8.365084E-3,1.2132879E-2,5.0731557E-3,9.673553E-2,2.0711131E-1,2.2213373E-2,9.886432E-3,-2.1908647E-1,-2.4526075E-3,-1.3216443E-1,1.81815E-2,-1.4196639E-1,6.981054E-2,-1.230998E-1,-4.6679713E-2,1.1135747E-1,-1.152399E-3,1.0589552E-2,3.7464793E-3,4.2369827E-3,9.3140383E-4,3.345413E-3,7.1758083E-3,-7.834096E-3,-1.39544E-1,-7.839739E-2,-1.5516874E-1,1.713003E-3,-6.8882684E-4,-5.8710305E-3,-1.4753925E-3,-1.1461739E-2,4.9246885E-3,-4.429476E-3,-1.8042879E-3,-2.7061895E-2,-4.304282E-3,2.2416325E-4,1.24671236E-1,-1.914033E-3,-5.2191755E-3,-9.438071E-2,2.939181E-4,-1.6511086E-1,-1.6275793E-3,-1.4599679E-3,8.1432034E-4,-4.4860784E-2,3.528615E-2,1.5799433E-1,8.0958866E-2,-2.1222369E-3,-4.043097E-3,-1.7796758E-1,-2.2217866E-3,-3.0146025E-3,-2.6104344E-2,2.1411853E-3,-1.5349251E-4,1.766714E-1,2.4615652E-3,6.328916E-4,3.5134153E-3,-3.033168E-3,-1.8656471E-1,-3.4901295E-2,4.3166374E-4,6.7161713E-3,2.7211972E-3,-4.9010077E-3,-7.266448E-3,-1.3389519E-3,-1.9603077E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,51,53,-1,55,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1,61,63,-1,-1,65,-1,67,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,81,83,85,-1,-1,-1,-1,87,-1,-1,-1,89,-1,-1,91,-1,-1,93,-1,95,-1,-1,-1,97,99,101,103,-1,-1,105,-1,-1,107,-1,-1,109,-1,-1,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.920766E0,1.0256224E0,1.5547042E0,2.4817514E-1,5.989643E-1,3.8119096E-1,5.940461E-1,2.6596737E-1,0E0,1.604569E-1,2.4370527E-1,4.304799E-1,0E0,1.4035225E-1,3.685422E-1,7.410109E-2,1.328175E-1,7.354072E-2,2.4934359E-2,3.0418038E-3,9.532853E-2,1.6114026E-1,2.4841711E-1,2.9470444E-2,3.4705102E-2,3.954053E-2,0E0,4.8107028E-2,0E0,1.2792856E-1,2.3019351E-1,0E0,5.022198E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.636712E-2,3.857082E-2,0E0,0E0,0E0,0E0,1.5117183E-2,2.9420853E-3,0E0,0E0,3.753078E-2,0E0,6.204748E-2,1.0846021E-2,2.6615962E-2,8.28788E-2,4.9325675E-3,5.4437146E-2,3.5268426E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0265008E-2,2.8841674E-2,3.4291387E-2,0E0,0E0,0E0,0E0,9.869109E-3,0E0,0E0,0E0,3.3765014E-2,0E0,0E0,2.0711452E-2,0E0,0E0,6.820515E-3,0E0,3.3424854E-2,0E0,0E0,0E0,1.8604241E-2,9.074406E-3,4.760623E-3,1.8590122E-2,0E0,0E0,5.4569244E-3,0E0,0E0,6.7249862E-3,0E0,0E0,8.876443E-3,0E0,0E0,0E0,0E0,3.2641888E-3,2.6401915E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,32,32,39,39,40,40,45,45,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,66,66,67,67,68,68,73,73,77,77,80,80,83,83,85,85,89,89,90,90,91,91,92,92,95,95,98,98,101,101,106,106,107,107],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,52,54,-1,56,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1,62,64,-1,-1,66,-1,68,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,82,84,86,-1,-1,-1,-1,88,-1,-1,-1,90,-1,-1,92,-1,-1,94,-1,96,-1,-1,-1,98,100,102,104,-1,-1,106,-1,-1,108,-1,-1,110,-1,-1,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,2.917845E8,9.0629734E8,1.9968E5,1.83E2,4.84E2,2.51383E5,5.61E2,-1.2697756E-2,1.3925089E9,4.9E1,1.11301E5,-7.664575E-3,4.3201213E9,1.83E3,6.641E3,1.83E2,1E0,2.231867E9,4.833E3,6.521E3,1.4181395E7,8.58437E6,2.3723983E9,1.1328217E10,1E0,4.626278E-3,4.0200136E7,-1.0307814E-3,1.460109E6,1.43497E5,2.4239533E-3,2.47796E5,4.227056E-3,8.0376136E-4,1.240708E-2,6.0001807E-3,-2.5253296E-3,4.543488E-3,9.02E2,1.3774316E11,1.5307982E-3,-8.365084E-3,1.2132879E-2,5.0731557E-3,3.1E2,2.8E1,2.2213373E-2,9.886432E-3,2.9736228E7,-2.4526075E-3,3.345233E-4,8.5E1,3.618E3,5.876475E7,4.805687E7,1.275E3,1.8E1,-1.152399E-3,1.0589552E-2,3.7464793E-3,4.2369827E-3,9.3140383E-4,3.345413E-3,7.1758083E-3,-7.834096E-3,1.2345292E7,1.9341E4,1.4397123E8,1.713003E-3,-6.8882684E-4,-5.8710305E-3,-1.4753925E-3,7.939E3,4.9246885E-3,-4.429476E-3,-1.8042879E-3,5.421011E7,-4.304282E-3,2.2416325E-4,2.7E2,-1.914033E-3,-5.2191755E-3,2.7653658E9,2.939181E-4,7.482E3,-1.6275793E-3,-1.4599679E-3,8.1432034E-4,2.177237E9,7.091E3,4.9097216E8,4.365884E6,-2.1222369E-3,-4.043097E-3,1.245E4,-2.2217866E-3,-3.0146025E-3,1.4688152E9,2.1411853E-3,-1.5349251E-4,1.1086E4,2.4615652E-3,6.328916E-4,3.5134153E-3,-3.033168E-3,1.0014492E8,3.044519E6,4.3166374E-4,6.7161713E-3,2.7211972E-3,-4.9010077E-3,-7.266448E-3,-1.3389519E-3,-1.9603077E-4],"split_indices":[2,16,16,22,47,4,3,0,0,16,4,8,0,11,19,3,47,45,10,7,7,6,2,16,21,36,0,18,0,6,27,0,6,0,0,0,0,0,0,17,1,0,0,0,0,17,17,0,0,23,0,63,4,7,16,18,0,17,0,0,0,0,0,0,0,0,21,7,27,0,0,0,0,20,0,0,0,18,0,0,17,0,0,11,0,3,0,0,0,10,3,16,6,0,0,7,0,0,11,0,0,0,0,0,0,0,16,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.92E2,2E2,9.2E1,1.3E2,7E1,4.7E1,4.5E1,1.27E2,3E0,5.7E1,1.3E1,4.4E1,3E0,3.3E1,1.2E1,4.3E1,8.4E1,4.8E1,9E0,7E0,6E0,3.5E1,9E0,1.1E1,2.2E1,9E0,3E0,4E1,3E0,6.5E1,1.9E1,3E0,4.5E1,4E0,5E0,3E0,4E0,3E0,3E0,2.8E1,7E0,5E0,4E0,8E0,3E0,7E0,1.5E1,6E0,3E0,3.6E1,4E0,5.9E1,6E0,8E0,1.1E1,1.2E1,3.3E1,2.5E1,3E0,4E0,3E0,4E0,3E0,3E0,1.2E1,2.6E1,1E1,1.9E1,4E1,3E0,3E0,5E0,3E0,6E0,5E0,9E0,3E0,2.8E1,5E0,3E0,2.2E1,3E0,7E0,1.6E1,3E0,3.6E1,4E0,3E0,3E0,2.2E1,6E0,1.1E1,1.1E1,1E1,6E0,3.1E1,5E0,5E0,1.7E1,3E0,3E0,8E0,3E0,4E0,7E0,4E0,2.7E1,1.4E1,3E0,5E0,3E0,1.7E1,1E1,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[2.0857658E-2,-8.900551E-2,2.1595927E-1,-1.440595E-1,1.9833196E-2,8.978444E-2,3.3295158E-1,-1.8883717E-1,-9.014944E-2,-2.6063187E-2,2.266501E-1,-1.8046396E-2,1.4763352E-1,4.5962462E-1,2.6191175E-1,3.0192756E-4,-1.9890435E-1,-1.490354E-1,-3.6109958E-2,-5.8345243E-2,9.3695484E-2,3.053703E-1,4.038802E-4,3.875166E-2,-4.4453316E-3,1.0899597E-1,8.692755E-3,8.020103E-3,6.190229E-1,1.8685777E-1,3.4952652E-1,-2.8055456E-1,-1.5723154E-1,-1.7447802E-1,-3.1699452E-3,5.1819623E-2,-6.512416E-2,-6.881286E-2,1.7081581E-3,3.9316895E-3,1.1307468E-3,4.1939383E-3,1.2372133E-2,7.297913E-2,-9.852431E-4,-5.562849E-4,1.3988505E-1,1.0115287E-2,2.5750978E-2,2.185435E-1,2.2924829E-3,1.2836428E-2,5.512543E-3,-1.215424E-2,-6.6542113E-3,-2.66033E-4,-1.6847861E-1,-6.7549213E-3,-2.5926866E-3,4.1504474E-3,-6.515262E-4,-3.0565974E-5,-7.858463E-2,-9.93869E-2,-3.46171E-2,7.921479E-5,3.4708735E-3,6.201317E-4,1.5891804E-1,7.768689E-3,2.8905119E-3,-6.2162825E-3,-1.0434363E-1,-2.7950776E-3,-8.350868E-4,-7.5338E-4,-1.1217071E-1,-5.521125E-2,6.0126185E-4,2.5716985E-3,5.8959327E-3,-1.5689047E-3,-5.185338E-3,-1.2538256E-1,-1.3258486E-3,-5.852448E-4,-2.5449777E-3,8.3210337E-4,-1.0014211E-3,-4.5335256E-3,-1.7829716E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,-1,43,-1,45,-1,-1,47,49,51,53,55,57,-1,59,61,63,-1,-1,-1,-1,-1,65,-1,-1,67,-1,-1,69,-1,-1,-1,-1,-1,-1,71,-1,-1,-1,-1,-1,73,75,77,-1,-1,-1,79,-1,-1,-1,81,-1,-1,-1,83,85,87,-1,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8922687E0,1.0574498E0,1.4314928E0,2.654109E-1,5.7543606E-1,3.0832207E-1,3.3855963E-1,1.2758112E-1,1.696375E-1,1.9876948E-1,1.8156499E-1,1.2809141E-1,1.2192607E-1,4.1608524E-1,1.6484237E-1,0E0,1.6004372E-1,1.6260803E-2,7.991091E-2,5.0300345E-2,1.4036916E-2,6.0556233E-2,0E0,3.309587E-2,0E0,1.0383326E-1,0E0,0E0,2.4500489E-1,6.138116E-2,6.342244E-2,5.8064222E-2,6.8976164E-2,3.626159E-2,0E0,4.7068425E-2,1.9793227E-2,3.5243094E-2,0E0,0E0,0E0,0E0,0E0,2.1852002E-2,0E0,0E0,4.5149207E-2,0E0,0E0,2.5299668E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.1480064E-2,0E0,0E0,0E0,0E0,0E0,5.9975907E-3,1.58301E-2,1.3813749E-2,0E0,0E0,0E0,1.9857764E-2,0E0,0E0,0E0,3.3167407E-2,0E0,0E0,0E0,9.84022E-3,9.161487E-3,7.209874E-3,0E0,0E0,0E0,0E0,6.2642545E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,43,43,46,46,49,49,56,56,62,62,63,63,64,64,68,68,72,72,76,76,77,77,78,78,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,-1,44,-1,46,-1,-1,48,50,52,54,56,58,-1,60,62,64,-1,-1,-1,-1,-1,66,-1,-1,68,-1,-1,70,-1,-1,-1,-1,-1,-1,72,-1,-1,-1,-1,-1,74,76,78,-1,-1,-1,80,-1,-1,-1,82,-1,-1,-1,84,86,88,-1,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,2.917845E8,8.912913E8,4.014E3,1.83E2,1.8703245E3,1.3219E4,3E0,1.358313E6,1.3925089E9,1.5268964E3,1.11301E5,1.83E2,5.58862E5,2.12632E5,3.0192756E-4,2.8895E4,2.379127E-2,4.0061934E9,3.72E2,3.6E1,4.264E3,4.038802E-4,5.3E2,-4.4453316E-3,3.718604E6,8.692755E-3,8.020103E-3,3.618E3,8.13E2,1.074E3,6.6362006E2,2.705253E2,3.28451E5,-3.1699452E-3,5.131E3,4.283E3,2.3912651E8,1.7081581E-3,3.9316895E-3,1.1307468E-3,4.1939383E-3,1.2372133E-2,4.445E3,-9.852431E-4,-5.562849E-4,6.3E1,1.0115287E-2,2.5750978E-2,3.91955E10,2.2924829E-3,1.2836428E-2,5.512543E-3,-1.215424E-2,-6.6542113E-3,-2.66033E-4,2.2296707E9,-6.7549213E-3,-2.5926866E-3,4.1504474E-3,-6.515262E-4,-3.0565974E-5,9.2921E4,5.46E2,7.164074E8,7.921479E-5,3.4708735E-3,6.201317E-4,3.3004024E9,7.768689E-3,2.8905119E-3,-6.2162825E-3,1.6525185E9,-2.7950776E-3,-8.350868E-4,-7.5338E-4,8.998854E7,2.282993E6,2.2599797E3,2.5716985E-3,5.8959327E-3,-1.5689047E-3,-5.185338E-3,1.5936E4,-1.3258486E-3,-5.852448E-4,-2.5449777E-3,8.3210337E-4,-1.0014211E-3,-4.5335256E-3,-1.7829716E-3],"split_indices":[2,16,16,3,47,5,7,17,9,16,5,8,47,18,3,0,18,63,10,4,19,3,0,17,0,6,0,0,7,17,19,5,5,6,0,3,3,21,0,0,0,0,0,3,0,0,19,0,0,21,0,0,0,0,0,0,10,0,0,0,0,0,28,28,16,0,0,0,11,0,0,0,11,0,0,0,18,18,5,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.75E2,9.8E1,1.16E2,5.9E1,4.8E1,5E1,6.2E1,5.4E1,4.9E1,1E1,1.7E1,3.1E1,1.6E1,3.4E1,3E0,5.9E1,2.5E1,2.9E1,3.9E1,1E1,7E0,3E0,1.2E1,5E0,2.5E1,6E0,8E0,8E0,2E1,1.4E1,1.8E1,4.1E1,1.5E1,1E1,7E0,2.2E1,3.6E1,3E0,6E0,4E0,3E0,4E0,8E0,4E0,5E0,2E1,4E0,4E0,1.5E1,5E0,1E1,4E0,6E0,1.2E1,3E0,3.8E1,1E1,5E0,3E0,4E0,4E0,1.8E1,1.8E1,1.8E1,3E0,5E0,3E0,1.7E1,1.2E1,3E0,2.6E1,1.2E1,1.5E1,3E0,3E0,1.5E1,1.1E1,7E0,5E0,1.2E1,7E0,5E0,1.2E1,3E0,5E0,6E0,4E0,3E0,9E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[4.12713E-3,-1.0756448E-1,1.4782232E-1,-1.6085331E-1,-2.6423464E-2,9.8585986E-2,3.0701602E-1,-1.8497074E-1,5.087964E-4,3.3707116E-2,-7.045842E-2,5.313826E-2,2.3280364E-1,4.1389847E-1,2.233526E-1,9.165712E-4,-1.9293277E-1,-6.269591E-2,4.0820898E-3,2.2813203E-3,4.9504186E-3,-3.454927E-2,-9.663598E-2,7.169018E-2,-9.615447E-3,1.6075653E-1,3.9510244E-1,5.447057E-3,4.7665516E-1,3.3454488E-3,2.4826942E-1,-2.745851E-1,-1.7267966E-1,-3.3893643E-3,3.1005E-5,-4.563184E-2,8.720679E-2,-5.148506E-2,1.210217E-3,-6.9918275E-2,-6.4180307E-3,4.486362E-2,3.1821725E-1,6.3529606E-3,1.5626125E-3,6.0706106E-3,1.618578E-2,1.7992314E-2,7.533354E-3,1.6762267E-1,1.0187518E-2,-9.974536E-3,-4.5718458E-3,-1.7989245E-1,-1.3820314E-3,-6.244094E-2,-2.3207754E-4,1.0756892E-3,3.4935176E-3,-2.1783574E-3,-1.7936535E-4,-8.76011E-2,-1.115484E-3,9.255087E-3,1.7900626E-1,1.2642129E-2,5.283264E-3,2.5206949E-3,6.4129354E-3,-1.8979433E-1,-4.0211645E-3,-2.369755E-3,-7.877394E-4,-1.3760942E-3,-3.3696238E-3,-4.1895383E-3,2.3010945E-2,8.2187794E-2,1.08425915E-2,-7.146695E-3,-5.2325437E-3,-5.4097344E-4,7.8001484E-2,5.995218E-3,-1.9031827E-3,-2.8002687E-2,5.0199185E-2,5.9481495E-4,3.409291E-3,1.369038E-2,-4.970487E-2,1.1957784E-4,7.044612E-2,-6.895691E-4,1.4827915E-3,-2.785924E-3,-2.5089415E-2,3.0971924E-3,4.5628258E-4,-9.212904E-5,-1.2032826E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,-1,35,-1,37,39,41,-1,43,45,-1,47,-1,49,51,53,-1,-1,55,57,59,-1,61,-1,63,65,-1,-1,-1,-1,-1,-1,67,-1,-1,-1,69,-1,71,-1,-1,-1,-1,-1,73,-1,75,77,-1,-1,-1,-1,79,-1,-1,-1,-1,-1,-1,81,83,-1,-1,-1,85,87,-1,-1,89,91,-1,-1,93,95,-1,97,-1,-1,-1,99,-1,-1,-1,-1],"loss_changes":[4.734467E0,7.1351874E-1,9.808495E-1,3.8925815E-1,1.7978074E-1,5.976179E-1,1.7581916E-1,1.5334654E-1,1.1971621E-1,1.09088674E-1,3.3321843E-2,5.175146E-1,2.2881377E-1,1.03601456E-1,3.0733228E-2,0E0,9.1496944E-2,2.722019E-2,0E0,1.0163655E-1,0E0,2.3929155E-2,4.6454698E-2,4.6850717E-1,0E0,6.8468004E-2,6.09051E-2,0E0,4.890108E-2,0E0,2.9476762E-2,2.6154041E-2,5.7783604E-2,0E0,0E0,9.883236E-3,6.696403E-3,1.0772131E-2,0E0,8.81204E-3,0E0,3.1837696E-1,1.5924811E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.803474E-3,0E0,0E0,0E0,1.2213707E-2,0E0,2.8673746E-3,0E0,0E0,0E0,0E0,0E0,4.718147E-3,0E0,1.0655069E-1,1.8625546E-1,0E0,0E0,0E0,0E0,1.1311173E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.4798824E-2,1.5806676E-1,0E0,0E0,0E0,5.156853E-2,2.2965036E-2,0E0,0E0,2.239641E-2,1.1973441E-2,0E0,0E0,1.1217809E-2,1.23395175E-2,0E0,1.1688519E-2,0E0,0E0,0E0,2.8985087E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,19,19,21,21,22,22,23,23,25,25,26,26,28,28,30,30,31,31,32,32,35,35,36,36,37,37,39,39,41,41,42,42,49,49,53,53,55,55,61,61,63,63,64,64,69,69,76,76,77,77,81,81,82,82,85,85,86,86,89,89,90,90,92,92,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,-1,36,-1,38,40,42,-1,44,46,-1,48,-1,50,52,54,-1,-1,56,58,60,-1,62,-1,64,66,-1,-1,-1,-1,-1,-1,68,-1,-1,-1,70,-1,72,-1,-1,-1,-1,-1,74,-1,76,78,-1,-1,-1,-1,80,-1,-1,-1,-1,-1,-1,82,84,-1,-1,-1,86,88,-1,-1,90,92,-1,-1,94,96,-1,98,-1,-1,-1,100,-1,-1,-1,-1],"split_conditions":[3.7372906E8,4.695E5,1.7833586E9,1.83E2,3.2933E4,3.48037E5,4.841179E-1,2.8156E5,1.268704E-2,1.83E2,1.6507992E9,2.93968E5,2.12632E5,3.84891E5,1.315E3,9.165712E-4,1.2083252E7,4.5761172E2,4.0820898E-3,7.069832E6,4.9504186E-3,2.564429E6,1.83E2,1.1906453E7,-9.615447E-3,6.8536E5,7.8667163E3,5.447057E-3,1E0,3.3454488E-3,7.1E1,1.916746E6,7.503E3,-3.3893643E-3,3.1005E-5,1.9669E4,6.4615297E-6,1.0500989E10,1.210217E-3,1.9341E4,-6.4180307E-3,1.83E2,1.719E3,6.3529606E-3,1.5626125E-3,6.0706106E-3,1.618578E-2,1.7992314E-2,7.533354E-3,6.5406E4,1.0187518E-2,-9.974536E-3,-4.5718458E-3,1.3953088E8,-1.3820314E-3,3.07E2,-2.3207754E-4,1.0756892E-3,3.4935176E-3,-2.1783574E-3,-1.7936535E-4,3.482E4,-1.115484E-3,8.099343E2,1.6844009E0,1.2642129E-2,5.283264E-3,2.5206949E-3,6.4129354E-3,4.24218E5,-4.0211645E-3,-2.369755E-3,-7.877394E-4,-1.3760942E-3,-3.3696238E-3,-4.1895383E-3,4.29906E5,3.2783824E7,1.08425915E-2,-7.146695E-3,-5.2325437E-3,5.421011E7,3.0371344E-1,5.995218E-3,-1.9031827E-3,4.5556176E8,7.38E2,5.9481495E-4,3.409291E-3,3.6E1,3.649951E9,1.1957784E-4,8.9977784E7,-6.895691E-4,1.4827915E-3,-2.785924E-3,1.3334E4,3.0971924E-3,4.5628258E-4,-9.212904E-5,-1.2032826E-3],"split_indices":[16,6,16,47,18,22,66,16,63,47,11,28,3,23,0,0,11,5,0,2,0,2,47,2,0,28,5,0,36,0,4,18,3,0,0,7,64,10,0,7,0,47,0,0,0,0,0,0,0,8,0,0,0,16,0,19,0,0,0,0,0,8,0,5,64,0,0,0,0,9,0,0,0,0,0,0,12,27,0,0,0,18,63,0,0,11,0,0,0,19,10,0,18,0,0,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.93E2,1.65E2,1.28E2,9.9E1,6.6E1,9.9E1,2.9E1,8.6E1,1.3E1,2.8E1,3.8E1,7.5E1,2.4E1,1.1E1,1.8E1,3E0,8.3E1,9E0,4E0,2.3E1,5E0,1.7E1,2.1E1,7.2E1,3E0,1.8E1,6E0,3E0,8E0,4E0,1.4E1,1.4E1,6.9E1,5E0,4E0,1.5E1,8E0,1.4E1,3E0,1.8E1,3E0,6.6E1,6E0,1.3E1,5E0,3E0,3E0,5E0,3E0,8E0,6E0,1E1,4E0,6.5E1,4E0,1E1,5E0,3E0,5E0,1E1,4E0,1.1E1,7E0,5.3E1,1.3E1,3E0,3E0,3E0,5E0,5.3E1,1.2E1,7E0,3E0,4E0,7E0,4E0,4.9E1,9E0,4E0,2.1E1,3.2E1,3.5E1,1.4E1,5E0,4E0,2.3E1,1.2E1,5E0,9E0,8E0,1.5E1,4E0,8E0,4E0,4E0,5E0,1E1,5E0,3E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-6.5840594E-3,-1.0172737E-1,1.6963755E-1,-1.6895878E-1,-5.6204505E-2,1.8276423E-1,-6.2567457E-3,-1.51098E-1,-1.2514751E-2,6.633279E-3,-6.523092E-2,1.0431604E-1,2.530142E-1,-1.9929972E-1,-7.441618E-2,-9.292588E-3,-5.4075077E-2,2.0908017E-2,1.6314648E-1,1.7473133E-1,3.2743916E-1,-2.2582644E-1,-1.3472809E-1,8.480411E-3,-1.0639967E-1,-9.570506E-2,-5.535388E-3,9.070918E-2,-5.079387E-2,2.0128873E-3,2.0700052E-1,1.14250325E-1,9.788418E-3,1.8627131E-2,2.6687527E-1,-7.767547E-3,-4.592978E-3,-1.6098247E-3,-1.6098753E-1,-2.4813195E-3,4.336174E-3,-1.8062115E-1,-5.6387417E-2,2.4825535E-3,-1.10203974E-1,-4.023309E-2,7.07118E-2,1.2707531E-1,-1.5130074E-4,1.3284225E-3,-4.325926E-3,-1.7333573E-3,1.8467447E-3,1.1070367E-2,1.5746781E-1,6.7576095E-2,1.6378815E-1,1.9083733E-3,2.965321E-1,-5.799419E-3,-2.6517208E-3,-2.6899122E-3,-7.1504815E-3,-9.361272E-2,-1.676685E-2,-9.411958E-2,-5.7419008E-3,-6.1964784E-2,4.0856972E-2,9.206125E-2,-5.263496E-4,2.0576967E-3,5.6087235E-3,1.02384716E-1,6.8731024E-3,1.3250651E-4,2.8413804E-3,6.25504E-3,2.4798235E-3,5.688893E-3,1.0887221E-2,-1.5532292E-3,-3.720039E-3,-1.2310112E-3,4.5934922E-4,-5.072301E-3,-7.387756E-2,9.166469E-5,-6.985619E-2,2.6696217E-3,-7.067401E-4,5.332559E-3,4.094168E-2,1.954212E-3,4.5145494E-3,-9.183922E-2,-3.5034683E-2,-8.76498E-2,-4.3275636E-2,7.806648E-5,2.2941E-3,-6.759426E-2,-3.6807244E-3,4.436817E-4,-5.826324E-2,-9.534639E-4,-1.078993E-1,-1.6779639E-3,-4.035743E-4,-1.19909E-3,-8.031222E-2,-2.8096985E-3,-5.660812E-4,-4.1087186E-3,-1.4926578E-3,-3.1476778E-3,-1.0951087E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,-1,-1,-1,59,-1,-1,61,63,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,73,75,77,-1,79,-1,-1,-1,-1,81,83,85,-1,87,89,91,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,95,-1,97,-1,-1,-1,99,-1,-1,101,103,105,107,-1,-1,109,-1,-1,111,-1,113,-1,-1,-1,115,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8291864E0,5.6282973E-1,5.181315E-1,2.357316E-1,2.8096157E-1,5.1368284E-1,0E0,2.4759889E-1,0E0,0E0,2.5860375E-1,2.3212165E-1,2.3992252E-1,4.3228388E-2,7.779066E-2,0E0,2.1366513E-1,1.0970985E-1,1.9744134E-1,1.7650229E-1,2.259686E-1,2.900958E-3,2.5317818E-2,1.0759341E-1,6.776689E-2,1.513322E-1,1.3512506E-1,3.8763754E-2,9.020545E-2,2.4736864E-2,9.5045865E-2,3.5174638E-2,0E0,0E0,1.1786091E-1,0E0,0E0,0E0,1.7427802E-3,0E0,0E0,1.3673991E-2,1.907884E-2,0E0,4.132998E-2,6.420955E-2,3.1245552E-2,1.2690768E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.3717513E-2,1.5396785E-2,8.8688135E-3,0E0,3.513086E-2,0E0,0E0,0E0,0E0,1.3947748E-3,5.878299E-3,4.6162993E-2,0E0,1.4519863E-2,2.2965774E-2,4.213062E-2,0E0,0E0,0E0,8.8439435E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.1899015E-2,0E0,8.030996E-3,0E0,0E0,0E0,1.034032E-2,0E0,0E0,5.3678155E-3,1.5544819E-2,1.3254747E-2,2.6118793E-3,0E0,0E0,2.4107993E-3,0E0,0E0,9.090586E-3,0E0,6.229207E-3,0E0,0E0,0E0,3.9536767E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,34,34,38,38,41,41,42,42,44,44,45,45,46,46,47,47,54,54,55,55,56,56,58,58,63,63,64,64,65,65,67,67,68,68,69,69,73,73,86,86,88,88,92,92,95,95,96,96,97,97,98,98,101,101,104,104,106,106,110,110],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,-1,-1,-1,60,-1,-1,62,64,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,74,76,78,-1,80,-1,-1,-1,-1,82,84,86,-1,88,90,92,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,96,-1,98,-1,-1,-1,100,-1,-1,102,104,106,108,-1,-1,110,-1,-1,112,-1,114,-1,-1,-1,116,-1,-1,-1,-1,-1,-1],"split_conditions":[2.85218E5,4.258E3,1.2952493E8,3.7824E4,3.453E3,3.15888E3,-6.2567457E-3,8.64E2,-1.2514751E-2,6.633279E-3,4.796E3,1.1594591E-1,1.034684E6,1.916746E6,1.3371136E9,-9.292588E-3,1.60885E6,2.54E2,1.49E3,7.8667163E3,1.3219E4,1.13161E5,1.646258E-2,5.1279686E8,3.7E1,1.06095E5,2.466E3,1E0,5.215832E10,1.038E3,2.154734E6,5.016E3,9.788418E-3,1.8627131E-2,8.1E2,-7.767547E-3,-4.592978E-3,-1.6098247E-3,1.3953088E8,-2.4813195E-3,4.336174E-3,9.89E2,1E0,2.4825535E-3,1.606E3,8.8327E4,2.05E2,1.49E2,-1.5130074E-4,1.3284225E-3,-4.325926E-3,-1.7333573E-3,1.8467447E-3,1.1070367E-2,3.51E3,3.7203822E-1,3.9972684E7,1.9083733E-3,6.6794E4,-5.799419E-3,-2.6517208E-3,-2.6899122E-3,-7.1504815E-3,5.01204E2,1.063E3,4.17E2,-5.7419008E-3,1.878288E6,8.5E1,3.134E3,-5.263496E-4,2.0576967E-3,5.6087235E-3,6.508427E9,6.8731024E-3,1.3250651E-4,2.8413804E-3,6.25504E-3,2.4798235E-3,5.688893E-3,1.0887221E-2,-1.5532292E-3,-3.720039E-3,-1.2310112E-3,4.5934922E-4,-5.072301E-3,3.1361875E8,9.166469E-5,1.9313741E3,2.6696217E-3,-7.067401E-4,5.332559E-3,8.0532E4,1.954212E-3,4.5145494E-3,8.388608E7,1.3579E4,5.019054E8,2.4292E4,7.806648E-5,2.2941E-3,1.219769E6,-3.6807244E-3,4.436817E-4,4.805687E7,-9.534639E-4,4.5787E4,-1.6779639E-3,-4.035743E-4,-1.19909E-3,9.400963E6,-2.8096985E-3,-5.660812E-4,-4.1087186E-3,-1.4926578E-3,-3.1476778E-3,-1.0951087E-3],"split_indices":[12,3,6,20,7,5,0,0,0,0,7,63,18,18,10,0,9,17,0,5,7,6,67,10,4,6,0,42,10,0,6,0,0,0,0,0,0,0,16,0,0,0,42,0,19,8,19,17,0,0,0,0,0,0,0,63,2,0,8,0,0,0,0,5,0,0,0,2,4,0,0,0,0,10,0,0,0,0,0,0,0,0,0,0,0,0,16,0,5,0,0,0,8,0,0,16,7,26,20,0,0,2,0,0,18,0,20,0,0,0,21,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.86E2,1E2,7.4E1,1.12E2,9.7E1,3E0,7E1,4E0,3E0,1.09E2,4.7E1,5E1,4.2E1,2.8E1,4E0,1.05E2,2E1,2.7E1,2.6E1,2.4E1,2.8E1,1.4E1,8E0,2E1,5.6E1,4.9E1,1E1,1E1,6E0,2.1E1,1.9E1,7E0,3E0,2.1E1,2.2E1,6E0,4E0,1E1,5E0,3E0,7E0,1.3E1,4E0,5.2E1,3.4E1,1.5E1,7E0,3E0,5E0,5E0,3E0,3E0,4E0,1.7E1,1.1E1,8E0,3E0,1.8E1,7E0,3E0,3E0,4E0,6E0,7E0,4.4E1,8E0,2.7E1,7E0,1.2E1,3E0,4E0,3E0,1E1,7E0,3E0,8E0,5E0,3E0,6E0,1.2E1,3E0,3E0,4E0,3E0,9E0,3.5E1,3E0,2.4E1,4E0,3E0,4E0,8E0,6E0,4E0,2.3E1,1.2E1,1.3E1,1.1E1,4E0,4E0,1.3E1,1E1,4E0,8E0,4E0,9E0,8E0,3E0,5E0,8E0,4E0,4E0,6E0,3E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[1.1845932E-2,-8.981595E-2,1.9432752E-1,-1.5389779E-1,-4.0604975E-2,1.5150015E-1,4.7917077E-1,-1.7241938E-1,3.434892E-2,-7.393955E-2,3.8791835E-2,1.0097102E-1,3.1512243E-1,2.341749E-2,9.269407E-3,-2.2111239E-1,-1.4245194E-1,3.8154048E-3,-1.2832743E-3,-5.6780335E-2,-9.74722E-3,8.750769E-3,-1.4436994E-2,2.3895303E-2,1.6010731E-1,4.061434E-1,1.16445474E-1,-3.7255327E-3,-2.4068978E-1,-1.431109E-3,-1.5539227E-1,4.213124E-3,-6.69846E-2,-3.7810013E-2,4.580018E-2,1.9473146E-1,-2.3018096E-2,2.649403E-1,1.3381502E-1,1.4841701E-2,4.9419873E-3,1.7449105E-3,4.6012853E-3,-8.306848E-3,-3.2938393E-3,-1.678474E-1,-9.875116E-2,-7.911659E-2,3.2736768E-3,1.0811592E-3,-5.7758734E-2,2.6835245E-3,2.119967E-4,9.808169E-3,1.1611873E-3,-1.0769193E-1,3.6633797E-2,3.5857812E-3,1.1338485E-2,4.667597E-2,1.6331804E-1,-6.085119E-3,-3.485068E-3,-9.926663E-4,-3.973758E-3,2.840483E-2,-9.072924E-2,-7.630506E-2,-4.5485157E-4,-7.4552097E-3,-3.227257E-2,-2.3526767E-2,7.508281E-2,2.6496376E-3,-1.7466348E-4,1.7422536E-1,2.0117543E-3,-1.9079818E-3,3.508045E-3,-7.307931E-2,-2.0888166E-1,-3.3587026E-3,-1.1442951E-3,8.9544215E-4,-2.3784665E-3,-1.9543984E-3,6.291197E-4,3.469375E-2,4.0877713E-3,3.4642208E-3,6.377677E-3,-8.821864E-2,1.8570456E-3,-2.237083E-3,-9.529362E-3,1.5666374E-3,3.876852E-4,-1.1945391E-1,-5.8395773E-2,-1.4065531E-1,-7.994671E-2,-4.4856977E-2,-3.9156233E-3,-2.797012E-3,-5.259506E-3,-9.655325E-2,-9.945933E-4,-5.250673E-2,-4.6453672E-4,-1.5898675E-3,-3.8490426E-3,-1.8672376E-3,-6.034752E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,-1,31,-1,-1,33,35,37,39,41,-1,43,-1,45,-1,47,49,51,53,55,57,59,-1,-1,-1,-1,-1,-1,61,63,65,-1,-1,67,-1,-1,-1,-1,69,71,-1,-1,73,75,-1,-1,-1,-1,77,79,81,-1,-1,83,85,87,-1,-1,89,-1,-1,-1,91,93,-1,-1,-1,-1,-1,-1,95,-1,-1,-1,97,-1,-1,-1,-1,-1,99,101,103,105,107,-1,-1,-1,109,-1,111,-1,-1,-1,-1,-1],"loss_changes":[5.341712E0,5.7715416E-1,1.1839118E0,2.8485262E-1,2.834141E-1,7.205415E-1,4.077587E-1,7.502413E-2,5.4706905E-2,2.7735224E-1,4.0090865E-1,3.197903E-1,3.3246565E-1,0E0,0E0,2.577877E-2,5.395788E-2,0E0,0E0,1.4473307E-1,0E0,0E0,3.974567E-2,2.6094648E-1,7.479882E-2,1.2349963E-1,5.445704E-3,0E0,2.407837E-2,0E0,1.30547285E-2,0E0,1.4720854E-1,3.0425932E-2,1.1236422E-2,1.11145884E-1,1.3526998E-1,5.4598033E-2,7.978642E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.120608E-2,1.2967169E-2,8.248803E-2,0E0,0E0,1.1868004E-2,0E0,0E0,0E0,0E0,9.532751E-2,3.8776286E-2,0E0,0E0,1.9015418E-2,1.6603053E-2,0E0,0E0,0E0,0E0,5.5915046E-2,1.0568726E-1,8.860581E-3,0E0,0E0,2.2062497E-2,1.2399336E-2,1.6560152E-2,0E0,0E0,1.3707459E-2,0E0,0E0,0E0,1.08038604E-1,6.446856E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.6297251E-3,0E0,0E0,0E0,3.6806107E-2,0E0,0E0,0E0,0E0,0E0,7.182449E-3,1.6807623E-2,1.6947091E-3,5.161844E-3,4.5954287E-3,0E0,0E0,0E0,1.6945899E-3,0E0,2.2557452E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,19,19,22,22,23,23,24,24,25,25,26,26,28,28,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,45,45,46,46,47,47,50,50,55,55,56,56,59,59,60,60,65,65,66,66,67,67,70,70,71,71,72,72,75,75,79,79,80,80,87,87,91,91,97,97,98,98,99,99,100,100,101,101,105,105,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,-1,32,-1,-1,34,36,38,40,42,-1,44,-1,46,-1,48,50,52,54,56,58,60,-1,-1,-1,-1,-1,-1,62,64,66,-1,-1,68,-1,-1,-1,-1,70,72,-1,-1,74,76,-1,-1,-1,-1,78,80,82,-1,-1,84,86,88,-1,-1,90,-1,-1,-1,92,94,-1,-1,-1,-1,-1,-1,96,-1,-1,-1,98,-1,-1,-1,-1,-1,100,102,104,106,108,-1,-1,-1,110,-1,112,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,9.10147E5,1.1741724E4,1.83E2,4.0317974E8,1.9367423E9,2.2374196E0,1.49015E5,2.302E3,1.11386E5,7.284E3,1.85155E5,1.312E3,2.341749E-2,9.269407E-3,2E1,2.9490748E-4,3.8154048E-3,-1.2832743E-3,3.308E3,-9.74722E-3,8.750769E-3,7.9E3,5.71365E5,6.2088E4,6.4401817E0,5.8882404E7,-3.7255327E-3,5.737332E9,-1.431109E-3,5.244E3,4.213124E-3,6.463662E9,5.5659E4,1.068557E6,1.1541341E9,2.511056E3,9.458E3,1.1328217E10,1.4841701E-2,4.9419873E-3,1.7449105E-3,4.6012853E-3,-8.306848E-3,-3.2938393E-3,1.3953088E8,1.37839E5,3.2416E4,3.2736768E-3,1.0811592E-3,2.2059E5,2.6835245E-3,2.119967E-4,9.808169E-3,1.1611873E-3,7.3504E4,1.138E3,3.5857812E-3,1.1338485E-2,1.3432413E10,2.308886E6,-6.085119E-3,-3.485068E-3,-9.926663E-4,-3.973758E-3,4.57473E5,1.83E2,3.067249E8,-4.5485157E-4,-7.4552097E-3,1.42E2,4.4457917E8,1.32E2,2.6496376E-3,-1.7466348E-4,1.38306E5,2.0117543E-3,-1.9079818E-3,3.508045E-3,2.379217E6,3.3012803E9,-3.3587026E-3,-1.1442951E-3,8.9544215E-4,-2.3784665E-3,-1.9543984E-3,6.291197E-4,1.15E2,4.0877713E-3,3.4642208E-3,6.377677E-3,1.833625E6,1.8570456E-3,-2.237083E-3,-9.529362E-3,1.5666374E-3,3.876852E-4,5.097E3,5.39E2,4.462E3,1.4908E4,2.4214E4,-3.9156233E-3,-2.797012E-3,-5.259506E-3,1E2,-9.945933E-4,6.3765E4,-4.6453672E-4,-1.5898675E-3,-3.8490426E-3,-1.8672376E-3,-6.034752E-4],"split_indices":[3,9,5,47,16,16,64,18,7,28,7,22,17,0,0,17,63,0,0,7,0,0,3,6,20,60,18,0,10,0,3,0,23,8,6,10,5,7,21,0,0,0,0,0,0,16,6,12,0,0,12,0,0,0,0,8,0,0,0,10,12,0,0,0,0,21,47,11,0,0,17,21,4,0,0,20,0,0,0,6,11,0,0,0,0,0,0,4,0,0,0,2,0,0,0,0,0,3,4,3,7,7,0,0,0,4,0,28,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.84E2,1.02E2,7.9E1,1.05E2,9E1,1.2E1,7.2E1,7E0,7.4E1,3.1E1,7E1,2E1,4E0,8E0,2.5E1,4.7E1,3E0,4E0,7E1,4E0,5E0,2.6E1,3.1E1,3.9E1,1.3E1,7E0,5E0,2E1,6E0,4.1E1,3E0,6.7E1,1.9E1,7E0,6E0,2.5E1,6E0,3.3E1,1E1,3E0,3E0,4E0,1.7E1,3E0,3.2E1,9E0,6.3E1,4E0,4E0,1.5E1,3E0,4E0,3E0,3E0,1E1,1.5E1,3E0,3E0,9E0,2.4E1,2.2E1,1E1,3E0,6E0,6E0,5.7E1,1E1,5E0,3E0,7E0,6E0,9E0,5E0,4E0,2.1E1,3E0,3E0,3E0,5.1E1,6E0,5E0,5E0,3E0,4E0,3E0,3E0,6E0,3E0,7E0,1.4E1,4.6E1,5E0,3E0,3E0,3E0,3E0,2.1E1,2.5E1,1.2E1,9E0,2.2E1,3E0,5E0,7E0,6E0,3E0,1.7E1,5E0,3E0,3E0,1.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-6.131802E-4,-9.274069E-2,1.9850771E-1,-1.5207708E-1,-3.365855E-2,1.517479E-1,5.332857E-1,-1.6631351E-1,-2.1771913E-2,1.1640303E-1,-4.5941014E-2,3.7813097E-1,1.15556635E-1,2.2001095E-2,6.963467E-3,-1.7406955E-1,-6.5055355E-2,4.056393E-3,-9.294546E-2,6.4513006E-4,6.687372E-3,-3.4359373E-2,-9.683517E-3,1.42916925E-2,4.888632E-3,-5.2056997E-3,1.7121142E-1,-1.9564974E-1,-1.2713924E-1,-8.1219856E-4,-2.7007437E-3,-1.5322116E-3,-3.5611389E-3,-9.292369E-3,-2.2543086E-2,7.41787E-2,-4.953407E-2,1.72482E-2,1.9181441E-1,-1.5572206E-3,-2.0321588E-1,-3.7027283E-5,-1.4163218E-1,-4.3883927E-2,9.491741E-2,3.4453638E-3,2.0435874E-4,-7.401581E-2,-7.4838656E-3,-2.4119602E-3,3.383563E-3,1.11731224E-1,2.2938395E-1,-2.1468453E-1,-4.299773E-3,-1.5209134E-1,-2.1658489E-3,-1.1806281E-1,-2.213518E-2,-5.958212E-4,1.2753493E-1,-1.0466723E-3,-2.8054614E-3,3.3587884E-4,-7.5745006E-4,5.7935715E-2,1.7262034E-1,1.0356234E-3,2.5090158E-1,-2.7056882E-1,-1.8955083E-1,-5.5046026E-3,-1.0576478E-1,-4.8126034E-3,-8.731735E-2,-3.1059459E-2,3.5591202E-3,7.6271425E-4,5.263316E-3,3.4780157E-4,2.4653852E-3,2.9242614E-3,6.79956E-3,9.579051E-3,4.173318E-3,-4.3271524E-3,-9.811752E-3,-6.519204E-3,-3.981044E-3,-3.9461744E-3,-1.7407693E-3,-3.2685841E-3,-1.1921312E-3,-6.8016715E-2,-9.2442725E-3,-9.0352274E-2,-2.0427406E-2,-2.5592573E-2,3.7455395E-2,-1.8843242E-3,-3.6631778E-3,8.1432133E-4,-1.7035161E-3,3.6070097E-4,-5.157161E-2,2.1968542E-3,1.4508456E-2,-2.8888287E-2,2.3069836E-2,-3.1898418E-4,-6.2033053E-2,1.1858467E-3,-3.685756E-4,-2.721418E-4,-1.355155E-3,1.3177487E-3,1.8920482E-5,-2.5788806E-3,-1.2957387E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,31,-1,-1,33,-1,-1,-1,35,37,39,41,-1,-1,-1,-1,-1,43,45,47,49,51,-1,53,-1,55,57,59,-1,-1,61,63,-1,-1,65,67,69,-1,71,-1,73,75,-1,77,-1,-1,-1,-1,79,81,-1,83,85,87,-1,89,-1,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,95,97,99,101,103,105,-1,-1,-1,-1,107,109,-1,111,113,115,-1,117,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.393352E0,6.995311E-1,1.3749115E0,1.8188715E-1,1.933465E-1,6.363307E-1,3.7585473E-1,5.9358597E-2,1.2741898E-1,6.625731E-2,2.7011052E-1,9.649098E-2,4.9152446E-1,0E0,0E0,5.918479E-2,3.8874634E-3,0E0,1.1499152E-3,0E0,0E0,2.6996577E-1,0E0,0E0,0E0,8.812993E-2,1.5529656E-1,5.1454306E-2,5.2789688E-2,0E0,0E0,0E0,0E0,0E0,2.2726217E-1,1.9378588E-2,1.5917465E-2,6.4573474E-2,1.0675299E-1,0E0,1.28519535E-2,0E0,9.984136E-3,1.1999728E-1,5.415681E-2,0E0,0E0,2.6190877E-3,2.2583655E-3,0E0,0E0,4.2408124E-2,1.149894E-1,9.110451E-3,0E0,1.193881E-3,0E0,3.382206E-3,7.44919E-2,0E0,3.7221268E-2,0E0,0E0,0E0,0E0,7.96482E-3,2.9154867E-3,0E0,1.08597994E-1,1.0311544E-2,3.1642914E-3,0E0,1.1985376E-3,0E0,4.107885E-3,4.5326307E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0475954E-2,2.9206607E-2,3.7822276E-3,1.3225643E-2,1.8897353E-2,6.0759364E-3,0E0,0E0,0E0,0E0,1.0629737E-2,5.487077E-3,0E0,4.4797696E-3,1.5338208E-3,3.5913335E-3,0E0,1.1072792E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,18,18,21,21,25,25,26,26,27,27,28,28,34,34,35,35,36,36,37,37,38,38,40,40,42,42,43,43,44,44,47,47,48,48,51,51,52,52,53,53,55,55,57,57,58,58,60,60,65,65,66,66,68,68,69,69,70,70,72,72,74,74,75,75,93,93,94,94,95,95,96,96,97,97,98,98,103,103,104,104,106,106,107,107,108,108,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,32,-1,-1,34,-1,-1,-1,36,38,40,42,-1,-1,-1,-1,-1,44,46,48,50,52,-1,54,-1,56,58,60,-1,-1,62,64,-1,-1,66,68,70,-1,72,-1,74,76,-1,78,-1,-1,-1,-1,80,82,-1,84,86,88,-1,90,-1,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,96,98,100,102,104,106,-1,-1,-1,-1,108,110,-1,112,114,116,-1,118,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,1.012905E6,1.2738551E4,4.579412E8,3.453E3,9.068E3,1.5860209E9,9.0417837E8,7.386E3,5.8111176E7,9.45839E5,5.105882E3,4.29906E5,2.2001095E-2,6.963467E-3,2.9736228E7,8.99687E5,4.056393E-3,6.532E3,6.4513006E-4,6.687372E-3,1.5262E4,-9.683517E-3,1.42916925E-2,4.888632E-3,4.3692328E-2,2.3595012E3,3.7E1,1.0644316E7,-8.1219856E-4,-2.7007437E-3,-1.5322116E-3,-3.5611389E-3,-9.292369E-3,1.83E2,1.3E1,2.654028E6,4.9749E4,1.6583E4,-1.5572206E-3,6.549254E6,-3.7027283E-5,4.939E3,1.530129E6,5.345E3,3.4453638E-3,2.0435874E-4,2.8E2,2.318509E6,-2.4119602E-3,3.383563E-3,1.0597183E0,2.2E1,6.6362006E2,-4.299773E-3,1.6871828E7,-2.1658489E-3,2.4702E4,4.365884E6,-5.958212E-4,6.763315E7,-1.0466723E-3,-2.8054614E-3,3.3587884E-4,-7.5745006E-4,2.2256E4,1.346289E7,1.0356234E-3,4.8778326E10,5.4E1,4.97264E5,-5.5046026E-3,7.35E2,-4.8126034E-3,1.46379E5,6.2528E4,3.5591202E-3,7.6271425E-4,5.263316E-3,3.4780157E-4,2.4653852E-3,2.9242614E-3,6.79956E-3,9.579051E-3,4.173318E-3,-4.3271524E-3,-9.811752E-3,-6.519204E-3,-3.981044E-3,-3.9461744E-3,-1.7407693E-3,-3.2685841E-3,-1.1921312E-3,5.5293235E9,6.378634E8,1.1026E4,2.7795588E-6,5.133E3,7.1E1,-1.8843242E-3,-3.6631778E-3,8.1432133E-4,-1.7035161E-3,1.333505E6,3.6E1,2.1968542E-3,6.23E2,2.57E2,1.9694447E10,-3.1898418E-4,1.8364E4,1.1858467E-3,-3.685756E-4,-2.721418E-4,-1.355155E-3,1.3177487E-3,1.8920482E-5,-2.5788806E-3,-1.2957387E-3],"split_indices":[3,9,5,16,7,7,16,11,7,16,12,5,12,0,0,23,2,0,3,0,0,8,0,0,0,63,5,20,21,0,0,0,0,0,47,19,2,20,7,0,18,0,3,9,7,0,0,17,1,0,0,59,4,5,0,27,0,28,6,0,16,0,0,0,0,18,2,0,21,17,9,0,0,0,12,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,11,16,7,63,3,4,0,0,0,0,6,19,0,17,4,10,0,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.92E2,2E2,9.2E1,9.9E1,1.01E2,8.2E1,1E1,8.9E1,1E1,7E0,9.4E1,1E1,7.2E1,6E0,4E0,8.2E1,7E0,3E0,7E0,4E0,3E0,9.1E1,3E0,7E0,3E0,2.3E1,4.9E1,5.4E1,2.8E1,3E0,4E0,3E0,4E0,3E0,8.8E1,8E0,1.5E1,6E0,4.3E1,3E0,5.1E1,3E0,2.5E1,7.5E1,1.3E1,5E0,3E0,9E0,6E0,3E0,3E0,1.5E1,2.8E1,4.2E1,9E0,2.1E1,4E0,1.6E1,5.9E1,3E0,1E1,3E0,6E0,3E0,3E0,9E0,6E0,3E0,2.5E1,1E1,3.2E1,1.3E1,8E0,6E0,1E1,5.6E1,3E0,3E0,7E0,3E0,6E0,3E0,3E0,1.7E1,8E0,3E0,7E0,2.5E1,7E0,5E0,3E0,7E0,3E0,2E1,3.6E1,1.3E1,7E0,2.7E1,9E0,7E0,6E0,3E0,4E0,1.4E1,1.3E1,3E0,6E0,6E0,8E0,3E0,1E1,3E0,3E0,3E0,3E0,4E0,4E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-2.300376E-3,-8.754219E-2,1.6314216E-1,-1.2374229E-1,9.942183E-3,8.768388E-2,2.6288393E-1,-1.1583484E-1,-1.18959965E-2,1.6635911E-1,-2.5038071E-2,2.073112E-1,3.83502E-2,3.9993787E-1,1.6685429E-1,-1.4863905E-1,-4.9863458E-2,8.544533E-3,-1.5656146E-3,-6.877711E-2,5.104841E-2,1.0355976E-1,1.0013554E-2,9.13878E-2,6.673346E-3,3.3727933E-3,4.5570138E-1,2.1741694E-1,-5.7372027E-3,1.6718482E-3,-1.5590206E-1,2.9501296E-3,-6.1334386E-2,-1.0000889E-1,-1.6198006E-2,8.496712E-2,2.4823012E-2,5.927842E-3,1.374777E-3,1.0947654E-3,1.14376485E-1,-4.28976E-2,5.2494675E-2,1.9450692E-2,2.6874465E-1,2.540663E-1,1.2810507E-1,-1.7092706E-1,-3.2456454E-2,1.530776E-2,-8.092014E-2,-1.1057906E-1,-1.1384762E-3,-3.8408685E-2,1.2175642E-3,3.4574536E-3,1.328802E-3,1.2282642E-3,3.828935E-5,1.3599243E-3,4.3823207E-3,-8.4599085E-2,-1.0466035E-3,-7.57481E-4,8.228016E-2,3.4107147E-3,1.1112406E-2,2.501759E-3,2.8878284E-1,2.288463E-3,5.1339613E-3,-9.381574E-3,-1.6074866E-1,5.0758846E-2,-4.5858514E-3,2.660203E-3,-2.5356535E-2,-9.3354665E-2,4.7784E-4,-4.121306E-3,-9.839255E-4,-2.0617924E-3,-4.7997505E-4,-3.7883718E-3,-9.771523E-4,-7.672692E-4,8.9170825E-4,3.750473E-4,3.4541814E-3,4.5828894E-3,1.0251149E-2,-1.90755E-1,-1.2572247E-1,6.1330575E-5,2.7979498E-3,-2.825393E-4,-1.1458115E-3,-6.8991214E-2,-4.508229E-3,-1.9727153E-1,-2.7173106E-3,-8.520505E-2,-1.3714993E-1,-7.607789E-2,-5.250025E-4,-6.5628546E-3,-3.0061342E-3,-3.4451382E-3,-1.2106627E-3,-4.685914E-3,-2.2300088E-3,-1.3801631E-3,-8.724424E-2,-1.2458732E-3,-9.696359E-2,-1.5527594E-3,-3.5152114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,-1,-1,33,35,37,-1,39,41,-1,43,45,-1,-1,47,-1,49,51,53,55,57,-1,-1,-1,59,61,63,-1,65,67,69,71,73,75,77,79,-1,81,-1,-1,-1,-1,-1,-1,-1,83,85,-1,87,-1,-1,-1,89,-1,-1,-1,91,93,-1,-1,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,99,101,-1,-1,-1,-1,103,-1,105,-1,107,109,111,-1,-1,-1,-1,-1,-1,-1,-1,113,-1,115,-1,-1],"loss_changes":[4.188658E0,6.929033E-1,7.3710966E-1,2.3516226E-1,2.996265E-1,3.4024727E-1,4.99202E-1,2.955675E-1,0E0,2.25952E-1,1.532687E-1,1.5082037E-1,7.163291E-2,2.32054E-1,5.098644E-1,1.4357948E-1,8.353948E-2,0E0,0E0,4.600209E-2,1.3012901E-2,4.1178644E-2,0E0,1.7214417E-2,6.585128E-2,0E0,2.2645879E-1,4.6010137E-2,0E0,0E0,1.6138148E-1,0E0,6.878795E-2,8.387655E-3,1.5851617E-2,2.4293326E-3,3.421268E-3,0E0,0E0,0E0,1.1518449E-2,2.434494E-2,3.590376E-2,0E0,6.301832E-2,7.550907E-2,4.766509E-3,5.1290274E-2,1.07931234E-1,2.9477086E-2,4.4254348E-2,1.7036289E-2,0E0,4.4680517E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.992389E-3,5.901779E-3,0E0,1.819539E-2,0E0,0E0,0E0,1.11579895E-2,0E0,0E0,0E0,5.5894256E-2,1.2201488E-2,0E0,0E0,8.759219E-4,2.6746511E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0371923E-2,5.908549E-3,0E0,0E0,0E0,0E0,7.3458627E-3,0E0,5.6928396E-3,0E0,7.3998645E-3,5.9260726E-3,3.6739185E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8518066E-3,0E0,1.2933612E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,23,23,24,24,26,26,27,27,30,30,32,32,33,33,34,34,35,35,36,36,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,61,61,62,62,64,64,68,68,72,72,73,73,76,76,77,77,91,91,92,92,97,97,99,99,101,101,102,102,103,103,112,112,114,114],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,-1,-1,34,36,38,-1,40,42,-1,44,46,-1,-1,48,-1,50,52,54,56,58,-1,-1,-1,60,62,64,-1,66,68,70,72,74,76,78,80,-1,82,-1,-1,-1,-1,-1,-1,-1,84,86,-1,88,-1,-1,-1,90,-1,-1,-1,92,94,-1,-1,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,100,102,-1,-1,-1,-1,104,-1,106,-1,108,110,112,-1,-1,-1,-1,-1,-1,-1,-1,114,-1,116,-1,-1],"split_conditions":[4.580147E6,4.3199274E8,6.6569E4,1.03199E5,9.603E3,1.457217E10,6.4401817E0,4.54205E5,-1.18959965E-2,7.031E3,1.1408653E9,1.83E2,2.37E2,4.7914E4,2.31E2,2.8156E5,4.4323E4,8.544533E-3,-1.5656146E-3,8.3552723E8,2.3E1,9.8E1,1.0013554E-2,4.0449E4,4.9749E4,3.3727933E-3,2.98E2,7.9937E4,-5.7372027E-3,1.6718482E-3,1.83E2,2.9501296E-3,2.2213E4,8.998854E7,5.581883E7,1.16E2,1.6553E4,5.927842E-3,1.374777E-3,1.0947654E-3,5.798156E2,6.1E2,6E1,1.9450692E-2,7.1E1,4E1,4.7016313E3,3.8775E2,5.347E3,1.3024E4,1.8659E4,4.3278136E9,-1.1384762E-3,1.09E2,1.2175642E-3,3.4574536E-3,1.328802E-3,1.2282642E-3,3.828935E-5,1.3599243E-3,4.3823207E-3,9.5E1,3.6392964E3,-7.57481E-4,4.365884E6,3.4107147E-3,1.1112406E-2,2.501759E-3,8.9847E4,2.288463E-3,5.1339613E-3,-9.381574E-3,3.191E3,2.005E3,-4.5858514E-3,2.660203E-3,5.072E3,6.1126E4,4.7784E-4,-4.121306E-3,-9.839255E-4,-2.0617924E-3,-4.7997505E-4,-3.7883718E-3,-9.771523E-4,-7.672692E-4,8.9170825E-4,3.750473E-4,3.4541814E-3,4.5828894E-3,1.0251149E-2,2.462322E3,3.94549E5,6.1330575E-5,2.7979498E-3,-2.825393E-4,-1.1458115E-3,4.22E2,-4.508229E-3,1.18E3,-2.7173106E-3,1.150242E6,3.0178302E7,5.40812E5,-5.250025E-4,-6.5628546E-3,-3.0061342E-3,-3.4451382E-3,-1.2106627E-3,-4.685914E-3,-2.2300088E-3,-1.3801631E-3,7.2318E4,-1.2458732E-3,5.324E3,-1.5527594E-3,-3.5152114E-3],"split_indices":[2,16,3,28,7,10,60,6,0,3,16,47,17,18,4,16,12,0,0,23,19,19,0,8,20,0,17,8,0,0,47,0,18,18,18,4,7,0,0,0,5,17,19,0,4,4,5,5,28,7,7,10,0,4,0,0,0,0,0,0,0,4,5,0,6,0,0,0,3,0,0,0,20,3,0,0,3,22,0,0,0,0,0,0,0,0,0,0,0,0,0,5,27,0,0,0,0,19,0,17,0,21,27,6,0,0,0,0,0,0,0,0,12,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.95E2,1.95E2,1E2,1.42E2,5.3E1,5.8E1,4.2E1,1.39E2,3E0,9E0,4.4E1,1.6E1,4.2E1,1.6E1,2.6E1,9.2E1,4.7E1,6E0,3E0,2.8E1,1.6E1,9E0,7E0,1.5E1,2.7E1,3E0,1.3E1,2.3E1,3E0,3E0,8.9E1,3E0,4.4E1,1.7E1,1.1E1,6E0,1E1,3E0,6E0,5E0,1E1,1.3E1,1.4E1,6E0,7E0,1.5E1,8E0,7.9E1,1E1,9E0,3.5E1,1.4E1,3E0,8E0,3E0,3E0,3E0,6E0,4E0,3E0,7E0,6E0,7E0,4E0,1E1,3E0,4E0,3E0,1.2E1,4E0,4E0,4E0,7.5E1,6E0,4E0,3E0,6E0,3.1E1,4E0,1.1E1,3E0,3E0,5E0,3E0,3E0,4E0,3E0,3E0,7E0,3E0,9E0,3.8E1,3.7E1,3E0,3E0,3E0,3E0,2.2E1,9E0,3.5E1,3E0,1E1,2.7E1,1.9E1,3E0,3.2E1,3E0,6E0,4E0,2.3E1,4E0,6E0,1.3E1,3E0,1E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[2.0688992E-2,-8.634809E-2,2.0338644E-1,-1.1535942E-1,1.8521437E-2,8.4653E-2,2.9404768E-1,-1.02664195E-1,-3.0882367E-1,7.711303E-3,-2.1030752E-2,1.1218227E-1,-7.482323E-3,2.4622677E-1,4.267543E-1,-1.4218359E-1,-4.5470435E-2,-5.227787E-3,-1.172291E-2,-5.8673188E-2,3.6935095E-2,2.3220265E-2,1.8853097E-1,3.2145494E-1,1.89072E-1,2.279573E-2,2.8562012E-1,3.140905E-3,-1.5665783E-1,-1.8998802E-2,-1.14794865E-1,-1.4061363E-1,-1.6522763E-2,2.530876E-3,1.2396336E-2,1.0887088E-1,-1.829804E-2,1.0235758E-1,2.7323306E-1,4.694879E-3,3.971011E-1,8.8276755E-4,2.0737271E-1,1.0489439E-2,4.304115E-3,-2.1540101E-1,-1.3775182E-1,-4.1442603E-2,5.4721244E-2,-6.9611385E-2,-7.134573E-3,-6.120983E-3,-1.7998784E-3,-2.8463803E-2,7.5422094E-4,-7.814718E-4,3.1581987E-2,7.670923E-4,5.365678E-3,-7.685229E-2,2.927727E-2,5.053719E-3,3.309103E-2,1.2919255E-2,4.592813E-3,5.074264E-3,1.47878E-2,2.2281055E-1,3.2529912E-3,-7.924364E-3,-4.5275576E-3,-1.425233E-1,-1.2137139E-3,-6.3063845E-2,-4.991262E-3,3.3891227E-3,1.11431E-4,-2.7407072E-3,-5.5556226E-4,-3.8367626E-4,-1.2417978E-3,2.2991034E-4,1.5491244E-3,-3.402552E-3,-9.265895E-4,-5.8486173E-4,1.803515E-3,-1.2599864E-3,3.1240266E-3,3.5654583E-3,2.3533088E-1,-1.7296029E-3,-1.4843369E-1,-7.387214E-4,-7.861645E-2,1.3032049E-3,-2.2683073E-2,3.547834E-3,8.143978E-3,-1.6373888E-1,-3.0994301E-3,-1.2405104E-3,-2.922559E-3,-3.5211552E-4,-1.8055263E-3,-1.7132364E-1,-1.8166613E-3,-5.648345E-4,5.449996E-4,-5.7099257E-3,-2.5744834E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,-1,-1,31,33,35,37,39,41,-1,43,-1,45,47,49,51,53,-1,55,57,59,61,63,-1,65,-1,67,-1,-1,69,71,73,75,77,-1,-1,-1,79,-1,-1,81,-1,-1,83,85,-1,87,-1,-1,-1,-1,89,-1,-1,-1,91,-1,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,97,-1,99,-1,101,-1,103,-1,-1,105,-1,-1,-1,107,-1,109,-1,-1,-1,-1,-1],"loss_changes":[5.4539633E0,5.375998E-1,1.0838532E0,3.0723262E-1,3.460036E-1,4.276391E-1,2.629981E-1,2.8962922E-1,5.7039857E-3,0E0,7.635427E-2,2.876839E-1,0E0,1.327064E-1,3.5380054E-1,2.825036E-1,9.8751225E-2,0E0,0E0,7.0207E-2,1.3347793E-2,7.666956E-2,1.3991106E-1,1.8118525E-1,7.7144265E-2,0E0,2.3683906E-2,0E0,5.1401734E-2,7.010467E-2,5.6989893E-2,1.8741071E-2,7.823287E-3,0E0,7.8028757E-3,3.0447967E-2,4.403608E-2,4.400973E-2,1.2672555E-1,0E0,1.0661435E-1,0E0,1.8514156E-2,0E0,0E0,2.1856427E-3,2.2313476E-2,2.4905175E-2,2.5555944E-2,8.288629E-3,0E0,0E0,0E0,1.5482288E-3,0E0,0E0,2.4865386E-3,0E0,0E0,6.6652633E-3,1.2441606E-2,0E0,3.6140654E-2,0E0,0E0,0E0,0E0,3.3926964E-3,0E0,0E0,0E0,2.0887494E-2,0E0,1.0673426E-2,1.1377845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1917114E-2,0E0,2.4936438E-2,0E0,3.3336058E-3,0E0,7.4406667E-3,0E0,0E0,2.12574E-2,0E0,0E0,0E0,2.3774556E-3,0E0,5.1994324E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,26,26,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,40,40,42,42,45,45,46,46,47,47,48,48,49,49,53,53,56,56,59,59,60,60,62,62,67,67,71,71,73,73,74,74,90,90,92,92,94,94,96,96,99,99,103,103,105,105],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,-1,-1,32,34,36,38,40,42,-1,44,-1,46,48,50,52,54,-1,56,58,60,62,64,-1,66,-1,68,-1,-1,70,72,74,76,78,-1,-1,-1,80,-1,-1,82,-1,-1,84,86,-1,88,-1,-1,-1,-1,90,-1,-1,-1,92,-1,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,98,-1,100,-1,102,-1,104,-1,-1,106,-1,-1,-1,108,-1,110,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,4.3199274E8,9.0629734E8,5.814E3,6.7995E4,4.84E2,2.51383E5,8.46944E5,4.9112267E-3,7.711303E-3,2.57951E5,1.78538E5,-7.482323E-3,2.8125507E-1,3.9527946E0,2.798795E2,5.527E3,-5.227787E-3,-1.172291E-2,6.44E4,1.1048E4,1.63E2,1.831E4,1.83E2,1.8318E4,2.279573E-2,8.4076E4,3.140905E-3,2.41247E5,1.83E2,1.266215E0,8.220982E8,8.415816E7,2.530876E-3,7.404E3,6.887E3,1E0,2.0679289E10,6.6148296E9,4.694879E-3,3.61273E5,8.8276755E-4,2.0121E4,1.0489439E-2,4.304115E-3,1.8285812E-1,6.591E3,1.146734E6,4.0061934E9,2.8125507E-1,-7.134573E-3,-6.120983E-3,-1.7998784E-3,7.5469E4,7.5422094E-4,-7.814718E-4,3.47989E5,7.670923E-4,5.365678E-3,1.28E2,5.575446E6,5.053719E-3,1.5712E4,1.2919255E-2,4.592813E-3,5.074264E-3,1.47878E-2,6.64063E5,3.2529912E-3,-7.924364E-3,-4.5275576E-3,9E0,-1.2137139E-3,1.158E3,1.2691E4,3.3891227E-3,1.11431E-4,-2.7407072E-3,-5.5556226E-4,-3.8367626E-4,-1.2417978E-3,2.2991034E-4,1.5491244E-3,-3.402552E-3,-9.265895E-4,-5.8486173E-4,1.803515E-3,-1.2599864E-3,3.1240266E-3,3.5654583E-3,2.0794068E7,-1.7296029E-3,1.1750605E8,-7.387214E-4,1.6035E4,1.3032049E-3,1.9694447E10,3.547834E-3,8.143978E-3,7.3831117E9,-3.0994301E-3,-1.2405104E-3,-2.922559E-3,2.42045E6,-1.8055263E-3,1.7156E4,-1.8166613E-3,-5.648345E-4,5.449996E-4,-5.7099257E-3,-2.5744834E-3],"split_indices":[2,16,16,0,12,4,3,9,64,0,12,22,0,63,60,5,3,0,0,8,20,17,7,47,8,0,19,0,9,47,64,16,18,0,3,3,43,10,11,0,12,0,7,0,0,66,3,6,10,63,0,0,0,8,0,0,12,0,0,4,2,0,7,0,0,0,0,12,0,0,0,17,0,0,20,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,2,0,16,0,18,0,10,0,0,10,0,0,0,2,0,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,1.75E2,1.02E2,1.37E2,3.8E1,4.5E1,5.7E1,1.3E2,7E0,5E0,3.3E1,4.2E1,3E0,4.4E1,1.3E1,7.6E1,5.4E1,3E0,4E0,2E1,1.3E1,2E1,2.2E1,1.7E1,2.7E1,3E0,1E1,4E0,7.2E1,4E1,1.4E1,6E0,1.4E1,4E0,9E0,6E0,1.4E1,1.2E1,1E1,6E0,1.1E1,3E0,2.4E1,7E0,3E0,1.5E1,5.7E1,3.1E1,9E0,1.1E1,3E0,3E0,3E0,1.1E1,3E0,3E0,6E0,3E0,3E0,6E0,8E0,6E0,6E0,4E0,6E0,3E0,8E0,2E1,4E0,9E0,6E0,5.4E1,3E0,1.9E1,1.2E1,4E0,5E0,8E0,3E0,5E0,6E0,3E0,3E0,3E0,3E0,3E0,5E0,3E0,3E0,3E0,1.7E1,4E0,5E1,6E0,1.3E1,3E0,9E0,3E0,1.4E1,3.7E1,1.3E1,4E0,9E0,6E0,3E0,3.4E1,3E0,3E0,3E0,3.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-5.143004E-5,-8.833709E-2,1.9170937E-1,-1.1313441E-1,4.6934105E-2,1.4164674E-1,4.8889124E-1,-1.6198966E-1,-7.2345056E-2,8.730568E-3,5.205292E-3,8.705658E-2,2.537776E-1,8.611695E-3,2.2614358E-2,-2.2898473E-1,-1.1674295E-1,-9.998447E-2,-3.3503275E-2,-6.9864884E-2,5.788981E-2,-5.2806423E-3,1.3850172E-1,1.2296683E-2,1.6341542E-1,-1.4655586E-1,-2.8339764E-1,-1.545627E-1,-3.8685627E-2,-9.082164E-3,-8.404884E-2,-7.47357E-2,-4.1250964E-3,-1.1552298E-3,-2.8650225E-3,2.006954E-2,7.669455E-2,3.5117425E-2,-3.6093069E-3,9.927167E-2,1.7221667E-1,8.271578E-2,7.2733457E-3,-8.489532E-2,-6.3118846E-3,-1.1573147E-2,-6.2493505E-3,-1.0310798E-3,-1.6613372E-1,1.6833303E-3,-7.0702456E-2,-1.20397255E-1,-3.6625803E-2,-6.136896E-4,-8.5303284E-2,-1.6989674E-2,2.1779488E-3,-2.5145206E-4,1.3819828E-3,9.325155E-2,1.2288403E-3,6.8948545E-2,-1.8229518E-2,1.17443815E-1,5.68192E-5,1.9406465E-1,1.5203691E-3,4.317813E-3,1.1152601E-3,-3.9641554E-3,-5.1018986E-4,-5.6701703E-3,-2.6656426E-3,-9.083593E-2,-8.6690125E-5,-6.7795314E-2,-1.4067836E-1,-1.2869682E-2,-3.403957E-3,-3.4802384E-3,-1.5142326E-3,-4.914633E-2,1.2931861E-3,3.880906E-3,1.3720173E-3,2.9715612E-3,3.482232E-4,3.9112946E-4,-1.4180106E-3,5.475952E-3,8.5708864E-2,6.7766816E-3,3.095226E-3,-1.6065013E-3,-3.5344393E-3,-4.02411E-3,-2.5413182E-2,-5.5628135E-3,-9.867209E-2,3.7154105E-2,-4.51464E-2,-1.987589E-3,-6.7945215E-4,1.8900997E-3,-1.7956479E-2,2.006101E-3,3.76291E-3,-1.367445E-3,-6.4096734E-5,-1.284331E-3,-3.7250435E-3,2.3867225E-4,2.0935638E-3,-2.1620926E-3,5.0506165E-4,-1.6370486E-3,-1.302889E-3,3.77347E-4,-3.8618944E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,-1,51,53,55,-1,-1,57,59,61,-1,63,65,67,-1,69,-1,-1,-1,-1,71,-1,73,75,77,-1,79,81,-1,-1,-1,83,-1,85,87,89,-1,91,-1,-1,-1,-1,-1,-1,-1,93,-1,95,97,99,-1,-1,-1,101,103,-1,-1,-1,-1,-1,-1,-1,105,-1,-1,-1,-1,-1,107,-1,109,111,113,-1,-1,-1,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,117,-1,-1,-1],"loss_changes":[4.994275E0,6.8289113E-1,1.3143768E0,3.290453E-1,2.9809362E-1,4.737277E-1,4.268725E-1,2.0667815E-1,9.863874E-2,0E0,1.1478548E-1,2.6674828E-1,2.3885727E-1,0E0,0E0,9.302235E-2,1.357218E-1,1.3594753E-1,4.9356382E-2,4.6098344E-3,1.0551184E-2,9.458325E-2,3.051877E-2,0E0,6.7111164E-2,2.611193E-2,4.57232E-2,4.0048122E-2,5.3217046E-2,0E0,8.522546E-2,8.374818E-3,2.4237106E-2,0E0,0E0,4.79771E-3,3.4553409E-3,3.0133488E-2,0E0,3.3458814E-2,3.9985538E-2,1.759921E-2,0E0,1.9180004E-2,0E0,0E0,0E0,0E0,9.621322E-3,0E0,1.7555788E-2,2.14006E-2,3.8049545E-2,0E0,7.135302E-3,1.29977E-2,0E0,0E0,0E0,4.5411997E-3,0E0,1.2585014E-2,6.026946E-3,1.2165591E-2,0E0,3.934324E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2825897E-3,0E0,2.1073982E-2,9.948641E-3,3.2354843E-2,0E0,0E0,0E0,1.5231296E-3,1.7635766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4679283E-3,0E0,0E0,0E0,0E0,0E0,2.713866E-3,0E0,6.4067915E-3,6.1521763E-3,1.7131185E-2,0E0,0E0,0E0,4.34386E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2479722E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,35,35,36,36,37,37,39,39,40,40,41,41,43,43,48,48,50,50,51,51,52,52,54,54,55,55,59,59,61,61,62,62,63,63,65,65,73,73,75,75,76,76,77,77,81,81,82,82,90,90,96,96,98,98,99,99,100,100,104,104,115,115],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,-1,52,54,56,-1,-1,58,60,62,-1,64,66,68,-1,70,-1,-1,-1,-1,72,-1,74,76,78,-1,80,82,-1,-1,-1,84,-1,86,88,90,-1,92,-1,-1,-1,-1,-1,-1,-1,94,-1,96,98,100,-1,-1,-1,102,104,-1,-1,-1,-1,-1,-1,-1,106,-1,-1,-1,-1,-1,108,-1,110,112,114,-1,-1,-1,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,118,-1,-1,-1],"split_conditions":[4.320722E6,4.805687E7,1.1170185E4,4.258E3,9.603E3,1.2084881E9,2.15195E5,7.325E4,1.9537698E3,8.730568E-3,7.078E3,4.9130436E-2,2.8125507E-1,8.611695E-3,2.2614358E-2,6.2468485E-4,8.64E2,4.796E3,7.939E3,9E0,8.814E3,3.5720756E7,3.51E3,1.2296683E-2,6.2901763E3,7.877782E2,6.6362006E2,1.942E3,2.1509E4,-9.082164E-3,1.374403E6,1.4556772E-5,3.43745E5,-1.1552298E-3,-2.8650225E-3,1.6198524E3,1.16E2,3.14E2,-3.6093069E-3,3.389E3,6.8536E5,2.226E4,7.2733457E-3,7.12E2,-6.3118846E-3,-1.1573147E-2,-6.2493505E-3,-1.0310798E-3,7.03651E5,1.6833303E-3,2.34E3,3.94549E5,1.71111E5,-6.136896E-4,1.0648158E-3,5.55386E5,2.1779488E-3,-2.5145206E-4,1.3819828E-3,2.2067E4,1.2288403E-3,1.4E1,4.951E3,2.305674E3,5.68192E-5,1.1408653E9,1.5203691E-3,4.317813E-3,1.1152601E-3,-3.9641554E-3,-5.1018986E-4,-5.6701703E-3,-2.6656426E-3,4.0148856E7,-8.6690125E-5,4E0,7.11736E6,2.4E2,-3.403957E-3,-3.4802384E-3,-1.5142326E-3,2.3395E4,7.1E1,3.880906E-3,1.3720173E-3,2.9715612E-3,3.482232E-4,3.9112946E-4,-1.4180106E-3,5.475952E-3,2.86E3,6.7766816E-3,3.095226E-3,-1.6065013E-3,-3.5344393E-3,-4.02411E-3,1.9E1,-5.5628135E-3,4.8E1,4.535E3,1.42894E5,-1.987589E-3,-6.7945215E-4,1.8900997E-3,2.51089E5,2.006101E-3,3.76291E-3,-1.367445E-3,-6.4096734E-5,-1.284331E-3,-3.7250435E-3,2.3867225E-4,2.0935638E-3,-2.1620926E-3,5.0506165E-4,1.25E2,-1.302889E-3,3.77347E-4,-3.8618944E-4],"split_indices":[2,18,5,3,7,16,3,18,5,0,3,63,63,0,0,64,0,7,20,19,3,23,0,0,5,5,5,3,12,0,9,64,12,0,0,5,4,17,0,0,28,18,0,20,0,0,0,0,2,0,0,27,12,0,63,18,0,0,0,22,0,19,3,5,0,16,0,0,0,0,0,0,0,16,0,19,27,17,0,0,0,18,4,0,0,0,0,0,0,0,0,0,0,0,0,0,19,0,19,3,12,0,0,0,12,0,0,0,0,0,0,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.93E2,2.01E2,9.2E1,1.7E2,3.1E1,8E1,1.2E1,7.6E1,9.4E1,4E0,2.7E1,5.5E1,2.5E1,7E0,5E0,2.9E1,4.7E1,5.4E1,4E1,1.1E1,1.6E1,2E1,3.5E1,9E0,1.6E1,1.3E1,1.6E1,3.1E1,1.6E1,3E0,5.1E1,1.6E1,2.4E1,5E0,6E0,6E0,1E1,1.5E1,5E0,1.8E1,1.7E1,8E0,8E0,7E0,6E0,7E0,9E0,3E0,2.8E1,4E0,1.2E1,2.8E1,2.3E1,3E0,1.3E1,2.1E1,3E0,3E0,3E0,6E0,4E0,9E0,6E0,1.5E1,3E0,1.4E1,3E0,3E0,5E0,4E0,3E0,2.4E1,4E0,9E0,3E0,9E0,1.9E1,1.8E1,5E0,7E0,6E0,7E0,1.4E1,3E0,3E0,6E0,3E0,3E0,3E0,4E0,1.1E1,1.1E1,3E0,4E0,5E0,3E0,6E0,9E0,1E1,7E0,1.1E1,4E0,3E0,3E0,1.1E1,8E0,3E0,3E0,3E0,3E0,7E0,4E0,3E0,8E0,3E0,7E0,4E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-8.951333E-3,-8.087678E-2,1.5927142E-1,1.4766783E-1,-9.2793375E-2,9.938638E-2,3.4294E-1,-5.9221406E-4,1.0217205E-2,-8.176106E-2,-1.0514948E-2,3.889258E-2,1.9858886E-1,2.399105E-3,4.0538377E-1,-1.08827665E-1,6.54169E-4,6.689102E-2,-7.6228525E-3,1.3613851E-1,2.4969433E-1,4.7161022E-1,2.8584404E-3,-1.5173173E-1,-7.6079465E-2,1.9967593E-2,-6.893958E-2,2.5395233E-2,2.1932581E-1,1.72789E-3,5.3698434E-3,4.4092983E-3,9.300289E-3,5.438014E-1,5.5760797E-3,2.285439E-3,-1.6542058E-1,-8.736395E-2,5.0858025E-2,2.9714666E-3,4.0155496E-3,5.519591E-4,-3.4855383E-3,5.149767E-2,-2.9597061E-3,9.916132E-3,3.3628854E-3,2.0087594E-2,8.607718E-3,-9.870812E-3,-1.4137805E-1,-9.231398E-2,6.394328E-4,-9.070325E-4,3.7718995E-3,4.017109E-2,-2.4595175E-2,-1.3041438E-2,7.596163E-2,-1.580953E-1,-8.4208906E-2,-2.1558991E-4,-9.7479634E-2,-6.828181E-4,6.684593E-2,-4.147867E-2,1.0273486E-2,-1.9750695E-3,9.083912E-4,1.1078872E-3,8.474532E-2,-5.5826167E-3,-2.663169E-3,-1.0069857E-1,-7.059851E-4,-1.0813649E-1,-5.4747693E-2,3.1599118E-3,3.939843E-2,-5.323078E-2,-5.564612E-5,-2.3781741E-4,8.1652985E-4,3.1920755E-3,1.4995331E-3,-4.098696E-3,-1.5643288E-3,-1.1349584E-1,-1.5780769E-3,-2.6582014E-3,-2.7016368E-2,4.7805268E-4,1.7412876E-3,-1.007364E-3,-2.21805E-3,-3.918122E-3,-7.625428E-2,-1.404237E-3,-1.6109477E-4,-5.945367E-4,-3.0777324E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,17,19,-1,21,23,25,27,-1,29,31,33,-1,35,37,39,41,43,45,-1,-1,-1,-1,47,-1,-1,49,51,53,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,59,61,-1,-1,-1,63,65,67,69,71,73,-1,75,-1,77,79,81,-1,-1,-1,83,-1,-1,85,-1,87,89,-1,91,93,-1,-1,-1,-1,-1,-1,-1,95,-1,-1,97,-1,-1,-1,-1,-1,99,-1,-1,-1,-1],"loss_changes":[3.3523426E0,5.420563E-1,8.7883353E-1,2.8772914E-1,4.506147E-1,3.763491E-1,3.0499434E-1,0E0,0E0,3.9713597E-1,0E0,3.323364E-1,4.2604268E-2,0E0,2.9357743E-1,1.7673028E-1,6.186077E-2,2.3479542E-1,0E0,2.393727E-2,1.3487577E-2,1.8582368E-1,0E0,1.8553519E-1,1.16172224E-1,6.37411E-2,3.5726946E-2,9.9686265E-2,4.9346775E-2,0E0,0E0,0E0,0E0,5.492544E-2,0E0,0E0,1.4000654E-1,4.0050447E-2,3.999484E-2,3.3803016E-2,0E0,0E0,0E0,4.2041577E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.2561123E-2,2.9863358E-2,0E0,0E0,0E0,2.4341663E-2,1.1611525E-2,1.7680887E-2,4.0163994E-3,2.7260482E-2,1.1141345E-2,0E0,2.3680031E-2,0E0,4.7293603E-3,5.9808716E-3,2.05342E-3,0E0,0E0,0E0,3.6351234E-3,0E0,0E0,7.6881424E-3,0E0,1.0592401E-2,9.346839E-3,0E0,1.7220331E-3,1.2822133E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.811406E-3,0E0,0E0,3.0718935E-3,0E0,0E0,0E0,0E0,0E0,1.0539837E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,33,33,36,36,37,37,38,38,39,39,43,43,50,50,51,51,55,55,56,56,57,57,58,58,59,59,60,60,62,62,64,64,65,65,66,66,70,70,73,73,75,75,76,76,78,78,79,79,87,87,90,90,96,96],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,18,20,-1,22,24,26,28,-1,30,32,34,-1,36,38,40,42,44,46,-1,-1,-1,-1,48,-1,-1,50,52,54,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,60,62,-1,-1,-1,64,66,68,70,72,74,-1,76,-1,78,80,82,-1,-1,-1,84,-1,-1,86,-1,88,90,-1,92,94,-1,-1,-1,-1,-1,-1,-1,96,-1,-1,98,-1,-1,-1,-1,-1,100,-1,-1,-1,-1],"split_conditions":[1.0259E4,3.342E3,1.83E2,4.2E1,1.17442E5,1.021304E6,7.992919E8,-5.9221406E-4,1.0217205E-2,1.027129E6,-1.0514948E-2,5.89E2,7.51842E5,2.399105E-3,1.07648E5,3.958E3,7.192367E10,6.6569E4,-7.6228525E-3,1E0,6.5E1,3.9622688E-1,2.8584404E-3,3.185029E2,2.466E3,5.710497E6,7.6565686E2,1E0,5.15297E5,1.72789E-3,5.3698434E-3,4.4092983E-3,9.300289E-3,2.3177024E7,5.5760797E-3,2.285439E-3,2.7708E4,1.1408653E9,4.352E3,1.2676195E9,4.0155496E-3,5.519591E-4,-3.4855383E-3,1.3323E4,-2.9597061E-3,9.916132E-3,3.3628854E-3,2.0087594E-2,8.607718E-3,-9.870812E-3,7.55418E5,5.9898485E-7,6.394328E-4,-9.070325E-4,3.7718995E-3,1.02E3,1.521867E6,3.069016E6,6.62E2,1.3953088E8,6E2,-2.1558991E-4,1.0901513E9,-6.828181E-4,2.4E1,1.004E3,5.501208E9,-1.9750695E-3,9.083912E-4,1.1078872E-3,6.2285466E8,-5.5826167E-3,-2.663169E-3,2.61E2,-7.059851E-4,6.44E2,1.59577E5,3.1599118E-3,3.28616E6,4.9744842E8,-5.564612E-5,-2.3781741E-4,8.1652985E-4,3.1920755E-3,1.4995331E-3,-4.098696E-3,-1.5643288E-3,1.754E4,-1.5780769E-3,-2.6582014E-3,3.4814704E8,4.7805268E-4,1.7412876E-3,-1.007364E-3,-2.21805E-3,-3.918122E-3,1E0,-1.404237E-3,-1.6109477E-4,-5.945367E-4,-3.0777324E-3],"split_indices":[3,7,47,17,8,12,23,0,0,6,0,4,18,0,20,3,10,3,0,45,4,63,0,5,0,2,5,36,12,0,0,0,0,18,0,0,8,16,3,11,0,0,0,7,0,0,0,0,0,0,2,63,0,0,0,0,6,18,0,16,17,0,25,0,19,17,11,0,0,0,1,0,0,4,0,4,12,0,2,21,0,0,0,0,0,0,0,7,0,0,21,0,0,0,0,0,42,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.93E2,8.2E1,9E0,1.84E2,6.3E1,1.9E1,5E0,4E0,1.77E2,7E0,4E1,2.3E1,4E0,1.5E1,1.33E2,4.4E1,3.7E1,3E0,1.2E1,1.1E1,1.2E1,3E0,5.6E1,7.7E1,3.5E1,9E0,3E1,7E0,4E0,8E0,4E0,7E0,9E0,3E0,3E0,5.3E1,7.1E1,6E0,3.1E1,4E0,3E0,6E0,2.5E1,5E0,3E0,4E0,6E0,3E0,6E0,4.7E1,6.8E1,3E0,3E0,3E0,1.3E1,1.8E1,7E0,1.8E1,3.5E1,1.2E1,4E0,6.4E1,4E0,9E0,1.2E1,6E0,3E0,4E0,4E0,1.4E1,2.8E1,7E0,9E0,3E0,5E1,1.4E1,3E0,6E0,9E0,3E0,3E0,3E0,9E0,5E0,5E0,4E0,4.5E1,5E0,6E0,8E0,3E0,3E0,5E0,4E0,3.5E1,1E1,4E0,4E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-4.9798824E-3,-6.788911E-2,1.4711003E-1,-9.952969E-2,2.3537626E-2,3.2602423E-1,1.185267E-1,-1.2116976E-1,-1.404578E-4,1.06183E-2,-8.201544E-3,2.9760913E-3,1.6263608E-2,4.853771E-2,2.0887044E-1,-1.9476694E-1,-9.965656E-2,3.3258747E-2,-6.7226766E-3,-3.183177E-2,1.0035856E-1,8.9709036E-2,-1.4280048E-1,2.744789E-1,1.2610704E-1,-8.14354E-3,-4.6878457E-3,-8.91259E-2,-8.938927E-3,-6.7633283E-3,3.835349E-3,-1.0341106E-1,-1.4053359E-2,4.847145E-3,-2.2332638E-6,4.9289983E-2,9.128446E-3,-1.157847E-2,1.9082177E-3,4.249129E-3,3.2284224E-1,3.9441424E-4,1.5143847E-1,6.8543516E-2,-1.025998E-1,1.9443365E-2,-2.009147E-3,-1.7050955E-3,-3.9618346E-3,-7.287573E-2,6.8487446E-3,1.924044E-2,5.513636E-3,1.1581336E-2,5.1178588E-3,5.8886562E-3,9.830264E-2,-3.937806E-5,3.5616534E-3,-1.3297926E-1,-7.6326355E-2,1.3566285E-3,-6.036665E-3,-9.866293E-4,-3.2357997E-3,-2.1065697E-2,5.361842E-2,-1.1331485E-2,8.9443706E-2,1.2351647E-3,4.30229E-3,-1.4879528E-1,-3.029243E-3,5.730416E-4,-8.508897E-2,-7.180154E-4,3.7796597E-4,1.4412663E-2,-4.172777E-2,1.02772465E-4,2.4068747E-3,-5.5249058E-2,3.3719238E-2,3.8101303E-3,9.955368E-4,-5.00595E-3,-2.3479622E-3,6.0347497E-4,-9.295227E-2,1.3117682E-3,-4.9989315E-4,-2.2387605E-3,-2.2372442E-4,-3.10002E-3,-3.806663E-4,7.190884E-5,4.6893794E-2,-1.2800032E-1,-6.646114E-2,2.3306303E-3,3.1092874E-4,-2.072973E-3,-1.4913373E-1,-8.004572E-2,-1.0270659E-3,-2.162407E-3,-5.439543E-3,-3.4495627E-3,-5.7555072E-2,-1.0084568E-3,-2.528743E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,-1,45,-1,47,49,-1,-1,51,-1,-1,-1,-1,53,-1,55,57,59,61,-1,-1,-1,63,65,67,-1,-1,-1,-1,69,-1,-1,71,73,-1,75,-1,-1,77,79,81,83,-1,-1,85,-1,-1,87,-1,-1,89,91,-1,-1,93,95,-1,-1,-1,-1,-1,97,-1,-1,-1,-1,-1,-1,-1,99,101,103,-1,-1,-1,105,107,-1,-1,-1,-1,109,-1,-1],"loss_changes":[2.7463946E0,5.895802E-1,3.909304E-1,3.2475233E-1,5.180172E-1,4.1377068E-1,4.577688E-1,1.7692053E-1,2.0212698E-1,0E0,1.2889428E-1,0E0,0E0,3.5154933E-1,1.3914561E-1,4.0394068E-2,1.5899295E-1,8.674559E-2,0E0,5.072195E-2,4.5406863E-2,2.6813436E-1,3.7198114E-1,7.4521065E-2,4.4289768E-2,0E0,0E0,2.0672822E-1,0E0,2.7466707E-2,0E0,1.4172196E-3,4.2302486E-2,0E0,0E0,1.1035138E-1,0E0,0E0,0E0,0E0,1.5445232E-2,0E0,3.7768483E-3,2.363551E-2,5.9291244E-2,7.7743735E-3,0E0,0E0,0E0,7.4261464E-3,3.5073794E-2,5.6816E-2,0E0,0E0,0E0,0E0,9.7029805E-3,0E0,0E0,9.973824E-3,4.1930676E-2,0E0,2.2866959E-3,0E0,0E0,1.3063369E-2,1.0428749E-2,3.945481E-2,9.879187E-3,0E0,0E0,1.9136071E-3,0E0,0E0,3.848514E-2,0E0,0E0,6.1535845E-3,1.0160258E-2,0E0,0E0,1.6689666E-2,4.0432857E-3,0E0,0E0,0E0,0E0,0E0,3.0488372E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.9497626E-3,1.3245285E-2,9.650297E-3,0E0,0E0,0E0,8.173406E-3,4.8112124E-3,0E0,0E0,0E0,0E0,4.149031E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,31,31,32,32,35,35,40,40,42,42,43,43,44,44,45,45,49,49,50,50,51,51,56,56,59,59,60,60,62,62,65,65,66,66,67,67,68,68,71,71,74,74,77,77,78,78,81,81,82,82,88,88,96,96,97,97,98,98,102,102,103,103,108,108],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,-1,46,-1,48,50,-1,-1,52,-1,-1,-1,-1,54,-1,56,58,60,62,-1,-1,-1,64,66,68,-1,-1,-1,-1,70,-1,-1,72,74,-1,76,-1,-1,78,80,82,84,-1,-1,86,-1,-1,88,-1,-1,90,92,-1,-1,94,96,-1,-1,-1,-1,-1,98,-1,-1,-1,-1,-1,-1,-1,100,102,104,-1,-1,-1,106,108,-1,-1,-1,-1,110,-1,-1],"split_conditions":[1.0773E4,3.1361875E8,6.281E3,1.460109E6,1.5469E4,3.9E1,9.31278E5,1.2590216E8,5.96025E5,1.06183E-2,1.83E2,2.9760913E-3,1.6263608E-2,4.9125667E10,9.869779E9,7.325E4,2.1997E5,3.718604E6,-6.7226766E-3,4.951E3,1.1268727E3,1.6208463E7,3.516E3,3.30553E5,2.6E1,-8.14354E-3,-4.6878457E-3,3.185029E2,-8.938927E-3,2.32877E5,3.835349E-3,4.371E3,1.1561E4,4.847145E-3,-2.2332638E-6,7.9503E4,9.128446E-3,-1.157847E-2,1.9082177E-3,4.249129E-3,6.57852E5,3.9441424E-4,3.4752463E10,3.7E1,3.3473062E9,3.6815E6,-2.009147E-3,-1.7050955E-3,-3.9618346E-3,1.1055E4,7.1E1,2.7307752E7,5.513636E-3,1.1581336E-2,5.1178588E-3,5.8886562E-3,6.2901763E3,-3.937806E-5,3.5616534E-3,1.0151622E8,3.919652E9,1.3566285E-3,2.402988E6,-9.866293E-4,-3.2357997E-3,6.437E3,1.7283224E7,3.435974E10,6.9585E4,1.2351647E-3,4.30229E-3,1.7372E4,-3.029243E-3,5.730416E-4,1.0758E4,-7.180154E-4,3.7796597E-4,6.4999E4,2.5425295E9,1.02772465E-4,2.4068747E-3,9.310384E6,1.669E3,3.8101303E-3,9.955368E-4,-5.00595E-3,-2.3479622E-3,6.0347497E-4,2.77E2,1.3117682E-3,-4.9989315E-4,-2.2387605E-3,-2.2372442E-4,-3.10002E-3,-3.806663E-4,7.190884E-5,3.4134E4,5.87E2,3.8786424E7,2.3306303E-3,3.1092874E-4,-2.072973E-3,7.83E2,1.1234E4,-1.0270659E-3,-2.162407E-3,-5.439543E-3,-3.4495627E-3,2.6304E4,-1.0084568E-3,-2.528743E-3],"split_indices":[3,16,7,6,8,4,12,10,12,0,47,0,0,10,11,18,22,6,0,3,5,2,0,28,19,0,0,5,0,12,0,3,7,0,0,8,0,0,0,0,22,0,21,4,10,2,0,0,0,7,19,18,0,0,0,0,5,0,0,16,10,0,6,0,0,3,27,13,8,0,0,7,0,0,8,0,0,8,10,0,0,18,0,0,0,0,0,0,17,0,0,0,0,0,0,0,3,28,16,0,0,0,0,20,0,0,0,0,20,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,2.02E2,8.3E1,1.5E2,5.2E1,1E1,7.3E1,1.23E2,2.7E1,4E0,4.8E1,5E0,5E0,4.2E1,3.1E1,2.6E1,9.7E1,2.4E1,3E0,4E1,8E0,3.5E1,7E0,1.6E1,1.5E1,1E1,1.6E1,9.3E1,4E0,1.7E1,7E0,7E0,3.3E1,5E0,3E0,3E1,5E0,3E0,4E0,5E0,1.1E1,3E0,1.2E1,7E0,8.6E1,1.2E1,5E0,3E0,4E0,8E0,2.5E1,2.5E1,5E0,8E0,3E0,6E0,6E0,3E0,4E0,3.8E1,4.8E1,6E0,6E0,4E0,4E0,1.6E1,9E0,1.8E1,7E0,3E0,3E0,2.5E1,1.3E1,4E0,4.4E1,3E0,3E0,6E0,1E1,3E0,6E0,9E0,9E0,4E0,3E0,2.2E1,3E0,3E0,4.1E1,3E0,3E0,5E0,5E0,4E0,5E0,3E0,6E0,1.6E1,2.5E1,3E0,3E0,5E0,1.1E1,1.7E1,8E0,3E0,8E0,6E0,1.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-3.5760193E-3,-5.458542E-2,1.9873452E-1,-1.16143875E-1,6.9118673E-3,1.311885E-1,3.1713524E-1,-1.4625297E-1,-5.7538453E-2,3.0298772E-1,-1.2342233E-2,7.973208E-2,1.8393272E-1,1.8524927E-1,1.3857228E-2,9.1995514E-4,-1.566517E-1,1.8840022E-2,-9.938994E-2,4.5534093E-3,1.2514095E-2,-2.7837152E-2,1.13936886E-1,4.222941E-3,5.0401885E-2,6.392646E-3,2.670289E-3,2.4465031E-3,6.9758478E-3,-1.9418365E-1,-1.112007E-1,-2.7030293E-2,5.24236E-2,-2.2174498E-1,-2.1569923E-2,3.2122897E-3,-3.5858396E-2,6.5550013E-3,6.1408605E-2,9.9375444E-5,6.7326546E-2,-2.1926944E-1,-4.595721E-3,-1.1975572E-1,-5.5040285E-4,-1.6514035E-3,1.2876869E-4,2.6528484E-3,-1.8248489E-4,-1.2294105E-2,-9.85297E-2,-4.8808943E-2,2.5480501E-3,-7.2668344E-2,-4.841212E-3,2.7046953E-3,3.9048493E-4,1.267841E-3,3.4240393E-3,-4.9044266E-3,-8.092214E-3,-4.036888E-3,-1.6165768E-3,-4.2203623E-3,-1.3298821E-3,-2.120703E-3,-4.8486708E-4,-1.2369687E-1,-3.650861E-2,1.6471142E-2,-1.411576E-1,-1.5964957E-1,-2.0373643E-3,3.138256E-3,-6.148404E-2,6.6836946E-2,-1.3621609E-2,-1.0715043E-3,-6.879999E-3,-3.0887476E-3,-7.4420753E-3,-7.552428E-4,1.2795777E-3,-3.565526E-4,-8.338941E-2,1.258234E-1,1.29640885E-2,-4.059239E-2,1.0829175E-2,-3.2182122E-3,-9.4517577E-4,5.036285E-3,1.8048919E-3,-1.3808598E-3,1.8462173E-3,-2.2954488E-2,-2.5412708E-3,2.5350899E-3,-1.0904555E-2,-3.8911175E-4,-1.0807819E-3,-2.0589477E-3,1.140984E-2,1.3616846E-3,-7.873284E-3,2.789848E-4,-7.2249223E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,31,33,-1,-1,35,37,-1,39,-1,-1,-1,-1,41,43,45,47,49,51,-1,53,-1,55,-1,57,59,-1,61,-1,-1,-1,-1,-1,-1,63,65,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,71,73,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,-1,-1,-1,89,91,93,95,97,-1,-1,-1,-1,-1,-1,99,-1,-1,101,-1,-1,-1,103,-1,105,-1,-1],"loss_changes":[2.9210145E0,8.563659E-1,4.14253E-1,1.9097972E-1,6.5361637E-1,8.846772E-2,2.192657E-1,1.3901198E-1,1.2973464E-1,4.199916E-2,2.1449365E-1,2.5083706E-2,1.0179281E-2,2.1341473E-2,0E0,0E0,9.781706E-2,2.4524365E-2,2.4278042E-1,0E0,0E0,1.02883846E-1,4.403889E-2,0E0,1.1803906E-2,0E0,0E0,0E0,0E0,1.641202E-2,2.5323719E-2,5.4777153E-3,1.6147701E-2,1.5974715E-1,5.050761E-2,0E0,1.0468837E-1,0E0,9.011265E-3,0E0,7.810414E-3,1.1135459E-2,0E0,5.8944523E-3,0E0,0E0,0E0,0E0,0E0,0E0,7.632509E-3,7.0740506E-3,0E0,7.298149E-2,1.5049142E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6813209E-2,2.593809E-2,6.911447E-2,4.7689542E-2,2.3387074E-2,0E0,1.1646319E-2,1.674331E-2,5.239088E-2,1.9563826E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.923124E-3,8.322641E-3,2.7258746E-2,7.6590795E-3,2.4479833E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.6370157E-4,0E0,0E0,1.6121803E-2,0E0,0E0,0E0,6.2901992E-3,0E0,1.8817277E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,21,21,22,22,24,24,29,29,30,30,31,31,32,32,33,33,34,34,36,36,38,38,40,40,41,41,43,43,50,50,51,51,53,53,54,54,67,67,68,68,69,69,70,70,71,71,73,73,74,74,75,75,76,76,84,84,85,85,86,86,87,87,88,88,95,95,98,98,102,102,104,104],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,32,34,-1,-1,36,38,-1,40,-1,-1,-1,-1,42,44,46,48,50,52,-1,54,-1,56,-1,58,60,-1,62,-1,-1,-1,-1,-1,-1,64,66,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,72,74,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,-1,-1,-1,90,92,94,96,98,-1,-1,-1,-1,-1,-1,100,-1,-1,102,-1,-1,-1,104,-1,106,-1,-1],"split_conditions":[1.9921019E9,4.995E3,1.09E2,3.4585872E8,3.719E3,7.1057E4,2.15195E5,2.8156E5,1.49E2,1.5165665E7,1.6429902E9,3.8248E4,3.9461447E11,6.378634E8,1.3857228E-2,9.1995514E-4,6.100984E6,1.7183E4,9.4202E4,4.5534093E-3,1.2514095E-2,6.2222845E2,6.0741784E7,4.222941E-3,3.3333E4,6.392646E-3,2.670289E-3,2.4465031E-3,6.9758478E-3,1.0016946E7,1.83E2,1.53E2,4.264E3,8.751819E2,5.9031E4,3.2122897E-3,2.004803E6,6.5550013E-3,8.58E2,9.9375444E-5,7.392E3,3.2189274E-3,-4.595721E-3,1.369E3,-5.5040285E-4,-1.6514035E-3,1.2876869E-4,2.6528484E-3,-1.8248489E-4,-1.2294105E-2,1E0,2.34E3,2.5480501E-3,2.96E2,4.348455E10,2.7046953E-3,3.9048493E-4,1.267841E-3,3.4240393E-3,-4.9044266E-3,-8.092214E-3,-4.036888E-3,-1.6165768E-3,-4.2203623E-3,-1.3298821E-3,-2.120703E-3,-4.8486708E-4,1.5202E4,1.9118268E3,2.44E2,2.2128516E7,2.9743274E8,-2.0373643E-3,6.6320845E-3,7.147E4,2.056506E3,3.044519E6,-1.0715043E-3,-6.879999E-3,-3.0887476E-3,-7.4420753E-3,-7.552428E-4,1.2795777E-3,-3.565526E-4,1.746735E6,1.7257E4,1.7234E4,1.0773E4,1.98315E5,-3.2182122E-3,-9.4517577E-4,5.036285E-3,1.8048919E-3,-1.3808598E-3,1.8462173E-3,2.51089E5,-2.5412708E-3,2.5350899E-3,2.6868372E3,-3.8911175E-4,-1.0807819E-3,-2.0589477E-3,1.15E2,1.3616846E-3,1.138E3,2.789848E-4,-7.2249223E-4],"split_indices":[27,3,4,11,7,8,3,16,17,18,16,18,1,16,0,0,18,22,18,0,0,5,18,0,3,0,0,0,0,26,47,4,3,5,28,0,9,0,0,0,0,66,0,17,0,0,0,0,0,0,19,0,0,17,10,0,0,0,0,0,0,0,0,0,0,0,0,7,5,17,23,11,0,63,8,5,2,0,0,0,0,0,0,0,2,7,28,3,12,0,0,0,0,0,0,12,0,0,5,0,0,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,2.25E2,5.6E1,1.12E2,1.13E2,3.7E1,1.9E1,7.3E1,3.9E1,6E0,1.07E2,2E1,1.7E1,1E1,9E0,4E0,6.9E1,1.4E1,2.5E1,3E0,3E0,9.6E1,1.1E1,6E0,1.4E1,1.4E1,3E0,3E0,7E0,3.6E1,3.3E1,6E0,8E0,9E0,1.6E1,5E0,9.1E1,3E0,8E0,4E0,1E1,2.2E1,1.4E1,3E1,3E0,3E0,3E0,5E0,3E0,3E0,6E0,1.3E1,3E0,4.1E1,5E1,5E0,3E0,7E0,3E0,9E0,1.3E1,2.7E1,3E0,3E0,3E0,8E0,5E0,1.6E1,2.5E1,4.4E1,6E0,9E0,7E0,1E1,1.5E1,1.6E1,2.8E1,3E0,3E0,6E0,3E0,6E0,4E0,5E0,1E1,7E0,9E0,1.3E1,1.5E1,7E0,3E0,4E0,3E0,4E0,5E0,1E1,3E0,3E0,1.2E1,6E0,4E0,3E0,9E0,3E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[1.1652295E-2,-7.050981E-2,1.595914E-1,-1.20698534E-1,-2.0100191E-2,1.1274818E-1,2.7200392E-1,-1.7513162E-1,-8.820718E-2,9.204129E-2,-5.4565154E-2,7.3355325E-2,2.8891864E-1,3.1480995E-1,1.15014045E-4,-1.8557076E-1,-1.7077909E-3,3.1806442E-3,-1.0360512E-1,2.2962044E-1,-4.1743033E-2,-4.3959506E-2,-7.1208375E-3,9.409888E-2,-7.910923E-3,1.2627501E-2,3.3133116E-3,1.4775765E-1,3.8473117E-1,-6.5609743E-3,-4.141942E-3,-1.2023203E-1,-4.6324026E-2,9.889408E-3,4.5509473E-3,-9.635458E-2,9.311499E-4,-6.9615334E-2,2.5869492E-2,7.285032E-2,1.1676285E-2,7.6720202E-3,-8.0665865E-4,2.837522E-1,1.8358335E-2,-1.2672831E-1,-8.638016E-4,-5.7246715E-2,-1.5146889E-4,-6.592806E-4,-4.768438E-3,-6.6846656E-3,-5.638487E-2,-3.0505955E-2,6.964309E-2,-7.9360325E-3,1.12366274E-1,1.2964497E-2,1.861075E-1,-1.3659991E-1,-6.20867E-2,-6.2853197E-4,-2.4467392E-3,-6.776333E-2,-8.132201E-4,-2.536632E-3,7.691132E-4,4.2356525E-4,3.6862886E-3,-2.326362E-3,8.643974E-3,-1.2757769E-4,1.2658694E-1,2.2877997E-3,7.46071E-3,-1.4691845E-1,-7.728558E-2,-2.9238532E-3,-5.7353295E-4,-4.1003567E-3,-5.30631E-2,9.411107E-4,-6.6667225E-4,-1.27523225E-2,3.7975255E-2,1.7066379E-3,1.3783047E-1,-2.5000717E-3,-1.5558499E-1,-3.3024077E-3,-1.0511418E-3,-6.435525E-2,-2.0655625E-2,-1.0486285E-3,1.897171E-4,2.0124072E-3,1.2676482E-4,2.8522548E-3,1.5308827E-1,-5.459066E-3,-3.1975165E-3,-3.1035868E-2,-7.3737495E-2,-1.1749602E-3,-1.2900497E-4,2.7950332E-3,5.6421296E-3,-2.2939698E-6,-1.7459774E-3,-1.0031508E-3,-2.580733E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,31,33,35,37,-1,39,-1,-1,-1,41,43,-1,-1,45,47,-1,-1,49,-1,51,53,55,-1,-1,-1,57,-1,59,-1,61,-1,-1,-1,-1,63,65,67,69,71,-1,73,75,77,-1,-1,79,81,-1,-1,-1,-1,-1,83,-1,85,-1,-1,87,89,-1,-1,-1,91,-1,-1,93,95,-1,97,-1,99,-1,-1,101,103,-1,-1,-1,-1,-1,105,-1,-1,107,109,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4389594E0,4.5803285E-1,5.01174E-1,1.4549518E-1,3.6059785E-1,4.8541582E-1,3.3209848E-1,3.22088E-2,1.8013999E-1,4.1451555E-1,1.15766406E-1,4.269275E-1,2.0955956E-1,2.351656E-1,0E0,4.9328804E-3,0E0,0E0,4.6812832E-2,3.189242E-2,4.910052E-2,1.23624325E-1,0E0,3.0465442E-1,0E0,0E0,0E0,1.4686607E-1,1.5677476E-1,0E0,0E0,2.2081852E-2,6.0948115E-3,0E0,0E0,2.446913E-2,0E0,7.9616666E-2,4.9011894E-2,1.76471E-1,0E0,0E0,0E0,7.611203E-2,0E0,1.6405523E-2,0E0,6.301839E-3,0E0,0E0,0E0,0E0,2.966392E-2,2.5529703E-2,2.6376527E-2,2.0891855E-2,6.171027E-2,0E0,3.080985E-2,8.842528E-3,7.2898604E-3,0E0,0E0,2.7421638E-2,5.9869965E-3,0E0,0E0,0E0,0E0,0E0,1.0525566E-2,0E0,1.9988596E-2,0E0,0E0,4.120171E-3,4.556518E-3,0E0,0E0,0E0,1.0747053E-2,0E0,0E0,3.8886014E-3,5.601472E-3,0E0,7.4629188E-3,0E0,1.277566E-3,0E0,0E0,5.2141547E-3,2.4917903E-3,0E0,0E0,0E0,0E0,0E0,1.41887665E-2,0E0,0E0,5.026008E-3,2.4341643E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,18,18,19,19,20,20,21,21,23,23,27,27,28,28,31,31,32,32,35,35,37,37,38,38,39,39,43,43,45,45,47,47,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,63,63,64,64,70,70,72,72,75,75,76,76,80,80,83,83,84,84,86,86,88,88,91,91,92,92,98,98,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,32,34,36,38,-1,40,-1,-1,-1,42,44,-1,-1,46,48,-1,-1,50,-1,52,54,56,-1,-1,-1,58,-1,60,-1,62,-1,-1,-1,-1,64,66,68,70,72,-1,74,76,78,-1,-1,80,82,-1,-1,-1,-1,-1,84,-1,86,-1,-1,88,90,-1,-1,-1,92,-1,-1,94,96,-1,98,-1,100,-1,-1,102,104,-1,-1,-1,-1,-1,106,-1,-1,108,110,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.91E3,1.042754E6,1.83E2,9.4202E4,9.068E3,2.308886E6,9.762892E9,2.2728746E-1,1.6028E4,8.1695386E8,6.9052E4,5.11E2,5.70167E5,7.6E1,1.15014045E-4,1.2995494E3,-1.7077909E-3,3.1806442E-3,4.0200136E7,4.41E3,1.2126E4,3.987307E6,-7.1208375E-3,1.0778861E4,-7.910923E-3,1.2627501E-2,3.3133116E-3,1.3432413E10,2.2497124E7,-6.5609743E-3,-4.141942E-3,1.253E3,1.5618E4,9.889408E-3,4.5509473E-3,2.367437E6,9.311499E-4,3.6267E4,4.07729E5,3.69126E5,1.1676285E-2,7.6720202E-3,-8.0665865E-4,5.019054E8,1.8358335E-2,8.24E2,-8.638016E-4,1.664E3,-1.5146889E-4,-6.592806E-4,-4.768438E-3,-6.6846656E-3,2.34E3,6.6E1,1.06454085E-4,9.8927E4,1.306E4,1.2964497E-2,2.317698E7,1E0,8.845961E2,-6.2853197E-4,-2.4467392E-3,4.09372E5,8.751819E2,-2.536632E-3,7.691132E-4,4.2356525E-4,3.6862886E-3,-2.326362E-3,1.5419E4,-1.2757769E-4,6.4762153E9,2.2877997E-3,7.46071E-3,4.1677676E7,4.445E3,-2.9238532E-3,-5.7353295E-4,-4.1003567E-3,2.2317708E3,9.411107E-4,-6.6667225E-4,6.12383E8,4.838464E7,1.7066379E-3,4.4276085E0,-2.5000717E-3,1.6395413E7,-3.3024077E-3,-1.0511418E-3,2.4702E4,1.3316E4,-1.0486285E-3,1.897171E-4,2.0124072E-3,1.2676482E-4,2.8522548E-3,2.8E1,-5.459066E-3,-3.1975165E-3,1.756E4,1.04345E5,-1.1749602E-3,-1.2900497E-4,2.7950332E-3,5.6421296E-3,-2.2939698E-6,-1.7459774E-3,-1.0031508E-3,-2.580733E-3],"split_indices":[3,9,47,18,7,12,11,63,12,11,7,4,20,17,0,5,0,0,18,7,28,2,0,5,0,0,0,10,2,0,0,17,28,0,0,2,0,8,18,12,0,0,0,26,0,0,0,20,0,0,0,0,0,4,63,12,7,0,18,36,5,0,0,6,5,0,0,0,0,0,7,0,10,0,0,10,3,0,0,0,5,0,0,16,18,0,60,0,27,0,0,28,7,0,0,0,0,0,17,0,0,7,12,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.81E2,1E2,9E1,9.1E1,7.2E1,2.8E1,3.2E1,5.8E1,2.1E1,7E1,6E1,1.2E1,2.4E1,4E0,2.9E1,3E0,4E0,5.4E1,1E1,1.1E1,6.7E1,3E0,5.7E1,3E0,7E0,5E0,8E0,1.6E1,2E1,9E0,4.1E1,1.3E1,4E0,6E0,6E0,5E0,4.9E1,1.8E1,5.4E1,3E0,5E0,3E0,1.2E1,4E0,3.8E1,3E0,1E1,3E0,3E0,3E0,3E0,4.6E1,8E0,1E1,1.8E1,3.6E1,4E0,8E0,3.2E1,6E0,4E0,6E0,3.8E1,8E0,4E0,4E0,5E0,5E0,3E0,1.5E1,4E0,3.2E1,3E0,5E0,2.6E1,6E0,3E0,3E0,6E0,3.2E1,3E0,5E0,9E0,6E0,5E0,2.7E1,4E0,2.2E1,3E0,3E0,2.3E1,9E0,4E0,5E0,3E0,3E0,8E0,1.9E1,1.6E1,6E0,6E0,1.7E1,4E0,5E0,6E0,1.3E1,3E0,3E0,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-7.7311276E-3,-7.374925E-2,1.408937E-1,-1.192894E-1,-2.5700191E-2,1.1014506E-1,2.800656E-1,-1.5262453E-1,-8.3074555E-2,6.2804837E-3,-3.670392E-2,4.7712512E-2,1.4484708E-1,1.3789596E-2,1.9492282E-1,-1.9618851E-1,-1.2911895E-1,-5.7073675E-2,-1.6715476E-1,-4.7896758E-2,4.5998776E-3,1.9216605E-1,-6.984798E-3,-1.13227E-3,1.5735239E-1,2.183489E-1,3.009134E-3,-7.069667E-3,-3.0902189E-3,-3.6378313E-4,-4.4708755E-3,-3.0291433E-2,-1.1448857E-1,-5.4532368E-2,-1.0104746E-2,-6.8252166E-3,-4.0069494E-2,9.254403E-3,2.4934774E-3,-4.669105E-3,2.2449812E-2,2.7082664E-1,1.2633607E-1,8.299586E-3,3.3645749E-3,-7.537251E-2,8.61601E-3,-9.185745E-4,-1.6464122E-1,-2.660106E-3,-1.8554814E-4,-3.213347E-2,-6.523436E-3,2.4769097E-3,3.1136048E-3,1.0292457E-2,4.175977E-3,-3.9638724E-4,1.4301006E-1,-3.0690022E-3,-6.552512E-4,-2.3561649E-2,5.7950046E-2,-6.26728E-3,-2.7651596E-3,-4.213517E-2,4.5168023E-3,-1.5153296E-2,1.9041653E-3,7.783644E-3,1.2564833E-1,3.0688106E-4,-3.7164137E-2,3.117993E-3,1.4637227E-4,-5.1084574E-2,6.2327575E-2,-1.6056818E-3,2.539158E-4,6.770473E-4,1.3610587E-1,-1.9572835E-4,-1.7574572E-3,-8.905213E-2,-3.5256524E-2,3.2484299E-3,2.6252482E-4,4.3994994E-4,-5.115933E-4,1.4911633E-3,1.4913908E-1,-1.0609857E-1,-3.7367135E-2,5.605065E-3,-4.218356E-2,5.054788E-3,2.2661497E-3,-3.7771945E-3,-1.3849278E-3,2.8779483E-4,-2.392711E-3,2.7621768E-3,-1.5708194E-3,-5.0863314E-2,-2.1645097E-2,-5.7141874E-2,-1.376079E-4,-7.290473E-3,-1.4277217E-3,-2.5333245E-3,-3.4705695E-2,4.037756E-3,-9.052924E-4,-1.5190089E-3,4.6821806E-4,5.3720654E-4,-4.1156582E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,29,31,33,35,-1,37,39,-1,41,43,-1,-1,-1,-1,-1,45,47,49,-1,-1,51,-1,-1,-1,53,55,57,-1,-1,59,61,-1,63,-1,-1,65,-1,67,-1,-1,-1,-1,69,-1,-1,71,73,-1,-1,75,-1,77,-1,-1,79,-1,81,-1,-1,83,85,-1,87,-1,89,-1,-1,91,93,-1,-1,-1,-1,-1,95,97,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,105,107,109,-1,111,-1,-1,113,115,-1,-1,-1,-1,-1],"loss_changes":[2.9343464E0,4.4988358E-1,3.628521E-1,1.1528683E-1,2.5499782E-1,1.5991211E-1,1.18563175E-1,2.9964209E-2,1.0632628E-1,0E0,2.0385428E-1,2.3047762E-1,1.158458E-1,0E0,8.103937E-3,1.8839955E-2,4.219687E-2,6.1115354E-2,1.812293E-1,1.10986546E-1,0E0,6.5211385E-2,9.266405E-2,0E0,1.2768435E-1,1.3503641E-2,0E0,0E0,0E0,0E0,0E0,5.3269103E-2,5.1340774E-2,1.0489786E-2,0E0,0E0,1.0961728E-1,0E0,0E0,0E0,2.7948247E-2,2.064401E-2,8.961171E-2,0E0,0E0,1.3441175E-2,2.8352115E-2,0E0,2.2099614E-3,0E0,0E0,1.5597382E-1,0E0,1.688459E-2,0E0,0E0,0E0,0E0,2.7938902E-2,0E0,0E0,5.3062886E-3,1.4106801E-2,0E0,0E0,8.077896E-2,0E0,6.9688093E-3,0E0,0E0,3.1051636E-2,0E0,4.003293E-3,0E0,0E0,4.313369E-2,1.3810357E-2,0E0,2.3832035E-3,0E0,2.5854707E-2,0E0,0E0,1.54183805E-2,1.608722E-2,0E0,0E0,0E0,0E0,0E0,3.8651228E-3,6.30033E-3,1.2647377E-2,4.346018E-2,7.45596E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.408474E-3,4.892368E-3,1.1296466E-2,0E0,2.7096749E-3,0E0,0E0,1.0524673E-2,1.9161317E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24,25,25,31,31,32,32,33,33,36,36,40,40,41,41,42,42,45,45,46,46,48,48,51,51,53,53,58,58,61,61,62,62,65,65,67,67,70,70,72,72,75,75,76,76,78,78,80,80,83,83,84,84,90,90,91,91,92,92,93,93,94,94,103,103,104,104,105,105,107,107,110,110,111,111],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,30,32,34,36,-1,38,40,-1,42,44,-1,-1,-1,-1,-1,46,48,50,-1,-1,52,-1,-1,-1,54,56,58,-1,-1,60,62,-1,64,-1,-1,66,-1,68,-1,-1,-1,-1,70,-1,-1,72,74,-1,-1,76,-1,78,-1,-1,80,-1,82,-1,-1,84,86,-1,88,-1,90,-1,-1,92,94,-1,-1,-1,-1,-1,96,98,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,106,108,110,-1,112,-1,-1,114,116,-1,-1,-1,-1,-1],"split_conditions":[1.0773E4,4.667E3,2.264E5,7.16E2,3.719E3,4.37849E5,4.7957E4,2.1159E4,2.2561E4,6.2804837E-3,1.1822208E10,5.71365E5,5.598581E7,1.3789596E-2,2.4066515E11,5.08868E5,1.52E2,1.6525185E9,3.648E3,4.692E3,4.5998776E-3,4.833E3,1.0917633E9,-1.13227E-3,3.6734461E9,1E0,3.009134E-3,-7.069667E-3,-3.0902189E-3,-3.6378313E-4,-4.4708755E-3,2.703815E8,7.6565686E2,1.783E3,-1.0104746E-2,-6.8252166E-3,6.79417E5,9.254403E-3,2.4934774E-3,-4.669105E-3,1.7833586E9,2.8125507E-1,2.1602268E9,8.299586E-3,3.3645749E-3,1.352E3,1.06E3,-9.185745E-4,4.014E3,-2.660106E-3,-1.8554814E-4,4.365884E6,-6.523436E-3,6.007078E6,3.1136048E-3,1.0292457E-2,4.175977E-3,-3.9638724E-4,5.0075E4,-3.0690022E-3,-6.552512E-4,4E1,6.023E3,-6.26728E-3,-2.7651596E-3,1.0737418E9,4.5168023E-3,9.8927E4,1.9041653E-3,7.783644E-3,8.2E1,3.0688106E-4,8.897305E-4,3.117993E-3,1.4637227E-4,3.28451E5,9.21E3,-1.6056818E-3,5.2582994E9,6.770473E-4,2.878E3,-1.9572835E-4,-1.7574572E-3,7.482E3,4.77915E5,3.2484299E-3,2.6252482E-4,4.3994994E-4,-5.115933E-4,1.4911633E-3,5.4686732E10,1.1422082E7,8.01E3,5E1,4.576E4,5.054788E-3,2.2661497E-3,-3.7771945E-3,-1.3849278E-3,2.8779483E-4,-2.392711E-3,2.7621768E-3,-1.5708194E-3,1E0,2.6973452E10,1.0038519E10,-1.376079E-4,8.5867E4,-1.4277217E-3,-2.5333245E-3,1.3141365E7,7.09E2,-9.052924E-4,-1.5190089E-3,4.6821806E-4,5.3720654E-4,-4.1156582E-4],"split_indices":[3,3,3,0,7,12,18,18,20,0,25,6,16,0,26,2,28,11,0,7,0,7,10,0,21,36,0,0,0,0,0,11,5,19,0,0,12,0,0,0,16,63,27,0,0,0,0,0,3,0,0,6,0,6,0,0,0,0,20,0,0,4,28,0,0,16,0,12,0,0,19,0,64,0,0,6,3,0,21,0,0,0,0,3,6,0,0,0,0,0,21,27,3,4,28,0,0,0,0,0,0,0,0,34,10,10,0,8,0,0,18,17,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.97E2,2.06E2,9.1E1,1.05E2,1.01E2,7.6E1,1.5E1,5.3E1,5.2E1,4E0,9.7E1,2.8E1,4.8E1,4E0,1.1E1,1.6E1,3.7E1,4.1E1,1.1E1,9.2E1,5E0,7E0,2.1E1,3E0,4.5E1,8E0,3E0,1.2E1,4E0,3E0,3.4E1,2.9E1,1.2E1,7E0,4E0,3E0,8.9E1,3E0,4E0,3E0,1.8E1,8E0,3.7E1,5E0,3E0,1.3E1,1.6E1,5E0,7E0,4E0,3E0,8.6E1,3E0,1.5E1,3E0,5E0,3E0,4E0,3.3E1,9E0,4E0,1E1,6E0,4E0,3E0,8.2E1,4E0,1.2E1,3E0,3E0,3E1,3E0,7E0,3E0,3E0,7.6E1,6E0,3E0,9E0,3E0,2.7E1,3E0,4E0,2.1E1,5.5E1,3E0,3E0,5E0,4E0,4E0,2.3E1,1.5E1,6E0,8E0,4.7E1,2E1,3E0,1.2E1,3E0,3E0,3E0,3E0,5E0,3.2E1,1.5E1,2.8E1,4E0,1E1,5E0,1.3E1,1.5E1,7E0,3E0,1.2E1,3E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[1.6152328E-3,-3.199611E-2,2.0744433E-1,-6.850899E-2,4.705722E-2,3.7400606E-1,1.3136898E-1,-8.346587E-2,8.9615576E-2,2.3176618E-2,2.2756664E-1,1.6753746E-2,4.531695E-3,2.3020122E-2,1.6479725E-1,-7.433975E-2,-9.533948E-3,1.8069543E-1,-4.028932E-2,4.154645E-2,-9.344849E-3,1.0011469E-2,3.173771E-3,1.6960171E-3,-1.7122361E-4,1.2947413E-1,7.6420833E-3,-1.4764567E-1,-5.0538406E-2,1.5165387E-3,7.7135707E-3,4.275556E-4,-2.6970804E-3,1.9834634E-2,5.7702954E-3,5.3706546E-3,5.2247915E-2,-1.6855755E-1,-9.768175E-2,1.0357219E-1,-6.1588373E-2,-4.087972E-2,5.9759907E-2,6.512669E-4,2.2918936E-3,-5.8058375E-3,-2.8199737E-3,-5.1445287E-4,-3.881775E-3,3.8033098E-4,6.1923484E-3,-7.823271E-2,5.4565683E-2,9.988035E-3,-1.4386374E-1,8.291704E-3,1.9541573E-2,-6.431965E-2,-1.0075832E-2,1.0402766E-1,-9.757385E-3,-2.0817262E-3,3.7935145E-2,-1.6845302E-3,-7.1559846E-3,-2.8997671E-3,3.911917E-2,2.6327746E-3,-7.387666E-2,1.6145436E-3,4.6789045E-3,-1.8198215E-3,1.2701815E-3,-3.018611E-3,2.7752558E-3,-1.191691E-3,5.0179355E-2,-8.4534764E-2,-1.9952965E-3,4.5841554E-4,-7.673513E-4,2.5261963E-2,2.8616455E-3,-5.7074115E-2,-1.00009196E-1,2.5351413E-3,-3.7663996E-2,1.6372954E-3,-3.03285E-4,-6.834184E-2,-2.883126E-2,-1.0739344E-1,-1.0584359E-3,-1.8259672E-3,-3.562894E-4,-8.6516497E-4,5.6213303E-4,-2.589122E-3,-1.2923087E-3,-1.1724583E-3,-2.6831866E-4,-1.216805E-1,-2.2901925E-3,-2.4081306E-3,-4.399817E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,35,-1,37,39,-1,-1,-1,-1,41,-1,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,63,-1,65,67,-1,69,71,-1,73,-1,-1,-1,75,-1,77,-1,-1,-1,-1,79,-1,-1,81,83,85,-1,-1,87,-1,89,91,-1,93,-1,95,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,103,-1,-1,-1],"loss_changes":[1.9714041E0,7.1040773E-1,4.577167E-1,4.0480822E-1,3.3095828E-1,3.3665895E-1,1.0124189E-1,2.7694464E-1,1.8475589E-1,4.1517308E-1,6.620169E-2,0E0,0E0,7.0067607E-3,2.5691569E-2,2.525103E-1,0E0,5.9568107E-2,1.7425492E-2,1.9677977E-1,0E0,0E0,0E0,0E0,0E0,4.034832E-2,0E0,2.1149278E-2,2.0051137E-1,0E0,0E0,0E0,0E0,1.4546117E-1,0E0,0E0,2.8069858E-3,7.142842E-3,2.2406906E-2,6.291288E-2,2.1113276E-1,1.2716249E-1,2.8140324E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.869298E-1,4.555668E-2,3.792442E-2,4.5809403E-2,0E0,7.0057526E-2,1.3171688E-1,0E0,1.0520384E-2,1.834706E-2,0E0,2.5965795E-2,0E0,0E0,0E0,2.4595302E-2,0E0,6.495151E-2,0E0,0E0,0E0,0E0,3.2143742E-3,0E0,0E0,2.0910673E-2,2.5737405E-2,3.7530765E-2,0E0,0E0,1.0488242E-2,0E0,7.2583705E-3,1.9270211E-2,0E0,3.9348733E-3,0E0,4.718664E-3,3.5899282E-3,1.2525972E-3,1.2573838E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.467483E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,25,25,27,27,28,28,33,33,36,36,37,37,38,38,39,39,40,40,41,41,42,42,51,51,52,52,53,53,54,54,56,56,57,57,59,59,60,60,62,62,66,66,68,68,73,73,76,76,77,77,78,78,81,81,83,83,84,84,86,86,88,88,89,89,90,90,91,91,101,101],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,36,-1,38,40,-1,-1,-1,-1,42,-1,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,64,-1,66,68,-1,70,72,-1,74,-1,-1,-1,76,-1,78,-1,-1,-1,-1,80,-1,-1,82,84,86,-1,-1,88,-1,90,92,-1,94,-1,96,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,104,-1,-1,-1],"split_conditions":[1.2084881E9,7.503E3,2.2397E4,6.490568E8,1.0358873E11,2.844E3,1.3321924E9,1.17442E5,5.5659E4,6.7975234E10,3.9010383E11,1.6753746E-2,4.531695E-3,3.3473062E9,8.375186E11,4.16E2,-9.533948E-3,1.741E4,1.40566E5,1.6208463E7,-9.344849E-3,1.0011469E-2,3.173771E-3,1.6960171E-3,-1.7122361E-4,2.1370164E7,7.6420833E-3,3.2072206E7,1.1799E4,1.5165387E-3,7.7135707E-3,4.275556E-4,-2.6970804E-3,7.281275E-1,5.7702954E-3,5.3706546E-3,2.1798E4,5.14524E5,2.753E3,3.1E1,1.914442E6,2.5582984E8,9.099E3,6.512669E-4,2.2918936E-3,-5.8058375E-3,-2.8199737E-3,-5.1445287E-4,-3.881775E-3,3.8033098E-4,6.1923484E-3,9.9838E4,1.83E2,2.036E3,1.52E3,8.291704E-3,1.878288E6,3.185029E2,-1.0075832E-2,4.223761E7,6.763315E7,-2.0817262E-3,1.83E2,-1.6845302E-3,-7.1559846E-3,-2.8997671E-3,3.0371344E-1,2.6327746E-3,2.466E3,1.6145436E-3,4.6789045E-3,-1.8198215E-3,1.2701815E-3,3.2353126E9,2.7752558E-3,-1.191691E-3,8.956271E6,7.973437E-4,2.5635E4,4.5841554E-4,-7.673513E-4,7.1E1,2.8616455E-3,1.0995E4,1.8364E4,2.5351413E-3,3.385204E7,1.6372954E-3,3.323062E3,1.0014492E8,1.014046E6,2.3933843E8,-1.0584359E-3,-1.8259672E-3,-3.562894E-4,-8.6516497E-4,5.6213303E-4,-2.589122E-3,-1.2923087E-3,-1.1724583E-3,-2.6831866E-4,1.245E4,-2.2901925E-3,-2.4081306E-3,-4.399817E-3],"split_indices":[16,3,8,16,23,0,21,8,8,23,13,0,0,10,13,0,0,8,12,2,0,0,0,0,0,6,0,25,8,0,0,0,0,64,0,0,7,2,3,17,6,27,7,0,0,0,0,0,0,0,0,28,17,22,0,0,2,5,0,21,16,0,47,0,0,0,63,0,0,0,0,0,0,10,0,0,2,63,8,0,0,4,0,20,7,0,16,0,5,16,6,16,0,0,0,0,0,0,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.44E2,3.9E1,1.67E2,7.7E1,1.1E1,2.8E1,1.53E2,1.4E1,6.9E1,8E0,6E0,5E0,7E0,2.1E1,1.48E2,5E0,8E0,6E0,6.6E1,3E0,4E0,4E0,3E0,4E0,1.6E1,5E0,3.5E1,1.13E2,3E0,5E0,3E0,3E0,5.8E1,8E0,1E1,6E0,2.3E1,1.2E1,7E0,1.06E2,2.3E1,3.5E1,3E0,3E0,1.9E1,4E0,3E0,9E0,4E0,3E0,9.3E1,1.3E1,1.6E1,7E0,5E0,3E1,8.9E1,4E0,7E0,6E0,4E0,1.2E1,4E0,3E0,4E0,2.6E1,5E0,8.4E1,4E0,3E0,3E0,3E0,7E0,5E0,3E0,2.3E1,7.3E1,1.1E1,4E0,3E0,1.5E1,8E0,2.8E1,4.5E1,3E0,8E0,7E0,8E0,1.9E1,9E0,4E1,5E0,4E0,4E0,3E0,5E0,1.2E1,7E0,6E0,3E0,2.7E1,1.3E1,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[7.512956E-4,-6.0404684E-2,1.3671777E-1,-9.7407125E-2,9.865707E-3,9.455193E-2,3.3636677E-1,-1.7139807E-1,-7.5952165E-2,1.2562332E-2,-1.20125245E-2,2.252871E-2,1.45731E-1,1.6676536E-2,2.0667033E-1,-6.248359E-3,-3.4298222E-3,-6.794036E-2,-7.3200017E-3,4.0979516E-3,-1.2166108E-1,-3.5605258E-3,4.396808E-2,2.2405443E-1,9.649002E-2,2.5223582E-3,7.886012E-3,2.1422664E-3,-7.54052E-2,-1.5036453E-2,5.9611313E-3,-2.0520438E-3,-5.733257E-3,6.093202E-2,-2.6243832E-3,9.433612E-3,5.1427325E-3,1.3722618E-1,2.395027E-2,-9.0197675E-2,-4.0047586E-2,-3.043851E-2,5.2139293E-2,4.907172E-3,3.8387824E-2,7.3017396E-2,2.1638608E-1,6.0917445E-2,-7.4581226E-4,1.5704408E-3,-9.725443E-2,-2.0988634E-2,-3.7326892E-3,1.7904141E-3,-3.9737675E-2,2.614837E-4,7.613075E-2,-6.5755364E-3,2.8338046E-3,3.907165E-3,2.1130834E-2,3.6392084E-3,8.233022E-3,9.0061926E-4,2.530902E-3,-1.3371429E-1,-7.849549E-2,-5.0486203E-2,6.027358E-4,-7.337736E-2,-2.7320273E-2,3.2416226E-3,1.0468741E-3,-6.4806006E-4,4.0548414E-4,1.20579934E-4,1.0697343E-3,-1.5065567E-1,-2.1325678E-3,-3.4005765E-2,-9.150638E-2,-5.421927E-4,-2.437154E-3,1.9901738E-2,-1.6618904E-2,-2.0353914E-4,-3.1458994E-3,-4.5091107E-2,7.5362795E-3,-5.210188E-3,-2.4646865E-3,1.2594282E-3,-6.2730394E-2,-1.0016871E-1,-6.2764063E-4,1.1986512E-3,6.606503E-5,-1.0704788E-5,-7.952813E-4,-6.4860925E-2,-2.3211185E-2,8.640393E-4,-1.0088712E-2,-7.855411E-4,-2.5051453E-3,-4.5327074E-4,-1.0888942E-1,-7.944848E-4,-2.4734903E-3,-9.996019E-4,-1.9378244E-4,-8.29487E-4,2.6118298E-4,-1.193734E-1,-1.627235E-3,-1.8693982E-3,-4.1595073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,-1,-1,27,-1,29,31,-1,33,35,37,-1,-1,-1,39,41,-1,-1,-1,43,-1,-1,-1,45,47,49,51,53,55,-1,57,59,61,63,-1,-1,65,67,-1,-1,69,-1,71,73,-1,-1,75,-1,-1,-1,-1,77,79,81,83,85,87,-1,-1,-1,-1,-1,-1,89,-1,91,93,-1,-1,95,97,-1,-1,99,101,-1,-1,-1,103,105,-1,-1,-1,-1,-1,107,109,-1,111,-1,-1,-1,113,-1,-1,-1,-1,-1,-1,115,-1,-1,-1],"loss_changes":[2.5194376E0,5.443977E-1,7.595228E-1,2.0446563E-1,6.1223334E-1,2.8791732E-1,2.7001393E-1,2.3509026E-2,1.1560017E-1,0E0,1.24152884E-1,1.0191578E-1,1.5579015E-1,0E0,3.4888417E-2,0E0,0E0,1.0943127E-1,0E0,2.1762225E-1,1.8067867E-2,0E0,6.883636E-2,2.8603196E-2,8.536312E-2,0E0,0E0,0E0,4.8167467E-2,6.056179E-2,0E0,0E0,0E0,4.9853556E-2,0E0,0E0,0E0,8.077252E-2,2.2314206E-2,7.2419286E-2,4.2418312E-2,3.982395E-2,1.0997621E-2,0E0,5.192347E-2,2.7548864E-2,3.6402047E-3,1.8832199E-3,0E0,0E0,3.4644604E-2,1.6589645E-2,0E0,0E0,1.5957162E-2,0E0,4.226513E-3,3.6327145E-3,0E0,0E0,1.3482631E-3,0E0,0E0,0E0,0E0,1.4020026E-2,2.3515403E-2,8.060554E-3,5.6485124E-3,1.7346054E-2,2.078101E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.907616E-4,0E0,2.7662624E-2,1.9878238E-2,0E0,0E0,2.3990911E-3,1.1772616E-3,0E0,0E0,8.010864E-3,4.3541193E-3,0E0,0E0,0E0,3.3078343E-3,2.1739662E-2,0E0,0E0,0E0,0E0,0E0,3.3990033E-3,1.431E-3,0E0,2.2071041E-3,0E0,0E0,0E0,1.1090249E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.3797235E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,17,17,19,19,20,20,22,22,23,23,24,24,28,28,29,29,33,33,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,50,50,51,51,54,54,56,56,57,57,60,60,65,65,66,66,67,67,68,68,69,69,70,70,77,77,79,79,80,80,83,83,84,84,87,87,88,88,92,92,93,93,99,99,100,100,102,102,106,106,113,113],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,-1,-1,28,-1,30,32,-1,34,36,38,-1,-1,-1,40,42,-1,-1,-1,44,-1,-1,-1,46,48,50,52,54,56,-1,58,60,62,64,-1,-1,66,68,-1,-1,70,-1,72,74,-1,-1,76,-1,-1,-1,-1,78,80,82,84,86,88,-1,-1,-1,-1,-1,-1,90,-1,92,94,-1,-1,96,98,-1,-1,100,102,-1,-1,-1,104,106,-1,-1,-1,-1,-1,108,110,-1,112,-1,-1,-1,114,-1,-1,-1,-1,-1,-1,116,-1,-1,-1],"split_conditions":[4.320722E6,2.917845E8,2.18449E5,3.0859E5,3.719E3,2.7729868E3,1.5860209E9,1.916746E6,1.2101E4,1.2562332E-2,8.89E2,1.4387648E-6,1.5581422E10,1.6676536E-2,6.5E1,-6.248359E-3,-3.4298222E-3,3.185029E2,-7.3200017E-3,2.601E3,1.6919283E8,-3.5605258E-3,2.305674E3,2.8125507E-1,4.62E2,2.5223582E-3,7.886012E-3,2.1422664E-3,6.63188E5,1.1156849E9,5.9611313E-3,-2.0520438E-3,-5.733257E-3,2.160042E9,-2.6243832E-3,9.433612E-3,5.1427325E-3,5.0600757E3,5.5659E4,1.0916E4,2.12834E5,1E0,4.77E2,4.907172E-3,4.365884E6,1.7100842E0,4.69698E5,3.9652E4,-7.4581226E-4,1.5704408E-3,4.7914E4,2.0446E6,-3.7326892E-3,1.7904141E-3,4.951E3,2.614837E-4,2.325128E6,1.914442E6,2.8338046E-3,3.907165E-3,5.6E1,3.6392084E-3,8.233022E-3,9.0061926E-4,2.530902E-3,1.7387058E3,2.09153E5,1.7707E4,1.17E2,1.0098E5,4.838464E7,3.2416226E-3,1.0468741E-3,-6.4806006E-4,4.0548414E-4,1.20579934E-4,1.0697343E-3,1.6404E4,-2.1325678E-3,9.710135E2,7.503E3,-5.421927E-4,-2.437154E-3,7.143424E7,1.08E2,-2.0353914E-4,-3.1458994E-3,3.644E4,1.56455E5,-5.210188E-3,-2.4646865E-3,1.2594282E-3,1.0203876E3,3.9818033E-7,-6.2764063E-4,1.1986512E-3,6.606503E-5,-1.0704788E-5,-7.952813E-4,6.208E3,5.3163274E8,8.640393E-4,1.878288E6,-7.855411E-4,-2.5051453E-3,-4.5327074E-4,2.1811952E7,-7.944848E-4,-2.4734903E-3,-9.996019E-4,-1.9378244E-4,-8.29487E-4,2.6118298E-4,2.748E3,-1.627235E-3,-1.8693982E-3,-4.1595073E-3],"split_indices":[2,16,3,9,7,5,16,18,0,0,19,63,10,0,4,0,0,5,0,0,21,0,5,63,17,0,0,0,6,16,0,0,0,11,0,0,0,5,8,8,12,45,0,0,6,64,28,8,0,0,18,9,0,0,3,0,2,6,0,0,19,0,0,0,0,5,18,28,4,12,18,0,0,0,0,0,0,7,0,5,3,0,0,16,17,0,0,28,12,0,0,0,5,63,0,0,0,0,0,3,16,0,2,0,0,0,18,0,0,0,0,0,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[3.01E2,2.08E2,9.3E1,1.36E2,7.2E1,7.8E1,1.5E1,2.9E1,1.07E2,3E0,6.9E1,3.3E1,4.5E1,5E0,1E1,2E1,9E0,1.03E2,4E0,6.1E1,8E0,4E0,2.9E1,1.6E1,2.9E1,3E0,7E0,5E0,9.8E1,5.6E1,5E0,5E0,3E0,2.6E1,3E0,6E0,1E1,1.8E1,1.1E1,6.8E1,3E1,4.6E1,1E1,4E0,2.2E1,1.1E1,7E0,6E0,5E0,3E0,6.5E1,2.5E1,5E0,4E0,4.2E1,4E0,6E0,1.2E1,1E1,5E0,6E0,3E0,4E0,3E0,3E0,2E1,4.5E1,1E1,1.5E1,1E1,3.2E1,3E0,3E0,7E0,5E0,3E0,3E0,1.5E1,5E0,1.1E1,3.4E1,5E0,5E0,7E0,8E0,3E0,7E0,2.1E1,1.1E1,1.2E1,3E0,3E0,8E0,3E1,4E0,3E0,4E0,3E0,5E0,1E1,1.1E1,5E0,6E0,3E0,5E0,3E0,2.7E1,3E0,7E0,7E0,4E0,3E0,3E0,2.2E1,5E0,4E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-3.00053E-3,-4.1909028E-2,1.9691585E-1,-9.301986E-2,7.360038E-3,5.3853247E-2,2.6905674E-1,-8.208514E-2,-8.713443E-3,-8.729675E-3,1.3551158E-1,7.132138E-2,3.730932E-5,3.2355395E-1,1.5046549E-1,-1.0876257E-1,-3.534831E-2,1.05580455E-2,-1.9109005E-1,2.3449173E-4,1.7325112E-1,4.2821778E-4,8.918262E-2,3.9303347E-1,3.460618E-3,1.7776367E-3,5.770757E-3,2.0608789E-4,-1.1426097E-1,5.301816E-3,-1.0345198E-1,-1.9320266E-2,1.2356772E-1,-1.7496922E-4,-1.2092017E-2,2.9030542E-3,7.669881E-3,3.4032443E-3,1.3541991E-3,6.36842E-3,1.37859145E-2,-3.0983106E-4,-1.1924919E-1,-3.4399595E-2,9.892994E-2,-6.783325E-3,-4.5282457E-2,-4.0498227E-2,4.6345104E-2,4.250741E-2,9.837715E-3,-1.5029961E-1,-9.787973E-2,5.7664607E-4,-5.4362766E-2,4.1101207E-3,1.231253E-3,-2.729085E-3,5.6827193E-5,-6.236775E-2,-1.800775E-2,-8.077253E-4,7.365117E-2,-5.8663264E-3,1.1388311E-1,-5.139796E-3,-2.1194508E-3,-1.0088169E-3,-1.0458878E-1,-7.144775E-2,1.9994582E-4,-6.9377355E-2,-3.9056216E-5,-3.9975613E-2,-3.185533E-3,1.0781585E-3,2.6252663E-3,1.5817375E-1,-4.584237E-4,-1.8180816E-3,-1.1259436E-1,-3.43127E-3,-7.8508275E-4,-5.3754024E-2,-3.3510122E-3,3.606573E-5,-5.379879E-2,-2.4187263E-2,3.6272757E-2,5.9119137E-3,2.5878893E-3,-1.222224E-1,-7.204295E-2,-2.1528982E-2,-2.23552E-3,-7.3670223E-4,-2.1064559E-3,-1.0585368E-3,1.1601292E-4,1.6266763E-3,4.1659267E-4,-4.1939463E-3,-1.7169205E-3,-1.0083605E-3,-3.3775247E-3,-1.1308392E-3,2.7557857E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,27,29,31,33,-1,35,-1,37,39,-1,-1,-1,-1,41,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,51,53,55,-1,57,59,61,63,-1,65,67,-1,69,-1,-1,-1,-1,71,73,-1,75,-1,77,-1,-1,-1,79,81,-1,83,-1,85,87,-1,-1,89,-1,-1,91,-1,-1,93,-1,-1,95,97,99,-1,-1,101,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1006546E0,5.6979406E-1,4.3961966E-1,1.9343567E-1,2.4036586E-1,1.4644839E-2,1.387403E-1,1.2831819E-1,0E0,3.678184E-1,6.1647743E-2,1.1248894E-2,0E0,2.429477E-1,2.0284161E-2,4.3157518E-2,1.11285396E-1,3.231585E-1,3.4061512E-1,0E0,3.252271E-2,0E0,2.5670528E-3,1.1585236E-2,0E0,0E0,0E0,0E0,3.2256722E-2,1.0004953E-1,8.406606E-2,1.0756694E-1,2.821593E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5764048E-2,2.081659E-2,9.075634E-3,0E0,2.0592399E-2,2.7404547E-2,3.8911458E-2,2.6138434E-1,0E0,7.699251E-3,1.4132351E-2,0E0,1.4932584E-2,0E0,0E0,0E0,0E0,1.23472065E-2,9.64103E-3,0E0,1.7133802E-3,0E0,7.034941E-2,0E0,0E0,0E0,7.119924E-3,1.55950375E-2,0E0,8.510321E-3,0E0,6.8772137E-3,1.6622493E-2,0E0,0E0,3.0762255E-3,0E0,0E0,2.5172234E-3,0E0,0E0,8.620817E-3,0E0,0E0,1.741048E-3,3.260943E-3,1.6752379E-3,0E0,0E0,5.2785575E-3,7.424768E-3,2.4659869E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,22,22,23,23,28,28,29,29,30,30,31,31,32,32,42,42,43,43,44,44,46,46,47,47,48,48,49,49,51,51,52,52,54,54,59,59,60,60,62,62,64,64,68,68,69,69,71,71,73,73,74,74,77,77,80,80,83,83,86,86,87,87,88,88,91,91,92,92,93,93],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,28,30,32,34,-1,36,-1,38,40,-1,-1,-1,-1,42,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,52,54,56,-1,58,60,62,64,-1,66,68,-1,70,-1,-1,-1,-1,72,74,-1,76,-1,78,-1,-1,-1,80,82,-1,84,-1,86,88,-1,-1,90,-1,-1,92,-1,-1,94,-1,-1,96,98,100,-1,-1,102,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2084881E9,5.013E3,4.580147E6,4.5794E4,3.48037E5,1.6121E4,2.2799654E0,1.039E3,-8.713443E-3,4.556168E10,5.6E1,4.89E2,3.730932E-5,1E0,2.1213516E7,2.302E3,1.0112496E8,1.83E2,5.96025E5,2.3449173E-4,7.4399194E8,4.2821778E-4,2.9E1,1.9E1,3.460618E-3,1.7776367E-3,5.770757E-3,2.0608789E-4,3.7E1,1.7028203E8,3.757E3,4.320722E6,8.2E1,-1.7496922E-4,-1.2092017E-2,2.9030542E-3,7.669881E-3,3.4032443E-3,1.3541991E-3,6.36842E-3,1.37859145E-2,-3.0983106E-4,6.059E3,1.746735E6,7.51885E5,-6.783325E-3,1.434897E6,1.618921E6,1.306E4,4.796E3,9.837715E-3,1.18E3,8.178743E-4,5.7664607E-4,6.723994E7,4.1101207E-3,1.231253E-3,-2.729085E-3,5.6827193E-5,5.89E2,1.7452324E3,-8.077253E-4,2.2256E4,-5.8663264E-3,5.7593933E9,-5.139796E-3,-2.1194508E-3,-1.0088169E-3,7.211698E7,2.61E2,1.9994582E-4,2.305674E3,-3.9056216E-5,5.236E3,5.3163274E8,1.0781585E-3,2.6252663E-3,3.4555535E9,-4.584237E-4,-1.8180816E-3,2.8809104E8,-3.43127E-3,-7.8508275E-4,8.383E3,-3.3510122E-3,3.606573E-5,1.6111E4,2.6304E4,7.52585E5,5.9119137E-3,2.5878893E-3,2.3933843E8,1.3731765E3,6.532E3,-2.23552E-3,-7.3670223E-4,-2.1064559E-3,-1.0585368E-3,1.1601292E-4,1.6266763E-3,4.1659267E-4,-4.1939463E-3,-1.7169205E-3,-1.0083605E-3,-3.3775247E-3,-1.1308392E-3,2.7557857E-5],"split_indices":[16,3,2,7,22,7,67,0,0,10,19,0,0,36,2,7,16,47,12,0,16,0,19,17,0,0,0,0,20,23,3,2,4,0,0,0,0,0,0,0,0,0,22,2,18,0,2,9,7,7,0,17,64,0,16,0,0,0,0,4,5,0,18,0,10,0,0,0,10,4,0,5,0,3,16,0,0,10,0,0,11,0,0,20,0,0,20,20,6,0,0,16,5,3,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,2.25E2,4.3E1,1.1E2,1.15E2,1.5E1,2.8E1,1.05E2,5E0,1.03E2,1.2E1,1.1E1,4E0,1.8E1,1E1,6.6E1,3.9E1,9.4E1,9E0,3E0,9E0,3E0,8E0,1.3E1,5E0,3E0,7E0,3E0,6.3E1,2.5E1,1.4E1,7.5E1,1.9E1,5E0,4E0,5E0,4E0,5E0,3E0,3E0,1E1,3E0,6E1,1.8E1,7E0,4E0,1E1,5.7E1,1.8E1,1.4E1,5E0,2.2E1,3.8E1,5E0,1.3E1,4E0,3E0,5E0,5E0,2.8E1,2.9E1,5E0,1.3E1,3E0,1.1E1,1.9E1,3E0,4E0,3.4E1,1E1,3E0,2.5E1,3E0,1.1E1,1.8E1,3E0,1E1,8E0,3E0,6E0,2.8E1,5E0,5E0,1.9E1,6E0,3E0,8E0,1.2E1,6E0,5E0,3E0,2.1E1,7E0,7E0,1.2E1,3E0,5E0,9E0,3E0,3E0,3E0,1.8E1,3E0,4E0,3E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[8.655856E-3,-4.611162E-2,1.5880437E-1,-6.9829196E-2,3.307861E-2,1.7614868E-1,-6.6342256E-3,-1.0908741E-1,-4.1183364E-2,3.0047777E-3,1.9059062E-1,1.3216025E-1,4.4885746E-1,7.713973E-4,-1.17643915E-1,2.3417452E-2,-6.3939445E-2,-1.027356E-2,4.3466995E-3,8.22789E-3,1.9849653E-3,7.683172E-2,1.921495E-1,7.196396E-3,1.8083427E-2,-1.4541107E-1,-8.446493E-2,-5.149517E-2,1.1510678E-1,-7.1459087E-3,-5.3849928E-2,-4.7436215E-2,5.4826614E-2,3.6353074E-2,1.3784437E-1,2.1952683E-1,1.0083529E-1,-1.6257088E-1,-6.932035E-2,-3.9099815E-4,-9.152724E-2,-8.807088E-2,-1.7325712E-2,7.277822E-4,4.6167644E-3,4.703818E-2,-6.420495E-2,-3.346963E-2,-3.7302186E-3,7.634638E-2,3.161779E-4,-3.0121785E-2,7.381445E-2,2.1621165E-1,-8.628556E-5,1.4362784E-1,8.590022E-3,4.3920125E-3,7.1499357E-4,-5.596208E-3,-3.0290147E-3,-6.4916076E-4,-3.0508314E-3,-1.2608856E-3,-9.6061714E-2,-1.4401072E-3,-3.520985E-3,-1.2856898E-3,6.7371706E-4,-1.9277453E-4,2.842467E-3,-1.0566661E-1,-4.9046803E-2,-5.025728E-2,1.2081764E-2,3.454347E-3,1.217229E-3,-1.3347756E-3,-1.7941152E-4,2.8037715E-3,9.597336E-4,2.558514E-3,8.733766E-3,5.2792532E-3,2.1553778E-3,-3.2803828E-3,-1.6453732E-3,-3.7474523E-3,-1.4259724E-3,-6.771207E-2,-2.5814464E-2,-6.537716E-2,-3.7472148E-4,-3.2823966E-4,1.0088136E-3,-3.2281663E-2,-8.457042E-2,-5.798116E-2,3.389238E-3,-2.4909065E-3,-7.0872885E-4,-1.2722701E-3,-3.713102E-4,-8.9851953E-4,-3.2142599E-3,-2.4021685E-3,-3.281346E-4,-7.25791E-4,2.8842123E-2,8.507771E-5,1.5396187E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,-1,33,35,-1,-1,37,39,41,43,-1,45,47,49,51,53,55,57,59,61,-1,63,65,67,-1,-1,69,71,73,-1,75,-1,77,79,81,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,85,-1,-1,-1,-1,-1,-1,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,95,97,99,-1,-1,-1,101,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,109,-1,-1],"loss_changes":[2.40029E0,4.0424466E-1,5.299715E-1,1.8106389E-1,2.3627682E-1,8.4038115E-1,0E0,8.1373096E-2,1.4409325E-1,7.677353E-2,5.1300436E-2,2.042985E-1,1.2882757E-1,0E0,4.76014E-2,1.8451332E-1,1.0286683E-1,9.937258E-2,0E0,0E0,0E0,8.3898485E-2,5.5361986E-2,0E0,0E0,3.3124447E-2,1.5231624E-2,1.7220687E-2,2.8191581E-2,0E0,7.5104326E-2,2.0977274E-2,1.3777234E-2,5.9498936E-2,1.5475541E-1,4.4303775E-2,2.216667E-2,4.5006275E-3,8.099489E-3,0E0,3.1219125E-3,1.5712157E-3,8.622803E-3,0E0,0E0,1.584689E-2,3.4411788E-2,1.819532E-2,0E0,7.874355E-3,0E0,2.2753794E-3,6.1735064E-3,4.625699E-2,0E0,6.205648E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.62521E-3,0E0,0E0,0E0,0E0,0E0,0E0,5.401358E-3,1.9066676E-2,9.164695E-3,3.3230272E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2799688E-2,2.1802407E-2,4.543435E-3,0E0,0E0,0E0,1.0468019E-3,1.1935063E-2,7.989328E-3,9.232577E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3559492E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,45,45,46,46,47,47,49,49,51,51,52,52,53,53,55,55,64,64,71,71,72,72,73,73,74,74,89,89,90,90,91,91,95,95,96,96,97,97,98,98,108,108],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,-1,34,36,-1,-1,38,40,42,44,-1,46,48,50,52,54,56,58,60,62,-1,64,66,68,-1,-1,70,72,74,-1,76,-1,78,80,82,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,86,-1,-1,-1,-1,-1,-1,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,96,98,100,-1,-1,-1,102,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,110,-1,-1],"split_conditions":[8.1188E4,7.76E3,6.25E2,6.76E2,1.83E2,1.1741724E4,-6.6342256E-3,1.5E1,9.068E3,8.58437E6,1.7156E4,7.2000364E9,2.15195E5,7.713973E-4,9.26E3,1E0,1.0882E4,4.48546E7,4.3466995E-3,8.22789E-3,1.9849653E-3,1.83E2,1.4495923E11,7.196396E-3,1.8083427E-2,2.586369E7,1.6E1,5.8042584E7,3.676E3,-7.1459087E-3,6.806435E-6,2.083E3,1.6121E4,4.365884E6,2.1009742E10,1.1962493E9,2.595033E6,6.549254E6,1.0542E4,-3.9099815E-4,4.93456E5,4.2954834E-5,1.158E3,7.277822E-4,4.6167644E-3,4.352E3,3.4585872E8,3.044519E6,-3.7302186E-3,1.16E2,3.161779E-4,1.24673E5,9.5842E4,3.9E1,-8.628556E-5,4.368265E7,8.590022E-3,4.3920125E-3,7.1499357E-4,-5.596208E-3,-3.0290147E-3,-6.4916076E-4,-3.0508314E-3,-1.2608856E-3,2.83098E5,-1.4401072E-3,-3.520985E-3,-1.2856898E-3,6.7371706E-4,-1.9277453E-4,2.842467E-3,1.369E3,6E2,1.6E2,1.37E4,3.454347E-3,1.217229E-3,-1.3347756E-3,-1.7941152E-4,2.8037715E-3,9.597336E-4,2.558514E-3,8.733766E-3,5.2792532E-3,2.1553778E-3,-3.2803828E-3,-1.6453732E-3,-3.7474523E-3,-1.4259724E-3,8.0793E4,1.0038519E10,2.1916846E3,-3.7472148E-4,-3.2823966E-4,1.0088136E-3,4.106E3,9.8E2,2.0037E4,6.1583864E7,-2.4909065E-3,-7.0872885E-4,-1.2722701E-3,-3.713102E-4,-8.9851953E-4,-3.2142599E-3,-2.4021685E-3,-3.281346E-4,-7.25791E-4,1.4002668E3,8.507771E-5,1.5396187E-3],"split_indices":[28,3,4,0,47,5,0,28,7,2,7,11,3,0,22,19,18,18,0,0,0,47,26,0,0,23,17,16,3,0,64,0,7,6,10,16,12,18,7,0,21,63,0,0,0,3,11,2,0,4,0,28,20,4,0,2,0,0,0,0,0,0,0,0,6,0,0,0,0,0,0,17,17,4,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,12,10,5,0,0,0,3,0,28,16,0,0,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.9E2,2.13E2,7.7E1,1.64E2,4.9E1,7.4E1,3E0,6.8E1,9.6E1,4.2E1,7E0,6.5E1,9E0,4E0,6.4E1,2.5E1,7.1E1,3.9E1,3E0,4E0,3E0,3.5E1,3E1,4E0,5E0,3.3E1,3.1E1,1.4E1,1.1E1,3E0,6.8E1,2.5E1,1.4E1,2.2E1,1.3E1,2.2E1,8E0,2.6E1,7E0,3E0,2.8E1,6E0,8E0,3E0,8E0,6E0,6.2E1,2.2E1,3E0,9E0,5E0,8E0,1.4E1,8E0,5E0,1E1,1.2E1,5E0,3E0,2.1E1,5E0,3E0,4E0,3E0,2.5E1,3E0,3E0,5E0,3E0,3E0,3E0,1.5E1,4.7E1,1.6E1,6E0,4E0,5E0,5E0,3E0,1E1,4E0,3E0,5E0,7E0,3E0,2.1E1,4E0,1.2E1,3E0,2.5E1,2.2E1,1.1E1,5E0,3E0,3E0,9E0,1.6E1,1E1,1.2E1,8E0,3E0,6E0,3E0,4E0,1.2E1,7E0,3E0,6E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[5.2161925E-3,-4.029542E-2,1.6687721E-1,-9.683051E-2,9.269784E-3,1.03460975E-1,2.3741412E-1,-1.2130749E-1,-4.4506893E-2,-8.0073215E-3,1.5740563E-1,1.25665E-2,6.554557E-2,2.5927526E-1,9.816978E-4,1.7399773E-3,-1.2959525E-1,1.6302927E-3,-6.148818E-2,9.675298E-3,-1.7124549E-1,7.518013E-4,6.2860204E-3,1.1799254E-1,9.242968E-3,1.5158957E-1,3.0469644E-1,-2.0668724E-1,-1.1008161E-1,-7.7766426E-2,6.2121224E-4,-6.472173E-3,2.2974414E-1,1.3503287E-3,-1.1456054E-2,1.6531804E-1,2.6767816E-2,3.693607E-2,-2.8968013E-3,7.948474E-2,7.7213473E-3,1.2359834E-2,6.5303002E-3,-3.786616E-3,-7.506281E-3,-1.354054E-1,-8.193255E-2,-8.879346E-2,-3.515113E-2,6.9946107E-3,-1.6027123E-2,3.4133599E-3,9.528283E-3,1.9323643E-3,6.350764E-3,-3.1347713E-4,1.8213263E-3,-8.8204324E-5,5.30806E-2,1.1467809E-3,3.3306482E-3,-5.0917207E-3,-3.334827E-3,-1.04617625E-1,-4.853406E-2,-2.2125524E-3,-3.7410923E-3,-2.0747398E-4,-1.772613E-3,-3.3803623E-2,5.4986812E-2,3.4512224E-4,6.970271E-2,-1.1720385E-1,-1.1076687E-3,-1.4063817E-2,-2.6071507E-3,-2.5398364E-2,-7.193653E-2,7.407279E-2,1.1686155E-2,3.0364266E-3,8.899743E-4,-1.9063525E-3,-4.1083605E-3,-7.750032E-4,6.335471E-5,3.1128312E-3,-4.302353E-2,-1.2959973E-3,-2.7907903E-3,2.8758645E-3,5.59399E-4,-4.7826115E-4,1.1365502E-3,2.11472E-3,-8.407612E-3,-8.574948E-2,-2.5333567E-2,-2.2842038E-2,1.9707888E-2,-1.2051609E-3,-3.4766067E-3,-6.491286E-2,-1.0858895E-3,-6.2032123E-3,-1.505362E-3,1.5164077E-3,-6.2675675E-4,-8.146266E-2,-5.0166674E-4,2.4983246E-2,-1.5039586E-2,-6.367072E-4,1.9779771E-4,-3.1883311E-3,-1.2589827E-3,-2.5881577E-4,1.666139E-3,-9.1050746E-4,3.556673E-4,3.7367933E-4,-3.5364434E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,27,-1,29,31,33,-1,-1,35,37,39,41,43,45,47,-1,49,51,-1,-1,53,55,57,-1,59,-1,-1,-1,-1,-1,61,63,65,67,-1,69,-1,-1,-1,-1,-1,-1,-1,71,-1,-1,-1,-1,73,75,-1,-1,-1,-1,77,79,-1,81,83,-1,85,-1,87,89,91,93,-1,-1,-1,-1,-1,-1,95,97,-1,-1,-1,-1,-1,-1,-1,99,101,103,105,107,-1,-1,109,111,113,-1,-1,-1,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,121,-1,-1],"loss_changes":[2.1771731E0,6.4819956E-1,2.66194E-1,1.3291931E-1,3.186209E-1,3.66067E-1,1.2779534E-1,1.1310065E-1,6.128466E-2,3.2494244E-1,6.144926E-2,0E0,9.6355125E-2,8.587396E-2,0E0,0E0,8.25516E-2,0E0,4.2268053E-2,3.6394957E-1,4.4797295E-1,0E0,0E0,6.896815E-2,5.0042886E-2,4.4592068E-2,6.740761E-2,3.2536983E-3,3.0250728E-2,9.096742E-3,0E0,2.0874187E-1,2.5992304E-2,0E0,0E0,2.5191545E-2,8.169996E-3,9.113217E-3,0E0,3.6778525E-3,0E0,0E0,0E0,0E0,0E0,4.61936E-3,1.76519E-2,3.014341E-3,3.647333E-3,0E0,1.1926003E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.2935986E-3,0E0,0E0,0E0,0E0,1.0988548E-2,1.4526103E-2,0E0,0E0,0E0,0E0,2.2211708E-2,1.4836654E-2,0E0,4.6420544E-3,9.063482E-4,0E0,1.3315998E-3,0E0,3.179234E-2,2.5884658E-3,9.701997E-3,4.913884E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.8577307E-2,2.7304664E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.378453E-3,7.783495E-3,2.7760986E-2,5.6809997E-3,9.505852E-3,0E0,0E0,7.710416E-3,7.287193E-3,1.7980604E-3,0E0,0E0,0E0,2.0847172E-3,0E0,6.6063465E-3,2.661241E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0209971E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,16,16,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,36,36,37,37,39,39,45,45,46,46,47,47,48,48,50,50,58,58,63,63,64,64,69,69,70,70,72,72,73,73,75,75,77,77,78,78,79,79,80,80,87,87,88,88,96,96,97,97,98,98,99,99,100,100,103,103,104,104,105,105,109,109,111,111,112,112,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,28,-1,30,32,34,-1,-1,36,38,40,42,44,46,48,-1,50,52,-1,-1,54,56,58,-1,60,-1,-1,-1,-1,-1,62,64,66,68,-1,70,-1,-1,-1,-1,-1,-1,-1,72,-1,-1,-1,-1,74,76,-1,-1,-1,-1,78,80,-1,82,84,-1,86,-1,88,90,92,94,-1,-1,-1,-1,-1,-1,96,98,-1,-1,-1,-1,-1,-1,-1,100,102,104,106,108,-1,-1,110,112,114,-1,-1,-1,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,122,-1,-1],"split_conditions":[1.0737418E9,4.44688E5,5.752002E9,6.549254E6,3.48037E5,3.068E3,8.04341E5,3E0,2.9609E4,8.23838E5,9.683002E6,1.25665E-2,1.02E2,6.28354E5,9.816978E-4,1.7399773E-3,5.328523E2,1.6302927E-3,6.490568E8,1.0368806E7,3.5849126E10,7.518013E-4,6.2860204E-3,4.049221E10,4.425011E6,3.5258776E7,2.054214E0,5.4E1,4.20172E5,7.1376E4,6.2121224E-4,2.5961E4,2.9663E4,1.3503287E-3,-1.1456054E-2,1.315E3,6.018E3,2.51089E5,-2.8968013E-3,1.01E2,7.7213473E-3,1.2359834E-2,6.5303002E-3,-3.786616E-3,-7.506281E-3,1E0,1.49015E5,6.4281E4,1.0098E5,6.9946107E-3,5.018E6,3.4133599E-3,9.528283E-3,1.9323643E-3,6.350764E-3,-3.1347713E-4,1.8213263E-3,-8.8204324E-5,1.6429902E9,1.1467809E-3,3.3306482E-3,-5.0917207E-3,-3.334827E-3,1.6486432E8,4.699127E7,-2.2125524E-3,-3.7410923E-3,-2.0747398E-4,-1.772613E-3,3.98E2,4.348455E10,3.4512224E-4,3.6E1,5.1279686E8,-1.1076687E-3,1.61E2,-2.6071507E-3,5.527E3,9.675527E7,2.5420546E7,2.379217E6,3.0364266E-3,8.899743E-4,-1.9063525E-3,-4.1083605E-3,-7.750032E-4,6.335471E-5,3.342E3,1.7140013E3,-1.2959973E-3,-2.7907903E-3,2.8758645E-3,5.59399E-4,-4.7826115E-4,1.1365502E-3,2.11472E-3,1.179118E6,3.3300438E7,3.3587363E8,3.3704907E-2,4.821E3,-1.2051609E-3,-3.4766067E-3,1.4947E4,1.02E2,1.229193E3,-1.505362E-3,1.5164077E-3,-6.2675675E-4,1.67E2,-5.0166674E-4,6.1E1,8.5591583E9,-6.367072E-4,1.9779771E-4,-3.1883311E-3,-1.2589827E-3,-2.5881577E-4,1.666139E-3,-9.1050746E-4,7.0345754E8,3.7367933E-4,-3.5364434E-4],"split_indices":[16,6,27,18,22,7,19,17,8,12,2,0,4,28,0,0,5,0,16,6,10,0,0,23,2,6,59,17,9,8,0,12,3,0,0,0,7,12,0,4,0,0,0,0,0,42,18,8,12,0,2,0,0,0,0,0,0,0,16,0,0,0,0,21,16,0,0,0,0,4,10,0,19,10,0,17,0,3,21,18,6,0,0,0,0,0,0,7,5,0,0,0,0,0,0,0,6,27,16,63,3,0,0,7,4,5,0,0,0,4,0,4,10,0,0,0,0,0,0,0,16,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.94E2,2.3E2,6.4E1,1.07E2,1.23E2,3.5E1,2.9E1,7.2E1,3.5E1,1.11E2,1.2E1,3E0,3.2E1,2.6E1,3E0,3E0,6.9E1,5E0,3E1,1.01E2,1E1,3E0,9E0,1.6E1,1.6E1,9E0,1.7E1,1.2E1,5.7E1,2.5E1,5E0,9.5E1,6E0,5E0,5E0,1E1,6E0,1.3E1,3E0,6E0,3E0,8E0,9E0,4E0,8E0,2.8E1,2.9E1,1.9E1,6E0,3E0,9.2E1,3E0,3E0,3E0,7E0,3E0,3E0,4E0,9E0,3E0,3E0,1.4E1,1.4E1,1.6E1,1.3E1,1.3E1,6E0,3E0,3E0,7.4E1,1.8E1,3E0,6E0,1.3E1,3E0,7E0,6E0,6.2E1,1.2E1,1.2E1,6E0,3E0,3E0,3E0,1E1,4E0,3E0,2.4E1,3.8E1,5E0,7E0,9E0,3E0,3E0,3E0,3E0,2.1E1,1E1,2.8E1,1.4E1,7E0,4E0,6E0,1E1,1.8E1,9E0,5E0,4E0,3E0,7E0,3E0,6E0,1.2E1,4E0,5E0,4E0,3E0,3E0,3E0,6E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[6.130507E-3,-4.8600085E-2,1.392564E-1,-6.43874E-2,1.389517E-1,9.1354236E-2,2.1274565E-1,4.975649E-2,-7.320135E-2,9.252643E-3,4.5321893E-2,6.0163587E-2,1.9030942E-1,7.558289E-2,2.5531727E-1,-4.9295712E-2,7.6580625E-3,-1.9366592E-1,-6.388371E-2,6.2244527E-2,4.7044355E-6,2.697708E-2,1.2956706E-1,2.1973152E-3,2.2340678E-1,5.3521623E-3,8.129349E-5,3.7784064E-1,1.8264055E-1,-4.0209536E-3,7.061651E-4,-7.289304E-3,-3.2984845E-3,-1.3673568E-1,-5.3321738E-2,8.3953066E-4,2.534114E-3,-1.1071317E-2,5.4486025E-2,1.5834737E-1,1.6689687E-3,9.700291E-3,3.2439262E-3,6.2019317E-3,1.449813E-2,6.8130665E-3,2.2746597E-3,-1.5315124E-1,-1.8302242E-3,2.4805188E-3,-5.8566168E-2,-2.7820468E-2,4.2187187E-4,7.092403E-2,-5.3654116E-4,2.653814E-3,5.8763465E-3,-2.7837898E-3,-5.413126E-3,-4.8161823E-2,-1.3553944E-1,-7.1834525E-5,-1.3753515E-3,9.5032895E-4,8.453192E-2,-6.0044073E-2,2.514719E-2,-6.827836E-3,-5.2410275E-2,1.3963163E-3,3.3713938E-3,-8.664921E-2,-4.1519996E-2,-6.1737397E-3,3.4704963E-3,-2.2571082E-3,-4.1016005E-4,-4.122224E-2,-1.0290409E-1,-3.5860058E-2,-3.8262694E-3,1.9001191E-2,-2.3775203E-3,9.242563E-5,-5.4737452E-2,-1.620107E-3,-1.0882856E-1,-7.0439726E-2,-2.8658988E-2,-4.0537486E-4,3.2423437E-2,-2.5222052E-3,-1.0757992E-3,-2.0685394E-3,-1.1850164E-1,-2.8724188E-3,-1.0878604E-3,-3.5328127E-3,-3.6736675E-2,2.598884E-4,1.461972E-3,-1.9035051E-3,-4.027184E-3,-1.5441435E-3,1.903967E-2,-2.309531E-3,-2.9684916E-2,1.1674239E-3,4.8591955E-5,-4.4921416E-4,-1.4087541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,33,35,-1,37,39,-1,41,-1,-1,43,45,-1,-1,-1,-1,47,49,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,59,61,-1,63,-1,-1,-1,-1,-1,65,67,-1,-1,-1,69,71,73,-1,75,-1,-1,77,79,81,-1,-1,-1,83,85,87,-1,89,-1,-1,91,-1,93,95,97,-1,99,-1,-1,-1,101,-1,-1,103,105,-1,-1,-1,-1,-1,107,-1,109,-1,-1,-1,-1],"loss_changes":[2.0760086E0,6.070565E-1,2.727492E-1,1.917752E-1,2.0943084E-1,1.4904702E-1,1.6755807E-1,2.7264562E-1,1.8293345E-1,0E0,8.400321E-3,9.093827E-2,3.3222556E-2,5.9210468E-2,1.4470136E-1,5.6608506E-2,0E0,1.2674898E-2,1.1813104E-1,2.7149543E-3,0E0,3.0956699E-2,2.0867571E-2,0E0,5.565834E-2,0E0,0E0,2.071464E-2,4.028064E-2,0E0,0E0,0E0,0E0,1.7516524E-2,1.0297805E-1,0E0,0E0,5.628985E-3,2.1066036E-2,1.4877468E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4738739E-3,0E0,0E0,1.04791194E-1,2.9562595E-3,0E0,5.3915754E-3,0E0,0E0,0E0,0E0,0E0,1.09523326E-1,9.073657E-2,0E0,0E0,0E0,3.7718862E-3,4.9135745E-2,4.7272585E-2,0E0,5.4292995E-3,0E0,0E0,2.750972E-2,2.4193734E-2,2.5259892E-2,0E0,0E0,0E0,7.9043135E-3,4.177958E-3,1.3419919E-2,0E0,5.0731306E-3,0E0,0E0,2.415821E-3,0E0,4.024327E-3,3.8718618E-3,1.0627173E-2,0E0,2.1646814E-3,0E0,0E0,0E0,8.4102154E-4,0E0,0E0,1.4956546E-2,7.7461153E-3,0E0,0E0,0E0,0E0,0E0,2.9653634E-3,0E0,7.140793E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,24,24,27,27,28,28,33,33,34,34,37,37,38,38,39,39,47,47,50,50,51,51,53,53,59,59,60,60,64,64,65,65,66,66,68,68,71,71,72,72,73,73,77,77,78,78,79,79,81,81,84,84,86,86,87,87,88,88,90,90,94,94,97,97,98,98,104,104,106,106],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,34,36,-1,38,40,-1,42,-1,-1,44,46,-1,-1,-1,-1,48,50,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,60,62,-1,64,-1,-1,-1,-1,-1,66,68,-1,-1,-1,70,72,74,-1,76,-1,-1,78,80,82,-1,-1,-1,84,86,88,-1,90,-1,-1,92,-1,94,96,98,-1,100,-1,-1,-1,102,-1,-1,104,106,-1,-1,-1,-1,-1,108,-1,110,-1,-1,-1,-1],"split_conditions":[8.1188E4,1.2084881E9,5.0600757E3,3.719E3,4.808E4,1.83E2,4.2E1,2.8E1,1.0882E4,9.252643E-3,1.607E3,1.78962E5,1.182761E6,1.2065335E7,3.9622688E-1,3.197612E-3,7.6580625E-3,1.0510289E3,2.65721E5,4.5556176E8,4.7044355E-6,4.01145E5,2.1131E4,2.1973152E-3,2.3278559E9,5.3521623E-3,8.129349E-5,1.310231E9,9.9181765E10,-4.0209536E-3,7.061651E-4,-7.289304E-3,-3.2984845E-3,1.3953088E8,1.6028E4,8.3953066E-4,2.534114E-3,1.24673E5,7.2000364E9,6.035628E10,1.6689687E-3,9.700291E-3,3.2439262E-3,6.2019317E-3,1.449813E-2,6.8130665E-3,2.2746597E-3,4.9403088E2,-1.8302242E-3,2.4805188E-3,1.606E3,1.2583364E10,4.2187187E-4,6.508427E9,-5.3654116E-4,2.653814E-3,5.8763465E-3,-2.7837898E-3,-5.413126E-3,2.414E3,3.0025E4,-7.1834525E-5,-1.3753515E-3,9.5032895E-4,6.7795E4,4.73E3,3.60589E6,-6.827836E-3,2.103005E6,1.3963163E-3,3.3713938E-3,6.350158E-4,1.0886205E9,4.445E3,3.4704963E-3,-2.2571082E-3,-4.1016005E-4,5.11E2,2.753E3,6.4966E4,-3.8262694E-3,6.2756E4,-2.3775203E-3,9.242563E-5,8.751819E2,-1.620107E-3,9.710135E2,6.493919E6,8.5E1,-4.0537486E-4,8.67E2,-2.5222052E-3,-1.0757992E-3,-2.0685394E-3,9.934327E7,-2.8724188E-3,-1.0878604E-3,6.4594E4,1.0699521E9,2.598884E-4,1.461972E-3,-1.9035051E-3,-4.027184E-3,-1.5441435E-3,1.59037E5,-2.309531E-3,1.033E3,1.1674239E-3,4.8591955E-5,-4.4921416E-4,-1.4087541E-3],"split_indices":[28,16,5,7,8,47,4,17,18,0,0,20,6,6,63,63,0,5,9,11,0,12,19,0,11,0,0,1,10,0,0,0,0,16,12,0,0,28,11,10,0,0,0,0,0,0,0,5,0,0,19,10,0,10,0,0,0,0,0,0,28,0,0,0,8,3,18,0,2,0,0,63,16,3,0,0,0,17,3,12,0,12,0,0,5,0,5,21,4,0,28,0,0,0,10,0,0,8,10,0,0,0,0,0,12,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.01E2,8.2E1,1.86E2,1.5E1,5.1E1,3.1E1,1.3E1,1.73E2,5E0,1E1,4E1,1.1E1,8E0,2.3E1,9E0,4E0,1.1E1,1.62E2,7E0,3E0,2.8E1,1.2E1,3E0,8E0,3E0,5E0,7E0,1.6E1,4E0,5E0,7E0,4E0,1.9E1,1.43E2,3E0,4E0,1.2E1,1.6E1,8E0,4E0,4E0,4E0,3E0,4E0,1.2E1,4E0,1.5E1,4E0,5E0,1.38E2,7E0,5E0,1.3E1,3E0,3E0,5E0,4E0,1.1E1,1.23E2,1.5E1,3E0,4E0,4E0,9E0,1.06E2,1.7E1,7E0,8E0,4E0,5E0,4.2E1,6.4E1,1.3E1,4E0,5E0,3E0,1.2E1,3E1,6.1E1,3E0,1E1,3E0,3E0,9E0,4E0,2.6E1,9E0,5.2E1,3E0,7E0,3E0,6E0,6E0,2E1,5E0,4E0,1.3E1,3.9E1,3E0,4E0,3E0,1.7E1,4E0,9E0,5E0,3.4E1,4E0,5E0,1.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[9.897505E-3,-5.6849137E-2,1.24900416E-1,-1.1169518E-1,-1.7888421E-2,7.12803E-2,2.0086758E-1,-9.6332096E-2,-1.0979747E-2,-3.4694005E-2,7.250927E-2,9.169442E-2,-7.4787014E-3,2.2003883E-1,7.2943576E-4,-4.8458714E-2,-1.4137219E-1,-4.93086E-2,5.0041337E-2,-3.5842232E-2,1.6076513E-1,7.179845E-2,1.0699365E-2,1.8016016E-1,1.1572358E-2,-6.9266945E-2,4.0210364E-3,-2.3142172E-1,-1.047638E-1,-6.68273E-2,7.933374E-3,2.0701736E-2,3.5691122E-3,-3.5077229E-3,2.0772049E-3,6.3608787E-3,2.102122E-3,2.6320943E-1,4.1762806E-2,1.4485604E-1,1.0548632E-2,3.9970822E-4,-8.182826E-2,-8.706399E-3,-3.7011565E-3,-4.1774213E-3,-5.1763173E-2,-3.7389507E-3,-5.7409756E-2,2.8424843E-3,-2.1337872E-2,1.7841652E-3,-3.761921E-4,1.1485849E-2,3.3409186E-3,5.9082415E-2,-6.564673E-2,1.576631E-1,1.1266557E-3,-1.1413873E-1,-3.1928618E-2,-7.346328E-5,-2.3383105E-3,-1.0072654E-2,-6.5059446E-2,-6.413495E-2,3.0621339E-2,5.146065E-3,4.0643226E-2,-4.0374924E-3,3.3956772E-4,1.11391805E-1,6.652706E-3,-1.3406067E-1,-6.7138934E-4,2.1880299E-4,-1.9494008E-3,3.5249948E-4,-1.0890808E-3,-7.20518E-2,-3.571919E-4,-2.7363321E-3,-7.084593E-4,-1.6201199E-4,1.8869328E-3,2.8502692E-2,3.5476738E-3,7.140564E-2,4.918658E-3,-4.737873E-3,-2.1799556E-3,-7.597326E-2,-1.0223159E-3,1.3722029E-2,5.5472862E-2,1.1857917E-3,3.3928773E-3,-7.9897724E-2,-8.866905E-4,-1.654597E-2,3.5949565E-2,7.749672E-2,-7.864005E-5,-8.600114E-2,-1.324593E-3,-9.252137E-4,4.506128E-5,-5.923732E-4,5.4095633E-2,3.4158886E-3,7.1921694E-4,-2.947354E-3,-1.0913489E-3,7.277485E-4,2.0716896E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,25,27,29,31,33,35,37,-1,39,-1,41,-1,43,45,47,49,51,-1,-1,-1,-1,-1,53,55,57,-1,-1,59,-1,-1,-1,61,-1,63,-1,65,-1,-1,-1,-1,67,69,71,-1,73,75,-1,-1,77,79,81,83,-1,85,-1,-1,87,-1,89,-1,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,93,-1,95,-1,-1,-1,97,-1,99,101,-1,-1,103,-1,105,107,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0489318E0,3.5912812E-1,3.848722E-1,2.1384943E-1,1.5443423E-1,3.8699135E-1,1.3167143E-1,1.376161E-1,0E0,1.07729144E-1,1.5876466E-1,2.4431422E-1,0E0,1.3546395E-1,0E0,1.3131458E-1,8.783871E-2,7.373494E-2,2.0909362E-2,6.57554E-2,1.8750131E-2,2.937128E-1,0E0,1.0880244E-1,0E0,3.2824114E-2,0E0,9.828925E-3,2.775234E-2,1.9346714E-2,4.4231288E-2,1.1895135E-2,0E0,0E0,0E0,0E0,0E0,6.743398E-2,9.1312245E-2,3.106141E-2,0E0,0E0,3.9887294E-2,0E0,0E0,0E0,1.0165865E-2,0E0,1.708828E-2,0E0,3.309573E-2,0E0,0E0,0E0,0E0,6.9698825E-2,3.321041E-2,2.9140115E-2,0E0,2.6937202E-2,1.3717574E-2,0E0,0E0,4.366904E-3,1.5048161E-2,5.1634684E-3,7.283117E-3,0E0,2.7265154E-2,0E0,0E0,1.2272581E-2,0E0,1.3537407E-3,0E0,0E0,0E0,0E0,0E0,3.4332275E-3,0E0,0E0,0E0,0E0,0E0,1.1995185E-2,0E0,6.6944025E-3,0E0,0E0,0E0,4.070744E-3,0E0,1.5396046E-2,1.4220189E-2,0E0,0E0,3.555715E-3,0E0,2.2291362E-3,1.381723E-2,1.0258157E-2,0E0,4.1408986E-3,0E0,0E0,0E0,0E0,1.7769225E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,25,25,27,27,28,28,29,29,30,30,31,31,37,37,38,38,39,39,42,42,46,46,48,48,50,50,55,55,56,56,57,57,59,59,60,60,63,63,64,64,65,65,66,66,68,68,71,71,73,73,79,79,85,85,87,87,91,91,93,93,94,94,97,97,99,99,100,100,101,101,103,103,108,108],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,26,28,30,32,34,36,38,-1,40,-1,42,-1,44,46,48,50,52,-1,-1,-1,-1,-1,54,56,58,-1,-1,60,-1,-1,-1,62,-1,64,-1,66,-1,-1,-1,-1,68,70,72,-1,74,76,-1,-1,78,80,82,84,-1,86,-1,-1,88,-1,90,-1,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,94,-1,96,-1,-1,-1,98,-1,100,102,-1,-1,104,-1,106,108,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,-1],"split_conditions":[9.896E3,4.07729E5,1.021304E6,9.2921E4,1.83E2,5.11E2,1.6321E4,2.39E2,-1.0979747E-2,1.9452E4,1.1458619E7,7.0231123E3,-7.4787014E-3,6.258553E7,7.2943576E-4,3.99507E5,7.131912E2,6.910113E8,6.4461936E7,1E0,7.505852E8,4.833E3,1.0699365E-2,4.0176924E9,1.1572358E-2,3.318E3,4.0210364E-3,2.4339E4,2.1245E4,3.0875E4,3.7661046E-4,7.56543E2,3.5691122E-3,-3.5077229E-3,2.0772049E-3,6.3608787E-3,2.102122E-3,3.7009846E3,1E0,7.20467E7,1.0548632E-2,3.9970822E-4,6.00237E5,-8.706399E-3,-3.7011565E-3,-4.1774213E-3,3.388763E7,-3.7389507E-3,7.75E2,2.8424843E-3,8.054554E8,1.7841652E-3,-3.761921E-4,1.1485849E-2,3.3409186E-3,4.7717344E8,2.6494702E3,1.7994546E0,1.1266557E-3,1.6397E4,5.1E1,-7.346328E-5,-2.3383105E-3,1.17E2,7.9E3,2.83326E9,1.5778465E9,5.146065E-3,3.937E3,-4.0374924E-3,3.3956772E-4,4.564531E6,6.652706E-3,1.305475E3,-6.7138934E-4,2.1880299E-4,-1.9494008E-3,3.5249948E-4,-1.0890808E-3,1.9343008E3,-3.571919E-4,-2.7363321E-3,-7.084593E-4,-1.6201199E-4,1.8869328E-3,1.6429902E9,3.5476738E-3,2.009388E6,4.918658E-3,-4.737873E-3,-2.1799556E-3,5.0373805E8,-1.0223159E-3,4.01145E5,2.5617332E10,1.1857917E-3,3.3928773E-3,2.8673078E7,-8.866905E-4,4.4598663E-1,8.6E1,1.16E2,-7.864005E-5,5.637464E-1,-1.324593E-3,-9.252137E-4,4.506128E-5,-5.923732E-4,6.4762153E9,3.4158886E-3,7.1921694E-4,-2.947354E-3,-1.0913489E-3,7.277485E-4,2.0716896E-3],"split_indices":[3,18,12,28,47,4,0,17,0,7,18,5,0,6,0,12,5,16,21,19,16,7,0,16,0,7,0,18,28,12,63,5,0,0,0,0,0,5,36,2,0,0,6,0,0,0,16,0,28,0,23,0,0,0,0,16,5,64,0,7,17,0,0,4,3,10,10,0,0,0,0,18,0,5,0,0,0,0,0,5,0,0,0,0,0,16,0,12,0,0,0,16,0,12,10,0,0,18,0,63,19,4,0,63,0,0,0,0,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.68E2,9.7E1,6.9E1,9.9E1,5.8E1,3.9E1,6.6E1,3E0,8.4E1,1.5E1,5.5E1,3E0,3.5E1,4E0,3.3E1,3.3E1,7.2E1,1.2E1,7E0,8E0,5.2E1,3E0,2.9E1,6E0,3E1,3E0,8E0,2.5E1,5.5E1,1.7E1,9E0,3E0,4E0,3E0,5E0,3E0,6E0,4.6E1,2.5E1,4E0,4E0,2.6E1,5E0,3E0,1.6E1,9E0,7E0,4.8E1,4E0,1.3E1,4E0,5E0,3E0,3E0,4E1,6E0,2.2E1,3E0,1.5E1,1.1E1,3E0,6E0,7E0,4.1E1,7E0,6E0,5E0,3.5E1,3E0,3E0,1.3E1,9E0,1.2E1,3E0,5E0,6E0,4E0,3E0,3.6E1,5E0,4E0,3E0,3E0,3E0,3.1E1,4E0,8E0,5E0,9E0,3E0,3.2E1,4E0,2.1E1,1E1,5E0,3E0,2.9E1,3E0,9E0,1.2E1,7E0,3E0,2.4E1,5E0,5E0,4E0,3E0,9E0,4E0,3E0,2.1E1,3E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-2.4921254E-3,-4.44816E-2,1.3623814E-1,1.2930062E-1,-5.4341044E-2,8.52343E-2,2.0675354E-1,4.47903E-2,9.242213E-3,-8.8872194E-2,-5.6903576E-3,1.2915686E-1,3.485389E-2,2.490679E-1,9.445061E-2,-3.5264099E-3,4.5135696E-3,-1.2706468E-1,-6.289772E-2,1.1386813E-1,-1.9238915E-2,6.3812085E-3,8.83066E-2,-1.2886184E-2,7.5085424E-2,5.831536E-3,1.0293633E-2,4.0434296E-3,7.7545573E-4,-6.1514867E-3,-1.0040924E-1,3.0287672E-3,-7.085641E-2,1.2760952E-3,6.3327746E-3,-5.143413E-3,-1.1588446E-2,1.0855402E-1,1.2825087E-3,-8.002685E-4,3.6348766E-4,1.058984E-3,3.3720392E-3,-1.1309997E-1,-4.1854206E-2,-8.597883E-2,-4.13761E-2,-2.2395741E-2,7.135428E-2,1.4123345E-3,4.1847248E-3,-1.2273023E-1,-1.3625153E-3,-2.226176E-4,-1.9774952E-3,-4.0732825E-4,-9.266083E-2,4.6332617E-4,-5.5340108E-2,-8.5269E-3,-9.348638E-2,2.9701984E-3,7.125472E-4,-4.1473517E-3,-1.880557E-3,-3.282604E-3,-5.2644156E-2,-3.7616335E-2,-7.107463E-2,-2.0015206E-2,4.8618488E-2,-3.9110295E-2,-5.7575414E-3,-2.7056418E-3,-5.695664E-4,-3.914139E-4,-1.4404792E-3,-2.855124E-3,-1.1494878E-3,-3.7504215E-2,1.2188318E-2,1.0468721E-2,3.3227364E-3,-5.7899666E-5,-2.445465E-3,1.18849985E-2,-5.7347167E-2,-1.6375666E-3,1.5599778E-3,7.558322E-4,-1.6612202E-4,3.0597553E-2,-7.6716655E-4,-3.2054824E-3,-4.603258E-2,8.2124065E-4,-1.5049151E-2,1.6830288E-4,1.5552781E-3,-1.0483545E-3,-1.8949516E-3,-8.7526644E-4,8.148669E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,35,-1,37,39,41,-1,-1,-1,-1,-1,43,-1,45,-1,-1,-1,47,49,-1,-1,-1,-1,-1,51,53,55,57,59,61,-1,-1,63,-1,-1,-1,-1,65,-1,67,69,71,-1,-1,-1,-1,-1,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,85,87,89,-1,-1,-1,91,93,95,-1,-1,-1,97,-1,-1,99,-1,101,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6545949E0,3.805467E-1,2.1975017E-1,1.4719181E-1,3.4724694E-1,8.401182E-2,1.047802E-1,1.479239E-1,0E0,1.1204344E-1,1.4311722E-1,4.1759104E-2,3.9380692E-2,3.3303976E-2,1.6708598E-2,0E0,0E0,6.274897E-2,9.912586E-2,4.7558807E-2,8.309442E-2,0E0,1.0393798E-2,3.1762156E-3,1.03229955E-2,0E0,0E0,0E0,0E0,0E0,2.2018075E-2,0E0,2.8095305E-2,0E0,0E0,0E0,6.9723405E-2,8.169815E-3,0E0,0E0,0E0,0E0,0E0,1.4569819E-2,5.0477814E-3,2.1365345E-2,2.1044675E-2,6.6247106E-2,7.223934E-3,0E0,0E0,2.3016036E-3,0E0,0E0,0E0,0E0,1.02927685E-2,0E0,3.1827986E-3,3.9050326E-2,4.407374E-2,0E0,0E0,0E0,0E0,0E0,9.490669E-3,1.6352311E-3,3.0651428E-3,2.8054288E-2,1.975198E-2,1.0866332E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.204246E-2,9.106927E-3,1.5448267E-3,0E0,0E0,0E0,7.413029E-3,6.407261E-3,5.4818224E-3,0E0,0E0,0E0,2.8940593E-3,0E0,0E0,1.539398E-3,0E0,2.2035427E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,22,22,23,23,24,24,30,30,32,32,36,36,37,37,43,43,44,44,45,45,46,46,47,47,48,48,51,51,56,56,58,58,59,59,60,60,66,66,67,67,68,68,69,69,70,70,71,71,79,79,80,80,81,81,85,85,86,86,87,87,91,91,94,94,96,96],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,36,-1,38,40,42,-1,-1,-1,-1,-1,44,-1,46,-1,-1,-1,48,50,-1,-1,-1,-1,-1,52,54,56,58,60,62,-1,-1,64,-1,-1,-1,-1,66,-1,68,70,72,-1,-1,-1,-1,-1,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,86,88,90,-1,-1,-1,92,94,96,-1,-1,-1,98,-1,-1,100,-1,102,-1,-1,-1,-1,-1,-1],"split_conditions":[8.0936586E9,1.0916E4,1.01248E5,3.6E1,1.530129E6,4.2E1,1.83E3,1.407E3,9.242213E-3,3.43514E5,3.28451E5,6E1,5.7407E4,3.558047E7,9.2E1,-3.5264099E-3,4.5135696E-3,8.626195E2,2.9609E4,2.57488E5,3.618E3,6.3812085E-3,8.470816E10,4.30805E3,7.4399194E8,5.831536E-3,1.0293633E-2,4.0434296E-3,7.7545573E-4,-6.1514867E-3,1.149807E9,3.0287672E-3,7.6058496E8,1.2760952E-3,6.3327746E-3,-5.143413E-3,3.0831E4,3E1,1.2825087E-3,-8.002685E-4,3.6348766E-4,1.058984E-3,3.3720392E-3,2.2728746E-1,2.3005E4,5.46E2,1.2928581E9,5.2633182E10,2.2605868E-6,1.4123345E-3,4.1847248E-3,3.0025E4,-1.3625153E-3,-2.226176E-4,-1.9774952E-3,-4.0732825E-4,1.105E3,4.6332617E-4,1.0821E5,1.36692E9,4.4748E5,2.9701984E-3,7.125472E-4,-4.1473517E-3,-1.880557E-3,-3.282604E-3,2E0,4.344E3,1.366323E6,1.22661E6,2.93118E3,2.2561E4,-5.7575414E-3,-2.7056418E-3,-5.695664E-4,-3.914139E-4,-1.4404792E-3,-2.855124E-3,-1.1494878E-3,9.8E2,7.2000855E9,1.6553E4,3.3227364E-3,-5.7899666E-5,-2.445465E-3,7.52585E5,5.47107E5,5.072E3,1.5599778E-3,7.558322E-4,-1.6612202E-4,4.7717344E8,-7.6716655E-4,-3.2054824E-3,6.254E3,8.2124065E-4,9.310384E6,1.6830288E-4,1.5552781E-3,-1.0483545E-3,-1.8949516E-3,-8.7526644E-4,8.148669E-5],"split_indices":[23,8,3,17,9,4,19,28,0,18,6,17,20,6,4,0,0,5,8,6,7,0,13,5,16,0,0,0,0,0,11,0,10,0,0,0,7,19,0,0,0,0,0,63,28,28,10,10,63,0,0,28,0,0,0,0,17,0,12,16,12,0,0,0,0,0,19,22,2,6,5,20,0,0,0,0,0,0,0,0,11,7,0,0,0,6,6,3,0,0,0,16,0,0,3,0,18,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.17E2,6.5E1,1.1E1,2.06E2,3.9E1,2.6E1,8E0,3E0,1.2E2,8.6E1,2E1,1.9E1,1.8E1,8E0,3E0,5E0,4.7E1,7.3E1,8E0,7.8E1,6E0,1.4E1,9E0,1E1,1.1E1,7E0,5E0,3E0,1.2E1,3.5E1,3E0,7E1,5E0,3E0,3E0,7.5E1,9E0,5E0,6E0,3E0,5E0,5E0,2.8E1,7E0,4.5E1,2.5E1,6.7E1,8E0,3E0,6E0,2.4E1,4E0,3E0,4E0,4E0,4.1E1,5E0,2E1,5.7E1,1E1,5E0,3E0,2.1E1,3E0,3.2E1,9E0,1.1E1,9E0,4.8E1,9E0,7E0,3E0,4E0,5E0,3E0,8E0,5E0,4E0,3.1E1,1.7E1,6E0,3E0,4E0,3E0,9E0,2.2E1,1.3E1,4E0,3E0,3E0,6E0,3E0,3E0,1.9E1,4E0,9E0,3E0,3E0,1.1E1,8E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-1.5625267E-3,-5.5778887E-2,1.1797766E-1,-7.844548E-2,2.5642063E-2,6.0565855E-2,2.2695495E-1,-7.0274934E-2,-1.1690606E-2,3.846236E-2,-2.5585282E-3,8.2472615E-2,-8.410018E-3,1.8014124E-1,1.5650555E-2,-1.00532204E-1,-2.4977034E-2,1.1875076E-1,1.7188309E-2,5.651192E-2,8.244377E-3,1.0498274E-1,2.2525129E-1,1.4694605E-3,-1.0616245E-1,3.2698703E-3,-3.564065E-2,4.544533E-3,1.9642604E-3,-1.7368158E-2,6.902993E-2,2.7767051E-2,1.4522342E-1,5.1778136E-3,-1.7421684E-4,9.710355E-3,4.8037185E-3,-1.4738151E-1,-8.26057E-2,-4.9434047E-2,2.0204095E-2,1.2099403E-2,-5.4728445E-2,8.251883E-2,5.8101694E-4,-1.8381305E-2,6.8198085E-2,-3.7329813E-4,1.9878548E-1,-1.5977883E-1,-2.6981402E-3,-3.3016257E-2,-9.3746305E-2,6.752559E-4,-5.4834366E-2,3.676955E-2,-7.119941E-4,-8.592028E-4,3.4831896E-2,-2.6546607E-3,1.8274775E-5,3.302261E-3,1.3496772E-3,-4.8369322E-2,2.0313082E-2,4.979415E-2,3.4090972E-3,8.001483E-3,2.3948455E-3,-5.406959E-3,-2.4721543E-3,-2.709052E-3,-6.6168266E-3,-7.185958E-4,-1.01911806E-1,-3.028404E-2,-6.690475E-2,-1.00909085E-4,1.842633E-3,1.5345627E-3,3.2420168E-4,-1.0052525E-2,-2.9722955E-3,1.354272E-3,-5.602299E-4,1.8476353E-3,4.064727E-4,1.1604029E-3,-1.5437827E-3,-4.4954577E-3,-8.852905E-2,2.7501439E-3,-1.6216657E-3,-7.45947E-2,-6.812435E-5,-7.26443E-4,1.6017744E-4,-5.8539007E-2,-9.902303E-2,4.850767E-4,-3.3015927E-4,-8.715003E-2,-1.3709527E-3,-2.3419173E-3,-8.667974E-4,-2.2442548E-3,-3.7403468E-3,-1.4232255E-3,-3.262518E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,-1,23,25,27,29,31,-1,33,35,-1,37,-1,39,-1,-1,41,43,45,47,-1,-1,-1,-1,49,51,53,55,57,59,61,-1,63,65,-1,67,69,-1,71,73,-1,75,77,-1,-1,79,-1,-1,-1,-1,81,83,85,-1,-1,-1,-1,-1,-1,87,-1,89,91,93,-1,-1,-1,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,97,99,-1,101,-1,-1,-1,103,105,-1,-1,107,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.827751E0,3.6004543E-1,5.3683364E-1,3.2810378E-1,6.1010648E-2,4.3752044E-1,2.7651286E-1,2.01707E-1,0E0,6.457318E-2,0E0,2.3798057E-1,0E0,6.789333E-2,0E0,7.80977E-2,8.594575E-2,1.7443448E-3,5.8525737E-2,1.2346758E-1,0E0,7.488926E-2,4.7922492E-2,0E0,7.133025E-2,0E0,4.4905238E-2,0E0,0E0,2.268051E-2,7.450111E-3,7.402077E-2,1.03100896E-1,0E0,0E0,0E0,0E0,1.0246098E-2,2.8394848E-2,1.8641658E-2,9.226568E-3,1.1511781E-2,1.3853418E-2,3.8450584E-3,0E0,2.3029523E-2,9.300932E-3,0E0,3.7247777E-2,3.4227371E-3,0E0,1.56454E-2,2.4551928E-2,0E0,1.07161105E-2,7.532742E-3,0E0,0E0,2.0635258E-3,0E0,0E0,0E0,0E0,1.7604299E-2,8.118667E-3,3.7978888E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.760468E-2,0E0,8.3474815E-3,1.07599795E-2,1.37922615E-2,0E0,0E0,0E0,0E0,1.4289488E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5888424E-3,1.2762526E-3,0E0,6.3337088E-3,0E0,0E0,0E0,2.9840544E-3,3.474459E-3,0E0,0E0,5.886972E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24,26,26,29,29,30,30,31,31,32,32,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,48,48,49,49,51,51,52,52,54,54,55,55,58,58,63,63,64,64,65,65,72,72,74,74,75,75,76,76,81,81,90,90,91,91,93,93,97,97,98,98,101,101],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,-1,24,26,28,30,32,-1,34,36,-1,38,-1,40,-1,-1,42,44,46,48,-1,-1,-1,-1,50,52,54,56,58,60,62,-1,64,66,-1,68,70,-1,72,74,-1,76,78,-1,-1,80,-1,-1,-1,-1,82,84,86,-1,-1,-1,-1,-1,-1,88,-1,90,92,94,-1,-1,-1,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,98,100,-1,102,-1,-1,-1,104,106,-1,-1,108,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,6.485728E8,1.2084881E9,1.03199E5,1.8550653E0,7.550548E7,1.2738551E4,9.98004E5,-1.1690606E-2,7.505852E8,-2.5585282E-3,1.05055E5,-8.410018E-3,1.7833586E9,1.5650555E-2,1.949E3,1.1799E4,7.284E3,1.5102771E9,1.83E2,8.244377E-3,1.5102771E9,1.2690033E9,1.4694605E-3,3.3E2,3.2698703E-3,2.431E3,4.544533E-3,1.9642604E-3,1.05E3,1.01E2,4.472153E6,7.143424E7,5.1778136E-3,-1.7421684E-4,9.710355E-3,4.8037185E-3,5.08868E5,1.484E3,5.8016E4,3.1E2,6.5012E4,1.484E3,1.16E2,5.8101694E-4,8.0532E4,4.7418696E3,-3.7329813E-4,2.466E3,1.3082312E-1,-2.6981402E-3,5.3688E4,2.9490748E-4,6.752559E-4,1.02E3,1.9E1,-7.119941E-4,-8.592028E-4,2.415193E6,-2.6546607E-3,1.8274775E-5,3.302261E-3,1.3496772E-3,2.1916846E3,1.11301E5,3.1208722E10,3.4090972E-3,8.001483E-3,2.3948455E-3,-5.406959E-3,-2.4721543E-3,-2.709052E-3,7.6E1,-7.185958E-4,5.58501E2,6.144E3,8.1188E4,-1.00909085E-4,1.842633E-3,1.5345627E-3,3.2420168E-4,1.914442E6,-2.9722955E-3,1.354272E-3,-5.602299E-4,1.8476353E-3,4.064727E-4,1.1604029E-3,-1.5437827E-3,-4.4954577E-3,1.0510289E3,7.43E2,-1.6216657E-3,6.059E3,-6.812435E-5,-7.26443E-4,1.6017744E-4,1.91801E5,1.2289E4,4.850767E-4,-3.3015927E-4,1.0018222E3,-1.3709527E-3,-2.3419173E-3,-8.667974E-4,-2.2442548E-3,-3.7403468E-3,-1.4232255E-3,-3.262518E-3],"split_indices":[2,16,16,28,64,2,5,9,0,16,0,3,0,16,0,7,8,7,16,47,0,16,1,0,0,0,0,0,0,17,19,6,16,0,0,0,0,2,20,12,17,8,17,4,0,8,5,0,0,63,0,6,63,0,0,17,0,0,2,0,0,0,0,5,8,1,0,0,0,0,0,0,17,0,5,3,28,0,0,0,0,6,0,0,0,0,0,0,0,0,5,0,0,3,0,0,0,6,7,0,0,5,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.93E2,8.7E1,1.51E2,4.2E1,5.8E1,2.9E1,1.48E2,3E0,3.8E1,4E0,5.5E1,3E0,2.6E1,3E0,8.8E1,6E1,7E0,3.1E1,4.9E1,6E0,1.1E1,1.5E1,3E0,8.5E1,4E0,5.6E1,4E0,3E0,1.9E1,1.2E1,3.8E1,1.1E1,7E0,4E0,6E0,9E0,2.9E1,5.6E1,4.5E1,1.1E1,1.1E1,8E0,9E0,3E0,1.8E1,2E1,3E0,8E0,2.3E1,6E0,1.1E1,4.5E1,3E0,4.2E1,8E0,3E0,4E0,7E0,5E0,3E0,5E0,4E0,1E1,8E0,1.5E1,5E0,5E0,3E0,2E1,3E0,3E0,8E0,5E0,4E1,1.5E1,2.7E1,3E0,5E0,4E0,3E0,6E0,4E0,5E0,3E0,1.2E1,3E0,4E0,4E0,8E0,3.2E1,6E0,9E0,2.4E1,3E0,3E0,3E0,1E1,2.2E1,3E0,3E0,1.6E1,8E0,6E0,4E0,1E1,1.2E1,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[6.9273007E-3,-4.774272E-2,9.7815365E-2,-6.340247E-2,1.7962424E-2,1.1323214E-1,-6.0605626E-3,-5.592008E-2,-8.569176E-3,3.2852953E-3,2.9227743E-3,6.733667E-2,2.0309918E-1,-6.8199106E-2,2.3619829E-2,-1.2660265E-2,2.9587455E-2,9.944182E-3,4.830219E-2,1.4791453E-1,1.050227E-2,2.2637174E-3,-7.335965E-2,-3.3296134E-2,6.4708483E-3,5.834263E-3,-2.3067646E-2,9.402559E-3,2.1761863E-3,-1.3572852E-1,6.8628624E-2,1.8216737E-1,4.8074707E-2,-9.6553594E-2,-4.7349855E-2,1.1045058E-3,-4.4026184E-3,-4.7343728E-4,6.79228E-4,-1.0614787E-3,2.4049556E-4,7.2373805E-4,-1.7895253E-4,-1.1571002E-4,-7.52997E-3,2.1468388E-2,1.0544798E-1,1.4198515E-1,9.571531E-3,-8.6712895E-4,3.1696628E-3,-1.03092946E-1,-3.8708888E-2,3.316196E-4,-5.28788E-2,-2.6824275E-2,1.6466394E-3,4.1232053E-2,-2.500789E-3,1.9153802E-1,7.159189E-2,9.217819E-2,5.802698E-3,-1.3182487E-1,-8.718865E-2,-2.1940717E-3,2.505919E-4,-6.7329206E-2,-3.809986E-2,-2.3025705E-3,2.3987878E-4,-1.3531189E-2,6.02382E-2,1.9675111E-3,7.93644E-3,8.627328E-2,1.9417644E-2,3.7783869E-3,1.0084395E-3,-1.5144035E-1,-2.8447146E-3,-9.438276E-2,-5.2721307E-2,-4.578591E-2,-2.8534727E-3,-1.16338935E-2,-5.8876336E-2,-1.3985031E-3,6.362821E-4,8.003228E-2,-2.3281814E-4,4.028838E-2,3.8702306E-3,-3.723833E-5,1.1310474E-3,-2.786986E-3,-5.66955E-3,-1.2066766E-3,-9.913988E-2,-7.278502E-4,-2.3268291E-3,-6.23089E-4,-6.125423E-2,-3.318604E-2,7.2854274E-4,-2.2477088E-3,-6.700367E-4,1.0576867E-3,1.0258887E-1,2.2256793E-3,1.8043866E-2,-1.6679827E-3,-3.4498249E-3,-2.3441354E-3,-9.2014397E-4,-4.52896E-4,-1.7241604E-3,3.8256114E-3,1.691598E-3,-4.649707E-5,1.0629203E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,17,19,21,23,25,27,-1,29,31,-1,-1,33,35,-1,37,39,41,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,-1,63,65,-1,67,69,-1,71,-1,73,75,77,-1,79,81,-1,-1,83,85,-1,-1,87,89,-1,-1,91,93,-1,-1,95,-1,97,99,101,-1,103,105,-1,-1,107,-1,109,-1,-1,-1,-1,-1,-1,111,-1,-1,-1,113,115,-1,-1,-1,-1,117,-1,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4555749E0,1.8943226E-1,5.064943E-1,2.0970625E-1,3.8187645E-2,4.1939545E-1,0E0,1.4225173E-1,0E0,1.3376067E-2,0E0,3.1158087E-1,1.896385E-1,9.451711E-2,2.0863934E-1,3.9841747E-3,8.573277E-3,0E0,2.5994104E-1,8.01571E-2,0E0,0E0,6.842375E-2,5.7106443E-2,0E0,2.819318E-3,4.180261E-3,1.8865247E-3,0E0,8.9989424E-2,1.03657186E-1,4.5721114E-2,3.289555E-2,1.98403E-2,1.9351348E-2,1.9544397E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.807639E-2,8.5386276E-2,1.3892531E-2,0E0,0E0,0E0,1.40962005E-2,1.1485858E-2,0E0,8.967102E-3,1.4325175E-2,0E0,2.579521E-2,0E0,4.9507737E-2,1.8240526E-2,1.0125034E-2,0E0,1.9574463E-3,4.6945214E-3,0E0,0E0,7.882141E-3,1.5261255E-2,0E0,0E0,7.831753E-3,2.4618357E-2,0E0,0E0,2.639021E-2,2.304683E-3,0E0,0E0,2.3670197E-3,0E0,4.673928E-3,3.6673732E-3,4.9463995E-3,0E0,1.1090557E-2,5.2160397E-3,0E0,0E0,1.1526123E-2,0E0,5.171204E-3,0E0,0E0,0E0,0E0,0E0,0E0,5.091816E-3,0E0,0E0,0E0,1.320038E-3,2.7522426E-3,0E0,0E0,0E0,0E0,1.0764971E-3,0E0,2.090896E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,45,45,46,46,47,47,51,51,52,52,54,54,55,55,57,57,59,59,60,60,61,61,63,63,64,64,67,67,68,68,71,71,72,72,75,75,76,76,79,79,81,81,82,82,83,83,85,85,86,86,89,89,91,91,98,98,102,102,103,103,108,108,110,110],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,18,20,22,24,26,28,-1,30,32,-1,-1,34,36,-1,38,40,42,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,-1,64,66,-1,68,70,-1,72,-1,74,76,78,-1,80,82,-1,-1,84,86,-1,-1,88,90,-1,-1,92,94,-1,-1,96,-1,98,100,102,-1,104,106,-1,-1,108,-1,110,-1,-1,-1,-1,-1,-1,112,-1,-1,-1,114,116,-1,-1,-1,-1,118,-1,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.66887E5,1.039919E6,5.11E2,1.71576E5,1.83E2,7.8908E4,-6.0605626E-3,1.83E2,-8.569176E-3,3.7935E4,2.9227743E-3,2.4540474E8,1.13E2,1E0,7.281275E-1,1.7234E4,5.275E3,9.944182E-3,6.162E3,2.608614E6,1.050227E-2,2.2637174E-3,8.3581E4,6.3346E4,6.4708483E-3,6.5261426E2,5.1012262E8,4.15E2,2.1761863E-3,2.235E3,2.8893403E3,3.590652E9,3.5541443E9,1.9537698E3,5.5659E4,5.3E1,-4.4026184E-3,-4.7343728E-4,6.79228E-4,-1.0614787E-3,2.4049556E-4,7.2373805E-4,-1.7895253E-4,-1.1571002E-4,-7.52997E-3,1E0,1.7655054E-1,4.2520176E11,9.571531E-3,-8.6712895E-4,3.1696628E-3,6.7452E4,1.1750605E8,3.316196E-4,2.83326E9,1.407E3,1.6466394E-3,3.36E5,-2.500789E-3,7.143424E7,5.9233395E10,1.117371E6,5.802698E-3,1.2621093E3,7.11E2,-2.1940717E-3,2.505919E-4,1E2,2.9911E4,-2.3025705E-3,2.3987878E-4,1.038E3,1.83E2,1.9675111E-3,7.93644E-3,9.035389E6,3.0826965E10,3.7783869E-3,1.0084395E-3,1.14903E5,-2.8447146E-3,7.6E1,1.5096E4,1.7234E4,-2.8534727E-3,1.5618E4,4.5291E4,-1.3985031E-3,6.362821E-4,6.546557E9,-2.3281814E-4,2.854041E6,3.8702306E-3,-3.723833E-5,1.1310474E-3,-2.786986E-3,-5.66955E-3,-1.2066766E-3,1.2590113E7,-7.278502E-4,-2.3268291E-3,-6.23089E-4,1.46379E5,1.12297E5,7.2854274E-4,-2.2477088E-3,-6.700367E-4,1.0576867E-3,1.74E2,2.2256793E-3,1.669E3,-1.6679827E-3,-3.4498249E-3,-2.3441354E-3,-9.2014397E-4,-4.52896E-4,-1.7241604E-3,3.8256114E-3,1.691598E-3,-4.649707E-5,1.0629203E-3],"split_indices":[12,6,4,22,47,3,0,47,0,20,0,11,4,38,64,28,3,0,20,12,0,0,12,12,0,5,16,17,0,0,5,16,16,5,8,4,0,0,0,0,0,0,0,0,0,36,63,13,0,0,0,18,16,0,10,28,0,12,0,16,26,12,0,5,0,0,0,4,28,0,0,0,47,0,0,2,1,0,0,21,0,4,7,28,0,28,28,0,0,10,0,2,0,0,0,0,0,0,11,0,0,0,12,12,0,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,1.82E2,1.09E2,1.47E2,3.5E1,1.04E2,5E0,1.43E2,4E0,3E1,5E0,7E1,3.4E1,1.24E2,1.9E1,1.9E1,1.1E1,4E0,6.6E1,2.5E1,9E0,4E0,1.2E2,1.5E1,4E0,7E0,1.2E1,8E0,3E0,6E0,6E1,1.8E1,7E0,6.2E1,5.8E1,1.2E1,3E0,3E0,4E0,9E0,3E0,4E0,4E0,3E0,3E0,2.7E1,3.3E1,1.5E1,3E0,3E0,4E0,5.5E1,7E0,5E0,5.3E1,8E0,4E0,2.3E1,4E0,8E0,2.5E1,8E0,7E0,1.7E1,3.8E1,4E0,3E0,2.5E1,2.8E1,3E0,5E0,6E0,1.7E1,3E0,5E0,1.9E1,6E0,5E0,3E0,1E1,7E0,3E1,8E0,1.4E1,1.1E1,1.3E1,1.5E1,3E0,3E0,1.3E1,4E0,9E0,1E1,3E0,3E0,4E0,6E0,3E0,2.7E1,4E0,4E0,6E0,8E0,8E0,5E0,1.1E1,4E0,5E0,8E0,3E0,6E0,5E0,2.2E1,5E0,3E0,5E0,3E0,5E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[3.8737191E-3,-3.8944427E-2,1.3930018E-1,-6.4313106E-2,2.6394775E-2,1.4893489E-2,1.9979027E-1,-8.7858535E-2,-3.1084541E-2,4.4702124E-2,-7.760231E-3,-1.926285E-3,3.377713E-2,1.5697312E-1,3.7819797E-1,-1.2578364E-1,-6.436053E-2,-1.8462632E-2,-4.896677E-3,9.351351E-3,2.5665998E-2,-1.7645959E-2,5.886824E-2,9.55794E-2,2.1333635E-1,1.5564838E-2,4.891549E-3,-1.5589367E-1,-8.404836E-2,3.387406E-4,-7.176329E-2,-3.5665307E-2,8.2982E-2,5.4019084E-3,8.936191E-3,-1.6899526E-3,6.959437E-4,4.22676E-4,2.7622175E-3,1.17747255E-1,1.7788906E-4,2.5613308E-1,1.3609126E-3,-1.9342972E-3,-5.505854E-3,-3.0188344E-3,-1.4121709E-3,-7.533798E-2,-3.2293325E-4,2.9778725E-2,-5.8243543E-2,3.6025771E-3,6.061191E-4,-5.013034E-2,4.2846274E-2,5.5476846E-3,5.6130726E-2,1.2093596E-2,1.587758E-1,-8.2373865E-2,-3.491477E-2,7.090751E-2,-1.8809485E-3,-1.1094914E-1,-3.3575304E-2,-1.5870157E-1,3.878256E-3,2.4487395E-3,6.494248E-2,4.5677152E-4,2.79545E-3,1.8083738E-3,6.2678265E-3,-8.828342E-2,-1.0682734E-3,-4.505175E-4,-1.5724485E-3,3.2862232E-3,6.214101E-4,-1.5882407E-3,-4.7609434E-3,-1.9860392E-2,-2.983653E-3,-7.66888E-3,-1.270888E-3,3.198148E-2,-7.9538114E-4,-4.6867948E-2,1.9177136E-3,2.664863E-2,8.351235E-2,-1.6152675E-3,-9.575023E-2,1.8518278E-3,-3.365567E-2,2.2731493E-4,1.7747575E-3,5.141962E-4,-3.1542995E-3,-7.329038E-5,1.8071963E-3,8.5734524E-4,9.7912155E-2,-1.00710094E-1,-1.2738324E-3,-4.456231E-2,-4.5586377E-4,4.09193E-3,1.8428756E-3,-3.5010935E-3,-1.7937688E-3,-2.6883997E-4,-1.7572027E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,-1,-1,33,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,-1,-1,-1,-1,-1,55,-1,57,-1,-1,-1,-1,-1,59,-1,61,63,-1,-1,65,67,-1,69,-1,71,73,75,77,-1,79,81,83,85,87,89,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,-1,93,-1,-1,-1,95,-1,97,-1,99,101,-1,103,-1,105,-1,-1,-1,-1,-1,-1,-1,107,109,-1,111,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6174663E0,3.5310218E-1,5.012244E-1,1.1697328E-1,3.0458617E-1,3.4481168E-2,2.8438413E-1,7.121813E-2,9.649416E-2,2.5926435E-1,0E0,0E0,2.5553772E-2,1.1110264E-1,1.17185116E-1,2.8366864E-2,3.2353073E-2,1.07666716E-1,0E0,0E0,2.7420008E-1,1.0715558E-2,1.4893834E-2,3.9274916E-2,1.2826431E-1,0E0,0E0,1.5853822E-2,2.1444336E-3,0E0,1.07189715E-2,7.857017E-2,1.460021E-2,1.0819577E-1,0E0,0E0,0E0,0E0,0E0,4.415442E-2,0E0,1.1496687E-1,0E0,0E0,0E0,0E0,0E0,1.1060178E-2,0E0,5.4936938E-2,4.6980873E-2,0E0,0E0,1.2375488E-1,2.7512576E-2,0E0,1.0361165E-2,0E0,2.5937676E-2,8.987516E-3,1.9399589E-3,1.4120877E-2,0E0,2.0024255E-2,2.0860877E-2,5.697596E-2,1.2841482E-2,3.660061E-2,1.1996746E-2,0E0,0E0,0E0,0E0,5.437374E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7559105E-2,0E0,0E0,0E0,4.2265663E-3,0E0,2.4055159E-2,0E0,6.9530313E-3,8.0394745E-3,0E0,4.0947497E-3,0E0,3.8336962E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.598923E-3,3.3619404E-3,0E0,4.264556E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,32,32,33,33,39,39,41,41,47,47,49,49,50,50,53,53,54,54,56,56,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,73,73,81,81,85,85,87,87,89,89,90,90,92,92,94,94,102,102,103,103,105,105],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,-1,-1,34,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,-1,-1,-1,-1,-1,56,-1,58,-1,-1,-1,-1,-1,60,-1,62,64,-1,-1,66,68,-1,70,-1,72,74,76,78,-1,80,82,84,86,88,90,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,-1,94,-1,-1,-1,96,-1,98,-1,100,102,-1,104,-1,106,-1,-1,-1,-1,-1,-1,-1,108,110,-1,112,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.0629734E8,1.4405825E9,1.315E3,1.012905E6,8.391885E8,1.3316E4,1.1170185E4,9.4202E4,1.33097E5,1.4933E4,-7.760231E-3,-1.926285E-3,1E0,1.8943199E9,1.3639678E0,1.2745082E3,1.2E1,7.205049E6,-4.896677E-3,9.351351E-3,7.8667163E3,5.8882404E7,8.13E2,4.9E2,2.2799654E0,1.5564838E-2,4.891549E-3,9.424458E-5,1.0203955E8,3.387406E-4,7.208E3,4.4021126E8,1.58E2,3.044519E6,8.936191E-3,-1.6899526E-3,6.959437E-4,4.22676E-4,2.7622175E-3,1.275083E9,1.7788906E-4,9.068E3,1.3609126E-3,-1.9342972E-3,-5.505854E-3,-3.0188344E-3,-1.4121709E-3,2.8809104E8,-3.2293325E-4,8.383E3,6.39277E5,3.6025771E-3,6.061191E-4,2.2728746E-1,5.8E1,5.5476846E-3,6.1501E4,1.2093596E-2,6.5406E4,5.2773176E7,1.2391E4,1.746735E6,-1.8809485E-3,1.6506441E9,6.254E3,1.5489E4,2.1467798E9,4.19E2,1.875E3,4.5677152E-4,2.79545E-3,1.8083738E-3,6.2678265E-3,3.24279E5,-1.0682734E-3,-4.505175E-4,-1.5724485E-3,3.2862232E-3,6.214101E-4,-1.5882407E-3,-4.7609434E-3,1.659826E9,-2.983653E-3,-7.66888E-3,-1.270888E-3,1.555368E8,-7.9538114E-4,1E2,1.9177136E-3,3.47989E5,5.86E3,-1.6152675E-3,7.7612E4,1.8518278E-3,1.9557E4,2.2731493E-4,1.7747575E-3,5.141962E-4,-3.1542995E-3,-7.329038E-5,1.8071963E-3,8.5734524E-4,9.489204E8,8.259344E7,-1.2738324E-3,3E1,-4.5586377E-4,4.09193E-3,1.8428756E-3,-3.5010935E-3,-1.7937688E-3,-2.6883997E-4,-1.7572027E-3],"split_indices":[16,25,0,9,16,7,5,18,8,8,0,0,42,16,59,5,17,2,0,0,5,18,17,17,67,0,0,63,16,0,3,11,4,2,0,0,0,0,0,16,0,7,0,0,0,0,0,11,0,20,6,0,0,63,4,0,8,0,8,18,28,2,0,11,3,7,10,19,0,0,0,0,0,2,0,0,0,0,0,0,0,11,0,0,0,11,0,19,0,12,3,0,8,0,7,0,0,0,0,0,0,0,1,21,0,17,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.11E2,6.6E1,1.52E2,5.9E1,2.2E1,4.4E1,8.8E1,6.4E1,5.6E1,3E0,4E0,1.8E1,3.7E1,7E0,3.2E1,5.6E1,5.9E1,5E0,3E0,5.3E1,6E0,1.2E1,1.9E1,1.8E1,4E0,3E0,1.7E1,1.5E1,5E0,5.1E1,5.1E1,8E0,5E1,3E0,3E0,3E0,5E0,7E0,1.5E1,4E0,1.4E1,4E0,3E0,1.4E1,1.1E1,4E0,4.8E1,3E0,1.3E1,3.8E1,5E0,3E0,2E1,3E1,7E0,8E0,5E0,9E0,4E1,8E0,9E0,4E0,1.1E1,2.7E1,6E0,1.4E1,1.1E1,1.9E1,4E0,4E0,3E0,6E0,3.5E1,5E0,4E0,4E0,5E0,4E0,5E0,6E0,2.3E1,4E0,3E0,3E0,7E0,7E0,6E0,5E0,7E0,1.2E1,7E0,2.8E1,3E0,2E1,4E0,3E0,3E0,3E0,4E0,3E0,3E0,9E0,2.5E1,3E0,1.2E1,8E0,4E0,5E0,2E1,5E0,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[1.8033838E-2,-1.2599683E-2,1.708094E-1,-3.630724E-2,1.0443753E-1,1.3241641E-1,2.957825E-1,-7.866291E-2,6.8678274E-3,2.8943673E-1,5.4924052E-2,1.06632136E-1,7.978457E-3,1.0808821E-2,4.8868926E-3,-1.21921495E-1,-5.227626E-2,1.7336687E-2,-7.669024E-3,4.521497E-3,1.128947E-2,8.460676E-2,-6.317229E-2,1.1965811E-1,9.734977E-4,-6.1384537E-3,-1.0180899E-1,-6.2180724E-2,-7.027902E-4,-1.4651027E-2,1.0492931E-1,1.0064778E-2,1.7073366E-1,1.9343691E-3,-5.492907E-3,1.299201E-1,1.5618518E-3,-1.10040344E-1,-6.693724E-4,-7.5177446E-2,-2.82395E-2,2.0874105E-3,-4.7424845E-2,-2.742876E-2,4.2501416E-2,7.1176365E-2,1.484411E-1,-2.0657384E-3,4.289532E-2,2.6500158E-3,6.587349E-3,1.4897133E-1,1.6452759E-3,-7.3753305E-2,-1.2408646E-1,-8.7235443E-4,-8.184248E-2,-3.755101E-2,2.923651E-4,-2.6363705E-3,-3.5103156E-5,-4.394005E-2,2.1732192E-2,-4.5319408E-4,1.959777E-3,3.1802815E-3,4.4721913E-2,5.754791E-3,2.6505794E-3,2.0755648E-3,1.318722E-4,5.484393E-3,2.5028451E-3,-2.8874287E-3,-1.2831696E-3,-4.484857E-3,-2.5825095E-3,-8.7281756E-2,-7.0645433E-4,-1.7208247E-2,-2.1456443E-3,-2.1019524E-2,-7.775853E-2,2.108377E-3,5.331903E-3,1.6962312E-3,6.3047494E-4,-9.5861E-2,-1.236629E-3,2.2777995E-5,-8.460586E-4,-3.0548003E-2,3.4019546E-4,-8.916258E-4,-9.6308045E-2,-4.748843E-4,1.7218491E-2,-1.2636072E-3,-1.0207812E-1,-1.8124849E-2,-1.6997394E-3,-3.7306037E-3,-1.4711675E-3,-2.2368611E-5,1.1364514E-3,-1.1331672E-1,-2.041192E-3,-1.2188625E-3,-2.4154437E-4,-4.20928E-3,-2.103402E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,29,-1,-1,-1,31,33,35,-1,-1,37,39,41,43,45,47,49,-1,-1,51,-1,53,-1,55,57,-1,59,61,63,65,67,-1,69,-1,-1,71,-1,73,75,-1,77,79,-1,-1,-1,81,83,-1,-1,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,87,-1,89,-1,91,93,-1,95,-1,-1,97,-1,-1,-1,99,-1,-1,101,-1,103,-1,105,107,-1,-1,-1,-1,-1,109,-1,-1,-1,-1,-1],"loss_changes":[1.2891667E0,6.4196515E-1,1.8154585E-1,3.5165235E-1,3.4134114E-1,8.370811E-2,6.620586E-3,1.0434693E-1,2.5047663E-1,2.377516E-2,1.1751821E-1,2.8222442E-2,0E0,0E0,0E0,3.0384481E-2,3.162636E-2,2.621129E-1,0E0,0E0,0E0,1.6520345E-1,1.0298988E-1,1.3406217E-2,0E0,0E0,1.7717361E-2,2.0815521E-2,3.6765113E-2,5.155498E-2,2.6150793E-2,3.9312992E-2,1.9129306E-2,0E0,0E0,2.691722E-2,0E0,4.609883E-3,0E0,9.637907E-3,5.901495E-3,0E0,1.1093315E-2,4.722243E-2,1.4378099E-2,7.2739124E-3,5.3323805E-3,0E0,8.948091E-3,0E0,0E0,1.419726E-2,0E0,1.8292442E-3,8.54522E-4,0E0,8.509621E-3,6.550392E-3,0E0,0E0,0E0,3.169646E-2,1.0417832E-2,0E0,0E0,0E0,9.721797E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.474514E-3,0E0,1.4820204E-3,0E0,8.449521E-3,1.3117917E-2,0E0,3.1189187E-3,0E0,0E0,4.438907E-3,0E0,0E0,0E0,4.849473E-3,0E0,0E0,6.5983757E-3,0E0,2.616552E-3,0E0,1.939699E-3,2.8608646E-3,0E0,0E0,0E0,0E0,0E0,4.0121824E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,21,21,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,39,39,40,40,42,42,43,43,44,44,45,45,46,46,48,48,51,51,53,53,54,54,56,56,57,57,61,61,62,62,66,66,77,77,79,79,81,81,82,82,84,84,87,87,91,91,94,94,96,96,98,98,99,99,105,105],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,30,-1,-1,-1,32,34,36,-1,-1,38,40,42,44,46,48,50,-1,-1,52,-1,54,-1,56,58,-1,60,62,64,66,68,-1,70,-1,-1,72,-1,74,76,-1,78,80,-1,-1,-1,82,84,-1,-1,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,88,-1,90,-1,92,94,-1,96,-1,-1,98,-1,-1,-1,100,-1,-1,102,-1,104,-1,106,108,-1,-1,-1,-1,-1,110,-1,-1,-1,-1,-1],"split_conditions":[9.28437E5,1.83E2,1.1741724E4,1.530129E6,2.4E1,3.9743017E9,1.4817759E9,1.49015E5,8.23838E5,6E1,1.1882E4,1.02E2,7.978457E-3,1.0808821E-2,4.8868926E-3,6.6362006E2,6.910113E8,4.365884E6,-7.669024E-3,4.521497E-3,1.128947E-2,4.352E3,4.9E1,8.28337E5,9.734977E-4,-6.1384537E-3,9.2701E4,5.209E3,7.031E3,4.2524388E7,1.2065335E7,4.57473E5,2.68331E5,1.9343691E-3,-5.492907E-3,3.81E2,1.5618518E-3,5.724E4,-6.693724E-4,6.24933E5,4.68E2,2.0874105E-3,1.31E2,2.34E3,4.79E2,9.4754234E8,7.9937E4,-2.0657384E-3,3.4311706E9,2.6500158E-3,6.587349E-3,2.2799654E0,1.6452759E-3,1.0033E4,1.2621093E3,-8.7235443E-4,6.2498252E7,7.6E1,2.923651E-4,-2.6363705E-3,-3.5103156E-5,2.203964E9,1.2791078E10,-4.5319408E-4,1.959777E-3,3.1802815E-3,2.5420546E7,5.754791E-3,2.6505794E-3,2.0755648E-3,1.318722E-4,5.484393E-3,2.5028451E-3,-2.8874287E-3,-1.2831696E-3,-4.484857E-3,-2.5825095E-3,1.105E3,-7.0645433E-4,6.6825376E8,-2.1456443E-3,2.6304E4,1.62E2,2.108377E-3,1.443451E6,1.6962312E-3,6.3047494E-4,4.1677676E7,-1.236629E-3,2.2777995E-5,-8.460586E-4,1.079E3,3.4019546E-4,-8.916258E-4,5.15E2,-4.748843E-4,2.930405E-5,-1.2636072E-3,6.6966E4,6.1346E4,-1.6997394E-3,-3.7306037E-3,-1.4711675E-3,-2.2368611E-5,1.1364514E-3,6.7E1,-2.041192E-3,-1.2188625E-3,-2.4154437E-4,-4.20928E-3,-2.103402E-3],"split_indices":[12,47,5,9,4,16,16,18,12,17,3,4,0,0,0,5,16,6,0,0,0,3,4,28,0,0,8,3,3,18,6,21,6,0,0,17,0,8,0,18,4,0,4,0,17,27,8,0,10,0,0,67,0,7,5,0,18,4,0,0,0,11,10,0,0,0,18,0,0,0,0,0,0,0,0,0,0,17,0,10,0,20,17,0,6,0,0,10,0,0,0,17,0,0,17,0,63,0,8,8,0,0,0,0,0,19,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.29E2,4.5E1,1.91E2,3.8E1,3.6E1,9E0,9.6E1,9.5E1,7E0,3.1E1,3.1E1,5E0,6E0,3E0,3.5E1,6.1E1,9.2E1,3E0,3E0,4E0,2.5E1,6E0,2.6E1,5E0,6E0,2.9E1,5.1E1,1E1,6.8E1,2.4E1,1.4E1,1.1E1,3E0,3E0,2.2E1,4E0,2.6E1,3E0,3.6E1,1.5E1,4E0,6E0,5.6E1,1.2E1,1.5E1,9E0,4E0,1E1,4E0,7E0,1.7E1,5E0,9E0,1.7E1,5E0,3.1E1,1.2E1,3E0,3E0,3E0,4.2E1,1.4E1,3E0,9E0,6E0,9E0,5E0,4E0,6E0,4E0,1.2E1,5E0,5E0,4E0,1.1E1,6E0,2.8E1,3E0,8E0,4E0,2.6E1,1.6E1,3E0,1.1E1,6E0,3E0,2.3E1,5E0,3E0,5E0,2E1,6E0,5E0,1.1E1,4E0,7E0,3E0,2E1,1.4E1,6E0,7E0,4E0,4E0,3E0,1.4E1,6E0,4E0,1E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-1.0433148E-2,-5.4804154E-2,7.3434904E-2,-6.040527E-2,6.880313E-3,8.8762574E-2,-7.384541E-3,-5.5090908E-2,-6.666699E-3,-3.2474434E-3,9.831241E-2,-6.894237E-2,3.076214E-3,9.230409E-3,8.385382E-2,-3.4721326E-2,-8.696742E-2,5.6288037E-3,-3.138459E-2,2.2346115E-2,1.2245782E-1,-4.7092013E-2,2.121494E-3,-1.354416E-1,-7.094799E-2,-4.1635912E-2,1.4696831E-3,5.5817984E-2,-6.3568175E-2,1.5314853E-1,8.3185904E-2,-8.095411E-2,-1.2678671E-2,-5.1799985E-3,-7.073063E-2,-5.9578955E-2,-1.1379265E-1,-3.1182723E-2,-2.3505974E-3,1.0338519E-2,1.3552165E-1,-3.726293E-3,-3.0507793E-4,1.08274974E-1,2.1265541E-1,7.059435E-5,1.0332444E-1,-4.1381303E-2,-3.4832174E-3,-2.163077E-2,1.2067483E-3,-2.7681198E-3,-1.0933407E-3,-6.69605E-2,5.402848E-4,-5.470876E-3,-2.6724213E-3,-3.9101105E-2,-8.62889E-3,-6.010127E-3,1.3298136E-3,6.2252614E-3,4.7728483E-4,1.4506023E-1,5.537192E-2,7.765465E-3,3.2932495E-3,1.2145766E-1,1.3123654E-3,-5.5224367E-4,-1.6987162E-3,-2.5631018E-2,5.72335E-5,-7.066629E-2,-9.565001E-5,-3.8030726E-4,-4.4489104E-2,2.1575908E-4,-7.018303E-4,-1.281357E-3,1.5864402E-2,5.80136E-3,2.377043E-3,2.3153555E-3,5.37287E-4,2.138822E-3,1.3542143E-1,-1.0645317E-3,-2.966091E-4,-7.473211E-2,-1.3551141E-4,-1.633035E-3,-5.716007E-4,8.529384E-4,1.3470303E-4,4.870707E-3,2.2460418E-3,-5.308393E-2,-8.595156E-2,-2.59983E-3,-3.742772E-2,-3.8713946E-3,-6.800494E-2,-4.6821753E-4,-1.3630692E-3,-2.4968493E-3,-4.55722E-2,-1.7610253E-3,-6.5903587E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,15,17,-1,19,21,23,-1,25,27,29,31,-1,33,35,37,-1,39,41,43,45,47,49,-1,51,53,55,57,-1,59,61,-1,-1,63,65,-1,67,69,-1,71,-1,-1,-1,73,-1,-1,-1,75,77,79,-1,-1,-1,81,83,-1,-1,85,-1,-1,-1,87,-1,89,-1,-1,91,-1,-1,-1,93,-1,-1,-1,-1,-1,95,-1,-1,97,-1,-1,-1,-1,-1,-1,-1,99,101,-1,103,-1,105,-1,-1,-1,107,-1,-1],"loss_changes":[1.0534339E0,2.9118645E-1,4.7527337E-1,1.3048619E-1,0E0,1.8016374E-1,0E0,1.428209E-1,0E0,0E0,2.2117221E-1,8.518797E-2,2.1269049E-1,0E0,1.9909126E-1,6.6590965E-2,6.2607884E-2,0E0,2.5594663E-2,1.0132246E-1,4.9934924E-2,5.257778E-2,0E0,2.4896473E-2,2.8291762E-2,6.582007E-3,0E0,8.922091E-2,2.7135294E-2,5.1742733E-2,4.055032E-2,2.0331085E-2,1.1589579E-2,0E0,1.5687831E-3,3.4381196E-2,1.0085419E-2,3.5915766E-3,0E0,8.856863E-3,5.9999734E-2,0E0,0E0,2.6996687E-2,1.0021657E-2,0E0,1.7734095E-2,2.1302756E-3,0E0,2.0120358E-3,0E0,0E0,0E0,1.2595028E-2,0E0,0E0,0E0,1.8266626E-3,1.5600722E-3,9.503098E-3,0E0,0E0,0E0,1.1735216E-2,4.5636427E-3,0E0,0E0,2.5244057E-3,0E0,0E0,0E0,1.8938361E-3,0E0,1.3465583E-2,0E0,0E0,1.2688339E-3,0E0,0E0,0E0,8.826726E-4,0E0,0E0,0E0,0E0,0E0,9.190291E-4,0E0,0E0,7.095635E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.212102E-3,1.0298461E-2,0E0,9.595435E-4,0E0,1.1848435E-3,0E0,0E0,0E0,9.437427E-4,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,39,39,40,40,43,43,44,44,46,46,47,47,49,49,53,53,57,57,58,58,59,59,63,63,64,64,67,67,71,71,73,73,76,76,80,80,86,86,89,89,97,97,98,98,100,100,102,102,106,106],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,16,18,-1,20,22,24,-1,26,28,30,32,-1,34,36,38,-1,40,42,44,46,48,50,-1,52,54,56,58,-1,60,62,-1,-1,64,66,-1,68,70,-1,72,-1,-1,-1,74,-1,-1,-1,76,78,80,-1,-1,-1,82,84,-1,-1,86,-1,-1,-1,88,-1,90,-1,-1,92,-1,-1,-1,94,-1,-1,-1,-1,-1,96,-1,-1,98,-1,-1,-1,-1,-1,-1,-1,100,102,-1,104,-1,106,-1,-1,-1,108,-1,-1],"split_conditions":[2.88414E5,1.1613266E9,4.84E2,1.03199E5,6.880313E-3,8.14E2,-7.384541E-3,7.992919E8,-6.666699E-3,-3.2474434E-3,2.59E3,7.318324E-4,8.563E3,9.230409E-3,6.371693E6,1.0347748E-3,7.325E4,5.6288037E-3,5.581883E7,2.0397697E10,5.5409517E0,4.695E5,2.121494E-3,2.2506252E7,3.8434416E8,6.402839E-1,1.4696831E-3,1.83E2,6.1E2,2.009388E6,1.306E4,1.34E2,1.5801E4,-5.1799985E-3,7.6E1,1.83E2,4.0449E4,2.462322E3,-2.3505974E-3,4.93153E5,2.9713324E9,-3.726293E-3,-3.0507793E-4,1.2690033E9,1.074E3,7.059435E-5,8.28337E5,5.1279686E8,-3.4832174E-3,1.267716E7,1.2067483E-3,-2.7681198E-3,-1.0933407E-3,1.9273E4,5.402848E-4,-5.470876E-3,-2.6724213E-3,3.3704907E-2,6.5012E4,1.3707E4,1.3298136E-3,6.2252614E-3,4.7728483E-4,1.1086E4,1E0,7.765465E-3,3.2932495E-3,8.2402E4,1.3123654E-3,-5.5224367E-4,-1.6987162E-3,1.179118E6,5.72335E-5,1.369E3,-9.565001E-5,-3.8030726E-4,2.21839E5,2.1575908E-4,-7.018303E-4,-1.281357E-3,3.2353126E9,5.80136E-3,2.377043E-3,2.3153555E-3,5.37287E-4,2.138822E-3,8.297E4,-1.0645317E-3,-2.966091E-4,1.2482E4,-1.3551141E-4,-1.633035E-3,-5.716007E-4,8.529384E-4,1.3470303E-4,4.870707E-3,2.2460418E-3,4.7274544E7,2.1047948E7,-2.59983E-3,1.67E2,-3.8713946E-3,1.268704E-2,-4.6821753E-4,-1.3630692E-3,-2.4968493E-3,3.916E3,-1.7610253E-3,-6.5903587E-4],"split_indices":[12,16,4,28,0,28,0,23,0,0,20,63,7,0,6,64,18,0,18,10,60,6,0,23,23,63,0,47,17,12,7,4,0,0,4,47,8,5,0,12,11,0,0,1,19,0,28,10,0,18,0,0,0,7,0,0,0,63,8,7,0,0,0,0,36,0,0,3,0,0,0,6,0,17,0,0,12,0,0,0,10,0,0,0,0,0,8,0,0,7,0,0,0,0,0,0,0,16,11,0,17,0,63,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.84E2,9.7E1,1.81E2,3E0,9.3E1,4E0,1.76E2,5E0,4E0,8.9E1,1.42E2,3.4E1,5E0,8.4E1,5E1,9.2E1,5E0,2.9E1,3.3E1,5.1E1,4.5E1,5E0,2.1E1,7.1E1,2.6E1,3E0,2.4E1,9E0,2.7E1,2.4E1,2.2E1,2.3E1,1.4E1,7E0,5.8E1,1.3E1,2.1E1,5E0,1.6E1,8E0,4E0,5E0,1.7E1,1E1,5E0,1.9E1,1E1,1.2E1,2E1,3E0,4E0,3E0,5.3E1,5E0,3E0,1E1,1.5E1,6E0,1.1E1,5E0,5E0,3E0,9E0,8E0,7E0,3E0,1.4E1,5E0,4E0,6E0,1.7E1,3E0,5E1,3E0,3E0,1.2E1,3E0,3E0,4E0,7E0,5E0,4E0,5E0,3E0,4E0,1E1,1.1E1,6E0,4.7E1,3E0,9E0,3E0,3E0,4E0,7E0,3E0,1.8E1,2.9E1,5E0,1.3E1,8E0,2.1E1,3E0,1E1,1.3E1,8E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-6.6194334E-4,-5.010691E-2,7.817637E-2,-5.694665E-2,7.927143E-3,8.907059E-2,-5.1406072E-3,-7.668531E-2,-2.6636016E-2,7.6816015E-2,1.1913252E-2,1.5609897E-3,-8.1056796E-2,-9.867629E-3,-9.553027E-2,2.1161687E-1,5.756117E-2,-9.0939805E-2,-2.919911E-2,-2.230211E-2,3.0707512E-3,-5.625189E-3,-5.428726E-2,1.101773E-2,1.0349851E-1,-7.92712E-3,9.15711E-2,-1.16409846E-1,-6.902978E-2,-1.5438158E-2,-2.2552444E-3,1.1567842E-3,-2.6784142E-2,-2.3415233E-3,-2.709601E-4,4.328752E-4,4.984115E-3,2.5197314E-2,-1.2509622E-1,6.9238365E-2,1.8069838E-1,-4.3327652E-4,-1.288579E-1,-5.5876747E-3,-8.027355E-2,-7.6247717E-4,3.575284E-4,-6.1336987E-2,-1.4522524E-2,8.570132E-3,6.672862E-2,-7.1896445E-3,1.4288955E-4,2.6876481E-2,9.375961E-2,6.8129892E-3,2.369376E-3,-9.200001E-2,-4.6282397E-3,5.8663543E-4,-1.0930172E-3,-7.721044E-5,-8.669649E-2,-7.3079765E-2,-6.3030777E-4,1.5383001E-3,-2.1635441E-2,2.011614E-3,-9.975804E-3,3.2839112E-3,4.749573E-4,4.9912907E-2,-1.5324551E-3,1.2879553E-1,6.3247025E-2,-3.4434497E-3,-1.5978031E-3,-4.0222295E-3,-6.931502E-2,-2.937209E-3,-8.8700023E-4,-2.856852E-2,4.6110124E-4,4.254455E-3,-1.6150048E-3,8.516503E-2,8.421674E-3,5.013195E-3,1.7553143E-3,4.1961968E-2,3.1810252E-3,-9.87977E-5,-2.4936094E-3,-3.4197256E-2,4.4270186E-4,-4.5745992E-4,4.913322E-4,1.3042907E-3,3.3427654E-3,1.3695835E-3,-1.1698096E-3,1.9964906E-3,5.119746E-4,-3.8608458E-2,-4.4624743E-5,-1.9715042E-3,-2.6290702E-2,1.2727112E-4,-3.3574365E-2,-3.686161E-4,-1.2574946E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,-1,17,19,21,23,25,27,29,31,-1,-1,33,-1,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,49,51,53,55,-1,57,59,61,-1,-1,63,65,67,69,-1,-1,71,73,-1,-1,75,-1,-1,-1,-1,77,79,-1,-1,81,-1,83,-1,-1,85,-1,87,89,-1,-1,-1,91,-1,-1,93,-1,95,-1,97,99,-1,-1,101,-1,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,105,-1,-1,107,-1,109,-1,-1],"loss_changes":[1.0720124E0,3.5660988E-1,2.8892237E-1,9.727377E-2,0E0,3.2285213E-1,0E0,5.8652997E-2,7.645878E-2,2.4478048E-1,0E0,0E0,4.7058105E-2,7.411812E-2,3.2990217E-2,1.3410294E-1,1.9672409E-1,3.7166238E-2,8.477826E-3,1.4014097E-2,0E0,0E0,7.853238E-3,0E0,3.4903847E-2,1.2346806E-1,1.02935195E-1,4.4400245E-2,3.2364562E-2,3.3288563E-3,0E0,0E0,1.9021902E-2,0E0,0E0,0E0,0E0,1.6691845E-2,9.008963E-2,4.7340855E-2,2.088824E-2,0E0,4.3759346E-3,6.023043E-3,1.9299299E-2,0E0,0E0,4.4528954E-3,1.698993E-2,1.96247E-2,1.1334069E-2,0E0,0E0,3.497757E-2,2.3925155E-2,0E0,0E0,2.3028702E-3,0E0,0E0,0E0,0E0,1.5279353E-2,4.9297027E-3,0E0,0E0,8.636647E-3,0E0,8.793627E-3,0E0,0E0,2.1222573E-2,0E0,1.7522752E-2,9.4836205E-3,0E0,0E0,0E0,1.4344662E-2,0E0,0E0,7.140424E-3,0E0,2.6565746E-3,0E0,2.4650022E-3,1.3767315E-2,0E0,0E0,5.809266E-3,0E0,0E0,0E0,3.5648681E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5248084E-3,0E0,0E0,3.657409E-3,0E0,1.1320068E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,24,24,25,25,26,26,27,27,28,28,29,29,32,32,37,37,38,38,39,39,40,40,42,42,43,43,44,44,47,47,48,48,49,49,50,50,53,53,54,54,57,57,62,62,63,63,66,66,68,68,71,71,73,73,74,74,78,78,81,81,83,83,85,85,86,86,89,89,93,93,103,103,106,106,108,108],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,-1,18,20,22,24,26,28,30,32,-1,-1,34,-1,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,50,52,54,56,-1,58,60,62,-1,-1,64,66,68,70,-1,-1,72,74,-1,-1,76,-1,-1,-1,-1,78,80,-1,-1,82,-1,84,-1,-1,86,-1,88,90,-1,-1,-1,92,-1,-1,94,-1,96,-1,98,100,-1,-1,102,-1,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,106,-1,-1,108,-1,110,-1,-1],"split_conditions":[2.88414E5,1.5102771E9,5.3408066E4,5.013E3,7.927143E-3,1.6189593E4,-5.1406072E-3,2.302E3,9.6E2,5.345E3,1.1913252E-2,1.5609897E-3,7.11868E5,1.83E2,1.5915E4,2.8543744E9,8.1188E4,1.49015E5,1.4366396E3,1.9452692E7,3.0707512E-3,-5.625189E-3,1.039919E6,1.101773E-2,2.7538784E3,5.96025E5,9.1697E4,6E0,3.5888664E7,3.8594732E7,-2.2552444E-3,1.1567842E-3,2.34725E5,-2.3415233E-3,-2.709601E-4,4.328752E-4,4.984115E-3,2.328E3,8.9003E4,2.029183E0,1.5831415E-1,-4.3327652E-4,5.6E1,1.09E2,2.253E3,-7.6247717E-4,3.575284E-4,3.649951E9,1.036345E6,8.1E2,4.1E2,-7.1896445E-3,1.4288955E-4,6.72444E5,5.37E2,6.8129892E-3,2.369376E-3,2.6E1,-4.6282397E-3,5.8663543E-4,-1.0930172E-3,-7.721044E-5,5.0529E4,4.425E3,-6.3030777E-4,1.5383001E-3,1.521867E6,2.011614E-3,1.27E2,3.2839112E-3,4.749573E-4,3.3730156E3,-1.5324551E-3,7.9937E4,4.293289E7,-3.4434497E-3,-1.5978031E-3,-4.0222295E-3,2.9490748E-4,-2.937209E-3,-8.8700023E-4,6.2498252E7,4.6110124E-4,3.1043533E8,-1.6150048E-3,1E2,5.4066E4,5.013195E-3,1.7553143E-3,1.667E3,3.1810252E-3,-9.87977E-5,-2.4936094E-3,1E0,4.4270186E-4,-4.5745992E-4,4.913322E-4,1.3042907E-3,3.3427654E-3,1.3695835E-3,-1.1698096E-3,1.9964906E-3,5.119746E-4,2.83326E9,-4.4624743E-5,-1.9715042E-3,5.395E3,1.2727112E-4,5.788302E8,-3.686161E-4,-1.2574946E-3],"split_indices":[12,16,5,3,0,5,0,7,19,7,0,0,6,47,20,10,28,18,5,11,0,0,6,0,5,12,8,17,16,27,0,0,6,0,0,0,0,0,7,67,63,0,17,4,3,0,0,10,2,0,17,0,0,28,19,0,0,17,0,0,0,0,8,28,0,0,6,0,4,0,0,5,0,8,2,0,0,0,63,0,0,18,0,21,0,4,8,0,0,19,0,0,0,34,0,0,0,0,0,0,0,0,0,10,0,0,3,0,11,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.68E2,1.05E2,1.65E2,3E0,1.01E2,4E0,9.9E1,6.6E1,9.8E1,3E0,3E0,9.6E1,5.4E1,1.2E1,1.1E1,8.7E1,8E1,1.6E1,4.9E1,5E0,3E0,9E0,4E0,7E0,3E1,5.7E1,3.5E1,4.5E1,1.3E1,3E0,3E0,4.6E1,6E0,3E0,3E0,4E0,2.4E1,6E0,4.7E1,1E1,4E0,3.1E1,7E0,3.8E1,1E1,3E0,1.1E1,3.5E1,1.8E1,6E0,3E0,3E0,1.8E1,2.9E1,7E0,3E0,1.1E1,2E1,4E0,3E0,3E0,3.5E1,8E0,3E0,3E0,3.2E1,4E0,1.4E1,3E0,3E0,1.4E1,4E0,1.2E1,1.7E1,7E0,4E0,9E0,2.6E1,5E0,3E0,2.7E1,5E0,1.1E1,3E0,7E0,7E0,8E0,4E0,1.2E1,5E0,3E0,2.3E1,2.4E1,3E0,4E0,7E0,3E0,4E0,4E0,3E0,6E0,6E0,2.1E1,3E0,6E0,1.5E1,3E0,1.2E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-1.7301029E-3,-3.7458308E-2,1.2304516E-1,-8.9646176E-2,-9.347313E-3,1.4020573E-1,-4.762811E-3,-1.523063E-1,-5.0918568E-2,-2.0313853E-2,1.2976584E-1,1.0368874E-1,2.2501767E-1,-9.622156E-2,-1.9751205E-1,-7.947581E-2,2.4717893E-3,-5.186764E-2,1.8342893E-3,7.835364E-3,4.628392E-2,1.4259118E-1,7.308753E-2,1.0600216E-2,1.2950978E-1,-3.9504142E-3,-6.1002687E-2,-7.924401E-3,-3.6589564E-3,1.1570258E-3,-9.363162E-2,2.3013796E-3,-1.7425949E-2,1.5010783E-3,-5.8850013E-2,1.6105162E-2,-4.7801934E-2,2.534107E-4,2.353794E-3,1.8036963E-1,7.754421E-2,-2.908404E-4,8.469681E-2,1.1716922E-3,5.9715644E-3,-2.643138E-3,-6.233036E-4,-1.0331461E-1,-1.3687407E-3,-1.0851531E-3,5.997214E-4,-3.1511406E-3,-4.0434986E-2,3.192191E-3,8.260959E-3,-7.775875E-3,-5.663411E-3,6.6362056E-3,2.9014165E-3,1.0750045E-3,3.6483794E-3,6.1489776E-2,3.7861587E-3,-1.8532585E-3,-1.133809E-1,-3.1099442E-2,-3.3801815E-3,8.249514E-4,2.654174E-3,-1.4346463E-3,2.5390545E-2,2.7111832E-3,2.3471806E-2,-4.353591E-3,-2.540123E-3,9.266274E-4,-4.0682778E-2,-9.358122E-3,5.672874E-2,1.5019402E-3,-2.4283469E-4,1.3861867E-3,-6.400392E-5,-4.7149867E-2,3.1890362E-4,1.0808539E-3,-1.3318805E-2,6.9634814E-4,2.3645838E-3,-3.024446E-2,-2.2366026E-3,-3.383493E-2,-5.60925E-3,-4.0249527E-2,3.16518E-4,-1.3686289E-3,-2.5779806E-4,-1.8981414E-2,1.4366161E-2,-2.01634E-4,-1.7593894E-3,-1.246073E-3,-1.1139393E-2,1.4422152E-3,7.5414777E-4,-9.091306E-4,-2.352102E-3,-3.0044018E-4,2.9082553E-4,3.9785705E-4,-3.0303214E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,-1,43,-1,45,-1,-1,-1,47,-1,49,-1,51,53,55,-1,-1,57,59,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,65,-1,67,69,-1,-1,-1,-1,-1,71,-1,-1,73,75,-1,77,-1,-1,79,-1,81,-1,-1,-1,83,85,87,-1,-1,-1,-1,89,-1,-1,91,-1,-1,93,-1,95,97,99,-1,-1,-1,101,103,-1,-1,-1,105,-1,107,-1,109,-1,-1,-1,-1],"loss_changes":[1.1815201E0,3.0137497E-1,2.9513526E-1,1.6696554E-1,2.0859072E-1,1.5010452E-1,0E0,4.9449623E-2,7.0331804E-2,8.815123E-2,8.361244E-2,3.838727E-2,1.1672592E-1,5.581662E-3,2.9145539E-2,5.238171E-2,2.4594098E-2,3.8508445E-2,5.3896274E-2,0E0,6.6411337E-3,2.7788758E-2,2.4599984E-2,0E0,4.5394093E-2,0E0,5.442094E-3,0E0,0E0,0E0,8.968964E-3,0E0,8.849298E-3,0E0,3.1273365E-2,3.7624005E-2,8.5824355E-2,0E0,0E0,4.6919584E-3,8.858006E-3,0E0,9.57796E-3,0E0,0E0,0E0,0E0,3.6017299E-3,0E0,0E0,0E0,0E0,1.7833918E-2,0E0,3.0276315E-2,1.821603E-2,0E0,0E0,0E0,0E0,0E0,1.0989454E-2,0E0,0E0,2.1419227E-3,1.9536356E-2,0E0,2.9563766E-2,0E0,0E0,5.956037E-3,0E0,3.577382E-3,0E0,0E0,0E0,9.62789E-3,7.929774E-3,3.1082425E-3,0E0,0E0,0E0,0E0,7.3374994E-3,0E0,0E0,6.295189E-3,0E0,0E0,6.811442E-3,0E0,2.1263799E-3,8.553239E-3,6.0235094E-3,0E0,0E0,0E0,2.5084862E-3,5.3524906E-3,0E0,0E0,0E0,2.1880572E-3,0E0,9.1967836E-4,0E0,1.2592386E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,24,24,26,26,30,30,32,32,34,34,35,35,36,36,39,39,40,40,42,42,47,47,52,52,54,54,55,55,61,61,64,64,65,65,67,67,70,70,72,72,76,76,77,77,78,78,83,83,86,86,89,89,91,91,92,92,93,93,97,97,98,98,102,102,104,104,106,106],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,-1,44,-1,46,-1,-1,-1,48,-1,50,-1,52,54,56,-1,-1,58,60,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,66,-1,68,70,-1,-1,-1,-1,-1,72,-1,-1,74,76,-1,78,-1,-1,80,-1,82,-1,-1,-1,84,86,88,-1,-1,-1,-1,90,-1,-1,92,-1,-1,94,-1,96,98,100,-1,-1,-1,102,104,-1,-1,-1,106,-1,108,-1,110,-1,-1,-1,-1],"split_conditions":[6.371693E6,4.258E3,1.084044E6,6.7452E4,1.8943199E9,2.15195E5,-4.762811E-3,1.52E2,7.76E2,1.864192E6,3.15E3,8.965656E9,3.43514E5,2.648E3,6.6362006E2,1.52E2,2.6E1,1.6028E4,2.6973452E10,7.835364E-3,4.89E2,2.2497124E7,3.9652E4,1.0600216E-2,1.09E2,-3.9504142E-3,1.039E3,-7.924401E-3,-3.6589564E-3,1.1570258E-3,1.3953088E8,2.3013796E-3,2.431E3,1.5010783E-3,5.013E3,9.6671375E2,4.4748E5,2.534107E-4,2.353794E-3,1.5035741E11,1.42E2,-2.908404E-4,1.05055E5,1.1716922E-3,5.9715644E-3,-2.643138E-3,-6.233036E-4,4.9403088E2,-1.3687407E-3,-1.0851531E-3,5.997214E-4,-3.1511406E-3,2.147E3,3.192191E-3,5.96025E5,1.914442E6,-5.663411E-3,6.6362056E-3,2.9014165E-3,1.0750045E-3,3.6483794E-3,1.4409099E10,3.7861587E-3,-1.8532585E-3,7.47E2,5.3E1,-3.3801815E-3,8.8573E4,2.654174E-3,-1.4346463E-3,5.941E3,2.7111832E-3,5.4066E4,-4.353591E-3,-2.540123E-3,9.266274E-4,5.89E2,4.283E3,4E2,1.5019402E-3,-2.4283469E-4,1.3861867E-3,-6.400392E-5,1.219769E6,3.1890362E-4,1.0808539E-3,1.1988E4,6.9634814E-4,2.3645838E-3,1.107136E6,-2.2366026E-3,7.069832E6,1.8283893E8,1.18399E5,3.16518E-4,-1.3686289E-3,-2.5779806E-4,1.5227322E8,1.068557E6,-2.01634E-4,-1.7593894E-3,-1.246073E-3,1.37586E5,1.4422152E-3,1.938487E6,-9.091306E-4,2.27905E5,-3.0044018E-4,2.9082553E-4,3.9785705E-4,-3.0303214E-4],"split_indices":[6,3,3,18,16,3,0,17,0,2,28,21,18,22,5,28,17,12,10,0,0,2,8,0,4,0,0,0,0,0,16,0,0,0,3,5,12,0,0,13,17,0,3,0,0,0,0,5,0,0,0,0,19,0,12,6,0,0,0,0,0,1,0,0,17,4,0,8,0,0,3,0,8,0,0,0,4,3,17,0,0,0,0,2,0,0,7,0,0,2,0,2,27,6,0,0,0,11,6,0,0,0,12,0,6,0,18,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.63E2,2.05E2,5.8E1,7.1E1,1.34E2,5.5E1,3E0,2.6E1,4.5E1,1.25E2,9E0,4E1,1.5E1,1.3E1,1.3E1,2.9E1,1.6E1,5.1E1,7.4E1,3E0,6E0,1.6E1,2.4E1,6E0,9E0,6E0,7E0,7E0,6E0,3E0,2.6E1,3E0,1.3E1,3E0,4.8E1,5.8E1,1.6E1,3E0,3E0,9E0,7E0,3E0,2.1E1,4E0,5E0,4E0,3E0,2.1E1,5E0,9E0,4E0,1.4E1,3.4E1,4E0,5.4E1,1.3E1,3E0,6E0,3E0,4E0,3E0,1.4E1,7E0,5E0,1.6E1,3.1E1,3E0,5E1,4E0,6E0,7E0,8E0,6E0,8E0,8E0,4E0,2.7E1,4.3E1,7E0,4E0,3E0,3E0,3E0,2.4E1,3E0,3E0,4E1,3E0,4E0,1.5E1,9E0,1E1,3E1,1.2E1,3E0,7E0,3E0,1.8E1,1.2E1,4E0,8E0,4E0,1.4E1,3E0,9E0,4E0,1E1,4E0,5E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[8.773932E-3,-3.7997484E-2,1.1147184E-1,1.08668536E-1,-5.0476722E-2,2.4754852E-1,8.135738E-2,-3.883635E-3,1.4397992E-2,-9.075089E-2,-3.136922E-2,1.4738724E-1,1.645453E-2,1.9047868E-2,1.4652017E-1,-3.6081765E-3,1.0488114E-1,-1.2962984E-1,-7.192774E-2,-2.1616712E-2,-1.3224871E-1,1.4209825E-3,2.0639549E-1,4.8300605E-2,-8.250092E-3,7.246197E-2,1.8844855E-1,1.0207266E-3,4.887299E-3,-5.0925505E-3,-8.3473034E-2,-2.3485178E-2,-8.633213E-2,-3.3124667E-2,6.5415E-2,9.991381E-4,-8.803544E-3,3.3942445E-3,7.914454E-3,-1.6750008E-2,7.30138E-2,-5.643281E-4,9.642137E-2,2.54105E-1,1.0708806E-1,-8.772208E-4,-3.4455513E-3,-1.4290392E-3,7.791788E-4,-3.192214E-3,-2.1661553E-3,-4.299686E-2,2.8316824E-2,-1.2772956E-3,3.586545E-3,9.142948E-3,-1.5981864E-3,3.2316424E-2,1.084959E-1,3.8090183E-3,2.055762E-3,1.0595477E-2,4.9399487E-3,1.2049197E-3,4.060551E-3,-5.9793618E-2,-3.0142644E-2,-2.4250941E-4,4.2038366E-2,-6.25563E-4,1.1405915E-3,-3.1255174E-4,5.674326E-2,4.1725114E-3,1.4336134E-3,-6.683992E-2,5.410939E-4,3.9933473E-4,-3.483325E-2,2.3740802E-4,5.4115027E-2,2.5594095E-3,7.283022E-4,-2.4250042E-2,-7.856099E-2,-5.0526746E-2,-1.703447E-2,6.816572E-4,2.2417244E-3,-1.2911323E-3,-1.13918235E-4,-8.643159E-2,-3.765194E-4,-5.75562E-2,-6.0109975E-4,-2.5644371E-2,1.0437636E-3,-5.6566834E-2,-1.0372307E-1,-6.53752E-2,-6.767461E-4,-1.1471441E-3,-4.9503017E-3,-2.3703347E-3,-6.8735203E-4,-3.7586966E-3,-1.8310719E-3,-9.016252E-4,-2.312558E-3,1.6307266E-4,-5.22531E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,-1,27,29,31,33,35,-1,37,39,-1,41,43,-1,-1,-1,45,47,49,51,53,-1,-1,-1,-1,55,57,-1,59,61,63,-1,-1,-1,-1,-1,-1,65,67,-1,-1,69,-1,71,73,-1,-1,-1,-1,-1,-1,75,77,-1,79,-1,-1,-1,81,-1,-1,83,-1,-1,85,-1,87,-1,-1,89,91,93,95,-1,-1,-1,-1,97,-1,99,-1,101,-1,103,105,107,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3972002E0,3.7154126E-1,3.5276675E-1,6.115492E-1,1.3951784E-1,3.4715724E-1,3.0578893E-1,1.6475913E-1,0E0,3.385365E-2,1.2208741E-1,7.008654E-2,0E0,3.3456892E-1,1.0123485E-1,0E0,1.92331E-2,1.0537267E-2,2.7279481E-2,1.1934902E-1,2.6219344E-1,0E0,5.955994E-3,6.0704485E-2,0E0,3.4033358E-2,9.565073E-2,0E0,0E0,0E0,9.008177E-3,1.2044266E-2,1.1476278E-3,6.430002E-2,7.211297E-2,0E0,0E0,0E0,0E0,9.824648E-3,3.5482556E-2,0E0,1.002878E-3,4.0130436E-2,1.1208922E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.762949E-2,7.6956293E-3,0E0,0E0,5.9479233E-3,0E0,1.4922955E-2,1.31466985E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1634907E-2,1.1009805E-2,0E0,4.2056125E-3,0E0,0E0,0E0,5.192278E-3,0E0,0E0,1.5566856E-2,0E0,0E0,1.2546908E-2,0E0,2.6168209E-3,0E0,0E0,2.814516E-3,1.31979585E-2,4.5228675E-3,1.1051971E-2,0E0,0E0,0E0,0E0,6.5264553E-3,0E0,4.338421E-3,0E0,4.1664867E-3,0E0,5.0400347E-3,1.9126385E-3,1.8544495E-3,0E0,0E0,9.86355E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,30,30,31,31,32,32,33,33,34,34,39,39,40,40,42,42,43,43,44,44,51,51,52,52,55,55,57,57,58,58,65,65,66,66,68,68,72,72,75,75,78,78,80,80,83,83,84,84,85,85,86,86,91,91,93,93,95,95,97,97,98,98,99,99,102,102],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,-1,28,30,32,34,36,-1,38,40,-1,42,44,-1,-1,-1,46,48,50,52,54,-1,-1,-1,-1,56,58,-1,60,62,64,-1,-1,-1,-1,-1,-1,66,68,-1,-1,70,-1,72,74,-1,-1,-1,-1,-1,-1,76,78,-1,80,-1,-1,-1,82,-1,-1,84,-1,-1,86,-1,88,-1,-1,90,92,94,96,-1,-1,-1,-1,98,-1,100,-1,102,-1,104,106,108,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,3.719E3,2.3880364E9,1.267716E7,7.03651E5,1.0431399E7,1.021304E6,3.15859E5,1.4397992E-2,2.876E3,2.981E3,2.748775E6,1.645453E-2,8.2686665E10,6.28354E5,-3.6081765E-3,2.3020366E-2,1.916746E6,9.7E1,1.83E2,8.544151E7,1.4209825E-3,4.803464E6,1.306E4,-8.250092E-3,1.2439753E-1,7.57418E5,1.0207266E-3,4.887299E-3,-5.0925505E-3,2.371E3,8.0612E7,3.15E2,1.1408653E9,5.345E3,9.991381E-4,-8.803544E-3,3.3942445E-3,7.914454E-3,1.62E2,6.240342E6,-5.643281E-4,4.5238292E10,1.8909816E0,2.80736E5,-8.772208E-4,-3.4455513E-3,-1.4290392E-3,7.791788E-4,-3.192214E-3,-2.1661553E-3,5.209E3,3.03125E5,-1.2772956E-3,3.586545E-3,3.61273E5,-1.5981864E-3,6.8315564E2,7.24024E5,3.8090183E-3,2.055762E-3,1.0595477E-2,4.9399487E-3,1.2049197E-3,4.060551E-3,1.9704E4,5.395E3,-2.4250941E-4,1.4562596E9,-6.25563E-4,1.1405915E-3,-3.1255174E-4,1.008599E8,4.1725114E-3,1.4336134E-3,8.63676E5,5.410939E-4,3.9933473E-4,8.172629E8,2.3740802E-4,6.549943E7,2.5594095E-3,7.283022E-4,2.4436589E8,2.235E3,7.7643576E7,5.421011E7,6.816572E-4,2.2417244E-3,-1.2911323E-3,-1.13918235E-4,1.3299E4,-3.765194E-4,1.5159761E10,-6.0109975E-4,2.943003E3,1.0437636E-3,7.43E2,3.28E2,3.122E4,-6.767461E-4,-1.1471441E-3,3.7742935E3,-2.3703347E-3,-6.8735203E-4,-3.7586966E-3,-1.8310719E-3,-9.016252E-4,-2.312558E-3,1.6307266E-4,-5.22531E-4],"split_indices":[2,7,11,18,2,2,12,21,0,3,0,18,0,23,28,0,63,18,4,47,16,0,2,7,0,63,22,0,0,0,3,23,0,16,7,0,0,0,0,17,6,0,26,64,20,0,0,0,0,0,0,3,12,0,0,12,0,5,12,0,0,0,0,0,0,7,3,0,16,0,0,0,16,0,0,9,0,0,23,0,27,0,0,10,0,18,18,0,0,0,0,7,0,10,0,5,0,0,4,18,0,0,5,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.89E2,1.99E2,9E1,1.5E1,1.84E2,1.5E1,7.5E1,1.2E1,3E0,5.8E1,1.26E2,1.2E1,3E0,3.9E1,3.6E1,6E0,6E0,1.7E1,4.1E1,1.16E2,1E1,5E0,7E0,3.6E1,3E0,1.4E1,2.2E1,3E0,3E0,9E0,8E0,1E1,3.1E1,1.03E2,1.3E1,5E0,5E0,3E0,4E0,1E1,2.6E1,3E0,1.1E1,1.1E1,1.1E1,3E0,5E0,7E0,3E0,1.6E1,1.5E1,8.9E1,1.4E1,4E0,9E0,6E0,4E0,1.3E1,1.3E1,5E0,6E0,5E0,6E0,3E0,8E0,3.7E1,5.2E1,4E0,1E1,3E0,3E0,5E0,8E0,9E0,4E0,3.4E1,3E0,5E0,4.7E1,3E0,7E0,4E0,4E0,8E0,2.6E1,2.4E1,2.3E1,3E0,4E0,4E0,4E0,2.3E1,3E0,1.9E1,5E0,2E1,3E0,1E1,1.3E1,1.5E1,4E0,1.3E1,7E0,6E0,4E0,9E0,4E0,3E0,1.2E1,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[4.499059E-3,-3.1620007E-2,1.3883561E-1,-5.4631602E-2,2.2989918E-2,1.1423415E-1,2.2977205E-1,-7.4932255E-2,-1.5592308E-2,8.035321E-3,4.9244664E-3,8.222481E-2,2.202468E-1,1.7703844E-3,9.07466E-3,-6.777721E-2,-6.5819263E-3,-4.6257963E-3,-5.053683E-3,-7.892563E-3,1.10419214E-1,-1.3626298E-5,9.4043784E-2,1.0511726E-2,1.3955782E-1,2.7107932E-2,-7.5131506E-2,-3.1818867E-2,5.001733E-2,4.681555E-2,-2.8065523E-2,4.525515E-3,1.6944745E-3,1.2827635E-1,5.8464974E-2,1.998327E-3,5.5888924E-3,-9.909128E-4,2.188852E-3,-1.24435425E-1,-6.2630184E-2,-4.2018536E-2,8.4278994E-4,2.5391063E-2,3.5075478E-3,4.0062363E-3,-1.221708E-2,-3.7576111E-3,-1.597788E-2,1.5464377E-1,1.0965414E-3,-1.4196381E-4,7.052974E-2,-4.361575E-3,-1.668604E-3,-6.9451645E-2,-9.9812085E-3,-1.7609957E-4,-4.8836533E-2,1.42428E-3,5.7178224E-3,1.9683642E-3,-5.2395996E-2,1.7296523E-3,-4.772202E-2,2.2822507E-3,1.7267624E-1,9.3627356E-2,5.2883715E-4,-6.155896E-2,-4.5305835E-3,4.092079E-4,-6.9281127E-4,-6.762291E-2,-3.491299E-2,7.6356326E-4,-5.863525E-4,-2.884006E-3,-6.749627E-5,3.894065E-2,-1.6971938E-2,-3.6807875E-3,-9.636138E-3,6.966549E-3,3.4583234E-3,1.4439916E-3,3.7044974E-3,-7.7338934E-2,-4.757567E-2,-1.4061119E-3,-2.9810814E-3,-3.6680067E-4,-1.4086677E-3,6.4407956E-5,2.1918183E-3,-4.421424E-2,9.243326E-3,-6.5612944E-4,2.9812503E-5,-8.487207E-2,-8.2764507E-4,-2.7008157E-2,-7.1597055E-2,-3.2964445E-4,-1.9150525E-3,-3.8507947E-4,1.0571614E-3,-8.7140925E-4,-2.9562386E-3,-3.692682E-2,-1.8955687E-3,-3.9239004E-3,-5.0611127E-2,5.3131433E-5,-4.670873E-2,4.4554452E-4,-5.523232E-4,-1.934568E-3,-4.5821816E-4,-1.9074611E-3,-9.424712E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,25,-1,27,-1,29,31,-1,33,-1,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,-1,-1,53,55,57,-1,59,-1,-1,61,-1,63,65,-1,-1,67,-1,-1,69,71,-1,73,-1,75,-1,77,79,81,-1,83,85,-1,87,-1,-1,-1,89,91,-1,-1,-1,-1,93,95,-1,97,-1,-1,-1,-1,99,101,-1,-1,-1,-1,-1,-1,103,105,-1,-1,107,-1,109,111,-1,-1,-1,-1,-1,-1,113,115,-1,117,-1,119,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3969458E0,2.8651816E-1,1.1021948E-1,1.2563533E-1,2.7718648E-1,1.5156007E-1,9.3859375E-2,8.346599E-2,8.6116E-2,0E0,8.740776E-2,3.911245E-2,4.878962E-2,0E0,0E0,7.2515875E-2,0E0,8.034497E-2,0E0,6.532413E-2,4.8022866E-3,0E0,3.5128683E-2,0E0,1.0337919E-2,2.1032017E-2,5.014479E-2,2.2397596E-2,2.3364171E-2,7.4857384E-2,4.4222947E-2,0E0,0E0,3.6961466E-2,1.4724001E-2,0E0,0E0,0E0,0E0,7.4335635E-3,2.696079E-2,7.4574724E-3,0E0,4.940534E-3,0E0,0E0,3.2682247E-2,0E0,2.200485E-2,7.3880255E-3,0E0,0E0,1.8455297E-2,0E0,0E0,2.9601395E-2,2.8929575E-3,0E0,4.43998E-3,0E0,3.8792305E-3,0E0,1.2910059E-2,1.8754425E-2,3.4423977E-2,0E0,5.2894354E-3,6.34858E-3,0E0,1.0221869E-2,0E0,0E0,0E0,1.9367635E-3,3.039347E-3,0E0,0E0,0E0,0E0,9.555414E-3,1.3263643E-2,0E0,1.1540675E-3,0E0,0E0,0E0,0E0,8.707315E-3,1.5852466E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.062688E-3,5.3974064E-3,0E0,0E0,7.179901E-3,0E0,5.1607527E-3,1.0721557E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.7375114E-3,1.9188944E-3,0E0,3.6317855E-3,0E0,8.89726E-4,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,15,15,17,17,19,19,20,20,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,39,39,40,40,41,41,43,43,46,46,48,48,49,49,52,52,55,55,56,56,58,58,60,60,62,62,63,63,64,64,66,66,67,67,69,69,73,73,74,74,79,79,80,80,82,82,87,87,88,88,95,95,96,96,99,99,101,101,102,102,109,109,110,110,112,112,114,114],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,26,-1,28,-1,30,32,-1,34,-1,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,-1,-1,54,56,58,-1,60,-1,-1,62,-1,64,66,-1,-1,68,-1,-1,70,72,-1,74,-1,76,-1,78,80,82,-1,84,86,-1,88,-1,-1,-1,90,92,-1,-1,-1,-1,94,96,-1,98,-1,-1,-1,-1,100,102,-1,-1,-1,-1,-1,-1,104,106,-1,-1,108,-1,110,112,-1,-1,-1,-1,-1,-1,114,116,-1,118,-1,120,-1,-1,-1,-1,-1,-1],"split_conditions":[2.4995E4,4.579412E8,1.1741724E4,1.27001E5,3.719E3,3.5541443E9,2.203964E9,1.13472E5,9.45839E5,8.035321E-3,6.64063E5,3.7889056E-2,4.368265E7,1.7703844E-3,9.07466E-3,3.185029E2,-6.5819263E-3,1.9669E4,-5.053683E-3,1.2126E4,6.007078E6,-1.3626298E-5,1.42E2,1.0511726E-2,2.7936E4,2.4249094E2,2.69E2,3.15196E5,1.14153E5,9E1,8.099343E2,4.525515E-3,1.6944745E-3,7.9937E4,2.89E2,1.998327E-3,5.5888924E-3,-9.909128E-4,2.188852E-3,4.1889E5,6.63188E5,7.801224E6,8.4278994E-4,1E2,3.5075478E-3,4.0062363E-3,1.6259238E7,-3.7576111E-3,1.5936E4,1.734E3,1.0965414E-3,-1.4196381E-4,5.121E4,-4.361575E-3,-1.668604E-3,4.86577E5,1.9E1,-1.7609957E-4,6.1E1,1.42428E-3,5.072E3,1.9683642E-3,7.721E3,7.1E1,7.5469E4,2.2822507E-3,7.3100853E0,3.389E3,5.2883715E-4,5.72466E5,-4.5305835E-3,4.092079E-4,-6.9281127E-4,1.038E3,1.02E3,7.6356326E-4,-5.863525E-4,-2.884006E-3,-6.749627E-5,3.6E1,3.649951E9,-3.6807875E-3,2.56282E5,6.966549E-3,3.4583234E-3,1.4439916E-3,3.7044974E-3,4.0200136E7,7.3893E4,-1.4061119E-3,-2.9810814E-3,-3.6680067E-4,-1.4086677E-3,6.4407956E-5,2.1918183E-3,4.6E1,7.7E1,-6.5612944E-4,2.9812503E-5,2.69E3,-8.2764507E-4,7.0319E4,8.0416E4,-3.2964445E-4,-1.9150525E-3,-3.8507947E-4,1.0571614E-3,-8.7140925E-4,-2.9562386E-3,2.401E3,4.115E3,-3.9239004E-3,4.44688E5,5.3131433E-5,1.0648841E7,4.4554452E-4,-5.523232E-4,-1.934568E-3,-4.5821816E-4,-1.9074611E-3,-9.424712E-4],"split_indices":[3,16,5,12,7,16,11,22,12,0,12,59,2,0,0,5,0,7,0,28,6,0,19,0,7,5,0,12,8,4,5,0,0,8,19,0,0,0,0,2,6,23,0,19,0,0,18,0,7,0,0,0,19,0,0,6,17,0,19,0,3,0,3,4,8,0,60,0,0,2,0,0,0,0,0,0,0,0,0,19,10,0,12,0,0,0,0,18,12,0,0,0,0,0,0,19,19,0,0,7,0,8,12,0,0,0,0,0,0,3,3,0,6,0,26,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,2.26E2,6E1,1.59E2,6.7E1,4.9E1,1.1E1,1.04E2,5.5E1,4E0,6.3E1,3.9E1,1E1,3E0,8E0,1E2,4E0,5.2E1,3E0,5.7E1,6E0,5E0,3.4E1,3E0,7E0,7E0,9.3E1,3.5E1,1.7E1,1.5E1,4.2E1,3E0,3E0,1.6E1,1.8E1,3E0,4E0,3E0,4E0,1.7E1,7.6E1,3E1,5E0,1.3E1,4E0,6E0,9E0,4E0,3.8E1,1.2E1,4E0,3E0,1.5E1,1.4E1,3E0,6.7E1,9E0,5E0,2.5E1,6E0,7E0,3E0,6E0,2.5E1,1.3E1,3E0,9E0,1E1,5E0,6.2E1,5E0,3E0,6E0,9E0,1.6E1,4E0,3E0,3E0,3E0,8E0,1.7E1,4E0,9E0,4E0,5E0,4E0,6E0,2.7E1,3.5E1,6E0,3E0,5E0,1.1E1,4E0,4E0,8E0,9E0,4E0,5E0,2.3E1,4E0,2E1,1.5E1,3E0,5E0,5E0,4E0,3E0,2E1,1.4E1,6E0,3E0,1.2E1,3E0,1.1E1,3E0,3E0,9E0,3E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[-1.3139575E-3,-4.1916132E-2,9.0978645E-2,-5.7314917E-2,4.725415E-2,6.413093E-2,2.1736212E-1,-6.807713E-2,1.144018E-2,3.5907333E-3,1.0627786E-2,2.1819055E-1,4.4778574E-2,2.5990313E-1,1.1394442E-3,1.8675294E-2,-7.5869076E-2,3.4544168E-3,-7.3668086E-3,-3.6531128E-2,3.345771E-2,2.9673136E-3,1.0337511E-2,-4.3715974E-3,5.5365723E-2,1.09151425E-2,4.985896E-3,-5.6785427E-2,3.739618E-3,-1.0595692E-1,-5.8253467E-2,3.3366708E-3,-1.7282497E-3,-1.8955644E-2,-2.1548313E-3,-4.039842E-4,4.9839612E-2,3.905191E-2,1.2280458E-1,-3.3724129E-3,5.597886E-4,-9.0818286E-2,-6.415713E-3,1.9524297E-3,-6.316384E-2,-2.1161757E-2,1.7349364E-2,9.398599E-5,-1.1922785E-3,2.1789754E-3,3.245113E-2,7.877257E-2,7.737343E-4,1.8660907E-3,4.763121E-3,-5.2822502E-3,-7.492883E-2,-8.820955E-2,-3.5421573E-2,-1.8477757E-4,-1.0072786E-3,9.409915E-4,2.4956727E-4,1.8614216E-4,1.8564586E-3,9.6185364E-2,1.5125582E-4,-3.7565306E-2,5.50229E-2,2.248103E-4,-8.2149476E-2,-4.982719E-3,-7.362234E-2,1.0986308E-3,-4.364104E-2,1.1077463E-1,8.802006E-4,-7.556925E-2,2.7521646E-2,2.1597867E-4,2.5217081E-3,-1.0076117E-1,-5.0616473E-2,-4.7782535E-4,-8.010079E-2,-5.2151095E-2,-1.0425174E-2,6.648032E-2,4.1488833E-3,1.45624535E-5,-1.1907482E-1,-6.747385E-5,1.6177868E-3,-1.4652028E-3,-3.4583628E-3,-3.0082311E-2,-2.4593652E-3,-8.466429E-2,-1.1813223E-3,-2.2239847E-2,-6.5994725E-2,8.232871E-4,-1.0522214E-3,5.177814E-4,3.2271002E-3,-5.1083304E-3,-1.5992351E-3,-1.6719798E-4,-1.735999E-3,-3.0462549E-3,-5.8836006E-2,-1.2495888E-3,-2.3238694E-4,-7.632404E-2,-9.390272E-4,-9.4202755E-4,-2.303204E-3,-1.0748265E-3,-2.7127136E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,-1,31,33,35,-1,-1,-1,37,-1,-1,39,-1,41,43,45,-1,47,-1,-1,49,51,53,-1,-1,55,-1,-1,57,59,61,-1,-1,-1,63,65,67,-1,-1,-1,69,71,73,-1,-1,-1,-1,-1,-1,75,-1,77,79,-1,81,-1,83,-1,85,87,-1,89,91,-1,-1,93,95,-1,97,99,101,103,-1,-1,105,-1,-1,-1,-1,107,-1,109,-1,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,115,-1,-1,117,-1,-1,-1,-1,-1],"loss_changes":[1.0792829E0,2.7733117E-1,2.8465384E-1,1.2731141E-1,3.694049E-1,2.1085459E-1,1.0691345E-1,1.0146046E-1,4.4066206E-2,3.3576854E-2,0E0,7.418415E-2,1.356181E-1,4.72669E-2,0E0,1.0224534E-1,6.590974E-2,0E0,1.05915265E-2,5.145127E-3,1.268471E-2,0E0,0E0,0E0,6.5616086E-2,0E0,0E0,3.0296043E-2,0E0,5.115038E-2,5.5213183E-2,6.5463446E-3,0E0,3.668579E-3,0E0,0E0,1.5286524E-3,8.0522075E-2,1.1002615E-2,0E0,0E0,3.893888E-2,0E0,0E0,5.593559E-2,9.139946E-4,1.1418385E-3,0E0,0E0,0E0,5.04765E-3,3.3083558E-2,6.0303055E-2,0E0,0E0,0E0,2.30494E-2,3.1341165E-2,2.5500152E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.8061876E-2,0E0,4.4130318E-2,1.2993086E-2,0E0,1.5440702E-2,0E0,1.3280958E-2,0E0,1.0197729E-2,5.073324E-3,0E0,3.6434896E-2,4.8193815E-3,0E0,0E0,2.8568506E-3,5.644474E-3,0E0,3.6269426E-3,1.09547E-2,8.04989E-3,1.030173E-2,0E0,0E0,1.1362299E-2,0E0,0E0,0E0,0E0,5.6554647E-3,0E0,2.1590292E-3,0E0,2.4592495E-3,5.3804517E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.042368E-3,0E0,0E0,2.3061112E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,24,24,27,27,29,29,30,30,31,31,33,33,36,36,37,37,38,38,41,41,44,44,45,45,46,46,50,50,51,51,52,52,56,56,57,57,58,58,65,65,67,67,68,68,70,70,72,72,74,74,75,75,77,77,78,78,81,81,82,82,84,84,85,85,86,86,87,87,90,90,95,95,97,97,99,99,100,100,110,110,113,113],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,-1,32,34,36,-1,-1,-1,38,-1,-1,40,-1,42,44,46,-1,48,-1,-1,50,52,54,-1,-1,56,-1,-1,58,60,62,-1,-1,-1,64,66,68,-1,-1,-1,70,72,74,-1,-1,-1,-1,-1,-1,76,-1,78,80,-1,82,-1,84,-1,86,88,-1,90,92,-1,-1,94,96,-1,98,100,102,104,-1,-1,106,-1,-1,-1,-1,108,-1,110,-1,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,116,-1,-1,118,-1,-1,-1,-1,-1],"split_conditions":[4.580147E6,4.48546E7,1.1546442E11,1.22661E6,1.83E2,1.0927762E9,8.74E2,1.0916E4,3.9076E4,7.078E3,1.0627786E-2,3.81E2,1.65728E5,4.331342E-1,1.1394442E-3,3.7E1,1.49015E5,3.4544168E-3,1.9694447E10,1.207E3,1.7452324E3,2.9673136E-3,1.0337511E-2,-4.3715974E-3,1.05055E5,1.09151425E-2,4.985896E-3,1.055E3,3.739618E-3,1.05914776E8,3.185029E2,1.038E3,-1.7282497E-3,8.1E1,-2.1548313E-3,-4.039842E-4,3.05031E5,2.239676E10,6.5406E4,-3.3724129E-3,5.597886E-4,2.315E3,-6.415713E-3,1.9524297E-3,1.1846196E9,2.1546708E9,5.097E3,9.398599E-5,-1.1922785E-3,2.1789754E-3,4.8892842E8,1.2717663E0,1.14E2,1.8660907E-3,4.763121E-3,-5.2822502E-3,5.758731E2,1.0382E4,3.2416E4,-1.8477757E-4,-1.0072786E-3,9.409915E-4,2.4956727E-4,1.8614216E-4,1.8564586E-3,1.1619143E1,1.5125582E-4,1.33E2,2.9424374E10,2.248103E-4,4.014E3,-4.982719E-3,2.92E2,1.0986308E-3,1.77451E5,3E1,8.802006E-4,5.5659E4,7.205049E6,2.1597867E-4,2.5217081E-3,2E1,2.6245E4,-4.7782535E-4,1.440233E6,8.4961E4,7.1E1,2.318509E6,4.1488833E-3,1.45624535E-5,1.3366E4,-6.747385E-5,1.6177868E-3,-1.4652028E-3,-3.4583628E-3,1.6671127E3,-2.4593652E-3,5.637464E-1,-1.1813223E-3,1.0018222E3,1.9422861E10,8.232871E-4,-1.0522214E-3,5.177814E-4,3.2271002E-3,-5.1083304E-3,-1.5992351E-3,-1.6719798E-4,-1.735999E-3,-3.0462549E-3,9.7E1,-1.2495888E-3,-2.3238694E-4,4.5E1,-9.390272E-4,-9.4202755E-4,-2.303204E-3,-1.0748265E-3,-2.7127136E-3],"split_indices":[2,18,23,6,47,11,17,8,8,3,0,17,16,63,0,4,18,0,10,0,5,0,0,0,3,0,0,20,0,16,5,0,0,4,0,0,12,10,8,0,0,3,0,0,10,21,3,0,0,0,27,59,19,0,0,0,5,7,12,0,0,0,0,0,0,60,0,4,10,0,3,0,0,0,12,4,0,8,2,0,0,17,18,0,2,12,17,1,0,0,3,0,0,0,0,5,0,63,0,5,10,0,0,0,0,0,0,0,0,0,4,0,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.99E2,8.7E1,1.7E2,2.9E1,7.3E1,1.4E1,1.47E2,2.3E1,2.6E1,3E0,7E0,6.6E1,1.1E1,3E0,1.2E1,1.35E2,3E0,2E1,1.1E1,1.5E1,4E0,3E0,3E0,6.3E1,5E0,6E0,7E0,5E0,4.8E1,8.7E1,1.7E1,3E0,8E0,3E0,4E0,1.1E1,5.2E1,1.1E1,4E0,3E0,4.3E1,5E0,3E0,8.4E1,6E0,1.1E1,4E0,4E0,4E0,7E0,2.5E1,2.7E1,4E0,7E0,6E0,3.7E1,4.3E1,4.1E1,3E0,3E0,4E0,7E0,4E0,3E0,2E1,5E0,1.6E1,1.1E1,3E0,3.4E1,6E0,3.7E1,4E0,3.7E1,1.6E1,4E0,1E1,6E0,4E0,7E0,2E1,1.4E1,4E0,3.3E1,2.9E1,8E0,6E0,1E1,4E0,6E0,3E0,3E0,3E0,1.7E1,9E0,5E0,2.9E1,4E0,1E1,1.9E1,3E0,5E0,3E0,3E0,3E0,3E0,5E0,4E0,1.9E1,1E1,4E0,6E0,1.4E1,5E0,4E0,6E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-9.582224E-3,-4.876266E-2,7.663503E-2,-7.618004E-2,-1.50397485E-2,9.0809435E-2,-7.617275E-3,-6.718804E-2,-1.9118515E-1,-2.9801562E-2,5.681961E-2,-3.932512E-3,1.2719272E-1,-9.1924705E-2,-1.329603E-2,-3.0668823E-3,-7.7027073E-3,1.8930457E-3,-3.450658E-2,1.4278787E-1,-3.6251098E-2,2.9880736E-2,-4.6610054E-2,1.0383812E-1,2.0364103E-1,1.1348904E-3,-1.00559376E-1,-2.3316057E-2,2.5820648E-2,-5.3306486E-2,-1.11033665E-2,5.7737916E-3,1.9751466E-3,7.214927E-4,-3.235636E-3,-8.120361E-4,5.444661E-2,-1.9110353E-3,-3.0383468E-4,8.42108E-2,8.233644E-3,1.4146397E-1,9.919782E-3,-9.013965E-2,-7.458678E-3,-6.728285E-3,-3.1702038E-2,-5.7207275E-4,2.0265675E-3,-2.5044545E-4,-5.9968594E-2,2.9914975E-2,-2.8454673E-2,2.925151E-3,4.8334053E-4,5.302808E-2,1.2244973E-1,2.0817064E-3,5.3155646E-3,-1.1343962E-1,-5.746045E-2,-6.8640173E-4,6.5976183E-4,-4.0898725E-4,-4.0908534E-2,-6.499866E-2,-6.051686E-4,5.3494014E-2,-6.0602423E-4,-3.4676346E-3,-5.539802E-2,7.4950635E-2,-1.7393879E-3,1.335387E-1,1.5283783E-3,-1.3165547E-1,-7.801301E-2,-4.0923357E-2,-2.6227941E-3,-1.5806429E-3,-4.59923E-4,-2.2529333E-3,-5.246676E-4,4.599702E-4,2.5533838E-3,-7.18966E-4,1.5895901E-2,-2.414826E-3,-5.4229377E-4,4.845699E-3,3.7196245E-2,2.4258068E-3,1.5313186E-1,-4.810833E-3,-2.9393416E-3,-3.0320156E-3,-1.3113447E-3,-6.4202435E-2,-2.0272814E-2,1.0118785E-3,-1.16451505E-4,-5.5586395E-4,5.1093463E-2,5.5199782E-3,2.515221E-3,-8.7303994E-4,-2.6081435E-3,-1.109193E-3,4.2384383E-4,6.375082E-2,2.1491614E-4,1.0709609E-3,2.8189921E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,-1,-1,-1,29,31,33,35,37,39,41,-1,43,45,47,49,51,-1,-1,-1,-1,-1,53,-1,-1,55,-1,57,-1,59,-1,61,63,-1,-1,-1,65,67,69,-1,-1,71,73,-1,-1,75,77,-1,-1,-1,79,81,-1,83,-1,85,87,89,-1,91,-1,93,95,97,-1,-1,-1,-1,-1,-1,-1,-1,99,-1,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,111,-1,-1,-1],"loss_changes":[9.293171E-1,1.7360246E-1,4.0337414E-1,9.470379E-2,9.291993E-2,2.866031E-1,0E0,1.2942237E-1,9.5002055E-3,3.204766E-2,1.2519312E-1,3.6026336E-2,8.597946E-2,7.7123106E-2,1.3144443E-2,0E0,0E0,0E0,2.9766604E-2,1.2824893E-2,3.2415994E-2,2.0326369E-2,4.6566036E-3,1.21616185E-1,4.0898323E-2,0E0,6.520963E-2,3.4034234E-3,1.2451554E-2,1.10587925E-2,2.362632E-2,0E0,0E0,0E0,0E0,0E0,1.2999237E-2,0E0,0E0,4.597208E-2,0E0,7.4989498E-3,0E0,3.9167166E-2,0E0,4.4107633E-3,2.4533272E-3,0E0,0E0,0E0,5.611576E-3,1.2500042E-2,1.5292289E-2,0E0,0E0,6.507562E-2,9.454459E-3,0E0,0E0,1.1670232E-2,7.3908046E-3,0E0,0E0,0E0,1.6985293E-3,5.909525E-3,0E0,5.955234E-3,0E0,5.1046535E-3,7.057138E-3,5.7184085E-2,0E0,6.697774E-3,0E0,1.4320314E-3,4.606597E-3,7.351497E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2363763E-3,0E0,0E0,0E0,1.2992917E-2,0E0,1.7414093E-3,0E0,0E0,0E0,0E0,2.7929433E-3,5.671933E-3,0E0,0E0,0E0,6.882943E-3,0E0,0E0,0E0,0E0,0E0,0E0,4.348211E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,36,36,39,39,41,41,43,43,45,45,46,46,50,50,51,51,52,52,55,55,56,56,59,59,60,60,64,64,65,65,67,67,69,69,70,70,71,71,73,73,75,75,76,76,77,77,86,86,90,90,92,92,97,97,98,98,102,102,109,109],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,-1,-1,-1,30,32,34,36,38,40,42,-1,44,46,48,50,52,-1,-1,-1,-1,-1,54,-1,-1,56,-1,58,-1,60,-1,62,64,-1,-1,-1,66,68,70,-1,-1,72,74,-1,-1,76,78,-1,-1,-1,80,82,-1,84,-1,86,88,90,-1,92,-1,94,96,98,-1,-1,-1,-1,-1,-1,-1,-1,100,-1,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,112,-1,-1,-1],"split_conditions":[2.7732336E3,2.748775E6,3.539647E4,1.17442E5,2.34E3,4.33272E5,-7.617275E-3,1.6744E4,1.83E2,3.185029E2,2.05E2,6.9585E4,1.75E2,3E0,2.402988E6,-3.0668823E-3,-7.7027073E-3,1.8930457E-3,1.129264E6,4.070204E6,2.5E1,3.204999E3,1.7586728E7,3.9743017E9,2.1942E4,1.1348904E-3,1.13472E5,1.9884E4,5.275E3,2.01E2,3.649951E9,5.7737916E-3,1.9751466E-3,7.214927E-4,-3.235636E-3,-8.120361E-4,1.5893E4,-1.9110353E-3,-3.0383468E-4,2.0464036E0,8.233644E-3,1.1906453E7,9.919782E-3,3.76E3,-7.458678E-3,1.2961985E3,9.270782E2,-5.7207275E-4,2.0265675E-3,-2.5044545E-4,1.165E3,2.41361E5,2.783874E6,2.925151E-3,4.8334053E-4,5.637395E3,1.5116796E10,2.0817064E-3,5.3155646E-3,3.8508E4,2.53387E5,-6.8640173E-4,6.5976183E-4,-4.0898725E-4,1.203673E6,7.9937E4,-6.051686E-4,4.7E1,-6.0602423E-4,1E0,1.578606E6,1.7655054E-1,-1.7393879E-3,1.3069112E10,1.5283783E-3,2.748E3,3.4713E4,1.501E4,-2.6227941E-3,-1.5806429E-3,-4.59923E-4,-2.2529333E-3,-5.246676E-4,4.599702E-4,2.5533838E-3,-7.18966E-4,6.6E2,-2.414826E-3,-5.4229377E-4,4.845699E-3,1.45E4,2.4258068E-3,8.28337E5,-4.810833E-3,-2.9393416E-3,-3.0320156E-3,-1.3113447E-3,2.4693763E9,1.2811E4,1.0118785E-3,-1.16451505E-4,-5.5586395E-4,1.2101E4,5.5199782E-3,2.515221E-3,-8.7303994E-4,-2.6081435E-3,-1.109193E-3,4.2384383E-4,5.5956E4,2.1491614E-4,1.0709609E-3,2.8189921E-3],"split_indices":[5,18,5,8,0,12,0,7,47,5,19,8,4,17,6,0,0,0,9,2,4,5,18,16,7,0,22,18,3,27,10,0,0,0,0,0,3,0,0,67,0,2,0,20,0,5,5,0,0,0,17,12,2,0,0,5,27,0,0,12,18,0,0,0,6,8,0,4,0,42,6,63,0,10,0,3,18,18,0,0,0,0,0,0,0,0,17,0,0,0,7,0,28,0,0,0,0,10,20,0,0,0,0,0,0,0,0,0,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.88E2,8.5E1,1.03E2,8.5E1,8.2E1,3E0,9.7E1,6E0,7.1E1,1.4E1,2.3E1,5.9E1,6.6E1,3.1E1,3E0,3E0,3E0,6.8E1,7E0,7E0,1.3E1,1E1,4.7E1,1.2E1,4E0,6.2E1,2.5E1,6E0,3.7E1,3.1E1,4E0,3E0,4E0,3E0,4E0,9E0,7E0,3E0,4.3E1,4E0,9E0,3E0,5.9E1,3E0,9E0,1.6E1,3E0,3E0,5E0,3.2E1,9E0,2.2E1,4E0,5E0,2.5E1,1.8E1,3E0,6E0,3.3E1,2.6E1,6E0,3E0,6E0,1E1,2.8E1,4E0,6E0,3E0,1.2E1,1E1,2.1E1,4E0,1.5E1,3E0,2E1,1.3E1,1.7E1,9E0,7E0,3E0,2.5E1,3E0,3E0,3E0,6E0,6E0,6E0,4E0,6E0,1.5E1,5E0,1E1,1.2E1,8E0,8E0,5E0,7E0,1E1,3E0,3E0,3E0,1.2E1,7E0,3E0,3E0,4E0,7E0,3E0,9E0,3E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[6.0660415E-3,-2.126211E-2,1.219844E-1,-3.9505623E-2,4.6156608E-2,1.6522089E-2,9.0740256E-2,-3.4356307E-2,-1.6995105E-1,2.0933889E-1,-2.72983E-3,1.12146474E-1,-6.594684E-3,-5.0913062E-2,1.8284999E-2,-7.5305155E-3,-2.0429422E-3,1.0730745E-2,7.31233E-2,-2.1151965E-2,3.430943E-2,8.628025E-2,2.0956996E-1,-4.2641133E-2,-1.04152635E-2,3.8639333E-2,-4.0909365E-2,1.108624E-3,3.010461E-3,-4.011243E-2,-4.4245934E-3,-9.0733534E-5,4.469156E-2,6.257518E-2,1.6624606E-1,7.7325106E-3,3.332705E-3,-5.7576172E-2,3.0699303E-2,2.1780983E-2,4.2509246E-3,-9.191359E-4,-3.883921E-3,-1.5168132E-3,-6.267971E-4,-1.2080924E-3,6.102429E-3,5.936031E-2,1.3458724E-4,-1.4644214E-2,7.9645194E-2,7.952979E-3,1.6793392E-3,2.548305E-3,-6.321707E-2,-2.6259369E-3,7.2657496E-2,2.6762611E-3,1.1883728E-2,-9.791458E-4,9.2589087E-4,1.3114676E-3,-2.0640636E-2,2.5007247E-3,6.955873E-4,4.645371E-4,-1.2894556E-3,1.1545842E-1,5.6604277E-2,-1.0954722E-1,-4.7756705E-2,-6.1876967E-4,1.3134038E-1,3.114417E-3,1.6044849E-3,-1.0913111E-3,3.5337707E-5,4.4216765E-3,1.5532823E-3,4.8091533E-4,6.671862E-2,-3.965182E-3,-5.2985642E-2,-6.183064E-2,-1.9893479E-2,-1.3247622E-3,8.532986E-4,2.3383405E-3,5.0975666E-3,1.7419852E-2,-2.1074023E-2,2.346038E-3,1.0153983E-3,-2.2973397E-3,-6.873776E-4,-6.783937E-2,-2.2914022E-2,-1.09611405E-2,-1.2720334E-3,5.7646446E-4,1.0328073E-3,1.2510078E-4,-1.1009269E-3,-7.3210366E-2,-8.3440257E-4,-1.7065371E-3,3.1323612E-5,6.45391E-4,-1.6912915E-2,5.043032E-4,-5.932674E-4,-4.627681E-2,-8.018096E-2,-2.2144945E-2,6.971347E-5,-1.9598005E-3,-5.3340435E-4,-3.9823223E-3,-7.141112E-2,-4.5301986E-4,-1.1418244E-3,-1.4254256E-3,-2.5117015E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,-1,-1,-1,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,-1,47,49,51,-1,-1,53,55,57,-1,59,-1,-1,-1,-1,61,63,-1,65,67,-1,-1,-1,69,-1,71,-1,73,-1,-1,-1,75,-1,-1,-1,-1,77,79,81,83,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,93,95,97,-1,-1,-1,-1,99,101,-1,-1,-1,-1,103,105,107,-1,109,-1,-1,-1,111,-1,-1,-1,-1,113,-1,-1,115,117,119,-1,-1,-1,-1,121,-1,-1,-1,-1],"loss_changes":[9.4660836E-1,2.994724E-1,6.502968E-1,1.224567E-1,4.1539627E-1,0E0,3.6468711E-1,1.6217123E-1,3.2853916E-2,1.7835826E-1,2.8701747E-2,1.10437155E-1,0E0,3.0420962E-1,5.57995E-2,0E0,0E0,0E0,2.3030862E-3,8.702961E-3,5.522551E-3,6.921214E-2,7.630557E-3,1.529248E-1,0E0,5.0671685E-2,3.816007E-2,0E0,0E0,9.632688E-4,5.8052675E-3,0E0,6.288413E-3,4.5982987E-2,7.009137E-2,0E0,0E0,9.371051E-2,1.18990995E-1,1.7231189E-2,0E0,8.7557845E-3,0E0,0E0,0E0,0E0,1.2884015E-2,3.8565323E-3,0E0,5.7507847E-3,1.6569316E-2,0E0,0E0,0E0,7.427558E-2,0E0,7.7482395E-2,0E0,8.833096E-3,0E0,0E0,0E0,2.3436942E-3,0E0,0E0,0E0,0E0,8.094147E-3,7.009864E-3,1.4299393E-2,3.2102317E-2,1.0988129E-2,4.3575317E-3,8.325708E-3,0E0,0E0,0E0,0E0,0E0,0E0,9.53421E-4,0E0,2.5466513E-3,1.1540547E-2,4.8255445E-3,0E0,0E0,0E0,0E0,3.6868574E-3,3.0820495E-3,0E0,0E0,0E0,0E0,8.908868E-3,6.52309E-3,4.3996833E-3,0E0,2.5833836E-3,0E0,0E0,0E0,4.1746795E-3,0E0,0E0,0E0,0E0,1.9445447E-3,0E0,0E0,3.7642624E-3,3.382966E-3,1.1143102E-3,0E0,0E0,0E0,0E0,1.7516762E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,29,29,30,30,32,32,33,33,34,34,37,37,38,38,39,39,41,41,46,46,47,47,49,49,50,50,54,54,56,56,58,58,62,62,67,67,68,68,69,69,70,70,71,71,72,72,73,73,80,80,82,82,83,83,84,84,89,89,90,90,95,95,96,96,97,97,99,99,103,103,108,108,111,111,112,112,113,113,118,118],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,-1,-1,-1,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,-1,48,50,52,-1,-1,54,56,58,-1,60,-1,-1,-1,-1,62,64,-1,66,68,-1,-1,-1,70,-1,72,-1,74,-1,-1,-1,76,-1,-1,-1,-1,78,80,82,84,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,94,96,98,-1,-1,-1,-1,100,102,-1,-1,-1,-1,104,106,108,-1,110,-1,-1,-1,112,-1,-1,-1,-1,114,-1,-1,116,118,120,-1,-1,-1,-1,122,-1,-1,-1,-1],"split_conditions":[2.1602268E9,2.4472736E7,2.3880364E9,1.33097E5,9.099E3,1.6522089E-2,1.2952493E8,1.22661E6,1.2609E5,2.702002E6,1.1156849E9,2.18449E5,-6.594684E-3,1.11386E5,1E0,-7.5305155E-3,-2.0429422E-3,1.0730745E-2,2.067E3,5.421011E7,7.4595E4,2.1866273E9,3.9461447E11,1.83E2,-1.04152635E-2,6.648864E6,1.78538E5,1.108624E-3,3.010461E-3,3.1043533E8,4.16E2,-9.0733534E-5,9.31023E8,6E1,5.2203887E3,7.7325106E-3,3.332705E-3,1.5E1,3.6E1,1.0382E4,4.2509246E-3,8.6E1,-3.883921E-3,-1.5168132E-3,-6.267971E-4,-1.2080924E-3,3.636015E8,4.7860208E-1,1.3458724E-4,3.2842822E3,3.0031217E10,7.952979E-3,1.6793392E-3,2.548305E-3,3.93461E5,-2.6259369E-3,3.5888664E7,2.6762611E-3,4.365884E6,-9.791458E-4,9.2589087E-4,1.3114676E-3,1.783892E6,2.5007247E-3,6.955873E-4,4.645371E-4,-1.2894556E-3,2.37E2,4.870603E0,6.549254E6,1.149807E9,3.15859E5,2.8E1,4.803464E6,1.6044849E-3,-1.0913111E-3,3.5337707E-5,4.4216765E-3,1.5532823E-3,4.8091533E-4,8.28337E5,-3.965182E-3,2.765E4,7.721E3,5.484E3,-1.3247622E-3,8.532986E-4,2.3383405E-3,5.0975666E-3,6.2764E4,2.44957E5,2.346038E-3,1.0153983E-3,-2.2973397E-3,-6.873776E-4,1.4397123E8,1.21E2,7.38E2,-1.2720334E-3,1.6175E4,1.0328073E-3,1.2510078E-4,-1.1009269E-3,1.01911224E3,-8.3440257E-4,-1.7065371E-3,3.1323612E-5,6.45391E-4,4.5681664E8,5.043032E-4,-5.932674E-4,7.3E1,1.1374049E3,3.9E2,6.971347E-5,-1.9598005E-3,-5.3340435E-4,-3.9823223E-3,1.3637682E3,-4.5301986E-4,-1.1418244E-3,-1.4254256E-3,-2.5117015E-3],"split_indices":[27,18,11,8,7,0,6,6,7,2,16,3,0,28,36,0,0,0,0,18,8,16,1,47,0,6,22,0,0,21,0,0,11,19,5,0,0,28,4,7,0,19,0,0,0,0,11,63,0,5,13,0,0,0,9,0,16,0,6,0,0,0,2,0,0,0,0,17,60,18,11,21,17,2,0,0,0,0,0,0,28,0,12,3,3,0,0,0,0,28,12,0,0,0,0,27,4,0,0,7,0,0,0,5,0,0,0,0,26,0,0,4,5,4,0,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.97E2,2.41E2,5.6E1,1.9E2,5.1E1,3E0,5.3E1,1.84E2,6E0,1.1E1,4E1,5E1,3E0,1.4E2,4.4E1,3E0,3E0,5E0,6E0,2.7E1,1.3E1,4.1E1,9E0,1.37E2,3E0,3.3E1,1.1E1,3E0,3E0,1.2E1,1.5E1,3E0,1E1,3.3E1,8E0,6E0,3E0,1.14E2,2.3E1,2.9E1,4E0,8E0,3E0,8E0,4E0,3E0,1.2E1,7E0,3E0,6E0,2.7E1,4E0,4E0,4E0,1.1E2,6E0,1.7E1,3E0,2.6E1,4E0,4E0,5E0,7E0,4E0,3E0,3E0,3E0,9E0,1.8E1,2.6E1,8.4E1,8E0,9E0,2.2E1,4E0,4E0,3E0,6E0,3E0,4E0,1.4E1,2E1,6E0,5.5E1,2.9E1,3E0,5E0,4E0,5E0,1.4E1,8E0,1.1E1,3E0,3E0,3E0,4.7E1,8E0,2.1E1,8E0,7E0,7E0,3E0,5E0,4.1E1,6E0,3E0,5E0,3E0,1.8E1,4E0,3E0,1E1,3.1E1,1.4E1,4E0,6E0,4E0,3E0,2.8E1,1E1,4E0,7E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[3.6302411E-3,-3.0461185E-2,1.2619086E-1,-4.8688326E-2,3.1965416E-2,8.8408574E-2,3.3886334E-1,-4.267514E-2,-6.72565E-3,1.7152661E-2,5.305134E-3,3.843617E-2,1.8621802E-1,1.6648917E-2,2.9848677E-3,-3.6386132E-2,-6.644255E-3,4.0181247E-3,4.2332243E-3,-3.4733373E-3,8.794579E-2,2.4189372E-1,6.4171396E-2,-7.089994E-2,-1.0935192E-2,-5.3809475E-2,2.8437156E-2,-4.322712E-2,3.6596205E-2,1.3879007E-1,4.7595263E-2,1.11578174E-1,1.1382433E-2,7.808941E-4,2.8339245E-3,1.0685935E-3,-7.781667E-2,3.7540585E-2,-2.1228269E-2,-3.653196E-3,-1.4673573E-2,-8.286521E-3,4.9049504E-2,-6.086904E-2,9.2172944E-5,5.330964E-2,-1.9248086E-4,2.3702378E-3,5.4479013E-3,-3.587765E-4,2.6637635E-3,1.7708113E-3,4.5144632E-3,-1.10607214E-1,-6.186825E-2,-6.736189E-2,1.0088392E-1,-2.9595908E-2,2.7957374E-2,1.3638036E-4,-7.994108E-4,-2.0805793E-3,2.7967986E-2,5.998259E-2,1.3692287E-4,-2.889371E-3,-8.234009E-4,6.0018006E-4,2.2654268E-3,-1.1808434E-1,-1.6473548E-3,1.1355485E-3,-6.946627E-2,-3.413369E-3,-3.8117223E-4,5.372944E-2,4.722874E-3,-3.4902092E-2,2.1629704E-2,-4.9309235E-4,5.1510237E-2,-2.0210342E-4,1.6020994E-3,2.8561156E-3,4.3006096E-2,-4.0799077E-3,-1.8746236E-3,-7.694351E-2,-4.3738127E-2,2.2281655E-3,7.9845043E-4,-4.033119E-2,1.5505076E-3,1.9776882E-3,-7.5927214E-4,2.227631E-3,5.315832E-4,2.867611E-4,2.1273275E-3,-8.193406E-2,-6.9290365E-4,-5.041547E-4,-1.653111E-3,-2.3116516E-3,-3.329454E-2,6.209915E-4,-5.3115486E-4,-2.0118318E-3,-9.590517E-2,-2.2181647E-2,-5.4782413E-2,-1.9988122E-3,-3.5905142E-3,-9.5112175E-3,-3.9968334E-2,-3.1623407E-3,-3.573034E-2,-8.7790063E-4,-3.3661015E-3,-6.844278E-5,-5.110832E-2,-3.9704013E-4,-1.3570344E-3,-3.2602172E-4,6.261653E-4,-6.454158E-4,-2.2040263E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,23,-1,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,57,-1,59,61,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,69,71,73,75,77,79,-1,-1,-1,81,83,-1,-1,-1,-1,-1,85,-1,-1,87,-1,-1,89,-1,91,93,-1,95,-1,-1,-1,97,-1,-1,99,101,-1,-1,103,105,-1,-1,-1,-1,-1,-1,107,-1,-1,-1,-1,109,-1,-1,-1,111,113,115,-1,-1,117,119,-1,121,-1,123,-1,125,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1904715E0,2.552988E-1,4.682271E-1,1.5779299E-1,9.753221E-2,2.567823E-1,3.4714425E-1,1.6548842E-1,0E0,6.485427E-2,0E0,7.7077635E-2,1.0678786E-1,0E0,0E0,1.4240536E-1,0E0,0E0,6.193924E-2,3.5033185E-2,2.8271362E-2,1.352495E-1,4.5327023E-3,5.2264303E-2,4.819858E-2,2.8699253E-2,2.3872249E-2,9.118697E-3,8.146266E-3,1.4286041E-3,2.3640275E-2,3.6367998E-3,0E0,0E0,0E0,0E0,2.714914E-2,1.19758576E-1,3.3341277E-2,0E0,1.8346254E-3,2.6323007E-2,9.539977E-3,5.8608595E-3,0E0,3.4213047E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.843976E-3,3.640397E-2,1.377682E-2,1.5893541E-2,1.923817E-2,1.3020368E-2,0E0,0E0,0E0,6.2861317E-3,3.9924756E-3,0E0,0E0,0E0,0E0,0E0,1.4333576E-3,0E0,0E0,4.9067587E-3,0E0,0E0,1.4196225E-3,0E0,1.2332626E-2,1.405013E-2,0E0,3.8110036E-3,0E0,0E0,0E0,8.855931E-3,0E0,0E0,7.086307E-3,1.762325E-3,0E0,0E0,9.572156E-3,3.2007224E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.8795431E-3,0E0,0E0,0E0,0E0,9.769291E-3,0E0,0E0,0E0,1.0049343E-3,6.8315305E-3,8.909967E-3,0E0,0E0,2.0912022E-3,5.371608E-3,0E0,1.2846487E-3,0E0,2.6659453E-3,0E0,4.0196534E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,15,15,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,36,36,37,37,38,38,40,40,41,41,42,42,43,43,45,45,53,53,54,54,55,55,56,56,57,57,58,58,62,62,63,63,69,69,72,72,75,75,77,77,78,78,80,80,84,84,87,87,88,88,91,91,92,92,99,99,104,104,108,108,109,109,110,110,113,113,114,114,116,116,118,118,120,120],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,24,-1,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,58,-1,60,62,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,70,72,74,76,78,80,-1,-1,-1,82,84,-1,-1,-1,-1,-1,86,-1,-1,88,-1,-1,90,-1,92,94,-1,96,-1,-1,-1,98,-1,-1,100,102,-1,-1,104,106,-1,-1,-1,-1,-1,-1,108,-1,-1,-1,-1,110,-1,-1,-1,112,114,116,-1,-1,118,120,-1,122,-1,124,-1,126,-1,-1,-1,-1,-1,-1],"split_conditions":[8.912913E8,7.529E3,1.2738551E4,1.24389E5,6.9052E4,1.83E2,1.7754914E0,1.11386E5,-6.72565E-3,5.61E2,5.305134E-3,2.981E3,5.2203887E3,1.6648917E-2,2.9848677E-3,6.92E2,-6.644255E-3,4.0181247E-3,7.2903E4,1.3925089E9,3.980565E3,1.8943199E9,8.6022E4,1.5E1,2.2155E4,4.3889E4,1.8277954E0,1.35E2,2.8709246E3,3.4182193E9,7.51842E5,7.581E3,1.1382433E-2,7.808941E-4,2.8339245E-3,1.0685935E-3,2.648E3,8.0145E4,1.914442E6,-3.653196E-3,1.6012754E3,1.5712E4,8.713217E-1,1.12E2,9.2172944E-5,1.02E3,-1.9248086E-4,2.3702378E-3,5.4479013E-3,-3.587765E-4,2.6637635E-3,1.7708113E-3,4.5144632E-3,8.35E2,9.692E3,9E0,3.1E1,7.164074E8,6.74207E5,1.3638036E-4,-7.994108E-4,-2.0805793E-3,2.0197088E8,1E2,1.3692287E-4,-2.889371E-3,-8.234009E-4,6.0018006E-4,2.2654268E-3,3.3E2,-1.6473548E-3,1.1355485E-3,1.1750605E8,-3.413369E-3,-3.8117223E-4,9.11E2,4.722874E-3,2.0292E4,1E3,-4.9309235E-4,2.748732E6,-2.0210342E-4,1.6020994E-3,2.8561156E-3,3.0826965E10,-4.0799077E-3,-1.8746236E-3,7.3831117E9,9.7E1,2.2281655E-3,7.9845043E-4,7.6058496E8,1.336082E6,1.9776882E-3,-7.5927214E-4,2.227631E-3,5.315832E-4,2.867611E-4,2.1273275E-3,7.214E3,-6.9290365E-4,-5.041547E-4,-1.653111E-3,-2.3116516E-3,2.203964E9,6.209915E-4,-5.3115486E-4,-2.0118318E-3,1.0022458E-2,1.6301E4,4.014E3,-1.9988122E-3,-3.5905142E-3,1.815494E9,9.4202E4,-3.1623407E-3,9.312365E2,-8.7790063E-4,1.0014492E8,-6.844278E-5,5.7E1,-3.9704013E-4,-1.3570344E-3,-3.2602172E-4,6.261653E-4,-6.454158E-4,-2.2040263E-3],"split_indices":[16,3,5,8,7,47,59,28,0,0,0,0,5,0,0,0,0,0,22,16,5,16,3,28,8,8,66,19,5,1,18,7,0,0,0,0,22,27,6,0,5,7,63,4,0,17,0,0,0,0,0,0,0,17,7,17,17,16,21,0,0,0,16,19,0,0,0,0,0,0,0,0,16,0,0,0,0,7,0,0,18,0,0,0,1,0,0,10,4,0,0,10,21,0,0,0,0,0,0,28,0,0,0,0,11,0,0,0,63,7,3,0,0,10,18,0,5,0,16,0,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.22E2,6.1E1,1.72E2,5E1,5.3E1,8E0,1.67E2,5E0,4.6E1,4E0,3.6E1,1.7E1,4E0,4E0,1.62E2,5E0,4E0,4.2E1,2E1,1.6E1,1.1E1,6E0,6.8E1,9.4E1,1.2E1,3E1,1E1,1E1,6E0,1E1,6E0,5E0,3E0,3E0,4E0,6.4E1,1.6E1,7.8E1,4E0,8E0,1.1E1,1.9E1,7E0,3E0,7E0,3E0,3E0,3E0,4E0,6E0,3E0,3E0,1.9E1,4.5E1,6E0,1E1,6.7E1,1.1E1,3E0,5E0,4E0,7E0,1.5E1,4E0,3E0,4E0,3E0,4E0,1.6E1,3E0,3E0,4.2E1,3E0,3E0,6E0,4E0,6.1E1,6E0,4E0,7E0,3E0,4E0,4E0,1.1E1,1.3E1,3E0,3.1E1,1.1E1,3E0,3E0,5.3E1,8E0,3E0,3E0,4E0,3E0,5E0,6E0,2.8E1,3E0,3E0,8E0,8E0,4.5E1,4E0,4E0,1.4E1,1.4E1,3.1E1,1.4E1,6E0,8E0,1.9E1,1.2E1,3E0,1.1E1,4E0,1.5E1,3E0,9E0,3E0,8E0,1.2E1,3E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[5.4556653E-3,-3.08421E-2,7.3835246E-2,1.4031824E-1,-3.956812E-2,8.754099E-2,-6.2162518E-3,-7.6944474E-4,1.1316738E-2,-6.818512E-2,-1.992621E-2,6.793606E-2,2.8490078E-1,-5.8151763E-2,-6.0189622E-3,-4.4663902E-2,-1.7559696E-3,7.1202633E-3,5.559191E-2,1.1405979E-2,4.0838453E-3,-6.434455E-2,1.735709E-3,-3.301254E-2,-4.9224775E-3,-1.276829E-2,4.4275027E-2,-6.3501224E-2,6.788168E-2,-1.08367816E-1,-4.832961E-2,5.258523E-3,-5.096905E-2,-3.0252999E-2,1.08601125E-2,2.9705453E-3,8.734666E-3,-3.583862E-3,3.9175054E-4,2.6310641E-2,8.770605E-2,-4.2133247E-3,-1.0504127E-3,7.924332E-4,-5.5520732E-2,-2.4461808E-2,2.7577826E-3,-6.0660217E-2,-4.2585362E-4,-1.6461133E-3,-2.377094E-2,1.672156E-3,-4.280369E-3,-3.068813E-4,6.9536077E-4,5.7551097E-2,-2.6462325E-3,1.4366002E-1,6.1219893E-2,-6.717457E-2,-9.180229E-3,-1.6709488E-3,2.2738191E-4,-6.7325994E-2,-4.452611E-4,-2.2193948E-3,-3.4408E-2,-8.0830755E-4,4.0607983E-3,8.3966844E-2,-5.8198033E-4,6.422099E-2,2.0514156E-1,2.3452384E-2,1.1001308E-1,-5.6918908E-2,-9.467256E-2,-7.9145026E-4,3.011151E-5,-1.5921172E-3,-2.9254851E-3,-5.1945687E-4,2.3914456E-4,-5.6786544E-4,-1.3745866E-3,-5.6052575E-4,1.2244479E-2,3.5995976E-3,3.615127E-2,2.955395E-3,4.4016977E-4,7.864704E-3,3.3757067E-3,-1.5267684E-3,5.692756E-2,4.3016914E-3,1.6835304E-3,-6.5799065E-2,-9.368158E-5,-1.7373388E-3,-3.923149E-3,-5.99145E-5,7.068238E-4,4.065931E-4,1.6298327E-3,2.4218764E-2,-2.705419E-2,2.5977232E-3,7.00667E-4,-2.3547066E-3,-7.4759003E-4,-1.1820086E-5,1.3760795E-3,-2.7987457E-4,-1.2441063E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,19,21,-1,23,25,-1,27,-1,-1,29,-1,31,-1,33,35,37,39,41,43,45,47,49,51,-1,53,-1,-1,55,57,-1,-1,-1,59,61,-1,63,-1,-1,65,-1,67,-1,-1,69,-1,71,73,75,77,-1,-1,79,-1,81,83,-1,85,87,-1,89,91,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.873164E-1,2.7588433E-1,3.6814922E-1,3.206371E-1,9.588331E-2,3.366137E-1,0E0,0E0,0E0,7.256603E-2,4.6668515E-2,1.4851776E-1,4.283732E-2,4.9277008E-2,0E0,5.0452396E-2,3.148671E-2,0E0,1.2140077E-1,0E0,0E0,3.9179504E-2,0E0,2.8443836E-2,0E0,2.0985337E-2,1.9669253E-2,3.031423E-2,5.7853132E-2,2.3024306E-2,2.654647E-2,3.555574E-2,9.43812E-3,2.7105343E-3,1.3908386E-2,0E0,2.0865456E-3,0E0,0E0,8.948658E-2,6.382343E-2,0E0,0E0,0E0,2.3145288E-2,9.885492E-3,0E0,5.9358105E-3,0E0,0E0,5.4255156E-3,0E0,3.055799E-3,0E0,0E0,4.1475236E-2,0E0,6.00861E-2,6.1917827E-2,4.481703E-3,1.5815657E-3,0E0,0E0,3.5010874E-3,0E0,1.32855E-3,1.4720503E-3,0E0,2.4978358E-3,1.59396E-2,0E0,9.902287E-3,5.811602E-3,1.7646726E-2,1.3462618E-2,1.3237432E-2,2.2800267E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4111253E-3,0E0,1.7447285E-3,0E0,0E0,0E0,0E0,9.19904E-3,5.7666637E-3,0E0,0E0,5.814716E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.2049655E-3,1.154209E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,21,21,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,39,39,40,40,44,44,45,45,47,47,50,50,52,52,55,55,57,57,58,58,59,59,60,60,63,63,65,65,66,66,68,68,69,69,71,71,72,72,73,73,74,74,75,75,76,76,86,86,88,88,93,93,94,94,97,97,105,105,106,106],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,20,22,-1,24,26,-1,28,-1,-1,30,-1,32,-1,34,36,38,40,42,44,46,48,50,52,-1,54,-1,-1,56,58,-1,-1,-1,60,62,-1,64,-1,-1,66,-1,68,-1,-1,70,-1,72,74,76,78,-1,-1,80,-1,82,84,-1,86,88,-1,90,92,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,3.453E3,6.25E2,3.139E3,4.258E3,1.2738551E4,-6.2162518E-3,-7.6944474E-4,1.1316738E-2,5.1041E4,2.83326E9,3.28451E5,5.3408066E4,2.352087E8,-6.0189622E-3,4.8102058E8,2.466E3,7.1202633E-3,8.288E3,1.1405979E-2,4.0838453E-3,2.748E3,1.735709E-3,2.433E3,-4.9224775E-3,8.327099E8,3.134E3,2.917845E8,2.93118E3,3.8508E4,1.8E1,1.4357E4,1.4397123E8,2.57488E5,3.7382904E7,2.9705453E-3,2.1009742E10,-3.583862E-3,3.9175054E-4,1E0,7.0909706E9,-4.2133247E-3,-1.0504127E-3,7.924332E-4,1.401993E6,4.1629E4,2.7577826E-3,1.6404E4,-4.2585362E-4,-1.6461133E-3,5.395E3,1.672156E-3,2.6245E4,-3.068813E-4,6.9536077E-4,4.9E2,-2.6462325E-3,1.45166E5,1.05055E5,7.2318E4,2.34E3,-1.6709488E-3,2.2738191E-4,6.126241E8,-4.452611E-4,4.939E3,1.44715E5,-8.0830755E-4,1.5491053E3,8.3E1,-5.8198033E-4,3.23E2,2.8125507E-1,7.1057E4,8.28337E5,5.99609E5,6.0946E4,-7.9145026E-4,3.011151E-5,-1.5921172E-3,-2.9254851E-3,-5.1945687E-4,2.3914456E-4,-5.6786544E-4,-1.3745866E-3,-5.6052575E-4,9.89E2,3.5995976E-3,4.472153E6,2.955395E-3,4.4016977E-4,7.864704E-3,3.3757067E-3,4.456605E9,3.4E1,4.3016914E-3,1.6835304E-3,1.0151622E8,-9.368158E-5,-1.7373388E-3,-3.923149E-3,-5.99145E-5,7.068238E-4,4.065931E-4,1.6298327E-3,1.0482055E10,4.90364E5,2.5977232E-3,7.00667E-4,-2.3547066E-3,-7.4759003E-4,-1.1820086E-5,1.3760795E-3,-2.7987457E-4,-1.2441063E-3],"split_indices":[12,7,4,3,3,5,0,0,0,20,10,6,5,16,0,11,0,0,20,0,0,3,0,20,0,25,0,16,5,12,17,7,27,6,27,0,10,0,0,36,21,0,0,0,2,12,0,7,0,0,3,0,18,0,0,17,0,22,3,12,0,0,0,10,0,3,12,0,5,19,0,17,63,8,28,9,8,0,0,0,0,0,0,0,0,0,0,0,6,0,0,0,0,11,4,0,0,16,0,0,0,0,0,0,0,21,18,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.8E2,9.5E1,8E0,1.72E2,9.1E1,4E0,5E0,3E0,6.9E1,1.03E2,8.4E1,7E0,6.5E1,4E0,4.3E1,6E1,5E0,7.9E1,4E0,3E0,6.2E1,3E0,4E1,3E0,4.9E1,1.1E1,7E0,7.2E1,1.5E1,4.7E1,1.3E1,2.7E1,2.8E1,2.1E1,4E0,7E0,4E0,3E0,2.4E1,4.8E1,1.1E1,4E0,4E0,4.3E1,1E1,3E0,2.1E1,6E0,5E0,2.3E1,5E0,1.6E1,3E0,4E0,1.9E1,5E0,1.4E1,3.4E1,3.4E1,9E0,5E0,5E0,1.8E1,3E0,8E0,1.5E1,4E0,1.2E1,1.4E1,5E0,7E0,7E0,2E1,1.4E1,2.7E1,7E0,3E0,6E0,1.2E1,6E0,3E0,5E0,6E0,9E0,3E0,9E0,8E0,6E0,4E0,3E0,4E0,3E0,1.2E1,8E0,9E0,5E0,2.3E1,4E0,4E0,3E0,4E0,5E0,3E0,3E0,6E0,6E0,4E0,4E0,1.9E1,4E0,3E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[1.8668387E-2,-1.544073E-2,1.4215037E-1,-5.323925E-2,2.1177325E-2,9.884615E-2,3.861812E-1,-4.5265004E-2,-5.9628244E-3,1.4768332E-1,-3.8205222E-3,8.2663886E-2,6.9053997E-3,1.6397828E-2,7.714271E-3,5.9672683E-2,-5.215799E-2,1.9599807E-1,5.6750774E-2,-1.3542974E-2,5.0231747E-2,-3.7942172E-4,8.907999E-2,3.4235457E-3,-6.214287E-5,-1.3828203E-2,-7.169412E-2,2.7677563E-3,2.402567E-1,3.1992593E-3,-3.4557743E-4,-2.0136518E-2,2.7146868E-2,3.0206586E-3,3.1509202E-2,3.7536394E-2,1.0420694E-1,-2.633509E-2,2.886833E-3,-1.276824E-1,-5.416492E-2,9.393472E-3,3.7253862E-3,-1.5726272E-2,-2.4492117E-3,2.7007662E-4,1.7375165E-3,4.8461277E-2,2.085924E-4,7.2093785E-2,-8.1318046E-4,5.4249605E-3,8.210587E-2,-5.668854E-2,4.2250962E-4,-6.629431E-3,-2.1872872E-3,-7.21776E-2,5.4252273E-3,-2.4014661E-2,1.7958561E-2,-4.4294074E-4,4.5815436E-4,2.209401E-4,2.5089164E-3,9.434547E-4,2.9582346E-3,4.6633533E-3,6.877868E-2,-2.7304183E-4,-2.5732925E-3,8.090459E-4,-1.4027891E-2,-7.741274E-2,-6.453577E-4,1.881254E-3,-2.4632491E-2,1.2832183E-2,-2.9655496E-2,1.1204827E-3,-3.319001E-4,4.365335E-2,2.9729726E-3,-9.237813E-4,-9.9951474E-5,-8.0952115E-2,-9.289648E-4,-1.3300475E-3,2.1105907E-4,1.3714572E-3,-4.3627835E-4,-3.816895E-2,-3.7985835E-3,1.930239E-3,-3.9011127E-4,-8.720213E-2,-1.4575262E-3,5.881658E-4,-4.3853678E-2,6.6144724E-4,-2.1584362E-2,-1.5900983E-3,-9.354838E-2,-4.805399E-2,-4.46752E-5,-9.942319E-4,-1.4676654E-4,-3.2029452E-3,-1.3477479E-3,-5.9394278E-2,-2.6094036E-2,-2.1267023E-3,-5.744306E-4,-1.5831183E-5,-1.1128838E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,27,29,31,33,-1,35,-1,-1,37,39,-1,41,-1,-1,43,45,-1,47,49,51,53,-1,55,57,-1,-1,59,-1,61,-1,63,-1,65,-1,-1,67,69,71,-1,-1,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,81,-1,-1,-1,83,85,-1,-1,87,89,91,-1,-1,93,-1,-1,-1,95,-1,-1,-1,-1,-1,97,99,-1,-1,101,-1,-1,103,-1,105,-1,107,109,-1,-1,-1,-1,-1,111,113,-1,-1,-1,-1],"loss_changes":[1.2067871E0,3.125518E-1,6.1701655E-1,1.0845813E-1,3.642376E-1,8.335316E-2,4.043734E-2,7.978584E-2,0E0,7.235128E-2,5.1655784E-2,3.0744463E-2,0E0,0E0,0E0,2.0336999E-2,7.389018E-2,3.7770122E-2,2.4087653E-2,2.2814736E-2,9.290401E-3,0E0,3.1954527E-2,0E0,0E0,4.767336E-2,5.7858467E-2,0E0,1.7594725E-2,0E0,0E0,1.6570298E-2,8.639247E-3,0E0,4.7774715E-3,2.7863367E-2,3.592235E-2,2.601389E-2,0E0,5.6133315E-2,5.6207255E-2,0E0,0E0,1.941639E-2,0E0,1.567255E-3,0E0,8.049669E-3,0E0,4.0861927E-3,0E0,0E0,1.3458461E-2,1.6137216E-2,6.7802873E-3,0E0,0E0,9.363949E-3,2.2156755E-2,1.1812128E-2,7.0280926E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.07804015E-2,0E0,0E0,0E0,1.8338321E-3,3.8502514E-3,0E0,0E0,4.9551823E-3,6.862427E-3,1.0421328E-2,0E0,0E0,1.346387E-2,0E0,0E0,0E0,3.4087896E-3,0E0,0E0,0E0,0E0,0E0,1.235202E-2,6.073038E-3,0E0,0E0,1.8563569E-3,0E0,0E0,5.8194995E-3,0E0,1.1262235E-3,0E0,2.6053935E-3,5.9211254E-3,0E0,0E0,0E0,0E0,0E0,4.090391E-3,2.5881017E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,28,28,31,31,32,32,34,34,35,35,36,36,37,37,39,39,40,40,43,43,45,45,47,47,49,49,52,52,53,53,54,54,57,57,58,58,59,59,60,60,68,68,72,72,73,73,76,76,77,77,78,78,81,81,85,85,91,91,92,92,95,95,98,98,100,100,102,102,103,103,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,28,30,32,34,-1,36,-1,-1,38,40,-1,42,-1,-1,44,46,-1,48,50,52,54,-1,56,58,-1,-1,60,-1,62,-1,64,-1,66,-1,-1,68,70,72,-1,-1,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,82,-1,-1,-1,84,86,-1,-1,88,90,92,-1,-1,94,-1,-1,-1,96,-1,-1,-1,-1,-1,98,100,-1,-1,102,-1,-1,104,-1,106,-1,108,110,-1,-1,-1,-1,-1,112,114,-1,-1,-1,-1],"split_conditions":[2.1602268E9,4.995E3,1.2738551E4,6.9052E4,9.099E3,4.0176924E9,8.3748E4,3.185029E2,-5.9628244E-3,5.6035635E9,1.36692E9,1.2862974E-1,6.9053997E-3,1.6397828E-2,7.714271E-3,2.0871878E8,6.216743E-4,1.829467E6,4.3406326E8,4.9749E4,6.9585E4,-3.7942172E-4,7.469149E6,3.4235457E-3,-6.214287E-5,2.6876314E8,7.2964703E2,2.7677563E-3,4.970634E6,3.1992593E-3,-3.4557743E-4,1.83E2,8E1,3.0206586E-3,1.1443E4,1.1408653E9,6.2088E4,1.850778E6,2.886833E-3,2.7708E4,5.13905E5,9.393472E-3,3.7253862E-3,2.846864E6,-2.4492117E-3,6.257519E6,1.7375165E-3,1.9118268E3,2.085924E-4,2.54044E5,-8.1318046E-4,5.4249605E-3,6.3E1,8.765375E-6,1.20791E5,-6.629431E-3,-2.1872872E-3,1.253E3,2.8895E4,6.0216E4,1.9966909E8,-4.4294074E-4,4.5815436E-4,2.209401E-4,2.5089164E-3,9.434547E-4,2.9582346E-3,4.6633533E-3,6.63395E5,-2.7304183E-4,-2.5732925E-3,8.090459E-4,6.8001E4,1.7088E4,-6.453577E-4,1.881254E-3,4.94E2,1.43E2,1.1663035E9,1.1204827E-3,-3.319001E-4,5.637395E3,2.9729726E-3,-9.237813E-4,-9.9951474E-5,2.1811952E7,-9.289648E-4,-1.3300475E-3,2.1105907E-4,1.3714572E-3,-4.3627835E-4,7.1E1,9.8E2,1.930239E-3,-3.9011127E-4,2E1,-1.4575262E-3,5.881658E-4,1.044089E8,6.6144724E-4,2.77E2,-1.5900983E-3,1.02E3,2.6369473E-1,-4.46752E-5,-9.942319E-4,-1.4676654E-4,-3.2029452E-3,-1.3477479E-3,5.737332E9,1.6E1,-2.1267023E-3,-5.744306E-4,-1.5831183E-5,-1.1128838E-3],"split_indices":[27,3,5,7,7,16,20,5,0,10,16,63,0,0,0,10,63,18,16,20,8,0,6,0,0,16,5,0,2,0,0,47,4,0,3,16,20,25,0,8,6,0,0,2,0,2,0,5,0,22,0,0,19,64,18,0,0,17,18,8,1,0,0,0,0,0,0,0,22,0,0,0,8,7,0,0,19,4,11,0,0,5,0,0,0,18,0,0,0,0,0,17,0,0,0,17,0,0,18,0,17,0,17,63,0,0,0,0,0,10,19,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,2.24E2,6.1E1,1.1E2,1.14E2,5.3E1,8E0,1.05E2,5E0,1.8E1,9.6E1,4.8E1,5E0,3E0,5E0,6E0,9.9E1,1.1E1,7E0,8.2E1,1.4E1,3E0,4.5E1,3E0,3E0,3.4E1,6.5E1,4E0,7E0,4E0,3E0,7.1E1,1.1E1,3E0,1.1E1,1.1E1,3.4E1,3.1E1,3E0,1.4E1,5.1E1,4E0,3E0,6.7E1,4E0,6E0,5E0,6E0,5E0,7E0,4E0,7E0,2.7E1,1.4E1,1.7E1,5E0,9E0,3.9E1,1.2E1,5.4E1,1.3E1,3E0,3E0,3E0,3E0,3E0,4E0,3E0,2.4E1,5E0,9E0,6E0,1.1E1,3.5E1,4E0,4E0,8E0,7E0,4.7E1,8E0,5E0,1.3E1,1.1E1,4E0,7E0,3.2E1,3E0,5E0,3E0,3E0,4E0,3.5E1,1.2E1,1E1,3E0,2.6E1,6E0,3E0,3.2E1,5E0,7E0,5E0,2.1E1,2.9E1,3E0,4E0,3E0,1.8E1,3E0,1.8E1,1.1E1,1.5E1,3E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[3.3386697E-3,-2.3764526E-2,9.5350504E-2,-4.3400552E-2,2.0925356E-2,6.5598555E-2,2.3003258E-1,-3.892581E-2,-6.3051227E-3,3.7206735E-2,-5.9597483E-3,2.5324004E-2,9.563158E-2,1.1542193E-2,2.03274E-3,-7.762687E-2,-2.1470193E-2,2.3341088E-2,7.481466E-3,-1.5000968E-3,3.4135208E-3,1.3564578E-1,5.7053313E-2,-3.2664626E-3,-5.7967853E-2,-1.4352227E-2,-6.1890455E-3,6.8410456E-2,-3.3659416E-3,2.0188008E-2,-2.1329818E-3,5.686649E-3,6.938917E-2,-2.9315712E-4,7.188113E-2,-6.198194E-5,-6.48929E-2,4.5322284E-2,-2.554794E-2,1.8556861E-2,1.725886E-1,-4.0878924E-3,1.0859461E-2,2.3312615E-3,1.0935912E-3,3.3764003E-3,5.323389E-4,2.8854497E-3,7.076814E-4,-9.269421E-4,-6.86782E-2,-6.757028E-3,4.2890925E-3,-3.2237012E-2,2.1496702E-3,-2.141793E-3,5.24667E-2,7.06771E-3,2.6543231E-3,-2.1592118E-2,4.649008E-2,-2.92736E-4,7.6426065E-4,-2.9064766E-3,-1.7385958E-3,-1.6616185E-3,2.6253786E-2,-5.3089853E-2,-1.6123239E-2,2.7997093E-2,3.0388515E-3,5.7884044E-4,-3.0136283E-2,2.8678596E-3,2.1313692E-2,1.8196316E-3,-3.4073796E-4,-6.850766E-2,-2.9730875E-4,-4.5757897E-2,-5.5021215E-3,1.4988992E-3,3.5252984E-4,-1.3286514E-3,-3.1919652E-4,-8.463563E-4,1.558071E-3,-2.5507165E-4,-7.967002E-2,-7.2681956E-4,-1.7550864E-3,1.2547789E-3,-1.8728115E-3,-9.042684E-2,-8.743432E-4,-3.3524044E-2,1.0570047E-2,-3.5268394E-3,-5.5198606E-2,-1.4533034E-3,-4.351288E-4,1.7218704E-3,1.525371E-3,-2.3419352E-3,-6.2615244E-4,7.41266E-4,-9.756498E-3,1.17094554E-4,-6.1449275E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,31,33,-1,35,37,-1,39,41,43,-1,-1,45,-1,47,-1,49,51,53,55,57,-1,59,-1,61,-1,-1,-1,-1,-1,63,65,-1,67,-1,-1,69,-1,-1,71,73,-1,-1,-1,-1,-1,75,77,79,81,-1,-1,83,-1,85,-1,-1,87,-1,89,91,-1,-1,-1,-1,-1,-1,-1,93,-1,-1,95,-1,97,-1,99,101,-1,103,-1,-1,105,-1,-1,-1,-1,107,-1,-1],"loss_changes":[6.9055885E-1,1.8869996E-1,2.3735464E-1,9.40378E-2,2.2833017E-1,6.1685264E-2,2.1332401E-1,9.696376E-2,0E0,1.6239691E-1,0E0,5.2126367E-2,3.8571298E-2,0E0,0E0,1.5217364E-2,1.21248364E-1,7.124673E-2,0E0,2.8025428E-2,0E0,2.5790155E-2,1.7331965E-2,0E0,1.035364E-2,6.7299664E-2,0E0,1.1105771E-1,6.820963E-2,1.4882106E-2,0E0,0E0,1.1400279E-2,0E0,1.0435231E-2,0E0,1.3014898E-3,7.416555E-2,5.4031894E-2,4.9659953E-2,1.1534125E-2,0E0,4.1473262E-2,0E0,2.9849147E-3,0E0,0E0,0E0,0E0,0E0,1.7804578E-3,1.764611E-2,0E0,2.5689438E-2,0E0,0E0,9.671934E-3,0E0,0E0,6.9664503E-3,1.6335703E-2,0E0,0E0,0E0,0E0,0E0,8.405805E-3,2.2355467E-2,1.4226883E-2,2.338559E-3,0E0,0E0,3.1948006E-3,0E0,1.6748924E-2,0E0,0E0,1.6275056E-2,0E0,1.2071654E-3,1.2531302E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.404764E-3,0E0,0E0,1.0721521E-2,0E0,5.4145902E-3,0E0,1.0176832E-3,8.388474E-3,0E0,3.605565E-3,0E0,0E0,5.6051356E-3,0E0,0E0,0E0,0E0,1.9447043E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,15,15,16,16,17,17,19,19,21,21,22,22,24,24,25,25,27,27,28,28,29,29,32,32,34,34,36,36,37,37,38,38,39,39,40,40,42,42,44,44,50,50,51,51,53,53,56,56,59,59,60,60,66,66,67,67,68,68,69,69,72,72,74,74,77,77,79,79,80,80,88,88,91,91,93,93,95,95,96,96,98,98,101,101,106,106],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,32,34,-1,36,38,-1,40,42,44,-1,-1,46,-1,48,-1,50,52,54,56,58,-1,60,-1,62,-1,-1,-1,-1,-1,64,66,-1,68,-1,-1,70,-1,-1,72,74,-1,-1,-1,-1,-1,76,78,80,82,-1,-1,84,-1,86,-1,-1,88,-1,90,92,-1,-1,-1,-1,-1,-1,-1,94,-1,-1,96,-1,98,-1,100,102,-1,104,-1,-1,106,-1,-1,-1,-1,108,-1,-1],"split_conditions":[1.0886205E9,7.529E3,2.51383E5,8.919402E9,6.0230115E10,6.92103E5,3.91E2,4.89E2,-6.3051227E-3,6.2901763E3,-5.9597483E-3,2.2403428E9,2.035576E7,1.1542193E-2,2.03274E-3,2.93E2,1.24389E5,1.95E2,7.481466E-3,5.559532E6,3.4135208E-3,7.3100853E0,3E1,-3.2664626E-3,2E1,2.9169E4,-6.1890455E-3,1.3880545E7,1.2437245E3,1.37507E5,-2.1329818E-3,5.686649E-3,5.7E1,-2.9315712E-4,1.328034E10,-6.198194E-5,4.1677676E7,2.0449717E8,3.0831E4,3.1E1,4.833E3,-4.0878924E-3,7.8581E4,2.3312615E-3,1.32E2,3.3764003E-3,5.323389E-4,2.8854497E-3,7.076814E-4,-9.269421E-4,6.6394E4,2.1467798E9,4.2890925E-3,1.0816216E3,2.1496702E-3,-2.141793E-3,8.5867E4,7.06771E-3,2.6543231E-3,9.1416E4,2.4636838E3,-2.92736E-4,7.6426065E-4,-2.9064766E-3,-1.7385958E-3,-1.6616185E-3,4.3683886E9,2.39E2,3.8786424E7,1E0,3.0388515E-3,5.7884044E-4,3.7732E4,2.8678596E-3,1.306E4,1.8196316E-3,-3.4073796E-4,1.14903E5,-2.9730875E-4,3.091182E6,2.51868E5,1.4988992E-3,3.5252984E-4,-1.3286514E-3,-3.1919652E-4,-8.463563E-4,1.558071E-3,-2.5507165E-4,4.7339E4,-7.2681956E-4,-1.7550864E-3,1.4110581E8,-1.8728115E-3,1.5712E4,-8.743432E-4,4.322E3,4.805687E7,-3.5268394E-3,6.6E1,-1.4533034E-3,-4.351288E-4,4.821E3,1.525371E-3,-2.3419352E-3,-6.2615244E-4,7.41266E-4,1.41437E5,1.17094554E-4,-6.1449275E-4],"split_indices":[16,3,3,25,1,12,19,0,0,5,0,16,6,0,0,0,8,17,0,2,0,60,19,0,17,12,0,18,5,12,0,0,4,0,27,0,10,26,7,17,7,0,22,0,4,0,0,0,0,0,8,10,0,5,0,0,8,0,0,12,5,0,0,0,0,0,10,4,16,45,0,0,28,0,7,0,0,21,0,21,12,0,0,0,0,0,0,0,28,0,0,11,0,7,0,3,18,0,4,0,0,3,0,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.13E2,6.2E1,1.48E2,6.5E1,5.2E1,1E1,1.45E2,3E0,6.1E1,4E0,2.3E1,2.9E1,5E0,5E0,4.4E1,1.01E2,5.8E1,3E0,1.8E1,5E0,1.3E1,1.6E1,1.8E1,2.6E1,9.8E1,3E0,2.1E1,3.7E1,1.4E1,4E0,7E0,6E0,3E0,1.3E1,3E0,2.3E1,1.5E1,8.3E1,1.5E1,6E0,3E0,3.4E1,3E0,1.1E1,3E0,3E0,9E0,4E0,3E0,2E1,1E1,5E0,7.8E1,5E0,4E0,1.1E1,3E0,3E0,1.8E1,1.6E1,8E0,3E0,6E0,1.4E1,4E0,6E0,3.3E1,4.5E1,8E0,3E0,3E0,1.5E1,5E0,1.1E1,3E0,3E0,2.4E1,9E0,1.1E1,3.4E1,3E0,5E0,9E0,6E0,4E0,7E0,4E0,2E1,4E0,7E0,3.1E1,3E0,1.6E1,4E0,6E0,2.5E1,9E0,7E0,3E0,3E0,2.1E1,4E0,4E0,3E0,7E0,1.4E1,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-5.572733E-3,-2.3188975E-2,1.4587612E-1,-3.9995346E-2,3.6818873E-2,2.8089416E-1,7.10733E-2,-3.4587346E-2,-9.3212975E-3,5.4028958E-2,-6.3127996E-3,4.0542513E-1,-2.3364658E-4,-1.6195907E-3,3.8520591E-3,-2.7950874E-2,-1.5257603E-1,-2.5515237E-3,7.1066886E-2,1.85195E-2,4.3183723E-3,-3.7643654E-4,2.8259726E-4,-4.1864995E-2,3.9014664E-2,-5.953499E-3,-1.8595367E-3,-2.4991645E-2,1.7442121E-3,5.419914E-2,4.241314E-3,-7.18974E-2,-2.5529169E-2,-3.5653908E-3,7.085391E-2,-1.0801374E-3,-3.1274502E-4,-6.988355E-3,6.805519E-2,-3.3553997E-3,-5.7614155E-2,-3.2468587E-2,2.806562E-2,6.9391313E-3,4.3781504E-2,-1.2596586E-3,8.6599984E-4,5.0262347E-2,5.0793677E-3,-3.6680363E-2,-6.667027E-2,-3.7772283E-2,1.4432028E-3,3.8618024E-2,-8.670721E-5,-9.479336E-3,8.957755E-2,6.220712E-2,-2.4364523E-4,-1.4839752E-3,-3.3801713E-4,-7.172381E-2,-6.741917E-4,-4.987075E-2,-1.9200353E-2,-1.6279586E-3,2.210619E-2,6.3516415E-4,1.602369E-3,5.5941608E-2,-3.6134354E-3,8.983925E-4,1.1388426E-1,7.1748815E-2,1.9645346E-4,-7.799338E-2,-1.0800912E-3,-4.3086357E-2,-3.5515777E-3,6.5096417E-3,-4.53172E-2,1.6088376E-3,3.0196758E-3,3.690872E-3,-7.1574104E-5,4.306549E-3,1.7882615E-3,8.352018E-4,2.5657876E-3,-8.337623E-2,-1.1898207E-3,-2.130438E-2,-4.9908154E-2,-1.3350606E-2,2.1746876E-2,-2.1092264E-2,-3.6283054E-3,-2.7092226E-4,4.4102265E-4,-1.935422E-3,-3.3694978E-3,-1.0520249E-3,-2.7386288E-4,-6.937906E-2,-3.293442E-2,1.0488342E-4,-7.714941E-4,1.0963501E-3,8.439838E-5,-1.0433026E-3,4.0170766E-4,-2.669354E-3,-1.2068623E-3,-4.6046716E-5,-4.4083636E-2,-1.7910235E-3,-6.244795E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,-1,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,35,-1,37,-1,39,41,-1,43,-1,-1,45,47,-1,49,51,53,-1,55,-1,-1,57,-1,59,61,63,65,67,-1,69,71,73,-1,-1,-1,75,-1,77,79,-1,81,-1,-1,83,-1,-1,85,87,-1,89,-1,91,-1,93,95,-1,97,-1,-1,-1,-1,-1,-1,99,-1,101,103,105,107,109,-1,-1,-1,-1,-1,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,115,-1,-1],"loss_changes":[7.424075E-1,2.5259268E-1,2.7292806E-1,2.5318682E-1,2.3108244E-1,3.6178225E-1,7.084679E-2,1.4607686E-1,0E0,5.0310105E-2,0E0,2.4545634E-1,0E0,1.0455917E-3,0E0,1.7214665E-1,2.0012364E-2,1.7905964E-2,3.381081E-2,0E0,0E0,0E0,0E0,7.273635E-2,1.6045138E-1,0E0,0E0,9.822627E-4,0E0,2.8453559E-2,0E0,1.8478543E-2,3.8102917E-2,0E0,9.634635E-2,0E0,0E0,8.679007E-3,3.518194E-2,0E0,4.9480945E-3,1.6061857E-2,3.9990405E-3,0E0,5.938839E-2,0E0,0E0,1.7049704E-2,0E0,2.9690098E-3,4.5057237E-3,1.6343743E-2,1.5091106E-2,9.14949E-4,0E0,8.696557E-2,1.6307667E-2,1.02686435E-2,0E0,0E0,0E0,2.8869286E-3,0E0,1.4115952E-2,2.1781847E-2,0E0,5.1694103E-3,0E0,0E0,2.7579121E-2,0E0,0E0,3.0168742E-3,4.1307583E-3,0E0,1.1144057E-3,0E0,5.3261966E-3,0E0,5.434793E-3,2.3747347E-2,0E0,9.7040436E-4,0E0,0E0,0E0,0E0,0E0,0E0,1.6302839E-3,0E0,1.4691683E-3,8.29085E-3,1.4888026E-3,2.2655358E-3,5.344738E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9389188E-3,6.6105705E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9143374E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,15,15,16,16,17,17,18,18,23,23,24,24,27,27,29,29,31,31,32,32,34,34,37,37,38,38,40,40,41,41,42,42,44,44,47,47,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,61,61,63,63,64,64,66,66,69,69,72,72,73,73,75,75,77,77,79,79,80,80,82,82,89,89,91,91,92,92,93,93,94,94,95,95,103,103,104,104,114,114],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,-1,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,36,-1,38,-1,40,42,-1,44,-1,-1,46,48,-1,50,52,54,-1,56,-1,-1,58,-1,60,62,64,66,68,-1,70,72,74,-1,-1,-1,76,-1,78,80,-1,82,-1,-1,84,-1,-1,86,88,-1,90,-1,92,-1,94,96,-1,98,-1,-1,-1,-1,-1,-1,100,-1,102,104,106,108,110,-1,-1,-1,-1,-1,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,116,-1,-1],"split_conditions":[1.56238E9,6.0620896E8,9.068E3,1.06056E5,5.3408066E4,2.054214E0,3.5464888E9,1.14153E5,-9.3212975E-3,1.864E3,-6.3127996E-3,5.101353E7,-2.3364658E-4,5.8882404E7,3.8520591E-3,1.83E2,1.8697E4,4.37849E5,2.3177024E7,1.85195E-2,4.3183723E-3,-3.7643654E-4,2.8259726E-4,7.19175E5,1.407E3,-5.953499E-3,-1.8595367E-3,1.3222029E0,1.7442121E-3,6E1,4.241314E-3,2.578E4,4.416E3,-3.5653908E-3,9.92E2,-1.0801374E-3,-3.1274502E-4,1.7169811E9,7.0231123E3,-3.3553997E-3,5.7995E4,5.581883E7,3.3831064E7,6.9391313E-3,4.796E3,-1.2596586E-3,8.6599984E-4,8.28337E5,5.0793677E-3,1.18E2,6.44E2,1.935849E6,5.097E3,8.1214E4,-8.670721E-5,1.306E3,5.9271344E2,1.0532446E0,-2.4364523E-4,-1.4839752E-3,-3.3801713E-4,1.058E3,-6.741917E-4,2.215E3,7.8E1,-1.6279586E-3,1.4607E4,6.3516415E-4,1.602369E-3,6.925979E9,-3.6134354E-3,8.983925E-4,1.080557E6,1.9E1,1.9645346E-4,4.4E1,-1.0800912E-3,1.1268727E3,-3.5515777E-3,8.383E3,2.1321451E8,1.6088376E-3,6.887E3,3.690872E-3,-7.1574104E-5,4.306549E-3,1.7882615E-3,8.352018E-4,2.5657876E-3,8.120952E-3,-1.1898207E-3,9.5E1,4.951E3,1.17E2,8.183E3,6.177E4,-3.6283054E-3,-2.7092226E-4,4.4102265E-4,-1.935422E-3,-3.3694978E-3,-1.0520249E-3,-2.7386288E-4,3.28E2,6.0216E4,1.0488342E-4,-7.714941E-4,1.0963501E-3,8.439838E-5,-1.0433026E-3,4.0170766E-4,-2.669354E-3,-1.2068623E-3,-4.6046716E-5,7.19E2,-1.7910235E-3,-6.244795E-4],"split_indices":[16,27,7,28,5,59,21,8,0,0,0,18,0,18,0,47,28,12,18,0,0,0,0,2,28,0,0,64,0,19,0,12,0,0,20,0,0,1,5,0,8,18,16,0,7,0,0,28,0,4,4,2,3,8,0,0,5,63,0,0,0,17,0,19,19,0,7,0,0,10,0,0,6,4,0,19,0,5,0,20,27,0,3,0,0,0,0,0,0,63,0,17,3,4,3,20,0,0,0,0,0,0,0,4,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.48E2,2.8E1,1.94E2,5.4E1,9E0,1.9E1,1.91E2,3E0,5.1E1,3E0,6E0,3E0,8E0,1.1E1,1.82E2,9E0,1.2E1,3.9E1,3E0,3E0,4E0,4E0,1.51E2,3.1E1,6E0,3E0,9E0,3E0,3.2E1,7E0,5.2E1,9.9E1,5E0,2.6E1,5E0,4E0,6E0,2.6E1,1.4E1,3.8E1,8.8E1,1.1E1,3E0,2.3E1,3E0,3E0,2.3E1,3E0,1.3E1,2.5E1,7.6E1,1.2E1,8E0,3E0,1.1E1,1.2E1,1.9E1,4E0,9E0,4E0,2.2E1,3E0,4.5E1,3.1E1,3E0,9E0,4E0,4E0,7E0,4E0,4E0,8E0,1.6E1,3E0,1.8E1,4E0,4.2E1,3E0,1.6E1,1.5E1,3E0,6E0,3E0,4E0,5E0,3E0,3E0,1.3E1,1.5E1,3E0,1.1E1,3.1E1,7E0,9E0,1.2E1,3E0,3E0,3E0,9E0,6E0,5E0,6E0,1.3E1,1.8E1,3E0,4E0,5E0,4E0,9E0,3E0,8E0,5E0,5E0,1.3E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[-2.7279244E-3,-3.934175E-2,8.30236E-2,-7.1537554E-2,-1.8866803E-2,9.721866E-2,-7.362617E-3,-1.14687726E-1,-4.983393E-2,-2.6573228E-2,4.8385493E-2,6.583984E-2,3.165622E-1,-8.963024E-2,-7.204919E-3,-6.255424E-2,4.4212807E-3,-2.0944351E-2,-4.5617083E-3,-4.6253015E-4,6.864883E-2,8.302099E-3,5.3923402E-2,1.5709443E-2,5.5800434E-3,-3.6174979E-3,-5.1088385E-2,-7.482073E-2,-4.31218E-2,1.1601892E-3,-1.8797329E-2,-2.8094593E-2,2.4783447E-2,9.114481E-4,3.3257313E-3,1.3702315E-2,7.737899E-2,-2.4646979E-2,-2.722824E-3,-8.272349E-2,-8.5831806E-4,-2.5077206E-3,-3.1730577E-2,-9.2479185E-4,-1.3407443E-4,1.6337543E-3,-3.2208685E-2,-1.5320034E-2,3.4781592E-3,8.257534E-2,-9.907853E-3,1.4613949E-1,5.2857094E-2,-1.249028E-3,-1.3935294E-4,-8.9469984E-2,-1.0198557E-3,-1.1616723E-3,-3.1006226E-4,-4.5197107E-2,-1.3587452E-2,-2.6281944E-3,1.7241117E-3,3.7728755E-3,8.786497E-4,3.0465145E-3,-1.3777366E-3,2.394584E-3,5.6817057E-3,2.4920827E-2,8.658164E-2,-1.3534668E-3,-3.0950103E-3,-3.593552E-2,-8.5454084E-2,-3.4075675E-3,-1.481036E-3,-1.2623551E-2,8.046308E-4,5.802823E-2,-7.442339E-3,3.562356E-3,1.0301722E-3,-4.5659892E-2,-6.1988723E-3,-1.3972792E-3,-3.1944977E-3,-1.2486209E-2,2.8205998E-2,1.1863139E-4,-7.760911E-4,2.3120092E-3,6.2365475E-4,-1.8571378E-3,1.8614218E-2,-2.4477644E-2,-6.4156264E-2,6.0501194E-4,-6.774353E-4,3.5307242E-3,-3.5262987E-2,1.3756949E-3,2.1316817E-4,1.2334094E-3,-3.4341708E-4,1.9994633E-4,-3.439832E-2,-7.077703E-2,-8.029377E-4,1.6212905E-2,-5.856683E-4,-5.836048E-4,-1.8163888E-3,-1.3922079E-3,-5.4275605E-4,-7.768519E-2,-1.0971715E-3,-2.0179813E-4,1.2859107E-3,-1.2378843E-3,-2.8193905E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,-1,35,-1,-1,-1,37,39,41,-1,43,45,47,-1,-1,49,51,53,-1,55,-1,-1,57,-1,-1,-1,59,61,-1,63,65,67,69,-1,-1,71,-1,-1,-1,73,75,-1,-1,-1,-1,77,-1,-1,-1,79,81,-1,-1,83,85,87,-1,89,-1,91,93,-1,-1,95,97,-1,-1,99,101,-1,-1,-1,-1,-1,103,105,107,-1,-1,109,111,-1,-1,-1,-1,-1,113,115,-1,117,-1,-1,-1,-1,-1,119,-1,-1,-1,-1,-1],"loss_changes":[8.91799E-1,1.2983254E-1,3.9839417E-1,6.6396326E-2,6.5033555E-2,5.4354244E-1,0E0,4.8309952E-2,3.6854163E-2,6.8973996E-2,1.7517485E-2,1.5318069E-1,1.6096711E-1,1.355122E-2,0E0,7.023409E-3,8.75387E-3,3.5681523E-2,0E0,0E0,1.1058088E-2,0E0,6.4978376E-2,0E0,0E0,0E0,6.7731105E-3,7.3082596E-3,5.0564967E-3,0E0,8.97693E-4,3.1803466E-2,5.1747236E-2,0E0,0E0,4.472295E-2,6.646553E-2,1.8453435E-3,0E0,4.3964684E-3,0E0,0E0,1.1934228E-3,0E0,0E0,0E0,2.0911478E-2,5.4631017E-2,0E0,1.0202568E-2,9.078846E-3,1.0839164E-2,2.9875003E-2,0E0,0E0,1.6602725E-3,0E0,0E0,0E0,1.6317025E-2,1.2385675E-2,0E0,0E0,0E0,0E0,5.819028E-3,0E0,0E0,0E0,2.1861069E-2,1.6178548E-2,0E0,0E0,1.2440447E-2,9.095222E-4,8.962433E-3,0E0,1.9622631E-3,0E0,3.941752E-3,1.5477431E-2,0E0,0E0,1.1358835E-2,4.848599E-3,0E0,0E0,8.880043E-3,1.9125561E-3,0E0,0E0,0E0,0E0,0E0,5.024639E-3,5.389463E-3,2.647899E-3,0E0,0E0,4.4278745E-3,2.6031453E-3,0E0,0E0,0E0,0E0,0E0,1.2889588E-3,9.006858E-4,0E0,5.586772E-3,0E0,0E0,0E0,0E0,0E0,9.0511143E-4,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,20,20,22,22,26,26,27,27,28,28,30,30,31,31,32,32,35,35,36,36,37,37,39,39,42,42,46,46,47,47,49,49,50,50,51,51,52,52,55,55,59,59,60,60,65,65,69,69,70,70,73,73,74,74,75,75,77,77,79,79,80,80,83,83,84,84,87,87,88,88,94,94,95,95,96,96,99,99,100,100,106,106,107,107,109,109,115,115],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,-1,36,-1,-1,-1,38,40,42,-1,44,46,48,-1,-1,50,52,54,-1,56,-1,-1,58,-1,-1,-1,60,62,-1,64,66,68,70,-1,-1,72,-1,-1,-1,74,76,-1,-1,-1,-1,78,-1,-1,-1,80,82,-1,-1,84,86,88,-1,90,-1,92,94,-1,-1,96,98,-1,-1,100,102,-1,-1,-1,-1,-1,104,106,108,-1,-1,110,112,-1,-1,-1,-1,-1,114,116,-1,118,-1,-1,-1,-1,-1,120,-1,-1,-1,-1,-1],"split_conditions":[1.0773E4,4.014E3,5.11E2,3.4713E4,2.402988E6,1.1741724E4,-7.362617E-3,1.685E3,1.099164E6,9.9838E4,4.667E3,1.1059E4,6.0793E4,1.402704E6,-7.204919E-3,3.95265E5,8.3E1,1.83E2,-4.5617083E-3,-4.6253015E-4,4.365884E6,8.302099E-3,3.9051435E0,1.5709443E-2,5.5800434E-3,-3.6174979E-3,5.5990905E-2,9.782E3,1.9452692E7,1.1601892E-3,3.412E3,3.7E1,1.29E2,9.114481E-4,3.3257313E-3,3.1884794E8,2.8125507E-1,4.16E2,-2.722824E-3,3.8909322E-1,-8.5831806E-4,-2.5077206E-3,7.0319E4,-9.2479185E-4,-1.3407443E-4,1.6337543E-3,1.618921E6,5.96131E6,3.4781592E-3,2.0259231E9,6.28354E5,2.5529967E10,4.7016313E3,-1.249028E-3,-1.3935294E-4,4.7484E4,-1.0198557E-3,-1.1616723E-3,-3.1006226E-4,2.215E3,2.5459167E10,-2.6281944E-3,1.7241117E-3,3.7728755E-3,8.786497E-4,6.726593E6,-1.3777366E-3,2.394584E-3,5.6817057E-3,3.2842822E3,1.275083E9,-1.3534668E-3,-3.0950103E-3,9.3E1,7.11E2,9.1697E4,-1.481036E-3,1.3366E4,8.046308E-4,9E1,1.6647108E10,3.562356E-3,1.0301722E-3,1.1E2,5.4642496E7,-1.3972792E-3,-3.1944977E-3,1.207E3,1.0456E5,1.1863139E-4,-7.760911E-4,2.3120092E-3,6.2365475E-4,-1.8571378E-3,3.134E3,4.162739E6,8.5194E4,6.0501194E-4,-6.774353E-4,8.374E3,1.482686E3,1.3756949E-3,2.1316817E-4,1.2334094E-3,-3.4341708E-4,1.9994633E-4,3.0178302E7,1.234965E6,-8.029377E-4,5.3163274E8,-5.856683E-4,-5.836048E-4,-1.8163888E-3,-1.3922079E-3,-5.4275605E-4,4.296264E8,-1.0971715E-3,-2.0179813E-4,1.2859107E-3,-1.2378843E-3,-2.8193905E-3],"split_indices":[3,3,4,18,6,5,0,0,2,28,3,3,8,23,0,2,4,47,0,0,6,0,60,0,0,0,64,28,11,0,3,20,17,0,0,11,63,0,0,63,0,0,8,0,0,0,9,21,0,16,28,23,5,0,0,6,0,0,0,19,10,0,0,0,0,2,0,0,0,5,16,0,0,19,0,8,0,3,0,19,1,0,0,4,21,0,0,0,8,0,0,0,0,0,0,27,8,0,0,3,5,0,0,0,0,0,27,9,0,16,0,0,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.98E2,8.4E1,7.6E1,1.22E2,8.1E1,3E0,2.4E1,5.2E1,1.1E2,1.2E1,7.2E1,9E0,2.1E1,3E0,4.2E1,1E1,1.06E2,4E0,3E0,9E0,3E0,6.9E1,3E0,6E0,1.2E1,9E0,2.4E1,1.8E1,4E0,6E0,9.2E1,1.4E1,5E0,4E0,2.6E1,4.3E1,6E0,3E0,2E1,4E0,3E0,1.5E1,3E0,3E0,4E0,8.8E1,1E1,4E0,6E0,2E1,1E1,3.3E1,3E0,3E0,1.7E1,3E0,1.2E1,3E0,5.1E1,3.7E1,5E0,5E0,3E0,3E0,1.5E1,5E0,4E0,6E0,1.9E1,1.4E1,3E0,1.4E1,4.3E1,8E0,2.9E1,8E0,9E0,6E0,9E0,1E1,9E0,5E0,3.2E1,1.1E1,3E0,5E0,2.3E1,6E0,4E0,5E0,6E0,3E0,3E0,7E0,1.6E1,1.6E1,4E0,7E0,1.4E1,9E0,3E0,3E0,4E0,3E0,4E0,1.2E1,1.3E1,3E0,9E0,5E0,6E0,3E0,7E0,5E0,1E1,3E0,5E0,4E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[7.095178E-3,-1.9136276E-2,1.11011535E-1,-2.3116494E-2,6.9368924E-3,3.1234738E-1,8.232545E-2,-5.219021E-2,2.835958E-3,1.4188325E-2,3.406416E-3,5.461373E-3,1.2169951E-1,-1.3626613E-1,-4.3260716E-2,9.420919E-3,-6.290427E-3,2.7532836E-2,-1.9907355E-3,6.0850333E-2,1.4042228E-1,-1.4067753E-3,-5.4622213E-3,3.167945E-3,-4.8657116E-2,-1.1470629E-2,7.2477914E-2,-5.656316E-4,4.0273454E-2,2.6164763E-3,3.1793886E-4,1.8403372E-1,8.98958E-2,2.7352313E-2,-5.7708226E-2,-1.4240517E-1,-5.5283605E-4,1.1591982E-1,3.2438826E-2,5.8419123E-2,-3.4193278E-4,2.1107312E-1,2.266965E-3,4.3961467E-4,1.3073304E-1,-3.1837162E-3,2.6261779E-3,1.2136318E-3,-6.156463E-2,8.399425E-4,-8.861722E-3,-1.3387262E-2,8.700487E-2,4.970088E-2,5.1300004E-3,5.2316632E-2,1.1384321E-4,2.24638E-3,8.6144434E-4,3.7057872E-3,8.2355505E-3,5.360814E-3,1.42714E-3,9.0952485E-4,-1.3418671E-3,-5.3526852E-2,-6.7255134E-3,-1.7742775E-2,1.0126042E-3,1.444374E-1,-1.5969395E-3,6.8704446E-4,2.1126396E-3,6.3863E-4,2.274505E-3,-9.1249645E-2,-4.3570917E-2,-2.3321152E-2,1.0697947E-2,6.550937E-3,2.1981136E-3,-1.8853988E-3,-3.3797906E-3,1.5797343E-3,-4.8761718E-2,8.3852075E-3,-2.7837422E-2,3.3923406E-2,-9.206808E-4,-6.1619785E-2,-2.9335415E-2,7.69977E-4,-4.2264903E-4,-8.611225E-3,-3.487946E-2,1.6795747E-3,4.034749E-4,-3.925111E-2,-7.2100684E-2,-5.8122758E-2,-1.1712703E-2,5.361544E-3,-8.1586256E-4,-4.896992E-2,-2.7847074E-2,-1.8019333E-3,-4.6755245E-4,-1.160799E-3,-2.6430397E-3,-2.6681865E-4,-2.5202623E-3,-3.289378E-2,1.6189854E-3,6.5209024E-4,-1.7585306E-4,-4.269421E-4,-1.9539287E-3,-3.2569386E-2,-1.4783626E-2,-1.6503673E-3,-2.0256173E-4,-2.1378126E-4,4.671223E-4,-1.2651407E-3,-3.678581E-4,-8.3786575E-4,-1.8699458E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,27,-1,29,31,-1,-1,-1,33,35,37,-1,39,-1,-1,41,43,45,47,49,51,53,55,57,-1,59,-1,-1,61,63,-1,-1,65,-1,-1,67,69,71,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,75,-1,77,-1,79,-1,-1,-1,-1,-1,81,83,85,87,-1,-1,-1,-1,-1,89,91,93,95,-1,97,99,-1,-1,101,103,-1,-1,105,107,109,111,113,-1,115,117,-1,-1,-1,-1,-1,-1,119,121,-1,-1,-1,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9808056E-1,2.2352007E-1,3.1504244E-1,1.7444542E-1,0E0,1.3902628E-1,1.5973729E-1,7.593739E-2,1.6242003E-1,0E0,0E0,2.9928349E-2,3.1329572E-2,2.3523971E-2,8.1243694E-2,1.5891436E-1,0E0,9.32769E-3,0E0,9.613253E-3,4.074794E-2,0E0,0E0,0E0,6.831154E-2,1.3000724E-1,4.842095E-2,0E0,1.17031E-2,0E0,0E0,2.5068998E-2,4.1801713E-2,1.8476743E-2,3.4332454E-2,1.639363E-1,9.668374E-2,3.2361567E-2,9.58165E-3,1.3712812E-3,0E0,1.3507277E-2,0E0,0E0,2.0460144E-2,1.0862149E-2,0E0,0E0,8.831912E-2,0E0,0E0,1.5207506E-2,9.3474135E-2,1.7615072E-3,0E0,4.555721E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6444986E-2,0E0,1.1196552E-2,0E0,2.2092313E-2,0E0,0E0,0E0,0E0,0E0,1.0382086E-3,3.4133554E-2,8.538697E-3,1.1923509E-2,0E0,0E0,0E0,0E0,0E0,1.3928786E-2,2.988079E-3,6.604623E-3,2.4696207E-3,0E0,5.8443844E-3,1.30166225E-2,0E0,0E0,3.6433688E-3,2.3026466E-3,0E0,0E0,4.9777385E-3,5.6963935E-3,9.403981E-3,5.1360866E-3,1.5619386E-3,0E0,3.8028099E-3,1.1841282E-3,0E0,0E0,0E0,0E0,0E0,0E0,3.0993493E-3,1.3743904E-3,0E0,0E0,0E0,0E0,2.2558495E-3,1.4223871E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,24,24,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,44,44,45,45,48,48,51,51,52,52,53,53,55,55,65,65,67,67,69,69,75,75,76,76,77,77,78,78,84,84,85,85,86,86,87,87,89,89,90,90,93,93,94,94,97,97,98,98,99,99,100,100,101,101,103,103,104,104,111,111,112,112,117,117,118,118],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,28,-1,30,32,-1,-1,-1,34,36,38,-1,40,-1,-1,42,44,46,48,50,52,54,56,58,-1,60,-1,-1,62,64,-1,-1,66,-1,-1,68,70,72,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,76,-1,78,-1,80,-1,-1,-1,-1,-1,82,84,86,88,-1,-1,-1,-1,-1,90,92,94,96,-1,98,100,-1,-1,102,104,-1,-1,106,108,110,112,114,-1,116,118,-1,-1,-1,-1,-1,-1,120,122,-1,-1,-1,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397685E9,7.14535E5,1.571E4,5.013E3,6.9368924E-3,2.844E3,6.64063E5,9.829E3,2.51383E5,1.4188325E-2,3.406416E-3,2.0397697E10,4.3E1,2E1,2.302E3,1.1822208E10,-6.290427E-3,1.6676827E-3,-1.9907355E-3,3.2E1,3.666791E-1,-1.4067753E-3,-5.4622213E-3,3.167945E-3,9.532393E-6,1.5262E4,1.08E2,-5.656316E-4,8.0532E4,2.6164763E-3,3.1793886E-4,1E0,1.05055E5,1.04E2,7E0,5.888E3,1.83E2,1.2158327E0,1.847262E0,1.16E2,-3.4193278E-4,3.060244E7,2.266965E-3,4.3961467E-4,6.635E3,3.969E3,2.6261779E-3,1.2136318E-3,2.16292E5,8.399425E-4,-8.861722E-3,2.29221E6,1.7257E4,6.3E1,5.1300004E-3,5.419321E-1,1.1384321E-4,2.24638E-3,8.6144434E-4,3.7057872E-3,8.2355505E-3,5.360814E-3,1.42714E-3,9.0952485E-4,-1.3418671E-3,1.055E3,-6.7255134E-3,4.838464E7,1.0126042E-3,7.091E3,-1.5969395E-3,6.8704446E-4,2.1126396E-3,6.3863E-4,2.274505E-3,7.45575E2,1.2361E4,5.236E3,7.571E4,6.550937E-3,2.1981136E-3,-1.8853988E-3,-3.3797906E-3,1.5797343E-3,3.4521626E8,6.6E2,7.8E1,6.259021E7,-9.206808E-4,1.2482E4,1E0,7.69977E-4,-4.2264903E-4,8.74E2,1.7140013E3,1.6795747E-3,4.034749E-4,1.0204246E8,8E0,8.808534E8,3.28451E5,1.3818E4,-8.1586256E-4,9.675527E7,1.3141365E7,-1.8019333E-3,-4.6755245E-4,-1.160799E-3,-2.6430397E-3,-2.6681865E-4,-2.5202623E-3,6.1E1,3.068292E6,6.5209024E-4,-1.7585306E-4,-4.269421E-4,-1.9539287E-3,1E0,1.48E2,-1.6503673E-3,-2.0256173E-4,-2.1378126E-4,4.671223E-4,-1.2651407E-3,-3.678581E-4,-8.3786575E-4,-1.8699458E-5],"split_indices":[16,8,8,3,0,0,12,18,3,0,0,10,4,17,7,25,0,63,0,4,63,0,0,0,64,8,19,0,8,0,0,36,3,17,4,3,47,64,64,4,0,2,0,0,0,3,0,0,22,0,0,6,7,19,0,63,0,0,0,0,0,0,0,0,0,20,0,18,0,7,0,0,0,0,0,5,12,3,8,0,0,0,0,0,11,17,4,18,0,7,42,0,0,0,5,0,0,16,19,11,6,7,0,21,18,0,0,0,0,0,0,17,2,0,0,0,0,36,4,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,2.33E2,5.8E1,2.3E2,3E0,6E0,5.2E1,1.08E2,1.22E2,3E0,3E0,1.8E1,3.4E1,9E0,9.9E1,1.19E2,3E0,1.4E1,4E0,9E0,2.5E1,3E0,6E0,3E0,9.6E1,9E1,2.9E1,3E0,1.1E1,6E0,3E0,1.2E1,1.3E1,1E1,8.6E1,6E0,8.4E1,1.3E1,1.6E1,8E0,3E0,9E0,3E0,5E0,8E0,7E0,3E0,3E0,8.3E1,3E0,3E0,7.4E1,1E1,6E0,7E0,9E0,7E0,5E0,3E0,4E0,5E0,5E0,3E0,4E0,3E0,8E1,3E0,6.8E1,6E0,7E0,3E0,3E0,3E0,4E0,5E0,1.5E1,6.5E1,5.7E1,1.1E1,3E0,4E0,6E0,9E0,3E0,6.2E1,7E0,5E1,7E0,4E0,3.6E1,2.6E1,4E0,3E0,1.4E1,3.6E1,3E0,4E0,1.3E1,2.3E1,9E0,1.7E1,8E0,6E0,1E1,2.6E1,7E0,6E0,6E0,1.7E1,3E0,6E0,6E0,1.1E1,3E0,5E0,3E0,7E0,1.8E1,8E0,3E0,3E0,7E0,4E0,1.3E1,5E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[1.008847E-2,-3.1482698E-3,2.2951874E-1,-3.385226E-2,4.6550814E-2,1.1962709E-2,4.221023E-3,-3.919306E-2,7.586011E-2,3.851696E-3,6.691631E-2,-3.530996E-2,-5.7896688E-3,4.7688535E-3,9.188993E-5,7.563974E-2,-3.3770397E-2,5.8276593E-3,5.3173147E-2,-3.1314775E-2,-5.739273E-3,6.8035675E-3,6.913698E-3,-3.8045058E-3,-1.5197383E-2,-3.6177527E-2,6.498801E-2,-6.0898542E-2,-1.6649695E-2,7.042029E-4,-3.0362236E-4,6.6561793E-4,-2.5458746E-2,-2.7036672E-3,2.9970668E-4,7.202813E-2,-1.4337141E-3,-6.6822484E-2,1.8609731E-4,2.2662838E-3,-2.1809718E-2,-3.611296E-2,-5.4170716E-5,4.439917E-3,6.068001E-2,-7.077224E-2,-2.4646998E-4,-3.597439E-2,8.529117E-3,-6.130663E-4,-1.5180712E-3,2.210667E-2,7.807043E-2,-6.2282108E-2,-4.4414476E-3,-5.1037423E-2,-6.859896E-3,3.4637507E-2,-2.3369558E-2,-1.1551164E-3,3.8249895E-2,1.15727566E-1,4.6214093E-2,-7.366142E-2,-2.0778047E-2,-2.0340524E-4,-5.5351276E-2,1.4310743E-2,-3.7929863E-2,1.570058E-2,2.0006273E-3,-3.9774608E-2,5.599418E-4,2.0256669E-4,1.8488145E-3,1.4860061E-1,1.5170678E-3,6.0399674E-2,1.3267432E-2,-7.969668E-2,-8.4781635E-4,4.8125765E-4,-1.2304669E-3,-7.94268E-2,-4.3480325E-2,-5.230659E-4,4.3082498E-2,-1.5854863E-4,-1.9027476E-3,-1.6358953E-4,2.6727222E-2,-4.6123852E-4,-1.682449E-3,2.2276388E-3,5.6898557E-3,6.8475964E-4,7.001033E-2,-3.2925626E-4,1.0766197E-3,-2.6893865E-3,-1.1798141E-3,-3.2735409E-3,-1.2942014E-3,-5.5003088E-2,-2.2572817E-2,1.6056631E-4,2.0903961E-3,9.7770106E-5,1.2982982E-3,2.684706E-3,9.3566364E-4,-2.2658627E-3,-3.5652272E-2,1.8164795E-4,-1.4012905E-3,-5.052328E-4,-1.4916554E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,-1,21,23,-1,25,27,-1,-1,29,-1,31,33,35,37,39,-1,-1,-1,41,-1,-1,43,-1,45,-1,-1,47,49,-1,-1,51,53,-1,55,57,-1,-1,59,61,63,-1,65,67,69,71,-1,73,75,77,79,81,-1,83,85,87,89,-1,91,-1,-1,-1,93,-1,95,97,99,-1,-1,-1,101,103,-1,105,-1,-1,-1,107,-1,-1,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1,115,-1,-1,-1,-1],"loss_changes":[8.168987E-1,4.0748125E-1,1.7499024E-1,9.9655256E-2,8.84403E-2,0E0,0E0,7.995553E-2,4.179857E-2,9.4382145E-2,9.8599255E-2,8.29633E-2,0E0,0E0,0E0,1.1047169E-1,3.4267314E-2,0E0,6.885806E-2,6.4931005E-2,0E0,0E0,2.4077168E-3,0E0,7.889148E-3,1.8182665E-2,4.6389416E-2,2.0136714E-2,4.7803886E-2,0E0,0E0,0E0,3.9922334E-3,0E0,0E0,3.1268716E-2,0E0,1.0207817E-2,0E0,0E0,4.242131E-2,1.1759568E-3,0E0,0E0,2.9802024E-2,1.594235E-2,0E0,2.9032916E-2,2.74598E-2,0E0,0E0,1.6351506E-2,3.303954E-2,1.736097E-2,0E0,8.138202E-3,1.632931E-2,8.403091E-3,1.0723181E-2,0E0,7.6093283E-3,2.4566576E-2,8.078631E-3,6.754622E-3,6.8056416E-3,0E0,7.978469E-3,1.3889313E-2,6.699794E-3,2.8857987E-3,0E0,2.747925E-3,0E0,0E0,0E0,7.89161E-3,0E0,3.399074E-3,3.6611157E-3,1.4159232E-3,0E0,0E0,0E0,6.3927695E-3,5.744882E-3,0E0,6.3376464E-3,0E0,0E0,0E0,2.456165E-3,0E0,0E0,0E0,0E0,0E0,3.05954E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.850715E-3,7.342705E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4056517E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,15,15,16,16,18,18,19,19,22,22,24,24,25,25,26,26,27,27,28,28,32,32,35,35,37,37,40,40,41,41,44,44,45,45,47,47,48,48,51,51,52,52,53,53,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,69,69,71,71,75,75,77,77,78,78,79,79,83,83,84,84,86,86,90,90,96,96,103,103,104,104,112,112],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,-1,22,24,-1,26,28,-1,-1,30,-1,32,34,36,38,40,-1,-1,-1,42,-1,-1,44,-1,46,-1,-1,48,50,-1,-1,52,54,-1,56,58,-1,-1,60,62,64,-1,66,68,70,72,-1,74,76,78,80,82,-1,84,86,88,90,-1,92,-1,-1,-1,94,-1,96,98,100,-1,-1,-1,102,104,-1,106,-1,-1,-1,108,-1,-1,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1,116,-1,-1,-1,-1],"split_conditions":[2.51383E5,7.9E3,4.0962806E0,5.191053E8,3.61273E5,1.1962709E-2,4.221023E-3,8.919402E9,2.1149E4,1.1326E4,1.1882E4,1.24389E5,-5.7896688E-3,4.7688535E-3,9.188993E-5,1.44E2,1.01911224E3,5.8276593E-3,5.710256E-2,3.558E3,-5.739273E-3,6.8035675E-3,1.1541341E9,-3.8045058E-3,8.922E3,2.486228E3,7.19833E6,2.761E3,1.1799E4,7.042029E-4,-3.0362236E-4,6.6561793E-4,3.25559E5,-2.7036672E-3,2.9970668E-4,1.9091807E-1,-1.4337141E-3,7.8581E4,1.8609731E-4,2.2662838E-3,1.595E3,9.63E2,-5.4170716E-5,4.439917E-3,2.029183E0,1.050246E6,-2.4646998E-4,3.1361875E8,7.51885E5,-6.130663E-4,-1.5180712E-3,1.306E4,1.69E2,2.2296707E9,-4.4414476E-3,1.5E1,2.0315E5,4.69E2,1.9452E4,-1.1551164E-3,3.43E2,7.3100853E0,4.90364E5,4.7E2,4E1,-2.0340524E-4,1.17E2,6.490568E8,7.0345754E8,8.2E1,2.0006273E-3,1.62E2,5.599418E-4,2.0256669E-4,1.8488145E-3,1.600673E7,1.5170678E-3,3.054E3,3.71E2,5.637464E-1,-8.4781635E-4,4.8125765E-4,-1.2304669E-3,1.064E3,9.547999E8,-5.230659E-4,7.143E3,-1.5854863E-4,-1.9027476E-3,-1.6358953E-4,4.058E3,-4.6123852E-4,-1.682449E-3,2.2276388E-3,5.6898557E-3,6.8475964E-4,1.02490554E11,-3.2925626E-4,1.0766197E-3,-2.6893865E-3,-1.1798141E-3,-3.2735409E-3,-1.2942014E-3,5.013E3,1.6507992E9,1.6056631E-4,2.0903961E-3,9.7770106E-5,1.2982982E-3,2.684706E-3,9.3566364E-4,-2.2658627E-3,6.507E3,1.8164795E-4,-1.4012905E-3,-5.052328E-4,-1.4916554E-3],"split_indices":[3,3,60,27,12,0,0,25,8,28,3,8,0,0,0,17,5,0,63,3,0,0,10,0,3,5,12,0,8,0,0,0,12,0,0,63,0,22,0,0,0,17,0,0,67,2,0,16,18,0,0,7,19,10,0,17,12,19,7,0,17,60,18,4,4,0,17,16,16,4,0,17,0,0,0,2,0,0,17,63,0,0,0,0,11,0,28,0,0,0,3,0,0,0,0,0,1,0,0,0,0,0,0,3,11,0,0,0,0,0,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.65E2,1.5E1,1.64E2,1.01E2,5E0,1E1,1.57E2,7E0,3.3E1,6.8E1,1.54E2,3E0,3E0,4E0,1.1E1,2.2E1,6E0,6.2E1,1.51E2,3E0,3E0,8E0,3E0,1.9E1,7E0,5.5E1,4.9E1,1.02E2,4E0,4E0,4E0,1.5E1,3E0,4E0,5.2E1,3E0,4.5E1,4E0,5E0,9.7E1,1E1,5E0,6E0,4.6E1,4.2E1,3E0,6.6E1,3.1E1,5E0,5E0,1.5E1,3.1E1,3.9E1,3E0,4.3E1,2.3E1,1.7E1,1.4E1,3E0,1.2E1,1.3E1,1.8E1,3E1,9E0,4E0,3.9E1,1.4E1,9E0,1.1E1,6E0,1E1,4E0,5E0,7E0,8E0,5E0,1.2E1,6E0,2.6E1,4E0,3E0,6E0,1.1E1,2.8E1,7E0,7E0,4E0,5E0,4E0,7E0,4E0,6E0,3E0,5E0,3E0,9E0,3E0,3E0,2.3E1,3E0,6E0,5E0,1.7E1,1.1E1,3E0,4E0,3E0,4E0,6E0,3E0,8E0,9E0,5E0,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[3.7905863E-3,-2.3246888E-2,1.0490193E-1,-3.0722326E-2,8.6945914E-2,1.0139254E-2,8.49821E-2,-2.5486829E-2,-9.799917E-3,3.5862576E-2,5.0727814E-3,4.3594606E-2,1.4606765E-1,-4.811538E-2,-1.3224878E-3,7.3751544E-5,1.6069692E-3,1.8250834E-2,8.21169E-2,2.178692E-1,1.0103764E-1,-4.1209552E-2,-5.217802E-3,5.328016E-3,-1.1319997E-2,3.3640757E-2,-2.0363212E-3,1.12958394E-1,1.2909493E-4,8.685434E-3,3.169173E-3,-7.6570825E-5,1.2508595E-1,-5.015327E-2,1.2233449E-2,-8.184252E-3,-1.1852958E-3,1.6447922E-2,5.7374015E-2,4.232945E-3,1.6823795E-3,1.8243041E-3,4.504581E-3,1.2333209E-3,-5.466118E-2,2.2563709E-2,-7.853546E-4,-1.572027E-2,5.9311982E-2,1.6081206E-3,1.3987928E-3,2.7404313E-3,7.6255057E-4,-3.521319E-2,-7.286688E-2,1.3627837E-3,9.071474E-3,6.042916E-2,-2.5151603E-2,5.4799477E-3,2.787221E-2,5.569568E-4,-3.8908786E-4,3.503359E-2,-4.9449977E-2,-1.03871495E-1,-5.3248227E-2,-2.1951002E-4,9.86268E-4,1.9868421E-5,2.9044822E-3,-1.95897E-2,-3.04671E-3,-1.0811947E-3,5.115836E-2,2.36271E-3,-8.58561E-5,-5.8350615E-2,-9.891331E-3,-3.807792E-3,-9.987378E-4,-2.5852318E-3,-3.7052806E-2,-1.46146715E-2,-5.0027143E-2,1.8263355E-2,2.6076108E-3,-6.354226E-2,-2.80666E-4,-1.096494E-3,7.338342E-4,-4.2996217E-2,-5.156292E-4,-1.8996453E-2,8.7737927E-4,-2.1513079E-3,-3.9626186E-4,-8.782555E-5,1.1166127E-3,-2.429968E-3,-1.2650226E-3,-2.430276E-2,-1.9599323E-3,-1.2997391E-2,-4.409259E-2,-1.1802411E-3,-2.278639E-4,-3.036083E-2,-3.3036438E-3,-4.5458367E-4,-1.7678017E-3,-4.0381015E-5,-1.1903855E-3,-6.2258507E-4,3.842844E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,17,19,21,23,-1,-1,25,27,29,31,33,-1,-1,35,37,-1,39,-1,-1,-1,-1,41,43,45,-1,47,49,51,-1,-1,-1,-1,-1,53,55,-1,57,59,-1,61,-1,-1,63,65,-1,67,69,71,-1,73,-1,-1,75,77,79,81,-1,-1,-1,-1,83,-1,-1,85,-1,-1,87,89,-1,-1,-1,91,93,95,97,-1,99,-1,-1,-1,101,-1,103,-1,-1,-1,-1,-1,-1,-1,105,-1,107,109,-1,-1,111,113,-1,-1,-1,-1,-1,-1],"loss_changes":[8.0344075E-1,1.937787E-1,2.3269844E-1,3.0465594E-1,4.848223E-2,0E0,1.4026654E-1,1.1749923E-1,0E0,4.6058893E-3,0E0,3.3648103E-2,5.2351177E-2,8.069384E-2,1.7701794E-1,0E0,0E0,3.098093E-2,3.327182E-2,2.306956E-2,4.00894E-2,5.127932E-2,0E0,0E0,2.4590753E-1,7.217156E-3,0E0,4.3821037E-3,0E0,0E0,0E0,0E0,5.2807927E-3,3.8385794E-2,6.605157E-3,0E0,8.622706E-2,6.4861346E-3,5.4642055E-3,0E0,0E0,0E0,0E0,0E0,2.823773E-2,3.0841557E-3,0E0,5.8256954E-2,6.152027E-2,0E0,2.3540189E-3,0E0,0E0,4.57363E-2,2.1232843E-2,0E0,3.2936276E-3,1.598752E-2,2.5590707E-2,0E0,2.4617478E-2,0E0,0E0,1.17677245E-2,1.2450434E-2,1.3140261E-2,9.736158E-3,0E0,0E0,0E0,0E0,9.414488E-3,0E0,0E0,1.0303926E-2,0E0,0E0,7.1759894E-3,7.0981123E-3,0E0,0E0,0E0,1.8115211E-3,1.1342706E-2,4.882233E-3,2.5083192E-3,0E0,4.8540235E-3,0E0,0E0,0E0,3.5372283E-3,0E0,7.5567905E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6569267E-3,0E0,7.4739545E-3,2.4692602E-3,0E0,0E0,3.0367123E-3,7.564427E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,24,24,25,25,27,27,32,32,33,33,34,34,36,36,37,37,38,38,44,44,45,45,47,47,48,48,50,50,53,53,54,54,56,56,57,57,58,58,60,60,63,63,64,64,65,65,66,66,71,71,74,74,77,77,78,78,82,82,83,83,84,84,85,85,87,87,91,91,93,93,101,101,103,103,104,104,107,107,108,108],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,18,20,22,24,-1,-1,26,28,30,32,34,-1,-1,36,38,-1,40,-1,-1,-1,-1,42,44,46,-1,48,50,52,-1,-1,-1,-1,-1,54,56,-1,58,60,-1,62,-1,-1,64,66,-1,68,70,72,-1,74,-1,-1,76,78,80,82,-1,-1,-1,-1,84,-1,-1,86,-1,-1,88,90,-1,-1,-1,92,94,96,98,-1,100,-1,-1,-1,102,-1,104,-1,-1,-1,-1,-1,-1,-1,106,-1,108,110,-1,-1,112,114,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0737418E9,3.97056E5,1.571E4,2.93968E5,4.566725E3,1.0139254E-2,3.7648073E11,4.995E3,-9.799917E-3,7.2000364E9,5.0727814E-3,1.83E2,3.954342E-1,6.9052E4,3.719E3,7.3751544E-5,1.6069692E-3,1E0,1.960921E11,7.14485E5,2.2491941E0,2.431E3,-5.217802E-3,5.328016E-3,1.5262E4,2.236864E6,-2.0363212E-3,3.5464888E9,1.2909493E-4,8.685434E-3,3.169173E-3,-7.6570825E-5,1.15540435E11,1E0,1E0,-8.184252E-3,8.1188E4,2.231867E9,8.470816E10,4.232945E-3,1.6823795E-3,1.8243041E-3,4.504581E-3,1.2333209E-3,1.17E2,3.4311706E9,-7.853546E-4,6.2222845E2,1.0618027E9,1.6081206E-3,2.5642102E3,2.7404313E-3,7.6255057E-4,5.806469E7,2.63E2,1.3627837E-3,5.601505E6,3.8208E4,7.0290915E10,5.4799477E-3,1.3334E4,5.569568E-4,-3.8908786E-4,3.649951E9,3.4521626E8,1.4544991E7,2.75E2,-2.1951002E-4,9.86268E-4,1.9868421E-5,2.9044822E-3,2.147E3,-3.04671E-3,-1.0811947E-3,1.21205E5,2.36271E-3,-8.58561E-5,5.581883E7,3.7E1,-3.807792E-3,-9.987378E-4,-2.5852318E-3,1.5571406E3,8.944641E7,2.194E3,8.4551E4,2.6076108E-3,7.35E2,-2.80666E-4,-1.096494E-3,7.338342E-4,1.1524414E3,-5.156292E-4,8.055685E8,8.7737927E-4,-2.1513079E-3,-3.9626186E-4,-8.782555E-5,1.1166127E-3,-2.429968E-3,-1.2650226E-3,2.9922E4,-1.9599323E-3,4.6585E5,8.54E2,-1.1802411E-3,-2.278639E-4,5.8016E4,4.07729E5,-4.5458367E-4,-1.7678017E-3,-4.0381015E-5,-1.1903855E-3,-6.2258507E-4,3.842844E-4],"split_indices":[16,22,8,28,5,0,13,3,0,11,0,47,63,7,7,0,0,36,13,18,59,0,0,0,8,6,0,21,0,0,0,0,1,38,31,0,28,10,13,0,0,0,0,0,4,10,0,5,21,0,5,0,0,16,4,0,2,22,10,0,7,0,0,10,11,27,0,0,0,0,0,19,0,0,18,0,0,18,19,0,0,0,5,18,0,20,0,17,0,0,0,5,0,16,0,0,0,0,0,0,0,12,0,6,0,0,0,12,18,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.92E2,2.31E2,6.1E1,2.17E2,1.4E1,4E0,5.7E1,2.14E2,3E0,9E0,5E0,3.5E1,2.2E1,1.1E2,1.04E2,3E0,6E0,2.2E1,1.3E1,7E0,1.5E1,1.05E2,5E0,5E0,9.9E1,1.9E1,3E0,9E0,4E0,4E0,3E0,3E0,1.2E1,9E1,1.5E1,3E0,9.6E1,1.2E1,7E0,6E0,3E0,3E0,9E0,4E0,8.6E1,1.2E1,3E0,7.8E1,1.8E1,3E0,9E0,3E0,4E0,4.3E1,4.3E1,4E0,8E0,8E0,7E1,3E0,1.5E1,4E0,5E0,7E0,3.6E1,1.5E1,2.8E1,5E0,3E0,3E0,5E0,6.6E1,4E0,4E0,1.1E1,3E0,4E0,2.9E1,7E0,1.2E1,3E0,9E0,1.9E1,5.8E1,8E0,6E0,5E0,2.6E1,3E0,4E0,3E0,1.4E1,5E0,5.3E1,5E0,5E0,3E0,3E0,3E0,1.6E1,1E1,8E0,6E0,4.4E1,9E0,4E0,4E0,1.5E1,2.9E1,3E0,6E0,3E0,1.2E1,1.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[8.2760425E-3,-2.4058279E-2,7.896032E-2,-1.8509429E-2,-9.243477E-3,9.113276E-2,-4.7059506E-3,1.3114952E-1,-2.5651477E-2,6.0006518E-2,2.780774E-1,-5.029849E-4,1.0253002E-2,-4.0622387E-2,1.7034993E-2,4.4049863E-2,1.2638552E-1,1.7102963E-1,1.4466217E-2,-9.902192E-2,-3.5286315E-2,5.3995475E-3,3.5594204E-3,8.680058E-2,6.2904265E-3,7.5809113E-3,5.9183348E-2,3.1524291E-3,7.658205E-3,-4.237838E-3,-1.9817636E-3,1.0319842E-3,-3.828463E-2,-1.21662915E-2,4.0372927E-2,1.28949E-1,4.4495687E-2,-3.163585E-2,4.607764E-2,2.100281E-4,3.0000606E-3,-6.912381E-2,-3.1625334E-2,-2.5151158E-2,9.745131E-3,5.5159938E-2,-3.3426913E-4,6.632059E-2,1.6763023E-1,-2.0856477E-4,1.996595E-3,8.060263E-3,-9.516693E-2,2.501068E-3,2.176572E-2,-8.223232E-2,-3.4539166E-4,6.8644485E-3,-3.583783E-2,-1.9035971E-3,-1.3675489E-2,1.0314031E-3,-3.7248884E-3,2.3109813E-3,2.6077527E-4,3.1903488E-4,3.416849E-3,2.7749455E-3,6.413392E-3,1.9100375E-2,-6.0499326E-4,-3.9861766E-3,-1.374641E-3,1.4914104E-3,1.4741452E-3,-1.4558278E-3,-3.1643403E-3,1.3682522E-3,-1.1814823E-2,-7.941064E-2,-3.0802617E-2,3.161145E-4,-2.4283905E-2,-4.5733995E-4,2.4151945E-4,-2.2321798E-4,1.0710438E-3,3.8305655E-4,-2.990444E-4,-6.4608257E-4,4.6989495E-5,-1.1948731E-3,-3.037637E-3,-7.2030886E-3,-3.8929388E-2,-9.444195E-4,-2.2007697E-4,-3.0538145E-2,2.1029532E-2,-5.125463E-2,-3.0561006E-2,-4.3962874E-2,-8.562475E-6,3.457541E-2,-6.890374E-5,-3.658792E-2,-2.62813E-3,2.3592582E-4,-3.6183335E-2,-1.7163365E-3,-6.095133E-4,3.3952467E-4,1.6081316E-3,-2.1368654E-3,-2.0020949E-2,-4.6992753E-2,-2.336373E-2,-5.423291E-5,-8.160964E-4,-1.8194434E-3,-1.0036918E-3,-1.2141389E-3,-2.2680775E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,-1,33,35,37,-1,39,-1,-1,-1,-1,-1,41,43,45,47,49,51,53,-1,-1,55,57,59,61,63,-1,65,67,-1,-1,69,71,-1,73,75,-1,77,79,-1,81,-1,83,-1,-1,-1,-1,-1,-1,85,-1,-1,-1,87,-1,-1,-1,-1,89,91,93,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,97,99,-1,-1,101,103,105,107,109,-1,111,-1,113,-1,-1,115,-1,-1,-1,-1,-1,117,119,121,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5556073E-1,2.8229073E-1,2.599883E-1,2.1072525E-1,0E0,4.837376E-1,0E0,2.5249705E-1,1.1955628E-1,7.387179E-2,1.4323294E-1,0E0,0E0,3.9510027E-2,9.891741E-2,9.953706E-2,8.8731796E-2,2.0699799E-2,0E0,5.339578E-3,2.7073935E-2,0E0,2.7121443E-2,4.5971766E-2,5.276305E-2,0E0,1.7305344E-2,0E0,0E0,0E0,0E0,0E0,2.307412E-2,9.6341595E-3,1.1080541E-2,2.2798285E-2,1.4700726E-2,4.6161935E-2,1.1378236E-2,0E0,0E0,1.5192166E-2,1.6946234E-2,7.3817614E-3,4.0237885E-3,8.029003E-3,0E0,1.4675042E-2,3.4149438E-3,0E0,0E0,3.916821E-3,5.239956E-3,0E0,5.291004E-3,5.6650117E-3,0E0,7.825732E-3,1.7958008E-2,0E0,4.4758366E-3,0E0,1.165948E-3,0E0,0E0,0E0,0E0,0E0,0E0,3.5517463E-3,0E0,0E0,0E0,8.961304E-4,0E0,0E0,0E0,0E0,9.0614124E-4,2.1902435E-3,1.6021565E-2,0E0,8.5792923E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.579483E-2,4.9315766E-3,0E0,0E0,5.271445E-3,3.5264688E-3,7.7108294E-3,8.786488E-3,1.098156E-3,0E0,2.0604664E-3,0E0,7.5565744E-3,0E0,0E0,3.7521124E-3,0E0,0E0,0E0,0E0,0E0,1.2283539E-3,9.5013157E-4,5.8284663E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,23,23,24,24,26,26,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,45,45,47,47,48,48,51,51,52,52,54,54,55,55,57,57,58,58,60,60,62,62,69,69,73,73,78,78,79,79,80,80,82,82,93,93,94,94,97,97,98,98,99,99,100,100,101,101,103,103,105,105,108,108,114,114,115,115,116,116],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,-1,34,36,38,-1,40,-1,-1,-1,-1,-1,42,44,46,48,50,52,54,-1,-1,56,58,60,62,64,-1,66,68,-1,-1,70,72,-1,74,76,-1,78,80,-1,82,-1,84,-1,-1,-1,-1,-1,-1,86,-1,-1,-1,88,-1,-1,-1,-1,90,92,94,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,98,100,-1,-1,102,104,106,108,110,-1,112,-1,114,-1,-1,116,-1,-1,-1,-1,-1,118,120,122,-1,-1,-1,-1,-1,-1],"split_conditions":[4.320722E6,1.06056E5,6.25E2,9.366E3,-9.243477E-3,1.1741724E4,-4.7059506E-3,1.1458619E7,1.3925508E9,3.7648073E11,1.833321E7,-5.029849E-4,1.0253002E-2,2.38E2,1.0018222E3,2.0679289E10,4.8454278E11,6.879718E6,1.4466217E-2,2.719E3,2.1509E4,5.3995475E-3,2.88414E5,2.8125507E-1,9.604939E6,7.5809113E-3,8.857E3,3.1524291E-3,7.658205E-3,-4.237838E-3,-1.9817636E-3,1.0319842E-3,3.8E1,6.26E2,5.078286E-1,3.99507E5,2.3807E4,4.37849E5,3.3464E4,2.100281E-4,3.0000606E-3,3.7732E4,7.05E2,5.26565E5,2.963893E9,1.0428423E9,-3.3426913E-4,7.4E1,1.0655E4,-2.0856477E-4,1.996595E-3,3.069059E6,1E0,2.501068E-3,9.2E1,1.25688E5,-3.4539166E-4,1.01109E5,7.477759E-5,-1.9035971E-3,6.103E3,1.0314031E-3,1.004E3,2.3109813E-3,2.6077527E-4,3.1903488E-4,3.416849E-3,2.7749455E-3,6.413392E-3,1.914442E6,-6.0499326E-4,-3.9861766E-3,-1.374641E-3,1.9E1,1.4741452E-3,-1.4558278E-3,-3.1643403E-3,1.3682522E-3,2.917845E8,4.328E4,7.318324E-4,3.161145E-4,3.4325E3,-4.5733995E-4,2.4151945E-4,-2.2321798E-4,1.0710438E-3,3.8305655E-4,-2.990444E-4,-6.4608257E-4,4.6989495E-5,-1.1948731E-3,-3.037637E-3,9.6607E4,4.058E3,-9.444195E-4,-2.2007697E-4,7.19E2,2.640421E6,4.495E3,1.37839E5,8.02283E5,-8.562475E-6,1.1159E5,-6.890374E-5,6.0706E4,-2.62813E-3,2.3592582E-4,2.9837253E9,-1.7163365E-3,-6.095133E-4,3.3952467E-4,1.6081316E-3,-2.1368654E-3,3.34961E5,2.1313655E-1,2.533269E-2,-5.423291E-5,-8.160964E-4,-1.8194434E-3,-1.0036918E-3,-1.2141389E-3,-2.2680775E-5],"split_indices":[2,28,4,8,0,5,0,18,25,13,18,0,0,0,5,10,13,12,0,3,12,0,12,63,6,0,0,0,0,0,0,0,4,17,63,12,3,12,3,0,0,28,28,6,10,11,0,4,3,0,0,18,34,0,4,6,0,6,63,0,3,0,17,0,0,0,0,0,0,6,0,0,0,4,0,0,0,0,16,12,63,0,5,0,0,0,0,0,0,0,0,0,0,12,3,0,0,0,21,20,6,21,0,12,0,6,0,0,10,0,0,0,0,0,2,63,67,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,1.96E2,8.9E1,1.93E2,3E0,8.5E1,4E0,8E0,1.85E2,7.4E1,1.1E1,5E0,3E0,1.37E2,4.8E1,6.1E1,1.3E1,8E0,3E0,1E1,1.27E2,3E0,4.5E1,2.8E1,3.3E1,4E0,9E0,5E0,3E0,4E0,6E0,5E0,1.22E2,3.2E1,1.3E1,1.3E1,1.5E1,1.7E1,1.6E1,4E0,5E0,2E1,1.02E2,2E1,1.2E1,1E1,3E0,6E0,7E0,4E0,1.1E1,1.1E1,6E0,6E0,1E1,1.6E1,4E0,1E1,9.2E1,4E0,1.6E1,4E0,8E0,7E0,3E0,3E0,3E0,3E0,4E0,8E0,3E0,3E0,3E0,6E0,4E0,6E0,1E1,3E0,7E0,8E0,8.4E1,5E0,1.1E1,4E0,4E0,3E0,5E0,3E0,3E0,4E0,3E0,3E0,5E0,2.2E1,6.2E1,8E0,3E0,1.2E1,1E1,2.3E1,3.9E1,8E0,4E0,6E0,4E0,1.7E1,6E0,5E0,3.4E1,5E0,3E0,3E0,3E0,5E0,1.2E1,1.7E1,1.7E1,3E0,9E0,9E0,8E0,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[3.4518759E-3,-3.834538E-2,6.247321E-2,-3.296695E-2,-5.327881E-3,4.7230747E-2,2.2658607E-1,-4.291361E-2,2.9703815E-2,6.5685697E-3,3.6936756E-2,2.6108618E-3,9.19978E-3,-3.7192374E-2,-1.3731334E-1,6.667722E-3,-1.1377359E-2,9.860344E-3,9.9152595E-2,-8.011425E-2,-2.5425423E-2,-5.8908244E-3,-1.8441288E-3,3.64398E-2,-3.8322587E-2,-6.632801E-3,5.2744392E-2,1.3504067E-1,3.7057757E-2,-3.2836208E-3,-5.405932E-2,3.163936E-2,-3.2269597E-2,1.7614249E-3,2.9125364E-4,-2.2876696E-3,-1.9510113E-2,4.0239906E-3,-2.750652E-3,2.4544686E-2,4.0813875E-3,9.4811074E-2,7.1033123E-3,-9.589623E-4,6.2651075E-2,-1.8294573E-2,-2.5299252E-3,4.7981504E-2,-2.8824445E-4,1.3686761E-2,-3.5799168E-2,-8.972819E-4,-6.71002E-5,-9.596964E-3,3.1905696E-2,-5.170109E-4,4.645408E-2,4.0875776E-3,4.4494204E-2,7.985376E-4,2.492651E-3,-1.42714E-3,3.9659426E-4,2.1925357E-3,7.1713096E-4,-1.2284906E-3,1.9994753E-3,-4.1194666E-2,2.7712854E-3,9.935857E-3,-1.9371442E-2,4.685115E-3,5.5138066E-2,2.3304615E-3,8.044626E-4,3.4158E-4,2.1648086E-3,-5.1411293E-2,-2.7562553E-2,1.9270899E-2,-6.7218376E-4,-5.4035574E-4,2.7604196E-2,-1.2867366E-3,-3.617699E-3,1.6040473E-3,-1.0419434E-3,5.048111E-4,2.4358903E-3,-5.9798587E-2,-2.0269549E-2,-1.1069038E-4,-3.634272E-2,7.1632916E-5,1.0139099E-3,1.3293783E-3,2.2558482E-4,-7.9055474E-4,2.7099156E-4,-3.1481396E-2,-7.1783856E-2,-2.485137E-4,-1.006695E-3,2.5509772E-4,-3.9066334E-4,-2.56296E-2,-2.3842133E-3,-1.2830705E-3,-2.2058048E-4,-4.1201472E-2,-8.743443E-2,-1.4497008E-3,-3.8920306E-3,-1.7738442E-3,-3.2325677E-4,-3.2488008E-3,-1.0248747E-3,-3.470788E-4,3.4963648E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,33,35,37,39,41,43,-1,45,47,49,-1,-1,-1,51,53,-1,55,-1,57,-1,-1,59,61,-1,63,-1,65,67,-1,-1,69,71,-1,73,-1,75,-1,-1,-1,-1,-1,-1,-1,-1,77,79,81,83,85,87,-1,-1,-1,-1,89,91,93,-1,-1,95,-1,97,-1,-1,-1,-1,99,101,103,105,-1,-1,-1,-1,-1,-1,107,109,-1,-1,-1,-1,111,-1,-1,-1,113,115,-1,117,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4873123E-1,9.9878445E-2,2.6196E-1,9.415603E-2,0E0,1.5551981E-1,5.435297E-2,6.456366E-2,1.5543446E-1,0E0,1.6074218E-1,0E0,0E0,6.00854E-2,1.5111387E-2,0E0,2.4588462E-2,4.8445754E-2,5.9621155E-2,9.31868E-3,3.9296E-2,0E0,0E0,3.0101975E-3,6.0061384E-3,4.2342532E-2,3.656357E-2,4.108849E-2,2.2397261E-2,0E0,1.0847826E-2,7.7270074E-3,1.4764123E-2,0E0,0E0,0E0,1.2538272E-3,1.7396092E-2,0E0,1.4091067E-2,0E0,1.701568E-2,0E0,0E0,3.1152256E-3,6.1272075E-3,0E0,2.6222765E-3,0E0,1.9948956E-2,1.7178923E-2,0E0,0E0,6.1110896E-3,9.227651E-3,0E0,3.7590414E-3,0E0,4.684237E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.610725E-3,4.6954723E-3,5.6570233E-3,6.6714548E-3,1.4996348E-2,5.2954536E-3,0E0,0E0,0E0,0E0,9.235702E-3,7.950032E-3,1.3889051E-3,0E0,0E0,1.6850107E-3,0E0,3.4836621E-3,0E0,0E0,0E0,0E0,8.261204E-3,1.1395719E-3,9.6590875E-4,8.205861E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.9969027E-3,6.44885E-3,0E0,0E0,0E0,0E0,8.684071E-3,0E0,0E0,0E0,3.3461908E-3,6.5395236E-3,0E0,1.235411E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,14,14,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,36,36,37,37,39,39,41,41,44,44,45,45,47,47,49,49,50,50,53,53,54,54,56,56,58,58,67,67,68,68,69,69,70,70,71,71,72,72,77,77,78,78,79,79,82,82,84,84,89,89,90,90,91,91,92,92,99,99,100,100,105,105,109,109,110,110,112,112],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,34,36,38,40,42,44,-1,46,48,50,-1,-1,-1,52,54,-1,56,-1,58,-1,-1,60,62,-1,64,-1,66,68,-1,-1,70,72,-1,74,-1,76,-1,-1,-1,-1,-1,-1,-1,-1,78,80,82,84,86,88,-1,-1,-1,-1,90,92,94,-1,-1,96,-1,98,-1,-1,-1,-1,100,102,104,106,-1,-1,-1,-1,-1,-1,108,110,-1,-1,-1,-1,112,-1,-1,-1,114,116,-1,118,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0317974E8,1.06056E5,4.2081E5,1.3825156E9,-5.327881E-3,5.617E3,9.2E1,3.99507E5,1.5469E4,6.5685697E-3,9.31278E5,2.6108618E-3,9.19978E-3,3.0859E5,7.208E3,6.667722E-3,5.275E3,3.4594485E3,4.4856325E11,7.325E4,2.964093E-6,-5.8908244E-3,-1.8441288E-3,8.563E3,3.348E4,1E0,2.7307752E7,6.0230115E10,1.3999609E11,-3.2836208E-3,2.4786025E-2,3.3816576E7,2.3796E4,1.7614249E-3,2.9125364E-4,-2.2876696E-3,2.2317708E3,3.69126E5,-2.750652E-3,1.0482055E10,4.0813875E-3,3.99444E5,7.1033123E-3,-9.589623E-4,6.3E1,7.175462E-3,-2.5299252E-3,5.814E3,-2.8824445E-4,3.5888664E7,1.460109E6,-8.972819E-4,-6.71002E-5,2.2368E4,8.649117E8,-5.170109E-4,4.7717344E8,4.0875776E-3,5.5659E4,7.985376E-4,2.492651E-3,-1.42714E-3,3.9659426E-4,2.1925357E-3,7.1713096E-4,-1.2284906E-3,1.9994753E-3,2.83E2,5.072E3,9.59513E5,6.1346E4,6.910113E8,1.49E3,2.3304615E-3,8.044626E-4,3.4158E-4,2.1648086E-3,1.6919283E8,1.2809E4,4.445E3,-6.7218376E-4,-5.4035574E-4,1.036345E6,-1.2867366E-3,1.878288E6,1.6040473E-3,-1.0419434E-3,5.048111E-4,2.4358903E-3,9.578E3,9.704549E6,9.72567E5,9.682833E7,7.1632916E-5,1.0139099E-3,1.3293783E-3,2.2558482E-4,-7.9055474E-4,2.7099156E-4,1.5963E4,8.33591E5,-2.485137E-4,-1.006695E-3,2.5509772E-4,-3.9066334E-4,9.955E3,-2.3842133E-3,-1.2830705E-3,-2.2058048E-4,4.6E1,4.259E4,-1.4497008E-3,6.216743E-4,-1.7738442E-3,-3.2325677E-4,-3.2488008E-3,-1.0248747E-3,-3.470788E-4,3.4963648E-4],"split_indices":[16,28,3,23,0,3,4,12,8,0,12,0,0,9,3,0,3,5,13,18,63,0,0,7,28,36,18,1,1,0,63,16,12,0,0,0,5,12,0,21,0,22,0,0,4,63,0,0,0,16,6,0,0,28,16,0,16,0,8,0,0,0,0,0,0,0,0,4,3,21,8,16,0,0,0,0,0,21,7,3,0,0,2,0,2,0,0,0,0,28,18,2,21,0,0,0,0,0,0,7,2,0,0,0,0,20,0,0,0,19,28,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,1.53E2,1.08E2,1.48E2,5E0,1E2,8E0,1.28E2,2E1,5E0,9.5E1,3E0,5E0,1.22E2,6E0,3E0,1.7E1,6.7E1,2.8E1,2.5E1,9.7E1,3E0,3E0,6E0,1.1E1,4.9E1,1.8E1,1.7E1,1.1E1,1.2E1,1.3E1,1E1,8.7E1,3E0,3E0,3E0,8E0,4.4E1,5E0,1.4E1,4E0,1.3E1,4E0,3E0,8E0,6E0,7E0,7E0,3E0,6E0,8.1E1,5E0,3E0,3E1,1.4E1,5E0,9E0,7E0,6E0,3E0,5E0,3E0,3E0,3E0,4E0,3E0,3E0,7.1E1,1E1,1E1,2E1,7E0,7E0,3E0,6E0,3E0,3E0,3.9E1,3.2E1,6E0,4E0,4E0,6E0,8E0,1.2E1,3E0,4E0,3E0,4E0,3E1,9E0,8E0,2.4E1,3E0,3E0,3E0,3E0,4E0,8E0,1E1,2E1,5E0,4E0,5E0,3E0,2E1,4E0,7E0,3E0,8E0,1.2E1,1E1,1E1,5E0,3E0,9E0,3E0,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[4.1761165E-3,-1.5256319E-2,1.0203207E-1,-3.0754881E-2,3.1727538E-2,2.0351318E-1,7.785679E-2,-2.573306E-2,-6.8052514E-3,4.5828946E-2,-6.042688E-3,3.9439264E-4,9.563387E-3,4.81987E-2,1.6794279E-1,-3.7311397E-2,1.7460665E-2,9.558247E-2,1.8562622E-2,3.3068113E-2,4.065993E-3,6.3026366E-3,2.4851458E-3,2.5720866E-3,-4.0308326E-2,8.960705E-3,-1.1218547E-2,5.9581175E-3,6.3631825E-2,-3.0022869E-2,7.3842525E-2,4.3218993E-2,-4.19962E-4,-9.539594E-2,-3.592918E-2,-4.7322758E-2,2.0764754E-3,2.7733078E-2,2.9762029E-3,-6.124278E-3,-2.984938E-3,9.4325766E-2,-7.647125E-5,2.5291411E-2,3.4212205E-3,-1.5075869E-3,-3.5252303E-3,-3.2047715E-2,-9.354148E-2,-4.416234E-4,-2.5165926E-3,1.9371085E-2,-1.9368928E-2,1.53893E-3,3.13095E-4,1.7973432E-2,-1.7156388E-3,5.144113E-2,5.4659043E-3,4.1079484E-2,-5.920838E-4,-3.6443062E-2,3.0117767E-2,-4.4969297E-3,-1.2200606E-3,8.36095E-3,1.6194668E-3,-1.253021E-3,-3.092969E-4,-7.6004136E-3,1.7799496E-3,-1.1118641E-4,2.6814477E-3,-2.8214598E-4,5.427112E-2,-4.1410893E-2,-9.533364E-3,2.6379384E-3,-3.0443608E-4,-4.0300319E-4,1.5877204E-2,-7.762527E-4,2.2956311E-4,3.531895E-4,2.1722682E-3,-6.4057715E-2,-3.0466476E-2,3.8021007E-3,-1.1215402E-3,2.4064034E-3,1.1421223E-3,-7.4913286E-2,-2.212685E-2,-7.998423E-3,-4.6163432E-2,1.2208198E-2,-6.523155E-4,3.9989818E-4,-2.6434383E-4,-3.4621367E-3,-5.8943976E-2,-4.0810304E-5,-1.1069361E-3,3.952792E-2,-2.7077453E-2,-5.2824657E-2,-1.6609669E-2,-2.1069584E-4,6.456806E-4,-6.518802E-2,-9.149085E-4,1.9655898E-3,-8.556911E-5,-3.4107286E-2,-8.5193795E-5,-3.389538E-2,-7.3280096E-2,-9.1301557E-4,-4.9350714E-5,-9.4367575E-4,-2.3083596E-3,-1.4648981E-3,-6.0887163E-4,-1.6482332E-4,-1.3270645E-3,-3.0130218E-3,-8.7532186E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":99,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,33,-1,35,-1,37,39,41,43,-1,45,47,49,51,53,-1,55,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,65,67,-1,-1,69,-1,71,-1,73,-1,75,77,-1,-1,79,-1,-1,-1,81,-1,-1,-1,-1,83,85,87,-1,-1,-1,89,-1,-1,-1,-1,91,93,95,-1,97,-1,99,101,103,105,107,-1,-1,-1,-1,109,-1,-1,111,113,115,117,-1,-1,119,-1,-1,-1,121,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7393205E-1,1.8447769E-1,1.06821835E-1,1.6693647E-1,2.0356287E-1,1.5745771E-1,1.04120076E-1,9.367223E-2,0E0,7.962148E-2,0E0,0E0,0E0,3.482785E-2,1.016131E-2,5.4860756E-2,3.0243942E-1,4.8283666E-2,1.0964902E-1,1.4138967E-2,0E0,0E0,0E0,0E0,3.1310588E-2,0E0,1.7858496E-2,0E0,1.5029229E-2,3.2402463E-2,2.988682E-2,2.444702E-2,0E0,1.7278567E-3,2.7190924E-2,9.296963E-3,1.0760114E-2,2.7884473E-3,0E0,2.0283608E-2,0E0,3.9819613E-2,0E0,1.4549784E-2,0E0,0E0,0E0,3.6072165E-2,1.5252106E-2,0E0,0E0,5.0298367E-3,2.1075276E-3,0E0,0E0,1.1874334E-2,0E0,1.9527685E-2,0E0,1.0338716E-2,0E0,1.5647158E-2,1.9237267E-2,0E0,0E0,2.239078E-3,0E0,0E0,0E0,2.1184064E-3,0E0,0E0,0E0,0E0,6.1252005E-3,2.3081869E-2,6.882626E-3,0E0,0E0,0E0,2.5555128E-3,0E0,0E0,0E0,0E0,1.2908295E-2,2.4318367E-2,3.07977E-3,0E0,8.4659806E-4,0E0,6.692484E-3,2.00256E-3,2.8273009E-2,7.1940273E-3,1.7508683E-3,0E0,0E0,0E0,0E0,1.76806E-3,0E0,0E0,8.339107E-3,3.6881026E-3,1.0293961E-2,1.5514754E-3,0E0,0E0,1.4083833E-3,0E0,0E0,0E0,2.0838566E-3,0E0,3.7974343E-3,1.1504717E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,13,13,14,14,15,15,16,16,17,17,18,18,19,19,24,24,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,39,39,41,41,43,43,47,47,48,48,51,51,52,52,55,55,57,57,59,59,61,61,62,62,65,65,69,69,74,74,75,75,76,76,80,80,85,85,86,86,87,87,89,89,91,91,92,92,93,93,94,94,95,95,100,100,103,103,104,104,105,105,106,106,109,109,113,113,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,34,-1,36,-1,38,40,42,44,-1,46,48,50,52,54,-1,56,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,66,68,-1,-1,70,-1,72,-1,74,-1,76,78,-1,-1,80,-1,-1,-1,82,-1,-1,-1,-1,84,86,88,-1,-1,-1,90,-1,-1,-1,-1,92,94,96,-1,98,-1,100,102,104,106,108,-1,-1,-1,-1,110,-1,-1,112,114,116,118,-1,-1,120,-1,-1,-1,122,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2084881E9,4.320722E6,2.2397E4,1.11386E5,3.11925E5,1.55E2,6.2901763E3,1.3925508E9,-6.8052514E-3,2.1009742E10,-6.042688E-3,3.9439264E-4,9.563387E-3,1.4657185E10,1.83E3,1E0,3.719E3,3.4108714E8,1.01056E5,5.105882E3,4.065993E-3,6.3026366E-3,2.4851458E-3,2.5720866E-3,1.78E2,8.960705E-3,1.4397123E8,5.9581175E-3,9.617851E6,1.83E2,7.3512E5,3.7868384E3,-4.19962E-4,6.4E1,2.0315E5,7.38E2,1.02E2,3.121111E6,2.9762029E-3,4.8242E4,-2.984938E-3,9.896584E10,-7.647125E-5,1.23E2,3.4212205E-3,-1.5075869E-3,-3.5252303E-3,6.910113E8,4.796E3,-4.416234E-4,-2.5165926E-3,7.419E3,2.8327882E8,1.53893E-3,3.13095E-4,4.37849E5,-1.7156388E-3,2.0292E4,5.4659043E-3,4.89E2,-5.920838E-4,6.00237E5,7.031E3,-4.4969297E-3,-1.2200606E-3,6.6378E4,1.6194668E-3,-1.253021E-3,-3.092969E-4,5.710497E6,1.7799496E-3,-1.1118641E-4,2.6814477E-3,-2.8214598E-4,4.9E2,6.7452E4,5.395E3,2.6379384E-3,-3.0443608E-4,-4.0300319E-4,5.493291E9,-7.762527E-4,2.2956311E-4,3.531895E-4,2.1722682E-3,1.0747238E8,8.2E1,4.4E2,-1.1215402E-3,1.71111E5,1.1421223E-3,4.5E1,1.0866E4,3.8374E4,5.244E3,6.72E2,-6.523155E-4,3.9989818E-4,-2.6434383E-4,-3.4621367E-3,6E0,-4.0810304E-5,-1.1069361E-3,5.870495E7,5.097E3,1.08547E6,7.07E2,-2.1069584E-4,6.456806E-4,2.2689E4,-9.149085E-4,1.9655898E-3,-8.556911E-5,1.4059E4,-8.5193795E-5,7.656413E7,1.6395413E7,-9.1301557E-4,-4.9350714E-5,-9.4367575E-4,-2.3083596E-3,-1.4648981E-3,-6.0887163E-4,-1.6482332E-4,-1.3270645E-3,-3.0130218E-3,-8.7532186E-4],"split_indices":[16,2,8,28,3,17,5,26,0,10,0,0,0,27,19,38,7,16,20,5,0,0,0,0,22,0,27,0,2,47,28,5,0,20,12,0,4,6,0,20,0,23,0,19,0,0,0,16,7,0,0,3,27,0,0,12,0,7,0,0,0,6,3,0,0,12,0,0,0,2,0,0,0,0,17,18,3,0,0,0,25,0,0,0,0,26,4,4,0,12,0,4,28,8,3,20,0,0,0,0,19,0,0,16,3,27,0,0,0,21,0,0,0,7,0,10,27,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[3E2,2.51E2,4.9E1,1.89E2,6.2E1,8E0,4.1E1,1.85E2,4E0,5.9E1,3E0,3E0,5E0,3.2E1,9E0,1.46E2,3.9E1,2E1,3.9E1,2.8E1,4E0,6E0,3E0,3E0,1.43E2,3E0,3.6E1,4E0,1.6E1,2.1E1,1.8E1,2.3E1,5E0,9E0,1.34E2,9E0,2.7E1,8E0,8E0,1.6E1,5E0,1.4E1,4E0,1.9E1,4E0,3E0,6E0,1.27E2,7E0,5E0,4E0,1.5E1,1.2E1,3E0,5E0,1.1E1,5E0,1E1,4E0,1.4E1,5E0,1.19E2,8E0,3E0,4E0,1.2E1,3E0,3E0,9E0,7E0,4E0,4E0,6E0,3E0,1.1E1,1E2,1.9E1,3E0,5E0,3E0,9E0,3E0,4E0,3E0,8E0,3.1E1,6.9E1,1.3E1,6E0,6E0,3E0,2.4E1,7E0,2.9E1,4E1,1E1,3E0,3E0,3E0,6E0,1.8E1,3E0,4E0,8E0,2.1E1,3.2E1,8E0,3E0,7E0,1.4E1,4E0,5E0,3E0,1.6E1,5E0,1.8E1,1.4E1,4E0,4E0,3E0,1.1E1,8E0,8E0,4E0,1.4E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[7.005075E-3,-8.651208E-3,1.5101637E-1,-1.5244809E-2,9.918145E-2,1.1827387E-1,8.9477915E-3,5.8173384E-3,-1.8434158E-2,1.5821452E-1,4.224668E-2,1.1475123E-3,1.3301633E-1,-2.8371664E-2,2.7984394E-2,2.187309E-3,6.725022E-3,5.1435165E-4,2.288195E-3,1.5406284E-1,1.2574373E-3,-2.4785021E-2,-6.342003E-3,6.496571E-2,1.3056214E-3,2.6881634E-3,5.492095E-3,7.8263596E-2,-2.9586365E-2,8.169379E-2,-4.5083917E-4,2.5162239E-2,-4.655268E-2,1.1054508E-4,5.502436E-3,-2.6093721E-2,-4.899108E-3,1.0949919E-1,6.9965364E-4,5.1399603E-2,3.793042E-3,-2.524547E-3,-5.6469213E-4,-3.1046195E-2,2.7847646E-2,1.5696696E-3,4.1382816E-3,2.246535E-3,5.00831E-4,-4.848245E-4,7.086636E-4,-2.7070839E-2,-4.3304306E-3,5.4989852E-2,-1.49051715E-2,2.0258478E-3,-3.6473207E-2,1.0541333E-3,2.5804322E-3,7.29125E-4,-1.5687435E-3,-2.3162587E-2,3.8948495E-2,-5.1441148E-2,-1.7379256E-2,-5.1720083E-2,7.1766465E-3,6.826099E-2,-4.080939E-4,-4.440834E-3,-4.399434E-2,-4.238561E-2,-2.309458E-3,-2.4373322E-3,-2.0332078E-2,2.3597868E-3,-1.9994536E-2,1.4164242E-4,2.950398E-3,-2.3345696E-2,-4.972202E-2,-2.369708E-3,-2.8024118E-2,3.4418836E-2,-1.5826995E-2,1.2950004E-4,-1.2748197E-3,-1.0905548E-3,2.9946238E-4,-3.72532E-4,-1.7951656E-3,-2.3801676E-3,-1.3195641E-3,-1.6021117E-3,-1.795757E-4,1.2393548E-4,1.5119032E-3,-1.2800071E-3,5.604035E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":100,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,-1,19,21,23,-1,-1,-1,-1,25,-1,27,-1,29,31,-1,-1,33,35,37,-1,39,41,-1,-1,43,-1,45,-1,47,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,55,-1,57,59,61,63,-1,-1,-1,-1,65,67,69,71,73,75,77,-1,-1,79,81,83,-1,85,-1,87,-1,-1,89,91,-1,93,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.438907E-1,1.8495144E-1,8.351886E-2,1.5583515E-1,4.373148E-2,2.4492204E-2,0E0,0E0,1.11963555E-1,1.784283E-2,5.6820232E-3,0E0,3.3528924E-2,1.1549106E-1,4.233968E-2,0E0,0E0,0E0,0E0,4.4416487E-3,0E0,9.92908E-2,0E0,2.492302E-2,3.0856537E-2,0E0,0E0,6.182833E-2,7.653773E-2,2.2627272E-2,0E0,9.897126E-3,6.9465954E-3,0E0,0E0,5.001892E-2,0E0,5.3086355E-3,0E0,4.187666E-3,4.1110483E-3,0E0,0E0,6.588322E-2,1.9386128E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.4940628E-2,0E0,2.9741041E-3,9.997771E-3,3.903312E-2,3.454484E-2,0E0,0E0,0E0,0E0,2.2031486E-2,2.6591578E-2,3.8387135E-2,2.1005189E-2,8.2532875E-3,2.4825944E-2,1.6032781E-2,0E0,0E0,6.526679E-3,7.3565207E-3,1.8417884E-2,0E0,3.4449762E-3,0E0,4.3832613E-3,0E0,0E0,5.4618837E-3,7.538691E-3,0E0,7.5325742E-3,3.655754E-3,2.2174483E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,12,12,13,13,14,14,19,19,21,21,23,23,24,24,27,27,28,28,29,29,31,31,32,32,35,35,37,37,39,39,40,40,43,43,44,44,51,51,53,53,54,54,55,55,56,56,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,74,74,76,76,79,79,80,80,82,82,83,83,84,84],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,-1,20,22,24,-1,-1,-1,-1,26,-1,28,-1,30,32,-1,-1,34,36,38,-1,40,42,-1,-1,44,-1,46,-1,48,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,56,-1,58,60,62,64,-1,-1,-1,-1,66,68,70,72,74,76,78,-1,-1,80,82,84,-1,86,-1,88,-1,-1,90,92,-1,94,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.01248E5,1.8943199E9,4.48E2,1.949E3,5.345E3,1.5111E4,8.9477915E-3,5.8173384E-3,1.0737651E10,1E0,1.5419E4,1.1475123E-3,4.885431E11,9.356612E9,1.73E2,2.187309E-3,6.725022E-3,5.1435165E-4,2.288195E-3,3.2842822E3,1.2574373E-3,1.1E1,-6.342003E-3,1.4110743E9,2.8065149E10,2.6881634E-3,5.492095E-3,2.0315E5,1.24389E5,8.501854E10,-4.5083917E-4,2.7797654E3,1.4066751E7,1.1054508E-4,5.502436E-3,4.168507E6,-4.899108E-3,2.2256E4,6.9965364E-4,9.5842E4,3.8126335E0,-2.524547E-3,-5.6469213E-4,3.751062E6,2.54E2,1.5696696E-3,4.1382816E-3,2.246535E-3,5.00831E-4,-4.848245E-4,7.086636E-4,5.3E1,-4.3304306E-3,1.3568E4,3.35495E5,3.6E1,2.1110523E9,1.0541333E-3,2.5804322E-3,7.29125E-4,-1.5687435E-3,1.02E3,3.7388639E9,2.2155E4,1.507738E6,3.552E3,1.8318E4,6.7E2,-4.080939E-4,-4.440834E-3,1E2,1.04E2,9.8E2,-2.4373322E-3,6.6378E4,2.3597868E-3,1.969156E6,1.4164242E-4,2.950398E-3,2.3623E4,9.93E2,-2.369708E-3,1.6138E4,1.83687E5,1.039919E6,1.2950004E-4,-1.2748197E-3,-1.0905548E-3,2.9946238E-4,-3.72532E-4,-1.7951656E-3,-2.3801676E-3,-1.3195641E-3,-1.6021117E-3,-1.795757E-4,1.2393548E-4,1.5119032E-3,-1.2800071E-3,5.604035E-4],"split_indices":[3,16,4,7,7,7,0,0,25,19,7,0,1,25,17,0,0,0,0,5,0,4,0,16,10,0,0,12,8,13,0,5,2,0,0,2,0,18,0,20,60,0,0,2,17,0,0,0,0,0,0,4,0,20,12,4,10,0,0,0,0,0,10,8,2,20,8,0,0,0,4,17,0,0,12,0,2,0,0,28,20,0,22,12,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.57E2,2.7E1,2.43E2,1.4E1,2.3E1,4E0,3E0,2.4E2,6E0,8E0,4E0,1.9E1,1.98E2,4.2E1,3E0,3E0,5E0,3E0,1.5E1,4E0,1.95E2,3E0,1.7E1,2.5E1,4E0,1.1E1,8E0,1.87E2,1.4E1,3E0,1.7E1,8E0,5E0,3E0,1.83E2,4E0,9E0,5E0,7E0,1E1,3E0,5E0,1.68E2,1.5E1,3E0,6E0,4E0,3E0,5E0,5E0,1.63E2,5E0,9E0,6E0,4E1,1.23E2,6E0,3E0,3E0,3E0,2.4E1,1.6E1,6.8E1,5.5E1,1.2E1,1.2E1,1E1,6E0,4E0,6.4E1,2E1,3.5E1,6E0,6E0,3E0,9E0,3E0,7E0,1.5E1,4.9E1,5E0,1.5E1,9E0,2.6E1,3E0,3E0,6E0,3E0,1.2E1,3E0,1.1E1,3.8E1,7E0,8E0,3E0,6E0,1.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[5.824997E-3,-1.4992477E-2,8.002141E-2,-1.0923231E-2,-7.0421686E-3,9.706364E-3,6.24558E-2,-1.5441821E-2,7.045091E-3,3.793131E-2,1.5710546E-1,5.676898E-3,-1.9238649E-2,2.5640555E-2,3.25075E-3,3.2754394E-3,6.5399203E-3,-3.949091E-2,2.1864215E-2,-6.1547705E-3,6.307703E-2,-3.4664113E-2,-6.0028867E-3,8.99842E-3,1.0234776E-1,-5.6042887E-2,3.2694146E-2,1.02271706E-1,2.883754E-2,-7.750451E-2,-2.6632834E-2,3.0783756E-2,-2.5491603E-2,5.4732016E-3,4.5684076E-4,-7.33487E-2,-3.7812354E-4,4.949677E-2,-1.3163363E-4,1.5958847E-3,3.990524E-3,4.3163136E-2,-2.2606424E-4,-2.9251194E-3,-3.9694704E-2,2.6267197E-2,-3.581881E-2,3.376202E-3,1.8243138E-2,-1.5955989E-3,-2.9913525E-3,-2.9965562E-3,-1.1352256E-3,5.836896E-4,2.000748E-3,2.0233244E-3,2.4959745E-4,-3.085722E-4,-1.9274575E-3,-4.7918063E-2,8.298869E-2,-2.6806341E-2,-8.604764E-2,5.608535E-3,5.544588E-2,-1.9595461E-2,9.4922824E-4,-1.9754572E-3,-6.155439E-4,6.2026165E-4,3.4617353E-3,-3.235956E-2,2.0982423E-3,-4.3719965E-3,-3.173837E-2,1.1760466E-2,-5.0963677E-4,2.3670588E-3,6.106667E-4,-2.7838936E-2,-8.0047685E-5,-5.894189E-2,-2.454893E-2,-2.0397268E-3,-2.818728E-6,8.3766365E-4,5.2712923E-3,-2.4247686E-4,-1.2397887E-3,-3.046726E-2,-7.5827405E-2,-3.3424992E-2,-2.7017882E-3,-2.9313608E-4,1.1327547E-2,-1.31974E-3,-3.117439E-4,-3.103135E-3,-1.1578487E-3,-3.8020078E-2,6.482551E-5,7.450049E-3,-4.7669886E-4,5.881497E-4,-2.1241212E-5,-3.351441E-4,-1.4792602E-3,-2.6585642E-4,5.6674064E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":101,"left_children":[1,3,5,7,-1,-1,9,11,-1,13,15,-1,17,19,-1,-1,-1,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,53,-1,-1,-1,55,-1,-1,57,59,61,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,67,69,71,73,75,77,79,-1,-1,-1,-1,-1,81,-1,-1,83,85,-1,-1,-1,87,-1,89,91,-1,-1,-1,93,-1,-1,95,97,99,101,-1,103,-1,-1,-1,-1,105,-1,107,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0896934E-1,1.6926394E-1,2.0685458E-1,2.1519828E-1,0E0,0E0,1.2170872E-1,1.4977017E-1,0E0,3.228709E-2,7.373333E-3,0E0,1.6568375E-1,4.6810113E-2,0E0,0E0,0E0,8.790296E-2,6.742103E-2,4.447046E-2,2.120684E-2,4.266405E-2,0E0,4.436458E-2,5.1288515E-2,6.8041384E-3,8.237194E-3,3.001213E-3,5.9541026E-3,6.502278E-3,5.4880016E-2,3.154132E-2,3.691696E-2,0E0,0E0,1.9799657E-3,0E0,2.4463441E-3,0E0,0E0,0E0,5.0915144E-3,0E0,0E0,3.6800979E-3,7.5510934E-2,4.069852E-2,0E0,1.473037E-2,1.0649269E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9253809E-3,1.357428E-2,4.3998975E-2,3.3534788E-2,3.483818E-3,3.8836598E-3,1.6231872E-3,0E0,0E0,0E0,0E0,0E0,1.4747769E-2,0E0,0E0,8.003093E-3,1.7139071E-3,0E0,0E0,0E0,1.4443253E-3,0E0,5.863499E-3,1.1932522E-2,0E0,0E0,0E0,1.4093112E-3,0E0,0E0,1.354056E-3,4.7334284E-3,7.241741E-3,2.4457045E-3,0E0,9.27737E-4,0E0,0E0,0E0,0E0,8.065794E-3,0E0,1.900506E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,9,9,10,10,12,12,13,13,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,41,41,44,44,45,45,46,46,48,48,49,49,59,59,60,60,61,61,62,62,63,63,64,64,65,65,71,71,74,74,75,75,79,79,81,81,82,82,86,86,89,89,90,90,91,91,92,92,94,94,99,99,101,101],"right_children":[2,4,6,8,-1,-1,10,12,-1,14,16,-1,18,20,-1,-1,-1,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,54,-1,-1,-1,56,-1,-1,58,60,62,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,68,70,72,74,76,78,80,-1,-1,-1,-1,-1,82,-1,-1,84,86,-1,-1,-1,88,-1,90,92,-1,-1,-1,94,-1,-1,96,98,100,102,-1,104,-1,-1,-1,-1,106,-1,108,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.0629734E8,5.5712092E7,1.0916E4,2.8358776E7,-7.0421686E-3,9.706364E-3,2.18449E5,1.949E3,7.045091E-3,3.1693525E0,4.90938E5,5.676898E-3,1.027129E6,2.5735368E7,3.25075E-3,3.2754394E-3,6.5399203E-3,1.63192E5,3.0831E4,6.898215E6,6.9369E4,2.7102E5,-6.0028867E-3,2.239676E10,1.1086E4,1.90063E7,4.0231396E7,1.1828E4,8.470816E10,1.916746E6,3.2416E4,5.671E3,1.83E2,5.4732016E-3,4.5684076E-4,1.3366E4,-3.7812354E-4,3.134E3,-1.3163363E-4,1.5958847E-3,3.990524E-3,1.02E3,-2.2606424E-4,-2.9251194E-3,4.1677676E7,3.9424896E7,1.606E3,3.376202E-3,2.1602268E9,5.710497E6,-2.9913525E-3,-2.9965562E-3,-1.1352256E-3,5.836896E-4,2.000748E-3,2.0233244E-3,2.4959745E-4,-3.085722E-4,-1.9274575E-3,9.07E2,2.253E3,2.844E3,5.7312E4,5.6988126E-1,1.5022E4,5.5293235E9,9.4922824E-4,-1.9754572E-3,-6.155439E-4,6.2026165E-4,3.4617353E-3,2E0,2.0982423E-3,-4.3719965E-3,1.194485E6,3.009E4,-5.0963677E-4,2.3670588E-3,6.106667E-4,2.51868E5,-8.0047685E-5,4.984018E8,4.760905E9,-2.0397268E-3,-2.818728E-6,8.3766365E-4,6.93652E5,-2.4247686E-4,-1.2397887E-3,4.39E2,1.6581868E9,4.3199274E8,3.6E2,-2.9313608E-4,1.25E2,-1.31974E-3,-3.117439E-4,-3.103135E-3,-1.1578487E-3,1.67E2,6.482551E-5,1E0,-4.7669886E-4,5.881497E-4,-2.1241212E-5,-3.351441E-4,-1.4792602E-3,-2.6585642E-4,5.6674064E-4],"split_indices":[16,2,8,2,0,0,3,7,0,67,3,0,6,18,0,0,0,22,7,6,8,9,0,10,0,18,2,3,13,18,12,7,47,0,0,3,0,0,0,0,0,17,0,0,10,16,19,0,27,2,0,0,0,0,0,0,0,0,0,0,3,0,22,63,7,11,0,0,0,0,0,19,0,0,2,18,0,0,0,12,0,11,10,0,0,0,18,0,0,0,11,16,4,0,4,0,0,0,0,17,0,42,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.63E2,2.06E2,5.7E1,2.03E2,3E0,3E0,5.4E1,2E2,3E0,4.4E1,1E1,3E0,1.97E2,3.8E1,6E0,6E0,4E0,1.32E2,6.5E1,2.1E1,1.7E1,1.29E2,3E0,5.7E1,8E0,9E0,1.2E1,7E0,1E1,1.9E1,1.1E2,3.5E1,2.2E1,4E0,4E0,6E0,3E0,8E0,4E0,3E0,4E0,7E0,3E0,1.3E1,6E0,1.6E1,9.4E1,4E0,3.1E1,1.7E1,5E0,3E0,3E0,3E0,5E0,4E0,3E0,3E0,3E0,7E0,9E0,8.1E1,1.3E1,2.4E1,7E0,1.1E1,6E0,4E0,3E0,3E0,6E0,7.7E1,4E0,6E0,7E0,1.9E1,5E0,4E0,3E0,7E0,4E0,1.6E1,6.1E1,3E0,4E0,5E0,1.4E1,3E0,4E0,7E0,9E0,4.3E1,1.8E1,4E0,1E1,4E0,3E0,5E0,4E0,3.8E1,5E0,1E1,8E0,6E0,4E0,9E0,2.9E1,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[4.902861E-3,-1.4534887E-2,9.849475E-2,-2.7684191E-2,2.1495294E-2,8.331903E-2,7.030469E-3,-2.326088E-2,-4.824216E-3,9.605998E-2,6.191532E-3,1.3022894E-1,6.089304E-2,5.077843E-2,-3.3894017E-2,1.977567E-2,7.2300783E-3,-4.6183695E-3,1.7585414E-2,5.003943E-2,1.8411614E-1,3.5260558E-2,8.930131E-2,-5.235474E-2,9.272554E-2,-2.9639019E-2,-4.8995516E-3,1.5649423E-3,-2.3346064E-4,5.726406E-3,6.682061E-2,1.1593114E-4,2.7028238E-3,7.36525E-3,2.6464511E-3,3.2464482E-3,7.047108E-2,1.06515065E-1,1.0806088E-3,-4.3043582E-4,-2.5187423E-3,5.9372693E-2,5.7277326E-3,-5.5472806E-2,-1.842435E-2,-1.27329165E-2,4.103267E-2,3.3035958E-3,8.31813E-4,-1.7543465E-2,1.4167862E-3,2.8472617E-3,8.3298626E-4,1.6526924E-3,3.8879914E-3,2.5059838E-2,3.065281E-3,-6.465469E-2,-2.5599873E-2,8.480001E-3,-2.6601655E-2,-8.461489E-7,-4.6154495E-2,5.5624925E-2,3.1781642E-4,-1.5000433E-3,7.4564246E-4,1.6804074E-3,-5.3685868E-5,-7.0573084E-2,-3.259023E-2,-1.211908E-3,-2.3417389E-4,1.5846627E-3,-4.169001E-3,-3.1969994E-2,9.898016E-4,1.0311542E-2,-1.5767594E-3,-2.19691E-3,-2.251927E-4,2.5385318E-3,5.3477875E-4,-2.6597052E-3,-1.3460512E-3,-1.4847098E-3,-3.5111982E-4,9.4364525E-4,-1.437244E-2,-4.2983342E-2,-2.0016354E-2,7.3102023E-4,-6.9985357E-3,3.3409745E-2,-9.524399E-3,8.3345326E-4,-1.0957746E-3,-4.90136E-2,5.777515E-5,1.0314649E-3,-3.0097255E-2,5.539853E-4,-1.8228192E-2,4.492016E-4,1.4865632E-3,7.5147836E-4,-7.972874E-4,-3.6604196E-4,4.929063E-4,-5.540266E-2,-4.578425E-4,-2.154545E-2,-1.9084276E-3,-1.109303E-3,-5.1352068E-5,-4.1156624E-2,-2.3387223E-3,-1.2260418E-3,-1.2492921E-2,-2.3056655E-3,-2.8614592E-2,-6.1859266E-4,2.4108485E-4,-4.2187405E-4,-1.1803396E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":102,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,27,-1,-1,29,31,33,35,37,39,41,43,-1,-1,-1,45,47,-1,-1,-1,-1,49,51,53,-1,-1,-1,55,-1,57,59,61,63,-1,-1,65,-1,-1,-1,-1,-1,67,-1,69,71,73,75,77,79,81,-1,-1,-1,-1,-1,83,85,-1,-1,-1,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,95,97,99,-1,101,103,105,107,-1,109,-1,-1,111,-1,113,-1,-1,-1,-1,-1,-1,115,-1,117,-1,-1,-1,119,-1,-1,121,-1,123,-1,-1,-1,-1],"loss_changes":[5.290041E-1,1.14774086E-1,7.279396E-2,9.2177704E-2,7.357806E-2,4.0462017E-2,0E0,1.3713156E-1,0E0,1.0343098E-1,9.631479E-2,5.128357E-2,2.0880945E-2,1.0002968E-1,7.150857E-2,6.5889917E-3,0E0,0E0,2.9979147E-2,1.0726385E-2,1.7624706E-2,2.1188654E-2,1.0814182E-2,6.0195234E-3,3.4909964E-2,4.1566417E-2,0E0,0E0,0E0,2.854684E-2,1.2022734E-2,0E0,0E0,0E0,0E0,1.0095414E-2,4.928436E-3,2.4595559E-3,0E0,0E0,0E0,1.3607357E-2,0E0,1.0413602E-2,2.310611E-2,1.2340212E-2,6.27104E-3,0E0,0E0,1.0542546E-2,0E0,0E0,0E0,0E0,0E0,5.891171E-3,0E0,3.9624125E-3,2.3758635E-3,1.3091413E-2,1.18812695E-2,1.1618148E-2,6.4444747E-3,7.7554025E-3,0E0,0E0,0E0,0E0,0E0,5.9041977E-3,1.5510791E-3,0E0,0E0,0E0,7.2545432E-3,7.960156E-3,2.5880858E-3,9.023525E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.8138495E-3,9.268798E-3,1.8808175E-2,0E0,3.3041548E-3,1.5918985E-3,6.0902727E-3,1.9413665E-3,0E0,5.8802366E-3,0E0,0E0,5.962046E-3,0E0,2.105168E-3,0E0,0E0,0E0,0E0,0E0,0E0,3.1545311E-3,0E0,3.1183977E-3,0E0,0E0,0E0,4.065346E-3,0E0,0E0,2.428314E-3,0E0,1.1432702E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,29,29,30,30,35,35,36,36,37,37,41,41,43,43,44,44,45,45,46,46,49,49,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,69,69,70,70,74,74,75,75,76,76,77,77,88,88,89,89,90,90,92,92,93,93,94,94,95,95,97,97,100,100,102,102,109,109,111,111,115,115,118,118,120,120],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,28,-1,-1,30,32,34,36,38,40,42,44,-1,-1,-1,46,48,-1,-1,-1,-1,50,52,54,-1,-1,-1,56,-1,58,60,62,64,-1,-1,66,-1,-1,-1,-1,-1,68,-1,70,72,74,76,78,80,82,-1,-1,-1,-1,-1,84,86,-1,-1,-1,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,96,98,100,-1,102,104,106,108,-1,110,-1,-1,112,-1,114,-1,-1,-1,-1,-1,-1,116,-1,118,-1,-1,-1,120,-1,-1,122,-1,124,-1,-1,-1,-1],"split_conditions":[2.6948815E9,7.529E3,1.6189593E4,1.16215E5,1.8454938E8,2.7804902E-1,7.030469E-3,1.9082E4,-4.824216E-3,5.870495E7,1.1188E4,3.50909E10,1.07E2,1.407E3,6.9052E4,1.58E2,7.2300783E-3,-4.6183695E-3,2.6865E4,7.0526E4,5.4664E4,6.6962534E-1,5.948E3,1.77E2,1.1445502E7,5.86253E5,-4.8995516E-3,1.5649423E-3,-2.3346064E-4,1.346326E9,1.8339203E9,1.1593114E-4,2.7028238E-3,7.36525E-3,2.6464511E-3,8.857E3,9.9032694E-1,4.4777285E3,1.0806088E-3,-4.3043582E-4,-2.5187423E-3,1.1117731E-2,5.7277326E-3,1.0151622E8,7.1711E4,1.01E2,3.6E1,3.3035958E-3,8.31813E-4,2.6566594E0,1.4167862E-3,2.8472617E-3,8.3298626E-4,1.6526924E-3,3.8879914E-3,3.308E3,3.065281E-3,1.916746E6,1.1489E4,9.603E3,2.0292E4,1.0397685E9,6.8001E4,1.1443E4,3.1781642E-4,-1.5000433E-3,7.4564246E-4,1.6804074E-3,-5.3685868E-5,5.08868E5,4.01E2,-1.211908E-3,-2.3417389E-4,1.5846627E-3,2.7795588E-6,5.209E3,7.7245E4,8.922E3,-1.5767594E-3,-2.19691E-3,-2.251927E-4,2.5385318E-3,5.3477875E-4,-2.6597052E-3,-1.3460512E-3,-1.4847098E-3,-3.5111982E-4,9.4364525E-4,8.99687E5,2.235E3,5.395E3,7.3102023E-4,1.4828954E10,1.92077E5,1.5772091E0,6.78404E5,-1.0957746E-3,1.068E3,5.777515E-5,1.0314649E-3,2.215E3,5.539853E-4,1E0,4.492016E-4,1.4865632E-3,7.5147836E-4,-7.972874E-4,-3.6604196E-4,4.929063E-4,1.19021E6,-4.578425E-4,1.4191E4,-1.9084276E-3,-1.109303E-3,-5.1352068E-5,2.9E1,-2.3387223E-3,-1.2260418E-3,2.8158697E-1,-2.3056655E-3,6.7488E4,-6.1859266E-4,2.4108485E-4,-4.2187405E-4,-1.1803396E-3],"split_indices":[27,3,5,28,16,63,0,8,0,16,8,23,4,28,7,4,0,0,3,8,7,63,0,20,18,9,0,0,0,16,27,0,0,0,0,0,63,5,0,0,0,63,0,16,12,19,19,0,0,66,0,0,0,0,0,7,0,18,7,7,7,16,8,3,0,0,0,0,0,2,0,0,0,0,63,3,8,3,0,0,0,0,0,0,0,0,0,0,2,0,3,0,10,12,67,9,0,17,0,0,19,0,42,0,0,0,0,0,0,2,0,7,0,0,0,19,0,0,63,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.89E2,2.4E2,4.9E1,1.76E2,6.4E1,4.5E1,4E0,1.71E2,5E0,1E1,5.4E1,1.3E1,3.2E1,2.1E1,1.5E2,7E0,3E0,3E0,5.1E1,6E0,7E0,1.8E1,1.4E1,6E0,1.5E1,1.46E2,4E0,3E0,4E0,4.2E1,9E0,3E0,3E0,4E0,3E0,1E1,8E0,1E1,4E0,3E0,3E0,1.2E1,3E0,4.3E1,1.03E2,2.8E1,1.4E1,4E0,5E0,7E0,3E0,5E0,3E0,3E0,7E0,7E0,5E0,3.2E1,1.1E1,2.4E1,7.9E1,2.1E1,7E0,9E0,5E0,4E0,3E0,3E0,4E0,2.6E1,6E0,6E0,5E0,5E0,1.9E1,6.6E1,1.3E1,1.8E1,3E0,4E0,3E0,5E0,4E0,1.7E1,9E0,3E0,3E0,4E0,1.5E1,3.3E1,3.3E1,3E0,1E1,8E0,1E1,9E0,6E0,2.9E1,4E0,5E0,2.8E1,3E0,7E0,4E0,4E0,3E0,7E0,5E0,4E0,2.4E1,5E0,2.3E1,5E0,3E0,4E0,1.5E1,9E0,7E0,1.6E1,3E0,1.2E1,1.2E1,4E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[7.6554306E-3,-1.3012437E-2,6.0353413E-2,-2.8363599E-2,4.1483425E-2,7.408836E-2,-7.3852385E-3,3.1511204E-3,-3.208537E-2,1.9968395E-1,3.9119325E-3,5.8052525E-2,1.94143E-1,-4.1937992E-2,-1.4174214E-2,8.971971E-3,2.5977418E-3,1.4193762E-2,-2.8842117E-3,-1.7276866E-3,6.368804E-2,2.3855753E-3,8.863094E-3,-6.471673E-2,-2.9445639E-2,2.1580337E-2,-2.104164E-2,2.0590792E-3,6.4171866E-2,5.7176957E-3,5.394353E-2,-7.255079E-2,5.1803136E-4,6.6808815E-4,-3.369188E-2,1.5218471E-3,1.2272822E-4,-1.2549193E-2,-5.8335096E-2,3.4031194E-2,-9.151181E-3,2.8144817E-3,8.003629E-4,4.2565543E-2,1.29691E-1,-8.219981E-2,-1.35638E-3,-2.210701E-2,-5.2078865E-2,-1.801314E-2,1.4118592E-3,2.4208787E-4,-2.978011E-3,1.6377649E-3,1.4365124E-4,-3.2408353E-2,8.11474E-3,8.394873E-3,6.321443E-2,5.825044E-3,2.019354E-3,-9.183759E-2,-4.567412E-2,-1.5883021E-2,-5.0068814E-2,-5.448468E-4,-5.6294605E-2,-2.3183333E-2,4.5987742E-4,-1.3320709E-3,-2.7685383E-4,-5.4424596E-3,1.3959822E-3,1.8976003E-2,-1.5003113E-3,1.7638035E-2,8.256252E-2,-1.1555112E-3,-3.2392514E-3,-1.748656E-4,-2.3979885E-3,-2.6116002E-2,-4.979772E-4,-7.276641E-4,-2.092746E-3,-6.471108E-2,-7.876072E-4,-1.7649174E-2,-1.4828772E-3,3.736969E-3,-7.223428E-4,8.199205E-3,1.9324125E-3,1.3911154E-3,4.3097534E-3,4.443585E-3,6.1789088E-2,-1.5166202E-2,-1.1166872E-3,1.0068287E-3,-1.1048704E-2,-9.3695725E-4,-7.068809E-2,-2.074496E-2,-7.482307E-5,-3.1896486E-4,4.4763452E-4,-1.0218911E-2,3.019695E-2,-2.4001398E-4,3.6809905E-4,2.8960738E-2,7.378976E-2,-8.5405965E-4,-4.0944768E-5,-2.1474702E-2,9.516377E-5,-1.5438871E-3,-2.8714652E-3,-1.2833107E-2,-1.1254719E-3,3.887535E-4,-1.1243764E-3,3.091118E-4,1.3079239E-3,1.2977095E-3,3.336686E-4,2.7210715E-3,8.303079E-4,-1.6237053E-4,-1.047314E-3,2.583499E-4,-1.7876213E-2,-1.9504485E-5,-7.25256E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":103,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,23,25,-1,-1,27,-1,-1,29,-1,-1,31,33,35,37,39,41,-1,43,45,-1,-1,47,-1,-1,49,51,53,55,-1,-1,57,59,61,-1,63,65,67,-1,-1,-1,-1,-1,69,71,73,75,-1,-1,77,79,81,83,-1,85,87,-1,-1,-1,89,-1,91,-1,93,95,-1,-1,-1,-1,97,99,-1,-1,101,-1,103,-1,105,-1,107,-1,-1,109,-1,111,113,-1,-1,115,-1,117,119,-1,-1,-1,121,123,-1,-1,125,127,-1,-1,129,-1,-1,-1,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,133,-1,-1],"loss_changes":[3.209969E-1,1.7853047E-1,3.4735608E-1,8.114493E-2,2.7857885E-1,1.4274362E-1,0E0,0E0,2.7994022E-2,6.214875E-2,3.877028E-2,4.875189E-2,6.731564E-2,2.7688608E-2,1.4941047E-2,0E0,0E0,2.1700727E-2,0E0,0E0,6.702876E-2,0E0,0E0,2.3958191E-2,1.5407648E-2,4.385692E-3,1.5103575E-2,1.1063897E-2,4.227169E-3,0E0,5.0500125E-2,6.2202215E-3,0E0,0E0,1.2481391E-2,0E0,0E0,1.3735561E-2,2.0955255E-2,3.7584128E-3,9.498785E-3,0E0,0E0,4.039333E-2,1.6111895E-2,4.2540133E-3,0E0,6.34015E-3,2.6037619E-3,6.844095E-3,0E0,0E0,0E0,0E0,0E0,1.7806701E-3,6.9271945E-3,1.4270656E-2,3.0289143E-2,0E0,0E0,5.2431524E-3,7.714524E-3,5.4972162E-3,1.4034193E-3,0E0,4.0418133E-3,3.492931E-3,0E0,0E0,0E0,1.800236E-3,0E0,8.357119E-3,0E0,3.9048542E-3,2.0285204E-2,0E0,0E0,0E0,0E0,1.4457861E-3,5.374595E-3,0E0,0E0,1.3472065E-3,0E0,1.3275947E-3,0E0,1.2479929E-3,0E0,7.1962494E-3,0E0,0E0,8.398596E-4,0E0,5.7422444E-3,1.7034595E-3,0E0,0E0,1.8156954E-3,0E0,1.1921376E-3,2.2513727E-3,0E0,0E0,0E0,5.9315953E-3,1.3291049E-3,0E0,0E0,1.0600523E-3,5.0390363E-3,0E0,0E0,1.1081186E-3,0E0,0E0,0E0,1.9317889E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1118492E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,20,20,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,34,34,37,37,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,49,49,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,66,66,67,67,71,71,73,73,75,75,76,76,81,81,82,82,85,85,87,87,89,89,91,91,94,94,96,96,97,97,100,100,102,102,103,103,107,107,108,108,111,111,112,112,115,115,119,119,132,132],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,24,26,-1,-1,28,-1,-1,30,-1,-1,32,34,36,38,40,42,-1,44,46,-1,-1,48,-1,-1,50,52,54,56,-1,-1,58,60,62,-1,64,66,68,-1,-1,-1,-1,-1,70,72,74,76,-1,-1,78,80,82,84,-1,86,88,-1,-1,-1,90,-1,92,-1,94,96,-1,-1,-1,-1,98,100,-1,-1,102,-1,104,-1,106,-1,108,-1,-1,110,-1,112,114,-1,-1,116,-1,118,120,-1,-1,-1,122,124,-1,-1,126,128,-1,-1,130,-1,-1,-1,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,134,-1,-1],"split_conditions":[8.1188E4,7.164074E8,6.36E2,2.302E3,9.603E3,4.2081E5,-7.3852385E-3,3.1511204E-3,6.00237E5,1.875641E6,1.843E3,1.4448E4,8.020398E3,7.325E4,1.1874E4,8.971971E-3,2.5977418E-3,3.4325E3,-2.8842117E-3,-1.7276866E-3,2.0072E4,2.3855753E-3,8.863094E-3,2.2728746E-1,8.272E3,2.71E2,1.83E2,9E1,2.6E1,5.7176957E-3,2.814E4,1.0816216E3,5.1803136E-4,6.6808815E-4,9.5269635E-2,1.5218471E-3,1.2272822E-4,4.365884E6,5.1E1,7.7245E4,2.83326E9,2.8144817E-3,8.003629E-4,1.6210217E0,3.562448E3,9.2701E4,-1.35638E-3,1.8090405E3,2.371E3,1E0,1.4118592E-3,2.4208787E-4,-2.978011E-3,1.6377649E-3,1.4365124E-4,1.33E2,1.85E2,6.72444E5,1.0431399E7,5.825044E-3,2.019354E-3,3.318E3,1.373E3,1.4394E4,2.34725E5,-5.448468E-4,2.1811952E7,2.2799654E0,4.5987742E-4,-1.3320709E-3,-2.7685383E-4,1.4562596E9,1.3959822E-3,9.232074E6,-1.5003113E-3,1.25E2,4.166891E0,-1.1555112E-3,-3.2392514E-3,-1.748656E-4,-2.3979885E-3,3.487E3,8.95E3,-7.276641E-4,-2.092746E-3,1.26E2,-7.876072E-4,4.9749E4,-1.4828772E-3,9.25E2,-7.223428E-4,4.1585E5,1.9324125E-3,1.3911154E-3,2.3278559E9,4.443585E-3,2.0110618E7,2.0582564E-3,-1.1166872E-3,1.0068287E-3,4.7421E4,-9.3695725E-4,3.95E2,4.1763E4,-7.482307E-5,-3.1896486E-4,4.4763452E-4,6.4311E4,6.257519E6,-2.4001398E-4,3.6809905E-4,1.6918336E7,3.9535236E0,-8.5405965E-4,-4.0944768E-5,1.074866E3,9.516377E-5,-1.5438871E-3,-2.8714652E-3,1.17E2,-1.1254719E-3,3.887535E-4,-1.1243764E-3,3.091118E-4,1.3079239E-3,1.2977095E-3,3.336686E-4,2.7210715E-3,8.303079E-4,-1.6237053E-4,-1.047314E-3,2.583499E-4,1.1055E4,-1.9504485E-5,-7.25256E-4],"split_indices":[28,16,4,7,7,3,0,0,6,2,0,8,5,18,18,0,0,5,0,0,8,0,0,63,12,4,47,4,19,0,7,5,0,0,66,0,0,6,17,8,10,0,0,66,5,8,0,5,3,44,0,0,0,0,0,4,4,28,2,0,0,7,0,22,6,0,18,67,0,0,0,16,0,18,0,19,60,0,0,0,0,3,28,0,0,17,0,20,0,0,0,22,0,0,11,0,6,67,0,0,22,0,0,28,0,0,0,20,2,0,0,2,67,0,0,5,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.93E2,2.11E2,8.2E1,1.65E2,4.6E1,7.9E1,3E0,4E0,1.61E2,8E0,3.8E1,7.1E1,8E0,1.03E2,5.8E1,4E0,4E0,3.5E1,3E0,3E0,6.8E1,4E0,4E0,3.5E1,6.8E1,9E0,4.9E1,2.9E1,6E0,4E0,6.4E1,3.2E1,3E0,5E0,6.3E1,3E0,6E0,4.1E1,8E0,7E0,2.2E1,3E0,3E0,5.7E1,7E0,2.3E1,9E0,4E1,2.3E1,3.8E1,3E0,3E0,5E0,4E0,3E0,9E0,1.3E1,2.2E1,3.5E1,3E0,4E0,1.7E1,6E0,3.4E1,6E0,3E0,2E1,3.3E1,5E0,6E0,3E0,1E1,3E0,1.9E1,3E0,1.1E1,2.4E1,3E0,1.4E1,3E0,3E0,2E1,1.4E1,3E0,3E0,1.5E1,5E0,2.8E1,5E0,7E0,3E0,1.6E1,3E0,3E0,8E0,5E0,1.9E1,1E1,1E1,3E0,1.1E1,3E0,1.2E1,2.3E1,5E0,3E0,4E0,9E0,7E0,3E0,5E0,6E0,1.3E1,5E0,5E0,6E0,5E0,7E0,5E0,1.6E1,7E0,5E0,4E0,3E0,4E0,3E0,3E0,1E1,3E0,3E0,3E0,3E0,1.3E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[5.0164303E-3,-2.214814E-2,6.272251E-2,-3.342363E-2,3.0800277E-2,7.458075E-2,-6.501565E-3,2.6980909E-3,-3.632731E-2,5.0938795E-3,7.7642356E-3,4.8453625E-2,3.167318E-1,-3.3179753E-2,-4.600388E-3,1.8098267E-2,-1.122298E-3,2.916684E-2,9.246752E-2,1.422232E-2,4.934571E-3,-5.065171E-2,-1.0969447E-2,3.9709434E-2,4.8108427E-3,5.1989537E-3,1.8425422E-2,1.1997462E-1,-1.7591429E-4,-2.8853966E-3,-4.2007096E-2,2.311957E-3,-1.605209E-2,3.337348E-4,1.967035E-3,-8.8463543E-4,1.4377213E-2,4.3716514E-3,5.6259107E-2,6.667898E-2,6.251075E-3,-4.6155456E-2,6.5417687E-4,9.91415E-4,-2.0622082E-2,2.097606E-5,9.747383E-4,3.350596E-2,-2.4981242E-2,7.6673396E-2,-4.4887356E-4,8.659455E-4,2.9371788E-3,-5.0293773E-2,-4.769676E-4,-3.8364682E-2,-1.0274274E-2,4.6611756E-2,-9.840216E-4,-6.8608765E-3,-2.5097784E-3,3.5238625E-3,1.6316193E-3,-4.473846E-2,-2.8988586E-3,-2.185627E-3,-2.202394E-2,5.403763E-3,-2.9959131E-2,-6.4703036E-4,6.234313E-2,1.392437E-2,-1.1450887E-3,-6.239428E-2,-3.5158254E-2,-3.0204507E-2,-1.5189988E-4,1.6646959E-2,-8.8045123E-4,-1.3694735E-4,-4.7526937E-2,4.5990117E-2,2.9526334E-3,-2.2245101E-4,1.1633631E-3,-2.181146E-3,-9.017179E-4,-1.9060172E-3,-2.6170595E-2,-1.2592608E-3,-2.269227E-4,-3.0810744E-4,3.59573E-2,-1.8355524E-3,-6.1239E-4,6.287336E-4,1.8772066E-3,-7.367535E-3,-3.620743E-2,4.4501856E-2,3.8675035E-4,-7.1038585E-4,4.1861874E-3,-4.651771E-2,-4.3005013E-4,5.0660083E-4,2.000219E-3,4.5551243E-4,-2.1970164E-4,-1.0031592E-3,-2.2349618E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":104,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,23,-1,25,27,-1,-1,29,31,33,35,-1,37,39,-1,-1,41,-1,43,-1,-1,-1,45,47,49,51,-1,53,-1,-1,55,-1,-1,57,59,61,-1,-1,-1,63,-1,65,67,69,-1,71,-1,-1,-1,73,-1,-1,75,77,79,-1,81,83,-1,85,87,89,-1,91,-1,-1,93,95,-1,-1,-1,-1,-1,-1,97,-1,-1,-1,99,-1,-1,-1,-1,101,103,105,-1,-1,107,109,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2619818E-1,1.11478016E-1,2.8815007E-1,5.5132195E-2,9.565669E-2,5.1609695E-1,0E0,0E0,4.5583144E-2,0E0,1.3406714E-2,6.2667266E-2,9.5837295E-2,5.6488827E-2,0E0,6.7007253E-3,0E0,7.5211585E-2,6.261402E-2,0E0,0E0,2.3871213E-2,2.892654E-2,5.0174687E-3,5.338015E-3,0E0,2.7402762E-2,5.8256358E-2,0E0,0E0,1.8779188E-2,0E0,1.4124272E-2,0E0,0E0,0E0,2.818151E-3,3.4188896E-2,2.113353E-2,9.272654E-3,0E0,7.68432E-3,0E0,0E0,1.0254459E-2,0E0,0E0,1.8220296E-2,1.8668627E-2,3.8263723E-3,0E0,0E0,0E0,8.562267E-3,0E0,8.639745E-3,1.188945E-2,1.9094381E-2,0E0,1.004428E-2,0E0,0E0,0E0,6.601952E-3,0E0,0E0,1.9809268E-3,8.55763E-3,7.474562E-3,0E0,2.5088862E-3,4.878779E-3,0E0,1.2383461E-3,6.2335245E-3,1.788781E-3,0E0,8.951173E-3,0E0,0E0,1.622064E-3,2.4488084E-3,0E0,0E0,0E0,0E0,0E0,0E0,5.017491E-3,0E0,0E0,0E0,1.5110737E-3,0E0,0E0,0E0,0E0,1.9608396E-3,3.618624E-3,2.5733458E-3,0E0,0E0,8.646996E-4,2.057897E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,10,10,11,11,12,12,13,13,15,15,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,30,30,32,32,36,36,37,37,38,38,39,39,41,41,44,44,47,47,48,48,49,49,53,53,55,55,56,56,57,57,59,59,63,63,66,66,67,67,68,68,70,70,71,71,73,73,74,74,75,75,77,77,80,80,81,81,88,88,92,92,97,97,98,98,99,99,102,102,103,103],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,24,-1,26,28,-1,-1,30,32,34,36,-1,38,40,-1,-1,42,-1,44,-1,-1,-1,46,48,50,52,-1,54,-1,-1,56,-1,-1,58,60,62,-1,-1,-1,64,-1,66,68,70,-1,72,-1,-1,-1,74,-1,-1,76,78,80,-1,82,84,-1,86,88,90,-1,92,-1,-1,94,96,-1,-1,-1,-1,-1,-1,98,-1,-1,-1,100,-1,-1,-1,-1,102,104,106,-1,-1,108,110,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.580147E6,7.164074E8,5.3408066E4,2.302E3,3.234E3,1.2738551E4,-6.501565E-3,2.6980909E-3,1.03199E5,5.0938795E-3,8.89E2,5.0600757E3,1.3774316E11,1.042754E6,-4.600388E-3,2.2368E4,-1.122298E-3,1.0927762E9,1.4595842E11,1.422232E-2,4.934571E-3,2.468E3,3.2416E4,2.68331E5,2.5339E4,5.1989537E-3,5.35875E5,7.808796E9,-1.7591429E-4,-2.8853966E-3,1.83E2,2.311957E-3,4.53E2,3.337348E-4,1.967035E-3,-8.8463543E-4,3.2318836E7,3.8719E4,1.5116796E10,6.5139646E10,6.251075E-3,4.0200136E7,6.5417687E-4,9.91415E-4,1.835475E6,2.097606E-5,9.747383E-4,4.78E2,1.83E2,1.9603042E7,-4.4887356E-4,8.659455E-4,2.9371788E-3,1.050246E6,-4.769676E-4,4.7914E4,1.9694447E10,9.642E3,-9.840216E-4,3.6060664E10,-2.5097784E-3,3.5238625E-3,1.6316193E-3,4.09E2,-2.8988586E-3,-2.185627E-3,4.68E2,1.5509368E9,4.995E3,-6.4703036E-4,1.83E2,8.956271E6,-1.1450887E-3,1.0151622E8,5.0529E4,1.4908E4,-1.5189988E-4,1.7234E4,-8.8045123E-4,-1.3694735E-4,1.754E4,1.49E2,2.9526334E-3,-2.2245101E-4,1.1633631E-3,-2.181146E-3,-9.017179E-4,-1.9060172E-3,6.2756E4,-1.2592608E-3,-2.269227E-4,-3.0810744E-4,1.1442958E10,-1.8355524E-3,-6.1239E-4,6.287336E-4,1.8772066E-3,2E0,7.000598E7,6.1E1,3.8675035E-4,-7.1038585E-4,1.3818E4,1.28200376E8,-4.3005013E-4,5.0660083E-4,2.000219E-3,4.5551243E-4,-2.1970164E-4,-1.0031592E-3,-2.2349618E-3],"split_indices":[2,16,5,7,28,5,0,0,28,0,19,5,1,9,0,28,0,11,23,0,0,3,12,6,28,0,22,27,0,0,47,0,22,0,0,0,18,18,27,26,0,18,0,0,9,0,0,17,47,6,0,0,0,2,0,18,10,7,0,10,0,0,0,0,0,0,4,21,3,0,47,2,0,16,8,7,0,28,0,0,7,4,0,0,0,0,0,0,12,0,0,0,10,0,0,0,0,19,21,17,0,0,7,26,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.84E2,8.6E1,1.52E2,3.2E1,8.3E1,3E0,3E0,1.49E2,4E0,2.8E1,7.6E1,7E0,1.46E2,3E0,2.3E1,5E0,5.4E1,2.2E1,3E0,4E0,8.1E1,6.5E1,8E0,1.5E1,3E0,5.1E1,1.7E1,5E0,1.3E1,6.8E1,3E0,6.2E1,4E0,4E0,3E0,1.2E1,3.8E1,1.3E1,1.1E1,6E0,6.4E1,4E0,5E0,5.7E1,7E0,5E0,1.9E1,1.9E1,1E1,3E0,5E0,6E0,5.6E1,8E0,2E1,3.7E1,1.6E1,3E0,1.5E1,4E0,3E0,7E0,5.1E1,5E0,6E0,1.4E1,2.1E1,1.6E1,3E0,1.3E1,9E0,6E0,1.6E1,3.5E1,9E0,5E0,1.6E1,5E0,7E0,9E0,1E1,3E0,5E0,4E0,1.3E1,3E0,8E0,2.7E1,6E0,3E0,7E0,9E0,6E0,3E0,4E0,6E0,1E1,1.7E1,6E0,3E0,4E0,6E0,1.1E1,6E0,3E0,3E0,3E0,3E0,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[4.9543297E-3,-9.318232E-4,1.8229647E-1,-3.9518557E-2,1.9667456E-2,2.6513964E-3,7.034303E-3,-3.401622E-2,-4.241132E-3,7.4715912E-3,1.2185593E-2,-6.1990395E-2,-1.5425393E-2,7.270049E-3,1.2771563E-1,-8.487292E-2,-3.142329E-2,-2.7718443E-2,5.1851477E-2,-6.936247E-3,3.4807395E-2,5.6419615E-3,1.5523471E-3,-1.3447881E-3,-2.994717E-3,-1.6760916E-2,-1.4750059E-3,1.2164613E-3,-3.2925565E-2,5.262744E-4,2.478007E-3,-1.7926397E-2,3.3649985E-2,4.1704E-3,2.3205034E-2,6.064539E-5,-9.4973185E-4,-4.3536335E-2,-1.1791056E-4,-3.2646686E-3,-1.37862945E-2,1.0581598E-2,7.092156E-2,-3.2613304E-2,3.2612626E-2,-3.7178077E-2,-2.637202E-3,3.7422616E-4,-1.0167204E-2,-8.350922E-3,-2.702592E-3,3.4257285E-2,-1.737625E-2,9.820269E-4,3.663465E-3,-2.4745695E-4,-1.7902554E-3,-4.878574E-3,4.2120945E-2,-4.685352E-2,-1.532695E-2,-6.445058E-4,7.178029E-5,-1.3382565E-2,2.246797E-3,1.8255581E-3,-2.572479E-4,-1.1212104E-3,2.8286618E-4,1.5610925E-2,-1.2719646E-3,5.2071467E-2,7.8517944E-4,-3.2572314E-2,-6.2275976E-2,7.875229E-5,-8.315294E-4,-2.0775605E-2,2.084201E-2,-7.804432E-5,9.574183E-4,2.5036104E-4,5.688312E-2,8.160881E-4,-6.1243196E-4,-1.6664505E-3,-4.5392412E-4,-2.47361E-3,-7.999328E-4,-1.0200865E-2,-3.2821823E-2,4.1433834E-2,-4.3517523E-4,2.467745E-3,3.7265945E-2,-2.4259137E-2,1.0160773E-2,-3.9301105E-2,-2.2029874E-4,3.5219567E-4,1.7657708E-3,2.0311965E-2,1.7298415E-3,-3.1973533E-2,1.1452272E-5,-6.9154245E-5,7.897656E-4,-3.0383371E-5,-4.5007486E-2,9.555613E-4,3.7755395E-5,-5.8911776E-4,-1.6644703E-3,-1.5987447E-3,-6.7899376E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":105,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,-1,37,-1,-1,39,41,-1,43,-1,-1,45,47,-1,49,51,53,55,57,59,-1,-1,61,63,-1,65,67,-1,-1,-1,-1,69,71,73,75,-1,-1,77,-1,-1,-1,-1,-1,79,-1,81,83,85,87,-1,-1,89,91,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,95,97,99,-1,-1,101,103,105,107,-1,-1,-1,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9245153E-1,2.1697885E-1,1.4587283E-2,4.3733343E-2,2.8010517E-1,0E0,0E0,4.6369515E-2,0E0,0E0,9.731701E-2,2.2513285E-2,4.775137E-2,6.557244E-2,1.8011145E-2,3.1485409E-3,2.8213225E-3,1.7617904E-2,6.771365E-3,5.0114635E-2,5.9656702E-2,0E0,0E0,0E0,0E0,2.4316986E-3,0E0,0E0,1.5660021E-2,0E0,0E0,2.9021643E-2,1.9884622E-2,0E0,2.8369647E-2,0E0,0E0,5.2537546E-3,1.534409E-3,0E0,3.171967E-2,1.1186003E-2,1.2127664E-2,4.159509E-3,1.6247008E-2,5.8358796E-3,0E0,0E0,8.9990045E-4,3.2392412E-2,0E0,8.99215E-3,3.9598113E-3,0E0,0E0,0E0,0E0,7.713818E-3,1.4766365E-2,2.596803E-3,2.1171763E-3,0E0,0E0,1.9662805E-2,0E0,0E0,0E0,0E0,0E0,1.8563549E-3,0E0,5.738206E-3,4.376147E-3,3.8798125E-3,2.9979795E-3,0E0,0E0,7.69048E-3,1.0466054E-2,0E0,0E0,0E0,7.2323754E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.0265222E-2,4.612254E-3,3.0833595E-3,0E0,0E0,3.097998E-3,3.9990516E-3,2.7067843E-3,4.9917772E-3,0E0,0E0,0E0,1.5798691E-3,0E0,3.3707693E-3,0E0,0E0,0E0,0E0,1.1810511E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,28,28,31,31,32,32,34,34,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,48,48,49,49,51,51,52,52,57,57,58,58,59,59,60,60,63,63,69,69,71,71,72,72,73,73,74,74,77,77,78,78,82,82,89,89,90,90,91,91,94,94,95,95,96,96,97,97,101,101,103,103,108,108],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,-1,38,-1,-1,40,42,-1,44,-1,-1,46,48,-1,50,52,54,56,58,60,-1,-1,62,64,-1,66,68,-1,-1,-1,-1,70,72,74,76,-1,-1,78,-1,-1,-1,-1,-1,80,-1,82,84,86,88,-1,-1,90,92,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,96,98,100,-1,-1,102,104,106,108,-1,-1,-1,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4276579E4,4.618E3,2.15195E5,5.1041E4,1.0916E4,2.6513964E-3,7.034303E-3,1.9529781E8,-4.241132E-3,7.4715912E-3,8.9003E4,2.876E3,9.433038E6,8.1188E4,2.0110618E7,4.518873E6,1.0238022E8,3E0,1.02E3,5.581883E7,2.9378E4,5.6419615E-3,1.5523471E-3,-1.3447881E-3,-2.994717E-3,2.96E2,-1.4750059E-3,1.2164613E-3,1.935849E6,5.262744E-4,2.478007E-3,4.796E3,1.1828E4,4.1704E-3,4.1E1,6.064539E-5,-9.4973185E-4,1.9537698E3,3.1177E4,-3.2646686E-3,5.59456E5,2.0315E5,1.315E3,7.59577E5,2.3278559E9,1.34028E5,-2.637202E-3,3.7422616E-4,1.85948E5,1.83E2,-2.702592E-3,1.116E3,3.6815E6,9.820269E-4,3.663465E-3,-2.4745695E-4,-1.7902554E-3,4.8612E4,1E0,6.63237E5,3.1E2,-6.445058E-4,7.178029E-5,1.9341E4,2.246797E-3,1.8255581E-3,-2.572479E-4,-1.1212104E-3,2.8286618E-4,1.3334E4,-1.2719646E-3,2.0830981E3,1.4E1,5.41808E5,2.63E2,7.875229E-5,-8.315294E-4,7.2925E4,1.29E2,-7.804432E-5,9.574183E-4,2.5036104E-4,1.4409099E10,8.160881E-4,-6.1243196E-4,-1.6664505E-3,-4.5392412E-4,-2.47361E-3,-7.999328E-4,6.44E4,3.5536144E8,2.01E2,-4.3517523E-4,2.467745E-3,1.8927E4,6.89E2,7.208E3,1.1268727E3,-2.2029874E-4,3.5219567E-4,1.7657708E-3,3.389E3,1.7298415E-3,3.8594732E7,1.1452272E-5,-6.9154245E-5,7.897656E-4,-3.0383371E-5,7.721E3,9.555613E-4,3.7755395E-5,-5.8911776E-4,-1.6644703E-3,-1.5987447E-3,-6.7899376E-4],"split_indices":[5,3,3,20,8,0,0,10,0,0,7,3,18,28,6,10,10,17,0,18,20,0,0,0,0,17,0,0,2,0,0,7,3,0,19,0,0,5,18,0,12,12,0,28,11,18,0,0,18,47,0,17,2,0,0,0,0,20,34,9,17,0,0,7,0,0,0,0,0,7,0,5,17,2,4,0,0,8,4,0,0,0,1,0,0,0,0,0,0,8,27,17,0,0,7,17,3,5,0,0,0,0,0,27,0,0,0,0,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,2.71E2,8E0,9.4E1,1.77E2,3E0,5E0,9E1,4E0,5E0,1.72E2,3.5E1,5.5E1,1.66E2,6E0,1.9E1,1.6E1,4.7E1,8E0,1.1E2,5.6E1,3E0,3E0,4E0,1.5E1,9E0,7E0,3E0,4.4E1,4E0,4E0,8.7E1,2.3E1,5E0,5.1E1,4E0,5E0,3.3E1,1.1E1,3E0,8.4E1,1.5E1,8E0,7E0,4.4E1,3E1,3E0,5E0,6E0,7.9E1,5E0,8E0,7E0,5E0,3E0,4E0,3E0,9E0,3.5E1,2E1,1E1,3E0,3E0,7.5E1,4E0,5E0,3E0,4E0,3E0,6E0,3E0,2.8E1,7E0,1.2E1,8E0,4E0,6E0,6.2E1,1.3E1,3E0,3E0,3E0,2.5E1,3E0,4E0,5E0,7E0,5E0,3E0,3.4E1,2.8E1,8E0,5E0,1.1E1,1.4E1,2E1,1.4E1,2.2E1,6E0,3E0,5E0,8E0,6E0,1.5E1,5E0,8E0,6E0,3E0,1.9E1,5E0,3E0,1E1,5E0,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[4.703618E-3,-2.2957359E-2,4.9945574E-2,-2.7830716E-2,3.619634E-3,5.9996147E-2,-6.8012862E-3,-2.2780169E-2,-1.3640997E-1,3.6597252E-2,1.101422E-1,-4.067402E-2,1.5963479E-3,-8.966978E-3,1.2829117E-3,1.674004E-2,1.01328015E-1,1.8188848E-1,3.151203E-2,-4.7187477E-2,1.6152412E-2,4.0625697E-3,-6.2169502E-3,3.8708173E-2,-1.9624554E-2,-2.3023262E-4,4.444788E-3,2.3915705E-1,7.4092165E-2,5.501522E-2,7.118967E-3,-7.428377E-2,-3.882934E-2,-3.839693E-4,2.9891219E-2,-1.3727546E-2,2.210525E-3,2.6040835E-2,4.0945318E-3,-6.5883756E-2,2.056016E-2,2.7152176E-3,9.445899E-3,-2.9242728E-4,4.466089E-3,8.34006E-4,2.1662076E-3,-9.3825476E-4,1.3507287E-3,-8.623944E-2,-3.5018535E-4,6.04494E-4,-4.1970428E-2,1.4974767E-3,1.8631588E-4,-3.360717E-3,-7.005386E-3,2.797981E-3,4.501742E-2,9.0894796E-4,-3.5490126E-3,4.3100484E-2,-1.1361288E-3,-3.115601E-3,-1.4735518E-3,-4.463289E-2,2.2943944E-4,-1.2396806E-2,2.1303385E-3,1.2758581E-3,-1.420117E-2,1.9588208E-4,5.292533E-2,2.0758964E-3,6.971086E-4,-3.886759E-2,-2.3080488E-3,1.3770255E-2,-1.6030265E-2,-7.715397E-4,-6.6515226E-5,2.1010493E-3,1.8475365E-4,-1.8641752E-3,-3.2658275E-2,8.845182E-4,-1.0883036E-4,-4.1347697E-2,-8.9459345E-3,-1.6793758E-2,-4.1505236E-2,-1.59381E-3,-5.3818093E-4,2.1922637E-2,-1.662344E-2,-7.703984E-4,-2.1444896E-4,-2.0793898E-2,-4.840443E-2,1.1160953E-3,2.3619067E-4,-2.4613349E-2,-1.9216383E-3,6.294868E-5,-1.1212933E-3,-7.8857027E-4,-5.5714674E-2,-1.4237817E-2,-1.1495505E-3,6.595649E-4,-6.7137944E-4,-2.1139209E-3,-8.577757E-4,-8.912135E-4,-2.1497248E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":106,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,-1,35,37,39,-1,-1,41,43,45,47,49,51,-1,53,55,-1,57,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,-1,65,-1,-1,-1,67,69,71,-1,-1,73,-1,-1,-1,75,-1,77,-1,-1,79,-1,81,-1,-1,83,-1,85,87,-1,-1,-1,-1,-1,89,-1,-1,91,93,95,97,-1,-1,99,101,-1,-1,103,105,-1,-1,107,109,-1,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3653167E-1,1.1333058E-1,2.804872E-1,8.5730866E-2,0E0,1.1213195E-1,0E0,6.811053E-2,1.86512E-1,8.699447E-2,1.6915247E-1,3.4074485E-2,6.601878E-2,0E0,0E0,4.389819E-2,6.478502E-2,8.1053734E-2,8.781385E-3,1.5460044E-2,4.214557E-3,0E0,3.6940865E-2,3.4801316E-2,4.0391747E-2,0E0,0E0,5.9297025E-2,3.8904395E-2,1.116015E-3,1.2596261E-2,1.270663E-2,1.2369268E-2,0E0,2.5365958E-3,3.52323E-2,0E0,1.3547214E-2,0E0,4.487687E-2,1.6629342E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9730106E-3,0E0,0E0,8.292213E-3,0E0,0E0,0E0,2.291723E-2,9.9562425E-3,4.9117394E-3,0E0,0E0,2.7315542E-3,0E0,0E0,0E0,6.2650144E-3,0E0,5.259821E-3,0E0,0E0,1.254315E-3,0E0,7.7818E-3,0E0,0E0,4.0217414E-3,0E0,1.7387513E-3,8.059888E-3,0E0,0E0,0E0,0E0,0E0,4.9170814E-3,0E0,0E0,1.1834148E-3,9.370186E-3,9.979582E-4,2.5657713E-3,0E0,0E0,1.2333216E-3,3.5941498E-3,0E0,0E0,2.623495E-3,1.7807893E-3,0E0,0E0,1.7947387E-3,5.5200597E-3,0E0,0E0,0E0,2.0125657E-3,1.0350947E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,37,37,39,39,40,40,49,49,52,52,56,56,57,57,58,58,61,61,65,65,67,67,70,70,72,72,75,75,77,77,78,78,84,84,87,87,88,88,89,89,90,90,93,93,94,94,97,97,98,98,101,101,102,102,106,106,107,107],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,-1,36,38,40,-1,-1,42,44,46,48,50,52,-1,54,56,-1,58,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,-1,66,-1,-1,-1,68,70,72,-1,-1,74,-1,-1,-1,76,-1,78,-1,-1,80,-1,82,-1,-1,84,-1,86,88,-1,-1,-1,-1,-1,90,-1,-1,92,94,96,98,-1,-1,100,102,-1,-1,104,106,-1,-1,108,110,-1,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.88414E5,1.5860209E9,6.25E2,1.11386E5,3.619634E-3,3.85E2,-6.8012862E-3,5.013E3,1.2081E4,1.01248E5,3.954342E-1,2.761E3,6.93E3,-8.966978E-3,1.2829117E-3,2.9188708E10,1.6301E4,4.1E2,8.28337E5,1.511E3,1.0338E4,4.0625697E-3,3.987307E6,1.243813E6,9.604939E6,-2.3023262E-4,4.444788E-3,7.143424E7,3.102E4,6.8917E4,7.0231123E3,1.9118268E3,3.318E3,-3.839693E-4,1.9897E4,2.0547E4,2.210525E-3,1.225E3,4.0945318E-3,5.71934E5,6.7975234E10,2.7152176E-3,9.445899E-3,-2.9242728E-4,4.466089E-3,8.34006E-4,2.1662076E-3,-9.3825476E-4,1.3507287E-3,8.095E3,-3.5018535E-4,6.04494E-4,1.864E3,1.4974767E-3,1.8631588E-4,-3.360717E-3,2.414E3,8.58E2,5.1E1,9.0894796E-4,-3.5490126E-3,8.01E3,-1.1361288E-3,-3.115601E-3,-1.4735518E-3,1.32E3,2.2943944E-4,1.41048E5,2.1303385E-3,1.2758581E-3,9.109177E8,1.9588208E-4,1.1619143E1,2.0758964E-3,6.971086E-4,3.73907E5,-2.3080488E-3,5.7728936E7,9.1455296E8,-7.715397E-4,-6.6515226E-5,2.1010493E-3,1.8475365E-4,-1.8641752E-3,6.3849E4,8.845182E-4,-1.0883036E-4,1.3696E4,5.395E3,6.78647E5,8.9042156E-4,-1.59381E-3,-5.3818093E-4,1.563477E6,1.85737E5,-7.703984E-4,-2.1444896E-4,7.5099E4,9.86712E2,1.1160953E-3,2.3619067E-4,5.92953E5,7.571E4,6.294868E-5,-1.1212933E-3,-7.8857027E-4,9.26E2,5.61E2,-1.1495505E-3,6.595649E-4,-6.7137944E-4,-2.1139209E-3,-8.577757E-4,-8.912135E-4,-2.1497248E-4],"split_indices":[12,16,4,28,0,19,0,3,7,3,63,0,12,0,0,10,7,17,28,28,7,0,2,12,6,0,0,16,7,3,5,5,7,0,7,8,0,0,0,12,23,0,0,0,0,0,0,0,0,7,0,0,0,0,0,0,0,0,17,0,0,3,0,0,0,0,0,6,0,0,11,0,60,0,0,2,0,11,10,0,0,0,0,0,8,0,0,20,3,2,63,0,0,2,12,0,0,8,5,0,0,6,8,0,0,0,17,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,1.66E2,1.01E2,1.61E2,5E0,9.8E1,3E0,1.55E2,6E0,6.8E1,3E1,8.9E1,6.6E1,3E0,3E0,5.3E1,1.5E1,1.5E1,1.5E1,8E1,9E0,3E0,6.3E1,3.3E1,2E1,4E0,1.1E1,9E0,6E0,7E0,8E0,1.7E1,6.3E1,3E0,6E0,5.8E1,5E0,3E1,3E0,9E0,1.1E1,3E0,6E0,3E0,3E0,3E0,4E0,4E0,4E0,1.4E1,3E0,3E0,6E1,3E0,3E0,3E0,5.5E1,1.4E1,1.6E1,3E0,6E0,8E0,3E0,1E1,4E0,5.7E1,3E0,5.2E1,3E0,4E0,1E1,3E0,1.3E1,3E0,5E0,4.9E1,8E0,6E0,4.6E1,5E0,5E0,1E1,3E0,1E1,3.9E1,3E0,3E0,9E0,3.7E1,1.5E1,2.4E1,6E0,3E0,7E0,3E1,8E0,7E0,7E0,1.7E1,3E0,4E0,1.9E1,1.1E1,3E0,4E0,5E0,1.2E1,1.1E1,8E0,5E0,6E0,8E0,4E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[1.3195643E-2,-7.077459E-3,9.0013504E-2,-2.3521217E-2,4.0422816E-2,7.031027E-3,1.490858E-1,-3.77398E-2,-1.1898759E-2,5.6919634E-2,-6.4583397E-3,-4.9164303E-2,3.473808E-2,2.7718785E-1,9.408173E-2,-5.0414138E-2,-2.1452313E-2,-2.1244776E-2,1.483133E-3,4.8238315E-2,3.9020823E-3,-2.6081603E-3,3.5150786E-4,5.515921E-2,-1.6263693E-3,3.601741E-1,2.0171197E-3,1.3183616E-1,2.0491015E-2,-5.955269E-2,-3.4920923E-2,-3.2350082E-2,2.1309182E-3,-2.6473075E-2,2.550372E-3,-9.185624E-3,2.1952655E-2,5.7577115E-2,1.651128E-2,2.2684155E-2,8.903297E-2,1.4792819E-2,5.4960325E-3,6.1436847E-2,7.1984143E-3,2.834642E-3,-2.104858E-2,-6.940647E-2,-3.432302E-2,-4.6051662E-2,-1.6043508E-4,-1.4825976E-2,-5.1712573E-2,-1.469939E-2,1.2703817E-3,-1.8820805E-2,-6.0468756E-2,-5.601879E-4,1.6438518E-2,9.693158E-3,-1.9163523E-2,1.6449123E-3,5.1828385E-3,4.144298E-2,1.0675279E-1,3.264635E-2,-8.9981727E-4,1.3821566E-3,-4.303313E-4,1.340911E-3,3.6743768E-3,8.830326E-3,3.9527593E-3,-1.9459268E-3,7.6024595E-4,-7.619739E-2,-9.105164E-4,-1.4724095E-3,-2.7723602E-4,-7.4749836E-4,-1.7259028E-3,3.534921E-4,-2.6247257E-2,-2.2738131E-3,-9.916069E-4,3.300923E-4,-9.2979864E-4,-2.2965236E-2,2.5110933E-4,-2.7832135E-3,-8.877114E-4,8.453894E-4,8.060351E-5,-1.965322E-4,1.123963E-3,-4.657609E-5,-2.5866283E-2,-2.7348063E-4,5.059508E-4,1.3918376E-4,4.6781946E-2,4.124764E-3,1.5444419E-3,1.1732741E-4,1.434173E-3,1.7786943E-3,-1.2812761E-3,-1.1488487E-3,-2.703874E-3,-1.1029177E-3,-1.8207383E-4,-4.017051E-2,-1.1754609E-2,-2.1516204E-4,-1.1916177E-3,3.0512732E-4,5.1049814E-2,-8.6015096E-4,-2.052025E-3,-5.3569707E-3,-8.987849E-4,7.5541105E-4,5.7073023E-2,-3.5180707E-5,-6.0971756E-4,2.1048128E-3,9.063207E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":107,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,-1,-1,-1,39,-1,41,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,-1,73,75,77,79,-1,81,83,85,-1,87,89,-1,91,93,95,-1,97,99,101,103,-1,-1,-1,-1,-1,105,-1,-1,-1,107,-1,-1,-1,-1,-1,-1,109,-1,-1,-1,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,117,119,-1,-1,-1,121,-1,-1,123,-1,-1,125,-1,-1,-1,-1],"loss_changes":[4.4759256E-1,1.7903772E-1,2.9306975E-1,2.7644277E-2,2.4604806E-1,4.2190056E-2,2.2053647E-1,1.4607415E-2,1.1912909E-2,2.5414288E-2,0E0,1.8114647E-2,3.412825E-2,1.5545422E-1,6.953631E-2,3.8669556E-3,9.064097E-3,7.0343744E-3,8.937023E-3,1.4160253E-2,0E0,0E0,0E0,1.4481146E-2,0E0,5.332321E-2,0E0,9.611857E-2,2.9922536E-2,3.8329512E-3,5.768398E-3,7.3767602E-3,8.1205405E-3,1.0828558E-2,3.3343665E-3,5.271761E-3,6.5786988E-3,2.6518717E-2,1.0239506E-2,7.1462365E-3,3.5743825E-3,0E0,0E0,3.4423966E-2,0E0,0E0,1.3748359E-2,2.5917515E-3,2.248453E-3,1.0003913E-3,0E0,4.414849E-3,2.176609E-3,3.4822784E-3,0E0,4.4821277E-3,4.455682E-3,0E0,8.9255814E-4,4.198344E-3,2.1000695E-3,0E0,1.5746613E-3,5.9412085E-3,5.165614E-3,3.2912195E-3,0E0,0E0,0E0,0E0,0E0,1.8005619E-2,0E0,0E0,0E0,1.4268011E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.456874E-3,0E0,0E0,0E0,0E0,6.0857963E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5337627E-3,0E0,0E0,0E0,3.8145706E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4188217E-3,2.1259713E-3,0E0,0E0,0E0,2.6477426E-3,0E0,0E0,9.6404273E-4,0E0,0E0,2.727989E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,23,23,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,46,46,47,47,48,48,49,49,51,51,52,52,53,53,55,55,56,56,58,58,59,59,60,60,62,62,63,63,64,64,65,65,71,71,75,75,82,82,87,87,96,96,100,100,111,111,112,112,116,116,119,119,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,-1,-1,-1,40,-1,42,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,-1,74,76,78,80,-1,82,84,86,-1,88,90,-1,92,94,96,-1,98,100,102,104,-1,-1,-1,-1,-1,106,-1,-1,-1,108,-1,-1,-1,-1,-1,-1,110,-1,-1,-1,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,118,120,-1,-1,-1,122,-1,-1,124,-1,-1,126,-1,-1,-1,-1],"split_conditions":[1.83E2,4.365884E6,4.3406326E8,1.09081E6,3.539647E4,2.219E3,2.99595E5,2.050352E6,7.9503E4,1.0778861E4,-6.4583397E-3,9.4661897E-4,1.6010206E10,2.844E3,1.9771761E0,2.63E2,4.951E3,5.101357E7,3.711818E6,1.199E3,3.9020823E-3,-2.6081603E-3,3.5150786E-4,5.1E1,-1.6263693E-3,1.571E4,2.0171197E-3,1.8943199E9,1.858566E9,2.2728746E-1,2.866E3,7.4539E4,8.95E3,7.385252E8,5.92953E5,2.2213E4,2.17568E5,8.800221E9,1.99916E5,8.808534E8,6.4762153E9,1.4792819E-2,5.4960325E-3,2.1E1,7.1984143E-3,2.834642E-3,8.674837E10,1.004E3,4.43019E5,3.65E2,-1.6043508E-4,1.09E2,7.07E2,1.18399E5,1.2703817E-3,5.50423E9,2.486228E3,-5.601879E-4,2.8466056E9,1.3637682E3,6E0,1.6449123E-3,1.0426E4,1.7E1,8.4128E4,3.389E3,-8.9981727E-4,1.3821566E-3,-4.303313E-4,1.340911E-3,3.6743768E-3,1.1663035E9,3.9527593E-3,-1.9459268E-3,7.6024595E-4,2E1,-9.105164E-4,-1.4724095E-3,-2.7723602E-4,-7.4749836E-4,-1.7259028E-3,3.534921E-4,1.163E3,-2.2738131E-3,-9.916069E-4,3.300923E-4,-9.2979864E-4,1E0,2.5110933E-4,-2.7832135E-3,-8.877114E-4,8.453894E-4,8.060351E-5,-1.965322E-4,1.123963E-3,-4.657609E-5,8.9969E4,-2.7348063E-4,5.059508E-4,1.3918376E-4,1.1541E4,4.124764E-3,1.5444419E-3,1.1732741E-4,1.434173E-3,1.7786943E-3,-1.2812761E-3,-1.1488487E-3,-2.703874E-3,-1.1029177E-3,-1.8207383E-4,3.8528742E9,2.4064786E0,-2.1516204E-4,-1.1916177E-3,3.0512732E-4,1.8484124E3,-8.6015096E-4,-2.052025E-3,2.66887E5,-8.987849E-4,7.5541105E-4,1.7648065E10,-3.5180707E-5,-6.0971756E-4,2.1048128E-3,9.063207E-4],"split_indices":[47,6,16,9,5,28,12,18,8,5,0,64,10,0,64,4,3,18,18,19,0,0,0,17,0,8,0,16,1,63,20,12,28,16,6,18,12,27,3,11,10,0,0,19,0,0,1,17,2,4,0,4,0,6,0,11,5,0,10,5,19,0,3,17,8,0,0,0,0,0,0,11,0,0,0,17,0,0,0,0,0,0,17,0,0,0,0,42,0,0,0,0,0,0,0,0,8,0,0,0,7,0,0,0,0,0,0,0,0,0,0,11,67,0,0,0,5,0,0,12,0,0,21,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,2.27E2,5.9E1,1.69E2,5.8E1,2.5E1,3.4E1,7.5E1,9.4E1,5.5E1,3E0,8E0,1.7E1,9E0,2.5E1,4.1E1,3.4E1,5.5E1,3.9E1,5E1,5E0,5E0,3E0,1.4E1,3E0,6E0,3E0,1.6E1,9E0,2.4E1,1.7E1,2.3E1,1.1E1,4.5E1,1E1,2.6E1,1.3E1,3.8E1,1.2E1,8E0,6E0,3E0,3E0,1E1,6E0,3E0,6E0,1.6E1,8E0,1.2E1,5E0,1.3E1,1E1,8E0,3E0,3.8E1,7E0,4E0,6E0,9E0,1.7E1,4E0,9E0,3E1,8E0,9E0,3E0,5E0,3E0,3E0,3E0,6E0,4E0,3E0,3E0,1.3E1,3E0,5E0,3E0,4E0,8E0,4E0,9E0,4E0,6E0,3E0,5E0,3.3E1,5E0,3E0,4E0,3E0,3E0,6E0,3E0,5E0,1.2E1,4E0,5E0,4E0,2.6E1,5E0,3E0,3E0,6E0,3E0,3E0,3E0,1E1,6E0,3E0,1.2E1,2.1E1,5E0,7E0,3E0,2.3E1,9E0,3E0,1.6E1,5E0,5E0,1.8E1,1.3E1,3E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[3.8955659E-3,-8.514737E-3,9.313845E-2,-4.2649363E-3,-8.721153E-3,7.288031E-2,6.4413706E-3,-8.477663E-3,5.40013E-3,4.8945054E-2,1.3557605E-1,-1.32291205E-2,1.0402935E-1,3.5901397E-2,2.9398391E-3,5.930832E-3,1.3145645E-3,-2.1207834E-2,2.5041085E-2,6.3894703E-3,3.5185464E-2,1.6541997E-2,2.5209212E-3,3.715608E-3,-2.3944022E-2,-3.7356848E-3,3.9234314E-2,3.9287744E-4,1.5891436E-3,2.0042225E-3,1.4554786E-3,-1.9989863E-2,-8.719065E-2,6.200825E-2,1.5637355E-2,-7.364705E-4,1.9492064E-2,-2.9244773E-2,-1.6054444E-3,-1.0175737E-2,-7.062992E-3,6.934094E-2,5.528595E-4,-1.6054396E-2,3.0366275E-2,9.905302E-4,1.0747102E-4,1.0364182E-2,-3.537913E-2,6.81642E-2,-1.3203989E-2,2.344343E-4,-7.1162166E-4,1.1674331E-3,7.734287E-2,1.0823463E-4,-1.0125897E-3,-2.5459974E-6,4.2552333E-2,1.6347736E-3,-1.0710798E-2,-3.1128462E-2,-2.9112194E-3,5.1493663E-4,2.9479112E-3,-6.94562E-3,-4.004625E-2,2.8008344E-3,1.1668818E-3,6.5861613E-4,1.9490913E-3,5.836292E-3,-9.087138E-4,-3.9178964E-2,-1.170847E-3,-1.44539075E-2,7.56652E-3,-5.7790155E-4,-2.21129E-3,7.568056E-4,-4.280433E-4,-4.2291082E-2,-1.5716149E-3,1.5077718E-3,-1.44425E-2,-9.375137E-3,-1.4524342E-3,1.6694305E-2,-2.1096983E-4,-4.5235153E-2,-2.7332033E-4,-1.0803752E-3,9.918449E-4,-7.4383785E-4,-3.5862707E-2,-2.0285454E-2,3.9348537E-3,1.8510927E-4,8.526135E-4,-1.5434633E-3,-6.2955543E-4,3.97211E-4,-3.8151236E-4,-1.6524012E-3,-3.6776936E-4,-3.294387E-4,-9.7565237E-4,9.306841E-4,-4.9219246E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":108,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,17,19,21,-1,-1,-1,23,25,-1,27,29,-1,-1,31,-1,33,-1,-1,35,-1,37,39,41,43,-1,45,47,49,51,-1,53,-1,55,57,-1,-1,59,61,63,65,-1,-1,-1,67,-1,-1,-1,69,-1,71,73,-1,-1,-1,75,77,-1,-1,-1,-1,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,-1,93,95,-1,97,-1,99,-1,-1,-1,101,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.17547E-1,2.7988467E-1,6.1263174E-2,1.8187463E-1,0E0,3.988403E-2,0E0,1.3181186E-1,0E0,9.989824E-3,2.9365584E-2,7.228762E-2,5.8052182E-2,1.5127044E-2,0E0,0E0,0E0,7.56685E-2,8.66602E-2,0E0,1.6790405E-3,6.162372E-3,0E0,0E0,4.627382E-2,0E0,1.9451398E-2,0E0,0E0,5.2327784E-3,0E0,3.0919649E-2,1.0978955E-1,5.055189E-3,9.823218E-3,0E0,1.1727572E-3,2.9766701E-2,5.1072072E-2,1.8275442E-3,0E0,1.4292598E-3,0E0,2.1993592E-3,5.1974943E-3,0E0,0E0,1.5026358E-2,2.1760166E-2,9.529378E-3,8.793248E-3,0E0,0E0,0E0,1.6137287E-3,0E0,0E0,0E0,2.7373359E-3,0E0,3.6435835E-3,2.3830332E-2,0E0,0E0,0E0,5.012625E-3,5.0964504E-3,0E0,0E0,0E0,0E0,2.6800258E-3,0E0,9.0625435E-3,1.4694909E-2,4.249678E-3,2.1716566E-3,0E0,0E0,0E0,0E0,6.7805797E-3,8.2865665E-3,0E0,5.2550314E-3,4.000658E-3,0E0,9.1943354E-4,0E0,3.6631376E-3,0E0,0E0,0E0,1.8903454E-3,2.0601023E-3,1.2011533E-3,6.68499E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,20,20,21,21,24,24,26,26,29,29,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,43,43,44,44,47,47,48,48,49,49,50,50,54,54,58,58,60,60,61,61,65,65,66,66,71,71,73,73,74,74,75,75,76,76,81,81,82,82,84,84,85,85,87,87,89,89,93,93,94,94,95,95,96,96],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,18,20,22,-1,-1,-1,24,26,-1,28,30,-1,-1,32,-1,34,-1,-1,36,-1,38,40,42,44,-1,46,48,50,52,-1,54,-1,56,58,-1,-1,60,62,64,66,-1,-1,-1,68,-1,-1,-1,70,-1,72,74,-1,-1,-1,76,78,-1,-1,-1,-1,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,-1,94,96,-1,98,-1,100,-1,-1,-1,102,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.117756E9,6.7975234E10,1.6189593E4,8.951542E3,-8.721153E-3,4.0176924E9,6.4413706E-3,1.8403553E9,5.40013E-3,1.4864444E1,5.105882E3,2.236864E6,1.92E3,1.2779106E7,2.9398391E-3,5.930832E-3,1.3145645E-3,1.949E3,6.162E3,6.3894703E-3,2.3E1,8.0976E4,2.5209212E-3,3.715608E-3,5.178273E8,-3.7356848E-3,1.4806E4,3.9287744E-4,1.5891436E-3,1.07E2,1.4554786E-3,2.1201018E8,1.864E3,6.21E2,4.01145E5,-7.364705E-4,2.1649884E9,2.1124447E-6,9.6671375E2,5.9170656E8,-7.062992E-3,4.618673E6,5.528595E-4,4.8655386E8,2.865148E-1,9.905302E-4,1.0747102E-4,1E0,1.14153E5,1.7136614E8,5.577429E-1,2.344343E-4,-7.1162166E-4,1.1674331E-3,1.9E2,1.0823463E-4,-1.0125897E-3,-2.5459974E-6,2.878E3,1.6347736E-3,2.54E2,4.695E5,-2.9112194E-3,5.1493663E-4,2.9479112E-3,1.4327996E9,3.4667E4,2.8008344E-3,1.1668818E-3,6.5861613E-4,1.9490913E-3,2.25E2,-9.087138E-4,4.579412E8,4.3016243E9,1.511511E8,1.1087451E9,-5.7790155E-4,-2.21129E-3,7.568056E-4,-4.280433E-4,1.18E3,4.47E2,1.5077718E-3,5.324E3,1.358313E6,-1.4524342E-3,8.3495496E7,-2.1096983E-4,3.8909322E-1,-2.7332033E-4,-1.0803752E-3,9.918449E-4,1.864E3,2.25E3,1.12297E5,1.3798331E10,1.8510927E-4,8.526135E-4,-1.5434633E-3,-6.2955543E-4,3.97211E-4,-3.8151236E-4,-1.6524012E-3,-3.6776936E-4,-3.294387E-4,-9.7565237E-4,9.306841E-4,-4.9219246E-4],"split_indices":[27,23,5,5,0,16,0,16,0,60,5,6,20,18,0,0,0,7,20,0,19,8,0,0,27,0,3,0,0,4,0,26,0,17,12,0,1,63,5,27,0,2,0,16,63,0,0,45,8,10,63,0,0,0,17,0,0,0,0,0,17,6,0,0,0,26,28,0,0,0,0,4,0,16,10,27,21,0,0,0,0,17,0,0,3,9,0,27,0,63,0,0,0,0,0,12,10,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,2.51E2,3.4E1,2.48E2,3E0,3E1,4E0,2.43E2,5E0,2.3E1,7E0,2.34E2,9E0,1.9E1,4E0,4E0,3E0,1.94E2,4E1,3E0,6E0,1.4E1,5E0,3E0,1.91E2,3E0,3.7E1,3E0,3E0,1E1,4E0,1.81E2,1E1,1.8E1,1.9E1,4E0,6E0,1.2E2,6.1E1,7E0,3E0,1.5E1,3E0,6E0,1.3E1,3E0,3E0,1.6E1,1.04E2,8E0,5.3E1,3E0,4E0,4E0,1.1E1,3E0,3E0,4E0,9E0,5E0,1.1E1,9.8E1,6E0,3E0,5E0,4.4E1,9E0,8E0,3E0,5E0,4E0,6E0,5E0,7.7E1,2.1E1,2.9E1,1.5E1,6E0,3E0,3E0,3E0,7.1E1,6E0,4E0,1.7E1,2.6E1,3E0,9E0,6E0,6.5E1,6E0,3E0,3E0,1.1E1,6E0,1.4E1,1.2E1,5E0,4E0,5.8E1,7E0,5E0,6E0,3E0,3E0,8E0,6E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[3.7827337E-4,-1.8536512E-2,7.499882E-2,-1.4947045E-2,-6.628958E-3,2.055577E-1,4.4986043E-2,-2.9035471E-2,4.9014755E-2,8.824692E-5,1.0954275E-2,-3.4546554E-3,7.6249875E-2,-2.4549337E-2,-4.8943656E-3,3.0286534E-2,4.730883E-3,-4.0501997E-2,2.7170865E-2,4.8924815E-2,1.0860622E-1,-2.145752E-2,-9.392082E-2,5.415652E-2,8.591465E-3,-4.9316086E-4,-2.193622E-3,4.376162E-2,-4.1212686E-4,3.0548614E-3,2.1858146E-2,4.2800503E-3,8.5765E-4,-4.4555567E-2,-1.0733345E-2,-4.6902215E-3,-6.004026E-4,-1.4784254E-4,7.412243E-2,-1.5220648E-2,4.32357E-2,1.9903379E-3,6.6155527E-4,-3.4721495E-4,3.3938315E-2,-1.9222897E-3,-3.6157444E-2,4.1626252E-2,-1.6729284E-2,3.953809E-3,1.4239834E-3,-5.2755564E-5,-1.3511104E-3,6.310342E-4,1.7219122E-3,6.040703E-4,1.5518824E-3,-2.7462395E-2,-1.9053037E-3,4.849191E-3,2.9660293E-3,1.8883822E-3,-2.1174585E-2,-1.6373095E-5,-3.193349E-2,-7.906241E-4,8.822439E-4,1.813882E-3,-2.507303E-2,-3.678156E-2,4.8356203E-5,-4.4689556E-3,-1.7121961E-2,-1.5315156E-3,-2.471864E-2,-2.0727713E-2,2.0820522E-3,-1.2440898E-2,-1.1241317E-3,-3.3259407E-2,-1.1475385E-2,3.5540597E-5,-7.3634484E-4,-2.8952716E-3,-4.4826563E-2,-5.5733663E-3,-1.0439074E-3,-5.324241E-4,5.40363E-4,-1.1702823E-3,-2.6498595E-3,-1.290496E-3,-5.7722405E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":109,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,-1,23,-1,25,27,29,31,33,35,37,39,-1,-1,41,-1,-1,43,-1,-1,45,47,-1,-1,-1,49,51,53,-1,-1,-1,55,-1,57,59,61,-1,-1,-1,-1,-1,-1,-1,-1,63,-1,65,-1,-1,67,-1,69,-1,-1,-1,71,73,-1,-1,75,-1,77,79,-1,81,-1,83,85,-1,-1,87,89,91,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8388816E-1,1.4289223E-1,2.0626667E-1,1.9426309E-1,0E0,2.7236956E-1,6.992772E-2,9.339541E-2,6.709692E-2,0E0,0E0,2.265776E-2,1.9442365E-2,3.4661435E-2,0E0,1.7141992E-2,0E0,5.221409E-3,7.855523E-3,1.9082136E-2,2.0416528E-2,4.039351E-2,2.4568263E-2,1.9108232E-2,1.6332172E-2,0E0,0E0,2.0845858E-3,0E0,0E0,5.098359E-3,0E0,0E0,4.462585E-3,3.648611E-2,0E0,0E0,0E0,1.2033619E-2,4.2915395E-3,8.906424E-4,0E0,0E0,0E0,1.0442259E-3,0E0,5.5845417E-3,2.184804E-2,3.5762943E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3581164E-3,0E0,5.981112E-3,0E0,0E0,3.1031366E-2,0E0,3.945628E-3,0E0,0E0,0E0,8.264865E-2,1.5591122E-3,0E0,0E0,2.7592637E-2,0E0,1.0689511E-3,9.589806E-3,0E0,1.0146346E-3,0E0,1.2514375E-2,6.1268364E-3,0E0,0E0,3.2319804E-3,5.2642226E-3,5.2778744E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,30,30,33,33,34,34,38,38,39,39,40,40,44,44,46,46,47,47,48,48,57,57,59,59,62,62,64,64,68,68,69,69,72,72,74,74,75,75,77,77,79,79,80,80,83,83,84,84,85,85],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,-1,24,-1,26,28,30,32,34,36,38,40,-1,-1,42,-1,-1,44,-1,-1,46,48,-1,-1,-1,50,52,54,-1,-1,-1,56,-1,58,60,62,-1,-1,-1,-1,-1,-1,-1,-1,64,-1,66,-1,-1,68,-1,70,-1,-1,-1,72,74,-1,-1,76,-1,78,80,-1,82,-1,84,86,-1,-1,88,90,92,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0737418E9,2.0788145E0,1.571E4,1.4758104E9,-6.628958E-3,7.6E1,6.0958E5,1.11386E5,6.0919204E3,8.824692E-5,1.0954275E-2,2.9138386E7,7.4074E4,1.33097E5,-4.8943656E-3,1.01E2,4.730883E-3,9.35E2,1.531E3,4.1756818E9,2.54487E5,5.99609E5,6.909286E6,1.038E3,1.0482055E10,-4.9316086E-4,-2.193622E-3,2.839147E6,-4.1212686E-4,3.0548614E-3,1.4867069E0,4.2800503E-3,8.5765E-4,5.111E3,6.352906E2,-4.6902215E-3,-6.004026E-4,-1.4784254E-4,1.425E4,8.58437E6,5.1889E4,1.9903379E-3,6.6155527E-4,-3.4721495E-4,1.6760237E1,-1.9222897E-3,1.639929E8,2.219E3,2.9169E4,3.953809E-3,1.4239834E-3,-5.2755564E-5,-1.3511104E-3,6.310342E-4,1.7219122E-3,6.040703E-4,1.5518824E-3,4.9418E4,-1.9053037E-3,1.4064E4,2.9660293E-3,1.8883822E-3,3.342E3,-1.6373095E-5,2.0230963E8,-7.906241E-4,8.822439E-4,1.813882E-3,1.3708E4,1.3249E4,4.8356203E-5,-4.4689556E-3,4.5448714E9,-1.5315156E-3,1.0816216E3,1.14928E5,2.0820522E-3,5.02E2,-1.1241317E-3,1.0866E4,3.03125E5,3.5540597E-5,-7.3634484E-4,3.8348E4,2.8304275E8,7.033241E8,-1.0439074E-3,-5.324241E-4,5.40363E-4,-1.1702823E-3,-2.6498595E-3,-1.290496E-3,-5.7722405E-5],"split_indices":[16,59,8,21,0,17,12,28,5,0,0,18,8,8,0,19,0,0,17,11,20,9,26,0,21,0,0,2,0,0,64,0,0,22,5,0,0,0,7,2,8,0,0,0,60,0,23,28,12,0,0,0,0,0,0,0,0,8,0,7,0,0,7,0,16,0,0,0,8,7,0,0,23,0,5,12,0,17,0,28,12,0,0,18,21,10,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,2.16E2,5.4E1,2.13E2,3E0,9E0,4.5E1,1.75E2,3.8E1,4E0,5E0,1.8E1,2.7E1,1.7E2,5E0,3.3E1,5E0,8E0,1E1,1.6E1,1.1E1,1.64E2,6E0,1.5E1,1.8E1,5E0,3E0,7E0,3E0,5E0,1.1E1,8E0,3E0,5.1E1,1.13E2,3E0,3E0,4E0,1.1E1,1.1E1,7E0,3E0,4E0,3E0,8E0,1.6E1,3.5E1,1.1E1,1.02E2,3E0,8E0,8E0,3E0,3E0,4E0,5E0,3E0,2.7E1,8E0,7E0,4E0,5E0,9.7E1,4E0,2.3E1,3E0,4E0,4E0,9.3E1,2E1,3E0,5E0,8.8E1,9E0,1.1E1,8.5E1,3E0,6E0,5E0,3.5E1,5E1,3E0,3E0,1E1,2.5E1,4E1,1E1,6E0,4E0,2.2E1,3E0,3E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[1.0093641E-2,-8.234773E-3,7.843437E-2,-3.101211E-2,1.1118898E-2,6.671049E-2,6.3219047E-3,-2.662513E-2,-4.3496005E-3,-1.8744504E-3,6.8258144E-2,1.0548209E-1,4.4548847E-2,1.6694238E-3,-3.0357953E-2,-7.3586754E-3,2.5979538E-2,1.5275998E-1,-5.6870252E-2,6.566188E-2,6.8179537E-3,-1.2225978E-2,5.7763353E-2,-4.192589E-2,-1.4586736E-2,-1.2359155E-2,2.785907E-2,-3.4438955E-4,3.808824E-2,7.0644857E-4,1.8644111E-1,1.3965438E-4,-3.167369E-3,2.9419892E-4,7.8002386E-2,9.6515944E-4,-1.4017939E-3,3.184183E-2,9.120532E-2,-1.4882497E-4,-4.4706393E-2,-5.767506E-3,-2.189195E-3,-8.254318E-3,-5.1641945E-2,1.5951947E-3,4.8855203E-5,5.1203094E-2,-1.3586828E-4,8.103184E-3,2.8992817E-3,8.8975E-2,1.0001194E-3,5.20233E-2,-1.0186876E-2,1.04892716E-1,1.1003957E-3,-5.4162946E-2,-2.9349139E-2,-2.5570031E-2,6.8493853E-3,-1.2449245E-2,1.5377497E-2,-2.4772172E-3,-4.3180882E-4,8.5344684E-4,2.0357517E-3,1.5872021E-3,3.450686E-3,7.8095304E-4,2.586977E-3,6.2992907E-4,-1.2037627E-3,1.6558077E-3,3.8772349E-3,-3.033807E-4,-5.8619693E-2,-3.6570333E-2,8.101913E-5,-1.8685734E-3,-1.2612496E-2,-1.1735036E-2,2.096219E-2,4.052567E-4,-1.4954441E-2,-1.2563212E-3,1.1455839E-3,-2.3437357E-3,-4.6690095E-2,-1.5008036E-3,-4.5417904E-4,9.213704E-6,-9.891015E-4,-2.6460376E-2,3.613998E-4,3.926485E-2,7.490655E-5,-3.0185161E-2,-9.017483E-3,-4.1918686E-4,3.4841744E-4,-5.341697E-4,-5.160283E-2,-1.9682829E-4,-1.2937027E-3,1.886669E-3,3.251466E-4,-1.160935E-3,-1.6080357E-4,5.763403E-3,-1.5332451E-2,-6.781006E-4,-1.8502682E-3,-3.0540018E-3,5.330291E-4,1.152374E-4,-1.9421719E-2,3.2182573E-4,-4.1474882E-4,7.697767E-5,-2.390392E-2,-1.0971703E-3,-1.2475038E-2,-4.9927265E-5,-6.862633E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":110,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,41,43,45,-1,47,-1,49,-1,-1,-1,51,-1,-1,53,55,-1,57,59,-1,61,63,-1,-1,65,-1,-1,-1,67,-1,69,71,73,-1,75,77,79,81,83,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,87,89,-1,-1,91,93,95,-1,97,99,-1,-1,101,-1,-1,-1,-1,103,-1,105,-1,107,109,-1,-1,-1,111,-1,-1,-1,-1,-1,-1,113,115,-1,-1,117,-1,-1,119,-1,-1,-1,121,-1,123,-1,-1],"loss_changes":[3.6138812E-1,1.00852944E-1,7.126188E-2,4.436101E-2,9.2189275E-2,4.4277698E-2,0E0,3.145875E-2,0E0,1.5772443E-2,2.518798E-1,7.077241E-2,2.921576E-2,0E0,1.72676E-2,1.5489399E-2,7.957707E-3,5.759123E-2,2.5846317E-2,1.0447264E-2,0E0,1.1882942E-2,2.3968726E-2,5.6015924E-3,1.986757E-2,1.1828694E-2,6.211913E-3,0E0,7.429553E-3,0E0,4.6545357E-2,0E0,0E0,0E0,3.9301068E-3,0E0,0E0,1.6645776E-2,6.5600723E-3,0E0,5.9621483E-3,9.669299E-3,0E0,7.1308017E-3,5.7420954E-3,0E0,0E0,1.2959559E-3,0E0,0E0,0E0,1.9435585E-3,0E0,8.279994E-3,6.3995593E-3,2.1208897E-3,0E0,5.628459E-3,5.154498E-3,5.5807745E-3,6.5406715E-3,3.9519574E-3,3.7429137E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3409778E-3,3.4070536E-3,0E0,0E0,2.8127958E-3,4.0165163E-3,4.683634E-3,0E0,4.701134E-3,1.1359606E-3,0E0,0E0,1.8049628E-3,0E0,0E0,0E0,0E0,1.7097294E-3,0E0,3.3576442E-3,0E0,2.0419545E-3,3.9153406E-3,0E0,0E0,0E0,1.4817752E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.2880745E-3,2.3123375E-3,0E0,0E0,1.1553182E-3,0E0,0E0,2.4036672E-3,0E0,0E0,0E0,1.9020196E-3,0E0,1.0297546E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,34,34,37,37,38,38,40,40,41,41,43,43,44,44,47,47,51,51,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,76,76,77,77,80,80,81,81,82,82,84,84,85,85,88,88,93,93,95,95,97,97,98,98,102,102,109,109,110,110,113,113,116,116,120,120,122,122],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,42,44,46,-1,48,-1,50,-1,-1,-1,52,-1,-1,54,56,-1,58,60,-1,62,64,-1,-1,66,-1,-1,-1,68,-1,70,72,74,-1,76,78,80,82,84,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,88,90,-1,-1,92,94,96,-1,98,100,-1,-1,102,-1,-1,-1,-1,104,-1,106,-1,108,110,-1,-1,-1,112,-1,-1,-1,-1,-1,-1,114,116,-1,-1,118,-1,-1,120,-1,-1,-1,122,-1,124,-1,-1],"split_conditions":[7.762619E6,5.013E3,4.48E2,1.78538E5,1.83E2,2.8125507E-1,6.3219047E-3,3.185029E2,-4.3496005E-3,6.2498252E7,1.2583364E10,7.684693E-1,1.306E4,1.6694238E-3,3.067249E8,3.636759E6,1E0,2.61029E5,1.7501315E10,6.1346E4,6.8179537E-3,5.5956E4,7.0231123E3,1.5E1,1E0,5.559532E6,2.318509E6,-3.4438955E-4,1.49E2,7.0644857E-4,2.1114904E8,1.3965438E-4,-3.167369E-3,2.9419892E-4,1.5831415E-1,9.6515944E-4,-1.4017939E-3,7.7433E4,5.121E4,-1.4882497E-4,7.47E2,3.584604E7,-2.189195E-3,3.25559E5,2.259E3,1.5951947E-3,4.8855203E-5,9.19E2,-1.3586828E-4,8.103184E-3,2.8992817E-3,3.4958506E-1,1.0001194E-3,5.315E3,9.1883635E8,1.373E3,1.1003957E-3,7E0,4.71E2,2.16E2,1.8622E4,1.37839E5,1.15E2,-2.4772172E-3,-4.3180882E-4,8.5344684E-4,2.0357517E-3,1.5872021E-3,3.450686E-3,7.8095304E-4,2.586977E-3,6.2992907E-4,-1.2037627E-3,1.6558077E-3,3.8772349E-3,-3.033807E-4,2.876E3,1E0,8.101913E-5,-1.8685734E-3,4.462E3,4.22E2,1.0242101E7,4.052567E-4,4.6585E5,8.4E1,1.1455839E-3,-2.3437357E-3,3.049E3,-1.5008036E-3,-4.5417904E-4,9.213704E-6,-9.891015E-4,6.23069E6,3.613998E-4,7.6E1,7.490655E-5,4.174378E7,6.5878E4,-4.1918686E-4,3.4841744E-4,-5.341697E-4,3.9E1,-1.9682829E-4,-1.2937027E-3,1.886669E-3,3.251466E-4,-1.160935E-3,-1.6080357E-4,3.189534E3,5.324E3,-6.781006E-4,-1.8502682E-3,1.563477E6,5.330291E-4,1.152374E-4,5.92953E5,3.2182573E-4,-4.1474882E-4,7.697767E-5,2.21839E5,-1.0971703E-3,5.788302E8,-4.9927265E-5,-6.862633E-4],"split_indices":[6,3,4,22,47,63,0,5,0,18,10,64,7,0,11,6,42,6,10,8,0,3,5,28,31,2,1,0,19,0,21,0,0,0,63,0,0,8,19,0,17,16,0,12,0,0,0,0,0,0,0,64,0,0,16,0,0,17,4,4,22,6,4,0,0,0,0,0,0,0,0,0,0,0,0,0,3,43,0,0,3,19,21,0,6,19,0,0,3,0,0,0,0,21,0,4,0,27,8,0,0,0,4,0,0,0,0,0,0,5,3,0,0,2,0,0,6,0,0,0,12,0,11,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.87E2,2.27E2,6E1,1.04E2,1.23E2,5.6E1,4E0,1.01E2,3E0,1.01E2,2.2E1,1.9E1,3.7E1,4E0,9.7E1,8.5E1,1.6E1,1.3E1,9E0,1.5E1,4E0,7E0,3E1,5.5E1,4.2E1,7.5E1,1E1,4E0,1.2E1,3E0,1E1,4E0,5E0,3E0,1.2E1,3E0,4E0,1.8E1,1.2E1,4E0,5.1E1,3.7E1,5E0,6.9E1,6E0,5E0,5E0,9E0,3E0,5E0,5E0,9E0,3E0,1.2E1,6E0,9E0,3E0,3E1,2.1E1,1.4E1,2.3E1,5.9E1,1E1,3E0,3E0,4E0,5E0,4E0,5E0,7E0,5E0,3E0,3E0,3E0,6E0,3E0,2.7E1,1.7E1,4E0,3E0,1.1E1,1E1,1.3E1,5E0,5.4E1,6E0,4E0,1E1,1.7E1,1.1E1,6E0,7E0,4E0,6E0,4E0,6E0,7E0,1.4E1,4E1,3E0,3E0,3E0,1.4E1,3E0,3E0,3E0,3E0,1.1E1,3E0,1.2E1,2.8E1,3E0,1.1E1,7E0,5E0,5E0,2.3E1,3E0,4E0,4E0,1.9E1,9E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[5.957761E-3,-1.1010423E-2,8.457098E-2,5.3887595E-3,-1.5677882E-2,6.1221633E-2,1.9950235E-1,-2.6030503E-2,1.5655031E-2,5.4001855E-3,4.903674E-2,7.96978E-3,2.881321E-3,-1.21582195E-1,-1.9340446E-2,5.927068E-2,-2.8113355E-3,7.630132E-2,1.79571E-2,-4.0911295E-2,-8.128628E-3,-2.4412611E-2,3.1656135E-2,3.2848772E-3,6.154626E-4,-3.1649247E-2,4.4225976E-2,4.719976E-4,8.5759684E-2,-7.975631E-3,3.5910223E-2,-2.5833754E-3,5.954353E-4,3.4871746E-2,-2.7182668E-2,6.562422E-2,-9.731807E-4,-8.844546E-2,1.589768E-2,9.711054E-5,5.771093E-2,3.6589846E-3,5.206261E-2,6.944921E-4,-1.1566011E-3,1.735615E-3,4.9389474E-4,1.07771884E-4,1.8565771E-3,-3.3501443E-2,-4.6124184E-3,3.1050143E-3,-1.5283683E-4,-9.092765E-3,-5.352467E-3,-3.7273194E-4,1.9005444E-3,2.2254242E-3,8.9769304E-4,3.5380252E-4,6.8094335E-2,-6.0861986E-2,-2.8540706E-2,-2.5706815E-2,1.1173579E-2,-9.793495E-4,4.6714785E-4,1.0887218E-3,2.7470784E-3,-9.765276E-4,-2.6147915E-3,-3.578918E-2,-1.5667118E-2,-6.366006E-3,-1.4207625E-3,-5.8050087E-4,2.289307E-2,-2.0042067E-2,-4.7241613E-2,9.406907E-3,-2.3167087E-2,-4.703953E-4,1.7816387E-4,4.0314645E-2,-5.3204165E-4,4.208051E-3,-2.8740013E-2,-6.1563578E-2,-3.1090785E-2,-1.3268764E-4,8.803281E-4,2.3748474E-4,-2.712759E-2,1.6954925E-3,5.60782E-4,5.788807E-4,-2.463801E-4,-3.839628E-2,-9.456076E-3,-9.0260117E-4,-2.1484245E-3,1.4559335E-4,-3.85151E-2,-3.0116564E-5,-3.0656695E-2,-4.2419494E-4,-1.4370392E-3,1.3360578E-4,-7.757697E-4,-1.6947332E-3,-2.74847E-4,-1.2114375E-3,-1.6394963E-2,-5.9766007E-5,-7.517612E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":111,"left_children":[1,3,5,-1,7,9,11,13,15,-1,17,-1,-1,19,21,23,25,27,29,31,-1,33,35,-1,-1,37,39,-1,41,43,45,-1,-1,47,49,51,-1,53,55,-1,57,-1,59,-1,-1,-1,-1,-1,-1,61,63,-1,-1,65,-1,-1,-1,-1,-1,-1,67,69,71,73,75,-1,-1,-1,-1,-1,-1,77,79,81,-1,-1,83,85,87,89,91,-1,-1,93,-1,95,97,99,101,-1,-1,-1,103,-1,-1,-1,-1,105,107,-1,-1,-1,109,-1,111,-1,-1,-1,-1,-1,-1,-1,113,-1,-1],"loss_changes":[3.7576368E-1,1.9582745E-1,1.2196478E-1,0E0,7.4041255E-2,4.7596842E-2,2.0153522E-2,1.06959656E-1,4.6075713E-2,0E0,3.252478E-2,0E0,0E0,1.0586783E-1,4.2417563E-2,2.724795E-2,5.7014488E-2,1.098536E-2,9.597878E-3,2.0184435E-2,0E0,2.50884E-2,3.351806E-2,0E0,0E0,7.1618706E-2,8.717135E-3,0E0,1.1012092E-2,8.210413E-3,3.617377E-3,0E0,0E0,4.8393235E-3,1.9954383E-2,2.2159345E-2,0E0,7.260716E-2,1.8466517E-2,0E0,2.1607243E-3,0E0,5.835969E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.3193078E-2,1.0943112E-2,0E0,0E0,3.966448E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.2512691E-3,7.484492E-3,8.304112E-3,4.709705E-3,6.9165537E-3,0E0,0E0,0E0,0E0,0E0,0E0,9.87374E-3,6.9878893E-3,8.661145E-4,0E0,0E0,1.0281401E-2,5.8159465E-3,5.9505254E-3,2.2973623E-3,3.587393E-3,0E0,0E0,1.9115843E-3,0E0,1.444943E-3,3.3611394E-3,1.0808036E-3,4.9335603E-3,0E0,0E0,0E0,2.2821743E-3,0E0,0E0,0E0,0E0,1.4598407E-3,1.694136E-3,0E0,0E0,0E0,5.9059653E-3,0E0,1.5725307E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.672683E-4,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,28,28,29,29,30,30,33,33,34,34,35,35,37,37,38,38,40,40,42,42,49,49,50,50,53,53,60,60,61,61,62,62,63,63,64,64,71,71,72,72,73,73,76,76,77,77,78,78,79,79,80,80,83,83,85,85,86,86,87,87,88,88,92,92,97,97,98,98,102,102,104,104,112,112],"right_children":[2,4,6,-1,8,10,12,14,16,-1,18,-1,-1,20,22,24,26,28,30,32,-1,34,36,-1,-1,38,40,-1,42,44,46,-1,-1,48,50,52,-1,54,56,-1,58,-1,60,-1,-1,-1,-1,-1,-1,62,64,-1,-1,66,-1,-1,-1,-1,-1,-1,68,70,72,74,76,-1,-1,-1,-1,-1,-1,78,80,82,-1,-1,84,86,88,90,92,-1,-1,94,-1,96,98,100,102,-1,-1,-1,104,-1,-1,-1,-1,106,108,-1,-1,-1,110,-1,112,-1,-1,-1,-1,-1,-1,-1,114,-1,-1],"split_conditions":[3.3333E4,1.0916E4,2.6339598E4,5.3887595E-3,3.6815E6,9.068E3,5.3408066E4,1.571E4,1.5581422E10,5.4001855E-3,1.95E2,7.96978E-3,2.881321E-3,8.109E3,1.83E2,1.1988E4,6.177E4,1.4E1,2.029183E0,1.501E4,-8.128628E-3,1E0,2.839147E6,3.2848772E-3,6.154626E-4,1.33E2,5.710497E6,4.719976E-4,1.99E2,6.7E1,4.90364E5,-2.5833754E-3,5.954353E-4,4E0,2.9138386E7,4.9711E4,-9.731807E-4,4.59516E5,1.600673E7,9.711054E-5,1.1059E4,3.6589846E-3,6.1346E4,6.944921E-4,-1.1566011E-3,1.735615E-3,4.9389474E-4,1.07771884E-4,1.8565771E-3,2.9510102E-4,7.164074E8,3.1050143E-3,-1.5283683E-4,5.480687E6,-5.352467E-3,-3.7273194E-4,1.9005444E-3,2.2254242E-3,8.9769304E-4,3.5380252E-4,3.2842822E3,3.6267E4,1.10686E5,5.3566998E8,1E0,-9.793495E-4,4.6714785E-4,1.0887218E-3,2.7470784E-3,-9.765276E-4,-2.6147915E-3,8.120952E-3,8.5E1,1.3299E4,-1.4207625E-3,-5.8050087E-4,1.207E3,6.0216E4,4.6553096E-1,2.42045E6,2.941625E6,-4.703953E-4,1.7816387E-4,1.554077E9,-5.3204165E-4,1.06095E5,2.963893E9,2.6E1,1.034684E6,-1.3268764E-4,8.803281E-4,2.3748474E-4,5.478E5,1.6954925E-3,5.60782E-4,5.788807E-4,-2.463801E-4,1E0,9.62256E5,-9.0260117E-4,-2.1484245E-3,1.4559335E-4,7.47E2,-3.0116564E-5,4.72323E8,-4.2419494E-4,-1.4370392E-3,1.3360578E-4,-7.757697E-4,-1.6947332E-3,-2.74847E-4,-1.2114375E-3,8.13E2,-5.9766007E-5,-7.517612E-4],"split_indices":[3,8,5,0,2,7,5,8,10,0,19,0,0,20,47,7,20,17,67,18,0,38,2,0,0,4,2,0,17,4,18,0,0,4,18,8,0,12,2,0,3,0,8,0,0,0,0,0,0,67,16,0,0,6,0,0,0,0,0,0,5,8,12,10,42,0,0,0,0,0,0,63,4,7,0,0,0,8,67,2,21,0,0,25,0,6,10,17,18,0,0,0,6,0,0,0,0,19,2,0,0,0,17,0,27,0,0,0,0,0,0,0,17,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.31E2,4.9E1,5E0,2.26E2,4.2E1,7E0,1.7E2,5.6E1,3E0,3.9E1,4E0,3E0,1E1,1.6E2,1.6E1,4E1,2E1,1.9E1,7E0,3E0,1.46E2,1.4E1,7E0,9E0,2.5E1,1.5E1,3E0,1.7E1,8E0,1.1E1,4E0,3E0,6E0,1.4E2,9E0,5E0,1.1E1,1.4E1,4E0,1.1E1,8E0,9E0,4E0,4E0,5E0,6E0,3E0,3E0,1.09E2,3.1E1,6E0,3E0,6E0,5E0,9E0,5E0,7E0,4E0,3E0,6E0,1.5E1,9.4E1,1.3E1,1.8E1,3E0,3E0,3E0,3E0,7E0,8E0,5.9E1,3.5E1,7E0,6E0,5E0,1.3E1,2.6E1,3.3E1,8E0,2.7E1,4E0,3E0,9E0,4E0,7E0,1.9E1,1.6E1,1.7E1,5E0,3E0,3E0,2.4E1,5E0,4E0,3E0,4E0,1.2E1,7E0,3E0,1.3E1,3E0,1.4E1,3E0,2.1E1,3E0,9E0,4E0,3E0,9E0,5E0,1.3E1,8E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.9479435E-4,-1.1485901E-2,9.987449E-2,-8.159192E-3,-7.077078E-3,3.5788536E-2,2.3831122E-1,-2.31008E-2,2.9204259E-2,5.5844965E-3,5.239061E-3,1.0091114E-2,2.123057E-3,-1.7748818E-2,-8.26699E-3,9.3155146E-2,1.2450572E-2,2.722834E-3,-1.5855689E-2,-2.109248E-2,3.3099628E-3,5.38078E-3,3.9435346E-2,2.4515362E-2,-5.136364E-3,-4.4837825E-2,1.1295263E-3,-5.5714224E-2,-1.626571E-2,-1.9281087E-4,2.2763135E-3,5.1583692E-2,9.221255E-3,-1.99559E-3,-1.1592443E-4,-2.0902215E-3,-4.2545516E-4,-1.1332851E-2,-5.56423E-2,1.2768233E-2,2.868755E-3,1.9877456E-2,-1.7773172E-2,-1.8887281E-2,3.201525E-2,-1.05442494E-1,-1.0237913E-2,-3.566459E-4,2.30962E-2,-5.1948056E-4,2.8796462E-2,-1.0409084E-3,-7.941533E-6,-2.0631276E-2,1.1720126E-3,-2.4480356E-3,8.832409E-2,-1.8231893E-3,-4.5091817E-3,2.230979E-4,-9.26813E-4,1.0492627E-3,1.753069E-4,4.1997384E-2,5.0620832E-3,1.1001894E-3,-2.2388889E-2,-2.6831213E-2,2.0992903E-3,3.451805E-3,1.3713649E-3,4.4397774E-4,1.5590925E-3,8.1980095E-4,-8.630792E-4,-2.4102272E-2,6.257747E-4,-3.630093E-3,-2.1413916E-3,-2.7096013E-2,-3.535102E-3,-9.474129E-4,5.7097204E-4,-2.9106578E-2,4.801136E-4,6.576761E-4,-1.1229454E-2,-8.2587876E-4,-1.6693338E-3,2.6076898E-4,-5.978499E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":112,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,-1,25,27,-1,-1,29,31,-1,33,-1,35,37,-1,-1,39,41,-1,-1,-1,-1,43,45,47,-1,49,51,53,55,57,59,-1,61,-1,63,-1,-1,65,-1,67,69,-1,-1,-1,-1,-1,-1,71,73,-1,75,77,-1,-1,-1,-1,-1,-1,-1,79,-1,81,-1,83,85,-1,-1,87,-1,-1,89,-1,-1,-1,-1],"loss_changes":[3.2316664E-1,1.7635882E-1,2.487548E-1,1.4247525E-1,0E0,7.962687E-2,9.594959E-2,2.2310509E-1,7.705332E-2,3.183092E-2,0E0,0E0,0E0,7.433669E-2,0E0,5.3042635E-2,1.2635939E-1,0E0,2.3721315E-2,2.8338216E-2,0E0,0E0,1.463348E-2,2.2707544E-2,0E0,6.8521984E-3,0E0,7.3715076E-3,2.9476497E-2,0E0,0E0,2.8003644E-2,1.0996634E-2,0E0,0E0,0E0,0E0,4.6123043E-2,3.735999E-2,2.965214E-3,0E0,9.137838E-3,2.799881E-3,1.2227416E-2,4.0968563E-2,5.5905282E-3,3.3852253E-3,0E0,1.1639823E-3,0E0,6.6803508E-3,0E0,0E0,1.1940133E-2,0E0,2.4848552E-2,2.348557E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.8741656E-3,6.536937E-3,0E0,8.567285E-3,9.889014E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.615646E-3,0E0,4.9329577E-3,0E0,8.458488E-3,2.9955753E-3,0E0,0E0,5.811684E-3,0E0,0E0,1.8539621E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,18,18,19,19,22,22,23,23,25,25,27,27,28,28,31,31,32,32,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,48,48,50,50,53,53,55,55,56,56,63,63,64,64,66,66,67,67,75,75,77,77,79,79,80,80,83,83,86,86],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,-1,26,28,-1,-1,30,32,-1,34,-1,36,38,-1,-1,40,42,-1,-1,-1,-1,44,46,48,-1,50,52,54,56,58,60,-1,62,-1,64,-1,-1,66,-1,68,70,-1,-1,-1,-1,-1,-1,72,74,-1,76,78,-1,-1,-1,-1,-1,-1,-1,80,-1,82,-1,84,86,-1,-1,88,-1,-1,90,-1,-1,-1,-1],"split_conditions":[5.96896E5,7.550548E7,1.2738551E4,4.168507E6,-7.077078E-3,3.9932396E9,1.3887953E0,1.16215E5,1.619E3,6.72444E5,5.239061E-3,1.0091114E-2,2.123057E-3,1.1822208E10,-8.26699E-3,1.1026E4,1.33097E5,2.722834E-3,2.3177024E7,2.75E2,3.3099628E-3,5.38078E-3,1.6505978E9,1.95E2,-5.136364E-3,2.2491941E0,1.1295263E-3,5.72466E5,1.975E3,-1.9281087E-4,2.2763135E-3,7.0133E4,6.1679645E10,-1.99559E-3,-1.1592443E-4,-2.0902215E-3,-4.2545516E-4,1.83E2,3.0025E4,7.514E3,2.868755E-3,1.843E3,4.80415E5,1.5102771E9,2.0772083E-1,6.6362006E2,5.484E3,-3.566459E-4,1.858566E9,-5.1948056E-4,3.3730156E3,-1.0409084E-3,-7.941533E-6,4.08E5,1.1720126E-3,5.345E3,1.7869706E-1,-1.8231893E-3,-4.5091817E-3,2.230979E-4,-9.26813E-4,1.0492627E-3,1.753069E-4,2.5E1,5.1889E4,1.1001894E-3,8.753932E7,6.920602E7,2.0992903E-3,3.451805E-3,1.3713649E-3,4.4397774E-4,1.5590925E-3,8.1980095E-4,-8.630792E-4,5.11E2,6.257747E-4,3.4713E4,-2.1413916E-3,1.831E4,2.640421E6,-9.474129E-4,5.7097204E-4,4.5291E4,4.801136E-4,6.576761E-4,6.1501E4,-8.2587876E-4,-1.6693338E-3,2.6076898E-4,-5.978499E-4],"split_indices":[28,2,5,2,0,16,59,28,0,28,0,0,0,25,0,3,8,0,18,0,0,0,11,17,0,59,0,2,19,0,0,8,10,0,0,0,0,47,28,7,0,0,27,16,64,5,3,0,1,0,5,0,0,11,0,7,63,0,0,0,0,0,0,4,8,0,18,16,0,0,0,0,0,0,0,19,0,18,0,7,21,0,0,28,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.56E2,2.8E1,2.53E2,3E0,2E1,8E0,1.81E2,7.2E1,1.7E1,3E0,5E0,3E0,1.78E2,3E0,1.4E1,5.8E1,3E0,1.4E1,1.74E2,4E0,5E0,9E0,5.5E1,3E0,9E0,5E0,2E1,1.54E2,4E0,5E0,1.9E1,3.6E1,6E0,3E0,1.6E1,4E0,1.38E2,1.6E1,1E1,9E0,2.6E1,1E1,1.18E2,2E1,7E0,9E0,3E0,7E0,5E0,2.1E1,5E0,5E0,1.15E2,3E0,1.3E1,7E0,4E0,3E0,5E0,4E0,4E0,3E0,1.3E1,8E0,3E0,1.12E2,1E1,3E0,4E0,3E0,3E0,1E1,5E0,3E0,1.08E2,4E0,7E0,3E0,9.4E1,1.4E1,3E0,4E0,9E1,4E0,3E0,1.1E1,8E1,1E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[1.9100739E-3,-1.6406111E-2,7.252411E-2,-2.0805893E-2,1.17931105E-1,9.459907E-2,-7.6641967E-3,-4.3795276E-2,-9.728384E-3,6.5629706E-3,8.01687E-5,1.2653701E-2,6.882647E-2,-3.6111027E-2,-5.0766976E-3,2.6822877E-3,-1.3005799E-2,4.225685E-2,1.0656804E-1,-6.2455438E-2,-2.3014503E-2,-4.84815E-3,-7.812811E-3,1.4393205E-2,8.517486E-2,1.4375387E-1,3.2820046E-2,-8.005331E-2,-1.582351E-2,-2.951936E-2,1.0760386E-2,-1.38176E-2,3.468028E-2,4.745488E-2,-1.16024865E-2,3.2031508E-3,1.0179807E-3,1.6534844E-1,1.732699E-3,1.0612761E-4,1.9802267E-3,-2.8923864E-3,-1.3672479E-3,-1.0156232E-3,1.2427423E-4,-5.0368506E-2,-1.9159762E-2,-1.0891358E-3,1.4254935E-3,-1.8048722E-2,2.9365124E-2,6.2491443E-2,-2.2960017E-3,4.3331698E-4,2.0024069E-3,-2.8425746E-2,5.6765956E-4,6.2075783E-3,2.4427243E-3,-6.8921223E-4,-1.85543E-3,-3.0072788E-2,-5.0939824E-3,-1.5554752E-2,-2.1506199E-3,4.355728E-6,1.2975172E-3,7.54256E-2,3.637239E-4,4.5096964E-5,-1.5000699E-3,-1.552247E-2,-4.4484228E-2,2.4095293E-4,-5.4546865E-4,-2.8085716E-2,-7.088264E-3,2.7781667E-3,1.1203137E-3,-7.6610426E-4,2.4939072E-5,-6.168943E-4,-1.8889325E-3,4.8379484E-4,-3.178768E-2,-2.8664093E-3,-3.628295E-2,-2.1344607E-3,-2.3501426E-2,-1.3006447E-2,8.100288E-3,-4.3873573E-4,-1.517668E-3,2.1922463E-4,-2.9157514E-2,4.809521E-3,-2.8896013E-2,8.8701915E-4,1.1073358E-3,-1.3417234E-3,-1.5523485E-2,-2.7642353E-4,4.7301856E-4,-1.8557148E-3,-1.4506908E-2,1.251848E-2,-6.1509926E-3,2.0516698E-5,-1.0971319E-3,2.681398E-4,-8.6803077E-4,-3.4437468E-5,7.5975986E-4,-4.014043E-4,1.7402206E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":113,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,17,19,-1,-1,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,59,61,-1,-1,63,65,67,-1,-1,-1,69,-1,-1,-1,-1,-1,71,73,75,-1,-1,-1,77,-1,-1,-1,79,81,-1,-1,83,85,-1,-1,-1,-1,-1,-1,-1,87,89,91,-1,93,95,97,-1,-1,-1,99,101,103,105,-1,-1,107,-1,-1,-1,109,111,113,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5169575E-1,1.3012144E-1,4.0651712E-1,5.305361E-2,6.895326E-2,3.8068998E-1,0E0,5.383134E-2,4.4697203E-2,0E0,0E0,0E0,4.5635745E-2,2.0988941E-2,0E0,0E0,9.815328E-2,3.584548E-2,5.0129622E-2,1.5966348E-2,1.0204177E-2,0E0,3.492607E-2,1.7739816E-2,6.0652196E-3,1.634401E-2,6.5753683E-3,1.7089844E-3,2.2890312E-3,7.242523E-3,1.3459013E-2,2.227266E-2,5.457748E-2,3.676096E-3,6.403778E-3,0E0,0E0,9.96691E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2931675E-3,3.991316E-3,0E0,0E0,1.2216788E-2,3.513432E-3,8.417435E-3,0E0,0E0,0E0,4.402398E-3,0E0,0E0,0E0,0E0,0E0,2.4548601E-3,2.0650327E-3,1.0940714E-2,0E0,0E0,0E0,1.8587708E-3,0E0,0E0,0E0,1.2325798E-3,1.391829E-3,0E0,0E0,7.181477E-3,7.7843647E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.5980875E-3,6.443911E-3,1.3264949E-3,0E0,5.856337E-3,8.631617E-3,5.2815014E-3,0E0,0E0,0E0,4.1344706E-3,2.1127488E-3,5.7837255E-3,1.9598352E-3,0E0,0E0,4.52106E-3,0E0,0E0,0E0,3.6391881E-3,1.3808478E-3,1.1629165E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,12,12,13,13,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,45,45,46,46,49,49,50,50,51,51,55,55,61,61,62,62,63,63,67,67,71,71,72,72,75,75,76,76,84,84,85,85,86,86,88,88,89,89,90,90,94,94,95,95,96,96,97,97,100,100,104,104,105,105,106,106],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,18,20,-1,-1,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,60,62,-1,-1,64,66,68,-1,-1,-1,70,-1,-1,-1,-1,-1,72,74,76,-1,-1,-1,78,-1,-1,-1,80,82,-1,-1,84,86,-1,-1,-1,-1,-1,-1,-1,88,90,92,-1,94,96,98,-1,-1,-1,100,102,104,106,-1,-1,108,-1,-1,-1,110,112,114,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.898215E6,1.8403553E9,5.3408066E4,3.908E3,5.345E3,2.3880364E9,-7.6641967E-3,1.04345E5,1.6028E4,6.5629706E-3,8.01687E-5,1.2653701E-2,8.391737E10,3.4713E4,-5.0766976E-3,2.6822877E-3,4.796E3,7.4311E4,9.9181765E10,3.175E3,1.83E2,-4.84815E-3,1.83E2,2.1009742E10,1.3469999E10,2.4354641E0,6.258553E7,5.8636E4,2.8E1,2.578E4,1.782131E6,3.636759E6,1.9694447E10,2.414E3,1.12E2,3.2031508E-3,1.0179807E-3,9.762892E9,1.732699E-3,1.0612761E-4,1.9802267E-3,-2.8923864E-3,-1.3672479E-3,-1.0156232E-3,1.2427423E-4,4.3085E4,2.91E2,-1.0891358E-3,1.4254935E-3,5.575446E6,2.239676E10,1.7156E4,-2.2960017E-3,4.3331698E-4,2.0024069E-3,9.311E4,5.6765956E-4,6.2075783E-3,2.4427243E-3,-6.8921223E-4,-1.85543E-3,2.4607566E-3,8.97E2,4.09372E5,-2.1506199E-3,4.355728E-6,1.2975172E-3,1.0618027E9,3.637239E-4,4.5096964E-5,-1.5000699E-3,2.0582564E-3,1.2289E4,2.4095293E-4,-5.4546865E-4,1.18399E5,1.8208394E0,2.7781667E-3,1.1203137E-3,-7.6610426E-4,2.4939072E-5,-6.168943E-4,-1.8889325E-3,4.8379484E-4,6.408463E-5,3.228882E6,5.6932E4,-2.1344607E-3,7.318324E-4,5.324E3,1.49E3,-4.3873573E-4,-1.517668E-3,2.1922463E-4,2.379127E-2,4.618E3,7.3E1,2.2368E4,1.1073358E-3,-1.3417234E-3,1.0922E4,-2.7642353E-4,4.7301856E-4,-1.8557148E-3,1.52E2,1.267716E7,2.702002E6,2.0516698E-5,-1.0971319E-3,2.681398E-4,-8.6803077E-4,-3.4437468E-5,7.5975986E-4,-4.014043E-4,1.7402206E-4],"split_indices":[6,16,5,3,7,11,0,12,12,0,0,0,26,18,0,0,7,8,10,3,47,0,47,10,21,67,6,8,19,12,27,6,10,0,4,0,0,11,0,0,0,0,0,0,0,6,4,0,0,2,10,7,0,0,0,20,0,0,0,0,0,63,17,6,0,0,0,21,0,0,0,67,7,0,0,6,64,0,0,0,0,0,0,0,64,18,8,0,63,3,0,0,0,0,63,3,4,28,0,0,20,0,0,0,17,18,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,2.15E2,5.5E1,2.09E2,6E0,5.2E1,3E0,6.7E1,1.42E2,3E0,3E0,3E0,4.9E1,6.4E1,3E0,4E0,1.38E2,3E1,1.9E1,2E1,4.4E1,4E0,1.34E2,1.9E1,1.1E1,1.2E1,7E0,1.4E1,6E0,3.7E1,7E0,1.18E2,1.6E1,8E0,1.1E1,8E0,3E0,9E0,3E0,4E0,3E0,1E1,4E0,3E0,3E0,1.1E1,2.6E1,3E0,4E0,1.08E2,1E1,1.3E1,3E0,3E0,5E0,7E0,4E0,6E0,3E0,3E0,8E0,1.4E1,1.2E1,1.04E2,4E0,3E0,7E0,1E1,3E0,3E0,4E0,8E0,6E0,6E0,6E0,4.1E1,6.3E1,7E0,3E0,5E0,3E0,3E0,3E0,3E0,3.8E1,5.6E1,7E0,6E0,3.2E1,2.9E1,2.7E1,3E0,4E0,5E0,2.7E1,1.4E1,1.5E1,2.2E1,5E0,1.3E1,1.4E1,6E0,8E0,4E0,1.1E1,8E0,1.4E1,8E0,6E0,4E0,7E0,4E0,4E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[9.330577E-3,-6.4527807E-3,7.09425E-2,-2.6355481E-3,-7.0366543E-3,-5.474839E-2,1.01524375E-1,-1.7426273E-2,5.3436466E-2,1.9456921E-3,-4.329712E-3,-4.4859264E-2,1.240405E-1,-2.828444E-2,9.847839E-4,3.850458E-2,5.5095716E-3,1.5438207E-3,-1.1348489E-3,1.4849022E-3,-4.011855E-3,1.7245957E-1,5.8899503E-2,-4.1157145E-2,-1.55899E-2,7.6267356E-3,-2.0663109E-2,2.2413E-2,6.425831E-2,1.271668E-1,1.1453107E-2,-7.701463E-3,1.08036704E-1,-5.5153962E-2,-2.7654666E-2,1.2790428E-3,-1.933011E-2,2.4229895E-2,-5.356577E-3,-2.5295147E-3,-1.2520426E-3,2.8689545E-2,-7.2163297E-4,8.251721E-4,2.4140556E-3,1.6295753E-1,-4.1191964E-4,2.0896816E-3,-1.7649742E-3,5.4934338E-2,6.0268114E-3,-6.41471E-2,-2.158015E-2,-3.6527835E-2,4.4826287E-4,-2.3964418E-2,1.3857071E-3,3.5479125E-2,1.9500009E-4,-1.0653731E-2,7.487132E-4,-1.2628864E-2,8.8529364E-4,-3.3744884E-4,3.665026E-2,2.5336817E-3,1.940949E-1,3.1004522E-3,3.4888493E-4,-3.944414E-2,-7.441452E-2,-1.5371699E-4,-1.0619076E-3,-1.5483537E-3,-1.9980645E-2,-1.4145563E-2,-4.469577E-2,-3.8597902E-4,6.385005E-4,1.828428E-4,4.6830628E-2,1.714224E-4,-1.942659E-2,6.201648E-5,-6.161665E-4,5.319477E-2,8.680384E-3,8.792458E-3,3.537507E-3,-1.655848E-3,-4.6952165E-4,-2.7069584E-3,-1.017414E-3,5.0785886E-5,-2.7342077E-2,-3.1756193E-3,-3.479949E-2,-1.7613197E-3,-1.9185708E-4,5.8616186E-4,1.8185206E-3,-2.4323387E-4,-8.684599E-4,7.361543E-4,2.064325E-3,8.0242596E-4,-5.7496084E-4,-1.1231384E-3,-2.9554675E-4,3.4087081E-3,-6.856874E-4,-1.8074188E-3,-4.6923713E-4,4.4438321E-4,-5.2495594E-3,1.6537866E-4,-5.364101E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":114,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,-1,-1,-1,-1,-1,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,-1,55,57,59,-1,61,63,-1,-1,-1,65,-1,-1,-1,67,-1,69,71,73,-1,75,77,79,-1,81,-1,83,-1,-1,85,-1,87,-1,-1,89,91,-1,-1,-1,93,95,97,-1,-1,-1,99,-1,101,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,107,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,-1,115,-1,-1],"loss_changes":[2.766316E-1,1.832559E-1,2.2850236E-1,1.8670465E-1,0E0,5.4525334E-2,1.60476E-1,3.5611887E-2,7.603E-2,1.5385688E-2,0E0,5.6560762E-2,1.1915892E-1,1.7660633E-2,9.791138E-3,1.6379759E-2,0E0,0E0,0E0,0E0,0E0,1.457538E-1,6.301116E-2,8.869275E-3,1.2769938E-2,1.1339118E-2,1.7890293E-2,8.52204E-3,4.713252E-3,1.02273464E-1,0E0,3.436327E-2,3.5974696E-2,6.465979E-3,1.157725E-2,0E0,5.3133983E-3,4.4867303E-3,4.795388E-3,0E0,4.600168E-3,8.18033E-3,0E0,0E0,0E0,2.6692837E-2,0E0,0E0,0E0,1.3555577E-2,0E0,2.137497E-3,1.184609E-3,3.8268901E-3,0E0,8.5278675E-3,2.9527089E-3,4.46965E-3,0E0,3.748229E-3,0E0,9.849245E-4,0E0,0E0,9.255953E-3,0E0,4.3200463E-2,0E0,0E0,1.6354667E-3,2.7295128E-3,0E0,0E0,0E0,1.9475468E-3,7.139808E-3,5.108947E-3,0E0,0E0,0E0,1.7372705E-3,0E0,1.3203598E-3,0E0,0E0,2.8448477E-3,4.3266835E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.136312E-4,2.6983044E-3,3.930752E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6027754E-3,0E0,0E0,0E0,0E0,1.271313E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,36,36,37,37,38,38,40,40,41,41,45,45,49,49,51,51,52,52,53,53,55,55,56,56,57,57,59,59,61,61,64,64,66,66,69,69,70,70,74,74,75,75,76,76,80,80,82,82,85,85,86,86,94,94,95,95,96,96,109,109,114,114],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,-1,-1,-1,-1,-1,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,-1,56,58,60,-1,62,64,-1,-1,-1,66,-1,-1,-1,68,-1,70,72,74,-1,76,78,80,-1,82,-1,84,-1,-1,86,-1,88,-1,-1,90,92,-1,-1,-1,94,96,98,-1,-1,-1,100,-1,102,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,108,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,-1,116,-1,-1],"split_conditions":[1.83E2,4.90938E5,5.305915E-4,4.472153E6,-7.0366543E-3,1.6010206E10,1.6E1,1.969156E6,2.4392496E4,2.398451E-4,-4.329712E-3,4E0,5.21992E5,2.315676E6,6.749936E9,1.34E2,5.5095716E-3,1.5438207E-3,-1.1348489E-3,1.4849022E-3,-4.011855E-3,3.204999E3,4.1E1,2.51E2,2.9609E4,1.17E2,3.5536144E8,1E0,5.10607E5,1.0618027E9,1.1453107E-2,5.7593933E9,4.799E3,2.2728746E-1,1.1234E4,1.2790428E-3,7.178534E8,3.05031E5,1.6216906E9,-2.5295147E-3,4.01145E5,2.2256E4,-7.2163297E-4,8.251721E-4,2.4140556E-3,1.5688386E-2,-4.1191964E-4,2.0896816E-3,-1.7649742E-3,1.69E2,6.0268114E-3,6.543E3,7.904483E-1,3.4702344E7,4.4826287E-4,1.37951E5,5.421011E7,2.8895E4,1.9500009E-4,5.275E3,7.487132E-4,3.07429E5,8.8529364E-4,-3.3744884E-4,9.69E2,2.5336817E-3,2.1114904E8,3.1004522E-3,3.4888493E-4,6.73792E7,2.9378E4,-1.5371699E-4,-1.0619076E-3,-1.5483537E-3,1.85948E5,1.2481975E-1,1.685836E6,-3.8597902E-4,6.385005E-4,1.828428E-4,6.1E1,1.714224E-4,1.92E4,6.201648E-5,-6.161665E-4,1.48E2,1.1962493E9,8.792458E-3,3.537507E-3,-1.655848E-3,-4.6952165E-4,-2.7069584E-3,-1.017414E-3,5.0785886E-5,1E0,1.5549E4,7.03651E5,-1.7613197E-3,-1.9185708E-4,5.8616186E-4,1.8185206E-3,-2.4323387E-4,-8.684599E-4,7.361543E-4,2.064325E-3,8.0242596E-4,-5.7496084E-4,-1.1231384E-3,-2.9554675E-4,1.1445502E7,-6.856874E-4,-1.8074188E-3,-4.6923713E-4,4.4438321E-4,1.5001173E8,1.6537866E-4,-5.364101E-4],"split_indices":[47,3,64,6,0,10,17,2,5,66,0,17,12,18,23,4,0,0,0,0,0,5,19,4,8,4,27,34,12,21,0,10,0,63,20,0,23,12,16,0,12,18,0,0,0,63,0,0,0,19,0,22,63,16,0,12,18,18,0,3,0,12,0,0,17,0,21,0,0,16,20,0,0,0,18,63,2,0,0,0,17,0,20,0,0,17,16,0,0,0,0,0,0,0,36,7,2,0,0,0,0,0,0,0,0,0,0,0,0,18,0,0,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.26E2,5.7E1,2.23E2,3E0,1.1E1,4.6E1,1.77E2,4.6E1,7E0,4E0,6E0,4E1,1.11E2,6.6E1,4.2E1,4E0,3E0,4E0,3E0,3E0,2.2E1,1.8E1,5.4E1,5.7E1,5.1E1,1.5E1,2.7E1,1.5E1,1.9E1,3E0,8E0,1E1,2.5E1,2.9E1,3E0,5.4E1,2.2E1,2.9E1,3E0,1.2E1,2.4E1,3E0,4E0,1.1E1,1.5E1,4E0,3E0,5E0,7E0,3E0,1.9E1,6E0,2.4E1,5E0,4.4E1,1E1,1.3E1,9E0,2.5E1,4E0,9E0,3E0,4E0,2E1,5E0,1E1,3E0,4E0,7E0,1.2E1,3E0,3E0,1.3E1,1.1E1,3.1E1,1.3E1,6E0,4E0,4E0,9E0,9E0,1.6E1,3E0,6E0,1.2E1,8E0,4E0,6E0,4E0,3E0,9E0,3E0,3E0,8E0,2.1E1,1E1,1E1,3E0,3E0,6E0,7E0,9E0,4E0,8E0,5E0,3E0,5E0,3E0,1.6E1,5E0,4E0,6E0,7E0,9E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[2.787173E-3,-1.1795731E-2,7.843486E-2,-9.142171E-3,-5.431044E-3,9.390979E-3,4.609444E-2,-2.3805398E-2,1.9057775E-2,2.5086569E-2,9.102842E-2,-2.0622151E-2,-4.5708558E-3,-3.3188388E-3,2.654835E-2,-3.1439823E-4,3.780277E-2,3.719932E-3,2.2302804E-4,-2.8953247E-3,-1.773092E-2,1.8932149E-2,3.7357465E-3,6.5985654E-4,-9.460635E-4,1.7464617E-2,2.6506865E-3,3.758029E-2,-2.0680632E-2,-1.404947E-2,3.0581245E-2,3.885757E-2,-6.1114132E-3,1.9331941E-3,2.8481788E-6,-2.3100665E-2,1.6486025E-3,1.1634558E-2,-4.673379E-2,1.4756438E-2,5.364771E-2,1.7369621E-3,6.11686E-4,-6.268176E-4,1.86702E-4,2.6903173E-2,-2.5934726E-2,6.134241E-4,-8.266817E-5,-2.5398987E-3,-5.6320237E-4,1.9018464E-2,-7.0697773E-4,6.2488414E-2,6.3276256E-4,-5.2142394E-4,2.3837502E-3,-3.9493656E-3,-2.2758046E-2,-2.8651708E-4,2.472996E-2,3.854098E-2,2.7235611E-3,-2.808264E-2,7.93136E-3,1.6024068E-3,1.2880469E-2,1.5541917E-3,3.816497E-4,-5.592133E-2,-2.5377486E-2,-5.0992863E-3,1.9741093E-3,3.4147512E-2,4.3817938E-4,-2.3178179E-3,-5.733842E-4,-1.3147995E-3,-1.9895868E-2,6.121148E-4,-1.3982257E-2,1.4938322E-3,4.2972024E-4,-6.022656E-4,3.549652E-4,9.313268E-3,-2.7583923E-2,1.654137E-5,-5.917132E-4,-5.634616E-4,1.6134874E-3,-1.1227418E-3,-1.425498E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":115,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,27,29,-1,-1,-1,31,-1,33,35,37,39,41,43,-1,-1,45,-1,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,59,-1,61,-1,-1,-1,-1,63,-1,65,67,-1,69,71,-1,73,-1,-1,75,77,79,-1,81,83,-1,-1,-1,85,-1,87,-1,-1,-1,-1,89,91,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2084244E-1,1.0006352E-1,3.1078398E-1,1.00107715E-1,0E0,0E0,3.7363045E-2,5.6894608E-2,7.8687966E-2,9.702001E-3,2.6025541E-2,2.9673204E-2,0E0,0E0,5.1658206E-2,7.259644E-3,1.6681056E-2,0E0,0E0,0E0,2.5413167E-2,2.8860897E-2,0E0,0E0,0E0,7.8028897E-3,0E0,6.7364294E-3,2.623815E-2,1.7333027E-2,1.928458E-2,1.3736198E-3,1.3862163E-3,0E0,0E0,2.0537674E-2,0E0,1.3139687E-3,7.0849545E-3,5.7387333E-3,5.382508E-3,0E0,0E0,0E0,0E0,1.7458279E-2,3.8088046E-2,0E0,0E0,0E0,0E0,5.163459E-3,0E0,5.7455003E-3,0E0,0E0,0E0,0E0,2.146586E-2,0E0,7.076768E-3,2.0272303E-3,0E0,6.940037E-3,1.4228789E-2,0E0,5.01264E-3,0E0,0E0,4.234178E-3,7.921115E-3,3.907597E-3,0E0,1.1654049E-3,2.8423443E-3,0E0,0E0,0E0,1.7772693E-2,0E0,8.3866925E-4,0E0,0E0,0E0,0E0,1.9484028E-2,1.031962E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,20,20,21,21,25,25,27,27,28,28,29,29,30,30,31,31,32,32,35,35,37,37,38,38,39,39,40,40,45,45,46,46,51,51,53,53,58,58,60,60,61,61,63,63,64,64,66,66,69,69,70,70,71,71,73,73,74,74,78,78,80,80,85,85,86,86],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,28,30,-1,-1,-1,32,-1,34,36,38,40,42,44,-1,-1,46,-1,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,60,-1,62,-1,-1,-1,-1,64,-1,66,68,-1,70,72,-1,74,-1,-1,76,78,80,-1,82,84,-1,-1,-1,86,-1,88,-1,-1,-1,-1,90,92,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1962493E9,1.21022024E8,1.571E4,1.039919E6,-5.431044E-3,9.390979E-3,3.7648073E11,9.57317E5,1.4387648E-6,4.564531E6,2.9331538E10,1.47192E5,-4.5708558E-3,-3.3188388E-3,8.9003E4,4.07729E5,4.425011E6,3.719932E-3,2.2302804E-4,-2.8953247E-3,3.7E1,6E1,3.7357465E-3,6.5985654E-4,-9.460635E-4,1.16E2,2.6506865E-3,1.746735E6,1.99564E9,5.131E3,5.675424E9,1.1443E4,3.03125E5,1.9331941E-3,2.8481788E-6,3.342E3,1.6486025E-3,3.1177E4,4.9E2,5.215832E10,8.81E2,1.7369621E-3,6.11686E-4,-6.268176E-4,1.86702E-4,2.3E1,1.1799E4,6.134241E-4,-8.266817E-5,-2.5398987E-3,-5.6320237E-4,1.105E3,-7.0697773E-4,2.0108912E7,6.3276256E-4,-5.2142394E-4,2.3837502E-3,-3.9493656E-3,2.3912651E8,-2.8651708E-4,1E2,2.2554214E9,2.7235611E-3,1.8E1,1.352E3,1.6024068E-3,5.097E3,1.5541917E-3,3.816497E-4,1.5618E4,4.01E2,3.805624E7,1.9741093E-3,4.56E3,1.8484124E3,-2.3178179E-3,-5.733842E-4,-1.3147995E-3,2.433E3,6.121148E-4,1.2654648E3,1.4938322E-3,4.2972024E-4,-6.022656E-4,3.549652E-4,2.4307465E-4,2.91E2,1.654137E-5,-5.917132E-4,-5.634616E-4,1.6134874E-3,-1.1227418E-3,-1.425498E-4],"split_indices":[16,6,8,6,0,0,13,6,63,18,27,2,0,0,7,18,2,0,0,0,20,19,0,0,0,4,0,2,23,3,21,3,12,0,0,7,0,18,17,10,17,0,0,0,0,17,8,0,0,0,0,0,0,2,0,0,0,0,21,0,19,1,0,4,0,0,3,0,0,28,0,27,0,19,5,0,0,0,20,0,5,0,0,0,0,63,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.89E2,2.43E2,4.6E1,2.4E2,3E0,5E0,4.1E1,1.58E2,8.2E1,2.9E1,1.2E1,1.55E2,3E0,4E0,7.8E1,1E1,1.9E1,9E0,3E0,5E0,1.5E2,7.3E1,5E0,6E0,4E0,1.4E1,5E0,7E0,1.43E2,1.9E1,5.4E1,7E0,7E0,4E0,3E0,1.39E2,4E0,1.1E1,8E0,3.3E1,2.1E1,3E0,4E0,3E0,4E0,7E0,1.32E2,7E0,4E0,3E0,5E0,3E1,3E0,1.6E1,5E0,4E0,3E0,3E0,1.29E2,5E0,2.5E1,9E0,7E0,1.1E2,1.9E1,7E0,1.8E1,6E0,3E0,8E0,1.02E2,1.6E1,3E0,6E0,1.2E1,5E0,3E0,2.5E1,7.7E1,4E0,1.2E1,3E0,3E0,4E0,8E0,1.6E1,6.1E1,3E0,9E0,1E1,6E0,4.6E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[8.071861E-3,-7.431731E-3,1.15121804E-1,-1.6488222E-2,5.022463E-2,6.68059E-2,2.5801507E-1,-2.2936117E-2,2.9005628E-2,7.495546E-3,1.9147037E-2,5.5580353E-3,4.7949813E-2,1.1907607E-2,3.041818E-3,-1.9214923E-2,-1.0751718E-1,1.0323025E-2,5.6940284E-2,6.399119E-2,3.4011938E-3,-8.1551576E-4,6.2425975E-2,-1.5753662E-2,-1.0646271E-1,-5.783741E-3,-9.103934E-4,-1.7313393E-2,2.5584016E-2,6.680329E-4,2.4030241E-3,2.9939592E-3,3.7717025E-4,-2.3191841E-2,3.192688E-2,7.987034E-2,1.3635142E-2,2.9807892E-2,-2.0324701E-2,-5.584559E-3,-4.1255736E-4,2.1681572E-4,-1.1920909E-3,1.2934554E-3,4.268747E-4,-8.795384E-5,-1.2984427E-3,1.349773E-3,3.0682667E-4,9.3585585E-4,3.229087E-3,-4.931215E-5,8.1738894E-4,4.967459E-3,5.1336337E-2,-4.964053E-2,-1.1233749E-2,7.0307456E-4,-4.1525942E-4,2.3294678E-4,2.7414872E-3,-7.4905425E-2,-3.249597E-2,2.7085477E-2,-1.7346472E-2,-7.3226006E-4,-8.604885E-2,-2.0721005E-8,-3.8685527E-2,5.0267413E-2,-1.652434E-2,-3.0137733E-2,-6.7511583E-3,-1.2683569E-3,-3.3620114E-3,-4.3480963E-2,-6.2607875E-4,-1.9139829E-4,6.966592E-2,-2.0593929E-3,1.1285656E-3,-3.8442567E-2,-1.4564447E-2,1.9951757E-2,-1.2270022E-2,-1.6130765E-3,-6.073587E-4,2.9412315E-3,6.3371833E-4,-4.9517922E-2,-2.25893E-2,1.8404068E-4,-2.9834734E-2,1.0232559E-3,-1.9660203E-5,-5.610653E-3,-2.994469E-2,-8.7419915E-4,-5.8434512E-2,-8.128711E-3,-1.9417586E-3,-2.127532E-4,-3.7718005E-2,-1.1189223E-2,2.0624099E-4,3.724025E-6,-3.8153496E-2,-9.927682E-4,-2.373535E-3,-6.0218986E-4,1.7924533E-4,-1.5519251E-3,-4.8830034E-4,7.475886E-3,-1.6878106E-2,-4.649792E-4,-1.4561772E-3,-1.8969522E-4,6.1081693E-4,-7.0665375E-4,-1.8948874E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":116,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,25,27,29,31,33,-1,35,37,39,-1,-1,41,43,-1,-1,-1,-1,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,61,63,-1,-1,-1,-1,65,67,69,71,-1,73,-1,75,77,79,81,83,-1,-1,85,-1,-1,87,-1,-1,89,91,93,95,-1,-1,-1,-1,97,99,-1,101,-1,-1,103,105,-1,107,109,-1,-1,111,113,-1,-1,115,-1,-1,-1,-1,-1,-1,117,119,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8547593E-1,1.3450217E-1,2.3821169E-1,6.578666E-2,1.9419764E-1,4.9609303E-2,1.2973893E-1,5.930887E-2,1.40251815E-2,0E0,2.167173E-2,0E0,2.9164627E-2,0E0,0E0,5.5096447E-2,4.0660895E-2,8.034611E-3,5.501576E-3,1.1046372E-2,1.8946474E-2,0E0,1.7635718E-2,3.850983E-2,4.1716203E-2,0E0,0E0,3.569319E-3,1.625889E-3,0E0,0E0,0E0,0E0,4.4502225E-3,2.2892635E-3,1.3700999E-2,1.2872784E-3,8.83618E-3,4.368282E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9954442E-3,1.2811499E-2,1.4458932E-2,3.0451894E-2,0E0,0E0,0E0,0E0,6.760545E-3,5.0282665E-3,1.9027915E-2,1.4840081E-2,0E0,6.1822236E-3,0E0,1.091376E-3,1.3499733E-2,1.937848E-2,5.906783E-3,9.381733E-3,0E0,0E0,1.6660225E-3,0E0,0E0,7.965475E-3,0E0,0E0,4.5000054E-3,6.055096E-3,2.6968634E-3,5.9525007E-3,0E0,0E0,0E0,0E0,2.0016916E-3,7.6945806E-3,0E0,1.7646998E-3,0E0,0E0,2.686346E-3,3.459053E-3,0E0,2.7928054E-3,1.843055E-3,0E0,0E0,1.1617579E-3,2.9931362E-3,0E0,0E0,1.1941902E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.1880468E-3,1.0422128E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,27,27,28,28,33,33,34,34,35,35,36,36,37,37,38,38,53,53,54,54,55,55,56,56,61,61,62,62,63,63,64,64,66,66,68,68,69,69,70,70,71,71,72,72,75,75,78,78,81,81,82,82,83,83,84,84,89,89,90,90,92,92,95,95,96,96,98,98,99,99,102,102,103,103,106,106,113,113,114,114],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,26,28,30,32,34,-1,36,38,40,-1,-1,42,44,-1,-1,-1,-1,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,62,64,-1,-1,-1,-1,66,68,70,72,-1,74,-1,76,78,80,82,84,-1,-1,86,-1,-1,88,-1,-1,90,92,94,96,-1,-1,-1,-1,98,100,-1,102,-1,-1,104,106,-1,108,110,-1,-1,112,114,-1,-1,116,-1,-1,-1,-1,-1,-1,118,120,-1,-1,-1,-1,-1,-1],"split_conditions":[5.0600757E3,4.805687E7,2.77873E5,6.6801E4,3.719E3,5.2203887E3,1.2756053E12,5.96025E5,5.59888E5,7.495546E-3,5.5659E4,5.5580353E-3,3.9652E4,1.1907607E-2,3.041818E-3,1.29864E5,1.3366E4,7.1E1,6.24933E5,3.43031E5,1.5200496E8,-8.1551576E-4,2.72E2,1.702784E-5,3.70531E5,-5.783741E-3,-9.103934E-4,9.311E4,1.14E2,6.680329E-4,2.4030241E-3,2.9939592E-3,3.7717025E-4,8E0,1.1288E4,2.326757E9,4.8E1,4.8E1,3.194984E6,-5.584559E-3,-4.1255736E-4,2.1681572E-4,-1.1920909E-3,1.2934554E-3,4.268747E-4,-8.795384E-5,-1.2984427E-3,1.349773E-3,3.0682667E-4,9.3585585E-4,3.229087E-3,-4.931215E-5,8.1738894E-4,3.969E3,1.43E2,7.25E2,2.5635E4,7.0307456E-4,-4.1525942E-4,2.3294678E-4,2.7414872E-3,2.86968E5,7.546657E7,4.0061934E9,4.669E3,-7.3226006E-4,5.705E3,-2.0721005E-8,9.4202E4,1.407E3,1E0,2.9837253E9,5.077E3,-1.2683569E-3,-3.3620114E-3,4.014E3,-6.2607875E-4,-1.9139829E-4,8.388608E7,-2.0593929E-3,1.1285656E-3,1.5025306E-1,1.0203876E3,6.6408E4,3.4158E4,-1.6130765E-3,-6.073587E-4,2.9412315E-3,6.3371833E-4,2E0,4.5291E4,1.8404068E-4,1.0995E4,1.0232559E-3,-1.9660203E-5,9.109177E8,7.8E1,-8.7419915E-4,3.722679E7,4.5697078E-1,-1.9417586E-3,-2.127532E-4,1.34028E5,7.8E1,2.0624099E-4,3.724025E-6,1.21205E5,-9.927682E-4,-2.373535E-3,-6.0218986E-4,1.7924533E-4,-1.5519251E-3,-4.8830034E-4,6.1E1,6.9E1,-4.649792E-4,-1.4561772E-3,-1.8969522E-4,6.1081693E-4,-7.0665375E-4,-1.8948874E-4],"split_indices":[5,18,3,20,7,5,13,12,22,0,8,0,8,0,0,28,3,4,18,6,21,0,19,66,18,0,0,20,19,0,0,0,0,19,3,27,4,17,26,0,0,0,0,0,0,0,0,0,0,0,0,0,0,3,4,28,8,0,0,0,0,16,10,10,3,0,21,0,18,28,19,10,3,0,0,3,0,0,16,0,0,63,5,18,20,0,0,0,0,19,28,0,20,0,0,11,19,0,16,63,0,0,18,4,0,0,18,0,0,0,0,0,0,4,19,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,2.55E2,3.6E1,2.21E2,3.4E1,2.8E1,8E0,1.94E2,2.7E1,4E0,3E1,3E0,2.5E1,4E0,4E0,1.87E2,7E0,1.7E1,1E1,7E0,2.3E1,4E0,2.1E1,1.81E2,6E0,3E0,4E0,6E0,1.1E1,4E0,6E0,4E0,3E0,1.2E1,1.1E1,1.5E1,6E0,1.6E1,1.65E2,3E0,3E0,3E0,3E0,4E0,7E0,6E0,6E0,7E0,4E0,5E0,1E1,3E0,3E0,8E0,8E0,3.8E1,1.27E2,4E0,4E0,4E0,4E0,1.4E1,2.4E1,1.7E1,1.1E2,3E0,1.1E1,4E0,2E1,1.1E1,6E0,4.9E1,6.1E1,4E0,7E0,1.5E1,5E0,3E0,8E0,3E0,3E0,3.1E1,1.8E1,1E1,5.1E1,1.1E1,4E0,5E0,3E0,1.7E1,1.4E1,8E0,1E1,6E0,4E0,3.8E1,1.3E1,6E0,1.1E1,1.1E1,3E0,3E0,7E0,2.6E1,1.2E1,3E0,1E1,5E0,6E0,6E0,5E0,4E0,3E0,6E0,2E1,3E0,7E0,3E0,3E0,1.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[5.0176885E-3,-8.035549E-3,9.7701326E-2,-2.7625913E-2,7.393248E-3,7.343003E-3,7.893214E-2,-4.001596E-3,-2.1998458E-2,6.147493E-3,2.1326384E-3,3.9153434E-2,1.0990587E-1,-1.7461075E-2,-4.364988E-3,-3.860231E-3,6.571696E-3,4.896629E-2,2.5016928E-4,1.1189327E-3,1.2261734E-1,-2.1698568E-2,3.6630858E-2,3.2802904E-3,3.7078739E-3,1.9292312E-3,9.015779E-4,4.537294E-3,2.305155E-3,1.2453648E-2,-2.8514288E-2,4.011612E-4,1.8567669E-3,3.2474592E-3,-4.990605E-4,-4.5458835E-2,4.81749E-2,-4.1248795E-2,-2.1147346E-2,-1.0702709E-2,2.543721E-2,1.9875448E-4,-2.7594815E-3,2.2919257E-3,2.0284338E-4,-1.7436051E-3,-2.3537872E-2,-2.5499383E-2,3.4460975E-3,-6.443763E-3,-5.5680905E-2,6.256645E-2,5.1125153E-3,-5.7446665E-5,-3.5424687E-2,-3.4959443E-2,-1.8169364E-2,5.3521345E-4,-3.3554615E-4,-1.7398965E-3,-2.2498684E-3,-8.3424465E-4,-2.541815E-3,7.48685E-2,5.204867E-4,-1.7471977E-2,2.4105472E-2,-1.4511235E-3,-3.889534E-4,-1.9356575E-2,-1.4339311E-3,2.7521187E-4,-2.434963E-2,-5.035362E-3,1.3228604E-3,6.237982E-4,3.086314E-3,-1.4317044E-4,-1.3650808E-3,-3.7315075E-4,4.522574E-2,-9.538817E-4,2.8919621E-5,-1.4930766E-2,-1.7791284E-3,-8.272974E-3,2.0345815E-2,5.7935255E-4,1.7974192E-3,-2.159696E-2,2.5924892E-4,1.0682714E-2,-1.2451061E-2,-2.6235476E-4,1.157269E-3,-1.0252216E-4,-9.0137677E-4,3.3784654E-2,-4.488856E-4,-2.7685268E-2,-2.7954055E-3,1.5582347E-3,3.448778E-4,-5.585325E-4,-1.5241426E-3,-3.949312E-4,3.6327756E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":117,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,15,17,19,21,-1,-1,23,25,-1,-1,27,29,31,33,-1,-1,-1,-1,-1,35,37,-1,-1,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,53,55,57,59,61,63,65,-1,67,69,71,-1,-1,-1,73,-1,-1,75,-1,77,79,-1,-1,81,-1,-1,83,85,-1,-1,-1,-1,-1,-1,87,-1,-1,89,-1,91,93,-1,-1,95,-1,97,99,-1,-1,-1,-1,101,-1,103,105,-1,-1,-1,-1,-1,-1],"loss_changes":[3.528431E-1,7.758171E-2,7.011774E-2,5.875057E-2,1.3866584E-1,0E0,3.6355942E-2,0E0,5.3303696E-2,0E0,7.718427E-2,4.5465175E-3,1.2931198E-2,2.486233E-2,0E0,0E0,4.860097E-2,1.0796934E-3,0E0,0E0,3.9373487E-3,2.316676E-2,3.3515086E-3,4.9481794E-2,0E0,0E0,0E0,0E0,0E0,3.7357982E-2,6.8204775E-3,0E0,0E0,0E0,3.467605E-2,1.50838345E-2,1.0157835E-2,5.360421E-3,5.867578E-3,1.7647818E-2,2.7601827E-2,0E0,0E0,0E0,0E0,0E0,3.556847E-3,2.55456E-3,1.8187711E-3,1.7249364E-2,3.497731E-3,6.2094703E-3,1.1144828E-2,0E0,1.4838595E-3,1.9800775E-3,4.831885E-3,0E0,0E0,0E0,9.987008E-3,0E0,0E0,9.801649E-3,0E0,3.7086816E-3,1.1079592E-2,0E0,0E0,1.9001146E-3,0E0,0E0,5.8734547E-3,6.4957486E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.5958529E-3,0E0,0E0,3.085135E-3,0E0,5.59771E-3,4.2955833E-3,0E0,0E0,1.6799355E-3,0E0,7.86756E-3,8.307096E-3,0E0,0E0,0E0,0E0,1.8430706E-3,0E0,3.6489256E-3,4.9409242E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,10,10,11,11,12,12,13,13,16,16,17,17,20,20,21,21,22,22,23,23,29,29,30,30,34,34,35,35,36,36,37,37,38,38,39,39,40,40,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,60,60,63,63,65,65,66,66,69,69,72,72,73,73,80,80,83,83,85,85,86,86,89,89,91,91,92,92,97,97,99,99,100,100],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,16,18,20,22,-1,-1,24,26,-1,-1,28,30,32,34,-1,-1,-1,-1,-1,36,38,-1,-1,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,54,56,58,60,62,64,66,-1,68,70,72,-1,-1,-1,74,-1,-1,76,-1,78,80,-1,-1,82,-1,-1,84,86,-1,-1,-1,-1,-1,-1,88,-1,-1,90,-1,92,94,-1,-1,96,-1,98,100,-1,-1,-1,-1,102,-1,104,106,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1328217E10,5.013E3,1.3219E4,9.05E3,3.068E3,7.343003E-3,6.3E1,-4.001596E-3,1.8558744E9,6.147493E-3,1.5E1,4.8778326E10,2.345689E7,1.443451E6,-4.364988E-3,-3.860231E-3,7.14535E5,4.62E2,2.5016928E-4,1.1189327E-3,2.4354641E0,2.9159E4,4.283E3,1E1,3.7078739E-3,1.9292312E-3,9.015779E-4,4.537294E-3,2.305155E-3,1.407E3,6.6E1,4.011612E-4,1.8567669E-3,3.2474592E-3,2.34E3,2.84E2,3.649951E9,6.494384E6,1.8468E4,2.810908E10,1.2051E4,1.9875448E-4,-2.7594815E-3,2.2919257E-3,2.0284338E-4,-1.7436051E-3,9.704549E6,6.059E3,4.18E2,2.3362E4,2.72074E5,2.84E2,1.01E2,-5.7446665E-5,7.283E3,8.59E2,1.0203876E3,5.3521345E-4,-3.3554615E-4,-1.7398965E-3,1.29864E5,-8.3424465E-4,-2.541815E-3,1E0,5.204867E-4,2.3768398E10,4.365884E6,-1.4511235E-3,-3.889534E-4,3.049E3,-1.4339311E-3,2.7521187E-4,1.2811E4,1.6429902E9,1.3228604E-3,6.237982E-4,3.086314E-3,-1.4317044E-4,-1.3650808E-3,-3.7315075E-4,1.08E2,-9.538817E-4,2.8919621E-5,3.3473062E9,-1.7791284E-3,6.0216E4,4.89E2,5.7935255E-4,1.7974192E-3,3.487E3,2.5924892E-4,7.031E3,2.83326E9,-2.6235476E-4,1.157269E-3,-1.0252216E-4,-9.0137677E-4,4.77915E5,-4.488856E-4,4.5556176E8,5.92E2,1.5582347E-3,3.448778E-4,-5.585325E-4,-1.5241426E-3,-3.949312E-4,3.6327756E-4],"split_indices":[21,3,7,18,7,0,4,0,23,0,17,21,2,6,0,0,8,17,0,0,67,8,3,4,0,0,0,0,0,28,4,0,0,0,0,0,10,18,7,10,7,0,0,0,0,0,18,22,17,8,12,19,4,0,20,28,5,0,0,0,28,0,0,19,0,23,6,0,0,3,0,0,20,16,0,0,0,0,0,0,17,0,0,10,0,8,0,0,0,3,0,3,10,0,0,0,0,6,0,11,17,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.9E2,2.55E2,3.5E1,1.12E2,1.43E2,3E0,3.2E1,5E0,1.07E2,3E0,1.4E2,1.5E1,1.7E1,1.04E2,3E0,4E0,1.36E2,1.1E1,4E0,3E0,1.4E1,9.7E1,7E0,1.33E2,3E0,6E0,5E0,9E0,5E0,1.6E1,8.1E1,4E0,3E0,4E0,1.29E2,6E0,1E1,2.8E1,5.3E1,9.3E1,3.6E1,3E0,3E0,6E0,4E0,1.5E1,1.3E1,4.5E1,8E0,8.6E1,7E0,1.2E1,2.4E1,5E0,8E0,1.8E1,2.7E1,4E0,4E0,6E0,8E1,4E0,3E0,9E0,3E0,1.1E1,1.3E1,5E0,3E0,8E0,1E1,5E0,2.2E1,7.6E1,4E0,3E0,6E0,8E0,3E0,5E0,8E0,5E0,3E0,1.8E1,4E0,6.8E1,8E0,3E0,5E0,1.4E1,4E0,1.2E1,5.6E1,3E0,5E0,4E0,1E1,6E0,6E0,2.1E1,3.5E1,3E0,3E0,1.5E1,6E0,2.1E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[2.000023E-3,-2.2876045E-2,3.327959E-2,-1.8253528E-2,-9.634898E-2,4.0332038E-2,-6.0636536E-3,-1.486623E-2,-3.6102172E-3,-4.879156E-3,-1.3992891E-3,3.357794E-2,7.387201E-3,-2.6657987E-2,5.882976E-3,1.6060227E-1,2.2879787E-2,1.626926E-3,-3.167578E-2,-9.5619465E-4,5.0962698E-2,7.4300873E-3,4.8651383E-4,-4.235091E-3,4.8166167E-2,-3.5815883E-2,-1.2521466E-2,2.5810389E-2,-1.4309605E-2,4.2399592E-4,2.4467674E-3,1.7954969E-3,-1.1497435E-2,3.8165983E-2,1.1414377E-1,-3.836855E-2,-5.688587E-5,-3.8977964E-3,-1.0441685E-3,2.5582607E-3,7.312372E-3,-8.141086E-3,-1.7850053E-3,-2.4081092E-2,1.1512257E-2,5.183522E-2,5.024828E-3,5.272736E-3,1.1570599E-3,-3.3640616E-2,-5.6840874E-2,-3.485265E-4,4.6355164E-4,-4.9256077E-3,1.1613528E-3,-1.1985638E-2,6.078489E-4,-1.4792802E-3,-1.20489765E-2,-8.559392E-3,3.253495E-2,3.097102E-2,6.887314E-2,-2.1329224E-2,1.3505084E-3,-4.367565E-2,-1.6602803E-2,-2.2561678E-3,-7.8128796E-4,-4.6436663E-4,2.4013958E-4,-2.0275816E-2,1.1624048E-3,-1.6673613E-2,4.1621266E-4,7.3331536E-4,-2.5245748E-2,1.6319287E-3,4.829468E-4,1.3713297E-3,1.9560564E-5,9.402136E-2,1.185764E-3,5.2786374E-5,-1.1882646E-3,-4.6749588E-2,-2.8756773E-4,-2.2415122E-2,2.459915E-4,-2.8803201E-2,-1.6729538E-4,2.8664837E-4,-4.7040108E-4,-9.1458E-3,-1.3717796E-3,-1.2758076E-3,-1.4630253E-4,1.3445361E-3,3.555193E-3,-1.6896505E-3,-8.6527504E-4,-5.883274E-3,-1.2326206E-3,-2.8695306E-4,-1.1605198E-3,2.331029E-4,-5.553562E-4,-1.8184343E-2,5.5089523E-4,-9.500669E-4,-7.426951E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":118,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,-1,17,19,21,23,-1,25,27,29,-1,-1,31,33,35,37,39,41,-1,-1,-1,43,45,47,49,-1,51,-1,-1,53,55,-1,57,59,61,63,-1,-1,65,67,-1,-1,69,-1,71,-1,-1,73,75,77,79,81,83,-1,85,87,-1,-1,-1,-1,89,91,93,-1,-1,95,-1,-1,-1,-1,97,-1,-1,-1,99,-1,101,-1,103,-1,-1,-1,105,-1,-1,-1,-1,-1,-1,-1,107,-1,-1,-1,-1,-1,109,-1,-1,-1],"loss_changes":[2.1785472E-1,5.1223546E-2,2.0143689E-1,4.5408353E-2,1.9694522E-2,1.4688654E-1,0E0,3.538925E-2,0E0,0E0,0E0,1.5667869E-1,0E0,3.723979E-2,1.638905E-2,8.846314E-2,7.562406E-2,0E0,6.4507127E-3,1.7168319E-2,5.6253187E-3,0E0,0E0,2.41772E-2,3.3251524E-2,6.103143E-3,2.7934762E-3,1.530226E-2,7.6701352E-3,0E0,0E0,0E0,1.4464986E-2,2.2843271E-2,2.1296084E-2,3.8520098E-3,0E0,1.8043941E-3,0E0,0E0,4.754339E-3,3.239389E-3,0E0,7.868772E-3,7.8737615E-3,1.0664292E-2,1.6467307E-2,0E0,0E0,8.678492E-3,3.477402E-3,0E0,0E0,1.2843686E-3,0E0,2.8575584E-3,0E0,0E0,2.7570357E-3,5.8048046E-3,2.1052323E-3,6.3311737E-3,1.18435845E-2,3.6405167E-3,0E0,3.213361E-3,3.2715197E-3,0E0,0E0,0E0,0E0,1.9075875E-3,1.4738E-3,3.3208043E-3,0E0,0E0,1.9049128E-3,0E0,0E0,0E0,0E0,3.9695054E-3,0E0,0E0,0E0,2.1256208E-3,0E0,4.4996236E-3,0E0,1.1205478E-3,0E0,0E0,0E0,2.25065E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.1401762E-3,0E0,0E0,0E0,0E0,0E0,1.1912147E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,32,32,33,33,34,34,35,35,37,37,40,40,41,41,43,43,44,44,45,45,46,46,49,49,50,50,53,53,55,55,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,71,71,72,72,73,73,76,76,81,81,85,85,87,87,89,89,93,93,101,101,107,107],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,-1,18,20,22,24,-1,26,28,30,-1,-1,32,34,36,38,40,42,-1,-1,-1,44,46,48,50,-1,52,-1,-1,54,56,-1,58,60,62,64,-1,-1,66,68,-1,-1,70,-1,72,-1,-1,74,76,78,80,82,84,-1,86,88,-1,-1,-1,-1,90,92,94,-1,-1,96,-1,-1,-1,-1,98,-1,-1,-1,100,-1,102,-1,104,-1,-1,-1,106,-1,-1,-1,-1,-1,-1,-1,108,-1,-1,-1,-1,-1,110,-1,-1,-1],"split_conditions":[4.0317974E8,1.33097E5,5.3408066E4,3.5655382E8,1.2609E5,2.4392496E4,-6.0636536E-3,5.21676E5,-3.6102172E-3,-4.879156E-3,-1.3992891E-3,4.833E3,7.387201E-3,3.185029E2,5.88764E5,2.702002E6,4.29906E5,1.626926E-3,1.3953088E8,2.4377E4,7.597E3,7.4300873E-3,4.8651383E-4,1.8001E5,4.0176924E9,1.6744E4,1.4158E4,7.62802E5,2.41361E5,4.2399592E-4,2.4467674E-3,1.7954969E-3,5.8882404E7,7.3512E5,4.413435E7,3.521722E6,-5.688587E-5,2.858911E6,-1.0441685E-3,2.5582607E-3,2.1009742E10,7.0507E4,-1.7850053E-3,6.44E4,2.5432851E8,3.6254539E3,5.9966224E-1,5.272736E-3,1.1570599E-3,9.56E3,2.48E2,-3.485265E-4,4.6355164E-4,3.385204E7,1.1613528E-3,1.8484124E3,6.078489E-4,-1.4792802E-3,8.622E4,2.433E3,6.9585E4,1.1619143E1,1.42E2,1.3628324E9,1.3505084E-3,1.1502465E7,2.5339E4,-2.2561678E-3,-7.8128796E-4,-4.6436663E-4,2.4013958E-4,2.280142E6,2.500539E3,1.207E3,4.1621266E-4,7.3331536E-4,1.064273E6,1.6319287E-3,4.829468E-4,1.3713297E-3,1.9560564E-5,6.2E1,1.185764E-3,5.2786374E-5,-1.1882646E-3,5.08868E5,-2.8756773E-4,5.4142E4,2.459915E-4,1.440233E6,-1.6729538E-4,2.8664837E-4,-4.7040108E-4,7.2925E4,-1.3717796E-3,-1.2758076E-3,-1.4630253E-4,1.3445361E-3,3.555193E-3,-1.6896505E-3,-8.6527504E-4,2.2066722E-3,-1.2326206E-3,-2.8695306E-4,-1.1605198E-3,2.331029E-4,-5.553562E-4,4.34786E5,5.5089523E-4,-9.500669E-4,-7.426951E-5],"split_indices":[16,8,5,16,7,5,0,6,0,0,0,7,0,5,12,2,12,0,16,18,3,0,0,6,16,7,28,6,12,0,0,0,18,28,2,27,0,21,0,0,10,28,0,8,23,5,63,0,0,22,17,0,0,16,0,5,0,0,8,20,8,60,19,16,0,21,28,0,0,0,0,2,5,0,0,0,2,0,0,0,0,4,0,0,0,2,0,12,0,2,0,0,0,8,0,0,0,0,0,0,0,64,0,0,0,0,0,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,1.55E2,1.23E2,1.47E2,8E0,1.2E2,3E0,1.43E2,4E0,3E0,5E0,1.17E2,3E0,9.1E1,5.2E1,8E0,1.09E2,5E0,8.6E1,4.6E1,6E0,5E0,3E0,5.3E1,5.6E1,7E1,1.6E1,1.5E1,3.1E1,3E0,3E0,5E0,4.8E1,5E1,6E0,6.5E1,5E0,1.2E1,4E0,3E0,1.2E1,2.8E1,3E0,3.1E1,1.7E1,3.5E1,1.5E1,3E0,3E0,5.4E1,1.1E1,9E0,3E0,9E0,3E0,2.5E1,3E0,1E1,2.1E1,9E0,8E0,1.7E1,1.8E1,9E0,6E0,3.3E1,2.1E1,7E0,4E0,5E0,4E0,1.5E1,1E1,1.8E1,3E0,3E0,6E0,3E0,5E0,1.2E1,5E0,9E0,9E0,4E0,5E0,3E1,3E0,1.7E1,4E0,9E0,6E0,7E0,3E0,1.5E1,3E0,3E0,3E0,3E0,6E0,2.2E1,8E0,9E0,8E0,3E0,6E0,5E0,1E1,6E0,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[9.902774E-4,4.0777237E-3,-5.4206154E-3,-1.3463726E-2,4.3915097E-2,-8.943451E-3,-7.163677E-3,3.4670454E-2,7.0860884E-3,-1.6591065E-2,3.7273474E-2,9.849847E-2,2.4293289E-2,-6.20335E-2,-1.1532085E-2,7.380115E-2,-4.3982018E-2,3.8531013E-2,5.238857E-3,1.1832987E-2,6.032457E-2,-6.809395E-2,-8.635871E-4,-2.5980748E-2,-5.69835E-3,1.546626E-2,1.24755934E-1,-1.700308E-4,-2.3782928E-3,6.53575E-5,2.1051206E-3,3.591825E-2,-2.0397978E-4,2.9762482E-2,5.1454455E-3,-1.0012571E-3,-2.4489325E-3,-3.777023E-2,-6.4214976E-3,-8.722415E-3,2.786351E-2,-1.1566313E-3,1.3721362E-3,6.111658E-3,1.5998908E-3,-3.677358E-4,5.026071E-2,-1.6179085E-2,1.578186E-2,-9.729589E-3,5.7171598E-2,-2.8455751E-2,-2.2918503E-3,1.1159716E-3,-7.961816E-4,-4.173938E-3,-2.7414823E-2,1.0481734E-4,1.2754678E-3,4.378583E-4,5.9021723E-2,-4.65301E-4,-1.614214E-3,1.3444861E-3,-6.9146915E-3,8.929621E-4,-1.4410365E-3,2.268329E-3,7.381685E-4,-1.6790498E-3,-2.0289727E-2,4.807575E-4,-4.140453E-4,2.1219963E-2,-7.2902045E-3,-3.4874305E-2,1.4375197E-5,8.319724E-4,2.1963045E-3,1.0233378E-3,-1.654721E-2,-1.9999295E-2,6.75427E-4,-2.638699E-2,2.7056643E-5,9.379952E-4,1.2986637E-4,-1.2531179E-2,9.1288965E-3,-1.6777124E-3,-1.7585488E-2,-9.834778E-4,-1.9892343E-4,1.3871753E-4,-1.1409845E-3,-3.9183226E-4,-1.1435437E-3,-1.6933971E-3,-9.207361E-3,-9.5238135E-4,1.8844597E-2,-7.910904E-4,-8.699608E-5,6.555641E-3,-1.3596354E-2,-4.6063494E-3,1.2856261E-3,6.197686E-4,-2.2003343E-3,-1.0214971E-3,-8.928687E-3,4.1674526E-4,-5.70634E-4,-3.8556967E-4,3.4030917E-4,-5.865573E-6,-6.436309E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":119,"left_children":[1,3,-1,5,7,9,-1,11,-1,13,15,17,19,21,23,25,27,29,-1,31,33,35,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,59,61,63,65,67,69,-1,71,-1,73,75,-1,-1,-1,77,79,-1,-1,81,-1,-1,-1,-1,-1,83,-1,-1,85,87,89,-1,-1,-1,-1,91,93,-1,95,-1,-1,-1,97,99,-1,101,-1,-1,-1,-1,-1,-1,-1,103,-1,105,-1,-1,107,109,111,-1,-1,113,-1,115,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4607595E-1,1.9206391E-1,0E0,1.7852965E-1,1.2921534E-1,6.7066856E-2,0E0,5.125781E-2,0E0,3.652719E-2,8.338393E-2,3.611408E-2,3.1162392E-2,1.586847E-3,1.2242533E-2,5.4546885E-2,1.002498E-2,6.7320513E-3,0E0,1.5662657E-2,4.9993806E-2,1.4735125E-3,0E0,9.442382E-3,1.1001321E-2,1.5492288E-2,3.9431527E-2,0E0,0E0,0E0,0E0,1.2796342E-2,9.7042285E-3,1.6792897E-2,0E0,0E0,0E0,6.069828E-3,2.374191E-3,8.217728E-3,2.4755904E-3,0E0,0E0,0E0,0E0,0E0,3.693398E-3,1.011851E-2,1.1495173E-2,1.0432363E-2,2.4816636E-3,3.2005813E-3,0E0,2.7034797E-3,0E0,6.4854296E-3,3.964375E-3,0E0,0E0,0E0,1.5973896E-3,7.788594E-3,0E0,0E0,4.435009E-3,0E0,0E0,0E0,0E0,0E0,2.341244E-3,0E0,0E0,1.1074618E-3,6.3101472E-3,3.555989E-3,0E0,0E0,0E0,0E0,1.2632413E-3,3.156767E-3,0E0,1.3861805E-3,0E0,0E0,0E0,6.842063E-3,7.3283324E-3,0E0,8.6998544E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.6866856E-3,0E0,7.6045743E-3,0E0,0E0,1.3766114E-3,3.1878087E-3,2.0720789E-3,0E0,0E0,1.1257575E-3,0E0,3.2873708E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,25,25,26,26,31,31,32,32,33,33,37,37,38,38,39,39,40,40,46,46,47,47,48,48,49,49,50,50,51,51,53,53,55,55,56,56,60,60,61,61,64,64,70,70,73,73,74,74,75,75,80,80,81,81,83,83,87,87,88,88,90,90,98,98,100,100,103,103,104,104,105,105,108,108,110,110],"right_children":[2,4,-1,6,8,10,-1,12,-1,14,16,18,20,22,24,26,28,30,-1,32,34,36,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,60,62,64,66,68,70,-1,72,-1,74,76,-1,-1,-1,78,80,-1,-1,82,-1,-1,-1,-1,-1,84,-1,-1,86,88,90,-1,-1,-1,-1,92,94,-1,96,-1,-1,-1,98,100,-1,102,-1,-1,-1,-1,-1,-1,-1,104,-1,106,-1,-1,108,110,112,-1,-1,114,-1,116,-1,-1,-1,-1,-1,-1],"split_conditions":[5.3408066E4,4.425011E6,-5.4206154E-3,1.11386E5,2.6339598E4,1.83E2,-7.163677E-3,1.619E3,7.0860884E-3,2.578E4,1.1768184E9,1.03E2,8.800221E9,8.656044E6,1.49015E5,2.8E1,1.0803774E10,1.3069112E10,5.238857E-3,3.318E4,3.9743017E9,2.908E3,-8.635871E-4,5.26565E5,1.728944E6,3.2E1,4.833E3,-1.700308E-4,-2.3782928E-3,6.53575E-5,2.1051206E-3,7.562E3,5.7407E4,5.9966224E-1,5.1454455E-3,-1.0012571E-3,-2.4489325E-3,2.147E3,9.1697E4,4.1477E4,1.296E3,-1.1566313E-3,1.3721362E-3,6.111658E-3,1.5998908E-3,-3.677358E-4,1E0,9.035389E6,7.2227E-1,4.0231396E7,2.6565576E0,4.5E1,-2.2918503E-3,5.275E3,-7.961816E-4,1E0,3.15196E5,1.0481734E-4,1.2754678E-3,4.378583E-4,5.86E3,1.17E2,-1.614214E-3,1.3444861E-3,1.058E3,8.929621E-4,-1.4410365E-3,2.268329E-3,7.381685E-4,-1.6790498E-3,3.66846E6,4.807575E-4,-4.140453E-4,5.078286E-1,5.421011E7,3.43E2,1.4375197E-5,8.319724E-4,2.1963045E-3,1.0233378E-3,1.28E2,1.05055E5,6.75427E-4,3.487E3,2.7056643E-5,9.379952E-4,1.2986637E-4,1.41E2,8.099343E2,-1.6777124E-3,6.8001E4,-9.834778E-4,-1.9892343E-4,1.3871753E-4,-1.1409845E-3,-3.9183226E-4,-1.1435437E-3,-1.6933971E-3,2.14E3,-9.5238135E-4,2.23334E5,-7.910904E-4,-8.699608E-5,2.25E2,1.0814E5,1.06E2,1.2856261E-3,6.197686E-4,4.875E3,-1.0214971E-3,2.8681312E8,4.1674526E-4,-5.70634E-4,-3.8556967E-4,3.4030917E-4,-5.865573E-6,-6.436309E-4],"split_indices":[5,2,0,28,5,47,0,0,0,12,11,19,27,21,18,17,10,10,0,18,16,21,0,6,6,4,7,0,0,0,0,20,20,63,0,0,0,19,8,28,0,0,0,0,0,0,19,2,64,2,67,4,0,3,0,45,12,0,0,0,3,4,0,0,17,0,0,0,0,0,21,0,0,63,18,17,0,0,0,0,4,20,0,3,0,0,0,20,5,0,8,0,0,0,0,0,0,0,20,0,6,0,0,4,6,4,0,0,28,0,16,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.73E2,4E0,1.9E2,8.3E1,1.87E2,3E0,8E1,3E0,1.61E2,2.6E1,1E1,7E1,1.5E1,1.46E2,1.8E1,8E0,6E0,4E0,5.3E1,1.7E1,1.2E1,3E0,4.1E1,1.05E2,9E0,9E0,4E0,4E0,3E0,3E0,1.7E1,3.6E1,1.4E1,3E0,3E0,9E0,2.5E1,1.6E1,9.7E1,8E0,3E0,6E0,4E0,5E0,4E0,1.3E1,1.8E1,1.8E1,6E0,8E0,2.1E1,4E0,1.2E1,4E0,7.9E1,1.8E1,3E0,5E0,3E0,1E1,1.3E1,5E0,8E0,1E1,3E0,3E0,5E0,3E0,4E0,1.7E1,6E0,6E0,8E0,7.1E1,1.4E1,4E0,3E0,7E0,4E0,9E0,7E0,3E0,1.3E1,4E0,5E0,3E0,5.4E1,1.7E1,6E0,8E0,3E0,6E0,3E0,4E0,6E0,7E0,3E0,5.1E1,3E0,1.4E1,5E0,3E0,1.1E1,4E1,7E0,7E0,4E0,7E0,7E0,3.3E1,3E0,4E0,4E0,3E0,1.9E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[1.1902583E-3,-6.855789E-3,9.5494345E-2,-1.1342888E-2,1.18547596E-1,6.7338524E-3,4.974648E-2,-2.1083333E-2,1.8532706E-2,1.979298E-3,4.8893653E-3,1.4136654E-2,2.8339305E-3,-1.84468E-2,-4.455314E-3,1.0988946E-2,4.612158E-2,-6.7225366E-4,1.0342081E-3,-1.5251692E-2,-3.5973762E-3,-2.41488E-2,1.825682E-2,2.0035923E-2,3.2126296E-3,-2.3823853E-3,-1.1746693E-2,3.2796763E-4,-1.727152E-3,-6.3082753E-3,2.9486287E-2,1.0722275E-3,1.8136954E-4,-1.7487476E-2,2.6203912E-2,1.4538234E-2,-2.9231675E-2,5.062513E-2,9.146906E-3,-2.02208E-2,7.339448E-4,-4.389771E-4,3.933687E-3,1.0319036E-3,-7.676988E-5,-1.3044511E-3,-3.4218899E-4,2.0553917E-3,9.101215E-4,8.797836E-4,-2.5859984E-4,-2.1985281E-2,1.6697189E-2,-4.6928912E-2,2.4627833E-2,-2.8237054E-2,-1.5952175E-2,5.5403023E-5,8.85161E-4,-6.7452766E-4,-1.9690099E-3,-5.077982E-4,5.1851884E-2,-2.9883241E-2,1.3963979E-4,-2.1366362E-2,-6.337712E-3,2.443219E-3,2.8409253E-4,-3.179618E-2,-5.5252942E-5,-1.1944621E-2,-3.156527E-2,-2.1064728E-3,-7.8971975E-4,-2.7397815E-2,-1.5482398E-3,-1.0779761E-3,-6.246215E-3,-4.1142564E-2,-4.5786073E-4,6.522304E-5,-4.5558892E-4,-4.7685788E-4,-1.0515702E-3,3.4291486E-4,-3.1077233E-4,-5.4680766E-4,-1.6091482E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":120,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,-1,21,23,-1,-1,25,-1,27,29,31,-1,-1,33,-1,-1,35,37,-1,-1,39,41,43,45,47,49,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,55,57,59,61,63,65,-1,-1,-1,-1,-1,67,69,-1,71,73,-1,-1,75,-1,77,79,81,-1,83,-1,-1,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1235295E-1,1.465206E-1,1.0403012E-1,7.314113E-2,7.7848136E-3,0E0,2.193731E-2,5.591181E-2,1.2396852E-2,0E0,0E0,6.9722505E-3,0E0,5.3749442E-2,0E0,1.3209719E-2,1.6205192E-2,0E0,0E0,3.6408726E-2,0E0,9.665767E-3,1.1771831E-2,1.7240234E-3,0E0,0E0,3.7931215E-2,0E0,0E0,7.1151964E-3,1.20055135E-2,0E0,0E0,1.6979326E-2,5.888249E-2,2.4483402E-3,1.0396433E-3,2.3340136E-3,5.218461E-3,9.552002E-3,0E0,2.3297492E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6414286E-3,1.0850352E-3,1.3074754E-3,1.5117258E-2,3.6152527E-3,3.5604537E-3,0E0,0E0,0E0,0E0,0E0,7.608505E-3,3.2860562E-3,0E0,3.947083E-3,2.027112E-3,0E0,0E0,2.8410032E-3,0E0,2.860036E-3,2.9024817E-3,1.173789E-3,0E0,2.6996657E-3,0E0,0E0,1.312454E-3,1.9292459E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,15,15,16,16,19,19,21,21,22,22,23,23,26,26,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,51,51,52,52,53,53,54,54,55,55,56,56,62,62,63,63,65,65,66,66,69,69,71,71,72,72,73,73,75,75,78,78,79,79],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,-1,22,24,-1,-1,26,-1,28,30,32,-1,-1,34,-1,-1,36,38,-1,-1,40,42,44,46,48,50,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,56,58,60,62,64,66,-1,-1,-1,-1,-1,68,70,-1,72,74,-1,-1,76,-1,78,80,82,-1,84,-1,-1,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8943199E9,1.1741724E4,9.068E3,1.30542E5,7.550548E7,6.7338524E-3,7.4661E4,1.12776E5,8.794227E9,1.979298E-3,4.8893653E-3,1.2439753E-1,2.8339305E-3,1.17442E5,-4.455314E-3,6E1,6.24933E5,-6.7225366E-4,1.0342081E-3,2.38E2,-3.5973762E-3,2.326757E9,1.6435003E-2,6.826978E0,3.2126296E-3,-2.3823853E-3,1.83E2,3.2796763E-4,-1.727152E-3,1.3035306E-3,1.858566E9,1.0722275E-3,1.8136954E-4,2.402988E6,8.5699437E8,4.56E3,2.3027863E-2,1.6817E4,5.397065E8,1.0886205E9,7.339448E-4,1.3557681E-3,3.933687E-3,1.0319036E-3,-7.676988E-5,-1.3044511E-3,-3.4218899E-4,2.0553917E-3,9.101215E-4,8.797836E-4,-2.5859984E-4,2.748732E6,6.62E2,1.099164E6,1.277E4,6.09E2,2.5425295E9,5.5403023E-5,8.85161E-4,-6.7452766E-4,-1.9690099E-3,-5.077982E-4,1.213102E6,1.04795E5,1.3963979E-4,1.5074E4,1.296E3,2.443219E-3,2.8409253E-4,9.68E2,-5.5252942E-5,2.578E4,6.9E1,1.199E3,-7.8971975E-4,8.9042156E-4,-1.5482398E-3,-1.0779761E-3,1.0939E4,6.3863E4,-4.5786073E-4,6.522304E-5,-4.5558892E-4,-4.7685788E-4,-1.0515702E-3,3.4291486E-4,-3.1077233E-4,-5.4680766E-4,-1.6091482E-3],"split_indices":[16,5,7,22,2,0,8,22,27,0,0,63,0,8,0,19,18,0,0,0,0,27,63,60,0,0,47,0,0,63,1,0,0,6,23,19,67,7,16,16,0,66,0,0,0,0,0,0,0,0,0,18,0,2,8,4,10,0,0,0,0,0,2,8,0,28,0,0,0,17,0,12,19,17,0,63,0,0,7,8,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,2.57E2,2.1E1,2.49E2,8E0,5E0,1.6E1,1.88E2,6.1E1,4E0,4E0,9E0,7E0,1.85E2,3E0,4.9E1,1.2E1,3E0,6E0,1.8E2,5E0,8E0,4.1E1,9E0,3E0,9E0,1.71E2,4E0,4E0,1.3E1,2.8E1,4E0,5E0,1.49E2,2.2E1,7E0,6E0,1.3E1,1.5E1,1.4E2,9E0,1.8E1,4E0,3E0,4E0,3E0,3E0,7E0,6E0,7E0,8E0,1.34E2,6E0,6E0,1.2E1,6.4E1,7E1,3E0,3E0,3E0,3E0,5E0,7E0,6.1E1,3E0,4.4E1,2.6E1,4E0,3E0,5.7E1,4E0,2.4E1,2E1,2.2E1,4E0,4.7E1,1E1,4E0,2E1,1.2E1,8E0,1.7E1,5E0,1.5E1,3.2E1,3E0,1.7E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[2.6502549E-3,-1.6531622E-2,3.1069502E-2,-1.2548509E-2,-8.727596E-2,3.912454E-2,-5.007869E-3,-1.8611597E-2,3.203631E-2,-4.764054E-3,-1.0379477E-3,2.9520245E-2,1.7897177E-1,-6.3455045E-2,-1.5529826E-2,9.94744E-3,3.795237E-3,7.2035883E-3,2.168916E-2,2.7879036E-3,7.2936974E-3,-6.402244E-4,-2.637012E-3,-1.3095394E-2,-2.1405346E-3,3.463059E-2,-6.6643315E-3,-7.382931E-2,2.9423103E-2,2.1601003E-2,-1.5809773E-2,2.997413E-4,1.6510233E-3,7.766577E-3,-8.2193664E-4,-3.160164E-5,-4.713475E-3,1.9661877E-2,7.362751E-2,-7.458186E-4,4.4098582E-2,-3.0310638E-2,-8.189698E-3,-1.3228047E-4,5.69777E-4,1.5238552E-2,2.670526E-3,9.370102E-2,-4.8401387E-4,1.2982333E-4,2.3542796E-3,-1.5679968E-4,-3.2860283E-2,1.354188E-3,-1.1999674E-2,1.8299619E-2,-1.0264579E-3,5.5247292E-2,5.666014E-3,-3.561871E-2,-2.4241215E-4,-3.56822E-2,-5.694512E-3,1.0124858E-2,4.2261407E-2,5.861905E-4,2.674123E-3,-4.0100075E-2,-1.8714912E-2,-4.743891E-2,-2.6334435E-4,1.315242E-3,-1.0495086E-2,5.1748483E-3,3.1882416E-2,3.1458784E-4,5.0845683E-2,-4.4001512E-2,-4.800577E-4,-2.0983374E-5,-8.896314E-4,-1.8277269E-2,-2.5357634E-3,-1.5324566E-2,1.3735922E-2,1.0253691E-2,-3.836042E-4,1.4891514E-3,2.6554198E-4,6.4056873E-4,1.8500999E-3,-2.7717207E-2,-5.1327027E-2,-5.977662E-5,-9.697943E-4,-6.3162073E-3,-3.4721874E-2,1.5288485E-3,-7.517857E-3,-2.4495209E-3,2.1748645E-2,-1.1695727E-3,-2.530639E-4,-8.11537E-4,-1.9072606E-3,-1.1308921E-3,-2.302712E-2,-1.689734E-4,-4.1652534E-2,1.3522494E-4,-5.5871095E-4,2.986083E-4,-9.085706E-3,1.4040628E-3,1.7239635E-4,4.0156633E-4,-2.0976954E-4,-1.165162E-3,-2.5985673E-5,-1.6084169E-3,-4.0331733E-4,-6.835764E-4,-8.61342E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":121,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,19,21,23,25,-1,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,45,47,-1,49,51,53,-1,-1,55,-1,57,-1,-1,-1,-1,59,-1,61,63,-1,65,-1,67,-1,69,71,73,75,-1,-1,77,79,81,-1,-1,83,85,87,-1,89,91,-1,-1,-1,93,-1,95,97,99,-1,-1,-1,-1,-1,101,103,-1,-1,105,107,-1,109,111,113,-1,-1,-1,-1,115,117,-1,119,-1,-1,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5696724E-1,4.737145E-2,1.8048665E-1,4.4929005E-2,2.5304005E-2,1.4514771E-1,0E0,1.9053161E-2,3.6761966E-2,0E0,0E0,1.564091E-1,1.1161491E-2,5.6116246E-3,1.612828E-2,7.201314E-3,0E0,0E0,7.826193E-2,0E0,0E0,0E0,0E0,1.2773154E-2,0E0,2.475217E-3,3.1938055E-3,4.216733E-2,3.9975315E-2,1.1094158E-2,1.334298E-2,0E0,0E0,8.986165E-4,0E0,0E0,0E0,2.1136228E-2,3.1665556E-2,0E0,7.849732E-3,2.5641061E-3,1.620685E-2,0E0,0E0,1.1547236E-2,0E0,3.459368E-2,0E0,0E0,0E0,0E0,2.3867935E-3,0E0,1.1294614E-2,1.3675606E-2,0E0,9.824444E-3,0E0,1.8251054E-3,0E0,4.785031E-3,1.4285122E-2,5.7585198E-3,4.523061E-3,0E0,0E0,1.8986017E-3,1.4325497E-3,8.612875E-3,0E0,0E0,6.8808165E-3,4.11537E-3,3.0168649E-3,0E0,1.7287992E-3,1.20176E-3,0E0,0E0,0E0,1.3062104E-3,0E0,8.168108E-3,7.532373E-3,5.310096E-3,0E0,0E0,0E0,0E0,0E0,1.2542568E-3,1.4036521E-3,0E0,0E0,2.9201047E-3,2.8453022E-3,0E0,8.800583E-4,1.4850021E-3,6.5783532E-3,0E0,0E0,0E0,0E0,2.0599084E-3,2.3119394E-3,0E0,2.2799484E-3,0E0,0E0,0E0,8.77775E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,12,12,13,13,14,14,15,15,18,18,23,23,25,25,26,26,27,27,28,28,29,29,30,30,33,33,37,37,38,38,40,40,41,41,42,42,45,45,47,47,52,52,54,54,55,55,57,57,59,59,61,61,62,62,63,63,64,64,67,67,68,68,69,69,72,72,73,73,74,74,76,76,77,77,81,81,83,83,84,84,85,85,91,91,92,92,95,95,96,96,98,98,99,99,100,100,105,105,106,106,108,108,112,112],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,20,22,24,26,-1,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,46,48,-1,50,52,54,-1,-1,56,-1,58,-1,-1,-1,-1,60,-1,62,64,-1,66,-1,68,-1,70,72,74,76,-1,-1,78,80,82,-1,-1,84,86,88,-1,90,92,-1,-1,-1,94,-1,96,98,100,-1,-1,-1,-1,-1,102,104,-1,-1,106,108,-1,110,112,114,-1,-1,-1,-1,116,118,-1,120,-1,-1,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0804082E9,1.17442E5,5.3408066E4,4.168507E6,5.5461E4,1.6189593E4,-5.007869E-3,9.89333E5,1.8522884E-4,-4.764054E-3,-1.0379477E-3,3.719E3,2.33E2,4.518873E6,4.72323E8,8.372779E-6,3.795237E-3,7.2035883E-3,4.73E3,2.7879036E-3,7.2936974E-3,-6.402244E-4,-2.637012E-3,1.6028E4,-2.1405346E-3,3.04326E5,2.32877E5,1.7079E4,1.83E2,1.34028E5,7.03651E5,2.997413E-4,1.6510233E-3,1.610428E6,-8.2193664E-4,-3.160164E-5,-4.713475E-3,7.0231123E3,8.2402E4,-7.458186E-4,2.49243E5,3.0597E5,6.4239606E2,-1.3228047E-4,5.69777E-4,4.8778326E10,2.670526E-3,3.4134E4,-4.8401387E-4,1.2982333E-4,2.3542796E-3,-1.5679968E-4,2.1811952E7,1.354188E-3,1.17E2,1.9038E4,-1.0264579E-3,4.3E1,5.666014E-3,1.4145E4,-2.4241215E-4,1.6671585E9,1.82E2,1.6429902E9,3.6386406E-1,5.861905E-4,2.674123E-3,2.14591E5,1.0816216E3,1.14928E5,-2.6334435E-4,1.315242E-3,1.039919E6,8.405997E8,6.3E1,3.1458784E-4,7.1376E4,2.9922E4,-4.800577E-4,-2.0983374E-5,-8.896314E-4,9.45E3,-2.5357634E-3,1.06E3,1.864E3,2.85218E5,-3.836042E-4,1.4891514E-3,2.6554198E-4,6.4056873E-4,1.8500999E-3,4.25E2,1.1026E4,-5.977662E-5,-9.697943E-4,1.37507E5,9.885E3,1.5288485E-3,2.3297848E7,6.71104E5,1.01E2,-1.1695727E-3,-2.530639E-4,-8.11537E-4,-1.9072606E-3,1.83935E5,1.610773E6,-1.689734E-4,3.222287E10,1.3522494E-4,-5.5871095E-4,2.986083E-4,3.3587363E8,1.4040628E-3,1.7239635E-4,4.0156633E-4,-2.0976954E-4,-1.165162E-3,-2.5985673E-5,-1.6084169E-3,-4.0331733E-4,-6.835764E-4,-8.61342E-5],"split_indices":[23,8,5,2,22,5,0,11,63,0,0,7,4,10,27,64,0,0,3,0,0,0,0,12,0,12,12,20,47,18,2,0,0,6,0,0,0,5,3,0,2,2,5,0,0,21,0,3,0,0,0,0,18,0,17,7,0,4,0,7,0,11,17,16,67,0,0,6,5,12,0,0,6,16,19,0,8,12,0,0,0,20,0,0,0,12,0,0,0,0,0,4,7,0,0,12,28,0,21,6,19,0,0,0,0,6,2,0,10,0,0,0,16,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.71E2,1.15E2,1.63E2,8E0,1.11E2,4E0,1.44E2,1.9E1,3E0,5E0,1.05E2,6E0,8E0,1.36E2,1.6E1,3E0,3E0,1.02E2,3E0,3E0,3E0,5E0,1.31E2,5E0,6E0,1E1,7E0,9.5E1,9E0,1.22E2,3E0,3E0,6E0,4E0,4E0,3E0,7.9E1,1.6E1,3E0,6E0,4.1E1,8.1E1,3E0,3E0,7.5E1,4E0,1.3E1,3E0,3E0,3E0,4E0,3.7E1,5E0,7.6E1,7.1E1,4E0,1E1,3E0,3.3E1,4E0,1.5E1,6.1E1,5.4E1,1.7E1,5E0,5E0,2.5E1,8E0,1E1,5E0,5E0,5.6E1,4.5E1,9E0,4E0,1.3E1,2.1E1,4E0,3E0,5E0,6E0,4E0,4.7E1,9E0,3.5E1,1E1,5E0,4E0,3E0,1E1,8E0,1.3E1,3E0,3E0,3.3E1,1.4E1,3E0,6E0,1.7E1,1.8E1,5E0,3E0,4E0,9E0,2.6E1,7E0,3E0,1.1E1,3E0,3E0,6E0,1.1E1,7E0,1.1E1,7E0,1.9E1,4E0,3E0,8E0,3E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[8.225428E-3,-9.409863E-3,7.198238E-2,-5.202353E-3,-1.3729623E-1,8.075585E-3,5.152931E-2,-8.7824045E-3,5.9820274E-3,4.8007953E-4,-8.2140695E-3,3.0236986E-2,9.14019E-2,-2.3861503E-2,9.686732E-3,4.0801775E-2,-1.5338355E-3,6.4935964E-3,5.810634E-2,-1.9939022E-2,-4.1744863E-3,3.3223256E-3,4.996933E-3,4.218405E-3,2.7840488E-2,9.2331685E-2,1.2339532E-2,-1.5588768E-2,-3.594116E-3,3.694316E-3,-1.667804E-3,2.5906677E-3,4.365139E-2,1.5638848E-3,3.7858249E-3,1.2605169E-3,-7.8125286E-4,-2.7786443E-2,2.6475715E-3,-2.2293036E-3,3.1228033E-3,-1.877565E-4,5.5477803E-4,2.7863944E-3,2.214246E-2,-2.431625E-3,-2.3487004E-2,-2.5091436E-2,2.0385234E-2,2.4763613E-3,-5.157289E-4,1.7120627E-3,-1.0147624E-3,-3.419349E-2,-1.7169205E-2,9.111812E-3,-4.2952377E-2,6.926324E-2,7.943156E-4,-7.0555746E-3,3.4738496E-2,-9.7446085E-4,6.006443E-4,-4.3405924E-2,-1.6063754E-2,6.2085455E-4,-2.0554965E-2,-3.1517385E-4,8.284484E-4,-2.0923575E-3,-3.2410072E-4,1.2042837E-3,2.9536688E-3,1.0651835E-2,-1.0583767E-3,2.4373692E-2,-1.1504088E-2,1.2945624E-3,3.9773103E-4,-6.5196224E-4,-1.6300872E-3,1.8731596E-4,-1.1180519E-3,-2.8640348E-2,-2.8030472E-3,-2.005989E-4,1.6326215E-2,1.5088154E-3,-9.6600816E-5,-8.203729E-3,-1.5285709E-3,-3.3269897E-2,-3.6443013E-4,-5.1205116E-4,1.5414471E-4,-1.0477238E-4,7.250899E-4,-1.2431392E-2,1.6194817E-2,-3.751738E-2,-2.2212388E-4,-1.4666366E-2,5.4519693E-4,9.904711E-4,-5.2141335E-5,-8.877339E-4,-1.7978863E-3,-5.9809885E-4,1.2637637E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":122,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,15,17,19,21,23,-1,-1,25,27,-1,29,-1,-1,31,33,35,37,-1,-1,39,41,43,-1,-1,-1,-1,45,47,-1,49,-1,-1,-1,51,-1,53,55,57,-1,59,-1,61,63,65,67,69,71,73,75,77,-1,-1,79,81,-1,83,-1,-1,-1,-1,-1,-1,85,-1,87,89,-1,-1,-1,-1,-1,-1,91,93,-1,95,-1,-1,97,-1,99,-1,-1,-1,-1,-1,101,103,105,-1,107,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1662595E-1,1.182021E-1,2.1027225E-1,1.4868028E-1,1.2941106E-1,0E0,4.477915E-2,5.9272707E-2,0E0,0E0,0E0,3.3213306E-2,5.8075592E-2,4.5980655E-2,8.8413276E-2,3.6122963E-2,0E0,0E0,2.3922827E-2,4.3744907E-2,0E0,5.191212E-2,0E0,0E0,1.226281E-2,4.2395443E-3,8.805677E-3,2.454207E-2,0E0,0E0,2.9357573E-2,1.7041923E-3,1.5620213E-2,0E0,0E0,0E0,0E0,1.1777058E-2,2.2652628E-2,0E0,2.2867035E-2,0E0,0E0,0E0,1.0118935E-2,0E0,3.606692E-3,1.1387613E-2,2.6756808E-2,0E0,1.915079E-2,0E0,5.745746E-3,2.977157E-3,5.457283E-3,2.4518874E-3,8.046521E-3,2.2992827E-3,7.3211077E-3,1.0067341E-2,1.0973215E-3,0E0,0E0,1.3194233E-3,3.8791809E-3,0E0,5.373899E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.730425E-3,0E0,5.684058E-3,6.9734175E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.7079618E-3,1.3695154E-3,0E0,1.6515166E-3,0E0,0E0,6.1839567E-3,0E0,1.981169E-3,0E0,0E0,0E0,0E0,0E0,3.5303314E-3,2.386796E-3,1.5577003E-3,0E0,3.4936192E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,11,11,12,12,13,13,14,14,15,15,18,18,19,19,21,21,24,24,25,25,26,26,27,27,30,30,31,31,32,32,37,37,38,38,40,40,44,44,46,46,47,47,48,48,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,63,63,64,64,66,66,73,73,75,75,76,76,83,83,84,84,86,86,89,89,91,91,97,97,98,98,99,99,101,101],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,16,18,20,22,24,-1,-1,26,28,-1,30,-1,-1,32,34,36,38,-1,-1,40,42,44,-1,-1,-1,-1,46,48,-1,50,-1,-1,-1,52,-1,54,56,58,-1,60,-1,62,64,66,68,70,72,74,76,78,-1,-1,80,82,-1,84,-1,-1,-1,-1,-1,-1,86,-1,88,90,-1,-1,-1,-1,-1,-1,92,94,-1,96,-1,-1,98,-1,100,-1,-1,-1,-1,-1,102,104,106,-1,108,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0737418E9,8.9219354E8,2.85218E5,7.8667163E3,1.693215E-1,8.075585E-3,1.15540435E11,5.013E3,5.9820274E-3,4.8007953E-4,-8.2140695E-3,9.97206E5,2.054214E0,2.16292E5,6.9052E4,1.2106E4,-1.5338355E-3,6.4935964E-3,9.9181765E10,6.9052E4,-4.1744863E-3,1.1E1,4.996933E-3,4.218405E-3,6.64063E5,1.8659E4,1.515252E10,1.02E3,-3.594116E-3,3.694316E-3,1.3708E4,9.26E2,1.1230755E-1,1.5638848E-3,3.7858249E-3,1.2605169E-3,-7.8125286E-4,4.094E3,3.584604E7,-2.2293036E-3,5.671E3,-1.877565E-4,5.5477803E-4,2.7863944E-3,1.3798331E10,-2.431625E-3,2.77E2,9.532393E-6,4.6592E4,2.4763613E-3,3.1561361E10,1.7120627E-3,9.9E1,2.3813066E9,3.0597E5,1.7088E4,2.63E2,4.7607956E2,1.6942586E7,5.275E3,6.6530247E0,-9.7446085E-4,6.006443E-4,4.4671544E7,8.0793E4,6.2085455E-4,1.4394E4,-3.1517385E-4,8.284484E-4,-2.0923575E-3,-3.2410072E-4,1.2042837E-3,2.9536688E-3,2.287572E6,-1.0583767E-3,6.6578E4,8.260109E9,1.2945624E-3,3.9773103E-4,-6.5196224E-4,-1.6300872E-3,1.8731596E-4,-1.1180519E-3,1.2779106E7,1.2051E4,-2.005989E-4,2.930405E-5,1.5088154E-3,-9.6600816E-5,4.23859E5,-1.5285709E-3,1.6397E4,-3.6443013E-4,-5.1205116E-4,1.5414471E-4,-1.0477238E-4,7.250899E-4,7.7643576E7,1.4821E4,8.120952E-3,-2.2212388E-4,1.4758104E9,5.4519693E-4,9.904711E-4,-5.2141335E-5,-8.877339E-4,-1.7978863E-3,-5.9809885E-4,1.2637637E-4],"split_indices":[16,16,12,5,63,0,1,3,0,0,0,28,59,22,7,8,0,0,10,7,0,4,0,0,12,7,11,0,0,0,8,17,63,0,0,0,0,12,16,0,7,0,0,0,10,0,17,64,12,0,23,0,19,10,2,7,4,5,18,3,60,0,0,10,12,0,22,0,0,0,0,0,0,2,0,8,11,0,0,0,0,0,0,18,7,0,63,0,0,12,0,7,0,0,0,0,0,18,7,63,0,21,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.2E2,6E1,2.14E2,6E0,5E0,5.5E1,2.11E2,3E0,3E0,3E0,3.7E1,1.8E1,1.16E2,9.5E1,3.3E1,4E0,3E0,1.5E1,1.13E2,3E0,9.2E1,3E0,3E0,3E1,8E0,7E0,1.09E2,4E0,3E0,8.9E1,1.2E1,1.8E1,4E0,4E0,4E0,3E0,6.5E1,4.4E1,5E0,8.4E1,8E0,4E0,5E0,1.3E1,4E0,6.1E1,1.7E1,2.7E1,3E0,8.1E1,5E0,8E0,2.1E1,4E1,6E0,1.1E1,7E0,2E1,6.9E1,1.2E1,3E0,5E0,1.3E1,8E0,3E0,3.7E1,3E0,3E0,6E0,5E0,4E0,3E0,1.6E1,4E0,8E0,6.1E1,9E0,3E0,4E0,9E0,4E0,4E0,2.5E1,1.2E1,4E0,1.2E1,4E0,4E0,5.7E1,4E0,1.9E1,6E0,4E0,8E0,3E0,9E0,4.9E1,8E0,1.6E1,3E0,4.6E1,3E0,4E0,4E0,1.2E1,4E0,3.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[5.7570203E-3,-3.4260373E-3,7.144818E-2,-1.1368298E-2,2.7187005E-2,1.1528898E-2,3.3376053E-2,-7.7660647E-3,-8.69528E-2,8.31521E-2,2.251099E-3,6.434814E-2,-6.422199E-3,-1.2746226E-2,1.850981E-2,-1.170753E-3,-4.541441E-3,-3.3347373E-4,1.2659936E-1,1.1747414E-2,-1.9594964E-3,3.198394E-2,1.144001E-1,-3.501158E-2,-7.2100614E-3,4.115972E-2,-6.502698E-2,6.4452514E-3,2.3806386E-3,4.2582E-2,-1.5898062E-3,5.700846E-2,-2.155357E-3,2.3735384E-3,4.778371E-3,-1.7952247E-3,-1.9767087E-2,1.794464E-3,-9.121167E-3,-2.6390167E-2,6.388022E-2,-3.0779636E-3,-5.850507E-4,1.8936215E-4,1.8498936E-3,1.3714205E-2,-2.4386842E-2,3.903438E-2,2.5334163E-3,-1.2735488E-3,1.754221E-3,1.9619295E-4,-2.8336793E-2,-6.7353947E-3,-1.8189752E-3,2.9314452E-4,-1.7797205E-3,7.6241486E-2,4.513011E-4,-2.13458E-4,9.017662E-4,-1.6775298E-3,-5.058306E-3,2.237409E-3,2.2043464E-4,4.4191378E-4,-4.308621E-4,-3.866452E-2,-6.482796E-5,-1.5129089E-2,7.1592843E-3,8.648214E-4,9.1753386E-2,5.9982896E-4,-8.8476675E-4,-1.7449461E-3,-6.201627E-4,-1.1259652E-2,-4.3113768E-2,3.3791617E-2,-1.917029E-3,3.6702172E-3,1.3592772E-3,-3.635429E-2,-6.390149E-3,-2.7831827E-4,-2.2196989E-3,3.0168824E-4,1.4430203E-3,2.3583963E-3,-1.1816577E-3,-4.958378E-2,-2.5967927E-5,2.0296494E-2,-9.794231E-3,-5.650788E-4,6.541335E-3,-7.0903916E-4,-1.9864491E-3,-1.5230913E-4,1.2956244E-3,3.6145633E-3,-1.569032E-2,2.2764059E-2,-3.7359595E-3,1.2824387E-2,-4.959518E-4,-2.1757944E-2,-6.3666897E-3,3.4478024E-4,1.3255059E-3,5.1700583E-5,-3.9786097E-4,7.6630875E-4,5.0827442E-5,-2.5959892E-4,-8.887501E-4,-3.2117352E-4,2.2406552E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":123,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,-1,-1,-1,27,29,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,51,-1,53,55,57,-1,-1,-1,-1,59,61,63,-1,-1,-1,65,67,69,-1,-1,-1,71,-1,-1,-1,-1,73,-1,-1,-1,-1,75,-1,77,79,-1,81,-1,-1,-1,-1,83,85,87,89,-1,-1,91,93,-1,-1,-1,-1,95,-1,97,-1,99,101,-1,103,-1,-1,-1,-1,105,107,109,111,113,-1,115,117,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.721698E-1,6.1336394E-2,3.7010056E-1,5.3779315E-2,7.2380915E-2,0E0,2.4365892E-1,2.5360681E-2,1.9512169E-2,6.631697E-2,2.289527E-2,4.331789E-2,0E0,1.9716436E-2,6.1331622E-2,0E0,0E0,0E0,2.7828217E-2,1.3638871E-2,0E0,1.6354945E-2,4.2827874E-3,9.024721E-3,1.6270285E-2,4.0054988E-2,8.295886E-3,0E0,0E0,5.1256195E-3,8.7077115E-3,1.217451E-3,2.1230368E-2,0E0,0E0,0E0,3.4271479E-3,0E0,1.4090715E-2,7.6845028E-3,1.0641351E-2,0E0,0E0,0E0,0E0,4.5500817E-3,5.095847E-3,7.3707933E-3,0E0,0E0,0E0,1.4703193E-3,3.7280973E-3,1.4453521E-2,0E0,0E0,0E0,9.023234E-3,0E0,0E0,0E0,0E0,4.2319554E-3,0E0,0E0,0E0,0E0,1.9710222E-3,0E0,7.852497E-3,1.142578E-2,0E0,7.3160306E-3,0E0,0E0,0E0,0E0,8.199381E-3,7.420929E-3,2.8143218E-3,5.4417527E-3,0E0,0E0,5.1330375E-3,5.558557E-3,0E0,0E0,0E0,0E0,2.8636255E-3,0E0,1.3143476E-3,0E0,3.6863615E-3,4.2468635E-3,0E0,4.7533656E-3,0E0,0E0,0E0,0E0,3.1758214E-3,1.9546486E-3,2.0004069E-3,8.491402E-4,1.444391E-3,0E0,1.5403684E-3,8.3948567E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,36,36,38,38,39,39,40,40,45,45,46,46,47,47,51,51,52,52,53,53,57,57,62,62,67,67,69,69,70,70,72,72,77,77,78,78,79,79,80,80,83,83,84,84,89,89,91,91,93,93,94,94,96,96,101,101,102,102,103,103,104,104,105,105,107,107,108,108],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,-1,-1,-1,28,30,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,52,-1,54,56,58,-1,-1,-1,-1,60,62,64,-1,-1,-1,66,68,70,-1,-1,-1,72,-1,-1,-1,-1,74,-1,-1,-1,-1,76,-1,78,80,-1,82,-1,-1,-1,-1,84,86,88,90,-1,-1,92,94,-1,-1,-1,-1,96,-1,98,-1,100,102,-1,104,-1,-1,-1,-1,106,108,110,112,114,-1,116,118,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.0600757E3,8.1188E4,4.06635E5,1.24389E5,4.2266383E9,1.1528898E-2,6.25E2,1.83E2,1.833625E6,1.412488E6,1.83E2,1.07E2,-6.422199E-3,3.99076E5,1.6506441E9,-1.170753E-3,-4.541441E-3,-3.3347373E-4,2.29221E6,5.6485933E-1,-1.9594964E-3,2.595033E6,6.879718E6,1E0,1E0,1.407E3,3.618E3,6.4452514E-3,2.3806386E-3,4.564531E6,2.9424374E10,8.640031E9,2.54E2,2.3735384E-3,4.778371E-3,-1.7952247E-3,6.9355E4,1.794464E-3,6.7266E4,2.72E2,7.403912E8,-3.0779636E-3,-5.850507E-4,1.8936215E-4,1.8498936E-3,9.321421E10,1.8232E4,1.275083E9,2.5334163E-3,-1.2735488E-3,1.754221E-3,2.371E3,4.014E3,2.103005E6,-1.8189752E-3,2.9314452E-4,-1.7797205E-3,9E0,4.513011E-4,-2.13458E-4,9.017662E-4,-1.6775298E-3,4.6360745E9,2.237409E-3,2.2043464E-4,4.4191378E-4,-4.308621E-4,1.27E2,-6.482796E-5,1.606E3,1.02E2,8.648214E-4,7.284E3,5.9982896E-4,-8.8476675E-4,-1.7449461E-3,-6.201627E-4,1.6608131E-4,1.01911224E3,1.3637682E3,5.39E2,3.6702172E-3,1.3592772E-3,1.340499E6,2.433E3,-2.7831827E-4,-2.2196989E-3,3.0168824E-4,1.4430203E-3,6.8E2,-1.1816577E-3,6.78647E5,-2.5967927E-5,6.485728E8,4.699127E7,-5.650788E-4,2.3E1,-7.0903916E-4,-1.9864491E-3,-1.5230913E-4,1.2956244E-3,8.16E2,7.3747E4,2.32877E5,7.385252E8,2.3005E4,-4.959518E-4,6.6378E4,5.421011E7,3.4478024E-4,1.3255059E-3,5.1700583E-5,-3.9786097E-4,7.6630875E-4,5.0827442E-5,-2.5959892E-4,-8.887501E-4,-3.2117352E-4,2.2406552E-4],"split_indices":[5,28,12,8,21,0,4,47,2,6,47,4,0,9,11,0,0,0,6,64,0,12,12,42,38,28,7,0,0,18,10,11,19,0,0,0,6,0,28,20,11,0,0,0,0,13,7,16,0,0,0,3,3,2,0,0,0,17,0,0,0,0,1,0,0,0,0,4,0,19,4,0,7,0,0,0,0,64,5,5,4,0,0,2,20,0,0,0,0,20,0,2,0,16,16,0,19,0,0,0,0,17,8,12,16,28,0,12,18,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.5E2,3.4E1,1.99E2,5.1E1,3E0,3.1E1,1.91E2,8E0,1.5E1,3.6E1,2.8E1,3E0,1.61E2,3E1,5E0,3E0,5E0,1E1,3.2E1,4E0,1.8E1,1E1,3.1E1,1.3E2,2.4E1,6E0,3E0,7E0,9E0,2.3E1,1E1,8E0,6E0,4E0,1.2E1,1.9E1,3E0,1.27E2,6E0,1.8E1,3E0,3E0,3E0,6E0,1.4E1,9E0,7E0,3E0,5E0,3E0,6E0,1.3E1,1.22E2,5E0,3E0,3E0,1.4E1,4E0,6E0,8E0,3E0,6E0,3E0,4E0,3E0,3E0,9E0,4E0,7.6E1,4.6E1,4E0,1E1,3E0,3E0,4E0,5E0,6.8E1,8E0,1.1E1,3.5E1,6E0,4E0,1E1,5.8E1,4E0,4E0,4E0,7E0,3.2E1,3E0,7E0,3E0,6E0,5.2E1,5E0,2.7E1,3E0,4E0,3E0,3E0,1.6E1,3.6E1,1E1,1.7E1,1.1E1,5E0,2.1E1,1.5E1,7E0,3E0,1.1E1,6E0,5E0,6E0,7E0,1.4E1,1.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[5.0780326E-3,-5.335303E-4,1.6281632E-1,-5.820622E-3,7.29573E-2,5.2596807E-2,1.0139405E-2,-1.3079111E-2,2.2649042E-2,1.3222842E-1,2.7363444E-2,2.9286088E-3,3.4205004E-5,-1.1084029E-2,-2.5557098E-3,3.6126315E-3,1.3237766E-2,5.675347E-3,1.4184065E-3,4.414467E-2,-5.5478234E-4,-1.6455391E-2,1.9768996E-2,2.1979956E-2,-3.540623E-2,2.6051482E-4,1.95778E-3,1.2867561E-3,-1.8339178E-2,4.7736153E-2,-4.3735646E-2,1.3526753E-3,5.055381E-2,-2.251454E-3,-2.2340313E-5,-5.1716372E-2,-1.5721269E-2,3.3575445E-3,9.1809526E-2,-4.3288385E-3,1.4367412E-2,2.8326925E-2,-5.274612E-2,2.8279537E-2,7.4861705E-2,-7.531245E-4,-2.4507472E-3,-2.3603572E-2,-9.16793E-3,-1.2280485E-3,2.8880836E-2,6.1864767E-4,1.16628475E-1,-8.3051797E-4,1.639844E-3,4.3763842E-2,1.1460226E-4,-9.6187905E-5,-2.482639E-3,1.6386118E-3,1.3431355E-2,3.0471887E-3,1.010455E-3,-1.694767E-2,-4.4140447E-2,1.487424E-2,-1.2058602E-2,1.607836E-3,2.011617E-4,4.466279E-3,1.9254499E-3,5.5155918E-2,3.231364E-4,-1.2804526E-4,7.9418236E-4,-2.280576E-2,-2.1670093E-3,-5.410078E-2,-3.030748E-4,-8.2322724E-5,8.665795E-4,-1.8747458E-2,-5.867087E-3,2.1507316E-3,8.6240866E-4,-3.052443E-2,-1.0567297E-2,4.244721E-4,-7.90566E-3,-6.8935135E-4,-2.1324519E-3,-1.14461676E-4,-2.4046008E-2,2.119499E-2,-1.0828369E-2,-3.9935615E-2,-1.3355208E-2,-2.4713385E-3,-8.0465956E-4,-2.023466E-4,-5.42678E-4,-1.0565843E-2,-3.0947996E-2,1.0085062E-3,1.8542205E-4,2.7374344E-4,-1.48445E-2,-1.0044363E-4,-1.5733761E-3,1.886729E-4,-7.21942E-4,-4.6753368E-4,2.6437204E-4,3.440946E-4,-2.85697E-4,-5.9525453E-4,2.552245E-4,-5.98764E-4,-1.5351628E-3,-9.239776E-4,-1.0369837E-2,-4.1231248E-4,1.8150448E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":124,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,-1,-1,23,-1,-1,25,-1,27,29,31,33,-1,-1,-1,35,37,39,41,43,-1,-1,45,47,49,51,-1,53,55,57,59,61,-1,-1,63,65,-1,67,-1,69,-1,-1,71,-1,-1,-1,-1,73,-1,-1,75,77,79,81,-1,-1,-1,-1,83,-1,-1,-1,85,87,89,-1,-1,-1,91,93,-1,-1,95,97,-1,99,-1,-1,-1,101,103,105,107,109,111,-1,113,-1,115,117,-1,-1,-1,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,121,-1,-1],"loss_changes":[2.5766796E-1,1.09996535E-1,1.511643E-1,5.4850977E-2,4.772737E-2,1.3750283E-2,0E0,2.7037334E-2,4.404116E-2,2.3324251E-2,9.741875E-3,0E0,0E0,3.4464195E-2,0E0,0E0,2.204098E-2,0E0,0E0,4.9187485E-3,0E0,1.9442428E-2,5.714514E-2,2.527585E-2,9.542762E-3,0E0,0E0,0E0,1.4128666E-2,4.272188E-2,5.4658614E-2,3.943527E-2,7.6288804E-3,0E0,0E0,6.517321E-3,8.048467E-3,1.3815127E-2,1.757624E-2,0E0,1.1599164E-2,6.7259567E-3,1.0687811E-2,3.011845E-3,3.915388E-3,0E0,0E0,9.093709E-3,6.3646883E-3,0E0,3.5024323E-3,0E0,1.7551631E-3,0E0,0E0,3.6724918E-3,0E0,0E0,0E0,0E0,1.6637483E-3,0E0,0E0,4.79942E-3,5.370442E-3,2.1689204E-3,3.2031396E-3,0E0,0E0,0E0,0E0,8.556228E-4,0E0,0E0,0E0,3.4560245E-3,1.6021318E-3,3.7426315E-3,0E0,0E0,0E0,2.9732287E-3,6.0027996E-3,0E0,0E0,3.3446383E-3,1.9243378E-3,0E0,8.9321705E-4,0E0,0E0,0E0,2.2781268E-3,9.1461465E-4,2.9990105E-3,4.7864653E-3,1.8749933E-3,1.6646927E-3,0E0,8.506457E-4,0E0,1.7592477E-3,2.8340258E-3,0E0,0E0,0E0,1.6249609E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0484648E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,16,16,19,19,21,21,22,22,23,23,24,24,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,47,47,48,48,50,50,52,52,55,55,60,60,63,63,64,64,65,65,66,66,71,71,75,75,76,76,77,77,81,81,82,82,85,85,86,86,88,88,92,92,93,93,94,94,95,95,96,96,97,97,99,99,101,101,102,102,106,106,120,120],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,-1,-1,24,-1,-1,26,-1,28,30,32,34,-1,-1,-1,36,38,40,42,44,-1,-1,46,48,50,52,-1,54,56,58,60,62,-1,-1,64,66,-1,68,-1,70,-1,-1,72,-1,-1,-1,-1,74,-1,-1,76,78,80,82,-1,-1,-1,-1,84,-1,-1,-1,86,88,90,-1,-1,-1,92,94,-1,-1,96,98,-1,100,-1,-1,-1,102,104,106,108,110,112,-1,114,-1,116,118,-1,-1,-1,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,122,-1,-1],"split_conditions":[1.0218795E4,1.8943199E9,2.264E5,4.29906E5,9.068E3,8.4986175E10,1.0139405E-2,3.48037E5,1.0927762E9,5.101353E7,1.09E3,2.9286088E-3,3.4205004E-5,1.83E2,-2.5557098E-3,3.6126315E-3,6.45689E5,5.675347E-3,1.4184065E-3,5.732E3,-5.5478234E-4,1E0,2.699E3,1.6208463E7,1.5042838E0,2.6051482E-4,1.95778E-3,1.2867561E-3,2.0713E4,2.500807E-2,1.277E4,2.7179317E10,4.90364E5,-2.251454E-3,-2.2340313E-5,1.3686E4,9.8927E4,1.852E3,9E0,-4.3288385E-3,6.494384E6,2.5420546E7,5.88764E5,6.79417E5,6.635E3,-7.531245E-4,-2.4507472E-3,7.1711E4,2.0906E4,-1.2280485E-3,2.017E3,6.1864767E-4,4.41E3,-8.3051797E-4,1.639844E-3,1.7867039E10,1.1460226E-4,-9.6187905E-5,-2.482639E-3,1.6386118E-3,1.360833E6,3.0471887E-3,1.010455E-3,1.916746E6,9.2701E4,8.297E4,1.8723524E3,1.607836E-3,2.011617E-4,4.466279E-3,1.9254499E-3,5.530942E9,3.231364E-4,-1.2804526E-4,7.9418236E-4,1.25688E5,2.748E3,4.96E2,-3.030748E-4,-8.2322724E-5,8.665795E-4,6.350158E-4,7.1E1,2.1507316E-3,8.6240866E-4,1.3249E4,6.2756E4,4.244721E-4,9.106517E-2,-6.8935135E-4,-2.1324519E-3,-1.14461676E-4,2.4540474E8,8.922E3,5.728E3,1.258E3,5.5296956E7,4.1106E4,-8.0465956E-4,2.351E3,-5.42678E-4,4.3199274E8,1.76375E5,1.0085062E-3,1.8542205E-4,2.7374344E-4,5.229E3,-1.0044363E-4,-1.5733761E-3,1.886729E-4,-7.21942E-4,-4.6753368E-4,2.6437204E-4,3.440946E-4,-2.85697E-4,-5.9525453E-4,2.552245E-4,-5.98764E-4,-1.5351628E-3,-9.239776E-4,5.581883E7,-4.1231248E-4,1.8150448E-4],"split_indices":[5,16,3,12,7,1,0,22,11,18,17,0,0,47,0,0,28,0,0,0,0,38,0,2,64,0,0,0,23,63,8,10,18,0,0,25,12,3,17,0,18,18,12,12,0,0,0,12,18,0,20,0,7,0,0,10,0,0,0,0,12,0,0,18,8,8,5,0,0,0,0,21,0,0,0,6,3,0,0,0,0,63,4,0,0,7,12,0,63,0,0,0,11,3,3,28,11,8,0,20,0,16,12,0,0,0,20,0,0,0,0,0,0,0,0,0,0,0,0,0,18,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.9E2,2.81E2,9E0,2.63E2,1.8E1,6E0,3E0,2.1E2,5.3E1,7E0,1.1E1,3E0,3E0,2.05E2,5E0,4E0,4.9E1,4E0,3E0,8E0,3E0,1.75E2,3E1,4.2E1,7E0,3E0,5E0,5E0,1.7E2,2.1E1,9E0,2.5E1,1.7E1,3E0,4E0,1.1E1,1.59E2,1.1E1,1E1,3E0,6E0,1.7E1,8E0,1E1,7E0,6E0,5E0,7.1E1,8.8E1,4E0,7E0,3E0,7E0,3E0,3E0,1E1,7E0,3E0,5E0,3E0,7E0,4E0,3E0,5.5E1,1.6E1,9E0,7.9E1,3E0,4E0,4E0,3E0,7E0,3E0,3E0,4E0,3.9E1,1.6E1,1.2E1,4E0,4E0,5E0,3.7E1,4.2E1,4E0,3E0,2.3E1,1.6E1,4E0,1.2E1,4E0,8E0,1E1,2.7E1,6E0,3.6E1,1.4E1,9E0,1.1E1,5E0,7E0,5E0,1E1,1.7E1,3E0,3E0,6E0,3E1,3E0,1.1E1,3E0,6E0,5E0,6E0,3E0,4E0,7E0,3E0,1.1E1,6E0,6E0,2.4E1,2.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[-4.2522843E-3,-9.463954E-4,-7.631386E-3,-6.163673E-3,2.0564392E-1,-2.2887204E-2,1.5586394E-2,9.788572E-3,1.7954904E-3,-1.6832553E-2,-7.908241E-2,4.6350416E-3,1.1101481E-2,-1.3649514E-2,-2.8080684E-3,-7.47218E-3,-2.147472E-2,-3.4274785E-3,1.5094734E-2,-1.860979E-2,2.275067E-2,-1.5549394E-3,4.315982E-4,1.44670275E-5,3.6143973E-2,-3.0740678E-2,-7.851146E-3,1.0105872E-2,1.9738204E-3,1.976688E-2,-1.1208824E-2,4.055985E-3,2.7353084E-2,-6.0445197E-2,-2.2323398E-2,-1.6368825E-2,4.7912918E-2,5.537422E-4,-3.8424978E-4,3.6243144E-2,-1.35512715E-2,-1.7101297E-2,9.322047E-4,1.51413595E-2,6.857999E-2,-7.100821E-2,-6.94223E-4,-1.6002033E-2,-4.6533227E-2,-4.955623E-3,-2.1380877E-2,-1.0205868E-4,2.3814396E-3,1.1109216E-4,1.3848804E-3,-1.0251084E-3,2.3994602E-4,-1.1433369E-2,-2.0548569E-3,3.3029478E-2,-4.5817294E-3,2.6022797E-3,9.648064E-4,-1.0049915E-3,-3.1092335E-3,-2.0893883E-2,4.95414E-3,-2.095136E-3,-6.010051E-4,6.8552734E-5,-4.8168038E-4,-2.780481E-2,-5.9000286E-3,-1.6827622E-2,8.168439E-4,1.3368147E-2,4.938786E-2,-1.6857889E-2,9.884541E-4,3.1880243E-4,-2.5823364E-2,6.4223266E-4,-4.838536E-4,-3.5020158E-2,-5.3889225E-3,2.4598735E-4,-4.9310364E-4,-2.8471977E-2,-4.878369E-3,-2.6880554E-4,9.4118254E-4,7.151173E-4,1.8624241E-3,2.3526629E-4,-2.72555E-2,-1.0878177E-3,-6.125632E-3,-4.3303184E-2,-1.9164655E-2,-4.367328E-4,1.989884E-4,-1.2419518E-2,-1.2739513E-3,4.197307E-4,-3.667813E-4,2.0335459E-4,-1.3695258E-3,3.61437E-4,-9.858054E-4,-4.9077265E-4,-1.6157407E-3,-9.6930744E-4,-1.4109508E-4,-7.868675E-4,-1.0140149E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":125,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,17,19,-1,-1,21,-1,23,25,27,-1,-1,29,31,33,35,37,-1,39,41,-1,43,45,47,49,51,-1,-1,53,55,57,-1,59,61,63,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,73,-1,75,77,-1,-1,-1,-1,79,81,-1,-1,-1,-1,83,85,87,-1,89,91,93,-1,-1,95,-1,-1,97,99,-1,-1,101,103,-1,-1,-1,-1,-1,105,-1,107,109,111,-1,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1991137E-1,3.0629596E-1,0E0,1.01112135E-1,8.631992E-2,5.2086905E-2,6.808541E-2,0E0,0E0,3.085557E-2,1.2726994E-1,0E0,5.7012122E-2,2.5339967E-2,0E0,0E0,1.18794255E-2,0E0,3.6503717E-2,1.5681103E-2,7.6711904E-3,0E0,0E0,1.5296284E-2,3.447155E-2,1.2902439E-2,3.2132983E-2,2.3955195E-3,0E0,1.4215106E-2,1.0909884E-2,0E0,2.1616925E-2,3.3965968E-3,6.2951744E-3,3.2118503E-3,1.2220392E-2,0E0,0E0,3.6312863E-3,3.6961243E-3,9.6010845E-3,0E0,1.2852453E-2,2.3115352E-3,6.145645E-3,0E0,3.999061E-3,3.4376513E-3,1.3792276E-3,3.8138684E-3,0E0,0E0,0E0,0E0,0E0,0E0,7.5752535E-3,0E0,5.4507013E-3,8.3156405E-3,0E0,0E0,0E0,0E0,4.9621174E-3,2.6978583E-3,0E0,0E0,0E0,0E0,4.3399557E-3,1.7752462E-3,4.3136245E-3,0E0,3.6912297E-3,1.0163616E-3,3.7170858E-3,0E0,0E0,4.032474E-3,0E0,0E0,1.9280724E-3,8.409625E-4,0E0,0E0,2.3607472E-3,2.1334689E-3,0E0,0E0,0E0,0E0,0E0,5.40242E-3,0E0,4.170482E-3,1.7485563E-3,1.3244885E-3,0E0,0E0,1.1568242E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,16,16,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,29,29,30,30,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,57,57,59,59,60,60,65,65,66,66,71,71,72,72,73,73,75,75,76,76,77,77,80,80,83,83,84,84,87,87,88,88,94,94,96,96,97,97,98,98,101,101],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,18,20,-1,-1,22,-1,24,26,28,-1,-1,30,32,34,36,38,-1,40,42,-1,44,46,48,50,52,-1,-1,54,56,58,-1,60,62,64,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,74,-1,76,78,-1,-1,-1,-1,80,82,-1,-1,-1,-1,84,86,88,-1,90,92,94,-1,-1,96,-1,-1,98,100,-1,-1,102,104,-1,-1,-1,-1,-1,106,-1,108,110,112,-1,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.539647E4,1.1741724E4,-7.631386E-3,4.0317974E8,3.954342E-1,2.5624927E9,1.0916E4,9.788572E-3,1.7954904E-3,1.17442E5,1.2081E4,4.6350416E-3,1.277E4,1.460109E6,-2.8080684E-3,-7.47218E-3,8.0976E4,-3.4274785E-3,2.601E3,1E0,6.648864E6,-1.5549394E-3,4.315982E-4,9.5E1,2.839147E6,2.719E3,1.83E2,7.51885E5,1.9738204E-3,1.6121E4,1.6583E4,4.055985E-3,2.51383E5,3.36E2,1.5581422E10,1.646258E-2,4.1E1,5.537422E-4,-3.8424978E-4,1.6E1,3.9182508E9,5.71934E5,9.322047E-4,5.2944245E10,3.9461447E11,1.77E2,-6.94223E-4,4.6585E5,3.2162281E10,4.058E3,1.3953088E8,-1.0205868E-4,2.3814396E-3,1.1109216E-4,1.3848804E-3,-1.0251084E-3,2.3994602E-4,5.10607E5,-2.0548569E-3,3.2E1,1.03880065E11,2.6022797E-3,9.648064E-4,-1.0049915E-3,-3.1092335E-3,7.656413E7,7.11868E5,-2.095136E-3,-6.010051E-4,6.8552734E-5,-4.8168038E-4,1.6404E4,1.2554E4,3.649951E9,8.168439E-4,3.294E3,3.482E4,4.69698E5,9.884541E-4,3.1880243E-4,2E1,6.4223266E-4,-4.838536E-4,1.5025306E-1,2.34E2,2.4598735E-4,-4.9310364E-4,1.3316E4,2.486228E3,-2.6880554E-4,9.4118254E-4,7.151173E-4,1.8624241E-3,2.3526629E-4,3.8639E4,-1.0878177E-3,1.94784E5,2E0,2.9E1,-4.367328E-4,1.989884E-4,1.09E2,-1.2739513E-3,4.197307E-4,-3.667813E-4,2.0335459E-4,-1.3695258E-3,3.61437E-4,-9.858054E-4,-4.9077265E-4,-1.6157407E-3,-9.6930744E-4,-1.4109508E-4,-7.868675E-4,-1.0140149E-5],"split_indices":[5,5,0,16,63,23,8,0,0,8,7,0,8,6,0,0,8,0,0,42,6,0,0,4,2,3,47,18,0,7,7,0,3,20,10,67,17,0,0,17,10,12,0,23,1,20,0,6,10,3,16,0,0,0,0,0,0,12,0,4,10,0,0,0,0,10,6,0,0,0,0,7,7,10,0,0,8,28,0,0,19,0,0,63,19,0,0,7,5,0,0,0,0,0,18,0,6,19,19,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,2.82E2,3E0,2.76E2,6E0,1.56E2,1.2E2,3E0,3E0,1.42E2,1.4E1,3E0,1.17E2,1.37E2,5E0,3E0,1.1E1,3E0,1.14E2,1.21E2,1.6E1,6E0,5E0,6.7E1,4.7E1,5.6E1,6.5E1,1.3E1,3E0,2.4E1,4.3E1,3E0,4.4E1,1.1E1,4.5E1,5.7E1,8E0,1E1,3E0,1.6E1,8E0,3.8E1,5E0,3.5E1,9E0,8E0,3E0,3.7E1,8E0,1.8E1,3.9E1,3E0,5E0,3E0,1.3E1,4E0,4E0,3.5E1,3E0,1.8E1,1.7E1,6E0,3E0,4E0,4E0,3E1,7E0,4E0,4E0,1.1E1,7E0,2.7E1,1.2E1,3.1E1,4E0,9E0,9E0,1.3E1,4E0,4E0,2.6E1,4E0,3E0,2E1,7E0,5E0,7E0,1.5E1,1.6E1,4E0,5E0,3E0,6E0,4E0,9E0,1.8E1,8E0,1.2E1,8E0,4E0,3E0,7E0,8E0,4E0,1.2E1,3E0,6E0,5E0,3E0,3E0,9E0,4E0,4E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[3.37144E-3,-6.685987E-3,6.607129E-2,-1.3818216E-2,4.7713745E-2,1.4669803E-1,4.2368446E-2,-3.89442E-2,-4.113522E-3,7.6181213E-3,1.4233169E-2,7.76864E-4,6.9337995E-3,6.455095E-2,8.395094E-3,-2.6575511E-2,-6.315005E-3,-7.0311264E-3,3.3285774E-3,1.1721363E-3,1.1546605E-3,2.5564476E-4,8.57127E-2,-1.0901042E-3,2.4459934E-2,-9.112506E-2,-1.5022917E-2,1.0928108E-2,-1.8655896E-2,1.929894E-2,-1.9367035E-2,3.3226716E-3,1.1925981E-3,2.8370126E-4,1.4718724E-3,-4.4055986E-3,-1.1686746E-3,1.7103729E-3,-2.0877697E-2,1.7265495E-2,-1.9217222E-3,-1.5306079E-2,-2.0100079E-3,-1.4746664E-4,1.0301303E-3,-1.118322E-3,-1.027899E-4,-2.1375369E-3,-1.3338032E-2,4.6028543E-2,-1.0077269E-3,-1.7560344E-2,7.668499E-4,-2.1016674E-2,8.1182085E-3,1.2330939E-2,8.9588575E-2,-2.779537E-2,9.747639E-3,1.2830739E-2,-2.0197874E-2,-2.6420906E-2,1.3207254E-4,1.0719072E-3,-5.9282675E-4,1.965341E-3,-1.6108846E-2,5.488356E-4,3.959754E-3,-6.7416746E-5,-1.4349928E-3,2.1824818E-3,1.5746548E-3,9.0611784E-4,-1.8335358E-4,-3.5772186E-2,-1.1521754E-2,-1.996835E-3,-1.2395369E-3,2.3358567E-4,-9.335143E-4,-4.0381183E-3,7.770462E-4,-2.3311608E-2,-1.9601316E-3,2.9832139E-3,-2.0807674E-2,3.296217E-4,-4.4531855E-4,1.9664434E-3,-8.222894E-4,-9.793793E-4,-9.0973364E-5,1.19345505E-2,-2.9882204E-4,-3.192872E-2,-1.3650714E-2,2.184078E-4,-4.178568E-4,6.551975E-4,3.9245E-6,2.5363084E-5,-1.425841E-3,-2.0293212E-4,-7.6874095E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":126,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,-1,27,-1,29,-1,-1,31,-1,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,47,49,-1,51,-1,-1,-1,-1,-1,-1,53,55,57,59,-1,61,63,65,67,69,71,73,75,77,-1,-1,-1,-1,79,-1,-1,-1,-1,81,-1,-1,-1,83,85,87,-1,-1,-1,89,-1,91,-1,93,95,-1,-1,97,-1,-1,-1,99,-1,101,103,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6881171E-1,9.028531E-2,6.44947E-2,4.9826995E-2,1.6744217E-1,6.2173158E-2,2.2066504E-2,1.0564375E-1,4.746891E-2,0E0,6.7407517E-3,0E0,0E0,2.0882271E-2,9.654239E-3,3.8899638E-2,0E0,3.068458E-2,0E0,6.330716E-3,0E0,0E0,7.2534606E-3,0E0,2.924344E-3,1.5090704E-2,1.986478E-2,2.6991904E-2,1.2098942E-2,2.8769088E-3,1.8784706E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4218042E-2,2.8428433E-2,0E0,7.827835E-3,0E0,0E0,0E0,0E0,0E0,0E0,6.5960544E-3,2.9720534E-2,1.006671E-2,6.6717453E-3,0E0,4.045236E-3,7.9637E-3,1.8998176E-2,1.9725338E-2,4.220514E-3,7.301691E-3,2.1470704E-3,9.587061E-3,7.008372E-3,0E0,0E0,0E0,0E0,2.9215575E-3,0E0,0E0,0E0,0E0,3.1024576E-3,0E0,0E0,0E0,6.7194663E-3,6.6619804E-3,1.4453919E-3,0E0,0E0,0E0,2.3748185E-3,0E0,2.6830658E-3,0E0,2.3068392E-3,1.9519366E-3,0E0,0E0,1.168524E-3,0E0,0E0,0E0,1.1910914E-3,0E0,4.4856677E-3,1.3527013E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,17,17,19,19,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,38,38,39,39,41,41,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,66,66,71,71,75,75,76,76,77,77,81,81,83,83,85,85,86,86,89,89,93,93,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,-1,28,-1,30,-1,-1,32,-1,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,48,50,-1,52,-1,-1,-1,-1,-1,-1,54,56,58,60,-1,62,64,66,68,70,72,74,76,78,-1,-1,-1,-1,80,-1,-1,-1,-1,82,-1,-1,-1,84,86,88,-1,-1,-1,90,-1,92,-1,94,96,-1,-1,98,-1,-1,-1,100,-1,102,104,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1328217E10,4.805687E7,1.6940264E10,9.0037585E2,5.218E3,6.8917E4,1.3628324E9,4.4032362E8,3.102E4,7.6181213E-3,1.6429902E9,7.76864E-4,6.9337995E-3,2.0794068E7,5.352599E3,9.05E3,-6.315005E-3,1.02E2,3.3285774E-3,1.7501605E3,1.1546605E-3,2.5564476E-4,2.2338E4,-1.0901042E-3,3.9704165E0,6.3E1,2.348E3,4.3740602E10,5.59456E5,1.01055334E3,3.067249E8,3.3226716E-3,1.1925981E-3,2.8370126E-4,1.4718724E-3,-4.4055986E-3,-1.1686746E-3,1.7103729E-3,7.273E3,4.3501E4,-1.9217222E-3,1.9557E4,-2.0100079E-3,-1.4746664E-4,1.0301303E-3,-1.118322E-3,-1.027899E-4,-2.1375369E-3,4.352E3,4.1443497E-1,9.8927E4,6.313056E-7,7.668499E-4,4.14853E5,1.0922E4,1.062E3,1.6E1,8.388608E7,8E1,2.25E2,1.3E1,7.656413E7,1.3207254E-4,1.0719072E-3,-5.9282675E-4,1.965341E-3,1.6506441E9,5.488356E-4,3.959754E-3,-6.7416746E-5,-1.4349928E-3,1E0,1.5746548E-3,9.0611784E-4,-1.8335358E-4,5.598581E7,1.5200496E8,4.9418E4,-1.2395369E-3,2.3358567E-4,-9.335143E-4,8.8157E4,7.770462E-4,3.71E3,-1.9601316E-3,1.219769E6,1.8703245E3,3.296217E-4,-4.4531855E-4,1.355E3,-8.222894E-4,-9.793793E-4,-9.0973364E-5,2.7700895E-2,-2.9882204E-4,1.4366396E3,4.5697078E-1,2.184078E-4,-4.178568E-4,6.551975E-4,3.9245E-6,2.5363084E-5,-1.425841E-3,-2.0293212E-4,-7.6874095E-4],"split_indices":[21,18,21,5,7,3,16,23,7,0,16,0,0,2,5,18,0,4,0,5,0,0,7,0,59,4,7,26,12,5,11,0,0,0,0,0,0,0,7,8,0,7,0,0,0,0,0,0,3,64,12,63,0,6,20,0,17,16,4,4,19,10,0,0,0,0,11,0,0,0,0,34,0,0,0,16,21,8,0,0,0,8,0,20,0,2,5,0,0,19,0,0,0,67,0,5,63,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.3E2,3.6E1,2.04E2,2.6E1,7E0,2.9E1,5.6E1,1.48E2,3E0,2.3E1,3E0,4E0,1.7E1,1.2E1,5.3E1,3E0,1.45E2,3E0,1.5E1,8E0,5E0,1.2E1,3E0,9E0,7E0,4.6E1,5.7E1,8.8E1,8E0,7E0,8E0,4E0,6E0,3E0,3E0,4E0,3E0,4.3E1,5.3E1,4E0,8.3E1,5E0,3E0,5E0,3E0,4E0,5E0,3.8E1,2E1,3.3E1,7.9E1,4E0,2.8E1,1E1,1.2E1,8E0,9E0,2.4E1,6E0,7.3E1,2.3E1,5E0,5E0,5E0,4E0,8E0,3E0,5E0,4E0,5E0,2.1E1,3E0,3E0,3E0,2.5E1,4.8E1,8E0,1.5E1,3E0,5E0,1.7E1,4E0,1.8E1,7E0,1.9E1,2.9E1,4E0,4E0,1.4E1,3E0,1.3E1,5E0,1.1E1,8E0,1E1,1.9E1,1.1E1,3E0,6E0,5E0,3E0,7E0,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[5.0951038E-3,-2.7603367E-3,8.0130085E-2,6.555038E-4,-6.828709E-3,-1.5797189E-3,1.1323051E-1,-2.5434315E-3,4.1535613E-3,9.994573E-3,6.3102946E-2,1.0117501E-1,-6.078492E-3,3.2559964E-3,2.9693853E-2,-3.463893E-4,6.946423E-3,-7.509094E-2,-2.3375596E-3,-1.970574E-4,1.9493835E-3,-7.439152E-3,-6.7125154E-3,-8.627926E-3,4.0040337E-2,1.4242603E-3,-1.8552857E-3,-1.9421866E-2,4.7172275E-3,7.92242E-2,-9.442668E-2,1.5504928E-3,-2.1811767E-2,-1.9627996E-3,2.0722015E-2,4.722333E-2,1.1610299E-1,-4.274715E-3,-1.0444033E-3,-4.648059E-2,-1.782073E-2,1.1777815E-2,-6.502113E-3,3.612469E-2,-2.5816262E-4,5.747886E-2,2.3223642E-4,1.364824E-3,6.9546443E-3,-2.1211284E-3,-2.8372373E-2,-2.0646276E-2,1.0531097E-2,1.7514713E-4,8.5849594E-4,9.590975E-4,-1.7940754E-2,1.4302724E-3,5.2325334E-4,-9.505804E-4,4.114002E-4,2.5185714E-3,3.4435835E-2,-3.5894304E-4,-1.2849559E-3,-1.8409362E-2,-1.6837283E-3,-1.4435653E-4,9.792511E-4,4.282734E-4,-2.3667364E-4,4.63377E-4,-5.484873E-3,-9.804623E-4,-1.9529357E-4,3.2061868E-4,1.6191752E-3,-2.0974955E-2,1.6472756E-3,-3.456891E-4,4.33823E-4,-2.3306211E-2,-2.3256077E-3,5.954919E-4,-6.085421E-4,-8.239135E-4,-1.4045165E-4,3.2507553E-4,-4.5982155E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":127,"left_children":[1,3,5,7,-1,-1,9,11,-1,-1,13,15,17,-1,19,-1,-1,21,23,-1,-1,25,-1,27,29,-1,-1,31,33,35,37,-1,39,41,43,45,47,-1,-1,49,51,53,55,57,59,61,-1,-1,-1,-1,63,65,67,69,-1,71,73,-1,-1,-1,-1,-1,75,-1,-1,77,-1,-1,-1,-1,-1,-1,79,-1,-1,-1,-1,81,83,-1,-1,85,87,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6172579E-1,1.7822953E-1,1.1675349E-1,1.0135635E-1,0E0,0E0,1.8806493E-1,8.8977866E-2,0E0,0E0,1.9878052E-2,1.04968965E-1,6.0236454E-2,0E0,1.2531428E-2,0E0,0E0,1.06781214E-1,5.979114E-2,0E0,0E0,2.5901163E-2,0E0,2.8180743E-2,1.6028003E-1,0E0,0E0,1.8496323E-2,9.448966E-3,2.166076E-2,1.2340266E-2,0E0,9.509366E-3,4.006077E-3,8.411248E-3,5.3296275E-3,6.450769E-2,0E0,0E0,3.0525047E-3,7.587487E-3,2.7600487E-3,4.1138204E-3,1.7540567E-3,4.9577164E-3,2.5688633E-3,0E0,0E0,0E0,0E0,1.3445062E-3,4.986245E-3,2.8245188E-3,1.0862115E-3,0E0,2.6764758E-3,2.6394408E-3,0E0,0E0,0E0,0E0,0E0,2.217303E-3,0E0,0E0,4.1511226E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.2371174E-3,0E0,0E0,0E0,0E0,3.0326229E-3,3.8133913E-3,0E0,0E0,2.6211329E-3,1.4815866E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,10,10,11,11,12,12,14,14,17,17,18,18,21,21,23,23,24,24,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,50,50,51,51,52,52,53,53,55,55,56,56,62,62,65,65,72,72,77,77,78,78,81,81,82,82],"right_children":[2,4,6,8,-1,-1,10,12,-1,-1,14,16,18,-1,20,-1,-1,22,24,-1,-1,26,-1,28,30,-1,-1,32,34,36,38,-1,40,42,44,46,48,-1,-1,50,52,54,56,58,60,62,-1,-1,-1,-1,64,66,68,70,-1,72,74,-1,-1,-1,-1,-1,76,-1,-1,78,-1,-1,-1,-1,-1,-1,80,-1,-1,-1,-1,82,84,-1,-1,86,88,-1,-1,-1,-1,-1,-1],"split_conditions":[6.28354E5,1.3195825E11,6.018E3,1.0778861E4,-6.828709E-3,-1.5797189E-3,1.066E4,3.308E3,4.1535613E-3,9.994573E-3,1.1003741E10,2.2032343E-4,1.5262E4,3.2559964E-3,8.0689E4,-3.463893E-4,6.946423E-3,5.987E3,1.83E2,-1.970574E-4,1.9493835E-3,4.7E1,-6.7125154E-3,1.530129E6,5.814E3,1.4242603E-3,-1.8552857E-3,4.3E1,6.648864E6,1.81E2,2.748775E6,1.5504928E-3,2.4253E4,5.133E3,1.7193291E11,1.7156E4,4.1106E4,-4.274715E-3,-1.0444033E-3,2.719E3,7.76E3,8.1214E4,1.669E3,4.58E2,2.1120247E9,6.216E4,2.3223642E-4,1.364824E-3,6.9546443E-3,-2.1211284E-3,3.56279E5,2.215E3,1.1813E4,1.501E4,8.5849594E-4,2.44E2,2.0408E4,1.4302724E-3,5.2325334E-4,-9.505804E-4,4.114002E-4,2.5185714E-3,5.671E3,-3.5894304E-4,-1.2849559E-3,9.547999E8,-1.6837283E-3,-1.4435653E-4,9.792511E-4,4.282734E-4,-2.3667364E-4,4.63377E-4,5.581883E7,-9.804623E-4,-1.9529357E-4,3.2061868E-4,1.6191752E-3,1.253E3,1.06E3,-3.456891E-4,4.33823E-4,1.6744E4,7.2391014E8,5.954919E-4,-6.085421E-4,-8.239135E-4,-1.4045165E-4,3.2507553E-4,-4.5982155E-4],"split_indices":[28,26,7,5,0,0,7,7,0,0,11,64,8,0,8,0,0,3,47,0,0,4,0,9,0,0,0,22,6,19,18,0,12,3,13,7,8,0,0,3,3,8,0,17,1,28,0,0,0,0,2,19,7,18,0,17,7,0,0,0,0,0,7,0,0,11,0,0,0,0,0,0,18,0,0,0,0,17,0,0,0,7,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.48E2,2.5E1,2.45E2,3E0,5E0,2E1,2.4E2,5E0,3E0,1.7E1,7E0,2.33E2,7E0,1E1,4E0,3E0,1.1E1,2.22E2,5E0,5E0,8E0,3E0,1.94E2,2.8E1,4E0,4E0,1.07E2,8.7E1,2.2E1,6E0,3E0,1.04E2,6.2E1,2.5E1,1.3E1,9E0,3E0,3E0,1.3E1,9.1E1,1.5E1,4.7E1,1.4E1,1.1E1,1E1,3E0,6E0,3E0,5E0,8E0,8.3E1,8E0,9E0,6E0,2.9E1,1.8E1,9E0,5E0,3E0,8E0,4E0,6E0,4E0,4E0,7.9E1,4E0,5E0,3E0,3E0,6E0,9E0,2E1,8E0,1E1,3E0,3E0,7E1,9E0,1.6E1,4E0,6.2E1,8E0,5E0,4E0,5.5E1,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[7.326637E-3,-2.2633914E-4,1.5570483E-1,-2.4808489E-2,1.8842036E-2,1.0863859E-2,7.544155E-2,-3.707359E-3,-1.9920498E-2,6.3485056E-3,1.2850175E-2,3.8653172E-3,5.8672164E-4,-1.605471E-2,-3.3003362E-3,8.415308E-3,3.842162E-3,1.06155295E-2,-2.1967975E-2,3.072852E-2,-8.221725E-3,-1.7366698E-2,6.9060944E-2,-2.4891073E-2,5.4987316E-4,1.0483339E-1,1.804357E-2,-4.178971E-3,-2.1895333E-3,-4.5778826E-2,2.1644142E-2,3.1822042E-3,7.0804544E-4,3.3338368E-4,-2.6289888E-2,6.3982955E-3,7.9618604E-4,8.736556E-3,2.3057018E-3,-1.0005664E-2,1.9581988E-2,-2.0736724E-3,-2.0503024E-4,2.5634074E-5,1.1935951E-3,-2.29462E-2,-1.6491323E-3,-1.90592E-3,1.625924E-2,-2.3981038E-2,-4.9271495E-3,3.1248575E-2,-1.3110496E-3,-1.4373785E-3,-1.9687692E-2,1.5520379E-3,3.3830278E-2,-2.956879E-4,-1.3419879E-3,8.588607E-4,-8.572811E-3,4.2210665E-2,1.7161626E-2,-2.8492648E-2,-1.0823372E-2,1.1896175E-2,-1.7938499E-3,5.0215647E-2,1.2423349E-2,5.225527E-3,-1.4266331E-2,1.6072246E-3,5.8415055E-4,-2.1567832E-4,9.124088E-4,-2.270367E-2,-1.6338825E-3,-2.1517437E-2,-5.025366E-3,1.1464835E-3,1.2994986E-3,2.826187E-2,2.6256305E-3,-2.9777904E-4,7.4143754E-4,-1.9220402E-4,6.207146E-4,-6.399242E-3,-6.9871946E-4,-2.5122482E-2,-6.148676E-5,-3.0438937E-2,-9.462108E-5,3.0956988E-4,-9.384863E-3,2.3650263E-4,-3.684436E-4,1.2320472E-3,2.793815E-4,3.27775E-4,-1.48622645E-2,-4.28364E-4,-9.95814E-4,-1.3317862E-3,-2.9486278E-4,-8.909636E-4,-1.3598974E-4,-9.288788E-4,-2.7428009E-3,3.0623918E-4,-4.60743E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":128,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,15,-1,-1,17,-1,19,-1,21,23,25,27,29,31,33,-1,35,37,-1,39,41,43,-1,-1,-1,45,-1,-1,47,-1,49,51,-1,-1,-1,-1,53,-1,-1,55,57,59,61,-1,-1,63,65,67,-1,-1,-1,69,71,73,75,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,-1,91,93,95,-1,97,-1,-1,-1,-1,-1,99,-1,101,-1,103,-1,-1,105,-1,-1,-1,-1,-1,107,-1,-1,-1,-1,-1,-1,-1,109,-1,-1],"loss_changes":[3.2391664E-1,1.2984012E-1,1.7882195E-1,5.115255E-2,1.6420665E-1,0E0,2.5905974E-2,0E0,3.5398647E-2,0E0,7.0795625E-2,0E0,0E0,1.7896518E-2,0E0,5.4822817E-2,0E0,3.554345E-2,1.0926902E-2,5.7329472E-2,6.206678E-2,1.7616559E-2,7.642783E-3,4.512567E-3,0E0,6.280328E-2,2.6804438E-2,0E0,1.41497925E-2,6.202027E-3,2.223223E-3,0E0,0E0,0E0,5.746726E-3,0E0,0E0,2.5497584E-2,0E0,4.211264E-3,1.6752664E-2,0E0,0E0,0E0,0E0,4.426077E-3,0E0,0E0,1.1318011E-2,3.8093245E-3,5.5295806E-3,2.2096243E-3,0E0,0E0,4.9209595E-3,1.5475963E-2,6.3913595E-3,0E0,0E0,0E0,3.3741156E-3,9.516105E-4,2.8585973E-3,3.1012818E-3,2.0620166E-3,6.687804E-3,0E0,5.2663404E-3,2.5128524E-3,2.161366E-3,1.6171383E-3,0E0,0E0,0E0,0E0,1.3602851E-3,0E0,1.8994343E-3,1.6222853E-3,1.4115553E-3,0E0,1.2365151E-3,0E0,0E0,0E0,0E0,0E0,2.3966925E-3,0E0,1.3055764E-3,0E0,1.4824593E-3,0E0,0E0,1.6766818E-3,0E0,0E0,0E0,0E0,0E0,1.7865545E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1289162E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,10,10,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,34,34,37,37,39,39,40,40,45,45,48,48,49,49,50,50,51,51,54,54,55,55,56,56,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,75,75,77,77,78,78,79,79,81,81,87,87,89,89,91,91,94,94,100,100,108,108],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,16,-1,-1,18,-1,20,-1,22,24,26,28,30,32,34,-1,36,38,-1,40,42,44,-1,-1,-1,46,-1,-1,48,-1,50,52,-1,-1,-1,-1,54,-1,-1,56,58,60,62,-1,-1,64,66,68,-1,-1,-1,70,72,74,76,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,-1,92,94,96,-1,98,-1,-1,-1,-1,-1,100,-1,102,-1,104,-1,-1,106,-1,-1,-1,-1,-1,108,-1,-1,-1,-1,-1,-1,-1,110,-1,-1],"split_conditions":[1.1741724E4,5.013E3,4.06635E5,9.05E3,1.0758E4,1.0863859E-2,1.4817759E9,-3.707359E-3,2.16292E5,6.3485056E-3,8.9003E4,3.8653172E-3,5.8672164E-4,2.5635E4,-3.3003362E-3,9.9E1,3.842162E-3,6.8E1,3.46841E6,1.0362922E3,6.281E3,1.063E3,9.554299E2,3.7E1,5.4987316E-4,4.833E3,8.2E1,-4.178971E-3,4.23859E5,1.3E1,5.9625816E7,3.1822042E-3,7.0804544E-4,3.3338368E-4,4.619E3,6.3982955E-3,7.9618604E-4,2.0547E4,2.3057018E-3,2.2E1,2.163512E6,-2.0736724E-3,-2.0503024E-4,2.5634074E-5,1.1935951E-3,2.578E4,-1.6491323E-3,-1.90592E-3,3.4330098E3,7.281E3,2.7E1,3.23E2,-1.3110496E-3,-1.4373785E-3,1.49015E5,1E0,5.482996E-1,-2.956879E-4,-1.3419879E-3,8.588607E-4,5.941E3,8.501854E10,1.875E3,1.975E3,8.5E1,7.502168E8,-1.7938499E-3,4.2E1,6.3029236E-1,3.15E2,1.43E2,1.6072246E-3,5.8415055E-4,-2.1567832E-4,9.124088E-4,2.94E2,-1.6338825E-3,7.92929E5,2.97229E5,8.1864E4,1.2994986E-3,4.54366E5,2.6256305E-3,-2.9777904E-4,7.4143754E-4,-1.9220402E-4,6.207146E-4,7.529E3,-6.9871946E-4,1E0,-6.148676E-5,4.6E1,-9.462108E-5,3.0956988E-4,2.77E2,2.3650263E-4,-3.684436E-4,1.2320472E-3,2.793815E-4,3.27775E-4,9.21E3,-4.28364E-4,-9.95814E-4,-1.3317862E-3,-2.9486278E-4,-8.909636E-4,-1.3598974E-4,-9.288788E-4,1.2178E4,3.0623918E-4,-4.60743E-4],"split_indices":[5,3,12,18,8,0,16,0,22,0,7,0,0,8,0,4,0,17,2,5,7,0,5,20,0,7,4,0,12,17,23,0,0,0,3,0,0,8,0,19,12,0,0,0,0,12,0,0,5,3,19,17,0,0,18,36,63,0,0,0,3,13,0,19,4,16,0,4,63,4,4,0,0,0,0,19,0,2,9,8,0,28,0,0,0,0,0,3,0,19,0,19,0,0,17,0,0,0,0,0,3,0,0,0,0,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.88E2,2.75E2,1.3E1,1.2E2,1.55E2,3E0,1E1,5E0,1.15E2,4E0,1.51E2,5E0,5E0,1.11E2,4E0,1.46E2,5E0,2E1,9.1E1,6.2E1,8.4E1,1.4E1,6E0,8.5E1,6E0,8E0,5.4E1,3E0,8.1E1,8E0,6E0,3E0,3E0,3E0,8.2E1,3E0,5E0,4.7E1,7E0,6E1,2.1E1,5E0,3E0,3E0,3E0,7.4E1,8E0,4E0,4.3E1,1.5E1,4.5E1,1.8E1,3E0,8E0,6.6E1,2.4E1,1.9E1,9E0,6E0,4E0,4.1E1,9E0,9E0,3.2E1,3.4E1,2.1E1,3E0,1E1,9E0,1.2E1,2.9E1,6E0,3E0,3E0,6E0,2.7E1,5E0,1.1E1,2.3E1,1.6E1,5E0,7E0,3E0,3E0,6E0,7E0,5E0,1.5E1,1.4E1,2.4E1,3E0,7E0,4E0,5E0,1.8E1,1.1E1,5E0,4E0,3E0,5E0,1E1,9E0,1.5E1,4E0,3E0,3E0,1.5E1,4E0,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[1.0923175E-2,4.0147617E-3,1.604738E-1,-3.264258E-3,7.042241E-2,9.25831E-3,4.505205E-2,-1.4715373E-2,2.9924033E-2,6.2654475E-3,4.7729764E-2,-2.9197524E-4,2.5538662E-3,-1.1396156E-2,-7.3653586E-2,4.282405E-3,2.290203E-2,1.659185E-2,8.585148E-2,2.504402E-2,-1.505933E-2,-4.231143E-3,-9.6910755E-4,2.7081778E-3,1.6172852E-2,-6.0454523E-4,3.1162132E-2,4.918299E-3,4.1648492E-2,5.0250385E-2,-2.9450147E-3,-4.794131E-2,-9.435365E-3,2.4695512E-2,-1.522832E-2,5.534717E-2,-2.874264E-4,6.606171E-4,1.8554719E-3,5.90063E-4,2.0328863E-3,-9.1690884E-4,7.462743E-4,-6.7830435E-4,-5.3902943E-2,-5.872747E-3,-5.6230914E-2,1.77225E-2,3.0090725E-3,-1.549361E-3,4.7442648E-3,2.3370583E-3,7.806854E-4,-7.274037E-4,-5.8360264E-2,2.7188437E-2,-1.0268598E-2,-9.49158E-3,-3.9903694E-3,-7.482274E-4,2.2520302E-2,-4.6590331E-4,7.407866E-4,-2.5350593E-3,-1.3496857E-3,-9.70156E-3,5.0292592E-2,-2.1875214E-2,-9.3398185E-4,-6.3994183E-4,1.05274696E-4,-2.0635724E-3,3.1068657E-2,3.111254E-4,-8.576209E-4,1.9586853E-4,2.200741E-3,-2.523997E-2,2.796807E-4,4.3364796E-3,-1.8004607E-2,5.388413E-4,-3.60754E-4,1.6533942E-3,2.2275863E-2,-2.2083027E-2,-1.7525468E-3,-6.831334E-4,6.90854E-3,-3.251749E-2,1.8443012E-5,-1.3341042E-4,2.6665652E-2,-3.7002403E-2,-1.4151621E-2,2.1629423E-2,-4.3820692E-4,-3.5559962E-4,-1.8216849E-3,1.1224684E-3,2.451947E-4,-1.500683E-3,-4.7653366E-4,-4.46277E-3,-2.0901192E-2,3.0518662E-2,-1.2974207E-4,1.5030966E-2,-8.856019E-3,-5.11649E-4,1.9630202E-4,-1.2780754E-3,-4.6652503E-4,4.0199946E-5,1.291952E-3,-4.4141363E-4,8.412921E-4,-7.212994E-6,-6.856079E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":129,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,19,21,-1,23,25,27,29,31,-1,-1,-1,33,-1,35,-1,37,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,53,55,57,59,-1,-1,61,-1,-1,-1,63,65,67,69,-1,-1,71,-1,-1,-1,-1,73,75,77,79,-1,-1,81,83,-1,-1,-1,-1,85,-1,87,89,-1,-1,-1,91,93,-1,-1,95,97,-1,-1,99,101,103,105,107,-1,-1,-1,-1,-1,-1,109,111,113,-1,115,117,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.041372E-1,1.3702714E-1,1.7782834E-1,9.773133E-2,6.964043E-2,0E0,1.5565639E-2,3.652571E-2,4.4742234E-2,0E0,2.8251335E-2,0E0,0E0,2.4671724E-2,2.120971E-2,0E0,2.4597783E-2,8.316176E-3,2.618628E-2,1.214152E-2,3.0245546E-2,0E0,0E0,0E0,1.5898122E-2,0E0,1.1159967E-2,0E0,1.520127E-3,2.549369E-3,6.666503E-3,2.7084835E-3,2.3509968E-2,2.0406967E-2,8.771926E-3,1.9952692E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3323762E-3,1.9753678E-2,3.0482765E-2,8.975267E-3,0E0,0E0,3.4930615E-3,0E0,0E0,0E0,2.059348E-3,1.4124913E-2,1.2872422E-2,9.931372E-4,0E0,0E0,8.260028E-3,0E0,0E0,0E0,0E0,2.5543198E-3,7.57532E-3,5.669834E-3,6.1060716E-3,0E0,0E0,2.0789045E-3,4.0312465E-3,0E0,0E0,0E0,0E0,3.2232963E-3,0E0,3.565071E-3,4.3324563E-3,0E0,0E0,0E0,2.662085E-3,4.801281E-3,0E0,0E0,5.2007064E-3,4.02705E-3,0E0,0E0,2.7041147E-3,2.51255E-3,1.9339868E-3,3.7725847E-3,4.4302624E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.7872932E-3,1.5934436E-3,3.328018E-3,0E0,4.188883E-3,2.358272E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,13,13,14,14,16,16,17,17,18,18,19,19,20,20,24,24,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,44,44,45,45,46,46,47,47,50,50,54,54,55,55,56,56,57,57,60,60,65,65,66,66,67,67,68,68,71,71,72,72,77,77,79,79,80,80,84,84,85,85,88,88,89,89,92,92,93,93,94,94,95,95,96,96,103,103,104,104,105,105,107,107,108,108],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,20,22,-1,24,26,28,30,32,-1,-1,-1,34,-1,36,-1,38,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,54,56,58,60,-1,-1,62,-1,-1,-1,64,66,68,70,-1,-1,72,-1,-1,-1,-1,74,76,78,80,-1,-1,82,84,-1,-1,-1,-1,86,-1,88,90,-1,-1,-1,92,94,-1,-1,96,98,-1,-1,100,102,104,106,108,-1,-1,-1,-1,-1,-1,110,112,114,-1,116,118,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1741724E4,1.8943199E9,2.2374196E0,8.1188E4,1.0916E4,9.25831E-3,6.1132E4,1.33097E5,1.8674554E-2,6.2654475E-3,3.4877544E7,-2.9197524E-4,2.5538662E-3,2.7795588E-6,5.5461E4,4.282405E-3,2.6208E4,2.61029E5,1.03880065E11,1E0,9.57049E5,-4.231143E-3,-9.6910755E-4,2.7081778E-3,6.45689E5,-6.0454523E-4,1.2585319E0,4.918299E-3,1.8956E4,3.185029E2,6.78E2,4.048635E-5,1.1828E4,1.3195825E11,2.54E2,3.0873373E8,-2.874264E-4,6.606171E-4,1.8554719E-3,5.90063E-4,2.0328863E-3,-9.1690884E-4,7.462743E-4,-6.7830435E-4,5.705E3,2.2825E4,1.49E3,6E1,3.0090725E-3,-1.549361E-3,5.6E1,2.3370583E-3,7.806854E-4,-7.274037E-4,7.25E2,9.406E3,4.73E3,4.0835712E8,-3.9903694E-3,-7.482274E-4,1.63192E5,-4.6590331E-4,7.407866E-4,-2.5350593E-3,-1.3496857E-3,2.31776E5,5.598581E7,2.431E3,8.89E2,-6.3994183E-4,1.05274696E-4,1.25E2,8.3748E4,3.111254E-4,-8.576209E-4,1.9586853E-4,2.200741E-3,4.591E3,2.796807E-4,1.8E1,7.7433E4,5.388413E-4,-3.60754E-4,1.6533942E-3,2.0986812E10,1E0,-1.7525468E-3,-6.831334E-4,8.5E1,2.1110523E9,1.8443012E-5,-1.3341042E-4,1.7728317E10,1.916746E6,6.6254E4,1.6366E4,5.71365E5,-3.5559962E-4,-1.8216849E-3,1.1224684E-3,2.451947E-4,-1.500683E-3,-4.7653366E-4,1.033905E6,1.9452692E7,6.607E3,-1.2974207E-4,1.1E2,5.5296E4,-5.11649E-4,1.9630202E-4,-1.2780754E-3,-4.6652503E-4,4.0199946E-5,1.291952E-3,-4.4141363E-4,8.412921E-4,-7.212994E-6,-6.856079E-4],"split_indices":[5,16,64,28,8,0,8,8,63,0,2,0,0,63,22,0,20,6,10,45,23,0,0,0,28,0,64,0,0,5,0,66,3,26,19,1,0,0,0,0,0,0,0,0,21,8,0,19,0,0,17,0,0,0,28,8,3,11,0,0,22,0,0,0,0,18,16,0,19,0,0,19,20,0,0,0,0,3,0,4,8,0,0,0,26,42,0,0,4,10,0,0,21,18,8,20,6,0,0,0,0,0,0,18,11,28,0,4,22,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.94E2,2.82E2,1.2E1,2.55E2,2.7E1,5E0,7E0,1.9E2,6.5E1,3E0,2.4E1,3E0,4E0,1.81E2,9E0,3E0,6.2E1,1.4E1,1E1,1.6E1,1.65E2,3E0,6E0,5E0,5.7E1,4E0,1E1,3E0,7E0,8E0,8E0,2.3E1,1.42E2,4.5E1,1.2E1,6E0,4E0,4E0,3E0,3E0,5E0,4E0,4E0,5E0,1.8E1,1.33E2,9E0,4.2E1,3E0,4E0,8E0,3E0,3E0,3E0,1.5E1,1.5E1,1.18E2,6E0,3E0,4E0,3.8E1,4E0,4E0,5E0,1E1,6E0,9E0,5.2E1,6.6E1,3E0,3E0,1E1,2.8E1,3E0,3E0,3E0,6E0,4.7E1,5E0,5.1E1,1.5E1,3E0,7E0,7E0,2.1E1,4.4E1,3E0,4E0,4.7E1,8E0,7E0,3E0,1.8E1,1.4E1,3E1,1.5E1,3.2E1,5E0,3E0,1.2E1,6E0,9E0,5E0,1.3E1,1.7E1,1.1E1,4E0,1.1E1,2.1E1,6E0,7E0,3E0,1.4E1,3E0,8E0,3E0,8E0,1.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[8.405344E-3,2.0103543E-3,1.3524798E-1,-1.3469212E-4,4.812763E-3,1.0458426E-2,5.3985693E-2,-1.2251063E-2,1.7716521E-2,2.4796887E-3,-1.8058355E-4,-1.6740875E-2,2.7821463E-2,4.928716E-3,1.1283884E-2,-4.4997733E-2,-1.3477217E-2,4.0476866E-2,-3.427119E-4,5.645528E-2,7.1056187E-3,-1.8028377E-3,-6.16346E-4,1.4096529E-3,-1.601679E-2,1.7784906E-4,5.040986E-2,3.5825244E-3,3.3750557E-4,-5.181253E-3,2.1641973E-2,-5.8791142E-2,-1.31045E-2,2.944246E-2,2.3980888E-3,-2.4406394E-2,-3.6663495E-4,3.8277596E-2,-6.3621933E-3,-2.2957972E-3,-9.1510464E-4,-1.5131355E-2,9.315827E-4,1.30352E-3,3.5499374E-4,1.7955524E-4,-1.1700002E-3,-7.19328E-4,1.7354664E-3,2.5096912E-2,2.2448509E-3,-1.8859088E-3,7.989017E-3,-1.6768001E-2,8.1149756E-4,-4.2173928E-3,1.1672449E-2,3.207551E-2,3.674045E-5,-6.5612555E-3,9.948954E-4,-1.4190889E-2,-5.413372E-2,4.5913985E-4,-7.1218824E-3,1.6603068E-2,-8.032823E-5,1.5187002E-3,2.956391E-4,2.247065E-4,-6.9204776E-4,-1.1862042E-2,-1.667526E-3,-2.2472017E-3,-8.0218754E-4,9.0067834E-4,-3.8612148E-4,4.0973113E-3,1.0716239E-3,-1.597537E-2,-2.73653E-4,3.6432387E-4,-2.552834E-4,5.002072E-4,-1.8914466E-4,-1.2991966E-3,-1.4060004E-2,-4.2853155E-4,7.7100387E-3,3.3076838E-4,-5.159723E-4,9.4953354E-4,4.46048E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":130,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,15,17,-1,19,21,23,25,-1,27,29,-1,-1,-1,31,-1,33,-1,-1,35,37,39,41,43,-1,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,55,57,-1,-1,59,61,-1,63,65,67,-1,69,-1,71,73,-1,75,77,-1,-1,-1,-1,-1,79,-1,-1,-1,81,-1,83,-1,85,87,-1,-1,-1,-1,-1,89,-1,91,-1,-1,-1,-1],"loss_changes":[2.3658602E-1,8.8297516E-2,1.9822967E-1,5.9913624E-2,0E0,0E0,1.5542489E-2,3.0152239E-2,9.5823124E-2,0E0,0E0,1.3134442E-2,8.707102E-3,0E0,1.998256E-2,3.1967815E-3,2.0336092E-2,4.2347554E-3,0E0,2.1522649E-2,1.7972764E-2,0E0,0E0,0E0,1.5423842E-2,0E0,2.8576776E-3,0E0,0E0,5.0819456E-3,2.16734E-2,1.016181E-3,1.0920009E-2,9.781593E-4,0E0,4.1414145E-3,2.1180094E-3,1.02758445E-2,1.3959306E-2,0E0,0E0,8.213699E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.5336726E-3,3.6412599E-3,0E0,0E0,5.20495E-3,1.0209501E-2,0E0,1.5509108E-3,1.1591429E-3,5.4448433E-3,0E0,2.173044E-3,0E0,8.845845E-3,1.46636E-3,0E0,9.4917056E-4,2.3680355E-3,0E0,0E0,0E0,0E0,0E0,4.9053123E-3,0E0,0E0,0E0,1.0097892E-3,0E0,1.0042682E-3,0E0,2.9823612E-3,3.0175028E-3,0E0,0E0,0E0,0E0,0E0,3.6155963E-3,0E0,2.4395E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,24,24,26,26,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,41,41,48,48,49,49,52,52,53,53,55,55,56,56,57,57,59,59,61,61,62,62,64,64,65,65,71,71,75,75,77,77,79,79,80,80,86,86,88,88],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,16,18,-1,20,22,24,26,-1,28,30,-1,-1,-1,32,-1,34,-1,-1,36,38,40,42,44,-1,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,56,58,-1,-1,60,62,-1,64,66,68,-1,70,-1,72,74,-1,76,78,-1,-1,-1,-1,-1,80,-1,-1,-1,82,-1,84,-1,86,88,-1,-1,-1,-1,-1,90,-1,92,-1,-1,-1,-1],"split_conditions":[2.264E5,7.14535E5,6.0793E4,2.917845E8,4.812763E-3,1.0458426E-2,1.5801E4,1.801545E6,3.068E3,2.4796887E-3,-1.8058355E-4,2.1E1,3.14E2,4.928716E-3,6.7995E4,1.8697E4,3.185029E2,2.1124447E-6,-3.427119E-4,7.386E3,4.29906E5,-1.8028377E-3,-6.16346E-4,1.4096529E-3,2.227E3,1.7784906E-4,9.5329E4,3.5825244E-3,3.3750557E-4,6.1346E4,1.7689897E11,2.1159E4,2.4744986E9,1.51731E5,2.3980888E-3,4.411652E8,4.77E2,2.93968E5,5.6E1,-2.2957972E-3,-9.1510464E-4,5.309E3,9.315827E-4,1.30352E-3,3.5499374E-4,1.7955524E-4,-1.1700002E-3,-7.19328E-4,5.581883E7,6.0741784E7,2.2448509E-3,-1.8859088E-3,6.6962534E-1,1.98168E5,8.1149756E-4,2.31776E5,1.3798331E10,1.858566E9,3.674045E-5,4.6936613E-1,9.948954E-4,2.215E3,1E0,4.5913985E-4,7.2925E4,5.4404634E8,-8.032823E-5,1.5187002E-3,2.956391E-4,2.247065E-4,-6.9204776E-4,4.591E3,-1.667526E-3,-2.2472017E-3,-8.0218754E-4,9.896E3,-3.8612148E-4,1.4291E4,1.0716239E-3,1.1E1,5.111E3,3.6432387E-4,-2.552834E-4,5.002072E-4,-1.8914466E-4,-1.2991966E-3,3.3816576E7,-4.2853155E-4,5.8016E4,3.3076838E-4,-5.159723E-4,9.4953354E-4,4.46048E-5],"split_indices":[3,8,8,16,0,0,0,6,7,0,0,4,17,0,12,28,5,63,0,7,12,0,0,0,3,0,8,0,0,8,13,18,26,12,0,16,0,28,19,0,0,19,0,0,0,0,0,0,18,18,0,0,63,12,0,18,10,1,0,63,0,19,19,0,8,11,0,0,0,0,0,3,0,0,0,3,0,7,0,17,22,0,0,0,0,0,16,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,2.78E2,1.3E1,2.75E2,3E0,3E0,1E1,1.64E2,1.11E2,7E0,3E0,1.48E2,1.6E1,4E0,1.07E2,1.4E1,1.34E2,1.2E1,4E0,8E0,9.9E1,9E0,5E0,5E0,1.29E2,3E0,9E0,3E0,5E0,5.4E1,4.5E1,7E0,1.22E2,6E0,3E0,1E1,4.4E1,2.8E1,1.7E1,4E0,3E0,1.17E2,5E0,3E0,3E0,3E0,7E0,3E0,4.1E1,2.1E1,7E0,3E0,1.4E1,1.13E2,4E0,2.6E1,1.5E1,1.6E1,5E0,9E0,5E0,1.07E2,6E0,3E0,2.3E1,1.1E1,4E0,9E0,7E0,5E0,4E0,1.02E2,5E0,3E0,3E0,9E0,1.4E1,7E0,4E0,7.5E1,2.7E1,4E0,5E0,3E0,4E0,4E0,7.1E1,1E1,1.7E1,5E0,6.6E1,3E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-6.093359E-4,-6.0570957E-3,7.3783055E-2,-2.3648483E-3,-8.168576E-3,1.16085015E-1,3.5907995E-2,6.870302E-2,-5.936621E-3,1.842103E-3,5.644321E-3,-1.4777786E-4,2.1081085E-3,-7.052434E-3,6.158463E-3,-1.6020253E-2,1.2744552E-2,-1.1483632E-3,1.2117712E-3,-1.3343995E-2,-3.9402773E-3,4.5886873E-3,6.699267E-3,-5.5676863E-2,-6.4418977E-3,1.1051607E-2,-1.758311E-3,-2.354956E-2,-6.5148575E-3,-1.1447568E-2,4.049947E-2,3.738603E-2,2.098044E-3,-3.960052E-2,1.6272939E-3,-1.2952435E-2,7.020971E-4,1.4925806E-2,2.689288E-3,-7.817335E-4,4.901387E-2,-2.0129804E-3,8.748189E-3,-7.22212E-2,-1.52418595E-2,-1.5625838E-2,7.6942397E-3,3.175198E-5,9.2279E-4,5.155306E-4,6.417828E-2,-2.3196877E-3,2.5691465E-2,-1.0348257E-3,-3.0334436E-3,6.429633E-4,-3.0944884E-2,-6.2537645E-3,-2.2667198E-2,1.660586E-2,-6.748152E-4,2.8035622E-3,8.6084165E-4,2.8145215E-2,-1.0643809E-2,3.9067432E-2,4.3959618E-3,-1.408721E-3,-4.66757E-4,-7.876605E-4,-3.1429117E-3,-1.3446033E-3,-1.7822446E-2,8.0781564E-4,1.2659915E-4,1.044142E-4,1.6814126E-3,-2.8838377E-2,3.8957908E-3,1.547298E-3,1.4632051E-4,-4.6382318E-4,6.2235503E-4,-9.7612655E-3,8.63564E-3,-1.4010825E-2,-1.2979936E-3,-1.5716433E-3,-1.4530135E-2,-3.6032673E-3,9.3890517E-4,-1.8241642E-2,1.4269862E-3,5.955426E-4,-5.3979096E-4,-1.0400743E-3,-2.089389E-2,-7.0571323E-4,6.2224167E-6,2.4085697E-4,-5.07236E-4,-7.758326E-4,-1.6975875E-5,4.3837004E-4,-1.7719096E-4,6.222611E-4,-2.1844184E-4,-1.2110961E-3,-4.5599169E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":131,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,-1,19,21,-1,-1,23,-1,-1,25,27,29,31,-1,33,-1,35,37,39,41,43,-1,45,-1,47,-1,-1,49,-1,51,53,55,57,59,-1,-1,-1,61,63,65,-1,-1,-1,67,69,71,73,-1,-1,-1,75,77,79,81,-1,-1,-1,83,-1,85,-1,-1,-1,-1,87,89,-1,-1,-1,-1,91,93,95,-1,-1,97,99,-1,101,103,-1,-1,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.187863E-1,2.490899E-1,2.7875409E-2,6.8944186E-2,0E0,2.1358468E-2,1.4677954E-2,1.2210724E-1,4.8804685E-2,0E0,0E0,0E0,0E0,1.2857646E-2,0E0,4.620096E-2,7.045628E-2,0E0,0E0,4.7851965E-2,0E0,0E0,2.409112E-2,1.0364722E-1,3.4064766E-2,1.9491881E-2,0E0,2.579078E-2,0E0,6.78912E-3,1.4165325E-2,1.6249642E-2,2.7626252E-2,1.2407346E-2,0E0,7.0198383E-3,0E0,1.8872845E-3,0E0,0E0,7.979281E-3,0E0,1.0936109E-2,3.1464659E-3,6.701177E-3,7.183373E-3,4.179023E-3,0E0,0E0,0E0,7.966258E-3,9.428941E-3,6.3797655E-3,0E0,0E0,0E0,1.0518278E-3,2.6998105E-3,5.0867274E-3,1.1924366E-3,0E0,0E0,0E0,4.629656E-3,7.784059E-3,4.1320957E-3,3.0878275E-3,0E0,0E0,0E0,3.4363435E-3,0E0,3.894968E-3,0E0,0E0,0E0,0E0,3.0083135E-3,3.3269736E-3,0E0,0E0,0E0,0E0,2.682687E-3,4.320529E-3,4.0850807E-3,0E0,0E0,9.840163E-4,2.0043624E-3,0E0,1.648406E-3,1.1735825E-3,0E0,0E0,2.1211808E-3,2.8438298E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,15,15,16,16,19,19,22,22,23,23,24,24,25,25,27,27,29,29,30,30,31,31,32,32,33,33,35,35,37,37,40,40,42,42,43,43,44,44,45,45,46,46,50,50,51,51,52,52,56,56,57,57,58,58,59,59,63,63,64,64,65,65,66,66,70,70,72,72,77,77,78,78,83,83,84,84,85,85,88,88,89,89,91,91,92,92,95,95,96,96],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,-1,20,22,-1,-1,24,-1,-1,26,28,30,32,-1,34,-1,36,38,40,42,44,-1,46,-1,48,-1,-1,50,-1,52,54,56,58,60,-1,-1,-1,62,64,66,-1,-1,-1,68,70,72,74,-1,-1,-1,76,78,80,82,-1,-1,-1,84,-1,86,-1,-1,-1,-1,88,90,-1,-1,-1,-1,92,94,96,-1,-1,98,100,-1,102,104,-1,-1,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2436111E11,1.0789621E11,2.2799654E0,3.308E3,-8.168576E-3,9.69111E5,8.375186E11,6.24933E5,4.579412E8,1.842103E-3,5.644321E-3,-1.4777786E-4,2.1081085E-3,3.15859E5,6.158463E-3,1.117371E6,5.275E3,-1.1483632E-3,1.2117712E-3,2.7708E4,-3.9402773E-3,4.5886873E-3,8.70498E5,1.13472E5,5.710497E6,2.54E2,-1.758311E-3,2.6238548E7,-6.5148575E-3,9.2921E4,9.5329E4,1E0,3.07E2,2.291E3,1.6272939E-3,7.503E3,7.020971E-4,3.2933E4,2.689288E-3,-7.817335E-4,1.506E3,-2.0129804E-3,4.2524388E7,3.6516666E2,2.0497083E3,8.858666E-3,4.6936613E-1,3.175198E-5,9.2279E-4,5.155306E-4,2.05E2,4.78E2,1.16E2,-1.0348257E-3,-3.0334436E-3,6.429633E-4,1.6507992E9,3.15E2,1.961056E-2,3.2353126E9,-6.748152E-4,2.8035622E-3,8.6084165E-4,1.5789E4,8.4E1,4.7860208E-1,1.32E2,-1.408721E-3,-4.66757E-4,-7.876605E-4,1.1994204E3,-1.3446033E-3,1.606E3,8.0781564E-4,1.2659915E-4,1.044142E-4,1.6814126E-3,1.6941404E7,4.1586E4,1.547298E-3,1.4632051E-4,-4.6382318E-4,6.2235503E-4,2.61E2,5.395E3,2.3E2,-1.2979936E-3,-1.5716433E-3,1.48E2,5.8896E4,9.3890517E-4,6.494384E6,5.5E2,5.955426E-4,-5.3979096E-4,1.2837E4,4.35E2,-7.0571323E-4,6.2224167E-6,2.4085697E-4,-5.07236E-4,-7.758326E-4,-1.6975875E-5,4.3837004E-4,-1.7719096E-4,6.222611E-4,-2.1844184E-4,-1.2110961E-3,-4.5599169E-4],"split_indices":[1,1,67,7,0,28,13,18,16,0,0,0,0,21,0,12,3,0,0,8,0,0,28,22,2,17,0,21,0,28,8,19,17,3,0,3,0,18,0,0,0,0,18,5,5,66,63,0,0,0,19,17,4,0,0,0,11,0,64,10,0,0,0,7,19,63,4,0,0,0,5,0,19,0,0,0,0,18,7,0,0,0,0,4,3,17,0,0,4,8,0,18,17,0,0,7,17,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.91E2,2.72E2,1.9E1,2.69E2,3E0,8E0,1.1E1,1.2E1,2.57E2,5E0,3E0,5E0,6E0,8E0,4E0,1.67E2,9E1,5E0,3E0,1.64E2,3E0,3E0,8.7E1,2.2E1,1.42E2,8.2E1,5E0,1.9E1,3E0,1.29E2,1.3E1,2E1,6.2E1,1.6E1,3E0,1.24E2,5E0,9E0,4E0,3E0,1.7E1,5E0,5.7E1,6E0,1E1,1.1E2,1.4E1,5E0,4E0,6E0,1.1E1,3.5E1,2.2E1,3E0,3E0,3E0,7E0,4.8E1,6.2E1,1.1E1,3E0,6E0,5E0,7E0,2.8E1,1.3E1,9E0,3E0,4E0,6E0,4.2E1,1.1E1,5.1E1,6E0,5E0,4E0,3E0,1.2E1,1.6E1,1E1,3E0,4E0,5E0,2.7E1,1.5E1,4.5E1,6E0,4E0,8E0,1.3E1,3E0,1.5E1,1.2E1,1.1E1,4E0,1.6E1,2.9E1,5E0,3E0,7E0,6E0,1.1E1,4E0,4E0,8E0,3E0,1.3E1,7E0,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[3.8988774E-3,6.740361E-3,-6.468445E-3,3.9642802E-4,2.0265335E-1,-1.9661854E-3,4.1929623E-3,2.9910053E-3,1.0190694E-2,-1.3379698E-2,1.607914E-2,2.5443775E-3,-1.6099358E-2,7.038485E-2,4.305121E-3,-1.3604834E-2,-3.0001134E-3,1.03831396E-1,-6.2407606E-4,5.5062707E-4,4.2250544E-2,-1.12773115E-2,-3.2878048E-3,4.6655715E-2,7.567839E-3,4.279283E-3,-3.016839E-2,1.806474E-3,4.6190954E-4,-3.840204E-2,-7.0679155E-3,1.750499E-2,2.5151032E-3,-2.6631134E-2,7.398307E-3,-1.5453739E-3,-2.0258638E-4,-5.4609712E-2,-2.1695474E-2,-1.0559305E-2,1.6794637E-2,1.6531985E-3,-6.6713046E-4,-1.3398693E-3,-1.6028057E-4,2.258416E-2,1.3262399E-4,-2.422773E-3,-9.1073895E-4,-1.0898577E-3,-1.2520891E-4,1.1471076E-3,-1.2035784E-2,1.8474166E-3,4.848801E-3,2.7752537E-2,-2.4501162E-4,-1.4055118E-2,9.423467E-3,-4.60369E-2,-9.803252E-3,7.028227E-4,-3.3126704E-3,2.3233346E-4,1.1127985E-3,-5.7512574E-4,-1.3894658E-3,-4.1693874E-4,1.4624902E-2,-3.4925345E-4,-2.2440364E-3,-1.1482826E-2,9.15332E-4,-5.369949E-4,1.2187925E-4,-5.4493494E-4,1.3806624E-2,6.1512473E-3,2.2585953E-2,-9.5012495E-3,-1.6224225E-3,7.9878606E-5,7.889935E-4,7.9020066E-4,-1.529971E-4,1.195455E-3,3.4336816E-4,-2.535562E-4,-1.5494248E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":132,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,13,-1,15,17,19,21,-1,23,-1,25,27,29,-1,31,-1,33,35,-1,-1,37,39,41,-1,43,45,-1,-1,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,59,-1,61,63,-1,65,67,69,71,-1,73,-1,-1,75,-1,-1,77,-1,-1,79,-1,-1,-1,-1,81,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7015804E-1,3.5662016E-1,0E0,8.585875E-2,8.310577E-2,5.685428E-2,0E0,0E0,0E0,4.375307E-2,6.81124E-2,0E0,3.0467652E-2,5.9062243E-2,1.2656198E-2,3.2025293E-2,0E0,9.411149E-2,0E0,9.521431E-3,2.2906037E-3,1.7768817E-2,0E0,8.726509E-3,0E0,7.3313895E-3,3.5313973E-3,0E0,0E0,4.5014806E-3,1.1674596E-2,1.0124263E-2,0E0,2.0652576E-3,7.4998266E-3,0E0,0E0,3.3066887E-3,2.4670656E-3,8.682015E-3,8.45135E-3,0E0,0E0,0E0,0E0,3.644636E-3,6.327804E-3,0E0,0E0,0E0,0E0,0E0,8.505901E-3,0E0,2.1517274E-3,2.410695E-3,0E0,7.4310265E-3,3.5647138E-3,5.075558E-3,7.519843E-3,0E0,1.1072151E-3,0E0,0E0,3.5265489E-3,0E0,0E0,1.4800886E-3,0E0,0E0,7.850444E-3,0E0,0E0,0E0,0E0,9.0902415E-4,2.7246557E-3,1.6852226E-3,6.146501E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9,10,10,12,12,13,13,14,14,15,15,17,17,19,19,20,20,21,21,23,23,25,25,26,26,29,29,30,30,31,31,33,33,34,34,37,37,38,38,39,39,40,40,45,45,46,46,52,52,54,54,55,55,57,57,58,58,59,59,60,60,62,62,65,65,68,68,71,71,76,76,77,77,78,78,79,79],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,14,-1,16,18,20,22,-1,24,-1,26,28,30,-1,32,-1,34,36,-1,-1,38,40,42,-1,44,46,-1,-1,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,60,-1,62,64,-1,66,68,70,72,-1,74,-1,-1,76,-1,-1,78,-1,-1,80,-1,-1,-1,-1,82,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.084044E6,2.77873E5,-6.468445E-3,7.14535E5,1.5042838E0,4.8655386E8,4.1929623E-3,2.9910053E-3,1.0190694E-2,1.949E3,9.603E3,2.5443775E-3,5.9512E4,2.7804902E-1,8.951542E3,1.46852E5,-3.0001134E-3,2.05E2,-6.2407606E-4,1E0,4.062E3,2.91642E5,-3.2878048E-3,1.1828E4,7.567839E-3,4.77E2,2.486228E3,1.806474E-3,4.6190954E-4,2.578E4,4.580147E6,1.5268964E3,2.5151032E-3,1.1408653E9,1.8829668E-1,-1.5453739E-3,-2.0258638E-4,6.6362006E2,7.325E4,3.67002E5,4.890295E6,1.6531985E-3,-6.6713046E-4,-1.3398693E-3,-1.6028057E-4,1.395E3,1.8308563E0,-2.422773E-3,-9.1073895E-4,-1.0898577E-3,-1.2520891E-4,1.1471076E-3,2.468E3,1.8474166E-3,3.1177E4,1E0,-2.4501162E-4,3.4325E3,3.1E1,1.595E3,5.309E3,7.028227E-4,5.617E3,2.3233346E-4,1.1127985E-3,1.37E4,-1.3894658E-3,-4.1693874E-4,6.546557E9,-3.4925345E-4,-2.2440364E-3,2.215E3,9.15332E-4,-5.369949E-4,1.2187925E-4,-5.4493494E-4,1.225E3,1.3219E4,1.01635706E9,2.88414E5,-1.6224225E-3,7.9878606E-5,7.889935E-4,7.9020066E-4,-1.529971E-4,1.195455E-3,3.4336816E-4,-2.535562E-4,-1.5494248E-3],"split_indices":[3,3,0,8,64,16,0,0,0,7,7,0,7,63,5,28,0,19,0,36,0,9,0,3,0,0,5,0,0,12,2,5,0,16,63,0,0,5,18,2,2,0,0,0,0,17,67,0,0,0,0,0,3,0,18,42,0,5,17,0,19,0,3,0,0,7,0,0,10,0,0,19,0,0,0,0,0,7,1,12,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.89E2,2.86E2,3E0,2.78E2,8E0,2.74E2,4E0,5E0,3E0,1.68E2,1.06E2,4E0,1.64E2,1.8E1,8.8E1,1.6E2,4E0,1.3E1,5E0,8.1E1,7E0,1.57E2,3E0,1E1,3E0,7.3E1,8E0,4E0,3E0,2E1,1.37E2,6E0,4E0,6E0,6.7E1,4E0,4E0,9E0,1.1E1,1.2E2,1.7E1,3E0,3E0,3E0,3E0,2.1E1,4.6E1,4E0,5E0,6E0,5E0,3E0,1.17E2,3E0,1.4E1,1.8E1,3E0,1.8E1,2.8E1,6E0,1.11E2,4E0,1E1,5E0,1.3E1,1.3E1,5E0,5E0,2.3E1,3E0,3E0,1.07E2,4E0,3E0,7E0,6E0,7E0,1.2E1,1.1E1,1.03E2,4E0,4E0,3E0,4E0,8E0,4E0,7E0,1E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.2359628E-2,7.075317E-3,1.511579E-1,-3.0319155E-3,4.7561456E-2,8.297901E-3,6.863291E-2,-1.2035539E-2,2.816878E-2,-2.2190375E-2,6.429887E-2,1.0036008E-3,2.8625377E-3,-1.958992E-2,-1.9007943E-3,3.233477E-2,-4.5469095E-4,-1.6763997E-3,2.3293812E-3,9.7000524E-2,3.4368366E-2,8.4045716E-4,-2.1885635E-2,2.5400752E-3,-4.3500308E-2,2.5353363E-2,6.0581937E-2,8.385954E-4,-7.073798E-4,4.5463316E-2,1.350343E-1,6.979103E-2,-9.7319484E-4,-4.1676544E-2,-1.8351872E-2,-2.733781E-2,7.737247E-3,-3.7411132E-4,-2.0762905E-3,1.4582331E-2,4.9486496E-2,2.4743963E-3,8.071436E-4,3.6928745E-4,2.1312886E-3,5.5808444E-3,2.387947E-3,5.37495E-4,1.0532909E-1,-1.3051821E-3,2.1120667E-2,-8.157898E-4,-1.818917E-3,-1.372382E-2,-3.224446E-2,-1.7476884E-4,-1.1572401E-3,1.8102907E-2,7.0446293E-4,2.7128657E-2,5.6063365E-3,6.1751053E-2,4.051225E-4,1.388384E-3,4.544875E-3,-2.0451055E-4,1.1561188E-3,-1.552004E-2,5.0705054E-4,-1.5550768E-3,-9.025011E-3,1.0019914E-2,1.2661441E-3,1.1924162E-2,-4.9347393E-3,1.6781615E-4,1.2523707E-3,-5.470778E-4,6.5935694E-4,9.5232873E-4,2.4184603E-3,-1.9464562E-2,-1.7326443E-3,5.1064588E-5,-5.7397486E-4,1.5002223E-2,1.5256902E-6,-2.949075E-4,2.3612343E-2,4.2385733E-4,-9.213634E-3,-1.4764239E-2,4.4254144E-4,-1.5968218E-2,-1.2724111E-3,4.6529432E-4,-3.9695267E-4,6.072392E-5,6.8640645E-4,1.1492595E-3,8.353898E-5,-7.3237426E-4,-1.0248667E-4,-8.016241E-4,-3.005553E-5,-2.5907123E-2,-9.620072E-3,-1.0853224E-3,-3.9156422E-4,4.080057E-4,-1.307927E-2,-1.5782466E-2,1.19309334E-4,-2.5108756E-4,-8.620356E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":133,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,23,25,-1,-1,27,29,31,-1,33,35,37,39,41,-1,-1,43,45,47,49,51,53,55,57,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,63,-1,65,-1,-1,67,69,-1,-1,71,73,75,77,79,-1,-1,-1,-1,-1,81,-1,-1,83,85,-1,87,89,-1,-1,91,-1,-1,-1,93,95,-1,-1,97,-1,-1,99,-1,101,103,-1,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,107,109,-1,-1,-1,111,113,-1,-1,-1],"loss_changes":[1.998967E-1,1.0858294E-1,7.030311E-2,6.0174845E-2,6.330247E-2,0E0,2.548635E-3,1.2672009E-2,8.914266E-3,8.183053E-3,3.875366E-2,0E0,0E0,1.0538295E-2,1.3411631E-2,7.141065E-3,0E0,0E0,4.6086526E-3,3.3063978E-2,3.0113328E-2,0E0,5.599372E-3,1.0460033E-2,3.9370237E-3,8.896105E-3,2.6994143E-3,0E0,0E0,6.2929634E-3,1.2802616E-2,2.0883024E-2,1.2222934E-2,1.7168783E-3,4.6368334E-3,1.6919114E-3,4.1425154E-3,0E0,0E0,2.8885324E-3,4.2010993E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.524599E-3,0E0,3.8867453E-3,0E0,0E0,3.4741582E-3,6.652007E-3,0E0,0E0,3.5530534E-3,2.2732979E-3,2.6364923E-3,1.5675239E-3,1.2206938E-3,0E0,0E0,0E0,0E0,0E0,3.1205872E-3,0E0,0E0,8.693237E-4,8.9363975E-4,0E0,3.1738034E-3,1.9803343E-3,0E0,0E0,2.8450864E-3,0E0,0E0,0E0,2.5256984E-3,2.5803624E-3,0E0,0E0,9.54794E-4,0E0,0E0,1.9403119E-3,0E0,1.5602527E-3,9.583845E-4,0E0,2.274517E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.199808E-3,2.1710128E-3,0E0,0E0,0E0,1.1021309E-3,1.5468318E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,48,48,50,50,53,53,54,54,57,57,58,58,59,59,60,60,61,61,67,67,70,70,71,71,73,73,74,74,77,77,81,81,82,82,85,85,88,88,90,90,91,91,93,93,105,105,106,106,110,110,111,111],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,24,26,-1,-1,28,30,32,-1,34,36,38,40,42,-1,-1,44,46,48,50,52,54,56,58,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,64,-1,66,-1,-1,68,70,-1,-1,72,74,76,78,80,-1,-1,-1,-1,-1,82,-1,-1,84,86,-1,88,90,-1,-1,92,-1,-1,-1,94,96,-1,-1,98,-1,-1,100,-1,102,104,-1,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,108,110,-1,-1,-1,112,114,-1,-1,-1],"split_conditions":[1.2738551E4,1.83E2,3E1,8.1188E4,9.4661897E-4,8.297901E-3,2.4392496E4,1.10686E5,2.5054106E11,2.3813066E9,2.0072E4,1.0036008E-3,2.8625377E-3,1.5E1,6.5232E4,1.0751E4,-4.5469095E-4,-1.6763997E-3,3.068E3,2.31776E5,1.1768184E9,8.4045716E-4,3.36E2,4.89E2,1.352E3,4.799528E3,1.7648065E10,8.385954E-4,-7.073798E-4,1E0,1.4933E4,9.1416E4,8.649117E8,3.1909866E-3,9.72567E5,1.1E2,4.2E1,-3.7411132E-4,-2.0762905E-3,7.4275E4,1.52E3,2.4743963E-3,8.071436E-4,3.6928745E-4,2.1312886E-3,5.5808444E-3,2.387947E-3,5.37495E-4,1.0655E4,-1.3051821E-3,3E1,-8.157898E-4,-1.818917E-3,6.29E2,2.81E2,-1.7476884E-4,-1.1572401E-3,5.7315456E7,7.6E1,1.3334E4,1.763175E6,4.171707E9,4.051225E-4,1.388384E-3,4.544875E-3,-2.0451055E-4,1.1561188E-3,7.19175E5,5.0705054E-4,-1.5550768E-3,1.5597291E3,1.2691E4,1.2661441E-3,1.969156E6,7.43E2,1.6781615E-4,1.2523707E-3,7.1E1,6.5935694E-4,9.5232873E-4,2.4184603E-3,1.2559658E7,6.8913E4,5.1064588E-5,-5.7397486E-4,3E1,1.5256902E-6,-2.949075E-4,8.922E3,4.2385733E-4,1.4908E4,5.9966224E-1,4.4254144E-4,1E0,-1.2724111E-3,4.6529432E-4,-3.9695267E-4,6.072392E-5,6.8640645E-4,1.1492595E-3,8.353898E-5,-7.3237426E-4,-1.0248667E-4,-8.016241E-4,-3.005553E-5,1.2625E4,2.371E3,-1.0853224E-3,-3.9156422E-4,4.080057E-4,2.3241706E0,1.1002E4,1.19309334E-4,-2.5108756E-4,-8.620356E-4],"split_indices":[5,47,19,28,64,0,5,12,1,10,8,0,0,28,28,0,0,0,7,18,11,0,20,0,0,5,21,0,0,19,8,12,16,67,2,4,19,0,0,20,19,0,0,0,0,0,0,0,3,0,19,0,0,4,4,0,0,18,4,7,12,11,0,0,0,0,0,2,0,0,5,20,0,2,0,0,0,4,0,0,0,18,12,0,0,17,0,0,3,0,7,63,0,42,0,0,0,0,0,0,0,0,0,0,0,7,3,0,0,0,64,22,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.64E2,9E0,2.12E2,5.2E1,3E0,6E0,1.65E2,4.7E1,1E1,4.2E1,3E0,3E0,9.4E1,7.1E1,4.3E1,4E0,4E0,6E0,1.9E1,2.3E1,4E0,9E1,6.5E1,6E0,3.6E1,7E0,3E0,3E0,9E0,1E1,1.1E1,1.2E1,1.2E1,7.8E1,9E0,5.6E1,3E0,3E0,2.6E1,1E1,4E0,3E0,4E0,5E0,5E0,5E0,5E0,6E0,4E0,8E0,7E0,5E0,6E1,1.8E1,3E0,6E0,2.2E1,3.4E1,1E1,1.6E1,7E0,3E0,3E0,3E0,3E0,5E0,5.7E1,3E0,1E1,8E0,1.7E1,5E0,1.1E1,2.3E1,4E0,6E0,1.2E1,4E0,3E0,4E0,4.4E1,1.3E1,4E0,4E0,1.1E1,6E0,4E0,7E0,4E0,1.9E1,6E0,6E0,3.9E1,5E0,5E0,8E0,4E0,7E0,4E0,3E0,5E0,1.4E1,3E0,3E0,1.4E1,2.5E1,8E0,6E0,3E0,2.2E1,1.9E1,3E0,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[4.9314643E-3,-6.4472493E-4,1.572259E-1,-1.0383764E-2,2.3960857E-2,4.2920608E-2,1.0234599E-2,4.2851772E-2,-1.3762488E-2,3.4766473E-2,-1.3989336E-2,-3.7568522E-4,2.7934322E-3,9.862648E-4,4.066631E-3,-2.7941754E-3,-1.20907165E-2,2.8987559E-2,3.5365727E-3,-1.497867E-3,2.1071802E-3,-1.0052345E-3,7.177766E-4,-8.796114E-3,-5.123299E-2,2.0520922E-2,2.5538374E-3,1.3963021E-3,-1.4611984E-2,1.7301976E-3,-1.0241583E-2,-4.2645936E-3,8.628388E-3,4.9776345E-2,1.3066752E-2,-1.4483737E-3,4.2307048E-4,-1.7061278E-2,7.225719E-4,-2.2982951E-4,5.698279E-4,-3.1580642E-4,2.9335867E-3,1.9853596E-2,-3.6003313E-3,-1.4493179E-2,-1.8078396E-3,5.579027E-3,-2.6545832E-2,3.475844E-2,6.41116E-3,-4.431791E-4,3.398085E-4,-3.3428118E-2,-1.1462395E-2,2.1678742E-3,1.2856015E-3,-1.5469721E-3,1.4740117E-4,7.055797E-4,1.975589E-3,-1.783852E-4,2.0550981E-2,-1.5105639E-3,2.7935655E-4,8.67301E-4,-1.3664466E-2,-7.764425E-4,2.1206545E-2,1.1792401E-3,1.1503151E-4,-2.6860692E-2,-7.4054087E-3,-4.5357784E-3,9.115392E-3,1.0646172E-3,1.2996195E-4,-1.4535977E-4,-3.196586E-2,3.8242298E-3,-1.3120865E-2,9.094635E-4,-6.3613435E-4,1.6921695E-2,-9.052323E-5,-2.177465E-2,-4.313473E-2,1.832909E-2,-7.373101E-3,-3.0691754E-2,-3.2716626E-4,5.061905E-4,-3.3670242E-3,1.285353E-4,9.2871446E-4,-1.0168595E-3,-1.6281113E-4,-1.675218E-3,-6.1121257E-4,9.6239697E-4,9.95928E-5,-4.8409338E-4,3.7721274E-4,-1.3527522E-3,-1.5444926E-4,3.3605786E-4,-3.0298866E-4,4.2064153E-4,-2.199204E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":134,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,-1,27,-1,-1,29,31,33,-1,-1,35,-1,37,-1,39,41,43,-1,-1,45,47,-1,-1,-1,-1,49,51,53,-1,55,57,59,61,-1,-1,63,65,67,-1,-1,-1,-1,-1,-1,69,-1,-1,-1,71,73,75,-1,-1,77,79,81,83,-1,-1,-1,85,87,89,91,-1,93,-1,95,97,99,101,103,105,-1,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4043138E-1,6.5908715E-2,1.7007363E-1,3.6023777E-2,3.2472227E-2,1.7774299E-2,0E0,4.1816823E-2,2.166311E-2,2.328989E-2,9.527331E-3,0E0,0E0,6.8757213E-3,0E0,0E0,2.3148712E-2,2.3097683E-2,0E0,0E0,9.604083E-3,0E0,0E0,1.5920259E-2,6.923817E-2,1.0471407E-2,0E0,0E0,9.024573E-3,0E0,1.2499286E-2,0E0,1.4141672E-3,2.5539523E-2,4.822389E-3,0E0,0E0,9.529062E-3,8.755905E-3,0E0,0E0,0E0,0E0,5.78788E-3,2.0166228E-3,5.2408297E-3,0E0,6.4770547E-3,6.9160927E-3,3.4386627E-3,2.9871091E-3,0E0,0E0,7.796949E-3,7.6415185E-3,2.9313713E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.0535819E-3,0E0,0E0,0E0,6.573952E-3,1.751935E-3,1.2927356E-3,0E0,0E0,2.8215777E-3,3.7085807E-3,2.8370214E-3,1.2580277E-3,0E0,0E0,0E0,1.4510639E-3,3.3834567E-3,8.532232E-3,1.7003345E-3,0E0,1.1187172E-3,0E0,1.9733254E-3,9.4777346E-4,1.4938235E-3,1.9325344E-3,4.493825E-3,2.3483178E-3,0E0,1.2852178E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,16,16,17,17,20,20,23,23,24,24,25,25,28,28,30,30,32,32,33,33,34,34,37,37,38,38,43,43,44,44,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,62,62,66,66,67,67,68,68,71,71,72,72,73,73,74,74,78,78,79,79,80,80,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,92,92],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,-1,28,-1,-1,30,32,34,-1,-1,36,-1,38,-1,40,42,44,-1,-1,46,48,-1,-1,-1,-1,50,52,54,-1,56,58,60,62,-1,-1,64,66,68,-1,-1,-1,-1,-1,-1,70,-1,-1,-1,72,74,76,-1,-1,78,80,82,84,-1,-1,-1,86,88,90,92,-1,94,-1,96,98,100,102,104,106,-1,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2738551E4,2.839763E3,1.693988E7,1E1,7.59577E5,5.981604E7,1.0234599E-2,2.2059E5,8.454E3,4.1586E4,5.752002E9,-3.7568522E-4,2.7934322E-3,1.275947E6,4.066631E-3,-2.7941754E-3,8.919402E9,3.259142E11,3.5365727E-3,-1.497867E-3,1.4595842E11,-1.0052345E-3,7.177766E-4,2.302E3,3.07E2,2.9609E4,2.5538374E-3,1.3963021E-3,6.2901763E3,1.7301976E-3,5.47107E5,-4.2645936E-3,6.726593E6,4.22E2,1.38E2,-1.4483737E-3,4.2307048E-4,4.97209E5,3.47989E5,-2.2982951E-4,5.698279E-4,-3.1580642E-4,2.9335867E-3,5.5666803E10,3.47989E5,6.3444E4,-1.8078396E-3,1.1148E5,6.039798E8,1E0,8.4551E4,-4.431791E-4,3.398085E-4,7.35E2,8.272E3,4.2524388E7,1.2856015E-3,-1.5469721E-3,1.4740117E-4,7.055797E-4,1.975589E-3,-1.783852E-4,1.9028718E11,-1.5105639E-3,2.7935655E-4,8.67301E-4,1.49015E5,4.6545E4,1.207E3,1.1792401E-3,1.1503151E-4,2E1,1.41048E5,9.675527E7,1.3321924E9,1.0646172E-3,1.2996195E-4,-1.4535977E-4,2.4848E4,6.23E2,7.11E2,9.8E2,-6.3613435E-4,3.068292E6,-9.052323E-5,1.6608131E-4,1.0112496E8,9.702562E6,1.2745082E3,9.76E2,9.33E2,5.061905E-4,2.876E3,1.285353E-4,9.2871446E-4,-1.0168595E-3,-1.6281113E-4,-1.675218E-3,-6.1121257E-4,9.6239697E-4,9.95928E-5,-4.8409338E-4,3.7721274E-4,-1.3527522E-3,-1.5444926E-4,3.3605786E-4,-3.0298866E-4,4.2064153E-4,-2.199204E-4],"split_indices":[5,5,18,4,28,6,0,12,18,7,27,0,0,2,0,0,25,13,0,0,23,0,0,7,17,8,0,0,5,0,6,0,2,28,4,0,0,6,12,0,0,0,0,13,12,6,0,8,16,36,20,0,0,17,12,18,0,0,0,0,0,0,13,0,0,0,18,28,0,0,0,17,6,21,21,0,0,0,27,17,0,0,0,2,0,64,16,18,5,17,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.73E2,9E0,1.96E2,7.7E1,6E0,3E0,1.1E1,1.85E2,6E1,1.7E1,3E0,3E0,8E0,3E0,3E0,1.82E2,5.7E1,3E0,5E0,1.2E1,3E0,5E0,1.69E2,1.3E1,5E1,7E0,3E0,9E0,3E0,1.66E2,5E0,8E0,9E0,4.1E1,4E0,5E0,1.02E2,6.4E1,3E0,5E0,4E0,5E0,2.9E1,1.2E1,9.7E1,5E0,5.5E1,9E0,1.3E1,1.6E1,7E0,5E0,1.2E1,8.5E1,5.1E1,4E0,5E0,4E0,1E1,3E0,9E0,7E0,9E0,3E0,4E0,8.1E1,4.5E1,6E0,3E0,4E0,2.5E1,5.6E1,3.3E1,1.2E1,3E0,3E0,5E0,2E1,1.9E1,3.7E1,2.5E1,8E0,7E0,5E0,1.2E1,8E0,8E0,1.1E1,1.5E1,2.2E1,5E0,2E1,4E0,3E0,7E0,5E0,5E0,3E0,4E0,4E0,8E0,3E0,1E1,5E0,1E1,1.2E1,3E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[2.5046465E-3,5.526946E-3,-6.919709E-3,6.610262E-4,1.5575635E-1,-1.1069279E-2,2.7114727E-2,1.0122433E-2,7.7216374E-4,-6.935313E-3,-6.6212313E-3,9.615503E-2,1.7796313E-2,-1.5511963E-2,3.0422814E-2,5.0597102E-3,1.5303517E-3,3.4999974E-2,3.3794667E-3,-2.5646353E-2,-7.3894067E-3,-2.2537215E-3,1.450724E-1,-9.875037E-3,4.6571407E-2,-4.490185E-2,1.5698107E-2,-4.4507366E-2,-2.1301895E-2,-9.069036E-4,-1.3563645E-2,3.5533182E-2,-7.7602156E-2,6.7674397E-3,8.8016014E-4,-7.355419E-4,2.8369308E-4,6.846747E-2,1.6039986E-2,-2.2962783E-3,1.9239356E-4,-1.3176911E-2,2.3387466E-2,-1.9147263E-4,-1.8250954E-3,3.8309608E-4,-2.3811378E-2,-4.319766E-3,8.5549813E-4,-1.6146228E-3,-8.769762E-3,5.668491E-3,4.9434137E-3,-4.3061874E-3,-5.7473156E-4,8.277612E-2,9.921828E-4,-1.3337017E-3,1.7853351E-3,2.4523225E-4,-8.748275E-4,3.16273E-2,-1.165539E-3,3.4199923E-4,-3.2041915E-4,-3.975899E-2,-1.9067684E-2,3.3087426E-4,-8.889738E-3,-1.4521205E-2,1.0511832E-2,-1.4372371E-3,2.458572E-2,3.0510575E-3,1.2251975E-3,5.0540303E-4,-1.0877362E-2,2.0978993E-2,1.7623962E-3,-6.1231334E-4,4.2982268E-4,-1.5769419E-3,-3.6556768E-4,-2.1291535E-2,2.3761211E-4,-1.8233526E-2,-2.0287386E-3,-1.9583892E-4,-3.0075423E-2,9.867563E-4,-8.048226E-5,3.879139E-2,-1.8996724E-4,-6.608125E-4,4.8083282E-5,3.044732E-2,1.4342798E-4,-1.6796004E-5,-2.3007853E-2,-1.1086249E-4,-8.705953E-4,-4.2173E-4,4.0896805E-3,-1.3698213E-3,-4.0502253E-4,5.367517E-4,1.5151472E-3,1.1941476E-3,3.6041046E-4,-1.0094636E-2,-8.477549E-4,3.323512E-4,-2.5596155E-4,1.6605343E-4,-5.98106E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":135,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,59,61,-1,-1,63,65,67,-1,-1,69,71,-1,-1,-1,73,-1,75,-1,-1,-1,77,79,-1,-1,81,83,-1,85,87,89,-1,91,-1,-1,-1,93,95,-1,-1,-1,-1,-1,97,-1,99,101,-1,103,-1,-1,105,-1,-1,-1,107,-1,-1,109,-1,-1,-1,111,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,-1,-1],"loss_changes":[1.9179876E-1,2.0969526E-1,0E0,8.687662E-2,1.8067555E-1,1.5466425E-1,5.3616554E-2,0E0,0E0,6.166976E-2,0E0,2.2196263E-2,1.8979454E-2,1.2651969E-2,1.3519561E-1,0E0,0E0,1.8634278E-2,2.6279563E-2,4.7058277E-3,3.5098558E-3,8.5325785E-2,5.562909E-2,2.1413555E-3,1.7620284E-2,1.2531502E-2,8.079236E-3,5.304085E-3,3.1653717E-3,4.2420113E-3,7.455008E-3,6.9635004E-2,3.1175148E-2,0E0,0E0,0E0,0E0,6.06665E-3,8.978246E-3,0E0,0E0,2.5363036E-3,5.70162E-3,0E0,0E0,8.468666E-4,3.175715E-3,2.760039E-3,0E0,0E0,4.6994206E-3,1.727982E-2,0E0,0E0,0E0,2.2991449E-3,0E0,1.7965334E-3,0E0,0E0,0E0,4.022289E-3,2.3280764E-3,0E0,0E0,2.327647E-3,2.6275981E-3,0E0,1.9503487E-3,4.016491E-3,2.6977193E-3,0E0,5.859143E-3,0E0,0E0,0E0,8.6649315E-4,2.3674825E-3,0E0,0E0,0E0,0E0,0E0,1.3782457E-3,0E0,1.6012318E-3,1.3408128E-3,0E0,1.8659839E-3,0E0,0E0,8.630138E-4,0E0,0E0,0E0,8.649295E-4,0E0,0E0,1.2816954E-3,0E0,0E0,0E0,1.0581795E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.260876E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,37,37,38,38,41,41,42,42,45,45,46,46,47,47,50,50,51,51,55,55,57,57,61,61,62,62,65,65,66,66,68,68,69,69,70,70,72,72,76,76,77,77,83,83,85,85,86,86,88,88,91,91,95,95,98,98,102,102,109,109],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,60,62,-1,-1,64,66,68,-1,-1,70,72,-1,-1,-1,74,-1,76,-1,-1,-1,78,80,-1,-1,82,84,-1,86,88,90,-1,92,-1,-1,-1,94,96,-1,-1,-1,-1,-1,98,-1,100,102,-1,104,-1,-1,106,-1,-1,-1,108,-1,-1,110,-1,-1,-1,112,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,-1,-1],"split_conditions":[1.2952493E8,2.77873E5,-6.919709E-3,4.580147E6,3.91E2,1.16215E5,1.619E3,1.0122433E-2,7.7216374E-4,1.83E2,-6.6212313E-3,6.7932027E9,1.99E2,2.34725E5,1.1127108E9,5.0597102E-3,1.5303517E-3,1E0,3.07E2,2.719E3,1.033E3,1.875641E6,2.702002E6,3.389E3,2.3819132E0,1.28E2,1.50749E5,1E0,7.656413E7,4.48546E7,1.09081E6,6.910113E8,1.3708E4,6.7674397E-3,8.8016014E-4,-7.355419E-4,2.8369308E-4,2.84E2,4.6523023E0,-2.2962783E-3,1.9239356E-4,1.4806E4,6.6861953E3,-1.9147263E-4,-1.8250954E-3,2.53E2,2.6804724E7,2.3E1,8.5549813E-4,-1.6146228E-3,1.610428E6,1.852E3,4.9434137E-3,-4.3061874E-3,-5.7473156E-4,2.1871E4,9.921828E-4,9.31278E5,1.7853351E-3,2.4523225E-4,-8.748275E-4,9.4008E4,1.1962493E9,3.4199923E-4,-3.2041915E-4,2.1811952E7,8.9091E4,3.3087426E-4,1.3326E4,5.2439684E7,4.7E1,-1.4372371E-3,3.7388639E9,3.0510575E-3,1.2251975E-3,5.0540303E-4,2.5059106E0,6.86E2,1.7623962E-3,-6.1231334E-4,4.2982268E-4,-1.5769419E-3,-3.6556768E-4,6.9355E4,2.3761211E-4,1.85948E5,1.11E2,-1.9583892E-4,7.3747E4,9.867563E-4,-8.048226E-5,9.11E2,-1.8996724E-4,-6.608125E-4,4.8083282E-5,4.38E2,1.4342798E-4,-1.6796004E-5,3.364E3,-1.1086249E-4,-8.705953E-4,-4.2173E-4,7.7021E4,-1.3698213E-3,-4.0502253E-4,5.367517E-4,1.5151472E-3,1.1941476E-3,3.6041046E-4,1.066E4,-8.477549E-4,3.323512E-4,-2.5596155E-4,1.6605343E-4,-5.98106E-4],"split_indices":[6,3,0,2,19,28,0,0,0,47,0,10,17,6,26,0,0,19,17,3,0,2,2,0,66,4,22,19,10,18,9,16,8,0,0,0,0,19,59,0,0,3,5,0,0,0,11,19,0,0,6,3,0,0,0,7,0,12,0,0,0,3,16,0,0,18,8,0,20,27,4,0,10,0,0,0,59,17,0,0,0,0,0,6,0,18,4,0,8,0,0,0,0,0,0,19,0,0,3,0,0,0,8,0,0,0,0,0,0,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.89E2,2.86E2,3E0,2.78E2,8E0,1.93E2,8.5E1,3E0,5E0,1.9E2,3E0,9E0,7.6E1,1.55E2,3.5E1,3E0,6E0,3.4E1,4.2E1,6.8E1,8.7E1,2.8E1,7E0,7E0,2.7E1,8E0,3.4E1,1.1E1,5.7E1,4.3E1,4.4E1,1.9E1,9E0,4E0,3E0,4E0,3E0,1.5E1,1.2E1,5E0,3E0,7E0,2.7E1,3E0,8E0,6E0,5.1E1,3.9E1,4E0,4E0,4E1,1.6E1,3E0,4E0,5E0,1E1,5E0,9E0,3E0,3E0,4E0,2E1,7E0,3E0,3E0,1E1,4.1E1,9E0,3E1,3.1E1,9E0,4E0,1.2E1,7E0,3E0,3E0,6E0,1.5E1,5E0,3E0,4E0,7E0,3E0,3.8E1,3E0,1.2E1,1.8E1,2.1E1,1E1,3E0,6E0,8E0,4E0,3E0,3E0,9E0,6E0,3E0,3.5E1,5E0,7E0,6E0,1.2E1,5E0,5E0,3E0,5E0,6E0,3E0,8E0,2.7E1,8E0,4E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[4.482833E-3,-3.1630756E-4,1.3937032E-1,-1.3598892E-2,2.2899047E-2,7.38917E-3,1.3193345E-3,-9.0907775E-3,-5.4581123E-3,4.8398594E-3,1.7571706E-2,-1.1215403E-2,2.1184997E-3,-2.892815E-3,3.0161137E-2,-4.5499407E-2,-7.650055E-3,-1.7370675E-2,3.1129926E-2,2.3632096E-2,3.8280464E-3,-2.2166757E-2,-3.5393585E-3,-1.0251263E-2,3.161283E-2,9.2230726E-4,-3.1224966E-2,2.0979585E-4,2.0692225E-3,1.5569838E-2,6.669244E-2,-4.2699557E-2,1.5327206E-3,-2.0936824E-2,-6.0997885E-3,1.2016674E-2,1.8670526E-3,-1.1927026E-2,-6.3505486E-2,-1.4205582E-3,2.0220606E-2,4.4313814E-3,2.658863E-4,-5.568132E-4,-1.8484818E-3,1.0372935E-3,-9.509543E-4,-2.762798E-2,-8.755142E-3,-8.294218E-3,1.21468315E-2,1.0934512E-3,-4.1654386E-4,-2.5188228E-2,3.915083E-4,-2.5353432E-3,-9.191717E-4,1.1736102E-2,3.2552946E-2,-2.1981819E-2,-1.5111269E-3,3.0014518E-4,-1.3644629E-2,-3.1250152E-3,-1.8224642E-2,2.1345453E-2,-3.729662E-4,-3.0685862E-4,-1.4899544E-3,2.1504413E-2,-7.0913825E-3,1.7232487E-2,5.146452E-2,7.710324E-5,-2.5956912E-2,8.254768E-5,-2.1133615E-2,-7.808829E-4,-1.9729348E-2,-1.1453899E-2,-9.890014E-4,2.5881222E-4,1.1577295E-3,9.70134E-4,4.7856197E-3,2.8593966E-5,-6.849146E-4,8.1847777E-4,-1.774874E-4,6.7285704E-4,2.1122575E-3,-1.0630807E-3,-1.6531076E-4,-9.743675E-4,-6.9054186E-5,-3.7505215E-3,8.820161E-3,4.9263228E-5,-1.0555182E-3,-6.920787E-4,-4.662344E-3,-2.3945278E-4,4.3803282E-4,3.981229E-4,-6.0745827E-3,-1.3940812E-4,5.345794E-4,-2.9179463E-4,3.125989E-4,-1.1154135E-2,7.216376E-5,-1.2590692E-4,-6.961489E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":136,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,15,-1,17,19,21,23,25,27,29,-1,31,-1,33,35,-1,37,-1,-1,39,41,43,45,47,49,51,-1,53,55,-1,57,-1,-1,-1,-1,-1,-1,59,61,63,65,-1,-1,67,-1,-1,-1,69,71,73,-1,-1,75,77,79,81,-1,-1,-1,83,85,87,89,-1,91,-1,93,95,97,99,-1,-1,-1,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,103,105,-1,-1,-1,107,-1,-1,-1,109,-1,-1,-1,-1,111,-1,-1,-1],"loss_changes":[1.884212E-1,8.726888E-2,7.93187E-2,1.2501867E-1,6.729657E-2,0E0,0E0,2.8747762E-2,0E0,0E0,2.5850916E-2,2.067624E-2,0E0,1.975123E-2,3.3120904E-2,2.1625604E-2,1.6358625E-2,1.874816E-2,9.372882E-3,1.9444268E-2,0E0,6.391462E-3,0E0,6.437691E-3,4.474435E-3,0E0,1.2947412E-2,0E0,0E0,1.5052967E-2,3.6188073E-2,1.6251178E-3,7.6285168E-3,3.100004E-3,4.425168E-3,4.27481E-3,0E0,5.0982637E-3,2.0172298E-3,0E0,4.640378E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.8038712E-3,1.5416298E-3,4.9222875E-3,2.8671226E-3,0E0,0E0,2.8619068E-3,0E0,0E0,0E0,5.6698783E-3,4.618082E-3,2.194007E-3,0E0,0E0,1.6322634E-3,2.5145728E-3,2.4410626E-3,1.4616367E-3,0E0,0E0,0E0,2.7428698E-3,1.26415E-3,2.3770302E-3,2.0893104E-3,0E0,2.6270496E-3,0E0,1.4964654E-3,1.6867548E-3,2.271741E-3,1.4690936E-3,0E0,0E0,0E0,0E0,9.640474E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7608837E-3,1.5057008E-3,0E0,0E0,0E0,1.0372833E-3,0E0,0E0,0E0,1.7147623E-3,0E0,0E0,0E0,0E0,1.8116764E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,47,47,48,48,49,49,50,50,53,53,57,57,58,58,59,59,62,62,63,63,64,64,65,65,69,69,70,70,71,71,72,72,74,74,76,76,77,77,78,78,79,79,84,84,95,95,96,96,100,100,104,104,109,109],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,16,-1,18,20,22,24,26,28,30,-1,32,-1,34,36,-1,38,-1,-1,40,42,44,46,48,50,52,-1,54,56,-1,58,-1,-1,-1,-1,-1,-1,60,62,64,66,-1,-1,68,-1,-1,-1,70,72,74,-1,-1,76,78,80,82,-1,-1,-1,84,86,88,90,-1,92,-1,94,96,98,100,-1,-1,-1,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,104,106,-1,-1,-1,108,-1,-1,-1,110,-1,-1,-1,-1,112,-1,-1,-1],"split_conditions":[1.2738551E4,2.85218E5,3.954342E-1,1.06056E5,1.3E1,7.38917E-3,1.3193345E-3,1.5102771E9,-5.4581123E-3,4.8398594E-3,7.9788E4,1.571E4,2.1184997E-3,1.36692E9,1.96342E5,4.105E3,1.83E2,7.848431E2,4.580147E6,1.15540435E11,3.8280464E-3,3.6054228E-3,-3.5393585E-3,4.97264E5,1.0446692E3,9.2230726E-4,5.59456E5,2.0979585E-4,2.0692225E-3,1.9391E4,1.153969E6,3.5590376E7,5.870495E7,8.964E3,1.576081E6,1.719E3,1.8670526E-3,2.207E3,2.6519E4,-1.4205582E-3,7.1376E4,4.4313814E-3,2.658863E-4,-5.568132E-4,-1.8484818E-3,1.0372935E-3,-9.509543E-4,1.095E3,3.59392E5,1.207E3,2.32877E5,1.0934512E-3,-4.1654386E-4,1.9694447E10,3.915083E-4,-2.5353432E-3,-9.191717E-4,2.9424374E10,1.1962493E9,4.3085E4,-1.5111269E-3,3.0014518E-4,1.1002E4,5.577429E-1,1.6868966E8,7.8E1,-3.729662E-4,-3.0685862E-4,-1.4899544E-3,5.3736E4,2.5817E4,5.482996E-1,5.1E1,7.710324E-5,9.310384E6,8.254768E-5,6.76E2,1.1173069E9,6.1E1,1.754E4,-9.890014E-4,2.5881222E-4,1.1577295E-3,9.70134E-4,1.6174E4,2.8593966E-5,-6.849146E-4,8.1847777E-4,-1.774874E-4,6.7285704E-4,2.1122575E-3,-1.0630807E-3,-1.6531076E-4,-9.743675E-4,-6.9054186E-5,2.1047948E7,5.159E3,4.9263228E-5,-1.0555182E-3,-6.920787E-4,1.64738E5,-2.3945278E-4,4.3803282E-4,3.981229E-4,2.3912651E8,-1.3940812E-4,5.345794E-4,-2.9179463E-4,3.125989E-4,1.4158E4,7.216376E-5,-1.2590692E-4,-6.961489E-4],"split_indices":[5,12,63,28,4,0,0,16,0,0,28,8,0,16,8,7,47,5,2,1,0,63,0,9,5,0,12,0,0,8,12,16,16,22,6,0,0,0,7,0,8,0,0,0,0,0,0,17,2,0,12,0,0,10,0,0,0,10,16,6,0,0,22,63,16,19,0,0,0,8,7,63,4,0,18,0,0,11,4,7,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,11,3,0,0,0,12,0,0,0,21,0,0,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.9E2,2.81E2,9E0,1.79E2,1.02E2,4E0,5E0,1.75E2,4E0,3E0,9.9E1,1.71E2,4E0,3.8E1,6.1E1,1.5E1,1.56E2,2.7E1,1.1E1,5.8E1,3E0,1.2E1,3E0,1.47E2,9E0,6E0,2.1E1,7E0,4E0,5E1,8E0,6E0,6E0,4E1,1.07E2,6E0,3E0,1.4E1,7E0,3E0,4.7E1,3E0,5E0,3E0,3E0,3E0,3E0,2.5E1,1.5E1,9.6E1,1.1E1,3E0,3E0,9E0,5E0,4E0,3E0,2.9E1,1.8E1,2.1E1,4E0,3E0,1.2E1,6.4E1,3.2E1,8E0,3E0,6E0,3E0,1.9E1,1E1,1.1E1,7E0,3E0,1.8E1,4E0,8E0,5.7E1,7E0,2.2E1,1E1,5E0,3E0,1.2E1,7E0,7E0,3E0,8E0,3E0,3E0,4E0,1.3E1,5E0,5E0,3E0,4.4E1,1.3E1,3E0,4E0,8E0,1.4E1,3E0,4E0,5E0,3.9E1,5E0,8E0,1.1E1,3E0,2.4E1,1.5E1,1.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"113","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"3.1950474E-1","boost_from_average":"1","num_class":"0","num_feature":"82","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,1]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.1-orc.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.1-orc.json deleted file mode 100644 index 9ad114c10..000000000 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc_2.1-orc.json +++ /dev/null @@ -1 +0,0 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","diskBytesSpilled_sum","duration_sum","duration_max","duration_min","duration_mean","executorCPUTime_sum","executorDeserializeCPUTime_sum","executorDeserializeTime_sum","executorRunTime_sum","input_bytesRead_sum","input_recordsRead_sum","jvmGCTime_sum","memoryBytesSpilled_sum","output_bytesWritten_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_fetchWaitTime_sum","sr_localBlocksFetched_sum","sr_localBytesRead_sum","sr_remoteBlocksFetched_sum","sr_remoteBytesRead_sum","sr_remoteBytesReadToDisk_sum","sr_totalBytesRead_sum","sw_bytesWritten_sum","sw_recordsWritten_sum","sw_writeTime_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_Project","sqlOp_Scan orc ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","resourceProfileId","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","hasSqlID","cache_hit_ratio","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilledBool","memoryBytesSpilledBool","sqlOp_CartesianProduct","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_RunningWindowFunction","sqlOp_Scan parquet ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit"],"feature_types":["int","int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","i","float","float","float","float","float","float","float","float","float","float","i","i","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"130"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99,100,101,102,103,104,105,106,107,108,109,110,111,112,113,114,115,116,117,118,119,120,121,122,123,124,125,126,127,128,129,130],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[3.8459204E-2,-4.302819E-1,9.241068E-1,-8.891553E-1,-1.4127856E-1,5.9571767E-1,6.0652304E-2,-9.8783445E-1,-2.1576181E-2,-3.3674455E-1,2.5534147E-2,1.283566E-2,3.259903E-2,-4.680447E-2,-2.8899014E-2,-4.4110736E-1,6.7314324E-3,-2.1419922E-2,1.1517308E-2,-2.1067461E-2,-1.1577523E-2,5.5721E-3,-7.022175E-2,-1.1136285E-1,3.132191E-3,-1.6343698E-1,-8.4375206E-4,-3.798952E-3,-9.834931E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,-1,19,-1,21,-1,-1,-1,-1,23,25,-1,27,-1,-1,-1],"loss_changes":[6.598966E1,1.3666441E1,7.5798683E0,1.0700989E0,2.1280022E0,1.330637E0,0E0,1.4331818E-1,0E0,1.6051981E0,4.0852994E-1,0E0,0E0,0E0,0E0,5.4730892E-2,0E0,2.3450594E-1,0E0,0E0,0E0,0E0,1.4818448E-1,8.7897316E-2,0E0,3.71297E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,15,15,17,17,22,22,23,23,25,25],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,-1,20,-1,22,-1,-1,-1,-1,24,26,-1,28,-1,-1,-1],"split_conditions":[2.358256E6,1.001171E6,4.1126E4,2.865605E8,1.878295E6,4.615651E6,6.0652304E-2,7.00693E5,-2.1576181E-2,1.2075767E9,9.969132E8,1.283566E-2,3.259903E-2,-4.680447E-2,-2.8899014E-2,7.27488E5,6.7314324E-3,3.388615E7,1.1517308E-2,-2.1067461E-2,-1.1577523E-2,5.5721E-3,3.1646586E8,1.466404E6,3.132191E-3,6.3494E4,-8.4375206E-4,-3.798952E-3,-9.834931E-3],"split_indices":[6,2,3,16,2,6,0,2,0,16,16,0,0,0,0,6,0,16,0,0,0,0,16,6,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.57E2,1.03E2,5.4E1,3.9E1,6.4E1,3.3E1,2.1E1,3E1,9E0,2.9E1,3.5E1,1.2E1,2.1E1,2.2E1,8E0,2.4E1,5E0,3E1,5E0,1.8E1,6E0,7E0,2.3E1,1.8E1,5E0,1.1E1,7E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.7972056E-2,-4.3183073E-1,7.724213E-1,-7.215583E-1,-1.334666E-1,4.9128598E-1,1.087423E0,-8.685789E-1,-4.5759845E-1,-2.700145E-1,5.259501E-2,3.3414614E-1,6.373629E-1,3.3689592E-2,5.9703693E-2,-4.5265168E-2,-2.8186994E-2,-2.2781203E-2,-1.400898E-2,-3.4471995E-1,3.221572E-3,1.4757626E-1,-3.3232814E-3,7.27473E-3,1.6836287E-2,3.5029512E-2,9.963035E-3,-1.6453132E-2,-8.082274E-3,2.389883E-3,8.674618E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.361204E1,8.461681E0,4.690693E0,1.5429173E0,1.2845172E0,5.445266E-1,1.3336697E0,4.9394798E-1,8.313656E-3,7.7032495E-1,2.8023976E-1,9.123397E-2,8.9814997E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.3646326E-2,0E0,5.0862193E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,19,19,21,21],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.753E4,1.244363E6,2.8512E5,1.215705E8,2.400113E6,7.948114E6,3.4492E4,7.00693E5,1.8338531E3,1.2075767E9,5.6E1,1E0,2.18E2,3.3689592E-2,5.9703693E-2,-4.5265168E-2,-2.8186994E-2,-2.2781203E-2,-1.400898E-2,9.86E2,3.221572E-3,1.712E3,-3.3232814E-3,7.27473E-3,1.6836287E-2,3.5029512E-2,9.963035E-3,-1.6453132E-2,-8.082274E-3,2.389883E-3,8.674618E-3],"split_indices":[12,2,12,16,2,2,3,2,5,16,19,29,4,0,0,0,0,0,0,0,0,20,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.56E2,9.8E1,5.8E1,4.9E1,4.9E1,3.2E1,2.6E1,3E1,1.9E1,2.8E1,2.1E1,1.7E1,1.5E1,1.4E1,1.2E1,1.5E1,1.5E1,1.1E1,8E0,2.3E1,5E0,1.2E1,9E0,5E0,1.2E1,1E1,5E0,1.8E1,5E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.9333268E-2,-4.0151566E-1,8.0334973E-1,-7.900194E-1,-1.5086178E-1,4.4686183E-1,1.1257248E0,-8.888789E-1,-1.8304113E-2,-3.010097E-1,1.5537634E-2,5.2604413E-1,9.259177E-3,3.1848878E-2,5.9641853E-2,-4.431738E-2,-2.969515E-2,2.9556067E-3,-3.556196E-1,-3.7179887E-2,1.2836915E-2,1.5096332E-2,3.0881481E-2,-4.2006546E-1,-2.1424463E-1,-6.7764424E-2,4.654821E-3,-2.05423E-2,-8.655597E-3,-1.1997694E-2,-4.2638304E-3,-1.4261001E-1,1.1921475E-2,-9.371355E-3,-1.7388556E-3,-1.7591541E-3,3.051133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,27,29,31,-1,-1,-1,-1,-1,33,35,-1,-1,-1,-1],"loss_changes":[6.2353626E1,1.1858553E1,7.0818634E0,1.3860683E0,1.9035009E0,5.0174236E-1,2.244442E0,2.8508568E-1,0E0,8.295245E-1,5.543001E-1,5.4004145E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.2888947E-1,1.4799118E-1,0E0,0E0,0E0,1.6465235E-1,6.50242E-2,1.6273184E-1,0E0,0E0,0E0,0E0,0E0,9.518063E-2,4.5212578E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,18,18,19,19,23,23,24,24,25,25,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,28,30,32,-1,-1,-1,-1,-1,34,36,-1,-1,-1,-1],"split_conditions":[2.35198E6,9.31995E5,2.8512E5,1.4089133E8,2.313201E6,2.18E2,4.3962E4,5.85951E5,-1.8304113E-2,6.5E1,1.3957454E9,4.241883E6,9.259177E-3,3.1848878E-2,5.9641853E-2,-4.431738E-2,-2.969515E-2,2.9556067E-3,6.98458E5,4.436935E6,1.2836915E-2,1.5096332E-2,3.0881481E-2,5.911733E8,2.8418045E8,1.8619835E8,4.654821E-3,-2.05423E-2,-8.655597E-3,-1.1997694E-2,-4.2638304E-3,1.348232E6,1.32E2,-9.371355E-3,-1.7388556E-3,-1.7591541E-3,3.051133E-3],"split_indices":[6,9,12,16,2,4,3,2,0,20,16,6,0,0,0,0,0,0,6,2,0,0,0,16,21,16,0,0,0,0,0,6,17,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.22E2,6.5E1,4.7E1,7.5E1,3.2E1,3.3E1,3.6E1,1.1E1,3.9E1,3.6E1,2.3E1,9E0,1.4E1,1.9E1,2E1,1.6E1,5E0,3.4E1,3.1E1,5E0,1.3E1,1E1,2.2E1,1.2E1,2.6E1,5E0,1.7E1,5E0,7E0,5E0,1.3E1,1.3E1,7E0,6E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.9193029E-2,-3.6197194E-1,7.381787E-1,-6.871509E-1,-1.3382106E-1,4.5602673E-1,1.1375225E0,-3.454906E-2,-1.6457127E-2,-2.0672745E-1,2.3578712E-1,2.7506644E-1,2.7170638E-2,3.509323E-2,5.7013884E-2,-3.815543E-1,-5.742283E-2,4.8091803E-3,1.3520215E-2,4.885661E-3,1.4053473E-2,-1.8360483E-2,-1.1199992E-2,-1.1711729E-1,5.680225E-2,9.2849194E-4,-1.7034312E-1,5.7188203E-3,-1.7134708E-3,-9.334503E-3,-2.7132859E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,-1,-1,21,23,-1,-1,-1,-1,-1,-1,25,27,-1,29,-1,-1,-1,-1],"loss_changes":[4.904848E1,8.588261E0,6.467148E0,1.2556152E0,1.9350197E0,9.735875E-1,3.5651016E-1,0E0,0E0,1.5181351E0,7.7369034E-2,1.1702216E-1,0E0,0E0,0E0,2.398324E-2,2.3166648E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6567409E-1,9.070112E-2,0E0,6.2033683E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,15,15,16,16,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,-1,-1,22,24,-1,-1,-1,-1,-1,-1,26,28,-1,30,-1,-1,-1,-1],"split_conditions":[9.753E4,9.31995E5,2.4455E4,1.4089133E8,9.188361E8,4.241883E6,1.5992037E11,-3.454906E-2,-1.6457127E-2,1.878295E6,3.2373845E3,1E0,2.7170638E-2,3.509323E-2,5.7013884E-2,1.62E2,7.781E4,4.8091803E-3,1.3520215E-2,4.885661E-3,1.4053473E-2,-1.8360483E-2,-1.1199992E-2,1E0,2.7008596E9,9.2849194E-4,1.216696E6,5.7188203E-3,-1.7134708E-3,-9.334503E-3,-2.7132859E-3],"split_indices":[12,9,3,16,16,6,13,0,0,2,5,29,0,0,0,17,12,0,0,0,0,0,0,44,23,0,6,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.77E2,1.16E2,6.1E1,4.7E1,6.9E1,3.7E1,2.4E1,3.4E1,1.3E1,5.8E1,1.1E1,1.9E1,1.8E1,1E1,1.4E1,2.6E1,3.2E1,5E0,6E0,5E0,1.4E1,1.8E1,8E0,2.1E1,1.1E1,6E0,1.5E1,6E0,5E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.5391695E-3,-3.946949E-1,6.415311E-1,-7.125547E-1,-1.3199028E-1,3.42932E-1,1.0182788E0,-8.132552E-1,-1.726327E-2,-2.3585324E-1,1.8135652E-1,2.2485496E-1,4.894565E-1,2.8319312E-2,5.229002E-2,-3.9784644E-2,-2.5608838E-2,-3.670682E-1,-1.0570135E-1,2.751982E-3,1.2735328E-2,2.5650886E-1,3.8034685E-3,2.9298209E-2,8.7741865E-3,-1.7426506E-2,-9.135832E-3,2.1540502E-3,-1.5726294E-1,7.400196E-3,1.3283533E-2,-1.0838922E-2,-1.07000306E-1,-6.3406695E-3,-1.3862086E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,33,-1,-1],"loss_changes":[4.5688663E1,9.17481E0,7.413061E0,1.2746143E0,2.0605712E0,5.819907E-1,1.2148933E0,2.88723E-1,0E0,7.6386046E-1,1.9062352E-1,8.2267284E-2,7.56474E-1,0E0,0E0,0E0,0E0,3.3237457E-2,2.076307E-1,0E0,0E0,2.7431488E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.039837E-2,0E0,0E0,0E0,3.5700336E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,17,17,18,18,21,21,28,28,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,34,-1,-1],"split_conditions":[1.760607E6,1.038321E6,2.8512E5,1.4089133E8,7.64454E8,4.200155E6,4.3962E4,6.21271E5,-1.726327E-2,1.66882E6,2.9950735E3,7.0588E4,2.18E2,2.8319312E-2,5.229002E-2,-3.9784644E-2,-2.5608838E-2,5.922601E8,1E0,2.751982E-3,1.2735328E-2,3.67248E5,3.8034685E-3,2.9298209E-2,8.7741865E-3,-1.7426506E-2,-9.135832E-3,2.1540502E-3,4.0221E4,7.400196E-3,1.3283533E-2,-1.0838922E-2,2.644432E6,-6.3406695E-3,-1.3862086E-3],"split_indices":[6,2,12,16,16,6,3,2,0,9,5,20,4,0,0,0,0,16,44,0,0,18,0,0,0,0,0,0,18,0,0,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.78E2,1.1E2,6.8E1,4.9E1,6.1E1,3.9E1,2.9E1,3.6E1,1.3E1,4.6E1,1.5E1,2.3E1,1.6E1,1.1E1,1.8E1,2.3E1,1.3E1,2.2E1,2.4E1,8E0,7E0,1.8E1,5E0,9E0,7E0,1.7E1,5E0,6E0,1.8E1,8E0,1E1,5E0,1.3E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.8508604E-3,-2.967171E-1,7.097907E-1,-5.656524E-1,-1.3872038E-2,4.1333163E-1,4.5324598E-2,-7.9242116E-1,-4.1159844E-1,-7.1249485E-2,1.354503E-1,3.2403275E-1,2.4031831E-2,-3.842665E-2,-1.6662592E-2,-4.5150864E-1,-7.1864906E-3,-1.3791338E-1,1.4127976E-1,8.0882395E-3,5.7415594E-4,6.9958842E-3,3.6793023E-1,-2.0808943E-2,-1.1279885E-2,-1.4584361E-2,-7.4663036E-2,8.385816E-3,2.919299E-3,9.6493615E-3,1.8298917E-2,1.9860063E-3,-1.0404144E-1,-1.3312201E-1,-1.879556E-4,-8.686196E-3,-2.9286642E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,-1,-1,23,-1,25,27,-1,-1,-1,29,-1,-1,-1,31,-1,-1,-1,-1,-1,33,35,-1,-1,-1],"loss_changes":[3.6844826E1,9.196261E0,4.6371574E0,1.8397865E0,5.2371484E-1,1.770587E-1,0E0,5.3098965E-1,3.1003237E-1,6.4287746E-1,1.0122335E-1,8.479047E-2,0E0,0E0,0E0,3.7653923E-2,0E0,3.9704823E-1,2.876155E-2,0E0,0E0,0E0,2.0357132E-2,0E0,0E0,0E0,1.0005072E-1,0E0,0E0,0E0,0E0,0E0,6.323454E-2,6.0599297E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,22,22,26,26,32,32,33,33],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,-1,-1,24,-1,26,28,-1,-1,-1,30,-1,-1,-1,32,-1,-1,-1,-1,-1,34,36,-1,-1,-1],"split_conditions":[1.07218E5,1.405389E6,3.0409E4,7.05246E5,1.804488E6,2.865E3,4.5324598E-2,2.865605E8,4.355446E-1,8.949362E8,5.6E1,9.275E3,2.4031831E-2,-3.842665E-2,-1.6662592E-2,7.984941E8,-7.1864906E-3,1.878295E6,4.550049E9,8.0882395E-3,5.7415594E-4,6.9958842E-3,1.337099E9,-2.0808943E-2,-1.1279885E-2,-1.4584361E-2,8.18647E5,8.385816E-3,2.919299E-3,9.6493615E-3,1.8298917E-2,1.9860063E-3,7.337122E-1,1.235866E6,-1.879556E-4,-8.686196E-3,-2.9286642E-3],"split_indices":[12,2,3,2,6,0,0,16,63,16,19,3,0,0,0,16,0,2,10,0,0,0,21,0,0,0,6,0,0,0,0,0,63,6,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,1.2E2,5.1E1,6.1E1,5.9E1,2.8E1,2.3E1,2.3E1,3.8E1,4.3E1,1.6E1,1.9E1,9E0,1.8E1,5E0,3.2E1,6E0,3.3E1,1E1,1.1E1,5E0,5E0,1.4E1,2.7E1,5E0,7E0,2.6E1,5E0,5E0,5E0,9E0,5E0,2.1E1,1.6E1,5E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.862732E-2,-3.331382E-1,6.364794E-1,-6.618091E-1,-1.2506208E-1,3.3134815E-1,9.407367E-1,-7.6267844E-1,-1.772496E-2,-2.5448853E-1,2.873252E-2,4.2010686E-1,1.4840807E-1,2.6012E-2,5.103764E-2,-4.5387484E-2,-2.9074958E-2,6.737845E-3,-3.1234834E-1,-4.3807507E-2,1.1665064E-1,2.5305867E-1,3.1340756E-2,1.46152945E-2,-4.096066E-3,-1.4772076E-2,-5.0961925E-3,-1.043827E-1,5.4570697E-3,7.564739E-3,4.774814E-4,1.3223918E-2,6.6661746E-3,-6.0432516E-3,-2.3133087E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,25,27,29,31,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8741177E1,8.60258E0,5.351431E0,9.5415115E-1,1.5798143E0,4.5725346E-1,1.6674995E0,2.1715164E-1,0E0,1.06545E0,2.4108072E-1,8.6348534E-1,5.734272E-1,0E0,0E0,0E0,0E0,0E0,1.5444231E-1,2.2782233E-1,9.993781E-2,2.8847456E-2,0E0,0E0,0E0,0E0,0E0,2.0675242E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,18,18,19,19,20,20,21,21,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,26,28,30,32,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.753E4,1.008791E6,2.8512E5,1.4089133E8,2.12954E6,2.18E2,3.4492E4,1.37E2,-1.772496E-2,6.5E1,2.7143E4,7.948114E6,3.2744336E3,2.6012E-2,5.103764E-2,-4.5387484E-2,-2.9074958E-2,6.737845E-3,3.0149117E9,1.804488E6,5.6882E4,1.20743E5,3.1340756E-2,1.46152945E-2,-4.096066E-3,-1.4772076E-2,-5.0961925E-3,2.53E2,5.4570697E-3,7.564739E-3,4.774814E-4,1.3223918E-2,6.6661746E-3,-6.0432516E-3,-2.3133087E-3],"split_indices":[12,2,12,16,2,4,3,0,0,20,7,2,5,0,0,0,0,0,26,6,22,28,0,0,0,0,0,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,1.26E2,6E1,4.8E1,7.8E1,3.1E1,2.9E1,3.3E1,1.5E1,4.2E1,3.6E1,2E1,1.1E1,1.3E1,1.6E1,6E0,2.7E1,5E0,3.7E1,2E1,1.6E1,1.4E1,6E0,6E0,5E0,3.2E1,5E0,1.5E1,5E0,1E1,6E0,8E0,6E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-6.183554E-2,-3.355048E-1,5.164821E-1,-6.8621683E-1,-1.62561E-1,2.5760937E-1,8.1682503E-1,-7.508378E-1,-1.5277884E-2,-2.711945E-1,-2.5271144E-2,3.0759227E-1,-8.371688E-5,1.9358682E-2,4.091266E-2,-4.3497376E-2,-2.8457712E-2,-3.24414E-1,2.4640262E-3,-5.6700103E-2,4.9336106E-3,1.5038028E-1,4.1285568E-1,-3.881482E-1,-2.0081173E-1,2.2986669E-3,-9.886311E-2,1.4136429E-3,1.1478022E-2,1.120715E-2,2.2580124E-2,-1.8289661E-2,-9.848631E-3,-3.416562E-3,-1.1097284E-2,-6.4487252E-3,-5.365485E-2,-3.0926133E-3,-1.4860771E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,25,-1,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,37,-1,-1],"loss_changes":[2.7078802E1,6.9242496E0,4.06586E0,5.627136E-1,1.1713743E0,4.0271997E-1,7.090912E-1,1.03178024E-1,0E0,7.8555465E-1,1.6289401E-1,3.6818027E-1,0E0,0E0,0E0,0E0,0E0,2.2140718E-1,0E0,1.4354154E-1,0E0,1.5090537E-1,1.1178517E-1,2.59161E-2,7.832056E-2,0E0,4.0692195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9718036E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,17,17,19,19,21,21,22,22,23,23,24,24,26,26,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,26,-1,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,38,-1,-1],"split_conditions":[2.210099E6,8.38657E5,2.8512E5,2.8681312E8,1.878295E6,3.35E2,7.523642E9,1.2858E4,-1.5277884E-2,1.2075767E9,7.9540166E8,3.3012173E9,-8.371688E-5,1.9358682E-2,4.091266E-2,-4.3497376E-2,-2.8457712E-2,1.62E2,2.4640262E-3,1E0,4.9336106E-3,1.84E2,1.0300973E7,6.60937E5,7.952E3,2.2986669E-3,2.01E2,1.4136429E-3,1.1478022E-2,1.120715E-2,2.2580124E-2,-1.8289661E-2,-9.848631E-3,-3.416562E-3,-1.1097284E-2,-6.4487252E-3,2.807764E6,-3.0926133E-3,-1.4860771E-3],"split_indices":[6,2,12,16,2,4,21,7,0,16,16,11,0,0,0,0,0,17,0,44,0,19,2,6,28,0,4,0,0,0,0,0,0,0,0,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.69E2,1.15E2,5.4E1,3.7E1,7.8E1,3E1,2.4E1,3E1,7E0,4.3E1,3.5E1,2.5E1,5E0,7E0,1.7E1,6E0,2.4E1,3.7E1,6E0,2.9E1,6E0,1.1E1,1.4E1,2.3E1,1.4E1,8E0,2.1E1,6E0,5E0,7E0,7E0,1.8E1,5E0,5E0,9E0,9E0,1.2E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.3730377E-2,-4.1902357E-1,4.2514673E-1,-6.5887636E-1,-2.4450068E-1,1.6878414E-1,7.594859E-1,-7.0691097E-1,-1.2084417E-2,-3.1271097E-1,-3.721182E-2,6.59869E-2,2.779272E-1,5.45222E-1,4.1101344E-2,-4.0634923E-2,-2.7307432E-2,-3.5655096E-1,-1.5674825E-1,-8.244833E-3,2.5106282E-3,-2.8362978E-2,1.7056285E-1,3.103734E-1,7.1607376E-3,1.3618083E-2,2.9036913E-2,-2.032265E-2,-2.9362968E-1,-2.305224E-3,-1.023767E-2,-5.560112E-3,2.6986739E-3,9.627296E-3,3.6572407E-3,7.874225E-3,1.5409467E-2,-1.4789578E-2,-8.771463E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,-1,-1,31,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.058813E1,3.7737274E0,6.6430626E0,5.199146E-1,7.8016067E-1,4.9776232E-1,8.1384087E-1,3.6684036E-2,0E0,2.3128271E-1,2.2641188E-1,2.5150588E-1,3.0359745E-2,2.963748E-1,0E0,0E0,0E0,8.287954E-2,7.656714E-2,0E0,0E0,1.3291548E-1,3.3590436E-2,2.5048018E-2,0E0,0E0,0E0,0E0,2.827382E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,21,21,22,22,23,23,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,-1,-1,32,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.400113E6,8.38657E5,5.625966E6,2.8681312E8,1.882904E6,1.07218E5,1.2862438E9,1.2858E4,-1.2084417E-2,3.48E2,3.135717E6,2.4424E4,1.6946304E7,7.6E1,4.1101344E-2,-4.0634923E-2,-2.7307432E-2,1.7513405E3,4.79E2,-8.244833E-3,2.5106282E-3,1.733293E6,2.6362E4,7.6162E4,7.1607376E-3,1.3618083E-2,2.9036913E-2,-2.032265E-2,2.4127E4,-2.305224E-3,-1.023767E-2,-5.560112E-3,2.6986739E-3,9.627296E-3,3.6572407E-3,7.874225E-3,1.5409467E-2,-1.4789578E-2,-8.771463E-3],"split_indices":[2,2,6,16,2,12,16,7,0,17,18,7,18,19,0,0,0,5,0,0,0,6,28,28,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,9.3E1,7.8E1,3.8E1,5.5E1,4.5E1,3.3E1,3.3E1,5E0,4.1E1,1.4E1,2.4E1,2.1E1,1.7E1,1.6E1,6E0,2.7E1,3.1E1,1E1,5E0,9E0,1.3E1,1.1E1,1.5E1,6E0,7E0,1E1,9E0,2.2E1,5E0,5E0,6E0,7E0,6E0,5E0,5E0,1E1,1.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-6.1245495E-3,-2.8095084E-1,5.859268E-1,-5.1802397E-1,-1.0786921E-1,4.120535E-1,4.6957545E-2,-5.569098E-1,-6.9610556E-3,-2.1454027E-1,1.948652E-2,2.7298975E-1,5.6685424E-1,-2.55612E-2,-1.438679E-2,-2.1425972E-3,-2.600588E-1,-3.9642997E-2,8.486675E-3,3.2676694E-1,2.887675E-3,1.786543E-2,2.9632404E-2,-1.31869875E-2,-6.009076E-3,-7.21691E-2,3.0536917E-3,7.662985E-3,1.6211135E-2,-5.978467E-3,-3.92409E-2,-1.3596231E-4,-2.8783358E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,-1,-1,-1,23,25,-1,27,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,31,-1,-1],"loss_changes":[2.6686552E1,4.539817E0,3.986292E0,5.664587E-1,9.008019E-1,7.063408E-1,0E0,8.119106E-2,0E0,2.5835288E-1,3.2839912E-1,2.3420393E-1,3.9868355E-2,0E0,0E0,0E0,1.0256779E-1,9.061095E-2,0E0,5.6306005E-2,0E0,0E0,0E0,0E0,0E0,3.516893E-2,0E0,0E0,0E0,0E0,1.330095E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,16,16,17,17,19,19,25,25,30,30],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,-1,-1,-1,24,26,-1,28,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,32,-1,-1],"split_conditions":[5.166426E6,1.15856E6,5.94024E5,2.56E2,2.313201E6,2.3284E5,4.6957545E-2,6.502E3,-6.9610556E-3,4.66838E5,1.0656E4,1.96E2,2.7637E4,-2.55612E-2,-1.438679E-2,-2.1425972E-3,1.882904E6,3.1149E4,8.486675E-3,1.07218E5,2.887675E-3,1.786543E-2,2.9632404E-2,-1.31869875E-2,-6.009076E-3,1.208075E6,3.0536917E-3,7.662985E-3,1.6211135E-2,-5.978467E-3,1.195E3,-1.3596231E-4,-2.8783358E-3],"split_indices":[2,2,12,17,2,12,0,3,0,6,3,17,3,0,0,0,2,7,0,12,0,0,0,0,0,6,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.62E2,1.11E2,5.1E1,4.6E1,6.5E1,3.9E1,1.2E1,4.1E1,5E0,3.5E1,3E1,2.2E1,1.7E1,3.5E1,6E0,8E0,2.7E1,2.3E1,7E0,1.7E1,5E0,9E0,8E0,1.9E1,8E0,1.8E1,5E0,5E0,1.2E1,5E0,1.3E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-3.4319174E-2,-2.7380013E-1,4.7519073E-1,-5.4904974E-1,-1.10842474E-1,2.7612212E-1,6.787784E-1,-6.384794E-1,-3.3358708E-1,-1.6121496E-1,1.037651E-2,3.244564E-1,1.2530562E-3,2.2502407E-2,3.665661E-2,-3.9203428E-2,-2.4032371E-2,-8.006974E-3,-1.7035889E-2,-2.7546617E-1,-1.9650156E-2,6.563735E-3,3.7151223E-1,-2.9881272E-1,-5.7929074E-3,-6.1467048E-2,5.1682178E-2,1.1823095E-2,1.8986097E-2,-1.720244E-2,-1.1112099E-2,1.9228366E-3,-9.957781E-2,4.6380404E-3,-8.9951145E-4,-6.2212027E-3,-1.3147346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,27,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[2.1361225E1,5.2767715E0,2.082138E0,6.348362E-1,1.372799E0,3.4944153E-1,2.7683353E-1,1.9008827E-1,5.614519E-2,1.0782418E0,0E0,1.4891076E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.3646536E-2,9.5646545E-2,0E0,3.5440922E-3,3.0323982E-2,0E0,8.481682E-2,4.953813E-2,0E0,0E0,0E0,0E0,0E0,4.0460676E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,19,19,20,20,22,22,23,23,25,25,26,26,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,28,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[2.269467E6,1.001171E6,2.8512E5,1.4089133E8,1.5063692E9,3.35E2,4.3962E4,1.37E2,3.22E2,1.914061E6,1.037651E-2,1.16415E5,1.2530562E-3,2.2502407E-2,3.665661E-2,-3.9203428E-2,-2.4032371E-2,-8.006974E-3,-1.7035889E-2,7.432659E-1,6.9504E4,6.563735E-3,1.2535E4,1.217902E6,-5.7929074E-3,1.25234E5,3.91034E9,1.1823095E-2,1.8986097E-2,-1.720244E-2,-1.1112099E-2,1.9228366E-3,1.466404E6,4.6380404E-3,-8.9951145E-4,-6.2212027E-3,-1.3147346E-3],"split_indices":[6,2,12,16,16,4,3,0,0,9,0,12,0,0,0,0,0,0,0,63,8,0,3,2,0,21,25,0,0,0,0,0,6,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,1.18E2,5.5E1,4.3E1,7.5E1,2.9E1,2.6E1,2.9E1,1.4E1,6.6E1,9E0,2.4E1,5E0,1.5E1,1.1E1,5E0,2.4E1,5E0,9E0,3.6E1,3E1,6E0,1.8E1,3E1,6E0,1.9E1,1.1E1,9E0,9E0,7E0,2.3E1,5E0,1.4E1,6E0,5E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.518222E-2,-3.5578614E-1,4.0686136E-1,-6.028731E-1,-2.2816823E-1,1.6617844E-1,6.951626E-1,-3.829586E-2,-4.881199E-1,-2.863877E-1,-7.433487E-2,6.395866E-2,2.6593825E-1,5.103198E-1,3.89357E-2,-2.3549657E-2,-1.1394052E-2,-3.1602076E-1,-2.6562675E-3,-1.6420491E-1,5.1413607E-3,-2.2758968E-2,1.4610782E-1,5.1693344E-3,3.0426738E-1,6.0058975E-1,6.0848934E-3,-3.482535E-1,-6.969304E-3,-9.281196E-3,-3.5060153E-3,-2.7449036E-3,6.634126E-4,7.7577075E-3,3.2434696E-3,1.4851721E-2,7.1384762E-3,2.1045541E-2,3.39044E-2,-1.6293775E-2,-9.435539E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,-1,-1,-1,27,-1,29,-1,31,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5668608E1,2.5111933E0,6.337673E0,4.5380592E-1,4.8920918E-1,5.13257E-1,1.0733604E0,0E0,1.1606598E-1,2.54776E-1,3.1287563E-1,1.9566731E-1,1.1095929E-1,7.198353E-1,0E0,0E0,0E0,1.2581348E-1,0E0,3.167352E-2,0E0,2.2281647E-2,1.8584847E-2,0E0,4.5671344E-2,2.1868229E-2,0E0,1.5264273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,17,17,19,19,21,21,22,22,24,24,25,25,27,27],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,-1,-1,-1,28,-1,30,-1,32,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.400113E6,7.8591E5,6.476062E6,1.37E2,3.48E2,3.733076E8,1.2862438E9,-3.829586E-2,1.8077296E8,1.973751E6,2.9697178E3,4.460193E6,8.8449E4,4.6665716E7,3.89357E-2,-2.3549657E-2,-1.1394052E-2,6.142029E8,-2.6562675E-3,3.1839E4,5.1413607E-3,1E0,1.5E2,5.1693344E-3,1.9840434E7,1.4357605E7,6.0848934E-3,6.69829E5,-6.969304E-3,-9.281196E-3,-3.5060153E-3,-2.7449036E-3,6.634126E-4,7.7577075E-3,3.2434696E-3,1.4851721E-2,7.1384762E-3,2.1045541E-2,3.39044E-2,-1.6293775E-2,-9.435539E-3],"split_indices":[2,2,6,0,17,16,16,0,16,2,5,2,12,2,0,0,0,16,0,7,0,29,17,0,18,6,0,6,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,8.3E1,9.2E1,2.7E1,5.6E1,5.1E1,4.1E1,6E0,2.1E1,4E1,1.6E1,2.6E1,2.5E1,2.3E1,1.8E1,1.6E1,5E0,3.5E1,5E0,1.1E1,5E0,1.3E1,1.3E1,6E0,1.9E1,1.8E1,5E0,2.8E1,7E0,6E0,5E0,6E0,7E0,8E0,5E0,1.4E1,5E0,1.3E1,5E0,2.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.19968E-2,-2.4777587E-1,4.8822427E-1,-4.8981944E-1,-1.11554705E-1,3.144498E-1,4.0361583E-2,-5.600421E-1,-2.7928528E-1,-2.3697613E-1,-3.5513514E-3,1.7202812E-1,4.0095517E-1,-2.7638597E-2,-1.6816754E-2,-7.3952274E-3,-1.4262237E-2,-3.0725682E-1,-6.4953275E-2,6.281724E-3,-2.6148401E-2,8.07332E-2,1.489069E-2,5.2154106E-1,2.4025072E-1,-1.4402289E-2,-7.705518E-3,5.673995E-3,-9.723442E-3,-5.4342456E-2,4.8230686E-3,7.746952E-3,-1.2028345E-3,1.4887015E-2,3.0784687E-2,1.6123058E-2,1.2226977E-3,-7.656879E-2,1.7088155E-3,-1.1363198E-1,-2.0770472E-2,-7.4964343E-3,-1.4858831E-3,7.075388E-4,-2.1605769E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,-1,-1,25,27,-1,29,31,-1,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,39,-1,41,43,-1,-1,-1,-1],"loss_changes":[2.1350468E1,4.000848E0,4.108719E0,4.940958E-1,1.0828333E0,4.6079445E-1,0E0,1.6529942E-1,1.2438655E-2,4.2736864E-1,1.5039104E-1,2.503968E-1,3.7854528E-1,0E0,0E0,0E0,0E0,1.8947601E-2,3.9816996E-1,0E0,1.562613E-1,1.4110652E-1,0E0,2.2372627E-1,3.4597868E-1,0E0,0E0,0E0,0E0,7.15722E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.347751E-2,0E0,6.816383E-2,1.3552347E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,20,20,21,21,23,23,24,24,29,29,37,37,39,39,40,40],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,-1,-1,26,28,-1,30,32,-1,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,40,-1,42,44,-1,-1,-1,-1],"split_conditions":[1.02979E5,1.003136E6,5.94024E5,1.8077296E8,1.882904E6,3.857587E6,4.0361583E-2,7.00693E5,4.43E2,3.48E2,4.550049E9,1.1613266E9,1.71E2,-2.7638597E-2,-1.6816754E-2,-7.3952274E-3,-1.4262237E-2,1.625561E6,1.3204E4,6.281724E-3,2.210099E6,3.059E4,1.489069E-2,2.0485E4,3.2744336E3,-1.4402289E-2,-7.705518E-3,5.673995E-3,-9.723442E-3,3.37E2,4.8230686E-3,7.746952E-3,-1.2028345E-3,1.4887015E-2,3.0784687E-2,1.6123058E-2,1.2226977E-3,5.309569E6,1.7088155E-3,2.826849E6,1.185915E6,-7.4964343E-3,-1.4858831E-3,7.075388E-4,-2.1605769E-3],"split_indices":[12,2,12,16,2,6,0,2,0,17,10,16,4,0,0,0,0,9,28,0,6,20,0,3,5,0,0,0,0,4,0,0,0,0,0,0,0,18,0,2,6,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,1.22E2,5.7E1,4.3E1,7.9E1,4.2E1,1.5E1,3.1E1,1.2E1,3.6E1,4.3E1,1.7E1,2.5E1,2E1,1.1E1,5E0,7E0,2.5E1,1.1E1,5E0,3.8E1,1.2E1,5E0,1.3E1,1.2E1,2E1,5E0,5E0,6E0,3.2E1,6E0,6E0,6E0,8E0,5E0,7E0,5E0,2.6E1,6E0,1.5E1,1.1E1,8E0,7E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.788101E-3,-2.4758166E-1,5.2553314E-1,-4.7949526E-1,-1.05132416E-1,3.620034E-1,8.986342E-1,-3.343824E-2,-3.968194E-1,-1.4479683E-1,1.0700334E-2,2.5922126E-1,5.12823E-1,2.866746E-2,5.020606E-2,-4.310118E-1,-6.473861E-3,-2.3264444E-1,-1.863472E-2,1.3875192E-2,3.4974783E-3,1.3614451E-2,2.5131233E-2,-2.027795E-2,-1.1880754E-2,-2.4776536E-1,-4.6119518E-3,-5.2480143E-2,3.350754E-3,-1.1602564E-2,-6.003212E-3,-7.481476E-2,8.532079E-4,-4.4106096E-3,-9.99381E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,23,-1,25,27,-1,-1,-1,-1,-1,-1,29,-1,31,-1,-1,-1,33,-1,-1,-1],"loss_changes":[2.3307146E1,3.8808641E0,3.2134533E0,7.6231E-1,1.0775489E0,5.181303E-1,2.7902794E-1,0E0,2.5044775E-1,7.4931145E-1,0E0,2.582456E-1,2.2785664E-2,0E0,0E0,4.4183254E-2,0E0,5.062151E-2,9.76698E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.3120632E-2,0E0,3.6854308E-2,0E0,0E0,0E0,2.0843305E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,15,15,17,17,18,18,25,25,27,27,31,31],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,24,-1,26,28,-1,-1,-1,-1,-1,-1,30,-1,32,-1,-1,-1,34,-1,-1,-1],"split_conditions":[2.35198E6,1.003136E6,6.25049E5,1.45E2,1.4817759E9,2.8512E5,1.0966112E4,-3.343824E-2,1.2194085E7,1.070527E6,1.0700334E-2,1.52E2,7.3E1,2.866746E-2,5.020606E-2,1.8077296E8,-6.473861E-3,8.663E3,1.806E3,1.3875192E-2,3.4974783E-3,1.3614451E-2,2.5131233E-2,-2.027795E-2,-1.1880754E-2,1.9951667E10,-4.6119518E-3,3.1149E4,3.350754E-3,-1.1602564E-2,-6.003212E-3,1.747361E6,8.532079E-4,-4.4106096E-3,-9.99381E-4],"split_indices":[6,2,12,0,16,12,5,0,18,6,0,17,19,0,0,16,0,3,0,0,0,0,0,0,0,10,0,7,0,0,0,6,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.18E2,5.7E1,4.4E1,7.4E1,4.1E1,1.6E1,8E0,3.6E1,6.7E1,7E0,2.6E1,1.5E1,1E1,6E0,3.1E1,5E0,3.9E1,2.8E1,1.9E1,7E0,5E0,1E1,2.4E1,7E0,3.4E1,5E0,2.1E1,7E0,2.8E1,6E0,1.6E1,5E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.1189753E-2,-2.4367955E-1,4.3489724E-1,-4.7844088E-1,-7.984707E-2,2.745275E-1,7.8339046E-1,-5.3308845E-1,-9.876939E-3,-1.9220835E-1,1.8286611E-3,1.728273E-1,1.6078299E-2,2.141774E-2,3.8441144E-2,-3.5660375E-2,-2.0629188E-2,-3.4854412E-3,-9.878697E-3,-2.4440462E-2,6.4564254E-3,2.5976506E-1,7.1487285E-2,-6.261445E-2,5.7313005E-3,6.886201E-3,1.3534019E-2,5.1689753E-3,2.099287E-4,-1.2086727E-1,-2.161939E-2,-6.933785E-3,-2.8635426E-3,1.7312843E-3,-7.536768E-2,-5.3491914E-3,-6.8168214E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1,23,-1,25,27,29,-1,-1,-1,-1,-1,31,33,-1,-1,-1,35,-1,-1],"loss_changes":[1.9183218E1,4.8464127E0,3.249979E0,5.9589195E-1,6.978179E-1,3.5627103E-1,1.0912895E-1,3.3472443E-1,0E0,1.046108E-1,1.763161E-1,1.8406636E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4010177E-1,0E0,1.2660146E-2,3.6988344E-2,7.380541E-2,0E0,0E0,0E0,0E0,0E0,1.6871572E-2,6.886886E-2,0E0,0E0,0E0,2.9100206E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,19,19,21,21,22,22,23,23,29,29,30,30,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1,24,-1,26,28,30,-1,-1,-1,-1,-1,32,34,-1,-1,-1,36,-1,-1],"split_conditions":[9.753E4,9.31995E5,4.3962E4,1.8077296E8,1.882904E6,3.857587E6,1.5992037E11,3.7681E4,-9.876939E-3,7.952E3,5.166426E6,1.11E2,1.6078299E-2,2.141774E-2,3.8441144E-2,-3.5660375E-2,-2.0629188E-2,-3.4854412E-3,-9.878697E-3,7.808233E8,6.4564254E-3,7.04906E8,1.31729E5,2.644432E6,5.7313005E-3,6.886201E-3,1.3534019E-2,5.1689753E-3,2.099287E-4,1.800758E6,2.3348113E10,-6.933785E-3,-2.8635426E-3,1.7312843E-3,4.352129E9,-5.3491914E-3,-6.8168214E-4],"split_indices":[12,9,3,16,2,6,13,21,0,28,2,4,0,0,0,0,0,0,0,16,0,16,12,2,0,0,0,0,0,18,10,0,0,0,11,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.26E2,6.1E1,5.1E1,7.5E1,4.3E1,1.8E1,4.1E1,1E1,3.1E1,4.4E1,2.2E1,2.1E1,6E0,1.2E1,5E0,3.6E1,8E0,2.3E1,3.8E1,6E0,1.1E1,1.1E1,3.1E1,7E0,5E0,6E0,6E0,5E0,1.2E1,1.9E1,6E0,6E0,9E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.1144923E-2,-2.848554E-1,3.8784957E-1,-4.8116404E-1,-1.3941996E-1,1.524637E-1,6.059233E-1,-5.754484E-1,-2.8609365E-1,-1.7842296E-1,1.0533932E-2,9.841009E-2,9.886675E-3,5.042171E-1,3.9232824E-2,-3.003258E-2,-2.0168966E-2,-1.4249338E-2,-7.1751666E-3,-2.3230073E-1,-4.4325534E-2,1.8038571E-1,3.57743E-5,2.5635151E-2,1.1097773E-2,-2.5061077E-1,-4.9232044E-3,-8.785979E-2,1.3276369E-3,4.5275814E-3,1.0464383E-2,-2.097158E-3,2.1000206E-3,-2.8003225E-1,-6.155909E-3,-4.879459E-3,-2.1510245E-3,-1.3744E-2,-8.565963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,-1,-1,-1,-1,-1,25,27,29,31,-1,-1,33,-1,35,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1],"loss_changes":[1.8785213E1,2.8450403E0,3.500822E0,6.262264E-1,9.3411815E-1,1.1780429E-1,6.1937046E-1,3.9528847E-2,2.4950743E-2,3.87071E-1,0E0,1.7740802E-1,0E0,4.5312452E-1,0E0,0E0,0E0,0E0,0E0,5.7069063E-2,5.798859E-2,2.9736847E-2,2.7741434E-2,0E0,0E0,6.688905E-2,0E0,4.646994E-3,0E0,0E0,0E0,0E0,0E0,1.1208415E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,19,19,20,20,21,21,22,22,25,25,27,27,33,33],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,-1,-1,-1,-1,-1,26,28,30,32,-1,-1,34,-1,36,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1],"split_conditions":[1.804488E6,9.31995E5,6.476062E6,1.3567034E8,1.4817759E9,3.329577E6,2.440942E9,4.1E1,1.2194085E7,1.070527E6,1.0533932E-2,6.6221E4,9.886675E-3,2.32E2,3.9232824E-2,-3.003258E-2,-2.0168966E-2,-1.4249338E-2,-7.1751666E-3,6.7842746E8,2.9178448E8,2.2E1,2.7603E4,2.5635151E-2,1.1097773E-2,4.005696E8,-4.9232044E-3,1.466404E6,1.3276369E-3,4.5275814E-3,1.0464383E-2,-2.097158E-3,2.1000206E-3,1.455479E6,-6.155909E-3,-4.879459E-3,-2.1510245E-3,-1.3744E-2,-8.565963E-3],"split_indices":[6,9,6,16,16,6,16,17,18,6,0,22,0,4,0,0,0,0,0,16,16,17,7,0,0,23,0,6,0,0,0,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.7E2,1.01E2,6.9E1,4.2E1,5.9E1,3.4E1,3.5E1,2.7E1,1.5E1,5.4E1,5E0,2.1E1,1.3E1,2.8E1,7E0,1.1E1,1.6E1,1E1,5E0,3.8E1,1.6E1,1.1E1,1E1,2E1,8E0,3.2E1,6E0,1E1,6E0,6E0,5E0,5E0,5E0,2.4E1,8E0,5E0,5E0,1.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.2266054E-2,-2.2305503E-1,4.383946E-1,-4.5476827E-1,-7.760259E-2,2.4175441E-1,6.4558524E-1,-5.14713E-1,-9.006278E-3,-1.6528179E-1,4.6968494E-2,3.8136047E-1,1.5948908E-1,5.182044E-1,4.559729E-2,-3.2238785E-2,-1.9625708E-2,-2.0650916E-1,6.284524E-3,-1.3598111E-2,1.0477225E-1,1.0800439E-2,2.1664036E-2,2.3629785E-1,-5.3738793E-3,1.26298405E-2,2.4313983E-2,-2.3832674E-1,-1.1964614E-1,3.100107E-3,-5.477746E-2,7.1698693E-3,1.4679931E-3,1.4396335E-1,1.5573402E-2,-1.2372735E-2,-4.6485798E-3,-1.5535329E-3,-8.144812E-3,-4.715238E-4,-4.231624E-3,9.262867E-3,1.2638848E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,35,37,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9451254E1,4.177615E0,2.6551256E0,6.085949E-1,8.597019E-1,3.6411238E-1,1.1362839E0,2.2150707E-1,0E0,6.1624706E-1,1.1695695E-1,7.735181E-2,5.717677E-1,7.888794E-2,0E0,0E0,0E0,8.4799886E-2,0E0,6.313235E-2,6.642902E-2,0E0,0E0,1.7117119E-1,0E0,0E0,0E0,1.423974E-1,6.652516E-2,0E0,2.1207388E-2,0E0,0E0,9.5907986E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,19,19,20,20,23,23,27,27,28,28,30,30,33,33],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,36,38,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.07218E5,9.31995E5,2.8512E5,1.3567034E8,2.12954E6,7E1,1.0966112E4,3.7681E4,-9.006278E-3,1.4817759E9,6.9504E4,5.8678E4,2.11E2,1.9412997E9,4.559729E-2,-3.2238785E-2,-1.9625708E-2,6.60937E5,6.284524E-3,3E1,8.3168E4,1.0800439E-2,2.1664036E-2,4.241883E6,-5.3738793E-3,1.26298405E-2,2.4313983E-2,5.922601E8,6.0676E4,3.100107E-3,5.9005E4,7.1698693E-3,1.4679931E-3,9.304E4,1.5573402E-2,-1.2372735E-2,-4.6485798E-3,-1.5535329E-3,-8.144812E-3,-4.715238E-4,-4.231624E-3,9.262867E-3,1.2638848E-3],"split_indices":[12,9,12,16,2,19,5,21,0,16,8,28,4,27,0,0,0,6,0,17,8,0,0,6,0,0,0,16,12,0,8,0,0,28,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.92E2,1.24E2,6.8E1,4.7E1,7.7E1,3.6E1,3.2E1,3.7E1,1E1,4.5E1,3.2E1,1.2E1,2.4E1,2.6E1,6E0,6E0,3.1E1,4E1,5E0,1.6E1,1.6E1,7E0,5E0,1.9E1,5E0,5E0,2.1E1,2.8E1,1.2E1,5E0,1.1E1,8E0,8E0,1.2E1,7E0,2E1,8E0,6E0,6E0,6E0,5E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.0559558E-2,-1.8084738E-1,4.5834744E-1,-2.8800634E-1,2.1917863E-2,3.379899E-1,7.7121884E-1,-3.3761808E-1,-2.9242052E-2,-1.0168206E-2,7.622209E-3,3.7560415E-1,5.2767165E-4,2.409115E-2,4.3891415E-2,-4.2171362E-1,-2.6516816E-1,4.627596E-3,-6.9979145E-3,5.6449277E-3,-4.4680014E-2,2.50154E-1,2.3392262E-2,-1.964765E-2,-1.0787869E-2,-2.8428447E-1,-5.333912E-3,-1.2735991E-1,1.6310502E-2,1.3951016E-2,1.8675223E-1,-3.0107096E-1,-7.1651773E-3,-7.1318643E-3,-2.797228E-3,5.0677054E-2,-1.7469309E-3,4.561486E-3,1.04362955E-2,-1.0278008E-2,-1.5624267E-2,7.151682E-4,3.339977E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,-1,-1,23,25,-1,-1,-1,27,29,-1,-1,-1,31,-1,33,35,-1,37,39,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.669112E1,2.461589E0,2.1493483E0,9.401207E-1,1.9777733E-1,5.838666E-1,3.186102E-1,2.6592398E-1,2.4751067E-1,1.6946787E-1,0E0,7.308574E-1,0E0,0E0,0E0,2.2755623E-2,6.3643456E-2,0E0,0E0,0E0,1.4327262E-1,5.4840922E-2,0E0,0E0,0E0,1.3084888E-2,0E0,1.6891435E-2,3.4940775E-2,0E0,4.390496E-2,1.2736797E-2,0E0,0E0,0E0,8.495342E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,15,15,16,16,20,20,21,21,25,25,27,27,28,28,30,30,31,31,35,35],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,-1,-1,24,26,-1,-1,-1,28,30,-1,-1,-1,32,-1,34,36,-1,38,40,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.07218E5,1.878295E6,6.25049E5,3.48E2,9.969132E8,3.38511E7,1.0966112E4,6.31395E5,2.4625E4,3.388615E7,7.622209E-3,6.820745E6,5.2767165E-4,2.409115E-2,4.3891415E-2,1.3567034E8,6.142029E8,4.627596E-3,-6.9979145E-3,5.6449277E-3,6.6325E4,2.878453E3,2.3392262E-2,-1.964765E-2,-1.0787869E-2,1.525147E6,-5.333912E-3,1.004526E6,1.37E2,1.3951016E-2,7.04906E8,2.4151E4,-7.1651773E-3,-7.1318643E-3,-2.797228E-3,1E0,-1.7469309E-3,4.561486E-3,1.04362955E-2,-1.0278008E-2,-1.5624267E-2,7.151682E-4,3.339977E-3],"split_indices":[12,2,12,17,16,2,5,2,7,16,0,6,0,0,0,16,16,0,0,0,12,5,0,0,0,2,0,6,19,0,16,28,0,0,0,41,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.12E2,6.3E1,7.3E1,3.9E1,4.7E1,1.6E1,6.1E1,1.2E1,3.3E1,6E0,4.2E1,5E0,1E1,6E0,2.6E1,3.5E1,6E0,6E0,6E0,2.7E1,2.5E1,1.7E1,2.1E1,5E0,3E1,5E0,1.1E1,1.6E1,1E1,1.5E1,2.5E1,5E0,6E0,5E0,1E1,6E0,7E0,8E0,1.4E1,1.1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.7057758E-2,-2.2780924E-1,3.5751662E-1,-3.991772E-1,-1.02149144E-1,2.5038213E-1,3.1086389E-2,-2.9126776E-2,-3.4744474E-1,-1.3198519E-1,5.0079566E-3,1.0013749E-1,3.3139715E-1,-3.7179026E-1,-5.730467E-3,-2.3244648E-1,-7.092031E-2,1.6530904E-1,9.234397E-5,4.0522933E-1,2.517807E-1,-1.7035656E-2,-8.943376E-3,-1.210821E-2,-3.9298013E-3,3.3660063E-3,-9.9313095E-2,3.967653E-3,9.801531E-3,1.3178607E-2,2.3004584E-2,1.6790025E-1,1.4243108E-2,-1.4493592E-1,-3.892893E-2,2.1292916E-3,1.13059785E-2,-2.6269353E-3,-7.831198E-3,-3.323181E-3,8.6368027E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,-1,-1,13,15,-1,17,19,21,-1,23,25,27,-1,29,31,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,35,-1,37,39,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4685735E1,2.5217838E0,2.3677664E0,4.6174288E-1,4.6811724E-1,6.1193466E-1,0E0,0E0,1.8273115E-1,3.6386347E-1,0E0,1.2741488E-1,9.7831964E-2,4.053688E-2,0E0,1.22306705E-1,1.7578885E-1,3.2493293E-2,0E0,4.2209864E-2,6.40142E-2,0E0,0E0,0E0,0E0,0E0,8.802429E-2,0E0,0E0,0E0,0E0,1.0677162E-1,0E0,4.4731736E-2,3.6463197E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,26,26,31,31,33,33,34,34],"right_children":[2,4,6,8,10,12,-1,-1,14,16,-1,18,20,22,-1,24,26,28,-1,30,32,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,36,-1,38,40,-1,-1,-1,-1,-1,-1],"split_conditions":[1.804488E6,1.003136E6,4.3200794E9,1.37E2,1.2340036E9,3.329577E6,3.1086389E-2,-2.9126776E-2,1.2194085E7,6.88588E5,5.0079566E-3,4.1265E4,1.33E2,5.4354E4,-5.730467E-3,1.43E2,1E0,3.67248E5,9.234397E-5,3.0409E4,2.71764E5,-1.7035656E-2,-8.943376E-3,-1.210821E-2,-3.9298013E-3,3.3660063E-3,1.070527E6,3.967653E-3,9.801531E-3,1.3178607E-2,2.3004584E-2,2.838E3,1.4243108E-2,7.61979E5,5.309569E6,2.1292916E-3,1.13059785E-2,-2.6269353E-3,-7.831198E-3,-3.323181E-3,8.6368027E-4],"split_indices":[6,2,27,0,16,6,0,0,18,6,0,20,4,28,0,17,44,18,0,3,12,0,0,0,0,0,6,0,0,0,0,0,0,6,18,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.84E2,1.18E2,6.6E1,4.9E1,6.9E1,5.2E1,1.4E1,6E0,4.3E1,6.1E1,8E0,1.9E1,3.3E1,3.8E1,5E0,2.2E1,3.9E1,1.1E1,8E0,1.5E1,1.8E1,3.3E1,5E0,1.6E1,6E0,6E0,3.3E1,6E0,5E0,1E1,5E0,1E1,8E0,1.8E1,1.5E1,5E0,5E0,6E0,1.2E1,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.0504069E-2,-1.478156E-1,4.4089317E-1,-2.8353155E-1,3.00689E-2,3.327948E-1,3.033851E-2,-3.285231E-1,-7.688565E-2,-8.705006E-3,8.966037E-3,2.2728853E-1,4.1111067E-1,-1.7762601E-2,-2.567682E-1,-7.6143723E-3,3.5913081E-3,-7.364898E-2,5.389568E-2,1.2415965E-2,4.2252042E-3,9.533035E-3,2.0111738E-2,-5.4597687E-3,-1.2038466E-2,3.0621147E-4,-9.644558E-2,1.1321693E-1,-1.7666428E-3,-1.4322372E-3,-5.167579E-3,2.0999264E-3,6.1524E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,29,31,-1,-1,-1,-1,-1],"loss_changes":[1.2377368E1,3.0262458E0,1.1425877E0,6.3485146E-1,3.7574002E-1,2.1228313E-1,0E0,2.2194767E-1,2.317685E-1,1.9098407E-1,0E0,1.06751084E-1,9.0792894E-2,0E0,4.826355E-2,0E0,0E0,4.297085E-2,1.3894011E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.125512E-2,2.0309985E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,14,14,17,17,18,18,26,26,27,27],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,30,32,-1,-1,-1,-1,-1],"split_conditions":[3.857587E6,1.66882E6,6.1123697E9,7.192E3,1.3957454E9,6.9937E4,3.033851E-2,5.4784E5,1.19E2,1.747361E6,8.966037E-3,6.5E1,4.234255E9,-1.7762601E-2,1.0926412E8,-7.6143723E-3,3.5913081E-3,6.487865E9,9.1309E4,1.2415965E-2,4.2252042E-3,9.533035E-3,2.0111738E-2,-5.4597687E-3,-1.2038466E-2,3.0621147E-4,1.656E3,2.754594E9,-1.7666428E-3,-1.4322372E-3,-5.167579E-3,2.0999264E-3,6.1524E-3],"split_indices":[6,9,27,3,16,8,0,9,17,6,0,17,23,0,11,0,0,10,22,0,0,0,0,0,0,0,20,11,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,1.24E2,4.9E1,7E1,5.4E1,3.6E1,1.3E1,5.7E1,1.3E1,4.5E1,9E0,1.7E1,1.9E1,2.5E1,3.2E1,8E0,5E0,2.2E1,2.3E1,1.1E1,6E0,5E0,1.4E1,5E0,2.7E1,5E0,1.7E1,1.4E1,9E0,5E0,1.2E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.4477609E-2,-1.881838E-1,3.8096845E-1,-2.8153497E-1,2.189748E-2,2.8209886E-1,7.395362E-1,-3.8920525E-1,-2.0388949E-1,-1.9096417E-2,8.487467E-3,3.170759E-1,-1.4436248E-3,2.096521E-2,4.1983094E-2,-1.8871266E-2,-8.9777345E-3,-2.2876714E-1,3.6584324E-4,-4.8860565E-2,1.9779883E-3,1.8490072E-1,4.5286247E-1,-6.698012E-3,-2.4418814E-1,1.2609281E-3,-7.496297E-2,4.5159073E-3,2.1082231E-1,1.3959312E-2,2.2712309E-2,-1.1263378E-2,-5.956019E-3,-5.4348353E-3,-1.4306597E-3,1.0863431E-2,4.0678014E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,-1,-1,-1,-1,23,-1,25,-1,27,29,-1,31,-1,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3143837E1,2.241857E0,2.0248194E0,5.810938E-1,2.5837398E-1,5.7270384E-1,2.8718233E-1,1.6450453E-1,2.553965E-1,5.973141E-2,0E0,7.418556E-1,0E0,0E0,0E0,0E0,0E0,1.24640465E-2,0E0,4.4784248E-2,0E0,3.0961275E-2,3.1144142E-2,0E0,1.5583277E-2,0E0,2.9366292E-2,0E0,5.7409286E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,17,17,19,19,21,21,22,22,24,24,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,-1,-1,-1,-1,24,-1,26,-1,28,30,-1,32,-1,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.269467E6,1.882904E6,7.37159E5,6.2808E5,1.3957454E9,3.0530524E7,1.0966112E4,1.1848904E8,8.663E3,1.272E3,8.487467E-3,6.476062E6,-1.4436248E-3,2.096521E-2,4.1983094E-2,-1.8871266E-2,-8.9777345E-3,1.4105509E8,3.6584324E-4,8.092E3,1.9779883E-3,1.10659E5,2.2246E4,-6.698012E-3,1.3024335E10,1.2609281E-3,2.5484189E8,4.5159073E-3,3.386103E3,1.3959312E-2,2.2712309E-2,-1.1263378E-2,-5.956019E-3,-5.4348353E-3,-1.4306597E-3,1.0863431E-2,4.0678014E-3],"split_indices":[6,2,12,2,16,2,5,16,3,0,0,6,0,0,0,0,0,11,0,20,0,12,3,0,10,0,16,0,5,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.13E2,6.2E1,7.8E1,3.5E1,5E1,1.2E1,3.1E1,4.7E1,2.9E1,6E0,4.5E1,5E0,7E0,5E0,2.4E1,7E0,4.2E1,5E0,2E1,9E0,2.4E1,2.1E1,9E0,3.3E1,5E0,1.5E1,7E0,1.7E1,9E0,1.2E1,2.8E1,5E0,6E0,9E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.0914478E-3,-1.9680676E-1,2.940038E-1,-3.551127E-1,-7.670754E-2,1.5639405E-1,4.944033E-1,-3.7760913E-1,-6.861258E-3,-1.0384814E-1,7.4765235E-3,1.9501147E-1,-4.143832E-3,1.5624657E-2,2.5567014E-2,-1.7686626E-2,-1.1458907E-2,-1.4281906E-1,-2.8644994E-2,1.1488077E-1,2.829866E-1,-8.765088E-3,-1.02055185E-1,1.4037285E-3,-3.7792334E-3,1.5616307E-1,6.941891E-4,1.4454875E-2,8.441253E-3,-1.7608577E-2,-7.461876E-3,4.1483045E-3,9.68163E-3,-3.2762615E-3,1.8672346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,23,25,27,-1,29,-1,-1,31,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[9.414118E0,1.8359773E0,1.7151785E0,1.2763834E-1,4.01453E-1,4.067489E-1,1.14385605E-1,1.11756325E-2,0E0,1.483072E-1,0E0,2.1422303E-1,0E0,0E0,0E0,0E0,0E0,5.989045E-2,6.972532E-2,7.9235494E-2,5.02789E-3,0E0,1.250027E-1,0E0,0E0,3.509745E-2,0E0,0E0,0E0,4.1376892E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,17,17,18,18,19,19,20,20,22,22,25,25,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,24,26,28,-1,30,-1,-1,32,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[1.804488E6,9.31995E5,2.92746E5,5.139292E8,1.5063692E9,1.4308215E0,5.590087E9,5.501E3,-6.861258E-3,2.313201E6,7.4765235E-3,4.2712787E8,-4.143832E-3,1.5624657E-2,2.5567014E-2,-1.7686626E-2,-1.1458907E-2,6.512479E7,1.8650388E10,1.48E2,1.26E2,-8.765088E-3,5.7437E4,1.4037285E-3,-3.7792334E-3,1.46635E5,6.941891E-4,1.4454875E-2,8.441253E-3,3.90767E5,-7.461876E-3,4.1483045E-3,9.68163E-3,-3.2762615E-3,1.8672346E-3],"split_indices":[6,9,12,16,16,65,27,3,0,2,0,27,0,0,0,0,0,16,10,19,19,0,12,0,0,12,0,0,0,6,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.61E2,9.7E1,6.4E1,4.1E1,5.6E1,3.9E1,2.5E1,3.6E1,5E0,5.1E1,5E0,3.4E1,5E0,1.2E1,1.3E1,2.7E1,9E0,3.3E1,1.8E1,1.9E1,1.5E1,1.2E1,2.1E1,9E0,9E0,1.3E1,6E0,8E0,7E0,1E1,1.1E1,8E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.7618868E-2,-1.9962358E-1,2.8721076E-1,-3.514768E-1,-9.0398744E-2,1.4104569E-1,5.199788E-1,-2.8309174E-2,-2.967541E-1,-1.18184835E-1,6.493508E-3,2.013499E-1,-1.979609E-2,1.4897345E-2,2.9139299E-2,-3.1860054E-1,-5.3903293E-3,-1.6607423E-1,-6.265388E-2,1.3817646E-1,1.2250689E-2,3.4708355E-3,-5.054906E-3,-1.4720178E-2,-8.662386E-3,-1.9694628E-1,-3.6940798E-3,-1.026342E-1,8.464283E-3,1.5654936E-1,3.421972E-3,-1.0651478E-2,-5.7592555E-3,-1.3096298E-1,-1.7200616E-3,2.5344973E-3,-1.8571909E-3,3.452117E-3,8.754581E-3,-7.053649E-3,-3.2030041E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,-1,-1,23,-1,25,27,29,-1,-1,-1,-1,-1,31,-1,33,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.88414E0,1.7263584E0,1.8999681E0,5.2292776E-1,4.3625963E-1,3.628651E-1,3.524289E-1,0E0,1.08269215E-1,1.4117742E-1,0E0,9.6524E-2,1.1410726E-1,0E0,0E0,9.2659E-3,0E0,5.875677E-2,8.048189E-2,6.3965023E-3,0E0,0E0,0E0,0E0,0E0,3.2073677E-2,0E0,2.7078822E-2,3.0306578E-2,2.5059491E-2,0E0,0E0,0E0,9.313852E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,15,15,17,17,18,18,19,19,25,25,27,27,28,28,29,29,33,33],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,-1,-1,24,-1,26,28,30,-1,-1,-1,-1,-1,32,-1,34,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.804488E6,9.31995E5,2.8512E5,8.8E1,1.5063692E9,1.62E2,1.5992037E11,-2.8309174E-2,1.2194085E7,1.882904E6,6.493508E-3,4.241883E6,3.386103E3,1.4897345E-2,2.9139299E-2,2.9672634E8,-5.3903293E-3,7.192E3,5.309569E6,2.9522E4,1.2250689E-2,3.4708355E-3,-5.054906E-3,-1.4720178E-2,-8.662386E-3,1.06E2,-3.6940798E-3,2.53E2,6.487865E9,2E1,3.421972E-3,-1.0651478E-2,-5.7592555E-3,6.775791E7,-1.7200616E-3,2.5344973E-3,-1.8571909E-3,3.452117E-3,8.754581E-3,-7.053649E-3,-3.2030041E-3],"split_indices":[6,9,12,0,16,17,13,0,18,2,0,6,5,0,0,16,0,3,18,7,0,0,0,0,0,17,0,4,10,17,0,0,0,16,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.62E2,1.05E2,5.7E1,4.3E1,6.2E1,3.6E1,2.1E1,5E0,3.8E1,5.6E1,6E0,2.6E1,1E1,1.1E1,1E1,3.3E1,5E0,2.9E1,2.7E1,1.6E1,1E1,5E0,5E0,2.7E1,6E0,2E1,9E0,1.7E1,1E1,1.1E1,5E0,1E1,1E1,1.1E1,6E0,5E0,5E0,5E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.6214986E-3,-1.6928522E-1,3.2172891E-1,-3.240682E-1,-6.2089626E-2,1.3909338E-1,4.3581384E-1,-2.5923736E-2,-2.7667037E-1,-9.200021E-2,8.51598E-2,7.0182376E-2,1.8670322E-1,4.8017532E-1,6.493111E-3,-3.074897E-1,-3.6248465E-3,-1.5161866E-1,-2.4837581E-2,6.0954434E-3,3.2257504E-4,4.148291E-3,1.467657E-3,4.9239118E-3,1.0209983E-2,3.7642032E-1,3.284379E-2,-1.4552267E-2,-9.13753E-3,-3.7350229E-3,-7.675257E-3,-5.3678386E-2,1.4160677E-3,1.0460329E-2,1.7921071E-2,-1.612701E-5,-7.269088E-2,-4.8704757E-3,-1.6883792E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,37,-1,-1],"loss_changes":[9.027462E0,1.8547807E0,1.1061945E0,4.2407942E-1,3.0732822E-1,6.2356114E-2,3.6152124E-1,0E0,2.1912146E-1,2.2446969E-1,4.9823046E-2,6.8008006E-3,1.8968344E-2,5.361085E-1,0E0,1.9228935E-2,0E0,3.2960415E-2,4.79986E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.0863705E-3,0E0,0E0,0E0,0E0,0E0,1.923031E-2,0E0,0E0,0E0,0E0,1.4203012E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,25,25,31,31,36,36],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,38,-1,-1],"split_conditions":[2.35198E6,1.15856E6,6.820745E6,4.2E1,1.2340036E9,2.0803306E9,2.63E2,-2.5923736E-2,9.273477E6,2.313201E6,4.5590743E9,2.16E2,7.04906E8,1.11464E5,6.493111E-3,6.112E3,-3.6248465E-3,6.487865E9,6.9504E4,6.0954434E-3,3.2257504E-4,4.148291E-3,1.467657E-3,4.9239118E-3,1.0209983E-2,9.352579E8,3.284379E-2,-1.4552267E-2,-9.13753E-3,-3.7350229E-3,-7.675257E-3,4.70769E-5,1.4160677E-3,1.0460329E-2,1.7921071E-2,-1.612701E-5,1E0,-4.8704757E-3,-1.6883792E-3],"split_indices":[6,2,6,4,16,21,17,0,18,2,10,4,16,3,0,3,0,10,8,0,0,0,0,0,0,27,0,0,0,0,0,62,0,0,0,0,41,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.67E2,1.12E2,5.5E1,4.5E1,6.7E1,2.2E1,3.3E1,5E0,4E1,5.6E1,1.1E1,1E1,1.2E1,2.8E1,5E0,3.4E1,6E0,2.9E1,2.7E1,6E0,5E0,5E0,5E0,6E0,6E0,2.2E1,6E0,2.5E1,9E0,9E0,2E1,1.8E1,9E0,6E0,1.6E1,5E0,1.3E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.157269E-3,-1.733987E-1,3.203396E-1,-3.4114224E-1,-9.375555E-2,2.0203651E-1,5.529584E-1,-2.494499E-2,-2.8554884E-1,-1.1160126E-1,6.066623E-3,1.397322E-1,1.36691695E-2,7.0294356E-1,3.5270435E-1,-1.35545265E-2,-5.6575797E-3,-1.6847801E-1,-3.3462986E-2,8.397952E-2,1.8326707E-1,1.9430943E-2,3.6818147E-2,2.08539E-2,7.3692733E-3,-1.9750047E-1,-9.6325934E-2,-7.517909E-2,2.5541944E-2,5.2325316E-3,4.4273256E-4,3.937824E-3,1.0041992E-2,-9.30331E-3,-4.4514793E-3,-1.9938644E-4,-7.183115E-3,-5.6148055E-3,-6.197328E-4,3.3320035E-3,-4.8715988E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,-1,21,23,-1,-1,25,27,29,31,-1,-1,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.960419E0,1.4776535E0,1.6495318E0,2.9786873E-1,3.3948994E-1,2.60916E-1,3.8280678E-1,0E0,8.059311E-2,3.1879246E-1,0E0,5.7424903E-2,0E0,2.312088E-2,1.7231274E-1,0E0,0E0,6.8924904E-2,8.068681E-2,3.8548715E-2,4.8925817E-2,0E0,0E0,0E0,0E0,2.1606207E-2,8.691928E-2,6.0116395E-2,2.6954439E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,13,13,14,14,17,17,18,18,19,19,20,20,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,-1,22,24,-1,-1,26,28,30,32,-1,-1,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.804488E6,8.10263E5,3.4492E4,4.5E1,1.5063692E9,6.820745E6,8.705043E10,-2.494499E-2,1.4659149E8,1.882904E6,6.066623E-3,1.3574205E9,1.36691695E-2,9.4378E4,3.70613E5,-1.35545265E-2,-5.6575797E-3,1.62E2,3.135717E6,3.274975E3,7.04906E8,1.9430943E-2,3.6818147E-2,2.08539E-2,7.3692733E-3,7.27488E5,7.952E3,7.7E1,6.0676E4,5.2325316E-3,4.4273256E-4,3.937824E-3,1.0041992E-2,-9.30331E-3,-4.4514793E-3,-1.9938644E-4,-7.183115E-3,-5.6148055E-3,-6.197328E-4,3.3320035E-3,-4.8715988E-4],"split_indices":[6,2,3,4,16,6,10,0,21,2,0,21,0,3,18,0,0,17,18,5,16,0,0,0,0,6,28,17,12,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.12E2,6.3E1,3.5E1,7.7E1,4.3E1,2E1,5E0,3E1,7.2E1,5E0,2.9E1,1.4E1,1E1,1E1,2.5E1,5E0,4.1E1,3.1E1,1.4E1,1.5E1,5E0,5E0,5E0,5E0,2.8E1,1.3E1,1.8E1,1.3E1,9E0,5E0,6E0,9E0,2.3E1,5E0,6E0,7E0,9E0,9E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.2513704E-2,-1.5313104E-1,2.934182E-1,-2.3828018E-1,1.3407502E-4,2.4125054E-1,3.0494038E-2,-1.4145856E-2,-1.9904089E-1,-4.253883E-2,7.920919E-3,1.5515381E-1,3.5557982E-1,-2.1773036E-1,-4.311098E-3,-1.347947E-2,-8.844399E-2,2.0900644E-1,-1.2931299E-3,8.958957E-3,1.6712269E-2,-2.3195153E-1,-3.988216E-3,-4.661445E-2,2.8014018E-3,-5.185916E-3,-1.4494447E-3,1.6166445E-2,1.5781878E-1,-1.0935233E-2,-7.138577E-3,-8.3831206E-2,1.28258E-3,1.0451647E-1,8.640761E-3,-4.978494E-3,-1.7296253E-3,2.9609413E-3,5.6692124E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,-1,-1,13,15,-1,17,19,21,-1,23,25,27,-1,-1,-1,29,-1,31,-1,-1,-1,-1,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1],"loss_changes":[8.280804E0,1.4616764E0,1.2266984E0,1.7794466E-1,3.2499567E-1,5.354326E-1,0E0,0E0,7.594061E-2,4.416296E-2,0E0,3.6442602E-1,2.406454E-2,5.6073427E-2,0E0,5.960564E-2,1.7862745E-2,1.7285109E-1,0E0,0E0,0E0,7.333994E-3,0E0,4.7718868E-2,0E0,0E0,0E0,0E0,2.853185E-2,0E0,0E0,1.0179773E-2,0E0,2.3555309E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,21,21,23,23,28,28,31,31,33,33],"right_children":[2,4,6,8,10,12,-1,-1,14,16,-1,18,20,22,-1,24,26,28,-1,-1,-1,30,-1,32,-1,-1,-1,-1,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1],"split_conditions":[4.503171E6,1.882904E6,8.704162E0,1.84146E5,9.969132E8,2.92746E5,3.0494038E-2,-1.4145856E-2,6.7842746E8,1.251E3,7.920919E-3,6.7595E4,2.00939E5,3.48E2,-4.311098E-3,1.154E3,3.601764E6,2.0589468E10,-1.2931299E-3,8.958957E-3,1.6712269E-2,1.244363E6,-3.988216E-3,5.180788E8,2.8014018E-3,-5.185916E-3,-1.4494447E-3,1.6166445E-2,4.615651E6,-1.0935233E-2,-7.138577E-3,2.526709E6,1.28258E-3,3.006846E6,8.640761E-3,-4.978494E-3,-1.7296253E-3,2.9609413E-3,5.6692124E-3],"split_indices":[2,2,59,6,16,12,0,0,16,0,0,20,28,17,0,0,2,10,0,0,0,2,0,27,0,0,0,0,6,0,0,2,0,18,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,1.11E2,6.5E1,7.1E1,4E1,5.9E1,6E0,2E1,5.1E1,3.3E1,7E0,3.5E1,2.4E1,4.2E1,9E0,2.1E1,1.2E1,2.7E1,8E0,5E0,1.9E1,3.7E1,5E0,1.5E1,6E0,7E0,5E0,5E0,2.2E1,2.7E1,1E1,1E1,5E0,1.1E1,1.1E1,5E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.4121323E-3,-1.546002E-1,2.4238844E-1,-1.8044248E-1,9.322786E-3,1.1805179E-1,4.1410398E-1,-2.708445E-1,-8.840335E-2,1.4627743E-1,-3.303271E-3,1.1314355E-2,2.3629416E-2,-2.9538766E-1,-5.0328323E-3,-1.4017324E-1,-2.5268741E-2,1.1231962E-1,1.3225287E-2,-1.3509765E-2,-6.527776E-3,-7.649307E-3,-9.852398E-2,-7.454523E-2,2.9741993E-2,8.068203E-2,1.0299313E-2,-5.941519E-3,-1.3413014E-3,-4.0147984E-3,-1.5696975E-3,-5.605061E-4,3.5599917E-3,6.7390017E-3,5.2268006E-2,7.929211E-2,-5.7210884E-4,1.1611158E-3,4.865004E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,-1,-1,-1,-1,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1],"loss_changes":[6.8577094E0,1.1327684E0,1.445652E0,8.746729E-1,0E0,2.4073702E-1,4.3130636E-1,1.7207718E-1,1.7622402E-1,1.6288662E-1,0E0,0E0,0E0,4.8535824E-2,0E0,2.6566088E-2,7.269625E-2,1.0614437E-1,0E0,0E0,0E0,0E0,3.6604285E-2,6.111145E-3,2.9735224E-2,4.8483312E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.814043E-2,2.2460684E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,22,22,23,23,24,24,25,25,34,34,35,35],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,-1,-1,-1,-1,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1],"split_conditions":[1.804488E6,1.5063692E9,2.8512E5,1.15856E6,9.322786E-3,2.25005E5,1.5992037E11,6.949662E8,1.973751E6,2.8100613E9,-3.303271E-3,1.1314355E-2,2.3629416E-2,5.922601E8,-5.0328323E-3,3.1633773E8,3.135717E6,3.8049E4,1.3225287E-2,-1.3509765E-2,-6.527776E-3,-7.649307E-3,3.8956667E9,1.466404E6,1.018E3,1E0,1.0299313E-2,-5.941519E-3,-1.3413014E-3,-4.0147984E-3,-1.5696975E-3,-5.605061E-4,3.5599917E-3,6.7390017E-3,6.800478E10,2.377024E6,-5.7210884E-4,1.1611158E-3,4.865004E-3],"split_indices":[6,16,12,2,0,28,13,26,2,1,0,0,0,16,0,16,18,7,0,0,0,0,10,6,0,44,0,0,0,0,0,0,0,0,10,6,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.83E2,1.14E2,6.9E1,1.07E2,7E0,4.1E1,2.8E1,5.3E1,5.4E1,3.6E1,5E0,1.4E1,1.4E1,4.5E1,8E0,2.9E1,2.5E1,3.1E1,5E0,4E1,5E0,1.4E1,1.5E1,1.3E1,1.2E1,2.6E1,5E0,9E0,6E0,8E0,5E0,7E0,5E0,6E0,2E1,1.4E1,6E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.665236E-2,-9.972738E-2,3.6790395E-1,-1.8921988E-1,1.3086193E-2,2.579512E-1,5.510851E-1,-2.2546081E-1,-4.8285272E-2,-1.7359463E-2,1.1981034E-1,3.006607E-1,5.2861404E-3,1.7573545E-2,3.2650586E-2,-2.8074807E-1,-1.487434E-1,2.2891602E-3,-5.7814815E-3,5.4509756E-3,-4.181878E-2,6.7953733E-3,2.9162818E-3,2.3864171E-1,1.8982317E-2,-1.3090166E-2,-8.108381E-3,-3.4120495E-3,-1.706283E-1,-1.0618661E-1,-1.3741785E-2,7.1860827E-3,1.1970398E-2,-8.229736E-3,-4.3807128E-3,-7.0906193E-3,-1.8675588E-3,1.6990183E-2,-5.8694586E-2,-1.6160069E-2,2.6383693E-3,-3.874822E-3,-6.0305034E-4,-2.2732043E-3,9.800858E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,29,-1,-1,31,-1,-1,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1],"loss_changes":[8.213547E0,1.3048561E0,9.2533445E-1,3.5686088E-1,1.905385E-1,1.6813898E-1,2.5441027E-1,2.0000601E-1,1.4261365E-1,1.652693E-1,1.4316067E-2,1.12683296E-1,0E0,0E0,0E0,2.523899E-3,2.6815891E-2,0E0,0E0,0E0,7.083103E-2,0E0,0E0,7.5547695E-3,0E0,0E0,0E0,0E0,5.0037503E-3,3.5856336E-2,4.1060384E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.6908327E-2,1.4973633E-2,1.64277E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,20,20,23,23,28,28,29,29,30,30,37,37,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,30,-1,-1,32,-1,-1,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1],"split_conditions":[3.857587E6,1.882904E6,1.5992037E11,7.118E3,1.0656E4,2.18E2,1.0966112E4,8.96079E5,3.6353846E8,5.32E2,1.8787456E7,4.3962E4,5.2861404E-3,1.7573545E-2,3.2650586E-2,1.1E2,3.82193E5,2.2891602E-3,-5.7814815E-3,5.4509756E-3,1.070527E6,6.7953733E-3,2.9162818E-3,1.4683288E9,1.8982317E-2,-1.3090166E-2,-8.108381E-3,-3.4120495E-3,9.861057E9,1.5048677E-2,2.8756E4,7.1860827E-3,1.1970398E-2,-8.229736E-3,-4.3807128E-3,-7.0906193E-3,-1.8675588E-3,2.8096E4,3.135717E6,1.546345E6,2.6383693E-3,-3.874822E-3,-6.0305034E-4,-2.2732043E-3,9.800858E-4],"split_indices":[6,2,13,3,3,4,5,2,11,20,18,3,0,0,0,17,6,0,0,0,6,0,0,27,0,0,0,0,10,66,20,0,0,0,0,0,0,7,18,6,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.8E2,1.28E2,5.2E1,7.1E1,5.7E1,3.4E1,1.8E1,5.6E1,1.5E1,4.5E1,1.2E1,2.5E1,9E0,1.2E1,6E0,3.1E1,2.5E1,7E0,8E0,6E0,3.9E1,6E0,6E0,1.9E1,6E0,2.4E1,7E0,7E0,1.8E1,1.1E1,2.8E1,8E0,1.1E1,1.3E1,5E0,5E0,6E0,1.7E1,1.1E1,1E1,7E0,6E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[9.872651E-3,-1.6467573E-1,2.2215594E-1,-3.212364E-1,-8.3973676E-2,1.6983865E-1,2.5624596E-2,-2.3643678E-2,-2.54647E-1,4.6856888E-3,-1.04220815E-1,9.892902E-2,2.7054182E-1,-1.1955869E-2,-6.037095E-3,-1.7266987E-1,-3.0201772E-2,7.133364E-2,8.789817E-3,3.121849E-1,2.2669614E-3,-1.378104E-1,-1.0756476E-2,4.095009E-3,-5.7660867E-2,3.0740669E-2,6.281221E-3,9.861558E-3,1.547804E-2,-6.8510063E-3,-2.3651437E-3,-9.150451E-2,-1.6778417E-2,3.6584781E-3,-4.355107E-3,-5.379106E-3,-1.8402304E-3,7.993268E-4,-2.6524758E-3,-3.3444066E-2,2.1692738E-3,-4.0403043E-4,-2.2721419E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,15,17,19,-1,-1,21,23,25,-1,27,-1,29,-1,-1,31,33,-1,-1,-1,-1,-1,35,37,-1,39,-1,-1,-1,-1,41,-1,-1,-1],"loss_changes":[6.817411E0,1.249136E0,1.4849482E0,3.6120272E-1,2.7461016E-1,5.0165653E-1,0E0,0E0,2.489698E-2,0E0,3.0891013E-1,1.12863034E-1,2.568848E-1,0E0,0E0,4.8331738E-2,1.1098217E-1,1.0458118E-1,0E0,1.1372089E-2,0E0,2.9934973E-2,0E0,0E0,3.443858E-2,4.6936415E-2,0E0,0E0,0E0,0E0,0E0,1.67277E-2,2.1182206E-2,0E0,2.6820926E-2,0E0,0E0,0E0,0E0,4.4703605E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,10,10,11,11,12,12,15,15,16,16,17,17,19,19,21,21,24,24,25,25,31,31,32,32,34,34,39,39],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,16,18,20,-1,-1,22,24,26,-1,28,-1,30,-1,-1,32,34,-1,-1,-1,-1,-1,36,38,-1,40,-1,-1,-1,-1,42,-1,-1,-1],"split_conditions":[8.4606E4,8.10263E5,1.338554E0,3.7681E4,6.5E1,6.820745E6,2.5624596E-2,-2.3643678E-2,8.84679E7,4.6856888E-3,6.88588E5,1.5522031E9,6.685129E7,-1.1955869E-2,-6.037095E-3,3.3173E4,3.394521E7,3.857587E6,8.789817E-3,7.3379085E8,2.2669614E-3,2.3216149E-1,-1.0756476E-2,4.095009E-3,2.616974E6,6.1677E4,6.281221E-3,9.861558E-3,1.547804E-2,-6.8510063E-3,-2.3651437E-3,2.3549E4,1.5016E4,3.6584781E-3,1.0656E4,-5.379106E-3,-1.8402304E-3,7.993268E-4,-2.6524758E-3,1.061E3,2.1692738E-3,-4.0403043E-4,-2.2721419E-3],"split_indices":[12,2,63,21,20,6,0,0,21,0,6,16,2,0,0,28,16,6,0,16,0,63,0,0,18,22,0,0,0,0,0,28,20,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.82E2,1E2,8.2E1,3.3E1,6.7E1,7.3E1,9E0,6E0,2.7E1,6E0,6.1E1,4.4E1,2.9E1,2.2E1,5E0,3.1E1,3E1,3.6E1,8E0,2.4E1,5E0,2.3E1,8E0,5E0,2.5E1,2.4E1,1.2E1,1E1,1.4E1,1.8E1,5E0,1.3E1,1.2E1,9E0,1.5E1,7E0,6E0,7E0,5E0,1E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.8293975E-3,-8.686024E-2,4.2004627E-1,-2.755925E-1,9.562007E-4,3.1871805E-1,3.0246234E-2,-2.3834627E-2,-2.3038998E-1,-9.1903225E-2,5.083655E-2,7.0505557E-3,1.5425059E-2,-2.4699494E-1,-3.9105206E-3,-1.287007E-1,4.95542E-3,6.834383E-2,-6.0470537E-3,-2.7058908E-1,-6.264444E-3,-9.7460926E-2,-9.005775E-3,3.0710727E-2,1.7461959E-1,-1.3170503E-2,-8.387408E-3,-1.3506342E-1,-2.3451544E-4,-3.379756E-2,8.882992E-2,4.348066E-3,8.714301E-3,-6.9558783E-3,-2.7910327E-3,-6.473975E-2,1.4102984E-3,9.874958E-5,1.2236935E-1,-4.3865503E-3,-1.4505198E-3,3.244142E-3,6.694154E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,-1,-1,19,-1,21,-1,23,-1,25,-1,27,-1,29,31,-1,-1,33,-1,35,37,-1,-1,-1,-1,39,-1,-1,41,-1,-1,-1,-1],"loss_changes":[6.2307615E0,2.4531856E0,5.811424E-1,4.4884038E-1,4.7712183E-1,6.83136E-2,0E0,0E0,7.606268E-2,2.8675675E-1,2.3161072E-1,0E0,0E0,5.1490784E-2,0E0,5.507177E-2,0E0,2.4260548E-1,0E0,1.28855705E-2,0E0,8.237995E-2,0E0,1.7921321E-1,1.0152608E-2,0E0,0E0,1.9192249E-2,0E0,4.913971E-2,7.230872E-2,0E0,0E0,0E0,0E0,1.4687456E-2,0E0,0E0,1.5409172E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,13,13,15,15,17,17,19,19,21,21,23,23,24,24,27,27,29,29,30,30,35,35,38,38],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,-1,-1,20,-1,22,-1,24,-1,26,-1,28,-1,30,32,-1,-1,34,-1,36,38,-1,-1,-1,-1,40,-1,-1,42,-1,-1,-1,-1],"split_conditions":[2.92746E5,1.038321E6,8.504903E3,3.3778E4,2.313201E6,1.51688E5,3.0246234E-2,-2.3834627E-2,1.2194085E7,1.4817759E9,1.3391612E0,7.0505557E-3,1.5425059E-2,1.0254491E8,-3.9105206E-3,2.0901188E7,4.95542E-3,3.585693E6,-6.0470537E-3,5.438E3,-6.264444E-3,8.694222E6,-9.005775E-3,6.9504E4,2.0712E4,-1.3170503E-2,-8.387408E-3,2.8E2,-2.3451544E-4,1.94717E6,1E0,4.348066E-3,8.714301E-3,-6.9558783E-3,-2.7910327E-3,4.6E1,1.4102984E-3,9.874958E-5,9.078E3,-4.3865503E-3,-1.4505198E-3,3.244142E-3,6.694154E-3],"split_indices":[12,2,5,6,2,20,0,0,18,16,65,0,0,21,0,18,0,6,0,3,0,18,0,8,7,0,0,4,0,6,29,0,0,0,0,17,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.47E2,2.8E1,4.6E1,1.01E2,2.2E1,6E0,5E0,4.1E1,3.5E1,6.6E1,5E0,1.7E1,3.6E1,5E0,3E1,5E0,6.1E1,5E0,2.8E1,8E0,2.3E1,7E0,4.6E1,1.5E1,1.8E1,1E1,1.6E1,7E0,2.2E1,2.4E1,5E0,1E1,1.1E1,5E0,1.5E1,7E0,7E0,1.7E1,6E0,9E0,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.090485E-3,-1.2669584E-1,2.6857686E-1,-2.756721E-1,-6.3114114E-2,1.7911255E-1,4.474958E-1,-2.3800708E-2,-2.1255724E-1,-9.0675086E-2,1.280337E-1,1.1464959E-1,2.1999675E-1,6.7152986E-3,5.4137784E-1,-1.0196454E-2,-3.9021226E-3,3.4079351E-3,-1.0671344E-1,2.994032E-3,7.2511425E-3,7.952657E-3,7.4651286E-2,5.488807E-3,1.0668372E-2,1.4721341E-2,3.3053428E-2,-1.4442001E-1,-2.6660569E-2,9.883252E-4,4.5534917E-3,-7.2098994E-3,-9.92773E-2,8.979067E-4,-5.322277E-2,-8.15611E-4,-1.3129956E-1,-4.2181937E-3,-1.3745013E-4,-3.5153166E-3,-6.811137E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,31,33,-1,-1,-1,35,-1,37,-1,39,-1,-1,-1,-1],"loss_changes":[6.1454344E0,1.100317E0,8.6425257E-1,4.7970772E-1,4.5413992E-1,8.283353E-2,4.4049025E-1,0E0,5.6905508E-2,2.0861667E-1,1.351276E-2,3.5035536E-2,2.4513125E-2,0E0,3.61516E-1,0E0,0E0,0E0,2.0140904E-1,0E0,0E0,0E0,1.7700478E-2,0E0,0E0,0E0,0E0,2.6724458E-2,2.9950758E-2,0E0,0E0,0E0,4.141833E-2,0E0,3.23092E-2,0E0,2.508074E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,14,14,18,18,22,22,27,27,28,28,32,32,34,34,36,36],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,32,34,-1,-1,-1,36,-1,38,-1,40,-1,-1,-1,-1],"split_conditions":[2.35198E6,8.10263E5,4.3962E4,3.7681E4,9.133E3,3.059E4,6.495E4,-2.3800708E-2,5.477601E8,6.5E1,6.12E2,6.784E3,2.0803306E9,6.7152986E-3,1.0966112E4,-1.0196454E-2,-3.9021226E-3,3.4079351E-3,1.037547E6,2.994032E-3,7.2511425E-3,7.952657E-3,2.5125E4,5.488807E-3,1.0668372E-2,1.4721341E-2,3.3053428E-2,6.88588E5,1.58985E5,9.883252E-4,4.5534917E-3,-7.2098994E-3,7.93065E5,8.979067E-4,7.7E1,-8.15611E-4,8.7E2,-4.2181937E-3,-1.3745013E-4,-3.5153166E-3,-6.811137E-3],"split_indices":[6,2,3,21,3,20,8,0,26,20,0,3,21,0,5,0,0,0,6,0,0,0,7,0,0,0,0,6,18,0,0,0,6,0,17,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.17E2,5.8E1,3.4E1,8.3E1,4E1,1.8E1,5E0,2.9E1,7.3E1,1E1,1.7E1,2.3E1,5E0,1.3E1,2.4E1,5E0,6E0,6.7E1,5E0,5E0,5E0,1.2E1,6E0,1.7E1,8E0,5E0,4.5E1,2.2E1,5E0,7E0,2.9E1,1.6E1,8E0,1.4E1,5E0,1.1E1,7E0,7E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.8397257E-2,-1.4820828E-1,2.0264643E-1,-3.0320892E-1,-9.018917E-2,9.288761E-2,3.2352492E-1,-2.1358533E-2,-9.7907735E-3,-1.1907515E-1,7.0723738E-3,-1.7612396E-3,1.0971023E-1,2.659473E-1,2.6845902E-2,-1.4519873E-1,-3.4507617E-2,1.3684943E-1,4.0067182E-4,3.120238E-1,1.3305437E-3,-1.7602064E-1,-5.8160335E-2,-3.930321E-4,-3.1103243E-3,1.025182E-1,7.4728923E-3,1.003465E-2,1.6998377E-2,-8.105656E-3,-4.4992585E-3,-4.9070963E-3,1.042669E-3,7.619561E-3,4.593869E-2,9.5091824E-4,2.6222114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,17,19,-1,21,23,25,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1],"loss_changes":[5.3230534E0,7.460519E-1,1.1242197E0,2.2797966E-1,4.8558402E-1,1.0977465E-1,5.246687E-1,0E0,0E0,1.23123586E-1,0E0,0E0,1.1333287E-1,3.8106322E-1,0E0,1.078009E-1,1.3418037E-2,2.1378458E-2,0E0,1.01212025E-1,0E0,3.470838E-3,6.05687E-2,0E0,0E0,6.9439724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7889758E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,12,12,13,13,15,15,16,16,17,17,19,19,21,21,22,22,25,25,34,34],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,18,20,-1,22,24,26,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1],"split_conditions":[2.565132E6,7.00693E5,6.820745E6,6.8E1,1.2340036E9,1.8448E4,1.0362809E1,-2.1358533E-2,-9.7907735E-3,4.9089632E8,7.0723738E-3,-1.7612396E-3,2.48E2,4.831919E7,2.6845902E-2,1.486843E6,1.6175099E8,1.337099E9,4.0067182E-4,3.4492E4,1.3305437E-3,6.142029E8,2.01E2,-3.930321E-4,-3.1103243E-3,1.2236E4,7.4728923E-3,1.003465E-2,1.6998377E-2,-8.105656E-3,-4.4992585E-3,-4.9070963E-3,1.042669E-3,7.619561E-3,1.9844815E9,9.5091824E-4,2.6222114E-3],"split_indices":[9,2,6,4,16,7,59,0,0,23,0,0,19,6,0,2,27,21,0,3,0,16,4,0,0,28,0,0,0,0,0,0,0,0,11,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,8.5E1,8.6E1,2.2E1,6.3E1,4.6E1,4E1,5E0,1.7E1,5.7E1,6E0,5E0,4.1E1,3.5E1,5E0,4.3E1,1.4E1,3.2E1,9E0,2.9E1,6E0,3.1E1,1.2E1,9E0,5E0,1.8E1,1.4E1,1.6E1,1.3E1,2.6E1,5E0,7E0,5E0,7E0,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.6401347E-2,-1.1989745E-1,2.105022E-1,-2.2896735E-1,-3.340006E-2,1.6781628E-1,2.411394E-2,-2.1760711E-2,-1.9276074E-1,-4.8922945E-2,5.712896E-2,1.0581152E-1,3.045891E-1,-2.1146616E-1,-7.51844E-4,3.8966786E-2,-6.657418E-2,-3.3973035E-4,4.9111526E-3,1.3917026E-1,-4.159412E-3,7.95072E-3,1.5000517E-2,-9.8356875E-3,-6.065253E-3,-2.557454E-3,5.675551E-3,-1.04605526E-1,-2.451954E-2,8.660073E-2,9.774738E-3,-1.1796564E-1,-1.666623E-3,2.1768551E-2,-5.4810133E-2,6.617713E-3,3.8617346E-2,-2.5602113E-3,-6.186181E-3,-1.1800667E-3,2.9219722E-3,-3.7527326E-3,-5.6276296E-4,3.4951996E-3,-1.0862542E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,23,-1,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,31,33,35,-1,37,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.346227E0,1.1905036E0,7.53942E-1,4.5052004E-1,1.0397449E-1,4.1724098E-1,0E0,0E0,1.6381371E-1,9.877865E-2,4.042715E-2,2.6819384E-1,1.3966799E-2,1.4979839E-2,0E0,1.05349004E-1,8.0603644E-2,0E0,0E0,1.2768155E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.9455254E-2,3.7647575E-2,6.278063E-2,0E0,2.2696793E-2,0E0,2.6063647E-2,1.9795518E-2,0E0,2.4059832E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,27,27,28,28,29,29,31,31,33,33,34,34,36,36],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,24,-1,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,32,34,36,-1,38,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.377024E6,1.145303E6,1.338554E0,8.8E1,9.188361E8,1.66368E5,2.411394E-2,-2.1760711E-2,8.548E3,6.02E2,1.5950401E9,9.319305E6,2.160306E9,6.094E3,-7.51844E-4,1.7744E4,2.341617E6,-3.3973035E-4,4.9111526E-3,4.615651E6,-4.159412E-3,7.95072E-3,1.5000517E-2,-9.8356875E-3,-6.065253E-3,-2.557454E-3,5.675551E-3,2.9409E4,2.975001E6,1.33E2,9.774738E-3,7.15E2,-1.666623E-3,2.565132E6,1.94717E6,6.617713E-3,2.065E3,-2.5602113E-3,-6.186181E-3,-1.1800667E-3,2.9219722E-3,-3.7527326E-3,-5.6276296E-4,3.4951996E-3,-1.0862542E-4],"split_indices":[6,2,63,0,16,20,0,0,3,22,25,6,27,3,0,7,2,0,0,6,0,0,0,0,0,0,0,7,2,4,0,0,0,9,6,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.83E2,1.26E2,5.7E1,5.5E1,7.1E1,5.2E1,5E0,5E0,5E1,6.1E1,1E1,3.7E1,1.5E1,4.5E1,5E0,1E1,5.1E1,5E0,5E0,3.2E1,5E0,5E0,1E1,3.6E1,9E0,5E0,5E0,2.6E1,2.5E1,2.1E1,1.1E1,2.1E1,5E0,1E1,1.5E1,8E0,1.3E1,7E0,1.4E1,5E0,5E0,8E0,7E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.463082E-2,-1.834152E-1,1.3806148E-1,-2.0059824E-2,-1.571226E-1,4.9344447E-2,3.1056476E-1,-2.0120987E-1,-8.785405E-2,6.9310606E-2,-5.612643E-3,2.4437885E-1,2.236977E-2,-2.1213122E-1,-4.378947E-3,-7.975206E-4,-1.0916321E-1,2.7597617E-2,1.5883291E-1,6.0926187E-3,1.2286617E-2,-2.2353946E-1,-5.0600795E-3,-7.3248535E-2,-7.666981E-3,-1.7677309E-2,1.00006595E-1,8.144162E-3,4.702953E-3,-1.03712445E-2,-6.080791E-3,-5.2831583E-3,-4.0519822E-2,-4.0932674E-2,1.8037237E-3,1.5004289E-3,5.7896827E-3,-1.1081456E-4,-3.1315554E-3,-4.192375E-3,3.1670523E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,25,27,-1,-1,29,-1,31,-1,33,35,-1,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0715384E0,4.7239542E-1,1.2549056E0,0E0,1.960932E-1,2.078643E-1,2.4900985E-1,2.4992347E-2,4.1004613E-2,1.8606296E-1,0E0,5.0174713E-2,0E0,1.4690638E-2,0E0,0E0,3.967318E-2,1.1978705E-1,2.5829375E-3,0E0,0E0,4.835725E-3,0E0,2.0123422E-2,0E0,3.3250507E-2,2.5431752E-2,0E0,0E0,0E0,0E0,0E0,1.2864443E-2,4.585121E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,16,16,17,17,18,18,21,21,23,23,25,25,26,26,32,32,33,33],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,26,28,-1,-1,30,-1,32,-1,34,36,-1,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.12954E6,2.711E3,2.92746E5,-2.0059824E-2,1.8077296E8,1.1377422E0,1.11464E5,2.2446582E8,7.441345E-4,3.857587E6,-5.612643E-3,2.09895E5,2.236977E-2,1.4206566E10,-4.378947E-3,-7.975206E-4,1.7997152E7,1.0902801E9,5.701E4,6.0926187E-3,1.2286617E-2,6.247E3,-5.0600795E-3,9.79389E5,-7.666981E-3,5.505271E9,9.0452E4,8.144162E-3,4.702953E-3,-1.03712445E-2,-6.080791E-3,-5.2831583E-3,1.898141E9,1E0,1.8037237E-3,1.5004289E-3,5.7896827E-3,-1.1081456E-4,-3.1315554E-3,-4.192375E-3,3.1670523E-4],"split_indices":[2,3,12,0,16,63,3,21,62,6,0,28,0,10,0,0,18,16,8,0,0,3,0,2,0,11,12,0,0,0,0,0,26,29,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.56E2,7.4E1,8.2E1,5E0,6.9E1,5.5E1,2.7E1,4.1E1,2.8E1,5E1,5E0,2.2E1,5E0,3.6E1,5E0,7E0,2.1E1,3.5E1,1.5E1,7E0,1.5E1,3.1E1,5E0,1.5E1,6E0,2.2E1,1.3E1,8E0,7E0,2.5E1,6E0,5E0,1E1,1.6E1,6E0,5E0,8E0,5E0,5E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.9935203E-3,-1.10007115E-1,2.2894405E-1,-2.0875188E-2,-8.947961E-2,1.0050016E-1,3.2043222E-1,-1.3205804E-1,9.18254E-3,1.6725633E-1,-4.564983E-3,2.5716305E-1,2.500217E-2,-1.6267143E-1,-3.976047E-2,5.3238613E-3,-2.337201E-2,4.3749036E-3,8.538865E-3,1.960838E-1,1.3469401E-2,-1.7973225E-1,-7.188498E-2,2.4806953E-3,-7.3498935E-2,-1.0685471E-1,2.8943641E-2,5.2490863E-3,1.0019165E-2,-1.926832E-1,-3.8964986E-3,-3.0446838E-4,-5.4477216E-3,-4.3370253E-3,-1.536851E-3,-2.9219405E-3,-5.6285043E-3,-4.1406677E-4,2.9866758E-3,-8.854577E-3,-4.9223467E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,-1,-1,27,-1,29,31,-1,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5189147E0,8.492372E-1,6.611111E-1,0E0,4.7591192E-1,3.734581E-1,3.9563084E-1,2.1675003E-1,1.3136832E-1,1.8430829E-2,0E0,3.0597448E-2,0E0,7.846439E-2,7.261636E-2,0E0,1.2538141E-1,0E0,0E0,1.2993336E-2,0E0,4.020369E-2,3.6920346E-2,0E0,1.2347646E-2,1.0694414E-3,2.7962405E-2,0E0,0E0,1.1303186E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,16,16,19,19,21,21,22,22,24,24,25,25,26,26,29,29],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,-1,-1,28,-1,30,32,-1,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.753E4,8.8E1,2.8512E5,-2.0875188E-2,7.118E3,2.19E2,1.0966112E4,7.27488E5,1.3419E4,2.228899E6,-4.564983E-3,4.3962E4,2.500217E-2,6.7842746E8,1E0,5.3238613E-3,1.973751E6,4.3749036E-3,8.538865E-3,2.21513E5,1.3469401E-2,8.9850285E8,5.409E3,2.4806953E-3,2.2038938E3,5.530292E8,1.00677864E8,5.2490863E-3,1.0019165E-2,2.5495654E3,-3.8964986E-3,-3.0446838E-4,-5.4477216E-3,-4.3370253E-3,-1.536851E-3,-2.9219405E-3,-5.6285043E-3,-4.1406677E-4,2.9866758E-3,-8.854577E-3,-4.9223467E-3],"split_indices":[12,0,12,0,3,4,5,6,28,18,0,3,0,16,44,0,2,0,0,28,0,26,3,0,5,11,16,0,0,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.17E2,5.8E1,5E0,1.12E2,2.5E1,3.3E1,7.8E1,3.4E1,1.9E1,6E0,2.8E1,5E0,5.8E1,2E1,7E0,2.7E1,7E0,1.2E1,1.5E1,1.3E1,4.8E1,1E1,5E0,1.5E1,1E1,1.7E1,6E0,9E0,4.1E1,7E0,5E0,5E0,8E0,7E0,5E0,5E0,9E0,8E0,3.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[6.2637795E-3,-9.012155E-2,2.137425E-1,-1.5103416E-1,1.9045036E-2,1.4468767E-1,3.2606468E-1,-1.6450235E-1,1.070786E-3,6.3917423E-3,-2.7168193E-3,1.6426343E-3,1.6628352E-1,3.9939243E-1,4.3918598E-3,-1.8764015E-1,-9.109167E-2,-7.191489E-2,3.060693E-2,1.22566596E-1,9.047916E-3,2.3154102E-2,1.1680764E-2,-2.043369E-1,-3.879038E-3,-2.0857763E-3,-4.9890424E-3,-6.511605E-4,-4.8227715E-3,-1.8399352E-3,6.124517E-2,2.6314282E-3,6.7174477E-3,-9.612187E-3,-5.7382225E-3,9.183741E-2,2.2704178E-6,4.995137E-3,2.5900314E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,-1,19,21,-1,23,25,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,37,-1,-1,-1],"loss_changes":[3.2789211E0,7.476342E-1,3.5615492E-1,1.7428482E-1,1.1406602E-1,7.234663E-2,2.7390456E-1,9.671116E-2,0E0,0E0,8.5214905E-2,0E0,2.4796963E-2,9.987664E-2,0E0,6.429434E-2,1.2631565E-2,2.4738591E-2,5.831421E-2,2.1947429E-2,0E0,0E0,0E0,1.9674897E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.369187E-2,0E0,0E0,0E0,0E0,2.0246282E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,23,23,30,30,35,35],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,-1,20,22,-1,24,26,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,38,-1,-1,-1],"split_conditions":[1.07218E5,1.882904E6,2.3378E4,9.188361E8,4.5623E4,2.5125E4,1.33865185E11,1.4331645E8,1.070786E-3,6.3917423E-3,6.3062E4,1.6426343E-3,8.2025E4,1.26E2,4.3918598E-3,8.330361E8,1.5654093E9,5.618E3,6.519E3,6.819532E6,9.047916E-3,2.3154102E-2,1.1680764E-2,6.502E3,-3.879038E-3,-2.0857763E-3,-4.9890424E-3,-6.511605E-4,-4.8227715E-3,-1.8399352E-3,3.2689406E10,2.6314282E-3,6.7174477E-3,-9.612187E-3,-5.7382225E-3,2.9E1,2.2704178E-6,4.995137E-3,2.5900314E-3],"split_indices":[12,2,3,16,12,7,26,21,0,0,12,0,8,19,0,11,10,3,3,2,0,0,0,3,0,0,0,0,0,0,10,0,0,0,0,19,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.62E2,1.11E2,5.1E1,7.1E1,4E1,3.3E1,1.8E1,6.6E1,5E0,5E0,3.5E1,6E0,2.7E1,1.3E1,5E0,4.9E1,1.7E1,1.1E1,2.4E1,1.5E1,1.2E1,5E0,8E0,4.1E1,8E0,7E0,1E1,5E0,6E0,7E0,1.7E1,6E0,9E0,3.2E1,9E0,1.1E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.7033186E-2,-1.0600612E-1,1.6120139E-1,-2.882348E-1,-7.032262E-2,1.636426E-2,1.20324895E-1,-2.0079521E-2,-8.468109E-3,-9.0275444E-2,8.352107E-2,4.9397912E-2,2.3718098E-1,-1.1592625E-1,-2.3662858E-2,6.7996336E-3,-6.4203143E-4,9.045792E-2,-6.5666116E-3,1.1775734E-2,6.2296605E-3,-1.4338465E-1,-6.074455E-2,3.0393537E-2,-7.216788E-2,6.101197E-2,7.4234586E-3,-8.203803E-3,-1.1692202E-1,8.135946E-4,-8.383149E-2,-7.6785736E-4,3.8980758E-3,-5.1507205E-3,-1.0054545E-3,1.343424E-2,4.344639E-3,-1.2652E-1,-2.5770513E-3,-1.6038005E-3,-4.5529297E-3,-1.1370215E-3,2.2120206E-3,-6.4065927E-3,-3.6434107E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,19,21,23,-1,-1,25,-1,-1,-1,27,29,31,33,35,-1,-1,37,-1,39,-1,-1,-1,-1,41,-1,43,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8081057E0,7.470324E-1,4.7032404E-1,2.1831799E-1,3.1450152E-1,0E0,4.109844E-1,0E0,0E0,1.4932823E-1,8.747671E-2,2.8439716E-1,2.0022154E-2,8.8263035E-2,7.036561E-2,0E0,0E0,5.527021E-2,0E0,0E0,0E0,2.7564108E-2,4.3716006E-2,3.8323898E-2,2.881299E-2,3.899102E-2,0E0,0E0,8.639932E-3,0E0,1.3526112E-2,0E0,0E0,0E0,0E0,1.7497186E-2,0E0,8.680105E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,12,12,13,13,14,14,17,17,21,21,22,22,23,23,24,24,25,25,28,28,30,30,35,35,37,37],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,20,22,24,-1,-1,26,-1,-1,-1,28,30,32,34,36,-1,-1,38,-1,40,-1,-1,-1,-1,42,-1,44,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.503171E6,3.671E3,1.9997997E10,1.4198665E-3,1.2340036E9,1.636426E-2,1.66368E5,-2.0079521E-2,-8.468109E-3,2.526709E6,5.48734E8,2.24E2,3.648E3,6.024596E8,2.6322297E10,6.7996336E-3,-6.4203143E-4,1.70835E5,-6.5666116E-3,1.1775734E-2,6.2296605E-3,1.6409624E3,9.67E3,1.0931615E9,4.315718E10,2.7603E4,7.4234586E-3,-8.203803E-3,2.56E2,8.135946E-4,8.7E1,-7.6785736E-4,3.8980758E-3,-5.1507205E-3,-1.0054545E-3,3.235961E6,4.344639E-3,6.73E2,-2.5770513E-3,-1.6038005E-3,-4.5529297E-3,-1.1370215E-3,2.2120206E-3,-6.4065927E-3,-3.6434107E-3],"split_indices":[2,3,10,63,16,0,20,0,0,2,11,17,19,26,10,0,0,28,0,0,0,5,28,11,10,7,0,0,17,0,17,0,0,0,0,6,0,20,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.17E2,5.8E1,1.8E1,9.9E1,8E0,5E1,5E0,1.3E1,8.8E1,1.1E1,3.2E1,1.8E1,6.3E1,2.5E1,6E0,5E0,2.7E1,5E0,1.2E1,6E0,4.1E1,2.2E1,1.2E1,1.3E1,2.1E1,6E0,1.3E1,2.8E1,5E0,1.7E1,7E0,5E0,6E0,7E0,1E1,1.1E1,2.3E1,5E0,6E0,1.1E1,5E0,5E0,1.4E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-8.618315E-3,-9.726051E-2,2.0961386E-1,-1.9039962E-1,-3.085839E-2,1.5602313E-1,1.7824287E-2,-1.91443E-2,-1.5388426E-1,-4.4084545E-2,3.5840536E-3,2.1032602E-1,8.207631E-2,-1.7395943E-1,-7.500928E-4,-9.2991546E-2,-5.6006624E-3,1.373525E-1,1.2109702E-2,1.3284466E-1,-2.1850758E-3,-4.8971106E-3,-8.328117E-3,-6.344734E-2,-6.6166823E-3,2.4151562E-3,-2.3694355E-2,3.2776757E-3,7.28364E-3,3.7803543E-3,7.276115E-3,-9.6320994E-2,-7.705529E-4,-4.0547192E-2,1.209864E-3,-2.5061085E-3,-5.058961E-3,-2.6370864E-3,-2.3457093E-2,3.815479E-5,-2.0912653E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,-1,13,15,-1,17,19,21,-1,23,25,27,-1,29,-1,-1,-1,31,-1,-1,33,-1,-1,-1,-1,35,-1,37,-1,-1,-1,-1,39,-1,-1],"loss_changes":[3.3476548E0,7.5483465E-1,4.6887016E-1,3.9475548E-1,1.1269655E-1,1.473738E-1,0E0,0E0,1.2324607E-1,1.2369986E-1,0E0,7.454705E-2,1.3411146E-1,2.0211697E-2,0E0,4.059258E-2,4.320732E-2,1.1363357E-2,0E0,7.477045E-3,0E0,0E0,0E0,2.991265E-2,0E0,0E0,2.7117942E-2,0E0,0E0,0E0,0E0,2.488166E-3,0E0,6.39366E-3,0E0,0E0,0E0,0E0,8.372569E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,19,19,23,23,26,26,31,31,33,33,38,38],"right_children":[2,4,6,8,10,12,-1,-1,14,16,-1,18,20,22,-1,24,26,28,-1,30,-1,-1,-1,32,-1,-1,34,-1,-1,-1,-1,36,-1,38,-1,-1,-1,-1,40,-1,-1],"split_conditions":[3.329577E6,1.13976E6,7.37159E5,1.39E2,1.4817759E9,1.86E2,1.7824287E-2,-1.91443E-2,8.691167E8,2.313201E6,3.5840536E-3,4.833515E10,1.41E2,4.1329212E7,-7.500928E-4,5.8802E4,3.2322E4,5.0597597E10,1.2109702E-2,6.021393E6,-2.1850758E-3,-4.8971106E-3,-8.328117E-3,4.9146E4,-6.6166823E-3,2.4151562E-3,7.808233E8,3.2776757E-3,7.28364E-3,3.7803543E-3,7.276115E-3,3.82193E5,-7.705529E-4,1.429952E6,1.209864E-3,-2.5061085E-3,-5.058961E-3,-2.6370864E-3,2.9745808E3,3.815479E-5,-2.0912653E-3],"split_indices":[6,2,12,0,16,4,0,0,25,2,0,13,17,11,0,12,18,10,0,6,0,0,0,12,0,0,16,0,0,0,0,6,0,6,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,1.22E2,4.9E1,5E1,7.2E1,4E1,9E0,5E0,4.5E1,6.5E1,7E0,2.2E1,1.8E1,3.9E1,6E0,2.8E1,3.7E1,1.2E1,1E1,1.3E1,5E0,1E1,2.9E1,2E1,8E0,8E0,2.9E1,5E0,7E0,7E0,6E0,1.1E1,9E0,2.2E1,7E0,5E0,6E0,9E0,1.3E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-7.0476937E-3,-1.316627E-1,1.2554355E-1,-1.8503614E-2,-1.0779791E-1,5.79348E-2,2.401126E-1,-1.3918608E-1,6.305679E-3,1.16131775E-1,2.8012415E-2,2.900932E-1,7.4803497E-6,-1.7443405E-1,-1.08883835E-1,4.416863E-3,-3.7974577E-2,3.5674232E-3,6.3119093E-3,1.043188E-2,4.8555606E-3,1.9915527E-2,2.2234288E-1,-4.940366E-3,-8.302091E-3,-5.786579E-3,-5.0449356E-2,-3.2741013E-3,7.7427627E-4,-2.1854421E-2,3.8494335E-3,5.9664724E-3,1.0970904E-2,-3.4268997E-3,-8.374445E-4,-3.789502E-3,7.934688E-3,1.3474927E-3,-1.1333904E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,-1,7,9,11,13,15,17,19,21,-1,23,25,-1,27,-1,-1,29,-1,-1,31,-1,-1,-1,33,-1,-1,35,-1,-1,-1,-1,-1,-1,37,-1,-1],"loss_changes":[2.6932325E0,5.402087E-1,5.980129E-1,0E0,2.8381646E-1,8.555786E-2,3.4773028E-1,4.7228694E-2,7.94741E-2,4.835129E-3,5.0054893E-2,1.6878033E-1,0E0,4.557848E-3,4.3153554E-2,0E0,2.8157271E-2,0E0,0E0,7.7247806E-2,0E0,0E0,1.2851E-2,0E0,0E0,0E0,9.022085E-3,0E0,0E0,4.3265916E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3243893E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,19,19,22,22,26,26,29,29,36,36],"right_children":[2,4,6,-1,8,10,12,14,16,18,20,22,-1,24,26,-1,28,-1,-1,30,-1,-1,32,-1,-1,-1,34,-1,-1,36,-1,-1,-1,-1,-1,-1,38,-1,-1],"split_conditions":[2.539959E6,2.58E3,6.820745E6,-1.8503614E-2,6.7842746E8,1.04E2,4.6665716E7,1.6807637E3,6.5E1,2.7143378E9,2.694E3,2.6459034E10,7.4803497E-6,9.67E3,1.7307954E8,4.416863E-3,1.8117093E-1,3.5674232E-3,6.3119093E-3,1.2535E4,4.8555606E-3,1.9915527E-2,2.160306E9,-4.940366E-3,-8.302091E-3,-5.786579E-3,4.927E3,-3.2741013E-3,7.7427627E-4,1.9535E4,3.8494335E-3,5.9664724E-3,1.0970904E-2,-3.4268997E-3,-8.374445E-4,-3.789502E-3,1.4521729E9,1.3474927E-3,-1.1333904E-3],"split_indices":[2,3,6,0,16,4,2,5,20,21,0,10,0,28,16,0,62,0,0,3,0,0,27,0,0,0,3,0,0,7,0,0,0,0,0,0,26,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.61E2,8.3E1,7.8E1,5E0,7.8E1,5E1,2.8E1,6.1E1,1.7E1,1.6E1,3.4E1,2.3E1,5E0,2.6E1,3.5E1,5E0,1.2E1,9E0,7E0,2.9E1,5E0,5E0,1.8E1,7E0,1.9E1,2.4E1,1.1E1,7E0,5E0,2.1E1,8E0,6E0,1.2E1,5E0,6E0,6E0,1.5E1,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.376726E-3,-1.0519142E-1,1.7757036E-1,-2.1614292E-1,-4.816873E-2,1.115236E-1,3.2117495E-1,-1.7948322E-2,-1.6832972E-1,-6.484277E-2,4.4127885E-2,8.1588976E-2,8.463587E-3,1.820344E-2,1.0060012E-2,-8.046906E-3,-3.3232302E-3,-8.114254E-2,9.388619E-4,-7.235893E-4,4.254674E-3,9.79231E-2,2.0064146E-4,-1.0780335E-1,-5.542619E-2,6.0246594E-2,5.523938E-3,-5.6456574E-3,-2.2363777E-3,-7.672827E-2,7.1572594E-4,3.77821E-3,1.5716485E-3,-7.373655E-4,-9.4370656E-2,-2.3777774E-3,-4.9259374E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,35,-1,-1],"loss_changes":[3.07161E0,6.285906E-1,5.5222774E-1,2.4790084E-1,1.0751045E-1,9.2692494E-2,6.9093466E-2,0E0,2.8830111E-2,8.338252E-2,3.724242E-2,4.2308167E-2,0E0,0E0,0E0,0E0,0E0,2.7620167E-2,0E0,0E0,0E0,2.2585064E-2,0E0,2.1402597E-2,4.256837E-2,5.813159E-3,0E0,0E0,0E0,2.0573758E-2,0E0,0E0,0E0,0E0,5.652502E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,17,17,21,21,23,23,24,24,25,25,29,29,34,34],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,36,-1,-1],"split_conditions":[4.460193E6,8.59522E5,3.4492E4,3.7681E4,1.2340036E9,1.6204663E7,1E0,-1.7948322E-2,6.112E3,3.601764E6,3.9692026E8,1.62E2,8.463587E-3,1.820344E-2,1.0060012E-2,-8.046906E-3,-3.3232302E-3,2.2505884E3,9.388619E-4,-7.235893E-4,4.254674E-3,1E0,2.0064146E-4,4.2349253E9,6.881934E-2,6.684E3,5.523938E-3,-5.6456574E-3,-2.2363777E-3,4.50039E5,7.1572594E-4,3.77821E-3,1.5716485E-3,-7.373655E-4,2.2622E4,-2.3777774E-3,-4.9259374E-3],"split_indices":[2,2,3,21,16,2,29,0,3,2,21,17,0,0,0,0,0,5,0,0,0,41,0,26,62,3,0,0,0,6,0,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.61E2,1E2,6.1E1,3.3E1,6.7E1,4.3E1,1.8E1,5E0,2.8E1,5.7E1,1E1,3.3E1,1E1,7E0,1.1E1,2.3E1,5E0,4.8E1,9E0,5E0,5E0,2.7E1,6E0,2.2E1,2.6E1,1.3E1,1.4E1,1.5E1,7E0,2E1,6E0,5E0,8E0,5E0,1.5E1,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.01080015E-2,-5.0854046E-2,2.602557E-1,-1.21108204E-1,2.1822425E-2,1.779362E-1,1.7574549E-2,-1.3134418E-1,1.2540767E-3,3.7338175E-2,-6.2459684E-3,4.4271685E-3,8.788769E-3,-1.6599728E-1,-9.771357E-2,3.4293253E-3,1.1090396E-1,-1.816144E-1,-3.8686707E-3,-1.2504266E-3,-1.08734734E-1,4.8537157E-3,-1.3411844E-2,2.4262911E-3,1.424929E-1,-8.493212E-3,-4.9337824E-3,-7.3119616E-3,-8.511068E-2,2.7469194E-3,-2.6771232E-2,3.506804E-3,7.336716E-3,-9.674834E-2,-1.2332057E-3,-5.6892302E-2,3.784724E-2,-1.1482848E-1,-2.774605E-3,-4.065352E-3,-3.359302E-2,2.8482298E-3,-2.184901E-4,-3.0033742E-3,-5.8918227E-3,9.3379714E-7,-4.6523288E-2,-2.98456E-3,-7.8655075E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,-1,-1,17,19,21,23,25,-1,-1,27,-1,29,-1,31,-1,-1,-1,33,-1,35,-1,-1,37,-1,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,47,-1,-1],"loss_changes":[2.0181928E0,8.14469E-1,2.0968246E-1,1.2800872E-1,2.074985E-1,1.0041118E-2,0E0,7.235074E-2,0E0,1.8172047E-1,0E0,0E0,0E0,2.739644E-2,2.809304E-2,9.405915E-2,3.226775E-2,3.8067102E-3,0E0,0E0,3.4352332E-2,0E0,4.7582477E-2,0E0,8.768082E-3,0E0,0E0,0E0,1.5131757E-2,0E0,7.806194E-2,0E0,0E0,5.89782E-3,0E0,1.9674927E-2,1.5595902E-2,2.7715862E-3,0E0,0E0,7.8245755E-3,0E0,0E0,0E0,0E0,0E0,6.866239E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,13,13,14,14,15,15,16,16,17,17,20,20,22,22,24,24,28,28,30,30,33,33,35,35,36,36,37,37,40,40,46,46],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,-1,-1,18,20,22,24,26,-1,-1,28,-1,30,-1,32,-1,-1,-1,34,-1,36,-1,-1,38,-1,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,48,-1,-1],"split_conditions":[2.8512E5,1.882904E6,2.4E2,8.663E3,1.5342771E0,7.089974E8,1.7574549E-2,8.96079E5,1.2540767E-3,1.1257E4,-6.2459684E-3,4.4271685E-3,8.788769E-3,2.8681312E8,1.2628019E8,5.73E2,1.337099E9,2.41701E5,-3.8686707E-3,-1.2504266E-3,1.7980087E3,4.8537157E-3,5.353857E9,2.4262911E-3,7.9E1,-8.493212E-3,-4.9337824E-3,-7.3119616E-3,1.1004841E9,2.7469194E-3,4.503171E6,3.506804E-3,7.336716E-3,1.3958365E8,-1.2332057E-3,1.070527E6,7.28E3,4.50612E5,-2.774605E-3,-4.065352E-3,1E0,2.8482298E-3,-2.184901E-4,-3.0033742E-3,-5.8918227E-3,9.3379714E-7,2.7196716E3,-2.98456E-3,-7.8655075E-4],"split_indices":[12,2,4,3,65,16,0,2,0,3,0,0,0,16,11,20,21,6,0,0,5,0,10,0,4,0,0,0,11,0,2,0,0,16,0,6,3,6,0,0,19,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.81E2,1.58E2,2.3E1,8E1,7.8E1,1.6E1,7E0,7.5E1,5E0,7.2E1,6E0,5E0,1.1E1,3.5E1,4E1,5E1,2.2E1,2.8E1,7E0,6E0,3.4E1,6E0,4.4E1,9E0,1.3E1,2.2E1,6E0,8E0,2.6E1,6E0,3.8E1,5E0,8E0,2.1E1,5E0,2.6E1,1.2E1,1.2E1,9E0,9E0,1.7E1,7E0,5E0,5E0,7E0,5E0,1.2E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.3073697E-2,-1.0555248E-1,1.2097118E-1,-1.6993085E-2,-8.799693E-2,9.294746E-2,1.6019931E-2,-1.19403526E-1,-1.4582195E-2,1.2126521E-1,-5.5276966E-3,-1.3824165E-1,-6.351231E-2,-5.1943336E-2,3.7827806E-3,8.5475385E-2,1.3222882E-2,-1.4544941E-1,-2.5114147E-3,-3.8917458E-3,-1.2633129E-3,-1.6364953E-2,-4.775947E-3,3.959526E-2,1.2955573E-1,-1.0820117E-1,-1.6069224E-1,3.8784082E-4,-1.8648804E-3,-8.31098E-4,5.6941073E-2,8.2918E-2,7.361292E-3,-5.933292E-3,-2.1455428E-3,-7.620066E-3,-3.1162181E-3,2.1769524E-2,4.285443E-3,4.3652826E-3,2.145401E-3,1.6690035E-3,7.2979805E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,17,19,21,-1,23,-1,25,-1,-1,-1,27,-1,29,31,33,35,-1,-1,-1,37,39,-1,-1,-1,-1,-1,41,-1,-1,-1,-1,-1],"loss_changes":[2.0205982E0,5.0378335E-1,3.9341313E-1,0E0,2.3574656E-1,3.6980557E-1,0E0,6.702781E-2,1.2555185E-1,3.057329E-1,0E0,2.2460222E-2,1.5154175E-2,4.7289528E-2,0E0,8.3856344E-2,0E0,9.865761E-3,0E0,0E0,0E0,1.1030076E-2,0E0,2.4477549E-2,2.6132911E-2,2.1159068E-2,2.7231574E-2,0E0,0E0,0E0,2.3973178E-2,2.010636E-3,0E0,0E0,0E0,0E0,0E0,3.5769497E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,21,21,23,23,24,24,25,25,26,26,30,30,31,31,37,37],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,18,20,22,-1,24,-1,26,-1,-1,-1,28,-1,30,32,34,36,-1,-1,-1,38,40,-1,-1,-1,-1,-1,42,-1,-1,-1,-1,-1],"split_conditions":[1.789754E6,2.618E3,1.2856582E0,-1.6993085E-2,1.67E2,1.5357575E0,1.6019931E-2,6.88588E5,2.695687E9,4.3962E4,-5.5276966E-3,8.937147E8,2.05E2,2.8956E4,3.7827806E-3,1.2538E4,1.3222882E-2,1.9012777E-3,-2.5114147E-3,-3.8917458E-3,-1.2633129E-3,2.5384E4,-4.775947E-3,2.9673362E9,7.337122E-1,5.6213E4,6.502E3,3.8784082E-4,-1.8648804E-3,-8.31098E-4,3E0,2.5997338E9,7.361292E-3,-5.933292E-3,-2.1455428E-3,-7.620066E-3,-3.1162181E-3,2.003E3,4.285443E-3,4.3652826E-3,2.145401E-3,1.6690035E-3,7.2979805E-5],"split_indices":[6,3,63,0,17,65,0,6,25,3,0,26,4,28,0,3,0,65,0,0,0,7,0,11,63,27,3,0,0,0,19,21,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,1.07E2,6.1E1,5E0,1.02E2,5.6E1,5E0,7.1E1,3.1E1,5E1,6E0,5.2E1,1.9E1,2.3E1,8E0,4.3E1,7E0,4.7E1,5E0,1E1,9E0,1.5E1,8E0,2.2E1,2.1E1,1.6E1,3.1E1,8E0,7E0,5E0,1.7E1,1.1E1,1E1,1E1,6E0,2.6E1,5E0,1E1,7E0,6E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.3637749E-3,-5.3940143E-2,2.6506862E-1,-1.3224448E-1,2.4047775E-2,1.8848036E-1,2.3569698E-2,-1.6394824E-2,-1.09814614E-1,4.108634E-2,-5.588797E-3,4.084358E-3,2.201884E-1,-1.3262829E-1,-3.5272747E-2,1.895681E-2,6.8987333E-3,5.975481E-3,1.0930121E-2,-1.454594E-1,-6.92468E-2,3.4397373E-3,-8.521222E-2,2.9906087E-2,-3.691265E-3,-1.6154934E-1,-8.909042E-2,-3.9745774E-3,-1.5665061E-3,-1.711012E-3,-5.7770284E-3,1.3631905E-2,7.659248E-2,-3.3441486E-3,-1.7397925E-1,-4.780651E-3,-2.5683378E-3,4.6310844E-3,-7.893908E-3,1.2242135E-3,4.7791093E-3,-4.877568E-3,-8.266756E-3,-2.4850799E-2,2.271397E-2,-2.7298927E-3,-1.0905497E-2,1.8746619E-3,-3.5152695E-4,5.756592E-3,-1.6547048E-3,8.349641E-4,-2.8495636E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,23,-1,-1,-1,25,27,-1,29,31,-1,33,35,-1,-1,-1,-1,37,39,-1,41,-1,-1,-1,43,-1,-1,-1,-1,45,47,-1,49,-1,-1,51,-1,-1,-1],"loss_changes":[2.5497546E0,9.314125E-1,5.299332E-1,3.7363255E-1,2.0480278E-1,5.4625154E-2,0E0,0E0,1.1606264E-1,1.7820026E-1,0E0,0E0,1.1038303E-2,3.38431E-2,1.0926403E-1,7.0542485E-2,0E0,0E0,0E0,2.5514364E-2,4.7361962E-3,0E0,2.3011439E-2,4.0743962E-2,0E0,2.150774E-2,9.737015E-4,0E0,0E0,0E0,0E0,8.443953E-2,1.9289963E-2,0E0,5.7046413E-3,0E0,0E0,0E0,1.8729966E-2,0E0,0E0,0E0,0E0,1.1407999E-2,8.440211E-3,0E0,8.317709E-3,0E0,0E0,2.0964378E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,12,12,13,13,14,14,15,15,19,19,20,20,22,22,23,23,25,25,26,26,31,31,32,32,34,34,38,38,43,43,44,44,46,46,49,49],"right_children":[2,4,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,24,-1,-1,-1,26,28,-1,30,32,-1,34,36,-1,-1,-1,-1,38,40,-1,42,-1,-1,-1,44,-1,-1,-1,-1,46,48,-1,50,-1,-1,52,-1,-1,-1],"split_conditions":[2.92746E5,1.882904E6,9.751728E3,2.58E3,1.5342771E0,5.16177E5,2.3569698E-2,-1.6394824E-2,2.56E2,4.200155E6,-5.588797E-3,4.084358E-3,2.572168E7,2.2446582E8,8.194472E7,1E0,6.8987333E-3,5.975481E-3,1.0930121E-2,5.13112E9,2.3412E4,3.4397373E-3,5.567E3,1.1257E4,-3.691265E-3,1.93E3,5.90433E5,-3.9745774E-3,-1.5665061E-3,-1.711012E-3,-5.7770284E-3,5.32E2,1.337099E9,-3.3441486E-3,3.9895258E8,-4.780651E-3,-2.5683378E-3,4.6310844E-3,2.856E4,1.2242135E-3,4.7791093E-3,-4.877568E-3,-8.266756E-3,2.400113E6,7.7E1,-2.7298927E-3,8.172E3,1.8746619E-3,-3.5152695E-4,1E0,-1.6547048E-3,8.349641E-4,-2.8495636E-4],"split_indices":[12,2,5,3,65,22,0,0,17,6,0,0,2,21,11,32,0,0,0,10,7,0,20,3,0,28,21,0,0,0,0,20,21,0,10,0,0,0,7,0,0,0,0,2,19,0,3,0,0,19,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.8E2,1.51E2,2.9E1,7.5E1,7.6E1,2.4E1,5E0,5E0,7E1,6.9E1,7E0,7E0,1.7E1,5.3E1,1.7E1,5.9E1,1E1,6E0,1.1E1,4.3E1,1E1,5E0,1.2E1,5.4E1,5E0,3.2E1,1.1E1,5E0,5E0,7E0,5E0,4.1E1,1.3E1,5E0,2.7E1,5E0,6E0,7E0,3.4E1,6E0,7E0,7E0,2E1,2.2E1,1.2E1,5E0,1.7E1,7E0,5E0,1.1E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[7.5267684E-3,-6.4431824E-2,1.6014892E-1,-1.19750604E-1,8.802661E-3,1.23967014E-1,1.5986437E-2,-1.6026312E-1,-9.152761E-2,-9.429034E-3,5.672264E-3,5.641974E-2,2.0737174E-1,-7.828059E-3,-3.4965063E-3,-1.05692826E-1,-4.071127E-2,-5.7160836E-2,3.132207E-2,9.990206E-2,-5.5393535E-3,4.531172E-3,1.0462927E-2,-7.859673E-2,-6.16876E-3,-3.886747E-3,6.290575E-4,-4.727783E-3,-3.5742674E-2,1.5000799E-2,3.6325662E-3,5.8053706E-2,9.417762E-3,-8.565825E-4,-4.402969E-3,-2.603578E-3,-6.0649905E-5,2.1921752E-3,-6.1989147E-3,8.218663E-2,2.1570419E-4,-1.808536E-3,7.8481407E-4,2.0781192E-3,4.583812E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,23,25,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,41,43,-1,-1,-1,-1,-1],"loss_changes":[1.9542167E0,4.9125445E-1,3.7968695E-1,6.422496E-2,1.1829528E-1,2.6998818E-1,0E0,2.6938796E-2,2.6649565E-2,9.3211204E-2,0E0,2.438589E-1,4.8805952E-2,0E0,0E0,2.0760268E-2,3.0593963E-2,2.0234019E-2,2.0777207E-2,1.0384835E-1,0E0,0E0,0E0,2.5612116E-2,0E0,0E0,0E0,0E0,1.3879586E-2,1.599303E-2,0E0,2.3946904E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.26733165E-2,5.2627996E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,17,17,18,18,19,19,23,23,28,28,29,29,31,31,38,38,39,39],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,24,26,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,42,44,-1,-1,-1,-1,-1],"split_conditions":[5.166426E6,6.588E3,1.7782382E0,7.00693E5,1.5522031E9,1.66368E5,1.5986437E-2,9.6E1,6.88588E5,2.5159E4,5.672264E-3,2.8413632E7,3.2E1,-7.828059E-3,-3.4965063E-3,5.53E2,1.6E2,3.5E1,1.2538E4,4.833515E10,-5.5393535E-3,4.531172E-3,1.0462927E-2,1.47E2,-6.16876E-3,-3.886747E-3,6.290575E-4,-4.727783E-3,1.6029493E8,1.37E2,3.6325662E-3,7.5E1,9.417762E-3,-8.565825E-4,-4.402969E-3,-2.603578E-3,-6.0649905E-5,2.1921752E-3,9.009353E9,1.1725978E11,2.1570419E-4,-1.808536E-3,7.8481407E-4,2.0781192E-3,4.583812E-3],"split_indices":[2,3,66,2,16,20,0,17,6,7,0,2,4,0,0,0,4,17,3,13,0,0,0,4,0,0,0,0,16,4,0,17,0,0,0,0,0,0,10,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,1.2E2,5.6E1,6.8E1,5.2E1,4.9E1,7E0,2.6E1,4.2E1,4.6E1,6E0,2.8E1,2.1E1,2E1,6E0,3.2E1,1E1,2.1E1,2.5E1,2.3E1,5E0,6E0,1.5E1,2E1,1.2E1,5E0,5E0,5E0,1.6E1,2E1,5E0,1.8E1,5E0,6E0,1.4E1,9E0,7E0,7E0,1.3E1,1.2E1,6E0,5E0,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[7.795661E-3,-4.0498942E-2,2.393499E-1,-8.811625E-2,3.3726014E-2,1.623843E-1,2.0472113E-2,-1.3360289E-1,-4.9186014E-2,1.9686686E-2,5.7262373E-3,8.061795E-3,2.8212843E-3,-1.5685871E-1,-2.1015792E-3,3.8088665E-3,-7.0255175E-2,4.9652024E-3,2.9294828E-3,-1.1193309E-1,-7.827708E-3,-8.3448686E-2,1.0917379E-3,-2.024278E-2,3.389207E-2,-6.3988105E-3,-2.0535777E-3,-9.5198855E-2,-2.1450897E-4,4.0302253E-3,-2.4333852E-3,1.8784092E-3,3.0227276E-3,-4.6037906E-3,-2.6628547E-3,1.5257228E-3,-1.3918695E-3,1.5710185E-3,-1.4207093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,-1,19,-1,-1,21,23,-1,25,-1,27,-1,29,31,-1,-1,33,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9443212E0,5.119244E-1,4.5418036E-1,1.4925951E-1,7.541559E-2,4.042232E-2,0E0,7.193881E-2,1.470018E-1,3.5277776E-2,0E0,0E0,0E0,1.1237502E-2,0E0,0E0,5.6134894E-2,2.9853586E-2,0E0,2.23466E-2,0E0,3.4598082E-2,0E0,1.9186616E-2,2.13792E-2,0E0,0E0,5.1625073E-3,0E0,1.6666481E-2,0E0,1.4091543E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,13,13,16,16,17,17,19,19,21,21,23,23,24,24,27,27,29,29,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,-1,20,-1,-1,22,24,-1,26,-1,28,-1,30,32,-1,-1,34,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.820745E6,1.070527E6,1.11464E5,5.131E3,1.5522031E9,5.2022126E11,2.0472113E-2,4.0876403E8,2.81E3,1.4683288E9,5.7262373E-3,8.061795E-3,2.8212843E-3,5.261E3,-2.1015792E-3,3.8088665E-3,7.563164E8,4.460193E6,2.9294828E-3,2.6E1,-7.827708E-3,9.133E3,1.0917379E-3,1.5351937E9,1E0,-6.3988105E-3,-2.0535777E-3,8.330361E8,-2.1450897E-4,2.6322297E10,-2.4333852E-3,1.9802E4,3.0227276E-3,-4.6037906E-3,-2.6628547E-3,1.5257228E-3,-1.3918695E-3,1.5710185E-3,-1.4207093E-3],"split_indices":[6,6,3,3,16,10,0,26,28,27,0,0,0,28,0,0,21,2,0,17,0,3,0,21,29,0,0,11,0,10,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.72E2,1.43E2,2.9E1,8.7E1,5.6E1,2.3E1,6E0,3.9E1,4.8E1,5E1,6E0,1.8E1,5E0,3E1,9E0,6E0,4.2E1,3.9E1,1.1E1,1.2E1,1.8E1,3.7E1,5E0,2.1E1,1.8E1,7E0,5E0,3.2E1,5E0,1.3E1,8E0,1E1,8E0,2.3E1,9E0,7E0,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.966306E-3,-9.859927E-2,8.266345E-2,-1.1761108E-1,-6.003287E-3,6.0798176E-2,1.7958183E-2,-1.5065011E-1,-8.301044E-2,4.5169275E-3,-3.4188796E-3,2.747582E-2,1.4246936E-1,-7.724268E-3,-1.06047414E-1,-1.0135679E-1,-1.3390075E-3,-5.159284E-3,1.2568462E-1,1.0272388E-1,1.8359087E-1,-1.7683639E-3,-6.021202E-3,-5.4096906E-3,-6.8442985E-2,2.0928737E-2,-7.675652E-3,7.196208E-2,8.386792E-3,6.244663E-3,1.941916E-3,1.0071776E-2,5.1039234E-3,-1.7703157E-3,-3.909283E-3,1.7122003E-4,4.637318E-3,4.6305014E-3,1.1278578E-3,5.1718682E-2,-2.2745105E-2,-5.858738E-4,4.3719397E-3,-3.74479E-2,1.5257213E-3,-2.4462435E-3,-1.727881E-2,3.564655E-4,-1.5983992E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,19,-1,21,23,-1,25,27,29,31,-1,-1,-1,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,-1,-1,-1,41,43,-1,-1,45,-1,-1,47,-1,-1],"loss_changes":[1.412612E0,1.3308573E-1,6.866798E-1,5.9514225E-2,1.1897473E-1,2.519043E-1,0E0,1.7627E-2,2.857405E-2,0E0,0E0,2.1859214E-1,2.5269508E-2,0E0,2.3682207E-2,9.3578845E-3,0E0,2.352462E-1,4.687491E-2,3.1227812E-2,1.0741055E-2,0E0,0E0,0E0,3.4349188E-3,8.1446804E-2,0E0,1.45729445E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.6064924E-2,0E0,0E0,0E0,3.9215866E-2,2.4214946E-2,0E0,0E0,7.284023E-3,0E0,0E0,6.2052608E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,24,24,25,25,27,27,35,35,39,39,40,40,43,43,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,20,-1,22,24,-1,26,28,30,32,-1,-1,-1,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,-1,-1,-1,42,44,-1,-1,46,-1,-1,48,-1,-1],"split_conditions":[1.882904E6,6.7842746E8,1.7903596E0,4.729E3,4.7182283E-4,1.174667E9,1.7958183E-2,8.3E1,1E2,4.5169275E-3,-3.4188796E-3,3.541E3,4.0659575E10,-7.724268E-3,1.4899E4,1E0,-1.3390075E-3,7.915464E8,1.8195216E7,1.2159447E9,1E0,-1.7683639E-3,-6.021202E-3,-5.4096906E-3,1.1062383E8,1.2535E4,-7.675652E-3,7.1999734E9,8.386792E-3,6.244663E-3,1.941916E-3,1.0071776E-2,5.1039234E-3,-1.7703157E-3,-3.909283E-3,1.841E3,4.637318E-3,4.6305014E-3,1.1278578E-3,5.0771E4,2.35198E6,-5.858738E-4,4.3719397E-3,1.429952E6,1.5257213E-3,-2.4462435E-3,1.382E3,3.564655E-4,-1.5983992E-3],"split_indices":[2,16,66,3,62,16,0,17,17,0,0,0,13,0,28,41,0,16,2,11,29,0,0,0,21,3,0,11,0,0,0,0,0,0,0,22,0,0,0,8,6,0,0,6,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,7.5E1,9.8E1,6.2E1,1.3E1,9.3E1,5E0,3E1,3.2E1,5E0,8E0,6.7E1,2.6E1,1.7E1,1.3E1,2.3E1,9E0,5.1E1,1.6E1,1.5E1,1.1E1,5E0,8E0,1.2E1,1.1E1,4.5E1,6E0,1E1,6E0,8E0,7E0,5E0,6E0,6E0,5E0,3.7E1,8E0,5E0,5E0,1.1E1,2.6E1,5E0,6E0,2.1E1,5E0,1E1,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.6755497E-3,-6.6049896E-2,1.9002251E-1,-1.3992871E-1,-1.0743794E-2,1.38372E-1,1.934715E-2,-1.593063E-2,-1.1070738E-1,-2.9605733E-2,5.6459744E-2,1.857106E-1,-2.7218882E-3,-1.2384559E-1,3.6969697E-4,-4.989276E-2,1.6797476E-2,1.0338895E-2,6.079644E-3,1.1172953E-1,2.7955133E-1,-5.783739E-3,-1.9197864E-3,-6.317228E-2,1.1142818E-3,4.5465004E-2,-8.7760866E-4,-4.8334998E-4,1.1878902E-3,3.3366408E-3,5.928994E-3,1.5818134E-2,7.963925E-3,-9.486895E-2,-2.2166068E-2,3.273189E-3,3.648911E-4,-4.6005263E-3,-2.6296263E-3,1.2135371E-3,-2.6155466E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,-1,21,-1,23,25,27,-1,29,31,-1,-1,33,-1,35,-1,-1,-1,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1],"loss_changes":[2.159787E0,5.33756E-1,5.141889E-1,3.278799E-1,9.8008305E-2,3.72674E-1,0E0,0E0,8.090383E-2,5.7321068E-2,6.350447E-2,1.8398142E-1,0E0,2.6008427E-2,0E0,4.367874E-2,2.1015406E-2,4.638036E-3,0E0,5.7787597E-3,3.8455606E-2,0E0,0E0,4.4688493E-2,0E0,1.1424813E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.4388425E-4,3.374178E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,19,19,20,20,23,23,25,25,33,33,34,34],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,-1,22,-1,24,26,28,-1,30,32,-1,-1,34,-1,36,-1,-1,-1,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1],"split_conditions":[3.329577E6,1.15856E6,8.704162E0,4E1,9.188361E8,3.5713768E0,1.934715E-2,-1.593063E-2,8.9850285E8,1.747361E6,2.0379508E-1,1.5245079E7,-2.7218882E-3,1.2194085E7,3.6969697E-4,3.51E2,6.1E1,3.9692026E8,6.079644E-3,7.337122E-1,1.5805948E11,-5.783739E-3,-1.9197864E-3,3.1633773E8,1.1142818E-3,1.25E2,-8.7760866E-4,-4.8334998E-4,1.1878902E-3,3.3366408E-3,5.928994E-3,1.5818134E-2,7.963925E-3,6.1377E4,6.0676E4,3.273189E-3,3.648911E-4,-4.6005263E-3,-2.6296263E-3,1.2135371E-3,-2.6155466E-3],"split_indices":[6,2,59,4,16,59,0,0,25,6,62,6,0,18,0,4,19,21,0,63,10,0,0,16,0,4,0,0,0,0,0,0,0,12,12,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,1.3E2,4.3E1,5.5E1,7.5E1,3.7E1,6E0,5E0,5E1,5.9E1,1.6E1,3E1,7E0,4.5E1,5E0,4.1E1,1.8E1,1.1E1,5E0,1.8E1,1.2E1,4E1,5E0,3.5E1,6E0,1E1,8E0,5E0,6E0,9E0,9E0,5E0,7E0,1.9E1,1.6E1,5E0,5E0,1.3E1,6E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-3.1289675E-3,-7.3166646E-2,1.1652198E-1,-1.4740981E-2,-5.750097E-2,5.118995E-2,1.8652517E-1,-7.029384E-2,3.8491404E-3,8.284322E-2,-5.045936E-3,9.983571E-2,1.0301628E-2,-1.113761E-1,-4.5702484E-2,5.548398E-2,7.988821E-3,2.1562346E-3,5.4637957E-3,-1.2820795E-1,-2.035484E-3,9.212127E-3,-6.0164526E-2,6.9119275E-2,7.799296E-4,-3.5117813E-3,-6.3982573E-3,2.2336564E-3,-1.7485883E-3,-7.07888E-2,-2.0621935E-2,1.5626627E-3,4.2973952E-3,-5.196705E-2,-1.13974415E-1,5.662399E-4,-2.4607813E-3,-6.388559E-2,1.9066212E-4,-6.0536056E-3,-3.0665537E-3,-7.6002665E-2,-6.4891047E-4,-4.0403083E-3,-2.0649969E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,-1,7,9,11,13,-1,15,-1,17,-1,19,21,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,-1,33,35,-1,-1,37,39,-1,-1,41,-1,-1,-1,43,-1,-1,-1],"loss_changes":[1.4750385E0,4.4063085E-1,2.8772599E-1,0E0,2.0375994E-1,1.9436696E-1,1.0945284E-1,9.391618E-2,0E0,7.2078004E-2,0E0,1.0433704E-2,0E0,3.325045E-2,5.058743E-2,1.1587173E-2,0E0,0E0,0E0,1.2783468E-2,0E0,3.0998247E-2,1.9545749E-2,1.4069624E-2,0E0,0E0,0E0,0E0,0E0,2.5777414E-2,1.5146682E-2,0E0,0E0,1.9750677E-2,2.143085E-3,0E0,0E0,1.3125047E-2,0E0,0E0,0E0,4.6508163E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,9,9,11,11,13,13,14,14,15,15,19,19,21,21,22,22,23,23,29,29,30,30,33,33,34,34,37,37,41,41],"right_children":[2,4,6,-1,8,10,12,14,-1,16,-1,18,-1,20,22,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,-1,34,36,-1,-1,38,40,-1,-1,42,-1,-1,-1,44,-1,-1,-1],"split_conditions":[1.747361E6,8.8E1,2.8512E5,-1.4740981E-2,1.5063692E9,3.803042E-1,4.744117E3,8.10263E5,3.8491404E-3,6.021393E6,-5.045936E-3,1.36215E5,1.0301628E-2,2.8681312E8,9.608E3,3.2744336E3,7.988821E-3,2.1562346E-3,5.4637957E-3,1.05E4,-2.035484E-3,2.943459E3,2.160768E6,2.423267E6,7.799296E-4,-3.5117813E-3,-6.3982573E-3,2.2336564E-3,-1.7485883E-3,9.4E1,1.289283E6,1.5626627E-3,4.2973952E-3,7.2269994E-1,1.05E2,5.662399E-4,-2.4607813E-3,1.1662405E9,1.9066212E-4,-6.0536056E-3,-3.0665537E-3,5.0364403E8,-6.4891047E-4,-4.0403083E-3,-2.0649969E-3],"split_indices":[6,0,12,0,16,62,5,2,0,6,0,20,0,16,28,5,0,0,0,28,0,5,2,6,0,0,0,0,0,19,6,0,0,66,4,0,0,11,0,0,0,16,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,1.1E2,6.4E1,5E0,1.05E2,3.4E1,3E1,9.7E1,8E0,2.9E1,5E0,1.2E1,1.8E1,3.5E1,6.2E1,2.4E1,5E0,5E0,7E0,2.7E1,8E0,1.3E1,4.9E1,1.7E1,7E0,9E0,1.8E1,7E0,6E0,3.8E1,1.1E1,9E0,8E0,2.8E1,1E1,6E0,5E0,2.3E1,5E0,5E0,5E0,1.8E1,5E0,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[3.537319E-3,-6.908597E-2,1.4277318E-1,-1.4204815E-2,-5.4735437E-2,1.10683665E-1,1.8265355E-2,-9.0785965E-2,-2.9467074E-3,6.6246204E-2,1.5275519E-1,-1.0337259E-1,-2.5531761E-2,4.2043175E-3,-1.9823773E-2,1.0082204E-1,-9.5112843E-4,1.6705255E-1,3.163939E-3,-1.319958E-1,-7.315883E-2,5.5404555E-4,-2.8751523E-3,-3.4089636E-2,3.0149268E-3,5.312753E-3,1.2346584E-3,5.490841E-3,8.484207E-3,-3.179302E-3,-6.203493E-3,-1.7376858E-3,-8.617414E-2,-4.6330128E-2,3.3955704E-4,-4.640197E-3,-2.4182547E-3,-3.5380658E-3,-2.2531768E-2,-3.9208528E-2,4.048595E-4,-2.377729E-3,-6.486457E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,-1,7,9,-1,11,13,15,17,19,21,-1,23,25,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,-1,39,41,-1,-1,-1],"loss_changes":[1.7996463E0,4.1262203E-1,4.93366E-1,0E0,2.0883772E-1,9.4575346E-2,0E0,5.156851E-2,8.065455E-2,9.047231E-2,2.0022988E-2,3.7908137E-2,1.9340964E-2,0E0,5.435189E-2,2.8390914E-2,0E0,1.4765859E-3,0E0,5.8315694E-3,9.155646E-3,0E0,0E0,1.8810447E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.502375E-3,2.1316655E-2,0E0,0E0,0E0,0E0,9.911734E-3,3.651211E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,19,19,20,20,23,23,32,32,33,33,38,38,39,39],"right_children":[2,4,6,-1,8,10,-1,12,14,16,18,20,22,-1,24,26,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,-1,40,42,-1,-1,-1],"split_conditions":[1.07218E5,3.3778E4,2.338928E0,-1.4204815E-2,6.88588E5,9.4806E4,1.8265355E-2,6.7842746E8,4.5623E4,1.8787456E7,1.5357575E0,4.743E3,2.943459E3,4.2043175E-3,1.1257E4,2.06842E5,-9.5112843E-4,1.85215E5,3.163939E-3,3.03719E5,3.565448E7,5.5404555E-4,-2.8751523E-3,3.1442E4,3.0149268E-3,5.312753E-3,1.2346584E-3,5.490841E-3,8.484207E-3,-3.179302E-3,-6.203493E-3,-1.7376858E-3,2.551424E6,2.2505884E3,3.3955704E-4,-4.640197E-3,-2.4182547E-3,-3.5380658E-3,1.6507361E9,6.2914E4,4.048595E-4,-2.377729E-3,-6.486457E-4],"split_indices":[12,6,58,0,6,20,0,16,12,18,66,3,5,0,3,12,0,20,0,9,16,0,0,7,0,0,0,0,0,0,0,0,18,5,0,0,0,0,11,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,1.16E2,6E1,5E0,1.11E2,5.5E1,5E0,6.5E1,4.6E1,2.8E1,2.7E1,5.4E1,1.1E1,6E0,4E1,2E1,8E0,2.2E1,5E0,2.6E1,2.8E1,6E0,5E0,3.5E1,5E0,1.5E1,5E0,1.1E1,1.1E1,5E0,2.1E1,9E0,1.9E1,2.7E1,8E0,1E1,9E0,1E1,1.7E1,1.1E1,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.138518E-3,-6.911124E-2,1.12995364E-1,-1.4085492E-2,-5.366726E-2,8.577541E-2,1.7178653E-2,-1.063967E-1,-1.5617538E-2,1.0709872E-1,-3.588035E-3,-1.188333E-1,-1.6207139E-3,-2.8291505E-2,4.594962E-3,4.9607042E-2,1.733997E-1,-5.4924367E-3,-2.7426567E-3,2.4054134E-3,-3.7548795E-2,2.8920263E-2,4.4327127E-3,3.497545E-3,2.0203653E-1,-5.00234E-2,-1.0071035E-3,-7.912892E-4,5.2127674E-2,6.0716937E-3,1.1500568E-2,-6.043645E-2,-1.4926848E-4,2.1889457E-3,-1.5276746E-3,6.995059E-2,7.2774384E-4,-8.077304E-2,-3.3359215E-2,3.8689976E-3,1.7284036E-3,-9.6675985E-2,-1.5496829E-3,-2.6589765E-3,-7.1074275E-5,-5.353931E-3,-2.644532E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,17,-1,19,-1,21,23,-1,-1,-1,25,27,-1,-1,29,31,33,-1,35,-1,-1,37,-1,-1,-1,39,-1,41,43,-1,-1,45,-1,-1,-1,-1,-1],"loss_changes":[1.4206185E0,4.056784E-1,5.0850713E-1,0E0,2.0499137E-1,2.452107E-1,0E0,3.262821E-2,9.644247E-2,2.1629173E-1,0E0,6.2129498E-3,0E0,4.5305748E-2,0E0,3.252823E-2,5.59237E-2,0E0,0E0,0E0,2.3197621E-2,2.8242517E-2,0E0,0E0,4.068911E-2,1.8234126E-2,2.610213E-2,0E0,9.575948E-3,0E0,0E0,1.4375895E-2,0E0,0E0,0E0,2.7304403E-3,0E0,8.805744E-3,1.3018761E-2,0E0,0E0,3.821984E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,11,11,13,13,15,15,16,16,20,20,21,21,24,24,25,25,26,26,28,28,31,31,35,35,37,37,38,38,41,41],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,18,-1,20,-1,22,24,-1,-1,-1,26,28,-1,-1,30,32,34,-1,36,-1,-1,38,-1,-1,-1,40,-1,42,44,-1,-1,46,-1,-1,-1,-1,-1],"split_conditions":[8.8449E4,1.1562E4,1.0362809E1,-1.4085492E-2,1.001171E6,3.5713768E0,1.7178653E-2,2.4434292E7,1.7015084E9,6.820745E6,-3.588035E-3,8.59522E5,-1.6207139E-3,1.7726E4,4.594962E-3,3.38E2,4.1E1,-5.4924367E-3,-2.7426567E-3,2.4054134E-3,1.78E2,1.454E3,4.4327127E-3,3.497545E-3,1.5522577E7,1.466404E6,1E0,-7.912892E-4,2.06842E5,6.0716937E-3,1.1500568E-2,2.2632957E3,-1.4926848E-4,2.1889457E-3,-1.5276746E-3,2.065E3,7.2774384E-4,4.005696E8,5.29E2,3.8689976E-3,1.7284036E-3,1.1768006E9,-1.5496829E-3,-2.6589765E-3,-7.1074275E-5,-5.353931E-3,-2.644532E-3],"split_indices":[12,23,59,0,2,59,0,27,16,6,0,2,0,18,0,17,17,0,0,0,17,0,0,0,6,6,19,0,12,0,0,5,0,0,0,0,0,23,0,0,0,11,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.77E2,1.07E2,7E1,5E0,1.02E2,6.5E1,5E0,4.2E1,6E1,5.8E1,7E0,3.5E1,7E0,5.5E1,5E0,3.2E1,2.6E1,3E1,5E0,5E0,5E1,2.4E1,8E0,7E0,1.9E1,3.7E1,1.3E1,8E0,1.6E1,1.1E1,8E0,3E1,7E0,5E0,8E0,1E1,6E0,1.6E1,1.4E1,5E0,5E0,1.1E1,5E0,7E0,7E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.1301422E-3,-6.039286E-2,1.0178702E-1,-1.4109695E-2,-4.449335E-2,7.171713E-2,1.5057663E-2,-5.477222E-2,4.5523127E-3,9.322331E-2,-5.996029E-3,-9.076495E-2,-3.2989796E-2,6.9487326E-2,1.0739657E-2,-1.08291686E-1,-6.9332693E-4,-4.1125543E-2,2.173534E-3,7.9866506E-2,-7.947192E-4,-9.1040686E-2,-6.525358E-3,-4.150214E-3,-2.7621472E-2,-4.9601977E-6,9.012329E-2,-1.0105971E-1,-1.9994718E-3,-9.962385E-3,-5.3121056E-2,4.4462673E-2,1.0433514E-1,-5.1408624E-3,-2.5295238E-3,1.4093021E-3,-2.2388263E-2,-1.0569848E-3,-6.1714273E-2,2.9541526E-3,6.0372154E-4,5.1605585E-3,2.6323982E-3,-2.2668126E-3,-3.1675368E-3,-3.3692042E-3,-1.5367566E-3,-1.3918605E-3,1.1326357E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,-1,15,17,19,-1,21,-1,23,-1,25,-1,27,-1,-1,29,-1,31,33,-1,35,37,39,41,-1,-1,-1,43,-1,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1],"loss_changes":[1.0774866E0,4.3701363E-1,4.579183E-1,0E0,1.6324219E-1,2.8261396E-1,0E0,7.305452E-2,0E0,1.8242413E-1,0E0,4.5526296E-2,4.3807544E-2,4.6054244E-2,0E0,7.006407E-3,0E0,3.8908727E-2,0E0,3.6105156E-2,0E0,5.7932734E-3,0E0,0E0,2.0634856E-2,0E0,2.0668924E-2,6.1495155E-3,0E0,1.5948085E-2,3.2247268E-3,6.887613E-3,1.0613978E-2,0E0,0E0,0E0,1.2897516E-2,0E0,3.0256659E-3,0E0,0E0,0E0,0E0,0E0,1.3372014E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,11,11,12,12,13,13,15,15,17,17,19,19,21,21,24,24,26,26,27,27,29,29,30,30,31,31,32,32,36,36,38,38,44,44],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,-1,16,18,20,-1,22,-1,24,-1,26,-1,28,-1,-1,30,-1,32,34,-1,36,38,40,42,-1,-1,-1,44,-1,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1],"split_conditions":[1.789754E6,1.45E2,1.338554E0,-1.4109695E-2,1.5063692E9,1.1512978E0,1.5057663E-2,1.8248318E3,4.5523127E-3,4.37004E9,-5.996029E-3,8.008664E8,1.0965E4,1E0,1.0739657E-2,7E1,-6.9332693E-4,9.5751E4,2.173534E-3,1.1386601E10,-7.947192E-4,1.03E2,-6.525358E-3,-4.150214E-3,2.845111E3,-4.9601977E-6,3.9019544E7,5.081E3,-1.9994718E-3,5.55E2,5.5539E4,7.638841E6,1.8787456E7,-5.1408624E-3,-2.5295238E-3,1.4093021E-3,1.05E2,-1.0569848E-3,1.6273E4,2.9541526E-3,6.0372154E-4,5.1605585E-3,2.6323982E-3,-2.2668126E-3,1.739898E6,-3.3692042E-3,-1.5367566E-3,-1.3918605E-3,1.1326357E-3],"split_indices":[6,0,63,0,16,63,0,5,0,27,0,21,3,35,0,19,0,18,0,10,0,17,0,0,5,0,21,3,0,22,8,2,18,0,0,0,4,0,28,0,0,0,0,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.72E2,1.07E2,6.5E1,5E0,1.02E2,5.9E1,6E0,9.6E1,6E0,5.4E1,5E0,3.5E1,6.1E1,4.8E1,6E0,2.8E1,7E0,5.6E1,5E0,4.3E1,5E0,2.2E1,6E0,1E1,4.6E1,5E0,3.8E1,1.7E1,5E0,2.8E1,1.8E1,1E1,2.8E1,1.1E1,6E0,6E0,2.2E1,5E0,1.3E1,5E0,5E0,2E1,8E0,8E0,1.4E1,7E0,6E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.01938285E-2,-4.2751823E-2,1.3452539E-1,-8.772398E-2,2.8883282E-3,1.0333965E-1,1.6009165E-2,-1.0057063E-1,2.3229783E-3,-2.6223594E-2,3.281974E-2,1.21080175E-1,-1.9196263E-3,-1.1758746E-1,-7.891226E-2,-5.2496742E-2,-4.101447E-3,1.040811E-2,5.5956272E-3,9.1131374E-2,9.331404E-3,-3.2233158E-3,-5.6561916E-3,-5.927003E-3,-5.816936E-2,-3.4332273E-3,-4.9582444E-4,-3.2825902E-2,1.3732166E-3,4.4218186E-2,-1.208124E-2,5.880487E-2,1.258438E-1,-8.911178E-4,-7.029293E-2,-2.5220623E-3,-1.0464523E-4,-4.7804794E-4,4.0163584E-3,-2.7642613E-3,1.4551075E-2,7.730267E-2,5.8524305E-4,3.0486824E-3,6.326455E-3,-4.0005445E-3,-5.0307356E-2,-5.524841E-4,1.5623171E-3,2.2565667E-3,4.487871E-3,-2.747091E-3,-1.278471E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,27,29,-1,31,-1,-1,-1,-1,33,-1,-1,35,-1,37,39,41,43,-1,45,-1,-1,-1,-1,-1,47,49,-1,-1,-1,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[1.190658E0,2.609254E-1,3.5323322E-1,1.21236324E-1,5.662775E-2,1.3338876E-1,0E0,1.2465775E-2,0E0,1.8964894E-2,6.707123E-2,1.0268879E-1,0E0,6.7032874E-3,2.6317984E-2,1.5249994E-2,2.0442005E-2,2.1064108E-2,0E0,3.187436E-2,0E0,0E0,0E0,0E0,9.091936E-3,0E0,0E0,8.214995E-3,0E0,3.0017333E-2,2.4125453E-2,1.5758112E-2,6.989956E-3,0E0,2.647519E-3,0E0,0E0,0E0,0E0,0E0,7.3892446E-3,3.8310215E-3,0E0,0E0,0E0,0E0,1.0765828E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,19,19,24,24,27,27,29,29,30,30,31,31,32,32,34,34,40,40,41,41,46,46],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,28,30,-1,32,-1,-1,-1,-1,34,-1,-1,36,-1,38,40,42,44,-1,46,-1,-1,-1,-1,-1,48,50,-1,-1,-1,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[3.329577E6,1.365503E6,1.7903596E0,3.48E2,3.135717E6,1.473603E0,1.6009165E-2,1.7015095E3,2.3229783E-3,1E0,1.9844815E9,1.5992037E11,-1.9196263E-3,9.67E3,2.6E1,3.525199E6,2.4771E4,2.6E1,5.5956272E-3,1.1034199E10,9.331404E-3,-3.2233158E-3,-5.6561916E-3,-5.927003E-3,5.239E3,-3.4332273E-3,-4.9582444E-4,3.16E2,1.3732166E-3,8.210746E7,9.7E2,3.312442E6,3E1,-8.911178E-4,1.97E2,-2.5220623E-3,-1.0464523E-4,-4.7804794E-4,4.0163584E-3,-2.7642613E-3,8.047E4,7.031403E8,5.8524305E-4,3.0486824E-3,6.326455E-3,-4.0005445E-3,2.228942E6,-5.524841E-4,1.5623171E-3,2.2565667E-3,4.487871E-3,-2.747091E-3,-1.278471E-3],"split_indices":[6,2,66,17,18,66,0,5,0,29,11,13,0,28,17,2,7,19,0,11,0,0,0,0,3,0,0,4,0,21,0,18,4,0,20,0,0,0,0,0,8,16,0,0,0,0,18,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,1.26E2,5.3E1,6.3E1,6.3E1,4.8E1,5E0,5.8E1,5E0,3.2E1,3.1E1,4.3E1,5E0,3E1,2.8E1,1.4E1,1.8E1,2.6E1,5E0,3.4E1,9E0,8E0,2.2E1,6E0,2.2E1,8E0,6E0,1E1,8E0,1E1,1.6E1,1.9E1,1.5E1,6E0,1.6E1,5E0,5E0,5E0,5E0,5E0,1.1E1,1.3E1,6E0,5E0,1E1,6E0,1E1,5E0,6E0,8E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[9.896335E-3,-4.0293474E-2,9.498226E-2,-4.8204824E-2,4.40463E-3,7.574531E-2,1.1571196E-2,-6.7283444E-2,1.6604185E-2,8.839188E-2,-2.120814E-3,-1.0334733E-1,-4.7641784E-2,5.372111E-2,-2.49895E-2,6.1364908E-2,1.0864299E-1,-3.224868E-3,-5.324541E-3,-8.265016E-2,-2.8814211E-2,4.2040106E-3,1.5054583E-4,-2.6322927E-3,3.8645422E-4,7.9181686E-2,1.8500478E-4,1.2804943E-1,7.870398E-2,-4.1883667E-3,-2.357702E-3,-4.7302566E-2,-7.697937E-3,4.8856894E-3,1.6807801E-3,3.2679748E-3,6.3468697E-3,4.497021E-3,1.2699033E-3,-4.6958835E-4,-6.334132E-2,1.4875665E-3,-3.7383404E-2,-3.512343E-3,-1.3069587E-3,-2.200111E-3,-7.912848E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,-1,-1,-1,-1,33,-1,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,43,-1,45,-1,-1,-1,-1],"loss_changes":[7.2553265E-1,1.243996E-1,1.861006E-1,1.272368E-1,0E0,9.5495194E-2,0E0,5.1036328E-2,3.8394377E-2,2.23414E-2,0E0,5.8173835E-3,3.255201E-2,2.7506739E-2,1.4877962E-2,2.5109515E-2,6.7712963E-3,0E0,0E0,1.676254E-3,1.3690453E-2,0E0,0E0,0E0,0E0,2.1002173E-2,0E0,3.8865507E-3,1.3898626E-2,0E0,0E0,1.045474E-2,2.3598623E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.027812E-3,0E0,1.8445076E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,25,25,27,27,28,28,31,31,32,32,40,40,42,42],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,-1,-1,-1,-1,34,-1,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,44,-1,46,-1,-1,-1,-1],"split_conditions":[9.753E4,1.7015084E9,8.704162E0,1.07E3,4.40463E-3,1.1554533E0,1.1571196E-2,1.7980087E3,1.502E3,3.715703E9,-2.120814E-3,1.4899E4,2.228942E6,6.9713E4,4.8858527E9,9.2977E4,1.5E2,-3.224868E-3,-5.324541E-3,1.402E3,3.0960486E8,4.2040106E-3,1.5054583E-4,-2.6322927E-3,3.8645422E-4,1.2166E4,1.8500478E-4,2.6572972E9,9.22E2,-4.1883667E-3,-2.357702E-3,5.398E3,2.6E1,4.8856894E-3,1.6807801E-3,3.2679748E-3,6.3468697E-3,4.497021E-3,1.2699033E-3,-4.6958835E-4,1E0,1.4875665E-3,1.38E2,-3.512343E-3,-1.3069587E-3,-2.200111E-3,-7.912848E-4],"split_indices":[12,16,59,0,0,63,0,5,0,21,0,28,18,28,11,28,17,0,0,22,11,0,0,0,0,3,0,27,19,0,0,28,19,0,0,0,0,0,0,0,41,0,19,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,1.06E2,6.2E1,1.01E2,5E0,5.7E1,5E0,7.8E1,2.3E1,5.2E1,5E0,2.6E1,5.2E1,1.2E1,1.1E1,2.4E1,2.8E1,1.2E1,1.4E1,1.7E1,3.5E1,6E0,6E0,5E0,6E0,1.8E1,6E0,1.5E1,1.3E1,1E1,7E0,1.8E1,1.7E1,9E0,9E0,5E0,1E1,8E0,5E0,6E0,1.2E1,7E0,1E1,7E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.0386558E-2,-6.602393E-2,9.118457E-2,-1.3654539E-2,-5.170055E-2,1.1837884E-2,7.063382E-2,-8.802337E-2,-2.2271108E-2,2.1286892E-2,1.4817272E-1,-1.02289245E-1,-3.0739142E-2,6.800144E-4,-5.5347927E-2,5.298928E-2,-6.3749203E-3,8.4740214E-2,8.466257E-3,-5.112943E-3,-2.6338114E-3,3.2433665E-5,-2.4921598E-3,-2.3734695E-3,2.031961E-2,-4.0541915E-3,-3.2219023E-2,3.6341E-2,4.203655E-3,4.597932E-3,2.1829254E-3,-1.2309244E-2,5.38277E-2,-2.1954726E-3,-4.6079842E-4,5.1223904E-2,-3.1080493E-4,-1.3630929E-3,9.2867E-4,3.299753E-3,5.3233496E-4,2.851074E-3,8.6155575E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,-1,7,-1,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,-1,-1,-1,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.893207E-1,3.833096E-1,2.1023327E-1,0E0,1.1348736E-1,0E0,2.1494535E-1,3.608176E-2,4.6345793E-2,1.987262E-1,4.7475606E-2,1.6607493E-2,9.170687E-3,4.1114315E-2,1.8712707E-2,1.8857896E-2,0E0,2.6018992E-3,0E0,0E0,0E0,0E0,0E0,0E0,3.127583E-2,0E0,6.127501E-3,1.5888896E-2,0E0,0E0,0E0,1.0196592E-2,1.1768281E-2,0E0,0E0,6.6999607E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,24,24,26,26,27,27,31,31,32,32,35,35],"right_children":[2,4,6,-1,8,-1,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,-1,-1,-1,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.503171E6,2.0584E4,1.308E3,-1.3654539E-2,1.15856E6,1.1837884E-2,1.66368E5,1.5070434E8,7.7E1,2.8413632E7,5.518522E3,5.583E3,1.5004E4,1E0,1.882904E6,4.949877E6,-6.3749203E-3,1E0,8.466257E-3,-5.112943E-3,-2.6338114E-3,3.2433665E-5,-2.4921598E-3,-2.3734695E-3,1.19E2,-4.0541915E-3,2.7143E4,2.228899E6,4.203655E-3,4.597932E-3,2.1829254E-3,1.466404E6,8.753E3,-2.1954726E-3,-4.6079842E-4,1.3086891E10,-3.1080493E-4,-1.3630929E-3,9.2867E-4,3.299753E-3,5.3233496E-4,2.851074E-3,8.6155575E-4],"split_indices":[2,23,0,0,2,0,20,21,19,2,5,3,12,29,2,18,0,29,0,0,0,0,0,0,17,0,7,18,0,0,0,6,3,0,0,23,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,1.12E2,6.1E1,5E0,1.07E2,5E0,5.6E1,4.7E1,6E1,3.5E1,2.1E1,3.7E1,1E1,3.6E1,2.4E1,3E1,5E0,1E1,1.1E1,2.6E1,1.1E1,5E0,5E0,9E0,2.7E1,8E0,1.6E1,2.3E1,7E0,5E0,5E0,1.4E1,1.3E1,8E0,8E0,1.7E1,6E0,9E0,5E0,8E0,5E0,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.10038025E-2,-2.8805526E-2,1.1567446E-1,-4.596783E-2,3.393895E-2,8.6585164E-2,1.1559556E-2,-6.4051606E-2,3.1850503E-3,1.571368E-2,4.54893E-3,1.0874158E-1,-3.2977494E-3,-8.142919E-2,-4.1033126E-2,3.4541525E-3,-1.9100754E-2,2.4528613E-3,-6.6434257E-3,9.660452E-4,1.20481834E-1,-5.1628385E-2,-4.452539E-3,-5.407808E-2,-6.4693247E-3,-4.2495083E-2,7.813154E-4,-1.4903696E-3,9.4265764E-4,9.987603E-2,7.071183E-3,-8.3719677E-4,-6.416427E-2,-7.13352E-2,-5.916412E-4,2.783865E-4,-7.960577E-4,-3.3456301E-3,-4.891424E-4,5.6121284E-3,7.7116184E-2,-1.5830934E-3,-3.5157958E-3,-3.931063E-3,-1.8592752E-3,1.6929862E-3,4.1244654E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,-1,21,23,-1,25,-1,27,-1,29,31,-1,33,35,37,-1,-1,-1,39,-1,-1,41,43,-1,-1,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8670335E-1,1.4908716E-1,2.0270646E-1,9.6201956E-2,3.6789637E-2,1.6987616E-1,0E0,2.8259158E-2,5.2292123E-2,2.3001043E-2,0E0,3.800559E-2,0E0,2.1723032E-2,1.5881944E-2,0E0,2.1535741E-2,0E0,1.3941279E-2,0E0,1.3768554E-2,6.9043636E-3,0E0,1.7215066E-2,1.7795847E-3,1.5171714E-2,0E0,0E0,0E0,7.0702434E-3,0E0,0E0,3.4775622E-3,5.851768E-3,0E0,0E0,0E0,0E0,0E0,0E0,7.6590106E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,13,13,14,14,16,16,18,18,20,20,21,21,23,23,24,24,25,25,29,29,32,32,33,33,40,40],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,-1,22,24,-1,26,-1,28,-1,30,32,-1,34,36,38,-1,-1,-1,40,-1,-1,42,44,-1,-1,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2538E4,1.9717047E9,1.7903596E0,6.7842746E8,6.021393E6,1.018047E8,1.1559556E-2,1.244363E6,2.81E3,3.8227876E9,4.54893E-3,1.2779233E10,-3.2977494E-3,1.2689E4,2.5484189E8,3.4541525E-3,1.8117093E-1,2.4528613E-3,5.5325594E9,9.660452E-4,7.2487605E-1,1.34E2,-4.452539E-3,2.8E2,5.6E1,7.115E3,7.813154E-4,-1.4903696E-3,9.4265764E-4,7E1,7.071183E-3,-8.3719677E-4,5.251E4,6.8E1,-5.916412E-4,2.783865E-4,-7.960577E-4,-3.3456301E-3,-4.891424E-4,5.6121284E-3,6.820745E6,-1.5830934E-3,-3.5157958E-3,-3.931063E-3,-1.8592752E-3,1.6929862E-3,4.1244654E-3],"split_indices":[3,11,66,16,6,2,0,2,28,11,0,10,0,28,16,0,62,0,11,0,58,4,0,4,19,3,0,0,0,19,0,0,8,17,0,0,0,0,0,0,6,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.36E2,5.1E1,1.07E2,2.9E1,4.4E1,7E0,7.8E1,2.9E1,2.4E1,5E0,3.9E1,5E0,4.3E1,3.5E1,6E0,2.3E1,8E0,1.6E1,5E0,3.4E1,1.9E1,2.4E1,2.5E1,1E1,1.4E1,9E0,8E0,8E0,2.5E1,9E0,6E0,1.3E1,1.7E1,8E0,5E0,5E0,6E0,8E0,9E0,1.6E1,6E0,7E0,9E0,8E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.1638526E-3,-6.410741E-2,9.733998E-2,-1.2629664E-2,-4.9513377E-2,6.946835E-2,1.4937821E-2,-8.180187E-2,-2.9116858E-2,4.5495078E-2,1.0367845E-1,-9.3314245E-2,-1.2398005E-3,-4.7969688E-2,-3.599604E-3,2.422007E-2,7.5893834E-2,5.1360917E-3,2.1335052E-3,-5.4994253E-3,-7.330701E-2,-7.733466E-2,-2.7102446E-2,1.4354059E-3,-2.366983E-2,2.7134365E-3,3.037061E-3,5.371958E-3,5.816118E-4,-4.777296E-2,-4.0232325E-3,-4.2073647E-3,-1.9424221E-3,-3.8939754E-3,-4.995692E-2,-4.8553623E-2,9.7636005E-4,1.9111317E-3,-2.1471782E-2,-2.8505013E-3,-9.722599E-4,3.5603344E-4,-7.5529236E-4,-2.8862134E-3,-1.1113072E-3,-2.9895222E-3,-7.5065607E-4,-1.808127E-3,2.351273E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,-1,7,9,-1,11,13,15,17,19,-1,21,23,25,27,-1,-1,-1,29,31,33,-1,35,37,-1,-1,-1,39,-1,-1,-1,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0911204E0,3.2098165E-1,4.6085864E-1,0E0,6.149973E-2,4.8591495E-2,0E0,2.0094126E-2,2.917274E-2,2.4031192E-2,1.4187425E-2,1.0380834E-2,0E0,1.935085E-2,2.0550242E-2,2.4201013E-2,4.5595378E-2,0E0,0E0,0E0,5.714692E-3,4.47125E-3,1.1474652E-2,0E0,2.1707518E-2,1.8820526E-2,0E0,0E0,0E0,3.463352E-3,0E0,0E0,0E0,2.0805092E-3,2.6732292E-3,6.324595E-3,0E0,0E0,6.654595E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,20,20,21,21,22,22,24,24,25,25,29,29,33,33,34,34,35,35,38,38],"right_children":[2,4,6,-1,8,10,-1,12,14,16,18,20,-1,22,24,26,28,-1,-1,-1,30,32,34,-1,36,38,-1,-1,-1,40,-1,-1,-1,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.789754E6,8.461E3,1.11464E5,-1.2629664E-2,1.8248318E3,6.476062E6,1.4937821E-2,6.3565E4,9.203419E6,4.949877E6,1.5860209E9,9.6E1,-1.2398005E-3,4.59E2,7.952E3,6.490565E8,1.8787456E7,5.1360917E-3,2.1335052E-3,-5.4994253E-3,2.9819E4,3.6657E4,7.61E2,1.4354059E-3,6.488686E-1,1.9738E4,3.037061E-3,5.371958E-3,5.816118E-4,4.33E2,-4.0232325E-3,-4.2073647E-3,-1.9424221E-3,1.8650388E10,1.108E3,7.118E3,9.7636005E-4,1.9111317E-3,2.102E3,-2.8505013E-3,-9.722599E-4,3.5603344E-4,-7.5529236E-4,-2.8862134E-3,-1.1113072E-3,-2.9895222E-3,-7.5065607E-4,-1.808127E-3,2.351273E-4],"split_indices":[6,7,3,0,5,6,0,28,18,18,16,4,0,0,28,16,18,0,0,0,12,12,0,0,65,18,0,0,0,20,0,0,0,10,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,1.01E2,7E1,5E0,9.6E1,6.4E1,6E0,3.6E1,6E1,3.9E1,2.5E1,2.9E1,7E0,3.4E1,2.6E1,2.4E1,1.5E1,1.9E1,6E0,9E0,2E1,1.3E1,2.1E1,9E0,1.7E1,1.7E1,7E0,8E0,7E0,1E1,1E1,7E0,6E0,1.1E1,1E1,1.1E1,6E0,6E0,1.1E1,5E0,5E0,6E0,5E0,5E0,5E0,6E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[6.0412716E-3,-2.3238398E-2,1.389476E-1,-1.2245036E-2,-1.2430859E-2,1.620042E-2,8.422329E-2,6.758001E-3,-1.9510536E-2,1.11878514E-1,-1.5636767E-3,-5.9220646E-2,1.1012306E-2,1.3204291E-1,2.048139E-3,-7.331759E-2,-1.5305914E-3,-5.022399E-4,7.630325E-2,2.614744E-3,6.857256E-3,-5.510375E-3,-5.991029E-2,-8.6821144E-4,1.0128698E-3,-2.265746E-2,1.612673E-2,4.899832E-3,9.4433373E-4,-4.348294E-2,-3.7311786E-3,-2.9415528E-3,-4.217726E-2,3.181663E-2,-1.4894354E-3,-3.0806544E-3,-3.0469468E-2,-2.9373813E-2,2.008101E-3,-2.730426E-3,-7.212759E-4,-4.7422652E-4,4.249484E-2,-5.2782572E-5,-4.2923156E-2,-7.6892626E-5,-2.273581E-3,1.9512845E-2,6.9970764E-2,-2.4795611E-3,-6.5207353E-4,4.3449818E-5,1.5534419E-3,3.943293E-3,1.6557222E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,-1,7,-1,9,-1,11,13,-1,15,17,19,-1,21,23,25,27,-1,-1,-1,29,-1,-1,31,33,-1,-1,35,-1,37,39,41,-1,-1,43,45,-1,-1,-1,-1,47,-1,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1],"loss_changes":[7.114735E-1,4.136443E-1,3.8810945E-1,0E0,1.7472655E-1,0E0,9.69675E-2,0E0,1.7034131E-1,2.3927987E-2,0E0,4.949625E-2,6.0181975E-2,2.132523E-2,0E0,2.7209997E-2,6.366089E-3,2.5786266E-2,2.1159574E-2,0E0,0E0,0E0,1.3508275E-2,0E0,0E0,1.142145E-2,3.2623295E-2,0E0,0E0,7.3707923E-3,0E0,2.2052921E-2,6.8079997E-3,1.45903975E-2,0E0,0E0,7.236032E-3,6.807983E-3,0E0,0E0,0E0,0E0,1.442096E-2,0E0,4.5915134E-3,0E0,0E0,4.4304733E-3,3.7527606E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,8,8,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,22,22,25,25,26,26,29,29,31,31,32,32,33,33,36,36,37,37,42,42,44,44,47,47,48,48],"right_children":[2,4,6,-1,8,-1,10,-1,12,14,-1,16,18,20,-1,22,24,26,28,-1,-1,-1,30,-1,-1,32,34,-1,-1,36,-1,38,40,42,-1,-1,44,46,-1,-1,-1,-1,48,-1,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1],"split_conditions":[6.476062E6,2.711E3,2.5296833E10,-1.2245036E-2,2.2E1,1.620042E-2,1.1377422E0,6.758001E-3,6.88588E5,1.96E2,-1.5636767E-3,6.7842746E8,1.5522031E9,5.9081E4,2.048139E-3,1.4E1,1.0114802E7,1.747361E6,6.53E2,2.614744E-3,6.857256E-3,-5.510375E-3,2.4151E4,-8.6821144E-4,1.0128698E-3,1.2026E4,1.71E2,4.899832E-3,9.4433373E-4,2.39E2,-3.7311786E-3,6.3494E4,3.135717E6,1.8448E4,-1.4894354E-3,-3.0806544E-3,1.3555108E-3,1.800758E6,2.008101E-3,-2.730426E-3,-7.212759E-4,-4.7422652E-4,1.3574205E9,-5.2782572E-5,1.2512887E-1,-7.6892626E-5,-2.273581E-3,4.879321E6,5.2E1,-2.4795611E-3,-6.5207353E-4,4.3449818E-5,1.5534419E-3,3.943293E-3,1.6557222E-3],"split_indices":[6,3,10,0,20,0,63,0,6,17,0,16,16,20,0,17,21,6,17,0,0,0,28,0,0,20,17,0,0,0,0,8,18,7,0,0,63,18,0,0,0,0,21,0,62,0,0,2,19,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.81E2,1.49E2,3.2E1,5E0,1.44E2,5E0,2.7E1,5E0,1.39E2,2.2E1,5E0,6E1,7.9E1,1.6E1,6E0,4.8E1,1.2E1,6.8E1,1.1E1,5E0,1.1E1,8E0,4E1,7E0,5E0,2.9E1,3.9E1,6E0,5E0,2.6E1,1.4E1,1.5E1,1.4E1,3E1,9E0,7E0,1.9E1,1E1,5E0,7E0,7E0,6E0,2.4E1,6E0,1.3E1,5E0,5E0,1.4E1,1E1,8E0,5E0,7E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.2146747E-3,-4.1229E-2,1.1703707E-1,-1.1613425E-2,-3.0106481E-2,8.232467E-2,1.4105005E-2,-5.8067277E-2,1.4446334E-2,1.6302677E-2,1.3260064E-1,-7.519653E-2,-3.1629384E-2,4.7546006E-3,-1.2712164E-3,6.600302E-2,-4.1071097E-3,2.533165E-3,6.595789E-3,-8.14517E-2,-8.6945074E-4,-1.1410859E-2,-4.461076E-3,1.2063692E-2,-2.6676739E-3,1.3563789E-3,4.4332896E-3,-8.788534E-2,-1.9172896E-3,1.1681586E-3,-2.6883839E-2,-1.4685234E-3,2.6017935E-3,-4.8546987E-3,-3.1376109E-3,-2.584431E-3,2.010642E-3,2.0525586E-3,-1.6164005E-2,5.507506E-4,-3.8986027E-4,-2.6660815E-3,1.1864948E-2,1.003351E-3,-1.803046E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,-1,7,9,-1,11,13,15,17,19,21,-1,23,25,-1,-1,-1,27,-1,29,-1,31,-1,-1,-1,33,-1,-1,35,37,-1,-1,-1,-1,39,-1,41,-1,-1,-1,43,-1,-1],"loss_changes":[8.234205E-1,3.1710267E-1,2.8821903E-1,0E0,1.5655538E-1,1.2729168E-1,0E0,3.2491893E-2,7.2908506E-2,1.04976356E-1,2.1603405E-2,1.4282584E-2,4.449775E-2,0E0,3.505434E-2,1.29934475E-2,0E0,0E0,0E0,6.037593E-3,0E0,1.6077094E-2,0E0,2.3250194E-2,0E0,0E0,0E0,4.7013164E-3,0E0,0E0,1.7867178E-2,2.1427177E-2,0E0,0E0,0E0,0E0,1.38956E-3,0E0,2.968244E-2,0E0,0E0,0E0,2.7637915E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,19,19,21,21,23,23,27,27,30,30,31,31,36,36,38,38,42,42],"right_children":[2,4,6,-1,8,10,-1,12,14,16,18,20,22,-1,24,26,-1,-1,-1,28,-1,30,-1,32,-1,-1,-1,34,-1,-1,36,38,-1,-1,-1,-1,40,-1,42,-1,-1,-1,44,-1,-1],"split_conditions":[1.0656E4,2.58E3,1.338554E0,-1.1613425E-2,1.882904E6,1.174667E9,1.4105005E-2,1.8077296E8,2.5522802E-6,2.18E2,1.2779233E10,1.2751797E10,1.509977E8,4.7546006E-3,8.489E3,1.5713978E8,-4.1071097E-3,2.533165E-3,6.595789E-3,1.0254491E8,-8.6945074E-4,6.02E2,-4.461076E-3,1.07218E5,-2.6676739E-3,1.3563789E-3,4.4332896E-3,1.85E2,-1.9172896E-3,1.1681586E-3,7E0,7.7583334E8,2.6017935E-3,-4.8546987E-3,-3.1376109E-3,-2.584431E-3,1.4345135E3,2.0525586E-3,1.216696E6,5.507506E-4,-3.8986027E-4,-2.6660815E-3,7.9214E4,1.003351E-3,-1.803046E-4],"split_indices":[3,3,63,0,2,16,0,16,63,4,10,10,27,0,3,1,0,0,0,21,0,22,0,12,0,0,0,4,0,0,19,11,0,0,0,0,5,0,6,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.72E2,1.29E2,4.3E1,5E0,1.24E2,3.8E1,5E0,7.6E1,4.8E1,1.7E1,2.1E1,4.5E1,3.1E1,6E0,4.2E1,1.2E1,5E0,5E0,1.6E1,4E1,5E0,2.5E1,6E0,3.5E1,7E0,7E0,5E0,3.3E1,7E0,7E0,1.8E1,2.8E1,7E0,1.1E1,2.2E1,8E0,1E1,6E0,2.2E1,5E0,5E0,8E0,1.4E1,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.713461E-4,-2.4328373E-2,1.2879363E-1,-5.8427352E-2,2.6088933E-2,9.705507E-3,9.742016E-2,-7.145132E-2,-1.3472245E-2,3.2091658E-3,9.451429E-2,6.732608E-3,5.3927075E-2,-9.23433E-2,-5.3548478E-2,-3.2849487E-2,9.6555555E-4,-9.166307E-3,3.8077587E-3,1.4950876E-3,5.290834E-3,4.215277E-3,8.512056E-4,-1.8098062E-3,-4.627677E-3,-3.35232E-3,-4.0367413E-2,-2.3852615E-3,-6.401486E-4,5.1288893E-3,-4.0770485E-3,-2.593466E-4,-5.211924E-2,3.005605E-2,-8.587027E-3,-3.29063E-3,-3.0321376E-2,-4.0225612E-4,2.8385564E-3,5.573684E-3,-1.6413494E-3,-2.1351718E-3,-1.5583562E-4,-6.4745353E-4,1.3262745E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,33,-1,-1,35,37,39,-1,41,-1,-1,43,-1,-1,-1,-1,-1],"loss_changes":[5.34819E-1,2.5424355E-1,5.7599217E-2,5.056086E-2,9.363006E-2,0E0,4.606071E-2,2.0488918E-2,1.52107645E-2,4.8636064E-2,1.9986138E-2,0E0,1.8671554E-2,1.64873E-2,8.855067E-3,4.74781E-3,0E0,4.9854428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0613494E-2,0E0,0E0,1.256645E-2,0E0,0E0,7.859953E-3,1.845722E-2,1.0071617E-2,0E0,5.795462E-3,0E0,0E0,9.059006E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,26,26,29,29,32,32,33,33,34,34,36,36,39,39],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,34,-1,-1,36,38,40,-1,42,-1,-1,44,-1,-1,-1,-1,-1],"split_conditions":[1.5522031E9,6.2914E4,2.4386E4,1.5070434E8,3.246E3,9.705507E-3,1.8454934E9,1.7015095E3,2.44E2,2.3537487E9,2.054301E3,6.732608E-3,1.05E2,3.035E3,2.94E2,4.569E3,9.6555555E-4,7.822274E6,3.8077587E-3,1.4950876E-3,5.290834E-3,4.215277E-3,8.512056E-4,-1.8098062E-3,-4.627677E-3,-3.35232E-3,5.251E4,-2.3852615E-3,-6.401486E-4,7.9E1,-4.0770485E-3,-2.593466E-4,6.588E3,2.3717349E3,1.11877E5,-3.29063E-3,2.9E2,-4.0225612E-4,2.8385564E-3,6.9504E4,-1.6413494E-3,-2.1351718E-3,-1.5583562E-4,-6.4745353E-4,1.3262745E-3],"split_indices":[16,12,7,21,0,0,16,5,17,27,5,0,4,12,0,3,0,2,0,0,0,0,0,0,0,0,8,0,0,4,0,0,3,5,12,0,4,0,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.72E2,1.46E2,2.6E1,8.7E1,5.9E1,5E0,2.1E1,6.7E1,2E1,4.5E1,1.4E1,8E0,1.3E1,2.9E1,3.8E1,1.3E1,7E0,4E1,5E0,5E0,9E0,5E0,8E0,7E0,2.2E1,1.2E1,2.6E1,5E0,8E0,3.5E1,5E0,7E0,1.9E1,1.2E1,2.3E1,8E0,1.1E1,6E0,6E0,1.6E1,7E0,6E0,5E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[7.646503E-3,-5.0967935E-2,1.00483015E-1,-1.1947661E-2,-3.7877686E-2,7.812532E-2,1.4226461E-2,3.4871208E-3,-4.4498987E-2,8.87943E-2,-1.8150723E-3,-6.631444E-2,-1.8023483E-2,7.004954E-2,1.0729385E-2,-7.405189E-2,-1.0514867E-3,-3.17356E-2,2.4614818E-2,3.812655E-2,9.735544E-2,-8.589728E-2,-4.384491E-2,-4.2061776E-2,7.4431545E-4,2.062225E-3,7.4140386E-5,-4.7545397E-4,5.5828117E-2,6.641499E-2,1.25762E-1,-9.744021E-2,-1.9890778E-3,-1.430044E-4,-2.7847402E-3,-2.66921E-2,-4.325052E-3,8.346559E-2,-3.239595E-4,3.4752474E-3,1.3588729E-3,6.4085913E-3,3.1950688E-3,-2.6426348E-3,-4.760593E-3,-4.684207E-2,3.2116866E-3,2.036861E-3,5.445079E-3,-7.274505E-4,-2.6302978E-3,5.3518516E-4,-2.781881E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,-1,7,9,-1,-1,11,13,-1,15,17,19,-1,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1],"loss_changes":[9.846023E-1,3.133151E-1,3.2739097E-1,0E0,8.621411E-2,8.670995E-2,0E0,0E0,5.713986E-2,1.5938738E-1,0E0,1.6184747E-2,2.8260246E-2,4.423091E-2,0E0,1.2693286E-2,0E0,1.8854111E-2,6.1437055E-3,2.4036746E-2,1.7135948E-2,1.045455E-2,1.193512E-2,2.3651022E-2,0E0,0E0,0E0,0E0,3.558113E-2,4.976459E-3,4.7528595E-3,3.6945194E-3,0E0,0E0,0E0,1.5214451E-2,0E0,1.5448712E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.0703734E-3,1.0323673E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,28,28,29,29,30,30,31,31,35,35,37,37,45,45,46,46],"right_children":[2,4,6,-1,8,10,-1,-1,12,14,-1,16,18,20,-1,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1],"split_conditions":[9.0582E4,3.7E1,4.766891E9,-1.1947661E-2,2.5E1,4.312765E7,1.4226461E-2,3.4871208E-3,6.247E3,9.0573E4,-1.8150723E-3,6.88588E5,3.9692026E8,6.820745E6,1.0729385E-2,8.3E1,-1.0514867E-3,1.733293E6,6.6653E4,1E0,8.7165575E10,8.330361E8,1.530688E9,7.98E2,7.4431545E-4,2.062225E-3,7.4140386E-5,-4.7545397E-4,1.8787456E7,7.1999734E9,1.7536711E11,4.12445E5,-1.9890778E-3,-1.430044E-4,-2.7847402E-3,1.7681605E8,-4.325052E-3,6.588871E6,-3.239595E-4,3.4752474E-3,1.3588729E-3,6.4085913E-3,3.1950688E-3,-2.6426348E-3,-4.760593E-3,3.0021282E3,1.28E2,2.036861E-3,5.445079E-3,-7.274505E-4,-2.6302978E-3,5.3518516E-4,-2.781881E-4],"split_indices":[12,4,16,0,20,6,0,0,3,3,0,6,21,6,0,17,0,6,12,29,10,11,10,0,0,0,0,0,18,11,10,2,0,0,0,16,0,18,0,0,0,0,0,0,0,5,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,1.1E2,6.9E1,5E0,1.05E2,6.4E1,5E0,5E0,1E2,5.9E1,5E0,5.4E1,4.6E1,5.4E1,5E0,4.5E1,9E0,3.5E1,1.1E1,2.6E1,2.8E1,3.1E1,1.4E1,2.9E1,6E0,5E0,6E0,7E0,1.9E1,1.5E1,1.3E1,2.3E1,8E0,5E0,9E0,2.4E1,5E0,1.3E1,6E0,1E1,5E0,8E0,5E0,7E0,1.6E1,1.4E1,1E1,8E0,5E0,5E0,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.0866184E-2,-6.302272E-2,4.7410253E-2,-1.1714365E-2,-4.894422E-2,3.5277646E-2,7.9013575E-3,-6.3385844E-2,1.3948331E-2,5.080286E-2,-7.3001515E-3,-6.779874E-2,-5.8387865E-5,-2.5946848E-2,2.926991E-3,4.1050687E-2,6.157141E-3,-6.0997017E-2,-4.9271896E-3,-2.3225513E-3,2.463013E-4,4.5896256E-3,5.454423E-2,-6.528815E-2,-4.0809598E-4,2.6279699E-2,-2.1998265E-3,6.160435E-2,3.8826207E-4,-5.275121E-2,-8.486507E-2,2.805098E-3,-2.7002968E-4,4.0270437E-2,9.018277E-2,-6.635358E-2,-3.3951152E-2,-1.043884E-1,-1.3489424E-3,6.450473E-2,9.210068E-3,5.4054046E-3,5.7503507E-2,-3.500182E-3,-1.8816539E-3,-7.231303E-5,-4.9960274E-2,-2.862919E-3,-5.3670057E-3,3.8392525E-3,1.0968882E-3,-7.7617983E-4,1.2353704E-3,1.3724306E-3,3.228963E-3,-3.1061E-3,-1.0758208E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,-1,7,9,-1,11,13,15,-1,17,-1,19,-1,21,-1,23,-1,-1,-1,25,27,29,-1,31,-1,33,-1,35,37,-1,-1,39,41,43,45,47,-1,49,51,-1,53,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5921596E-1,2.6731548E-1,1.346532E-1,0E0,8.444503E-2,2.6319E-1,0E0,2.0445168E-2,3.9880678E-2,6.1109737E-2,0E0,1.4581859E-2,0E0,9.774702E-3,0E0,3.418889E-2,0E0,1.3351887E-2,0E0,0E0,0E0,2.5179304E-2,1.5888557E-2,9.758487E-3,0E0,1.8897722E-2,0E0,2.3194656E-2,0E0,7.4647814E-3,1.8896535E-2,0E0,0E0,1.9729506E-2,1.2678906E-2,3.2978505E-3,9.223314E-3,2.7443469E-3,0E0,1.1643227E-2,7.2035193E-3,0E0,2.3966432E-3,0E0,0E0,0E0,4.6865065E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,11,11,13,13,15,15,17,17,21,21,22,22,23,23,25,25,27,27,29,29,30,30,33,33,34,34,35,35,36,36,37,37,39,39,40,40,42,42,46,46],"right_children":[2,4,6,-1,8,10,-1,12,14,16,-1,18,-1,20,-1,22,-1,24,-1,-1,-1,26,28,30,-1,32,-1,34,-1,36,38,-1,-1,40,42,44,46,48,-1,50,52,-1,54,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.09274E6,1.402264E-6,1.6202324E11,-1.1714365E-2,3.9692026E8,4.2250056E7,7.9013575E-3,8.888E3,2.7616E4,1.85215E5,-7.3001515E-3,1.739898E6,-5.8387865E-5,2.3188192E8,2.926991E-3,1.272E3,6.157141E-3,6.69829E5,-4.9271896E-3,-2.3225513E-3,2.463013E-4,1.8650388E10,1E0,2.202E3,-4.0809598E-4,6.2952E4,-2.1998265E-3,2.518512E6,3.8826207E-4,2.49E2,1.4331645E8,2.805098E-3,-2.7002968E-4,2.32E2,4.1265E4,4.1945936E7,7.441345E-4,1.3390451E-2,-1.3489424E-3,3.014154E-1,2.051E3,5.4054046E-3,6.4184E4,-3.500182E-3,-1.8816539E-3,-7.231303E-5,5.67E2,-2.862919E-3,-5.3670057E-3,3.8392525E-3,1.0968882E-3,-7.7617983E-4,1.2353704E-3,1.3724306E-3,3.228963E-3,-3.1061E-3,-1.0758208E-3],"split_indices":[6,62,1,0,21,6,0,3,7,20,0,2,0,11,0,0,0,6,0,0,0,10,35,20,0,28,0,18,0,0,21,0,0,4,20,16,62,62,0,62,0,0,20,0,0,0,20,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.82E2,9.6E1,8.6E1,5E0,9.1E1,8E1,6E0,7.4E1,1.7E1,7.5E1,5E0,6.9E1,5E0,1E1,7E0,6.9E1,6E0,6.2E1,7E0,5E0,5E0,1.9E1,5E1,5.7E1,5E0,1.4E1,5E0,4.3E1,7E0,3.7E1,2E1,6E0,8E0,2.6E1,1.7E1,2E1,1.7E1,1.4E1,6E0,1.4E1,1.2E1,7E0,1E1,1.1E1,9E0,6E0,1.1E1,6E0,8E0,8E0,6E0,5E0,7E0,5E0,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.1704591E-3,-2.5448015E-2,1.3041839E-1,-1.13304695E-2,-1.4529521E-2,8.3785415E-2,8.990358E-3,-4.8557434E-2,1.4910369E-2,5.197714E-3,1.7505389E-3,1.9926087E-3,-5.77843E-2,-1.1441745E-2,3.9435063E-2,-7.258013E-2,-2.7522802E-2,-2.6700615E-3,3.1761355E-3,5.86262E-3,1.2818119E-2,-9.511588E-4,-8.17071E-2,1.1597355E-3,-4.732541E-2,-1.1124028E-2,2.6287662E-2,-1.5646421E-3,3.5381936E-2,-9.008366E-2,-1.7510435E-3,-2.5284574E-3,-9.499251E-4,3.1887015E-4,-1.1609363E-3,-6.772802E-5,2.1712498E-3,3.130036E-3,7.5082183E-3,-4.312406E-3,-2.0081222E-3,1.324904E-3,-7.162986E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,-1,7,9,-1,11,13,-1,-1,-1,15,17,19,21,23,-1,25,-1,27,-1,29,-1,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,41,-1,-1,-1,-1],"loss_changes":[5.641013E-1,3.4391484E-1,8.1403464E-2,0E0,1.3195695E-1,2.451916E-2,0E0,5.5663243E-2,4.6336908E-2,0E0,0E0,0E0,2.3076072E-2,2.5638659E-2,9.172274E-2,1.5370414E-2,2.268138E-2,0E0,9.546386E-3,0E0,3.4210965E-2,0E0,7.185474E-3,0E0,2.805192E-3,5.3276895E-3,7.2608455E-3,0E0,2.072768E-2,5.060941E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.602439E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,12,12,13,13,14,14,15,15,16,16,18,18,20,20,22,22,24,24,25,25,26,26,28,28,29,29,38,38],"right_children":[2,4,6,-1,8,10,-1,12,14,-1,-1,-1,16,18,20,22,24,-1,26,-1,28,-1,30,-1,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,42,-1,-1,-1,-1],"split_conditions":[6.820745E6,2.1569896E7,3.2189102E10,-1.13304695E-2,6.35E3,3.4025794E-1,8.990358E-3,1E0,1.03E2,5.197714E-3,1.7505389E-3,1.9926087E-3,1.15856E6,1.414E3,1.8214622E-3,1.93E3,1.516891E6,-2.6700615E-3,3.1281E4,5.86262E-3,3.9692026E8,-9.511588E-4,2.9435686E3,1.1597355E-3,1.348232E6,1.712E3,2.5965E4,-1.5646421E-3,1.1482625E7,1.0254491E8,-1.7510435E-3,-2.5284574E-3,-9.499251E-4,3.1887015E-4,-1.1609363E-3,-6.772802E-5,2.1712498E-3,3.130036E-3,6.1E1,-4.312406E-3,-2.0081222E-3,1.324904E-3,-7.162986E-4],"split_indices":[6,11,1,0,3,65,0,44,17,0,0,0,2,22,63,28,2,0,20,0,21,0,5,0,6,20,7,0,18,21,0,0,0,0,0,0,0,0,19,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.62E2,1.35E2,2.7E1,5E0,1.3E2,1.8E1,9E0,6E1,7E1,9E0,9E0,5E0,5.5E1,3.4E1,3.6E1,3.6E1,1.9E1,7E0,2.7E1,7E0,2.9E1,6E0,3E1,5E0,1.4E1,1.7E1,1E1,9E0,2E1,2.4E1,6E0,9E0,5E0,8E0,9E0,5E0,5E0,8E0,1.2E1,1.9E1,5E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.0716244E-2,-1.4209182E-2,1.6081531E-1,-3.9865796E-2,5.9352595E-2,1.2404372E-2,1.0040069E-1,-5.4610685E-2,-5.172831E-3,8.263598E-3,3.6041293E-2,1.2730385E-1,9.119245E-4,-5.8370642E-2,1.19762095E-4,3.8903095E-3,-2.325839E-2,-1.6677236E-2,8.408825E-2,3.850796E-3,7.1887877E-3,-8.279067E-2,-4.6436355E-2,-4.607889E-2,-6.4202365E-3,3.6758255E-2,-4.6221423E-3,1.5023694E-3,1.1708073E-1,-1.4472423E-3,-9.749281E-2,9.777606E-4,-5.391781E-2,-2.5816723E-3,-9.1537414E-4,8.1635464E-4,-2.6392449E-2,2.5940281E-3,5.269211E-4,3.1460137E-3,6.2227105E-3,-2.6948622E-3,-5.0032544E-3,-7.494924E-4,-5.7703238E-2,-3.3423098E-4,-1.7776756E-3,-3.3092175E-3,-4.464422E-2,-2.714988E-3,-3.305284E-2,-4.2696E-4,-4.9087703E-2,-1.2032203E-3,-2.724746E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,-1,21,-1,-1,23,25,27,-1,-1,29,31,33,35,37,-1,-1,39,-1,41,-1,43,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,47,-1,-1,-1,49,-1,51,-1,53,-1,-1],"loss_changes":[6.831985E-1,2.9867947E-1,1.652736E-1,5.9568137E-2,1.17402405E-1,0E0,3.7981257E-2,1.7843515E-2,6.347264E-2,0E0,9.2589095E-2,5.929664E-3,0E0,1.8596083E-2,0E0,0E0,1.1616072E-2,8.9709096E-2,2.7104259E-2,0E0,0E0,1.4280304E-2,2.901829E-2,3.0254219E-3,1.0027427E-2,5.9992075E-3,0E0,0E0,2.341196E-3,0E0,3.9505064E-3,0E0,5.824998E-3,0E0,0E0,0E0,2.6425468E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.0237964E-3,0E0,0E0,0E0,3.6429912E-3,0E0,6.6095274E-3,0E0,1.4368277E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,13,13,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,28,28,30,30,32,32,36,36,44,44,48,48,50,50,52,52],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,-1,22,-1,-1,24,26,28,-1,-1,30,32,34,36,38,-1,-1,40,-1,42,-1,44,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,48,-1,-1,-1,50,-1,52,-1,54,-1,-1],"split_conditions":[1.5522031E9,5.166426E6,1E0,1.914061E6,1.467E3,1.2404372E-2,3.90786E0,8.008664E8,1E0,8.263598E-3,5.8815314E9,8.982518E9,9.119245E-4,1.32E2,1.19762095E-4,3.8903095E-3,6.523E3,6.800478E10,1.3746902E7,3.850796E-3,7.1887877E-3,3.9895258E8,1.47E2,1.348232E6,7.1E1,1.79E3,-4.6221423E-3,1.5023694E-3,3.5E1,-1.4472423E-3,1.3390451E-2,9.777606E-4,6.7739E4,-2.5816723E-3,-9.1537414E-4,8.1635464E-4,3.073658E6,2.5940281E-3,5.269211E-4,3.1460137E-3,6.2227105E-3,-2.6948622E-3,-5.0032544E-3,-7.494924E-4,4.985E3,-3.3423098E-4,-1.7776756E-3,-3.3092175E-3,2.63E2,-2.714988E-3,6.1894E4,-4.2696E-4,2.202E3,-1.2032203E-3,-2.724746E-3],"split_indices":[16,2,42,9,0,0,59,21,44,0,11,11,0,4,0,0,3,10,2,0,0,10,4,6,19,0,0,0,17,0,62,0,6,0,0,0,2,0,0,0,0,0,0,0,3,0,0,0,0,0,8,0,20,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.81E2,1.56E2,2.5E1,1.16E2,4E1,7E0,1.8E1,8.1E1,3.5E1,5E0,3.5E1,1.3E1,5E0,7.6E1,5E0,5E0,3E1,1.7E1,1.8E1,8E0,5E0,2.3E1,5.3E1,1.2E1,1.8E1,1.1E1,6E0,8E0,1E1,6E0,1.7E1,5E0,4.8E1,7E0,5E0,8E0,1E1,5E0,6E0,5E0,5E0,7E0,1E1,5E0,4.3E1,5E0,5E0,1.6E1,2.7E1,9E0,1.8E1,8E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-9.933938E-4,-2.707718E-2,6.162756E-2,-5.61464E-2,-2.4425695E-3,3.7340254E-2,1.019204E-2,-6.420299E-2,-5.879879E-4,4.4849943E-2,-2.7108142E-2,7.0043795E-2,-8.157903E-3,-3.1195896E-2,-7.368533E-2,1.9095307E-2,3.3355602E-3,-5.0933793E-2,-6.9632954E-3,4.3608997E-2,6.819899E-3,-2.4071755E-3,-2.708864E-4,-1.482241E-3,-7.917426E-2,1.7921143E-3,-1.0956014E-4,-7.3006816E-2,-9.410314E-4,9.436276E-3,-2.6392413E-3,5.453679E-2,-5.601747E-4,-3.8639395E-3,-1.8500962E-3,-4.0394925E-3,-1.8024662E-3,-7.9245446E-4,2.9344143E-2,2.4165667E-5,6.6780515E-2,1.7466632E-3,5.237891E-4,4.9314465E-2,3.979805E-3,2.6669672E-3,1.1833804E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,-1,27,29,31,-1,-1,-1,-1,33,-1,-1,35,-1,37,-1,39,-1,-1,-1,-1,-1,-1,41,-1,43,-1,-1,45,-1,-1,-1],"loss_changes":[2.7771187E-1,8.579592E-2,2.0001006E-1,1.8044874E-2,7.8204595E-2,3.4310687E-1,0E0,1.209861E-2,0E0,1.7242294E-2,2.0763755E-2,8.434141E-2,0E0,6.8412814E-3,4.943028E-3,6.7466698E-3,0E0,1.139329E-2,2.2506941E-2,2.0437535E-2,0E0,0E0,0E0,0E0,6.5693855E-3,0E0,0E0,2.9947162E-3,0E0,1.1518849E-2,0E0,1.7808795E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.7412342E-3,0E0,4.879698E-3,0E0,0E0,2.2693276E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,17,17,18,18,19,19,24,24,27,27,29,29,31,31,38,38,40,40,43,43],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,-1,28,30,32,-1,-1,-1,-1,34,-1,-1,36,-1,38,-1,40,-1,-1,-1,-1,-1,-1,42,-1,44,-1,-1,46,-1,-1,-1],"split_conditions":[1.0431E4,1.13976E6,1.3336757E0,6.024596E8,1.2192E4,4.2250056E7,1.019204E-2,1.9012777E-3,-5.879879E-4,2.341617E6,2.856E4,3.41893E5,-8.157903E-3,2.37E2,2.731002E7,2.71E2,3.3355602E-3,7.041E3,2.0803306E9,2.0901188E7,6.819899E-3,-2.4071755E-3,-2.708864E-4,-1.482241E-3,2.95E2,1.7921143E-3,-1.0956014E-4,2.695687E9,-9.410314E-4,1.2E1,-2.6392413E-3,5.398E3,-5.601747E-4,-3.8639395E-3,-1.8500962E-3,-4.0394925E-3,-1.8024662E-3,-7.9245446E-4,2.48E2,2.4165667E-5,4.949877E6,1.7466632E-3,5.237891E-4,5.8E1,3.979805E-3,2.6669672E-3,1.1833804E-3],"split_indices":[3,2,63,26,22,6,0,65,0,2,7,28,0,0,11,20,0,3,21,18,0,0,0,0,4,0,0,25,0,19,0,28,0,0,0,0,0,0,19,0,18,0,0,17,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,1.19E2,4.9E1,5.4E1,6.5E1,4.4E1,5E0,4.5E1,9E0,2.2E1,4.3E1,3.9E1,5E0,1.1E1,3.4E1,1.3E1,9E0,1.9E1,2.4E1,3.1E1,8E0,5E0,6E0,5E0,2.9E1,6E0,7E0,1E1,9E0,1.9E1,5E0,2.6E1,5E0,2.2E1,7E0,5E0,5E0,8E0,1.1E1,5E0,2.1E1,6E0,5E0,1.4E1,7E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[8.763443E-3,-5.154379E-3,1.05497455E-2,-4.6557173E-2,2.7150474E-2,-5.4946396E-2,2.5174848E-3,1.4355871E-2,6.440302E-2,-6.068595E-2,-6.553331E-4,4.05884E-3,5.7980996E-3,7.930485E-2,2.8283414E-4,-4.414091E-3,-5.057415E-2,-2.5722135E-3,1.5742505E-2,1.8461007E-3,8.9501046E-2,-5.5066593E-2,-3.688952E-4,3.3865396E-2,-1.1343787E-3,2.6618827E-3,5.1222146E-3,-7.1067765E-2,-4.4697084E-2,8.621677E-3,5.5168506E-2,2.1853063E-2,-1.4870126E-2,-3.5651524E-3,-1.735591E-3,-7.069697E-4,-5.6139894E-2,1.860633E-3,-9.69512E-4,6.7381025E-4,3.4891511E-3,2.5105456E-4,1.6147409E-3,-2.326205E-3,1.4915536E-3,-2.8282965E-3,-1.2903656E-3,1.163847E-3,-6.006556E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,-1,5,7,9,-1,11,13,15,-1,-1,17,19,-1,-1,21,-1,23,-1,25,27,-1,29,31,-1,-1,33,35,37,39,41,43,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,47,-1,-1,-1,-1],"loss_changes":[5.750905E-1,2.2596593E-1,0E0,6.787892E-2,4.435543E-2,1.4903069E-2,0E0,4.791358E-2,2.0201907E-2,1.942417E-2,0E0,0E0,4.3412298E-2,3.3840463E-3,0E0,0E0,8.996204E-3,0E0,1.7778173E-2,0E0,3.9624497E-3,4.290059E-3,0E0,1.4581911E-2,1.0113197E-2,0E0,0E0,2.0680055E-3,8.874148E-3,1.562727E-2,1.352898E-2,2.712932E-3,1.2653721E-2,0E0,0E0,0E0,2.917178E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1266185E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,12,12,13,13,16,16,18,18,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,36,36,44,44],"right_children":[2,4,-1,6,8,10,-1,12,14,16,-1,-1,18,20,-1,-1,22,-1,24,-1,26,28,-1,30,32,-1,-1,34,36,38,40,42,44,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,48,-1,-1,-1,-1],"split_conditions":[8.504903E3,1.882904E6,1.05497455E-2,1.4817759E9,6.820745E6,1.4771324E9,2.5174848E-3,8.44966E5,4.114E3,9.6E1,-6.553331E-4,4.05884E-3,1.070527E6,4.1E1,2.8283414E-4,-4.414091E-3,4.13822E8,-2.5722135E-3,1.44E2,1.8461007E-3,4.408E3,9.7786E4,-3.688952E-4,6.490565E8,1.85221E5,2.6618827E-3,5.1222146E-3,3.33044E5,1.3555108E-3,5.1269E4,4.1E1,7.4659E4,5.8334584E7,-3.5651524E-3,-1.735591E-3,-7.069697E-4,2.61E2,1.860633E-3,-9.69512E-4,6.7381025E-4,3.4891511E-3,2.5105456E-4,1.6147409E-3,-2.326205E-3,4.9676266E8,-2.8282965E-3,-1.2903656E-3,1.163847E-3,-6.006556E-4],"split_indices":[5,2,0,16,6,25,0,6,19,4,0,0,6,17,0,0,23,0,4,0,0,18,0,16,18,0,0,6,63,22,4,12,16,0,0,0,4,0,0,0,0,0,0,0,16,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,1.67E2,9E0,7.3E1,9.4E1,6.8E1,5E0,7.1E1,2.3E1,5.9E1,9E0,6E0,6.5E1,1.8E1,5E0,1E1,4.9E1,8E0,5.7E1,5E0,1.3E1,4.4E1,5E0,2.7E1,3E1,8E0,5E0,1.5E1,2.9E1,1.3E1,1.4E1,1.1E1,1.9E1,1E1,5E0,9E0,2E1,6E0,7E0,6E0,8E0,6E0,5E0,5E0,1.4E1,1.4E1,6E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.2135167E-3,-2.191082E-2,1.2364191E-1,-1.0609203E-2,-1.2136483E-2,8.331247E-2,1.0951429E-2,-2.9641384E-2,6.817133E-2,5.9305253E-3,3.6471102E-2,-2.208009E-2,-7.0736245E-3,9.466201E-4,8.170875E-2,3.4050518E-3,3.296972E-5,-3.289405E-2,7.3781563E-3,4.2617405E-3,2.077436E-3,-4.762755E-2,-1.1595561E-3,-7.249158E-2,-3.5734825E-2,1.7639453E-2,-3.7094362E-2,-4.0037055E-3,-5.2116986E-2,-6.173654E-2,-2.6818022E-2,3.052145E-2,-7.055665E-4,-2.3822233E-3,-7.336165E-4,-1.0316073E-3,-3.192921E-3,-1.2935122E-3,-3.5106065E-3,-1.2302451E-2,-5.490282E-2,-2.8284749E-5,1.9013794E-3,-2.3216147E-3,-5.918801E-4,-3.4307446E-3,-1.3209924E-3,1.3867405E-3,-1.47139905E-2,-1.2060181E-3,-6.970902E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,-1,7,9,-1,11,13,-1,15,17,-1,-1,19,-1,-1,21,-1,-1,-1,23,25,27,29,31,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,43,45,-1,-1,-1,47,-1,-1,-1,49,-1,-1],"loss_changes":[4.8116082E-1,3.0616915E-1,1.1767715E-1,0E0,1.9601838E-1,4.9210712E-2,0E0,1.10568844E-1,1.39594525E-2,0E0,1.9229418E-2,2.3084E-1,0E0,0E0,5.679056E-3,0E0,0E0,4.8571028E-2,0E0,0E0,0E0,1.840432E-2,2.3622641E-2,3.8425103E-3,9.936959E-3,1.0509268E-2,3.3391304E-3,0E0,6.0599893E-3,4.8179626E-3,1.522357E-2,6.9385674E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2896818E-2,5.2116625E-3,0E0,0E0,0E0,1.0522441E-2,0E0,0E0,0E0,2.666812E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,10,10,11,11,14,14,17,17,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,39,39,40,40,44,44,48,48],"right_children":[2,4,6,-1,8,10,-1,12,14,-1,16,18,-1,-1,20,-1,-1,22,-1,-1,-1,24,26,28,30,32,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,44,46,-1,-1,-1,48,-1,-1,-1,50,-1,-1],"split_conditions":[1.5063692E9,1.66E2,1.11464E5,-1.0609203E-2,3.541E3,1.6010006E10,1.0951429E-2,1.94591E5,5.625966E6,5.9305253E-3,1E0,1.887E4,-7.0736245E-3,9.466201E-4,4.74794E5,3.4050518E-3,3.296972E-5,3.6754374E8,7.3781563E-3,4.2617405E-3,2.077436E-3,5.131E3,4.3822035E-1,1.8033E4,8.01998E5,3.2744336E3,2.64684E9,-4.0037055E-3,6.1513E4,1.6187E4,2.13E2,2.090253E3,-7.055665E-4,-2.3822233E-3,-7.336165E-4,-1.0316073E-3,-3.192921E-3,-1.2935122E-3,-3.5106065E-3,5.5219E4,1.4941494E9,-2.8284749E-5,1.9013794E-3,-2.3216147E-3,9.248437E7,-3.4307446E-3,-1.3209924E-3,1.3867405E-3,4.55E2,-1.2060181E-3,-6.970902E-5],"split_indices":[16,0,3,0,0,10,0,28,6,0,29,3,0,0,12,0,0,21,0,0,0,3,62,7,18,5,10,0,8,28,17,5,0,0,0,0,0,0,0,27,10,0,0,0,26,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,1.42E2,2.6E1,5E0,1.37E2,2.1E1,5E0,1.13E2,2.4E1,9E0,1.2E1,1.08E2,5E0,6E0,1.8E1,5E0,7E0,1.03E2,5E0,1.1E1,7E0,7E1,3.3E1,2.1E1,4.9E1,2.2E1,1.1E1,9E0,1.2E1,1.1E1,3.8E1,1.6E1,6E0,5E0,6E0,6E0,6E0,5E0,6E0,2.6E1,1.2E1,5E0,1.1E1,5E0,2.1E1,5E0,7E0,6E0,1.5E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[7.3986966E-4,-4.6996213E-2,4.051223E-2,-1.1563905E-2,-2.8933931E-2,5.002974E-2,-4.8268563E-3,-3.764258E-2,3.3821687E-3,3.553333E-2,9.554767E-3,-4.459351E-2,-3.3028176E-4,1.9260844E-2,7.171856E-2,-6.526333E-4,-4.829609E-2,1.3058742E-3,-1.1440335E-3,5.5714753E-3,7.973426E-2,3.4703975E-4,8.6522676E-2,-6.77991E-2,-3.3987984E-2,1.9908643E-3,-9.25382E-4,5.010917E-3,1.3693671E-3,1.8683829E-3,1.0115526E-1,-3.7407302E-3,-1.5015743E-3,-4.3029662E-2,-2.6626373E-4,-1.0273612E-2,1.5961834E-3,5.5774143E-3,2.7009274E-3,-2.2040799E-3,-7.716081E-4,5.2065216E-4,-2.4035086E-3,-1.1684132E-3,1.3491246E-2,2.4503116E-2,-3.2586488E-4,-1.9842307E-6,1.664967E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,-1,7,9,-1,11,-1,13,-1,15,17,19,21,-1,23,-1,-1,25,27,-1,29,31,33,-1,35,-1,-1,-1,37,-1,-1,39,-1,41,-1,-1,-1,-1,-1,43,-1,-1,45,47,-1,-1,-1],"loss_changes":[3.3415484E-1,3.0726212E-1,1.4359075E-1,0E0,7.242707E-2,2.150043E-1,0E0,1.8140137E-2,0E0,4.8668653E-2,0E0,5.7529807E-3,1.01787355E-2,4.922633E-2,2.3856625E-2,0E0,1.2206301E-2,0E0,0E0,1.3049394E-2,1.5054941E-2,0E0,9.103075E-3,9.608887E-3,7.8063197E-3,0E0,1.5798535E-2,0E0,0E0,0E0,7.780254E-3,0E0,0E0,3.6512055E-3,0E0,1.7435122E-2,0E0,0E0,0E0,0E0,0E0,1.09803025E-2,0E0,0E0,5.02927E-3,4.6906127E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,9,9,11,11,12,12,13,13,14,14,16,16,19,19,20,20,22,22,23,23,24,24,26,26,30,30,33,33,35,35,41,41,44,44,45,45],"right_children":[2,4,6,-1,8,10,-1,12,-1,14,-1,16,18,20,22,-1,24,-1,-1,26,28,-1,30,32,34,-1,36,-1,-1,-1,38,-1,-1,40,-1,42,-1,-1,-1,-1,-1,44,-1,-1,46,48,-1,-1,-1],"split_conditions":[6.88588E5,4.5E1,4.2250056E7,-1.1563905E-2,6.83E2,9.0573E4,-4.8268563E-3,6.7842746E8,3.3821687E-3,3.016E3,9.554767E-3,3.035E3,1.96E2,1.3957454E9,1.6107015E3,-6.526333E-4,4.985E3,1.3058742E-3,-1.1440335E-3,1.617E3,1.8345702E9,3.4703975E-4,8.098088E10,8.3E1,5.55064E8,1.9908643E-3,1.7261E4,5.010917E-3,1.3693671E-3,1.8683829E-3,1E0,-3.7407302E-3,-1.5015743E-3,9.6E1,-2.6626373E-4,1.31729E5,1.5961834E-3,5.5774143E-3,2.7009274E-3,-2.2040799E-3,-7.716081E-4,6.523E3,-2.4035086E-3,-1.1684132E-3,2.2672263E10,2.1955977E3,-3.2586488E-4,-1.9842307E-6,1.664967E-3],"split_indices":[6,4,6,0,17,3,0,16,0,0,0,12,4,16,5,0,3,0,0,22,16,0,10,17,11,0,3,0,0,0,43,0,0,17,0,12,0,0,0,0,0,3,0,0,10,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,7.9E1,9.5E1,5E0,7.4E1,9E1,5E0,6.9E1,5E0,8.4E1,6E0,5.8E1,1.1E1,5.9E1,2.5E1,7E0,5.1E1,5E0,6E0,4.9E1,1E1,5E0,2E1,2E1,3.1E1,6E0,4.3E1,5E0,5E0,6E0,1.4E1,1.2E1,8E0,2.3E1,8E0,3.5E1,8E0,7E0,7E0,1.7E1,6E0,2.9E1,6E0,9E0,2E1,1.3E1,7E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-3.8915413E-4,-1.0888496E-2,1.4420864E-1,-5.827282E-3,-6.8023265E-3,1.0170791E-2,2.600084E-3,-2.8294068E-2,3.32296E-2,-3.3691928E-2,2.4754722E-3,4.0150475E-2,-1.7617622E-3,-4.121669E-2,2.5368242E-3,4.0921182E-3,3.1131497E-2,-4.6448164E-2,-7.864603E-3,2.4727136E-2,-1.0853477E-3,-1.5854781E-3,3.90806E-2,-5.6838755E-2,-2.8967649E-2,6.7024137E-4,-1.3077348E-3,-1.4975225E-6,1.9801469E-3,2.7029019E-2,7.606497E-2,-1.088581E-3,-6.3923694E-2,-5.8692778E-3,-6.8331264E-2,1.2154594E-2,5.7004523E-2,4.530457E-3,1.5562129E-3,-3.813773E-3,-5.3264227E-2,-1.8265676E-2,1.1391817E-3,-3.925422E-3,-1.5424014E-3,2.1050544E-2,-9.168907E-4,3.3861578E-3,1.025626E-3,-8.4244745E-4,-6.164947E-2,-1.8346949E-3,-9.50439E-5,2.7637659E-3,3.7423637E-2,-2.9869827E-3,-1.6294413E-3,9.709903E-4,-7.4983545E-4,2.200634E-3,7.9398113E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,15,-1,17,19,-1,21,23,25,27,-1,-1,29,31,33,-1,-1,-1,-1,35,37,-1,39,41,43,45,47,-1,-1,-1,49,51,-1,-1,-1,53,-1,-1,-1,-1,55,-1,-1,57,59,-1,-1,-1,-1,-1,-1],"loss_changes":[2.915429E-1,1.3041979E-1,8.384159E-2,1.5362218E-1,0E0,0E0,0E0,5.284562E-2,3.4217864E-2,2.8728895E-2,0E0,2.5605313E-2,0E0,1.4814526E-2,1.216857E-2,0E0,2.9303022E-2,1.1975974E-2,7.1304813E-3,5.6234645E-3,0E0,0E0,1.8892884E-2,8.801535E-3,2.6876012E-2,0E0,0E0,0E0,0E0,1.5804261E-2,8.626789E-3,0E0,4.5962036E-3,8.402795E-3,4.6626553E-3,8.112309E-3,6.4501576E-3,0E0,0E0,0E0,6.7841113E-3,5.6445E-3,0E0,0E0,0E0,6.1842036E-3,0E0,0E0,0E0,0E0,1.0753274E-3,0E0,0E0,4.6562385E-3,1.8321127E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,11,11,13,13,14,14,16,16,17,17,18,18,19,19,22,22,23,23,24,24,29,29,30,30,32,32,33,33,34,34,35,35,36,36,40,40,41,41,45,45,50,50,53,53,54,54],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,16,-1,18,20,-1,22,24,26,28,-1,-1,30,32,34,-1,-1,-1,-1,36,38,-1,40,42,44,46,48,-1,-1,-1,50,52,-1,-1,-1,54,-1,-1,-1,-1,56,-1,-1,58,60,-1,-1,-1,-1,-1,-1],"split_conditions":[7.37159E5,8.355255E3,1.7536711E11,1.154E3,-6.8023265E-3,1.0170791E-2,2.600084E-3,1.4817759E9,1E0,8.008664E8,2.4754722E-3,3.383443E7,-1.7617622E-3,1.3924256E9,6.1E1,4.0921182E-3,1.9535E4,1.3958365E8,2.0521914E10,2.3944454E8,-1.0853477E-3,-1.5854781E-3,2.378739E10,3.035E3,1.7997152E7,6.7024137E-4,-1.3077348E-3,-1.4975225E-6,1.9801469E-3,9.2421E4,3.44675E5,-1.088581E-3,2.6E1,6.501315E8,1.3820372E9,8.5565235E8,2.48E2,4.530457E-3,1.5562129E-3,-3.813773E-3,3.446673E7,1.5157233E3,1.1391817E-3,-3.925422E-3,-1.5424014E-3,8.11246E5,-9.168907E-4,3.3861578E-3,1.025626E-3,-8.4244745E-4,4.734E3,-1.8346949E-3,-9.50439E-5,5.6E1,7.4256E4,-2.9869827E-3,-1.6294413E-3,9.709903E-4,-7.4983545E-4,2.200634E-3,7.9398113E-4],"split_indices":[12,5,10,0,0,0,0,16,32,21,0,16,0,11,19,0,7,16,10,16,0,0,26,12,18,0,0,0,0,8,12,0,17,16,10,16,19,0,0,0,16,5,0,0,0,18,0,0,0,0,20,0,0,19,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.9E2,1.78E2,1.2E1,1.73E2,5E0,5E0,7E0,1.1E2,6.3E1,1.04E2,6E0,5.8E1,5E0,8.6E1,1.8E1,7E0,5.1E1,7.4E1,1.2E1,1E1,8E0,5E0,4.6E1,4.5E1,2.9E1,6E0,6E0,5E0,5E0,3.6E1,1E1,9E0,3.6E1,1.9E1,1E1,2.5E1,1.1E1,5E0,5E0,9E0,2.7E1,1.4E1,5E0,5E0,5E0,2E1,5E0,6E0,5E0,6E0,2.1E1,5E0,9E0,1E1,1E1,1.5E1,6E0,5E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-4.2694947E-4,-9.853536E-3,7.3010437E-3,-4.6790484E-3,1.3540062E-1,6.215467E-3,-1.030607E-2,2.5727618E-3,8.508151E-3,-4.3946546E-2,1.2955184E-2,-5.1208418E-2,1.6952117E-5,2.0285483E-2,-4.2213155E-3,-6.26501E-2,-3.2565635E-2,8.110452E-3,5.5106E-3,-3.186063E-3,-3.532172E-2,-4.342116E-2,-6.1591633E-4,-3.2113683E-3,3.0453708E-2,-2.3580042E-3,-4.6841693E-4,-1.2288935E-3,-2.3859502E-3,1.2135504E-2,-2.4423914E-2,-1.54001E-3,7.057419E-2,-3.386708E-3,3.508233E-2,-1.6233174E-3,-2.352672E-3,1.0312666E-3,-1.0361779E-3,4.18018E-3,1.697582E-3,-1.3885804E-2,7.898042E-4,8.4753986E-4,1.9961833E-3,-7.635018E-4,9.799293E-4,-1.332658E-3,1.0544867E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,-1,3,5,7,-1,9,-1,-1,11,13,15,-1,17,-1,19,21,23,-1,-1,25,27,-1,29,31,-1,-1,-1,-1,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1508914E-1,0E0,2.691964E-1,1.3471697E-1,5.678174E-2,0E0,1.226424E-1,0E0,0E0,2.0624965E-2,7.672609E-2,9.691536E-3,0E0,1.1251104E-1,0E0,6.752208E-3,3.994884E-3,2.0335104E-2,0E0,0E0,4.479085E-3,8.6390413E-4,0E0,1.7875893E-2,3.4765974E-2,0E0,0E0,0E0,0E0,1.1516748E-2,1.5371395E-2,9.500405E-3,5.8546998E-3,4.7621015E-3,1.2811366E-3,5.7414146E-3,0E0,0E0,0E0,0E0,0E0,3.8729939E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,9,9,10,10,11,11,13,13,15,15,16,16,17,17,20,20,21,21,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,41,41],"right_children":[2,-1,4,6,8,-1,10,-1,-1,12,14,16,-1,18,-1,20,22,24,-1,-1,26,28,-1,30,32,-1,-1,-1,-1,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.58E3,-9.853536E-3,6.117756E9,2.2E1,7.74639E5,6.215467E-3,6.88588E5,2.5727618E-3,8.508151E-3,4.9791E4,4.44E2,5.628E3,1.6952117E-5,5.518522E3,-4.2213155E-3,1.6287474E3,1.6273E4,2.2964326E-1,5.5106E-3,-3.186063E-3,3.2458E4,5.806467E7,-6.1591633E-4,2.8464446E3,6.490565E8,-2.3580042E-3,-4.6841693E-4,-1.2288935E-3,-2.3859502E-3,7.781E4,8.7137146E8,2.2883E4,2.9607E4,2.8E2,5.2751875E8,1.4976E4,-2.352672E-3,1.0312666E-3,-1.0361779E-3,4.18018E-3,1.697582E-3,8.01998E5,7.898042E-4,8.4753986E-4,1.9961833E-3,-7.635018E-4,9.799293E-4,-1.332658E-3,1.0544867E-4],"split_indices":[3,0,27,20,12,0,6,0,0,12,4,3,0,5,0,5,28,62,0,0,12,16,0,5,16,0,0,0,0,12,23,7,7,4,23,28,0,0,0,0,0,18,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,5E0,1.74E2,1.6E2,1.4E1,5E0,1.55E2,7E0,7E0,6.3E1,9.2E1,5.4E1,9E0,8.7E1,5E0,3.2E1,2.2E1,7.9E1,8E0,2.2E1,1E1,1.3E1,9E0,5.3E1,2.6E1,5E0,5E0,7E0,6E0,3.1E1,2.2E1,1.5E1,1.1E1,1.9E1,1.2E1,1.3E1,9E0,7E0,8E0,5E0,6E0,1.3E1,6E0,6E0,6E0,8E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.6227722E-3,-5.1173E-2,3.8504772E-2,-1.08572785E-2,-3.5714883E-2,2.617635E-2,8.062234E-3,2.158106E-3,-4.3287877E-2,3.8008343E-2,-7.0525687E-3,-4.7569346E-2,1.9586543E-4,2.769468E-2,7.984798E-3,-5.2066974E-2,-3.1987773E-4,8.8201165E-2,1.3023779E-2,-5.6315724E-2,-1.0809936E-3,1.9242837E-3,4.7250064E-3,-1.5257065E-3,6.404233E-2,-3.4501937E-3,-4.5860536E-2,2.3910892E-3,-9.224374E-3,3.2802725E-3,1.4401686E-3,-6.1159123E-2,-2.1002539E-2,-2.577107E-2,2.0000119E-2,-3.905068E-2,-3.9810822E-3,-1.9244748E-3,-6.499031E-3,-1.4402906E-3,-3.960579E-2,3.8615823E-2,-1.1684226E-3,-2.3328834E-3,-6.529542E-4,2.150805E-4,-7.3512876E-4,-1.3419952E-3,1.2252473E-3,-5.13982E-2,-6.801901E-5,2.5550032E-3,1.0023927E-4,-2.6445035E-3,-1.1340718E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,-1,7,9,-1,-1,11,13,-1,15,-1,17,-1,19,-1,21,23,25,-1,-1,-1,27,29,-1,31,-1,33,-1,-1,35,37,39,41,43,-1,-1,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1],"loss_changes":[3.7776464E-1,2.507234E-1,1.8443148E-1,0E0,5.3733364E-2,2.2385427E-1,0E0,0E0,1.5350074E-2,1.3394842E-1,0E0,1.1925548E-2,0E0,7.6967545E-2,0E0,5.680725E-3,0E0,1.0266468E-2,5.354285E-2,8.733571E-3,0E0,0E0,0E0,2.5214748E-2,3.0402169E-3,0E0,1.3166606E-2,0E0,2.5177434E-2,0E0,0E0,1.1719383E-2,5.0714687E-3,1.1051068E-2,1.7576212E-2,4.4033974E-3,0E0,0E0,1.3831283E-3,1.2106693E-2,9.341981E-3,9.995373E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.1629483E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,8,8,9,9,11,11,13,13,15,15,17,17,18,18,19,19,23,23,24,24,26,26,28,28,31,31,32,32,33,33,34,34,35,35,38,38,39,39,40,40,41,41,49,49],"right_children":[2,4,6,-1,8,10,-1,-1,12,14,-1,16,-1,18,-1,20,-1,22,24,26,-1,-1,-1,28,30,-1,32,-1,34,-1,-1,36,38,40,42,44,-1,-1,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1],"split_conditions":[1.882904E6,7.5356634E-6,7.74639E5,-1.08572785E-2,1.421E3,2.635962E7,8.062234E-3,2.158106E-3,8.663E3,6.8191E4,-7.0525687E-3,8.330361E8,1.9586543E-4,1E0,7.984798E-3,1.151526E9,-3.1987773E-4,1.479E4,3.1588893E10,1.203429E6,-1.0809936E-3,1.9242837E-3,4.7250064E-3,5.353857E9,4.74794E5,-3.4501937E-3,1.7980087E3,2.3910892E-3,2.653003E6,3.2802725E-3,1.4401686E-3,1.3733394E3,2.53083E5,4.1348E4,1.3086891E10,7.2E1,-3.9810822E-3,-1.9244748E-3,3.6079565E8,2.3549E4,7.0622264E9,1E0,-1.1684226E-3,-2.3328834E-3,-6.529542E-4,2.150805E-4,-7.3512876E-4,-1.3419952E-3,1.2252473E-3,1.601E3,-6.801901E-5,2.5550032E-3,1.0023927E-4,-2.6445035E-3,-1.1340718E-3],"split_indices":[2,62,12,0,28,6,0,0,3,3,0,11,0,44,0,25,0,18,26,18,0,0,0,10,12,0,5,0,6,0,0,5,6,28,23,17,0,0,11,28,23,35,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,8.4E1,1.04E2,5E0,7.9E1,9.7E1,7E0,6E0,7.3E1,9.2E1,5E0,6.7E1,6E0,8.7E1,5E0,6E1,7E0,1.6E1,7.1E1,5.1E1,9E0,6E0,1E1,5.6E1,1.5E1,1.4E1,3.7E1,6E0,5E1,1E1,5E0,2.2E1,1.5E1,3.2E1,1.8E1,1.4E1,8E0,5E0,1E1,1.2E1,2E1,1.3E1,5E0,8E0,6E0,5E0,5E0,6E0,6E0,1.5E1,5E0,8E0,5E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.2872891E-2,-4.132386E-3,1.1113919E-1,5.41092E-3,-1.0288152E-2,1.2800488E-2,5.380845E-2,-2.87233E-2,2.5313094E-2,1.0272057E-1,-2.5616444E-3,-4.477599E-2,-1.4279177E-2,8.288484E-3,4.2006165E-2,2.6043747E-3,6.8679675E-3,-5.3105455E-2,-4.5010072E-4,1.4130973E-3,-1.99313E-2,-1.7699502E-2,2.8526308E-2,5.1963907E-2,3.918238E-4,-9.646114E-4,-6.1174765E-2,-1.348334E-2,-1.9928254E-3,3.948449E-4,-2.00571E-3,2.8237116E-3,-4.8917637E-4,3.1752955E-2,3.3077751E-3,-3.02199E-3,-1.1826772E-3,-2.1927355E-2,3.161497E-3,1.998882E-3,4.3980833E-4,-4.0766615E-2,3.4489986E-3,-4.7975336E-4,8.2426873E-4,-8.568606E-4,-2.7111873E-3,-8.034215E-4,1.2384014E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,7,-1,9,11,13,15,-1,17,19,21,23,-1,-1,25,-1,-1,27,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,39,-1,-1,-1,41,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9456326E-1,1.2034948E-1,2.557088E-1,0E0,9.589264E-2,0E0,1.2158212E-1,2.1619871E-2,1.3861477E-2,2.5890857E-2,0E0,1.2370236E-2,1.43576255E-2,1.4179348E-2,7.755883E-3,0E0,0E0,7.654853E-3,0E0,0E0,7.1809236E-3,9.483692E-3,2.2281557E-2,6.7410395E-3,0E0,0E0,5.8170184E-3,5.550744E-3,0E0,0E0,0E0,0E0,0E0,3.1915363E-3,0E0,0E0,0E0,1.2570469E-2,3.3230893E-3,0E0,0E0,5.5489466E-3,7.059271E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,20,20,21,21,22,22,23,23,26,26,27,27,33,33,37,37,38,38,41,41,42,42],"right_children":[2,4,6,-1,8,-1,10,12,14,16,-1,18,20,22,24,-1,-1,26,-1,-1,28,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,40,-1,-1,-1,42,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5406E4,2.2E1,4.6024E4,5.41092E-3,8.4606E4,1.2800488E-2,1.5357575E0,4.19345E5,3.510581E10,3.4357E4,-2.5616444E-3,6.024596E8,2.71E2,1.0656E4,4.2711E4,2.6043747E-3,6.8679675E-3,1.93E3,-4.5010072E-4,1.4130973E-3,1.8E2,3.092315E6,1.5749014E9,1.0300973E7,3.918238E-4,-9.646114E-4,6.094E3,2.9E2,-1.9928254E-3,3.948449E-4,-2.00571E-3,2.8237116E-3,-4.8917637E-4,4.072092E6,3.3077751E-3,-3.02199E-3,-1.1826772E-3,4.2088798E-1,1.7287E4,1.998882E-3,4.3980833E-4,1.99E2,1.26E2,-4.7975336E-4,8.2426873E-4,-8.568606E-4,-2.7111873E-3,-8.034215E-4,1.2384014E-3],"split_indices":[3,20,20,0,12,0,66,6,10,7,0,26,20,3,7,0,0,28,0,0,19,2,11,2,0,0,3,4,0,0,0,0,0,6,0,0,0,66,28,0,0,4,17,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.5E2,2.5E1,6E0,1.44E2,5E0,2E1,9.5E1,4.9E1,1.4E1,6E0,4.4E1,5.1E1,2.5E1,2.4E1,9E0,5E0,3.5E1,9E0,5E0,4.6E1,1.1E1,1.4E1,1.8E1,6E0,8E0,2.7E1,3.8E1,8E0,6E0,5E0,7E0,7E0,1.1E1,7E0,2.1E1,6E0,2.5E1,1.3E1,6E0,5E0,1.4E1,1.1E1,7E0,6E0,8E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.244879E-3,-9.54308E-3,1.0583062E-2,-9.904845E-3,-1.992095E-3,-2.0049235E-2,4.19506E-2,-3.7460044E-2,-7.778956E-3,5.0144717E-2,-3.6492624E-4,-4.7841247E-2,-3.4348925E-3,-1.6107049E-2,2.3417074E-2,2.2834692E-2,8.177501E-2,-4.5695642E-4,-5.431888E-2,5.902479E-4,-6.8631E-4,-3.782303E-2,4.7274805E-3,2.693725E-3,-2.5027455E-4,-8.773018E-3,4.8295964E-2,5.6573655E-2,4.639262E-3,-6.468836E-2,-3.0781584E-2,-2.2272791E-3,-2.0028306E-2,3.9288215E-2,-1.6245333E-2,9.6303906E-4,-1.889376E-3,8.672352E-4,2.9301168E-3,1.4660775E-3,3.228108E-3,-3.2483547E-3,-1.6738165E-3,-6.395931E-4,-1.9408508E-3,2.2032298E-4,-1.7225869E-3,-3.503406E-4,3.2399655E-3,-3.0424675E-2,9.56329E-4,-2.097593E-3,-7.333593E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,-1,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,29,-1,-1,31,33,-1,-1,35,37,39,-1,41,43,-1,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,51,-1,-1,-1],"loss_changes":[4.8634636E-1,2.933753E-1,0E0,0E0,1.389065E-1,2.61994E-2,2.1444418E-2,1.7849997E-2,1.9602634E-2,3.6140956E-2,0E0,8.872472E-3,2.9218753E-3,2.68729E-2,1.8217333E-2,2.0543015E-2,6.34332E-3,0E0,6.063089E-3,0E0,0E0,5.7931095E-3,2.3108162E-2,0E0,0E0,1.37272095E-2,6.1624646E-3,2.3109503E-3,0E0,2.7870834E-3,1.9969894E-3,0E0,7.0232777E-3,2.043575E-2,1.1487135E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7957382E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,25,25,26,26,27,27,29,29,30,30,32,32,33,33,34,34,49,49],"right_children":[2,4,-1,-1,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,30,-1,-1,32,34,-1,-1,36,38,40,-1,42,44,-1,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,52,-1,-1,-1],"split_conditions":[1.0966112E4,2.618E3,1.0583062E-2,-9.904845E-3,1.07218E5,1.15856E6,4.5628892E7,1.0254491E8,3.16E2,2.966E3,-3.6492624E-4,1.445E3,9.5E1,3.1633773E8,3.394521E7,1E0,4.08326E5,-4.5695642E-4,9.7E1,5.902479E-4,-6.8631E-4,6.86E3,5.7437E4,2.693725E-3,-2.5027455E-4,2.003E3,3.6724277E3,4.316E3,4.639262E-3,5.114E3,6.1513E4,-2.2272791E-3,1.29E2,3.90767E5,9.86E2,9.6303906E-4,-1.889376E-3,8.672352E-4,2.9301168E-3,1.4660775E-3,3.228108E-3,-3.2483547E-3,-1.6738165E-3,-6.395931E-4,-1.9408508E-3,2.2032298E-4,-1.7225869E-3,-3.503406E-4,3.2399655E-3,7.549E3,9.56329E-4,-2.097593E-3,-7.333593E-4],"split_indices":[5,3,0,0,12,2,2,21,4,0,0,28,4,16,16,29,18,0,17,0,0,3,12,0,0,0,5,0,0,3,8,0,4,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.85E2,1.78E2,7E0,5E0,1.73E2,1.23E2,5E1,5E1,7.3E1,4.3E1,7E0,3.8E1,1.2E1,5.8E1,1.5E1,2.4E1,1.9E1,6E0,3.2E1,5E0,7E0,2.8E1,3E1,6E0,9E0,1.1E1,1.3E1,1.1E1,8E0,2.1E1,1.1E1,1.5E1,1.3E1,1.1E1,1.9E1,6E0,5E0,6E0,7E0,6E0,5E0,1.4E1,7E0,6E0,5E0,6E0,7E0,5E0,6E0,1.4E1,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.961372E-3,-2.3550801E-2,7.997459E-2,-9.473354E-3,-1.4074665E-2,1.1058577E-2,5.330045E-2,-2.0846233E-2,4.5868546E-2,3.4291204E-2,7.055121E-3,1.88998E-3,-2.4216978E-2,2.1995844E-4,3.4980802E-3,5.96551E-2,-3.6197861E-3,-2.8578795E-3,-1.8377392E-2,5.772957E-4,7.264567E-2,-2.2062466E-2,6.7742163E-4,3.6648963E-2,9.573983E-2,-1.2080806E-2,-4.1609205E-2,2.03972E-4,2.7286545E-3,4.7936523E-3,2.5599343E-3,-2.1322504E-2,3.0185005E-2,-5.1193282E-2,1.5983406E-4,6.212362E-4,-2.7587343E-2,2.6549627E-3,-2.3957813E-4,-3.4778964E-2,-3.757485E-3,-2.1883198E-3,-1.8973839E-2,-2.5538967E-3,-2.114965E-2,5.6676246E-4,-2.5373105E-2,-1.572449E-3,-3.2072852E-4,-1.3185571E-3,-1.4885113E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,-1,7,-1,9,11,13,15,-1,-1,17,-1,-1,19,-1,-1,21,-1,23,25,-1,27,29,31,33,-1,-1,-1,-1,35,37,39,-1,-1,41,-1,-1,43,-1,-1,45,-1,47,-1,49,-1,-1,-1,-1],"loss_changes":[3.699503E-1,2.3531047E-1,2.0387173E-1,0E0,5.1694874E-2,0E0,8.095477E-2,2.5482163E-2,1.7789258E-2,1.1495845E-1,0E0,0E0,2.4818517E-2,0E0,0E0,1.77363E-2,0E0,0E0,1.2334373E-2,0E0,1.644276E-2,1.64718E-2,0E0,8.80633E-3,8.736551E-4,2.3689479E-2,1.2770064E-2,0E0,0E0,0E0,0E0,1.130745E-2,1.2358699E-2,1.0752574E-2,0E0,0E0,7.2063915E-3,0E0,0E0,4.5850053E-3,0E0,0E0,6.873926E-3,0E0,2.4044588E-3,0E0,2.7650055E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,12,12,15,15,18,18,20,20,21,21,23,23,24,24,25,25,26,26,31,31,32,32,33,33,36,36,39,39,42,42,44,44,46,46],"right_children":[2,4,6,-1,8,-1,10,12,14,16,-1,-1,18,-1,-1,20,-1,-1,22,-1,24,26,-1,28,30,32,34,-1,-1,-1,-1,36,38,40,-1,-1,42,-1,-1,44,-1,-1,46,-1,48,-1,50,-1,-1,-1,-1],"split_conditions":[4.200155E6,1.3267355E8,1.461E3,-9.473354E-3,1.2538E4,1.1058577E-2,1.0362809E1,1.0620748E-6,1.4027468E10,1.1377422E0,7.055121E-3,1.88998E-3,1.6E1,2.1995844E-4,3.4980802E-3,3.2E1,-3.6197861E-3,-2.8578795E-3,3.2659E4,5.772957E-4,5.530237E9,3.0853E4,6.7742163E-4,4.833515E10,9.767E3,2.7265E4,8.156809E-1,2.03972E-4,2.7286545E-3,4.7936523E-3,2.5599343E-3,2.81E3,1.027E3,9.0452E4,1.5983406E-4,6.212362E-4,1.97E2,2.6549627E-3,-2.3957813E-4,7.5E1,-3.757485E-3,-2.1883198E-3,6.779055E6,-2.5538967E-3,5.468E3,5.6676246E-4,3.2558454E7,-1.572449E-3,-3.2072852E-4,-1.3185571E-3,-1.4885113E-4],"split_indices":[6,10,0,0,3,0,59,62,10,63,0,0,17,0,0,4,0,0,7,0,11,22,0,13,0,7,63,0,0,0,0,28,20,12,0,0,20,0,0,19,0,0,23,0,3,0,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.29E2,4.6E1,5E0,1.24E2,5E0,4.1E1,1.12E2,1.2E1,3.6E1,5E0,5E0,1.07E2,6E0,6E0,3E1,6E0,1.2E1,9.5E1,7E0,2.3E1,8.6E1,9E0,1E1,1.3E1,5.8E1,2.8E1,5E0,5E0,8E0,5E0,4.8E1,1E1,2.3E1,5E0,7E0,4.1E1,5E0,5E0,1.7E1,6E0,1E1,3.1E1,5E0,1.2E1,5E0,2.6E1,5E0,7E0,2.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[5.01004E-3,-7.523812E-3,1.0691043E-1,-2.9343944E-2,1.4388074E-2,6.595153E-2,8.357917E-3,-4.938897E-2,-2.0197917E-2,5.377663E-2,4.0245354E-3,1.3515672E-3,3.5069315E-3,-6.509395E-2,-6.213727E-4,-4.013465E-2,-9.456055E-3,1.5124137E-4,3.5312222E-3,-3.3372216E-2,1.7679986E-2,-4.4931162E-2,-4.1277125E-3,-5.9935637E-4,-2.2028273E-3,-2.0356323E-3,3.2533356E-3,-2.926795E-3,-1.5258686E-2,2.6861483E-3,6.272313E-3,-2.7357743E-3,-7.3047687E-4,1.2657344E-2,-1.0558127E-3,2.6564774E-4,-1.7971854E-3,-1.1020054E-3,1.6237881E-2,2.0004378E-3,8.958485E-4,2.5020996E-3,1.8652926E-3,-4.273051E-4,6.7552173E-4,-2.2456583E-3,2.2495605E-2,3.6953832E-3,2.5961255E-3,7.597833E-4,-4.602409E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,-1,23,25,-1,-1,27,29,31,-1,-1,-1,-1,33,-1,35,-1,37,-1,-1,39,-1,-1,-1,-1,41,-1,43,-1,45,-1,-1,-1,47,49,-1,-1,-1],"loss_changes":[2.3325558E-1,7.8355804E-2,5.6656122E-2,1.4035702E-2,3.325637E-2,5.066864E-3,0E0,1.2875069E-2,1.2041625E-2,2.2921812E-2,3.4294207E-2,0E0,0E0,5.85445E-3,0E0,4.764192E-3,1.8581873E-2,0E0,0E0,9.959245E-3,2.4132174E-2,4.9560424E-3,0E0,0E0,0E0,0E0,8.04846E-3,0E0,7.4421344E-3,0E0,1.2974763E-2,0E0,0E0,8.933238E-3,0E0,0E0,0E0,0E0,1.8199854E-2,0E0,2.956139E-3,0E0,2.7555553E-2,0E0,0E0,0E0,1.2295916E-2,2.7219446E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19,20,20,21,21,26,26,28,28,30,30,33,33,38,38,40,40,42,42,46,46,47,47],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,-1,24,26,-1,-1,28,30,32,-1,-1,-1,-1,34,-1,36,-1,38,-1,-1,40,-1,-1,-1,-1,42,-1,44,-1,46,-1,-1,-1,48,50,-1,-1,-1],"split_conditions":[2.8100613E9,1.882904E6,8.504903E3,1.25E2,1E0,6.7E1,8.357917E-3,4.5180562E-1,2.63E2,1.3895729E3,1.5235041E-2,1.3515672E-3,3.5069315E-3,2.2254E4,-6.213727E-4,6.7739E4,1.6807637E3,1.5124137E-4,3.5312222E-3,6.3062E4,4.2E1,3.645E3,-4.1277125E-3,-5.9935637E-4,-2.2028273E-3,-2.0356323E-3,8.027122E8,-2.926795E-3,8.954E3,2.6861483E-3,6.7179E4,-2.7357743E-3,-7.3047687E-4,2.56E2,-1.0558127E-3,2.6564774E-4,-1.7971854E-3,-1.1020054E-3,1.00770764E-1,2.0004378E-3,5.89247E5,2.5020996E-3,2.2964326E-1,-4.273051E-4,6.7552173E-4,-2.2456583E-3,3.1977E4,2.4404E4,2.5961255E-3,7.597833E-4,-4.602409E-4],"split_indices":[1,2,5,4,44,19,0,62,0,5,66,0,0,7,0,6,5,0,0,12,19,3,0,0,0,0,16,0,3,0,28,0,0,20,0,0,0,0,62,0,6,0,62,0,0,0,7,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.81E2,1.62E2,1.9E1,8.1E1,8.1E1,1.4E1,5E0,2.4E1,5.7E1,1.6E1,6.5E1,5E0,9E0,1.6E1,8E0,1.9E1,3.8E1,6E0,1E1,1.7E1,4.8E1,1.1E1,5E0,6E0,1.3E1,9E0,2.9E1,5E0,1.2E1,9E0,3.9E1,6E0,5E0,2.2E1,7E0,7E0,5E0,9E0,3E1,5E0,1.7E1,7E0,2.3E1,1E1,7E0,6E0,1.7E1,1.2E1,5E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-5.30129E-4,-9.188648E-3,7.6548248E-3,-1.3246857E-2,4.3456756E-2,-1.9379737E-2,3.0225113E-3,5.9423227E-2,-3.4515334E-3,1.5038324E-3,-2.4218492E-2,3.751458E-2,9.49258E-3,-3.3936545E-2,-2.0857672E-3,6.486175E-2,2.3523258E-2,-2.3361506E-2,-7.795839E-2,1.0047955E-3,-1.2024185E-2,1.4097114E-3,3.8984432E-3,9.215302E-3,2.360392E-3,-4.1862752E-2,-1.0248625E-2,-4.31035E-3,-1.9278298E-3,3.5381824E-4,-1.2758686E-3,2.1777678E-2,-1.3097442E-3,-2.7957538E-3,-2.4986474E-2,5.4240634E-4,-1.9059228E-2,4.0012973E-4,1.5885109E-3,-5.503306E-5,-1.730922E-3,-2.9899728E-2,6.473739E-4,-1.5133232E-4,-4.4002276E-2,-2.414226E-3,-1.1068075E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,-1,3,5,7,9,-1,11,-1,-1,13,15,-1,17,19,21,23,25,27,-1,29,-1,-1,31,-1,33,35,-1,-1,-1,-1,37,-1,-1,39,-1,41,-1,-1,-1,-1,43,-1,-1,45,-1,-1],"loss_changes":[2.748969E-1,0E0,1.1581702E-1,5.092396E-2,1.16217084E-1,2.4799567E-2,0E0,1.6845079E-1,0E0,0E0,1.821161E-2,1.6074903E-2,0E0,2.5645196E-2,7.0138886E-3,9.060852E-3,1.3491837E-2,1.1458594E-2,3.3679083E-3,0E0,7.164853E-3,0E0,0E0,1.1986386E-2,0E0,5.889587E-3,6.230465E-3,0E0,0E0,0E0,0E0,2.9841643E-3,0E0,0E0,4.4749826E-3,0E0,8.526244E-3,0E0,0E0,0E0,0E0,6.184458E-3,0E0,0E0,9.1695413E-4,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,7,7,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,25,25,26,26,31,31,34,34,36,36,41,41,44,44],"right_children":[2,-1,4,6,8,10,-1,12,-1,-1,14,16,-1,18,20,22,24,26,28,-1,30,-1,-1,32,-1,34,36,-1,-1,-1,-1,38,-1,-1,40,-1,42,-1,-1,-1,-1,44,-1,-1,46,-1,-1],"split_conditions":[2.1569896E7,-9.188648E-3,4.503171E6,1.7015084E9,9.6779176E7,1E0,3.0225113E-3,1.11464E5,-3.4515334E-3,1.5038324E-3,2.493638E-1,4.2E1,9.49258E-3,3.0853E4,3.645513E-1,3.3219E4,2.1E2,3.33044E5,1.07E3,1.0047955E-3,2.0989612E9,1.4097114E-3,3.8984432E-3,1.62E2,2.360392E-3,4.1945936E7,5.626E3,-4.31035E-3,-1.9278298E-3,3.5381824E-4,-1.2758686E-3,2.9565E4,-1.3097442E-3,-2.7957538E-3,5.3195E4,5.4240634E-4,1.0856773E7,4.0012973E-4,1.5885109E-3,-5.503306E-5,-1.730922E-3,4.612E4,6.473739E-4,-1.5133232E-4,2.17E2,-2.414226E-3,-1.1068075E-3],"split_indices":[11,0,2,16,2,44,0,3,0,0,62,19,0,22,62,18,17,6,0,0,25,0,0,17,0,16,3,0,0,0,0,7,0,0,8,0,27,0,0,0,0,12,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.58E2,5E0,1.53E2,9.7E1,5.6E1,9.1E1,6E0,5E1,6E0,7E0,8.4E1,4.5E1,5E0,5.8E1,2.6E1,1.4E1,3.1E1,4.8E1,1E1,7E0,1.9E1,7E0,7E0,2.2E1,9E0,1.9E1,2.9E1,5E0,5E0,9E0,1E1,1.7E1,5E0,7E0,1.2E1,8E0,2.1E1,1E1,7E0,5E0,7E0,1.6E1,5E0,6E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0275718E-2,-4.6343976E-3,7.728577E-2,1.7979271E-3,-6.9065928E-3,5.3479664E-2,7.6600313E-3,-6.891739E-3,7.5444416E-3,1.08476825E-1,1.6899198E-2,-1.5960556E-2,3.0982308E-2,6.63526E-3,2.0449858E-3,3.322488E-2,-9.292275E-4,1.216592E-3,-1.8704686E-2,4.0103268E-2,-3.3280445E-4,-1.1314832E-5,3.1571079E-3,-2.7409686E-2,-1.7302353E-3,5.2985277E-2,8.444076E-4,-4.5384925E-2,-1.9311892E-2,1.3088921E-2,-1.8790369E-3,1.3978455E-3,3.1469355E-3,-2.5982286E-3,-1.0420413E-3,-3.8558334E-2,-1.07561005E-2,2.2895714E-2,-1.16473E-3,-8.0788607E-4,-2.0396973E-3,8.593793E-4,-1.7501501E-2,3.584437E-2,1.08157976E-4,-1.0399202E-2,-1.7175947E-3,2.3307127E-3,4.6532118E-4,-1.944538E-2,-1.2489434E-3,-1.6955445E-4,-1.309635E-3,4.3298004E-4,-5.3421763E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,17,19,-1,-1,21,-1,-1,23,25,-1,-1,-1,27,29,31,-1,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,45,47,-1,49,-1,-1,-1,51,53,-1,-1,-1,-1],"loss_changes":[1.8120027E-1,1.475005E-1,6.777148E-2,2.152328E-1,0E0,5.4497845E-2,0E0,4.7893744E-2,0E0,2.2394523E-2,1.1929605E-2,1.3993433E-2,9.81343E-3,0E0,0E0,1.7042212E-2,0E0,0E0,1.5615411E-2,4.8575327E-3,0E0,0E0,0E0,9.312719E-3,2.3219943E-2,2.8990135E-3,0E0,4.9676895E-3,7.81869E-3,1.1532582E-2,0E0,0E0,0E0,0E0,0E0,1.5930198E-3,7.7377944E-3,5.9919553E-3,0E0,0E0,0E0,0E0,4.2464407E-3,5.619975E-3,0E0,1.9623106E-3,0E0,0E0,0E0,1.8552593E-3,1.717217E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,11,11,12,12,15,15,18,18,19,19,23,23,24,24,25,25,27,27,28,28,29,29,35,35,36,36,37,37,42,42,43,43,45,45,49,49,50,50],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,18,20,-1,-1,22,-1,-1,24,26,-1,-1,-1,28,30,32,-1,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,46,48,-1,50,-1,-1,-1,52,54,-1,-1,-1,-1],"split_conditions":[1.5097532E9,3.5211442E0,6.737724E0,5.830747E3,-6.9065928E-3,1.9367423E9,7.6600313E-3,3.203437E6,7.5444416E-3,1.344E3,4.290848E10,1.48E2,1E0,6.63526E-3,2.0449858E-3,6.759545E10,-9.292275E-4,1.216592E-3,2.5484189E8,1.00770764E-1,-3.3280445E-4,-1.1314832E-5,3.1571079E-3,5.544997E6,2.0911702E7,9.3813E4,8.444076E-4,2.6919014E3,1.8448E4,7.506021E-1,-1.8790369E-3,1.3978455E-3,3.1469355E-3,-2.5982286E-3,-1.0420413E-3,3.8E1,1.5383122E7,1.41E2,-1.16473E-3,-8.0788607E-4,-2.0396973E-3,8.593793E-4,1.7893435E9,7.3461664E-1,1.08157976E-4,6.071E3,-1.7175947E-3,2.3307127E-3,4.6532118E-4,1.88744E5,1.14E2,-1.6955445E-4,-1.309635E-3,4.3298004E-4,-5.3421763E-4],"split_indices":[16,59,59,5,0,16,0,6,0,0,21,28,35,0,0,10,0,0,16,62,0,0,0,23,18,8,0,5,7,62,0,0,0,0,0,17,23,4,0,0,0,0,25,63,0,3,0,0,0,6,17,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.8E2,1.48E2,3.2E1,1.43E2,5E0,2.7E1,5E0,1.37E2,6E0,1E1,1.7E1,1.11E2,2.6E1,5E0,5E0,1.2E1,5E0,6E0,1.05E2,2.1E1,5E0,7E0,5E0,6.9E1,3.6E1,1.2E1,9E0,2E1,4.9E1,2.7E1,9E0,7E0,5E0,1.1E1,9E0,1.4E1,3.5E1,2.2E1,5E0,5E0,9E0,6E0,2.9E1,1.3E1,9E0,2.3E1,6E0,7E0,6E0,1.1E1,1.2E1,5E0,6E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-2.6316978E-3,-9.091787E-3,4.6002306E-3,-9.117377E-3,8.507575E-2,-2.5486734E-3,-7.1169417E-3,1.9270381E-4,1.1031518E-1,-9.861075E-3,7.194873E-3,7.610592E-3,4.7880318E-2,-2.3380289E-2,2.2363922E-2,-6.3481036E-4,4.4661625E-3,-3.3429235E-2,-8.839555E-3,5.3520832E-2,4.2262324E-3,1.7873092E-4,-3.7116073E-2,2.647818E-2,-2.1086508E-2,7.967135E-4,3.0260014E-3,-1.7725239E-3,1.6262334E-2,-3.258101E-3,-2.7524844E-2,1.6654526E-4,1.9522217E-3,-3.3039156E-2,7.597728E-4,3.1961896E-2,-9.5785654E-4,-1.2432854E-2,-4.274346E-2,-3.544464E-4,-4.0894132E-2,4.3498806E-4,1.9275696E-3,-1.9663295E-3,-1.8227078E-3,-2.610193E-3,-2.9942356E-2,-2.3124914E-3,-1.000371E-3,-6.921953E-4,5.974574E-4,-5.331559E-4,-1.6775052E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,-1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,19,-1,-1,21,23,25,27,-1,29,31,33,-1,-1,-1,35,-1,37,-1,-1,39,-1,41,-1,43,45,-1,47,-1,-1,49,-1,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6274073E-1,0E0,1.8954799E-1,1.4822856E-1,5.040869E-2,1.7543814E-1,0E0,0E0,6.763376E-2,6.0207836E-2,0E0,0E0,3.8873628E-2,1.3878044E-2,2.2943495E-2,0E0,0E0,8.037694E-3,1.8292254E-2,7.097192E-3,1.5231231E-2,0E0,1.6663559E-2,4.378669E-3,1.4838292E-2,0E0,0E0,0E0,1.3898063E-2,0E0,9.344786E-3,0E0,0E0,4.3655112E-3,0E0,3.85377E-3,0E0,6.977879E-3,2.7952269E-3,0E0,2.6965272E-3,0E0,0E0,4.1318294E-3,0E0,0E0,1.6378677E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,8,8,9,9,12,12,13,13,14,14,17,17,18,18,19,19,20,20,22,22,23,23,24,24,28,28,30,30,33,33,35,35,37,37,38,38,40,40,43,43,46,46],"right_children":[2,-1,4,6,8,10,-1,-1,12,14,-1,-1,16,18,20,-1,-1,22,24,26,28,-1,30,32,34,-1,-1,-1,36,-1,38,-1,-1,40,-1,42,-1,44,46,-1,48,-1,-1,50,-1,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[2.618E3,-9.091787E-3,1.5522031E9,3.5713768E0,4.07768E5,1.5522577E7,-7.1169417E-3,1.9270381E-4,1.69409E7,3.601764E6,7.194873E-3,7.610592E-3,1.2779233E10,3.135717E6,6.059E4,-6.3481036E-4,4.4661625E-3,1.9385E4,2.506355E-3,1.5E1,4.8348E4,1.7873092E-4,4.0221E4,1.18159E5,1.7694E4,7.967135E-4,3.0260014E-3,-1.7725239E-3,3.876437E6,-3.258101E-3,1.8552737E-2,1.6654526E-4,1.9522217E-3,1.659E4,7.597728E-4,2.5054538E9,-9.5785654E-4,3.2608296E3,7.4E1,-3.544464E-4,2.453417E8,4.3498806E-4,1.9275696E-3,1.7459727E-3,-1.8227078E-3,-2.610193E-3,1.03E2,-2.3124914E-3,-1.000371E-3,-6.921953E-4,5.974574E-4,-5.331559E-4,-1.6775052E-3],"split_indices":[3,0,16,59,18,6,0,0,18,2,0,0,10,18,8,0,0,18,63,17,28,0,18,27,20,0,0,0,18,0,63,0,0,12,0,11,0,5,4,0,11,0,0,63,0,0,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,5E0,1.7E2,1.46E2,2.4E1,1.41E2,5E0,6E0,1.8E1,1.36E2,5E0,8E0,1E1,9.6E1,4E1,5E0,5E0,5.6E1,4E1,1.4E1,2.6E1,5E0,5.1E1,1E1,3E1,5E0,9E0,5E0,2.1E1,9E0,4.2E1,5E0,5E0,2.3E1,7E0,1.5E1,6E0,2.2E1,2E1,6E0,1.7E1,6E0,9E0,1.7E1,5E0,7E0,1.3E1,9E0,8E0,9E0,8E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.6406242E-3,-7.831071E-3,6.8243735E-2,-1.7289269E-3,-7.076499E-3,4.259416E-2,6.7737033E-3,-9.962822E-3,7.482386E-3,4.7651026E-3,1.8034587E-2,4.3420857E-3,-1.4517094E-2,-8.5997744E-4,2.5298388E-3,-2.1622444E-2,3.2618724E-2,-2.4739053E-2,9.143888E-4,1.962049E-2,2.5101989E-3,-2.1264017E-2,-3.0113775E-3,1.2149293E-3,1.7581206E-4,-2.5004877E-3,-1.8230058E-2,-1.514894E-2,-2.2931914E-3,-2.1198336E-2,-3.1410332E-3,-2.6558971E-2,-3.791532E-3,4.9725655E-3,-1.4834551E-3,1.4630909E-4,-1.3492046E-3,8.2999567E-4,-8.457789E-4,1.1275519E-3,-2.2568229E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,-1,15,-1,-1,17,19,21,-1,23,-1,25,-1,-1,-1,-1,27,29,-1,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3763925E-1,1.484858E-1,5.5835694E-2,2.1985097E-1,0E0,3.563998E-2,0E0,7.471512E-2,0E0,0E0,2.8263437E-2,0E0,4.8263438E-2,0E0,0E0,1.6970675E-2,5.0192717E-3,1.6372278E-2,0E0,1.681143E-3,0E0,1.0869127E-2,0E0,0E0,0E0,0E0,9.98668E-3,6.8342686E-3,0E0,5.777929E-3,8.4281415E-3,6.3531026E-3,6.056252E-3,5.84668E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,12,12,15,15,16,16,17,17,19,19,21,21,26,26,27,27,29,29,30,30,31,31,32,32,33,33],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,-1,16,-1,-1,18,20,22,-1,24,-1,26,-1,-1,-1,-1,28,30,-1,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5522031E9,3.5713768E0,9.0573E4,2.3378E4,-7.076499E-3,1.1270022E-3,6.7737033E-3,1E0,7.482386E-3,4.7651026E-3,2.3762707E-1,4.3420857E-3,1.2538E4,-8.5997744E-4,2.5298388E-3,1.22666E5,4.833515E10,2.0901188E7,9.143888E-4,1.20743E5,2.5101989E-3,5.9E1,-3.0113775E-3,1.2149293E-3,1.7581206E-4,-2.5004877E-3,3.9281719E3,6.502E3,-2.2931914E-3,3.0089152E7,1.251E3,2.715E3,3.1E1,7E0,-1.4834551E-3,1.4630909E-4,-1.3492046E-3,8.2999567E-4,-8.457789E-4,1.1275519E-3,-2.2568229E-4],"split_indices":[16,59,3,3,0,63,0,30,0,0,62,0,3,0,0,28,13,18,0,28,0,4,0,0,0,0,5,3,0,27,0,3,19,19,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.84E2,1.57E2,2.7E1,1.52E2,5E0,2.2E1,5E0,1.46E2,6E0,5E0,1.7E1,5E0,1.41E2,9E0,8E0,1.23E2,1.8E1,1.15E2,8E0,1.3E1,5E0,1.08E2,7E0,8E0,5E0,7E0,1.01E2,9.4E1,7E0,6.2E1,3.2E1,4.7E1,1.5E1,2.6E1,6E0,6E0,4.1E1,6E0,9E0,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.151796E-3,-7.9117445E-3,1.4248016E-1,-8.46283E-3,-8.438262E-4,2.3185597E-3,1.0382699E-2,4.765937E-3,-6.315147E-3,-3.0315462E-3,9.728879E-2,3.7420746E-3,-7.5449417E-3,7.0902477E-3,1.2021122E-3,-2.3307934E-2,1.0798908E-2,-3.157542E-3,-1.791746E-2,2.6456458E-3,3.61776E-3,-2.7200948E-2,4.155157E-3,-7.567038E-3,2.827188E-2,-9.9592935E-3,-3.332044E-2,1.8369174E-2,-9.2700677E-4,-1.9701945E-2,2.258847E-3,2.2570277E-3,8.655047E-3,3.5423329E-4,-9.123438E-4,-4.4156548E-2,-1.769278E-2,-3.6317355E-4,1.4892946E-3,-3.307405E-4,-1.2502061E-3,1.0570006E-3,-5.186053E-3,-5.968525E-4,1.2894238E-3,-3.0723361E-2,-2.8246036E-3,-1.3917732E-3,-2.6133537E-4,-1.050344E-3,6.2585813E-3,-1.7877967E-3,-4.4725864E-4,-4.188394E-4,1.0349816E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,-1,7,-1,-1,9,-1,11,13,-1,15,-1,-1,17,19,-1,21,-1,23,25,27,29,31,33,35,37,-1,39,41,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,49,-1,-1,51,-1,-1,-1,-1,53,-1,-1,-1,-1],"loss_changes":[2.7242514E-1,2.154144E-1,9.8186135E-2,0E0,1.2899092E-1,0E0,0E0,1.1139753E-1,0E0,5.7938263E-2,5.005987E-2,0E0,3.986579E-2,0E0,0E0,1.8263586E-2,2.2675732E-2,0E0,1.4046965E-2,0E0,1.5931763E-2,4.7326423E-3,7.970081E-3,4.8139514E-3,7.8288745E-3,2.9389318E-3,5.145751E-3,6.169872E-3,0E0,1.7343601E-3,3.885557E-3,0E0,5.5343853E-3,0E0,0E0,3.5350136E-3,2.4674535E-3,0E0,0E0,0E0,0E0,0E0,4.0396475E-3,0E0,0E0,2.5148923E-3,0E0,0E0,0E0,0E0,3.548277E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,9,9,10,10,12,12,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,32,32,35,35,36,36,42,42,45,45,50,50],"right_children":[2,4,6,-1,8,-1,-1,10,-1,12,14,-1,16,-1,-1,18,20,-1,22,-1,24,26,28,30,32,34,36,38,-1,40,42,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,50,-1,-1,52,-1,-1,-1,-1,54,-1,-1,-1,-1],"split_conditions":[7.74639E5,2.5E1,1.0966112E4,-8.46283E-3,3.38511E7,2.3185597E-3,1.0382699E-2,1.8195216E7,-6.315147E-3,2.2E1,3.44675E5,3.7420746E-3,1.878295E6,7.0902477E-3,1.2021122E-3,1.6E1,8.76496E5,-3.157542E-3,4.277561E8,2.6456458E-3,1.0544393E10,5.261E3,9.6703277E8,2.05E2,6.400218E9,2.18E2,2.61E2,1.8619835E8,-9.2700677E-4,9.2E1,2.34E2,2.2570277E-3,6.820745E6,3.5423329E-4,-9.123438E-4,2.556831E6,9.7786E4,-3.6317355E-4,1.4892946E-3,-3.307405E-4,-1.2502061E-3,1.0570006E-3,1.20343E5,-5.968525E-4,1.2894238E-3,1.634654E6,-2.8246036E-3,-1.3917732E-3,-2.6133537E-4,-1.050344E-3,3.2373845E3,-1.7877967E-3,-4.4725864E-4,-4.188394E-4,1.0349816E-3],"split_indices":[12,4,5,0,2,0,0,2,0,20,12,0,2,0,0,17,6,0,25,0,25,28,21,4,21,4,4,16,0,17,4,0,6,0,0,18,18,0,0,0,0,0,27,0,0,18,0,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,1.63E2,1.2E1,5E0,1.58E2,7E0,5E0,1.53E2,5E0,1.42E2,1.1E1,6E0,1.36E2,5E0,6E0,7.3E1,6.3E1,6E0,6.7E1,7E0,5.6E1,4.7E1,2E1,3.9E1,1.7E1,1.3E1,3.4E1,1.3E1,7E0,1.7E1,2.2E1,7E0,1E1,5E0,8E0,1.9E1,1.5E1,5E0,8E0,8E0,9E0,5E0,1.7E1,5E0,5E0,1.3E1,6E0,6E0,9E0,6E0,1.1E1,8E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[2.2261597E-3,-5.7805437E-3,9.807829E-3,-2.4473418E-2,1.5862267E-2,-1.9539151E-2,-3.499977E-3,3.0981682E-3,1.0283045E-2,-2.7516631E-2,2.8148855E-3,-2.9654609E-2,1.72029E-2,-2.321863E-2,-2.7829949E-3,-1.2559553E-2,1.7349948E-3,-2.1008528E-3,-2.7208982E-4,2.1863732E-2,-1.306882E-3,-3.141211E-2,-5.178398E-3,7.301493E-5,-1.108477E-3,9.672167E-3,6.2383477E-2,-8.588609E-3,-4.249353E-2,1.160957E-2,-1.8980848E-3,2.2029171E-3,3.3104918E-3,1.5339261E-3,3.2741812E-3,7.158964E-4,-1.0603897E-3,-5.437605E-2,-2.1346819E-2,-4.6135482E-4,1.5407066E-3,-1.4370433E-2,1.1401261E-2,-2.967752E-3,-1.4079732E-3,-1.3907675E-3,-3.173911E-4,-1.5366116E-3,6.898277E-4,-5.415471E-4,1.8458521E-2,2.912249E-2,-3.7332077E-4,1.7193574E-3,4.4916253E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,19,21,-1,23,-1,-1,-1,25,-1,27,29,-1,-1,31,33,35,37,39,-1,-1,41,-1,-1,-1,-1,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,51,53,-1,-1,-1],"loss_changes":[3.0769098E-1,6.795078E-2,0E0,2.3175314E-2,2.3257274E-2,1.50570795E-2,0E0,0E0,2.0408273E-2,8.1779435E-3,1.35601945E-2,4.46047E-3,1.4278218E-2,8.262638E-3,0E0,3.1483846E-3,0E0,0E0,0E0,2.755457E-2,0E0,9.498794E-3,1.2724297E-2,0E0,0E0,1.1512702E-2,1.8367209E-3,5.8935536E-3,5.3724274E-3,7.7291694E-3,0E0,0E0,5.885932E-3,0E0,0E0,0E0,0E0,2.625987E-3,1.3966253E-3,0E0,0E0,8.729532E-3,4.9337037E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.5862383E-3,2.689017E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,32,32,37,37,38,38,41,41,42,42,50,50,51,51],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,20,22,-1,24,-1,-1,-1,26,-1,28,30,-1,-1,32,34,36,38,40,-1,-1,42,-1,-1,-1,-1,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,52,54,-1,-1,-1],"split_conditions":[8.539015E3,2.216174E6,9.807829E-3,8.221E4,4.68E2,5.477601E8,-3.499977E-3,3.0981682E-3,6.151E3,2.6732274E7,1.62E2,3.949527E6,4.5338184E7,4.093E4,-2.7829949E-3,9.163062E6,1.7349948E-3,-2.1008528E-3,-2.7208982E-4,4.316E3,-1.306882E-3,4.0622024E7,5.5589E4,7.301493E-5,-1.108477E-3,6.6653E4,2.00939E5,2.6E1,3.2E2,4.7166E4,-1.8980848E-3,2.2029171E-3,5.1637E4,1.5339261E-3,3.2741812E-3,7.158964E-4,-1.0603897E-3,3.59E2,3.33044E5,-4.6135482E-4,1.5407066E-3,7.9635E4,1.1171634E9,-2.967752E-3,-1.4079732E-3,-1.3907675E-3,-3.173911E-4,-1.5366116E-3,6.898277E-4,-5.415471E-4,2.4686491E0,1.8279E5,-3.7332077E-4,1.7193574E-3,4.4916253E-4],"split_indices":[5,9,0,8,20,26,0,0,3,27,17,2,2,12,0,18,0,0,0,0,0,11,12,0,0,12,28,17,4,12,0,0,28,0,0,0,0,0,6,0,0,8,11,0,0,0,0,0,0,0,59,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,1.66E2,5E0,8.9E1,7.7E1,8.3E1,6E0,6E0,7.1E1,6.1E1,2.2E1,1E1,6.1E1,5.6E1,5E0,1.6E1,6E0,5E0,5E0,5.6E1,5E0,3.8E1,1.8E1,8E0,8E0,4.4E1,1.2E1,1.3E1,2.5E1,1.3E1,5E0,5E0,3.9E1,5E0,7E0,5E0,8E0,1.5E1,1E1,7E0,6E0,1.2E1,2.7E1,8E0,7E0,5E0,5E0,7E0,5E0,6E0,2.1E1,1.5E1,6E0,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[4.5215455E-3,-3.386566E-3,8.502421E-3,1.6999824E-3,-5.296409E-3,-6.9675497E-3,5.8927694E-3,-2.7035397E-2,1.1933151E-2,-1.99492E-2,-5.9296552E-2,-1.3630936E-3,3.2993767E-2,-2.673039E-2,1.1098394E-2,-3.4231087E-3,-1.0434305E-3,2.8381876E-3,-1.1894026E-2,4.3357506E-2,-9.696922E-5,-1.7623529E-3,-3.5882764E-2,-6.3147105E-4,1.3716792E-3,1.4019136E-3,-1.9688765E-2,6.252835E-2,2.5197733E-2,6.55341E-4,-5.085865E-4,-4.4865724E-2,-2.2147004E-2,-3.032836E-3,-1.003815E-2,1.4763508E-3,3.5271277E-3,-2.4603212E-5,1.9461839E-3,-2.2863944E-3,-1.0982002E-3,-3.236365E-2,8.630985E-5,8.490343E-4,-3.1889703E-2,-6.3665264E-4,-2.082386E-3,-1.0945439E-2,8.909277E-4,2.5431813E-5,-2.5772254E-3,-6.310002E-5,-8.326526E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,-1,7,-1,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,-1,29,31,-1,-1,-1,33,35,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,45,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1],"loss_changes":[2.6398978E-1,1.0160796E-1,0E0,1.8928209E-1,0E0,5.8734257E-2,0E0,1.5962742E-2,2.244714E-2,1.3560662E-2,6.9266707E-3,3.5730407E-2,1.1410203E-2,1.1819635E-2,6.6912645E-3,0E0,0E0,0E0,1.5727615E-2,6.779745E-3,0E0,2.663799E-3,3.695093E-3,0E0,0E0,0E0,1.7564181E-2,3.0389614E-3,7.01888E-3,0E0,0E0,1.9411109E-3,4.254007E-3,0E0,7.877838E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.5778264E-3,0E0,5.530364E-3,9.734933E-3,0E0,0E0,1.1311382E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,18,18,19,19,21,21,22,22,26,26,27,27,28,28,31,31,32,32,34,34,41,41,43,43,44,44,47,47],"right_children":[2,4,-1,6,-1,8,-1,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,-1,30,32,-1,-1,-1,34,36,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,46,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1],"split_conditions":[3.2261983E9,1.2498039E0,8.502421E-3,3.4492E4,-5.296409E-3,1.882904E6,5.8927694E-3,7E1,1.0416E4,1.0829598E8,7.1161E4,4.68E2,2.0901188E7,4.286E3,2.3944454E8,-3.4231087E-3,-1.0434305E-3,2.8381876E-3,6.0954E8,1.2E2,-9.696922E-5,3.113E3,2.61E2,-6.3147105E-4,1.3716792E-3,1.4019136E-3,1.037547E6,3.9E1,4.7E2,6.55341E-4,-5.085865E-4,4.8E2,3.41E2,-3.032836E-3,8.172E3,1.4763508E-3,3.5271277E-3,-2.4603212E-5,1.9461839E-3,-2.2863944E-3,-1.0982002E-3,3.59E2,8.630985E-5,6.9937E4,8.156355E8,-6.3665264E-4,-2.082386E-3,4.091078E6,8.909277E-4,2.5431813E-5,-2.5772254E-3,-6.310002E-5,-8.326526E-4],"split_indices":[16,58,0,3,0,2,0,19,3,21,28,20,18,28,16,0,0,0,11,19,0,3,4,0,0,0,6,17,19,0,0,0,20,0,3,0,0,0,0,0,0,4,0,8,23,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,1.68E2,6E0,1.62E2,6E0,1.53E2,9E0,7.4E1,7.9E1,6.2E1,1.2E1,4.9E1,3E1,5.1E1,1.1E1,7E0,5E0,6E0,4.3E1,2.3E1,7E0,1.4E1,3.7E1,5E0,6E0,6E0,3.7E1,1E1,1.3E1,5E0,9E0,2.1E1,1.6E1,5E0,3.2E1,5E0,5E0,6E0,7E0,1.4E1,7E0,1.1E1,5E0,2.2E1,1E1,6E0,5E0,1.4E1,8E0,5E0,5E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.1670625E-3,-9.171418E-3,1.0811375E-2,-1.0286829E-3,9.317695E-2,4.366773E-3,-5.292271E-3,-6.041471E-4,1.6009933E-1,-1.9455807E-2,1.9595692E-2,3.1406647E-3,1.0952773E-2,-2.422369E-2,1.4420485E-3,2.6826901E-2,-6.7439135E-3,-5.617739E-2,-1.8639198E-2,2.7089897E-3,1.4976343E-2,-8.576633E-4,4.1190503E-4,-3.0923001E-3,-1.2962567E-3,-1.448441E-2,-2.7331812E-3,1.7523391E-3,3.6870323E-2,6.4558815E-3,-2.2735868E-2,8.3594245E-4,-8.3157085E-3,2.297321E-3,7.8962656E-4,2.1840334E-3,-1.1407623E-2,-1.7866854E-2,-2.1560509E-3,5.296984E-4,-9.1772905E-4,-1.0191596E-3,2.4340303E-4,-2.6812393E-2,-2.6082394E-3,-3.6587123E-2,-1.3471371E-2,3.872357E-4,-5.033193E-4,-8.9681585E-4,-2.0974001E-3,-7.295558E-5,-1.1766908E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,-1,3,5,7,-1,9,-1,11,13,15,-1,-1,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,-1,39,-1,-1,-1,41,43,-1,-1,-1,-1,-1,45,47,49,51,-1,-1,-1,-1,-1,-1],"loss_changes":[2.827161E-1,0E0,1.6295646E-1,6.387883E-2,1.5261681E-1,0E0,5.043647E-2,0E0,8.603448E-2,2.3576621E-2,1.0085214E-2,0E0,0E0,1.3942301E-2,0E0,1.6192202E-2,2.6914664E-3,2.5574304E-3,1.2432413E-2,0E0,9.200226E-3,0E0,0E0,0E0,0E0,1.1896358E-2,0E0,3.8381387E-3,2.6001576E-3,1.6141646E-2,5.478412E-3,0E0,3.8944897E-3,0E0,0E0,0E0,3.172724E-3,5.7994183E-3,0E0,0E0,0E0,0E0,0E0,3.028052E-3,1.8438308E-3,1.7715283E-3,2.0239658E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,8,8,9,9,10,10,13,13,15,15,16,16,17,17,18,18,20,20,25,25,27,27,28,28,29,29,30,30,32,32,36,36,37,37,43,43,44,44,45,45,46,46],"right_children":[2,-1,4,6,8,-1,10,-1,12,14,16,-1,-1,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,-1,40,-1,-1,-1,42,44,-1,-1,-1,-1,-1,46,48,50,52,-1,-1,-1,-1,-1,-1],"split_conditions":[8.461E3,-9.171418E-3,2.5406E4,6.7E1,6.6643E4,4.366773E-3,6.2521696E8,-6.041471E-4,2.613671E6,9.237E3,4.252951E-1,3.1406647E-3,1.0952773E-2,1.6E1,1.4420485E-3,1.0781E4,1.7307954E8,8.71E2,6.52E2,2.7089897E-3,1.0544393E10,-8.576633E-4,4.1190503E-4,-3.0923001E-3,-1.2962567E-3,1.47E2,-2.7331812E-3,1.28E2,1.76286E5,1.4899E4,3.9281719E3,8.3594245E-4,1.7326219E8,2.297321E-3,7.8962656E-4,2.1840334E-3,1.003136E6,5.530292E8,-2.1560509E-3,5.296984E-4,-9.1772905E-4,-1.0191596E-3,2.4340303E-4,5.583E3,2.0521914E10,4.888E3,4.6568102E8,3.872357E-4,-5.033193E-4,-8.9681585E-4,-2.0974001E-3,-7.295558E-5,-1.1766908E-3],"split_indices":[7,0,3,22,8,0,21,0,18,0,62,0,0,17,0,20,16,0,17,0,26,0,0,0,0,4,0,4,12,28,5,0,27,0,0,0,2,11,0,0,0,0,0,3,10,28,11,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,5E0,1.66E2,1.46E2,2E1,5E0,1.41E2,8E0,1.2E1,9E1,5.1E1,7E0,5E0,8.3E1,7E0,4E1,1.1E1,1.1E1,7.2E1,9E0,3.1E1,6E0,5E0,6E0,5E0,6.7E1,5E0,2E1,1.1E1,1.9E1,4.8E1,7E0,1.3E1,5E0,6E0,5E0,1.4E1,4.2E1,6E0,5E0,8E0,8E0,6E0,2.6E1,1.6E1,1.4E1,1.2E1,7E0,9E0,7E0,7E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.939928E-3,-1.3309541E-2,8.008521E-3,-8.135084E-3,-6.390095E-3,3.692708E-4,-7.460548E-3,-1.2567844E-2,3.7809096E-2,2.3362169E-3,-1.6224904E-2,-9.500499E-6,4.4945918E-2,-1.3323235E-2,-2.398178E-3,3.0185746E-3,3.5399914E-2,-2.6733719E-2,-3.7782684E-3,-2.9006047E-4,4.9027216E-2,-3.0103665E-2,-8.984363E-5,4.3870113E-3,-1.7710194E-2,1.3429546E-3,2.722831E-3,-2.1235698E-3,-2.3066128E-2,-2.1233717E-3,1.4631341E-3,-3.6130816E-2,3.9950456E-3,-7.847418E-3,-3.3989698E-2,9.836825E-4,-8.681107E-3,-6.8713416E-4,-2.352341E-3,-2.6174556E-4,5.8142655E-4,2.6847096E-4,-9.045714E-4,-2.0650355E-3,-6.439264E-4,2.7400898E-4,-1.5678305E-2,-1.1696457E-3,6.856594E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,-1,-1,5,7,-1,9,11,-1,13,-1,15,17,-1,-1,19,21,23,-1,25,27,-1,29,31,-1,-1,-1,33,35,-1,37,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,-1,-1],"loss_changes":[2.5949913E-1,1.8682887E-1,0E0,0E0,1.6911483E-1,7.167659E-2,0E0,2.7456192E-2,1.0322612E-2,0E0,1.1065001E-2,0E0,4.9481764E-3,1.2571251E-2,0E0,0E0,1.4649998E-2,3.347367E-3,6.7894207E-3,0E0,2.7139038E-3,3.5773441E-3,0E0,7.2790296E-3,8.940471E-3,0E0,0E0,0E0,4.388063E-3,5.370624E-3,0E0,3.4975503E-3,1.1049273E-3,2.4708817E-3,3.4025013E-3,0E0,2.819867E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.4945756E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,8,8,10,10,12,12,13,13,16,16,17,17,18,18,20,20,21,21,23,23,24,24,28,28,29,29,31,31,32,32,33,33,34,34,36,36,46,46],"right_children":[2,4,-1,-1,6,8,-1,10,12,-1,14,-1,16,18,-1,-1,20,22,24,-1,26,28,-1,30,32,-1,-1,-1,34,36,-1,38,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,-1,-1],"split_conditions":[1.0510942E1,2.58E3,8.008521E-3,-8.135084E-3,7.685281E3,1.0431E4,-7.460548E-3,1.66E2,1.4027468E10,2.3362169E-3,1E0,-9.500499E-6,1.90978E5,5.29E2,-2.398178E-3,3.0185746E-3,6.820745E6,9.183341E8,1.77E2,-2.9006047E-4,2.5891152E7,8.4E1,-8.984363E-5,8.9386E4,5.38E2,1.3429546E-3,2.722831E-3,-2.1235698E-3,2.710413E3,5.73E2,1.4631341E-3,2.0325911E3,2.01E2,5.89E2,2.95E2,9.836825E-4,2.1579E4,-6.8713416E-4,-2.352341E-3,-2.6174556E-4,5.8142655E-4,2.6847096E-4,-9.045714E-4,-2.0650355E-3,-6.439264E-4,2.7400898E-4,6.8718E4,-1.1696457E-3,6.856594E-5],"split_indices":[59,3,0,0,5,3,0,28,10,0,32,0,12,0,0,0,6,16,19,0,2,4,0,8,19,0,0,0,5,20,0,5,4,20,4,0,28,0,0,0,0,0,0,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.62E2,1.56E2,6E0,5E0,1.51E2,1.46E2,5E0,1.09E2,3.7E1,5E0,1.04E2,6E0,3.1E1,9.8E1,6E0,7E0,2.4E1,4E1,5.8E1,6E0,1.8E1,3.5E1,5E0,3.7E1,2.1E1,9E0,9E0,8E0,2.7E1,3.1E1,6E0,1.1E1,1E1,1.2E1,1.5E1,6E0,2.5E1,6E0,5E0,5E0,5E0,6E0,6E0,8E0,7E0,8E0,1.7E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.1073758E-2,-8.576922E-3,-4.578539E-3,-9.1603575E-3,5.4329704E-3,-2.7930685E-3,-7.883142E-3,-1.854042E-2,2.76267E-2,-2.3631504E-2,2.8501999E-2,-5.224786E-4,3.4475632E-2,-5.6331877E-2,-1.884948E-2,2.1764478E-3,1.04263774E-4,4.3725483E-3,2.4505822E-2,-1.1556766E-3,-3.2243566E-3,-3.7883192E-2,-1.3335133E-2,1.5802825E-2,2.4279885E-3,-1.4358562E-2,-3.7737186E-3,-2.7425135E-2,2.8668432E-3,2.7587313E-2,-7.1655697E-4,2.8111148E-5,-1.285541E-3,-3.554673E-2,-1.1543459E-2,1.375179E-3,-6.0557434E-3,3.951036E-4,3.68675E-2,-1.9173732E-3,-2.1646649E-2,1.1554861E-5,-1.1929506E-3,5.3572194E-3,-1.98173E-2,6.7908043E-4,1.9400929E-3,-3.3012507E-4,-1.4020258E-3,8.6478307E-4,-4.263591E-4,-1.3992927E-3,-2.8340204E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,-1,3,5,-1,7,-1,9,11,13,15,-1,17,19,21,-1,-1,-1,23,-1,-1,25,27,29,-1,31,-1,33,35,37,-1,-1,-1,39,41,-1,43,-1,45,-1,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1348865E-1,0E0,1.0343548E-1,1.8289776E-1,0E0,7.859423E-2,0E0,2.6539203E-2,1.5599839E-2,1.4262814E-2,6.017263E-3,0E0,2.898788E-2,4.328292E-3,8.626213E-3,0E0,0E0,0E0,1.0767717E-2,0E0,0E0,2.0470891E-2,1.5830345E-2,1.3742251E-2,0E0,3.1893947E-3,0E0,4.362097E-3,8.623728E-3,4.13849E-3,0E0,0E0,0E0,1.8454809E-3,2.617288E-3,0E0,4.189729E-3,0E0,2.0551085E-3,0E0,1.3798233E-3,0E0,0E0,3.473359E-3,1.7492315E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,18,18,21,21,22,22,23,23,25,25,27,27,28,28,29,29,33,33,34,34,36,36,38,38,40,40,43,43,44,44],"right_children":[2,-1,4,6,-1,8,-1,10,12,14,16,-1,18,20,22,-1,-1,-1,24,-1,-1,26,28,30,-1,32,-1,34,36,38,-1,-1,-1,40,42,-1,44,-1,46,-1,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3267355E8,-8.576922E-3,1.0362809E1,4.2250056E7,5.4329704E-3,1.7171203E9,-7.883142E-3,1.5063692E9,1.6849949E3,2.3E1,1.8345702E9,-5.224786E-4,2.0316896E3,1.2E1,1.1903926E9,2.1764478E-3,1.04263774E-4,4.3725483E-3,3.3765684E7,-1.1556766E-3,-3.2243566E-3,2.9939E4,2.616931E6,6.865654E10,2.4279885E-3,5.8704936E7,-3.7737186E-3,7.6086336E8,1.3358E4,1.45549E5,-7.1655697E-4,2.8111148E-5,-1.285541E-3,9.7E1,2.5705E4,1.375179E-3,1.852532E7,3.951036E-4,5.3E1,-1.9173732E-3,1.33274984E8,1.1554861E-5,-1.1929506E-3,5.6E1,1E0,6.7908043E-4,1.9400929E-3,-3.3012507E-4,-1.4020258E-3,8.6478307E-4,-4.263591E-4,-1.3992927E-3,-2.8340204E-4],"split_indices":[10,0,59,6,0,11,0,16,5,17,16,0,5,17,10,0,0,0,2,0,0,12,18,10,0,16,0,11,28,18,0,0,0,17,7,0,18,0,17,0,26,0,0,19,42,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.77E2,5E0,1.72E2,1.67E2,5E0,1.62E2,5E0,1.07E2,5.5E1,9.7E1,1E1,8E0,4.7E1,1.1E1,8.6E1,5E0,5E0,5E0,4.2E1,5E0,6E0,1.8E1,6.8E1,3.4E1,8E0,1.3E1,5E0,3.6E1,3.2E1,2.5E1,9E0,7E0,6E0,2.3E1,1.3E1,7E0,2.5E1,9E0,1.6E1,1.3E1,1E1,8E0,5E0,1.4E1,1.1E1,5E0,1.1E1,5E0,5E0,7E0,7E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.2485727E-3,-8.19129E-3,7.831554E-3,2.6847064E-3,6.537446E-3,-7.4618114E-3,3.4048684E-2,-2.7006768E-2,5.670364E-4,6.011054E-3,1.5824333E-2,3.9456136E-4,-3.4275137E-2,2.0141571E-3,-3.7806253E-3,3.61885E-2,-2.1513177E-3,-4.3019623E-2,4.0723218E-4,1.7383018E-3,-2.9283578E-2,9.647354E-3,5.5491395E-2,-2.113379E-3,-7.3976093E-4,-2.641252E-2,8.017998E-3,3.7811082E-4,-2.1440734E-3,1.4183533E-3,-6.363527E-4,3.1075943E-3,1.3480135E-3,-5.2341574E-4,-1.7999003E-3,1.1473905E-2,-1.0407834E-3,1.554095E-3,2.538118E-2,1.6649887E-2,-8.335874E-3,1.9743538E-3,8.15687E-3,2.5166554E-4,1.0979504E-3,2.9713288E-4,-1.0654671E-3,-4.7213695E-4,9.326388E-4,6.490469E-4,-4.1252404E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,15,-1,17,-1,19,21,-1,23,-1,25,27,29,31,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,37,-1,39,41,43,45,-1,47,-1,-1,49,-1,-1,-1,-1,-1],"loss_changes":[2.2159731E-1,0E0,1.2525906E-1,5.3399332E-2,0E0,1.9944593E-2,7.528372E-2,1.000385E-2,1.8207436E-2,0E0,4.951901E-2,0E0,1.2221854E-2,0E0,1.1873813E-2,1.3809644E-2,0E0,2.8398745E-3,0E0,1.258628E-2,1.1718183E-2,7.669977E-3,3.9851107E-3,0E0,0E0,2.2847028E-3,6.6604074E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.272304E-3,0E0,4.9162926E-3,7.154204E-3,1.0583953E-3,2.7827956E-3,0E0,3.4897847E-3,0E0,0E0,2.1294944E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,17,17,19,19,20,20,21,21,22,22,25,25,26,26,35,35,37,37,38,38,39,39,40,40,42,42,45,45],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,16,-1,18,-1,20,22,-1,24,-1,26,28,30,32,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,38,-1,40,42,44,46,-1,48,-1,-1,50,-1,-1,-1,-1,-1],"split_conditions":[2.711E3,-8.19129E-3,1.717542E6,3.857587E6,6.537446E-3,1.7015095E3,1.45957E5,2.731002E7,1E0,6.011054E-3,7.3373047E3,3.9456136E-4,1.43E2,2.0141571E-3,8.2025E4,6.490565E8,-2.1513177E-3,1.5070434E8,4.0723218E-4,2.63E2,5.79E4,6.2E1,3.64582E5,-2.113379E-3,-7.3976093E-4,5.735E3,4.3822035E-1,3.7811082E-4,-2.1440734E-3,1.4183533E-3,-6.363527E-4,3.1075943E-3,1.3480135E-3,-5.2341574E-4,-1.7999003E-3,2.5714466E8,-1.0407834E-3,8.2E1,1.0781E4,1.7287E4,6.1224E4,1.9743538E-3,7.0940457E9,2.5166554E-4,1.0979504E-3,2.3628674E3,-1.0654671E-3,-4.7213695E-4,9.326388E-4,6.490469E-4,-4.1252404E-4],"split_indices":[3,0,12,6,0,5,12,11,44,0,5,0,17,0,8,16,0,21,0,0,28,17,12,0,0,3,62,0,0,0,0,0,0,0,0,21,0,17,20,28,12,0,10,0,0,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,5E0,1.71E2,1.66E2,5E0,1.26E2,4E1,3.6E1,9E1,5E0,3.5E1,6E0,3E1,7E0,8.3E1,2.7E1,8E0,2.5E1,5E0,6.9E1,1.4E1,1.2E1,1.5E1,2E1,5E0,1.2E1,5.7E1,5E0,9E0,6E0,6E0,8E0,7E0,7E0,5E0,5.2E1,5E0,3.1E1,2.1E1,1.2E1,1.9E1,9E0,1.2E1,6E0,6E0,1.3E1,6E0,5E0,7E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-5.690708E-3,-9.013237E-3,1.3886307E-3,-1.012877E-2,6.600986E-2,3.0402113E-3,-1.4045592E-2,3.7182827E-2,5.0774235E-3,-8.606113E-3,-6.238619E-2,3.6007692E-3,8.463155E-4,-1.619637E-2,3.0477865E-2,-7.589888E-4,-5.215102E-3,-1.7443597E-3,1.8120814E-3,-3.26038E-2,-7.7677034E-3,1.8085241E-2,2.1105567E-3,-4.491146E-2,-1.0055234E-2,-1.1697594E-2,7.7549147E-4,1.3028058E-3,-1.12357004E-4,-2.245489E-3,-1.1447315E-3,4.5666166E-4,-1.7090457E-3,-5.101754E-3,-3.246417E-2,-8.451266E-4,-1.5480134E-3,-6.8610767E-4,-2.1542064E-3,-1.6500397E-2,1.5560162E-2,2.313635E-3,-3.3939395E-2,4.8184107E-4,1.2258359E-3,-8.7115495E-4,9.1981806E-4,-2.0657934E-3,-5.526079E-4,-5.8347575E-4,6.220323E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,-1,3,5,7,-1,9,11,-1,13,15,-1,17,19,21,-1,-1,-1,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,35,37,39,-1,-1,-1,41,43,45,47,49,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.504921E-1,0E0,1.2793852E-1,4.6602983E-2,3.3389248E-2,0E0,3.6275774E-2,2.9568685E-2,0E0,3.8169123E-2,3.3891313E-2,0E0,1.9917322E-2,1.454407E-2,3.7779808E-3,0E0,0E0,0E0,0E0,9.539127E-3,7.4330433E-3,3.4792265E-3,0E0,1.4591552E-3,8.768922E-3,8.463674E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.3033924E-3,3.4667049E-3,1.1557235E-2,0E0,0E0,0E0,7.624775E-3,4.1413363E-3,5.4347827E-3,2.819865E-3,2.288305E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,13,13,14,14,19,19,20,20,21,21,23,23,24,24,25,25,33,33,34,34,35,35,39,39,40,40,41,41,42,42,43,43],"right_children":[2,-1,4,6,8,-1,10,12,-1,14,16,-1,18,20,22,-1,-1,-1,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,36,38,40,-1,-1,-1,42,44,46,48,50,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.066E4,-9.013237E-3,1.5522031E9,1E0,6.5688E4,3.0402113E-3,3.89E2,4.053981E3,5.0774235E-3,1.4058E4,7.986E3,3.6007692E-3,5.518522E3,4.743E3,1.1630274E0,-7.589888E-4,-5.215102E-3,-1.7443597E-3,1.8120814E-3,9.7E1,3.56E2,3.876437E6,2.1105567E-3,1.2909914E7,5.6E1,1E0,7.7549147E-4,1.3028058E-3,-1.12357004E-4,-2.245489E-3,-1.1447315E-3,4.5666166E-4,-1.7090457E-3,1.082E3,8.078E3,7.85E2,-1.5480134E-3,-6.8610767E-4,-2.1542064E-3,5.72E2,2.16002E3,4.41E2,1.4659149E8,3.0491E4,1.2258359E-3,-8.7115495E-4,9.1981806E-4,-2.0657934E-3,-5.526079E-4,-5.8347575E-4,6.220323E-4],"split_indices":[7,0,16,30,3,0,4,5,0,3,3,0,5,3,66,0,0,0,0,17,4,18,0,27,19,35,0,0,0,0,0,0,0,19,3,0,0,0,0,20,5,22,21,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,5E0,1.7E2,1.45E2,2.5E1,6E0,1.39E2,1.7E1,8E0,1.26E2,1.3E1,7E0,1E1,1.06E2,2E1,8E0,5E0,5E0,5E0,3.5E1,7.1E1,1.3E1,7E0,2.2E1,1.3E1,6.2E1,9E0,8E0,5E0,1.5E1,7E0,8E0,5E0,4.8E1,1.4E1,4.3E1,5E0,8E0,6E0,2.2E1,2.1E1,1.1E1,1.1E1,1E1,1.1E1,5E0,6E0,6E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-4.634185E-3,-1.2291611E-2,8.7997615E-3,-8.120169E-3,-5.5301036E-3,3.45234E-3,-8.856146E-3,-1.522484E-2,3.129409E-2,-1.1368292E-2,-4.0886193E-3,2.443766E-3,8.106803E-3,-1.5994914E-2,2.678717E-2,-9.888799E-4,1.4541892E-3,-2.2482093E-2,1.8899535E-3,1.9128727E-3,5.43472E-4,-5.0183687E-2,-1.66206E-2,-2.4073642E-2,2.7735427E-2,-9.934526E-4,-2.963865E-3,7.5494166E-4,-1.9495714E-2,1.545915E-4,-1.6126722E-3,-6.038508E-4,4.788875E-2,-3.241883E-2,-7.492509E-3,2.9874644E-3,8.4456237E-4,-3.73005E-2,-4.5187716E-4,1.6044253E-2,-1.7548986E-2,-7.878713E-4,-1.9110184E-3,-7.583583E-5,1.3596867E-3,-1.121949E-3,3.0972195E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,-1,5,-1,7,9,11,13,-1,-1,15,17,19,-1,-1,21,23,-1,-1,25,27,29,31,-1,-1,-1,33,-1,-1,-1,35,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6464495E-1,1.8811652E-1,0E0,0E0,4.4610325E-2,0E0,3.9087832E-2,3.856863E-2,1.1571083E-2,2.2671346E-2,0E0,0E0,1.0061425E-2,1.3153018E-2,2.8739795E-3,0E0,0E0,1.26428455E-2,2.1469977E-2,0E0,0E0,5.2580796E-3,7.1633067E-3,5.8194343E-3,1.4067018E-2,0E0,0E0,0E0,9.839822E-3,0E0,0E0,0E0,5.1291157E-3,2.9530898E-3,8.565951E-3,0E0,0E0,2.3295581E-3,0E0,3.0091784E-3,5.207614E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,8,8,9,9,12,12,13,13,14,14,17,17,18,18,21,21,22,22,23,23,24,24,28,28,32,32,33,33,34,34,37,37,39,39,40,40],"right_children":[2,4,-1,-1,6,-1,8,10,12,14,-1,-1,16,18,20,-1,-1,22,24,-1,-1,26,28,30,32,-1,-1,-1,34,-1,-1,-1,36,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,-1,-1],"split_conditions":[1.717542E6,1.20184E5,8.7997615E-3,-8.120169E-3,1E0,3.45234E-3,2.92746E5,2.59434E5,7.0552E4,1.2538E4,-4.0886193E-3,2.443766E-3,1.3478712E11,6.2521696E8,3.876437E6,-9.888799E-4,1.4541892E-3,9.6E1,2.7616E4,1.9128727E-3,5.43472E-4,1.3390451E-2,1.35E2,1.7E1,3.2037274E8,-9.934526E-4,-2.963865E-3,7.5494166E-4,4.4854E4,1.545915E-4,-1.6126722E-3,-6.038508E-4,2.0815739E9,2.56E2,5.5589E4,2.9874644E-3,8.4456237E-4,1.4081E4,-4.5187716E-4,4.9791E4,2.856E4,-7.878713E-4,-1.9110184E-3,-7.583583E-5,1.3596867E-3,-1.121949E-3,3.0972195E-4],"split_indices":[12,16,0,0,37,0,12,12,8,3,0,0,10,21,18,0,0,4,7,0,0,62,0,17,16,0,0,0,12,0,0,0,23,17,12,0,0,12,0,12,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.65E2,1.6E2,5E0,5E0,1.55E2,5E0,1.5E2,1.3E2,2E1,1.25E2,5E0,9E0,1.1E1,1.12E2,1.3E1,5E0,6E0,8.2E1,3E1,5E0,8E0,1.3E1,6.9E1,1.5E1,1.5E1,6E0,7E0,5E0,6.4E1,5E0,1E1,5E0,1E1,3E1,3.4E1,5E0,5E0,2.4E1,6E0,1E1,2.4E1,7E0,1.7E1,5E0,5E0,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0549721E-3,-7.539728E-3,6.9739306E-3,1.3383554E-3,7.2639612E-3,7.1760267E-3,-7.000687E-3,1.5172622E-3,6.888512E-3,4.3497225E-3,-3.5907598E-3,-1.697542E-2,1.885584E-2,-1.3710534E-2,-2.251459E-3,9.988579E-3,4.6010908E-2,-1.8414581E-2,1.3141049E-2,1.6478367E-2,-6.273596E-4,2.6450318E-3,9.935752E-4,-2.543141E-2,-5.2268687E-3,1.1864573E-3,-4.413645E-4,3.773894E-2,4.275623E-3,-2.505821E-3,-2.0602299E-2,-1.8727763E-2,7.722423E-3,9.284635E-4,2.3309574E-3,-6.2156306E-4,1.2737683E-2,1.1117204E-4,-2.6460394E-2,-5.7628728E-5,-1.820845E-3,-3.2896723E-4,9.6095464E-4,1.8373677E-3,-3.0030285E-3,-1.5321783E-3,-1.5820899E-4,2.863644E-4,-5.962377E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,-1,3,5,-1,7,-1,9,-1,-1,11,13,15,17,-1,19,21,23,25,27,-1,-1,-1,29,31,-1,-1,33,35,-1,37,39,41,-1,-1,-1,43,-1,45,-1,-1,-1,-1,-1,47,-1,-1,-1,-1],"loss_changes":[1.8740064E-1,0E0,1.579311E-1,1.6259861E-1,0E0,1.4127204E-1,0E0,8.106716E-2,0E0,0E0,4.6587862E-2,1.0252921E-2,1.3506297E-2,1.1609985E-2,0E0,7.3515684E-3,3.3688378E-3,6.992342E-3,4.8461873E-3,9.178387E-3,0E0,0E0,0E0,6.5388903E-3,5.03978E-3,0E0,0E0,2.2175405E-3,3.9418647E-3,0E0,6.169904E-3,5.547633E-3,3.4377435E-3,0E0,0E0,0E0,7.982511E-3,0E0,6.9784597E-3,0E0,0E0,0E0,0E0,0E0,1.312754E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,23,23,24,24,27,27,28,28,30,30,31,31,32,32,36,36,38,38,44,44],"right_children":[2,-1,4,6,-1,8,-1,10,-1,-1,12,14,16,18,-1,20,22,24,26,28,-1,-1,-1,30,32,-1,-1,34,36,-1,38,40,42,-1,-1,-1,44,-1,46,-1,-1,-1,-1,-1,48,-1,-1,-1,-1],"split_conditions":[1.53656E5,-7.539728E-3,4.766891E9,4.831919E7,7.2639612E-3,9.0573E4,-7.000687E-3,2.2E1,6.888512E-3,4.3497225E-3,6.7179E4,3.5916E4,3.774E3,6.0700047E-1,-2.251459E-3,1.6946304E7,1.97E2,2.8E2,1.47E2,1.10745E5,-6.273596E-4,2.6450318E-3,9.935752E-4,1.1301477E3,1.9535E4,1.1864573E-3,-4.413645E-4,2.374017E9,2.377024E6,-2.505821E-3,4.386E3,1.0208E4,1.9515E4,9.284635E-4,2.3309574E-3,-6.2156306E-4,1.90978E5,1.1117204E-4,2.042779E6,-5.7628728E-5,-1.820845E-3,-3.2896723E-4,9.6095464E-4,1.8373677E-3,2.9607E4,-1.5321783E-3,-1.5820899E-4,2.863644E-4,-5.962377E-4],"split_indices":[16,0,16,6,0,3,0,20,0,0,28,20,0,63,0,18,19,4,4,28,0,0,0,5,7,0,0,21,6,0,3,28,28,0,0,0,12,0,2,0,0,0,0,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.8E2,5E0,1.75E2,1.7E2,5E0,1.65E2,5E0,1.6E2,5E0,7E0,1.53E2,9.6E1,5.7E1,8.9E1,7E0,4.4E1,1.3E1,7.6E1,1.3E1,3.5E1,9E0,7E0,6E0,4.9E1,2.7E1,8E0,5E0,1.2E1,2.3E1,5E0,4.4E1,1.3E1,1.4E1,7E0,5E0,7E0,1.6E1,9E0,3.5E1,8E0,5E0,7E0,7E0,5E0,1.1E1,2.5E1,1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.0909393E-3,-7.9681305E-3,8.55765E-3,-4.1595043E-4,1.0921989E-1,3.6089586E-3,-3.331749E-3,5.024903E-4,1.0369525E-2,-1.478519E-2,1.8456496E-2,-2.6496608E-2,-3.639023E-3,5.5689994E-2,1.08547285E-2,-4.3191236E-2,-8.243359E-3,1.6350934E-2,-1.710313E-2,1.1504862E-3,3.945897E-3,-4.8732543E-3,2.6281191E-2,-1.2812787E-3,-2.404376E-3,-8.974705E-4,3.572691E-4,-1.00392775E-4,1.2675298E-3,-2.9120501E-2,3.3029082E-4,1.29080545E-2,-3.3737317E-2,3.5156548E-2,-1.8555028E-4,-3.7852564E-4,-1.8239582E-3,2.666379E-2,-4.5221825E-3,-2.2042026E-3,-1.2959923E-4,4.6000823E-2,1.0020263E-4,3.1839914E-4,2.0970178E-3,-1.0591284E-3,6.972663E-4,2.4751673E-2,3.379875E-3,3.1793298E-4,1.7973166E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,-1,3,5,7,9,-1,-1,-1,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,35,-1,37,39,41,-1,-1,-1,43,45,-1,-1,47,-1,-1,-1,-1,-1,49,-1,-1,-1],"loss_changes":[2.1171975E-1,0E0,1.5451021E-1,4.8554547E-2,1.7284542E-1,4.151216E-2,0E0,0E0,0E0,8.780122E-3,2.3170896E-2,9.700093E-3,9.968942E-3,1.18721835E-2,1.73546E-2,1.1808425E-3,3.608793E-3,3.6169104E-3,6.742795E-3,0E0,0E0,1.8912151E-2,9.904938E-3,0E0,0E0,0E0,0E0,0E0,0E0,3.5232594E-3,0E0,5.6351903E-3,7.0709474E-3,9.872235E-3,0E0,0E0,0E0,5.033225E-3,4.8392625E-3,0E0,0E0,1.2111027E-2,0E0,0E0,0E0,0E0,0E0,3.7171496E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,29,29,31,31,32,32,33,33,37,37,38,38,41,41,47,47],"right_children":[2,-1,4,6,8,10,-1,-1,-1,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,36,-1,38,40,42,-1,-1,-1,44,46,-1,-1,48,-1,-1,-1,-1,-1,50,-1,-1,-1],"split_conditions":[1.8891E4,-7.9681305E-3,1.338554E0,1.5357575E0,4.744117E3,1.664399E6,-3.331749E-3,5.024903E-4,1.0369525E-2,8.5E1,1.2236E4,2.61E2,1.47E2,5.3507E4,2.423267E6,1.351E4,3.33044E5,9.163062E6,7.115E3,1.1504862E-3,3.945897E-3,3.282958E6,1E0,-1.2812787E-3,-2.404376E-3,-8.974705E-4,3.572691E-4,-1.00392775E-4,1.2675298E-3,3.3732E4,3.3029082E-4,8.233E3,8.29E4,2.008E3,-1.8555028E-4,-3.7852564E-4,-1.8239582E-3,7.2032E4,2.644432E6,-2.2042026E-3,-1.2959923E-4,3.4357E4,1.0020263E-4,3.1839914E-4,2.0970178E-3,-1.0591284E-3,6.972663E-4,1.174443E0,3.379875E-3,3.1793298E-4,1.7973166E-3],"split_indices":[8,0,63,65,5,9,0,0,0,17,28,4,4,8,6,28,6,18,3,0,0,2,35,0,0,0,0,0,0,12,0,3,8,19,0,0,0,8,2,0,0,7,0,0,0,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.75E2,5E0,1.7E2,1.57E2,1.3E1,1.5E2,7E0,8E0,5E0,6.7E1,8.3E1,3.2E1,3.5E1,1.3E1,7E1,1.6E1,1.6E1,1.4E1,2.1E1,8E0,5E0,3.5E1,3.5E1,9E0,7E0,9E0,7E0,6E0,8E0,1.4E1,7E0,2.2E1,1.3E1,2.7E1,8E0,6E0,8E0,1.2E1,1E1,8E0,5E0,2E1,7E0,7E0,5E0,5E0,5E0,1.3E1,7E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-5.3083203E-3,-1.0153318E-2,4.9068783E-3,-5.1622917E-3,-6.3236738E-3,2.983373E-3,-7.903965E-3,-1.3365188E-2,2.3372296E-2,-2.4103478E-2,-2.5888262E-3,-2.4513554E-4,3.821763E-2,-1.09894825E-2,-3.247149E-2,6.713949E-3,-1.4208808E-2,2.6786474E-3,5.84459E-4,-1.6367196E-3,3.5413334E-3,-2.4174433E-3,-2.4295863E-2,-1.1208611E-2,1.5094665E-2,-2.7247563E-2,5.94082E-3,8.503526E-4,-5.314799E-3,-3.0872442E-2,-1.3635084E-2,3.5913225E-4,-1.0643338E-3,1.4212077E-3,4.5522936E-3,1.1335091E-4,-3.8091835E-2,-4.2078024E-4,1.1228537E-3,-5.5772334E-4,1.2691338E-4,-7.570407E-4,-1.6288273E-3,-1.0791185E-3,-9.724783E-5,-5.757207E-4,1.0370768E-3,-2.155006E-3,-8.623185E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,-1,-1,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,-1,29,31,33,35,37,-1,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0096208E-1,1.1289388E-1,0E0,3.3595E-2,0E0,0E0,2.7470628E-2,1.567548E-2,1.0542941E-2,7.105343E-3,7.5541646E-3,0E0,8.375403E-3,1.0918569E-2,6.335411E-3,6.0271416E-3,8.30638E-3,0E0,0E0,0E0,2.7859155E-3,0E0,1.8226188E-3,3.5300422E-3,4.823591E-3,6.2481314E-3,4.2556217E-3,0E0,8.349254E-4,1.0094848E-3,1.7157665E-3,0E0,0E0,0E0,6.447941E-3,0E0,1.920484E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,20,20,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,34,34,36,36],"right_children":[2,4,-1,6,-1,-1,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,-1,30,32,34,36,38,-1,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.74639E5,8.504903E3,4.9068783E-3,1.5E1,-6.3236738E-3,2.983373E-3,1.109494E9,5.0929E4,9.706545E6,1.05E4,7.1E1,-2.4513554E-4,2.0901188E7,1.84E2,9.319117E8,1.20343E5,3.456766E3,2.6786474E-3,5.84459E-4,-1.6367196E-3,2.7E1,-2.4174433E-3,3.8956667E9,2.2E1,5.14717E5,3.7129E4,2.0822124E9,8.503526E-4,5.4674416E7,1.3733394E3,2.2038938E3,3.5913225E-4,-1.0643338E-3,1.4212077E-3,3.5E1,1.1335091E-4,1.21E2,-4.2078024E-4,1.1228537E-3,-5.5772334E-4,1.2691338E-4,-7.570407E-4,-1.6288273E-3,-1.0791185E-3,-9.724783E-5,-5.757207E-4,1.0370768E-3,-2.155006E-3,-8.623185E-4],"split_indices":[12,5,0,20,0,0,16,12,18,28,19,0,18,22,10,27,5,0,0,0,17,0,10,17,27,18,11,0,16,5,5,0,0,0,19,0,4,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,1.68E2,6E0,1.63E2,5E0,5E0,1.58E2,1.35E2,2.3E1,6.7E1,6.8E1,8E0,1.5E1,2.7E1,4E1,3.8E1,3E1,7E0,8E0,9E0,1.8E1,9E0,3.1E1,1.2E1,2.6E1,1.8E1,1.2E1,6E0,1.2E1,1.8E1,1.3E1,5E0,7E0,9E0,1.7E1,5E0,1.3E1,7E0,5E0,6E0,6E0,7E0,1.1E1,6E0,7E0,9E0,8E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[3.697578E-3,-7.423016E-3,9.343805E-3,5.3911326E-3,5.4645734E-3,-6.0325516E-3,4.9751718E-2,-1.7192427E-2,9.251092E-3,7.569352E-3,2.0221192E-2,-2.1974964E-3,-2.3254815E-2,5.8913864E-3,1.7051322E-3,4.502505E-2,-3.1099634E-3,1.6675494E-3,-1.4208499E-2,-2.6235413E-2,3.2501077E-4,2.5726093E-2,4.7121165E-4,3.238324E-2,2.9610505E-3,-1.07722E-3,-5.170593E-3,-2.283274E-2,-2.167128E-3,1.8967781E-3,2.9910108E-4,1.4951263E-3,-5.604632E-3,2.1847964E-3,3.993058E-4,-5.7652773E-4,2.2125474E-4,-3.721494E-2,-1.5393095E-2,7.900761E-3,-1.3483683E-2,-2.01348E-3,-6.4013316E-4,-2.922742E-2,-6.518581E-3,-2.8648655E-4,9.3306886E-4,-2.6557686E-2,5.5474555E-4,-5.9455214E-4,-1.7745806E-3,-1.6192883E-2,5.4277387E-4,-4.2307626E-2,4.0890506E-4,-1.1987495E-4,-1.3603811E-3,-2.4587335E-3,-1.05811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,-1,3,-1,5,7,9,11,13,-1,15,17,19,21,-1,23,-1,-1,25,27,-1,29,31,33,-1,-1,35,37,-1,-1,-1,-1,39,-1,-1,-1,-1,41,43,45,47,-1,-1,49,51,-1,-1,53,-1,-1,-1,55,-1,57,-1,-1,-1,-1,-1],"loss_changes":[1.8702249E-1,0E0,8.0819346E-2,0E0,9.144243E-2,2.454964E-2,1.3161938E-1,7.4940193E-3,5.880273E-3,0E0,7.354428E-2,1.2674834E-2,5.5943206E-3,6.008302E-3,0E0,5.0139576E-3,0E0,0E0,1.7652549E-3,3.168866E-3,0E0,3.71797E-3,9.428919E-3,6.9333017E-3,0E0,0E0,1.054685E-3,4.6700723E-3,0E0,0E0,0E0,0E0,4.256509E-3,0E0,0E0,0E0,0E0,2.5402047E-3,3.9837584E-3,3.0643633E-3,8.8989E-3,0E0,0E0,1.7650407E-3,4.2354655E-3,0E0,0E0,1.001555E-2,0E0,0E0,0E0,2.9347944E-3,0E0,1.6746651E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,18,18,19,19,21,21,22,22,23,23,26,26,27,27,32,32,37,37,38,38,39,39,40,40,43,43,44,44,47,47,51,51,53,53],"right_children":[2,-1,4,-1,6,8,10,12,14,-1,16,18,20,22,-1,24,-1,-1,26,28,-1,30,32,34,-1,-1,36,38,-1,-1,-1,-1,40,-1,-1,-1,-1,42,44,46,48,-1,-1,50,52,-1,-1,54,-1,-1,-1,56,-1,58,-1,-1,-1,-1,-1],"split_conditions":[2.618E3,-7.423016E-3,2.2E1,5.3911326E-3,6.820745E6,2.216174E6,2.5296833E10,1.05E4,1.5522031E9,7.569352E-3,1.1512978E0,1.41E2,1.1639729E9,3.2322E4,1.7051322E-3,6.66367E5,-3.1099634E-3,1.6675494E-3,4.1E1,8.221E4,3.2501077E-4,4.678294E6,3.7E1,3.64582E5,2.9610505E-3,-1.07722E-3,1.5782253E8,2.35243E5,-2.167128E-3,1.8967781E-3,2.9910108E-4,1.4951263E-3,3.282958E6,2.1847964E-3,3.993058E-4,-5.7652773E-4,2.2125474E-4,1.0696113E7,6.9E1,5.6725E4,5.5015055E9,-2.01348E-3,-6.4013316E-4,2.2708E4,5.78082E5,-2.8648655E-4,9.3306886E-4,3.857587E6,5.5474555E-4,-5.9455214E-4,-1.7745806E-3,1.2E1,5.4277387E-4,1.84E2,4.0890506E-4,-1.1987495E-4,-1.3603811E-3,-2.4587335E-3,-1.05811E-3],"split_indices":[3,0,20,0,6,9,10,28,16,0,63,4,11,18,0,12,0,0,17,8,0,2,4,12,0,0,11,6,0,0,0,0,2,0,0,0,0,27,17,28,11,0,0,28,6,0,0,6,0,0,0,19,0,19,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,5E0,1.83E2,5E0,1.78E2,1.42E2,3.6E1,8.2E1,6E1,6E0,3E1,2.4E1,5.8E1,5.5E1,5E0,2.4E1,6E0,5E0,1.9E1,5.3E1,5E0,1.1E1,4.4E1,1.7E1,7E0,8E0,1.1E1,4.8E1,5E0,5E0,6E0,6E0,3.8E1,9E0,8E0,6E0,5E0,1.5E1,3.3E1,1.4E1,2.4E1,1E1,5E0,1.2E1,2.1E1,7E0,7E0,1.6E1,8E0,6E0,6E0,1.4E1,7E0,1.1E1,5E0,8E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.7637893E-3,-8.486776E-3,8.931279E-3,1.2840953E-4,9.636456E-2,4.6027894E-3,-4.230821E-3,4.865581E-4,9.70411E-3,-6.7392393E-4,-3.2410559E-3,-8.093522E-3,4.661267E-2,-1.8386982E-2,1.0337918E-2,5.992157E-2,3.4000038E-4,-5.0548635E-2,-1.2352253E-2,2.214295E-3,3.858371E-3,1.2881032E-3,3.758666E-3,-9.386509E-4,-2.9776571E-3,-1.568867E-2,4.7541494E-4,2.410844E-2,-3.955737E-3,-1.0648989E-2,-3.2702237E-2,2.4251848E-4,1.6866233E-3,-1.3565432E-3,4.4714403E-3,-1.6790528E-2,4.7911258E-4,-1.8835871E-3,-5.8114657E-4,-1.0912378E-2,1.2925735E-3,-5.8465567E-3,-2.5800576E-2,1.9053814E-4,-9.420158E-4,3.2575033E-4,-6.5448345E-4,-1.4602985E-3,-2.9687624E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,-1,3,5,7,-1,9,-1,-1,11,-1,13,15,17,19,21,-1,23,25,-1,27,-1,-1,-1,-1,29,-1,31,33,35,37,-1,-1,-1,39,41,-1,-1,-1,43,-1,45,47,-1,-1,-1,-1,-1,-1],"loss_changes":[2.394013E-1,0E0,1.262125E-1,6.931765E-2,1.5854008E-1,0E0,3.6065567E-2,0E0,0E0,4.914507E-2,0E0,2.3146603E-2,9.35033E-3,1.4514385E-2,1.132061E-2,8.610163E-3,0E0,4.6921466E-3,5.366971E-3,0E0,6.26658E-3,0E0,0E0,0E0,0E0,4.7105895E-3,0E0,2.7512806E-3,6.7678024E-3,6.426704E-3,2.059917E-3,0E0,0E0,0E0,9.21941E-3,3.497568E-3,0E0,0E0,0E0,2.5394934E-3,0E0,2.3036618E-3,2.6817229E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,9,9,11,11,12,12,13,13,14,14,15,15,17,17,18,18,20,20,25,25,27,27,28,28,29,29,30,30,34,34,35,35,39,39,41,41,42,42],"right_children":[2,-1,4,6,8,-1,10,-1,-1,12,-1,14,16,18,20,22,-1,24,26,-1,28,-1,-1,-1,-1,30,-1,32,34,36,38,-1,-1,-1,40,42,-1,-1,-1,44,-1,46,48,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0045E4,-8.486776E-3,1.3336757E0,2.2E1,6.5078E4,4.6027894E-3,5.09E2,4.865581E-4,9.70411E-3,2.4455E4,-3.2410559E-3,6.331471E8,1.33865185E11,9.6E1,7.8E2,5.8E1,3.4000038E-4,9.489291E7,5.530237E9,2.214295E-3,1.1411912E-1,1.2881032E-3,3.758666E-3,-9.386509E-4,-2.9776571E-3,8.489E3,4.7541494E-4,1.0122584E-1,4.1920558E-1,5.80292E5,2.44E2,2.4251848E-4,1.6866233E-3,-1.3565432E-3,3.0303E4,7.58454E5,4.7911258E-4,-1.8835871E-3,-5.8114657E-4,1.00495E5,1.2925735E-3,5.3471E4,1.217902E6,1.9053814E-4,-9.420158E-4,3.2575033E-4,-6.5448345E-4,-1.4602985E-3,-2.9687624E-4],"split_indices":[8,0,63,20,3,0,4,0,0,3,0,21,26,4,0,17,0,26,11,0,62,0,0,0,0,3,0,63,63,6,17,0,0,0,7,2,0,0,0,28,0,8,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,5E0,1.63E2,1.49E2,1.4E1,5E0,1.44E2,9E0,5E0,1.38E2,6E0,1.2E2,1.8E1,7.7E1,4.3E1,1.3E1,5E0,1.1E1,6.6E1,5E0,3.8E1,7E0,6E0,5E0,6E0,5.8E1,8E0,1E1,2.8E1,4.6E1,1.2E1,5E0,5E0,6E0,2.2E1,3.6E1,1E1,7E0,5E0,1.4E1,8E0,1.7E1,1.9E1,6E0,8E0,7E0,1E1,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.1589504E-3,-8.147115E-3,8.1512E-3,1.2401646E-3,4.9052075E-3,-1.2252523E-2,1.23086795E-2,-6.1363503E-3,-2.2847063E-3,2.5072305E-3,7.4594985E-3,1.5797674E-3,-1.0562194E-2,-7.987654E-3,1.8621644E-2,-4.0415686E-2,-3.1937019E-3,1.6321986E-3,-1.7427342E-2,2.564526E-2,-5.4191025E-3,-7.6095044E-4,-2.4730868E-3,-8.428916E-3,7.1999803E-4,-6.280658E-3,-1.7572854E-3,1.3856496E-2,2.7803208E-3,1.3686093E-3,-1.8022424E-3,8.401815E-3,-1.684374E-2,-9.738541E-4,6.2663057E-3,-3.800802E-3,2.5505383E-2,-3.043757E-4,1.2003798E-3,-1.414309E-3,-1.1333653E-2,7.836631E-4,-3.6726377E-4,-1.1148488E-3,9.688703E-4,3.4283887E-4,1.4056583E-3,7.3840245E-5,-1.7036814E-2,-1.1593682E-3,-1.622167E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,-1,3,5,-1,7,9,11,-1,-1,13,-1,15,17,19,21,23,-1,25,27,29,-1,-1,31,-1,33,-1,35,-1,-1,-1,37,39,-1,41,43,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,49,-1,-1],"loss_changes":[2.1952496E-1,0E0,1.1583783E-1,2.2851102E-2,0E0,1.6598536E-2,1.7956309E-2,1.1810553E-2,0E0,0E0,1.3425343E-2,0E0,1.2189535E-2,1.4788999E-2,7.737358E-3,3.1189565E-3,4.880706E-3,0E0,6.8738014E-3,1.4819093E-2,1.5806176E-2,0E0,0E0,5.381741E-3,0E0,4.16273E-3,0E0,5.856202E-3,0E0,0E0,0E0,4.0095476E-3,1.7761285E-3,0E0,2.2101908E-3,7.3500425E-3,1.7573806E-3,0E0,0E0,0E0,1.5145435E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6776482E-3,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,10,10,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,25,25,27,27,31,31,32,32,34,34,35,35,36,36,40,40,48,48],"right_children":[2,-1,4,6,-1,8,10,12,-1,-1,14,-1,16,18,20,22,24,-1,26,28,30,-1,-1,32,-1,34,-1,36,-1,-1,-1,38,40,-1,42,44,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,50,-1,-1],"split_conditions":[1.72232E5,-8.147115E-3,7.9463E4,1.882904E6,4.9052075E-3,1.509977E8,4.68E2,2.715E3,-2.2847063E-3,2.5072305E-3,4.503171E6,1.5797674E-3,2.5E1,3.4416284E9,3.0409E4,1.9829087E3,3.432137E7,1.6321986E-3,1.259E3,8.9422E4,7.0378E4,-7.6095044E-4,-2.4730868E-3,4.1715623E-4,7.1999803E-4,2.601885E6,-1.7572854E-3,2.8735796E-2,2.7803208E-3,1.3686093E-3,-1.8022424E-3,5.9816E4,1.1301477E3,-9.738541E-4,8.9E1,2.53E2,3.443987E10,-3.043757E-4,1.2003798E-3,-1.414309E-3,1.2689E4,7.836631E-4,-3.6726377E-4,-1.1148488E-3,9.688703E-4,3.4283887E-4,1.4056583E-3,7.3840245E-5,7.1955553E9,-1.1593682E-3,-1.622167E-4],"split_indices":[16,0,3,2,0,27,20,3,0,0,2,0,17,10,3,5,27,0,0,8,8,0,0,62,0,2,0,62,0,0,0,27,5,0,17,4,10,0,0,0,28,0,0,0,0,0,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.65E2,5E0,1.6E2,1.51E2,9E0,6.8E1,8.3E1,6E1,8E0,7E0,7.6E1,5E0,5.5E1,3.2E1,4.4E1,1E1,4.5E1,5E0,2.7E1,3.4E1,1E1,5E0,5E0,3.6E1,9E0,1.9E1,8E0,2.7E1,7E0,5E0,5E0,1.2E1,2.4E1,8E0,1.1E1,1.1E1,1.6E1,7E0,5E0,5E0,1.9E1,6E0,5E0,6E0,5E0,5E0,1.1E1,6E0,1.3E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-1.3698239E-3,-7.4856128E-3,4.825168E-3,3.8221015E-3,6.050859E-4,-9.412718E-3,3.0392926E-2,-4.568046E-3,-2.973133E-3,4.1987948E-2,-1.8467098E-3,-1.1998665E-2,1.2613023E-2,2.0477165E-2,4.090467E-3,-1.5689097E-2,1.1905709E-2,3.576635E-2,1.4755391E-3,3.485716E-2,5.661658E-3,-1.1927423E-2,-1.8510978E-3,1.5637099E-3,-6.110227E-4,5.032966E-4,2.3587034E-3,-8.576906E-3,1.4430701E-3,6.356841E-4,2.1897873E-3,-6.400733E-4,8.11115E-4,-1.524663E-3,-6.7318366E-3,1.0646401E-2,-1.3582516E-3,1.0650998E-3,-1.064651E-2,-4.2070882E-4,1.272627E-3,-1.4022218E-2,6.195788E-4,-1.3462168E-3,-1.025736E-2,-7.2912604E-4,-1.7233854E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,-1,3,-1,5,7,9,11,-1,13,-1,15,17,19,-1,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,-1,-1,43,-1,-1,45,-1,-1],"loss_changes":[1.8000388E-1,0E0,5.7562415E-2,0E0,4.743466E-2,3.3533264E-2,3.54963E-2,1.432226E-2,0E0,3.7026726E-2,0E0,7.067755E-3,8.714797E-3,5.3418707E-3,0E0,6.32938E-3,7.3178275E-3,4.2485343E-3,7.904168E-3,3.309084E-3,3.949714E-3,7.0556216E-3,0E0,0E0,0E0,0E0,0E0,8.56467E-3,0E0,0E0,0E0,0E0,0E0,0E0,6.4757485E-3,4.4115414E-3,0E0,0E0,4.0565403E-3,0E0,0E0,2.2801245E-3,0E0,0E0,1.3577724E-3,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,27,27,34,34,35,35,38,38,41,41,44,44],"right_children":[2,-1,4,-1,6,8,10,12,-1,14,-1,16,18,20,-1,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,-1,-1,44,-1,-1,46,-1,-1],"split_conditions":[1.7E1,-7.4856128E-3,2.2E1,3.8221015E-3,1.54417E5,4.3069697E3,1.2458633E11,1.09274E6,-2.973133E-3,7.030546E10,-1.8467098E-3,2.7129277E9,2.288E3,1.8340538E9,4.090467E-3,8.221E4,3.4974E4,8.7975E4,3.147503E6,9.886422E6,2.6572972E9,2.3E1,-1.8510978E-3,1.5637099E-3,-6.110227E-4,5.032966E-4,2.3587034E-3,2.2672263E10,1.4430701E-3,6.356841E-4,2.1897873E-3,-6.400733E-4,8.11115E-4,-1.524663E-3,4.286E3,7.428226E9,-1.3582516E-3,1.0650998E-3,8.078E3,-4.2070882E-4,1.272627E-3,8.131426E7,6.195788E-4,-1.3462168E-3,2.45009E6,-7.2912604E-4,-1.7233854E-4],"split_indices":[4,0,20,0,12,5,23,6,0,23,0,26,20,27,0,8,22,12,6,6,27,17,0,0,0,0,0,10,0,0,0,0,0,0,28,10,0,0,3,0,0,11,0,0,18,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.69E2,5E0,1.64E2,7E0,1.57E2,1.18E2,3.9E1,1.1E2,8E0,3.4E1,5E0,7.7E1,3.3E1,2.5E1,9E0,6.7E1,1E1,1E1,2.3E1,1.2E1,1.3E1,6E1,7E0,5E0,5E0,5E0,5E0,1.8E1,5E0,6E0,6E0,5E0,8E0,1E1,5E1,1E1,8E0,5E0,4.5E1,5E0,5E0,4E1,5E0,6E0,3.4E1,1.6E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.0178126E-3,-7.4042995E-3,3.773888E-3,-1.3268543E-3,6.4080586E-3,3.7962762E-3,-5.6793457E-3,-2.3692267E-3,-3.98704E-3,-1.096252E-2,2.7897753E-2,-2.5626212E-2,-4.8391623E-3,2.829569E-3,1.455422E-2,-3.410293E-2,-9.86973E-3,1.9457728E-3,-8.353317E-3,2.2917924E-2,-8.091438E-4,-4.3089828E-4,-4.207417E-2,3.3687547E-4,-8.94691E-4,-1.1208024E-2,8.646698E-4,-9.7788055E-5,3.4807853E-2,-2.2845392E-3,-1.0466931E-3,-2.8064065E-3,-2.0465197E-2,2.4042227E-3,4.4438144E-4,-1.4715013E-2,8.147163E-3,2.0501774E-4,-2.467517E-2,-1.4658027E-3,-1.2141544E-3,-3.918495E-4,1.5130099E-2,-1.1347697E-2,-3.8134627E-2,-6.73651E-4,5.5635837E-4,1.0742994E-3,1.1217753E-4,-8.8483654E-4,3.6634566E-4,-8.8520953E-4,-2.4285712E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,-1,3,5,-1,-1,7,9,-1,11,13,15,17,-1,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,-1,-1,33,-1,-1,35,37,-1,-1,39,41,-1,43,45,-1,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7441739E-1,0E0,1.2714869E-1,6.544769E-2,0E0,0E0,4.572224E-2,4.1140437E-2,0E0,1.0920288E-2,1.6304048E-2,4.4716895E-3,1.5655002E-2,0E0,7.897836E-3,4.027026E-3,2.7753126E-3,0E0,6.895204E-3,6.6588707E-3,0E0,0E0,1.9028429E-3,0E0,0E0,5.8534844E-3,0E0,0E0,6.928956E-3,0E0,0E0,5.4679974E-3,3.9294893E-3,0E0,0E0,3.4312145E-3,2.7666353E-3,0E0,5.1280577E-3,2.3805194E-3,0E0,0E0,1.8425044E-3,3.1653398E-3,3.5598557E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,22,22,25,25,28,28,31,31,32,32,35,35,36,36,38,38,39,39,42,42,43,43,44,44],"right_children":[2,-1,4,6,-1,-1,8,10,-1,12,14,16,18,-1,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,-1,-1,34,-1,-1,36,38,-1,-1,40,42,-1,44,46,-1,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2E1,-7.4042995E-3,7.034231E11,2.2E1,6.4080586E-3,3.7962762E-3,6.685129E7,1.2535E4,-3.98704E-3,4.3405E5,3.1E1,2.616974E6,1.6004977E7,2.829569E-3,1.7782382E0,3.3219E4,1.445E3,1.9457728E-3,1E0,1.492E3,-8.091438E-4,-4.3089828E-4,5.3781E4,3.3687547E-4,-8.94691E-4,1.01E2,8.646698E-4,-9.7788055E-5,3.44675E5,-2.2845392E-3,-1.0466931E-3,1.24E2,9.301267E-3,2.4042227E-3,4.4438144E-4,7.0643283E8,2.221362E6,2.0501774E-4,1.79E2,2.4E1,-1.2141544E-3,-3.918495E-4,2.7968428E-1,2.8096E4,1.7681605E8,-6.73651E-4,5.5635837E-4,1.0742994E-3,1.1217753E-4,-8.8483654E-4,3.6634566E-4,-8.8520953E-4,-2.4285712E-3],"split_indices":[4,0,13,20,0,0,2,3,0,27,19,18,23,0,66,18,28,0,43,0,0,0,8,0,0,17,0,0,12,0,0,4,63,0,0,26,27,0,4,17,0,0,63,7,16,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.78E2,5E0,1.73E2,1.68E2,5E0,7E0,1.61E2,1.56E2,5E0,1.22E2,3.4E1,3.5E1,8.7E1,8E0,2.6E1,2.2E1,1.3E1,5E0,8.2E1,2.1E1,5E0,6E0,1.6E1,5E0,8E0,7.5E1,7E0,7E0,1.4E1,9E0,7E0,4E1,3.5E1,7E0,7E0,1.9E1,2.1E1,5E0,3E1,1E1,9E0,6E0,1.5E1,1.6E1,1.4E1,5E0,5E0,8E0,7E0,1.1E1,5E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.9769543E-4,-5.050554E-3,3.3555545E-3,-1.4226246E-3,-4.432704E-3,2.9745405E-3,-4.5554386E-3,-1.6816212E-2,1.0969819E-2,1.11526E-3,-1.971124E-2,7.1065263E-3,1.5684734E-3,-1.6197456E-2,-2.685141E-3,1.756735E-2,-6.459161E-4,-1.8799597E-3,-1.2457335E-2,1.8441159E-3,2.0143166E-3,-1.0141964E-3,6.638373E-3,-1.5593785E-3,-1.0081148E-2,4.964314E-4,-5.6658965E-4,-2.750085E-3,1.938549E-2,-6.0436735E-3,-2.707323E-2,6.4738193E-3,-9.052173E-4,1.5808915E-3,9.543625E-5,-1.2880032E-2,5.880319E-3,-2.022351E-3,-3.9169727E-6,6.8265916E-4,-1.0074391E-4,1.01092686E-4,-1.557993E-2,1.290399E-2,-5.5252254E-4,-9.252574E-4,-1.9459752E-4,-1.2742697E-4,9.2128536E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,-1,-1,7,9,11,-1,13,15,-1,17,-1,19,21,-1,23,-1,25,-1,27,-1,29,-1,-1,31,33,35,37,39,-1,-1,-1,41,43,-1,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1],"loss_changes":[6.331128E-2,5.7975944E-2,0E0,3.5401743E-2,0E0,0E0,2.9688979E-2,1.1174792E-2,6.399652E-3,0E0,1.1049792E-2,4.9582594E-3,0E0,7.2030146E-3,0E0,9.896346E-3,6.0782484E-3,0E0,3.4550717E-3,0E0,2.5336023E-3,0E0,3.4042336E-3,0E0,4.2271046E-3,0E0,0E0,2.9551259E-3,3.3952328E-3,4.396123E-3,6.232622E-3,1.002118E-3,0E0,0E0,0E0,1.4310423E-3,2.76762E-3,0E0,0E0,0E0,0E0,0E0,1.7176648E-3,2.008789E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,10,10,11,11,13,13,15,15,16,16,18,18,20,20,22,22,24,24,27,27,28,28,29,29,30,30,31,31,35,35,36,36,42,42,43,43],"right_children":[2,4,-1,6,-1,-1,8,10,12,-1,14,16,-1,18,-1,20,22,-1,24,-1,26,-1,28,-1,30,-1,-1,32,34,36,38,40,-1,-1,-1,42,44,-1,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1],"split_conditions":[1.3972062E11,4.3821412E7,3.3555545E-3,2.2E1,-4.432704E-3,2.9745405E-3,1.09274E6,2.8182E4,1.5385991E7,1.11526E-3,1.035E3,2.2672263E10,1.5684734E-3,1.6E1,-2.685141E-3,7E1,1.9717047E9,-1.8799597E-3,8.1E1,1.8441159E-3,3.664424E9,-1.0141964E-3,8.2E1,-1.5593785E-3,1.403E4,4.964314E-4,-5.6658965E-4,9.7457E4,4.1747935E10,3.2037274E8,1.11E2,1.993E3,-9.052173E-4,1.5808915E-3,9.543625E-5,3.9895258E8,1.65E2,-2.022351E-3,-3.9169727E-6,6.8265916E-4,-1.0074391E-4,1.01092686E-4,2.57E2,8.9E1,-5.5252254E-4,-9.252574E-4,-1.9459752E-4,-1.2742697E-4,9.2128536E-4],"split_indices":[13,2,0,20,0,0,6,18,2,0,0,10,0,17,0,19,11,0,22,0,21,0,19,0,20,0,0,12,10,16,19,0,0,0,0,10,4,0,0,0,0,0,4,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,1.65E2,9E0,1.6E2,5E0,6E0,1.54E2,8.6E1,6.8E1,5E0,8.1E1,6E1,8E0,7.6E1,5E0,2.5E1,3.5E1,8E0,6.8E1,9E0,1.6E1,8E0,2.7E1,5E0,6.3E1,1E1,6E0,1.6E1,1.1E1,5.2E1,1.1E1,1.1E1,5E0,5E0,6E0,3.3E1,1.9E1,6E0,5E0,5E0,6E0,5E0,2.8E1,1.4E1,5E0,1.8E1,1E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.643796E-3,-8.364624E-3,8.496321E-3,4.9732663E-3,4.5120916E-3,1.4182465E-3,4.615709E-2,-1.3335352E-2,1.1730007E-2,3.4017714E-3,-1.7069653E-4,-3.556831E-2,-8.469353E-3,1.9610825E-3,8.3886245E-3,-8.2372647E-4,-2.0224268E-3,-3.3558963E-3,-2.768838E-2,1.8190031E-3,3.20112E-2,-1.0010686E-3,-1.7604981E-4,-3.5798742E-4,-1.8576188E-3,-1.1379366E-2,1.3133737E-2,4.392043E-2,3.8089427E-5,3.7039383E-3,-4.980293E-4,-1.2282364E-3,-2.3338457E-3,-5.330062E-3,2.147101E-2,9.0506853E-4,2.4363098E-3,1.4565251E-2,-2.779134E-3,7.61171E-4,-1.0493761E-2,6.0959626E-4,-9.213239E-4,3.0352531E-2,-5.2214127E-5,1.9015446E-5,1.1464866E-3,-6.4968056E-4,2.41061E-4,6.325504E-4,-2.1218643E-2,3.923687E-2,4.7078266E-4,-4.4214563E-4,-1.4170839E-3,8.78629E-4,2.1827212E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,15,17,-1,19,-1,-1,21,23,25,27,-1,29,-1,-1,31,33,35,-1,37,-1,39,-1,41,43,-1,-1,45,47,-1,49,-1,-1,51,-1,-1,-1,-1,-1,-1,53,55,-1,-1,-1,-1,-1],"loss_changes":[2.3209164E-1,0E0,5.5977527E-2,2.3945663E-2,0E0,2.3281032E-2,2.0990731E-2,6.5144487E-3,9.607325E-3,0E0,0E0,1.103065E-3,5.115691E-3,0E0,1.2822655E-2,0E0,0E0,2.6731628E-3,2.8387075E-3,1.00056315E-2,6.6367183E-3,0E0,1.6986526E-3,0E0,0E0,1.3179941E-2,5.6562E-3,2.428593E-3,0E0,2.0825046E-3,0E0,4.680077E-3,0E0,3.9484706E-3,5.0693117E-3,0E0,0E0,1.8073518E-3,1.9917763E-3,0E0,5.1310426E-3,0E0,0E0,2.7058031E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.2925747E-3,1.4630258E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,14,14,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,29,29,31,31,33,33,34,34,37,37,38,38,40,40,43,43,50,50,51,51],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,16,18,-1,20,-1,-1,22,24,26,28,-1,30,-1,-1,32,34,36,-1,38,-1,40,-1,42,44,-1,-1,46,48,-1,50,-1,-1,52,-1,-1,-1,-1,-1,-1,54,56,-1,-1,-1,-1,-1],"split_conditions":[1.066E4,-8.364624E-3,1.442786E4,8.03E2,4.5120916E-3,6.88588E5,1.14071E5,1.6E1,5.1622E4,3.4017714E-3,-1.7069653E-4,9.539214E9,3.5E1,1.9610825E-3,1.51688E5,-8.2372647E-4,-2.0224268E-3,4.9333E4,2.228942E6,6.881934E-2,1.750994E11,-1.0010686E-3,3.077206E3,-3.5798742E-4,-1.8576188E-3,8.945914E8,1.186263E6,2.8044E4,3.8089427E-5,6.779055E6,-4.980293E-4,3.565157E7,-2.3338457E-3,5E1,1.5342771E0,9.0506853E-4,2.4363098E-3,5.9816E4,1.5061888E8,7.61171E-4,1.4793389E9,6.0959626E-4,-9.213239E-4,1.94E2,-5.2214127E-5,1.9015446E-5,1.1464866E-3,-6.4968056E-4,2.41061E-4,6.325504E-4,2.45009E6,1.251E3,4.7078266E-4,-4.4214563E-4,-1.4170839E-3,8.78629E-4,2.1827212E-3],"split_indices":[7,0,5,17,0,6,28,17,12,0,0,10,19,0,20,0,0,18,18,62,10,0,5,0,0,23,18,7,0,23,0,16,0,4,65,0,0,27,23,0,11,0,0,19,0,0,0,0,0,0,18,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,5E0,1.68E2,1.63E2,5E0,1.51E2,1.2E1,6.2E1,8.9E1,7E0,5E0,1E1,5.2E1,7E0,8.2E1,5E0,5E0,4.2E1,1E1,6.5E1,1.7E1,5E0,3.7E1,5E0,5E0,3E1,3.5E1,1.2E1,5E0,2.8E1,9E0,2.5E1,5E0,1.1E1,2.4E1,5E0,7E0,1E1,1.8E1,8E0,1.7E1,5E0,6E0,1.7E1,7E0,5E0,5E0,7E0,1.1E1,5E0,1.2E1,1.1E1,6E0,7E0,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-5.740499E-3,-6.909762E-3,-1.5771129E-4,3.306597E-3,-4.037907E-3,-4.5102495E-3,1.7904138E-2,-1.3966884E-2,4.340634E-3,2.2125851E-2,-9.150338E-4,-8.498003E-3,-1.5511896E-3,-3.5992335E-3,2.5753563E-2,1.4166214E-2,4.9782153E-2,-9.182123E-4,-2.1515267E-3,1.9808374E-2,-1.1789055E-2,1.2741965E-4,1.9802E-3,2.8819756E-3,4.467941E-2,1.1357033E-3,2.8478322E-3,8.435176E-4,-9.863717E-3,1.9574582E-3,-3.724048E-4,-6.9574583E-3,-1.0828894E-3,1.1573297E-3,-5.116022E-3,7.991359E-4,2.776082E-3,-1.1890244E-3,6.222117E-5,1.1220827E-3,-1.6608015E-2,-1.7459467E-2,6.0716126E-4,-3.9128354E-4,5.088983E-4,-4.1648885E-4,7.390094E-4,-1.1246236E-3,-2.0433898E-4,1.4730028E-4,-1.3846535E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,-1,3,5,-1,7,9,11,13,15,-1,17,-1,19,21,23,25,-1,27,29,31,-1,-1,33,35,-1,-1,-1,37,-1,-1,39,-1,-1,41,-1,-1,-1,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4477515E-1,0E0,5.3097177E-2,1.8337829E-2,0E0,8.849081E-3,9.653013E-3,5.7148645E-3,9.443563E-3,1.0521617E-2,0E0,3.2971061E-3,0E0,8.093391E-3,6.5859733E-3,1.3988281E-2,2.3444854E-3,0E0,4.9941353E-3,8.188529E-3,1.7776862E-3,0E0,0E0,5.995357E-3,4.3247584E-3,0E0,0E0,0E0,3.7965768E-3,0E0,0E0,1.8899648E-3,0E0,0E0,5.879415E-3,0E0,0E0,0E0,1.6748382E-3,2.5219214E-3,1.0808604E-3,4.5649214E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24,28,28,31,31,34,34,38,38,39,39,40,40,41,41],"right_children":[2,-1,4,6,-1,8,10,12,14,16,-1,18,-1,20,22,24,26,-1,28,30,32,-1,-1,34,36,-1,-1,-1,38,-1,-1,40,-1,-1,42,-1,-1,-1,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1869768E7,-6.909762E-3,3.14331E5,2.0822124E9,-4.037907E-3,7.115196E-2,9.8744E4,5.2597154E-2,3.55E2,1.81812E5,-9.150338E-4,3.1E2,-1.5511896E-3,5.89E2,2.6456423E3,3.312442E6,1.09E2,-9.182123E-4,4.286E3,7.5802153E-1,7.9929E4,1.2741965E-4,1.9802E-3,2.535E3,1.33E2,1.1357033E-3,2.8478322E-3,8.435176E-4,2.2505884E3,1.9574582E-3,-3.724048E-4,9.13532E9,-1.0828894E-3,1.1573297E-3,7.0926E4,7.991359E-4,2.776082E-3,-1.1890244E-3,6.08E2,2.8721827E8,1.7198737E10,1.33E2,6.0716126E-4,-3.9128354E-4,5.088983E-4,-4.1648885E-4,7.390094E-4,-1.1246236E-3,-2.0433898E-4,1.4730028E-4,-1.3846535E-3],"split_indices":[11,0,3,11,0,62,8,65,17,20,0,0,0,20,5,18,4,0,28,62,8,0,0,20,4,0,0,0,5,0,0,10,0,0,8,0,0,0,0,11,10,4,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.69E2,5E0,1.64E2,1.59E2,5E0,1.04E2,5.5E1,5E1,5.4E1,5E1,5E0,4.1E1,9E0,4E1,1.4E1,4E1,1E1,1.3E1,2.8E1,1E1,3E1,7E0,7E0,3E1,1E1,5E0,5E0,7E0,2.1E1,5E0,5E0,2.3E1,7E0,7E0,2.3E1,5E0,5E0,7E0,1.4E1,1.3E1,1E1,1.4E1,9E0,8E0,6E0,8E0,5E0,5E0,5E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.7966194E-3,1.0231393E-3,6.302144E-3,-2.9397497E-3,3.0495506E-2,2.381342E-2,-7.5149843E-3,4.458203E-2,-4.2788798E-4,2.4114556E-3,1.1317666E-2,-1.7579236E-3,-4.9952827E-3,9.07797E-4,2.5370498E-3,-7.797989E-4,1.4676852E-3,-2.923362E-3,-1.3191913E-3,-5.268983E-4,5.4664304E-4,-7.507357E-3,2.1432355E-2,8.1187434E-4,-9.795529E-3,3.899739E-2,-5.1245396E-4,-5.865885E-3,-2.131783E-2,6.326159E-4,2.66618E-3,-1.6549427E-2,2.7228561E-3,-1.3991208E-3,-2.2814985E-4,-1.3988833E-3,-8.380139E-3,9.732968E-3,-1.1506818E-2,1.352273E-4,-8.335103E-4,-2.1317287E-3,2.0054974E-2,5.4950347E-6,-8.2025665E-4,-5.5856234E-4,5.4315577E-4,2.501136E-4,1.3129918E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,15,-1,17,-1,-1,19,-1,21,-1,-1,-1,23,25,-1,27,29,-1,31,33,-1,-1,35,37,-1,-1,-1,39,41,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3807578E-1,1.9711329E-2,0E0,1.8416861E-2,1.1790488E-2,8.016603E-3,1.0364953E-2,3.6231E-3,0E0,0E0,4.6141497E-3,0E0,6.1843116E-3,0E0,0E0,1.9539415E-3,0E0,1.2665307E-2,0E0,0E0,0E0,5.8575347E-3,1.0946269E-2,0E0,3.859464E-3,5.6093577E-3,0E0,6.1928593E-3,3.6334917E-3,0E0,0E0,3.5302723E-3,3.902557E-3,0E0,0E0,0E0,2.642332E-3,3.226906E-3,1.1042214E-3,0E0,0E0,2.1802548E-3,1.8385043E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,10,10,12,12,15,15,17,17,21,21,22,22,24,24,25,25,27,27,28,28,31,31,32,32,36,36,37,37,38,38,41,41,42,42],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,16,-1,18,-1,-1,20,-1,22,-1,-1,-1,24,26,-1,28,30,-1,32,34,-1,-1,36,38,-1,-1,-1,40,42,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1],"split_conditions":[1.442786E4,5.2596548E10,6.302144E-3,1E0,1.4732313E11,2.644956E-6,1.6E1,4.8E1,-4.2788798E-4,2.4114556E-3,6.512479E7,-1.7579236E-3,7.8E2,9.07797E-4,2.5370498E-3,5.7073E4,1.4676852E-3,1.0902801E9,-1.3191913E-3,-5.268983E-4,5.4664304E-4,2.412E4,2.26E2,8.1187434E-4,3.1199257E10,2.9950735E3,-5.1245396E-4,5.264E3,5.5015055E9,6.326159E-4,2.66618E-3,6.494468E6,2.13E2,-1.3991208E-3,-2.2814985E-4,-1.3988833E-3,1.2628019E8,5.9E1,5.353857E9,1.352273E-4,-8.335103E-4,4.6E2,6.7063E4,5.4950347E-6,-8.2025665E-4,-5.5856234E-4,5.4315577E-4,2.501136E-4,1.3129918E-3],"split_indices":[5,13,0,44,26,63,17,4,0,0,16,0,17,0,0,12,0,16,0,0,0,18,4,0,10,5,0,3,11,0,0,23,17,0,0,0,11,17,10,0,0,0,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.73E2,1.67E2,6E0,1.48E2,1.9E1,2.1E1,1.27E2,1.4E1,5E0,5E0,1.6E1,8E0,1.19E2,6E0,8E0,1.1E1,5E0,1.11E2,8E0,6E0,5E0,9.4E1,1.7E1,7E0,8.7E1,1.1E1,6E0,6.6E1,2.1E1,6E0,5E0,2.9E1,3.7E1,1.2E1,9E0,9E0,2E1,2.5E1,1.2E1,1E1,1E1,1.2E1,1.3E1,5E0,7E0,7E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.900749E-3,-8.681983E-4,4.6996744E-3,4.021572E-3,-6.2208576E-3,3.046356E-4,4.7158324E-3,3.1798491E-3,-2.8128468E-3,-8.182464E-3,2.1265205E-2,-1.0475516E-2,1.8542039E-3,2.0375703E-3,1.1499651E-2,-1.697778E-2,2.9238425E-3,-1.4349783E-4,2.1240639E-2,7.781762E-4,-1.9317562E-2,-7.3157144E-3,2.0660328E-2,1.3414982E-3,3.2103612E-4,-1.2228119E-2,-3.255334E-2,7.779584E-4,-1.8172108E-2,1.8442094E-3,-3.6168898E-5,-1.8020337E-2,6.0219704E-3,-5.908297E-4,-4.236618E-2,8.944401E-3,-5.3101534E-4,-1.4198333E-3,7.015904E-5,-1.3626066E-2,-1.5273248E-3,-3.451352E-4,7.619642E-4,-5.3038538E-2,-7.8644796E-4,9.1769634E-4,-2.019712E-4,1.5044928E-4,-8.2110363E-4,-2.8596555E-3,-1.2936728E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":99,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,13,15,-1,-1,17,19,21,-1,23,-1,25,27,29,-1,-1,31,33,35,37,-1,-1,39,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1155834E-1,1.24562584E-1,0E0,6.724394E-2,0E0,3.842784E-2,0E0,0E0,2.124826E-2,1.6012602E-2,6.9097346E-3,1.1288914E-2,0E0,0E0,3.3828304E-3,7.4296966E-3,7.9605365E-3,0E0,1.6196319E-3,0E0,7.296346E-3,2.474444E-3,7.4590603E-3,0E0,0E0,5.922484E-3,4.65155E-3,1.9058443E-3,3.4601921E-3,0E0,0E0,2.6556011E-3,2.3680967E-3,0E0,3.721591E-3,1.900869E-3,0E0,0E0,0E0,3.1639733E-3,0E0,0E0,0E0,1.562152E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,20,20,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,34,34,35,35,39,39,43,43],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,14,16,-1,-1,18,20,22,-1,24,-1,26,28,30,-1,-1,32,34,36,38,-1,-1,40,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.74639E5,3.0530524E7,4.6996744E-3,6.5078E4,-6.2208576E-3,2.2E1,4.7158324E-3,3.1798491E-3,1.2538E4,5.41E2,3.1E1,6.331471E8,1.8542039E-3,2.0375703E-3,1.495E3,1.7362E4,3.38E2,-1.4349783E-4,3.9556157E10,7.781762E-4,6.8424E4,7.7722E4,9.7318803E8,1.3414982E-3,3.2103612E-4,5.309569E6,8.786887E-4,9.703059E6,1.9717047E9,1.8442094E-3,-3.6168898E-5,3.75E2,3.538E4,-5.908297E-4,2.434628E7,7.541677E-1,-5.3101534E-4,-1.4198333E-3,7.015904E-5,3.565157E7,-1.5273248E-3,-3.451352E-4,7.619642E-4,2.34E2,-7.8644796E-4,9.1769634E-4,-2.019712E-4,1.5044928E-4,-8.2110363E-4,-2.8596555E-3,-1.2936728E-3],"split_indices":[12,2,0,3,0,20,0,0,3,4,19,21,0,0,0,18,17,0,10,0,8,8,21,0,0,18,62,18,11,0,0,4,12,0,18,66,0,0,0,16,0,0,0,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.87E2,1.78E2,9E0,1.73E2,5E0,1.68E2,5E0,6E0,1.62E2,1.33E2,2.9E1,1.28E2,5E0,7E0,2.2E1,8.6E1,4.2E1,9E0,1.3E1,5E0,8.1E1,2.7E1,1.5E1,7E0,6E0,5.4E1,2.7E1,1.6E1,1.1E1,7E0,8E0,4.1E1,1.3E1,1E1,1.7E1,1E1,6E0,6E0,5E0,3.4E1,7E0,6E0,7E0,1.1E1,6E0,5E0,5E0,8E0,2.6E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[2.9524541E-3,-1.9750656E-3,7.004918E-2,3.6051532E-3,-6.526161E-3,-2.4337857E-4,6.398804E-3,2.6887017E-3,1.3221578E-3,-1.3367963E-2,1.3093044E-2,1.0214355E-2,-1.8511862E-2,2.1929424E-3,1.0179569E-2,-4.0965635E-4,1.0317081E-3,-1.9799166E-3,-1.4463554E-2,1.5228083E-3,2.743669E-2,4.3276942E-4,-1.922973E-2,1.0226719E-3,-3.2728515E-3,3.5542592E-2,1.6334215E-4,-1.3905286E-2,-2.1712708E-3,-1.3497309E-2,1.1967025E-2,2.2008594E-2,2.5111416E-3,-1.8207563E-2,-1.6651443E-6,-2.1563793E-2,8.5869915E-4,2.092768E-2,-4.7730867E-4,3.5381172E-4,1.5168585E-3,-2.2314856E-2,-2.0237177E-4,-1.0554119E-4,-2.8178867E-2,1.323321E-3,1.4635245E-4,-1.4189115E-3,-5.567946E-4,-3.982213E-2,-2.5562942E-4,-2.3658294E-3,-8.2071195E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":100,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,13,15,17,-1,19,-1,-1,-1,21,23,25,-1,27,-1,29,31,-1,33,-1,35,37,39,-1,41,-1,43,-1,45,-1,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,-1,-1],"loss_changes":[5.799693E-2,1.3556719E-1,7.029103E-2,2.0970894E-2,0E0,0E0,0E0,0E0,2.6804052E-2,8.559134E-3,8.796743E-3,3.6421034E-3,5.539693E-3,0E0,1.2058872E-2,0E0,0E0,0E0,6.1165392E-3,5.8551026E-3,5.0527174E-3,0E0,6.295057E-3,0E0,7.3292176E-3,4.68616E-3,0E0,2.266395E-3,0E0,7.91631E-3,4.1281E-3,2.1987548E-3,0E0,1.5129987E-3,0E0,2.8448207E-3,0E0,2.2089612E-3,0E0,0E0,0E0,1.7288597E-3,0E0,0E0,4.1851094E-3,0E0,0E0,0E0,0E0,2.3059081E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9,10,10,11,11,12,12,14,14,18,18,19,19,20,20,22,22,24,24,25,25,27,27,29,29,30,30,31,31,33,33,35,35,37,37,41,41,44,44,49,49],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,14,16,18,-1,20,-1,-1,-1,22,24,26,-1,28,-1,30,32,-1,34,-1,36,38,40,-1,42,-1,44,-1,46,-1,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,-1,-1],"split_conditions":[1.7782382E0,4.831919E7,1.7261E4,6.7E1,-6.526161E-3,-2.4337857E-4,6.398804E-3,2.6887017E-3,1.882904E6,4.432E3,5.7073E4,1.84E2,9.319117E8,2.1929424E-3,5.530639E9,-4.0965635E-4,1.0317081E-3,-1.9799166E-3,1.4345135E3,6.815E3,1.7536711E11,4.3276942E-4,8.1559E4,1.0226719E-3,7.337122E-1,2.7121158E7,1.6334215E-4,5.0364403E8,-2.1712708E-3,4.8858527E9,1.65E2,1.23E2,2.5111416E-3,6.502E3,-1.6651443E-6,1E0,8.5869915E-4,4.5356333E3,-4.7730867E-4,3.5381172E-4,1.5168585E-3,1E0,-2.0237177E-4,-1.0554119E-4,1.84E2,1.323321E-3,1.4635245E-4,-1.4189115E-3,-5.567946E-4,5.9170656E8,-2.5562942E-4,-2.3658294E-3,-8.2071195E-4],"split_indices":[66,6,3,22,0,0,0,0,2,28,12,22,10,0,11,0,0,0,5,22,10,0,8,0,63,2,0,16,0,11,4,4,0,3,0,41,0,5,0,0,0,41,0,0,19,0,0,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,1.63E2,1.1E1,1.58E2,5E0,6E0,5E0,5E0,1.53E2,6.8E1,8.5E1,1.2E1,5.6E1,5E0,8E1,5E0,7E0,6E0,5E1,5.4E1,2.6E1,8E0,4.2E1,9E0,4.5E1,1.9E1,7E0,3.7E1,5E0,2.7E1,1.8E1,1.3E1,6E0,2.8E1,9E0,2.2E1,5E0,1.3E1,5E0,7E0,6E0,2.1E1,7E0,6E0,1.6E1,8E0,5E0,9E0,1.2E1,1E1,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.088175E-3,-7.2464533E-3,2.2066988E-3,6.0813418E-3,-4.147643E-3,1.0304623E-3,5.5488637E-3,-3.145632E-3,3.7432987E-2,1.8452639E-3,-5.2836523E-3,2.229044E-3,3.6951355E-4,-1.4353703E-2,9.754214E-3,-6.8930327E-3,-3.2219116E-2,1.8395277E-3,5.007637E-3,-1.1207359E-2,1.1140718E-3,-2.5960621E-3,-2.1342635E-2,-3.3667425E-3,1.8013556E-2,-2.8074274E-2,-6.408739E-3,-4.4770557E-5,-1.2691617E-3,-1.612726E-2,1.16159115E-2,2.6162328E-2,-5.614078E-5,-4.6971822E-4,-1.7767671E-3,-1.3498521E-2,5.5127046E-3,-1.1462147E-3,-1.7360765E-4,-3.8706753E-4,1.1141737E-3,2.7342778E-4,1.7231848E-3,-1.2837583E-3,-7.2537567E-3,5.114562E-4,-1.8068013E-4,7.9333106E-5,-5.871734E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":101,"left_children":[1,-1,3,5,-1,7,-1,9,11,-1,13,-1,-1,15,17,19,21,-1,23,25,-1,-1,27,29,31,33,35,-1,-1,37,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,47,-1,-1,-1,-1],"loss_changes":[1.6346525E-1,0E0,5.8973603E-2,9.1579854E-2,0E0,2.2160705E-2,0E0,1.2945397E-2,5.4929554E-3,0E0,1.7325679E-2,0E0,0E0,1.0307658E-2,7.1688746E-3,8.258106E-3,5.549798E-3,0E0,4.7440566E-3,3.947914E-3,0E0,0E0,2.7989987E-3,5.349366E-3,2.707236E-3,1.9677961E-3,3.5397517E-3,0E0,0E0,1.7423884E-3,3.930792E-3,2.9326072E-3,0E0,0E0,0E0,2.37048E-3,1.0068385E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1491284E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,10,10,13,13,14,14,15,15,16,16,18,18,19,19,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,35,35,36,36,44,44],"right_children":[2,-1,4,6,-1,8,-1,10,12,-1,14,-1,-1,16,18,20,22,-1,24,26,-1,-1,28,30,32,34,36,-1,-1,38,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,48,-1,-1,-1,-1],"split_conditions":[2.7197143E-7,-7.2464533E-3,5.079193E7,7.1848E4,-4.147643E-3,7.194782E10,5.5488637E-3,1.35E2,2.616974E6,1.8452639E-3,8.008664E8,2.229044E-3,3.6951355E-4,7.132E4,1.752E3,2.4625E4,1.3820372E9,1.8395277E-3,2.98E2,3.440772E7,1.1140718E-3,-2.5960621E-3,2.1955977E3,2.0822124E9,1.1291503E0,3.31E2,2.57E2,-4.4770557E-5,-1.2691617E-3,2.7196716E3,2.6E1,2.0794651E-1,-5.614078E-5,-4.6971822E-4,-1.7767671E-3,1.72975E5,5.2618624E7,-1.1462147E-3,-1.7360765E-4,-3.8706753E-4,1.1141737E-3,2.7342778E-4,1.7231848E-3,-1.2837583E-3,5.6776E4,5.114562E-4,-1.8068013E-4,7.9333106E-5,-5.871734E-4],"split_indices":[62,0,2,3,0,10,0,0,18,0,21,0,0,8,22,7,10,0,17,16,0,0,5,11,65,0,4,0,0,5,17,62,0,0,0,6,16,0,0,0,0,0,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.59E2,5E0,1.54E2,1.49E2,5E0,1.44E2,5E0,1.3E2,1.4E1,5E0,1.25E2,9E0,5E0,7.8E1,4.7E1,5.6E1,2.2E1,5E0,4.2E1,5E1,6E0,5E0,1.7E1,2.6E1,1.6E1,1E1,4E1,5E0,1.2E1,1.4E1,1.2E1,1.1E1,5E0,5E0,5E0,2.5E1,1.5E1,7E0,7E0,5E0,7E0,5E0,6E0,6E0,1.9E1,9E0,6E0,8E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.9158671E-3,-1.1305097E-3,5.227802E-3,3.795146E-3,-6.096663E-3,3.8100455E-3,7.172263E-4,-5.515578E-3,2.7635891E-2,-7.3464303E-3,1.2548836E-3,3.6743876E-2,-7.174693E-4,-1.8879069E-2,-2.8031887E-3,4.319916E-2,3.503281E-4,-8.25629E-3,-1.9934713E-3,-6.988364E-3,1.4243125E-3,2.8580202E-2,2.9834495E-3,1.7730002E-3,-8.377349E-4,-1.3803041E-3,-4.338026E-3,6.6301343E-4,1.8206352E-3,5.731934E-4,-3.1684045E-4,-1.8478137E-3,-1.0015944E-3,5.1289666E-3,-1.1088594E-2,8.9898705E-4,9.5746125E-4,-2.5844103E-2,3.437733E-3,-1.4279841E-4,6.259516E-4,-5.1378564E-4,-1.7565945E-3,-2.8322166E-4,7.4756576E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":102,"left_children":[1,3,-1,5,-1,-1,7,9,11,13,-1,15,-1,17,19,21,-1,23,-1,25,-1,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1],"loss_changes":[8.477651E-2,1.192771E-1,0E0,4.337004E-2,0E0,0E0,2.766815E-2,8.617926E-3,1.3048163E-2,6.6537107E-3,0E0,4.4731237E-3,0E0,9.869322E-3,1.4020412E-2,5.7689734E-3,0E0,2.9957122E-3,0E0,5.347476E-3,0E0,2.0040125E-3,0E0,1.6343461E-3,0E0,0E0,3.5259253E-3,0E0,0E0,0E0,0E0,4.5053647E-3,0E0,2.8241025E-3,6.519146E-3,1.8970029E-3,0E0,2.566263E-3,2.2854493E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,17,17,19,19,21,21,23,23,26,26,31,31,33,33,34,34,35,35,37,37,38,38],"right_children":[2,4,-1,6,-1,-1,8,10,12,14,-1,16,-1,18,20,22,-1,24,-1,26,-1,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0510942E1,1.2498039E0,5.227802E-3,2.2E1,-6.096663E-3,3.8100455E-3,1.3029E4,7.112E3,2.008E3,1.05E2,1.2548836E-3,4.156497E-1,-7.174693E-4,8.2155E4,2.8884864E8,2.3378E4,3.503281E-4,1.2406315E-1,-1.9934713E-3,2E1,1.4243125E-3,9.780779E-1,2.9834495E-3,6.929641E8,-8.377349E-4,-1.3803041E-3,3.9405898E3,6.6301343E-4,1.8206352E-3,5.731934E-4,-3.1684045E-4,4.174E3,-1.0015944E-3,4.6192E4,6.86E3,1.4908578E-1,9.5746125E-4,4.743E3,1E0,-1.4279841E-4,6.259516E-4,-5.1378564E-4,-1.7565945E-3,-2.8322166E-4,7.4756576E-4],"split_indices":[59,58,0,20,0,0,3,0,19,4,0,62,0,8,27,3,0,63,0,17,0,66,0,11,0,0,5,0,0,0,0,20,0,12,3,63,0,3,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.78E2,1.73E2,5E0,1.68E2,5E0,5E0,1.63E2,1.33E2,3E1,1.27E2,6E0,2.5E1,5E0,3.5E1,9.2E1,2E1,5E0,2.6E1,9E0,8.3E1,9E0,1.4E1,6E0,1.4E1,1.2E1,7E0,7.6E1,8E0,6E0,6E0,8E0,6.8E1,8E0,3.9E1,2.9E1,3.2E1,7E0,1.4E1,1.5E1,2.5E1,7E0,8E0,6E0,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[3.0471697E-3,-1.3577425E-3,6.703154E-2,1.7657952E-3,-3.912147E-3,1.0380964E-3,4.640328E-3,2.704129E-3,-4.3359175E-4,-3.7093626E-3,3.257141E-2,-1.9627435E-3,-1.989959E-3,1.8677829E-3,4.4113252E-4,-5.7892846E-3,1.3026192E-2,1.2926055E-3,-7.969834E-3,2.0600685E-3,3.17804E-2,-1.9296517E-3,-5.843289E-3,-5.283734E-4,1.0618954E-2,7.415429E-4,1.8015039E-3,3.1186366E-3,-1.2253666E-2,8.478865E-5,8.9116517E-4,1.6193565E-3,-2.0567253E-3,-1.680282E-2,5.7099824E-4,-1.4118701E-2,4.2738076E-3,-1.0097697E-2,-2.8270002E-2,-9.791897E-6,-1.0698483E-3,8.664707E-4,-2.015648E-4,-9.0122863E-4,4.424701E-5,-1.6799145E-3,-2.2931173E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":103,"left_children":[1,3,5,7,-1,-1,-1,-1,9,11,13,15,-1,-1,-1,17,19,-1,21,23,25,-1,27,-1,29,-1,-1,31,33,-1,-1,-1,35,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.2514285E-2,4.8410494E-2,1.7858453E-2,2.2531219E-2,0E0,0E0,0E0,0E0,1.7960094E-2,1.0956673E-2,3.0118544E-3,8.449356E-3,0E0,0E0,0E0,9.2685325E-3,6.104358E-3,0E0,8.296145E-3,2.5534008E-3,8.4344205E-4,0E0,6.127842E-3,0E0,1.0655507E-3,0E0,0E0,7.990568E-3,7.365739E-3,0E0,0E0,0E0,3.1146898E-3,3.7896493E-3,0E0,2.0161048E-3,3.831659E-3,4.1240132E-3,4.26652E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8,9,9,10,10,11,11,15,15,16,16,18,18,19,19,20,20,22,22,24,24,27,27,28,28,32,32,33,33,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,-1,-1,-1,-1,10,12,14,16,-1,-1,-1,18,20,-1,22,24,26,-1,28,-1,30,-1,-1,32,34,-1,-1,-1,36,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.74639E5,7.3133384E3,4.312765E7,1E0,-3.912147E-3,1.0380964E-3,4.640328E-3,2.704129E-3,8.9422E4,7.12E2,1.48E2,6.993197E-1,-1.989959E-3,1.8677829E-3,4.4113252E-4,3.017E3,7.046429E8,1.2926055E-3,1.06E2,7.9E1,4.9E1,-1.9296517E-3,6.365E3,-5.283734E-4,3.4626413E8,7.415429E-4,1.8015039E-3,6.4E1,3.857587E6,8.478865E-5,8.9116517E-4,1.6193565E-3,3.33044E5,3.425962E6,5.7099824E-4,1.8781646E9,5.596E3,6.247E3,5.5015055E9,-9.791897E-6,-1.0698483E-3,8.664707E-4,-2.015648E-4,-9.0122863E-4,4.424701E-5,-1.6799145E-3,-2.2931173E-4],"split_indices":[12,5,6,30,0,0,0,0,8,17,19,63,0,0,0,3,16,0,22,4,17,0,22,0,16,0,0,4,6,0,0,0,6,2,0,10,3,3,11,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.85E2,1.74E2,1.1E1,1.69E2,5E0,6E0,5E0,5E0,1.64E2,1.5E2,1.4E1,1.45E2,5E0,9E0,5E0,1.16E2,2.9E1,6E0,1.1E2,1.9E1,1E1,5E0,1.05E2,7E0,1.2E1,5E0,5E0,4.4E1,6.1E1,7E0,5E0,5E0,3.9E1,5.2E1,9E0,1.3E1,2.6E1,3.4E1,1.8E1,6E0,7E0,9E0,1.7E1,1.7E1,1.7E1,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.8903929E-4,-7.844739E-3,6.3807503E-3,2.9148816E-3,5.4322843E-2,3.0579455E-3,1.9144944E-4,6.003199E-3,-1.6563205E-3,2.3102886E-3,-2.2125759E-3,-5.306952E-3,1.5103967E-2,-9.806291E-3,9.250352E-3,3.7502486E-2,4.686103E-3,-2.2559218E-2,-5.9180525E-3,1.2566586E-3,-1.7882622E-3,4.514636E-4,2.6403985E-3,-3.9477805E-3,1.4379494E-3,2.2301447E-5,-1.5910357E-3,4.692762E-4,-7.8000547E-3,5.7013385E-4,-5.7352433E-4,3.3872502E-3,-1.0712055E-3,-8.285625E-4,-4.529662E-3,-6.6532874E-3,1.0058398E-3,9.4199745E-4,-1.4758601E-2,-1.2412905E-2,2.1888455E-4,-2.465623E-4,5.548234E-4,-1.6804706E-4,-9.802519E-4,-9.007617E-5,-9.031964E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":104,"left_children":[1,-1,3,5,7,-1,9,-1,-1,11,-1,13,15,17,19,21,23,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,35,-1,-1,37,39,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0049213E-1,0E0,2.7092058E-2,2.814823E-2,8.968327E-2,0E0,1.6181566E-2,0E0,0E0,1.4113319E-2,0E0,6.0510524E-3,1.2424875E-2,3.337543E-3,4.797035E-3,9.989729E-3,9.401465E-3,5.1469207E-3,1.8093463E-3,0E0,2.700374E-3,0E0,0E0,4.6150237E-3,0E0,0E0,0E0,0E0,1.7426915E-3,0E0,0E0,4.6929163E-3,0E0,0E0,2.2526733E-3,1.1375173E-3,0E0,2.1663012E-3,1.0894737E-3,8.9993456E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,28,28,31,31,34,34,35,35,37,37,38,38,39,39],"right_children":[2,-1,4,6,8,-1,10,-1,-1,12,-1,14,16,18,20,22,24,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,36,-1,-1,38,40,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7197143E-7,-7.844739E-3,9.0573E4,4.6E1,1.0699701E0,3.0579455E-3,8.88E2,6.003199E-3,-1.6563205E-3,2.7451E4,-2.2125759E-3,1.5081332E9,1.0595E4,5.4753E4,1.1415993E0,6.22281E5,2.42889E5,1.9385E4,2.731002E7,1.2566586E-3,5.337197E-1,4.514636E-4,2.6403985E-3,5.691233E10,1.4379494E-3,2.2301447E-5,-1.5910357E-3,4.692762E-4,1.73297E5,5.7013385E-4,-5.7352433E-4,3.1819E4,-1.0712055E-3,-8.285625E-4,6.1437E4,1.83E2,1.0058398E-3,2.57E2,2.13E2,8E1,2.1888455E-4,-2.465623E-4,5.548234E-4,-1.6804706E-4,-9.802519E-4,-9.007617E-5,-9.031964E-4],"split_indices":[62,0,3,22,58,0,17,0,0,7,0,21,20,18,65,6,28,18,11,0,62,0,0,10,0,0,0,0,6,0,0,7,0,0,12,4,0,4,4,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,5E0,1.63E2,1.53E2,1E1,5E0,1.48E2,5E0,5E0,1.43E2,5E0,9E1,5.3E1,6.9E1,2.1E1,1.6E1,3.7E1,1.5E1,5.4E1,7E0,1.4E1,8E0,8E0,2.9E1,8E0,6E0,9E0,5E0,4.9E1,6E0,8E0,2.2E1,7E0,1E1,3.9E1,1.5E1,7E0,2.6E1,1.3E1,1E1,5E0,1.7E1,9E0,6E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.2025183E-4,-7.146976E-3,5.802754E-3,-1.1686811E-3,7.4283E-2,4.320464E-3,-6.151907E-3,2.3881526E-4,8.2081035E-3,-4.0252865E-3,2.3820912E-2,1.7341293E-3,-6.5351073E-3,4.3357827E-2,5.6755925E-3,-3.2448612E-2,-3.399799E-3,2.5709603E-2,3.3827517E-3,1.4409828E-2,-1.0643066E-3,-1.9338136E-3,-6.6270336E-4,-8.199925E-4,-1.3402986E-3,1.9642632E-3,1.2848438E-2,2.8803882E-3,1.5455445E-3,-7.1974155E-3,8.237467E-3,1.087522E-3,-5.9398368E-5,-5.664115E-4,5.731722E-4,-3.3467764E-3,-1.5503129E-3,1.5365103E-3,1.6708863E-3,-1.6102647E-2,2.7371251E-3,-6.775557E-3,1.6876157E-2,-1.3184397E-3,-1.4593186E-4,1.6391093E-2,-5.1954817E-3,-9.617709E-4,-2.1997494E-4,1.2040457E-3,1.4637351E-4,1.2309876E-3,1.7130339E-4,-5.747688E-4,3.1601314E-5,-3.9360646E-4,2.9013632E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":105,"left_children":[1,-1,3,5,7,9,-1,-1,-1,11,13,-1,15,17,19,21,23,25,-1,27,-1,-1,-1,29,-1,-1,31,33,-1,35,37,-1,-1,-1,-1,39,-1,-1,41,43,45,47,49,-1,-1,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6669774E-1,0E0,8.2677975E-2,1.2183668E-1,1.2423421E-1,2.4996007E-2,0E0,0E0,0E0,1.2120212E-2,1.6061079E-2,0E0,8.258102E-3,1.1264533E-2,6.9151656E-3,1.5792027E-3,6.526598E-3,3.3920417E-3,0E0,4.740447E-3,0E0,0E0,0E0,5.0269277E-3,0E0,0E0,1.920154E-3,2.4463309E-3,0E0,5.4430715E-3,6.315789E-3,0E0,0E0,0E0,0E0,3.6139267E-3,0E0,0E0,3.967259E-3,2.5983925E-3,3.5511295E-3,1.9958764E-3,1.5005653E-3,0E0,0E0,1.6563702E-3,1.0006692E-3,0E0,9.6622534E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,23,23,26,26,27,27,29,29,30,30,35,35,38,38,39,39,40,40,41,41,42,42,45,45,46,46,48,48],"right_children":[2,-1,4,6,8,10,-1,-1,-1,12,14,-1,16,18,20,22,24,26,-1,28,-1,-1,-1,30,-1,-1,32,34,-1,36,38,-1,-1,-1,-1,40,-1,-1,42,44,46,48,50,-1,-1,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7E1,-7.146976E-3,1.3336757E0,1.0893652E0,1.5385991E7,4.503171E6,-6.151907E-3,2.3881526E-4,8.2081035E-3,1.49E2,1E0,1.7341293E-3,1.5E1,3.443E4,1.4509117E11,3.1861102E7,1.14131E5,2.5522802E-6,3.3827517E-3,8.7165575E10,-1.0643066E-3,-1.9338136E-3,-6.6270336E-4,5.3561E4,-1.3402986E-3,1.9642632E-3,2.127E3,6.1E1,1.5455445E-3,2.8678E4,5.7437E4,1.087522E-3,-5.9398368E-5,-5.664115E-4,5.731722E-4,4.05131E6,-1.5503129E-3,1.5365103E-3,2.6778312E8,1.84146E5,1.3411E4,2.0538752E3,2.6899552E-1,-1.3184397E-3,-1.4593186E-4,5.53E2,3.19899E5,-9.617709E-4,9.109177E8,1.2040457E-3,1.4637351E-4,1.2309876E-3,1.7130339E-4,-5.747688E-4,3.1601314E-5,-3.9360646E-4,2.9013632E-4],"split_indices":[4,0,63,63,2,2,0,0,0,28,29,0,17,7,10,21,12,63,0,10,0,0,0,12,0,0,0,17,0,7,12,0,0,0,0,23,0,0,27,6,28,5,62,0,0,20,6,0,11,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.77E2,5E0,1.72E2,1.57E2,1.5E1,1.52E2,5E0,1E1,5E0,1.07E2,4.5E1,5E0,1.02E2,2.1E1,2.4E1,1E1,9.2E1,1.5E1,6E0,1.9E1,5E0,5E0,5E0,8.5E1,7E0,5E0,1E1,1.3E1,6E0,5E1,3.5E1,5E0,5E0,5E0,8E0,4.5E1,5E0,6E0,2.9E1,1.4E1,3.1E1,1.9E1,1E1,6E0,8E0,1.1E1,2E1,5E0,1.4E1,5E0,5E0,5E0,6E0,8E0,1.2E1,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.7764873E-3,-7.1751024E-3,3.4974192E-3,-3.3454248E-3,8.532009E-2,2.1498218E-4,-3.1887738E-3,6.9753844E-3,4.050293E-4,-6.8283966E-3,2.2832654E-2,-1.04754865E-2,2.261343E-2,3.167972E-2,-2.9835723E-5,1.5242822E-2,-1.4024822E-2,1.67209E-3,2.587913E-4,6.5083045E-4,1.6458052E-3,-1.3702014E-4,1.4904612E-3,-3.243292E-2,-1.0806712E-2,-5.2305864E-4,-1.978402E-3,-1.307906E-2,5.450069E-4,-9.318916E-3,-1.5833211E-3,-1.4232062E-2,-1.00999E-3,-2.2606112E-2,-5.9838137E-3,4.3442185E-4,-8.768621E-3,-4.4297785E-4,-1.7778049E-3,-1.1678E-3,4.4055827E-4,-7.9834135E-4,3.0449097E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":106,"left_children":[1,-1,3,5,7,9,-1,-1,-1,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,25,27,-1,-1,29,-1,31,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6280629E-1,0E0,8.70699E-2,3.5757445E-2,6.649146E-2,2.2139978E-2,0E0,0E0,0E0,1.1618847E-2,6.8872906E-3,8.885655E-3,2.9153698E-3,1.863474E-3,0E0,4.277436E-3,4.6256073E-3,0E0,0E0,0E0,0E0,0E0,0E0,2.6209643E-3,4.0272614E-3,0E0,0E0,5.5058626E-3,0E0,2.3941812E-3,0E0,2.4203658E-3,2.0436207E-3,3.6507938E-3,6.9492343E-3,0E0,2.2265771E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,15,15,16,16,23,23,24,24,27,27,29,29,31,31,32,32,33,33,34,34,36,36],"right_children":[2,-1,4,6,8,10,-1,-1,-1,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,26,28,-1,-1,30,-1,32,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7309324E7,-7.1751024E-3,1.845821E9,3.5713768E0,2.115E3,4.503171E6,-3.1887738E-3,6.9753844E-3,4.050293E-4,9.078E3,4.832502E3,1.445E3,2.607E3,2E0,-2.9835723E-5,3.04085E6,9.319117E8,1.67209E-3,2.587913E-4,6.5083045E-4,1.6458052E-3,-1.3702014E-4,1.4904612E-3,4.4468006E8,7.594183E-1,-5.2305864E-4,-1.978402E-3,1.9118E4,5.450069E-4,1E0,-1.5833211E-3,2.4152776E3,6.5553044E9,6.128E3,3.8956667E9,4.3442185E-4,1.1662405E9,-4.4297785E-4,-1.7778049E-3,-1.1678E-3,4.4055827E-4,-7.9834135E-4,3.0449097E-4],"split_indices":[11,0,16,59,0,2,0,0,0,3,5,28,20,19,0,18,10,0,0,0,0,0,0,10,63,0,0,20,0,41,0,5,10,3,10,0,11,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.59E2,5E0,1.54E2,1.43E2,1.1E1,1.37E2,6E0,5E0,6E0,1.05E2,3.2E1,9.4E1,1.1E1,2.3E1,9E0,1.1E1,8.3E1,5E0,6E0,7E0,1.6E1,6E0,5E0,1.1E1,7.2E1,5E0,6E0,6.6E1,6E0,5.8E1,8E0,3.6E1,2.2E1,1.7E1,1.9E1,9E0,1.3E1,1.1E1,6E0,8E0,1.1E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[6.879416E-5,-7.7649206E-3,6.276714E-3,9.517032E-3,-3.7081658E-3,5.2425317E-3,5.383515E-3,3.3983253E-3,1.9504738E-3,3.8989559E-3,-2.069362E-3,1.4894617E-3,1.8222557E-3,-2.5364459E-3,1.3525811E-2,6.36933E-3,-8.525949E-3,4.472832E-3,2.9198555E-2,-8.355793E-4,1.505001E-3,-1.16134565E-2,5.226977E-4,1.3916353E-2,-4.588328E-3,1.7375108E-3,4.4457766E-4,-1.2633758E-2,1.038562E-2,-4.4068093E-3,-2.6983036E-2,1.1238381E-3,7.7977485E-5,2.6164047E-4,-6.335638E-4,-4.8989216E-3,-1.1909909E-3,4.451539E-3,9.820937E-4,-8.139318E-3,6.2642334E-4,-3.7067294E-2,-2.5494222E-4,2.9846566E-4,-8.6123706E-4,4.978955E-4,-1.8043157E-4,-9.332927E-4,-2.234122E-4,-2.3310136E-3,-7.7548553E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":107,"left_children":[1,-1,3,5,-1,7,-1,-1,9,11,-1,13,-1,15,17,19,21,23,25,27,-1,29,-1,31,33,-1,-1,35,37,39,41,-1,-1,-1,-1,43,-1,45,-1,47,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9667242E-1,0E0,5.1656954E-2,8.060664E-2,0E0,3.8575716E-2,0E0,0E0,1.5189052E-2,1.3709269E-2,0E0,6.963692E-3,0E0,5.8151507E-3,4.977261E-3,8.940743E-3,4.253491E-3,2.1175146E-3,2.216897E-3,4.897239E-3,0E0,6.1945887E-3,0E0,1.676243E-3,1.4527807E-3,0E0,0E0,1.9064429E-3,1.2658013E-3,2.917522E-3,3.62112E-3,0E0,0E0,0E0,0E0,2.3981826E-3,0E0,8.8321685E-4,0E0,1.250701E-3,0E0,2.8352048E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,27,27,28,28,29,29,30,30,35,35,37,37,39,39,41,41],"right_children":[2,-1,4,6,-1,8,-1,-1,10,12,-1,14,-1,16,18,20,22,24,26,28,-1,30,-1,32,34,-1,-1,36,38,40,42,-1,-1,-1,-1,44,-1,46,-1,48,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.73432E5,-7.7649206E-3,2.77015E5,4.8604E4,-3.7081658E-3,2.2E1,5.383515E-3,3.3983253E-3,9.62E2,2.3078E4,-2.069362E-3,9.32074E-1,1.8222557E-3,2.0832268E-3,2.7616E4,8.298118E6,1.1904228E9,4.3822035E-1,8.7091637E-1,1.6921284E10,1.505001E-3,1.4521729E9,5.226977E-4,1E0,5.89E2,1.7375108E-3,4.4457766E-4,5.735E3,8.7165575E10,2.5714466E8,8.9422E4,1.1238381E-3,7.7977485E-5,2.6164047E-4,-6.335638E-4,3.1426326E3,-1.1909909E-3,1.755287E6,9.820937E-4,1.0114802E7,6.2642334E-4,6.881934E-2,-2.5494222E-4,2.9846566E-4,-8.6123706E-4,4.978955E-4,-1.8043157E-4,-9.332927E-4,-2.234122E-4,-2.3310136E-3,-7.7548553E-4],"split_indices":[16,0,3,3,0,20,0,0,17,3,0,65,0,63,7,18,27,62,63,10,0,26,0,29,20,0,0,3,10,21,8,0,0,0,0,5,0,6,0,21,0,62,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,5E0,1.71E2,1.66E2,5E0,1.61E2,5E0,6E0,1.55E2,1.5E2,5E0,1.42E2,8E0,1.07E2,3.5E1,4.3E1,6.4E1,2.3E1,1.2E1,3.5E1,8E0,5.6E1,8E0,1.1E1,1.2E1,7E0,5E0,1.7E1,1.8E1,3.9E1,1.7E1,5E0,6E0,6E0,6E0,1.2E1,5E0,1.3E1,5E0,3.3E1,6E0,1.1E1,6E0,7E0,5E0,7E0,6E0,5E0,2.8E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[3.9379485E-3,-7.1111065E-3,9.417237E-3,4.008683E-3,9.313124E-2,4.2057307E-3,6.6673855E-4,-1.988788E-4,7.6511805E-3,3.6198348E-3,-3.4558752E-3,6.3953834E-4,1.9411112E-3,-1.5288681E-3,1.3785355E-3,-3.002925E-3,1.2019901E-3,-1.472866E-3,-1.3621286E-3,-8.241245E-3,9.305791E-3,-1.3742608E-3,-5.616945E-3,1.4383072E-2,-6.9775287E-4,-9.692043E-4,-1.8117385E-2,1.0598597E-2,1.5692916E-3,5.565465E-4,-3.0225495E-4,-1.0565905E-3,-3.484752E-4,-8.587442E-5,8.653872E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":108,"left_children":[1,-1,3,5,7,-1,9,-1,-1,11,-1,13,-1,15,-1,17,-1,19,-1,21,23,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7250496E-1,0E0,8.2031116E-2,5.301948E-2,8.909138E-2,0E0,3.9663397E-2,0E0,0E0,1.9695811E-2,0E0,1.0247446E-2,0E0,6.2099057E-3,0E0,5.8663078E-3,0E0,9.63367E-3,0E0,4.764213E-3,6.7791E-3,0E0,4.2817523E-3,3.1292625E-3,0E0,4.582647E-3,1.0410366E-3,4.4456683E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,9,9,11,11,13,13,15,15,17,17,19,19,20,20,22,22,23,23,25,25,26,26,27,27],"right_children":[2,-1,4,6,8,-1,10,-1,-1,12,-1,14,-1,16,-1,18,-1,20,-1,22,24,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7E1,-7.1111065E-3,1.7903596E0,4.6E1,1.483466E6,4.2057307E-3,4.831919E7,-1.988788E-4,7.6511805E-3,3.3765684E7,-3.4558752E-3,7.12E2,1.9411112E-3,5.93E2,1.3785355E-3,3.4824E4,1.2019901E-3,1.235866E6,-1.3621286E-3,1.4E1,8.6033555E10,-1.3742608E-3,7.6064E4,8.2025E4,-6.9775287E-4,1.47E2,7.64E2,5.1637E4,1.5692916E-3,5.565465E-4,-3.0225495E-4,-1.0565905E-3,-3.484752E-4,-8.587442E-5,8.653872E-4],"split_indices":[4,0,66,22,18,0,6,0,0,2,0,17,0,4,0,7,0,6,0,17,13,0,8,8,0,4,0,28,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.86E2,5E0,1.81E2,1.71E2,1E1,5E0,1.66E2,5E0,5E0,1.61E2,5E0,1.51E2,1E1,1.42E2,9E0,1.36E2,6E0,1.3E2,6E0,8E1,5E1,7E0,7.3E1,4.2E1,8E0,5.4E1,1.9E1,3.7E1,5E0,1.6E1,3.8E1,1.1E1,8E0,1.6E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.5814454E-3,3.53139E-5,4.3092645E-3,3.442869E-3,-3.3132173E-3,-4.8389313E-5,3.96019E-3,3.2506369E-3,-3.645772E-3,-1.0338131E-2,1.0528469E-2,-6.9641215E-3,-2.0883733E-3,1.9880845E-3,3.2611014E-3,-2.3830574E-2,-3.6312893E-3,-8.41772E-3,1.2259262E-2,-1.4329924E-4,-1.8069467E-3,-1.3267839E-3,-1.3815329E-3,-1.1305251E-3,5.213238E-4,2.0296095E-2,-6.064499E-4,9.69295E-3,-5.743469E-3,1.2912702E-2,1.4364535E-3,1.5534442E-2,7.77989E-6,-7.8073493E-3,3.899637E-4,1.3311788E-4,9.1356726E-4,2.6098793E-4,1.1210822E-3,-1.3329857E-2,-3.3520287E-4,-3.699672E-4,-1.134212E-3,3.6173608E-4,-3.2705022E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":109,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,13,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,-1,-1,29,-1,31,33,35,-1,37,-1,39,-1,-1,-1,-1,-1,41,43,-1,-1,-1,-1],"loss_changes":[5.6343913E-2,4.218472E-2,0E0,4.7545437E-2,0E0,4.0753383E-2,0E0,0E0,1.3863416E-2,1.2195757E-2,1.1813378E-2,5.085065E-3,0E0,0E0,4.3067113E-3,5.2789245E-3,4.6550725E-3,6.7174416E-3,5.1164427E-3,0E0,0E0,0E0,3.5820229E-3,0E0,0E0,1.2515364E-3,0E0,1.1644124E-3,1.6753147E-3,9.642241E-4,0E0,1.1072864E-3,0E0,1.9349863E-3,0E0,0E0,0E0,0E0,0E0,1.4382573E-3,1.3582696E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,22,22,25,25,27,27,28,28,29,29,31,31,33,33,39,39,40,40],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,14,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,-1,-1,30,-1,32,34,36,-1,38,-1,40,-1,-1,-1,-1,-1,42,44,-1,-1,-1,-1],"split_conditions":[1.717542E6,1.07858846E11,4.3092645E-3,5.3560664E3,-3.3132173E-3,2.2E1,3.96019E-3,3.2506369E-3,9.0582E4,1.3957454E9,4.354E3,4.0221E4,-2.0883733E-3,1.9880845E-3,9.973E3,1.7986E4,1.59E2,9.1735E4,8.6033555E10,-1.4329924E-4,-1.8069467E-3,-1.3267839E-3,1.05E4,-1.1305251E-3,5.213238E-4,1.5115035E9,-6.064499E-4,4.66838E5,8.8035E4,3.6422E4,1.4364535E-3,3.5699836E7,7.77989E-6,5.78082E5,3.899637E-4,1.3311788E-4,9.1356726E-4,2.6098793E-4,1.1210822E-3,1.072373E6,1.9867595E10,-3.699672E-4,-1.134212E-3,3.6173608E-4,-3.2705022E-4],"split_indices":[12,1,0,5,0,20,0,0,12,16,20,18,0,0,3,18,22,20,13,0,0,0,28,0,0,27,0,6,28,7,0,21,0,6,0,0,0,0,0,2,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.66E2,1.61E2,5E0,1.55E2,6E0,1.5E2,5E0,6E0,1.44E2,9.8E1,4.6E1,9.1E1,7E0,7E0,3.9E1,1.4E1,7.7E1,1.7E1,2.2E1,7E0,7E0,5E0,7.2E1,9E0,8E0,1.7E1,5E0,2E1,5.2E1,1.2E1,5E0,1.2E1,8E0,4.6E1,6E0,6E0,6E0,7E0,5E0,2.6E1,2E1,2E1,6E0,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[6.176478E-3,3.3506956E-3,3.7603057E-3,5.7640183E-3,-2.8099262E-3,1.8060104E-3,4.8740325E-3,2.4233959E-3,-6.159318E-4,-3.0171084E-3,1.6995465E-3,-5.4169446E-3,1.1754138E-2,-1.7736894E-3,-2.7302832E-3,-4.498164E-5,1.079723E-3,-1.6077543E-2,6.0027896E-4,5.2470644E-4,-6.1608414E-4,-2.2321367E-3,-1.6980924E-3,-1.8804597E-3,1.241037E-3,-3.9958413E-4,3.481467E-4,7.2292873E-4,-4.134228E-3,-2.7776279E-2,-2.224468E-5,-1.6295369E-3,-5.235718E-4,7.59124E-4,-2.0455042E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":110,"left_children":[1,3,-1,5,-1,7,-1,-1,9,11,-1,13,15,-1,17,19,-1,21,23,-1,-1,25,-1,27,-1,-1,-1,-1,29,31,33,-1,-1,-1,-1],"loss_changes":[3.971258E-2,2.8484922E-2,0E0,6.980053E-2,0E0,2.11102E-2,0E0,0E0,1.4933928E-2,5.310061E-3,0E0,1.2001044E-2,3.2181914E-3,0E0,5.3441166E-3,2.2070399E-3,0E0,7.498557E-3,6.7510353E-3,0E0,0E0,1.2037712E-3,0E0,3.8095536E-3,0E0,0E0,0E0,0E0,7.8738555E-3,1.3621738E-3,5.767324E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8,9,9,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,28,28,29,29,30,30],"right_children":[2,4,-1,6,-1,8,-1,-1,10,12,-1,14,16,-1,18,20,-1,22,24,-1,-1,26,-1,28,-1,-1,-1,-1,30,32,34,-1,-1,-1,-1],"split_conditions":[9.77508E5,2.5742707E10,3.7603057E-3,7.9463E4,-2.8099262E-3,2.2E1,4.8740325E-3,2.4233959E-3,4.65E2,9.076228E6,1.6995465E-3,1.4E1,4.9E1,-1.7736894E-3,1E0,1.3E1,1.079723E-3,6.490565E8,7.12E2,5.2470644E-4,-6.1608414E-4,1.7198737E10,-1.6980924E-3,5.261E3,1.241037E-3,-3.9958413E-4,3.481467E-4,7.2292873E-4,1.97E2,2.9435686E3,5.89E2,-1.6295369E-3,-5.235718E-4,7.59124E-4,-2.0455042E-4],"split_indices":[12,21,0,3,0,20,0,0,4,2,0,17,17,0,29,17,0,16,17,0,0,10,0,28,0,0,0,0,20,5,20,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,1.71E2,5E0,1.66E2,5E0,1.61E2,5E0,6E0,1.55E2,1.47E2,8E0,1.27E2,2E1,8E0,1.19E2,1.1E1,9E0,2.3E1,9.6E1,6E0,5E0,1.5E1,8E0,8.9E1,7E0,9E0,6E0,9E0,8E1,1.1E1,6.9E1,6E0,5E0,1.4E1,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.952648E-3,2.7680793E-3,4.0157526E-3,5.876382E-3,-3.4945155E-3,2.2021967E-3,4.3298206E-3,5.579933E-4,1.8622489E-3,-9.651967E-4,1.379019E-3,-4.6803975E-3,1.4460821E-2,1.2748421E-3,-6.4700404E-3,-5.955197E-4,2.447882E-2,4.7367465E-4,-8.484036E-3,3.9858133E-2,1.5607104E-4,-1.0241451E-2,5.676901E-4,2.4307847E-3,8.987868E-4,-2.7175147E-2,-7.873834E-3,-1.6218587E-3,-5.526792E-4,-6.2214313E-3,-1.2239806E-3,-7.7693287E-4,-1.6788614E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":111,"left_children":[1,3,-1,5,-1,7,-1,9,-1,11,-1,13,15,-1,17,-1,19,-1,21,23,-1,25,-1,-1,-1,27,29,-1,-1,31,-1,-1,-1],"loss_changes":[4.605824E-2,4.269473E-2,0E0,5.430139E-2,0E0,1.0221979E-2,0E0,7.074481E-3,0E0,8.26312E-3,0E0,7.224174E-3,8.21751E-3,0E0,3.9645676E-3,0E0,6.6084433E-3,0E0,3.9099692E-3,2.5138296E-3,0E0,3.5146298E-3,0E0,0E0,0E0,1.123229E-3,2.5916924E-3,0E0,0E0,2.0765103E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,9,9,11,11,12,12,14,14,16,16,18,18,19,19,21,21,25,25,26,26,29,29],"right_children":[2,4,-1,6,-1,8,-1,10,-1,12,-1,14,16,-1,18,-1,20,-1,22,24,-1,26,-1,-1,-1,28,30,-1,-1,32,-1,-1,-1],"split_conditions":[1.717542E6,5.294278E7,4.0157526E-3,8.539015E3,-3.4945155E-3,2.8413632E7,4.3298206E-3,4.85E2,1.8622489E-3,3.48E2,1.379019E-3,2.4E1,2.7616E4,1.2748421E-3,2.6E2,-5.955197E-4,1.8787456E7,4.7367465E-4,4.9250466E-1,9.6703277E8,1.5607104E-4,2.3E1,5.676901E-4,2.4307847E-3,8.987868E-4,2.08298E5,3.4633E4,-1.6218587E-3,-5.526792E-4,4.8714E4,-1.2239806E-3,-7.7693287E-4,-1.6788614E-4],"split_indices":[12,2,0,5,0,2,0,4,0,17,0,4,7,0,28,0,18,0,58,21,0,17,0,0,0,18,7,0,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,1.63E2,5E0,1.58E2,5E0,1.53E2,5E0,1.48E2,5E0,1.42E2,6E0,1.15E2,2.7E1,5E0,1.1E2,7E0,2E1,1.1E1,9.9E1,1.1E1,9E0,9.2E1,7E0,5E0,6E0,1E1,8.2E1,5E0,5E0,7.7E1,5E0,1.2E1,6.5E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.0884751E-3,-7.0899357E-3,4.7150454E-3,-1.8869637E-3,7.700036E-2,3.6193654E-3,-5.1519484E-3,2.728539E-4,7.432652E-3,3.3101533E-3,5.638469E-4,-4.7004637E-3,1.47225335E-2,-1.3640121E-3,-2.1891272E-2,1.9007212E-2,-5.2424584E-4,-3.523644E-3,1.2700455E-3,-5.7586627E-3,-2.204516E-3,2.4611415E-2,-9.9640696E-5,-7.744637E-3,1.2383588E-2,-1.0384843E-3,7.0886913E-4,7.4780667E-3,3.5923213E-2,-1.6109988E-2,5.2451435E-3,2.380865E-2,-2.1272738E-4,1.0380606E-3,-5.582816E-4,6.5325655E-4,2.206772E-3,-9.804015E-3,-1.7850118E-3,-4.502038E-3,1.22277085E-2,1.6968118E-3,2.0834115E-4,3.748002E-4,-1.405165E-2,-7.9742656E-4,5.373308E-4,9.283587E-4,1.2359548E-4,-8.2120614E-4,2.858377E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":112,"left_children":[1,-1,3,5,7,9,-1,-1,-1,-1,11,13,15,17,19,21,-1,23,-1,25,-1,27,-1,29,31,-1,-1,33,35,37,39,41,-1,-1,-1,-1,-1,43,-1,45,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6182627E-1,0E0,7.984243E-2,9.8919444E-2,9.133788E-2,3.2616317E-2,0E0,0E0,0E0,0E0,1.0727811E-2,5.9925513E-3,4.695545E-3,5.957583E-3,7.5573875E-3,4.11286E-3,0E0,5.7360283E-3,0E0,5.147367E-3,0E0,4.9642064E-3,0E0,7.364994E-3,3.6650451E-3,0E0,0E0,4.2700414E-3,4.150901E-3,6.0075074E-3,1.8926784E-3,2.969334E-3,0E0,0E0,0E0,0E0,0E0,2.776699E-3,0E0,3.0026573E-3,1.2991128E-3,0E0,0E0,0E0,2.870642E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,37,37,39,39,40,40,44,44],"right_children":[2,-1,4,6,8,10,-1,-1,-1,-1,12,14,16,18,20,22,-1,24,-1,26,-1,28,-1,30,32,-1,-1,34,36,38,40,42,-1,-1,-1,-1,-1,44,-1,46,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1],"split_conditions":[7.095E4,-7.0899357E-3,1.7903596E0,4.831919E7,2.613671E6,2.5E1,-5.1519484E-3,2.728539E-4,7.432652E-3,3.3101533E-3,1.1257E4,8.527E3,4.3822035E-1,1E0,3.0545714E-1,1.5805948E11,-5.2424584E-4,6.6299E4,1.2700455E-3,5.309569E6,-2.204516E-3,2.0822124E9,-9.9640696E-5,2.95E2,1.1415993E0,-1.0384843E-3,7.0886913E-4,1.2159447E9,7.2E1,2.160768E6,3.33044E5,2.48E2,-2.1272738E-4,1.0380606E-3,-5.582816E-4,6.5325655E-4,2.206772E-3,2.81E3,-1.7850118E-3,9.7786E4,4.3E1,1.6968118E-3,2.0834115E-4,3.748002E-4,6.519988E8,-7.9742656E-4,5.373308E-4,9.283587E-4,1.2359548E-4,-8.2120614E-4,2.858377E-4],"split_indices":[6,0,66,6,18,20,0,0,0,0,3,3,62,43,66,10,0,28,0,18,0,11,0,4,65,0,0,11,4,2,6,19,0,0,0,0,0,28,0,18,17,0,0,0,23,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,5E0,1.66E2,1.53E2,1.3E1,1.47E2,6E0,8E0,5E0,5E0,1.42E2,1.04E2,3.8E1,8.8E1,1.6E1,3.3E1,5E0,8.3E1,5E0,1.1E1,5E0,2.6E1,7E0,6.6E1,1.7E1,6E0,5E0,1.1E1,1.5E1,4E1,2.6E1,1E1,7E0,6E0,5E0,7E0,8E0,3.3E1,7E0,1.1E1,1.5E1,5E0,5E0,6E0,2.7E1,6E0,5E0,7E0,8E0,2.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-6.271669E-4,-6.781987E-3,5.0041173E-3,1.0058144E-3,4.835708E-3,4.241529E-3,-3.6118315E-3,2.8927028E-3,1.7319103E-3,-4.205255E-3,2.2111805E-2,-1.023264E-2,4.989418E-3,3.2403085E-2,6.9159786E-3,-6.2235906E-3,-1.6218773E-3,1.6233409E-3,-6.303732E-4,7.918662E-4,1.8515667E-3,-2.3150582E-4,5.9169525E-4,-1.0954991E-3,-2.6128714E-3,1.3754504E-2,-1.1392765E-2,-8.9716725E-3,5.7094498E-3,-2.2302386E-4,9.341765E-4,-1.9979298E-2,3.612967E-4,-5.28615E-3,-1.0233889E-3,1.0720163E-2,-4.8489036E-4,-1.240528E-3,-4.0664893E-4,-6.28778E-4,5.735813E-5,-9.455996E-5,6.7553564E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":113,"left_children":[1,-1,3,5,-1,7,-1,-1,9,11,13,15,17,19,21,23,-1,-1,25,-1,-1,-1,-1,-1,27,29,31,33,35,-1,-1,37,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.489281E-1,0E0,6.967862E-2,4.3713156E-2,0E0,2.4015704E-2,0E0,0E0,1.8242806E-2,6.535177E-3,5.092973E-3,7.4331337E-3,8.529268E-3,2.0877216E-3,1.2890974E-3,4.228135E-3,0E0,0E0,6.504145E-3,0E0,0E0,0E0,0E0,0E0,2.9089022E-3,2.706469E-3,4.2190123E-3,1.5298778E-3,2.1416787E-3,0E0,0E0,1.2655258E-3,0E0,1.5839685E-3,0E0,1.1982932E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,18,18,24,24,25,25,26,26,27,27,28,28,31,31,33,33,35,35],"right_children":[2,-1,4,6,-1,8,-1,-1,10,12,14,16,18,20,22,24,-1,-1,26,-1,-1,-1,-1,-1,28,30,32,34,36,-1,-1,38,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1569896E7,-6.781987E-3,7.034231E11,8.6855156E10,4.835708E-3,6.7E1,-3.6118315E-3,2.8927028E-3,9.076228E6,7.115196E-2,3.47019E5,3.8305012E7,4.1804355E-1,1.55E2,3.0525E4,1.67E2,-1.6218773E-3,1.6233409E-3,7E1,7.918662E-4,1.8515667E-3,-2.3150582E-4,5.9169525E-4,-1.0954991E-3,2.88E2,6.1854714E-1,2.98E2,7.031403E8,3.86E2,-2.2302386E-4,9.341765E-4,1.28E2,3.612967E-4,5.441268E7,-1.0233889E-3,2.92E2,-4.8489036E-4,-1.240528E-3,-4.0664893E-4,-6.28778E-4,5.735813E-5,-9.455996E-5,6.7553564E-4],"split_indices":[11,0,13,1,0,22,0,0,2,62,12,27,66,4,7,22,0,0,19,0,0,0,0,0,4,66,17,16,4,0,0,19,0,16,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.69E2,5E0,1.64E2,1.59E2,5E0,1.54E2,5E0,5E0,1.49E2,1.16E2,3.3E1,7E1,4.6E1,1.9E1,1.4E1,6.2E1,8E0,6E0,4E1,9E0,1E1,5E0,9E0,9E0,5.3E1,1.7E1,2.3E1,3E1,2.3E1,5E0,1.2E1,1.6E1,7E0,2.5E1,5E0,1.8E1,5E0,8E0,8E0,1E1,1.5E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.3239166E-3,3.4726542E-4,3.9102808E-3,2.9408273E-3,-3.1947307E-3,1.8475265E-3,9.896279E-4,2.8183742E-3,-1.5186783E-3,-8.4330956E-4,2.9265426E-2,9.648235E-4,-1.5734322E-2,4.1320246E-2,1.177213E-4,-1.573361E-3,1.9513855E-2,2.5283935E-4,-1.7966965E-3,2.5203996E-3,1.0403902E-3,1.5116026E-4,-1.3744526E-3,-1.3268516E-4,1.3571905E-3,-3.0652124E-3,1.3536316E-3,-5.698675E-3,9.507859E-3,-1.02614395E-4,-1.2199432E-2,1.234615E-3,-3.306484E-3,-4.6969598E-4,1.8465564E-4,-7.485784E-4,1.2588945E-4,3.8927203E-4,-6.5385475E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":114,"left_children":[1,3,-1,5,-1,-1,7,9,-1,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,25,-1,-1,-1,27,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[4.618944E-2,3.3594064E-2,0E0,1.3023291E-2,0E0,0E0,1.0764836E-2,1.5131834E-2,0E0,3.7308217E-3,5.973799E-3,5.8687297E-3,8.449996E-3,2.3567583E-3,0E0,5.6799157E-3,4.012433E-3,0E0,0E0,0E0,0E0,1.051777E-2,0E0,0E0,0E0,3.232811E-3,0E0,2.9090382E-3,4.1843224E-3,2.0777527E-3,2.8127963E-3,0E0,1.7034945E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,21,21,25,25,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,-1,6,-1,-1,8,10,-1,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,26,-1,-1,-1,28,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[6.4E2,7.685281E3,3.9102808E-3,1.48E2,-3.1947307E-3,1.8475265E-3,3.648E3,6.820745E6,-1.5186783E-3,2.0911702E7,2.18E2,1.1257E4,2.9336E4,8.7091637E-1,1.177213E-4,1.6106268E9,6.7E1,2.5283935E-4,-1.7966965E-3,2.5203996E-3,1.0403902E-3,7.984941E8,-1.3744526E-3,-1.3268516E-4,1.3571905E-3,8.1996E4,1.3536316E-3,5.737E3,1.10745E5,1.5447E4,7.1537E4,1.234615E-3,2.2532697E-1,-4.6969598E-4,1.8465564E-4,-7.485784E-4,1.2588945E-4,3.8927203E-4,-6.5385475E-4],"split_indices":[4,5,0,28,0,0,19,6,0,18,4,3,7,63,0,16,17,0,0,0,0,16,0,0,0,28,0,3,28,7,8,0,62,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,1.74E2,5E0,1.69E2,5E0,7E0,1.62E2,1.55E2,7E0,1.37E2,1.8E1,1.23E2,1.4E1,1.2E1,6E0,1.09E2,1.4E1,8E0,6E0,5E0,7E0,1.04E2,5E0,5E0,9E0,9.5E1,9E0,7.9E1,1.6E1,4.3E1,3.6E1,6E0,1E1,1.2E1,3.1E1,2.7E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.6986597E-3,-1.5772012E-3,4.6600206E-3,3.1183134E-3,-5.727505E-3,-4.1555974E-4,3.6422082E-3,-3.0424104E-3,3.22358E-2,1.9626722E-2,-5.9170933E-3,2.6854838E-3,-3.1907743E-4,1.0254179E-4,1.6921954E-3,-2.3867497E-2,-3.7735705E-3,-1.492801E-3,-3.7742735E-4,9.903412E-3,-8.043776E-3,1.9736325E-3,6.620896E-4,-1.2730626E-2,5.0262082E-3,-6.187332E-4,5.312915E-3,-1.4142242E-3,-1.0304393E-2,1.1703014E-2,-7.885914E-4,-3.380401E-4,5.9252296E-4,-5.9374757E-3,-2.064922E-2,5.3086197E-3,9.960204E-4,-1.3545844E-2,1.0182318E-2,-3.8083878E-4,-1.8786739E-3,6.047406E-4,-3.6646603E-4,-1.825797E-4,-1.1430193E-3,6.946981E-4,-4.6763027E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":115,"left_children":[1,3,-1,5,-1,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,-1,-1,21,23,-1,25,27,29,-1,31,-1,33,35,-1,-1,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.892965E-2,1.0450653E-1,0E0,4.7157373E-2,0E0,1.3821313E-2,0E0,9.825952E-3,1.4350282E-2,5.531619E-3,5.027389E-3,0E0,0E0,0E0,0E0,1.9063E-3,7.0880842E-3,0E0,0E0,9.43435E-3,5.699637E-3,0E0,1.7315256E-3,2.911495E-3,4.0662517E-3,0E0,2.155038E-3,0E0,2.6782453E-3,1.3005785E-3,0E0,0E0,0E0,5.656968E-3,4.430666E-3,1.7561521E-3,0E0,3.5597337E-3,9.849836E-4,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,9,9,10,10,15,15,16,16,19,19,20,20,22,22,23,23,24,24,26,26,28,28,29,29,33,33,34,34,35,35,37,37,38,38],"right_children":[2,4,-1,6,-1,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,-1,-1,22,24,-1,26,28,30,-1,32,-1,34,36,-1,-1,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.717542E6,4.2250056E7,4.6600206E-3,9.0573E4,-5.727505E-3,7.12E2,3.6422082E-3,1.7726E4,1.852532E7,1.8853599E3,1.96E2,2.6854838E-3,-3.1907743E-4,1.0254179E-4,1.6921954E-3,1.8126E5,6.89E2,-1.492801E-3,-3.7742735E-4,6.4E1,7.594183E-1,1.9736325E-3,3.70676E5,4.607122E6,5.188088E-1,-6.187332E-4,3.1E2,-1.4142242E-3,1.78E2,4.833515E10,-7.885914E-4,-3.380401E-4,5.9252296E-4,1.9844815E9,1.495E3,7.7583334E8,9.960204E-4,3.432137E7,1E0,-3.8083878E-4,-1.8786739E-3,6.047406E-4,-3.6646603E-4,-1.825797E-4,-1.1430193E-3,6.946981E-4,-4.6763027E-5],"split_indices":[12,6,0,3,0,17,0,18,18,5,22,0,0,0,0,6,20,0,0,4,63,0,2,21,62,0,0,0,17,13,0,0,0,11,0,11,0,27,35,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.76E2,1.7E2,6E0,1.65E2,5E0,1.59E2,6E0,1.48E2,1.1E1,1.6E1,1.32E2,6E0,5E0,9E0,7E0,1.3E1,1.19E2,7E0,6E0,2.8E1,9.1E1,5E0,2.3E1,6.7E1,2.4E1,5E0,1.8E1,6E0,6.1E1,1.9E1,5E0,7E0,1.1E1,4.4E1,1.7E1,1.3E1,6E0,3E1,1.4E1,1.2E1,5E0,8E0,5E0,1.8E1,1.2E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[9.480762E-4,-2.798668E-3,4.776966E-3,1.7590171E-3,-5.704959E-3,-2.3954846E-3,1.7174134E-2,1.420724E-3,-4.287872E-3,2.7120534E-2,3.046099E-3,-2.5730502E-2,-1.6401788E-3,3.9963116E-4,3.21622E-2,6.490569E-4,-4.6428337E-4,-1.5140997E-3,-3.4931742E-4,9.3544193E-4,-2.3825545E-2,1.8759016E-3,8.281428E-4,-7.1617897E-4,1.223479E-3,-2.1924924E-3,9.657357E-5,1.6472391E-3,-1.6477808E-2,-3.7577355E-4,1.2666571E-3,-1.4511361E-3,-8.0357E-5,4.7823535E-3,-7.4390396E-3,5.1530066E-4,1.296378E-3,-1.6177103E-3,-1.716608E-2,2.7304818E-4,-4.4671437E-4,-4.757657E-4,2.77649E-4,-1.4410127E-3,-1.3676612E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":116,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,15,17,19,-1,21,-1,-1,-1,-1,23,25,-1,-1,27,-1,-1,-1,29,31,33,-1,-1,-1,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1],"loss_changes":[7.307207E-2,1.0166601E-1,0E0,1.086426E-2,0E0,9.001141E-3,4.9762195E-3,0E0,7.2248057E-3,1.6075196E-3,2.7469685E-3,1.9458812E-3,6.5935124E-3,0E0,1.4670528E-3,0E0,0E0,0E0,0E0,4.6742884E-3,8.3621945E-3,0E0,0E0,3.7150236E-3,0E0,0E0,0E0,4.828663E-3,3.129622E-3,3.0231003E-3,0E0,0E0,0E0,5.0885924E-3,1.954081E-3,2.6558877E-3,0E0,1.7728494E-3,2.7878114E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9,10,10,11,11,12,12,14,14,19,19,20,20,23,23,27,27,28,28,29,29,33,33,34,34,35,35,37,37,38,38],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,16,18,20,-1,22,-1,-1,-1,-1,24,26,-1,-1,28,-1,-1,-1,30,32,34,-1,-1,-1,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1],"split_conditions":[7.034231E11,4.831919E7,4.776966E-3,1.6353E4,-5.704959E-3,1.5825307E8,1.2918454E9,1.420724E-3,1.2368878E3,4.9E1,7.3133384E3,4.8E1,2.1035208E7,3.9963116E-4,1.94E2,6.490569E-4,-4.6428337E-4,-1.5140997E-3,-3.4931742E-4,6.44E2,1.32E2,1.8759016E-3,8.281428E-4,3.86E2,1.223479E-3,-2.1924924E-3,9.657357E-5,9.613032E10,4.6E1,3.1E1,1.2666571E-3,-1.4511361E-3,-8.0357E-5,1.086582E7,1.78E2,3.1426326E3,1.296378E-3,7.969174E7,2.77419E-1,2.7304818E-4,-4.4671437E-4,-4.757657E-4,2.77649E-4,-1.4410127E-3,-1.3676612E-4],"split_indices":[13,6,0,3,0,10,16,0,5,17,5,17,18,0,19,0,0,0,0,17,4,0,0,4,0,0,0,10,17,19,0,0,0,18,17,5,0,16,62,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.78E2,1.73E2,5E0,1.68E2,5E0,1.33E2,3.5E1,6E0,1.27E2,2E1,1.5E1,1.3E1,1.14E2,5E0,1.5E1,8E0,7E0,8E0,5E0,1.03E2,1.1E1,7E0,8E0,9.8E1,5E0,5E0,6E0,8.6E1,1.2E1,8.1E1,5E0,5E0,7E0,4.7E1,3.4E1,4.1E1,6E0,2.2E1,1.2E1,2.7E1,1.4E1,1E1,1.2E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.1356657E-3,4.891175E-3,-4.055423E-3,1.933834E-3,3.835137E-3,-1.8007903E-3,2.0153932E-2,1.3865028E-2,-6.3008387E-3,3.877269E-2,8.313917E-3,2.1595918E-3,5.4373643E-3,-9.310669E-3,1.3582786E-2,2.5001224E-3,6.024432E-4,-4.59191E-4,1.7816037E-2,7.4736675E-4,-1.3567383E-3,-2.9695255E-3,-2.209832E-2,-1.3375252E-4,1.4423535E-3,2.650172E-4,1.3314766E-3,-7.741719E-3,5.5446906E-4,-6.298797E-3,1.3693344E-3,-3.6685582E-2,-1.271044E-2,1.8294698E-4,-8.1047957E-4,-2.0146508E-2,-9.27467E-4,-6.9001917E-4,-2.397403E-3,-2.4146675E-3,-1.6606532E-3,-2.927547E-4,-1.1726707E-3,3.2607706E-3,-9.0373843E-4,-5.1939895E-4,2.7432299E-4,-7.6931934E-3,1.157698E-2,-6.1216246E-4,2.729364E-4,1.7904935E-2,9.571512E-5,3.737217E-4,1.1949441E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":117,"left_children":[1,3,-1,5,-1,7,9,11,13,15,17,-1,19,21,23,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,33,-1,35,-1,37,39,-1,-1,41,43,-1,-1,45,-1,-1,-1,47,-1,-1,-1,49,51,-1,-1,53,-1,-1,-1],"loss_changes":[6.438466E-2,4.2773098E-2,0E0,1.160398E-2,0E0,1.0097927E-2,6.0705924E-3,9.3354415E-3,6.7694085E-3,4.2929556E-3,3.587821E-3,0E0,2.1668556E-3,7.799946E-3,4.961374E-3,0E0,0E0,0E0,1.7521512E-3,0E0,1.7142863E-3,7.7437605E-3,3.9158985E-3,0E0,0E0,0E0,0E0,1.7574914E-3,0E0,4.518532E-3,0E0,3.701495E-3,5.3862687E-3,0E0,0E0,1.3552424E-3,3.7951262E-3,0E0,0E0,1.3951092E-3,0E0,0E0,0E0,3.5506918E-3,0E0,0E0,0E0,1.588991E-3,1.2671344E-3,0E0,0E0,9.160582E-4,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,18,18,20,20,21,21,22,22,27,27,29,29,31,31,32,32,35,35,36,36,39,39,43,43,47,47,48,48,51,51],"right_children":[2,4,-1,6,-1,8,10,12,14,16,18,-1,20,22,24,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,34,-1,36,-1,38,40,-1,-1,42,44,-1,-1,46,-1,-1,-1,48,-1,-1,-1,50,52,-1,-1,54,-1,-1,-1],"split_conditions":[4.831919E7,8.539015E3,-4.055423E-3,3.51E2,3.835137E-3,6.779055E6,2.2018733E3,1.41E2,2.6572972E9,4.4200656E7,3.388615E7,2.1595918E-3,1E0,1.212077E7,3.876394E6,2.5001224E-3,6.024432E-4,-4.59191E-4,6.883E3,7.4736675E-4,9.50206E5,1.086582E7,3.9692026E8,-1.3375252E-4,1.4423535E-3,2.650172E-4,1.3314766E-3,2.8861E4,5.5446906E-4,1.8448E4,1.3693344E-3,5.36E2,7.021018E9,1.8294698E-4,-8.1047957E-4,1.9E1,8.221E4,-6.9001917E-4,-2.397403E-3,8.047E4,-1.6606532E-3,-2.927547E-4,-1.1726707E-3,1.203429E6,-9.0373843E-4,-5.1939895E-4,2.7432299E-4,2.3412058E3,1.33E2,-6.1216246E-4,2.729364E-4,2.5705E4,9.571512E-5,3.737217E-4,1.1949441E-3],"split_indices":[6,5,0,4,0,23,5,4,27,16,16,0,44,18,18,0,0,0,3,0,21,18,21,0,0,0,0,21,0,7,0,17,21,0,0,17,8,0,0,8,0,0,0,18,0,0,0,5,4,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.8E2,1.74E2,6E0,1.69E2,5E0,1.41E2,2.8E1,3.1E1,1.1E2,1E1,1.8E1,5E0,2.6E1,9.6E1,1.4E1,5E0,5E0,6E0,1.2E1,9E0,1.7E1,6.5E1,3.1E1,8E0,6E0,7E0,5E0,1.2E1,5E0,6E1,5E0,1.1E1,2E1,6E0,6E0,1.6E1,4.4E1,6E0,5E0,1.5E1,5E0,6E0,1E1,3.7E1,7E0,7E0,8E0,1.6E1,2.1E1,1.1E1,5E0,1.2E1,9E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-6.3799466E-3,-7.195798E-3,-9.0975646E-4,5.7220566E-3,-4.2873334E-2,2.607115E-3,3.5390866E-3,-8.915575E-3,-6.252887E-3,3.0454353E-3,-2.4929695E-4,-2.5782434E-2,1.0336571E-3,-3.1785094E-3,3.1748712E-2,-3.4441423E-4,-1.7454729E-3,-2.151621E-2,-8.0035534E-4,7.454106E-4,1.9018213E-3,-1.516404E-3,-4.988321E-4,-2.5158594E-3,8.757857E-4,-1.44731E-4,-2.167042E-2,1.667446E-3,-1.2312514E-3,3.6815673E-4,-1.9370151E-3,-1.5504926E-2,5.1686065E-3,-1.2499682E-3,-9.201462E-5,2.2687842E-4,1.7737238E-2,9.7506697E-4,-1.6757466E-4,1.2776455E-3,4.2247655E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":118,"left_children":[1,-1,3,5,7,9,-1,11,-1,-1,13,15,-1,17,19,-1,-1,21,23,-1,-1,-1,-1,25,-1,27,29,31,-1,-1,-1,33,35,-1,-1,37,39,-1,-1,-1,-1],"loss_changes":[1.5589868E-1,0E0,4.894688E-2,3.5470523E-2,8.053705E-2,2.8220415E-2,0E0,1.10572195E-2,0E0,0E0,1.3410427E-2,2.989144E-3,0E0,5.6956727E-3,1.3374509E-3,0E0,0E0,1.6042227E-3,4.2384975E-3,0E0,0E0,0E0,0E0,4.9474174E-3,0E0,5.0311284E-3,8.578188E-3,5.671599E-3,0E0,0E0,0E0,2.7547155E-3,4.8406157E-3,0E0,0E0,5.2109784E-3,4.227231E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,7,7,10,10,11,11,13,13,14,14,17,17,18,18,23,23,25,25,26,26,27,27,31,31,32,32,35,35,36,36],"right_children":[2,-1,4,6,8,10,-1,12,-1,-1,14,16,-1,18,20,-1,-1,22,24,-1,-1,-1,-1,26,-1,28,30,32,-1,-1,-1,34,36,-1,-1,38,40,-1,-1,-1,-1],"split_conditions":[1.5140202E8,-7.195798E-3,1.87875E7,9.48749E5,3.1392E4,2.2E1,3.5390866E-3,3.274975E3,-6.252887E-3,3.0454353E-3,1E0,2.9409E4,1.0336571E-3,1.96E2,1.38E2,-3.4441423E-4,-1.7454729E-3,1.8126E5,9.0166E4,7.454106E-4,1.9018213E-3,-1.516404E-3,-4.988321E-4,8.221E4,8.757857E-4,1.8454934E9,7.264501E-2,1.5E1,-1.2312514E-3,3.6815673E-4,-1.9370151E-3,8.56088E5,8.57207E8,-1.2499682E-3,-9.201462E-5,6.413E3,3.1885722E-1,9.7506697E-4,-1.6757466E-4,1.2776455E-3,4.2247655E-5],"split_indices":[10,0,18,12,3,20,0,5,0,0,43,7,0,22,4,0,0,6,8,0,0,0,0,8,0,16,66,17,0,0,0,6,21,0,0,28,62,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,5E0,1.74E2,1.51E2,2.3E1,1.46E2,5E0,1.8E1,5E0,5E0,1.41E2,1.2E1,6E0,1.3E2,1.1E1,6E0,6E0,1.4E1,1.16E2,6E0,5E0,5E0,9E0,1.08E2,8E0,9.7E1,1.1E1,9.2E1,5E0,5E0,6E0,1.5E1,7.7E1,7E0,8E0,5.6E1,2.1E1,8E0,4.8E1,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-9.409554E-5,-7.245858E-3,5.7623396E-3,1.3265802E-3,5.5337898E-3,5.725331E-3,-5.0715343E-3,1.7591973E-3,4.6802405E-3,3.6719646E-3,-1.5473162E-3,-3.067978E-3,9.871741E-4,-9.639082E-3,1.9996322E-3,-6.0519176E-3,-1.8379991E-3,9.798639E-3,-5.8481465E-3,-1.4822898E-3,-2.7152647E-3,-5.750478E-4,1.3355394E-2,1.0478086E-3,-1.0661468E-2,-5.2104588E-3,7.5434835E-4,1.5862477E-3,8.4754145E-3,-2.180678E-2,-1.6621127E-3,2.842031E-4,-8.158045E-3,-2.179583E-3,1.737833E-2,-1.3779437E-3,-4.67248E-4,-9.243391E-3,5.1559624E-4,-5.540283E-4,-1.0534441E-4,3.5118635E-4,-3.5340994E-4,1.3142715E-3,1.18470736E-4,-8.0253714E-4,2.624327E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":119,"left_children":[1,-1,3,5,-1,7,-1,9,-1,-1,11,13,-1,15,17,19,-1,21,23,-1,25,-1,27,-1,29,31,-1,-1,33,35,37,-1,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7100063E-1,0E0,9.109273E-2,8.596137E-2,0E0,6.4224064E-2,0E0,4.2345405E-2,0E0,0E0,5.5813133E-3,4.747845E-3,0E0,6.966956E-3,5.0148997E-3,5.215779E-3,0E0,3.5261307E-3,6.14764E-3,0E0,2.6848249E-3,0E0,3.7303753E-3,0E0,3.5745767E-3,1.6770278E-3,0E0,0E0,2.9785545E-3,1.3962756E-3,2.2545038E-3,0E0,9.593908E-4,9.7279425E-4,3.092486E-3,0E0,0E0,2.0634914E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,10,10,11,11,13,13,14,14,15,15,17,17,18,18,20,20,22,22,24,24,25,25,28,28,29,29,30,30,32,32,33,33,34,34,37,37],"right_children":[2,-1,4,6,-1,8,-1,10,-1,-1,12,14,-1,16,18,20,-1,22,24,-1,26,-1,28,-1,30,32,-1,-1,34,36,38,-1,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.72232E5,-7.245858E-3,1.717542E6,1.61025E5,5.5337898E-3,9.0573E4,-5.0715343E-3,2.2E1,4.6802405E-3,3.6719646E-3,4.65E2,3.489065E-2,9.871741E-4,3.8305012E7,7E1,1.8126E5,-1.8379991E-3,8.48E3,9.675392E-2,-1.4822898E-3,1.07218E5,-5.750478E-4,5.89E2,1.0478086E-3,1.0569768E0,4.695E3,7.5434835E-4,1.5862477E-3,2.8681312E8,5.911733E8,7.502168E8,2.842031E-4,2.61E2,1.382443E9,1.1559897E7,-1.3779437E-3,-4.67248E-4,1.0564923E10,5.1559624E-4,-5.540283E-4,-1.0534441E-4,3.5118635E-4,-3.5340994E-4,1.3142715E-3,1.18470736E-4,-8.0253714E-4,2.624327E-4],"split_indices":[16,0,12,3,0,3,0,20,0,0,4,62,0,27,19,6,0,12,62,0,12,0,20,0,66,3,0,0,16,16,16,0,4,10,18,0,0,21,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,5E0,1.69E2,1.64E2,5E0,1.59E2,5E0,1.54E2,5E0,5E0,1.49E2,1.41E2,8E0,6.1E1,8E1,5.6E1,5E0,4E1,4E1,5E0,5.1E1,5E0,3.5E1,5E0,3.5E1,4.6E1,5E0,5E0,3E1,1.5E1,2E1,9E0,3.7E1,1.4E1,1.6E1,7E0,8E0,1.3E1,7E0,2E1,1.7E1,5E0,9E0,8E0,8E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.1129134E-3,-6.911936E-3,3.5134144E-3,-2.2438564E-3,5.79879E-2,8.418586E-4,-3.3813226E-3,-1.57424E-5,6.7505753E-3,1.3888279E-3,-9.343117E-4,4.034809E-4,-4.0474063E-4,-8.208276E-3,7.113564E-3,7.493312E-3,-1.1068764E-2,-2.6622713E-5,2.6552217E-2,-1.8255976E-4,8.671064E-4,-1.4332926E-3,-8.201119E-3,9.027847E-3,-1.2960311E-2,2.2134027E-3,1.335928E-2,-1.024905E-2,3.4442305E-4,1.6922903E-3,1.6624249E-3,-2.2653773E-2,2.4543918E-4,1.0399232E-3,4.2961037E-5,-4.6491846E-3,-2.6260259E-2,8.572484E-3,-6.11043E-4,-1.5713831E-3,-3.5538294E-4,-8.908369E-3,3.0562564E-4,-1.814676E-3,-3.1394296E-4,1.5452173E-2,-1.940502E-4,1.6925632E-4,-1.2412507E-2,-8.930026E-5,1.2327255E-3,-7.93211E-4,1.4627574E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":120,"left_children":[1,-1,3,5,7,9,-1,11,-1,-1,13,-1,-1,15,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,35,-1,37,-1,39,-1,-1,-1,41,43,45,-1,-1,-1,47,-1,-1,-1,49,-1,-1,51,-1,-1,-1,-1],"loss_changes":[1.5176135E-1,0E0,5.250539E-2,3.534839E-2,8.972279E-2,8.117928E-3,0E0,1.028667E-3,0E0,0E0,8.253984E-3,0E0,0E0,3.4065936E-3,9.300012E-3,1.8167039E-3,3.663619E-3,5.972388E-3,5.0597154E-3,0E0,0E0,0E0,1.9593576E-3,6.330721E-3,3.910317E-3,0E0,1.6603507E-3,4.4506025E-3,0E0,2.8191071E-3,0E0,2.4187411E-3,0E0,0E0,0E0,1.9977733E-3,3.4974394E-3,1.6808235E-3,0E0,0E0,0E0,1.3474696E-3,0E0,0E0,0E0,2.7435997E-3,0E0,0E0,2.1496075E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,7,7,10,10,13,13,14,14,15,15,16,16,17,17,18,18,22,22,23,23,24,24,26,26,27,27,29,29,31,31,35,35,36,36,37,37,41,41,45,45,48,48],"right_children":[2,-1,4,6,8,10,-1,12,-1,-1,14,-1,-1,16,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,36,-1,38,-1,40,-1,-1,-1,42,44,46,-1,-1,-1,48,-1,-1,-1,50,-1,-1,52,-1,-1,-1,-1],"split_conditions":[1.72232E5,-6.911936E-3,1.338554E0,4.5628892E7,6.4399E4,3.017E3,-3.3813226E-3,4.61E2,6.7505753E-3,1.3888279E-3,3.135717E6,4.034809E-4,-4.0474063E-4,1E0,2.325882E9,6.97E3,1.4E1,1.66E2,3.9556157E10,-1.8255976E-4,8.671064E-4,-1.4332926E-3,5.5015055E9,1.5063692E9,3.7188347E-2,2.2134027E-3,1E0,3.329881E6,3.4442305E-4,6.519E3,1.6624249E-3,5.9339E4,2.4543918E-4,1.0399232E-3,4.2961037E-5,1.3116845E10,7.0988134E8,6.1E1,-6.11043E-4,-1.5713831E-3,-3.5538294E-4,1.1554496E8,3.0562564E-4,-1.814676E-3,-3.1394296E-4,1.26E2,-1.940502E-4,1.6925632E-4,8.008664E8,-8.930026E-5,1.2327255E-3,-7.93211E-4,1.4627574E-4],"split_indices":[16,0,63,2,8,3,0,0,0,0,18,0,0,44,11,3,17,4,10,0,0,0,11,16,62,0,42,9,0,3,0,12,0,0,0,10,27,19,0,0,0,11,0,0,0,17,0,0,21,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.71E2,5E0,1.66E2,1.51E2,1.5E1,1.46E2,5E0,1E1,5E0,7E0,1.39E2,5E0,5E0,7.3E1,6.6E1,1.1E1,6.2E1,4.9E1,1.7E1,6E0,5E0,6E0,5.6E1,2.9E1,2E1,5E0,1.2E1,5E1,6E0,2.4E1,5E0,1.3E1,7E0,6E0,6E0,3.8E1,1.2E1,1.7E1,7E0,6E0,7E0,2.8E1,1E1,6E0,6E0,1.1E1,6E0,6E0,2.2E1,5E0,6E0,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[2.985328E-4,-7.311363E-3,6.11681E-3,2.611862E-3,4.505928E-3,6.3768323E-3,-3.705337E-3,2.9354703E-3,4.1623013E-3,3.1638774E-3,1.7066389E-4,-5.2070976E-3,7.4487E-3,1.2038527E-3,-7.942633E-3,2.7305366E-2,2.8981685E-3,-1.3958269E-3,-4.541045E-3,4.9458357E-4,1.8065762E-3,-2.6303546E-3,1.8881831E-2,-6.594177E-3,8.5044384E-4,3.7936163E-3,-1.8471086E-2,3.6995907E-4,1.3680403E-3,1.864006E-3,-1.2561505E-2,-3.352912E-3,1.211379E-3,-3.6127766E-4,-1.1909214E-3,-3.526569E-3,7.182139E-4,-1.802029E-2,1.6214956E-4,6.208073E-4,-8.726855E-3,1.8159173E-4,-5.7470176E-4,-5.3734204E-4,-1.610663E-3,3.86337E-4,-4.9978175E-4,6.3512955E-5,-7.0988387E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":121,"left_children":[1,-1,3,5,-1,7,-1,9,-1,-1,11,13,15,-1,17,19,21,-1,23,-1,-1,25,27,29,-1,31,33,-1,-1,35,37,39,-1,-1,-1,41,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.748177E-1,0E0,5.861909E-2,5.5889767E-2,0E0,4.932991E-2,0E0,3.0178612E-2,0E0,0E0,5.9879967E-3,8.112203E-3,5.794159E-3,0E0,6.559516E-3,2.2309534E-3,4.8214328E-3,0E0,3.7294193E-3,0E0,0E0,4.242249E-3,1.54971E-3,3.4555623E-3,0E0,5.2300747E-3,8.5236225E-4,0E0,0E0,2.3390055E-3,2.7811504E-3,2.3910226E-3,0E0,0E0,0E0,1.6851323E-3,0E0,2.5640847E-3,1.4131307E-3,0E0,1.4818351E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,10,10,11,11,12,12,14,14,15,15,16,16,18,18,21,21,22,22,23,23,25,25,26,26,29,29,30,30,31,31,35,35,37,37,38,38,40,40],"right_children":[2,-1,4,6,-1,8,-1,10,-1,-1,12,14,16,-1,18,20,22,-1,24,-1,-1,26,28,30,-1,32,34,-1,-1,36,38,40,-1,-1,-1,42,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7739E4,-7.311363E-3,7.034231E11,1.1686055E11,4.505928E-3,1.5522577E7,-3.705337E-3,2.5E1,4.1623013E-3,3.1638774E-3,6.2521696E8,1.7726E4,2.7779735E9,1.2038527E-3,1.6E1,8.694222E6,2.6572972E9,-1.3958269E-3,8.1996E4,4.9458357E-4,1.8065762E-3,7.021018E9,3.1392E4,1.3419E4,8.5044384E-4,7.7647084E9,7.1012E4,3.6995907E-4,1.3680403E-3,1.2254326E7,7.041E3,4.7E1,1.211379E-3,-3.6127766E-4,-1.1909214E-3,1.05E4,7.182139E-4,7.98E2,8.941E3,6.208073E-4,2.779416E8,1.8159173E-4,-5.7470176E-4,-5.3734204E-4,-1.610663E-3,3.86337E-4,-4.9978175E-4,6.3512955E-5,-7.0988387E-4],"split_indices":[6,0,13,26,0,6,0,20,0,0,21,18,25,0,17,18,27,0,28,0,0,21,3,28,0,23,8,0,0,18,3,4,0,0,0,28,0,0,3,0,27,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.77E2,5E0,1.72E2,1.67E2,5E0,1.61E2,6E0,1.56E2,5E0,5E0,1.51E2,8.7E1,6.4E1,6E0,8.1E1,1.1E1,5.3E1,9E0,7.2E1,6E0,5E0,4E1,1.3E1,6.7E1,5E0,2.9E1,1.1E1,8E0,5E0,2.8E1,3.9E1,2.3E1,6E0,6E0,5E0,2.1E1,7E0,2.7E1,1.2E1,5E0,1.8E1,1.2E1,9E0,2.2E1,5E0,7E0,5E0,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.7528366E-3,-7.131291E-3,3.250587E-3,-1.3117131E-3,5.529152E-2,1.5100267E-3,-3.0035563E-3,5.384525E-3,-1.1677375E-4,1.8094916E-2,-1.0028534E-3,1.8558801E-3,3.0341465E-4,-1.2653688E-3,6.381467E-4,4.6206554E-4,-5.125902E-4,-8.0031116E-4,2.341783E-3,-7.4948097E-4,2.2175837E-2,3.897902E-3,-1.3590331E-2,5.253703E-4,1.5441503E-3,-1.4114731E-3,1.4575464E-2,-5.397029E-3,-1.7459568E-3,8.957363E-3,-1.172385E-2,1.3633983E-3,2.496083E-3,-7.8263326E-4,8.119626E-4,-8.497339E-4,1.752633E-3,-1.7090358E-2,7.927471E-5,-7.1507343E-4,5.9246353E-4,5.077733E-4,-2.5150448E-4,2.910601E-4,-6.880866E-4,-1.0945861E-3,-4.1701973E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":122,"left_children":[1,-1,3,5,7,9,-1,-1,-1,11,13,-1,15,-1,17,-1,-1,-1,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,-1,39,-1,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6061291E-1,0E0,3.829817E-2,2.8484879E-2,5.1628593E-2,6.019467E-3,0E0,0E0,0E0,8.24382E-3,5.8055026E-3,0E0,1.610936E-3,0E0,3.92195E-3,0E0,0E0,0E0,6.822285E-3,5.8415798E-3,1.5820456E-3,4.102338E-3,5.410441E-3,0E0,0E0,5.3444547E-3,4.7522914E-3,1.64885E-3,0E0,7.683026E-3,1.8476201E-3,0E0,3.3595227E-3,0E0,1.1341977E-3,2.3478372E-3,0E0,2.295936E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,9,9,10,10,12,12,14,14,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,29,29,30,30,32,32,34,34,35,35,37,37],"right_children":[2,-1,4,6,8,10,-1,-1,-1,12,14,-1,16,-1,18,-1,-1,-1,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,-1,40,-1,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.432002E7,-7.131291E-3,1.7903596E0,1.7087803E0,7.7E1,1E0,-3.0035563E-3,5.384525E-3,-1.1677375E-4,4.510046E10,3.4403656E7,1.8558801E-3,1.350694E-6,-1.2653688E-3,3.4E1,4.6206554E-4,-5.125902E-4,-8.0031116E-4,9.1735E4,7.7E1,7.4118E4,7.115196E-2,3.0491E4,5.253703E-4,1.5441503E-3,1.3555108E-3,2.6E1,5.502E3,-1.7459568E-3,2.8958064E-4,2.95E2,1.3633983E-3,4.3923E4,-7.8263326E-4,7.89E2,2.3844E4,1.752633E-3,1.1E1,7.927471E-5,-7.1507343E-4,5.9246353E-4,5.077733E-4,-2.5150448E-4,2.910601E-4,-6.880866E-4,-1.0945861E-3,-4.1701973E-5],"split_indices":[11,0,66,66,19,44,0,0,0,10,16,0,62,0,4,0,0,0,20,19,8,62,7,0,0,63,19,3,0,62,4,0,28,0,0,7,0,19,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.65E2,5E0,1.6E2,1.48E2,1.2E1,1.43E2,5E0,5E0,7E0,1.8E1,1.25E2,7E0,1.1E1,6E0,1.19E2,6E0,5E0,9E0,1.1E2,9.6E1,1.4E1,7.1E1,2.5E1,9E0,5E0,4.8E1,2.3E1,2E1,5E0,2.4E1,2.4E1,9E0,1.4E1,6E0,1.4E1,1.9E1,5E0,1.7E1,7E0,5E0,9E0,5E0,9E0,1.3E1,6E0,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-7.084794E-4,-6.8950555E-3,5.0494378E-3,3.1466945E-3,2.4876853E-3,-4.3135897E-5,2.9256889E-3,2.2231485E-3,-2.4579912E-3,-3.1519716E-3,9.41123E-3,-7.3433213E-4,-1.5391988E-3,2.5780484E-2,4.3122284E-3,-1.9059695E-2,2.728915E-3,-1.7197795E-4,1.7910448E-3,-1.1972992E-2,9.770008E-3,-1.5681422E-4,-1.5933579E-3,1.1196408E-3,1.3856421E-4,-1.043058E-5,-9.600829E-4,4.1965013E-3,1.3157842E-3,-9.127368E-3,6.8964455E-3,-1.404572E-4,9.7291486E-4,-1.4133532E-2,1.0716854E-4,1.3236156E-3,1.3835438E-3,4.9743196E-3,-6.4309547E-4,-7.16294E-3,-1.0937848E-3,-4.942656E-3,8.461E-3,-2.425148E-3,8.320255E-4,1.1532024E-4,-6.959379E-4,-4.7004406E-4,3.2884823E-4,7.2436035E-4,-3.192916E-5,2.7220786E-4,-6.552957E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":123,"left_children":[1,-1,3,-1,5,7,-1,9,-1,11,13,15,-1,17,19,21,23,-1,-1,25,27,-1,-1,-1,29,-1,-1,31,-1,33,35,37,-1,39,-1,-1,41,43,-1,45,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5383035E-1,0E0,2.7988043E-2,0E0,2.5975088E-2,1.9764636E-2,0E0,5.786547E-3,0E0,6.650901E-3,5.230707E-3,5.192699E-3,0E0,6.9622416E-3,4.561033E-3,3.3766911E-3,4.093626E-3,0E0,0E0,1.5237634E-3,4.1890093E-3,0E0,0E0,0E0,4.0078848E-3,0E0,0E0,2.4358013E-3,0E0,1.6002974E-3,4.7370438E-3,2.014008E-3,0E0,1.267592E-3,0E0,0E0,1.4297954E-3,2.1311508E-3,0E0,1.1615036E-3,0E0,1.3170027E-3,1.1338468E-3,1.6201584E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,20,20,24,24,27,27,29,29,30,30,31,31,33,33,36,36,37,37,39,39,41,41,42,42,43,43],"right_children":[2,-1,4,-1,6,8,-1,10,-1,12,14,16,-1,18,20,22,24,-1,-1,26,28,-1,-1,-1,30,-1,-1,32,-1,34,36,38,-1,40,-1,-1,42,44,-1,46,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5407658E-7,-6.8950555E-3,2.2E1,3.1466945E-3,3.9801333E11,9.4378E4,2.9256889E-3,2.565132E6,-2.4579912E-3,6.7436E4,1.6215E4,1.6E1,-1.5391988E-3,5.368E3,7.5E1,1.9829087E3,3.2926774E8,-1.7197795E-4,1.7910448E-3,2.861685E3,1.6946304E7,-1.5681422E-4,-1.5933579E-3,1.1196408E-3,2.8481894E8,-1.043058E-5,-9.600829E-4,2.61E2,1.3157842E-3,6.0700047E-1,2.9416174E9,1.78E2,9.7291486E-4,9.265308E6,1.0716854E-4,1.3236156E-3,2.26E2,6.2689E4,-6.4309547E-4,1.2689E4,-1.0937848E-3,1.1299395E10,3.0021282E3,9.7457E4,8.320255E-4,1.1532024E-4,-6.959379E-4,-4.7004406E-4,3.2884823E-4,7.2436035E-4,-3.192916E-5,2.7220786E-4,-6.552957E-4],"split_indices":[62,0,20,0,10,3,0,9,0,12,22,17,0,3,4,5,10,0,0,5,18,0,0,0,11,0,0,17,0,63,10,17,0,18,0,0,4,20,0,28,0,10,5,12,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,5E0,1.63E2,5E0,1.58E2,1.53E2,5E0,1.48E2,5E0,8.5E1,6.3E1,8E1,5E0,1.4E1,4.9E1,1.2E1,6.8E1,5E0,9E0,1.2E1,3.7E1,7E0,5E0,6E0,6.2E1,6E0,6E0,3E1,7E0,2.6E1,3.6E1,2.5E1,5E0,1.8E1,8E0,6E0,3E1,1.9E1,6E0,1.2E1,6E0,1.6E1,1.4E1,1.3E1,6E0,6E0,6E0,1.1E1,5E0,7E0,7E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-7.823011E-3,-7.058302E-3,-2.649862E-3,1.6568805E-3,-2.340346E-2,-6.998403E-4,1.6872415E-3,-2.861422E-3,-1.0663938E-2,1.6109823E-3,-2.3418528E-3,-1.9249897E-2,8.612956E-4,-1.19664855E-2,1.8933186E-3,-8.97594E-3,-1.6170235E-3,-1.1643672E-3,-7.7205817E-3,9.733169E-3,-4.3845866E-3,3.6815304E-4,-8.5485866E-4,-1.2871627E-2,2.5878652E-4,4.204696E-3,1.6395812E-3,5.065147E-4,-2.3550393E-2,-9.1222033E-4,-6.1260504E-3,8.888894E-3,-9.3746925E-4,-3.9597773E-3,9.209413E-4,-1.6332302E-3,-2.5125706E-4,2.860903E-4,-6.681534E-4,-4.561886E-4,1.2857607E-2,3.5956415E-4,-9.4372295E-3,1.761762E-2,6.0235536E-5,-1.9016474E-2,-4.9962E-5,1.163917E-4,2.2766773E-2,-2.2901906E-4,-1.2265733E-3,4.7101916E-4,1.3268323E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":124,"left_children":[1,-1,3,5,7,9,-1,-1,11,-1,13,15,-1,17,19,21,-1,-1,23,25,27,-1,-1,29,-1,31,-1,33,35,-1,37,39,-1,41,-1,-1,-1,-1,-1,-1,43,-1,45,47,-1,49,-1,-1,51,-1,-1,-1,-1],"loss_changes":[1.4684981E-1,0E0,1.612725E-2,1.3106755E-2,1.5948934E-2,8.855627E-3,0E0,0E0,6.9045885E-3,0E0,5.607046E-3,3.3245645E-3,0E0,2.4173283E-3,4.7675883E-3,2.751353E-3,0E0,0E0,2.4668598E-3,6.4582718E-3,5.0739916E-3,0E0,0E0,1.2258645E-3,0E0,4.642207E-3,0E0,4.130478E-3,2.49915E-3,0E0,1.8468702E-3,2.5906286E-3,0E0,2.5498196E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.450785E-3,0E0,2.0435762E-3,1.4019986E-3,0E0,1.3537994E-3,0E0,0E0,9.515444E-4,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,8,8,10,10,11,11,13,13,14,14,15,15,18,18,19,19,20,20,23,23,25,25,27,27,28,28,30,30,31,31,33,33,40,40,42,42,43,43,45,45,48,48],"right_children":[2,-1,4,6,8,10,-1,-1,12,-1,14,16,-1,18,20,22,-1,-1,24,26,28,-1,-1,30,-1,32,-1,34,36,-1,38,40,-1,42,-1,-1,-1,-1,-1,-1,44,-1,46,48,-1,50,-1,-1,52,-1,-1,-1,-1],"split_conditions":[2.5407658E-7,-7.058302E-3,1.8787456E7,3.38511E7,1.9840392E7,1.312094E8,1.6872415E-3,-2.861422E-3,8.553513E6,1.6109823E-3,3.6923E4,3.0367E4,8.612956E-4,8.4E1,1.44E2,7E1,-1.6170235E-3,-1.1643672E-3,7.59E2,9.203419E6,9.969132E8,3.6815304E-4,-8.5485866E-4,5.27222E5,2.5878652E-4,8.1951E4,1.6395812E-3,7.3613E4,1.2852292E7,-9.1222033E-4,1.05E4,5.7742E4,-9.3746925E-4,5.591E4,9.209413E-4,-1.6332302E-3,-2.5125706E-4,2.860903E-4,-6.681534E-4,-4.561886E-4,3.014154E-1,3.5956415E-4,3.5094325E-3,7.203963E7,6.0235536E-5,5.4007E7,-4.9962E-5,1.163917E-4,2.2375688E3,-2.2901906E-4,-1.2265733E-3,4.7101916E-4,1.3268323E-3],"split_indices":[62,0,18,2,18,10,0,0,2,0,12,7,0,4,4,19,0,0,20,18,16,0,0,21,0,8,0,8,18,0,28,8,0,8,0,0,0,0,0,0,62,0,62,16,0,16,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.84E2,5E0,1.79E2,1.49E2,3E1,1.41E2,8E0,6E0,2.4E1,5E0,1.36E2,1.9E1,5E0,4.1E1,9.5E1,1.3E1,6E0,8E0,3.3E1,4.2E1,5.3E1,5E0,8E0,2.4E1,9E0,3.6E1,6E0,4.3E1,1E1,1E1,1.4E1,3.1E1,5E0,3.6E1,7E0,5E0,5E0,6E0,8E0,5E0,2.6E1,1.1E1,2.5E1,1.8E1,8E0,1.1E1,1.4E1,5E0,1.3E1,5E0,6E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[5.9716175E-3,2.6081046E-3,4.4990727E-3,2.6151462E-2,-5.984669E-4,6.5908876E-6,3.6541577E-2,1.0811798E-3,-1.8545529E-3,2.201852E-3,5.8674894E-4,-7.5244653E-4,1.9734353E-3,8.821512E-4,-2.1241198E-3,-1.6887873E-2,3.962123E-4,-1.4840473E-3,-8.916689E-3,4.9775257E-3,-7.146308E-3,6.8583606E-5,-6.249292E-4,8.11521E-3,-1.1910109E-2,-2.8490913E-3,-1.5103305E-3,-6.448736E-4,1.5141701E-2,-1.1146668E-4,-9.096347E-4,2.1030589E-3,-8.7905646E-4,7.3904325E-3,-1.4263211E-3,9.274648E-3,1.355693E-3,5.9732224E-4,-6.5419287E-3,1.0826202E-3,6.55763E-5,9.0318645E-4,5.8518457E-5,-9.79301E-4,4.599871E-3,6.509675E-4,-2.1379461E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":125,"left_children":[1,3,-1,5,7,-1,9,11,-1,-1,-1,13,-1,-1,15,17,19,-1,21,23,25,-1,-1,27,29,31,-1,33,35,-1,-1,37,-1,39,-1,41,-1,-1,43,-1,-1,-1,-1,-1,45,-1,-1],"loss_changes":[5.8523E-2,1.3227006E-2,0E0,5.667585E-3,1.0952172E-2,0E0,4.1397214E-3,1.2172381E-2,0E0,0E0,0E0,4.21242E-3,0E0,0E0,5.104401E-3,2.4250895E-3,4.11302E-3,0E0,8.7220676E-4,4.0177694E-3,5.2109137E-3,0E0,0E0,3.8875807E-3,9.4946765E-4,3.467477E-3,0E0,7.7061364E-3,2.980791E-3,0E0,0E0,3.2936414E-3,0E0,2.4234885E-3,0E0,2.4182529E-3,0E0,0E0,3.4680774E-3,0E0,0E0,0E0,0E0,0E0,1.2486811E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,11,11,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24,25,25,27,27,28,28,31,31,33,33,35,35,38,38,44,44],"right_children":[2,4,-1,6,8,-1,10,12,-1,-1,-1,14,-1,-1,16,18,20,-1,22,24,26,-1,-1,28,30,32,-1,34,36,-1,-1,38,-1,40,-1,42,-1,-1,44,-1,-1,-1,-1,-1,46,-1,-1],"split_conditions":[1.442786E4,1E0,4.4990727E-3,6.88588E5,2.008E3,6.5908876E-6,6.13E3,7.74639E5,-1.8545529E-3,2.201852E-3,5.8674894E-4,3.2077E5,1.9734353E-3,8.821512E-4,4.524E3,3.347741E6,2.14E2,-1.4840473E-3,7.713631E6,1E0,1.3957454E9,6.8583606E-5,-6.249292E-4,1.03E2,7.952434E9,3.9281719E3,-1.5103305E-3,3.48E2,2.325882E9,-1.1146668E-4,-9.096347E-4,9.307657E6,-8.7905646E-4,1.636E3,-1.4263211E-3,1.44E2,1.355693E-3,5.9732224E-4,5.0929E4,1.0826202E-3,6.55763E-5,9.0318645E-4,5.8518457E-5,-9.79301E-4,1.71E2,6.509675E-4,-2.1379461E-4],"split_indices":[5,44,0,6,19,0,0,12,0,0,0,2,0,0,3,27,4,0,27,35,16,0,0,4,25,5,0,17,11,0,0,23,0,22,0,4,0,0,12,0,0,0,0,0,17,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.79E2,1.74E2,5E0,2E1,1.54E2,6E0,1.4E1,1.49E2,5E0,8E0,6E0,1.44E2,5E0,8E0,1.36E2,1.9E1,1.17E2,5E0,1.4E1,7.3E1,4.4E1,5E0,9E0,6.2E1,1.1E1,3.9E1,5E0,2.8E1,3.4E1,6E0,5E0,3.1E1,8E0,2.3E1,5E0,2.6E1,8E0,1.3E1,1.8E1,5E0,1.8E1,1E1,1.6E1,7E0,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.1776442E-3,-7.0506753E-3,-4.56027E-4,-3.605092E-3,3.6257582E-3,1.033459E-3,-5.3275595E-3,3.1382802E-3,-2.2438234E-3,1.3832969E-3,1.9525378E-3,2.8592448E-3,-1.0503728E-3,-1.4995978E-3,1.5605003E-2,-1.5730845E-2,5.499165E-4,2.995685E-2,4.610589E-3,7.878098E-5,-1.3539006E-3,1.73263E-2,-1.9734148E-3,1.8336958E-3,2.1265082E-4,-3.791019E-4,1.0508128E-2,2.1939512E-4,1.2565183E-3,-8.245403E-4,4.3715842E-4,9.3115965E-4,-4.0531426E-4,7.993232E-3,-4.6689655E-3,5.1849324E-4,-3.877195E-4,-2.4300267E-5,-7.936587E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":126,"left_children":[1,-1,3,5,-1,7,-1,9,-1,11,-1,13,-1,15,17,19,21,23,25,-1,-1,27,29,-1,-1,-1,31,-1,-1,-1,33,-1,-1,35,37,-1,-1,-1,-1],"loss_changes":[1.5015341E-1,0E0,4.3436315E-2,8.749043E-2,0E0,1.7160008E-2,0E0,1.0815307E-2,0E0,5.4775476E-3,0E0,7.6346756E-3,0E0,3.0128665E-3,5.3845365E-3,3.541248E-3,3.886438E-3,4.331338E-3,1.7379868E-3,0E0,0E0,1.5470367E-3,3.2707725E-3,0E0,0E0,0E0,3.4124502E-3,0E0,0E0,0E0,2.7766598E-3,0E0,0E0,2.0073992E-3,2.3796095E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,26,26,30,30,33,33,34,34],"right_children":[2,-1,4,6,-1,8,-1,10,-1,12,-1,14,-1,16,18,20,22,24,26,-1,-1,28,30,-1,-1,-1,32,-1,-1,-1,34,-1,-1,36,38,-1,-1,-1,-1],"split_conditions":[6.7E1,-7.0506753E-3,4.8801543E11,3.5713768E0,3.6257582E-3,8.83E2,-5.3275595E-3,3.5234257E11,-2.2438234E-3,1.6429902E9,1.9525378E-3,6.7842746E8,-1.0503728E-3,1.2028641E3,4.2E1,3.73595E5,1.421E3,1.852532E7,2.8646374E8,7.878098E-5,-1.3539006E-3,2.5003E4,1.97E2,1.8336958E-3,2.1265082E-4,-3.791019E-4,4.832502E3,2.1939512E-4,1.2565183E-3,-8.245403E-4,4.174E3,9.3115965E-4,-4.0531426E-4,8.233E3,1.78E2,5.1849324E-4,-3.877195E-4,-2.4300267E-5,-7.936587E-4],"split_indices":[22,0,13,59,0,17,0,10,0,16,0,16,0,5,19,2,28,18,27,0,0,21,20,0,0,0,5,0,0,0,20,0,0,3,17,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.68E2,5E0,1.63E2,1.58E2,5E0,1.53E2,5E0,1.48E2,5E0,1.43E2,5E0,1.36E2,7E0,1.02E2,3.4E1,1.2E1,9E1,1.4E1,2E1,6E0,6E0,1.1E1,7.9E1,9E0,5E0,6E0,1.4E1,6E0,5E0,9E0,7E1,9E0,5E0,2.8E1,4.2E1,2.3E1,5E0,3.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.9735916E-4,-6.665985E-3,5.593465E-3,2.9697993E-3,2.5468622E-3,2.8497363E-3,5.843884E-4,2.1917932E-3,-1.7677803E-3,6.625857E-5,1.7768148E-3,1.9349106E-2,-2.125476E-3,1.3293683E-3,-1.0429174E-4,-6.1740745E-3,7.3406524E-3,5.786607E-4,-7.6665897E-3,1.1994136E-3,-2.884316E-4,-1.131074E-3,-5.490705E-3,9.857356E-4,-5.766352E-3,-2.6155068E-3,-1.758385E-2,-1.0859828E-2,5.313123E-4,-5.3540384E-3,9.925845E-3,-1.5802172E-3,1.4119044E-4,-9.980664E-4,-2.635538E-3,-6.793477E-5,-5.614182E-4,7.6341536E-4,-2.4186154E-5,3.3733327E-4,-5.509662E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":127,"left_children":[1,-1,3,5,-1,-1,7,9,-1,11,-1,13,15,-1,-1,17,19,-1,21,-1,23,-1,25,-1,27,29,31,33,-1,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4526188E-1,0E0,2.3358002E-2,2.4193566E-2,0E0,0E0,1.05234375E-2,1.2523304E-2,0E0,6.2115067E-3,0E0,3.694924E-3,5.1041627E-3,0E0,0E0,2.7988572E-3,6.1536897E-3,0E0,3.3120178E-3,0E0,3.8890808E-3,0E0,2.7008543E-3,0E0,2.4165334E-3,2.285561E-3,5.9319925E-3,1.9147138E-3,0E0,1.5197786E-3,9.610468E-4,0E0,0E0,0E0,1.4432549E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,9,9,11,11,12,12,15,15,16,16,18,18,20,20,22,22,24,24,25,25,26,26,27,27,29,29,30,30,34,34],"right_children":[2,-1,4,6,-1,-1,8,10,-1,12,-1,14,16,-1,-1,18,20,-1,22,-1,24,-1,26,-1,28,30,32,34,-1,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5407658E-7,-6.665985E-3,1.11464E5,2.5E1,2.5468622E-3,2.8497363E-3,1.2458633E11,7.12E2,-1.7677803E-3,1.7726E4,1.7768148E-3,1.1934E4,2.8646374E8,1.3293683E-3,-1.0429174E-4,6.989E3,6.1E1,5.786607E-4,7.484321E8,1.1994136E-3,1.00770764E-1,-1.131074E-3,2.8800161E9,9.857356E-4,1.83E2,1.1768006E9,5.5015055E9,2.93283E-1,5.313123E-4,5.503231E8,4.174E3,-1.5802172E-3,1.4119044E-4,-9.980664E-4,9.381936E-1,-6.793477E-5,-5.614182E-4,7.6341536E-4,-2.4186154E-5,3.3733327E-4,-5.509662E-4],"split_indices":[62,0,3,20,0,0,23,17,0,18,0,20,27,0,0,12,19,0,10,0,62,0,11,0,4,11,11,62,0,11,20,0,0,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.74E2,5E0,1.69E2,1.62E2,7E0,5E0,1.57E2,1.52E2,5E0,1.45E2,7E0,1.4E1,1.31E2,9E0,5E0,9.2E1,3.9E1,6E0,8.6E1,1E1,2.9E1,8E0,7.8E1,5E0,2.4E1,6.4E1,1.4E1,1.9E1,5E0,5.3E1,1.1E1,7E0,7E0,7E0,1.2E1,3.6E1,1.7E1,6E0,5E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.0413287E-3,-6.9527584E-3,3.8317547E-3,-1.4220175E-3,5.2209524E-3,1.8447074E-3,-3.773833E-3,-1.7986648E-3,-1.3465636E-3,-4.0905757E-4,-1.0694521E-3,1.1114298E-3,-2.0616676E-3,-2.3690833E-2,4.4488837E-4,-1.4498082E-3,-3.0732478E-4,2.3248526E-3,-1.0702333E-3,5.778241E-4,1.0324122E-3,7.4495224E-4,-1.2608039E-3,7.5936556E-4,-3.0440444E-3,-4.333146E-5,-8.641982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":128,"left_children":[1,-1,3,5,-1,-1,7,9,-1,11,-1,-1,13,15,17,-1,-1,19,-1,21,-1,-1,23,-1,25,-1,-1],"loss_changes":[1.539225E-1,0E0,9.810864E-2,1.6084539E-2,0E0,0E0,7.859922E-3,4.375316E-3,0E0,5.6965877E-3,0E0,0E0,6.7873793E-3,1.8279571E-3,5.3604986E-3,0E0,0E0,3.9361194E-3,0E0,3.0230735E-3,0E0,0E0,3.0475294E-3,0E0,2.8330958E-3,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,9,9,12,12,13,13,14,14,17,17,19,19,22,22,24,24],"right_children":[2,-1,4,6,-1,-1,8,10,-1,12,-1,-1,14,16,18,-1,-1,20,-1,22,-1,-1,24,-1,26,-1,-1],"split_conditions":[8.419447E-7,-6.9527584E-3,1.11464E5,1.689E3,5.2209524E-3,1.8447074E-3,7.79E2,1E0,-1.3465636E-3,4.286E3,-1.0694521E-3,1.1114298E-3,1.97E2,2.9435686E3,1.845821E9,-1.4498082E-3,-3.0732478E-4,1.5522031E9,-1.0702333E-3,2.9E1,1.0324122E-3,7.4495224E-4,1E0,7.5936556E-4,1.082E3,-4.333146E-5,-8.641982E-4],"split_indices":[62,0,3,27,0,0,17,32,0,28,0,0,20,5,16,0,0,16,0,4,0,0,44,0,19,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.65E2,5E0,1.6E2,1.54E2,6E0,7E0,1.47E2,1.38E2,9E0,1.31E2,7E0,7E0,1.24E2,1.2E1,1.12E2,7E0,5E0,1.05E2,7E0,9.8E1,7E0,9E0,8.9E1,7E0,8.2E1,7.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.2591593E-3,-6.3054953E-3,3.4174717E-3,6.3655083E-4,3.7454308E-3,4.752195E-3,-5.1862677E-3,2.040846E-3,3.506149E-3,3.199604E-3,-5.78879E-4,9.748035E-4,-1.6077112E-2,-2.3846794E-3,7.948119E-3,-1.6930689E-3,-3.6733298E-5,-1.869051E-4,-1.2301956E-3,1.1809967E-2,-5.813815E-4,-3.3009262E-3,8.664391E-3,1.5075627E-2,-1.7255964E-4,-6.4757944E-4,-1.1665778E-3,1.781168E-2,-1.0156917E-3,1.4175709E-3,1.1092602E-2,3.686109E-3,-5.6795166E-3,3.412528E-3,1.5290158E-3,4.2848568E-4,1.7244013E-2,-1.7280075E-4,5.060954E-4,-1.2523346E-4,-7.5854757E-4,-4.3173807E-4,8.0158707E-4,3.949712E-4,-3.0648604E-4,1.1066133E-3,1.2761916E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":129,"left_children":[1,-1,3,5,-1,7,-1,9,-1,-1,11,13,15,17,19,-1,-1,21,-1,23,-1,25,27,29,-1,-1,31,33,-1,-1,35,37,39,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2707181E-1,0E0,4.2105805E-2,8.857071E-2,0E0,3.5491664E-2,0E0,3.1594142E-2,0E0,0E0,3.964683E-3,3.5329636E-3,5.1580165E-3,5.7785036E-3,4.1819084E-3,0E0,0E0,2.647097E-3,0E0,2.2216872E-3,0E0,1.7308192E-3,7.7174613E-3,2.0657452E-3,0E0,0E0,1.3568279E-3,4.839462E-3,0E0,0E0,1.9605919E-3,1.8598993E-3,1.0022722E-3,2.5688591E-3,0E0,8.342821E-4,2.132384E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,10,10,11,11,12,12,13,13,14,14,17,17,19,19,21,21,22,22,23,23,26,26,27,27,30,30,31,31,32,32,33,33,35,35,36,36],"right_children":[2,-1,4,6,-1,8,-1,10,-1,-1,12,14,16,18,20,-1,-1,22,-1,24,-1,26,28,30,-1,-1,32,34,-1,-1,36,38,40,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5338E4,-6.3054953E-3,7.9193126E11,4.312765E7,3.7454308E-3,7.1848E4,-5.1862677E-3,2.6E1,3.506149E-3,3.199604E-3,2.1035208E7,3.601764E6,2.0538752E3,1.259E3,4.81374E5,-1.6930689E-3,-3.6733298E-5,3.2037274E8,-1.2301956E-3,1E0,-5.813815E-4,4.0221E4,4.2644E4,1.882E4,-1.7255964E-4,-6.4757944E-4,1.148E3,4.02722E5,-1.0156917E-3,1.4175709E-3,8.945914E8,6.922972E-4,4.4456246E-1,5E0,1.5290158E-3,3.565157E7,3.238106E-1,-1.7280075E-4,5.060954E-4,-1.2523346E-4,-7.5854757E-4,-4.3173807E-4,8.0158707E-4,3.949712E-4,-3.0648604E-4,1.1066133E-3,1.2761916E-4],"split_indices":[25,0,13,6,0,3,0,20,0,0,18,2,5,0,12,0,0,16,0,35,0,18,22,7,0,0,20,6,0,0,23,62,62,19,0,16,62,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.88E2,5E0,1.83E2,1.78E2,5E0,1.73E2,5E0,1.68E2,5E0,5E0,1.63E2,1.49E2,1.4E1,1.01E2,4.8E1,5E0,9E0,9.4E1,7E0,4.1E1,7E0,7E1,2.4E1,3.4E1,7E0,1E1,6E1,1.9E1,5E0,5E0,2.9E1,2.9E1,3.1E1,1.1E1,8E0,1.1E1,1.8E1,1.5E1,1.4E1,2.6E1,5E0,6E0,5E0,5E0,6E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"82","num_nodes":"47","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"1.7869648E-1","boost_from_average":"1","num_class":"0","num_feature":"82","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,1]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg index 94a8c7745..7bb45d686 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"61"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0436389036","gamma":"0.0876344517","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0436389036","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.0876344517","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.779665112"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"61"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0436389036","gamma":"0.0876344517","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0436389036","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.0876344517","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.779665112"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json index 0f46af9d6..aaca0db8a 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Scan parquet ","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_emr","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"61"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.78048E-2,-1.8552697E-1,4.901243E-1,-3.3088633E-1,3.0965138E-2,2.4628823E-1,8.852223E-1,-4.659008E-1,-1.3334489E-1,-1.5243791E-1,1.4812984E-1,2.971024E-1,-1.2563897E-2,5.130134E-2,6.705256E-1,-1.8754591E-1,-5.3297985E-1,9.115728E-2,-2.2618434E-1,-2.66628E-1,1.2518686E-2,2.2718608E-1,4.980866E-2,4.346211E-1,1.5858072E-1,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,-6.1508644E-1,-3.1681323E-1,7.0068543E-3,-7.234637E-4,-2.9873994E-1,1.068607E-2,-7.976921E-3,-2.4829587E-2,-7.415798E-2,4.8681386E-3,1.2035196E-2,4.350847E-2,-3.566741E-2,9.323246E-3,2.2088567E-2,1.0012548E-2,2.3240472E-1,2.6123306E-2,-1.783583E-2,-6.754464E-1,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4057398E1,5.265595E0,7.200094E0,2.60497E0,1.4837892E0,6.482475E-1,1.0196114E0,1.0016766E0,8.960098E-1,5.119209E-1,3.1514597E-1,7.120998E-1,2.5112772E-1,0E0,1.8040752E-1,5.0723886E-1,6.393223E-1,9.9871E-2,5.187807E-1,3.0155098E-1,1.1128237E-1,1.9333434E-1,2.8770238E-1,1.774602E-1,2.0905662E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3500404E-1,1.0710728E-1,0E0,0E0,1.3412046E-1,3.246751E-1,0E0,0E0,1.1355559E-1,0E0,0E0,9.497507E-2,1.2899351E-1,0E0,0E0,0E0,9.763676E-2,1.0520158E-1,0E0,3.8818073E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,35,35,36,36,39,39,42,42,43,43,47,47,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,2.7558826E-2,2.0224316E-1,1.9415865E2,2.4118989E8,2.05E0,7.1239825E5,1.433458E10,8.938651E4,7.919613E5,3.2744168E6,7.3E1,6.035389E6,5.130134E-2,2.5E1,2.7287993E0,1.021924E6,1.1936242E7,4.145324E6,2.7575057E0,6.406965E-1,1.2183861E7,2.1469998E5,6.75602E5,2.8994352E7,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,3.12986E10,1.858E3,7.0068543E-3,-7.234637E-4,3.1100525E5,4.7E3,-7.976921E-3,-2.4829587E-2,4.565566E-1,4.8681386E-3,1.2035196E-2,6.7E2,1.526402E6,9.323246E-3,2.2088567E-2,1.0012548E-2,1E0,1.479938E6,-1.783583E-2,7.359468E8,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"split_indices":[54,40,40,54,7,55,35,33,35,34,52,8,1,0,3,56,31,47,1,60,41,34,30,9,1,0,0,0,0,0,0,33,0,0,0,35,0,0,0,41,0,0,0,9,0,0,0,102,34,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,1.66E2,7.6E1,9.9E1,6.7E1,4.8E1,2.8E1,5.8E1,4.1E1,2.6E1,4.1E1,4E1,8E0,1E1,1.8E1,1.2E1,4.6E1,1.2E1,2.9E1,1.5E1,1.1E1,2.2E1,1.9E1,1.9E1,2.1E1,4E0,4E0,1.6E1,2E0,2E0,1E1,3.2E1,1.4E1,7E0,5E0,2.2E1,7E0,1.3E1,2E0,6E0,5E0,1.7E1,5E0,1.3E1,6E0,1.3E1,6E0,1.3E1,8E0,9E0,2.3E1,2E0,1.2E1,6E0,1.6E1,3E0,4E0,3E0,3E0,2E0,3E0,7E0,6E0,7E0,6E0,4E0,4E0,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.0040125E-2,-2.1713006E-1,4.0957847E-1,-3.5627815E-1,9.978797E-3,2.0629005E-1,8.082211E-1,-4.7081342E-1,-1.7093469E-1,-5.8718614E-2,1.0682804E-2,6.47079E-2,2.835063E-1,2.307691E-2,4.1448183E-2,9.978426E-4,-4.971977E-1,1.4917599E-3,-3.1879982E-1,-1.7213058E-1,8.313783E-2,1.541623E-1,-7.4060783E-3,3.6715305E-1,1.0359522E-2,-1.8699963E-1,-5.3688544E-1,-9.420938E-2,1.2580958E-2,-1.8095352E-2,-1.4491154E-1,-2.1190276E-2,-1.1271654E-1,-1.211096E-2,2.5133708E-1,9.38944E-3,-7.0502064E-3,1.9026043E-5,3.9769462E-1,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,-3.106009E-1,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,-1.4537086E-1,7.170774E-3,-6.025724E-2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,4.859038E-1,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,5.357659E-1,3.981885E-3,7.6941703E-3,2.5353398E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,-1,-1,55,-1,-1,-1,-1,-1,57,-1,-1,-1,-1,59,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[2.0510149E1,4.996633E0,6.1543484E0,1.9999018E0,9.984025E-1,5.610745E-1,5.2623177E-1,7.9289055E-1,9.9481404E-1,7.8654516E-1,0E0,4.4556248E-1,7.730229E-1,0E0,0E0,0E0,6.0133266E-1,5.4866415E-1,2.8996253E-1,4.4809937E-1,3.5550103E-1,3.2364437E-1,0E0,2.9102945E-1,1.4653622E-1,9.1507256E-2,6.157198E-1,1.915977E-1,0E0,0E0,1.275117E-1,0E0,2.4100256E-1,1.6567487E-1,2.9098767E-1,0E0,0E0,0E0,3.2995868E-1,0E0,0E0,0E0,0E0,0E0,1.7156088E-1,0E0,0E0,0E0,0E0,1.24156505E-1,0E0,9.488517E-2,0E0,0E0,0E0,0E0,2.6596856E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2510395E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,30,30,32,32,33,33,34,34,38,38,44,44,49,49,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,-1,-1,56,-1,-1,-1,-1,-1,58,-1,-1,-1,-1,60,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.5665247E2,1.7613442E-1,2.0224316E-1,1.9415865E2,9.749826E6,4.633287E2,3.856E3,3.8757736E-7,1.5248811E1,1.5225257E-1,1.0682804E-2,3.6217573E0,2.1030048E10,2.307691E-2,4.1448183E-2,9.978426E-4,1.1266387E10,3.165913E6,3.03E9,1.386452E0,1.7573398E2,1.9647126E7,-7.4060783E-3,2.34E3,2.51838E6,2.85E2,1.021924E6,5.63651E5,1.2580958E-2,-1.8095352E-2,2.3407683E0,-2.1190276E-2,2.7538432E8,2.424E3,1.2730925E0,9.38944E-3,-7.0502064E-3,1.9026043E-5,5.792986E4,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,1.892E3,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,2.5187686E-1,7.170774E-3,3.56E2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,1.1107071E7,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,3.302E3,3.981885E-3,7.6941703E-3,2.5353398E-2],"split_indices":[54,43,40,54,9,54,2,40,58,40,0,56,5,0,0,0,33,1,5,55,54,47,0,2,34,0,31,9,0,0,56,0,7,2,44,0,0,0,30,0,0,0,0,0,2,0,0,0,0,44,0,10,0,0,0,0,9,0,0,0,0,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,1.57E2,7.7E1,9.7E1,6E1,5.2E1,2.5E1,5.9E1,3.8E1,4.7E1,1.3E1,1.9E1,3.3E1,1E1,1.5E1,3E0,5.6E1,1.8E1,2E1,2.6E1,2.1E1,1.4E1,5E0,2.5E1,8E0,7E0,4.9E1,1.4E1,4E0,1.2E1,8E0,3E0,2.3E1,1.4E1,7E0,1.2E1,2E0,2E0,2.3E1,5E0,3E0,5E0,2E0,3.6E1,1.3E1,7E0,7E0,6E0,2E0,2.1E1,2E0,1.2E1,2E0,2E0,5E0,8E0,1.5E1,6E0,7E0,8E0,1.3E1,7E0,5E0,1.3E1,2E0,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.5844442E-2,-2.1939752E-1,3.5634845E-1,-3.067927E-1,7.70621E-2,1.8756232E-1,7.0530635E-1,-3.6990502E-1,-1.6308445E-2,3.355599E-2,1.0062139E-2,-4.2225484E-2,2.3436439E-1,1.2260787E-2,7.853399E-1,-4.3145156E-1,-2.022222E-1,1.7129338E-1,-1.4914486E-1,-7.0826644E-3,8.609564E-3,7.4184006E-3,-1.4439643E-2,2.7769628E-1,6.108423E-3,1.9752327E-2,3.9388932E-2,-3.5376546E-1,-6.491721E-1,-1.04918115E-1,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,4.5147832E-2,-5.3988285E-3,2.0416094E-1,4.445212E-1,-5.368166E-3,3.978625E-3,-5.482469E-1,-2.6226673E-1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,-3.322308E-2,-8.5412245E-4,7.761204E-3,2.3548925E-1,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,-2.9240888E-1,-3.515024E-3,6.6058734E-3,2.612413E-1,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,49,51,-1,-1,53,55,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,-1,-1,61,-1,-1,63,-1,-1,-1,-1,-1],"loss_changes":[1.863963E1,4.1431956E0,4.991946E0,2.2489424E0,2.3938012E-1,6.55169E-1,7.3028564E-1,9.5510197E-1,5.9871566E-1,2.0101649E-1,0E0,7.301139E-1,4.9284196E-1,0E0,4.888401E-1,1.0050564E0,4.5257246E-1,2.7720916E-1,2.4958798E-1,1.5788087E-1,0E0,0E0,0E0,4.2448258E-1,1.1006676E-1,0E0,0E0,8.527217E-1,3.161502E-1,1.413299E-1,0E0,0E0,0E0,0E0,0E0,1.5807515E-1,0E0,3.994676E-1,2.7063847E-1,0E0,0E0,1.5897751E-1,3.2701302E-1,0E0,0E0,0E0,1.2576254E-1,0E0,0E0,1.6656256E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.2975593E-1,0E0,0E0,1.3126945E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,27,27,28,28,29,29,35,35,37,37,38,38,41,41,42,42,46,46,49,49,56,56,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,50,52,-1,-1,54,56,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,-1,-1,62,-1,-1,64,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,6.005889E-1,2.0224316E-1,2.727992E6,9.750871E6,7.12E2,2.744E3,1.1670399E-1,1.6E1,1E0,1.0062139E-2,3.9378E5,2.8812434E7,1.2260787E-2,2.2324745E5,3.98231E-2,3.4047124E-1,6.0578914E4,2.6017979E11,4.33E2,8.609564E-3,7.4184006E-3,-1.4439643E-2,7.2259143E2,1.04E2,1.9752327E-2,3.9388932E-2,3.3617156E-4,1.868632E7,1.86204E5,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,7.948E4,-5.3988285E-3,5.130435E0,9.521E3,-5.368166E-3,3.978625E-3,5.8013948E7,9.3E1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,2.3544362E6,-8.5412245E-4,7.761204E-3,2.1061536E3,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,1.65E9,-3.515024E-3,6.6058734E-3,7.83495E5,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"split_indices":[54,41,40,1,9,0,2,41,3,101,0,1,47,0,30,44,44,34,33,0,0,0,0,54,8,0,0,41,47,9,0,0,0,0,0,1,0,56,2,0,0,7,11,0,0,0,53,0,0,4,0,0,0,0,0,0,5,0,0,35,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.58E2,8.6E1,1.22E2,3.6E1,5.9E1,2.7E1,1E2,2.2E1,2.9E1,7E0,1E1,4.9E1,5E0,2.2E1,7.2E1,2.8E1,9E0,1.3E1,2.4E1,5E0,6E0,4E0,4.1E1,8E0,7E0,1.5E1,5.5E1,1.7E1,1.9E1,9E0,2E0,7E0,2E0,1.1E1,1.7E1,7E0,3E1,1.1E1,3E0,5E0,1.6E1,3.9E1,1.5E1,2E0,7E0,1.2E1,1.2E1,5E0,2.8E1,2E0,6E0,5E0,1.3E1,3E0,4E0,3.5E1,1E1,2E0,2.5E1,3E0,2.5E1,1E1,1.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[1.0417009E-2,-1.92214E-1,4.0404752E-1,-3.015024E-1,-7.670976E-5,1.7257118E-1,7.666108E-1,-3.8266033E-1,-1.3505321E-2,-5.040442E-2,1.1856716E-2,2.0422818E-1,-5.236619E-3,4.0671965E-1,9.444278E-1,-4.4783047E-1,-2.4137214E-1,-8.257532E-2,9.555998E-3,-1.7077446E-2,-2.2676414E-2,-1.2199888E-3,2.4267583E-1,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,-1.7631327E-1,-5.2404433E-1,-4.6340473E-2,-1.6422002E-2,8.984605E-3,-1.4559102E-1,-8.014499E-2,7.040712E-2,3.7856903E-2,1.1994875E-2,8.994386E-3,-1.1225067E-2,-5.8429706E-1,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,-1.4150277E-1,4.9856484E-2,1.1765836E-2,1.939237E-2,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,-1,35,-1,-1,-1,-1,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0656195E1,3.5906997E0,7.253089E0,2.5425148E0,8.754045E-1,5.339109E-1,1.7917042E0,6.7456627E-1,4.1952434E-1,5.01606E-1,0E0,4.531319E-1,0E0,9.731603E-2,3.5835075E-1,1.0871277E0,7.457906E-1,3.9352587E-1,0E0,0E0,2.7846104E-1,0E0,2.731049E-1,0E0,0E0,0E0,0E0,4.8621753E-1,5.456524E-1,1.2857142E-1,0E0,0E0,3.2480013E-1,2.623061E-1,1.9802536E-1,1.5393375E-1,0E0,0E0,0E0,2.466135E-1,0E0,0E0,0E0,0E0,0E0,1.9748119E-1,1.3966252E-1,0E0,1.250949E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,20,20,22,22,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,45,45,46,46,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,-1,36,-1,-1,-1,-1,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0845802E6,1.7613442E-1,1.5972575E0,2.727992E6,2.03693E11,1.25440235E1,7.2610445E8,1.9415865E2,1.22E2,2.9927E4,1.1856716E-2,4.0123675E-2,-5.236619E-3,1.3589149E6,2.2324745E5,2.431768E2,1.2247422E1,2.6017979E11,9.555998E-3,-1.7077446E-2,1.1343225E6,-1.2199888E-3,3.4021906E2,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,1.0395554E5,6.6037946E2,6.0515753E2,-1.6422002E-2,8.984605E-3,2.3407683E0,9.74979E6,1.2355705E6,1.19881555E-1,1.1994875E-2,8.994386E-3,-1.1225067E-2,1.51E3,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,4.4E1,5.3164E4,1.1765836E-2,6.0039372E0,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"split_indices":[53,43,43,1,33,59,12,54,8,9,0,40,0,49,30,57,58,33,0,0,35,0,54,0,0,0,0,30,4,4,0,0,56,9,35,40,0,0,0,2,0,0,0,0,0,10,31,0,56,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.7E2,8.7E1,1.08E2,6.2E1,5.4E1,3.3E1,8.4E1,2.4E1,5.3E1,9E0,4.9E1,5E0,1.2E1,2.1E1,5.6E1,2.8E1,1.9E1,5E0,3E0,5E1,7E0,4.2E1,1E1,2E0,4E0,1.7E1,1.3E1,4.3E1,1.2E1,1.6E1,3E0,1.6E1,3.1E1,1.9E1,6E0,3.6E1,2E0,1.1E1,3.4E1,9E0,3E0,9E0,1.4E1,2E0,2.1E1,1E1,3E0,1.6E1,2E0,4E0,2.2E1,1.2E1,3E0,1.8E1,4E0,6E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-2.2585599E-2,-1.9692676E-1,3.6539125E-1,-2.9716375E-1,-2.7832244E-2,1.6051705E-1,7.1087146E-1,-3.7421748E-1,1.4051265E-2,-6.813547E-2,1.1468622E-2,-8.938615E-2,2.114588E-1,4.520597E-1,9.747266E-1,-4.3956405E-1,-2.1756326E-1,-5.356781E-2,1.1113507E-2,-1.736248E-1,1.3845394E-2,4.104848E-3,-6.774424E-3,2.3279007E-1,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,-2.5017148E-1,-4.929268E-1,1.6659833E-3,-1.6076026E-2,9.1863185E-2,-6.740727E-3,-7.370915E-2,-3.767319E-1,-5.412376E-3,5.573497E-2,9.279637E-2,2.8638762E-1,-2.9118592E-1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,3.14243E-1,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,-1,47,-1,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1,-1],"loss_changes":[1.6576199E1,2.8561993E0,5.265172E0,2.5471687E0,7.675839E-1,6.4269626E-1,1.4714527E0,7.744961E-1,3.715908E-1,4.9534598E-1,0E0,1.2332737E-1,2.3372102E-1,1.488092E-1,2.703581E-1,4.729681E-1,8.93371E-1,2.783168E-1,0E0,4.7957075E-1,1.9750053E-1,0E0,0E0,2.585504E-1,0E0,0E0,0E0,0E0,0E0,1.6383308E-1,2.5936127E-1,1.265059E-1,0E0,2.4251655E-1,0E0,1.5377875E-1,1.1529529E-1,0E0,2.033091E-1,1.5228397E-1,2.5932693E-1,1.0870254E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3727503E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,29,29,30,30,31,31,33,33,35,35,36,36,38,38,39,39,40,40,41,41,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,-1,48,-1,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1,-1],"split_conditions":[4.083633E6,1.7613442E-1,1.5972575E0,2.246309E6,2.03693E11,3.359399E-1,1.6525185E9,1.9415865E2,1.22E2,7.251646E-2,1.1468622E-2,2.2183135E-1,1.25440235E1,1.6790323E8,4.6E1,2.4747229E2,1.2288136E1,4.7E3,1.1113507E-2,3.4047124E-1,2.0358758E8,4.104848E-3,-6.774424E-3,1.0705044E3,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,1.499E3,1.021924E6,2.797224E2,-1.6076026E-2,1.445113E6,-6.740727E-3,9.593298E6,6.9268906E5,-5.412376E-3,2.7352352E6,1.8E1,1.1107071E7,2.923913E1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,6.464779E6,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"split_indices":[53,43,43,1,33,44,5,54,8,40,0,41,59,7,8,57,58,0,0,44,7,0,0,57,0,0,0,0,0,2,31,54,0,9,0,47,35,0,50,3,9,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,31,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.68E2,7.5E1,1.05E2,6.3E1,4.8E1,2.7E1,8.4E1,2.1E1,5.6E1,7E0,8E0,4E1,1.5E1,1.2E1,5.8E1,2.6E1,1.7E1,4E0,2.4E1,3.2E1,2E0,6E0,3.7E1,3E0,2E0,1.3E1,2E0,1E1,1.4E1,4.4E1,1.1E1,1.5E1,7E0,1E1,1.7E1,7E0,7E0,2.5E1,1.1E1,2.6E1,1.2E1,2E0,3.8E1,6E0,7E0,4E0,4E0,3E0,1.3E1,4E0,4E0,3E0,1.4E1,1.1E1,8E0,3E0,2.4E1,2E0,3E0,9E0,1.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-8.34782E-3,-1.667577E-1,4.3978554E-1,-3.0709088E-1,2.6771413E-2,1.9930072E-1,7.418625E-1,-4.0431347E-1,-1.2503225E-1,-6.46251E-2,1.2227376E-1,2.7885714E-1,3.0976776E-2,3.739352E-1,8.703366E-1,-4.299911E-1,4.975969E-3,1.0842858E-1,-2.3695372E-1,-1.4191392E-1,1.8143274E-1,1.5553638E-1,-3.5369173E-3,2.103493E-2,1.11278795E-1,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,-4.528867E-1,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,-2.1131526E-1,5.121328E-2,8.2729507E-4,1.0838011E-2,7.531465E-2,2.1820368E-1,6.924229E-3,-5.724572E-3,-4.900431E-1,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,-4.1408205E-1,-6.208327E-1,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,45,-1,-1,-1,-1,47,49,-1,-1,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.7394594E1,4.9239187E0,4.501422E0,1.807971E0,6.800027E-1,4.868381E-1,9.980869E-1,9.540901E-1,1.0186052E0,7.864486E-1,2.696029E-1,7.975136E-1,3.8219005E-1,1.8597853E-1,5.8754253E-1,3.8879967E-1,0E0,1.0245718E-1,1.4958644E-1,4.2632413E-1,1.0402632E-1,1.4946616E-1,0E0,0E0,1.9317481E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9329586E-1,0E0,0E0,0E0,0E0,2.1796727E-1,1.8252599E-1,0E0,0E0,1.0288309E-1,1.249153E-1,0E0,0E0,2.2973442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3063469E-1,2.7027082E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,24,24,32,32,37,37,38,38,41,41,42,42,45,45,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,46,-1,-1,-1,-1,48,50,-1,-1,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.4210773E-2,2.0224316E-1,1.773747E2,6.75553E5,4.3147E5,2.174459E5,4.108148E6,9.165388E4,1.5225257E-1,2.4219654E0,3.5630266E8,1.4E1,5.76E3,1.6507233E3,6E0,4.975969E-3,1.1936242E7,2.1055435E1,2.5055168E9,6.765595E7,2.151E3,-3.5369173E-3,2.103493E-2,8.96347E7,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,1.437883E6,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,2.5335E4,4.933E4,8.2729507E-4,1.0838011E-2,5.546848E1,9.173913E5,6.924229E-3,-5.724572E-3,1.499E3,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,1.53091E5,2.373E3,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"split_indices":[54,40,40,54,9,9,30,9,35,40,55,7,3,2,4,10,0,47,58,5,5,2,0,0,46,0,0,0,0,0,0,0,1,0,0,0,0,9,9,0,0,54,49,0,0,2,0,0,0,0,0,0,0,0,0,9,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.8E2,6.3E1,1.04E2,7.6E1,3.6E1,2.7E1,6.7E1,3.7E1,3.9E1,3.7E1,2.4E1,1.2E1,8E0,1.9E1,6.4E1,3E0,1.2E1,2.5E1,3E1,9E0,3.2E1,5E0,1E1,1.4E1,4E0,8E0,3E0,5E0,1E1,9E0,5E0,5.9E1,7E0,5E0,1.4E1,1.1E1,2.2E1,8E0,3E0,6E0,1.5E1,1.7E1,1.2E1,2E0,4.8E1,1.1E1,2E0,2E1,4E0,4E0,2E0,1.3E1,1.5E1,2E0,3.3E1,1.5E1,2.4E1,9E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.9497597E-3,-1.8956819E-1,3.4107372E-1,-3.159441E-1,-1.5801242E-2,1.5268305E-1,6.5795535E-1,-3.6932546E-1,7.846604E-3,-5.214277E-2,1.0556764E-1,-1.2807088E-2,2.3161104E-1,4.560374E-1,8.566107E-1,-2.9471084E-1,-5.894394E-1,1.2438455E-1,-7.3430347E-3,-1.3571183E-1,1.1542113E-2,7.1960697E-3,-4.9598026E-3,1.3622053E-1,-1.1057054E-2,6.492877E-2,3.2902324E-1,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,-2.0076081E-1,-6.4882565E-1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,9.8576896E-2,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,2.6239768E-1,2.3697935E-2,-2.679108E-1,-6.2030084E-2,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,4.4976994E-3,-3.273256E-1,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6410194E1,3.4866414E0,5.391714E0,1.5930605E0,3.054343E-1,7.736126E-1,9.7316456E-1,1.1237602E0,3.082666E-1,2.8363693E-1,2.2685544E-1,7.515435E-1,6.2811255E-1,2.3523116E-1,7.9534054E-1,9.5042276E-1,3.015952E-1,2.1883255E-1,0E0,9.567392E-2,1.899715E-1,0E0,0E0,2.035301E-1,0E0,1.436969E-1,2.1399069E-1,0E0,0E0,0E0,0E0,0E0,3.7464368E-1,1.0184717E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1995159E-1,0E0,0E0,0E0,0E0,1.2598455E-1,0E0,4.546547E-1,1.02785856E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.053444E-1,1.2376809E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,26,26,32,32,33,33,40,40,45,45,47,47,48,48,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.1306552E-1,1.8829846E-1,3.402546E6,2.424E3,3.165913E6,1.4968063E3,5.1653096E-3,3.771E3,5.670282E-1,3.2410275E2,1.1936242E7,6.2143734E4,2.0270285E8,2.65826E6,3.3617156E-4,1.8018788E7,3.995065E6,-7.3430347E-3,2.8507206E-1,2.6801108E2,7.1960697E-3,-4.9598026E-3,1.9816086E0,-1.1057054E-2,2.51838E6,8.0267444E2,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,8.456733E10,7.5E1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,7E0,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,8.0774775E6,2.3697935E-2,2.3498028E6,3.948533E5,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,7.1E1,6E0,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"split_indices":[54,41,40,1,2,1,4,40,2,44,54,47,30,7,49,41,47,31,0,44,4,0,0,55,0,34,54,0,0,0,0,0,33,8,0,0,0,0,0,0,8,0,0,0,0,34,0,47,30,0,0,0,0,0,0,0,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.58E2,9.1E1,9.1E1,6.7E1,5.8E1,3.3E1,7.8E1,1.3E1,5.2E1,1.5E1,1.9E1,3.9E1,1.8E1,1.5E1,6E1,1.8E1,8E0,5E0,2.2E1,3E1,1.2E1,3E0,1.2E1,7E0,1.5E1,2.4E1,3E0,1.5E1,9E0,6E0,1.9E1,4.1E1,1.5E1,3E0,3E0,5E0,1.4E1,8E0,1.7E1,1.3E1,1E1,2E0,6E0,9E0,2E1,4E0,2.7E1,1.4E1,1.2E1,3E0,9E0,4E0,1.5E1,5E0,5E0,2.2E1,1E1,4E0,3E0,2E0,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.0813884E-3,-2.0281385E-1,3.2203957E-1,-2.7021798E-1,6.471507E-2,1.3456307E-1,6.046468E-1,-3.230015E-1,-2.4385922E-2,-4.1049863E-3,1.0497509E-1,-1.7012404E-1,1.8569289E-1,7.2859275E-1,3.886841E-1,-2.9304287E-1,-5.693168E-1,7.866887E-3,-7.695989E-2,1.368179E-1,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.1308988E-1,1.2391388E-1,4.675196E-2,5.738674E-1,9.75509E-3,2.0310232E-2,-3.7767187E-1,-1.9021347E-1,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,9.295686E-2,2.2762196E-2,-4.9231676E-3,1.4885345E-1,8.6237835E-3,2.6556352E-2,-4.5262882E-1,-2.0554197E-1,-2.254751E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.7958388E-2,1.8340372E-1,-3.690774E-1,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,-2.566865E-1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,-2.7930382E-1,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,-1,-1,51,53,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[1.6330715E1,2.812533E0,5.0049305E0,1.6011333E0,2.1443705E-1,9.507282E-1,7.339792E-1,5.858507E-1,2.613468E-1,0E0,1.8759432E-1,2.9529205E-1,3.6259735E-1,6.0491276E-1,9.870744E-2,7.3748016E-1,1.4939356E-1,0E0,3.703289E-1,1.539154E-1,0E0,0E0,0E0,6.859244E-1,2.2933495E-1,0E0,1.1461353E-1,0E0,0E0,5.5208206E-1,3.4554946E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2862067E-1,0E0,0E0,1.4594752E-1,0E0,0E0,4.9916983E-1,1.4001942E-1,4.1745436E-1,0E0,0E0,0E0,1.3290837E-1,1.4290059E-1,2.839706E-1,0E0,0E0,0E0,0E0,1.334548E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1397505E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,23,23,24,24,26,26,29,29,30,30,37,37,40,40,43,43,44,44,45,45,49,49,50,50,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,-1,-1,52,54,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.1259552E2,4.9615598E-1,1.2711891E0,2.727992E6,1.8681856E8,7.5E2,9.8110194E5,1E0,2.6017979E11,-4.1049863E-3,1E0,3.9378E5,6.5E1,2.7479675E-1,2.0787016E5,1.31654E5,3.7274265E2,7.866887E-3,3.2829787E12,3.1277522E6,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.302E3,3.0327642E-1,4.675196E-2,3.301E3,9.75509E-3,2.0310232E-2,1.3738942E0,2.0766662E6,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,1.4958695E5,2.2762196E-2,-4.9231676E-3,1.7377981E9,8.6237835E-3,2.6556352E-2,4.969844E3,7.07816E10,4.5595855E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.5E1,4.089454E4,1.9415865E2,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,2E1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,2.5605E4,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"split_indices":[54,41,41,1,7,0,35,77,33,0,77,1,8,40,30,9,4,0,33,53,0,0,0,2,44,0,2,0,0,55,50,0,0,0,0,0,0,30,0,0,12,0,0,49,33,58,0,0,0,3,30,54,0,0,0,0,3,0,0,0,0,0,0,31,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.54E2,9.5E1,1.23E2,3.1E1,5.8E1,3.7E1,1.01E2,2.2E1,6E0,2.5E1,8E0,5E1,2.2E1,1.5E1,9.2E1,9E0,4E0,1.8E1,2.2E1,3E0,3E0,5E0,1.5E1,3.5E1,5E0,1.7E1,6E0,9E0,4.9E1,4.3E1,2E0,7E0,1.3E1,5E0,1.3E1,9E0,8E0,7E0,3E0,3.2E1,2E0,1.5E1,3.3E1,1.6E1,3.7E1,6E0,2E0,6E0,7E0,2.5E1,2.7E1,6E0,1E1,6E0,3E0,3.4E1,4E0,3E0,4E0,2.1E1,2.5E1,2E0,3E1,4E0,2E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-6.4136046E-3,-1.6499257E-1,2.6240352E-1,-2.5779408E-1,-1.8667922E-4,1.3376746E-1,5.1147556E-1,-3.5323516E-1,-8.524396E-2,-9.4428E-2,1.0056048E-1,-9.662582E-2,1.73225E-1,3.220377E-2,3.9076596E-1,-1.4722008E-1,-4.0586957E-1,-1.8771532E-1,1.03829466E-1,-1.9139503E-1,7.505271E-2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,2.0356877E-1,-2.7897414E-3,4.3953964E-1,5.554044E-3,-9.59708E-3,4.6992786E-3,-4.2843238E-1,-4.7170566E-3,-1.2011155E-3,-2.7610272E-1,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,9.2738315E-2,2.768057E-1,7.707731E-3,2.085054E-2,-3.687053E-1,-5.6087404E-1,-1.6907474E-2,-7.4428786E-3,9.665818E-3,-4.444956E-2,3.618798E-1,4.56879E-3,-3.8534006E-1,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,4.9356914E-1,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,41,-1,43,-1,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,-1,57,59,-1,61,-1,-1,-1,-1,-1,-1,63,-1,-1,-1,-1],"loss_changes":[1.0913624E1,2.4623237E0,2.975348E0,1.6661301E0,5.696739E-1,5.985464E-1,5.864744E-1,6.559181E-1,7.557257E-1,5.238199E-1,2.0130792E-1,5.320686E-1,4.088E-1,0E0,2.3964357E-1,3.0165756E-1,3.0306053E-1,3.463729E-1,1.8510923E-1,2.6532936E-1,1.640884E-1,0E0,0E0,0E0,0E0,3.7205648E-1,0E0,1.0671902E-1,0E0,0E0,0E0,1.7289925E-1,0E0,0E0,1.2194073E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.8078758E-1,3.8673472E-1,0E0,0E0,1.6442966E-1,1.01364136E-1,0E0,0E0,0E0,2.1470177E-1,1.7472506E-1,0E0,8.7862015E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.314799E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,27,27,31,31,34,34,41,41,42,42,45,45,46,46,50,50,51,51,53,53,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,42,-1,44,-1,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,-1,58,60,-1,62,-1,-1,-1,-1,-1,-1,64,-1,-1,-1,-1],"split_conditions":[3.3103763E2,2.5106274E-2,1.8829846E-1,1.9415865E2,9.50379E5,7.5E2,7.5921344E5,2.431768E2,5.99999E9,1.2185057E0,1.3E1,3.13E2,1.25440235E1,3.220377E-2,2.56701E5,3.160848E6,7.4E1,3.0364912E5,4.654E3,2.587E3,1.7468556E2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,4.871647E2,-2.7897414E-3,1.5789307E8,5.554044E-3,-9.59708E-3,4.6992786E-3,1.912E3,-4.7170566E-3,-1.2011155E-3,5.316779E5,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,1.1572553E7,1.8751362E7,7.707731E-3,2.085054E-2,7.872337E5,1.2967741E0,-1.6907474E-2,-7.4428786E-3,9.665818E-3,1.1E1,5.5053414E9,4.56879E-3,1.0169992E0,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,1.6851284E7,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"split_indices":[54,40,40,54,9,0,35,57,5,43,3,0,59,0,31,9,3,35,2,2,54,0,0,0,0,54,0,5,0,0,0,2,0,0,35,0,0,0,0,0,0,47,46,0,0,53,55,0,0,0,3,5,0,55,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.54E2,1.6E2,9.4E1,1.02E2,5.8E1,6.3E1,3.1E1,6.5E1,3.7E1,3E1,2.8E1,9E0,5.4E1,9E0,2.2E1,1.4E1,5.1E1,2.4E1,1.3E1,1.9E1,1.1E1,1.9E1,9E0,4E0,5E0,4.8E1,6E0,1.8E1,4E0,1.1E1,3E0,4.7E1,4E0,9E0,1.5E1,1E1,3E0,1.7E1,2E0,7E0,4E0,2E1,2.8E1,3E0,1.5E1,3.5E1,1.2E1,6E0,9E0,1E1,1E1,1.8E1,1E1,3.3E1,2E0,4E0,8E0,4E0,6E0,1.1E1,7E0,2E0,3.1E1,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.2190764E-2,-1.8730694E-1,2.753607E-1,-2.6419362E-1,5.672748E-2,1.16373606E-1,5.309576E-1,-3.1258324E-1,-1.8713716E-2,-1.0737888E-1,1.2148461E-1,-1.23983465E-1,1.630511E-1,4.4448983E-2,4.534144E-1,-2.807187E-1,-2.4374343E-2,1.0378545E-1,-1.09028615E-1,-7.691218E-3,4.587331E-4,1.4246969E-1,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,2.1820955E-1,-6.480758E-3,5.3746756E-3,4.961946E-1,-2.0117857E-2,-2.3034851E-1,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1.7839447E-1,6.969961E-2,3.162982E-1,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,-2.1047248E-1,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.4450694E-1,9.220269E-3,-2.461917E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,-7.4021526E-2,-2.9362077E-1,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,-1,-1,37,39,-1,41,-1,43,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.2237465E1,2.848199E0,3.6767783E0,1.360198E0,4.0624598E-1,6.7093295E-1,8.615341E-1,5.956049E-1,2.3258556E-1,9.1160655E-2,1.3540426E-1,3.5389587E-1,4.596188E-1,0E0,3.909912E-1,6.838703E-1,0E0,1.7328064E-1,1.0814984E-1,0E0,0E0,1.2905765E-1,0E0,0E0,0E0,5.169381E-1,1.1691285E-1,0E0,1.8245935E-1,0E0,3.9284968E-1,0E0,0E0,0E0,0E0,0E0,1.09398365E-1,1.9227347E-1,2.0228004E-1,0E0,0E0,0E0,0E0,0E0,3.2970786E-1,0E0,0E0,0E0,0E0,1.4234447E-1,0E0,4.1922283E-1,0E0,0E0,0E0,2.4974391E-1,2.2993183E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,25,25,26,26,28,28,30,30,36,36,37,37,38,38,44,44,49,49,51,51,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,-1,-1,38,40,-1,42,-1,44,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[3.3103763E2,4.4269168E-1,1.7842576E-1,2.727992E6,2.1123085E8,7.5E2,1.431E4,1E0,1.8000048E10,1.3402277E0,1.1771224E8,3.9378E5,4.183787E0,4.4448983E-2,1E0,2.647867E-4,-2.4374343E-2,2.686459E6,1.6868966E8,-7.691218E-3,4.587331E-4,2.25E2,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,4.5971466E2,4.9863962E2,5.3746756E-3,2.9743274E8,-2.0117857E-2,1E0,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1E0,1.1572553E7,8.312613E5,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,2.933E3,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.488E3,9.220269E-3,6.7897725E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,2.7287993E0,6.767991E5,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"split_indices":[54,44,40,1,7,0,9,77,5,43,34,1,59,0,18,41,0,31,7,0,0,0,0,0,0,54,54,0,5,0,68,0,0,0,0,0,77,47,49,0,0,0,0,0,10,0,0,0,0,2,0,58,0,0,0,56,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.5E2,9.1E1,1.14E2,3.6E1,5.7E1,3.4E1,9.5E1,1.9E1,1E1,2.6E1,9E0,4.8E1,3E0,3.1E1,8.6E1,9E0,8E0,1.1E1,6E0,4E0,2.4E1,2E0,4E0,5E0,3.6E1,1.2E1,4E0,2.7E1,1.7E1,6.9E1,4E0,4E0,8E0,3E0,5E0,1.9E1,1.5E1,2.1E1,5E0,7E0,5E0,2.2E1,2E0,6.7E1,1.6E1,3E0,9E0,6E0,8E0,1.3E1,5.3E1,1.4E1,3E0,5E0,1.2E1,4.1E1,2E0,1E1,3.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.1875391E-3,-1.9125406E-1,2.4149728E-1,-3.2296014E-1,-4.532346E-2,9.999164E-2,4.7495925E-1,-3.5961127E-1,-3.3613036E-3,4.6751868E-2,-1.47186E-1,-3.1877293E-3,1.2955628E-1,1.2192243E-1,6.471233E-1,-2.8752145E-1,-5.190118E-1,-3.20057E-2,7.1740914E-3,-1.8803763E-1,5.201352E-3,1.4700729E-1,-4.1586305E-3,6.593719E-4,9.316681E-3,9.0452635E-1,5.182496E-1,4.057908E-2,-3.2754722E-1,-5.1887655E-3,-5.5908924E-1,3.8056083E-3,-3.9105513E-3,-2.4499905E-1,-2.4137127E-3,4.0515624E-2,1.8407537E-1,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,-2.7528796E-1,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,1.9600414E-1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,1.5410087E-1,4.4805845E-3,9.989807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,-1,33,-1,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,47,-1,49,51,-1,-1,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,57,-1,-1],"loss_changes":[1.1762257E1,2.7338152E0,3.6525607E0,6.483345E-1,6.540341E-1,3.7268925E-1,2.4738722E0,5.976982E-1,0E0,3.480822E-1,3.8151592E-1,0E0,2.5336754E-1,1.4349061E-1,4.6006012E-1,6.3051796E-1,2.3819494E-1,1.6576359E-1,0E0,2.0569742E-1,0E0,2.1754622E-1,0E0,0E0,0E0,2.0789146E-1,1.3972569E-1,1.1563162E-1,1.779995E-1,0E0,1.1710739E-1,0E0,0E0,1.0057318E-1,0E0,2.0077212E-1,1.1754835E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.099534E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.906554E-2,0E0,0E0,0E0,0E0,8.8078976E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,21,21,25,25,26,26,27,27,28,28,30,30,33,33,35,35,36,36,44,44,51,51,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,-1,34,-1,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,48,-1,50,52,-1,-1,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,58,-1,-1],"split_conditions":[3.5300907E-1,1.9415865E2,1.5972575E0,1.848519E-2,1.560615E1,1.8681856E8,2.744E3,5.039375E-3,-3.3613036E-3,3.165913E6,5.4878284E2,-3.1877293E-3,9.723711E0,6.765595E7,2.7479675E-1,1.0395554E5,1.47887E5,9.165388E4,7.1740914E-3,3.1692016E0,5.201352E-3,2.2599022E8,-4.1586305E-3,6.593719E-4,9.316681E-3,4.063E3,2.0467837E0,4.56E2,1.7565872E-3,-5.1887655E-3,6.6037946E2,3.8056083E-3,-3.9105513E-3,3.2829E4,-2.4137127E-3,1.19532734E5,1E0,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,4.5E2,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,5.7E1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,2.63E2,4.4805845E-3,9.989807E-3],"split_indices":[44,54,43,40,58,7,2,40,0,1,54,0,59,5,40,30,1,35,0,60,0,12,0,0,0,2,55,0,60,0,4,0,0,9,0,30,103,0,0,0,0,0,0,0,10,0,0,0,0,0,0,8,0,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,7.4E1,6.8E1,7E1,4.1E1,6.4E1,1E1,3.6E1,3.2E1,1E1,6E1,1.4E1,2.7E1,4.6E1,1.8E1,2.2E1,1.4E1,2.8E1,4E0,5.6E1,4E0,7E0,7E0,7E0,2E1,5E0,4.1E1,2E0,1.6E1,7E0,1.5E1,1.9E1,9E0,1.5E1,4.1E1,2E0,5E0,1.7E1,3E0,2E0,3E0,9E0,3.2E1,1.4E1,2E0,8E0,1.1E1,3E0,1.2E1,3.9E1,2E0,2.3E1,9E0,1.1E1,2.8E1,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.9416563E-3,-1.3463952E-1,3.7984976E-1,-2.3853315E-1,1.360329E-2,1.961173E-1,5.876671E-1,-3.1009814E-1,-1.1053116E-1,-3.428409E-2,1.00340895E-1,2.3631513E-2,1.5987208E-1,1.1244957E-2,6.9594884E-1,-6.4688744E-3,-3.5531205E-1,1.6310263E-3,-2.0668645E-1,-7.293062E-2,1.3145976E-1,9.800815E-3,-3.9949227E-2,-4.0140087E-3,1.8721226E-1,4.3395724E-2,2.3986792E-2,3.5975182E-3,-3.7373397E-1,-1.2723958E-2,-3.946153E-3,-9.6303485E-2,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,2.38724E-1,1.5173156E-3,-3.9501062E-1,-4.1339686E-3,8.0993416E-4,-1.22839086E-1,1.2143473E-2,4.7381404E-3,-4.0761718E-1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,-4.2140347E-1,-2.2882612E-2,-1.5165369E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,-1,29,31,33,-1,35,-1,37,-1,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,43,-1,45,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,-1],"loss_changes":[1.2767532E1,2.8833227E0,2.3666296E0,9.7337484E-1,3.2742375E-1,3.641218E-1,8.926077E-1,4.596181E-1,5.9082735E-1,3.3626413E-1,4.9703383E-1,0E0,2.5133562E-1,0E0,4.1397285E-1,0E0,4.6627617E-1,0E0,2.203567E-1,2.0018259E-1,1.2457131E-1,0E0,2.0385656E-1,0E0,2.3512948E-1,0E0,0E0,0E0,2.7126503E-1,0E0,0E0,1.1810273E-1,0E0,0E0,0E0,0E0,0E0,8.816862E-2,0E0,1.6809559E-1,0E0,0E0,9.8291665E-2,0E0,0E0,9.2234135E-2,0E0,0E0,0E0,0E0,1.4487553E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,18,18,19,19,20,20,22,22,24,24,28,28,31,31,37,37,39,39,42,42,45,45,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,-1,30,32,34,-1,36,-1,38,-1,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,44,-1,46,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.427E3,8.9397516E4,3.856E3,2.4747229E2,1.3048244E1,1.4191151E6,9E0,2.3631513E-2,7.5E2,1.1244957E-2,2.7479675E-1,-6.4688744E-3,3.8757736E-7,1.6310263E-3,3.03E9,4.222002E8,1.31903E5,9.800815E-3,1.3116745E1,-4.0140087E-3,4.183787E0,4.3395724E-2,2.3986792E-2,3.5975182E-3,7.4E1,-1.2723958E-2,-3.946153E-3,2.6639402E-1,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,4.3147E5,1.5173156E-3,9.4983295E2,-4.1339686E-3,8.0993416E-4,4.9615598E-1,1.2143473E-2,4.7381404E-3,4.699202E1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,9.6713684E1,-2.2882612E-2,-1.5165369E-2],"split_indices":[54,43,40,54,2,35,2,57,58,35,3,0,0,0,40,0,40,0,5,7,1,0,58,0,59,0,0,0,3,0,0,44,0,0,0,0,0,9,0,4,0,0,41,0,0,54,0,0,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.86E2,6.4E1,1.09E2,7.7E1,3.5E1,2.9E1,6.9E1,4E1,5E1,2.7E1,2E0,3.3E1,8E0,2.1E1,1.6E1,5.3E1,1.6E1,2.4E1,4.1E1,9E0,1.4E1,1.3E1,3E0,3E1,5E0,1.6E1,2E0,5.1E1,1.3E1,1.1E1,3.7E1,4E0,6E0,3E0,4E0,9E0,2.2E1,8E0,4.7E1,4E0,7E0,3E1,1.6E1,6E0,4.5E1,2E0,1.1E1,1.9E1,3E0,4.2E1,1.5E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[4.442649E-3,-1.2406673E-1,3.1314126E-1,-2.231523E-1,2.7577164E-2,1.5845458E-1,5.6371045E-1,-2.9410875E-1,-5.8644965E-2,-2.6454885E-2,1.4853756E-1,2.294401E-2,1.3163756E-1,2.7344096E-3,6.19114E-1,-9.231322E-2,-3.2586154E-1,-1.3393481E-1,2.8952213E-2,-4.4858553E-3,1.6983777E-2,1.7551805E-1,-3.9204443E-3,2.2740224E-1,3.826856E-2,1.9222386E-2,8.226514E-1,-7.2305724E-3,7.988117E-3,-3.752356E-1,-1.8137792E-1,3.932153E-3,-7.2262976E-3,-2.8384456E-3,1.1973143E-1,4.0585145E-2,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,-1.2312204E-1,5.3003617E-3,1.6423428E-2,4.1150242E-2,-3.9276025E-1,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,9.395361E-2,3.7756322E-3,-8.526156E-3,-3.2505807E-1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,-3.9313173E-1,-1.9234711E-2,-7.488049E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,-1,39,41,-1,43,-1,-1,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,53,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,61,-1,-1],"loss_changes":[1.0392895E1,2.7859118E0,2.895176E0,1.2901125E0,4.8757413E-1,4.1345346E-1,7.6624584E-1,4.6970892E-1,2.3470895E-1,1.741834E-1,1.6369265E-1,0E0,4.1154587E-1,0E0,5.829191E-1,2.727965E-1,4.0990114E-1,1.4968318E-1,1.5284702E-1,0E0,1.21554464E-1,9.423089E-2,0E0,2.5063312E-1,3.5061046E-1,0E0,2.3580599E-1,0E0,0E0,1.4392567E-1,3.3857572E-1,0E0,0E0,0E0,1.4192495E-1,1.1094484E-1,0E0,0E0,0E0,0E0,0E0,1.5323834E-1,0E0,0E0,0E0,2.5782776E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.29731E-2,0E0,0E0,1.570108E-1,0E0,0E0,0E0,0E0,1.24889135E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,26,26,29,29,30,30,34,34,35,35,41,41,45,45,52,52,55,55,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,-1,40,42,-1,44,-1,-1,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,54,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,62,-1,-1],"split_conditions":[3.705974E2,2.1619815E-1,2.0224316E-1,4.5537444E9,2.427E3,8.9397516E4,1E0,1.1563724E8,2.0256741E2,1.503E3,5.4281917E0,2.294401E-2,1.3844932E7,2.7344096E-3,1.6507233E3,3.160848E6,2.2348747E2,1.903E3,4.231592E6,-4.4858553E-3,2.302E3,3.5734247E2,-3.9204443E-3,5.264E4,1.1218195E6,1.9222386E-2,1.61628E5,-7.2305724E-3,7.988117E-3,9.0814667E2,4.1081388E5,3.932153E-3,-7.2262976E-3,-2.8384456E-3,4.3560153E5,6.4796996E-1,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,1.4153711E-3,5.3003617E-3,1.6423428E-2,4.1150242E-2,1.549E3,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,1.5130641E0,3.7756322E-3,-8.526156E-3,1.5E1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,1E0,-1.9234711E-2,-7.488049E-3],"split_indices":[54,43,40,5,2,35,18,5,54,2,56,0,47,0,4,9,54,0,1,0,2,54,0,9,49,0,30,0,0,57,30,0,0,0,34,41,0,0,0,0,0,40,0,0,0,2,0,0,0,0,0,0,58,0,0,3,0,0,0,0,105,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,1.84E2,7.6E1,1.11E2,7.3E1,4.8E1,2.8E1,7.7E1,3.4E1,5.1E1,2.2E1,2E0,4.6E1,3E0,2.5E1,1.1E1,6.6E1,1.8E1,1.6E1,1.8E1,3.3E1,2E1,2E0,2.2E1,2.4E1,1.5E1,1E1,9E0,2E0,4.8E1,1.8E1,2E0,1.6E1,8E0,8E0,2.9E1,4E0,1.8E1,2E0,1E1,1.2E1,8E0,1.6E1,3E0,7E0,4.4E1,4E0,1.4E1,4E0,3E0,5E0,1.3E1,1.6E1,2E0,6E0,3.2E1,1.2E1,8E0,8E0,1.4E1,1.8E1,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.2187884E-2,-1.0560222E-1,2.8632665E-1,-1.6603857E-1,6.5966286E-2,1.4586735E-1,4.827894E-1,-2.163441E-1,3.5765186E-2,-2.1459074E-3,9.6117355E-2,2.2276511E-2,1.16351E-1,3.9712414E-3,5.3956425E-1,-1.9085151E-1,-4.2924163E-1,1.310548E-1,-6.960556E-2,8.844785E-3,5.8431923E-2,-3.4577996E-2,1.8520117E-1,1.25307795E-2,2.6944714E-2,-3.2403958E-1,-1.5634167E-1,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,5.7000898E-2,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,4.627612E-2,-1.0834768E-2,2.3908277E-1,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,-1.3881442E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,-1.12805806E-1,-3.083408E-1,-7.1671814E-2,-2.4239872E-1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,-1,47,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1],"loss_changes":[7.844618E0,1.7736785E0,1.9440131E0,1.2868705E0,1.6094884E-1,4.141271E-1,6.1820793E-1,4.6623707E-1,2.7018133E-1,0E0,1.327295E-1,0E0,4.4284636E-1,0E0,3.3383512E-1,3.7219834E-1,2.928431E-1,1.1195864E-1,1.3693142E-1,0E0,1.6056333E-1,2.5339377E-1,1.8323988E-1,0E0,0E0,1.318934E-1,3.0327547E-1,0E0,0E0,0E0,1.019789E-1,0E0,0E0,0E0,0E0,1.6095355E-1,0E0,1.4495695E-1,0E0,0E0,0E0,0E0,2.8267586E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2379E-1,1.0256362E-1,2.1890551E-1,9.2020154E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,30,30,35,35,37,37,42,42,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,-1,48,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,4.9615598E-1,1.9530877E-1,2.727992E6,1.8681856E8,8.9397516E4,1.2354571E0,1E0,1.6E1,-2.1459074E-3,2.0383387E0,2.2276511E-2,1.479938E6,3.9712414E-3,7.2610445E8,3.3617156E-4,2.22105E5,1.2051221E0,1.7108529E3,8.844785E-3,2.2599022E8,2.629593E0,1.8990228E0,1.25307795E-2,2.6944714E-2,1.7565872E-3,3.4955364E7,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,4.654E3,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,6.2143734E4,-1.0834768E-2,1.6851284E7,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,3.219065E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,8.3170245E2,2.07298E5,6.214038E6,1.2271346E1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"split_indices":[54,41,40,1,7,35,55,77,3,0,60,0,34,0,12,41,1,55,4,0,12,56,55,0,0,60,7,0,0,0,2,0,0,0,0,30,0,1,0,0,0,0,44,0,0,0,0,0,0,57,1,9,58,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.69E2,7.2E1,1.25E2,4.4E1,4.3E1,2.9E1,1E2,2.5E1,9E0,3.5E1,2E0,4.1E1,4E0,2.5E1,9.1E1,9E0,1.3E1,1.2E1,8E0,2.7E1,1.3E1,2.8E1,7E0,1.8E1,1.7E1,7.4E1,2E0,7E0,4E0,9E0,1E1,2E0,1.7E1,1E1,1E1,3E0,1.9E1,9E0,6E0,1.1E1,3E0,7.1E1,6E0,3E0,5E0,5E0,1.3E1,6E0,6.3E1,8E0,4.9E1,1.4E1,4E0,4E0,4.4E1,5E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[8.535432E-5,-1.3134183E-1,2.4541517E-1,-2.1222556E-1,-3.1241742E-3,1.1487452E-1,4.674134E-1,-2.7328297E-1,-3.8918845E-2,-8.365333E-2,9.0494335E-2,2.394816E-3,1.6543052E-1,2.984512E-1,6.282826E-1,-2.368147E-1,-2.039022E-2,-7.2169475E-2,5.6061996E-3,-1.5359263E-1,5.1900256E-2,2.500614E-4,6.980609E-3,5.4644295E-3,-7.968769E-2,1.8786167E-1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,7.535615E-1,1.2825073E-2,-2.2865131E-2,-2.539767E-1,4.7256513E-3,-1.1367583E-1,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.703592E-1,1.374485E-1,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,-2.2705331E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,-2.3829083E-1,-2.8504506E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,-1,-1,-1,39,41,-1,-1,-1,43,-1,45,47,-1,49,-1,-1,-1,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1],"loss_changes":[8.318686E0,1.7416182E0,2.5544882E0,1.0792751E0,5.0514543E-1,3.2942206E-1,7.09455E-1,4.2694187E-1,1.6511148E-1,3.4927028E-1,1.8162525E-1,2.0153141E-1,2.3894668E-1,1.7641377E-1,4.4881153E-1,2.3653197E-1,0E0,1.9207498E-1,0E0,2.4937719E-1,1.370348E-1,0E0,0E0,0E0,2.0430191E-1,1.1672115E-1,0E0,0E0,0E0,1.8685484E-1,0E0,1.2997259E-1,1.903727E-1,0E0,2.3296595E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.380126E-2,9.920034E-2,0E0,0E0,0E0,0E0,0E0,1.8071914E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.499064E-1,1.1059618E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,24,24,25,25,29,29,31,31,32,32,34,34,41,41,42,42,48,48,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,-1,-1,-1,40,42,-1,-1,-1,44,-1,46,48,-1,50,-1,-1,-1,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1],"split_conditions":[3.4021906E2,2.5106274E-2,1.8829846E-1,1.781973E6,9.50379E5,3.359399E-1,5.6E1,2.418166E0,1.22E2,1.5225257E-1,5.3164E4,1.1936242E7,5.615313E3,2.598887E7,3.2307692E0,1.1623532E6,-2.039022E-2,2.207E3,5.6061996E-3,2.587E3,8.180814E7,2.500614E-4,6.980609E-3,5.4644295E-3,1.0520055E5,5.7E1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,1.4475617E0,1.2825073E-2,2.1426034E5,1.4614057E-3,4.7256513E-3,2.4616451E0,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.7307575E5,1.0899524E3,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,4.2446044E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,1.3496346E0,1.1306552E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"split_indices":[54,40,40,1,9,44,8,55,8,40,31,47,4,9,56,47,0,0,0,2,34,0,0,0,35,8,0,0,0,55,0,30,60,0,56,0,0,0,0,0,0,30,4,0,0,0,0,0,58,0,0,0,0,0,0,0,55,41,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.56E2,1.67E2,8.9E1,1.02E2,6.5E1,5.7E1,3.2E1,7.5E1,2.7E1,3.5E1,3E1,1.8E1,3.9E1,1.7E1,1.5E1,6.5E1,1E1,2.3E1,4E0,2.3E1,1.2E1,1.4E1,1.6E1,7E0,1.1E1,3.6E1,3E0,1.5E1,2E0,1E1,5E0,5E0,6E1,4E0,1.9E1,2E1,3E0,9E0,3E0,2E0,9E0,1.2E1,2.4E1,4E0,6E0,2E0,3E0,6E0,5.4E1,1.7E1,2E0,7E0,5E0,5E0,1.9E1,2E0,5.2E1,3.1E1,2.1E1,2.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-7.534086E-3,-8.694848E-2,3.8782644E-1,-1.7279062E-1,7.5343594E-2,1.2632059E-1,5.605976E-1,-2.4380171E-1,-4.4355273E-2,1.0239507E-1,-7.070103E-2,-8.078534E-4,9.607972E-3,6.888959E-1,4.1058496E-1,-2.8629518E-1,-1.11263916E-1,-1.6469195E-2,-1.506756E-2,4.8591096E-2,1.8556008E-1,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,-6.6435128E-3,-3.0423003E-1,-1.0063401E-2,2.9212602E-3,-4.0811297E-2,7.316056E-3,-2.3700453E-2,1.1168387E-1,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,-3.2206443E-1,-3.669968E-3,-6.68522E-2,7.2118444E-3,4.6604417E-2,-4.6691555E-3,8.108817E-3,-7.8055467E-3,-3.4489202E-1,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,-3.753321E-1,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,47,-1,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,-1],"loss_changes":[7.88305E0,2.924702E0,1.8267727E0,1.2386718E0,2.9564223E-1,2.4874002E-1,1.9830894E-1,4.6376085E-1,4.8201522E-1,2.6840913E-1,1.2740509E-1,0E0,0E0,3.1174803E-1,9.529376E-2,3.2900333E-1,5.045887E-1,0E0,2.2792198E-1,1.8052885E-1,1.6800141E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.805466E-2,2.1632099E-1,0E0,0E0,2.3643395E-1,0E0,1.16251014E-1,1.8746912E-1,0E0,0E0,0E0,0E0,1.5240574E-1,0E0,1.1973961E-1,0E0,9.648324E-2,0E0,0E0,1.9439551E-1,1.6225243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4404154E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,27,27,28,28,31,31,33,33,34,34,39,39,41,41,43,43,46,46,47,47,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,48,-1,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,-1],"split_conditions":[1.311002E0,3.051E3,2.787E3,1.5157113E-2,1.50321455E1,2.024E3,2.9898608E-1,2.2348747E2,2.718E4,4.871647E2,1.471E4,-8.078534E-4,9.607972E-3,1.792184E0,3.4048372E6,1E1,2.8041378E5,-1.6469195E-2,1.3431462E7,6.477987E-1,1.8751362E7,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,7.1E1,6.8E1,-1.0063401E-2,2.9212602E-3,1.00849255E3,7.316056E-3,4.303E3,4.99E3,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,1E0,-3.669968E-3,2.1799392E6,7.2118444E-3,2.5063604E7,-4.6691555E-3,8.108817E-3,6.074E3,4.239E3,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,3.1872916E2,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"split_indices":[41,2,2,40,59,2,40,54,9,54,0,0,0,44,47,10,30,0,9,59,46,0,0,0,0,0,0,0,3,0,0,4,0,2,2,0,0,0,0,105,0,50,0,47,0,0,2,10,0,0,0,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,2.08E2,4.1E1,1.36E2,7.2E1,1.7E1,2.4E1,8.7E1,4.9E1,6.1E1,1.1E1,7E0,1E1,1.1E1,1.3E1,6.5E1,2.2E1,3E0,4.6E1,3.8E1,2.3E1,8E0,3E0,5E0,6E0,8E0,5E0,4E0,6.1E1,1.3E1,9E0,4.1E1,5E0,1.8E1,2E1,1.4E1,9E0,2E0,2E0,5.6E1,5E0,3.7E1,4E0,1E1,8E0,1.2E1,8E0,4.8E1,8E0,2E1,1.7E1,7E0,3E0,6E0,2E0,3.9E1,9E0,1.4E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.6526852E-2,-1.3732727E-1,1.8663803E-1,-2.051646E-1,-2.9185692E-2,7.73927E-2,3.695984E-1,-2.6628572E-1,-9.389659E-2,-8.110631E-2,7.3622145E-2,4.680393E-3,1.8481968E-1,1.6930054E-3,3.9849573E-1,-2.2754145E-1,-2.0849528E-2,6.2429474E-4,-1.7701784E-1,-3.0688336E-1,-5.0080888E-2,4.623225E-3,-4.2899745E-3,1.03257135E-1,-1.1058583E-1,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,-2.4562204E-1,7.5825364E-3,-1.0521853E-2,-6.476905E-2,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,1.3644478E-1,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,-2.6290497E-1,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,-2.9959735E-1,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,31,33,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,-1,43,-1,-1,-1,-1,45,-1,-1,-1,-1,47,-1,-1,-1,-1,49,-1,-1,-1],"loss_changes":[6.3821125E0,1.1897857E0,1.9101899E0,6.5507936E-1,3.4759343E-1,4.8343337E-1,3.2867527E-1,4.1569996E-1,3.3530772E-1,2.8583866E-1,1.3266498E-1,4.4319636E-1,5.4417163E-1,0E0,2.268424E-1,4.489975E-1,0E0,0E0,1.3546717E-1,1.6951728E-1,1.4221326E-1,0E0,0E0,1.3188584E-1,1.2820828E-1,0E0,0E0,0E0,0E0,1.5322685E-1,0E0,0E0,9.677847E-2,0E0,0E0,0E0,0E0,1.2525946E-1,0E0,0E0,0E0,0E0,2.1090531E-1,0E0,0E0,0E0,0E0,1.8816972E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,18,18,19,19,20,20,23,23,24,24,29,29,32,32,37,37,42,42,47,47],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,32,34,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,-1,44,-1,-1,-1,-1,46,-1,-1,-1,-1,48,-1,-1,-1,-1,50,-1,-1,-1],"split_conditions":[3.3103763E2,1.1670399E-1,1.7842576E-1,1.9415865E2,2.7538432E8,6.483E3,1E0,4.1938826E-2,3.0364912E5,3.3789E4,1.1771224E8,1.1936242E7,6.2143734E4,1.6930054E-3,7.2610445E8,1.1661899E8,-2.0849528E-2,6.2429474E-4,1.3469651E3,2.151E3,1.2473118E0,4.623225E-3,-4.2899745E-3,1.9816086E0,1.4813267E3,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,6E0,7.5825364E-3,-1.0521853E-2,2.8659306E0,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,3.8692224E8,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,4.426498E0,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,1.1078758E0,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"split_indices":[54,41,40,54,7,2,18,44,35,9,34,47,30,0,12,7,0,0,4,2,55,0,0,55,57,0,0,0,0,10,0,0,60,0,0,0,0,7,0,0,0,0,58,0,0,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.62E2,9.6E1,9.9E1,6.3E1,6.1E1,3.5E1,6.3E1,3.6E1,4.2E1,2.1E1,3.7E1,2.4E1,3E0,3.2E1,5.5E1,8E0,1.6E1,2E1,4E0,3.8E1,1.8E1,3E0,2E1,1.7E1,1.4E1,1E1,1.2E1,2E1,5.3E1,2E0,1.2E1,8E0,2E0,2E0,9E0,2.9E1,1.7E1,3E0,1.2E1,5E0,5E0,4.8E1,6E0,2E0,1.5E1,2E0,3.7E1,1.1E1,5E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.752704E-3,-1.0890242E-1,2.1311827E-1,-1.7942376E-1,-4.630112E-3,8.463161E-2,4.3359125E-1,-2.6526305E-1,-4.0561736E-2,-8.0085285E-2,6.466574E-2,1.7120663E-2,6.155428E-2,2.8981546E-1,6.182841E-1,-1.9980457E-1,-3.8043132E-1,2.1313105E-2,-7.922915E-3,-1.4972365E-1,5.4258812E-2,-3.958611E-6,5.6440667E-3,-6.662196E-2,1.1769473E-1,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,-1.3539992E-1,-2.9197437E-1,-4.6381897E-1,-6.432207E-3,-5.7250984E-2,5.7095457E-3,-7.0690963E-4,-2.4091765E-1,-1.8624915E-3,6.908497E-3,7.913083E-2,-2.8256294E-3,-1.419902E-1,4.703381E-3,1.525669E-1,-8.453891E-4,2.933514E-3,-7.012109E-3,-3.371898E-1,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,7.1813464E-2,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,43,-1,-1,-1,-1,45,47,49,-1,51,-1,-1,53,-1,-1,55,-1,57,-1,59,-1,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,65,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1809554E0,1.2198377E0,2.597793E0,1.1693025E0,3.607325E-1,4.0189618E-1,7.2366E-1,3.8385105E-1,3.4410596E-1,3.1685543E-1,1.5055917E-1,0E0,4.1800576E-1,1.5071511E-1,4.769044E-1,2.0384932E-1,3.3971214E-1,2.4841867E-1,0E0,2.639835E-1,1.2779592E-1,1.0247372E-1,0E0,2.5190163E-1,1.941486E-1,0E0,0E0,0E0,0E0,9.141275E-2,1.3482976E-1,1.2367654E-1,0E0,1.1534542E-1,0E0,0E0,1.0435748E-1,0E0,0E0,1.1921386E-1,0E0,1.5446314E-1,0E0,1.8064415E-1,0E0,1.0156908E-1,0E0,1.0845828E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.946659E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,29,29,30,30,31,31,33,33,36,36,39,39,41,41,43,43,45,45,47,47,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,44,-1,-1,-1,-1,46,48,50,-1,52,-1,-1,54,-1,-1,56,-1,58,-1,60,-1,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,66,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.7613442E-1,1.8829846E-1,1.9415865E2,9.50379E5,8.9397516E4,1.623319E3,1.607E3,2.142943E1,1.2185057E0,6.6727E4,1.7120663E-2,3.359399E-1,2.0270285E8,2.65826E6,4.3399894E5,6.608892E2,6.135367E9,-7.922915E-3,2.8507206E-1,1.7468556E2,1.2495134E6,5.6440667E-3,5.4878284E2,3.2383578E7,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,1.1623532E6,3.342E3,2.1710082E7,-6.432207E-3,2.2E1,5.7095457E-3,-7.0690963E-4,6.2684685E-2,-1.8624915E-3,6.908497E-3,1.6537195E0,-2.8256294E-3,1.409674E7,4.703381E-3,1.7111111E0,-8.453891E-4,4.14E2,-7.012109E-3,1E0,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,1.0685258E6,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"split_indices":[54,43,40,54,9,35,4,2,58,43,31,0,44,7,49,30,4,5,0,44,54,47,0,54,1,0,0,0,0,47,0,34,0,3,0,0,40,0,0,55,0,47,0,55,0,2,0,105,0,0,0,0,0,0,0,0,0,0,0,0,35,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.65E2,9.2E1,9.8E1,6.7E1,5.9E1,3.3E1,6E1,3.8E1,3.2E1,3.5E1,3E0,5.6E1,2E1,1.3E1,4E1,2E1,2.7E1,1.1E1,2.1E1,1.1E1,1.8E1,1.7E1,1.7E1,3.9E1,3E0,1.7E1,9E0,4E0,2.5E1,1.5E1,1.4E1,6E0,1.6E1,1.1E1,9E0,1.2E1,6E0,5E0,8E0,1E1,1.2E1,5E0,3.1E1,8E0,4E0,2.1E1,1.2E1,3E0,1.2E1,2E0,1.1E1,5E0,5E0,7E0,5E0,3E0,3E0,9E0,1.5E1,1.6E1,2E0,2E0,8E0,4E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-4.591145E-3,-1.0742565E-1,2.1129104E-1,-1.6721527E-1,-3.2239808E-3,9.313745E-2,3.8930318E-1,-2.2656383E-1,-3.6957923E-2,-4.179782E-2,9.553272E-2,1.1645269E-1,-6.1090174E-3,9.100737E-3,4.8732993E-1,-1.9007076E-1,-3.574318E-1,8.0027975E-2,-3.7945511E-3,3.2766287E-3,-6.838066E-2,5.598663E-3,-2.234741E-3,-1.6062098E-3,1.4490457E-1,3.515733E-2,1.7769938E-2,2.4345317E-3,-2.1554193E-1,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,-8.616666E-2,7.324565E-3,2.3872483E-1,1.1299209E-1,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,-1.5753022E-1,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.572733E0,1.0585915E0,1.6673958E0,8.2653E-1,2.4398685E-1,2.8807345E-1,4.6537733E-1,2.8546977E-1,2.1166551E-1,1.4805996E-1,9.310773E-2,2.0591748E-1,0E0,0E0,1.8203974E-1,2.947247E-1,2.1712995E-1,1.3969424E-1,0E0,0E0,1.7411983E-1,0E0,0E0,0E0,8.9793384E-2,0E0,0E0,1.5928355E-1,2.97374E-1,0E0,0E0,0E0,0E0,1.2090984E-1,0E0,9.095979E-2,1.5865728E-1,0E0,0E0,0E0,1.3265473E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,20,20,24,24,27,27,28,28,33,33,35,35,36,36,40,40],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0471938E6,1.529448E-1,1.2711891E0,2.8704E4,9.74979E6,1.25440235E1,3.301E3,4.1295107E-2,1.0520055E5,4.46E2,1E0,3.359399E-1,-6.1090174E-3,9.100737E-3,1.431E4,1.1695156E5,1.1E1,2.00902E2,-3.7945511E-3,3.2766287E-3,2.65852E6,5.598663E-3,-2.234741E-3,-1.6062098E-3,8.312613E5,3.515733E-2,1.7769938E-2,1E0,2.647867E-4,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,1.1343225E6,7.324565E-3,9.616974E2,3.7724147E8,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,5.0090684E5,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"split_indices":[50,41,41,11,9,59,2,44,35,11,77,44,0,0,9,30,3,54,0,0,1,0,0,0,49,0,0,8,41,0,0,0,0,35,0,4,7,0,0,0,30,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.69E2,8E1,1.07E2,6.2E1,4.9E1,3.1E1,7.3E1,3.4E1,4.5E1,1.7E1,4.5E1,4E0,1.2E1,1.9E1,5.9E1,1.4E1,1E1,2.4E1,8E0,3.7E1,1.4E1,3E0,7E0,3.8E1,2E0,1.7E1,7E0,5.2E1,6E0,8E0,2E0,8E0,3.5E1,2E0,8E0,3E1,3E0,4E0,1.6E1,3.6E1,2.4E1,1.1E1,3E0,5E0,1.7E1,1.3E1,3.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.739959E-3,-1.0096392E-1,2.598716E-1,-1.7680193E-1,-3.243053E-2,1.19685955E-1,4.226019E-1,-8.241986E-2,-2.739108E-1,-8.830396E-2,4.9990978E-2,1.8303644E-2,8.3055995E-2,2.0394288E-3,4.746282E-1,-1.0853977E-2,-3.597446E-2,-3.5187337E-1,-1.5231963E-1,-2.7551723E-1,-2.8254518E-2,9.033917E-3,5.8722845E-3,-8.941312E-3,1.0602045E-1,6.4259046E-1,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.3898046E-2,-4.325442E-3,3.7839024E-3,-3.718894E-2,1.2833083E-1,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,29,31,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4447184E0,8.7263286E-1,1.5462923E0,7.1065116E-1,4.1887993E-1,3.9337415E-1,6.0357904E-1,3.1546003E-1,3.161056E-1,5.992862E-1,1.2694931E-1,0E0,2.597763E-1,0E0,4.780264E-1,0E0,2.3357649E-1,1.2557936E-1,1.1205655E-1,3.2829583E-1,1.5759072E-1,9.657455E-2,0E0,0E0,1.6324815E-1,1.374054E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4665566E-1,0E0,0E0,1.1835869E-1,1.3544708E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,17,17,18,18,19,19,20,20,21,21,24,24,25,25,35,35,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,30,32,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,1.6375645E2,2.0224316E-1,1.69607E5,3.2764925E5,1.2E1,1.2354571E0,4.0561585E-3,5.3597E2,3.6477748E7,7.128343E5,1.8303644E-2,1.3E1,2.0394288E-3,2.9898608E-1,-1.0853977E-2,1.2245615E0,3.9647983E2,9.714086E0,6.779745E6,3.642097E6,9.598357E0,5.8722845E-3,-8.941312E-3,9.723711E0,2.4214827E5,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.142943E1,-4.325442E-3,3.7839024E-3,8.1262054E2,1.7021878E7,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"split_indices":[54,54,40,1,30,3,55,43,4,12,30,0,3,0,40,0,55,4,49,12,34,58,0,0,59,30,0,0,0,0,0,0,0,0,0,58,0,0,4,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,1.68E2,6.9E1,7.9E1,8.9E1,3.8E1,3.1E1,4.1E1,3.8E1,5.3E1,3.6E1,3E0,3.5E1,4E0,2.7E1,8E0,3.3E1,2.2E1,1.6E1,1.2E1,4.1E1,2.5E1,1.1E1,2E0,3.3E1,1.1E1,1.6E1,7E0,2.6E1,1.2E1,1E1,2E0,1.4E1,8E0,4E0,2.4E1,1.7E1,9E0,1.6E1,3E1,3E0,4E0,7E0,1.5E1,9E0,1.1E1,5E0,2.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.7042104E-3,-7.20367E-2,2.6177105E-1,-1.2977166E-1,4.7325946E-2,1.1269311E-1,4.0182242E-1,-1.1040865E-1,-3.0936033E-1,1.5922122E-2,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,-1.6456133E-1,-5.346679E-2,-1.7062483E-2,-2.109847E-3,-2.8860811E-2,1.2831074E-1,-2.0478494E-1,5.846947E-4,8.376397E-3,-6.544534E-2,-5.855391E-3,-6.571167E-3,1.2199961E-2,1.0790331E-3,-1.0410925E-1,-2.5988343E-1,-7.7399775E-2,7.660132E-3,3.2776052E-3,-5.569639E-2,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,-5.8419913E-2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,2.7633142E-3,1.8774311E-3,-5.525428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,25,27,29,-1,-1,31,33,-1,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,-1,-1,45,-1,-1],"loss_changes":[4.8915606E0,1.3223984E0,1.1447959E0,4.1442585E-1,2.5857258E-1,1.2807718E-1,6.305504E-1,3.5506094E-1,2.3678052E-1,2.6542392E-1,0E0,0E0,0E0,0E0,0E0,4.3223488E-1,1.8600516E-1,0E0,0E0,1.0552636E-1,2.2810276E-1,2.4475026E-1,0E0,0E0,1.7779067E-1,1.3734703E-1,0E0,0E0,0E0,1.1772323E-1,1.4893365E-1,1.1551082E-1,0E0,0E0,1.1556485E-1,0E0,0E0,0E0,0E0,1.1671467E-1,0E0,0E0,0E0,0E0,1.0962085E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,15,15,16,16,19,19,20,20,21,21,24,24,25,25,29,29,30,30,31,31,34,34,39,39,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,26,28,30,-1,-1,32,34,-1,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,-1,-1,46,-1,-1],"split_conditions":[1.3394321E0,7.629E3,2.0787016E5,3.3135505E0,5.2328894E5,7.323253E8,1.623319E3,1.66863E5,1.5157113E-2,8.834E3,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,2.951829E9,1.55E2,-1.7062483E-2,-2.109847E-3,2.1445148E0,5.2638875E6,6.8618E4,5.846947E-4,8.376397E-3,6.098826E6,3.1791415E11,-6.571167E-3,1.2199961E-2,1.0790331E-3,1.3182504E0,1.7218688E8,2.2733334E1,7.660132E-3,3.2776052E-3,4.1E1,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,2.0256741E2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,4.4326835E0,1.8774311E-3,-5.525428E-3],"split_indices":[44,10,30,55,30,33,4,9,40,2,0,0,0,0,0,5,11,0,0,55,45,31,0,0,50,33,0,0,0,58,7,58,0,0,8,0,0,0,0,54,0,0,0,0,56,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.9E2,5.6E1,1.28E2,6.2E1,2.8E1,2.8E1,1.17E2,1.1E1,5.1E1,1.1E1,6E0,2.2E1,2E1,8E0,5.9E1,5.8E1,8E0,3E0,3.7E1,1.4E1,4.8E1,1.1E1,2E0,5.6E1,3.2E1,5E0,5E0,9E0,1.8E1,3E1,5.4E1,2E0,1.2E1,2E1,1.2E1,6E0,2.1E1,9E0,4.8E1,6E0,5E0,1.5E1,2.9E1,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.6167102E-2,-9.1617934E-2,2.0921354E-1,-1.5892246E-1,1.5889113E-4,1.0256023E-1,3.3971918E-1,-2.3020288E-1,-4.6520796E-2,-1.315317E-1,3.3601597E-2,1.9421874E-2,6.8316124E-2,2.9019982E-2,2.54235E-1,-7.909159E-2,-2.7484733E-1,5.3665806E-3,-8.432639E-3,-3.2230811E-3,-1.557175E-2,-2.764732E-3,5.4405066E-3,-5.779646E-2,1.18016526E-1,5.7629147E-3,3.5522467E-1,7.019195E-3,-5.448651E-3,-3.0026987E-1,3.8947738E-4,-5.0486766E-2,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,1.5366653E-1,-2.1028195E-3,1.7452618E-2,3.831069E-3,-2.4670707E-1,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,-1,-1,33,-1,35,37,-1,39,-1,-1,41,-1,43,-1,-1,-1,-1,-1,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.134416E0,1.1242383E0,8.142085E-1,8.305547E-1,3.4793866E-1,3.800948E-1,5.817108E-1,4.100237E-1,3.2077062E-1,1.8154746E-1,2.089919E-1,0E0,2.1365988E-1,0E0,2.3687482E-1,1.9577982E-1,3.561766E-1,2.3537263E-1,0E0,0E0,0E0,1.8856645E-1,0E0,8.858822E-2,1.4960828E-1,0E0,1.08366966E-1,0E0,0E0,1.7012787E-1,0E0,1.1466624E-1,0E0,0E0,0E0,0E0,0E0,9.963548E-2,0E0,0E0,0E0,1.5310645E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,21,21,23,23,24,24,26,26,29,29,31,31,37,37,41,41],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,-1,-1,34,-1,36,38,-1,40,-1,-1,42,-1,44,-1,-1,-1,-1,-1,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,1.9530877E-1,1.80025E2,1.8681856E8,8.9397516E4,1.8681E4,1.433458E10,2.142943E1,2.151E3,2.704375E6,1.9421874E-2,1.479938E6,2.9019982E-2,6.7E1,2.7287993E0,3.402546E6,6.135367E9,-8.432639E-3,-3.2230811E-3,-1.557175E-2,9.143112E5,5.4405066E-3,9.71709E5,2.30273E6,5.7629147E-3,6.3898524E7,7.019195E-3,-5.448651E-3,4.1938826E-2,3.8947738E-4,9.165388E4,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,5.7E1,-2.1028195E-3,1.7452618E-2,3.831069E-3,1.0142518E0,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"split_indices":[54,43,40,54,7,35,9,33,58,2,9,0,34,0,8,56,1,5,0,0,0,35,0,31,9,0,34,0,0,44,0,35,0,0,0,0,0,8,0,0,0,60,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.81E2,6E1,1.04E2,7.7E1,3.4E1,2.6E1,6.3E1,4.1E1,1.5E1,6.2E1,2E0,3.2E1,4E0,2.2E1,1.5E1,4.8E1,3.1E1,1E1,1.3E1,2E0,4.5E1,1.7E1,9E0,2.3E1,1.1E1,1.1E1,2E0,1.3E1,4.4E1,4E0,2.2E1,9E0,2.2E1,2.3E1,2E0,7E0,1.9E1,4E0,9E0,2E0,3.2E1,1.2E1,5E0,1.7E1,7E0,1.2E1,2.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[6.0046404E-3,-9.218389E-2,2.1822302E-1,-1.7119181E-1,2.2936372E-2,1.080785E-1,3.675177E-1,-2.2839919E-1,-7.109087E-2,4.932593E-2,-3.4302478E-3,1.885684E-2,8.236004E-2,1.287323E-1,4.4497648E-1,-2.0051514E-1,-2.1602163E-2,4.6547647E-3,-1.2568788E-1,-1.1251967E-2,4.7163595E-3,1.5830024E-1,1.2744217E-2,9.789093E-3,3.2283113E-4,3.5435382E-1,3.0562386E-2,-1.4268523E-2,-1.7262784E-1,-6.920345E-3,9.3187834E-4,-4.0626727E-2,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,5.3271845E-2,1.711268E-2,5.001436E-3,-1.4809307E-1,-1.3950878E-2,-6.9640996E-3,7.891728E-3,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,29,31,-1,33,35,-1,-1,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8752337E0,1.4591926E0,1.1771345E0,5.277815E-1,1.8073346E-1,3.3109027E-1,5.1609373E-1,3.4681034E-1,3.6278534E-1,1.8984374E-1,0E0,0E0,2.2002146E-1,1.0272367E-1,2.8410387E-1,1.4103603E-1,0E0,0E0,1.3881755E-1,1.4883459E-1,0E0,1.2233335E-1,2.0214176E-1,0E0,0E0,1.2719083E-1,0E0,0E0,1.2584579E-1,0E0,0E0,1.397791E-1,0E0,0E0,0E0,0E0,9.9094376E-2,0E0,0E0,9.3803644E-2,0E0,0E0,9.208519E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,19,19,21,21,22,22,25,25,28,28,31,31,36,36,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,30,32,-1,34,36,-1,-1,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6984555E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.260197E0,8.9397516E4,2.2324745E5,5.6835636E-2,8.938651E4,1.7573398E2,-3.4302478E-3,1.885684E-2,1.3844932E7,2.81002E5,1.2461E4,1.7565872E-3,-2.1602163E-2,4.6547647E-3,1.7725672E8,2.424E3,4.7163595E-3,1.8990228E0,1.312252E6,9.789093E-3,3.2283113E-4,4.7201157E-1,3.0562386E-2,-1.4268523E-2,1.269689E1,-6.920345E-3,9.3187834E-4,8E0,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,2.0215728E0,1.711268E-2,5.001436E-3,8.938651E4,-1.3950878E-2,-6.9640996E-3,2.176974E8,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"split_indices":[54,43,40,54,55,35,30,59,35,54,0,0,47,9,2,60,0,0,12,2,0,55,31,0,0,40,0,0,58,0,0,3,0,0,0,0,37,0,0,35,0,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.59E2,7.3E1,9.4E1,6.5E1,4.3E1,3E1,5.9E1,3.5E1,5.2E1,1.3E1,2E0,4.1E1,8E0,2.2E1,5.5E1,4E0,8E0,2.7E1,2.6E1,2.6E1,1.9E1,2.2E1,4E0,4E0,1.8E1,4E0,8E0,4.7E1,2.2E1,5E0,2.3E1,3E0,1.3E1,6E0,3E0,1.9E1,1.5E1,3E0,4.2E1,5E0,6E0,1.7E1,1.6E1,3E0,1.7E1,2.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.3128728E-2,-7.6841384E-2,2.5555712E-1,-1.3273934E-1,6.3761264E-2,8.655129E-2,4.008202E-1,-1.6426608E-1,-1.455439E-2,1.4814147E-2,3.6547862E-2,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,-1.4727283E-1,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,5.5398457E-2,-4.1562365E-3,-1.9884297E-1,-8.877025E-2,-5.379106E-4,9.418675E-2,-2.5566745E-1,-8.6616755E-2,-7.036469E-4,-1.4556472E-1,5.3083412E-3,-5.8485573E-4,-2.0745803E-1,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,-7.540202E-2,-9.3988655E-3,-2.2761422E-1,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,-1,25,27,-1,29,31,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,-1,43,-1,-1,-1,-1,-1],"loss_changes":[4.1965976E0,1.5646251E0,1.1191728E0,5.266993E-1,4.1526425E-1,1.3371919E-1,5.893326E-1,3.3139777E-1,9.107072E-2,0E0,1.3694459E-1,0E0,0E0,0E0,0E0,2.9913235E-1,0E0,0E0,0E0,1.246268E-1,0E0,3.2860088E-1,2.5498408E-1,0E0,9.024802E-2,4.5705009E-1,9.878099E-2,1.712665E-1,1.3560718E-1,0E0,0E0,1.3404357E-1,0E0,0E0,0E0,0E0,0E0,1.554379E-1,0E0,9.5205545E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,31,31,37,37,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,-1,26,28,-1,30,32,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,-1,44,-1,-1,-1,-1,-1],"split_conditions":[1.4272063E0,3.165913E6,2.174459E5,7.087292E-2,2.207E3,7.002603E1,1.6507233E3,1E0,1.1721691E5,1.4814147E-2,3.605082E7,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,1.73507E5,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,1.462136E6,-4.1562365E-3,1.3738942E0,1.833E5,-5.379106E-4,1.9879582E7,1.0665042E8,1.2288136E1,1.1942446E0,1.2271346E1,5.3083412E-3,-5.8485573E-4,8.672049E9,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,9.3E4,-9.3988655E-3,6.7001674E-3,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"split_indices":[44,1,30,40,0,58,4,77,30,0,47,0,0,0,0,9,0,0,0,34,0,55,31,0,34,7,58,55,58,0,0,5,0,0,0,0,0,12,0,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.97E2,4.6E1,1.41E2,5.6E1,2.2E1,2.4E1,1.11E2,3E1,4E0,5.2E1,1.9E1,3E0,1.6E1,8E0,1.04E2,7E0,5E0,2.5E1,4.6E1,6E0,5.4E1,5E1,1.7E1,2.9E1,3.5E1,1.9E1,2E1,3E1,2.3E1,6E0,3.2E1,3E0,1E1,9E0,3E0,1.7E1,1.6E1,1.4E1,2.9E1,3E0,2E0,1.4E1,8E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.4699904E-3,-8.2270786E-2,1.7413537E-1,-1.3975397E-1,3.89982E-3,7.456945E-2,3.0463025E-1,-1.7296548E-1,6.0730726E-3,-9.173903E-2,3.4571238E-2,1.801554E-2,4.9235556E-2,1.7605738E-3,3.2971406E-1,-2.1693262E-1,-1.0940522E-1,7.686381E-2,-3.4685563E-3,-2.7480236E-1,-8.64333E-4,1.01417145E-5,4.4262665E-3,-7.801436E-2,7.197917E-2,8.4248185E-3,3.7063202E-1,-1.9041932E-1,-2.0984167E-2,-4.680374E-2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,2.399795E-2,3.9364552E-1,4.087419E-3,-1.0795465E-3,-2.0910926E-1,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,5.9234392E-2,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5455477E0,8.9215004E-1,9.777329E-1,5.2410936E-1,2.1730821E-1,3.6530188E-1,2.0546484E-1,2.22188E-1,1.3329768E-1,2.3158492E-1,1.2854296E-1,0E0,1.3006273E-1,0E0,9.619784E-2,2.65311E-1,1.5102664E-1,2.2796245E-1,0E0,1.5092689E-1,0E0,0E0,0E0,1.5338422E-1,1.3331974E-1,0E0,1.03357315E-1,1.4346242E-1,0E0,1.5426993E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3046479E-1,9.462261E-2,0E0,0E0,1.1954689E-1,0E0,0E0,0E0,9.6233115E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,19,19,23,23,24,24,26,26,27,27,29,29,38,38,39,39,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.8509178E-1,1.8829846E-1,2.219883E6,1.8681856E8,8.9397516E4,1E0,5.283708E0,1.8000048E10,8E0,2.069532E6,1.801554E-2,7.5E2,1.7605738E-3,2.2324745E5,5.144033E-2,1.2288136E1,3.131673E6,-3.4685563E-3,3.56E2,-8.64333E-4,1.01417145E-5,4.4262665E-3,3.13E2,8.312613E5,8.4248185E-3,1.8373182E8,6E0,-2.0984167E-2,6.0515753E2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,6.925995E6,1.7598395E3,4.087419E-3,-1.0795465E-3,7.5646E4,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,2.8812434E7,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"split_indices":[54,41,40,1,7,35,18,58,5,3,9,0,0,0,30,59,58,31,0,0,0,0,0,0,49,0,34,10,0,4,0,0,0,0,0,0,0,0,53,4,0,0,9,0,0,0,47,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.79E2,7.6E1,1.07E2,7.2E1,4.4E1,3.2E1,8.7E1,2E1,1.7E1,5.5E1,2E0,4.2E1,3E0,2.9E1,5E1,3.7E1,1.1E1,9E0,4E0,1.3E1,3.7E1,1.8E1,6E0,3.6E1,8E0,2.1E1,4.7E1,3E0,2E1,1.7E1,7E0,4E0,2E0,2E0,2E0,4E0,1.3E1,2.3E1,1.9E1,2E0,5E0,4.2E1,9E0,1.1E1,4E0,1.9E1,1.4E1,5E0,2.4E1,1.8E1,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.0756515E-4,-9.093915E-2,1.3531232E-1,-1.3121241E-1,3.0161556E-2,6.471361E-2,2.7240935E-1,-1.5608981E-1,-4.138077E-2,5.6453147E-3,-1.665546E-3,1.331529E-2,4.4281885E-2,1.4165652E-4,2.9921746E-1,-1.3490759E-1,-3.431053E-1,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,1.077188E-1,-1.1128485E-2,1.87167E-2,1.0148881E-2,-1.18122905E-1,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,5.707247E-2,-4.600261E-3,3.8549118E-2,-1.6461147E-1,-4.9828332E-2,6.6000526E-3,6.4204754E-3,4.2778063E-3,-2.1197742E-3,-2.1401511E-1,-2.0505309E-3,7.0574425E-2,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,35,-1,37,39,41,-1,43,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9934402E0,7.1114016E-1,9.309007E-1,2.3713171E-1,1.1703685E-1,3.1068093E-1,2.37149E-1,2.9043174E-1,1.1233303E-1,0E0,1.0321621E-1,0E0,2.1913269E-1,0E0,1.5956187E-1,2.0051861E-1,2.3705983E-1,0E0,0E0,0E0,0E0,2.0704418E-1,1.6331983E-1,0E0,0E0,2.2356677E-1,0E0,0E0,0E0,0E0,1.08736545E-1,0E0,1.2394792E-1,2.3981667E-1,1.24148294E-1,0E0,9.681947E-2,0E0,0E0,2.0636463E-1,0E0,8.820005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,21,21,22,22,25,25,30,30,32,32,33,33,34,34,36,36,39,39,41,41],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,36,-1,38,40,42,-1,44,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.5343175E0,2.2348747E2,1.0632315E6,8.9397516E4,1E0,2.8659306E0,2.2616908E2,5.6453147E-3,7.432E3,1.331529E-2,1.1572553E7,1.4165652E-4,3.2307692E0,3.0962145E0,1.020435E4,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,4.0012E4,1.4813267E3,1.87167E-2,1.0148881E-2,1.73507E5,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,7E0,-4.600261E-3,8.303E3,2.951829E9,4.4E1,6.6000526E-3,1.9816086E0,4.2778063E-3,-2.1197742E-3,1.42E2,-2.0505309E-3,7.510128E7,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"split_indices":[54,40,43,54,50,35,18,60,54,0,10,0,47,0,56,55,34,0,0,0,0,9,57,0,0,9,0,0,0,0,3,0,0,5,10,0,55,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.44E2,9.7E1,1.08E2,3.6E1,6.5E1,3.2E1,8.4E1,2.4E1,8E0,2.8E1,4E0,6.1E1,3E0,2.9E1,7.7E1,7E0,2E0,2.2E1,2.4E1,4E0,2.8E1,3.3E1,8E0,2.1E1,7.2E1,5E0,3E0,4E0,6E0,2.2E1,1.1E1,2.2E1,4.2E1,3E1,7E0,1.5E1,1.3E1,9E0,2.9E1,1.3E1,6E0,2.4E1,1.1E1,4E0,6E0,2.3E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.0671469E-2,-4.057193E-2,2.5710663E-1,-9.11908E-2,7.657581E-2,1.0005701E-1,3.5594457E-1,-1.3330418E-1,3.5337575E-2,6.0731232E-2,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,-1.425013E-1,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,-2.7416292E-1,-1.2234801E-1,-3.3937553E-3,-2.139736E-2,-1.513492E-1,-3.387177E-2,-1.1739622E-1,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,-8.5462265E-2,-4.7841365E-3,7.8635005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,27,29,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[3.1548471E0,1.2344693E0,6.318512E-1,7.7734864E-1,2.7344996E-1,1.175154E-1,3.9797258E-1,1.8897259E-1,1.1001776E-1,1.5682302E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.4241185E-1,0E0,0E0,0E0,0E0,5.137621E-1,2.3093772E-1,0E0,0E0,1.8179822E-1,9.140951E-2,1.8776286E-1,0E0,0E0,0E0,0E0,1.0951534E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,16,16,21,21,22,22,25,25,26,26,27,27,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,28,30,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.5648934E0,3.165913E6,2.174459E5,6.408734E-2,1.4122614E2,1E0,1.7598395E3,1E1,1.599097E0,1.25440235E1,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,1.0187571E5,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,8.3E1,1.1670399E-1,-3.3937553E-3,-2.139736E-2,3.671916E0,1E0,1E0,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,1.8E1,-4.7841365E-3,7.8635005E-4],"split_indices":[43,1,30,40,58,8,4,10,55,59,0,0,0,0,0,0,30,0,0,0,0,10,41,0,0,56,100,102,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,2.06E2,4.2E1,1.44E2,6.2E1,1.7E1,2.5E1,1.08E2,3.6E1,6E1,2E0,7E0,1E1,1.9E1,6E0,5E0,1.03E2,2.2E1,1.4E1,5E1,1E1,1.2E1,9.1E1,7E0,5E0,6.8E1,2.3E1,5.1E1,1.7E1,2E0,2.1E1,9E0,4.2E1,3.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.4315777E-2,-1.0690338E-1,1.1009746E-1,-1.4777888E-1,2.0821875E-2,2.117857E-2,1.7040308E-1,-1.806778E-1,-5.4927632E-2,2.9963165E-4,6.637665E-3,3.901988E-2,-4.5500887E-3,7.291719E-2,2.6567936E-1,-1.4094864E-1,-2.987772E-1,-2.4235064E-4,-2.5972742E-1,8.2612615E-3,-1.8001318E-2,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,9.182606E-2,1.6551235E-1,2.0925388E-2,-2.1879025E-1,-9.1945045E-2,-3.4581E-1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,-1,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9952228E0,7.809812E-1,5.9160745E-1,3.3525777E-1,9.964019E-2,1.07021526E-1,5.936415E-1,3.473227E-1,3.1202346E-1,1.1755685E-1,0E0,9.129217E-2,0E0,1.14367574E-1,6.274619E-1,2.2401261E-1,1.6717279E-1,0E0,9.88501E-2,0E0,1.3668847E-1,0E0,0E0,0E0,1.5995112E-1,2.5624245E-1,0E0,1.0733318E-1,1.4268285E-1,2.0528078E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,18,18,20,20,24,24,25,25,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,-1,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.1677386E-1,2.796589E6,3.4021906E2,1.1670399E-1,5.5446387E2,1.9E1,2.174459E5,5.740651E-3,3.4047124E-1,2.207E3,6.637665E-3,2.0358758E8,-4.5500887E-3,1.9241573E-1,7.2259143E2,4.949154E-4,5.906883E5,-2.4235064E-4,2.07298E5,8.2612615E-3,1E1,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,8.303E3,1.9816086E0,2.0925388E-2,1.7565872E-3,4.8831225E5,7.5E1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"split_indices":[44,1,54,41,54,3,30,40,44,0,0,7,0,59,54,41,35,0,1,0,3,0,0,0,0,55,0,60,30,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.48E2,1.1E2,1.12E2,3.6E1,4.5E1,6.5E1,8.2E1,3E1,3.2E1,4E0,4E1,5E0,3.3E1,3.2E1,6.3E1,1.9E1,2.5E1,5E0,2E0,3E1,1E1,3E1,3E0,3E1,2.3E1,9E0,2.3E1,4E1,1.5E1,4E0,3E0,2E0,1E1,2E1,2E1,1E1,1.8E1,5E0,8E0,1.5E1,3.1E1,9E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.5204665E-2,-6.9504075E-2,1.5126899E-1,-1.1839543E-1,2.2287078E-2,7.562342E-2,2.5493225E-1,-1.4351347E-1,-1.2403568E-2,-5.980414E-3,3.6651608E-2,5.731069E-2,9.66742E-3,6.0085384E-3,3.3954874E-1,-1.3105348E-2,-1.2924905E-1,3.58532E-2,-7.821619E-3,1.6858906E-2,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,-9.744661E-2,-1.9808635E-1,4.93121E-3,-2.631804E-3,3.7851118E-2,-3.5518215E-3,-1.1887726E-1,1.7840436E-3,-5.2316664E-3,-2.911272E-1,6.4006194E-2,-1.814889E-3,-1.4108726E-1,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.0247637E-1,-1.5689014E-1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1],"loss_changes":[2.3878818E0,8.9570975E-1,4.904294E-1,3.4431982E-1,1.6523217E-1,9.151134E-2,2.2111058E-1,1.9603395E-1,2.1574554E-1,0E0,1.2080229E-1,1.1728562E-1,0E0,0E0,9.910309E-2,0E0,1.9523811E-1,1.6260572E-1,0E0,1.1651751E-1,0E0,0E0,0E0,0E0,0E0,2.094084E-1,1.8337464E-1,0E0,0E0,9.8238766E-2,0E0,1.4792949E-1,0E0,0E0,2.5384462E-1,1.126506E-1,0E0,1.1237788E-1,0E0,0E0,0E0,0E0,8.794698E-2,1.0356009E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,16,16,17,17,19,19,25,25,26,26,29,29,31,31,34,34,35,35,37,37,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1],"split_conditions":[1.2185057E0,1.781973E6,2.0787016E5,6.808926E6,9.315286E1,4.328683E0,3.778E3,3.4903356E7,1.409674E7,-5.980414E-3,4.43001E5,1.1832633E0,9.66742E-3,6.0085384E-3,4.7201157E-1,-1.3105348E-2,4.352025E0,4.8593006E5,-7.821619E-3,1.939974E7,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,3.9376082E0,8.287425E6,4.93121E-3,-2.631804E-3,1.25440235E1,-3.5518215E-3,2.2E1,1.7840436E-3,-5.2316664E-3,4.752366E0,5.6024827E9,-1.814889E-3,2.427E3,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.1770233E0,4.2E1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"split_indices":[43,1,30,9,54,37,2,7,47,0,30,41,0,0,40,0,56,35,0,34,0,0,0,0,0,56,47,0,0,59,0,3,0,0,58,5,0,2,0,0,0,0,56,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.62E2,1.98E2,6.4E1,1.29E2,6.9E1,3.8E1,2.6E1,1.04E2,2.5E1,5E0,6.4E1,3.5E1,3E0,1.2E1,1.4E1,7E0,9.7E1,2E1,5E0,5.4E1,1E1,8E0,2.7E1,1.2E1,2E0,6.8E1,2.9E1,1.1E1,9E0,4.5E1,9E0,5.9E1,9E0,1.7E1,1.2E1,3.4E1,1.1E1,4.9E1,1E1,4E0,8E0,1.1E1,2.3E1,4.4E1,5E0,3E0,2E1,3.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.1777132E-3,-6.0567748E-2,1.9037953E-1,-1.3071354E-1,5.439912E-4,8.848939E-2,2.9270196E-1,-1.7778724E-1,-1.4740403E-3,-7.950044E-2,3.19054E-2,3.1027812E-4,1.2356255E-1,2.2399522E-2,2.3053867E-1,-1.3397929E-1,-2.7813753E-1,1.4052213E-4,-1.3808084E-1,7.668798E-2,-2.3035506E-2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.975542E0,8.102633E-1,6.309147E-1,3.9554834E-1,2.5858772E-1,9.331095E-2,3.067577E-1,2.2210789E-1,0E0,1.4097291E-1,1.838358E-1,0E0,1.22035086E-1,0E0,9.3972206E-2,2.5386703E-1,9.3033075E-2,0E0,1.3659179E-1,1.5234712E-1,1.16067775E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,1.9415865E2,2.174459E5,1.848519E-2,8.461558E10,5.958114E4,1.8681E4,1.607E3,-1.4740403E-3,1.2271346E1,2.305287E1,3.1027812E-4,1.718354E8,2.2399522E-2,3.778E3,6.581686E6,6.608892E2,1.4052213E-4,4.6955062E5,1.2119408E3,6.657047E2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"split_indices":[44,54,30,40,33,30,9,2,0,58,58,0,7,0,2,9,4,0,35,4,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.88E2,6.2E1,8.7E1,1.01E2,3.2E1,3E1,5.8E1,2.9E1,2.8E1,7.3E1,1E1,2.2E1,5E0,2.5E1,4.2E1,1.6E1,1.2E1,1.6E1,4E1,3.3E1,6E0,1.6E1,1.1E1,1.4E1,4E1,2E0,1.2E1,4E0,4E0,1.2E1,3E1,1E1,2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.9849198E-3,-6.908789E-2,1.1069185E-1,-1.1135524E-1,4.8647843E-2,1.8852478E-2,1.622326E-1,-9.491666E-2,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,5.3762157E-2,2.2892316E-1,6.1087083E-2,-1.4048992E-1,-5.2776527E-2,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,-1.5545653E-1,-1.0782661E-3,-8.035333E-3,-2.0769975E-1,-9.872578E-2,-2.4276859E-1,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1],"loss_changes":[1.8784683E0,7.269919E-1,4.5608807E-1,2.1379781E-1,3.8304985E-1,1.0936768E-1,4.042337E-1,1.7887038E-1,0E0,0E0,0E0,0E0,9.703164E-2,2.1851015E-1,1.605283E-1,2.029301E-1,1.871668E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.09901786E-1,0E0,0E0,1.7119968E-1,1.0259637E-1,1.2492943E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,14,14,15,15,16,16,24,24,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1],"split_conditions":[4.259334E-1,3.343327E6,3.1259552E2,3.7979183E0,2.207E3,2.1123085E8,1.0568718E7,1.3466058E0,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,1.6070912E0,1.2461E4,1.25440235E1,4.6376264E9,2.2733334E1,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,6.7329E4,-1.0782661E-3,-8.035333E-3,8.640031E9,1.1710652E0,4.19891E0,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"split_indices":[44,1,54,55,0,7,47,55,0,0,0,0,60,2,59,33,58,0,0,0,0,0,0,0,9,0,0,5,55,58,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.44E2,9.6E1,1.06E2,3.8E1,3.5E1,6.1E1,9.6E1,1E1,3E0,3.5E1,1E1,2.5E1,3.6E1,2.5E1,4.5E1,5.1E1,3E0,2.2E1,3.2E1,4E0,2E1,5E0,2E0,4.3E1,4.3E1,8E0,2.1E1,2.2E1,1.8E1,3E0,8E0,1.4E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0547296E-2,-4.2344254E-2,1.742515E-1,-1.0237627E-1,3.5556585E-2,2.740988E-1,7.519926E-2,-7.69858E-2,-2.2364104E-1,7.678524E-3,2.2331232E-2,1.4626542E-1,3.7681144E-1,4.2196088E-2,8.162208E-3,-4.9231295E-2,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,4.149654E-2,-7.011419E-4,9.022605E-3,4.7000867E-1,6.24682E-3,-3.9323145E-3,3.069897E-3,-7.009031E-2,6.090464E-2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.163362E0,8.8750696E-1,5.826844E-1,3.122326E-1,1.5018767E-1,3.3878827E-1,1.1022253E-1,2.4382734E-1,2.6022464E-1,0E0,1.0540541E-1,1.5061551E-1,2.8114796E-1,1.0317968E-1,0E0,1.6967641E-1,0E0,0E0,0E0,0E0,1.09696545E-1,0E0,0E0,9.1567755E-2,0E0,0E0,0E0,1.7041293E-1,1.1754744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,20,20,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.5971466E2,2.7558826E-2,9.8110194E5,1.2460274E-2,2.3755E4,7.116346E2,2.0787016E5,3.774648E0,5.906883E5,7.678524E-3,1.496E3,2.4894228E6,2.660842E6,1.5E1,8.162208E-3,3.901529E6,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,1.6815423E5,-7.011419E-4,9.022605E-3,5.505334E9,6.24682E-3,-3.9323145E-3,3.069897E-3,2.797224E2,2.1374529E2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"split_indices":[54,40,35,40,9,54,30,56,35,0,2,34,31,3,0,9,0,0,0,0,30,0,0,5,0,0,0,54,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,1.88E2,6E1,1.06E2,8.2E1,2.9E1,3.1E1,8.9E1,1.7E1,6E0,7.6E1,1.4E1,1.5E1,2.5E1,6E0,7.1E1,1.8E1,8E0,9E0,1.6E1,6E1,4E0,1E1,1E1,5E0,4E0,2.1E1,6E1,1.1E1,2.2E1,3.8E1,6E0,4E0,5.4E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.9339443E-3,-1.0605051E-1,7.4423306E-2,-1.4161547E-1,-2.8023109E-2,5.737777E-2,3.6554796E-1,-1.08674295E-1,-2.4172173E-1,-5.5312406E-2,5.1337723E-3,2.0626027E-2,1.1801036E-1,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,-2.894412E-1,4.924759E-4,-5.1691607E-3,-2.6259364E-3,4.9695414E-2,4.6624467E-4,1.5626377E-1,-1.5225716E-2,-4.396114E-3,1.359324E-1,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.0659106E0,3.0406094E-1,7.134572E-1,2.243501E-1,1.496107E-1,3.117638E-1,1.7670429E-1,1.5272987E-1,2.0493579E-1,1.3169232E-1,0E0,2.1203475E-1,2.1592683E-1,0E0,0E0,0E0,0E0,0E0,1.3315952E-1,0E0,0E0,0E0,1.3145865E-1,0E0,1.2719214E-1,0E0,0E0,8.859834E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,18,18,22,22,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.7613442E-1,2.8704E4,4.328683E0,5.1653096E-3,1.22E2,1.2446029E0,2.174459E5,1.0665042E8,3.7274265E2,2.8488008E11,5.1337723E-3,1.805E3,1.7468556E2,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,2.1710082E7,4.924759E-4,-5.1691607E-3,-2.6259364E-3,1.325E0,4.6624467E-4,1.1560919E3,-1.5225716E-2,-4.396114E-3,8.8356856E5,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"split_indices":[43,11,37,40,8,44,30,7,4,33,0,2,54,0,0,0,0,0,34,0,0,0,55,0,54,0,0,49,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.1E2,1.47E2,7.5E1,3.5E1,1.4E2,7E0,5.8E1,1.7E1,3E1,5E0,8.8E1,5.2E1,4E0,3E0,5.3E1,5E0,3E0,1.4E1,1.5E1,1.5E1,2.3E1,6.5E1,1.4E1,3.8E1,1E1,4E0,1.3E1,5.2E1,3.5E1,3E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.345121E-3,-6.562803E-2,1.1551625E-1,-8.816371E-2,4.981665E-4,3.869827E-2,2.208219E-1,-1.4935678E-1,-4.5134716E-2,1.6114242E-2,6.035789E-3,2.2871085E-2,1.6961017E-1,-2.867197E-1,-1.2161767E-1,-6.569096E-2,3.5917396E-3,-3.0824807E-3,5.086014E-2,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,-1.0346314E-1,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.6731131E0,2.8574848E-1,5.9006786E-1,3.276595E-1,0E0,9.675721E-2,3.9257836E-1,1.6326678E-1,2.0450398E-1,1.1511861E-1,0E0,0E0,1.3534534E-1,3.4041297E-1,1.645894E-1,9.34082E-2,0E0,0E0,1.10176116E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0852566E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[3.7136658E2,9.1318935E-2,1.8829846E-1,4.426498E0,4.981665E-4,1E0,1.431E4,9.975593E4,6.767991E5,1.312252E6,6.035789E-3,2.2871085E-2,1.1207349E0,2.83E2,5.22241E5,1.7012288E8,3.5917396E-3,-3.0824807E-3,8.303E3,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,2.22105E5,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"split_indices":[54,40,40,58,0,105,9,30,30,31,0,0,55,0,12,7,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.63E2,7.3E1,1.26E2,3.7E1,4.3E1,3E1,5.1E1,7.5E1,3.6E1,7E0,3E0,2.7E1,7E0,4.4E1,6.5E1,1E1,1E1,2.6E1,3E0,2.4E1,4E0,3E0,2.6E1,1.8E1,3.3E1,3.2E1,1.7E1,9E0,2E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.5972516E-3,-6.415109E-2,1.004628E-1,-9.126678E-2,7.985584E-5,7.822119E-2,3.8027394E-1,-1.3547492E-1,-1.7177572E-2,1.2492152E-1,-1.8076582E-2,2.4130752E-2,6.7940415E-3,-2.0095895E-1,-7.504207E-2,3.5000306E-3,-4.5314096E-2,2.2494642E-1,8.771981E-2,-6.7835734E-3,8.8194094E-4,-1.3448668E-1,-3.6327535E-1,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,-8.5903674E-2,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[1.4975804E0,2.8284806E-1,4.9924803E-1,3.645115E-1,0E0,3.6202925E-1,1.4661348E-1,2.6193678E-1,1.21453926E-1,1.8120694E-1,1.4514734E-1,0E0,0E0,3.07199E-1,2.7256477E-1,0E0,1.0324016E-1,1.1432767E-1,8.934039E-2,0E0,0E0,1.8839267E-1,1.3768709E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.900878E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,17,17,18,18,21,21,22,22,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[3.4021906E2,5.530166E-2,2.069929E3,1.6408995E11,7.985584E-5,1.8018788E7,5.1145615E3,1.386452E0,2.6017979E11,8E0,1.628E3,2.4130752E-2,6.7940415E-3,1.609E3,3.9069E5,3.5000306E-3,1E0,4.0123675E-2,4.0012E4,-6.7835734E-3,8.8194094E-4,1.393E3,4.7201145E10,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,1.1710652E0,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"split_indices":[54,40,54,33,0,47,4,55,33,3,0,0,0,2,31,0,101,40,9,0,0,12,33,0,0,0,0,0,0,0,0,0,55,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.57E2,8.4E1,1.11E2,4.6E1,7.9E1,5E0,6.9E1,4.2E1,5.3E1,2.6E1,2E0,3E0,3.2E1,3.7E1,9E0,3.3E1,1.3E1,4E1,5E0,2.1E1,2.4E1,8E0,3.1E1,6E0,2.5E1,8E0,2E0,1.1E1,9E0,3.1E1,4E0,2E1,4E0,4E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.0949485E-4,-7.023364E-2,1.07531965E-1,-9.5697366E-2,5.999637E-3,4.481289E-2,2.2272988E-1,-1.2741327E-1,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,3.131387E-2,6.709275E-3,1.5677929E-2,-1.9639476E-1,-7.4869595E-2,-1.0507656E-2,5.4045306E-3,-2.6656508E-1,-3.303782E-3,-3.9352458E-2,-1.0309419E-2,-5.7465717E-2,3.690726E-2,-8.487278E-3,-2.1258341E-2,-5.933776E-2,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,19,21,23,-1,25,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9060686E0,2.956339E-1,7.1252954E-1,3.6945975E-1,9.6911065E-2,2.3330902E-1,2.7442348E-1,3.0063093E-1,0E0,0E0,0E0,0E0,2.4897242E-1,0E0,0E0,2.9316032E-1,2.830122E-1,1.0231397E-1,0E0,2.7091873E-1,0E0,9.987865E-2,0E0,1.0716895E-1,1.2748241E-1,0E0,0E0,1.180381E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,15,15,16,16,17,17,19,19,21,21,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,20,22,24,-1,26,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.2711891E0,1.769817E6,9.750871E6,3.4903356E7,1.6507233E3,7.5646E4,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,2.8041378E5,6.709275E-3,1.5677929E-2,1.3738942E0,3.0962145E0,1.0685258E6,5.4045306E-3,6.4146477E-3,-3.303782E-3,2.0766662E6,-1.0309419E-2,2.0250492E0,2.51838E6,-8.487278E-3,-2.1258341E-2,4.6376264E9,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"split_indices":[54,40,41,1,9,7,4,9,0,0,0,0,30,0,0,55,55,35,0,40,0,50,0,59,34,0,0,33,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.51E2,9.9E1,1.13E2,3.8E1,6.5E1,3.4E1,8.6E1,2.7E1,2.8E1,1E1,2E0,6.3E1,2.4E1,1E1,3.6E1,5E1,4.4E1,1.9E1,2.2E1,1.4E1,4.2E1,8E0,2.2E1,2.2E1,1.8E1,4E0,3.6E1,6E0,1.6E1,6E0,9E0,1.3E1,4E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.23015465E-2,-3.6813665E-2,1.6186102E-1,-8.2913704E-2,3.2519907E-2,3.2985918E-3,2.6083997E-1,-9.4713494E-2,1.0172514E-3,2.072838E-2,1.34672765E-2,7.684792E-3,4.2016974E-1,-1.21286646E-1,-3.079713E-2,8.215429E-2,-2.6565844E-2,8.788486E-3,2.4215858E-2,-3.9600227E-3,-1.8518592E-1,1.9243775E-3,-4.5282864E-3,6.782496E-3,1.653277E-2,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8129876E0,5.9720445E-1,5.011734E-1,1.4259148E-1,2.3776487E-1,0E0,3.040551E-1,1.6780907E-1,0E0,2.1425411E-1,0E0,0E0,1.535585E-1,1.2175703E-1,1.739935E-1,1.5128689E-1,8.827151E-2,0E0,0E0,0E0,9.1700196E-2,0E0,0E0,0E0,9.928738E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,12,12,13,13,14,14,15,15,16,16,20,20,24,24],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,2.427E3,2.2324745E5,6.4796996E-1,1.4122614E2,3.2985918E-3,1.7598395E3,2.0256741E2,1.0172514E-3,1.3844932E7,1.34672765E-2,7.684792E-3,2.4394053E9,1.607E3,7.336684E2,1.6254681E0,6.925995E6,8.788486E-3,2.4215858E-2,-3.9600227E-3,1.7842995E8,1.9243775E-3,-4.5282864E-3,6.782496E-3,4.488E3,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"split_indices":[44,2,30,41,58,0,4,54,0,47,0,0,5,2,4,55,53,0,0,0,7,0,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.85E2,6E1,1.11E2,7.4E1,3.3E1,2.7E1,1E2,1.1E1,7.2E1,2E0,1.9E1,8E0,7E1,3E1,3.1E1,4.1E1,4E0,4E0,4.9E1,2.1E1,1.5E1,1.5E1,1.4E1,1.7E1,2.5E1,1.6E1,1.3E1,8E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.4086864E-3,-5.6603998E-2,1.0975099E-1,-1.0331851E-1,7.5110937E-3,8.584382E-2,1.7369809E-2,-1.5483831E-1,-2.4599966E-2,-6.041252E-3,1.6805973E-2,4.8704322E-2,1.6501543E-1,-1.18140824E-1,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.352543E-2,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.7069752E-2,3.0004673E-3,-3.8697182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.617809E0,4.5603204E-1,6.2057626E-1,3.530805E-1,9.050379E-2,2.5365812E-1,0E0,3.3182883E-1,1.1731397E-1,0E0,1.0263331E-1,2.5914997E-1,2.4869812E-1,9.6805155E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2964478E-1,0E0,0E0,0E0,0E0,0E0,8.78565E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,20,20,26,26],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[3.4021906E2,1.1670399E-1,1.3821641E0,1.6796057E2,8.89E2,2.968068E5,1.7369809E-2,7.33452E-3,2.939566E5,-6.041252E-3,1.10018056E8,8.9397516E4,2.3842E4,6E0,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.0327642E-1,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.615313E3,3.0004673E-3,-3.8697182E-3],"split_indices":[54,41,36,54,2,30,0,40,35,0,34,35,9,10,0,0,0,0,0,0,44,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,8.7E1,6.4E1,8.8E1,6E0,5.2E1,3.5E1,3E0,6.1E1,6.1E1,2.7E1,4.5E1,7E0,1.7E1,1.8E1,5.5E1,6E0,2E0,5.9E1,5E0,2.2E1,5E0,4E1,1.2E1,4.7E1,4.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.5817066E-3,-3.926012E-2,1.1888153E-1,-7.5602576E-2,5.771031E-2,6.1283264E-2,8.008875E-3,-9.3968615E-2,8.966747E-3,1.21591035E-2,3.8098045E-2,8.102908E-3,1.4816463E-3,-1.2472329E-1,-4.609013E-2,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,-1.0609691E-1,-2.3297147E-1,-7.477302E-3,-1.8048729E-1,-1.9920397E-1,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0151035E0,6.5632963E-1,1.823703E-1,2.1052504E-1,2.1035415E-1,9.229251E-2,0E0,1.576975E-1,1.7606108E-1,0E0,1.1426413E-1,0E0,0E0,1.1037278E-1,2.3217589E-1,0E0,0E0,0E0,0E0,1.10173464E-1,9.862223E-2,1.0037966E-1,1.288043E-1,1.144903E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1427817E0,3.165913E6,2.174459E5,5.8536225E5,2.207E3,1.1832633E0,8.008875E-3,1.1670399E-1,3.2744165E2,1.21591035E-2,1E1,8.102908E-3,1.4816463E-3,1.4031231E-2,2.587E3,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,1.1408451E0,5.1607388E5,2.8507206E-1,7.12E2,7.5608E4,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"split_indices":[43,1,30,30,0,41,0,41,4,0,3,0,0,40,2,0,0,0,0,55,35,44,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,1.84E2,5.1E1,1.34E2,5E1,2.8E1,2.3E1,1.1E2,2.4E1,3E0,4.7E1,4E0,2.4E1,6.6E1,4.4E1,9E0,1.5E1,1.4E1,3.3E1,5.8E1,8E0,3.5E1,9E0,1E1,4.8E1,5E0,3E0,2E1,1.5E1,2E0,7E0,8E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-6.5358323E-3,-5.01191E-2,1.1463253E-1,-6.566863E-2,3.5350563E-3,6.801129E-2,2.2740725E-1,-9.271781E-2,1.6208226E-4,3.614062E-3,4.4614673E-3,2.6938426E-1,5.7348673E-4,-7.9707764E-2,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,-1.20486856E-1,-4.1228443E-2,-2.9383153E-1,-9.8548844E-2,3.4898527E-3,-6.404513E-2,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2573216E0,3.6213574E-1,3.1576872E-1,2.9499376E-1,0E0,1.0101345E-1,1.5835595E-1,1.3562757E-1,0E0,1.0294547E-1,0E0,1.21626854E-1,0E0,1.5508604E-1,0E0,0E0,0E0,0E0,0E0,1.5738219E-1,1.544519E-1,1.2428334E-1,2.1362385E-1,0E0,1.0190545E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,5.8536225E5,5.76E3,3.165913E6,3.5350563E-3,1.7468556E2,3.41212E7,1E0,1.6208226E-4,1.2495134E6,4.4614673E-3,2.9375613E-1,5.7348673E-4,1.73507E5,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,9.675635E4,1.2473118E0,9.48E2,4.43001E5,3.4898527E-3,1.8979991E6,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"split_indices":[44,30,2,1,0,54,53,85,0,47,0,40,0,9,0,0,0,0,0,30,55,0,30,0,50,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.74E2,6.2E1,1.56E2,1.8E1,4.5E1,1.7E1,1.12E2,4.4E1,1.6E1,2.9E1,1.4E1,3E0,1.01E2,1.1E1,9E0,7E0,6E0,8E0,4.8E1,5.3E1,4E0,4.4E1,8E0,4.5E1,2E0,2E0,3.8E1,6E0,3.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.814724E-3,-4.223491E-2,1.1547112E-1,-8.021347E-2,1.0189096E-3,1.424386E-1,-2.2660806E-3,-5.76489E-2,-2.370273E-1,-5.7016924E-2,1.174304E-3,-1.1189182E-3,1.5711717E-1,-3.7425077E-3,2.5416018E-2,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.0735268E-1,2.2521736E-1,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.3238825E-1,3.9426633E-3,2.2386394E-2,9.90202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.2799627E0,2.77701E-1,3.414572E-1,3.0479503E-1,1.2174119E-1,1.6192663E-1,0E0,1.8898878E-1,1.3618523E-1,2.1964492E-1,0E0,0E0,1.7577171E-1,0E0,9.3134895E-2,0E0,0E0,0E0,0E0,1.2113613E-1,3.1686854E-1,0E0,0E0,0E0,0E0,1.3124907E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,14,14,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[3.799535E2,1.1306552E-1,4.183787E0,1.4031231E-2,1.83437E5,1.2055675E0,-2.2660806E-3,4.239E3,1.8018788E7,1.5086387E8,1.174304E-3,-1.1189182E-3,7.3667456E2,-3.7425077E-3,1.7E1,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.57257E0,3.21879E6,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.0256798E0,3.9426633E-3,2.2386394E-2,9.90202E-3],"split_indices":[54,41,59,40,9,55,0,10,47,7,0,0,54,0,3,0,0,0,0,55,31,0,0,0,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.68E2,7.3E1,8.9E1,7.9E1,6.3E1,1E1,7.9E1,1E1,2.4E1,5.5E1,5E0,5.8E1,5.9E1,2E1,5E0,5E0,2E0,2.2E1,3.5E1,2.3E1,1.1E1,9E0,1.2E1,2.3E1,1.2E1,1.1E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.0411531E-3,-5.0822284E-2,8.184148E-2,-8.0308676E-2,-6.2485766E-3,1.5205893E-1,6.5722945E-3,-1.08080305E-1,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,2.7616316E-1,-1.5828831E-2,1.3877054E-2,-8.242552E-2,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,-1.7038673E-1,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[9.903048E-1,2.005888E-1,4.6938437E-1,2.0104021E-1,1.332165E-1,2.687695E-1,3.1198943E-1,1.609565E-1,0E0,0E0,0E0,0E0,1.3902038E-1,9.690636E-2,0E0,1.4167267E-1,0E0,0E0,0E0,0E0,0E0,1.059061E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,15,15,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[3.2769293E2,1.529448E-1,1.0568718E7,5.5015685E9,9.74979E6,7.115615E2,1.4122614E2,1.3244295E5,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,1.8373182E8,1.128757E6,1.3877054E-2,6.7001674E-3,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,2.647867E-4,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"split_indices":[54,41,47,5,9,54,58,49,0,0,0,0,34,31,0,60,0,0,0,0,0,41,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.52E2,8.8E1,9.1E1,6.1E1,4.5E1,4.3E1,6.7E1,2.4E1,4.7E1,1.4E1,3.3E1,1.2E1,4.1E1,2E0,5.5E1,1.2E1,1E1,2E0,1E1,3.1E1,1.3E1,4.2E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.606068E-4,-4.0152058E-2,1.3688791E-1,-7.5740196E-2,5.956878E-3,1.0230381E-1,3.1381202E-1,-9.340738E-2,3.2788247E-2,6.146593E-3,-6.0711894E-3,9.786554E-3,6.620768E-2,1.6971182E-2,1.4838147E-3,-6.620601E-2,-1.9074214E-1,5.796871E-3,-1.0807036E-3,-2.5402438E-2,2.655493E-3,-2.4867726E-3,8.962364E-2,-1.7166455E-1,-3.69803E-2,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,25,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.397829E0,3.1715095E-1,3.2278395E-1,2.120583E-1,1.3876921E-1,2.0526332E-1,1.7587048E-1,2.362498E-1,9.541947E-2,0E0,1.03748016E-1,0E0,1.20494455E-1,0E0,0E0,2.2480932E-1,2.1755302E-1,0E0,0E0,1.3877995E-1,0E0,0E0,1.2990308E-1,1.283262E-1,1.8078491E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,15,15,16,16,19,19,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,26,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.633287E2,2.0358758E8,1.9292948E3,3.402546E6,2.3755E4,2.02983E5,3.41212E7,1.2869189E5,1.8000048E10,6.146593E-3,1.9425332E7,9.786554E-3,1.9241573E-1,1.6971182E-2,1.4838147E-3,1E0,9.975593E4,5.796871E-3,-1.0807036E-3,4.327E3,2.655493E-3,-2.4867726E-3,3.3996282E0,1.9E1,1.081E3,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"split_indices":[54,7,54,1,9,31,53,49,5,0,9,0,59,0,0,102,30,0,0,11,0,0,59,3,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.92E2,5.7E1,1.08E2,8.4E1,4.9E1,8E0,9.3E1,1.5E1,6E0,7.8E1,1E1,3.9E1,6E0,2E0,7.4E1,1.9E1,5E0,1E1,6.1E1,1.7E1,6E0,3.3E1,1.5E1,5.9E1,3E0,1.6E1,1.6E1,4.5E1,2.1E1,1.2E1,6E0,9E0,2.2E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.4908163E-3,-4.9458664E-2,8.817324E-2,6.931918E-2,-6.1066084E-2,6.9811605E-2,3.090321E-1,-5.98609E-4,7.956278E-3,-8.879114E-3,-5.08166E-2,1.4328934E-1,3.887047E-2,1.7160837E-2,2.8655254E-3,-1.0146142E-1,-3.0556865E-2,1.2192971E-3,2.755345E-1,8.296684E-2,-6.053145E-4,-6.862668E-2,-1.2999088E-2,1.7750489E-3,-5.1321223E-2,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,-1.3914266E-1,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,17,19,-1,-1,21,23,-1,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[1.1073979E0,2.1379605E-1,3.6368233E-1,1.3386652E-1,1.9253713E-1,1.9677043E-1,1.176427E-1,0E0,0E0,0E0,1.3131228E-1,3.889135E-1,1.4999606E-1,0E0,0E0,2.181817E-1,1.4260218E-1,0E0,1.17243946E-1,9.568061E-2,0E0,1.2064642E-1,0E0,0E0,8.933057E-2,0E0,0E0,0E0,0E0,1.0965279E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,11,11,12,12,15,15,16,16,18,18,19,19,21,21,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,18,20,-1,-1,22,24,-1,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[3.2769293E2,1.1623532E6,1.3758811E0,1E0,1.7565872E-3,2.3842E4,6.901E3,-5.98609E-4,7.956278E-3,-8.879114E-3,4.0835E4,7.6411234E4,9.380036E6,1.7160837E-2,2.8655254E-3,2.9138212E0,1.217759E0,1.2192971E-3,8.7292106E5,1.9643908E5,-6.053145E-4,5.3091907E2,-1.2999088E-2,1.7750489E-3,3.0962145E0,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,1.0288912E5,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"split_indices":[54,47,36,8,60,9,0,0,0,0,9,30,47,0,0,60,55,0,35,30,0,4,0,0,55,0,0,0,0,35,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,1.3E1,1.38E2,8.8E1,6E0,8E0,5E0,8E0,1.3E2,2.5E1,6.3E1,4E0,2E0,3.6E1,9.4E1,1.4E1,1.1E1,3.4E1,2.9E1,3.2E1,4E0,2.1E1,7.3E1,3E0,8E0,6E0,2.8E1,1.3E1,1.9E1,6.4E1,9E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.576465E-3,-3.1257384E-2,1.1510225E-1,-6.0538724E-2,4.1183308E-2,1.9610284E-2,9.548955E-2,-1.616713E-1,-4.8437126E-2,1.07467435E-2,2.4615793E-2,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,-3.3256467E-2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,-4.011636E-2,4.8860135E-3,-9.835392E-2,-1.4123669E-2,7.2520383E-4,-1.8231265E-1,7.876769E-5,-6.9828774E-3,-2.5798842E-1,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[9.6316123E-1,4.0339252E-1,3.4389424E-1,1.5739092E-1,1.8081136E-1,0E0,1.22983694E-1,1.2277371E-1,1.5455526E-1,0E0,1.18286856E-1,0E0,0E0,0E0,0E0,1.0909271E-1,0E0,0E0,0E0,1.5149972E-1,0E0,3.022702E-1,1.6528234E-1,0E0,1.5847087E-1,0E0,0E0,1.461187E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.2185057E0,3.165913E6,1E0,6.7001674E-3,2.207E3,1.9610284E-2,2.0787016E5,2.647867E-4,2.0017123E0,1.07467435E-2,1E1,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,4.5971466E2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,4.0054E4,4.8860135E-3,1.8259804E-1,2.2912737E1,7.2520383E-4,1.3738942E0,7.876769E-5,-6.9828774E-3,1.622E3,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"split_indices":[44,1,13,60,0,0,30,41,55,0,3,0,0,0,0,54,0,0,0,9,0,60,58,0,55,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.88E2,5.8E1,1.34E2,5.4E1,2E0,5.6E1,1.3E1,1.21E2,3E0,5.1E1,3.2E1,2.4E1,9E0,4E0,1.04E2,1.7E1,1.5E1,3.6E1,1E2,4E0,3E1,7E1,1.3E1,1.7E1,6.4E1,6E0,1E1,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.720544E-3,-4.5713637E-2,8.041132E-2,4.5472946E-2,-5.675482E-2,4.4581052E-2,1.6859385E-1,-1.5774339E-3,5.743392E-3,-6.856326E-2,2.9137306E-4,8.072134E-3,3.0883143E-2,1.6059604E-2,5.5405004E-3,-6.1498176E-2,-1.2888393E-2,7.4021585E-2,-2.9413155E-3,-9.75898E-3,-5.4604717E-2,7.0612812E-3,1.7218302E-3,-2.8861985E-3,3.3276238E-2,-7.160593E-3,-4.491236E-2,-1.0041681E-3,3.9801714E-3,4.2815937E-4,-6.328989E-2,-7.997919E-3,-2.0749776E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,17,-1,-1,19,-1,21,23,-1,25,-1,-1,-1,27,-1,29,-1,-1,-1,31,-1,-1],"loss_changes":[9.9902195E-1,1.4655435E-1,3.463596E-1,1.1724831E-1,9.633365E-2,1.5005943E-1,2.1894133E-1,0E0,0E0,1.5487564E-1,0E0,0E0,1.096528E-1,0E0,0E0,1.0625979E-1,0E0,9.375355E-2,1.00620806E-1,0E0,1.0192835E-1,0E0,0E0,0E0,9.3399785E-2,0E0,9.716697E-2,0E0,0E0,0E0,1.2727156E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,12,12,15,15,17,17,18,18,20,20,24,24,26,26,30,30],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,18,-1,-1,20,-1,22,24,-1,26,-1,-1,-1,28,-1,30,-1,-1,-1,32,-1,-1],"split_conditions":[2.797224E2,1.1696849E6,1.5972575E0,2.25E2,4.343271E5,8.9397516E4,1.8426E4,-1.5774339E-3,5.743392E-3,2.9892595E6,2.9137306E-4,8.072134E-3,1.1572553E7,1.6059604E-2,5.5405004E-3,1.3525729E6,-1.2888393E-2,4.0012E4,1.4813267E3,-9.75898E-3,1.7565872E-3,7.0612812E-3,1.7218302E-3,-2.8861985E-3,5.415429E4,-7.160593E-3,1.1710652E0,-1.0041681E-3,3.9801714E-3,4.2815937E-4,1E0,-7.997919E-3,-2.0749776E-3],"split_indices":[54,47,43,0,49,35,9,0,0,53,0,0,47,0,0,47,0,9,57,0,60,0,0,0,30,0,55,0,0,0,68,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,1.5E1,1.27E2,8E1,3.1E1,8E0,7E0,1.07E2,2E1,6E0,7.4E1,4E0,2.7E1,1.05E2,2E0,3.2E1,4.2E1,3E0,1.02E2,8E0,2.4E1,1.5E1,2.7E1,7E0,9.5E1,1.4E1,1.3E1,2.4E1,7.1E1,7E0,6.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.432755E-3,-3.2541275E-2,1.2062466E-1,-6.6113114E-2,7.3056045E-4,7.963434E-2,2.6087046E-1,-2.4810357E-2,-1.07705206E-1,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,-1.1733579E-1,-3.5330087E-3,-1.7337382E-1,-5.0983664E-2,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,3.909637E-2,-2.709494E-1,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0041343E0,3.2042485E-1,2.954054E-1,1.9488263E-1,0E0,1.0474539E-1,2.4700874E-1,1.15738496E-1,2.0347583E-1,0E0,0E0,0E0,0E0,1.0116187E-1,1.1524878E-1,1.1706191E-1,1.1447847E-1,0E0,0E0,0E0,1.1847381E-1,1.464771E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15,16,16,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.3341238E9,2.2973125E3,1.19331E5,7.3056045E-4,2.0787016E5,2.174459E5,3.559166E7,6.6037946E2,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,4.3399894E5,1.09E3,7.359468E8,1.48068905E1,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,1.2765957E0,1.721649E8,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"split_indices":[44,5,4,31,0,30,30,7,4,0,0,0,0,30,2,5,58,0,0,0,55,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.92E2,5.4E1,1.14E2,7.8E1,4.3E1,1.1E1,5.8E1,5.6E1,2.1E1,2.2E1,6E0,5E0,1E1,4.8E1,2.5E1,3.1E1,5E0,5E0,2.1E1,2.7E1,8E0,1.7E1,1.5E1,1.6E1,7E0,2E1,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.4172934E-3,-4.873043E-2,5.479038E-2,2.9505575E-3,-7.196424E-2,1.9374692E-1,3.8230103E-2,-2.232737E-3,2.4846569E-3,-2.2767627E-1,-5.168247E-2,1.6559279E-3,3.4322336E-1,6.4282104E-2,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,9.1448665E-2,1.5415065E-2,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.809334E-1,1.707184E-1,2.5722635E-1,1.343776E-1,2.9772854E-1,2.6643854E-1,1.8172754E-1,0E0,0E0,1.076473E-1,2.4328896E-1,0E0,1.7182684E-1,9.815651E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.945521E-2,1.7858982E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.47887E5,1.431E4,1.081E3,2.07684E5,9.675635E4,1.8018788E7,-2.232737E-3,2.4846569E-3,1.3738942E0,9.476661E1,1.6559279E-3,9.63E2,2.1949912E7,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,7.3667456E2,1.25836E5,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"split_indices":[54,1,9,2,1,30,47,0,0,55,54,0,0,34,0,0,0,0,0,0,0,54,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.41E2,1.14E2,4.4E1,9.7E1,1.1E1,1.03E2,2.2E1,2.2E1,1E1,8.7E1,6E0,5E0,7.4E1,2.9E1,4E0,6E0,1.7E1,7E1,2E0,3E0,4.7E1,2.7E1,3.9E1,8E0,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.1405309E-3,-4.1295454E-2,7.539111E-2,-4.773554E-2,2.649901E-3,5.8098752E-2,2.4969846E-1,-3.5499748E-2,-1.1151391E-1,7.256275E-2,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,-2.639264E-2,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,-3.14371E-2,-4.9293704E-2,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1],"loss_changes":[7.971391E-1,1.0867721E-1,2.6279783E-1,1.1465886E-1,0E0,1.4647299E-1,1.3158771E-1,9.851079E-2,1.7702582E-1,1.16677344E-1,0E0,0E0,0E0,0E0,9.4192E-2,0E0,0E0,0E0,0E0,0E0,9.489479E-2,9.564981E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,14,14,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1],"split_conditions":[5.9570425E5,2.407794E7,1.3821641E0,2.9138212E0,2.649901E-3,1E0,3.41212E7,1.4781966E-3,3.67E2,2.3581433E5,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,1E1,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,3.3341238E9,5.77032E5,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"split_indices":[49,9,36,60,0,85,53,60,0,30,0,0,0,0,10,0,0,0,0,0,5,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.6E2,9.1E1,1.51E2,9E0,8.4E1,7E0,1.28E2,2.3E1,7.5E1,9E0,5E0,2E0,1.1E1,1.17E2,2E0,2.1E1,3.8E1,3.7E1,3E0,1.14E2,8.2E1,3.2E1,6.9E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.8670357E-3,-4.5970436E-2,5.2696485E-2,6.0541928E-2,-5.781559E-2,2.443929E-2,1.2535055E-1,-9.904153E-4,6.934784E-3,-7.859939E-3,-4.945917E-2,3.5898713E-3,3.4437096E-3,2.2750926E-1,1.946528E-3,-6.620565E-2,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,-1.1629289E-1,-4.547119E-2,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.025647E-1,1.8644026E-1,2.186605E-1,1.2642913E-1,1.2496975E-1,9.5626205E-2,2.3731667E-1,0E0,0E0,0E0,9.588456E-2,0E0,9.368942E-2,1.0582209E-1,0E0,8.921841E-2,0E0,0E0,0E0,0E0,0E0,1.2605882E-1,9.986128E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.1623532E6,6.657047E2,1E0,1.7565872E-3,1E1,2.45226E6,-9.904153E-4,6.934784E-3,-7.859939E-3,2.9073395E9,3.5898713E-3,5.1332E4,3.0256798E0,1.946528E-3,4.0054E4,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,2.8516157E0,2.704375E6,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"split_indices":[54,47,54,8,60,3,31,0,0,0,5,0,9,56,0,9,0,0,0,0,0,60,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.44E2,1.07E2,1.4E1,1.3E2,7.8E1,2.9E1,8E0,6E0,7E0,1.23E2,2E1,5.8E1,1.2E1,1.7E1,9E1,3.3E1,2.2E1,3.6E1,4E0,8E0,2.5E1,6.5E1,2.2E1,3E0,4.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.004087E-3,-2.8343314E-2,7.549351E-2,-5.5867407E-2,2.2657372E-2,1.4241742E-2,5.9626795E-2,-6.4280997E-3,-4.644786E-2,4.461195E-2,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,-5.3805247E-2,4.6029454E-3,4.787177E-5,-8.447469E-3,-4.588249E-2,-1.3219344E-3,-5.0460016E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,-1,-1,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[5.418513E-1,2.4271044E-1,2.6314437E-1,8.962074E-2,1.0539442E-1,0E0,1.0044208E-1,0E0,1.2327367E-1,1.2701635E-1,0E0,0E0,0E0,0E0,1.0074824E-1,0E0,0E0,0E0,9.979747E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,14,14,18,18],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,-1,-1,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[9.821357E-1,7.422E3,1.215951E5,9.87574E2,2.1949912E7,1.4241742E-2,2.216135E5,-6.4280997E-3,1.8E1,1.1572553E7,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,1.3525729E6,4.6029454E-3,4.787177E-5,-8.447469E-3,2.0134454E0,-1.3219344E-3,-5.0460016E-3],"split_indices":[43,10,35,35,34,0,30,0,10,47,0,0,0,0,47,0,0,0,55,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.71E2,7E1,1.11E2,6E1,3E0,6.7E1,9E0,1.02E2,4.7E1,1.3E1,3.8E1,2.9E1,4E0,9.8E1,1.9E1,2.8E1,4E0,9.4E1,7.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.1594397E-2,-1.194265E-2,9.3121335E-2,-3.9583016E-2,2.608233E-2,1.6468607E-2,7.527401E-2,-5.18759E-2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,4.0722504E-4,-1.7415832E-3,3.3587667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,-1,15,-1,-1],"loss_changes":[4.712434E-1,2.0177807E-1,2.5062546E-1,1.2383151E-1,1.3076647E-1,0E0,1.04620725E-1,1.0193059E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.9496724E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.3394321E0,2.427E3,1.431E4,9.74979E6,2.637815E1,1.6468607E-2,2.174459E5,2.0256741E2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,3.4234106E5,-1.7415832E-3,3.3587667E-3],"split_indices":[44,2,9,9,58,0,30,54,0,0,0,0,0,0,30,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.9E2,5.4E1,1.1E2,8E1,2E0,5.2E1,9.7E1,1.3E1,5.1E1,2.9E1,2.8E1,2.4E1,7E1,2.7E1,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-8.11151E-3,-2.3770498E-2,1.3511473E-1,-4.903885E-2,3.7162323E-4,2.8097564E-1,1.2597005E-3,-4.1468557E-2,-9.574549E-3,1.8381007E-2,7.012952E-3,-1.3489653E-3,-6.7639E-2,-9.2418713E-4,6.152402E-3,-8.961814E-2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[5.4596394E-1,1.7908293E-1,3.6382854E-1,1.495364E-1,0E0,1.009686E-1,0E0,1.2485254E-1,0E0,0E0,0E0,1.3845807E-1,1.4358482E-1,0E0,0E0,9.946346E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,11,11,12,12,15,15],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[7.2259143E2,2.2348747E2,2.45226E6,2.1405182E2,3.7162323E-4,3.0256798E0,1.2597005E-3,4.37E2,-9.574549E-3,1.8381007E-2,7.012952E-3,4.02E2,5.8536225E5,-9.2418713E-4,6.152402E-3,4.73E2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"split_indices":[54,54,31,54,0,56,0,0,0,0,0,0,30,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.18E2,2.3E1,1.22E2,9.6E1,9E0,1.4E1,1.18E2,4E0,3E0,6E0,4.7E1,7.1E1,4.2E1,5E0,5.7E1,1.4E1,2E0,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.816721E-3,-3.212387E-2,7.8340545E-2,-5.9203006E-2,-2.047665E-3,1.05734095E-1,-1.2828258E-3,-4.2488243E-2,-1.6570118E-1,5.200363E-3,-1.1345699E-2,6.766135E-2,1.9234525E-1,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,1.8420164E-2,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.675061E-1,1.4077646E-1,2.3921198E-1,1.54948E-1,9.489868E-2,1.9802713E-1,0E0,1.2677193E-1,1.3189653E-1,0E0,9.6501775E-2,1.6102695E-1,1.9053322E-1,0E0,0E0,0E0,0E0,0E0,8.944257E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.5442211E8,3.6207435E0,1.2869189E5,2.3755E4,5.5303E9,-1.2828258E-3,7.92582E5,3.396945E7,5.200363E-3,3.7230852E0,9.90141E6,6.464779E6,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,4.33121E0,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"split_indices":[54,7,59,49,9,5,0,12,12,0,56,47,31,0,0,0,0,0,56,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.72E2,7.9E1,9E1,8.2E1,6.3E1,1.6E1,7.9E1,1.1E1,5E0,7.7E1,4.5E1,1.8E1,5.4E1,2.5E1,6E0,5E0,3.2E1,4.5E1,3.4E1,1.1E1,1E1,8E0,1.7E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.911687E-4,-2.2826223E-2,8.1994176E-2,-2.6359616E-2,9.769677E-3,1.5213658E-2,6.624614E-2,-5.9948202E-2,-4.920752E-3,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9672684E-1,1.8459845E-1,2.2044265E-1,1.4588554E-1,0E0,0E0,1.2346178E-1,1.278441E-1,1.16657905E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.4122614E2,1E0,5.283708E0,9.769677E-3,1.5213658E-2,1.1598054E8,3.901529E6,5.8536225E5,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"split_indices":[44,58,13,58,0,0,34,9,30,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,2.04E2,5.7E1,2.02E2,2E0,2E0,5.5E1,7.8E1,1.24E2,4.5E1,1E1,6.5E1,1.3E1,1.05E2,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-4.2027067E-3,-2.6648754E-2,8.658342E-2,-3.084103E-3,-2.5529137E-3,1.4771055E-2,2.9963553E-3,4.294585E-3,-1.1864337E-2,-5.120143E-2,4.8290283E-4,1.1069501E-3,-3.521963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[5.0360364E-1,2.0985942E-1,1.97959E-1,0E0,1.22443795E-1,0E0,0E0,0E0,1.0790554E-1,1.0145053E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.4125891E0,5.283708E0,1E0,-3.084103E-3,9.165388E4,1.4771055E-2,2.9963553E-3,4.294585E-3,1.1670399E-1,3.712E4,4.8290283E-4,1.1069501E-3,-3.521963E-3],"split_indices":[44,58,13,0,35,0,0,0,41,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.97E2,4.8E1,6.9E1,1.28E2,2E0,4.6E1,1E1,1.18E2,4.3E1,7.5E1,1.2E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.173793E-3,-2.6264051E-2,8.658401E-2,-6.669964E-2,-8.2213007E-4,1.4341326E-2,7.03292E-2,1.6294725E-2,-8.671885E-2,2.6084114E-2,-4.4527646E-2,8.693059E-4,1.125322E-1,-1.6205347E-3,6.1647515E-3,-2.2572479E-1,-6.812656E-2,4.0135182E-2,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.266243E-1,2.0775451E-1,1.8148205E-1,1.3100863E-1,1.4815538E-1,0E0,1.0412836E-1,1.1208219E-1,1.4744174E-1,1.12065986E-1,9.200167E-2,0E0,9.0320885E-2,0E0,0E0,9.239629E-2,1.0407472E-1,1.1784806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5098532E0,5.283708E0,1E0,3.6E1,1.7303246E7,1.4341326E-2,2.174459E5,7.014272E7,9.975593E4,5.8414636E0,2.094E3,8.693059E-4,4.7201157E-1,-1.6205347E-3,6.1647515E-3,7.725141E4,1.9415865E2,2.4256398E8,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"split_indices":[43,58,13,10,47,0,30,7,30,56,2,0,40,0,0,30,54,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.01E2,5.1E1,7.7E1,1.24E2,2E0,4.9E1,1.5E1,6.2E1,7.7E1,4.7E1,2.3E1,2.6E1,1.1E1,4E0,6E0,5.6E1,6.8E1,9E0,7E0,4E1,2.1E1,5E0,3E0,3E0,4.8E1,8E0,2E0,6.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[5.15778E-3,-1.8465955E-2,8.479578E-2,-4.582774E-2,2.8703757E-2,1.1748621E-2,2.8553656E-3,-6.113014E-2,7.824005E-4,7.2038495E-3,8.444443E-3,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,-1,-1,-1,-1],"loss_changes":[4.6061453E-1,2.4524778E-1,1.8237612E-1,1.1817169E-1,1.9200628E-1,0E0,0E0,1.1865306E-1,0E0,0E0,1.02744974E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2446029E0,1.781973E6,1.8426E4,5.4279834E-1,2.207E3,1.1748621E-2,2.8553656E-3,4.1150245E-1,7.824005E-4,7.2038495E-3,1E1,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"split_indices":[44,1,9,43,0,0,0,44,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.88E2,5.5E1,1.19E2,6.9E1,4E0,5.1E1,9.6E1,2.3E1,8E0,6.1E1,9.4E1,2E0,1.8E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[2.6694385E-3,-1.2630397E-2,9.763298E-2,-2.7587097E-3,-3.7950324E-3,2.2221443E-1,6.921976E-4,4.2745727E-3,-9.810083E-3,4.104886E-3,1.3904189E-2,-3.826968E-2,8.1882236E-4,-4.5390716E-3,-2.056427E-2,-1.03351854E-1,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,-1,-1,11,-1,-1,13,-1,-1,15,17,-1,-1,-1],"loss_changes":[3.4557673E-1,1.5018955E-1,3.3265132E-1,1.3020961E-1,0E0,1.3063407E-1,0E0,0E0,1.3977444E-1,0E0,0E0,9.7968146E-2,0E0,0E0,9.9550575E-2,1.2142539E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,11,11,14,14,15,15],"right_children":[2,4,6,8,-1,10,-1,-1,12,-1,-1,14,-1,-1,16,18,-1,-1,-1],"split_conditions":[6.56755E2,2.4219654E0,2.660842E6,1.1623532E6,-3.7950324E-3,2.1061536E3,6.921976E-4,4.2745727E-3,1.7179383E5,4.104886E-3,1.3904189E-2,1E0,8.1882236E-4,-4.5390716E-3,1.27318E5,1.3738942E0,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"split_indices":[54,55,31,47,0,4,0,0,34,0,0,102,0,0,30,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2.04E2,3.2E1,1.81E2,2.3E1,1.2E1,2E1,1.1E1,1.7E2,6E0,6E0,8.5E1,8.5E1,1.7E1,6.8E1,1.1E1,5.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.2459247E-4,-2.1839963E-2,5.5326115E-2,6.403799E-2,-2.9754521E-2,6.706761E-2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,3.54164E-2,1.4349867E-1,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,13,15,-1,-1,-1,-1],"loss_changes":[3.0025756E-1,1.2017081E-1,1.2092781E-1,9.031686E-2,1.1387043E-1,1.5458077E-1,0E0,0E0,0E0,0E0,0E0,1.247524E-1,1.0393754E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,11,11,12,12],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,14,16,-1,-1,-1,-1],"split_conditions":[3.799535E2,1.1623532E6,6.236639E3,2.0535674E0,1.4781966E-3,9.386909E2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,9E0,7.529323E4,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"split_indices":[54,47,4,55,60,54,0,0,0,0,0,3,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.73E2,7E1,1.4E1,1.59E2,6.5E1,5E0,8E0,6E0,6E0,1.53E2,4.7E1,1.8E1,8E0,3.9E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.029568E-4,-6.5150703E-3,1.8107072E-1,-3.652972E-2,2.4049532E-2,1.0449934E-2,-1.5273974E-3,-2.0122712E-2,-9.696651E-2,3.5459854E-2,-2.3149115E-3,4.5559215E-3,-2.7940093E-2,-1.3360148E-3,-1.7182508E-1,6.808551E-3,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,4.025893E-2,3.4104274E-3,-9.366535E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[3.37975E-1,2.2380325E-1,1.3455206E-1,1.2001395E-1,1.08324245E-1,0E0,0E0,9.831457E-2,1.2388055E-1,1.0830644E-1,0E0,0E0,1.4380896E-1,0E0,1.03295654E-1,1.3083223E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.5229566E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,12,12,14,14,15,15,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[1.9292948E3,2.0718746E8,6.236639E3,4.352025E0,2.2979798E0,1.0449934E-2,-1.5273974E-3,1E1,5.694004E6,3.642194E0,-2.3149115E-3,4.5559215E-3,9.476661E1,-1.3360148E-3,4.752366E0,4.854831E8,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,1.9647126E7,3.4104274E-3,-9.366535E-4],"split_indices":[54,7,4,56,55,0,0,10,47,56,0,0,54,0,58,12,0,0,0,0,0,0,47,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.42E2,9E0,1.22E2,1.2E2,7E0,2E0,9.7E1,2.5E1,1.05E2,1.5E1,5E0,9.2E1,1.4E1,1.1E1,5.9E1,4.6E1,2.5E1,6.7E1,4E0,7E0,2E1,3.9E1,2.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_emr","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"61"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.78048E-2,-1.8552697E-1,4.901243E-1,-3.3088633E-1,3.0965138E-2,2.4628823E-1,8.852223E-1,-4.659008E-1,-1.3334489E-1,-1.5243791E-1,1.4812984E-1,2.971024E-1,-1.2563897E-2,5.130134E-2,6.705256E-1,-1.8754591E-1,-5.3297985E-1,9.115728E-2,-2.2618434E-1,-2.66628E-1,1.2518686E-2,2.2718608E-1,4.980866E-2,4.346211E-1,1.5858072E-1,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,-6.1508644E-1,-3.1681323E-1,7.0068543E-3,-7.234637E-4,-2.9873994E-1,1.068607E-2,-7.976921E-3,-2.4829587E-2,-7.415798E-2,4.8681386E-3,1.2035196E-2,4.350847E-2,-3.566741E-2,9.323246E-3,2.2088567E-2,1.0012548E-2,2.3240472E-1,2.6123306E-2,-1.783583E-2,-6.754464E-1,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,57,-1,-1,59,61,-1,-1,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4057398E1,5.265595E0,7.200094E0,2.60497E0,1.4837892E0,6.482475E-1,1.0196114E0,1.0016766E0,8.960098E-1,5.119209E-1,3.1514597E-1,7.120998E-1,2.5112772E-1,0E0,1.8040752E-1,5.0723886E-1,6.393223E-1,9.9871E-2,5.187807E-1,3.0155098E-1,1.1128237E-1,1.9333434E-1,2.8770238E-1,1.774602E-1,2.0905662E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3500404E-1,1.0710728E-1,0E0,0E0,1.3412046E-1,3.246751E-1,0E0,0E0,1.1355559E-1,0E0,0E0,9.497507E-2,1.2899351E-1,0E0,0E0,0E0,9.763676E-2,1.0520158E-1,0E0,3.8818073E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,35,35,36,36,39,39,42,42,43,43,47,47,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,58,-1,-1,60,62,-1,-1,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,2.7558826E-2,2.0224316E-1,1.9415865E2,2.4118989E8,2.05E0,7.1239825E5,1.433458E10,8.938651E4,7.919613E5,3.2744168E6,7.3E1,6.035389E6,5.130134E-2,2.5E1,2.7287993E0,1.021924E6,1.1936242E7,4.145324E6,2.7575057E0,6.406965E-1,1.2183861E7,2.1469998E5,6.75602E5,2.8994352E7,-7.4108643E-3,6.4239698E-3,3.0993018E-2,9.69258E-3,8.810053E-3,-1.2075094E-2,3.12986E10,1.858E3,7.0068543E-3,-7.234637E-4,3.1100525E5,4.7E3,-7.976921E-3,-2.4829587E-2,4.565566E-1,4.8681386E-3,1.2035196E-2,6.7E2,1.526402E6,9.323246E-3,2.2088567E-2,1.0012548E-2,1E0,1.479938E6,-1.783583E-2,7.359468E8,-2.7743543E-3,-1.5312129E-2,-5.9876E-3,-1.5172383E-2,9.682812E-3,-7.0001236E-3,2.4765234E-3,-8.139826E-3,-4.2919717E-3,6.0669715E-3,-5.2555413E-3,2.8933594E-3,1.3521158E-2,4.8310217E-3,-3.4630008E-3,5.514987E-3,-4.2413525E-2,-2.3838768E-2],"split_indices":[53,39,39,53,7,54,34,32,34,33,51,8,1,0,3,55,30,46,1,59,40,33,29,9,1,0,0,0,0,0,0,32,0,0,0,34,0,0,0,40,0,0,0,9,0,0,0,109,33,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,1.66E2,7.6E1,9.9E1,6.7E1,4.8E1,2.8E1,5.8E1,4.1E1,2.6E1,4.1E1,4E1,8E0,1E1,1.8E1,1.2E1,4.6E1,1.2E1,2.9E1,1.5E1,1.1E1,2.2E1,1.9E1,1.9E1,2.1E1,4E0,4E0,1.6E1,2E0,2E0,1E1,3.2E1,1.4E1,7E0,5E0,2.2E1,7E0,1.3E1,2E0,6E0,5E0,1.7E1,5E0,1.3E1,6E0,1.3E1,6E0,1.3E1,8E0,9E0,2.3E1,2E0,1.2E1,6E0,1.6E1,3E0,4E0,3E0,3E0,2E0,3E0,7E0,6E0,7E0,6E0,4E0,4E0,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.0040125E-2,-2.1713006E-1,4.0957847E-1,-3.5627815E-1,9.978797E-3,2.0629005E-1,8.082211E-1,-4.7081342E-1,-1.7093469E-1,-5.8718614E-2,1.0682804E-2,6.47079E-2,2.835063E-1,2.307691E-2,4.1448183E-2,9.978426E-4,-4.971977E-1,1.4917599E-3,-3.1879982E-1,-1.7213058E-1,8.313783E-2,1.541623E-1,-7.4060783E-3,3.6715305E-1,1.0359522E-2,-1.8699963E-1,-5.3688544E-1,-9.420938E-2,1.2580958E-2,-1.8095352E-2,-1.4491154E-1,-2.1190276E-2,-1.1271654E-1,-1.211096E-2,2.5133708E-1,9.38944E-3,-7.0502064E-3,1.9026043E-5,3.9769462E-1,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,-3.106009E-1,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,-1.4537086E-1,7.170774E-3,-6.025724E-2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,4.859038E-1,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,5.357659E-1,3.981885E-3,7.6941703E-3,2.5353398E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,-1,-1,55,-1,-1,-1,-1,-1,57,-1,-1,-1,-1,59,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[2.0510149E1,4.996633E0,6.1543484E0,1.9999018E0,9.984025E-1,5.610745E-1,5.2623177E-1,7.9289055E-1,9.9481404E-1,7.8654516E-1,0E0,4.4556248E-1,7.730229E-1,0E0,0E0,0E0,6.0133266E-1,5.4866415E-1,2.8996253E-1,4.4809937E-1,3.5550103E-1,3.2364437E-1,0E0,2.9102945E-1,1.4653622E-1,9.1507256E-2,6.157198E-1,1.915977E-1,0E0,0E0,1.275117E-1,0E0,2.4100256E-1,1.6567487E-1,2.9098767E-1,0E0,0E0,0E0,3.2995868E-1,0E0,0E0,0E0,0E0,0E0,1.7156088E-1,0E0,0E0,0E0,0E0,1.24156505E-1,0E0,9.488517E-2,0E0,0E0,0E0,0E0,2.6596856E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2510395E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,30,30,32,32,33,33,34,34,38,38,44,44,49,49,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,-1,-1,56,-1,-1,-1,-1,-1,58,-1,-1,-1,-1,60,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.5665247E2,1.7613442E-1,2.0224316E-1,1.9415865E2,9.749826E6,4.633287E2,3.856E3,3.8757736E-7,1.5248811E1,1.5225257E-1,1.0682804E-2,3.6217573E0,2.1030048E10,2.307691E-2,4.1448183E-2,9.978426E-4,1.1266387E10,3.165913E6,3.03E9,1.386452E0,1.7573398E2,1.9647126E7,-7.4060783E-3,2.34E3,2.51838E6,2.85E2,1.021924E6,5.63651E5,1.2580958E-2,-1.8095352E-2,2.3407683E0,-2.1190276E-2,2.7538432E8,2.424E3,1.2730925E0,9.38944E-3,-7.0502064E-3,1.9026043E-5,5.792986E4,-3.9077615E-3,6.8788184E-3,-1.0855099E-2,-5.1025632E-5,-2.6532287E-2,1.892E3,1.0237377E-3,-8.732227E-3,-9.148189E-3,2.374431E-3,2.5187686E-1,7.170774E-3,3.56E2,8.752215E-3,-2.3710665E-3,1.5809635E-2,8.583337E-3,1.1107071E7,-6.489753E-3,-1.8041462E-2,-1.7558772E-3,-8.84009E-3,7.2434393E-4,-6.663172E-3,3.302E3,3.981885E-3,7.6941703E-3,2.5353398E-2],"split_indices":[53,42,39,53,9,53,2,39,57,39,0,55,5,0,0,0,32,1,5,54,53,46,0,2,33,0,30,9,0,0,55,0,7,2,43,0,0,0,29,0,0,0,0,0,2,0,0,0,0,43,0,10,0,0,0,0,9,0,0,0,0,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,1.57E2,7.7E1,9.7E1,6E1,5.2E1,2.5E1,5.9E1,3.8E1,4.7E1,1.3E1,1.9E1,3.3E1,1E1,1.5E1,3E0,5.6E1,1.8E1,2E1,2.6E1,2.1E1,1.4E1,5E0,2.5E1,8E0,7E0,4.9E1,1.4E1,4E0,1.2E1,8E0,3E0,2.3E1,1.4E1,7E0,1.2E1,2E0,2E0,2.3E1,5E0,3E0,5E0,2E0,3.6E1,1.3E1,7E0,7E0,6E0,2E0,2.1E1,2E0,1.2E1,2E0,2E0,5E0,8E0,1.5E1,6E0,7E0,8E0,1.3E1,7E0,5E0,1.3E1,2E0,2E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.5844442E-2,-2.1939752E-1,3.5634845E-1,-3.067927E-1,7.70621E-2,1.8756232E-1,7.0530635E-1,-3.6990502E-1,-1.6308445E-2,3.355599E-2,1.0062139E-2,-4.2225484E-2,2.3436439E-1,1.2260787E-2,7.853399E-1,-4.3145156E-1,-2.022222E-1,1.7129338E-1,-1.4914486E-1,-7.0826644E-3,8.609564E-3,7.4184006E-3,-1.4439643E-2,2.7769628E-1,6.108423E-3,1.9752327E-2,3.9388932E-2,-3.5376546E-1,-6.491721E-1,-1.04918115E-1,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,4.5147832E-2,-5.3988285E-3,2.0416094E-1,4.445212E-1,-5.368166E-3,3.978625E-3,-5.482469E-1,-2.6226673E-1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,-3.322308E-2,-8.5412245E-4,7.761204E-3,2.3548925E-1,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,-2.9240888E-1,-3.515024E-3,6.6058734E-3,2.612413E-1,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,49,51,-1,-1,53,55,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,-1,-1,61,-1,-1,63,-1,-1,-1,-1,-1],"loss_changes":[1.863963E1,4.1431956E0,4.991946E0,2.2489424E0,2.3938012E-1,6.55169E-1,7.3028564E-1,9.5510197E-1,5.9871566E-1,2.0101649E-1,0E0,7.301139E-1,4.9284196E-1,0E0,4.888401E-1,1.0050564E0,4.5257246E-1,2.7720916E-1,2.4958798E-1,1.5788087E-1,0E0,0E0,0E0,4.2448258E-1,1.1006676E-1,0E0,0E0,8.527217E-1,3.161502E-1,1.413299E-1,0E0,0E0,0E0,0E0,0E0,1.5807515E-1,0E0,3.994676E-1,2.7063847E-1,0E0,0E0,1.5897751E-1,3.2701302E-1,0E0,0E0,0E0,1.2576254E-1,0E0,0E0,1.6656256E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.2975593E-1,0E0,0E0,1.3126945E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24,27,27,28,28,29,29,35,35,37,37,38,38,41,41,42,42,46,46,49,49,56,56,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,50,52,-1,-1,54,56,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,-1,-1,62,-1,-1,64,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,6.005889E-1,2.0224316E-1,2.727992E6,9.750871E6,7.12E2,2.744E3,1.1670399E-1,1.6E1,1E0,1.0062139E-2,3.9378E5,2.8812434E7,1.2260787E-2,2.2324745E5,3.98231E-2,3.4047124E-1,6.0578914E4,2.6017979E11,4.33E2,8.609564E-3,7.4184006E-3,-1.4439643E-2,7.2259143E2,1.04E2,1.9752327E-2,3.9388932E-2,3.3617156E-4,1.868632E7,1.86204E5,-1.6434768E-2,-5.0489525E-3,1.1237176E-2,5.6425002E-3,-9.003896E-3,7.948E4,-5.3988285E-3,5.130435E0,9.521E3,-5.368166E-3,3.978625E-3,5.8013948E7,9.3E1,-3.0504808E-2,-7.2826403E-3,-9.090324E-3,2.3544362E6,-8.5412245E-4,7.761204E-3,2.1061536E3,-7.2760447E-3,2.5448397E-2,9.10704E-3,-2.6254488E-2,-9.790089E-3,3.14645E-4,1.65E9,-3.515024E-3,6.6058734E-3,7.83495E5,4.0272932E-4,-1.5757626E-2,-4.5160223E-3,8.119221E-3,1.583806E-2],"split_indices":[53,40,39,1,9,0,2,40,3,108,0,1,46,0,29,43,43,33,32,0,0,0,0,53,8,0,0,40,46,9,0,0,0,0,0,1,0,55,2,0,0,7,11,0,0,0,52,0,0,4,0,0,0,0,0,0,5,0,0,34,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.58E2,8.6E1,1.22E2,3.6E1,5.9E1,2.7E1,1E2,2.2E1,2.9E1,7E0,1E1,4.9E1,5E0,2.2E1,7.2E1,2.8E1,9E0,1.3E1,2.4E1,5E0,6E0,4E0,4.1E1,8E0,7E0,1.5E1,5.5E1,1.7E1,1.9E1,9E0,2E0,7E0,2E0,1.1E1,1.7E1,7E0,3E1,1.1E1,3E0,5E0,1.6E1,3.9E1,1.5E1,2E0,7E0,1.2E1,1.2E1,5E0,2.8E1,2E0,6E0,5E0,1.3E1,3E0,4E0,3.5E1,1E1,2E0,2.5E1,3E0,2.5E1,1E1,1.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[1.0417009E-2,-1.92214E-1,4.0404752E-1,-3.015024E-1,-7.670976E-5,1.7257118E-1,7.666108E-1,-3.8266033E-1,-1.3505321E-2,-5.040442E-2,1.1856716E-2,2.0422818E-1,-5.236619E-3,4.0671965E-1,9.444278E-1,-4.4783047E-1,-2.4137214E-1,-8.257532E-2,9.555998E-3,-1.7077446E-2,-2.2676414E-2,-1.2199888E-3,2.4267583E-1,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,-1.7631327E-1,-5.2404433E-1,-4.6340473E-2,-1.6422002E-2,8.984605E-3,-1.4559102E-1,-8.014499E-2,7.040712E-2,3.7856903E-2,1.1994875E-2,8.994386E-3,-1.1225067E-2,-5.8429706E-1,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,-1.4150277E-1,4.9856484E-2,1.1765836E-2,1.939237E-2,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,-1,35,-1,-1,-1,-1,37,39,41,-1,-1,43,45,47,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0656195E1,3.5906997E0,7.253089E0,2.5425148E0,8.754045E-1,5.339109E-1,1.7917042E0,6.7456627E-1,4.1952434E-1,5.01606E-1,0E0,4.531319E-1,0E0,9.731603E-2,3.5835075E-1,1.0871277E0,7.457906E-1,3.9352587E-1,0E0,0E0,2.7846104E-1,0E0,2.731049E-1,0E0,0E0,0E0,0E0,4.8621753E-1,5.456524E-1,1.2857142E-1,0E0,0E0,3.2480013E-1,2.623061E-1,1.9802536E-1,1.5393375E-1,0E0,0E0,0E0,2.466135E-1,0E0,0E0,0E0,0E0,0E0,1.9748119E-1,1.3966252E-1,0E0,1.250949E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,20,20,22,22,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,45,45,46,46,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,-1,36,-1,-1,-1,-1,38,40,42,-1,-1,44,46,48,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0845802E6,1.7613442E-1,1.5972575E0,2.727992E6,2.03693E11,1.25440235E1,7.2610445E8,1.9415865E2,1.22E2,2.9927E4,1.1856716E-2,4.0123675E-2,-5.236619E-3,1.3589149E6,2.2324745E5,2.431768E2,1.2247422E1,2.6017979E11,9.555998E-3,-1.7077446E-2,1.1343225E6,-1.2199888E-3,3.4021906E2,1.953203E-2,5.294022E-3,2.0418731E-2,4.4700544E-2,1.0395554E5,6.6037946E2,6.0515753E2,-1.6422002E-2,8.984605E-3,2.3407683E0,9.74979E6,1.2355705E6,1.19881555E-1,1.1994875E-2,8.994386E-3,-1.1225067E-2,1.51E3,-1.1379077E-2,-8.541172E-3,7.875472E-4,-8.701612E-3,7.5052767E-3,4.4E1,5.3164E4,1.1765836E-2,6.0039372E0,-6.401078E-3,6.153494E-3,-2.063075E-2,-3.2148127E-2,3.323836E-3,-7.849785E-3,-3.6212897E-3,6.005565E-3,-8.30004E-4,8.945503E-3],"split_indices":[52,42,42,1,32,58,12,53,8,9,0,39,0,48,29,56,57,32,0,0,34,0,53,0,0,0,0,29,4,4,0,0,55,9,34,39,0,0,0,2,0,0,0,0,0,10,30,0,55,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.7E2,8.7E1,1.08E2,6.2E1,5.4E1,3.3E1,8.4E1,2.4E1,5.3E1,9E0,4.9E1,5E0,1.2E1,2.1E1,5.6E1,2.8E1,1.9E1,5E0,3E0,5E1,7E0,4.2E1,1E1,2E0,4E0,1.7E1,1.3E1,4.3E1,1.2E1,1.6E1,3E0,1.6E1,3.1E1,1.9E1,6E0,3.6E1,2E0,1.1E1,3.4E1,9E0,3E0,9E0,1.4E1,2E0,2.1E1,1E1,3E0,1.6E1,2E0,4E0,2.2E1,1.2E1,3E0,1.8E1,4E0,6E0,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-2.2585599E-2,-1.9692676E-1,3.6539125E-1,-2.9716375E-1,-2.7832244E-2,1.6051705E-1,7.1087146E-1,-3.7421748E-1,1.4051265E-2,-6.813547E-2,1.1468622E-2,-8.938615E-2,2.114588E-1,4.520597E-1,9.747266E-1,-4.3956405E-1,-2.1756326E-1,-5.356781E-2,1.1113507E-2,-1.736248E-1,1.3845394E-2,4.104848E-3,-6.774424E-3,2.3279007E-1,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,-2.5017148E-1,-4.929268E-1,1.6659833E-3,-1.6076026E-2,9.1863185E-2,-6.740727E-3,-7.370915E-2,-3.767319E-1,-5.412376E-3,5.573497E-2,9.279637E-2,2.8638762E-1,-2.9118592E-1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,3.14243E-1,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,-1,47,-1,49,51,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1,-1],"loss_changes":[1.6576199E1,2.8561993E0,5.265172E0,2.5471687E0,7.675839E-1,6.4269626E-1,1.4714527E0,7.744961E-1,3.715908E-1,4.9534598E-1,0E0,1.2332737E-1,2.3372102E-1,1.488092E-1,2.703581E-1,4.729681E-1,8.93371E-1,2.783168E-1,0E0,4.7957075E-1,1.9750053E-1,0E0,0E0,2.585504E-1,0E0,0E0,0E0,0E0,0E0,1.6383308E-1,2.5936127E-1,1.265059E-1,0E0,2.4251655E-1,0E0,1.5377875E-1,1.1529529E-1,0E0,2.033091E-1,1.5228397E-1,2.5932693E-1,1.0870254E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3727503E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,29,29,30,30,31,31,33,33,35,35,36,36,38,38,39,39,40,40,41,41,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,-1,48,-1,50,52,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1,-1],"split_conditions":[4.083633E6,1.7613442E-1,1.5972575E0,2.246309E6,2.03693E11,3.359399E-1,1.6525185E9,1.9415865E2,1.22E2,7.251646E-2,1.1468622E-2,2.2183135E-1,1.25440235E1,1.6790323E8,4.6E1,2.4747229E2,1.2288136E1,4.7E3,1.1113507E-2,3.4047124E-1,2.0358758E8,4.104848E-3,-6.774424E-3,1.0705044E3,-1.9224216E-3,5.212598E-3,2.1428604E-2,1.5521408E-2,4.6036705E-2,1.499E3,1.021924E6,2.797224E2,-1.6076026E-2,1.445113E6,-6.740727E-3,9.593298E6,6.9268906E5,-5.412376E-3,2.7352352E6,1.8E1,1.1107071E7,2.923913E1,4.7776944E-4,-2.2964986E-2,-1.0335838E-2,-3.3362042E-3,5.512411E-3,1.00689195E-2,-4.5685326E-3,-5.43244E-3,3.6311205E-3,-8.220353E-3,-2.2604892E-2,5.7539586E-3,-1.922653E-3,6.9679106E-3,-3.5292034E-3,6.464779E6,-1.7980508E-3,-3.8156614E-3,-1.4992879E-2,1.7633969E-2,6.0686995E-3],"split_indices":[52,42,42,1,32,43,5,53,8,39,0,40,58,7,8,56,57,0,0,43,7,0,0,56,0,0,0,0,0,2,30,53,0,9,0,46,34,0,49,3,9,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,30,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.68E2,7.5E1,1.05E2,6.3E1,4.8E1,2.7E1,8.4E1,2.1E1,5.6E1,7E0,8E0,4E1,1.5E1,1.2E1,5.8E1,2.6E1,1.7E1,4E0,2.4E1,3.2E1,2E0,6E0,3.7E1,3E0,2E0,1.3E1,2E0,1E1,1.4E1,4.4E1,1.1E1,1.5E1,7E0,1E1,1.7E1,7E0,7E0,2.5E1,1.1E1,2.6E1,1.2E1,2E0,3.8E1,6E0,7E0,4E0,4E0,3E0,1.3E1,4E0,4E0,3E0,1.4E1,1.1E1,8E0,3E0,2.4E1,2E0,3E0,9E0,1.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-8.34782E-3,-1.667577E-1,4.3978554E-1,-3.0709088E-1,2.6771413E-2,1.9930072E-1,7.418625E-1,-4.0431347E-1,-1.2503225E-1,-6.46251E-2,1.2227376E-1,2.7885714E-1,3.0976776E-2,3.739352E-1,8.703366E-1,-4.299911E-1,4.975969E-3,1.0842858E-1,-2.3695372E-1,-1.4191392E-1,1.8143274E-1,1.5553638E-1,-3.5369173E-3,2.103493E-2,1.11278795E-1,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,-4.528867E-1,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,-2.1131526E-1,5.121328E-2,8.2729507E-4,1.0838011E-2,7.531465E-2,2.1820368E-1,6.924229E-3,-5.724572E-3,-4.900431E-1,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,-4.1408205E-1,-6.208327E-1,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,45,-1,-1,-1,-1,47,49,-1,-1,51,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.7394594E1,4.9239187E0,4.501422E0,1.807971E0,6.800027E-1,4.868381E-1,9.980869E-1,9.540901E-1,1.0186052E0,7.864486E-1,2.696029E-1,7.975136E-1,3.8219005E-1,1.8597853E-1,5.8754253E-1,3.8879967E-1,0E0,1.0245718E-1,1.4958644E-1,4.2632413E-1,1.0402632E-1,1.4946616E-1,0E0,0E0,1.9317481E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9329586E-1,0E0,0E0,0E0,0E0,2.1796727E-1,1.8252599E-1,0E0,0E0,1.0288309E-1,1.249153E-1,0E0,0E0,2.2973442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3063469E-1,2.7027082E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,24,24,32,32,37,37,38,38,41,41,42,42,45,45,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,46,-1,-1,-1,-1,48,50,-1,-1,52,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.4210773E-2,2.0224316E-1,1.773747E2,6.75553E5,4.3147E5,2.174459E5,4.108148E6,9.165388E4,1.5225257E-1,2.4219654E0,3.5630266E8,1.4E1,5.76E3,1.6507233E3,6E0,4.975969E-3,1.1936242E7,2.1055435E1,2.5055168E9,6.765595E7,2.151E3,-3.5369173E-3,2.103493E-2,8.96347E7,-8.429581E-3,6.6356896E-3,5.2484437E-3,2.097822E-2,2.5783971E-2,4.75987E-2,-5.6455536E-3,1.437883E6,7.794654E-3,-1.4084585E-4,-6.567549E-3,-1.4194764E-2,2.5335E4,4.933E4,8.2729507E-4,1.0838011E-2,5.546848E1,9.173913E5,6.924229E-3,-5.724572E-3,1.499E3,-1.1495543E-2,-2.0872124E-2,-7.118079E-3,7.945114E-3,-3.9223093E-3,1.0614912E-2,1.4815459E-3,1.080852E-2,-5.124271E-4,1.53091E5,2.373E3,-2.049398E-2,-1.0203345E-2,-2.9396933E-2,-7.3074135E-3],"split_indices":[53,39,39,53,9,9,29,9,34,39,54,7,3,2,4,10,0,46,57,5,5,2,0,0,45,0,0,0,0,0,0,0,1,0,0,0,0,9,9,0,0,53,48,0,0,2,0,0,0,0,0,0,0,0,0,9,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.8E2,6.3E1,1.04E2,7.6E1,3.6E1,2.7E1,6.7E1,3.7E1,3.9E1,3.7E1,2.4E1,1.2E1,8E0,1.9E1,6.4E1,3E0,1.2E1,2.5E1,3E1,9E0,3.2E1,5E0,1E1,1.4E1,4E0,8E0,3E0,5E0,1E1,9E0,5E0,5.9E1,7E0,5E0,1.4E1,1.1E1,2.2E1,8E0,3E0,6E0,1.5E1,1.7E1,1.2E1,2E0,4.8E1,1.1E1,2E0,2E1,4E0,4E0,2E0,1.3E1,1.5E1,2E0,3.3E1,1.5E1,2.4E1,9E0,1.3E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[4.9497597E-3,-1.8956819E-1,3.4107372E-1,-3.159441E-1,-1.5801242E-2,1.5268305E-1,6.5795535E-1,-3.6932546E-1,7.846604E-3,-5.214277E-2,1.0556764E-1,-1.2807088E-2,2.3161104E-1,4.560374E-1,8.566107E-1,-2.9471084E-1,-5.894394E-1,1.2438455E-1,-7.3430347E-3,-1.3571183E-1,1.1542113E-2,7.1960697E-3,-4.9598026E-3,1.3622053E-1,-1.1057054E-2,6.492877E-2,3.2902324E-1,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,-2.0076081E-1,-6.4882565E-1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,9.8576896E-2,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,2.6239768E-1,2.3697935E-2,-2.679108E-1,-6.2030084E-2,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,4.4976994E-3,-3.273256E-1,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6410194E1,3.4866414E0,5.391714E0,1.5930605E0,3.054343E-1,7.736126E-1,9.7316456E-1,1.1237602E0,3.082666E-1,2.8363693E-1,2.2685544E-1,7.515435E-1,6.2811255E-1,2.3523116E-1,7.9534054E-1,9.5042276E-1,3.015952E-1,2.1883255E-1,0E0,9.567392E-2,1.899715E-1,0E0,0E0,2.035301E-1,0E0,1.436969E-1,2.1399069E-1,0E0,0E0,0E0,0E0,0E0,3.7464368E-1,1.0184717E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1995159E-1,0E0,0E0,0E0,0E0,1.2598455E-1,0E0,4.546547E-1,1.02785856E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.053444E-1,1.2376809E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,23,23,25,25,26,26,32,32,33,33,40,40,45,45,47,47,48,48,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.1306552E-1,1.8829846E-1,3.402546E6,2.424E3,3.165913E6,1.4968063E3,5.1653096E-3,3.771E3,5.670282E-1,3.2410275E2,1.1936242E7,6.2143734E4,2.0270285E8,2.65826E6,3.3617156E-4,1.8018788E7,3.995065E6,-7.3430347E-3,2.8507206E-1,2.6801108E2,7.1960697E-3,-4.9598026E-3,1.9816086E0,-1.1057054E-2,2.51838E6,8.0267444E2,6.0668467E-3,2.2115694E-2,4.5977116E-2,1.9761955E-2,-2.0827556E-2,8.456733E10,7.5E1,-8.92567E-3,1.3039206E-2,-5.5079587E-4,-3.3527883E-3,-9.546827E-3,-2.4783772E-3,7E0,8.381679E-3,-4.973257E-3,-2.1943972E-3,6.0695503E-3,8.0774775E6,2.3697935E-2,2.3498028E6,3.948533E5,-3.0791823E-2,-1.3182735E-2,6.8768305E-3,-1.7086562E-3,1.3424839E-2,4.278042E-3,7.1E1,6E0,-4.936902E-3,2.7404095E-3,-6.3049076E-3,8.799093E-3,-2.473953E-3,-1.5291102E-2],"split_indices":[53,40,39,1,2,1,4,39,2,43,53,46,29,7,48,40,46,30,0,43,4,0,0,54,0,33,53,0,0,0,0,0,32,8,0,0,0,0,0,0,8,0,0,0,0,33,0,46,29,0,0,0,0,0,0,0,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.58E2,9.1E1,9.1E1,6.7E1,5.8E1,3.3E1,7.8E1,1.3E1,5.2E1,1.5E1,1.9E1,3.9E1,1.8E1,1.5E1,6E1,1.8E1,8E0,5E0,2.2E1,3E1,1.2E1,3E0,1.2E1,7E0,1.5E1,2.4E1,3E0,1.5E1,9E0,6E0,1.9E1,4.1E1,1.5E1,3E0,3E0,5E0,1.4E1,8E0,1.7E1,1.3E1,1E1,2E0,6E0,9E0,2E1,4E0,2.7E1,1.4E1,1.2E1,3E0,9E0,4E0,1.5E1,5E0,5E0,2.2E1,1E1,4E0,3E0,2E0,2E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.0813884E-3,-2.0281385E-1,3.2203957E-1,-2.7021798E-1,6.471507E-2,1.3456307E-1,6.046468E-1,-3.230015E-1,-2.4385922E-2,-4.1049863E-3,1.0497509E-1,-1.7012404E-1,1.8569289E-1,7.2859275E-1,3.886841E-1,-2.9304287E-1,-5.693168E-1,7.866887E-3,-7.695989E-2,1.368179E-1,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.1308988E-1,1.2391388E-1,4.675196E-2,5.738674E-1,9.75509E-3,2.0310232E-2,-3.7767187E-1,-1.9021347E-1,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,9.295686E-2,2.2762196E-2,-4.9231676E-3,1.4885345E-1,8.6237835E-3,2.6556352E-2,-4.5262882E-1,-2.0554197E-1,-2.254751E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.7958388E-2,1.8340372E-1,-3.690774E-1,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,-2.566865E-1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,-2.7930382E-1,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,47,-1,-1,49,-1,-1,51,53,55,-1,-1,-1,57,59,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,65,-1,-1,-1],"loss_changes":[1.6330715E1,2.812533E0,5.0049305E0,1.6011333E0,2.1443705E-1,9.507282E-1,7.339792E-1,5.858507E-1,2.613468E-1,0E0,1.8759432E-1,2.9529205E-1,3.6259735E-1,6.0491276E-1,9.870744E-2,7.3748016E-1,1.4939356E-1,0E0,3.703289E-1,1.539154E-1,0E0,0E0,0E0,6.859244E-1,2.2933495E-1,0E0,1.1461353E-1,0E0,0E0,5.5208206E-1,3.4554946E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2862067E-1,0E0,0E0,1.4594752E-1,0E0,0E0,4.9916983E-1,1.4001942E-1,4.1745436E-1,0E0,0E0,0E0,1.3290837E-1,1.4290059E-1,2.839706E-1,0E0,0E0,0E0,0E0,1.334548E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1397505E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,23,23,24,24,26,26,29,29,30,30,37,37,40,40,43,43,44,44,45,45,49,49,50,50,51,51,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,48,-1,-1,50,-1,-1,52,54,56,-1,-1,-1,58,60,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,66,-1,-1,-1],"split_conditions":[3.1259552E2,4.9615598E-1,1.2711891E0,2.727992E6,1.8681856E8,7.5E2,9.8110194E5,1E0,2.6017979E11,-4.1049863E-3,1E0,3.9378E5,6.5E1,2.7479675E-1,2.0787016E5,1.31654E5,3.7274265E2,7.866887E-3,3.2829787E12,3.1277522E6,-4.554369E-3,2.899179E-3,-1.3068826E-2,3.302E3,3.0327642E-1,4.675196E-2,3.301E3,9.75509E-3,2.0310232E-2,1.3738942E0,2.0766662E6,-8.267942E-3,-2.7954973E-2,-7.107389E-3,5.948831E-3,8.842744E-3,1.3525004E-3,1.4958695E5,2.2762196E-2,-4.9231676E-3,1.7377981E9,8.6237835E-3,2.6556352E-2,4.969844E3,7.07816E10,4.5595855E-1,1.2384782E-3,-8.575237E-3,8.890647E-3,1.5E1,4.089454E4,1.9415865E2,-3.1514846E-2,-1.1951719E-2,-3.0021057E-3,4.538159E-3,2E1,5.4449462E-3,-5.238814E-3,3.3148154E-4,9.383389E-3,-1.7325062E-2,-1.7338246E-4,2.5605E4,-2.841845E-3,-1.3653175E-3,-1.288786E-2],"split_indices":[53,40,40,1,7,0,34,82,32,0,82,1,8,39,29,9,4,0,32,52,0,0,0,2,43,0,2,0,0,54,49,0,0,0,0,0,0,29,0,0,12,0,0,48,32,57,0,0,0,3,29,53,0,0,0,0,3,0,0,0,0,0,0,30,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.54E2,9.5E1,1.23E2,3.1E1,5.8E1,3.7E1,1.01E2,2.2E1,6E0,2.5E1,8E0,5E1,2.2E1,1.5E1,9.2E1,9E0,4E0,1.8E1,2.2E1,3E0,3E0,5E0,1.5E1,3.5E1,5E0,1.7E1,6E0,9E0,4.9E1,4.3E1,2E0,7E0,1.3E1,5E0,1.3E1,9E0,8E0,7E0,3E0,3.2E1,2E0,1.5E1,3.3E1,1.6E1,3.7E1,6E0,2E0,6E0,7E0,2.5E1,2.7E1,6E0,1E1,6E0,3E0,3.4E1,4E0,3E0,4E0,2.1E1,2.5E1,2E0,3E1,4E0,2E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-6.4136046E-3,-1.6499257E-1,2.6240352E-1,-2.5779408E-1,-1.8667922E-4,1.3376746E-1,5.1147556E-1,-3.5323516E-1,-8.524396E-2,-9.4428E-2,1.0056048E-1,-9.662582E-2,1.73225E-1,3.220377E-2,3.9076596E-1,-1.4722008E-1,-4.0586957E-1,-1.8771532E-1,1.03829466E-1,-1.9139503E-1,7.505271E-2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,2.0356877E-1,-2.7897414E-3,4.3953964E-1,5.554044E-3,-9.59708E-3,4.6992786E-3,-4.2843238E-1,-4.7170566E-3,-1.2011155E-3,-2.7610272E-1,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,9.2738315E-2,2.768057E-1,7.707731E-3,2.085054E-2,-3.687053E-1,-5.6087404E-1,-1.6907474E-2,-7.4428786E-3,9.665818E-3,-4.444956E-2,3.618798E-1,4.56879E-3,-3.8534006E-1,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,4.9356914E-1,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,41,-1,43,-1,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,-1,-1,-1,57,59,-1,61,-1,-1,-1,-1,-1,-1,63,-1,-1,-1,-1],"loss_changes":[1.0913624E1,2.4623237E0,2.975348E0,1.6661301E0,5.696739E-1,5.985464E-1,5.864744E-1,6.559181E-1,7.557257E-1,5.238199E-1,2.0130792E-1,5.320686E-1,4.088E-1,0E0,2.3964357E-1,3.0165756E-1,3.0306053E-1,3.463729E-1,1.8510923E-1,2.6532936E-1,1.640884E-1,0E0,0E0,0E0,0E0,3.7205648E-1,0E0,1.0671902E-1,0E0,0E0,0E0,1.7289925E-1,0E0,0E0,1.2194073E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.8078758E-1,3.8673472E-1,0E0,0E0,1.6442966E-1,1.01364136E-1,0E0,0E0,0E0,2.1470177E-1,1.7472506E-1,0E0,8.7862015E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.314799E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,27,27,31,31,34,34,41,41,42,42,45,45,46,46,50,50,51,51,53,53,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,42,-1,44,-1,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,-1,-1,-1,58,60,-1,62,-1,-1,-1,-1,-1,-1,64,-1,-1,-1,-1],"split_conditions":[3.3103763E2,2.5106274E-2,1.8829846E-1,1.9415865E2,9.50379E5,7.5E2,7.5921344E5,2.431768E2,5.99999E9,1.2185057E0,1.3E1,3.13E2,1.25440235E1,3.220377E-2,2.56701E5,3.160848E6,7.4E1,3.0364912E5,4.654E3,2.587E3,1.7468556E2,6.830433E-3,-9.3465403E-4,6.7638247E-3,-1.2664262E-2,4.871647E2,-2.7897414E-3,1.5789307E8,5.554044E-3,-9.59708E-3,4.6992786E-3,1.912E3,-4.7170566E-3,-1.2011155E-3,5.316779E5,7.230653E-3,-4.0257815E-3,-5.8244118E-3,-2.0735325E-2,-9.1362983E-4,9.322331E-3,1.1572553E7,1.8751362E7,7.707731E-3,2.085054E-2,7.872337E5,1.2967741E0,-1.6907474E-2,-7.4428786E-3,9.665818E-3,1.1E1,5.5053414E9,4.56879E-3,1.0169992E0,-2.4994707E-3,-1.2912342E-2,-2.8180597E-2,-8.713767E-3,3.1759718E-3,1.0644849E-2,1.6851284E7,-4.7289496E-3,-1.7423468E-2,2.4869641E-2,7.6975627E-3],"split_indices":[53,39,39,53,9,0,34,56,5,42,3,0,58,0,30,9,3,34,2,2,53,0,0,0,0,53,0,5,0,0,0,2,0,0,34,0,0,0,0,0,0,46,45,0,0,52,54,0,0,0,3,5,0,54,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.54E2,1.6E2,9.4E1,1.02E2,5.8E1,6.3E1,3.1E1,6.5E1,3.7E1,3E1,2.8E1,9E0,5.4E1,9E0,2.2E1,1.4E1,5.1E1,2.4E1,1.3E1,1.9E1,1.1E1,1.9E1,9E0,4E0,5E0,4.8E1,6E0,1.8E1,4E0,1.1E1,3E0,4.7E1,4E0,9E0,1.5E1,1E1,3E0,1.7E1,2E0,7E0,4E0,2E1,2.8E1,3E0,1.5E1,3.5E1,1.2E1,6E0,9E0,1E1,1E1,1.8E1,1E1,3.3E1,2E0,4E0,8E0,4E0,6E0,1.1E1,7E0,2E0,3.1E1,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.2190764E-2,-1.8730694E-1,2.753607E-1,-2.6419362E-1,5.672748E-2,1.16373606E-1,5.309576E-1,-3.1258324E-1,-1.8713716E-2,-1.0737888E-1,1.2148461E-1,-1.23983465E-1,1.630511E-1,4.4448983E-2,4.534144E-1,-2.807187E-1,-2.4374343E-2,1.0378545E-1,-1.09028615E-1,-7.691218E-3,4.587331E-4,1.4246969E-1,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,2.1820955E-1,-6.480758E-3,5.3746756E-3,4.961946E-1,-2.0117857E-2,-2.3034851E-1,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1.7839447E-1,6.969961E-2,3.162982E-1,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,-2.1047248E-1,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.4450694E-1,9.220269E-3,-2.461917E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,-7.4021526E-2,-2.9362077E-1,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,-1,-1,37,39,-1,41,-1,43,-1,-1,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,51,-1,-1,-1,-1,53,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[1.2237465E1,2.848199E0,3.6767783E0,1.360198E0,4.0624598E-1,6.7093295E-1,8.615341E-1,5.956049E-1,2.3258556E-1,9.1160655E-2,1.3540426E-1,3.5389587E-1,4.596188E-1,0E0,3.909912E-1,6.838703E-1,0E0,1.7328064E-1,1.0814984E-1,0E0,0E0,1.2905765E-1,0E0,0E0,0E0,5.169381E-1,1.1691285E-1,0E0,1.8245935E-1,0E0,3.9284968E-1,0E0,0E0,0E0,0E0,0E0,1.09398365E-1,1.9227347E-1,2.0228004E-1,0E0,0E0,0E0,0E0,0E0,3.2970786E-1,0E0,0E0,0E0,0E0,1.4234447E-1,0E0,4.1922283E-1,0E0,0E0,0E0,2.4974391E-1,2.2993183E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,25,25,26,26,28,28,30,30,36,36,37,37,38,38,44,44,49,49,51,51,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,-1,-1,38,40,-1,42,-1,44,-1,-1,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,52,-1,-1,-1,-1,54,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[3.3103763E2,4.4269168E-1,1.7842576E-1,2.727992E6,2.1123085E8,7.5E2,1.431E4,1E0,1.8000048E10,1.3402277E0,1.1771224E8,3.9378E5,4.183787E0,4.4448983E-2,1E0,2.647867E-4,-2.4374343E-2,2.686459E6,1.6868966E8,-7.691218E-3,4.587331E-4,2.25E2,-4.0970817E-3,3.8234557E-3,-1.2203717E-2,4.5971466E2,4.9863962E2,5.3746756E-3,2.9743274E8,-2.0117857E-2,1E0,-1.826782E-3,9.979132E-3,-7.1767885E-3,1.8741065E-3,-4.4707387E-5,1E0,1.1572553E7,8.312613E5,4.342951E-3,-3.7167845E-3,1.0622292E-2,2.3589615E-2,-2.6361788E-2,2.933E3,9.140778E-3,7.63858E-5,6.791509E-3,-2.7498945E-3,4.488E3,9.220269E-3,6.7897725E-1,-2.9609771E-3,7.894639E-3,2.3833603E-2,2.7287993E0,6.767991E5,8.274363E-3,-6.0741752E-3,-1.3709839E-2,-8.685913E-4],"split_indices":[53,43,39,1,7,0,9,82,5,42,33,1,58,0,17,40,0,30,7,0,0,0,0,0,0,53,53,0,5,0,67,0,0,0,0,0,82,46,48,0,0,0,0,0,10,0,0,0,0,2,0,57,0,0,0,55,29,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.5E2,9.1E1,1.14E2,3.6E1,5.7E1,3.4E1,9.5E1,1.9E1,1E1,2.6E1,9E0,4.8E1,3E0,3.1E1,8.6E1,9E0,8E0,1.1E1,6E0,4E0,2.4E1,2E0,4E0,5E0,3.6E1,1.2E1,4E0,2.7E1,1.7E1,6.9E1,4E0,4E0,8E0,3E0,5E0,1.9E1,1.5E1,2.1E1,5E0,7E0,5E0,2.2E1,2E0,6.7E1,1.6E1,3E0,9E0,6E0,8E0,1.3E1,5.3E1,1.4E1,3E0,5E0,1.2E1,4.1E1,2E0,1E1,3.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.1875391E-3,-1.9125406E-1,2.4149728E-1,-3.2296014E-1,-4.532346E-2,9.999164E-2,4.7495925E-1,-3.5961127E-1,-3.3613036E-3,4.6751868E-2,-1.47186E-1,-3.1877293E-3,1.2955628E-1,1.2192243E-1,6.471233E-1,-2.8752145E-1,-5.190118E-1,-3.20057E-2,7.1740914E-3,-1.8803763E-1,5.201352E-3,1.4700729E-1,-4.1586305E-3,6.593719E-4,9.316681E-3,9.0452635E-1,5.182496E-1,4.057908E-2,-3.2754722E-1,-5.1887655E-3,-5.5908924E-1,3.8056083E-3,-3.9105513E-3,-2.4499905E-1,-2.4137127E-3,4.0515624E-2,1.8407537E-1,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,-2.7528796E-1,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,1.9600414E-1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,1.5410087E-1,4.4805845E-3,9.989807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,-1,33,-1,35,-1,-1,-1,37,39,41,43,-1,45,-1,-1,47,-1,49,51,-1,-1,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,57,-1,-1],"loss_changes":[1.1762257E1,2.7338152E0,3.6525607E0,6.483345E-1,6.540341E-1,3.7268925E-1,2.4738722E0,5.976982E-1,0E0,3.480822E-1,3.8151592E-1,0E0,2.5336754E-1,1.4349061E-1,4.6006012E-1,6.3051796E-1,2.3819494E-1,1.6576359E-1,0E0,2.0569742E-1,0E0,2.1754622E-1,0E0,0E0,0E0,2.0789146E-1,1.3972569E-1,1.1563162E-1,1.779995E-1,0E0,1.1710739E-1,0E0,0E0,1.0057318E-1,0E0,2.0077212E-1,1.1754835E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.099534E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.906554E-2,0E0,0E0,0E0,0E0,8.8078976E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,21,21,25,25,26,26,27,27,28,28,30,30,33,33,35,35,36,36,44,44,51,51,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,-1,34,-1,36,-1,-1,-1,38,40,42,44,-1,46,-1,-1,48,-1,50,52,-1,-1,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,58,-1,-1],"split_conditions":[3.5300907E-1,1.9415865E2,1.5972575E0,1.848519E-2,1.560615E1,1.8681856E8,2.744E3,5.039375E-3,-3.3613036E-3,3.165913E6,5.4878284E2,-3.1877293E-3,9.723711E0,6.765595E7,2.7479675E-1,1.0395554E5,1.47887E5,9.165388E4,7.1740914E-3,3.1692016E0,5.201352E-3,2.2599022E8,-4.1586305E-3,6.593719E-4,9.316681E-3,4.063E3,2.0467837E0,4.56E2,1.7565872E-3,-5.1887655E-3,6.6037946E2,3.8056083E-3,-3.9105513E-3,3.2829E4,-2.4137127E-3,1.19532734E5,1E0,1.5193312E-2,4.5033395E-2,2.4348166E-2,9.166431E-3,8.03364E-3,-3.3689898E-3,-2.0390192E-2,4.5E2,-2.5996326E-2,-8.27393E-3,-5.8151674E-3,-1.3457775E-2,1.0246426E-2,-9.766758E-4,5.7E1,-1.5855534E-3,-1.3729388E-2,-6.693243E-3,1.225977E-2,2.63E2,4.4805845E-3,9.989807E-3],"split_indices":[43,53,42,39,57,7,2,39,0,1,53,0,58,5,39,29,1,34,0,59,0,12,0,0,0,2,54,0,59,0,4,0,0,9,0,29,110,0,0,0,0,0,0,0,10,0,0,0,0,0,0,8,0,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,7.4E1,6.8E1,7E1,4.1E1,6.4E1,1E1,3.6E1,3.2E1,1E1,6E1,1.4E1,2.7E1,4.6E1,1.8E1,2.2E1,1.4E1,2.8E1,4E0,5.6E1,4E0,7E0,7E0,7E0,2E1,5E0,4.1E1,2E0,1.6E1,7E0,1.5E1,1.9E1,9E0,1.5E1,4.1E1,2E0,5E0,1.7E1,3E0,2E0,3E0,9E0,3.2E1,1.4E1,2E0,8E0,1.1E1,3E0,1.2E1,3.9E1,2E0,2.3E1,9E0,1.1E1,2.8E1,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.9416563E-3,-1.3463952E-1,3.7984976E-1,-2.3853315E-1,1.360329E-2,1.961173E-1,5.876671E-1,-3.1009814E-1,-1.1053116E-1,-3.428409E-2,1.00340895E-1,2.3631513E-2,1.5987208E-1,1.1244957E-2,6.9594884E-1,-6.4688744E-3,-3.5531205E-1,1.6310263E-3,-2.0668645E-1,-7.293062E-2,1.3145976E-1,9.800815E-3,-3.9949227E-2,-4.0140087E-3,1.8721226E-1,4.3395724E-2,2.3986792E-2,3.5975182E-3,-3.7373397E-1,-1.2723958E-2,-3.946153E-3,-9.6303485E-2,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,2.38724E-1,1.5173156E-3,-3.9501062E-1,-4.1339686E-3,8.0993416E-4,-1.22839086E-1,1.2143473E-2,4.7381404E-3,-4.0761718E-1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,-4.2140347E-1,-2.2882612E-2,-1.5165369E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,-1,29,31,33,-1,35,-1,37,-1,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,43,-1,45,-1,-1,47,-1,-1,49,-1,-1,-1,-1,51,-1,-1],"loss_changes":[1.2767532E1,2.8833227E0,2.3666296E0,9.7337484E-1,3.2742375E-1,3.641218E-1,8.926077E-1,4.596181E-1,5.9082735E-1,3.3626413E-1,4.9703383E-1,0E0,2.5133562E-1,0E0,4.1397285E-1,0E0,4.6627617E-1,0E0,2.203567E-1,2.0018259E-1,1.2457131E-1,0E0,2.0385656E-1,0E0,2.3512948E-1,0E0,0E0,0E0,2.7126503E-1,0E0,0E0,1.1810273E-1,0E0,0E0,0E0,0E0,0E0,8.816862E-2,0E0,1.6809559E-1,0E0,0E0,9.8291665E-2,0E0,0E0,9.2234135E-2,0E0,0E0,0E0,0E0,1.4487553E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,18,18,19,19,20,20,22,22,24,24,28,28,31,31,37,37,39,39,42,42,45,45,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,-1,30,32,34,-1,36,-1,38,-1,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,44,-1,46,-1,-1,48,-1,-1,50,-1,-1,-1,-1,52,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.427E3,8.9397516E4,3.856E3,2.4747229E2,1.3048244E1,1.4191151E6,9E0,2.3631513E-2,7.5E2,1.1244957E-2,2.7479675E-1,-6.4688744E-3,3.8757736E-7,1.6310263E-3,3.03E9,4.222002E8,1.31903E5,9.800815E-3,1.3116745E1,-4.0140087E-3,4.183787E0,4.3395724E-2,2.3986792E-2,3.5975182E-3,7.4E1,-1.2723958E-2,-3.946153E-3,2.6639402E-1,5.2056527E-3,1.4899529E-3,1.1734482E-2,5.593701E-3,-5.237527E-3,4.3147E5,1.5173156E-3,9.4983295E2,-4.1339686E-3,8.0993416E-4,4.9615598E-1,1.2143473E-2,4.7381404E-3,4.699202E1,-3.0564563E-3,-8.539529E-3,-3.185156E-3,-6.873583E-3,9.6713684E1,-2.2882612E-2,-1.5165369E-2],"split_indices":[53,42,39,53,2,34,2,56,57,34,3,0,0,0,39,0,39,0,5,7,1,0,57,0,58,0,0,0,3,0,0,43,0,0,0,0,0,9,0,4,0,0,40,0,0,53,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.86E2,6.4E1,1.09E2,7.7E1,3.5E1,2.9E1,6.9E1,4E1,5E1,2.7E1,2E0,3.3E1,8E0,2.1E1,1.6E1,5.3E1,1.6E1,2.4E1,4.1E1,9E0,1.4E1,1.3E1,3E0,3E1,5E0,1.6E1,2E0,5.1E1,1.3E1,1.1E1,3.7E1,4E0,6E0,3E0,4E0,9E0,2.2E1,8E0,4.7E1,4E0,7E0,3E1,1.6E1,6E0,4.5E1,2E0,1.1E1,1.9E1,3E0,4.2E1,1.5E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[4.442649E-3,-1.2406673E-1,3.1314126E-1,-2.231523E-1,2.7577164E-2,1.5845458E-1,5.6371045E-1,-2.9410875E-1,-5.8644965E-2,-2.6454885E-2,1.4853756E-1,2.294401E-2,1.3163756E-1,2.7344096E-3,6.19114E-1,-9.231322E-2,-3.2586154E-1,-1.3393481E-1,2.8952213E-2,-4.4858553E-3,1.6983777E-2,1.7551805E-1,-3.9204443E-3,2.2740224E-1,3.826856E-2,1.9222386E-2,8.226514E-1,-7.2305724E-3,7.988117E-3,-3.752356E-1,-1.8137792E-1,3.932153E-3,-7.2262976E-3,-2.8384456E-3,1.1973143E-1,4.0585145E-2,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,-1.2312204E-1,5.3003617E-3,1.6423428E-2,4.1150242E-2,-3.9276025E-1,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,9.395361E-2,3.7756322E-3,-8.526156E-3,-3.2505807E-1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,-3.9313173E-1,-1.9234711E-2,-7.488049E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,-1,39,41,-1,43,-1,-1,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,53,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,57,-1,-1,59,-1,-1,-1,-1,61,-1,-1],"loss_changes":[1.0392895E1,2.7859118E0,2.895176E0,1.2901125E0,4.8757413E-1,4.1345346E-1,7.6624584E-1,4.6970892E-1,2.3470895E-1,1.741834E-1,1.6369265E-1,0E0,4.1154587E-1,0E0,5.829191E-1,2.727965E-1,4.0990114E-1,1.4968318E-1,1.5284702E-1,0E0,1.21554464E-1,9.423089E-2,0E0,2.5063312E-1,3.5061046E-1,0E0,2.3580599E-1,0E0,0E0,1.4392567E-1,3.3857572E-1,0E0,0E0,0E0,1.4192495E-1,1.1094484E-1,0E0,0E0,0E0,0E0,0E0,1.5323834E-1,0E0,0E0,0E0,2.5782776E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.29731E-2,0E0,0E0,1.570108E-1,0E0,0E0,0E0,0E0,1.24889135E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,26,26,29,29,30,30,34,34,35,35,41,41,45,45,52,52,55,55,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,-1,40,42,-1,44,-1,-1,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,54,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,58,-1,-1,60,-1,-1,-1,-1,62,-1,-1],"split_conditions":[3.705974E2,2.1619815E-1,2.0224316E-1,4.5537444E9,2.427E3,8.9397516E4,1E0,1.1563724E8,2.0256741E2,1.503E3,5.4281917E0,2.294401E-2,1.3844932E7,2.7344096E-3,1.6507233E3,3.160848E6,2.2348747E2,1.903E3,4.231592E6,-4.4858553E-3,2.302E3,3.5734247E2,-3.9204443E-3,5.264E4,1.1218195E6,1.9222386E-2,1.61628E5,-7.2305724E-3,7.988117E-3,9.0814667E2,4.1081388E5,3.932153E-3,-7.2262976E-3,-2.8384456E-3,4.3560153E5,6.4796996E-1,-5.586704E-3,8.611193E-3,-9.2164805E-4,1.4815791E-2,5.020672E-3,1.4153711E-3,5.3003617E-3,1.6423428E-2,4.1150242E-2,1.549E3,-6.217078E-3,-1.09454645E-2,2.758889E-3,-1.9478701E-3,9.136003E-3,-1.1834154E-3,1.5130641E0,3.7756322E-3,-8.526156E-3,1.5E1,-2.3320947E-2,5.983659E-4,7.14614E-3,-9.4766645E-3,1E0,-1.9234711E-2,-7.488049E-3],"split_indices":[53,42,39,5,2,34,17,5,53,2,55,0,46,0,4,9,53,0,1,0,2,53,0,9,48,0,29,0,0,56,29,0,0,0,33,40,0,0,0,0,0,39,0,0,0,2,0,0,0,0,0,0,57,0,0,3,0,0,0,0,112,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,1.84E2,7.6E1,1.11E2,7.3E1,4.8E1,2.8E1,7.7E1,3.4E1,5.1E1,2.2E1,2E0,4.6E1,3E0,2.5E1,1.1E1,6.6E1,1.8E1,1.6E1,1.8E1,3.3E1,2E1,2E0,2.2E1,2.4E1,1.5E1,1E1,9E0,2E0,4.8E1,1.8E1,2E0,1.6E1,8E0,8E0,2.9E1,4E0,1.8E1,2E0,1E1,1.2E1,8E0,1.6E1,3E0,7E0,4.4E1,4E0,1.4E1,4E0,3E0,5E0,1.3E1,1.6E1,2E0,6E0,3.2E1,1.2E1,8E0,8E0,1.4E1,1.8E1,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.2187884E-2,-1.0560222E-1,2.8632665E-1,-1.6603857E-1,6.5966286E-2,1.4586735E-1,4.827894E-1,-2.163441E-1,3.5765186E-2,-2.1459074E-3,9.6117355E-2,2.2276511E-2,1.16351E-1,3.9712414E-3,5.3956425E-1,-1.9085151E-1,-4.2924163E-1,1.310548E-1,-6.960556E-2,8.844785E-3,5.8431923E-2,-3.4577996E-2,1.8520117E-1,1.25307795E-2,2.6944714E-2,-3.2403958E-1,-1.5634167E-1,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,5.7000898E-2,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,4.627612E-2,-1.0834768E-2,2.3908277E-1,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,-1.3881442E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,-1.12805806E-1,-3.083408E-1,-7.1671814E-2,-2.4239872E-1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,43,-1,-1,-1,-1,45,-1,47,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1],"loss_changes":[7.844618E0,1.7736785E0,1.9440131E0,1.2868705E0,1.6094884E-1,4.141271E-1,6.1820793E-1,4.6623707E-1,2.7018133E-1,0E0,1.327295E-1,0E0,4.4284636E-1,0E0,3.3383512E-1,3.7219834E-1,2.928431E-1,1.1195864E-1,1.3693142E-1,0E0,1.6056333E-1,2.5339377E-1,1.8323988E-1,0E0,0E0,1.318934E-1,3.0327547E-1,0E0,0E0,0E0,1.019789E-1,0E0,0E0,0E0,0E0,1.6095355E-1,0E0,1.4495695E-1,0E0,0E0,0E0,0E0,2.8267586E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2379E-1,1.0256362E-1,2.1890551E-1,9.2020154E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,30,30,35,35,37,37,42,42,49,49,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,44,-1,-1,-1,-1,46,-1,48,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,4.9615598E-1,1.9530877E-1,2.727992E6,1.8681856E8,8.9397516E4,1.2354571E0,1E0,1.6E1,-2.1459074E-3,2.0383387E0,2.2276511E-2,1.479938E6,3.9712414E-3,7.2610445E8,3.3617156E-4,2.22105E5,1.2051221E0,1.7108529E3,8.844785E-3,2.2599022E8,2.629593E0,1.8990228E0,1.25307795E-2,2.6944714E-2,1.7565872E-3,3.4955364E7,-2.4506203E-3,-2.2495558E-2,1.10385325E-2,4.654E3,-5.1111244E-3,5.5782446E-3,-1.0233678E-4,6.6581275E-3,6.2143734E4,-1.0834768E-2,1.6851284E7,2.5711106E-3,-1.9701567E-2,-9.718517E-3,-1.8884597E-2,3.219065E-1,5.4832366E-3,-3.3770227E-3,-3.2335538E-3,6.9358586E-3,1.2943303E-2,3.9228504E-3,8.3170245E2,2.07298E5,6.214038E6,1.2271346E1,-1.8095627E-2,-6.124552E-3,-4.122259E-3,4.8529482E-3,-1.2064778E-3,-1.1926983E-2],"split_indices":[53,40,39,1,7,34,54,82,3,0,59,0,33,0,12,40,1,54,4,0,12,55,54,0,0,59,7,0,0,0,2,0,0,0,0,29,0,1,0,0,0,0,43,0,0,0,0,0,0,56,1,9,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.69E2,7.2E1,1.25E2,4.4E1,4.3E1,2.9E1,1E2,2.5E1,9E0,3.5E1,2E0,4.1E1,4E0,2.5E1,9.1E1,9E0,1.3E1,1.2E1,8E0,2.7E1,1.3E1,2.8E1,7E0,1.8E1,1.7E1,7.4E1,2E0,7E0,4E0,9E0,1E1,2E0,1.7E1,1E1,1E1,3E0,1.9E1,9E0,6E0,1.1E1,3E0,7.1E1,6E0,3E0,5E0,5E0,1.3E1,6E0,6.3E1,8E0,4.9E1,1.4E1,4E0,4E0,4.4E1,5E0,2E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[8.535432E-5,-1.3134183E-1,2.4541517E-1,-2.1222556E-1,-3.1241742E-3,1.1487452E-1,4.674134E-1,-2.7328297E-1,-3.8918845E-2,-8.365333E-2,9.0494335E-2,2.394816E-3,1.6543052E-1,2.984512E-1,6.282826E-1,-2.368147E-1,-2.039022E-2,-7.2169475E-2,5.6061996E-3,-1.5359263E-1,5.1900256E-2,2.500614E-4,6.980609E-3,5.4644295E-3,-7.968769E-2,1.8786167E-1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,7.535615E-1,1.2825073E-2,-2.2865131E-2,-2.539767E-1,4.7256513E-3,-1.1367583E-1,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.703592E-1,1.374485E-1,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,-2.2705331E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,-2.3829083E-1,-2.8504506E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,-1,-1,-1,39,41,-1,-1,-1,43,-1,45,47,-1,49,-1,-1,-1,-1,-1,-1,51,53,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1],"loss_changes":[8.318686E0,1.7416182E0,2.5544882E0,1.0792751E0,5.0514543E-1,3.2942206E-1,7.09455E-1,4.2694187E-1,1.6511148E-1,3.4927028E-1,1.8162525E-1,2.0153141E-1,2.3894668E-1,1.7641377E-1,4.4881153E-1,2.3653197E-1,0E0,1.9207498E-1,0E0,2.4937719E-1,1.370348E-1,0E0,0E0,0E0,2.0430191E-1,1.1672115E-1,0E0,0E0,0E0,1.8685484E-1,0E0,1.2997259E-1,1.903727E-1,0E0,2.3296595E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.380126E-2,9.920034E-2,0E0,0E0,0E0,0E0,0E0,1.8071914E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.499064E-1,1.1059618E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,24,24,25,25,29,29,31,31,32,32,34,34,41,41,42,42,48,48,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,-1,-1,-1,40,42,-1,-1,-1,44,-1,46,48,-1,50,-1,-1,-1,-1,-1,-1,52,54,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1],"split_conditions":[3.4021906E2,2.5106274E-2,1.8829846E-1,1.781973E6,9.50379E5,3.359399E-1,5.6E1,2.418166E0,1.22E2,1.5225257E-1,5.3164E4,1.1936242E7,5.615313E3,2.598887E7,3.2307692E0,1.1623532E6,-2.039022E-2,2.207E3,5.6061996E-3,2.587E3,8.180814E7,2.500614E-4,6.980609E-3,5.4644295E-3,1.0520055E5,5.7E1,-3.6401555E-3,1.4552465E-2,5.3134456E-4,1.4475617E0,1.2825073E-2,2.1426034E5,1.4614057E-3,4.7256513E-3,2.4616451E0,-4.4478294E-3,-1.6864581E-2,4.8584836E-3,-4.78538E-3,6.920267E-3,-6.2490604E-3,2.7307575E5,1.0899524E3,1.7607713E-2,3.9098855E-2,6.022819E-3,-6.0138283E-3,-1.8731354E-2,4.2446044E-1,-6.649831E-3,6.827729E-3,6.8941894E-3,1.6370468E-2,4.1440513E-4,7.373306E-3,2.058034E-3,1.3496346E0,1.1306552E-1,-6.958366E-3,-1.3107529E-2,-1.6079455E-3],"split_indices":[53,39,39,1,9,43,8,54,8,39,30,46,4,9,55,46,0,0,0,2,33,0,0,0,34,8,0,0,0,54,0,29,59,0,55,0,0,0,0,0,0,29,4,0,0,0,0,0,57,0,0,0,0,0,0,0,54,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.56E2,1.67E2,8.9E1,1.02E2,6.5E1,5.7E1,3.2E1,7.5E1,2.7E1,3.5E1,3E1,1.8E1,3.9E1,1.7E1,1.5E1,6.5E1,1E1,2.3E1,4E0,2.3E1,1.2E1,1.4E1,1.6E1,7E0,1.1E1,3.6E1,3E0,1.5E1,2E0,1E1,5E0,5E0,6E1,4E0,1.9E1,2E1,3E0,9E0,3E0,2E0,9E0,1.2E1,2.4E1,4E0,6E0,2E0,3E0,6E0,5.4E1,1.7E1,2E0,7E0,5E0,5E0,1.9E1,2E0,5.2E1,3.1E1,2.1E1,2.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-7.534086E-3,-8.694848E-2,3.8782644E-1,-1.7279062E-1,7.5343594E-2,1.2632059E-1,5.605976E-1,-2.4380171E-1,-4.4355273E-2,1.0239507E-1,-7.070103E-2,-8.078534E-4,9.607972E-3,6.888959E-1,4.1058496E-1,-2.8629518E-1,-1.11263916E-1,-1.6469195E-2,-1.506756E-2,4.8591096E-2,1.8556008E-1,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,-6.6435128E-3,-3.0423003E-1,-1.0063401E-2,2.9212602E-3,-4.0811297E-2,7.316056E-3,-2.3700453E-2,1.1168387E-1,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,-3.2206443E-1,-3.669968E-3,-6.68522E-2,7.2118444E-3,4.6604417E-2,-4.6691555E-3,8.108817E-3,-7.8055467E-3,-3.4489202E-1,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,-3.753321E-1,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,47,-1,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,-1],"loss_changes":[7.88305E0,2.924702E0,1.8267727E0,1.2386718E0,2.9564223E-1,2.4874002E-1,1.9830894E-1,4.6376085E-1,4.8201522E-1,2.6840913E-1,1.2740509E-1,0E0,0E0,3.1174803E-1,9.529376E-2,3.2900333E-1,5.045887E-1,0E0,2.2792198E-1,1.8052885E-1,1.6800141E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.805466E-2,2.1632099E-1,0E0,0E0,2.3643395E-1,0E0,1.16251014E-1,1.8746912E-1,0E0,0E0,0E0,0E0,1.5240574E-1,0E0,1.1973961E-1,0E0,9.648324E-2,0E0,0E0,1.9439551E-1,1.6225243E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4404154E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,27,27,28,28,31,31,33,33,34,34,39,39,41,41,43,43,46,46,47,47,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,48,-1,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,-1],"split_conditions":[1.311002E0,3.051E3,2.787E3,1.5157113E-2,1.50321455E1,2.024E3,2.9898608E-1,2.2348747E2,2.718E4,4.871647E2,1.471E4,-8.078534E-4,9.607972E-3,1.792184E0,3.4048372E6,1E1,2.8041378E5,-1.6469195E-2,1.3431462E7,6.477987E-1,1.8751362E7,-5.779751E-3,3.7484926E-3,1.6429951E-2,3.7453175E-2,2.1347104E-2,9.786794E-3,7.1E1,6.8E1,-1.0063401E-2,2.9212602E-3,1.00849255E3,7.316056E-3,4.303E3,4.99E3,1.0969074E-2,2.9807193E-3,-5.8213337E-3,5.338141E-3,1E0,-3.669968E-3,2.1799392E6,7.2118444E-3,2.5063604E7,-4.6691555E-3,8.108817E-3,6.074E3,4.239E3,-7.069322E-3,-5.1215505E-3,-1.8372193E-4,4.659592E-3,-3.7263283E-3,-4.290292E-3,8.988804E-3,3.1872916E2,-8.232157E-3,-1.1219157E-2,-1.8725993E-2],"split_indices":[40,2,2,39,58,2,39,53,9,53,0,0,0,43,46,10,29,0,9,58,45,0,0,0,0,0,0,0,3,0,0,4,0,2,2,0,0,0,0,112,0,49,0,46,0,0,2,10,0,0,0,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,2.08E2,4.1E1,1.36E2,7.2E1,1.7E1,2.4E1,8.7E1,4.9E1,6.1E1,1.1E1,7E0,1E1,1.1E1,1.3E1,6.5E1,2.2E1,3E0,4.6E1,3.8E1,2.3E1,8E0,3E0,5E0,6E0,8E0,5E0,4E0,6.1E1,1.3E1,9E0,4.1E1,5E0,1.8E1,2E1,1.4E1,9E0,2E0,2E0,5.6E1,5E0,3.7E1,4E0,1E1,8E0,1.2E1,8E0,4.8E1,8E0,2E1,1.7E1,7E0,3E0,6E0,2E0,3.9E1,9E0,1.4E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.6526852E-2,-1.3732727E-1,1.8663803E-1,-2.051646E-1,-2.9185692E-2,7.73927E-2,3.695984E-1,-2.6628572E-1,-9.389659E-2,-8.110631E-2,7.3622145E-2,4.680393E-3,1.8481968E-1,1.6930054E-3,3.9849573E-1,-2.2754145E-1,-2.0849528E-2,6.2429474E-4,-1.7701784E-1,-3.0688336E-1,-5.0080888E-2,4.623225E-3,-4.2899745E-3,1.03257135E-1,-1.1058583E-1,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,-2.4562204E-1,7.5825364E-3,-1.0521853E-2,-6.476905E-2,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,1.3644478E-1,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,-2.6290497E-1,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,-2.9959735E-1,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,31,33,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,-1,43,-1,-1,-1,-1,45,-1,-1,-1,-1,47,-1,-1,-1,-1,49,-1,-1,-1],"loss_changes":[6.3821125E0,1.1897857E0,1.9101899E0,6.5507936E-1,3.4759343E-1,4.8343337E-1,3.2867527E-1,4.1569996E-1,3.3530772E-1,2.8583866E-1,1.3266498E-1,4.4319636E-1,5.4417163E-1,0E0,2.268424E-1,4.489975E-1,0E0,0E0,1.3546717E-1,1.6951728E-1,1.4221326E-1,0E0,0E0,1.3188584E-1,1.2820828E-1,0E0,0E0,0E0,0E0,1.5322685E-1,0E0,0E0,9.677847E-2,0E0,0E0,0E0,0E0,1.2525946E-1,0E0,0E0,0E0,0E0,2.1090531E-1,0E0,0E0,0E0,0E0,1.8816972E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,18,18,19,19,20,20,23,23,24,24,29,29,32,32,37,37,42,42,47,47],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,32,34,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,-1,44,-1,-1,-1,-1,46,-1,-1,-1,-1,48,-1,-1,-1,-1,50,-1,-1,-1],"split_conditions":[3.3103763E2,1.1670399E-1,1.7842576E-1,1.9415865E2,2.7538432E8,6.483E3,1E0,4.1938826E-2,3.0364912E5,3.3789E4,1.1771224E8,1.1936242E7,6.2143734E4,1.6930054E-3,7.2610445E8,1.1661899E8,-2.0849528E-2,6.2429474E-4,1.3469651E3,2.151E3,1.2473118E0,4.623225E-3,-4.2899745E-3,1.9816086E0,1.4813267E3,2.187987E-3,1.5346674E-2,1.1007364E-2,2.051293E-2,6E0,7.5825364E-3,-1.0521853E-2,2.8659306E0,-2.288021E-3,-2.0032069E-2,2.4146098E-3,-3.6459875E-3,3.8692224E8,-3.1377056E-3,-7.1148453E-3,9.379643E-4,-2.772861E-3,4.426498E0,-5.4085446E-3,4.140586E-3,7.279652E-3,-3.099007E-3,1.1078758E0,-5.446295E-3,-4.1352003E-3,-1.4303169E-2],"split_indices":[53,40,39,53,7,2,17,43,34,9,33,46,29,0,12,7,0,0,4,2,54,0,0,54,56,0,0,0,0,10,0,0,59,0,0,0,0,7,0,0,0,0,57,0,0,0,0,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.62E2,9.6E1,9.9E1,6.3E1,6.1E1,3.5E1,6.3E1,3.6E1,4.2E1,2.1E1,3.7E1,2.4E1,3E0,3.2E1,5.5E1,8E0,1.6E1,2E1,4E0,3.8E1,1.8E1,3E0,2E1,1.7E1,1.4E1,1E1,1.2E1,2E1,5.3E1,2E0,1.2E1,8E0,2E0,2E0,9E0,2.9E1,1.7E1,3E0,1.2E1,5E0,5E0,4.8E1,6E0,2E0,1.5E1,2E0,3.7E1,1.1E1,5E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[6.752704E-3,-1.0890242E-1,2.1311827E-1,-1.7942376E-1,-4.630112E-3,8.463161E-2,4.3359125E-1,-2.6526305E-1,-4.0561736E-2,-8.0085285E-2,6.466574E-2,1.7120663E-2,6.155428E-2,2.8981546E-1,6.182841E-1,-1.9980457E-1,-3.8043132E-1,2.1313105E-2,-7.922915E-3,-1.4972365E-1,5.4258812E-2,-3.958611E-6,5.6440667E-3,-6.662196E-2,1.1769473E-1,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,-1.3539992E-1,-2.9197437E-1,-4.6381897E-1,-6.432207E-3,-5.7250984E-2,5.7095457E-3,-7.0690963E-4,-2.4091765E-1,-1.8624915E-3,6.908497E-3,7.913083E-2,-2.8256294E-3,-1.419902E-1,4.703381E-3,1.525669E-1,-8.453891E-4,2.933514E-3,-7.012109E-3,-3.371898E-1,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,7.1813464E-2,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,-1,41,43,-1,-1,-1,-1,45,47,49,-1,51,-1,-1,53,-1,-1,55,-1,57,-1,59,-1,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,65,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1809554E0,1.2198377E0,2.597793E0,1.1693025E0,3.607325E-1,4.0189618E-1,7.2366E-1,3.8385105E-1,3.4410596E-1,3.1685543E-1,1.5055917E-1,0E0,4.1800576E-1,1.5071511E-1,4.769044E-1,2.0384932E-1,3.3971214E-1,2.4841867E-1,0E0,2.639835E-1,1.2779592E-1,1.0247372E-1,0E0,2.5190163E-1,1.941486E-1,0E0,0E0,0E0,0E0,9.141275E-2,1.3482976E-1,1.2367654E-1,0E0,1.1534542E-1,0E0,0E0,1.0435748E-1,0E0,0E0,1.1921386E-1,0E0,1.5446314E-1,0E0,1.8064415E-1,0E0,1.0156908E-1,0E0,1.0845828E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.946659E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,29,29,30,30,31,31,33,33,36,36,39,39,41,41,43,43,45,45,47,47,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,-1,42,44,-1,-1,-1,-1,46,48,50,-1,52,-1,-1,54,-1,-1,56,-1,58,-1,60,-1,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,66,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4021906E2,1.7613442E-1,1.8829846E-1,1.9415865E2,9.50379E5,8.9397516E4,1.623319E3,1.607E3,2.142943E1,1.2185057E0,6.6727E4,1.7120663E-2,3.359399E-1,2.0270285E8,2.65826E6,4.3399894E5,6.608892E2,6.135367E9,-7.922915E-3,2.8507206E-1,1.7468556E2,1.2495134E6,5.6440667E-3,5.4878284E2,3.2383578E7,2.58879E-3,1.4179813E-2,3.2457218E-2,1.06330365E-2,1.1623532E6,3.342E3,2.1710082E7,-6.432207E-3,2.2E1,5.7095457E-3,-7.0690963E-4,6.2684685E-2,-1.8624915E-3,6.908497E-3,1.6537195E0,-2.8256294E-3,1.409674E7,4.703381E-3,1.7111111E0,-8.453891E-4,4.14E2,-7.012109E-3,1E0,-3.1433352E-3,-2.196492E-2,-6.021441E-3,-4.8626456E-3,2.6465752E-3,-1.5119841E-2,-5.744365E-3,7.089132E-3,-2.8640376E-3,1.920513E-3,-8.823391E-3,9.985977E-3,1.0685258E6,-5.571306E-3,5.7846652E-3,-1.7685302E-2,-6.4244005E-3,-1.0005281E-3,5.4799393E-3],"split_indices":[53,42,39,53,9,34,4,2,57,42,30,0,43,7,48,29,4,5,0,43,53,46,0,53,1,0,0,0,0,46,0,33,0,3,0,0,39,0,0,54,0,46,0,54,0,2,0,112,0,0,0,0,0,0,0,0,0,0,0,0,34,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.65E2,9.2E1,9.8E1,6.7E1,5.9E1,3.3E1,6E1,3.8E1,3.2E1,3.5E1,3E0,5.6E1,2E1,1.3E1,4E1,2E1,2.7E1,1.1E1,2.1E1,1.1E1,1.8E1,1.7E1,1.7E1,3.9E1,3E0,1.7E1,9E0,4E0,2.5E1,1.5E1,1.4E1,6E0,1.6E1,1.1E1,9E0,1.2E1,6E0,5E0,8E0,1E1,1.2E1,5E0,3.1E1,8E0,4E0,2.1E1,1.2E1,3E0,1.2E1,2E0,1.1E1,5E0,5E0,7E0,5E0,3E0,3E0,9E0,1.5E1,1.6E1,2E0,2E0,8E0,4E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-4.591145E-3,-1.0742565E-1,2.1129104E-1,-1.6721527E-1,-3.2239808E-3,9.313745E-2,3.8930318E-1,-2.2656383E-1,-3.6957923E-2,-4.179782E-2,9.553272E-2,1.1645269E-1,-6.1090174E-3,9.100737E-3,4.8732993E-1,-1.9007076E-1,-3.574318E-1,8.0027975E-2,-3.7945511E-3,3.2766287E-3,-6.838066E-2,5.598663E-3,-2.234741E-3,-1.6062098E-3,1.4490457E-1,3.515733E-2,1.7769938E-2,2.4345317E-3,-2.1554193E-1,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,-8.616666E-2,7.324565E-3,2.3872483E-1,1.1299209E-1,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,-1.5753022E-1,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,-1,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.572733E0,1.0585915E0,1.6673958E0,8.2653E-1,2.4398685E-1,2.8807345E-1,4.6537733E-1,2.8546977E-1,2.1166551E-1,1.4805996E-1,9.310773E-2,2.0591748E-1,0E0,0E0,1.8203974E-1,2.947247E-1,2.1712995E-1,1.3969424E-1,0E0,0E0,1.7411983E-1,0E0,0E0,0E0,8.9793384E-2,0E0,0E0,1.5928355E-1,2.97374E-1,0E0,0E0,0E0,0E0,1.2090984E-1,0E0,9.095979E-2,1.5865728E-1,0E0,0E0,0E0,1.3265473E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,20,20,24,24,27,27,28,28,33,33,35,35,36,36,40,40],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,-1,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0471938E6,1.529448E-1,1.2711891E0,2.8704E4,9.74979E6,1.25440235E1,3.301E3,4.1295107E-2,1.0520055E5,4.46E2,1E0,3.359399E-1,-6.1090174E-3,9.100737E-3,1.431E4,1.1695156E5,1.1E1,2.00902E2,-3.7945511E-3,3.2766287E-3,2.65852E6,5.598663E-3,-2.234741E-3,-1.6062098E-3,8.312613E5,3.515733E-2,1.7769938E-2,1E0,2.647867E-4,-7.442835E-3,-2.020768E-2,-5.1234667E-3,5.9762294E-3,1.1343225E6,7.324565E-3,9.616974E2,3.7724147E8,-6.3963938E-3,5.2870996E-3,-1.4362599E-2,5.0090684E5,-5.437794E-3,4.8082587E-5,3.001421E-3,1.3625586E-2,7.5903274E-3,1.1593201E-3,-7.907431E-3,-2.1945388E-4],"split_indices":[49,40,40,11,9,58,2,43,34,11,82,43,0,0,9,29,3,53,0,0,1,0,0,0,48,0,0,8,40,0,0,0,0,34,0,4,7,0,0,0,29,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.69E2,8E1,1.07E2,6.2E1,4.9E1,3.1E1,7.3E1,3.4E1,4.5E1,1.7E1,4.5E1,4E0,1.2E1,1.9E1,5.9E1,1.4E1,1E1,2.4E1,8E0,3.7E1,1.4E1,3E0,7E0,3.8E1,2E0,1.7E1,7E0,5.2E1,6E0,8E0,2E0,8E0,3.5E1,2E0,8E0,3E1,3E0,4E0,1.6E1,3.6E1,2.4E1,1.1E1,3E0,5E0,1.7E1,1.3E1,3.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.739959E-3,-1.0096392E-1,2.598716E-1,-1.7680193E-1,-3.243053E-2,1.19685955E-1,4.226019E-1,-8.241986E-2,-2.739108E-1,-8.830396E-2,4.9990978E-2,1.8303644E-2,8.3055995E-2,2.0394288E-3,4.746282E-1,-1.0853977E-2,-3.597446E-2,-3.5187337E-1,-1.5231963E-1,-2.7551723E-1,-2.8254518E-2,9.033917E-3,5.8722845E-3,-8.941312E-3,1.0602045E-1,6.4259046E-1,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.3898046E-2,-4.325442E-3,3.7839024E-3,-3.718894E-2,1.2833083E-1,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,29,31,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4447184E0,8.7263286E-1,1.5462923E0,7.1065116E-1,4.1887993E-1,3.9337415E-1,6.0357904E-1,3.1546003E-1,3.161056E-1,5.992862E-1,1.2694931E-1,0E0,2.597763E-1,0E0,4.780264E-1,0E0,2.3357649E-1,1.2557936E-1,1.1205655E-1,3.2829583E-1,1.5759072E-1,9.657455E-2,0E0,0E0,1.6324815E-1,1.374054E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4665566E-1,0E0,0E0,1.1835869E-1,1.3544708E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,16,16,17,17,18,18,19,19,20,20,21,21,24,24,25,25,35,35,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,30,32,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,1.6375645E2,2.0224316E-1,1.69607E5,3.2764925E5,1.2E1,1.2354571E0,4.0561585E-3,5.3597E2,3.6477748E7,7.128343E5,1.8303644E-2,1.3E1,2.0394288E-3,2.9898608E-1,-1.0853977E-2,1.2245615E0,3.9647983E2,9.714086E0,6.779745E6,3.642097E6,9.598357E0,5.8722845E-3,-8.941312E-3,9.723711E0,2.4214827E5,1.4319989E-2,5.041815E-3,-3.4707645E-3,-1.0666101E-2,-1.9501286E-2,2.1405213E-3,-7.961441E-3,-5.674847E-3,-2.1045778E-2,2.142943E1,-4.325442E-3,3.7839024E-3,8.1262054E2,1.7021878E7,-4.0754387E-3,1.4860256E-2,3.2775253E-2,3.5986751E-3,-3.1506687E-3,-4.0486967E-3,3.499221E-3,6.559736E-3,-2.516474E-3],"split_indices":[53,53,39,1,29,3,54,42,4,12,29,0,3,0,39,0,54,4,48,12,33,57,0,0,58,29,0,0,0,0,0,0,0,0,0,57,0,0,4,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,1.68E2,6.9E1,7.9E1,8.9E1,3.8E1,3.1E1,4.1E1,3.8E1,5.3E1,3.6E1,3E0,3.5E1,4E0,2.7E1,8E0,3.3E1,2.2E1,1.6E1,1.2E1,4.1E1,2.5E1,1.1E1,2E0,3.3E1,1.1E1,1.6E1,7E0,2.6E1,1.2E1,1E1,2E0,1.4E1,8E0,4E0,2.4E1,1.7E1,9E0,1.6E1,3E1,3E0,4E0,7E0,1.5E1,9E0,1.1E1,5E0,2.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.7042104E-3,-7.20367E-2,2.6177105E-1,-1.2977166E-1,4.7325946E-2,1.1269311E-1,4.0182242E-1,-1.1040865E-1,-3.0936033E-1,1.5922122E-2,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,-1.6456133E-1,-5.346679E-2,-1.7062483E-2,-2.109847E-3,-2.8860811E-2,1.2831074E-1,-2.0478494E-1,5.846947E-4,8.376397E-3,-6.544534E-2,-5.855391E-3,-6.571167E-3,1.2199961E-2,1.0790331E-3,-1.0410925E-1,-2.5988343E-1,-7.7399775E-2,7.660132E-3,3.2776052E-3,-5.569639E-2,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,-5.8419913E-2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,2.7633142E-3,1.8774311E-3,-5.525428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,23,-1,-1,25,27,29,-1,-1,31,33,-1,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,-1,-1,45,-1,-1],"loss_changes":[4.8915606E0,1.3223984E0,1.1447959E0,4.1442585E-1,2.5857258E-1,1.2807718E-1,6.305504E-1,3.5506094E-1,2.3678052E-1,2.6542392E-1,0E0,0E0,0E0,0E0,0E0,4.3223488E-1,1.8600516E-1,0E0,0E0,1.0552636E-1,2.2810276E-1,2.4475026E-1,0E0,0E0,1.7779067E-1,1.3734703E-1,0E0,0E0,0E0,1.1772323E-1,1.4893365E-1,1.1551082E-1,0E0,0E0,1.1556485E-1,0E0,0E0,0E0,0E0,1.1671467E-1,0E0,0E0,0E0,0E0,1.0962085E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,15,15,16,16,19,19,20,20,21,21,24,24,25,25,29,29,30,30,31,31,34,34,39,39,44,44],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,24,-1,-1,26,28,30,-1,-1,32,34,-1,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,-1,-1,46,-1,-1],"split_conditions":[1.3394321E0,7.629E3,2.0787016E5,3.3135505E0,5.2328894E5,7.323253E8,1.623319E3,1.66863E5,1.5157113E-2,8.834E3,7.831671E-3,-6.56654E-4,6.40056E-3,1.2324539E-2,2.7744697E-2,2.951829E9,1.55E2,-1.7062483E-2,-2.109847E-3,2.1445148E0,5.2638875E6,6.8618E4,5.846947E-4,8.376397E-3,6.098826E6,3.1791415E11,-6.571167E-3,1.2199961E-2,1.0790331E-3,1.3182504E0,1.7218688E8,2.2733334E1,7.660132E-3,3.2776052E-3,4.1E1,-6.8780296E-3,4.4190368E-4,-1.3394651E-2,-5.6889546E-3,2.0256741E2,-8.692944E-3,-7.550676E-3,-3.5856658E-4,-4.2443806E-3,4.4326835E0,1.8774311E-3,-5.525428E-3],"split_indices":[43,10,29,54,29,32,4,9,39,2,0,0,0,0,0,5,11,0,0,54,44,30,0,0,49,32,0,0,0,57,7,57,0,0,8,0,0,0,0,53,0,0,0,0,55,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.9E2,5.6E1,1.28E2,6.2E1,2.8E1,2.8E1,1.17E2,1.1E1,5.1E1,1.1E1,6E0,2.2E1,2E1,8E0,5.9E1,5.8E1,8E0,3E0,3.7E1,1.4E1,4.8E1,1.1E1,2E0,5.6E1,3.2E1,5E0,5E0,9E0,1.8E1,3E1,5.4E1,2E0,1.2E1,2E1,1.2E1,6E0,2.1E1,9E0,4.8E1,6E0,5E0,1.5E1,2.9E1,1.9E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.6167102E-2,-9.1617934E-2,2.0921354E-1,-1.5892246E-1,1.5889113E-4,1.0256023E-1,3.3971918E-1,-2.3020288E-1,-4.6520796E-2,-1.315317E-1,3.3601597E-2,1.9421874E-2,6.8316124E-2,2.9019982E-2,2.54235E-1,-7.909159E-2,-2.7484733E-1,5.3665806E-3,-8.432639E-3,-3.2230811E-3,-1.557175E-2,-2.764732E-3,5.4405066E-3,-5.779646E-2,1.18016526E-1,5.7629147E-3,3.5522467E-1,7.019195E-3,-5.448651E-3,-3.0026987E-1,3.8947738E-4,-5.0486766E-2,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,1.5366653E-1,-2.1028195E-3,1.7452618E-2,3.831069E-3,-2.4670707E-1,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,-1,-1,33,-1,35,37,-1,39,-1,-1,41,-1,43,-1,-1,-1,-1,-1,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.134416E0,1.1242383E0,8.142085E-1,8.305547E-1,3.4793866E-1,3.800948E-1,5.817108E-1,4.100237E-1,3.2077062E-1,1.8154746E-1,2.089919E-1,0E0,2.1365988E-1,0E0,2.3687482E-1,1.9577982E-1,3.561766E-1,2.3537263E-1,0E0,0E0,0E0,1.8856645E-1,0E0,8.858822E-2,1.4960828E-1,0E0,1.08366966E-1,0E0,0E0,1.7012787E-1,0E0,1.1466624E-1,0E0,0E0,0E0,0E0,0E0,9.963548E-2,0E0,0E0,0E0,1.5310645E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,21,21,23,23,24,24,26,26,29,29,31,31,37,37,41,41],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,-1,-1,34,-1,36,38,-1,40,-1,-1,42,-1,44,-1,-1,-1,-1,-1,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2659024E2,2.1619815E-1,1.9530877E-1,1.80025E2,1.8681856E8,8.9397516E4,1.8681E4,1.433458E10,2.142943E1,2.151E3,2.704375E6,1.9421874E-2,1.479938E6,2.9019982E-2,6.7E1,2.7287993E0,3.402546E6,6.135367E9,-8.432639E-3,-3.2230811E-3,-1.557175E-2,9.143112E5,5.4405066E-3,9.71709E5,2.30273E6,5.7629147E-3,6.3898524E7,7.019195E-3,-5.448651E-3,4.1938826E-2,3.8947738E-4,9.165388E4,5.8167437E-3,-2.9417104E-3,2.5878935E-3,-8.79671E-3,1.4604852E-4,5.7E1,-2.1028195E-3,1.7452618E-2,3.831069E-3,1.0142518E0,-1.8028947E-2,3.1684902E-3,-3.871347E-3,1.0628724E-2,3.7759228E-3,-1.2045199E-2,-3.0022117E-3],"split_indices":[53,42,39,53,7,34,9,32,57,2,9,0,33,0,8,55,1,5,0,0,0,34,0,30,9,0,33,0,0,43,0,34,0,0,0,0,0,8,0,0,0,59,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.81E2,6E1,1.04E2,7.7E1,3.4E1,2.6E1,6.3E1,4.1E1,1.5E1,6.2E1,2E0,3.2E1,4E0,2.2E1,1.5E1,4.8E1,3.1E1,1E1,1.3E1,2E0,4.5E1,1.7E1,9E0,2.3E1,1.1E1,1.1E1,2E0,1.3E1,4.4E1,4E0,2.2E1,9E0,2.2E1,2.3E1,2E0,7E0,1.9E1,4E0,9E0,2E0,3.2E1,1.2E1,5E0,1.7E1,7E0,1.2E1,2.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[6.0046404E-3,-9.218389E-2,2.1822302E-1,-1.7119181E-1,2.2936372E-2,1.080785E-1,3.675177E-1,-2.2839919E-1,-7.109087E-2,4.932593E-2,-3.4302478E-3,1.885684E-2,8.236004E-2,1.287323E-1,4.4497648E-1,-2.0051514E-1,-2.1602163E-2,4.6547647E-3,-1.2568788E-1,-1.1251967E-2,4.7163595E-3,1.5830024E-1,1.2744217E-2,9.789093E-3,3.2283113E-4,3.5435382E-1,3.0562386E-2,-1.4268523E-2,-1.7262784E-1,-6.920345E-3,9.3187834E-4,-4.0626727E-2,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,5.3271845E-2,1.711268E-2,5.001436E-3,-1.4809307E-1,-1.3950878E-2,-6.9640996E-3,7.891728E-3,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,-1,-1,29,31,-1,33,35,-1,-1,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,43,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8752337E0,1.4591926E0,1.1771345E0,5.277815E-1,1.8073346E-1,3.3109027E-1,5.1609373E-1,3.4681034E-1,3.6278534E-1,1.8984374E-1,0E0,0E0,2.2002146E-1,1.0272367E-1,2.8410387E-1,1.4103603E-1,0E0,0E0,1.3881755E-1,1.4883459E-1,0E0,1.2233335E-1,2.0214176E-1,0E0,0E0,1.2719083E-1,0E0,0E0,1.2584579E-1,0E0,0E0,1.397791E-1,0E0,0E0,0E0,0E0,9.9094376E-2,0E0,0E0,9.3803644E-2,0E0,0E0,9.208519E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,19,19,21,21,22,22,25,25,28,28,31,31,36,36,39,39,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,-1,-1,30,32,-1,34,36,-1,-1,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,44,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1,-1],"split_conditions":[3.6984555E2,2.1619815E-1,2.0224316E-1,1.9415865E2,2.260197E0,8.9397516E4,2.2324745E5,5.6835636E-2,8.938651E4,1.7573398E2,-3.4302478E-3,1.885684E-2,1.3844932E7,2.81002E5,1.2461E4,1.7565872E-3,-2.1602163E-2,4.6547647E-3,1.7725672E8,2.424E3,4.7163595E-3,1.8990228E0,1.312252E6,9.789093E-3,3.2283113E-4,4.7201157E-1,3.0562386E-2,-1.4268523E-2,1.269689E1,-6.920345E-3,9.3187834E-4,8E0,7.3230257E-3,9.153763E-3,1.429756E-3,-8.425798E-3,2.0215728E0,1.711268E-2,5.001436E-3,8.938651E4,-1.3950878E-2,-6.9640996E-3,2.176974E8,3.6878204E-3,-4.049611E-3,-8.999809E-3,-4.4575413E-3,-3.237285E-3,2.9179305E-3],"split_indices":[53,42,39,53,54,34,29,58,34,53,0,0,46,9,2,59,0,0,12,2,0,54,30,0,0,39,0,0,57,0,0,3,0,0,0,0,36,0,0,34,0,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.59E2,7.3E1,9.4E1,6.5E1,4.3E1,3E1,5.9E1,3.5E1,5.2E1,1.3E1,2E0,4.1E1,8E0,2.2E1,5.5E1,4E0,8E0,2.7E1,2.6E1,2.6E1,1.9E1,2.2E1,4E0,4E0,1.8E1,4E0,8E0,4.7E1,2.2E1,5E0,2.3E1,3E0,1.3E1,6E0,3E0,1.9E1,1.5E1,3E0,4.2E1,5E0,6E0,1.7E1,1.6E1,3E0,1.7E1,2.5E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.3128728E-2,-7.6841384E-2,2.5555712E-1,-1.3273934E-1,6.3761264E-2,8.655129E-2,4.008202E-1,-1.6426608E-1,-1.455439E-2,1.4814147E-2,3.6547862E-2,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,-1.4727283E-1,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,5.5398457E-2,-4.1562365E-3,-1.9884297E-1,-8.877025E-2,-5.379106E-4,9.418675E-2,-2.5566745E-1,-8.6616755E-2,-7.036469E-4,-1.4556472E-1,5.3083412E-3,-5.8485573E-4,-2.0745803E-1,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,-7.540202E-2,-9.3988655E-3,-2.2761422E-1,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,-1,25,27,-1,29,31,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,-1,43,-1,-1,-1,-1,-1],"loss_changes":[4.1965976E0,1.5646251E0,1.1191728E0,5.266993E-1,4.1526425E-1,1.3371919E-1,5.893326E-1,3.3139777E-1,9.107072E-2,0E0,1.3694459E-1,0E0,0E0,0E0,0E0,2.9913235E-1,0E0,0E0,0E0,1.246268E-1,0E0,3.2860088E-1,2.5498408E-1,0E0,9.024802E-2,4.5705009E-1,9.878099E-2,1.712665E-1,1.3560718E-1,0E0,0E0,1.3404357E-1,0E0,0E0,0E0,0E0,0E0,1.554379E-1,0E0,9.5205545E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,31,31,37,37,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,-1,26,28,-1,30,32,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,-1,44,-1,-1,-1,-1,-1],"split_conditions":[1.4272063E0,3.165913E6,2.174459E5,7.087292E-2,2.207E3,7.002603E1,1.6507233E3,1E0,1.1721691E5,1.4814147E-2,3.605082E7,2.124302E-3,1.1096259E-2,1.1448107E-2,2.6962893E-2,1.73507E5,-1.6005408E-2,4.2363442E-3,-1.7348974E-3,1.462136E6,-4.1562365E-3,1.3738942E0,1.833E5,-5.379106E-4,1.9879582E7,1.0665042E8,1.2288136E1,1.1942446E0,1.2271346E1,5.3083412E-3,-5.8485573E-4,8.672049E9,-2.572419E-2,-6.447848E-4,-6.850457E-3,8.138498E-3,-1.8443794E-3,9.3E4,-9.3988655E-3,6.7001674E-3,-1.9297525E-4,-1.2330733E-2,-1.2630431E-3,-1.4284167E-2,-7.7012517E-3],"split_indices":[43,1,29,39,0,57,4,82,29,0,46,0,0,0,0,9,0,0,0,33,0,54,30,0,33,7,57,54,57,0,0,5,0,0,0,0,0,12,0,59,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.97E2,4.6E1,1.41E2,5.6E1,2.2E1,2.4E1,1.11E2,3E1,4E0,5.2E1,1.9E1,3E0,1.6E1,8E0,1.04E2,7E0,5E0,2.5E1,4.6E1,6E0,5.4E1,5E1,1.7E1,2.9E1,3.5E1,1.9E1,2E1,3E1,2.3E1,6E0,3.2E1,3E0,1E1,9E0,3E0,1.7E1,1.6E1,1.4E1,2.9E1,3E0,2E0,1.4E1,8E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.4699904E-3,-8.2270786E-2,1.7413537E-1,-1.3975397E-1,3.89982E-3,7.456945E-2,3.0463025E-1,-1.7296548E-1,6.0730726E-3,-9.173903E-2,3.4571238E-2,1.801554E-2,4.9235556E-2,1.7605738E-3,3.2971406E-1,-2.1693262E-1,-1.0940522E-1,7.686381E-2,-3.4685563E-3,-2.7480236E-1,-8.64333E-4,1.01417145E-5,4.4262665E-3,-7.801436E-2,7.197917E-2,8.4248185E-3,3.7063202E-1,-1.9041932E-1,-2.0984167E-2,-4.680374E-2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,2.399795E-2,3.9364552E-1,4.087419E-3,-1.0795465E-3,-2.0910926E-1,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,5.9234392E-2,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,33,-1,-1,-1,35,37,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,45,47,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5455477E0,8.9215004E-1,9.777329E-1,5.2410936E-1,2.1730821E-1,3.6530188E-1,2.0546484E-1,2.22188E-1,1.3329768E-1,2.3158492E-1,1.2854296E-1,0E0,1.3006273E-1,0E0,9.619784E-2,2.65311E-1,1.5102664E-1,2.2796245E-1,0E0,1.5092689E-1,0E0,0E0,0E0,1.5338422E-1,1.3331974E-1,0E0,1.03357315E-1,1.4346242E-1,0E0,1.5426993E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3046479E-1,9.462261E-2,0E0,0E0,1.1954689E-1,0E0,0E0,0E0,9.6233115E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,19,19,23,23,24,24,26,26,27,27,29,29,38,38,39,39,42,42,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,34,-1,-1,-1,36,38,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,46,48,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.8509178E-1,1.8829846E-1,2.219883E6,1.8681856E8,8.9397516E4,1E0,5.283708E0,1.8000048E10,8E0,2.069532E6,1.801554E-2,7.5E2,1.7605738E-3,2.2324745E5,5.144033E-2,1.2288136E1,3.131673E6,-3.4685563E-3,3.56E2,-8.64333E-4,1.01417145E-5,4.4262665E-3,3.13E2,8.312613E5,8.4248185E-3,1.8373182E8,6E0,-2.0984167E-2,6.0515753E2,-7.6962537E-3,-1.5265922E-3,1.0492754E-2,-1.823212E-2,-1.7546682E-3,4.956243E-3,-7.7399914E-3,6.5061823E-3,6.925995E6,1.7598395E3,4.087419E-3,-1.0795465E-3,7.5646E4,-5.9797377E-3,1.4088032E-3,-5.3129233E-3,2.8812434E7,1.3396224E-2,2.3770304E-2,-1.12283025E-2,-5.8779106E-3,4.3623364E-3,-2.2894272E-3],"split_indices":[53,40,39,1,7,34,17,57,5,3,9,0,0,0,29,58,57,30,0,0,0,0,0,0,48,0,33,10,0,4,0,0,0,0,0,0,0,0,52,4,0,0,9,0,0,0,46,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.79E2,7.6E1,1.07E2,7.2E1,4.4E1,3.2E1,8.7E1,2E1,1.7E1,5.5E1,2E0,4.2E1,3E0,2.9E1,5E1,3.7E1,1.1E1,9E0,4E0,1.3E1,3.7E1,1.8E1,6E0,3.6E1,8E0,2.1E1,4.7E1,3E0,2E1,1.7E1,7E0,4E0,2E0,2E0,2E0,4E0,1.3E1,2.3E1,1.9E1,2E0,5E0,4.2E1,9E0,1.1E1,4E0,1.9E1,1.4E1,5E0,2.4E1,1.8E1,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[3.0756515E-4,-9.093915E-2,1.3531232E-1,-1.3121241E-1,3.0161556E-2,6.471361E-2,2.7240935E-1,-1.5608981E-1,-4.138077E-2,5.6453147E-3,-1.665546E-3,1.331529E-2,4.4281885E-2,1.4165652E-4,2.9921746E-1,-1.3490759E-1,-3.431053E-1,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,1.077188E-1,-1.1128485E-2,1.87167E-2,1.0148881E-2,-1.18122905E-1,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,5.707247E-2,-4.600261E-3,3.8549118E-2,-1.6461147E-1,-4.9828332E-2,6.6000526E-3,6.4204754E-3,4.2778063E-3,-2.1197742E-3,-2.1401511E-1,-2.0505309E-3,7.0574425E-2,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,23,25,27,-1,-1,-1,-1,29,31,-1,-1,33,-1,-1,-1,-1,35,-1,37,39,41,-1,43,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9934402E0,7.1114016E-1,9.309007E-1,2.3713171E-1,1.1703685E-1,3.1068093E-1,2.37149E-1,2.9043174E-1,1.1233303E-1,0E0,1.0321621E-1,0E0,2.1913269E-1,0E0,1.5956187E-1,2.0051861E-1,2.3705983E-1,0E0,0E0,0E0,0E0,2.0704418E-1,1.6331983E-1,0E0,0E0,2.2356677E-1,0E0,0E0,0E0,0E0,1.08736545E-1,0E0,1.2394792E-1,2.3981667E-1,1.24148294E-1,0E0,9.681947E-2,0E0,0E0,2.0636463E-1,0E0,8.820005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,14,14,15,15,16,16,21,21,22,22,25,25,30,30,32,32,33,33,34,34,36,36,39,39,41,41],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,24,26,28,-1,-1,-1,-1,30,32,-1,-1,34,-1,-1,-1,-1,36,-1,38,40,42,-1,44,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.5343175E0,2.2348747E2,1.0632315E6,8.9397516E4,1E0,2.8659306E0,2.2616908E2,5.6453147E-3,7.432E3,1.331529E-2,1.1572553E7,1.4165652E-4,3.2307692E0,3.0962145E0,1.020435E4,6.2638987E-3,-2.779869E-3,-1.2150024E-3,5.653453E-3,4.0012E4,1.4813267E3,1.87167E-2,1.0148881E-2,1.73507E5,-1.3817679E-2,-2.2815822E-2,-5.703726E-3,1.129112E-2,7E0,-4.600261E-3,8.303E3,2.951829E9,4.4E1,6.6000526E-3,1.9816086E0,4.2778063E-3,-2.1197742E-3,1.42E2,-2.0505309E-3,7.510128E7,-3.558664E-3,2.3899402E-3,-4.8392727E-3,-1.7717372E-3,-1.1157474E-2,-3.3650168E-3,6.3307164E-3],"split_indices":[53,39,42,53,49,34,17,59,53,0,10,0,46,0,55,54,33,0,0,0,0,9,56,0,0,9,0,0,0,0,3,0,0,5,10,0,54,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.44E2,9.7E1,1.08E2,3.6E1,6.5E1,3.2E1,8.4E1,2.4E1,8E0,2.8E1,4E0,6.1E1,3E0,2.9E1,7.7E1,7E0,2E0,2.2E1,2.4E1,4E0,2.8E1,3.3E1,8E0,2.1E1,7.2E1,5E0,3E0,4E0,6E0,2.2E1,1.1E1,2.2E1,4.2E1,3E1,7E0,1.5E1,1.3E1,9E0,2.9E1,1.3E1,6E0,2.4E1,1.1E1,4E0,6E0,2.3E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.0671469E-2,-4.057193E-2,2.5710663E-1,-9.11908E-2,7.657581E-2,1.0005701E-1,3.5594457E-1,-1.3330418E-1,3.5337575E-2,6.0731232E-2,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,-1.425013E-1,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,-2.7416292E-1,-1.2234801E-1,-3.3937553E-3,-2.139736E-2,-1.513492E-1,-3.387177E-2,-1.1739622E-1,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,-8.5462265E-2,-4.7841365E-3,7.8635005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,21,-1,-1,-1,-1,23,25,-1,-1,27,29,31,-1,-1,-1,-1,33,-1,-1],"loss_changes":[3.1548471E0,1.2344693E0,6.318512E-1,7.7734864E-1,2.7344996E-1,1.175154E-1,3.9797258E-1,1.8897259E-1,1.1001776E-1,1.5682302E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.4241185E-1,0E0,0E0,0E0,0E0,5.137621E-1,2.3093772E-1,0E0,0E0,1.8179822E-1,9.140951E-2,1.8776286E-1,0E0,0E0,0E0,0E0,1.0951534E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,16,16,21,21,22,22,25,25,26,26,27,27,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,22,-1,-1,-1,-1,24,26,-1,-1,28,30,32,-1,-1,-1,-1,34,-1,-1],"split_conditions":[1.5648934E0,3.165913E6,2.174459E5,6.408734E-2,1.4122614E2,1E0,1.7598395E3,1E1,1.599097E0,1.25440235E1,1.628707E-2,-4.702962E-5,7.179186E-3,1.1374972E-2,2.5194194E-2,2.109063E-3,1.0187571E5,3.4082092E-3,-1.4220915E-3,3.6385055E-3,-2.1726242E-3,8.3E1,1.1670399E-1,-3.3937553E-3,-2.139736E-2,3.671916E0,1E0,1E0,-1.0518167E-2,5.7697096E-3,-2.3992807E-3,-1.0603058E-2,1.8E1,-4.7841365E-3,7.8635005E-4],"split_indices":[42,1,29,39,57,8,4,10,54,58,0,0,0,0,0,0,29,0,0,0,0,10,40,0,0,55,107,109,0,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,2.06E2,4.2E1,1.44E2,6.2E1,1.7E1,2.5E1,1.08E2,3.6E1,6E1,2E0,7E0,1E1,1.9E1,6E0,5E0,1.03E2,2.2E1,1.4E1,5E1,1E1,1.2E1,9.1E1,7E0,5E0,6.8E1,2.3E1,5.1E1,1.7E1,2E0,2.1E1,9E0,4.2E1,3.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.4315777E-2,-1.0690338E-1,1.1009746E-1,-1.4777888E-1,2.0821875E-2,2.117857E-2,1.7040308E-1,-1.806778E-1,-5.4927632E-2,2.9963165E-4,6.637665E-3,3.901988E-2,-4.5500887E-3,7.291719E-2,2.6567936E-1,-1.4094864E-1,-2.987772E-1,-2.4235064E-4,-2.5972742E-1,8.2612615E-3,-1.8001318E-2,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,9.182606E-2,1.6551235E-1,2.0925388E-2,-2.1879025E-1,-9.1945045E-2,-3.4581E-1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,-1,31,-1,33,-1,-1,-1,35,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9952228E0,7.809812E-1,5.9160745E-1,3.3525777E-1,9.964019E-2,1.07021526E-1,5.936415E-1,3.473227E-1,3.1202346E-1,1.1755685E-1,0E0,9.129217E-2,0E0,1.14367574E-1,6.274619E-1,2.2401261E-1,1.6717279E-1,0E0,9.88501E-2,0E0,1.3668847E-1,0E0,0E0,0E0,1.5995112E-1,2.5624245E-1,0E0,1.0733318E-1,1.4268285E-1,2.0528078E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,18,18,20,20,24,24,25,25,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,-1,32,-1,34,-1,-1,-1,36,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.1677386E-1,2.796589E6,3.4021906E2,1.1670399E-1,5.5446387E2,1.9E1,2.174459E5,5.740651E-3,3.4047124E-1,2.207E3,6.637665E-3,2.0358758E8,-4.5500887E-3,1.9241573E-1,7.2259143E2,4.949154E-4,5.906883E5,-2.4235064E-4,2.07298E5,8.2612615E-3,1E1,-1.7809185E-3,2.8840087E-3,-4.0084883E-3,8.303E3,1.9816086E0,2.0925388E-2,1.7565872E-3,4.8831225E5,7.5E1,-3.8627759E-3,-1.5368976E-2,-2.1764718E-3,3.1842561E-3,-2.8275766E-3,6.1712214E-3,-4.8843765E-4,9.540881E-3,-1.3216786E-3,-1.3721985E-2,-6.6030337E-3,-5.3792605E-3,7.628713E-4,-1.8609913E-2,-6.1238795E-3],"split_indices":[43,1,53,40,53,3,29,39,43,0,0,7,0,58,53,40,34,0,1,0,3,0,0,0,0,54,0,59,29,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,1.48E2,1.1E2,1.12E2,3.6E1,4.5E1,6.5E1,8.2E1,3E1,3.2E1,4E0,4E1,5E0,3.3E1,3.2E1,6.3E1,1.9E1,2.5E1,5E0,2E0,3E1,1E1,3E1,3E0,3E1,2.3E1,9E0,2.3E1,4E1,1.5E1,4E0,3E0,2E0,1E1,2E1,2E1,1E1,1.8E1,5E0,8E0,1.5E1,3.1E1,9E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.5204665E-2,-6.9504075E-2,1.5126899E-1,-1.1839543E-1,2.2287078E-2,7.562342E-2,2.5493225E-1,-1.4351347E-1,-1.2403568E-2,-5.980414E-3,3.6651608E-2,5.731069E-2,9.66742E-3,6.0085384E-3,3.3954874E-1,-1.3105348E-2,-1.2924905E-1,3.58532E-2,-7.821619E-3,1.6858906E-2,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,-9.744661E-2,-1.9808635E-1,4.93121E-3,-2.631804E-3,3.7851118E-2,-3.5518215E-3,-1.1887726E-1,1.7840436E-3,-5.2316664E-3,-2.911272E-1,6.4006194E-2,-1.814889E-3,-1.4108726E-1,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.0247637E-1,-1.5689014E-1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,37,-1,-1,39,41,-1,43,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1],"loss_changes":[2.3878818E0,8.9570975E-1,4.904294E-1,3.4431982E-1,1.6523217E-1,9.151134E-2,2.2111058E-1,1.9603395E-1,2.1574554E-1,0E0,1.2080229E-1,1.1728562E-1,0E0,0E0,9.910309E-2,0E0,1.9523811E-1,1.6260572E-1,0E0,1.1651751E-1,0E0,0E0,0E0,0E0,0E0,2.094084E-1,1.8337464E-1,0E0,0E0,9.8238766E-2,0E0,1.4792949E-1,0E0,0E0,2.5384462E-1,1.126506E-1,0E0,1.1237788E-1,0E0,0E0,0E0,0E0,8.794698E-2,1.0356009E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,16,16,17,17,19,19,25,25,26,26,29,29,31,31,34,34,35,35,37,37,42,42,43,43],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,38,-1,-1,40,42,-1,44,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1],"split_conditions":[1.2185057E0,1.781973E6,2.0787016E5,6.808926E6,9.315286E1,4.328683E0,3.778E3,3.4903356E7,1.409674E7,-5.980414E-3,4.43001E5,1.1832633E0,9.66742E-3,6.0085384E-3,4.7201157E-1,-1.3105348E-2,4.352025E0,4.8593006E5,-7.821619E-3,1.939974E7,5.772692E-3,6.825668E-3,1.0215753E-3,1.6276985E-2,3.5540734E-3,3.9376082E0,8.287425E6,4.93121E-3,-2.631804E-3,1.25440235E1,-3.5518215E-3,2.2E1,1.7840436E-3,-5.2316664E-3,4.752366E0,5.6024827E9,-1.814889E-3,2.427E3,-3.1052163E-4,-2.0820443E-2,-6.7839916E-3,-7.9719536E-4,1.1770233E0,4.2E1,4.1407915E-5,-1.9681456E-3,5.485692E-3,-7.671554E-3,-1.2717871E-3],"split_indices":[42,1,29,9,53,36,2,7,46,0,29,40,0,0,39,0,55,34,0,33,0,0,0,0,0,55,46,0,0,58,0,3,0,0,57,5,0,2,0,0,0,0,55,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.62E2,1.98E2,6.4E1,1.29E2,6.9E1,3.8E1,2.6E1,1.04E2,2.5E1,5E0,6.4E1,3.5E1,3E0,1.2E1,1.4E1,7E0,9.7E1,2E1,5E0,5.4E1,1E1,8E0,2.7E1,1.2E1,2E0,6.8E1,2.9E1,1.1E1,9E0,4.5E1,9E0,5.9E1,9E0,1.7E1,1.2E1,3.4E1,1.1E1,4.9E1,1E1,4E0,8E0,1.1E1,2.3E1,4.4E1,5E0,3E0,2E1,3.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.1777132E-3,-6.0567748E-2,1.9037953E-1,-1.3071354E-1,5.439912E-4,8.848939E-2,2.9270196E-1,-1.7778724E-1,-1.4740403E-3,-7.950044E-2,3.19054E-2,3.1027812E-4,1.2356255E-1,2.2399522E-2,2.3053867E-1,-1.3397929E-1,-2.7813753E-1,1.4052213E-4,-1.3808084E-1,7.668798E-2,-2.3035506E-2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.975542E0,8.102633E-1,6.309147E-1,3.9554834E-1,2.5858772E-1,9.331095E-2,3.067577E-1,2.2210789E-1,0E0,1.4097291E-1,1.838358E-1,0E0,1.22035086E-1,0E0,9.3972206E-2,2.5386703E-1,9.3033075E-2,0E0,1.3659179E-1,1.5234712E-1,1.16067775E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,1.9415865E2,2.174459E5,1.848519E-2,8.461558E10,5.958114E4,1.8681E4,1.607E3,-1.4740403E-3,1.2271346E1,2.305287E1,3.1027812E-4,1.718354E8,2.2399522E-2,3.778E3,6.581686E6,6.608892E2,1.4052213E-4,4.6955062E5,1.2119408E3,6.657047E2,3.560035E-5,7.2805816E-3,6.1105206E-3,1.2549705E-2,-6.640934E-3,6.9566006E-3,-1.4063048E-2,-4.7039767E-3,-1.2212133E-2,-3.182782E-3,1.6720688E-3,7.7614193E-3,-3.032945E-3,2.10811E-3],"split_indices":[43,53,29,39,32,29,9,2,0,57,57,0,7,0,2,9,4,0,34,4,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.88E2,6.2E1,8.7E1,1.01E2,3.2E1,3E1,5.8E1,2.9E1,2.8E1,7.3E1,1E1,2.2E1,5E0,2.5E1,4.2E1,1.6E1,1.2E1,1.6E1,4E1,3.3E1,6E0,1.6E1,1.1E1,1.4E1,4E1,2E0,1.2E1,4E0,4E0,1.2E1,3E1,1E1,2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.9849198E-3,-6.908789E-2,1.1069185E-1,-1.1135524E-1,4.8647843E-2,1.8852478E-2,1.622326E-1,-9.491666E-2,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,5.3762157E-2,2.2892316E-1,6.1087083E-2,-1.4048992E-1,-5.2776527E-2,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,-1.5545653E-1,-1.0782661E-3,-8.035333E-3,-2.0769975E-1,-9.872578E-2,-2.4276859E-1,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1],"loss_changes":[1.8784683E0,7.269919E-1,4.5608807E-1,2.1379781E-1,3.8304985E-1,1.0936768E-1,4.042337E-1,1.7887038E-1,0E0,0E0,0E0,0E0,9.703164E-2,2.1851015E-1,1.605283E-1,2.029301E-1,1.871668E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.09901786E-1,0E0,0E0,1.7119968E-1,1.0259637E-1,1.2492943E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,14,14,15,15,16,16,24,24,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1],"split_conditions":[4.259334E-1,3.343327E6,3.1259552E2,3.7979183E0,2.207E3,2.1123085E8,1.0568718E7,1.3466058E0,-1.074348E-2,1.4919681E-2,6.421078E-4,-2.8529011E-3,1.6070912E0,1.2461E4,1.25440235E1,4.6376264E9,2.2733334E1,8.6216675E-3,1.152717E-3,8.296459E-3,1.9169042E-2,4.385657E-3,-3.7981148E-3,5.491974E-3,6.7329E4,-1.0782661E-3,-8.035333E-3,8.640031E9,1.1710652E0,4.19891E0,4.713934E-4,-2.812652E-4,-6.437278E-3,-7.989193E-3,-1.8365461E-2],"split_indices":[43,1,53,54,0,7,46,54,0,0,0,0,59,2,58,32,57,0,0,0,0,0,0,0,9,0,0,5,54,57,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.44E2,9.6E1,1.06E2,3.8E1,3.5E1,6.1E1,9.6E1,1E1,3E0,3.5E1,1E1,2.5E1,3.6E1,2.5E1,4.5E1,5.1E1,3E0,2.2E1,3.2E1,4E0,2E1,5E0,2E0,4.3E1,4.3E1,8E0,2.1E1,2.2E1,1.8E1,3E0,8E0,1.4E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0547296E-2,-4.2344254E-2,1.742515E-1,-1.0237627E-1,3.5556585E-2,2.740988E-1,7.519926E-2,-7.69858E-2,-2.2364104E-1,7.678524E-3,2.2331232E-2,1.4626542E-1,3.7681144E-1,4.2196088E-2,8.162208E-3,-4.9231295E-2,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,4.149654E-2,-7.011419E-4,9.022605E-3,4.7000867E-1,6.24682E-3,-3.9323145E-3,3.069897E-3,-7.009031E-2,6.090464E-2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,-1,-1,-1,-1,29,-1,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.163362E0,8.8750696E-1,5.826844E-1,3.122326E-1,1.5018767E-1,3.3878827E-1,1.1022253E-1,2.4382734E-1,2.6022464E-1,0E0,1.0540541E-1,1.5061551E-1,2.8114796E-1,1.0317968E-1,0E0,1.6967641E-1,0E0,0E0,0E0,0E0,1.09696545E-1,0E0,0E0,9.1567755E-2,0E0,0E0,0E0,1.7041293E-1,1.1754744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,20,20,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,-1,-1,-1,-1,30,-1,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.5971466E2,2.7558826E-2,9.8110194E5,1.2460274E-2,2.3755E4,7.116346E2,2.0787016E5,3.774648E0,5.906883E5,7.678524E-3,1.496E3,2.4894228E6,2.660842E6,1.5E1,8.162208E-3,3.901529E6,-7.772476E-3,-1.5098738E-2,-3.9781453E-3,-2.0838454E-3,1.6815423E5,-7.011419E-4,9.022605E-3,5.505334E9,6.24682E-3,-3.9323145E-3,3.069897E-3,2.797224E2,2.1374529E2,-6.269786E-4,3.202133E-3,1.30279E-2,2.68844E-2,-3.8370257E-3,3.4939845E-3,7.000282E-3,-1.4439927E-3],"split_indices":[53,39,34,39,9,53,29,55,34,0,2,33,30,3,0,9,0,0,0,0,29,0,0,5,0,0,0,53,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,1.88E2,6E1,1.06E2,8.2E1,2.9E1,3.1E1,8.9E1,1.7E1,6E0,7.6E1,1.4E1,1.5E1,2.5E1,6E0,7.1E1,1.8E1,8E0,9E0,1.6E1,6E1,4E0,1E1,1E1,5E0,4E0,2.1E1,6E1,1.1E1,2.2E1,3.8E1,6E0,4E0,5.4E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.9339443E-3,-1.0605051E-1,7.4423306E-2,-1.4161547E-1,-2.8023109E-2,5.737777E-2,3.6554796E-1,-1.08674295E-1,-2.4172173E-1,-5.5312406E-2,5.1337723E-3,2.0626027E-2,1.1801036E-1,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,-2.894412E-1,4.924759E-4,-5.1691607E-3,-2.6259364E-3,4.9695414E-2,4.6624467E-4,1.5626377E-1,-1.5225716E-2,-4.396114E-3,1.359324E-1,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,25,-1,-1,-1,27,-1,29,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.0659106E0,3.0406094E-1,7.134572E-1,2.243501E-1,1.496107E-1,3.117638E-1,1.7670429E-1,1.5272987E-1,2.0493579E-1,1.3169232E-1,0E0,2.1203475E-1,2.1592683E-1,0E0,0E0,0E0,0E0,0E0,1.3315952E-1,0E0,0E0,0E0,1.3145865E-1,0E0,1.2719214E-1,0E0,0E0,8.859834E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,18,18,22,22,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,26,-1,-1,-1,28,-1,30,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[1.7613442E-1,2.8704E4,4.328683E0,5.1653096E-3,1.22E2,1.2446029E0,2.174459E5,1.0665042E8,3.7274265E2,2.8488008E11,5.1337723E-3,1.805E3,1.7468556E2,7.1073123E-3,2.3020085E-2,-5.4246816E-3,2.1882658E-3,-1.0226086E-4,2.1710082E7,4.924759E-4,-5.1691607E-3,-2.6259364E-3,1.325E0,4.6624467E-4,1.1560919E3,-1.5225716E-2,-4.396114E-3,8.8356856E5,1.1336594E-3,7.5424393E-3,-1.3949588E-3,8.128547E-3,3.5234066E-4],"split_indices":[42,11,36,39,8,43,29,7,4,32,0,2,53,0,0,0,0,0,33,0,0,0,54,0,53,0,0,48,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,1.1E2,1.47E2,7.5E1,3.5E1,1.4E2,7E0,5.8E1,1.7E1,3E1,5E0,8.8E1,5.2E1,4E0,3E0,5.3E1,5E0,3E0,1.4E1,1.5E1,1.5E1,2.3E1,6.5E1,1.4E1,3.8E1,1E1,4E0,1.3E1,5.2E1,3.5E1,3E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-9.345121E-3,-6.562803E-2,1.1551625E-1,-8.816371E-2,4.981665E-4,3.869827E-2,2.208219E-1,-1.4935678E-1,-4.5134716E-2,1.6114242E-2,6.035789E-3,2.2871085E-2,1.6961017E-1,-2.867197E-1,-1.2161767E-1,-6.569096E-2,3.5917396E-3,-3.0824807E-3,5.086014E-2,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,-1.0346314E-1,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,29,-1,-1,-1,-1],"loss_changes":[1.6731131E0,2.8574848E-1,5.9006786E-1,3.276595E-1,0E0,9.675721E-2,3.9257836E-1,1.6326678E-1,2.0450398E-1,1.1511861E-1,0E0,0E0,1.3534534E-1,3.4041297E-1,1.645894E-1,9.34082E-2,0E0,0E0,1.10176116E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0852566E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,18,18,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,30,-1,-1,-1,-1],"split_conditions":[3.7136658E2,9.1318935E-2,1.8829846E-1,4.426498E0,4.981665E-4,1E0,1.431E4,9.975593E4,6.767991E5,1.312252E6,6.035789E-3,2.2871085E-2,1.1207349E0,2.83E2,5.22241E5,1.7012288E8,3.5917396E-3,-3.0824807E-3,8.303E3,-1.0358182E-3,8.455525E-3,-2.755693E-3,-2.1579653E-2,-7.4758423E-3,-1.9462655E-3,-1.1825111E-3,2.22105E5,4.20352E-3,-1.5737375E-3,3.913872E-3,-5.1850714E-3],"split_indices":[53,39,39,57,0,112,9,29,29,30,0,0,54,0,12,7,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.63E2,7.3E1,1.26E2,3.7E1,4.3E1,3E1,5.1E1,7.5E1,3.6E1,7E0,3E0,2.7E1,7E0,4.4E1,6.5E1,1E1,1E1,2.6E1,3E0,2.4E1,4E0,3E0,2.6E1,1.8E1,3.3E1,3.2E1,1.7E1,9E0,2E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.5972516E-3,-6.415109E-2,1.004628E-1,-9.126678E-2,7.985584E-5,7.822119E-2,3.8027394E-1,-1.3547492E-1,-1.7177572E-2,1.2492152E-1,-1.8076582E-2,2.4130752E-2,6.7940415E-3,-2.0095895E-1,-7.504207E-2,3.5000306E-3,-4.5314096E-2,2.2494642E-1,8.771981E-2,-6.7835734E-3,8.8194094E-4,-1.3448668E-1,-3.6327535E-1,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,-8.5903674E-2,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,-1,-1,-1,-1],"loss_changes":[1.4975804E0,2.8284806E-1,4.9924803E-1,3.645115E-1,0E0,3.6202925E-1,1.4661348E-1,2.6193678E-1,1.21453926E-1,1.8120694E-1,1.4514734E-1,0E0,0E0,3.07199E-1,2.7256477E-1,0E0,1.0324016E-1,1.1432767E-1,8.934039E-2,0E0,0E0,1.8839267E-1,1.3768709E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.900878E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,17,17,18,18,21,21,22,22,32,32],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,-1,-1,-1,-1],"split_conditions":[3.4021906E2,5.530166E-2,2.069929E3,1.6408995E11,7.985584E-5,1.8018788E7,5.1145615E3,1.386452E0,2.6017979E11,8E0,1.628E3,2.4130752E-2,6.7940415E-3,1.609E3,3.9069E5,3.5000306E-3,1E0,4.0123675E-2,4.0012E4,-6.7835734E-3,8.8194094E-4,1.393E3,4.7201145E10,-1.4674163E-3,-1.10690445E-2,-3.3288016E-3,2.1461432E-3,1.04273124E-4,1.1426416E-2,7.4855336E-3,2.5653914E-3,-1.3599539E-2,1.1710652E0,-2.1256128E-2,-7.279162E-3,4.2979253E-4,-5.868709E-3],"split_indices":[53,39,53,32,0,46,4,54,32,3,0,0,0,2,30,0,108,39,9,0,0,12,32,0,0,0,0,0,0,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.57E2,8.4E1,1.11E2,4.6E1,7.9E1,5E0,6.9E1,4.2E1,5.3E1,2.6E1,2E0,3E0,3.2E1,3.7E1,9E0,3.3E1,1.3E1,4E1,5E0,2.1E1,2.4E1,8E0,3.1E1,6E0,2.5E1,8E0,2E0,1.1E1,9E0,3.1E1,4E0,2E1,4E0,4E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.0949485E-4,-7.023364E-2,1.07531965E-1,-9.5697366E-2,5.999637E-3,4.481289E-2,2.2272988E-1,-1.2741327E-1,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,3.131387E-2,6.709275E-3,1.5677929E-2,-1.9639476E-1,-7.4869595E-2,-1.0507656E-2,5.4045306E-3,-2.6656508E-1,-3.303782E-3,-3.9352458E-2,-1.0309419E-2,-5.7465717E-2,3.690726E-2,-8.487278E-3,-2.1258341E-2,-5.933776E-2,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,19,21,23,-1,25,-1,27,-1,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9060686E0,2.956339E-1,7.1252954E-1,3.6945975E-1,9.6911065E-2,2.3330902E-1,2.7442348E-1,3.0063093E-1,0E0,0E0,0E0,0E0,2.4897242E-1,0E0,0E0,2.9316032E-1,2.830122E-1,1.0231397E-1,0E0,2.7091873E-1,0E0,9.987865E-2,0E0,1.0716895E-1,1.2748241E-1,0E0,0E0,1.180381E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,15,15,16,16,17,17,19,19,21,21,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,20,22,24,-1,26,-1,28,-1,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1259552E2,7.087292E-2,1.2711891E0,1.769817E6,9.750871E6,3.4903356E7,1.6507233E3,7.5646E4,2.7345368E-4,-1.06787E-3,3.7435559E-3,1.3870817E-2,2.8041378E5,6.709275E-3,1.5677929E-2,1.3738942E0,3.0962145E0,1.0685258E6,5.4045306E-3,6.4146477E-3,-3.303782E-3,2.0766662E6,-1.0309419E-2,2.0250492E0,2.51838E6,-8.487278E-3,-2.1258341E-2,4.6376264E9,3.137896E-3,-4.30191E-3,2.2077754E-3,-2.2389342E-3,4.245212E-3,3.8171443E-3,-3.481661E-3],"split_indices":[53,39,40,1,9,7,4,9,0,0,0,0,29,0,0,54,54,34,0,39,0,49,0,58,33,0,0,32,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,1.51E2,9.9E1,1.13E2,3.8E1,6.5E1,3.4E1,8.6E1,2.7E1,2.8E1,1E1,2E0,6.3E1,2.4E1,1E1,3.6E1,5E1,4.4E1,1.9E1,2.2E1,1.4E1,4.2E1,8E0,2.2E1,2.2E1,1.8E1,4E0,3.6E1,6E0,1.6E1,6E0,9E0,1.3E1,4E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.23015465E-2,-3.6813665E-2,1.6186102E-1,-8.2913704E-2,3.2519907E-2,3.2985918E-3,2.6083997E-1,-9.4713494E-2,1.0172514E-3,2.072838E-2,1.34672765E-2,7.684792E-3,4.2016974E-1,-1.21286646E-1,-3.079713E-2,8.215429E-2,-2.6565844E-2,8.788486E-3,2.4215858E-2,-3.9600227E-3,-1.8518592E-1,1.9243775E-3,-4.5282864E-3,6.782496E-3,1.653277E-2,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,17,19,21,23,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8129876E0,5.9720445E-1,5.011734E-1,1.4259148E-1,2.3776487E-1,0E0,3.040551E-1,1.6780907E-1,0E0,2.1425411E-1,0E0,0E0,1.535585E-1,1.2175703E-1,1.739935E-1,1.5128689E-1,8.827151E-2,0E0,0E0,0E0,9.1700196E-2,0E0,0E0,0E0,9.928738E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,12,12,13,13,14,14,15,15,16,16,20,20,24,24],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,18,20,22,24,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,2.427E3,2.2324745E5,6.4796996E-1,1.4122614E2,3.2985918E-3,1.7598395E3,2.0256741E2,1.0172514E-3,1.3844932E7,1.34672765E-2,7.684792E-3,2.4394053E9,1.607E3,7.336684E2,1.6254681E0,6.925995E6,8.788486E-3,2.4215858E-2,-3.9600227E-3,1.7842995E8,1.9243775E-3,-4.5282864E-3,6.782496E-3,4.488E3,-2.7373503E-3,1.3223722E-3,-1.0304237E-2,-3.7255022E-3,3.6801142E-3,-2.6460723E-3],"split_indices":[43,2,29,40,57,0,4,53,0,46,0,0,5,2,4,54,52,0,0,0,7,0,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.85E2,6E1,1.11E2,7.4E1,3.3E1,2.7E1,1E2,1.1E1,7.2E1,2E0,1.9E1,8E0,7E1,3E1,3.1E1,4.1E1,4E0,4E0,4.9E1,2.1E1,1.5E1,1.5E1,1.4E1,1.7E1,2.5E1,1.6E1,1.3E1,8E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.4086864E-3,-5.6603998E-2,1.0975099E-1,-1.0331851E-1,7.5110937E-3,8.584382E-2,1.7369809E-2,-1.5483831E-1,-2.4599966E-2,-6.041252E-3,1.6805973E-2,4.8704322E-2,1.6501543E-1,-1.18140824E-1,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.352543E-2,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.7069752E-2,3.0004673E-3,-3.8697182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,-1,-1,-1,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.617809E0,4.5603204E-1,6.2057626E-1,3.530805E-1,9.050379E-2,2.5365812E-1,0E0,3.3182883E-1,1.1731397E-1,0E0,1.0263331E-1,2.5914997E-1,2.4869812E-1,9.6805155E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2964478E-1,0E0,0E0,0E0,0E0,0E0,8.78565E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,20,20,26,26],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,-1,-1,-1,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[3.4021906E2,1.1670399E-1,1.3821641E0,1.6796057E2,8.89E2,2.968068E5,1.7369809E-2,7.33452E-3,2.939566E5,-6.041252E-3,1.10018056E8,8.9397516E4,2.3842E4,6E0,-1.5120619E-2,1.4864015E-3,-3.4421992E-3,1.3453299E-3,-4.2668623E-3,1.466474E-2,3.0327642E-1,1.5272733E-2,4.782356E-3,4.8452977E-4,-5.855167E-3,-2.4431827E-3,5.615313E3,3.0004673E-3,-3.8697182E-3],"split_indices":[53,40,35,53,2,29,0,39,34,0,33,34,9,10,0,0,0,0,0,0,43,0,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,8.7E1,6.4E1,8.8E1,6E0,5.2E1,3.5E1,3E0,6.1E1,6.1E1,2.7E1,4.5E1,7E0,1.7E1,1.8E1,5.5E1,6E0,2E0,5.9E1,5E0,2.2E1,5E0,4E1,1.2E1,4.7E1,4.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-4.5817066E-3,-3.926012E-2,1.1888153E-1,-7.5602576E-2,5.771031E-2,6.1283264E-2,8.008875E-3,-9.3968615E-2,8.966747E-3,1.21591035E-2,3.8098045E-2,8.102908E-3,1.4816463E-3,-1.2472329E-1,-4.609013E-2,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,-1.0609691E-1,-2.3297147E-1,-7.477302E-3,-1.8048729E-1,-1.9920397E-1,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,19,21,-1,-1,-1,-1,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0151035E0,6.5632963E-1,1.823703E-1,2.1052504E-1,2.1035415E-1,9.229251E-2,0E0,1.576975E-1,1.7606108E-1,0E0,1.1426413E-1,0E0,0E0,1.1037278E-1,2.3217589E-1,0E0,0E0,0E0,0E0,1.10173464E-1,9.862223E-2,1.0037966E-1,1.288043E-1,1.144903E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,13,13,14,14,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,20,22,-1,-1,-1,-1,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1427817E0,3.165913E6,2.174459E5,5.8536225E5,2.207E3,1.1832633E0,8.008875E-3,1.1670399E-1,3.2744165E2,1.21591035E-2,1E1,8.102908E-3,1.4816463E-3,1.4031231E-2,2.587E3,-4.1670795E-3,3.2158294E-3,4.820973E-3,2.2023922E-4,1.1408451E0,5.1607388E5,2.8507206E-1,7.12E2,7.5608E4,-3.6233417E-3,-1.3476894E-2,-2.6595518E-3,1.6670758E-3,-2.9222148E-3,-1.5712187E-2,-3.953265E-3,-1.0793618E-2,5.0636334E-4],"split_indices":[42,1,29,29,0,40,0,40,4,0,3,0,0,39,2,0,0,0,0,54,34,43,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,1.84E2,5.1E1,1.34E2,5E1,2.8E1,2.3E1,1.1E2,2.4E1,3E0,4.7E1,4E0,2.4E1,6.6E1,4.4E1,9E0,1.5E1,1.4E1,3.3E1,5.8E1,8E0,3.5E1,9E0,1E1,4.8E1,5E0,3E0,2E1,1.5E1,2E0,7E0,8E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-6.5358323E-3,-5.01191E-2,1.1463253E-1,-6.566863E-2,3.5350563E-3,6.801129E-2,2.2740725E-1,-9.271781E-2,1.6208226E-4,3.614062E-3,4.4614673E-3,2.6938426E-1,5.7348673E-4,-7.9707764E-2,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,-1.20486856E-1,-4.1228443E-2,-2.9383153E-1,-9.8548844E-2,3.4898527E-3,-6.404513E-2,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,19,-1,-1,-1,-1,-1,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2573216E0,3.6213574E-1,3.1576872E-1,2.9499376E-1,0E0,1.0101345E-1,1.5835595E-1,1.3562757E-1,0E0,1.0294547E-1,0E0,1.21626854E-1,0E0,1.5508604E-1,0E0,0E0,0E0,0E0,0E0,1.5738219E-1,1.544519E-1,1.2428334E-1,2.1362385E-1,0E0,1.0190545E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,13,13,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,20,-1,-1,-1,-1,-1,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,5.8536225E5,5.76E3,3.165913E6,3.5350563E-3,1.7468556E2,3.41212E7,1E0,1.6208226E-4,1.2495134E6,4.4614673E-3,2.9375613E-1,5.7348673E-4,1.73507E5,-8.534754E-3,3.1009237E-3,-3.541013E-3,1.640823E-2,6.830766E-3,9.675635E4,1.2473118E0,9.48E2,4.43001E5,3.4898527E-3,1.8979991E6,-1.857731E-2,-2.7934995E-3,-2.9194602E-3,-1.1380917E-2,-3.8123874E-3,1.2494803E-3],"split_indices":[43,29,2,1,0,53,52,90,0,46,0,39,0,9,0,0,0,0,0,29,54,0,29,0,49,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.74E2,6.2E1,1.56E2,1.8E1,4.5E1,1.7E1,1.12E2,4.4E1,1.6E1,2.9E1,1.4E1,3E0,1.01E2,1.1E1,9E0,7E0,6E0,8E0,4.8E1,5.3E1,4E0,4.4E1,8E0,4.5E1,2E0,2E0,3.8E1,6E0,3.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.814724E-3,-4.223491E-2,1.1547112E-1,-8.021347E-2,1.0189096E-3,1.424386E-1,-2.2660806E-3,-5.76489E-2,-2.370273E-1,-5.7016924E-2,1.174304E-3,-1.1189182E-3,1.5711717E-1,-3.7425077E-3,2.5416018E-2,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.0735268E-1,2.2521736E-1,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.3238825E-1,3.9426633E-3,2.2386394E-2,9.90202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,-1,21,-1,-1,-1,-1,23,25,-1,-1,-1,-1,27,-1,-1,-1],"loss_changes":[1.2799627E0,2.77701E-1,3.414572E-1,3.0479503E-1,1.2174119E-1,1.6192663E-1,0E0,1.8898878E-1,1.3618523E-1,2.1964492E-1,0E0,0E0,1.7577171E-1,0E0,9.3134895E-2,0E0,0E0,0E0,0E0,1.2113613E-1,3.1686854E-1,0E0,0E0,0E0,0E0,1.3124907E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,14,14,19,19,20,20,25,25],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,-1,22,-1,-1,-1,-1,24,26,-1,-1,-1,-1,28,-1,-1,-1],"split_conditions":[3.799535E2,1.1306552E-1,4.183787E0,1.4031231E-2,1.83437E5,1.2055675E0,-2.2660806E-3,4.239E3,1.8018788E7,1.5086387E8,1.174304E-3,-1.1189182E-3,7.3667456E2,-3.7425077E-3,1.7E1,-1.4939204E-2,-4.0240847E-3,-1.3577036E-2,-9.335999E-4,1.57257E0,3.21879E6,3.659237E-3,-2.0619174E-3,8.104374E-3,2.6372606E-3,3.0256798E0,3.9426633E-3,2.2386394E-2,9.90202E-3],"split_indices":[53,40,58,39,9,54,0,10,46,7,0,0,53,0,3,0,0,0,0,54,30,0,0,0,0,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.68E2,7.3E1,8.9E1,7.9E1,6.3E1,1E1,7.9E1,1E1,2.4E1,5.5E1,5E0,5.8E1,5.9E1,2E1,5E0,5E0,2E0,2.2E1,3.5E1,2.3E1,1.1E1,9E0,1.2E1,2.3E1,1.2E1,1.1E1,3E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.0411531E-3,-5.0822284E-2,8.184148E-2,-8.0308676E-2,-6.2485766E-3,1.5205893E-1,6.5722945E-3,-1.08080305E-1,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,2.7616316E-1,-1.5828831E-2,1.3877054E-2,-8.242552E-2,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,-1.7038673E-1,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,-1,-1,-1],"loss_changes":[9.903048E-1,2.005888E-1,4.6938437E-1,2.0104021E-1,1.332165E-1,2.687695E-1,3.1198943E-1,1.609565E-1,0E0,0E0,0E0,0E0,1.3902038E-1,9.690636E-2,0E0,1.4167267E-1,0E0,0E0,0E0,0E0,0E0,1.059061E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,15,15,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,-1,-1,-1],"split_conditions":[3.2769293E2,1.529448E-1,1.0568718E7,5.5015685E9,9.74979E6,7.115615E2,1.4122614E2,1.3244295E5,-6.7966845E-5,-1.3902943E-3,3.3218598E-3,4.3697814E-3,1.8373182E8,1.128757E6,1.3877054E-2,6.7001674E-3,-9.176361E-3,1.3980723E-2,9.6033316E-4,-4.21256E-3,5.414545E-4,2.647867E-4,-2.2635548E-3,-9.845437E-3,-1.1284981E-3],"split_indices":[53,40,46,5,9,53,57,48,0,0,0,0,33,30,0,59,0,0,0,0,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,1.52E2,8.8E1,9.1E1,6.1E1,4.5E1,4.3E1,6.7E1,2.4E1,4.7E1,1.4E1,3.3E1,1.2E1,4.1E1,2E0,5.5E1,1.2E1,1E1,2E0,1E1,3.1E1,1.3E1,4.2E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.606068E-4,-4.0152058E-2,1.3688791E-1,-7.5740196E-2,5.956878E-3,1.0230381E-1,3.1381202E-1,-9.340738E-2,3.2788247E-2,6.146593E-3,-6.0711894E-3,9.786554E-3,6.620768E-2,1.6971182E-2,1.4838147E-3,-6.620601E-2,-1.9074214E-1,5.796871E-3,-1.0807036E-3,-2.5402438E-2,2.655493E-3,-2.4867726E-3,8.962364E-2,-1.7166455E-1,-3.69803E-2,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,25,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.397829E0,3.1715095E-1,3.2278395E-1,2.120583E-1,1.3876921E-1,2.0526332E-1,1.7587048E-1,2.362498E-1,9.541947E-2,0E0,1.03748016E-1,0E0,1.20494455E-1,0E0,0E0,2.2480932E-1,2.1755302E-1,0E0,0E0,1.3877995E-1,0E0,0E0,1.2990308E-1,1.283262E-1,1.8078491E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,15,15,16,16,19,19,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,26,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.633287E2,2.0358758E8,1.9292948E3,3.402546E6,2.3755E4,2.02983E5,3.41212E7,1.2869189E5,1.8000048E10,6.146593E-3,1.9425332E7,9.786554E-3,1.9241573E-1,1.6971182E-2,1.4838147E-3,1E0,9.975593E4,5.796871E-3,-1.0807036E-3,4.327E3,2.655493E-3,-2.4867726E-3,3.3996282E0,1.9E1,1.081E3,-1.7786428E-2,-5.6076376E-3,2.285919E-3,-2.3389077E-3,5.903482E-3,2.3846359E-4,-1.2048039E-2,-3.5523775E-3,-4.652304E-3,2.6779488E-4],"split_indices":[53,7,53,1,9,30,52,48,5,0,9,0,58,0,0,109,29,0,0,11,0,0,58,3,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,1.92E2,5.7E1,1.08E2,8.4E1,4.9E1,8E0,9.3E1,1.5E1,6E0,7.8E1,1E1,3.9E1,6E0,2E0,7.4E1,1.9E1,5E0,1E1,6.1E1,1.7E1,6E0,3.3E1,1.5E1,5.9E1,3E0,1.6E1,1.6E1,4.5E1,2.1E1,1.2E1,6E0,9E0,2.2E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.4908163E-3,-4.9458664E-2,8.817324E-2,6.931918E-2,-6.1066084E-2,6.9811605E-2,3.090321E-1,-5.98609E-4,7.956278E-3,-8.879114E-3,-5.08166E-2,1.4328934E-1,3.887047E-2,1.7160837E-2,2.8655254E-3,-1.0146142E-1,-3.0556865E-2,1.2192971E-3,2.755345E-1,8.296684E-2,-6.053145E-4,-6.862668E-2,-1.2999088E-2,1.7750489E-3,-5.1321223E-2,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,-1.3914266E-1,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,17,19,-1,-1,21,23,-1,25,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[1.1073979E0,2.1379605E-1,3.6368233E-1,1.3386652E-1,1.9253713E-1,1.9677043E-1,1.176427E-1,0E0,0E0,0E0,1.3131228E-1,3.889135E-1,1.4999606E-1,0E0,0E0,2.181817E-1,1.4260218E-1,0E0,1.17243946E-1,9.568061E-2,0E0,1.2064642E-1,0E0,0E0,8.933057E-2,0E0,0E0,0E0,0E0,1.0965279E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,11,11,12,12,15,15,16,16,18,18,19,19,21,21,24,24,29,29],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,18,20,-1,-1,22,24,-1,26,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[3.2769293E2,1.1623532E6,1.3758811E0,1E0,1.7565872E-3,2.3842E4,6.901E3,-5.98609E-4,7.956278E-3,-8.879114E-3,4.0835E4,7.6411234E4,9.380036E6,1.7160837E-2,2.8655254E-3,2.9138212E0,1.217759E0,1.2192971E-3,8.7292106E5,1.9643908E5,-6.053145E-4,5.3091907E2,-1.2999088E-2,1.7750489E-3,3.0962145E0,1.8956685E-2,7.606837E-3,-1.2165711E-3,4.663323E-3,1.0288912E5,-6.909854E-4,-1.610483E-3,-6.104915E-3,-8.095495E-3,1.0104963E-3],"split_indices":[53,46,35,8,59,9,0,0,0,0,9,29,46,0,0,59,54,0,34,29,0,4,0,0,54,0,0,0,0,34,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.51E2,9.4E1,1.3E1,1.38E2,8.8E1,6E0,8E0,5E0,8E0,1.3E2,2.5E1,6.3E1,4E0,2E0,3.6E1,9.4E1,1.4E1,1.1E1,3.4E1,2.9E1,3.2E1,4E0,2.1E1,7.3E1,3E0,8E0,6E0,2.8E1,1.3E1,1.9E1,6.4E1,9E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.576465E-3,-3.1257384E-2,1.1510225E-1,-6.0538724E-2,4.1183308E-2,1.9610284E-2,9.548955E-2,-1.616713E-1,-4.8437126E-2,1.07467435E-2,2.4615793E-2,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,-3.3256467E-2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,-4.011636E-2,4.8860135E-3,-9.835392E-2,-1.4123669E-2,7.2520383E-4,-1.8231265E-1,7.876769E-5,-6.9828774E-3,-2.5798842E-1,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,-1,19,-1,-1,-1,21,-1,23,25,-1,27,-1,-1,29,-1,-1,-1],"loss_changes":[9.6316123E-1,4.0339252E-1,3.4389424E-1,1.5739092E-1,1.8081136E-1,0E0,1.22983694E-1,1.2277371E-1,1.5455526E-1,0E0,1.18286856E-1,0E0,0E0,0E0,0E0,1.0909271E-1,0E0,0E0,0E0,1.5149972E-1,0E0,3.022702E-1,1.6528234E-1,0E0,1.5847087E-1,0E0,0E0,1.461187E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,15,15,19,19,21,21,22,22,24,24,27,27],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,-1,20,-1,-1,-1,22,-1,24,26,-1,28,-1,-1,30,-1,-1,-1],"split_conditions":[1.2185057E0,3.165913E6,1E0,6.7001674E-3,2.207E3,1.9610284E-2,2.0787016E5,2.647867E-4,2.0017123E0,1.07467435E-2,1E1,2.2835857E-3,6.486561E-3,-9.641291E-3,-4.718611E-4,4.5971466E2,-5.8607063E-3,4.196861E-3,-3.0516347E-4,4.0054E4,4.8860135E-3,1.8259804E-1,2.2912737E1,7.2520383E-4,1.3738942E0,7.876769E-5,-6.9828774E-3,1.622E3,-2.4206224E-3,-4.5868345E-3,-1.6053442E-2],"split_indices":[43,1,13,59,0,0,29,40,54,0,3,0,0,0,0,53,0,0,0,9,0,59,57,0,54,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.88E2,5.8E1,1.34E2,5.4E1,2E0,5.6E1,1.3E1,1.21E2,3E0,5.1E1,3.2E1,2.4E1,9E0,4E0,1.04E2,1.7E1,1.5E1,3.6E1,1E2,4E0,3E1,7E1,1.3E1,1.7E1,6.4E1,6E0,1E1,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[9.720544E-3,-4.5713637E-2,8.041132E-2,4.5472946E-2,-5.675482E-2,4.4581052E-2,1.6859385E-1,-1.5774339E-3,5.743392E-3,-6.856326E-2,2.9137306E-4,8.072134E-3,3.0883143E-2,1.6059604E-2,5.5405004E-3,-6.1498176E-2,-1.2888393E-2,7.4021585E-2,-2.9413155E-3,-9.75898E-3,-5.4604717E-2,7.0612812E-3,1.7218302E-3,-2.8861985E-3,3.3276238E-2,-7.160593E-3,-4.491236E-2,-1.0041681E-3,3.9801714E-3,4.2815937E-4,-6.328989E-2,-7.997919E-3,-2.0749776E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,17,-1,-1,19,-1,21,23,-1,25,-1,-1,-1,27,-1,29,-1,-1,-1,31,-1,-1],"loss_changes":[9.9902195E-1,1.4655435E-1,3.463596E-1,1.1724831E-1,9.633365E-2,1.5005943E-1,2.1894133E-1,0E0,0E0,1.5487564E-1,0E0,0E0,1.096528E-1,0E0,0E0,1.0625979E-1,0E0,9.375355E-2,1.00620806E-1,0E0,1.0192835E-1,0E0,0E0,0E0,9.3399785E-2,0E0,9.716697E-2,0E0,0E0,0E0,1.2727156E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,12,12,15,15,17,17,18,18,20,20,24,24,26,26,30,30],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,18,-1,-1,20,-1,22,24,-1,26,-1,-1,-1,28,-1,30,-1,-1,-1,32,-1,-1],"split_conditions":[2.797224E2,1.1696849E6,1.5972575E0,2.25E2,4.343271E5,8.9397516E4,1.8426E4,-1.5774339E-3,5.743392E-3,2.9892595E6,2.9137306E-4,8.072134E-3,1.1572553E7,1.6059604E-2,5.5405004E-3,1.3525729E6,-1.2888393E-2,4.0012E4,1.4813267E3,-9.75898E-3,1.7565872E-3,7.0612812E-3,1.7218302E-3,-2.8861985E-3,5.415429E4,-7.160593E-3,1.1710652E0,-1.0041681E-3,3.9801714E-3,4.2815937E-4,1E0,-7.997919E-3,-2.0749776E-3],"split_indices":[53,46,42,0,48,34,9,0,0,52,0,0,46,0,0,46,0,9,56,0,59,0,0,0,29,0,54,0,0,0,67,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,1.42E2,1.11E2,1.5E1,1.27E2,8E1,3.1E1,8E0,7E0,1.07E2,2E1,6E0,7.4E1,4E0,2.7E1,1.05E2,2E0,3.2E1,4.2E1,3E0,1.02E2,8E0,2.4E1,1.5E1,2.7E1,7E0,9.5E1,1.4E1,1.3E1,2.4E1,7.1E1,7E0,6.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.432755E-3,-3.2541275E-2,1.2062466E-1,-6.6113114E-2,7.3056045E-4,7.963434E-2,2.6087046E-1,-2.4810357E-2,-1.07705206E-1,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,-1.1733579E-1,-3.5330087E-3,-1.7337382E-1,-5.0983664E-2,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,3.909637E-2,-2.709494E-1,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0041343E0,3.2042485E-1,2.954054E-1,1.9488263E-1,0E0,1.0474539E-1,2.4700874E-1,1.15738496E-1,2.0347583E-1,0E0,0E0,0E0,0E0,1.0116187E-1,1.1524878E-1,1.1706191E-1,1.1447847E-1,0E0,0E0,0E0,1.1847381E-1,1.464771E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15,16,16,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.3341238E9,2.2973125E3,1.19331E5,7.3056045E-4,2.0787016E5,2.174459E5,3.559166E7,6.6037946E2,1.1824071E-3,5.517124E-3,4.271217E-3,1.7785117E-2,4.3399894E5,1.09E3,7.359468E8,1.48068905E1,-4.4437486E-4,-8.943034E-3,-2.5148224E-3,1.2765957E0,1.721649E8,-5.0164736E-3,5.2093086E-4,-4.6782806E-3,6.1944374E-3,-8.495531E-5,-1.48065435E-2,-9.232005E-4],"split_indices":[43,5,4,30,0,29,29,7,4,0,0,0,0,29,2,5,57,0,0,0,54,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,1.92E2,5.4E1,1.14E2,7.8E1,4.3E1,1.1E1,5.8E1,5.6E1,2.1E1,2.2E1,6E0,5E0,1E1,4.8E1,2.5E1,3.1E1,5E0,5E0,2.1E1,2.7E1,8E0,1.7E1,1.5E1,1.6E1,7E0,2E1,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-2.4172934E-3,-4.873043E-2,5.479038E-2,2.9505575E-3,-7.196424E-2,1.9374692E-1,3.8230103E-2,-2.232737E-3,2.4846569E-3,-2.2767627E-1,-5.168247E-2,1.6559279E-3,3.4322336E-1,6.4282104E-2,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,9.1448665E-2,1.5415065E-2,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.809334E-1,1.707184E-1,2.5722635E-1,1.343776E-1,2.9772854E-1,2.6643854E-1,1.8172754E-1,0E0,0E0,1.076473E-1,2.4328896E-1,0E0,1.7182684E-1,9.815651E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.945521E-2,1.7858982E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.47887E5,1.431E4,1.081E3,2.07684E5,9.675635E4,1.8018788E7,-2.232737E-3,2.4846569E-3,1.3738942E0,9.476661E1,1.6559279E-3,9.63E2,2.1949912E7,-1.2294926E-3,-1.49789965E-2,-4.9137124E-3,-6.78596E-3,-1.0749992E-3,2.8906337E-3,2.0298863E-2,7.3667456E2,1.25836E5,2.9390594E-3,8.22135E-3,5.372944E-3,-1.8365231E-3],"split_indices":[53,1,9,2,1,29,46,0,0,54,53,0,0,33,0,0,0,0,0,0,0,53,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.55E2,1.41E2,1.14E2,4.4E1,9.7E1,1.1E1,1.03E2,2.2E1,2.2E1,1E1,8.7E1,6E0,5E0,7.4E1,2.9E1,4E0,6E0,1.7E1,7E1,2E0,3E0,4.7E1,2.7E1,3.9E1,8E0,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.1405309E-3,-4.1295454E-2,7.539111E-2,-4.773554E-2,2.649901E-3,5.8098752E-2,2.4969846E-1,-3.5499748E-2,-1.1151391E-1,7.256275E-2,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,-2.639264E-2,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,-3.14371E-2,-4.9293704E-2,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,-1,-1,21,23,-1,-1,-1],"loss_changes":[7.971391E-1,1.0867721E-1,2.6279783E-1,1.1465886E-1,0E0,1.4647299E-1,1.3158771E-1,9.851079E-2,1.7702582E-1,1.16677344E-1,0E0,0E0,0E0,0E0,9.4192E-2,0E0,0E0,0E0,0E0,0E0,9.489479E-2,9.564981E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,14,14,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,-1,-1,22,24,-1,-1,-1],"split_conditions":[5.9570425E5,2.407794E7,1.3821641E0,2.9138212E0,2.649901E-3,1E0,3.41212E7,1.4781966E-3,3.67E2,2.3581433E5,-2.5152352E-3,1.4133884E-2,7.897464E-4,-5.328078E-3,1E1,-1.4945791E-2,-3.2706773E-3,1.4116977E-3,4.8842696E-3,5.465062E-3,3.3341238E9,5.77032E5,6.2960485E-4,-1.4587467E-3,-5.459354E-3],"split_indices":[48,9,35,59,0,90,52,59,0,29,0,0,0,0,10,0,0,0,0,0,5,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.6E2,9.1E1,1.51E2,9E0,8.4E1,7E0,1.28E2,2.3E1,7.5E1,9E0,5E0,2E0,1.1E1,1.17E2,2E0,2.1E1,3.8E1,3.7E1,3E0,1.14E2,8.2E1,3.2E1,6.9E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.8670357E-3,-4.5970436E-2,5.2696485E-2,6.0541928E-2,-5.781559E-2,2.443929E-2,1.2535055E-1,-9.904153E-4,6.934784E-3,-7.859939E-3,-4.945917E-2,3.5898713E-3,3.4437096E-3,2.2750926E-1,1.946528E-3,-6.620565E-2,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,-1.1629289E-1,-4.547119E-2,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,-1,21,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[6.025647E-1,1.8644026E-1,2.186605E-1,1.2642913E-1,1.2496975E-1,9.5626205E-2,2.3731667E-1,0E0,0E0,0E0,9.588456E-2,0E0,9.368942E-2,1.0582209E-1,0E0,8.921841E-2,0E0,0E0,0E0,0E0,0E0,1.2605882E-1,9.986128E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,15,15,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,-1,22,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[2.7468146E2,1.1623532E6,6.657047E2,1E0,1.7565872E-3,1E1,2.45226E6,-9.904153E-4,6.934784E-3,-7.859939E-3,2.9073395E9,3.5898713E-3,5.1332E4,3.0256798E0,1.946528E-3,4.0054E4,-1.3890445E-4,2.3350706E-3,-1.2118951E-3,1.5357346E-2,5.808953E-3,2.8516157E0,2.704375E6,-3.5807842E-3,-1.2397304E-2,-9.232241E-4,-4.762593E-3],"split_indices":[53,46,53,8,59,3,30,0,0,0,5,0,9,55,0,9,0,0,0,0,0,59,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.44E2,1.07E2,1.4E1,1.3E2,7.8E1,2.9E1,8E0,6E0,7E0,1.23E2,2E1,5.8E1,1.2E1,1.7E1,9E1,3.3E1,2.2E1,3.6E1,4E0,8E0,2.5E1,6.5E1,2.2E1,3E0,4.8E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.004087E-3,-2.8343314E-2,7.549351E-2,-5.5867407E-2,2.2657372E-2,1.4241742E-2,5.9626795E-2,-6.4280997E-3,-4.644786E-2,4.461195E-2,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,-5.3805247E-2,4.6029454E-3,4.787177E-5,-8.447469E-3,-4.588249E-2,-1.3219344E-3,-5.0460016E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,-1,-1,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[5.418513E-1,2.4271044E-1,2.6314437E-1,8.962074E-2,1.0539442E-1,0E0,1.0044208E-1,0E0,1.2327367E-1,1.2701635E-1,0E0,0E0,0E0,0E0,1.0074824E-1,0E0,0E0,0E0,9.979747E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,14,14,18,18],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,-1,-1,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[9.821357E-1,7.422E3,1.215951E5,9.87574E2,2.1949912E7,1.4241742E-2,2.216135E5,-6.4280997E-3,1.8E1,1.1572553E7,-2.3667053E-3,1.0788076E-3,4.4955257E-3,4.7356063E-3,1.3525729E6,4.6029454E-3,4.787177E-5,-8.447469E-3,2.0134454E0,-1.3219344E-3,-5.0460016E-3],"split_indices":[42,10,34,34,33,0,29,0,10,46,0,0,0,0,46,0,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,1.71E2,7E1,1.11E2,6E1,3E0,6.7E1,9E0,1.02E2,4.7E1,1.3E1,3.8E1,2.9E1,4E0,9.8E1,1.9E1,2.8E1,4E0,9.4E1,7.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.1594397E-2,-1.194265E-2,9.3121335E-2,-3.9583016E-2,2.608233E-2,1.6468607E-2,7.527401E-2,-5.18759E-2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,4.0722504E-4,-1.7415832E-3,3.3587667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,-1,15,-1,-1],"loss_changes":[4.712434E-1,2.0177807E-1,2.5062546E-1,1.2383151E-1,1.3076647E-1,0E0,1.04620725E-1,1.0193059E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.9496724E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1.3394321E0,2.427E3,1.431E4,9.74979E6,2.637815E1,1.6468607E-2,2.174459E5,2.0256741E2,2.1511593E-3,2.4513733E-3,-1.1758952E-3,1.3938246E-3,5.347099E-3,-3.1317002E-3,3.4234106E5,-1.7415832E-3,3.3587667E-3],"split_indices":[43,2,9,9,57,0,29,53,0,0,0,0,0,0,29,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.9E2,5.4E1,1.1E2,8E1,2E0,5.2E1,9.7E1,1.3E1,5.1E1,2.9E1,2.8E1,2.4E1,7E1,2.7E1,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-8.11151E-3,-2.3770498E-2,1.3511473E-1,-4.903885E-2,3.7162323E-4,2.8097564E-1,1.2597005E-3,-4.1468557E-2,-9.574549E-3,1.8381007E-2,7.012952E-3,-1.3489653E-3,-6.7639E-2,-9.2418713E-4,6.152402E-3,-8.961814E-2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,-1,13,15,-1,-1,17,-1,-1,-1],"loss_changes":[5.4596394E-1,1.7908293E-1,3.6382854E-1,1.495364E-1,0E0,1.009686E-1,0E0,1.2485254E-1,0E0,0E0,0E0,1.3845807E-1,1.4358482E-1,0E0,0E0,9.946346E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,11,11,12,12,15,15],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,-1,14,16,-1,-1,18,-1,-1,-1],"split_conditions":[7.2259143E2,2.2348747E2,2.45226E6,2.1405182E2,3.7162323E-4,3.0256798E0,1.2597005E-3,4.37E2,-9.574549E-3,1.8381007E-2,7.012952E-3,4.02E2,5.8536225E5,-9.2418713E-4,6.152402E-3,4.73E2,9.537837E-4,-1.1887216E-2,-3.413695E-3],"split_indices":[53,53,30,53,0,55,0,0,0,0,0,0,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.18E2,2.3E1,1.22E2,9.6E1,9E0,1.4E1,1.18E2,4E0,3E0,6E0,4.7E1,7.1E1,4.2E1,5E0,5.7E1,1.4E1,2E0,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.816721E-3,-3.212387E-2,7.8340545E-2,-5.9203006E-2,-2.047665E-3,1.05734095E-1,-1.2828258E-3,-4.2488243E-2,-1.6570118E-1,5.200363E-3,-1.1345699E-2,6.766135E-2,1.9234525E-1,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,1.8420164E-2,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,-1,-1,-1],"loss_changes":[6.675061E-1,1.4077646E-1,2.3921198E-1,1.54948E-1,9.489868E-2,1.9802713E-1,0E0,1.2677193E-1,1.3189653E-1,0E0,9.6501775E-2,1.6102695E-1,1.9053322E-1,0E0,0E0,0E0,0E0,0E0,8.944257E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7136658E2,1.5442211E8,3.6207435E0,1.2869189E5,2.3755E4,5.5303E9,-1.2828258E-3,7.92582E5,3.396945E7,5.200363E-3,3.7230852E0,9.90141E6,6.464779E6,-3.0265606E-3,6.97293E-4,-1.1118152E-2,-1.4908579E-3,-2.2907674E-3,4.33121E0,4.406163E-3,-1.5327517E-3,1.2167102E-2,2.8492063E-3,3.2075641E-3,-7.158521E-4],"split_indices":[53,7,58,48,9,5,0,12,12,0,55,46,30,0,0,0,0,0,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,1.72E2,7.9E1,9E1,8.2E1,6.3E1,1.6E1,7.9E1,1.1E1,5E0,7.7E1,4.5E1,1.8E1,5.4E1,2.5E1,6E0,5E0,3.2E1,4.5E1,3.4E1,1.1E1,1E1,8E0,1.7E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.911687E-4,-2.2826223E-2,8.1994176E-2,-2.6359616E-2,9.769677E-3,1.5213658E-2,6.624614E-2,-5.9948202E-2,-4.920752E-3,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9672684E-1,1.8459845E-1,2.2044265E-1,1.4588554E-1,0E0,0E0,1.2346178E-1,1.278441E-1,1.16657905E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.4122614E2,1E0,5.283708E0,9.769677E-3,1.5213658E-2,1.1598054E8,3.901529E6,5.8536225E5,3.8505462E-3,-1.3849295E-3,-3.3979027E-3,1.2565526E-3,-7.89868E-4,2.8441993E-3],"split_indices":[43,57,13,57,0,0,33,9,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,2.04E2,5.7E1,2.02E2,2E0,2E0,5.5E1,7.8E1,1.24E2,4.5E1,1E1,6.5E1,1.3E1,1.05E2,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-4.2027067E-3,-2.6648754E-2,8.658342E-2,-3.084103E-3,-2.5529137E-3,1.4771055E-2,2.9963553E-3,4.294585E-3,-1.1864337E-2,-5.120143E-2,4.8290283E-4,1.1069501E-3,-3.521963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[5.0360364E-1,2.0985942E-1,1.97959E-1,0E0,1.22443795E-1,0E0,0E0,0E0,1.0790554E-1,1.0145053E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.4125891E0,5.283708E0,1E0,-3.084103E-3,9.165388E4,1.4771055E-2,2.9963553E-3,4.294585E-3,1.1670399E-1,3.712E4,4.8290283E-4,1.1069501E-3,-3.521963E-3],"split_indices":[43,57,13,0,34,0,0,0,40,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,1.97E2,4.8E1,6.9E1,1.28E2,2E0,4.6E1,1E1,1.18E2,4.3E1,7.5E1,1.2E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.173793E-3,-2.6264051E-2,8.658401E-2,-6.669964E-2,-8.2213007E-4,1.4341326E-2,7.03292E-2,1.6294725E-2,-8.671885E-2,2.6084114E-2,-4.4527646E-2,8.693059E-4,1.125322E-1,-1.6205347E-3,6.1647515E-3,-2.2572479E-1,-6.812656E-2,4.0135182E-2,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.266243E-1,2.0775451E-1,1.8148205E-1,1.3100863E-1,1.4815538E-1,0E0,1.0412836E-1,1.1208219E-1,1.4744174E-1,1.12065986E-1,9.200167E-2,0E0,9.0320885E-2,0E0,0E0,9.239629E-2,1.0407472E-1,1.1784806E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5098532E0,5.283708E0,1E0,3.6E1,1.7303246E7,1.4341326E-2,2.174459E5,7.014272E7,9.975593E4,5.8414636E0,2.094E3,8.693059E-4,4.7201157E-1,-1.6205347E-3,6.1647515E-3,7.725141E4,1.9415865E2,2.4256398E8,-3.2064416E-3,-6.229188E-3,-1.0594416E-3,6.114475E-3,-3.2122486E-4,-2.9088247E-3,-1.4329344E-2,-3.7311227E-3,1.4850899E-3,-6.79218E-3,2.1078652E-3],"split_indices":[42,57,13,10,46,0,29,7,29,55,2,0,39,0,0,29,53,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.01E2,5.1E1,7.7E1,1.24E2,2E0,4.9E1,1.5E1,6.2E1,7.7E1,4.7E1,2.3E1,2.6E1,1.1E1,4E0,6E0,5.6E1,6.8E1,9E0,7E0,4E1,2.1E1,5E0,3E0,3E0,4.8E1,8E0,2E0,6.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[5.15778E-3,-1.8465955E-2,8.479578E-2,-4.582774E-2,2.8703757E-2,1.1748621E-2,2.8553656E-3,-6.113014E-2,7.824005E-4,7.2038495E-3,8.444443E-3,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,13,-1,-1,-1,-1],"loss_changes":[4.6061453E-1,2.4524778E-1,1.8237612E-1,1.1817169E-1,1.9200628E-1,0E0,0E0,1.1865306E-1,0E0,0E0,1.02744974E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,10,10],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.2446029E0,1.781973E6,1.8426E4,5.4279834E-1,2.207E3,1.1748621E-2,2.8553656E-3,4.1150245E-1,7.824005E-4,7.2038495E-3,1E1,-2.3664224E-3,-1.1317383E-2,3.045416E-3,-7.9580717E-4],"split_indices":[43,1,9,42,0,0,0,43,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.88E2,5.5E1,1.19E2,6.9E1,4E0,5.1E1,9.6E1,2.3E1,8E0,6.1E1,9.4E1,2E0,1.8E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[2.6694385E-3,-1.2630397E-2,9.763298E-2,-2.7587097E-3,-3.7950324E-3,2.2221443E-1,6.921976E-4,4.2745727E-3,-9.810083E-3,4.104886E-3,1.3904189E-2,-3.826968E-2,8.1882236E-4,-4.5390716E-3,-2.056427E-2,-1.03351854E-1,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,-1,-1,11,-1,-1,13,-1,-1,15,17,-1,-1,-1],"loss_changes":[3.4557673E-1,1.5018955E-1,3.3265132E-1,1.3020961E-1,0E0,1.3063407E-1,0E0,0E0,1.3977444E-1,0E0,0E0,9.7968146E-2,0E0,0E0,9.9550575E-2,1.2142539E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,11,11,14,14,15,15],"right_children":[2,4,6,8,-1,10,-1,-1,12,-1,-1,14,-1,-1,16,18,-1,-1,-1],"split_conditions":[6.56755E2,2.4219654E0,2.660842E6,1.1623532E6,-3.7950324E-3,2.1061536E3,6.921976E-4,4.2745727E-3,1.7179383E5,4.104886E-3,1.3904189E-2,1E0,8.1882236E-4,-4.5390716E-3,1.27318E5,1.3738942E0,-1.3446219E-4,-1.0741342E-2,-1.239619E-3],"split_indices":[53,54,30,46,0,4,0,0,33,0,0,109,0,0,29,54,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2.04E2,3.2E1,1.81E2,2.3E1,1.2E1,2E1,1.1E1,1.7E2,6E0,6E0,8.5E1,8.5E1,1.7E1,6.8E1,1.1E1,5.7E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[5.2459247E-4,-2.1839963E-2,5.5326115E-2,6.403799E-2,-2.9754521E-2,6.706761E-2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,3.54164E-2,1.4349867E-1,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,13,15,-1,-1,-1,-1],"loss_changes":[3.0025756E-1,1.2017081E-1,1.2092781E-1,9.031686E-2,1.1387043E-1,1.5458077E-1,0E0,0E0,0E0,0E0,0E0,1.247524E-1,1.0393754E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,11,11,12,12],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,14,16,-1,-1,-1,-1],"split_conditions":[3.799535E2,1.1623532E6,6.236639E3,2.0535674E0,1.4781966E-3,9.386909E2,-3.624268E-3,5.5723083E-3,-1.1760796E-3,-6.7548724E-3,-1.0420042E-3,9E0,7.529323E4,6.178837E-3,4.6440098E-4,2.1579028E-3,9.050841E-3],"split_indices":[53,46,4,54,59,53,0,0,0,0,0,3,29,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.73E2,7E1,1.4E1,1.59E2,6.5E1,5E0,8E0,6E0,6E0,1.53E2,4.7E1,1.8E1,8E0,3.9E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.029568E-4,-6.5150703E-3,1.8107072E-1,-3.652972E-2,2.4049532E-2,1.0449934E-2,-1.5273974E-3,-2.0122712E-2,-9.696651E-2,3.5459854E-2,-2.3149115E-3,4.5559215E-3,-2.7940093E-2,-1.3360148E-3,-1.7182508E-1,6.808551E-3,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,4.025893E-2,3.4104274E-3,-9.366535E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[3.37975E-1,2.2380325E-1,1.3455206E-1,1.2001395E-1,1.08324245E-1,0E0,0E0,9.831457E-2,1.2388055E-1,1.0830644E-1,0E0,0E0,1.4380896E-1,0E0,1.03295654E-1,1.3083223E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.5229566E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,12,12,14,14,15,15,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[1.9292948E3,2.0718746E8,6.236639E3,4.352025E0,2.2979798E0,1.0449934E-2,-1.5273974E-3,1E1,5.694004E6,3.642194E0,-2.3149115E-3,4.5559215E-3,9.476661E1,-1.3360148E-3,4.752366E0,4.854831E8,3.1106474E-3,-3.974098E-3,-1.4802981E-4,-1.24530345E-2,-3.4642415E-3,-2.4974837E-3,1.9647126E7,3.4104274E-3,-9.366535E-4],"split_indices":[53,7,4,55,54,0,0,10,46,55,0,0,53,0,57,12,0,0,0,0,0,0,46,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.42E2,9E0,1.22E2,1.2E2,7E0,2E0,9.7E1,2.5E1,1.05E2,1.5E1,5E0,9.2E1,1.4E1,1.1E1,5.9E1,4.6E1,2.5E1,6.7E1,4E0,7E0,2E1,3.9E1,2.4E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics index 70e919724..dd751b17f 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics @@ -46,64 +46,71 @@ 44,diskBytesSpilled_mean,0.00030759146,1407298.6859824378,5836681.87707674,0.0,0.0,0.0,0.0,56913253.10538942 45,sqlOp_SortMergeJoin,0.00025351142,0.629746835443038,0.4836382609453798,0.0,0.0,1.0,1.0,1.0 46,sqlOp_SubqueryOutputBroadcast,0.00011199158,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 -47,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_LocalTableScan,0.0,0.0031645569620253164,0.056254395046301184,0.0,0.0,0.0,0.0,1.0 -51,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +47,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sqlOp_LocalTableScan,0.0,0.0031645569620253164,0.056254395046301184,0.0,0.0,0.0,0.0,1.0 52,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.10443037974683544,0.30630308399462325,0.0,0.0,0.0,0.0,1.0 -53,sqlOp_HashAggregate,0.0,0.990506329113924,0.09712565886833821,0.0,1.0,1.0,1.0,1.0 -54,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -57,sqlOp_Project,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -58,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_WindowGroupLimit,0.0,0.0189873417721519,0.1366965815037675,0.0,0.0,0.0,0.0,1.0 -60,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan text ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_ReusedSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 -67,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Filter,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -74,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -77,maxMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 -78,maxOnHeapMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 -79,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,executorMemory,0.0,16384.0,0.0,16384.0,16384.0,16384.0,16384.0,16384.0 -81,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -84,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,sqlOp_Scan parquet ,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -88,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -89,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,sqlOp_ColumnarToRow,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -94,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -95,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,platform_emr,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -98,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -100,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sqlOp_BroadcastExchange,0.0,0.9240506329113924,0.2653372528417936,0.0,1.0,1.0,1.0,1.0 -104,sqlOp_BroadcastNestedLoopJoin,0.0,0.028481012658227847,0.16660637848720578,0.0,0.0,0.0,0.0,1.0 -105,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +53,sqlOp_ReusedSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +54,sqlOp_HashAggregate,0.0,0.990506329113924,0.09712565886833821,0.0,1.0,1.0,1.0,1.0 +55,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,sqlOp_WindowGroupLimit,0.0,0.0189873417721519,0.1366965815037675,0.0,0.0,0.0,0.0,1.0 +57,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_Filter,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +59,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +60,sqlOp_Project,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +61,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +78,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +82,sqlOp_Scan parquet,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +83,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +85,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +87,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +90,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,executorMemory,0.0,16384.0,0.0,16384.0,16384.0,16384.0,16384.0,16384.0 +93,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,maxOnHeapMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 +95,sqlOp_ColumnarToRow,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +96,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_BroadcastNestedLoopJoin,0.0,0.028481012658227847,0.16660637848720578,0.0,0.0,0.0,0.0,1.0 +106,maxMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 +107,sqlOp_BroadcastExchange,0.0,0.9240506329113924,0.2653372528417936,0.0,1.0,1.0,1.0,1.0 +108,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,platform_emr,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +114,sqlOp_WindowSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg index f1909ca15..836236f24 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"83"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0461990498","gamma":"0.0150604062","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0461990498","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"10","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.0150604062","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.671188235"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"83"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0461990498","gamma":"0.0150604062","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0461990498","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"10","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.0150604062","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.671188235"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json index 691af8005..d17cd986b 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_ObjectHashAggregate","sqlOp_Project","sqlOp_Scan parquet ","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_onprem","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan JDBCRelation","sqlOp_Scan OneRowRelation","sqlOp_Scan csv ","sqlOp_Scan json ","sqlOp_Scan orc ","sqlOp_Scan text ","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"83"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-4.5668036E-3,-2.3331916E-1,6.575819E-1,-7.3830986E-1,-2.9545367E-2,7.585778E-1,-9.047681E-1,-9.8671955E-1,-5.689058E-1,-2.1748337E-1,1.7790031E-1,4.7353148E-1,9.98321E-1,-5.0334964E-2,-1.9184373E-2,-1.009283E0,-1.9581432E-2,-7.572694E-1,-2.6597184E-1,-3.4746066E-1,2.4795938E-2,3.172561E-1,1.9483695E-2,3.2106972E-1,8.287864E-1,8.6895436E-2,8.755892E-1,-4.7518518E-2,-2.757553E-2,-6.920537E-1,-1.1876076E0,3.4099087E-2,-4.1624886E-1,-5.5970305E-1,-2.3217718E-1,-1.1214657E-2,3.8214678E-1,2.0068753E-1,4.5767924E-1,7.318743E-2,-3.5774136E-1,-6.0188014E-2,4.028545E-1,8.6816084E-1,2.0080749E-2,5.641912E-1,1.1029484E0,-3.3124804E-2,-1.3965118E-2,-6.606181E-2,-3.2697614E-2,-7.3586625E-1,-2.912001E-1,-5.0898236E-1,-3.7304517E-2,-1.3420476E-1,-4.1529045E-1,-2.2086361E-1,2.3870613E-2,2.2889538E-2,8.889135E-3,3.0159533E-1,-9.188867E-2,5.2135813E-1,-9.34718E-3,1.5662609E-2,2.2340204E-1,-2.1273324E-1,-2.7960172E-2,-1.0369004E-2,4.438314E-2,2.2642371E-1,5.3501856E-1,2.715325E-2,4.549749E-2,6.6693574E-1,1.7469372E-1,1.2068436E0,6.308485E-1,-3.998909E-2,-2.1338705E-2,-4.135335E-1,-1.0867557E-1,-5.4981214E-1,-1.2486093E-2,-1.0617143E-2,-2.5443727E-1,-8.711125E-2,-5.048686E-1,8.628427E-3,-1.9532233E-2,-1.7093034E-1,5.5031493E-2,6.1803424E-1,2.2785096E-1,1.0172568E-1,-1.824982E-2,4.6515972E-3,5.708142E-1,-8.626919E-2,7.5669944E-2,1.1983855E-1,3.8197517E-1,-1.3491115E-2,-2.7342E-3,-1.4498445E-3,5.8407313E-3,3.234492E-1,-9.9598905E-3,6.387942E-1,2.3165841E-1,7.3940027E-1,1.0149398E-2,-7.78564E-3,1.6972052E-2,1.2719901E0,2.2384929E-2,3.457816E-2,1.2448564E-2,-1.0324918E-2,-4.790183E-1,-8.777231E-3,-1.1875937E-2,-5.716248E-1,-1.1855732E-2,-4.4726193E-2,1.0953399E-2,-3.516006E-1,-8.671726E-2,3.5364935E-4,-8.473294E-3,-5.602632E-1,-4.3281703E-3,-1.1312302E-2,-1.5357796E-3,1.956537E-1,2.4461351E-2,3.5201266E-2,1.8253269E-2,7.594841E-2,2.8369758E-1,-2.4591943E-2,1.4264061E-2,4.69715E-1,3.320946E-2,8.841761E-3,-1.5208796E-1,-9.6291214E-2,1.2223156E-1,1.9617124E-1,1.3227642E-2,1.0226706E-2,2.4505768E-2,2.6079942E-2,2.1761277E-1,6.965718E-1,1.1330703E-2,4.0160073E-3,1.3949478E-2,3.624573E-2,1.4810829E-2,6.3704684E-2,4.4903275E-2,-2.6027502E-2,-9.909527E-3,-3.1908199E-3,1.9831837E-3,-1.4045271E-2,-5.9957147E-1,4.5774165E-3,-8.285467E-2,-5.9024906E-3,-1.7932279E-2,-9.371145E-3,3.2423376E-3,-6.361793E-1,-1.116317E-2,5.031539E-3,1.2207902E-2,-1.4691992E-1,6.4051405E-2,-1.1300913E-3,5.477288E-3,2.2185285E-1,1.9904312E-2,3.4397193E-3,-5.4847435E-3,2.4269028E-2,1.0960829E-2,-6.01094E-2,-1.3799966E-2,-9.214199E-3,3.1130412E-3,1.6172273E-1,-5.6327586E-3,1.2809288E-3,1.3326791E-2,-3.5169765E-3,4.6169646E-3,1.0040688E-1,1.6919017E-2,1.1915705E-2,3.5820182E-2,-2.899007E-2,-1.3642825E-2,-8.5165E-3,-4.8935594E-4,-3.4403258E-3,3.116593E-3,-2.0048281E-2,-3.329749E-2,-4.7226588E-4,-1.1971596E-2,-9.0992794E-4,5.181902E-3,4.1945907E-3,1.27073815E-2,-5.0962237E-3,3.344545E-3,1.1613483E-2,3.99133E-3,8.124161E-3,-1.8657715E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,35,37,39,41,43,-1,45,-1,-1,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,-1,-1,-1,-1,79,81,83,-1,85,87,89,91,-1,-1,93,95,97,-1,99,101,103,-1,-1,105,107,109,-1,-1,111,113,115,117,-1,-1,119,121,123,-1,125,127,129,131,-1,-1,133,135,137,139,141,-1,-1,143,145,147,149,151,-1,-1,-1,-1,153,-1,155,157,159,-1,-1,-1,161,-1,-1,-1,-1,163,165,-1,167,-1,169,-1,171,173,-1,-1,175,-1,-1,-1,177,179,-1,-1,181,183,185,-1,187,-1,-1,189,191,193,195,197,-1,-1,-1,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,203,-1,205,-1,-1,-1,207,209,-1,-1,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,217,-1,-1,-1,219,-1,-1,-1,-1,-1,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5101564E2,7.622995E1,4.108082E1,8.402031E0,2.0662775E1,1.5996719E1,9.946165E-1,6.51741E-1,7.1595535E0,8.763061E0,5.5570397E0,5.8001347E0,1.2434647E1,0E0,0E0,3.0021667E-2,0E0,1.3952293E0,8.002494E0,4.298765E0,1.2653905E0,2.1015854E0,2.4369311E0,2.518735E0,1.3476181E-1,0E0,7.590439E0,0E0,0E0,4.7367477E-1,3.4502983E-2,0E0,1.5842652E0,3.807373E-1,2.0791326E0,6.6727525E-1,9.815848E-2,2.2419546E0,2.627181E0,9.038132E-1,4.0952802E-1,2.6760477E-1,1.4178333E0,3.6981392E-1,0E0,1.9244442E0,2.475891E0,0E0,0E0,0E0,0E0,2.6675224E-2,7.1423984E-1,3.8747025E-1,0E0,1.1711794E0,1.1613255E0,1.1489779E0,4.8287967E-1,0E0,0E0,1.1659727E0,1.2140188E0,1.0867891E0,0E0,4.775105E-1,4.304738E-1,1.1663926E-1,0E0,0E0,6.6127405E-2,1.356319E0,1.0168295E0,0E0,0E0,1.1339455E0,8.554605E-1,1.8776474E0,3.9338207E-1,0E0,0E0,1.4184737E-1,2.2009441E-1,1.4748955E-1,0E0,3.4856838E-1,6.2135935E-1,9.266107E-2,6.763859E-1,0E0,0E0,1.0383153E-1,2.8753528E-1,1.9785881E-2,3.7716627E-1,3.4755725E-1,0E0,0E0,3.9562225E-1,5.5907613E-1,4.0410244E-1,1.520252E-1,1.4487875E-1,0E0,0E0,0E0,0E0,5.14987E-1,0E0,4.4307423E-1,8.564776E-2,3.914795E-1,0E0,0E0,0E0,1.9971466E-1,0E0,0E0,0E0,0E0,1.9156551E-1,3.413704E-2,0E0,1.3182831E-1,0E0,2.0560172E-1,0E0,1.5495229E-1,1.6780338E-1,0E0,0E0,5.235777E-1,0E0,0E0,0E0,4.2620093E-2,3.9615512E-1,0E0,0E0,6.27986E-2,2.1381283E-1,9.2746355E-2,0E0,2.889619E-1,0E0,0E0,3.0570364E-1,1.9916046E-1,3.95755E-1,1.6301173E-1,7.733794E-2,0E0,0E0,0E0,2.9416418E-1,5.9475803E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2915039E-1,0E0,2.1072836E-1,0E0,0E0,0E0,5.492708E-2,6.3435555E-2,0E0,0E0,0E0,1.6607939E-1,1.915888E-1,0E0,0E0,1.56322E-1,0E0,0E0,0E0,0E0,0E0,1.1441429E-1,0E0,0E0,0E0,2.0790786E-1,0E0,0E0,0E0,0E0,0E0,1.3456574E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,51,51,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,63,63,65,65,66,66,67,67,70,70,71,71,72,72,75,75,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,95,95,98,98,99,99,100,100,101,101,102,102,107,107,109,109,110,110,111,111,115,115,120,120,121,121,123,123,125,125,127,127,128,128,131,131,135,135,136,136,139,139,140,140,141,141,143,143,146,146,147,147,148,148,149,149,150,150,154,154,155,155,168,168,170,170,174,174,175,175,179,179,180,180,183,183,189,189,193,193,199,199],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,36,38,40,42,44,-1,46,-1,-1,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,-1,-1,-1,-1,80,82,84,-1,86,88,90,92,-1,-1,94,96,98,-1,100,102,104,-1,-1,106,108,110,-1,-1,112,114,116,118,-1,-1,120,122,124,-1,126,128,130,132,-1,-1,134,136,138,140,142,-1,-1,144,146,148,150,152,-1,-1,-1,-1,154,-1,156,158,160,-1,-1,-1,162,-1,-1,-1,-1,164,166,-1,168,-1,170,-1,172,174,-1,-1,176,-1,-1,-1,178,180,-1,-1,182,184,186,-1,188,-1,-1,190,192,194,196,198,-1,-1,-1,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,204,-1,206,-1,-1,-1,208,210,-1,-1,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,218,-1,-1,-1,220,-1,-1,-1,-1,-1,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.9661028E7,9.9E1,9.31E2,1.1598715E3,1.665909E3,1.4912975E-1,6.76E2,1.5859042E6,3.875E0,2.9251662E6,1.059448E7,-5.0334964E-2,-1.9184373E-2,8.875E0,-1.9581432E-2,4.6463413E0,2.74614E5,4.48E2,7.4711205E6,4.265829E2,9.932432E0,1.4828649E9,2.6662E4,8.6895436E-2,1.7587205E6,-4.7518518E-2,-2.757553E-2,7.997723E6,6.8E1,3.4099087E-2,9.876542E-2,2E0,2.835E3,5.8899284E1,3.52E2,5.9192824E0,4.455844E1,1.4603988E6,8.4797815E2,2.6463525E8,4.087E3,1.2003246E6,2.0080749E-2,2.3876712E7,2.1257812E7,-3.3124804E-2,-1.3965118E-2,-6.606181E-2,-3.2697614E-2,4.9094446E2,2.026453E6,2.0145264E2,-3.7304517E-2,1E0,1.5714285E0,3.5714287E-1,3.4187552E6,2.2889538E-2,8.889135E-3,1.407037E2,2.1347518E0,1.089E3,-9.34718E-3,5E0,1.2585366E1,1.079E3,-2.7960172E-2,-1.0369004E-2,7.7E1,1.0494994E7,5.5454545E0,2.715325E-2,4.549749E-2,4.233978E0,2.9925186E-2,1E0,1.8545505E2,-3.998909E-2,-2.1338705E-2,6.782E4,3.401282E1,6.920598E7,-1.2486093E-2,1.3366154E6,2.118835E6,2.33E2,1.559733E6,8.628427E-3,-1.9532233E-2,1.9751773E0,3.355864E0,1.8992E4,1E0,3.9892856E2,-1.824982E-2,4.6515972E-3,6.965855E9,9.1E1,6.228074E5,7.7E1,2.239E3,-1.3491115E-2,-2.7342E-3,-1.4498445E-3,5.8407313E-3,1.3902439E0,-9.9598905E-3,6.797565E1,2.427869E8,1.4603988E6,1.0149398E-2,-7.78564E-3,1.6972052E-2,1.6063418E7,2.2384929E-2,3.457816E-2,1.2448564E-2,-1.0324918E-2,6.9740294E5,2.9711267E2,-1.1875937E-2,1.9E1,-1.1855732E-2,3.53E2,1.0953399E-2,1.3529412E0,1E0,3.5364935E-4,-8.473294E-3,1.2739792E1,-4.3281703E-3,-1.1312302E-2,-1.5357796E-3,9.66E2,7.3809524E0,3.5201266E-2,1.8253269E-2,1.8181819E0,8.684966E-1,1E0,1.4264061E-2,1.1394967E7,3.320946E-2,8.841761E-3,2.5E0,3.1095755E0,1.03339244E6,5.80418E7,7.6825094E0,1.0226706E-2,2.4505768E-2,2.6079942E-2,1.9505818E8,3.9149782E6,1.1330703E-2,4.0160073E-3,1.3949478E-2,3.624573E-2,1.4810829E-2,6.3704684E-2,4.4903275E-2,-2.6027502E-2,-9.909527E-3,-3.1908199E-3,1.9831837E-3,-1.4045271E-2,4.03E2,4.5774165E-3,4.0481758E3,-5.9024906E-3,-1.7932279E-2,-9.371145E-3,2.0671213E2,8E0,-1.116317E-2,5.031539E-3,1.2207902E-2,5.4E0,2.9547384E9,-1.1300913E-3,5.477288E-3,1.5484655E6,1.9904312E-2,3.4397193E-3,-5.4847435E-3,2.4269028E-2,1.0960829E-2,9.557399E6,-1.3799966E-2,-9.214199E-3,3.1130412E-3,1.48E1,-5.6327586E-3,1.2809288E-3,1.3326791E-2,-3.5169765E-3,4.6169646E-3,1.4172E4,1.6919017E-2,1.1915705E-2,3.5820182E-2,-2.899007E-2,-1.3642825E-2,-8.5165E-3,-4.8935594E-4,-3.4403258E-3,3.116593E-3,-2.0048281E-2,-3.329749E-2,-4.7226588E-4,-1.1971596E-2,-9.0992794E-4,5.181902E-3,4.1945907E-3,1.27073815E-2,-5.0962237E-3,3.344545E-3,1.1613483E-2,3.99133E-3,8.124161E-3,-1.8657715E-3],"split_indices":[2,43,60,44,2,67,4,53,2,43,69,43,12,0,0,68,0,68,5,2,43,67,69,46,10,0,43,0,0,9,0,0,53,6,44,62,0,71,73,43,4,7,2,66,0,60,66,0,0,0,0,4,9,70,0,26,69,68,60,0,0,67,69,2,0,69,71,10,0,0,0,62,69,0,0,68,72,6,73,0,0,9,73,7,0,62,9,0,9,0,0,71,73,9,8,67,0,0,46,10,60,0,2,0,0,0,0,68,0,73,7,43,0,0,0,62,0,0,0,0,62,4,0,67,0,44,0,68,27,0,0,71,0,0,0,12,73,0,0,71,57,6,0,60,0,0,68,53,43,7,71,0,0,0,5,60,0,0,0,0,0,0,0,0,0,0,0,0,2,0,48,0,0,0,4,8,0,0,0,73,46,0,0,43,0,0,0,0,0,60,0,0,0,73,0,0,0,0,0,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.95E2,7.4E2,2.55E2,2.12E2,5.28E2,2.4E2,1.5E1,8.4E1,1.28E2,2.77E2,2.51E2,1.11E2,1.29E2,1E1,5E0,8E1,4E0,7.8E1,5E1,1.8E2,9.7E1,1.33E2,1.18E2,7.9E1,3.2E1,1.4E1,1.15E2,7.5E1,5E0,7E1,8E0,6E0,4.4E1,6.2E1,1.18E2,8.9E1,8E0,7.4E1,5.9E1,1.04E2,1.4E1,1.4E1,6.5E1,2.8E1,4E0,5E1,6.5E1,6.5E1,5E0,4E0,4E0,1.1E1,3.3E1,5.4E1,8E0,7.8E1,4E1,1.2E1,7.7E1,4E0,4E0,5.5E1,1.9E1,5.4E1,5E0,7.6E1,2.8E1,1E1,4E0,5E0,9E0,2.9E1,3.6E1,1E1,1.8E1,3.9E1,1.1E1,5.2E1,1.3E1,6E0,5E0,1.9E1,1.4E1,4.5E1,9E0,3.9E1,3.9E1,9E0,3.1E1,4E0,8E0,1E1,6.7E1,9E0,4.6E1,1.2E1,7E0,6E0,4.8E1,2.8E1,4.8E1,1.8E1,1E1,6E0,4E0,5E0,4E0,2.4E1,5E0,2.6E1,1E1,3.3E1,6E0,4E0,7E0,4.7E1,5E0,9E0,4E0,6E0,1.3E1,9E0,5E0,4.1E1,4E0,3.5E1,4E0,2.4E1,1.5E1,5E0,4E0,2.7E1,4E0,6E0,4E0,1.1E1,5.6E1,4E0,5E0,1.3E1,3.3E1,8E0,4E0,3.1E1,1.7E1,5E0,2.3E1,1E1,3.8E1,1E1,8E0,6E0,4E0,6E0,1.8E1,2.2E1,4E0,4E0,6E0,2.9E1,4E0,3.2E1,1.5E1,9E0,4E0,4E0,5E0,5E0,3.6E1,7E0,2.8E1,4E0,2E1,6E0,9E0,2.1E1,6E0,6E0,5E0,1E1,4.6E1,4E0,9E0,2.5E1,8E0,4E0,4E0,2.4E1,7E0,1.5E1,8E0,6E0,4E0,3.3E1,5E0,4E0,6E0,4E0,4E0,1.1E1,7E0,4E0,1.8E1,3.2E1,4E0,1.1E1,1.7E1,4E0,5E0,8E0,1.3E1,5E0,5E0,1.7E1,2.9E1,8E0,1.7E1,1.1E1,4E0,1.4E1,1.9E1,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[1.3471586E-2,-2.2610085E-1,7.3868704E-1,-6.5098697E-1,-3.5431497E-2,8.4848815E-1,-1.5004842E-1,-8.9800495E-1,-4.8295447E-1,-2.2586998E-1,1.7531002E-1,4.6864507E-1,1.1211873E0,-8.2168055E-1,2.4668853E-1,-4.416024E-2,-6.1319447E-1,1.7007243E-2,-5.28793E-1,-2.930738E-1,5.057889E-2,3.0750713E-1,3.9650675E-2,1.7390183E-1,6.044502E-1,1.7208714E0,9.987359E-1,-4.5871697E-2,-1.9293519E-2,3.8010448E-1,-9.373652E-3,-3.344311E-2,-1.8930763E-2,-5.636617E-1,9.437213E-3,-2.7136397E-1,-5.556443E-2,-1.9204797E-2,3.378035E-1,1.8561703E-1,4.7336155E-1,7.310219E-2,-4.070584E-1,1.6812421E-2,4.7227135E-1,7.144502E-1,4.9912896E-2,5.6426976E-2,9.535076E-2,6.8532443E-1,1.126565E0,7.007599E-3,4.493776E-1,-6.2934786E-1,-2.4185112E-1,-5.183244E-1,-1.9883876E-1,6.202525E-2,-3.1783548E-1,9.521465E-3,2.1003593E-2,1.1989366E-1,2.4307655E-2,5.542791E-1,-6.889406E-3,-3.158213E-2,1.3950029E-1,-4.08395E-3,-2.976633E-2,-8.820557E-3,1.066468E-1,2.6161164E-2,1.1375045E-2,3.7389975E-2,3.7879828E-1,-5.8667203E-3,1.4920981E-2,4.2009376E-2,5.289257E-1,2.4025898E-2,1.1790373E0,1.3040129E-2,2.7417619E-2,-6.601182E-1,-3.2561103E-1,-4.49822E-3,-1.6475413E-2,-4.1487828E-1,-2.7016576E-2,-1.3602722E-1,-3.4765822E-1,-1.6290968E-2,1.7797509E-1,-2.3467105E-2,-4.9169087E-3,-2.3018967E-2,2.1575898E-1,4.4199905E-1,3.565774E-2,6.239491E-2,-2.3622614E-1,-1.15486704E-1,1.9155489E-1,1.4931753E-1,-7.8964967E-4,7.838179E-3,2.1996956E-2,9.713744E-3,6.057634E-1,1.2241651E0,2.7329534E-2,-3.5721904E-1,-6.9430983E-1,-2.3551518E-3,-2.376916E-2,-2.0790437E-2,-1.1102127E-2,-9.744354E-2,-3.903768E-1,-5.040833E-1,-2.9203922E-1,-8.791621E-2,1.0489454E-1,1.1815381E-2,3.6013946E-3,9.236729E-2,-1.690374E-2,2.5943175E-1,-9.6785446E-4,8.035426E-3,4.7194386E-1,1.3761136E-1,-8.0701485E-2,-1.3299356E-1,-1.7043263E-2,-1.3238124E-2,-7.622262E-3,2.6666063E-1,6.465602E-2,2.151845E-3,1.1286171E-2,1.5127124E-2,3.3154383E-2,1.3348699E0,4.6511665E-2,-2.0508487E-2,-6.794053E-3,-7.2029847E-1,-1.606041E-2,-4.970796E-2,-1.6955613E-1,-5.098529E-1,-4.9097845E-3,-2.9359356E-2,-1.4757413E-2,-1.8703546E-2,-1.9996542E-1,2.4301666E-3,-1.4379577E-1,8.326625E-3,3.9622452E-4,1.8074419E-1,-5.091463E-3,2.1009976E-2,2.0830798E-1,1.1900312E-2,4.9501446E-1,2.2191374E-1,-1.7397186E-3,1.122717E-3,-8.7665E-3,-1.00850025E-2,-1.8634964E-4,1.6903182E-3,-2.3241725E-3,3.962152E-1,1.8344365E-1,1.5655515E-1,-4.0289376E-2,3.6649484E-2,6.500713E-2,-3.3923816E-2,-1.964705E-2,-6.456711E-4,-1.4390323E-2,-3.4553998E-3,-1.2255195E-2,-1.00895045E-2,-2.9807769E-2,-9.6507213E-4,-1.0844586E-2,-8.552617E-3,-2.641438E-3,4.1510696E-3,1.0688406E-2,4.239972E-3,1.0899175E-2,1.2333621E-2,2.4224138E-2,1.2011358E-2,4.3316246E-3,1.0150402E-2,2.1707145E-2,1.0621646E-2,-2.236304E-3,1.9091156E-3,9.386613E-3,2.0133564E-3,-7.6885647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,-1,81,83,85,87,89,91,93,-1,-1,95,-1,97,-1,99,101,-1,-1,-1,103,-1,-1,-1,105,-1,-1,-1,107,-1,109,-1,-1,111,113,-1,-1,115,-1,117,119,121,123,-1,-1,125,127,129,-1,131,133,135,137,139,-1,-1,-1,-1,141,143,-1,145,147,-1,-1,-1,-1,149,151,153,155,157,159,-1,-1,161,-1,163,-1,-1,165,167,169,171,-1,-1,173,175,177,-1,-1,-1,-1,179,-1,-1,-1,181,-1,183,185,187,-1,-1,-1,-1,189,-1,191,-1,-1,193,-1,-1,195,-1,197,199,-1,-1,-1,-1,-1,-1,-1,201,203,205,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.791209E2,6.2761826E1,2.510884E1,9.528E0,2.1551044E1,2.3091461E1,7.930076E0,9.959717E-1,5.851471E0,5.2539377E0,4.560919E0,3.7775688E0,7.97435E0,3.4642363E-1,1.216778E0,0E0,1.3123655E-1,0E0,3.6425667E0,4.156475E0,1.1289285E0,2.5200014E0,1.9275302E0,1.491922E0,3.9902802E0,1.0773582E0,3.7149963E0,0E0,0E0,1.6918921E-1,0E0,0E0,0E0,2.6380768E0,0E0,3.891058E0,0E0,1.135572E0,7.556093E-2,1.6378746E0,2.802637E0,8.315556E-1,6.234549E-1,4.3092486E-1,9.4311E-2,1.4448681E0,6.206651E-1,0E0,0E0,7.45142E-1,1.8692017E0,0E0,9.7367525E-2,8.037758E-1,3.63248E-1,1.01145744E-1,1.5791879E0,3.3888727E-1,3.4786725E-1,0E0,0E0,8.9549017E-1,0E0,8.3132267E-1,0E0,9.1863567E-1,9.9106896E-1,0E0,0E0,0E0,8.703281E-2,0E0,0E0,0E0,2.0421124E-1,0E0,0E0,0E0,4.3580246E-1,0E0,1.1581421E0,0E0,0E0,7.669296E-1,5.9625506E-1,0E0,0E0,3.294611E-2,0E0,1.1788642E0,2.9649878E-1,2.0886058E-1,9.674963E-2,0E0,0E0,1.1019844E0,4.086063E-1,1.9084072E-1,0E0,3.662911E-1,1.5639597E-1,2.3768076E-1,5.641396E-1,1.0571629E-1,0E0,0E0,0E0,0E0,3.255248E-1,2.2045898E-1,0E0,1.5335834E-1,5.334358E-1,0E0,0E0,0E0,0E0,3.6678624E-1,4.8891044E-1,8.81927E-2,3.345101E-1,1.2534547E-1,6.376294E-2,0E0,0E0,3.958801E-1,0E0,2.3482108E-1,0E0,0E0,2.0850182E-2,3.3341855E-1,1.42023E-1,1.0912365E-1,0E0,0E0,1.8824793E-2,3.3700037E-1,2.3777288E-1,0E0,0E0,0E0,0E0,4.4866943E-1,0E0,0E0,0E0,1.8115997E-2,0E0,6.22355E-1,3.7017024E-1,2.930348E-1,0E0,0E0,0E0,0E0,1.4453489E-1,0E0,2.8795391E-2,0E0,0E0,4.821509E-2,0E0,0E0,6.843603E-2,0E0,3.391266E-2,4.8820138E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.8751326E-2,2.804619E-1,6.144482E-2,1.3419731E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,63,63,65,65,66,66,70,70,74,74,78,78,80,80,83,83,84,84,87,87,89,89,90,90,91,91,92,92,95,95,96,96,97,97,99,99,100,100,101,101,102,102,103,103,108,108,109,109,111,111,112,112,117,117,118,118,119,119,120,120,121,121,122,122,125,125,127,127,130,130,131,131,132,132,133,133,136,136,137,137,138,138,143,143,147,147,149,149,150,150,151,151,156,156,158,158,161,161,164,164,166,166,167,167,175,175,176,176,177,177,178,178],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,-1,82,84,86,88,90,92,94,-1,-1,96,-1,98,-1,100,102,-1,-1,-1,104,-1,-1,-1,106,-1,-1,-1,108,-1,110,-1,-1,112,114,-1,-1,116,-1,118,120,122,124,-1,-1,126,128,130,-1,132,134,136,138,140,-1,-1,-1,-1,142,144,-1,146,148,-1,-1,-1,-1,150,152,154,156,158,160,-1,-1,162,-1,164,-1,-1,166,168,170,172,-1,-1,174,176,178,-1,-1,-1,-1,180,-1,-1,-1,182,-1,184,186,188,-1,-1,-1,-1,190,-1,192,-1,-1,194,-1,-1,196,-1,198,200,-1,-1,-1,-1,-1,-1,-1,202,204,206,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.856934E7,1.16E2,9.31E2,1.1459359E3,1.3655363E2,7.6588124E-2,6E0,2.331083E6,3.6666667E0,5.486433E5,1.059448E7,1.665909E3,3.0070068E7,-4.416024E-2,2.0081382E-7,1.7007243E-2,8.909296E4,1.1290322E1,6.9307615E6,4.265829E2,1.2294118E1,4.032683E6,1E0,2.35184E5,1.7587205E6,-4.5871697E-2,-1.9293519E-2,1.4595416E3,-9.373652E-3,-3.344311E-2,-1.8930763E-2,7.997723E6,9.437213E-3,3.89E2,-5.556443E-2,8E0,1.1311654E2,1.9090909E0,2.48334E5,1.47E3,1.4787234E1,1.5204346E8,1.8333334E0,2E0,1.038946E6,5.6426976E-2,9.535076E-2,2.5817454E8,1E0,7.007599E-3,7.802E-2,3.3238492E2,1.8E0,3.1E1,2.2970297E0,1.821E3,1.5933333E2,9.521465E-3,2.1003593E-2,1.212945E6,2.4307655E-2,1.2774637E10,-6.889406E-3,4.911E3,3.2892792E2,-4.08395E-3,-2.976633E-2,-8.820557E-3,2.8E1,2.6161164E-2,1.1375045E-2,3.7389975E-2,9.0505896E2,-5.8667203E-3,1.4920981E-2,4.2009376E-2,6.319223E8,2.4025898E-2,1E0,1.3040129E-2,2.7417619E-2,9.935484E-1,1.3209776E6,-4.49822E-3,-1.6475413E-2,5.4E0,-2.7016576E-2,8.931E3,1E0,5.95E2,7.597862E6,-2.3467105E-2,-4.9169087E-3,4.68418E5,3.2857144E0,1.124E3,3.565774E-2,5.44E2,1.5E1,2.686063E6,3.093458E0,1.409E3,-7.8964967E-4,7.838179E-3,2.1996956E-2,9.713744E-3,4.3948618E8,1.7103828E7,2.7329534E-2,6.848509E7,5.0953402E4,-2.3551518E-3,-2.376916E-2,-2.0790437E-2,-1.1102127E-2,1E0,1.6E1,2.57E1,6.47E2,3.3817584E7,1.4285715E0,1.1815381E-2,3.6013946E-3,1.8E1,-1.690374E-2,1.2825651E0,-9.6785446E-4,8.035426E-3,4.797342E2,2.6647997E9,1.0674361E8,1E0,-1.7043263E-2,-1.3238124E-2,2.8655008E2,2.11429E5,1.5222016E8,2.151845E-3,1.1286171E-2,1.5127124E-2,3.3154383E-2,4.454E3,4.6511665E-2,-2.0508487E-2,-6.794053E-3,1.3091388E0,-1.606041E-2,1.1203416E1,1.6153846E0,9.2789966E-1,-4.9097845E-3,-2.9359356E-2,-1.4757413E-2,-1.8703546E-2,3.449123E0,2.4301666E-3,2.3112903E2,8.326625E-3,3.9622452E-4,4.745397E6,-5.091463E-3,2.1009976E-2,6.479833E5,1.1900312E-2,5.1604336E7,1.4839433E5,-1.7397186E-3,1.122717E-3,-8.7665E-3,-1.00850025E-2,-1.8634964E-4,1.6903182E-3,-2.3241725E-3,5.185192E2,1.26E2,6E0,4.9E1,3.6649484E-2,6.500713E-2,-3.3923816E-2,-1.964705E-2,-6.456711E-4,-1.4390323E-2,-3.4553998E-3,-1.2255195E-2,-1.00895045E-2,-2.9807769E-2,-9.6507213E-4,-1.0844586E-2,-8.552617E-3,-2.641438E-3,4.1510696E-3,1.0688406E-2,4.239972E-3,1.0899175E-2,1.2333621E-2,2.4224138E-2,1.2011358E-2,4.3316246E-3,1.0150402E-2,2.1707145E-2,1.0621646E-2,-2.236304E-3,1.9091156E-3,9.386613E-3,2.0133564E-3,-7.6885647E-3],"split_indices":[2,43,60,44,2,67,71,57,3,43,69,43,12,4,58,0,52,0,48,69,43,67,68,60,6,1,43,0,0,4,0,0,0,9,0,2,0,8,4,68,1,2,71,7,68,32,9,0,0,12,27,0,72,67,68,10,68,44,4,0,0,9,0,46,0,44,4,0,0,0,71,0,0,0,4,0,0,0,12,0,6,0,0,68,60,0,0,73,0,44,29,2,60,0,0,9,69,2,0,10,3,12,68,10,0,0,0,0,7,62,0,7,48,0,0,0,0,26,3,61,2,7,68,0,0,3,0,69,0,0,4,46,7,26,0,0,4,9,5,0,0,0,0,2,0,0,0,53,0,71,68,68,0,0,0,0,73,0,67,0,0,60,0,0,43,0,7,48,0,0,0,0,0,0,0,67,8,8,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.74E2,2.55E2,2.39E2,5.35E2,2.27E2,2.8E1,9.5E1,1.44E2,2.81E2,2.54E2,9.6E1,1.31E2,1E1,1.8E1,7.7E1,1.8E1,7E0,1.37E2,2.26E2,5.5E1,1.28E2,1.26E2,3.1E1,6.5E1,2E1,1.11E2,6E0,4E0,1.4E1,4E0,1E1,8E0,1.31E2,6E0,2.22E2,4E0,4.5E1,1E1,7.5E1,5.3E1,1.18E2,8E0,2.1E1,1E1,5.4E1,1.1E1,1E1,1E1,3.4E1,7.7E1,4E0,1E1,1.08E2,2.3E1,4.9E1,1.73E2,3.6E1,9E0,6E0,4E0,6.4E1,1.1E1,4.7E1,6E0,4.6E1,7.2E1,4E0,4E0,6E0,1.5E1,6E0,4E0,4.1E1,1.3E1,7E0,4E0,1.2E1,2.2E1,7E0,7E1,6E0,4E0,9.7E1,1.1E1,1.1E1,1.2E1,2.2E1,2.7E1,1.23E2,5E1,2.2E1,1.4E1,4E0,5E0,2.6E1,3.8E1,3.3E1,1.4E1,3.2E1,1.4E1,1.2E1,6E1,1.1E1,4E0,5E0,8E0,5E0,1.7E1,6.4E1,6E0,1.1E1,8.6E1,5E0,6E0,1.7E1,5E0,1.08E2,1.5E1,1.1E1,3.9E1,1.4E1,8E0,7E0,7E0,2E1,6E0,3.2E1,6E0,4E0,2.9E1,2.1E1,1.1E1,9E0,5E0,4E0,8E0,3.7E1,2.3E1,6E0,5E0,6E0,1.1E1,3.9E1,2.5E1,7E0,4E0,7.9E1,7E0,6.6E1,4.2E1,1E1,5E0,5E0,6E0,1.6E1,2.3E1,4E0,1E1,4E0,4E0,1.4E1,6E0,5E0,2.7E1,4E0,2.5E1,1.4E1,7E0,6E0,5E0,5E0,4E0,4E0,4E0,1.3E1,2.4E1,1.2E1,1.1E1,6E0,3.3E1,7.4E1,5E0,5.9E1,7E0,2.2E1,2E1,4E0,6E0,4E0,1.9E1,6E0,4E0,6E0,8E0,6E0,2.1E1,4E0,2.1E1,1E1,4E0,5E0,8E0,2E1,4E0,4E0,8E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"209","size_leaf_vector":"1"}},{"base_weights":[-5.7131415E-3,-2.1280274E-1,6.4551926E-1,-6.5236235E-1,-1.1418982E-2,7.558895E-1,-1.8537945E-1,-1.0013372E0,-5.050563E-1,-1.9263324E-1,1.7599282E-1,5.0836426E-1,1.0645207E0,-6.724832E-1,1.7671658E-1,-1.3869389E0,-8.3693385E-1,-8.493203E-1,-4.0978426E-1,-2.9323986E-1,1.7531294E-5,1.0551816E-1,4.7692347E-1,3.4867108E-1,7.809663E-1,1.2617676E0,6.7906153E-1,-4.0480398E-2,-1.3340322E-2,3.5177413E-1,-1.071945E-2,-3.424277E-2,-7.1736574E-2,-9.073491E-1,-5.2073604E-1,-9.080936E-1,-1.3469453E-2,-5.8451164E-1,-1.629588E-1,-2.6605478E-1,-4.7920655E-2,-4.2612037E-1,5.5164583E-2,1.2563364E-1,-2.8618587E-2,1.8970884E-1,5.530684E-1,4.8565168E-2,5.006389E-1,8.5673195E-1,4.301134E-1,8.1547685E-2,1.1213405E0,5.0695276E-1,1.0084851E0,8.504519E-3,2.003051E-2,-4.3692246E-2,-2.3995148E-2,-9.402905E-3,-3.3181816E-2,-4.461401E-2,-2.2543114E-2,-3.4037873E-1,-6.393693E-1,3.5912238E-2,-3.044859E-1,-4.7926924E-1,-1.7449664E-1,4.4251833E-4,-3.6534168E-2,-1.4875851E-2,1.5091062E-1,1.6019753E-1,-1.6385362E-1,4.930058E-3,1.1777003E-2,9.275461E-3,6.2385917E-1,1.4891837E-2,-7.70621E-2,6.118049E-1,2.0568298E-2,4.150985E-2,1.9314384E-2,2.6112651E-2,8.406481E-3,6.12446E-1,5.606765E-2,3.3181302E-2,3.4322318E-1,2.5779746E-2,5.4800406E-2,-2.5190907E-2,1.3960036E-3,-7.3375976E-1,-4.7934717E-1,-1.3277434E-1,-4.0765643E-1,-5.0291383E-1,-7.226789E-3,-2.4905093E-1,-1.7495241E-2,2.2120388E-2,-1.6507284E-1,2.0853803E-1,-5.4340563E-3,2.6271075E-1,6.8277985E-2,-3.2757118E-1,-1.2719233E-2,6.793986E-1,1.247167E-2,-1.2805148E-2,-9.710938E-3,6.8976617E-1,2.85556E-1,-8.303377E-3,1.0045475E-2,1.5552431E-2,3.4197014E-2,5.7263775E-3,4.0598503E-1,-1.6412728E-2,-3.5428498E-2,-8.322831E-3,-2.4187442E-2,-2.1534975E-1,5.061374E-3,-4.7666577E-1,-4.3011047E-3,-1.08891465E-2,-5.285037E-1,-3.1408364E-1,6.8513486E-3,2.5285916E-2,-1.3646984E-1,1.017859E-1,-5.130827E-2,-1.283611E-2,1.5091422E-4,3.032222E-1,1.1829586E-1,2.9116493E-1,-1.2637876E-2,-1.8512274E-1,1.1118296E-1,-7.440535E-3,-2.0304175E-2,7.4566975E-3,-6.5473993E-3,3.322434E-2,1.666092E-2,-5.1218826E-2,4.243391E-3,3.4646105E-2,4.7037703E-1,7.2624353E-3,1.7669909E-2,2.1585908E-2,1.0477241E-2,-1.236151E-1,-1.606056E-2,-5.556968E-1,-8.443694E-3,-2.5964834E-2,-1.5681634E-2,-2.0712969E-1,-4.3908915E-1,1.4772722E-1,-8.671995E-3,1.3502525E-1,-2.5744956E-2,-1.6614421E-3,-8.720762E-3,-3.0788453E-3,6.840154E-3,-1.1333294E-1,5.087547E-2,1.740212E-2,4.419779E-3,3.644565E-2,1.2117101E-2,1.8387146E-1,4.721319E-1,-1.3897703E-4,-2.5107273E-1,2.8319046E-1,7.43159E-2,1.503767E-3,-4.4892463E-3,1.1996412E-2,2.6221275E-2,-2.2326733E-3,-8.237311E-3,-2.762764E-2,-1.4282202E-2,1.1469987E-5,-1.1021158E-2,-2.2399798E-2,-5.234065E-3,8.955E-3,2.4776862E-3,2.861803E-3,8.011894E-3,-3.1723324E-3,4.627169E-3,2.5558582E-4,-8.315084E-3,6.157977E-3,-1.9272601E-3,-2.8932558E-3,4.3339166E-3,-2.726627E-3,1.3586501E-2,1.6983192E-2,3.0254316E-2,-6.7836526E-3,-1.448177E-2,8.466768E-3,1.7149435E-2,2.0640255E-3,1.604899E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,63,65,67,-1,69,71,73,-1,75,77,79,81,83,85,-1,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,93,95,-1,97,99,101,-1,-1,103,105,107,109,-1,-1,-1,111,-1,113,115,117,-1,-1,-1,-1,119,-1,-1,121,-1,-1,-1,-1,123,125,127,129,131,-1,133,135,137,139,141,-1,143,145,147,149,151,-1,-1,153,155,157,-1,-1,-1,-1,-1,159,-1,-1,-1,-1,161,-1,163,-1,-1,165,167,169,171,173,175,177,-1,-1,179,181,183,-1,185,187,-1,-1,-1,-1,-1,-1,189,-1,-1,191,-1,-1,-1,-1,193,-1,195,-1,-1,-1,197,199,201,-1,203,205,-1,-1,-1,-1,207,209,-1,-1,211,-1,213,215,-1,217,219,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.391816E2,6.930384E1,2.2975037E1,1.214016E1,1.830529E1,1.6265518E1,5.410189E0,3.4036102E0,5.3917465E0,5.3106565E0,5.569791E0,5.1257553E0,6.5181503E0,7.63299E-1,1.3695884E0,9.9077606E-1,7.075691E-1,9.481716E-1,5.8994236E0,3.3647804E0,2.2560303E0,3.2278268E0,9.88575E-1,3.6242008E0,8.2772446E-1,2.4739532E0,1.4731293E0,0E0,0E0,9.849846E-2,0E0,0E0,0E0,1.3433838E-1,6.0559464E-1,3.273468E-1,0E0,8.671684E-1,8.088253E0,3.332635E0,0E0,1.7553664E0,5.722629E-1,2.1374733E0,0E0,3.7742227E-2,8.4738255E-1,9.8786324E-1,2.815895E0,2.8396225E-1,2.3242116E-1,0E0,1.6984177E0,6.1827564E-1,2.18606E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3073114E0,6.3006973E-1,0E0,8.6708355E-1,3.1826973E-1,1.447079E0,0E0,0E0,2.8148764E-1,5.8162445E-1,1.7630029E0,5.649306E-1,0E0,0E0,0E0,4.4557285E-1,0E0,2.66863E-1,8.967304E-1,4.7284296E-1,0E0,0E0,0E0,0E0,1.0319781E-1,0E0,0E0,1.698699E-1,0E0,0E0,0E0,0E0,2.1626472E-1,2.62753E-1,4.4381347E-1,6.5025806E-1,2.0365429E-1,0E0,1.4000812E0,2.121355E-1,2.4506144E-1,1.9074523E-1,2.204653E-1,0E0,1.4376955E0,1.1186697E0,1.3422465E-1,3.1488374E-1,8.2829475E-2,0E0,0E0,7.215386E-2,9.60331E-2,6.42761E-2,0E0,0E0,0E0,0E0,0E0,2.5754333E-2,0E0,0E0,0E0,0E0,1.5119374E-1,0E0,5.023408E-1,0E0,0E0,8.1721306E-2,8.048301E-1,5.210902E-1,1.7707913E-1,5.1027894E-2,1.6573937E-1,1.4519995E-1,0E0,0E0,1.7740571E-1,1.9078872E-1,1.5635972E0,0E0,1.7940146E-1,5.306573E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.935592E-2,0E0,0E0,5.7418108E-2,0E0,0E0,0E0,0E0,3.6671773E-2,0E0,2.379036E-2,0E0,0E0,0E0,2.4772656E-1,3.923688E-1,3.7329063E-2,0E0,1.731725E-2,1.2576392E-1,0E0,0E0,0E0,0E0,1.119104E-1,7.424067E-2,0E0,0E0,7.407257E-2,0E0,1.4897128E0,3.513689E-1,0E0,2.5511563E-2,6.389427E-2,5.753132E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,63,63,64,64,66,66,67,67,68,68,71,71,72,72,73,73,74,74,78,78,80,80,81,81,82,82,87,87,90,90,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,107,107,108,108,109,109,110,110,111,111,114,114,115,115,116,116,122,122,127,127,129,129,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,145,145,146,146,153,153,156,156,161,161,163,163,167,167,168,168,169,169,171,171,172,172,177,177,178,178,181,181,183,183,184,184,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,64,66,68,-1,70,72,74,-1,76,78,80,82,84,86,-1,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,94,96,-1,98,100,102,-1,-1,104,106,108,110,-1,-1,-1,112,-1,114,116,118,-1,-1,-1,-1,120,-1,-1,122,-1,-1,-1,-1,124,126,128,130,132,-1,134,136,138,140,142,-1,144,146,148,150,152,-1,-1,154,156,158,-1,-1,-1,-1,-1,160,-1,-1,-1,-1,162,-1,164,-1,-1,166,168,170,172,174,176,178,-1,-1,180,182,184,-1,186,188,-1,-1,-1,-1,-1,-1,190,-1,-1,192,-1,-1,-1,-1,194,-1,196,-1,-1,-1,198,200,202,-1,204,206,-1,-1,-1,-1,208,210,-1,-1,212,-1,214,216,-1,218,220,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8988005E5,3.2856784E7,3.3817584E7,9.31E2,7.459E3,1.3655363E2,2E0,8E1,1.5859042E6,2.8530578E6,1.087521E3,7.153514E1,1.4916515E3,4.4910244E3,4.869229E7,1.7E1,9.643839E6,6.15E2,1.05039425E1,1.45064E5,1.2294118E1,4.8581E4,5.486433E5,2.6506329E1,7.9E1,3.8537518E2,-4.0480398E-2,-1.3340322E-2,1.4595416E3,-1.071945E-2,-3.424277E-2,-7.1736574E-2,2.568306E0,2.3E1,5.159652E-7,-1.3469453E-2,2.7317073E0,2.7E1,4.33E2,-4.7920655E-2,1.1015E4,8.216578E9,2.1129E4,-2.8618587E-2,6.5718125E6,4.745397E6,7.6937294E-1,2E0,1E0,6.439711E1,8.1547685E-2,1.8598528E6,8.710612E1,2.9622424E3,8.504519E-3,2.003051E-2,-4.3692246E-2,-2.3995148E-2,-9.402905E-3,-3.3181816E-2,-4.461401E-2,-2.2543114E-2,1E1,1.852364E6,3.5912238E-2,3.3E1,2.3043478E2,2.240836E6,4.4251833E-4,-3.6534168E-2,9.1E1,7.77E2,3.7105262E0,2.2503355E1,4.930058E-3,1.1777003E-2,9.275461E-3,2E0,1.4891837E-2,8.295515E7,2E0,8.399457E2,4.150985E-2,1.9314384E-2,2.6112651E-2,8.406481E-3,3.3412E5,5.606765E-2,3.3181302E-2,7.25E2,2.5779746E-2,5.4800406E-2,-2.5190907E-2,1.3960036E-3,2E0,2.2E1,2.152E3,1.363176E7,1.9E1,-7.226789E-3,3.541824E2,2.1578948E0,1.4067796E0,1.3902439E0,2.1111E4,-5.4340563E-3,2.8366232E0,2.7015875E2,2.956111E2,2.884188E2,2E1,1.247167E-2,-1.2805148E-2,7.792289E9,7.234179E6,1.2E1,-8.303377E-3,1.0045475E-2,1.5552431E-2,3.4197014E-2,5.7263775E-3,1.39442E5,-1.6412728E-2,-3.5428498E-2,-8.322831E-3,-2.4187442E-2,3.9157894E0,5.061374E-3,3.2874417E2,-4.3011047E-3,-1.08891465E-2,1.585814E6,3.6981132E0,3.9649122E0,3.4846212E5,1.3E1,7.117E3,1.0772152E1,-1.283611E-2,1.5091422E-4,1.1E1,2.3534782E2,7.6937294E-1,-1.2637876E-2,4.878788E0,1.26E2,-7.440535E-3,-2.0304175E-2,7.4566975E-3,-6.5473993E-3,3.322434E-2,1.666092E-2,1.1E1,4.243391E-3,3.4646105E-2,5.1792985E6,7.2624353E-3,1.7669909E-2,2.1585908E-2,1.0477241E-2,1E0,-1.606056E-2,1.266544E0,-8.443694E-3,-2.5964834E-2,-1.5681634E-2,1.2877E4,2.7E1,1.8578552E7,-8.671995E-3,1.4473684E-1,1.746E3,-1.6614421E-3,-8.720762E-3,-3.0788453E-3,6.840154E-3,1.2E1,1E0,1.740212E-2,4.419779E-3,1.3E1,1.2117101E-2,1.196966E6,1.935E3,-1.3897703E-4,1.0142858E1,6.7148806E5,3.3412E5,1.503767E-3,-4.4892463E-3,1.1996412E-2,2.6221275E-2,-2.2326733E-3,-8.237311E-3,-2.762764E-2,-1.4282202E-2,1.1469987E-5,-1.1021158E-2,-2.2399798E-2,-5.234065E-3,8.955E-3,2.4776862E-3,2.861803E-3,8.011894E-3,-3.1723324E-3,4.627169E-3,2.5558582E-4,-8.315084E-3,6.157977E-3,-1.9272601E-3,-2.8932558E-3,4.3339166E-3,-2.726627E-3,1.3586501E-2,1.6983192E-2,3.0254316E-2,-6.7836526E-3,-1.448177E-2,8.466768E-3,1.7149435E-2,2.0640255E-3,1.604899E-2],"split_indices":[2,43,60,7,2,2,71,32,44,43,43,67,71,4,4,46,3,62,2,69,7,68,1,43,71,0,71,0,0,4,0,0,0,68,3,52,0,69,0,2,0,9,46,44,0,60,60,53,6,6,73,0,43,73,67,0,0,0,0,0,0,0,0,3,9,0,8,4,9,0,0,0,2,69,71,0,0,0,32,0,7,32,67,0,0,0,0,1,0,0,0,0,0,0,0,32,10,2,9,67,0,67,68,68,68,9,0,53,4,4,67,3,0,0,46,62,3,0,0,0,0,0,44,0,0,0,0,71,0,67,0,0,9,71,69,43,3,1,73,0,0,8,4,53,0,73,10,0,0,0,0,0,0,3,0,0,60,0,0,0,0,26,0,53,0,0,0,9,3,5,0,71,44,0,0,0,0,3,26,0,0,3,0,9,2,0,73,43,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,7.82E2,2.48E2,2.45E2,5.37E2,2.19E2,2.9E1,7.1E1,1.74E2,2.73E2,2.64E2,1.23E2,9.6E1,1.2E1,1.7E1,1.9E1,5.2E1,3.6E1,1.38E2,1.79E2,9.4E1,2.15E2,4.9E1,7.9E1,4.4E1,6.2E1,3.4E1,7E0,5E0,1.2E1,5E0,5E0,1.4E1,4.1E1,1.1E1,3.2E1,4E0,8E1,5.8E1,1.74E2,5E0,1E1,8.4E1,2.1E2,5E0,1.1E1,3.8E1,2.7E1,5.2E1,3.5E1,9E0,1.1E1,5.1E1,2.4E1,1E1,5E0,7E0,3.6E1,5E0,5E0,6E0,2.7E1,5E0,1.6E1,6.4E1,7E0,5.1E1,5.1E1,1.23E2,5E0,5E0,4.9E1,3.5E1,1.88E2,2.2E1,6E0,5E0,7E0,3.1E1,8E0,1.9E1,4.2E1,1E1,3.1E1,4E0,5E0,4E0,9E0,4.2E1,9E0,1.5E1,4E0,6E0,1E1,6E0,3.8E1,2.6E1,2E1,3.1E1,4.7E1,4E0,8.3E1,4E1,4E1,9E0,2.9E1,6E0,8.8E1,1E2,1E1,1.2E1,2.6E1,5E0,4E0,1.5E1,3.3E1,9E0,5E0,5E0,4E0,5E0,4E0,1.1E1,4E0,3.4E1,4E0,2.2E1,1.5E1,5E0,2.5E1,6E0,5E0,4.2E1,6.6E1,1.7E1,3E1,1E1,1.9E1,2.1E1,5E0,4E0,1.3E1,1.6E1,8.4E1,4E0,1.4E1,8.6E1,5E0,5E0,5E0,7E0,2.2E1,4E0,1.1E1,4E0,2.4E1,9E0,5E0,4E0,7E0,4E0,1E1,5E0,1.9E1,6E0,3.4E1,8E0,3.7E1,2.9E1,1E1,7E0,9E0,2.1E1,4E0,6E0,4E0,1.5E1,1.3E1,8E0,9E0,4E0,1.1E1,5E0,5.4E1,3E1,4E0,1E1,1.4E1,7.2E1,4E0,7E0,4E0,5E0,5E0,5E0,1.5E1,4E0,5E0,3.2E1,2.5E1,4E0,6E0,4E0,4E0,5E0,1.6E1,5E0,5E0,8E0,4E0,4E0,4E0,7E0,1.7E1,3.7E1,2.1E1,9E0,5E0,5E0,8E0,6E0,6.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[-2.7321313E-2,-2.1315159E-1,6.306059E-1,-6.0968655E-1,-3.1799924E-2,7.2708744E-1,-5.9831965E-1,-8.8215536E-1,-4.7226286E-1,-2.0722754E-1,1.3934027E-1,8.293986E-2,6.703266E-1,-7.6598215E-1,-1.9742332E-3,-9.230321E-1,-6.1476046E-1,-5.123159E-1,3.2396268E-2,-4.908164E-1,-1.3956308E-1,8.3611384E-2,4.0579364E-1,4.7080368E-1,8.793944E-1,-4.0456094E-2,-1.9186908E-2,-3.615302E-2,-4.6607025E-2,-2.1061836E-2,-3.7400108E-2,1.5865411E-1,-5.477417E-1,-5.198807E-1,-5.2498714E-3,-1.2169319E-1,-4.0987976E-2,2.1637887E-1,-5.33264E-4,1.7481834E-1,5.268182E-1,1.8728648E-1,6.160114E-1,9.235906E-1,8.279611E-3,-1.2028875E-2,2.5222281E-2,-6.198929E-1,-2.2404358E-1,-1.082487E-2,-5.405165E-1,-1.6959687E-1,4.6009153E-2,2.5578412E-1,-1.0140608E-2,2.3873476E-2,-3.665751E-1,4.7975876E-3,1.383455E-2,5.8828557E-1,4.1362466E-3,-7.1966544E-2,2.8597155E-1,7.824462E-1,4.319763E-1,9.632503E-1,1.4324811E-2,-6.794125E-1,-2.871075E-1,-7.158672E-2,-2.9669267E-1,-2.6181515E-2,-1.6252914E-2,-1.1395624E-1,-3.5214362E-1,3.2989262E-3,1.6241241E-2,1.1122572E-1,4.8269832E-1,5.7618495E-2,-1.851813E-1,-2.8752903E-3,-2.760847E-2,1.4357027E-2,6.527789E-1,1.8022266E-3,-6.5119686E-3,2.2731097E-2,1.8925148E-1,8.800229E-3,8.526836E-1,3.4963164E-1,2.9152198E-2,7.874639E-1,5.2728053E-2,-2.3668635E-1,-7.176559E-1,5.290712E-3,-3.8834456E-1,-5.3105573E-3,-7.527128E-4,-1.6047608E-2,-5.570708E-3,-2.509221E-2,-1.9591716E-1,-5.0313705E-1,-1.4803258E-1,9.098035E-3,-5.211884E-2,2.0809026E-1,-6.796483E-2,5.4342055E-1,2.1304288E-3,3.8789086E-2,1.6075788E-2,-3.3229384E-1,1.5198614E-2,1.7619217E-2,3.2974865E-2,-3.0607564E-4,2.7367184E-1,9.658273E-1,2.3954164E-2,3.9227602E-1,3.8595155E-3,8.813821E-1,6.2130105E-1,-1.5695505E-2,-3.034763E-3,-6.6468513E-1,-1.0128735E0,-2.6655287E-2,-2.6167172E-1,-1.051066E-2,1.6605336E-2,-2.5957415E-1,-8.310299E-2,-6.68079E-3,-6.0008013E-1,-1.804626E-2,-1.87105E-2,-8.626734E-4,-1.6607787E-2,1.8373882E-2,2.756532E-1,-1.4515831E-2,1.01857E-1,1.8210046E-2,2.7924657E-2,-1.0048564E-1,6.1597485E-2,-1.9242357E-2,-8.902351E-3,6.1707953E-3,-4.9069044E-3,1.476883E-2,5.8638025E-3,3.70601E-2,5.860416E-2,1.9564262E-2,9.020591E-3,4.287401E-2,2.2662248E-2,1.7723396E-2,3.214333E-2,-3.265898E-2,-2.2636048E-2,-5.324868E-2,-2.5816234E-2,-6.2112897E-3,-1.5400013E-2,2.2749745E-3,-8.187207E-3,-6.239129E-3,-1.5513657E-2,-9.91701E-3,-1.9199622E-3,-3.824728E-2,-1.6531495E-2,-3.667687E-3,3.083856E-3,-2.4959133E-3,3.1207814E-3,4.0119765E-3,-3.1166603E-3,4.8071994E-3,1.4476486E-2,-1.2227817E-4,7.1349684E-3,6.553441E-3,-8.920388E-3,8.233219E-3,1.6653351E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,45,47,49,-1,51,-1,53,55,57,59,61,63,65,-1,-1,-1,67,69,-1,71,73,75,77,-1,79,81,-1,-1,83,-1,85,87,89,91,93,-1,95,97,99,101,-1,-1,103,105,107,-1,109,111,113,115,-1,-1,-1,117,-1,-1,-1,119,-1,121,123,-1,125,-1,127,129,-1,131,-1,-1,-1,-1,133,135,137,139,-1,141,143,145,147,-1,149,-1,151,153,-1,-1,-1,155,157,-1,159,-1,161,163,-1,-1,165,167,-1,169,-1,171,173,175,-1,177,179,-1,181,-1,183,185,-1,187,-1,-1,189,191,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2545379E2,5.7510433E1,2.7231544E1,8.9817505E0,1.654137E1,1.1529106E1,1.5508342E0,3.3976746E-1,8.302612E0,5.150878E0,4.108984E0,0E0,7.970459E0,1.3645267E-1,0E0,5.4271698E-2,6.415844E-2,3.985115E0,0E0,5.1097107E-1,2.8446512E0,2.5919433E0,1.2496796E0,4.207363E0,2.8361664E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6027203E0,3.5045319E0,1.5621662E-1,0E0,1.7510853E0,0E0,1.6019707E0,1.2862974E0,1.2812662E-1,7.8057575E-1,9.6720326E-1,1.7754593E0,1.9155502E0,0E0,0E0,0E0,2.3204575E0,3.1103635E-1,0E0,1.813507E-2,1.6764069E0,6.3827497E-1,2.6689067E0,0E0,9.6303517E-1,5.955895E-1,0E0,0E0,2.8039265E-1,0E0,8.971529E-2,4.4498086E-1,1.2928009E0,3.53014E-1,1.7619934E0,0E0,1.6417923E0,8.983244E-1,2.4501812E-2,1.3657093E-1,0E0,0E0,9.492065E-1,1.1274762E0,4.824133E-1,0E0,9.167177E-1,8.232727E-1,6.261544E-1,5.6514E-1,0E0,0E0,0E0,7.532787E-2,0E0,0E0,0E0,3.3253253E-1,0E0,6.713581E-1,2.7360535E-1,0E0,1.7026138E-1,0E0,1.5389633E-1,8.59272E-1,0E0,2.5520897E-1,0E0,0E0,0E0,0E0,5.439538E-1,4.6676445E-1,6.778736E-1,5.9392595E-1,0E0,5.510912E-1,4.3074036E-1,8.26516E-1,1.6768456E-2,0E0,3.58796E-1,0E0,4.906094E-2,1.4353059E-1,0E0,0E0,0E0,5.3943157E-2,7.2143555E-2,0E0,5.8703184E-2,0E0,3.7015915E-2,8.700609E-2,0E0,0E0,2.8822327E-1,1.806736E-1,0E0,6.4600706E-2,0E0,3.5032257E-1,3.6281037E-1,1.3317958E-1,0E0,6.418886E-1,7.2921544E-2,0E0,1.1639744E-1,0E0,6.462759E-2,1.2911427E-1,0E0,6.63499E-2,0E0,0E0,3.8643885E-1,2.796137E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,47,47,48,48,50,50,51,51,52,52,53,53,55,55,56,56,59,59,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,73,73,74,74,75,75,77,77,78,78,79,79,80,80,84,84,88,88,90,90,91,91,93,93,95,95,96,96,98,98,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,113,113,115,115,116,116,120,120,121,121,123,123,125,125,126,126,129,129,130,130,132,132,134,134,135,135,136,136,138,138,139,139,141,141,143,143,144,144,146,146,149,149,150,150],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,46,48,50,-1,52,-1,54,56,58,60,62,64,66,-1,-1,-1,68,70,-1,72,74,76,78,-1,80,82,-1,-1,84,-1,86,88,90,92,94,-1,96,98,100,102,-1,-1,104,106,108,-1,110,112,114,116,-1,-1,-1,118,-1,-1,-1,120,-1,122,124,-1,126,-1,128,130,-1,132,-1,-1,-1,-1,134,136,138,140,-1,142,144,146,148,-1,150,-1,152,154,-1,-1,-1,156,158,-1,160,-1,162,164,-1,-1,166,168,-1,170,-1,172,174,176,-1,178,180,-1,182,-1,184,186,-1,188,-1,-1,190,192,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,4.217427E7,8.8E1,9.31E2,2.2968E4,2.629E3,1.0795462E-6,6.68411E2,3.95E2,2.8530578E6,8.293986E-2,2.36433E6,1.665909E3,-1.9742332E-3,6.48334E5,4.9558692E7,6E0,3.2396268E-2,7.9016544E5,9.639872E0,1.7142857E0,6.009E4,8.5023944E2,1.0424884E-7,-4.0456094E-2,-1.9186908E-2,-3.615302E-2,-4.6607025E-2,-2.1061836E-2,-3.7400108E-2,1.0332258E2,7.997723E6,1.9E1,-5.2498714E-3,2.9652428E6,-4.0987976E-2,1.0767881E7,1.2294118E1,6.965855E9,8.0605554E2,6.516E3,9E1,7.153514E1,8.279611E-3,-1.2028875E-2,2.5222281E-2,3.2479605E2,1.8E0,-1.082487E-2,1.585814E6,1.23E2,3.1847827E2,4.797342E2,-1.0140608E-2,2.2968E4,1.4787234E1,4.7975876E-3,1.383455E-2,3.7788504E7,4.1362466E-3,1.3E1,3.3412E5,1.2825651E0,1.000501E6,7.459E3,1.4324811E-2,1.9186046E0,4.6847186E2,2.019537E2,4.6404468E2,-2.6181515E-2,-1.6252914E-2,1E0,7.6E2,6.728972E-1,1.6241241E-2,3.4E3,1E0,4.86392E5,3.1797794E1,-2.8752903E-3,-2.760847E-2,1.4357027E-2,3.474468E2,1.8022266E-3,-6.5119686E-3,2.2731097E-2,8.7390656E2,8.800229E-3,2.1298597E0,5.712652E8,2.9152198E-2,5.0741018E8,5.2728053E-2,1.1E1,4.9735293E0,5.290712E-3,5.100098E4,-5.3105573E-3,-7.527128E-4,-1.6047608E-2,-5.570708E-3,4.82E2,2.118835E6,1.0865825E0,9E0,9.098035E-3,5.6530495E6,5.5E1,5E0,1.798E3,2.1304288E-3,2.8788235E2,1.6075788E-2,7.3131656E5,4.383864E7,1.7619217E-2,3.2974865E-2,-3.0607564E-4,7.683389E7,2.64538E3,2.3954164E-2,4.4895835E0,3.8595155E-3,2.395631E7,3.422592E6,-1.5695505E-2,-3.034763E-3,1.318849E6,8.237624E0,-2.6655287E-2,1.9032816E7,-1.051066E-2,1.6527965E7,1.22896E5,5.3E2,-6.68079E-3,1E0,2.9417648E2,-1.87105E-2,4.1973075E6,-1.6607787E-2,1.3529412E0,5.6151875E5,-1.4515831E-2,7.9016544E5,1.8210046E-2,2.7924657E-2,1.089E3,1E0,-1.9242357E-2,-8.902351E-3,6.1707953E-3,-4.9069044E-3,1.476883E-2,5.8638025E-3,3.70601E-2,5.860416E-2,1.9564262E-2,9.020591E-3,4.287401E-2,2.2662248E-2,1.7723396E-2,3.214333E-2,-3.265898E-2,-2.2636048E-2,-5.324868E-2,-2.5816234E-2,-6.2112897E-3,-1.5400013E-2,2.2749745E-3,-8.187207E-3,-6.239129E-3,-1.5513657E-2,-9.91701E-3,-1.9199622E-3,-3.824728E-2,-1.6531495E-2,-3.667687E-3,3.083856E-3,-2.4959133E-3,3.1207814E-3,4.0119765E-3,-3.1166603E-3,4.8071994E-3,1.4476486E-2,-1.2227817E-4,7.1349684E-3,6.553441E-3,-8.920388E-3,8.233219E-3,1.6653351E-3],"split_indices":[2,43,60,44,2,12,0,52,67,2,43,0,43,4,0,46,46,3,0,43,69,68,1,67,52,0,0,0,0,0,0,67,9,67,0,43,0,62,68,46,4,44,8,71,0,0,0,67,68,0,9,0,4,4,0,44,71,0,0,7,0,3,1,69,43,2,0,69,4,4,4,0,0,26,2,69,0,44,6,1,71,0,0,0,4,0,0,0,4,0,53,47,0,7,0,3,68,0,43,0,0,0,0,2,9,71,8,0,43,0,8,2,0,70,0,43,12,0,0,0,47,4,0,68,0,62,43,0,0,9,71,0,5,0,60,9,2,0,29,4,0,43,0,68,66,0,43,0,0,2,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.024E3,7.99E2,2.25E2,2.5E2,5.49E2,2.09E2,1.6E1,8.2E1,1.68E2,2.71E2,2.78E2,9E0,2E2,1.2E1,4E0,6.9E1,1.3E1,1.63E2,5E0,5.1E1,2.2E2,2.31E2,4.7E1,1.04E2,9.6E1,8E0,4E0,3E1,3.9E1,9E0,4E0,8E0,1.55E2,4.7E1,4E0,2.16E2,4E0,8.9E1,1.42E2,1.7E1,3E1,3.6E1,6.8E1,9E1,6E0,4E0,4E0,1.26E2,2.9E1,4E0,4.3E1,1.68E2,4.8E1,8.2E1,7E0,1.34E2,8E0,1.2E1,5E0,2.6E1,4E0,1E1,2.6E1,3.4E1,3.4E1,8.4E1,6E0,1.06E2,2E1,1E1,1.9E1,3.6E1,7E0,1.3E2,3.8E1,4.3E1,5E0,5.1E1,3.1E1,1.16E2,1.8E1,4E0,4E0,6E0,2E1,4E0,6E0,7E0,1.9E1,4E0,3E1,2.6E1,8E0,4.5E1,3.9E1,9E0,9.7E1,4E0,1.6E1,5E0,5E0,1.4E1,5E0,6.3E1,6.7E1,2.1E1,1.7E1,9E0,3.4E1,3.3E1,1.8E1,2.7E1,4E0,1.1E2,6E0,1E1,8E0,5E0,1.5E1,6E0,1.3E1,2.1E1,9E0,2.2E1,4E0,2.6E1,1.9E1,5E0,4E0,8.5E1,1.2E1,5E0,1.1E1,1E1,5.3E1,4.2E1,2.5E1,5E0,1.6E1,1.2E1,5E0,3E1,4E0,9E0,2.4E1,7E0,1.1E1,1E1,1.7E1,1.5E1,9.5E1,5E0,5E0,4E0,4E0,9E0,4E0,1.6E1,5E0,1.8E1,4E0,2.2E1,4E0,6E0,1.3E1,6.6E1,1.9E1,8E0,4E0,5E0,6E0,4.6E1,7E0,1.7E1,2.5E1,5E0,2E1,7E0,9E0,7E0,5E0,1.7E1,1.3E1,5E0,4E0,5E0,1.9E1,4E0,7E0,4E0,1.1E1,1.6E1,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"193","size_leaf_vector":"1"}},{"base_weights":[8.976589E-3,-1.8565607E-1,5.751452E-1,-5.6272644E-1,-2.3799082E-2,6.2997967E-1,-2.76464E-1,-7.7734095E-1,-4.4256908E-1,-2.1286839E-1,1.2544522E-1,2.6550955E-1,7.993267E-1,4.3805264E-2,-3.2085083E-2,-8.1119436E-1,-5.5827997E-3,-4.7796077E-1,2.0840403E-2,-4.5805752E-1,-1.431906E-1,2.819183E-2,3.1765613E-1,1.5147437E-1,6.3087624E-1,1.403616E0,7.234894E-1,-1.037821E-2,1.6501375E-2,-3.9388802E-2,-2.1298224E-2,-7.748547E-2,-5.2942514E-1,-4.893122E-1,-9.062538E-3,-4.8304686E-1,-1.1806283E-1,1.6423035E-1,-5.593352E-2,-1.1546289E-2,3.5989767E-1,9.9220954E-2,2.211636E-2,2.016987E-2,3.5880797E-2,3.6084726E-2,7.382241E-2,5.482194E-1,9.394883E-1,-1.85779E-2,5.3337198E-2,-5.7060766E-1,-1.9487064E-1,-1.4499942E-2,-5.382861E-1,-1.003566E-2,-3.750208E-2,-1.6307665E-1,7.102636E-2,-1.2485949E-1,2.2788653E-1,-1.2284219E-2,-2.6112384E-1,4.6732873E-3,-8.757485E-3,3.8353354E-1,-4.6064984E-3,-5.8206096E-2,2.1282099E-1,1.7325893E-3,5.804768E-1,9.9883366E-1,9.209491E-3,1.3763387E-2,-8.8465676E-2,-6.1320615E-1,-2.8467864E-1,-1.3864814E-2,-1.4656074E-3,-1.4489231E-2,-2.620424E-2,-1.2501457E-1,-3.0542275E-1,-1.4960766E-3,1.5735818E-2,-1.7184617E-2,5.3753514E-2,2.5714293E-1,-3.8416544E-3,-4.5830585E-2,1.8086722E-1,-2.8967168E-2,-4.0176123E-1,2.6755875E-1,5.107791E-1,-9.77747E-3,-3.8284925E-3,3.148756E-1,-5.5413647E-3,4.393887E-1,7.174278E-1,1.0461825E0,1.285394E-2,-7.5843628E-3,2.2770879E-4,-5.5861133E-1,-8.444426E-1,1.9463387E-3,-4.3705767E-1,-2.5320935E-1,-4.7029063E-2,-2.8062437E-2,-1.954484E-1,8.442496E-3,-6.283745E-2,5.73461E-3,-1.2645593E-3,1.9284983E-1,2.015421E-2,-1.522078E-1,5.430307E-3,1.5427817E-2,4.1712478E-2,-5.9377323E-3,3.528872E-3,-2.2562139E-2,-7.6467153E-3,1.6048105E-1,4.5160702E-1,1.0057352E-2,5.5892915E-1,-4.9862205E-3,3.362375E-2,5.703676E-3,3.718224E-1,4.9299067E-1,2.4838407E-1,1.233643E-2,3.5229653E-2,2.9737024E-2,5.0002776E-2,-5.8186895E-1,-8.254169E-3,-4.769092E-2,-2.6963526E-2,-2.3705885E-2,-1.1233391E-2,-6.110349E-2,-2.9792884E-1,2.6852906E-2,-1.15500115E-1,-2.787465E-1,3.3788867E-3,-1.0285062E-2,-1.0577184E-2,2.4601297E-1,5.1241584E-2,9.319882E-4,-2.1062881E-1,1.6773035E-1,-4.8790377E-2,5.2741645E-3,-1.8054264E-3,1.9213195E-1,8.572016E-5,2.77106E-2,8.10215E-3,2.7518088E-2,1.1159483E-2,3.4063791E-3,1.4545783E-4,2.1889698E-2,1.0033796E-2,2.5096098E-2,8.924695E-3,4.0383646E-3,1.5147768E-2,-2.8510474E-2,-2.0666638E-2,-4.3014597E-4,-4.6511157E-3,-1.6400533E-2,-4.911072E-3,-3.5677375E-3,3.1142822E-3,4.9126344E-3,-7.147473E-3,-1.4926865E-2,-4.9747466E-3,-3.3059209E-3,3.122686E-3,1.32718235E-2,5.2311765E-3,7.835372E-3,-3.4389351E-3,-1.5331047E-2,-4.2531174E-3,1.9013898E-3,1.2627951E-2,-8.465574E-3,4.5229162E-6,1.6677508E-2,6.5962044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,-1,33,35,37,39,41,43,45,47,-1,-1,-1,-1,49,51,53,-1,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,71,-1,73,75,77,-1,79,-1,-1,81,83,85,87,89,91,-1,-1,93,-1,95,97,-1,99,101,-1,-1,103,105,107,-1,-1,-1,-1,109,111,113,-1,-1,115,117,-1,119,121,123,125,127,129,-1,131,133,-1,135,137,139,-1,-1,-1,141,143,-1,145,147,149,-1,151,-1,153,-1,-1,155,-1,157,159,-1,161,-1,-1,-1,-1,163,165,-1,167,-1,169,-1,171,173,175,-1,-1,-1,-1,177,-1,-1,-1,-1,-1,179,181,183,185,187,-1,189,-1,191,193,-1,195,197,199,-1,-1,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0897934E2,4.4905777E1,1.198864E1,5.355011E0,1.4586942E1,1.4477592E1,2.1725507E0,1.6355019E0,4.770111E0,3.8296175E0,5.393369E0,3.112617E0,6.408577E0,9.2148066E-1,0E0,6.796303E-1,0E0,2.8237972E0,0E0,3.1328678E-1,1.4715998E0,2.2179337E0,1.3530397E0,9.7847974E-1,1.7074823E-1,8.082886E-1,5.012245E0,0E0,0E0,0E0,0E0,7.4344707E-1,1.5834732E0,2.091322E-1,0E0,8.7217784E-1,1.4415362E0,1.3902282E0,1.0715619E0,2.5992435E-1,9.7247887E-1,9.602246E-1,0E0,0E0,0E0,0E0,0E0,1.3456078E0,2.6120338E0,0E0,4.77219E-1,1.1530685E0,2.4802244E-1,0E0,3.64933E-2,0E0,0E0,6.986644E-1,6.4568603E-1,6.379125E-1,5.764792E-1,6.5624213E-1,6.740104E-1,0E0,0E0,1.0734329E0,0E0,1.9108456E-1,1.0905383E0,0E0,1.1918983E0,1.7219849E0,0E0,0E0,6.444004E-2,7.1496964E-1,8.151921E-1,0E0,0E0,0E0,0E0,1.0778015E0,8.1122875E-1,3.1662092E-1,0E0,0E0,5.478035E-2,5.607197E-1,0E0,4.6978372E-1,3.0558726E-1,1.04214124E-1,1.8515182E-1,8.143618E-1,4.1457844E-1,0E0,7.380738E-2,2.1570921E-1,0E0,2.9731607E-1,5.68861E-1,1.03645325E-1,0E0,0E0,0E0,5.941353E-1,1.813879E-1,0E0,3.748107E-2,3.3402133E-1,3.5215706E-1,0E0,5.1258624E-1,0E0,1.80965E-1,0E0,0E0,3.1111252E-1,0E0,2.8622818E-1,5.273847E-1,0E0,5.714111E-2,0E0,0E0,0E0,0E0,1.5026277E-1,5.498185E-1,0E0,2.289772E-1,0E0,1.6880456E-2,0E0,1.8097973E-1,3.6187077E-1,1.1210269E-1,0E0,0E0,0E0,0E0,7.489395E-2,0E0,0E0,0E0,0E0,0E0,1.750857E-2,3.2314444E-1,1.4872387E-1,3.2958877E-1,9.419072E-2,0E0,8.595689E-2,0E0,1.3731492E-1,2.0600218E-1,0E0,2.7127624E-1,1.8925777E-1,2.9580626E-1,0E0,0E0,1.56807E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,47,47,48,48,50,50,51,51,52,52,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,74,74,75,75,76,76,81,81,82,82,83,83,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,99,99,100,100,101,101,105,105,106,106,108,108,109,109,110,110,112,112,114,114,117,117,119,119,120,120,122,122,127,127,128,128,130,130,132,132,134,134,135,135,136,136,141,141,147,147,148,148,149,149,150,150,151,151,153,153,155,155,156,156,158,158,159,159,160,160,163,163],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,-1,34,36,38,40,42,44,46,48,-1,-1,-1,-1,50,52,54,-1,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,72,-1,74,76,78,-1,80,-1,-1,82,84,86,88,90,92,-1,-1,94,-1,96,98,-1,100,102,-1,-1,104,106,108,-1,-1,-1,-1,110,112,114,-1,-1,116,118,-1,120,122,124,126,128,130,-1,132,134,-1,136,138,140,-1,-1,-1,142,144,-1,146,148,150,-1,152,-1,154,-1,-1,156,-1,158,160,-1,162,-1,-1,-1,-1,164,166,-1,168,-1,170,-1,172,174,176,-1,-1,-1,-1,178,-1,-1,-1,-1,-1,180,182,184,186,188,-1,190,-1,192,194,-1,196,198,200,-1,-1,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,3.9661028E7,8.8E1,8.1E2,8.7115955E2,9.7172376E7,1.1408248E0,6.68411E2,3.89E2,2.0766992E6,3.1993368E6,1.059448E7,1E1,-3.2085083E-2,1.0795462E-6,-5.5827997E-3,1.9186046E0,2.0840403E-2,8.599521E5,1.45064E5,1.75E0,2.9077E4,5.53648E8,1.2003246E6,1.2655613E3,2.8045622E6,-1.037821E-2,1.6501375E-2,-3.9388802E-2,-2.1298224E-2,3.6E2,7.997723E6,1.68E2,-9.062538E-3,1.0153885E-5,3.3486558E6,1E0,9.639872E0,2.426015E-1,9.29554E2,2.4361508E6,2.211636E-2,2.016987E-2,3.5880797E-2,3.6084726E-2,7.382241E-2,1.2825651E0,6.292039E0,-1.85779E-2,6.2E1,3.2479605E2,1.85552E6,-1.4499942E-2,2.2116136E5,-1.003566E-2,-3.750208E-2,3.6507E4,2.0833333E0,3.1157124E-1,2.2968E4,3.1797794E1,5.194E3,4.6732873E-3,-8.757485E-3,4.2210345E2,-4.6064984E-3,1.5137369E6,6.1095314E-8,1.7325893E-3,1.1251919E6,1.0424884E-7,9.209491E-3,1.3763387E-2,1.037E3,7.382199E0,4.046E3,-1.3864814E-2,-1.4656074E-3,-1.4489231E-2,-2.620424E-2,5.47E2,5.4E0,9.4658756E-1,1.5735818E-2,-1.7184617E-2,9.783615E4,4.7185095E2,-3.8416544E-3,4.7567694E5,8.931E3,1.43E2,1.212945E6,5.405423E6,1.9501E4,-9.77747E-3,3.8121E4,4.1032645E6,-5.5413647E-3,2.5345264E7,2.0750147E8,2.707947E8,1.285394E-2,-7.5843628E-3,2.2770879E-4,6.077143E5,3.6981132E0,1.9463387E-3,7.582512E7,2.82E2,1.6E0,-2.8062437E-2,1.0937695E6,8.442496E-3,5.504831E8,5.73461E-3,-1.2645593E-3,1.5132743E0,2.015421E-2,1E0,1.99693E9,1.5427817E-2,4.699871E2,-5.9377323E-3,3.528872E-3,-2.2562139E-2,-7.6467153E-3,4.0911578E2,1.162E3,1.0057352E-2,4.1104166E8,-4.9862205E-3,2.8608696E1,5.703676E-3,1.6942337E1,4.2964826E0,6.2193125E-2,1.233643E-2,3.5229653E-2,2.9737024E-2,5.0002776E-2,2.240836E6,-8.254169E-3,-4.769092E-2,-2.6963526E-2,-2.3705885E-2,-1.1233391E-2,1.9E1,1.4576986E7,1E0,7.9634375E2,7.6E2,3.3788867E-3,9.356961E6,-1.0577184E-2,2.3072304E8,2.749236E6,9.319882E-4,9.44363E5,7.026624E7,2.1340206E0,5.2741645E-3,-1.8054264E-3,7E0,8.572016E-5,2.77106E-2,8.10215E-3,2.7518088E-2,1.1159483E-2,3.4063791E-3,1.4545783E-4,2.1889698E-2,1.0033796E-2,2.5096098E-2,8.924695E-3,4.0383646E-3,1.5147768E-2,-2.8510474E-2,-2.0666638E-2,-4.3014597E-4,-4.6511157E-3,-1.6400533E-2,-4.911072E-3,-3.5677375E-3,3.1142822E-3,4.9126344E-3,-7.147473E-3,-1.4926865E-2,-4.9747466E-3,-3.3059209E-3,3.122686E-3,1.32718235E-2,5.2311765E-3,7.835372E-3,-3.4389351E-3,-1.5331047E-2,-4.2531174E-3,1.9013898E-3,1.2627951E-2,-8.465574E-3,4.5229162E-6,1.6677508E-2,6.5962044E-3],"split_indices":[2,43,60,44,2,67,60,53,67,2,43,43,12,3,0,52,0,69,0,43,7,68,1,7,66,67,43,0,0,0,0,2,9,44,0,53,43,8,69,53,67,60,0,0,0,0,0,69,53,0,0,67,12,0,43,0,0,1,68,53,44,71,44,0,0,67,0,62,52,0,43,52,0,0,2,69,44,0,0,0,0,2,73,68,0,0,48,4,0,43,44,0,9,43,9,0,1,60,0,60,7,7,0,0,0,62,71,0,7,44,68,0,43,0,5,0,0,68,0,26,46,0,67,0,0,0,0,4,10,0,7,0,73,0,71,68,72,0,0,0,0,9,0,0,0,0,0,0,5,29,48,2,0,60,0,7,12,0,9,7,68,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.87E2,7.35E2,2.52E2,2.2E2,5.15E2,2.37E2,1.5E1,7.7E1,1.43E2,2.27E2,2.88E2,7.6E1,1.61E2,9E0,6E0,7.3E1,4E0,1.38E2,5E0,4.9E1,1.78E2,1.92E2,9.6E1,5.9E1,1.7E1,1.6E1,1.45E2,5E0,4E0,6.4E1,9E0,1.6E1,1.22E2,4.3E1,6E0,1.1E1,1.67E2,7.3E1,1.19E2,1.1E1,8.5E1,5.2E1,7E0,9E0,8E0,5E0,1.1E1,8.2E1,6.3E1,4E0,1.2E1,1.08E2,1.4E1,1.1E1,3.2E1,7E0,4E0,1.35E2,3.2E1,1.3E1,6E1,9.9E1,2E1,7E0,4E0,8.1E1,4E0,2.2E1,3E1,5E0,7.7E1,5.8E1,5E0,4E0,8E0,9.3E1,1.5E1,8E0,6E0,5E0,2.7E1,1.08E2,2.7E1,2.6E1,6E0,5E0,8E0,5.5E1,5E0,8.5E1,1.4E1,8E0,1.2E1,4.4E1,3.7E1,5E0,1.7E1,2.3E1,7E0,4E1,3.7E1,5.4E1,4E0,4E0,4E0,7.8E1,1.5E1,5E0,1E1,4E1,6.8E1,6E0,2.1E1,6E0,2E1,4E0,4E0,4.2E1,1.3E1,2.7E1,5.8E1,6E0,8E0,4E0,4E0,8E0,4E0,2.9E1,1.5E1,6E0,3.1E1,4E0,1.3E1,6E0,1.7E1,3E1,1E1,4E0,3.3E1,6E0,4.8E1,7.3E1,5E0,7E0,8E0,6E0,4E0,8E0,3.2E1,3.3E1,3.5E1,1.6E1,5E0,1.6E1,4E0,3E1,1.2E1,7E0,2E1,1.4E1,4.4E1,4E0,4E0,2.4E1,5E0,9E0,6E0,2.7E1,4E0,5E0,8E0,9E0,8E0,2.5E1,5E0,4E0,6E0,5.5E1,1.8E1,4E0,4E0,2.4E1,8E0,9E0,2.4E1,5E0,3E1,1.2E1,4E0,9E0,7E0,2.2E1,8E0,6E0,6E0,9E0,1.1E1,7E0,7E0,1.1E1,3.3E1,4E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"203","size_leaf_vector":"1"}},{"base_weights":[2.5899991E-2,-1.9419216E-1,5.6307477E-1,-5.480851E-1,-1.6844464E-2,-6.132231E-1,6.104611E-1,-7.590241E-1,-4.331985E-1,-1.6650082E-1,1.4112706E-1,-3.4462553E-2,-1.2852693E-2,3.1988806E-1,7.7225965E-1,-6.60953E-1,-4.053686E-2,-3.6462346E-1,-9.5011306E-1,-4.4129208E-1,-1.069993E-1,2.9474112E-1,1.5751341E-2,1.5035282E-1,6.103874E-1,9.355166E-1,4.9535182E-1,-3.4822058E-2,-4.5412448E-1,-5.099062E-1,-1.3240522E-1,-1.2051009E0,-2.5928061E-2,-3.4435704E-1,-6.0574263E-1,-1.6271408E-1,7.9785064E-2,2.10817E-1,4.4555056E-1,7.495849E-2,-2.979508E-1,-1.1562876E-1,2.3606525E-1,6.521973E-1,9.4247125E-3,8.18089E-1,6.682793E-2,1.0034611E0,4.039347E-1,-2.3135023E-2,-1.0750364E-2,-5.448496E-1,-2.1894737E-1,2.403337E-2,-2.5201276E-1,-3.447157E-2,-6.998315E-2,-1.7853767E-2,-8.329051E-3,-1.5855588E-2,-3.0953916E-2,-3.6070856E-1,-9.5359966E-2,1.8069474E-1,-3.775566E-2,1.6005524E-1,4.9938202E-1,5.095464E-1,4.5501604E-3,1.4347373E-1,-3.2632295E-2,-1.7336248E-1,-3.0581964E-2,-1.3022984E-2,6.846515E-4,2.784489E-1,-9.681204E-3,2.2284646E-2,3.5152428E-2,8.643828E-1,4.8850462E-1,2.0195067E-2,6.0435697E-2,5.3143036E-1,4.268525E-2,-4.9129814E-1,-6.93085E-1,-3.5155436E-3,-1.3384149E-2,-1.8813774E-1,-4.759241E-1,-1.7540015E-1,-5.3724694E-1,-1.3317022E-1,6.989068E-2,8.843524E-2,3.194664E-1,-1.0942415E-1,8.5886896E-2,2.0008877E-1,-3.15232E-3,2.8689932E-2,1.4543351E-2,1.710361E-2,3.013517E-2,-4.9161958E-3,1.693727E-1,-1.8062603E-1,2.8509917E-2,-6.3914515E-4,-2.3867954E-1,-3.1393045E-3,3.197293E-3,4.2300195E-1,1.6525213E-1,9.7641134E-1,6.860471E-1,1.2540972E-2,2.8521437E-2,3.4125382E-1,6.6992146E-1,2.1136494E-1,-1.0555592E-2,-2.7661588E-2,-3.5579148E-1,-3.688066E-2,-2.1333795E-2,-2.251704E-1,6.15445E-4,-2.8207693E-2,-1.01252515E-2,-2.2520797E-1,1.340996E-3,-2.25969E-1,-7.7138627E-1,-9.460886E-3,-1.8760873E-1,1.7805059E-2,1.1727069E-2,1.493668E-1,-4.050449E-3,6.199278E-3,1.9432344E-2,-1.6709466E-1,1.8172765E-3,7.148093E-3,-5.885711E-6,1.937808E-2,1.7126915E-1,-8.527477E-3,9.799177E-2,5.077055E-2,2.2464107E-1,-2.4591428E-1,-1.2465415E-3,9.393035E-2,-1.24808066E-1,-1.4112135E-2,-5.589379E-3,1.0473481E-2,4.909368E-1,-3.731076E-5,2.1057786E-1,2.4195883E-2,1.0056611E0,1.1438845E-2,7.508684E-1,4.195695E-1,3.48971E-3,1.5414384E-2,3.317483E-2,1.0831631E-3,1.4571091E-2,-1.8158314E-3,-4.0485722E-1,-1.2284252E-3,-2.683146E-1,-1.5851019E-2,-6.814049E-3,-1.6526833E-2,-1.0469066E-3,-4.748191E-2,-1.7853165E-2,-9.79328E-2,7.960233E-2,-1.12324744E-1,-2.5538152E-1,7.7016495E-2,-4.375171E-3,8.708977E-3,2.2654105E-3,-1.2472493E-2,-1.6801341E-3,5.627967E-4,1.908505E-1,1.1876486E-3,6.9611794E-3,-5.1907017E-3,1.2351117E-1,2.7262053E-1,9.663919E-2,-1.5007516E-2,-5.4422948E-3,1.5850979E-1,9.070066E-3,-3.568707E-4,-1.1103784E-2,2.631044E-2,1.0885399E-2,2.6020753E-1,1.0997864E-1,1.0956743E0,3.504616E-2,2.376837E-2,3.8961507E-2,2.3762712E-2,1.0358896E-2,-2.0228496E-2,-8.560411E-3,-1.428076E-2,-3.418675E-3,1.5080068E-3,-8.294648E-3,2.0374921E-4,5.2380306E-3,-7.723287E-3,8.1116444E-4,-1.4845412E-2,-7.698857E-3,5.2661817E-3,-2.280954E-4,2.2682855E-3,9.481751E-3,1.0832815E-2,2.8570371E-3,8.336831E-3,1.5360508E-2,1.07226E-2,4.919231E-4,1.0295539E-3,1.0005695E-2,3.7886964E-3,-3.0204777E-3,7.2013107E-3,1.4736199E-2,1.6959865E-3,7.449649E-3,5.29985E-2,2.8122451E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,-1,79,-1,81,83,-1,-1,85,87,-1,89,-1,-1,-1,-1,-1,-1,91,93,95,97,99,101,103,-1,105,107,109,-1,-1,111,113,-1,-1,-1,115,117,-1,-1,119,121,123,125,-1,-1,127,129,131,133,135,137,139,141,143,145,147,149,-1,-1,-1,-1,-1,151,153,155,-1,157,-1,-1,159,161,163,165,-1,-1,167,169,171,-1,-1,173,-1,-1,175,-1,-1,-1,177,-1,179,181,183,185,187,-1,189,-1,-1,-1,191,-1,-1,-1,-1,193,-1,195,197,199,201,-1,203,205,-1,-1,-1,207,-1,209,-1,211,-1,213,215,-1,-1,-1,-1,-1,-1,217,-1,219,-1,-1,-1,-1,-1,-1,221,223,225,227,229,-1,-1,-1,-1,-1,-1,231,-1,-1,-1,233,235,237,-1,-1,239,241,-1,-1,-1,-1,243,245,247,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.20820076E2,4.5493286E1,1.6929268E1,5.5358353E0,1.1465957E1,3.2609987E-1,1.3199051E1,3.9554977E-1,5.289707E0,4.024057E0,4.5405936E0,0E0,0E0,5.0302134E0,7.809494E0,6.1725426E-1,0E0,4.7231846E0,1.1085796E0,4.928131E-1,2.1567519E0,1.2476654E0,2.4554505E0,1.5547571E0,5.1260567E-1,5.6573105E0,2.846161E0,0E0,6.9381475E-2,7.652416E-1,4.5023556E0,2.911539E-1,0E0,1.3752985E-1,2.3151398E-2,2.0859861E0,5.7615006E-1,9.4543624E-1,7.715993E-1,8.2293034E-1,8.376204E-1,3.2894325E-1,1.0215862E0,2.6061726E-1,0E0,1.0272217E0,0E0,1.1537552E0,2.7868376E0,0E0,0E0,2.7073288E-1,8.914286E-2,0E0,5.969732E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2143288E0,7.590878E-1,3.04626E-1,2.1320744E-1,3.9919102E-1,2.9009104E-2,3.7505722E-1,0E0,4.58107E-1,4.0329087E-1,1.7383623E-1,0E0,0E0,5.6437027E-2,6.9915605E-1,0E0,0E0,0E0,9.683914E-1,2.239654E-1,0E0,0E0,9.5658684E-1,8.258761E-1,6.605797E-1,6.941986E-2,0E0,0E0,2.93962E-1,2.1189547E-1,2.2035992E-1,1.2833385E0,6.5821123E-1,2.158435E-1,1.9551578E-1,1.3097978E-1,1.3525873E-1,5.3308338E-2,2.3793173E-1,2.566434E-1,0E0,0E0,0E0,0E0,0E0,3.9500177E-1,1.237697E-1,3.3268738E-1,0E0,5.7365417E-2,0E0,0E0,1.5541339E-1,2.1091986E-1,1.08860016E-1,8.088131E-1,0E0,0E0,4.0531397E-1,1.6530609E-1,2.0765361E-1,0E0,0E0,4.3441868E-1,0E0,0E0,2.7559638E-1,0E0,0E0,0E0,1.05151296E-1,0E0,2.5977653E-1,6.884546E-1,2.5206307E-1,3.1826448E-1,1.3386676E-1,0E0,4.181066E-2,0E0,0E0,0E0,1.3812086E-1,0E0,0E0,0E0,0E0,1.3785636E-1,0E0,3.0402087E-2,2.630603E-1,2.3159552E-1,5.274129E-2,0E0,1.2460758E-1,1.3341984E-1,0E0,0E0,0E0,1.4664006E-1,0E0,8.1222296E-2,0E0,1.4737701E-1,0E0,2.2190475E-1,1.8092537E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.999657E-2,0E0,1.9117963E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7635658E-1,4.0136524E-2,2.4515104E-1,1.5128946E-1,3.9953195E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.149565E-2,0E0,0E0,0E0,8.815101E-2,1.0552621E-1,1.4895368E-1,0E0,0E0,9.800756E-2,6.509405E-2,0E0,0E0,0E0,0E0,3.6113918E-2,2.789019E-2,5.406952E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,51,51,52,52,54,54,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,74,74,75,75,79,79,80,80,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,106,106,107,107,108,108,110,110,113,113,114,114,115,115,116,116,119,119,120,120,121,121,124,124,127,127,131,131,133,133,134,134,135,135,136,136,137,137,139,139,143,143,148,148,150,150,151,151,152,152,153,153,155,155,156,156,160,160,162,162,164,164,166,166,167,167,174,174,176,176,183,183,184,184,185,185,186,186,187,187,194,194,198,198,199,199,200,200,203,203,204,204,209,209,210,210,211,211],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,-1,80,-1,82,84,-1,-1,86,88,-1,90,-1,-1,-1,-1,-1,-1,92,94,96,98,100,102,104,-1,106,108,110,-1,-1,112,114,-1,-1,-1,116,118,-1,-1,120,122,124,126,-1,-1,128,130,132,134,136,138,140,142,144,146,148,150,-1,-1,-1,-1,-1,152,154,156,-1,158,-1,-1,160,162,164,166,-1,-1,168,170,172,-1,-1,174,-1,-1,176,-1,-1,-1,178,-1,180,182,184,186,188,-1,190,-1,-1,-1,192,-1,-1,-1,-1,194,-1,196,198,200,202,-1,204,206,-1,-1,-1,208,-1,210,-1,212,-1,214,216,-1,-1,-1,-1,-1,-1,218,-1,220,-1,-1,-1,-1,-1,-1,222,224,226,228,230,-1,-1,-1,-1,-1,-1,232,-1,-1,-1,234,236,238,-1,-1,240,242,-1,-1,-1,-1,244,246,248,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,1.972052E5,1E0,9.9E1,9.31E2,2.3308511E5,8.625455E2,9.55188E5,8.462282E0,3.79E2,3.3358974E0,-3.4462553E-2,-1.2852693E-2,2.308821E6,5.489183E1,3.9125E1,-4.053686E-2,6.83E2,7.3E1,1E0,3.3486558E6,4.371613E2,2E1,1E0,1E0,1.7379E4,1.2083E4,-3.4822058E-2,5.2755904E0,7.997723E6,2.7E1,8.317E3,-2.5928061E-2,3E0,3.7043128E5,1E0,3.5364E4,6.0052995E6,1E0,1.9E1,4.1E1,1E0,8.920169E6,1.0410659E5,9.4247125E-3,1E0,6.682793E-2,2.8016653E3,2.707256E7,-2.3135023E-2,-1.0750364E-2,1.2E1,1.2E1,2.403337E-2,2.694192E9,-3.447157E-2,-6.998315E-2,-1.7853767E-2,-8.329051E-3,-1.5855588E-2,-3.0953916E-2,7.4142776E7,7.814751E6,3.3832976E7,1.594941E3,1.0292E4,1.9913E4,5.460753E9,4.5501604E-3,1.4473684E-1,4.591837E0,1.4081731E1,-3.0581964E-2,-1.3022984E-2,3.414E3,8E0,-9.681204E-3,2.2284646E-2,3.5152428E-2,2.1089442E0,4.797448E8,2.0195067E-2,6.0435697E-2,1.3388E4,3.0070068E7,2.95E2,2.2E0,-3.5155436E-3,-1.3384149E-2,3.1158695E6,8.57E2,8.2E1,1.027972E0,1.5132743E0,8E0,6.34E2,2.6732E4,1.3683688E7,1.5E1,1.1160929E0,1.007E3,2.8689932E-2,1.4543351E-2,1.710361E-2,3.013517E-2,-4.9161958E-3,7.709291E7,8.847733E4,3.4444444E0,-6.3914515E-4,9.936263E6,-3.1393045E-3,3.197293E-3,4.2010452E2,4.3333566E2,9.2789966E-1,9.5E2,1.2540972E-2,2.8521437E-2,3.6440072E0,2.7317073E0,6.619098E8,-1.0555592E-2,-2.7661588E-2,3E1,-3.688066E-2,-2.1333795E-2,1.4285715E0,6.15445E-4,-2.8207693E-2,-1.01252515E-2,2.1651703E2,1.340996E-3,3.8E2,5.4E0,2.732381E2,7.619497E0,1.7288135E0,1.1727069E-2,1E0,-4.050449E-3,6.199278E-3,1.9432344E-2,2.1557376E0,1.8172765E-3,7.148093E-3,-5.885711E-6,1.937808E-2,4.0449125E5,-8.527477E-3,3.1096E4,4.135135E0,2.0158867E8,3.0291306E2,-1.2465415E-3,1.390602E8,2.7012987E1,-1.4112135E-2,-5.589379E-3,1.0473481E-2,5.7819215E6,-3.731076E-5,3.1530054E0,2.4195883E-2,1.2820834E7,1.1438845E-2,2.5247778E6,6.439711E1,3.48971E-3,1.5414384E-2,3.317483E-2,1.0831631E-3,1.4571091E-2,-1.8158314E-3,1.432632E6,-1.2284252E-3,2.1340163E8,-1.5851019E-2,-6.814049E-3,-1.6526833E-2,-1.0469066E-3,-4.748191E-2,-1.7853165E-2,1.01417E0,4.0481758E3,2.884188E2,1.339646E6,1.4E1,-4.375171E-3,8.708977E-3,2.2654105E-3,-1.2472493E-2,-1.6801341E-3,5.627967E-4,2.2734E4,1.1876486E-3,6.9611794E-3,-5.1907017E-3,1.0439024E1,8.995735E5,5E0,-1.5007516E-2,-5.4422948E-3,5.617715E6,9E1,-3.568707E-4,-1.1103784E-2,2.631044E-2,1.0885399E-2,8.4797815E2,2.427869E8,3.4332926E0,3.504616E-2,2.376837E-2,3.8961507E-2,2.3762712E-2,1.0358896E-2,-2.0228496E-2,-8.560411E-3,-1.428076E-2,-3.418675E-3,1.5080068E-3,-8.294648E-3,2.0374921E-4,5.2380306E-3,-7.723287E-3,8.1116444E-4,-1.4845412E-2,-7.698857E-3,5.2661817E-3,-2.280954E-4,2.2682855E-3,9.481751E-3,1.0832815E-2,2.8570371E-3,8.336831E-3,1.5360508E-2,1.07226E-2,4.919231E-4,1.0295539E-3,1.0005695E-2,3.7886964E-3,-3.0204777E-3,7.2013107E-3,1.4736199E-2,1.6959865E-3,7.449649E-3,5.29985E-2,2.8122451E-2],"split_indices":[2,43,17,44,2,48,67,46,69,2,69,0,0,43,71,4,0,2,0,26,43,67,3,27,6,2,9,0,69,9,0,9,0,8,43,29,9,43,6,8,3,26,62,48,0,6,0,4,60,0,0,8,3,0,46,0,0,0,0,0,0,5,60,7,48,44,9,46,0,71,69,73,0,0,2,32,0,0,0,53,7,0,0,2,58,2,68,0,0,60,2,0,68,68,8,2,1,60,3,69,2,0,0,0,0,0,7,48,68,0,60,0,0,67,67,68,10,0,0,68,69,7,0,0,0,0,0,71,0,0,0,67,0,12,73,4,73,68,0,28,0,0,0,69,0,0,0,0,43,0,9,69,7,67,0,7,73,0,0,0,66,0,68,0,9,0,43,73,0,0,0,0,0,0,12,0,7,0,0,0,0,0,0,68,48,67,9,3,0,0,0,0,0,0,9,0,0,0,73,43,3,0,0,12,8,0,0,0,0,4,7,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,7.24E2,2.96E2,2.41E2,4.83E2,1.1E1,2.85E2,8.3E1,1.58E2,2.48E2,2.35E2,7E0,4E0,1.03E2,1.82E2,4.9E1,3.4E1,1.41E2,1.7E1,4.3E1,2.05E2,1.05E2,1.3E2,6.6E1,3.7E1,1.13E2,6.9E1,3.2E1,1.7E1,8.6E1,5.5E1,9E0,8E0,2.9E1,1.4E1,1.58E2,4.7E1,6.9E1,3.6E1,1.1E2,2E1,1.6E1,5E1,3.3E1,4E0,9.4E1,1.9E1,9E0,6E1,1.3E1,4E0,7.6E1,1E1,8E0,4.7E1,5E0,4E0,2.2E1,7E0,4E0,1E1,3.9E1,1.19E2,2.5E1,2.2E1,6E1,9E0,3E1,6E0,6.7E1,4.3E1,1.6E1,4E0,6E0,1E1,4.6E1,4E0,1.5E1,1.8E1,8.1E1,1.3E1,4E0,5E0,4.4E1,1.6E1,5.9E1,1.7E1,4E0,6E0,3.8E1,9E0,2E1,1.9E1,9.7E1,2.2E1,1.6E1,9E0,1.4E1,8E0,4.8E1,1.2E1,4E0,5E0,1.7E1,1.3E1,6E0,6.1E1,1.2E1,3.1E1,5E0,1.1E1,5E0,5E0,1.9E1,2.7E1,4.7E1,3.4E1,6E0,7E0,2E1,2.4E1,1E1,6E0,3.1E1,2.8E1,1E1,7E0,3.2E1,6E0,5E0,4E0,1.6E1,4E0,9E0,1E1,3E1,6.7E1,1.8E1,4E0,1.2E1,4E0,4E0,5E0,1E1,4E0,4E0,4E0,4E0,4.4E1,4E0,8E0,2E1,4.1E1,8E0,4E0,2.2E1,9E0,6E0,5E0,6E0,1.3E1,6E0,2.1E1,4E0,4.3E1,5E0,2.9E1,1.5E1,5E0,4E0,2E1,4E0,6E0,4E0,2.4E1,6E0,2.6E1,5E0,1.1E1,5E0,4E0,5E0,5E0,1.5E1,1.5E1,3.3E1,3.4E1,1.2E1,6E0,8E0,4E0,5E0,5E0,5E0,3.9E1,4E0,4E0,6E0,1.4E1,2.9E1,1.2E1,4E0,4E0,1.2E1,1E1,5E0,4E0,9E0,4E0,1.3E1,8E0,2.9E1,1.4E1,1E1,1.9E1,9E0,6E0,2E1,4E0,2.1E1,5E0,6E0,9E0,5E0,1E1,2.3E1,1E1,1.8E1,1.6E1,8E0,4E0,4E0,3.5E1,4E0,1E1,1.3E1,1.6E1,4E0,8E0,4E0,8E0,5E0,5E0,6E0,7E0,4E0,4E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"249","size_leaf_vector":"1"}},{"base_weights":[6.798278E-3,-2.9250383E-1,3.2333016E-1,-5.4509145E-1,-1.3893844E-1,1.4397086E-1,6.3014203E-1,-7.511041E-1,-4.2006698E-1,-3.5289416E-1,-5.0119236E-2,-4.0301904E-2,1.9171517E-1,7.1512866E-1,2.2618681E-2,-9.9959886E-1,-6.9973826E-1,-6.033013E-1,-2.9881284E-1,-4.2399275E-1,-1.2785693E-1,-1.0241604E-1,8.260068E-2,4.0933564E-2,1.591436E-1,6.866623E-2,6.468455E-1,-2.665817E-1,3.099342E-1,-2.5027623E-2,-5.611118E-2,-3.5596162E-2,-5.5265373E-1,-3.6661573E-2,-5.016208E-1,2.8036669E-2,-3.575571E-1,-3.2117665E-1,-5.392631E-1,-1.8218723E-1,5.3145397E-3,-1.884132E-1,-2.8034728E-3,-1.7533407E-1,1.3838987E-1,-9.642109E-2,2.6033372E-1,2.0197384E-1,6.962432E-1,-1.9054212E-2,9.287665E-4,-6.836584E-3,2.5750862E-2,-1.0586381E-2,-2.8907977E-2,-1.20543E-2,-5.250168E-1,-2.9863966E-1,-5.8031213E-1,-3.5235187E-1,-3.9635E-3,-4.5235357E-1,-3.8829327E-2,-9.630789E-2,-2.584358E-1,-2.783935E-1,-3.0009903E-2,-9.539431E-2,5.8326732E-2,-1.5262059E-2,4.978653E-3,9.87916E-2,1.6342035E-2,-1.3543558E-2,-5.288925E-1,2.2214657E-1,2.958916E-2,-1.2322253E-3,3.418286E-1,5.4083705E-2,6.473597E-1,-2.6804253E-2,-1.5329408E-2,-1.705044E-1,-3.7021998E-1,-3.2409858E-2,-1.8558647E-2,-3.7659407E-1,-6.078864E-3,-2.2322569E-2,-1.2192952E-2,-6.845971E-3,-1.3111394E-3,-1.463086E-2,-6.321943E-3,-1.8398893E-1,-3.9660004E-1,9.742137E-2,-1.8942219E-1,2.3122365E-2,-1.529319E-1,-1.7521594E-2,1.932074E-1,1.3263568E-1,-5.8842912E-2,-1.347676E-1,8.889626E-2,-1.2324528E-2,-3.0462394E-2,2.4081929E-1,-1.3385908E-2,2.0845033E-2,6.5702763E-3,4.8914114E-1,7.5950795E-1,-9.660245E-2,-1.6806886E-2,-2.7759965E-3,-4.059909E-1,-2.3605634E-2,-1.4212878E-2,-1.0439892E-1,-2.8454685E-1,-2.4071284E-2,-2.705018E-1,-2.6948992E-3,8.922826E-3,-1.0214229E-2,-3.782071E-3,7.119939E-3,-3.4070218E-3,-3.6927234E-4,-1.9772571E-1,-6.125142E-2,5.6821585E-2,-1.4373868E-4,2.538525E-1,8.0264695E-2,1.068138E-2,-8.271929E-3,3.3786534E-3,8.9172255E-3,-2.3354761E-1,1.17935985E-1,-3.956408E-3,3.507249E-1,1.3423313E-1,5.385398E-1,1.8779878E-1,8.2583904E-1,2.8557053E-1,-1.4705496E-1,1.7037791E-4,-4.6494633E-1,-2.2389881E-1,-7.8066083E-4,-1.5021509E-1,-6.403517E-3,-1.5202354E-2,-1.599685E-2,-2.9976934E-3,-2.386771E-1,-2.7671948E-3,2.7382474E-3,-9.508635E-2,-9.336852E-4,4.521208E-3,4.5469464E-3,1.4413992E-2,3.927989E-2,6.297303E-3,5.585332E-3,-6.672683E-2,-3.1092018E-1,-4.079159E-3,1.4590019E-1,-3.1294962E-4,3.9253423E-1,-1.4641273E-2,8.984514E-2,3.7829068E-1,8.155576E-3,5.6955206E-1,1.1485854E-2,4.1311714E-3,2.4195999E-2,4.1874755E-2,-4.9958057E-3,2.615165E-2,-1.2944418E-3,-9.381672E-3,-1.5504474E-2,-2.4640905E-2,-1.536285E-2,-4.437479E-3,-3.7726064E-3,-1.1252776E-2,-5.464779E-3,-1.3124496E-2,-7.428129E-4,-5.984501E-3,-1.7113466E-4,5.483901E-3,-3.7516936E-4,-5.1737204E-3,-8.49017E-3,-1.7347082E-2,2.1123588E-3,9.338601E-3,1.2624343E-2,2.470133E-2,-6.8272697E-3,4.5620366E-3,-6.8897502E-3,5.6473007E-3,2.1985939E-2,4.9627996E-3,2.7737483E-2,1.5430196E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,-1,-1,53,-1,55,-1,57,59,61,63,-1,65,67,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,81,83,85,87,-1,89,-1,91,93,95,97,99,101,-1,-1,103,-1,105,107,109,-1,-1,111,-1,113,-1,-1,115,117,-1,-1,119,-1,-1,-1,-1,-1,-1,-1,121,123,125,127,129,131,133,135,137,139,141,143,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,153,-1,-1,155,157,-1,159,-1,-1,-1,-1,-1,-1,-1,161,163,165,-1,167,169,-1,-1,-1,171,173,175,-1,177,179,181,183,185,187,189,-1,191,193,-1,195,-1,-1,-1,-1,197,-1,-1,199,-1,-1,-1,-1,201,-1,-1,203,205,-1,207,-1,209,211,213,215,-1,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.4738495E1,1.9861668E1,2.665364E1,4.6812706E0,6.0823965E0,1.5112545E1,9.22583E0,2.1956253E-1,2.5525303E0,1.4497738E0,1.5909684E0,0E0,6.530362E0,7.3060303E0,1.9937334E0,3.26231E-1,2.2984123E-1,5.3050995E-1,4.2604527E0,6.6826916E-1,3.3767235E-1,1.4043611E0,9.596126E-1,0E0,7.3436575E0,0E0,3.0678177E0,5.100673E-1,1.4422456E0,0E0,0E0,0E0,4.2396593E-1,0E0,5.44796E-2,0E0,7.716341E-1,2.6329756E-1,4.5751858E-1,1.06074035E-1,0E0,1.2459252E0,4.4139153E-1,5.622345E-1,4.255196E-1,2.8960526E0,2.8045444E0,5.207529E-1,2.280243E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2728119E-1,4.7802162E-1,2.1276474E-2,1.3776827E-1,0E0,2.9323578E-2,0E0,3.455674E-2,2.7499557E-2,5.519471E-1,6.8881947E-1,2.1493018E-1,4.8421752E-1,0E0,0E0,2.5854942E-1,0E0,8.688341E-1,2.687223E-1,1.8476362E0,0E0,0E0,1.5415096E-1,0E0,1.7755051E0,0E0,0E0,2.933979E-1,3.8607264E-1,0E0,0E0,1.299963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.34406E-1,2.3564172E-1,2.9452467E-1,3.2994866E-2,1.5094508E-1,1.3475919E-1,1.04261845E-1,2.0318782E-1,1.8276405E-1,1.5587352E-1,4.5626467E-1,2.027004E-1,0E0,0E0,2.0615168E0,0E0,0E0,0E0,6.8781567E-1,1.9333076E0,9.162052E-2,0E0,0E0,2.501645E-1,0E0,0E0,7.7641055E-2,4.5341134E-2,0E0,1.9560897E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.6012194E-2,8.719295E-2,3.9916288E-2,0E0,8.678162E-2,5.6662843E-2,0E0,0E0,0E0,1.2665567E-1,1.860795E-1,1.1607301E-1,0E0,1.3500109E0,9.7350204E-1,4.0465736E-1,3.161651E-2,9.8553467E-1,1.1655376E0,7.4324995E-2,0E0,3.338909E-2,1.0694945E-1,0E0,6.331104E-2,0E0,0E0,0E0,0E0,3.1996906E-2,0E0,0E0,4.1467562E-2,0E0,0E0,0E0,0E0,5.8475614E-2,0E0,0E0,2.2963595E-2,2.9507041E-2,0E0,1.4286971E-1,0E0,1.1932125E0,1.6555646E-1,6.3753015E-1,3.1901288E-1,0E0,9.731102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,26,26,27,27,28,28,32,32,34,34,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,56,56,57,57,58,58,59,59,61,61,63,63,64,64,65,65,66,66,67,67,68,68,71,71,73,73,74,74,75,75,78,78,80,80,83,83,84,84,87,87,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,109,109,113,113,114,114,115,115,118,118,121,121,122,122,124,124,132,132,133,133,134,134,136,136,137,137,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,150,150,151,151,153,153,154,154,156,156,161,161,164,164,169,169,172,172,173,173,175,175,177,177,178,178,179,179,180,180,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,-1,-1,54,-1,56,-1,58,60,62,64,-1,66,68,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,82,84,86,88,-1,90,-1,92,94,96,98,100,102,-1,-1,104,-1,106,108,110,-1,-1,112,-1,114,-1,-1,116,118,-1,-1,120,-1,-1,-1,-1,-1,-1,-1,122,124,126,128,130,132,134,136,138,140,142,144,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,154,-1,-1,156,158,-1,160,-1,-1,-1,-1,-1,-1,-1,162,164,166,-1,168,170,-1,-1,-1,172,174,176,-1,178,180,182,184,186,188,190,-1,192,194,-1,196,-1,-1,-1,-1,198,-1,-1,200,-1,-1,-1,-1,202,-1,-1,204,206,-1,208,-1,210,212,214,216,-1,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,4.27E3,3.557841E7,5.02E2,1E0,2.856934E7,2E0,3.06E2,9.860918E5,2.0766992E6,-4.0301904E-2,1.2E1,3.6E1,1.3E1,1.5E1,3.62E2,3.89E2,2E0,4.325E0,4.0705118E6,1.559733E6,5.8899284E1,4.0933564E-2,4.8709216E5,6.866623E-2,3.257507E5,4.9236734E5,2.0618556E-2,-2.5027623E-2,-5.611118E-2,-3.5596162E-2,1.0005E5,-3.6661573E-2,6.695E3,2.8036669E-2,1.9E1,4.67E2,7.668863E6,3.429012E7,5.3145397E-3,3.4734247E2,4.8709216E5,1.7010834E3,7.168071E6,2.3E1,5.405423E6,1.8452264E6,1.0918E4,-1.9054212E-2,9.287665E-4,-6.836584E-3,2.5750862E-2,-1.0586381E-2,-2.8907977E-2,-1.20543E-2,3.201807E6,3.464E0,6.81531E-2,2.241E2,-3.9635E-3,1.9637654E2,-3.8829327E-2,1.0865825E0,6.7723384E7,3.6981132E0,1.798E3,2.812736E5,9.24E2,-1.5262059E-2,4.978653E-3,2.8302418E10,1.6342035E-2,3.6161574E4,4E1,1.6776881E7,2.958916E-2,-1.2322253E-3,1.8996E4,5.4083705E-2,7.098E3,-2.6804253E-2,-1.5329408E-2,1.1769393E5,2.3636363E0,-3.2409858E-2,-1.8558647E-2,1.1967312E3,-6.078864E-3,-2.2322569E-2,-1.2192952E-2,-6.845971E-3,-1.3111394E-3,-1.463086E-2,-6.321943E-3,1E0,6.176636E5,8.23E2,1.835821E1,2.0939393E0,1.5599597E8,2.9427118E2,5.1E1,5.1586456E7,1E1,3.82E2,4.2904656E8,-1.2324528E-2,-3.0462394E-2,3.464E0,-1.3385908E-2,2.0845033E-2,6.5702763E-3,4.3657026E0,1.9500381E2,1.3009709E0,-1.6806886E-2,-2.7759965E-3,3.53274E2,-2.3605634E-2,-1.4212878E-2,2.5454545E0,2.4645097E2,-2.4071284E-2,1.0563E4,-2.6948992E-3,8.922826E-3,-1.0214229E-2,-3.782071E-3,7.119939E-3,-3.4070218E-3,-3.6927234E-4,3.990487E5,1.2051282E0,7.24E2,-1.4373868E-4,1.977157E7,3.235955E0,1.068138E-2,-8.271929E-3,3.3786534E-3,5.513889E0,3.3238492E2,3.687715E6,-3.956408E-3,6.929741E8,8.5023944E2,1E0,4.75356E6,1.3543621E6,9.132011E6,1E0,1.7037791E-4,8.91E2,2.0939393E0,-7.8066083E-4,1.7755102E0,-6.403517E-3,-1.5202354E-2,-1.599685E-2,-2.9976934E-3,6.789622E7,-2.7671948E-3,2.7382474E-3,1.98E2,-9.336852E-4,4.521208E-3,4.5469464E-3,1.4413992E-2,1.0404834E7,6.297303E-3,5.585332E-3,1.4E1,2.35184E5,-4.079159E-3,3.53274E2,-3.1294962E-4,5.372237E2,5E0,2.1924414E2,1.2979348E3,8.155576E-3,1.766486E7,1.1485854E-2,4.1311714E-3,2.4195999E-2,4.1874755E-2,-4.9958057E-3,2.615165E-2,-1.2944418E-3,-9.381672E-3,-1.5504474E-2,-2.4640905E-2,-1.536285E-2,-4.437479E-3,-3.7726064E-3,-1.1252776E-2,-5.464779E-3,-1.3124496E-2,-7.428129E-4,-5.984501E-3,-1.7113466E-4,5.483901E-3,-3.7516936E-4,-5.1737204E-3,-8.49017E-3,-1.7347082E-2,2.1123588E-3,9.338601E-3,1.2624343E-2,2.470133E-2,-6.8272697E-3,4.5620366E-3,-6.8897502E-3,5.6473007E-3,2.1985939E-2,4.9627996E-3,2.7737483E-2,1.5430196E-2],"split_indices":[2,43,2,7,2,17,60,32,2,43,43,0,33,0,3,3,1,1,10,69,43,9,62,0,43,0,43,43,72,0,0,0,5,0,9,0,3,2,5,7,0,67,43,48,43,3,43,60,9,0,0,0,0,0,0,0,9,69,53,4,0,67,0,71,7,71,44,43,2,0,0,46,0,48,3,62,0,0,44,0,2,0,0,43,73,0,0,48,0,0,0,0,0,0,0,26,43,2,71,68,46,67,10,7,8,0,7,0,0,69,0,0,0,69,73,68,0,0,4,0,0,73,4,0,44,0,0,0,0,0,0,0,43,68,2,0,5,71,0,0,0,69,67,62,0,5,67,15,62,43,1,29,0,44,68,0,68,0,0,0,0,7,0,0,10,0,0,0,0,60,0,0,3,1,0,4,0,4,6,67,4,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,5.13E2,4.85E2,1.93E2,3.2E2,3.07E2,1.78E2,7.1E1,1.22E2,9.3E1,2.27E2,1.3E1,2.94E2,1.56E2,2.2E1,9E0,6.2E1,4.7E1,7.5E1,7E1,2.3E1,1.63E2,6.4E1,1.2E1,2.82E2,1.1E1,1.45E2,1.1E1,1.1E1,4E0,5E0,3.9E1,2.3E1,1.4E1,3.3E1,4E0,7.1E1,3.9E1,3.1E1,1.9E1,4E0,8.7E1,7.6E1,1.1E1,5.3E1,8E1,2.02E2,1.5E1,1.3E2,7E0,4E0,4E0,7E0,5E0,1.8E1,4E0,2.9E1,5.8E1,1.3E1,3.4E1,5E0,2.6E1,5E0,1E1,9E0,5.5E1,3.2E1,3E1,4.6E1,7E0,4E0,4.6E1,7E0,6.8E1,1.2E1,1.85E2,1.7E1,6E0,9E0,1E1,1.2E2,2.1E1,8E0,2.2E1,3.6E1,6E0,7E0,3E1,4E0,2.1E1,5E0,5E0,5E0,5E0,4E0,3.2E1,2.3E1,1.8E1,1.4E1,1E1,2E1,3E1,1.6E1,3.8E1,8E0,3.1E1,3.7E1,5E0,7E0,1.79E2,6E0,5E0,4E0,5.2E1,6.8E1,1.7E1,5E0,4E0,3.2E1,8E0,2.2E1,1.9E1,1.3E1,1E1,1.3E1,7E0,1.1E1,1E1,4E0,4E0,6E0,5E0,1.5E1,1.9E1,1.1E1,4E0,1.2E1,2.6E1,1.2E1,4E0,4E0,1.3E1,1.8E1,3.2E1,5E0,8.7E1,9.2E1,4.4E1,8E0,5.9E1,9E0,1.1E1,6E0,2.3E1,9E0,7E0,1.2E1,4E0,9E0,9E0,4E0,1.1E1,4E0,4E0,1.5E1,4E0,7E0,4E0,8E0,1.6E1,1E1,5E0,8E0,1.1E1,7E0,2.6E1,6E0,7.8E1,9E0,7.9E1,1.3E1,4E0,4E1,4E0,4E0,1.4E1,4.5E1,4E0,5E0,4E0,7E0,1E1,1.3E1,4E0,5E0,8E0,4E0,4E0,7E0,5E0,1E1,1.1E1,5E0,4E0,4E0,5E0,6E0,1E1,1.6E1,4.4E1,3.4E1,4E0,5E0,9E0,7E1,9E0,4E0,3.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-9.604084E-3,-1.7769752E-1,5.3377515E-1,-5.027555E-1,-1.1044416E-2,5.953061E-1,-5.064273E-1,-6.957941E-1,-3.919236E-1,-1.5918878E-1,1.365706E-1,4.4329765E-1,8.657004E-1,-3.2116305E-2,-1.0088568E-2,-6.123844E-1,-8.735108E-1,-4.9474064E-1,-9.532652E-2,-2.2376442E-1,8.322249E-2,2.804698E-1,2.5025155E-2,1.8223284E-1,6.0022587E-1,1.1787167E0,6.684661E-1,-3.4130882E-2,-3.930898E-1,-5.0244346E-2,-6.9417316E-1,-4.3455964E-1,-9.8442465E-1,3.1324722E-2,-2.0894806E-1,-2.0233789E-1,-4.786559E-2,3.7036147E-2,3.1517094E-1,3.03033E-1,-7.965279E-3,4.1885342E-2,-2.1352336E-2,3.462123E-2,2.964725E-1,4.8907918E-1,8.248178E-1,1.2883264E0,3.3090036E-2,7.35882E-1,2.6460323E-1,-5.182883E-1,1.2078336E-2,-1.9782528E-2,-3.65654E-2,-2.3568341E-1,-4.9656954E-1,-1.9125406E-2,-1.1919587E0,-1.1826896E-1,-1.704303E-2,-3.6924347E-1,-1.2643513E-1,-6.714747E-3,5.7381816E-2,1.921139E-2,6.9976863E-3,2.2981884E-1,4.318533E-1,-1.08133584E-1,8.7874055E-2,-8.603366E-3,8.323961E-2,1.94815E-1,1.9868504E-2,5.381012E-1,2.0396389E-1,4.410762E-2,2.7326044E-2,3.589514E-2,6.4087585E-2,8.632517E-1,5.22856E-1,1.7521976E-2,4.4819764E-3,-2.7962103E-2,-3.1575984E-1,-3.2417864E-1,1.1731296E-2,-5.764673E-1,-3.855013E-1,-6.996596E-2,-2.9155284E-2,-3.753598E-2,-2.3228638E-1,-1.3757024E-3,-4.1190678E-1,-1.0416744E-1,-2.2085348E-2,1.3092345E-1,2.4690751E-2,1.690638E-1,2.521731E-2,4.6240836E-1,6.361989E-3,-1.82799E-1,8.934329E-2,-1.0108118E-1,1.2350115E-1,1.4519924E-1,-3.2465826E-3,2.4155864E-1,3.3954915E-3,6.183139E-1,2.883272E-1,1.7264051E-3,1.3574328E-2,2.0680983E-2,9.18468E-1,3.2642597E-1,2.93844E-2,-1.8470675E-2,-7.7873715E-3,-4.2645237E-1,-2.4735157E-3,-4.7742063E-1,-3.3033323E-2,-2.4655715E-1,-4.4549042E-1,2.4755226E-2,-9.565652E-2,-1.5569443E-2,-5.742614E-3,-2.6252744E-1,-4.7748247E-1,-6.2056348E-2,-2.1061331E-1,3.3671854E-3,8.615587E-3,-2.0534683E-2,9.800891E-2,-3.6841254E-3,1.8792307E-1,9.079898E-3,5.0147074E-1,2.8283142E-3,-2.468648E-1,-2.4922558E-3,8.956172E-3,-1.089296E-2,6.0731443E-3,8.775925E-2,3.5744572E-1,-3.2624108E-4,2.1505071E-1,4.6179355E-3,1.4281034E-2,8.376324E-3,6.618336E-1,7.028194E-3,1.7947633E-2,4.588095E-2,2.1530917E-2,2.0102886E-2,7.0421374E-3,-2.2482986E-2,-7.973739E-3,-2.3983054E-2,-1.0031399E-2,-1.3448424E-2,-4.9971286E-3,-9.1125015E-3,-4.816604E-1,-1.1962886E-3,3.2548907E-3,-6.571131E-3,-1.3835018E-3,-3.20505E-1,-5.4215016E-3,-1.1348064E-2,-5.432325E-1,-1.04204014E-1,4.9513765E-2,-1.1366056E-1,-1.2545401E-2,-4.2906925E-3,2.5561439E-2,6.4209225E-3,5.9352885E-4,1.505344E-1,3.1558603E-1,2.4542887E-2,1.1656909E-2,-1.3030397E-2,-3.9224033E-3,5.7654423E-3,-3.6772718E-3,1.1004653E-1,-6.1053853E-3,2.3943515E-2,9.805783E-3,4.3220334E-3,1.2525193E-2,2.3565728E-2,3.500608E-2,-2.5173325E-2,-1.2396623E-2,-1.926584E-2,-9.612037E-3,-3.0312736E-2,-1.771572E-2,-2.2826523E-3,-9.571008E-3,9.623021E-3,-7.610982E-4,-9.670485E-3,-7.391419E-4,3.0173867E-3,-1.4645348E-3,7.753292E-3,3.3735766E-4,2.1841483E-2,6.2726284E-3,1.3855398E-2,3.977356E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,53,55,57,-1,59,61,-1,63,65,67,-1,69,-1,71,73,75,77,79,-1,81,83,85,-1,-1,-1,87,89,-1,91,93,-1,95,97,-1,99,-1,-1,101,103,105,107,-1,109,111,-1,113,115,-1,-1,-1,-1,117,119,-1,-1,-1,121,123,-1,125,127,-1,-1,129,131,-1,133,135,-1,137,139,141,-1,143,-1,145,147,149,151,153,-1,155,-1,157,159,-1,-1,-1,161,163,-1,-1,-1,165,-1,167,-1,169,171,173,175,-1,-1,177,179,181,183,-1,-1,185,187,-1,189,-1,191,-1,193,-1,-1,-1,195,197,199,-1,201,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,205,-1,-1,-1,-1,207,-1,-1,209,211,213,215,-1,-1,217,-1,-1,219,221,-1,-1,-1,-1,-1,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.316876E1,4.2194374E1,1.5726601E1,5.3755417E0,1.13058E1,8.987541E0,6.0936093E-1,8.73291E-1,5.1370296E0,4.0573187E0,4.1502686E0,5.953951E0,4.2128067E0,0E0,0E0,1.5943947E0,3.198948E-1,3.2853851E0,4.0958652E0,3.3409758E0,5.7753414E-1,1.2069321E0,1.2268742E0,9.509324E-1,1.8931618E0,5.979614E-1,1.221077E0,0E0,2.3480525E0,0E0,1.2691975E-1,1.2888489E0,1.0455866E0,0E0,5.365578E-1,2.487033E0,0E0,1.8274027E-1,8.533102E-2,9.193506E-1,0E0,9.960845E-1,0E0,2.9466662E-1,3.5079265E-1,7.8477E-1,3.1436348E-1,8.51326E-2,0E0,7.424984E-1,1.3615894E-1,2.1397972E-1,0E0,0E0,0E0,1.3388417E0,5.4094315E-1,0E0,6.721506E-1,2.3981497E-1,0E0,8.8566494E-1,1.0385175E0,0E0,1.00146025E-1,0E0,0E0,1.2709115E0,2.7785206E-1,5.1765144E-1,7.5299567E-1,0E0,2.1945965E-1,9.566301E-2,0E0,9.036169E-1,1.5369049E-1,0E0,0E0,0E0,0E0,1.8549728E-1,2.1479893E-1,0E0,0E0,0E0,4.395187E-2,6.69147E-1,0E0,3.1735516E-1,2.1523142E-1,0E0,0E0,6.3914776E-2,8.2500994E-2,0E0,4.1340733E-1,5.786269E-1,0E0,2.8332159E-2,1.04803786E-1,3.0121863E-1,0E0,2.5454903E-1,0E0,4.0594524E-1,1.6022004E-1,2.6124227E-1,7.4626195E-1,1.71689E-1,0E0,1.0303539E-1,0E0,6.407356E-1,1.4552057E-1,0E0,0E0,0E0,2.8013992E-1,1.039083E-1,0E0,0E0,0E0,1.9109583E-1,0E0,2.2378206E-1,0E0,5.480963E-2,1.3845062E-1,2.2655735E-2,2.3286693E-2,0E0,0E0,1.22310996E-1,4.3059254E-1,4.5659184E-1,1.8773448E-1,0E0,0E0,6.906527E-2,4.0088408E-2,0E0,2.2456312E-1,0E0,8.106756E-2,0E0,9.0023994E-2,0E0,0E0,0E0,1.2181306E-1,4.1644692E-1,1.7020512E-1,0E0,4.9563766E-2,0E0,0E0,0E0,1.3730907E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.505351E-1,0E0,0E0,0E0,0E0,5.4321647E-2,0E0,0E0,2.5345707E-1,3.8470095E-1,2.8632823E-1,1.4613031E-1,0E0,0E0,3.1661548E-2,0E0,0E0,1.1268771E-1,2.7497137E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.1528503E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,34,34,35,35,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,55,55,56,56,58,58,59,59,61,61,62,62,64,64,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,81,81,82,82,86,86,87,87,89,89,90,90,93,93,94,94,96,96,97,97,99,99,100,100,101,101,103,103,105,105,106,106,107,107,108,108,109,109,111,111,113,113,114,114,118,118,119,119,123,123,125,125,127,127,128,128,129,129,130,130,133,133,134,134,135,135,136,136,139,139,140,140,142,142,144,144,146,146,150,150,151,151,152,152,154,154,158,158,172,172,177,177,180,180,181,181,182,182,183,183,186,186,189,189,190,190,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,54,56,58,-1,60,62,-1,64,66,68,-1,70,-1,72,74,76,78,80,-1,82,84,86,-1,-1,-1,88,90,-1,92,94,-1,96,98,-1,100,-1,-1,102,104,106,108,-1,110,112,-1,114,116,-1,-1,-1,-1,118,120,-1,-1,-1,122,124,-1,126,128,-1,-1,130,132,-1,134,136,-1,138,140,142,-1,144,-1,146,148,150,152,154,-1,156,-1,158,160,-1,-1,-1,162,164,-1,-1,-1,166,-1,168,-1,170,172,174,176,-1,-1,178,180,182,184,-1,-1,186,188,-1,190,-1,192,-1,194,-1,-1,-1,196,198,200,-1,202,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,206,-1,-1,-1,-1,208,-1,-1,210,212,214,216,-1,-1,218,-1,-1,220,222,-1,-1,-1,-1,-1,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.2116136E5,4.217427E7,9.9E1,9.31E2,1.5142261E3,1.0201538E-1,1.4504054E8,8.57E2,2.331083E6,3.21875E0,9.155886E5,2.3601625E1,-3.2116305E-2,-1.0088568E-2,3.89E2,3.3817584E7,9.932432E0,2.04115E5,1.1290322E1,7.168071E6,1.4676277E7,2.673585E1,4.032683E6,4.391553E6,1.4595818E7,2.856934E7,-3.4130882E-2,1.1634076E4,-5.0244346E-2,4.217427E7,1.3208092E0,2.49E2,3.1324722E-2,1.6942337E1,1.7934177E2,-4.786559E-2,3.72E2,3.52E2,5.144203E2,-7.965279E-3,1.221E3,-2.1352336E-2,8.295515E7,1.29443414E5,2E0,1.4598765E0,1E1,3.3090036E-2,2.171312E0,2.2365898E3,1.9363927E-6,1.2078336E-2,-1.9782528E-2,-3.65654E-2,6.47E2,1.339646E6,-1.9125406E-2,6.8E1,4.3E1,-1.704303E-2,9E0,1.7479E4,-6.714747E-3,1.2E1,1.921139E-2,6.9976863E-3,6.0052995E6,2E0,1.5115256E6,2.4080281E2,-8.603366E-3,5.8485405E1,6.364486E0,1.9868504E-2,2E0,8E0,4.410762E-2,2.7326044E-2,3.589514E-2,6.4087585E-2,1.8697667E3,7.958E3,1.7521976E-2,4.4819764E-3,-2.7962103E-2,5.5E1,1.302273E6,1.1731296E-2,1.88E2,9.263158E0,-6.996596E-2,-2.9155284E-2,4.046E3,5.2767E4,-1.3757024E-3,3.2E1,2.1851852E0,-2.2085348E-2,1.1E1,2.7840833E2,3.7760816E5,2.521731E-2,2.9816E4,6.361989E-3,1.9095E4,4.7563504E7,2.0227273E0,2.295413E6,1.8471494E7,-3.2465826E-3,1.0784811E3,3.3954915E-3,2.0750147E8,8.5023944E2,1.7264051E-3,1.3574328E-2,2.0680983E-2,2.5842668E7,2E0,2.93844E-2,-1.8470675E-2,-7.7873715E-3,6.666667E-1,-2.4735157E-3,2.7976523E2,-3.3033323E-2,4.5087484E5,3.9961785E-2,2.7777777E0,1.1770261E8,-1.5569443E-2,-5.742614E-3,1E0,1.1010101E0,5E0,9.538462E0,3.3671854E-3,8.615587E-3,1.3071136E3,1E0,-3.6841254E-3,7.6569915E-1,9.079898E-3,1.8809018E0,2.8283142E-3,4.88E2,-2.4922558E-3,8.956172E-3,-1.089296E-2,1.1631579E1,9.639872E0,1.4E1,-3.2624108E-4,2.848973E6,4.6179355E-3,1.4281034E-2,8.376324E-3,6.255748E6,7.028194E-3,1.7947633E-2,4.588095E-2,2.1530917E-2,2.0102886E-2,7.0421374E-3,-2.2482986E-2,-7.973739E-3,-2.3983054E-2,-1.0031399E-2,-1.3448424E-2,-4.9971286E-3,-9.1125015E-3,1.5292561E2,-1.1962886E-3,3.2548907E-3,-6.571131E-3,-1.3835018E-3,8.3E1,-5.4215016E-3,-1.1348064E-2,4.795239E7,1E0,2.3786941E4,5.0857143E0,-1.2545401E-2,-4.2906925E-3,1.880118E-3,6.4209225E-3,5.9352885E-4,1.8E1,1.124641E0,2.4542887E-2,1.1656909E-2,-1.3030397E-2,-3.9224033E-3,5.7654423E-3,-3.6772718E-3,6.4722223E0,-6.1053853E-3,2.3943515E-2,9.805783E-3,4.3220334E-3,1.2525193E-2,2.3565728E-2,3.500608E-2,-2.5173325E-2,-1.2396623E-2,-1.926584E-2,-9.612037E-3,-3.0312736E-2,-1.771572E-2,-2.2826523E-3,-9.571008E-3,9.623021E-3,-7.610982E-4,-9.670485E-3,-7.391419E-4,3.0173867E-3,-1.4645348E-3,7.753292E-3,3.3735766E-4,2.1841483E-2,6.2726284E-3,1.3855398E-2,3.977356E-3],"split_indices":[2,43,60,44,2,67,53,46,2,43,69,43,71,0,0,1,7,69,5,69,43,62,68,60,43,62,60,0,43,0,60,68,2,0,71,70,0,44,0,4,0,2,0,7,48,6,69,3,0,57,67,52,0,0,0,2,9,0,0,8,0,3,44,0,3,0,0,43,6,43,67,0,73,69,0,32,8,0,0,0,0,67,2,0,0,0,0,9,0,10,73,0,0,44,1,0,10,68,0,3,67,43,0,1,0,12,7,68,43,5,0,4,0,7,67,0,0,0,62,32,0,0,0,71,0,4,0,60,53,71,7,0,0,8,68,8,73,0,0,48,26,0,57,0,57,0,4,0,0,0,73,69,3,0,66,0,0,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,67,0,0,0,0,12,0,0,7,26,62,69,0,0,53,0,0,3,53,0,0,0,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.78E2,2.4E2,2.63E2,5.15E2,2.27E2,1.3E1,9.4E1,1.69E2,2.57E2,2.58E2,1.47E2,8E1,7E0,6E0,6.7E1,2.7E1,1.25E2,4.4E1,2.03E2,5.4E1,1.12E2,1.46E2,5.6E1,9.1E1,2.9E1,5.1E1,4.1E1,2.6E1,1E1,1.7E1,1.13E2,1.2E1,5E0,3.9E1,1.99E2,4E0,4.6E1,8E0,1.07E2,5E0,1.42E2,4E0,2.5E1,3.1E1,6.3E1,2.8E1,2.2E1,7E0,4.3E1,8E0,2.2E1,4E0,6E0,1.1E1,2.8E1,8.5E1,4E0,8E0,2.6E1,1.3E1,6.1E1,1.38E2,4E0,4.2E1,4E0,4E0,7E1,3.7E1,3.3E1,1.09E2,4E0,2.1E1,1.9E1,1.2E1,5.3E1,1E1,1.6E1,1.2E1,5E0,1.7E1,2.5E1,1.8E1,4E0,4E0,1.4E1,8E0,2.4E1,4E0,4.7E1,3.8E1,4E0,4E0,1.6E1,1E1,7E0,5.4E1,1.31E2,7E0,1.2E1,3E1,6E1,1E1,3.3E1,4E0,2.4E1,9E0,1.7E1,9.2E1,1.5E1,6E0,1.3E1,6E0,3.9E1,1.4E1,4E0,6E0,4E0,2.1E1,8E0,1E1,4E0,4E0,1.7E1,7E0,3E1,1.7E1,1.3E1,2.5E1,8E0,8E0,4E0,6E0,1.8E1,3.6E1,9.5E1,3.6E1,7E0,5E0,1.9E1,1.1E1,4E0,5.6E1,5E0,2.8E1,5E0,1.9E1,4E0,5E0,7E0,1E1,8.1E1,1.1E1,5E0,1E1,5E0,8E0,4E0,3.5E1,7E0,7E0,1.7E1,4E0,4E0,4E0,1.3E1,4E0,2.5E1,5E0,9E0,4E0,4E0,2.1E1,4E0,4E0,4E0,4E0,1.2E1,6E0,9E0,2.7E1,6.9E1,2.6E1,1.5E1,2.1E1,7E0,1.2E1,7E0,4E0,4.5E1,1.1E1,2.4E1,4E0,1.5E1,4E0,4E0,6E0,7.4E1,7E0,4E0,7E0,4E0,6E0,1.6E1,1.9E1,1.5E1,6E0,5E0,7E0,1.4E1,1.3E1,4.6E1,2.3E1,7E0,1.9E1,7E0,8E0,7E0,5E0,4E1,5E0,5E0,6E0,7E0,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[6.180792E-3,-1.6503148E-1,4.7275862E-1,-5.0639033E-1,-1.129815E-2,-6.803958E-1,5.4719776E-1,1.8274754E-2,-5.280954E-1,-1.6625014E-1,1.1087337E-1,-3.9352454E-2,-1.7605912E-2,1.3548586E0,4.8434323E-1,-6.065767E-1,-3.0085424E-1,-1.2424855E-1,-5.163365E-1,4.6761047E-2,2.686597E-1,4.5754105E-2,7.792276E-2,3.3147123E-1,7.643023E-1,-5.4452336E-1,-1.0120652E0,-3.3467343E-1,6.379688E-3,-1.7157865E-1,6.810231E-2,-5.913036E-1,-5.229772E-3,8.2113095E-2,-2.3808844E-1,2.206505E-1,2.2615165E-2,1.6233137E-1,4.803148E-1,7.994446E-1,2.2713481E-3,-6.016546E-1,-3.5165614E-1,-2.0412194E-2,-1.2107033E0,-4.5247874E-1,-2.5665715E-1,-3.8528496E-1,-1.127225E-1,1.734763E-1,1.8517341E-2,-6.655063E-1,-1.4194483E-2,1.07354075E-1,-1.8962681E-1,-2.392149E-2,-8.3356254E-2,1.07245244E-1,3.375724E-1,3.3497408E-1,5.235329E-2,5.8765614E-1,2.4847795E-1,5.636321E-1,9.4491065E-1,-6.7435145E-1,-4.8269066E-1,-4.9582702E-1,-8.968568E-2,-7.161074E-2,-3.326427E-2,-2.2630611E-2,-1.0853086E-2,-3.0224049E-3,-3.0056164E-1,-3.0194435E-1,-5.634421E-1,-7.476078E-2,-2.0266244E-1,1.4698161E-3,1.1921103E-2,-6.736173E-3,5.0641637E-2,-2.244549E-2,-4.2857196E-2,6.4812E-3,1.5728682E-1,-2.97582E-1,1.8431344E-3,-1.4241126E-2,1.729825E-2,-2.1520426E-2,1.6255073E-1,3.9851704E-1,1.5145347E-1,2.513066E-2,2.1373E-1,-3.0838272E-1,1.3425767E-1,4.5744762E-1,8.594612E-1,3.4686705E-1,-7.858146E-3,1.4932823E-2,6.2763226E-1,4.9509685E-2,3.5136476E-2,-3.2075506E-2,-1.3317501E-2,-8.192813E-4,-5.5199087E-1,-1.1377268E-2,-2.473519E-2,-8.901832E-3,1.886736E-3,-3.4619373E-1,-6.3525126E-3,-2.2607056E-2,-2.3402435E-1,-3.4160864E-2,-1.4916765E-2,-1.6875029E-1,-2.025404E-2,-8.9234844E-2,-3.0028555E-1,1.2991291E-1,-1.0866737E-2,4.3241512E-2,-1.776235E-2,-7.973841E-3,2.1443602E-1,-7.090227E-3,-1.8114451E-2,6.0518007E-3,-5.462109E-3,4.420927E-3,-9.4596185E-3,2.274301E-1,5.3403392E-2,2.3255488E-1,4.7733608E-1,1.9099436E-3,1.1702081E-2,2.999682E-1,1.1590164E-1,-3.993294E-3,-2.1651287E-2,3.341527E-1,2.1458121E-2,7.0477296E-3,5.174998E-1,4.65909E-2,2.6137244E-2,2.1844682E-1,2.5887595E-2,3.205296E-2,1.3973525E-2,-5.954478E-1,-5.945938E-3,-2.4844922E-1,-1.8800398E-2,-5.5004717E-3,-1.2569496E-2,-7.12295E-4,-1.8868068E-1,-8.481989E-2,2.828785E-2,-1.4959584E-1,5.0678095E-3,-3.65702E-1,-7.2277533E-3,8.353372E-3,1.5093553E-3,-4.2204354E-3,1.7792116E-3,-4.433357E-2,1.2921506E-1,-8.109424E-2,7.055284E-2,2.3125322E-1,-4.505623E-3,1.7135364E-1,1.4623275E-2,4.7639236E-3,-2.4074948E-4,4.0995292E-3,1.6568195E-2,1.2153308E-2,2.5447248E-2,1.6693758E-2,7.683984E-3,-3.5270027E-6,9.820194E-3,2.3110531E-1,2.1871414E-2,-9.265563E-3,7.094084E-2,5.201964E-3,5.694655E-1,3.5538576E-3,2.592104E-1,-2.043926E-2,-3.1079069E-2,-5.291665E-3,-1.4257282E-2,-1.2106815E-2,-4.519799E-3,1.220963E-3,-7.2071347E-3,5.0935005E-3,-8.90217E-4,-9.215974E-3,-1.9180061E-3,-1.2051613E-2,-2.2234317E-2,-4.1192146E-3,4.457565E-3,9.330464E-3,2.3287753E-4,5.383695E-4,-7.138954E-3,-2.3208114E-3,5.4996517E-3,3.7311022E-3,1.22307185E-2,4.3693925E-3,9.825735E-3,1.3281742E-2,5.4156007E-3,-3.6526588E-3,5.0203376E-3,2.8532118E-2,1.5245063E-2,7.564245E-3,1.6637865E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,-1,47,49,51,-1,53,55,57,-1,59,61,63,-1,65,67,-1,69,71,73,75,77,79,81,83,-1,85,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,115,117,119,121,123,-1,-1,-1,125,-1,-1,127,129,131,-1,-1,133,135,137,139,141,-1,143,145,147,149,151,153,-1,-1,155,-1,-1,-1,-1,-1,157,-1,-1,-1,-1,159,-1,-1,161,-1,-1,163,165,167,169,171,173,175,-1,177,179,-1,-1,-1,-1,-1,-1,181,183,185,187,-1,-1,189,191,-1,-1,193,195,-1,197,-1,-1,199,-1,-1,-1,201,-1,203,-1,-1,-1,-1,205,207,209,211,-1,213,-1,-1,-1,-1,-1,215,217,219,221,223,-1,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,227,-1,-1,229,-1,231,-1,233,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.1240005E1,3.9038086E1,2.3806591E1,4.731186E0,9.748823E0,5.475435E-1,1.2390434E1,0E0,3.84803E0,3.273695E0,2.9008594E0,0E0,0E0,5.0655365E-1,1.0018154E1,3.65094E0,9.3690205E-1,1.8688982E0,6.5838957E-1,2.093973E0,7.720475E-1,0E0,0E0,3.8745651E0,2.0717506E0,1.408184E0,1.8437462E0,3.9854956E-1,0E0,2.019351E0,2.1101129E-1,2.170825E-1,0E0,1.2824098E0,9.5225835E-1,8.9196634E-1,0E0,1.4064816E0,1.9194889E0,2.2116356E0,0E0,6.4310455E-1,1.315198E0,0E0,1.3815918E0,5.5664062E-2,2.825966E-1,3.4954214E-1,4.287045E-1,1.4751896E-1,1.604698E-1,2.5580263E-1,0E0,8.502102E-1,4.0832222E-1,0E0,3.6752677E-1,2.6699176E-1,3.2530355E-1,6.1856604E-1,1.4300581E0,1.5887585E0,1.2121193E0,3.048668E-1,3.0662918E-1,3.156147E-1,1.4724045E0,9.531975E-2,1.9608359E-1,0E0,0E0,0E0,0E0,0E0,1.681776E-1,2.1640873E-1,1.8460774E-1,4.7302693E-1,3.9398372E-1,0E0,0E0,0E0,1.23307906E-1,0E0,0E0,8.358092E-1,1.0689397E0,8.964944E-2,0E0,0E0,2.0039521E-1,2.773286E-1,1.7197114E-1,2.1601772E-1,1.0167009E-1,0E0,1.3396406E-1,2.7963167E-1,8.722045E-1,6.5344334E-1,1.7351437E-1,5.0814795E-1,0E0,0E0,3.2805347E-1,0E0,0E0,0E0,0E0,0E0,6.591568E-1,0E0,0E0,0E0,0E0,2.9530048E-2,0E0,0E0,5.946493E-2,0E0,0E0,1.0099387E-1,1.9044825E-1,2.445545E-1,1.3097334E-1,4.8538357E-2,6.313289E-2,4.04773E-1,0E0,1.7797852E-1,4.6324515E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.5702178E-2,3.2590926E-2,1.4949113E-1,1.0702038E-1,0E0,0E0,2.1925747E-2,1.2333231E-1,0E0,0E0,9.150219E-2,2.9986197E-1,0E0,6.3022995E-1,0E0,0E0,7.236326E-2,0E0,0E0,0E0,1.2808132E-1,0E0,5.3262115E-2,0E0,0E0,0E0,0E0,1.7434561E-1,2.100132E-1,1.3855205E-1,7.259676E-2,0E0,3.1965733E-2,0E0,0E0,0E0,0E0,0E0,1.790477E-1,2.4288192E-1,1.1077051E-1,9.3834616E-2,3.8177347E-1,0E0,1.796195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.051114E-2,0E0,0E0,1.3309339E-1,0E0,1.3042927E-1,0E0,5.6667805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,74,74,75,75,76,76,77,77,78,78,82,82,85,85,86,86,87,87,90,90,91,91,92,92,93,93,94,94,96,96,97,97,98,98,99,99,100,100,101,101,104,104,110,110,115,115,118,118,121,121,122,122,123,123,124,124,125,125,126,126,127,127,129,129,130,130,137,137,138,138,139,139,140,140,143,143,144,144,147,147,148,148,150,150,153,153,157,157,159,159,164,164,165,165,166,166,167,167,169,169,175,175,176,176,177,177,178,178,179,179,181,181,193,193,196,196,198,198,200,200],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,-1,48,50,52,-1,54,56,58,-1,60,62,64,-1,66,68,-1,70,72,74,76,78,80,82,84,-1,86,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,116,118,120,122,124,-1,-1,-1,126,-1,-1,128,130,132,-1,-1,134,136,138,140,142,-1,144,146,148,150,152,154,-1,-1,156,-1,-1,-1,-1,-1,158,-1,-1,-1,-1,160,-1,-1,162,-1,-1,164,166,168,170,172,174,176,-1,178,180,-1,-1,-1,-1,-1,-1,182,184,186,188,-1,-1,190,192,-1,-1,194,196,-1,198,-1,-1,200,-1,-1,-1,202,-1,204,-1,-1,-1,-1,206,208,210,212,-1,214,-1,-1,-1,-1,-1,216,218,220,222,224,-1,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,228,-1,-1,230,-1,232,-1,234,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.972052E5,1E0,6E0,8.1E2,1.5900173E3,6.743651E7,1.8274754E-2,1.559733E6,6.2233735E-6,2.0766992E6,-3.9352454E-2,-1.7605912E-2,2.35184E5,2.8530578E6,4.6463413E0,5.185192E2,3.2911258E6,1.3E1,2.3E1,6.0052995E6,4.5754105E-2,7.792276E-2,5.709E3,6.292039E0,1.4912975E-1,9E0,1.901875E2,6.379688E-3,1.3920766E2,5.894636E6,7.714286E0,-5.229772E-3,2.1129E4,5.095006E5,4.4935583E2,2.2615165E-2,3.53969E5,4.05E2,1.2400667E3,2.2713481E-3,8.7601E4,8.01E2,-2.0412194E-2,1.753E3,8.390338E7,1.5522388E0,1.1244019E2,2.165E3,4.49239E6,2.234E3,5.4E1,-1.4194483E-2,1.295313E6,3.3704692E10,-2.392149E-2,4.236E3,2.41E2,7.7630096E2,4E0,7.70947E5,3.4915986E3,4.1325716E7,1.0950326E6,1.6063418E7,3.1530054E0,1.5E1,5.4009717E3,4.6750406E4,-7.161074E-2,-3.326427E-2,-2.2630611E-2,-1.0853086E-2,-3.0224049E-3,1.3999298E5,1.02E2,2.8024E4,5.47E2,1.9637654E2,1.4698161E-3,1.1921103E-2,-6.736173E-3,3.5364E4,-2.244549E-2,-4.2857196E-2,1.28636E7,4.8709216E5,1.97525E1,1.8431344E-3,-1.4241126E-2,5.857143E0,2.8055556E0,1.28636E7,5.281628E2,1.8645384E1,2.513066E-2,3.3382E4,1.4283589E6,3.1226995E0,9.49372E4,2.1298597E0,4.435876E2,-7.858146E-3,1.4932823E-2,1.8201E4,4.9509685E-2,3.5136476E-2,-3.2075506E-2,-1.3317501E-2,-8.192813E-4,2E1,-1.1377268E-2,-2.473519E-2,-8.901832E-3,1.886736E-3,1.6E1,-6.3525126E-3,-2.2607056E-2,2.0487332E-3,-3.4160864E-2,-1.4916765E-2,2.16E2,3.0283334E2,1.4390917E8,7.54E2,1.422162E3,1.0079E5,8.6363555E4,-1.776235E-2,1.387E3,1E0,-7.090227E-3,-1.8114451E-2,6.0518007E-3,-5.462109E-3,4.420927E-3,-9.4596185E-3,8.051603E7,1.1716E4,6.965855E9,2.36433E6,1.9099436E-3,1.1702081E-2,6.941074E6,4.8709216E5,-3.993294E-3,-2.1651287E-2,2.379747E0,2.9585715E2,7.0477296E-3,1.36E2,4.65909E-2,2.6137244E-2,4.6750406E4,2.5887595E-2,3.205296E-2,1.3973525E-2,2.6210527E0,-5.945938E-3,2.6699028E0,-1.8800398E-2,-5.5004717E-3,-1.2569496E-2,-7.12295E-4,4.6E1,5.5555553E0,1.8949389E6,1.5422421E2,5.0678095E-3,5.494667E2,-7.2277533E-3,8.353372E-3,1.5093553E-3,-4.2204354E-3,1.7792116E-3,1.47176E5,1.1362371E6,5E0,2.8846003E5,5.1604336E7,-4.505623E-3,6.009E4,1.4623275E-2,4.7639236E-3,-2.4074948E-4,4.0995292E-3,1.6568195E-2,1.2153308E-2,2.5447248E-2,1.6693758E-2,7.683984E-3,-3.5270027E-6,9.820194E-3,6.968415E6,2.1871414E-2,-9.265563E-3,1.345672E6,5.201964E-3,1.6007428E7,3.5538576E-3,1E0,-2.043926E-2,-3.1079069E-2,-5.291665E-3,-1.4257282E-2,-1.2106815E-2,-4.519799E-3,1.220963E-3,-7.2071347E-3,5.0935005E-3,-8.90217E-4,-9.215974E-3,-1.9180061E-3,-1.2051613E-2,-2.2234317E-2,-4.1192146E-3,4.457565E-3,9.330464E-3,2.3287753E-4,5.383695E-4,-7.138954E-3,-2.3208114E-3,5.4996517E-3,3.7311022E-3,1.22307185E-2,4.3693925E-3,9.825735E-3,1.3281742E-2,5.4156007E-3,-3.6526588E-3,5.0203376E-3,2.8532118E-2,1.5245063E-2,7.564245E-3,1.6637865E-2],"split_indices":[2,43,17,3,2,70,7,0,9,52,43,0,0,1,43,68,67,43,8,3,43,0,0,2,53,53,0,4,0,70,60,71,0,44,43,70,0,9,8,4,0,9,2,0,44,7,68,67,44,43,12,0,0,9,46,0,12,10,4,8,9,4,66,62,62,68,0,43,48,0,0,0,0,0,43,2,12,2,67,0,0,0,9,0,0,60,43,73,0,0,68,71,60,4,71,0,9,43,69,48,53,71,0,0,10,0,0,0,0,0,6,0,0,0,0,3,0,0,53,0,0,44,4,5,2,48,9,48,0,2,79,0,0,0,0,0,0,7,44,46,43,0,0,60,43,0,0,69,67,0,0,0,0,48,0,0,0,71,0,68,0,0,0,0,0,73,60,67,0,48,0,0,0,0,0,1,43,8,43,7,0,1,0,0,0,0,0,0,0,0,0,0,0,9,0,0,9,0,66,0,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,7.43E2,2.72E2,2.3E2,5.13E2,1.6E1,2.56E2,5E0,2.25E2,2.26E2,2.87E2,9E0,7E0,1.7E1,2.39E2,1.66E2,5.9E1,2.03E2,2.3E1,2.05E2,8.2E1,1E1,7E0,1.56E2,8.3E1,1.46E2,2E1,5.5E1,4E0,1.63E2,4E1,1.9E1,4E0,1.83E2,2.2E1,6.9E1,1.3E1,7.4E1,8.2E1,7.9E1,4E0,1.11E2,3.5E1,6E0,1.4E1,2E1,3.5E1,3.4E1,1.29E2,1.2E1,2.8E1,1.4E1,5E0,1.68E2,1.5E1,7E0,1.5E1,3.6E1,3.3E1,2.8E1,4.6E1,5.5E1,2.7E1,3.2E1,4.7E1,6.6E1,4.5E1,2.2E1,1.3E1,7E0,7E0,1.6E1,4E0,7E0,2.8E1,2.5E1,9E0,9.2E1,3.7E1,5E0,7E0,4E0,2.4E1,1E1,4E0,5.6E1,1.12E2,1E1,5E0,4E0,1.1E1,1.1E1,2.5E1,2.4E1,9E0,9E0,1.9E1,8E0,3.8E1,3.9E1,1.6E1,2.2E1,5E0,8E0,2.4E1,2.5E1,2.2E1,6.2E1,4E0,6E0,3.9E1,4E0,1.8E1,7E0,6E0,2.1E1,7E0,5E0,2E1,4E0,5E0,3.3E1,5.9E1,1.8E1,1.9E1,1E1,1.4E1,5.2E1,4E0,2.9E1,8.3E1,5E0,5E0,6E0,5E0,7E0,4E0,1.5E1,1E1,9E0,1.5E1,5E0,4E0,9E0,1E1,4E0,4E0,1.3E1,2.5E1,7E0,3.2E1,9E0,7E0,1.5E1,7E0,1.9E1,5E0,3.5E1,4E0,1E1,1.1E1,6E0,1.4E1,4E0,2.9E1,2.5E1,3.4E1,1.4E1,4E0,1.2E1,7E0,6E0,4E0,5E0,9E0,2.6E1,2.6E1,1.5E1,1.4E1,7.9E1,4E0,1.1E1,4E0,5E0,5E0,5E0,4E0,5E0,1E1,5E0,4E0,5E0,5E0,9E0,4E0,4E0,2.1E1,4E0,2.8E1,4E0,1.1E1,1.4E1,2.1E1,4E0,6E0,1.5E1,1.4E1,1E1,1.5E1,1.2E1,2.2E1,9E0,5E0,8E0,4E0,2E1,6E0,1.6E1,1E1,7E0,8E0,4E0,1E1,1.5E1,6.4E1,5E0,6E0,5E0,4E0,4E0,1.7E1,2.2E1,6E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"235","size_leaf_vector":"1"}},{"base_weights":[1.4377204E-2,-1.5660624E-1,4.9529484E-1,-4.8663077E-1,-2.1084787E-3,-5.829388E-1,5.4487056E-1,2.9609978E-1,-5.1911163E-1,-1.6130061E-1,1.134766E-1,-3.1639814E-2,-1.4011223E-2,1.3125441E0,4.939846E-1,3.288987E-2,-4.6090093E-3,-6.983173E-1,-4.0818495E-1,-2.4788819E-1,6.9673453E-3,7.799139E-2,4.5886874E-1,3.4380168E-2,6.706123E-2,3.5069853E-1,7.50795E-1,-6.149155E-1,-8.846754E-1,-4.7176003E-1,3.2110855E-3,-3.9059E-1,-1.7390017E-1,-2.7103266E-1,5.5396438E-2,1.4779969E-1,-1.6382823E-2,2.3503827E-1,2.5766423E-2,9.7157516E-2,4.5448008E-1,5.896566E-1,8.7109065E-1,-6.482583E-1,-9.039893E-3,-2.1030493E-2,-9.478067E-1,-4.1875008E-1,-8.000923E-1,1.7956136E-2,-1.5118612E-1,-2.8482133E-1,-4.650639E-1,-2.5084972E-1,-5.3899504E-2,-5.529362E-4,-2.2403058E-2,-4.924974E-3,1.7627378E-1,9.2411615E-2,2.7075124E-1,2.0955662E-1,-4.5018412E-2,2.8565787E-3,1.8289194E-2,2.7418974E-1,-6.251715E-2,4.8526943E-1,-7.1584913E-3,1.265036E-2,2.9511178E-2,9.343002E-1,1.7807515E-2,-3.1096052E-2,-1.731282E-2,-4.7187775E-2,-2.4150196E-2,-5.386295E-1,-2.7420583E-1,-1.7646331E-2,-1.0127735E0,-1.3990566E-2,-4.50798E-2,-3.202222E-1,-7.5404365E-3,-1.1275629E-2,-2.33795E-2,-1.4055581E-1,-3.3882025E-1,-1.10522784E-1,5.1063456E-2,5.1522065E-2,-7.241517E-2,2.7589345E-1,9.3746796E-2,1.4835468E-1,-1.3073278E-2,3.2628033E-1,-1.2055591E-2,1.520202E-2,-2.1922132E-3,-1.2874891E-1,4.624318E-3,1.860571E-2,1.3351272E-1,3.8614534E-3,-1.13053136E-1,4.25396E-1,8.7166077E-1,2.323293E-2,4.4732958E-2,-3.0387348E-2,-4.7319612E-1,5.3308133E-2,-3.6560488E-1,-6.0503263E-2,-2.7562568E-2,2.1564204E-3,-6.7529925E-3,-8.010528E-3,-1.6387561E-2,3.213173E-2,-2.4402682E-1,-4.5196897E-1,-2.0449527E-1,-5.7164866E-2,-1.247199E-2,-3.1442575E-3,1.055183E-1,-3.3468108E-3,1.0682799E-1,3.0929053E-3,-1.1144002E-1,1.7358132E-2,5.5846944E-3,9.972215E-3,1.0431361E-2,2.3153735E-2,2.0525607E-1,1.2611057E-1,-1.9242454E-1,1.7359488E-2,8.3915464E-4,3.5793418E-3,-2.3604223E-1,4.1813046E-2,-1.4734697E-1,4.511797E-4,8.970255E-3,-7.2414996E-3,-2.0755947E-1,-3.7772248E-3,4.4937763E-1,4.681434E-2,2.4832327E-2,-2.2883609E-2,-1.3626187E-2,1.165859E-2,-3.5924413E-3,-2.906079E-1,-2.3053942E-2,5.409874E-3,-1.53148E-3,-1.3219251E-2,-3.960894E-3,-7.624876E-3,-5.33857E-1,-1.5479107E-3,-2.5869983E-1,-4.8279176E-3,3.266586E-3,1.8617671E-3,6.5732696E-3,5.6622062E-2,8.751726E-3,-6.577063E-3,-2.3720202E-3,-2.4738437E-3,3.303116E-3,9.3819596E-2,-5.593421E-3,4.828695E-2,2.562378E-1,-3.7309334E-3,1.9170468E-1,-1.9731954E-2,-2.5587803E-2,9.149446E-2,-1.1240177E-2,-2.6808238E-1,-3.6911797E-3,8.559756E-3,1.2190492E-2,-1.0530989E-2,1.2568238E-3,1.8117186E-3,-2.0673494E-3,-2.913659E-3,-1.35537E-2,3.4046194E-1,5.7156485E-1,-1.7561594E-2,-8.670689E-3,-2.8586531E-2,-1.3454394E-2,-1.5883882E-2,-8.000104E-3,6.2799673E-3,-1.1510374E-6,2.0641368E-3,5.8707697E-3,7.5796368E-3,-5.2727587E-3,8.692511E-3,1.3919663E-2,1.2261971E-2,4.467113E-3,-5.3287987E-3,3.1615566E-3,6.265921E-4,8.923013E-3,-1.4147436E-2,-6.7963875E-3,-5.1409528E-3,2.3398283E-3,6.366647E-4,1.7056411E-2,1.5334917E-2,2.7908405E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,-1,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,-1,75,77,79,-1,81,83,85,87,89,-1,-1,91,93,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,109,-1,-1,-1,-1,-1,111,113,-1,115,-1,117,119,-1,-1,-1,121,123,125,127,129,131,133,135,137,139,141,-1,-1,-1,143,145,-1,147,-1,149,151,153,-1,-1,-1,155,157,159,-1,-1,-1,-1,-1,-1,161,163,165,167,169,-1,-1,171,-1,173,-1,175,-1,-1,177,-1,179,181,183,185,-1,-1,187,189,191,193,-1,-1,195,197,-1,199,-1,-1,-1,-1,-1,-1,201,-1,-1,-1,-1,-1,-1,203,-1,205,-1,-1,-1,-1,207,-1,-1,-1,-1,-1,209,-1,211,213,-1,215,-1,217,219,-1,221,-1,-1,223,-1,-1,-1,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.214206E1,3.757717E1,1.4322632E1,6.128292E0,9.273622E0,1.3433027E-1,9.156639E0,1.7170925E0,4.2075424E0,3.09055E0,3.5430548E0,0E0,0E0,1.0506058E-1,8.457668E0,0E0,0E0,7.771225E-1,3.715458E0,1.4102278E0,9.978342E-1,1.7548567E0,4.6635056E-1,0E0,0E0,4.008028E0,1.1146317E0,7.3983574E-1,2.192421E-1,1.8057613E0,1.2490338E0,2.3775864E-1,8.5447526E-1,6.037253E-1,4.6023214E-1,1.016917E0,7.470035E-1,2.5410682E-1,0E0,1.3174495E0,2.1204605E0,4.3020344E-1,1.0519333E0,1.3487625E-1,0E0,0E0,9.171867E-2,1.7237167E0,1.0360098E0,0E0,2.2770771E-1,4.2442918E-2,1.1205006E-1,5.0542593E-1,2.3149088E-1,0E0,0E0,1.6754472E-1,1.4678335E-1,6.3336235E-1,1.4728086E0,4.148702E-1,4.2456967E-1,0E0,0E0,3.440467E-1,1.9403213E-1,2.0351944E0,0E0,0E0,0E0,9.919739E-2,0E0,0E0,0E0,0E0,0E0,1.4082909E-1,1.5461471E0,0E0,4.5406246E-1,0E0,9.9581204E-2,1.9532084E-2,0E0,0E0,0E0,4.9028122E-1,3.7873387E-1,1.9725138E-1,9.799653E-2,1.7228341E-1,1.1623266E-1,9.385514E-2,1.4155445E-1,4.9393535E-1,9.728532E-1,6.4865494E-1,0E0,0E0,0E0,5.403665E-1,3.7351626E-1,0E0,8.817053E-2,0E0,1.884945E-1,1.1556816E0,7.0693016E-2,0E0,0E0,0E0,1.5640259E-2,3.3268875E-1,2.551918E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.489652E-2,8.87841E-2,2.8782153E-1,1.3962722E-1,1.2845194E-1,0E0,0E0,1.8242985E-2,0E0,5.9779093E-2,0E0,2.2424966E-2,0E0,0E0,3.9001126E-2,0E0,2.4577151E-1,3.7174964E-1,3.1400722E-1,6.546844E-1,0E0,0E0,4.1293845E-1,8.3642006E-2,2.2298366E-1,1.8909898E-1,0E0,0E0,1.9179584E-2,1.0549575E-1,0E0,9.765167E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.8665218E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.702423E-2,0E0,2.7824879E-2,0E0,0E0,0E0,0E0,5.3915963E-2,0E0,0E0,0E0,0E0,0E0,2.0469725E-2,0E0,2.631826E-1,5.37529E-2,0E0,9.3395054E-2,0E0,1.0072256E-1,1.0828302E-1,0E0,3.3397198E-2,0E0,0E0,2.1930218E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5483685E-1,1.11252785E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,71,71,77,77,78,78,80,80,82,82,83,83,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,101,101,102,102,104,104,106,106,107,107,108,108,112,112,113,113,114,114,121,121,122,122,123,123,124,124,125,125,128,128,130,130,132,132,135,135,137,137,138,138,139,139,140,140,143,143,144,144,145,145,146,146,149,149,150,150,152,152,159,159,166,166,168,168,173,173,179,179,181,181,182,182,184,184,186,186,187,187,189,189,192,192,199,199,200,200],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,-1,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,-1,76,78,80,-1,82,84,86,88,90,-1,-1,92,94,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,110,-1,-1,-1,-1,-1,112,114,-1,116,-1,118,120,-1,-1,-1,122,124,126,128,130,132,134,136,138,140,142,-1,-1,-1,144,146,-1,148,-1,150,152,154,-1,-1,-1,156,158,160,-1,-1,-1,-1,-1,-1,162,164,166,168,170,-1,-1,172,-1,174,-1,176,-1,-1,178,-1,180,182,184,186,-1,-1,188,190,192,194,-1,-1,196,198,-1,200,-1,-1,-1,-1,-1,-1,202,-1,-1,-1,-1,-1,-1,204,-1,206,-1,-1,-1,-1,208,-1,-1,-1,-1,-1,210,-1,212,214,-1,216,-1,218,220,-1,222,-1,-1,224,-1,-1,-1,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,6E0,8.1E2,5.4051904E8,2.4481E4,1.89991E5,7.7E1,1.5859042E6,5.1167645E6,-3.1639814E-2,-1.4011223E-2,1.2328386E3,2.8045622E6,3.288987E-2,-4.6090093E-3,1.4504054E8,3.513913E2,3.79E2,1.0203835E-5,2.0342858E0,3.7788504E7,3.4380168E-2,6.706123E-2,7.075161E2,7.459E3,4.56E2,1.4E1,4.9735293E0,2.2E0,3.1E1,1.559733E6,6.83E2,1.65281E8,5.372237E2,9E0,1.2E1,2.5766423E-2,1.124641E0,4.1325716E7,3.532346E-1,4.9036694E0,5.159652E-7,-9.039893E-3,-2.1030493E-2,1E0,3.95E2,3.89E2,1.7956136E-2,2.87834E5,1E0,1E1,4.9E1,3.0283334E2,-5.529362E-4,-2.2403058E-2,2.4915715E-3,2.1111E4,5.0149255E0,1.4676277E7,8.4683485E-2,1.244E3,2.8565787E-3,1.8289194E-2,2.276E4,9.47E4,4.907764E3,-7.1584913E-3,1.265036E-2,2.9511178E-2,1E0,1.7807515E-2,-3.1096052E-2,-1.731282E-2,-4.7187775E-2,-2.4150196E-2,1.5827801E4,2.3961E4,-1.7646331E-2,7.3E1,-1.3990566E-2,3.01E2,1.9E1,-7.5404365E-3,-1.1275629E-2,-2.33795E-2,1.664557E0,1E0,2.4067588E0,1.944578E6,5.08E2,1.2368386E0,3.52E2,2.6138635E2,5.201557E7,2.2842104E0,5.2409735E-7,-1.2055591E-2,1.520202E-2,-2.1922132E-3,1E0,9.932432E0,1.860571E-2,3.0714026E8,3.8614534E-3,1.3E1,1.2825651E0,2.3385885E0,2.323293E-2,4.4732958E-2,-3.0387348E-2,1.901875E2,1.610025E6,1.0885427E9,-6.0503263E-2,-2.7562568E-2,2.1564204E-3,-6.7529925E-3,-8.010528E-3,-1.6387561E-2,1.2916666E0,1.1320151E7,1.8181819E0,1.8738E4,1.802361E6,-1.247199E-2,-3.1442575E-3,3.875E0,-3.3468108E-3,6.23E2,3.0929053E-3,2.8266037E2,1.7358132E-2,5.5846944E-3,1.4E1,1.0431361E-2,4.1221875E-1,2.560356E5,9.7E2,2.5751933E-1,1.7359488E-2,8.3915464E-4,3.792E3,5.821732E2,1.0439024E1,2.402121E4,4.511797E-4,8.970255E-3,2.6845297E5,4.6847186E2,-3.7772248E-3,1.4839433E5,4.681434E-2,2.4832327E-2,-2.2883609E-2,-1.3626187E-2,1.165859E-2,-3.5924413E-3,6.76E2,-2.3053942E-2,5.409874E-3,-1.53148E-3,-1.3219251E-2,-3.960894E-3,-7.624876E-3,5E0,-1.5479107E-3,4.0663E4,-4.8279176E-3,3.266586E-3,1.8617671E-3,6.5732696E-3,5.51E2,8.751726E-3,-6.577063E-3,-2.3720202E-3,-2.4738437E-3,3.303116E-3,1.75E0,-5.593421E-3,2.992648E-2,7.6937294E-1,-3.7309334E-3,8.051603E7,-1.9731954E-2,6.6414773E-1,3E0,-1.1240177E-2,1E0,-3.6911797E-3,8.559756E-3,2.9427118E2,-1.0530989E-2,1.2568238E-3,1.8117186E-3,-2.0673494E-3,-2.913659E-3,-1.35537E-2,2.7726634E5,1.0476191E0,-1.7561594E-2,-8.670689E-3,-2.8586531E-2,-1.3454394E-2,-1.5883882E-2,-8.000104E-3,6.2799673E-3,-1.1510374E-6,2.0641368E-3,5.8707697E-3,7.5796368E-3,-5.2727587E-3,8.692511E-3,1.3919663E-2,1.2261971E-2,4.467113E-3,-5.3287987E-3,3.1615566E-3,6.265921E-4,8.923013E-3,-1.4147436E-2,-6.7963875E-3,-5.1409528E-3,2.3398283E-3,6.366647E-4,1.7056411E-2,1.5334917E-2,2.7908405E-2],"split_indices":[2,43,17,3,2,7,12,9,44,43,43,0,0,67,43,0,0,46,67,2,53,68,7,0,0,67,2,10,3,68,73,10,9,2,5,4,3,3,0,53,66,53,53,52,0,0,106,2,2,0,12,29,3,0,4,0,0,53,9,71,62,57,2,0,0,10,9,4,0,0,0,29,0,0,0,0,0,43,1,0,0,0,0,67,0,0,0,68,29,68,9,2,68,0,4,7,69,52,0,0,0,26,69,0,5,0,3,69,53,0,0,0,4,5,46,0,0,0,0,0,0,68,5,71,9,12,0,0,69,0,2,0,4,0,0,3,0,53,62,2,57,0,0,44,4,73,48,0,0,43,4,0,48,0,0,0,0,0,0,2,0,0,0,0,0,0,6,0,9,0,0,0,0,2,0,0,0,0,0,68,0,53,53,0,7,0,53,8,0,30,0,0,67,0,0,0,0,0,0,43,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,7.36E2,2.61E2,2.34E2,5.02E2,1.1E1,2.5E2,9E0,2.25E2,2.11E2,2.91E2,7E0,4E0,1.4E1,2.36E2,4E0,5E0,8.4E1,1.41E2,1.39E2,7.2E1,2.65E2,2.6E1,4E0,1E1,1.53E2,8.3E1,6.1E1,2.3E1,1.22E2,1.9E1,4.6E1,9.3E1,1E1,6.2E1,1.52E2,1.13E2,9E0,1.7E1,4.5E1,1.08E2,3.8E1,4.5E1,5.6E1,5E0,4E0,1.9E1,1.07E2,1.5E1,5E0,1.4E1,2.1E1,2.5E1,5.6E1,3.7E1,5E0,5E0,4.2E1,2E1,1.06E2,4.6E1,1.2E1,1.01E2,5E0,4E0,2.1E1,2.4E1,1.03E2,5E0,6E0,3.2E1,3.9E1,6E0,5E1,6E0,1.5E1,4E0,5.7E1,5E1,6E0,9E0,5E0,9E0,1.5E1,6E0,5E0,2E1,2.6E1,3E1,2.4E1,1.3E1,2.3E1,1.9E1,8E0,1.2E1,6.9E1,3.7E1,4.2E1,4E0,8E0,4E0,3.7E1,6.4E1,1E1,1.1E1,6E0,1.8E1,9.1E1,1.2E1,4E0,3.5E1,1.7E1,4E1,1.1E1,3.9E1,4E0,5E0,5E0,4E0,4E0,1.1E1,1E1,1.6E1,1.5E1,1.5E1,1.9E1,5E0,4E0,9E0,7E0,1.6E1,4E0,1.5E1,4E0,4E0,8E0,4E0,2.2E1,4.7E1,2.1E1,1.6E1,3.6E1,6E0,1.7E1,2E1,5.2E1,1.2E1,4E0,7E0,9E0,9E0,4E0,8.7E1,7E0,5E0,3.4E1,6E0,4E0,7E0,2.7E1,1.2E1,4E0,6E0,1.2E1,4E0,4E0,1.1E1,4E0,1.1E1,1.4E1,5E0,4E0,5E0,1.1E1,5E0,9E0,6E0,4E0,4E0,1.5E1,7E0,1.2E1,3.5E1,5E0,1.6E1,6E0,1E1,1.3E1,4E0,1.6E1,4E0,8E0,4.4E1,8E0,4E0,4E0,5E0,4E0,5E0,4.8E1,3.9E1,1.3E1,1.4E1,7E0,4E0,4E0,7E0,4E0,7E0,7E0,8E0,7E0,5E0,1.6E1,1.9E1,8E0,8E0,5E0,5E0,8E0,5E0,1.1E1,5E0,1E1,3.4E1,4E0,4.4E1,6E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[4.282539E-3,-1.5980986E-1,4.428412E-1,-4.5826387E-1,-9.64763E-3,-2.8706862E-2,4.8953718E-1,-6.6484565E-1,-3.5655844E-1,-1.7110102E-1,1.2145714E-1,2.0425431E-1,5.975026E-1,-6.287699E-1,-4.9905527E-2,-4.7406977E-1,-1.5504657E-1,-3.5407957E-1,-9.038243E-2,-1.948989E-2,1.9048887E-1,4.245525E-1,6.0930364E-2,1.1161122E0,5.404194E-1,-4.0212203E-2,-5.7448566E-1,-4.203092E-1,-8.556885E-1,5.480729E-1,-2.8238264E-1,-2.4379371E-1,-4.819723E-1,-6.863111E-2,-2.8434414E-2,1.8420845E-2,-2.9343253E-1,1.5160462E-1,4.2861778E-1,2.7033815E-1,2.7421888E-2,1.9839756E-1,-5.2444313E-2,3.0397184E-2,5.9613295E-2,4.2763776E-1,7.372566E-1,-2.7440354E-2,-1.3120598E-2,-2.3259267E-1,-4.647628E-1,-1.4245516E-2,-5.5558696E-2,1.17730005E-2,3.238991E-2,-2.2586961E-1,-2.9085081E-2,-3.053415E-1,-6.4963534E-2,-1.2324744E-2,-5.143408E-1,-1.0789787E-1,1.8881796E-2,1.4778185E-1,-3.637012E-2,-2.1648295E-2,4.8374466E-4,8.579984E-2,2.879833E-1,5.6159967E-3,4.9930394E-1,1.02200955E-1,1.8869437E-2,2.6119836E-3,2.6528913E-1,4.206135E-2,-2.078595E-1,4.638437E-1,-4.2280406E-2,7.809948E-1,9.187501E-3,-1.7436896E-2,-5.001263E-2,-5.210282E-1,-3.6299708E-1,-1.5876178E-1,-4.3716303E-1,-1.6737146E-2,-1.0280319E-2,3.6121877E-3,-9.114485E-3,-1.5027487E-2,-5.5692536E-1,-1.6829827E-1,2.5324164E-2,5.002059E-2,-1.1673954E-2,-1.5445804E-3,2.1004426E-1,-1.4001645E-1,6.1578788E-2,1.0641734E-1,-1.6641088E-2,-5.280974E-3,3.2465306E-1,2.5540506E-2,1.0539849E-2,1.3102842E-3,7.188572E-3,2.316542E-3,1.6000278E-2,9.190085E-3,-2.150581E-2,-1.4457218E-2,-1.857162E-3,1.0171475E-1,4.9673375E-1,-1.2682702E-2,9.166736E-3,8.893733E-3,8.217765E-1,-7.426931E-3,2.338216E-3,-2.4966089E-2,-1.1738144E-2,-1.8443297E-1,-4.3074203E-1,-3.051808E-1,-8.7806605E-2,-2.4272174E-2,-1.04512945E-2,-3.1522036E-2,-1.8650971E-2,-1.01339296E-1,-3.750304E-1,-1.3413919E-2,6.840111E-3,-5.4356554E-3,7.517972E-2,1.0871677E-2,4.493147E-3,1.9456498E-2,-2.0477846E-1,9.0122856E-2,-4.58138E-3,-1.2808391E-2,1.5958446E-1,3.6497775E-1,1.9443567E-3,-3.8519052E-3,2.0067447E-3,1.3277823E-2,-6.5351385E-3,2.1206862E-1,5.427489E-1,1.8452225E-2,3.918162E-2,-2.7686933E-3,-1.1893801E-2,-2.5489086E-2,-1.1202827E-2,-6.0975584E-3,-1.7337618E-2,-1.3765106E-2,-3.830511E-2,-3.1301405E-2,-1.841327E-1,-2.5823444E-1,-2.7890079E-2,6.411128E-2,-6.587412E-2,1.1746271E-1,-1.5169332E-2,-3.6964463E-3,5.314416E-3,-5.311406E-3,-1.0859397E-2,-2.8596302E-3,1.2057656E-1,4.6179496E-2,-1.349823E-1,2.1995944E-1,8.5637145E-2,2.7764356E-1,4.8408473E-1,3.9302735E-3,1.2337889E-2,5.0258005E-1,3.4941804E-2,-1.0424447E-4,-7.744306E-3,3.7727735E-3,-2.8495346E-3,-2.9315157E-3,-1.20281475E-2,-5.2722883E-3,-1.5432335E-2,-1.3650125E-3,5.629392E-3,-8.359306E-4,-7.7323965E-3,6.391512E-3,9.6006435E-5,-3.9546434E-3,3.7544388E-3,-2.256193E-4,6.871089E-3,-9.2264666E-4,3.528156E-3,-1.1525731E-2,-1.5668486E-3,1.1844793E-2,7.766568E-4,5.55371E-3,-2.998259E-3,8.021801E-3,1.7764298E-2,1.1768102E-2,2.491563E-2,2.4774328E-2,1.3376156E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,-1,73,75,-1,-1,77,79,-1,-1,81,83,-1,-1,-1,-1,85,-1,87,89,-1,91,93,95,97,99,-1,-1,101,103,-1,105,107,-1,-1,109,111,113,115,117,119,-1,-1,121,123,125,127,129,-1,-1,-1,-1,-1,131,133,135,137,-1,-1,139,141,143,145,-1,-1,147,-1,-1,-1,-1,-1,-1,-1,149,-1,-1,151,153,-1,-1,-1,155,-1,-1,-1,-1,157,159,161,163,-1,-1,-1,-1,165,167,169,-1,-1,171,-1,-1,173,175,177,-1,179,181,183,-1,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,189,191,193,195,-1,197,199,201,203,-1,-1,-1,-1,-1,205,207,209,211,213,215,217,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.268259E1,3.2935722E1,1.39701805E1,4.931526E0,1.0392763E1,0E0,8.009335E0,5.196037E-1,3.8810444E0,3.2173467E0,2.640493E0,2.3106086E0,5.051178E0,4.9687958E-1,0E0,1.7945328E0,5.795312E0,8.275175E-1,1.7166666E0,9.400883E-1,1.6202416E0,5.953741E-1,7.2732866E-1,5.8794785E-1,3.547451E0,0E0,1.9900513E-1,6.7629814E-1,1.9076719E0,2.7006102E-1,9.172845E-1,3.984151E-1,7.1755886E-2,5.1405317E-1,0E0,5.7240003E-1,5.9043854E-1,1.3932416E0,4.819212E-1,3.5299194E-1,0E0,1.8427515E-1,3.906209E-1,0E0,0E0,1.9421749E0,1.3046684E0,0E0,0E0,5.1001084E-1,2.4031067E-1,0E0,0E0,0E0,0E0,6.112082E-1,0E0,5.1428556E-2,2.237905E-1,0E0,2.0124912E-2,8.330685E-1,4.1008863E-1,2.7781165E-1,5.872886E-1,0E0,0E0,1.0314445E0,7.8946114E-1,0E0,1.6419601E-1,3.1073406E-2,0E0,0E0,2.267437E-1,1.7379001E-1,1.6358727E-1,1.2000694E0,5.5757487E-1,1.209774E0,0E0,0E0,1.19574584E-1,1.0060692E-1,2.834618E-1,3.6799222E-1,8.584547E-2,0E0,0E0,0E0,0E0,0E0,8.955145E-2,9.448173E-1,1.5888289E-1,1.902465E-1,0E0,0E0,2.8432906E-2,2.931623E-1,1.4658654E-1,6.612884E-1,0E0,0E0,5.201764E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.58561E-2,0E0,0E0,4.9219903E-1,1.1423969E0,0E0,0E0,0E0,2.5062943E-1,0E0,0E0,0E0,0E0,7.547602E-2,3.4448504E-1,9.6161604E-2,2.694549E-1,0E0,0E0,0E0,0E0,3.1316352E-1,2.9812407E-1,1.09964944E-1,0E0,0E0,1.4695047E-1,0E0,0E0,9.476501E-2,2.7362287E-2,1.2783307E-1,0E0,2.4405715E-1,3.026122E-1,2.9208374E-1,0E0,0E0,0E0,0E0,0E0,8.203459E-2,3.2339096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0684687E-1,1.1141836E-1,2.1343923E-1,1.0389882E-1,0E0,6.308546E-2,7.4564174E-2,6.2366217E-2,9.525315E-2,0E0,0E0,0E0,0E0,0E0,7.842755E-2,4.7456782E-2,1.1883013E-1,2.842903E-1,1.847666E-1,2.2224092E-1,6.528759E-2,0E0,0E0,3.612442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,41,41,42,42,45,45,46,46,49,49,50,50,55,55,57,57,58,58,60,60,61,61,62,62,63,63,64,64,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,82,82,83,83,84,84,85,85,86,86,92,92,93,93,94,94,95,95,98,98,99,99,100,100,101,101,104,104,112,112,115,115,116,116,120,120,125,125,126,126,127,127,128,128,133,133,134,134,135,135,138,138,141,141,142,142,143,143,145,145,146,146,147,147,153,153,154,154,164,164,165,165,166,166,167,167,169,169,170,170,171,171,172,172,178,178,179,179,180,180,181,181,182,182,183,183,184,184,187,187],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,-1,74,76,-1,-1,78,80,-1,-1,82,84,-1,-1,-1,-1,86,-1,88,90,-1,92,94,96,98,100,-1,-1,102,104,-1,106,108,-1,-1,110,112,114,116,118,120,-1,-1,122,124,126,128,130,-1,-1,-1,-1,-1,132,134,136,138,-1,-1,140,142,144,146,-1,-1,148,-1,-1,-1,-1,-1,-1,-1,150,-1,-1,152,154,-1,-1,-1,156,-1,-1,-1,-1,158,160,162,164,-1,-1,-1,-1,166,168,170,-1,-1,172,-1,-1,174,176,178,-1,180,182,184,-1,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,190,192,194,196,-1,198,200,202,204,-1,-1,-1,-1,-1,206,208,210,212,214,216,218,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,2.16577E5,1E0,7.7E1,8.1E2,-2.8706862E-2,7.445455E2,1.05039425E1,6.76E2,4.58E2,7.502167E5,9E0,1.059448E7,8.8324326E-1,-4.9905527E-2,8.719101E0,2.74614E5,3.875E0,4.6E1,2.3E1,5.405423E6,5.766206E6,7.44488E5,1.767635E3,2.9251662E6,-4.0212203E-2,6.1454544E0,2E0,3.89E2,1.9E1,2.5E1,1.000501E6,3.808001E5,2.331083E6,-2.8434414E-2,1.88E2,2.22E3,5.163891E2,3.3852024E7,5.404E2,2.7421888E-2,2.3625128E6,1.6E1,3.0397184E-2,5.9613295E-2,3.0070068E7,2E0,-2.7440354E-2,-1.3120598E-2,3.6E2,4.60349E5,-1.4245516E-2,-5.5558696E-2,1.17730005E-2,3.238991E-2,1.7479E4,-2.9085081E-2,4.3827028E5,6.642857E0,-1.2324744E-2,1.9913E4,7.06E2,7.89E2,1.4473684E-1,1.445E3,-2.1648295E-2,4.8374466E-4,2.41E3,1.071E3,5.6159967E-3,1.639E3,6.185E3,1.8869437E-2,2.6119836E-3,1E0,1.124641E0,1.453785E6,1.0476191E0,9.4578236E-1,1E0,9.187501E-3,-1.7436896E-2,1.2E1,3.0588236E0,1.4338E4,1.6580646E2,5.25E0,-1.6737146E-2,-1.0280319E-2,3.6121877E-3,-9.114485E-3,-1.5027487E-2,4.795239E7,2.5493E4,1.6802971E6,5.02E2,-1.1673954E-2,-1.5445804E-3,4.957196E2,3.6086957E0,1.6533886E8,5.4007E7,-1.6641088E-2,-5.280974E-3,5.929839E8,2.5540506E-2,1.0539849E-2,1.3102842E-3,7.188572E-3,2.316542E-3,1.6000278E-2,9.190085E-3,2.8003533E0,-1.4457218E-2,-1.857162E-3,2.6402525E5,9.143923E4,-1.2682702E-2,9.166736E-3,8.893733E-3,1.06314E5,-7.426931E-3,2.338216E-3,-2.4966089E-2,-1.1738144E-2,9.9E1,1.8224286E2,2.9157894E0,3.741267E7,-2.4272174E-2,-1.04512945E-2,-3.1522036E-2,-1.8650971E-2,1E0,4.1361522E6,5.539245E6,6.840111E-3,-5.4356554E-3,1.6E1,1.0871677E-2,4.493147E-3,1.3E1,9.31E2,3.9960475E0,-4.58138E-3,3.774648E0,3.0416667E0,1.8062708E6,1.9443567E-3,-3.8519052E-3,2.0067447E-3,1.3277823E-2,-6.5351385E-3,3.1E2,4.907764E3,1.8452225E-2,3.918162E-2,-2.7686933E-3,-1.1893801E-2,-2.5489086E-2,-1.1202827E-2,-6.0975584E-3,-1.7337618E-2,-1.3765106E-2,4.7220547E1,7.784E3,1.6808511E0,2.28E0,-2.7890079E-2,2.0671213E2,1.3E1,1E0,3.5510652E6,-3.6964463E-3,5.314416E-3,-5.311406E-3,-1.0859397E-2,-2.8596302E-3,3.2892792E2,2.0467092E6,5.092769E7,9.51E3,1.7500242E7,1.724907E8,2.02E2,3.9302735E-3,1.2337889E-2,2.2040408E7,3.4941804E-2,-1.0424447E-4,-7.744306E-3,3.7727735E-3,-2.8495346E-3,-2.9315157E-3,-1.20281475E-2,-5.2722883E-3,-1.5432335E-2,-1.3650125E-3,5.629392E-3,-8.359306E-4,-7.7323965E-3,6.391512E-3,9.6006435E-5,-3.9546434E-3,3.7544388E-3,-2.256193E-4,6.871089E-3,-9.2264666E-4,3.528156E-3,-1.1525731E-2,-1.5668486E-3,1.1844793E-2,7.766568E-4,5.55371E-3,-2.998259E-3,8.021801E-3,1.7764298E-2,1.1768102E-2,2.491563E-2,2.4774328E-2,1.3376156E-2],"split_indices":[2,43,17,44,2,0,67,69,2,2,43,3,12,68,0,69,5,69,3,3,43,60,9,67,43,0,69,32,2,0,3,43,43,43,0,10,10,70,7,4,0,60,3,0,0,58,6,0,0,2,9,0,0,0,0,44,0,43,73,0,9,2,2,71,2,0,0,2,2,0,2,44,0,0,68,53,9,68,49,27,0,0,3,68,12,67,69,0,0,0,0,0,7,1,43,2,0,0,4,69,5,7,0,0,5,0,0,0,0,0,0,0,68,0,0,48,48,0,0,0,1,0,0,0,0,0,4,69,7,0,0,0,0,26,60,5,0,0,3,0,0,3,2,69,0,69,69,43,0,0,0,0,0,8,4,0,0,0,0,0,0,0,0,0,73,1,68,71,0,4,3,30,43,0,0,0,0,0,4,43,5,44,12,7,10,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,7.34E2,2.74E2,2.45E2,4.89E2,1.1E1,2.63E2,7.9E1,1.66E2,2.19E2,2.7E2,7.3E1,1.9E2,7.5E1,4E0,1.04E2,6.2E1,6.6E1,1.53E2,8.9E1,1.81E2,2.8E1,4.5E1,1.7E1,1.73E2,1.1E1,6.4E1,9.3E1,1.1E1,9E0,5.3E1,3.7E1,2.9E1,1.48E2,5E0,7.9E1,1E1,1.57E2,2.4E1,1.6E1,1.2E1,2E1,2.5E1,6E0,1.1E1,1.12E2,6.1E1,5.9E1,5E0,1.9E1,7.4E1,5E0,6E0,4E0,5E0,4.7E1,6E0,2.7E1,1E1,5E0,2.4E1,1.02E2,4.6E1,2.3E1,5.6E1,6E0,4E0,1.07E2,5E1,5E0,1.9E1,8E0,8E0,7E0,1.3E1,1.6E1,9E0,1.04E2,8E0,5.6E1,5E0,1E1,9E0,4.5E1,2.9E1,3.7E1,1E1,1.4E1,1.3E1,5E0,5E0,6E0,1.8E1,7E1,3.2E1,4.2E1,4E0,6E0,1.7E1,2.7E1,2.9E1,1.03E2,4E0,4E0,4.6E1,1.5E1,4E0,4E0,4E0,4E0,9E0,4E0,1.2E1,5E0,4E0,9E0,9.5E1,4E0,4E0,4E0,5.2E1,4E0,5E0,4.1E1,4E0,9E0,2E1,1.1E1,2.6E1,6E0,4E0,8E0,1E1,5.4E1,1.6E1,2.5E1,7E0,5E0,3.7E1,1.3E1,4E0,8E0,1.9E1,2.5E1,4E0,3.2E1,7.1E1,4E1,6E0,6E0,6E0,5E0,4E0,1.4E1,8.1E1,4E0,4.8E1,4E0,5E0,1.1E1,9E0,4E0,7E0,4E0,2.2E1,3E1,2.4E1,1.2E1,4E0,1E1,1.5E1,2.5E1,1.2E1,4E0,4E0,6E0,1.3E1,4E0,2.1E1,2.2E1,1E1,3.8E1,3.3E1,2.5E1,1.5E1,5E0,9E0,7.1E1,1E1,1.8E1,4E0,6E0,2.4E1,1E1,1.4E1,5E0,7E0,4E0,6E0,1.1E1,4E0,2.1E1,4E0,7E0,5E0,4E0,1.7E1,7E0,1.5E1,4E0,6E0,3.2E1,6E0,2.7E1,6E0,1.4E1,1.1E1,4E0,1.1E1,6E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[1.0388326E-2,-1.4030552E-1,4.7239754E-1,-4.348304E-1,-1.2833244E-2,-5.790869E-1,5.2626246E-1,-5.837202E-1,-3.2643813E-1,-1.360659E-1,1.2067847E-1,-3.274073E-2,-1.4311097E-2,3.134193E-1,6.3941395E-1,-5.323711E-1,-3.7736334E-2,-7.323976E-2,-4.166818E-1,-1.8608682E-1,4.474006E-2,2.010919E-1,1.8798243E-2,-1.12498924E-1,3.7654564E-1,6.8760234E-1,-7.1448244E-2,-3.0266145E-2,-3.587346E-1,-2.4892873E-1,2.1274523E-1,-7.369641E-1,-3.4930846E-1,-3.253191E-1,-1.24986306E-1,-9.495562E-3,1.6182142E-1,1.0867217E-1,3.2406023E-1,3.726113E-2,-1.8752895E-2,-1.0087916E-3,-8.045053E-3,4.3161398E-1,1.1904129E-2,1.1011767E0,6.295695E-1,-1.0426399E-2,7.3351082E-3,-2.4215284E-1,-2.5655964E-2,-3.673265E-1,-7.561075E-2,2.6332855E-2,-2.287433E-2,-1.8815547E-2,-8.797719E-1,-4.0034622E-1,-9.780679E-2,-3.643531E-1,-1.8678468E-1,-4.7936368E-1,-8.720403E-2,8.236923E-2,-7.038707E-2,2.3411028E-1,2.1924276E-3,7.45788E-2,2.2304257E-2,3.87757E-1,2.1035975E-2,6.268567E-2,-1.4882028E-1,5.064681E-1,2.4709468E-1,-6.804852E-3,7.813111E-3,3.4257486E-2,5.9078656E-2,4.0932286E-1,7.4131685E-1,-3.1504092E-1,1.901843E-3,-8.280606E-3,-2.0877583E-2,3.6218555E-3,-8.076267E-3,-3.437439E-3,2.0113823E-3,-5.5872116E-2,-2.396147E-2,-4.272993E-1,-2.299726E-1,-1.4218594E-2,-1.2882934E-2,-5.5001173E-3,-3.8231453E-1,-4.089209E-3,-9.711663E-3,-2.734341E-1,-3.8647167E-2,-4.0184204E-2,-2.0043829E-1,1.1519904E-2,8.872906E-3,-1.0236356E-2,4.9308194E-3,4.8747207E-3,1.436282E-2,1.385777E-1,-3.1050999E-2,3.0358925E-1,2.7466597E-2,-5.1160636E-3,7.912424E-3,2.2566853E-1,3.604424E-2,-1.4750997E-2,8.752503E-4,5.362695E-1,7.1535977E-3,2.5675008E-3,2.8888908E-1,9.555297E-3,4.518391E-1,8.5291106E-1,5.8210063E-1,-1.6858865E-2,-6.3681426E-3,-1.17612695E-2,-4.522976E-1,-5.4745353E-3,-1.2515189E-2,-5.9187347E-3,2.7138807E-3,-2.2638229E-1,-4.078414E-1,-5.9702927E-3,-1.7189099E-2,8.165699E-2,-8.0282755E-2,-2.1667428E-2,-1.5982631E-1,-1.5348726E-3,2.8551018E-3,7.133243E-2,-9.020835E-3,-3.4746742E-3,1.6944005E-1,-2.7185538E-1,1.14984185E-1,5.0500883E-3,3.2894447E-1,1.8673599E-3,1.366226E-2,-7.78244E-2,6.852726E-2,5.714867E-1,1.5875896E-2,5.0973957E-3,3.353413E-1,4.889097E-1,7.976E-3,1.4878836E-2,9.059156E-1,9.089502E-3,6.2051976E-1,-4.75416E-1,-1.33524025E-2,-1.3847336E-2,-4.9782274E-3,-1.9682078E-2,-1.0800449E-2,1.2242195E-1,-2.147313E-3,6.240829E-3,-9.374977E-2,-2.0394827E-1,5.0579747E-3,4.847221E-3,1.6002495E-4,2.4208382E-1,1.147353E-1,6.509094E-4,-2.2088427E-2,-5.4375856E-4,1.7014807E-1,2.3618416E-1,1.8400008E-2,1.3297886E-3,-1.37443E-1,-5.8076844E-2,1.0477056E-1,3.8360858E-1,2.9782811E-2,9.697665E-3,1.9074026E-2,5.2090096E-1,1.1925462E-2,1.968422E-2,4.3577157E-2,1.5370795E-2,2.9887285E-2,-1.8692313E-2,-2.606006E-2,1.0476403E-3,7.5558624E-3,-6.3167918E-3,-1.3731056E-3,-3.2650512E-3,-1.159511E-2,3.3349416E-3,1.3188432E-2,3.732447E-3,1.016734E-2,9.53655E-3,4.0367595E-3,1.3284303E-2,5.6239846E-3,-1.1354617E-2,-1.802359E-3,-5.509594E-3,2.946002E-3,1.3176004E-3,8.583301E-3,8.835795E-3,2.2807637E-2,2.6123086E-2,1.4159386E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,-1,73,75,77,79,-1,-1,81,-1,83,85,-1,87,-1,89,91,93,95,97,99,101,103,105,107,-1,109,-1,111,113,115,117,119,121,-1,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,129,131,133,-1,-1,135,-1,-1,137,-1,139,141,143,-1,-1,145,-1,-1,147,149,151,-1,-1,-1,153,155,-1,-1,157,-1,-1,159,-1,161,163,165,-1,-1,-1,167,-1,-1,-1,-1,169,171,-1,-1,173,175,-1,177,-1,-1,179,-1,-1,181,183,185,-1,187,-1,-1,189,191,193,-1,-1,195,197,-1,-1,199,-1,201,203,-1,-1,-1,-1,-1,205,-1,-1,207,209,-1,-1,-1,211,213,-1,-1,-1,215,217,-1,-1,219,221,223,225,-1,-1,-1,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.2751595E1,2.9578901E1,1.4860748E1,3.6504135E0,9.082026E0,2.3421955E-1,5.700058E0,7.501259E-1,3.1684208E0,2.6039767E0,2.1672423E0,0E0,0E0,2.3814993E0,5.49572E0,1.5554047E0,0E0,1.9622456E0,1.9651566E0,1.8734517E0,4.0158534E-1,1.6467595E0,9.404141E-1,6.3259274E-2,1.520958E0,2.882866E0,4.264229E-1,0E0,7.135663E-1,4.647174E-1,1.2756351E0,4.4209957E-1,1.0775833E0,2.9965782E-1,2.0736883E0,2.519243E-1,1.5423673E-1,1.0609074E0,1.2078476E0,5.4866683E-1,0E0,0E0,0E0,7.864485E-1,3.0022097E-1,1.3768768E-1,2.969963E0,0E0,0E0,5.292337E-1,0E0,1.4171755E-1,1.8176433E-1,0E0,3.7461735E-2,0E0,6.820288E-1,2.3236275E-1,2.3817374E-1,1.8359327E-1,2.0534039E-2,8.745477E-1,7.5863826E-1,1.376725E-1,3.08537E-1,8.56778E-2,0E0,5.451183E-1,0E0,7.2448397E-1,2.5652924E-1,4.304177E-1,4.0643933E-1,3.9899254E-1,1.520387E-1,0E0,0E0,0E0,0E0,3.1118298E-1,1.0493011E0,1.2646317E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4823723E-1,2.2643149E-2,1.0724579E-1,0E0,0E0,9.87339E-2,0E0,0E0,1.0837275E-1,0E0,5.0977814E-1,3.8313055E-1,2.9049342E-2,0E0,0E0,2.6649553E-1,0E0,0E0,3.4264457E-1,1.1214024E0,1.5615511E-1,0E0,0E0,0E0,1.6973954E-1,3.3082727E-1,0E0,0E0,8.8534355E-2,0E0,0E0,1.0400152E-1,0E0,3.088894E-1,1.0869789E0,4.553089E-1,0E0,0E0,0E0,5.8882713E-2,0E0,0E0,0E0,0E0,4.601291E-2,2.702427E-2,0E0,0E0,1.4150064E-1,2.3889303E-1,0E0,4.7386074E-1,0E0,0E0,3.3830076E-2,0E0,0E0,1.4707863E-1,7.1447974E-1,1.4184761E-1,0E0,1.2459707E-1,0E0,0E0,1.3107294E-1,3.2269472E-1,2.201395E-1,0E0,0E0,3.974712E-2,1.0917282E-1,0E0,0E0,3.8630295E-1,0E0,5.074787E-2,4.80299E-2,0E0,0E0,0E0,0E0,0E0,7.8338176E-2,0E0,0E0,1.9958001E-1,1.8911779E-1,0E0,0E0,0E0,1.120739E-1,7.876623E-2,0E0,0E0,0E0,2.399829E-2,6.4926445E-2,0E0,0E0,1.2903346E-1,1.2687759E-1,3.2829916E-1,1.5965712E-1,0E0,0E0,0E0,9.413195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,43,43,44,44,45,45,46,46,49,49,51,51,52,52,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,69,69,70,70,71,71,72,72,73,73,74,74,79,79,80,80,81,81,91,91,92,92,93,93,96,96,99,99,101,101,102,102,103,103,106,106,109,109,110,110,111,111,115,115,116,116,119,119,122,122,124,124,125,125,126,126,130,130,135,135,136,136,139,139,140,140,142,142,145,145,148,148,149,149,150,150,152,152,155,155,156,156,157,157,160,160,161,161,164,164,166,166,167,167,173,173,176,176,177,177,181,181,182,182,186,186,187,187,190,190,191,191,192,192,193,193,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,-1,74,76,78,80,-1,-1,82,-1,84,86,-1,88,-1,90,92,94,96,98,100,102,104,106,108,-1,110,-1,112,114,116,118,120,122,-1,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,130,132,134,-1,-1,136,-1,-1,138,-1,140,142,144,-1,-1,146,-1,-1,148,150,152,-1,-1,-1,154,156,-1,-1,158,-1,-1,160,-1,162,164,166,-1,-1,-1,168,-1,-1,-1,-1,170,172,-1,-1,174,176,-1,178,-1,-1,180,-1,-1,182,184,186,-1,188,-1,-1,190,192,194,-1,-1,196,198,-1,-1,200,-1,202,204,-1,-1,-1,-1,-1,206,-1,-1,208,210,-1,-1,-1,212,214,-1,-1,-1,216,218,-1,-1,220,222,224,226,-1,-1,-1,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,1.2E2,9.48E2,9.3326636E-2,9.9347424E2,5.17259E8,1.5522388E0,2.331083E6,3.887931E0,-3.274073E-2,-1.4311097E-2,2.8204132E7,3.206931E2,3E1,-3.7736334E-2,1.5292561E2,1E0,4.67E2,2.9254214E2,4.371613E2,2.5401869E1,1.1E1,2.2E1,1.059448E7,4.7263342E2,-3.0266145E-2,1.388E1,1.2E1,2.3E1,1.654E3,3.2874417E2,1.8154167E2,2.86152E5,1.4285715E0,9.244374E6,2.1578948E0,1E0,1E0,-1.8752895E-2,-1.0087916E-3,-8.045053E-3,1.835821E1,1.9719212E1,4.626E3,1.3711089E6,-1.0426399E-2,7.3351082E-3,4.7837E4,-2.5655964E-2,1.16E0,1.5077367E3,2.6332855E-2,9E0,-1.8815547E-2,6.27907E0,7.997723E6,3.478E4,1.9E1,1.2707424E0,1.2916666E0,2.712E3,5.405423E6,8.611817E6,4.6E1,2.1924276E-3,2.5E0,2.2304257E-2,1.28738E5,2.9149944E5,9E0,4.265829E2,2E0,3.202146E5,-6.804852E-3,7.813111E-3,3.4257486E-2,5.9078656E-2,9.49372E4,1.7103828E7,3.015873E0,1.901843E-3,-8.280606E-3,-2.0877583E-2,3.6218555E-3,-8.076267E-3,-3.437439E-3,2.0113823E-3,-5.5872116E-2,-2.396147E-2,7.090909E0,2.29E2,4.7280664E3,-1.2882934E-2,-5.5001173E-3,2.6003375E5,-4.089209E-3,-9.711663E-3,1E0,-3.8647167E-2,1.3364486E0,1.19111115E2,1.4E1,8.872906E-3,-1.0236356E-2,1.7544615E0,4.8747207E-3,1.436282E-2,1.007E3,2.026453E6,3.655004E7,2.7466597E-2,-5.1160636E-3,7.912424E-3,1.0542694E1,3.82E2,-1.4750997E-2,8.752503E-4,2.2539758E5,7.1535977E-3,2.5675008E-3,1E0,9.555297E-3,4.4895835E0,1E0,1E0,-1.6858865E-2,-6.3681426E-3,-1.17612695E-2,6.926E3,-5.4745353E-3,-1.2515189E-2,-5.9187347E-3,2.7138807E-3,3.8E1,9.860918E5,-5.9702927E-3,-1.7189099E-2,1E0,1.4E1,-2.1667428E-2,2.0200005E8,-1.5348726E-3,2.8551018E-3,1.6E1,-9.020835E-3,-3.4746742E-3,5.92E2,3.1888113E0,6.642857E0,5.0500883E-3,5.5885565E2,1.8673599E-3,1.366226E-2,1.7161617E2,1E0,4.6847186E2,1.5875896E-2,5.0973957E-3,5.489183E1,1E0,7.976E-3,1.4878836E-2,1.32476E5,9.089502E-3,1.5822886E0,1.5E1,-1.33524025E-2,-1.3847336E-2,-4.9782274E-3,-1.9682078E-2,-1.0800449E-2,8.754E3,-2.147313E-3,6.240829E-3,7.09E2,1.08E2,5.0579747E-3,4.847221E-3,1.6002495E-4,1.124E3,1.25789E5,6.509094E-4,-2.2088427E-2,-5.4375856E-4,2.23099E7,1.3881669E6,1.8400008E-2,1.3297886E-3,1.6869704E6,4.888889E0,4.162742E5,5.941442E6,2.9782811E-2,9.697665E-3,1.9074026E-2,2.9E1,1.1925462E-2,1.968422E-2,4.3577157E-2,1.5370795E-2,2.9887285E-2,-1.8692313E-2,-2.606006E-2,1.0476403E-3,7.5558624E-3,-6.3167918E-3,-1.3731056E-3,-3.2650512E-3,-1.159511E-2,3.3349416E-3,1.3188432E-2,3.732447E-3,1.016734E-2,9.53655E-3,4.0367595E-3,1.3284303E-2,5.6239846E-3,-1.1354617E-2,-1.802359E-3,-5.509594E-3,2.946002E-3,1.3176004E-3,8.583301E-3,8.835795E-3,2.2807637E-2,2.6123086E-2,1.4159386E-2],"split_indices":[2,43,17,44,2,53,67,46,68,43,69,0,0,5,73,70,0,67,29,2,70,67,69,3,3,12,71,0,71,3,0,1,67,67,7,71,60,68,6,19,0,0,0,71,71,2,43,0,0,9,0,69,48,0,8,0,71,9,44,67,68,68,44,43,60,0,0,69,0,1,48,3,67,32,43,0,0,0,0,48,62,68,0,0,0,0,0,0,0,0,0,73,10,48,0,0,43,0,0,8,0,68,67,3,0,0,68,0,0,2,9,7,0,0,0,71,4,0,0,48,0,0,29,0,68,15,26,0,0,0,10,0,0,0,0,0,43,0,0,26,10,0,5,0,0,3,0,0,10,71,73,0,4,0,0,67,29,4,0,0,71,23,0,0,1,0,53,8,0,0,0,0,0,1,0,0,2,10,0,0,0,2,1,0,0,0,9,43,0,0,60,69,62,60,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.043E3,7.87E2,2.56E2,2.37E2,5.5E2,1.2E1,2.44E2,9.8E1,1.39E2,2.86E2,2.64E2,7E0,5E0,8.6E1,1.58E2,8.3E1,1.5E1,3.7E1,1.02E2,2.24E2,6.2E1,1.47E2,1.17E2,1.1E1,7.5E1,1.48E2,1E1,4.7E1,3.6E1,2.3E1,1.4E1,1.6E1,8.6E1,6.7E1,1.57E2,4.3E1,1.9E1,8.5E1,6.2E1,1.13E2,4E0,5E0,6E0,6.5E1,1E1,1.6E1,1.32E2,6E0,4E0,2.4E1,1.2E1,1.3E1,1E1,5E0,9E0,6E0,1E1,7.1E1,1.5E1,5.1E1,1.6E1,1.4E1,1.43E2,1.7E1,2.6E1,1.1E1,8E0,7.9E1,6E0,5.1E1,1.1E1,1E2,1.3E1,4.5E1,2E1,5E0,5E0,7E0,9E0,4.6E1,8.6E1,1.9E1,5E0,5E0,8E0,4E0,6E0,5E0,4E0,4E0,6E0,6E1,1.1E1,1.1E1,4E0,4E0,4.7E1,4E0,1.2E1,1E1,4E0,1.02E2,4.1E1,1.1E1,6E0,8E0,1.8E1,5E0,6E0,4.9E1,3E1,3.8E1,1.3E1,6E0,5E0,1.3E1,8.7E1,6E0,7E0,4.1E1,4E0,4E0,1.6E1,9E0,3.7E1,4.8E1,3.8E1,1.4E1,5E0,9E0,5.1E1,4E0,7E0,4E0,7E0,8E0,3.9E1,5E0,5E0,2.5E1,7.7E1,4E0,3.7E1,6E0,5E0,1.4E1,4E0,6E0,4.3E1,1.1E1,1.9E1,5E0,3.3E1,4E0,9E0,1.9E1,6.8E1,3.3E1,8E0,4E0,1.2E1,3.2E1,5E0,5E0,4.3E1,4E0,3.4E1,4.3E1,8E0,4E0,4E0,3.4E1,5E0,1.9E1,6E0,4E0,7.3E1,3.2E1,5E0,9E0,5E0,1.7E1,2.6E1,5E0,6E0,6E0,1.3E1,1.6E1,1.7E1,7E0,1.2E1,1.5E1,5.3E1,1.1E1,2.2E1,6E0,6E0,2.7E1,5E0,4E0,3.9E1,4E0,3E1,2.7E1,1.6E1,6E0,1.3E1,4.3E1,3E1,9E0,2.3E1,4E0,1.3E1,2.1E1,5E0,8E0,5E0,1E1,6E0,5E0,7E0,1E1,5E0,2.8E1,2.5E1,5E0,6E0,2.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[-6.069969E-3,-1.39584E-1,4.168373E-1,-4.165128E-1,-1.692058E-2,4.6412694E-1,-5.636922E-1,-3.6771145E-1,-6.8826175E-1,-1.2340638E-1,9.475499E-2,1.1592902E0,4.1920882E-1,-3.1316992E-2,-1.239372E-2,-2.764794E-2,-2.95202E-1,-5.5115956E-1,-5.0783135E-2,-1.905333E-1,1.5783371E-2,1.8670271E-1,9.308463E-3,3.617117E-2,6.5777734E-2,2.5238645E-1,5.974901E-1,9.19941E-2,-3.5077587E-1,-9.668635E-3,-6.3124573E-1,-2.6704106E-1,-6.037407E-2,-1.902264E-2,1.8447621E-1,1.4883727E-1,4.2964143E-1,2.7907675E-2,-1.4016416E-2,2.1528159E-1,3.7666537E-2,4.4670904E-1,7.004935E-1,1.843852E-2,-9.007192E-2,-3.835936E-1,-1.4497967E-1,-3.1929143E-2,-1.8278925E-2,-1.8729578E-1,-4.479107E-1,8.411741E-3,-1.7532864E-1,-6.9576964E-2,5.9868664E-2,1.3444636E-2,9.1451146E-2,1.2956489E-2,2.4056609E-1,9.602993E-3,2.3317957E-2,-1.4078775E-3,2.4419786E-1,5.267276E-2,2.875718E-1,4.47477E-3,4.7700074E-1,7.6333183E-1,3.698196E-1,-7.621937E-3,3.8831128E-4,-3.9534763E-1,-4.2695575E-3,-4.9766037E-3,-2.3075259E-1,-2.910212E-1,-9.30281E-2,-3.7576923E-1,-3.2020617E-2,7.1031146E-2,-1.0258791E-1,5.824676E-4,-2.4173677E-1,-2.1200903E-1,1.5893976E-3,1.5678743E-2,1.6614638E-1,5.87003E-3,1.405876E-3,1.10222556E-1,-2.0189263E-1,-1.3502848E-3,2.624422E-1,1.11344926E-1,-3.5708643E-2,8.19227E-4,3.783038E-1,1.8270288E-1,-2.8766137E-2,3.2911843E-1,-8.354947E-3,3.5599896E-1,2.6182918E-2,7.9507124E-1,1.4135402E-2,3.64063E-3,2.335258E-2,-3.3707044E-1,-4.7670263E-1,5.5863406E-3,-6.0001863E-3,-1.6127607E-2,-1.5943821E-3,-2.9655686E-3,-3.1444147E-1,2.7288396E-2,-1.230318E-1,-1.0543343E-2,-4.281391E-1,3.4443801E-3,7.352528E-3,1.4278679E-3,-1.4604068E-1,-1.630179E-2,-9.5327705E-2,-7.322823E-2,-1.9916672E-2,3.375031E-2,-5.668018E-3,6.965579E-2,-4.3535763E-3,1.0984553E-2,2.831896E-3,2.071923E-1,-5.757317E-3,-1.9149872E-2,-2.6035389E-2,2.8901154E-1,1.10567905E-1,-4.011358E-2,1.9771707E-1,-4.7556735E-3,-2.2344165E-1,2.1848662E-2,8.736158E-3,1.1240756E-3,1.1254942E-2,2.849864E-2,-9.044208E-3,9.710715E-2,3.6936462E-1,2.4912652E-2,2.4675043E-1,3.7781075E-2,2.0149276E-2,-2.4418607E-1,-3.8531062E-1,-3.404038E-2,-4.1112208E-1,-9.160771E-3,-1.5732905E-2,-1.6468738E-3,4.4976445E-3,-6.2192682E-2,-1.7739557E-1,-2.175964E-2,-1.1202959E-2,-5.877514E-2,4.0916502E-3,-9.827798E-3,-1.5146399E-3,-7.0481887E-3,-3.5027225E-4,-7.862704E-3,2.9995697E-4,8.753806E-2,-3.038841E-3,-7.617977E-4,5.0962986E-3,1.0533791E-1,1.6271347E-2,1.11285366E-1,-6.3159033E-3,5.706902E-3,-9.936016E-3,1.4836109E-1,3.144091E-1,1.5741846E-3,6.902647E-3,-7.4920487E-3,2.439277E-3,3.5132426E-3,1.0489423E-2,-2.4694036E-1,2.5583463E-2,-1.4868317E-2,-4.3025813E-3,4.087016E-3,-2.8321285E-2,5.378837E-4,7.2296844E-3,3.958921E-1,2.7818275E-3,4.9096565E-3,1.3313295E-2,-3.2483886E-3,-1.3930216E-2,-1.887233E-2,-9.289061E-3,-2.327068E-2,-1.4219126E-2,-7.5650658E-3,-1.258577E-4,-1.0275359E-2,-1.9765997E-3,1.2900188E-3,-5.1884297E-3,5.4901233E-4,5.962981E-3,-1.8559955E-3,2.0216622E-3,7.915611E-3,1.5577816E-3,1.4711095E-3,7.7831917E-3,8.856118E-3,3.0809408E-3,1.1301039E-2,1.7829586E-2,-5.616756E-3,-1.4918382E-2,1.1033503E-2,4.401701E-4,-4.265691E-3,1.3740485E-3,2.468877E-2,1.4867516E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,29,-1,31,33,35,37,-1,-1,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,-1,65,67,-1,69,71,73,-1,-1,75,77,79,81,83,85,-1,87,89,91,-1,-1,93,95,97,99,-1,101,103,105,-1,-1,107,-1,109,111,113,115,117,-1,119,121,-1,123,125,127,129,131,-1,-1,133,135,-1,137,139,141,-1,143,145,147,149,-1,151,-1,153,-1,-1,-1,155,157,-1,-1,-1,-1,-1,159,161,163,-1,165,167,-1,-1,169,-1,171,173,-1,175,-1,177,-1,-1,-1,179,181,183,-1,185,187,189,191,193,195,-1,-1,-1,-1,197,-1,199,201,-1,203,-1,-1,205,207,-1,209,-1,-1,-1,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,217,219,-1,-1,221,-1,223,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,229,231,-1,-1,-1,233,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.906312E1,2.699833E1,1.1900219E1,2.9827728E0,6.576001E0,7.008114E0,2.2292304E-1,3.320097E0,1.422535E0,2.6471834E0,2.119548E0,2.7886963E-1,6.600483E0,0E0,0E0,0E0,3.496665E0,6.619015E-1,0E0,1.8858919E0,5.49561E-1,1.1319208E0,8.285201E-1,0E0,0E0,2.3100548E0,1.3799553E0,1.2101727E0,8.966675E-1,0E0,5.0992966E-2,1.6469278E0,5.707963E-1,3.152526E-1,1.3195938E-1,1.4178E0,1.7818308E-1,8.4999305E-1,0E0,1.3131709E0,0E0,4.5188427E-1,1.0267811E0,0E0,1.04661435E-1,3.7758636E-1,2.5102952E-1,0E0,0E0,8.040533E-1,3.7633705E-1,3.2702065E-1,3.4002143E-1,4.8688287E-1,1.4298357E-1,0E0,1.786182E-2,1.0044371E0,4.0900922E-1,0E0,0E0,4.6420935E-1,4.7921723E-1,3.86478E-1,1.5774946E0,0E0,2.651682E-1,5.1159286E-1,4.3391085E-1,0E0,0E0,3.8928795E-1,0E0,1.5722403E-1,2.8984672E-1,1.92312E-1,1.6941464E-1,1.3411283E-1,0E0,1.7811963E-1,1.03125855E-1,0E0,2.9175115E-1,4.5493335E-1,1.3603643E-1,1.430191E-1,5.3008273E-2,0E0,0E0,3.723287E-1,9.8055714E-1,0E0,2.2226238E-1,3.7422144E-1,5.3353965E-1,0E0,8.959854E-2,1.291348E-1,2.2553302E-1,6.3643646E-1,0E0,2.873497E-1,0E0,3.472519E-2,0E0,0E0,0E0,2.240467E-1,4.8177433E-1,0E0,0E0,0E0,0E0,0E0,4.1629553E-2,4.756647E-2,1.1134267E-1,0E0,5.2872658E-2,1.0053718E-1,0E0,0E0,9.120992E-2,0E0,4.9063466E-2,8.613539E-2,0E0,5.3693768E-2,0E0,5.7856083E-2,0E0,0E0,0E0,2.2875565E-1,2.6044804E-1,3.3405617E-1,0E0,1.4267969E-1,2.7592897E-2,1.33308E-1,4.6972096E-2,5.941316E-1,1.3670337E-1,0E0,0E0,0E0,0E0,6.383695E-2,0E0,5.907937E-2,4.764223E-1,0E0,5.6659102E-2,0E0,0E0,2.4798572E-1,1.1013079E-1,0E0,2.1811104E-1,0E0,0E0,0E0,0E0,1.1430286E-1,1.05626225E-1,0E0,0E0,5.4187227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.303025E-2,3.1301804E-2,0E0,0E0,5.094424E-2,0E0,3.5522006E-2,0E0,0E0,0E0,2.2607997E-2,1.252265E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6460927E-2,2.4423552E-1,0E0,0E0,0E0,3.991334E-2,0E0,0E0,4.0260315E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,61,61,62,62,63,63,64,64,66,66,67,67,68,68,71,71,73,73,74,74,75,75,76,76,77,77,79,79,80,80,82,82,83,83,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,96,96,97,97,98,98,99,99,101,101,103,103,107,107,108,108,114,114,115,115,116,116,118,118,119,119,122,122,124,124,125,125,127,127,129,129,133,133,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,147,147,149,149,150,150,152,152,155,155,156,156,158,158,163,163,164,164,167,167,175,175,176,176,179,179,181,181,185,185,186,186,193,193,194,194,198,198,201,201],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,30,-1,32,34,36,38,-1,-1,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,-1,66,68,-1,70,72,74,-1,-1,76,78,80,82,84,86,-1,88,90,92,-1,-1,94,96,98,100,-1,102,104,106,-1,-1,108,-1,110,112,114,116,118,-1,120,122,-1,124,126,128,130,132,-1,-1,134,136,-1,138,140,142,-1,144,146,148,150,-1,152,-1,154,-1,-1,-1,156,158,-1,-1,-1,-1,-1,160,162,164,-1,166,168,-1,-1,170,-1,172,174,-1,176,-1,178,-1,-1,-1,180,182,184,-1,186,188,190,192,194,196,-1,-1,-1,-1,198,-1,200,202,-1,204,-1,-1,206,208,-1,210,-1,-1,-1,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,218,220,-1,-1,222,-1,224,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,230,232,-1,-1,-1,234,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.7909248E5,9.7172376E7,1E0,9.31E2,2.4481E4,1.665909E3,3.89E2,3.9935064E0,1.5859042E6,3.7326388E0,6.23694E5,1.8062708E6,-3.1316992E-2,-1.239372E-2,-2.764794E-2,9.83871E-1,1.0013907E1,-5.0783135E-2,2.956111E2,5.5426865E6,5.405423E6,1.2294118E1,3.617117E-2,6.5777734E-2,7.6255713E3,6.355E3,2E0,3.3238492E2,-9.668635E-3,1.6074808E5,3.4698796E0,1.934E3,2.732381E2,5.894636E6,1.084437E6,1E0,2.445E3,-1.4016416E-2,7.075161E2,3.7666537E-2,1E0,5.6926E-3,1.843852E-2,6.923077E-1,1.803948E7,1.1544592E1,-3.1929143E-2,-1.8278925E-2,4.33E2,5.4051723E0,1E0,1.2E1,2.331083E6,1.2397727E1,1.3444636E-2,1.4E1,3.72381E5,3.7760816E5,9.602993E-3,2.3317957E-2,9.777778E0,9.68E2,2.0421052E0,3.0070068E7,4.47477E-3,1.0089981E3,1E0,1.3655363E2,-7.621937E-3,3.8831128E-4,1.2E1,-4.2695575E-3,1E0,1.1939979E8,1.0201707E-3,9E0,7.106795E6,-3.2020617E-2,3.82E2,1.1652361E6,5.824676E-4,3.90225E2,3E0,2.0487332E-3,1E0,1.5539158E3,5.87003E-3,1.405876E-3,1.2562814E0,6.9508715E0,-1.3502848E-3,2.23099E7,1.5026077E7,2.0734E4,8.19227E-4,3.4779343E1,3.9896116E7,1.7E1,6.996357E4,-8.354947E-3,2.5959E4,2.6182918E-2,2.4080592E7,1.4135402E-2,3.64063E-3,2.335258E-2,1.4E1,1.63757E4,5.5863406E-3,-6.0001863E-3,-1.6127607E-2,-1.5943821E-3,-2.9655686E-3,1E0,6.407767E-1,1.6808511E0,-1.0543343E-2,3.1540035E9,2.8E1,7.352528E-3,1.4278679E-3,5.9506125E8,-1.630179E-2,2.335E3,3.39E2,-1.9916672E-2,3.5364E4,-5.668018E-3,5.44E2,-4.3535763E-3,1.0984553E-2,2.831896E-3,2.9080848E6,1.57E2,4.5154482E-1,-2.6035389E-2,5.1604336E7,4.8E2,1.1E1,3.7788504E7,2.1924414E2,2.2503355E1,2.1848662E-2,8.736158E-3,1.1240756E-3,1.1254942E-2,1.1E1,-9.044208E-3,2.829581E10,4.2964826E0,2.4912652E-2,4E0,3.7781075E-2,2.0149276E-2,2E0,4.6463413E0,-3.404038E-2,1.547E3,-9.160771E-3,-1.5732905E-2,-1.6468738E-3,4.4976445E-3,1.407037E2,6.9E2,-2.175964E-2,-1.1202959E-2,1.4473684E-1,4.0916502E-3,-9.827798E-3,-1.5146399E-3,-7.0481887E-3,-3.5027225E-4,-7.862704E-3,2.9995697E-4,1.2E1,6.47E2,-7.617977E-4,5.0962986E-3,7.7983914E4,1.6271347E-2,1.7638037E0,-6.3159033E-3,5.706902E-3,-9.936016E-3,4.6E1,4.797342E2,1.5741846E-3,6.902647E-3,-7.4920487E-3,2.439277E-3,3.5132426E-3,1.0489423E-2,1.196966E6,7E0,-1.4868317E-2,-4.3025813E-3,4.087016E-3,2.8003533E0,5.378837E-4,7.2296844E-3,9.643839E6,2.7818275E-3,4.9096565E-3,1.3313295E-2,-3.2483886E-3,-1.3930216E-2,-1.887233E-2,-9.289061E-3,-2.327068E-2,-1.4219126E-2,-7.5650658E-3,-1.258577E-4,-1.0275359E-2,-1.9765997E-3,1.2900188E-3,-5.1884297E-3,5.4901233E-4,5.962981E-3,-1.8559955E-3,2.0216622E-3,7.915611E-3,1.5577816E-3,1.4711095E-3,7.7831917E-3,8.856118E-3,3.0809408E-3,1.1301039E-2,1.7829586E-2,-5.616756E-3,-1.4918382E-2,1.1033503E-2,4.401701E-4,-4.265691E-3,1.3740485E-3,2.468877E-2,1.4867516E-2],"split_indices":[2,43,60,6,2,12,4,1,68,43,69,1,43,0,0,0,68,71,0,4,43,43,68,0,0,67,2,10,67,0,48,71,44,4,60,9,29,2,0,67,0,15,72,0,68,9,71,0,0,2,69,26,3,43,73,0,3,9,43,0,0,73,10,68,58,0,4,6,71,0,0,8,0,13,7,53,3,5,0,4,60,0,67,8,53,26,48,0,0,68,71,0,9,5,44,0,71,5,3,48,0,9,0,9,0,0,0,3,43,0,0,0,0,0,73,71,68,0,46,0,0,0,46,0,44,44,0,9,0,12,0,0,0,66,0,53,0,7,10,3,7,67,71,0,0,0,0,3,0,46,68,0,8,0,0,32,68,0,10,0,0,0,0,67,2,0,0,71,0,0,0,0,0,0,0,3,2,0,0,48,0,68,0,0,0,0,4,0,0,0,0,0,0,9,3,0,0,0,68,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.044E3,7.94E2,2.5E2,2.43E2,5.51E2,2.39E2,1.1E1,2.08E2,3.5E1,2.82E2,2.69E2,1.3E1,2.26E2,7E0,4E0,4.8E1,1.6E2,2.8E1,7E0,1.9E2,9.2E1,1.29E2,1.4E2,7E0,6E0,1.18E2,1.08E2,2E1,1.4E2,6E0,2.2E1,1.19E2,7.1E1,7.7E1,1.5E1,1.13E2,1.6E1,1.33E2,7E0,1.12E2,6E0,4.6E1,6.2E1,7E0,1.3E1,1.2E2,2E1,1.6E1,6E0,8.4E1,3.5E1,4.5E1,2.6E1,4.7E1,3E1,6E0,9E0,4.6E1,6.7E1,5E0,1.1E1,1.18E2,1.5E1,3.5E1,7.7E1,4E0,4.2E1,5.1E1,1.1E1,7E0,6E0,1.15E2,5E0,8E0,1.2E1,3.9E1,4.5E1,2.9E1,6E0,2.9E1,1.6E1,7E0,1.9E1,1.5E1,3.2E1,2.2E1,8E0,5E0,4E0,3.2E1,1.4E1,5E0,6.2E1,2.7E1,9.1E1,6E0,9E0,1.3E1,2.2E1,7.1E1,6E0,2E1,2.2E1,4.7E1,4E0,4E0,7E0,7E1,4.5E1,4E0,4E0,7E0,5E0,4E0,3.5E1,9E0,3.6E1,9E0,2E1,1.7E1,1.2E1,4E0,1.2E1,1E1,9E0,1E1,5E0,2.6E1,6E0,1.5E1,7E0,4E0,4E0,1.7E1,1.5E1,1E1,4E0,5.2E1,1E1,1E1,1.7E1,7.9E1,1.2E1,5E0,4E0,4E0,9E0,1.7E1,5E0,1.1E1,6E1,6E0,1.4E1,4.3E1,4E0,2.6E1,4.4E1,7E0,3.8E1,8E0,2.7E1,5E0,4E0,1.8E1,1.8E1,1.5E1,5E0,1E1,7E0,7E0,5E0,5E0,4E0,4E0,6E0,1E1,1.6E1,5E0,1E1,1.1E1,6E0,8E0,7E0,6E0,4E0,9E0,4.3E1,4E0,6E0,4E0,6E0,4E0,1.3E1,8E0,7.1E1,6E0,6E0,8E0,9E0,5E0,6E0,5.5E1,5E0,4E0,1E1,7E0,1.9E1,3.8E1,6E0,1.8E1,2E1,6E0,1.2E1,1.3E1,5E0,4E0,6E0,4E0,6E0,9E0,7E0,5E0,6E0,4E0,4E0,5E0,4E0,2.4E1,1.9E1,4E0,4E0,4E0,6.7E1,4E0,5E0,1.7E1,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[8.597817E-3,-1.2879427E-1,4.256468E-1,-3.8591677E-1,-3.969808E-3,4.8857808E-1,-3.875296E-2,-5.740737E-1,-3.0354354E-1,-1.351161E-1,9.396026E-2,6.364543E-2,4.5750412E-1,-4.631766E-1,1.5592837E-1,-4.4338825E-1,-7.1076024E-1,-3.5070378E-1,2.542251E-2,-2.0304939E-1,-1.0437911E-2,5.080184E-2,2.8214893E-1,3.692196E-1,6.972939E-1,-2.5734607E-2,-1.1915106E-2,2.7503145E-1,-6.402389E-3,-4.653785E-1,-1.0974521E-2,-6.29406E-1,-4.9571794E-2,1.263199E-2,-3.7785378E-1,1.9859787E-2,-1.10807136E-1,-1.7515352E-1,-4.2941716E-1,1.7590694E-2,-1.39172375E-2,6.307764E-2,-2.044582E-2,8.2467906E-2,3.35956E-1,1.3044524E-1,4.5310163E-1,7.579595E-1,2.5552644E-3,2.8878017E-3,3.6595058E-1,-2.2936359E-2,-1.2272982E-2,-3.387607E-2,-4.0385985E-1,-3.350037E-1,-6.8834525E-1,-1.1423461E-2,-3.9302424E-2,-2.7000123E-1,-1.2789492E-1,-1.2143544E-2,-2.9357424E-2,-1.623632E-2,8.819292E-3,1.4592235E-1,1.4928196E-2,9.02584E-5,5.4531014E-3,3.8234964E-1,3.935705E-3,7.739058E-2,1.6302979E-2,6.589423E-1,3.963303E-1,4.2970426E-2,2.8703295E-2,2.2502005E-2,8.49337E-3,-2.2833815E-2,-1.067643E-2,-4.057083E-1,-2.4064533E-1,-1.3344708E-2,-8.307296E-1,1.5412492E-3,-7.4951584E-3,-3.0325803E-1,-4.192686E-3,-8.940315E-2,-2.3930645E-1,-9.753107E-3,8.43118E-4,8.251604E-2,2.4233465E-1,-2.1061499E-2,1.0581072E-1,1.8188989E-1,4.331278E-1,1.7360204E-1,3.889039E-2,3.307639E-2,1.3818471E-2,5.214149E-4,4.1759622E-1,-2.628421E-3,-4.3016022E-1,-3.1865135E-2,-3.243929E-1,-5.8177676E-2,-2.1207329E-2,-5.5476143E-3,-1.4923227E-2,-1.2677565E-1,-1.2395965E-2,-4.294914E-3,-2.88374E-1,-4.682604E-2,9.375582E-2,1.2210748E-1,-1.5241143E-2,2.8454947E-1,-2.74521E-3,-6.2461957E-2,9.004178E-2,-7.915614E-2,1.718667E-1,2.2793636E-3,1.2846289E-2,1.1451453E-2,2.2317665E-2,2.1162769E-3,1.1603518E-2,-4.8818752E-2,1.1231229E-1,2.7839504E-3,4.4240287E-1,-4.7453263E-1,-3.1379494E-1,-1.2640013E-2,5.6744467E-2,-2.9149225E-1,-2.0662732E-2,-1.7108446E-1,-5.713807E-2,-6.658633E-2,6.471014E-2,-1.6361587E-2,-8.2384255E-3,1.0217967E-2,-9.620748E-2,5.180747E-3,1.4862325E-3,7.362799E-2,3.0866265E-1,1.6457209E-1,3.666162E-1,-8.473663E-2,1.3333295E-2,1.8417571E-1,-6.650711E-2,-5.2396295E-5,-8.692818E-3,1.1907593E-1,1.209602E-2,3.9888298E-4,-5.4185004E-3,8.2136495E-3,5.5828143E-4,4.9335462E-1,1.9502313E-1,-1.8970046E-2,-2.715469E-2,-7.6681697E-3,-1.55885415E-2,-1.7287824E-3,1.0797878E-2,-1.7378371E-2,-1.0187471E-2,7.745435E-5,-9.295376E-3,2.5393679E-3,-4.1901073E-3,-2.0631547E-4,-4.2035575E-3,5.1161516E-3,-1.6028775E-4,-1.9313481E-3,4.868096E-3,-5.922737E-3,1.4695912E-3,-1.0086013E-3,5.485132E-3,6.548525E-3,1.8309431E-2,1.1172498E-2,2.4375792E-3,1.9502638E-2,8.983798E-3,6.6987835E-3,-5.088571E-3,1.7708457E-3,1.0187459E-2,-7.0623914E-3,2.0943263E-3,7.5206594E-3,9.939047E-5,1.6885199E-2,2.5971701E-2,2.066766E-3,1.5694655E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,-1,51,-1,53,-1,-1,55,-1,57,59,61,63,-1,65,-1,67,69,71,73,75,-1,-1,77,-1,-1,-1,79,81,83,-1,85,87,89,-1,-1,91,-1,93,95,-1,-1,97,-1,99,-1,101,103,-1,-1,-1,-1,-1,-1,105,107,-1,109,-1,-1,111,-1,113,115,-1,117,119,121,123,125,127,129,131,133,-1,-1,-1,135,-1,137,139,141,-1,-1,-1,-1,143,145,-1,147,149,151,153,-1,155,-1,157,159,161,163,-1,-1,-1,-1,-1,-1,165,167,-1,169,171,173,-1,175,177,-1,179,181,183,185,-1,-1,187,189,-1,-1,191,193,195,197,199,-1,201,203,-1,-1,205,-1,-1,-1,-1,-1,207,209,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.827084E1,2.4551277E1,7.3837013E0,3.7005157E0,6.6397686E0,5.5028152E0,2.6336706E0,1.0245762E0,2.7394009E0,1.8697062E0,2.3897562E0,0E0,4.3152046E0,4.9002647E-2,8.098125E-1,7.503605E-2,3.5609818E-1,2.712862E0,1.3148947E0,8.277917E-1,6.492217E-1,1.4935653E0,5.6322813E-1,3.1455154E0,2.1780167E0,0E0,0E0,2.8628445E-1,0E0,9.164715E-2,0E0,4.3002796E-1,0E0,0E0,1.7384796E0,0E0,1.6591617E-1,5.436938E-1,3.4774804E-1,4.3105894E-1,0E0,9.4319725E-1,0E0,3.699687E-2,4.6768665E-1,4.8015428E-1,1.1177711E0,6.4554214E-1,0E0,0E0,1.5307367E-1,0E0,0E0,0E0,7.31715E-2,7.830162E-1,7.2702694E-1,0E0,1.2153871E-1,2.2285032E-1,3.559867E-1,0E0,0E0,2.0653522E-1,0E0,5.118475E-1,4.961245E-1,0E0,0E0,2.8766203E-1,0E0,1.2659828E-1,0E0,2.7820778E-1,7.7317715E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.066866E-1,1.0212688E0,0E0,1.3333883E0,0E0,0E0,8.791113E-2,0E0,1.9125122E-1,1.3116431E-1,0E0,2.612763E-1,9.2200714E-1,4.5001972E-1,5.0699437E-1,5.389855E-1,1.0101345E-1,1.3934946E-1,8.761391E-2,1.7935678E-1,0E0,0E0,0E0,7.7112865E-1,0E0,2.0736885E-1,4.0108418E-1,3.9402008E-2,0E0,0E0,0E0,0E0,1.290747E-1,1.004122E-1,0E0,5.2670717E-2,1.10842735E-1,1.903607E-2,4.294787E-1,0E0,2.3504996E-1,0E0,6.5004814E-1,4.5434594E-1,1.018431E-1,1.20788395E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.214985E-2,9.6289545E-2,0E0,9.6255875E-1,1.0608387E-1,2.082491E-2,0E0,2.4206087E-1,1.334076E-1,0E0,1.4082551E-1,7.646449E-2,2.0815466E-2,3.1768225E-2,0E0,0E0,9.845888E-2,9.008677E-2,0E0,0E0,1.791074E-1,9.0132356E-2,1.0818079E-1,8.0421925E-2,4.595884E-1,0E0,9.223825E-2,1.2283397E-1,0E0,0E0,1.12092614E-1,0E0,0E0,0E0,0E0,0E0,4.0167427E-1,3.3273482E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,31,31,34,34,36,36,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,50,50,54,54,55,55,56,56,58,58,59,59,60,60,63,63,65,65,66,66,69,69,71,71,73,73,74,74,81,81,82,82,84,84,87,87,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,104,104,106,106,107,107,108,108,113,113,114,114,116,116,117,117,118,118,119,119,121,121,123,123,124,124,125,125,126,126,133,133,134,134,136,136,137,137,138,138,140,140,141,141,143,143,144,144,145,145,146,146,149,149,150,150,153,153,154,154,155,155,156,156,157,157,159,159,160,160,163,163,169,169,170,170],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,-1,52,-1,54,-1,-1,56,-1,58,60,62,64,-1,66,-1,68,70,72,74,76,-1,-1,78,-1,-1,-1,80,82,84,-1,86,88,90,-1,-1,92,-1,94,96,-1,-1,98,-1,100,-1,102,104,-1,-1,-1,-1,-1,-1,106,108,-1,110,-1,-1,112,-1,114,116,-1,118,120,122,124,126,128,130,132,134,-1,-1,-1,136,-1,138,140,142,-1,-1,-1,-1,144,146,-1,148,150,152,154,-1,156,-1,158,160,162,164,-1,-1,-1,-1,-1,-1,166,168,-1,170,172,174,-1,176,178,-1,180,182,184,186,-1,-1,188,190,-1,-1,192,194,196,198,200,-1,202,204,-1,-1,206,-1,-1,-1,-1,-1,208,210,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,2.16577E5,2.856934E7,7.7E1,8.1E2,3.3817584E7,1.465E4,9.55188E5,3.541824E2,1.5859042E6,2.8530578E6,6.364543E-2,4.1973075E6,1.7026364E3,7.5491136E-1,2.0376764E-7,7.45E0,6E0,1.8729E4,2E0,7.89E2,1.2294118E1,4.6712E4,7.445455E2,6.292039E0,-2.5734607E-2,-1.1915106E-2,1.5896863E3,-6.402389E-3,2.7546012E0,-1.0974521E-2,1.8987958E0,-4.9571794E-2,1.263199E-2,8.719101E0,1.9859787E-2,7.70947E5,4.39E2,1.7913036E6,5.5426865E6,-1.39172375E-2,1.4392524E0,-2.044582E-2,2.524828E2,1.0039E4,6.103178E8,2.6793549E1,3.9914E4,2.5552644E-3,2.8878017E-3,5.2224144E5,-2.2936359E-2,-1.2272982E-2,-3.387607E-2,7E0,8.81059E5,1.7915686E2,-1.1423461E-2,8.215373E6,1.7566667E2,1.36E2,-1.2143544E-2,-2.9357424E-2,3.0502679E-6,8.819292E-3,7.5491136E-1,1.4603988E6,9.02584E-5,5.4531014E-3,3.2911258E6,3.935705E-3,3.56E3,1.6302979E-2,4.8149962E8,1E0,4.2970426E-2,2.8703295E-2,2.2502005E-2,8.49337E-3,-2.2833815E-2,-1.067643E-2,1.4953704E0,1.8E0,-1.3344708E-2,6.8E1,1.5412492E-3,-7.4951584E-3,1.9E1,-4.192686E-3,2.118835E6,9.836066E-1,-9.753107E-3,2.485E2,4.5158855E6,2.0110958E0,9.751103E6,4.7563504E7,1.935E3,6.4258E4,4.3566666E2,1.0039E4,3.307639E-2,1.3818471E-2,5.214149E-4,3.808001E5,-2.628421E-3,2.2470966E-1,3.6E2,1.04618106E6,-5.8177676E-2,-2.1207329E-2,-5.5476143E-3,-1.4923227E-2,5.9565744E5,3.056087E2,-4.294914E-3,4.5172415E0,5.56E2,1.6E1,5.372237E2,-1.5241143E-2,1.387E3,-2.74521E-3,1.0564016E3,2.5417458E1,1.72959E5,6.328E2,2.2793636E-3,1.2846289E-2,1.1451453E-2,2.2317665E-2,2.1162769E-3,1.1603518E-2,2.73E2,1.99E2,2.7839504E-3,1E0,6E0,5.912179E3,-1.2640013E-2,1.4E1,1.7915686E2,-2.0662732E-2,9E0,2.9276064E3,5.13657E6,5.539245E6,-1.6361587E-2,-8.2384255E-3,5.08E2,1.118178E6,5.180747E-3,1.4862325E-3,1.071E3,1.47E3,3.6245735E0,2.277593E5,3.5E1,1.3333295E-2,9.72E2,6.648E3,-5.2396295E-5,-8.692818E-3,4.7185095E2,1.209602E-2,3.9888298E-4,-5.4185004E-3,8.2136495E-3,5.5828143E-4,1.1251919E6,4.797448E8,-1.8970046E-2,-2.715469E-2,-7.6681697E-3,-1.55885415E-2,-1.7287824E-3,1.0797878E-2,-1.7378371E-2,-1.0187471E-2,7.745435E-5,-9.295376E-3,2.5393679E-3,-4.1901073E-3,-2.0631547E-4,-4.2035575E-3,5.1161516E-3,-1.6028775E-4,-1.9313481E-3,4.868096E-3,-5.922737E-3,1.4695912E-3,-1.0086013E-3,5.485132E-3,6.548525E-3,1.8309431E-2,1.1172498E-2,2.4375792E-3,1.9502638E-2,8.983798E-3,6.6987835E-3,-5.088571E-3,1.7708457E-3,1.0187459E-2,-7.0623914E-3,2.0943263E-3,7.5206594E-3,9.939047E-5,1.6885199E-2,2.5971701E-2,2.066766E-3,1.5694655E-2],"split_indices":[2,43,60,44,2,7,44,46,67,43,43,0,43,4,53,52,69,3,1,6,2,68,1,67,53,0,0,4,0,68,0,68,0,0,69,0,9,2,60,43,0,68,0,4,44,7,73,9,0,0,43,0,0,0,6,9,4,0,12,67,0,0,0,53,0,53,43,0,0,43,0,2,0,5,26,0,0,0,0,0,0,69,68,0,0,0,0,67,0,9,71,0,67,62,53,9,7,2,1,67,44,0,0,0,43,0,53,2,60,0,0,0,0,43,4,0,69,2,3,4,0,2,0,4,73,12,4,0,0,0,0,0,0,0,8,0,6,8,48,0,3,4,0,3,62,9,5,0,0,2,9,0,0,2,2,71,48,0,0,44,44,0,0,4,0,0,0,0,0,43,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,7.64E2,2.51E2,2.49E2,5.15E2,2.21E2,3E1,7.4E1,1.75E2,2.2E2,2.95E2,6E0,2.15E2,9E0,2.1E1,4E1,3.4E1,1.53E2,2.2E1,1.42E2,7.8E1,2.41E2,5.4E1,1.59E2,5.6E1,5E0,4E0,1.5E1,6E0,3.5E1,5E0,3E1,4E0,6E0,1.47E2,5E0,1.7E1,1.28E2,1.4E1,7.2E1,6E0,2.36E2,5E0,1.2E1,4.2E1,4.2E1,1.17E2,5.1E1,5E0,5E0,1E1,2.9E1,6E0,1.9E1,1.1E1,1.31E2,1.6E1,5E0,1.2E1,4.1E1,8.7E1,9E0,5E0,6.1E1,1.1E1,8.6E1,1.5E2,4E0,8E0,3.5E1,7E0,3.5E1,7E0,2.3E1,9.4E1,2E1,3.1E1,5E0,5E0,6E0,5E0,7.3E1,5.8E1,5E0,1.1E1,8E0,4E0,3.4E1,7E0,6.6E1,2.1E1,4E0,5.7E1,5.3E1,3.3E1,1.08E2,4.2E1,8E0,2.7E1,9E0,2.6E1,1.9E1,4E0,5E0,8.9E1,5E0,6.8E1,1.7E1,4.1E1,4E0,7E0,4E0,3E1,4.4E1,2.2E1,6E0,1.5E1,3.8E1,1.9E1,4.9E1,4E0,2.9E1,4E0,7.9E1,2.9E1,1.1E1,3.1E1,4E0,4E0,7E0,2E1,4E0,5E0,1.2E1,1.4E1,6E0,8.3E1,4.7E1,2.1E1,4E0,1.3E1,3.5E1,6E0,2.6E1,1.8E1,1.3E1,9E0,8E0,7E0,1.8E1,2E1,1.4E1,5E0,4E1,9E0,1.3E1,1.6E1,7.5E1,4E0,1.8E1,1.1E1,7E0,4E0,2.1E1,1E1,7E0,5E0,8E0,6E0,6.8E1,1.5E1,3.3E1,1.4E1,4E0,1.7E1,9E0,4E0,1.4E1,2.1E1,4E0,2.2E1,4E0,1.4E1,4E0,9E0,5E0,4E0,1.2E1,6E0,1.6E1,4E0,1.3E1,2.7E1,4E0,5E0,7E0,6E0,1.1E1,5E0,7E0,6.8E1,4E0,1.4E1,6E0,5E0,1.5E1,6E0,2.6E1,4.2E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"211","size_leaf_vector":"1"}},{"base_weights":[3.220995E-3,-1.2848191E-1,3.9728412E-1,-2.554451E-1,4.9032506E-2,-3.9518216E-1,4.3124267E-1,-3.9445123E-1,-1.5593712E-1,-6.254721E-2,1.4237507E-1,-2.3489894E-2,-7.279638E-3,1.9914469E-1,5.425681E-1,-3.6553645E-1,-8.639063E-1,-9.621763E-2,-5.5182266E-1,-4.52914E-2,-2.1142263E-2,1.6072036E-1,-7.876543E-2,1.0174252E-1,4.7259173E-1,5.721472E-1,-9.554424E-3,-5.348445E-1,-3.0329442E-1,-1.1962927E-2,-5.6550313E-2,4.4917592E-1,-1.1809183E-1,-7.22109E-1,-2.87986E-1,-9.610479E-2,2.4908427E-2,1.0262005E-1,2.3670018E-1,-7.750425E-3,1.5798523E-3,-8.152097E-2,1.5307474E-1,1.6141608E-2,2.6479976E-2,5.93647E-2,5.4065937E-1,-1.2771418E-2,-2.6180707E-2,-9.120253E-2,-3.320098E-1,1.0742796E-2,2.6609905E-2,-2.6936036E-1,-3.7991833E-2,-1.819198E-2,-3.6015943E-2,-2.7202473E-3,-1.6906269E-2,-6.44165E-2,-2.9478228E-1,-1.7224561E-2,1.4185068E-1,3.2275656E-1,7.220895E-2,-9.732608E-3,2.674715E-1,-8.080454E-3,-1.0206928E-2,3.3646703E-1,6.766417E-2,4.661095E-1,7.7206975E-1,9.574252E-3,-1.9897221E-1,-2.5316775E-1,-3.908813E-1,-3.996865E-1,-1.543419E-1,-1.5883903E-1,1.4510805E-2,-2.4133502E-2,-1.9332422E-1,-1.9171244E-2,-3.1213132E-3,-5.4154865E-2,1.1744698E-1,7.7865735E-2,1.23371985E-2,2.119341E-2,8.293495E-3,9.585977E-2,-1.4595016E-2,3.1908882E-1,1.5934035E-1,-3.2006756E-3,2.351885E-3,8.581722E-3,1.9232757E-2,1.1739641E-1,-6.5108836E-3,7.5008446E-1,4.0807354E-1,8.432998E-1,2.321695E-2,-1.2538147E-2,-1.3313857E-3,-2.7091724E-1,-1.4848002E-3,-2.1639124E-2,-2.8742635E-1,-2.3689473E-1,-2.4222445E-2,-5.7015553E-2,-2.2496103E-1,-7.381873E-2,-2.8952903E-1,4.1781794E-2,-1.5989126E-1,1.8479452E-2,-1.0176079E-1,-2.4506535E-1,-2.662048E-3,-6.7606554E-3,-1.4837989E-1,1.978625E-3,8.477528E-3,5.658173E-3,5.9344375E-4,1.120397E-1,-5.528606E-3,2.5520054E-1,4.1095024E-1,1.9344163E-1,-9.759751E-5,-1.0161004E-3,1.7425226E-1,3.885535E-2,1.9072609E-2,2.0911404E-1,4.7076926E-1,2.3250774E-2,4.1472994E-2,-1.5806496E-2,-2.315277E-1,-2.3873536E-1,-2.232034E-2,-3.4274857E-3,-1.586334E-2,-2.580456E-3,-9.651801E-3,-1.2431198E-2,-1.6153818E-4,-8.438632E-3,-2.1150062E-2,-1.576765E-2,-6.363268E-3,1.2111861E-1,-2.8508376E-2,-1.1337749E-2,1.9923884E-3,-1.4758602E-2,6.5053906E-3,-1.554788E-1,9.3379774E-4,-4.6505737E-3,-1.4076071E-2,-5.6894593E-2,4.938022E-3,-5.731622E-4,-1.5534965E-2,1.4330468E-1,-2.0489521E-2,1.6641114E-2,1.9518864E-1,2.1520257E-2,9.105019E-3,1.6813251E-3,1.0759529E-2,2.3198147E-1,1.0593004E-1,2.789836E-1,7.8698323E-4,5.118178E-1,2.3247096E-1,-4.2572315E-3,-1.1924129E-2,-4.202423E-3,-1.24546895E-2,-4.7498825E-3,3.495662E-3,1.7887024E-3,-3.4026618E-3,6.856046E-3,-6.184847E-4,3.8336935E-3,-2.7734078E-3,-6.407575E-3,4.454952E-4,-3.4182416E-3,-1.1632907E-2,-7.1993507E-3,-8.0827257E-4,2.2610591E-3,8.585639E-3,2.2618072E-3,-6.13617E-3,4.183312E-3,1.0350049E-2,5.4446855E-3,1.3324968E-2,1.1464513E-3,6.871469E-3,1.7427403E-2,4.469738E-3,2.5338389E-2,1.3185349E-2,-2.4528322E-3,1.6875928E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,71,-1,-1,73,75,-1,-1,77,79,-1,-1,-1,-1,81,83,85,87,89,91,-1,93,-1,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,121,-1,-1,123,125,127,-1,-1,-1,129,-1,131,133,-1,-1,-1,-1,135,-1,137,139,141,-1,-1,-1,143,-1,-1,145,147,-1,149,151,153,155,157,159,161,163,165,-1,167,169,-1,-1,-1,-1,171,-1,173,175,177,-1,-1,179,-1,-1,181,183,-1,-1,-1,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,-1,193,195,-1,-1,197,-1,199,-1,-1,-1,201,-1,-1,-1,203,205,-1,207,-1,-1,-1,-1,209,211,213,-1,215,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.283259E1,1.7208862E1,7.0327263E0,6.0938263E0,3.3313334E0,2.1593273E-1,6.2223587E0,2.1954384E0,6.1174884E0,9.7463465E-1,7.185576E-1,0E0,0E0,2.1092985E0,3.775711E0,1.6924877E0,1.8618231E0,2.773095E0,1.3195868E0,5.053537E-1,0E0,6.8643665E-1,1.4648151E-1,5.863254E-1,5.9955597E-2,2.8740883E0,0E0,2.2191334E-1,7.6754856E-1,0E0,0E0,1.1331332E-1,2.6554205E0,7.042885E-2,2.4636161E-1,4.9726284E-1,2.980709E-1,5.9929144E-1,1.0062056E0,0E0,0E0,9.144924E-2,7.2908795E-1,0E0,0E0,0E0,2.2968788E0,0E0,0E0,5.880052E-1,4.3481922E-1,0E0,0E0,1.0775442E0,9.2230046E-1,0E0,0E0,0E0,0E0,3.692152E-1,2.7236444E-1,2.3016608E-1,1.0737461E-1,1.3191128E-1,7.9218E-1,0E0,3.1194973E-1,0E0,3.6018983E-2,1.03150606E-1,3.6925775E-1,1.6551208E0,2.6735115E-1,0E0,1.5237588E-1,1.9529629E-1,3.7856007E-1,5.8584166E-1,2.7584398E-1,4.6752465E-1,4.9180454E-1,1.8712448E-1,1.05243266E-1,0E0,0E0,1.5976277E-1,4.142897E-2,3.3231974E-2,0E0,0E0,0E0,2.8889984E-1,0E0,1.4748669E-1,1.2704086E-1,0E0,0E0,0E0,0E0,2.2573695E-1,0E0,2.356205E-1,1.1614208E0,4.7662735E-2,0E0,0E0,0E0,5.544424E-2,0E0,0E0,1.7053485E-1,2.690333E-1,0E0,1.565683E-1,2.3355317E-1,1.566447E-1,8.6580396E-2,4.9977022E-1,2.5365123E-1,1.52524E-1,1.3418582E-1,7.263613E-2,0E0,1.485198E-1,3.023915E-1,0E0,0E0,0E0,0E0,3.0955702E-1,0E0,1.0613406E-1,9.594011E-2,1.0925019E-1,0E0,0E0,5.4309845E-2,0E0,0E0,3.441807E-1,6.244354E-1,0E0,0E0,0E0,1.0215378E-1,9.401369E-2,0E0,0E0,0E0,1.2542138E-1,0E0,0E0,0E0,0E0,6.57149E-2,0E0,0E0,1.5611923E-1,1.7368001E-1,0E0,0E0,9.206462E-2,0E0,9.1257304E-2,0E0,0E0,0E0,6.743945E-2,0E0,0E0,0E0,2.317313E-1,1.2351875E-1,0E0,4.0075958E-2,0E0,0E0,0E0,0E0,3.04268E-2,3.450325E-2,3.0365694E-1,0E0,3.7445068E-1,5.124518E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,46,46,49,49,50,50,53,53,54,54,59,59,60,60,61,61,62,62,63,63,64,64,66,66,68,68,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,86,86,87,87,91,91,93,93,94,94,99,99,101,101,102,102,103,103,107,107,110,110,111,111,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,124,124,129,129,131,131,132,132,133,133,136,136,139,139,140,140,144,144,145,145,149,149,154,154,157,157,158,158,161,161,163,163,167,167,171,171,172,172,174,174,179,179,180,180,181,181,183,183,184,184],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,72,-1,-1,74,76,-1,-1,78,80,-1,-1,-1,-1,82,84,86,88,90,92,-1,94,-1,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,122,-1,-1,124,126,128,-1,-1,-1,130,-1,132,134,-1,-1,-1,-1,136,-1,138,140,142,-1,-1,-1,144,-1,-1,146,148,-1,150,152,154,156,158,160,162,164,166,-1,168,170,-1,-1,-1,-1,172,-1,174,176,178,-1,-1,180,-1,-1,182,184,-1,-1,-1,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,-1,194,196,-1,-1,198,-1,200,-1,-1,-1,202,-1,-1,-1,204,206,-1,208,-1,-1,-1,-1,210,212,214,-1,216,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,7.785302E5,1E0,4.97E2,9.48E2,2.3308511E5,8.5023944E2,4.5E1,2.3E1,3.4E1,8.695652E0,-2.3489894E-2,-7.279638E-3,3.1993368E6,3.206931E2,3E1,5.5E1,2.74614E5,6.5139695E-3,2.331083E6,-2.1142263E-2,4.2210345E2,2.0394794E8,1E0,5.941442E6,3.3817584E7,-9.554424E-3,1.957E3,1.2051282E0,-1.1962927E-2,-5.6550313E-2,1.9E1,1.7302156E5,3.9401392E6,2.12791E5,1.4992306E8,2.9254214E2,1.8992E4,1.5043378E0,-7.750425E-3,1.5798523E-3,2.0939393E0,1.1057851E1,1.6141608E-2,2.6479976E-2,5.93647E-2,4.391553E6,-1.2771418E-2,-2.6180707E-2,1.9E1,8.6875E0,1.0742796E-2,2.6609905E-2,1.4912975E-1,7.69E2,-1.819198E-2,-3.6015943E-2,-2.7202473E-3,-1.6906269E-2,4.135135E0,1.01573E5,1E0,2.9816E4,9.8933E4,2.511E3,-9.732608E-3,3.875E0,-8.080454E-3,1.7909248E5,4.3566666E2,1.2544625E5,1.307E3,1.6808511E0,9.574252E-3,1E0,4.58E2,1.463E2,1.01E2,2.3412812E0,2E0,7.955228E9,1E0,4.3460325E6,-1.9171244E-2,-3.1213132E-3,6.83E2,2.0833333E0,1.4067796E0,1.23371985E-2,2.119341E-2,8.293495E-3,1E0,-1.4595016E-2,3.18E3,3.1206896E0,-3.2006756E-3,2.351885E-3,8.581722E-3,1.9232757E-2,5.595208E2,-6.5108836E-3,3.256391E1,3.3580637E8,4.029152E8,2.321695E-2,-1.2538147E-2,-1.3313857E-3,7.2864324E-1,-1.4848002E-3,-2.1639124E-2,4.046E3,1.0130841E1,-2.4222445E-2,1.32476E5,6.5162756E2,1E0,1E0,2.1340206E0,1.82E3,8.051603E7,1.559733E6,4.5650104E2,-2.662048E-3,4.49239E6,6.0052995E6,1.978625E-3,8.477528E-3,5.658173E-3,5.9344375E-4,3.472E3,-5.528606E-3,1.9197379E9,1.9E1,7.755551E7,-9.759751E-5,-1.0161004E-3,3.0588236E0,3.885535E-2,1.9072609E-2,1.260841E2,1E0,2.3250774E-2,4.1472994E-2,-1.5806496E-2,4.4908694E2,2.855464E5,-2.232034E-2,-3.4274857E-3,-1.586334E-2,2.5055911E2,-9.651801E-3,-1.2431198E-2,-1.6153818E-4,-8.438632E-3,1.2707424E0,-1.576765E-2,-6.363268E-3,4.41537E6,6.8808866E0,-1.1337749E-2,1.9923884E-3,4.48E2,6.5053906E-3,8.7601E4,9.3379774E-4,-4.6505737E-3,-1.4076071E-2,5.342944E6,4.938022E-3,-5.731622E-4,-1.5534965E-2,5.4007E7,2.5E0,1.6641114E-2,3.221296E6,2.1520257E-2,9.105019E-3,1.6813251E-3,1.0759529E-2,1.5E1,5.657353E1,2.1298597E0,7.8698323E-4,7.61E2,1.4595818E7,-4.2572315E-3,-1.1924129E-2,-4.202423E-3,-1.24546895E-2,-4.7498825E-3,3.495662E-3,1.7887024E-3,-3.4026618E-3,6.856046E-3,-6.184847E-4,3.8336935E-3,-2.7734078E-3,-6.407575E-3,4.454952E-4,-3.4182416E-3,-1.1632907E-2,-7.1993507E-3,-8.0827257E-4,2.2610591E-3,8.585639E-3,2.2618072E-3,-6.13617E-3,4.183312E-3,1.0350049E-2,5.4446855E-3,1.3324968E-2,1.1464513E-3,6.871469E-3,1.7427403E-2,4.469738E-3,2.5338389E-2,1.3185349E-2,-2.4528322E-3,1.6875928E-2],"split_indices":[2,43,17,2,2,48,67,6,3,3,61,0,0,43,73,70,0,5,53,43,0,67,7,27,60,7,0,9,68,0,0,0,43,60,12,5,70,9,69,0,0,68,71,0,0,0,43,0,0,0,73,0,0,53,2,0,0,0,0,69,1,28,1,1,2,0,69,0,43,67,48,44,68,0,8,2,70,0,68,32,46,26,60,0,0,2,68,68,0,0,0,79,0,44,68,0,0,0,0,67,0,73,7,7,0,0,0,73,0,0,44,71,0,1,4,8,30,68,2,7,9,48,0,43,43,0,0,0,0,10,0,46,3,5,0,0,68,0,0,71,6,0,0,0,66,60,0,0,0,4,0,0,0,0,68,0,0,60,73,0,0,2,0,9,0,0,0,60,0,0,0,7,68,0,60,0,0,0,0,3,71,53,0,8,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,7.62E2,2.54E2,4.44E2,3.18E2,1E1,2.44E2,1.84E2,2.6E2,1.45E2,1.73E2,6E0,4E0,8E1,1.64E2,1.75E2,9E0,2.27E2,3.3E1,1.4E2,5E0,1.6E2,1.3E1,6E1,2E1,1.58E2,6E0,4.5E1,1.3E2,4E0,5E0,8E0,2.19E2,1.9E1,1.4E1,8.1E1,5.9E1,9.2E1,6.8E1,7E0,6E0,1.3E1,4.7E1,1.1E1,9E0,5E0,1.53E2,6E0,3.9E1,1.6E1,1.14E2,4E0,4E0,7.5E1,1.44E2,4E0,1.5E1,4E0,1E1,7.1E1,1E1,4.4E1,1.5E1,1E1,8.2E1,4E0,6.4E1,5E0,8E0,1.4E1,3.3E1,1.18E2,3.5E1,4E0,1.2E1,5.1E1,6.3E1,3.4E1,4.1E1,4.3E1,1.01E2,5.5E1,1.6E1,6E0,4E0,3.5E1,9E0,1.1E1,4E0,4E0,6E0,7.8E1,4E0,4.2E1,2.2E1,4E0,4E0,6E0,8E0,2.7E1,6E0,1.8E1,1E2,2.6E1,9E0,8E0,4E0,4.7E1,4E0,3.4E1,2.9E1,1.6E1,1.8E1,1.8E1,2.3E1,2.7E1,1.6E1,8.8E1,1.3E1,3.6E1,1.9E1,1.1E1,5E0,2.4E1,1.1E1,5E0,4E0,6E0,5E0,7.3E1,5E0,2.7E1,1.5E1,1.8E1,4E0,8E0,1.9E1,1.3E1,5E0,2.5E1,7.5E1,5E0,2.1E1,1.4E1,3.3E1,2.5E1,4E0,7E0,9E0,1.4E1,4E0,1.9E1,4E0,8E0,1.9E1,1.1E1,5E0,4.1E1,4.7E1,9E0,4E0,2.9E1,7E0,1.3E1,6E0,4E0,7E0,1.7E1,7E0,7E0,4E0,5.9E1,1.4E1,8E0,1.9E1,1.1E1,4E0,4E0,1.4E1,9E0,1E1,1.8E1,7E0,6.3E1,1.2E1,6E0,2.7E1,5E0,2E1,6E0,8E0,9E0,1E1,3.4E1,7E0,1E1,3.7E1,4E0,2.5E1,8E0,5E0,4E0,1.3E1,1.9E1,4E1,9E0,5E0,5E0,1.4E1,4E0,5E0,4E0,6E0,1.1E1,7E0,5.3E1,1E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-4.4613914E-3,-1.9807957E-1,2.0014012E-1,-3.745944E-1,-9.339277E-2,2.4397902E-1,-5.112069E-1,-4.272029E-1,-2.056768E-1,-2.4976265E-1,-3.2554865E-2,1.2366045E-1,5.148828E-1,-3.2893173E-2,-3.8525602E-1,-3.9716187E-1,-6.6508645E-1,-2.5036967E-1,-7.9253405E-2,-1.6846828E-1,-4.205584E-1,-6.1155513E-2,7.092352E-2,3.402724E-1,6.586562E-2,6.2717986E-1,2.866561E-1,-8.778473E-3,-1.9504804E-2,-4.8264036E-1,-2.6317227E-1,-7.724392E-3,-8.973108E-1,-2.898917E-1,-5.6628706E-3,1.678695E-3,-1.2912284E-1,-2.0375587E-1,4.246028E-2,-3.2448512E-1,-3.3945143E-2,-4.4609047E-2,-3.218436E-1,1.6104831E-1,2.254725E-2,2.3030083E-1,4.8914957E-1,9.570672E-2,-2.0294215E-1,5.04491E-1,8.890067E-1,3.494736E-1,-3.648233E-3,-2.8487349E-2,-2.0430839E-2,1.521193E-2,-3.3034527E-1,-5.3707164E-2,-1.846122E-2,-1.6118526E-2,-1.0319226E-2,-2.2626396E-3,-8.056722E-3,-2.507658E-1,-8.392392E-2,-2.1121674E-4,4.1767093E-3,-9.395055E-3,-1.6742427E-2,6.4748465E-3,-9.873822E-2,-2.2507835E-2,-7.288339E-3,4.5214454E-3,1.0523641E-2,-6.8514915E-3,6.300965E-2,2.8972468E-1,1.711443E-2,2.4119057E-2,1.30020855E-2,1.7322531E-1,2.6793553E-2,-9.353181E-2,-3.9921367E-1,6.3372624E-1,2.465392E-1,1.0068929E0,1.730918E-2,1.009027E-3,3.93158E-1,-2.7900547E-1,-2.2283504E-2,-2.7296433E-1,-3.8428288E-3,-6.0890834E-3,1.192751E-3,5.172601E-2,-1.398573E-1,-1.1622585E-1,9.600464E-3,2.4009965E-2,7.501921E-3,3.495059E-1,1.7546707E-1,-1.2547317E-3,3.0870188E-3,2.0642166E-1,1.5180469E-2,-1.3186507E-1,6.2306833E-2,-1.759049E-1,1.451697E-3,-2.3611257E-2,-9.586669E-3,4.4171327E-1,7.382635E-1,2.8708488E-1,1.6192159E-3,5.0096635E-2,2.3613272E-2,3.2663527E-1,2.6048169E-2,-3.1818485E-1,-4.0383595E-3,-1.7529752E-2,-2.1945564E-1,7.809879E-2,-1.3863076E-1,-2.4217283E-2,-2.1805292E-1,-1.7754853E-1,-4.014939E-3,8.396162E-2,-1.6245382E-2,6.5664626E-3,4.091542E-1,1.8692205E-3,1.3185545E-2,2.1174159E-2,1.8297236E-1,-4.3492056E-2,7.360973E-3,9.450751E-3,-2.2338599E-1,7.9136506E-2,-1.0171062E-2,-1.1788716E-2,-3.110122E-3,5.527112E-1,1.9669945E-3,3.607859E-2,1.8216664E-2,4.285529E-3,3.4567493E-1,7.658027E-3,1.8598875E-2,-1.1427673E-2,-3.8196558E-1,-6.8128845E-3,-1.3173806E-2,4.3001838E-2,2.0977922E-1,-1.3725284E-3,-1.0155769E-2,1.4527359E-3,-3.980914E-3,-2.9408297E-1,-1.736542E-3,-2.4079856E-1,-1.0260431E-1,6.268127E-2,-1.24915786E-1,5.71508E-3,1.2670245E-3,-3.16025E-3,2.1106717E-3,2.3716556E-2,1.1725698E-2,1.7361093E-2,2.2473401E-1,7.4811797E-3,-6.7882463E-3,-6.691366E-3,6.303831E-3,-1.4504919E-2,-4.063392E-3,9.252638E-3,1.5028353E-1,1.269861E-2,2.9399319E-2,1.801399E-2,8.278784E-3,-1.9761374E-2,-9.464315E-3,-6.204706E-4,5.1875E-3,4.5663933E-3,1.1462093E-2,-6.2363097E-3,-1.8219126E-2,-9.189131E-3,-1.8190295E-2,-5.0416205E-4,-6.594905E-3,6.896549E-4,6.8944483E-3,-2.5300044E-3,-8.050146E-3,5.3206435E-3,-3.3448602E-3,1.1482408E-2,2.964767E-3,-4.5070266E-3,3.7624273E-3,-4.783242E-3,3.7754192E-3,-7.2395295E-4,8.979616E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,-1,-1,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,87,89,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,93,95,-1,-1,-1,-1,97,99,-1,-1,-1,-1,-1,101,103,105,-1,-1,107,109,111,113,115,117,119,-1,-1,121,123,-1,125,-1,-1,-1,127,129,131,-1,133,-1,135,137,-1,-1,139,141,143,145,147,-1,-1,-1,149,151,153,-1,-1,-1,155,-1,157,-1,-1,159,161,163,165,167,169,171,173,175,-1,177,-1,-1,-1,179,181,-1,183,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,-1,-1,193,-1,-1,195,197,-1,-1,-1,-1,199,-1,201,203,205,207,-1,-1,-1,-1,-1,-1,209,211,213,-1,-1,-1,-1,-1,215,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0248436E1,9.610933E0,1.5529947E1,1.636116E0,3.1217172E0,1.5125486E1,4.5907068E-1,7.8635406E-1,2.5136828E-1,1.2065444E0,7.0870996E-1,4.03376E0,3.4851875E0,0E0,6.400156E-2,1.3962612E0,1.5842981E0,1.4124036E-1,8.539321E-2,4.8501372E-1,6.369829E-1,7.92261E-1,2.2308365E-1,1.0006618E0,2.0809536E0,2.6032143E0,1.153008E0,0E0,0E0,1.6039467E-1,2.2577612E0,0E0,8.554096E-1,2.412486E-2,0E0,0E0,2.7740464E-2,2.8878593E-1,2.2963624E-2,3.6514282E-2,0E0,4.9006712E-1,2.2158325E-1,5.360627E-2,2.5130102E-1,5.144441E-1,4.452467E-2,1.2347145E0,5.209882E-1,2.0773048E0,1.3811302E0,5.925355E-1,0E0,0E0,0E0,0E0,2.477827E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2238097E-1,9.2722684E-2,0E0,0E0,0E0,0E0,6.164307E-1,4.851762E-1,0E0,0E0,0E0,0E0,0E0,1.08491085E-1,1.5724778E-1,2.3821253E-2,0E0,0E0,5.687363E-1,7.068926E-1,1.9079934E-1,8.695078E-2,5.5708885E-1,1.9573855E-1,3.2915497E-1,0E0,0E0,2.3590374E-1,2.6494098E-1,0E0,1.10440016E-1,0E0,0E0,0E0,3.6726367E-1,1.9365522E-1,5.635351E-1,0E0,5.4458227E-2,0E0,1.8923283E-1,1.814073E-1,0E0,0E0,4.477644E-1,1.7608398E-1,2.995784E-1,5.0230813E-1,7.7501416E-2,0E0,0E0,0E0,7.739277E-1,8.8378906E-2,1.95642E-1,0E0,0E0,0E0,2.8758144E-1,0E0,5.007553E-2,0E0,0E0,7.618415E-2,2.8213874E-1,7.306486E-2,3.7193272E-2,1.6731554E-1,2.2652495E-1,2.4973874E-1,1.6830124E-2,4.835005E-2,0E0,1.1251593E-1,0E0,0E0,0E0,5.7705927E-1,8.019138E-2,0E0,2.1570726E-1,1.4412886E-1,4.808964E-1,0E0,0E0,0E0,1.5047073E-1,0E0,0E0,0E0,0E0,4.0616393E-2,0E0,0E0,0E0,3.398776E-2,0E0,0E0,2.0091042E-1,3.0746281E-2,0E0,0E0,0E0,0E0,9.035039E-2,0E0,1.0109317E-1,9.414825E-2,8.0435514E-2,2.852711E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.6636644E-1,2.4016619E-1,9.339764E-2,0E0,0E0,0E0,0E0,0E0,4.1713798E-1,3.5421288E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,56,56,63,63,64,64,69,69,70,70,76,76,77,77,78,78,81,81,82,82,83,83,84,84,85,85,86,86,87,87,90,90,91,91,93,93,97,97,98,98,99,99,101,101,103,103,104,104,107,107,108,108,109,109,110,110,111,111,115,115,116,116,117,117,121,121,123,123,126,126,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,136,136,140,140,141,141,143,143,144,144,145,145,149,149,154,154,158,158,161,161,162,162,167,167,169,169,170,170,171,171,172,172,179,179,180,180,181,181,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,-1,-1,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,88,90,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,94,96,-1,-1,-1,-1,98,100,-1,-1,-1,-1,-1,102,104,106,-1,-1,108,110,112,114,116,118,120,-1,-1,122,124,-1,126,-1,-1,-1,128,130,132,-1,134,-1,136,138,-1,-1,140,142,144,146,148,-1,-1,-1,150,152,154,-1,-1,-1,156,-1,158,-1,-1,160,162,164,166,168,170,172,174,176,-1,178,-1,-1,-1,180,182,-1,184,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,-1,-1,194,-1,-1,196,198,-1,-1,-1,-1,200,-1,202,204,206,208,-1,-1,-1,-1,-1,-1,210,212,214,-1,-1,-1,-1,-1,216,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,5.1987316E7,1.339646E6,4.97E2,1.0236667E3,4.4522205E6,4.6463413E0,8.0509944E5,4.325E0,2.331083E6,7E0,1.6007428E7,-3.2893173E-2,1E0,2.55E2,1.3E1,6.54E2,1.0526316E0,2.9652428E6,7.668863E6,2.829581E10,2.1111E4,1.0254199E10,9.932432E0,1.9180963E3,4.24038E8,-8.778473E-3,-1.9504804E-2,8.0833334E-1,1.1E1,-7.724392E-3,7.3E1,8.147158E4,-5.6628706E-3,1.678695E-3,4.03E2,2.0399521E2,1.3E1,1.1967312E3,-3.3945143E-2,1E0,2.2084616E2,5.405423E6,9.693913E3,1.5723623E0,9.244374E6,1E1,4E1,8E0,1.4497429E7,1.2841364E3,-3.648233E-3,-2.8487349E-2,-2.0430839E-2,1.521193E-2,1.3439851E7,-5.3707164E-2,-1.846122E-2,-1.6118526E-2,-1.0319226E-2,-2.2626396E-3,-8.056722E-3,2.5882354E0,6.920598E7,-2.1121674E-4,4.1767093E-3,-9.395055E-3,-1.6742427E-2,3.493E3,3.2E1,-2.2507835E-2,-7.288339E-3,4.5214454E-3,1.0523641E-2,-6.8514915E-3,1.273801E6,1.02E2,8.328548E6,2.4119057E-2,1.30020855E-2,8E0,2.2783158E5,1.8285715E1,6.001692E-7,1.7942307E8,4.9735293E0,1.2816234E7,1.730918E-2,1.009027E-3,3.8181802E6,3.1158695E6,-2.2283504E-2,1.1967312E3,-3.8428288E-3,-6.0890834E-3,1.192751E-3,1E0,3.074398E0,4.39776E6,9.600464E-3,2.0487332E-3,7.501921E-3,1.357E3,1.9539816E7,-1.2547317E-3,3.0870188E-3,9.63381E5,1.94261E5,3.63E2,2.277593E5,2.0312773E8,1.451697E-3,-2.3611257E-2,-9.586669E-3,5.526E3,1.7103828E7,2.9697892E-1,1.6192159E-3,5.0096635E-2,2.3613272E-2,2.337765E3,2.6048169E-2,8E0,-4.0383595E-3,-1.7529752E-2,3.2E1,8.15736E7,1E0,7.579E3,9.61E2,3.4919708E2,1.3E1,1.411E3,2.5384614E0,6.5664626E-3,5.55E2,1.8692205E-3,1.3185545E-2,2.1174159E-2,1.977157E7,6.4356956E0,7.360973E-3,9.0927E4,6.808571E2,4.5960168E2,-1.0171062E-2,-1.1788716E-2,-3.110122E-3,1.25E0,1.9669945E-3,3.607859E-2,1.8216664E-2,4.285529E-3,8.255237E1,7.658027E-3,1.8598875E-2,-1.1427673E-2,2.8998098E1,-6.8128845E-3,-1.3173806E-2,3.388854E2,8.62E2,-1.3725284E-3,-1.0155769E-2,1.4527359E-3,-3.980914E-3,2.866353E2,-1.736542E-3,9.455644E5,1E0,9.31E2,1.5E1,5.71508E-3,1.2670245E-3,-3.16025E-3,2.1106717E-3,2.3716556E-2,1.1725698E-2,2.3600838E6,5.2409735E-7,2.4796897E3,-6.7882463E-3,-6.691366E-3,6.303831E-3,-1.4504919E-2,-4.063392E-3,7.026624E7,4.8709216E5,1.269861E-2,2.9399319E-2,1.801399E-2,8.278784E-3,-1.9761374E-2,-9.464315E-3,-6.204706E-4,5.1875E-3,4.5663933E-3,1.1462093E-2,-6.2363097E-3,-1.8219126E-2,-9.189131E-3,-1.8190295E-2,-5.0416205E-4,-6.594905E-3,6.896549E-4,6.8944483E-3,-2.5300044E-3,-8.050146E-3,5.3206435E-3,-3.3448602E-3,1.1482408E-2,2.964767E-3,-4.5070266E-3,3.7624273E-3,-4.783242E-3,3.7754192E-3,-7.2395295E-4,8.979616E-3],"split_indices":[2,43,60,9,2,67,62,68,60,69,43,3,66,0,15,2,0,2,71,43,5,46,9,46,69,67,47,0,0,68,10,0,0,43,0,0,2,4,3,48,0,26,67,43,62,68,60,8,3,32,62,4,0,0,0,0,60,0,0,0,0,0,0,71,7,0,0,0,0,44,3,0,0,0,0,0,9,0,9,0,0,32,43,69,52,12,68,9,0,0,43,60,0,48,0,0,0,19,71,9,0,53,0,2,12,0,0,60,1,0,48,7,0,0,0,2,62,53,0,0,0,67,0,8,0,0,10,7,8,44,2,70,8,44,69,0,44,0,0,0,5,69,0,1,4,67,0,0,0,68,0,0,0,0,73,0,0,0,73,0,0,4,44,0,0,0,0,67,0,62,8,2,3,0,0,0,0,0,0,62,52,48,0,0,0,0,0,7,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.014E3,5.21E2,4.93E2,1.93E2,3.28E2,4.65E2,2.8E1,1.46E2,4.7E1,9.1E1,2.37E2,3.23E2,1.42E2,9E0,1.9E1,1.32E2,1.4E1,3.4E1,1.3E1,6.3E1,2.8E1,1.86E2,5.1E1,6.7E1,2.56E2,9.4E1,4.8E1,4E0,1.5E1,7.9E1,5.3E1,5E0,9E0,2.5E1,9E0,4E0,9E0,5.4E1,9E0,2.3E1,5E0,1.76E2,1E1,1.7E1,3.4E1,4E1,2.7E1,2.31E2,2.5E1,6.6E1,2.8E1,4.1E1,7E0,1.5E1,6.4E1,5E0,4.8E1,5E0,4E0,1.1E1,1.4E1,4E0,5E0,3.8E1,1.6E1,5E0,4E0,7E0,1.6E1,9.1E1,8.5E1,4E0,6E0,1E1,7E0,6E0,2.8E1,3.1E1,9E0,2.2E1,5E0,1.08E2,1.23E2,1.7E1,8E0,4.3E1,2.3E1,2.2E1,6E0,5E0,3.6E1,3.8E1,1E1,3.3E1,5E0,1.1E1,5E0,7E1,2.1E1,8.1E1,4E0,2.1E1,7E0,1.9E1,1.2E1,5E0,4E0,8.9E1,1.9E1,2.2E1,1.01E2,1E1,7E0,4E0,4E0,1.7E1,2.6E1,1.9E1,4E0,1.8E1,4E0,2.8E1,8E0,3.1E1,7E0,9E0,2.4E1,6.2E1,8E0,9E0,1.2E1,5.2E1,2.9E1,8E0,1.3E1,5E0,1.4E1,6E0,6E0,6E0,8.3E1,1.4E1,5E0,9E0,1.3E1,9.6E1,5E0,5E0,5E0,1.3E1,4E0,2.2E1,4E0,5E0,1.4E1,1E1,1.8E1,1.7E1,1.4E1,1.3E1,1.1E1,5E1,1.2E1,4E0,4E0,5E0,4E0,8E0,4E0,2.7E1,2.5E1,1.9E1,1E1,4E0,4E0,7E0,6E0,7E0,7E0,1.7E1,6.6E1,1E1,4E0,4E0,5E0,7E0,6E0,4.9E1,4.7E1,4E0,9E0,1E1,4E0,1E1,4E0,2.8E1,2.2E1,4E0,8E0,4E0,4E0,2.3E1,4E0,8E0,1.7E1,1.3E1,6E0,5E0,5E0,8E0,9E0,5.7E1,9E0,4E0,6E0,1.9E1,3E1,1E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[3.2505463E-3,-1.1654815E-1,3.6172676E-1,-3.590055E-1,-1.0095838E-2,-4.5593828E-1,4.0589243E-1,-5.265351E-1,-2.8014255E-1,-1.14962645E-1,9.217608E-2,-1.11706415E-2,-2.3904162E-2,1.0189188E0,3.708496E-1,-7.593408E-1,-4.1117904E-1,-4.1346416E-1,-2.1395218E-1,-1.6278714E-1,4.9649157E-2,1.6716285E-1,2.4673175E-2,3.0435188E-2,5.6986243E-2,2.9117757E-1,6.2600684E-1,-1.5495122E-2,-8.820214E-1,-2.0719653E-2,1.6941458E-3,-6.6070645E-3,-4.3224E-1,1.6305804E-2,-2.5471857E-1,-2.3922963E-1,-5.7356335E-2,7.908246E-2,-6.7375004E-2,9.513606E-2,2.5741884E-1,9.834508E-3,6.5021114E-3,9.256604E-2,3.666409E-1,7.5896996E-1,4.5508462E-1,-5.306592E-2,-3.192911E-2,-4.5905703E-1,-9.328848E-3,-3.917255E-1,-1.816627E-1,-2.0540728E-1,-4.8547304E-1,-2.5786358E-1,-2.0155588E-2,5.466478E-2,9.260237E-3,1.2802752E-3,-1.0397415E-2,1.234197E-1,-9.112795E-3,2.9675186E-1,-1.1783756E-3,-1.4289995E-1,3.198535E-2,1.701426E-1,-5.1127076E-2,4.4959348E-1,2.244506E-1,4.4486746E-2,2.9615002E-2,5.400847E-1,2.5657594E-1,-5.59759E-1,-3.9877042E-1,-2.676839E-1,-2.6219243E-2,-2.08072E-1,4.1356627E-3,-3.022815E-1,-1.5142633E-1,-1.0892647E-2,-3.599395E-2,-2.4272904E-3,-1.691322E-2,1.18870055E-2,-3.8883276E-2,-3.799378E-3,7.484267E-2,-6.754106E-2,1.4957097E-1,2.1982755E-1,1.9160535E-2,-1.9103704E-1,1.8349694E-3,4.8519544E-2,-9.21015E-3,1.568868E-2,1.13634735E-1,-1.312766E-1,3.487478E-3,4.7322676E-1,-6.343543E-4,3.0808827E-1,-1.85981E-2,2.7130585E-2,1.3859407E-2,2.9573445E-3,1.7291486E-2,-3.0396149E-2,-1.474656E-2,-1.4725634E-2,-2.3049546E-2,-3.7483564E-1,-1.1671515E-1,-2.5105888E-1,-9.0784915E-2,-1.601454E-1,-3.6172333E-1,-1.152518E-1,-1.4908189E-2,8.4625065E-2,-8.084283E-2,-4.8527718E-3,1.16478406E-1,-6.4006546E-3,7.840552E-4,1.5481961E-2,1.2702042E-1,2.6843554E-1,6.038011E-2,-2.4007928E-1,-1.6228916E-3,-3.0662296E-3,1.591481E-1,1.8448913E-1,6.110338E-2,-9.1805327E-4,-9.60999E-3,4.2353433E-1,6.0785896E-1,2.3549022E-1,3.0307941E-2,5.5388524E-3,-8.0485465E-3,-9.137015E-3,-1.9937396E-2,-1.2191463E-2,2.348425E-4,-4.0918523E-3,-2.6411435E-1,-3.68976E-2,-1.082397E-2,-9.446531E-3,-2.2325153E-3,-4.3337488E-1,-2.4201532E-1,1.13993045E-2,-1.373914E-1,-2.925816E-4,1.09302774E-1,2.5645468E-2,-1.1783019E-1,3.9501796E-3,-2.2889606E-3,7.961968E-3,2.7847271E-3,7.358406E-2,1.816897E-1,1.9170721E-1,1.7719213E-2,5.412028E-3,-3.9092152E-4,-4.9004625E-3,-1.3574363E-2,3.6472507E-2,-1.4243434E-1,8.738414E-2,1.523735E-2,1.0475402E-2,4.085527E-3,-8.1262493E-4,1.0562591E-1,2.566952E-2,3.455461E-1,3.1130448E-2,1.3322173E-2,1.3223937E-3,2.8040037E-1,-1.3813071E-2,-7.877615E-3,-4.636489E-3,1.4402999E-3,-2.222312E-2,-1.117366E-2,-3.419595E-3,-1.4634069E-2,-3.15442E-3,3.506412E-3,-7.409692E-3,-1.1445563E-3,2.3210004E-3,7.836609E-3,-4.517476E-3,3.6690293E-3,1.2596359E-3,-6.8607777E-3,-6.2639493E-4,6.9613336E-3,1.0762418E-2,5.0152917E-3,4.3124678E-3,1.0390745E-2,-2.8561084E-3,2.8797733E-3,-3.5921042E-4,-1.0899195E-2,1.3185804E-3,1.1227438E-2,7.2503136E-3,5.8515766E-4,1.7867856E-2,4.1635795E-3,1.42089855E-2,8.1980685E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,-1,75,-1,77,79,81,83,85,87,89,-1,-1,-1,91,-1,93,-1,95,97,99,101,103,105,-1,-1,107,109,111,113,115,-1,117,-1,119,121,-1,-1,-1,-1,-1,123,-1,125,127,129,131,-1,133,-1,135,-1,-1,137,139,-1,141,-1,143,145,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,153,155,157,159,-1,161,163,165,167,-1,-1,-1,169,171,173,175,-1,177,179,181,183,-1,-1,185,187,189,-1,-1,-1,-1,-1,-1,-1,-1,191,193,-1,-1,-1,195,197,199,201,-1,203,205,207,-1,-1,-1,-1,209,211,213,-1,-1,-1,-1,-1,215,217,219,-1,-1,-1,-1,221,-1,223,-1,-1,-1,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5263226E1,2.038639E1,9.7188225E0,3.03656E0,5.909308E0,5.9205532E-2,5.0141563E0,1.728796E0,1.3764238E0,2.1493962E0,1.4081249E0,0E0,0E0,1.9198227E-1,4.6763306E0,9.5239735E-1,9.0197945E-1,2.123928E-1,2.7013812E0,1.6868105E0,2.188814E-1,8.336222E-1,5.0474364E-1,0E0,0E0,2.7238321E0,9.272804E-1,0E0,1.223011E-1,0E0,0E0,0E0,2.1611881E-1,0E0,9.8981E-1,9.1636133E-1,6.661143E-1,1.3720903E-1,2.011526E-1,6.445767E-1,6.5301085E-1,0E0,5.2235645E-1,5.883543E-1,1.4678326E0,1.04400635E-1,3.1700563E-1,0E0,0E0,4.390812E-2,0E0,6.3312626E-1,5.27107E-1,5.2422667E-1,8.427727E-1,2.9187888E-1,4.101042E-1,1.24939755E-1,0E0,0E0,0E0,3.54679E-1,0E0,3.717084E-1,0E0,1.8340248E-1,4.5720655E-1,2.869624E-1,2.0272456E-1,9.153366E-1,1.0424592E0,0E0,0E0,2.767706E-2,2.0269978E-1,1.20646E-1,8.942413E-2,3.283118E-1,0E0,3.0492496E-1,0E0,2.6312733E-1,4.088887E-1,0E0,0E0,0E0,0E0,0E0,3.8515317E-1,0E0,1.2665604E-1,5.6357764E-2,2.1266389E-1,2.3955548E-1,0E0,1.139043E-1,0E0,6.282425E-1,0E0,0E0,8.922315E-2,9.845179E-2,0E0,2.710762E-1,0E0,8.053863E-1,3.226034E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5203686E-2,1.985236E-1,7.631159E-2,1.3828415E-1,5.455941E-2,1.1812568E-1,1.7261052E-1,0E0,4.3340355E-2,2.1965146E-1,6.077222E-2,6.810048E-2,0E0,0E0,0E0,1.5022486E-1,1.4726806E-1,3.6161963E-2,5.5258095E-2,0E0,4.2398834E-1,3.9986098E-1,2.4596423E-2,6.141933E-2,0E0,0E0,4.3641567E-1,2.061801E-1,2.9855704E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.9386225E-2,6.79294E-2,0E0,0E0,0E0,2.0558596E-2,1.2723851E-1,5.6573566E-2,1.306861E-1,0E0,4.4922665E-2,1.0732127E-1,1.865294E-1,0E0,0E0,0E0,0E0,2.0359685E-1,7.5941324E-2,3.4529686E-2,0E0,0E0,0E0,0E0,0E0,1.5637608E-1,2.121692E-1,2.3117812E-1,0E0,0E0,0E0,0E0,5.0480276E-2,0E0,3.967309E-1,0E0,0E0,0E0,2.0917177E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,61,61,63,63,65,65,66,66,67,67,68,68,69,69,70,70,73,73,74,74,75,75,76,76,77,77,79,79,81,81,82,82,88,88,90,90,91,91,92,92,93,93,95,95,97,97,100,100,101,101,103,103,105,105,106,106,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,130,130,131,131,132,132,133,133,135,135,136,136,137,137,138,138,141,141,142,142,143,143,152,152,153,153,157,157,158,158,159,159,160,160,162,162,163,163,164,164,169,169,170,170,171,171,177,177,178,178,179,179,184,184,186,186,190,190],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,-1,76,-1,78,80,82,84,86,88,90,-1,-1,-1,92,-1,94,-1,96,98,100,102,104,106,-1,-1,108,110,112,114,116,-1,118,-1,120,122,-1,-1,-1,-1,-1,124,-1,126,128,130,132,-1,134,-1,136,-1,-1,138,140,-1,142,-1,144,146,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,154,156,158,160,-1,162,164,166,168,-1,-1,-1,170,172,174,176,-1,178,180,182,184,-1,-1,186,188,190,-1,-1,-1,-1,-1,-1,-1,-1,192,194,-1,-1,-1,196,198,200,202,-1,204,206,208,-1,-1,-1,-1,210,212,214,-1,-1,-1,-1,-1,216,218,220,-1,-1,-1,-1,222,-1,224,-1,-1,-1,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8988005E5,1E0,3.434405E7,9.24E2,3.714144E8,2.4481E4,2E0,2.71E2,2.331083E6,3.7105262E0,-1.11706415E-2,-2.3904162E-2,6.23694E5,4.391553E6,1.5E1,3.7066666E2,2.416E3,1.2E1,2.61E2,1.8E1,4.265829E2,4.0869565E0,3.0435188E-2,5.6986243E-2,7.601393E2,1.6386554E0,-1.5495122E-2,3.6255838E2,-2.0719653E-2,1.6941458E-3,-6.6070645E-3,3.328629E6,1.6305804E-2,2.4360857E4,2.1E1,1E0,6.9307615E6,1.6923077E0,4.5158855E6,7.978062E6,9.834508E-3,2.1924414E2,1.124641E0,6.9E1,2.5959E4,8.81059E5,-5.306592E-2,-3.192911E-2,3.89E2,-9.328848E-3,5.17259E8,8.909296E4,1.5933333E2,2.85E2,3.1275E4,2.3112903E2,1.3020051E-5,9.260237E-3,1.2802752E-3,-1.0397415E-2,4.0449125E5,-9.112795E-3,1.0029973E8,-1.1783756E-3,9.48E0,2.5243637E-6,6E0,3.149243E8,3.2E1,5.712652E8,4.4486746E-2,2.9615002E-2,1.647E4,1.3E1,2.7546012E0,9.183673E-1,3.183206E-8,-2.6219243E-2,7.997723E6,4.1356627E-3,3.1E1,2E0,-1.0892647E-2,-3.599395E-2,-2.4272904E-3,-1.691322E-2,1.18870055E-2,1.3364486E0,-3.799378E-3,9.22E2,1.4327235E7,1.3600995E2,5.4983668E7,1.9160535E-2,2.008E3,1.8349694E-3,2.008E3,-9.21015E-3,1.568868E-2,1E1,1.2E1,3.487478E-3,1.711E4,-6.343543E-4,4.907764E3,6.619098E8,2.7130585E-2,1.3859407E-2,2.9573445E-3,1.7291486E-2,-3.0396149E-2,-1.474656E-2,-1.4725634E-2,-2.3049546E-2,2.56E2,4.9E1,7.090909E0,5.4444447E0,1E0,1.329762E2,6.055E3,-1.4908189E-2,2.2007043E0,3.373913E0,3.863E3,9.166947E6,-6.4006546E-3,7.840552E-4,1.5481961E-2,2.378914E6,1.763E3,1.1090909E1,2.083138E6,-1.6228916E-3,1.3254E4,6.875854E2,3.744E3,1.5E1,-9.1805327E-4,-9.60999E-3,1.798E3,9.2732E4,1.7068776E10,3.0307941E-2,5.5388524E-3,-8.0485465E-3,-9.137015E-3,-1.9937396E-2,-1.2191463E-2,2.348425E-4,-4.0918523E-3,8.01E2,2.132956E8,-1.082397E-2,-9.446531E-3,-2.2325153E-3,4.7563504E7,1.5277778E0,4.0906172E5,1.03860024E8,-2.925816E-4,1.3717398E4,6.07E2,2.57E2,3.9501796E-3,-2.2889606E-3,7.961968E-3,2.7847271E-3,2.2461708E6,9E0,1.2151898E0,1.7719213E-2,5.412028E-3,-3.9092152E-4,-4.9004625E-3,-1.3574363E-2,1.14E3,1.9E1,1.836095E6,1.523735E-2,1.0475402E-2,4.085527E-3,-8.1262493E-4,7.362415E1,2.566952E-2,2.7578741E1,3.1130448E-2,1.3322173E-2,1.3223937E-3,2.5345264E7,-1.3813071E-2,-7.877615E-3,-4.636489E-3,1.4402999E-3,-2.222312E-2,-1.117366E-2,-3.419595E-3,-1.4634069E-2,-3.15442E-3,3.506412E-3,-7.409692E-3,-1.1445563E-3,2.3210004E-3,7.836609E-3,-4.517476E-3,3.6690293E-3,1.2596359E-3,-6.8607777E-3,-6.2639493E-4,6.9613336E-3,1.0762418E-2,5.0152917E-3,4.3124678E-3,1.0390745E-2,-2.8561084E-3,2.8797733E-3,-3.5921042E-4,-1.0899195E-2,1.3185804E-3,1.1227438E-2,7.2503136E-3,5.8515766E-4,1.7867856E-2,4.1635795E-3,1.42089855E-2,8.1980685E-3],"split_indices":[2,43,17,7,2,7,12,32,2,43,69,0,0,1,43,3,4,9,33,70,3,67,73,0,0,67,69,0,4,0,0,0,9,0,43,3,13,43,68,62,62,0,67,53,8,9,9,0,0,1,0,46,48,4,2,1,67,53,0,0,0,43,0,5,0,71,52,8,7,3,47,0,0,44,8,68,71,52,0,9,0,10,6,0,0,0,0,0,68,0,44,12,67,5,0,2,0,2,0,0,3,3,0,44,0,4,7,0,0,0,0,0,0,0,0,10,0,73,69,8,4,12,0,69,69,12,60,0,0,0,9,2,73,12,0,44,67,10,3,0,0,44,44,46,0,0,0,0,0,0,0,0,2,46,0,0,0,7,68,43,7,0,48,2,44,0,0,0,0,43,8,68,0,0,0,0,0,2,8,43,0,0,0,0,73,0,71,0,0,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.052E3,7.89E2,2.63E2,2.4E2,5.49E2,1.3E1,2.5E2,7.5E1,1.65E2,2.71E2,2.78E2,4E0,9E0,1.2E1,2.38E2,2.3E1,5.2E1,5.3E1,1.12E2,2.1E2,6.1E1,1.31E2,1.47E2,6E0,6E0,1.83E2,5.5E1,6E0,1.7E1,4.8E1,4E0,4E0,4.9E1,7E0,1.05E2,1.21E2,8.9E1,4.9E1,1.2E1,7.4E1,5.7E1,1.2E1,1.35E2,5.1E1,1.32E2,2.9E1,2.6E1,5E0,1.2E1,4.3E1,6E0,3.5E1,7E1,1.08E2,1.3E1,1.3E1,7.6E1,4.2E1,7E0,8E0,4E0,6.8E1,6E0,5E1,7E0,1.9E1,1.16E2,3.3E1,1.8E1,8.2E1,5E1,8E0,2.1E1,1.7E1,9E0,1.3E1,3E1,2.2E1,1.3E1,6.4E1,6E0,3.7E1,7.1E1,8E0,5E0,5E0,8E0,4E0,7.2E1,5E0,3.7E1,8E0,6E1,3.2E1,1.8E1,1.5E1,4E0,1.09E2,7E0,7E0,2.6E1,1.1E1,7E0,7.8E1,4E0,3.7E1,1.3E1,1.3E1,4E0,4E0,5E0,8E0,5E0,1.9E1,1.1E1,1.2E1,1E1,4.6E1,1.8E1,1.2E1,2.5E1,6E1,1.1E1,1.8E1,5.4E1,1.3E1,2.4E1,4E0,4E0,5E0,5.5E1,2.4E1,8E0,1.1E1,4E0,7.5E1,3.4E1,1E1,1.6E1,5E0,6E0,6E1,1.8E1,3.2E1,5E0,7E0,6E0,4E0,8E0,4E0,6E0,4E0,4.2E1,1.4E1,4E0,8E0,4E0,1.4E1,1.1E1,9E0,5.1E1,4E0,1.4E1,1.4E1,4E1,4E0,9E0,1.1E1,1.3E1,2.9E1,2.6E1,1.6E1,8E0,4E0,4E0,4E0,7E0,5.9E1,1.6E1,2.5E1,9E0,6E0,4E0,6E0,1E1,2E1,4E1,1.4E1,4E0,6E0,2.6E1,2.9E1,1.3E1,7E0,7E0,1E1,4E0,4E0,7E0,4E0,5E0,4.2E1,9E0,8E0,6E0,4E0,1E1,7E0,3.3E1,1.4E1,1.5E1,1.4E1,1.2E1,5E0,1.1E1,1.2E1,4.7E1,7E0,9E0,1.9E1,6E0,6E0,4E0,3.4E1,6E0,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[9.038484E-3,-1.2883458E-1,2.5897896E-1,-3.4229952E-1,-2.2767497E-2,-4.104476E-1,2.9649198E-1,-2.956752E-1,-5.1426053E-1,-9.994397E-2,8.2556054E-2,-2.1128622E-1,-2.4282899E-2,1.7741084E-1,4.522518E-1,-4.2905805E-1,-1.8625176E-1,-1.9732893E-1,-5.968157E-1,-8.501878E-2,-3.3037852E-2,5.9052687E-2,3.5282513E-1,-2.869962E-3,-1.4700239E-2,3.4714615E-1,1.0899194E-1,7.490896E-1,3.6288962E-1,-4.9013755E-1,-3.4248704E-1,1.7432038E-2,-2.2666878E-1,-4.1610957E-3,-1.1353574E-2,-6.827386E-1,-1.2158992E-2,-1.2904565E-1,5.340388E-2,1.0425414E-1,-3.7144285E-2,6.8349433E-3,1.9120734E-2,2.4442317E-1,4.448084E-1,1.2789638E-1,-1.6036898E-2,4.6931082E-1,9.2146915E-1,5.40026E-1,2.4629833E-1,-1.579187E-2,-5.350738E-1,-5.937914E-3,-3.6521524E-1,-1.9714724E-1,-2.5926823E-2,-2.3892788E-2,-8.3399636E-1,-2.2614989E-1,-8.668468E-2,7.3612556E-2,-8.8562425E-3,4.4637952E-2,1.4767136E-1,-1.4357184E-1,8.642603E-2,1.3238703E-1,1.7590212E-2,4.8416483E-1,7.712279E-3,6.384352E-2,2.3062661E-1,3.0198673E-2,7.6339548E-3,2.4037002E-2,4.5702185E-2,6.253453E-1,3.0034307E-1,2.928491E-1,2.9202359E-2,-2.5938027E-2,-1.2954414E-2,-1.4005636E-2,-2.1072648E-2,-1.6577858E-1,-1.676818E-2,-4.501945E-2,-2.3455385E-2,-1.8234089E-1,-3.7888932E-1,-2.1259548E-2,-1.5575287E-1,-4.1580193E-2,1.1476305E-1,-6.599024E-3,6.762143E-2,1.6652918E-1,-5.867957E-3,-9.1943525E-2,-1.8186273E-2,1.4028837E-1,-5.79676E-3,1.6521081E-1,6.4692704E-4,1.2622806E-2,2.426811E-2,2.0737039E-1,2.8578728E-2,2.7084735E-1,-9.686181E-3,1.3115358E-2,3.0613964E-2,5.8611194E-3,1.9153757E-2,2.5180168E-2,2.3151086E-1,1.2744788E-2,-9.7050145E-2,-1.9552541E-1,2.5068957E-2,-2.1722208E-1,-4.8436772E-2,-8.439384E-3,-2.5162708E-2,-4.962803E-2,5.6526614E-3,1.7392533E-3,-1.7572029E-1,-9.547308E-2,1.9103154E-3,8.470641E-2,1.0887848E-2,-3.8268674E-2,9.727466E-2,1.2087663E-2,1.400573E-1,-1.6351989E-2,-1.6096766E-1,3.9882407E-2,1.1863329E-2,3.417695E-3,1.0366536E-2,2.832335E-1,-1.4482354E-3,1.0993777E-2,1.0036747E-2,3.687134E-1,1.5354046E-1,1.7514554E-1,2.0302527E-2,-9.418493E-3,3.7603674E-4,-2.6285863E-1,-1.3030097E-1,-9.0386387E-4,3.813366E-3,-2.3222806E-1,-4.6654847E-3,-4.7475668E-3,6.450564E-4,-9.573933E-2,4.0566247E-2,-2.4257724E-1,-1.21034555E-1,-1.2344385E-3,-6.3225767E-3,7.3939515E-3,5.7465035E-2,1.8067614E-3,-4.0687923E-3,4.295742E-2,1.4647219E-1,1.6614866E-1,-4.251073E-3,-4.428699E-3,2.5530807E-3,-4.714015E-3,-1.0280934E-2,8.606419E-2,-2.3663742E-3,1.6952554E-2,8.5489005E-3,-5.757856E-2,3.9939586E-2,2.2588288E-2,2.307433E-1,2.1240956E-1,4.2739182E-4,2.3145424E-1,-8.865486E-3,-5.257754E-3,-1.3586165E-2,-7.855505E-3,-6.4492127E-4,-7.4248053E-3,-1.2885934E-2,-7.843881E-4,-6.574897E-3,-3.31523E-3,5.322587E-3,-1.2294298E-2,-4.1806474E-3,-1.6670658E-3,-7.3210024E-3,4.6119912E-4,4.2712283E-3,-2.9135854E-3,3.9420547E-3,5.457052E-4,8.389474E-3,1.0541899E-2,4.8453845E-3,6.2551233E-3,9.018273E-4,-8.8055115E-3,-2.4561705E-5,2.3880196E-4,7.7648554E-3,5.467038E-3,1.5072199E-2,1.2010223E-2,2.1380277E-3,6.979607E-3,1.672586E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,57,-1,59,61,63,65,-1,-1,67,69,71,-1,73,75,77,79,-1,81,-1,83,85,-1,-1,87,89,91,93,-1,95,97,99,101,103,-1,105,-1,107,109,-1,-1,-1,-1,111,113,115,117,-1,-1,-1,-1,119,-1,-1,-1,121,123,125,127,129,131,-1,133,135,-1,137,-1,139,-1,141,-1,-1,-1,143,145,147,-1,-1,-1,-1,-1,-1,149,-1,151,153,155,157,159,-1,-1,161,-1,-1,163,165,-1,167,-1,169,171,-1,173,175,177,179,-1,-1,-1,181,-1,183,-1,185,187,189,-1,-1,-1,191,193,-1,-1,195,-1,-1,-1,197,199,201,203,-1,-1,-1,205,-1,-1,207,209,211,-1,-1,-1,-1,-1,213,-1,-1,-1,215,217,-1,219,221,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5768444E1,1.513944E1,9.3881E0,1.6278458E0,3.6498191E0,3.4768152E-1,6.411585E0,2.503439E0,1.0857916E0,2.3172398E0,1.1860889E0,1.2375435E-1,0E0,2.2847843E0,3.7336025E0,2.4986362E-1,2.3405576E0,3.49994E-2,8.522997E-1,1.5548478E0,0E0,7.6916337E-1,1.2639368E-1,0E0,0E0,4.631114E-1,1.2860588E0,1.2073002E0,2.303996E0,1.1034775E-1,1.2812328E-1,0E0,8.0781317E-1,0E0,0E0,2.698984E-1,0E0,7.742882E-1,3.2120812E-1,3.0401254E-1,7.6203173E-1,0E0,0E0,4.0779436E-1,2.2538614E-1,8.999188E-1,0E0,7.328346E-1,2.910614E-2,7.52862E-1,7.291846E-1,0E0,5.7444572E-2,0E0,4.22554E-2,3.9233136E-1,0E0,0E0,1.0111141E-1,3.1901193E-1,6.112449E-1,2.797057E-1,0E0,2.3372027E-1,3.7857032E-1,3.6405146E-1,3.257645E-1,6.7631155E-2,0E0,6.469536E-2,0E0,4.344194E-1,9.9534893E-1,0E0,0E0,0E0,0E0,2.322731E-1,2.2487509E-1,8.019414E-1,4.6276113E-1,0E0,0E0,0E0,0E0,4.3068504E-1,0E0,0E0,0E0,2.0900309E-1,2.9082143E-1,2.9533422E-1,2.6148736E-1,7.545705E-2,1.3630474E-1,0E0,1.5130182E-1,1.2701488E-1,0E0,1.3824415E-1,0E0,2.470968E-1,0E0,5.898586E-2,0E0,0E0,0E0,3.1674695E-1,2.3317374E-1,5.0592566E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.090804E-1,0E0,1.1402033E-1,2.512145E-1,2.9832404E-2,4.1084528E-2,3.872416E-2,0E0,0E0,2.5342226E-1,0E0,0E0,1.9000721E-1,2.4794668E-2,0E0,6.468722E-2,0E0,4.5833245E-2,9.258211E-2,0E0,3.3620977E-1,8.501391E-2,2.808845E-2,5.910361E-2,0E0,0E0,0E0,3.8953424E-2,0E0,1.3330185E-1,0E0,3.1843972E-1,2.0189536E-1,8.917923E-1,0E0,0E0,0E0,1.10788345E-1,1.5942556E-1,0E0,0E0,6.203127E-2,0E0,0E0,0E0,1.4432275E-1,1.8434021E-1,6.899512E-2,1.05576575E-1,0E0,0E0,0E0,4.5593224E-2,0E0,0E0,9.07336E-2,8.7715864E-2,1.585064E-1,0E0,0E0,0E0,0E0,0E0,2.6900962E-2,0E0,0E0,0E0,1.5166724E-1,2.0853072E-1,0E0,1.1168337E-1,1.2228173E-1,0E0,3.2316256E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,32,32,35,35,37,37,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,54,54,55,55,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,71,71,72,72,77,77,78,78,79,79,80,80,85,85,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,99,99,101,101,103,103,107,107,108,108,109,109,116,116,118,118,119,119,120,120,121,121,122,122,125,125,128,128,129,129,131,131,133,133,134,134,136,136,137,137,138,138,139,139,143,143,145,145,147,147,148,148,149,149,153,153,154,154,157,157,161,161,162,162,163,163,164,164,168,168,171,171,172,172,173,173,179,179,183,183,184,184,186,186,187,187,189,189],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,58,-1,60,62,64,66,-1,-1,68,70,72,-1,74,76,78,80,-1,82,-1,84,86,-1,-1,88,90,92,94,-1,96,98,100,102,104,-1,106,-1,108,110,-1,-1,-1,-1,112,114,116,118,-1,-1,-1,-1,120,-1,-1,-1,122,124,126,128,130,132,-1,134,136,-1,138,-1,140,-1,142,-1,-1,-1,144,146,148,-1,-1,-1,-1,-1,-1,150,-1,152,154,156,158,160,-1,-1,162,-1,-1,164,166,-1,168,-1,170,172,-1,174,176,178,180,-1,-1,-1,182,-1,184,-1,186,188,190,-1,-1,-1,192,194,-1,-1,196,-1,-1,-1,198,200,202,204,-1,-1,-1,206,-1,-1,208,210,212,-1,-1,-1,-1,-1,214,-1,-1,-1,216,218,-1,220,222,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,1E0,2.1E1,9.31E2,1.91E2,1.1459359E3,2.71E2,1E0,1.1290322E1,5.197839E6,1E0,-2.4282899E-2,8E0,3.4583332E1,3.89E2,2E0,4.0869565E0,6.03E2,2.331083E6,-3.3037852E-2,5.194E3,5.45901E6,-2.869962E-3,-1.4700239E-2,5.691698E2,1.0428572E1,1.562752E3,2.2842104E0,2.416E3,4.456432E6,1.7432038E-2,3.2856784E7,-4.1610957E-3,-1.1353574E-2,4.6463413E0,-1.2158992E-2,1.8149019E2,2.7E1,2.378914E6,7.9016544E5,6.8349433E-3,1.9120734E-2,1E0,1.4553613E1,6.9664386E2,-1.6036898E-2,1.934E3,9E0,2.395631E7,2.3876712E7,-1.579187E-2,7.338571E2,-5.937914E-3,1.5272727E0,3.78E2,-2.5926823E-2,-2.3892788E-2,1.746E3,2.4E0,1.7755102E0,3.0479442E9,-8.8562425E-3,5.4603375E5,1E0,1.28636E7,1.2906634E7,3.222E3,1.7590212E-2,1.000501E6,7.712279E-3,2.992648E-2,1.6063418E7,3.0198673E-2,7.6339548E-3,2.4037002E-2,4.5702185E-2,1.4992306E8,5.854E3,3.3382E4,1.2176E4,-2.5938027E-2,-1.2954414E-2,-1.4005636E-2,-2.1072648E-2,1.3022917E8,-1.676818E-2,-4.501945E-2,-2.3455385E-2,5.41E2,4.9963706E5,3.961275E2,1.6580646E2,2.8080197E2,6.7723384E7,-6.599024E-3,1E0,1.880118E-3,-5.867957E-3,3.78E2,-1.8186273E-2,5.4007E7,-5.79676E-3,4.23025E2,6.4692704E-4,1.2622806E-2,2.426811E-2,1.302273E6,6.0450394E8,3.8396227E0,-9.686181E-3,1.3115358E-2,3.0613964E-2,5.8611194E-3,1.9153757E-2,2.5180168E-2,4.435876E2,1.2744788E-2,2.9925186E-2,8.147158E4,7.8918E4,1.3337367E7,8.8790035E-1,-8.439384E-3,-2.5162708E-2,4.39776E6,5.6526614E-3,1.7392533E-3,2.3238889E2,5.2846E4,1.9103154E-3,8.5E1,1.0887848E-2,1.821E3,6.077143E5,1.2087663E-2,1E0,1.445E3,1.1860938E8,1.3062992E1,1.1863329E-2,3.417695E-3,1.0366536E-2,1E0,-1.4482354E-3,3.0805944E7,1.0036747E-2,1.2629019E8,4.771E3,1.8545505E2,2.0302527E-2,-9.418493E-3,3.7603674E-4,1E0,1.605076E6,-9.0386387E-4,3.813366E-3,1.6667E4,-4.6654847E-3,-4.7475668E-3,6.450564E-4,1.4473684E-1,8.856631E0,1.339646E6,1E0,-1.2344385E-3,-6.3225767E-3,7.3939515E-3,1.5539158E3,1.8067614E-3,-4.0687923E-3,2.9432205E6,2.906237E2,8E0,-4.251073E-3,-4.428699E-3,2.5530807E-3,-4.714015E-3,-1.0280934E-2,3.2283451E4,-2.3663742E-3,1.6952554E-2,8.5489005E-3,1.5204346E8,1.3261502E0,2.2588288E-2,1.1342433E3,7.0296685E6,4.2739182E-4,2.5134058E3,-8.865486E-3,-5.257754E-3,-1.3586165E-2,-7.855505E-3,-6.4492127E-4,-7.4248053E-3,-1.2885934E-2,-7.843881E-4,-6.574897E-3,-3.31523E-3,5.322587E-3,-1.2294298E-2,-4.1806474E-3,-1.6670658E-3,-7.3210024E-3,4.6119912E-4,4.2712283E-3,-2.9135854E-3,3.9420547E-3,5.457052E-4,8.389474E-3,1.0541899E-2,4.8453845E-3,6.2551233E-3,9.018273E-4,-8.8055115E-3,-2.4561705E-5,2.3880196E-4,7.7648554E-3,5.467038E-3,1.5072199E-2,1.2010223E-2,2.1380277E-3,6.979607E-3,1.672586E-2],"split_indices":[2,43,17,3,2,0,67,2,20,69,43,15,0,3,73,1,10,73,10,43,0,44,60,0,0,67,68,67,69,9,7,0,60,0,0,68,0,70,3,9,43,0,0,23,71,67,0,44,3,62,60,0,48,0,71,0,0,0,44,71,68,46,0,60,79,60,12,2,0,43,0,53,62,0,0,0,0,5,2,9,9,0,0,0,0,7,0,0,0,2,43,70,67,4,7,0,29,53,0,0,0,7,0,4,0,0,0,9,7,69,0,0,0,0,0,0,71,0,72,43,1,5,71,0,0,9,0,0,67,9,0,10,0,44,62,0,19,2,5,71,0,0,0,23,0,5,0,5,2,73,0,0,0,15,12,0,0,9,0,0,0,71,73,9,26,0,0,0,48,0,0,60,4,8,0,0,0,0,0,48,0,0,0,7,57,0,4,62,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.036E3,6.68E2,3.68E2,2.21E2,4.47E2,1.9E1,3.49E2,1.76E2,4.5E1,2.58E2,1.89E2,8E0,1.1E1,1.99E2,1.5E2,7.8E1,9.8E1,1E1,3.5E1,2.53E2,5E0,1.75E2,1.4E1,4E0,4E0,5.6E1,1.43E2,3.3E1,1.17E2,4.3E1,3.5E1,6E0,9.2E1,4E0,6E0,2.7E1,8E0,1.92E2,6.1E1,1.19E2,5.6E1,4E0,1E1,2.9E1,2.7E1,1.38E2,5E0,1.4E1,1.9E1,4.5E1,7.2E1,1.2E1,3.1E1,4E0,3.1E1,8.6E1,6E0,1.5E1,1.2E1,5.7E1,1.35E2,5.7E1,4E0,5.1E1,6.8E1,3E1,2.6E1,1.7E1,1.2E1,2.3E1,4E0,8.6E1,5.2E1,8E0,6E0,4E0,1.5E1,3.2E1,1.3E1,5.9E1,1.3E1,2.7E1,4E0,2.1E1,1E1,7.4E1,1.2E1,7E0,5E0,4.6E1,1.1E1,7E1,6.5E1,1.5E1,4.2E1,5E0,4.6E1,6.4E1,4E0,2.6E1,4E0,2.1E1,5E0,1.3E1,4E0,5E0,1.8E1,1.6E1,7E1,4.8E1,4E0,4E0,2.8E1,6E0,7E0,1E1,4.9E1,4E0,9E0,6.4E1,1E1,3.6E1,1E1,6E0,5E0,5.9E1,1.1E1,6E0,5.9E1,9E0,6E0,3.5E1,7E0,1E1,3.6E1,1.2E1,5.2E1,1.3E1,1.3E1,1.2E1,9E0,6E0,7E0,1.2E1,4E0,6.5E1,5E0,2.5E1,2.3E1,4E1,9E0,4E0,5E0,3E1,3.4E1,6E0,4E0,3.1E1,5E0,5E0,5E0,3.9E1,2E1,2.5E1,3.4E1,4E0,5E0,8E0,2.7E1,4E0,6E0,1.8E1,1.8E1,4.7E1,5E0,6E0,7E0,8E0,5E0,8E0,4E0,5E0,7E0,1.9E1,4.6E1,1.2E1,1.3E1,1.6E1,7E0,3.5E1,5E0,6E0,2.4E1,2.5E1,9E0,1.4E1,1.7E1,1.5E1,2.4E1,8E0,1.2E1,2.1E1,4E0,1.1E1,2.3E1,1.2E1,1.5E1,5E0,1.3E1,4E0,1.4E1,2.2E1,2.5E1,4E0,4E0,5E0,1.4E1,3.7E1,9E0,7E0,6E0,1.2E1,4E0,2.3E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[3.913325E-3,-9.931895E-2,3.1083712E-1,-3.1763157E-1,-2.1071972E-3,-4.4784325E-1,3.5547167E-1,-4.5457786E-1,-2.5262824E-1,-9.130332E-2,9.1489136E-2,-2.5141533E-2,-9.822276E-3,4.3573078E-2,3.1894782E-1,-4.1667438E-1,-4.2016767E-2,-3.936701E-1,-1.9808681E-1,-6.4099506E-2,-3.05111E-1,1.7501368E-1,3.434601E-2,1.5826014E-1,4.5669937E-1,-3.713674E-1,-2.9297657E-2,-2.4053128E-2,-3.258181E-1,3.4862563E-1,-2.451318E-1,-2.639635E-1,-3.4193274E-2,-2.065337E-1,-2.5996389E-2,2.1157478E-1,2.4137817E-2,1.2612889E-2,1.9797083E-2,1.2478939E-1,3.3153426E-2,3.062011E-1,5.5495554E-1,-2.0340558E-2,-2.4390937E-1,-1.8274166E-1,-3.7385923E-1,8.407582E-3,2.2123247E-2,-3.3229895E-2,-2.209109E-1,-1.82136E-1,-3.5616052E-1,-1.097188E-1,-1.6712171E-3,-2.4444067E-1,-2.4057087E-3,1.7775781E-1,1.7915823E-2,9.569802E-2,-8.437751E-3,3.8218956E-2,-1.368748E-1,-4.61842E-2,1.6069409E-1,3.5291427E-1,-2.0565116E-3,5.8687085E-1,-4.2657557E-4,-1.3189177E-1,-1.6830482E-2,-4.1841697E-3,-1.0584011E-2,-1.9104682E-2,-6.853139E-3,9.133671E-3,-2.3891242E-1,-9.636414E-3,-3.065895E-3,-8.395773E-3,-1.9998362E-2,2.4143733E-2,-1.5036044E-1,-2.6829498E-2,1.2720603E-1,-5.235127E-3,-1.308249E-2,1.508029E-2,1.5215199E-1,1.3449143E-1,-1.1228424E-3,-9.066222E-2,6.833433E-2,-7.916652E-5,-1.1171382E-2,-8.8848695E-3,5.5086453E-2,1.8045875E-1,-9.218476E-3,4.0806955E-1,1.8441735E-3,3.2386098E-2,4.6930048E-1,-8.346902E-3,-3.3596912E-3,-2.7991658E-1,-7.335519E-2,-3.6704876E-3,4.3540094E-3,-1.982777E-1,-1.2981921E-2,-7.813894E-2,3.0807743E-2,1.8298261E-1,-6.121228E-3,1.6421536E-1,-1.1758386E-3,2.7201432E-3,8.056629E-3,-1.0870488E-2,-4.50208E-2,-5.41519E-2,9.450346E-2,-1.7841109E-3,5.891995E-3,3.3544743E-1,1.3843986E-1,4.3896627E-1,5.0901463E-3,4.9769557E-1,1.0852653E-2,-1.2632258E-3,-2.9235902E-1,-3.0671412E-3,-1.3167296E-1,-3.5097697E-1,-1.1094296E-1,1.8554485E-3,-3.3200176E-3,-1.1270994E-1,4.6013794E-3,8.948036E-2,-5.161618E-2,8.999942E-2,1.4914457E-2,2.9274397E-2,1.8403146E-1,-9.048598E-3,1.4358122E-2,3.214653E-3,-7.4449317E-3,2.0508094E-2,1.3625069E-1,1.0811578E-2,1.9417403E-2,9.204352E-2,2.596275E-1,9.556691E-3,2.1355174E-2,2.5890335E-2,1.5765306E-2,-1.0036199E-2,-1.617885E-2,-2.627989E-3,2.3682076E-3,-8.100321E-3,-1.639086E-3,-2.5239557E-2,-9.399405E-3,2.6148575E-4,-8.734189E-3,1.9759787E-3,-5.905007E-3,2.4998833E-3,-3.8167748E-3,3.16947E-3,7.113256E-3,-7.1091275E-4,-5.467072E-3,-3.112851E-4,5.9766853E-3,-1.2287435E-3,3.6631525E-3,3.609185E-3,9.420872E-3,4.633323E-3,-1.0461089E-3,2.9267166E-3,-1.7201228E-3,6.909068E-3,-1.1526215E-3,4.6080537E-3,1.392952E-2,9.913788E-4,6.5795113E-3,6.3021085E-3,1.3699164E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,-1,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,-1,57,59,-1,61,63,-1,65,67,-1,69,71,73,-1,-1,-1,75,77,79,81,83,85,-1,87,-1,89,-1,91,93,95,97,99,-1,101,-1,103,-1,-1,-1,-1,-1,-1,105,-1,-1,-1,-1,107,109,111,113,-1,-1,-1,115,117,-1,119,121,-1,-1,-1,123,125,-1,127,-1,-1,129,-1,-1,131,133,-1,-1,135,137,139,141,143,-1,145,-1,-1,-1,-1,147,149,151,-1,-1,153,155,157,-1,159,-1,-1,161,163,165,167,169,-1,-1,171,173,175,177,179,-1,181,183,-1,185,187,-1,189,191,-1,-1,193,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2982693E1,1.65317E1,9.031397E0,2.0216312E0,4.5165052E0,2.2429752E-1,5.0224094E0,9.314451E-1,1.1913033E0,1.5915806E0,1.2540224E0,0E0,0E0,0E0,5.129442E0,4.7395802E-1,0E0,2.1805143E-1,3.197588E0,1.4673373E0,6.709602E-1,5.8512497E-1,5.432236E-1,1.9698946E0,1.6932831E0,4.3719292E-1,0E0,0E0,1.5163422E-1,1.2988806E-1,1.1273355E0,1.7654634E-1,5.303656E-1,1.2779498E-1,0E0,4.4356394E-1,3.4301212E-1,0E0,4.3815967E-1,6.519908E-1,0E0,8.6762667E-1,1.3558903E0,0E0,2.75622E-1,2.1977395E-2,1.4442635E-1,0E0,0E0,0E0,8.6108685E-1,4.4336677E-2,1.0866916E-1,3.572557E-1,4.9624702E-1,6.441629E-2,0E0,2.3202395E-1,0E0,8.2408905E-2,0E0,5.301715E-1,2.2651586E-1,2.9177466E-1,6.5543747E-1,7.7782726E-1,0E0,6.238079E-1,0E0,2.7269974E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.854744E-1,0E0,0E0,0E0,0E0,1.2367914E-1,3.2656598E-1,3.809461E-1,3.863781E-1,0E0,0E0,0E0,1.4344203E-1,2.8950542E-2,0E0,1.6104124E-1,3.577069E-1,0E0,0E0,0E0,8.6390056E-2,4.9409437E-1,0E0,3.105855E-1,0E0,0E0,1.3599586E-1,0E0,0E0,2.5284433E-1,8.977506E-2,0E0,0E0,4.5449173E-1,4.6694864E-2,1.9502687E-1,2.996163E-1,2.440188E-1,0E0,1.5772676E-1,0E0,0E0,0E0,0E0,1.9090365E-1,1.2327917E-1,2.7831393E-1,0E0,0E0,4.7944784E-2,3.5282135E-1,9.347296E-2,0E0,1.3253593E-1,0E0,0E0,2.6717567E-1,3.5077408E-2,4.4179976E-2,2.6347935E-1,2.287741E-1,0E0,0E0,1.1820942E-1,9.515066E-2,3.7902355E-2,6.1025783E-2,5.4189637E-2,0E0,2.7259042E-2,9.570241E-2,0E0,5.3145107E-2,3.641135E-2,0E0,2.0317085E-1,3.198514E-1,0E0,0E0,1.7472222E-1,4.13841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,38,38,39,39,41,41,42,42,44,44,45,45,46,46,50,50,51,51,52,52,53,53,54,54,55,55,57,57,59,59,61,61,62,62,63,63,64,64,65,65,67,67,69,69,76,76,81,81,82,82,83,83,84,84,88,88,89,89,91,91,92,92,96,96,97,97,99,99,102,102,105,105,106,106,109,109,110,110,111,111,112,112,113,113,115,115,120,120,121,121,122,122,125,125,126,126,127,127,129,129,132,132,133,133,134,134,135,135,136,136,139,139,140,140,141,141,142,142,143,143,145,145,146,146,148,148,149,149,151,151,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,-1,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,-1,58,60,-1,62,64,-1,66,68,-1,70,72,74,-1,-1,-1,76,78,80,82,84,86,-1,88,-1,90,-1,92,94,96,98,100,-1,102,-1,104,-1,-1,-1,-1,-1,-1,106,-1,-1,-1,-1,108,110,112,114,-1,-1,-1,116,118,-1,120,122,-1,-1,-1,124,126,-1,128,-1,-1,130,-1,-1,132,134,-1,-1,136,138,140,142,144,-1,146,-1,-1,-1,-1,148,150,152,-1,-1,154,156,158,-1,160,-1,-1,162,164,166,168,170,-1,-1,172,174,176,178,180,-1,182,184,-1,186,188,-1,190,192,-1,-1,194,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.8988005E5,1E0,3.3832976E7,9.31E2,9.3326636E-2,2.4481E4,6.9934395E-5,2.71E2,2.9802957E0,3.1706784E0,-2.5141533E-2,-9.822276E-3,4.3573078E-2,1.7587205E6,4.1361522E6,-4.2016767E-2,3.89E2,2.5E-1,3.89E2,2.0303884E6,1.0292E4,1.3208092E0,5.428175E3,5.97E3,4.622222E1,-2.9297657E-2,-2.4053128E-2,1.16E2,1.9E1,1.0479E4,3.8275862E0,1E0,1.339646E6,-2.5996389E-2,1.28738E5,4.68418E5,1.2612889E-2,1E0,1E0,3.3153426E-2,1.2673605E7,2.2313573E2,-2.0340558E-2,1E0,1.2E2,3.328629E6,8.407582E-3,2.2123247E-2,-3.3229895E-2,2.2E1,3E0,4.4761734E5,5E-1,5.5593027E9,3.1488764E0,-2.4057087E-3,1.9913E4,1.7915823E-2,1.9851096E7,-8.437751E-3,1E0,6.763312E7,2.0939393E0,3.0070068E7,1.9269184E7,-2.0565116E-3,1.7288135E0,-4.2657557E-4,1.16408E5,-1.6830482E-2,-4.1841697E-3,-1.0584011E-2,-1.9104682E-2,-6.853139E-3,9.133671E-3,7.997723E6,-9.636414E-3,-3.065895E-3,-8.395773E-3,-1.9998362E-2,5.44E2,1.2673605E7,2.956111E2,1.3508157E7,-5.235127E-3,-1.308249E-2,1.508029E-2,3.472E3,1.42946E5,-1.1228424E-3,4.8709216E5,3.7990784E7,-7.916652E-5,-1.1171382E-2,-8.8848695E-3,1.090477E6,1.4007937E0,-9.218476E-3,5.526E3,1.8441735E-3,3.2386098E-2,1.9995576E7,-8.346902E-3,-3.3596912E-3,1E0,1.8E0,-3.6704876E-3,4.3540094E-3,5.4E0,6.346204E-7,3.011152E6,1E0,1.65E2,-6.121228E-3,4.0906172E5,-1.1758386E-3,2.7201432E-3,8.056629E-3,-1.0870488E-2,4.88E2,5.2663404E-1,5.095006E5,-1.7841109E-3,5.891995E-3,1.1772152E0,1.9616238E2,2.0045958E6,5.0901463E-3,1.407E3,1.0852653E-2,-1.2632258E-3,1.6213593E1,1.3823239E2,5.7E3,2.8055556E0,1.5277778E0,1.8554485E-3,-3.3200176E-3,1.631108E6,1E0,6.5E1,4.017544E0,6.078218E2,1.4914457E-2,1.2E1,1.071E3,-9.048598E-3,1.5724638E1,1.15E2,-7.4449317E-3,1.26E2,2.954124E6,1.0811578E-2,1.9417403E-2,5.615327E5,1.7755102E0,9.556691E-3,2.1355174E-2,2.5890335E-2,1.5765306E-2,-1.0036199E-2,-1.617885E-2,-2.627989E-3,2.3682076E-3,-8.100321E-3,-1.639086E-3,-2.5239557E-2,-9.399405E-3,2.6148575E-4,-8.734189E-3,1.9759787E-3,-5.905007E-3,2.4998833E-3,-3.8167748E-3,3.16947E-3,7.113256E-3,-7.1091275E-4,-5.467072E-3,-3.112851E-4,5.9766853E-3,-1.2287435E-3,3.6631525E-3,3.609185E-3,9.420872E-3,4.633323E-3,-1.0461089E-3,2.9267166E-3,-1.7201228E-3,6.909068E-3,-1.1526215E-3,4.6080537E-3,1.392952E-2,9.913788E-4,6.5795113E-3,6.3021085E-3,1.3699164E-2],"split_indices":[2,43,17,7,2,53,12,52,2,68,69,0,0,0,43,60,0,1,73,2,60,44,68,67,2,4,0,0,2,0,9,69,29,9,0,1,9,0,19,27,0,60,73,0,8,44,9,0,0,0,10,8,43,68,46,73,0,9,0,60,0,29,7,68,58,62,0,69,0,7,0,0,0,0,0,0,9,0,0,0,0,12,60,4,60,0,0,0,10,1,0,43,7,0,0,0,60,68,0,2,0,0,9,0,0,13,68,0,0,73,52,9,26,0,0,43,0,0,0,0,4,57,43,0,0,68,71,43,0,0,0,0,73,67,10,71,68,0,0,5,30,0,69,48,0,3,2,0,73,0,0,10,62,0,0,43,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,7.78E2,2.61E2,2.39E2,5.39E2,1.4E1,2.47E2,7.5E1,1.64E2,2.76E2,2.63E2,9E0,5E0,1.3E1,2.34E2,7.1E1,4E0,4.4E1,1.2E2,2.46E2,3E1,1.06E2,1.57E2,1.09E2,1.25E2,6.1E1,1E1,1.3E1,3.1E1,9E0,1.11E2,3.1E1,2.15E2,2.3E1,7E0,8.5E1,2.1E1,8E0,1.49E2,1.04E2,5E0,5.1E1,7.4E1,3.8E1,2.3E1,9E0,2.2E1,5E0,4E0,4E0,1.07E2,1.8E1,1.3E1,6.4E1,1.51E2,1.8E1,5E0,7.3E1,1.2E1,1.6E1,5E0,1.34E2,1.5E1,1.8E1,8.6E1,4.5E1,6E0,7E1,4E0,1.3E1,1E1,4E0,5E0,1.8E1,4E0,4E0,1.03E2,1.4E1,4E0,5E0,8E0,1.5E1,4.9E1,1.27E2,2.4E1,5E0,1.3E1,9E0,6.4E1,1.2E1,4E0,2.5E1,1.09E2,7E0,8E0,7E0,1.1E1,8.2E1,4E0,3.8E1,7E0,3.3E1,3.7E1,6E0,7E0,8.2E1,2.1E1,6E0,9E0,3.6E1,1.3E1,6.7E1,6E1,2E1,4E0,6E1,4E0,5E0,7E0,5E0,2E1,1.9E1,9E1,5E0,6E0,1.6E1,6.6E1,3.4E1,4E0,3.2E1,5E0,4E0,7.8E1,1E1,1.1E1,1.2E1,2.4E1,7E0,6E0,4.7E1,2E1,3.5E1,2.5E1,1.3E1,7E0,8E0,5.2E1,5E0,1.5E1,1.3E1,6E0,3.3E1,5.7E1,9E0,7E0,4.9E1,1.7E1,4E0,3E1,2.1E1,1.1E1,3.6E1,4.2E1,5E0,5E0,7E0,4E0,4E0,8E0,1E1,1.4E1,4E0,4.3E1,1.3E1,7E0,2.8E1,7E0,1.7E1,8E0,4E0,9E0,4E0,4E0,9E0,4.3E1,4E0,1.1E1,5E0,8E0,8E0,2.5E1,4.8E1,9E0,2.1E1,2.8E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-2.1113707E-3,-1.0200969E-1,2.9720458E-1,-3.0300242E-1,-7.6127104E-3,-4.0487608E-1,3.3611038E-1,-4.32657E-1,-2.3474638E-1,-8.9797586E-2,8.5915886E-2,-2.2989823E-2,-6.3940496E-3,1.9984965E-1,4.3687844E-1,-4.8388547E-1,-2.7575174E-1,-3.1926566E-1,-9.4179496E-2,-2.3252317E-1,-6.087035E-2,1.5551583E-1,1.09599745E-2,3.3258677E-2,3.0441844E-1,3.7213E-2,3.862008E-1,-4.2371097E-1,-3.2307148E-2,-1.2989971E-1,-1.8416071E-2,-2.7564287E-1,-5.7377714E-1,3.7168902E-1,-1.6870017E-1,-1.5832473E-1,-2.6235604E-1,-4.6340823E-2,-3.5253155E-1,1.7503917E-1,-7.3555475E-3,3.770238E-2,-1.6522206E-1,-7.3001226E-3,6.3377075E-2,3.429147E-1,-5.881937E-4,4.063643E-1,-8.269427E-3,-4.4448215E-1,-1.0609695E-2,-1.1470621E-2,1.3799048E-3,-3.544665E-1,-2.0092966E-1,-1.1726351E-2,-3.6128372E-2,2.182578E-2,9.083242E-3,-8.356933E-2,-3.2560232E-1,-1.1683327E-2,-4.266479E-3,-3.1641376E-1,-1.7025228E-1,-7.5457096E-2,5.7075273E-2,-4.0173042E-4,-2.9457076E-2,1.3323134E-1,3.30252E-1,9.147734E-3,1.752083E-1,-1.2054948E-4,-2.914087E-1,3.6195736E-2,1.0630331E-2,3.89102E-1,1.699616E-1,3.0364278E-1,5.201724E-1,-1.6423136E-2,-2.2907563E-2,-3.745789E-1,-6.0390932E-3,4.752744E-3,-2.4677978E-1,-1.4514278E-1,1.6414579E-2,-1.7861769E-2,-7.6578357E-3,-8.806472E-3,-1.727421E-2,-1.05719855E-2,-3.6487186E-3,-5.0267972E-2,-2.2558868E-1,8.590074E-2,-6.110608E-3,1.4697424E-1,-4.605972E-3,1.7539717E-2,1.0079776E-2,2.7939972E-2,-9.807906E-3,1.1873181E-2,8.407445E-2,-1.7347636E-2,-6.885414E-3,7.664771E-3,1.2261668E-3,4.2744943E-1,2.1214108E-1,3.9057012E-3,1.0323624E-2,1.3112285E-3,3.2573983E-1,5.5834115E-1,8.002984E-3,-1.814009E-2,-8.166824E-3,-2.0136353E-1,-1.7184913E-2,7.246878E-4,-2.1648586E-1,-3.1306863E-2,4.3064235E-3,-9.680902E-2,2.788872E-3,-6.1936937E-2,-3.1377172E-1,3.7719805E-2,1.6785988E-1,1.073693E-1,2.5471452E-1,2.0379356E-1,6.174806E-3,5.137581E-4,6.045468E-3,3.122814E-2,-6.33057E-3,2.4400657E-1,2.176655E-2,1.4226803E-2,2.5292698E-3,1.7177637E-1,3.6794648E-1,4.9142686E-1,3.2846943E-2,-2.3944591E-1,4.5805506E-4,-1.2775961E-2,-5.629924E-3,-3.0526544E-3,7.704906E-4,1.2867505E-3,-1.3116398E-1,4.4401463E-2,-6.0443643E-2,-6.868443E-3,9.546565E-4,-4.096277E-1,-6.098741E-3,6.5622004E-3,-1.930035E-3,3.8364648E-3,1.1193912E-2,2.1227369E-1,8.599105E-2,1.2040161E-1,3.1434587E-1,1.3900035E-2,3.0470886E-3,-7.5299838E-3,2.3498027E-2,4.6168263E-3,1.1008066E-3,6.1910963E-3,1.4100073E-2,9.821588E-3,2.20022E-3,3.9764947E-1,8.01268E-3,2.4961524E-2,1.704951E-2,-1.1986963E-2,-6.5753767E-3,1.9012865E-3,-6.5830667E-3,-1.111884E-2,-5.067491E-3,6.215874E-4,6.2839515E-3,-4.8681768E-3,1.6382809E-3,-2.540426E-2,-1.0370468E-2,2.1250919E-3,-6.3480353E-3,1.525366E-2,4.5152805E-3,-1.952035E-3,4.439086E-3,8.087964E-3,1.9244282E-3,1.7417217E-2,6.795048E-3,2.045517E-3,-4.0745554E-3,-2.1880982E-3,1.471022E-3,2.2327043E-2,1.1705415E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,-1,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,-1,79,-1,81,-1,-1,-1,83,85,-1,-1,-1,-1,87,89,-1,-1,91,93,95,97,-1,-1,99,101,103,105,-1,107,109,-1,111,113,115,117,-1,-1,119,-1,-1,121,123,125,-1,-1,-1,-1,-1,-1,127,129,131,-1,133,-1,-1,-1,135,-1,-1,137,-1,-1,-1,139,141,143,-1,-1,-1,145,147,-1,-1,-1,149,-1,-1,151,153,-1,155,157,159,161,163,165,167,169,171,173,-1,-1,175,-1,177,-1,-1,-1,179,181,183,-1,185,-1,-1,-1,-1,-1,187,189,191,193,-1,-1,195,-1,-1,197,-1,-1,199,201,203,205,-1,-1,-1,207,-1,209,-1,-1,-1,-1,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0828514E1,1.4663904E1,7.183359E0,2.0843143E0,4.058572E0,2.7714157E-1,3.27672E0,5.4176044E-1,1.9183521E0,1.1459987E0,1.2868767E0,0E0,0E0,1.8364182E0,2.3041878E0,5.073824E-1,3.604071E-1,9.7429657E-1,2.2498646E0,6.2499523E-2,9.7952265E-1,8.143754E-1,5.716131E-1,2.5196993E-1,7.9795694E-1,0E0,1.4972763E0,1.0349655E-1,0E0,2.3439042E-1,0E0,4.5291328E-1,7.342825E-1,6.585169E-2,7.074363E-1,7.646945E-2,1.054554E-1,6.8180233E-1,1.0726861E0,7.4118257E-1,0E0,4.1067266E-1,3.2755244E-1,0E0,1.5934475E-1,3.971858E-1,0E0,1.256609E0,0E0,2.9431343E-2,0E0,0E0,0E0,1.4003944E-1,6.9400346E-1,0E0,0E0,0E0,0E0,2.301676E-1,1.2124705E-1,0E0,0E0,6.614208E-2,5.6786835E-2,6.542363E-1,2.842557E-1,0E0,0E0,3.251629E-1,5.5583E-2,3.7213796E-1,1.1256564E-1,0E0,5.1782787E-2,1.4948344E-1,0E0,2.1469021E-1,4.5086473E-2,3.9219618E-1,6.4485455E-1,0E0,0E0,6.642771E-2,0E0,0E0,1.6390634E-1,2.6729858E-1,5.7893336E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.7560678E-1,3.4134865E-1,1.6741857E-1,0E0,3.6521983E-1,0E0,0E0,0E0,3.1259516E-1,0E0,0E0,3.2674477E-2,0E0,0E0,0E0,1.16272494E-1,1.7361403E-1,1.3393152E-1,0E0,0E0,0E0,3.3319092E-1,1.6131306E-1,0E0,0E0,0E0,2.678914E-1,0E0,0E0,5.722308E-2,1.7785849E-2,0E0,2.7314502E-1,1.9213514E-1,7.471498E-2,2.2469199E-1,1.2022841E-1,8.089721E-2,1.3649082E-1,1.5437448E-1,1.00586504E-1,2.2068545E-1,0E0,0E0,4.749781E-2,0E0,1.9682765E-2,0E0,0E0,0E0,6.5886766E-2,2.0257378E-1,2.7907372E-2,0E0,1.7505407E-2,0E0,0E0,0E0,0E0,0E0,1.319717E-1,1.1470795E-1,1.22922346E-1,1.286989E-1,0E0,0E0,1.3626087E-1,0E0,0E0,1.4917064E-1,0E0,0E0,1.1627385E-1,8.0442876E-2,3.145042E-2,1.1223614E-1,0E0,0E0,0E0,1.6171852E-1,0E0,2.6834093E-2,0E0,0E0,0E0,0E0,3.7173986E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,47,47,49,49,53,53,54,54,59,59,60,60,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,77,77,78,78,79,79,80,80,83,83,86,86,87,87,88,88,95,95,96,96,97,97,99,99,103,103,106,106,110,110,111,111,112,112,116,116,117,117,121,121,124,124,125,125,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,141,141,145,145,146,146,147,147,149,149,155,155,156,156,157,157,158,158,161,161,164,164,167,167,168,168,169,169,170,170,174,174,176,176,181,181],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,-1,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,-1,80,-1,82,-1,-1,-1,84,86,-1,-1,-1,-1,88,90,-1,-1,92,94,96,98,-1,-1,100,102,104,106,-1,108,110,-1,112,114,116,118,-1,-1,120,-1,-1,122,124,126,-1,-1,-1,-1,-1,-1,128,130,132,-1,134,-1,-1,-1,136,-1,-1,138,-1,-1,-1,140,142,144,-1,-1,-1,146,148,-1,-1,-1,150,-1,-1,152,154,-1,156,158,160,162,164,166,168,170,172,174,-1,-1,176,-1,178,-1,-1,-1,180,182,184,-1,186,-1,-1,-1,-1,-1,188,190,192,194,-1,-1,196,-1,-1,198,-1,-1,200,202,204,206,-1,-1,-1,208,-1,210,-1,-1,-1,-1,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,7.7E1,9.48E2,1.5900173E3,9.9347424E2,1.4956522E1,7.24E2,3.79E2,3.774648E0,-2.2989823E-2,-6.3940496E-3,6.7148806E5,1.059448E7,1.0195397E8,3.2856784E7,7.382199E0,2.31E2,1.4452E4,3.6E1,2.83475E5,9.639872E0,8.147158E4,2E0,3.7213E-2,7.06191E7,5.707547E0,-3.2307148E-2,6.8E1,-1.8416071E-2,2.55E2,2.49E2,1.0101351E4,2.0734E4,4.9E2,4.795239E7,2.331083E6,5.69E2,6.009825E2,-7.3555475E-3,3.1797794E1,5.144203E2,-7.3001226E-3,7.098E3,2E0,-5.881937E-4,2.8530578E6,-8.269427E-3,1E0,-1.0609695E-2,-1.1470621E-2,1.3799048E-3,1.8E1,8.414097E-1,-1.1726351E-2,-3.6128372E-2,2.182578E-2,9.083242E-3,1.3022917E8,2.9994638E2,-1.1683327E-2,-4.266479E-3,3.390625E0,3E0,2.87E2,1.8E1,-4.0173042E-4,-2.9457076E-2,1E0,7.403968E2,5.1034E4,3.093458E0,-1.2054948E-4,1.2218E4,3.3382E4,1.0630331E-2,7.7191065E6,1.8062708E6,1E0,5.489183E1,-1.6423136E-2,-2.2907563E-2,3.328629E6,-6.0390932E-3,4.752744E-3,2.1E1,3.716E3,4.080551E-1,-1.7861769E-2,-7.6578357E-3,-8.806472E-3,-1.727421E-2,-1.05719855E-2,-3.6487186E-3,6.99E2,9.935484E-1,1.4285715E0,-6.110608E-3,2.8802464E8,-4.605972E-3,1.7539717E-2,1.0079776E-2,7.9555137E3,-9.807906E-3,1.1873181E-2,1.909E3,-1.7347636E-2,-6.885414E-3,7.664771E-3,4.539777E5,4.6134964E2,1.8667632E0,3.9057012E-3,1.0323624E-2,1.3112285E-3,1.0168186E5,6.663214E6,8.002984E-3,-1.814009E-2,-8.166824E-3,1.157602E6,-1.7184913E-2,7.246878E-4,7.0093E4,1.1358E4,4.3064235E-3,1.2916666E0,9.6E1,3.57E2,8.49E2,3.5714287E-1,4.5866325E6,7.8571427E-1,5.4007E7,1.3E1,3.257507E5,5.137581E-4,6.045468E-3,1.5998265E6,-6.33057E-3,5.941442E6,2.176655E-2,1.4226803E-2,2.5292698E-3,5.5933E5,2E1,2.9366477E0,3.2846943E-2,6.763314E7,4.5805506E-4,-1.2775961E-2,-5.629924E-3,-3.0526544E-3,7.704906E-4,2.8361E4,2.590909E0,7.1E1,1.432632E6,-6.868443E-3,9.546565E-4,7.6116E4,-6.098741E-3,6.5622004E-3,5.405423E6,3.8364648E-3,1.1193912E-2,3.074398E0,3.6E1,5.5426865E6,8.051603E7,1.3900035E-2,3.0470886E-3,-7.5299838E-3,4.2E2,4.6168263E-3,2.6463525E8,6.1910963E-3,1.4100073E-2,9.821588E-3,2.20022E-3,1.9347133E5,8.01268E-3,2.4961524E-2,1.704951E-2,-1.1986963E-2,-6.5753767E-3,1.9012865E-3,-6.5830667E-3,-1.111884E-2,-5.067491E-3,6.215874E-4,6.2839515E-3,-4.8681768E-3,1.6382809E-3,-2.540426E-2,-1.0370468E-2,2.1250919E-3,-6.3480353E-3,1.525366E-2,4.5152805E-3,-1.952035E-3,4.439086E-3,8.087964E-3,1.9244282E-3,1.7417217E-2,6.795048E-3,2.045517E-3,-4.0745554E-3,-2.1880982E-3,1.471022E-3,2.2327043E-2,1.1705415E-2],"split_indices":[2,43,17,44,2,70,67,73,2,2,69,0,0,43,12,46,60,69,44,9,3,1,69,43,6,0,59,69,0,0,0,2,2,43,44,1,7,43,2,70,0,71,4,0,2,32,0,43,0,20,0,0,0,3,68,0,0,0,0,7,67,0,0,69,8,0,3,0,0,6,4,44,68,0,44,9,0,62,43,26,71,0,0,9,0,0,8,44,53,0,0,0,0,0,0,2,68,68,0,5,0,0,0,62,0,0,2,0,0,0,43,4,57,0,0,0,48,43,0,0,0,12,0,0,1,44,0,68,0,0,2,68,43,68,7,3,43,0,0,62,0,60,0,0,0,9,3,53,0,7,0,0,0,0,0,1,69,0,12,0,0,12,0,0,43,0,0,71,0,43,7,0,0,0,0,0,7,0,0,0,0,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.72E2,2.57E2,2.46E2,5.26E2,1.3E1,2.44E2,8.3E1,1.63E2,2.8E2,2.46E2,9E0,4E0,1.05E2,1.39E2,6.1E1,2.2E1,1.01E2,6.2E1,4.6E1,2.34E2,1.27E2,1.19E2,4.1E1,6.4E1,1.5E1,1.24E2,5E1,1.1E1,1.1E1,1.1E1,8.8E1,1.3E1,8E0,5.4E1,1.5E1,3.1E1,2.24E2,1E1,1.2E2,7E0,1.04E2,1.5E1,5E0,3.6E1,5.7E1,7E0,1.2E2,4E0,4.4E1,6E0,6E0,5E0,4.1E1,4.7E1,6E0,7E0,4E0,4E0,3.6E1,1.8E1,5E0,1E1,1.8E1,1.3E1,1.75E2,4.9E1,5E0,5E0,9.6E1,2.4E1,8.7E1,1.7E1,7E0,8E0,3.2E1,4E0,4.4E1,1.3E1,6.5E1,5.5E1,1.9E1,2.5E1,3.7E1,4E0,6E0,4.1E1,2.2E1,1.4E1,1.2E1,6E0,7E0,1.1E1,7E0,6E0,1.51E2,2.4E1,4.3E1,6E0,9.1E1,5E0,1.5E1,9E0,8.1E1,6E0,8E0,9E0,4E0,4E0,6E0,2.6E1,3.5E1,9E0,6E0,7E0,5E0,6E1,4.9E1,6E0,3.3E1,4E0,3.2E1,9E0,7E0,1.5E1,9E0,5E0,8E1,7.1E1,9E0,1.5E1,2.8E1,1.5E1,6.8E1,2.3E1,8E0,7.3E1,4E0,5E0,2.2E1,4E0,8E0,2.7E1,5E0,4E0,1.4E1,4.6E1,3.7E1,1.2E1,2.7E1,5E0,8E0,7E0,5E0,4E0,2.1E1,5.9E1,4.3E1,2.8E1,4E0,5E0,9E0,6E0,7E0,2.1E1,8E0,7E0,1E1,5.8E1,8E0,1.5E1,4E0,4E0,6E0,6.7E1,6E0,1.6E1,4E0,4E0,1E1,4E0,3.9E1,7E0,2.4E1,1.3E1,2.1E1,6E0,1.7E1,4E0,8E0,5.1E1,3.3E1,1E1,1.9E1,9E0,4E0,5E0,1.6E1,5E0,4E0,6E0,4E0,5.4E1,4E0,4E0,1E1,5E0,5.7E1,1E1,6E0,1E1,2.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[1.4223594E-2,-8.634216E-2,3.0193153E-1,-2.7222386E-1,-6.765204E-3,7.940106E-1,2.5887245E-1,-4.040506E-1,-1.9873267E-1,-1.9419628E-1,3.405146E-2,4.0034797E-2,1.7948333E-2,-3.1415752E-1,2.9465175E-1,-3.5044417E-1,-5.3188396E-1,-2.8547212E-1,-4.5296635E-2,-1.10069335E-1,-2.6626772E-1,-2.4604792E-2,8.945693E-2,-2.5621648E-3,-1.8626895E-2,2.5246102E-1,5.9453166E-1,-3.798945E-1,-3.3814332E-3,-3.2094732E-2,-3.3830726E-1,-2.3854767E-1,-4.3946558E-1,2.438176E-2,-1.398615E-1,-3.6346477E-2,-1.7054267E-1,-1.8699847E-1,-3.5463905E-1,-1.1870257E-1,3.8861327E-3,1.0711176E-1,-1.8679948E-1,7.600477E-2,3.0271718E-1,1.3814948E-2,6.457696E-1,-1.8644042E-2,-2.1673195E-1,-7.077512E-3,-1.9020766E-2,-3.0667937E-1,-1.5621948E-1,-5.764379E-1,-2.6254186E-1,1.785968E-4,-1.918815E-1,-6.8915084E-2,2.8638325E-3,-2.1919231E-1,-1.8370199E-3,-1.3457746E-2,-1.2549819E-1,-2.8971314E-1,-2.4505343E-2,5.7134286E-2,-1.7573352E-1,-1.1868533E-2,2.2696266E-1,5.4524697E-2,1.6540694E-1,-8.068745E-4,-2.527867E-1,1.9602962E-2,1.4412609E-2,3.270215E-1,-1.413914E-1,3.3096164E-2,1.5074941E-2,-4.93375E-3,-1.3089308E-2,-2.1446358E-2,-2.6522526E-1,6.500218E-3,-1.9773133E-1,-1.43390065E-2,-3.1606384E-2,-1.4578584E-2,-6.2741884E-3,-4.3213803E-2,3.0139075E-3,-2.7602658E-1,-1.0649476E-1,-2.5482537E-4,-4.758274E-3,-4.357136E-3,-1.2010289E-2,-3.8744456E-3,-1.0793365E-2,-1.4583387E-2,-7.110835E-3,1.5007585E-4,4.751945E-3,-2.5428542E-1,-5.1831815E-2,3.7173957E-2,-7.588196E-2,5.351729E-3,1.3871613E-2,3.405184E-2,2.109637E-1,1.7860527E-1,-3.3879743E-3,-1.4970619E-2,-5.3922664E-3,7.656629E-2,-1.002134E-1,6.5897727E-1,2.9645064E-1,-1.0526263E-2,-1.2316033E-3,-1.7388396E-1,-1.418976E-2,-2.4303529E-1,-8.914783E-2,2.676173E-4,-3.8612033E-3,-1.4714607E-2,-6.7019206E-3,1.4255417E-3,-1.7226124E-1,-2.7240536E-3,-3.360536E-1,2.2319464E-2,-8.275968E-3,5.644833E-2,-1.0463053E-1,-2.444161E-2,-1.7546345E-1,1.2201927E-1,-1.5779171E-2,1.6101167E-2,5.1329285E-3,5.2408434E-2,2.0906413E-1,-1.5491253E-2,1.2345568E-1,-9.0388E-3,-7.442658E-4,1.6963217E-2,3.7650026E-2,3.2575783E-1,1.3192868E-1,-3.6361762E-3,-9.58352E-3,-8.411567E-3,-1.510531E-2,-9.4763655E-4,-6.603029E-3,-9.883966E-3,-4.064374E-3,-2.4326963E-2,-2.0158608E-1,3.7598812E-3,-1.8694615E-3,2.863704E-2,1.4276624E-1,-8.576235E-4,-9.433755E-3,-7.633371E-2,8.7890975E-2,-1.5007787E-3,-2.0385443E-1,6.7740284E-2,1.0025926E-2,-9.127981E-3,5.693843E-3,1.0643449E-1,-1.175741E-3,2.3600961E-1,2.5935834E-2,2.5258532E-3,-4.4623865E-3,1.1203726E-2,7.2118066E-2,-2.6310363E-3,3.3956462E-1,-1.8976894E-3,1.8226767E-1,-1.234629E-2,-3.2039825E-3,1.9455624E-3,-6.038761E-3,4.492031E-3,8.926657E-3,-6.647862E-3,-3.8139103E-4,9.4821565E-3,1.2269466E-3,-1.0759202E-2,-4.718485E-3,5.160075E-3,1.1655367E-3,5.1182546E-3,-1.3258873E-3,7.995434E-3,5.0633756E-4,1.1686313E-2,-1.1417135E-3,4.007029E-3,-9.792607E-4,1.2326427E-4,5.1006023E-3,9.331165E-3,1.7439645E-2,1.1267652E-2,5.0161807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,-1,77,-1,79,-1,-1,81,83,85,87,89,91,93,-1,95,-1,-1,97,99,-1,101,103,105,107,109,111,-1,113,115,-1,117,119,-1,-1,-1,-1,-1,121,-1,123,-1,-1,-1,-1,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,131,133,135,137,-1,-1,139,141,143,-1,-1,-1,145,147,149,151,-1,-1,153,-1,155,157,-1,-1,-1,-1,-1,159,-1,161,163,-1,165,167,169,171,173,175,-1,-1,177,179,181,183,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,189,-1,-1,191,193,-1,-1,195,197,-1,199,201,-1,-1,203,205,-1,207,209,-1,-1,-1,211,-1,213,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9827642E1,1.1299171E1,5.4517746E0,2.138277E0,4.107179E0,2.812481E-1,5.1567135E0,3.6867237E-1,1.9701042E0,5.527544E-1,1.435397E0,0E0,0E0,3.2311928E-1,2.7978554E0,4.621029E-1,4.5838213E-1,5.816426E-1,3.0542905E0,1.9888163E-1,2.9094434E-1,5.7686174E-1,1.1271461E0,0E0,0E0,1.8038015E0,2.3083878E-1,1.2103939E-1,0E0,0E0,9.998667E-2,3.7277794E-1,3.575673E-1,0E0,3.4970886E-1,7.473025E-2,1.5034372E-1,1.4946353E-1,1.0883498E-1,5.114477E-1,5.861001E-1,6.4701366E-1,1.5201959E-1,6.193949E-1,1.773242E0,0E0,2.8507996E-1,0E0,3.5632372E-2,0E0,0E0,1.3810015E-1,4.7154558E-1,1.0285854E-1,3.1056046E-2,4.2341985E-2,2.220751E-1,3.6162883E-2,0E0,6.7139804E-2,0E0,0E0,7.044408E-2,1.8370748E-2,0E0,3.1696096E-2,3.56663E-1,4.9256375E-1,5.4809332E-2,3.5793018E-1,3.3134794E-1,0E0,5.913955E-2,2.7385336E-1,0E0,1.347168E0,8.319841E-2,0E0,0E0,0E0,0E0,0E0,7.677555E-2,0E0,1.357503E-1,0E0,0E0,0E0,0E0,1.8286915E-2,0E0,4.8365593E-2,1.7694445E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.4752095E-1,1.5503596E-1,2.4916035E-1,3.4404194E-1,0E0,0E0,4.4904298E-1,1.3984573E-1,3.5804892E-1,0E0,0E0,0E0,1.184586E-1,1.0119216E-1,2.469325E-1,6.368799E-1,0E0,0E0,2.5191784E-2,0E0,5.3030014E-2,3.7670232E-2,0E0,0E0,0E0,0E0,0E0,2.6849806E-2,0E0,3.037399E-1,4.408544E-2,0E0,1.8555996E-1,9.0473436E-2,2.7485105E-1,8.674151E-2,1.7608303E-1,2.5877574E-1,0E0,0E0,8.68358E-2,3.738339E-1,6.2183775E-2,9.25259E-2,0E0,0E0,0E0,0E0,6.4812946E-1,2.077915E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.696691E-2,0E0,0E0,1.364909E-1,2.4864078E-2,0E0,0E0,1.4599709E-1,1.0251513E-1,0E0,3.2327116E-2,4.389397E-2,0E0,0E0,2.2004953E-1,7.444134E-2,0E0,3.0227232E-1,3.3359654E-2,0E0,0E0,0E0,3.693171E-2,0E0,5.2381897E-1,0E0,5.526066E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,82,82,84,84,89,89,91,91,92,92,103,103,104,104,105,105,106,106,109,109,110,110,111,111,115,115,116,116,117,117,118,118,121,121,123,123,124,124,130,130,132,132,133,133,135,135,136,136,137,137,138,138,139,139,140,140,143,143,144,144,145,145,146,146,151,151,152,152,162,162,165,165,166,166,169,169,170,170,172,172,173,173,176,176,177,177,179,179,180,180,184,184,186,186,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,-1,78,-1,80,-1,-1,82,84,86,88,90,92,94,-1,96,-1,-1,98,100,-1,102,104,106,108,110,112,-1,114,116,-1,118,120,-1,-1,-1,-1,-1,122,-1,124,-1,-1,-1,-1,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,132,134,136,138,-1,-1,140,142,144,-1,-1,-1,146,148,150,152,-1,-1,154,-1,156,158,-1,-1,-1,-1,-1,160,-1,162,164,-1,166,168,170,172,174,176,-1,-1,178,180,182,184,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,190,-1,-1,192,194,-1,-1,196,198,-1,200,202,-1,-1,204,206,-1,208,210,-1,-1,-1,212,-1,214,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.6E1,7.7E1,4.97E2,6.7321223E-1,6.829817E4,1.4504054E8,7.24E2,1.6923077E0,1.037E3,4.0034797E-2,1.7948333E-2,1.0322473E8,6.663214E6,4.56E2,1.4956522E1,1.7E1,2.04115E5,8.510249E-3,4.0753E4,1E0,9.639872E0,-2.5621648E-3,-1.8626895E-2,7.075161E2,3.53562E5,3.1530054E0,-3.3814332E-3,-3.2094732E-2,7.042942E7,2.55E2,6.593462E4,2.438176E-2,1.5522388E0,4.49239E6,6.769866E7,1.43E2,7.668863E6,5E-1,2.0200005E8,1.5115256E6,5.737478E7,6.103178E8,7.134432E7,1.3814948E-2,1.4379113E9,-1.8644042E-2,4.233978E0,-7.077512E-3,-1.9020766E-2,3.9E1,1E0,1.6E1,5E0,1.1E3,9.458613E3,2.0409561E6,2.8638325E-3,1.188E0,-1.8370199E-3,-1.3457746E-2,1.31625E2,4.48E2,-2.4505343E-2,7.52E2,8.1E2,1E0,1.6923077E0,3.299862E1,3.1027884E5,-8.068745E-4,3.38E2,1.9347133E5,1.4412609E-2,1.2083E4,9.4578236E-1,3.3096164E-2,1.5074941E-2,-4.93375E-3,-1.3089308E-2,-2.1446358E-2,1.4E1,6.500218E-3,8E0,-1.43390065E-2,-3.1606384E-2,-1.4578584E-2,-6.2741884E-3,2.02E2,3.0139075E-3,1E0,1.2038125E7,-2.5482537E-4,-4.758274E-3,-4.357136E-3,-1.2010289E-2,-3.8744456E-3,-1.0793365E-2,-1.4583387E-2,-7.110835E-3,1.5007585E-4,4.751945E-3,6.07E2,1.5E1,1E0,2E0,5.351729E-3,1.3871613E-2,1.4915254E0,3.13E2,3.7788504E7,-3.3879743E-3,-1.4970619E-2,-5.3922664E-3,3.0990322E5,1.4521204E0,2.8016653E3,1.7172844E7,-1.0526263E-2,-1.2316033E-3,1.00019E5,-1.418976E-2,5.80563E5,2.3266666E0,2.676173E-4,-3.8612033E-3,-1.4714607E-2,-6.7019206E-3,1.4255417E-3,2.58466E5,-2.7240536E-3,2.019537E2,1E1,-8.275968E-3,7.919006E7,6.763314E7,8.7E1,1.1356604E2,8.684966E-1,4.186E4,1.6101167E-2,5.1329285E-3,1.103E3,2.23099E7,3.921E3,5.7777777E0,-9.0388E-3,-7.442658E-4,1.6963217E-2,3.7650026E-2,3.167E3,1.465E4,-3.6361762E-3,-9.58352E-3,-8.411567E-3,-1.510531E-2,-9.4763655E-4,-6.603029E-3,-9.883966E-3,-4.064374E-3,-2.4326963E-2,1.4386049E6,3.7598812E-3,-1.8694615E-3,9.07826E0,2.28E0,-8.576235E-4,-9.433755E-3,2.118835E6,2E0,-1.5007787E-3,5.1042255E2,9.6E1,1.0025926E-2,-9.127981E-3,1.0439024E1,3.412336E7,-1.175741E-3,5.2409735E-7,2.7777777E0,2.5258532E-3,-4.4623865E-3,1.1203726E-2,8.11E2,-2.6310363E-3,8.747289E5,-1.8976894E-3,6.072304E5,-1.234629E-2,-3.2039825E-3,1.9455624E-3,-6.038761E-3,4.492031E-3,8.926657E-3,-6.647862E-3,-3.8139103E-4,9.4821565E-3,1.2269466E-3,-1.0759202E-2,-4.718485E-3,5.160075E-3,1.1655367E-3,5.1182546E-3,-1.3258873E-3,7.995434E-3,5.0633756E-4,1.1686313E-2,-1.1417135E-3,4.007029E-3,-9.792607E-4,1.2326427E-4,5.1006023E-3,9.331165E-3,1.7439645E-2,1.1267652E-2,5.0161807E-3],"split_indices":[2,43,0,44,2,53,43,46,2,68,2,0,0,5,43,10,73,8,5,53,12,29,69,0,0,67,1,68,0,0,60,2,43,0,68,43,7,2,5,68,5,43,5,7,59,0,12,0,68,0,0,2,21,3,69,2,48,60,0,68,0,0,67,2,0,12,2,26,68,71,48,0,0,48,0,9,49,0,0,0,0,0,3,0,8,0,0,0,0,10,0,27,12,0,0,0,0,0,0,0,0,0,0,2,3,19,32,0,0,68,0,7,0,0,0,43,53,4,60,0,0,5,0,12,68,0,0,0,0,0,1,0,4,3,0,7,7,0,67,57,9,0,0,10,9,2,71,0,0,0,0,2,44,0,0,0,0,0,0,0,0,0,43,0,0,71,71,0,0,9,8,0,4,0,0,0,73,7,0,52,69,0,0,0,0,0,43,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.63E2,2.66E2,2.28E2,5.35E2,2E1,2.46E2,8E1,1.48E2,9.5E1,4.4E2,1.6E1,4E0,1.4E1,2.32E2,5.9E1,2.1E1,9.4E1,5.4E1,4.5E1,5E1,2.14E2,2.26E2,4E0,1E1,2.05E2,2.7E1,5.3E1,6E0,1E1,1.1E1,7.4E1,2E1,7E0,4.7E1,2.1E1,2.4E1,2.8E1,2.2E1,4.9E1,1.65E2,2.13E2,1.3E1,4.6E1,1.59E2,5E0,2.2E1,4.5E1,8E0,4E0,7E0,3.9E1,3.5E1,1E1,1E1,1.3E1,3.4E1,1.6E1,5E0,1.7E1,7E0,9E0,1.9E1,1.8E1,4E0,1.2E1,3.7E1,1.55E2,1E1,1.13E2,1E2,4E0,9E0,3.8E1,8E0,1.51E2,8E0,1.7E1,5E0,4E0,4E0,6E0,3.3E1,4E0,3.1E1,4E0,6E0,6E0,4E0,8E0,5E0,1.6E1,1.8E1,6E0,1E1,5E0,1.2E1,1.5E1,4E0,1.4E1,4E0,6E0,6E0,2.2E1,1.5E1,8.8E1,6.7E1,5E0,5E0,1.01E2,1.2E1,9.5E1,5E0,5E0,4E0,2.6E1,1.2E1,1.1E1,1.4E2,4E0,4E0,1.2E1,2.1E1,2.1E1,1E1,4E0,4E0,1.1E1,5E0,6E0,1.2E1,7E0,1.5E1,1E1,5E0,7.8E1,1E1,4.5E1,2.2E1,3.6E1,6.5E1,4E0,8E0,1.9E1,7.6E1,9E0,1.7E1,5E0,7E0,5E0,6E0,1.18E2,2.2E1,4E0,8E0,1.4E1,7E0,5E0,5E0,7E0,5E0,5E0,1E1,5E0,5E0,6E1,1.8E1,6E0,4E0,3.1E1,1.4E1,4E0,1.8E1,2.4E1,1.2E1,6E0,5.9E1,1.1E1,8E0,6.6E1,1E1,5E0,4E0,4E0,1.3E1,4E0,1.14E2,5E0,1.7E1,6E0,4E0,5.6E1,4E0,1.1E1,7E0,1.5E1,1.6E1,4E0,1E1,1.3E1,5E0,1.1E1,1.3E1,1.4E1,4.5E1,6E0,5E0,6.2E1,4E0,4E0,6E0,5E0,8E0,2.6E1,8.8E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[1.1543609E-2,-8.338619E-2,3.051977E-1,-2.646278E-1,-3.4403836E-3,-3.7307516E-1,3.4792015E-1,-4.4438487E-1,-2.0215484E-1,-9.747073E-2,6.0152967E-2,-2.2339892E-2,-9.977077E-3,2.5551268E-1,5.1589066E-1,-3.5952973E-1,-6.9145286E-1,-2.3411669E-1,1.2810075E-1,-5.7790004E-2,-2.326693E-1,3.556915E-2,2.69155E-1,1.2523043E-1,3.551997E-1,6.626197E-1,2.2994138E-1,-3.8405544E-1,-7.7624316E-3,-1.7001208E-2,-3.622165E-2,-4.2762083E-1,-1.9992173E-1,2.8149905E-2,-1.4602683E-1,-8.362085E-2,9.852671E-2,-3.1050396E-1,-2.1463407E-2,1.14119865E-1,-7.978343E-3,1.4414949E-3,3.1053227E-1,2.2622343E-2,9.431877E-2,2.8278008E-1,4.7222808E-1,7.161346E-1,9.639213E-3,3.1366363E-1,-3.6032856E-3,-9.934622E-3,-4.0205657E-1,-8.603879E-3,-2.3185492E-2,-4.4929963E-2,-2.2759324E-1,3.441804E-4,-1.27123995E-2,-5.3285617E-2,-1.8312359E-1,6.300127E-2,1.02064395E-2,-1.6993105E-1,-4.6681356E-1,5.875403E-3,-9.363669E-2,1.3160466E-1,-1.1453167E-2,6.3196644E-3,-2.851087E-1,3.6019534E-1,5.0858995E-3,-2.9385116E-2,1.7119771E-1,2.0045193E-1,3.356893E-1,1.2639584E-2,2.4458908E-2,7.5985986E-1,1.8848816E-2,2.0518926E-1,2.1233212E-2,-2.591244E-1,-2.0399967E-2,-2.0485388E-1,2.0883279E-1,-2.8055418E-1,-1.5207374E-1,-1.4177686E-1,-2.4549743E-2,-1.0495798E-1,-2.6992512E-1,1.15091E-1,-1.3637824E-3,-1.19288955E-2,-2.4217737E-4,-3.022225E-1,-3.5502557E-2,-7.2634085E-3,6.42378E-5,6.72919E-2,2.542933E-1,-3.192825E-2,8.0160305E-2,-2.6347525E-3,-2.1074397E-2,3.8972566E-1,8.401162E-3,8.1129834E-2,-1.5587047E-1,1.6458493E-2,1.1194983E-1,2.489145E-1,-1.547148E-3,3.5823494E-1,6.5739336E-3,3.7956085E-2,2.4137238E-2,2.5682065E-1,2.335932E-3,-1.4580121E-2,-6.446454E-3,-1.4555292E-2,-3.8262142E-3,7.306444E-5,1.7293112E-2,-3.2006103E-1,-1.8519649E-1,-1.9031139E-1,4.326568E-2,-1.7694527E-1,2.2338717E-3,1.4073353E-2,-5.9194125E-2,-1.6171026E-1,7.129953E-4,-1.5398336E-3,-1.6305007E-2,7.289084E-3,2.585521E-3,-1.6314238E-2,-7.40696E-3,9.140162E-2,-1.1101867E-2,2.8632256E-1,8.160982E-4,-6.1204627E-2,5.801139E-2,-4.7625955E-3,1.6524196E-1,2.0019598E-2,8.762965E-3,7.8591466E-4,7.566744E-3,-1.0268122E-2,-2.4226597E-3,6.4916926E-4,1.39918E-1,6.4655896E-3,1.5030578E-2,1.7991068E-2,8.268109E-3,1.4572807E-2,5.700786E-3,-2.4285667E-1,-1.8167546E-2,-9.113013E-2,-1.5634745E-2,-2.2198202E-1,-1.1111426E-1,-1.84157E-3,5.43947E-3,-1.2372878E-2,-9.939814E-2,-2.0517794E-2,6.940438E-2,-9.404668E-2,2.4697002E-2,-3.5157346E-3,-8.8380575E-3,1.3244663E-1,6.370217E-4,3.975059E-3,3.1492576E-1,2.7158529E-2,-1.0737796E-1,1.1556205E-1,-4.6668056E-2,2.1825334E-2,-6.8090423E-3,1.936915E-1,-7.2736363E-4,1.5811813E-1,2.7241095E-3,-3.5474019E-3,-1.2960464E-2,-9.305543E-4,-8.974389E-3,-6.4456435E-3,-1.2947898E-2,-2.2215578E-3,-6.504209E-3,-5.7108663E-3,-1.4361461E-3,3.832691E-3,-2.9108948E-3,9.4427145E-4,5.979993E-3,-6.3294666E-3,-2.2214286E-3,2.6789808E-3,-2.1632232E-3,1.1095829E-2,4.425329E-3,-2.0961347E-3,1.9590857E-3,8.213827E-3,1.655673E-2,4.260382E-3,-3.8593335E-3,-7.925799E-3,-2.474342E-3,6.598231E-4,6.542611E-3,-5.9609422E-3,3.6020759E-3,-3.8036013E-3,2.5642654E-3,1.0251868E-2,1.4232753E-3,3.6959965E-3,8.329804E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,-1,73,75,77,79,-1,81,-1,-1,83,-1,-1,85,87,-1,-1,89,91,93,-1,95,97,-1,99,101,-1,103,105,107,-1,109,111,113,115,-1,-1,117,-1,119,-1,121,-1,123,125,127,129,131,133,135,137,139,-1,-1,-1,141,-1,-1,-1,143,145,147,149,-1,-1,151,-1,153,155,-1,157,159,-1,161,-1,-1,-1,163,-1,-1,-1,-1,-1,-1,-1,165,167,169,171,173,-1,175,177,179,-1,-1,-1,-1,-1,-1,-1,181,-1,183,-1,185,187,189,191,-1,-1,-1,-1,-1,-1,-1,193,-1,-1,-1,-1,-1,-1,195,-1,197,-1,199,201,-1,-1,-1,203,205,207,209,211,-1,-1,213,215,-1,217,219,221,223,225,227,-1,229,-1,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7957724E1,1.0982136E1,7.229288E0,2.5155144E0,3.1571677E0,1.5595078E-1,3.4537048E0,9.8833656E-1,1.8666615E0,1.1292517E0,1.6089178E0,0E0,0E0,1.9166231E0,3.2461567E0,1.5152693E-1,1.3071775E-1,9.6659184E-1,2.1996148E0,6.785125E-1,7.8241396E-1,9.689647E-1,3.1522393E-1,7.08601E-1,5.868511E-1,1.1381836E0,7.721162E-1,3.5241127E-2,0E0,0E0,0E0,3.1870747E-1,5.7734776E-1,0E0,2.2006813E-1,4.2187893E-1,9.043807E-2,6.897249E-1,1.6209728E-1,6.719564E-1,7.269231E-1,0E0,2.3907995E-1,0E0,6.0417473E-1,1.7228603E-1,2.0735884E-1,3.23246E-1,0E0,2.6979947E-1,0E0,0E0,8.323383E-2,0E0,0E0,9.356003E-1,4.2128706E-1,0E0,0E0,2.7877834E-1,1.955707E-1,9.9785306E-2,0E0,2.8940648E-1,5.621488E-1,0E0,6.0639888E-2,7.482611E-1,0E0,4.9680316E-1,3.2511312E-1,4.499674E-2,0E0,3.6210883E-1,3.8771605E-1,2.8043252E-1,8.855057E-2,0E0,0E0,1.568203E-1,0E0,1.0676861E-1,0E0,2.3493469E-2,0E0,1.621843E-1,3.080803E-1,1.9313383E-1,3.8169992E-1,1.8909329E-1,1.1456929E-1,1.3231261E-1,2.828126E-1,2.3979247E-2,0E0,0E0,0E0,3.006506E-2,0E0,0E0,0E0,5.0400937E-1,2.4505019E-1,3.0899096E-1,4.343098E-1,0E0,0E0,7.4831486E-2,0E0,7.141037E-2,7.074347E-2,0E0,8.291304E-2,1.2102866E-1,0E0,8.686447E-2,0E0,0E0,0E0,4.7103286E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6440296E-1,2.8625238E-1,8.068001E-2,6.0410865E-2,1.3545632E-1,0E0,7.989654E-2,1.3389459E-1,1.8377662E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2716707E-1,0E0,1.3903475E-1,0E0,3.619581E-1,1.8009982E-1,1.2076971E-1,1.5751868E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6037365E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3489258E-1,0E0,1.0589628E-1,0E0,1.0027349E-1,1.9412532E-2,0E0,0E0,0E0,1.9316971E-2,1.1954331E-1,4.438863E-2,5.659473E-2,3.6049075E-2,0E0,0E0,1.4285707E-1,4.035822E-2,0E0,8.525276E-2,2.3095919E-1,1.9115108E-1,4.811704E-2,1.2297499E-1,9.9283375E-2,0E0,1.12653494E-1,0E0,1.8501848E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,49,49,52,52,55,55,56,56,59,59,60,60,61,61,63,63,64,64,66,66,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,97,97,101,101,102,102,103,103,104,104,107,107,109,109,110,110,112,112,113,113,115,115,119,119,127,127,128,128,129,129,130,130,131,131,133,133,134,134,135,135,143,143,145,145,147,147,148,148,149,149,150,150,158,158,165,165,167,167,169,169,170,170,174,174,175,175,176,176,177,177,178,178,181,181,182,182,184,184,185,185,186,186,187,187,188,188,189,189,191,191,193,193],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,-1,74,76,78,80,-1,82,-1,-1,84,-1,-1,86,88,-1,-1,90,92,94,-1,96,98,-1,100,102,-1,104,106,108,-1,110,112,114,116,-1,-1,118,-1,120,-1,122,-1,124,126,128,130,132,134,136,138,140,-1,-1,-1,142,-1,-1,-1,144,146,148,150,-1,-1,152,-1,154,156,-1,158,160,-1,162,-1,-1,-1,164,-1,-1,-1,-1,-1,-1,-1,166,168,170,172,174,-1,176,178,180,-1,-1,-1,-1,-1,-1,-1,182,-1,184,-1,186,188,190,192,-1,-1,-1,-1,-1,-1,-1,194,-1,-1,-1,-1,-1,-1,196,-1,198,-1,200,202,-1,-1,-1,204,206,208,210,212,-1,-1,214,216,-1,218,220,222,224,226,228,-1,230,-1,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,1E0,3.3817584E7,8.1E2,9.3326636E-2,1.6975454E3,2.2E1,2.41E3,4.5709968E0,3.7951445E6,-2.2339892E-2,-9.977077E-3,1.6802971E6,1.2792593E2,2.46875E0,4.869229E7,1.0479E4,2.3E1,4.1973075E6,2.6801266E2,1.5132743E0,3.3875784E7,5E1,4.391553E6,4.262058E0,6.747343E8,2E0,-7.7624316E-3,-1.7001208E-2,-3.622165E-2,4E0,9.935484E-1,2.8149905E-2,2E1,3.6245735E0,1.6341463E0,2.6742857E0,3.1316666E2,1.4676277E7,1.2294118E1,1.4414949E-3,8.931E3,2.2622343E-2,8.190129E9,1.0089981E3,1.7068776E10,8.913649E-3,9.639213E-3,3.8771296E3,-3.6032856E-3,-9.934622E-3,1.863E3,-8.603879E-3,-2.3185492E-2,1.8236995E0,1.559733E6,3.441804E-4,-1.27123995E-2,1.3920766E2,9.6E1,6.34E2,1.02064395E-2,8.39777E5,1.1742609E7,5.875403E-3,7.09E2,4.797342E2,-1.1453167E-2,1.4603988E6,4E1,1.8E1,5.0858995E-3,1.2E1,1.3864E4,9.7944025E2,1E0,1.2639584E-2,2.4458908E-2,1.557718E7,1.8848816E-2,5.501002E9,2.1233212E-2,8.209663E3,-2.0399967E-2,8.579633E-2,4.13E2,3.9682608E2,6.4522544E5,2.331083E6,1.4007937E0,3.0948E2,1.45064E5,7.318515E6,-1.3637824E-3,-1.19288955E-2,-2.4217737E-4,1.7E1,-3.5502557E-2,-7.2634085E-3,6.42378E-5,2.511E3,2.0319266E-7,9.751103E6,6.763312E7,-2.6347525E-3,-2.1074397E-2,1E0,8.401162E-3,3.1720988E6,2.0312773E8,1.6458493E-2,8.620714E2,1.2544625E5,-1.547148E-3,1.8039524E7,6.5739336E-3,3.7956085E-2,2.4137238E-2,5.6516E4,2.335932E-3,-1.4580121E-2,-6.446454E-3,-1.4555292E-2,-3.8262142E-3,7.306444E-5,1.7293112E-2,3.962617E0,8.9535345E2,1.5152774E2,9.593451E4,2.6818182E0,2.2338717E-3,2.732381E2,2.347364E7,2.227328E4,7.129953E-4,-1.5398336E-3,-1.6305007E-2,7.289084E-3,2.585521E-3,-1.6314238E-2,-7.40696E-3,5.2767E4,-1.1101867E-2,3.292496E-2,8.160982E-4,2.67637E5,4.3E1,2.1651245E6,2.03353E-6,2.0019598E-2,8.762965E-3,7.8591466E-4,7.566744E-3,-1.0268122E-2,-2.4226597E-3,6.4916926E-4,1.2792593E2,6.4655896E-3,1.5030578E-2,1.7991068E-2,8.268109E-3,1.4572807E-2,5.700786E-3,2.90099E0,-1.8167546E-2,6.185E3,-1.5634745E-2,1.1967312E3,1.3E1,-1.84157E-3,5.43947E-3,-1.2372878E-2,2.707673E6,1E1,2.6818182E0,1.084437E6,1E0,-3.5157346E-3,-8.8380575E-3,7.8571427E-1,1.047E3,3.975059E-3,5.281628E2,7.755551E7,1.244E3,3.0536E4,1.7022566E7,1.0738284E4,-6.8090423E-3,1.7022608E7,-7.2736363E-4,1.4E1,2.7241095E-3,-3.5474019E-3,-1.2960464E-2,-9.305543E-4,-8.974389E-3,-6.4456435E-3,-1.2947898E-2,-2.2215578E-3,-6.504209E-3,-5.7108663E-3,-1.4361461E-3,3.832691E-3,-2.9108948E-3,9.4427145E-4,5.979993E-3,-6.3294666E-3,-2.2214286E-3,2.6789808E-3,-2.1632232E-3,1.1095829E-2,4.425329E-3,-2.0961347E-3,1.9590857E-3,8.213827E-3,1.655673E-2,4.260382E-3,-3.8593335E-3,-7.925799E-3,-2.474342E-3,6.598231E-4,6.542611E-3,-5.9609422E-3,3.6020759E-3,-3.8036013E-3,2.5642654E-3,1.0251868E-2,1.4232753E-3,3.6959965E-3,8.329804E-3],"split_indices":[2,43,17,7,2,53,67,3,2,69,43,0,0,43,73,68,46,9,0,43,70,68,7,0,43,53,47,32,0,0,0,10,68,0,3,71,68,71,4,62,68,0,44,0,46,4,46,72,0,67,0,0,9,0,0,69,9,0,0,70,0,2,0,9,5,0,2,4,0,43,3,3,0,3,10,70,6,0,0,66,0,5,0,43,0,53,2,4,62,43,68,4,7,60,0,0,0,3,0,0,0,2,52,9,7,0,0,31,0,60,7,0,4,48,0,9,0,0,0,9,0,0,0,0,0,0,0,71,4,67,43,69,0,4,5,66,0,0,0,0,0,0,0,1,0,73,0,9,8,62,52,0,0,0,0,0,0,0,73,0,0,0,0,0,0,69,0,44,0,48,3,0,0,0,5,3,69,9,30,0,0,68,2,0,4,5,2,1,9,62,0,9,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.001E3,7.57E2,2.44E2,2.31E2,5.26E2,1.4E1,2.3E2,5.8E1,1.73E2,2.12E2,3.14E2,7E0,7E0,1.5E2,8E1,4.5E1,1.3E1,1.58E2,1.5E1,1.65E2,4.7E1,2.82E2,3.2E1,6.6E1,8.4E1,5.2E1,2.8E1,3.9E1,6E0,4E0,9E0,2.2E1,1.36E2,5E0,1E1,1.42E2,2.3E1,3.4E1,1.3E1,1E2,1.82E2,5E0,2.7E1,4E0,6.2E1,5.4E1,3E1,4.6E1,6E0,2.2E1,6E0,5E0,3.4E1,6E0,1.6E1,2.1E1,1.15E2,5E0,5E0,1.1E2,3.2E1,1.9E1,4E0,1.9E1,1.5E1,4E0,9E0,9.6E1,4E0,1.74E2,8E0,2.1E1,6E0,2.4E1,3.8E1,2.3E1,3.1E1,8E0,2.2E1,3.9E1,7E0,1.4E1,8E0,9E0,2.5E1,1.3E1,8E0,6.6E1,4.9E1,2.6E1,8.4E1,1.8E1,1.4E1,1.2E1,7E0,1.2E1,7E0,1.1E1,4E0,5E0,4E0,6.4E1,3.2E1,1.15E2,5.9E1,4E0,4E0,1.7E1,4E0,1.3E1,1.1E1,8E0,3E1,1.9E1,4E0,2.7E1,4E0,2.9E1,1E1,1E1,4E0,5E0,4E0,6E0,7E0,4E0,4E0,4.5E1,2.1E1,4.1E1,8E0,2.2E1,4E0,4E1,4.4E1,1.2E1,6E0,4E0,1E1,6E0,6E0,7E0,4E0,6E1,4E0,2.8E1,4E0,8.7E1,2.8E1,3E1,2.9E1,1.3E1,4E0,8E0,5E0,6E0,5E0,7E0,2.3E1,9E0,1E1,2.2E1,5E0,6E0,4E0,2.4E1,2.1E1,1.4E1,7E0,2.8E1,1.3E1,4E0,4E0,9E0,1.3E1,2.5E1,1.5E1,3.1E1,1.3E1,4E0,8E0,4.1E1,1.9E1,4E0,2.4E1,3E1,5.7E1,1.8E1,1E1,2.6E1,4E0,2.5E1,4E0,1.8E1,5E0,5E0,1.9E1,9E0,5E0,1.3E1,1.5E1,5E0,8E0,9E0,4E0,7E0,1.8E1,9E0,6E0,1.5E1,1.6E1,9E0,4E0,9E0,3.2E1,9E0,1E1,7E0,1.7E1,1.9E1,1.1E1,2.5E1,3.2E1,4E0,1.4E1,6E0,4E0,6E0,2E1,2.1E1,4E0,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[-3.9656907E-3,-9.501214E-2,2.7065217E-1,-2.784495E-1,-9.365859E-3,-3.6793208E-1,3.017125E-1,-4.4126916E-1,-2.2017272E-1,-1.7643312E-1,2.1908537E-2,-2.1849465E-2,-8.505184E-3,1.5604289E-1,4.175075E-1,-3.507889E-1,-7.0138806E-1,-2.3586765E-1,9.097859E-3,-1.4819905E-1,-2.6608398E-2,3.0838398E-2,-2.1082714E-2,2.8791332E-1,1.0092565E-1,5.0469875E-1,2.5951615E-1,-2.0058792E-2,-2.6352626E-1,-1.7305136E-2,-4.054622E-2,2.1098584E-3,-2.636969E-1,-1.8473676E-1,-5.438283E-2,-5.381478E-2,6.6572115E-2,1.5014201E-1,1.7512841E-2,1.2969783E-1,-7.551079E-3,5.437801E-1,1.9590639E-1,2.812694E-2,1.86803E-1,-1.603637E-1,-2.1972109E-2,-2.7934167E-1,2.3219657E-1,-3.302879E-1,-1.7906928E-1,-1.1146918E-1,-2.4207743E-1,1.619824E-2,-1.09921984E-1,-1.9888683E-1,-3.408119E-2,2.1942863E-2,1.4924604E-1,1.7885772E-3,1.0573727E-2,2.3934389E-2,1.9963692E-1,4.638127E-1,6.910533E-1,1.28993485E-2,1.8524277E-3,1.1050021E-1,3.6835158E-1,-2.061758E-1,4.2250715E-4,-6.0287667E-3,-1.7200807E-2,1.6376795E-2,6.724597E-4,-4.2167246E-1,-2.4743077E-1,-2.2192113E-1,-6.0843613E-2,-9.97553E-3,-6.9336176E-2,-2.721887E-1,-1.7132155E-3,-3.5586585E-3,3.717867E-3,-6.844297E-3,-1.8772351E-3,-1.376032E-2,-3.4678967E-3,7.466982E-3,-4.6565972E-2,3.2273047E-2,-1.7515205E-2,2.2839847E-1,7.858436E-2,-3.4470063E-2,1.3834985E-1,1.7988099E-2,1.5056755E-1,1.0686118E-2,2.2362962E-2,3.714944E-2,2.0898255E-2,1.2367412E-2,4.418352E-2,1.1051776E-2,2.1141248E-2,-1.143979E-2,-5.7026832E-3,-3.279505E-1,-5.030666E-1,-2.6878613E-1,-6.6011306E-4,-1.2509137E-1,-2.6261213E-1,-9.1879135E-3,1.2518891E-2,1.7837885E-4,-4.059825E-3,-3.0420086E-1,-6.5529323E-3,-9.235642E-2,3.6423183E-3,1.7449673E-2,2.3503724E-1,2.5325418E-1,-1.8933563E-3,-2.1648366E-2,1.376548E-1,-4.706644E-3,9.66299E-3,1.0267954E-3,9.797057E-3,2.8059423E-2,1.9189501E-1,9.446828E-2,-3.8922029E-3,-1.7883098E-2,-5.340247E-3,-2.667475E-2,-1.5185724E-2,-2.9505622E-1,-2.6615907E-3,-7.375062E-3,-4.7378655E-4,-2.841051E-1,-3.5496268E-3,6.878219E-2,-4.735933E-3,-7.836087E-3,-1.5603953E-2,-3.6249865E-2,-1.4716834E-1,7.715605E-2,-4.8646215E-2,-1.1873774E-2,7.4849084E-2,1.8090488E-3,1.6288273E-2,1.6469273E-1,3.308046E-1,3.084686E-2,-1.0090223E-1,1.0215925E-2,5.1329482E-2,4.0874323E-3,-3.519478E-2,-4.97195E-3,8.558977E-3,1.2989512E-3,2.2955103E-1,-6.5646076E-4,1.4093477E-1,-1.5428957E-2,-8.987475E-3,-1.4099805E-2,-6.715339E-3,4.6868604E-3,7.3111086E-4,1.4541056E-3,-5.130876E-3,-7.5847246E-3,-1.6679307E-3,4.994398E-3,-2.1293836E-4,-6.8203514E-3,1.4341802E-3,-4.3626237E-3,7.148556E-4,4.195675E-3,-5.6436253E-3,1.6728593E-3,8.770242E-3,9.296893E-3,1.732066E-2,3.204331E-3,-2.4183947E-3,-6.6490406E-3,-1.7418164E-3,5.538608E-3,-2.6759E-3,6.589656E-4,-3.5857034E-3,1.4160664E-2,4.2908005E-3,3.2838914E-3,8.34704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,-1,35,-1,37,39,41,43,-1,45,-1,-1,47,49,51,53,55,57,59,-1,61,-1,63,65,-1,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,-1,-1,109,111,113,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,121,123,-1,125,127,129,131,-1,133,-1,-1,-1,-1,-1,135,-1,-1,-1,-1,137,139,141,-1,143,145,-1,147,-1,-1,149,-1,151,153,155,157,159,-1,161,163,-1,165,-1,-1,167,169,171,-1,-1,-1,-1,-1,173,-1,-1,-1,175,-1,177,-1,-1,-1,179,181,183,185,187,189,-1,-1,191,193,195,197,-1,199,-1,201,-1,-1,-1,203,-1,205,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5003695E1,1.1796147E1,5.049837E0,2.1706238E0,2.6842532E0,1.4458358E-1,3.9493904E0,1.21317E0,1.2116995E0,8.2832646E-1,1.8633032E0,0E0,0E0,7.486539E-1,1.6873856E0,2.3488188E-1,5.352783E-1,1.1396675E0,0E0,2.5389636E-1,0E0,1.2922211E0,0E0,3.3242273E-1,6.069044E-1,9.071808E-1,1.1308231E0,0E0,4.9255216E-1,0E0,0E0,1.295269E0,8.087139E-1,2.0408261E-1,9.194082E-2,3.5679632E-1,1.1036274E0,1.1395872E-1,0E0,5.130117E-1,0E0,5.509834E-1,1.315833E-1,0E0,5.4110885E-1,1.4943129E-1,0E0,7.596862E-2,2.8760093E-1,5.3927326E-1,3.4113073E-1,9.9455565E-2,1.7508519E-1,7.211452E-2,2.8412238E-2,1.614477E-1,2.836265E-1,8.2428366E-1,5.6932116E-1,0E0,0E0,1.9855772E-1,3.3237934E-1,1.2808514E-1,2.839775E-1,0E0,0E0,3.1134298E-1,2.8845072E-2,2.0869553E-2,0E0,0E0,0E0,0E0,0E0,1.3252687E-1,2.30165E-1,1.680944E-1,2.0501871E-1,0E0,2.748178E-2,6.534195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4633583E-1,5.7298344E-1,0E0,3.3846617E-1,3.4046504E-1,6.0482375E-2,8.088446E-2,0E0,1.7181867E-1,0E0,0E0,0E0,0E0,0E0,1.5649688E-1,0E0,0E0,0E0,0E0,2.1898055E-1,5.6430817E-2,2.2154713E-1,0E0,6.5802634E-2,1.1792421E-1,0E0,9.765879E-2,0E0,0E0,3.302896E-2,0E0,1.6688243E-1,2.0363745E-1,3.0385548E-1,2.8547662E-1,2.5457764E-1,0E0,9.464241E-2,2.4806172E-1,0E0,4.907278E-2,0E0,0E0,2.3323132E-1,1.5389436E-1,8.785039E-2,0E0,0E0,0E0,0E0,0E0,8.36575E-2,0E0,0E0,0E0,4.6357393E-2,0E0,1.5694037E-2,0E0,0E0,0E0,1.5042421E-1,4.6960175E-2,5.617225E-2,2.4910985E-1,2.7238593E-1,2.029925E-1,0E0,0E0,6.742704E-2,5.6003332E-2,4.8486833E-2,1.904314E-2,0E0,1.4925034E-1,0E0,1.998904E-2,0E0,0E0,0E0,1.924293E-1,0E0,2.046971E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,21,21,23,23,24,24,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,69,69,75,75,76,76,77,77,78,78,80,80,81,81,90,90,91,91,93,93,94,94,95,95,96,96,98,98,104,104,109,109,110,110,111,111,113,113,114,114,116,116,119,119,121,121,122,122,123,123,124,124,125,125,127,127,128,128,130,130,133,133,134,134,135,135,141,141,145,145,147,147,151,151,152,152,153,153,154,154,155,155,156,156,159,159,160,160,161,161,162,162,164,164,166,166,170,170,172,172],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,-1,36,-1,38,40,42,44,-1,46,-1,-1,48,50,52,54,56,58,60,-1,62,-1,64,66,-1,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,-1,-1,110,112,114,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,122,124,-1,126,128,130,132,-1,134,-1,-1,-1,-1,-1,136,-1,-1,-1,-1,138,140,142,-1,144,146,-1,148,-1,-1,150,-1,152,154,156,158,160,-1,162,164,-1,166,-1,-1,168,170,172,-1,-1,-1,-1,-1,174,-1,-1,-1,176,-1,178,-1,-1,-1,180,182,184,186,188,190,-1,-1,192,194,196,198,-1,200,-1,202,-1,-1,-1,204,-1,206,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.972052E5,1E0,3.3817584E7,4.97E2,9.3326636E-2,1.087521E3,2.4545455E0,2.8366232E0,1E0,4.6E1,-2.1849465E-2,-8.505184E-3,7E0,8E0,3.6E1,2.8914285E0,5.597832E-4,9.097859E-3,8.599521E5,-2.6608398E-2,6.374563E5,-2.1082714E-2,1.2774637E10,1.2118524E7,4.9036694E0,1.2083E4,-2.0058792E-2,3.7509E4,-1.7305136E-2,-4.054622E-2,4.1142857E1,5.100098E4,3.2E1,3.5364E4,1E0,3.9742767E2,6.4348956E2,1.7512841E-2,3.3193566E5,-7.551079E-3,1.9617874E3,2.0003653E0,2.812694E-2,9.0577506E-2,2.6412E5,-2.1972109E-2,6.737099E7,2E0,8.3925985E-2,7.9E2,8.3E1,7.026624E7,8.754E3,1E0,5.0200376E7,1.5E1,2.483E3,2.9375E0,1.7885772E-3,1.0573727E-2,2.3625128E6,2.08E3,1.9675E4,1.7493458E7,1.28993485E-2,1.8524277E-3,4.229649E6,4.907764E3,5.6E1,4.2250715E-4,-6.0287667E-3,-1.7200807E-2,1.6376795E-2,6.724597E-4,3.6E2,1.3091103E0,9.117022E0,1.7356666E3,-9.97553E-3,1.2E1,2.241E2,-1.7132155E-3,-3.5586585E-3,3.717867E-3,-6.844297E-3,-1.8772351E-3,-1.376032E-2,-3.4678967E-3,7.466982E-3,4.220534E5,7.4711205E6,-1.7515205E-2,3.0561172E5,1.574E3,1.5204346E8,5.130175E2,1.7988099E-2,4.032683E6,1.0686118E-2,2.2362962E-2,3.714944E-2,2.0898255E-2,1.2367412E-2,2.2807484E7,1.1051776E-2,2.1141248E-2,-1.143979E-2,-5.7026832E-3,4.1142857E1,1.9885094E-8,5.13657E6,-6.6011306E-4,1.4276666E2,7.997723E6,-9.1879135E-3,3.3594444E1,1.7837885E-4,-4.059825E-3,1.2608696E0,-6.5529323E-3,1E0,6.933837E-1,2.378914E6,1E0,1.0641107E8,-1.8933563E-3,5.5586E4,4.857143E0,-4.706644E-3,1.4195632E6,1.0267954E-3,9.797057E-3,2.0094643E0,2.7662E4,2.6776E4,-3.8922029E-3,-1.7883098E-2,-5.340247E-3,-2.667475E-2,-1.5185724E-2,1.0433326E8,-2.6615907E-3,-7.375062E-3,-4.7378655E-4,1.547E3,-3.5496268E-3,1.364E3,-4.735933E-3,-7.836087E-3,-1.5603953E-2,4.2609915E6,4.032683E6,1.5998265E6,9.558022E7,3.2883008E6,8.561815E7,1.8090488E-3,1.6288273E-2,4.371613E2,4.7185095E2,2.141096E1,4.7563504E7,1.0215925E-2,1.3736341E8,4.0874323E-3,2.9355192E1,-4.97195E-3,8.558977E-3,1.2989512E-3,4.205298E0,-6.5646076E-4,1.562752E3,-1.5428957E-2,-8.987475E-3,-1.4099805E-2,-6.715339E-3,4.6868604E-3,7.3111086E-4,1.4541056E-3,-5.130876E-3,-7.5847246E-3,-1.6679307E-3,4.994398E-3,-2.1293836E-4,-6.8203514E-3,1.4341802E-3,-4.3626237E-3,7.148556E-4,4.195675E-3,-5.6436253E-3,1.6728593E-3,8.770242E-3,9.296893E-3,1.732066E-2,3.204331E-3,-2.4183947E-3,-6.6490406E-3,-1.7418164E-3,5.538608E-3,-2.6759E-3,6.589656E-4,-3.5857034E-3,1.4160664E-2,4.2908005E-3,3.2838914E-3,8.34704E-3],"split_indices":[2,43,17,7,2,53,67,73,53,106,3,0,0,3,32,2,68,57,0,43,0,43,0,46,62,53,9,0,1,0,0,67,43,10,9,29,67,4,0,43,0,4,68,0,72,7,0,7,10,53,2,12,7,1,8,7,10,2,69,0,0,60,44,9,66,0,0,60,4,44,0,0,0,0,0,2,57,73,48,0,3,4,0,0,0,0,0,0,0,0,66,43,0,48,2,7,67,0,60,0,0,0,0,0,62,0,0,0,0,67,52,9,0,67,9,0,73,0,0,68,0,26,57,9,29,5,0,1,69,0,60,0,0,57,9,44,0,0,0,0,0,7,0,0,0,10,0,2,0,0,0,60,60,62,7,60,12,0,0,67,4,61,7,0,7,0,73,0,0,0,69,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,7.5E2,2.48E2,2.38E2,5.12E2,1.1E1,2.37E2,6.1E1,1.77E2,8E1,4.32E2,6E0,5E0,1.06E2,1.31E2,4.7E1,1.4E1,1.71E2,6E0,7.6E1,4E0,4.25E2,7E0,3E1,7.6E1,8.3E1,4.8E1,2.2E1,2.5E1,6E0,8E0,1.8E1,1.53E2,5.4E1,2.2E1,1.26E2,2.99E2,1.3E1,1.7E1,6.9E1,7E0,7.3E1,1E1,7E0,4.1E1,1.8E1,7E0,8E0,1E1,8.4E1,6.9E1,2.5E1,2.9E1,1E1,1.2E1,1.4E1,1.12E2,1.95E2,1.04E2,6E0,7E0,2.8E1,4.1E1,5E1,2.3E1,6E0,4E0,3E1,1.1E1,1.4E1,4E0,4E0,4E0,6E0,4E0,3.8E1,4.6E1,5E1,1.9E1,6E0,1.9E1,2.5E1,4E0,4E0,6E0,7E0,5E0,7E0,7E0,6E0,1.06E2,1.91E2,4E0,4.8E1,5.6E1,1.9E1,9E0,7E0,3.4E1,5E0,4.5E1,1.4E1,9E0,8E0,2.2E1,6E0,5E0,8E0,6E0,2E1,1.8E1,4.2E1,4E0,1.6E1,3.4E1,6E0,1.3E1,4E0,1.5E1,1.9E1,6E0,5.5E1,5.1E1,1.79E2,1.2E1,4.4E1,4E0,2.1E1,3.5E1,7E0,1.2E1,4E0,5E0,9E0,2.5E1,1.6E1,6E0,1.5E1,5E0,1.1E1,7E0,3.7E1,5E0,1.2E1,4E0,3E1,4E0,9E0,4E0,5E0,1.4E1,2.8E1,2.7E1,2.1E1,3E1,1.19E2,6E1,5E0,7E0,2.2E1,2.2E1,1.3E1,8E0,1.7E1,1.8E1,4E0,8E0,5E0,4E0,5E0,2E1,5E0,1.1E1,2.5E1,1.2E1,2.5E1,5E0,5E0,4E0,1.5E1,1.3E1,2.3E1,4E0,1.5E1,6E0,1.3E1,1.7E1,2.9E1,9E1,5.6E1,4E0,4E0,1.8E1,7E0,1.5E1,9E0,4E0,4E0,4E0,1.1E1,7E0,4E0,4E0,1.2E1,8E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"207","size_leaf_vector":"1"}},{"base_weights":[3.6584307E-3,-8.717372E-2,2.6933727E-1,-1.9764858E-1,1.747211E-2,3.1734458E-1,-1.3559663E-1,-3.3104765E-1,-1.5139028E-1,-4.751485E-2,6.8291664E-2,3.481294E-2,2.7624777E-1,-4.2068493E-1,1.372734E-2,-2.1810003E-1,-3.7600914E-1,-1.3114409E-1,-4.616602E-1,-8.735668E-2,7.129579E-2,4.550107E-2,2.2154155E-1,1.398515E-1,3.7319E-1,-2.5613237E-2,-8.134605E-3,1.4498642E-1,-1.6281003E-1,-1.39393695E-2,-1.4558814E-1,-4.174836E-1,-2.3760873E-1,-2.134523E-1,-5.4366227E-2,-3.4546994E-2,-3.1609958E-1,-1.8754998E-1,-2.8570468E-2,1.05273254E-1,-7.7200104E-3,5.653491E-2,-1.9249128E-1,1.4698611E-1,1.6100287E-2,1.1064811E-1,2.9183999E-2,3.960429E-1,6.3921496E-2,1.2552225E-2,-4.0077963E-3,-1.1661417E-2,-1.8775862E-3,-1.7426285E-1,-1.6715435E-3,-2.9512233E-1,-5.237878E-1,-1.0580909E-1,-1.5359684E-2,-2.5049242E-1,-1.1815903E-1,2.0022953E-2,-7.944116E-2,-5.7165525E-3,-1.7918082E-2,-1.4690425E-1,-3.51875E-1,-5.6904454E-2,7.284781E-2,8.705399E-2,9.951149E-3,-3.8742732E-2,8.637103E-2,-1.6535815E-2,5.2856794E-4,6.1813336E-2,1.1282641E-2,-8.473943E-2,1.3368845E-1,2.6736724E-1,4.7632608E-1,1.1844745E-2,-4.948767E-3,-3.588991E-3,-1.1999852E-2,-1.8307015E-1,-1.8160485E-2,-5.6335413E-1,-1.131592E-2,-4.64755E-4,-8.334148E-3,-1.8143399E-1,-3.07392E-1,-9.175192E-3,-1.6222003E-1,-3.5944086E-2,-2.6765063E-1,-1.9269049E-1,-3.3465356E-2,-2.6381325E-2,-2.879998E-3,-8.2045004E-2,2.8358858E-2,-5.3661563E-3,8.364929E-3,1.1922198E-1,4.408848E-2,7.3120794E-3,-6.1274584E-2,1.4336166E-1,4.445573E-2,-2.9693093E-4,5.1500276E-3,-6.913837E-3,4.6684203E-4,5.759979E-2,1.7818967E-1,3.1480128E-1,5.5637907E-2,5.3607506E-1,3.2572708E-1,-1.31378705E-2,-1.9006245E-3,-2.814211E-2,-1.3204679E-2,-2.0752932E-1,-6.4006355E-4,-3.2659554E-1,-5.3283176E-3,-3.0754057E-3,2.740203E-3,-5.1902775E-3,-9.2632305E-3,-7.218088E-2,3.7500963E-2,-3.2925078E-1,-2.8742468E-3,-1.2656488E-1,-2.4597009E-1,1.3971746E-2,-5.1816422E-3,-1.8467259E-2,-1.2120399E-1,-1.27165755E-2,4.570811E-3,1.8031854E-3,-2.599539E-3,2.1877536E-3,1.3422905E-1,5.5799777E-5,4.85894E-3,-3.0685423E-2,-1.10736955E-2,1.6237536E-1,-1.2248257E-3,-5.7272278E-2,1.0098505E-1,2.0620754E-2,9.620597E-3,2.0337267E-1,-4.1178144E-3,3.6593717E-1,6.0018757E-3,-9.654969E-3,1.2329838E-2,5.579276E-1,1.0524922E-2,7.307919E-3,3.6347908E-1,-1.02076165E-1,-2.4671112E-1,-3.789711E-1,-2.112526E-1,1.8101072E-2,-1.036983E-1,1.4309485E-1,-6.9076917E-3,-1.6559511E-2,-8.123538E-3,-7.3194234E-3,-1.4387994E-3,-1.3026931E-2,-5.4315035E-3,-1.9422622E-3,3.1041286E-3,-5.6895424E-2,3.0730215E-3,-1.4101261E-1,-2.5369686E-3,1.8590864E-3,-2.9361583E-3,4.1070725E-3,8.30705E-3,-5.9224144E-2,4.2153742E-2,1.2688813E-1,2.28272E-1,2.3988217E-2,-1.8472953E-2,1.5314375E-2,7.130514E-2,5.5312112E-2,-3.171402E-2,1.4755993E-1,2.9879907E-1,9.34993E-3,1.8799089E-2,2.1985715E-2,3.28682E-2,8.785764E-3,1.919603E-2,-9.236898E-3,-1.669248E-4,-1.3185396E-2,-7.040399E-3,-9.734367E-3,-1.9539097E-2,-1.2630345E-2,-2.8701501E-3,-4.281962E-3,2.3834284E-3,3.5186842E-4,-6.084418E-3,2.3185445E-3,8.732369E-3,4.232548E-3,-2.45839E-3,2.5006486E-3,-4.720996E-3,-1.9990192E-3,-7.3178234E-3,-9.091291E-4,-6.9615114E-3,4.0126187E-3,-1.3332478E-3,7.975032E-3,2.7829132E-3,1.187323E-2,4.720372E-3,-1.2999049E-3,5.8013718E-3,2.0875838E-3,8.559218E-3,4.879857E-3,6.726875E-4,-3.108075E-3,4.2195295E-4,8.844434E-3,2.327079E-3,1.599773E-2,7.4821156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,51,-1,53,55,57,59,61,-1,63,65,67,69,-1,71,73,75,-1,77,-1,79,81,-1,-1,-1,-1,83,-1,85,87,89,-1,91,93,-1,95,-1,-1,97,99,101,103,105,-1,107,109,-1,-1,111,-1,113,115,117,119,-1,-1,-1,-1,121,-1,123,-1,-1,-1,125,127,129,131,133,135,137,139,-1,-1,141,143,145,-1,147,149,-1,151,153,155,-1,-1,-1,-1,157,159,161,163,165,167,-1,-1,-1,-1,169,-1,171,-1,-1,-1,-1,-1,173,175,177,-1,179,181,183,-1,185,187,189,-1,-1,-1,-1,191,-1,-1,193,-1,195,-1,197,199,201,-1,203,-1,205,-1,-1,-1,207,-1,-1,209,211,213,215,217,219,221,223,225,-1,-1,-1,-1,-1,-1,-1,-1,227,-1,229,-1,-1,-1,-1,-1,231,233,235,237,239,-1,-1,241,243,245,247,249,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5289871E1,9.032387E0,5.2334538E0,2.2947922E0,1.3308935E0,4.076231E0,1.24032E0,4.1058636E-1,1.7234836E0,8.440813E-1,7.792599E-1,0E0,2.8567734E0,2.2948349E-1,4.8704785E-1,1.3798487E-1,2.911272E-1,1.6820073E0,4.7165155E-1,7.7800965E-1,3.821002E-1,5.308244E-1,2.2326982E-1,1.2659254E0,8.7934685E-1,0E0,0E0,3.7593743E-1,8.826588E-2,0E0,5.016467E-2,5.088024E-1,1.8986428E-1,4.2573023E-1,1.7467504E0,0E0,1.3142407E-1,2.7558446E-1,2.484E-1,6.1503947E-2,0E0,5.434621E-1,3.0773142E-1,1.5053433E-1,0E0,4.119885E-1,0E0,1.1019726E0,3.5665295E-1,0E0,0E0,0E0,0E0,9.539741E-2,0E0,2.1345782E-1,1.4887285E-1,6.2460452E-2,0E0,3.0573225E-1,1.8098527E-1,0E0,1.0873787E0,0E0,0E0,2.0650327E-1,5.3549767E-1,1.4594564E-1,1.6178617E-1,4.4499785E-2,0E0,2.1389848E-1,3.399999E-1,0E0,0E0,4.138354E-2,0E0,6.307938E-2,2.6061928E-1,4.6455765E-1,4.7325325E-1,0E0,0E0,0E0,0E0,1.895966E-1,0E0,1.0550308E-1,0E0,0E0,0E0,1.8953872E-1,1.4305449E-1,5.11176E-2,2.9031038E-2,2.9546228E-1,2.9588842E-1,7.1040034E-2,5.009608E-2,0E0,0E0,1.2722299E-1,4.7642812E-2,2.9314958E-2,0E0,1.5213877E-2,4.440259E-2,0E0,2.2186998E-1,2.0133007E-1,4.9458593E-1,0E0,0E0,0E0,0E0,1.6839492E-1,3.5472524E-1,3.21182E-1,6.147865E-1,2.0970535E-1,8.7980986E-2,0E0,0E0,0E0,0E0,1.3182676E-1,0E0,1.8575239E-1,0E0,0E0,0E0,0E0,0E0,2.1244073E-1,1.7449191E-1,2.205038E-2,0E0,4.067844E-2,3.6194026E-2,2.9653106E-2,0E0,7.236607E-2,3.4276187E-2,3.2172166E-2,0E0,0E0,0E0,0E0,1.7073601E-2,0E0,0E0,7.940978E-2,0E0,9.965205E-2,0E0,8.7258524E-1,3.5263324E-1,4.895884E-2,0E0,2.0090568E-1,0E0,1.18709564E-1,0E0,0E0,0E0,2.4076462E-1,0E0,0E0,8.3284855E-2,1.1490561E-1,5.504179E-2,1.2540674E-1,1.3127655E-1,7.898476E-2,1.7243844E-1,3.744553E-2,1.2805308E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.095936E-2,0E0,3.3928603E-2,0E0,0E0,0E0,0E0,0E0,9.469057E-2,3.7424363E-2,1.05436265E-1,3.8950086E-2,1.4120728E-1,0E0,0E0,1.4045513E-1,3.1268362E-2,1.6593087E-2,1.22567415E-1,4.7875762E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,47,47,48,48,53,53,55,55,56,56,57,57,59,59,60,60,62,62,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,77,77,78,78,79,79,80,80,85,85,87,87,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,105,105,106,106,108,108,109,109,110,110,115,115,116,116,117,117,118,118,119,119,120,120,125,125,127,127,133,133,134,134,135,135,137,137,138,138,139,139,141,141,142,142,143,143,148,148,151,151,153,153,155,155,156,156,157,157,159,159,161,161,165,165,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,185,185,187,187,193,193,194,194,195,195,196,196,197,197,200,200,201,201,202,202,203,203,204,204],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,52,-1,54,56,58,60,62,-1,64,66,68,70,-1,72,74,76,-1,78,-1,80,82,-1,-1,-1,-1,84,-1,86,88,90,-1,92,94,-1,96,-1,-1,98,100,102,104,106,-1,108,110,-1,-1,112,-1,114,116,118,120,-1,-1,-1,-1,122,-1,124,-1,-1,-1,126,128,130,132,134,136,138,140,-1,-1,142,144,146,-1,148,150,-1,152,154,156,-1,-1,-1,-1,158,160,162,164,166,168,-1,-1,-1,-1,170,-1,172,-1,-1,-1,-1,-1,174,176,178,-1,180,182,184,-1,186,188,190,-1,-1,-1,-1,192,-1,-1,194,-1,196,-1,198,200,202,-1,204,-1,206,-1,-1,-1,208,-1,-1,210,212,214,216,218,220,222,224,226,-1,-1,-1,-1,-1,-1,-1,-1,228,-1,230,-1,-1,-1,-1,-1,232,234,236,238,240,-1,-1,242,244,246,248,250,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,4.4761734E5,1E0,1.16E2,8.39E2,3.6E1,1.0201538E-1,1E0,3E1,3.3486558E6,3.7951445E6,3.481294E-2,1.3026532E6,7E0,1.1E1,4.2E1,9.558022E7,6.07E2,1E0,1E0,7.89E2,1.7500046E8,3.962617E0,1.0534078E4,1.8545505E2,-2.5613237E-2,-8.134605E-3,4.433E3,2.8E1,-1.39393695E-2,1.5832983E-8,6.0921145E4,7.080933E2,1.852364E6,2.00087E5,-3.4546994E-2,2.1095E4,2.43E2,5.751617E6,6.7723384E7,-7.7200104E-3,4.7563504E7,1.7694E4,3.7788504E7,1.6100287E-2,6.4258E4,2.9183999E-2,1.1293673E3,2.4344263E0,1.2552225E-2,-4.0077963E-3,-1.1661417E-2,-1.8775862E-3,6.101E3,-1.6715435E-3,5E0,1.318849E6,1.7E1,-1.5359684E-2,1E0,1.3809524E0,2.0022953E-2,2.0734E4,-5.7165525E-3,-1.7918082E-2,1.1362371E6,3.304531E3,6.887749E7,4.9E1,1.9668E4,9.951149E-3,3.7974394E-6,4.9390244E0,-1.6535815E-2,5.2856794E-4,3E0,1.1282641E-2,2.0939393E0,8.072743E4,7.7191065E6,1.853776E7,1.1844745E-2,-4.948767E-3,-3.588991E-3,-1.1999852E-2,3.6E1,-1.8160485E-2,7.7E1,-1.131592E-2,-4.64755E-4,-8.334148E-3,3.0416667E0,5.57E2,7.1428573E-1,3.478747E4,1.158E3,1.609E3,9.836066E-1,2.141096E1,-2.6381325E-2,-2.879998E-3,3.6E1,4.1226246E4,1.3529412E0,8.364929E-3,4.33E2,1.3935602E1,7.3120794E-3,1.764442E7,1E0,4.326193E2,-2.9693093E-4,5.1500276E-3,-6.913837E-3,4.6684203E-4,2.4693803E2,4.1325716E7,2E0,1E0,2.53E2,7.15244E5,-1.31378705E-2,-1.9006245E-3,-2.814211E-2,-1.3204679E-2,3.8575E4,-6.4006355E-4,1.7909248E5,-5.3283176E-3,-3.0754057E-3,2.740203E-3,-5.1902775E-3,-9.2632305E-3,1.3867E4,1.3520833E1,1.1939979E8,-2.8742468E-3,1.7171982E6,1.605076E6,1.512E4,-5.1816422E-3,2.6530768E2,8.1407714E-1,1E0,4.570811E-3,1.8031854E-3,-2.599539E-3,2.1877536E-3,2.0518206E3,5.5799777E-5,4.85894E-3,4.699871E2,-1.10736955E-2,1.7642576E8,-1.2248257E-3,1.18033766E5,1.684E0,4.5179688E1,9.620597E-3,1.767635E3,-4.1178144E-3,4.6134964E2,6.0018757E-3,-9.654969E-3,1.2329838E-2,1.6975454E3,1.0524922E-2,7.307919E-3,8E0,3.103E3,1.86619E5,4.0869565E0,4.58E2,1.5152774E2,2.978142E0,1.04E2,2.3266666E0,-1.6559511E-2,-8.123538E-3,-7.3194234E-3,-1.4387994E-3,-1.3026931E-2,-5.4315035E-3,-1.9422622E-3,3.1041286E-3,1.7676923E2,3.0730215E-3,1E1,-2.5369686E-3,1.8590864E-3,-2.9361583E-3,4.1070725E-3,8.30705E-3,1.2E1,2.75E0,4.05E2,1.57E2,1E0,-1.8472953E-2,1.5314375E-2,2.445E3,5.117647E0,2.3266666E0,3.776793E8,5.501002E9,9.34993E-3,1.8799089E-2,2.1985715E-2,3.28682E-2,8.785764E-3,1.919603E-2,-9.236898E-3,-1.669248E-4,-1.3185396E-2,-7.040399E-3,-9.734367E-3,-1.9539097E-2,-1.2630345E-2,-2.8701501E-3,-4.281962E-3,2.3834284E-3,3.5186842E-4,-6.084418E-3,2.3185445E-3,8.732369E-3,4.232548E-3,-2.45839E-3,2.5006486E-3,-4.720996E-3,-1.9990192E-3,-7.3178234E-3,-9.091291E-4,-6.9615114E-3,4.0126187E-3,-1.3332478E-3,7.975032E-3,2.7829132E-3,1.187323E-2,4.720372E-3,-1.2999049E-3,5.8013718E-3,2.0875838E-3,8.559218E-3,4.879857E-3,6.726875E-4,-3.108075E-3,4.2195295E-4,8.844434E-3,2.327079E-3,1.599773E-2,7.4821156E-3],"split_indices":[2,43,6,44,2,0,53,24,3,43,43,0,43,6,8,2,7,2,13,29,2,12,71,4,73,0,0,10,3,0,52,60,4,9,5,0,1,0,60,7,0,7,10,7,0,1,0,4,69,0,0,0,0,9,0,12,9,3,0,8,68,0,44,0,0,43,48,7,0,12,0,53,71,0,0,8,0,68,48,62,62,0,0,0,0,2,0,44,0,0,0,71,10,71,62,2,2,71,61,0,0,10,48,68,0,2,73,0,60,19,70,0,0,0,0,71,66,32,8,8,1,0,0,0,0,9,0,43,0,0,0,0,0,1,73,7,0,60,12,9,0,67,53,26,0,0,0,0,48,0,0,67,0,7,0,48,69,71,0,67,0,4,0,0,0,67,0,0,3,1,12,73,2,67,69,10,68,0,0,0,0,0,0,0,0,67,0,3,0,0,0,0,0,8,68,10,0,23,0,0,2,69,68,12,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.046E3,7.8E2,2.66E2,3.79E2,4.01E2,2.38E2,2.8E1,9.6E1,2.83E2,1.76E2,2.25E2,1.9E1,2.19E2,9E0,1.9E1,2.9E1,6.7E1,2.67E2,1.6E1,1.32E2,4.4E1,1.97E2,2.8E1,9.2E1,1.27E2,5E0,4E0,1.1E1,8E0,1.2E1,1.7E1,5E1,1.7E1,1.28E2,1.39E2,4E0,1.2E1,4.8E1,8.4E1,3.9E1,5E0,1.89E2,8E0,1.9E1,9E0,8.8E1,4E0,1.18E2,9E0,7E0,4E0,4E0,4E0,1.3E1,4E0,2.5E1,2.5E1,8E0,9E0,9.1E1,3.7E1,6E0,1.33E2,4E0,8E0,4E1,8E0,6.6E1,1.8E1,3.5E1,4E0,4.5E1,1.44E2,4E0,4E0,1.1E1,8E0,9E0,7.9E1,4.7E1,7.1E1,4E0,5E0,7E0,6E0,1.3E1,1.2E1,2.1E1,4E0,4E0,4E0,4.3E1,4.8E1,1.1E1,2.6E1,1.09E2,2.4E1,2.8E1,1.2E1,4E0,4E0,5.1E1,1.5E1,1.1E1,7E0,1.9E1,1.6E1,4E0,4.1E1,6E1,8.4E1,5E0,6E0,5E0,4E0,3E1,4.9E1,3.8E1,9E0,4.9E1,2.2E1,7E0,6E0,1.7E1,4E0,3.7E1,6E0,4.3E1,5E0,6E0,5E0,1.3E1,1.3E1,7.3E1,3.6E1,1.8E1,6E0,1.4E1,1.4E1,8E0,4E0,2E1,3.1E1,1E1,5E0,6E0,5E0,4E0,1.5E1,1E1,6E0,3.6E1,5E0,5.4E1,6E0,3E1,5.4E1,2.5E1,5E0,4.5E1,4E0,2.9E1,9E0,4E0,5E0,4.5E1,4E0,5E0,1.7E1,1.1E1,2.6E1,2.8E1,1.5E1,1.9E1,5.4E1,1E1,2.6E1,1.4E1,4E0,1E1,4E0,1E1,4E0,4E0,4E0,1.4E1,6E0,2.3E1,8E0,5E0,5E0,9E0,6E0,2.6E1,1E1,3.7E1,1.7E1,2.5E1,5E0,5E0,4.9E1,1.5E1,1E1,3E1,1.5E1,7E0,2.2E1,3.2E1,1.3E1,5E0,1.2E1,5E0,6E0,1.7E1,9E0,7E0,2.1E1,1E1,5E0,4E0,1.5E1,1.1E1,4.3E1,4E0,6E0,8E0,1.8E1,4E0,1E1,4E0,1.9E1,1.9E1,7E0,6E0,4E0,2.1E1,1.6E1,1.3E1,4E0,1.7E1,8E0,4.1E1,8E0,6E0,9E0,5E0,5E0,2E1,1E1,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"251","size_leaf_vector":"1"}},{"base_weights":[-2.99391E-3,-8.831429E-2,2.4912943E-1,-2.6406372E-1,-1.5137249E-2,6.127956E-1,2.1382841E-1,-3.5943136E-1,-2.1115749E-1,-9.3182884E-2,4.57518E-2,3.1338148E-2,1.6913528E-2,-2.1640071E-1,2.4496906E-1,-2.95355E-1,-4.3520728E-1,-4.5271154E-2,-2.502184E-1,-1.7284678E-1,-4.950818E-2,-7.196651E-2,8.058637E-2,-1.0896794E-3,-1.2654556E-2,1.4805791E-1,3.2795694E-1,-4.214891E-3,-3.1222135E-1,-5.516985E-1,-2.7941078E-1,-2.5595522E-1,7.2868645E-2,-2.76405E-1,1.2859181E-2,1.576933E-2,-2.2300647E-1,-1.21104665E-1,3.9490927E-3,-3.0619962E-2,-2.7614754E-1,6.265724E-2,2.4083194E-1,9.1958314E-2,2.658024E-1,4.0736428E-1,2.054631E-1,-1.0115258E-2,-1.6603213E-2,-1.9671071E-2,-3.2649584E-2,-3.5516988E-3,-1.9103112E-2,-1.4849484E-2,-5.225477E-3,1.8012885E-2,-3.0757098E-2,-2.4766478E-1,-4.3157107E-1,1.0021293E-2,-5.3721853E-3,-6.6432714E-2,3.7694715E-3,-1.8739763E-1,-2.1558696E-2,-1.5851676E-1,4.5616136E-4,-1.1833065E-2,9.114149E-3,8.394522E-3,-1.2428759E-1,-6.4939214E-3,-1.507793E-2,1.2589181E-2,1.1591291E-1,3.2262672E-3,3.0713496E-1,-1.8852325E-2,1.2636074E-1,3.2381943E-1,-6.2581357E-3,3.4137793E-2,3.666432E-1,2.4937008E-1,-4.614155E-3,-8.232715E-2,2.1855973E-3,-2.0621872E-1,-3.7024492E-1,-8.3336495E-3,-5.002674E-1,-5.276657E-3,-2.4777374E-4,-1.8875582E-2,-1.6470881E-1,-1.4460988E-2,-1.3912079E-1,-4.388345E-2,5.3428244E-3,1.0040181E-2,-1.6188158E-1,2.9081505E-2,-5.888991E-3,-1.8223038E-1,1.0149796E-3,2.704253E-2,-1.4224796E-2,1.3864143E-1,-2.0600121E-2,9.547849E-3,1.7254243E-2,3.7968703E-2,2.1366796E-1,3.6912245E-1,5.6471373E-3,1.1089947E-1,3.9944065E-1,1.7773433E-1,2.3478601E-2,-1.4606287E-4,-7.4315965E-3,-2.5133523E-1,-8.815539E-2,-8.5149985E-3,-4.116537E-1,-3.2499995E-2,-1.4355317E-2,-8.8783694E-5,-1.8382254E-1,-4.267908E-4,-1.574555E-1,-3.9355992E-3,1.8129069E-3,-5.0451376E-2,7.8654446E-2,-1.37755E-2,2.8323957E-3,-1.898887E-2,8.068132E-2,-3.8668283E-3,-1.0012353E-2,4.0165115E-2,-9.568277E-3,2.094816E-1,5.4692373E-2,4.9777194E-3,-8.8733844E-2,9.831434E-2,-9.091543E-2,2.5220698E-1,-3.1418859E-3,2.0643573E-2,2.5416845E-1,-2.5801815E-3,1.1802392E-2,3.1811407E-1,4.6263516E-1,7.929093E-2,1.1666393E-2,-6.084328E-3,-2.6380596E-1,1.1817312E-3,-1.1485079E-1,-2.1868676E-2,-1.1414294E-2,-2.099745E-1,-6.963592E-2,-1.1265667E-1,-1.863606E-1,-1.1348648E-1,2.8509617E-2,1.24131896E-1,6.7105573E-3,4.246519E-2,-4.8807436E-3,1.2765835E-1,6.033308E-6,3.4689752E-3,1.201243E-1,1.6168655E-1,1.4664508E-2,7.527092E-2,-5.394145E-3,-6.124588E-4,-6.9031436E-3,5.178071E-2,1.7528737E-1,-7.75603E-4,-8.839531E-3,4.5169266E-3,2.8536078E-1,1.4764176E-2,7.474719E-3,8.617172E-3,1.5907837E-2,1.2407149E-2,2.2620369E-2,-9.172108E-3,7.343055E-3,-1.341973E-2,-8.324688E-3,-6.6140885E-3,-5.8214075E-4,-5.578171E-3,-1.2115863E-2,-6.068335E-4,-5.2912077E-3,-6.5090233E-3,-2.5052743E-4,-5.3579533E-3,-9.900675E-3,-2.3328223E-4,-6.28536E-3,-6.032966E-4,2.768627E-3,2.5625895E-3,8.164715E-3,-1.3723923E-3,3.949327E-3,-2.6384208E-3,5.1668244E-3,3.1676488E-3,8.099799E-3,9.294547E-4,-1.048663E-2,4.782991E-4,7.733759E-3,1.066699E-2,4.6780817E-3,5.370828E-3,-7.2246086E-4,3.9856415E-3,-4.5347452E-4,1.1775024E-2,2.8015731E-3,9.146807E-3,1.6794136E-2,-2.6498754E-3,1.8871386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,87,89,-1,-1,91,-1,93,-1,95,97,99,-1,101,103,-1,-1,105,107,-1,109,-1,111,113,-1,-1,115,117,-1,119,-1,121,123,-1,125,-1,-1,-1,127,-1,129,131,-1,133,135,137,-1,139,-1,141,-1,143,145,-1,-1,147,149,151,-1,153,155,157,-1,-1,-1,159,161,-1,163,-1,-1,-1,165,-1,167,-1,-1,169,171,-1,-1,173,175,-1,-1,177,-1,179,181,-1,183,185,187,189,-1,-1,191,-1,-1,193,195,197,-1,-1,199,-1,201,-1,-1,203,205,207,209,211,213,215,217,219,-1,221,-1,223,225,227,-1,229,-1,-1,-1,231,233,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,237,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2630335E1,1.0104609E1,3.2838192E0,1.0896244E0,2.6464446E0,1.2349701E-1,3.322029E0,2.6475143E-1,9.67103E-1,8.4039116E-1,1.2893082E0,0E0,0E0,1.8205786E-1,1.7851229E0,1.3332558E-1,4.8106194E-1,7.653563E-1,8.440256E-1,8.171563E-1,6.090274E-1,5.993813E-1,6.8019164E-1,0E0,0E0,6.823368E-1,1.1029844E0,0E0,1.1046934E-1,6.993055E-2,4.298731E-1,6.6516876E-2,6.690387E-1,3.8812065E-1,3.4145224E-1,1.0822474E-1,5.025053E-1,3.0932593E-1,2.830429E-1,2.2396141E-1,3.583449E-2,5.824842E-1,2.5065565E-1,1.3245192E0,8.3538723E-1,7.146969E-1,6.9523096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.8308115E-2,4.1137314E-1,1.858468E-1,0E0,0E0,2.5650207E-2,0E0,2.3881245E-1,0E0,1.1227834E-1,9.1903515E-2,2.8388146E-1,0E0,1.2737457E-1,1.5654552E-1,0E0,0E0,5.3639454E-1,3.3174634E-1,0E0,3.6121607E-2,0E0,5.3209555E-1,2.288518E-1,0E0,0E0,5.176811E-1,7.2551584E-1,0E0,6.1662704E-2,0E0,3.7201214E-1,1.0121131E-1,0E0,2.4360752E-1,0E0,0E0,0E0,1.770004E-1,0E0,1.12850785E-1,4.79773E-2,0E0,3.194126E-1,3.5289887E-1,9.826852E-2,0E0,2.604106E-2,0E0,3.4631383E-1,0E0,5.297166E-1,1.494748E-1,0E0,0E0,2.8893167E-1,3.970946E-1,1.0084152E-1,0E0,2.3122764E-1,1.5605354E-1,2.4503076E-1,0E0,0E0,0E0,4.4331312E-2,6.973393E-2,0E0,6.584978E-2,0E0,0E0,0E0,1.3767469E-1,0E0,3.3391714E-2,0E0,0E0,2.0730536E-1,1.1695267E-1,0E0,0E0,1.16320334E-1,7.1879975E-2,0E0,0E0,3.0767363E-1,0E0,2.0146489E-1,1.6050184E-1,0E0,4.8405014E-2,8.0469385E-2,8.611426E-2,1.3082421E-1,0E0,0E0,2.0800054E-2,0E0,0E0,3.5827875E-2,3.0908108E-2,1.2124563E-1,0E0,0E0,5.2469492E-2,0E0,5.0308466E-2,0E0,0E0,1.5959918E-1,2.6398048E-2,5.6761548E-2,2.235037E-2,5.5586487E-2,2.5596896E-2,6.96502E-2,4.5569938E-2,9.6188895E-2,0E0,2.1779373E-2,0E0,2.8466728E-1,1.6952386E-1,1.271683E-1,0E0,1.471216E-1,0E0,0E0,0E0,3.6962986E-2,6.666455E-2,0E0,0E0,0E0,9.8748684E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.4035225E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,56,56,57,57,58,58,61,61,63,63,65,65,66,66,67,67,69,69,70,70,73,73,74,74,76,76,78,78,79,79,82,82,83,83,85,85,87,87,88,88,90,90,94,94,96,96,97,97,99,99,100,100,101,101,103,103,105,105,107,107,108,108,111,111,112,112,113,113,115,115,116,116,117,117,121,121,122,122,124,124,128,128,130,130,133,133,134,134,137,137,138,138,141,141,143,143,144,144,146,146,147,147,148,148,149,149,152,152,155,155,156,156,157,157,160,160,162,162,165,165,166,166,167,167,168,168,169,169,170,170,171,171,172,172,173,173,175,175,177,177,178,178,179,179,181,181,185,185,186,186,190,190,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,88,90,-1,-1,92,-1,94,-1,96,98,100,-1,102,104,-1,-1,106,108,-1,110,-1,112,114,-1,-1,116,118,-1,120,-1,122,124,-1,126,-1,-1,-1,128,-1,130,132,-1,134,136,138,-1,140,-1,142,-1,144,146,-1,-1,148,150,152,-1,154,156,158,-1,-1,-1,160,162,-1,164,-1,-1,-1,166,-1,168,-1,-1,170,172,-1,-1,174,176,-1,-1,178,-1,180,182,-1,184,186,188,190,-1,-1,192,-1,-1,194,196,198,-1,-1,200,-1,202,-1,-1,204,206,208,210,212,214,216,218,220,-1,222,-1,224,226,228,-1,230,-1,-1,-1,232,234,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,238,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,5.6E1,7.7E1,8.1E2,6.7321223E-1,9.183432E4,1.139605E6,9.935484E-1,1E0,4.539777E5,3.1338148E-2,1.6913528E-2,3.63E2,5.97E3,5E0,1E0,3.06E2,4.2210345E2,5.714286E-1,6.072304E5,1.9E1,5.405423E6,-1.0896794E-3,-1.2654556E-2,2.8530578E6,8.992806E-4,-4.214891E-3,1E0,2.3060504E7,7.042942E7,5.4E1,2.00087E5,4.6463413E0,9.566785E5,5.44E2,3.7568388E0,4.39776E6,1.9E1,2E1,2.9E1,4.5432812E2,1E0,1.4598765E0,1E0,1.0918E4,3.3571344E7,-1.0115258E-2,-1.6603213E-2,-1.9671071E-2,-3.2649584E-2,-3.5516988E-3,-1.9103112E-2,-1.4849484E-2,-5.225477E-3,1.8012885E-2,9.14E2,8.499432E5,2.49E2,1.0021293E-2,-5.3721853E-3,1.3026532E6,3.7694715E-3,1E1,-2.1558696E-2,8.2765434E1,1.185E3,3.6507E4,9.114149E-3,2.3719013E0,1.445E3,-6.4939214E-3,-1.507793E-2,2.41E3,1E0,3.2262672E-3,2.47E2,-1.8852325E-2,5.723202E5,1.6864318E11,-6.2581357E-3,3.4137793E-2,1.0322473E8,2.9622424E3,-4.614155E-3,6.2478176E7,2.1855973E-3,1.19111115E2,2E0,-8.3336495E-3,7.3E1,-5.276657E-3,-2.4777374E-4,-1.8875582E-2,1.6666666E0,-1.4460988E-2,4.1719616E5,6.07E2,5.3428244E-3,1.2463E4,3.6977E4,2.6369516E5,-5.888991E-3,3.074398E0,1.0149796E-3,1.81E2,-1.4224796E-2,1.9545455E0,1.3358E4,9.547849E-3,1.7254243E-2,4.4522205E6,9.4163445E1,4.026E3,5.6471373E-3,1.034073E6,3.7951445E6,2.234478E3,2.3478601E-2,-1.4606287E-4,-7.4315965E-3,6.48334E5,1.9166666E0,-8.5149985E-3,3.3025316E5,-3.2499995E-2,-1.4355317E-2,-8.8783694E-5,2.866353E2,-4.267908E-4,9.966666E0,-3.9355992E-3,1.8129069E-3,2.3112903E2,4.9E4,-1.37755E-2,2.8323957E-3,1.281479E5,2.3625128E6,-3.8668283E-3,-1.0012353E-2,9.750085E6,-9.568277E-3,1.11791E5,1.481649E8,4.9777194E-3,2E0,8.620714E2,1.1E1,2.8608696E1,-3.1418859E-3,2.0643573E-2,9.6732026E-1,-2.5801815E-3,1.1802392E-2,1.1074919E0,1.3586957E1,1.5E1,1.1666393E-2,-6.084328E-3,2.026453E6,1.1817312E-3,3.221198E0,-2.1868676E-2,-1.1414294E-2,1.3556826E6,1.9184E4,7.7E1,1.625E0,5.4E0,2.4E1,5.75E2,3.5834332E0,1.1497368E-2,-4.8807436E-3,3.9123364E2,6.033308E-6,9.575663E6,1.088E3,1.9197379E9,1.4664508E-2,1E0,-5.394145E-3,-6.124588E-4,-6.9031436E-3,3.0594551E1,1.7172844E7,-7.75603E-4,-8.839531E-3,4.5169266E-3,1.3748E4,1.4764176E-2,7.474719E-3,8.617172E-3,1.5907837E-2,1.2407149E-2,2.2620369E-2,5.32E2,7.343055E-3,-1.341973E-2,-8.324688E-3,-6.6140885E-3,-5.8214075E-4,-5.578171E-3,-1.2115863E-2,-6.068335E-4,-5.2912077E-3,-6.5090233E-3,-2.5052743E-4,-5.3579533E-3,-9.900675E-3,-2.3328223E-4,-6.28536E-3,-6.032966E-4,2.768627E-3,2.5625895E-3,8.164715E-3,-1.3723923E-3,3.949327E-3,-2.6384208E-3,5.1668244E-3,3.1676488E-3,8.099799E-3,9.294547E-4,-1.048663E-2,4.782991E-4,7.733759E-3,1.066699E-2,4.6780817E-3,5.370828E-3,-7.2246086E-4,3.9856415E-3,-4.5347452E-4,1.1775024E-2,2.8015731E-3,9.146807E-3,1.6794136E-2,-2.6498754E-3,1.8871386E-3],"split_indices":[2,43,0,44,2,53,43,46,68,29,43,0,0,0,2,0,26,2,67,68,43,3,43,0,0,43,72,0,24,7,60,2,5,68,60,12,68,9,3,8,0,4,29,69,6,9,66,0,0,0,0,0,0,0,0,0,2,60,2,0,0,43,0,3,0,4,44,1,0,53,2,0,0,2,6,0,0,0,43,46,0,0,5,67,0,7,0,67,8,0,0,0,0,0,69,0,60,2,0,1,12,66,0,71,0,8,0,68,9,0,0,62,73,2,0,9,43,4,0,0,0,46,68,0,62,0,0,0,67,0,73,0,0,67,12,0,0,62,60,0,0,9,0,1,12,0,8,4,3,73,0,0,68,0,0,68,73,3,0,0,9,0,68,0,0,60,1,0,68,73,0,2,69,53,0,4,0,9,44,46,0,30,0,0,0,73,60,0,0,0,44,0,0,0,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E3,7.85E2,2.65E2,2.3E2,5.55E2,2.2E1,2.43E2,8E1,1.5E2,2.43E2,3.12E2,1.6E1,6E0,1.6E1,2.27E2,4.6E1,3.4E1,2.9E1,1.21E2,8.5E1,1.58E2,7.1E1,2.41E2,4E0,1.2E1,1.06E2,1.21E2,4E0,4.2E1,1.8E1,1.6E1,1E1,1.9E1,1.1E2,1.1E1,1.8E1,6.7E1,6.7E1,9.1E1,6E1,1.1E1,2.18E2,2.3E1,7.3E1,3.3E1,7.2E1,4.9E1,1.6E1,2.6E1,1.2E1,6E0,7E0,9E0,6E0,4E0,4E0,1.5E1,9.5E1,1.5E1,4E0,7E0,8E0,1E1,6E1,7E0,5.1E1,1.6E1,8.5E1,6E0,4.3E1,1.7E1,4E0,7E0,1.13E2,1.05E2,7E0,1.6E1,4E0,6.9E1,2.9E1,4E0,6E0,6.6E1,4.3E1,6E0,9E0,6E0,7.3E1,2.2E1,4E0,1.1E1,4E0,4E0,4E0,5.6E1,4E0,4.7E1,1.2E1,4E0,7.5E1,1E1,3.8E1,5E0,1.2E1,5E0,1.09E2,4E0,9E1,1.5E1,8E0,8E0,3.5E1,3.4E1,2.3E1,6E0,8E0,5.8E1,3.5E1,8E0,5E0,4E0,5.2E1,2.1E1,5E0,1.7E1,4E0,7E0,6E0,5E1,6E0,4.1E1,8E0,4E0,4E1,3.5E1,6E0,4E0,2E1,1.8E1,4E0,8E0,1.04E2,5E0,4.8E1,4.2E1,5E0,1E1,2.4E1,1.1E1,3E1,4E0,1.2E1,1.1E1,4E0,4E0,2.8E1,3E1,1.6E1,1.9E1,6E0,4.6E1,4E0,1.7E1,1.1E1,6E0,4E1,1E1,1.8E1,2.3E1,2.2E1,1.8E1,2.1E1,1.4E1,1.2E1,8E0,1.1E1,7E0,7.2E1,3.2E1,3.5E1,1.3E1,3.8E1,4E0,5E0,5E0,1.6E1,8E0,7E0,4E0,6E0,2.4E1,5E0,6E0,6E0,2.2E1,5E0,2.5E1,8E0,8E0,3.3E1,1.3E1,1.3E1,4E0,1.6E1,2.4E1,5E0,5E0,1.4E1,4E0,8E0,1.5E1,4E0,1.8E1,8E0,1E1,1E1,1.1E1,1E1,4E0,5E0,7E0,6E0,5E0,6.8E1,4E0,1E1,2.2E1,1.5E1,2E1,2.6E1,1.2E1,1E1,6E0,4E0,4E0,1.3E1,1.1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"239","size_leaf_vector":"1"}},{"base_weights":[8.385376E-3,-8.538872E-2,2.0839147E-1,-2.301312E-1,-1.4239848E-2,-3.2906845E-1,2.3747429E-1,-3.666555E-1,-1.7427713E-1,-7.8797385E-2,6.739256E-2,-3.815999E-1,-5.852169E-3,1.3782695E-1,3.538121E-1,-2.868886E-1,-4.448873E-1,-1.8602254E-2,-1.5831576E-1,-6.749313E-2,-2.9546818E-2,4.5568712E-2,2.4685593E-1,-2.1202961E-2,-1.15376385E-2,2.5929764E-1,8.518831E-2,7.612947E-1,3.0647954E-1,-3.0748403E-1,-4.703944E-3,-2.2583077E-2,-1.0368227E-2,-3.2264754E-2,-2.0078294E-1,-1.0796738E-1,3.296368E-2,5.540572E-4,9.257787E-2,3.8242887E-3,2.7871877E-1,1.3354308E-3,2.7870327E-1,6.6096395E-2,1.9891214E-2,2.4160905E-2,4.2729717E-2,3.4492496E-1,3.5163887E-2,-1.5100135E-2,-6.533194E-3,-1.4553766E-1,2.7664334E-2,-2.3006064E-1,-9.171461E-2,-1.8874885E-1,-7.331491E-2,-2.0556806E-1,6.499129E-2,-4.755107E-2,5.7392977E-2,1.0381096E-1,-5.5044745E-3,1.6947904E-1,1.705334E-2,2.934425E-1,4.5807837E-3,8.473108E-2,-1.0542178E-2,3.176181E-1,2.441365E-2,-9.526243E-3,1.2529123E-1,-4.9157947E-4,-1.9843249E-1,1.08595295E-2,-1.8624386E-2,-1.8568096E-1,-3.2673618E-1,3.4390914E-3,-1.4528327E-1,-1.2332242E-1,-2.2360787E-1,-2.8357996E-2,-1.4377968E-1,-5.0849398E-3,-1.2009749E-2,-4.880094E-2,8.863702E-2,-1.558001E-2,-1.5282294E-1,-2.546018E-2,9.690646E-2,1.2702686E-1,-2.4658425E-2,3.6861175E-3,9.977854E-3,2.113888E-1,1.599295E-2,-6.0455836E-2,1.0545327E-1,4.6006906E-1,2.690929E-1,2.036041E-1,-1.8816956E-3,-4.8382836E-3,-1.1247085E-2,-6.8514557E-3,2.563649E-2,-2.2101967E-1,-9.688479E-2,-2.7479225E-1,-2.13782E-2,-1.68762E-1,-1.8985339E-3,-1.1184934E-3,-1.7114916E-1,-1.5763903E-1,-2.7348912E-1,-4.3729156E-2,7.7849017E-3,-9.073622E-2,-2.73696E-1,-4.9986714E-3,1.3226691E-3,6.71694E-2,9.53378E-3,-8.465152E-2,2.3370089E-2,-9.30596E-3,-2.0551227E-3,-6.669334E-2,2.1946914E-3,1.2639599E-1,-3.5028288E-3,1.0315654E-1,1.0591707E-2,-8.797705E-2,4.939164E-3,1.6794829E-2,1.4658184E-1,8.65815E-2,-1.5820373E-2,1.8924217E-1,4.5217186E-2,1.019785E-2,2.3627738E-2,2.0017476E-1,3.3111122E-1,1.4480535E-2,2.450834E-3,7.2311927E-3,-2.2939708E-2,-1.0854046E-1,-2.4612446E-1,-9.645308E-6,-1.2104075E-1,-4.1258195E-3,-3.24094E-1,-1.0412486E-2,-4.6051564E-3,-1.02068875E-2,-4.148843E-3,-2.1493214E-3,-1.8824986E-1,-1.4288767E-2,-5.4772226E-3,-1.3185384E-2,-1.3305894E-1,-1.4188997E-1,-1.4728785E-2,-2.001718E-2,-1.665338E-1,9.54931E-2,2.9355342E-2,-6.8681487E-3,-4.646384E-4,-4.583179E-3,5.57374E-2,-3.7209212E-4,-5.7158275E-3,4.124596E-2,1.7703688E-1,1.1909425E-1,-1.7003579E-3,-7.177065E-4,-6.1760047E-3,1.02022635E-2,8.7373115E-2,-1.6423877E-3,8.035295E-3,2.1612059E-1,2.490472E-4,7.457233E-3,9.858267E-3,2.7113795E-1,1.2643786E-1,2.2591102E-1,4.1052514E-1,-4.1065407E-3,5.6956057E-4,-2.134921E-3,-7.4670273E-3,-1.2298635E-2,-7.2204466E-3,-6.5089376E-3,-2.2728448E-3,-1.7132578E-2,-9.439976E-3,-1.0521625E-2,-4.81459E-3,-2.1480166E-3,1.7590827E-3,-6.1685685E-5,-1.1617987E-2,-8.336323E-3,-3.0561646E-3,3.0857178E-3,-2.6728727E-3,-3.6108214E-3,-9.81382E-3,5.9812306E-4,5.0489143E-3,-4.0784515E-3,3.1945214E-3,1.0648882E-3,5.0846254E-3,5.369802E-3,-8.4117777E-4,2.8404922E-3,9.7217085E-3,6.401373E-3,-5.817069E-4,6.1227125E-3,7.2585413E-4,4.5768563E-3,1.3541119E-2,-2.2696569E-3,2.8209276E-3,1.4939151E-2,4.808647E-3,8.884772E-3,1.6050225E-3,1.2984555E-2,6.1633885E-3,2.1035686E-2,8.135332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,35,-1,37,39,-1,-1,41,43,45,47,49,-1,-1,-1,51,53,55,57,59,61,-1,63,-1,65,67,-1,-1,-1,69,71,-1,-1,73,75,77,79,81,83,85,87,89,91,93,-1,95,-1,97,-1,99,-1,101,-1,-1,103,-1,105,-1,107,109,111,-1,113,115,117,119,121,-1,-1,123,125,127,129,131,133,135,137,-1,-1,139,-1,141,143,145,147,149,-1,-1,-1,-1,151,153,155,157,-1,159,-1,-1,161,163,165,167,-1,169,171,-1,-1,173,-1,175,177,-1,-1,179,-1,181,-1,183,-1,185,-1,-1,187,189,-1,191,193,-1,-1,195,197,-1,-1,-1,199,201,203,-1,205,-1,207,-1,-1,-1,-1,-1,209,-1,-1,211,213,215,217,-1,219,221,223,-1,-1,-1,225,-1,-1,227,229,231,-1,-1,-1,-1,233,-1,-1,235,-1,-1,237,239,241,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8979494E1,7.0922275E0,5.126911E0,1.6670742E0,2.4453144E0,1.324079E-1,3.51021E0,2.7798653E-1,5.3042316E-1,1.5986202E0,7.9165006E-1,1.7974257E-2,0E0,1.0457177E0,2.4861736E0,1.0209775E-1,1.905551E-1,0E0,8.1788826E-1,1.0403669E0,0E0,3.8931125E-1,9.19435E-2,0E0,0E0,2.1548891E-1,7.495988E-1,6.2181473E-2,1.32302E0,5.347538E-2,0E0,0E0,0E0,2.755599E-1,3.4612656E-1,4.956305E-1,5.7869065E-1,2.624822E-1,2.264179E-1,0E0,1.1521256E-1,0E0,9.263992E-2,6.3901997E-1,0E0,0E0,0E0,3.9342785E-1,3.956443E-1,0E0,0E0,9.7783685E-2,2.635795E-1,3.254695E-1,2.4223363E-1,9.558189E-2,4.0457904E-1,1.8135667E-2,1.8191183E-1,1.72633E-1,1.4651519E-1,2.5963002E-1,0E0,2.4470925E-2,0E0,1.0855198E-1,0E0,3.2829434E-1,0E0,5.6744385E-1,0E0,0E0,1.7731379E-1,0E0,1.6686827E-2,0E0,1.3581477E-1,1.7941499E-1,6.358719E-2,0E0,4.1989148E-2,9.540084E-2,6.8768024E-2,2.4945322E-1,3.1972635E-1,0E0,0E0,5.8288127E-2,1.2514013E-1,1.12282336E-1,5.4234266E-2,4.8297122E-2,1.6239232E-1,1.5449905E-1,1.2603638E-1,0E0,0E0,1.340515E-1,0E0,6.101183E-1,4.658718E-1,1.8214226E-1,2.6283216E-1,1.3219628E-1,0E0,0E0,0E0,0E0,1.1803363E-1,1.0035491E-1,4.6680763E-2,1.72243E-1,0E0,4.0116966E-2,0E0,0E0,3.6854923E-2,4.9065024E-2,6.316447E-2,2.0288068E-1,0E0,1.4176857E-1,1.6726506E-1,0E0,0E0,4.860638E-2,0E0,7.012955E-2,1.1264998E-1,0E0,0E0,3.244455E-2,0E0,1.0481188E-1,0E0,1.4045393E-1,0E0,3.1033158E-2,0E0,0E0,4.6649694E-2,1.12860635E-1,0E0,1.9155884E-1,2.2849253E-1,0E0,0E0,1.6978943E-1,2.33078E-1,0E0,0E0,0E0,3.3658046E-2,2.5619522E-2,2.3350716E-2,0E0,1.5884325E-2,0E0,1.9402266E-2,0E0,0E0,0E0,0E0,0E0,1.9413322E-2,0E0,0E0,9.909747E-2,2.9603454E-1,5.230266E-2,6.011582E-2,0E0,2.3953825E-2,2.932635E-2,1.09083965E-1,0E0,0E0,0E0,3.652709E-2,0E0,0E0,5.1156364E-2,4.8802495E-2,1.4273459E-1,0E0,0E0,0E0,0E0,3.0277222E-2,0E0,0E0,2.7733946E-1,0E0,0E0,1.358888E-1,1.3211071E-1,1.20173424E-1,5.94247E-2,1.4038372E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,33,33,34,34,35,35,36,36,37,37,38,38,40,40,42,42,43,43,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,65,65,67,67,69,69,72,72,74,74,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,97,97,99,99,100,100,101,101,102,102,103,103,108,108,109,109,110,110,111,111,113,113,116,116,117,117,118,118,119,119,121,121,122,122,125,125,127,127,128,128,131,131,133,133,135,135,137,137,140,140,141,141,143,143,144,144,147,147,148,148,152,152,153,153,154,154,156,156,158,158,164,164,167,167,168,168,169,169,170,170,172,172,173,173,174,174,178,178,181,181,182,182,183,183,188,188,191,191,194,194,195,195,196,196,197,197,198,198],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,36,-1,38,40,-1,-1,42,44,46,48,50,-1,-1,-1,52,54,56,58,60,62,-1,64,-1,66,68,-1,-1,-1,70,72,-1,-1,74,76,78,80,82,84,86,88,90,92,94,-1,96,-1,98,-1,100,-1,102,-1,-1,104,-1,106,-1,108,110,112,-1,114,116,118,120,122,-1,-1,124,126,128,130,132,134,136,138,-1,-1,140,-1,142,144,146,148,150,-1,-1,-1,-1,152,154,156,158,-1,160,-1,-1,162,164,166,168,-1,170,172,-1,-1,174,-1,176,178,-1,-1,180,-1,182,-1,184,-1,186,-1,-1,188,190,-1,192,194,-1,-1,196,198,-1,-1,-1,200,202,204,-1,206,-1,208,-1,-1,-1,-1,-1,210,-1,-1,212,214,216,218,-1,220,222,224,-1,-1,-1,226,-1,-1,228,230,232,-1,-1,-1,-1,234,-1,-1,236,-1,-1,238,240,242,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.194E3,2.0976269E5,1E0,7.7E1,9.31E2,1.02006085E-1,1.087521E3,1.139605E6,3.07E2,5.433518E0,3.7951445E6,2E1,-5.852169E-3,8E0,2.670351E6,2.0376764E-7,7.7983914E4,-1.8602254E-2,1.4339622E0,2.0766992E6,-2.9546818E-2,2.437372E6,5.2692295E6,-2.1202961E-2,-1.15376385E-2,1.6869704E6,3.3778846E0,2.480065E6,2.3876712E7,3.1530054E0,-4.703944E-3,-2.2583077E-2,-1.0368227E-2,1.20267E5,8.57E2,4.33E2,1.1127184E-5,4.4970587E2,1E0,3.8242887E-3,7.82261E6,1.3354308E-3,2.8827406E5,1.0428572E1,1.9891214E-2,2.4160905E-2,4.2729717E-2,6.701152E0,2.9925186E-2,-1.5100135E-2,-6.533194E-3,5.445545E-1,1.4806053E4,7.485264E5,9.237895E0,1.6923077E0,4.066351E0,5E0,4.92E2,3E0,1E0,2.9789968E1,-5.5044745E-3,3.5364E4,1.705334E-2,5.691698E2,4.5807837E-3,1.4598765E0,-1.0542178E-2,1.0717949E1,2.441365E-2,-9.526243E-3,6.747343E8,-4.9157947E-4,1.0795E4,1.08595295E-2,3.06E2,1.7915686E2,1.3999298E5,3.4390914E-3,1.363176E7,2.8E1,1.7277137E4,1.97525E1,1.9197379E9,-5.0849398E-3,-1.2009749E-2,4.1973075E6,6.2521395E6,2.5084135E6,1.1362371E6,4.285714E0,3.7241504E0,7.955228E9,3.6966666E2,3.6861175E-3,9.977854E-3,1.2375429E8,1.599295E-2,1.9495455E6,4.159558E0,3.0238653E8,1.6466942E3,6.072304E5,-1.8816956E-3,-4.8382836E-3,-1.1247085E-2,-6.8514557E-3,9.384615E0,1.00019E5,2.11429E5,1.26E2,-2.13782E-2,6.829817E4,-1.8985339E-3,-1.1184934E-3,6.5998E4,2.6845297E5,1.7907799E2,2.3695534E9,7.7849017E-3,1.345672E6,3.4402E4,-4.9986714E-3,1.3226691E-3,1.5016073E4,9.53378E-3,5.3871745E-1,1E0,-9.30596E-3,-2.0551227E-3,1.01E2,2.1946914E-3,1.98E2,-3.5028288E-3,3.8435754E8,1.0591707E-2,2.4545455E0,4.939164E-3,1.6794829E-2,4.1290735E2,9.8933E4,-1.5820373E-2,2.4911E4,3.3382E4,1.019785E-2,2.3627738E-2,4.7837E4,1.5416006E5,1.4480535E-2,2.450834E-3,7.2311927E-3,1.8711711E2,8.657E2,1.02166664E2,-9.645308E-6,6.07E2,-4.1258195E-3,2.0262831E2,-1.0412486E-2,-4.6051564E-3,-1.02068875E-2,-4.148843E-3,-2.1493214E-3,9.917037E2,-1.4288767E-2,-5.4772226E-3,6.83E2,1.5043378E0,5.2083335E0,7.8E1,-2.001718E-2,8E0,1.09333336E2,1.3478261E1,-6.8681487E-3,-4.646384E-4,-4.583179E-3,4.4091418E4,-3.7209212E-4,-5.7158275E-3,2.28E4,4.6991256E7,1E0,-1.7003579E-3,-7.177065E-4,-6.1760047E-3,1.02022635E-2,3.1993368E6,-1.6423877E-3,8.035295E-3,5.6440186E2,2.490472E-4,7.457233E-3,7.1E1,1E0,2.72E0,3.09627E5,8.847283E8,-4.1065407E-3,5.6956057E-4,-2.134921E-3,-7.4670273E-3,-1.2298635E-2,-7.2204466E-3,-6.5089376E-3,-2.2728448E-3,-1.7132578E-2,-9.439976E-3,-1.0521625E-2,-4.81459E-3,-2.1480166E-3,1.7590827E-3,-6.1685685E-5,-1.1617987E-2,-8.336323E-3,-3.0561646E-3,3.0857178E-3,-2.6728727E-3,-3.6108214E-3,-9.81382E-3,5.9812306E-4,5.0489143E-3,-4.0784515E-3,3.1945214E-3,1.0648882E-3,5.0846254E-3,5.369802E-3,-8.4117777E-4,2.8404922E-3,9.7217085E-3,6.401373E-3,-5.817069E-4,6.1227125E-3,7.2585413E-4,4.5768563E-3,1.3541119E-2,-2.2696569E-3,2.8209276E-3,1.4939151E-2,4.808647E-3,8.884772E-3,1.6050225E-3,1.2984555E-2,6.1633885E-3,2.1035686E-2,8.135332E-3],"split_indices":[2,43,17,44,2,57,67,46,1,68,43,6,0,3,66,52,48,0,68,43,0,9,60,0,0,60,57,1,60,68,0,0,0,9,2,2,57,67,79,0,43,0,48,68,0,0,0,50,72,0,0,68,43,60,71,68,69,8,2,68,29,73,0,9,0,67,0,69,0,71,0,0,47,0,9,0,2,4,43,0,9,10,66,73,46,0,0,43,43,60,43,69,68,46,67,0,0,7,0,43,69,7,67,43,0,0,0,0,73,5,9,10,0,43,0,0,12,43,67,46,0,9,9,0,0,62,0,57,29,0,0,0,0,10,0,7,0,68,0,0,70,1,0,10,9,0,0,9,48,0,0,0,4,48,67,0,2,0,67,0,0,0,0,0,48,0,0,2,69,69,10,0,8,4,73,0,0,0,48,0,0,1,7,19,0,0,0,0,43,0,0,4,0,0,8,6,69,9,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,6.88E2,3.22E2,2.26E2,4.62E2,1.6E1,3.06E2,6.4E1,1.62E2,2.58E2,2.04E2,1.2E1,4E0,1.66E2,1.4E2,3.4E1,3E1,9E0,1.53E2,2.54E2,4E0,1.83E2,2.1E1,6E0,6E0,4.9E1,1.17E2,1.3E1,1.27E2,3E1,4E0,2.4E1,6E0,3.9E1,1.14E2,1.81E2,7.3E1,9.4E1,8.9E1,4E0,1.7E1,4E0,4.5E1,1.12E2,5E0,7E0,6E0,1.11E2,1.6E1,2.6E1,4E0,1.3E1,2.6E1,8.9E1,2.5E1,5.3E1,1.28E2,8E0,6.5E1,5.1E1,4.3E1,8.5E1,4E0,9E0,8E0,4.1E1,4E0,1.06E2,6E0,9.9E1,1.2E1,4E0,1.2E1,4E0,9E0,4E0,2.2E1,6.3E1,2.6E1,6E0,1.9E1,2E1,3.3E1,7.9E1,4.9E1,4E0,4E0,1.1E1,5.4E1,4E1,1.1E1,1.4E1,2.9E1,7.2E1,1.3E1,4E0,5E0,1.8E1,2.3E1,1.3E1,9.3E1,2.3E1,7.6E1,8E0,4E0,4E0,5E0,5E0,1.7E1,4.4E1,1.9E1,2.1E1,5E0,1.5E1,4E0,7E0,1.3E1,1.6E1,1.7E1,7.4E1,5E0,3.6E1,1.3E1,6E0,5E0,4.7E1,7E0,1.4E1,2.6E1,7E0,4E0,9E0,5E0,2.5E1,4E0,6E1,1.2E1,9E0,4E0,4E0,1.4E1,9E0,4E0,3.8E1,5.5E1,5E0,1.8E1,3.8E1,3.8E1,4E0,4E0,4E0,1.3E1,9E0,3.5E1,4E0,1.5E1,5E0,1.6E1,7E0,8E0,7E0,6E0,4E0,1.2E1,1.3E1,4E0,5.6E1,1.8E1,2.1E1,1.5E1,4E0,9E0,2.6E1,2.1E1,7E0,7E0,5E0,2.1E1,5E0,4E0,1E1,1.5E1,5.4E1,6E0,4E0,5E0,5E0,9E0,4E0,5E0,3.3E1,5E0,1.2E1,4.3E1,1.8E1,2E1,1.8E1,2E1,4E0,9E0,5E0,4E0,2.7E1,8E0,1.1E1,4E0,1E1,6E0,7E0,5E0,3.4E1,2.2E1,9E0,9E0,1.3E1,8E0,5E0,1E1,4E0,5E0,4E0,2.2E1,5E0,1.6E1,1.4E1,7E0,4E0,6E0,4E0,1.1E1,4.7E1,7E0,5E0,4E0,1.4E1,1.9E1,2E1,2.3E1,1.3E1,5E0,1.1E1,9E0,1E1,8E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"247","size_leaf_vector":"1"}},{"base_weights":[4.4282814E-3,-7.089756E-2,2.4218602E-1,-2.0437197E-1,-8.261404E-3,2.8415385E-1,-8.598903E-2,-1.8150964E-1,-4.5935646E-1,-9.294011E-2,5.2365724E-2,1.0600522E-1,3.4112406E-1,-1.5499134E-1,9.440856E-3,-1.9990972E-1,1.657073E-2,-2.907893E-2,-1.5777435E-2,-2.0898332E-1,-5.925382E-2,2.0580392E-2,1.360249E-1,3.554532E-2,2.1839052E-1,4.0324628E-1,1.7871171E-1,-1.398593E-2,-2.9483128E-1,-3.4354174E-1,-1.600544E-1,-1.3231076E-1,-2.5994238E-1,-8.98192E-2,2.8414747E-2,7.378349E-2,-6.5065715E-3,8.728703E-2,2.7384022E-1,9.920785E-2,-1.16205495E-2,4.738623E-3,2.4817392E-1,4.8966122E-1,3.1450874E-1,2.5352243E-1,-4.2203434E-2,-6.1995494E-3,4.9995813E-3,-1.6558258E-2,-7.9238815E-3,-4.6093622E-1,-2.9160646E-1,2.0438117E-1,-1.8067156E-1,-7.6184296E-3,-1.7975752E-3,-1.8328145E-1,-1.4750147E-2,-1.8464176E-1,-5.0108705E-2,9.0700425E-2,-3.2459646E-2,9.518787E-2,-3.891413E-2,-1.58949E-2,7.5983666E-3,-6.7246966E-2,1.1015578E-1,1.1786991E-1,1.8202102E-2,1.3118407E-1,1.3573053E-3,-4.3895687E-3,5.8842584E-2,5.5208523E-3,1.2986845E-2,3.9819297E-1,5.9044045E-1,4.2621452E-1,2.0997998E-1,1.5474588E-1,2.6790693E-2,-6.1836056E-3,2.5626253E-3,-1.3171322E-2,-2.4725582E-2,-7.5487415E-3,-3.0701473E-1,7.7513396E-5,1.8791415E-2,-2.0266195E-1,-3.2269064E-2,-1.1578427E-2,-5.7540867E-3,-7.8102365E-2,-3.1165498E-1,-6.324879E-2,5.5606193E-3,-2.0883323E-3,1.3096264E-1,2.9990165E-3,-7.7523096E-3,1.1098491E-1,-2.3818745E-3,-7.921826E-3,4.5830626E-3,-7.313378E-2,3.0233769E-2,4.5355887E-4,-6.0457014E-3,-5.4546865E-3,1.3486981E-1,-7.790265E-5,8.612825E-3,8.616152E-3,2.9208383E-3,-1.06385596E-4,4.1441983E-3,2.0027258E-2,1.0591589E-2,1.7179692E-2,2.9448232E-2,4.8606926E-1,1.9264043E-3,1.5037008E-1,1.3138838E-2,-4.5302436E-3,9.1990195E-3,-1.5037169E-2,-8.121841E-3,-1.8360811E-1,-3.5809505E-1,2.2791203E-2,-1.2045073E-1,-1.07284226E-1,1.9668234E-3,-1.685911E-3,-4.2326507E-1,-4.998092E-2,-9.597459E-3,6.3006915E-2,9.610854E-3,-3.9417487E-2,2.7359598E-3,1.3972326E-1,5.6301165E-2,-3.9379608E-2,-1.8409701E-2,7.110269E-2,-8.992044E-2,4.881099E-3,-7.3879417E-3,9.77548E-2,1.2968812E-2,1.2038146E-2,2.3867035E-2,8.8894E-3,-1.0467523E-3,-1.5443225E-1,-2.4996966E-1,-2.0567698E-2,-1.04516465E-2,-1.8106676E-3,3.784915E-3,-7.8015993E-3,-2.2148767E-3,-1.0186115E-3,-1.4806482E-1,-3.0158894E-2,-7.6714233E-3,-6.956638E-2,2.2432987E-2,7.837078E-5,4.518229E-3,-3.0988848E-3,5.8769487E-4,7.572765E-2,1.9353291E-1,-1.0267993E-3,1.1372247E-1,-6.88002E-2,6.220237E-2,4.0727045E-2,1.3610534E-1,-7.429002E-3,-2.3357257E-2,6.493661E-2,1.459348E-1,-7.599928E-3,2.7608543E-3,-1.2748771E-2,-2.8213009E-3,-9.0355445E-3,-3.98174E-3,-2.4131336E-3,-9.389699E-3,-3.1999135E-3,3.4211902E-3,-9.884589E-4,5.171651E-3,1.1303496E-2,5.980526E-3,2.4330695E-3,-2.166809E-3,2.1575398E-3,6.714982E-3,6.1021145E-3,-4.183727E-3,7.989451E-3,2.8440674E-5,4.2089876E-3,-1.0054508E-3,-6.8957923E-4,8.829163E-3,-5.13425E-3,2.5509289E-3,4.0809037E-3,-1.1029723E-3,8.457262E-3,2.759687E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,83,-1,-1,-1,-1,85,87,89,91,-1,-1,93,-1,95,97,99,101,103,105,107,-1,109,111,113,-1,115,-1,-1,117,-1,-1,119,121,123,125,127,-1,-1,-1,-1,-1,-1,129,-1,-1,131,133,-1,-1,135,137,139,-1,-1,141,143,-1,145,-1,-1,-1,147,149,-1,-1,151,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,155,-1,157,-1,-1,-1,-1,-1,159,161,163,165,167,-1,-1,169,171,-1,173,-1,175,-1,177,179,181,-1,183,185,-1,-1,187,-1,-1,-1,-1,-1,189,191,-1,-1,-1,-1,-1,-1,-1,193,-1,-1,195,197,-1,-1,-1,-1,199,201,203,205,207,209,211,213,-1,215,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8535387E1,6.5695257E0,3.4538612E0,1.379837E0,2.756598E0,2.2068024E0,6.126642E-1,2.3776798E0,1.8588877E-1,8.632474E-1,8.2950985E-1,4.277634E-1,1.6128674E0,4.6911293E-1,0E0,1.2330275E0,0E0,0E0,0E0,1.6096497E-1,4.7148627E-1,3.2905322E-1,5.537443E-1,1.0624728E-1,4.698515E-2,7.6618767E-1,8.039868E-1,2.0548818E-1,3.261125E-2,1.5823984E-1,1.374351E0,6.05582E-2,7.338679E-2,4.8025644E-1,1.7735785E-1,1.8944547E-1,2.4660249E-1,2.3711762E-1,3.6584604E-1,2.9638574E-2,1.2899071E-1,0E0,3.4207225E-2,3.0875874E-1,6.3444424E-1,1.070837E0,1.2378834E-1,0E0,0E0,0E0,0E0,1.7166376E-2,2.095294E-2,4.0952381E-1,5.449796E-1,0E0,0E0,2.8109133E-2,0E0,4.8987174E-1,2.1593298E-1,1.317706E-1,1.1568097E-1,1.5640038E-1,2.550196E-1,3.850623E-1,0E0,4.5406885E-2,1.635961E-1,9.047711E-2,0E0,2.5802672E-2,0E0,0E0,2.2850301E-2,0E0,0E0,1.0340643E-1,2.9957771E-2,6.47171E-1,1.0805762E-1,3.4482598E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7915487E-2,0E0,0E0,3.6960554E-1,1.1441724E-1,0E0,0E0,8.2040966E-2,5.066397E-1,1.5640551E-1,0E0,0E0,8.116335E-2,5.0029255E-2,0E0,8.485317E-2,0E0,0E0,0E0,6.9813865E-1,4.0415004E-1,0E0,0E0,2.0567635E-1,2.2697365E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.7575245E-2,0E0,1.5713403E-1,0E0,0E0,0E0,0E0,0E0,2.1890545E-1,7.156801E-2,5.819227E-2,2.350095E-2,5.9302866E-2,0E0,0E0,6.0008526E-1,1.1643958E-1,0E0,2.329855E-2,0E0,1.815864E-2,0E0,1.1620033E-1,7.25374E-2,1.8359569E-1,0E0,1.14901185E-1,9.532726E-2,0E0,0E0,5.218953E-2,0E0,0E0,0E0,0E0,0E0,2.1477342E-1,1.7457223E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.842454E-2,0E0,0E0,1.3767433E-1,9.033047E-2,0E0,0E0,0E0,0E0,6.922447E-2,3.386545E-2,3.202723E-2,1.6528234E-2,2.1739471E-1,9.536918E-2,1.3735428E-1,1.606979E-1,0E0,8.8898525E-2,5.4055437E-2,3.706959E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,51,51,52,52,53,53,54,54,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,71,71,74,74,77,77,78,78,79,79,80,80,81,81,88,88,91,91,92,92,95,95,96,96,97,97,100,100,101,101,103,103,107,107,108,108,111,111,112,112,123,123,125,125,131,131,132,132,133,133,134,134,135,135,138,138,139,139,141,141,143,143,145,145,146,146,147,147,149,149,150,150,153,153,159,159,160,160,168,168,171,171,172,172,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,84,-1,-1,-1,-1,86,88,90,92,-1,-1,94,-1,96,98,100,102,104,106,108,-1,110,112,114,-1,116,-1,-1,118,-1,-1,120,122,124,126,128,-1,-1,-1,-1,-1,-1,130,-1,-1,132,134,-1,-1,136,138,140,-1,-1,142,144,-1,146,-1,-1,-1,148,150,-1,-1,152,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,156,-1,158,-1,-1,-1,-1,-1,160,162,164,166,168,-1,-1,170,172,-1,174,-1,176,-1,178,180,182,-1,184,186,-1,-1,188,-1,-1,-1,-1,-1,190,192,-1,-1,-1,-1,-1,-1,-1,194,-1,-1,196,198,-1,-1,-1,-1,200,202,204,206,208,210,212,214,-1,216,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.0976269E5,2E0,4.217427E7,7.9E2,7.601393E2,1.465E4,6.68411E2,1.0479E4,3.95E2,2.0766992E6,5.4051904E8,6.9E1,3.91157E5,9.440856E-3,4.9E2,1.657073E-2,-2.907893E-2,-1.5777435E-2,3.1E1,2.8045622E6,1.592E3,2.831224E10,7.6937294E-1,1.2003246E6,2.1298597E0,5.712652E8,5.071E3,7.650379E8,8.0833334E-1,4E0,1E0,1.7142857E0,1E0,3.5364E4,2.3719013E0,2.0240436E0,2.9077E4,1E0,1.6673345E7,2.89196E6,4.738623E-3,6.1236826E8,1.6975454E3,1.684E0,3.5675005E3,4.7837E4,-6.1995494E-3,4.9995813E-3,-1.6558258E-2,-7.9238815E-3,1.00019E5,2.88E0,2.123E3,1.1939979E8,-7.6184296E-3,-1.7975752E-3,2.55E2,-1.4750147E-2,2.3514E4,2E1,8.612209E-6,7.45E2,1E0,4.6712E4,1.056178E6,7.5983666E-3,3.693316E7,1.9376624E0,1.87627E5,1.8202102E-2,4.4971E4,1.3573053E-3,-4.3895687E-3,1.5822886E0,5.5208523E-3,1.2986845E-2,8E0,1.1E1,3.2952412E5,1.5E1,5.94E2,2.6790693E-2,-6.1836056E-3,2.5626253E-3,-1.3171322E-2,-2.4725582E-2,-7.5487415E-3,5.513889E0,7.7513396E-5,1.8791415E-2,3.1E1,2.3412812E0,-1.1578427E-2,-5.7540867E-3,1.3E1,1.0865825E0,5.44E2,5.5606193E-3,-2.0883323E-3,2.6732E4,5.69E2,-7.7523096E-3,8E0,-2.3818745E-3,-7.921826E-3,4.5830626E-3,8.81059E5,3.84E2,4.5355887E-4,-6.0457014E-3,2.3670635E1,2.483E3,-7.790265E-5,8.612825E-3,8.616152E-3,2.9208383E-3,-1.06385596E-4,4.1441983E-3,2.0027258E-2,1.0591589E-2,1.7179692E-2,2.9448232E-2,2.0659652E0,1.9264043E-3,1.4E1,1.3138838E-2,-4.5302436E-3,9.1990195E-3,-1.5037169E-2,-8.121841E-3,1.2E1,4.6E1,4.306E4,1.862E3,3.3272727E0,1.9668234E-3,-1.685911E-3,2.6801266E2,6.953366E7,-9.597459E-3,7.597862E6,9.610854E-3,1.4162437E0,2.7359598E-3,1.047E3,4.0119403E2,2.8E1,-1.8409701E-2,9.751103E6,4.6816683E-1,4.881099E-3,-7.3879417E-3,5.144203E2,1.2968812E-2,1.2038146E-2,2.3867035E-2,8.8894E-3,-1.0467523E-3,4.7866177E2,7.997723E6,-2.0567698E-2,-1.04516465E-2,-1.8106676E-3,3.784915E-3,-7.8015993E-3,-2.2148767E-3,-1.0186115E-3,5.0149255E0,-3.0158894E-2,-7.6714233E-3,6.763314E7,4.358214E6,7.837078E-5,4.518229E-3,-3.0988848E-3,5.8769487E-4,2.7777777E0,3.3253515E6,2.12E2,3.1706784E0,8.118812E-1,1.9545455E0,1.7787506E4,1.2E2,-7.429002E-3,4.99E2,4.498404E2,1.2183674E1,-7.599928E-3,2.7608543E-3,-1.2748771E-2,-2.8213009E-3,-9.0355445E-3,-3.98174E-3,-2.4131336E-3,-9.389699E-3,-3.1999135E-3,3.4211902E-3,-9.884589E-4,5.171651E-3,1.1303496E-2,5.980526E-3,2.4330695E-3,-2.166809E-3,2.1575398E-3,6.714982E-3,6.1021145E-3,-4.183727E-3,7.989451E-3,2.8440674E-5,4.2089876E-3,-1.0054508E-3,-6.8957923E-4,8.829163E-3,-5.13425E-3,2.5509289E-3,4.0809037E-3,-1.1029723E-3,8.457262E-3,2.759687E-3],"split_indices":[2,43,6,60,2,67,44,67,9,2,43,7,8,1,0,1,0,0,0,10,43,44,46,53,66,53,47,2,5,68,10,8,68,29,9,53,53,1,29,47,62,0,7,67,69,67,9,0,0,0,0,5,69,1,7,0,0,2,0,1,3,53,2,19,1,9,0,7,69,1,0,9,0,0,57,0,0,32,3,48,3,0,0,0,0,0,0,0,69,0,0,3,68,0,0,8,71,10,0,0,1,2,0,8,0,0,0,9,0,0,0,73,2,0,0,0,0,0,0,0,0,0,0,57,0,3,0,0,0,0,0,8,3,1,2,69,0,0,70,7,0,60,0,68,0,2,4,71,0,9,53,0,0,4,0,0,0,0,0,67,9,0,0,0,0,0,0,0,71,0,0,7,5,0,0,0,0,69,60,10,69,68,68,48,0,0,0,4,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.033E3,7.85E2,2.48E2,2.5E2,5.35E2,2.2E2,2.8E1,2.31E2,1.9E1,2.23E2,3.12E2,5.4E1,1.66E2,2.3E1,5E0,2.24E2,7E0,6E0,1.3E1,4.9E1,1.74E2,2.27E2,8.5E1,3.4E1,2E1,1.19E2,4.7E1,1.2E1,1.1E1,4.7E1,1.77E2,2.1E1,2.8E1,1.29E2,4.5E1,7.6E1,1.51E2,6.4E1,2.1E1,1.4E1,2E1,5E0,1.5E1,5.8E1,6.1E1,3.5E1,1.2E1,6E0,6E0,6E0,5E0,1.2E1,3.5E1,9E0,1.68E2,1.5E1,6E0,1.4E1,1.4E1,3.7E1,9.2E1,2.2E1,2.3E1,6.4E1,1.2E1,1.44E2,7E0,8E0,5.6E1,1E1,1.1E1,9E0,5E0,9E0,1.1E1,4E0,1.1E1,3.3E1,2.5E1,2.8E1,3.3E1,2.8E1,7E0,6E0,6E0,5E0,7E0,5E0,3E1,5E0,4E0,1.46E2,2.2E1,5E0,9E0,2.1E1,1.6E1,8.6E1,6E0,5E0,1.7E1,1.9E1,4E0,5.8E1,6E0,6E0,6E0,6.4E1,8E1,4E0,4E0,1E1,4.6E1,4E0,6E0,4E0,5E0,4E0,7E0,2.6E1,7E0,6E0,1.9E1,2.4E1,4E0,2E1,1.3E1,4E0,2.4E1,2.5E1,5E0,1.32E2,1.4E1,1.4E1,8E0,1.7E1,4E0,5E0,1.1E1,8E1,6E0,1E1,7E0,1.1E1,8E0,3.7E1,2.1E1,5.9E1,5E0,6E1,2E1,6E0,4E0,3.8E1,8E0,4E0,2E1,1.6E1,4E0,9.4E1,3.8E1,7E0,7E0,7E0,7E0,4E0,4E0,6E0,1.1E1,5E0,6E0,6.3E1,1.7E1,4E0,6E0,7E0,4E0,1.8E1,1.9E1,1.1E1,1E1,4.6E1,1.3E1,4.2E1,1.8E1,9E0,1.1E1,2.4E1,1.4E1,9E1,4E0,3.3E1,5E0,5E0,6E0,5.7E1,6E0,6E0,1.1E1,5E0,1.3E1,9E0,1E1,5E0,6E0,4E0,6E0,4E0,4.2E1,4E0,9E0,2.3E1,1.9E1,5E0,1.3E1,5E0,6E0,1.9E1,5E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[2.5721124E-4,-6.941781E-2,2.1714778E-1,-2.0841785E-1,-1.1861273E-2,2.6116636E-1,-3.2985624E-2,-1.874136E-1,-4.045349E-1,-1.5475622E-1,1.889853E-2,2.1016075E-1,4.4465017E-1,-1.5201495E-2,7.024072E-2,-2.6033053E-1,-1.2859589E-1,-2.1156715E-2,-8.994043E-3,-1.3139848E-1,-2.250427E-2,2.5277276E-2,-1.5591447E-2,2.643263E-1,8.177779E-2,5.2652365E-1,2.0261356E-1,1.6935204E-1,-8.427964E-2,-2.9067558E-1,-1.4533873E-1,1.6115719E-1,-1.5745144E-1,-1.7316738E-1,-8.1896804E-2,-5.4016E-3,9.334349E-2,1.1552466E-1,3.166208E-1,4.711955E-2,1.3531619E-2,1.5834536E-2,6.438996E-1,3.1449094E-3,1.1773628E-2,3.99937E-2,1.5063236E-2,2.5682794E-3,-7.4456334E-3,-3.2147348E-1,-2.2857998E-1,-6.641187E-2,-2.1105298E-1,3.803326E-3,1.1055024E-2,-1.277601E-1,-3.1648016E-1,-1.4948812E-1,-1.2743434E-2,-2.2059928E-3,-1.08393565E-1,-6.1240386E-2,4.302475E-2,1.03435725E-1,-9.045641E-3,1.8546598E-1,-5.427583E-3,4.648768E-1,2.517334E-1,6.544048E-2,-5.1990584E-3,3.3385865E-2,1.8741027E-2,-6.1183185E-3,7.2737103E-3,-1.8258354E-1,-1.601116E-2,-4.6916767E-3,-2.5283355E-1,-8.951615E-4,-4.7298083E-3,-1.2539495E-2,-5.336325E-3,-1.424571E-1,3.1856492E-3,-1.9143803E-2,-6.0713263E-3,-2.0149945E-1,-1.1628604E-1,2.1152708E-3,-2.6716453E-3,-1.2655258E-1,8.596487E-4,-2.5652793E-1,-4.3023027E-2,6.0954146E-2,-9.001108E-2,3.0621938E-2,1.603076E-1,-1.2320047E-4,2.3263249E-1,2.3992082E-2,1.4339581E-2,1.7017592E-1,3.059089E-1,-2.7941069E-2,1.0504462E-1,-1.1340461E-2,-3.842874E-3,-2.7481338E-1,-5.688928E-3,-1.0213446E-1,-2.1679962E-1,-3.421872E-3,-1.1138808E-2,-3.179752E-3,-1.3882832E-1,-8.746116E-2,-7.4929795E-3,-3.431194E-3,-1.7375572E-2,-1.5040724E-2,-3.1650174E-2,7.369837E-2,-5.5258647E-2,-1.3014556E-1,2.6066182E-3,-1.0831684E-1,6.0576033E-2,1.7547688E-1,-2.0043545E-3,4.7190785E-3,1.2993889E-2,2.186632E-1,6.588124E-2,3.39173E-1,5.5275876E-3,2.6303676E-3,-6.3040005E-3,9.76267E-3,7.409952E-2,-8.616283E-3,-1.5098357E-2,-1.1394966E-1,7.986102E-4,-4.629351E-3,-2.4934334E-1,-7.442143E-3,-2.2514497E-3,-5.6348983E-3,-1.1858534E-3,-4.725372E-2,9.8782554E-2,1.3226067E-2,1.069804E-1,4.2094857E-2,-9.62348E-3,-2.9536115E-3,-8.063637E-3,1.2176153E-3,-1.2713758E-2,8.9630626E-2,-6.4605415E-2,1.0361151E-1,2.2378138E-1,1.197937E-2,5.4405904E-3,9.944889E-3,-5.8465656E-3,3.6956805E-1,5.233609E-3,9.348919E-2,6.3083536E-4,-2.6966045E-3,-6.9168294E-3,-1.2859753E-2,-5.801923E-3,-1.6738634E-3,-1.1154813E-2,8.751863E-3,3.2852514E-4,2.3848298E-3,-1.5936765E-3,5.5873496E-3,-4.5351812E-4,-2.6152588E-3,5.4206196E-3,2.7820508E-3,7.192777E-3,-8.781809E-3,2.34366E-3,2.8736636E-3,9.688542E-3,5.5319746E-3,1.1517445E-2,1.0970537E-2,1.8744828E-2,7.1072887E-4,5.65228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,-1,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,-1,73,-1,-1,-1,75,77,79,81,-1,-1,83,85,87,-1,89,91,93,95,97,-1,99,-1,101,103,105,-1,-1,-1,-1,-1,107,-1,-1,109,-1,-1,-1,-1,111,-1,-1,-1,113,115,-1,-1,117,-1,119,121,123,125,127,129,-1,131,-1,-1,133,135,137,139,-1,-1,141,-1,143,145,-1,-1,-1,147,149,-1,-1,-1,-1,151,153,155,157,-1,159,161,163,-1,-1,-1,165,167,169,-1,-1,-1,-1,171,-1,-1,173,-1,-1,175,-1,-1,-1,-1,177,179,181,183,185,-1,-1,-1,-1,-1,187,189,191,193,-1,-1,-1,-1,195,-1,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5414054E1,6.1738076E0,2.7392235E0,8.5514355E-1,2.4072196E0,1.881237E0,1.184429E0,8.500166E-1,1.4623117E-1,6.890199E-1,1.0337775E0,1.1396914E0,7.850561E-1,0E0,4.5839968E-1,2.7529144E-1,9.952483E-1,0E0,0E0,1.7603993E-1,0E0,9.276626E-1,0E0,8.738594E-1,3.593689E-1,4.911518E-1,7.400721E-2,3.5182512E-1,1.3800666E-1,6.229925E-2,9.4902545E-2,4.8054814E-2,4.6007466E-1,7.78923E-2,9.266901E-2,8.327839E-1,4.229468E-1,5.432877E-1,7.102909E-1,1.389912E-1,0E0,0E0,6.333256E-2,0E0,0E0,2.4361694E-1,0E0,0E0,0E0,9.357071E-2,5.750668E-2,1.6625367E-2,3.2097965E-2,0E0,0E0,2.729149E-1,2.3541677E-1,4.9097538E-2,0E0,3.468129E-2,7.845992E-2,4.977035E-1,3.9905316E-1,5.499928E-1,0E0,2.224595E-1,0E0,4.9082756E-2,2.218833E-1,1.5456519E-1,0E0,0E0,0E0,0E0,0E0,3.5841018E-2,0E0,0E0,2.887392E-2,0E0,0E0,0E0,0E0,2.3266053E-1,0E0,0E0,0E0,5.784613E-2,2.0400047E-2,0E0,0E0,2.6214063E-2,0E0,2.3358792E-1,4.1385034E-1,2.2030461E-1,1.2488291E-1,2.562685E-1,2.3935354E-1,0E0,9.817135E-2,0E0,0E0,1.221323E-1,1.8246865E-1,1.2750067E-1,7.9711616E-2,0E0,0E0,2.1702766E-2,0E0,8.239561E-2,8.505356E-2,0E0,0E0,0E0,2.8249264E-2,3.0987598E-2,0E0,0E0,0E0,0E0,2.6517516E-1,2.6425362E-1,2.3226213E-1,3.5874456E-2,0E0,2.5446662E-1,1.8764544E-1,2.1981311E-1,0E0,0E0,0E0,4.3266058E-2,3.1469896E-1,1.6411138E-1,0E0,0E0,0E0,0E0,2.685155E-2,0E0,0E0,9.463805E-2,0E0,0E0,4.403293E-2,0E0,0E0,0E0,0E0,2.3740658E-1,1.1500035E-1,8.972961E-2,1.3978243E-1,8.090342E-2,0E0,0E0,0E0,0E0,0E0,7.028574E-2,1.543665E-1,1.1876455E-1,7.900262E-2,0E0,0E0,0E0,0E0,2.7895212E-2,0E0,3.868772E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,42,42,45,45,49,49,50,50,51,51,52,52,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,65,65,67,67,68,68,69,69,75,75,78,78,83,83,87,87,88,88,91,91,93,93,94,94,95,95,96,96,97,97,98,98,100,100,103,103,104,104,105,105,106,106,109,109,111,111,112,112,116,116,117,117,122,122,123,123,124,124,125,125,127,127,128,128,129,129,133,133,134,134,135,135,140,140,143,143,146,146,151,151,152,152,153,153,154,154,155,155,161,161,162,162,163,163,164,164,169,169,171,171],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,-1,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,-1,74,-1,-1,-1,76,78,80,82,-1,-1,84,86,88,-1,90,92,94,96,98,-1,100,-1,102,104,106,-1,-1,-1,-1,-1,108,-1,-1,110,-1,-1,-1,-1,112,-1,-1,-1,114,116,-1,-1,118,-1,120,122,124,126,128,130,-1,132,-1,-1,134,136,138,140,-1,-1,142,-1,144,146,-1,-1,-1,148,150,-1,-1,-1,-1,152,154,156,158,-1,160,162,164,-1,-1,-1,166,168,170,-1,-1,-1,-1,172,-1,-1,174,-1,-1,176,-1,-1,-1,-1,178,180,182,184,186,-1,-1,-1,-1,-1,188,190,192,194,-1,-1,-1,-1,196,-1,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.3876712E7,4.217427E7,4.97E2,1.4176E4,1.4598765E0,2.71E2,1.6074808E5,1E0,4.4E1,8E0,1.2792593E2,-1.5201495E-2,7.7175174E8,1.9E1,1.1E1,-2.1156715E-2,-8.994043E-3,3.56E2,-2.250427E-2,2.0766992E6,-1.5591447E-2,6.9664386E2,2.8016653E3,1.9705E4,4.988E3,6.355E3,3.592428E7,1.16E2,1E0,1.07882355E2,3.6349E4,1.9530877E5,1.2051282E0,1.212945E6,1.0089981E3,4.75356E6,1.5282027E5,1.6677892E2,1.3531619E-2,1.5834536E-2,1.61E9,3.1449094E-3,1.1773628E-2,1.7431098E8,1.5063236E-2,2.5682794E-3,-7.4456334E-3,1.863E3,2.0395E4,3.83E2,6.44E2,3.803326E-3,1.1055024E-2,5.185192E2,1.775373E1,1.0538846E3,-1.2743434E-2,4.33E2,1E0,2.83408E5,7.1E1,4.6066E4,-9.045641E-3,1.02E2,-5.427583E-3,1.2909952E1,5.831829E6,1.0901037E10,-5.1990584E-3,3.3385865E-2,1.8741027E-2,-6.1183185E-3,7.2737103E-3,1.1E1,-1.601116E-2,-4.6916767E-3,3.328629E6,-8.951615E-4,-4.7298083E-3,-1.2539495E-2,-5.336325E-3,1.7E1,3.1856492E-3,-1.9143803E-2,-6.0713263E-3,2.4636364E1,3.806E3,2.1152708E-3,-2.6716453E-3,9.695652E0,8.596487E-4,4.5801528E-2,7E0,1E0,2.0316172E6,2.2007043E0,2.5345264E7,-1.2320047E-4,1.9376624E0,2.3992082E-2,1.4339581E-2,1.934E3,2.6912618E7,1.2473E4,1.0092541E8,-1.1340461E-2,-3.842874E-3,2.1E1,-5.688928E-3,2.0263722E8,5.236559E0,-3.421872E-3,-1.1138808E-2,-3.179752E-3,2E0,1.2123E4,-7.4929795E-3,-3.431194E-3,-1.7375572E-2,-1.5040724E-2,4.888E3,8.69E2,5.4007E7,1.4E1,2.6066182E-3,1E0,1.4E1,2.485E2,-2.0043545E-3,4.7190785E-3,1.2993889E-2,3.7951445E6,1.8667632E0,2.057143E0,5.5275876E-3,2.6303676E-3,-6.3040005E-3,9.76267E-3,1.213E5,-8.616283E-3,-1.5098357E-2,2.7777777E0,7.986102E-4,-4.629351E-3,8.43E2,-7.442143E-3,-2.2514497E-3,-5.6348983E-3,-1.1858534E-3,3.607317E1,2.6595745E0,1E0,3.471E3,1.14E3,-9.62348E-3,-2.9536115E-3,-8.063637E-3,1.2176153E-3,-1.2713758E-2,6.7723384E7,2.8266037E2,6.887749E7,1.0029973E8,1.197937E-2,5.4405904E-3,9.944889E-3,-5.8465656E-3,1.0775862E0,5.233609E-3,2.8802464E8,6.3083536E-4,-2.6966045E-3,-6.9168294E-3,-1.2859753E-2,-5.801923E-3,-1.6738634E-3,-1.1154813E-2,8.751863E-3,3.2852514E-4,2.3848298E-3,-1.5936765E-3,5.5873496E-3,-4.5351812E-4,-2.6152588E-3,5.4206196E-3,2.7820508E-3,7.192777E-3,-8.781809E-3,2.34366E-3,2.8736636E-3,9.688542E-3,5.5319746E-3,1.1517445E-2,1.0970537E-2,1.8744828E-2,7.1072887E-4,5.65228E-3],"split_indices":[2,43,60,60,2,2,69,2,48,106,3,32,73,0,7,3,10,0,0,2,0,43,0,67,4,2,44,2,60,44,20,67,44,66,68,9,4,62,48,73,0,0,5,0,0,5,0,0,0,9,9,12,1,0,0,67,71,48,0,2,28,7,8,1,0,0,0,71,60,46,0,0,0,0,0,5,0,0,9,0,0,0,0,3,0,0,0,67,1,0,0,73,0,73,3,19,62,69,60,0,69,0,0,44,62,44,7,0,0,0,0,7,69,0,0,0,32,1,0,0,0,0,10,2,7,3,0,8,8,67,0,0,0,43,57,68,0,0,0,0,10,0,0,71,0,0,10,0,0,0,0,73,68,26,10,2,0,0,0,0,0,7,4,7,5,0,0,0,0,68,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.71E2,2.47E2,2.25E2,5.46E2,2.1E2,3.7E1,2.05E2,2E1,9.6E1,4.5E2,1.66E2,4.4E1,9E0,2.8E1,9E1,1.15E2,1.5E1,5E0,9.1E1,5E0,4.43E2,7E0,1.16E2,5E1,3.2E1,1.2E1,1.7E1,1.1E1,7E1,2E1,1E1,1.05E2,4.8E1,4.3E1,3.06E2,1.37E2,3.1E1,8.5E1,4.4E1,6E0,1.4E1,1.8E1,4E0,8E0,1E1,7E0,4E0,7E0,4.4E1,2.6E1,1E1,1E1,6E0,4E0,9E1,1.5E1,4.1E1,7E0,1.1E1,3.2E1,1.42E2,1.64E2,1.33E2,4E0,2.4E1,7E0,2.4E1,6.1E1,4E1,4E0,1.2E1,6E0,4E0,6E0,8E0,3.6E1,5E0,2.1E1,5E0,5E0,5E0,5E0,8.4E1,6E0,9E0,6E0,1.4E1,2.7E1,6E0,5E0,2.8E1,4E0,1.1E1,1.31E2,1.45E2,1.9E1,5.9E1,7.4E1,5E0,1.9E1,1.6E1,8E0,2.6E1,3.5E1,1.2E1,2.8E1,4E0,4E0,1.7E1,4E0,5.6E1,2.8E1,4E0,1E1,1E1,1.7E1,1.5E1,1.3E1,5E0,6E0,4E0,1.27E2,1.31E2,1.4E1,1.5E1,4E0,1E1,4.9E1,6.9E1,5E0,6E0,1.3E1,1.7E1,9E0,2.9E1,6E0,7E0,5E0,5E0,2.3E1,8E0,9E0,5.1E1,5E0,7E0,2.1E1,1.3E1,4E0,9E0,6E0,1.14E2,1.3E1,4.7E1,8.4E1,9E0,5E0,7E0,8E0,6E0,4E0,4E1,9E0,2.9E1,4E1,1.1E1,6E0,5E0,4E0,2.5E1,4E0,1.7E1,6E0,2.1E1,3E1,1.6E1,5E0,1.09E2,5E0,6E0,7E0,2.6E1,2.1E1,7.5E1,9E0,4E0,5E0,2.9E1,1.1E1,4E0,5E0,2.2E1,7E0,9E0,3.1E1,7E0,1.8E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[-5.3168205E-3,-7.74734E-2,2.0169191E-1,-2.1571895E-1,-1.1063464E-2,1.3282856E-1,3.6131305E-1,-1.9371118E-1,-3.9438784E-1,-1.2515052E-1,2.0699512E-2,9.7009264E-2,5.8387464E-1,3.8022965E-1,-2.7943772E-4,-3.0554298E-1,-1.5399313E-1,-1.7895313E-1,-5.313147E-1,-1.519736E-1,3.9994195E-2,-6.532466E-2,4.1314524E-2,1.2865941E-1,-2.1402809E-1,3.1359896E-2,1.2808675E-2,4.9003658E-1,3.2540062E-1,-2.1329568E-1,-3.4083948E-1,1.1649526E-1,-1.7514001E-1,-1.3769596E-2,-2.3702849E-3,-3.6308873E-2,-2.9995766E-1,-1.2228492E-1,-2.4604942E-1,-2.210123E-2,4.511144E-3,-1.6732469E-1,-1.944176E-2,3.3107933E-2,1.5556166E-2,4.0220708E-1,1.0441745E-1,-1.5473763E-2,-6.933846E-2,1.6368799E-2,2.7545583E-2,2.344124E-1,3.8828143E-1,-1.144036E-2,-3.1243395E-3,-1.645332E-2,-7.3663923E-3,-4.727472E-3,2.2512001E-1,-1.5317227E-1,-3.496251E-1,-7.40218E-3,-1.6927782E-2,-1.6040571E-1,-8.918237E-2,-2.9165658E-1,-1.0871704E-3,2.1256988E-3,-3.9635994E-3,-6.320663E-2,-2.9409254E-1,-9.239928E-3,5.7223067E-3,1.0242157E-1,2.3927146E-3,2.2257049E-2,1.0584036E-2,-6.5492444E-2,1.2629864E-1,-4.2622528E-4,-5.3398423E-3,1.4539092E-2,8.228761E-3,1.890475E-2,9.131394E-3,4.570507E-3,1.4150088E-2,-1.6706988E-1,7.2310865E-2,-1.8060554E-2,-9.643882E-3,-1.01115964E-1,-2.0501201E-1,-1.04588315E-1,2.3978092E-3,-1.7310146E-2,-8.290944E-3,1.4051679E-3,-5.22323E-3,-4.314174E-3,-2.3851106E-2,9.6280344E-2,-2.7791766E-2,7.13368E-2,2.1080156E-1,-1.7589917E-2,1.1751634E-1,9.905234E-4,-1.2276243E-1,1.5467829E-1,3.0117992E-2,-2.1173723E-1,-1.0027598E-1,-1.6374715E-3,7.6507195E-3,-1.4358059E-1,-1.6352473E-3,-1.0641862E-2,-4.065583E-3,-7.0854135E-2,-7.4924002E-3,1.7874026E-3,7.697704E-3,-6.9604285E-2,9.6243724E-2,9.594987E-2,-7.492316E-3,1.5364879E-2,1.3179964E-1,-7.489912E-2,7.6373997E-3,4.0003918E-2,1.7660488E-1,-7.77869E-3,-1.5871506E-3,1.354796E-1,2.0589838E-2,-8.870381E-3,6.870691E-2,-1.8367597E-1,-3.554447E-1,-1.4009155E-1,-6.9369463E-4,-9.562683E-3,-3.0865753E-3,-5.7975124E-3,-2.4362074E-3,-6.7767436E-3,-3.841082E-2,1.8019229E-3,6.7304294E-3,7.2633E-2,1.812663E-1,1.791936E-1,5.4190175E-5,-4.928792E-2,-1.372968E-2,2.3710981E-2,-1.0721794E-1,7.766926E-3,-1.6710917E-2,1.1481164E-2,8.703412E-2,1.904918E-1,7.306057E-2,1.22991525E-1,-3.8058117E-3,-9.950286E-3,-3.0121743E-3,-2.0771096E-2,-8.609469E-3,-8.783466E-3,-5.0339014E-3,-2.618099E-3,1.2636482E-3,-3.939506E-3,2.731606E-5,2.0682104E-3,4.7070077E-3,1.0218954E-2,2.6412944E-3,1.0034836E-2,4.164094E-3,8.341585E-4,-4.223221E-3,-3.3481237E-3,2.1724629E-3,-8.852959E-3,-2.0215022E-3,1.9006011E-3,-3.761425E-3,7.8010594E-4,7.10566E-3,6.305115E-3,1.4021178E-2,-5.677469E-3,5.6100124E-3,3.424116E-3,9.708883E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,-1,49,51,53,55,57,59,-1,-1,-1,61,63,65,67,-1,69,71,73,-1,75,77,-1,79,-1,-1,81,83,-1,-1,-1,-1,-1,85,87,89,-1,-1,91,93,95,-1,-1,-1,97,99,-1,101,103,105,-1,-1,107,109,-1,-1,-1,-1,-1,-1,-1,-1,111,113,-1,-1,115,117,119,-1,-1,-1,-1,-1,-1,-1,121,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,143,-1,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,153,155,157,159,161,-1,-1,163,-1,-1,165,167,169,171,173,-1,-1,-1,-1,-1,175,-1,-1,177,179,181,-1,183,-1,185,187,-1,189,-1,191,193,195,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5833953E1,7.2137837E0,2.9638271E0,9.2628956E-1,1.9304225E0,3.0614972E0,5.7213783E-1,9.701071E-1,7.013869E-1,5.199224E-1,7.422021E-1,1.799525E0,2.1929169E-1,3.101492E-1,0E0,1.1940527E-1,9.985213E-1,1.6713586E-1,7.067418E-1,2.492106E-1,6.301643E-2,3.758315E-1,8.058327E-1,1.0455413E0,2.7417797E-1,0E0,0E0,1.346879E-1,2.2365904E-1,7.828534E-2,5.7002544E-2,3.320417E-1,5.5590105E-1,0E0,0E0,0E0,3.4148574E-2,8.412242E-2,2.2654927E-1,4.299224E-2,0E0,3.11382E-1,2.6013216E-1,7.001904E-1,0E0,6.866813E-2,5.7338154E-1,0E0,2.3952786E-2,0E0,0E0,5.6952596E-2,4.275036E-2,0E0,0E0,0E0,0E0,0E0,6.187865E-2,4.6019626E-1,1.7315626E-2,0E0,0E0,7.1834326E-2,1.0106379E-1,1.0975349E-1,0E0,0E0,0E0,7.344839E-2,4.4231218E-1,0E0,1.574592E-1,3.2522774E-1,5.288754E-1,0E0,0E0,9.224757E-2,3.6383367E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.81541E-1,9.6343994E-2,0E0,0E0,4.2361245E-2,3.605622E-2,6.621677E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.0267115E-2,2.0377596E-1,4.753556E-1,1.6782218E-1,2.8394052E-1,1.486584E-1,0E0,3.8571596E-2,5.206454E-1,2.8747773E-1,2.573223E-1,2.2193867E-1,0E0,0E0,3.48496E-2,0E0,0E0,0E0,2.0114452E-2,0E0,0E0,0E0,6.409441E-2,2.2617184E-2,1.3205391E-1,0E0,0E0,9.265354E-2,3.2693344E-1,2.5546297E-1,1.1906559E-1,1.00750804E-1,0E0,0E0,3.2602966E-1,0E0,0E0,2.3977716E-1,2.4512148E-1,1.0726166E-1,4.158306E-2,2.8238486E-2,0E0,0E0,0E0,0E0,0E0,4.0259857E-2,0E0,0E0,4.3531835E-2,6.1675876E-2,1.7663985E-2,0E0,1.5767989E-1,0E0,2.7472693E-1,8.2678944E-2,0E0,4.8269242E-2,0E0,4.4241786E-2,2.7597857E-1,4.698837E-1,7.3884726E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,51,51,52,52,58,58,59,59,60,60,63,63,64,64,65,65,69,69,70,70,72,72,73,73,74,74,77,77,78,78,87,87,88,88,91,91,92,92,93,93,101,101,102,102,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,112,112,115,115,119,119,123,123,124,124,125,125,128,128,129,129,130,130,131,131,132,132,135,135,138,138,139,139,140,140,141,141,142,142,148,148,151,151,152,152,153,153,155,155,157,157,158,158,160,160,162,162,163,163,164,164,165,165],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,-1,50,52,54,56,58,60,-1,-1,-1,62,64,66,68,-1,70,72,74,-1,76,78,-1,80,-1,-1,82,84,-1,-1,-1,-1,-1,86,88,90,-1,-1,92,94,96,-1,-1,-1,98,100,-1,102,104,106,-1,-1,108,110,-1,-1,-1,-1,-1,-1,-1,-1,112,114,-1,-1,116,118,120,-1,-1,-1,-1,-1,-1,-1,122,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,144,-1,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,154,156,158,160,162,-1,-1,164,-1,-1,166,168,170,172,174,-1,-1,-1,-1,-1,176,-1,-1,178,180,182,-1,184,-1,186,188,-1,190,-1,192,194,196,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,2.9251662E6,4.4895835E0,5.47E2,5.428175E3,2E0,4.54E2,3.89E2,2.9652428E6,1E0,3.9661028E7,1.3342042E0,3.3382E4,-2.7943772E-4,2.416E3,2.6333334E1,1.317484E1,1.753E3,6.2233735E-6,4.49239E6,8.1E2,7.168071E6,5.1E1,1.5900173E3,3.1359896E-2,1.2808675E-2,5.3700186E8,6.426937E6,3.07E2,2.7546012E0,1.3E2,4.217427E7,-1.3769596E-2,-2.3702849E-3,-3.6308873E-2,6.185E3,4.3827028E5,9E0,1.2E1,4.511144E-3,6.83E2,4.539777E5,1.5197045E0,1.5556166E-2,6.7321223E-1,2.8204132E7,-1.5473763E-2,2.6654E4,1.6368799E-2,2.7545583E-2,9.4658756E-1,3.3765998E0,-1.144036E-2,-3.1243395E-3,-1.645332E-2,-7.3663923E-3,-4.727472E-3,2.638E3,4.4970587E2,1.5416006E5,-7.40218E-3,-1.6927782E-2,3.1E1,7.9559E5,3.9595376E7,-1.0871704E-3,2.1256988E-3,-3.9635994E-3,1.4598765E0,6.278306E7,-9.239928E-3,1.3E1,4.797342E2,5.862504E2,2.2257049E-2,1.0584036E-2,1.3E1,1.6007428E7,-4.2622528E-4,-5.3398423E-3,1.4539092E-2,8.228761E-3,1.890475E-2,9.131394E-3,4.570507E-3,1.4150088E-2,1.339646E6,1.7500242E7,-1.8060554E-2,-9.643882E-3,3.3820656E7,1.9470909E2,5.1E1,2.3978092E-3,-1.7310146E-2,-8.290944E-3,1.4051679E-3,-5.22323E-3,-4.314174E-3,-2.3851106E-2,1.295313E6,2.364E3,4.5620965E6,2E0,4.4761734E5,2.8209653E-2,9.905234E-4,2.1340163E8,3.4915986E3,1.102794E3,2.0734E4,1.1995476E6,-1.6374715E-3,7.6507195E-3,4.017544E0,-1.6352473E-3,-1.0641862E-2,-4.065583E-3,6.14E2,-7.4924002E-3,1.7874026E-3,7.697704E-3,3.6966666E2,6.33E2,7.6569915E-1,-7.492316E-3,1.5364879E-2,9.07826E0,5.465798E7,6.4E1,1.125584E-3,9E0,-7.77869E-3,-1.5871506E-3,2.9308079E1,2.0589838E-2,-8.870381E-3,3.85374E8,5.57E2,6.160415E6,1.631108E6,1.030671E6,-9.562683E-3,-3.0865753E-3,-5.7975124E-3,-2.4362074E-3,-6.7767436E-3,1.6E1,1.8019229E-3,6.7304294E-3,8.7E1,3.7509E4,1.124641E0,5.4190175E-5,7.9146667E0,-1.372968E-2,2.5437157E2,9.884021E0,7.766926E-3,3.774872E7,1.1481164E-2,2.6662E4,7.177158E6,8.237181E9,3.2952412E5,-3.8058117E-3,-9.950286E-3,-3.0121743E-3,-2.0771096E-2,-8.609469E-3,-8.783466E-3,-5.0339014E-3,-2.618099E-3,1.2636482E-3,-3.939506E-3,2.731606E-5,2.0682104E-3,4.7070077E-3,1.0218954E-2,2.6412944E-3,1.0034836E-2,4.164094E-3,8.341585E-4,-4.223221E-3,-3.3481237E-3,2.1724629E-3,-8.852959E-3,-2.0215022E-3,1.9006011E-3,-3.761425E-3,7.8010594E-4,7.10566E-3,6.305115E-3,1.4021178E-2,-5.677469E-3,5.6100124E-3,3.424116E-3,9.708883E-3],"split_indices":[2,43,43,68,2,67,6,1,2,43,29,60,53,9,0,9,62,69,44,52,43,2,43,0,70,0,0,7,60,1,68,2,60,0,0,0,44,43,8,3,0,2,43,68,0,53,5,0,9,0,0,68,53,0,0,0,0,0,1,67,48,0,0,10,12,7,0,0,0,69,5,0,3,4,67,0,0,3,66,0,0,0,0,0,0,0,0,9,12,0,0,7,4,10,0,0,0,0,0,0,0,9,2,62,8,43,53,0,7,4,70,44,60,0,0,69,0,0,0,1,0,0,0,67,0,57,0,0,71,5,8,53,8,0,0,71,0,0,47,10,12,5,12,0,0,0,0,0,3,0,0,0,1,53,0,73,0,4,71,0,7,0,10,60,46,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.058E3,7.85E2,2.73E2,2.54E2,5.31E2,1.92E2,8.1E1,2.28E2,2.6E1,1.15E2,4.16E2,1.79E2,1.3E1,7.7E1,4E0,5.8E1,1.7E2,1.1E1,1.5E1,9.9E1,1.6E1,8E1,3.36E2,1.63E2,1.6E1,9E0,4E0,2.3E1,5.4E1,1.8E1,4E1,1.2E1,1.58E2,5E0,6E0,6E0,9E0,7.7E1,2.2E1,8E0,8E0,2.4E1,5.6E1,3.28E2,8E0,1.2E1,1.51E2,8E0,8E0,1.2E1,1.1E1,2.4E1,3E1,1.4E1,4E0,3.6E1,4E0,4E0,8E0,1.42E2,1.6E1,4E0,5E0,3.4E1,4.3E1,1.8E1,4E0,4E0,4E0,1.4E1,1E1,6E0,5E1,1E2,2.28E2,7E0,5E0,1.7E1,1.34E2,4E0,4E0,8E0,1.6E1,2.6E1,4E0,4E0,4E0,1.34E2,8E0,1.1E1,5E0,1.6E1,1.8E1,3.9E1,4E0,9E0,9E0,5E0,9E0,6E0,4E0,1.3E1,3.7E1,7.9E1,2.1E1,1.95E2,3.3E1,7E0,1E1,1.03E2,3.1E1,7.9E1,5.5E1,4E0,4E0,9E0,7E0,1.4E1,4E0,2.6E1,1.3E1,8E0,5E0,2.8E1,9E0,7.2E1,7E0,7E0,1.4E1,5.9E1,1.36E2,1.5E1,1.8E1,6E0,4E0,9.8E1,5E0,4E0,2.7E1,6.8E1,1.1E1,3.9E1,1.6E1,4E0,5E0,5E0,2.1E1,7E0,2.1E1,5E0,4E0,5.8E1,1.4E1,1E1,4E0,5.4E1,5E0,1.2E2,1.6E1,4E0,1.1E1,9E0,9E0,5.1E1,4.7E1,2E1,7E0,5.3E1,1.5E1,6E0,5E0,1.3E1,2.6E1,5E0,1.1E1,9E0,1.2E1,3.1E1,2.7E1,1E1,4E0,6E0,4E0,2.1E1,3.3E1,2.3E1,9.7E1,6E0,1E1,6E0,5E0,5E0,4E0,3.6E1,1.5E1,9E0,3.8E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[-9.581912E-3,-7.257247E-2,1.7942704E-1,-1.9418085E-1,-1.7906478E-2,-3.5965043E-1,2.0962237E-1,-1.6565062E-1,-3.8026592E-1,-1.2550124E-2,-2.6842918E-2,-2.3116974E-2,-1.0435031E-2,1.0108888E-1,2.8178123E-1,-1.5956992E-2,-1.3240612E-1,-5.664697E-1,-2.6462272E-1,-6.357436E-2,4.211078E-2,7.2013184E-2,2.8789636E-1,3.04593E-1,-9.930838E-2,6.466385E-2,-1.5498063E-1,-1.1142762E-2,-3.6836233E-2,-6.2297476E-3,-3.341127E-1,-4.581332E-2,-1.7267416E-1,2.0968283E-2,1.3817504E-1,5.1410656E-2,1.3865546E-2,1.6300699E-2,8.400301E-3,3.9637175E-1,2.3511447E-1,-8.551965E-3,2.9365002E-4,-8.7408796E-2,1.5608201E-2,-2.111568E-1,-1.0661881E-1,-1.8770915E-2,-8.416108E-3,-1.9438569E-2,-1.268761E-1,-2.324132E-1,-3.9913446E-2,-2.0647852E-2,7.2986744E-2,3.5533722E-2,1.8669394E-1,7.659787E-2,-8.622999E-2,3.3156115E-1,3.1152323E-2,3.555974E-1,1.6828786E-1,-1.3954026E-3,-6.8889037E-3,-2.4515393E-1,-9.3656994E-2,-7.471838E-2,-1.39719015E-2,-4.5223065E-2,3.9984792E-2,-9.4256885E-2,-2.0000146E-1,-1.6699068E-1,-1.783897E-2,1.1126087E-3,-7.103217E-2,3.2815162E-2,-7.65202E-2,1.4088078E-1,3.0322812E-2,3.80353E-3,-1.6813305E-3,1.0851359E-3,2.1000262E-1,9.989559E-3,1.5330155E-1,1.4128158E-4,-6.5795444E-3,3.895956E-1,1.8751533E-1,3.4678753E-3,3.9922535E-1,9.4451435E-2,2.5016007E-1,-1.6030985E-1,-3.1861877E-1,-6.0339727E-2,-7.7713244E-3,-1.06611826E-1,1.06922075E-1,1.0290782E-2,-7.21214E-2,-5.059808E-3,6.0526878E-2,-1.173335E-2,-7.076105E-2,-1.16590075E-2,-5.06098E-3,-1.0104442E-2,-2.8673087E-3,-8.059999E-4,-5.1009133E-3,-5.1831216E-2,1.4132656E-1,-2.2734904E-1,-8.436309E-3,2.059346E-3,1.6258979E-1,-2.3621519E-3,1.5853576E-1,1.6545305E-1,1.363455E-2,-4.0741548E-2,9.986728E-2,2.0863858E-1,7.6677024E-2,2.731075E-1,2.0024195E-2,3.3940226E-4,2.6874238E-1,1.0049854E-2,2.06835E-2,1.3586044E-1,-6.457043E-3,2.5187032E-3,2.84829E-1,-1.9281705E-1,-8.251519E-2,-6.3519846E-3,-3.5062477E-1,-3.810116E-3,-1.8515399E-4,-2.0755851E-1,-7.2851844E-2,8.5797915E-3,-1.87239E-3,3.1531107E-2,-4.5437623E-2,5.6683127E-2,-9.2034005E-2,4.04152E-2,9.143933E-3,-9.0005174E-2,1.2483862E-3,-1.189828E-1,1.0979742E-2,1.9397499E-1,-1.7005035E-3,-8.496999E-2,-2.6820421E-2,6.3791305E-2,-5.7213634E-2,1.8103935E-3,1.8076679E-1,3.0098017E-2,-1.0225083E-1,9.999024E-3,2.9828758E-3,9.143727E-3,3.5519006E-3,-1.002226E-1,1.0105338E-3,9.627098E-3,3.5622675E-2,5.269908E-3,1.091391E-2,2.9007126E-2,7.678993E-3,1.4638257E-2,6.860326E-3,5.6368955E-3,1.5483941E-2,1.6306031E-1,-5.0777383E-4,8.077706E-3,1.46527635E-2,-1.1176726E-2,-5.7484065E-3,-6.5746172E-3,-8.746909E-4,-1.8007085E-2,-9.23389E-3,-4.8850356E-3,-1.2704869E-2,-2.1737895E-3,-9.500405E-3,1.6851719E-4,4.5868917E-3,-5.4463306E-3,1.0696977E-3,-1.7372688E-3,5.013853E-3,-3.3590717E-3,-1.5900875E-2,2.7876692E-3,-2.6236782E-3,-1.9486265E-3,-6.2377723E-3,-2.3766218E-3,2.4804354E-3,-1.2117583E-2,-2.527635E-3,-2.295875E-3,3.259657E-3,4.09689E-3,1.1639896E-2,-8.153879E-3,1.467654E-3,-5.6455034E-4,4.4573303E-3,-7.0261434E-3,-1.6735867E-4,3.938377E-3,9.00426E-3,2.5920656E-3,-3.2891848E-3,-1.8424853E-3,-8.389362E-3,-2.5470501E-3,-6.6360096E-3,4.0666E-3,-1.5884526E-3,3.1150458E-3,-6.5819285E-4,1.2904538E-2,5.1943446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,49,51,53,55,57,-1,-1,-1,59,61,-1,-1,63,-1,65,67,-1,-1,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,-1,101,103,105,107,109,-1,-1,111,113,115,117,119,-1,-1,-1,121,123,125,-1,-1,127,129,-1,131,133,135,137,139,141,-1,143,145,147,149,-1,151,-1,153,-1,-1,-1,-1,-1,-1,155,157,159,161,-1,163,165,167,169,-1,171,173,175,177,179,-1,-1,181,-1,-1,183,-1,-1,185,187,189,-1,191,-1,-1,193,195,-1,-1,197,199,201,203,205,-1,207,209,211,213,215,-1,217,-1,219,221,-1,223,225,227,-1,-1,-1,-1,229,-1,-1,231,-1,-1,233,-1,-1,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2240258E1,5.1220093E0,4.2587376E0,1.2058716E0,1.6003832E0,1.505487E-1,1.8824387E0,1.2031603E0,5.1762867E-1,1.4780744E0,0E0,0E0,0E0,5.169921E-1,1.2994747E0,0E0,8.0190086E-1,6.3375187E-1,1.4432836E-1,5.230262E-1,5.170484E-1,3.934447E-1,2.5388718E-2,7.867861E-1,8.277353E-2,8.188601E-1,4.121716E-1,0E0,0E0,0E0,6.858122E-2,5.042422E-1,2.921015E-1,4.5828137E-1,2.2244811E-1,2.8741452E-1,0E0,0E0,0E0,8.21187E-1,5.9232616E-1,0E0,0E0,4.1383535E-2,0E0,2.6864958E-1,5.27087E-1,0E0,0E0,2.7760446E-1,1.1949545E-1,1.969943E-1,2.7599968E-2,3.5498926E-1,2.6801097E-1,5.684987E-2,1.1299467E-1,3.509967E-1,6.56547E-2,3.3656597E-1,0E0,3.127277E-1,3.0281007E-1,0E0,0E0,2.9797363E-1,3.4562618E-2,4.574708E-1,0E0,1.8888918E-1,1.7554389E-1,1.3964123E-1,5.2874148E-2,9.497136E-2,0E0,0E0,1.706541E-2,5.6761605E-1,5.9368473E-1,6.801742E-2,2.4762201E-1,0E0,0E0,0E0,5.3560495E-2,1.7733341E-1,1.19687736E-1,0E0,0E0,5.5248737E-2,2.3217899E-1,0E0,1.1400533E-1,3.0994967E-1,1.5403879E-1,5.489856E-2,1.1990929E-1,1.7123844E-2,0E0,2.098043E-1,1.4694121E-1,5.1147655E-2,2.2322768E-1,0E0,1.2740256E-1,0E0,5.2771196E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.4893135E-1,2.6035976E-1,8.5583895E-1,1.4809561E-1,0E0,5.7966113E-2,1.5871291E-1,5.13137E-2,4.2295992E-2,0E0,9.53024E-2,9.017858E-2,2.0031273E-2,5.9202842E-2,1.8362343E-2,0E0,0E0,6.630117E-2,0E0,0E0,1.05645E-1,0E0,0E0,1.9231796E-2,3.3710003E-2,3.532484E-2,0E0,6.3732624E-2,0E0,0E0,8.0796E-2,1.6504192E-1,0E0,0E0,5.820383E-2,6.2363863E-2,6.2739775E-2,3.3556515E-1,8.864807E-2,0E0,5.889103E-2,2.7631111E-2,1.4222875E-1,7.218829E-2,1.0585451E-1,0E0,1.5518844E-1,0E0,4.3420322E-2,1.265544E-1,0E0,1.7138124E-2,1.0147102E-1,5.213912E-2,0E0,0E0,0E0,0E0,1.7530844E-2,0E0,0E0,3.9709907E-2,0E0,0E0,1.9240392E-2,0E0,0E0,0E0,0E0,0E0,9.808725E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,43,43,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,76,76,77,77,78,78,79,79,80,80,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,104,104,106,106,113,113,114,114,115,115,116,116,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,127,127,130,130,133,133,136,136,137,137,138,138,140,140,143,143,144,144,147,147,148,148,149,149,150,150,151,151,153,153,154,154,155,155,156,156,157,157,159,159,161,161,162,162,164,164,165,165,166,166,171,171,174,174,177,177,183,183],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,50,52,54,56,58,-1,-1,-1,60,62,-1,-1,64,-1,66,68,-1,-1,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,-1,102,104,106,108,110,-1,-1,112,114,116,118,120,-1,-1,-1,122,124,126,-1,-1,128,130,-1,132,134,136,138,140,142,-1,144,146,148,150,-1,152,-1,154,-1,-1,-1,-1,-1,-1,156,158,160,162,-1,164,166,168,170,-1,172,174,176,178,180,-1,-1,182,-1,-1,184,-1,-1,186,188,190,-1,192,-1,-1,194,196,-1,-1,198,200,202,204,206,-1,208,210,212,214,216,-1,218,-1,220,222,-1,224,226,228,-1,-1,-1,-1,230,-1,-1,232,-1,-1,234,-1,-1,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,2.6E1,5.9E1,2.5330253E-2,9.616803E2,3.9E1,3.355864E0,9.48E2,-2.6842918E-2,-2.3116974E-2,-1.0435031E-2,5.6530495E6,3.206931E2,-1.5956992E-2,2.6333334E1,4E0,1.0772152E1,1.137832E-7,2.8971518E6,3.1639907E0,2.6846153E-1,5.932344E1,3.6086957E0,3.25E2,1.3823239E2,-1.1142762E-2,-3.6836233E-2,-6.2297476E-3,8.215373E6,2E0,2.8266037E2,2.437372E6,5.1942E4,2E0,1.3865546E-2,1.6300699E-2,8.400301E-3,1.9705E4,1.5277778E0,-8.551965E-3,2.9365002E-4,1.31306E5,1.5608201E-2,4.39776E6,4.217427E7,-1.8770915E-2,-8.416108E-3,2.732439E2,1.9E1,5.4E1,8.303459E5,3.72381E5,3.64E2,4.284608E6,4.745397E6,4.1032645E6,1.6E1,9.44363E5,3.1152323E-2,3.714144E8,5.620512E8,-1.3954026E-3,-6.8889037E-3,1E0,5.6842103E0,4.7866177E2,-1.39719015E-2,1.1772152E0,1.0909091E0,1.43E2,5.0710382E0,2.1578947E2,-1.783897E-2,1.1126087E-3,1.7352032E1,8.583682E4,1.048796E6,1.4979E4,5.7420593E2,3.80353E-3,-1.6813305E-3,1.0851359E-3,2.1111112E0,4.8709216E5,3.0977E4,1.4128158E-4,-6.5795444E-3,1.2562814E0,5.2141205E6,3.4678753E-3,7.659164E1,1.28636E7,1E0,2.590909E0,6.642857E0,7.2140925E5,-7.7713244E-3,1.9595902E3,6.743651E7,7.09E2,7.9634375E2,-5.059808E-3,3.7029E4,-1.173335E-2,4.629112E6,-1.16590075E-2,-5.06098E-3,-1.0104442E-2,-2.8673087E-3,-8.059999E-4,-5.1009133E-3,3.693316E7,8.695652E0,9.706078E4,3.1226995E0,2.059346E-3,5.4674416E7,6.4E1,4.857143E0,7E0,1.363455E-2,2.1340163E8,1.3333334E0,6.978874E5,6.624E3,1.0609756E0,2.0024195E-2,3.3940226E-4,2.5205562E5,1.0049854E-2,2.06835E-2,1.9407146E7,-6.457043E-3,2.5187032E-3,2.4344263E0,6.44E2,3.62E2,-6.3519846E-3,4.39E2,-3.810116E-3,-1.8515399E-4,9.6E1,4.76E2,8.5797915E-3,-1.87239E-3,5.57E2,1.994E3,1.1117E4,2.8302418E10,1E0,9.143933E-3,3.440772E7,2.08E2,2.3257812E2,2.3818183E0,1.609E3,-1.7005035E-3,1.373E3,-2.6820421E-2,1.071E3,1.058E3,1.8103935E-3,8.7578294E5,2.923409E7,8.7692904E7,9.999024E-3,2.9828758E-3,9.143727E-3,3.5519006E-3,1.5E1,1.0105338E-3,9.627098E-3,2.0394794E8,5.269908E-3,1.091391E-2,4.771E3,7.678993E-3,1.4638257E-2,6.860326E-3,5.6368955E-3,1.5483941E-2,2E0,-5.0777383E-4,8.077706E-3,1.46527635E-2,-1.1176726E-2,-5.7484065E-3,-6.5746172E-3,-8.746909E-4,-1.8007085E-2,-9.23389E-3,-4.8850356E-3,-1.2704869E-2,-2.1737895E-3,-9.500405E-3,1.6851719E-4,4.5868917E-3,-5.4463306E-3,1.0696977E-3,-1.7372688E-3,5.013853E-3,-3.3590717E-3,-1.5900875E-2,2.7876692E-3,-2.6236782E-3,-1.9486265E-3,-6.2377723E-3,-2.3766218E-3,2.4804354E-3,-1.2117583E-2,-2.527635E-3,-2.295875E-3,3.259657E-3,4.09689E-3,1.1639896E-2,-8.153879E-3,1.467654E-3,-5.6455034E-4,4.4573303E-3,-7.0261434E-3,-1.6735867E-4,3.938377E-3,9.00426E-3,2.5920656E-3,-3.2891848E-3,-1.8424853E-3,-8.389362E-3,-2.5470501E-3,-6.6360096E-3,4.0666E-3,-1.5884526E-3,3.1150458E-3,-6.5819285E-4,1.2904538E-2,5.1943446E-3],"split_indices":[2,43,17,3,3,53,67,2,73,2,0,0,0,43,73,0,62,8,73,52,43,57,53,73,69,2,67,0,0,0,12,32,4,9,1,6,0,0,0,2,68,0,0,9,0,9,60,0,0,67,8,0,43,9,10,43,60,60,3,9,0,7,7,0,0,8,71,67,0,68,68,2,69,70,0,0,73,48,9,1,67,0,0,0,68,43,44,0,0,68,60,0,73,60,14,68,73,60,0,48,7,2,48,0,1,0,9,0,0,0,0,0,0,7,61,48,69,0,7,8,69,8,0,7,68,43,2,68,0,0,48,0,0,9,0,0,69,1,1,0,2,0,0,10,0,0,0,10,12,1,46,19,0,7,0,48,68,2,0,2,0,2,10,0,60,9,5,0,0,0,0,3,0,0,7,0,0,2,0,0,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.026E3,7.7E2,2.56E2,2.38E2,5.32E2,1.3E1,2.43E2,2.08E2,3E1,5.28E2,4E0,5E0,8E0,9.8E1,1.45E2,3.1E1,1.77E2,1E1,2E1,2.73E2,2.55E2,8.6E1,1.2E1,1.37E2,8E0,1.8E1,1.59E2,5E0,5E0,8E0,1.2E1,2.36E2,3.7E1,2.1E2,4.5E1,8E1,6E0,6E0,6E0,5.7E1,8E1,4E0,4E0,1.2E1,6E0,7.2E1,8.7E1,7E0,5E0,1.79E2,5.7E1,2.5E1,1.2E1,1.17E2,9.3E1,1.5E1,3E1,6.8E1,1.2E1,4.8E1,9E0,2.7E1,5.3E1,7E0,5E0,5.5E1,1.7E1,7.6E1,1.1E1,1.25E2,5.4E1,4.1E1,1.6E1,1.9E1,6E0,4E0,8E0,6E1,5.7E1,3.5E1,5.8E1,9E0,6E0,4E0,2.6E1,3.7E1,3.1E1,5E0,7E0,3.3E1,1.5E1,4E0,2.3E1,2.9E1,2.4E1,2.7E1,2.8E1,1.3E1,4E0,6.5E1,1.1E1,4.1E1,8.4E1,6E0,4.8E1,4E0,3.7E1,9E0,7E0,1.2E1,7E0,4E0,4E0,3.4E1,2.6E1,1.7E1,4E1,7E0,2.8E1,4.7E1,1.1E1,1.9E1,7E0,2.4E1,1.3E1,1.7E1,1.4E1,1.1E1,2.2E1,5E0,1E1,6E0,1.7E1,2.5E1,4E0,4E0,2E1,1.8E1,9E0,5E0,2.3E1,9E0,4E0,1.5E1,5E1,7E0,4E0,3E1,1.1E1,1.1E1,7.3E1,4.3E1,5E0,2.9E1,8E0,1.6E1,1.8E1,2E1,6E0,1.3E1,4E0,1.6E1,2.4E1,4E0,2.4E1,3.6E1,1.1E1,6E0,5E0,1.3E1,6E0,1.2E1,1.2E1,4E0,9E0,5E0,1.2E1,1E1,4E0,7E0,4E0,4E0,6E0,2.1E1,4E0,6E0,1.4E1,9E0,9E0,4E0,5E0,1.7E1,6E0,7E0,8E0,4.3E1,7E0,2.2E1,8E0,5E0,6E0,4E0,7E0,6.9E1,4E0,3.6E1,7E0,1.5E1,1.4E1,4E0,4E0,4E0,1.2E1,9E0,9E0,8E0,1.2E1,7E0,6E0,5E0,1.1E1,8E0,1.6E1,4E0,2E1,2.9E1,7E0,7E0,4E0,7E0,5E0,5E0,4E0,5E0,5E0,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[8.72172E-3,-5.2378815E-2,1.8862124E-1,-1.4497288E-1,2.8107746E-2,2.2104456E-1,-4.3223172E-2,-2.8720894E-1,-1.0349912E-1,-5.3254906E-2,5.86618E-2,8.073839E-2,2.7825445E-1,-2.4189971E-1,8.807774E-2,-2.4866153E-1,-2.9690595E-2,-1.7638516E-1,-4.863835E-2,-9.128427E-2,8.682048E-2,4.8905227E-2,1.303921E-2,3.1544287E-2,2.4589539E-1,5.186504E-1,2.4941117E-1,-3.5997012E-3,-1.4142607E-2,1.5876757E-2,2.7034823E-3,-1.761185E-2,-2.2367401E-1,-1.62861E-1,-2.0514648E-2,2.0292308E-2,-7.1081586E-2,-2.1198758E-1,-4.1937176E-2,4.346774E-2,9.684896E-3,-1.4100128E-1,5.9390966E-2,1.0537662E-1,-4.0721823E-2,4.8275758E-3,1.4143654E-2,1.3774401E-2,3.0353699E-2,2.1816213E-2,2.1961127E-1,-5.3911596E-3,3.973649E-3,-1.5521282E-1,-2.4970925E-1,-1.4594304E-2,-1.430656E-1,-7.994017E-3,-1.3078807E-1,-8.0597945E-2,-3.4760985E-1,5.8276732E-3,-6.1842825E-2,3.9057387E-3,-4.0861764E-3,-5.4247882E-2,-1.40287345E-2,6.439775E-2,-9.971807E-3,-1.0422347E-3,1.5369838E-1,2.283647E-3,-1.0644382E-2,2.4693705E-1,9.715158E-2,-1.4571238E-3,-8.978063E-3,-1.2164489E-2,-4.2037857E-3,-8.9428425E-2,-2.0278858E-1,5.5312263E-3,-2.3124851E-2,-9.876157E-2,-2.8085992E-1,-5.663785E-3,9.016345E-4,-2.270514E-2,-8.068586E-3,-2.459085E-2,-1.1584179E-1,8.762837E-4,-4.564232E-3,4.2571846E-2,1.850129E-1,2.9195794E-3,8.805893E-3,4.2759046E-2,-4.4508157E-3,2.1421364E-1,4.018031E-1,1.703014E-2,1.22557245E-2,4.4476306E-3,-1.0588076E-1,-1.3681443E-1,-2.779559E-1,3.590126E-2,-6.642312E-2,-1.7129269E-1,-6.507932E-2,-1.5911153E-2,-8.186132E-3,-4.507132E-2,2.9461032E-3,-9.514544E-3,-5.4056033E-2,2.4281483E-2,1.2361709E-1,2.6249638E-1,8.5557684E-2,4.2141047E-3,-2.826333E-4,1.0423103E-1,2.4100588E-1,2.1852339E-2,1.0741991E-2,6.1002336E-3,-8.818392E-2,-2.001791E-1,-7.799874E-2,-1.7800367E-1,-7.733494E-4,-4.945717E-3,-3.118713E-1,5.4359782E-2,-2.9369479E-3,-1.3686177E-1,-5.772263E-3,-1.8939708E-1,-3.654349E-3,-3.6622614E-2,-7.570231E-3,-6.185178E-2,1.3657555E-3,-7.4566645E-3,1.6373991E-3,6.5352987E-3,1.1890642E-1,1.5681271E-1,-1.1835407E-2,1.6850555E-2,2.0135957E-1,-8.26217E-5,6.313497E-3,1.639012E-3,-1.9929293E-3,4.280685E-2,7.74868E-3,2.532417E-1,3.1546345E-3,2.6112532E-3,-8.966066E-3,-2.8425588E-3,-1.2502305E-2,-2.5173726E-3,-6.3366937E-3,-3.0830633E-3,-9.316043E-3,-1.608188E-2,-7.1748444E-3,8.384456E-4,3.765922E-3,-1.7980152E-3,-8.435483E-3,2.7885148E-3,-1.8783283E-3,-1.0050917E-2,-3.877687E-3,-6.5479623E-3,-3.4059043E-4,-1.1249786E-3,-3.8220037E-3,-6.0070277E-4,2.7662155E-3,6.9019375E-3,8.042589E-5,5.2760583E-3,9.539327E-3,2.090719E-3,-3.074931E-3,1.1207516E-2,5.9967865E-3,4.18515E-3,-1.5084115E-3,5.0713434E-3,1.2412042E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,43,45,47,49,-1,-1,-1,51,-1,53,55,-1,-1,57,59,61,63,-1,65,67,69,71,-1,-1,-1,-1,-1,73,-1,-1,75,77,-1,79,81,83,85,87,-1,89,-1,-1,91,-1,93,-1,-1,95,97,-1,99,101,-1,-1,-1,-1,103,105,-1,107,109,111,-1,-1,-1,-1,113,115,-1,-1,117,119,-1,-1,121,-1,123,125,127,-1,-1,129,131,133,135,137,139,141,-1,-1,143,-1,-1,145,147,149,151,153,-1,155,157,159,-1,-1,-1,161,163,165,167,-1,-1,169,171,-1,173,175,177,-1,179,-1,181,-1,-1,-1,183,185,187,189,-1,191,-1,-1,-1,-1,193,-1,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.101283E1,5.578603E0,1.9189138E0,2.0231476E0,9.9999356E-1,1.7675323E0,8.560711E-1,9.0894794E-1,1.0750372E0,5.9389293E-1,6.2315285E-1,5.2847695E-1,9.6618366E-1,1.1305791E-1,4.354735E-1,1.5321207E-1,0E0,3.475933E-1,1.7393423E0,5.093129E-1,1.1867106E-1,5.679323E-1,0E0,2.8174105E-1,9.5800996E-2,2.9226875E-1,8.4577847E-1,0E0,0E0,0E0,1.691855E-1,0E0,7.353306E-2,2.941072E-1,0E0,0E0,5.639649E-1,4.1746747E-1,2.1858262E-1,1.10424176E-1,0E0,1.951932E-1,3.8183808E-1,1.6420215E-1,2.2077356E-1,0E0,0E0,0E0,0E0,0E0,4.0433693E-1,0E0,0E0,9.0636015E-2,7.0994616E-2,0E0,3.0540562E-1,1.457744E-1,3.4234512E-1,6.1256036E-2,2.1274018E-1,0E0,1.1181073E-1,0E0,0E0,3.7750762E-2,0E0,6.844804E-1,0E0,0E0,5.142185E-2,9.593152E-2,0E0,4.4097376E-1,3.322513E-1,0E0,0E0,0E0,0E0,1.7822868E-1,1.9504929E-1,0E0,1.7361847E-1,1.476481E-1,2.4611235E-2,0E0,0E0,0E0,0E0,6.6212535E-2,1.1691299E-1,0E0,0E0,3.2904413E-1,2.8578675E-1,0E0,0E0,3.548234E-2,0E0,2.341876E-1,9.4928026E-2,2.2946034E-1,0E0,0E0,1.1847687E-1,1.3055676E-1,8.833504E-2,5.6703277E-2,1.658244E-1,1.840043E-2,1.23305246E-1,0E0,0E0,3.8147766E-2,0E0,0E0,1.4983103E-1,3.0820155E-1,1.8620956E-1,6.407273E-2,8.5050195E-2,0E0,1.6855368E-2,6.69201E-2,1.3064575E-1,0E0,0E0,0E0,1.6419947E-1,9.077996E-2,4.9550608E-2,3.9165437E-2,0E0,0E0,4.23671E-2,2.3103848E-2,0E0,7.351133E-2,5.32076E-2,2.9256344E-2,0E0,1.1271305E-1,0E0,1.6775578E-2,0E0,0E0,0E0,1.6335548E-1,1.03336334E-1,4.6262145E-2,3.1129247E-2,0E0,1.5717804E-2,0E0,0E0,0E0,0E0,4.2618744E-2,0E0,1.1433411E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,30,30,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,50,50,53,53,54,54,56,56,57,57,58,58,59,59,60,60,62,62,65,65,67,67,70,70,71,71,73,73,74,74,79,79,80,80,82,82,83,83,84,84,89,89,90,90,93,93,94,94,97,97,99,99,100,100,101,101,104,104,105,105,106,106,107,107,108,108,109,109,110,110,113,113,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,128,128,129,129,130,130,131,131,134,134,135,135,137,137,138,138,139,139,141,141,143,143,147,147,148,148,149,149,150,150,152,152,157,157,159,159],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,44,46,48,50,-1,-1,-1,52,-1,54,56,-1,-1,58,60,62,64,-1,66,68,70,72,-1,-1,-1,-1,-1,74,-1,-1,76,78,-1,80,82,84,86,88,-1,90,-1,-1,92,-1,94,-1,-1,96,98,-1,100,102,-1,-1,-1,-1,104,106,-1,108,110,112,-1,-1,-1,-1,114,116,-1,-1,118,120,-1,-1,122,-1,124,126,128,-1,-1,130,132,134,136,138,140,142,-1,-1,144,-1,-1,146,148,150,152,154,-1,156,158,160,-1,-1,-1,162,164,166,168,-1,-1,170,172,-1,174,176,178,-1,180,-1,182,-1,-1,-1,184,186,188,190,-1,192,-1,-1,-1,-1,194,-1,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.414E3,4.539777E5,2.856934E7,7.7E1,1E0,8.625455E2,1.3655363E2,1.05039425E1,5.41E2,6.33E2,7.82261E6,3.5510652E6,1.059448E7,3.6451373E8,1.2083E4,9.2058825E-1,-2.9690595E-2,3.1E1,2.00087E5,1.9750606E2,9.50283E5,1.6262903E2,1.303921E-2,1.5382514E1,5.2692295E6,5.467E3,7.714286E0,-3.5997012E-3,-1.4142607E-2,1.5876757E-2,1.01E3,-1.761185E-2,3.5675676E0,5.4E1,-2.0514648E-2,2.0292308E-2,4.064E3,1.0526316E0,1E1,2.5818555E2,9.684896E-3,1.5222145E2,2.3544883E8,1.1656439E9,1E0,4.8275758E-3,1.4143654E-2,1.3774401E-2,3.0353699E-2,2.1816213E-2,1.2792593E2,-5.3911596E-3,3.973649E-3,1.2E1,3E0,-1.4594304E-2,8.960784E0,3.2931E4,4.9735293E0,4.39E2,5.4E0,5.8276732E-3,5.524E3,3.9057387E-3,-4.0861764E-3,2.4481E4,-1.40287345E-2,5.817547E2,-9.971807E-3,-1.0422347E-3,1.29443414E5,6.0641945E1,-1.0644382E-2,4.73909E0,1.5E1,-1.4571238E-3,-8.978063E-3,-1.2164489E-2,-4.2037857E-3,7.3333335E-1,4.612245E0,5.5312263E-3,1E0,8.75E2,7.5391846E2,-5.663785E-3,9.016345E-4,-2.270514E-2,-8.068586E-3,1.535E3,1.5E1,8.762837E-4,-4.564232E-3,8.237181E9,3.8275862E0,2.9195794E-3,8.805893E-3,4.6816683E-1,-4.4508157E-3,1.000501E6,1.7655972E0,2.0277777E0,1.22557245E-2,4.4476306E-3,1E0,8E0,4.7721977E4,1E0,8.83425E7,2.026453E6,2.2587704E7,-1.5911153E-2,-8.186132E-3,1.28636E7,2.9461032E-3,-9.514544E-3,1.2218E4,1.427636E5,2.9411764E0,1.500502E6,1.98E2,4.2141047E-3,1.1E1,7.607E3,3.0546486E8,2.1852339E-2,1.0741991E-2,6.1002336E-3,1.6677892E2,1.1E1,3.373828E-1,2.992648E-2,-7.733494E-4,-4.945717E-3,1.9470909E2,1.2748E4,-2.9369479E-3,6.76E2,1.75E0,2.3847478E5,-3.654349E-3,2.4177586E2,-7.570231E-3,1.4E1,1.3657555E-3,-7.4566645E-3,1.6373991E-3,9.750085E6,2.2681375E6,1.7903225E0,3.3E1,1.6850555E-2,6.726722E2,-8.26217E-5,6.313497E-3,1.639012E-3,-1.9929293E-3,3.019898E8,7.74868E-3,1.0089981E3,3.1546345E-3,2.6112532E-3,-8.966066E-3,-2.8425588E-3,-1.2502305E-2,-2.5173726E-3,-6.3366937E-3,-3.0830633E-3,-9.316043E-3,-1.608188E-2,-7.1748444E-3,8.384456E-4,3.765922E-3,-1.7980152E-3,-8.435483E-3,2.7885148E-3,-1.8783283E-3,-1.0050917E-2,-3.877687E-3,-6.5479623E-3,-3.4059043E-4,-1.1249786E-3,-3.8220037E-3,-6.0070277E-4,2.7662155E-3,6.9019375E-3,8.042589E-5,5.2760583E-3,9.539327E-3,2.090719E-3,-3.074931E-3,1.1207516E-2,5.9967865E-3,4.18515E-3,-1.5084115E-3,5.0713434E-3,1.2412042E-2],"split_indices":[2,43,60,44,29,67,71,69,2,0,43,43,12,7,9,68,0,8,5,70,1,4,0,71,60,2,71,0,0,0,8,0,69,2,0,0,44,71,3,4,0,4,12,46,6,0,0,0,0,0,73,0,0,3,8,0,73,9,68,2,73,0,44,0,0,12,0,67,0,0,48,73,0,50,3,0,0,0,0,68,69,0,26,2,70,0,0,0,0,2,3,0,0,46,69,0,0,53,0,43,49,68,0,0,13,8,43,19,7,9,12,0,0,60,0,0,44,48,68,9,10,0,3,2,47,0,0,0,73,3,53,53,0,0,4,1,0,2,68,43,0,4,0,3,0,0,0,9,43,68,8,0,4,0,0,0,0,7,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.47E2,2.53E2,3.47E2,4E2,2.22E2,3.1E1,7.7E1,2.7E2,1.09E2,2.91E2,6.5E1,1.57E2,1.2E1,1.9E1,7.1E1,6E0,1.15E2,1.55E2,8.6E1,2.3E1,2.8E2,1.1E1,5.1E1,1.4E1,1.5E1,1.42E2,4E0,8E0,4E0,1.5E1,9E0,6.2E1,1.11E2,4E0,6E0,1.49E2,2.4E1,6.2E1,1.8E1,5E0,1.4E1,2.66E2,2.5E1,2.6E1,5E0,9E0,7E0,8E0,1.5E1,1.27E2,6E0,9E0,1.9E1,4.3E1,1.1E1,1E2,7.3E1,7.6E1,1.3E1,1.1E1,6E0,5.6E1,1.4E1,4E0,1E1,4E0,2.62E2,4E0,7E0,1.8E1,2.2E1,4E0,1.03E2,2.4E1,5E0,1.4E1,3.9E1,4E0,5.4E1,4.6E1,7E0,6.6E1,6.4E1,1.2E1,9E0,4E0,5E0,6E0,3.4E1,2.2E1,4E0,6E0,2.23E2,3.9E1,6E0,1.2E1,1.6E1,6E0,8.7E1,1.6E1,1.7E1,7E0,4E0,5E1,2.6E1,2E1,2.8E1,3.8E1,1.9E1,4.5E1,6E0,6E0,2.8E1,6E0,8E0,1.4E1,1.83E2,4E1,2.1E1,1.8E1,7E0,9E0,1.8E1,6.9E1,1E1,6E0,8E0,9E0,1E1,4E1,1.9E1,7E0,4E0,1.6E1,2.4E1,4E0,1.7E1,2.1E1,1.5E1,4E0,3.6E1,9E0,2.3E1,5E0,6E0,8E0,1.55E2,2.8E1,3.2E1,8E0,6E0,1.5E1,7E0,1.1E1,5E0,4E0,1E1,8E0,6.4E1,5E0,4E0,5E0,4E0,6E0,3E1,1E1,4E0,1.5E1,1.2E1,4E0,1.1E1,1.3E1,6E0,1.1E1,7E0,1.4E1,1.1E1,4E0,7E0,2.9E1,9E0,1.4E1,1.14E2,4.1E1,2.2E1,6E0,1.9E1,1.3E1,4E0,4E0,8E0,7E0,6E0,4E0,7E0,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-6.840664E-3,-7.564314E-2,1.9535907E-1,-1.9300523E-1,-2.1521175E-2,-3.0387625E-1,2.1447061E-1,-1.6691664E-1,-3.4692943E-1,-8.0028E-2,2.1416238E-2,-1.721789E-2,-7.416122E-3,5.991051E-1,1.8573333E-1,-2.0868088E-1,-8.339015E-2,-4.9642673E-1,-2.4282059E-1,-5.5640046E-2,-2.6173565E-1,1.0791406E-2,1.7121804E-1,1.8752169E-2,3.5375725E-2,6.153804E-2,2.328578E-1,-1.9223677E-1,-1.9543128E-2,1.2934754E-2,-1.2123926E-1,-1.0904279E-2,-3.5152256E-2,-1.9409508E-3,-2.851105E-1,-1.4052862E-1,-1.5613228E-2,-1.3557872E-1,-2.7016891E-2,-2.4180397E-2,6.0915094E-2,6.5613404E-2,1.1626313E-2,3.2972004E-2,9.303139E-3,2.5004095E-1,-9.377705E-3,-2.119346E-1,-7.990358E-2,-2.077344E-1,-3.6266204E-2,-1.5910942E-2,-6.5945326E-3,-1.8337767E-1,-9.328806E-2,1.5711425E-2,-8.075538E-2,-9.197322E-3,-5.216909E-2,-1.1941902E-1,-7.477994E-3,8.0042176E-2,-1.517489E-1,-5.2269764E-4,5.4877093E-3,1.5570532E-1,6.157E-3,2.2106372E-1,4.368353E-1,-2.1859212E-1,-1.3124169E-3,-1.2622821E-2,-8.669574E-3,-2.4108933E-1,-2.5025285E-3,1.6260386E-2,-1.4757274E-1,-3.908957E-3,-2.3108338E-1,-4.4649086E-4,-1.3085946E-1,-1.741995E-2,6.118278E-2,-1.1001991E-1,6.4516813E-3,-5.695077E-3,2.0137683E-3,-2.7138608E-2,-2.8045768E-1,9.3885595E-3,-1.0671488E-1,1.6980128E-1,3.4680944E-2,-1.2606748E-2,-1.1788013E-3,1.3482728E-3,1.0865502E-2,3.5373196E-2,-9.267539E-2,2.7034536E-1,1.5341043E-1,1.0501004E-2,2.1841403E-2,-2.0876442E-1,-1.7341226E-2,1.9915015E-3,-2.7630979E-3,-1.7403053E-1,-1.408249E-2,-2.2346012E-2,5.0646264E-2,-9.9995565E-3,-3.571021E-3,-1.3162834E-2,-1.562435E-1,-3.3883373E-3,-7.0597418E-3,5.0022574E-3,-3.538772E-2,2.5663525E-4,1.1904978E-1,-8.802719E-3,-3.23352E-2,1.7780726E-3,-2.1295685E-3,1.3409492E-3,-7.749313E-3,-1.9454127E-2,-3.868254E-3,5.086483E-4,8.497947E-3,-1.9072951E-1,-4.0342066E-2,1.0185063E-1,2.6930583E-1,8.930754E-2,-8.0569506E-2,7.236593E-2,-5.8642894E-2,1.1766521E-3,-9.026096E-3,1.7983173E-1,3.1910643E-1,1.029651E-1,3.3951056E-1,-1.3897629E-2,-1.907007E-1,-1.6402863E-3,-1.0218113E-2,7.861163E-4,-2.4435818E-3,4.6941247E-3,2.8159659E-4,-1.0388775E-2,-2.8447777E-3,-6.0307216E-2,7.929756E-2,-3.3707142E-2,5.030587E-3,7.5726593E-3,5.6892954E-4,6.3205147E-3,-6.1954847E-3,-7.278001E-3,8.822312E-3,-4.1713733E-3,-1.1689367E-2,8.8830624E-4,-3.185609E-3,1.4517672E-1,8.4653875E-4,1.6219297E-2,7.581044E-3,3.3748668E-2,1.530615E-1,1.1560535E-2,-2.000307E-2,3.919705E-2,5.767181E-3,-3.920738E-4,-5.4114377E-3,-3.4258162E-5,2.3252983E-1,2.515274E-1,3.9982662E-1,-2.1985779E-2,1.3787624E-1,8.8514695E-3,1.884963E-2,-1.124831E-2,-7.3551587E-3,-4.0056063E-3,1.2728353E-3,6.5676756E-3,2.6573543E-5,-3.5093222E-3,3.5097462E-4,3.1461755E-3,-1.1642853E-3,6.547251E-5,6.554649E-3,2.2026922E-3,8.144722E-3,-1.2287642E-3,3.7268945E-3,8.080138E-3,2.866218E-3,2.8895813E-3,-5.443667E-3,-1.6372912E-3,2.9130038E-3,1.332312E-2,2.5725174E-3,1.2843513E-2,7.203258E-3,2.037287E-2,1.0006106E-2,-6.540608E-3,5.5992655E-3,1.036429E-2,3.2748797E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,-1,-1,-1,51,53,55,57,-1,59,61,63,-1,65,-1,67,-1,69,71,73,75,-1,-1,77,79,81,83,-1,85,87,89,91,93,-1,-1,95,97,99,101,103,-1,105,-1,107,-1,109,111,-1,113,-1,115,117,119,121,123,-1,-1,125,127,129,131,133,135,-1,-1,-1,-1,137,139,141,143,-1,-1,145,-1,-1,-1,147,-1,149,151,-1,-1,-1,153,-1,-1,-1,155,157,159,-1,161,-1,-1,-1,-1,-1,-1,163,-1,165,167,169,171,173,175,177,179,-1,-1,181,183,185,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,191,193,195,-1,-1,-1,197,-1,-1,199,-1,-1,-1,-1,201,-1,-1,-1,203,205,207,-1,209,-1,-1,-1,-1,211,213,215,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4246592E1,4.8486733E0,2.5458717E0,9.098673E-1,1.3180709E0,3.8817823E-2,2.6631165E0,7.081895E-1,4.0863276E-1,9.6922505E-1,4.8041293E-1,0E0,0E0,2.364521E-1,1.3610392E0,4.0507698E-1,1.010421E0,7.156551E-1,1.7484891E-1,6.671763E-1,9.8036885E-1,4.9930102E-1,1.5338236E-1,0E0,0E0,2.569952E-1,1.3239107E0,2.7071905E-1,0E0,0E0,4.7020048E-1,0E0,0E0,0E0,1.0277665E-1,1.1041856E-1,2.7674374E-1,9.554705E-2,0E0,2.663469E-1,4.880603E-1,4.224673E-2,0E0,1.8334258E-1,0E0,7.9112816E-1,0E0,1.2819242E-1,1.4987557E-1,1.508503E-1,2.011824E-1,0E0,0E0,1.3122296E-1,9.9274665E-2,1.3966641E-1,1.1300105E-1,0E0,7.346066E-2,3.6390662E-1,2.4202682E-1,4.3384224E-1,1.4594364E-1,0E0,0E0,9.369987E-2,1.3899104E-1,4.3863106E-1,5.0651073E-2,8.494806E-2,0E0,3.939328E-2,0E0,5.8071494E-2,0E0,3.29921E-2,3.651063E-2,0E0,4.0088654E-2,0E0,1.6570061E-2,1.2578955E-1,1.3746837E-1,2.0025715E-1,2.1974608E-2,0E0,0E0,1.3911098E-1,2.1374452E-1,1.9210884E-1,1.09407604E-1,2.1598136E-1,4.6678448E-1,0E0,0E0,0E0,0E0,1.329265E-1,1.3844222E-1,3.045249E-1,5.56479E-1,0E0,0E0,1.1224699E-1,0E0,0E0,0E0,9.0109795E-2,0E0,1.5328394E-2,2.8894953E-2,0E0,0E0,0E0,5.7616442E-2,0E0,0E0,0E0,1.4196138E-1,7.7461936E-2,9.375256E-2,0E0,7.165874E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5774725E-1,0E0,3.346306E-2,2.3483142E-2,8.057438E-2,6.2831044E-2,1.719392E-1,7.842184E-1,4.209459E-2,3.12761E-2,0E0,0E0,2.9516435E-1,1.8309212E-1,2.2550237E-1,5.3544402E-2,0E0,1.07503414E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.515312E-2,4.4456627E-2,2.8501181E-2,0E0,0E0,0E0,2.898022E-2,0E0,0E0,1.1212287E-1,0E0,0E0,0E0,0E0,3.7106156E-2,0E0,0E0,0E0,8.1428446E-2,3.4864247E-2,1.393122E-1,0E0,3.4284696E-2,0E0,0E0,0E0,0E0,2.2100937E-1,3.512764E-2,7.039118E-2,2.2263767E-1,2.157852E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,30,30,34,34,35,35,36,36,37,37,39,39,40,40,41,41,43,43,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,68,68,69,69,71,71,73,73,75,75,76,76,78,78,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,97,97,98,98,99,99,100,100,103,103,107,107,109,109,110,110,114,114,118,118,119,119,120,120,122,122,129,129,131,131,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144,146,146,155,155,156,156,157,157,161,161,164,164,169,169,173,173,174,174,175,175,177,177,182,182,183,183,184,184,185,185,186,186],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,-1,-1,-1,52,54,56,58,-1,60,62,64,-1,66,-1,68,-1,70,72,74,76,-1,-1,78,80,82,84,-1,86,88,90,92,94,-1,-1,96,98,100,102,104,-1,106,-1,108,-1,110,112,-1,114,-1,116,118,120,122,124,-1,-1,126,128,130,132,134,136,-1,-1,-1,-1,138,140,142,144,-1,-1,146,-1,-1,-1,148,-1,150,152,-1,-1,-1,154,-1,-1,-1,156,158,160,-1,162,-1,-1,-1,-1,-1,-1,164,-1,166,168,170,172,174,176,178,180,-1,-1,182,184,186,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,192,194,196,-1,-1,-1,198,-1,-1,200,-1,-1,-1,-1,202,-1,-1,-1,204,206,208,-1,210,-1,-1,-1,-1,212,214,216,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.972052E5,1E0,2.6E1,7.785302E5,8.76E2,4.70127E5,7.24E2,3.355864E0,2.1E1,6.9307615E6,-1.721789E-2,-7.416122E-3,1.94261E5,8.399457E2,3.9E1,2.7E1,3.292496E-2,7.6255396E2,5.51E2,6.0692043E0,1.158E3,1E0,1.8752169E-2,3.5375725E-2,3.5510652E6,7.134432E7,9.566785E5,-1.9543128E-2,1.2934754E-2,1.4912975E-1,-1.0904279E-2,-3.5152256E-2,-1.9409508E-3,7.919006E7,1.561132E2,1.9E1,3.9E2,-2.7016891E-2,1.5961905E2,4.7220547E1,9.51E3,1.1626313E-2,3.414E3,9.303139E-3,6.663214E6,-9.377705E-3,1.317484E1,2.5493E4,1.7302156E5,3.38E2,-1.5910942E-2,-6.5945326E-3,1.001791E6,9.144571E5,5.5356906E4,1.862E3,-9.197322E-3,2.5E1,4.5418963E-4,6.346204E-7,2.308943E0,1.5714285E0,-5.2269764E-4,5.4877093E-3,3.3193566E5,2.115238E5,2.9308079E1,3.97371E5,4.233978E0,-1.3124169E-3,1.9903773E2,-8.669574E-3,6.0893228E7,-2.5025285E-3,6.0875E4,5.25E0,-3.908957E-3,5.7E1,-4.4649086E-4,2.087E2,2.6E1,9.61E2,3.011152E6,3.62349E5,-5.695077E-3,2.0137683E-3,7.89E2,7.3809524E0,3.2E1,5.5356906E4,1.28738E5,1.2629019E8,-1.2606748E-2,-1.1788013E-3,1.3482728E-3,1.0865502E-2,6.3055553E0,1E1,1.1772152E0,3.8537518E2,1.0501004E-2,2.1841403E-2,8.0833334E-1,-1.7341226E-2,1.9915015E-3,-2.7630979E-3,1.9166666E0,-1.408249E-2,2.29E2,8.910034E0,-9.9995565E-3,-3.571021E-3,-1.3162834E-2,1.1430505E3,-3.3883373E-3,-7.0597418E-3,5.0022574E-3,1.06314E5,1.4381613E7,1.8729467E8,-8.802719E-3,3.1316226E0,1.7780726E-3,-2.1295685E-3,1.3409492E-3,-7.749313E-3,-1.9454127E-2,-3.868254E-3,1.5881818E2,8.497947E-3,3.1E1,3.3272727E0,9.94E2,6E0,5.372237E2,2.364E3,4.924E3,2.5503825E6,1.1766521E-3,-9.026096E-3,3.714144E8,6.355E3,3.2608695E0,4.907764E3,-1.3897629E-2,1.0891155E4,-1.6402863E-3,-1.0218113E-2,7.861163E-4,-2.4435818E-3,4.6941247E-3,2.8159659E-4,-1.0388775E-2,-2.8447777E-3,1.3205625E7,5.342944E6,3.011152E6,5.030587E-3,7.5726593E-3,5.6892954E-4,2.99E2,-6.1954847E-3,-7.278001E-3,1.9650501E6,-4.1713733E-3,-1.1689367E-2,8.8830624E-4,-3.185609E-3,1.6446976E3,8.4653875E-4,1.6219297E-2,7.581044E-3,1.1E1,2.7012987E1,1.1962106E-1,-2.000307E-2,1.1E1,5.767181E-3,-3.920738E-4,-5.4114377E-3,-3.4258162E-5,2E0,1.6007428E7,1.853776E7,1.4342732E2,8.710612E1,8.8514695E-3,1.884963E-2,-1.124831E-2,-7.3551587E-3,-4.0056063E-3,1.2728353E-3,6.5676756E-3,2.6573543E-5,-3.5093222E-3,3.5097462E-4,3.1461755E-3,-1.1642853E-3,6.547251E-5,6.554649E-3,2.2026922E-3,8.144722E-3,-1.2287642E-3,3.7268945E-3,8.080138E-3,2.866218E-3,2.8895813E-3,-5.443667E-3,-1.6372912E-3,2.9130038E-3,1.332312E-2,2.5725174E-3,1.2843513E-2,7.203258E-3,2.037287E-2,1.0006106E-2,-6.540608E-3,5.5992655E-3,1.036429E-2,3.2748797E-3],"split_indices":[2,43,17,3,43,0,12,2,73,3,43,0,0,1,67,8,0,73,48,2,68,2,29,0,0,43,59,60,0,0,53,0,0,0,7,4,8,48,0,70,73,44,0,2,0,43,0,69,1,43,0,0,0,5,60,48,2,0,3,53,52,69,69,0,0,43,48,71,1,68,0,4,0,5,0,1,69,0,0,0,4,0,2,9,1,0,0,2,73,3,48,1,5,0,0,0,0,69,3,68,71,0,0,68,0,0,0,68,0,10,71,0,0,0,48,0,0,0,1,5,7,0,68,0,0,0,0,0,0,48,0,0,69,10,8,4,2,2,47,0,0,7,2,69,4,0,43,0,0,0,0,0,0,0,0,9,60,9,0,0,0,0,0,0,66,0,0,0,0,48,0,0,0,3,73,53,0,8,0,0,0,0,6,66,62,71,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,7.63E2,2.59E2,2.4E2,5.23E2,9E0,2.5E2,2.07E2,3.3E1,2.21E2,3.02E2,5E0,4E0,1.6E1,2.34E2,1.37E2,7E1,1.2E1,2.1E1,1.96E2,2.5E1,2.83E2,1.9E1,9E0,7E0,6.5E1,1.69E2,1.29E2,8E0,6E0,6.4E1,7E0,5E0,4E0,1.7E1,6.2E1,1.34E2,1.9E1,6E0,1.67E2,1.16E2,9E0,1E1,5.5E1,1E1,1.63E2,6E0,1.09E2,2E1,3.1E1,3.3E1,1.1E1,6E0,3.1E1,3.1E1,9.1E1,4.3E1,1E1,9E0,2.4E1,1.43E2,1.07E2,9E0,4E0,5E0,9E0,4.6E1,1.43E2,2E1,1.05E2,4E0,1.3E1,7E0,2.5E1,6E0,2.3E1,1E1,1.1E1,2E1,1E1,2.1E1,5.3E1,3.8E1,3.2E1,1.1E1,5E0,4E0,1.6E1,8E0,1.23E2,2E1,3.5E1,7.2E1,4E0,5E0,4E0,5E0,3.6E1,1E1,8.1E1,6.2E1,4E0,1.6E1,1.01E2,4E0,6E0,7E0,1.4E1,1.1E1,1.1E1,1.2E1,4E0,6E0,1E1,1E1,7E0,1.4E1,6E0,4.7E1,1.9E1,1.9E1,1.5E1,1.7E1,7E0,4E0,1.2E1,4E0,4E0,4E0,1.18E2,5E0,8E0,1.2E1,2.2E1,1.3E1,4.9E1,2.3E1,2.6E1,1E1,5E0,5E0,3E1,5.1E1,5E1,1.2E1,1.4E1,8.7E1,4E0,1E1,5E0,6E0,5E0,7E0,5E0,5E0,3.9E1,8E0,1.5E1,4E0,1.3E1,6E0,1.3E1,4E0,5E0,1.13E2,4E0,4E0,4E0,8E0,1.4E1,8E0,6E0,7E0,2.7E1,2.2E1,1.9E1,4E0,1.7E1,9E0,6E0,4E0,7E0,2.3E1,3E1,2.1E1,1.1E1,3.9E1,5E0,7E0,3E1,5.7E1,3E1,9E0,4E0,4E0,7E0,8E0,4E0,9E0,1.08E2,5E0,4E0,1E1,1.2E1,1.5E1,1.7E1,5E0,1.4E1,5E0,4E0,1.3E1,1.7E1,6E0,2.2E1,8E0,1.6E1,5E0,6E0,5E0,1.6E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-2.8888178E-3,-1.0378601E-1,1.03640884E-1,-1.7752095E-1,-4.3336123E-2,-2.8024974E-1,1.2771633E-1,-1.5946168E-1,-4.2828533E-1,-2.2663085E-2,-2.1770269E-1,-3.737383E-1,-1.400324E-1,7.256638E-2,2.6645783E-1,-2.0144477E-1,-7.7175714E-2,-1.762753E-1,-3.7833456E-2,-1.1101938E-2,-1.3162233E-1,-2.7249513E-2,-1.091418E-1,-1.882733E-2,-9.442346E-3,-2.26086E-3,-9.343133E-3,4.241862E-2,1.814922E-1,3.3945683E-2,2.3605664E-1,-2.6189423E-1,-1.6384618E-1,-5.902484E-2,-1.34531185E-2,-1.0062221E-2,-3.8291423E-3,8.8417195E-3,-1.6937181E-2,-1.0936117E-2,-7.1762465E-2,2.5273997E-3,-1.6103171E-1,5.24376E-2,-2.3033181E-1,1.3372271E-1,2.632323E-1,3.0090496E-1,1.1237551E-1,-1.8231781E-1,-2.9394698E-1,-1.1706685E-1,-2.0916119E-1,3.4688085E-2,-7.7425525E-2,-2.3373311E-2,1.3407475E-1,1.3794715E-4,-9.3332425E-2,-9.410978E-3,-1.2709263E-3,6.12179E-2,-2.0704196E-1,-1.3680625E-2,-4.8654717E-3,8.1075914E-2,2.4052992E-1,7.573041E-3,1.6519506E-2,3.369364E-1,9.153348E-2,4.2898804E-2,2.7307785E-1,-1.0592125E-2,-5.4114023E-3,-1.7695993E-1,-3.335862E-1,-1.3754368E-1,-4.8206348E-2,-2.647402E-3,-2.2166401E-1,6.4088334E-3,-2.2024706E-3,-1.1233871E-1,-1.6010825E-2,-1.1673879E-1,-1.2039128E-2,1.0470365E-3,9.451013E-3,-9.660795E-4,-5.5535776E-3,-7.306476E-3,1.055014E-1,-1.7359061E-2,1.4180505E-4,1.15015484E-1,-1.5274567E-2,4.1195396E-3,1.3099863E-2,1.6887656E-3,3.5382012E-1,-1.1621586E-3,7.3528434E-3,8.465699E-2,-9.989625E-3,6.3697295E-3,1.6504375E-2,-1.1958338E-2,-4.78911E-3,-9.370715E-3,-1.6632874E-2,-3.141064E-4,-1.5272544E-1,2.7652434E-4,-3.8264103E-3,-2.3514044E-1,-3.2117467E-3,-1.4402582E-1,-4.9925137E-2,-3.7213653E-2,3.326454E-3,3.2567137E-3,-1.795748E-1,3.361361E-2,-4.0257577E-2,1.3968155E-2,-1.63348E-2,1.5979484E-1,6.683572E-2,1.5146789E-1,1.3550721E-3,2.5317718E-3,-3.285927E-3,3.7045714E-1,8.346115E-3,-5.6934517E-2,1.3873275E-1,-1.704616E-1,-2.770229E-3,-1.5368074E-1,-2.6456314E-1,-8.46026E-3,-9.743069E-2,1.4720956E-4,-4.4718855E-3,-3.224681E-3,4.7905123E-4,-2.4685125E-1,2.6873744E-3,5.8459133E-2,-4.3845914E-2,-6.0763683E-2,1.2372842E-2,3.586358E-2,-1.3195297E-1,1.19522125E-1,2.7565986E-1,3.6398917E-2,1.5409309E-1,1.7750493E-1,2.2300573E-3,4.180528E-1,2.6491722E-1,2.59703E-3,-7.331607E-3,9.685445E-3,7.351236E-2,-5.6178286E-3,-1.0853061E-2,-8.500792E-3,-3.1583244E-3,-6.0907956E-3,-1.2990744E-2,-5.485076E-4,-6.0274336E-3,-4.691669E-3,-1.535808E-2,7.7819085E-4,4.014255E-3,8.9214253E-4,-3.339842E-3,-4.8270114E-3,-8.9502777E-4,-3.4885944E-3,2.916251E-3,8.705812E-3,1.0193696E-3,-9.336256E-3,9.5539633E-4,7.5618178E-3,1.1169775E-3,8.491481E-3,1.591753E-2,9.2139846E-4,7.552971E-3,9.486959E-3,3.960368E-3,9.777831E-3,3.405903E-3,2.0701755E-2,1.3645905E-2,3.7264237E-3,1.5198209E-2,5.622386E-4,5.097148E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,-1,-1,-1,-1,43,45,-1,47,49,51,53,-1,-1,-1,-1,55,-1,57,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,89,-1,-1,91,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,-1,107,109,111,113,-1,115,-1,-1,117,119,121,123,-1,-1,-1,-1,125,127,-1,-1,129,131,-1,-1,-1,133,-1,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,137,-1,-1,139,-1,141,143,145,-1,-1,147,149,151,153,-1,155,157,159,-1,-1,-1,161,-1,163,165,167,-1,169,171,-1,173,-1,-1,-1,-1,175,-1,177,179,181,183,185,187,189,191,193,195,197,-1,199,201,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1490206E1,2.4393477E0,4.8418083E0,1.0502748E0,1.0872861E0,3.5403466E-1,3.7299428E0,7.8592443E-1,1.430757E0,3.4117547E-1,1.2283708E0,1.9706964E-2,6.943387E-2,1.1482234E0,1.8055296E0,3.0483055E-1,2.9358885E-1,2.4608076E-2,0E0,2.9508433E-1,1.4657792E-1,0E0,2.2986373E-1,0E0,0E0,0E0,0E0,7.6903045E-1,2.6155567E-1,0E0,1.0248938E0,9.1795206E-2,1.8113518E-1,1.3201442E-1,0E0,0E0,0E0,0E0,2.3736948E-1,0E0,2.9300451E-2,0E0,1.0835254E-1,6.232615E-1,5.1064074E-2,2.616517E-1,1.943475E-1,6.179395E-1,5.130435E-1,3.129816E-2,1.2182188E-1,6.38718E-2,7.843256E-2,1.1800276E-1,1.332849E-1,2.4410196E-1,7.3905006E-2,0E0,2.473712E-2,0E0,0E0,7.897406E-1,3.2017112E-1,0E0,0E0,1.1621013E-1,7.890695E-2,0E0,0E0,3.5729122E-1,1.1447441E-1,3.8621104E-1,1.1005843E-1,0E0,0E0,4.5190156E-2,2.3774147E-2,7.438916E-2,2.4883984E-2,0E0,7.386875E-2,0E0,0E0,7.191771E-2,4.746671E-2,3.0678138E-1,2.6930907E-1,0E0,0E0,0E0,0E0,7.637111E-1,3.2100832E-1,0E0,0E0,7.676566E-2,4.3035794E-2,0E0,0E0,0E0,1.3185978E-1,0E0,0E0,2.375987E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.571712E-2,0E0,0E0,5.5212498E-2,0E0,2.9527068E-2,3.7649244E-2,3.173459E-2,0E0,0E0,3.1825638E-1,1.5652499E-1,1.3997442E-1,3.1784278E-1,0E0,2.7035546E-1,2.4403962E-1,3.979683E-2,0E0,0E0,0E0,2.0069122E-1,0E0,1.12548366E-1,8.633819E-2,5.961919E-2,0E0,2.1052778E-2,2.4721503E-2,0E0,3.8021833E-2,0E0,0E0,0E0,0E0,1.5276265E-1,0E0,7.070933E-2,3.732783E-2,1.6626453E-1,1.6979533E-1,1.7690632E-1,1.4334238E-1,2.0654905E-1,3.791082E-2,1.4521608E-1,6.650138E-2,3.4002006E-2,0E0,1.8774033E-2,2.3312235E-1,0E0,0E0,0E0,2.8017692E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,27,27,28,28,30,30,31,31,32,32,33,33,38,38,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,80,80,83,83,84,84,85,85,86,86,91,91,92,92,95,95,96,96,100,100,103,103,112,112,115,115,117,117,118,118,119,119,122,122,123,123,124,124,125,125,127,127,128,128,129,129,133,133,135,135,136,136,137,137,139,139,140,140,142,142,147,147,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,161,161,162,162,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,-1,-1,-1,-1,44,46,-1,48,50,52,54,-1,-1,-1,-1,56,-1,58,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,90,-1,-1,92,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,-1,108,110,112,114,-1,116,-1,-1,118,120,122,124,-1,-1,-1,-1,126,128,-1,-1,130,132,-1,-1,-1,134,-1,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,138,-1,-1,140,-1,142,144,146,-1,-1,148,150,152,154,-1,156,158,160,-1,-1,-1,162,-1,164,166,168,-1,170,172,-1,174,-1,-1,-1,-1,176,-1,178,180,182,184,186,188,190,192,194,196,198,-1,200,202,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,4.97E2,1E0,4.4895835E0,3.1316226E0,1.7540792E5,1.1459359E3,3.51E2,1E0,1E0,1E0,1.3141646E3,1.91E2,2.9652428E6,3.3817584E7,3.89E2,2.1E1,8.5714287E-1,-3.7833456E-2,1.2E1,8.314423E6,-2.7249513E-2,2.857143E0,-1.882733E-2,-9.442346E-3,-2.26086E-3,-9.343133E-3,1.2294118E1,2.593592E6,3.3945683E-2,4.1681065E1,5E1,3.1E1,8.638015E4,-1.34531185E-2,-1.0062221E-2,-3.8291423E-3,8.8417195E-3,7.4711205E6,-1.0936117E-2,3.202146E5,2.5273997E-3,2.0134516E6,1.5449402E7,1.594941E3,1E0,1.81E2,2.2040408E7,4.435876E2,3.5070792E2,2.416E3,2E0,3.3305264E2,2.90099E0,8.599521E5,1.6666667E-1,3.3852024E7,1.3794715E-4,2.5242718E-2,-9.410978E-3,-1.2709263E-3,3.850926E2,4.44E2,-1.3680625E-2,-4.8654717E-3,3.744E3,7.156528E7,7.573041E-3,1.6519506E-2,5.80418E7,4.771E3,3.206931E2,1.11436836E5,-1.0592125E-2,-5.4114023E-3,1.2571428E1,8.683527E3,1E1,3.927456E7,-2.647402E-3,1.901875E2,6.4088334E-3,-2.2024706E-3,1.339646E6,9.458613E3,3.151751E-1,1.5357143E0,1.0470365E-3,9.451013E-3,-9.660795E-4,-5.5535776E-3,2.954124E6,3.6828358E0,-1.7359061E-2,1.4180505E-4,2.4E0,7.617426E4,4.1195396E-3,1.3099863E-2,1.6887656E-3,1E0,-1.1621586E-3,7.3528434E-3,9.408E3,-9.989625E-3,6.3697295E-3,1.6504375E-2,-1.1958338E-2,-4.78911E-3,-9.370715E-3,-1.6632874E-2,-3.141064E-4,1.0666667E1,2.7652434E-4,-3.8264103E-3,1.625E0,-3.2117467E-3,1.9225262E2,1.2E1,9.356961E6,3.326454E-3,3.2567137E-3,8.1E2,1.5532663E1,1.6802971E6,6.16E2,-1.63348E-2,3.51E3,7.445455E2,1.5723623E0,1.3550721E-3,2.5317718E-3,-3.285927E-3,9.44363E5,8.346115E-3,3.0238653E8,8.710612E1,3.407316E7,-2.770229E-3,4.162742E5,9.185E3,-8.46026E-3,1.4E1,1.4720956E-4,-4.4718855E-3,-3.224681E-3,4.7905123E-4,2E0,2.6873744E-3,1.8791208E0,1E0,3.4289816E2,2.7857144E0,3.4662805E-4,4.6404468E2,5.556659E6,1E0,1.8953742E6,5.388794E6,5.1291872E8,2.2300573E-3,1.7103828E7,5.831829E6,2.59703E-3,-7.331607E-3,9.685445E-3,2.2365898E3,-5.6178286E-3,-1.0853061E-2,-8.500792E-3,-3.1583244E-3,-6.0907956E-3,-1.2990744E-2,-5.485076E-4,-6.0274336E-3,-4.691669E-3,-1.535808E-2,7.7819085E-4,4.014255E-3,8.9214253E-4,-3.339842E-3,-4.8270114E-3,-8.9502777E-4,-3.4885944E-3,2.916251E-3,8.705812E-3,1.0193696E-3,-9.336256E-3,9.5539633E-4,7.5618178E-3,1.1169775E-3,8.491481E-3,1.591753E-2,9.2139846E-4,7.552971E-3,9.486959E-3,3.960368E-3,9.777831E-3,3.405903E-3,2.0701755E-2,1.3645905E-2,3.7264237E-3,1.5198209E-2,5.622386E-4,5.097148E-3],"split_indices":[2,2,17,68,68,48,67,2,6,6,29,67,0,43,7,1,8,71,0,10,5,0,73,0,0,0,0,68,9,0,71,44,10,43,0,0,0,0,43,0,43,0,60,62,48,23,0,66,71,48,9,8,48,69,43,73,7,0,57,0,0,67,0,0,0,10,7,0,0,7,2,73,48,0,0,70,43,3,7,0,4,0,0,9,48,68,68,0,0,0,0,62,69,0,0,69,48,0,0,0,6,0,0,2,0,0,0,0,0,0,0,0,73,0,0,68,0,4,3,60,0,0,2,73,43,0,0,10,67,68,0,0,0,9,0,7,73,7,0,62,12,0,3,0,0,0,0,8,0,69,26,4,69,53,4,62,29,43,62,7,0,62,60,0,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.067E3,5.48E2,5.19E2,2.46E2,3.02E2,3E1,4.89E2,2.31E2,1.5E1,2.71E2,3.1E1,1.7E1,1.3E1,3.51E2,1.38E2,1.52E2,7.9E1,1E1,5E0,2.46E2,2.5E1,6E0,2.5E1,1.3E1,4E0,6E0,7E0,2.76E2,7.5E1,7E0,1.31E2,5.6E1,9.6E1,7.4E1,5E0,6E0,4E0,6E0,2.4E2,8E0,1.7E1,6E0,1.9E1,2.67E2,9E0,4.9E1,2.6E1,8.5E1,4.6E1,1.8E1,3.8E1,4.9E1,4.7E1,1.2E1,6.2E1,2.31E2,9E0,4E0,1.3E1,1.4E1,5E0,2.59E2,8E0,5E0,4E0,3.4E1,1.5E1,1.4E1,1.2E1,7.2E1,1.3E1,3.3E1,1.3E1,9E0,9E0,1.1E1,2.7E1,3.7E1,1.2E1,4E0,4.3E1,5E0,7E0,3.9E1,2.3E1,2.4E1,2.07E2,4E0,5E0,4E0,9E0,1.02E2,1.57E2,4E0,4E0,2.5E1,9E0,4E0,1.1E1,4E0,6.8E1,5E0,8E0,2.9E1,4E0,6E0,7E0,4E0,7E0,6E0,2.1E1,4E0,3.3E1,5E0,7E0,3.9E1,4E0,2.5E1,1.4E1,1.9E1,4E0,6E0,1.8E1,7.9E1,1.28E2,9.7E1,5E0,6.4E1,9.3E1,1.7E1,8E0,4E0,5E0,6.1E1,7E0,8E0,2.1E1,2.7E1,6E0,1.2E1,2.7E1,1.2E1,1.3E1,7E0,7E0,1.1E1,8E0,1.4E1,4E0,6E1,1.9E1,9.2E1,3.6E1,8.5E1,1.2E1,4.9E1,1.5E1,7E1,2.3E1,1.3E1,4E0,4E1,2.1E1,4E0,4E0,9E0,1.2E1,1.7E1,1E1,8E0,4E0,4E0,2.3E1,4E0,9E0,6E0,8E0,2.5E1,3.5E1,6E0,1.3E1,4.4E1,4.8E1,1.3E1,2.3E1,6E0,7.9E1,8E0,4E0,3.3E1,1.6E1,8E0,7E0,6.3E1,7E0,1.2E1,1.1E1,9E0,4E0,3E1,1E1,6E0,1.5E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"205","size_leaf_vector":"1"}},{"base_weights":[-2.3427939E-3,-5.623861E-2,1.5676703E-1,-1.3198337E-1,1.8639259E-2,-2.9502624E-1,1.8686032E-1,-1.1840211E-1,-3.6169827E-1,3.3162467E-2,-1.0273241E-1,-1.826237E-2,-5.0650304E-3,5.7936394E-1,1.6126613E-1,-2.2093423E-1,-8.746913E-2,-4.4488412E-1,-1.0701462E-3,9.307009E-3,9.7468786E-2,-2.6139313E-1,-4.1132398E-2,1.748581E-2,3.1540528E-2,7.998693E-2,2.140466E-1,-1.7831215E-1,-2.825526E-1,-1.4985313E-1,-3.467525E-2,-5.6049544E-1,-9.986859E-3,1.6116574E-2,-1.0327695E-2,1.4700647E-1,3.0877972E-3,-6.101707E-3,-1.8025063E-2,-6.814604E-3,-6.22091E-3,5.999928E-2,1.6007181E-2,1.5136747E-1,2.8882807E-1,-1.9712232E-1,-3.472857E-4,-3.3322403E-1,-1.8675955E-1,-1.2366861E-1,-2.0969711E-1,1.0904815E-2,-4.9897004E-2,-1.50895035E-2,-3.368131E-2,-1.0956182E-2,7.0743464E-2,1.2300843E-1,1.2319498E-2,4.746086E-2,-6.541598E-2,1.8706367E-3,-3.4743544E-2,7.546723E-2,-7.0990347E-3,2.2647834E-1,9.6260704E-2,3.17631E-1,-4.2286147E-3,-1.1765553E-3,-2.1324803E-1,-9.120772E-3,-3.676226E-1,-1.9430023E-3,-1.1383385E-2,-1.3343324E-2,-1.08061954E-1,-2.5948998E-1,-1.0164127E-1,-2.1878144E-2,-9.547743E-2,-6.876728E-2,1.8321907E-2,-7.173154E-2,9.2431605E-2,9.3851976E-2,2.2049987E-1,-1.6503694E-3,9.50001E-2,1.0518869E-3,-6.6644624E-3,-3.2587184E-3,2.4912076E-4,4.0731695E-2,1.7590047E-1,2.085931E-3,2.5166938E-1,1.1742903E-1,-7.5403583E-4,2.8857383E-1,2.0203322E-2,-4.8413756E-3,-1.0262682E-2,-2.3152672E-2,-1.2201006E-2,3.9538456E-4,-1.1738142E-1,-1.7621845E-1,-3.584483E-1,-1.6830821E-3,-6.368249E-3,-3.3268414E-2,4.4435696E-3,-1.056216E-2,-6.304716E-2,-4.1113973E-2,-1.3080865E-2,4.2379634E-3,1.17666805E-2,6.068921E-4,-8.140293E-3,1.7759284E-1,4.599027E-2,8.028012E-3,6.2466465E-2,6.238012E-3,1.3677929E-2,5.5254367E-3,1.4653923E-3,5.2801795E-2,-4.7400123E-3,2.0351318E-1,2.4189707E-3,1.3605975E-1,1.3544919E-2,1.4673632E-1,4.1701887E-2,3.1746763E-1,4.824167E-3,-1.3392566E-1,-5.2085243E-2,-9.941129E-3,-2.5528905E-3,-2.013959E-2,-8.952433E-3,-6.040955E-2,2.0263355E-2,-6.8422225E-3,-1.0169511E-1,-1.3024366E-1,3.0384844E-2,1.9406306E-2,-1.3725142E-1,5.4798927E-3,1.0472691E-2,-9.15994E-3,1.6057804E-1,8.553412E-2,-2.21314E-3,-2.292266E-3,6.6084415E-2,3.7688268E-3,1.0965808E-2,2.277029E-3,9.037467E-3,1.8887774E-3,1.8174064E-1,-4.352378E-3,5.6101177E-3,1.2731254E-2,2.1059021E-2,-4.447894E-3,-8.290691E-3,-9.5859176E-4,-4.9006348E-3,-3.3029483E-3,1.3655274E-4,-6.0596596E-4,6.691559E-3,1.4183397E-3,-5.456215E-3,-1.0743662E-3,-5.308265E-3,-2.358725E-3,-1.7983975E-2,3.2831766E-3,-2.9287226E-3,-1.443607E-3,2.3336082E-3,-1.4271461E-3,-1.09691955E-2,-2.4505972E-3,4.282196E-3,3.6186844E-3,8.956619E-3,6.3879024E-3,1.4070527E-3,3.7921395E-3,-2.8258574E-4,4.579095E-3,1.1755486E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,39,-1,-1,41,43,45,47,49,51,53,-1,55,-1,57,59,-1,-1,61,-1,63,-1,65,67,69,-1,71,73,75,77,-1,79,-1,-1,81,83,85,-1,87,89,-1,91,93,-1,95,97,99,-1,-1,101,-1,103,-1,-1,-1,105,107,109,111,113,115,117,119,121,123,125,-1,127,-1,-1,-1,-1,129,131,-1,133,135,-1,137,-1,-1,-1,-1,-1,-1,139,141,143,-1,-1,145,-1,-1,147,149,-1,151,-1,-1,-1,153,155,-1,157,-1,-1,-1,-1,159,-1,161,-1,163,-1,165,167,169,-1,171,173,-1,-1,-1,-1,175,177,179,181,183,185,187,189,-1,-1,191,193,195,-1,-1,197,-1,-1,-1,-1,-1,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.472691E0,4.1881056E0,3.4541159E0,1.102148E0,6.5913343E-1,2.5409234E-1,2.267335E0,1.0844002E0,5.5292034E-1,5.099337E-1,3.801884E-1,0E0,0E0,3.2236576E-2,9.3524885E-1,1.5972185E-1,8.807533E-1,3.0190825E-1,0E0,3.904989E-1,4.203334E-1,1.3164288E-1,9.536655E-2,0E0,0E0,4.5328116E-1,5.843706E-1,1.5915251E-1,8.8240385E-2,1.6680622E-1,6.175291E-1,1.5608215E-1,0E0,3.527534E-1,0E0,1.3661587E-1,1.0037583E-1,0E0,0E0,3.179416E-2,0E0,2.892081E-1,0E0,2.8863275E-1,6.8639946E-1,1.191318E-1,0E0,1.617837E-2,1.018261E-1,1.9560921E-1,1.673199E-1,0E0,1.7670676E-1,0E0,0E0,2.720757E-1,2.5074163E-1,1.242733E-1,0E0,8.248366E-2,9.366595E-2,0E0,2.192907E-2,2.6843485E-1,0E0,1.3023949E-1,1.091595E-1,6.8546295E-2,0E0,0E0,1.626277E-2,0E0,6.1178565E-2,0E0,0E0,0E0,9.012538E-2,1.3460457E-1,2.5668189E-2,1.2153788E-1,2.1443698E-1,3.0847922E-1,3.5513106E-1,9.9841386E-2,2.647276E-1,9.19843E-2,3.1072438E-2,0E0,1.6444102E-2,0E0,0E0,0E0,0E0,1.0921896E-1,5.756992E-2,0E0,8.970201E-2,7.8216374E-2,0E0,2.2225118E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.512534E-2,5.880198E-2,4.311502E-2,0E0,0E0,1.1891006E-1,0E0,0E0,9.488122E-2,3.1621808E-1,0E0,2.2157183E-1,0E0,0E0,0E0,4.2602062E-2,2.9216972E-1,0E0,8.302341E-2,0E0,0E0,0E0,0E0,7.946089E-2,0E0,4.6670675E-2,0E0,3.6872208E-2,0E0,9.26131E-2,1.4147733E-1,1.1204529E-1,0E0,8.557236E-2,2.6560873E-2,0E0,0E0,0E0,0E0,3.8138762E-2,1.1833511E-1,8.298921E-2,2.4258286E-2,4.31387E-1,1.11325234E-1,1.4811152E-1,9.921907E-2,0E0,0E0,1.4794476E-1,2.5888652E-2,7.130657E-2,0E0,0E0,5.825652E-2,0E0,0E0,0E0,0E0,0E0,9.493512E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,36,36,39,39,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,55,55,56,56,57,57,59,59,60,60,62,62,63,63,65,65,66,66,67,67,70,70,72,72,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,93,93,94,94,96,96,97,97,99,99,106,106,107,107,108,108,111,111,114,114,115,115,117,117,121,121,122,122,124,124,129,129,131,131,133,133,135,135,136,136,137,137,139,139,140,140,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,155,155,156,156,157,157,160,160,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,40,-1,-1,42,44,46,48,50,52,54,-1,56,-1,58,60,-1,-1,62,-1,64,-1,66,68,70,-1,72,74,76,78,-1,80,-1,-1,82,84,86,-1,88,90,-1,92,94,-1,96,98,100,-1,-1,102,-1,104,-1,-1,-1,106,108,110,112,114,116,118,120,122,124,126,-1,128,-1,-1,-1,-1,130,132,-1,134,136,-1,138,-1,-1,-1,-1,-1,-1,140,142,144,-1,-1,146,-1,-1,148,150,-1,152,-1,-1,-1,154,156,-1,158,-1,-1,-1,-1,160,-1,162,-1,164,-1,166,168,170,-1,172,174,-1,-1,-1,-1,176,178,180,182,184,186,188,190,-1,-1,192,194,196,-1,-1,198,-1,-1,-1,-1,-1,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,4.9963706E5,1E0,4E1,5.1071826E-8,9.3326636E-2,2.2968E4,7.7E1,6.0078123E-6,3.9742767E2,2.3238889E2,-1.826237E-2,-5.0650304E-3,1.63757E4,1.0665628E6,1E0,5.41E2,8E0,-1.0701462E-3,4.5620965E6,2.3818183E0,5.90352E5,2.01E2,1.748581E-2,3.1540528E-2,7.6255713E3,7.098E3,1.6576804E7,1.5389474E1,1.3306123E1,1.1E1,1E1,-9.986859E-3,6.946813E7,-1.0327695E-2,4.284608E6,1.2218E4,-6.101707E-3,-1.8025063E-2,2.1470589E1,-6.22091E-3,3.0070068E7,1.6007181E-2,1.1057851E1,4.1325716E7,5E0,-3.472857E-4,9E0,7.8918E4,3.9E1,1.9470909E2,1.0904815E-2,1.9E1,-1.50895035E-2,-3.368131E-2,2.1578947E2,5.4E1,2.483E3,1.2319498E-2,2.476353E-2,4.9E1,1.8706367E-3,5.1042255E2,1.5142261E3,-7.0990347E-3,1.2157188E6,1.0975045E7,2.1511E4,-4.2286147E-3,-1.1765553E-3,1.798E3,-9.120772E-3,2.1609572E7,-1.9430023E-3,-1.1383385E-2,-1.3343324E-2,8.8E1,3.50986E5,1.84E2,4.233978E0,1.0201538E-1,1.8E1,3.2E1,1.3287E4,9.538462E0,1.6939986E9,6.257143E2,-1.6503694E-3,2.0316172E6,1.0518869E-3,-6.6644624E-3,-3.2587184E-3,2.4912076E-4,1.1290322E1,3.7798166E0,2.085931E-3,1E0,2.9157894E0,-7.5403583E-4,2E0,2.0203322E-2,-4.8413756E-3,-1.0262682E-2,-2.3152672E-2,-1.2201006E-2,3.9538456E-4,3.201807E6,2.511352E6,1.6595395E5,-1.6830821E-3,-6.368249E-3,6.077143E5,4.4435696E-3,-1.056216E-2,3.401282E1,2.9652428E6,-1.3080865E-2,6.485856E7,1.17666805E-2,6.068921E-4,-8.140293E-3,1.14E3,1.2768175E6,8.028012E-3,1.9882522E0,6.238012E-3,1.3677929E-2,5.5254367E-3,1.4653923E-3,4.6712E4,-4.7400123E-3,3.8153846E0,2.4189707E-3,2.1512408E5,1.3544919E-2,3.6522612E6,4.488263E0,1.9717247E9,4.824167E-3,1E0,2.707673E6,-9.941129E-3,-2.5528905E-3,-2.013959E-2,-8.952433E-3,1.3205625E7,4.6134964E2,5.49E2,1.8571428E0,1.925508E6,2.6210527E0,1.7587205E6,3.2075E4,5.4798927E-3,1.0472691E-2,1.4E1,1.281E3,1.0092541E8,-2.21314E-3,-2.292266E-3,1.2820834E7,3.7688268E-3,1.0965808E-2,2.277029E-3,9.037467E-3,1.8887774E-3,5.306E3,-4.352378E-3,5.6101177E-3,1.2731254E-2,2.1059021E-2,-4.447894E-3,-8.290691E-3,-9.5859176E-4,-4.9006348E-3,-3.3029483E-3,1.3655274E-4,-6.0596596E-4,6.691559E-3,1.4183397E-3,-5.456215E-3,-1.0743662E-3,-5.308265E-3,-2.358725E-3,-1.7983975E-2,3.2831766E-3,-2.9287226E-3,-1.443607E-3,2.3336082E-3,-1.4271461E-3,-1.09691955E-2,-2.4505972E-3,4.282196E-3,3.6186844E-3,8.956619E-3,6.3879024E-3,1.4070527E-3,3.7921395E-3,-2.8258574E-4,4.579095E-3,1.1755486E-2],"split_indices":[2,43,17,3,52,53,12,44,52,67,67,0,0,43,43,15,2,32,0,62,68,9,0,0,0,67,2,5,73,73,10,8,0,7,0,43,44,0,0,61,0,58,0,71,66,0,0,0,1,2,4,0,8,0,0,70,0,2,0,53,8,0,4,67,0,62,60,2,0,0,9,0,60,0,0,0,44,60,10,68,53,3,3,1,73,46,4,0,62,0,0,0,0,69,68,0,29,69,0,32,0,0,0,0,0,0,9,5,43,0,0,62,0,0,73,43,0,7,0,0,0,2,62,0,68,0,0,0,0,1,0,69,0,48,0,60,69,5,0,8,5,0,0,0,0,9,4,0,68,43,71,43,1,0,0,3,44,7,0,0,9,0,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.86E2,7.37E2,2.49E2,3.66E2,3.71E2,1.5E1,2.34E2,3.47E2,1.9E1,3.32E2,3.9E1,9E0,6E0,1.3E1,2.21E2,7.9E1,2.68E2,1.5E1,4E0,2.43E2,8.9E1,1E1,2.9E1,6E0,7E0,8.8E1,1.33E2,4.9E1,3E1,1.22E2,1.46E2,9E0,6E0,2.37E2,6E0,5.8E1,3.1E1,6E0,4E0,2.2E1,7E0,8.3E1,5E0,7.4E1,5.9E1,4.4E1,5E0,1.8E1,1.2E1,8.7E1,3.5E1,7E0,1.39E2,5E0,4E0,1.59E2,7.8E1,5E1,8E0,1.9E1,1.2E1,8E0,1.4E1,7.8E1,5E0,3E1,4.4E1,5.5E1,4E0,4E0,4E1,5E0,1.3E1,4E0,8E0,6E0,8.1E1,2.3E1,1.2E1,8.7E1,5.2E1,5.3E1,1.06E2,1E1,6.8E1,4E1,1E1,7E0,1.2E1,6E0,6E0,7E0,7E0,5.9E1,1.9E1,4E0,2.6E1,3.7E1,7E0,4.7E1,8E0,4E0,3.6E1,4E0,9E0,6E0,7.5E1,1.4E1,9E0,5E0,7E0,8E1,7E0,9E0,4.3E1,4.8E1,5E0,1.01E2,5E0,6E0,4E0,2.3E1,4.5E1,1E1,3E1,6E0,4E0,8E0,4E0,5.5E1,4E0,1.5E1,4E0,8E0,1.8E1,2.6E1,1.1E1,4E1,7E0,5.9E1,1.6E1,1E1,4E0,5E0,4E0,5.3E1,2.7E1,1.8E1,2.5E1,2.1E1,2.7E1,9.2E1,9E0,1.2E1,1.1E1,3.1E1,1.4E1,2.5E1,5E0,6E0,4.9E1,4E0,1.1E1,4E0,4E0,7E0,1.9E1,4E0,7E0,3.3E1,7E0,3.4E1,2.5E1,1.1E1,5E0,4.5E1,8E0,2.2E1,5E0,1.4E1,4E0,4E0,2.1E1,1.7E1,4E0,1.9E1,8E0,3.5E1,5.7E1,5E0,4E0,2.2E1,9E0,5E0,9E0,1.2E1,1.3E1,4E1,9E0,1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[-3.7201808E-3,-5.2778203E-2,1.3622658E-1,-1.0779125E-1,2.8416028E-2,1.6271453E-1,-2.0433286E-1,-1.8276516E-1,-6.9586866E-2,-1.660413E-2,7.807986E-2,4.2367348E-1,1.4083019E-1,-1.9697698E-2,-1.0189746E-1,-1.9264838E-1,4.378566E-3,-1.3625574E-1,-3.219716E-2,-8.073628E-3,-9.6092345E-3,-1.9239467E-2,1.0486154E-1,2.2238046E-2,1.0093383E-2,9.120168E-2,2.448266E-1,-6.5098745E-3,1.9902634E-4,4.2371196E-3,-2.030865E-1,-1.2584434E-1,-1.5233027E-2,-1.4324683E-2,-1.6117576E-1,4.5197673E-2,-2.8210383E-2,6.2002182E-2,-7.5146645E-2,1.563233E-1,5.1622476E-2,5.3254947E-2,1.9003272E-1,2.681881E-1,-3.165949E-3,-1.8626359E-1,-3.1143346E-1,-1.5406427E-1,-8.2578465E-2,-1.1710958E-3,-1.3650678E-1,-2.4947971E-1,4.1078646E-3,7.8360856E-2,-5.258918E-3,-4.8107103E-2,3.313864E-2,3.8908428E-4,6.100071E-3,-1.3480869E-1,-2.0037354E-3,1.3065596E-1,1.4026982E-2,1.3842005E-1,1.3194412E-2,-2.2411019E-2,9.1375254E-2,2.3884837E-1,1.7470121E-4,2.2469002E-1,3.813664E-1,-2.4950086E-1,-1.489723E-1,-1.7351644E-2,-7.251859E-3,-1.0222929E-1,-1.8334144E-1,-1.1884007E-1,-1.799592E-2,-4.8711717E-2,2.3742288E-2,-8.246425E-3,-1.3399397E-6,-1.6387108E-1,-2.0294687E-2,2.10715E-2,1.376623E-1,-1.5259614E-2,-3.0737117E-2,1.5241931E-2,4.583055E-3,-1.5907167E-3,-9.827345E-3,2.0346134E-3,-2.6266773E-3,7.951623E-2,1.6847312E-1,1.6358165E-1,1.86106E-3,3.7603915E-2,-8.082834E-3,-1.2975219E-2,1.410646E-2,2.0414834E-1,4.896818E-2,3.3730572E-1,1.6533695E-1,8.374456E-3,-6.9652623E-3,2.3724182E-1,2.6215327E-3,2.0265052E-2,1.1777775E-2,-1.5467033E-2,-2.1312988E-1,-9.513864E-2,-2.3620494E-1,-2.4398442E-3,-1.3165367E-1,-2.6639467E-3,-1.9605726E-1,-1.4719027E-1,-5.896057E-4,-3.652433E-3,1.4035088E-2,3.0709058E-3,-9.1966264E-2,1.1734547E-2,6.3127475E-3,-2.4482403E-3,-2.0292285E-1,-2.0088296E-2,5.6573905E-2,4.427066E-3,1.049974E-2,5.0690905E-3,-8.101064E-2,5.6614406E-2,-3.879958E-2,-1.5586324E-3,4.797947E-3,1.0187592E-2,1.1876606E-1,3.6456757E-3,8.757406E-3,-4.4908606E-2,6.867655E-2,5.0229605E-2,-5.479357E-3,3.3455284E-3,2.4078888E-1,-7.904197E-3,6.843712E-2,1.9115336E-2,7.6878057E-3,9.614157E-2,1.0431385E-2,2.4852222E-1,4.077983E-3,-1.2565714E-1,-1.1456709E-2,1.870128E-3,-1.1665622E-1,-2.6607448E-1,-4.425153E-3,-7.7513563E-3,-3.652908E-3,-2.0956075E-1,-3.3782672E-3,-8.432478E-3,-3.505158E-3,2.7623072E-3,-1.2561356E-3,3.647485E-3,-1.8755503E-2,-2.1379977E-2,-7.443925E-3,7.988813E-2,-5.0174096E-3,-1.187403E-2,-4.500849E-3,1.4127657E-3,-2.7240717E-3,2.1736983E-4,3.951917E-3,3.2079108E-2,-4.9249128E-2,-1.2078163E-1,-3.1795375E-2,3.62021E-3,8.30691E-5,-3.098265E-3,3.940629E-4,8.457033E-2,8.49701E-3,2.0760712E-3,-4.7432054E-3,7.246409E-3,2.5563816E-2,-2.993375E-2,1.2704432E-1,6.986435E-3,1.3566222E-2,-2.964809E-3,9.289515E-2,4.3040756E-4,6.393469E-3,1.6590248E-1,2.6738778E-1,-1.8351442E-3,-8.37886E-3,-3.1916583E-3,-8.34454E-3,-1.3724857E-2,-5.4687564E-3,-5.771147E-3,-1.0581565E-2,-2.7714788E-3,1.506153E-3,-3.0180826E-3,2.8991923E-3,6.0761184E-3,-1.266455E-3,-1.4005963E-3,2.159663E-3,3.5238713E-3,-1.0352954E-3,-4.372605E-3,7.488627E-5,-1.9527575E-3,-6.417076E-3,1.552799E-4,-3.811379E-3,5.5943653E-3,1.0061627E-3,4.105663E-3,1.9326035E-4,-3.472781E-3,-3.0725062E-4,9.633285E-3,2.2896563E-3,5.334121E-3,-1.3267531E-3,3.5266308E-3,1.052502E-2,5.9843413E-3,1.3464372E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,35,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,-1,85,-1,87,89,-1,-1,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,-1,131,133,-1,135,137,-1,-1,-1,-1,-1,139,141,143,-1,145,-1,-1,147,149,151,153,155,-1,-1,157,-1,-1,-1,-1,159,161,163,-1,165,-1,167,169,-1,-1,171,173,175,177,-1,-1,179,181,183,-1,-1,185,187,189,191,-1,-1,-1,193,-1,-1,195,197,199,-1,-1,201,-1,203,-1,-1,205,-1,207,-1,209,-1,-1,211,213,-1,-1,-1,215,-1,-1,-1,-1,-1,-1,217,219,-1,221,223,-1,-1,-1,-1,-1,-1,225,227,229,231,-1,-1,-1,-1,233,-1,-1,-1,-1,235,237,239,-1,-1,-1,241,-1,-1,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.167918E0,3.4558551E0,2.4778628E0,1.3077388E0,7.011248E-1,1.3794842E0,4.1143525E-1,4.4342375E-1,7.6088107E-1,2.6771158E-1,3.903674E-1,1.1971378E-1,1.185389E0,0E0,6.275605E-2,4.638586E-1,0E0,1.8007803E-1,4.5411432E-1,1.7183612E-1,0E0,1.5454498E-1,3.1248403E-1,0E0,0E0,5.899303E-1,5.674114E-1,0E0,0E0,0E0,2.0628405E-1,1.1669934E-1,0E0,2.810884E-1,5.5220246E-1,2.4097246E-1,1.4318895E-1,5.1424187E-2,8.7006986E-2,1.8339312E-1,1.9496588E-1,3.3952376E-1,4.0775204E-1,2.5892353E-1,0E0,2.6468086E-1,1.1940634E-1,7.686496E-2,1.0090348E-1,1.8947577E-1,9.7418875E-2,2.204994E-1,0E0,1.228009E-1,0E0,4.2119813E-1,3.2775342E-2,0E0,0E0,7.869843E-2,2.775989E-2,8.716953E-2,0E0,3.7857294E-2,1.9947144E-1,3.814703E-1,3.6302853E-1,1.9755173E-1,3.0067497E-1,9.8263025E-2,1.8528938E-2,7.134223E-2,3.6730015E-1,0E0,0E0,2.9399306E-2,5.102122E-2,8.232668E-2,3.4359895E-2,1.2347327E-1,1.4074244E-1,0E0,0E0,4.986608E-2,0E0,3.0361911E-2,4.570231E-2,0E0,1.5161939E-1,5.585656E-2,0E0,0E0,0E0,0E0,0E0,6.965536E-2,4.882914E-2,1.5833259E-2,0E0,1.0051168E-1,0E0,0E0,1.785375E-1,8.392513E-2,2.6019835E-1,1.1407268E-1,7.032287E-2,0E0,0E0,6.040573E-2,0E0,0E0,0E0,0E0,7.555199E-2,1.5938577E-1,1.0453725E-1,0E0,1.5230447E-2,0E0,4.2776823E-2,4.1592747E-2,0E0,0E0,2.426122E-2,4.4542007E-2,1.4300957E-1,1.1000459E-1,0E0,0E0,3.2031745E-2,2.1500658E-2,1.6124956E-2,0E0,0E0,7.4936666E-2,6.466657E-2,1.6548164E-2,1.528679E-2,0E0,0E0,0E0,2.2322446E-2,0E0,0E0,6.1698616E-2,1.0243173E-1,1.8237695E-1,0E0,0E0,3.1340778E-2,0E0,1.8110633E-1,0E0,0E0,4.2729057E-2,0E0,2.5779963E-2,0E0,5.0241843E-2,0E0,0E0,1.2749565E-1,7.932532E-2,0E0,0E0,0E0,2.250719E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.642689E-2,6.325996E-2,0E0,1.0886927E-1,1.035655E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.358866E-2,3.941317E-2,2.115795E-2,3.0580873E-2,0E0,0E0,0E0,0E0,2.7175508E-2,0E0,0E0,0E0,0E0,2.6698302E-2,1.5298711E-2,8.4357694E-2,0E0,0E0,0E0,1.2954587E-1,0E0,0E0,4.3611705E-2,7.8309774E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,21,21,22,22,25,25,26,26,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,55,55,56,56,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,79,79,80,80,83,83,85,85,86,86,88,88,89,89,95,95,96,96,97,97,99,99,102,102,103,103,104,104,105,105,106,106,109,109,114,114,115,115,116,116,118,118,120,120,121,121,124,124,125,125,126,126,127,127,130,130,131,131,132,132,135,135,136,136,137,137,138,138,142,142,145,145,146,146,147,147,150,150,152,152,155,155,157,157,159,159,162,162,163,163,167,167,174,174,175,175,177,177,178,178,185,185,186,186,187,187,188,188,193,193,198,198,199,199,200,200,204,204,207,207,208,208],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,36,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,-1,86,-1,88,90,-1,-1,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,-1,132,134,-1,136,138,-1,-1,-1,-1,-1,140,142,144,-1,146,-1,-1,148,150,152,154,156,-1,-1,158,-1,-1,-1,-1,160,162,164,-1,166,-1,168,170,-1,-1,172,174,176,178,-1,-1,180,182,184,-1,-1,186,188,190,192,-1,-1,-1,194,-1,-1,196,198,200,-1,-1,202,-1,204,-1,-1,206,-1,208,-1,210,-1,-1,212,214,-1,-1,-1,216,-1,-1,-1,-1,-1,-1,218,220,-1,222,224,-1,-1,-1,-1,-1,-1,226,228,230,232,-1,-1,-1,-1,234,-1,-1,-1,-1,236,238,240,-1,-1,-1,242,-1,-1,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.9016544E5,4.217427E7,8.147158E4,1.071E3,3.6E1,2.5330253E-2,1.482958E0,5.47E2,2.8461537E0,5.036081E7,6.441311E-1,2.9251662E6,-1.9697698E-2,2.3308511E5,7E0,4.378566E-3,1E0,2.2E1,9.917037E2,-9.6092345E-3,3.1842105E0,8E0,2.2238046E-2,1.0093383E-2,1.5142261E3,5.2964807E0,-6.5098745E-3,1.9902634E-4,4.2371196E-3,4.217427E7,3.72E2,-1.5233027E-2,8.233192E9,3.7725E1,3.7103873E2,3.3486558E6,5.1942E4,1.2839025E1,3.549E3,2.1028344E4,6.648E3,2.856934E7,1.6466942E3,-3.165949E-3,3.175E1,1.5416006E5,3.1E1,1.339646E6,2.9211267E2,1.731E3,4E1,4.1078646E-3,1.1015625E-4,-5.258918E-3,1.2324481E2,3.1847827E2,3.8908428E-4,6.100071E-3,1.5115256E6,1.3254E4,2E0,1.4026982E-2,1.5E1,1.2619E4,1.4076087E0,1.5209424E1,5.6516E4,2.9622424E3,9.244374E6,1.5454545E0,9.2058825E-1,1.8797773E-1,-1.7351644E-2,-7.251859E-3,1.6E1,1.0443479E5,8E0,1.5292561E2,1.6808511E0,1.9099288E7,-8.246425E-3,-1.3399397E-6,3.7509E4,-2.0294687E-2,5.8899284E1,3.5589743E0,-1.5259614E-2,1.6153846E0,2.0487332E-3,4.583055E-3,-1.5907167E-3,-9.827345E-3,2.0346134E-3,-2.6266773E-3,5.441268E7,3.44E2,4.1388846E2,1.86106E-3,3.1609525E2,-8.082834E-3,-1.2975219E-2,1.1555496E8,4.6847186E2,3.97371E5,3.7936268E-3,1.3567028E5,8.374456E-3,-6.9652623E-3,1.562E5,2.6215327E-3,2.0265052E-2,1.1777775E-2,-1.5467033E-2,3.7452831E0,9.935484E-1,1.1408248E0,-2.4398442E-3,2.8003533E0,-2.6639467E-3,7.997723E6,1.9545455E0,-5.896057E-4,-3.652433E-3,2.1790265E2,1E1,9.966666E0,3.8E1,6.3127475E-3,-2.4482403E-3,4.88451E5,9.836066E-1,1.623E3,4.427066E-3,1.049974E-2,4.9390244E0,4.1361522E6,5.41E2,1.3809524E0,-1.5586324E-3,4.797947E-3,1.0187592E-2,1.9505818E8,3.6456757E-3,8.757406E-3,1E0,7E0,2.4358286E7,-5.479357E-3,3.3455284E-3,1E0,-7.904197E-3,2.5711747E5,1.9115336E-2,7.6878057E-3,9.67E2,1.0431385E-2,1.5137369E6,4.077983E-3,2.8877066E4,-1.1456709E-2,1.870128E-3,1.31E2,2.8027795E4,-4.425153E-3,-7.7513563E-3,-3.652908E-3,1.2608696E0,-3.3782672E-3,-8.432478E-3,-3.505158E-3,2.7623072E-3,-1.2561356E-3,3.647485E-3,2.0116506E4,1.4381613E7,-7.443925E-3,4.45E2,1.535E3,-1.187403E-2,-4.500849E-3,1.4127657E-3,-2.7240717E-3,2.1736983E-4,3.951917E-3,1E0,4.0119403E2,5.6666665E0,4.267606E0,3.62021E-3,8.30691E-5,-3.098265E-3,3.940629E-4,9.94E2,8.49701E-3,2.0760712E-3,-4.7432054E-3,7.246409E-3,2.2307692E0,1.5204346E8,8.818731E0,6.986435E-3,1.3566222E-2,-2.964809E-3,1.2792593E2,4.3040756E-4,6.393469E-3,6.000379E8,3.0238653E8,-1.8351442E-3,-8.37886E-3,-3.1916583E-3,-8.34454E-3,-1.3724857E-2,-5.4687564E-3,-5.771147E-3,-1.0581565E-2,-2.7714788E-3,1.506153E-3,-3.0180826E-3,2.8991923E-3,6.0761184E-3,-1.266455E-3,-1.4005963E-3,2.159663E-3,3.5238713E-3,-1.0352954E-3,-4.372605E-3,7.488627E-5,-1.9527575E-3,-6.417076E-3,1.552799E-4,-3.811379E-3,5.5943653E-3,1.0061627E-3,4.105663E-3,1.9326035E-4,-3.472781E-3,-3.0725062E-4,9.633285E-3,2.2896563E-3,5.334121E-3,-1.3267531E-3,3.5266308E-3,1.052502E-2,5.9843413E-3,1.3464372E-2],"split_indices":[2,43,60,43,2,0,53,57,2,68,7,53,43,0,48,3,0,106,3,48,0,69,8,0,0,67,53,0,0,0,60,2,0,46,71,70,43,1,73,10,48,44,60,67,0,70,48,10,9,4,2,3,0,53,0,67,4,0,0,43,44,8,0,3,44,69,71,9,67,60,68,68,57,0,0,3,43,8,67,68,9,0,0,1,0,62,69,0,68,53,0,0,0,0,0,7,10,4,0,67,0,0,5,4,1,72,48,0,0,10,0,0,0,0,69,68,53,0,68,0,9,68,0,0,4,3,73,0,0,0,9,71,44,0,0,71,60,2,68,0,0,0,5,0,0,26,3,5,0,0,23,0,43,0,0,0,0,62,0,60,0,0,44,48,0,0,0,68,0,0,0,0,0,0,48,5,0,44,2,0,0,0,0,0,0,26,4,73,71,0,0,0,0,10,0,0,0,0,71,7,71,0,0,0,73,0,0,5,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,7.72E2,2.7E2,4.6E2,3.12E2,2.51E2,1.9E1,1.54E2,3.06E2,1.64E2,1.48E2,1.8E1,2.33E2,5E0,1.4E1,1.49E2,5E0,1.09E2,1.97E2,1.58E2,6E0,3.2E1,1.16E2,1.3E1,5E0,1.59E2,7.4E1,1E1,4E0,5E0,1.44E2,1.05E2,4E0,1.74E2,2.3E1,4.3E1,1.15E2,1.3E1,1.9E1,5.8E1,5.8E1,1.16E2,4.3E1,6.9E1,5E0,1.27E2,1.7E1,6.2E1,4.3E1,1.58E2,1.6E1,1.7E1,6E0,3.6E1,7E0,8.7E1,2.8E1,8E0,5E0,1E1,9E0,5.1E1,7E0,1.7E1,4.1E1,3.9E1,7.7E1,3.4E1,9E0,5.2E1,1.7E1,4.5E1,8.2E1,1.1E1,6E0,2.4E1,3.8E1,2.7E1,1.6E1,5.4E1,1.04E2,1.2E1,4E0,1.3E1,4E0,1.9E1,1.7E1,4E0,8.3E1,2.3E1,5E0,5E0,5E0,5E0,4E0,2.3E1,2.8E1,1.3E1,4E0,3.7E1,4E0,4E0,3.5E1,2E1,5.7E1,1.3E1,2.1E1,4E0,5E0,4.8E1,4E0,1E1,7E0,1.1E1,3.4E1,5.2E1,3E1,1E1,1.4E1,4E0,3.4E1,2.1E1,6E0,5E0,1.1E1,2.5E1,2.9E1,9.5E1,9E0,4E0,9E0,9E0,1E1,1.3E1,4E0,4.9E1,3.4E1,1.3E1,1E1,4E0,1.9E1,1.2E1,1.6E1,4E0,9E0,1E1,2.7E1,2.8E1,7E0,5E0,1.5E1,4E0,5.3E1,8E0,5E0,1.1E1,1E1,4.4E1,4E0,1.1E1,2.3E1,7E0,4.5E1,2.4E1,6E0,7E0,7E0,3E1,4E0,1.3E1,8E0,5E0,6E0,5E0,2E1,1.5E1,1.4E1,1.8E1,7.7E1,5E0,4E0,4E0,5E0,4E0,6E0,3.3E1,1.6E1,1.8E1,1.6E1,9E0,4E0,6E0,4E0,1.2E1,4E0,4E0,6E0,8E0,1.9E1,1.4E1,1.4E1,7E0,8E0,8E0,4.5E1,4E0,7E0,1E1,3.4E1,5E0,6E0,2.7E1,1.8E1,1.9E1,5E0,7E0,2.3E1,1.1E1,9E0,1E1,5E0,1.2E1,6E0,5.2E1,2.5E1,1.8E1,1.5E1,8E0,8E0,4E0,1.4E1,1E1,6E0,7E0,5E0,4E0,1.5E1,4E0,1E1,6E0,8E0,3.8E1,7E0,5E0,5E0,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"247","size_leaf_vector":"1"}},{"base_weights":[-5.6042373E-3,-6.0386825E-2,1.2993024E-1,-1.5974328E-1,-1.3175338E-2,-3.243074E-1,1.5108468E-1,1.9640605E-1,-1.7365013E-1,-8.107625E-3,-2.2728685E-2,-2.524669E-2,-1.6484608E-1,6.4293586E-2,1.8931831E-1,1.4133215E-2,2.1995762E-3,-1.455832E-1,-2.8846595E-1,-5.3682692E-2,3.3824727E-2,-1.0733918E-2,-2.974399E-3,-2.7115382E-2,1.1999552E-1,2.8986552E-1,1.5005909E-1,-2.1175928E-1,-9.8635666E-2,-4.4349626E-1,-1.7600514E-1,-8.8702016E-2,5.1713306E-3,8.3594695E-2,-7.110025E-3,7.0060813E-3,-8.400357E-3,1.3772951E-1,-3.594129E-3,1.9723479E-2,2.1709171E-1,2.184235E-1,9.141413E-2,-2.9992566E-1,-1.8575497E-1,4.8802393E-3,-1.14146076E-1,-3.0517232E-2,-1.2244924E-2,-2.369582E-3,-2.1085495E-1,-7.84992E-2,-1.6382184E-2,-1.09717175E-1,3.299895E-2,6.779765E-2,2.2247683E-1,7.53344E-3,-1.04338124E-1,5.3205886E-3,-2.1514324E-2,6.215589E-2,1.7960593E-1,2.1804802E-3,2.429715E-1,1.6468567E-1,2.24881E-2,1.0880122E-1,-8.18845E-3,-6.986421E-3,-1.8994104E-2,-1.3678305E-1,-2.2019397E-1,-1.6975993E-1,-6.656845E-2,-1.4315485E-2,-1.4628866E-1,6.1695995E-3,-8.906914E-2,-2.9593755E-2,-1.0161552E-2,1.6191307E-2,9.682485E-3,8.215516E-3,1.0017252E-1,4.7066435E-3,1.2789598E-2,9.347914E-2,-1.1641701E-2,-4.5176525E-2,-9.938889E-3,-4.856461E-4,-4.486872E-3,1.120196E-1,6.076592E-3,2.1792844E-1,3.5728407E-3,1.1433973E-1,1.3062398E-2,1.870648E-1,-2.1032027E-3,1.7800152E-1,5.5134304E-2,-8.049488E-3,-4.2519826E-2,-5.101213E-3,-2.3152794E-1,-2.1118683E-1,-7.282702E-2,-1.3332805E-1,-1.44184865E-2,-2.095544E-3,-8.413984E-3,-6.916259E-2,-1.6833213E-1,-3.8833516E-3,6.253603E-4,6.3711755E-2,-9.6031455E-3,3.47807E-2,-7.8296244E-2,7.642113E-2,1.8293333E-1,3.4571737E-2,7.424689E-3,5.619764E-3,-2.1548659E-2,-3.5654642E-3,-3.106019E-4,-3.0075978E-3,1.2585211E-3,7.483405E-3,2.3891607E-3,-2.4689632E-3,2.5253573E-3,1.1354262E-2,4.7459104E-3,8.334024E-3,1.1742724E-3,1.5804507E-2,1.4363039E-1,3.170089E-3,2.0704596E-1,-5.1479917E-3,1.760666E-1,7.642351E-4,-4.300111E-3,-1.135412E-2,-5.268347E-3,-1.4717348E-1,-1.21041965E-2,-1.2980774E-1,1.5836431E-3,-2.2744222E-3,-7.625631E-3,3.1777162E-2,-4.873954E-3,-1.0042541E-1,3.3298023E-2,-3.259218E-1,-6.5730415E-2,1.2241098E-3,4.7077537E-3,-5.6773257E-3,1.20887365E-2,1.1478007E-2,5.6452784E-3,-5.6093144E-3,-9.016676E-4,1.0075476E-1,1.3852444E-2,3.4378578E-3,1.0112955E-2,4.5929444E-3,-4.114629E-3,-7.141527E-2,9.774558E-3,7.47523E-2,1.7047983E-1,2.2311167E-1,4.683377E-3,2.7128326E-2,-5.1033003E-3,4.0393667E-3,9.36807E-3,-8.357427E-3,-3.6020183E-3,-2.5368736E-3,-7.880929E-3,-6.4360676E-4,3.2814434E-3,1.7691898E-3,-4.986447E-3,-4.4692855E-4,4.286147E-3,-2.5783906E-2,-3.608875E-3,1.1859142E-4,-5.854548E-3,1.5286996E-3,-3.8179467E-3,-2.0952502E-3,1.3314765E-3,8.042869E-3,3.325956E-3,-1.8886509E-3,2.3460074E-3,-4.840639E-3,1.867129E-4,-3.2826783E-3,2.0799898E-3,-2.2710275E-3,6.25005E-3,6.351626E-3,1.0893004E-2,1.2034015E-2,6.213594E-3,-2.6580174E-3,2.4874485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,-1,-1,63,65,67,69,71,-1,73,-1,-1,-1,75,77,-1,79,81,83,85,87,89,-1,91,93,95,-1,97,99,-1,101,-1,-1,-1,103,105,107,109,-1,111,-1,113,115,-1,117,-1,119,121,-1,-1,123,125,127,-1,129,-1,131,133,135,-1,137,-1,139,-1,141,143,-1,145,-1,147,149,151,153,155,-1,-1,157,159,-1,-1,161,163,165,167,169,171,173,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,177,-1,179,181,183,-1,-1,-1,-1,185,-1,187,-1,-1,-1,189,-1,191,193,195,197,-1,-1,-1,199,201,-1,-1,-1,203,205,-1,-1,-1,-1,207,209,211,213,215,-1,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3511014E0,3.3048859E0,2.787603E0,1.1584172E0,1.1582612E0,3.7047112E-1,8.9373446E-1,1.3209131E-1,6.6373825E-1,9.09574E-1,0E0,0E0,4.606831E-2,4.360445E-1,7.025628E-1,0E0,0E0,5.321238E-1,6.542058E-1,4.703731E-1,5.058501E-1,0E0,0E0,1.7548345E-1,1.9661987E-1,4.259796E-1,5.3330636E-1,1.1165476E-1,3.478999E-1,4.8341203E-1,1.0218817E-1,3.601104E-1,2.785689E-1,2.3063636E-1,1.9584063E-1,8.885164E-2,0E0,1.4390928E-1,0E0,0E0,1.4681816E-1,8.283036E-1,3.7429255E-1,1.8588364E-1,6.9387436E-2,0E0,2.5065088E-1,0E0,0E0,0E0,8.2524896E-2,3.2343298E-1,0E0,1.4364082E-1,2.0436262E-1,1.9593135E-1,4.3910086E-2,1.9885078E-1,1.0826635E-1,0E0,3.652046E-2,5.2449413E-2,1.0464311E-1,0E0,1.2625837E-1,2.6254976E-1,0E0,2.575773E-1,0E0,0E0,0E0,8.791059E-2,1.5493631E-2,1.6532314E-1,1.895607E-1,0E0,4.2375714E-2,0E0,1.9821203E-1,2.6976772E-2,0E0,8.028647E-2,0E0,8.7756634E-2,1.1575526E-1,0E0,0E0,8.157377E-2,1.333448E-1,1.5477557E-2,0E0,3.581347E-2,0E0,2.1752939E-2,3.1838868E-2,4.5310736E-2,0E0,4.827781E-2,0E0,2.8081524E-1,0E0,8.42793E-2,3.0708963E-1,0E0,2.8414294E-2,0E0,2.0484447E-2,6.227684E-2,9.599407E-2,5.5920184E-2,1.3799325E-1,0E0,0E0,3.492903E-1,3.9897025E-1,0E0,0E0,2.998861E-2,1.0715246E-1,5.76699E-2,2.0441346E-2,7.929799E-2,3.8353354E-2,1.13076106E-1,0E0,0E0,1.4596242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1229825E-2,0E0,1.8127084E-2,1.0157322E-1,1.5413284E-2,0E0,0E0,0E0,0E0,2.324143E-2,0E0,2.1174312E-2,0E0,0E0,0E0,4.045987E-2,0E0,9.058535E-2,6.73502E-2,5.317664E-1,7.1142726E-2,0E0,0E0,0E0,7.634213E-2,2.4961041E-2,0E0,0E0,0E0,6.762332E-2,3.441491E-2,0E0,0E0,0E0,0E0,9.105116E-2,1.6844496E-1,1.0415837E-1,2.7917206E-2,3.1197786E-2,0E0,5.5115484E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,40,40,41,41,42,42,43,43,44,44,46,46,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,64,64,65,65,67,67,71,71,72,72,73,73,74,74,76,76,78,78,79,79,81,81,83,83,84,84,87,87,88,88,89,89,91,91,93,93,94,94,95,95,97,97,99,99,101,101,102,102,104,104,106,106,107,107,108,108,109,109,110,110,113,113,114,114,117,117,118,118,119,119,120,120,121,121,122,122,123,123,126,126,140,140,142,142,143,143,144,144,149,149,151,151,155,155,157,157,158,158,159,159,160,160,164,164,165,165,169,169,170,170,175,175,176,176,177,177,178,178,179,179,181,181],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,-1,-1,64,66,68,70,72,-1,74,-1,-1,-1,76,78,-1,80,82,84,86,88,90,-1,92,94,96,-1,98,100,-1,102,-1,-1,-1,104,106,108,110,-1,112,-1,114,116,-1,118,-1,120,122,-1,-1,124,126,128,-1,130,-1,132,134,136,-1,138,-1,140,-1,142,144,-1,146,-1,148,150,152,154,156,-1,-1,158,160,-1,-1,162,164,166,168,170,172,174,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,178,-1,180,182,184,-1,-1,-1,-1,186,-1,188,-1,-1,-1,190,-1,192,194,196,198,-1,-1,-1,200,202,-1,-1,-1,204,206,-1,-1,-1,-1,208,210,212,214,216,-1,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1.972052E5,1E0,7E0,5.6E1,1.2877E4,6.9664386E2,1.1634076E4,2.2E1,8.49E2,-2.2728685E-2,-2.524669E-2,4.27E3,4.7567694E5,1.0717949E1,1.4133215E-2,2.1995762E-3,2.71E2,2E0,2.61E2,3.3358974E0,-1.0733918E-2,-2.974399E-3,4.233978E0,1E0,1.4839433E5,5.932344E1,9.2058825E-1,2.2E1,3.428562E2,6.48334E5,1.925673E1,1.727279E-4,1.8857143E0,1E0,2.7662E4,-8.400357E-3,4.6134964E2,-3.594129E-3,1.9723479E-2,8.118812E-1,3.4915986E3,4.023131E2,3.959E3,1.1066655E-2,4.8802393E-3,1.4912975E-1,-3.0517232E-2,-1.2244924E-2,-2.369582E-3,7.336111E4,7E0,-1.6382184E-2,3.4919708E2,8.257819E9,1.295313E6,5.6530495E6,9E0,2.9690266E4,5.3205886E-3,3.257507E5,7.4010696E0,7.403968E2,2.1804802E-3,1.4831979E0,4.2964826E0,2.24881E-2,6.9E1,-8.18845E-3,-6.986421E-3,-1.8994104E-2,1.8187732E4,1.3947369E0,4.8E0,1.5152774E2,-1.4315485E-2,2.234E3,6.1695995E-3,6.83E2,1.1643481E6,-1.0161552E-2,1.4339622E0,9.682485E-3,1.683263E7,1.19216E8,4.7066435E-3,1.2789598E-2,3.1609525E2,3.757994E0,3.6572136E7,-9.938889E-3,1.5204346E8,-4.486872E-3,5E0,8.909296E4,1.4433751E0,3.5728407E-3,9.5139277E-1,1.3062398E-2,2.1412018E6,-2.1032027E-3,1.4320667E3,7.802E-2,-8.049488E-3,3.1416278E2,-5.101213E-3,5.0710382E0,5.449711E8,8.39E2,2.9157894E0,3.38E2,-2.095544E-3,-8.413984E-3,3.1540035E9,1.5077367E3,-3.8833516E-3,6.253603E-4,2.5677419E0,2.2E1,4.07E2,4.1957852E2,1.8454931E8,5.4007E7,3.7066666E2,7.424689E-3,5.619764E-3,3.35E2,-3.5654642E-3,-3.106019E-4,-3.0075978E-3,1.2585211E-3,7.483405E-3,2.3891607E-3,-2.4689632E-3,2.5253573E-3,1.1354262E-2,4.7459104E-3,8.334024E-3,1.1742724E-3,1.5804507E-2,4.186E4,3.170089E-3,2.0666666E0,1.7291568E2,7.555942E6,7.642351E-4,-4.300111E-3,-1.135412E-2,-5.268347E-3,5.15E2,-1.21041965E-2,4.0753E4,1.5836431E-3,-2.2744222E-3,-7.625631E-3,7.89E2,-4.873954E-3,8.414097E-1,4.49239E6,5.1942E4,1.9090909E0,1.2241098E-3,4.7077537E-3,-5.6773257E-3,1.6533333E1,4.0449125E5,5.6452784E-3,-5.6093144E-3,-9.016676E-4,1.8452264E6,3.472499E-1,3.4378578E-3,1.0112955E-2,4.5929444E-3,-4.114629E-3,1.1840488E8,7.56484E5,1.2586393E3,7.120907E6,8.847283E8,4.683377E-3,7.25E2,-5.1033003E-3,4.0393667E-3,9.36807E-3,-8.357427E-3,-3.6020183E-3,-2.5368736E-3,-7.880929E-3,-6.4360676E-4,3.2814434E-3,1.7691898E-3,-4.986447E-3,-4.4692855E-4,4.286147E-3,-2.5783906E-2,-3.608875E-3,1.1859142E-4,-5.854548E-3,1.5286996E-3,-3.8179467E-3,-2.0952502E-3,1.3314765E-3,8.042869E-3,3.325956E-3,-1.8886509E-3,2.3460074E-3,-4.840639E-3,1.867129E-4,-3.2826783E-3,2.0799898E-3,-2.2710275E-3,6.25005E-3,6.351626E-3,1.0893004E-2,1.2034015E-2,6.213594E-3,-2.6580174E-3,2.4874485E-3],"split_indices":[2,43,17,3,3,9,67,43,3,2,0,0,2,43,71,0,0,2,32,70,69,0,0,68,6,48,73,68,10,67,46,73,53,68,19,9,0,4,0,0,68,4,73,9,53,0,53,0,0,0,60,3,0,70,46,9,43,3,48,0,43,71,4,0,53,68,0,8,0,0,0,43,68,69,67,0,12,0,2,43,0,68,0,60,5,0,0,67,73,5,0,7,0,3,48,53,0,53,0,60,0,4,72,0,62,0,69,46,2,69,0,0,0,46,48,0,0,69,0,0,4,7,7,4,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,9,0,68,73,1,0,0,0,0,44,0,12,0,0,0,2,0,68,43,1,68,0,0,0,73,43,0,0,0,60,53,0,0,0,0,7,12,4,60,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.88E2,7.04E2,2.84E2,2.26E2,4.78E2,1.2E1,2.72E2,8E0,2.18E2,4.74E2,4E0,4E0,8E0,8.4E1,1.88E2,4E0,4E0,1.77E2,4.1E1,2.27E2,2.47E2,4E0,4E0,3.2E1,5.2E1,5.1E1,1.37E2,7.2E1,1.05E2,1.6E1,2.5E1,1.42E2,8.5E1,1.11E2,1.36E2,2.7E1,5E0,4.8E1,4E0,1.6E1,3.5E1,6.2E1,7.5E1,1.4E1,5.8E1,7E0,9.8E1,6E0,1E1,6E0,1.9E1,1.38E2,4E0,1.6E1,6.9E1,1.01E2,1E1,1.19E2,1.7E1,5E0,2.2E1,1.8E1,3E1,5E0,3E1,5.3E1,9E0,7.1E1,4E0,7E0,7E0,2.6E1,3.2E1,4.4E1,5.4E1,6E0,1.3E1,6E0,1.32E2,1E1,6E0,6.4E1,5E0,3.6E1,6.5E1,4E0,6E0,2.1E1,9.8E1,1.2E1,5E0,1.8E1,4E0,9E0,9E0,2.1E1,9E0,8E0,2.2E1,4.8E1,5E0,3E1,4.1E1,1.8E1,8E0,4E0,2.8E1,3E1,1.4E1,2.3E1,3.1E1,4E0,9E0,1.07E2,2.5E1,4E0,6E0,2.2E1,4.2E1,2.8E1,8E0,5.2E1,1.3E1,1.2E1,9E0,6E0,9.2E1,6E0,6E0,5E0,1.3E1,4E0,5E0,4E0,5E0,1.6E1,5E0,4E0,4E0,9E0,3.9E1,7E0,2.3E1,2.8E1,1.3E1,4E0,4E0,2.4E1,4E0,1.5E1,1.5E1,9E0,5E0,7E0,1.6E1,2.1E1,1E1,8.2E1,2.5E1,9E0,1.6E1,1.2E1,1E1,6E0,3.6E1,2.3E1,5E0,4E0,4E0,3.7E1,1.5E1,4E0,9E0,8E0,4E0,3.5E1,5.7E1,1.2E1,2.7E1,1.9E1,4E0,2.2E1,6E0,4E0,9E0,9E0,6E0,4E0,5E0,1E1,1.1E1,4E0,7.8E1,1.5E1,1E1,4E0,5E0,8E0,8E0,3E1,6E0,5E0,1.8E1,9E0,2.8E1,6E0,9E0,2.4E1,1.1E1,1.7E1,4E1,4E0,8E0,2E1,7E0,1.2E1,7E0,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-3.7557508E-3,-5.356728E-2,1.4186162E-1,-1.528055E-1,-2.857376E-3,-2.832127E-1,1.6357172E-1,-1.3140261E-1,-3.302597E-1,-1.1550596E-1,1.8353786E-2,-1.8730268E-2,-8.493633E-3,6.835811E-2,2.1737468E-1,-1.6718604E-1,-8.6870246E-2,-5.949048E-1,-1.648869E-1,-9.126658E-2,-1.8444842E-2,-9.19258E-3,6.997228E-2,1.09520346E-1,-7.306444E-3,2.8212065E-1,1.00879565E-1,-1.3432595E-2,-1.5529728E-1,1.274936E-2,-1.0777171E-1,-3.4556713E-2,-1.3500018E-2,6.179515E-4,-2.1677454E-1,-1.4211644E-1,-4.7929402E-2,-4.268616E-2,3.2748528E-2,7.840632E-2,-8.635644E-3,4.638027E-2,1.6900975E-1,-4.1179787E-2,4.0469444E-3,2.3710096E-1,4.330049E-1,2.6837327E-2,1.6909108E-1,-1.4538892E-1,-1.4599904E-2,-1.7023702E-1,-2.100985E-2,-1.3649008E-2,-5.582979E-3,-8.309649E-2,-1.7927557E-1,1.3527782E-3,-6.69579E-2,-7.4526265E-2,4.766257E-2,5.0661497E-2,-6.89106E-2,2.434058E-2,1.3213399E-1,-3.7332933E-2,9.68107E-2,2.0912513E-1,-3.390654E-3,1.5296387E-3,-7.80872E-2,2.8110296E-1,1.1583066E-1,2.4640094E-2,1.1977588E-2,-8.712043E-3,7.95725E-2,1.3394545E-1,1.4692688E-2,-1.6311975E-1,-8.5737474E-2,-1.1284824E-1,-1.3032554E-2,5.8899667E-2,-6.592208E-2,-6.0413913E-3,-1.1566941E-3,-9.1470005E-3,-3.8590443E-3,-8.554575E-2,1.5727554E-3,-5.64944E-2,-1.8141877E-2,-5.6810346E-2,1.16797656E-1,1.7292814E-2,8.1288956E-2,-2.2312049E-2,-7.676224E-3,-3.63934E-2,6.519692E-2,1.5005349E-1,-9.975056E-3,-3.116995E-3,8.4780186E-4,4.3281596E-2,6.0893246E-3,1.16314115E-2,1.1572087E-1,-2.0492954E-2,-7.273446E-3,1.6446128E-2,2.1358742E-1,-1.2744347E-3,1.4747547E-1,1.4035816E-1,-5.8292816E-3,9.476695E-3,5.8280297E-2,-2.1458541E-4,-1.707936E-1,2.1300022E-2,-6.3551646E-3,-1.4219256E-1,2.1160326E-3,2.1370817E-2,5.446219E-3,1.6521169E-3,-8.4359266E-2,-3.255745E-2,-1.2656389E-1,-4.2470887E-2,-1.78845E-1,-9.1749795E-2,1.0411858E-3,1.5287383E-1,1.01232676E-4,1.2553751E-1,-1.4810339E-2,1.0559643E-1,2.5457121E-2,-3.823277E-3,4.6852478E-4,-9.754159E-2,1.8690983E-3,1.0593614E-1,-5.7380903E-3,1.6477413E-1,3.0704768E-4,2.0880303E-3,-2.917539E-3,3.8176242E-3,-2.184008E-4,1.246536E-3,7.871543E-3,-3.0559883E-3,2.195523E-3,2.5345418E-1,1.3537777E-3,9.006914E-3,1.765918E-3,2.7478794E-3,8.435069E-3,-9.773464E-4,4.2581656E-3,-1.0102244E-1,-1.849062E-1,3.1487294E-3,-9.838736E-4,-7.738598E-2,-1.8368787E-1,2.8991008E-3,-1.0890296E-3,-4.64967E-3,-1.084472E-4,-3.0389505E-3,1.0488047E-3,-7.977407E-3,-3.3038724E-3,-7.425573E-2,1.3656159E-2,-1.2256857E-2,-1.0178325E-3,-6.5754154E-3,-2.2484954E-3,8.95076E-3,8.593619E-2,2.5366854E-3,8.639962E-3,5.5932276E-2,-4.6011843E-2,9.361874E-3,7.714603E-2,-1.8858064E-3,6.199686E-2,-1.0518959E-2,-1.4058007E-2,7.20199E-2,1.143643E-2,-2.843269E-3,2.264298E-2,1.04351275E-1,2.014991E-1,5.483878E-3,1.3614059E-2,-6.9777896E-3,-6.994794E-4,-1.0109652E-2,-7.368951E-3,-2.4036414E-4,-4.76596E-3,-1.0060209E-2,-4.8676725E-3,1.4966434E-3,-4.370702E-3,6.086785E-3,-1.6152324E-3,1.3124244E-3,5.8338824E-3,4.190093E-3,1.1915165E-5,-3.4006445E-3,2.697135E-4,7.1319556E-3,2.7713517E-3,4.112183E-4,4.0392824E-3,2.9411428E-3,-2.9955518E-3,2.2150865E-3,5.711781E-3,2.2634848E-3,-1.1109737E-3,2.9519508E-3,8.189894E-3,5.9749857E-3,1.2125979E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,-1,49,-1,51,-1,-1,-1,53,55,57,59,61,63,-1,65,67,69,-1,71,73,75,77,79,-1,81,83,-1,-1,85,87,-1,89,91,93,95,97,99,101,103,105,107,-1,-1,109,111,113,-1,-1,-1,115,117,-1,119,121,123,-1,125,127,-1,-1,-1,-1,129,-1,131,-1,133,135,137,139,141,-1,143,145,147,149,-1,-1,151,-1,-1,153,155,-1,-1,157,-1,159,161,-1,-1,163,-1,165,167,-1,169,-1,171,-1,-1,173,175,177,179,181,183,-1,185,-1,187,189,191,193,-1,-1,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,205,207,-1,-1,209,211,-1,-1,-1,-1,-1,-1,-1,-1,213,215,-1,-1,-1,-1,-1,217,-1,-1,219,221,-1,223,-1,225,227,-1,229,-1,-1,231,233,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.398838E0,3.8241575E0,2.4457111E0,9.3269825E-1,1.2063813E0,8.332908E-2,1.2555885E0,3.5357976E-1,1.0835652E0,5.137789E-1,6.049515E-1,0E0,0E0,2.842217E-1,1.163269E0,1.4141941E-1,8.260407E-1,2.4481821E-1,1.6939294E-1,1.5846509E-1,0E0,3.9192057E-1,3.3532184E-1,2.1377504E-1,1.096934E-1,5.8340263E-1,2.8839296E-1,0E0,1.4197063E-1,0E0,5.383841E-1,0E0,0E0,0E0,6.994367E-2,5.9669316E-2,6.366733E-2,4.4944263E-1,2.2929724E-1,4.150787E-1,0E0,1.3026503E-1,3.0706972E-1,7.1131304E-2,0E0,3.943305E-1,2.4526739E-1,3.4444174E-1,1.14438E-1,1.05531216E-1,0E0,3.404932E-1,1.5820612E-1,0E0,0E0,3.8243175E-2,1.9299865E-2,0E0,6.674351E-2,6.375787E-1,3.0276778E-1,1.06926024E-1,8.239066E-2,1.8351261E-1,1.848464E-1,2.137516E-2,2.9887542E-2,7.442236E-2,0E0,0E0,7.446534E-2,2.171936E-1,1.0845107E-1,0E0,0E0,0E0,3.1786966E-1,1.2808108E-1,0E0,1.0379219E-1,1.5824623E-1,1.9273269E-1,0E0,3.2899674E-2,5.550968E-2,0E0,0E0,0E0,0E0,5.8910385E-2,0E0,1.8114063E-1,0E0,4.9197737E-2,1.0302907E-1,1.8255906E-1,7.195404E-2,2.8302189E-2,0E0,1.4492512E-1,1.2776071E-1,1.3401198E-1,2.9258605E-2,0E0,0E0,1.7394174E-2,0E0,0E0,4.390958E-2,3.7301842E-2,0E0,0E0,2.3501801E-1,0E0,9.11127E-2,5.402416E-2,0E0,0E0,3.9638434E-2,0E0,6.2061787E-2,2.1410324E-2,0E0,7.721692E-2,0E0,2.1937333E-2,0E0,0E0,3.1817272E-2,2.7194906E-2,2.8265774E-2,1.7941299E-1,1.4329469E-1,1.9219846E-2,0E0,3.5328925E-2,0E0,4.1837946E-2,9.3071856E-2,8.928457E-2,4.6121582E-2,0E0,0E0,3.026628E-1,0E0,1.16652995E-1,2.8318616E-2,1.0767686E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1502409E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6439435E-2,2.282834E-2,0E0,0E0,2.7371578E-2,3.0427039E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4196849E-1,2.170464E-1,0E0,0E0,0E0,0E0,0E0,1.7299362E-2,0E0,0E0,2.5137812E-2,4.1925438E-2,0E0,3.181915E-2,0E0,1.5427865E-2,5.6512907E-2,0E0,2.3892798E-2,0E0,0E0,1.5942406E-2,5.9474915E-2,1.2006509E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,55,55,56,56,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,76,76,77,77,79,79,80,80,81,81,83,83,84,84,89,89,91,91,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,105,105,108,108,109,109,112,112,114,114,115,115,118,118,120,120,121,121,123,123,125,125,128,128,129,129,130,130,131,131,132,132,133,133,135,135,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,157,157,165,165,166,166,169,169,170,170,179,179,180,180,186,186,189,189,190,190,192,192,194,194,195,195,197,197,200,200,201,201,202,202],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,-1,50,-1,52,-1,-1,-1,54,56,58,60,62,64,-1,66,68,70,-1,72,74,76,78,80,-1,82,84,-1,-1,86,88,-1,90,92,94,96,98,100,102,104,106,108,-1,-1,110,112,114,-1,-1,-1,116,118,-1,120,122,124,-1,126,128,-1,-1,-1,-1,130,-1,132,-1,134,136,138,140,142,-1,144,146,148,150,-1,-1,152,-1,-1,154,156,-1,-1,158,-1,160,162,-1,-1,164,-1,166,168,-1,170,-1,172,-1,-1,174,176,178,180,182,184,-1,186,-1,188,190,192,194,-1,-1,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,206,208,-1,-1,210,212,-1,-1,-1,-1,-1,-1,-1,-1,214,216,-1,-1,-1,-1,-1,218,-1,-1,220,222,-1,224,-1,226,228,-1,230,-1,-1,232,234,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,2.0976269E5,1E0,4.586207E0,4.97E2,3.698E3,9.616803E2,1.3390654E2,2E0,1E0,1.9791039E6,-1.8730268E-2,-8.493633E-3,1.6942337E1,1.6007428E7,9E0,2.04115E5,3.3E1,3.7990784E7,3.51E2,-1.8444842E-2,1.852364E6,1.0358873E3,7.233077E2,4.439357E8,2.2365898E3,1.4E1,-1.3432595E-2,4.306E4,1.274936E-2,1.4912975E-1,-3.4556713E-2,-1.3500018E-2,6.179515E-4,9.237895E0,1.6667E4,1.2051282E0,1.409E3,5.2E1,5.4007E7,-8.635644E-3,1.1656439E9,3.2E1,4.383864E7,4.0469444E-3,2.7578741E1,1.159499E6,3.3580637E8,5.428175E3,2.71E2,-1.4599904E-2,5.884866E7,1E0,-1.3649008E-2,-5.582979E-3,1.4E1,5.0149255E0,1.3527782E-3,7.026624E7,2.511E3,1.594E3,1.6009701E7,1.6869704E6,2.5437157E2,1.3558896E7,3.164034E0,9.1233586E4,7.4390995E6,-3.390654E-3,1.5296387E-3,3.3803125E6,2.115238E5,9.812221E9,2.4640094E-2,1.1977588E-2,-8.712043E-3,3.85374E8,2.88721E5,1.4692688E-2,2.4E1,5.2117184E4,5.595208E2,-1.3032554E-2,3.7439062E2,2.5454E4,-6.0413913E-3,-1.1566941E-3,-9.1470005E-3,-3.8590443E-3,3.429012E7,1.5727554E-3,1.318849E6,-1.8141877E-2,1.6E1,2.453E4,7.9555137E3,1.8729467E8,2.9849844E7,-7.676224E-3,7.271717E6,2.964867E4,1E0,1.09E2,-3.116995E-3,8.4780186E-4,7.4010696E0,6.0893246E-3,1.16314115E-2,2.35184E5,6.5062125E5,-7.273446E-3,1.6446128E-2,2.3385885E0,-1.2744347E-3,8.710612E1,8E0,-5.8292816E-3,9.476695E-3,1.4608225E5,-2.1458541E-4,2.416E3,7.84664E1,-6.3551646E-3,6.23E2,2.1160326E-3,1.7161617E2,5.446219E-3,1.6521169E-3,1.946094E6,7.1E1,1.9903773E2,3.90225E2,1.3881669E6,2.2190831E5,1.0411858E-3,5.3386245E0,1.01232676E-4,1.177E3,3.1226995E0,1.4195632E6,1.6E1,-3.823277E-3,4.6852478E-4,2.6210527E0,1.8690983E-3,3.6977E4,3.5814576E7,1.1860938E8,3.0704768E-4,2.0880303E-3,-2.917539E-3,3.8176242E-3,-2.184008E-4,1.246536E-3,7.871543E-3,-3.0559883E-3,2.195523E-3,2.6402525E5,1.3537777E-3,9.006914E-3,1.765918E-3,2.7478794E-3,8.435069E-3,-9.773464E-4,4.2581656E-3,3.07E2,3.89E2,3.1487294E-3,-9.838736E-4,1E0,9.24E2,2.8991008E-3,-1.0890296E-3,-4.64967E-3,-1.084472E-4,-3.0389505E-3,1.0488047E-3,-7.977407E-3,-3.3038724E-3,1.5714285E0,4.3E1,-1.2256857E-2,-1.0178325E-3,-6.5754154E-3,-2.2484954E-3,8.95076E-3,3.4444444E0,2.5366854E-3,8.639962E-3,3.92E2,1.0000181E0,9.361874E-3,2E0,-1.8858064E-3,1.2783505E0,2.0399521E2,-1.4058007E-2,3.7035952E6,1.143643E-2,-2.843269E-3,3.2892792E2,1.609E3,4.4971E4,5.483878E-3,1.3614059E-2,-6.9777896E-3,-6.994794E-4,-1.0109652E-2,-7.368951E-3,-2.4036414E-4,-4.76596E-3,-1.0060209E-2,-4.8676725E-3,1.4966434E-3,-4.370702E-3,6.086785E-3,-1.6152324E-3,1.3124244E-3,5.8338824E-3,4.190093E-3,1.1915165E-5,-3.4006445E-3,2.697135E-4,7.1319556E-3,2.7713517E-3,4.112183E-4,4.0392824E-3,2.9411428E-3,-2.9955518E-3,2.2150865E-3,5.711781E-3,2.2634848E-3,-1.1109737E-3,2.9519508E-3,8.189894E-3,5.9749857E-3,1.2125979E-2],"split_indices":[2,43,17,68,2,2,67,67,32,106,43,0,0,71,66,70,5,3,7,2,0,9,4,70,12,67,3,0,1,0,53,0,0,0,71,9,68,10,8,7,0,46,3,12,0,71,9,7,67,2,0,5,26,0,0,0,71,0,7,2,2,5,60,4,9,53,48,62,0,0,66,48,46,0,0,0,47,9,0,2,62,67,0,4,9,0,0,0,0,7,0,9,0,3,44,62,7,5,0,60,62,6,0,0,0,71,0,0,1,43,0,0,53,0,73,3,0,0,48,0,9,67,0,2,0,67,0,0,60,0,4,67,43,62,0,69,0,2,69,60,73,0,0,71,0,12,7,5,0,0,0,0,0,0,0,0,0,48,0,0,0,0,0,0,0,1,1,0,0,17,2,0,0,0,0,0,0,0,0,69,0,0,0,0,0,0,68,0,0,10,57,0,8,0,68,4,0,43,0,0,4,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.59E2,2.59E2,2.56E2,5.03E2,1.2E1,2.47E2,2.3E2,2.6E1,7.9E1,4.24E2,4E0,8E0,9E1,1.57E2,1.26E2,1.04E2,9E0,1.7E1,7.4E1,5E0,2.77E2,1.47E2,5.8E1,3.2E1,1E2,5.7E1,9E0,1.17E2,5E0,9.9E1,5E0,4E0,4E0,1.3E1,3.3E1,4.1E1,1.54E2,1.23E2,1.43E2,4E0,2.9E1,2.9E1,2.4E1,8E0,7.9E1,2.1E1,2.8E1,2.9E1,1.12E2,5E0,5.7E1,4.2E1,6E0,7E0,1.4E1,1.9E1,8E0,3.3E1,1.14E2,4E1,1.05E2,1.8E1,7.2E1,7.1E1,1.1E1,1.8E1,2.5E1,4E0,8E0,1.6E1,5.7E1,2.2E1,1.2E1,9E0,5E0,2.3E1,2.5E1,4E0,8.5E1,2.7E1,3.9E1,1.8E1,1.5E1,2.7E1,7E0,7E0,1.5E1,4E0,2.8E1,5E0,1.09E2,5E0,1.6E1,2.4E1,5.1E1,5.4E1,1.3E1,5E0,2.9E1,4.3E1,6.3E1,8E0,7E0,4E0,8E0,1E1,1.6E1,9E0,1E1,6E0,2.5E1,3.2E1,4E0,1.8E1,1.8E1,5E0,1.2E1,1.3E1,4E0,8.1E1,9E0,1.8E1,3.3E1,6E0,1E1,5E0,4E0,2.3E1,1.3E1,1.5E1,9.9E1,1E1,1.1E1,5E0,1.8E1,6E0,1.1E1,4E1,3.7E1,1.7E1,4E0,9E0,1.6E1,1.3E1,2.7E1,1.6E1,5.7E1,6E0,4E0,4E0,4E0,4E0,4E0,5E0,6E0,4E0,2.6E1,6E0,1.2E1,6E0,7E0,1.1E1,4E0,9E0,1.5E1,6.6E1,4E0,5E0,1.4E1,1.9E1,5E0,5E0,1.9E1,4E0,8E0,5E0,7E0,8E0,6.3E1,3.6E1,6E0,4E0,4E0,7E0,1E1,8E0,6E0,5E0,1.2E1,2.8E1,7E0,3E1,6E0,1.1E1,1.2E1,4E0,2.3E1,4E0,5E0,1.1E1,2.3E1,3.4E1,7E0,1.9E1,9E0,6E0,2.5E1,4.1E1,4E0,1E1,1.2E1,7E0,1E1,5.3E1,1E1,2.6E1,4E0,4E0,7E0,5E0,1.8E1,1E1,4E0,2.6E1,4E0,7E0,5E0,7E0,1.7E1,6E0,7E0,4E0,1.6E1,7E0,1.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[4.803224E-4,-6.4988546E-2,1.21288955E-1,-1.1059974E-1,1.0745003E-2,-1.976288E-1,1.3855101E-1,-1.4435022E-1,-3.824352E-2,-1.4367324E-1,2.1822821E-2,-1.8122297E-2,-1.1957463E-1,9.712047E-2,2.5266978E-1,-1.3449155E-1,-4.3064338E-1,9.492502E-2,-5.5827163E-2,-4.490818E-2,-1.7173467E-2,7.91889E-3,9.406123E-2,-3.2540635E-4,-8.953963E-3,7.566279E-2,2.1347274E-1,2.8531857E-2,2.1964534E-1,-1.277048E-1,-2.202736E-2,-6.1311997E-3,-2.968037E-2,-5.8277004E-4,6.6967052E-3,-4.308805E-2,-2.0673989E-1,2.084897E-3,-5.9379395E-3,-4.7458272E-2,2.214873E-2,2.39022E-2,2.0004493E-1,9.030686E-2,-1.4208077E-1,4.2823735E-3,2.4661657E-1,2.5455597E-1,6.712854E-3,-1.7614657E-1,-9.713884E-2,-1.0351726E-1,-4.804799E-3,-1.4698131E-2,-2.4940043E-3,2.204983E-2,-9.282545E-2,1.5735382E-1,1.1036502E-2,6.765682E-2,-7.202281E-3,5.1831864E-3,1.0752917E-2,9.983039E-2,-5.824326E-3,-1.7747069E-2,-6.841284E-3,6.5612113E-3,2.7227536E-1,3.7616786E-1,2.0156719E-1,5.2179517E-3,-4.6420004E-3,-7.661795E-2,-2.0191501E-1,1.7577179E-1,-1.1540851E-1,1.4363092E-4,-1.17628835E-1,4.173631E-3,-2.0582218E-2,7.042988E-2,-1.3296179E-3,-1.3204718E-1,2.466316E-3,2.3801855E-3,9.416779E-3,3.913172E-2,-2.7490882E-2,1.97283E-2,6.333131E-3,5.607387E-2,1.4529963E-1,3.4835741E-3,-3.4297463E-3,1.3654399E-2,7.255779E-3,2.1689147E-2,1.1488178E-2,1.5455331E-1,2.8439677E-1,-5.7331333E-3,-2.4864047E-3,-2.1296485E-1,-4.0944275E-3,3.2724605E-3,1.0423297E-2,-9.6584104E-2,-2.0386173E-1,-9.7910434E-2,-1.0542079E-2,-6.9274396E-2,3.6720706E-3,5.900914E-3,-4.408583E-5,-1.0289645E-3,-1.6715394E-1,6.889242E-3,7.121398E-2,-4.984149E-2,4.7353823E-2,3.5678826E-3,-1.1948405E-3,-2.1087103E-2,7.6592214E-2,2.6549757E-1,8.5739076E-2,1.0424673E-2,7.90671E-2,1.5542647E-2,7.141104E-3,1.5845262E-3,-2.1311706E-3,-1.8088901E-1,-2.51622E-1,-8.0336615E-2,-1.1105385E-2,-3.5130077E-3,-2.2945558E-1,-7.057203E-2,-6.3128164E-3,1.1483617E-3,-4.359377E-3,3.632732E-2,-1.8513006E-3,-1.3865606E-3,-2.1429503E-1,4.0211953E-2,-3.510861E-2,7.59381E-3,5.463084E-2,-1.04452804E-1,-2.2676118E-2,-1.439573E-2,8.026256E-3,1.920989E-2,-6.0866047E-3,8.942626E-2,-3.2350274E-3,1.6472772E-2,9.477966E-3,-2.5429567E-2,1.0178783E-1,-4.0402506E-3,1.4478526E-1,-9.441583E-3,-3.7248184E-3,-1.8939136E-2,-9.839627E-3,-5.2954275E-3,3.3181882E-4,-1.1807099E-2,-4.137211E-3,-2.1972875E-3,-5.7290085E-3,4.3375455E-3,-5.3100684E-4,-1.5052858E-2,-4.805458E-3,-1.6188619E-3,2.5970554E-3,1.2557821E-4,-6.8897875E-3,2.012465E-3,5.4968237E-3,-2.4611356E-3,-6.8824627E-3,-3.7292093E-3,3.9702593E-4,2.7175676E-3,-3.9368607E-3,-2.8369667E-3,2.6757447E-3,5.9484793E-3,1.1066947E-3,-5.294183E-3,3.1795034E-3,9.655174E-3,2.7038043E-3,8.366757E-3,3.057394E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,-1,47,49,-1,-1,-1,-1,-1,51,53,-1,-1,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,-1,-1,81,83,85,87,89,-1,-1,-1,91,-1,-1,93,-1,95,97,99,-1,-1,101,103,105,107,-1,109,-1,111,113,-1,115,-1,-1,-1,117,119,121,-1,123,125,-1,-1,-1,-1,-1,-1,127,129,-1,131,133,-1,-1,-1,135,137,139,-1,141,143,-1,-1,-1,145,147,149,151,153,-1,-1,155,157,159,161,-1,163,-1,-1,-1,-1,165,167,169,-1,-1,171,173,-1,-1,-1,175,-1,-1,177,179,181,-1,183,185,187,189,-1,191,-1,193,-1,-1,-1,195,197,-1,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.146455E0,2.3086586E0,2.020256E0,1.0128174E0,4.3421087E-1,2.4174702E-1,1.6018939E0,7.4366236E-1,3.1907216E-1,3.6621127E-1,2.3629305E-1,0E0,1.23946235E-1,6.1799717E-1,9.610038E-1,5.802903E-1,4.826603E-1,8.790864E-2,2.1986881E-1,1.0367721E-1,0E0,1.5786476E-1,2.7777562E-1,0E0,0E0,7.015799E-1,1.3326001E-1,0E0,6.3014793E-1,3.8563108E-1,0E0,0E0,0E0,0E0,0E0,2.5629294E-1,1.3598934E-1,0E0,0E0,1.3133563E-1,2.3714969E-1,1.9935629E-1,1.698786E-2,4.3090224E-1,4.5568228E-1,0E0,4.184544E-2,3.9459372E-1,1.5938137E-1,2.5125837E-1,8.6291754E-1,6.506628E-2,1.0543821E-1,0E0,0E0,4.383269E-2,1.5041453E-1,4.85242E-2,1.6127618E-1,6.3846014E-2,0E0,0E0,0E0,3.798195E-1,0E0,0E0,6.1028562E-2,0E0,1.527667E-2,1.3308072E-1,1.6243505E-1,0E0,0E0,8.064334E-2,8.377552E-2,4.155594E-2,2.4505806E-1,0E0,6.0500085E-2,0E0,7.111403E-2,3.6933497E-2,0E0,7.335377E-2,0E0,0E0,0E0,8.8468775E-2,1.0774812E-1,3.6076866E-2,0E0,1.6151989E-1,6.5734696E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7535573E-1,6.958568E-2,0E0,1.7636184E-2,4.624152E-2,0E0,0E0,0E0,2.9142213E-1,7.088065E-2,2.6359677E-2,0E0,4.9573705E-2,6.0023487E-2,0E0,0E0,0E0,9.054324E-2,6.2989004E-2,5.744861E-2,7.032472E-2,1.19558014E-1,0E0,0E0,1.0063494E-1,1.5679595E-1,9.719443E-2,1.1855608E-1,0E0,2.2058652E-1,0E0,0E0,0E0,0E0,8.710039E-2,1.02329135E-1,3.6389846E-1,0E0,0E0,5.7490945E-2,2.1152787E-2,0E0,0E0,0E0,6.714228E-2,0E0,0E0,1.0140324E-1,3.2075748E-2,8.666871E-2,0E0,2.188699E-2,2.8521672E-2,6.2405165E-2,6.2051054E-2,0E0,5.6605503E-2,0E0,1.8747121E-1,0E0,0E0,0E0,8.352239E-2,2.5212848E-1,0E0,2.798155E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,28,28,29,29,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,59,59,63,63,66,66,68,68,69,69,70,70,73,73,74,74,75,75,76,76,78,78,80,80,81,81,83,83,87,87,88,88,89,89,91,91,92,92,99,99,100,100,102,102,103,103,107,107,108,108,109,109,111,111,112,112,116,116,117,117,118,118,119,119,120,120,123,123,124,124,125,125,126,126,128,128,133,133,134,134,135,135,138,138,139,139,143,143,146,146,147,147,148,148,150,150,151,151,152,152,153,153,155,155,157,157,161,161,162,162,164,164],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,-1,48,50,-1,-1,-1,-1,-1,52,54,-1,-1,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,-1,-1,82,84,86,88,90,-1,-1,-1,92,-1,-1,94,-1,96,98,100,-1,-1,102,104,106,108,-1,110,-1,112,114,-1,116,-1,-1,-1,118,120,122,-1,124,126,-1,-1,-1,-1,-1,-1,128,130,-1,132,134,-1,-1,-1,136,138,140,-1,142,144,-1,-1,-1,146,148,150,152,154,-1,-1,156,158,160,162,-1,164,-1,-1,-1,-1,166,168,170,-1,-1,172,174,-1,-1,-1,176,-1,-1,178,180,182,-1,184,186,188,190,-1,192,-1,194,-1,-1,-1,196,198,-1,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,7.785302E5,1E0,1.380863E6,5.862757E1,7.937523E-3,1.5142261E3,1.5126086E2,7E0,3.4E1,4.9321495E6,-1.8122297E-2,1E0,4.391553E6,3.3817584E7,4.8E1,4.68418E5,1.4979E4,1.2906634E7,1.3881669E6,-1.7173467E-2,2.3043478E2,3.741267E7,-3.2540635E-4,-8.953963E-3,1.7493458E7,4.4631204E2,2.8531857E-2,2.5130852E2,2.71E2,-2.202736E-2,-6.1311997E-3,-2.968037E-2,-5.8277004E-4,6.6967052E-3,1.5292561E2,2.956111E2,2.084897E-3,-5.9379395E-3,1.422162E3,2.1188422E2,1.7544615E0,2.9565217E0,1.6254545E1,3.167E3,4.2823735E-3,1.7068776E10,1.5277778E0,6.6477684E7,1E0,1.1E1,1.1010101E0,7.696402E3,-1.4698131E-2,-2.4940043E-3,1.4735735E2,1.605076E6,2.4568965E2,3.373913E0,3.074398E0,-7.202281E-3,5.1831864E-3,1.0752917E-2,6.9664386E2,-5.824326E-3,-1.7747069E-2,1.5043378E0,6.5612113E-3,2.5400869E5,8.053062E8,3.5675005E3,5.2179517E-3,-4.6420004E-3,3.9125E1,3.3502305E0,1.07882355E2,1.3922E4,1.4363092E-4,8.0509944E5,4.173631E-3,2.87834E5,3.9152692E6,-1.3296179E-3,1.1481482E0,2.466316E-3,2.3801855E-3,9.416779E-3,1.196966E6,6.2136955E0,5.51E2,6.333131E-3,1.6525911E9,1.0717949E1,3.4835741E-3,-3.4297463E-3,1.3654399E-2,7.255779E-3,2.1689147E-2,1.1488178E-2,2.7578741E1,3.2295492E0,-5.7331333E-3,6.44E2,2.4636364E1,-4.0944275E-3,3.2724605E-3,1.0423297E-2,2.1609572E7,1.9470909E2,3.0588236E0,-1.0542079E-2,2.94E2,1.474E3,5.900914E-3,-4.408583E-5,-1.0289645E-3,4.1E1,3.9914E4,1.552356E0,3.0291306E2,6.887749E7,3.5678826E-3,-1.1948405E-3,1.8578552E7,1E0,1.6521566E5,1.9E1,1.0424673E-2,2.453E4,1.5542647E-2,7.141104E-3,1.5845262E-3,-2.1311706E-3,6.1E1,2E0,3.35E2,-1.1105385E-2,-3.5130077E-3,1.96E2,4.3992E4,-6.3128164E-3,1.1483617E-3,-4.359377E-3,3.3924444E5,-1.8513006E-3,-1.3865606E-3,1.1E1,1.2E1,8.69E2,7.59381E-3,1.574E3,2.402121E4,4.8927066E4,1.43E2,8.026256E-3,1.5204346E8,-6.0866047E-3,1.3864E4,-3.2350274E-3,1.6472772E-2,9.477966E-3,1.2E2,1.1662862E8,-4.0402506E-3,1.2820834E7,-9.441583E-3,-3.7248184E-3,-1.8939136E-2,-9.839627E-3,-5.2954275E-3,3.3181882E-4,-1.1807099E-2,-4.137211E-3,-2.1972875E-3,-5.7290085E-3,4.3375455E-3,-5.3100684E-4,-1.5052858E-2,-4.805458E-3,-1.6188619E-3,2.5970554E-3,1.2557821E-4,-6.8897875E-3,2.012465E-3,5.4968237E-3,-2.4611356E-3,-6.8824627E-3,-3.7292093E-3,3.9702593E-4,2.7175676E-3,-3.9368607E-3,-2.8369667E-3,2.6757447E-3,5.9484793E-3,1.1066947E-3,-5.294183E-3,3.1795034E-3,9.655174E-3,2.7038043E-3,8.366757E-3,3.057394E-3],"split_indices":[2,43,17,9,62,53,67,61,3,3,43,0,15,43,7,3,9,1,12,43,0,4,7,0,0,66,4,0,73,2,0,0,0,0,0,67,4,0,0,48,67,68,73,69,2,0,46,68,59,20,10,68,62,0,0,67,12,70,69,71,0,0,0,67,0,0,69,0,48,7,67,0,0,4,68,67,44,0,60,0,12,43,0,68,0,0,0,9,69,2,0,46,71,0,0,0,0,0,0,71,53,0,1,67,0,0,0,60,4,68,0,44,10,0,0,0,10,9,69,67,7,0,0,5,6,48,73,0,44,0,0,0,0,2,10,67,0,0,8,1,0,0,0,43,0,0,3,3,2,0,2,48,62,0,0,7,0,10,0,0,0,0,12,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,6.67E2,3.61E2,4.16E2,2.51E2,1.8E1,3.43E2,2.83E2,1.33E2,1.6E1,2.35E2,4E0,1.4E1,2.53E2,9E1,2.75E2,8E0,1.5E1,1.18E2,1.2E1,4E0,1.98E2,3.7E1,6E0,8E0,2.15E2,3.8E1,6E0,8.4E1,2.71E2,4E0,4E0,4E0,5E0,1E1,1.1E2,8E0,6E0,6E0,4E1,1.58E2,2.3E1,1.4E1,2.02E2,1.3E1,9E0,2.9E1,7.2E1,1.2E1,1.03E2,1.68E2,4.2E1,6.8E1,4E0,4E0,1.6E1,2.4E1,1.1E1,1.47E2,1.9E1,4E0,5E0,9E0,1.94E2,8E0,4E0,9E0,7E0,2.2E1,2E1,5.2E1,6E0,6E0,2.2E1,8.1E1,1E1,1.58E2,5E0,3.7E1,9E0,5.9E1,8E0,8E0,1.9E1,5E0,4E0,7E0,8.5E1,6.2E1,1.2E1,7E0,1E2,9.4E1,4E0,5E0,1.7E1,5E0,1E1,1E1,3.5E1,1.7E1,1.3E1,9E0,7.3E1,8E0,4E0,6E0,1.32E2,2.6E1,3.3E1,4E0,1.9E1,4E1,4E0,4E0,5E0,1.4E1,4.3E1,4.2E1,4.8E1,1.4E1,5E0,7E0,2.1E1,7.9E1,3E1,6.4E1,1.7E1,1.8E1,1.1E1,6E0,5E0,4E0,4.3E1,3E1,1.2E2,1.2E1,5E0,2.1E1,2.1E1,1.2E1,4E0,1.5E1,2.3E1,1.7E1,4E0,1E1,2.4E1,1.9E1,5E0,3.7E1,1.5E1,3.3E1,1E1,4E0,1.6E1,5E0,7.3E1,6E0,1E1,2E1,8E0,5.6E1,5E0,1.3E1,3.4E1,9E0,4E0,2.6E1,8.6E1,3.4E1,1.7E1,4E0,1.6E1,5E0,1E1,1.3E1,4E0,6E0,4E0,2E1,1.5E1,4E0,3.3E1,4E0,8E0,7E0,1.1E1,2.2E1,5E0,5E0,5E0,1.1E1,4.5E1,2.8E1,4E0,4E0,1.5E1,4.1E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[-4.8119277E-3,-8.19449E-2,6.674725E-2,-6.0867727E-2,-2.3381202E-1,2.2733299E-2,1.7460561E-1,-1.2819494E-1,4.277983E-3,-1.6598009E-1,-2.7359715E-2,3.1408943E-2,-1.7334734E-1,1.8926386E-2,1.5694648E-1,-2.1095286E-1,-1.0319144E-1,1.5616835E-2,-1.4997721E-2,-1.1989513E-2,-1.11577906E-1,-1.2657218E-1,3.9631903E-2,-2.2440404E-1,-7.457351E-4,1.2680195E-1,2.6176533E-1,-1.866383E-1,-1.768178E-2,-9.140318E-2,-1.4859333E-2,4.396741E-3,-1.7916892E-1,-5.5355404E-2,-1.9814077E-1,-6.1191046E-3,-1.6930033E-2,1.6500445E-2,7.585727E-2,-5.640702E-3,-1.2937548E-2,6.5104023E-4,1.4319506E-1,6.5657133E-3,2.907409E-1,-1.1193723E-2,-1.2528831E-1,-7.586996E-2,-1.7860232E-1,-4.750737E-3,1.5629835E-2,-2.0091496E-2,-9.818152E-2,-9.136819E-2,3.94363E-3,-5.0382144E-3,-1.1096451E-2,-3.6236395E-3,2.2583473E-3,4.433173E-2,-6.4346627E-3,8.827237E-2,-7.4022366E-3,8.998234E-2,-6.165467E-3,1.2226552E-1,1.475295E-2,1.4521345E-2,6.388002E-3,-1.7084765E-3,-1.6992027E-1,4.4337574E-2,-8.4808104E-2,-1.2674099E-1,-1.5705595E-2,-6.724795E-2,2.1932188E-2,-7.828827E-3,-3.741805E-2,-1.2933938E-1,1.5349538E-3,2.5929652E-2,1.1204304E-1,-3.1998172E-2,3.6123767E-2,6.0704805E-2,1.4671421E-1,1.4191905E-4,7.3408578E-3,1.4411539E-1,1.8676901E-2,-3.9969385E-3,-9.377069E-3,-1.6101873E-3,4.755746E-3,-1.0512735E-1,-4.3142404E-2,-1.4917591E-1,-1.5748345E-3,-8.000582E-3,-4.8192218E-2,-2.1289203E-2,5.2353542E-2,-4.263576E-3,3.28917E-4,-2.9202914E-3,-7.90162E-3,3.888936E-2,-7.663315E-3,3.050958E-2,8.130047E-3,4.236355E-3,-1.1095308E-1,-2.3079947E-2,8.204179E-2,1.319442E-2,1.1738741E-1,2.2552916E-1,1.04881756E-1,1.1253367E-1,2.1508346E-1,-2.669938E-3,7.798438E-2,-6.846816E-2,-1.3848919E-1,6.092308E-2,-6.866946E-2,-8.476732E-3,-3.8694555E-3,-2.4941197E-2,-1.24132805E-1,-9.40167E-2,2.4256876E-2,7.8029774E-2,-1.8609444E-2,2.3689168E-2,8.667296E-2,-9.649653E-4,3.1533272E-3,-1.5281502E-2,4.482122E-3,-1.7335914E-1,1.2810153E-3,-4.263336E-3,2.1281246E-2,4.1914728E-2,8.280619E-3,3.435392E-2,-5.482454E-3,1.6697888E-1,4.7498997E-2,5.488922E-3,1.2000602E-2,3.7214328E-2,1.4040731E-1,1.7796963E-1,5.8243457E-2,1.3032056E-2,1.4699897E-1,6.570476E-3,5.292767E-4,-1.0127713E-2,-2.1635348E-3,-6.853345E-3,-1.5673038E-3,4.7726654E-3,2.9359382E-4,6.57511E-5,-4.13437E-3,-2.3574363E-3,1.3467293E-3,-1.02446E-3,-8.280099E-3,-5.945158E-3,1.9709137E-3,5.0180973E-3,-1.858477E-3,2.5033061E-3,6.4490414E-3,-2.0649957E-3,3.4100832E-3,2.2316587E-3,-5.1489077E-4,5.842822E-3,5.429285E-4,-1.4401516E-3,4.0613585E-3,-3.6001569E-3,-1.3163268E-2,3.9859163E-3,-1.3917804E-3,2.867904E-3,-7.010903E-5,-1.741727E-3,3.0876757E-3,9.491881E-3,5.2937944E-3,-2.8379129E-3,3.8349226E-3,-1.7605657E-3,3.9592516E-3,7.992866E-3,9.720368E-4,1.0105526E-2,3.1918418E-3,-1.1080394E-3,3.8449534E-3,9.017076E-3,2.6915255E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,-1,31,-1,33,35,37,39,-1,41,43,45,-1,47,-1,49,51,53,55,57,-1,59,61,-1,-1,63,65,-1,67,-1,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,81,83,85,-1,87,-1,89,-1,-1,-1,-1,91,93,95,97,-1,99,101,-1,103,105,-1,107,109,111,113,115,117,-1,-1,119,121,-1,-1,-1,-1,123,125,127,-1,-1,129,131,133,-1,-1,-1,-1,135,-1,137,-1,139,141,143,145,147,149,151,153,155,157,-1,159,161,163,165,167,-1,-1,169,171,173,175,177,179,181,183,-1,-1,185,-1,187,-1,-1,189,191,-1,193,-1,195,197,-1,-1,199,201,203,205,-1,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5857563E0,1.5446205E0,2.488349E0,1.8821764E0,1.3360584E0,6.416659E-1,5.6666136E-1,4.1484094E-1,1.413979E0,2.3056722E-1,0E0,4.7138312E-1,1.24800056E-1,0E0,4.186468E-1,1.2904906E-1,3.9291716E-1,0E0,6.630623E-1,0E0,1.5450239E-1,5.178684E-1,2.8532565E-1,3.4111798E-2,0E0,2.3359144E-1,6.615305E-2,1.1987877E-1,0E0,1.9910324E-1,0E0,5.7280076E-1,4.1292566E-1,1.1928689E-1,2.2502065E-2,5.554225E-2,0E0,1.343311E-1,4.0396422E-1,0E0,0E0,1.7972594E-1,3.2960844E-1,0E0,4.2825222E-2,0E0,8.436707E-2,1.4880842E-1,1.4937681E-1,3.0664724E-1,0E0,0E0,7.1617976E-2,8.945426E-2,0E0,0E0,0E0,0E0,0E0,1.154675E-1,1.2735727E-1,1.9570339E-1,0E0,5.3415954E-2,0E0,2.0327759E-1,0E0,0E0,0E0,0E0,2.378866E-2,4.9995825E-2,1.0213816E-1,3.4409463E-2,0E0,1.0267411E-1,1.7073062E-1,0E0,2.7538143E-2,2.7789697E-2,0E0,1.9536744E-1,9.8888785E-2,2.093153E-1,1.2150829E-1,2.3543623E-1,1.08227015E-1,0E0,0E0,1.4389288E-1,8.160494E-2,0E0,0E0,0E0,0E0,9.325147E-2,1.1841242E-1,1.8297434E-2,0E0,0E0,8.16574E-2,1.8115298E-1,1.3992882E-1,0E0,0E0,0E0,0E0,5.0241813E-2,0E0,2.082568E-2,0E0,9.383805E-2,2.0386156E-1,6.336149E-2,8.9992106E-2,1.4079614E-1,1.3070345E-1,1.662612E-2,6.2084675E-2,1.8147123E-1,5.8892608E-2,0E0,4.110597E-2,1.2673208E-1,3.927654E-2,2.015857E-2,5.1206112E-2,0E0,0E0,5.5108212E-2,5.7815403E-2,1.0499832E-1,1.8933123E-1,7.464054E-2,5.393332E-2,4.8557434E-2,4.9202308E-2,0E0,0E0,7.325361E-2,0E0,1.4814809E-1,0E0,0E0,4.603523E-2,1.6732477E-2,0E0,1.03154555E-1,0E0,2.2823393E-2,7.472238E-2,0E0,0E0,4.3438476E-2,6.640881E-2,9.1423035E-2,6.56774E-2,0E0,4.4210047E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,44,44,46,46,47,47,48,48,49,49,52,52,53,53,59,59,60,60,61,61,63,63,65,65,70,70,71,71,72,72,73,73,75,75,76,76,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,89,89,90,90,95,95,96,96,97,97,100,100,101,101,102,102,107,107,109,109,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,125,125,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,141,141,144,144,145,145,147,147,149,149,150,150,153,153,154,154,155,155,156,156,158,158],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,-1,32,-1,34,36,38,40,-1,42,44,46,-1,48,-1,50,52,54,56,58,-1,60,62,-1,-1,64,66,-1,68,-1,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,82,84,86,-1,88,-1,90,-1,-1,-1,-1,92,94,96,98,-1,100,102,-1,104,106,-1,108,110,112,114,116,118,-1,-1,120,122,-1,-1,-1,-1,124,126,128,-1,-1,130,132,134,-1,-1,-1,-1,136,-1,138,-1,140,142,144,146,148,150,152,154,156,158,-1,160,162,164,166,168,-1,-1,170,172,174,176,178,180,182,184,-1,-1,186,-1,188,-1,-1,190,192,-1,194,-1,196,198,-1,-1,200,202,204,206,-1,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.072304E5,1E0,3.921E3,6.83E2,9.639872E0,2.8088E1,7.9E1,3.89E2,2.00087E5,5E0,-2.7359715E-2,1.45064E5,1.1997242E0,1.8926386E-2,5.1167645E6,6.737403E7,1.2218E4,1.5616835E-2,1.1290322E1,-1.1989513E-2,1.8414415E1,1.1015E4,1E0,1.8E1,-7.457351E-4,1E0,8.0605554E2,3.6E1,-1.768178E-2,1.4E1,-1.4859333E-2,5.428175E3,1.5E1,1E0,1.25E0,2.0467092E6,-1.6930033E-2,1.5723623E0,3.174172E5,-5.640702E-3,-1.2937548E-2,2.4661344E5,4.43837E0,6.5657133E-3,1.6E-3,-1.1193723E-2,2.7444445E1,8.414097E-1,2E1,1.8988005E5,1.5629835E-2,-2.0091496E-2,6.593462E4,1.3E1,3.94363E-3,-5.0382144E-3,-1.1096451E-2,-3.6236395E-3,2.2583473E-3,1.047E3,1.2839025E1,2.308821E6,-7.4022366E-3,1.3388E4,-6.165467E-3,1.4078995E2,1.475295E-2,1.4521345E-2,6.388002E-3,-1.7084765E-3,5.3E1,3.56E2,1.3823239E2,7.997723E6,-1.5705595E-2,8.01E2,4.220534E5,-7.828827E-3,4.727453E7,1.3439851E7,1.5349538E-3,1.3062992E1,1.411E3,8E0,1.6802971E6,7.5491136E-1,7E0,1.4191905E-4,7.3408578E-3,1.6466942E3,3.4915986E3,-3.9969385E-3,-9.377069E-3,-1.6101873E-3,4.755746E-3,1.257E4,5.8114325E5,6.946813E7,-1.5748345E-3,-8.000582E-3,3.154992E7,1.159499E6,2.847E4,-4.263576E-3,3.28917E-4,-2.9202914E-3,-7.90162E-3,3.0291306E2,-7.663315E-3,1.1E1,8.130047E-3,4.39776E6,2.84E2,3.8343322E6,1.670046E1,4.76E2,3.8275862E0,2.379747E0,4.569524E2,4.320314E8,1.0851E4,-2.669938E-3,5.43E2,3.964269E3,2.3E1,1.4865319E5,1.1146532E8,-8.476732E-3,-3.8694555E-3,2.1340163E8,1.670046E1,5.821732E2,2.232008E-2,1.01573E5,8.076923E0,2.4915715E-3,1.500502E6,-9.649653E-4,3.1533272E-3,1.574E3,4.482122E-3,2.9077E4,1.2810153E-3,-4.263336E-3,1.1643481E6,4.646154E0,8.280619E-3,1.158E3,-5.482454E-3,9E0,4.857143E0,5.488922E-3,1.2000602E-2,4.0705118E6,3.43E2,1.3240695E3,7.618E3,1.3032056E-2,1.3091103E0,6.570476E-3,5.292767E-4,-1.0127713E-2,-2.1635348E-3,-6.853345E-3,-1.5673038E-3,4.7726654E-3,2.9359382E-4,6.57511E-5,-4.13437E-3,-2.3574363E-3,1.3467293E-3,-1.02446E-3,-8.280099E-3,-5.945158E-3,1.9709137E-3,5.0180973E-3,-1.858477E-3,2.5033061E-3,6.4490414E-3,-2.0649957E-3,3.4100832E-3,2.2316587E-3,-5.1489077E-4,5.842822E-3,5.429285E-4,-1.4401516E-3,4.0613585E-3,-3.6001569E-3,-1.3163268E-2,3.9859163E-3,-1.3917804E-3,2.867904E-3,-7.010903E-5,-1.741727E-3,3.0876757E-3,9.491881E-3,5.2937944E-3,-2.8379129E-3,3.8349226E-3,-1.7605657E-3,3.9592516E-3,7.992866E-3,9.720368E-4,1.0105526E-2,3.1918418E-3,-1.1080394E-3,3.8449534E-3,9.017076E-3,2.6915255E-3],"split_indices":[43,6,2,2,69,61,0,1,5,6,0,7,53,0,43,7,44,0,69,0,73,9,27,3,0,27,4,2,0,8,0,67,8,16,68,43,0,68,48,0,0,48,50,0,72,0,4,68,3,43,0,0,43,8,0,0,0,0,0,2,73,43,0,2,0,73,0,0,0,0,44,2,67,9,0,2,66,0,7,60,0,71,44,8,43,53,3,0,0,67,4,0,0,0,0,12,60,7,0,0,12,9,10,0,0,0,0,67,0,3,0,9,0,60,71,0,69,69,67,5,2,0,8,43,3,43,46,0,0,7,71,4,53,1,69,53,9,0,0,2,0,1,0,0,43,71,0,2,0,8,69,0,0,43,0,67,10,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,4.86E2,5.24E2,4.28E2,5.8E1,3.73E2,1.51E2,2.1E2,2.18E2,5E1,8E0,3.58E2,1.5E1,9E0,1.42E2,4.7E1,1.63E2,1.1E1,2.07E2,1.7E1,3.3E1,1.7E1,3.41E2,1.1E1,4E0,1.12E2,3E1,4.3E1,4E0,1.56E2,7E0,1.86E2,2.1E1,2.1E1,1.2E1,1.2E1,5E0,2.09E2,1.32E2,5E0,6E0,1.3E1,9.9E1,7E0,2.3E1,2.1E1,2.2E1,1.34E2,2.2E1,1.82E2,4E0,4E0,1.7E1,1.7E1,4E0,5E0,7E0,5E0,7E0,9.4E1,1.15E2,1.26E2,6E0,8E0,5E0,9E1,9E0,1.9E1,4E0,8E0,1.4E1,9E0,1.25E2,1.8E1,4E0,5.4E1,1.28E2,7E0,1E1,1.3E1,4E0,7.5E1,1.9E1,7.2E1,4.3E1,8.7E1,3.9E1,4E0,4E0,7.4E1,1.6E1,5E0,9E0,4E0,5E0,8.3E1,4.2E1,1.4E1,4E0,7E0,4.7E1,5.3E1,7.5E1,4E0,6E0,6E0,7E0,7.1E1,4E0,9E0,1E1,5E1,2.2E1,1.9E1,2.4E1,4.8E1,3.9E1,1.2E1,2.7E1,5.3E1,2.1E1,7E0,9E0,4.1E1,4.2E1,8E0,3.4E1,8E0,6E0,3.7E1,1E1,2E1,3.3E1,5.5E1,2E1,5.5E1,1.6E1,4E0,5E0,4.2E1,8E0,1.5E1,7E0,7E0,1.2E1,1.8E1,6E0,4.2E1,6E0,2.2E1,1.7E1,4E0,8E0,1E1,1.7E1,2.3E1,3E1,9E0,1.2E1,4E0,5E0,4E0,3.7E1,3.8E1,4E0,4E0,4E0,8E0,2.6E1,2.5E1,1.2E1,4E0,6E0,1.6E1,4E0,1.4E1,1.9E1,4.1E1,1.4E1,1.6E1,4E0,3.2E1,2.3E1,1E1,6E0,3.7E1,5E0,9E0,6E0,5E0,7E0,1.2E1,6E0,1.3E1,2.9E1,1.1E1,1.1E1,4E0,1.3E1,4E0,6E0,1.3E1,4E0,1.6E1,7E0,7E0,2.3E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"209","size_leaf_vector":"1"}},{"base_weights":[-5.0273514E-4,-1.2543014E-1,4.433076E-2,-1.0155056E-1,-2.711262E-1,1.177078E-2,2.009337E-1,-1.5354086E-1,-3.6586028E-2,-4.6052542E-1,-1.8392186E-1,-3.558003E-2,4.612676E-2,2.3762597E-2,1.7290747E-1,-1.427859E-1,-1.79971E-2,2.6996106E-1,-8.72707E-2,-6.3834502E-3,-2.8873853E-2,-2.0180507E-1,-2.9971502E-3,-2.6035894E-2,-1.488079E-1,3.1877626E-2,1.5861045E-1,2.0439933E-1,-3.5970823E-3,-1.948772E-1,-1.0245779E-1,5.1168823E-3,1.6806778E-2,-5.2321207E-2,-2.1511768E-1,-4.169671E-3,-2.3586677E-1,-9.047526E-3,-8.3322965E-2,-7.8002766E-2,-1.243493E-2,1.1171669E-1,1.732415E-2,-1.1329408E-3,1.796398E-1,3.4045517E-1,1.64429E-1,5.8520515E-2,-6.834624E-3,-2.168251E-1,-2.0474137E-3,-8.677662E-2,-1.3639349E-2,-8.476217E-2,3.0500285E-2,-1.1320735E-2,-3.8030786E-3,-1.2915283E-2,-6.1682626E-3,-3.2096844E-2,1.8989265E-2,-1.1183121E-2,-6.612635E-2,-5.1507945E-3,2.1135827E-4,7.6993674E-2,9.329905E-3,-9.390459E-2,2.8018365E-2,1.2619777E-2,1.4764638E-1,6.0773E-3,1.7535836E-2,-3.4592901E-3,1.8138115E-1,3.177586E-4,5.0440314E-3,-1.8143508E-1,-2.7184168E-1,-4.1874185E-2,-1.2552673E-1,-1.4925282E-1,-4.6230275E-2,6.256472E-2,-3.9093117E-3,1.5024849E-2,-6.891583E-2,8.406424E-3,8.694847E-3,1.886704E-2,-9.5699534E-2,9.940017E-2,1.499649E-2,3.931263E-2,-2.54917E-1,3.2674097E-2,-9.46087E-3,9.450849E-2,2.1504937E-1,1.3978823E-1,2.4620543E-1,-9.693711E-2,-9.545233E-3,-1.5372083E-2,-7.8240605E-3,3.4368723E-3,-7.603807E-2,-1.5857081E-1,-5.4866362E-2,-2.7530256E-3,-7.81642E-3,-6.396603E-2,3.207615E-3,5.728842E-4,4.4554253E-3,-2.3151629E-2,5.7724264E-2,-1.1677748E-2,-4.284088E-2,7.073717E-2,-1.8669238E-2,5.03168E-3,-1.295545E-3,-5.0552033E-2,-1.3818641E-1,7.60942E-4,1.1310651E-1,2.848938E-3,-1.5787615E-3,-1.7716123E-3,4.0062843E-3,-1.9272873E-2,-2.3181534E-3,9.634987E-3,7.565743E-2,6.3074734E-3,1.7241014E-3,1.1610202E-2,5.267861E-3,1.6217683E-1,4.9737714E-2,2.9040825E-1,1.514912E-1,-7.0272554E-3,-1.0338688E-3,-9.687571E-2,1.4254802E-3,-8.141663E-3,-3.5213497E-3,-8.661685E-4,-4.3371185E-3,-6.5384167E-3,-1.0749666E-1,2.9847318E-2,-6.0474083E-2,1.2997195E-2,6.2078577E-3,-7.10495E-2,2.183724E-3,1.2312038E-4,9.706163E-2,2.4653893E-2,-6.442012E-2,-4.548244E-3,-7.940892E-4,-4.016721E-3,-8.113032E-3,6.0717477E-3,9.642598E-4,6.811439E-2,-1.3134049E-2,-1.2004864E-2,9.938767E-2,1.7437801E-1,3.1434253E-3,5.0107064E-3,-7.980086E-4,8.947797E-3,1.6891176E-2,8.7557975E-3,3.1391324E-3,-5.445959E-3,-1.6181574E-3,-2.544946E-3,1.6982077E-3,-8.188024E-3,-2.9414995E-3,-1.0493287E-3,2.916391E-3,-4.929272E-3,-3.091876E-4,-2.236729E-3,2.2427244E-3,-1.4561897E-3,-5.2550673E-3,1.7367838E-3,-1.5441832E-3,5.4332195E-3,9.2994294E-4,-5.3042206E-3,2.3342576E-3,-1.824139E-3,-6.8961596E-3,1.2160112E-3,5.499531E-3,-4.0633734E-3,3.8748537E-4,-3.1079208E-3,1.5848759E-3,7.897111E-3,2.1629792E-3,8.711605E-3,4.3768445E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,61,63,-1,65,67,-1,69,71,73,75,-1,77,-1,79,-1,81,83,-1,-1,-1,-1,85,87,-1,89,-1,-1,91,-1,93,95,-1,97,-1,-1,-1,99,-1,-1,101,103,105,107,109,111,113,-1,115,117,-1,119,121,123,125,127,129,131,133,-1,135,137,139,141,143,-1,-1,-1,-1,145,147,149,-1,-1,151,-1,-1,-1,153,155,-1,157,159,161,-1,-1,163,165,-1,167,-1,-1,-1,-1,-1,-1,169,171,-1,-1,-1,-1,173,175,177,179,-1,-1,181,-1,-1,-1,-1,-1,183,185,187,189,191,-1,193,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,203,205,207,209,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6625023E0,9.0140295E-1,3.7912822E0,7.7459097E-1,5.264075E-1,1.0053666E0,1.0354099E0,2.6928306E-1,1.6394327E0,4.968083E-1,4.4389427E-2,2.7709788E-1,5.687446E-1,0E0,6.6219187E-1,2.4052572E-1,0E0,1.8377757E-1,3.908816E-1,0E0,0E0,6.610286E-2,0E0,2.337907E-1,1.4943555E-1,3.6872214E-1,1.5845251E-1,4.9327755E-1,1.790143E-1,1.6552782E-1,1.920293E-1,0E0,0E0,1.9603552E-1,5.5333853E-2,0E0,3.8695753E-2,1.214543E-1,1.3413253E-1,3.922581E-2,0E0,1.3797683E-1,3.249867E-1,0E0,6.738305E-2,1.2984681E-1,3.3812308E-1,3.5875898E-2,0E0,4.0241957E-2,0E0,1.13061905E-1,0E0,1.22317314E-1,8.281002E-2,0E0,0E0,0E0,0E0,1.7973764E-1,1.4102226E-1,0E0,1.2999536E-1,0E0,0E0,4.990752E-2,0E0,5.248125E-1,2.7549082E-1,0E0,8.354396E-2,0E0,0E0,0E0,1.6780066E-1,0E0,0E0,4.5918465E-2,4.350257E-2,1.3673691E-1,7.535648E-2,2.3882598E-2,7.418635E-2,2.7692549E-2,0E0,7.632162E-2,2.6722285E-1,0E0,1.3883686E-1,6.219068E-2,6.5905094E-2,2.8826743E-2,2.734955E-2,5.4866306E-2,3.4448367E-1,2.409037E-1,0E0,3.816046E-2,1.5300751E-2,8.870673E-2,7.32317E-2,3.3617347E-2,0E0,0E0,0E0,0E0,6.0901597E-2,2.1469176E-2,1.6557511E-2,0E0,0E0,7.429293E-2,0E0,0E0,0E0,5.1256258E-2,7.367456E-2,0E0,6.619941E-2,4.454039E-2,1.1456628E-1,0E0,0E0,3.026804E-2,2.1168083E-2,0E0,3.61194E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1393777E-1,1.7801422E-1,0E0,0E0,0E0,0E0,3.17809E-2,4.5158178E-2,6.0135722E-2,2.2072852E-2,0E0,0E0,2.4452105E-2,0E0,0E0,0E0,0E0,0E0,3.145268E-2,4.0673092E-2,2.0674776E-2,3.6574565E-2,3.5096187E-2,0E0,5.0426483E-2,2.7735056E-2,0E0,2.608031E-2,1.14049956E-1,5.33489E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.264222E-2,1.87523E-1,5.093741E-2,2.4400109E-1,1.9424796E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,49,49,51,51,53,53,54,54,59,59,60,60,62,62,65,65,67,67,68,68,70,70,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,98,98,99,99,100,100,101,101,106,106,107,107,108,108,111,111,115,115,116,116,118,118,119,119,120,120,123,123,124,124,126,126,133,133,134,134,139,139,140,140,141,141,142,142,145,145,151,151,152,152,153,153,154,154,155,155,157,157,158,158,160,160,161,161,162,162,169,169,170,170,171,171,172,172,173,173],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,62,64,-1,66,68,-1,70,72,74,76,-1,78,-1,80,-1,82,84,-1,-1,-1,-1,86,88,-1,90,-1,-1,92,-1,94,96,-1,98,-1,-1,-1,100,-1,-1,102,104,106,108,110,112,114,-1,116,118,-1,120,122,124,126,128,130,132,134,-1,136,138,140,142,144,-1,-1,-1,-1,146,148,150,-1,-1,152,-1,-1,-1,154,156,-1,158,160,162,-1,-1,164,166,-1,168,-1,-1,-1,-1,-1,-1,170,172,-1,-1,-1,-1,174,176,178,180,-1,-1,182,-1,-1,-1,-1,-1,184,186,188,190,192,-1,194,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,204,206,208,210,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,2.6E1,5.97E3,5.41E2,3.355864E0,9.24E2,1.059448E7,1E0,3E1,4E0,4.7721977E4,6.4153387E-6,4.5866325E6,2.3762597E-2,7.3274844E9,3.89E2,-1.79971E-2,7.635E3,4.217427E7,-6.3834502E-3,-2.8873853E-2,4.1665E3,-2.9971502E-3,2E0,1.8949389E6,6E0,4.745397E6,3.3382E4,2.5652106E7,2.135961E4,9.687E3,5.1168823E-3,1.6806778E-2,4.8614563E2,9.643839E6,-4.169671E-3,1E0,2.4568965E2,1.43E2,1.034073E6,-1.243493E-2,5.691698E2,1.4598765E0,-1.1329408E-3,1.4076087E0,7.0296685E6,1.36E2,2.8582963E-1,-6.834624E-3,9.55188E5,-2.0474137E-3,1.9668E4,-1.3639349E-2,7.026624E7,3.4779343E1,-1.1320735E-2,-3.8030786E-3,-1.2915283E-2,-6.1682626E-3,5.3061223E0,2.2871187E2,-1.1183121E-2,9.5E1,-5.1507945E-3,2.1135827E-4,2.2461708E6,9.329905E-3,5.404E2,3.8304348E1,1.2619777E-2,1.9290142E0,6.0773E-3,1.7535836E-2,-3.4592901E-3,1.4176E4,3.177586E-4,5.0440314E-3,1.957E3,1.2E1,9.9E1,1.1244019E2,6.07E2,3.2874417E2,3.88E2,-3.9093117E-3,1.1356604E2,8.860256E3,8.406424E-3,2.7563172E6,1.880118E-3,8E0,1.6964E4,4.577732E2,4.0695653E0,3.596E3,2.445E3,-9.46087E-3,2.041E3,6.7553375E4,5.657353E1,1.2792593E2,3.2E1,-9.545233E-3,-1.5372083E-2,-7.8240605E-3,3.4368723E-3,7.6811076E-3,3.3025316E5,2.241E2,-2.7530256E-3,-7.81642E-3,1.9545455E0,3.207615E-3,5.728842E-4,4.4554253E-3,3.5714287E-1,2.8361E4,-1.1677748E-2,1.6223962E7,2.7777777E0,2.90099E0,5.03168E-3,-1.295545E-3,6.23E2,3.125E0,7.60942E-4,1.4521204E0,2.848938E-3,-1.5787615E-3,-1.7716123E-3,4.0062843E-3,-1.9272873E-2,-2.3181534E-3,9.777778E0,3.7043128E5,6.3074734E-3,1.7241014E-3,1.1610202E-2,5.267861E-3,1.0905187E9,4.1104166E8,4.73909E0,6.2136906E-1,-7.0272554E-3,-1.0338688E-3,2.49E2,1.4254802E-3,-8.141663E-3,-3.5213497E-3,-8.661685E-4,-4.3371185E-3,1.4929079E2,5.25E0,4.49239E6,1.4141E4,1.407037E2,6.2078577E-3,1.5132743E0,1.901875E2,1.2312038E-4,2.4361508E6,3.7974394E-6,1.676E3,-4.548244E-3,-7.940892E-4,-4.016721E-3,-8.113032E-3,6.0717477E-3,9.642598E-4,1.9882522E0,3.0291306E2,1.02E2,7.202917E7,2.6E1,3.1434253E-3,5.0107064E-3,-7.980086E-4,8.947797E-3,1.6891176E-2,8.7557975E-3,3.1391324E-3,-5.445959E-3,-1.6181574E-3,-2.544946E-3,1.6982077E-3,-8.188024E-3,-2.9414995E-3,-1.0493287E-3,2.916391E-3,-4.929272E-3,-3.091876E-4,-2.236729E-3,2.2427244E-3,-1.4561897E-3,-5.2550673E-3,1.7367838E-3,-1.5441832E-3,5.4332195E-3,9.2994294E-4,-5.3042206E-3,2.3342576E-3,-1.824139E-3,-6.8961596E-3,1.2160112E-3,5.499531E-3,-4.0633734E-3,3.8748537E-4,-3.1079208E-3,1.5848759E-3,7.897111E-3,2.1629792E-3,8.711605E-3,4.3768445E-3],"split_indices":[43,3,2,2,73,2,12,6,0,8,43,52,43,0,5,1,0,1,60,0,0,62,0,32,60,3,60,9,58,43,44,0,0,4,62,0,26,70,2,9,0,67,69,0,69,62,0,49,0,46,0,12,0,7,71,0,0,0,0,73,67,0,10,0,0,43,0,4,68,0,68,0,0,0,2,0,0,9,3,44,67,2,67,10,0,67,62,0,60,53,8,1,4,71,2,2,0,2,48,71,73,2,0,0,0,0,53,62,4,0,0,68,0,0,0,68,1,0,5,69,69,0,0,2,71,0,53,0,0,0,0,0,0,73,43,0,0,0,0,7,7,50,49,0,0,2,0,0,0,0,0,67,69,43,9,67,0,68,4,0,60,53,44,0,0,0,0,0,0,68,67,0,12,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.009E3,2.66E2,7.43E2,2.3E2,3.6E1,6.16E2,1.27E2,1.27E2,1.03E2,1E1,2.6E1,2.59E2,3.57E2,9E0,1.18E2,1.23E2,4E0,1.4E1,8.9E1,4E0,6E0,2.2E1,4E0,2.4E2,1.9E1,3.18E2,3.9E1,1E2,1.8E1,5.2E1,7.1E1,6E0,8E0,7.1E1,1.8E1,6E0,1.6E1,1.86E2,5.4E1,1.3E1,6E0,4.8E1,2.7E2,4E0,3.5E1,2.1E1,7.9E1,1.3E1,5E0,4.5E1,7E0,6.7E1,4E0,5.1E1,2E1,1.4E1,4E0,1E1,6E0,1.02E2,8.4E1,4E0,5E1,9E0,4E0,3.6E1,1.2E1,2.3E1,2.47E2,7E0,2.8E1,4E0,1.7E1,5E0,7.4E1,7E0,6E0,3E1,1.5E1,3.2E1,3.5E1,1.8E1,3.3E1,1.6E1,4E0,4.5E1,5.7E1,4E0,8E1,1.3E1,3.7E1,2.6E1,1E1,1.3E1,1E1,2.43E2,4E0,1.7E1,1.1E1,4.7E1,2.7E1,8E0,2.2E1,8E0,7E0,7E0,2.5E1,2.3E1,1.2E1,4E0,1.4E1,2.9E1,4E0,7E0,9E0,2.4E1,2.1E1,6E0,5.1E1,2.4E1,5.6E1,4E0,9E0,1.9E1,1.8E1,4E0,2.2E1,5E0,5E0,5E0,8E0,5E0,5E0,1.59E2,8.4E1,9E0,8E0,7E0,4E0,3.7E1,1E1,1.7E1,1E1,4E0,4E0,2.1E1,4E0,1.8E1,5E0,7E0,5E0,1.3E1,1.6E1,1E1,1.4E1,1.4E1,7E0,3.1E1,2E1,7E0,1.7E1,2.9E1,2.7E1,7E0,1.2E1,9E0,9E0,1.8E1,4E0,4.4E1,1.15E2,1.8E1,6.6E1,3.2E1,5E0,5E0,5E0,9E0,8E0,6E0,4E0,1.5E1,6E0,6E0,7E0,5E0,1.1E1,4E0,6E0,7E0,7E0,5E0,9E0,1.7E1,1.4E1,1E1,1E1,1.3E1,4E0,4E0,2.5E1,2.2E1,5E0,2.5E1,1.9E1,2.5E1,9E1,8E0,1E1,2.7E1,3.9E1,2.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[1.1542375E-3,-5.76815E-2,1.068986E-1,-1.3268349E-1,-2.0550696E-2,-1.4995354E-1,1.2255918E-1,-1.2017352E-1,-2.4729492E-1,-8.86327E-2,1.6061844E-3,1.2959312E-3,-2.1845846E-1,1.8485563E-2,1.1080396E-1,-2.0795041E-1,-1.0434892E-1,-2.0038301E-2,-1.2309187E-1,-6.996615E-2,-1.6445434E-2,-3.311938E-3,1.0969806E-2,-2.9613188E-1,-3.524436E-3,9.177153E-2,2.6950178E-1,-6.725694E-3,-1.2040149E-2,9.711703E-3,-1.2355179E-1,-3.3570593E-4,-8.622217E-3,5.130638E-2,-9.074831E-2,-7.682282E-2,9.8082265E-3,-8.458637E-3,-1.721166E-2,8.068845E-2,2.7543744E-1,3.297408E-1,1.3756038E-1,-7.711062E-2,1.174303E-1,-1.127049E-1,-2.63053E-1,-9.3462586E-4,3.957514E-3,-8.916907E-3,-8.114612E-2,5.2357037E-2,-1.096071E-1,4.1632015E-2,-1.2792008E-2,3.5423167E-2,1.19343095E-1,1.7810117E-2,4.3378366E-4,6.9425576E-3,1.6544815E-2,3.8725317E-3,8.606136E-3,-1.6418867E-3,-7.0751617E-3,1.0048811E-2,6.870214E-4,-1.019315E-1,-1.0123459E-2,-1.6845658E-2,-6.216617E-3,2.5020705E-3,-9.610132E-2,-1.5455208E-3,4.904986E-3,-1.4406097E-1,-2.4317175E-2,1.4849249E-2,8.093883E-2,1.4323044E-3,-9.819756E-2,1.2038257E-1,1.4053195E-2,-4.906671E-2,1.3894041E-1,-1.3363954E-1,-6.323991E-2,-1.4279623E-1,-7.180374E-2,-2.0152305E-1,-1.0812687E-1,1.1029331E-3,-3.9046765E-3,-6.527125E-3,2.6111165E-2,1.0386447E-1,3.69592E-2,7.5321584E-3,-7.894034E-3,-3.9806597E-2,-8.828309E-3,1.9458464E-1,1.1414602E-2,2.3166211E-2,-8.321872E-3,3.370835E-2,-1.1756204E-2,1.6452897E-1,2.0416373E-2,-7.5284176E-2,-1.6244921E-1,-2.5253227E-2,-1.0038481E-1,-1.7571834E-1,-1.5262672E-3,8.3889463E-4,-8.176051E-2,-4.400068E-3,-1.3340265E-2,-7.920805E-4,-6.7401864E-3,3.2237347E-3,7.075076E-2,5.254402E-2,6.6081984E-3,3.900055E-3,8.0245466E-4,-1.1625355E-1,1.6852802E-2,-3.7231839E-3,1.1153442E-3,3.4418919E-3,1.2922096E-2,-5.374962E-3,4.150367E-3,2.9706245E-2,-4.1438416E-3,3.5956511E-3,-1.6613628E-4,2.2103499E-1,1.3144529E-1,-5.0971475E-3,7.439299E-2,-4.1886596E-3,2.020825E-4,-8.350153E-3,-4.717289E-3,3.4931733E-3,-2.1971525E-3,-1.4555298E-3,-8.170462E-3,-9.469792E-3,-2.4391005E-3,-4.382608E-3,-7.343796E-4,-1.5869667E-3,2.291051E-3,1.7959994E-3,5.8052777E-3,3.691448E-3,-8.370355E-5,-8.759674E-3,-2.2999875E-4,1.4840086E-3,-1.7780653E-3,-1.6564471E-3,2.0127238E-3,1.1070752E-2,3.3547985E-3,7.1514794E-3,-3.8565067E-3,5.220347E-3,-2.600876E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,-1,31,33,-1,35,-1,37,-1,39,41,-1,-1,43,45,-1,-1,47,49,51,53,-1,-1,55,57,59,61,63,65,67,69,-1,-1,-1,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,-1,-1,-1,-1,87,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,-1,-1,121,123,125,127,-1,129,-1,131,133,135,-1,137,-1,139,141,143,145,147,149,151,-1,-1,153,-1,-1,-1,-1,155,157,159,-1,-1,-1,161,163,-1,-1,-1,-1,-1,-1,165,-1,-1,-1,167,169,-1,171,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.526348E0,1.8742855E0,1.5267992E0,2.9001594E-1,6.8197757E-1,2.7440012E-1,1.1107273E0,2.602744E-1,4.3290412E-1,5.290116E-1,3.9746946E-1,0E0,1.540944E-1,0E0,1.0052257E0,6.059897E-2,3.8231063E-1,0E0,1.0167748E-1,2.69288E-1,0E0,3.2481703E-1,0E0,1.8177211E-2,0E0,6.006639E-1,2.4076939E-1,0E0,0E0,2.52217E-1,1.9581223E-1,0E0,0E0,4.10563E-2,7.271284E-2,2.2172725E-1,2.0623092E-1,0E0,0E0,5.031893E-1,4.942106E-1,6.841874E-2,1.8422857E-2,4.197003E-2,1.1993627E-1,1.3817811E-1,8.145499E-2,0E0,0E0,0E0,1.7517936E-1,5.4347377E-2,1.1697805E-1,1.2407483E-1,2.0423296E-1,2.4341156E-1,5.216763E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4981043E-1,0E0,0E0,0E0,0E0,7.5974226E-2,0E0,0E0,3.8090587E-2,3.9732635E-2,1.3028987E-1,4.455778E-2,1.5368478E-1,1.2257518E-1,2.1609136E-1,1.9705957E-1,2.9761422E-1,4.2124128E-1,1.0545385E-1,8.212565E-2,8.453268E-2,4.8062652E-2,6.521124E-2,6.622201E-2,0E0,0E0,0E0,6.9246545E-2,5.4805994E-2,1.5268533E-2,1.646397E-1,0E0,4.3674428E-2,0E0,1.427058E-1,1.3068056E-1,8.0166966E-2,0E0,2.1676803E-2,0E0,1.8593025E-1,1.9169639E-1,3.1027839E-2,3.2322764E-2,7.3339805E-2,1.5244117E-1,5.7515204E-2,0E0,0E0,3.9054394E-2,0E0,0E0,0E0,0E0,8.18454E-2,3.5369493E-2,2.2451662E-2,0E0,0E0,0E0,8.0680445E-2,1.12776116E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.243001E-2,0E0,0E0,0E0,9.576225E-2,3.8933206E-1,0E0,1.02219135E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,21,21,23,23,25,25,26,26,29,29,30,30,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,50,50,51,51,52,52,53,53,54,54,55,55,56,56,67,67,72,72,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,94,94,95,95,96,96,97,97,99,99,101,101,102,102,103,103,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,116,116,121,121,122,122,123,123,127,127,128,128,135,135,139,139,140,140,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,-1,32,34,-1,36,-1,38,-1,40,42,-1,-1,44,46,-1,-1,48,50,52,54,-1,-1,56,58,60,62,64,66,68,70,-1,-1,-1,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,-1,-1,-1,-1,88,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,-1,-1,122,124,126,128,-1,130,-1,132,134,136,-1,138,-1,140,142,144,146,148,150,152,-1,-1,154,-1,-1,-1,-1,156,158,160,-1,-1,-1,162,164,-1,-1,-1,-1,-1,-1,166,-1,-1,-1,168,170,-1,172,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,4.8465605E3,4.9735293E0,1E0,1E0,3.6E1,3.6E1,2E0,3.5531914E0,7.82261E6,1.2959312E-3,7.978062E6,1.8485563E-2,6.0052995E6,6.48334E5,1E0,-2.0038301E-2,4.0869565E0,5E-1,-1.6445434E-2,1.5292561E2,1.0969806E-2,1.1558494E8,-3.524436E-3,5.428175E3,1.9047619E0,-6.725694E-3,-1.2040149E-2,4.03E2,3.0977E4,-3.3570593E-4,-8.622217E-3,7.52E2,1.4076087E0,3.1488764E0,3.0416667E0,-8.458637E-3,-1.721166E-2,1.1806228E6,4.1325716E7,1.4081731E1,5.5177975E6,6.737099E7,2.00087E5,3.2856784E7,5.27E2,-9.3462586E-4,3.957514E-3,-8.916907E-3,1.6386554E0,1.525E4,1.585814E6,1.058E3,1E0,2.5959E4,7.549744E7,1.7810117E-2,4.3378366E-4,6.9425576E-3,1.6544815E-2,3.8725317E-3,8.606136E-3,-1.6418867E-3,-7.0751617E-3,1.0048811E-2,6.870214E-4,1.8224286E2,-1.0123459E-2,-1.6845658E-2,-6.216617E-3,2.5020705E-3,3.3820656E7,-1.5455208E-3,4.904986E-3,1.1430505E3,4.4091418E4,6.620166E-6,1.1939979E8,2.673585E1,6.763312E7,7.362415E1,3.0070068E7,8.620714E2,2E0,2.772925E3,1.2E1,4.8E0,1.2783505E0,8.238114E2,3.2E1,1.1029331E-3,-3.9046765E-3,-6.527125E-3,2.6086957E0,2.378914E6,7.3333335E-1,3.1706784E0,-7.894034E-3,3.3064186E7,-8.828309E-3,4.204547E5,1.5477113E2,1.6254545E1,-8.321872E-3,2.152E3,-1.1756204E-2,5.6516E4,8.399457E2,3.3857143E1,1.8E6,1.9741463E2,8.7578294E5,8.49E2,-1.5262672E-3,8.3889463E-4,1.2673605E7,-4.400068E-3,-1.3340265E-2,-7.920805E-4,-6.7401864E-3,1.24E3,6.7723384E7,2E0,6.6081984E-3,3.900055E-3,8.0245466E-4,9.24E2,4.3E1,-3.7231839E-3,1.1153442E-3,3.4418919E-3,1.2922096E-2,-5.374962E-3,4.150367E-3,8.2103E4,-4.1438416E-3,3.5956511E-3,-1.6613628E-4,1.683263E7,2.9366477E0,-5.0971475E-3,3.5510652E6,-4.1886596E-3,2.020825E-4,-8.350153E-3,-4.717289E-3,3.4931733E-3,-2.1971525E-3,-1.4555298E-3,-8.170462E-3,-9.469792E-3,-2.4391005E-3,-4.382608E-3,-7.343796E-4,-1.5869667E-3,2.291051E-3,1.7959994E-3,5.8052777E-3,3.691448E-3,-8.370355E-5,-8.759674E-3,-2.2999875E-4,1.4840086E-3,-1.7780653E-3,-1.6564471E-3,2.0127238E-3,1.1070752E-2,3.3547985E-3,7.1514794E-3,-3.8565067E-3,5.220347E-3,-2.600876E-3],"split_indices":[2,43,43,68,29,8,10,2,32,68,43,0,62,0,43,46,21,0,73,68,0,67,0,60,0,67,69,0,0,2,44,0,0,12,69,73,69,0,0,43,66,73,60,7,5,60,0,0,0,0,69,1,9,2,19,9,7,0,0,0,0,0,0,0,0,0,0,4,0,0,0,0,7,0,0,48,48,53,7,68,7,73,58,4,6,62,8,69,68,48,10,0,0,0,69,9,68,69,0,5,0,43,71,69,0,2,0,9,67,67,5,4,60,2,0,0,60,0,0,0,0,44,7,8,0,0,0,2,8,0,0,0,0,0,0,1,0,0,0,60,57,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,6.73E2,3.74E2,2.22E2,4.51E2,2.1E1,3.53E2,2.02E2,2E1,1.1E2,3.41E2,6E0,1.5E1,1.3E1,3.4E2,2.9E1,1.73E2,7E0,1.3E1,1.04E2,6E0,3.35E2,6E0,9E0,6E0,3.05E2,3.5E1,1.5E1,1.4E1,2.5E1,1.48E2,5E0,8E0,1.5E1,8.9E1,5E1,2.85E2,5E0,4E0,2.89E2,1.6E1,2.3E1,1.2E1,1.4E1,1.1E1,1.39E2,9E0,5E0,1E1,6E0,8.3E1,1E1,4E1,1.18E2,1.67E2,1.34E2,1.55E2,1.1E1,5E0,4E0,1.9E1,7E0,5E0,1E1,4E0,5E0,6E0,1.28E2,1.1E1,4E0,5E0,8E0,7.5E1,4E0,6E0,2.8E1,1.2E1,7.1E1,4.7E1,1.44E2,2.3E1,2.6E1,1.08E2,1.6E1,1.39E2,6.9E1,5.9E1,2.4E1,5.1E1,9E0,1.9E1,7E0,5E0,4E0,6.7E1,3E1,1.7E1,1.4E2,4E0,1.5E1,8E0,1.5E1,1.1E1,1.04E2,4E0,1.2E1,4E0,1.14E2,2.5E1,2.4E1,4.5E1,3E1,2.9E1,1.8E1,6E0,5E0,4.6E1,5E0,4E0,6E0,1.3E1,4.5E1,2.2E1,1.4E1,1.6E1,4E0,1.3E1,9E0,1.31E2,9E0,6E0,7E0,8E0,4E0,7E0,9.9E1,5E0,5E0,7E0,4E1,7.4E1,7E0,1.8E1,2E1,4E0,3.3E1,1.2E1,5E0,2.5E1,1.6E1,1.3E1,1.4E1,4E0,3.8E1,8E0,2.5E1,2E1,1.5E1,7E0,9E0,5E0,5E0,4E0,1.03E2,2.8E1,1.7E1,8.2E1,3.5E1,5E0,6.7E1,7E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"173","size_leaf_vector":"1"}},{"base_weights":[-1.3648425E-3,-4.643758E-2,1.2990181E-1,-3.1971816E-2,-2.3962098E-1,3.364901E-1,1.0956251E-1,-1.2884802E-1,-8.204965E-3,-1.8504345E-1,-2.939666E-2,1.7244022E-2,5.982114E-3,-7.7699996E-3,1.2363854E-1,-1.8769374E-1,-9.407046E-2,1.3923354E-2,-1.2019227E-2,-1.1522745E-1,-3.8488916E-1,1.03314966E-1,2.783451E-1,-2.3445815E-1,-1.574157E-1,-4.6898846E-2,-1.2055755E-1,-6.422348E-2,1.646271E-2,-1.4091845E-2,-8.297858E-2,-7.904948E-3,-2.1731677E-2,3.8110703E-2,1.3086286E-1,1.5881384E-2,8.615531E-3,-6.019431E-3,-1.22196E-2,-3.614537E-3,-7.7784923E-3,2.7469278E-3,-6.4881206E-2,-1.2851924E-1,-1.4535735E-3,-5.080084E-2,-1.8941809E-1,-6.767619E-3,9.0190314E-2,-1.3917547E-2,-1.12399995E-1,-1.3047696E-2,6.508588E-2,1.4266464E-1,-5.4888823E-3,2.6243364E-3,-2.8953925E-3,-8.2422554E-2,1.0056944E-3,-1.742598E-1,-1.07081674E-1,-4.0266406E-2,-1.3905594E-1,-1.1049805E-2,-4.96888E-3,-1.6899758E-3,-1.3199069E-2,1.02852836E-1,-4.620193E-3,2.962765E-3,-3.126654E-3,-1.696859E-1,-3.4480296E-2,4.01074E-2,1.11497855E-2,7.116663E-2,1.620457E-1,-4.4580832E-2,-6.3913036E-3,-4.2986376E-3,-9.496773E-3,-7.158731E-2,-7.2409557E-3,-6.275233E-2,5.4375557E-3,-1.781616E-1,-2.1513053E-3,-1.170128E-1,7.2911386E-3,1.4994432E-1,4.31407E-2,-1.079468E-2,-4.642897E-3,-4.1099167E-3,1.1894957E-3,-4.169197E-2,6.3070446E-2,1.0758131E-1,2.5152206E-2,2.0418164E-1,1.200258E-1,-3.435467E-3,3.4748582E-4,-6.092643E-4,-3.9851996E-3,-1.6120221E-2,-8.916895E-2,2.2807406E-2,-5.4019717E-3,-9.701979E-3,-3.9367694E-3,-2.4814813E-3,-1.3328407E-2,-2.5004733E-3,8.146435E-2,1.6666955E-1,4.9268856E-4,-2.0551838E-2,1.0149157E-1,1.0272469E-3,-3.5312236E-3,1.4029053E-1,1.312462E-2,2.1578704E-3,6.3079577E-3,-3.3562683E-4,2.9215373E-3,1.421513E-1,2.563523E-1,1.2189953E-2,1.5568084E-1,-3.504771E-3,9.823864E-4,-2.1566E-3,-5.3018993E-3,4.1064695E-3,-8.5137336E-4,-3.4009924E-3,5.8237165E-3,2.6133207E-3,-8.3855167E-4,5.628635E-3,-2.068917E-3,5.780465E-3,1.1389654E-2,2.8093476E-3,-2.9004205E-3,9.170078E-3,2.5837212E-3,8.167031E-3,1.1392484E-3,-3.0959942E-3,2.1333117E-3,8.1001725E-3,1.2493079E-3,1.4434329E-2,7.401501E-3,6.6161314E-3,-4.207273E-3,4.4999705E-3,1.0532091E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,61,63,65,67,69,71,-1,73,75,-1,-1,-1,77,-1,79,81,83,85,-1,-1,87,-1,89,-1,-1,-1,91,93,95,-1,97,99,101,-1,-1,-1,103,-1,105,107,109,-1,111,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,-1,-1,-1,-1,131,133,135,-1,-1,-1,137,-1,139,141,143,-1,145,147,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.064577E0,2.124577E0,1.0640211E0,1.6359792E0,1.0297062E0,1.2671018E-1,9.590945E-1,2.688551E-1,6.7905635E-1,6.2536657E-1,0E0,0E0,0E0,0E0,6.912625E-1,3.530526E-2,1.0679549E-1,0E0,8.4263045E-1,2.0115706E-1,1.3886237E-1,3.614459E-1,9.212375E-2,2.1862984E-2,1.631403E-2,3.053166E-2,3.6388755E-2,3.264659E-1,6.2917733E-1,0E0,6.5487504E-2,0E0,0E0,5.544626E-1,4.3831038E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.892436E-2,3.9793164E-2,3.232944E-2,0E0,1.6360933E-1,5.167353E-2,3.9608648E-1,2.2222221E-1,5.0494954E-2,9.6814275E-2,0E0,2.4567217E-1,1.7830944E-1,0E0,0E0,0E0,3.8271025E-2,0E0,1.9005895E-2,5.5750042E-2,1.688833E-1,6.0424447E-2,0E0,0E0,2.8671393E-1,0E0,2.2692168E-1,0E0,0E0,0E0,3.405574E-2,3.8384136E-2,1.0030442E-1,0E0,4.9237728E-2,1.6494155E-1,2.2282878E-2,0E0,0E0,0E0,1.9429564E-2,0E0,1.3406757E-1,1.1956794E-1,2.0580828E-2,0E0,3.9209622E-1,1.8685645E-1,1.05379105E-1,1.4331341E-1,0E0,0E0,0E0,0E0,2.8351832E-2,1.5628836E-1,2.3588896E-2,1.8978842E-2,1.3183975E-1,2.1341628E-1,0E0,0E0,0E0,0E0,9.31837E-2,7.07615E-2,1.3515425E-1,0E0,0E0,0E0,1.281458E-1,0E0,2.1179464E-1,1.5835415E-1,1.0520291E-1,0E0,6.909832E-2,7.729587E-2,0E0,0E0,6.314829E-2,7.1961775E-2,0E0,0E0,0E0,0E0,9.4563544E-2,9.55354E-2,2.1597436E-1,1.5473652E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,33,33,34,34,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,57,57,59,59,60,60,61,61,62,62,65,65,67,67,71,71,72,72,73,73,75,75,76,76,77,77,81,81,83,83,84,84,85,85,87,87,88,88,89,89,90,90,95,95,96,96,97,97,98,98,99,99,100,100,105,105,106,106,107,107,111,111,113,113,114,114,115,115,117,117,118,118,121,121,122,122,127,127,128,128,129,129,130,130],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,62,64,66,68,70,72,-1,74,76,-1,-1,-1,78,-1,80,82,84,86,-1,-1,88,-1,90,-1,-1,-1,92,94,96,-1,98,100,102,-1,-1,-1,104,-1,106,108,110,-1,112,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,-1,-1,-1,-1,132,134,136,-1,-1,-1,138,-1,140,142,144,-1,146,148,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,9.932432E0,5.6E1,3.56E2,2E1,1.84E2,8.147158E4,4.54E2,7.336111E4,4.9E1,-2.939666E-2,1.7244022E-2,5.982114E-3,-7.7699996E-3,6.9307615E6,1.2916666E0,1.9E1,1.3923354E-2,4.3827028E5,3.909839E-5,1.1E1,3.3580637E8,3.3382E4,6.2886596E-1,7E0,1.00019E5,7.997723E6,3.2856784E7,4.265829E2,-1.4091845E-2,4.1665E3,-7.904948E-3,-2.1731677E-2,1.3809524E0,7.134432E7,1.5881384E-2,8.615531E-3,-6.019431E-3,-1.22196E-2,-3.614537E-3,-7.7784923E-3,6.695E3,3E0,1.2139681E3,-1.4535735E-3,3.0084E4,1.731E3,2.511E3,1.1043767E3,1.3E1,3.541824E2,-1.3047696E-2,7.459E3,6.9664386E2,-5.4888823E-3,2.6243364E-3,-2.8953925E-3,1.7544615E0,1.0056944E-3,3.3305264E2,4.325E0,3.839424E2,1E0,-1.1049805E-2,-4.96888E-3,5.862757E1,-1.3199069E-2,3.5419354E0,-4.620193E-3,2.962765E-3,-3.126654E-3,2.56E2,1.0118329E4,6.726722E2,1.11497855E-2,4.569524E2,1.5918367E0,1.52E2,-6.3913036E-3,-4.2986376E-3,-9.496773E-3,3.2E1,-7.2409557E-3,1.5132743E0,1E0,1.266E3,-2.1513053E-3,9.0217394E-1,1.559287E5,3.4052496E8,1.552E3,-1.079468E-2,-4.642897E-3,-4.1099167E-3,1.1894957E-3,6.63651E4,1.6942337E1,2.6862775E6,5.53648E8,1.1618335E0,3.2608695E0,-3.435467E-3,3.4748582E-4,-6.092643E-4,-3.9851996E-3,7.56484E5,1.0142858E1,5.449711E8,-5.4019717E-3,-9.701979E-3,-3.9367694E-3,7.82E2,-1.3328407E-2,4E0,1.3333334E0,1.09652E5,4.9268856E-4,1.798E3,4.111111E0,1.0272469E-3,-3.5312236E-3,1.3205668E7,2.2715E4,2.1578704E-3,6.3079577E-3,-3.3562683E-4,2.9215373E-3,5E0,2.1089442E0,2.7578741E1,2.4077083E3,-3.504771E-3,9.823864E-4,-2.1566E-3,-5.3018993E-3,4.1064695E-3,-8.5137336E-4,-3.4009924E-3,5.8237165E-3,2.6133207E-3,-8.3855167E-4,5.628635E-3,-2.068917E-3,5.780465E-3,1.1389654E-2,2.8093476E-3,-2.9004205E-3,9.170078E-3,2.5837212E-3,8.167031E-3,1.1392484E-3,-3.0959942E-3,2.1333117E-3,8.1001725E-3,1.2493079E-3,1.4434329E-2,7.401501E-3,6.6161314E-3,-4.207273E-3,4.4999705E-3,1.0532091E-2],"split_indices":[2,69,0,2,6,10,43,1,60,3,0,0,0,0,43,68,10,0,43,53,8,7,9,68,0,5,9,60,67,0,62,0,0,69,59,0,0,0,0,0,0,9,8,48,0,44,2,2,67,0,67,0,2,67,0,0,0,68,0,48,69,4,19,0,0,62,0,69,0,0,0,10,48,4,0,67,68,44,0,0,0,10,0,68,19,2,0,68,48,7,2,0,0,0,0,48,71,62,7,68,69,0,0,0,0,12,73,46,0,0,0,2,0,73,68,1,0,44,69,0,0,9,44,0,0,0,0,6,53,71,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.023E3,7.62E2,2.61E2,7.1E2,5.2E1,2.2E1,2.39E2,1.39E2,5.71E2,4.7E1,5E0,1.8E1,4E0,1.1E1,2.28E2,5E1,8.9E1,6E0,5.65E2,3.6E1,1.1E1,2.03E2,2.5E1,1.7E1,3.3E1,3.3E1,5.6E1,1.99E2,3.66E2,4E0,3.2E1,4E0,7E0,6.1E1,1.42E2,1.3E1,1.2E1,5E0,1.2E1,5E0,2.8E1,9E0,2.4E1,5.1E1,5E0,1.81E2,1.8E1,2.79E2,8.7E1,1E1,2.2E1,4E0,5.7E1,1.36E2,6E0,5E0,4E0,2E1,4E0,1.4E1,3.7E1,1.63E2,1.8E1,1E1,8E0,2.75E2,4E0,8.2E1,5E0,4E0,6E0,1.2E1,1E1,5.1E1,6E0,3E1,1.06E2,1.3E1,7E0,5E0,9E0,2.3E1,1.4E1,1.09E2,5.4E1,1.2E1,6E0,1.9E1,2.56E2,4.5E1,3.7E1,5E0,7E0,5E0,5E0,1.1E1,4E1,1.6E1,1.4E1,5.1E1,5.5E1,8E0,5E0,5E0,1.8E1,4E1,6.9E1,4.8E1,6E0,8E0,4E0,1.2E1,7E0,2.27E2,2.9E1,4E1,5E0,1.8E1,1.9E1,4E0,7E0,1.5E1,2.5E1,6E0,1E1,8E0,6E0,2.5E1,2.6E1,1.4E1,4.1E1,1.5E1,2.5E1,2.7E1,4.2E1,1.8E1,3E1,8E0,4E0,4.7E1,1.8E2,2.2E1,7E0,2.8E1,1.2E1,6E0,1.2E1,5E0,1.4E1,1.1E1,4E0,7E0,1.8E1,1.9E1,6E0,1.5E1,1.1E1,6E0,8E0,2.4E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-2.3163706E-3,-4.3742355E-2,1.2968592E-1,-1.1102305E-1,-1.1658012E-2,8.893596E-2,2.077015E-1,-1.0250346E-1,-2.0608572E-2,-6.083685E-3,-2.345919E-1,1.02474086E-1,-8.679268E-3,3.111256E-1,1.2365165E-1,-8.4660165E-2,-1.9096068E-1,-4.335501E-2,2.5062487E-2,-8.14113E-2,-2.1408575E-2,1.307893E-1,2.9094229E-2,2.6379682E-3,3.5844988E-1,1.5902634E-1,4.275372E-3,-9.175615E-2,1.0311958E-2,-1.4437537E-1,-2.2984022E-2,-9.431513E-2,-2.3252202E-2,3.1502713E-2,-1.2919953E-2,1.5573751E-3,-8.327399E-3,-5.9242765E-3,1.4147119E-1,7.192187E-2,-8.044763E-3,1.0673161E-2,1.8483171E-2,2.7062166E-1,1.1890965E-1,4.742078E-3,-2.6675216E-3,-8.077384E-2,-2.2548053E-1,-9.565029E-3,-8.075886E-2,-1.251941E-1,4.3581687E-2,-1.1334203E-1,-1.4552204E-2,8.3757125E-2,7.1165347E-3,1.5431722E-1,-1.1361421E-3,8.1400096E-4,1.1968596E-1,-5.0875306E-2,2.390062E-2,4.9015544E-3,1.760288E-2,3.975848E-2,7.047192E-3,-1.2097469E-1,-2.7006762E-2,-1.4662112E-2,-3.4708255E-3,-4.3994233E-2,-6.594967E-3,-8.983631E-2,-2.1288743E-2,3.6841158E-3,-1.3964837E-3,-4.7106115E-4,-1.6389525E-1,2.4860755E-2,-3.3029135E-2,3.8901668E-2,1.1957992E-1,-3.081001E-3,1.4776576E-1,1.3359898E-2,1.4105667E-1,5.19689E-3,-6.3412455E-3,7.966141E-3,1.012034E-3,-4.2694362E-4,-4.5731384E-3,-1.2872173E-3,2.0908068E-3,-2.7223488E-3,5.3299638E-3,-1.0546174E-2,-1.0783026E-1,2.0469642E-1,-6.1550986E-2,-5.502692E-3,1.097008E-2,-1.0630176E-1,-5.4800945E-5,-3.7516362E-3,-9.49331E-3,-6.4669102E-3,5.0825156E-2,-4.7684547E-2,5.4219753E-2,5.8565225E-2,-2.2882926E-3,1.3279818E-1,1.2547638E-4,-3.92845E-2,2.6936859E-2,9.001348E-3,2.7892513E-3,9.377825E-2,1.853339E-1,-7.85614E-2,-7.4599828E-3,5.0631985E-3,1.3716438E-2,-4.0502585E-2,-1.473619E-1,2.7592587E-3,-1.8470054E-3,-4.9784373E-2,-1.4897035E-1,1.4266383E-3,-1.8684632E-3,3.2081798E-2,4.7341147E-3,-5.59169E-2,2.8386528E-3,4.7767456E-3,6.6411955E-4,-1.1531069E-3,7.7354796E-2,1.02882884E-1,1.962206E-1,-1.3367072E-2,-2.275188E-2,-3.5706526E-3,8.042419E-2,1.3515028E-1,5.812674E-2,1.0124369E-2,8.997898E-2,-4.4659856E-3,-2.0337415E-4,1.0180435E-4,-2.7191702E-3,-8.4482925E-3,-2.1657597E-3,-4.703047E-3,-6.4426457E-4,-1.0158487E-2,-4.609284E-3,-9.525904E-4,2.254283E-3,-8.0058066E-4,-3.372192E-3,9.4284205E-4,5.790955E-3,1.5408425E-4,5.404589E-3,4.3060756E-3,1.0701922E-2,-4.781559E-3,2.3125742E-4,8.74795E-4,-3.619816E-3,1.08540235E-2,2.4840198E-3,3.1417976E-3,9.341854E-3,-3.8126172E-3,4.335523E-3,5.1972517E-3,1.2449378E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,-1,41,43,45,47,-1,49,-1,51,53,55,-1,-1,-1,-1,57,59,61,-1,-1,63,65,-1,-1,67,69,-1,71,73,75,77,79,81,83,85,87,-1,89,91,93,-1,-1,95,-1,97,99,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,109,111,113,115,117,-1,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,121,123,125,-1,127,129,-1,-1,-1,131,133,135,137,139,-1,141,-1,143,145,-1,-1,147,149,151,-1,-1,-1,153,155,-1,-1,157,159,-1,-1,161,-1,163,-1,-1,-1,-1,165,167,169,-1,171,173,175,177,179,-1,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.654453E0,1.6975689E0,7.667732E-1,6.793349E-1,6.62178E-1,6.3382363E-1,6.9172263E-1,3.7684703E-1,0E0,6.0705507E-1,4.1790777E-1,3.2240415E-1,0E0,4.243307E-1,2.0201927E-1,4.7706652E-1,5.1366234E-1,2.4162403E-1,5.681142E-1,1.0848279E-1,0E0,3.30796E-1,7.215449E-2,0E0,7.2004795E-2,1.3346338E-1,7.913114E-2,2.8285062E-1,0E0,1.3077933E-1,0E0,2.907496E-1,1.3316101E-1,3.5620385E-1,0E0,0E0,0E0,0E0,1.9985628E-1,5.2359037E-2,3.54219E-2,0E0,0E0,1.23048484E-1,7.11292E-2,0E0,0E0,4.0957606E-1,1.8339062E-1,0E0,3.9302245E-2,6.1125803E-1,3.7109327E-2,7.654405E-2,1.1588156E-1,1.3835764E-1,2.7583617E-1,1.3804603E-1,1.7011037E-1,0E0,5.2954093E-2,2.1119406E-2,1.7842624E-2,0E0,0E0,8.141511E-2,0E0,1.291213E-1,6.7693174E-1,0E0,0E0,5.9106953E-2,0E0,7.43168E-2,0E0,0E0,0E0,0E0,1.7705292E-2,4.181229E-2,1.4068162E-1,7.419587E-2,7.533151E-2,1.9667351E-1,4.1715175E-2,0E0,1.7730188E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4275563E-1,6.391525E-2,1.2588277E-1,0E0,2.4744278E-2,9.627879E-2,0E0,0E0,0E0,3.1703446E-2,2.3817562E-2,8.698554E-2,2.986348E-2,5.529017E-2,0E0,6.329346E-2,0E0,3.3240622E-1,1.6183195E-1,0E0,0E0,6.1468035E-2,1.318196E-1,8.819911E-2,0E0,0E0,0E0,4.716234E-2,4.137495E-2,0E0,0E0,3.5863835E-2,6.179875E-2,0E0,0E0,2.0383798E-2,0E0,5.606109E-2,0E0,0E0,0E0,0E0,7.3967054E-2,4.453203E-2,2.5852442E-2,0E0,1.7534298E-1,1.0919836E-1,1.3564631E-1,8.080357E-2,1.433868E-1,0E0,1.6839348E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,38,38,39,39,40,40,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,65,65,67,67,68,68,71,71,73,73,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,98,98,99,99,100,100,102,102,103,103,107,107,108,108,109,109,110,110,111,111,113,113,115,115,116,116,119,119,120,120,121,121,125,125,126,126,129,129,130,130,133,133,135,135,140,140,141,141,142,142,144,144,145,145,146,146,147,147,148,148,150,150],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,-1,42,44,46,48,-1,50,-1,52,54,56,-1,-1,-1,-1,58,60,62,-1,-1,64,66,-1,-1,68,70,-1,72,74,76,78,80,82,84,86,88,-1,90,92,94,-1,-1,96,-1,98,100,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,110,112,114,116,118,-1,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,122,124,126,-1,128,130,-1,-1,-1,132,134,136,138,140,-1,142,-1,144,146,-1,-1,148,150,152,-1,-1,-1,154,156,-1,-1,158,160,-1,-1,162,-1,164,-1,-1,-1,-1,166,168,170,-1,172,174,176,178,180,-1,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8439855E5,1.6466942E3,3.8304348E1,1.2674157E1,9.7172376E7,1.7103828E7,1E0,-2.0608572E-2,8.23E2,1E0,8E0,-8.679268E-3,1.5599597E8,2.5130852E2,6.68411E2,7.45E0,1.5933333E2,1.4676277E7,1.4787234E1,-2.1408575E-2,6.6903913E-1,1.4E1,2.6379682E-3,4.48375E5,1.5277778E0,3.85374E8,3.0977E4,1.0311958E-2,5E0,-2.2984022E-2,4.57E2,5.8182236E1,9.538462E0,-1.2919953E-2,1.5573751E-3,-8.327399E-3,-5.9242765E-3,5.2409735E-7,7.601393E2,2.6844707E8,1.0673161E-2,1.8483171E-2,4.420569E1,1.270394E5,4.742078E-3,-2.6675216E-3,3.72E2,1.266E3,-9.565029E-3,1E0,4.5865917E8,3.5714287E-1,5.2224144E5,1.3619632E0,1.188E3,2.779E3,6.229955E4,2.6750835E6,8.1400096E-4,4.4522205E6,1.3342042E0,4.6750406E4,4.9015544E-3,1.760288E-2,8.93E2,7.047192E-3,6.6E0,1.6E1,-1.4662112E-2,-3.4708255E-3,7.026624E7,-6.594967E-3,4.92E2,-2.1288743E-2,3.6841158E-3,-1.3964837E-3,-4.7106115E-4,1.08712E5,2.4346602E2,3.7066666E2,1E0,1.1208894E0,1.273801E6,4.5E0,1.3359898E-2,1.606722E8,5.19689E-3,-6.3412455E-3,7.966141E-3,1.012034E-3,-4.2694362E-4,-4.5731384E-3,-1.2872173E-3,2.0908068E-3,-2.7223488E-3,5.3299638E-3,-1.0546174E-2,6.3623324E-2,1.9E1,4.142857E0,-5.502692E-3,1.6E1,3.1E1,-5.4800945E-5,-3.7516362E-3,-9.49331E-3,1.901875E2,2.12E2,1.6100003E8,9.207547E0,1.4953704E0,-2.2882926E-3,1.2E1,1.2547638E-4,5E0,1.5736016E6,9.001348E-3,2.7892513E-3,2.5054401E1,1.7103828E7,1.99E2,-7.4599828E-3,5.0631985E-3,1.3716438E-2,1.8E0,5.3776E4,2.7592587E-3,-1.8470054E-3,4.9E2,9.502128E1,1.4266383E-3,-1.8684632E-3,5.08E2,4.7341147E-3,1.1074469E6,2.8386528E-3,4.7767456E-3,6.6411955E-4,-1.1531069E-3,1.7143776E7,3.655004E7,2.8655008E2,-1.3367072E-2,3.64E2,1.3705882E1,1.4872362E-1,1.8863014E1,2.9355192E1,1.0124369E-2,1.3695652E0,-4.4659856E-3,-2.0337415E-4,1.0180435E-4,-2.7191702E-3,-8.4482925E-3,-2.1657597E-3,-4.703047E-3,-6.4426457E-4,-1.0158487E-2,-4.609284E-3,-9.525904E-4,2.254283E-3,-8.0058066E-4,-3.372192E-3,9.4284205E-4,5.790955E-3,1.5408425E-4,5.404589E-3,4.3060756E-3,1.0701922E-2,-4.781559E-3,2.3125742E-4,8.74795E-4,-3.619816E-3,1.08540235E-2,2.4840198E-3,3.1417976E-3,9.341854E-3,-3.8126172E-3,4.335523E-3,5.1972517E-3,1.2449378E-3],"split_indices":[2,43,67,68,69,60,62,6,0,2,6,32,0,46,73,67,69,4,62,71,0,68,3,0,1,68,47,44,0,6,0,0,62,73,0,0,0,0,52,67,7,0,0,71,48,0,0,2,2,0,106,5,68,43,68,2,2,48,43,0,62,53,48,0,0,8,0,67,10,0,0,7,0,2,0,0,0,0,7,4,4,19,53,9,69,0,12,0,0,0,0,0,0,0,0,0,0,0,53,0,68,0,3,10,0,0,0,4,10,5,73,69,0,8,0,3,62,0,0,73,62,2,0,0,0,68,1,0,0,1,67,0,0,2,0,60,0,0,0,0,5,7,4,0,10,71,57,73,73,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.032E3,7.86E2,2.46E2,2.53E2,5.33E2,1.63E2,8.3E1,2.48E2,5E0,5.21E2,1.2E1,1.56E2,7E0,3.6E1,4.7E1,2.08E2,4E1,2.37E2,2.84E2,8E0,4E0,1.12E2,4.4E1,6E0,3E1,3.6E1,1.1E1,2.04E2,4E0,3.6E1,4E0,6.6E1,1.71E2,2.79E2,5E0,4E0,4E0,4E0,1.08E2,2E1,2.4E1,9E0,2.1E1,8E0,2.8E1,4E0,7E0,1.9E2,1.4E1,1.7E1,1.9E1,5.4E1,1.2E1,1.4E1,1.57E2,8.8E1,1.91E2,9.9E1,9E0,1E1,1E1,1E1,1.4E1,4E0,4E0,9E0,1.9E1,1.08E2,8.2E1,8E0,6E0,1.3E1,6E0,5E1,4E0,8E0,4E0,5E0,9E0,5E1,1.07E2,4E1,4.8E1,1.79E2,1.2E1,7E0,9.2E1,5E0,4E0,6E0,4E0,6E0,4E0,4E0,1E1,4E0,5E0,1E1,9.8E1,1E1,7.2E1,5E0,8E0,4.2E1,8E0,4E0,5E0,2.3E1,2.7E1,9.2E1,1.5E1,3.3E1,7E0,4.3E1,5E0,8.1E1,9.8E1,7E0,5E0,4.6E1,4.6E1,6.5E1,3.3E1,6E0,4E0,5.9E1,1.3E1,4E0,4E0,1.9E1,2.3E1,1.1E1,1.2E1,2.1E1,6E0,8.6E1,6E0,6E0,9E0,6E0,2.7E1,3.1E1,1.2E1,4E0,7.7E1,6.3E1,3.5E1,2E1,2.6E1,3.3E1,1.3E1,5.2E1,1.3E1,1.8E1,4.1E1,9E0,4E0,7E0,1.2E1,8E0,1.5E1,5E0,1.6E1,2.7E1,5.9E1,1.3E1,1.4E1,4E0,2.7E1,4E0,8E0,1.9E1,5.8E1,4.9E1,1.4E1,4E0,3.1E1,1.1E1,9E0,5E0,2.1E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[1.7595345E-3,-4.604001E-2,9.6235365E-2,-1.1945923E-1,-1.0832919E-2,3.736891E-1,8.147172E-2,-9.8599575E-2,-2.7050975E-1,-6.886885E-3,-1.6953487E-2,8.473743E-3,1.9316973E-2,-1.4762089E-1,9.934878E-2,1.3500729E-1,-1.09373026E-1,-2.1028584E-2,-2.0486724E-1,-4.6014715E-2,1.782059E-2,-1.080661E-1,-1.1470033E-2,5.8647767E-2,1.4758804E-1,1.3014028E-2,-1.7870535E-3,-1.3100278E-1,-5.6821752E-2,-1.0959197E-2,-2.4903528E-3,-1.7599622E-2,-8.317143E-2,4.2048253E-2,-2.7355488E-2,-6.2773395E-3,-1.3967625E-4,6.88182E-2,-5.6689575E-2,1.5992443E-1,-3.3664703E-3,-1.1612602E-1,-2.1794435E-1,-1.2858361E-1,-1.6495831E-2,-9.081926E-2,4.7345733E-4,-6.795864E-2,-1.3194508E-2,1.1827146E-2,9.2539124E-2,-5.6345433E-2,3.4197066E-2,1.7180946E-1,4.736045E-2,2.1479174E-3,-6.4626858E-3,1.9397405E-1,9.4942704E-2,-1.5816578E-1,-7.565414E-2,-2.568005E-1,-3.0287288E-3,-2.9726156E-3,-7.6712733E-3,3.5225672E-3,-4.4709455E-2,-5.1574036E-2,-7.2225532E-3,1.642953E-2,-5.7172E-2,-1.4236178E-1,-3.4530643E-2,-6.482677E-3,2.0018592E-2,1.0792761E-1,-2.0332953E-2,-3.987955E-2,-1.4621301E-1,5.1721134E-3,-4.0996403E-2,1.9274902E-1,2.0934418E-3,-5.3314183E-2,6.4319156E-2,3.2232958E-1,1.6412465E-1,2.9443393E-2,1.5648606E-1,-2.4898688E-3,-1.8300876E-1,3.500623E-2,-1.07007146E-1,-1.3859208E-2,-6.8937135E-3,-6.735104E-2,2.4628008E-3,-4.575952E-3,-1.657172E-4,5.641105E-3,3.8313204E-3,-4.8381225E-3,7.0951093E-4,-7.486937E-2,-1.16012655E-2,-5.3010643E-2,1.7319338E-3,7.063823E-2,-1.6112518E-2,1.2887438E-1,2.989721E-2,-3.2878083E-3,1.5969548E-3,2.4510561E-3,-5.5810045E-2,-8.785496E-3,-2.9672084E-3,-7.760685E-2,1.8044072E-3,4.2414702E-3,9.705485E-3,-8.020313E-3,-5.9973323E-3,1.0236308E-1,2.6608733E-3,5.888042E-3,1.7629962E-2,8.491013E-2,2.1679874E-1,6.883945E-2,-4.6299775E-3,2.3849605E-4,1.8649268E-1,-2.0220388E-1,-2.432959E-3,-2.0935365E-3,4.6620397E-3,-1.6317834E-1,-6.973752E-2,-2.7008492E-2,-4.8713614E-3,3.6684677E-2,-2.1578815E-2,-1.1000487E-1,8.039836E-4,-2.8681986E-2,-5.0548567E-3,6.1379494E-3,4.7287244E-2,-7.388654E-2,2.96053E-2,4.853903E-2,1.5816322E-1,-2.2769286E-3,7.9363555E-2,-4.1126978E-2,-7.236221E-3,-5.611879E-3,-1.3361542E-3,-3.42289E-3,1.9650548E-3,1.2937905E-1,6.474979E-3,-2.7502578E-2,5.5280607E-2,7.678604E-3,4.2154018E-2,2.4456646E-1,6.6431584E-3,4.554645E-3,2.4831984E-2,1.19303465E-2,5.4118815E-3,-6.7189825E-3,-1.1882401E-2,-1.0256936E-2,-4.8764395E-3,1.4483294E-3,-3.9959946E-3,-3.5947496E-3,6.684414E-4,3.0958077E-3,-2.0195788E-3,1.06457E-3,-2.1839691E-3,-6.664084E-3,-1.8509475E-3,-2.122441E-3,7.712547E-4,1.1582002E-3,3.832561E-3,4.7430227E-4,-5.484657E-3,6.6732673E-4,5.716895E-3,-1.0076571E-3,5.6365225E-3,2.6749328E-3,8.013383E-3,6.4799343E-3,1.1980372E-4,-2.738094E-3,9.392785E-4,7.1157697E-3,3.6971763E-4,-5.3268857E-3,3.3406836E-3,-6.0323866E-5,-5.6981924E-3,-4.1920433E-4,5.579428E-3,4.774754E-3,-1.6400062E-3,8.504729E-3,1.454819E-2,-9.786241E-4,3.0436092E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,-1,29,31,33,35,-1,37,39,-1,-1,41,43,-1,-1,45,47,49,51,-1,-1,53,55,57,-1,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,-1,-1,85,87,89,91,93,-1,-1,-1,-1,95,97,-1,99,101,103,105,-1,107,109,111,113,115,-1,117,119,-1,121,123,125,127,129,131,-1,133,135,137,-1,-1,139,-1,-1,-1,141,-1,-1,-1,143,-1,145,-1,147,149,151,153,-1,-1,-1,155,-1,-1,157,-1,-1,-1,159,-1,161,163,-1,-1,165,167,169,-1,-1,171,173,-1,-1,-1,175,177,179,-1,181,183,185,-1,187,-1,-1,189,191,193,195,197,-1,199,201,-1,-1,-1,-1,-1,203,205,207,209,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5971026E0,1.7460828E0,1.3704648E0,6.6336656E-1,6.411064E-1,6.704068E-2,1.3498001E0,5.030161E-1,2.219162E-1,4.3972442E-1,0E0,0E0,0E0,6.872165E-2,5.8625054E-1,2.4019821E-1,2.0397711E-1,0E0,8.976817E-2,1.8435901E-1,3.065188E-1,5.509244E-2,0E0,1.9862425E-1,3.8696432E-1,0E0,0E0,1.441276E-1,1.5993524E-1,0E0,0E0,1.3369673E-1,2.146346E-1,2.766175E-1,1.7679854E-1,0E0,0E0,3.29987E-1,1.2668796E-1,2.7143478E-1,0E0,1.8199694E-1,8.9745104E-2,3.7450433E-2,1.0022424E-1,4.3361157E-2,7.636251E-2,1.7415881E-1,0E0,1.4613466E-1,1.2030578E-1,9.331225E-2,1.9168481E-1,6.0133874E-2,2.2196496E-1,0E0,0E0,2.727611E-1,1.848517E-1,1.326096E-1,2.1191806E-1,2.028513E-2,0E0,0E0,0E0,0E0,6.794954E-2,3.1539E-2,0E0,4.616598E-2,6.368832E-2,1.4267936E-1,6.9863915E-2,0E0,2.0232198E-1,9.421611E-2,2.757657E-2,8.882888E-2,2.382189E-2,0E0,5.2854706E-2,1.9199014E-2,0E0,6.479424E-2,2.5775716E-1,1.24649405E-1,2.7784264E-1,1.294945E-1,1.08014405E-1,0E0,9.943998E-2,7.868431E-2,8.70139E-2,0E0,0E0,3.4033477E-2,0E0,0E0,0E0,4.895681E-2,0E0,0E0,0E0,5.0544694E-2,0E0,5.3123437E-2,0E0,6.0551405E-2,1.7387208E-1,1.0291058E-1,5.8747545E-2,0E0,0E0,0E0,6.757656E-2,0E0,0E0,2.2113673E-2,0E0,0E0,0E0,4.6573278E-2,0E0,1.7531699E-1,6.976964E-2,0E0,0E0,9.7271815E-2,4.9973845E-2,2.2425406E-2,0E0,0E0,6.925994E-2,8.123171E-2,0E0,0E0,0E0,3.759569E-2,5.353117E-2,2.8518198E-2,0E0,6.87595E-2,3.6631614E-2,2.0564884E-2,0E0,2.4261696E-2,0E0,0E0,2.5657915E-2,1.10627115E-1,5.1019773E-2,7.501811E-2,4.2080402E-2,0E0,4.171247E-2,5.1374175E-2,0E0,0E0,0E0,0E0,0E0,1.5730798E-1,1.3661493E-1,7.013682E-2,6.852773E-2,0E0,1.03464484E-1,6.53888E-2,0E0,0E0,1.8395005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,27,27,28,28,31,31,32,32,33,33,34,34,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,66,66,67,67,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,80,80,81,81,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91,92,92,95,95,99,99,103,103,105,105,107,107,108,108,109,109,110,110,114,114,117,117,121,121,123,123,124,124,127,127,128,128,129,129,132,132,133,133,137,137,138,138,139,139,141,141,142,142,143,143,145,145,148,148,149,149,150,150,151,151,152,152,154,154,155,155,161,161,162,162,163,163,164,164,166,166,167,167,170,170],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,-1,30,32,34,36,-1,38,40,-1,-1,42,44,-1,-1,46,48,50,52,-1,-1,54,56,58,-1,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,-1,-1,86,88,90,92,94,-1,-1,-1,-1,96,98,-1,100,102,104,106,-1,108,110,112,114,116,-1,118,120,-1,122,124,126,128,130,132,-1,134,136,138,-1,-1,140,-1,-1,-1,142,-1,-1,-1,144,-1,146,-1,148,150,152,154,-1,-1,-1,156,-1,-1,158,-1,-1,-1,160,-1,162,164,-1,-1,166,168,170,-1,-1,172,174,-1,-1,-1,176,178,180,-1,182,184,186,-1,188,-1,-1,190,192,194,196,198,-1,200,202,-1,-1,-1,-1,-1,204,206,208,210,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.935E3,1.972052E5,3.6E1,2.6E1,5.6E1,2.00033E5,8.147158E4,6E0,5.6190475E1,6.99E2,-1.6953487E-2,8.473743E-3,1.9316973E-2,3.6672198E-8,5.709E3,1.89991E5,8.147158E4,-2.1028584E-2,4.4E1,1.53053E5,2E0,9.234473E-9,-1.1470033E-2,2.0319266E-7,7.06191E7,1.3014028E-2,-1.7870535E-3,2.7E1,1.463E2,-1.0959197E-2,-2.4903528E-3,6.613774E5,1.1949272E7,4.6066E4,9.750085E6,-6.2773395E-3,-1.3967625E-4,2.5959E4,1.1621469E1,2.5503825E6,-3.3664703E-3,3.8714287E1,6.593462E4,6.1E1,9.593451E4,3.1E1,3.3512E4,4.0832064E7,-1.3194508E-2,5.6591835E1,6.1114804E-8,2.8025E4,2.4E1,2.871795E0,5.9754E4,2.1479174E-3,-6.4626858E-3,2.5959E4,1.9616238E2,3.8E1,1.2E1,3.515484E1,-3.0287288E-3,-2.9726156E-3,-7.6712733E-3,3.5225672E-3,6.354E3,4.9E2,-7.2225532E-3,9E0,3.84E2,1.64388E5,1.8E7,-6.482677E-3,1E0,1.0988551E8,5E0,2.4243538E3,1.007E3,5.1721134E-3,1.594E3,2.297E3,2.0934418E-3,1.136E3,1.3922E4,1.4194314E3,8.439E3,1.3475722E2,2.234478E3,-2.4898688E-3,1.16E2,4.1142857E1,1.7467743E-2,-1.3859208E-2,-6.8937135E-3,4.8581E4,2.4628008E-3,-4.575952E-3,-1.657172E-4,7.619497E0,3.8313204E-3,-4.8381225E-3,7.0951093E-4,7.785302E5,-1.16012655E-2,9.776408E5,1.7319338E-3,3.63E2,2.026453E6,2.5912744E2,1.7728971E1,-3.2878083E-3,1.5969548E-3,2.4510561E-3,1.4172E4,-8.785496E-3,-2.9672084E-3,1.4260629E8,1.8044072E-3,4.2414702E-3,9.705485E-3,6.27E2,-5.9973323E-3,2.459733E0,3.2585382E10,5.888042E-3,1.7629962E-2,9.237895E0,1.7103828E7,1.752E3,-4.6299775E-3,2.3849605E-4,8.837831E-2,7.1E1,-2.432959E-3,-2.0935365E-3,4.6620397E-3,4.488263E0,1E0,1.4473684E-1,-4.8713614E-3,3.979933E0,3.6401E4,4.03E2,8.039836E-4,2.4332604E0,-5.0548567E-3,6.1379494E-3,1.0894737E1,2.1525E4,1.8002772E5,3E0,7.619497E0,-2.2769286E-3,1.7790684E5,3.19E2,-7.236221E-3,-5.611879E-3,-1.3361542E-3,-3.42289E-3,1.9650548E-3,4.797448E8,6.9508715E0,8.072743E4,2.3472653E-1,7.678604E-3,2.2007043E0,1.0494994E7,6.6431584E-3,4.554645E-3,2.0805957E-2,1.19303465E-2,5.4118815E-3,-6.7189825E-3,-1.1882401E-2,-1.0256936E-2,-4.8764395E-3,1.4483294E-3,-3.9959946E-3,-3.5947496E-3,6.684414E-4,3.0958077E-3,-2.0195788E-3,1.06457E-3,-2.1839691E-3,-6.664084E-3,-1.8509475E-3,-2.122441E-3,7.712547E-4,1.1582002E-3,3.832561E-3,4.7430227E-4,-5.484657E-3,6.6732673E-4,5.716895E-3,-1.0076571E-3,5.6365225E-3,2.6749328E-3,8.013383E-3,6.4799343E-3,1.1980372E-4,-2.738094E-3,9.392785E-4,7.1157697E-3,3.6971763E-4,-5.3268857E-3,3.3406836E-3,-6.0323866E-5,-5.6981924E-3,-4.1920433E-4,5.579428E-3,4.774754E-3,-1.6400062E-3,8.504729E-3,1.454819E-2,-9.786241E-4,3.0436092E-3],"split_indices":[2,43,10,3,3,5,43,3,62,2,0,0,0,52,2,9,43,0,3,9,32,52,0,52,59,0,0,8,70,0,0,60,60,1,9,0,0,9,71,47,0,70,43,10,43,10,1,7,0,62,52,44,8,68,1,0,0,9,71,44,33,73,0,0,0,0,10,1,0,8,0,62,5,0,26,7,8,62,2,0,2,2,0,44,44,67,2,73,4,0,44,67,53,0,0,1,0,0,0,73,0,0,0,43,0,62,0,44,9,4,73,0,0,0,44,0,0,7,0,0,0,44,0,53,46,0,0,71,62,0,0,0,72,44,0,0,0,69,21,71,0,69,9,2,0,71,0,0,73,9,48,8,73,0,48,0,0,0,0,0,0,7,71,48,53,0,69,62,0,0,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,6.75E2,3.41E2,2.18E2,4.57E2,1.6E1,3.25E2,1.93E2,2.5E1,4.53E2,4E0,4E0,1.2E1,2.3E1,3.02E2,8E0,1.85E2,5E0,2E1,1.75E2,2.78E2,1.8E1,5E0,1.65E2,1.37E2,4E0,4E0,1.3E2,5.5E1,1.6E1,4E0,1E2,7.5E1,1.81E2,9.7E1,1.4E1,4E0,1.52E2,1.3E1,1.3E2,7E0,1.13E2,1.7E1,1.9E1,3.6E1,1.9E1,8.1E1,7.1E1,4E0,1.14E2,6.7E1,6.6E1,3.1E1,2.5E1,1.27E2,6E0,7E0,8.4E1,4.6E1,5.4E1,5.9E1,1.3E1,4E0,8E0,1.1E1,8E0,2.8E1,1.3E1,6E0,6.4E1,1.7E1,2.1E1,5E1,5E0,1.09E2,5.9E1,8E0,5.7E1,9E0,1.5E1,1.6E1,2.1E1,4E0,1.8E1,1.09E2,1.4E1,7E1,2.3E1,2.3E1,1.1E1,4.3E1,1.3E1,4.6E1,8E0,5E0,2.3E1,5E0,6E0,7E0,5.6E1,8E0,1E1,7E0,1.4E1,7E0,4E1,1E1,4.5E1,6.4E1,4.6E1,1.3E1,4E0,4E0,8E0,4.9E1,5E0,4E0,1.1E1,5E0,4E0,1.7E1,1.2E1,6E0,6.7E1,4.2E1,4E0,1E1,2.9E1,4.1E1,1.8E1,5E0,4E0,1.9E1,3.7E1,6E0,6E0,7E0,1.7E1,2.9E1,1.2E1,1.1E1,2.6E1,3E1,1E1,4E0,2.9E1,1.1E1,1.1E1,3.4E1,2.8E1,3.6E1,1.3E1,3.3E1,5E0,8E0,4.4E1,5E0,5E0,6E0,5E0,7E0,5.2E1,1.5E1,2.7E1,1.5E1,9E0,2E1,2.8E1,1.3E1,1E1,8E0,8E0,1.1E1,2E1,1.7E1,7E0,1E1,4E0,2.5E1,5E0,7E0,1.9E1,7E0,1.1E1,1.9E1,6E0,4E0,2.1E1,8E0,2.2E1,1.2E1,1E1,1.8E1,3.2E1,4E0,7E0,6E0,5E0,2.8E1,4E0,4E0,3.4E1,1E1,4.3E1,9E0,5E0,1E1,2.2E1,5E0,8E0,7E0,1.1E1,9E0,1.7E1,1.1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-4.246676E-3,-6.808504E-2,5.8847517E-2,-1.24956235E-1,-3.197511E-2,7.25163E-2,-1.878514E-1,-1.0967272E-1,-2.89086E-1,-7.940078E-2,-1.0550625E-2,3.926635E-2,1.5051953E-1,-2.3990951E-1,-2.2365642E-4,-1.8054748E-1,-8.5415095E-2,-1.2362893E-1,-2.0285517E-2,-6.1006423E-2,-1.2663756E-2,-1.9150887E-2,1.4990428E-1,4.6509113E-2,-1.3270856E-1,1.8738891E-1,8.367987E-2,-1.4224235E-2,-8.497253E-3,-1.12483455E-2,-1.5302731E-1,3.4755883E-3,-9.085308E-2,-1.9021727E-3,-8.378598E-3,3.4661505E-2,-8.086697E-2,-7.2961594E-3,-1.2680723E-2,3.8711128E-3,9.816399E-3,3.8670145E-2,1.7892972E-1,-1.2233176E-3,-1.8878269E-1,1.6144902E-1,1.6803693E-2,1.1860967E-2,6.0304508E-2,-2.1567966E-3,-7.842399E-3,1.1266137E-2,-9.841681E-2,-2.2521977E-4,2.7197797E-3,-6.1837643E-2,-1.74032E-1,-1.715104E-2,7.461865E-3,1.2748428E-2,3.466672E-2,9.915375E-3,1.9959281E-3,-1.8064856E-3,-1.389236E-2,1.9271572E-1,1.03793584E-1,2.5398724E-2,1.1706977E-1,3.3249524E-3,-1.9033191E-3,-1.2271502E-1,-4.8634205E-2,-8.125813E-2,-4.508185E-3,-1.168239E-2,-6.0444773E-4,4.4353185E-3,-4.218284E-2,5.3904797E-3,6.13476E-2,1.6982117E-1,1.570477E-2,-3.8522992E-3,1.3461204E-1,-6.3506454E-2,5.9083413E-2,1.8047484E-3,1.6315807E-1,-9.565868E-2,-1.641413E-1,-9.196156E-2,-5.0656255E-3,-9.1702044E-2,1.1462504E-3,3.364774E-3,-4.2802643E-2,1.6971877E-2,-7.107972E-2,-6.7158826E-2,4.4043483E-3,-1.4942856E-1,1.9011347E-2,7.560417E-2,-8.150623E-2,2.0148331E-1,8.079732E-2,1.5145561E-1,1.3956367E-3,3.846139E-4,-5.665702E-3,1.752344E-2,1.02828525E-1,3.7424928E-3,9.171824E-3,-6.809631E-2,-1.6560681E-1,-2.2617683E-1,-5.223122E-3,-6.6056475E-2,-6.7318887E-3,1.8655114E-2,-4.1323737E-3,-6.2383365E-2,-1.3287278E-1,-1.1996457E-4,-3.8149245E-3,2.5052862E-2,-5.794073E-3,-4.846777E-3,-8.051778E-4,-9.425922E-2,-1.6991132E-3,-2.2909991E-2,5.1837415E-2,-1.8868674E-2,1.4377348E-3,-4.9310926E-2,3.409211E-2,1.0709327E-1,3.241865E-2,2.4693515E-3,-1.7741615E-1,7.040766E-3,1.1711209E-2,-1.824229E-4,5.9281304E-3,2.946607E-3,7.694757E-3,-9.253373E-4,3.7704797E-3,1.7401703E-3,6.9692265E-3,-4.020863E-3,2.170386E-4,-8.869812E-3,-3.6828346E-3,-5.5350126E-3,-1.2828617E-2,-4.4012032E-3,-6.8798754E-4,-7.7901117E-4,1.9768511E-3,-4.1316934E-3,-3.4449282E-4,-9.569709E-3,-4.346737E-3,-2.4460553E-4,2.775861E-3,-1.1375634E-3,-5.2239527E-3,-2.4104996E-3,1.2883787E-3,-2.7757033E-3,1.5397029E-3,5.1759507E-5,4.0610735E-3,-9.3171693E-4,-5.6071896E-3,1.0261538E-3,7.070875E-3,5.9179016E-3,4.5398576E-4,-2.8310283E-3,2.1788187E-3,-2.7921349E-3,-1.1961488E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,41,43,45,47,-1,-1,-1,49,-1,51,-1,-1,53,55,-1,57,-1,-1,59,61,-1,63,65,-1,-1,67,-1,-1,69,71,-1,-1,73,75,77,-1,-1,79,-1,-1,-1,-1,81,83,85,87,-1,-1,89,91,93,95,-1,-1,97,99,101,103,105,-1,-1,107,109,111,-1,113,115,117,119,121,123,-1,-1,125,127,129,131,133,135,137,139,141,143,145,147,-1,-1,-1,149,151,-1,-1,153,155,157,-1,159,-1,161,-1,163,165,-1,-1,167,-1,-1,-1,169,171,173,175,-1,-1,177,179,181,183,-1,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0761423E0,1.0289729E0,1.730008E0,4.5744872E-1,3.1312805E-1,1.2468004E0,2.5607204E-1,2.9404402E-1,3.4282207E-1,3.2711142E-1,2.9818097E-1,4.29667E-1,3.391838E-1,2.3290992E-2,0E0,4.0672302E-2,1.2564307E-1,3.537406E-2,0E0,1.719045E-1,0E0,1.8117213E-1,2.7703434E-2,3.3252007E-1,7.839517E-2,3.6209536E-1,1.9891614E-1,0E0,0E0,0E0,5.2546978E-2,0E0,1.0306585E-1,0E0,0E0,1.5601469E-2,1.2043044E-1,0E0,1.5539777E-1,0E0,0E0,2.8490737E-1,7.79292E-2,0E0,1.3901716E-1,1.2781835E-1,0E0,0E0,9.213552E-2,0E0,0E0,3.481302E-2,1.4281607E-1,0E0,0E0,6.9775745E-2,1.4895931E-1,1.0393341E-1,0E0,0E0,2.3869076E-1,0E0,0E0,0E0,0E0,1.1754084E-1,1.9004968E-1,9.658921E-2,5.8338553E-2,0E0,0E0,7.620764E-2,7.881805E-2,5.434078E-2,5.3466383E-2,0E0,0E0,9.9663034E-2,1.0385272E-1,3.1282997E-1,3.320192E-1,1.1619902E-1,0E0,0E0,4.2904586E-2,3.9247792E-2,4.0013142E-2,0E0,1.5880048E-2,8.897549E-2,7.2669566E-2,1.9616753E-2,4.570368E-2,4.3615013E-2,0E0,0E0,1.8974999E-2,1.0669867E-1,2.5397733E-2,1.027309E-1,4.2646885E-2,5.738407E-1,1.4180899E-1,1.9506961E-1,2.0363793E-1,4.816413E-2,5.6886256E-2,1.9828439E-2,0E0,0E0,0E0,3.2922707E-2,2.8740212E-2,0E0,0E0,5.417095E-2,1.6421348E-2,3.7951946E-2,0E0,2.2493325E-2,0E0,1.6151093E-2,0E0,3.9447434E-2,2.9479772E-2,0E0,0E0,9.174736E-2,0E0,0E0,0E0,5.0936073E-2,2.8337257E-2,4.5901388E-2,2.1851707E-2,0E0,0E0,5.0053746E-2,1.5489137E-1,1.703397E-1,8.9916594E-2,0E0,7.0151925E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26,30,30,32,32,35,35,36,36,38,38,41,41,42,42,44,44,45,45,48,48,51,51,52,52,55,55,56,56,57,57,60,60,65,65,66,66,67,67,68,68,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81,84,84,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,111,111,112,112,115,115,116,116,117,117,119,119,121,121,123,123,124,124,127,127,131,131,132,132,133,133,134,134,137,137,138,138,139,139,140,140,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,42,44,46,48,-1,-1,-1,50,-1,52,-1,-1,54,56,-1,58,-1,-1,60,62,-1,64,66,-1,-1,68,-1,-1,70,72,-1,-1,74,76,78,-1,-1,80,-1,-1,-1,-1,82,84,86,88,-1,-1,90,92,94,96,-1,-1,98,100,102,104,106,-1,-1,108,110,112,-1,114,116,118,120,122,124,-1,-1,126,128,130,132,134,136,138,140,142,144,146,148,-1,-1,-1,150,152,-1,-1,154,156,158,-1,160,-1,162,-1,164,166,-1,-1,168,-1,-1,-1,170,172,174,176,-1,-1,178,180,182,184,-1,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.071E3,1.972052E5,7.042942E7,4.6463413E0,1E0,1.087521E3,2.3308511E5,5.4E1,1.169375E2,3.7568388E0,2.0200005E8,2.1894855E1,6.6933334E1,1.1015E4,-2.2365642E-4,1.7857143E1,7E0,7.6811076E-3,-2.0285517E-2,6.728972E-1,-1.2663756E-2,2.35E2,7.4654E4,7.82261E6,3.4E1,2.1511E4,3.401282E1,-1.4224235E-2,-8.497253E-3,-1.12483455E-2,1.9363636E1,3.4755883E-3,5.7018555E4,-1.9021727E-3,-8.378598E-3,4.48E2,2.87E2,-7.2961594E-3,2.8034655E6,3.8711128E-3,9.816399E-3,2.3E1,1.3E1,-1.2233176E-3,2.4E1,2.3385885E0,1.6803693E-2,1.1860967E-2,7.802E-2,-2.1567966E-3,-7.842399E-3,1.16E2,1.339646E6,-2.2521977E-4,2.7197797E-3,5E0,1.409E3,1E0,7.461865E-3,1.2748428E-2,4.371613E2,9.915375E-3,1.9959281E-3,-1.8064856E-3,-1.389236E-2,3.478E4,1.5484655E6,3.3272727E0,1.14061164E5,3.3249524E-3,-1.9033191E-3,2.23607E5,1.901875E2,4.0119403E2,6.346204E-7,-1.168239E-2,-6.0444773E-4,1E0,3.011152E6,4E0,2.8827406E5,9.557399E6,1.570477E-2,-3.8522992E-3,2.5E1,1.6677892E2,7.073537E5,1.8047484E-3,2.101E3,1.2E1,2.135961E4,1.7559428E2,1.8E1,2.6686933E-2,1.1462504E-3,3.364774E-3,4.612245E0,1.2E1,2.988E4,4.831007E6,8.69E2,1.24E2,2.16577E5,2E0,2.67637E5,1.6E1,1.7026364E3,2.23462E5,1.3956367E-3,3.846139E-4,-5.665702E-3,8.25E2,3.3580637E8,3.7424928E-3,9.171824E-3,4.7181977E4,3.2216358E-1,4.6991256E7,-5.223122E-3,9.72E2,-6.7318887E-3,5.249058E5,-4.1323737E-3,4.2609915E6,1.48404255E1,-1.1996457E-4,-3.8149245E-3,5.4E1,-5.794073E-3,-4.846777E-3,-8.051778E-4,1.5132743E0,1.4E1,2.1818182E0,1.89965E4,-1.8868674E-2,1.4377348E-3,8.998703E-1,2.5176924E1,2E0,7.8461537E0,2.4693515E-3,1.2E2,7.040766E-3,1.1711209E-2,-1.824229E-4,5.9281304E-3,2.946607E-3,7.694757E-3,-9.253373E-4,3.7704797E-3,1.7401703E-3,6.9692265E-3,-4.020863E-3,2.170386E-4,-8.869812E-3,-3.6828346E-3,-5.5350126E-3,-1.2828617E-2,-4.4012032E-3,-6.8798754E-4,-7.7901117E-4,1.9768511E-3,-4.1316934E-3,-3.4449282E-4,-9.569709E-3,-4.346737E-3,-2.4460553E-4,2.775861E-3,-1.1375634E-3,-5.2239527E-3,-2.4104996E-3,1.2883787E-3,-2.7757033E-3,1.5397029E-3,5.1759507E-5,4.0610735E-3,-9.3171693E-4,-5.6071896E-3,1.0261538E-3,7.070875E-3,5.9179016E-3,4.5398576E-4,-2.8310283E-3,2.1788187E-3,-2.7921349E-3,-1.1961488E-2],"split_indices":[2,43,60,68,29,67,48,2,67,68,5,69,71,9,0,4,3,53,0,69,0,2,1,43,3,2,73,0,0,0,4,0,60,0,0,2,0,0,62,0,0,0,3,0,0,53,0,0,72,0,0,44,9,0,0,8,10,26,0,0,67,0,0,0,0,44,43,69,48,0,0,9,4,4,52,0,0,19,9,3,48,60,0,0,3,73,43,0,0,8,43,4,3,57,0,0,69,71,1,60,2,0,43,32,9,3,4,1,0,0,0,8,7,0,0,48,53,7,0,44,0,60,0,60,61,0,0,0,0,0,0,68,3,71,48,0,0,57,71,6,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,5.02E2,5.08E2,1.94E2,3.08E2,4.82E2,2.6E1,1.79E2,1.5E1,9.5E1,2.13E2,3.39E2,1.43E2,2E1,6E0,4.4E1,1.35E2,8E0,7E0,8.8E1,7E0,2.03E2,1E1,3.26E2,1.3E1,9.1E1,5.2E1,7E0,1.3E1,1.1E1,3.3E1,4E0,1.31E2,4E0,4E0,1.5E1,7.3E1,8E0,1.95E2,6E0,4E0,3.09E2,1.7E1,5E0,8E0,8.1E1,1E1,5E0,4.7E1,5E0,2.8E1,9E0,1.22E2,6E0,9E0,6.2E1,1.1E1,1.91E2,4E0,4E0,3.05E2,1.3E1,4E0,4E0,4E0,5.1E1,3E1,3E1,1.7E1,4E0,5E0,8.1E1,4.1E1,4.6E1,1.6E1,7E0,4E0,1.03E2,8.8E1,1.46E2,1.59E2,4.6E1,5E0,4E0,2.6E1,8E0,2.2E1,7E0,1E1,5.1E1,3E1,2E1,2.1E1,4.2E1,4E0,5E0,1.1E1,8.9E1,1.4E1,5.7E1,3.1E1,1.1E1,1.35E2,1.45E2,1.4E1,3.3E1,1.3E1,2.2E1,4E0,4E0,4E0,1.2E1,1E1,4E0,6E0,3.8E1,1.3E1,1.2E1,1.8E1,1.5E1,5E0,1.7E1,4E0,2.6E1,1.6E1,6E0,5E0,8.5E1,4E0,8E0,6E0,4E1,1.7E1,2E1,1.1E1,4E0,7E0,2.4E1,1.11E2,8.3E1,6.2E1,6E0,8E0,1.9E1,1.4E1,5E0,8E0,4E0,1.8E1,8E0,4E0,5E0,5E0,3E1,8E0,9E0,4E0,5E0,7E0,9E0,6E0,7E0,1E1,1.7E1,9E0,4E0,1.2E1,4.6E1,3.9E1,9E0,3.1E1,6E0,1.1E1,1.2E1,8E0,5E0,6E0,1.8E1,6E0,1.02E2,9E0,6.8E1,1.5E1,8E0,5.4E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[3.3297176E-3,-9.517032E-2,4.093917E-2,-7.589505E-2,-2.047397E-1,5.6218677E-3,1.35447E-1,-1.03723176E-1,3.9400604E-2,-1.5781309E-1,-2.3609966E-2,-3.5599433E-2,3.4445647E-2,2.018905E-2,1.2374736E-1,5.7373635E-2,-1.1288989E-1,1.1896653E-2,-1.1942283E-2,-1.7154962E-1,-1.6441952E-3,-2.5915174E-2,-2.3540896E-1,2.0166505E-2,1.0572814E-1,1.0986993E-1,1.3505884E-2,9.171345E-3,-2.3857206E-3,-1.0469012E-1,-1.2850202E-2,-4.5681067E-2,5.904921E-2,-1.9346522E-1,-9.161186E-2,-2.037825E-2,-9.737564E-3,-1.92913E-2,-2.050034E-3,2.4700554E-2,-8.195852E-3,7.24137E-2,9.736071E-3,1.1811017E-1,-5.0577535E-3,-1.2530367E-1,-5.4875832E-2,-4.7120946E-3,-5.9018005E-4,3.9337403E-3,1.2113429E-5,-3.963647E-3,-9.696643E-3,-1.2340919E-3,-6.384194E-3,1.3450192E-2,-3.5481364E-2,6.465759E-2,-4.6955356E-3,3.9165895E-2,1.2743217E-1,-3.98957E-3,1.247095E-1,-1.06488764E-1,-2.0684081E-1,-2.6475715E-2,-9.369074E-2,-2.8757644E-3,2.5149453E-2,-4.293659E-3,2.8145934E-2,-5.223333E-2,7.059656E-3,7.6398745E-2,-2.0471564E-2,-1.5931932E-2,1.3771853E-1,-1.8382893E-3,5.462749E-2,2.4348241E-3,8.603761E-3,1.4945285E-1,6.6638686E-2,-1.3527343E-1,-2.5583832E-2,-2.692773E-1,-4.0070787E-3,-4.8865356E-2,2.766803E-2,-8.288341E-3,-5.7927426E-2,3.6776708E-3,-5.5572303E-4,4.2710274E-2,-3.0675915E-3,-4.1916486E-2,-7.771451E-3,4.3633394E-3,-4.9401806E-3,6.3400075E-2,1.8184727E-1,-5.6452624E-2,2.567968E-3,-3.488993E-2,6.400086E-2,3.202812E-3,9.513486E-3,7.452817E-2,-6.550628E-4,1.7600492E-1,8.962082E-2,3.4909096E-2,1.9641206E-1,-1.5257972E-1,-7.524455E-3,6.721595E-2,-7.30793E-2,-1.6596582E-2,-9.118207E-3,-3.8823816E-3,-9.356743E-4,3.4212247E-3,-1.5489962E-3,-1.2179071E-3,-5.1467298E-3,2.8590102E-2,5.791706E-3,-1.2005941E-2,-8.84788E-2,-3.2254998E-3,5.03941E-3,4.7472272E-2,1.311127E-1,1.0810228E-2,3.8300692E-3,-5.5680884E-4,-4.5479448E-3,2.7110377E-2,-5.971857E-2,1.0920867E-1,6.9148587E-3,1.4481733E-3,5.27718E-3,1.3534814E-1,1.201341E-2,2.2169113E-2,1.1989972E-1,5.615528E-3,8.921606E-2,1.2231885E-2,3.469839E-3,-3.802598E-3,-7.929313E-3,2.7879747E-3,-2.9026833E-3,4.999125E-3,2.1167596E-4,-1.7738995E-3,-7.50784E-3,8.6023175E-4,3.6325655E-3,-2.8788622E-3,3.031062E-4,-8.70642E-3,-1.7986735E-3,2.8219619E-3,-2.2161666E-4,7.038843E-4,3.240193E-3,9.038444E-3,3.8805315E-3,-2.0097329E-3,4.4410056E-3,1.2077744E-3,-3.9940276E-3,1.1817028E-3,6.3428674E-3,-1.050409E-3,2.0927086E-3,5.188983E-3,1.0133847E-2,-1.5733469E-3,4.3168785E-3,6.6986037E-3,1.4355014E-3,-2.237358E-3,1.4948554E-3,9.953913E-4,6.7327907E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,45,-1,47,49,51,53,55,-1,-1,-1,57,-1,59,-1,61,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,69,71,73,75,77,79,-1,81,83,85,87,89,-1,91,-1,93,95,97,99,101,103,105,-1,107,-1,-1,109,111,113,115,117,-1,119,121,-1,123,-1,-1,125,-1,127,-1,-1,129,131,133,135,-1,137,139,-1,-1,141,-1,143,145,147,149,151,153,155,157,-1,-1,-1,-1,-1,-1,-1,-1,159,-1,161,163,-1,165,167,169,-1,-1,-1,-1,171,173,175,177,-1,-1,179,-1,181,183,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7269514E0,5.7159185E-1,2.4291286E0,7.677009E-1,5.086857E-1,6.3213587E-1,6.444812E-1,2.9000854E-1,5.293951E-1,5.6015015E-2,0E0,4.1766602E-1,3.1645578E-1,0E0,4.1709304E-1,1.7950486E-1,2.1750617E-1,0E0,9.610893E-2,4.0088475E-2,0E0,2.1065018E-1,3.2946062E-1,2.39079E-1,1.6665989E-1,3.3646822E-1,0E0,0E0,0E0,1.7282248E-1,0E0,6.8500035E-2,1.9922905E-2,2.6199758E-2,2.351398E-2,1.05214775E-1,0E0,0E0,0E0,3.0217084E-1,0E0,7.000798E-2,0E0,2.4420547E-1,0E0,1.6666985E-1,5.594392E-2,0E0,2.6957976E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0267708E-1,1.0156444E-1,1.10511065E-1,2.4062705E-1,3.51351E-2,5.609648E-2,0E0,2.3143435E-1,2.3407888E-1,1.4162177E-1,4.0144295E-2,5.704689E-2,0E0,2.5252316E-2,0E0,8.132444E-2,1.1500034E-1,4.346033E-2,1.20057166E-1,4.1450188E-2,2.128688E-1,3.7035853E-2,0E0,3.2550618E-2,0E0,0E0,1.7009997E-1,2.0726153E-1,1.6466653E-1,1.2832405E-1,1.9666433E-2,0E0,2.1443143E-2,3.086966E-2,0E0,2.5758862E-2,0E0,0E0,5.47702E-2,0E0,1.3124464E-1,0E0,0E0,2.4307435E-2,8.760643E-2,3.219551E-2,1.7457087E-2,0E0,1.7556931E-1,6.8924636E-2,0E0,0E0,2.727621E-2,0E0,2.3811865E-1,7.495177E-2,6.790532E-2,6.303039E-2,7.424271E-2,4.1328244E-2,2.5179472E-2,5.269234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0486768E-2,0E0,5.5518754E-2,1.7593735E-1,0E0,1.848474E-2,5.1533163E-2,3.2213032E-2,0E0,0E0,0E0,0E0,1.650054E-1,1.8818924E-1,3.172031E-2,1.5825287E-2,0E0,0E0,8.370173E-2,0E0,5.5276446E-2,5.345133E-2,4.347816E-2,5.4229297E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,19,19,21,21,22,22,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,39,39,41,41,43,43,45,45,46,46,48,48,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,68,68,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,81,81,82,82,83,83,84,84,85,85,87,87,88,88,90,90,93,93,95,95,98,98,99,99,100,100,101,101,103,103,104,104,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,125,125,127,127,128,128,130,130,131,131,132,132,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,148,148],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,46,-1,48,50,52,54,56,-1,-1,-1,58,-1,60,-1,62,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,70,72,74,76,78,80,-1,82,84,86,88,90,-1,92,-1,94,96,98,100,102,104,106,-1,108,-1,-1,110,112,114,116,118,-1,120,122,-1,124,-1,-1,126,-1,128,-1,-1,130,132,134,136,-1,138,140,-1,-1,142,-1,144,146,148,150,152,154,156,158,-1,-1,-1,-1,-1,-1,-1,-1,160,-1,162,164,-1,166,168,170,-1,-1,-1,-1,172,174,176,178,-1,-1,180,-1,182,184,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0976269E5,1.0657745E7,3.812E3,2.9994638E2,3.7798166E0,8.1E2,2.4481E4,7E0,1.176582E6,1.4916515E3,-2.3609966E-2,3.6E1,2.8530578E6,2.018905E-2,8.2471845E6,1.3050649E4,3.0977E4,1.1896653E-2,6.3351805E4,8.579633E-2,-1.6441952E-3,1E0,2.6676828E2,1.4676277E7,2.0833333E0,7.134432E7,1.3505884E-2,9.171345E-3,-2.3857206E-3,1.339646E6,-1.2850202E-2,8.147158E4,3.1095755E0,1.5933E4,2.4203823E0,2.6654E4,-9.737564E-3,-1.92913E-2,-2.050034E-3,1.7903225E0,-8.195852E-3,6E0,9.736071E-3,1.55E2,-5.0577535E-3,1.2E1,2.41E2,-4.7120946E-3,1.6E1,3.9337403E-3,1.2113429E-5,-3.963647E-3,-9.696643E-3,-1.2340919E-3,-6.384194E-3,1.62E2,6.887749E7,6.1114804E-8,8.7115955E2,2.02E4,2.742234E6,-3.98957E-3,8.7E1,2.55E2,7.709291E7,1.4276666E2,3.72E2,-2.8757644E-3,2.670362E7,-4.293659E-3,1.9E1,6.763314E7,2.4915715E-3,1.47176E5,6.6414773E-1,9.751103E6,1.7E1,-1.8382893E-3,2.682E3,2.4348241E-3,8.603761E-3,1.7103828E7,3.8537518E2,2.4E1,2.855464E5,1.746E3,-4.0070787E-3,1.33162E6,1.8E1,-8.288341E-3,6.642857E0,3.6776708E-3,-5.5572303E-4,5.6530495E6,-3.0675915E-3,1E0,-7.771451E-3,4.3633394E-3,3.000176E5,6.083812E2,1.72797E5,2.0211798E6,2.567968E-3,1.5123151E6,2.5417458E1,3.202812E-3,9.513486E-3,6.009E4,-6.550628E-4,1.4916515E3,5.97E3,1.4176E4,6.880842E-1,2.416E3,1E0,1.5E1,1.441215E5,-1.6596582E-2,-9.118207E-3,-3.8823816E-3,-9.356743E-4,3.4212247E-3,-1.5489962E-3,-1.2179071E-3,-5.1467298E-3,3.0751158E4,5.791706E-3,7.117E3,2.019537E2,-3.2254998E-3,3.617263E6,1.411E3,1.9197379E9,1.0810228E-2,3.8300692E-3,-5.5680884E-4,-4.5479448E-3,4.0911578E2,1E0,6.1786E4,5.460753E9,1.4481733E-3,5.27718E-3,4.9833015E1,1.201341E-2,3.1991906E0,9.0754684E1,2.453E4,1.9428571E0,1.2231885E-2,3.469839E-3,-3.802598E-3,-7.929313E-3,2.7879747E-3,-2.9026833E-3,4.999125E-3,2.1167596E-4,-1.7738995E-3,-7.50784E-3,8.6023175E-4,3.6325655E-3,-2.8788622E-3,3.031062E-4,-8.70642E-3,-1.7986735E-3,2.8219619E-3,-2.2161666E-4,7.038843E-4,3.240193E-3,9.038444E-3,3.8805315E-3,-2.0097329E-3,4.4410056E-3,1.2077744E-3,-3.9940276E-3,1.1817028E-3,6.3428674E-3,-1.050409E-3,2.0927086E-3,5.188983E-3,1.0133847E-2,-1.5733469E-3,4.3168785E-3,6.6986037E-3,1.4355014E-3,-2.237358E-3,1.4948554E-3,9.953913E-4,6.7327907E-3],"split_indices":[43,60,2,67,68,2,12,3,5,4,0,3,43,0,43,43,44,0,48,53,0,106,4,62,68,59,0,0,0,9,0,43,53,1,69,9,0,0,0,68,0,8,0,10,0,8,10,0,8,0,0,0,0,0,0,2,7,52,67,12,9,0,8,2,7,67,2,0,5,0,8,7,53,1,53,9,3,0,10,0,0,62,71,3,60,44,0,5,3,0,69,0,0,43,0,26,0,0,43,4,1,43,0,60,73,0,0,1,0,4,2,2,57,9,20,3,43,0,0,0,0,0,0,0,0,48,0,1,4,0,5,44,46,0,0,0,0,4,8,1,46,0,0,73,0,53,71,44,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.004E3,2.77E2,7.27E2,2.37E2,4E1,5.3E2,1.97E2,1.91E2,4.6E1,3.6E1,4E0,2.18E2,3.12E2,6E0,1.91E2,1E1,1.81E2,8E0,3.8E1,3.2E1,4E0,2.09E2,9E0,2.61E2,5.1E1,1.78E2,1.3E1,4E0,6E0,1.74E2,7E0,2.6E1,1.2E1,2.4E1,8E0,2.04E2,5E0,4E0,5E0,2.56E2,5E0,4E1,1.1E1,1.72E2,6E0,1.22E2,5.2E1,1.1E1,1.5E1,8E0,4E0,4E0,2E1,4E0,4E0,6.3E1,1.41E2,1.08E2,1.48E2,2.6E1,1.4E1,5E0,1.67E2,1.01E2,2.1E1,3.1E1,2.1E1,4E0,1.1E1,7E0,5.6E1,1.01E2,4E1,9.5E1,1.3E1,1.38E2,1E1,4E0,2.2E1,7E0,7E0,1.16E2,5.1E1,7.4E1,2.7E1,1.3E1,8E0,2.2E1,9E0,5E0,1.6E1,4E0,7E0,4.9E1,7E0,9.4E1,7E0,4E0,3.6E1,8.6E1,9E0,9E0,4E0,1.12E2,2.6E1,6E0,4E0,1.7E1,5E0,7.9E1,3.7E1,4.2E1,9E0,6.5E1,9E0,9E0,1.8E1,4E0,9E0,9E0,1.3E1,5E0,4E0,1.1E1,5E0,4.3E1,6E0,5.8E1,3.6E1,4E0,3.2E1,7.1E1,1.5E1,5E0,4E0,5E0,4E0,3.2E1,8E1,1.4E1,1.2E1,9E0,8E0,5.5E1,2.4E1,1.2E1,2.5E1,2.8E1,1.4E1,5E0,4E0,1.5E1,5E1,4E0,5E0,5E0,4E0,1.4E1,4E0,3.7E1,6E0,1.5E1,4.3E1,1.1E1,2.5E1,4E0,2.8E1,3E1,4.1E1,5E0,1E1,1.6E1,1.6E1,1.9E1,6.1E1,4E0,1E1,7E0,5E0,4.5E1,1E1,7E0,5E0,1.9E1,6E0,9E0,1.9E1,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[-2.5421267E-3,-4.54704E-2,7.471239E-2,-1.09337635E-1,-1.33620575E-2,1.0278751E-1,-8.404298E-2,-1.0111751E-1,-1.7944142E-2,-8.394736E-3,-2.2760764E-1,1.9427637E-2,9.310652E-2,-1.9356988E-2,-5.022023E-2,-8.435361E-2,-2.1126117E-1,-1.332153E-2,1.1324026E-1,-1.1398182E-3,-1.966273E-2,2.6305959E-2,1.217016E-1,-7.619313E-2,3.8039568E-3,-1.5917991E-1,-5.500472E-2,-2.5255677E-1,-3.532036E-3,-5.942694E-2,3.542078E-3,1.3865676E-2,9.241316E-3,-6.367484E-3,1.1543242E-1,1.5685134E-1,5.840395E-2,-2.7696842E-2,-1.5158741E-1,-7.2341606E-2,-1.9477642E-1,7.969432E-2,-7.33234E-2,-1.9888097E-1,-1.6335959E-2,-4.9246356E-2,-1.2355577E-2,1.2362886E-2,-8.839086E-2,2.4679608E-3,-1.3149148E-3,1.1127053E-2,-5.1141405E-3,2.3229497E-2,8.101115E-3,1.3353586E-1,1.3239269E-2,7.806513E-2,-8.309853E-2,-6.772005E-2,4.7144773E-3,-9.600239E-3,-6.4190663E-4,7.998607E-4,-1.1114062E-1,-1.215757E-2,-1.6683866E-1,8.65128E-3,-6.619609E-4,-4.2575836E-2,-9.408454E-2,-4.4017024E-3,-1.0416772E-2,-2.1093674E-2,-9.473748E-2,1.7409757E-2,-5.095121E-2,2.4142342E-3,-1.1512344E-1,-5.0133485E-2,3.266131E-2,3.584761E-3,-2.6576682E-3,1.8481413E-1,8.088663E-2,4.8281103E-2,1.9162512E-1,-8.544575E-3,2.5753442E-3,1.9589511E-3,-1.02603264E-1,-1.3831886E-3,-6.837394E-3,-8.4553035E-3,-2.4286485E-3,-2.4226228E-2,-1.0709931E-1,-7.313269E-2,-1.5756802E-1,-6.6714636E-3,-5.3081783E-3,-3.9686467E-2,-1.4527717E-1,7.143194E-3,4.8257496E-2,-9.067044E-2,1.1515733E-3,-2.9145833E-4,-7.0757326E-3,5.572348E-4,-6.090037E-3,8.611992E-2,1.2832938E-2,2.6268777E-1,1.3486584E-1,-7.571906E-3,1.06119685E-1,-3.2952398E-2,6.665692E-2,5.6689526E-3,1.2153334E-2,2.3366521E-4,-1.6178727E-1,-4.07812E-2,2.2711768E-3,-7.0660263E-3,-9.92913E-4,-1.2123521E-1,-4.080731E-2,-8.279315E-3,-3.224055E-3,-5.5296108E-2,2.0928886E-2,9.3635026E-4,-6.501933E-2,-1.8784513E-1,-3.487928E-4,-4.1923635E-2,1.800284E-2,8.1211045E-2,7.104659E-3,-5.814261E-3,-1.7329064E-3,1.084086E-3,6.6925497E-3,2.4810946E-3,-1.8613518E-4,1.3753464E-2,5.530243E-3,-1.822666E-3,1.5653424E-1,1.287163E-1,3.0749848E-2,3.8311942E-4,-3.88558E-3,1.2206044E-1,1.0784502E-2,-3.452554E-3,-1.1610145E-2,2.3218493E-3,-2.5222346E-3,-6.790782E-3,-1.2962726E-3,-5.549409E-3,-1.2334522E-3,-5.8116326E-3,-1.2548242E-3,-2.4443984E-3,2.2274922E-3,-1.0832128E-3,-4.5489715E-3,-1.0776526E-2,-3.3850789E-3,2.2743465E-3,-3.4247932E-3,1.0696541E-3,-4.5897663E-3,2.7066993E-3,7.071643E-3,-1.6484723E-3,2.4148275E-3,9.705331E-3,3.842353E-3,4.059716E-3,8.01699E-3,4.703545E-3,-9.25853E-4,7.4518058E-3,3.941479E-3,-1.1742665E-3,4.7277724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,35,37,-1,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,-1,-1,79,-1,81,-1,83,-1,85,87,89,-1,-1,-1,-1,91,-1,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,107,109,111,-1,-1,113,115,117,119,-1,-1,-1,121,-1,-1,-1,-1,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,-1,-1,-1,143,145,147,149,-1,151,153,155,-1,-1,-1,157,159,-1,-1,-1,161,163,-1,-1,165,167,-1,169,171,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,183,185,-1,-1,187,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.425925E0,1.3601847E0,1.6532998E0,4.699943E-1,4.6954185E-1,9.2670035E-1,6.133747E-1,3.8285613E-1,0E0,2.61592E-1,3.9131492E-1,0E0,5.8130956E-1,0E0,1.853061E-1,4.0849745E-1,1.3656747E-1,3.25207E-1,1.4385058E-1,0E0,0E0,2.7155736E-1,4.6696687E-1,1.5668058E-1,0E0,1.4973998E-1,3.4635958E-1,4.357028E-2,0E0,2.2401384E-1,2.5004607E-1,1.6588666E-2,0E0,1.2717147E-1,1.3404658E-1,3.7608337E-1,2.2349128E-1,1.523081E-1,1.2869799E-1,6.1059132E-2,2.6779413E-2,1.7461634E-1,7.387358E-2,2.2602916E-2,0E0,1.363169E-1,0E0,9.067723E-2,1.0753298E-1,0E0,0E0,8.078943E-2,0E0,5.2756384E-2,0E0,3.0362272E-1,0E0,2.2412497E-1,1.5172553E-1,8.8862E-2,0E0,0E0,0E0,0E0,3.1483725E-2,0E0,4.3997407E-2,0E0,0E0,5.77884E-2,8.4243536E-2,0E0,0E0,9.122833E-2,1.0830721E-1,8.231406E-2,6.5549836E-2,0E0,9.422311E-2,8.287967E-2,4.6594705E-2,0E0,0E0,1.9406223E-1,3.8799167E-1,8.578949E-2,3.664857E-2,0E0,0E0,0E0,1.09139115E-1,0E0,0E0,0E0,0E0,5.1816482E-2,3.9887086E-2,8.212891E-2,1.9789249E-2,8.1680715E-2,0E0,3.321284E-2,1.2170005E-1,1.0572588E-1,8.757159E-2,2.146358E-2,0E0,0E0,0E0,0E0,0E0,4.0767103E-2,2.3648227E-2,7.4499846E-2,1.4970922E-1,0E0,8.971262E-2,2.3905348E-2,1.41078E-1,0E0,0E0,0E0,6.694397E-2,4.5501724E-2,0E0,0E0,0E0,4.8669428E-2,3.595761E-2,0E0,0E0,4.0220015E-2,8.1127815E-2,0E0,1.8547483E-2,6.6167295E-2,0E0,1.09338894E-1,1.00211866E-1,4.9555868E-2,5.9839338E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1608368E-1,6.1625123E-2,5.256903E-2,0E0,0E0,1.939997E-2,8.214453E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,51,51,53,53,55,55,57,57,58,58,59,59,64,64,66,66,69,69,70,70,73,73,74,74,75,75,76,76,78,78,79,79,80,80,83,83,84,84,85,85,86,86,90,90,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,111,111,112,112,113,113,114,114,116,116,117,117,118,118,122,122,123,123,127,127,128,128,131,131,132,132,134,134,135,135,137,137,138,138,139,139,140,140,150,150,151,151,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,36,38,-1,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,-1,-1,80,-1,82,-1,84,-1,86,88,90,-1,-1,-1,-1,92,-1,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,108,110,112,-1,-1,114,116,118,120,-1,-1,-1,122,-1,-1,-1,-1,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,-1,-1,-1,144,146,148,150,-1,152,154,156,-1,-1,-1,158,160,-1,-1,-1,162,164,-1,-1,166,168,-1,170,172,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,184,186,-1,-1,188,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.8988005E5,1E0,6.9934395E-5,6.0692043E0,3.3817584E7,7.87E3,1.977157E7,-1.7944142E-2,7.82261E6,1.3586957E1,1.9427637E-2,5.817547E2,-1.9356988E-2,3.2462872E5,4.9E2,3.1126543E1,2.1222334E2,1.9186046E0,-1.1398182E-3,-1.966273E-2,7.127857E6,5.489183E1,4.01969E5,3.8039568E-3,5.3E1,9.83871E-1,3.2E1,-3.532036E-3,3.4E1,1.465E4,5.51E2,9.241316E-3,1.8E1,1E0,1.4176E4,3.206931E2,4.27E3,1.3026532E6,1E0,1.7857143E1,1.32E2,1.0142858E1,7.0093E4,-1.6335959E-2,1E0,-1.2355577E-2,2E0,2.5818555E2,2.4679608E-3,-1.3149148E-3,5.9754E4,-5.1141405E-3,1.1544592E1,8.101115E-3,3.72381E5,1.3239269E-2,3.8537518E2,6.9879E4,1E0,4.7144773E-3,-9.600239E-3,-6.4190663E-4,7.998607E-4,2.3E1,-1.215757E-2,1.76421E6,8.65128E-3,-6.619609E-4,2.0303884E6,3.1316226E0,-4.4017024E-3,-1.0416772E-2,1E0,1.703125E0,4.5432812E2,9E0,2.4142342E-3,1.9E1,1.136E3,1.1283241E6,3.584761E-3,-2.6576682E-3,2.124E3,6.27907E0,3.808001E5,4.43837E0,-8.544575E-3,2.5753442E-3,1.9589511E-3,2.511E3,-1.3831886E-3,-6.837394E-3,-8.4553035E-3,-2.4286485E-3,1.6580646E2,9.14E2,3.95E2,2.897656E8,8.038E3,-5.3081783E-3,8.454545E0,9.5E2,3.990487E5,2E0,7.89E2,1.1515733E-3,-2.9145833E-4,-7.0757326E-3,5.572348E-4,-6.090037E-3,5.9653606E5,9.47E4,1.3587301E1,2.9355192E1,-7.571906E-3,2.3922667E1,7.25723E2,8.710612E1,5.6689526E-3,1.2153334E-2,2.3366521E-4,1.3711089E6,2.6412E5,2.2711768E-3,-7.0660263E-3,-9.92913E-4,2.8461537E0,7.7E1,-8.279315E-3,-3.224055E-3,1.3E1,1.3020051E-5,9.3635026E-4,2.9016996E4,1.8149019E2,-3.487928E-4,1.5132743E0,1E0,2.0277777E0,1.404E3,-5.814261E-3,-1.7329064E-3,1.084086E-3,6.6925497E-3,2.4810946E-3,-1.8613518E-4,1.3753464E-2,5.530243E-3,-1.822666E-3,6E0,1.2E1,2.83E3,3.8311942E-4,-3.88558E-3,1.03339244E6,3.002566E0,-3.452554E-3,-1.1610145E-2,2.3218493E-3,-2.5222346E-3,-6.790782E-3,-1.2962726E-3,-5.549409E-3,-1.2334522E-3,-5.8116326E-3,-1.2548242E-3,-2.4443984E-3,2.2274922E-3,-1.0832128E-3,-4.5489715E-3,-1.0776526E-2,-3.3850789E-3,2.2743465E-3,-3.4247932E-3,1.0696541E-3,-4.5897663E-3,2.7066993E-3,7.071643E-3,-1.6484723E-3,2.4148275E-3,9.705331E-3,3.842353E-3,4.059716E-3,8.01699E-3,4.703545E-3,-9.25853E-4,7.4518058E-3,3.941479E-3,-1.1742665E-3,4.7277724E-3],"split_indices":[2,43,6,52,68,7,9,5,0,43,73,0,67,0,48,1,73,4,69,0,0,60,71,1,0,44,68,3,0,3,44,2,0,3,23,2,73,2,43,24,4,12,73,1,0,26,0,6,4,0,0,1,0,71,0,9,0,71,2,8,0,0,0,0,44,0,46,0,0,60,68,0,0,19,68,4,8,0,8,44,60,0,0,44,71,43,50,0,0,0,2,0,0,0,0,67,2,2,46,1,0,73,10,43,32,2,0,0,0,0,0,60,9,71,73,0,71,67,73,0,0,0,43,7,0,0,0,68,10,0,0,3,53,0,48,70,0,68,79,68,2,0,0,0,0,0,0,0,0,0,8,8,2,0,0,43,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.031E3,6.63E2,3.68E2,2.21E2,4.42E2,3.13E2,5.5E1,2.16E2,5E0,4.33E2,9E0,8E0,3.05E2,4E0,5.1E1,1.89E2,2.7E1,4.17E2,1.6E1,5E0,4E0,9.2E1,2.13E2,4.3E1,8E0,5.2E1,1.37E2,2E1,7E0,1.11E2,3.06E2,8E0,8E0,6.8E1,2.4E1,1.36E2,7.7E1,2.7E1,1.6E1,1.6E1,3.6E1,1.6E1,1.21E2,1.5E1,5E0,1.07E2,4E0,2.8E2,2.6E1,4E0,4E0,5.9E1,9E0,1E1,1.4E1,1.17E2,1.9E1,6.8E1,9E0,2.1E1,6E0,1.1E1,5E0,5E0,1.1E1,8E0,2.8E1,7E0,9E0,5E1,7.1E1,4E0,1.1E1,6.7E1,4E1,2.6E2,2E1,4E0,2.2E1,1.5E1,4.4E1,6E0,4E0,5.8E1,5.9E1,5.5E1,1.3E1,5E0,4E0,5E0,1.6E1,4E0,7E0,2.4E1,4E0,4E1,1E1,5.5E1,1.6E1,5.9E1,8E0,2E1,2E1,1.96E2,6.4E1,1.3E1,7E0,6E0,1.6E1,9E0,6E0,1.1E1,3.3E1,2.1E1,3.7E1,5E0,5.4E1,1E1,4.5E1,8E0,5E0,6E0,1E1,3.3E1,7E0,6E0,4E0,2.1E1,3.4E1,1.2E1,4E0,2.1E1,3.8E1,6E0,1.4E1,1.5E1,5E0,3.5E1,1.61E2,3.5E1,2.9E1,7E0,6E0,6E0,5E0,9E0,2.4E1,1.6E1,5E0,4E0,3.3E1,4.1E1,1.3E1,6E0,4E0,2.2E1,2.3E1,6E0,4E0,4E0,2.9E1,1.6E1,5E0,4E0,3E1,5E0,1.6E1,1E1,2.8E1,7E0,7E0,1E1,5E0,9E0,2.6E1,1.55E2,6E0,2.8E1,7E0,1.5E1,1.4E1,1.8E1,1.5E1,2.3E1,1.8E1,5E0,8E0,9E0,1.3E1,1.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-1.6768986E-3,-3.4966692E-2,9.1516346E-2,-6.739187E-2,1.5456984E-2,1.0763499E-1,-1.07800454E-1,-5.2370828E-2,-1.547599E-1,1.9478336E-2,-1.0464207E-2,2.0494487E-2,9.771774E-2,-8.86112E-3,3.7827522E-3,-1.3803354E-1,-3.8602564E-2,-9.705497E-2,-3.9143035E-1,7.961811E-3,9.7763024E-2,7.964134E-2,2.091503E-1,3.285471E-3,-3.5930462E-3,-7.501039E-2,-2.0544358E-1,1.0062904E-1,-4.6550117E-2,-1.309202E-1,4.237979E-3,-2.515075E-2,-1.7462966E-3,4.1632973E-2,-1.2664159E-2,-2.2505E-3,1.1642825E-1,5.4190245E-2,1.388433E-1,2.5647342E-1,1.0101846E-1,-6.518899E-3,-4.8759773E-2,-3.2511433E-3,-1.0478092E-2,1.1660193E-2,4.3333333E-2,-3.5339423E-2,-1.3117544E-1,-9.181514E-2,-8.221557E-3,-2.4021226E-3,2.194635E-3,1.4204666E-2,7.7023216E-2,8.0237497E-4,-1.4468968E-1,1.4548971E-1,1.7520034E-4,1.1522659E-2,4.4459067E-2,2.0912269E-1,7.068141E-2,1.3075406E-2,4.8180684E-3,2.2034561E-3,6.7632147E-3,6.2657185E-5,-7.07703E-2,-2.8989706E-3,4.047658E-3,-3.1303614E-2,-1.2008373E-2,-1.7737614E-1,2.968251E-3,4.992811E-4,-1.1801108E-1,2.9725919E-2,-5.6677135E-3,8.975173E-3,4.8828438E-2,-2.679227E-2,3.845852E-2,-1.4495105E-2,2.8808778E-3,1.6173092E-1,1.9256801E-3,2.2927333E-2,1.0094499E-1,2.3838468E-1,4.397981E-3,-3.2774562E-3,1.03562936E-1,-6.0479003E-3,-1.4568117E-3,-6.277974E-2,-4.0931082E-3,-2.3537235E-1,-5.9300605E-3,-6.481607E-3,-2.6139272E-3,5.352191E-2,-2.241417E-2,2.4598083E-2,5.0526774E-3,-1.4538056E-1,-1.2765515E-2,5.607442E-3,1.2290839E-2,-2.8369317E-3,3.0765005E-3,3.3858994E-3,8.153802E-3,-1.362983E-1,3.8940895E-2,1.2367919E-1,-1.104112E-3,6.4709918E-3,2.706289E-1,2.3289045E-2,7.1208873E-3,-1.2806566E-2,-8.651352E-2,-2.080394E-2,3.853658E-2,-4.4824574E-3,-1.3216604E-2,2.5242904E-2,1.0797634E-1,3.2110284E-3,-6.6505514E-2,-1.4431196E-3,4.570221E-2,-1.8732371E-3,-1.0216361E-2,1.2648586E-2,-4.126894E-2,-2.6493251E-2,6.1093733E-2,3.4651808E-3,-1.3382404E-2,8.724209E-3,7.902209E-2,1.6011976E-1,6.684649E-2,7.4967253E-3,1.4551862E-2,-1.36519E-3,3.0146171E-3,1.2837243E-3,-2.0682854E-3,-5.330412E-3,-2.7189185E-3,8.006539E-4,-1.9510069E-3,3.610932E-3,-8.1835163E-4,4.666905E-4,3.303524E-3,2.0861109E-3,7.5430125E-3,-5.617559E-3,4.0883533E-4,4.1358406E-3,7.0317305E-4,2.858946E-3,-1.2262574E-4,-2.4527307E-3,1.3239514E-3,7.4568996E-4,-5.464974E-3,4.5384523E-3,-2.5098824E-3,2.7692802E-3,-1.8188662E-3,8.140432E-3,1.7950365E-3,8.773918E-3,2.7231404E-3,6.678367E-3,-4.7211503E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,61,63,65,-1,67,-1,-1,-1,69,71,73,75,-1,-1,-1,77,79,81,83,85,-1,-1,87,89,91,-1,-1,-1,-1,-1,93,-1,-1,95,-1,97,-1,-1,99,101,-1,-1,103,105,107,-1,109,111,-1,113,115,117,-1,-1,119,-1,-1,121,123,125,-1,-1,-1,127,129,131,-1,133,135,-1,137,-1,-1,-1,-1,139,141,143,-1,-1,145,147,-1,149,151,153,155,-1,-1,157,159,-1,161,-1,163,-1,-1,165,167,169,171,-1,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.229679E0,1.2550843E0,8.9188766E-1,6.0476685E-1,2.972863E-1,8.0297303E-1,1.975645E-1,4.6644235E-1,8.9068925E-1,2.669872E-1,0E0,0E0,4.828894E-1,0E0,6.044476E-2,2.1825123E-1,3.8755083E-1,1.9265217E-1,6.826532E-1,1.8109734E-1,1.0956398E-1,3.164283E-1,1.4807415E-1,0E0,0E0,4.5170188E-2,5.9051156E-2,1.5427318E-1,3.0654085E-1,6.091845E-2,3.8964592E-2,0E0,0E0,9.500469E-2,2.8814012E-1,0E0,1.1074269E-1,2.7767336E-1,2.91803E-1,6.3414335E-2,2.2052176E-2,0E0,2.5466818E-2,0E0,0E0,0E0,7.595815E-2,2.5562325E-1,3.544889E-1,6.848256E-2,0E0,0E0,0E0,1.2475047E-1,1.3266331E-1,1.5481836E-1,3.7513608E-1,3.8779497E-2,0E0,0E0,1.7525658E-1,8.065164E-2,1.6567805E-1,0E0,0E0,0E0,0E0,0E0,3.36269E-2,0E0,0E0,2.4555314E-1,0E0,5.791509E-2,0E0,0E0,1.9443631E-2,6.5717354E-2,0E0,0E0,5.0117612E-2,1.4119728E-1,1.3520245E-1,0E0,4.095189E-2,1.7093003E-2,0E0,2.7813292E-1,1.1648506E-1,3.817749E-2,0E0,0E0,1.1007944E-1,0E0,0E0,1.5663701E-1,1.1122024E-1,6.344092E-2,0E0,0E0,0E0,5.2711368E-2,7.35201E-2,3.3243068E-2,0E0,6.2510654E-2,5.7024326E-2,0E0,9.436458E-2,0E0,0E0,0E0,0E0,3.4580347E-1,1.1800045E-1,5.971104E-2,0E0,0E0,3.0590415E-2,2.8536215E-2,0E0,5.8332376E-2,6.5071344E-2,9.224582E-2,9.945354E-2,0E0,0E0,1.678972E-2,3.431256E-2,0E0,5.089094E-2,0E0,2.5351577E-2,0E0,0E0,3.1808384E-2,3.2107137E-2,1.1149431E-1,9.96535E-2,0E0,0E0,1.4276846E-1,1.5710282E-1,4.9098134E-2,7.9258405E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,38,38,39,39,40,40,42,42,46,46,47,47,48,48,49,49,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,68,68,71,71,73,73,76,76,77,77,80,80,81,81,82,82,84,84,85,85,87,87,88,88,89,89,92,92,95,95,96,96,97,97,101,101,102,102,103,103,105,105,106,106,108,108,113,113,114,114,115,115,118,118,119,119,121,121,122,122,123,123,124,124,127,127,128,128,130,130,132,132,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,62,64,66,-1,68,-1,-1,-1,70,72,74,76,-1,-1,-1,78,80,82,84,86,-1,-1,88,90,92,-1,-1,-1,-1,-1,94,-1,-1,96,-1,98,-1,-1,100,102,-1,-1,104,106,108,-1,110,112,-1,114,116,118,-1,-1,120,-1,-1,122,124,126,-1,-1,-1,128,130,132,-1,134,136,-1,138,-1,-1,-1,-1,140,142,144,-1,-1,146,148,-1,150,152,154,156,-1,-1,158,160,-1,162,-1,164,-1,-1,166,168,170,172,-1,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.785302E5,3.5056704E7,1E0,4.4E1,3.3817584E7,1.3655363E2,3.89E2,9.639872E0,5.817547E2,-1.0464207E-2,2.0494487E-2,6.5590655E6,-8.86112E-3,2.342E4,6.7E1,6E0,5.7E1,1.753E3,2.308943E0,1.2895454E3,1.5142261E3,1.8029197E0,3.285471E-3,-3.5930462E-3,8.8324326E-1,1.3E1,1.89991E5,2.0734E4,5.6E1,5E0,-2.515075E-2,-1.7462966E-3,1.047E3,2.954124E6,-2.2505E-3,2.0319266E-7,5.6E1,8.710612E1,1E0,5.941442E6,-6.518899E-3,3.774648E0,-3.2511433E-3,-1.0478092E-2,1.1660193E-2,1.6595395E5,4.6E1,4.5179688E1,1.2E1,-8.221557E-3,-2.4021226E-3,2.194635E-3,1.683263E7,6.923077E-1,6.848509E7,4.1957852E2,3.361317E1,1.7520034E-4,1.1522659E-2,2.9251662E6,3.1095755E0,6.516E3,1.3075406E-2,4.8180684E-3,2.2034561E-3,6.7632147E-3,6.2657185E-5,2.8E1,-2.8989706E-3,4.047658E-3,1.5292561E2,-1.2008373E-2,2.0421052E0,2.968251E-3,4.992811E-4,3.3E1,6E0,-5.6677135E-3,8.975173E-3,1.28738E5,1.901875E2,3.168978E-2,-1.4495105E-2,1.535E3,1.8425926E0,1.9256801E-3,1.4598765E0,1E0,1.990351E1,4.397981E-3,-3.2774562E-3,1.270394E5,-6.0479003E-3,-1.4568117E-3,4.577342E0,7.09617E6,1.0609756E0,-5.9300605E-3,-6.481607E-3,-2.6139272E-3,2.6676828E2,3.5714287E-1,3.1096E4,5.0526774E-3,1.3938298E2,1E0,5.607442E-3,1.2768175E6,-2.8369317E-3,3.0765005E-3,3.3858994E-3,8.153802E-3,1.38132E5,1.0901037E10,1.500502E6,-1.104112E-3,6.4709918E-3,1.670046E1,2.337765E3,7.1208873E-3,1.31E2,2.511352E6,8.454545E0,2.5417458E1,-4.4824574E-3,-1.3216604E-2,4.28649E5,1.5E1,3.2110284E-3,8.1E2,-1.4431196E-3,2.3773398E3,-1.8732371E-3,-1.0216361E-2,3.491E3,1.9E1,1.1E1,1.990351E1,3.4651808E-3,-1.3382404E-2,1.0717949E1,5.8475E5,4.5866325E6,1.552356E0,7.4967253E-3,1.4551862E-2,-1.36519E-3,3.0146171E-3,1.2837243E-3,-2.0682854E-3,-5.330412E-3,-2.7189185E-3,8.006539E-4,-1.9510069E-3,3.610932E-3,-8.1835163E-4,4.666905E-4,3.303524E-3,2.0861109E-3,7.5430125E-3,-5.617559E-3,4.0883533E-4,4.1358406E-3,7.0317305E-4,2.858946E-3,-1.2262574E-4,-2.4527307E-3,1.3239514E-3,7.4568996E-4,-5.464974E-3,4.5384523E-3,-2.5098824E-3,2.7692802E-3,-1.8188662E-3,8.140432E-3,1.7950365E-3,8.773918E-3,2.7231404E-3,6.678367E-3,-4.7211503E-5],"split_indices":[2,43,60,6,3,7,71,1,69,67,0,0,43,0,9,44,3,6,44,69,48,67,69,0,0,68,5,9,44,0,8,0,0,2,62,0,52,0,73,19,60,0,69,0,0,0,43,3,71,3,0,0,0,60,68,7,4,71,0,0,43,53,44,0,0,0,0,0,2,0,0,67,0,68,0,0,3,8,0,0,1,4,53,0,2,69,0,69,6,73,0,0,48,0,0,73,12,68,0,0,0,4,68,9,0,4,26,0,62,0,0,0,0,9,46,9,0,0,71,67,0,44,5,73,73,0,0,9,3,0,2,0,48,0,0,12,3,8,73,0,0,71,1,43,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,7.66E2,2.73E2,4.66E2,3E2,2.53E2,2E1,3.99E2,6.7E1,2.96E2,4E0,6E0,2.47E2,1.1E1,9E0,5.4E1,3.45E2,5.5E1,1.2E1,2.59E2,3.7E1,2.14E2,3.3E1,5E0,4E0,2.9E1,2.5E1,1.8E1,3.27E2,4.1E1,1.4E1,8E0,4E0,9.8E1,1.61E2,4E0,3.3E1,1.51E2,6.3E1,2.2E1,1.1E1,7E0,2.2E1,4E0,2.1E1,4E0,1.4E1,2.9E2,3.7E1,2.4E1,1.7E1,6E0,8E0,5.6E1,4.2E1,1.47E2,1.4E1,2.6E1,7E0,6E0,1.45E2,3E1,3.3E1,1.8E1,4E0,6E0,5E0,7E0,1.5E1,4E0,1E1,2.86E2,4E0,3E1,7E0,5E0,1.9E1,5.1E1,5E0,7E0,3.5E1,8.5E1,6.2E1,6E0,8E0,2.2E1,4E0,1.06E2,3.9E1,2.3E1,7E0,6E0,2.7E1,5E0,1E1,1.32E2,1.54E2,1.2E1,1.8E1,1.3E1,6E0,3.5E1,1.6E1,2.6E1,9E0,8E0,7.7E1,1.4E1,4.8E1,4E0,4E0,4E0,1.8E1,9E0,9.7E1,3.3E1,6E0,7E0,1.6E1,1.1E1,1.6E1,4.3E1,8.9E1,1.11E2,4.3E1,4E0,8E0,2.4E1,1.1E1,5E0,1.1E1,7E0,1.9E1,4E0,4E0,4.1E1,3.6E1,2.7E1,2.1E1,4E0,5E0,5.6E1,4.1E1,1.9E1,1.4E1,6E0,1E1,5E0,6E0,1.9E1,2.4E1,4.2E1,4.7E1,4E1,7.1E1,2.5E1,1.8E1,1.9E1,5E0,6E0,5E0,6E0,5E0,7E0,1.2E1,9E0,3.2E1,3.1E1,5E0,1.9E1,8E0,1.6E1,5E0,2.7E1,2.9E1,1.1E1,3E1,1.4E1,5E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[-2.3257365E-3,-9.361101E-2,2.9138543E-2,-7.504461E-2,-2.1700005E-1,-4.3321406E-3,1.011941E-1,-1.1467243E-1,-3.931716E-2,-1.2133072E-1,-2.167969E-2,-3.6855094E-2,2.0264916E-2,1.1289062E-1,-7.688294E-4,-1.4771907E-1,-7.8278266E-2,1.1763657E-2,-6.197982E-2,-1.60153E-1,-4.319978E-2,-3.0141326E-2,-1.5133677E-2,2.4603236E-2,-9.040706E-3,1.0391565E-1,1.1384624E-2,1.3134174E-1,-1.0640603E-1,-1.0529617E-1,-1.7624575E-1,1.9840815E-3,-9.1524586E-2,-3.8676467E-2,-1.3649903E-1,-7.770432E-2,-1.232758E-2,2.9120783E-4,-3.568988E-3,-4.25794E-2,2.9313734E-2,6.865808E-2,6.948213E-3,6.336656E-2,1.3131529E-1,1.2039306E-2,-1.9101286E-3,-9.710885E-4,-9.285439E-3,-1.8202295E-3,-1.287337E-1,-2.0858306E-1,-3.2538017E-3,-1.2343109E-1,-5.3145684E-2,-1.4649846E-2,-9.6331365E-2,-8.9190565E-3,-3.6211764E-3,-7.5402656E-3,2.6586677E-3,-3.4477558E-2,-1.9628441E-1,6.643207E-2,-8.497266E-2,3.4993973E-2,9.441506E-2,-2.985897E-2,4.1015346E-2,7.423596E-2,-4.2558517E-3,1.493828E-1,2.5508147E-2,-7.6652523E-3,-3.3778017E-3,-2.4586946E-1,-2.575395E-3,-1.5095799E-1,-5.4662734E-2,-1.7122427E-2,-5.6848805E-3,2.5977049E-2,-3.9675407E-2,-7.3686386E-3,-9.345938E-3,-4.9747836E-2,9.4321783E-4,-3.1843367E-3,-1.313834E-2,4.1217147E-3,8.762426E-4,1.9442398E-3,-8.16296E-3,9.34223E-2,8.7435124E-4,8.1993686E-4,1.0738812E-1,-1.584113E-2,-1.3796739E-2,9.281836E-2,-2.1185244E-3,2.3200482E-3,9.659797E-2,1.7556436E-1,8.20661E-2,9.7870804E-2,-3.9834836E-3,-1.3162144E-2,-7.2342553E-3,-8.248221E-3,-2.011797E-3,4.8022768E-5,-4.593682E-3,6.649162E-4,-3.5711122E-3,-1.5925014E-2,6.588088E-2,-9.046201E-3,-9.591206E-2,2.3565071E-3,-3.6128052E-3,-5.725016E-2,4.075345E-2,1.3056016E-2,-3.7999048E-3,6.9344183E-4,6.946276E-3,-4.660225E-3,3.1354763E-2,2.837775E-2,1.262539E-1,3.9971434E-2,-4.8893124E-2,1.17042884E-1,-7.2920835E-4,-5.25034E-2,3.1982567E-2,3.6384843E-3,-2.1711405E-2,1.451414E-1,5.885555E-2,1.4906694E-1,1.3861813E-2,5.108682E-2,1.0761651E-2,9.2859525E-4,6.2019355E-3,-2.261614E-3,2.1580318E-3,4.2798687E-3,2.0968488E-4,-2.5722475E-3,5.3882714E-2,-6.505277E-3,-5.8119214E-4,-2.8470319E-2,-7.3642984E-2,3.9775637E-3,-1.766854E-4,-1.1563249E-2,6.301725E-2,5.4900598E-2,-2.3220961E-3,-8.796674E-4,3.239513E-3,1.4647314E-1,3.1010772E-3,1.08308464E-1,6.893903E-3,-1.3479684E-1,-1.6083207E-2,6.465944E-2,2.0353024E-1,4.8096015E-4,-8.272238E-2,9.167583E-2,-2.3340499E-2,-2.5334295E-3,4.6874394E-4,1.6548668E-1,2.2980636E-3,-1.4143132E-3,8.551146E-2,1.2755826E-2,1.2556845E-1,7.33895E-2,-2.8515377E-3,1.741479E-4,4.775488E-3,-4.848594E-4,-5.516882E-3,-7.39282E-3,-2.9940906E-3,-1.0144135E-3,2.3578054E-3,3.9681285E-3,4.8866356E-4,3.6606938E-3,-6.693271E-4,2.9359742E-3,7.4189096E-3,6.2821656E-3,1.9575302E-3,1.088083E-3,-2.4281596E-3,-2.3489571E-3,-1.0297211E-2,9.076303E-4,-4.2149E-3,5.7357843E-3,1.0329944E-3,1.2627515E-2,5.8501055E-3,-1.3251698E-3,-6.4646406E-3,1.3527403E-3,5.563906E-3,-3.0664308E-3,1.216456E-3,5.0566373E-3,9.165458E-3,6.077624E-3,2.2166981E-3,1.0801344E-2,5.1971795E-3,4.294069E-3,7.1184384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,35,37,39,-1,41,-1,43,-1,45,47,49,51,-1,53,55,57,59,-1,-1,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,-1,77,79,81,83,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,-1,109,111,113,-1,115,117,-1,119,121,123,-1,-1,-1,-1,-1,-1,125,127,-1,129,131,-1,133,135,137,139,141,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,153,-1,-1,155,157,159,-1,-1,-1,-1,161,163,165,167,169,171,-1,173,175,-1,177,179,181,183,-1,185,-1,-1,-1,-1,-1,-1,-1,-1,187,-1,-1,189,191,-1,-1,193,195,197,-1,-1,-1,199,-1,201,203,205,207,209,211,-1,213,215,217,-1,-1,219,-1,-1,221,-1,223,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9582477E0,5.885105E-1,1.8478379E0,3.2254136E-1,7.6362646E-1,4.199306E-1,2.8989124E-1,1.1953163E-1,8.351613E-1,7.194519E-2,0E0,4.3123996E-1,2.849426E-1,2.3565936E-1,3.768261E-1,4.9212337E-2,9.064275E-2,0E0,1.9442368E-1,1.287958E-1,1.734406E-2,1.6541861E-1,0E0,2.28193E-1,0E0,2.2055125E-1,0E0,2.7862227E-1,1.1691502E-1,3.3539206E-2,9.480685E-2,0E0,5.344248E-2,1.21876225E-1,7.3616385E-2,1.3660975E-1,0E0,0E0,0E0,2.2236672E-1,1.7108765E-1,6.953129E-2,2.6580077E-1,1.5049243E-1,2.2941351E-1,0E0,0E0,0E0,0E0,0E0,2.5096208E-2,1.2306404E-1,0E0,4.0965676E-2,5.8331028E-2,6.610015E-2,1.4136745E-1,0E0,0E0,0E0,0E0,9.5242694E-2,8.138168E-2,3.0751914E-2,1.2397021E-1,7.56733E-2,4.4840574E-2,3.7957314E-1,2.4598771E-1,1.2843063E-1,0E0,1.6888762E-1,1.6205882E-1,0E0,0E0,2.1597743E-2,0E0,4.5546174E-2,2.2547279E-2,3.3351928E-2,0E0,4.2850472E-2,6.8031736E-2,0E0,5.3856578E-2,8.584648E-2,5.5490278E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.0242854E-2,8.069519E-2,0E0,5.6040943E-2,1.8273656E-1,0E0,1.333895E-1,1.0648812E-1,3.844759E-2,1.03218555E-1,1.9935656E-1,1.3299358E-1,3.1246841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9181164E-2,2.091834E-2,8.223039E-2,5.0608307E-2,0E0,0E0,5.1686585E-2,2.054212E-2,5.9769113E-2,0E0,0E0,0E0,0E0,4.119456E-2,1.9149698E-2,2.8425872E-2,8.288889E-2,1.7257862E-1,1.7259061E-1,0E0,4.839112E-2,1.2432909E-1,0E0,1.744135E-2,4.2231977E-2,8.847342E-2,1.5339875E-1,0E0,7.246068E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9368546E-2,0E0,0E0,6.9041744E-2,4.457295E-2,0E0,0E0,2.259264E-2,1.8284276E-2,2.8167114E-2,0E0,0E0,0E0,1.671055E-2,0E0,1.6133487E-2,2.6987493E-2,1.1439049E-1,1.253054E-1,6.517204E-2,4.5903146E-2,0E0,4.85404E-2,2.877155E-2,4.449857E-2,0E0,0E0,1.7003655E-2,0E0,0E0,4.2070195E-2,0E0,5.1519215E-2,2.4649315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,25,25,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,39,39,40,40,41,41,42,42,43,43,44,44,50,50,51,51,53,53,54,54,55,55,56,56,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,77,77,78,78,79,79,81,81,82,82,84,84,85,85,86,86,93,93,94,94,96,96,97,97,99,99,100,100,101,101,102,102,103,103,104,104,105,105,115,115,116,116,117,117,118,118,121,121,122,122,123,123,128,128,129,129,130,130,131,131,132,132,133,133,135,135,136,136,138,138,139,139,140,140,141,141,143,143,152,152,155,155,156,156,159,159,160,160,161,161,165,165,167,167,168,168,169,169,170,170,171,171,172,172,174,174,175,175,176,176,179,179,182,182,184,184,185,185],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,36,38,40,-1,42,-1,44,-1,46,48,50,52,-1,54,56,58,60,-1,-1,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,-1,78,80,82,84,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,-1,110,112,114,-1,116,118,-1,120,122,124,-1,-1,-1,-1,-1,-1,126,128,-1,130,132,-1,134,136,138,140,142,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,154,-1,-1,156,158,160,-1,-1,-1,-1,162,164,166,168,170,172,-1,174,176,-1,178,180,182,184,-1,186,-1,-1,-1,-1,-1,-1,-1,-1,188,-1,-1,190,192,-1,-1,194,196,198,-1,-1,-1,200,-1,202,204,206,208,210,212,-1,214,216,218,-1,-1,220,-1,-1,222,-1,224,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,4.4895835E0,2.652E3,1.19111115E2,1E0,8.1E2,2.3876712E7,1.3E1,7.336111E4,8.237624E0,-2.167969E-2,1E0,1.2294118E1,9.059046E6,5.2224144E5,1.00033E5,2.88E0,1.1763657E-2,4.76E2,2.9E1,6.763314E7,3.2911258E6,-1.5133677E-2,8.960784E0,-9.040706E-3,1.1342433E3,1.1384624E-2,3.827E5,1.3784861E5,2.8282208E0,2.2470966E-1,1.9840815E-3,1.4473684E-1,1.3439851E7,9.3326636E-2,1.6254545E1,-1.232758E-2,2.9120783E-4,-3.568988E-3,8.84E2,8E0,1.071E3,3.850926E2,1.2118524E7,4.9036694E0,1.2039306E-2,-1.9101286E-3,-9.710885E-4,-9.285439E-3,-1.8202295E-3,4E-1,7.2896E5,-3.2538017E-3,6.44E2,7E0,8E0,1.0590052E8,-8.9190565E-3,-3.6211764E-3,-7.5402656E-3,2.6586677E-3,1.559733E6,2.124E3,9.244374E6,5.75E2,1.6580646E2,7.2864324E-1,2.41E3,3.8275862E0,4.9963706E5,-4.2558517E-3,8.7E1,4.88E2,-7.6652523E-3,-3.3778017E-3,2.72E0,-2.575395E-3,5.0406504E0,1.2445087E2,5.6321215E4,-5.6848805E-3,2.7976523E2,1.0885427E9,-7.3686386E-3,1.8729467E8,1.5512196E1,4.888889E0,-3.1843367E-3,-1.313834E-2,4.1217147E-3,8.762426E-4,1.9442398E-3,-8.16296E-3,8.9E2,1E1,8.1993686E-4,2.492E2,1.5522388E0,-1.3796739E-2,3.9888642E0,5.62E2,4.0663E4,9.318287E0,1.9E1,4.91027E0,2.8389828E7,-3.9834836E-3,-1.3162144E-2,-7.2342553E-3,-8.248221E-3,-2.011797E-3,4.8022768E-5,-4.593682E-3,6.649162E-4,-3.5711122E-3,2.0134516E6,1.6595395E5,8.39E2,2.58466E5,2.3565071E-3,-3.6128052E-3,1.4695653E0,1.8645384E1,1.142E3,-3.7999048E-3,6.9344183E-4,6.946276E-3,-4.660225E-3,1E0,4.07E2,1E0,4.05E2,3.49E2,1.95E2,-7.2920835E-4,3.8795E4,9.822E3,3.6384843E-3,2.1340163E8,6.5590655E6,1.5724638E1,1.0586236E5,1.3861813E-2,9.100503E0,1.0761651E-2,9.2859525E-4,6.2019355E-3,-2.261614E-3,2.1580318E-3,4.2798687E-3,2.0968488E-4,-2.5722475E-3,1.24272164E5,-6.505277E-3,-5.8119214E-4,1.137832E-7,2.8E0,3.9775637E-3,-1.766854E-4,5E0,2.7432E4,5.831829E6,-2.3220961E-3,-8.796674E-4,3.239513E-3,4.3827028E5,3.1010772E-3,3.7509E4,3.7253174E2,7.09415E5,4.2E2,3E0,1.5416006E5,4.8096015E-4,1.08E2,1.609E3,1.862E3,-2.5334295E-3,4.6874394E-4,5.893737E2,2.2980636E-3,-1.4143132E-3,3.307E3,1.2755826E-2,3.596E3,3.0546486E8,-2.8515377E-3,1.741479E-4,4.775488E-3,-4.848594E-4,-5.516882E-3,-7.39282E-3,-2.9940906E-3,-1.0144135E-3,2.3578054E-3,3.9681285E-3,4.8866356E-4,3.6606938E-3,-6.693271E-4,2.9359742E-3,7.4189096E-3,6.2821656E-3,1.9575302E-3,1.088083E-3,-2.4281596E-3,-2.3489571E-3,-1.0297211E-2,9.076303E-4,-4.2149E-3,5.7357843E-3,1.0329944E-3,1.2627515E-2,5.8501055E-3,-1.3251698E-3,-6.4646406E-3,1.3527403E-3,5.563906E-3,-3.0664308E-3,1.216456E-3,5.0566373E-3,9.165458E-3,6.077624E-3,2.2166981E-3,1.0801344E-2,5.1971795E-3,4.294069E-3,7.1184384E-4],"split_indices":[43,68,2,67,6,2,60,3,60,71,0,106,68,43,43,5,69,0,0,3,7,43,0,73,0,4,0,10,48,69,53,0,71,60,53,69,0,0,0,10,8,2,67,62,53,0,0,0,0,0,73,9,0,1,8,32,7,0,0,0,0,9,44,60,2,67,73,2,69,43,0,8,0,0,0,69,0,69,4,43,0,4,46,0,7,71,71,0,0,0,0,0,0,2,3,0,4,68,0,61,10,9,71,8,50,62,0,0,0,0,0,0,0,0,0,60,43,2,1,0,0,68,71,44,0,0,0,0,26,0,19,10,10,0,0,1,44,0,7,43,73,48,0,69,0,0,0,0,0,0,0,0,43,0,0,52,69,0,0,69,1,60,0,0,0,43,0,1,4,12,0,8,48,0,10,2,2,0,0,4,0,0,2,0,2,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,2.63E2,7.65E2,2.3E2,3.3E1,5.23E2,2.42E2,1.08E2,1.22E2,2.5E1,8E0,2.25E2,2.98E2,2.17E2,2.5E1,5.5E1,5.3E1,8E0,1.14E2,1.6E1,9E0,2.21E2,4E0,2.93E2,5E0,2.05E2,1.2E1,1.1E1,1.4E1,2.4E1,3.1E1,5E0,4.8E1,8.8E1,2.6E1,1E1,6E0,4E0,5E0,1.83E2,3.8E1,8.3E1,2.1E2,8.4E1,1.21E2,6E0,5E0,8E0,6E0,7E0,1.7E1,2.3E1,8E0,2.5E1,2.3E1,6.3E1,2.5E1,1.2E1,1.4E1,6E0,4E0,1.75E2,8E0,2.9E1,9E0,3.7E1,4.6E1,1.01E2,1.09E2,7.9E1,5E0,1.03E2,1.8E1,9E0,8E0,1.8E1,5E0,1.7E1,8E0,1.6E1,7E0,2.4E1,3.9E1,1.4E1,1.1E1,1.22E2,5.3E1,4E0,4E0,1.9E1,1E1,4E0,5E0,1.3E1,2.4E1,7E0,3.9E1,9.7E1,4E0,4.9E1,6E1,1.9E1,6E1,7.3E1,3E1,1.1E1,7E0,1.1E1,7E0,1.3E1,4E0,4E0,4E0,1.1E1,5E0,1.2E1,1.2E1,2.6E1,1.3E1,6E0,5E0,1.13E2,9E0,4.7E1,6E0,6E0,7E0,5E0,1.9E1,8E0,3.1E1,3.6E1,6.1E1,4E1,9E0,2.4E1,3.6E1,4E0,1.5E1,2.5E1,3.5E1,6.2E1,1.1E1,2.6E1,4E0,4E0,7E0,8E0,4E0,8E0,4E0,1.5E1,1.1E1,8E0,5E0,4.2E1,7.1E1,4E0,5E0,3.2E1,1.5E1,1.5E1,4E0,4E0,4E0,2.2E1,9E0,1.1E1,2.5E1,1.6E1,4.5E1,2.6E1,1.4E1,8E0,1.6E1,1.7E1,1.9E1,7E0,8E0,2E1,5E0,8E0,2.7E1,8E0,5.4E1,2.2E1,4E0,6E0,5E0,3.6E1,6E0,5E0,6.6E1,2.8E1,4E0,1E1,5E0,1.1E1,4E0,4E0,1.8E1,7E0,4E0,2E1,5E0,9E0,7E0,3.1E1,1.4E1,1E1,1.6E1,6E0,8E0,9E0,7E0,6E0,1.1E1,1E1,9E0,9E0,1.1E1,1.1E1,1.6E1,4E0,5E1,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[1.1009438E-3,-3.9112236E-2,7.401193E-2,-6.565131E-2,1.1799968E-2,9.642514E-2,-5.0982617E-2,-5.7125136E-2,-1.785663E-1,4.2089024E-3,1.4745581E-1,2.640706E-1,8.514786E-2,-1.3052501E-1,8.411319E-3,-8.500051E-2,-1.606398E-2,-1.3229026E-1,-1.7716803E-2,-3.3999672E-3,1.2328587E-1,2.1924123E-3,8.84822E-3,1.4566637E-2,4.643914E-3,1.23720154E-1,3.293409E-2,-2.8229157E-2,-2.015771E-1,8.455084E-2,-5.214153E-2,-7.6764576E-2,-2.5022778E-1,1.6954329E-2,-7.8922436E-2,-1.4988591E-1,-1.3125762E-3,7.1650865E-3,-8.685387E-2,2.9813452E-3,9.442373E-3,1.020845E-1,1.8986122E-1,2.0646023E-2,1.0955011E-2,-3.3969164E-3,1.0059561E-3,-1.0332464E-1,-1.4569315E-2,3.2375936E-4,6.2584677E-3,-8.223674E-3,1.9642137E-2,-1.6757555E-1,-6.591733E-2,-9.0988737E-4,-2.4158787E-2,8.109917E-3,6.0118563E-3,-9.401221E-3,-6.02769E-2,-3.3532484E-3,-7.7777305E-3,1.5964994E-2,-5.755993E-2,-1.1471005E-2,-1.5268955E-2,1.02831E-2,8.1624605E-2,2.6887571E-3,2.0287366E-1,2.8626302E-2,-6.2100897E-3,-1.9861078E-3,-6.6061923E-3,-3.379097E-3,5.064358E-3,-8.985815E-3,-4.1306373E-3,-5.707424E-2,-1.684672E-1,-4.080267E-2,2.5809642E-2,-3.4349825E-2,-6.545361E-3,2.1768924E-2,-7.385123E-2,-5.8555426E-3,-7.7722454E-4,2.6048787E-3,-4.7550974E-3,-2.1161705E-2,9.5475465E-2,2.1868771E-1,2.8222546E-3,-3.5680735E-3,3.522247E-2,3.0007742E-2,-6.511559E-2,-3.6720086E-3,-9.882433E-3,-1.5785502E-2,-9.1760166E-2,3.4780245E-2,-4.1845776E-2,-1.70479E-2,-5.985214E-3,-9.127189E-3,3.700356E-2,-5.6874775E-3,1.2595323E-6,-3.8020848E-3,5.323314E-4,1.0407894E-1,-3.943781E-3,1.0890828E-2,4.8748157E-3,-4.9988687E-2,4.4427574E-2,-1.7183678E-2,5.515672E-3,-6.930566E-2,2.0799518E-2,-1.9589649E-3,9.4224914E-4,-7.7101826E-3,-6.4023543E-4,4.5027938E-2,-2.0264322E-3,-3.6868136E-3,2.0699066E-4,-3.1238016E-2,1.605347E-3,6.6782977E-3,-7.2012E-3,-2.4768012E-2,5.0535183E-2,8.743726E-2,8.303924E-3,2.0616E-4,-5.36937E-3,5.4291193E-3,3.662511E-2,-2.3333067E-3,1.8280034E-3,-4.64554E-3,-2.6723002E-3,3.8047738E-3,-2.0751213E-3,-1.8756782E-3,2.4080144E-3,-3.35718E-3,-5.0970225E-4,1.0832227E-3,-1.5949009E-3,1.5376982E-3,-3.7117896E-3,6.5935426E-3,1.9816665E-3,5.313785E-3,1.0745503E-3,9.09989E-4,3.8047836E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,-1,-1,67,69,71,-1,-1,-1,73,-1,-1,-1,-1,75,77,79,-1,-1,-1,81,-1,83,-1,-1,85,87,-1,89,-1,91,-1,93,95,-1,-1,-1,-1,-1,-1,-1,97,99,101,103,105,-1,107,109,-1,-1,-1,-1,111,113,115,-1,-1,117,119,121,-1,-1,123,125,127,129,131,-1,133,135,-1,-1,-1,-1,137,-1,-1,-1,139,141,143,-1,145,147,-1,-1,-1,-1,149,-1,-1,-1,151,-1,153,-1,155,157,159,-1,-1,-1,-1,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0023127E0,8.925421E-1,1.0246727E0,4.0762532E-1,2.331377E-1,5.617058E-1,2.656607E-1,4.6219552E-1,2.3375797E-1,1.9612405E-1,4.3792754E-2,1.2750483E-1,5.8200073E-1,1.6874972E-1,1.5655395E-1,3.087443E-1,3.4379223E-1,4.3264776E-2,0E0,1.8048808E-1,4.8753113E-2,0E0,0E0,0E0,0E0,2.1959591E-1,3.087452E-1,2.6517067E-2,1.23927295E-1,5.8273993E-2,1.7516066E-1,2.1650028E-1,6.812315E-1,1.8809797E-1,1.2012792E-1,1.9186378E-2,0E0,1.0458145E-1,2.6201448E-1,0E0,0E0,2.974E-1,5.672252E-2,1.5110719E-1,0E0,0E0,0E0,1.5393563E-2,0E0,0E0,0E0,0E0,1.1654924E-1,3.3123553E-2,1.7901105E-1,0E0,0E0,0E0,9.651597E-2,0E0,1.02767706E-1,0E0,0E0,8.5534886E-2,5.98725E-2,0E0,1.1257855E-1,0E0,1.6004872E-1,0E0,6.735933E-2,8.2322314E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3710368E-1,4.7697335E-2,3.8070694E-2,4.5759827E-2,6.3679166E-2,0E0,7.188278E-2,3.6393583E-2,0E0,0E0,0E0,0E0,2.8922964E-2,1.6135865E-1,3.686881E-2,0E0,0E0,8.693366E-2,7.3758274E-2,6.515962E-2,0E0,0E0,2.2051597E-2,5.6215197E-2,5.43594E-2,1.6183186E-2,2.6810357E-2,0E0,1.1933007E-1,8.6510345E-2,0E0,0E0,0E0,0E0,1.026454E-1,0E0,0E0,0E0,4.01902E-2,5.1582277E-2,2.4691237E-2,0E0,5.4718196E-2,4.0106725E-2,0E0,0E0,0E0,0E0,3.7351184E-2,0E0,0E0,0E0,2.263704E-2,0E0,3.328061E-2,0E0,6.397338E-2,5.30473E-2,1.3716805E-1,0E0,0E0,0E0,0E0,6.805012E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,43,43,47,47,52,52,53,53,54,54,58,58,60,60,63,63,64,64,66,66,68,68,70,70,71,71,79,79,80,80,81,81,82,82,83,83,85,85,86,86,91,91,92,92,93,93,96,96,97,97,98,98,101,101,102,102,103,103,104,104,105,105,107,107,108,108,113,113,117,117,118,118,119,119,121,121,122,122,127,127,131,131,133,133,135,135,136,136,137,137,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,-1,-1,68,70,72,-1,-1,-1,74,-1,-1,-1,-1,76,78,80,-1,-1,-1,82,-1,84,-1,-1,86,88,-1,90,-1,92,-1,94,96,-1,-1,-1,-1,-1,-1,-1,98,100,102,104,106,-1,108,110,-1,-1,-1,-1,112,114,116,-1,-1,118,120,122,-1,-1,124,126,128,130,132,-1,134,136,-1,-1,-1,-1,138,-1,-1,-1,140,142,144,-1,146,148,-1,-1,-1,-1,150,-1,-1,-1,152,-1,154,-1,156,158,160,-1,-1,-1,-1,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,9.860918E5,1E0,1.0657745E7,7.82261E6,1.2083E4,1.9674084E5,6.99E2,7.627907E0,4.1E1,5.51E2,1.562E5,8E0,2.5345264E7,1.7234043E1,1.335955E2,7.453E3,5.1485147E0,-1.7716803E-2,2.8302418E10,1.0717949E1,2.1924123E-3,8.84822E-3,1.4566637E-2,4.643914E-3,6.46E2,4.91027E0,6E0,5E0,2.652E3,2.0900433E1,3.4E1,5.5E1,1.1634076E4,3.4402E4,1.4979E4,-1.3125762E-3,1E0,1.3254E4,2.9813452E-3,9.442373E-3,5.6E1,3.307E3,3.0392378E7,1.0955011E-2,-3.3969164E-3,1.0059561E-3,5.1917776E-2,-1.4569315E-2,3.2375936E-4,6.2584677E-3,-8.223674E-3,3.7827366E8,5.0363636E1,3.1E1,-9.0988737E-4,-2.4158787E-2,8.109917E-3,6.613774E5,-9.401221E-3,8.998703E-1,-3.3532484E-3,-7.7777305E-3,1E0,7.694314E-7,-1.1471005E-2,2.0217392E0,1.02831E-2,3.7043128E5,2.6887571E-3,4.7E1,8.147158E4,-6.2100897E-3,-1.9861078E-3,-6.6061923E-3,-3.379097E-3,5.064358E-3,-8.985815E-3,-4.1306373E-3,8.414097E-1,1.6E1,3.3E0,2.6789763E9,1.6533886E8,-6.545361E-3,1.7733E4,5.4814816E0,-5.8555426E-3,-7.7722454E-4,2.6048787E-3,-4.7550974E-3,1.6869566E1,1E0,2.8146256E5,2.8222546E-3,-3.5680735E-3,5.4814816E0,1.1E1,1.073125E1,-3.6720086E-3,-9.882433E-3,3.2638438E5,6.74502E0,1E0,3.57E2,6.5062125E5,-5.985214E-3,2.72E2,6.078218E2,-5.6874775E-3,1.2595323E-6,-3.8020848E-3,5.323314E-4,8.83E3,-3.943781E-3,1.0890828E-2,4.8748157E-3,6.1454544E0,3.8197617E9,9.183432E4,5.515672E-3,3.3820656E7,1.279012E7,-1.9589649E-3,9.4224914E-4,-7.7101826E-3,-6.4023543E-4,8.48E3,-2.0264322E-3,-3.6868136E-3,2.0699066E-4,8.818731E0,1.605347E-3,1E0,-7.2012E-3,3.6507E4,8.691756E-1,4.643602E8,8.303924E-3,2.0616E-4,-5.36937E-3,5.4291193E-3,1.836095E6,-2.3333067E-3,1.8280034E-3,-4.64554E-3,-2.6723002E-3,3.8047738E-3,-2.0751213E-3,-1.8756782E-3,2.4080144E-3,-3.35718E-3,-5.0970225E-4,1.0832227E-3,-1.5949009E-3,1.5376982E-3,-3.7117896E-3,6.5935426E-3,1.9816665E-3,5.313785E-3,1.0745503E-3,9.09989E-4,3.8047836E-3],"split_indices":[2,43,6,60,43,9,48,2,69,8,2,10,32,60,71,61,44,69,0,46,71,0,0,0,0,0,50,8,8,2,71,2,0,43,9,1,0,6,44,0,0,0,2,58,0,0,0,53,0,0,0,0,7,62,8,0,0,0,60,0,57,0,0,79,52,0,71,0,43,0,8,43,0,0,0,0,0,0,0,68,3,68,46,5,0,1,71,0,0,0,0,73,31,48,0,0,71,3,69,0,0,60,69,19,0,43,0,10,48,0,0,0,0,2,0,0,0,69,46,43,0,7,7,0,0,0,0,1,0,0,0,71,0,30,0,1,68,7,0,0,0,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,6.59E2,3.63E2,4.33E2,2.26E2,3.08E2,5.5E1,4.04E2,2.9E1,2.15E2,1.1E1,1.8E1,2.9E2,2.3E1,3.2E1,2.4E2,1.64E2,2.5E1,4E0,2.03E2,1.2E1,4E0,7E0,1.3E1,5E0,1.66E2,1.24E2,1E1,1.3E1,1.4E1,1.8E1,2.3E2,1E1,1.08E2,5.6E1,2.1E1,4E0,1.81E2,2.2E1,8E0,4E0,1.27E2,3.9E1,1.18E2,6E0,5E0,5E0,8E0,5E0,6E0,8E0,6E0,1.2E1,2.3E1,2.07E2,6E0,4E0,6E0,1.02E2,6E0,5E1,5E0,1.6E1,1.6E2,2.1E1,6E0,1.6E1,1.7E1,1.1E2,4E0,3.5E1,1.13E2,5E0,4E0,4E0,6E0,6E0,1.6E1,7E0,1.92E2,1.5E1,3E1,7.2E1,3.9E1,1.1E1,1.51E2,9E0,7E0,1.4E1,9E0,7E0,1.3E1,9.7E1,3.1E1,4E0,6E0,1.07E2,1.6E1,1.76E2,6E0,9E0,2.1E1,9E0,6.4E1,8E0,3.4E1,5E0,5E1,1.01E2,5E0,4E0,4E0,9E0,9.3E1,4E0,2.6E1,5E0,1E1,9.7E1,1.1E1,5E0,1.68E2,8E0,1.2E1,9E0,4E0,5E0,5.7E1,7E0,4E0,4E0,2.7E1,7E0,4.6E1,4E0,1.8E1,8.3E1,7.8E1,1.5E1,6E0,4E0,8E0,8.9E1,7E0,4E0,4.3E1,1.25E2,4E0,4E0,4E0,5.3E1,8E0,1.9E1,3.3E1,1.3E1,9E0,9E0,5E0,7.8E1,5.4E1,2.4E1,6.6E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[-1.7823981E-3,-4.0926073E-2,6.908415E-2,-8.466238E-2,-1.5719987E-2,3.4781877E-2,1.426687E-1,-6.65194E-2,-2.2224681E-1,-7.045499E-2,2.00722E-3,4.7250196E-2,-1.661988E-1,1.8625151E-2,1.224276E-1,8.938055E-2,-7.689234E-2,-3.3350907E-2,-4.3763816E-1,-5.9620846E-2,-9.797345E-3,-1.9559527E-3,8.344906E-3,1.6832297E-1,3.7220646E-2,-1.7415055E-3,-1.265517E-2,1.3706036E-1,-3.760608E-2,1.0930074E-2,6.8394765E-3,-1.0896457E-1,-5.3991854E-2,-5.618569E-3,1.782589E-2,-2.4994403E-2,-7.578095E-3,-9.8991886E-2,-2.288893E-2,1.421071E-2,-3.6013566E-2,9.800655E-3,3.1952346E-3,-6.388553E-2,5.154758E-2,-1.0179935E-3,1.491423E-1,-4.9310275E-3,2.442503E-3,-2.3401529E-3,3.440138E-3,-1.8716906E-1,-8.632807E-2,-3.3540778E-2,-9.6131526E-2,3.0979766E-3,-1.5881547E-3,4.572157E-4,-1.1829479E-1,-1.0720574E-2,-6.2556736E-2,2.6304418E-2,-2.0596504E-2,4.977003E-3,-4.773916E-2,1.3387327E-2,-1.9496838E-2,7.179611E-2,1.1792234E-2,1.7904058E-1,1.0037008E-1,-2.2123496E-1,-2.5109504E-3,1.3077036E-3,-9.602929E-2,-4.3801606E-2,2.5729572E-3,-2.8782912E-2,-1.6550991E-1,-8.080106E-3,-8.779007E-2,-5.083314E-4,-2.7835057E-3,-1.0223103E-3,-4.587428E-3,4.0390793E-2,-1.4233276E-2,-1.2336766E-1,1.3342861E-2,-1.1219512E-1,-2.3116386E-2,6.9491334E-2,-4.966726E-3,8.692444E-2,-1.0145965E-2,-1.0988743E-2,5.948917E-3,1.8852025E-1,3.0973414E-3,-1.1658761E-2,1.4079677E-1,-7.1715E-3,-1.4275366E-2,-8.131578E-2,-7.2465115E-3,-2.5789065E-2,-8.602322E-2,-1.4941373E-1,5.6944806E-2,-9.179282E-3,-2.7952767E-3,-5.4082677E-2,-5.434599E-3,-1.3082469E-2,3.5828743E-3,1.1836862E-2,6.163217E-2,-6.3834814E-3,2.4491977E-3,-1.09247165E-2,-2.5168979E-3,-1.6950825E-2,6.552288E-2,-6.0261615E-2,-1.6776241E-1,-8.509088E-3,-1.0564108E-2,5.828922E-4,6.093556E-3,5.5383097E-2,1.2170904E-1,6.0001258E-2,-9.330804E-2,6.4652073E-3,-9.366883E-2,1.6613105E-1,1.3439045E-2,-2.1326744E-3,1.8007722E-3,9.645941E-2,9.746061E-3,-9.264131E-2,2.0176284E-3,-3.802059E-2,9.327427E-4,-6.6219675E-3,-1.6317331E-3,-1.1597686E-2,-8.2730426E-4,-3.2230516E-4,4.5167725E-3,-3.4073547E-3,4.7528482E-7,1.1356301E-3,-1.9241748E-3,-4.3642693E-3,8.031804E-2,7.39972E-2,-8.077768E-4,-4.180844E-3,1.604562E-2,-1.5501073E-3,1.0083345E-3,4.472581E-3,-1.0474909E-3,-3.4120228E-4,-8.596113E-2,-3.4949712E-3,-1.029082E-2,2.2867149E-2,-5.1443856E-2,9.102348E-2,-4.608941E-2,5.3078935E-2,1.488305E-1,-1.2253443E-3,4.923842E-3,2.323397E-3,-9.120735E-3,-3.668458E-3,1.829681E-2,-5.8822064E-3,-1.5961746E-3,1.465743E-1,1.3548806E-2,1.9197954E-3,5.775997E-3,-6.1476664E-3,-3.3300633E-3,-7.5694895E-4,-5.4485486E-3,-1.4037599E-3,1.0096745E-3,6.1842E-3,1.0308623E-3,4.1973614E-3,1.0633453E-3,2.3505064E-3,-1.4537458E-3,-5.8272686E-3,-9.499235E-4,-3.1516328E-4,5.358671E-3,-3.3846407E-3,1.4188082E-3,5.24261E-4,5.2724076E-3,2.1776587E-3,-6.70844E-3,-4.4813997E-4,3.811271E-3,7.7153766E-3,1.392189E-3,1.4253506E-3,-2.3414674E-3,1.0679259E-2,5.7965866E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,-1,41,43,-1,-1,45,47,-1,49,51,53,-1,55,-1,-1,57,59,61,63,-1,-1,65,67,-1,69,-1,-1,-1,-1,71,73,75,77,-1,-1,-1,79,81,83,85,87,-1,89,91,-1,93,95,97,99,101,-1,-1,103,105,-1,107,109,-1,111,113,-1,-1,-1,115,117,119,121,123,125,127,-1,129,131,133,-1,135,-1,137,139,-1,-1,141,-1,143,145,147,149,-1,-1,151,-1,153,-1,155,157,-1,159,-1,-1,161,163,165,167,-1,169,-1,-1,171,173,175,177,179,181,183,-1,-1,-1,185,-1,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,191,193,195,-1,-1,197,-1,-1,-1,-1,-1,199,-1,-1,201,203,205,207,209,211,-1,-1,-1,-1,-1,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8766673E0,7.3508644E-1,9.26437E-1,5.942868E-1,4.125184E-1,6.4184344E-1,5.675175E-1,3.5779214E-1,1.1246276E0,1.4691907E-1,2.283005E-1,2.8395385E-1,1.9732544E-1,0E0,2.6274514E-1,1.6848947E-1,1.4394867E-1,7.444237E-2,2.7895808E-1,1.3945916E-1,0E0,1.7450075E-1,0E0,6.353611E-2,3.2486933E-1,0E0,0E0,1.9742656E-1,6.818693E-2,0E0,4.2650983E-2,1.3254523E-1,1.01053625E-1,0E0,3.0574983E-2,0E0,0E0,9.995845E-2,2.4429344E-2,9.104566E-2,1.7706119E-1,0E0,0E0,7.805198E-1,1.561259E-1,0E0,1.1910081E-1,0E0,0E0,0E0,0E0,6.941813E-2,7.722819E-2,7.843513E-2,1.7746517E-1,0E0,0E0,0E0,5.371189E-2,2.0707555E-2,1.5626907E-2,9.193957E-2,1.969734E-1,0E0,1.4817952E-1,1.7025469E-1,0E0,1.6079628E-1,1.7995669E-1,4.787779E-2,1.7374176E-1,3.3131897E-2,0E0,0E0,4.273194E-2,5.482045E-2,0E0,2.2567782E-1,5.4175317E-2,0E0,2.1805331E-2,3.5368964E-2,0E0,0E0,0E0,7.144691E-2,8.71788E-2,9.6197665E-2,6.908185E-2,6.492126E-2,1.3957712E-1,5.867447E-2,0E0,1.1284906E-1,1.2810396E-1,8.286195E-2,0E0,7.086134E-2,0E0,2.101843E-2,6.675249E-2,0E0,0E0,7.765496E-2,0E0,3.1136673E-2,5.8868155E-2,1.157819E-1,3.4604877E-2,0E0,0E0,1.5961781E-2,0E0,3.437814E-2,0E0,5.80431E-2,6.781703E-2,0E0,4.9473673E-2,0E0,0E0,1.8757867E-2,4.6347134E-2,1.9748509E-2,4.3928772E-2,0E0,9.137602E-2,0E0,0E0,2.1756987E-1,8.702904E-2,5.1187996E-2,1.5943608E-1,5.0220292E-2,1.5497029E-2,6.8903446E-2,0E0,0E0,0E0,2.460298E-2,0E0,3.0306697E-2,0E0,7.241744E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.000048E-2,2.8069824E-2,4.8611194E-2,0E0,0E0,5.7869907E-2,0E0,0E0,0E0,0E0,0E0,2.3679346E-2,0E0,0E0,1.0308711E-1,5.6024097E-2,7.968804E-2,1.5469018E-1,3.0349419E-2,7.26192E-2,0E0,0E0,0E0,0E0,0E0,3.880467E-2,0E0,0E0,4.418516E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,27,27,28,28,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,43,43,44,44,46,46,51,51,52,52,53,53,54,54,58,58,59,59,60,60,61,61,62,62,64,64,65,65,67,67,68,68,69,69,70,70,71,71,74,74,75,75,77,77,78,78,80,80,81,81,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,97,97,99,99,100,100,103,103,105,105,106,106,107,107,108,108,111,111,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,139,139,141,141,143,143,155,155,156,156,157,157,160,160,166,166,169,169,170,170,171,171,172,172,173,173,174,174,180,180,183,183],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,-1,42,44,-1,-1,46,48,-1,50,52,54,-1,56,-1,-1,58,60,62,64,-1,-1,66,68,-1,70,-1,-1,-1,-1,72,74,76,78,-1,-1,-1,80,82,84,86,88,-1,90,92,-1,94,96,98,100,102,-1,-1,104,106,-1,108,110,-1,112,114,-1,-1,-1,116,118,120,122,124,126,128,-1,130,132,134,-1,136,-1,138,140,-1,-1,142,-1,144,146,148,150,-1,-1,152,-1,154,-1,156,158,-1,160,-1,-1,162,164,166,168,-1,170,-1,-1,172,174,176,178,180,182,184,-1,-1,-1,186,-1,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,192,194,196,-1,-1,198,-1,-1,-1,-1,-1,200,-1,-1,202,204,206,208,210,212,-1,-1,-1,-1,-1,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1E0,7.098E3,3.7568388E0,5.02E2,4.217427E7,1.2646534E4,7E0,8.317E3,4.519E3,7.82261E6,4.125E0,1.7E1,1.8625151E-2,3.206931E2,1.3050649E4,9.9E1,3.3857143E1,2.277451E2,1.463E2,-9.797345E-3,1.4E1,8.344906E-3,1.94261E5,5.9754E4,-1.7415055E-3,-1.265517E-2,2.0778275E-1,6.9879E4,1.0930074E-2,6.228E1,9.2058825E-1,1.1891E4,-5.618569E-3,3.6E1,-2.4994403E-2,-7.578095E-3,2.824856E-3,2.1481E4,1E0,8.6E1,9.800655E-3,3.1952346E-3,2.466E3,4.6603775E0,-1.0179935E-3,7.153514E1,-4.9310275E-3,2.442503E-3,-2.3401529E-3,3.440138E-3,3.89E2,5E0,1.3E1,1.7717391E0,3.0979766E-3,-1.5881547E-3,4.572157E-4,9.917037E2,4.5726547E5,4.5087484E5,4.9390244E0,2.0217392E0,4.977003E-3,2.7534653E2,1.1E1,-1.9496838E-2,2E0,1.836095E6,1.7655972E0,1.3388E4,4.456432E6,-2.5109504E-3,1.3077036E-3,3.2856784E7,6.4153387E-6,2.5729572E-3,1.169375E2,3.8575E4,-8.080106E-3,4.325E0,3.8767453E4,-2.7835057E-3,-1.0223103E-3,-4.587428E-3,1.6494E4,5.633663E0,3.2962964E0,4.4970587E2,1.9578822E5,1.5E1,1E0,-4.966726E-3,1.3457517E10,1.8645384E1,2.2E1,5.948917E-3,2.5045E4,3.0973414E-3,3.8923203E8,3.8537518E2,-7.1715E-3,-1.4275366E-2,2.28E4,-7.2465115E-3,2.866353E2,2.0549193E2,1E1,1.4386049E6,-9.179282E-3,-2.7952767E-3,1.6341463E0,-5.434599E-3,8.201515E5,3.5828743E-3,4.0351807E2,4.519E3,-6.3834814E-3,7.5E-1,-1.09247165E-2,-2.5168979E-3,1.0003492E7,2.978142E0,2.0261577E2,3.9939122E5,-8.509088E-3,8.931E3,5.828922E-4,6.093556E-3,1.4872598E8,5.5177975E6,3.72381E5,1E1,2.5367088E2,9.525663E2,1.752E3,1.3439045E-2,-2.1326744E-3,1.8007722E-3,1E0,9.746061E-3,4.294737E1,2.0176284E-3,2.8387096E0,9.327427E-4,-6.6219675E-3,-1.6317331E-3,-1.1597686E-2,-8.2730426E-4,-3.2230516E-4,4.5167725E-3,-3.4073547E-3,4.7528482E-7,1.1356301E-3,-1.9241748E-3,4.5E1,8.82E2,1.5321098E8,-8.077768E-4,-4.180844E-3,1.6643229E1,-1.5501073E-3,1.0083345E-3,4.472581E-3,-1.0474909E-3,-3.4120228E-4,1.09924164E5,-3.4949712E-3,-1.029082E-2,6.1986052E7,2.593592E6,4.9963706E5,1.5822886E0,1.1481482E0,2.1E1,-1.2253443E-3,4.923842E-3,2.323397E-3,-9.120735E-3,-3.668458E-3,5.388794E6,-5.8822064E-3,-1.5961746E-3,7.0285716E0,1.3548806E-2,1.9197954E-3,5.775997E-3,-6.1476664E-3,-3.3300633E-3,-7.5694895E-4,-5.4485486E-3,-1.4037599E-3,1.0096745E-3,6.1842E-3,1.0308623E-3,4.1973614E-3,1.0633453E-3,2.3505064E-3,-1.4537458E-3,-5.8272686E-3,-9.499235E-4,-3.1516328E-4,5.358671E-3,-3.3846407E-3,1.4188082E-3,5.24261E-4,5.2724076E-3,2.1776587E-3,-6.70844E-3,-4.4813997E-4,3.811271E-3,7.7153766E-3,1.392189E-3,1.4253506E-3,-2.3414674E-3,1.0679259E-2,5.7965866E-3],"split_indices":[2,29,2,68,2,60,48,3,9,44,43,73,3,0,73,43,44,67,67,70,0,8,0,1,1,0,0,53,2,0,62,68,44,0,3,0,0,53,1,30,10,0,0,44,69,0,71,0,0,0,0,1,0,8,69,0,0,0,48,62,60,71,71,0,70,8,0,6,43,49,2,7,0,0,60,52,0,67,9,0,69,48,0,0,0,1,71,69,67,62,8,27,0,46,71,3,0,2,0,47,71,0,0,1,0,67,4,3,43,0,0,68,0,60,0,4,44,0,68,0,0,60,69,70,60,0,44,0,0,12,60,9,8,67,4,0,0,0,0,29,0,62,0,71,0,0,0,0,0,0,0,0,0,0,0,0,2,7,0,0,73,0,0,0,0,0,66,0,0,5,9,43,57,68,3,0,0,0,0,0,62,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.035E3,6.67E2,3.68E2,2.43E2,4.24E2,2.52E2,1.16E2,2.16E2,2.7E1,1.03E2,3.21E2,2.38E2,1.4E1,7E0,1.09E2,1.3E1,2.03E2,1.5E1,1.2E1,9.7E1,6E0,3.15E2,6E0,1.7E1,2.21E2,7E0,7E0,1E2,9E0,4E0,9E0,8.3E1,1.2E2,5E0,1E1,8E0,4E0,4.6E1,5.1E1,2.14E2,1.01E2,1.1E1,6E0,2.7E1,1.94E2,7E0,9.3E1,5E0,4E0,5E0,4E0,1.7E1,6.6E1,8.2E1,3.8E1,5E0,5E0,7E0,3.9E1,4E1,1.1E1,1.59E2,5.5E1,7E0,9.4E1,2.3E1,4E0,1.28E2,6.6E1,5.6E1,3.7E1,1.3E1,4E0,5E0,6.1E1,7.4E1,8E0,2E1,1.8E1,1.2E1,2.7E1,3.4E1,6E0,6E0,5E0,1.18E2,4.1E1,1.3E1,4.2E1,2.5E1,6.9E1,1.6E1,7E0,1.08E2,2E1,5.6E1,1E1,5.1E1,5E0,1E1,2.7E1,9E0,4E0,5.1E1,1E1,5.3E1,2.1E1,8E0,1.2E1,1.3E1,5E0,1.4E1,1.3E1,3E1,4E0,5.1E1,6.7E1,4E0,3.7E1,4E0,9E0,2.7E1,1.5E1,1.4E1,1.1E1,4E0,6.5E1,9E0,7E0,5.8E1,5E1,1.1E1,9E0,4.7E1,9E0,4.4E1,7E0,6E0,4E0,1.8E1,9E0,4.7E1,4E0,4.2E1,1.1E1,9E0,1.2E1,4E0,4E0,5E0,7E0,1E1,4E0,1.3E1,1.7E1,4.2E1,9E0,5.8E1,9E0,4E0,3.3E1,1.9E1,8E0,1.1E1,4E0,5E0,9E0,5E0,6E0,3.6E1,2.9E1,4.3E1,1.5E1,1.5E1,3.5E1,4E0,7E0,4E0,5E0,5E0,4.2E1,5E0,4E0,4E1,4E0,7E0,1.1E1,1.4E1,3.3E1,3.4E1,8E0,2.1E1,2.1E1,4E0,5E0,4.3E1,1.5E1,1.9E1,1.4E1,5E0,4E0,2.8E1,8E0,2.3E1,6E0,1E1,3.3E1,8E0,7E0,5E0,1E1,3E1,5E0,3.6E1,6E0,6E0,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-2.246716E-3,-7.778354E-2,2.6095033E-2,-6.0222115E-2,-2.0953187E-1,-1.3702271E-3,8.0813855E-2,-7.349078E-2,4.4343524E-2,-3.8790664E-1,-1.0271237E-1,1.966771E-3,-9.652232E-3,1.6765498E-2,7.482498E-2,-6.616697E-2,-1.587768E-1,1.0149643E-2,-3.2918174E-2,-2.2362106E-2,-7.230836E-3,-1.4324442E-1,2.1276942E-4,-5.9582265E-3,7.996801E-2,6.428155E-2,1.7563073E-1,-9.900151E-2,-4.2671546E-2,-1.3761507E-2,-9.30642E-2,2.6838355E-2,-4.6815434E-3,-1.2896681E-3,-1.793611E-1,-6.2180858E-2,2.1290486E-3,1.01124294E-1,-2.418476E-3,6.837701E-2,-6.06279E-3,1.0513901E-2,1.12644E-3,-8.052767E-2,-2.0621286E-1,6.570158E-3,-5.5054422E-2,-5.5885073E-3,-1.1193283E-3,-2.6713351E-3,5.596371E-3,-1.2341322E-2,-4.7161425E-3,-5.3587213E-2,-6.50655E-3,5.279459E-3,-8.045472E-3,6.594445E-2,1.4484148E-1,7.723972E-2,-3.0845776E-2,-9.442909E-2,3.2384848E-4,-1.3958449E-2,-4.3673394E-3,-4.132916E-2,-1.2295029E-1,-6.0812492E-2,1.2907062E-3,9.902494E-3,-9.700982E-2,9.194517E-2,5.7258084E-4,7.790057E-3,2.4970826E-3,6.985779E-2,1.8089138E-1,2.043353E-2,-8.247202E-3,-2.6567988E-2,-1.09578416E-1,-2.1340686E-3,1.85485E-3,-8.3295904E-2,-2.5017614E-2,-6.638835E-3,-1.6682023E-3,-9.239915E-4,-7.999419E-2,1.4523398E-2,-5.5371094E-2,-1.5578517E-2,2.8349573E-3,2.590712E-3,6.577774E-3,3.1344898E-2,8.6996354E-2,1.0408034E-2,2.9937655E-3,8.240165E-2,-2.8716796E-3,-3.4051596E-3,1.8808118E-3,-1.2041012E-1,-4.4448044E-2,-7.4528984E-4,-4.51952E-3,-4.939115E-2,3.6760774E-3,-6.321094E-3,-5.6336038E-2,7.0906878E-3,6.129466E-2,9.6387195E-4,-8.617695E-2,-2.5871943E-3,1.6265125E-3,-3.3372953E-3,3.9881147E-2,1.7698576E-1,6.657143E-2,-6.3252065E-4,7.4849008E-3,-3.7093188E-3,-6.952874E-3,9.4599277E-4,-4.642216E-3,1.5494056E-3,-2.7639477E-3,-2.230725E-3,1.4149114E-3,1.1848741E-3,-3.3805314E-3,1.6459491E-3,-2.0856343E-4,4.851957E-3,1.080785E-3,-5.902269E-3,-8.4228953E-4,5.3548994E-3,8.861318E-4,8.991867E-3,4.2320495E-3,4.317499E-3,-2.3936133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,31,-1,-1,33,-1,35,37,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,-1,59,-1,-1,-1,61,63,-1,65,-1,-1,-1,-1,-1,-1,67,-1,69,-1,71,73,75,77,79,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,95,97,99,-1,101,103,-1,-1,105,107,-1,-1,-1,109,111,113,-1,115,-1,-1,117,119,-1,-1,121,-1,-1,-1,123,125,-1,-1,127,129,-1,131,133,135,-1,137,-1,-1,-1,139,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2285092E0,6.444689E-1,1.1377809E0,3.528955E-1,5.88936E-1,3.5018238E-1,4.002546E-1,1.2940884E-1,4.0011975E-1,1.911695E-1,9.635559E-2,3.0832E-1,0E0,0E0,2.533568E-1,1.5623945E-1,1.2766632E-1,0E0,8.857402E-2,0E0,0E0,6.163782E-2,0E0,2.06106E-1,1.3406324E-1,1.9073576E-1,1.7516172E-1,1.5372622E-1,2.8942782E-1,0E0,2.203741E-2,1.0280403E-1,0E0,0E0,5.4124832E-2,3.0981809E-2,2.213914E-1,5.1967084E-2,0E0,1.984849E-1,0E0,0E0,0E0,8.432683E-2,9.999162E-2,0E0,1.0301763E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2918215E-2,0E0,1.8585946E-1,0E0,3.2277003E-2,2.7586013E-2,1.442188E-1,1.4752237E-1,6.288397E-2,2.4085097E-2,0E0,0E0,6.5001294E-2,2.904895E-2,3.6759675E-2,0E0,1.139128E-1,4.0865272E-1,2.0959139E-2,0E0,0E0,0E0,1.2466395E-1,5.2401185E-2,8.189341E-2,0E0,4.4229094E-2,3.1332552E-2,0E0,0E0,2.4773046E-2,5.0535973E-2,0E0,0E0,0E0,3.6659285E-2,1.2158606E-1,6.0509697E-2,0E0,2.6731653E-2,0E0,0E0,5.6817323E-2,2.3267531E-1,0E0,0E0,7.106991E-2,0E0,0E0,0E0,4.1030645E-2,3.479987E-2,0E0,0E0,3.5758115E-2,4.9091406E-2,0E0,3.6385737E-2,1.0083707E-1,7.70946E-2,0E0,4.8524186E-2,0E0,0E0,0E0,8.7484166E-2,1.833278E-2,3.549119E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,34,34,35,35,36,36,37,37,39,39,43,43,44,44,46,46,53,53,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,69,69,70,70,71,71,75,75,76,76,77,77,79,79,80,80,83,83,84,84,88,88,89,89,90,90,92,92,95,95,96,96,99,99,103,103,104,104,107,107,108,108,110,110,111,111,112,112,114,114,118,118,119,119,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,32,-1,-1,34,-1,36,38,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,-1,60,-1,-1,-1,62,64,-1,66,-1,-1,-1,-1,-1,-1,68,-1,70,-1,72,74,76,78,80,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,96,98,100,-1,102,104,-1,-1,106,108,-1,-1,-1,110,112,114,-1,116,-1,-1,118,120,-1,-1,122,-1,-1,-1,124,126,-1,-1,128,130,-1,132,134,136,-1,138,-1,-1,-1,140,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0162934E5,4.6463413E0,5.862504E2,2.83E3,2E0,1E0,1.4E1,2.8025E4,1.11791E5,1.2E1,8.499432E5,4.5866325E6,-9.652232E-3,1.6765498E-2,6.663214E6,2.71E2,6.15E2,1.0149643E-2,4.36853E5,-2.2362106E-2,-7.230836E-3,1.00011E5,2.1276942E-4,4.33E2,1.9E1,5.619497E7,1.213274E9,6.769866E7,2.00087E5,-1.3761507E-2,2.2E1,2.35184E5,-4.6815434E-3,-1.2896681E-3,8.2E1,3.310366E4,5.6E1,3.7788504E7,-2.418476E-3,7.34E0,-6.06279E-3,1.0513901E-2,1.12644E-3,1.9E1,1.76421E6,6.570158E-3,3.2856784E7,-5.5885073E-3,-1.1193283E-3,-2.6713351E-3,5.596371E-3,-1.2341322E-2,-4.7161425E-3,5.30577E-1,-6.50655E-3,4.75356E6,-8.045472E-3,7.6116E4,1.3922E4,4.907764E3,6.70019E1,1.957E3,7.701384E3,-1.3958449E-2,-4.3673394E-3,1.8401923E2,1.7443357E8,2.57E2,1.2907062E-3,3.0084E4,2.1512408E5,2.3043478E2,5.7258084E-4,7.790057E-3,2.4970826E-3,8.1407714E-1,3.2295492E0,2.7664675E5,-8.247202E-3,3.4E1,2.7546012E0,-2.1340686E-3,1.85485E-3,3.1E1,1.453785E6,-6.638835E-3,-1.6682023E-3,-9.239915E-4,3.01E2,1.812513E8,1.5215946E0,-1.5578517E-2,8.599521E5,2.590712E-3,6.577774E-3,1.0775862E0,3.3382E4,1.0408034E-2,2.9937655E-3,3E0,-2.8716796E-3,-3.4051596E-3,1.8808118E-3,1.5661134E0,7.9634375E2,-7.4528984E-4,-4.51952E-3,2.019537E2,1.5152774E2,-6.321094E-3,1.5310282E8,1.3364486E0,1.2E1,9.6387195E-4,1.3092E4,-2.5871943E-3,1.6265125E-3,-3.3372953E-3,2E0,8.7E1,2.9366477E0,-6.3252065E-4,7.4849008E-3,-3.7093188E-3,-6.952874E-3,9.4599277E-4,-4.642216E-3,1.5494056E-3,-2.7639477E-3,-2.230725E-3,1.4149114E-3,1.1848741E-3,-3.3805314E-3,1.6459491E-3,-2.0856343E-4,4.851957E-3,1.080785E-3,-5.902269E-3,-8.4228953E-4,5.3548994E-3,8.861318E-4,8.991867E-3,4.2320495E-3,4.317499E-3,-2.3936133E-3],"split_indices":[43,68,67,2,32,106,0,44,1,8,60,43,0,0,43,2,2,0,1,0,0,5,0,2,8,62,7,7,5,0,71,1,0,0,0,48,3,7,0,61,0,0,0,3,46,0,60,0,0,0,0,0,0,53,0,62,0,12,44,4,73,9,43,0,0,70,7,44,0,44,48,4,0,0,0,53,53,48,0,2,68,0,0,0,9,0,0,0,44,5,69,0,43,0,0,68,9,0,0,8,0,0,0,68,48,0,0,4,67,0,46,68,3,0,10,0,0,0,32,8,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,2.83E2,7.56E2,2.51E2,3.2E1,5.04E2,2.52E2,2.23E2,2.8E1,1.1E1,2.1E1,4.97E2,7E0,4E0,2.48E2,2.07E2,1.6E1,8E0,2E1,7E0,4E0,1.5E1,6E0,4.52E2,4.5E1,2.26E2,2.2E1,8.5E1,1.22E2,4E0,1.2E1,1.1E1,9E0,4E0,1.1E1,5.6E1,3.96E2,3.9E1,6E0,2.22E2,4E0,1.6E1,6E0,7.4E1,1.1E1,7E0,1.15E2,8E0,4E0,6E0,5E0,4E0,7E0,5.2E1,4E0,3.9E2,6E0,2.3E1,1.6E1,2.04E2,1.8E1,6.3E1,1.1E1,5E0,6E0,9.7E1,1.8E1,4.8E1,4E0,3.74E2,1.6E1,1.5E1,8E0,1.2E1,4E0,1.92E2,1.2E1,1.4E1,4E0,1.2E1,5.1E1,5E0,6E0,2.6E1,7.1E1,1.4E1,4E0,1.6E1,3.2E1,3.5E2,2.4E1,4E0,1.2E1,1E1,5E0,6E1,1.32E2,8E0,4E0,8E0,6E0,7E0,5E0,4.3E1,8E0,5E0,2.1E1,3.8E1,3.3E1,8E0,2.4E1,3.03E2,4.7E1,7E0,1.7E1,4E0,8E0,4E0,5.6E1,2.3E1,1.09E2,4E0,4E0,2E1,2.3E1,4E0,4E0,4E0,3.4E1,1.1E1,2.2E1,4E0,2E1,8.7E1,2.16E2,2.1E1,2.6E1,1E1,7E0,1.1E1,4.5E1,1.8E1,5E0,8.9E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[-2.567656E-3,-5.573658E-2,3.8479526E-2,-4.8533235E-2,-2.565958E-1,2.383949E-2,1.2916835E-1,-8.2375884E-2,-2.506622E-2,-1.01484045E-1,-2.524882E-2,3.0910918E-2,-1.4459175E-1,1.7562997E-1,2.1786032E-2,-7.585837E-2,-1.6404082E-1,7.0511973E-3,-3.0742949E-2,5.20484E-4,-9.116005E-3,1.9763673E-2,1.1887469E-1,-1.1419385E-2,-4.3389145E-3,1.33511E-1,3.1543034E-1,-2.4449607E-2,4.9294084E-3,-1.03909306E-1,-5.7610177E-2,-1.1580421E-2,-4.130033E-3,-2.3796668E-2,-1.12480275E-1,9.005744E-3,1.6761377E-2,-1.0729339E-3,1.3416049E-1,1.5866417E-1,-1.0213535E-3,1.8529927E-2,7.5269165E-3,-5.9708287E-3,4.4458758E-2,-8.619171E-2,-1.0779947E-2,-8.496841E-3,-7.923898E-2,-2.7775029E-2,4.8866454E-3,-8.242472E-3,-5.9152335E-2,2.1102395E-2,-8.611082E-2,7.2912045E-2,1.5482672E-1,1.2456751E-2,1.09403E-1,4.7205263E-3,-9.54947E-4,-2.3603722E-2,-9.9006094E-2,-5.842033E-2,1.3578452E-2,-8.600494E-2,3.3565337E-4,-4.5528298E-4,-4.440114E-2,-4.649631E-3,-4.0993546E-5,-2.8081415E-2,2.806836E-2,-2.7918817E-2,-8.692029E-3,5.204056E-3,-9.764041E-4,2.9035744E-3,7.7642733E-3,1.5350669E-3,1.2866788E-1,-2.9587694E-3,1.743244E-3,-1.1882344E-1,-2.7939728E-3,-4.856889E-3,4.3033942E-4,3.2742673E-3,-2.8316148E-2,-6.6414364E-3,-7.1578465E-2,1.208197E-2,-5.976793E-2,-3.6334645E-2,-1.1628438E-2,3.4833124E-3,-8.955902E-2,3.29158E-2,-3.3604197E-2,-4.1238335E-3,3.502559E-3,1.5145591E-1,9.654395E-4,-8.666436E-2,-1.5984003E-1,6.828456E-4,-2.4901757E-3,-5.582565E-2,-6.2511154E-3,-3.733035E-2,3.219132E-2,-6.5755304E-3,1.3982398E-3,-2.4311658E-2,-9.987207E-2,-3.6536732E-3,2.0446392E-2,-6.3733445E-3,-2.338618E-3,1.0011721E-2,5.3484317E-2,-5.435336E-3,-1.2975327E-2,7.96133E-3,2.8979066E-3,-1.6586041E-3,-4.958629E-3,-9.009088E-3,-3.3081567E-3,1.12197835E-4,-3.0778085E-3,-5.7873293E-3,-9.332417E-5,2.0442274E-3,-1.7622969E-3,4.8791593E-5,-2.178759E-3,-7.192644E-3,-1.6158909E-3,3.5146428E-3,-2.0182246E-4,1.2474451E-3,-2.26424E-3,9.338563E-4,3.910048E-3,2.6906333E-3,-1.7253913E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,-1,-1,35,37,-1,-1,39,41,43,-1,45,47,-1,-1,49,51,-1,53,-1,55,57,-1,-1,-1,-1,59,61,-1,63,65,67,-1,-1,69,71,73,75,77,-1,79,-1,-1,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,-1,-1,-1,-1,-1,-1,101,-1,-1,103,-1,-1,-1,-1,105,-1,107,109,111,113,-1,115,117,119,121,-1,-1,123,-1,125,127,-1,-1,129,-1,131,133,-1,-1,135,137,-1,139,-1,-1,141,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.204221E0,6.241642E-1,7.537026E-1,3.362584E-1,6.191022E-1,5.9046566E-1,3.9031708E-1,8.016801E-2,2.5989658E-1,1.21084E-1,0E0,4.6145254E-1,8.075446E-2,2.817402E-1,1.0075929E-1,7.745004E-2,5.5182368E-2,0E0,1.3699488E-1,0E0,0E0,2.1814148E-1,1.1823678E-1,0E0,0E0,1.7573696E-1,9.1415405E-2,1.2850341E-1,0E0,1.2435657E-1,1.06615275E-1,0E0,0E0,1.204668E-1,5.625072E-2,0E0,1.8714371E-1,0E0,4.9468398E-2,1.782251E-1,0E0,0E0,0E0,0E0,4.3463226E-2,4.337862E-2,0E0,3.6059678E-2,4.1594774E-2,1.00816876E-1,0E0,0E0,2.892045E-2,1.3749069E-1,9.568407E-2,5.4694273E-2,3.2042444E-2,0E0,3.6815196E-2,0E0,0E0,2.9701915E-2,2.870065E-2,3.2617927E-2,5.7214536E-2,4.3865412E-2,0E0,6.392962E-2,2.2161794E-1,0E0,0E0,9.7338155E-2,1.0574001E-1,8.312775E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.086428E-2,0E0,0E0,2.5719225E-2,0E0,0E0,0E0,0E0,1.6233398E-2,0E0,4.6658218E-2,7.175084E-2,1.1580853E-1,9.9935815E-2,0E0,4.9238116E-2,2.5052026E-2,1.5257299E-1,4.355167E-2,0E0,0E0,2.252385E-2,0E0,1.6080424E-2,2.3269981E-2,0E0,0E0,2.8460115E-2,0E0,6.4955235E-2,4.49384E-2,0E0,0E0,6.612628E-2,7.0620835E-2,0E0,4.036531E-2,0E0,0E0,1.567538E-1,1.7551258E-1,0E0,4.0358942E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,25,25,26,26,27,27,29,29,30,30,33,33,34,34,36,36,38,38,39,39,44,44,45,45,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,64,64,65,65,67,67,68,68,71,71,72,72,73,73,80,80,83,83,88,88,90,90,91,91,92,92,93,93,95,95,96,96,97,97,98,98,101,101,103,103,104,104,107,107,109,109,110,110,113,113,114,114,116,116,119,119,120,120,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,-1,-1,36,38,-1,-1,40,42,44,-1,46,48,-1,-1,50,52,-1,54,-1,56,58,-1,-1,-1,-1,60,62,-1,64,66,68,-1,-1,70,72,74,76,78,-1,80,-1,-1,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,-1,-1,-1,-1,-1,-1,102,-1,-1,104,-1,-1,-1,-1,106,-1,108,110,112,114,-1,116,118,120,122,-1,-1,124,-1,126,128,-1,-1,130,-1,132,134,-1,-1,136,138,-1,140,-1,-1,142,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.31E2,6.0692043E0,1.6466942E3,4.13E2,1.6493898E6,4.217427E7,2.2040408E7,3.578882E1,1.2E1,4.6683E7,-2.524882E-2,4.7429975E6,1.0479E4,3.8771296E3,2.6108465E0,3.9125E1,3.3817584E7,7.0511973E-3,8.7E1,5.20484E-4,-9.116005E-3,2.04115E5,1.0378262E10,-1.1419385E-2,-4.3389145E-3,1E1,1.159499E6,1.3655363E2,4.9294084E-3,2.161111E1,4.577342E0,-1.1580421E-2,-4.130033E-3,3.5E1,1.31306E5,9.005744E-3,1.2294118E1,-1.0729339E-3,1E0,1.6386554E0,-1.0213535E-3,1.8529927E-2,7.5269165E-3,-5.9708287E-3,2.5130852E2,1.957E3,-1.0779947E-2,5.684E3,3.95E2,7.9227E4,4.8866454E-3,-8.242472E-3,6.354E3,1.8439855E5,3.3924444E5,3.82E2,1.28738E5,1.2456751E-2,4.1032645E6,4.7205263E-3,-9.54947E-4,2.125E1,3.9E1,1.8187732E4,1.2E1,3.3817584E7,3.3565337E-4,1.1E1,2.8302418E10,-4.649631E-3,-4.0993546E-5,3.63E2,1.6673345E7,5.0563266E2,-8.692029E-3,5.204056E-3,-9.764041E-4,2.9035744E-3,7.7642733E-3,1.5350669E-3,1.0905187E9,-2.9587694E-3,1.743244E-3,2.6412E5,-2.7939728E-3,-4.856889E-3,4.3033942E-4,3.2742673E-3,2.3412812E0,-6.6414364E-3,1.5E1,7.5793734E8,1.994E3,9.07826E0,-1.1628438E-2,5.1E1,1.6E1,1.862E3,4.48375E5,-4.1238335E-3,3.502559E-3,1.2792593E2,9.654395E-4,2.88E0,4.631579E0,6.828456E-4,-2.4901757E-3,2.0563E4,-6.2511154E-3,3.3817584E7,1E0,-6.5755304E-3,1.3982398E-3,1E0,1.95E2,-3.6536732E-3,1.5853742E1,-6.3733445E-3,-2.338618E-3,8.009E3,5.2692295E6,-5.435336E-3,5.003246E1,7.96133E-3,2.8979066E-3,-1.6586041E-3,-4.958629E-3,-9.009088E-3,-3.3081567E-3,1.12197835E-4,-3.0778085E-3,-5.7873293E-3,-9.332417E-5,2.0442274E-3,-1.7622969E-3,4.8791593E-5,-2.178759E-3,-7.192644E-3,-1.6158909E-3,3.5146428E-3,-2.0182246E-4,1.2474451E-3,-2.26424E-3,9.338563E-4,3.910048E-3,2.6906333E-3,-1.7253913E-3],"split_indices":[2,68,67,2,60,60,66,61,10,7,0,43,9,67,50,4,7,0,8,0,0,5,46,0,0,6,9,71,0,67,73,0,0,3,9,0,68,0,27,69,0,0,0,0,73,9,0,9,2,9,0,0,10,43,43,0,1,0,60,0,0,4,2,43,33,7,0,8,46,0,0,0,47,67,0,0,0,0,0,0,7,0,0,7,0,0,0,0,68,0,8,46,12,71,0,0,3,2,1,0,0,73,0,69,69,0,0,12,0,7,19,0,0,26,10,0,73,0,0,44,60,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,4.39E2,5.69E2,4.25E2,1.4E1,4.91E2,7.8E1,1.73E2,2.52E2,1E1,4E0,4.72E2,1.9E1,5.4E1,2.4E1,1.62E2,1.1E1,7E0,2.45E2,5E0,5E0,4.2E2,5.2E1,5E0,1.4E1,4.3E1,1.1E1,1.6E1,8E0,6.2E1,1E2,4E0,7E0,2.27E2,1.8E1,6E0,4.14E2,5E0,4.7E1,3.7E1,6E0,6E0,5E0,6E0,1E1,5.6E1,6E0,3.1E1,6.9E1,2.21E2,6E0,7E0,1.1E1,3.98E2,1.6E1,1.3E1,3.4E1,1E1,2.7E1,5E0,5E0,1E1,4.6E1,9E0,2.2E1,6.4E1,5E0,8.4E1,1.37E2,6E0,5E0,4.9E1,3.49E2,1.1E1,5E0,9E0,4E0,5E0,2.9E1,6E0,2.1E1,6E0,4E0,2.9E1,1.7E1,5E0,4E0,9E0,1.3E1,1.1E1,5.3E1,7E1,1.4E1,1.33E2,4E0,3.3E1,1.6E1,3.24E2,2.5E1,7E0,4E0,1.7E1,4E0,1.8E1,1.1E1,5E0,8E0,4.4E1,9E0,2E1,5E1,7E0,7E0,1.13E2,2E1,5E0,2.8E1,6E0,1E1,1.54E2,1.7E2,4E0,2.1E1,1.3E1,4E0,6E0,1.2E1,7E0,4E0,7E0,3.7E1,5E0,1.5E1,4.3E1,7E0,5.4E1,5.9E1,1E1,1E1,8E0,2E1,1.2E2,3.4E1,8.3E1,8.7E1,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[3.7052329E-3,-1.3392239E-2,1.1293301E-1,-3.8338415E-2,2.3857282E-2,1.631091E-2,9.9645585E-2,-7.9138584E-2,-1.8889999E-2,1.0220474E-2,8.719398E-2,8.8257894E-2,1.1515891E-2,-6.900184E-2,-1.884737E-2,-1.07211374E-1,-7.146279E-3,-5.1901132E-2,3.0515952E-2,7.0185095E-2,1.627654E-1,-3.3053036E-3,9.6773274E-2,-6.064249E-2,-9.086762E-3,-7.579222E-3,-7.711027E-2,1.5004916E-1,-1.685171E-2,-1.7882802E-2,-1.104559E-1,4.8900127E-2,2.192085E-3,5.4561675E-2,6.166278E-3,9.617119E-3,3.0791692E-3,1.1367391E-1,3.3723872E-2,-6.82102E-2,5.340218E-3,-3.0035583E-2,-1.4521934E-1,3.5626607E-3,8.600763E-3,-8.644546E-3,-9.2343025E-2,-8.9038186E-2,3.185411E-3,-1.1556455E-2,-5.86331E-2,5.569412E-2,-6.5417853E-3,-7.152324E-2,2.7092975E-2,1.15076E-3,7.5273104E-2,1.3409376E-1,5.43253E-2,2.963826E-4,5.224055E-3,-4.7544133E-2,-1.0290723E-1,-7.5654134E-2,1.858653E-2,-1.0400868E-2,-3.5629508E-3,1.4883763E-3,-7.268112E-2,3.9379247E-3,-1.2294161E-1,-2.9599204E-4,-8.635267E-3,-5.6373373E-2,3.2842286E-2,-7.337421E-3,2.8057268E-3,4.40668E-2,1.69264E-1,1.3119429E-2,-1.071209E-2,4.7601566E-2,-3.7232894E-2,5.6640417E-3,1.9868102E-3,1.4637332E-1,1.2322719E-4,7.3623896E-2,7.856037E-5,1.1585838E-3,-3.6526981E-3,-8.506636E-2,-2.4752598E-2,-7.048921E-3,-5.9334535E-2,-5.853912E-3,-9.4698905E-4,3.216998E-3,-1.2496985E-3,-2.4807671E-2,1.7891357E-2,-9.8883994E-2,2.6558195E-3,-1.5565075E-1,2.1051473E-3,-8.9713087E-4,-4.4466844E-3,3.2608781E-3,4.6004853E-4,-2.9323524E-2,3.0723794E-3,1.0482689E-1,3.0395623E-2,9.968748E-3,2.8176096E-3,2.7873942E-3,-3.0307194E-2,7.570677E-3,8.164338E-2,-7.4500754E-2,2.0100423E-3,1.2345921E-1,1.1908433E-2,7.607614E-4,4.2900825E-3,3.0274796E-5,-9.848085E-2,-3.4731673E-4,-9.076999E-2,-5.730533E-3,-3.259769E-2,1.2830986E-3,-7.012057E-2,6.547583E-2,6.3389516E-3,-1.2570807E-1,-7.560943E-4,-5.411486E-3,-1.21586835E-2,1.4184571E-3,-1.7781866E-3,7.824094E-4,-3.6483293E-3,1.7556569E-3,1.2224585E-1,5.5783026E-2,3.3848137E-3,9.5232425E-4,-3.4726185E-3,1.0377073E-3,-2.0016988E-3,9.488311E-4,9.4034225E-2,-1.3008444E-3,-6.1791227E-3,1.1894656E-3,1.371726E-1,-7.4322456E-3,-3.5986681E-3,-2.763639E-3,1.4912166E-3,-1.2982892E-3,-5.291629E-3,-2.5039813E-3,1.416145E-4,-7.4322213E-4,1.4791973E-3,4.4492108E-4,-5.0209644E-3,3.8492386E-3,-1.7016523E-3,6.380402E-4,-2.895762E-3,-9.252627E-3,-4.145214E-3,2.6918112E-3,6.654921E-3,1.6192447E-3,7.643047E-3,-3.6111416E-3,9.59977E-4,5.3370306E-3,2.2070282E-3,3.2869226E-3,7.3763076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,35,-1,37,39,-1,-1,41,43,45,47,49,51,53,55,-1,-1,-1,57,59,61,-1,63,65,-1,-1,67,69,71,73,-1,75,77,-1,79,81,-1,83,85,87,89,-1,91,93,95,97,-1,-1,99,101,-1,103,-1,-1,105,107,-1,109,111,113,115,-1,117,119,-1,-1,121,-1,123,-1,-1,-1,125,127,-1,129,-1,-1,-1,-1,131,133,135,-1,137,-1,-1,-1,-1,139,141,-1,143,145,-1,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,157,159,161,-1,163,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,175,177,179,-1,-1,-1,-1,-1,181,-1,-1,-1,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9455996E0,8.381537E-1,4.1469884E-1,4.2699432E-1,3.1177196E-1,0E0,2.0801866E-1,5.5182314E-1,3.7989414E-1,3.7855935E-1,7.0841014E-2,1.7974281E-1,0E0,1.6952473E-1,0E0,6.092608E-2,4.9836463E-1,1.4542796E-1,1.17573485E-1,4.611045E-2,3.4127384E-2,0E0,1.2510109E-1,2.2260833E-1,0E0,0E0,9.1824666E-2,3.569433E-2,1.8940955E-1,7.224123E-2,1.8078822E-1,1.8519932E-1,1.6713972E-1,2.5761992E-2,0E0,0E0,0E0,1.0784495E-1,7.158613E-2,1.06250286E-1,0E0,4.3549493E-2,4.793948E-2,0E0,0E0,1.8049617E-1,1.7349139E-1,8.776646E-2,6.897188E-2,0E0,1.2971076E-1,1.6816932E-1,0E0,3.1519854E-1,9.1480196E-2,0E0,3.443353E-2,1.125797E-1,2.6280023E-2,4.131667E-2,0E0,8.255811E-2,1.1430943E-1,2.5563471E-2,2.5179796E-2,0E0,0E0,1.04395226E-1,1.3529351E-1,0E0,1.5039843E-1,0E0,0E0,1.7288294E-2,3.1743065E-2,0E0,3.0601783E-2,9.8154455E-2,4.72762E-2,3.4634795E-2,0E0,7.024355E-2,5.439528E-2,0E0,0E0,1.3153076E-1,0E0,1.9361801E-2,0E0,0E0,0E0,4.26113E-2,1.0147237E-1,0E0,5.1317275E-2,0E0,0E0,0E0,0E0,1.10039085E-1,8.14925E-2,6.877366E-2,0E0,6.0289502E-2,0E0,0E0,0E0,0E0,1.8851176E-2,2.4303338E-2,0E0,2.0962834E-2,6.907034E-2,0E0,0E0,0E0,2.2107825E-2,1.9921387E-2,1.92689E-2,2.9183663E-2,0E0,7.016677E-2,0E0,0E0,0E0,0E0,2.8570592E-2,9.3126304E-2,2.2104442E-2,0E0,1.8655686E-2,3.2547813E-2,1.0560502E-1,5.6007534E-2,6.3264325E-2,4.6427578E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5321314E-2,1.12425655E-1,7.254931E-2,0E0,0E0,0E0,0E0,0E0,1.7345369E-2,0E0,0E0,0E0,5.9919596E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,37,37,38,38,39,39,41,41,42,42,45,45,46,46,47,47,48,48,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,68,68,70,70,73,73,74,74,76,76,77,77,78,78,79,79,81,81,82,82,85,85,87,87,91,91,92,92,94,94,99,99,100,100,101,101,103,103,108,108,109,109,111,111,112,112,116,116,117,117,118,118,119,119,121,121,126,126,127,127,128,128,130,130,131,131,132,132,133,133,134,134,135,135,144,144,145,145,146,146,152,152,156,156],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,36,-1,38,40,-1,-1,42,44,46,48,50,52,54,56,-1,-1,-1,58,60,62,-1,64,66,-1,-1,68,70,72,74,-1,76,78,-1,80,82,-1,84,86,88,90,-1,92,94,96,98,-1,-1,100,102,-1,104,-1,-1,106,108,-1,110,112,114,116,-1,118,120,-1,-1,122,-1,124,-1,-1,-1,126,128,-1,130,-1,-1,-1,-1,132,134,136,-1,138,-1,-1,-1,-1,140,142,-1,144,146,-1,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,158,160,162,-1,164,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,176,178,180,-1,-1,-1,-1,-1,182,-1,-1,-1,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,8.995735E5,1.2646534E4,3.9595376E7,4.5866325E6,1.631091E-2,9.059046E6,5E0,2.35E2,2.485E2,6.459696E6,9.9688476E-1,1.1515891E-2,2.707256E7,-1.884737E-2,5.4E1,1.2E1,2.6210527E0,9.113598E6,2.712766E0,4.026E3,-3.3053036E-3,1.0907809E9,5E1,-9.086762E-3,-7.579222E-3,3.14403E5,1.4E1,1.7172844E7,1.241995E2,2.0671213E2,1.853776E7,1.0909091E0,2.8387096E0,6.166278E-3,9.617119E-3,3.0791692E-3,8.7E1,4.435876E2,8E0,5.340218E-3,1.0101351E4,8.960784E0,3.5626607E-3,8.600763E-3,2.2715E4,1.2E1,1.2131707E2,9.971819E8,-1.1556455E-2,9.582126E7,8.118189E6,-6.5417853E-3,2.41E3,6.346204E-7,1.15076E-3,1E0,4.6E1,5.712652E8,3.5056704E7,5.224055E-3,1.16408E5,4.9948E4,4.9E2,1.26E2,-1.0400868E-2,-3.5629508E-3,7.69E2,9.0754684E1,3.9379247E-3,2.3E1,-2.9599204E-4,-8.635267E-3,1.68991E6,1.7676923E2,-7.337421E-3,1.7E1,1.1149426E0,1E0,1.2E1,-1.071209E-2,1.157602E6,1.056178E6,5.6640417E-3,1.9868102E-3,1.3439851E7,1.2322719E-4,2.8646374E8,7.856037E-5,1.1585838E-3,-3.6526981E-3,1.11E2,1E0,-7.048921E-3,4.312981E5,-5.853912E-3,-9.4698905E-4,3.216998E-3,-1.2496985E-3,7.547528E6,2.23607E5,1E0,2.6558195E-3,7E0,2.1051473E-3,-8.9713087E-4,-4.4466844E-3,3.2608781E-3,9.695652E0,2.7005264E2,3.0723794E-3,1.1090909E1,1.676E3,9.968748E-3,2.8176096E-3,2.7873942E-3,1.3809524E0,2.6699028E0,1.4956522E1,7E0,2.0100423E-3,7.785302E5,1.1908433E-2,7.607614E-4,4.2900825E-3,3.0274796E-5,1E1,6.44E2,1.5365228E5,-5.730533E-3,8.315271E0,1.6507974E8,1.625E0,2E1,4.9469455E6,4.080551E-1,-7.560943E-4,-5.411486E-3,-1.21586835E-2,1.4184571E-3,-1.7781866E-3,7.824094E-4,-3.6483293E-3,1.7556569E-3,1.142E3,2.364E3,4.646154E0,9.5232425E-4,-3.4726185E-3,1.0377073E-3,-2.0016988E-3,9.488311E-4,2.295413E6,-1.3008444E-3,-6.1791227E-3,1.1894656E-3,1.2697166E3,-7.4322456E-3,-3.5986681E-3,-2.763639E-3,1.4912166E-3,-1.2982892E-3,-5.291629E-3,-2.5039813E-3,1.416145E-4,-7.4322213E-4,1.4791973E-3,4.4492108E-4,-5.0209644E-3,3.8492386E-3,-1.7016523E-3,6.380402E-4,-2.895762E-3,-9.252627E-3,-4.145214E-3,2.6918112E-3,6.654921E-3,1.6192447E-3,7.643047E-3,-3.6111416E-3,9.59977E-4,5.3370306E-3,2.2070282E-3,3.2869226E-3,7.3763076E-3],"split_indices":[2,43,48,7,43,0,43,6,2,67,9,69,0,60,0,2,33,71,60,68,2,0,7,3,0,0,9,3,60,67,4,62,68,71,0,0,0,8,71,32,0,43,73,0,0,44,3,70,46,0,5,60,0,2,52,0,26,6,47,60,0,7,9,1,44,0,0,2,71,0,8,0,0,43,67,0,3,68,19,3,0,12,9,0,0,60,0,12,0,0,0,1,26,0,60,0,0,0,0,5,9,28,0,8,0,0,0,0,73,70,0,73,44,0,0,0,69,68,73,6,0,43,0,0,0,0,3,1,43,0,71,46,68,3,62,53,0,0,0,0,0,0,0,0,44,2,71,0,0,0,0,0,43,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,9E2,1.4E2,5.39E2,3.61E2,6E0,1.34E2,1.73E2,3.66E2,2.98E2,6.3E1,1.26E2,8E0,1.69E2,4E0,4.2E1,3.24E2,7.3E1,2.25E2,5.3E1,1E1,6E0,1.2E2,1.6E2,9E0,1.3E1,2.9E1,1.8E1,3.06E2,4.7E1,2.6E1,1.36E2,8.9E1,4.4E1,9E0,6E0,4E0,9.4E1,2.6E1,1.54E2,6E0,1.8E1,1.1E1,7E0,1.1E1,2.77E2,2.9E1,1E1,3.7E1,6E0,2E1,1.32E2,4E0,2.2E1,6.7E1,1.9E1,2.5E1,6.9E1,2.5E1,1.9E1,7E0,9.8E1,5.6E1,9E0,9E0,4E0,7E0,2.4E2,3.7E1,4E0,2.5E1,6E0,4E0,1.2E1,2.5E1,7E0,1.3E1,1.21E2,1.1E1,1.5E1,7E0,5.1E1,1.6E1,9E0,1.6E1,6.3E1,6E0,1.8E1,7E0,1.5E1,4E0,3.6E1,6.2E1,2.5E1,3.1E1,4E0,5E0,4E0,5E0,9.2E1,1.48E2,3.1E1,6E0,2.1E1,4E0,7E0,5E0,1.1E1,1.4E1,9E0,4E0,2.1E1,1E2,7E0,4E0,7E0,8E0,2.4E1,2.7E1,1.1E1,5E0,5.4E1,9E0,5E0,1.3E1,5E0,3.1E1,4.6E1,1.6E1,8E0,2.3E1,5.9E1,3.3E1,2.8E1,1.2E2,2.3E1,8E0,1.7E1,4E0,8E0,6E0,5E0,4E0,5E0,1.6E1,5.1E1,4.9E1,4E0,4E0,1.9E1,5E0,5E0,2.2E1,7E0,4E0,7E0,4.7E1,6E0,2.5E1,1.6E1,3E1,5E0,1.1E1,1.4E1,9E0,3.8E1,2.1E1,1.1E1,2.2E1,2.4E1,4E0,1.09E2,1.1E1,6E0,1.7E1,5E0,1.1E1,4.4E1,7E0,8E0,4.1E1,1.4E1,8E0,1.3E1,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[-1.6896375E-3,-1.7278235E-2,1.0297271E-1,-4.4850104E-2,1.9811345E-2,1.6026318E-2,8.9046225E-2,-3.1389132E-2,-1.2141681E-1,2.8260003E-3,4.4210114E-2,4.547129E-2,1.4953184E-1,-6.733951E-2,-3.556275E-3,-3.775534E-2,-1.601674E-1,9.572094E-3,-1.3296606E-1,3.524323E-2,1.4289385E-1,2.8366694E-2,1.3575728E-1,1.662894E-1,-2.215249E-3,-1.4110914E-1,-5.950135E-2,1.08520485E-1,-2.2649748E-2,2.023681E-3,-6.4413175E-2,-2.904859E-1,-1.0108153E-1,3.0521434E-3,8.763196E-2,1.1556856E-3,-1.5132353E-2,-7.901481E-2,4.6831153E-2,9.348502E-3,3.716158E-3,4.7347613E-2,-3.2728433E-3,1.16381E-3,8.680404E-3,1.7969016E-3,1.8612354E-1,-1.0776713E-3,-7.996828E-3,3.7161887E-2,-6.4268805E-2,1.9950835E-1,6.2352218E-2,-8.378911E-2,-4.31869E-3,1.7515651E-3,-9.236442E-2,-1.1113867E-1,-2.1064032E-2,1.8901301E-3,-1.23835586E-1,7.6817484E-3,-9.8623894E-2,-6.587486E-4,6.6759977E-3,5.7972938E-2,-1.5577735E-2,-5.913482E-2,6.056969E-2,5.4327203E-3,2.1132376E-2,2.0330381E-1,3.952182E-3,4.524027E-3,-1.4337078E-3,-5.832092E-2,-1.6418974E-1,1.1305392E-2,4.032403E-3,1.1760663E-2,4.3074563E-3,-6.4046737E-3,-4.8607647E-2,1.0349239E-1,-1.2531519E-2,-6.113991E-3,-1.6560567E-3,-8.242859E-3,-9.99243E-4,-1.9602416E-1,-7.325049E-2,-1.1980067E-2,2.502334E-2,-6.9456054E-3,-1.255789E-3,6.4543122E-3,-2.3885854E-3,-2.346451E-2,-6.1406204E-3,7.270009E-2,-1.2208145E-2,-4.8326543E-3,3.712066E-2,3.1616518E-3,1.007249E-2,-7.388208E-2,-3.203063E-2,-2.8101031E-3,-1.0843636E-2,2.15063E-3,-1.1545226E-3,-1.7161662E-2,-7.3208436E-3,-5.3513714E-4,7.5063356E-3,1.9968376E-2,-3.025174E-2,-4.1439617E-3,-1.4096334E-2,-9.6212946E-2,-1.1046136E-3,-1.6949673E-3,-8.9006545E-3,3.688281E-2,-3.0597564E-2,-2.748403E-3,1.7957533E-3,8.150835E-2,-1.9443603E-3,4.0046684E-3,-5.6911085E-2,5.9090937E-3,2.2678224E-2,-5.28683E-2,-1.1962713E-1,3.2593027E-2,-4.6608146E-2,1.9286551E-3,-4.566239E-2,-1.0608669E-2,7.143529E-2,2.7843105E-2,-4.3032102E-2,-5.5495957E-3,-1.3466601E-3,-3.2579906E-2,2.3046885E-2,2.549738E-2,4.60747E-3,1.9420171E-3,-5.276887E-2,7.102301E-2,7.2105103E-3,-9.555584E-2,1.1102412E-3,1.7279412E-3,-2.23772E-2,8.9306E-4,-3.5921354E-3,-7.922401E-3,-4.008574E-3,-1.8745323E-3,4.1880566E-3,-9.312093E-4,-3.3869282E-3,5.7790417E-4,-3.8722993E-3,-1.973576E-3,1.102998E-3,1.9556483E-4,4.2582313E-3,2.9506234E-3,-2.8114258E-3,-3.4112425E-3,-7.950707E-4,1.059894E-3,-2.6423233E-3,-2.0099138E-3,2.0885812E-3,6.782996E-4,4.7667264E-3,-3.8253152E-4,-3.7655951E-3,3.587164E-3,-2.3929263E-3,-1.5564652E-3,-6.389795E-3,-3.2880423E-3,1.4271927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,-1,55,57,59,61,63,-1,-1,65,67,-1,-1,69,-1,-1,-1,-1,71,-1,-1,73,75,77,79,81,83,-1,85,87,-1,-1,89,91,93,-1,-1,95,-1,97,99,-1,101,103,-1,-1,-1,105,107,-1,-1,109,-1,-1,111,113,115,-1,-1,-1,-1,117,119,121,123,-1,-1,-1,-1,125,-1,127,129,-1,131,-1,-1,133,135,-1,-1,-1,-1,137,-1,-1,-1,139,141,-1,-1,143,-1,145,-1,147,149,-1,-1,151,-1,-1,153,-1,155,157,159,161,163,-1,165,167,169,171,173,-1,-1,175,177,179,-1,-1,181,183,-1,185,-1,-1,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7116323E0,9.3454885E-1,4.311037E-1,5.362557E-1,1.6149944E-1,0E0,3.3583617E-1,4.470295E-1,2.4682355E-1,2.1288401E-1,1.3654563E-1,1.1506051E-1,1.866765E-1,1.0435647E-1,5.438614E-1,5.935027E-2,3.7873638E-1,1.122097E-1,3.4633535E-1,1.9965431E-1,3.199938E-2,1.2763725E-1,6.4438E-2,1.190989E-1,0E0,6.377515E-2,8.441746E-2,1.4298806E-1,2.4284074E-1,0E0,6.0704984E-2,4.2410934E-1,1.2784246E-1,9.725519E-2,1.0057299E-1,0E0,0E0,5.1467836E-1,2.0011708E-1,0E0,0E0,1.0064004E-1,0E0,0E0,0E0,0E0,5.6909323E-2,0E0,0E0,4.0332474E-2,9.145838E-2,3.951463E-2,3.9528564E-2,8.9734316E-2,1.500498E-1,0E0,3.140591E-2,5.034226E-2,0E0,0E0,1.061461E-1,6.747031E-2,2.9166162E-2,0E0,0E0,9.686451E-2,0E0,3.8990267E-2,1.06826484E-1,0E0,8.831669E-2,5.6548595E-2,0E0,0E0,0E0,6.3908935E-2,5.133152E-2,0E0,0E0,1.5227805E-2,0E0,0E0,1.0782492E-1,8.333595E-2,9.102121E-2,0E0,0E0,0E0,0E0,1.2343022E-1,2.1492176E-2,1.7249525E-1,7.0390135E-2,0E0,0E0,0E0,0E0,2.9260274E-2,0E0,1.0757071E-1,8.45062E-2,0E0,4.6922967E-2,0E0,0E0,9.076446E-2,5.9926346E-2,0E0,0E0,0E0,0E0,4.5558415E-2,0E0,0E0,0E0,8.888538E-2,7.693039E-2,0E0,0E0,1.8948063E-2,0E0,6.8760276E-2,0E0,5.9710108E-2,3.2815367E-2,0E0,0E0,6.397331E-2,0E0,0E0,4.296054E-2,0E0,2.339222E-2,1.2906748E-1,3.7944794E-2,5.4655436E-2,3.4546264E-2,0E0,4.1000366E-2,4.0881924E-2,2.8873593E-2,6.459721E-2,6.554972E-2,0E0,0E0,5.5977616E-2,7.5711645E-2,6.1495405E-2,0E0,0E0,1.8503688E-2,7.397497E-2,0E0,1.9145362E-2,0E0,0E0,2.5591712E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,37,37,38,38,41,41,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,75,75,76,76,79,79,82,82,83,83,84,84,89,89,90,90,91,91,92,92,97,97,99,99,100,100,102,102,105,105,106,106,111,111,115,115,116,116,119,119,121,121,123,123,124,124,127,127,130,130,132,132,133,133,134,134,135,135,136,136,138,138,139,139,140,140,141,141,142,142,145,145,146,146,147,147,150,150,151,151,153,153,156,156],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,-1,56,58,60,62,64,-1,-1,66,68,-1,-1,70,-1,-1,-1,-1,72,-1,-1,74,76,78,80,82,84,-1,86,88,-1,-1,90,92,94,-1,-1,96,-1,98,100,-1,102,104,-1,-1,-1,106,108,-1,-1,110,-1,-1,112,114,116,-1,-1,-1,-1,118,120,122,124,-1,-1,-1,-1,126,-1,128,130,-1,132,-1,-1,134,136,-1,-1,-1,-1,138,-1,-1,-1,140,142,-1,-1,144,-1,146,-1,148,150,-1,-1,152,-1,-1,154,-1,156,158,160,162,164,-1,166,168,170,172,174,-1,-1,176,178,180,-1,-1,182,184,-1,186,-1,-1,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,7.785302E5,3.3817584E7,2.2E1,4.6134964E2,1.6026318E-2,2.36433E6,6.15E2,2.8266037E2,3.0812828E6,4.9321495E6,4.7263342E2,7.153514E1,7.785714E0,3.5E1,6.123E3,4.5E0,7.82261E6,3.9123364E2,1.4598765E0,1.4615384E0,3.592428E7,1.3E1,8.901939E2,-2.215249E-3,5.3E1,5.445545E-1,7.336111E4,1E0,2.023681E-3,6.5E1,1.08712E5,9.207547E0,6.5590655E6,1E0,1.1556856E-3,-1.5132353E-2,2.7664675E5,4.727453E7,9.348502E-3,3.716158E-3,3.0594551E1,-3.2728433E-3,1.16381E-3,8.680404E-3,1.7969016E-3,2.395631E7,-1.0776713E-3,-7.996828E-3,3.8575E4,5.178788E6,2.4595E5,8.754E3,1.7467743E-2,3.3382E4,1.7515651E-3,2.94E2,3E1,-2.1064032E-2,1.8901301E-3,3.88E2,3.1540035E9,7.9E2,-6.587486E-4,6.6759977E-3,1.65915E5,-1.5577735E-2,2.75E0,3.1530054E0,5.4327203E-3,2.0528813E8,1E0,3.952182E-3,4.524027E-3,-1.4337078E-3,1.6547934E2,2.241E2,1.1305392E-2,4.032403E-3,1.834412E6,4.3074563E-3,-6.4046737E-3,1.606722E8,6.999242E7,1.48E1,-6.113991E-3,-1.6560567E-3,-8.242859E-3,-9.99243E-4,1.7144266E4,1.3050649E4,1.857E3,3.9682608E2,-6.9456054E-3,-1.255789E-3,6.4543122E-3,-2.3885854E-3,1.835821E1,-6.1406204E-3,6.1657715E8,1.925673E1,-4.8326543E-3,5.7E3,3.1616518E-3,1.007249E-2,6.737099E7,3.1096E4,-2.8101031E-3,-1.0843636E-2,2.15063E-3,-1.1545226E-3,1.4915254E0,-7.3208436E-3,-5.3513714E-4,7.5063356E-3,5.0857143E0,4.5683857E-2,-4.1439617E-3,-1.4096334E-2,2E1,-1.1046136E-3,2.485E2,-8.9006545E-3,5.603014E6,4.6712E4,-2.748403E-3,1.7957533E-3,1E0,-1.9443603E-3,4.0046684E-3,3.1E2,5.9090937E-3,1.7291568E2,6.0921145E4,1.76421E6,7.9547346E-4,1.7161617E2,1.9286551E-3,9.75458E5,2.2190831E5,2.16577E5,4.2964826E0,1.3631483E8,-5.5495957E-3,-1.3466601E-3,6.4722223E0,1.1E1,1.3177083E1,4.60747E-3,1.9420171E-3,1.1262003E0,5.526E3,7.2105103E-3,2.427869E8,1.1102412E-3,1.7279412E-3,3.4915986E3,8.9306E-4,-3.5921354E-3,-7.922401E-3,-4.008574E-3,-1.8745323E-3,4.1880566E-3,-9.312093E-4,-3.3869282E-3,5.7790417E-4,-3.8722993E-3,-1.973576E-3,1.102998E-3,1.9556483E-4,4.2582313E-3,2.9506234E-3,-2.8114258E-3,-3.4112425E-3,-7.950707E-4,1.059894E-3,-2.6423233E-3,-2.0099138E-3,2.0885812E-3,6.782996E-4,4.7667264E-3,-3.8253152E-4,-3.7655951E-3,3.587164E-3,-2.3929263E-3,-1.5564652E-3,-6.389795E-3,-3.2880423E-3,1.4271927E-3],"split_indices":[2,43,7,3,4,0,43,2,4,66,43,71,71,67,0,43,73,43,4,69,68,60,3,67,0,44,68,60,29,0,44,7,73,43,29,0,0,48,7,0,0,73,0,0,0,0,62,0,0,9,12,9,1,53,9,0,44,3,0,0,10,46,2,0,0,1,0,68,68,0,7,29,0,0,0,4,4,0,0,5,0,0,12,7,73,0,0,0,0,48,43,44,4,0,0,0,0,71,0,7,73,0,10,0,0,7,9,0,0,0,0,68,0,0,0,69,53,0,0,6,0,67,0,9,1,0,0,23,0,0,8,0,73,60,46,53,67,0,66,62,43,68,7,0,0,73,3,71,0,0,68,2,0,7,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,9.12E2,1.35E2,5.23E2,3.89E2,6E0,1.29E2,4.46E2,7.7E1,2.3E2,1.59E2,7.6E1,5.3E1,1.94E2,2.52E2,2.5E1,5.2E1,2.2E2,1E1,1.47E2,1.2E1,6.5E1,1.1E1,4.9E1,4E0,1.7E1,1.77E2,3.6E1,2.16E2,6E0,1.9E1,1.5E1,3.7E1,2.04E2,1.6E1,6E0,4E0,1.3E1,1.34E2,5E0,7E0,5.5E1,1E1,4E0,7E0,7E0,4.2E1,4E0,1.3E1,8E0,1.69E2,1.1E1,2.5E1,4.9E1,1.67E2,4E0,1.5E1,8E0,7E0,5E0,3.2E1,1.96E2,8E0,6E0,1E1,9E0,4E0,1.5E1,1.19E2,1.4E1,4.1E1,3.5E1,7E0,4E0,4E0,1.61E2,8E0,7E0,4E0,1E1,1.5E1,1.8E1,3.1E1,1.1E1,1.56E2,8E0,7E0,4E0,4E0,1.2E1,2E1,9.2E1,1.04E2,4E0,4E0,5E0,4E0,1.1E1,4E0,1.02E2,1.7E1,4E0,3.7E1,4E0,3.1E1,1E2,6.1E1,4E0,4E0,5E0,5E0,2.5E1,6E0,4E0,7E0,5.5E1,1.01E2,7E0,5E0,1.3E1,7E0,8.8E1,4E0,8.6E1,1.8E1,7E0,4E0,9.5E1,7E0,5E0,1.2E1,4E0,3.3E1,7E1,3E1,1.1E1,5E1,8E0,1.7E1,3.5E1,2E1,1.8E1,8.3E1,9E0,4E0,3.9E1,4.9E1,7.4E1,1.2E1,4E0,1.4E1,8.5E1,1E1,8E0,4E0,2.5E1,8E0,1.8E1,5.2E1,1E1,2E1,5E0,6E0,2.6E1,2.4E1,7E0,1E1,1.8E1,1.7E1,5E0,1.5E1,1.3E1,5E0,3.7E1,4.6E1,1.2E1,2.7E1,1.2E1,3.7E1,6.6E1,8E0,6E0,8E0,8.1E1,4E0,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[3.7375215E-4,-2.2854786E-2,7.1063854E-2,-5.8360323E-2,1.0228556E-2,8.425252E-2,-3.8396563E-2,-4.3050762E-2,-1.4425461E-1,-3.8919474E-3,4.4126812E-2,2.1412544E-1,7.600787E-2,-8.458971E-3,1.855634E-2,-7.499242E-2,-9.817644E-3,-2.7764913E-1,-1.09219946E-1,1.0057027E-3,-1.2814783E-2,5.195271E-2,-6.6124233E-3,5.608803E-3,1.2762782E-2,4.6425205E-2,1.12194456E-1,-7.7419385E-2,1.3187157E-1,-7.061742E-2,-9.158675E-3,8.57492E-3,-1.88121E-2,-1.6626748E-3,-1.8969288E-2,-6.0778484E-2,-1.8828017E-1,-2.7380306E-2,2.4074294E-2,9.161428E-2,1.839152E-2,6.436518E-2,-5.232756E-2,1.2440169E-1,-9.069431E-4,-7.1217644E-4,-7.444602E-3,8.472259E-3,2.0179723E-3,-8.366807E-2,-3.0026326E-2,-2.7148295E-2,6.470269E-2,-8.010336E-2,2.396186E-3,-1.0680201E-1,-1.6745647E-2,-1.869584E-2,-7.0748855E-3,-5.630425E-2,3.18294E-2,5.9690524E-2,1.5382847E-1,4.316089E-2,-2.6562203E-2,1.940227E-2,1.1073856E-1,-8.700417E-3,-3.5067042E-3,8.8989146E-2,1.7802781E-1,-6.796249E-2,-1.1799029E-1,-3.81711E-3,-3.888446E-3,-1.6301822E-2,-7.552228E-2,-3.822054E-4,6.415212E-3,1.0924598E-3,-1.0593182E-1,-6.579887E-3,-9.84997E-4,2.7597444E-2,-3.753783E-2,-5.691611E-3,4.2805477E-4,-6.407973E-3,5.142304E-2,6.270709E-3,3.995773E-2,8.671582E-3,2.792569E-3,-4.433734E-3,6.541674E-2,-8.136432E-2,4.0618684E-2,-5.8471568E-2,3.6659908E-2,6.982494E-2,1.5573172E-1,1.5408626E-3,-2.328266E-3,1.165052E-1,3.0721609E-2,1.0859114E-2,1.315351E-1,-1.24975234E-1,-5.3757183E-2,-9.391959E-2,-8.30506E-3,4.103508E-3,-2.2957949E-2,-6.3306004E-2,-3.725974E-3,-4.7688512E-4,-1.0559738E-1,-7.248426E-3,-7.508661E-2,6.478108E-2,-4.0311858E-2,3.4639683E-2,-4.8735023E-2,1.5622701E-2,-5.602467E-3,6.097619E-2,4.8066482E-5,2.9032072E-3,5.1991437E-3,1.7157947E-3,-3.0516386E-3,-3.9130453E-5,4.044495E-3,-1.6181631E-3,-5.9868502E-3,-1.0374942E-3,3.6899229E-3,-6.05112E-3,5.4038176E-4,5.6653224E-2,-2.7900375E-3,4.9900594E-3,-9.71328E-3,2.879077E-3,1.7211762E-1,2.4420454E-3,1.4494091E-1,3.4271793E-3,-8.127193E-4,8.26875E-3,3.332406E-3,-1.8985095E-3,-7.909023E-3,4.1781426E-3,-8.0964744E-2,-1.172697E-1,5.1939E-4,2.1786975E-2,-2.8872078E-3,-1.2262E-2,-7.5702346E-3,-4.35586E-2,1.9688427E-2,-1.4707293E-1,-1.8735223E-3,-9.2844013E-4,-5.05943E-3,2.044092E-2,5.1416717E-3,-4.255829E-3,-3.2574005E-4,-7.035108E-4,3.3464157E-3,-1.23647325E-1,-3.5097986E-2,2.1621373E-3,5.303579E-3,4.0756516E-2,1.0062565E-1,2.6816274E-3,-1.142844E-3,4.214352E-3,2.893743E-2,-3.059964E-3,1.8020627E-3,9.515024E-3,4.612682E-3,7.3838206E-3,2.4686127E-3,-2.1000549E-3,1.2562141E-3,2.9393926E-3,-9.4918493E-4,-5.96852E-3,-2.1517708E-3,-7.7835103E-3,-3.8962588E-3,-8.35593E-4,2.9879336E-3,2.017854E-3,-2.5773207E-3,3.8914723E-4,-3.3399707E-3,-1.8202577E-3,1.6612908E-3,-3.5519698E-3,-9.326895E-3,-1.0680539E-3,2.821849E-3,-2.077275E-3,-9.6590705E-3,1.1493452E-3,-2.2182593E-3,6.698775E-4,-1.8111728E-3,8.8162813E-4,3.5410144E-3,1.6282144E-3,5.675965E-3,3.232429E-4,3.0703444E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,41,43,45,47,49,-1,-1,51,-1,-1,53,55,57,59,61,63,65,67,69,-1,-1,-1,-1,-1,71,73,75,77,79,-1,81,-1,83,-1,85,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,-1,113,115,-1,-1,-1,117,-1,-1,119,121,-1,-1,123,125,-1,127,-1,-1,129,131,133,135,137,139,141,143,-1,-1,145,147,-1,149,151,153,155,-1,-1,157,159,161,-1,163,-1,165,167,169,171,173,175,-1,177,-1,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,-1,-1,183,-1,185,-1,187,-1,189,-1,-1,-1,-1,191,193,195,-1,197,-1,199,-1,201,203,205,-1,-1,-1,207,-1,-1,-1,-1,-1,209,211,213,-1,215,217,-1,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6863446E0,9.08595E-1,3.6991894E-1,4.842862E-1,1.9211456E-1,2.2678661E-1,2.3415089E-1,3.368274E-1,2.3138726E-1,3.8068515E-1,1.8035841E-1,4.1351438E-2,2.2494793E-1,0E0,2.3859516E-1,7.335484E-2,2.7945817E-1,3.38638E-1,1.6472709E-1,1.8401024E-1,0E0,1.5005928E-1,0E0,0E0,0E0,2.1596271E-1,1.5692794E-1,5.98095E-2,3.7421122E-2,8.118659E-2,0E0,0E0,1.06882095E-1,0E0,0E0,6.946021E-2,2.0255941E-1,1.3474771E-1,9.8410495E-2,9.409565E-2,7.1295395E-2,2.105298E-1,1.2548997E-1,1.5127277E-1,0E0,0E0,0E0,0E0,0E0,5.6128144E-2,5.643835E-2,7.117568E-2,7.691259E-2,7.217732E-2,0E0,3.6549374E-2,0E0,1.04834594E-1,0E0,6.254794E-2,1.06717214E-1,4.859513E-2,4.2425722E-2,4.371965E-2,8.798298E-2,7.337125E-2,8.11677E-2,0E0,2.7605744E-2,8.460185E-2,5.9221506E-2,6.13119E-2,3.664863E-2,5.1688693E-2,0E0,6.715151E-2,4.7932282E-2,0E0,0E0,0E0,2.1230996E-2,0E0,0E0,9.0990454E-2,7.0383936E-2,0E0,0E0,1.2588629E-1,4.517995E-2,0E0,2.353527E-2,0E0,0E0,3.8317557E-2,4.1130565E-2,2.4511077E-2,2.902758E-2,5.240955E-2,8.879821E-2,8.609478E-2,2.4478793E-2,0E0,0E0,5.9301853E-2,2.6100218E-2,0E0,4.6277404E-2,5.499567E-2,1.0910152E-1,7.007365E-2,0E0,0E0,4.381951E-2,1.2117654E-1,8.572939E-2,0E0,3.9897308E-2,0E0,2.6196413E-2,4.5159906E-2,2.1751745E-2,2.3719955E-2,7.125728E-2,5.582016E-2,0E0,5.92331E-2,0E0,0E0,2.2002196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.280963E-2,0E0,0E0,3.0120505E-2,0E0,2.5905073E-2,0E0,2.5512457E-2,0E0,1.6238572E-2,0E0,0E0,0E0,0E0,3.511108E-2,7.192993E-2,2.4579614E-2,0E0,2.1691162E-2,0E0,4.382789E-2,0E0,5.1397614E-2,5.717989E-2,2.2950247E-2,0E0,0E0,0E0,2.4425022E-2,0E0,0E0,0E0,0E0,0E0,6.453876E-2,5.0777078E-2,1.9962048E-2,0E0,4.0541872E-2,3.3465147E-2,0E0,0E0,0E0,1.7646499E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,25,25,26,26,27,27,28,28,29,29,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,50,50,51,51,52,52,53,53,55,55,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,80,80,83,83,84,84,87,87,88,88,90,90,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,103,103,104,104,106,106,107,107,108,108,109,109,112,112,113,113,114,114,116,116,118,118,119,119,120,120,121,121,122,122,123,123,125,125,128,128,139,139,142,142,144,144,146,146,148,148,153,153,154,154,155,155,157,157,159,159,161,161,162,162,163,163,167,167,173,173,174,174,175,175,177,177,178,178,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,42,44,46,48,50,-1,-1,52,-1,-1,54,56,58,60,62,64,66,68,70,-1,-1,-1,-1,-1,72,74,76,78,80,-1,82,-1,84,-1,86,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,-1,114,116,-1,-1,-1,118,-1,-1,120,122,-1,-1,124,126,-1,128,-1,-1,130,132,134,136,138,140,142,144,-1,-1,146,148,-1,150,152,154,156,-1,-1,158,160,162,-1,164,-1,166,168,170,172,174,176,-1,178,-1,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,-1,-1,184,-1,186,-1,188,-1,190,-1,-1,-1,-1,192,194,196,-1,198,-1,200,-1,202,204,206,-1,-1,-1,208,-1,-1,-1,-1,-1,210,212,214,-1,216,218,-1,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.984E3,4.539777E5,1E0,2.2E1,4.577732E2,3.6E1,3.812E3,5.51E2,3.355864E0,2.511E3,1.1043767E3,1.0952997E3,2.839532E10,-8.458971E-3,2.2539758E5,5.0363636E1,2.04115E5,8E0,2.1772727E4,1.5859042E6,-1.2814783E-2,3.875E0,-6.6124233E-3,5.608803E-3,1.2762782E-2,4.88E2,3.7715748E7,4.9627705E0,3.08564E5,1.6840201E2,-9.158675E-3,8.57492E-3,8.13362E0,-1.6626748E-3,-1.8969288E-2,6.0078123E-6,3.7798166E0,1.70595E5,8.612209E-6,1.09652E5,9.822E3,1.1043767E3,5.11E2,1.6975454E3,-9.069431E-4,-7.1217644E-4,-7.444602E-3,8.472259E-3,2.0179723E-3,6.737099E7,2.732381E2,3.82E2,2.0480016E7,6.123E3,2.396186E-3,5.159652E-7,-1.6745647E-2,1.3902439E0,-7.0748855E-3,2.2715E4,1.185E3,9.46E2,3.43E2,4.4E0,3.4779343E1,3.64E2,1.4992306E8,-8.700417E-3,3.3266933E0,1.6673345E7,8.710612E1,7.785714E0,1.2380953E0,2.90099E0,-3.888446E-3,5.4E1,8.910034E0,-3.822054E-4,6.415212E-3,1.0924598E-3,7.336111E4,-6.579887E-3,-9.84997E-4,2.988E4,5.095006E5,-5.691611E-3,4.2805477E-4,1.2E1,3.7439062E2,6.270709E-3,1.0410659E5,8.671582E-3,2.792569E-3,1.01E2,4E0,2.2E1,3.53562E5,2.0939393E0,9.46898E2,2.059E3,8.822294E6,1.5408626E-3,-2.328266E-3,6.149315E6,1.752E3,1.0859114E-2,2.01E0,5.6E1,6.666982E4,4E0,-8.30506E-3,4.103508E-3,2.992648E-2,1.0439024E1,8.579633E-2,-4.7688512E-4,5.4051723E0,-7.248426E-3,9.99277E4,2.2084616E2,6.23E2,1.8736842E0,6.072304E5,7.6569915E-1,-5.602467E-3,2.8282208E0,4.8066482E-5,2.9032072E-3,3.588785E0,1.7157947E-3,-3.0516386E-3,-3.9130453E-5,4.044495E-3,-1.6181631E-3,-5.9868502E-3,-1.0374942E-3,3.6899229E-3,-6.05112E-3,5.4038176E-4,2.0588236E0,-2.7900375E-3,4.9900594E-3,1.3795834E3,2.879077E-3,1.853776E7,2.4420454E-3,1.4379113E9,3.4271793E-3,3.08E2,8.26875E-3,3.332406E-3,-1.8985095E-3,-7.909023E-3,3.5419354E0,3.774648E0,3.89E2,5.1939E-4,1.4392524E0,-2.8872078E-3,1.397998E6,-7.5702346E-3,5.3776E4,1.4929079E2,1.9067797E1,-1.8735223E-3,-9.2844013E-4,-5.05943E-3,3.0751158E4,5.1416717E-3,-4.255829E-3,-3.2574005E-4,-7.035108E-4,3.3464157E-3,3.2296E4,4E0,1E0,5.303579E-3,2.6138635E2,7.09E2,2.6816274E-3,-1.142844E-3,4.214352E-3,2.1111112E0,-3.059964E-3,1.8020627E-3,9.515024E-3,4.612682E-3,7.3838206E-3,2.4686127E-3,-2.1000549E-3,1.2562141E-3,2.9393926E-3,-9.4918493E-4,-5.96852E-3,-2.1517708E-3,-7.7835103E-3,-3.8962588E-3,-8.35593E-4,2.9879336E-3,2.017854E-3,-2.5773207E-3,3.8914723E-4,-3.3399707E-3,-1.8202577E-3,1.6612908E-3,-3.5519698E-3,-9.326895E-3,-1.0680539E-3,2.821849E-3,-2.077275E-3,-9.6590705E-3,1.1493452E-3,-2.2182593E-3,6.698775E-4,-1.8111728E-3,8.8162813E-4,3.5410144E-3,1.6282144E-3,5.675965E-3,3.232429E-4,3.0703444E-3],"split_indices":[2,43,6,3,4,10,2,2,73,2,67,48,46,0,48,61,5,0,48,43,0,69,0,0,0,0,62,61,1,4,0,0,69,0,0,52,68,1,53,1,44,67,0,67,0,0,0,0,0,7,4,0,5,43,0,52,0,68,0,44,44,44,0,69,71,10,5,0,68,47,73,67,71,69,0,0,71,0,0,0,60,0,0,1,43,0,0,8,4,0,48,0,0,0,8,71,1,68,67,44,62,0,0,60,0,0,68,44,60,8,0,0,53,73,53,0,69,0,43,67,2,68,43,57,0,69,0,0,71,0,0,0,0,0,0,0,0,0,0,69,0,0,67,0,62,0,12,0,8,0,0,0,0,69,69,1,0,68,0,60,0,1,67,73,0,0,0,48,0,0,0,0,0,12,73,30,0,4,2,0,0,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,7.72E2,2.53E2,3.72E2,4E2,2.26E2,2.7E1,3.17E2,5.5E1,2.83E2,1.17E2,1.2E1,2.14E2,7E0,2E1,1.61E2,1.56E2,1E1,4.5E1,2.79E2,4E0,1.13E2,4E0,6E0,6E0,1.19E2,9.5E1,1.1E1,9E0,1.57E2,4E0,6E0,1.5E2,4E0,6E0,2.9E1,1.6E1,1.25E2,1.54E2,5.1E1,6.2E1,1.01E2,1.8E1,8.7E1,8E0,7E0,4E0,5E0,4E0,1.18E2,3.9E1,1.37E2,1.3E1,2.5E1,4E0,1.2E1,4E0,1.18E2,7E0,1.3E1,1.41E2,3.5E1,1.6E1,4E1,2.2E1,5.2E1,4.9E1,4E0,1.4E1,5.4E1,3.3E1,8.3E1,3.5E1,2.7E1,1.2E1,1.13E2,2.4E1,7E0,6E0,5E0,2E1,8E0,4E0,3.4E1,8.4E1,6E0,7E0,4.8E1,9.3E1,6E0,2.9E1,1.1E1,5E0,1.3E1,2.7E1,1.2E1,1E1,9E0,4.3E1,2.7E1,2.2E1,8E0,6E0,3.6E1,1.8E1,1.3E1,2E1,1.5E1,6.8E1,2.7E1,8E0,4E0,2.3E1,2.3E1,9E1,8E0,1.6E1,6E0,1.4E1,2.2E1,1.2E1,1.1E1,7.3E1,4.1E1,7E0,7.8E1,1.5E1,1.7E1,1.2E1,8E0,5E0,7E0,2E1,7E0,5E0,4E0,6E0,4E0,5E0,3.6E1,7E0,1.8E1,9E0,4E0,1.8E1,1.2E1,2.4E1,7E0,1.1E1,1E1,1E1,6E0,9E0,2.2E1,4.6E1,2.2E1,5E0,1.1E1,1.2E1,1.6E1,7E0,3.3E1,5.7E1,9E0,7E0,6E0,8E0,1.2E1,1E1,4E0,8E0,5E0,6E0,1E1,6.3E1,3.7E1,4E0,5.3E1,2.5E1,4E0,8E0,1.5E1,2.1E1,4E0,5E0,1.1E1,7E0,2E1,4E0,4E0,7E0,6E0,1.6E1,1.8E1,2.8E1,7E0,1.5E1,6E0,5E0,7E0,9E0,1.2E1,2.1E1,1.2E1,4.5E1,5E0,4E0,6E0,6E0,6E0,4E0,1.1E1,5.2E1,2.9E1,8E0,3.4E1,1.9E1,7E0,1.8E1,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-4.676902E-3,-2.7015233E-2,6.269562E-2,-1.4433283E-1,-2.026201E-2,7.7048E-2,-5.1342774E-2,-9.494207E-2,-1.610266E-2,-6.3832276E-2,-1.9735934E-3,4.491648E-2,1.1286066E-1,-1.2745735E-1,-5.681625E-4,-1.1262371E-1,-2.938342E-2,-5.8491174E-2,-1.0141822E-2,8.9425035E-3,-5.341162E-3,1.5622101E-2,9.0033844E-2,1.543186E-1,4.7166493E-2,-1.5672048E-3,-8.751095E-3,4.3330923E-2,-3.0651113E-3,-1.3539714E-3,-5.7882154E-3,7.4794714E-4,-3.191422E-3,-9.907242E-2,-4.4012662E-2,-1.892396E-2,3.1855647E-2,1.0822931E-1,3.3164932E-3,1.4114897E-1,3.090772E-2,1.3716716E-1,1.3015919E-2,6.0195044E-2,-3.072016E-3,-5.882701E-4,3.93597E-3,-7.8674644E-2,-9.776861E-3,2.4785649E-2,-5.5658653E-2,-4.1122038E-2,1.3640688E-2,-2.4848082E-3,7.637008E-2,6.878172E-3,2.121992E-3,3.5483663E-3,-1.5115118E-2,9.114572E-3,1.0523726E-1,-2.7630284E-2,8.2565285E-2,1.4738992E-1,8.7286456E-4,1.8635593E-2,3.5730659E-3,-1.3134022E-1,-4.6591673E-2,3.951455E-3,-2.1455933E-4,-6.1301332E-2,2.0187856E-3,-1.1591104E-2,-3.529352E-2,2.9082855E-2,-5.7990752E-2,-4.078118E-2,2.527182E-2,8.7465055E-2,-3.0207979E-3,-4.6664895E-3,-6.565478E-3,5.4957136E-4,6.054689E-3,1.6593332E-3,-3.6105593E-3,1.4986608E-3,7.219949E-3,8.436715E-2,1.6638876E-1,-1.131794E-3,3.3744434E-3,-2.4045259E-3,-7.69905E-3,2.8641973E-2,-7.286297E-2,1.9043525E-3,-1.4012039E-3,-5.3395648E-2,-1.3524973E-1,-7.0584394E-2,-4.4139833E-3,3.4816034E-2,-5.546807E-3,-2.6899775E-2,-1.1126453E-1,-1.0510528E-2,-8.301122E-3,7.641999E-2,1.2568398E-3,8.1047835E-3,6.731646E-2,8.0370903E-4,-3.5109697E-3,2.2227303E-3,6.039655E-3,1.818477E-1,2.915891E-3,3.3756718E-3,-9.938544E-4,-5.5721467E-3,-5.0246067E-2,-5.949569E-2,5.75592E-4,-2.1886944E-3,-8.00468E-3,-1.6279727E-2,-5.578866E-2,4.6154104E-2,-2.8410831E-2,7.7605054E-2,2.4406698E-2,-2.8837048E-3,1.9822894E-3,-2.5136247E-3,-7.2642816E-3,2.9133026E-2,-6.688009E-2,6.576708E-4,4.515419E-3,2.1913093E-2,-5.6261286E-2,3.962466E-2,1.2559676E-1,-1.2319154E-2,4.6064172E-2,6.051044E-3,1.041517E-2,-1.1298243E-4,-3.4160744E-3,-1.8129412E-3,-3.7574493E-3,-2.947179E-3,5.0381785E-3,2.8153555E-3,-1.392568E-3,-2.459323E-3,2.901479E-3,4.643642E-3,-3.637462E-4,2.4517943E-4,2.3325812E-3,-6.185418E-4,2.5433982E-3,3.1915025E-3,-6.629384E-3,1.583631E-3,-1.5894301E-3,-5.4530525E-3,7.744604E-4,3.6098657E-3,-1.6096738E-4,7.241138E-4,7.5832903E-3,1.0984136E-4,-2.4812461E-3,4.3974826E-3,2.0313096E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,-1,67,-1,69,71,73,75,77,79,-1,-1,-1,81,-1,83,85,87,89,-1,91,-1,93,95,-1,97,99,-1,-1,101,103,105,107,109,111,-1,-1,113,-1,-1,-1,-1,-1,-1,115,117,-1,-1,-1,-1,119,121,-1,-1,123,125,127,129,131,-1,133,135,-1,137,139,141,-1,143,145,-1,-1,-1,147,-1,-1,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,161,-1,-1,-1,-1,163,165,-1,-1,167,169,171,173,175,177,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5894581E0,6.2534726E-1,4.3441367E-1,4.0062398E-1,5.9829426E-1,2.6419806E-1,1.15868226E-1,3.8421363E-2,0E0,1.7176777E-1,3.5003278E-1,1.6380414E-1,2.923113E-1,6.312376E-2,5.773878E-2,2.5873601E-2,1.7400166E-2,1.2255162E-1,0E0,0E0,2.6433644E-1,8.738933E-2,1.437271E-1,1.0777593E-1,6.915995E-2,0E0,0E0,2.9250408E-2,0E0,0E0,0E0,0E0,0E0,1.11916184E-1,1.303767E-1,2.7742112E-1,2.1420023E-1,1.5954398E-2,9.462356E-2,3.1656146E-2,7.485564E-2,7.096934E-2,0E0,2.7061254E-2,0E0,0E0,0E0,7.667437E-2,0E0,3.6465917E-2,8.0046386E-2,2.6991242E-1,1.7424512E-1,8.606886E-2,1.0165897E-1,0E0,0E0,0E0,4.0460773E-2,0E0,4.182124E-2,4.133342E-2,4.2965256E-2,5.3014398E-2,0E0,3.2296866E-2,0E0,4.1722536E-2,6.533447E-2,0E0,2.371108E-2,6.936258E-2,0E0,0E0,2.427063E-1,1.1448563E-1,4.338435E-2,2.04315E-1,5.76351E-2,8.921772E-2,0E0,0E0,2.6666917E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.8924244E-2,5.5575132E-2,0E0,0E0,0E0,0E0,2.1625318E-2,1.9858018E-2,0E0,0E0,4.8947155E-2,3.1878382E-2,4.139595E-1,1.4692824E-1,5.3781584E-2,0E0,4.9789E-2,1.7583951E-2,0E0,6.773403E-2,1.8493675E-2,4.0440455E-2,0E0,7.248983E-2,2.907692E-2,0E0,0E0,0E0,5.0305963E-2,0E0,0E0,0E0,0E0,2.0207822E-2,4.5457184E-2,0E0,0E0,0E0,0E0,1.3890344E-1,4.5832537E-2,1.8900472E-1,4.7788307E-2,5.0546292E-2,0E0,0E0,0E0,0E0,2.1678159E-2,1.3491501E-1,0E0,0E0,1.8588802E-2,4.2577066E-2,5.6720186E-2,6.098543E-2,2.3242194E-2,2.2095801E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,27,27,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,47,47,49,49,50,50,51,51,52,52,53,53,54,54,58,58,60,60,61,61,62,62,63,63,65,65,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,82,82,89,89,90,90,95,95,96,96,99,99,100,100,101,101,102,102,103,103,105,105,106,106,108,108,109,109,110,110,112,112,113,113,117,117,122,122,123,123,128,128,129,129,130,130,131,131,132,132,137,137,138,138,141,141,142,142,143,143,144,144,145,145,146,146],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,-1,68,-1,70,72,74,76,78,80,-1,-1,-1,82,-1,84,86,88,90,-1,92,-1,94,96,-1,98,100,-1,-1,102,104,106,108,110,112,-1,-1,114,-1,-1,-1,-1,-1,-1,116,118,-1,-1,-1,-1,120,122,-1,-1,124,126,128,130,132,-1,134,136,-1,138,140,142,-1,144,146,-1,-1,-1,148,-1,-1,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,162,-1,-1,-1,-1,164,166,-1,-1,168,170,172,174,176,178,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,3.964269E3,2.856934E7,4.6463413E0,5.02E2,7.098E3,1.3655363E2,5.7E1,-1.610266E-2,4.5E1,2.04115E5,5.9671835E6,8.710612E1,4.797448E8,2.342E4,1.2E1,1.86E2,3.89E2,-1.0141822E-2,8.9425035E-3,2.0766992E6,5.617715E6,1.1621469E1,8.604875E3,1.0534078E4,-1.5672048E-3,-8.751095E-3,3.592428E7,-3.0651113E-3,-1.3539714E-3,-5.7882154E-3,7.4794714E-4,-3.191422E-3,2.161111E1,8E0,2.026453E6,5.441268E7,7.6825094E0,3.9914E4,4.27E3,2.95E2,4.931361E-2,1.3015919E-2,1.6326721E0,-3.072016E-3,-5.882701E-4,3.93597E-3,1.2571428E1,-9.776861E-3,3.2226672E7,1.321E3,7E0,3.84E2,2.524828E2,1.8393531E0,6.878172E-3,2.121992E-3,3.5483663E-3,8.295515E7,9.114572E-3,3.0984934E8,1.0657745E7,1.4916515E3,6.7321223E-1,8.7286456E-4,5.063952E-2,3.5730659E-3,1.11E2,2.88E0,3.951455E-3,2.1277844E-4,2.7E1,2.0187856E-3,-1.1591104E-2,3.7439062E2,1E0,5.94E2,2.5247778E6,1.125584E-3,9.4658756E-1,-3.0207979E-3,-4.6664895E-3,1.7E1,5.4957136E-4,6.054689E-3,1.6593332E-3,-3.6105593E-3,1.4986608E-3,7.219949E-3,1.7379E4,4.262058E0,-1.131794E-3,3.3744434E-3,-2.4045259E-3,-7.69905E-3,1.7217412E4,3.887931E0,1.9043525E-3,-1.4012039E-3,1.605076E6,1.2E1,1.16408E5,1.798E3,6E0,-5.546807E-3,8.172E3,2.7012987E1,-1.0510528E-2,6.2521395E6,3.5070792E2,7.2744107E0,8.1047835E-3,6.1319604E2,9.643839E6,-3.5109697E-3,2.2227303E-3,6.039655E-3,1.562752E3,2.915891E-3,3.3756718E-3,-9.938544E-4,-5.5721467E-3,1.8E1,5.217E4,5.75592E-4,-2.1886944E-3,-8.00468E-3,-1.6279727E-2,2.4E1,3.807421E6,3.299862E1,1.6566709E6,1.5026077E7,-2.8837048E-3,1.9822894E-3,-2.5136247E-3,-7.2642816E-3,4.0705118E6,1.1626786E2,6.576708E-4,4.515419E-3,3.82E2,2.33E2,3.839424E2,9.68E2,4.5620965E6,4.6779457E1,6.051044E-3,1.041517E-2,-1.1298243E-4,-3.4160744E-3,-1.8129412E-3,-3.7574493E-3,-2.947179E-3,5.0381785E-3,2.8153555E-3,-1.392568E-3,-2.459323E-3,2.901479E-3,4.643642E-3,-3.637462E-4,2.4517943E-4,2.3325812E-3,-6.185418E-4,2.5433982E-3,3.1915025E-3,-6.629384E-3,1.583631E-3,-1.5894301E-3,-5.4530525E-3,7.744604E-4,3.6098657E-3,-1.6096738E-4,7.241138E-4,7.5832903E-3,1.0984136E-4,-2.4812461E-3,4.3974826E-3,2.0313096E-4],"split_indices":[2,43,60,68,2,2,71,6,0,6,5,60,73,7,9,3,6,1,0,0,43,12,71,4,4,0,0,60,0,0,0,0,0,67,10,9,7,71,9,2,0,72,0,53,0,0,0,70,0,7,10,3,0,4,53,0,0,0,7,0,7,60,4,53,0,72,0,1,69,0,53,8,0,0,4,79,0,43,53,68,0,0,3,0,0,0,0,0,0,2,53,0,0,0,0,43,69,0,0,12,3,7,44,3,0,44,73,0,43,48,71,0,67,62,0,0,0,67,0,0,0,0,70,12,0,0,0,0,3,62,71,43,5,0,0,0,0,43,4,0,0,4,0,4,10,62,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.054E3,7.92E2,2.62E2,4.2E1,7.5E2,2.33E2,2.9E1,3.5E1,7E0,2.21E2,5.29E2,1.24E2,1.09E2,1.1E1,1.8E1,2.7E1,8E0,2.15E2,6E0,8E0,5.21E2,7.6E1,4.8E1,6.6E1,4.3E1,5E0,6E0,1.1E1,7E0,4E0,2.3E1,4E0,4E0,5.5E1,1.6E2,3.82E2,1.39E2,8E0,6.8E1,2.5E1,2.3E1,6E1,6E0,3.9E1,4E0,5E0,6E0,4.8E1,7E0,2.3E1,1.37E2,2.27E2,1.55E2,7.9E1,6E1,4E0,4E0,1.3E1,5.5E1,8E0,1.7E1,1.1E1,1.2E1,5.5E1,5E0,1.2E1,2.7E1,1.7E1,3.1E1,6E0,1.7E1,1.3E2,7E0,5E0,2.22E2,1.28E2,2.7E1,3.3E1,4.6E1,5.6E1,4E0,4E0,5.1E1,4E0,1.3E1,5E0,6E0,8E0,4E0,1.4E1,4.1E1,7E0,5E0,6E0,1.1E1,8E0,2.3E1,7E0,1E1,1.19E2,1.1E1,1.03E2,1.19E2,1.24E2,4E0,1.8E1,9E0,4E0,2.9E1,1.4E1,3.2E1,9E0,4.7E1,4.7E1,4E0,9E0,5E0,3.5E1,6E0,4E0,4E0,6E0,1.7E1,1.09E2,1E1,4E0,7E0,4E0,9.9E1,3.8E1,8.1E1,2.3E1,1.01E2,1.2E1,6E0,5E0,4E0,1.8E1,1.1E1,4E0,1E1,2.4E1,8E0,3.3E1,1.4E1,3.7E1,1E1,1.8E1,1.7E1,6E0,1.1E1,5.8E1,5.1E1,9.5E1,4E0,3.2E1,6E0,6.4E1,1.7E1,1.8E1,5E0,5.9E1,4.2E1,7E0,1.1E1,4E0,7E0,2E1,4E0,4E0,4E0,1.7E1,1.6E1,4E0,1E1,2.8E1,9E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"179","size_leaf_vector":"1"}},{"base_weights":[1.8820602E-3,-2.3254154E-2,6.0664844E-2,-5.817044E-2,-4.1928925E-3,-9.252778E-2,6.8936184E-2,-4.7223926E-2,-2.159959E-1,-5.2362785E-2,8.787443E-3,-6.736836E-4,-5.850275E-3,4.4177786E-2,1.0824084E-1,-5.3769454E-2,4.6137717E-2,9.752715E-4,-3.6037058E-1,-3.6191095E-2,-1.0993559E-1,5.4022055E-3,6.5471455E-3,5.4822262E-2,-1.4119607E-1,1.394868E-1,-4.8247185E-3,7.382396E-2,-5.8908287E-2,3.6359744E-3,5.573894E-4,-2.163606E-2,-7.334285E-3,-6.334675E-2,-1.2614475E-2,-6.461285E-3,-2.5178462E-3,1.9779388E-2,-1.5173376E-2,6.0100798E-2,-3.5424551E-3,2.4684942E-3,-1.6008442E-2,1.2143163E-1,1.2305972E-2,3.0801974E-2,-7.091179E-3,-1.17766685E-4,6.256841E-3,-7.7221796E-2,-3.4447607E-2,-8.518524E-2,-2.7579462E-4,-6.2074084E-2,1.088716E-2,-6.835149E-4,3.4545116E-2,5.4946378E-2,-2.8622575E-2,6.372312E-2,-3.1338425E-3,1.4537615E-1,3.9426208E-2,-4.7660664E-2,8.2752876E-2,-6.494196E-2,-1.8335184E-1,-4.438151E-2,1.3905023E-3,-4.4974156E-2,-4.966688E-3,-4.166776E-3,3.0381273E-4,-7.781972E-3,3.3882523E-3,6.3535824E-2,-1.562934E-2,4.8185106E-2,-8.890355E-3,-1.5526025E-2,6.4513925E-3,-1.07028455E-1,-1.6015884E-2,9.540378E-2,4.4773288E-2,2.086556E-1,1.1762326E-1,-2.1872087E-3,7.379692E-2,7.913354E-4,-4.7547147E-3,5.4156943E-3,1.0624671E-3,-5.5279616E-2,-1.22451484E-1,-1.0283447E-2,-3.8761199E-3,-5.5431057E-2,1.4309211E-3,-3.989234E-3,-4.1560328E-4,9.28315E-4,-2.2553199E-3,1.5722326E-3,5.8921976E-3,-4.038363E-2,2.5596613E-2,3.7701253E-2,1.1580935E-1,-3.2595225E-2,1.9301457E-3,1.375967E-3,-3.0071088E-3,-2.1518843E-2,-1.8064602E-1,-5.7524736E-3,-4.8969863E-3,5.4504536E-2,1.3583387E-1,-3.12618E-3,5.37707E-2,4.7856667E-3,1.1263395E-2,9.045327E-3,1.049118E-1,6.038466E-3,1.2777855E-3,-1.8153297E-2,-7.628401E-2,-1.0370394E-2,-4.8994023E-2,-6.645577E-2,-1.1319116E-3,-4.728548E-3,-6.529911E-2,-1.1599367E-3,6.519381E-2,4.3444466E-2,-2.745755E-3,2.424189E-3,7.0940354E-3,-2.5433872E-3,9.325106E-4,1.3196211E-3,-3.1090914E-3,-4.4090957E-3,-1.0613119E-2,-3.6048897E-2,1.5763143E-2,-1.1187796E-3,6.706382E-2,7.248702E-3,1.537885E-4,1.026793E-1,3.6721017E-2,1.1499044E-1,1.8928207E-3,-5.172863E-5,-2.879527E-3,-2.4938239E-3,-5.6707766E-3,-4.5329537E-3,9.1259036E-4,-1.2672386E-3,-3.8949156E-3,-3.1517833E-3,1.6754561E-3,-1.8356799E-3,1.2716099E-3,4.4266082E-4,-4.026241E-3,4.5792023E-3,-1.3520306E-3,2.477007E-3,1.7522495E-4,-3.183424E-5,-3.7510216E-3,1.4656065E-3,-1.9369116E-3,7.3491996E-3,1.7981133E-3,2.4840212E-4,5.6303227E-3,4.980934E-4,3.1788892E-3,9.120172E-3,4.500747E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,37,-1,39,41,43,45,47,49,-1,-1,-1,-1,51,53,-1,-1,55,57,59,-1,-1,-1,61,-1,63,-1,-1,-1,65,67,69,-1,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,-1,99,-1,-1,-1,101,-1,103,105,107,109,111,-1,113,115,117,119,121,123,-1,125,-1,-1,-1,-1,127,129,-1,-1,131,-1,-1,-1,-1,-1,-1,-1,133,135,137,139,141,-1,-1,-1,143,145,147,-1,149,151,-1,153,-1,-1,-1,155,-1,-1,157,159,-1,161,163,165,167,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,175,177,-1,179,-1,-1,181,183,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4996736E0,4.7296107E-1,3.9157927E-1,4.2343718E-1,2.8870562E-1,4.0685713E-2,2.7645004E-1,1.467008E-1,5.553214E-1,8.726418E-2,1.6303253E-1,0E0,0E0,3.609448E-1,3.9281845E-1,1.4925438E-1,1.680354E-2,0E0,1.4330113E-1,4.913953E-2,2.7211338E-2,1.05611645E-1,0E0,1.2261045E-1,4.1811293E-1,1.6696012E-1,1.3714084E-1,4.2692162E-2,9.2755556E-2,0E0,0E0,0E0,0E0,4.460047E-2,5.065822E-2,0E0,0E0,6.3475035E-2,1.4020173E-1,8.032316E-2,0E0,0E0,0E0,1.4742768E-1,0E0,8.954436E-2,0E0,0E0,0E0,1.4578718E-1,6.096653E-2,1.8501371E-2,0E0,2.7617157E-2,3.5549108E-2,8.64162E-2,7.270843E-2,1.4541936E-1,1.20852254E-1,9.1783404E-2,0E0,8.082855E-2,6.0157947E-2,3.3983726E-2,2.408287E-2,5.4355353E-2,2.8152108E-2,6.916009E-2,0E0,1.5597679E-2,0E0,0E0,0E0,2.8434437E-2,0E0,2.7761877E-2,7.553158E-2,6.156738E-2,3.7395038E-2,3.3378836E-2,0E0,1.0312788E-1,9.913029E-2,9.0156436E-2,1.0682958E-1,3.7520826E-2,2.3178458E-2,0E0,3.314446E-2,0E0,0E0,0E0,0E0,7.347754E-2,1.0141474E-1,0E0,0E0,4.242766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0555954E-2,5.7853438E-2,4.8326865E-2,2.0629838E-2,2.6002686E-2,0E0,0E0,0E0,2.2556836E-2,1.5714139E-2,6.4449176E-2,0E0,3.2273136E-2,8.043331E-2,0E0,7.544774E-2,0E0,0E0,0E0,2.1066189E-2,0E0,0E0,2.7019009E-2,5.658865E-2,0E0,3.5709593E-2,3.879094E-2,3.5091456E-2,2.3674473E-2,4.5012444E-2,0E0,5.417341E-2,3.125538E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.539129E-2,5.47331E-2,0E0,6.372522E-2,0E0,0E0,4.4162273E-2,5.9186332E-2,3.000325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,37,37,38,38,39,39,43,43,45,45,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,73,73,75,75,76,76,77,77,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,88,88,93,93,94,94,97,97,105,105,106,106,107,107,108,108,109,109,113,113,114,114,115,115,117,117,118,118,120,120,124,124,127,127,128,128,130,130,131,131,132,132,133,133,134,134,136,136,137,137,147,147,148,148,150,150,153,153,154,154,155,155],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,38,-1,40,42,44,46,48,50,-1,-1,-1,-1,52,54,-1,-1,56,58,60,-1,-1,-1,62,-1,64,-1,-1,-1,66,68,70,-1,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,-1,100,-1,-1,-1,102,-1,104,106,108,110,112,-1,114,116,118,120,122,124,-1,126,-1,-1,-1,-1,128,130,-1,-1,132,-1,-1,-1,-1,-1,-1,-1,134,136,138,140,142,-1,-1,-1,144,146,148,-1,150,152,-1,154,-1,-1,-1,156,-1,-1,158,160,-1,162,164,166,168,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,176,178,-1,180,-1,-1,182,184,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1E0,1E0,9.369565E0,5.02E2,1.4E1,1.3627213E3,9.454259E10,1.654E3,1.706875E1,7.82261E6,-6.736836E-4,-5.850275E-3,3.3386944E5,1.8545505E2,7E0,8.238114E2,9.752715E-4,8.01E2,1.631108E6,2.1478824E2,9E0,6.5471455E-3,1.0428572E1,1.52179E5,6.5026445E3,8.127911E7,1.9E1,3.168978E-2,3.6359744E-3,5.573894E-4,-2.163606E-2,-7.334285E-3,1.6496063E2,2.6086957E0,-6.461285E-3,-2.5178462E-3,1.8E7,4.273336E-3,2E1,-3.5424551E-3,2.4684942E-3,-1.6008442E-2,1.2820834E7,1.2305972E-2,2.8181964E-1,-7.091179E-3,-1.17766685E-4,6.256841E-3,2.4E1,1.559733E6,1.5365228E5,-2.7579462E-4,4.0705118E6,6.763314E7,2.63E2,1E0,5.72202E2,4.727453E7,3.596E3,-3.1338425E-3,1.6666666E0,1.762852E7,1.8421053E0,7.184408E-1,7.24E2,3.8E1,1.3745962E0,1.3905023E-3,1.9666272E5,-4.966688E-3,-4.166776E-3,3.0381273E-4,2.3412812E0,3.3882523E-3,4.0351807E2,7.058982E-1,1.24147E5,1.1046409E7,2.732439E2,6.4513925E-3,3.14403E5,1.1251582E10,5.928425E2,1.7302156E5,1.7265024E3,1E0,-2.1872087E-3,3.3765998E0,7.913354E-4,-4.7547147E-3,5.4156943E-3,1.0624671E-3,2E0,1.5132743E0,-1.0283447E-2,-3.8761199E-3,1E2,1.4309211E-3,-3.989234E-3,-4.1560328E-4,9.28315E-4,-2.2553199E-3,1.5722326E-3,5.8921976E-3,3.3358974E0,8.15736E7,5.6818056E7,1.2E1,5.217E4,1.9301457E-3,1.375967E-3,-3.0071088E-3,1.16E2,2.75E0,6.0875E4,-4.8969863E-3,2.1924414E2,1.2979348E3,-3.12618E-3,1.552356E0,4.7856667E-3,1.1263395E-2,9.045327E-3,8.847283E8,6.038466E-3,1.2777855E-3,6.843911E6,2.0007E5,-1.0370394E-2,5E0,6.81531E-2,4.9390244E0,2.6043478E2,2.0162934E5,-1.1599367E-3,3.588785E0,1.14E3,-2.745755E-3,2.424189E-3,7.0940354E-3,-2.5433872E-3,9.325106E-4,1.3196211E-3,-3.1090914E-3,-4.4090957E-3,-1.0613119E-2,8E1,4.7220547E1,-1.1187796E-3,3.1157124E-1,7.248702E-3,1.537885E-4,2.5454E4,7.177158E6,1.3358E4,1.8928207E-3,-5.172863E-5,-2.879527E-3,-2.4938239E-3,-5.6707766E-3,-4.5329537E-3,9.1259036E-4,-1.2672386E-3,-3.8949156E-3,-3.1517833E-3,1.6754561E-3,-1.8356799E-3,1.2716099E-3,4.4266082E-4,-4.026241E-3,4.5792023E-3,-1.3520306E-3,2.477007E-3,1.7522495E-4,-3.183424E-5,-3.7510216E-3,1.4656065E-3,-1.9369116E-3,7.3491996E-3,1.7981133E-3,2.4840212E-4,5.6303227E-3,4.980934E-4,3.1788892E-3,9.120172E-3,4.500747E-3],"split_indices":[2,29,17,69,2,3,67,46,1,73,43,0,0,48,73,3,48,0,2,5,4,8,0,68,1,4,59,0,53,0,0,0,0,4,69,0,0,5,53,6,0,0,0,9,0,49,0,0,0,3,9,43,0,43,7,44,30,48,7,2,0,69,62,68,49,10,3,53,0,60,0,0,0,68,0,4,53,1,60,67,0,9,46,67,43,4,13,0,53,0,0,0,0,32,68,0,0,6,0,0,0,0,0,0,0,69,7,12,3,12,0,0,0,0,71,1,0,67,4,0,69,0,0,0,7,0,0,60,5,0,8,53,71,67,43,0,71,10,0,0,0,0,0,0,0,0,0,0,73,0,53,0,0,9,60,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.013E3,7.1E2,3.03E2,2.5E2,4.6E2,1.5E1,2.88E2,2.35E2,1.5E1,9.7E1,3.63E2,5E0,1E1,1.78E2,1.1E2,2.2E2,1.5E1,6E0,9E0,7.7E1,2E1,3.55E2,8E0,1.69E2,9E0,8.6E1,2.4E1,8E0,2.12E2,7E0,8E0,5E0,4E0,3.5E1,4.2E1,1.2E1,8E0,2.09E2,1.46E2,1.63E2,6E0,5E0,4E0,7.7E1,9E0,2E1,4E0,4E0,4E0,1.2E2,9.2E1,2.5E1,1E1,1.3E1,2.9E1,8.8E1,1.21E2,2.3E1,1.23E2,1.59E2,4E0,5.9E1,1.8E1,8E0,1.2E1,1.09E2,1.1E1,8E1,1.2E1,1E1,1.5E1,9E0,4E0,2.3E1,6E0,1.6E1,7.2E1,9.2E1,2.9E1,1.3E1,1E1,1.6E1,1.07E2,5.8E1,1.01E2,1.6E1,4.3E1,5E0,1.3E1,4E0,4E0,7E0,5E0,9.5E1,1.4E1,7E0,4E0,7E1,1E1,4E0,6E0,1.4E1,9E0,1.2E1,4E0,4.5E1,2.7E1,8.1E1,1.1E1,2E1,9E0,7E0,6E0,8E0,8E0,9.7E1,1E1,3E1,2.8E1,7E0,9.4E1,5E0,1.1E1,4E0,3.9E1,5E0,8E0,3.5E1,6E1,5E0,9E0,5.8E1,1.2E1,1.9E1,2.6E1,1.2E1,1.5E1,7.7E1,4E0,5E0,6E0,1.4E1,6E0,4E0,4E0,4E0,4E0,4E1,5.7E1,4E0,2.6E1,2.4E1,4E0,2.3E1,7.1E1,3.3E1,6E0,2.6E1,9E0,4.2E1,1.8E1,5E0,4E0,1.9E1,3.9E1,4E0,8E0,9E0,1E1,6E0,2E1,1.1E1,4E0,6.1E1,1.6E1,2.3E1,1.7E1,4.5E1,1.2E1,5E0,2.1E1,4E0,1.9E1,4E1,3.1E1,4E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[1.423794E-3,-5.0470393E-2,2.6536688E-2,-4.087611E-2,-1.5317702E-1,1.2591555E-2,1.1108404E-1,-8.011143E-2,-2.2470837E-2,-1.158281E-1,-1.3943634E-2,2.1700311E-2,-1.2029618E-1,1.2241098E-1,-5.627696E-3,-6.958135E-2,-1.2009375E-2,-1.1587812E-2,-5.7770766E-2,-7.2058383E-4,-1.4138368E-1,1.3850019E-2,8.25383E-2,-8.737344E-2,-1.478885E-2,9.8302804E-2,1.1615753E-2,-1.4650612E-1,-5.2909844E-2,2.0234497E-2,-2.6525741E-2,-2.550336E-2,-1.1192556E-1,-8.094936E-3,-3.8955284E-3,8.154739E-3,1.0790355E-2,1.0820098E-2,1.04724824E-1,-1.111358E-3,-4.6237097E-3,5.5580497E-2,1.286719E-1,-3.4692339E-3,-7.952699E-3,-6.1069567E-2,2.5494979E-3,1.702953E-3,6.182404E-2,-5.810492E-2,-8.361054E-3,-6.5425164E-3,6.7385836E-3,-6.0565653E-3,-1.656692E-3,-1.1868992E-2,2.1842804E-2,2.7411855E-3,-1.6625805E-2,7.854613E-2,7.5161792E-3,-1.8854549E-3,7.300622E-2,1.0140893E-2,9.734331E-2,-9.645782E-2,-2.5936764E-2,-2.9446915E-2,3.2763194E-2,4.7156163E-3,4.0735537E-4,5.032486E-4,-8.1467345E-2,3.6263962E-3,-1.4718059E-2,2.7741652E-2,-3.1366963E-3,-1.6080143E-2,4.536533E-3,1.0692011E-1,1.5913406E-2,-2.6327455E-3,1.2245688E-3,5.2073663E-3,4.0012702E-2,1.3121872E-1,4.1809496E-2,7.0301397E-3,7.624368E-2,1.1526459E-3,-1.2662338E-1,3.2327892E-3,-7.617263E-2,-4.1114476E-5,-4.2120614E-3,2.458413E-3,-9.431402E-4,-2.3647617E-3,-6.7651845E-3,-2.1264525E-2,3.266782E-3,2.0308178E-3,-1.5219009E-3,-6.8575954E-3,-8.124784E-2,1.862808E-5,1.319592E-1,6.750579E-2,7.871137E-3,8.38086E-4,4.308135E-3,3.1677843E-3,8.323019E-3,1.1964786E-2,3.3929597E-3,8.863961E-2,6.9045724E-5,-1.4481018E-1,-4.78835E-4,-2.9059453E-3,3.244578E-2,-5.766223E-3,-1.1129848E-3,-4.5741063E-2,9.578449E-4,-2.897477E-4,-5.906653E-3,-2.2751654E-2,-6.0806843E-3,7.307735E-3,1.7276135E-3,-1.7481922E-5,9.903697E-2,-6.351451E-2,1.8653069E-2,-1.6403464E-3,1.854359E-3,3.013373E-3,6.8379147E-3,-4.1314885E-3,-9.117624E-3,5.18E-3,7.723988E-5,-7.7845005E-4,-3.3960405E-3,-5.3273607E-4,2.3458803E-3,-2.8938593E-4,4.877667E-3,-2.8835372E-3,1.3580353E-3,5.1651024E-3,8.2982413E-4,-1.53484745E-2,-8.3737E-4,-7.1175564E-5,2.4650353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,31,-1,33,35,37,39,-1,41,-1,43,45,47,49,51,53,-1,-1,-1,55,57,59,-1,-1,61,63,-1,-1,65,-1,67,69,71,73,-1,75,-1,-1,77,79,-1,81,83,-1,-1,85,-1,87,89,91,93,95,-1,-1,-1,97,-1,99,101,-1,103,-1,105,107,-1,-1,-1,109,111,113,-1,115,-1,117,119,121,-1,-1,-1,-1,-1,-1,123,-1,-1,-1,125,127,-1,129,131,133,-1,-1,-1,-1,135,-1,137,-1,139,-1,-1,141,-1,-1,143,145,147,-1,149,-1,-1,-1,-1,151,153,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3267074E0,3.2089025E-1,8.074118E-1,2.1845156E-1,1.20483816E-1,7.167713E-1,2.708E-1,1.6560113E-1,7.962212E-2,5.9269696E-2,0E0,2.629674E-1,2.2201258E-1,2.6454866E-1,0E0,1.11516416E-1,0E0,7.706961E-2,8.2769796E-2,0E0,2.1909356E-2,2.4246335E-1,9.911105E-2,2.4952143E-2,0E0,9.664345E-2,0E0,1.6362786E-2,7.216802E-2,3.9972074E-2,6.261303E-2,1.2075242E-1,2.1548346E-2,0E0,0E0,0E0,1.2122238E-1,2.229071E-2,6.00605E-2,0E0,0E0,6.1769046E-2,1.09532654E-1,0E0,0E0,8.825824E-2,0E0,3.67631E-2,3.3234306E-2,6.536582E-2,4.0357567E-2,0E0,4.2639554E-2,0E0,0E0,7.527511E-2,1.6224712E-1,0E0,2.0660046E-2,4.157366E-2,0E0,0E0,4.899499E-2,0E0,3.0044079E-2,1.3500276E-1,5.592984E-2,3.3409826E-2,2.1748917E-2,0E0,0E0,0E0,5.1523507E-2,0E0,3.8973965E-2,2.2113571E-2,0E0,9.199871E-2,0E0,5.5955574E-2,1.260192E-1,0E0,0E0,0E0,1.926822E-2,1.8306315E-2,1.8946722E-2,0E0,2.581188E-2,0E0,5.9816033E-2,5.033967E-2,3.245856E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.4311194E-2,0E0,0E0,0E0,1.08038686E-1,5.2989602E-2,0E0,3.6236137E-2,8.779669E-2,2.0514865E-1,0E0,0E0,0E0,0E0,1.7474303E-2,0E0,2.3707837E-2,0E0,5.510235E-2,0E0,0E0,4.3948423E-2,0E0,0E0,2.2828713E-2,2.174061E-2,8.2883425E-2,0E0,2.2518221E-2,0E0,0E0,0E0,0E0,2.6967585E-2,4.2085797E-1,1.6196641E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,41,41,42,42,45,45,47,47,48,48,49,49,50,50,52,52,55,55,56,56,58,58,59,59,62,62,64,64,65,65,66,66,67,67,68,68,72,72,74,74,75,75,77,77,79,79,80,80,84,84,85,85,86,86,88,88,90,90,91,91,92,92,99,99,103,103,104,104,106,106,107,107,108,108,113,113,115,115,117,117,120,120,123,123,124,124,125,125,127,127,132,132,133,133,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,32,-1,34,36,38,40,-1,42,-1,44,46,48,50,52,54,-1,-1,-1,56,58,60,-1,-1,62,64,-1,-1,66,-1,68,70,72,74,-1,76,-1,-1,78,80,-1,82,84,-1,-1,86,-1,88,90,92,94,96,-1,-1,-1,98,-1,100,102,-1,104,-1,106,108,-1,-1,-1,110,112,114,-1,116,-1,118,120,122,-1,-1,-1,-1,-1,-1,124,-1,-1,-1,126,128,-1,130,132,134,-1,-1,-1,-1,136,-1,138,-1,140,-1,-1,142,-1,-1,144,146,148,-1,150,-1,-1,-1,-1,152,154,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.09E2,1.925673E1,1.5142261E3,8.147158E4,3.7377258E4,1E0,7.134432E7,1.2E1,2E0,1.375E0,-1.3943634E-2,4.5866325E6,3.7798166E0,6.5026445E3,-5.627696E-3,1.2571428E1,-1.2009375E-2,3.5364E4,1.89E2,-7.2058383E-4,2.5428572E0,2.04115E5,1.91E2,1.2E1,-1.478885E-2,1.5822886E0,1.1615753E-2,5.3E1,1.0428572E1,2.732381E2,1.5422421E2,1.43E2,1.162E3,-8.094936E-3,-3.8955284E-3,8.154739E-3,7.122041E7,2.0317461E0,3.201807E6,-1.111358E-3,-4.6237097E-3,1.1481482E0,2.5959E4,-3.4692339E-3,-7.952699E-3,7.7E1,2.5494979E-3,6.38E2,5.5177975E6,3.806E3,1.6379E2,-6.5425164E-3,3.4741783E0,-6.0565653E-3,-1.656692E-3,5E1,1.7039736E4,2.7411855E-3,3.6401E4,3.9914E4,7.5161792E-3,-1.8854549E-3,1.9514073E3,1.0140893E-2,1.6038339E0,1.2714286E1,2.5524476E0,4.878788E0,4.8064E4,4.7156163E-3,4.0735537E-4,5.032486E-4,1.703125E0,3.6263962E-3,3.7568388E0,2.1222334E2,-3.1366963E-3,9.6E1,4.536533E-3,1.2131076E2,3.3382E4,-2.6327455E-3,1.2245688E-3,5.2073663E-3,6.103178E8,1.599E3,1E0,7.0301397E-3,7.0993075E0,1.1526459E-3,2.1478824E2,5.4E1,9.876542E-2,-4.1114476E-5,-4.2120614E-3,2.458413E-3,-9.431402E-4,-2.3647617E-3,-6.7651845E-3,1.7594937E0,3.266782E-3,2.0308178E-3,-1.5219009E-3,2.511E3,1.2951E4,1.862808E-5,1.5E1,2.426015E-1,1E0,8.38086E-4,4.308135E-3,3.1677843E-3,8.323019E-3,7.3835544E5,3.3929597E-3,1.6327E4,6.9045724E-5,5.853E3,-4.78835E-4,-2.9059453E-3,9.2E1,-5.766223E-3,-1.1129848E-3,1.5357143E0,1E1,3.5E1,-5.906653E-3,1.2739792E1,-6.0806843E-3,7.307735E-3,1.7276135E-3,-1.7481922E-5,4.27E3,3.1157124E-1,1.1643481E6,-1.6403464E-3,1.854359E-3,3.013373E-3,6.8379147E-3,-4.1314885E-3,-9.117624E-3,5.18E-3,7.723988E-5,-7.7845005E-4,-3.3960405E-3,-5.3273607E-4,2.3458803E-3,-2.8938593E-4,4.877667E-3,-2.8835372E-3,1.3580353E-3,5.1651024E-3,8.2982413E-4,-1.53484745E-2,-8.3737E-4,-7.1175564E-5,2.4650353E-3],"split_indices":[2,73,67,43,48,83,59,8,32,68,0,43,68,4,0,70,0,9,10,0,68,5,0,3,0,53,0,44,68,4,67,2,10,0,0,0,7,69,9,0,0,68,9,0,0,44,0,44,60,1,67,0,68,0,0,3,62,0,9,9,0,0,70,0,57,67,68,73,44,0,0,0,68,0,68,4,0,8,0,48,9,0,0,0,7,10,28,0,50,0,4,2,53,0,0,0,0,0,0,71,0,0,0,2,44,0,3,53,8,0,0,0,0,43,0,2,0,9,0,0,44,0,0,68,8,3,0,71,0,0,0,0,2,53,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,3.31E2,6.85E2,3.04E2,2.7E1,5.89E2,9.6E1,9.6E1,2.08E2,2.3E1,4E0,5.52E2,3.7E1,9.2E1,4E0,9.2E1,4E0,1.6E2,4.8E1,5E0,1.8E1,4.9E2,6.2E1,3.3E1,4E0,7.9E1,1.3E1,1.5E1,7.7E1,5.1E1,1.09E2,3.1E1,1.7E1,1E1,8E0,8E0,4.82E2,1.5E1,4.7E1,6E0,2.7E1,3.4E1,4.5E1,5E0,1E1,7.2E1,5E0,3.6E1,1.5E1,3.9E1,7E1,6E0,2.5E1,1.3E1,4E0,1.58E2,3.24E2,5E0,1E1,3.4E1,1.3E1,5E0,2.9E1,1E1,3.5E1,3.5E1,3.7E1,1.8E1,1.8E1,8E0,7E0,1E1,2.9E1,4E0,6.6E1,2E1,5E0,1.53E2,5E0,2E1,3.04E2,5E0,5E0,1.7E1,1.7E1,9E0,2E1,8E0,2.7E1,7E0,2.8E1,2.4E1,1.3E1,1.3E1,5E0,1.3E1,5E0,2.1E1,8E0,6.2E1,4E0,1.6E1,4E0,1.35E2,1.8E1,4E0,1.6E1,4E1,2.64E2,1.3E1,4E0,5E0,4E0,1.1E1,9E0,2.3E1,4E0,2.4E1,4E0,7E0,1.7E1,6E0,7E0,2.9E1,3.3E1,1.29E2,6E0,9E0,9E0,1.2E1,4E0,1.3E1,2.7E1,3.4E1,2.3E2,4E0,7E0,1.8E1,5E0,1.3E1,1.1E1,4E0,1.3E1,1.5E1,1.4E1,2.7E1,6E0,1.23E2,6E0,5E0,4E0,2.3E1,4E0,4E0,3E1,1.46E2,8.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-2.3690518E-3,-4.6454236E-2,2.4270676E-2,-3.9487075E-2,-1.7790103E-1,1.2806815E-2,9.6692406E-2,-6.5399684E-2,-1.0367591E-2,-1.0324124E-2,-3.8157013E-1,2.2095663E-2,-4.8716985E-2,2.1900548E-1,7.713679E-2,-5.4802455E-2,-1.7812215E-1,-1.51273925E-2,8.280719E-2,-3.3877736E-3,2.086152E-3,-2.3489974E-2,-8.240744E-3,4.320459E-2,-9.208643E-3,-3.247114E-2,-1.4176326E-1,4.8005264E-3,1.323014E-2,9.144171E-2,-3.214184E-2,-1.17009215E-1,-4.7070682E-2,-1.5294679E-2,-6.460483E-2,-4.4045746E-3,-6.2572256E-2,-1.9115207E-5,6.90522E-3,1.4513594E-2,7.912101E-2,-3.743456E-2,5.928565E-3,-1.01075694E-1,-4.4577237E-4,-1.600472E-3,-9.148606E-3,1.02016166E-1,-3.1666807E-3,-4.510205E-3,2.4424011E-3,-1.4236003E-1,-5.767855E-4,-2.887059E-2,-7.6194726E-2,-3.0261488E-4,-4.4740615E-3,2.4722386E-2,-2.5758069E-2,-7.7478774E-2,6.9002045E-4,2.3605159E-2,-9.570162E-2,1.0650416E-1,3.300488E-2,-7.3764555E-2,-1.34646855E-2,1.05447076E-1,-4.812503E-3,-6.7789786E-2,-8.964123E-3,-3.090284E-2,8.842449E-2,1.1827256E-1,4.2806864E-2,-7.7682324E-3,-2.4023156E-3,-6.4046825E-3,-6.521758E-2,-1.3017711E-2,-5.7793178E-2,3.5074838E-2,-2.2289632E-3,2.9889024E-3,-3.2385856E-2,-1.9217745E-3,-5.2168015E-3,3.1509407E-2,-8.345172E-2,-1.2850322E-2,7.5271144E-4,7.9006106E-2,1.4786023E-1,-4.6273023E-2,6.134398E-2,-1.7573074E-3,-5.4621366E-3,-3.3920713E-2,3.7924357E-2,1.4922781E-1,2.5651226E-4,6.2303375E-3,-4.5400973E-2,-5.406925E-3,-3.0288946E-2,4.670073E-3,-4.595578E-3,7.395668E-4,6.3691614E-3,9.071165E-3,9.961424E-2,-4.0681256E-4,3.8774093E-3,-5.3135406E-2,1.0892414E-2,-7.025822E-3,-2.7026778E-2,-8.574363E-2,-2.4958434E-2,-1.1691692E-3,5.6429226E-2,-5.2172947E-3,-2.5322085E-2,4.2205244E-2,-3.798462E-2,-1.3420903E-3,-6.6029313E-3,3.759445E-2,1.22226916E-1,8.221263E-3,4.456291E-3,-5.069393E-3,6.9462217E-4,1.0558689E-1,1.6246017E-2,2.7053368E-3,-4.9223363E-2,-6.691448E-4,4.688343E-3,2.4992712E-3,9.910259E-3,-4.065815E-3,6.682679E-2,-4.6778736E-3,-6.907592E-4,9.347339E-4,-3.500155E-3,-1.4445928E-3,1.9964813E-3,5.2844435E-2,1.2996086E-1,-3.3647951E-3,-7.4026546E-5,5.5946827E-2,-6.9313142E-3,-4.1447632E-2,2.3322594E-3,1.15156574E-4,-1.0217599E-1,-3.503672E-2,1.4055921E-3,1.5535273E-3,-1.1627731E-3,5.110874E-3,4.086439E-2,-4.5741526E-3,-1.6006589E-2,4.6745043E-2,-3.5995934E-3,7.6965173E-3,-6.7384406E-3,6.0235422E-2,-1.2260028E-3,1.5047327E-1,-9.633956E-4,-3.4186582E-4,6.292773E-3,-1.760535E-3,5.5657398E-2,-8.316235E-2,9.3243754E-4,1.0042654E-2,-4.619761E-2,5.8167763E-3,3.131555E-4,1.087767E-4,7.070837E-2,8.7557705E-3,3.7370187E-3,6.297011E-4,4.4497224E-3,-2.2760548E-3,7.525329E-4,-3.0182349E-3,-2.3613703E-4,-6.422422E-3,-3.1489157E-3,-7.412458E-4,-4.9800836E-3,8.622268E-4,3.6478778E-3,-1.3873229E-3,3.78451E-4,3.99316E-3,1.4709192E-3,1.8237171E-3,-1.1570201E-3,4.405428E-3,9.0722676E-4,5.3884136E-3,9.9619245E-3,7.995641E-4,4.8918626E-3,-4.680241E-3,-1.2458394E-3,5.7625137E-5,4.3145483E-3,-3.5074928E-3,-2.730087E-4,4.252089E-3,6.424683E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,-1,-1,61,63,65,67,69,71,-1,-1,73,-1,-1,-1,75,-1,77,79,-1,-1,81,83,85,-1,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,-1,-1,113,115,-1,117,119,-1,-1,121,-1,-1,123,125,-1,-1,127,129,131,133,-1,-1,135,137,139,-1,141,143,-1,145,147,-1,-1,-1,-1,149,-1,-1,151,153,-1,155,157,159,161,163,-1,165,167,169,-1,-1,171,173,-1,-1,-1,-1,175,177,-1,179,-1,-1,-1,-1,181,183,-1,-1,-1,-1,-1,-1,185,187,-1,-1,189,191,193,-1,-1,195,197,-1,-1,-1,-1,199,-1,201,203,-1,205,-1,207,-1,209,-1,-1,-1,-1,211,213,-1,215,217,-1,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2565796E0,3.6204702E-1,5.5252683E-1,2.890737E-1,6.7866534E-1,3.3082342E-1,2.0077026E-1,2.3273116E-1,8.1968464E-2,4.5257922E-2,1.4134037E-1,3.321197E-1,1.10279575E-1,6.328726E-2,1.2799692E-1,8.443338E-2,2.7375138E-1,8.8245615E-2,4.9989052E-2,0E0,0E0,0E0,0E0,3.0799502E-1,8.695603E-2,1.4496261E-1,5.9435293E-2,0E0,0E0,1.2710792E-1,6.0827993E-2,5.121824E-2,8.6850375E-2,0E0,1.9953046E-2,8.957742E-2,3.8129225E-2,0E0,0E0,1.7059988E-1,1.6443783E-1,6.0835384E-2,1.4267583E-1,5.1823854E-2,1.2724204E-1,0E0,0E0,5.9426606E-2,0E0,0E0,0E0,2.8538644E-2,0E0,8.532194E-2,2.2582644E-1,0E0,0E0,4.777235E-2,5.185471E-2,2.9637948E-2,0E0,1.3463739E-1,2.7016523E-1,8.270675E-2,1.1741844E-1,3.9984316E-2,4.7539912E-2,5.602649E-2,5.451651E-2,2.784992E-2,0E0,8.58385E-2,4.0755257E-2,5.619371E-2,3.469716E-2,0E0,0E0,5.392696E-2,1.2857936E-1,0E0,5.2924722E-2,4.1860282E-2,0E0,0E0,4.1518524E-2,0E0,0E0,1.1012448E-1,3.1436995E-2,0E0,0E0,8.8476926E-2,3.2333255E-2,5.6115575E-2,7.41047E-2,0E0,0E0,4.8167996E-2,4.4771757E-2,4.4289827E-2,0E0,6.01671E-2,4.270429E-2,0E0,2.6804287E-2,3.459743E-2,0E0,0E0,0E0,0E0,5.558172E-2,0E0,0E0,1.8154286E-2,3.973663E-2,0E0,3.4507282E-2,4.6652466E-2,1.7252829E-2,1.8383233E-2,2.4161115E-2,0E0,4.852505E-2,7.289794E-2,9.9676974E-2,0E0,0E0,4.225469E-2,1.0417509E-1,0E0,0E0,0E0,0E0,6.674807E-2,4.4963993E-2,0E0,6.763481E-2,0E0,0E0,0E0,0E0,4.9800407E-2,4.8813827E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.697053E-2,6.0661912E-2,0E0,0E0,2.2603411E-2,3.61915E-2,2.1362014E-2,0E0,0E0,2.436024E-2,3.2816865E-2,0E0,0E0,0E0,0E0,2.2207748E-2,0E0,2.264145E-2,7.001647E-2,0E0,1.6595505E-2,0E0,2.8071135E-2,0E0,2.1151334E-2,0E0,0E0,0E0,0E0,2.1283206E-2,1.664035E-2,0E0,4.6066996E-2,2.4699166E-2,0E0,0E0,0E0,1.5682295E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,47,47,51,51,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,77,77,78,78,80,80,81,81,84,84,87,87,88,88,91,91,92,92,93,93,94,94,97,97,98,98,99,99,101,101,102,102,104,104,105,105,110,110,113,113,114,114,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,127,127,128,128,133,133,134,134,136,136,141,141,142,142,149,149,150,150,153,153,154,154,155,155,158,158,159,159,164,164,166,166,167,167,169,169,171,171,173,173,178,178,179,179,181,181,182,182,186,186],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,-1,-1,62,64,66,68,70,72,-1,-1,74,-1,-1,-1,76,-1,78,80,-1,-1,82,84,86,-1,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,-1,-1,114,116,-1,118,120,-1,-1,122,-1,-1,124,126,-1,-1,128,130,132,134,-1,-1,136,138,140,-1,142,144,-1,146,148,-1,-1,-1,-1,150,-1,-1,152,154,-1,156,158,160,162,164,-1,166,168,170,-1,-1,172,174,-1,-1,-1,-1,176,178,-1,180,-1,-1,-1,-1,182,184,-1,-1,-1,-1,-1,-1,186,188,-1,-1,190,192,194,-1,-1,196,198,-1,-1,-1,-1,200,-1,202,204,-1,206,-1,208,-1,210,-1,-1,-1,-1,212,214,-1,216,218,-1,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.01E2,9.639872E0,1.6466942E3,1.5422421E2,3.89E2,1E0,8.025477E0,6.9E2,1.5512196E1,1.317484E1,1.2916E4,8E0,2.614094E0,1.400229E6,3.206931E2,7.785714E0,1.5077367E3,8E0,6E0,-3.3877736E-3,2.086152E-3,-2.3489974E-2,-8.240744E-3,4.716184E2,6.953366E7,1.4598765E0,3.3512E4,4.8005264E-3,1.323014E-2,1.625E0,4.4615383E0,5.0363636E1,3.53969E5,-1.5294679E-2,3.8795E4,1.4392524E0,7.31E2,-1.9115207E-5,6.90522E-3,4.5620965E6,1.6485592E0,8.315271E0,3.168978E-2,2.060658E5,2.10377E5,-1.600472E-3,-9.148606E-3,1.3948464E7,-3.1666807E-3,-4.510205E-3,2.4424011E-3,1.4806053E4,-5.767855E-4,3.5318289E3,1.76421E6,-3.0261488E-4,-4.4740615E-3,3.0923106E-10,1.556431E6,5.524E3,6.9002045E-4,1.2300693E0,1.065548E0,1.0378262E10,1E0,6.185E3,1.935E3,5.1212654E0,5.1034E4,1.7851852E1,-8.964123E-3,8.625455E2,1.1772152E0,1.3208092E0,9.837502E1,-7.7682324E-3,-2.4023156E-3,1.0891155E4,1.2139681E3,-1.3017711E-2,2.240836E6,5.14E2,-2.2289632E-3,2.9889024E-3,1.952653E2,-1.9217745E-3,-5.2168015E-3,1E0,1.7544615E0,-1.2850322E-2,7.5271144E-4,6.4610266E2,1.6521566E5,6.9508715E0,4.7E3,-1.7573074E-3,-5.4621366E-3,1.06E2,6.0147805E0,2.6043478E2,2.5651226E-4,5.8165E4,1.9505818E8,-5.406925E-3,3.6451373E8,1.317E3,-4.595578E-3,7.395668E-4,6.3691614E-3,9.071165E-3,2.337765E3,-4.0681256E-4,3.8774093E-3,3.2E1,1.870878E6,-7.025822E-3,6E0,3.466E3,2.022131E6,8.6875E0,1.6877083E2,-5.2172947E-3,2.9726138E5,5.857143E0,3.3272727E0,-1.3420903E-3,-6.6029313E-3,2.297E3,1.4523518E8,8.221263E-3,4.456291E-3,-5.069393E-3,6.9462217E-4,4E0,9.812221E9,2.7053368E-3,2.558E3,-6.691448E-4,4.688343E-3,2.4992712E-3,9.910259E-3,6.627033E7,7.973882E7,-4.6778736E-3,-6.907592E-4,9.347339E-4,-3.500155E-3,-1.4445928E-3,1.9964813E-3,9.643839E6,1.4676277E7,-3.3647951E-3,-7.4026546E-5,5.4E1,1.43E2,5.6516E4,2.3322594E-3,1.15156574E-4,1.178E4,1.3935602E1,1.4055921E-3,1.5535273E-3,-1.1627731E-3,5.110874E-3,1.4E1,-4.5741526E-3,1.084437E6,3.0989232E-3,-3.5995934E-3,4.0903897E2,-6.7384406E-3,1.2526851E6,-1.2260028E-3,2.4545455E0,-9.633956E-4,-3.4186582E-4,6.292773E-3,-1.760535E-3,1.6438356E1,2.0130434E1,9.3243754E-4,6.63651E4,3.8104E4,5.8167763E-3,3.131555E-4,1.087767E-4,1.1863237E2,8.7557705E-3,3.7370187E-3,6.297011E-4,4.4497224E-3,-2.2760548E-3,7.525329E-4,-3.0182349E-3,-2.3613703E-4,-6.422422E-3,-3.1489157E-3,-7.412458E-4,-4.9800836E-3,8.622268E-4,3.6478778E-3,-1.3873229E-3,3.78451E-4,3.99316E-3,1.4709192E-3,1.8237171E-3,-1.1570201E-3,4.405428E-3,9.0722676E-4,5.3884136E-3,9.9619245E-3,7.995641E-4,4.8918626E-3,-4.680241E-3,-1.2458394E-3,5.7625137E-5,4.3145483E-3,-3.5074928E-3,-2.730087E-4,4.252089E-3,6.424683E-4],"split_indices":[2,69,67,67,2,6,71,2,71,69,1,32,68,1,73,67,48,32,8,0,0,0,0,70,7,69,1,0,0,61,69,62,9,0,1,68,2,0,0,62,53,71,53,48,1,0,0,9,0,0,0,43,0,62,46,0,0,52,5,44,0,53,53,46,27,44,2,68,44,73,0,67,68,68,73,0,0,43,48,0,9,2,0,0,70,0,0,19,68,0,0,70,48,71,2,0,0,0,69,67,0,10,5,0,7,2,0,0,0,0,67,0,0,4,46,0,8,1,12,73,67,0,43,68,69,0,0,2,12,0,0,0,0,3,46,0,10,0,0,0,0,12,47,0,0,0,0,0,0,62,62,0,0,2,2,9,0,0,1,73,0,0,0,0,3,0,9,53,0,67,0,43,0,68,0,0,0,0,71,71,0,48,10,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.068E3,4.02E2,6.66E2,3.83E2,1.9E1,5.76E2,9E1,2.02E2,1.81E2,1.1E1,8E0,5.01E2,7.5E1,1.1E1,7.9E1,1.86E2,1.6E1,1.73E2,8E0,5E0,6E0,4E0,4E0,2.99E2,2.02E2,6.5E1,1E1,5E0,6E0,7E1,9E0,1.9E1,1.67E2,6E0,1E1,1.42E2,3.1E1,4E0,4E0,1.67E2,1.32E2,7E1,1.32E2,2E1,4.5E1,4E0,6E0,6.6E1,4E0,5E0,4E0,1.5E1,4E0,1.04E2,6.3E1,4E0,6E0,6E1,8.2E1,2.6E1,5E0,1.55E2,1.2E1,8.2E1,5E1,2.7E1,4.3E1,1.2E1,1.2E2,1.6E1,4E0,3.4E1,1.1E1,5.1E1,1.5E1,1.1E1,4E0,6.5E1,3.9E1,4E0,5.9E1,5.3E1,7E0,5E0,7.7E1,1.4E1,1.2E1,1.45E2,1E1,4E0,8E0,5.1E1,3.1E1,1.3E1,3.7E1,1.6E1,1.1E1,3.1E1,1.2E1,8E0,4E0,9.5E1,2.5E1,6E0,1E1,2.3E1,1.1E1,5E0,6E0,8E0,4.3E1,7E0,8E0,1.7E1,4.8E1,1.1E1,2.8E1,3.1E1,2.8E1,2E1,3.3E1,5E0,7.2E1,1.26E2,1.9E1,6E0,4E0,2.7E1,2.4E1,1.8E1,1.3E1,6E0,7E0,1.8E1,1.9E1,4E0,2.7E1,7E0,5E0,4E0,4E0,8.2E1,1.3E1,8E0,1.7E1,5E0,5E0,1.2E1,1.1E1,1.8E1,2.5E1,1.2E1,5E0,1.3E1,3.5E1,2.4E1,4E0,5E0,2.6E1,2.4E1,4E0,8E0,1.2E1,6E0,2.7E1,7E0,6.5E1,1.22E2,4E0,1.4E1,5E0,2E1,7E0,2E1,4E0,4E0,1.4E1,8E0,1.1E1,1.8E1,9E0,6.2E1,2E1,6E0,7E0,5E0,1.3E1,1E1,1.5E1,7E0,6E0,1.2E1,2.3E1,1.4E1,1E1,1.1E1,1.5E1,2E1,4E0,1.8E1,9E0,4.1E1,2.4E1,3.2E1,9E1,7E0,7E0,1E1,1E1,1.5E1,5E0,7E0,4E0,1.3E1,5E0,5.7E1,5E0,1.1E1,9E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[3.1734907E-4,-2.5441412E-2,4.4444777E-2,-1.930243E-2,-1.1689011E-1,5.1820323E-2,-8.1651054E-2,-5.8866374E-2,-8.555098E-3,-2.7244818E-1,-2.264929E-2,3.028332E-2,8.010094E-2,-1.4328314E-1,-2.3486719E-2,-4.4790212E-2,-1.3263816E-1,-1.4765708E-2,3.70305E-2,6.172962E-4,-3.7759373E-1,-4.6482105E-2,1.6994231E-3,3.5040934E-2,-7.584718E-3,8.7630995E-2,-5.4392726E-3,-8.991346E-3,-2.4494752E-3,1.3180748E-3,-3.707882E-3,-6.523228E-3,-3.1407785E-2,-9.862959E-3,-9.145552E-2,-2.7298667E-2,1.1616464E-2,5.6367002E-2,-1.1782863E-2,-2.0760357E-2,-9.313337E-3,-4.9811127E-3,-1.7176581E-3,7.7325464E-3,6.706582E-2,1.6933034E-1,7.343165E-2,4.8997873E-3,-3.846071E-2,-2.1971301E-3,-5.725041E-3,-1.9333543E-2,-7.130554E-2,1.8790262E-2,-9.090453E-2,8.776974E-2,2.5256924E-2,1.4223348E-3,-5.5488846E-3,2.6358298E-3,-1.7664249E-3,2.3178311E-2,-1.1952087E-2,1.0394326E-2,5.3271286E-2,8.88285E-3,3.6925517E-3,6.51994E-2,1.0560282E-2,-4.358769E-2,1.4208389E-3,-2.5222566E-2,5.9746668E-2,-9.945379E-3,-5.1019914E-2,2.466904E-2,-3.960379E-2,-7.2662435E-3,-2.9322098E-4,5.6587663E-3,1.8851146E-3,2.463497E-3,-1.37553755E-2,7.4817287E-3,1.4445846E-2,4.443088E-3,7.0251875E-2,1.3766012E-1,5.296325E-2,-5.094637E-2,1.1885586E-2,-3.6514066E-3,-5.364996E-2,7.324804E-3,7.9231156E-4,1.415675E-3,-8.419614E-2,-1.2184613E-3,3.587004E-2,1.4462165E-3,-4.3761693E-3,-2.8119825E-3,2.1034083E-3,2.6040884E-2,-7.082199E-2,1.4593147E-3,-2.8696032E-3,7.726795E-2,-1.2283513E-3,4.527957E-3,9.850517E-3,-7.6740836E-3,6.893581E-2,-3.321744E-2,-8.0785654E-2,2.3800759E-3,-2.12408E-3,-3.5158604E-2,1.4836361E-2,-6.842919E-2,-5.845265E-3,-1.14167064E-1,-8.700106E-4,1.5398306E-2,-4.5961887E-2,6.464244E-2,1.8580774E-2,-4.6848338E-2,3.5051797E-2,-1.5942835E-3,-5.301727E-3,5.421723E-2,1.0862454E-1,5.122702E-3,-3.846949E-2,7.789783E-2,-2.7975119E-3,-4.4645816E-3,-1.0336055E-3,-4.408084E-3,-5.1937375E-4,-3.1197525E-3,-7.035668E-4,-9.4183546E-4,1.5935844E-3,-7.452188E-4,-3.982482E-3,-1.4736438E-3,1.5623561E-3,-1.8472258E-3,-6.0651773E-3,-5.055231E-4,2.2746518E-3,-4.2294003E-3,8.284892E-4,8.9585455E-4,3.8397154E-3,-6.036367E-4,1.4129496E-3,1.7207733E-4,-3.7506458E-3,2.0657287E-3,-1.007813E-3,1.0732767E-3,3.7291192E-3,6.259252E-3,1.1105543E-3,1.8018075E-3,-2.8517907E-3,4.9809343E-3,2.2626228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,47,-1,49,51,53,55,57,-1,-1,-1,59,61,63,65,67,-1,69,-1,-1,71,73,75,77,79,81,-1,-1,-1,-1,83,-1,-1,85,-1,-1,87,-1,89,-1,91,93,-1,95,97,99,-1,-1,-1,-1,-1,101,-1,103,105,107,109,111,113,115,117,119,-1,-1,-1,121,123,125,-1,-1,-1,-1,127,129,-1,-1,131,-1,-1,-1,133,135,137,139,-1,-1,141,143,145,147,149,-1,151,153,155,157,159,161,-1,-1,163,165,-1,167,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1503019E0,3.564796E-1,3.5113066E-1,2.546205E-1,5.8022606E-1,2.1257943E-1,7.191533E-2,1.2697732E-1,1.3445985E-1,4.5581985E-1,3.853836E-2,1.9263433E-1,2.3675478E-1,3.6021084E-2,3.7727233E-2,1.3619429E-1,4.706779E-2,1.3819677E-1,5.446531E-2,0E0,4.8367143E-2,5.1982917E-2,0E0,1.7264192E-1,0E0,1.5809727E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.812418E-2,0E0,1.548405E-2,9.781864E-2,1.00995876E-1,3.7837803E-2,8.209168E-2,0E0,0E0,0E0,2.7935635E-2,4.5045808E-1,1.8625724E-1,2.77071E-2,1.4730597E-1,0E0,3.4343094E-2,0E0,0E0,1.13741904E-1,1.145965E-1,4.4574272E-2,4.9515665E-2,2.7612E-2,2.482224E-2,0E0,0E0,0E0,0E0,1.22473866E-1,0E0,0E0,7.016404E-2,0E0,0E0,1.0087919E-1,0E0,3.611979E-2,0E0,1.3777676E-1,6.882788E-2,0E0,1.0824139E-1,3.367976E-2,4.98669E-2,0E0,0E0,0E0,0E0,0E0,3.0700976E-2,0E0,9.8735265E-2,4.3515056E-2,4.4859022E-2,3.003341E-2,1.0319358E-1,3.805095E-2,2.7593968E-2,7.566383E-2,6.81321E-2,0E0,0E0,0E0,5.270286E-2,2.7473414E-2,3.9092645E-2,0E0,0E0,0E0,0E0,5.8783587E-2,1.5185334E-2,0E0,0E0,3.6755413E-2,0E0,0E0,0E0,8.8239126E-2,1.02197796E-1,3.2226328E-2,2.741395E-2,0E0,0E0,2.9937278E-2,5.747947E-2,6.7394614E-2,2.587847E-2,1.8876106E-2,0E0,2.4603913E-2,3.0768512E-2,2.3462072E-2,2.015243E-2,1.7667213E-2,4.3831363E-2,0E0,0E0,2.7686536E-2,5.1270485E-2,0E0,3.664307E-2,6.307554E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,25,25,32,32,34,34,35,35,36,36,37,37,38,38,42,42,43,43,44,44,45,45,46,46,48,48,51,51,52,52,53,53,54,54,55,55,56,56,61,61,64,64,67,67,69,69,71,71,72,72,74,74,75,75,76,76,82,82,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,96,96,97,97,98,98,103,103,104,104,107,107,111,111,112,112,113,113,114,114,117,117,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,127,127,128,128,131,131,132,132,134,134,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,48,-1,50,52,54,56,58,-1,-1,-1,60,62,64,66,68,-1,70,-1,-1,72,74,76,78,80,82,-1,-1,-1,-1,84,-1,-1,86,-1,-1,88,-1,90,-1,92,94,-1,96,98,100,-1,-1,-1,-1,-1,102,-1,104,106,108,110,112,114,116,118,120,-1,-1,-1,122,124,126,-1,-1,-1,-1,128,130,-1,-1,132,-1,-1,-1,134,136,138,140,-1,-1,142,144,146,148,150,-1,152,154,156,158,160,162,-1,-1,164,166,-1,168,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.82E3,9.639872E0,4.217427E7,8.147158E4,2E0,1.2456025E3,1.2376862E10,5.17259E8,3.422592E6,8.317E3,1.6254545E1,1.5449402E7,4.1325716E7,4.5576923E1,2.5427E4,1.7857143E1,6.15E2,3.422351E6,1E0,6.172962E-4,5.75E2,6.76E2,1.6994231E-3,1.8062708E6,-7.584718E-3,9.318287E0,-5.4392726E-3,-8.991346E-3,-2.4494752E-3,1.3180748E-3,-3.707882E-3,-6.523228E-3,6E0,-9.862959E-3,1.5757076E9,2.87E2,1.6438356E1,2.1111E4,6.2521395E6,-2.0760357E-2,-9.313337E-3,-4.9811127E-3,3.37231E7,1.68991E6,1.1057693E0,5.1578946E0,8.363565E0,4.8997873E-3,1.3745962E0,-2.1971301E-3,-5.725041E-3,1.187E3,5.1942E4,5.4444447E0,1.23E2,1.67227E5,2.0487332E-3,1.4223348E-3,-5.5488846E-3,2.6358298E-3,-1.7664249E-3,4E0,-1.1952087E-2,1.0394326E-2,2.483E3,8.88285E-3,3.6925517E-3,1.524E3,1.0560282E-2,7.163082E7,1.4208389E-3,1E0,6.904904E3,-9.945379E-3,6E0,8.8790035E-1,5E0,-7.2662435E-3,-2.9322098E-4,5.6587663E-3,1.8851146E-3,2.463497E-3,1.3478261E1,7.4817287E-3,2.2E1,2.6844707E8,8.82E4,7.008608E7,1.0901037E10,2.04115E5,6.03E2,4.33E2,4E0,7.324804E-3,7.9231156E-4,1.415675E-3,1.034073E6,2.6950342E2,1E0,1.4462165E-3,-4.3761693E-3,-2.8119825E-3,2.1034083E-3,2.29E2,5.943433E4,1.4593147E-3,-2.8696032E-3,2.01E0,-1.2283513E-3,4.527957E-3,9.850517E-3,1.3358E4,1.514526E6,4.91271E5,3.257732E0,2.3800759E-3,-2.12408E-3,1.1430505E3,3.3832976E7,3.0052083E0,9E0,1.027972E0,-8.700106E-4,7.09E2,1.6153846E0,1.7733E4,2.9417648E2,2.6845297E5,2.831224E10,-1.5942835E-3,-5.301727E-3,6.9664386E2,5.8475E5,5.122702E-3,7.5456814E-4,9.4163445E1,-2.7975119E-3,-4.4645816E-3,-1.0336055E-3,-4.408084E-3,-5.1937375E-4,-3.1197525E-3,-7.035668E-4,-9.4183546E-4,1.5935844E-3,-7.452188E-4,-3.982482E-3,-1.4736438E-3,1.5623561E-3,-1.8472258E-3,-6.0651773E-3,-5.055231E-4,2.2746518E-3,-4.2294003E-3,8.284892E-4,8.9585455E-4,3.8397154E-3,-6.036367E-4,1.4129496E-3,1.7207733E-4,-3.7506458E-3,2.0657287E-3,-1.007813E-3,1.0732767E-3,3.7291192E-3,6.259252E-3,1.1105543E-3,1.8018075E-3,-2.8517907E-3,4.9809343E-3,2.2626228E-3],"split_indices":[2,69,60,43,32,70,46,46,43,9,69,62,66,73,9,4,2,9,30,0,2,2,0,43,0,71,0,0,0,0,0,0,3,0,46,0,71,9,43,0,0,0,5,43,69,69,50,0,53,0,0,10,1,69,0,1,53,0,0,0,0,73,0,0,2,0,0,44,0,7,0,26,48,0,8,71,8,0,0,0,0,0,73,0,3,7,10,5,46,5,10,2,6,0,0,0,9,67,26,0,0,0,0,10,48,0,0,68,0,0,0,9,10,46,68,0,0,48,7,69,8,68,0,2,68,1,4,43,46,0,0,67,1,0,53,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,6.38E2,3.72E2,5.99E2,3.9E1,3.52E2,2E1,1.27E2,4.72E2,1.4E1,2.5E1,2.01E2,1.51E2,9E0,1.1E1,1.08E2,1.9E1,4.16E2,5.6E1,4E0,1E1,1.8E1,7E0,1.97E2,4E0,1.46E2,5E0,5E0,4E0,6E0,5E0,1.2E1,9.6E1,5E0,1.4E1,2.82E2,1.34E2,4E1,1.6E1,6E0,4E0,7E0,1.1E1,1.07E2,9E1,2E1,1.26E2,4E0,9.2E1,7E0,7E0,2.4E2,4.2E1,1.26E2,8E0,1.9E1,2.1E1,1.2E1,4E0,4E0,7E0,1.02E2,5E0,6E0,8.4E1,1.5E1,5E0,1.21E2,5E0,8.6E1,6E0,2.24E2,1.6E1,4E0,3.8E1,1.15E2,1.1E1,4E0,4E0,1E1,9E0,1.2E1,9E0,5E0,9.7E1,2.2E1,6.2E1,1.6E1,1.05E2,7.6E1,1E1,1.28E2,9.6E1,4E0,1.2E1,1.1E1,2.7E1,3.5E1,8E1,5E0,6E0,5E0,4E0,8.6E1,1.1E1,1.6E1,6E0,5.8E1,4E0,1.2E1,4E0,2.2E1,8.3E1,4.9E1,2.7E1,6E0,4E0,4.7E1,8.1E1,7.3E1,2.3E1,1.8E1,9E0,2.6E1,9E0,2.9E1,5.1E1,9E0,7.7E1,7E0,4E0,3.5E1,2.3E1,4E0,1.8E1,7.8E1,5E0,6E0,4.3E1,2.2E1,5E0,1.7E1,3E1,2.9E1,5.2E1,1.9E1,5.4E1,1.4E1,9E0,4E0,1.4E1,1.5E1,1.1E1,5E0,4E0,9E0,2E1,1.4E1,3.7E1,4E0,5E0,6.6E1,1.1E1,1.7E1,1.8E1,1.7E1,6E0,4E0,1.4E1,3.7E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[2.8358696E-3,-2.1731937E-2,4.7262184E-2,-1.914325E-2,-1.0804873E-2,-1.07442856E-1,5.4537464E-2,-4.850239E-2,-4.782098E-3,-1.0055192E-2,-5.679092E-2,1.2139469E-2,4.8979595E-2,-4.3503724E-2,-1.5974875E-1,-3.4631126E-2,4.1432115E-3,2.755532E-4,-4.4701556E-3,3.7976187E-2,9.047904E-2,-3.45927E-2,-1.11122794E-1,-1.1393424E-2,-1.8910074E-3,-2.544626E-2,-8.264027E-3,1.8162321E-3,5.3703445E-3,6.246678E-2,1.574164E-2,1.1415972E-1,-6.775713E-2,-1.3162982E-1,-2.8366704E-2,-1.2663113E-2,-4.1569676E-2,2.0889657E-2,-4.246881E-2,-1.015258E-3,4.3923184E-3,1.04059055E-1,3.7159696E-2,-8.334721E-3,2.2537494E-2,1.2962312E-1,3.7638366E-2,-7.364217E-3,2.576431E-3,-8.075818E-3,-2.0724342E-3,6.308548E-2,-3.364144E-2,-8.0281794E-2,1.4198109E-3,-1.0233842E-2,1.9831716E-3,-6.898858E-2,-3.0415172E-3,-9.7341975E-3,3.1508435E-2,7.90356E-2,8.067667E-3,4.8366394E-2,-6.035828E-2,5.7871625E-2,-4.7866236E-3,9.6582316E-2,1.7765927E-1,-5.571634E-4,5.0647175E-3,7.6603927E-3,-1.5261787E-3,-5.3869475E-2,-1.464016E-2,-5.2134483E-3,-1.3355284E-3,-2.9182574E-3,1.6908598E-3,-4.7223542E-2,-1.0392852E-2,1.789234E-2,-4.146781E-3,-6.584291E-3,-5.6866505E-3,-2.8734442E-2,4.7082268E-2,9.949901E-2,1.976186E-2,1.5820231E-2,7.8003354E-2,-7.739879E-3,2.7205874E-3,1.496291E-2,1.1670819E-1,4.5652285E-2,-5.2709714E-2,4.4692498E-2,1.2049575E-1,9.118442E-3,3.6517426E-3,-4.1236207E-2,-1.503538E-1,8.923282E-2,-2.6142867E-2,-5.873108E-4,-2.8685401E-3,3.1307526E-3,-1.987494E-2,-5.7339385E-2,-1.0628119E-3,1.4283924E-3,-2.8482913E-3,3.0428344E-2,5.573799E-3,1.1780362E-1,1.2823052E-3,2.5670915E-3,-1.585374E-3,-1.5597975E-2,5.80315E-2,5.9674694E-3,4.745268E-2,-4.6728486E-3,4.6099126E-3,1.4337245E-1,1.7921563E-3,2.496411E-2,7.332925E-3,-3.266591E-2,-8.985747E-3,2.9854354E-4,3.7812009E-3,1.3690108E-1,1.7249534E-3,-1.3511529E-3,-4.6082414E-3,-9.808591E-3,-2.6945747E-3,6.3687614E-3,1.0516869E-3,-3.0566845E-3,-3.6080249E-4,1.376148E-3,-3.0977794E-3,-9.797035E-4,-9.130371E-3,-2.3079491E-3,2.611778E-4,-1.131352E-3,1.8780392E-3,1.5895964E-3,6.2102866E-3,-1.4444111E-3,2.0291891E-3,5.431144E-3,9.371942E-4,1.0213414E-3,6.905895E-3,-1.8031822E-3,1.8403545E-3,4.0617585E-3,9.6385805E-3,-8.982889E-4,2.262778E-3,1.116433E-3,-2.138018E-3,7.4168895E-3,2.904172E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,-1,39,-1,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,-1,65,67,69,-1,-1,-1,-1,71,73,75,-1,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,-1,-1,101,103,-1,-1,-1,-1,105,-1,107,-1,109,-1,111,113,115,117,119,121,-1,-1,123,125,127,129,131,133,-1,-1,135,137,139,141,-1,-1,-1,143,145,147,-1,-1,149,-1,151,-1,-1,-1,153,155,-1,157,159,-1,161,-1,163,-1,165,-1,-1,-1,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1372341E0,3.6402002E-1,4.2342842E-1,2.7973107E-1,0E0,8.25364E-2,3.979709E-1,1.1427647E-1,1.1922956E-1,0E0,3.3183564E-2,0E0,1.5501636E-1,1.22692555E-1,8.279735E-2,1.3191229E-1,8.978397E-2,0E0,0E0,1.4930898E-1,2.775293E-1,1.0924263E-1,2.6065782E-1,0E0,0E0,7.8185186E-2,0E0,8.9612134E-2,0E0,1.3402104E-1,1.9868553E-1,6.9667816E-2,1.22093916E-1,3.336823E-2,8.750458E-2,0E0,5.2848704E-2,1.8957775E-2,7.507357E-2,9.38855E-2,0E0,7.272667E-2,9.362622E-2,0E0,1.3737431E-1,6.537312E-2,4.4255488E-2,0E0,0E0,0E0,0E0,1.04219414E-1,6.37836E-2,1.6814038E-2,0E0,3.206031E-2,0E0,1.3534802E-1,5.6065947E-2,9.238241E-2,6.702777E-2,4.4125527E-2,0E0,7.1108654E-2,1.2488814E-1,1.5477902E-1,1.9819859E-1,3.556326E-2,2.3280442E-2,0E0,0E0,0E0,0E0,9.2531875E-2,1.0747701E-1,0E0,0E0,0E0,0E0,1.9221015E-2,0E0,4.8437238E-2,0E0,7.1401075E-2,0E0,3.136102E-2,6.493725E-2,3.350061E-2,2.3205133E-2,4.997466E-2,5.483286E-2,0E0,0E0,6.2090542E-2,4.845524E-2,8.960797E-2,1.13011725E-1,1.6515747E-2,2.4901718E-2,0E0,0E0,4.8295423E-2,3.893374E-2,2.5948353E-2,5.8179356E-2,0E0,0E0,0E0,3.7141807E-2,1.2253891E-1,7.613755E-2,0E0,0E0,2.7381256E-2,0E0,2.58694E-2,0E0,0E0,0E0,2.188846E-2,3.3565894E-2,0E0,6.3929886E-2,4.89602E-2,0E0,5.0415844E-2,0E0,3.827802E-2,0E0,3.0515894E-2,0E0,0E0,0E0,2.141428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,27,27,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,51,51,52,52,53,53,55,55,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,73,73,74,74,79,79,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,104,104,108,108,109,109,110,110,113,113,115,115,119,119,120,120,122,122,123,123,125,125,127,127,129,129,133,133],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,-1,40,-1,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,-1,66,68,70,-1,-1,-1,-1,72,74,76,-1,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,-1,-1,102,104,-1,-1,-1,-1,106,-1,108,-1,110,-1,112,114,116,118,120,122,-1,-1,124,126,128,130,132,134,-1,-1,136,138,140,142,-1,-1,-1,144,146,148,-1,-1,150,-1,152,-1,-1,-1,154,156,-1,158,160,-1,162,-1,164,-1,166,-1,-1,-1,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,5.8E1,1E0,1.972052E5,-1.0804873E-2,1.6427984E1,3.3817584E7,3.0977E4,1E0,-1.0055192E-2,1E1,1.2139469E-2,6.7947706E8,4.6463413E0,3.3064186E7,3.7568388E0,3.7E1,2.755532E-4,-4.4701556E-3,1.1736916E0,3.206931E2,6E0,2E0,-1.1393424E-2,-1.8910074E-3,1E0,-8.264027E-3,6.663214E6,5.3703445E-3,6E0,1.0280637E0,7.3274844E9,6.9879E4,1.4806053E4,6E0,-1.2663113E-2,1.0428572E1,1.1E1,5E0,9.751103E6,4.3923184E-3,7.8114265E-1,2.3308511E5,-8.334721E-3,2.67637E5,1.3E1,1.2567214E7,-7.364217E-3,2.576431E-3,-8.075818E-3,-2.0724342E-3,1.3050649E4,2.135961E4,8.579633E-2,1.4198109E-3,7.54E2,1.9831716E-3,1.339646E6,2E0,9.750042E6,8.0509944E5,2.936348E10,8.067667E-3,1.3026532E6,9.525663E2,1.3758875E3,4.519E3,1.4176E4,2.8389828E7,-5.571634E-4,5.0647175E-3,7.6603927E-3,-1.5261787E-3,1.965084E4,1.813506E-5,-5.2134483E-3,-1.3355284E-3,-2.9182574E-3,1.6908598E-3,1.4452E4,-1.0392852E-2,6.346204E-7,-4.146781E-3,1.5933333E2,-5.6866505E-3,1.6121496E1,1E0,6.441311E-1,1E0,5.691698E2,2.0778275E-1,-7.739879E-3,2.7205874E-3,3.13173E0,1.016E4,6.2868685E6,2.8146256E5,1.1223777E7,5.0554064E7,9.118442E-3,3.6517426E-3,3.2856784E7,2E0,6.0921145E4,1.9595902E3,-5.873108E-4,-2.8685401E-3,3.1307526E-3,6.76E2,1.5222145E2,3.202146E5,1.4283924E-3,-2.8482913E-3,5.55E2,5.573799E-3,3.7253174E2,1.2823052E-3,2.5670915E-3,-1.585374E-3,6.4356956E0,2.0421052E0,5.9674694E-3,6.2850784E1,4.1032645E6,4.6099126E-3,5E0,1.7921563E-3,7.0093E4,7.332925E-3,8E0,-8.985747E-3,2.9854354E-4,3.7812009E-3,1E0,1.7249534E-3,-1.3511529E-3,-4.6082414E-3,-9.808591E-3,-2.6945747E-3,6.3687614E-3,1.0516869E-3,-3.0566845E-3,-3.6080249E-4,1.376148E-3,-3.0977794E-3,-9.797035E-4,-9.130371E-3,-2.3079491E-3,2.611778E-4,-1.131352E-3,1.8780392E-3,1.5895964E-3,6.2102866E-3,-1.4444111E-3,2.0291891E-3,5.431144E-3,9.371942E-4,1.0213414E-3,6.905895E-3,-1.8031822E-3,1.8403545E-3,4.0617585E-3,9.6385805E-3,-8.982889E-4,2.262778E-3,1.116433E-3,-2.138018E-3,7.4168895E-3,2.904172E-3],"split_indices":[2,3,17,43,0,73,7,44,29,0,6,0,7,68,5,68,3,0,0,53,73,67,32,0,0,68,0,43,0,8,69,5,2,43,3,0,68,3,8,9,0,53,48,0,9,3,58,0,0,0,0,43,43,53,0,2,0,9,32,9,60,46,0,43,4,70,44,2,62,0,0,0,0,43,53,0,0,0,0,9,0,52,0,4,0,73,28,53,8,67,57,0,0,53,2,60,48,59,59,0,0,60,10,60,48,0,0,0,2,4,43,0,0,44,0,4,0,0,0,69,68,0,73,60,0,8,0,1,0,3,0,0,0,19,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,6.7E2,3.7E2,6.63E2,7E0,1.6E1,3.54E2,2.17E2,4.46E2,4E0,1.2E1,8E0,3.46E2,2.09E2,8E0,1.02E2,3.44E2,5E0,7E0,2.75E2,7.1E1,1.86E2,2.3E1,4E0,4E0,9.7E1,5E0,3.38E2,6E0,1.3E2,1.45E2,6.2E1,9E0,1E1,1.76E2,6E0,1.7E1,2.6E1,7.1E1,3.29E2,9E0,4.8E1,8.2E1,4E0,1.41E2,5.1E1,1.1E1,5E0,4E0,6E0,4E0,9E0,1.67E2,1.1E1,6E0,1.1E1,1.5E1,4.2E1,2.9E1,2.6E2,6.9E1,3.7E1,1.1E1,7.4E1,8E0,6.1E1,8E1,3.2E1,1.9E1,7E0,4E0,4E0,5E0,8E1,8.7E1,6E0,5E0,5E0,6E0,3.8E1,4E0,2.4E1,5E0,2.54E2,6E0,1.4E1,5.5E1,2.7E1,1E1,3.6E1,3.8E1,4E0,4E0,3.6E1,2.5E1,3.9E1,4.1E1,1.1E1,2.1E1,1.5E1,4E0,7.2E1,8E0,8E0,7.9E1,1.2E1,2.6E1,1E1,1.4E1,2.4E1,2.3E2,5E0,9E0,4.6E1,9E0,2.1E1,6E0,6E0,4E0,2.1E1,1.5E1,1.3E1,2.5E1,3E1,6E0,1.8E1,7E0,3.4E1,5E0,3.7E1,4E0,6E0,5E0,1.7E1,4E0,6.1E1,1.1E1,4E0,4E0,4E0,4E0,2.4E1,5.5E1,7E0,7E0,2E1,4E0,2.7E1,2.03E2,7E0,3.9E1,4E0,1.7E1,1.7E1,4E0,5E0,1E1,2.1E1,4E0,1.7E1,1.3E1,1.1E1,7E0,1.2E1,2.2E1,7E0,3E1,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[2.9454422E-3,-9.563474E-3,7.428527E-2,-2.9429536E-2,2.0322178E-2,4.1506413E-2,1.13783166E-1,-1.7095927E-2,-9.8724216E-2,2.3066541E-2,-6.2965574E-3,-4.1470453E-3,4.9289316E-2,1.3534188E-1,-4.0188786E-2,-5.0270803E-2,3.7640538E-3,-6.295235E-2,-1.7997329E-1,2.6324706E-2,-7.5294375E-2,6.173823E-2,-5.4395396E-2,1.6529757E-1,8.126788E-2,-6.065235E-3,2.7232047E-3,-4.616113E-2,-8.085477E-3,1.0416386E-1,-7.4531315E-3,-7.370393E-2,2.7397985E-3,-3.2464108E-1,-4.371797E-2,4.2655105E-3,4.3851E-2,-1.3487922E-2,4.167971E-3,-1.3443623E-3,7.2128266E-2,-1.4028888E-5,-4.509399E-3,1.3111286E-1,1.1876941E-2,9.881935E-2,-7.767354E-5,-1.0347914E-1,-3.748169E-2,1.3016593E-1,1.7415573E-3,-6.1022893E-2,-1.5484404E-3,1.5956135E-3,-8.622015E-2,-5.3663175E-3,-1.9735374E-2,-5.6289393E-3,9.3964546E-4,-8.858226E-3,5.2812744E-2,6.9785826E-3,3.736328E-2,8.584616E-2,5.409155E-4,1.5262698E-1,-1.2382794E-4,1.960861E-3,6.2812814E-3,-1.2023802E-3,-5.5518528E-3,-1.0085321E-2,-6.550774E-2,8.8766284E-2,9.474076E-3,-5.375932E-3,4.015815E-3,2.9827037E-3,-5.020529E-3,-1.0165612E-1,-8.88543E-6,-8.319991E-3,-1.0652757E-3,1.7654244E-3,5.5940794E-3,-1.7961536E-2,4.559733E-2,1.0166748E-1,-1.0409621E-3,9.179849E-3,4.7230236E-3,-1.6026724E-2,3.4700744E-3,2.086195E-4,-7.323711E-2,1.1003499E-3,5.1911105E-3,-2.7842843E-3,3.6193852E-3,1.4941422E-2,-1.7769432E-2,-1.1799691E-1,-2.7672232E-3,1.0791896E-2,-4.690106E-2,4.2978145E-2,-1.0148094E-1,5.5032045E-2,-1.847993E-2,9.680315E-3,8.564236E-2,3.5728882E-3,-3.6594528E-3,1.4066939E-2,-2.9211087E-2,-5.983574E-2,-6.85991E-3,5.559923E-3,4.667273E-3,1.2511676E-2,-3.4337644E-2,-3.7651986E-3,-6.74811E-3,3.1818952E-3,5.3375266E-3,-4.6710996E-3,-1.1250079E-2,-4.1055446E-4,4.440304E-3,-4.0270337E-5,-7.7803126E-3,3.386677E-2,8.028684E-2,-5.821384E-3,1.627714E-2,3.8258277E-2,4.8316563E-3,-8.3033217E-4,3.3440406E-3,-5.3379415E-3,-6.6886307E-4,-3.6172136E-3,-7.629177E-4,1.7429817E-3,-9.52301E-4,-1.1119111E-3,1.9207463E-3,-2.42766E-3,-2.3811366E-4,-1.1721759E-3,1.2622371E-3,7.354085E-4,-2.525038E-3,1.9855238E-3,-2.1034812E-3,1.6549384E-3,4.8858863E-3,-1.2219751E-3,3.4645093E-3,2.8443632E-3,-5.243642E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,55,57,59,61,-1,-1,-1,63,-1,-1,65,-1,67,-1,69,71,73,-1,75,77,-1,79,-1,-1,-1,-1,81,83,-1,85,87,-1,89,-1,-1,-1,-1,-1,91,93,95,-1,-1,97,-1,99,101,-1,-1,103,-1,-1,105,107,109,111,-1,-1,113,-1,-1,115,-1,-1,-1,-1,117,119,121,-1,123,125,127,129,131,133,-1,135,-1,-1,137,139,141,-1,143,-1,145,147,-1,-1,149,-1,-1,151,-1,-1,-1,-1,153,155,-1,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9130956E-1,5.0525177E-1,1.8818694E-1,4.3504015E-1,1.4927793E-1,8.922307E-2,2.2784162E-1,3.012211E-1,2.124654E-1,1.0934158E-1,0E0,0E0,1.05328545E-1,8.2136154E-2,8.9014895E-2,7.804695E-2,3.025988E-1,7.6665744E-2,4.3917787E-1,1.2546313E-1,4.2079395E-1,6.910908E-2,2.100739E-2,8.238542E-2,3.3651367E-2,0E0,0E0,7.7709496E-2,0E0,4.175678E-2,7.6453224E-2,7.207152E-2,0E0,1.8553889E-1,6.753443E-2,9.285038E-2,1.2024295E-1,0E0,0E0,0E0,5.204025E-2,0E0,0E0,8.388391E-2,0E0,3.401336E-2,0E0,2.4024278E-2,1.0990426E-1,4.071629E-2,0E0,8.685171E-2,5.059369E-2,0E0,6.106487E-2,0E0,0E0,0E0,0E0,1.5326431E-1,2.6272096E-2,0E0,7.917313E-2,7.1588576E-2,0E0,3.7735105E-2,0E0,0E0,0E0,0E0,0E0,3.8661696E-2,3.8758546E-2,1.8780746E-2,0E0,0E0,6.2057383E-2,0E0,5.3454533E-2,1.9544423E-2,0E0,0E0,6.0832735E-2,0E0,0E0,1.2142332E-1,9.16996E-2,5.811286E-2,6.126675E-2,0E0,0E0,2.818452E-2,0E0,0E0,5.565524E-2,0E0,0E0,0E0,0E0,6.566116E-2,6.4647704E-2,1.5755832E-2,0E0,7.057266E-2,4.331084E-2,3.943531E-2,6.718852E-2,6.730369E-2,7.679725E-2,0E0,3.1899065E-2,0E0,0E0,4.240682E-2,6.065211E-2,4.349293E-2,0E0,6.392194E-2,0E0,4.990501E-2,4.3838553E-2,0E0,0E0,5.8593817E-2,0E0,0E0,1.8558875E-2,0E0,0E0,0E0,0E0,5.4763347E-2,6.3031524E-2,0E0,4.2122483E-2,1.573117E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,40,40,43,43,45,45,47,47,48,48,49,49,51,51,52,52,54,54,59,59,60,60,62,62,63,63,65,65,71,71,72,72,73,73,76,76,78,78,79,79,82,82,85,85,86,86,87,87,88,88,91,91,94,94,99,99,100,100,101,101,103,103,104,104,105,105,106,106,107,107,108,108,110,110,113,113,114,114,115,115,117,117,119,119,120,120,123,123,126,126,131,131,132,132,134,134,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,56,58,60,62,-1,-1,-1,64,-1,-1,66,-1,68,-1,70,72,74,-1,76,78,-1,80,-1,-1,-1,-1,82,84,-1,86,88,-1,90,-1,-1,-1,-1,-1,92,94,96,-1,-1,98,-1,100,102,-1,-1,104,-1,-1,106,108,110,112,-1,-1,114,-1,-1,116,-1,-1,-1,-1,118,120,122,-1,124,126,128,130,132,134,-1,136,-1,-1,138,140,142,-1,144,-1,146,148,-1,-1,150,-1,-1,152,-1,-1,-1,-1,154,156,-1,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.467E3,8.995735E5,1.767635E3,2.2E1,4.9E1,1.1840488E8,2.5130852E2,5.51E2,6.0692043E0,1.9269184E7,-6.2965574E-3,-4.1470453E-3,6.364486E0,1.159499E6,4.591837E0,6.651E4,2.6E1,6.2233735E-6,1.746E3,1.158E3,2.395631E7,1.3935602E1,1.172766E3,3.5675005E3,1.7655972E0,-6.065235E-3,2.7232047E-3,3.4E1,-8.085477E-3,3.66025E5,1.84E2,1E0,2.7397985E-3,1.04109E5,2.6530768E2,4.5866325E6,6.923077E-1,-1.3487922E-2,4.167971E-3,-1.3443623E-3,1.4379113E9,-1.4028888E-5,-4.509399E-3,1.8776652E7,1.1876941E-2,1.2816234E7,-7.767354E-5,1.3888889E1,3.9961785E-2,6.743651E7,1.7415573E-3,1.1151079E0,5E0,1.5956135E-3,5.095006E5,-5.3663175E-3,-1.9735374E-2,-5.6289393E-3,9.3964546E-4,1.4110284E2,2.712766E0,6.9785826E-3,2.4294034E2,3.0561172E5,5.409155E-4,1.3947369E0,-1.2382794E-4,1.960861E-3,6.2812814E-3,-1.2023802E-3,-5.5518528E-3,7.2132964E0,1.2368386E0,1.7467743E-2,9.474076E-3,-5.375932E-3,6.819212E6,2.9827037E-3,1.4195632E6,2.8998098E1,-8.88543E-6,-8.319991E-3,9.03E2,1.7654244E-3,5.5940794E-3,1.0439024E1,2E0,1.43E2,5.3700186E8,9.179849E-3,4.7230236E-3,2.00033E5,3.4700744E-3,2.086195E-4,7.547528E6,1.1003499E-3,5.1911105E-3,-2.7842843E-3,3.6193852E-3,4.586207E0,3.14403E5,9.874E3,-2.7672232E-3,2.0097298E1,9.99E2,5.80418E7,1.1E1,6.5718125E6,1.4598765E0,9.680315E-3,1.606722E8,3.5728882E-3,-3.6594528E-3,4.9E2,4.9069305E1,1.3600995E2,-6.85991E-3,2.3412812E0,4.667273E-3,5.860227E5,1.3631483E8,-3.7651986E-3,-6.74811E-3,2.485E2,5.3375266E-3,-4.6710996E-3,3.0416667E0,-4.1055446E-4,4.440304E-3,-4.0270337E-5,-7.7803126E-3,2.23099E7,6.763312E7,-5.821384E-3,8.134772E2,2.4180895E6,4.8316563E-3,-8.3033217E-4,3.3440406E-3,-5.3379415E-3,-6.6886307E-4,-3.6172136E-3,-7.629177E-4,1.7429817E-3,-9.52301E-4,-1.1119111E-3,1.9207463E-3,-2.42766E-3,-2.3811366E-4,-1.1721759E-3,1.2622371E-3,7.354085E-4,-2.525038E-3,1.9855238E-3,-2.1034812E-3,1.6549384E-3,4.8858863E-3,-1.2219751E-3,3.4645093E-3,2.8443632E-3,-5.243642E-4],"split_indices":[2,43,67,3,3,7,73,2,68,62,0,0,69,9,69,1,0,52,44,2,62,73,4,67,49,0,0,2,0,9,44,20,0,5,67,43,68,0,0,0,12,0,0,60,0,9,0,4,53,7,0,61,3,0,43,0,0,0,0,4,68,0,67,48,0,68,0,0,0,0,0,69,68,53,0,0,12,0,60,73,0,0,2,0,0,73,6,0,7,0,0,5,0,0,5,0,0,0,0,68,9,1,0,73,2,7,8,60,69,0,12,0,0,1,62,67,0,68,0,62,7,0,0,67,0,0,69,0,0,0,0,9,7,0,70,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,8.49E2,1.48E2,5.1E2,3.39E2,8.2E1,6.6E1,4.34E2,7.6E1,3.34E2,5E0,4E0,7.8E1,5.8E1,8E0,1.67E2,2.67E2,5.4E1,2.2E1,3.24E2,1E1,7E1,8E0,3.6E1,2.2E1,4E0,4E0,1.63E2,4E0,2.6E1,2.41E2,5E1,4E0,1E1,1.2E1,1.44E2,1.8E2,4E0,6E0,7E0,6.3E1,4E0,4E0,2.8E1,8E0,1.8E1,4E0,2E1,1.43E2,1.8E1,8E0,2.3E1,2.18E2,5E0,4.5E1,4E0,6E0,5E0,7E0,1.14E2,3E1,9E0,1.71E2,5.1E1,1.2E1,2.4E1,4E0,8E0,1E1,4E0,1.6E1,7.3E1,7E1,1.3E1,5E0,1.2E1,1.1E1,1E1,2.08E2,3.8E1,7E0,4E0,1.1E2,2.6E1,4E0,2.2E1,1.49E2,4.3E1,8E0,1.1E1,1.3E1,6.9E1,4E0,7E0,6.3E1,4E0,9E0,6E0,5E0,8.1E1,1.27E2,2.6E1,1.2E1,8.8E1,2.2E1,1.3E1,9E0,1.3E2,1.9E1,4E0,3.9E1,4E0,4E0,2.1E1,4.8E1,5.5E1,8E0,7.4E1,7E0,4.5E1,8.2E1,1.3E1,1.3E1,8.3E1,5E0,8E0,1.4E1,7E0,6E0,4E0,5E0,7.2E1,5.8E1,4E0,1.5E1,1.2E1,2.7E1,1.4E1,7E0,6E0,4.2E1,3.8E1,1.7E1,3.3E1,4.1E1,2E1,2.5E1,5E1,3.2E1,3.8E1,4.5E1,9E0,5E0,6.5E1,7E0,2.2E1,3.6E1,9E0,6E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[2.0819542E-3,-1.7877074E-2,3.76012E-2,9.933179E-4,-4.062655E-2,5.258583E-2,-5.2450586E-2,-1.16662E-2,3.8404495E-2,-7.575788E-2,-2.2136638E-2,1.5701999E-1,4.148614E-2,-1.0884336E-2,-2.4057562E-2,-1.7698543E-2,5.362315E-2,2.4370957E-2,4.53019E-3,-4.9488153E-2,-2.3734702E-1,6.727523E-4,-4.7405884E-2,1.7948905E-1,4.0548347E-4,4.580467E-2,-9.699776E-2,-8.577646E-2,1.3775699E-2,-1.4497971E-2,-5.791575E-3,4.737488E-3,2.0695077E-2,-1.1189776E-2,3.9343145E-2,-5.96843E-2,2.8277691E-3,-1.9832907E-2,-5.4348363E-3,-2.0492876E-2,4.6831917E-2,-8.729253E-3,-3.721653E-2,3.3127645E-3,9.046999E-3,1.2169534E-2,6.482141E-2,-8.060399E-3,-5.7684047E-6,-4.000148E-4,-1.236139E-1,8.4833086E-2,-4.555248E-2,-6.140423E-2,-9.027213E-3,3.7792719E-3,-4.1408776E-4,8.329023E-4,-3.505223E-2,5.7367004E-2,1.1096895E-3,-4.2713508E-2,-1.390629E-1,-6.9562527E-3,-4.7919643E-3,4.4867317E-3,1.7274745E-2,2.6133945E-3,-4.476868E-2,5.975969E-3,4.7956347E-3,5.7432503E-2,7.495726E-3,-7.3635615E-3,-1.9243296E-3,1.2752093E-1,-8.089096E-4,2.5987994E-2,-6.9975727E-3,1.3696582E-3,-9.171758E-2,5.0598115E-2,-1.3074896E-2,-3.5596162E-3,5.521968E-4,4.8075696E-3,3.7947826E-2,-1.4810581E-3,1.161024E-3,-8.900546E-2,-1.6046567E-2,-1.6721735E-3,-9.779852E-3,-4.091469E-2,1.6461527E-2,-9.887124E-4,2.1111395E-3,-6.1900683E-2,-9.517646E-3,1.3114582E-2,-3.6036347E-3,8.496984E-2,2.7851483E-2,2.9190194E-3,8.279868E-3,3.5521688E-3,-2.331671E-3,-8.6891875E-3,-5.348176E-2,-2.4155518E-3,4.4803945E-3,-1.7102772E-2,5.2796353E-2,4.4994867E-5,6.330651E-2,-1.5464537E-3,-1.087415E-1,2.415385E-2,-4.13166E-2,-3.5429904E-3,3.5088528E-5,2.952218E-2,-1.672692E-3,-3.0730983E-2,-7.503631E-2,3.6828525E-2,-5.2139565E-2,-1.5952941E-2,3.6106307E-2,1.0857082E-1,-3.3979516E-2,3.496704E-2,-3.146467E-3,-8.7539974E-4,-4.7180164E-3,1.7066026E-3,-2.1535968E-2,3.934041E-4,5.2244924E-3,4.645524E-3,6.231866E-4,-1.5208583E-3,-1.2805347E-1,2.1235573E-3,-1.9287746E-3,-6.4249575E-2,2.951061E-4,-3.7301495E-4,5.7123598E-2,-2.2669241E-3,6.5244216E-4,-1.1241767E-1,-4.8546452E-2,-1.3279982E-3,4.1395235E-3,-1.6305559E-4,-4.3534413E-3,1.7959678E-3,-3.5524596E-2,5.1395256E-2,-3.9253917E-2,1.193095E-1,4.493497E-2,-9.086982E-3,4.3069743E-2,-2.7071675E-2,4.799165E-2,-5.2263984E-4,-3.4198577E-3,-7.490021E-3,-2.5688484E-3,-4.786176E-3,-1.2110481E-3,3.5636595E-3,1.9590536E-4,-6.549292E-3,-2.0684865E-3,-3.534229E-3,-8.781887E-4,5.2576273E-4,-2.7509732E-3,-4.5676634E-4,2.684451E-3,-3.661148E-3,3.968595E-4,5.820149E-3,1.7171294E-3,-1.5692565E-3,5.982705E-3,4.145085E-3,-4.971527E-4,-3.1300636E-3,1.7768291E-3,3.223041E-3,8.920452E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,51,53,-1,-1,55,57,59,61,-1,-1,-1,63,65,-1,67,-1,-1,69,71,-1,-1,-1,73,75,77,79,81,-1,-1,-1,83,85,87,89,91,93,-1,-1,95,-1,97,-1,99,101,-1,-1,-1,103,-1,105,-1,-1,107,109,111,-1,-1,-1,113,-1,-1,115,117,-1,-1,119,121,-1,-1,123,125,127,-1,129,131,-1,-1,-1,-1,-1,133,-1,-1,135,137,-1,139,-1,141,143,145,-1,-1,147,-1,149,151,153,155,157,159,161,163,165,-1,-1,-1,-1,167,-1,-1,-1,-1,-1,169,-1,-1,171,-1,-1,173,-1,-1,175,177,-1,-1,-1,-1,-1,179,181,183,185,187,-1,189,191,193,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.251983E-1,2.8122628E-1,4.9933475E-1,1.7047313E-1,1.9124687E-1,3.5978878E-1,2.699357E-1,1.06894106E-1,7.379095E-2,4.212342E-1,1.1300161E-1,9.835112E-2,1.7608231E-1,0E0,1.109279E-1,8.329445E-2,3.492868E-2,4.037189E-2,0E0,1.0410555E-1,2.711805E-1,1.02566436E-1,1.2709537E-1,3.174132E-2,0E0,1.776681E-1,6.752383E-2,5.1452726E-2,1.3033552E-1,6.1289374E-2,0E0,0E0,2.7918976E-2,1.6668027E-2,3.645941E-2,1.0452375E-1,0E0,0E0,0E0,8.0398895E-2,4.7370106E-2,0E0,6.486076E-2,0E0,0E0,8.7517515E-2,1.1882007E-1,0E0,0E0,0E0,2.8545663E-2,6.3395604E-2,1.3274671E-1,7.172075E-2,5.30333E-2,0E0,0E0,0E0,2.658392E-2,2.8695159E-2,1.5146844E-2,8.3749846E-2,9.69311E-2,5.077308E-2,0E0,0E0,2.516125E-2,0E0,4.893236E-2,0E0,6.78358E-2,1.3366872E-1,0E0,0E0,0E0,2.1939024E-2,0E0,4.668976E-2,0E0,0E0,5.9947595E-2,7.1878284E-2,5.5028502E-2,0E0,0E0,0E0,2.5253791E-2,0E0,0E0,2.3759425E-2,4.6711914E-2,0E0,0E0,3.8821492E-2,2.7208135E-2,0E0,0E0,2.0212904E-2,5.723038E-2,6.005582E-2,0E0,2.451188E-1,5.837079E-2,0E0,0E0,0E0,0E0,0E0,2.326142E-2,0E0,0E0,4.7035083E-2,3.1073507E-2,0E0,2.8333165E-2,0E0,2.2141352E-2,2.7793076E-2,3.1042457E-2,0E0,0E0,3.2630235E-2,0E0,1.5498072E-2,3.2171145E-2,5.1023245E-2,3.0359399E-2,4.4303596E-2,5.9626386E-2,4.3841183E-2,1.9652459E-1,6.351751E-2,0E0,0E0,0E0,0E0,9.5213816E-2,0E0,0E0,0E0,0E0,0E0,2.5543764E-2,0E0,0E0,2.5721401E-2,0E0,0E0,1.8723667E-2,0E0,0E0,2.3427978E-2,1.836222E-2,0E0,0E0,0E0,0E0,0E0,3.4479495E-2,1.7847449E-2,1.7901774E-2,2.8740168E-2,8.44657E-2,0E0,2.8590323E-2,3.9927527E-2,3.862101E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,40,40,42,42,45,45,46,46,50,50,51,51,52,52,53,53,54,54,58,58,59,59,60,60,61,61,62,62,63,63,66,66,68,68,70,70,71,71,75,75,77,77,80,80,81,81,82,82,86,86,89,89,90,90,93,93,94,94,97,97,98,98,99,99,101,101,102,102,108,108,111,111,112,112,114,114,116,116,117,117,118,118,121,121,123,123,124,124,125,125,126,126,127,127,128,128,129,129,130,130,131,131,136,136,142,142,145,145,148,148,151,151,152,152,158,158,159,159,160,160,161,161,162,162,164,164,165,165,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,52,54,-1,-1,56,58,60,62,-1,-1,-1,64,66,-1,68,-1,-1,70,72,-1,-1,-1,74,76,78,80,82,-1,-1,-1,84,86,88,90,92,94,-1,-1,96,-1,98,-1,100,102,-1,-1,-1,104,-1,106,-1,-1,108,110,112,-1,-1,-1,114,-1,-1,116,118,-1,-1,120,122,-1,-1,124,126,128,-1,130,132,-1,-1,-1,-1,-1,134,-1,-1,136,138,-1,140,-1,142,144,146,-1,-1,148,-1,150,152,154,156,158,160,162,164,166,-1,-1,-1,-1,168,-1,-1,-1,-1,-1,170,-1,-1,172,-1,-1,174,-1,-1,176,178,-1,-1,-1,-1,-1,180,182,184,186,188,-1,190,192,194,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,3.875E0,1E0,3.422351E6,1E0,1.8992E4,4.5E0,2.0277777E0,6.965855E9,1.8256016E7,1.45625E1,8.837831E-2,3.206931E2,-1.0884336E-2,1.4598765E0,5.2E1,3.1842105E0,3.011015E8,4.53019E-3,4.5E1,3.6255838E2,1.6223962E7,6.8E1,4.5496914E9,4.0548347E-4,5.817547E2,3.82E3,1.7234043E1,2.4E0,7.7E1,-5.791575E-3,4.737488E-3,1.4473684E-1,3.72E2,1.5321098E8,6.18826E5,2.8277691E-3,-1.9832907E-2,-5.4348363E-3,6.6358675E8,1.0486312E9,-8.729253E-3,2E0,3.3127645E-3,9.046999E-3,9.9688476E-1,4.907764E3,-8.060399E-3,-5.7684047E-6,-4.000148E-4,5.467E3,1.7E1,2E1,4.6874816E4,1.6E1,3.7792719E-3,-4.1408776E-4,8.329023E-4,8E0,1.5123151E6,4E0,2.4636364E1,5.3E1,4.1719616E5,-4.7919643E-3,4.4867317E-3,1.3881669E6,2.6133945E-3,3.885813E2,5.975969E-3,2E1,1.7951853E1,7.495726E-3,-7.3635615E-3,-1.9243296E-3,1.2697166E3,-8.089096E-4,8.0274E4,-6.9975727E-3,1.3696582E-3,7E0,2.66E2,1.775373E1,-3.5596162E-3,5.521968E-4,4.8075696E-3,1.5849056E0,-1.4810581E-3,1.161024E-3,3.8E1,8E0,-1.6721735E-3,-9.779852E-3,2.240836E6,1E0,-9.887124E-4,2.1111395E-3,7.106795E6,7.836E3,6.185E3,-3.6036347E-3,3.174172E5,1.514526E6,2.9190194E-3,8.279868E-3,3.5521688E-3,-2.331671E-3,-8.6891875E-3,7.042942E7,-2.4155518E-3,4.4803945E-3,3.5511714E-1,2.23607E5,4.4994867E-5,2.0467092E6,-1.5464537E-3,1.863E3,2.9E1,1.802361E6,-3.5429904E-3,3.5088528E-5,2.1651703E2,-1.672692E-3,1.7E1,1.0041E5,2.2166292E1,1.763E3,3.819455E-1,1.1108876E11,1.6063418E7,1.4598765E0,1.4407171E8,-3.146467E-3,-8.7539974E-4,-4.7180164E-3,1.7066026E-3,1.0292E4,3.934041E-4,5.2244924E-3,4.645524E-3,6.231866E-4,-1.5208583E-3,1.00011E5,2.1235573E-3,-1.9287746E-3,4.6748266E0,2.951061E-4,-3.7301495E-4,1.946094E6,-2.2669241E-3,6.5244216E-4,2.0518E2,5.4051723E0,-1.3279982E-3,4.1395235E-3,-1.6305559E-4,-4.3534413E-3,1.7959678E-3,1.136E3,5.036081E7,1.2193182E1,4.7E1,8.710612E1,-9.086982E-3,1.0236775E1,2.5205562E5,4.857143E0,-5.2263984E-4,-3.4198577E-3,-7.490021E-3,-2.5688484E-3,-4.786176E-3,-1.2110481E-3,3.5636595E-3,1.9590536E-4,-6.549292E-3,-2.0684865E-3,-3.534229E-3,-8.781887E-4,5.2576273E-4,-2.7509732E-3,-4.5676634E-4,2.684451E-3,-3.661148E-3,3.968595E-4,5.820149E-3,1.7171294E-3,-1.5692565E-3,5.982705E-3,4.145085E-3,-4.971527E-4,-3.1300636E-3,1.7768291E-3,3.223041E-3,8.920452E-4],"split_indices":[2,69,6,9,29,9,73,68,46,5,73,72,73,0,69,8,69,46,0,3,4,5,0,46,0,67,0,71,69,44,0,0,71,44,7,9,0,0,0,46,46,0,32,0,0,69,4,0,0,0,2,8,6,60,0,0,0,0,8,60,3,67,0,60,0,0,43,0,67,0,3,71,0,0,0,4,0,1,0,0,12,1,71,0,0,0,71,0,0,44,10,0,0,9,30,0,0,5,44,44,0,48,10,0,0,0,0,0,60,0,0,69,9,0,43,0,9,3,12,0,0,67,0,3,1,73,2,57,46,62,69,5,0,0,0,0,44,0,0,0,0,0,5,0,0,69,0,0,60,0,0,67,69,0,0,0,0,0,44,7,73,8,73,0,71,48,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.021E3,6.54E2,3.67E2,3.58E2,2.96E2,3.15E2,5.2E1,2.68E2,9E1,1.01E2,1.95E2,2.9E1,2.86E2,6E0,4.6E1,2.46E2,2.2E1,7.4E1,1.6E1,8.8E1,1.3E1,1.03E2,9.2E1,2.5E1,4E0,2.78E2,8E0,1.7E1,2.9E1,2.4E2,6E0,8E0,1.4E1,2.2E1,5.2E1,8.1E1,7E0,4E0,9E0,7.1E1,3.2E1,5E0,8.7E1,4E0,2.1E1,1.01E2,1.77E2,4E0,4E0,6E0,1.1E1,1.3E1,1.6E1,2.4E1,2.16E2,4E0,1E1,1E1,1.2E1,3.5E1,1.7E1,6.8E1,1.3E1,6.2E1,9E0,1.1E1,2.1E1,6E0,8.1E1,5E0,9.6E1,1.66E2,1.1E1,7E0,4E0,9E0,4E0,1E1,6E0,6E0,1.8E1,1.3E1,2.03E2,6E0,6E0,9E0,2.6E1,7E0,1E1,2.4E1,4.4E1,6E0,7E0,2.5E1,3.7E1,9E0,1.2E1,5.4E1,2.7E1,8.8E1,8E0,8.5E1,8.1E1,5E0,4E0,6E0,4E0,4E0,1.4E1,4E0,9E0,1.92E2,1.1E1,1.1E1,1.5E1,7E0,1.7E1,1.7E1,2.7E1,1.3E1,1.2E1,3E1,7E0,1.7E1,3.7E1,1.3E1,1.4E1,3.9E1,4.9E1,7.1E1,1.4E1,7.6E1,5E0,9E0,5E0,1.4E1,1.78E2,7E0,4E0,8E0,7E0,4E0,1.3E1,1.3E1,4E0,1.8E1,9E0,1.3E1,1.7E1,1.2E1,5E0,1.4E1,2.3E1,6E0,7E0,7E0,7E0,1E1,2.9E1,4.1E1,8E0,6E1,1.1E1,4E0,1E1,1.3E1,6.3E1,1.5E2,2.8E1,8E0,5E0,8E0,1E1,1.2E1,5E0,9E0,5E0,1.1E1,1.2E1,1E1,1.9E1,4E0,3.7E1,4E0,4E0,5.5E1,5E0,6E0,5E0,5E0,5E0,8E0,5E0,3.5E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"195","size_leaf_vector":"1"}},{"base_weights":[-5.098481E-4,-1.4738046E-2,4.0954E-2,-1.9372182E-2,5.9637774E-2,-1.23512626E-1,4.744676E-2,-1.7254204E-2,-1.8077473E-1,4.593562E-2,5.7980376E-3,-2.4887423E-3,-8.425841E-3,1.241756E-2,4.2697247E-2,-2.8662639E-2,8.007986E-3,-1.3674575E-2,-1.3583421E-3,3.2477606E-2,5.1880954E-3,5.212447E-2,-3.798862E-2,-1.542429E-2,-5.676903E-2,1.2686985E-2,-6.1987303E-3,-5.8552832E-5,4.940772E-2,3.7044518E-2,8.7050445E-2,3.6837216E-3,-7.729276E-2,-1.194912E-2,-1.3688593E-1,-1.1681E-2,-4.563655E-2,-2.3688074E-2,2.2412676E-2,3.8328255E-4,6.8039946E-2,1.10149E-2,5.3999268E-2,9.8913275E-2,2.2469626E-3,-2.5181672E-2,-1.3818614E-1,-3.8508173E-2,3.4171776E-3,-8.849732E-3,-2.5334677E-3,-8.2773045E-2,-1.645134E-2,-1.2277105E-2,-4.426478E-3,2.6620286E-2,-3.583389E-3,4.267272E-3,7.463321E-4,-3.0195527E-3,1.8861081E-2,6.5743993E-3,3.9159935E-2,6.2475264E-2,1.2962498E-1,2.8180457E-3,-2.631192E-3,-2.4598902E-3,1.1437379E-3,-1.0430779E-2,-1.0605438E-3,-3.1210838E-2,-9.042371E-3,1.0638501E-2,-2.299947E-3,-9.830764E-3,-7.055862E-2,-5.971306E-3,-4.361153E-3,-2.1943724E-4,-2.9575764E-3,3.4335148E-2,-2.6447143E-2,2.4948966E-2,-2.2842064E-3,9.353032E-2,2.4337528E-2,7.833544E-2,1.7111906E-4,7.3710205E-3,3.300594E-3,-4.6692163E-2,-4.9228165E-3,7.3657366E-3,-6.4146735E-2,-1.0125736E-1,-3.6484636E-2,-1.7468985E-2,5.0467397E-3,4.2490862E-2,-9.65095E-5,1.3811332E-3,-1.0489206E-1,-1.7991523E-3,3.0959442E-2,5.7429476E-3,-2.4022316E-4,4.5577753E-3,1.3667204E-2,4.3751905E-3,8.733776E-4,-1.5174905E-3,-5.508142E-3,2.0170852E-3,-1.6322191E-3,8.7473524E-4,-1.7178691E-3,1.8310734E-3,-4.1364287E-3,-5.4006767E-3,-7.667031E-4,-4.4859826E-4,-5.6857583E-3,-2.0381626E-4,-3.969211E-3,8.557374E-4,2.5274237E-3,2.0049165E-3,-1.0617899E-3,-6.7551206E-3,-1.9675244E-3,1.9234428E-3,-6.853169E-4,-3.649284E-3,1.1041055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,-1,27,-1,29,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,-1,51,53,55,-1,57,59,61,63,65,67,69,71,73,-1,-1,75,77,79,-1,81,-1,-1,-1,-1,83,-1,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,-1,93,-1,95,-1,97,-1,-1,99,101,103,-1,105,107,109,-1,-1,-1,111,113,115,117,119,121,123,-1,125,127,-1,129,-1,131,-1,-1,-1,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9114933E-1,2.5827742E-1,2.79228E-1,2.3719943E-1,3.3943772E-2,3.1172812E-2,2.4818105E-1,2.0052996E-1,1.4826483E-1,3.02912E-2,0E0,0E0,0E0,0E0,1.8681666E-1,1.7735177E-1,1.4606614E-1,0E0,0E0,1.8915746E-2,0E0,1.11591935E-1,1.2646657E-1,1.3639097E-1,3.224242E-1,7.517694E-2,0E0,0E0,1.6287457E-2,6.736934E-2,6.516829E-2,0E0,5.9984647E-2,1.3032006E-1,2.9865876E-2,0E0,1.5687838E-1,3.5859473E-2,7.231554E-2,0E0,1.7441131E-2,3.838615E-2,1.16295636E-1,5.56888E-2,3.4776725E-2,1.8135723E-2,8.5657164E-2,1.2812224E-1,2.6381576E-1,0E0,0E0,8.6624116E-2,1.1262984E-1,1.52747E-2,0E0,6.6822745E-2,0E0,0E0,0E0,0E0,2.4935605E-2,0E0,6.3001595E-2,2.580931E-2,4.0832877E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.5758143E-2,0E0,0E0,1.19433E-1,0E0,5.906737E-2,0E0,1.1507625E-1,0E0,0E0,3.959897E-2,9.595109E-2,2.1558948E-2,0E0,5.230528E-2,5.019623E-2,1.9609943E-2,0E0,0E0,0E0,6.738642E-2,6.506628E-2,8.986241E-2,7.468872E-2,3.9114714E-2,6.8354115E-2,6.1287977E-2,0E0,3.236623E-2,2.8869137E-2,0E0,1.694569E-2,0E0,2.45696E-2,0E0,0E0,0E0,5.6874275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,19,19,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,32,32,33,33,34,34,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,51,51,52,52,53,53,55,55,60,60,62,62,63,63,64,64,71,71,74,74,76,76,78,78,81,81,82,82,83,83,85,85,86,86,87,87,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,102,102,104,104,108,108],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,-1,28,-1,30,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,-1,52,54,56,-1,58,60,62,64,66,68,70,72,74,-1,-1,76,78,80,-1,82,-1,-1,-1,-1,84,-1,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,-1,94,-1,96,-1,98,-1,-1,100,102,104,-1,106,108,110,-1,-1,-1,112,114,116,118,120,122,124,-1,126,128,-1,130,-1,132,-1,-1,-1,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.984E3,4.5866325E6,1E0,5.9E1,1.339646E6,7.472532E9,3.3817584E7,2.026453E6,4.0568292E2,2.712766E0,5.7980376E-3,-2.4887423E-3,-8.425841E-3,1.241756E-2,3.1879792E8,3.39498E5,1E0,-1.3674575E-2,-1.3583421E-3,1.2E1,5.1880954E-3,1.4817301E3,7.556899E-1,4.217427E7,9E0,5.88E2,-6.1987303E-3,-5.8552832E-5,5.294E3,1.1251919E6,1.2820834E7,3.6837216E-3,1.3475722E2,1.7667647E2,1.0479E4,-1.1681E-2,1.084437E6,2.8608696E1,4.643602E8,3.8328255E-4,2.138772E4,3.167E3,3.443E3,1.6326721E0,3.359873E6,1.1863237E2,2.0120485E0,7.45E2,7.336111E4,-8.849732E-3,-2.5334677E-3,3.72424E5,1.318E2,1.0066326E8,-4.426478E-3,1E0,-3.583389E-3,4.267272E-3,7.463321E-4,-3.0195527E-3,5.433518E0,6.5743993E-3,7.5052085E0,1.4052098E0,1E0,2.8180457E-3,-2.631192E-3,-2.4598902E-3,1.1437379E-3,-1.0430779E-2,-1.0605438E-3,8.2765434E1,-9.042371E-3,1.0638501E-2,1.647E4,-9.830764E-3,3.1275E4,-5.971306E-3,1.9791039E6,-2.1943724E-4,-2.9575764E-3,1.7E1,5.036784E6,2.9585715E2,-2.2842064E-3,8.76E2,1.6208625E6,5.501002E9,1.7111906E-4,7.3710205E-3,3.300594E-3,4.577342E0,4.8E1,6.2222223E0,1.3E1,9.393264E-6,2.0734E4,2.4E1,5.0467397E-3,1.2200055E7,5.44E2,1.3811332E-3,1.11E2,-1.7991523E-3,7.973882E7,5.7429476E-3,-2.4022316E-4,4.5577753E-3,3.921E3,4.3751905E-3,8.733776E-4,-1.5174905E-3,-5.508142E-3,2.0170852E-3,-1.6322191E-3,8.7473524E-4,-1.7178691E-3,1.8310734E-3,-4.1364287E-3,-5.4006767E-3,-7.667031E-4,-4.4859826E-4,-5.6857583E-3,-2.0381626E-4,-3.969211E-3,8.557374E-4,2.5274237E-3,2.0049165E-3,-1.0617899E-3,-6.7551206E-3,-1.9675244E-3,1.9234428E-3,-6.853169E-4,-3.649284E-3,1.1041055E-3],"split_indices":[2,43,17,3,9,46,7,9,4,68,0,0,0,0,47,9,79,0,0,3,0,67,57,60,3,2,0,0,12,43,9,0,73,70,9,0,9,73,7,0,62,2,2,53,1,73,50,2,60,0,0,9,70,7,0,19,0,0,0,0,68,0,71,53,30,0,0,0,0,0,0,4,0,0,44,0,1,0,43,0,0,3,9,67,0,0,43,5,0,0,0,73,10,61,3,52,44,8,0,5,10,0,0,0,47,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.45E2,2.55E2,7.02E2,4.3E1,9E0,2.46E2,6.94E2,8E0,3.7E1,6E0,5E0,4E0,4E0,2.42E2,4.78E2,2.16E2,4E0,4E0,3.2E1,5E0,2.17E2,2.5E1,3.26E2,1.52E2,2.1E2,6E0,1.1E1,2.1E1,1.53E2,6.4E1,6E0,1.9E1,3.18E2,8E0,7E0,1.45E2,4.4E1,1.66E2,7E0,1.4E1,6.1E1,9.2E1,5.6E1,8E0,1.1E1,8E0,1.16E2,2.02E2,4E0,4E0,6.3E1,8.2E1,3.9E1,5E0,1.6E2,6E0,9E0,5E0,5E0,5.6E1,1.2E1,8E1,2.7E1,2.9E1,4E0,4E0,7E0,4E0,4E0,4E0,1.12E2,4E0,4E0,1.98E2,4E0,5.9E1,7E0,7.5E1,3.5E1,4E0,1.4E2,2E1,5.2E1,4E0,1.6E1,6.4E1,2.1E1,6E0,1.8E1,1.1E1,7E1,4.2E1,1.72E2,2.6E1,3E1,2.9E1,6.8E1,7E0,1.13E2,2.7E1,1.2E1,8E0,4E0,4.8E1,1.2E1,4E0,7E0,5.7E1,1.6E1,5E0,6E1,1E1,1.6E1,2.6E1,1.37E2,3.5E1,5E0,2.1E1,2.5E1,5E0,2.3E1,6E0,5.8E1,1E1,3.9E1,7.4E1,9E0,1.8E1,4E0,4E0,3.9E1,9E0,5E0,5.2E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-2.3366245E-3,-3.3935077E-2,2.497764E-2,-2.9708486E-2,-1.417064E-1,1.0916311E-2,7.048774E-2,-5.9631854E-2,-1.710505E-2,-5.928723E-2,-1.5898887E-2,1.8216379E-2,-7.442907E-2,1.065097E-2,6.237094E-2,-6.490355E-3,-5.35755E-2,2.1833261E-2,-2.692233E-2,2.0630606E-3,-6.598452E-3,1.22842915E-1,1.4626896E-2,2.0293012E-2,-1.0934075E-1,7.504802E-2,1.263924E-2,2.874033E-3,-5.7744954E-2,4.9296217E-3,1.3267819E-2,-9.8109305E-2,-2.0045375E-2,3.0784048E-3,8.191784E-3,-1.9137459E-2,2.371617E-2,2.562346E-3,-1.1997796E-3,-6.3487045E-2,-1.717726E-1,5.0587215E-2,9.540118E-2,4.512991E-2,-5.0172056E-3,-4.9768772E-2,-1.0947064E-1,-1.1092251E-2,3.1169321E-2,-1.1857892E-1,-3.2473574E-5,-1.3697813E-1,-1.5462298E-2,-4.862707E-3,-1.2178419E-2,7.0062275E-3,2.062097E-2,-3.93082E-3,4.8236318E-5,-1.1115208E-2,-2.5332111E-3,-1.5583293E-2,7.068069E-2,1.5074437E-4,1.04736455E-1,5.898369E-3,6.16405E-4,-5.9721004E-2,-9.094102E-3,-1.3707307E-1,-9.85457E-4,2.3244622E-3,-1.6215943E-3,7.4759037E-3,2.9494262E-3,-1.9061627E-3,-1.6024344E-1,-2.5501123E-4,-1.1135857E-2,-1.8156148E-2,5.153377E-3,9.026356E-4,-4.2929705E-2,5.9875626E-2,1.4100203E-2,-3.0645975E-3,8.354735E-4,3.4172386E-2,4.3436084E-3,1.3397242E-1,6.27663E-2,1.3335259E-3,-6.647257E-2,2.3376613E-3,-5.1083006E-2,-1.4053211E-3,-8.620643E-3,-3.4271897E-4,2.7534587E-3,-4.703675E-3,-1.0241547E-2,-2.724116E-2,2.0867791E-2,-3.3161156E-2,2.4616713E-2,-5.1386394E-3,-1.7778542E-2,3.0733211E-2,1.102831E-1,-1.4536855E-2,2.4328878E-2,2.9816085E-3,-1.9684598E-4,1.5591888E-1,2.7542224E-3,1.4732647E-2,4.383693E-3,-7.45126E-2,-1.2692969E-2,-5.083454E-3,-6.1066314E-3,-8.612028E-3,-2.2743389E-2,1.0033327E-2,3.5435753E-3,1.8349178E-3,-2.3374162E-3,-1.2451997E-2,4.3717068E-2,-2.0938818E-3,1.480186E-2,4.4817934E-3,8.248015E-3,6.5334155E-3,1.8484099E-3,2.1140661E-2,-8.525985E-2,1.2984869E-2,7.387153E-2,9.635364E-3,4.96547E-3,2.6512677E-3,-1.0749991E-3,-4.4982047E-3,-2.5811733E-3,1.6035568E-3,-3.0436222E-3,2.394242E-3,-2.902059E-3,-6.9392077E-4,-2.77782E-3,-8.605957E-4,1.2717951E-3,-1.9096925E-3,1.6748591E-3,2.8841703E-3,-7.6629693E-4,3.143531E-3,-1.9126337E-3,-8.562794E-4,3.1547972E-3,-1.0556738E-4,4.88332E-3,1.106909E-4,-1.8539676E-2,8.1247033E-4,-4.814637E-3,5.3307763E-3,5.0010026E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,-1,27,29,31,-1,-1,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,-1,53,55,-1,-1,57,59,61,63,65,-1,67,69,71,73,75,-1,77,79,-1,81,-1,83,-1,-1,-1,-1,85,87,-1,89,-1,-1,91,93,95,-1,-1,-1,97,-1,-1,99,-1,-1,101,-1,103,105,107,109,-1,-1,111,-1,113,115,-1,117,-1,119,-1,-1,-1,-1,-1,-1,121,123,125,127,-1,129,131,133,135,137,-1,-1,139,-1,141,-1,143,145,-1,147,-1,149,151,-1,-1,-1,153,155,-1,157,-1,159,-1,-1,161,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0104896E-1,2.1625483E-1,3.575926E-1,1.7510304E-1,2.7991465E-1,2.6863444E-1,1.5710902E-1,6.005004E-2,1.2673634E-1,1.2794484E-1,0E0,1.4692304E-1,1.1465365E-1,0E0,7.9119E-2,0E0,6.6351265E-2,4.729026E-2,1.2727444E-1,0E0,0E0,2.7988851E-2,1.1830098E-1,1.771107E-2,6.0662955E-2,4.5457244E-2,1.11257784E-1,0E0,4.653603E-2,0E0,2.7382812E-2,4.5775697E-2,1.2757272E-1,0E0,0E0,4.7860697E-2,1.1685783E-1,0E0,0E0,2.2388108E-2,6.7287296E-2,6.492203E-2,4.6105564E-2,5.6021012E-2,0E0,4.4390082E-2,3.600219E-2,4.1570447E-2,2.7391069E-2,5.4596215E-2,0E0,1.21788606E-1,8.208712E-2,0E0,3.1012716E-2,0E0,7.517898E-2,0E0,0E0,0E0,0E0,2.1703668E-2,2.8313294E-2,0E0,5.171591E-2,0E0,0E0,5.514896E-2,6.0233835E-2,5.7708204E-2,0E0,0E0,0E0,1.7521743E-2,0E0,0E0,2.0508885E-2,0E0,0E0,8.211104E-2,0E0,4.5244843E-2,3.7531644E-2,5.8308467E-2,7.5340986E-2,0E0,0E0,1.8947946E-2,0E0,3.6396503E-2,3.254538E-2,0E0,3.4942567E-2,0E0,3.644753E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2963577E-1,2.5847668E-2,3.123142E-2,2.3909206E-2,0E0,1.6824268E-2,4.0927082E-2,2.7162075E-2,1.7338146E-1,1.0526112E-1,0E0,0E0,3.1133235E-2,0E0,1.7545E-2,0E0,2.5233984E-2,3.254157E-2,0E0,3.282278E-2,0E0,5.1932782E-2,1.9567434E-2,0E0,0E0,0E0,1.8380256E-2,2.5860652E-2,0E0,2.9747263E-2,0E0,3.6641683E-2,0E0,0E0,9.136312E-2,6.381247E-1,8.542231E-2,9.036061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,35,35,36,36,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,54,54,56,56,61,61,62,62,64,64,67,67,68,68,69,69,73,73,76,76,79,79,81,81,82,82,83,83,84,84,87,87,89,89,90,90,92,92,94,94,101,101,102,102,103,103,104,104,106,106,107,107,108,108,109,109,110,110,113,113,115,115,117,117,118,118,120,120,122,122,123,123,127,127,128,128,130,130,132,132,135,135,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,-1,28,30,32,-1,-1,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,-1,54,56,-1,-1,58,60,62,64,66,-1,68,70,72,74,76,-1,78,80,-1,82,-1,84,-1,-1,-1,-1,86,88,-1,90,-1,-1,92,94,96,-1,-1,-1,98,-1,-1,100,-1,-1,102,-1,104,106,108,110,-1,-1,112,-1,114,116,-1,118,-1,120,-1,-1,-1,-1,-1,-1,122,124,126,128,-1,130,132,134,136,138,-1,-1,140,-1,142,-1,144,146,-1,148,-1,150,152,-1,-1,-1,154,156,-1,158,-1,160,-1,-1,162,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.48E2,6.0692043E0,5.97E3,3.06E2,1.6493898E6,2.856934E7,1.2646534E4,6E0,3.4E1,3.3817584E7,-1.5898887E-2,2.04115E5,8.620714E2,1.065097E-2,2.7021693E8,-6.490355E-3,2.4E1,2.00087E5,6.593462E4,2.0630606E-3,-6.598452E-3,1.5E1,3.3193566E5,2.476353E-2,1.9885094E-8,1.4817301E3,8.81059E5,2.874033E-3,6.769866E7,4.9296217E-3,2.4E1,9.7199225E-1,1.65E2,3.0784048E-3,8.191784E-3,2.5454E4,9.9688476E-1,2.562346E-3,-1.1997796E-3,1.6E1,1.7308458E1,1E0,1.990351E1,6.9E1,-5.0172056E-3,2E1,4E0,1.8949389E6,1.1913043E0,1.1732674E1,-3.2473574E-5,1.01E2,2.2E1,-4.862707E-3,2.3600838E6,7.0062275E-3,2.28E0,-3.93082E-3,4.8236318E-5,-1.1115208E-2,-2.5332111E-3,1.0292E4,8.83E3,1.5074437E-4,2.3385885E0,5.898369E-3,6.16405E-4,8E0,1E0,3.14403E5,-9.85457E-4,2.3244622E-3,-1.6215943E-3,3.1857144E2,2.9494262E-3,-1.9061627E-3,1.5E1,-2.5501123E-4,-1.1135857E-2,2.9652428E6,5.153377E-3,2.7491847E5,8.35684E4,8.7151864E7,3E0,-3.0645975E-3,8.354735E-4,1.925508E6,4.3436084E-3,2.4508352E9,1.4176E4,1.3335259E-3,5.2083335E0,2.3376613E-3,6.44E2,-1.4053211E-3,-8.620643E-3,-3.4271897E-4,2.7534587E-3,-4.703675E-3,-1.0241547E-2,1.2954546E2,2.8198E2,9.99E2,6E0,-5.1386394E-3,2.0227273E0,1.6446976E3,2.8543878E6,4.5620965E6,1.0213677E3,2.9816085E-3,-1.9684598E-4,1.3009709E0,2.7542224E-3,2.0669324E3,4.383693E-3,3.3817584E7,1.3085094E2,-5.083454E-3,1.0146154E2,-8.612028E-3,2.4E1,7.318515E6,3.5435753E-3,1.8349178E-3,-2.3374162E-3,8.6363555E4,3.9416668E1,-2.0938818E-3,3.164034E0,4.4817934E-3,1.5272727E0,6.5334155E-3,1.8484099E-3,2.517059E6,3.289E3,1E0,6.797565E1,9.635364E-3,4.96547E-3,2.6512677E-3,-1.0749991E-3,-4.4982047E-3,-2.5811733E-3,1.6035568E-3,-3.0436222E-3,2.394242E-3,-2.902059E-3,-6.9392077E-4,-2.77782E-3,-8.605957E-4,1.2717951E-3,-1.9096925E-3,1.6748591E-3,2.8841703E-3,-7.6629693E-4,3.143531E-3,-1.9126337E-3,-8.562794E-4,3.1547972E-3,-1.0556738E-4,4.88332E-3,1.106909E-4,-1.8539676E-2,8.1247033E-4,-4.814637E-3,5.3307763E-3,5.0010026E-4],"split_indices":[2,68,2,2,60,60,48,67,0,7,0,5,4,0,47,0,2,5,43,0,0,0,43,53,52,67,9,0,7,0,0,53,12,0,0,9,69,0,0,8,61,29,73,8,0,3,8,60,71,73,0,0,71,0,62,0,71,0,0,0,0,44,2,0,53,0,0,3,20,9,0,0,0,67,0,0,3,0,0,43,0,66,48,7,8,0,0,43,0,12,2,0,69,0,1,0,0,0,0,0,0,4,67,2,8,0,68,48,60,62,70,0,0,68,0,4,0,7,4,0,4,0,8,60,0,0,0,48,71,0,53,0,71,0,0,62,44,79,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,4.83E2,5.59E2,4.66E2,1.7E1,4.28E2,1.31E2,1.37E2,3.29E2,1.3E1,4E0,3.95E2,3.3E1,5E0,1.26E2,8E0,1.29E2,6.6E1,2.63E2,6E0,7E0,1.2E1,3.83E2,9E0,2.4E1,1E2,2.6E1,4E0,1.25E2,5E0,6.1E1,2.2E1,2.41E2,7E0,5E0,8.1E1,3.02E2,5E0,4E0,1.5E1,9E0,4.7E1,5.3E1,2.1E1,5E0,1.1E2,1.5E1,2.6E1,3.5E1,1.8E1,4E0,8E0,2.33E2,5E0,7.6E1,6E0,2.96E2,1.1E1,4E0,5E0,4E0,1.1E1,3.6E1,5E0,4.8E1,5E0,1.6E1,8.8E1,2.2E1,1.1E1,4E0,7E0,1.9E1,2.1E1,1.4E1,7E0,1.1E1,4E0,4E0,2.29E2,4E0,5.4E1,2.2E1,4.1E1,2.55E2,4E0,7E0,1.5E1,2.1E1,2.7E1,2.1E1,6E0,8.2E1,9E0,1.3E1,4E0,7E0,1.7E1,4E0,7E0,4E0,1.86E2,4.3E1,2.2E1,3.2E1,5E0,1.7E1,2.7E1,1.4E1,6.7E1,1.88E2,8E0,7E0,2E1,7E0,9E0,1.2E1,7.1E1,1.1E1,5E0,8E0,4E0,1.82E2,3.7E1,6E0,4E0,1.8E1,1.1E1,2.1E1,9E0,8E0,6E0,2.1E1,9E0,5E0,4.5E1,2.2E1,1.54E2,3.4E1,8E0,1.2E1,4E0,5E0,3E1,4.1E1,6E0,5E0,4E0,4E0,1.52E2,3E1,1.4E1,2.3E1,7E0,4E0,1.6E1,5E0,4E0,4E0,1.5E1,6E0,3.6E1,9E0,1.8E1,4E0,1.49E2,5E0,2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-4.5186665E-4,-1.232754E-2,6.791436E-2,-7.3019736E-3,-1.1986783E-1,5.622875E-2,9.821498E-3,-3.862031E-2,4.5574345E-3,-5.202672E-2,-2.2494516E-1,6.8715096E-2,-1.8988553E-2,-3.50905E-2,-8.798329E-3,6.2457863E-3,2.3689324E-3,-1.2038623E-2,-9.681999E-3,-1.6154714E-2,-1.4987651E-3,3.7945434E-2,1.0940608E-1,1.04084895E-4,-3.3479459E-3,-4.1402604E-2,3.1994343E-2,-1.7647162E-2,1.6856538E-2,2.3580734E-3,-1.8790762E-3,5.4157972E-2,-7.975848E-2,1.3890329E-1,5.924813E-2,-3.4592424E-2,-9.539265E-3,9.611605E-3,3.8169099E-3,-7.954144E-3,-5.5693094E-2,7.216566E-4,3.537732E-2,7.475748E-3,3.7811138E-2,-6.551623E-3,-8.095593E-5,1.509483E-3,1.5291207E-1,3.152177E-2,4.6366863E-3,-1.8541634E-2,-5.3614337E-2,2.7443261E-3,-1.911738E-3,-2.6344648E-2,1.2626461E-2,1.1205216E-3,-6.4910285E-2,7.618614E-3,-8.210358E-3,4.5564607E-2,-4.3693557E-2,4.064799E-4,5.8580212E-2,7.876785E-3,2.772825E-3,2.7803134E-3,-7.7249296E-4,-5.1793877E-2,7.0877206E-3,-7.6312795E-2,-3.074104E-2,-3.449613E-2,6.624884E-2,4.4400066E-2,-7.1777026E-3,-3.344104E-2,-1.2182505E-1,1.3140672E-2,-7.1245424E-2,6.1933473E-2,-9.346008E-5,-1.1465894E-2,2.7383037E-2,1.8956016E-3,-1.2820043E-3,6.8836E-2,-8.67289E-4,-2.6966687E-2,-8.826871E-2,1.7107772E-2,-5.2004624E-3,5.1355433E-5,-8.601864E-2,-4.171948E-2,8.981079E-3,1.6906897E-2,-4.8552163E-2,4.93053E-4,5.016087E-3,6.374375E-2,-6.5613067E-4,-4.958542E-2,1.5265792E-2,1.6842806E-3,-2.0857288E-3,-7.179622E-3,-2.607858E-3,1.9787349E-2,-5.6860816E-2,-1.5366844E-4,-4.8411638E-3,3.7294164E-2,9.580491E-2,-3.755522E-2,4.569833E-2,-2.7183776E-3,7.637889E-2,3.7102813E-3,7.568201E-4,-3.0836621E-3,1.0747998E-3,-5.392878E-3,-3.3006503E-4,3.1681757E-3,4.949889E-5,-1.0641998E-3,-4.9084323E-3,-2.3102786E-3,4.358501E-4,2.782167E-3,-1.3352493E-3,2.3087817E-3,-2.8223842E-3,-3.4552247E-3,-1.4551475E-3,2.0838044E-4,3.847151E-3,-6.8767555E-4,-3.985468E-3,4.282039E-3,1.8890017E-4,-3.0996418E-4,1.7690883E-3,-5.667811E-3,-1.0058801E-3,5.650749E-4,3.2458662E-3,5.065372E-3,-4.668861E-4,-2.5756788E-3,9.3603204E-4,3.759572E-3,-1.0161304E-3,8.382565E-4,5.182506E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,-1,-1,27,29,-1,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,-1,55,57,59,61,-1,63,-1,-1,-1,65,67,-1,69,71,-1,-1,73,75,-1,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,-1,-1,119,-1,121,123,125,-1,-1,127,129,131,133,135,-1,-1,137,-1,139,141,-1,-1,-1,-1,143,145,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.338413E-1,4.7201926E-1,2.4502254E-1,3.1105202E-1,2.6631075E-1,1.3466966E-1,0E0,1.1657637E-1,1.7350456E-1,1.5497485E-1,3.4882414E-1,1.4859837E-1,2.401888E-2,9.7023636E-2,0E0,0E0,1.7399865E-1,4.00082E-2,0E0,0E0,0E0,1.3980736E-1,6.870532E-2,0E0,0E0,2.2557074E-1,2.1868177E-2,9.248954E-2,1.0396752E-1,0E0,0E0,1.047533E-1,4.3317165E-2,4.2624593E-2,2.0775795E-2,6.0069248E-2,0E0,4.0541988E-2,0E0,7.678715E-2,3.915572E-2,2.3149186E-1,1.3248476E-1,0E0,4.3480344E-2,0E0,0E0,0E0,3.550458E-2,2.0362806E-2,0E0,9.4112806E-2,4.490024E-2,0E0,0E0,8.304616E-2,6.0726658E-2,0E0,7.831499E-2,7.976023E-2,0E0,1.0762718E-1,2.829521E-1,2.5159061E-2,2.9851675E-2,0E0,0E0,0E0,0E0,4.1086927E-2,7.745904E-2,3.3911705E-2,2.0870771E-2,7.235059E-2,2.0012569E-2,4.2847946E-2,5.78636E-2,2.7009934E-2,2.6354477E-2,8.0222264E-2,2.6990686E-2,8.489132E-2,6.861842E-2,0E0,6.7863144E-2,0E0,0E0,1.7945185E-2,0E0,6.149973E-2,4.303725E-2,4.862351E-2,0E0,0E0,4.851055E-2,1.6157545E-2,2.3093084E-2,5.9796385E-2,3.2547817E-2,0E0,0E0,3.201489E-2,0E0,2.5446285E-2,3.4234248E-2,0E0,0E0,0E0,0E0,7.6748595E-2,3.197217E-2,0E0,0E0,5.118967E-2,6.559163E-2,2.4274735E-2,4.526421E-2,0E0,1.8811878E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,42,42,44,44,48,48,49,49,51,51,52,52,55,55,56,56,58,58,59,59,61,61,62,62,63,63,64,64,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,84,84,87,87,89,89,90,90,91,91,94,94,95,95,96,96,97,97,98,98,101,101,103,103,104,104,109,109,110,110,113,113,114,114,115,115,116,116,118,118],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,-1,-1,28,30,-1,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,-1,56,58,60,62,-1,64,-1,-1,-1,66,68,-1,70,72,-1,-1,74,76,-1,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,-1,-1,120,-1,122,124,126,-1,-1,128,130,132,134,136,-1,-1,138,-1,140,142,-1,-1,-1,-1,144,146,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.467E3,6.0692043E0,1.0534078E4,1.5152774E2,4.9E1,7.3274844E9,9.821498E-3,4.5E1,7.336111E4,2.0778275E-1,2.8E1,1.6975454E3,3.9661028E7,8.8474586E8,-8.798329E-3,6.2457863E-3,6.285983E5,2.857143E0,-9.681999E-3,-1.6154714E-2,-1.4987651E-3,3.0561172E5,1.7493458E7,1.04084895E-4,-3.3479459E-3,4.5865917E8,1.9290142E0,2.4218928E7,4.4935583E2,2.3580734E-3,-1.8790762E-3,1.81E2,1.36E2,6.6358675E8,1.2E1,4.201202E-2,-9.539265E-3,3.5714287E-1,3.8169099E-3,1.7161779E-1,6.613774E5,2.511E3,2.9366477E0,7.475748E-3,1.3543621E6,-6.551623E-3,-8.095593E-5,1.509483E-3,2.53E2,3.01E2,4.6366863E-3,3.54E1,3.1157124E-1,2.7443261E-3,-1.911738E-3,5.7E3,1.3586957E1,1.1205216E-3,9.6203804E-1,1.70595E5,-8.210358E-3,2E0,6.9508715E0,4.9538185E6,1E0,7.876785E-3,2.772825E-3,2.7803134E-3,-7.7249296E-4,6.0921145E4,1E0,2.6412E5,1.547E3,5.539245E6,2.5393645E5,5.9908E6,1.9E1,8.7961E4,2.909091E0,6.346204E-7,1.162E3,6.965855E9,2.9308079E1,-1.1465894E-2,1.3E1,1.8956016E-3,-1.2820043E-3,2.4262331E5,-8.67289E-4,7.785714E0,3.5070792E2,2.3603575E5,-5.2004624E-3,5.1355433E-5,1.1010101E0,9.29023E5,1.5E1,4.313463E-2,1.8987958E0,4.93053E-4,5.016087E-3,5.4603375E5,-6.5613067E-4,9.783615E4,1.8916E2,1.6842806E-3,-2.0857288E-3,-7.179622E-3,-2.607858E-3,1.71E2,1.4E1,-1.5366844E-4,-4.8411638E-3,2.62E3,2.1199985E-7,3.1566668E1,1.8729467E8,-2.7183776E-3,3.307E3,3.7102813E-3,7.568201E-4,-3.0836621E-3,1.0747998E-3,-5.392878E-3,-3.3006503E-4,3.1681757E-3,4.949889E-5,-1.0641998E-3,-4.9084323E-3,-2.3102786E-3,4.358501E-4,2.782167E-3,-1.3352493E-3,2.3087817E-3,-2.8223842E-3,-3.4552247E-3,-1.4551475E-3,2.0838044E-4,3.847151E-3,-6.8767555E-4,-3.985468E-3,4.282039E-3,1.8890017E-4,-3.0996418E-4,1.7690883E-3,-5.667811E-3,-1.0058801E-3,5.650749E-4,3.2458662E-3,5.065372E-3,-4.668861E-4,-2.5756788E-3,9.3603204E-4,3.759572E-3,-1.0161304E-3,8.382565E-4,5.182506E-3],"split_indices":[2,68,4,67,3,5,0,6,60,53,8,67,60,5,0,0,43,73,0,0,0,48,66,0,0,5,68,12,70,0,0,0,0,46,8,57,0,68,0,53,60,2,57,0,43,0,0,0,8,0,0,70,53,0,0,10,73,0,53,1,0,32,71,60,6,0,0,0,0,60,30,7,10,5,43,9,73,1,68,52,10,46,71,0,3,0,0,48,0,67,48,60,0,0,68,62,3,53,68,0,0,60,0,48,67,0,0,0,0,10,3,0,0,2,52,73,7,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,8.74E2,1.51E2,8.36E2,3.8E1,1.41E2,1E1,2.29E2,6.07E2,2.4E1,1.4E1,1.21E2,2E1,2.25E2,4E0,9E0,5.98E2,2E1,4E0,8E0,6E0,7E1,5.1E1,1.5E1,5E0,2.06E2,1.9E1,2.51E2,3.47E2,6E0,1.4E1,6.2E1,8E0,3.1E1,2E1,1.99E2,7E0,1.4E1,5E0,2.01E2,5E1,1.86E2,1.61E2,7E0,5.5E1,4E0,4E0,4E0,2.7E1,1.3E1,7E0,1.09E2,9E1,7E0,7E0,1.06E2,9.5E1,5E0,4.5E1,1.8E2,6E0,1.43E2,1.8E1,2E1,3.5E1,2.2E1,5E0,8E0,5E0,4.7E1,6.2E1,4.4E1,4.6E1,9.8E1,8E0,3.6E1,5.9E1,3E1,1.5E1,1.69E2,1.1E1,1.05E2,3.8E1,4E0,1.4E1,8E0,1.2E1,3.1E1,4E0,2.9E1,1.8E1,5.8E1,4E0,5E0,3.9E1,3.6E1,1E1,2.1E1,7.7E1,4E0,4E0,2.7E1,9E0,2E1,3.9E1,4E0,2.6E1,9E0,6E0,1.55E2,1.4E1,4E0,7E0,6.2E1,4.3E1,2.1E1,1.7E1,5E0,9E0,2.5E1,6E0,1.6E1,1.3E1,1.3E1,5E0,1.3E1,4.5E1,1E1,2.9E1,3.1E1,5E0,4E0,6E0,1.5E1,6E0,2.9E1,4.8E1,7E0,2E1,1.1E1,9E0,4E0,3.5E1,6.4E1,9.1E1,4E0,1E1,3.6E1,2.6E1,3.8E1,5E0,1.6E1,5E0,1.1E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-5.610202E-3,-2.4882287E-2,1.9601159E-2,-2.1088654E-2,-1.0734692E-1,3.3337165E-2,-3.329409E-2,-1.920235E-2,-6.76749E-3,-5.0205804E-2,-1.2932322E-2,7.034963E-2,1.8605804E-2,-7.561385E-2,-4.219084E-3,-3.5982504E-2,-8.757486E-3,-8.6721204E-2,2.284558E-3,3.6221486E-2,1.01760015E-1,6.303687E-3,7.63384E-2,-1.0180549E-1,7.938385E-4,-3.498649E-2,5.13732E-2,-3.1398974E-2,-9.969479E-3,-4.586434E-3,-5.447526E-2,1.0931337E-3,-1.2901136E-1,5.254168E-2,-1.9473212E-3,1.251664E-1,2.3713157E-2,-1.3147849E-1,1.3926416E-2,1.0949343E-1,1.1473775E-2,-6.844725E-2,-1.0083909E-2,1.8404823E-2,-7.399956E-2,6.8098265E-3,1.8153418E-2,-2.3674041E-2,-1.0536943E-1,-1.2631728E-3,-6.1594028E-2,-7.3459116E-3,-1.8739425E-2,-8.61497E-3,-3.212482E-3,2.9867974E-3,-2.1978188E-3,8.739188E-3,9.657545E-2,-1.7832507E-3,3.1176666E-3,-1.6504169E-2,2.2435505E-3,1.8965283E-2,-5.8336486E-3,6.9539277E-3,7.605873E-2,-3.957373E-3,5.09897E-2,-4.846418E-3,1.8332672E-4,-4.652249E-4,3.1203397E-3,-9.6827786E-4,-1.2031306E-1,3.271095E-3,-1.2215601E-2,-3.4606993E-2,3.2107074E-2,-9.164775E-3,-4.7183715E-2,-4.5385556E-3,7.5813584E-2,-7.9530117E-4,-5.052743E-3,1.1326865E-3,-4.520894E-2,5.964977E-3,3.0496719E-3,1.5158699E-2,6.2111383E-3,2.0621126E-3,6.073613E-3,3.6040123E-3,-1.1279836E-4,-2.0216068E-3,-8.1687E-3,-2.8147774E-3,1.1237209E-3,-4.480446E-3,-3.0128418E-2,-1.9423816E-3,2.563411E-3,-6.5087766E-4,-4.4960114E-3,-8.701281E-3,5.6998145E-2,4.872609E-3,6.0986367E-4,-1.0892138E-2,-4.841695E-3,4.283715E-2,2.9996873E-3,-2.5197765E-2,-3.8575483E-3,4.241282E-2,-1.3967468E-2,-1.735478E-4,1.181702E-1,-2.3454702E-3,1.1158811E-3,7.434839E-2,2.5124997E-2,-8.852257E-3,4.6914063E-2,-1.4146487E-3,1.3151615E-3,5.5539194E-3,1.0102954E-3,-3.1213704E-3,-4.4343178E-4,7.643544E-3,2.1832879E-3,5.292591E-4,4.1281376E-3,-1.8285833E-3,1.8947485E-3,-2.8258418E-3,2.3657916E-4,6.076838E-3,8.2495087E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,43,45,47,-1,49,51,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,81,83,-1,85,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,89,-1,-1,91,-1,93,-1,-1,-1,-1,-1,95,-1,97,99,101,-1,103,105,107,-1,-1,-1,109,-1,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,-1,115,117,-1,-1,119,-1,121,123,125,-1,127,129,-1,131,-1,-1,133,135,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.921937E-1,1.7730671E-1,3.200739E-1,1.2728222E-1,2.3248035E-1,1.8898124E-1,1.1165167E-1,9.485452E-2,0E0,7.706805E-2,0E0,1.0243991E-1,1.777873E-1,9.1677904E-2,9.587104E-2,1.6465202E-1,6.3822396E-2,7.307405E-2,0E0,6.488806E-2,9.019852E-2,2.2028844E-1,9.33696E-2,9.772739E-2,0E0,7.634847E-2,6.1125174E-2,1.140745E-1,0E0,5.8462597E-2,1.0161383E-1,0E0,2.4627939E-2,5.2779123E-2,0E0,5.3091526E-2,3.8061373E-2,4.6445948E-1,1.4227024E-1,2.8332144E-2,6.578051E-2,6.3775264E-2,0E0,2.3066547E-2,4.9064934E-2,0E0,2.644825E-2,1.14543654E-1,9.346111E-2,7.431366E-2,3.4161016E-2,0E0,2.6298203E-2,0E0,0E0,0E0,0E0,0E0,2.031818E-2,0E0,0E0,0E0,0E0,8.236056E-2,0E0,0E0,2.5471538E-2,0E0,1.7515557E-2,0E0,0E0,0E0,0E0,0E0,3.984329E-2,0E0,2.1060588E-2,4.0656433E-2,5.607393E-2,0E0,2.0198671E-2,7.273318E-2,2.089008E-2,0E0,0E0,0E0,2.7488498E-2,0E0,0E0,6.2804975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.657894E-2,0E0,0E0,0E0,0E0,7.192415E-2,6.734069E-2,0E0,0E0,1.5201393E-2,0E0,2.9945523E-2,6.859696E-2,4.0450916E-2,0E0,3.7332352E-2,5.5786673E-2,0E0,2.2354305E-2,0E0,0E0,1.7852925E-2,4.0688187E-2,7.633678E-2,6.650075E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,52,52,58,58,63,63,66,66,68,68,74,74,76,76,77,77,78,78,80,80,81,81,82,82,86,86,89,89,100,100,105,105,106,106,109,109,111,111,112,112,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,44,46,48,-1,50,52,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,82,84,-1,86,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,90,-1,-1,92,-1,94,-1,-1,-1,-1,-1,96,-1,98,100,102,-1,104,106,108,-1,-1,-1,110,-1,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,-1,116,118,-1,-1,120,-1,122,124,126,-1,128,130,-1,132,-1,-1,134,136,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.850926E2,6.0692043E0,1E0,4.093403E6,1.6493898E6,2.165E3,5E0,1.8149019E2,-6.76749E-3,7.86127E6,-1.2932322E-2,7.257846E2,3.9152692E6,1.7308458E1,2.3751075E5,3.4E1,1E0,6.48334E5,2.284558E-3,4.1104166E8,1.9719212E1,1.2825651E0,1.5277778E0,4E0,7.938385E-4,8.7390656E2,3.2888E4,1E0,-9.969479E-3,4.2438595E1,2.1410204E2,1.0931337E-3,1.70698E6,2.4E1,-1.9473212E-3,1.55E2,1.10198E7,3.0984934E8,7.1E1,6.624E3,1E0,1.3026532E6,-1.0083909E-2,1.28636E7,4.3400474E0,6.8098265E-3,2.8827406E5,4.436827E7,3.7936746E3,2.89196E6,1.7181714E7,-7.3459116E-3,3.3272727E0,-8.61497E-3,-3.212482E-3,2.9867974E-3,-2.1978188E-3,8.739188E-3,6.9508715E0,-1.7832507E-3,3.1176666E-3,-1.6504169E-2,2.2435505E-3,4.1E1,-5.8336486E-3,6.9539277E-3,7.858646E-3,-3.957373E-3,2.7578741E1,-4.846418E-3,1.8332672E-4,-4.652249E-4,3.1203397E-3,-9.6827786E-4,4E0,3.271095E-3,1.332E3,9E0,1.45064E5,-9.164775E-3,6.666667E-1,1.9719212E1,1.7903225E0,-7.9530117E-4,-5.052743E-3,1.1326865E-3,1.1111689E8,5.964977E-3,3.0496719E-3,2E0,6.2111383E-3,2.0621126E-3,6.073613E-3,3.6040123E-3,-1.1279836E-4,-2.0216068E-3,-8.1687E-3,-2.8147774E-3,1.1237209E-3,-4.480446E-3,6.0147805E0,-1.9423816E-3,2.563411E-3,-6.5087766E-4,-4.4960114E-3,4.8679228E5,3.3E1,4.872609E-3,6.0986367E-4,4.7272725E0,-4.841695E-3,9.237895E0,1.9616238E2,5.25E0,-3.8575483E-3,1.9095E4,9.5679015E-2,-1.735478E-4,4.44E2,-2.3454702E-3,1.1158811E-3,1.0315458E8,5.658231E9,1E0,3.6828618E8,-1.4146487E-3,1.3151615E-3,5.5539194E-3,1.0102954E-3,-3.1213704E-3,-4.4343178E-4,7.643544E-3,2.1832879E-3,5.292591E-4,4.1281376E-3,-1.8285833E-3,1.8947485E-3,-2.8258418E-3,2.3657916E-4,6.076838E-3,8.2495087E-4],"split_indices":[67,68,6,66,60,44,6,70,0,5,0,4,43,61,48,3,19,46,0,7,71,69,68,6,0,4,9,30,0,73,67,0,5,3,0,10,62,7,3,2,27,43,0,60,61,0,48,5,48,62,12,0,69,0,0,0,0,0,71,0,0,0,0,3,0,0,49,0,71,0,0,0,0,0,8,0,44,70,7,0,71,71,68,0,0,0,7,0,0,32,0,0,0,0,0,0,0,0,0,0,69,0,0,0,0,60,8,0,0,69,0,71,71,69,0,12,73,0,0,0,0,7,46,29,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.011E3,5.73E2,4.38E2,5.49E2,2.4E1,3.48E2,9E1,5.42E2,7E0,1.9E1,5E0,9.8E1,2.5E2,3.6E1,5.4E1,2.07E2,3.35E2,1.4E1,5E0,4.8E1,5E1,2.07E2,4.3E1,2.8E1,8E0,3.5E1,1.9E1,2.03E2,4E0,3.08E2,2.7E1,4E0,1E1,4E1,8E0,3.8E1,1.2E1,1E1,1.97E2,2.8E1,1.5E1,2.3E1,5E0,1.5E1,2E1,4E0,1.5E1,1.85E2,1.8E1,2.92E2,1.6E1,6E0,2.1E1,4E0,6E0,3.6E1,4E0,1E1,2.8E1,5E0,7E0,4E0,6E0,1.91E2,6E0,1.1E1,1.7E1,4E0,1.1E1,1.5E1,8E0,1E1,5E0,1E1,1E1,5E0,1E1,1.55E2,3E1,6E0,1.2E1,2.81E2,1.1E1,9E0,7E0,8E0,1.3E1,1.2E1,1.6E1,1.86E2,5E0,1.2E1,5E0,7E0,4E0,5E0,5E0,4E0,6E0,9E0,1.46E2,7E0,2.3E1,8E0,4E0,2.64E2,1.7E1,7E0,4E0,9E0,4E0,5.6E1,1.3E2,1.35E2,1.1E1,2.4E1,2.4E2,9E0,8E0,4E0,5E0,1.9E1,3.7E1,1.03E2,2.7E1,1.23E2,1.2E1,4E0,2E1,1.7E1,2.23E2,4E0,4E0,4E0,1.5E1,7E0,3E1,2.1E1,8.2E1,6E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[-2.3832999E-3,-1.616509E-2,3.9556604E-2,-9.288826E-3,-1.17473E-1,4.8942212E-2,-3.1051794E-2,-7.5559868E-3,-1.0151071E-2,-1.9056906E-1,-1.5779432E-2,3.2307252E-2,8.266856E-2,4.129138E-3,-5.220338E-2,-5.2623376E-2,1.3472438E-4,4.1708484E-4,-2.4303037E-1,2.2713726E-3,-4.936446E-2,4.6477012E-2,-1.5822701E-2,7.083084E-2,9.563075E-3,-8.976092E-2,4.8905646E-4,-1.0384677E-1,-4.247589E-2,1.3308501E-1,-2.6712634E-3,-1.6859977E-2,-1.3859129E-1,-7.630779E-2,9.139906E-4,2.0981997E-2,7.079216E-2,-7.521709E-3,7.638667E-3,8.336396E-2,-1.6964907E-3,-4.3454796E-2,-8.879673E-3,-6.0306154E-2,-9.06809E-3,-1.346596E-2,-5.7981852E-2,2.0907472E-3,8.423091E-3,2.7398719E-5,-3.8388066E-3,-4.4804253E-2,-1.3787973E-2,-1.5046317E-3,-5.6492793E-3,5.1904593E-2,-3.101977E-2,8.074858E-2,-2.431322E-4,-1.4040481E-3,3.8905915E-2,1.3170636E-1,5.2274413E-2,-3.6403323E-3,1.3329152E-3,-4.2736847E-3,4.4880612E-4,3.5509577E-3,-2.8575424E-2,-8.383783E-2,-2.6361382E-2,-9.664212E-3,2.5248775E-2,1.3585996E-3,-4.5820232E-3,7.898426E-2,5.1168203E-3,9.873965E-4,-6.765529E-3,5.6036357E-3,5.527306E-2,3.8422006E-3,-2.226313E-4,9.6920095E-2,1.0159718E-2,3.888192E-2,6.386809E-3,-5.034832E-3,-1.1351237E-2,-5.5202052E-2,-6.970725E-3,1.6563621E-3,-6.29166E-2,-6.9045634E-3,-8.141564E-3,7.029964E-2,7.913334E-3,4.4405475E-2,6.5875757E-3,-4.0224628E-3,4.6553884E-2,1.0834249E-3,-3.0769308E-3,-1.620746E-3,7.15016E-2,5.501881E-3,1.3413954E-3,4.9138326E-2,-1.5955053E-3,1.1279745E-2,-2.8113066E-3,-3.6164382E-3,-2.0812389E-2,-1.0118706E-3,-4.020491E-3,1.1471406E-2,-2.001307E-2,4.5745578E-2,9.993264E-2,2.6743922E-3,4.2896364E-3,4.2437166E-3,9.256104E-5,3.5194184E-3,-1.9554637E-4,4.8606703E-3,4.9386084E-2,-1.0677816E-3,6.028534E-2,2.2145114E-3,-7.578226E-4,1.1115515E-3,-2.8743246E-3,7.0295564E-4,-4.546169E-3,-7.7078314E-3,-7.253777E-4,8.60132E-4,4.157949E-3,5.8099045E-3,1.5710271E-3,-8.8851004E-5,4.814243E-3,1.20445875E-5,3.7936168E-3,1.7642291E-3,4.20249E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,29,-1,31,-1,33,35,37,39,-1,41,-1,43,45,47,49,-1,51,53,-1,55,57,-1,59,61,-1,63,-1,65,-1,67,69,-1,-1,71,-1,73,-1,-1,-1,75,77,79,-1,-1,81,83,85,-1,-1,-1,-1,-1,87,89,91,93,95,-1,-1,97,99,101,-1,-1,103,-1,-1,105,-1,107,-1,-1,109,111,-1,-1,113,115,-1,117,119,121,-1,-1,123,-1,-1,-1,125,-1,-1,127,-1,129,-1,-1,131,-1,-1,133,135,137,139,141,-1,-1,-1,-1,-1,-1,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.052348E-1,5.478455E-1,1.7315972E-1,2.6781717E-1,3.6866283E-1,1.2580657E-1,8.363092E-2,2.5441167E-1,0E0,3.058567E-1,5.0412472E-2,1.0644382E-1,9.572089E-2,0E0,6.462185E-2,5.120641E-2,2.3463522E-1,0E0,2.4226606E-1,0E0,2.9455896E-2,7.290444E-2,1.2539126E-1,9.7418725E-2,0E0,7.229191E-2,0E0,5.6583375E-2,4.0153474E-2,4.7966853E-2,1.3369302E-1,0E0,1.9652194E-1,1.8136837E-2,0E0,9.809184E-2,4.6616286E-2,0E0,3.9233334E-2,8.780053E-2,0E0,3.549418E-2,0E0,3.0209124E-2,0E0,4.723507E-2,4.6027035E-2,0E0,0E0,1.4617217E-1,0E0,4.3269873E-2,0E0,0E0,0E0,4.7741756E-2,8.511515E-2,4.9072236E-2,0E0,0E0,3.523575E-2,5.232632E-2,4.0679403E-2,0E0,0E0,0E0,0E0,0E0,3.8925767E-2,5.310884E-2,6.593473E-2,1.9735524E-1,1.2908582E-1,0E0,0E0,4.645495E-2,6.1351433E-2,3.0409753E-2,0E0,0E0,5.382988E-2,0E0,0E0,2.5138259E-2,0E0,2.8804958E-2,0E0,0E0,2.869269E-2,1.7417833E-2,0E0,0E0,1.5892394E-2,1.0284511E-1,0E0,2.8898597E-2,5.3594798E-2,3.402076E-2,0E0,0E0,1.6872836E-2,0E0,0E0,0E0,1.7169341E-2,0E0,0E0,2.71656E-2,0E0,1.9051056E-2,0E0,0E0,2.1933664E-2,0E0,0E0,7.472952E-2,1.5502375E-1,3.0738868E-2,2.977541E-2,5.3883057E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.2134607E-2,0E0,1.5508845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,32,32,33,33,35,35,36,36,38,38,39,39,41,41,43,43,45,45,46,46,49,49,51,51,55,55,56,56,57,57,60,60,61,61,62,62,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,80,80,83,83,85,85,88,88,89,89,92,92,93,93,95,95,96,96,97,97,100,100,104,104,107,107,109,109,112,112,115,115,116,116,117,117,118,118,119,119,126,126,128,128],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,30,-1,32,-1,34,36,38,40,-1,42,-1,44,46,48,50,-1,52,54,-1,56,58,-1,60,62,-1,64,-1,66,-1,68,70,-1,-1,72,-1,74,-1,-1,-1,76,78,80,-1,-1,82,84,86,-1,-1,-1,-1,-1,88,90,92,94,96,-1,-1,98,100,102,-1,-1,104,-1,-1,106,-1,108,-1,-1,110,112,-1,-1,114,116,-1,118,120,122,-1,-1,124,-1,-1,-1,126,-1,-1,128,-1,130,-1,-1,132,-1,-1,134,136,138,140,142,-1,-1,-1,-1,-1,-1,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,9.932432E0,2.856934E7,1.6776881E7,9.4E1,7.650379E8,2.0263722E8,2.71E2,-1.0151071E-2,1E0,8.303E3,4.88E2,3.8771296E3,4.129138E-3,1.3655363E2,8.8324326E-1,2.04115E5,4.1708484E-4,3.2214062E2,2.2713726E-3,1.23E4,2.35184E5,5.11E2,2E1,9.563075E-3,1.7026364E3,4.8905646E-4,6.737099E7,3.8275862E0,1.766298E2,3.713753E7,-1.6859977E-2,5.7420593E2,8.620714E2,9.139906E-4,2.08E3,2.9E1,-7.521709E-3,2.6172416E10,1.4388715E0,-1.6964907E-3,9.3326636E-2,-8.879673E-3,2.66E2,-9.06809E-3,2.4E1,7.336111E4,2.0907472E-3,8.423091E-3,4.4935583E2,-3.8388066E-3,2.5144437E2,-1.3787973E-2,-1.5046317E-3,-5.6492793E-3,2.8827406E5,9.427676E2,4.029152E8,-2.431322E-4,-1.4040481E-3,3.0977E4,1.188E0,9.059046E6,-3.6403323E-3,1.3329152E-3,-4.2736847E-3,4.4880612E-4,3.5509577E-3,5.4E1,6.48334E5,3.466E3,2.483E3,2.8E0,1.3585996E-3,-4.5820232E-3,1.57922E5,3.2952412E5,1E0,-6.765529E-3,5.6036357E-3,8.484036E5,3.8422006E-3,-2.226313E-4,1.2376238E0,1.0159718E-2,3.116962E7,6.386809E-3,-5.034832E-3,8E0,2.2970297E0,-6.970725E-3,1.6563621E-3,4.8121494E2,1.5357143E0,-8.141564E-3,1.10566504E8,1.0358873E3,2.4434292E7,6.5875757E-3,-4.0224628E-3,1.83154E6,1.0834249E-3,-3.0769308E-3,-1.620746E-3,1.1621469E1,5.501881E-3,1.3413954E-3,4.6750406E4,-1.5955053E-3,2.855464E5,-2.8113066E-3,-3.6164382E-3,2.6412E5,-1.0118706E-3,-4.020491E-3,1.6942337E1,1.16408E5,6.083812E2,2.1609572E7,3.2585382E10,4.2896364E-3,4.2437166E-3,9.256104E-5,3.5194184E-3,-1.9554637E-4,4.8606703E-3,2.0619047E1,-1.0677816E-3,8.072373E1,2.2145114E-3,-7.578226E-4,1.1115515E-3,-2.8743246E-3,7.0295564E-4,-4.546169E-3,-7.7078314E-3,-7.253777E-4,8.60132E-4,4.157949E-3,5.8099045E-3,1.5710271E-3,-8.8851004E-5,4.814243E-3,1.20445875E-5,3.7936168E-3,1.7642291E-3,4.20249E-3],"split_indices":[2,69,60,62,0,5,7,2,0,8,44,0,67,0,71,68,5,0,70,0,10,1,0,3,0,4,0,7,69,4,60,0,67,4,0,44,8,0,46,69,0,53,0,1,0,2,60,0,0,70,0,48,0,0,0,48,4,7,0,0,44,68,43,0,0,0,0,0,2,46,1,2,69,0,0,1,48,6,0,0,43,0,0,69,0,62,0,0,32,68,0,0,48,68,0,5,4,12,0,0,9,0,0,0,71,0,0,48,0,60,0,0,7,0,0,71,7,4,60,46,0,0,0,0,0,0,71,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.045E3,7.87E2,2.58E2,7.38E2,4.9E1,2.28E2,3E1,7.33E2,5E0,2.8E1,2.1E1,1.54E2,7.4E1,4E0,2.6E1,1.06E2,6.27E2,6E0,2.2E1,7E0,1.4E1,1.19E2,3.5E1,6.9E1,5E0,1.6E1,1E1,1.6E1,9E1,1.2E1,6.15E2,9E0,1.3E1,1E1,4E0,5.9E1,6E1,4E0,3.1E1,6.2E1,7E0,1.2E1,4E0,1.2E1,4E0,3.2E1,5.8E1,5E0,7E0,5.96E2,1.9E1,9E0,4E0,6E0,4E0,3.7E1,2.2E1,5.3E1,7E0,1.4E1,1.7E1,2.3E1,3.9E1,8E0,4E0,8E0,4E0,4E0,2.8E1,3.1E1,2.7E1,4.31E2,1.65E2,4E0,5E0,2.3E1,1.4E1,1.8E1,4E0,1.9E1,3.4E1,8E0,9E0,1.8E1,5E0,3.5E1,4E0,4E0,2.4E1,2.3E1,8E0,1E1,1.7E1,4.25E2,6E0,4.5E1,1.2E2,1.6E1,7E0,4E0,1E1,1.4E1,4E0,5E0,2.9E1,1.3E1,5E0,3.1E1,4E0,1.7E1,7E0,1.3E1,1E1,7E0,1E1,1.77E2,2.48E2,2.6E1,1.9E1,1.14E2,6E0,7E0,9E0,6E0,4E0,1E1,1.9E1,4E0,2.7E1,7E0,1E1,5E0,5E0,1.72E2,5E0,6E0,2.42E2,1.7E1,9E0,1.3E1,6E0,1.1E2,4E0,8E0,1.1E1,1.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-2.1796166E-3,-1.5955565E-2,3.6903016E-2,-4.0135726E-2,-3.23034E-3,4.6634834E-2,-2.1685509E-2,-2.7762825E-2,-1.353065E-1,-1.2487384E-3,-9.219274E-3,3.0976562E-2,1.04161955E-1,-7.263966E-2,4.1796535E-2,1.394602E-3,-3.4222674E-2,1.6698814E-3,-2.2671686E-1,-4.4431672E-3,8.920827E-2,8.803175E-2,1.8302405E-2,9.845702E-3,8.400307E-2,-2.9315378E-3,-1.4208128E-1,5.1902547E-3,-1.3311481E-2,-5.879757E-2,-1.7828032E-2,-3.740043E-2,3.3107377E-3,-2.015563E-2,-8.557978E-2,-1.0228534E-2,5.048864E-2,1.3515306E-1,1.9705143E-4,4.662847E-3,-2.0242562E-4,3.177532E-2,-1.5183443E-2,5.7543978E-2,1.295571E-1,-2.3803602E-3,2.5062182E-3,-9.56011E-3,-1.0566904E-3,-3.3320778E-3,1.413662E-3,-4.298241E-2,-9.409306E-3,3.229396E-2,-2.9114697E-2,4.1061343E-4,-3.520769E-3,-1.0646119E-2,7.2326744E-4,-7.701555E-3,-6.67388E-3,8.639728E-2,7.388428E-3,2.4377177E-3,8.070683E-3,1.8092034E-2,9.4918504E-2,8.549523E-4,-3.324707E-2,4.28422E-3,-8.469955E-3,2.0201996E-3,7.243657E-3,-5.9466425E-2,-1.6953155E-3,-1.6933965E-3,5.7313085E-2,-3.6026187E-2,3.3629553E-3,-3.6745653E-3,-7.550389E-2,1.251894E-3,9.9867254E-2,1.8392664E-3,-1.8120952E-2,-4.08743E-3,2.4077382E-2,1.3038072E-1,1.3795211E-3,-3.1572036E-3,-1.1180756E-2,-2.708137E-3,1.3194776E-3,-4.7482144E-2,-6.2304595E-3,1.9251611E-2,-2.003988E-3,4.5912773E-3,1.2934699E-3,-5.9667155E-2,-9.664591E-3,-4.4623144E-2,3.0047023E-3,-8.255444E-4,-9.818465E-2,6.4058197E-3,4.7211863E-2,1.3401976E-3,-1.953801E-3,4.7398186E-3,1.7963966E-2,8.694866E-3,1.9029209E-3,9.350864E-4,-2.6294128E-3,1.8676422E-3,-5.6194555E-2,-1.5724283E-3,1.8410534E-3,-1.9769514E-2,-8.605686E-2,2.4962673E-4,-3.1029787E-2,-3.1112049E-2,-7.1276873E-3,7.8498935E-3,-3.1770267E-2,-5.567701E-3,-9.4650546E-4,-1.8824694E-4,3.7921108E-3,1.5570229E-3,4.896436E-2,-2.8982072E-3,4.0008436E-4,1.7682387E-3,-2.2826153E-3,-4.811528E-3,-9.714286E-4,9.5939456E-4,-3.1087901E-3,1.1182362E-3,-2.1502655E-3,-3.5281477E-3,4.648629E-4,-3.7354912E-4,-5.5599217E-3,-3.7694263E-4,4.5463024E-3,4.512656E-3,7.610985E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,-1,43,45,47,-1,49,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,73,-1,75,77,-1,-1,-1,-1,79,-1,81,83,-1,-1,85,87,-1,89,-1,91,-1,-1,93,95,-1,97,99,-1,101,103,-1,105,-1,107,-1,109,111,-1,-1,113,-1,-1,115,-1,117,-1,-1,-1,119,121,123,125,-1,127,-1,129,-1,-1,-1,131,-1,-1,-1,-1,-1,133,-1,-1,135,137,-1,139,141,-1,143,145,-1,-1,-1,-1,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5567956E-1,2.3464392E-1,1.5438774E-1,3.0593616E-1,1.9465898E-1,2.0485252E-1,1.2918834E-1,8.863005E-2,3.7601048E-1,1.4400873E-1,0E0,1.3072094E-1,8.971977E-2,1.0607766E-1,7.147612E-2,0E0,8.396293E-2,3.823042E-2,4.9504668E-1,1.5337855E-1,6.576924E-2,3.977576E-2,6.860643E-2,0E0,4.4373453E-2,3.613718E-2,8.030677E-2,0E0,3.061473E-2,1.8370345E-1,7.335113E-2,1.6844671E-2,0E0,0E0,1.7958742E-1,1.467779E-1,7.055449E-2,2.6616395E-2,0E0,0E0,0E0,9.159539E-2,2.739149E-2,6.8153255E-2,2.8207362E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.2303612E-2,0E0,4.3501407E-2,7.6577306E-2,0E0,0E0,0E0,0E0,1.1685641E-1,0E0,1.7262861E-2,1.8917616E-2,0E0,0E0,5.9535533E-2,3.9941788E-2,0E0,2.1692153E-2,0E0,2.2101728E-2,0E0,0E0,4.3291867E-2,2.091658E-2,0E0,1.9415442E-2,6.112148E-2,0E0,1.1117773E-1,2.9618219E-2,0E0,3.4303486E-2,0E0,1.603558E-2,0E0,3.9438218E-2,5.4136246E-2,0E0,0E0,2.8455315E-2,0E0,0E0,3.9800413E-2,0E0,1.8918943E-2,0E0,0E0,0E0,5.304885E-2,1.5620215E-2,7.965191E-2,5.9221376E-2,0E0,2.7026638E-2,0E0,1.8218156E-2,0E0,0E0,0E0,4.112212E-2,0E0,0E0,0E0,0E0,0E0,1.9625574E-2,0E0,0E0,3.973873E-2,3.4279972E-2,0E0,3.8961586E-2,4.551473E-2,0E0,5.815331E-2,8.9237005E-2,0E0,0E0,0E0,0E0,5.161716E-2,4.2707123E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,41,41,42,42,43,43,44,44,51,51,53,53,54,54,59,59,61,61,62,62,65,65,66,66,68,68,70,70,73,73,74,74,76,76,77,77,79,79,80,80,82,82,84,84,86,86,87,87,90,90,93,93,95,95,99,99,100,100,101,101,102,102,104,104,106,106,110,110,116,116,119,119,120,120,122,122,123,123,125,125,126,126,131,131,132,132],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,-1,44,46,48,-1,50,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,74,-1,76,78,-1,-1,-1,-1,80,-1,82,84,-1,-1,86,88,-1,90,-1,92,-1,-1,94,96,-1,98,100,-1,102,104,-1,106,-1,108,-1,110,112,-1,-1,114,-1,-1,116,-1,118,-1,-1,-1,120,122,124,126,-1,128,-1,130,-1,-1,-1,132,-1,-1,-1,-1,-1,134,-1,-1,136,138,-1,140,142,-1,144,146,-1,-1,-1,-1,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.167E3,1E0,2.7555334E7,3.7798166E0,8.432552E6,1.4176E4,3.416447E-2,4.2175066E-1,3.95E2,5.5303917E8,-9.219274E-3,3.9914E4,8.025477E0,1.480315E0,5.735198E-1,1.394602E-3,1.5222145E2,1.3E2,2.277451E2,5.817547E2,8.051603E7,9.033567E6,8E0,9.845702E-3,4.73909E0,8.969567E-9,1.3655363E2,5.1902547E-3,1.01E3,1.1244019E2,1.2E1,8E1,3.3107377E-3,-2.015563E-2,4.5E0,4.093403E6,4.612245E0,8.7568E4,1.9705143E-4,4.662847E-3,-2.0242562E-4,7.659164E1,3.9041237E1,1.4E1,4.29E2,-2.3803602E-3,2.5062182E-3,-9.56011E-3,-1.0566904E-3,-3.3320778E-3,1.413662E-3,3.3527696E-1,-9.409306E-3,3.7788504E7,4.4E1,4.1061343E-4,-3.520769E-3,-1.0646119E-2,7.2326744E-4,2.2968E4,-6.67388E-3,4.7773536E7,1.763E3,2.4377177E-3,8.070683E-3,8.118812E-1,1.3208092E0,8.549523E-4,2.4199E4,4.28422E-3,3.6349E4,2.0201996E-3,7.243657E-3,6.737509E7,4.08E2,-1.6933965E-3,4E0,7.5982756E5,3.3629553E-3,1.5933333E2,1.9067797E1,1.251894E-3,1.8729467E8,1.8392664E-3,6.232639E2,-4.08743E-3,2.3636363E0,4.2904656E8,1.3795211E-3,-3.1572036E-3,1.800554E-2,-2.708137E-3,1.3194776E-3,2.1126761E0,-6.2304595E-3,2.3857143E1,-2.003988E-3,4.5912773E-3,1.2934699E-3,3.7509E4,8.0274E4,1.4209677E2,1E0,-8.255444E-4,2.9308079E1,6.4058197E-3,1.935E3,1.3401976E-3,-1.953801E-3,4.7398186E-3,7.618E3,8.694866E-3,1.9029209E-3,9.350864E-4,-2.6294128E-3,1.8676422E-3,5.0406504E0,-1.5724283E-3,1.8410534E-3,1E0,1.8383686E5,2.4962673E-4,1.3358E4,3.757994E0,-7.1276873E-3,3.7974394E-6,1.2375429E8,-5.567701E-3,-9.4650546E-4,-1.8824694E-4,3.7921108E-3,2.8E1,6.03235E5,-2.8982072E-3,4.0008436E-4,1.7682387E-3,-2.2826153E-3,-4.811528E-3,-9.714286E-4,9.5939456E-4,-3.1087901E-3,1.1182362E-3,-2.1502655E-3,-3.5281477E-3,4.648629E-4,-3.7354912E-4,-5.5599217E-3,-3.7694263E-4,4.5463024E-3,4.512656E-3,7.610985E-4],"split_indices":[2,29,60,68,62,2,72,68,2,5,0,9,71,68,53,0,4,2,67,67,7,60,32,0,50,52,71,0,8,67,3,44,0,0,73,66,69,12,0,0,0,73,73,3,0,0,0,0,0,0,0,71,0,7,3,0,0,0,0,44,0,7,2,0,0,68,68,0,44,0,44,0,0,7,44,0,8,43,0,4,73,0,7,0,67,0,73,7,0,0,72,0,0,69,0,4,0,0,0,1,1,70,19,0,71,0,2,0,0,0,10,0,0,0,0,0,69,0,0,6,48,0,9,73,0,53,7,0,0,0,0,3,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,7.62E2,2.68E2,2.62E2,5E2,2.3E2,3.8E1,2.33E2,2.9E1,4.96E2,4E0,1.82E2,4.8E1,2.1E1,1.7E1,2.3E1,2.1E2,1.2E1,1.7E1,4.8E2,1.6E1,3.2E1,1.5E2,6E0,4.2E1,1.1E1,1E1,7E0,1E1,8.3E1,1.27E2,8E0,4E0,6E0,1.1E1,4.35E2,4.5E1,1E1,6E0,2.8E1,4E0,1.07E2,4.3E1,2.8E1,1.4E1,6E0,5E0,6E0,4E0,4E0,6E0,7.6E1,7E0,2.3E1,1.04E2,4E0,4E0,4E0,7E0,4.28E2,7E0,2.4E1,2.1E1,4E0,6E0,8.9E1,1.8E1,1.5E1,2.8E1,1.8E1,1E1,4E0,1E1,5.4E1,2.2E1,6E0,1.7E1,9.8E1,6E0,4.05E2,2.3E1,5E0,1.9E1,9E0,1.2E1,4E0,8.5E1,1.1E1,7E0,1E1,1.8E1,4E0,6E0,4.8E1,6E0,1.5E1,7E0,6E0,1.1E1,5.1E1,4.7E1,5.6E1,3.49E2,7E0,1.6E1,1E1,9E0,4E0,8E0,5E0,8E1,6E0,5E0,1.1E1,7E0,4E0,4.4E1,4E0,1.1E1,2.1E1,3E1,2.8E1,1.9E1,5.1E1,5E0,3.07E2,4.2E1,1.2E1,4E0,4E0,5E0,5.3E1,2.7E1,4E1,4E0,7E0,1.4E1,2.3E1,7E0,8E0,1.1E1,1.1E1,4E1,7E0,3E2,3.4E1,8E0,4.9E1,4E0,1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-4.394324E-3,-1.4657774E-2,2.9410226E-2,-4.259483E-2,-6.3885613E-3,3.9567206E-2,-2.9179737E-2,-2.3801986E-2,-9.635634E-2,-4.623298E-3,-8.238197E-3,2.7196303E-2,7.688229E-2,3.4070916E-2,-8.423665E-2,-4.785559E-3,-1.7770644E-2,-1.668953E-1,-4.314906E-2,-2.6686054E-2,7.5233844E-3,-4.5937024E-2,3.2708857E-2,1.1982903E-1,4.0392622E-2,4.835181E-3,-4.0303577E-2,-1.1631969E-1,2.1543658E-4,9.7151855E-3,-3.6102008E-2,-2.4053605E-1,-2.906722E-4,-7.035766E-2,2.0623659E-3,-1.682742E-2,-8.981411E-2,3.362829E-2,-2.304895E-3,-6.3896207E-3,1.2290553E-3,7.2906494E-2,1.9254409E-2,1.597043E-1,2.5875287E-3,3.2092799E-3,1.5024359E-3,-4.0444606E-3,6.928843E-4,-8.983178E-3,-3.0318138E-3,2.989017E-2,-2.617782E-3,-4.8118174E-2,3.9548497E-3,-4.6595233E-3,-1.4563231E-2,-6.685542E-3,-1.3469676E-2,-2.8876303E-2,1.8395469E-2,-9.334916E-3,-5.6627978E-2,-3.545487E-3,3.8810607E-2,-1.0196628E-2,5.080911E-2,4.8319746E-2,5.6435647E-3,2.8097006E-2,-3.703017E-3,3.3276656E-3,8.808678E-3,1.952945E-3,-3.3348324E-3,4.6422314E-2,-2.417001E-3,-5.433554E-2,1.2662942E-3,-2.1104075E-3,1.5051365E-3,1.993359E-3,-3.1490335E-3,-2.381475E-2,-6.217197E-3,7.877063E-2,-2.1794997E-2,-7.714783E-2,5.9857586E-5,5.0976045E-2,-4.1335197E-3,-5.2364026E-3,-1.1284767E-2,5.1628323E-3,3.2245267E-2,2.8856678E-3,1.4006702E-4,6.374135E-3,2.2018397E-2,5.0710775E-3,3.3900246E-2,-4.018954E-2,-8.749277E-2,-3.4528847E-3,-1.6585376E-2,4.3282816E-3,8.438426E-4,-3.298905E-3,-5.078036E-3,1.1687091E-4,-4.7057704E-3,7.995242E-2,1.9949615E-2,-5.5630016E-3,2.66841E-2,-2.8890943E-2,1.3977782E-2,8.5900427E-4,3.764203E-3,-1.6856841E-3,3.0916678E-2,1.0237182E-3,3.965133E-3,-3.1024155E-3,-7.5311546E-4,-5.815104E-3,-1.7638017E-3,-7.032845E-3,-3.0021719E-3,-1.7917902E-3,8.817341E-4,8.689986E-2,6.4403075E-4,-3.6498033E-3,4.0334754E-2,-2.3064863E-4,2.5729307E-3,-1.16037935E-1,-2.0676663E-2,-5.7259955E-2,2.5728125E-2,6.1875656E-2,5.8869147E-3,-6.147545E-5,-3.2808736E-3,4.558375E-3,1.8771517E-3,5.738897E-4,3.572522E-3,-7.2325376E-4,-8.926262E-3,-2.0561318E-4,-2.5929569E-3,-4.9445713E-3,4.587254E-4,2.0109846E-3,-2.6904224E-4,1.1480399E-3,5.290877E-3,-2.572983E-3,1.0134846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,53,55,-1,57,-1,59,61,63,65,-1,-1,67,69,71,-1,-1,73,-1,-1,-1,-1,75,-1,77,79,-1,-1,-1,81,83,85,-1,87,-1,89,91,93,95,-1,97,-1,-1,-1,-1,-1,99,-1,101,-1,-1,-1,-1,-1,103,-1,105,107,109,-1,111,113,115,-1,-1,117,-1,-1,-1,119,-1,121,123,125,-1,127,-1,-1,129,-1,-1,-1,131,133,-1,135,137,139,-1,-1,-1,141,-1,-1,-1,-1,-1,-1,143,-1,-1,-1,145,-1,-1,147,-1,-1,149,151,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4703547E-1,1.7691827E-1,1.3981211E-1,1.7451459E-1,1.791532E-1,8.977169E-2,1.2475352E-1,6.1196752E-2,1.61692E-1,1.5777847E-1,0E0,6.237842E-2,7.194832E-2,9.346831E-2,5.4733828E-2,0E0,6.232463E-2,2.2319901E-1,6.763412E-2,1.285612E-1,9.75727E-2,7.73851E-2,7.485424E-2,4.70424E-2,3.1554762E-2,0E0,2.4825234E-2,4.2687103E-2,0E0,6.865272E-2,3.5808697E-2,1.03833914E-1,0E0,8.687762E-2,0E0,7.777135E-2,9.285672E-2,6.2464677E-2,1.1662583E-1,0E0,0E0,3.6479726E-2,9.630768E-2,2.1558136E-2,0E0,0E0,4.2105917E-2,0E0,0E0,0E0,0E0,5.5432133E-2,0E0,2.8255582E-2,2.7062578E-2,0E0,0E0,0E0,4.3195836E-2,6.955447E-2,1.1576258E-1,0E0,2.7437381E-2,0E0,5.245413E-2,2.7968952E-1,3.7124515E-2,1.5822046E-2,0E0,6.308238E-2,0E0,0E0,0E0,0E0,0E0,2.245424E-2,0E0,2.1342948E-2,0E0,0E0,0E0,0E0,0E0,4.6988428E-2,0E0,1.5437402E-2,4.6904568E-2,3.3728853E-2,0E0,6.8529576E-2,8.563318E-2,1.0857547E-1,0E0,0E0,1.797624E-2,0E0,0E0,0E0,5.058517E-2,0E0,1.577929E-2,2.4025545E-2,2.3583218E-2,0E0,5.3172532E-2,0E0,0E0,2.059815E-2,0E0,0E0,0E0,1.7133206E-2,8.185482E-2,0E0,1.7736774E-2,7.419665E-2,1.1196615E-1,0E0,0E0,0E0,6.39521E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.5660185E-2,0E0,0E0,0E0,1.55928135E-2,0E0,0E0,3.3115365E-2,0E0,0E0,6.669875E-2,5.6694962E-2,6.459583E-2,6.428022E-2,6.927925E-2,4.7676142E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,46,46,51,51,53,53,54,54,58,58,59,59,60,60,62,62,64,64,65,65,66,66,67,67,69,69,75,75,77,77,83,83,85,85,86,86,87,87,89,89,90,90,91,91,94,94,98,98,100,100,101,101,102,102,104,104,107,107,111,111,112,112,114,114,115,115,116,116,120,120,127,127,131,131,134,134,137,137,138,138,139,139,140,140,141,141,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,54,56,-1,58,-1,60,62,64,66,-1,-1,68,70,72,-1,-1,74,-1,-1,-1,-1,76,-1,78,80,-1,-1,-1,82,84,86,-1,88,-1,90,92,94,96,-1,98,-1,-1,-1,-1,-1,100,-1,102,-1,-1,-1,-1,-1,104,-1,106,108,110,-1,112,114,116,-1,-1,118,-1,-1,-1,120,-1,122,124,126,-1,128,-1,-1,130,-1,-1,-1,132,134,-1,136,138,140,-1,-1,-1,142,-1,-1,-1,-1,-1,-1,144,-1,-1,-1,146,-1,-1,148,-1,-1,150,152,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,3.3832976E7,1E0,6E0,1.6776881E7,1.9851073E3,1.8645384E1,9E0,9.66E2,7.09E2,-8.238197E-3,7.549744E7,1.0284041E2,5.7E1,4.6E1,-4.785559E-3,3.373913E0,1.9874651E-5,4.83929E9,3.477E4,9.777778E0,2.0342858E0,1.684E0,1.137863E6,2.0666666E0,4.835181E-3,4.6779457E1,4.433E3,2.1543658E-4,2.4796897E3,2.7777777E0,2.7E1,-2.906722E-4,1.8154167E2,2.0623659E-3,5.51E2,1E1,6.2886596E-1,2.1512408E5,-6.3896207E-3,1.2290553E-3,4.455844E1,1.6553869E7,5.36763E5,2.5875287E-3,3.2092799E-3,8.72419E6,-4.0444606E-3,6.928843E-4,-8.983178E-3,-3.0318138E-3,8E0,-2.617782E-3,2.153533E6,4.36875E1,-4.6595233E-3,-1.4563231E-2,-6.685542E-3,4.9094446E2,1.0828667E5,2.1924414E2,-9.334916E-3,2.73E3,-3.545487E-3,1E0,2.0232125E5,5E0,1.0130841E1,5.6435647E-3,5.617715E6,-3.703017E-3,3.3276656E-3,8.808678E-3,1.952945E-3,-3.3348324E-3,2.6E1,-2.417001E-3,4.5E0,1.2662942E-3,-2.1104075E-3,1.5051365E-3,1.993359E-3,-3.1490335E-3,5.4E1,-6.217197E-3,6.69E2,1.7214285E1,3E0,5.9857586E-5,3.7936746E3,7.9E2,5.338167E5,-1.1284767E-2,5.1628323E-3,3.264E3,2.8856678E-3,1.4006702E-4,6.374135E-3,1.1724138E0,5.0710775E-3,3.1026666E2,1.8187732E4,4.353644E6,-3.4528847E-3,2.49E2,4.3282816E-3,8.438426E-4,1.7594937E0,-5.078036E-3,1.1687091E-4,-4.7057704E-3,1.8E1,2.53692E5,-5.5630016E-3,9.166947E6,5.4E1,1.2397727E1,8.5900427E-4,3.764203E-3,-1.6856841E-3,1.775373E1,1.0237182E-3,3.965133E-3,-3.1024155E-3,-7.5311546E-4,-5.815104E-3,-1.7638017E-3,1E0,-3.0021719E-3,-1.7917902E-3,8.817341E-4,1.2673605E7,6.4403075E-4,-3.6498033E-3,5.603014E6,-2.3064863E-4,2.5729307E-3,1.17E2,3.401282E1,3.270662E6,2E0,9.293198E2,1.4390917E8,-6.147545E-5,-3.2808736E-3,4.558375E-3,1.8771517E-3,5.738897E-4,3.572522E-3,-7.2325376E-4,-8.926262E-3,-2.0561318E-4,-2.5929569E-3,-4.9445713E-3,4.587254E-4,2.0109846E-3,-2.6904224E-4,1.1480399E-3,5.290877E-3,-2.572983E-3,1.0134846E-3],"split_indices":[2,7,6,8,62,67,71,70,12,2,0,7,73,6,6,0,69,57,5,1,73,68,69,9,68,0,73,10,0,48,68,3,0,67,0,2,3,68,48,0,0,73,66,1,0,0,58,0,0,0,0,32,0,43,70,0,0,0,4,48,67,0,10,0,30,48,3,71,0,12,0,0,0,0,0,4,0,73,0,0,0,0,0,2,0,2,73,8,0,48,2,43,0,0,10,0,0,0,68,0,67,43,60,0,10,0,0,71,0,0,0,3,12,0,60,0,73,0,0,0,71,0,0,0,0,0,0,19,0,0,0,60,0,0,9,0,0,10,73,12,32,4,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,7.66E2,2.32E2,1.74E2,5.92E2,1.98E2,3.4E1,1.3E2,4.4E1,5.87E2,5E0,1.5E2,4.8E1,1.6E1,1.8E1,8E0,1.22E2,1.8E1,2.6E1,2.08E2,3.79E2,1E1,1.4E2,2.1E1,2.7E1,8E0,8E0,1.3E1,5E0,4.9E1,7.3E1,1.2E1,6E0,2E1,6E0,1.81E2,2.7E1,1.03E2,2.76E2,4E0,6E0,3.4E1,1.06E2,1.2E1,9E0,1.5E1,1.2E1,4E0,4E0,4E0,9E0,3.8E1,1.1E1,5.6E1,1.7E1,5E0,7E0,8E0,1.2E1,1.35E2,4.6E1,5E0,2.2E1,4E0,9.9E1,2.41E2,3.5E1,2.4E1,1E1,9.8E1,8E0,4E0,8E0,8E0,4E0,3.2E1,6E0,5.2E1,4E0,6E0,1.1E1,6E0,6E0,1.3E2,5E0,1.8E1,2.8E1,1.6E1,6E0,7.7E1,2.2E1,2.37E2,4E0,7E0,2.8E1,1.8E1,6E0,4E0,9.4E1,4E0,2.8E1,3.8E1,1.4E1,1.5E1,1.15E2,1.4E1,4E0,2.4E1,4E0,4E0,1.2E1,3.9E1,3.8E1,4E0,1.8E1,1.06E2,1.31E2,2.3E1,5E0,1.2E1,8.2E1,2.4E1,4E0,1.7E1,2.1E1,7E0,7E0,9.7E1,1.8E1,9E0,1.5E1,3.5E1,4E0,6E0,3.2E1,9E0,9E0,8E0,9.8E1,1.8E1,1.13E2,3.6E1,4.6E1,9E1,7E0,2.7E1,8E0,1.9E1,1.3E1,4E0,4E0,6.8E1,3E1,1E1,8E0,7.2E1,4.1E1,2.2E1,1.4E1,9E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-3.5064819E-3,-1.7043624E-2,2.9958205E-2,-1.5313692E-2,-8.929629E-3,-8.798362E-3,3.3642247E-2,-4.0011212E-2,-2.3347524E-3,4.154353E-2,-2.1721793E-2,-2.9777834E-2,-1.2304945E-1,-7.003079E-4,-6.0048187E-3,3.3120025E-2,9.920983E-2,-4.1052856E-2,5.0914856E-3,-3.6685344E-2,2.6397053E-2,9.714439E-3,-2.299082E-1,-9.401194E-3,2.0300383E-2,5.3411988E-3,2.8317055E-2,6.0303295E-3,2.5415237E-3,3.367115E-3,-5.9281316E-2,-4.758875E-2,-1.1238481E-2,2.5520409E-3,-7.0842216E-4,-3.6390233E-3,2.669943E-3,-1.8471073E-2,-1.4443144E-3,-1.1500854E-2,4.9408553E-3,4.9678985E-2,8.2272326E-4,6.7663215E-2,1.9386671E-2,-2.7409794E-2,-1.270614E-1,-1.4830986E-2,-6.260465E-2,2.6524141E-2,-2.7662931E-2,-4.8351116E-3,-5.0283622E-2,6.231547E-2,-2.119309E-3,-1.5919986E-3,1.63569E-2,2.8714927E-2,1.0322834E-1,3.582367E-2,-7.643471E-3,-5.8292434E-2,9.3525334E-4,-2.2455424E-3,-8.320666E-3,-3.2136608E-2,3.8670804E-3,-1.07570544E-1,-5.0696123E-2,-4.5756376E-5,2.978936E-3,1.4405181E-3,-4.415626E-2,-5.1902514E-2,3.7245703E-4,-9.0679474E-2,-2.0690499E-2,6.923772E-2,5.0315005E-4,4.1536473E-2,-1.7199343E-2,2.7676343E-3,-1.04027036E-4,5.687251E-3,8.7700016E-4,-1.0285859E-2,5.2666947E-2,4.4412473E-3,-1.5758364E-2,-3.8619095E-3,-5.050631E-5,-3.2476212E-3,-7.955341E-3,-5.44112E-2,-7.4191424E-3,-2.3419643E-2,-7.66727E-2,-5.782504E-3,-2.918293E-2,-4.1498486E-3,-3.494274E-4,7.58996E-3,-3.7308473E-2,-7.7146096E-3,-5.4111432E-2,5.2095467E-4,-4.085223E-2,3.0023122E-2,3.7781356E-3,2.2920173E-2,4.1903337E-3,1.3029128E-3,-4.396736E-3,7.978218E-3,-3.9117597E-3,3.2966908E-2,9.231447E-2,5.1219616E-2,-2.9362127E-2,-1.3554444E-3,1.2498882E-3,-5.1819924E-3,5.734586E-4,-1.5715151E-3,2.5612765E-3,-4.2935116E-3,-9.4207666E-5,-1.8896698E-3,1.7369419E-3,7.4920733E-4,-1.1212428E-3,-4.456104E-3,-9.261613E-4,2.2897439E-4,-3.5130295E-3,-4.555337E-3,-7.9966645E-4,3.103407E-3,-1.6365559E-3,2.1641464E-3,-2.1283876E-4,2.514252E-3,-8.094858E-4,-1.3055975E-3,1.777273E-3,3.8738707E-3,7.031666E-4,2.6839764E-3,5.639245E-3,3.8731173E-3,-2.1650958E-4,-5.8842753E-4,-4.6003815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,33,35,37,39,41,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,-1,79,81,83,85,87,89,-1,-1,-1,91,-1,93,95,-1,-1,-1,97,99,101,103,105,107,-1,109,111,-1,-1,-1,-1,113,115,-1,117,-1,-1,-1,119,121,-1,123,125,-1,127,-1,-1,129,131,-1,133,-1,135,137,-1,139,-1,141,-1,143,-1,145,147,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.639462E-1,2.1930282E-1,2.4593905E-1,2.3166256E-1,0E0,0E0,1.2819946E-1,2.0792654E-1,9.9005535E-2,1.2058103E-1,9.888673E-2,8.749877E-2,3.8528195E-1,8.607142E-2,0E0,8.547725E-2,3.711492E-2,7.2857335E-2,0E0,5.4679185E-2,3.0938441E-2,5.9854865E-2,4.937685E-1,8.257921E-2,7.8926384E-2,0E0,7.3531374E-2,0E0,0E0,0E0,5.916466E-2,6.744385E-2,3.856778E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.455011E-2,0E0,6.92682E-2,4.660246E-2,5.105865E-2,7.802534E-2,3.208524E-2,2.869974E-2,7.9424985E-2,4.541698E-2,1.990654E-2,4.310949E-2,6.889432E-2,5.550488E-2,1.668863E-2,0E0,0E0,5.056537E-2,1.9453004E-2,3.1150386E-2,8.519499E-2,5.7937868E-2,1.8721722E-2,0E0,0E0,0E0,3.5427988E-2,0E0,4.4813946E-2,5.2762672E-2,0E0,0E0,0E0,3.758832E-2,4.6401568E-2,6.9361575E-2,4.6771973E-2,1.9151341E-2,1.869303E-2,0E0,2.9040456E-2,3.7632167E-2,0E0,0E0,0E0,0E0,4.1433286E-2,5.9441313E-2,0E0,5.890653E-2,0E0,0E0,0E0,1.950987E-2,4.384624E-2,0E0,3.447579E-2,4.7145486E-2,0E0,2.5017329E-2,0E0,0E0,5.9212293E-2,3.99247E-2,0E0,1.9806955E-2,0E0,2.276638E-2,3.1557165E-2,0E0,1.7338216E-2,0E0,2.2952452E-2,0E0,2.871148E-2,0E0,4.839037E-2,1.9376278E-2,2.0450532E-2,6.06014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,26,26,30,30,31,31,32,32,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,56,56,57,57,58,58,59,59,60,60,61,61,65,65,67,67,68,68,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,85,85,86,86,88,88,92,92,93,93,95,95,96,96,98,98,101,101,102,102,104,104,106,106,107,107,109,109,111,111,113,113,115,115,116,116,117,117,118,118],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,34,36,38,40,42,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,-1,80,82,84,86,88,90,-1,-1,-1,92,-1,94,96,-1,-1,-1,98,100,102,104,106,108,-1,110,112,-1,-1,-1,-1,114,116,-1,118,-1,-1,-1,120,122,-1,124,126,-1,128,-1,-1,130,132,-1,134,-1,136,138,-1,140,-1,142,-1,144,-1,146,148,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.56E3,2.483E3,2.7920364E2,1E0,-8.929629E-3,-8.798362E-3,2.5345264E7,3.7798166E0,4.4E1,1.7379E4,4.7263342E2,7.6937294E-1,4.327E3,3.6698947E2,-6.0048187E-3,1.0918E4,5.4123135E1,2.94857E5,5.0914856E-3,7.3131656E5,1.5E1,3.3857143E1,2.2596774E2,3.5E1,1E0,5.3411988E-3,3.222E3,6.0303295E-3,2.5415237E-3,3.367115E-3,1.4839433E5,6.0921145E4,5.51E2,2.5520409E-3,-7.0842216E-4,-3.6390233E-3,2.669943E-3,-1.8471073E-2,-1.4443144E-3,1.8E1,4.9408553E-3,1.59E2,1.14E3,6.232639E2,1.8645384E1,1.8987958E0,4.797448E8,2.135961E4,1E0,6E0,1.1015E4,8.234307E7,2.3266666E0,1.7172844E7,-2.119309E-3,-1.5919986E-3,1.688E3,4.1228E2,2.1199985E-7,7.653112E2,1.0092541E8,1.514526E6,9.3525334E-4,-2.2455424E-3,-8.320666E-3,5.62201E5,3.8670804E-3,1.3050649E4,6.76E2,-4.5756376E-5,2.978936E-3,1.4405181E-3,1.6716E5,2.026453E6,1E0,1.8149019E2,5.80563E5,2.2116136E5,5.0315005E-4,2.682E3,2.0766992E6,2.7676343E-3,-1.04027036E-4,5.687251E-3,8.7700016E-4,4.4522205E6,1.5E1,4.4412473E-3,2E0,-3.8619095E-3,-5.050631E-5,-3.2476212E-3,1.1E1,3.964269E3,-7.4191424E-3,1.5141701E1,5.7E1,-5.782504E-3,2.0130434E1,-4.1498486E-3,-3.494274E-4,3.5014236E2,1.2E1,-7.7146096E-3,1E0,5.2095467E-4,1.079E3,1.2E1,3.7781356E-3,1.5747747E1,4.1903337E-3,2.7662E4,-4.396736E-3,2.2E0,-3.9117597E-3,7.5052085E0,1.1772152E0,1.2707424E0,1.0975045E7,-1.3554444E-3,1.2498882E-3,-5.1819924E-3,5.734586E-4,-1.5715151E-3,2.5612765E-3,-4.2935116E-3,-9.4207666E-5,-1.8896698E-3,1.7369419E-3,7.4920733E-4,-1.1212428E-3,-4.456104E-3,-9.261613E-4,2.2897439E-4,-3.5130295E-3,-4.555337E-3,-7.9966645E-4,3.103407E-3,-1.6365559E-3,2.1641464E-3,-2.1283876E-4,2.514252E-3,-8.094858E-4,-1.3055975E-3,1.777273E-3,3.8738707E-3,7.031666E-4,2.6839764E-3,5.639245E-3,3.8731173E-3,-2.1650958E-4,-5.8842753E-4,-4.6003815E-3],"split_indices":[2,2,48,29,0,0,60,68,3,2,71,53,1,67,0,9,73,1,0,43,3,67,67,3,26,0,2,0,0,0,48,60,2,0,0,0,0,0,0,3,0,0,2,67,71,68,7,43,18,8,9,46,68,60,0,0,2,4,52,4,7,10,0,0,0,46,0,43,2,0,0,0,7,9,30,70,12,43,0,10,43,0,0,0,0,62,8,0,8,0,0,0,3,43,0,73,6,0,71,0,0,4,3,0,8,0,10,71,0,73,0,9,0,68,0,71,68,68,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,7.28E2,2.94E2,7.22E2,6E0,4E0,2.9E2,2.48E2,4.74E2,2.54E2,3.6E1,2.22E2,2.6E1,4.69E2,5E0,2.23E2,3.1E1,3.2E1,4E0,1.98E2,2.4E1,1.2E1,1.4E1,3.32E2,1.37E2,1.1E1,2.12E2,1.7E1,1.4E1,4E0,2.8E1,1.38E2,6E1,1.4E1,1E1,4E0,8E0,7E0,7E0,3.27E2,5E0,5.4E1,8.3E1,3.8E1,1.74E2,2E1,8E0,4.4E1,9.4E1,1.8E1,4.2E1,2.8E2,4.7E1,4.8E1,6E0,2.5E1,5.8E1,1.9E1,1.9E1,1.08E2,6.6E1,1.2E1,8E0,4E0,4E0,3.8E1,6E0,1.8E1,7.6E1,1.1E1,7E0,9E0,3.3E1,2.7E1,2.53E2,1.9E1,2.8E1,4.2E1,6E0,3.3E1,2.5E1,9E0,1E1,1.5E1,4E0,2.9E1,7.9E1,4E0,6.2E1,8E0,4E0,1.4E1,2.4E1,1E1,8E0,3.8E1,3.8E1,4E0,2.9E1,1.4E1,1.3E1,2.13E2,4E1,5E0,1.4E1,1.1E1,1.7E1,1.1E1,3.1E1,2.5E1,8E0,2.1E1,4E0,2.4E1,5E0,5.4E1,2.5E1,1E1,5.2E1,1.5E1,9E0,5E0,5E0,3.4E1,4E0,3.1E1,7E0,2.5E1,4E0,1.68E2,4.5E1,8E0,3.2E1,4E0,1E1,4E0,1.3E1,7E0,4E0,1.3E1,1.2E1,5E0,1.6E1,1.1E1,1.3E1,1.3E1,4.1E1,1.3E1,1.2E1,6E0,4E0,4.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-7.3198804E-5,-2.2783266E-2,1.5895527E-2,-1.5977498E-2,-7.454215E-2,-5.250131E-2,2.058859E-2,-1.09443525E-2,-6.320875E-2,-3.5612783E-3,-1.4833216E-1,-6.835955E-2,1.3748431E-3,7.889349E-3,1.8671151E-2,-1.6785571E-2,1.4599447E-2,9.362277E-4,-8.7782174E-2,-3.6571737E-2,3.799857E-2,-1.7136347E-2,-5.3527907E-2,-5.5754366E-3,-4.963083E-2,5.5908125E-2,1.0829558E-2,-1.8994963E-2,2.7791401E-3,2.3128772E-2,-3.3127256E-3,-6.019664E-3,-3.5958882E-2,9.1654784E-4,-3.084298E-3,3.637656E-3,-1.6070552E-3,-3.8415352E-3,2.2650431E-4,3.671403E-4,-2.7932285E-3,3.9474778E-2,1.01657175E-1,8.21374E-3,1.0845626E-1,-2.699716E-2,2.6730103E-3,4.0224258E-2,-5.378261E-3,-3.255703E-3,1.0072666E-3,4.0847105E-3,1.9338377E-2,1.0643075E-3,5.551289E-3,2.5339709E-3,5.7102513E-2,6.8270857E-3,2.0562187E-3,-1.9593183E-2,-7.7448964E-2,1.0233871E-2,-4.7914716E-3,1.7784748E-2,4.5752055E-3,-2.1229796E-2,1.9899188E-3,-3.2505472E-3,9.678808E-2,7.0530996E-3,-5.872616E-2,4.3609206E-2,7.471516E-3,-1.4337468E-2,-6.728788E-2,-6.964446E-3,-1.87423E-3,-4.020971E-2,2.0320127E-2,2.4518825E-3,2.578389E-3,5.384609E-4,-3.600521E-3,-4.117588E-2,4.4826217E-2,7.0764334E-3,6.87196E-4,5.2980117E-2,1.7797659E-3,-2.6662592E-2,-1.2345341E-1,5.576743E-2,-1.8785038E-3,-2.5240244E-2,1.54081015E-2,-1.2521219E-1,-8.718359E-5,-3.304146E-3,8.282728E-4,2.6197698E-2,-2.2359039E-3,1.3245983E-3,-2.0207013E-3,1.2220579E-3,-1.7750418E-3,-2.1131195E-3,-7.300003E-3,-1.9109527E-4,3.406367E-3,2.5541404E-2,1.05892815E-1,-1.014036E-1,6.1785076E-3,1.6401062E-3,-4.9241115E-3,-1.4431817E-3,-8.822992E-3,5.106643E-3,7.6217875E-2,-1.3711936E-3,2.223794E-3,-5.828585E-4,2.2298857E-3,-7.821379E-3,-2.591053E-3,1.6174281E-3,-5.652077E-4,-8.688377E-4,2.2355414E-3,-1.2461173E-3,2.4622704E-3,6.725709E-3,1.6320407E-3,3.4086595E-4,-1.3799031E-2,2.3074378E-3,-3.403599E-6,1.5976295E-3,-1.9899944E-3,4.2402097E-3,1.2835037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,31,33,35,-1,37,-1,39,41,43,45,-1,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,51,53,55,57,59,61,63,65,-1,-1,-1,67,-1,-1,69,71,-1,-1,73,75,77,-1,79,-1,81,-1,83,85,87,89,91,-1,93,95,-1,-1,97,99,101,-1,103,-1,105,107,-1,-1,109,111,113,115,117,-1,119,121,123,-1,-1,-1,125,-1,-1,-1,-1,-1,127,-1,-1,-1,129,131,133,135,-1,-1,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.724419E-1,1.4816678E-1,1.9463833E-1,8.8691615E-2,2.5611803E-1,5.2910015E-2,1.5950966E-1,5.112463E-2,7.562348E-2,3.705932E-2,4.8660952E-1,2.6348755E-2,0E0,0E0,1.6262098E-1,4.8400886E-2,4.8770323E-2,0E0,5.7007954E-2,2.6869562E-2,3.8321793E-2,0E0,3.156471E-2,0E0,1.6693085E-2,6.8912685E-2,1.1734719E-1,4.7061235E-2,0E0,2.8969493E-2,0E0,0E0,2.9445304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.023091E-2,3.3599466E-2,1.2536526E-1,2.3595676E-2,7.2444245E-2,6.058511E-2,4.7064945E-2,1.8578941E-2,0E0,0E0,0E0,9.3038395E-2,0E0,0E0,1.1281223E-1,5.929704E-2,0E0,0E0,4.2651787E-2,6.146738E-2,3.643215E-2,0E0,1.6188353E-2,0E0,2.226067E-2,0E0,7.8425504E-2,5.314546E-2,9.1695145E-2,5.4819234E-2,4.6324387E-2,0E0,5.1629774E-2,6.416471E-2,0E0,0E0,2.3125919E-2,2.509746E-2,2.6687378E-2,0E0,1.5150642E-2,0E0,1.0920067E-1,2.7196083E-2,0E0,0E0,5.3917736E-2,1.5539856E-1,1.0314853E-1,5.0075084E-2,3.898015E-2,0E0,3.9218865E-2,4.020536E-2,1.7932579E-2,0E0,0E0,0E0,1.9055326E-2,0E0,0E0,0E0,0E0,0E0,1.692801E-2,0E0,0E0,0E0,4.079175E-2,3.126581E-2,3.0265477E-1,8.975745E-2,0E0,0E0,0E0,0E0,1.8531075E-2,1.780349E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,19,19,20,20,22,22,24,24,25,25,26,26,27,27,29,29,32,32,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,52,52,55,55,56,56,59,59,60,60,61,61,63,63,65,65,67,67,68,68,69,69,70,70,71,71,73,73,74,74,77,77,78,78,79,79,81,81,83,83,84,84,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,99,99,105,105,109,109,110,110,111,111,112,112,117,117,118,118],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,32,34,36,-1,38,-1,40,42,44,46,-1,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,52,54,56,58,60,62,64,66,-1,-1,-1,68,-1,-1,70,72,-1,-1,74,76,78,-1,80,-1,82,-1,84,86,88,90,92,-1,94,96,-1,-1,98,100,102,-1,104,-1,106,108,-1,-1,110,112,114,116,118,-1,120,122,124,-1,-1,-1,126,-1,-1,-1,-1,-1,128,-1,-1,-1,130,132,134,136,-1,-1,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,3.5531914E0,1E0,2.8E1,8.62E2,1.1805571E-1,1.4E1,2.7224753E2,1.1E1,8.5714287E-1,1E0,3.421907E6,1.3748431E-3,7.889349E-3,1.6954315E0,1.3062992E1,5.0394736E2,9.362277E-4,1.084437E6,8.75477E5,6.763314E7,-1.7136347E-2,9.822E3,-5.5754366E-3,9.82E2,1.6E1,4.435876E2,8.93004E5,2.7791401E-3,1E0,-3.3127256E-3,-6.019664E-3,4.5172415E0,9.1654784E-4,-3.084298E-3,3.637656E-3,-1.6070552E-3,-3.8415352E-3,2.2650431E-4,3.671403E-4,-2.7932285E-3,3.887E3,2.5454E4,2.680152E5,2.4704508E7,4.28649E5,8.416667E0,3.541824E2,7.1E1,-3.255703E-3,1.0072666E-3,4.0847105E-3,1.9001543E1,1.0643075E-3,5.551289E-3,1.4127854E7,1.9E1,6.8270857E-3,2.0562187E-3,1E0,1.2E1,2.35E2,-4.7914716E-3,9.008784E7,4.5752055E-3,4.0119403E2,1.9899188E-3,3.659E3,2E0,1.4814814E0,1.7540792E5,1.4E1,7.471516E-3,5.5E1,2.0217392E0,-6.964446E-3,-1.87423E-3,7.773675E1,2.6043478E2,9.22E2,2.578389E-3,1.7638037E0,-3.600521E-3,2.62E3,1.7790684E5,7.0764334E-3,6.87196E-4,4.6604937E-1,1.027972E0,4.2619047E0,6.9756895E6,9.3411E4,-1.8785038E-3,3.1316226E0,1.6808511E0,1.2269693E4,-8.718359E-5,-3.304146E-3,8.282728E-4,1.0115696E0,-2.2359039E-3,1.3245983E-3,-2.0207013E-3,1.2220579E-3,-1.7750418E-3,4.080551E-1,-7.300003E-3,-1.9109527E-4,3.406367E-3,1.1E1,7E0,4.822581E0,6E0,1.6401062E-3,-4.9241115E-3,-1.4431817E-3,-8.822992E-3,4.8E2,1.4780379E7,-1.3711936E-3,2.223794E-3,-5.828585E-4,2.2298857E-3,-7.821379E-3,-2.591053E-3,1.6174281E-3,-5.652077E-4,-8.688377E-4,2.2355414E-3,-1.2461173E-3,2.4622704E-3,6.725709E-3,1.6320407E-3,3.4086595E-4,-1.3799031E-2,2.3074378E-3,-3.403599E-6,1.5976295E-3,-1.9899944E-3,4.2402097E-3,1.2835037E-3],"split_indices":[2,68,17,8,44,57,0,67,3,71,29,12,0,0,69,71,70,0,9,46,7,0,44,0,2,8,71,9,0,26,0,0,69,0,0,0,0,0,0,0,0,44,9,48,1,9,71,67,0,0,0,0,71,0,0,66,8,0,0,30,3,2,0,5,0,4,0,2,8,71,48,8,0,0,71,0,0,67,67,44,0,68,0,2,48,0,0,71,68,69,60,1,0,68,68,62,0,0,0,53,0,0,0,0,0,53,0,0,0,3,3,71,3,0,0,0,0,10,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,4.23E2,6.02E2,3.75E2,4.8E1,3.8E1,5.64E2,3.4E2,3.5E1,2.5E1,2.3E1,3.2E1,6E0,6E0,5.58E2,2.77E2,6.3E1,8E0,2.7E1,1.4E1,1.1E1,6E0,1.7E1,7E0,2.5E1,9.6E1,4.62E2,2.7E2,7E0,5.8E1,5E0,1.4E1,1.3E1,5E0,9E0,7E0,4E0,1.1E1,6E0,4E0,2.1E1,7.2E1,2.4E1,4.51E2,1.1E1,1.97E2,7.3E1,3.6E1,2.2E1,8E0,5E0,2E1,5.2E1,5E0,1.9E1,4.05E2,4.6E1,6E0,5E0,1.73E2,2.4E1,6.9E1,4E0,2.7E1,9E0,1.7E1,5E0,4.1E1,1.1E1,3.78E2,2.7E1,4.2E1,4E0,1.57E2,1.6E1,7E0,1.7E1,1.1E1,5.8E1,2E1,7E0,1.3E1,4E0,2.3E1,1.8E1,6E0,5E0,3.8E1,3.4E2,1.9E1,8E0,3.7E1,5E0,1.15E2,4.2E1,8E0,8E0,7E0,4E0,5.4E1,4E0,1.3E1,7E0,8E0,5E0,1.8E1,5E0,7E0,1.1E1,2.6E1,1.2E1,1.3E1,3.27E2,1.1E1,8E0,4E0,4E0,1.1E1,2.6E1,1.09E2,6E0,2.3E1,1.9E1,4E0,4E0,4.4E1,1E1,1.4E1,4E0,9E0,1.7E1,7E0,5E0,9E0,4E0,4E1,2.87E2,7E0,4E0,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[-5.0379704E-3,-2.6458643E-2,1.7478172E-2,-2.170328E-2,-1.7119959E-1,-6.794838E-2,2.3113383E-2,-2.4321122E-2,3.901759E-2,-9.5182745E-4,-2.5147036E-1,-4.120725E-2,-9.429642E-3,1.1287728E-2,5.8693375E-2,3.0864634E-2,-2.6794646E-2,3.650673E-3,-7.802085E-4,-1.5879592E-2,-5.4195095E-3,-2.7823346E-3,1.7815157E-3,1.353236E-2,-4.8417076E-3,1.2800871E-1,4.09381E-2,-9.640937E-4,2.3824265E-3,-1.214309E-1,-2.4114335E-2,2.5119218E-3,3.229817E-2,8.202278E-3,7.9946116E-2,1.2456085E-2,6.5346465E-2,-1.1591618E-3,-1.2499488E-2,-3.165019E-2,-5.995764E-3,9.0057375E-3,-1.3227522E-2,3.7086245E-2,-2.1506944E-3,5.17834E-3,8.505047E-4,-4.042029E-3,2.6778769E-2,8.8830836E-2,2.928456E-2,-2.4758635E-2,-6.505052E-2,1.4850303E-2,-2.3982411E-2,5.9431978E-2,2.3047836E-3,6.3128434E-2,1.946525E-2,6.9702476E-2,-9.84424E-3,1.0425684E-1,3.316943E-4,-2.2041834E-4,4.133811E-3,-2.7720671E-2,5.227569E-3,-8.053179E-2,3.5749776E-3,2.2346413E-2,-2.3308652E-3,5.5097076E-3,-3.8838666E-2,9.825463E-2,5.950998E-3,-3.4965403E-2,9.2094E-3,3.2930467E-2,8.763298E-2,4.5370262E-2,-1.9602356E-2,8.9224346E-2,-1.295253E-4,-3.2192608E-3,1.5374078E-2,6.2608216E-3,7.2297126E-2,-1.6361516E-2,-5.2813537E-2,-1.7768513E-2,-1.0871591E-1,-5.1212655E-3,1.9724157E-3,-8.45056E-4,2.4254594E-3,-3.6917245E-3,-1.1644814E-3,3.0869665E-4,6.0495976E-3,-2.4311212E-3,2.5551277E-3,-6.585039E-2,1.2863841E-2,6.7187324E-2,5.4816236E-3,7.04264E-2,-1.0434325E-2,6.5210555E-3,6.5620534E-2,2.6777036E-2,9.0508685E-2,-4.5372076E-2,1.4495606E-3,5.55659E-3,1.6888417E-3,-4.9947044E-3,3.0148623E-3,4.5743897E-3,7.261632E-4,-1.3532762E-3,1.0650757E-3,-1.0867582E-3,-3.4820798E-3,7.823806E-4,-3.8906117E-3,-1.1792188E-2,-3.2153344E-3,2.120461E-3,-1.4019826E-3,-4.938263E-3,1.8210933E-4,1.8892603E-3,-1.8554955E-3,5.717072E-3,-6.5250543E-4,-2.7998232E-3,5.6951516E-4,1.3544664E-3,5.243582E-3,1.0600457E-3,-2.2008317E-3,3.6425916E-3,3.4793493E-4,3.9226608E-4,3.5401762E-3,9.350862E-4,5.5203293E-3,-4.794298E-3,-6.972665E-4,2.1345895E-3,-1.5191894E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,-1,-1,37,39,41,43,-1,45,47,49,-1,-1,51,53,55,-1,57,-1,-1,-1,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,87,-1,89,-1,91,-1,93,95,97,99,101,103,105,107,109,111,113,-1,-1,115,-1,117,119,121,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,129,131,133,135,137,139,-1,141,143,145,147,-1,-1,-1,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.073713E-1,3.6727583E-1,2.4852696E-1,8.408375E-2,2.0032391E-1,1.08104065E-1,2.0214224E-1,6.93517E-2,5.053501E-2,0E0,9.582144E-2,4.4798974E-2,0E0,9.600747E-2,1.4244509E-1,2.4639979E-2,1.1907962E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.374912E-2,0E0,4.294312E-2,6.6620395E-2,0E0,0E0,1.7998101E-1,6.3846946E-2,4.2646283E-1,5.1562324E-2,0E0,2.5626764E-2,6.8192795E-2,4.143694E-2,0E0,0E0,7.457909E-2,5.2474316E-2,7.4877694E-2,0E0,5.613233E-2,0E0,0E0,0E0,0E0,6.5137975E-2,3.8297564E-2,4.552545E-2,1.1588268E-1,1.2971382E-1,3.294612E-2,3.3074666E-2,5.339838E-2,5.1017273E-2,3.411357E-2,7.780492E-2,2.7143799E-2,3.5778712E-2,1.6179293E-2,0E0,0E0,0E0,7.638003E-2,0E0,8.82836E-2,0E0,3.319358E-2,0E0,3.0039666E-2,2.5981337E-2,4.4030115E-2,3.7601948E-2,4.664046E-2,3.5725202E-2,3.9758928E-2,2.2545427E-2,3.6182806E-2,4.2148583E-2,2.0134136E-2,0E0,0E0,1.759925E-2,0E0,2.1118551E-2,9.6494034E-2,5.3727373E-2,4.062638E-2,1.8285868E-1,3.485449E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.484336E-2,2.0964954E-2,4.7738485E-2,7.222712E-2,1.9691184E-2,1.5995158E-2,0E0,1.5539132E-2,3.0343806E-2,2.4055652E-2,3.5570186E-2,0E0,0E0,0E0,2.008178E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,23,23,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,43,43,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,65,65,67,67,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,84,84,86,86,87,87,88,88,89,89,90,90,91,91,101,101,102,102,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,115,115],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,-1,-1,38,40,42,44,-1,46,48,50,-1,-1,52,54,56,-1,58,-1,-1,-1,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,88,-1,90,-1,92,-1,94,96,98,100,102,104,106,108,110,112,114,-1,-1,116,-1,118,120,122,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,130,132,134,136,138,140,-1,142,144,146,148,-1,-1,-1,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,6.0692043E0,3.964269E3,1.6521566E5,3.89E2,3.5E0,1.3141646E3,3.5714287E-1,4E0,-9.5182745E-4,6.6E1,1.24141544E-1,-9.429642E-3,1.853776E7,1.3587301E1,9.147E3,1.342376E0,3.650673E-3,-7.802085E-4,-1.5879592E-2,-5.4195095E-3,-2.7823346E-3,1.7815157E-3,1.8062708E6,-4.8417076E-3,2.5400869E5,1.6876252E0,-9.640937E-4,2.3824265E-3,1.0280637E0,1.559733E6,1.7587205E6,3.9935064E0,8.202278E-3,3.9813398E6,5.97E3,1.034073E6,-1.1591618E-3,-1.2499488E-2,4.064E3,1E0,2.3514E4,-1.3227522E-2,2.8045622E6,-2.1506944E-3,5.17834E-3,8.505047E-4,-4.042029E-3,6.6858566E-1,2.8827406E5,1.1863237E2,3.653E3,2.0130434E1,1E0,5E1,1.6869704E6,3.77E2,2.152E3,7.4010696E0,1.5859042E6,5.369854E5,5.932344E1,3.316943E-4,-2.2041834E-4,4.133811E-3,1.82407E5,5.227569E-3,1.480315E0,3.5749776E-3,6.83E2,-2.3308652E-3,5.195011E7,5.75E2,9.7573663E2,1.365243E7,1.4787234E1,1.0918E4,1.0092541E8,7.6825094E0,5.235714E2,1E0,5.801167E-1,-1.295253E-4,-3.2192608E-3,5.48352E5,6.2608216E-3,3.6666667E0,5.75E2,5.680271E-2,2.732439E2,1.8149019E2,1.2E1,1.9724157E-3,-8.45056E-4,2.4254594E-3,-3.6917245E-3,-1.1644814E-3,3.0869665E-4,6.0495976E-3,-2.4311212E-3,2.5551277E-3,1.7443357E8,2.3666666E0,1.6673345E7,7.9559E5,6.347708E7,4.3024124E2,6.5210555E-3,5.4567E5,2.01E0,1.0641107E8,3.921E3,1.4495606E-3,5.55659E-3,1.6888417E-3,9.408E3,3.0148623E-3,4.5743897E-3,7.261632E-4,-1.3532762E-3,1.0650757E-3,-1.0867582E-3,-3.4820798E-3,7.823806E-4,-3.8906117E-3,-1.1792188E-2,-3.2153344E-3,2.120461E-3,-1.4019826E-3,-4.938263E-3,1.8210933E-4,1.8892603E-3,-1.8554955E-3,5.717072E-3,-6.5250543E-4,-2.7998232E-3,5.6951516E-4,1.3544664E-3,5.243582E-3,1.0600457E-3,-2.2008317E-3,3.6425916E-3,3.4793493E-4,3.9226608E-4,3.5401762E-3,9.350862E-4,5.5203293E-3,-4.794298E-3,-6.972665E-4,2.1345895E-3,-1.5191894E-3],"split_indices":[2,68,43,48,2,68,67,68,8,0,10,53,0,62,71,44,69,0,0,0,0,0,0,43,0,48,53,0,0,69,9,43,68,0,43,2,9,0,0,44,26,1,0,43,0,0,0,0,57,48,73,44,71,19,0,60,10,2,71,43,43,73,0,0,0,9,0,68,0,2,0,7,2,48,5,71,9,7,71,4,28,57,0,0,44,0,69,2,53,67,70,3,0,0,0,0,0,0,0,0,0,7,68,47,12,7,4,0,1,68,5,2,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E3,5.38E2,5.12E2,5.22E2,1.6E1,3.1E1,4.81E2,5.01E2,2.1E1,6E0,1E1,2.7E1,4E0,3.62E2,1.19E2,2.1E1,4.8E2,1.2E1,9E0,5E0,5E0,2.2E1,5E0,3.56E2,6E0,2.3E1,9.6E1,6E0,1.5E1,1.2E1,4.68E2,2.25E2,1.31E2,1E1,1.3E1,4.5E1,5.1E1,8E0,4E0,3.3E2,1.38E2,2.21E2,4E0,1.24E2,7E0,8E0,5E0,5E0,4E1,3E1,2.1E1,2.75E2,5.5E1,6.4E1,7.4E1,2.5E1,1.96E2,4.9E1,7.5E1,1.8E1,2.2E1,2.5E1,5E0,1.4E1,7E0,2.7E2,5E0,5E1,5E0,5.8E1,6E0,2.5E1,4.9E1,1.4E1,1.1E1,3E1,1.66E2,2.3E1,2.6E1,4.5E1,3E1,1.4E1,4E0,6E0,1.6E1,1.1E1,1.4E1,1.87E2,8.3E1,1.6E1,3.4E1,2.5E1,3.3E1,1.7E1,8E0,1.1E1,3.8E1,4E0,1E1,5E0,6E0,1.8E1,1.2E1,9E0,1.57E2,1.2E1,1.1E1,6E0,2E1,3.3E1,1.2E1,2E1,1E1,8E0,6E0,1.2E1,4E0,9E0,5E0,1.41E2,4.6E1,3.7E1,4.6E1,1.1E1,5E0,6E0,2.8E1,8E0,1.7E1,1.1E1,7E0,8E0,4E0,5E0,4E0,1.4E1,1.43E2,7E0,5E0,6E0,5E0,1.6E1,4E0,2.5E1,8E0,4E0,8E0,6E0,1.4E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-6.0611346E-4,-2.3694906E-2,1.5831003E-2,-1.8948995E-2,-6.957293E-2,-5.7419207E-2,2.126269E-2,-1.0498687E-2,-4.1141726E-2,-2.0615581E-2,-2.0948768E-1,-3.4817286E-2,-9.849156E-3,1.8930182E-2,8.173714E-3,-4.462574E-2,-2.4489863E-3,-5.6143887E-2,1.47852795E-2,1.8199385E-3,-6.1517353E-3,-1.5598287E-2,-6.3831935E-4,-5.3048957E-2,1.7511126E-3,5.199987E-3,4.3018747E-2,-1.6522886E-2,-9.01693E-2,4.6123814E-2,-6.120625E-3,-5.1253475E-3,-8.202377E-2,-7.4939905E-3,3.0851678E-3,-1.9282785E-3,3.165261E-2,-2.6445054E-2,-3.4498938E-3,6.368992E-2,-2.6283106E-3,3.3134618E-3,5.5787474E-2,-3.9011112E-3,1.046837E-3,-5.0963473E-4,-5.047909E-3,2.9819328E-3,-3.3783424E-4,-4.590922E-3,-2.9065527E-3,-1.271066E-3,1.8554521E-3,-5.5347722E-2,-2.1221828E-1,3.139617E-2,-4.5551486E-2,-1.1024267E-3,5.2146208E-2,-2.4895126E-3,5.827955E-4,8.370852E-2,2.341045E-2,2.4086763E-3,-8.8695064E-2,-1.5617399E-2,3.2399737E-3,6.247584E-2,-4.3428443E-2,2.6933176E-3,-7.9541194E-4,-3.6056782E-3,-2.4883013E-4,3.695911E-2,-8.129441E-2,-2.8430296E-3,-1.480468E-2,3.1989787E-3,-5.8812666E-4,-3.7955975E-3,7.6137817E-6,3.7556288E-3,-2.2715343E-4,5.608295E-2,6.1319177E-3,-1.1956391E-3,2.3396944E-3,3.8379703E-2,-1.0034807E-2,-2.112811E-1,4.374598E-2,2.8258592E-3,-3.1431753E-2,3.160368E-2,7.7247284E-2,7.787996E-4,-3.992921E-3,-5.1907557E-3,4.4240944E-2,-6.8492896E-4,3.8153015E-3,-6.227962E-4,-9.9536784E-2,5.9769296E-5,3.754632E-3,2.757065E-2,5.2936114E-3,-7.1155108E-3,-4.5019267E-3,-1.4979242E-2,-2.590533E-3,3.8339172E-3,-1.960762E-4,-1.5056144E-2,-3.7293495E-3,7.981172E-2,8.733244E-3,-1.1164962E-3,8.3216265E-2,1.2045125E-3,-3.596816E-2,-8.078828E-4,2.8073252E-3,-8.479045E-3,-7.439039E-2,3.626753E-2,-2.4095168E-3,-8.627255E-3,-5.679496E-5,-3.1907673E-3,8.477124E-3,4.756977E-3,1.2897695E-3,-1.1739854E-3,2.9774604E-3,8.793451E-2,-6.30025E-4,-1.1709518E-3,3.593029E-4,8.3589846E-5,-3.1039051E-3,-2.1760524E-3,-5.29617E-3,5.036642E-4,3.1325482E-3,2.3443638E-3,-2.8171093E-4,1.148037E-3,-1.9558978E-3,1.1586294E-3,-9.785327E-4,4.343763E-3,3.077762E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,-1,-1,-1,37,-1,39,41,43,45,47,49,51,53,55,-1,-1,57,59,-1,61,63,65,67,-1,69,-1,-1,-1,-1,-1,71,-1,-1,73,75,77,79,-1,81,-1,-1,83,85,87,89,91,-1,93,95,-1,-1,-1,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,103,-1,-1,-1,105,107,109,111,-1,113,115,117,-1,-1,119,121,-1,-1,-1,123,-1,-1,125,-1,-1,127,-1,-1,-1,-1,129,-1,131,133,-1,135,137,139,-1,-1,-1,141,143,-1,-1,145,-1,147,-1,-1,149,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.833039E-1,9.0129495E-2,2.3596549E-1,7.12066E-2,2.6282564E-1,1.3692643E-1,1.9634533E-1,7.62608E-2,8.892782E-2,7.5990215E-2,2.4607664E-1,5.1066257E-2,0E0,1.7936848E-1,0E0,6.634506E-2,4.058755E-2,1.0916898E-1,2.7143696E-2,3.5107207E-2,0E0,0E0,0E0,1.5284985E-2,0E0,1.5905058E-1,9.969139E-2,4.0660903E-2,2.9017553E-2,1.6222056E-2,6.2951826E-2,3.0507263E-2,1.7927337E-1,2.65912E-2,0E0,0E0,1.9447623E-2,1.6757783E-2,0E0,3.1100214E-2,1.3330339E-1,6.307578E-2,1.0158855E-1,0E0,3.0281406E-2,0E0,0E0,0E0,0E0,0E0,4.0919363E-2,0E0,0E0,1.15572244E-1,1.2706017E-1,1.5912052E-2,1.507495E-2,0E0,2.0357706E-2,0E0,0E0,2.9119536E-2,2.1445882E-2,1.3063815E-1,2.8524488E-1,4.9280144E-2,0E0,6.140238E-2,2.7380034E-2,0E0,0E0,0E0,4.398281E-2,2.7213737E-2,4.443024E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6115246E-2,0E0,0E0,0E0,5.8617055E-2,1.7217854E-1,1.395979E-1,1.730082E-2,0E0,2.5421642E-2,5.1122315E-2,5.8913052E-2,0E0,0E0,3.547268E-2,2.1463603E-2,0E0,0E0,0E0,4.7600687E-2,0E0,0E0,4.8347026E-2,0E0,0E0,1.7010549E-1,0E0,0E0,0E0,0E0,3.3632E-2,0E0,1.514972E-2,1.8291134E-2,0E0,4.1870236E-2,2.6373472E-2,3.6680236E-2,0E0,0E0,0E0,2.095975E-2,4.8059255E-2,0E0,0E0,6.352902E-2,0E0,1.6859137E-2,0E0,0E0,1.5763158E-2,0E0,4.143244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,36,36,37,37,39,39,40,40,41,41,42,42,44,44,50,50,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,64,64,65,65,67,67,68,68,72,72,73,73,74,74,83,83,87,87,88,88,89,89,90,90,92,92,93,93,94,94,97,97,98,98,102,102,105,105,108,108,113,113,115,115,116,116,118,118,119,119,120,120,124,124,125,125,128,128,130,130,133,133,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,-1,-1,-1,38,-1,40,42,44,46,48,50,52,54,56,-1,-1,58,60,-1,62,64,66,68,-1,70,-1,-1,-1,-1,-1,72,-1,-1,74,76,78,80,-1,82,-1,-1,84,86,88,90,92,-1,94,96,-1,-1,-1,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,104,-1,-1,-1,106,108,110,112,-1,114,116,118,-1,-1,120,122,-1,-1,-1,124,-1,-1,126,-1,-1,128,-1,-1,-1,-1,130,-1,132,134,-1,136,138,140,-1,-1,-1,142,144,-1,-1,146,-1,148,-1,-1,150,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,3.5964913E0,1E0,2.56E2,2.0303884E6,3.5E0,1.0534078E4,4.9E2,2.7224753E2,6.5199727E-1,3.0536E4,1.24141544E-1,-9.849156E-3,1.7068776E10,8.173714E-3,5.853E3,1.2E1,1.1244019E2,1.9495455E6,4.586207E0,-6.1517353E-3,-1.5598287E-2,-6.3831935E-4,1.618692E6,1.7511126E-3,4.2E1,1.1806228E6,9E0,2E0,5.912179E3,1.43E2,1.4702564E11,2.8302418E10,1.946094E6,3.0851678E-3,-1.9282785E-3,7.8625E1,7.980211E5,-3.4498938E-3,8.295515E7,3.2462872E5,8.93E2,3.1E1,-3.9011112E-3,1.9363636E1,-5.0963473E-4,-5.047909E-3,2.9819328E-3,-3.3783424E-4,-4.590922E-3,5.0363636E1,-1.271066E-3,1.8554521E-3,1.7934177E2,1.4386049E6,7.547528E6,2E0,-1.1024267E-3,5.788E3,-2.4895126E-3,5.827955E-4,6.737099E7,2.8470706E2,2.379747E0,3.524172E5,1.9E1,3.2399737E-3,6.255748E6,2.67637E5,2.6933176E-3,-7.9541194E-4,-3.6056782E-3,4.3297894E9,3.6507E4,2.4203823E0,-2.8430296E-3,-1.480468E-2,3.1989787E-3,-5.8812666E-4,-3.7955975E-3,7.6137817E-6,3.7556288E-3,-2.2715343E-4,2.2116136E5,6.1319177E-3,-1.1956391E-3,2.3396944E-3,4.9833015E1,1.027972E0,6.27907E0,8.7692904E7,2.8258592E-3,3.7481603E8,1.5994397E0,1E0,7.787996E-4,-3.992921E-3,1E0,5.08E2,-6.8492896E-4,3.8153015E-3,-6.227962E-4,3.1706784E0,5.9769296E-5,3.754632E-3,2.19724E5,5.2936114E-3,-7.1155108E-3,1E0,-1.4979242E-2,-2.590533E-3,3.8339172E-3,-1.960762E-4,3.4779343E1,-3.7293495E-3,4.911E3,1.3388E4,-1.1164962E-3,1E0,2.590909E0,1.6808511E0,-8.078828E-4,2.8073252E-3,-8.479045E-3,1.6E1,1.915148E7,-2.4095168E-3,-8.627255E-3,1.7142857E0,-3.1907673E-3,2.2092846E7,4.756977E-3,1.2897695E-3,5.929839E8,2.9774604E-3,1.7655972E0,-6.30025E-4,-1.1709518E-3,3.593029E-4,8.3589846E-5,-3.1039051E-3,-2.1760524E-3,-5.29617E-3,5.036642E-4,3.1325482E-3,2.3443638E-3,-2.8171093E-4,1.148037E-3,-1.9558978E-3,1.1586294E-3,-9.785327E-4,4.343763E-3,3.077762E-4],"split_indices":[2,68,17,10,60,68,4,1,67,57,1,53,0,46,0,9,33,67,43,68,0,0,0,62,0,0,43,70,32,48,2,46,46,60,0,0,4,66,0,7,48,8,3,0,4,0,0,0,0,0,62,0,0,70,43,5,8,0,12,0,0,7,67,69,48,8,0,60,9,0,0,0,46,1,69,0,0,0,0,0,0,0,0,43,0,0,0,73,68,71,5,0,47,69,14,0,0,30,2,0,0,0,69,0,0,1,0,0,15,0,0,0,0,71,0,44,2,0,106,69,68,0,0,0,3,12,0,0,71,0,60,0,0,5,0,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,4.19E2,5.89E2,3.81E2,3.8E1,4E1,5.49E2,2.77E2,1.04E2,2.9E1,9E0,3.6E1,4E0,5.42E2,7E0,5.2E1,2.25E2,8.2E1,2.2E1,2.5E1,4E0,5E0,4E0,2.9E1,7E0,3.46E2,1.96E2,3.3E1,1.9E1,1.5E1,2.1E2,2.8E1,5.4E1,1.6E1,6E0,1E1,1.5E1,1.4E1,1.5E1,4E1,3.06E2,4.8E1,1.48E2,6E0,2.7E1,4E0,1.5E1,1.1E1,4E0,6E0,2.04E2,1.9E1,9E0,4.6E1,8E0,8E0,8E0,4E0,1.1E1,8E0,6E0,2.6E1,1.4E1,2.9E2,1.6E1,3.8E1,1E1,1.39E2,9E0,6E0,2.1E1,6E0,1.98E2,1E1,3.6E1,4E0,4E0,4E0,4E0,4E0,4E0,7E0,4E0,1.8E1,8E0,5E0,9E0,7.4E1,2.16E2,8E0,8E0,6E0,3.2E1,4.6E1,9.3E1,4E0,5E0,1.79E2,1.9E1,5E0,5E0,8E0,2.8E1,6E0,1.2E1,6.6E1,8E0,7E0,2.09E2,4E0,4E0,4E0,4E0,2.5E1,7E0,1.4E1,3.2E1,5E0,8.8E1,1.49E2,3E1,4E0,1.5E1,5E0,2.3E1,6E1,6E0,4E0,2.05E2,7E0,1.8E1,9E0,5E0,2.8E1,4E0,8.4E1,4E0,2.9E1,1.2E2,1.4E1,1.6E1,1.5E1,8E0,3.4E1,2.6E1,2.1E1,1.84E2,1.4E1,4E0,1.2E1,1.6E1,7.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[5.9554825E-4,-2.1876628E-2,1.7934639E-2,-1.5649695E-2,-8.1401184E-2,-6.374116E-3,3.300077E-2,-4.8421673E-2,-6.5315063E-3,-9.275872E-3,-4.74066E-2,-4.124915E-3,-4.7660996E-3,3.0714719E-2,7.466498E-3,1.7306872E-2,-5.5210866E-2,7.44051E-2,-1.2798433E-2,-2.7949918E-2,-7.1453936E-3,-8.725113E-3,5.8712777E-2,6.656207E-2,2.2821276E-2,-1.1501417E-3,2.5893515E-3,-8.200504E-2,-3.4653146E-2,1.3743595E-3,9.208301E-2,-1.5352372E-2,4.38133E-3,-4.439506E-2,2.9176925E-3,-5.7045175E-3,-3.083982E-3,-1.1341189E-4,3.7503797E-3,5.5552263E-2,7.4794665E-3,4.1974522E-2,1.0176706E-2,-4.349663E-3,-4.4510656E-4,2.5233554E-5,-5.1169906E-2,7.236533E-3,5.751145E-2,-7.110355E-3,-4.8197847E-2,-4.808201E-3,-1.6929764E-2,-1.8330231E-2,8.926301E-3,7.0742674E-2,1.1256013E-2,1.344279E-2,6.1497293E-2,1.8943131E-2,-6.7825414E-2,-6.1883503E-3,-3.0879207E-2,4.7989172E-4,4.1434774E-3,-1.0886417E-2,7.818445E-2,-9.160255E-2,-1.4865039E-2,2.2064399E-3,-1.7261893E-3,-2.8859276E-2,1.7495886E-3,1.5174884E-2,-2.2956925E-3,5.894271E-2,5.6828447E-3,-2.3596713E-3,1.6765526E-3,5.973904E-2,-9.411233E-3,1.3350567E-1,4.8250157E-2,-5.6302366E-3,2.3363248E-2,-1.2650138E-2,4.891592E-4,4.9221294E-4,-2.413337E-3,1.4825744E-3,-2.4918726E-2,5.2794244E-4,5.5801272E-3,-1.0495287E-1,-1.4707184E-3,2.0835115E-2,-1.8893981E-3,-1.6419897E-2,-1.0731031E-1,2.0827616E-2,-1.8675007E-3,6.684442E-2,-2.308612E-4,3.9988137E-3,2.8540035E-5,-2.1667664E-3,8.927682E-3,7.96291E-3,2.7801772E-3,5.6005865E-2,-1.3671643E-3,5.9227683E-2,9.508964E-3,-1.1720821E-2,3.2616075E-2,-4.370065E-3,-1.9740622E-2,-2.96228E-3,-6.4389375E-3,-3.0324302E-4,3.5548224E-3,-3.1264443E-2,9.441818E-4,-8.904431E-3,4.7323437E-4,2.8400443E-2,-2.19275E-2,3.49129E-3,1.322988E-4,4.7796655E-2,-1.1387231E-3,4.0509272E-2,4.937208E-3,-2.9512115E-2,8.0354266E-2,2.7870711E-2,-1.593654E-2,2.0722274E-4,-1.6170022E-3,2.2537686E-4,2.6415347E-3,-1.3797373E-3,1.1044816E-4,-2.77567E-4,-2.601295E-3,9.765567E-4,3.2912276E-3,-2.532683E-3,9.287375E-4,3.7208193E-3,-3.5119473E-4,7.6728576E-4,2.7136812E-3,-4.1455464E-3,1.6913696E-3,2.2544197E-3,5.7380674E-3,6.4408325E-4,3.4326196E-3,-1.1569837E-4,-5.0608525E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,45,-1,47,49,-1,51,-1,53,-1,-1,-1,55,-1,57,59,-1,-1,-1,61,-1,63,65,67,-1,69,71,73,75,77,79,81,83,85,-1,87,-1,-1,89,91,93,95,-1,-1,97,-1,99,-1,101,-1,-1,-1,103,105,107,109,-1,111,-1,-1,-1,-1,113,115,-1,-1,117,-1,119,-1,121,123,125,-1,127,-1,-1,-1,-1,129,-1,-1,131,-1,133,135,137,139,-1,141,-1,-1,-1,-1,143,-1,-1,-1,145,147,-1,-1,149,-1,151,-1,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9900133E-1,1.640384E-1,2.1189645E-1,1.2070628E-1,1.6092828E-1,4.7904048E-2,1.00001514E-1,4.022497E-2,1.6224943E-1,0E0,6.663172E-2,6.352863E-2,0E0,9.808254E-2,0E0,1.610988E-2,4.122442E-2,1.5417591E-2,8.282385E-2,4.7750674E-2,0E0,3.5376836E-2,2.0812105E-2,5.7131976E-2,6.980637E-2,0E0,0E0,2.8977945E-2,2.7352761E-2,0E0,2.6699856E-2,7.816404E-2,0E0,4.0855873E-2,0E0,3.6184903E-2,0E0,0E0,0E0,3.8230658E-2,0E0,6.3230455E-2,1.21633574E-1,0E0,0E0,0E0,4.9612872E-2,0E0,1.7263532E-2,7.589542E-2,8.255276E-2,0E0,2.6900316E-2,6.3225806E-2,3.4085285E-2,1.926884E-2,2.6820078E-2,5.134888E-2,5.8423817E-2,1.0180987E-1,2.9364616E-1,0E0,2.4508104E-2,0E0,0E0,3.891562E-2,2.7058162E-2,1.6607001E-2,3.2449704E-2,0E0,0E0,8.5091494E-2,0E0,2.6927112E-2,0E0,1.902768E-2,0E0,0E0,0E0,2.5313951E-2,2.310673E-2,1.8119186E-2,3.7277356E-2,0E0,7.645259E-2,0E0,0E0,0E0,0E0,4.971044E-2,3.601417E-2,0E0,0E0,1.9173503E-2,0E0,2.3565665E-2,0E0,4.331491E-2,1.2248604E-1,2.5229808E-2,0E0,1.8207744E-2,0E0,0E0,0E0,0E0,3.119448E-2,0E0,0E0,3.8233235E-2,0E0,8.298649E-2,5.324046E-2,3.2225855E-2,2.4246797E-2,0E0,2.2254046E-2,0E0,0E0,0E0,0E0,3.507976E-2,0E0,0E0,0E0,1.8716916E-2,1.7692465E-2,0E0,0E0,2.0564776E-2,0E0,1.775983E-2,0E0,3.912237E-2,4.236895E-2,4.164854E-2,6.0078975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,33,33,35,35,39,39,41,41,42,42,46,46,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,65,65,66,66,67,67,68,68,71,71,73,73,75,75,79,79,80,80,81,81,82,82,84,84,89,89,90,90,93,93,95,95,97,97,98,98,99,99,101,101,106,106,109,109,111,111,112,112,113,113,114,114,116,116,121,121,125,125,126,126,129,129,131,131,133,133,134,134,135,135,136,136],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,46,-1,48,50,-1,52,-1,54,-1,-1,-1,56,-1,58,60,-1,-1,-1,62,-1,64,66,68,-1,70,72,74,76,78,80,82,84,86,-1,88,-1,-1,90,92,94,96,-1,-1,98,-1,100,-1,102,-1,-1,-1,104,106,108,110,-1,112,-1,-1,-1,-1,114,116,-1,-1,118,-1,120,-1,122,124,126,-1,128,-1,-1,-1,-1,130,-1,-1,132,-1,134,136,138,140,-1,142,-1,-1,-1,-1,144,-1,-1,-1,146,148,-1,-1,150,-1,152,-1,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9236734E5,5.27439E0,1.194E3,2.08E2,3.964269E3,1.25E2,2.88755E5,1.863E3,1.1E1,-9.275872E-3,3.6440647E5,1.5512196E1,-4.7660996E-3,5E0,7.466498E-3,1.1E1,1.63757E4,3.806E3,4.435876E2,2.5393645E5,-7.1453936E-3,1.9E1,4.745397E6,6.439711E1,9.47E4,-1.1501417E-3,2.5893515E-3,5.619529E3,6.0921145E4,1.3743595E-3,4.1957852E2,3.9E2,4.38133E-3,1.317484E1,2.9176925E-3,2.5367088E2,-3.083982E-3,-1.1341189E-4,3.7503797E-3,4.643602E8,7.4794665E-3,6.5199727E-1,3.3386944E5,-4.349663E-3,-4.4510656E-4,2.5233554E-5,2.512195E0,7.236533E-3,9.7573663E2,5.202E3,4.5454545E0,-4.808201E-3,3.3817584E7,8.8474586E8,2E0,2.4545455E0,2.58466E5,2.16948E3,1.2877E4,1.8434782E2,6.27907E0,-6.1883503E-3,4E0,4.7989172E-4,4.1434774E-3,3.0751158E4,1.1481482E0,1.2152382E5,1.4E1,2.2064399E-3,-1.7261893E-3,2.8302418E10,1.7495886E-3,1.45752E8,-2.2956925E-3,2.0342858E0,5.6828447E-3,-2.3596713E-3,1.6765526E-3,1E0,1.8729467E8,1.8383686E5,3.1790426E8,-5.6302366E-3,1.3208092E0,-1.2650138E-2,4.891592E-4,4.9221294E-4,-2.413337E-3,5.117647E0,2.560356E5,5.2794244E-4,5.5801272E-3,2.1251968E1,-1.4707184E-3,6.364486E0,-1.8893981E-3,1.449E3,1.3254E4,3.9444446E2,-1.8675007E-3,1E0,-2.308612E-4,3.9988137E-3,2.8540035E-5,-2.1667664E-3,1.9495455E6,7.96291E-3,2.7801772E-3,9.356961E6,-1.3671643E-3,5.5933E5,8.998703E-1,4.0695653E0,1.2860047E5,-4.370065E-3,1.3631483E8,-2.96228E-3,-6.4389375E-3,-3.0324302E-4,3.5548224E-3,1.703125E0,9.441818E-4,-8.904431E-3,4.7323437E-4,1.978531E6,4.797342E2,3.49129E-3,1.322988E-4,5.94E2,-1.1387231E-3,2.0094643E0,4.937208E-3,2.2145481E5,2.7748251E1,2.6862775E6,3.7035952E6,2.0722274E-4,-1.6170022E-3,2.2537686E-4,2.6415347E-3,-1.3797373E-3,1.1044816E-4,-2.77567E-4,-2.601295E-3,9.765567E-4,3.2912276E-3,-2.532683E-3,9.287375E-4,3.7208193E-3,-3.5119473E-4,7.6728576E-4,2.7136812E-3,-4.1455464E-3,1.6913696E-3,2.2544197E-3,5.7380674E-3,6.4408325E-4,3.4326196E-3,-1.1569837E-4,-5.0608525E-3],"split_indices":[43,68,2,2,43,8,2,9,10,0,43,71,0,3,0,5,43,1,71,43,0,73,60,73,9,0,0,48,60,0,4,0,0,69,0,67,0,0,0,7,0,57,48,0,0,0,69,0,48,2,69,0,7,5,32,68,1,48,9,67,71,0,10,0,0,48,68,48,3,0,0,46,0,7,0,68,0,0,0,19,7,48,47,0,68,0,0,0,0,69,62,0,0,73,0,69,0,44,44,67,0,19,0,0,0,0,43,0,0,60,0,9,57,71,43,0,7,0,0,0,0,68,0,0,0,62,4,0,0,0,0,57,0,48,73,62,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,4.45E2,5.77E2,4.04E2,4.1E1,2.21E2,3.56E2,8.7E1,3.17E2,8E0,3.3E1,2.17E2,4E0,3.51E2,5E0,8E0,7.9E1,2.2E1,2.95E2,2.9E1,4E0,2.03E2,1.4E1,6.2E1,2.89E2,4E0,4E0,3.3E1,4.6E1,7E0,1.5E1,2.89E2,6E0,2.5E1,4E0,1.94E2,9E0,4E0,1E1,5.7E1,5E0,1.14E2,1.75E2,2.8E1,5E0,1.5E1,3.1E1,4E0,1.1E1,2.32E2,5.7E1,7E0,1.8E1,1.04E2,9E1,4.2E1,1.5E1,4.7E1,6.7E1,1.58E2,1.7E1,5E0,2.6E1,5E0,6E0,2.23E2,9E0,2.4E1,3.3E1,4E0,1.4E1,8.8E1,1.6E1,8.2E1,8E0,3.6E1,6E0,4E0,1.1E1,1.5E1,3.2E1,9E0,5.8E1,4E0,1.54E2,4E0,1.3E1,9E0,1.7E1,1.19E2,1.04E2,4E0,5E0,1.9E1,5E0,1.4E1,1.9E1,7.7E1,1.1E1,7.5E1,7E0,3.2E1,4E0,1E1,5E0,1E1,2.2E1,5E0,4E0,5.3E1,5E0,4.2E1,1.12E2,8.4E1,3.5E1,6E0,9.8E1,1E1,9E0,1E1,4E0,5.5E1,2.2E1,6E0,5E0,6.4E1,1.1E1,2.8E1,4E0,1E1,1.2E1,4.2E1,1.1E1,8E0,3.4E1,6.5E1,4.7E1,5E1,3.4E1,1.7E1,1.8E1,6.7E1,3.1E1,2.8E1,2.7E1,5.6E1,8E0,6E0,5E0,6E0,4E0,1.9E1,2.3E1,4E0,4E0,2.1E1,1.3E1,5.1E1,1.4E1,4.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[5.650882E-4,-6.382714E-3,4.63556E-2,-4.511776E-3,-1.3412294E-1,3.9423298E-2,7.0782066E-3,-1.4508722E-2,1.11286E-2,-5.4246124E-2,-1.1599501E-2,-4.332077E-3,4.4395857E-2,-1.2030394E-2,-1.2337878E-1,1.491511E-2,-8.759417E-2,-6.0621193E-3,1.5511045E-3,5.4352574E-2,-1.1092375E-2,5.0111458E-2,-1.4803842E-2,-1.0342491E-2,-2.085917E-3,-3.6253892E-3,2.6633024E-2,-1.04349945E-2,-1.0413991E-3,4.1141868E-2,1.173865E-1,2.0090997E-2,-2.3763739E-3,-2.0018022E-3,8.5639596E-2,-3.736683E-2,-8.735509E-3,-1.0657948E-2,4.758725E-2,7.7458583E-3,3.7162602E-2,-2.2213517E-3,2.1347508E-3,9.1278896E-2,3.0705065E-2,1.3925987E-3,7.3223677E-3,-4.681101E-4,2.97662E-3,5.3506605E-3,1.7191686E-3,-2.4529317E-2,-1.0019165E-1,-3.1257533E-3,-3.690509E-2,-3.4193262E-3,-4.987276E-3,3.6360817E-3,-1.6929218E-4,1.741027E-2,-6.506097E-2,4.6195902E-2,-4.227491E-2,2.1418915E-3,6.282564E-3,1.4381922E-2,3.4968413E-3,-3.7633654E-2,1.8281268E-2,-8.855379E-3,-5.0259758E-2,-6.129496E-3,7.898671E-2,-8.247789E-2,-1.7147567E-2,-1.028234E-2,3.904352E-3,8.705057E-3,4.8260647E-3,1.933137E-5,-5.4296907E-3,3.6354393E-2,9.309365E-2,-3.188885E-3,2.2618618E-4,-1.3390383E-3,2.2053603E-2,2.0223854E-3,-4.3230806E-2,1.3954345E-4,3.213783E-3,-3.52644E-3,3.105137E-4,-3.7718036E-3,-4.6047056E-3,6.1938735E-3,1.602496E-3,-6.2720566E-3,-2.571711E-3,-3.7010428E-2,9.81731E-4,2.1557739E-2,-2.5822636E-2,1.77644E-2,-3.0035512E-3,5.429795E-3,2.80181E-2,1.8132549E-3,5.5008247E-3,1.5818675E-3,-5.221393E-3,-2.0724467E-4,-2.56587E-3,-3.340695E-4,2.2396864E-3,-2.6626028E-3,-2.946884E-4,1.8733478E-4,2.2877564E-3,-4.240829E-3,-6.0451834E-4,2.439537E-3,1.6893951E-4,5.124633E-3,1.0620741E-3,-2.3530629E-3,1.6627699E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,39,-1,41,43,45,47,-1,-1,49,51,53,55,57,59,61,-1,-1,63,65,-1,-1,-1,-1,-1,-1,67,69,71,73,75,-1,-1,-1,77,79,81,83,-1,-1,85,-1,87,89,-1,91,93,95,97,99,101,-1,103,-1,-1,-1,105,107,-1,-1,-1,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,115,-1,117,119,121,-1,-1,123,-1,-1,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3084688E-1,2.1520646E-1,9.544331E-2,1.3947466E-1,1.07824564E-1,9.026338E-2,0E0,1.4529555E-1,1.3139401E-1,6.5242454E-2,0E0,0E0,7.0215434E-2,9.260291E-2,8.422525E-2,7.309455E-2,1.5535119E-1,0E0,0E0,8.427182E-2,2.6195515E-2,8.0066144E-2,6.963955E-2,0E0,0E0,4.7687605E-2,4.068017E-2,0E0,2.2225477E-2,4.42916E-2,5.9779823E-2,1.6733892E-2,0E0,0E0,2.0534642E-2,8.464593E-2,6.372653E-2,8.13267E-2,2.5805533E-2,5.3905368E-2,9.6673846E-2,0E0,0E0,2.3818687E-2,5.471883E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.173938E-2,7.259257E-2,8.385006E-2,5.891289E-2,6.664728E-2,0E0,0E0,0E0,4.9863756E-2,3.0968875E-2,5.153945E-2,1.8003406E-2,0E0,0E0,1.9602552E-2,0E0,3.378859E-2,1.7007353E-2,0E0,1.9825704E-2,7.1741335E-2,2.4631418E-2,2.1015555E-2,3.7395407E-2,5.102671E-2,0E0,4.256098E-2,0E0,0E0,0E0,6.437777E-2,2.3298353E-2,0E0,0E0,0E0,1.6465941E-2,0E0,3.1226076E-2,0E0,0E0,0E0,0E0,5.819079E-2,0E0,0E0,0E0,0E0,0E0,1.9847237E-2,0E0,1.6439136E-2,5.6781143E-2,2.748986E-2,0E0,0E0,3.580842E-2,0E0,0E0,0E0,3.1975724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,51,51,52,52,53,53,54,54,55,55,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,72,72,73,73,74,74,75,75,77,77,81,81,82,82,86,86,88,88,93,93,99,99,101,101,102,102,103,103,106,106,110,110],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,40,-1,42,44,46,48,-1,-1,50,52,54,56,58,60,62,-1,-1,64,66,-1,-1,-1,-1,-1,-1,68,70,72,74,76,-1,-1,-1,78,80,82,84,-1,-1,86,-1,88,90,-1,92,94,96,98,100,102,-1,104,-1,-1,-1,106,108,-1,-1,-1,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,116,-1,118,120,122,-1,-1,124,-1,-1,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,3.8304348E1,9.62385E3,9.860918E5,6.9756895E6,1.1770261E8,7.0782066E-3,6.425E1,1.9269184E7,2.0619047E1,-1.1599501E-2,-4.332077E-3,7.3274844E9,6E0,1.75E2,3.49E2,2.395631E7,-6.0621193E-3,1.5511045E-3,3.558194E6,2.948485E8,3.3527696E-1,3.3832976E7,-1.0342491E-2,-2.085917E-3,9.750085E6,1E0,-1.04349945E-2,1.9E1,2.01E2,1.0912606E5,4.435876E2,-2.3763739E-3,-2.0018022E-3,2.0649375E6,7E0,3.9E2,1.9E1,1.6108533E8,2.1391304E1,5.0741018E8,-2.2213517E-3,2.1347508E-3,6E0,5.1167645E6,1.3925987E-3,7.3223677E-3,-4.681101E-4,2.97662E-3,5.3506605E-3,1.7191686E-3,4.7659574E0,5.764631E1,1.23E4,8.638015E4,1.3E1,-4.987276E-3,3.6360817E-3,-1.6929218E-4,5.306E3,2.1557376E0,3.6451373E8,2.9251662E6,2.1418915E-3,6.282564E-3,3.532346E-1,3.4968413E-3,4.5054945E-1,5.667774E0,-8.855379E-3,2.7E1,3.53562E5,2.3266666E0,1.775373E1,2.364E3,1.0865825E0,3.904352E-3,1.0046621E7,4.8260647E-3,1.933137E-5,-5.4296907E-3,2.0115254E2,5.4204545E0,-3.188885E-3,2.2618618E-4,-1.3390383E-3,1.116241E2,2.0223854E-3,6.0921145E4,1.3954345E-4,3.213783E-3,-3.52644E-3,3.105137E-4,1.0564016E3,-4.6047056E-3,6.1938735E-3,1.602496E-3,-6.2720566E-3,-2.571711E-3,1.58E2,9.81731E-4,6.23E2,2.0217392E0,1.422162E3,-3.0035512E-3,5.429795E-3,1.1160929E0,1.8132549E-3,5.5008247E-3,1.5818675E-3,1.5477113E2,-2.0724467E-4,-2.56587E-3,-3.340695E-4,2.2396864E-3,-2.6626028E-3,-2.946884E-4,1.8733478E-4,2.2877564E-3,-4.240829E-3,-6.0451834E-4,2.439537E-3,1.6893951E-4,5.124633E-3,1.0620741E-3,-2.3530629E-3,1.6627699E-3],"split_indices":[2,68,67,43,60,7,0,61,62,71,0,0,5,3,6,10,62,0,0,1,47,71,7,0,0,9,29,0,3,0,48,71,0,0,62,8,0,8,7,71,7,0,0,8,43,0,0,0,0,0,0,69,62,10,43,8,0,0,0,2,69,7,43,0,0,53,0,68,68,0,3,1,68,71,2,71,0,62,0,0,0,67,71,0,0,0,71,0,60,0,0,0,0,4,0,0,0,0,0,8,0,2,71,48,0,0,69,0,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.038E3,9.02E2,1.36E2,8.9E2,1.2E1,1.29E2,7E0,5.43E2,3.47E2,8E0,4E0,4E0,1.25E2,5.32E2,1.1E1,3.35E2,1.2E1,4E0,4E0,1.06E2,1.9E1,2.2E1,5.1E2,4E0,7E0,1.3E2,2.05E2,4E0,8E0,8.9E1,1.7E1,1.1E1,8E0,6E0,1.6E1,1.07E2,4.03E2,1.15E2,1.5E1,7.4E1,1.31E2,4E0,4E0,1.4E1,7.5E1,6E0,1.1E1,7E0,4E0,9E0,7E0,9E1,1.7E1,3.37E2,6.6E1,1.08E2,7E0,9E0,6E0,6.6E1,8E0,1.18E2,1.3E1,8E0,6E0,5.6E1,1.9E1,6.9E1,2.1E1,5E0,1.2E1,3.26E2,1.1E1,1.9E1,4.7E1,1.01E2,7E0,6.1E1,5E0,4E0,4E0,9.9E1,1.9E1,8E0,5E0,8E0,4.8E1,4E0,6.5E1,1.7E1,4E0,8E0,4E0,3.19E2,7E0,4E0,7E0,5E0,1.4E1,3.1E1,1.6E1,3.3E1,6.8E1,5.5E1,6E0,8E0,9.1E1,7E0,1.2E1,3.3E1,1.5E1,1.6E1,4.9E1,3E2,1.9E1,1.8E1,1.3E1,2.1E1,1.2E1,1E1,5.8E1,1.5E1,4E1,4E0,8.7E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[1.4329132E-3,-3.3973034E-2,9.006472E-3,-2.6577396E-2,-1.0070072E-1,-5.4913033E-3,2.2044191E-2,-2.1321176E-2,-6.6375877E-3,-7.2451485E-3,-5.072872E-2,-8.932805E-3,5.2504815E-2,1.892582E-2,8.861062E-2,-4.7090568E-2,-9.810872E-4,1.3020491E-5,-3.6921296E-3,-1.193331E-2,5.3073812E-2,6.366194E-3,2.1156404E-2,2.0924678E-2,-5.8740634E-3,8.931953E-3,2.9918449E-2,-3.435887E-2,-1.1341266E-1,8.072683E-2,-2.2234092E-2,-9.192986E-2,-8.753858E-3,7.4712727E-3,-1.47254765E-2,2.4482051E-3,-1.1671192E-3,1.5946405E-2,6.4122155E-2,2.4216874E-3,-9.7320374E-4,-8.411323E-2,-1.7748777E-2,-9.182542E-3,-1.6746321E-3,5.583645E-3,2.1826164E-3,-2.9036723E-2,1.6939746E-3,1.4866677E-5,-8.511134E-3,-6.1108726E-3,-8.401018E-2,-2.6792507E-3,1.1302668E-3,-6.3615977E-3,1.8324176E-2,7.6316044E-2,-1.3391825E-5,-5.040966E-4,-5.0698896E-3,1.5586565E-3,-3.842713E-2,-1.1178046E-2,-5.4418586E-2,-6.673053E-2,-3.536216E-3,-7.7013066E-3,-1.0084688E-3,4.0267454E-3,1.546076E-2,2.6335793E-3,5.395029E-3,-4.196974E-3,-1.2800932E-2,-3.2464698E-2,3.552997E-2,-3.2223521E-3,-4.291627E-4,-4.294266E-3,-7.520036E-4,2.0293554E-2,-1.1433555E-2,3.2213912E-3,1.3270903E-2,1.3108187E-2,-5.2610193E-3,2.492809E-5,-3.026385E-3,-5.1812316E-5,3.100221E-3,3.5256784E-2,-2.2797983E-2,-6.3956957E-3,-7.7431314E-3,3.137669E-2,4.9798666E-3,-1.7102346E-3,1.9841224E-3,-2.089834E-3,1.9213211E-3,-2.2421593E-3,3.395017E-4,-3.8993752E-3,-2.3666656E-4,2.5502827E-6,2.2970787E-3,-2.6004745E-3,4.136652E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,21,23,25,27,29,-1,-1,31,33,-1,35,37,-1,-1,39,41,43,45,47,49,51,-1,53,-1,-1,55,57,-1,-1,59,61,-1,-1,-1,-1,63,-1,-1,-1,65,67,-1,-1,-1,69,71,-1,-1,-1,-1,73,75,77,79,81,-1,-1,-1,83,-1,-1,-1,85,87,89,-1,-1,-1,-1,91,93,-1,95,97,-1,-1,-1,-1,-1,99,101,-1,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7998543E-1,8.812645E-2,1.6267644E-1,9.972252E-2,4.252772E-2,8.190326E-2,9.235619E-2,8.4342234E-2,0E0,0E0,1.6401028E-2,7.2449096E-2,5.8583103E-2,1.293792E-1,1.1714721E-1,5.6055173E-2,1.5981992E-1,0E0,0E0,9.304864E-2,1.3497083E-1,0E0,2.7939316E-2,9.123343E-2,0E0,0E0,1.7164223E-2,4.8918992E-2,6.5239236E-2,1.982233E-2,3.0314434E-2,1.1926381E-1,7.029548E-2,0E0,2.1767298E-2,0E0,0E0,1.4354266E-1,3.45812E-2,0E0,0E0,2.694162E-2,5.13354E-2,0E0,0E0,0E0,0E0,2.9308613E-2,0E0,0E0,0E0,5.3609833E-2,5.806119E-2,0E0,0E0,0E0,7.3750466E-2,2.225335E-2,0E0,0E0,0E0,0E0,4.4508692E-2,4.0923756E-2,1.8032372E-2,1.7007913E-2,6.27117E-2,0E0,0E0,0E0,4.2688422E-2,0E0,0E0,0E0,6.434387E-2,3.0570865E-2,1.5120134E-2,0E0,0E0,0E0,0E0,5.4321237E-2,1.1610294E-1,0E0,5.298472E-2,3.1504724E-2,0E0,0E0,0E0,0E0,0E0,3.3476762E-2,1.7424518E-2,0E0,4.858224E-2,6.377583E-2,5.9914872E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,37,37,38,38,41,41,42,42,47,47,51,51,52,52,56,56,57,57,62,62,63,63,64,64,65,65,66,66,70,70,74,74,75,75,76,76,81,81,82,82,84,84,85,85,91,91,92,92,94,94,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,22,24,26,28,30,-1,-1,32,34,-1,36,38,-1,-1,40,42,44,46,48,50,52,-1,54,-1,-1,56,58,-1,-1,60,62,-1,-1,-1,-1,64,-1,-1,-1,66,68,-1,-1,-1,70,72,-1,-1,-1,-1,74,76,78,80,82,-1,-1,-1,84,-1,-1,-1,86,88,90,-1,-1,-1,-1,92,94,-1,96,98,-1,-1,-1,-1,-1,100,102,-1,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.147158E4,9.51E3,3.4734247E2,3.8304348E1,3.3025316E5,1.559287E5,4.435876E2,2.71E2,-6.6375877E-3,-7.2451485E-3,4.3E1,2.2503355E1,8.691756E-1,7.134432E7,4.8647133E-1,1.3E2,2.04115E5,1.3020491E-5,-3.6921296E-3,1.16408E5,1.68574E5,6.366194E-3,5.323779E6,5.1167645E6,-5.8740634E-3,8.931953E-3,2.7021693E8,7.785714E0,1.1626786E2,3.8275862E0,1.0652307E7,6.23E2,2.2051188E6,7.4712727E-3,4.232328E3,2.4482051E-3,-1.1671192E-3,1.0280637E0,1.213274E9,2.4216874E-3,-9.7320374E-4,1.11E2,1.1E1,-9.182542E-3,-1.6746321E-3,5.583645E-3,2.1826164E-3,4.217427E7,1.6939746E-3,1.4866677E-5,-8.511134E-3,1.88E2,6.763314E7,-2.6792507E-3,1.1302668E-3,-6.3615977E-3,3.63E2,1.400229E6,-1.3391825E-5,-5.040966E-4,-5.0698896E-3,1.5586565E-3,1.8857143E0,1E0,1.3141646E3,4.3827028E5,5.3061223E0,-7.7013066E-3,-1.0084688E-3,4.0267454E-3,8.118812E-1,2.6335793E-3,5.395029E-3,-4.196974E-3,3.5318289E3,1.26375E1,5.831829E6,-3.2223521E-3,-4.291627E-4,-4.294266E-3,-7.520036E-4,2.277451E2,5.475771E0,3.2213912E-3,5.6516E4,3.6E1,-5.2610193E-3,2.492809E-5,-3.026385E-3,-5.1812316E-5,3.100221E-3,8.612209E-6,3.8575E4,-6.3956957E-3,4.9558692E7,1E0,1.4598765E0,-1.7102346E-3,1.9841224E-3,-2.089834E-3,1.9213211E-3,-2.2421593E-3,3.395017E-4,-3.8993752E-3,-2.3666656E-4,2.5502827E-6,2.2970787E-3,-2.6004745E-3,4.136652E-4],"split_indices":[43,44,67,68,62,48,71,2,0,0,8,71,68,59,57,2,5,0,0,7,9,0,12,43,0,0,47,67,4,69,66,2,66,0,48,0,0,69,7,0,0,1,3,0,0,0,0,60,0,0,0,2,7,0,0,0,44,1,0,0,0,0,68,27,67,43,73,0,0,0,68,0,0,0,62,73,60,0,0,0,0,67,73,0,9,2,0,0,0,0,0,53,9,0,46,29,69,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,1.83E2,8.59E2,1.66E2,1.7E1,4.07E2,4.52E2,1.6E2,6E0,7E0,1E1,3.85E2,2.2E1,4.33E2,1.9E1,7E1,9E1,4E0,6E0,3.68E2,1.7E1,5E0,1.7E1,4.28E2,5E0,6E0,1.3E1,6E1,1E1,1.8E1,7.2E1,1.3E1,3.55E2,6E0,1.1E1,1E1,7E0,3.85E2,4.3E1,9E0,4E0,1.4E1,4.6E1,4E0,6E0,7E0,1.1E1,6.5E1,7E0,7E0,6E0,3.44E2,1.1E1,5E0,6E0,5E0,3.8E2,3.6E1,7E0,4E0,1E1,1.3E1,3.3E1,3.9E1,2.6E1,1.3E1,3.31E2,4E0,7E0,1.4E1,3.66E2,2.6E1,1E1,1E1,2.3E1,2.7E1,1.2E1,1.9E1,7E0,8E0,5E0,8.2E1,2.49E2,1.3E1,3.53E2,1.9E1,4E0,1.4E1,1.3E1,6E0,6E0,6.1E1,2.1E1,6E0,2.43E2,1.1E2,2.43E2,7E0,1.2E1,4E0,5.7E1,1.1E1,1E1,7E0,2.36E2,4.1E1,6.9E1,1.4E1,2.29E2],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[1.1671787E-3,-1.6318643E-2,1.8855238E-2,-1.17567815E-2,-8.3146155E-2,-5.6021255E-2,2.3787173E-2,-3.5236605E-2,-5.9766294E-3,-1.2162982E-1,1.7529247E-2,-7.91766E-3,-3.3420462E-2,1.1393533E-1,2.0647649E-2,8.585303E-4,-4.0340494E-2,7.558665E-3,-2.1797104E-2,-1.9651461E-2,-2.0640755E-1,-1.4655216E-3,3.378295E-3,8.368036E-4,-2.2313385E-3,2.0753755E-3,6.153532E-3,-8.023789E-4,3.193607E-2,-5.0421815E-2,-1.2918692E-2,1.0862727E-2,-3.6059632E-3,-2.4322916E-2,3.3252272E-3,-2.1773067E-3,1.3047814E-3,-1.3436436E-2,-2.7457245E-3,-8.843282E-3,6.401614E-2,4.073136E-2,5.2496777E-3,-9.2098214E-2,-4.2823434E-2,-3.801867E-2,1.6598296E-3,1.6413512E-2,-4.13027E-2,1.0997318E-2,-3.1897742E-2,-1.3973577E-3,-5.8147974E-2,9.946798E-2,-1.0840854E-3,2.2004254E-2,6.711193E-2,2.9541608E-2,-8.2224295E-2,-6.1208676E-3,-1.5378622E-3,-2.4805384E-2,-6.9542E-2,2.1254164E-3,-3.1143334E-3,-2.6794795E-2,2.237313E-2,3.6854463E-4,-4.22169E-3,3.7591243E-3,-1.9616357E-3,-5.253208E-2,-5.3759953E-3,-3.91986E-2,6.461366E-3,-5.898438E-3,-2.4197485E-2,6.4986926E-3,2.0354849E-3,2.6862154E-2,-4.052917E-3,2.3195792E-2,7.851363E-2,7.5571276E-2,3.486751E-3,-1.1895893E-2,-8.488627E-3,-3.164914E-3,-5.282204E-4,-6.216524E-3,-2.1999117E-3,-8.763408E-6,-3.5414186E-3,3.0253574E-2,-1.69838E-2,-1.9466795E-2,2.043597E-3,-6.993608E-2,-2.1643689E-2,-1.2486806E-2,3.809162E-3,-6.079649E-2,2.1433367E-3,-6.6817887E-3,3.449045E-2,3.1877044E-4,-3.7676631E-3,-2.6951255E-3,3.0706078E-2,2.968216E-3,-8.8688095E-5,5.6172274E-3,6.477177E-2,9.613334E-2,-6.267547E-4,2.070991E-2,-3.141287E-3,-2.6898896E-3,2.7368255E-3,3.3460625E-3,2.524807E-2,-4.9086116E-2,3.7091244E-2,2.0651363E-3,-3.719452E-2,-3.9838914E-2,-1.09166816E-1,-2.0034993E-3,6.615832E-5,-1.4908453E-3,-4.9295325E-2,-2.780346E-2,-5.6431512E-3,3.7475287E-3,-3.3615E-3,-2.1279707E-4,4.874945E-2,1.934768E-2,6.0871515E-2,7.938859E-2,5.598643E-4,2.6723088E-3,6.3184304E-3,-2.9879278E-3,3.471733E-2,-7.803605E-4,1.5939849E-3,-4.884244E-4,-3.6423355E-3,-5.622856E-4,4.5570764E-3,-4.549525E-4,-3.9587575E-3,5.481957E-4,-2.3209727E-3,-6.2672775E-3,-1.5254904E-3,8.0606254E-4,-1.446732E-3,-9.0561016E-4,-4.565492E-3,-2.7453857E-3,8.774453E-4,9.408216E-4,-1.5628253E-3,3.05456E-3,-2.4745456E-4,2.711664E-3,5.094988E-4,1.1849916E-3,4.369715E-3,4.298175E-3,9.6806703E-4,6.52018E-5,2.6862174E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,49,-1,-1,-1,-1,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,-1,79,81,83,85,-1,-1,87,89,-1,-1,91,93,-1,-1,-1,95,97,99,101,103,-1,105,-1,-1,107,-1,109,111,113,115,-1,117,-1,-1,-1,-1,-1,-1,119,121,123,-1,125,127,129,-1,131,-1,133,135,-1,-1,-1,137,-1,-1,-1,139,141,-1,143,-1,-1,-1,-1,145,147,149,-1,151,153,155,-1,-1,157,159,161,-1,163,-1,-1,165,167,169,171,-1,-1,-1,-1,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1918597E-1,1.5724371E-1,1.907763E-1,6.5883555E-2,1.299859E-1,7.770377E-2,1.3406426E-1,2.7120002E-2,8.410559E-2,2.034356E-1,2.9701091E-2,0E0,2.23427E-2,1.528433E-2,1.131036E-1,0E0,2.3676038E-2,6.0815345E-2,4.4621155E-2,1.7123068E-2,1.4403385E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.500042E-2,7.160881E-2,1.632312E-2,3.2017063E-2,6.0037516E-2,0E0,4.7706194E-2,0E0,0E0,0E0,0E0,0E0,5.3151E-2,5.815901E-2,1.1215052E-1,1.6604725E-1,1.6968362E-2,2.5234304E-2,4.5085967E-2,0E0,4.849542E-2,5.008562E-2,2.9341387E-2,7.961738E-2,3.7981108E-2,4.1759305E-2,2.3478836E-2,0E0,7.485337E-2,4.574713E-2,7.28936E-2,2.1751365E-1,0E0,0E0,1.966922E-2,2.4952158E-2,0E0,0E0,3.0496193E-2,5.1378436E-2,0E0,0E0,0E0,2.3503939E-2,4.2568E-2,4.1623466E-2,4.3489974E-2,3.927986E-2,0E0,2.5533155E-2,0E0,0E0,4.469379E-2,0E0,2.1775529E-2,3.6518157E-2,4.162696E-2,5.0653566E-2,0E0,4.7239628E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7092025E-2,5.0426826E-2,2.55511E-2,0E0,5.7049096E-2,1.5601579E-2,2.4615955E-2,0E0,3.3039324E-2,0E0,5.0581615E-2,1.9178785E-2,0E0,0E0,0E0,4.1993797E-2,0E0,0E0,0E0,4.444571E-2,2.0093426E-2,0E0,4.1714117E-2,0E0,0E0,0E0,0E0,4.894001E-2,1.9793436E-2,3.455285E-2,0E0,2.1701159E-2,1.7265175E-2,3.9899647E-2,0E0,0E0,2.765806E-2,1.8651254E-2,2.0365711E-2,0E0,4.065009E-2,0E0,0E0,2.384675E-2,2.9795833E-2,3.794667E-2,3.4734488E-2,0E0,0E0,0E0,0E0,2.2545572E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,27,27,28,28,29,29,30,30,31,31,33,33,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,65,65,66,66,70,70,71,71,72,72,73,73,74,74,76,76,79,79,81,81,82,82,83,83,84,84,86,86,93,93,94,94,95,95,97,97,98,98,99,99,101,101,103,103,104,104,108,108,112,112,113,113,115,115,120,120,121,121,122,122,124,124,125,125,126,126,129,129,130,130,131,131,133,133,136,136,137,137,138,138,139,139,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,50,-1,-1,-1,-1,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,-1,80,82,84,86,-1,-1,88,90,-1,-1,92,94,-1,-1,-1,96,98,100,102,104,-1,106,-1,-1,108,-1,110,112,114,116,-1,118,-1,-1,-1,-1,-1,-1,120,122,124,-1,126,128,130,-1,132,-1,134,136,-1,-1,-1,138,-1,-1,-1,140,142,-1,144,-1,-1,-1,-1,146,148,150,-1,152,154,156,-1,-1,158,160,162,-1,164,-1,-1,166,168,170,172,-1,-1,-1,-1,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,8.719101E0,5.3E1,2.71E2,3.535931E2,9.7573663E2,2.82E2,1.957E3,1E0,3.89E2,1.2674157E1,-7.91766E-3,1.2E1,4.814389E4,8.484036E5,8.585303E-4,1.6E1,1.6942337E1,6.048589E-4,1.0428572E1,1E0,-1.4655216E-3,3.378295E-3,8.368036E-4,-2.2313385E-3,2.0753755E-3,6.153532E-3,2.4693803E2,2.1356704E0,8E0,1.43E2,1E0,-3.6059632E-3,3.2931E4,3.3252272E-3,-2.1773067E-3,1.3047814E-3,-1.3436436E-2,-2.7457245E-3,4.4744192E8,1.3745962E0,7.956885E-1,3.0561172E5,1.8736842E0,1.62E2,1.7387315E2,1.6598296E-3,1.1379E4,6.763314E7,4.97E2,1.559733E6,1E0,1.48E2,8.036266E-2,-1.0840854E-3,3.49E2,2.297E3,1.684E0,1.2825651E0,-6.1208676E-3,-1.5378622E-3,6.48334E5,1.84E2,2.1254164E-3,-3.1143334E-3,8E0,6.764151E0,3.6854463E-4,-4.22169E-3,3.7591243E-3,1.7697E4,4.4E2,1.6957658E6,1.8383686E5,5.4275E2,-5.898438E-3,1.559733E6,6.4986926E-3,2.0354849E-3,1.1283241E6,-4.052917E-3,1.0586236E5,7.4010696E0,2.596275E3,3.1790426E8,-1.1895893E-2,1.4817301E3,-3.164914E-3,-5.282204E-4,-6.216524E-3,-2.1999117E-3,-8.763408E-6,-3.5414186E-3,1.84E2,1.0722478E5,1.3529412E0,2.043597E-3,1.17E2,2.8266037E2,3.4E1,3.809162E-3,7.601393E2,2.1433367E-3,4.8064E4,5.619529E3,3.1877044E-4,-3.7676631E-3,-2.6951255E-3,2.7777777E0,2.968216E-3,-8.8688095E-5,5.6172274E-3,4.1973075E6,6.46E2,-6.267547E-4,1.7717391E0,-3.141287E-3,-2.6898896E-3,2.7368255E-3,3.3460625E-3,8.754E3,1.2894E4,1.4E1,2.0651363E-3,9.489462E6,1.625E0,7.694314E-7,-2.0034993E-3,6.615832E-5,1.2751362E8,3.9939122E5,2.067E3,-5.6431512E-3,2.1068998E6,-3.3615E-3,-2.1279707E-4,6.0091515E6,1.125584E-3,1E0,9.837502E1,5.598643E-4,2.6723088E-3,6.3184304E-3,-2.9879278E-3,4.3948618E8,-7.803605E-4,1.5939849E-3,-4.884244E-4,-3.6423355E-3,-5.622856E-4,4.5570764E-3,-4.549525E-4,-3.9587575E-3,5.481957E-4,-2.3209727E-3,-6.2672775E-3,-1.5254904E-3,8.0606254E-4,-1.446732E-3,-9.0561016E-4,-4.565492E-3,-2.7453857E-3,8.774453E-4,9.408216E-4,-1.5628253E-3,3.05456E-3,-2.4745456E-4,2.711664E-3,5.094988E-4,1.1849916E-3,4.369715E-3,4.298175E-3,9.6806703E-4,6.52018E-5,2.6862174E-3],"split_indices":[2,69,44,2,70,48,44,9,26,2,69,0,3,60,43,0,3,71,52,68,29,0,0,0,0,0,0,71,53,0,2,19,0,9,0,0,0,0,0,7,53,53,48,68,2,48,0,9,7,2,9,29,0,72,0,8,2,69,69,0,0,46,44,0,0,8,71,0,0,0,9,10,66,48,67,0,9,0,0,60,0,48,71,67,47,0,67,0,0,0,0,0,0,44,66,68,0,10,4,8,0,67,0,44,48,0,0,0,68,0,0,0,43,0,0,69,0,0,0,0,1,12,3,0,12,68,52,0,0,7,60,2,0,66,0,0,62,53,29,73,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,5.18E2,5.12E2,4.86E2,3.2E1,3.1E1,4.81E2,9.5E1,3.91E2,2.3E1,9E0,4E0,2.7E1,1.5E1,4.66E2,8E0,8.7E1,2.11E2,1.8E2,1.1E1,1.2E1,5E0,4E0,6E0,2.1E1,4E0,1.1E1,1.61E2,3.05E2,6.3E1,2.4E1,2.04E2,7E0,1.76E2,4E0,7E0,4E0,7E0,5E0,1.44E2,1.7E1,2.29E2,7.6E1,8E0,5.5E1,1.6E1,8E0,1.85E2,1.9E1,3.1E1,1.45E2,1.26E2,1.8E1,1.2E1,5E0,1.35E2,9.4E1,6E1,1.6E1,4E0,4E0,3.4E1,2.1E1,4E0,1.2E1,2.2E1,1.63E2,1E1,9E0,4E0,2.7E1,8.1E1,6.4E1,2.1E1,1.05E2,5E0,1.3E1,6E0,6E0,1.3E2,5E0,2E1,7.4E1,2.1E1,3.9E1,4E0,1.2E1,7E0,2.7E1,4E0,1.7E1,1.5E1,7E0,1.36E2,2.7E1,2E1,7E0,5.1E1,3E1,6E1,4E0,1.7E1,4E0,7.2E1,3.3E1,9E0,4E0,5E0,1.25E2,7E0,1.3E1,1.6E1,5.8E1,1.7E1,4E0,3.2E1,7E0,7E0,5E0,1.3E1,1.23E2,1.7E1,1E1,4E0,1.6E1,3E1,2.1E1,1.5E1,1.5E1,4.7E1,1.3E1,1.2E1,5E0,6.3E1,9E0,9E0,2.4E1,9.2E1,3.3E1,4.5E1,1.3E1,1E1,7E0,4E0,2.8E1,2.2E1,1.01E2,8E0,9E0,6E0,4E0,1.1E1,5E0,5E0,2.5E1,1.5E1,6E0,2.9E1,1.8E1,9E0,4E0,7E0,5E0,4.4E1,1.9E1,1.8E1,6E0,1.5E1,7.7E1,1.7E1,1.6E1,3.6E1,9E0,1.2E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-3.4122604E-3,-2.430848E-2,9.615928E-3,-1.9493194E-2,-1.2938137E-1,1.4941398E-2,-4.113631E-2,-1.576189E-2,-6.713998E-2,-1.2116654E-2,-5.2154955E-2,3.404663E-2,3.869755E-3,-9.680642E-3,-2.5365511E-2,-4.5177612E-2,-1.0360505E-2,-2.1487218E-1,6.372106E-3,9.4395614E-4,-4.204237E-3,2.5462193E-2,6.391187E-2,-6.043369E-2,1.0937199E-2,-3.910601E-2,3.5298183E-3,-8.04002E-3,-6.352307E-2,2.92759E-4,-2.0525085E-2,-3.4781634E-3,-1.43902395E-2,-1.47886565E-2,2.5264043E-3,6.745667E-2,1.3960125E-2,7.677619E-2,2.1064932E-2,-1.634125E-1,2.0904155E-2,2.8469943E-2,-4.453927E-3,-5.89141E-2,5.100592E-4,-1.5182093E-3,1.8113337E-3,9.6618006E-4,-3.4525094E-3,-7.189631E-3,3.8627807E-2,-6.6283636E-2,-1.05567835E-2,1.8204711E-3,-2.276437E-3,3.5901018E-3,-4.4359217E-4,2.3128465E-3,4.5508087E-2,8.90509E-2,1.1188345E-3,-1.0882863E-3,4.0768897E-3,-9.455465E-3,-1.6631957E-2,-2.6570057E-4,4.088667E-3,7.813883E-3,4.2457435E-2,6.957198E-4,-8.301094E-3,-4.130163E-2,-7.3849675E-3,1.4438928E-2,-1.6662836E-2,-1.058075E-3,6.0410284E-2,-4.1354834E-3,-1.5883277E-3,-6.850375E-3,-4.5682453E-3,-6.2522357E-3,3.1907817E-3,6.395616E-2,-3.237498E-4,5.1193666E-2,1.23251446E-1,-2.5711127E-3,1.4145382E-3,1.4943029E-2,-3.59765E-3,4.8511443E-3,2.9721396E-2,-5.66817E-3,6.7258045E-2,-3.552426E-3,-1.2119684E-3,1.5243745E-4,2.5778625E-3,-3.6472242E-2,-2.3530026E-3,3.5207032E-3,1.8926304E-4,-1.917663E-2,1.7084794E-2,-3.6043897E-2,6.5483446E-3,-1.00886034E-4,3.6870416E-3,3.405424E-3,-3.9663675E-4,6.8591866E-3,2.225105E-3,-1.7099573E-3,5.296983E-2,1.3106582E-2,6.0004137E-2,1.35541065E-2,-1.9758547E-2,4.5102823E-4,4.032157E-3,-3.6705396E-4,-2.844864E-3,2.3753118E-2,-1.996189E-2,9.818433E-4,-3.0247677E-2,-1.1644831E-2,1.8071781E-3,-8.0444245E-4,-4.400291E-3,3.0311547E-2,-1.7426096E-2,-2.5337454E-2,3.1201009E-2,3.9751194E-3,-1.2712923E-3,5.095153E-2,-2.8290223E-2,3.9076055E-3,1.6565343E-2,-9.108564E-4,5.9718102E-2,-5.727447E-2,-6.9668833E-3,3.9136843E-3,-2.7331631E-5,-2.6043411E-3,2.8058012E-6,-5.1617913E-5,-1.8449372E-3,4.8178105E-4,-2.8277594E-3,6.156484E-4,2.9867042E-3,-2.7970145E-3,9.3909475E-6,2.2863483E-3,-2.1491125E-3,-3.2629122E-4,2.5960503E-3,2.5885573E-4,3.5710714E-3,-1.2076759E-4,-3.81259E-3,-1.8508105E-3,2.5246267E-3,-2.178779E-3,8.830343E-4,3.728812E-3,-1.2060668E-4,-6.442816E-4,-3.8868415E-3,2.7928483E-3,-7.7613455E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,29,31,33,-1,-1,35,37,39,41,43,-1,45,47,49,51,-1,-1,53,-1,55,57,59,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,77,79,-1,-1,-1,-1,81,83,85,-1,-1,-1,87,-1,-1,-1,89,91,93,-1,95,-1,97,99,-1,101,-1,-1,103,-1,105,-1,107,-1,109,111,-1,-1,113,-1,-1,115,117,119,-1,-1,-1,-1,121,123,-1,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,133,135,137,139,141,143,-1,-1,-1,-1,145,147,-1,149,151,-1,-1,-1,153,155,157,159,-1,-1,161,163,-1,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.779184E-1,1.9581272E-1,1.7062047E-1,6.578879E-2,1.6078484E-1,1.2043826E-1,1.5403841E-1,5.5141896E-2,2.9459193E-1,0E0,3.5697617E-2,5.1955268E-2,1.6517586E-1,0E0,8.129669E-2,3.6279805E-2,3.216694E-2,9.7919494E-2,2.0233218E-2,0E0,0E0,7.827414E-2,2.3737773E-2,3.049547E-1,8.836733E-2,5.0315738E-2,0E0,2.3571357E-2,3.5632312E-2,4.2156037E-2,6.863279E-2,0E0,0E0,2.8106462E-2,0E0,2.839829E-2,4.7662977E-2,2.0423293E-2,3.805142E-2,4.8086843E-1,3.8347248E-2,4.3873988E-2,1.5803763E-1,5.739686E-2,0E0,0E0,0E0,0E0,0E0,2.544228E-2,3.3550337E-2,1.6933404E-2,4.057107E-2,0E0,0E0,0E0,0E0,5.531943E-2,3.435672E-2,2.9942974E-2,0E0,0E0,0E0,2.0217083E-2,0E0,0E0,0E0,3.9648652E-2,6.937079E-2,7.281602E-2,0E0,1.5144557E-2,0E0,1.7323423E-2,2.4292538E-2,0E0,1.570034E-2,0E0,0E0,3.6321416E-2,0E0,3.3032175E-2,0E0,2.7433239E-2,0E0,2.0824704E-2,1.936005E-2,0E0,0E0,3.7452903E-2,0E0,0E0,3.7955493E-2,4.279112E-2,1.6413674E-2,0E0,0E0,0E0,0E0,2.5294248E-2,2.3918843E-2,0E0,0E0,3.6897816E-2,2.7040808E-2,2.7020108E-2,3.5280492E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.344991E-2,5.0283797E-2,8.134108E-2,2.7424872E-2,4.4794347E-2,4.327106E-2,0E0,0E0,0E0,0E0,3.1215465E-2,2.2603339E-2,0E0,1.8015005E-2,2.1424718E-2,0E0,0E0,0E0,1.7218739E-2,2.357597E-2,4.1924838E-2,1.7609848E-2,0E0,0E0,3.1825155E-2,3.459797E-2,0E0,2.5910014E-2,4.906027E-2,2.1146942E-2,2.559235E-2,4.6757687E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,50,50,51,51,52,52,57,57,58,58,59,59,63,63,67,67,68,68,69,69,71,71,73,73,74,74,76,76,79,79,81,81,83,83,85,85,86,86,89,89,92,92,93,93,94,94,99,99,100,100,103,103,104,104,105,105,106,106,113,113,114,114,115,115,116,116,117,117,118,118,123,123,124,124,126,126,127,127,131,131,132,132,133,133,134,134,137,137,138,138,140,140,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,30,32,34,-1,-1,36,38,40,42,44,-1,46,48,50,52,-1,-1,54,-1,56,58,60,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,78,80,-1,-1,-1,-1,82,84,86,-1,-1,-1,88,-1,-1,-1,90,92,94,-1,96,-1,98,100,-1,102,-1,-1,104,-1,106,-1,108,-1,110,112,-1,-1,114,-1,-1,116,118,120,-1,-1,-1,-1,122,124,-1,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,134,136,138,140,142,144,-1,-1,-1,-1,146,148,-1,150,152,-1,-1,-1,154,156,158,160,-1,-1,162,164,-1,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,6.0692043E0,2.856934E7,1.6631816E8,2E0,3.72381E5,2.7920364E2,1.63757E4,5.1942E4,-1.2116654E-2,3.3817584E7,8.260895E8,1E0,-9.680642E-3,2.3751075E5,5.4009717E3,3.66025E5,3E0,2.906237E2,9.4395614E-4,-4.204237E-3,1.6869704E6,2.8277853E11,1.048796E6,1.8497453E0,2.0618556E-2,3.5298183E-3,2.5192308E0,8E0,2E1,8.39777E5,-3.4781634E-3,-1.43902395E-2,3.5714287E-1,2.5264043E-3,3.89E2,1.38132E5,1.0284041E2,4.7083335E0,1.8669039E1,1.5971001E1,5.1191E4,1.9500381E2,1.6322028E3,5.100592E-4,-1.5182093E-3,1.8113337E-3,9.6618006E-4,-3.4525094E-3,1.2E1,3.2E1,1.1732674E1,7.814751E6,1.8204711E-3,-2.276437E-3,3.5901018E-3,-4.4359217E-4,1.6322028E3,1.3E1,1.0717949E1,1.1188345E-3,-1.0882863E-3,4.0768897E-3,1.03860024E8,-1.6631957E-2,-2.6570057E-4,4.088667E-3,2.152E3,1.5037551E4,1.3388E4,-8.301094E-3,1.775373E1,-7.3849675E-3,2.1924414E2,1.4473684E-1,-1.058075E-3,6.763314E7,-4.1354834E-3,-1.5883277E-3,3.11626E8,-4.5682453E-3,3.6522612E6,3.1907817E-3,3.741267E7,-3.237498E-4,9E0,2.5503825E6,-2.5711127E-3,1.4145382E-3,9.7199225E-1,-3.59765E-3,4.8511443E-3,1.7E1,1.9E1,3.8396227E0,-3.552426E-3,-1.2119684E-3,1.5243745E-4,2.5778625E-3,4.0869565E0,2.0625E0,3.5207032E-3,1.8926304E-4,2.8E1,1.7391304E0,5.556659E6,2.308943E0,-1.00886034E-4,3.6870416E-3,3.405424E-3,-3.9663675E-4,6.8591866E-3,2.225105E-3,2.732002E7,2.884462E0,5.812E3,2.1818182E0,1.0901037E10,7.919006E7,4.5102823E-4,4.032157E-3,-3.6705396E-4,-2.844864E-3,1.5365228E5,2.5437157E2,9.818433E-4,3.088104E5,2.49E2,1.8071781E-3,-8.0444245E-4,-4.400291E-3,1.225E0,3.5589743E0,3.53E2,5.036784E6,3.9751194E-3,-1.2712923E-3,9.8933E4,9E0,3.9076055E-3,1.0130841E1,2.3266666E0,1.7022608E7,3.0588236E0,1.4872362E-1,3.9136843E-3,-2.7331631E-5,-2.6043411E-3,2.8058012E-6,-5.1617913E-5,-1.8449372E-3,4.8178105E-4,-2.8277594E-3,6.156484E-4,2.9867042E-3,-2.7970145E-3,9.3909475E-6,2.2863483E-3,-2.1491125E-3,-3.2629122E-4,2.5960503E-3,2.5885573E-4,3.5710714E-3,-1.2076759E-4,-3.81259E-3,-1.8508105E-3,2.5246267E-3,-2.178779E-3,8.830343E-4,3.728812E-3,-1.2060668E-4,-6.442816E-4,-3.8868415E-3,2.7928483E-3,-7.7613455E-4],"split_indices":[2,68,60,5,32,9,48,43,1,0,7,5,8,0,48,43,9,8,4,0,0,60,46,9,68,72,0,68,3,3,9,0,0,68,0,8,9,73,69,73,71,1,73,4,0,0,0,0,0,3,12,73,60,0,0,0,0,4,8,71,0,0,0,7,0,0,0,2,48,2,0,71,0,67,71,0,7,0,0,46,0,60,0,7,0,8,47,0,0,53,0,0,8,8,69,0,0,0,0,73,71,0,0,10,71,62,69,0,0,0,0,0,0,5,71,44,69,46,7,0,0,0,0,43,4,0,60,10,0,0,0,68,69,44,9,0,0,1,8,0,71,68,9,68,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.019E3,3.91E2,6.28E2,3.75E2,1.6E1,5.69E2,5.9E1,3.49E2,2.6E1,5E0,1.1E1,2.08E2,3.61E2,4E0,5.5E1,5.3E1,2.96E2,8E0,1.8E1,4E0,7E0,1.63E2,4.5E1,3.5E1,3.26E2,4.9E1,6E0,1.8E1,3.5E1,1.45E2,1.51E2,4E0,4E0,1.3E1,5E0,3.4E1,1.29E2,3.4E1,1.1E1,1.5E1,2E1,1.52E2,1.74E2,3.5E1,1.4E1,1.2E1,6E0,4E0,3.1E1,1.22E2,2.3E1,2.6E1,1.25E2,5E0,8E0,3E1,4E0,9.5E1,3.4E1,2.7E1,7E0,7E0,4E0,9E0,6E0,1.5E1,5E0,6.2E1,9E1,1.7E2,4E0,3.1E1,4E0,3.7E1,8.5E1,6E0,1.7E1,1.4E1,1.2E1,1.21E2,4E0,8.5E1,1E1,2.5E1,9E0,1.4E1,1.3E1,4E0,5E0,5.8E1,4E0,1.4E1,7.6E1,1.56E2,1.4E1,8E0,2.3E1,3E1,7E0,3.5E1,5E1,1.3E1,4E0,8E1,4.1E1,2.5E1,6E1,5E0,2E1,1E1,4E0,9E0,4E0,4.1E1,1.7E1,5E1,2.6E1,6.6E1,9E1,4E0,1E1,1.7E1,1.8E1,2E1,3E1,1.7E1,6.3E1,1.8E1,2.3E1,2E1,5E0,3E1,3E1,2.4E1,1.7E1,1.2E1,5E0,2.6E1,2.4E1,1.6E1,1E1,5.1E1,1.5E1,2.2E1,6.8E1,5E0,1.5E1,1E1,2E1,1.6E1,4.7E1,1.3E1,5E0,2.1E1,9E0,8E0,2.2E1,5E0,1.9E1,7E0,1E1,1E1,1.6E1,1.7E1,7E0,4E0,6E0,1.5E1,3.6E1,1.1E1,4E0,9E0,1.3E1,8E0,6E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[1.1045056E-4,-1.2815726E-2,1.8037228E-2,-1.0047119E-2,-9.884234E-2,2.6030619E-2,-2.7895272E-2,-1.3516349E-2,2.8885858E-2,1.9730919E-3,-1.849727E-1,5.351257E-3,2.3015238E-2,-3.895394E-2,2.6402446E-3,-2.3300089E-2,3.4412355E-3,-1.0403417E-2,5.4115847E-2,-1.2781561E-2,-2.1793044E-3,8.731328E-2,1.904624E-2,-5.320839E-2,1.5577832E-2,-8.889742E-3,-2.0766817E-2,8.538131E-3,-6.032947E-2,-1.9747985E-3,2.0074777E-3,1.0895453E-3,3.8380194E-3,1.1194588E-1,-6.421764E-4,1.0966069E-2,3.8813796E-2,-6.912194E-2,-6.271259E-3,4.667697E-3,-1.5536258E-3,-4.356134E-3,-1.850333E-2,-3.2385606E-3,1.1985731E-2,-8.4086595E-4,-6.511587E-3,2.1493388E-4,7.2192405E-3,1.346729E-2,-5.2106287E-3,5.8112144E-3,2.9883713E-2,-5.0095234E-2,-8.004593E-3,1.1779047E-3,-2.8735166E-3,1.4696375E-3,-2.8318046E-2,1.5294258E-2,-3.3248083E-3,5.098607E-4,3.7546866E-2,-3.442261E-2,3.904164E-2,-1.0026759E-2,-6.8698E-2,-3.5556927E-3,2.7173567E-3,-1.8700685E-2,-4.8779387E-2,-4.7632554E-3,2.4359833E-2,9.096348E-3,-1.4360896E-1,4.2818524E-2,-2.557734E-3,8.6203177E-4,-2.9242078E-3,-1.238015E-3,4.5631785E-2,-2.2741118E-3,1.4248629E-3,-5.178553E-3,-2.0266369E-3,7.542003E-3,-3.213402E-2,-8.339582E-3,-5.7941783E-2,-6.421848E-2,-6.1987992E-3,9.651417E-3,-5.2076153E-2,2.788856E-2,-2.1867906E-3,-4.310182E-2,1.693625E-2,-1.04953805E-2,-1.4468948E-3,3.3015613E-2,4.7215084E-3,2.8121676E-2,6.901285E-2,1.2956869E-3,-6.594403E-4,-7.989502E-4,-3.979578E-3,-8.8121585E-4,1.4106815E-3,-5.0460147E-3,7.296363E-4,-8.736107E-4,-3.9585796E-3,-2.6647928E-3,4.7477442E-4,8.5820846E-4,-2.8594034E-3,-3.5427369E-3,1.21667406E-4,1.0155893E-3,2.461328E-3,-3.6172033E-4,-3.9427355E-3,1.8423343E-4,1.8254595E-3,2.1788385E-3,1.546392E-4,4.2802566E-3,8.0725714E-4,3.6999774E-3,-2.1224305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,27,29,31,-1,-1,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,47,-1,49,51,53,55,-1,-1,-1,57,-1,59,-1,-1,-1,-1,61,-1,-1,63,65,-1,-1,-1,67,69,71,-1,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,-1,-1,-1,-1,101,-1,-1,-1,-1,103,105,107,109,111,113,115,117,119,-1,121,123,-1,-1,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.44469E-1,1.4500752E-1,1.6300927E-1,8.075556E-2,2.3954457E-1,9.941837E-2,6.4297885E-2,9.0832196E-2,4.9135122E-2,0E0,1.3856879E-1,0E0,9.173985E-2,4.6690777E-2,0E0,1.4467044E-1,6.584665E-2,3.6693778E-2,2.456697E-2,0E0,0E0,5.3912237E-2,5.4839283E-2,3.467253E-2,5.8063302E-2,0E0,5.533734E-2,5.1538415E-2,4.8378706E-2,0E0,0E0,0E0,0E0,8.010849E-2,0E0,7.797346E-2,7.390836E-2,6.0671598E-2,2.4682611E-2,0E0,0E0,0E0,6.55153E-2,0E0,5.1097475E-2,0E0,0E0,0E0,0E0,7.579915E-2,0E0,0E0,5.5523872E-2,2.2418097E-2,0E0,0E0,0E0,3.219898E-2,4.3167427E-2,3.1754952E-2,0E0,1.9806474E-1,4.400278E-2,1.9572716E-2,3.6367826E-2,1.9139552E-2,1.7710283E-2,3.2909174E-2,0E0,6.213027E-2,4.6277687E-2,3.8012605E-2,3.16787E-2,6.252349E-2,7.734038E-2,4.3776244E-2,0E0,0E0,0E0,0E0,2.8355211E-2,0E0,0E0,0E0,0E0,3.3948395E-2,2.1752847E-2,5.1911224E-2,1.2272502E-1,4.895675E-2,1.7556578E-2,2.860646E-2,1.7703831E-2,1.6071111E-2,0E0,2.9130146E-2,3.8430322E-2,0E0,0E0,2.9640816E-2,0E0,2.8613165E-2,2.5638789E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,28,28,33,33,35,35,36,36,37,37,38,38,42,42,44,44,49,49,52,52,53,53,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,80,80,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,96,96,99,99,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,28,30,32,-1,-1,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,48,-1,50,52,54,56,-1,-1,-1,58,-1,60,-1,-1,-1,-1,62,-1,-1,64,66,-1,-1,-1,68,70,72,-1,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,-1,-1,-1,-1,102,-1,-1,-1,-1,104,106,108,110,112,114,116,118,120,-1,122,124,-1,-1,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,6.0692043E0,2.856934E7,4.5866325E6,1.901875E2,1.2E1,4.435876E2,1.380863E6,1E0,1.9730919E-3,2.6235E4,5.351257E-3,1.0918E4,2.3308511E5,2.6402446E-3,4E0,1.2E1,1E0,1.81E2,-1.2781561E-2,-2.1793044E-3,1.4791E4,9.033567E6,2.0618556E-2,1E0,-8.889742E-3,6E0,3.7788504E7,1.3994015E7,-1.9747985E-3,2.0074777E-3,1.0895453E-3,3.8380194E-3,3.7509E4,-6.421764E-4,6.5026445E3,5E0,4.9E1,3.463702E2,4.667697E-3,-1.5536258E-3,-4.356134E-3,1.4615384E0,-3.2385606E-3,4.0903897E2,-8.4086595E-4,-6.511587E-3,2.1493388E-4,7.2192405E-3,1.7068776E10,-5.2106287E-3,5.8112144E-3,1.2895454E3,6.5E2,-8.004593E-3,1.1779047E-3,-2.8735166E-3,1.3E1,3.14403E5,1.0140845E0,-3.3248083E-3,2.1373269E8,2.5842668E7,1.194E3,1E1,5.17259E8,1.33E2,1E0,2.7173567E-3,1.65E2,9.5E2,1.2751362E8,5.123862E1,1E0,4.626E3,1.7743401E6,-2.557734E-3,8.6203177E-4,-2.9242078E-3,-1.238015E-3,1.9001543E1,-2.2741118E-3,1.4248629E-3,-5.178553E-3,-2.0266369E-3,3.883E4,8E0,5.5E1,2.2084616E2,3.56E2,2.8266037E2,1.8E1,1.3E1,7.884708E0,-2.1867906E-3,1.2585366E1,7.445455E2,-1.04953805E-2,-1.4468948E-3,4.076E4,4.7215084E-3,9.5596474E-1,2.7561485E6,1.2956869E-3,-6.594403E-4,-7.989502E-4,-3.979578E-3,-8.8121585E-4,1.4106815E-3,-5.0460147E-3,7.296363E-4,-8.736107E-4,-3.9585796E-3,-2.6647928E-3,4.7477442E-4,8.5820846E-4,-2.8594034E-3,-3.5427369E-3,1.21667406E-4,1.0155893E-3,2.461328E-3,-3.6172033E-4,-3.9427355E-3,1.8423343E-4,1.8254595E-3,2.1788385E-3,1.546392E-4,4.2802566E-3,8.0725714E-4,3.6999774E-3,-2.1224305E-4],"split_indices":[4,68,60,43,4,33,71,9,29,0,1,0,9,48,0,3,71,16,0,0,0,2,60,72,15,0,67,7,5,0,0,0,0,1,0,4,3,3,71,0,0,0,68,0,67,0,0,0,0,46,0,0,48,10,0,0,0,8,9,71,0,5,62,2,3,46,0,8,0,0,10,7,73,29,2,58,0,0,0,0,71,0,0,0,0,9,32,0,67,2,4,3,8,71,0,71,67,0,0,44,0,68,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.053E3,6.12E2,4.41E2,5.94E2,1.8E1,3.76E2,6.5E1,5.46E2,4.8E1,7E0,1.1E1,1.1E1,3.65E2,5.8E1,7E0,3.46E2,2E2,1.9E1,2.9E1,6E0,5E0,2E1,3.45E2,4.6E1,1.2E1,4E0,3.42E2,1.86E2,1.4E1,1.2E1,7E0,1.5E1,1.4E1,1.6E1,4E0,2.46E2,9.9E1,3.4E1,1.2E1,4E0,8E0,9E0,3.33E2,7E0,1.79E2,1E1,4E0,5E0,1.1E1,2.42E2,4E0,8E0,9.1E1,3E1,4E0,8E0,4E0,1.1E2,2.23E2,1.73E2,6E0,1.58E2,8.4E1,1.1E1,8E1,1E1,2E1,1.02E2,8E0,1.53E2,7E1,5.4E1,1.19E2,1.5E2,8E0,8E1,4E0,4E0,7E0,7E0,7.3E1,5E0,5E0,6E0,1.4E1,7.4E1,2.8E1,1.22E2,3.1E1,5.1E1,1.9E1,4.2E1,1.2E1,1.14E2,5E0,1.9E1,1.31E2,4E0,4E0,7E1,1E1,4.3E1,3E1,3.8E1,3.6E1,2.3E1,5E0,9.6E1,2.6E1,1.8E1,1.3E1,1.7E1,3.4E1,4E0,1.5E1,3.8E1,4E0,8E0,4E0,9.4E1,2E1,1.1E1,8E0,8.4E1,4.7E1,4.7E1,2.3E1,5E0,3.8E1,2.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[3.93045E-4,-1.3073961E-2,1.5106187E-2,6.039252E-2,-1.6067546E-2,1.2840821E-2,4.183979E-3,-1.3171644E-3,4.937781E-3,-1.2578861E-2,-6.315427E-2,1.4488584E-2,-3.915205E-3,-1.6600374E-2,2.839642E-2,-9.160539E-3,-1.0814054E-1,-2.6114536E-3,2.2298638E-2,2.7369767E-2,-2.0396769E-2,-5.7654246E-3,4.7308274E-2,2.9032074E-3,-3.614889E-2,-1.28113E-2,-4.454249E-2,9.089443E-3,-2.072299E-2,4.0063714E-3,1.9067867E-2,6.4171925E-2,-5.060151E-4,-1.4067255E-2,-5.613506E-2,5.745423E-2,-3.3637436E-4,5.193359E-4,-3.1033242E-3,-9.091091E-2,3.0665323E-3,-1.3860628E-2,3.2347526E-2,-2.3520426E-3,5.395782E-3,1.8976177E-3,3.028788E-2,1.9093285E-3,5.326736E-3,-5.4896926E-3,-1.1842894E-2,-1.08501256E-1,-6.9239717E-3,3.7535783E-3,1.9359307E-2,-6.4186123E-4,-7.0581343E-3,2.6896414E-3,-2.4070736E-2,3.5155264E-3,1.4133363E-2,2.9097486E-3,-7.587823E-4,7.510281E-3,-6.1715238E-2,4.9648867E-3,2.5730597E-2,-1.9521604E-3,-2.673824E-2,-1.3320321E-1,-5.468816E-4,-3.5595693E-2,3.7718985E-2,-1.2059099E-3,2.620339E-3,-3.5133367E-4,-3.1547733E-3,3.8654448E-3,-1.2719702E-3,-2.8051767E-3,1.2588836E-2,-1.2615975E-4,-5.550979E-3,5.3099122E-2,1.11167105E-2,-2.171132E-2,1.1895519E-2,-3.152733E-2,2.457812E-2,-1.7551357E-1,-4.7885578E-2,-2.3510227E-4,-3.729177E-3,5.113411E-3,-7.007122E-3,-1.6715389E-2,3.377651E-3,3.6446556E-2,-1.2793975E-3,6.6027895E-2,1.8545505E-2,3.801265E-2,-1.643462E-2,-6.074711E-4,-5.2675437E-3,8.450591E-4,-1.9476142E-3,-4.3062828E-4,-1.945756E-3,-1.6807514E-3,3.387191E-3,-1.25386E-2,-5.3577637E-3,-3.9051617E-3,-7.6920966E-5,1.7336402E-3,-2.3163406E-3,5.540255E-4,-2.39E-3,8.3220925E-4,4.0701777E-3,-1.7803643E-3,8.6021045E-4,4.569807E-3,1.8227851E-3,-1.7364054E-3,1.7217251E-3,2.5233924E-3,-2.8783127E-4,-6.54822E-3,1.6369525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,-1,19,21,23,25,27,29,31,33,-1,35,-1,37,-1,39,41,43,-1,45,47,-1,49,51,53,-1,-1,-1,55,-1,57,59,-1,61,63,65,-1,-1,-1,67,69,71,-1,73,-1,-1,-1,75,-1,77,-1,-1,79,81,-1,83,85,87,89,-1,91,93,-1,-1,-1,-1,-1,95,-1,97,-1,-1,99,101,103,105,107,109,111,113,-1,-1,-1,115,117,-1,119,121,123,125,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0150974E-1,1.1775222E-1,8.191388E-2,8.983888E-2,8.314797E-2,7.728656E-2,0E0,0E0,0E0,7.9072714E-2,8.402355E-2,6.2293567E-2,0E0,7.3171936E-2,1.3048084E-1,3.5306223E-2,1.9206207E-1,3.134078E-2,6.494801E-2,5.0065406E-2,8.9918986E-2,0E0,2.2018872E-2,0E0,1.9868134E-2,0E0,8.3181426E-2,4.848808E-2,4.6006266E-2,0E0,5.889356E-2,1.6257837E-2,0E0,7.828672E-2,1.5386857E-1,2.8338954E-2,0E0,0E0,0E0,5.028961E-2,0E0,3.5254423E-2,3.4781747E-2,0E0,4.1244015E-2,4.402208E-2,6.2304556E-2,0E0,0E0,0E0,4.9449306E-2,6.7668915E-2,4.230233E-2,0E0,2.5258288E-2,0E0,0E0,0E0,2.9330969E-2,0E0,3.5456855E-2,0E0,0E0,3.991979E-2,3.4141395E-2,0E0,6.980276E-2,5.580282E-2,3.3706546E-2,7.462129E-2,0E0,2.7121047E-2,4.3199234E-2,0E0,0E0,0E0,0E0,0E0,3.3386372E-2,0E0,3.511826E-2,0E0,0E0,2.6026368E-2,8.6582914E-2,6.48913E-2,4.2258702E-2,2.853109E-2,3.832032E-2,5.4167837E-2,1.5102407E-2,0E0,0E0,0E0,1.9168131E-2,2.4746615E-2,0E0,3.5546616E-2,5.1131885E-2,3.4549788E-2,2.0316612E-2,4.3690994E-2,1.4569485E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,26,26,27,27,28,28,30,30,31,31,33,33,34,34,35,35,39,39,41,41,42,42,44,44,45,45,46,46,50,50,51,51,52,52,54,54,58,58,60,60,63,63,64,64,66,66,67,67,68,68,69,69,71,71,72,72,78,78,80,80,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,94,94,95,95,97,97,98,98,99,99,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,-1,20,22,24,26,28,30,32,34,-1,36,-1,38,-1,40,42,44,-1,46,48,-1,50,52,54,-1,-1,-1,56,-1,58,60,-1,62,64,66,-1,-1,-1,68,70,72,-1,74,-1,-1,-1,76,-1,78,-1,-1,80,82,-1,84,86,88,90,-1,92,94,-1,-1,-1,-1,-1,96,-1,98,-1,-1,100,102,104,106,108,110,112,114,-1,-1,-1,116,118,-1,120,122,124,126,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.969174E7,7E0,9.059046E6,3.5443038E-1,6.0692043E0,1E0,4.183979E-3,-1.3171644E-3,4.937781E-3,2.008E3,1.2999985E6,3.808001E5,-3.915205E-3,7.3333335E-1,2.1126761E0,5.004138E-3,1.088E3,3.38E2,1.0164831E-2,4.31903E6,6.648E3,-5.7654246E-3,4.5620965E6,2.9032074E-3,1.1634076E4,-1.28113E-2,1.764442E7,4.062891E8,1.19E2,4.0063714E-3,8.1407714E-1,6.347708E7,-5.060151E-4,6E0,7.714286E0,3.015873E0,-3.3637436E-4,5.193359E-4,-3.1033242E-3,1.8471494E7,3.0665323E-3,7.807738E6,1.3209776E6,-2.3520426E-3,5.49E2,1.2769117E7,8.998703E-1,1.9093285E-3,5.326736E-3,-5.4896926E-3,1E0,2.378914E6,1.1362371E6,3.7535783E-3,5.6115704E0,-6.4186123E-4,-7.0581343E-3,2.6896414E-3,1.4195632E6,3.5155264E-3,3.9961785E-2,2.9097486E-3,-7.587823E-4,2.87834E5,7.3835544E5,4.9648867E-3,5.8606E4,4.1277572E5,2.3E1,8.39E2,-5.468816E-4,1.34214E5,7.82E2,-1.2059099E-3,2.620339E-3,-3.5133367E-4,-3.1547733E-3,3.8654448E-3,5.467E3,-2.8051767E-3,1.4327235E7,-1.2615975E-4,-5.550979E-3,1.016E4,1.5153E4,2.923526E5,1.3177083E1,1.7142857E0,7E0,2E0,5.095006E5,-2.3510227E-4,-3.729177E-3,5.113411E-3,1.3152658E7,1.393875E5,3.377651E-3,6.819212E6,2.194E3,2.1298597E0,1.3388E4,1.1E1,5.668E3,-6.074711E-4,-5.2675437E-3,8.450591E-4,-1.9476142E-3,-4.3062828E-4,-1.945756E-3,-1.6807514E-3,3.387191E-3,-1.25386E-2,-5.3577637E-3,-3.9051617E-3,-7.6920966E-5,1.7336402E-3,-2.3163406E-3,5.540255E-4,-2.39E-3,8.3220925E-4,4.0701777E-3,-1.7803643E-3,8.6021045E-4,4.569807E-3,1.8227851E-3,-1.7364054E-3,1.7217251E-3,2.5233924E-3,-2.8783127E-4,-6.54822E-3,1.6369525E-4],"split_indices":[7,3,43,71,68,79,0,0,0,2,60,43,0,68,69,53,44,0,53,60,44,0,62,0,43,0,60,46,8,0,53,7,0,67,71,68,0,0,0,5,0,46,60,0,0,62,57,0,0,0,26,9,43,0,69,0,0,0,60,0,53,0,0,12,43,0,9,60,3,2,0,1,2,0,0,0,0,0,2,0,12,0,0,2,10,60,71,68,8,32,43,0,0,0,60,48,0,12,2,53,2,8,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,5.3E2,4.85E2,2E1,5.1E2,4.72E2,1.3E1,7E0,1.3E1,4.76E2,3.4E1,4.65E2,7E0,4.34E2,4.2E1,1.6E1,1.8E1,1.46E2,3.19E2,3.4E1,4E2,4E0,3.8E1,4E0,1.2E1,4E0,1.4E1,8.9E1,5.7E1,1.4E1,3.05E2,1.7E1,1.7E1,3.41E2,5.9E1,3.2E1,6E0,5E0,7E0,1E1,4E0,4.5E1,4.4E1,2.6E1,3.1E1,1.21E2,1.84E2,1.3E1,4E0,6E0,3.35E2,2.8E1,3.1E1,1.9E1,1.3E1,5E0,5E0,5E0,4E1,1.2E1,3.2E1,8E0,2.3E1,1.12E2,9E0,9E0,1.75E2,2.02E2,1.33E2,2.2E1,6E0,1.9E1,1.2E1,6E0,7E0,3E1,1E1,5E0,2.7E1,7E0,1.05E2,5E0,4E0,6E1,1.15E2,8.3E1,1.19E2,1.22E2,1.1E1,1.4E1,8E0,1.2E1,7E0,4E0,8E0,2.3E1,4E0,3.8E1,6.7E1,4.3E1,1.7E1,5.8E1,5.7E1,7.7E1,6E0,1.07E2,1.2E1,4E1,8.2E1,5E0,6E0,4E0,1E1,4E0,4E0,4E0,4E0,1.3E1,1E1,2.9E1,9E0,2.3E1,4.4E1,1.8E1,2.5E1,4E0,1.3E1,4.2E1,1.6E1,7E0,5E1],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-1.554028E-3,-1.8304912E-2,1.6564656E-2,-1.5394103E-2,-1.174105E-1,4.5295143E-3,1.3435578E-2,-1.7329253E-2,2.893446E-2,7.616974E-4,-2.0850763E-1,-2.7874842E-2,1.9149173E-2,-1.4193781E-2,-5.370661E-2,6.4994946E-2,-9.626782E-4,-1.3356135E-2,-3.9830017E-3,-2.8058556E-3,-1.0309225E-2,4.101179E-3,1.6330482E-2,-2.546953E-2,-5.6612757E-3,-5.8101793E-3,-3.6794227E-2,1.7421166E-4,4.7487365E-3,6.8761827E-3,-2.5261184E-3,1.7881889E-2,-3.4079968E-3,-2.1883378E-2,-9.79488E-2,4.765879E-3,-2.6690217E-2,1.0881517E-3,-6.842333E-2,2.1666206E-2,-2.4684125E-3,1.5434419E-2,1.02140196E-1,-1.8405905E-2,-2.817911E-3,-6.939745E-3,-1.2055098E-3,9.4584463E-4,3.9208382E-3,-8.255553E-2,-9.86674E-3,-1.238414E-3,2.4044833E-3,-4.386652E-3,-6.0855004E-4,-5.4762646E-4,5.094096E-2,1.0483627E-2,4.634253E-2,7.2312662E-3,2.2500358E-3,-4.120051E-2,-9.466208E-3,5.538158E-3,-5.959844E-2,-6.128115E-4,-1.1181261E-2,-3.3496052E-2,3.211552E-2,3.636293E-3,2.5069484E-4,1.4981445E-2,-2.6025381E-2,2.7629407E-2,3.5036304E-3,-2.5828788E-2,-8.644425E-2,3.2633454E-2,-2.5208378E-2,2.6760949E-2,-1.4966975E-2,-8.099305E-4,-5.3122505E-3,-4.056157E-3,-2.0849917E-2,5.952784E-3,7.554004E-3,1.8604428E-2,-1.9266376E-2,3.207527E-3,-3.969943E-2,5.221785E-2,-1.4170104E-6,-1.921377E-3,1.0375849E-3,-6.2345397E-3,-2.0938758E-3,2.3278622E-3,-2.1058281E-3,-1.5122887E-4,-1.7460182E-3,1.591008E-3,-2.065975E-3,-1.4087139E-3,5.7482865E-4,1.2719618E-3,-1.5271936E-3,-1.1451181E-3,2.4683645E-3,-6.922327E-5,1.1849086E-3,3.1147618E-4,-2.618261E-3,3.176464E-4,-3.248578E-3,3.0440209E-4,5.0093806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,35,-1,37,-1,-1,39,-1,41,-1,43,45,47,49,51,53,55,-1,57,59,61,-1,-1,-1,63,-1,65,67,-1,-1,-1,-1,-1,69,71,73,-1,-1,75,77,79,81,-1,-1,83,85,-1,-1,87,89,91,-1,93,95,97,99,101,103,-1,-1,-1,105,-1,107,109,111,-1,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.101808E-1,1.5137413E-1,1.2386489E-1,4.48509E-2,1.864033E-1,0E0,1.12412475E-1,5.5811554E-2,4.084006E-2,0E0,6.377882E-2,3.285103E-2,8.038448E-2,4.392018E-2,4.2900458E-2,2.9692862E-2,0E0,0E0,0E0,0E0,3.0113984E-2,0E0,5.7510197E-2,4.906112E-2,5.757647E-2,0E0,3.961458E-2,0E0,0E0,2.7023563E-2,0E0,8.0086514E-2,0E0,2.4583153E-2,2.9879995E-2,5.3774714E-2,8.09523E-2,2.413808E-2,2.5131643E-2,2.4154678E-2,0E0,5.8579735E-2,2.4344124E-2,3.5270933E-2,0E0,0E0,0E0,4.731854E-2,0E0,2.1778351E-1,6.853256E-2,0E0,0E0,0E0,0E0,0E0,1.600309E-2,5.523633E-2,2.7157843E-2,0E0,0E0,3.1845495E-2,8.499063E-2,6.915836E-2,2.5944151E-2,0E0,0E0,2.8384332E-2,5.8425784E-2,0E0,0E0,3.7334062E-2,5.1050145E-2,2.312562E-2,0E0,2.9853227E-2,1.7818898E-2,5.0899398E-2,2.5549069E-2,4.434356E-2,3.484807E-2,0E0,0E0,0E0,2.2762176E-2,0E0,3.24804E-2,3.826867E-2,2.8596187E-2,0E0,4.7541924E-2,4.5454074E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,20,20,22,22,23,23,24,24,26,26,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,47,47,49,49,50,50,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,84,84,86,86,87,87,88,88,90,90,91,91],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,36,-1,38,-1,-1,40,-1,42,-1,44,46,48,50,52,54,56,-1,58,60,62,-1,-1,-1,64,-1,66,68,-1,-1,-1,-1,-1,70,72,74,-1,-1,76,78,80,82,-1,-1,84,86,-1,-1,88,90,92,-1,94,96,98,100,102,104,-1,-1,-1,106,-1,108,110,112,-1,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.194E3,6.27957E0,1.2E1,1.559287E5,9E0,4.5295143E-3,1.8439855E5,1.0700264E6,4E0,7.616974E-4,6.6E1,1.3631483E8,6.923077E-1,5.02E2,2.3238889E2,5.4674416E7,-9.626782E-4,-1.3356135E-2,-3.9830017E-3,-2.8058556E-3,5.16175E5,4.101179E-3,5.619497E7,3.578882E1,4.064E3,-5.8101793E-3,8.39E2,1.7421166E-4,4.7487365E-3,4.7220547E1,-2.5261184E-3,4.435876E2,-3.4079968E-3,2.3364584E1,3.3817584E7,3.653E3,5.1942E4,1.7171982E6,3.8192307E2,1.896E3,-2.4684125E-3,4.9321495E6,1.213274E9,3.89E2,-2.817911E-3,-6.939745E-3,-1.2055098E-3,1.683263E7,3.9208382E-3,1.7431098E8,1.925508E6,-1.238414E-3,2.4044833E-3,-4.386652E-3,-6.0855004E-4,-5.4762646E-4,2.0734E4,2.3E1,1.3E1,7.2312662E-3,2.2500358E-3,1.9E1,1.0443479E5,1E0,1E0,-6.128115E-4,-1.1181261E-2,6.15E2,2.331083E6,3.636293E-3,2.5069484E-4,3.7568388E0,1.592E3,1.3784861E5,3.5036304E-3,1.00037E5,7.336111E4,2.952381E0,3.2E1,1E0,1E0,-8.099305E-4,-5.3122505E-3,-4.056157E-3,7.89E2,5.952784E-3,2.2715E4,1.1772152E0,1.0743623E0,3.207527E-3,5.7420593E2,6.203939E2,-1.4170104E-6,-1.921377E-3,1.0375849E-3,-6.2345397E-3,-2.0938758E-3,2.3278622E-3,-2.1058281E-3,-1.5122887E-4,-1.7460182E-3,1.591008E-3,-2.065975E-3,-1.4087139E-3,5.7482865E-4,1.2719618E-3,-1.5271936E-3,-1.1451181E-3,2.4683645E-3,-6.922327E-5,1.1849086E-3,3.1147618E-4,-2.618261E-3,3.176464E-4,-3.248578E-3,3.0440209E-4,5.0093806E-3],"split_indices":[2,68,33,48,0,0,43,66,8,0,10,7,68,2,67,7,0,0,0,0,1,0,62,61,44,0,2,0,0,73,0,71,0,73,7,44,1,60,67,10,0,43,7,1,0,0,0,60,0,5,43,0,0,0,0,0,44,3,3,0,0,67,43,26,8,0,0,2,43,0,0,68,44,48,0,5,60,68,10,23,27,0,0,0,2,0,44,68,53,0,67,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,5.3E2,4.9E2,5.16E2,1.4E1,1.7E1,4.73E2,4.95E2,2.1E1,6E0,8E0,5.7E1,4.16E2,4.57E2,3.8E1,1.2E1,9E0,4E0,4E0,1.9E1,3.8E1,1.5E1,4.01E2,1.96E2,2.61E2,6E0,3.2E1,5E0,7E0,2.8E1,1E1,3.95E2,6E0,1.88E2,8E0,1.75E2,8.6E1,1.5E1,1.7E1,2.3E1,5E0,3.85E2,1E1,1.74E2,1.4E1,4E0,4E0,1.68E2,7E0,1.9E1,6.7E1,1E1,5E0,1.1E1,6E0,1.1E1,1.2E1,3.33E2,5.2E1,4E0,6E0,4.8E1,1.26E2,1.57E2,1.1E1,1.4E1,5E0,4.3E1,2.4E1,7E0,5E0,2.97E2,3.6E1,3.3E1,1.9E1,3.7E1,1.1E1,3.4E1,9.2E1,7.7E1,8E1,7E0,4E0,7E0,3.6E1,4E0,2E1,2.69E2,2.8E1,4E0,3.2E1,1.7E1,1.6E1,2.8E1,9E0,4E0,7E0,2.8E1,6E0,3.4E1,5.8E1,7E1,7E0,5.1E1,2.9E1,7E0,2.9E1,1.2E1,8E0,7E1,1.99E2,1.7E1,1.1E1,1.3E1,1.9E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"108","num_nodes":"117","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"108","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_ObjectHashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_onprem","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"83"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-4.5668036E-3,-2.3331916E-1,6.575819E-1,-7.3830986E-1,-2.9545367E-2,7.585778E-1,-9.047681E-1,-9.8671955E-1,-5.689058E-1,-2.1748337E-1,1.7790031E-1,4.7353148E-1,9.98321E-1,-5.0334964E-2,-1.9184373E-2,-1.009283E0,-1.9581432E-2,-7.572694E-1,-2.6597184E-1,-3.4746066E-1,2.4795938E-2,3.172561E-1,1.9483695E-2,3.2106972E-1,8.287864E-1,8.6895436E-2,8.755892E-1,-4.7518518E-2,-2.757553E-2,-6.920537E-1,-1.1876076E0,3.4099087E-2,-4.1624886E-1,-5.5970305E-1,-2.3217718E-1,-1.1214657E-2,3.8214678E-1,2.0068753E-1,4.5767924E-1,7.318743E-2,-3.5774136E-1,-6.0188014E-2,4.028545E-1,8.6816084E-1,2.0080749E-2,5.641912E-1,1.1029484E0,-3.3124804E-2,-1.3965118E-2,-6.606181E-2,-3.2697614E-2,-7.3586625E-1,-2.912001E-1,-5.0898236E-1,-3.7304517E-2,-1.3420476E-1,-4.1529045E-1,-2.2086361E-1,2.3870613E-2,2.2889538E-2,8.889135E-3,3.0159533E-1,-9.188867E-2,5.2135813E-1,-9.34718E-3,1.5662609E-2,2.2340204E-1,-2.1273324E-1,-2.7960172E-2,-1.0369004E-2,4.438314E-2,2.2642371E-1,5.3501856E-1,2.715325E-2,4.549749E-2,6.6693574E-1,1.7469372E-1,1.2068436E0,6.308485E-1,-3.998909E-2,-2.1338705E-2,-4.135335E-1,-1.0867557E-1,-5.4981214E-1,-1.2486093E-2,-1.0617143E-2,-2.5443727E-1,-8.711125E-2,-5.048686E-1,8.628427E-3,-1.9532233E-2,-1.7093034E-1,5.5031493E-2,6.1803424E-1,2.2785096E-1,1.0172568E-1,-1.824982E-2,4.6515972E-3,5.708142E-1,-8.626919E-2,7.5669944E-2,1.1983855E-1,3.8197517E-1,-1.3491115E-2,-2.7342E-3,-1.4498445E-3,5.8407313E-3,3.234492E-1,-9.9598905E-3,6.387942E-1,2.3165841E-1,7.3940027E-1,1.0149398E-2,-7.78564E-3,1.6972052E-2,1.2719901E0,2.2384929E-2,3.457816E-2,1.2448564E-2,-1.0324918E-2,-4.790183E-1,-8.777231E-3,-1.1875937E-2,-5.716248E-1,-1.1855732E-2,-4.4726193E-2,1.0953399E-2,-3.516006E-1,-8.671726E-2,3.5364935E-4,-8.473294E-3,-5.602632E-1,-4.3281703E-3,-1.1312302E-2,-1.5357796E-3,1.956537E-1,2.4461351E-2,3.5201266E-2,1.8253269E-2,7.594841E-2,2.8369758E-1,-2.4591943E-2,1.4264061E-2,4.69715E-1,3.320946E-2,8.841761E-3,-1.5208796E-1,-9.6291214E-2,1.2223156E-1,1.9617124E-1,1.3227642E-2,1.0226706E-2,2.4505768E-2,2.6079942E-2,2.1761277E-1,6.965718E-1,1.1330703E-2,4.0160073E-3,1.3949478E-2,3.624573E-2,1.4810829E-2,6.3704684E-2,4.4903275E-2,-2.6027502E-2,-9.909527E-3,-3.1908199E-3,1.9831837E-3,-1.4045271E-2,-5.9957147E-1,4.5774165E-3,-8.285467E-2,-5.9024906E-3,-1.7932279E-2,-9.371145E-3,3.2423376E-3,-6.361793E-1,-1.116317E-2,5.031539E-3,1.2207902E-2,-1.4691992E-1,6.4051405E-2,-1.1300913E-3,5.477288E-3,2.2185285E-1,1.9904312E-2,3.4397193E-3,-5.4847435E-3,2.4269028E-2,1.0960829E-2,-6.01094E-2,-1.3799966E-2,-9.214199E-3,3.1130412E-3,1.6172273E-1,-5.6327586E-3,1.2809288E-3,1.3326791E-2,-3.5169765E-3,4.6169646E-3,1.0040688E-1,1.6919017E-2,1.1915705E-2,3.5820182E-2,-2.899007E-2,-1.3642825E-2,-8.5165E-3,-4.8935594E-4,-3.4403258E-3,3.116593E-3,-2.0048281E-2,-3.329749E-2,-4.7226588E-4,-1.1971596E-2,-9.0992794E-4,5.181902E-3,4.1945907E-3,1.27073815E-2,-5.0962237E-3,3.344545E-3,1.1613483E-2,3.99133E-3,8.124161E-3,-1.8657715E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,35,37,39,41,43,-1,45,-1,-1,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,-1,-1,-1,-1,79,81,83,-1,85,87,89,91,-1,-1,93,95,97,-1,99,101,103,-1,-1,105,107,109,-1,-1,111,113,115,117,-1,-1,119,121,123,-1,125,127,129,131,-1,-1,133,135,137,139,141,-1,-1,143,145,147,149,151,-1,-1,-1,-1,153,-1,155,157,159,-1,-1,-1,161,-1,-1,-1,-1,163,165,-1,167,-1,169,-1,171,173,-1,-1,175,-1,-1,-1,177,179,-1,-1,181,183,185,-1,187,-1,-1,189,191,193,195,197,-1,-1,-1,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,203,-1,205,-1,-1,-1,207,209,-1,-1,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,217,-1,-1,-1,219,-1,-1,-1,-1,-1,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5101564E2,7.622995E1,4.108082E1,8.402031E0,2.0662775E1,1.5996719E1,9.946165E-1,6.51741E-1,7.1595535E0,8.763061E0,5.5570397E0,5.8001347E0,1.2434647E1,0E0,0E0,3.0021667E-2,0E0,1.3952293E0,8.002494E0,4.298765E0,1.2653905E0,2.1015854E0,2.4369311E0,2.518735E0,1.3476181E-1,0E0,7.590439E0,0E0,0E0,4.7367477E-1,3.4502983E-2,0E0,1.5842652E0,3.807373E-1,2.0791326E0,6.6727525E-1,9.815848E-2,2.2419546E0,2.627181E0,9.038132E-1,4.0952802E-1,2.6760477E-1,1.4178333E0,3.6981392E-1,0E0,1.9244442E0,2.475891E0,0E0,0E0,0E0,0E0,2.6675224E-2,7.1423984E-1,3.8747025E-1,0E0,1.1711794E0,1.1613255E0,1.1489779E0,4.8287967E-1,0E0,0E0,1.1659727E0,1.2140188E0,1.0867891E0,0E0,4.775105E-1,4.304738E-1,1.1663926E-1,0E0,0E0,6.6127405E-2,1.356319E0,1.0168295E0,0E0,0E0,1.1339455E0,8.554605E-1,1.8776474E0,3.9338207E-1,0E0,0E0,1.4184737E-1,2.2009441E-1,1.4748955E-1,0E0,3.4856838E-1,6.2135935E-1,9.266107E-2,6.763859E-1,0E0,0E0,1.0383153E-1,2.8753528E-1,1.9785881E-2,3.7716627E-1,3.4755725E-1,0E0,0E0,3.9562225E-1,5.5907613E-1,4.0410244E-1,1.520252E-1,1.4487875E-1,0E0,0E0,0E0,0E0,5.14987E-1,0E0,4.4307423E-1,8.564776E-2,3.914795E-1,0E0,0E0,0E0,1.9971466E-1,0E0,0E0,0E0,0E0,1.9156551E-1,3.413704E-2,0E0,1.3182831E-1,0E0,2.0560172E-1,0E0,1.5495229E-1,1.6780338E-1,0E0,0E0,5.235777E-1,0E0,0E0,0E0,4.2620093E-2,3.9615512E-1,0E0,0E0,6.27986E-2,2.1381283E-1,9.2746355E-2,0E0,2.889619E-1,0E0,0E0,3.0570364E-1,1.9916046E-1,3.95755E-1,1.6301173E-1,7.733794E-2,0E0,0E0,0E0,2.9416418E-1,5.9475803E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2915039E-1,0E0,2.1072836E-1,0E0,0E0,0E0,5.492708E-2,6.3435555E-2,0E0,0E0,0E0,1.6607939E-1,1.915888E-1,0E0,0E0,1.56322E-1,0E0,0E0,0E0,0E0,0E0,1.1441429E-1,0E0,0E0,0E0,2.0790786E-1,0E0,0E0,0E0,0E0,0E0,1.3456574E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,51,51,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,63,63,65,65,66,66,67,67,70,70,71,71,72,72,75,75,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86,87,87,88,88,91,91,92,92,93,93,94,94,95,95,98,98,99,99,100,100,101,101,102,102,107,107,109,109,110,110,111,111,115,115,120,120,121,121,123,123,125,125,127,127,128,128,131,131,135,135,136,136,139,139,140,140,141,141,143,143,146,146,147,147,148,148,149,149,150,150,154,154,155,155,168,168,170,170,174,174,175,175,179,179,180,180,183,183,189,189,193,193,199,199],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,36,38,40,42,44,-1,46,-1,-1,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,-1,-1,-1,-1,80,82,84,-1,86,88,90,92,-1,-1,94,96,98,-1,100,102,104,-1,-1,106,108,110,-1,-1,112,114,116,118,-1,-1,120,122,124,-1,126,128,130,132,-1,-1,134,136,138,140,142,-1,-1,144,146,148,150,152,-1,-1,-1,-1,154,-1,156,158,160,-1,-1,-1,162,-1,-1,-1,-1,164,166,-1,168,-1,170,-1,172,174,-1,-1,176,-1,-1,-1,178,180,-1,-1,182,184,186,-1,188,-1,-1,190,192,194,196,198,-1,-1,-1,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,204,-1,206,-1,-1,-1,208,210,-1,-1,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,218,-1,-1,-1,220,-1,-1,-1,-1,-1,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.9661028E7,9.9E1,9.31E2,1.1598715E3,1.665909E3,1.4912975E-1,6.76E2,1.5859042E6,3.875E0,2.9251662E6,1.059448E7,-5.0334964E-2,-1.9184373E-2,8.875E0,-1.9581432E-2,4.6463413E0,2.74614E5,4.48E2,7.4711205E6,4.265829E2,9.932432E0,1.4828649E9,2.6662E4,8.6895436E-2,1.7587205E6,-4.7518518E-2,-2.757553E-2,7.997723E6,6.8E1,3.4099087E-2,9.876542E-2,2E0,2.835E3,5.8899284E1,3.52E2,5.9192824E0,4.455844E1,1.4603988E6,8.4797815E2,2.6463525E8,4.087E3,1.2003246E6,2.0080749E-2,2.3876712E7,2.1257812E7,-3.3124804E-2,-1.3965118E-2,-6.606181E-2,-3.2697614E-2,4.9094446E2,2.026453E6,2.0145264E2,-3.7304517E-2,1E0,1.5714285E0,3.5714287E-1,3.4187552E6,2.2889538E-2,8.889135E-3,1.407037E2,2.1347518E0,1.089E3,-9.34718E-3,5E0,1.2585366E1,1.079E3,-2.7960172E-2,-1.0369004E-2,7.7E1,1.0494994E7,5.5454545E0,2.715325E-2,4.549749E-2,4.233978E0,2.9925186E-2,1E0,1.8545505E2,-3.998909E-2,-2.1338705E-2,6.782E4,3.401282E1,6.920598E7,-1.2486093E-2,1.3366154E6,2.118835E6,2.33E2,1.559733E6,8.628427E-3,-1.9532233E-2,1.9751773E0,3.355864E0,1.8992E4,1E0,3.9892856E2,-1.824982E-2,4.6515972E-3,6.965855E9,9.1E1,6.228074E5,7.7E1,2.239E3,-1.3491115E-2,-2.7342E-3,-1.4498445E-3,5.8407313E-3,1.3902439E0,-9.9598905E-3,6.797565E1,2.427869E8,1.4603988E6,1.0149398E-2,-7.78564E-3,1.6972052E-2,1.6063418E7,2.2384929E-2,3.457816E-2,1.2448564E-2,-1.0324918E-2,6.9740294E5,2.9711267E2,-1.1875937E-2,1.9E1,-1.1855732E-2,3.53E2,1.0953399E-2,1.3529412E0,1E0,3.5364935E-4,-8.473294E-3,1.2739792E1,-4.3281703E-3,-1.1312302E-2,-1.5357796E-3,9.66E2,7.3809524E0,3.5201266E-2,1.8253269E-2,1.8181819E0,8.684966E-1,1E0,1.4264061E-2,1.1394967E7,3.320946E-2,8.841761E-3,2.5E0,3.1095755E0,1.03339244E6,5.80418E7,7.6825094E0,1.0226706E-2,2.4505768E-2,2.6079942E-2,1.9505818E8,3.9149782E6,1.1330703E-2,4.0160073E-3,1.3949478E-2,3.624573E-2,1.4810829E-2,6.3704684E-2,4.4903275E-2,-2.6027502E-2,-9.909527E-3,-3.1908199E-3,1.9831837E-3,-1.4045271E-2,4.03E2,4.5774165E-3,4.0481758E3,-5.9024906E-3,-1.7932279E-2,-9.371145E-3,2.0671213E2,8E0,-1.116317E-2,5.031539E-3,1.2207902E-2,5.4E0,2.9547384E9,-1.1300913E-3,5.477288E-3,1.5484655E6,1.9904312E-2,3.4397193E-3,-5.4847435E-3,2.4269028E-2,1.0960829E-2,9.557399E6,-1.3799966E-2,-9.214199E-3,3.1130412E-3,1.48E1,-5.6327586E-3,1.2809288E-3,1.3326791E-2,-3.5169765E-3,4.6169646E-3,1.4172E4,1.6919017E-2,1.1915705E-2,3.5820182E-2,-2.899007E-2,-1.3642825E-2,-8.5165E-3,-4.8935594E-4,-3.4403258E-3,3.116593E-3,-2.0048281E-2,-3.329749E-2,-4.7226588E-4,-1.1971596E-2,-9.0992794E-4,5.181902E-3,4.1945907E-3,1.27073815E-2,-5.0962237E-3,3.344545E-3,1.1613483E-2,3.99133E-3,8.124161E-3,-1.8657715E-3],"split_indices":[2,43,60,44,2,67,4,53,2,43,69,43,12,0,0,68,0,68,5,2,43,67,69,46,10,0,43,0,0,9,0,0,53,6,44,62,0,71,73,43,4,7,2,66,0,60,66,0,0,0,0,4,9,70,0,26,69,68,60,0,0,67,69,2,0,69,71,10,0,0,0,62,69,0,0,68,72,6,73,0,0,9,73,7,0,62,9,0,9,0,0,71,73,9,8,67,0,0,46,10,60,0,2,0,0,0,0,68,0,73,7,43,0,0,0,62,0,0,0,0,62,4,0,67,0,44,0,68,27,0,0,71,0,0,0,12,73,0,0,71,57,6,0,60,0,0,68,53,43,7,71,0,0,0,5,60,0,0,0,0,0,0,0,0,0,0,0,0,2,0,48,0,0,0,4,8,0,0,0,73,46,0,0,43,0,0,0,0,0,60,0,0,0,73,0,0,0,0,0,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.95E2,7.4E2,2.55E2,2.12E2,5.28E2,2.4E2,1.5E1,8.4E1,1.28E2,2.77E2,2.51E2,1.11E2,1.29E2,1E1,5E0,8E1,4E0,7.8E1,5E1,1.8E2,9.7E1,1.33E2,1.18E2,7.9E1,3.2E1,1.4E1,1.15E2,7.5E1,5E0,7E1,8E0,6E0,4.4E1,6.2E1,1.18E2,8.9E1,8E0,7.4E1,5.9E1,1.04E2,1.4E1,1.4E1,6.5E1,2.8E1,4E0,5E1,6.5E1,6.5E1,5E0,4E0,4E0,1.1E1,3.3E1,5.4E1,8E0,7.8E1,4E1,1.2E1,7.7E1,4E0,4E0,5.5E1,1.9E1,5.4E1,5E0,7.6E1,2.8E1,1E1,4E0,5E0,9E0,2.9E1,3.6E1,1E1,1.8E1,3.9E1,1.1E1,5.2E1,1.3E1,6E0,5E0,1.9E1,1.4E1,4.5E1,9E0,3.9E1,3.9E1,9E0,3.1E1,4E0,8E0,1E1,6.7E1,9E0,4.6E1,1.2E1,7E0,6E0,4.8E1,2.8E1,4.8E1,1.8E1,1E1,6E0,4E0,5E0,4E0,2.4E1,5E0,2.6E1,1E1,3.3E1,6E0,4E0,7E0,4.7E1,5E0,9E0,4E0,6E0,1.3E1,9E0,5E0,4.1E1,4E0,3.5E1,4E0,2.4E1,1.5E1,5E0,4E0,2.7E1,4E0,6E0,4E0,1.1E1,5.6E1,4E0,5E0,1.3E1,3.3E1,8E0,4E0,3.1E1,1.7E1,5E0,2.3E1,1E1,3.8E1,1E1,8E0,6E0,4E0,6E0,1.8E1,2.2E1,4E0,4E0,6E0,2.9E1,4E0,3.2E1,1.5E1,9E0,4E0,4E0,5E0,5E0,3.6E1,7E0,2.8E1,4E0,2E1,6E0,9E0,2.1E1,6E0,6E0,5E0,1E1,4.6E1,4E0,9E0,2.5E1,8E0,4E0,4E0,2.4E1,7E0,1.5E1,8E0,6E0,4E0,3.3E1,5E0,4E0,6E0,4E0,4E0,1.1E1,7E0,4E0,1.8E1,3.2E1,4E0,1.1E1,1.7E1,4E0,5E0,8E0,1.3E1,5E0,5E0,1.7E1,2.9E1,8E0,1.7E1,1.1E1,4E0,1.4E1,1.9E1,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[1.3471586E-2,-2.2610085E-1,7.3868704E-1,-6.5098697E-1,-3.5431497E-2,8.4848815E-1,-1.5004842E-1,-8.9800495E-1,-4.8295447E-1,-2.2586998E-1,1.7531002E-1,4.6864507E-1,1.1211873E0,-8.2168055E-1,2.4668853E-1,-4.416024E-2,-6.1319447E-1,1.7007243E-2,-5.28793E-1,-2.930738E-1,5.057889E-2,3.0750713E-1,3.9650675E-2,1.7390183E-1,6.044502E-1,1.7208714E0,9.987359E-1,-4.5871697E-2,-1.9293519E-2,3.8010448E-1,-9.373652E-3,-3.344311E-2,-1.8930763E-2,-5.636617E-1,9.437213E-3,-2.7136397E-1,-5.556443E-2,-1.9204797E-2,3.378035E-1,1.8561703E-1,4.7336155E-1,7.310219E-2,-4.070584E-1,1.6812421E-2,4.7227135E-1,7.144502E-1,4.9912896E-2,5.6426976E-2,9.535076E-2,6.8532443E-1,1.126565E0,7.007599E-3,4.493776E-1,-6.2934786E-1,-2.4185112E-1,-5.183244E-1,-1.9883876E-1,6.202525E-2,-3.1783548E-1,9.521465E-3,2.1003593E-2,1.1989366E-1,2.4307655E-2,5.542791E-1,-6.889406E-3,-3.158213E-2,1.3950029E-1,-4.08395E-3,-2.976633E-2,-8.820557E-3,1.066468E-1,2.6161164E-2,1.1375045E-2,3.7389975E-2,3.7879828E-1,-5.8667203E-3,1.4920981E-2,4.2009376E-2,5.289257E-1,2.4025898E-2,1.1790373E0,1.3040129E-2,2.7417619E-2,-6.601182E-1,-3.2561103E-1,-4.49822E-3,-1.6475413E-2,-4.1487828E-1,-2.7016576E-2,-1.3602722E-1,-3.4765822E-1,-1.6290968E-2,1.7797509E-1,-2.3467105E-2,-4.9169087E-3,-2.3018967E-2,2.1575898E-1,4.4199905E-1,3.565774E-2,6.239491E-2,-2.3622614E-1,-1.15486704E-1,1.9155489E-1,1.4931753E-1,-7.8964967E-4,7.838179E-3,2.1996956E-2,9.713744E-3,6.057634E-1,1.2241651E0,2.7329534E-2,-3.5721904E-1,-6.9430983E-1,-2.3551518E-3,-2.376916E-2,-2.0790437E-2,-1.1102127E-2,-9.744354E-2,-3.903768E-1,-5.040833E-1,-2.9203922E-1,-8.791621E-2,1.0489454E-1,1.1815381E-2,3.6013946E-3,9.236729E-2,-1.690374E-2,2.5943175E-1,-9.6785446E-4,8.035426E-3,4.7194386E-1,1.3761136E-1,-8.0701485E-2,-1.3299356E-1,-1.7043263E-2,-1.3238124E-2,-7.622262E-3,2.6666063E-1,6.465602E-2,2.151845E-3,1.1286171E-2,1.5127124E-2,3.3154383E-2,1.3348699E0,4.6511665E-2,-2.0508487E-2,-6.794053E-3,-7.2029847E-1,-1.606041E-2,-4.970796E-2,-1.6955613E-1,-5.098529E-1,-4.9097845E-3,-2.9359356E-2,-1.4757413E-2,-1.8703546E-2,-1.9996542E-1,2.4301666E-3,-1.4379577E-1,8.326625E-3,3.9622452E-4,1.8074419E-1,-5.091463E-3,2.1009976E-2,2.0830798E-1,1.1900312E-2,4.9501446E-1,2.2191374E-1,-1.7397186E-3,1.122717E-3,-8.7665E-3,-1.00850025E-2,-1.8634964E-4,1.6903182E-3,-2.3241725E-3,3.962152E-1,1.8344365E-1,1.5655515E-1,-4.0289376E-2,3.6649484E-2,6.500713E-2,-3.3923816E-2,-1.964705E-2,-6.456711E-4,-1.4390323E-2,-3.4553998E-3,-1.2255195E-2,-1.00895045E-2,-2.9807769E-2,-9.6507213E-4,-1.0844586E-2,-8.552617E-3,-2.641438E-3,4.1510696E-3,1.0688406E-2,4.239972E-3,1.0899175E-2,1.2333621E-2,2.4224138E-2,1.2011358E-2,4.3316246E-3,1.0150402E-2,2.1707145E-2,1.0621646E-2,-2.236304E-3,1.9091156E-3,9.386613E-3,2.0133564E-3,-7.6885647E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,-1,-1,53,-1,55,-1,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,-1,81,83,85,87,89,91,93,-1,-1,95,-1,97,-1,99,101,-1,-1,-1,103,-1,-1,-1,105,-1,-1,-1,107,-1,109,-1,-1,111,113,-1,-1,115,-1,117,119,121,123,-1,-1,125,127,129,-1,131,133,135,137,139,-1,-1,-1,-1,141,143,-1,145,147,-1,-1,-1,-1,149,151,153,155,157,159,-1,-1,161,-1,163,-1,-1,165,167,169,171,-1,-1,173,175,177,-1,-1,-1,-1,179,-1,-1,-1,181,-1,183,185,187,-1,-1,-1,-1,189,-1,191,-1,-1,193,-1,-1,195,-1,197,199,-1,-1,-1,-1,-1,-1,-1,201,203,205,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.791209E2,6.2761826E1,2.510884E1,9.528E0,2.1551044E1,2.3091461E1,7.930076E0,9.959717E-1,5.851471E0,5.2539377E0,4.560919E0,3.7775688E0,7.97435E0,3.4642363E-1,1.216778E0,0E0,1.3123655E-1,0E0,3.6425667E0,4.156475E0,1.1289285E0,2.5200014E0,1.9275302E0,1.491922E0,3.9902802E0,1.0773582E0,3.7149963E0,0E0,0E0,1.6918921E-1,0E0,0E0,0E0,2.6380768E0,0E0,3.891058E0,0E0,1.135572E0,7.556093E-2,1.6378746E0,2.802637E0,8.315556E-1,6.234549E-1,4.3092486E-1,9.4311E-2,1.4448681E0,6.206651E-1,0E0,0E0,7.45142E-1,1.8692017E0,0E0,9.7367525E-2,8.037758E-1,3.63248E-1,1.01145744E-1,1.5791879E0,3.3888727E-1,3.4786725E-1,0E0,0E0,8.9549017E-1,0E0,8.3132267E-1,0E0,9.1863567E-1,9.9106896E-1,0E0,0E0,0E0,8.703281E-2,0E0,0E0,0E0,2.0421124E-1,0E0,0E0,0E0,4.3580246E-1,0E0,1.1581421E0,0E0,0E0,7.669296E-1,5.9625506E-1,0E0,0E0,3.294611E-2,0E0,1.1788642E0,2.9649878E-1,2.0886058E-1,9.674963E-2,0E0,0E0,1.1019844E0,4.086063E-1,1.9084072E-1,0E0,3.662911E-1,1.5639597E-1,2.3768076E-1,5.641396E-1,1.0571629E-1,0E0,0E0,0E0,0E0,3.255248E-1,2.2045898E-1,0E0,1.5335834E-1,5.334358E-1,0E0,0E0,0E0,0E0,3.6678624E-1,4.8891044E-1,8.81927E-2,3.345101E-1,1.2534547E-1,6.376294E-2,0E0,0E0,3.958801E-1,0E0,2.3482108E-1,0E0,0E0,2.0850182E-2,3.3341855E-1,1.42023E-1,1.0912365E-1,0E0,0E0,1.8824793E-2,3.3700037E-1,2.3777288E-1,0E0,0E0,0E0,0E0,4.4866943E-1,0E0,0E0,0E0,1.8115997E-2,0E0,6.22355E-1,3.7017024E-1,2.930348E-1,0E0,0E0,0E0,0E0,1.4453489E-1,0E0,2.8795391E-2,0E0,0E0,4.821509E-2,0E0,0E0,6.843603E-2,0E0,3.391266E-2,4.8820138E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.8751326E-2,2.804619E-1,6.144482E-2,1.3419731E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,63,63,65,65,66,66,70,70,74,74,78,78,80,80,83,83,84,84,87,87,89,89,90,90,91,91,92,92,95,95,96,96,97,97,99,99,100,100,101,101,102,102,103,103,108,108,109,109,111,111,112,112,117,117,118,118,119,119,120,120,121,121,122,122,125,125,127,127,130,130,131,131,132,132,133,133,136,136,137,137,138,138,143,143,147,147,149,149,150,150,151,151,156,156,158,158,161,161,164,164,166,166,167,167,175,175,176,176,177,177,178,178],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,-1,-1,54,-1,56,-1,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,-1,82,84,86,88,90,92,94,-1,-1,96,-1,98,-1,100,102,-1,-1,-1,104,-1,-1,-1,106,-1,-1,-1,108,-1,110,-1,-1,112,114,-1,-1,116,-1,118,120,122,124,-1,-1,126,128,130,-1,132,134,136,138,140,-1,-1,-1,-1,142,144,-1,146,148,-1,-1,-1,-1,150,152,154,156,158,160,-1,-1,162,-1,164,-1,-1,166,168,170,172,-1,-1,174,176,178,-1,-1,-1,-1,180,-1,-1,-1,182,-1,184,186,188,-1,-1,-1,-1,190,-1,192,-1,-1,194,-1,-1,196,-1,198,200,-1,-1,-1,-1,-1,-1,-1,202,204,206,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.856934E7,1.16E2,9.31E2,1.1459359E3,1.3655363E2,7.6588124E-2,6E0,2.331083E6,3.6666667E0,5.486433E5,1.059448E7,1.665909E3,3.0070068E7,-4.416024E-2,2.0081382E-7,1.7007243E-2,8.909296E4,1.1290322E1,6.9307615E6,4.265829E2,1.2294118E1,4.032683E6,1E0,2.35184E5,1.7587205E6,-4.5871697E-2,-1.9293519E-2,1.4595416E3,-9.373652E-3,-3.344311E-2,-1.8930763E-2,7.997723E6,9.437213E-3,3.89E2,-5.556443E-2,8E0,1.1311654E2,1.9090909E0,2.48334E5,1.47E3,1.4787234E1,1.5204346E8,1.8333334E0,2E0,1.038946E6,5.6426976E-2,9.535076E-2,2.5817454E8,1E0,7.007599E-3,7.802E-2,3.3238492E2,1.8E0,3.1E1,2.2970297E0,1.821E3,1.5933333E2,9.521465E-3,2.1003593E-2,1.212945E6,2.4307655E-2,1.2774637E10,-6.889406E-3,4.911E3,3.2892792E2,-4.08395E-3,-2.976633E-2,-8.820557E-3,2.8E1,2.6161164E-2,1.1375045E-2,3.7389975E-2,9.0505896E2,-5.8667203E-3,1.4920981E-2,4.2009376E-2,6.319223E8,2.4025898E-2,1E0,1.3040129E-2,2.7417619E-2,9.935484E-1,1.3209776E6,-4.49822E-3,-1.6475413E-2,5.4E0,-2.7016576E-2,8.931E3,1E0,5.95E2,7.597862E6,-2.3467105E-2,-4.9169087E-3,4.68418E5,3.2857144E0,1.124E3,3.565774E-2,5.44E2,1.5E1,2.686063E6,3.093458E0,1.409E3,-7.8964967E-4,7.838179E-3,2.1996956E-2,9.713744E-3,4.3948618E8,1.7103828E7,2.7329534E-2,6.848509E7,5.0953402E4,-2.3551518E-3,-2.376916E-2,-2.0790437E-2,-1.1102127E-2,1E0,1.6E1,2.57E1,6.47E2,3.3817584E7,1.4285715E0,1.1815381E-2,3.6013946E-3,1.8E1,-1.690374E-2,1.2825651E0,-9.6785446E-4,8.035426E-3,4.797342E2,2.6647997E9,1.0674361E8,1E0,-1.7043263E-2,-1.3238124E-2,2.8655008E2,2.11429E5,1.5222016E8,2.151845E-3,1.1286171E-2,1.5127124E-2,3.3154383E-2,4.454E3,4.6511665E-2,-2.0508487E-2,-6.794053E-3,1.3091388E0,-1.606041E-2,1.1203416E1,1.6153846E0,9.2789966E-1,-4.9097845E-3,-2.9359356E-2,-1.4757413E-2,-1.8703546E-2,3.449123E0,2.4301666E-3,2.3112903E2,8.326625E-3,3.9622452E-4,4.745397E6,-5.091463E-3,2.1009976E-2,6.479833E5,1.1900312E-2,5.1604336E7,1.4839433E5,-1.7397186E-3,1.122717E-3,-8.7665E-3,-1.00850025E-2,-1.8634964E-4,1.6903182E-3,-2.3241725E-3,5.185192E2,1.26E2,6E0,4.9E1,3.6649484E-2,6.500713E-2,-3.3923816E-2,-1.964705E-2,-6.456711E-4,-1.4390323E-2,-3.4553998E-3,-1.2255195E-2,-1.00895045E-2,-2.9807769E-2,-9.6507213E-4,-1.0844586E-2,-8.552617E-3,-2.641438E-3,4.1510696E-3,1.0688406E-2,4.239972E-3,1.0899175E-2,1.2333621E-2,2.4224138E-2,1.2011358E-2,4.3316246E-3,1.0150402E-2,2.1707145E-2,1.0621646E-2,-2.236304E-3,1.9091156E-3,9.386613E-3,2.0133564E-3,-7.6885647E-3],"split_indices":[2,43,60,44,2,67,71,57,3,43,69,43,12,4,58,0,52,0,48,69,43,67,68,60,6,1,43,0,0,4,0,0,0,9,0,2,0,8,4,68,1,2,71,7,68,32,9,0,0,12,27,0,72,67,68,10,68,44,4,0,0,9,0,46,0,44,4,0,0,0,71,0,0,0,4,0,0,0,12,0,6,0,0,68,60,0,0,73,0,44,29,2,60,0,0,9,69,2,0,10,3,12,68,10,0,0,0,0,7,62,0,7,48,0,0,0,0,26,3,61,2,7,68,0,0,3,0,69,0,0,4,46,7,26,0,0,4,9,5,0,0,0,0,2,0,0,0,53,0,71,68,68,0,0,0,0,73,0,67,0,0,60,0,0,43,0,7,48,0,0,0,0,0,0,0,67,8,8,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.74E2,2.55E2,2.39E2,5.35E2,2.27E2,2.8E1,9.5E1,1.44E2,2.81E2,2.54E2,9.6E1,1.31E2,1E1,1.8E1,7.7E1,1.8E1,7E0,1.37E2,2.26E2,5.5E1,1.28E2,1.26E2,3.1E1,6.5E1,2E1,1.11E2,6E0,4E0,1.4E1,4E0,1E1,8E0,1.31E2,6E0,2.22E2,4E0,4.5E1,1E1,7.5E1,5.3E1,1.18E2,8E0,2.1E1,1E1,5.4E1,1.1E1,1E1,1E1,3.4E1,7.7E1,4E0,1E1,1.08E2,2.3E1,4.9E1,1.73E2,3.6E1,9E0,6E0,4E0,6.4E1,1.1E1,4.7E1,6E0,4.6E1,7.2E1,4E0,4E0,6E0,1.5E1,6E0,4E0,4.1E1,1.3E1,7E0,4E0,1.2E1,2.2E1,7E0,7E1,6E0,4E0,9.7E1,1.1E1,1.1E1,1.2E1,2.2E1,2.7E1,1.23E2,5E1,2.2E1,1.4E1,4E0,5E0,2.6E1,3.8E1,3.3E1,1.4E1,3.2E1,1.4E1,1.2E1,6E1,1.1E1,4E0,5E0,8E0,5E0,1.7E1,6.4E1,6E0,1.1E1,8.6E1,5E0,6E0,1.7E1,5E0,1.08E2,1.5E1,1.1E1,3.9E1,1.4E1,8E0,7E0,7E0,2E1,6E0,3.2E1,6E0,4E0,2.9E1,2.1E1,1.1E1,9E0,5E0,4E0,8E0,3.7E1,2.3E1,6E0,5E0,6E0,1.1E1,3.9E1,2.5E1,7E0,4E0,7.9E1,7E0,6.6E1,4.2E1,1E1,5E0,5E0,6E0,1.6E1,2.3E1,4E0,1E1,4E0,4E0,1.4E1,6E0,5E0,2.7E1,4E0,2.5E1,1.4E1,7E0,6E0,5E0,5E0,4E0,4E0,4E0,1.3E1,2.4E1,1.2E1,1.1E1,6E0,3.3E1,7.4E1,5E0,5.9E1,7E0,2.2E1,2E1,4E0,6E0,4E0,1.9E1,6E0,4E0,6E0,8E0,6E0,2.1E1,4E0,2.1E1,1E1,4E0,5E0,8E0,2E1,4E0,4E0,8E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"209","size_leaf_vector":"1"}},{"base_weights":[-5.7131415E-3,-2.1280274E-1,6.4551926E-1,-6.5236235E-1,-1.1418982E-2,7.558895E-1,-1.8537945E-1,-1.0013372E0,-5.050563E-1,-1.9263324E-1,1.7599282E-1,5.0836426E-1,1.0645207E0,-6.724832E-1,1.7671658E-1,-1.3869389E0,-8.3693385E-1,-8.493203E-1,-4.0978426E-1,-2.9323986E-1,1.7531294E-5,1.0551816E-1,4.7692347E-1,3.4867108E-1,7.809663E-1,1.2617676E0,6.7906153E-1,-4.0480398E-2,-1.3340322E-2,3.5177413E-1,-1.071945E-2,-3.424277E-2,-7.1736574E-2,-9.073491E-1,-5.2073604E-1,-9.080936E-1,-1.3469453E-2,-5.8451164E-1,-1.629588E-1,-2.6605478E-1,-4.7920655E-2,-4.2612037E-1,5.5164583E-2,1.2563364E-1,-2.8618587E-2,1.8970884E-1,5.530684E-1,4.8565168E-2,5.006389E-1,8.5673195E-1,4.301134E-1,8.1547685E-2,1.1213405E0,5.0695276E-1,1.0084851E0,8.504519E-3,2.003051E-2,-4.3692246E-2,-2.3995148E-2,-9.402905E-3,-3.3181816E-2,-4.461401E-2,-2.2543114E-2,-3.4037873E-1,-6.393693E-1,3.5912238E-2,-3.044859E-1,-4.7926924E-1,-1.7449664E-1,4.4251833E-4,-3.6534168E-2,-1.4875851E-2,1.5091062E-1,1.6019753E-1,-1.6385362E-1,4.930058E-3,1.1777003E-2,9.275461E-3,6.2385917E-1,1.4891837E-2,-7.70621E-2,6.118049E-1,2.0568298E-2,4.150985E-2,1.9314384E-2,2.6112651E-2,8.406481E-3,6.12446E-1,5.606765E-2,3.3181302E-2,3.4322318E-1,2.5779746E-2,5.4800406E-2,-2.5190907E-2,1.3960036E-3,-7.3375976E-1,-4.7934717E-1,-1.3277434E-1,-4.0765643E-1,-5.0291383E-1,-7.226789E-3,-2.4905093E-1,-1.7495241E-2,2.2120388E-2,-1.6507284E-1,2.0853803E-1,-5.4340563E-3,2.6271075E-1,6.8277985E-2,-3.2757118E-1,-1.2719233E-2,6.793986E-1,1.247167E-2,-1.2805148E-2,-9.710938E-3,6.8976617E-1,2.85556E-1,-8.303377E-3,1.0045475E-2,1.5552431E-2,3.4197014E-2,5.7263775E-3,4.0598503E-1,-1.6412728E-2,-3.5428498E-2,-8.322831E-3,-2.4187442E-2,-2.1534975E-1,5.061374E-3,-4.7666577E-1,-4.3011047E-3,-1.08891465E-2,-5.285037E-1,-3.1408364E-1,6.8513486E-3,2.5285916E-2,-1.3646984E-1,1.017859E-1,-5.130827E-2,-1.283611E-2,1.5091422E-4,3.032222E-1,1.1829586E-1,2.9116493E-1,-1.2637876E-2,-1.8512274E-1,1.1118296E-1,-7.440535E-3,-2.0304175E-2,7.4566975E-3,-6.5473993E-3,3.322434E-2,1.666092E-2,-5.1218826E-2,4.243391E-3,3.4646105E-2,4.7037703E-1,7.2624353E-3,1.7669909E-2,2.1585908E-2,1.0477241E-2,-1.236151E-1,-1.606056E-2,-5.556968E-1,-8.443694E-3,-2.5964834E-2,-1.5681634E-2,-2.0712969E-1,-4.3908915E-1,1.4772722E-1,-8.671995E-3,1.3502525E-1,-2.5744956E-2,-1.6614421E-3,-8.720762E-3,-3.0788453E-3,6.840154E-3,-1.1333294E-1,5.087547E-2,1.740212E-2,4.419779E-3,3.644565E-2,1.2117101E-2,1.8387146E-1,4.721319E-1,-1.3897703E-4,-2.5107273E-1,2.8319046E-1,7.43159E-2,1.503767E-3,-4.4892463E-3,1.1996412E-2,2.6221275E-2,-2.2326733E-3,-8.237311E-3,-2.762764E-2,-1.4282202E-2,1.1469987E-5,-1.1021158E-2,-2.2399798E-2,-5.234065E-3,8.955E-3,2.4776862E-3,2.861803E-3,8.011894E-3,-3.1723324E-3,4.627169E-3,2.5558582E-4,-8.315084E-3,6.157977E-3,-1.9272601E-3,-2.8932558E-3,4.3339166E-3,-2.726627E-3,1.3586501E-2,1.6983192E-2,3.0254316E-2,-6.7836526E-3,-1.448177E-2,8.466768E-3,1.7149435E-2,2.0640255E-3,1.604899E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,-1,-1,-1,57,59,61,-1,63,65,67,-1,69,71,73,-1,75,77,79,81,83,85,-1,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,93,95,-1,97,99,101,-1,-1,103,105,107,109,-1,-1,-1,111,-1,113,115,117,-1,-1,-1,-1,119,-1,-1,121,-1,-1,-1,-1,123,125,127,129,131,-1,133,135,137,139,141,-1,143,145,147,149,151,-1,-1,153,155,157,-1,-1,-1,-1,-1,159,-1,-1,-1,-1,161,-1,163,-1,-1,165,167,169,171,173,175,177,-1,-1,179,181,183,-1,185,187,-1,-1,-1,-1,-1,-1,189,-1,-1,191,-1,-1,-1,-1,193,-1,195,-1,-1,-1,197,199,201,-1,203,205,-1,-1,-1,-1,207,209,-1,-1,211,-1,213,215,-1,217,219,221,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.391816E2,6.930384E1,2.2975037E1,1.214016E1,1.830529E1,1.6265518E1,5.410189E0,3.4036102E0,5.3917465E0,5.3106565E0,5.569791E0,5.1257553E0,6.5181503E0,7.63299E-1,1.3695884E0,9.9077606E-1,7.075691E-1,9.481716E-1,5.8994236E0,3.3647804E0,2.2560303E0,3.2278268E0,9.88575E-1,3.6242008E0,8.2772446E-1,2.4739532E0,1.4731293E0,0E0,0E0,9.849846E-2,0E0,0E0,0E0,1.3433838E-1,6.0559464E-1,3.273468E-1,0E0,8.671684E-1,8.088253E0,3.332635E0,0E0,1.7553664E0,5.722629E-1,2.1374733E0,0E0,3.7742227E-2,8.4738255E-1,9.8786324E-1,2.815895E0,2.8396225E-1,2.3242116E-1,0E0,1.6984177E0,6.1827564E-1,2.18606E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3073114E0,6.3006973E-1,0E0,8.6708355E-1,3.1826973E-1,1.447079E0,0E0,0E0,2.8148764E-1,5.8162445E-1,1.7630029E0,5.649306E-1,0E0,0E0,0E0,4.4557285E-1,0E0,2.66863E-1,8.967304E-1,4.7284296E-1,0E0,0E0,0E0,0E0,1.0319781E-1,0E0,0E0,1.698699E-1,0E0,0E0,0E0,0E0,2.1626472E-1,2.62753E-1,4.4381347E-1,6.5025806E-1,2.0365429E-1,0E0,1.4000812E0,2.121355E-1,2.4506144E-1,1.9074523E-1,2.204653E-1,0E0,1.4376955E0,1.1186697E0,1.3422465E-1,3.1488374E-1,8.2829475E-2,0E0,0E0,7.215386E-2,9.60331E-2,6.42761E-2,0E0,0E0,0E0,0E0,0E0,2.5754333E-2,0E0,0E0,0E0,0E0,1.5119374E-1,0E0,5.023408E-1,0E0,0E0,8.1721306E-2,8.048301E-1,5.210902E-1,1.7707913E-1,5.1027894E-2,1.6573937E-1,1.4519995E-1,0E0,0E0,1.7740571E-1,1.9078872E-1,1.5635972E0,0E0,1.7940146E-1,5.306573E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.935592E-2,0E0,0E0,5.7418108E-2,0E0,0E0,0E0,0E0,3.6671773E-2,0E0,2.379036E-2,0E0,0E0,0E0,2.4772656E-1,3.923688E-1,3.7329063E-2,0E0,1.731725E-2,1.2576392E-1,0E0,0E0,0E0,0E0,1.119104E-1,7.424067E-2,0E0,0E0,7.407257E-2,0E0,1.4897128E0,3.513689E-1,0E0,2.5511563E-2,6.389427E-2,5.753132E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,54,54,63,63,64,64,66,66,67,67,68,68,71,71,72,72,73,73,74,74,78,78,80,80,81,81,82,82,87,87,90,90,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,107,107,108,108,109,109,110,110,111,111,114,114,115,115,116,116,122,122,127,127,129,129,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,145,145,146,146,153,153,156,156,161,161,163,163,167,167,168,168,169,169,171,171,172,172,177,177,178,178,181,181,183,183,184,184,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,-1,-1,-1,58,60,62,-1,64,66,68,-1,70,72,74,-1,76,78,80,82,84,86,-1,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,94,96,-1,98,100,102,-1,-1,104,106,108,110,-1,-1,-1,112,-1,114,116,118,-1,-1,-1,-1,120,-1,-1,122,-1,-1,-1,-1,124,126,128,130,132,-1,134,136,138,140,142,-1,144,146,148,150,152,-1,-1,154,156,158,-1,-1,-1,-1,-1,160,-1,-1,-1,-1,162,-1,164,-1,-1,166,168,170,172,174,176,178,-1,-1,180,182,184,-1,186,188,-1,-1,-1,-1,-1,-1,190,-1,-1,192,-1,-1,-1,-1,194,-1,196,-1,-1,-1,198,200,202,-1,204,206,-1,-1,-1,-1,208,210,-1,-1,212,-1,214,216,-1,218,220,222,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8988005E5,3.2856784E7,3.3817584E7,9.31E2,7.459E3,1.3655363E2,2E0,8E1,1.5859042E6,2.8530578E6,1.087521E3,7.153514E1,1.4916515E3,4.4910244E3,4.869229E7,1.7E1,9.643839E6,6.15E2,1.05039425E1,1.45064E5,1.2294118E1,4.8581E4,5.486433E5,2.6506329E1,7.9E1,3.8537518E2,-4.0480398E-2,-1.3340322E-2,1.4595416E3,-1.071945E-2,-3.424277E-2,-7.1736574E-2,2.568306E0,2.3E1,5.159652E-7,-1.3469453E-2,2.7317073E0,2.7E1,4.33E2,-4.7920655E-2,1.1015E4,8.216578E9,2.1129E4,-2.8618587E-2,6.5718125E6,4.745397E6,7.6937294E-1,2E0,1E0,6.439711E1,8.1547685E-2,1.8598528E6,8.710612E1,2.9622424E3,8.504519E-3,2.003051E-2,-4.3692246E-2,-2.3995148E-2,-9.402905E-3,-3.3181816E-2,-4.461401E-2,-2.2543114E-2,1E1,1.852364E6,3.5912238E-2,3.3E1,2.3043478E2,2.240836E6,4.4251833E-4,-3.6534168E-2,9.1E1,7.77E2,3.7105262E0,2.2503355E1,4.930058E-3,1.1777003E-2,9.275461E-3,2E0,1.4891837E-2,8.295515E7,2E0,8.399457E2,4.150985E-2,1.9314384E-2,2.6112651E-2,8.406481E-3,3.3412E5,5.606765E-2,3.3181302E-2,7.25E2,2.5779746E-2,5.4800406E-2,-2.5190907E-2,1.3960036E-3,2E0,2.2E1,2.152E3,1.363176E7,1.9E1,-7.226789E-3,3.541824E2,2.1578948E0,1.4067796E0,1.3902439E0,2.1111E4,-5.4340563E-3,2.8366232E0,2.7015875E2,2.956111E2,2.884188E2,2E1,1.247167E-2,-1.2805148E-2,7.792289E9,7.234179E6,1.2E1,-8.303377E-3,1.0045475E-2,1.5552431E-2,3.4197014E-2,5.7263775E-3,1.39442E5,-1.6412728E-2,-3.5428498E-2,-8.322831E-3,-2.4187442E-2,3.9157894E0,5.061374E-3,3.2874417E2,-4.3011047E-3,-1.08891465E-2,1.585814E6,3.6981132E0,3.9649122E0,3.4846212E5,1.3E1,7.117E3,1.0772152E1,-1.283611E-2,1.5091422E-4,1.1E1,2.3534782E2,7.6937294E-1,-1.2637876E-2,4.878788E0,1.26E2,-7.440535E-3,-2.0304175E-2,7.4566975E-3,-6.5473993E-3,3.322434E-2,1.666092E-2,1.1E1,4.243391E-3,3.4646105E-2,5.1792985E6,7.2624353E-3,1.7669909E-2,2.1585908E-2,1.0477241E-2,1E0,-1.606056E-2,1.266544E0,-8.443694E-3,-2.5964834E-2,-1.5681634E-2,1.2877E4,2.7E1,1.8578552E7,-8.671995E-3,1.4473684E-1,1.746E3,-1.6614421E-3,-8.720762E-3,-3.0788453E-3,6.840154E-3,1.2E1,1E0,1.740212E-2,4.419779E-3,1.3E1,1.2117101E-2,1.196966E6,1.935E3,-1.3897703E-4,1.0142858E1,6.7148806E5,3.3412E5,1.503767E-3,-4.4892463E-3,1.1996412E-2,2.6221275E-2,-2.2326733E-3,-8.237311E-3,-2.762764E-2,-1.4282202E-2,1.1469987E-5,-1.1021158E-2,-2.2399798E-2,-5.234065E-3,8.955E-3,2.4776862E-3,2.861803E-3,8.011894E-3,-3.1723324E-3,4.627169E-3,2.5558582E-4,-8.315084E-3,6.157977E-3,-1.9272601E-3,-2.8932558E-3,4.3339166E-3,-2.726627E-3,1.3586501E-2,1.6983192E-2,3.0254316E-2,-6.7836526E-3,-1.448177E-2,8.466768E-3,1.7149435E-2,2.0640255E-3,1.604899E-2],"split_indices":[2,43,60,7,2,2,71,32,44,43,43,67,71,4,4,46,3,62,2,69,7,68,1,43,71,0,71,0,0,4,0,0,0,68,3,52,0,69,0,2,0,9,46,44,0,60,60,53,6,6,73,0,43,73,67,0,0,0,0,0,0,0,0,3,9,0,8,4,9,0,0,0,2,69,71,0,0,0,32,0,7,32,67,0,0,0,0,1,0,0,0,0,0,0,0,32,10,2,9,67,0,67,68,68,68,9,0,53,4,4,67,3,0,0,46,62,3,0,0,0,0,0,44,0,0,0,0,71,0,67,0,0,9,71,69,43,3,1,73,0,0,8,4,53,0,73,10,0,0,0,0,0,0,3,0,0,60,0,0,0,0,26,0,53,0,0,0,9,3,5,0,71,44,0,0,0,0,3,26,0,0,3,0,9,2,0,73,43,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,7.82E2,2.48E2,2.45E2,5.37E2,2.19E2,2.9E1,7.1E1,1.74E2,2.73E2,2.64E2,1.23E2,9.6E1,1.2E1,1.7E1,1.9E1,5.2E1,3.6E1,1.38E2,1.79E2,9.4E1,2.15E2,4.9E1,7.9E1,4.4E1,6.2E1,3.4E1,7E0,5E0,1.2E1,5E0,5E0,1.4E1,4.1E1,1.1E1,3.2E1,4E0,8E1,5.8E1,1.74E2,5E0,1E1,8.4E1,2.1E2,5E0,1.1E1,3.8E1,2.7E1,5.2E1,3.5E1,9E0,1.1E1,5.1E1,2.4E1,1E1,5E0,7E0,3.6E1,5E0,5E0,6E0,2.7E1,5E0,1.6E1,6.4E1,7E0,5.1E1,5.1E1,1.23E2,5E0,5E0,4.9E1,3.5E1,1.88E2,2.2E1,6E0,5E0,7E0,3.1E1,8E0,1.9E1,4.2E1,1E1,3.1E1,4E0,5E0,4E0,9E0,4.2E1,9E0,1.5E1,4E0,6E0,1E1,6E0,3.8E1,2.6E1,2E1,3.1E1,4.7E1,4E0,8.3E1,4E1,4E1,9E0,2.9E1,6E0,8.8E1,1E2,1E1,1.2E1,2.6E1,5E0,4E0,1.5E1,3.3E1,9E0,5E0,5E0,4E0,5E0,4E0,1.1E1,4E0,3.4E1,4E0,2.2E1,1.5E1,5E0,2.5E1,6E0,5E0,4.2E1,6.6E1,1.7E1,3E1,1E1,1.9E1,2.1E1,5E0,4E0,1.3E1,1.6E1,8.4E1,4E0,1.4E1,8.6E1,5E0,5E0,5E0,7E0,2.2E1,4E0,1.1E1,4E0,2.4E1,9E0,5E0,4E0,7E0,4E0,1E1,5E0,1.9E1,6E0,3.4E1,8E0,3.7E1,2.9E1,1E1,7E0,9E0,2.1E1,4E0,6E0,4E0,1.5E1,1.3E1,8E0,9E0,4E0,1.1E1,5E0,5.4E1,3E1,4E0,1E1,1.4E1,7.2E1,4E0,7E0,4E0,5E0,5E0,5E0,1.5E1,4E0,5E0,3.2E1,2.5E1,4E0,6E0,4E0,4E0,5E0,1.6E1,5E0,5E0,8E0,4E0,4E0,4E0,7E0,1.7E1,3.7E1,2.1E1,9E0,5E0,5E0,8E0,6E0,6.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"223","size_leaf_vector":"1"}},{"base_weights":[-2.7321313E-2,-2.1315159E-1,6.306059E-1,-6.0968655E-1,-3.1799924E-2,7.2708744E-1,-5.9831965E-1,-8.8215536E-1,-4.7226286E-1,-2.0722754E-1,1.3934027E-1,8.293986E-2,6.703266E-1,-7.6598215E-1,-1.9742332E-3,-9.230321E-1,-6.1476046E-1,-5.123159E-1,3.2396268E-2,-4.908164E-1,-1.3956308E-1,8.3611384E-2,4.0579364E-1,4.7080368E-1,8.793944E-1,-4.0456094E-2,-1.9186908E-2,-3.615302E-2,-4.6607025E-2,-2.1061836E-2,-3.7400108E-2,1.5865411E-1,-5.477417E-1,-5.198807E-1,-5.2498714E-3,-1.2169319E-1,-4.0987976E-2,2.1637887E-1,-5.33264E-4,1.7481834E-1,5.268182E-1,1.8728648E-1,6.160114E-1,9.235906E-1,8.279611E-3,-1.2028875E-2,2.5222281E-2,-6.198929E-1,-2.2404358E-1,-1.082487E-2,-5.405165E-1,-1.6959687E-1,4.6009153E-2,2.5578412E-1,-1.0140608E-2,2.3873476E-2,-3.665751E-1,4.7975876E-3,1.383455E-2,5.8828557E-1,4.1362466E-3,-7.1966544E-2,2.8597155E-1,7.824462E-1,4.319763E-1,9.632503E-1,1.4324811E-2,-6.794125E-1,-2.871075E-1,-7.158672E-2,-2.9669267E-1,-2.6181515E-2,-1.6252914E-2,-1.1395624E-1,-3.5214362E-1,3.2989262E-3,1.6241241E-2,1.1122572E-1,4.8269832E-1,5.7618495E-2,-1.851813E-1,-2.8752903E-3,-2.760847E-2,1.4357027E-2,6.527789E-1,1.8022266E-3,-6.5119686E-3,2.2731097E-2,1.8925148E-1,8.800229E-3,8.526836E-1,3.4963164E-1,2.9152198E-2,7.874639E-1,5.2728053E-2,-2.3668635E-1,-7.176559E-1,5.290712E-3,-3.8834456E-1,-5.3105573E-3,-7.527128E-4,-1.6047608E-2,-5.570708E-3,-2.509221E-2,-1.9591716E-1,-5.0313705E-1,-1.4803258E-1,9.098035E-3,-5.211884E-2,2.0809026E-1,-6.796483E-2,5.4342055E-1,2.1304288E-3,3.8789086E-2,1.6075788E-2,-3.3229384E-1,1.5198614E-2,1.7619217E-2,3.2974865E-2,-3.0607564E-4,2.7367184E-1,9.658273E-1,2.3954164E-2,3.9227602E-1,3.8595155E-3,8.813821E-1,6.2130105E-1,-1.5695505E-2,-3.034763E-3,-6.6468513E-1,-1.0128735E0,-2.6655287E-2,-2.6167172E-1,-1.051066E-2,1.6605336E-2,-2.5957415E-1,-8.310299E-2,-6.68079E-3,-6.0008013E-1,-1.804626E-2,-1.87105E-2,-8.626734E-4,-1.6607787E-2,1.8373882E-2,2.756532E-1,-1.4515831E-2,1.01857E-1,1.8210046E-2,2.7924657E-2,-1.0048564E-1,6.1597485E-2,-1.9242357E-2,-8.902351E-3,6.1707953E-3,-4.9069044E-3,1.476883E-2,5.8638025E-3,3.70601E-2,5.860416E-2,1.9564262E-2,9.020591E-3,4.287401E-2,2.2662248E-2,1.7723396E-2,3.214333E-2,-3.265898E-2,-2.2636048E-2,-5.324868E-2,-2.5816234E-2,-6.2112897E-3,-1.5400013E-2,2.2749745E-3,-8.187207E-3,-6.239129E-3,-1.5513657E-2,-9.91701E-3,-1.9199622E-3,-3.824728E-2,-1.6531495E-2,-3.667687E-3,3.083856E-3,-2.4959133E-3,3.1207814E-3,4.0119765E-3,-3.1166603E-3,4.8071994E-3,1.4476486E-2,-1.2227817E-4,7.1349684E-3,6.553441E-3,-8.920388E-3,8.233219E-3,1.6653351E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,45,47,49,-1,51,-1,53,55,57,59,61,63,65,-1,-1,-1,67,69,-1,71,73,75,77,-1,79,81,-1,-1,83,-1,85,87,89,91,93,-1,95,97,99,101,-1,-1,103,105,107,-1,109,111,113,115,-1,-1,-1,117,-1,-1,-1,119,-1,121,123,-1,125,-1,127,129,-1,131,-1,-1,-1,-1,133,135,137,139,-1,141,143,145,147,-1,149,-1,151,153,-1,-1,-1,155,157,-1,159,-1,161,163,-1,-1,165,167,-1,169,-1,171,173,175,-1,177,179,-1,181,-1,183,185,-1,187,-1,-1,189,191,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2545379E2,5.7510433E1,2.7231544E1,8.9817505E0,1.654137E1,1.1529106E1,1.5508342E0,3.3976746E-1,8.302612E0,5.150878E0,4.108984E0,0E0,7.970459E0,1.3645267E-1,0E0,5.4271698E-2,6.415844E-2,3.985115E0,0E0,5.1097107E-1,2.8446512E0,2.5919433E0,1.2496796E0,4.207363E0,2.8361664E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6027203E0,3.5045319E0,1.5621662E-1,0E0,1.7510853E0,0E0,1.6019707E0,1.2862974E0,1.2812662E-1,7.8057575E-1,9.6720326E-1,1.7754593E0,1.9155502E0,0E0,0E0,0E0,2.3204575E0,3.1103635E-1,0E0,1.813507E-2,1.6764069E0,6.3827497E-1,2.6689067E0,0E0,9.6303517E-1,5.955895E-1,0E0,0E0,2.8039265E-1,0E0,8.971529E-2,4.4498086E-1,1.2928009E0,3.53014E-1,1.7619934E0,0E0,1.6417923E0,8.983244E-1,2.4501812E-2,1.3657093E-1,0E0,0E0,9.492065E-1,1.1274762E0,4.824133E-1,0E0,9.167177E-1,8.232727E-1,6.261544E-1,5.6514E-1,0E0,0E0,0E0,7.532787E-2,0E0,0E0,0E0,3.3253253E-1,0E0,6.713581E-1,2.7360535E-1,0E0,1.7026138E-1,0E0,1.5389633E-1,8.59272E-1,0E0,2.5520897E-1,0E0,0E0,0E0,0E0,5.439538E-1,4.6676445E-1,6.778736E-1,5.9392595E-1,0E0,5.510912E-1,4.3074036E-1,8.26516E-1,1.6768456E-2,0E0,3.58796E-1,0E0,4.906094E-2,1.4353059E-1,0E0,0E0,0E0,5.3943157E-2,7.2143555E-2,0E0,5.8703184E-2,0E0,3.7015915E-2,8.700609E-2,0E0,0E0,2.8822327E-1,1.806736E-1,0E0,6.4600706E-2,0E0,3.5032257E-1,3.6281037E-1,1.3317958E-1,0E0,6.418886E-1,7.2921544E-2,0E0,1.1639744E-1,0E0,6.462759E-2,1.2911427E-1,0E0,6.63499E-2,0E0,0E0,3.8643885E-1,2.796137E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,31,31,32,32,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,47,47,48,48,50,50,51,51,52,52,53,53,55,55,56,56,59,59,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,73,73,74,74,75,75,77,77,78,78,79,79,80,80,84,84,88,88,90,90,91,91,93,93,95,95,96,96,98,98,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,113,113,115,115,116,116,120,120,121,121,123,123,125,125,126,126,129,129,130,130,132,132,134,134,135,135,136,136,138,138,139,139,141,141,143,143,144,144,146,146,149,149,150,150],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,46,48,50,-1,52,-1,54,56,58,60,62,64,66,-1,-1,-1,68,70,-1,72,74,76,78,-1,80,82,-1,-1,84,-1,86,88,90,92,94,-1,96,98,100,102,-1,-1,104,106,108,-1,110,112,114,116,-1,-1,-1,118,-1,-1,-1,120,-1,122,124,-1,126,-1,128,130,-1,132,-1,-1,-1,-1,134,136,138,140,-1,142,144,146,148,-1,150,-1,152,154,-1,-1,-1,156,158,-1,160,-1,162,164,-1,-1,166,168,-1,170,-1,172,174,176,-1,178,180,-1,182,-1,184,186,-1,188,-1,-1,190,192,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,4.217427E7,8.8E1,9.31E2,2.2968E4,2.629E3,1.0795462E-6,6.68411E2,3.95E2,2.8530578E6,8.293986E-2,2.36433E6,1.665909E3,-1.9742332E-3,6.48334E5,4.9558692E7,6E0,3.2396268E-2,7.9016544E5,9.639872E0,1.7142857E0,6.009E4,8.5023944E2,1.0424884E-7,-4.0456094E-2,-1.9186908E-2,-3.615302E-2,-4.6607025E-2,-2.1061836E-2,-3.7400108E-2,1.0332258E2,7.997723E6,1.9E1,-5.2498714E-3,2.9652428E6,-4.0987976E-2,1.0767881E7,1.2294118E1,6.965855E9,8.0605554E2,6.516E3,9E1,7.153514E1,8.279611E-3,-1.2028875E-2,2.5222281E-2,3.2479605E2,1.8E0,-1.082487E-2,1.585814E6,1.23E2,3.1847827E2,4.797342E2,-1.0140608E-2,2.2968E4,1.4787234E1,4.7975876E-3,1.383455E-2,3.7788504E7,4.1362466E-3,1.3E1,3.3412E5,1.2825651E0,1.000501E6,7.459E3,1.4324811E-2,1.9186046E0,4.6847186E2,2.019537E2,4.6404468E2,-2.6181515E-2,-1.6252914E-2,1E0,7.6E2,6.728972E-1,1.6241241E-2,3.4E3,1E0,4.86392E5,3.1797794E1,-2.8752903E-3,-2.760847E-2,1.4357027E-2,3.474468E2,1.8022266E-3,-6.5119686E-3,2.2731097E-2,8.7390656E2,8.800229E-3,2.1298597E0,5.712652E8,2.9152198E-2,5.0741018E8,5.2728053E-2,1.1E1,4.9735293E0,5.290712E-3,5.100098E4,-5.3105573E-3,-7.527128E-4,-1.6047608E-2,-5.570708E-3,4.82E2,2.118835E6,1.0865825E0,9E0,9.098035E-3,5.6530495E6,5.5E1,5E0,1.798E3,2.1304288E-3,2.8788235E2,1.6075788E-2,7.3131656E5,4.383864E7,1.7619217E-2,3.2974865E-2,-3.0607564E-4,7.683389E7,2.64538E3,2.3954164E-2,4.4895835E0,3.8595155E-3,2.395631E7,3.422592E6,-1.5695505E-2,-3.034763E-3,1.318849E6,8.237624E0,-2.6655287E-2,1.9032816E7,-1.051066E-2,1.6527965E7,1.22896E5,5.3E2,-6.68079E-3,1E0,2.9417648E2,-1.87105E-2,4.1973075E6,-1.6607787E-2,1.3529412E0,5.6151875E5,-1.4515831E-2,7.9016544E5,1.8210046E-2,2.7924657E-2,1.089E3,1E0,-1.9242357E-2,-8.902351E-3,6.1707953E-3,-4.9069044E-3,1.476883E-2,5.8638025E-3,3.70601E-2,5.860416E-2,1.9564262E-2,9.020591E-3,4.287401E-2,2.2662248E-2,1.7723396E-2,3.214333E-2,-3.265898E-2,-2.2636048E-2,-5.324868E-2,-2.5816234E-2,-6.2112897E-3,-1.5400013E-2,2.2749745E-3,-8.187207E-3,-6.239129E-3,-1.5513657E-2,-9.91701E-3,-1.9199622E-3,-3.824728E-2,-1.6531495E-2,-3.667687E-3,3.083856E-3,-2.4959133E-3,3.1207814E-3,4.0119765E-3,-3.1166603E-3,4.8071994E-3,1.4476486E-2,-1.2227817E-4,7.1349684E-3,6.553441E-3,-8.920388E-3,8.233219E-3,1.6653351E-3],"split_indices":[2,43,60,44,2,12,0,52,67,2,43,0,43,4,0,46,46,3,0,43,69,68,1,67,52,0,0,0,0,0,0,67,9,67,0,43,0,62,68,46,4,44,8,71,0,0,0,67,68,0,9,0,4,4,0,44,71,0,0,7,0,3,1,69,43,2,0,69,4,4,4,0,0,26,2,69,0,44,6,1,71,0,0,0,4,0,0,0,4,0,53,47,0,7,0,3,68,0,43,0,0,0,0,2,9,71,8,0,43,0,8,2,0,70,0,43,12,0,0,0,47,4,0,68,0,62,43,0,0,9,71,0,5,0,60,9,2,0,29,4,0,43,0,68,66,0,43,0,0,2,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.024E3,7.99E2,2.25E2,2.5E2,5.49E2,2.09E2,1.6E1,8.2E1,1.68E2,2.71E2,2.78E2,9E0,2E2,1.2E1,4E0,6.9E1,1.3E1,1.63E2,5E0,5.1E1,2.2E2,2.31E2,4.7E1,1.04E2,9.6E1,8E0,4E0,3E1,3.9E1,9E0,4E0,8E0,1.55E2,4.7E1,4E0,2.16E2,4E0,8.9E1,1.42E2,1.7E1,3E1,3.6E1,6.8E1,9E1,6E0,4E0,4E0,1.26E2,2.9E1,4E0,4.3E1,1.68E2,4.8E1,8.2E1,7E0,1.34E2,8E0,1.2E1,5E0,2.6E1,4E0,1E1,2.6E1,3.4E1,3.4E1,8.4E1,6E0,1.06E2,2E1,1E1,1.9E1,3.6E1,7E0,1.3E2,3.8E1,4.3E1,5E0,5.1E1,3.1E1,1.16E2,1.8E1,4E0,4E0,6E0,2E1,4E0,6E0,7E0,1.9E1,4E0,3E1,2.6E1,8E0,4.5E1,3.9E1,9E0,9.7E1,4E0,1.6E1,5E0,5E0,1.4E1,5E0,6.3E1,6.7E1,2.1E1,1.7E1,9E0,3.4E1,3.3E1,1.8E1,2.7E1,4E0,1.1E2,6E0,1E1,8E0,5E0,1.5E1,6E0,1.3E1,2.1E1,9E0,2.2E1,4E0,2.6E1,1.9E1,5E0,4E0,8.5E1,1.2E1,5E0,1.1E1,1E1,5.3E1,4.2E1,2.5E1,5E0,1.6E1,1.2E1,5E0,3E1,4E0,9E0,2.4E1,7E0,1.1E1,1E1,1.7E1,1.5E1,9.5E1,5E0,5E0,4E0,4E0,9E0,4E0,1.6E1,5E0,1.8E1,4E0,2.2E1,4E0,6E0,1.3E1,6.6E1,1.9E1,8E0,4E0,5E0,6E0,4.6E1,7E0,1.7E1,2.5E1,5E0,2E1,7E0,9E0,7E0,5E0,1.7E1,1.3E1,5E0,4E0,5E0,1.9E1,4E0,7E0,4E0,1.1E1,1.6E1,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"193","size_leaf_vector":"1"}},{"base_weights":[8.976589E-3,-1.8565607E-1,5.751452E-1,-5.6272644E-1,-2.3799082E-2,6.2997967E-1,-2.76464E-1,-7.7734095E-1,-4.4256908E-1,-2.1286839E-1,1.2544522E-1,2.6550955E-1,7.993267E-1,4.3805264E-2,-3.2085083E-2,-8.1119436E-1,-5.5827997E-3,-4.7796077E-1,2.0840403E-2,-4.5805752E-1,-1.431906E-1,2.819183E-2,3.1765613E-1,1.5147437E-1,6.3087624E-1,1.403616E0,7.234894E-1,-1.037821E-2,1.6501375E-2,-3.9388802E-2,-2.1298224E-2,-7.748547E-2,-5.2942514E-1,-4.893122E-1,-9.062538E-3,-4.8304686E-1,-1.1806283E-1,1.6423035E-1,-5.593352E-2,-1.1546289E-2,3.5989767E-1,9.9220954E-2,2.211636E-2,2.016987E-2,3.5880797E-2,3.6084726E-2,7.382241E-2,5.482194E-1,9.394883E-1,-1.85779E-2,5.3337198E-2,-5.7060766E-1,-1.9487064E-1,-1.4499942E-2,-5.382861E-1,-1.003566E-2,-3.750208E-2,-1.6307665E-1,7.102636E-2,-1.2485949E-1,2.2788653E-1,-1.2284219E-2,-2.6112384E-1,4.6732873E-3,-8.757485E-3,3.8353354E-1,-4.6064984E-3,-5.8206096E-2,2.1282099E-1,1.7325893E-3,5.804768E-1,9.9883366E-1,9.209491E-3,1.3763387E-2,-8.8465676E-2,-6.1320615E-1,-2.8467864E-1,-1.3864814E-2,-1.4656074E-3,-1.4489231E-2,-2.620424E-2,-1.2501457E-1,-3.0542275E-1,-1.4960766E-3,1.5735818E-2,-1.7184617E-2,5.3753514E-2,2.5714293E-1,-3.8416544E-3,-4.5830585E-2,1.8086722E-1,-2.8967168E-2,-4.0176123E-1,2.6755875E-1,5.107791E-1,-9.77747E-3,-3.8284925E-3,3.148756E-1,-5.5413647E-3,4.393887E-1,7.174278E-1,1.0461825E0,1.285394E-2,-7.5843628E-3,2.2770879E-4,-5.5861133E-1,-8.444426E-1,1.9463387E-3,-4.3705767E-1,-2.5320935E-1,-4.7029063E-2,-2.8062437E-2,-1.954484E-1,8.442496E-3,-6.283745E-2,5.73461E-3,-1.2645593E-3,1.9284983E-1,2.015421E-2,-1.522078E-1,5.430307E-3,1.5427817E-2,4.1712478E-2,-5.9377323E-3,3.528872E-3,-2.2562139E-2,-7.6467153E-3,1.6048105E-1,4.5160702E-1,1.0057352E-2,5.5892915E-1,-4.9862205E-3,3.362375E-2,5.703676E-3,3.718224E-1,4.9299067E-1,2.4838407E-1,1.233643E-2,3.5229653E-2,2.9737024E-2,5.0002776E-2,-5.8186895E-1,-8.254169E-3,-4.769092E-2,-2.6963526E-2,-2.3705885E-2,-1.1233391E-2,-6.110349E-2,-2.9792884E-1,2.6852906E-2,-1.15500115E-1,-2.787465E-1,3.3788867E-3,-1.0285062E-2,-1.0577184E-2,2.4601297E-1,5.1241584E-2,9.319882E-4,-2.1062881E-1,1.6773035E-1,-4.8790377E-2,5.2741645E-3,-1.8054264E-3,1.9213195E-1,8.572016E-5,2.77106E-2,8.10215E-3,2.7518088E-2,1.1159483E-2,3.4063791E-3,1.4545783E-4,2.1889698E-2,1.0033796E-2,2.5096098E-2,8.924695E-3,4.0383646E-3,1.5147768E-2,-2.8510474E-2,-2.0666638E-2,-4.3014597E-4,-4.6511157E-3,-1.6400533E-2,-4.911072E-3,-3.5677375E-3,3.1142822E-3,4.9126344E-3,-7.147473E-3,-1.4926865E-2,-4.9747466E-3,-3.3059209E-3,3.122686E-3,1.32718235E-2,5.2311765E-3,7.835372E-3,-3.4389351E-3,-1.5331047E-2,-4.2531174E-3,1.9013898E-3,1.2627951E-2,-8.465574E-3,4.5229162E-6,1.6677508E-2,6.5962044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,-1,33,35,37,39,41,43,45,47,-1,-1,-1,-1,49,51,53,-1,55,57,59,61,63,65,67,-1,-1,-1,-1,-1,69,71,-1,73,75,77,-1,79,-1,-1,81,83,85,87,89,91,-1,-1,93,-1,95,97,-1,99,101,-1,-1,103,105,107,-1,-1,-1,-1,109,111,113,-1,-1,115,117,-1,119,121,123,125,127,129,-1,131,133,-1,135,137,139,-1,-1,-1,141,143,-1,145,147,149,-1,151,-1,153,-1,-1,155,-1,157,159,-1,161,-1,-1,-1,-1,163,165,-1,167,-1,169,-1,171,173,175,-1,-1,-1,-1,177,-1,-1,-1,-1,-1,179,181,183,185,187,-1,189,-1,191,193,-1,195,197,199,-1,-1,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0897934E2,4.4905777E1,1.198864E1,5.355011E0,1.4586942E1,1.4477592E1,2.1725507E0,1.6355019E0,4.770111E0,3.8296175E0,5.393369E0,3.112617E0,6.408577E0,9.2148066E-1,0E0,6.796303E-1,0E0,2.8237972E0,0E0,3.1328678E-1,1.4715998E0,2.2179337E0,1.3530397E0,9.7847974E-1,1.7074823E-1,8.082886E-1,5.012245E0,0E0,0E0,0E0,0E0,7.4344707E-1,1.5834732E0,2.091322E-1,0E0,8.7217784E-1,1.4415362E0,1.3902282E0,1.0715619E0,2.5992435E-1,9.7247887E-1,9.602246E-1,0E0,0E0,0E0,0E0,0E0,1.3456078E0,2.6120338E0,0E0,4.77219E-1,1.1530685E0,2.4802244E-1,0E0,3.64933E-2,0E0,0E0,6.986644E-1,6.4568603E-1,6.379125E-1,5.764792E-1,6.5624213E-1,6.740104E-1,0E0,0E0,1.0734329E0,0E0,1.9108456E-1,1.0905383E0,0E0,1.1918983E0,1.7219849E0,0E0,0E0,6.444004E-2,7.1496964E-1,8.151921E-1,0E0,0E0,0E0,0E0,1.0778015E0,8.1122875E-1,3.1662092E-1,0E0,0E0,5.478035E-2,5.607197E-1,0E0,4.6978372E-1,3.0558726E-1,1.04214124E-1,1.8515182E-1,8.143618E-1,4.1457844E-1,0E0,7.380738E-2,2.1570921E-1,0E0,2.9731607E-1,5.68861E-1,1.03645325E-1,0E0,0E0,0E0,5.941353E-1,1.813879E-1,0E0,3.748107E-2,3.3402133E-1,3.5215706E-1,0E0,5.1258624E-1,0E0,1.80965E-1,0E0,0E0,3.1111252E-1,0E0,2.8622818E-1,5.273847E-1,0E0,5.714111E-2,0E0,0E0,0E0,0E0,1.5026277E-1,5.498185E-1,0E0,2.289772E-1,0E0,1.6880456E-2,0E0,1.8097973E-1,3.6187077E-1,1.1210269E-1,0E0,0E0,0E0,0E0,7.489395E-2,0E0,0E0,0E0,0E0,0E0,1.750857E-2,3.2314444E-1,1.4872387E-1,3.2958877E-1,9.419072E-2,0E0,8.595689E-2,0E0,1.3731492E-1,2.0600218E-1,0E0,2.7127624E-1,1.8925777E-1,2.9580626E-1,0E0,0E0,1.56807E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,47,47,48,48,50,50,51,51,52,52,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,74,74,75,75,76,76,81,81,82,82,83,83,86,86,87,87,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,99,99,100,100,101,101,105,105,106,106,108,108,109,109,110,110,112,112,114,114,117,117,119,119,120,120,122,122,127,127,128,128,130,130,132,132,134,134,135,135,136,136,141,141,147,147,148,148,149,149,150,150,151,151,153,153,155,155,156,156,158,158,159,159,160,160,163,163],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,-1,34,36,38,40,42,44,46,48,-1,-1,-1,-1,50,52,54,-1,56,58,60,62,64,66,68,-1,-1,-1,-1,-1,70,72,-1,74,76,78,-1,80,-1,-1,82,84,86,88,90,92,-1,-1,94,-1,96,98,-1,100,102,-1,-1,104,106,108,-1,-1,-1,-1,110,112,114,-1,-1,116,118,-1,120,122,124,126,128,130,-1,132,134,-1,136,138,140,-1,-1,-1,142,144,-1,146,148,150,-1,152,-1,154,-1,-1,156,-1,158,160,-1,162,-1,-1,-1,-1,164,166,-1,168,-1,170,-1,172,174,176,-1,-1,-1,-1,178,-1,-1,-1,-1,-1,180,182,184,186,188,-1,190,-1,192,194,-1,196,198,200,-1,-1,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,3.9661028E7,8.8E1,8.1E2,8.7115955E2,9.7172376E7,1.1408248E0,6.68411E2,3.89E2,2.0766992E6,3.1993368E6,1.059448E7,1E1,-3.2085083E-2,1.0795462E-6,-5.5827997E-3,1.9186046E0,2.0840403E-2,8.599521E5,1.45064E5,1.75E0,2.9077E4,5.53648E8,1.2003246E6,1.2655613E3,2.8045622E6,-1.037821E-2,1.6501375E-2,-3.9388802E-2,-2.1298224E-2,3.6E2,7.997723E6,1.68E2,-9.062538E-3,1.0153885E-5,3.3486558E6,1E0,9.639872E0,2.426015E-1,9.29554E2,2.4361508E6,2.211636E-2,2.016987E-2,3.5880797E-2,3.6084726E-2,7.382241E-2,1.2825651E0,6.292039E0,-1.85779E-2,6.2E1,3.2479605E2,1.85552E6,-1.4499942E-2,2.2116136E5,-1.003566E-2,-3.750208E-2,3.6507E4,2.0833333E0,3.1157124E-1,2.2968E4,3.1797794E1,5.194E3,4.6732873E-3,-8.757485E-3,4.2210345E2,-4.6064984E-3,1.5137369E6,6.1095314E-8,1.7325893E-3,1.1251919E6,1.0424884E-7,9.209491E-3,1.3763387E-2,1.037E3,7.382199E0,4.046E3,-1.3864814E-2,-1.4656074E-3,-1.4489231E-2,-2.620424E-2,5.47E2,5.4E0,9.4658756E-1,1.5735818E-2,-1.7184617E-2,9.783615E4,4.7185095E2,-3.8416544E-3,4.7567694E5,8.931E3,1.43E2,1.212945E6,5.405423E6,1.9501E4,-9.77747E-3,3.8121E4,4.1032645E6,-5.5413647E-3,2.5345264E7,2.0750147E8,2.707947E8,1.285394E-2,-7.5843628E-3,2.2770879E-4,6.077143E5,3.6981132E0,1.9463387E-3,7.582512E7,2.82E2,1.6E0,-2.8062437E-2,1.0937695E6,8.442496E-3,5.504831E8,5.73461E-3,-1.2645593E-3,1.5132743E0,2.015421E-2,1E0,1.99693E9,1.5427817E-2,4.699871E2,-5.9377323E-3,3.528872E-3,-2.2562139E-2,-7.6467153E-3,4.0911578E2,1.162E3,1.0057352E-2,4.1104166E8,-4.9862205E-3,2.8608696E1,5.703676E-3,1.6942337E1,4.2964826E0,6.2193125E-2,1.233643E-2,3.5229653E-2,2.9737024E-2,5.0002776E-2,2.240836E6,-8.254169E-3,-4.769092E-2,-2.6963526E-2,-2.3705885E-2,-1.1233391E-2,1.9E1,1.4576986E7,1E0,7.9634375E2,7.6E2,3.3788867E-3,9.356961E6,-1.0577184E-2,2.3072304E8,2.749236E6,9.319882E-4,9.44363E5,7.026624E7,2.1340206E0,5.2741645E-3,-1.8054264E-3,7E0,8.572016E-5,2.77106E-2,8.10215E-3,2.7518088E-2,1.1159483E-2,3.4063791E-3,1.4545783E-4,2.1889698E-2,1.0033796E-2,2.5096098E-2,8.924695E-3,4.0383646E-3,1.5147768E-2,-2.8510474E-2,-2.0666638E-2,-4.3014597E-4,-4.6511157E-3,-1.6400533E-2,-4.911072E-3,-3.5677375E-3,3.1142822E-3,4.9126344E-3,-7.147473E-3,-1.4926865E-2,-4.9747466E-3,-3.3059209E-3,3.122686E-3,1.32718235E-2,5.2311765E-3,7.835372E-3,-3.4389351E-3,-1.5331047E-2,-4.2531174E-3,1.9013898E-3,1.2627951E-2,-8.465574E-3,4.5229162E-6,1.6677508E-2,6.5962044E-3],"split_indices":[2,43,60,44,2,67,60,53,67,2,43,43,12,3,0,52,0,69,0,43,7,68,1,7,66,67,43,0,0,0,0,2,9,44,0,53,43,8,69,53,67,60,0,0,0,0,0,69,53,0,0,67,12,0,43,0,0,1,68,53,44,71,44,0,0,67,0,62,52,0,43,52,0,0,2,69,44,0,0,0,0,2,73,68,0,0,48,4,0,43,44,0,9,43,9,0,1,60,0,60,7,7,0,0,0,62,71,0,7,44,68,0,43,0,5,0,0,68,0,26,46,0,67,0,0,0,0,4,10,0,7,0,73,0,71,68,72,0,0,0,0,9,0,0,0,0,0,0,5,29,48,2,0,60,0,7,12,0,9,7,68,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.87E2,7.35E2,2.52E2,2.2E2,5.15E2,2.37E2,1.5E1,7.7E1,1.43E2,2.27E2,2.88E2,7.6E1,1.61E2,9E0,6E0,7.3E1,4E0,1.38E2,5E0,4.9E1,1.78E2,1.92E2,9.6E1,5.9E1,1.7E1,1.6E1,1.45E2,5E0,4E0,6.4E1,9E0,1.6E1,1.22E2,4.3E1,6E0,1.1E1,1.67E2,7.3E1,1.19E2,1.1E1,8.5E1,5.2E1,7E0,9E0,8E0,5E0,1.1E1,8.2E1,6.3E1,4E0,1.2E1,1.08E2,1.4E1,1.1E1,3.2E1,7E0,4E0,1.35E2,3.2E1,1.3E1,6E1,9.9E1,2E1,7E0,4E0,8.1E1,4E0,2.2E1,3E1,5E0,7.7E1,5.8E1,5E0,4E0,8E0,9.3E1,1.5E1,8E0,6E0,5E0,2.7E1,1.08E2,2.7E1,2.6E1,6E0,5E0,8E0,5.5E1,5E0,8.5E1,1.4E1,8E0,1.2E1,4.4E1,3.7E1,5E0,1.7E1,2.3E1,7E0,4E1,3.7E1,5.4E1,4E0,4E0,4E0,7.8E1,1.5E1,5E0,1E1,4E1,6.8E1,6E0,2.1E1,6E0,2E1,4E0,4E0,4.2E1,1.3E1,2.7E1,5.8E1,6E0,8E0,4E0,4E0,8E0,4E0,2.9E1,1.5E1,6E0,3.1E1,4E0,1.3E1,6E0,1.7E1,3E1,1E1,4E0,3.3E1,6E0,4.8E1,7.3E1,5E0,7E0,8E0,6E0,4E0,8E0,3.2E1,3.3E1,3.5E1,1.6E1,5E0,1.6E1,4E0,3E1,1.2E1,7E0,2E1,1.4E1,4.4E1,4E0,4E0,2.4E1,5E0,9E0,6E0,2.7E1,4E0,5E0,8E0,9E0,8E0,2.5E1,5E0,4E0,6E0,5.5E1,1.8E1,4E0,4E0,2.4E1,8E0,9E0,2.4E1,5E0,3E1,1.2E1,4E0,9E0,7E0,2.2E1,8E0,6E0,6E0,9E0,1.1E1,7E0,7E0,1.1E1,3.3E1,4E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"203","size_leaf_vector":"1"}},{"base_weights":[2.5899991E-2,-1.9419216E-1,5.6307477E-1,-5.480851E-1,-1.6844464E-2,-6.132231E-1,6.104611E-1,-7.590241E-1,-4.331985E-1,-1.6650082E-1,1.4112706E-1,-3.4462553E-2,-1.2852693E-2,3.1988806E-1,7.7225965E-1,-6.60953E-1,-4.053686E-2,-3.6462346E-1,-9.5011306E-1,-4.4129208E-1,-1.069993E-1,2.9474112E-1,1.5751341E-2,1.5035282E-1,6.103874E-1,9.355166E-1,4.9535182E-1,-3.4822058E-2,-4.5412448E-1,-5.099062E-1,-1.3240522E-1,-1.2051009E0,-2.5928061E-2,-3.4435704E-1,-6.0574263E-1,-1.6271408E-1,7.9785064E-2,2.10817E-1,4.4555056E-1,7.495849E-2,-2.979508E-1,-1.1562876E-1,2.3606525E-1,6.521973E-1,9.4247125E-3,8.18089E-1,6.682793E-2,1.0034611E0,4.039347E-1,-2.3135023E-2,-1.0750364E-2,-5.448496E-1,-2.1894737E-1,2.403337E-2,-2.5201276E-1,-3.447157E-2,-6.998315E-2,-1.7853767E-2,-8.329051E-3,-1.5855588E-2,-3.0953916E-2,-3.6070856E-1,-9.5359966E-2,1.8069474E-1,-3.775566E-2,1.6005524E-1,4.9938202E-1,5.095464E-1,4.5501604E-3,1.4347373E-1,-3.2632295E-2,-1.7336248E-1,-3.0581964E-2,-1.3022984E-2,6.846515E-4,2.784489E-1,-9.681204E-3,2.2284646E-2,3.5152428E-2,8.643828E-1,4.8850462E-1,2.0195067E-2,6.0435697E-2,5.3143036E-1,4.268525E-2,-4.9129814E-1,-6.93085E-1,-3.5155436E-3,-1.3384149E-2,-1.8813774E-1,-4.759241E-1,-1.7540015E-1,-5.3724694E-1,-1.3317022E-1,6.989068E-2,8.843524E-2,3.194664E-1,-1.0942415E-1,8.5886896E-2,2.0008877E-1,-3.15232E-3,2.8689932E-2,1.4543351E-2,1.710361E-2,3.013517E-2,-4.9161958E-3,1.693727E-1,-1.8062603E-1,2.8509917E-2,-6.3914515E-4,-2.3867954E-1,-3.1393045E-3,3.197293E-3,4.2300195E-1,1.6525213E-1,9.7641134E-1,6.860471E-1,1.2540972E-2,2.8521437E-2,3.4125382E-1,6.6992146E-1,2.1136494E-1,-1.0555592E-2,-2.7661588E-2,-3.5579148E-1,-3.688066E-2,-2.1333795E-2,-2.251704E-1,6.15445E-4,-2.8207693E-2,-1.01252515E-2,-2.2520797E-1,1.340996E-3,-2.25969E-1,-7.7138627E-1,-9.460886E-3,-1.8760873E-1,1.7805059E-2,1.1727069E-2,1.493668E-1,-4.050449E-3,6.199278E-3,1.9432344E-2,-1.6709466E-1,1.8172765E-3,7.148093E-3,-5.885711E-6,1.937808E-2,1.7126915E-1,-8.527477E-3,9.799177E-2,5.077055E-2,2.2464107E-1,-2.4591428E-1,-1.2465415E-3,9.393035E-2,-1.24808066E-1,-1.4112135E-2,-5.589379E-3,1.0473481E-2,4.909368E-1,-3.731076E-5,2.1057786E-1,2.4195883E-2,1.0056611E0,1.1438845E-2,7.508684E-1,4.195695E-1,3.48971E-3,1.5414384E-2,3.317483E-2,1.0831631E-3,1.4571091E-2,-1.8158314E-3,-4.0485722E-1,-1.2284252E-3,-2.683146E-1,-1.5851019E-2,-6.814049E-3,-1.6526833E-2,-1.0469066E-3,-4.748191E-2,-1.7853165E-2,-9.79328E-2,7.960233E-2,-1.12324744E-1,-2.5538152E-1,7.7016495E-2,-4.375171E-3,8.708977E-3,2.2654105E-3,-1.2472493E-2,-1.6801341E-3,5.627967E-4,1.908505E-1,1.1876486E-3,6.9611794E-3,-5.1907017E-3,1.2351117E-1,2.7262053E-1,9.663919E-2,-1.5007516E-2,-5.4422948E-3,1.5850979E-1,9.070066E-3,-3.568707E-4,-1.1103784E-2,2.631044E-2,1.0885399E-2,2.6020753E-1,1.0997864E-1,1.0956743E0,3.504616E-2,2.376837E-2,3.8961507E-2,2.3762712E-2,1.0358896E-2,-2.0228496E-2,-8.560411E-3,-1.428076E-2,-3.418675E-3,1.5080068E-3,-8.294648E-3,2.0374921E-4,5.2380306E-3,-7.723287E-3,8.1116444E-4,-1.4845412E-2,-7.698857E-3,5.2661817E-3,-2.280954E-4,2.2682855E-3,9.481751E-3,1.0832815E-2,2.8570371E-3,8.336831E-3,1.5360508E-2,1.07226E-2,4.919231E-4,1.0295539E-3,1.0005695E-2,3.7886964E-3,-3.0204777E-3,7.2013107E-3,1.4736199E-2,1.6959865E-3,7.449649E-3,5.29985E-2,2.8122451E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,57,59,61,63,65,67,69,71,73,75,77,-1,79,-1,81,83,-1,-1,85,87,-1,89,-1,-1,-1,-1,-1,-1,91,93,95,97,99,101,103,-1,105,107,109,-1,-1,111,113,-1,-1,-1,115,117,-1,-1,119,121,123,125,-1,-1,127,129,131,133,135,137,139,141,143,145,147,149,-1,-1,-1,-1,-1,151,153,155,-1,157,-1,-1,159,161,163,165,-1,-1,167,169,171,-1,-1,173,-1,-1,175,-1,-1,-1,177,-1,179,181,183,185,187,-1,189,-1,-1,-1,191,-1,-1,-1,-1,193,-1,195,197,199,201,-1,203,205,-1,-1,-1,207,-1,209,-1,211,-1,213,215,-1,-1,-1,-1,-1,-1,217,-1,219,-1,-1,-1,-1,-1,-1,221,223,225,227,229,-1,-1,-1,-1,-1,-1,231,-1,-1,-1,233,235,237,-1,-1,239,241,-1,-1,-1,-1,243,245,247,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.20820076E2,4.5493286E1,1.6929268E1,5.5358353E0,1.1465957E1,3.2609987E-1,1.3199051E1,3.9554977E-1,5.289707E0,4.024057E0,4.5405936E0,0E0,0E0,5.0302134E0,7.809494E0,6.1725426E-1,0E0,4.7231846E0,1.1085796E0,4.928131E-1,2.1567519E0,1.2476654E0,2.4554505E0,1.5547571E0,5.1260567E-1,5.6573105E0,2.846161E0,0E0,6.9381475E-2,7.652416E-1,4.5023556E0,2.911539E-1,0E0,1.3752985E-1,2.3151398E-2,2.0859861E0,5.7615006E-1,9.4543624E-1,7.715993E-1,8.2293034E-1,8.376204E-1,3.2894325E-1,1.0215862E0,2.6061726E-1,0E0,1.0272217E0,0E0,1.1537552E0,2.7868376E0,0E0,0E0,2.7073288E-1,8.914286E-2,0E0,5.969732E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2143288E0,7.590878E-1,3.04626E-1,2.1320744E-1,3.9919102E-1,2.9009104E-2,3.7505722E-1,0E0,4.58107E-1,4.0329087E-1,1.7383623E-1,0E0,0E0,5.6437027E-2,6.9915605E-1,0E0,0E0,0E0,9.683914E-1,2.239654E-1,0E0,0E0,9.5658684E-1,8.258761E-1,6.605797E-1,6.941986E-2,0E0,0E0,2.93962E-1,2.1189547E-1,2.2035992E-1,1.2833385E0,6.5821123E-1,2.158435E-1,1.9551578E-1,1.3097978E-1,1.3525873E-1,5.3308338E-2,2.3793173E-1,2.566434E-1,0E0,0E0,0E0,0E0,0E0,3.9500177E-1,1.237697E-1,3.3268738E-1,0E0,5.7365417E-2,0E0,0E0,1.5541339E-1,2.1091986E-1,1.08860016E-1,8.088131E-1,0E0,0E0,4.0531397E-1,1.6530609E-1,2.0765361E-1,0E0,0E0,4.3441868E-1,0E0,0E0,2.7559638E-1,0E0,0E0,0E0,1.05151296E-1,0E0,2.5977653E-1,6.884546E-1,2.5206307E-1,3.1826448E-1,1.3386676E-1,0E0,4.181066E-2,0E0,0E0,0E0,1.3812086E-1,0E0,0E0,0E0,0E0,1.3785636E-1,0E0,3.0402087E-2,2.630603E-1,2.3159552E-1,5.274129E-2,0E0,1.2460758E-1,1.3341984E-1,0E0,0E0,0E0,1.4664006E-1,0E0,8.1222296E-2,0E0,1.4737701E-1,0E0,2.2190475E-1,1.8092537E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.999657E-2,0E0,1.9117963E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7635658E-1,4.0136524E-2,2.4515104E-1,1.5128946E-1,3.9953195E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.149565E-2,0E0,0E0,0E0,8.815101E-2,1.0552621E-1,1.4895368E-1,0E0,0E0,9.800756E-2,6.509405E-2,0E0,0E0,0E0,0E0,3.6113918E-2,2.789019E-2,5.406952E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,51,51,52,52,54,54,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,74,74,75,75,79,79,80,80,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,106,106,107,107,108,108,110,110,113,113,114,114,115,115,116,116,119,119,120,120,121,121,124,124,127,127,131,131,133,133,134,134,135,135,136,136,137,137,139,139,143,143,148,148,150,150,151,151,152,152,153,153,155,155,156,156,160,160,162,162,164,164,166,166,167,167,174,174,176,176,183,183,184,184,185,185,186,186,187,187,194,194,198,198,199,199,200,200,203,203,204,204,209,209,210,210,211,211],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,58,60,62,64,66,68,70,72,74,76,78,-1,80,-1,82,84,-1,-1,86,88,-1,90,-1,-1,-1,-1,-1,-1,92,94,96,98,100,102,104,-1,106,108,110,-1,-1,112,114,-1,-1,-1,116,118,-1,-1,120,122,124,126,-1,-1,128,130,132,134,136,138,140,142,144,146,148,150,-1,-1,-1,-1,-1,152,154,156,-1,158,-1,-1,160,162,164,166,-1,-1,168,170,172,-1,-1,174,-1,-1,176,-1,-1,-1,178,-1,180,182,184,186,188,-1,190,-1,-1,-1,192,-1,-1,-1,-1,194,-1,196,198,200,202,-1,204,206,-1,-1,-1,208,-1,210,-1,212,-1,214,216,-1,-1,-1,-1,-1,-1,218,-1,220,-1,-1,-1,-1,-1,-1,222,224,226,228,230,-1,-1,-1,-1,-1,-1,232,-1,-1,-1,234,236,238,-1,-1,240,242,-1,-1,-1,-1,244,246,248,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,1.972052E5,1E0,9.9E1,9.31E2,2.3308511E5,8.625455E2,9.55188E5,8.462282E0,3.79E2,3.3358974E0,-3.4462553E-2,-1.2852693E-2,2.308821E6,5.489183E1,3.9125E1,-4.053686E-2,6.83E2,7.3E1,1E0,3.3486558E6,4.371613E2,2E1,1E0,1E0,1.7379E4,1.2083E4,-3.4822058E-2,5.2755904E0,7.997723E6,2.7E1,8.317E3,-2.5928061E-2,3E0,3.7043128E5,1E0,3.5364E4,6.0052995E6,1E0,1.9E1,4.1E1,1E0,8.920169E6,1.0410659E5,9.4247125E-3,1E0,6.682793E-2,2.8016653E3,2.707256E7,-2.3135023E-2,-1.0750364E-2,1.2E1,1.2E1,2.403337E-2,2.694192E9,-3.447157E-2,-6.998315E-2,-1.7853767E-2,-8.329051E-3,-1.5855588E-2,-3.0953916E-2,7.4142776E7,7.814751E6,3.3832976E7,1.594941E3,1.0292E4,1.9913E4,5.460753E9,4.5501604E-3,1.4473684E-1,4.591837E0,1.4081731E1,-3.0581964E-2,-1.3022984E-2,3.414E3,8E0,-9.681204E-3,2.2284646E-2,3.5152428E-2,2.1089442E0,4.797448E8,2.0195067E-2,6.0435697E-2,1.3388E4,3.0070068E7,2.95E2,2.2E0,-3.5155436E-3,-1.3384149E-2,3.1158695E6,8.57E2,8.2E1,1.027972E0,1.5132743E0,8E0,6.34E2,2.6732E4,1.3683688E7,1.5E1,1.1160929E0,1.007E3,2.8689932E-2,1.4543351E-2,1.710361E-2,3.013517E-2,-4.9161958E-3,7.709291E7,8.847733E4,3.4444444E0,-6.3914515E-4,9.936263E6,-3.1393045E-3,3.197293E-3,4.2010452E2,4.3333566E2,9.2789966E-1,9.5E2,1.2540972E-2,2.8521437E-2,3.6440072E0,2.7317073E0,6.619098E8,-1.0555592E-2,-2.7661588E-2,3E1,-3.688066E-2,-2.1333795E-2,1.4285715E0,6.15445E-4,-2.8207693E-2,-1.01252515E-2,2.1651703E2,1.340996E-3,3.8E2,5.4E0,2.732381E2,7.619497E0,1.7288135E0,1.1727069E-2,1E0,-4.050449E-3,6.199278E-3,1.9432344E-2,2.1557376E0,1.8172765E-3,7.148093E-3,-5.885711E-6,1.937808E-2,4.0449125E5,-8.527477E-3,3.1096E4,4.135135E0,2.0158867E8,3.0291306E2,-1.2465415E-3,1.390602E8,2.7012987E1,-1.4112135E-2,-5.589379E-3,1.0473481E-2,5.7819215E6,-3.731076E-5,3.1530054E0,2.4195883E-2,1.2820834E7,1.1438845E-2,2.5247778E6,6.439711E1,3.48971E-3,1.5414384E-2,3.317483E-2,1.0831631E-3,1.4571091E-2,-1.8158314E-3,1.432632E6,-1.2284252E-3,2.1340163E8,-1.5851019E-2,-6.814049E-3,-1.6526833E-2,-1.0469066E-3,-4.748191E-2,-1.7853165E-2,1.01417E0,4.0481758E3,2.884188E2,1.339646E6,1.4E1,-4.375171E-3,8.708977E-3,2.2654105E-3,-1.2472493E-2,-1.6801341E-3,5.627967E-4,2.2734E4,1.1876486E-3,6.9611794E-3,-5.1907017E-3,1.0439024E1,8.995735E5,5E0,-1.5007516E-2,-5.4422948E-3,5.617715E6,9E1,-3.568707E-4,-1.1103784E-2,2.631044E-2,1.0885399E-2,8.4797815E2,2.427869E8,3.4332926E0,3.504616E-2,2.376837E-2,3.8961507E-2,2.3762712E-2,1.0358896E-2,-2.0228496E-2,-8.560411E-3,-1.428076E-2,-3.418675E-3,1.5080068E-3,-8.294648E-3,2.0374921E-4,5.2380306E-3,-7.723287E-3,8.1116444E-4,-1.4845412E-2,-7.698857E-3,5.2661817E-3,-2.280954E-4,2.2682855E-3,9.481751E-3,1.0832815E-2,2.8570371E-3,8.336831E-3,1.5360508E-2,1.07226E-2,4.919231E-4,1.0295539E-3,1.0005695E-2,3.7886964E-3,-3.0204777E-3,7.2013107E-3,1.4736199E-2,1.6959865E-3,7.449649E-3,5.29985E-2,2.8122451E-2],"split_indices":[2,43,17,44,2,48,67,46,69,2,69,0,0,43,71,4,0,2,0,26,43,67,3,27,6,2,9,0,69,9,0,9,0,8,43,29,9,43,6,8,3,26,62,48,0,6,0,4,60,0,0,8,3,0,46,0,0,0,0,0,0,5,60,7,48,44,9,46,0,71,69,73,0,0,2,32,0,0,0,53,7,0,0,2,58,2,68,0,0,60,2,0,68,68,8,2,1,60,3,69,2,0,0,0,0,0,7,48,68,0,60,0,0,67,67,68,10,0,0,68,69,7,0,0,0,0,0,71,0,0,0,67,0,12,73,4,73,68,0,28,0,0,0,69,0,0,0,0,43,0,9,69,7,67,0,7,73,0,0,0,66,0,68,0,9,0,43,73,0,0,0,0,0,0,12,0,7,0,0,0,0,0,0,68,48,67,9,3,0,0,0,0,0,0,9,0,0,0,73,43,3,0,0,12,8,0,0,0,0,4,7,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,7.24E2,2.96E2,2.41E2,4.83E2,1.1E1,2.85E2,8.3E1,1.58E2,2.48E2,2.35E2,7E0,4E0,1.03E2,1.82E2,4.9E1,3.4E1,1.41E2,1.7E1,4.3E1,2.05E2,1.05E2,1.3E2,6.6E1,3.7E1,1.13E2,6.9E1,3.2E1,1.7E1,8.6E1,5.5E1,9E0,8E0,2.9E1,1.4E1,1.58E2,4.7E1,6.9E1,3.6E1,1.1E2,2E1,1.6E1,5E1,3.3E1,4E0,9.4E1,1.9E1,9E0,6E1,1.3E1,4E0,7.6E1,1E1,8E0,4.7E1,5E0,4E0,2.2E1,7E0,4E0,1E1,3.9E1,1.19E2,2.5E1,2.2E1,6E1,9E0,3E1,6E0,6.7E1,4.3E1,1.6E1,4E0,6E0,1E1,4.6E1,4E0,1.5E1,1.8E1,8.1E1,1.3E1,4E0,5E0,4.4E1,1.6E1,5.9E1,1.7E1,4E0,6E0,3.8E1,9E0,2E1,1.9E1,9.7E1,2.2E1,1.6E1,9E0,1.4E1,8E0,4.8E1,1.2E1,4E0,5E0,1.7E1,1.3E1,6E0,6.1E1,1.2E1,3.1E1,5E0,1.1E1,5E0,5E0,1.9E1,2.7E1,4.7E1,3.4E1,6E0,7E0,2E1,2.4E1,1E1,6E0,3.1E1,2.8E1,1E1,7E0,3.2E1,6E0,5E0,4E0,1.6E1,4E0,9E0,1E1,3E1,6.7E1,1.8E1,4E0,1.2E1,4E0,4E0,5E0,1E1,4E0,4E0,4E0,4E0,4.4E1,4E0,8E0,2E1,4.1E1,8E0,4E0,2.2E1,9E0,6E0,5E0,6E0,1.3E1,6E0,2.1E1,4E0,4.3E1,5E0,2.9E1,1.5E1,5E0,4E0,2E1,4E0,6E0,4E0,2.4E1,6E0,2.6E1,5E0,1.1E1,5E0,4E0,5E0,5E0,1.5E1,1.5E1,3.3E1,3.4E1,1.2E1,6E0,8E0,4E0,5E0,5E0,5E0,3.9E1,4E0,4E0,6E0,1.4E1,2.9E1,1.2E1,4E0,4E0,1.2E1,1E1,5E0,4E0,9E0,4E0,1.3E1,8E0,2.9E1,1.4E1,1E1,1.9E1,9E0,6E0,2E1,4E0,2.1E1,5E0,6E0,9E0,5E0,1E1,2.3E1,1E1,1.8E1,1.6E1,8E0,4E0,4E0,3.5E1,4E0,1E1,1.3E1,1.6E1,4E0,8E0,4E0,8E0,5E0,5E0,6E0,7E0,4E0,4E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"249","size_leaf_vector":"1"}},{"base_weights":[6.798278E-3,-2.9250383E-1,3.2333016E-1,-5.4509145E-1,-1.3893844E-1,1.4397086E-1,6.3014203E-1,-7.511041E-1,-4.2006698E-1,-3.5289416E-1,-5.0119236E-2,-4.0301904E-2,1.9171517E-1,7.1512866E-1,2.2618681E-2,-9.9959886E-1,-6.9973826E-1,-6.033013E-1,-2.9881284E-1,-4.2399275E-1,-1.2785693E-1,-1.0241604E-1,8.260068E-2,4.0933564E-2,1.591436E-1,6.866623E-2,6.468455E-1,-2.665817E-1,3.099342E-1,-2.5027623E-2,-5.611118E-2,-3.5596162E-2,-5.5265373E-1,-3.6661573E-2,-5.016208E-1,2.8036669E-2,-3.575571E-1,-3.2117665E-1,-5.392631E-1,-1.8218723E-1,5.3145397E-3,-1.884132E-1,-2.8034728E-3,-1.7533407E-1,1.3838987E-1,-9.642109E-2,2.6033372E-1,2.0197384E-1,6.962432E-1,-1.9054212E-2,9.287665E-4,-6.836584E-3,2.5750862E-2,-1.0586381E-2,-2.8907977E-2,-1.20543E-2,-5.250168E-1,-2.9863966E-1,-5.8031213E-1,-3.5235187E-1,-3.9635E-3,-4.5235357E-1,-3.8829327E-2,-9.630789E-2,-2.584358E-1,-2.783935E-1,-3.0009903E-2,-9.539431E-2,5.8326732E-2,-1.5262059E-2,4.978653E-3,9.87916E-2,1.6342035E-2,-1.3543558E-2,-5.288925E-1,2.2214657E-1,2.958916E-2,-1.2322253E-3,3.418286E-1,5.4083705E-2,6.473597E-1,-2.6804253E-2,-1.5329408E-2,-1.705044E-1,-3.7021998E-1,-3.2409858E-2,-1.8558647E-2,-3.7659407E-1,-6.078864E-3,-2.2322569E-2,-1.2192952E-2,-6.845971E-3,-1.3111394E-3,-1.463086E-2,-6.321943E-3,-1.8398893E-1,-3.9660004E-1,9.742137E-2,-1.8942219E-1,2.3122365E-2,-1.529319E-1,-1.7521594E-2,1.932074E-1,1.3263568E-1,-5.8842912E-2,-1.347676E-1,8.889626E-2,-1.2324528E-2,-3.0462394E-2,2.4081929E-1,-1.3385908E-2,2.0845033E-2,6.5702763E-3,4.8914114E-1,7.5950795E-1,-9.660245E-2,-1.6806886E-2,-2.7759965E-3,-4.059909E-1,-2.3605634E-2,-1.4212878E-2,-1.0439892E-1,-2.8454685E-1,-2.4071284E-2,-2.705018E-1,-2.6948992E-3,8.922826E-3,-1.0214229E-2,-3.782071E-3,7.119939E-3,-3.4070218E-3,-3.6927234E-4,-1.9772571E-1,-6.125142E-2,5.6821585E-2,-1.4373868E-4,2.538525E-1,8.0264695E-2,1.068138E-2,-8.271929E-3,3.3786534E-3,8.9172255E-3,-2.3354761E-1,1.17935985E-1,-3.956408E-3,3.507249E-1,1.3423313E-1,5.385398E-1,1.8779878E-1,8.2583904E-1,2.8557053E-1,-1.4705496E-1,1.7037791E-4,-4.6494633E-1,-2.2389881E-1,-7.8066083E-4,-1.5021509E-1,-6.403517E-3,-1.5202354E-2,-1.599685E-2,-2.9976934E-3,-2.386771E-1,-2.7671948E-3,2.7382474E-3,-9.508635E-2,-9.336852E-4,4.521208E-3,4.5469464E-3,1.4413992E-2,3.927989E-2,6.297303E-3,5.585332E-3,-6.672683E-2,-3.1092018E-1,-4.079159E-3,1.4590019E-1,-3.1294962E-4,3.9253423E-1,-1.4641273E-2,8.984514E-2,3.7829068E-1,8.155576E-3,5.6955206E-1,1.1485854E-2,4.1311714E-3,2.4195999E-2,4.1874755E-2,-4.9958057E-3,2.615165E-2,-1.2944418E-3,-9.381672E-3,-1.5504474E-2,-2.4640905E-2,-1.536285E-2,-4.437479E-3,-3.7726064E-3,-1.1252776E-2,-5.464779E-3,-1.3124496E-2,-7.428129E-4,-5.984501E-3,-1.7113466E-4,5.483901E-3,-3.7516936E-4,-5.1737204E-3,-8.49017E-3,-1.7347082E-2,2.1123588E-3,9.338601E-3,1.2624343E-2,2.470133E-2,-6.8272697E-3,4.5620366E-3,-6.8897502E-3,5.6473007E-3,2.1985939E-2,4.9627996E-3,2.7737483E-2,1.5430196E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,-1,-1,53,-1,55,-1,57,59,61,63,-1,65,67,69,71,73,75,77,79,-1,-1,-1,-1,-1,-1,-1,81,83,85,87,-1,89,-1,91,93,95,97,99,101,-1,-1,103,-1,105,107,109,-1,-1,111,-1,113,-1,-1,115,117,-1,-1,119,-1,-1,-1,-1,-1,-1,-1,121,123,125,127,129,131,133,135,137,139,141,143,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,153,-1,-1,155,157,-1,159,-1,-1,-1,-1,-1,-1,-1,161,163,165,-1,167,169,-1,-1,-1,171,173,175,-1,177,179,181,183,185,187,189,-1,191,193,-1,195,-1,-1,-1,-1,197,-1,-1,199,-1,-1,-1,-1,201,-1,-1,203,205,-1,207,-1,209,211,213,215,-1,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.4738495E1,1.9861668E1,2.665364E1,4.6812706E0,6.0823965E0,1.5112545E1,9.22583E0,2.1956253E-1,2.5525303E0,1.4497738E0,1.5909684E0,0E0,6.530362E0,7.3060303E0,1.9937334E0,3.26231E-1,2.2984123E-1,5.3050995E-1,4.2604527E0,6.6826916E-1,3.3767235E-1,1.4043611E0,9.596126E-1,0E0,7.3436575E0,0E0,3.0678177E0,5.100673E-1,1.4422456E0,0E0,0E0,0E0,4.2396593E-1,0E0,5.44796E-2,0E0,7.716341E-1,2.6329756E-1,4.5751858E-1,1.06074035E-1,0E0,1.2459252E0,4.4139153E-1,5.622345E-1,4.255196E-1,2.8960526E0,2.8045444E0,5.207529E-1,2.280243E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2728119E-1,4.7802162E-1,2.1276474E-2,1.3776827E-1,0E0,2.9323578E-2,0E0,3.455674E-2,2.7499557E-2,5.519471E-1,6.8881947E-1,2.1493018E-1,4.8421752E-1,0E0,0E0,2.5854942E-1,0E0,8.688341E-1,2.687223E-1,1.8476362E0,0E0,0E0,1.5415096E-1,0E0,1.7755051E0,0E0,0E0,2.933979E-1,3.8607264E-1,0E0,0E0,1.299963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.34406E-1,2.3564172E-1,2.9452467E-1,3.2994866E-2,1.5094508E-1,1.3475919E-1,1.04261845E-1,2.0318782E-1,1.8276405E-1,1.5587352E-1,4.5626467E-1,2.027004E-1,0E0,0E0,2.0615168E0,0E0,0E0,0E0,6.8781567E-1,1.9333076E0,9.162052E-2,0E0,0E0,2.501645E-1,0E0,0E0,7.7641055E-2,4.5341134E-2,0E0,1.9560897E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.6012194E-2,8.719295E-2,3.9916288E-2,0E0,8.678162E-2,5.6662843E-2,0E0,0E0,0E0,1.2665567E-1,1.860795E-1,1.1607301E-1,0E0,1.3500109E0,9.7350204E-1,4.0465736E-1,3.161651E-2,9.8553467E-1,1.1655376E0,7.4324995E-2,0E0,3.338909E-2,1.0694945E-1,0E0,6.331104E-2,0E0,0E0,0E0,0E0,3.1996906E-2,0E0,0E0,4.1467562E-2,0E0,0E0,0E0,0E0,5.8475614E-2,0E0,0E0,2.2963595E-2,2.9507041E-2,0E0,1.4286971E-1,0E0,1.1932125E0,1.6555646E-1,6.3753015E-1,3.1901288E-1,0E0,9.731102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,26,26,27,27,28,28,32,32,34,34,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,56,56,57,57,58,58,59,59,61,61,63,63,64,64,65,65,66,66,67,67,68,68,71,71,73,73,74,74,75,75,78,78,80,80,83,83,84,84,87,87,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,109,109,113,113,114,114,115,115,118,118,121,121,122,122,124,124,132,132,133,133,134,134,136,136,137,137,141,141,142,142,143,143,145,145,146,146,147,147,148,148,149,149,150,150,151,151,153,153,154,154,156,156,161,161,164,164,169,169,172,172,173,173,175,175,177,177,178,178,179,179,180,180,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,-1,-1,54,-1,56,-1,58,60,62,64,-1,66,68,70,72,74,76,78,80,-1,-1,-1,-1,-1,-1,-1,82,84,86,88,-1,90,-1,92,94,96,98,100,102,-1,-1,104,-1,106,108,110,-1,-1,112,-1,114,-1,-1,116,118,-1,-1,120,-1,-1,-1,-1,-1,-1,-1,122,124,126,128,130,132,134,136,138,140,142,144,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,154,-1,-1,156,158,-1,160,-1,-1,-1,-1,-1,-1,-1,162,164,166,-1,168,170,-1,-1,-1,172,174,176,-1,178,180,182,184,186,188,190,-1,192,194,-1,196,-1,-1,-1,-1,198,-1,-1,200,-1,-1,-1,-1,202,-1,-1,204,206,-1,208,-1,210,212,214,216,-1,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,4.27E3,3.557841E7,5.02E2,1E0,2.856934E7,2E0,3.06E2,9.860918E5,2.0766992E6,-4.0301904E-2,1.2E1,3.6E1,1.3E1,1.5E1,3.62E2,3.89E2,2E0,4.325E0,4.0705118E6,1.559733E6,5.8899284E1,4.0933564E-2,4.8709216E5,6.866623E-2,3.257507E5,4.9236734E5,2.0618556E-2,-2.5027623E-2,-5.611118E-2,-3.5596162E-2,1.0005E5,-3.6661573E-2,6.695E3,2.8036669E-2,1.9E1,4.67E2,7.668863E6,3.429012E7,5.3145397E-3,3.4734247E2,4.8709216E5,1.7010834E3,7.168071E6,2.3E1,5.405423E6,1.8452264E6,1.0918E4,-1.9054212E-2,9.287665E-4,-6.836584E-3,2.5750862E-2,-1.0586381E-2,-2.8907977E-2,-1.20543E-2,3.201807E6,3.464E0,6.81531E-2,2.241E2,-3.9635E-3,1.9637654E2,-3.8829327E-2,1.0865825E0,6.7723384E7,3.6981132E0,1.798E3,2.812736E5,9.24E2,-1.5262059E-2,4.978653E-3,2.8302418E10,1.6342035E-2,3.6161574E4,4E1,1.6776881E7,2.958916E-2,-1.2322253E-3,1.8996E4,5.4083705E-2,7.098E3,-2.6804253E-2,-1.5329408E-2,1.1769393E5,2.3636363E0,-3.2409858E-2,-1.8558647E-2,1.1967312E3,-6.078864E-3,-2.2322569E-2,-1.2192952E-2,-6.845971E-3,-1.3111394E-3,-1.463086E-2,-6.321943E-3,1E0,6.176636E5,8.23E2,1.835821E1,2.0939393E0,1.5599597E8,2.9427118E2,5.1E1,5.1586456E7,1E1,3.82E2,4.2904656E8,-1.2324528E-2,-3.0462394E-2,3.464E0,-1.3385908E-2,2.0845033E-2,6.5702763E-3,4.3657026E0,1.9500381E2,1.3009709E0,-1.6806886E-2,-2.7759965E-3,3.53274E2,-2.3605634E-2,-1.4212878E-2,2.5454545E0,2.4645097E2,-2.4071284E-2,1.0563E4,-2.6948992E-3,8.922826E-3,-1.0214229E-2,-3.782071E-3,7.119939E-3,-3.4070218E-3,-3.6927234E-4,3.990487E5,1.2051282E0,7.24E2,-1.4373868E-4,1.977157E7,3.235955E0,1.068138E-2,-8.271929E-3,3.3786534E-3,5.513889E0,3.3238492E2,3.687715E6,-3.956408E-3,6.929741E8,8.5023944E2,1E0,4.75356E6,1.3543621E6,9.132011E6,1E0,1.7037791E-4,8.91E2,2.0939393E0,-7.8066083E-4,1.7755102E0,-6.403517E-3,-1.5202354E-2,-1.599685E-2,-2.9976934E-3,6.789622E7,-2.7671948E-3,2.7382474E-3,1.98E2,-9.336852E-4,4.521208E-3,4.5469464E-3,1.4413992E-2,1.0404834E7,6.297303E-3,5.585332E-3,1.4E1,2.35184E5,-4.079159E-3,3.53274E2,-3.1294962E-4,5.372237E2,5E0,2.1924414E2,1.2979348E3,8.155576E-3,1.766486E7,1.1485854E-2,4.1311714E-3,2.4195999E-2,4.1874755E-2,-4.9958057E-3,2.615165E-2,-1.2944418E-3,-9.381672E-3,-1.5504474E-2,-2.4640905E-2,-1.536285E-2,-4.437479E-3,-3.7726064E-3,-1.1252776E-2,-5.464779E-3,-1.3124496E-2,-7.428129E-4,-5.984501E-3,-1.7113466E-4,5.483901E-3,-3.7516936E-4,-5.1737204E-3,-8.49017E-3,-1.7347082E-2,2.1123588E-3,9.338601E-3,1.2624343E-2,2.470133E-2,-6.8272697E-3,4.5620366E-3,-6.8897502E-3,5.6473007E-3,2.1985939E-2,4.9627996E-3,2.7737483E-2,1.5430196E-2],"split_indices":[2,43,2,7,2,17,60,32,2,43,43,0,33,0,3,3,1,1,10,69,43,9,62,0,43,0,43,43,72,0,0,0,5,0,9,0,3,2,5,7,0,67,43,48,43,3,43,60,9,0,0,0,0,0,0,0,9,69,53,4,0,67,0,71,7,71,44,43,2,0,0,46,0,48,3,62,0,0,44,0,2,0,0,43,73,0,0,48,0,0,0,0,0,0,0,26,43,2,71,68,46,67,10,7,8,0,7,0,0,69,0,0,0,69,73,68,0,0,4,0,0,73,4,0,44,0,0,0,0,0,0,0,43,68,2,0,5,71,0,0,0,69,67,62,0,5,67,15,62,43,1,29,0,44,68,0,68,0,0,0,0,7,0,0,10,0,0,0,0,60,0,0,3,1,0,4,0,4,6,67,4,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,5.13E2,4.85E2,1.93E2,3.2E2,3.07E2,1.78E2,7.1E1,1.22E2,9.3E1,2.27E2,1.3E1,2.94E2,1.56E2,2.2E1,9E0,6.2E1,4.7E1,7.5E1,7E1,2.3E1,1.63E2,6.4E1,1.2E1,2.82E2,1.1E1,1.45E2,1.1E1,1.1E1,4E0,5E0,3.9E1,2.3E1,1.4E1,3.3E1,4E0,7.1E1,3.9E1,3.1E1,1.9E1,4E0,8.7E1,7.6E1,1.1E1,5.3E1,8E1,2.02E2,1.5E1,1.3E2,7E0,4E0,4E0,7E0,5E0,1.8E1,4E0,2.9E1,5.8E1,1.3E1,3.4E1,5E0,2.6E1,5E0,1E1,9E0,5.5E1,3.2E1,3E1,4.6E1,7E0,4E0,4.6E1,7E0,6.8E1,1.2E1,1.85E2,1.7E1,6E0,9E0,1E1,1.2E2,2.1E1,8E0,2.2E1,3.6E1,6E0,7E0,3E1,4E0,2.1E1,5E0,5E0,5E0,5E0,4E0,3.2E1,2.3E1,1.8E1,1.4E1,1E1,2E1,3E1,1.6E1,3.8E1,8E0,3.1E1,3.7E1,5E0,7E0,1.79E2,6E0,5E0,4E0,5.2E1,6.8E1,1.7E1,5E0,4E0,3.2E1,8E0,2.2E1,1.9E1,1.3E1,1E1,1.3E1,7E0,1.1E1,1E1,4E0,4E0,6E0,5E0,1.5E1,1.9E1,1.1E1,4E0,1.2E1,2.6E1,1.2E1,4E0,4E0,1.3E1,1.8E1,3.2E1,5E0,8.7E1,9.2E1,4.4E1,8E0,5.9E1,9E0,1.1E1,6E0,2.3E1,9E0,7E0,1.2E1,4E0,9E0,9E0,4E0,1.1E1,4E0,4E0,1.5E1,4E0,7E0,4E0,8E0,1.6E1,1E1,5E0,8E0,1.1E1,7E0,2.6E1,6E0,7.8E1,9E0,7.9E1,1.3E1,4E0,4E1,4E0,4E0,1.4E1,4.5E1,4E0,5E0,4E0,7E0,1E1,1.3E1,4E0,5E0,8E0,4E0,4E0,7E0,5E0,1E1,1.1E1,5E0,4E0,4E0,5E0,6E0,1E1,1.6E1,4.4E1,3.4E1,4E0,5E0,9E0,7E1,9E0,4E0,3.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-9.604084E-3,-1.7769752E-1,5.3377515E-1,-5.027555E-1,-1.1044416E-2,5.953061E-1,-5.064273E-1,-6.957941E-1,-3.919236E-1,-1.5918878E-1,1.365706E-1,4.4329765E-1,8.657004E-1,-3.2116305E-2,-1.0088568E-2,-6.123844E-1,-8.735108E-1,-4.9474064E-1,-9.532652E-2,-2.2376442E-1,8.322249E-2,2.804698E-1,2.5025155E-2,1.8223284E-1,6.0022587E-1,1.1787167E0,6.684661E-1,-3.4130882E-2,-3.930898E-1,-5.0244346E-2,-6.9417316E-1,-4.3455964E-1,-9.8442465E-1,3.1324722E-2,-2.0894806E-1,-2.0233789E-1,-4.786559E-2,3.7036147E-2,3.1517094E-1,3.03033E-1,-7.965279E-3,4.1885342E-2,-2.1352336E-2,3.462123E-2,2.964725E-1,4.8907918E-1,8.248178E-1,1.2883264E0,3.3090036E-2,7.35882E-1,2.6460323E-1,-5.182883E-1,1.2078336E-2,-1.9782528E-2,-3.65654E-2,-2.3568341E-1,-4.9656954E-1,-1.9125406E-2,-1.1919587E0,-1.1826896E-1,-1.704303E-2,-3.6924347E-1,-1.2643513E-1,-6.714747E-3,5.7381816E-2,1.921139E-2,6.9976863E-3,2.2981884E-1,4.318533E-1,-1.08133584E-1,8.7874055E-2,-8.603366E-3,8.323961E-2,1.94815E-1,1.9868504E-2,5.381012E-1,2.0396389E-1,4.410762E-2,2.7326044E-2,3.589514E-2,6.4087585E-2,8.632517E-1,5.22856E-1,1.7521976E-2,4.4819764E-3,-2.7962103E-2,-3.1575984E-1,-3.2417864E-1,1.1731296E-2,-5.764673E-1,-3.855013E-1,-6.996596E-2,-2.9155284E-2,-3.753598E-2,-2.3228638E-1,-1.3757024E-3,-4.1190678E-1,-1.0416744E-1,-2.2085348E-2,1.3092345E-1,2.4690751E-2,1.690638E-1,2.521731E-2,4.6240836E-1,6.361989E-3,-1.82799E-1,8.934329E-2,-1.0108118E-1,1.2350115E-1,1.4519924E-1,-3.2465826E-3,2.4155864E-1,3.3954915E-3,6.183139E-1,2.883272E-1,1.7264051E-3,1.3574328E-2,2.0680983E-2,9.18468E-1,3.2642597E-1,2.93844E-2,-1.8470675E-2,-7.7873715E-3,-4.2645237E-1,-2.4735157E-3,-4.7742063E-1,-3.3033323E-2,-2.4655715E-1,-4.4549042E-1,2.4755226E-2,-9.565652E-2,-1.5569443E-2,-5.742614E-3,-2.6252744E-1,-4.7748247E-1,-6.2056348E-2,-2.1061331E-1,3.3671854E-3,8.615587E-3,-2.0534683E-2,9.800891E-2,-3.6841254E-3,1.8792307E-1,9.079898E-3,5.0147074E-1,2.8283142E-3,-2.468648E-1,-2.4922558E-3,8.956172E-3,-1.089296E-2,6.0731443E-3,8.775925E-2,3.5744572E-1,-3.2624108E-4,2.1505071E-1,4.6179355E-3,1.4281034E-2,8.376324E-3,6.618336E-1,7.028194E-3,1.7947633E-2,4.588095E-2,2.1530917E-2,2.0102886E-2,7.0421374E-3,-2.2482986E-2,-7.973739E-3,-2.3983054E-2,-1.0031399E-2,-1.3448424E-2,-4.9971286E-3,-9.1125015E-3,-4.816604E-1,-1.1962886E-3,3.2548907E-3,-6.571131E-3,-1.3835018E-3,-3.20505E-1,-5.4215016E-3,-1.1348064E-2,-5.432325E-1,-1.04204014E-1,4.9513765E-2,-1.1366056E-1,-1.2545401E-2,-4.2906925E-3,2.5561439E-2,6.4209225E-3,5.9352885E-4,1.505344E-1,3.1558603E-1,2.4542887E-2,1.1656909E-2,-1.3030397E-2,-3.9224033E-3,5.7654423E-3,-3.6772718E-3,1.1004653E-1,-6.1053853E-3,2.3943515E-2,9.805783E-3,4.3220334E-3,1.2525193E-2,2.3565728E-2,3.500608E-2,-2.5173325E-2,-1.2396623E-2,-1.926584E-2,-9.612037E-3,-3.0312736E-2,-1.771572E-2,-2.2826523E-3,-9.571008E-3,9.623021E-3,-7.610982E-4,-9.670485E-3,-7.391419E-4,3.0173867E-3,-1.4645348E-3,7.753292E-3,3.3735766E-4,2.1841483E-2,6.2726284E-3,1.3855398E-2,3.977356E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,53,55,57,-1,59,61,-1,63,65,67,-1,69,-1,71,73,75,77,79,-1,81,83,85,-1,-1,-1,87,89,-1,91,93,-1,95,97,-1,99,-1,-1,101,103,105,107,-1,109,111,-1,113,115,-1,-1,-1,-1,117,119,-1,-1,-1,121,123,-1,125,127,-1,-1,129,131,-1,133,135,-1,137,139,141,-1,143,-1,145,147,149,151,153,-1,155,-1,157,159,-1,-1,-1,161,163,-1,-1,-1,165,-1,167,-1,169,171,173,175,-1,-1,177,179,181,183,-1,-1,185,187,-1,189,-1,191,-1,193,-1,-1,-1,195,197,199,-1,201,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,205,-1,-1,-1,-1,207,-1,-1,209,211,213,215,-1,-1,217,-1,-1,219,221,-1,-1,-1,-1,-1,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.316876E1,4.2194374E1,1.5726601E1,5.3755417E0,1.13058E1,8.987541E0,6.0936093E-1,8.73291E-1,5.1370296E0,4.0573187E0,4.1502686E0,5.953951E0,4.2128067E0,0E0,0E0,1.5943947E0,3.198948E-1,3.2853851E0,4.0958652E0,3.3409758E0,5.7753414E-1,1.2069321E0,1.2268742E0,9.509324E-1,1.8931618E0,5.979614E-1,1.221077E0,0E0,2.3480525E0,0E0,1.2691975E-1,1.2888489E0,1.0455866E0,0E0,5.365578E-1,2.487033E0,0E0,1.8274027E-1,8.533102E-2,9.193506E-1,0E0,9.960845E-1,0E0,2.9466662E-1,3.5079265E-1,7.8477E-1,3.1436348E-1,8.51326E-2,0E0,7.424984E-1,1.3615894E-1,2.1397972E-1,0E0,0E0,0E0,1.3388417E0,5.4094315E-1,0E0,6.721506E-1,2.3981497E-1,0E0,8.8566494E-1,1.0385175E0,0E0,1.00146025E-1,0E0,0E0,1.2709115E0,2.7785206E-1,5.1765144E-1,7.5299567E-1,0E0,2.1945965E-1,9.566301E-2,0E0,9.036169E-1,1.5369049E-1,0E0,0E0,0E0,0E0,1.8549728E-1,2.1479893E-1,0E0,0E0,0E0,4.395187E-2,6.69147E-1,0E0,3.1735516E-1,2.1523142E-1,0E0,0E0,6.3914776E-2,8.2500994E-2,0E0,4.1340733E-1,5.786269E-1,0E0,2.8332159E-2,1.04803786E-1,3.0121863E-1,0E0,2.5454903E-1,0E0,4.0594524E-1,1.6022004E-1,2.6124227E-1,7.4626195E-1,1.71689E-1,0E0,1.0303539E-1,0E0,6.407356E-1,1.4552057E-1,0E0,0E0,0E0,2.8013992E-1,1.039083E-1,0E0,0E0,0E0,1.9109583E-1,0E0,2.2378206E-1,0E0,5.480963E-2,1.3845062E-1,2.2655735E-2,2.3286693E-2,0E0,0E0,1.22310996E-1,4.3059254E-1,4.5659184E-1,1.8773448E-1,0E0,0E0,6.906527E-2,4.0088408E-2,0E0,2.2456312E-1,0E0,8.106756E-2,0E0,9.0023994E-2,0E0,0E0,0E0,1.2181306E-1,4.1644692E-1,1.7020512E-1,0E0,4.9563766E-2,0E0,0E0,0E0,1.3730907E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.505351E-1,0E0,0E0,0E0,0E0,5.4321647E-2,0E0,0E0,2.5345707E-1,3.8470095E-1,2.8632823E-1,1.4613031E-1,0E0,0E0,3.1661548E-2,0E0,0E0,1.1268771E-1,2.7497137E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.1528503E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,34,34,35,35,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,55,55,56,56,58,58,59,59,61,61,62,62,64,64,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,81,81,82,82,86,86,87,87,89,89,90,90,93,93,94,94,96,96,97,97,99,99,100,100,101,101,103,103,105,105,106,106,107,107,108,108,109,109,111,111,113,113,114,114,118,118,119,119,123,123,125,125,127,127,128,128,129,129,130,130,133,133,134,134,135,135,136,136,139,139,140,140,142,142,144,144,146,146,150,150,151,151,152,152,154,154,158,158,172,172,177,177,180,180,181,181,182,182,183,183,186,186,189,189,190,190,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,54,56,58,-1,60,62,-1,64,66,68,-1,70,-1,72,74,76,78,80,-1,82,84,86,-1,-1,-1,88,90,-1,92,94,-1,96,98,-1,100,-1,-1,102,104,106,108,-1,110,112,-1,114,116,-1,-1,-1,-1,118,120,-1,-1,-1,122,124,-1,126,128,-1,-1,130,132,-1,134,136,-1,138,140,142,-1,144,-1,146,148,150,152,154,-1,156,-1,158,160,-1,-1,-1,162,164,-1,-1,-1,166,-1,168,-1,170,172,174,176,-1,-1,178,180,182,184,-1,-1,186,188,-1,190,-1,192,-1,194,-1,-1,-1,196,198,200,-1,202,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,206,-1,-1,-1,-1,208,-1,-1,210,212,214,216,-1,-1,218,-1,-1,220,222,-1,-1,-1,-1,-1,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.2116136E5,4.217427E7,9.9E1,9.31E2,1.5142261E3,1.0201538E-1,1.4504054E8,8.57E2,2.331083E6,3.21875E0,9.155886E5,2.3601625E1,-3.2116305E-2,-1.0088568E-2,3.89E2,3.3817584E7,9.932432E0,2.04115E5,1.1290322E1,7.168071E6,1.4676277E7,2.673585E1,4.032683E6,4.391553E6,1.4595818E7,2.856934E7,-3.4130882E-2,1.1634076E4,-5.0244346E-2,4.217427E7,1.3208092E0,2.49E2,3.1324722E-2,1.6942337E1,1.7934177E2,-4.786559E-2,3.72E2,3.52E2,5.144203E2,-7.965279E-3,1.221E3,-2.1352336E-2,8.295515E7,1.29443414E5,2E0,1.4598765E0,1E1,3.3090036E-2,2.171312E0,2.2365898E3,1.9363927E-6,1.2078336E-2,-1.9782528E-2,-3.65654E-2,6.47E2,1.339646E6,-1.9125406E-2,6.8E1,4.3E1,-1.704303E-2,9E0,1.7479E4,-6.714747E-3,1.2E1,1.921139E-2,6.9976863E-3,6.0052995E6,2E0,1.5115256E6,2.4080281E2,-8.603366E-3,5.8485405E1,6.364486E0,1.9868504E-2,2E0,8E0,4.410762E-2,2.7326044E-2,3.589514E-2,6.4087585E-2,1.8697667E3,7.958E3,1.7521976E-2,4.4819764E-3,-2.7962103E-2,5.5E1,1.302273E6,1.1731296E-2,1.88E2,9.263158E0,-6.996596E-2,-2.9155284E-2,4.046E3,5.2767E4,-1.3757024E-3,3.2E1,2.1851852E0,-2.2085348E-2,1.1E1,2.7840833E2,3.7760816E5,2.521731E-2,2.9816E4,6.361989E-3,1.9095E4,4.7563504E7,2.0227273E0,2.295413E6,1.8471494E7,-3.2465826E-3,1.0784811E3,3.3954915E-3,2.0750147E8,8.5023944E2,1.7264051E-3,1.3574328E-2,2.0680983E-2,2.5842668E7,2E0,2.93844E-2,-1.8470675E-2,-7.7873715E-3,6.666667E-1,-2.4735157E-3,2.7976523E2,-3.3033323E-2,4.5087484E5,3.9961785E-2,2.7777777E0,1.1770261E8,-1.5569443E-2,-5.742614E-3,1E0,1.1010101E0,5E0,9.538462E0,3.3671854E-3,8.615587E-3,1.3071136E3,1E0,-3.6841254E-3,7.6569915E-1,9.079898E-3,1.8809018E0,2.8283142E-3,4.88E2,-2.4922558E-3,8.956172E-3,-1.089296E-2,1.1631579E1,9.639872E0,1.4E1,-3.2624108E-4,2.848973E6,4.6179355E-3,1.4281034E-2,8.376324E-3,6.255748E6,7.028194E-3,1.7947633E-2,4.588095E-2,2.1530917E-2,2.0102886E-2,7.0421374E-3,-2.2482986E-2,-7.973739E-3,-2.3983054E-2,-1.0031399E-2,-1.3448424E-2,-4.9971286E-3,-9.1125015E-3,1.5292561E2,-1.1962886E-3,3.2548907E-3,-6.571131E-3,-1.3835018E-3,8.3E1,-5.4215016E-3,-1.1348064E-2,4.795239E7,1E0,2.3786941E4,5.0857143E0,-1.2545401E-2,-4.2906925E-3,1.880118E-3,6.4209225E-3,5.9352885E-4,1.8E1,1.124641E0,2.4542887E-2,1.1656909E-2,-1.3030397E-2,-3.9224033E-3,5.7654423E-3,-3.6772718E-3,6.4722223E0,-6.1053853E-3,2.3943515E-2,9.805783E-3,4.3220334E-3,1.2525193E-2,2.3565728E-2,3.500608E-2,-2.5173325E-2,-1.2396623E-2,-1.926584E-2,-9.612037E-3,-3.0312736E-2,-1.771572E-2,-2.2826523E-3,-9.571008E-3,9.623021E-3,-7.610982E-4,-9.670485E-3,-7.391419E-4,3.0173867E-3,-1.4645348E-3,7.753292E-3,3.3735766E-4,2.1841483E-2,6.2726284E-3,1.3855398E-2,3.977356E-3],"split_indices":[2,43,60,44,2,67,53,46,2,43,69,43,71,0,0,1,7,69,5,69,43,62,68,60,43,62,60,0,43,0,60,68,2,0,71,70,0,44,0,4,0,2,0,7,48,6,69,3,0,57,67,52,0,0,0,2,9,0,0,8,0,3,44,0,3,0,0,43,6,43,67,0,73,69,0,32,8,0,0,0,0,67,2,0,0,0,0,9,0,10,73,0,0,44,1,0,10,68,0,3,67,43,0,1,0,12,7,68,43,5,0,4,0,7,67,0,0,0,62,32,0,0,0,71,0,4,0,60,53,71,7,0,0,8,68,8,73,0,0,48,26,0,57,0,57,0,4,0,0,0,73,69,3,0,66,0,0,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,67,0,0,0,0,12,0,0,7,26,62,69,0,0,53,0,0,3,53,0,0,0,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.78E2,2.4E2,2.63E2,5.15E2,2.27E2,1.3E1,9.4E1,1.69E2,2.57E2,2.58E2,1.47E2,8E1,7E0,6E0,6.7E1,2.7E1,1.25E2,4.4E1,2.03E2,5.4E1,1.12E2,1.46E2,5.6E1,9.1E1,2.9E1,5.1E1,4.1E1,2.6E1,1E1,1.7E1,1.13E2,1.2E1,5E0,3.9E1,1.99E2,4E0,4.6E1,8E0,1.07E2,5E0,1.42E2,4E0,2.5E1,3.1E1,6.3E1,2.8E1,2.2E1,7E0,4.3E1,8E0,2.2E1,4E0,6E0,1.1E1,2.8E1,8.5E1,4E0,8E0,2.6E1,1.3E1,6.1E1,1.38E2,4E0,4.2E1,4E0,4E0,7E1,3.7E1,3.3E1,1.09E2,4E0,2.1E1,1.9E1,1.2E1,5.3E1,1E1,1.6E1,1.2E1,5E0,1.7E1,2.5E1,1.8E1,4E0,4E0,1.4E1,8E0,2.4E1,4E0,4.7E1,3.8E1,4E0,4E0,1.6E1,1E1,7E0,5.4E1,1.31E2,7E0,1.2E1,3E1,6E1,1E1,3.3E1,4E0,2.4E1,9E0,1.7E1,9.2E1,1.5E1,6E0,1.3E1,6E0,3.9E1,1.4E1,4E0,6E0,4E0,2.1E1,8E0,1E1,4E0,4E0,1.7E1,7E0,3E1,1.7E1,1.3E1,2.5E1,8E0,8E0,4E0,6E0,1.8E1,3.6E1,9.5E1,3.6E1,7E0,5E0,1.9E1,1.1E1,4E0,5.6E1,5E0,2.8E1,5E0,1.9E1,4E0,5E0,7E0,1E1,8.1E1,1.1E1,5E0,1E1,5E0,8E0,4E0,3.5E1,7E0,7E0,1.7E1,4E0,4E0,4E0,1.3E1,4E0,2.5E1,5E0,9E0,4E0,4E0,2.1E1,4E0,4E0,4E0,4E0,1.2E1,6E0,9E0,2.7E1,6.9E1,2.6E1,1.5E1,2.1E1,7E0,1.2E1,7E0,4E0,4.5E1,1.1E1,2.4E1,4E0,1.5E1,4E0,4E0,6E0,7.4E1,7E0,4E0,7E0,4E0,6E0,1.6E1,1.9E1,1.5E1,6E0,5E0,7E0,1.4E1,1.3E1,4.6E1,2.3E1,7E0,1.9E1,7E0,8E0,7E0,5E0,4E1,5E0,5E0,6E0,7E0,6.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[6.180792E-3,-1.6503148E-1,4.7275862E-1,-5.0639033E-1,-1.129815E-2,-6.803958E-1,5.4719776E-1,1.8274754E-2,-5.280954E-1,-1.6625014E-1,1.1087337E-1,-3.9352454E-2,-1.7605912E-2,1.3548586E0,4.8434323E-1,-6.065767E-1,-3.0085424E-1,-1.2424855E-1,-5.163365E-1,4.6761047E-2,2.686597E-1,4.5754105E-2,7.792276E-2,3.3147123E-1,7.643023E-1,-5.4452336E-1,-1.0120652E0,-3.3467343E-1,6.379688E-3,-1.7157865E-1,6.810231E-2,-5.913036E-1,-5.229772E-3,8.2113095E-2,-2.3808844E-1,2.206505E-1,2.2615165E-2,1.6233137E-1,4.803148E-1,7.994446E-1,2.2713481E-3,-6.016546E-1,-3.5165614E-1,-2.0412194E-2,-1.2107033E0,-4.5247874E-1,-2.5665715E-1,-3.8528496E-1,-1.127225E-1,1.734763E-1,1.8517341E-2,-6.655063E-1,-1.4194483E-2,1.07354075E-1,-1.8962681E-1,-2.392149E-2,-8.3356254E-2,1.07245244E-1,3.375724E-1,3.3497408E-1,5.235329E-2,5.8765614E-1,2.4847795E-1,5.636321E-1,9.4491065E-1,-6.7435145E-1,-4.8269066E-1,-4.9582702E-1,-8.968568E-2,-7.161074E-2,-3.326427E-2,-2.2630611E-2,-1.0853086E-2,-3.0224049E-3,-3.0056164E-1,-3.0194435E-1,-5.634421E-1,-7.476078E-2,-2.0266244E-1,1.4698161E-3,1.1921103E-2,-6.736173E-3,5.0641637E-2,-2.244549E-2,-4.2857196E-2,6.4812E-3,1.5728682E-1,-2.97582E-1,1.8431344E-3,-1.4241126E-2,1.729825E-2,-2.1520426E-2,1.6255073E-1,3.9851704E-1,1.5145347E-1,2.513066E-2,2.1373E-1,-3.0838272E-1,1.3425767E-1,4.5744762E-1,8.594612E-1,3.4686705E-1,-7.858146E-3,1.4932823E-2,6.2763226E-1,4.9509685E-2,3.5136476E-2,-3.2075506E-2,-1.3317501E-2,-8.192813E-4,-5.5199087E-1,-1.1377268E-2,-2.473519E-2,-8.901832E-3,1.886736E-3,-3.4619373E-1,-6.3525126E-3,-2.2607056E-2,-2.3402435E-1,-3.4160864E-2,-1.4916765E-2,-1.6875029E-1,-2.025404E-2,-8.9234844E-2,-3.0028555E-1,1.2991291E-1,-1.0866737E-2,4.3241512E-2,-1.776235E-2,-7.973841E-3,2.1443602E-1,-7.090227E-3,-1.8114451E-2,6.0518007E-3,-5.462109E-3,4.420927E-3,-9.4596185E-3,2.274301E-1,5.3403392E-2,2.3255488E-1,4.7733608E-1,1.9099436E-3,1.1702081E-2,2.999682E-1,1.1590164E-1,-3.993294E-3,-2.1651287E-2,3.341527E-1,2.1458121E-2,7.0477296E-3,5.174998E-1,4.65909E-2,2.6137244E-2,2.1844682E-1,2.5887595E-2,3.205296E-2,1.3973525E-2,-5.954478E-1,-5.945938E-3,-2.4844922E-1,-1.8800398E-2,-5.5004717E-3,-1.2569496E-2,-7.12295E-4,-1.8868068E-1,-8.481989E-2,2.828785E-2,-1.4959584E-1,5.0678095E-3,-3.65702E-1,-7.2277533E-3,8.353372E-3,1.5093553E-3,-4.2204354E-3,1.7792116E-3,-4.433357E-2,1.2921506E-1,-8.109424E-2,7.055284E-2,2.3125322E-1,-4.505623E-3,1.7135364E-1,1.4623275E-2,4.7639236E-3,-2.4074948E-4,4.0995292E-3,1.6568195E-2,1.2153308E-2,2.5447248E-2,1.6693758E-2,7.683984E-3,-3.5270027E-6,9.820194E-3,2.3110531E-1,2.1871414E-2,-9.265563E-3,7.094084E-2,5.201964E-3,5.694655E-1,3.5538576E-3,2.592104E-1,-2.043926E-2,-3.1079069E-2,-5.291665E-3,-1.4257282E-2,-1.2106815E-2,-4.519799E-3,1.220963E-3,-7.2071347E-3,5.0935005E-3,-8.90217E-4,-9.215974E-3,-1.9180061E-3,-1.2051613E-2,-2.2234317E-2,-4.1192146E-3,4.457565E-3,9.330464E-3,2.3287753E-4,5.383695E-4,-7.138954E-3,-2.3208114E-3,5.4996517E-3,3.7311022E-3,1.22307185E-2,4.3693925E-3,9.825735E-3,1.3281742E-2,5.4156007E-3,-3.6526588E-3,5.0203376E-3,2.8532118E-2,1.5245063E-2,7.564245E-3,1.6637865E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,-1,47,49,51,-1,53,55,57,-1,59,61,63,-1,65,67,-1,69,71,73,75,77,79,81,83,-1,85,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,115,117,119,121,123,-1,-1,-1,125,-1,-1,127,129,131,-1,-1,133,135,137,139,141,-1,143,145,147,149,151,153,-1,-1,155,-1,-1,-1,-1,-1,157,-1,-1,-1,-1,159,-1,-1,161,-1,-1,163,165,167,169,171,173,175,-1,177,179,-1,-1,-1,-1,-1,-1,181,183,185,187,-1,-1,189,191,-1,-1,193,195,-1,197,-1,-1,199,-1,-1,-1,201,-1,203,-1,-1,-1,-1,205,207,209,211,-1,213,-1,-1,-1,-1,-1,215,217,219,221,223,-1,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,227,-1,-1,229,-1,231,-1,233,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.1240005E1,3.9038086E1,2.3806591E1,4.731186E0,9.748823E0,5.475435E-1,1.2390434E1,0E0,3.84803E0,3.273695E0,2.9008594E0,0E0,0E0,5.0655365E-1,1.0018154E1,3.65094E0,9.3690205E-1,1.8688982E0,6.5838957E-1,2.093973E0,7.720475E-1,0E0,0E0,3.8745651E0,2.0717506E0,1.408184E0,1.8437462E0,3.9854956E-1,0E0,2.019351E0,2.1101129E-1,2.170825E-1,0E0,1.2824098E0,9.5225835E-1,8.9196634E-1,0E0,1.4064816E0,1.9194889E0,2.2116356E0,0E0,6.4310455E-1,1.315198E0,0E0,1.3815918E0,5.5664062E-2,2.825966E-1,3.4954214E-1,4.287045E-1,1.4751896E-1,1.604698E-1,2.5580263E-1,0E0,8.502102E-1,4.0832222E-1,0E0,3.6752677E-1,2.6699176E-1,3.2530355E-1,6.1856604E-1,1.4300581E0,1.5887585E0,1.2121193E0,3.048668E-1,3.0662918E-1,3.156147E-1,1.4724045E0,9.531975E-2,1.9608359E-1,0E0,0E0,0E0,0E0,0E0,1.681776E-1,2.1640873E-1,1.8460774E-1,4.7302693E-1,3.9398372E-1,0E0,0E0,0E0,1.23307906E-1,0E0,0E0,8.358092E-1,1.0689397E0,8.964944E-2,0E0,0E0,2.0039521E-1,2.773286E-1,1.7197114E-1,2.1601772E-1,1.0167009E-1,0E0,1.3396406E-1,2.7963167E-1,8.722045E-1,6.5344334E-1,1.7351437E-1,5.0814795E-1,0E0,0E0,3.2805347E-1,0E0,0E0,0E0,0E0,0E0,6.591568E-1,0E0,0E0,0E0,0E0,2.9530048E-2,0E0,0E0,5.946493E-2,0E0,0E0,1.0099387E-1,1.9044825E-1,2.445545E-1,1.3097334E-1,4.8538357E-2,6.313289E-2,4.04773E-1,0E0,1.7797852E-1,4.6324515E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.5702178E-2,3.2590926E-2,1.4949113E-1,1.0702038E-1,0E0,0E0,2.1925747E-2,1.2333231E-1,0E0,0E0,9.150219E-2,2.9986197E-1,0E0,6.3022995E-1,0E0,0E0,7.236326E-2,0E0,0E0,0E0,1.2808132E-1,0E0,5.3262115E-2,0E0,0E0,0E0,0E0,1.7434561E-1,2.100132E-1,1.3855205E-1,7.259676E-2,0E0,3.1965733E-2,0E0,0E0,0E0,0E0,0E0,1.790477E-1,2.4288192E-1,1.1077051E-1,9.3834616E-2,3.8177347E-1,0E0,1.796195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.051114E-2,0E0,0E0,1.3309339E-1,0E0,1.3042927E-1,0E0,5.6667805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,74,74,75,75,76,76,77,77,78,78,82,82,85,85,86,86,87,87,90,90,91,91,92,92,93,93,94,94,96,96,97,97,98,98,99,99,100,100,101,101,104,104,110,110,115,115,118,118,121,121,122,122,123,123,124,124,125,125,126,126,127,127,129,129,130,130,137,137,138,138,139,139,140,140,143,143,144,144,147,147,148,148,150,150,153,153,157,157,159,159,164,164,165,165,166,166,167,167,169,169,175,175,176,176,177,177,178,178,179,179,181,181,193,193,196,196,198,198,200,200],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,-1,48,50,52,-1,54,56,58,-1,60,62,64,-1,66,68,-1,70,72,74,76,78,80,82,84,-1,86,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,116,118,120,122,124,-1,-1,-1,126,-1,-1,128,130,132,-1,-1,134,136,138,140,142,-1,144,146,148,150,152,154,-1,-1,156,-1,-1,-1,-1,-1,158,-1,-1,-1,-1,160,-1,-1,162,-1,-1,164,166,168,170,172,174,176,-1,178,180,-1,-1,-1,-1,-1,-1,182,184,186,188,-1,-1,190,192,-1,-1,194,196,-1,198,-1,-1,200,-1,-1,-1,202,-1,204,-1,-1,-1,-1,206,208,210,212,-1,214,-1,-1,-1,-1,-1,216,218,220,222,224,-1,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,228,-1,-1,230,-1,232,-1,234,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.972052E5,1E0,6E0,8.1E2,1.5900173E3,6.743651E7,1.8274754E-2,1.559733E6,6.2233735E-6,2.0766992E6,-3.9352454E-2,-1.7605912E-2,2.35184E5,2.8530578E6,4.6463413E0,5.185192E2,3.2911258E6,1.3E1,2.3E1,6.0052995E6,4.5754105E-2,7.792276E-2,5.709E3,6.292039E0,1.4912975E-1,9E0,1.901875E2,6.379688E-3,1.3920766E2,5.894636E6,7.714286E0,-5.229772E-3,2.1129E4,5.095006E5,4.4935583E2,2.2615165E-2,3.53969E5,4.05E2,1.2400667E3,2.2713481E-3,8.7601E4,8.01E2,-2.0412194E-2,1.753E3,8.390338E7,1.5522388E0,1.1244019E2,2.165E3,4.49239E6,2.234E3,5.4E1,-1.4194483E-2,1.295313E6,3.3704692E10,-2.392149E-2,4.236E3,2.41E2,7.7630096E2,4E0,7.70947E5,3.4915986E3,4.1325716E7,1.0950326E6,1.6063418E7,3.1530054E0,1.5E1,5.4009717E3,4.6750406E4,-7.161074E-2,-3.326427E-2,-2.2630611E-2,-1.0853086E-2,-3.0224049E-3,1.3999298E5,1.02E2,2.8024E4,5.47E2,1.9637654E2,1.4698161E-3,1.1921103E-2,-6.736173E-3,3.5364E4,-2.244549E-2,-4.2857196E-2,1.28636E7,4.8709216E5,1.97525E1,1.8431344E-3,-1.4241126E-2,5.857143E0,2.8055556E0,1.28636E7,5.281628E2,1.8645384E1,2.513066E-2,3.3382E4,1.4283589E6,3.1226995E0,9.49372E4,2.1298597E0,4.435876E2,-7.858146E-3,1.4932823E-2,1.8201E4,4.9509685E-2,3.5136476E-2,-3.2075506E-2,-1.3317501E-2,-8.192813E-4,2E1,-1.1377268E-2,-2.473519E-2,-8.901832E-3,1.886736E-3,1.6E1,-6.3525126E-3,-2.2607056E-2,2.0487332E-3,-3.4160864E-2,-1.4916765E-2,2.16E2,3.0283334E2,1.4390917E8,7.54E2,1.422162E3,1.0079E5,8.6363555E4,-1.776235E-2,1.387E3,1E0,-7.090227E-3,-1.8114451E-2,6.0518007E-3,-5.462109E-3,4.420927E-3,-9.4596185E-3,8.051603E7,1.1716E4,6.965855E9,2.36433E6,1.9099436E-3,1.1702081E-2,6.941074E6,4.8709216E5,-3.993294E-3,-2.1651287E-2,2.379747E0,2.9585715E2,7.0477296E-3,1.36E2,4.65909E-2,2.6137244E-2,4.6750406E4,2.5887595E-2,3.205296E-2,1.3973525E-2,2.6210527E0,-5.945938E-3,2.6699028E0,-1.8800398E-2,-5.5004717E-3,-1.2569496E-2,-7.12295E-4,4.6E1,5.5555553E0,1.8949389E6,1.5422421E2,5.0678095E-3,5.494667E2,-7.2277533E-3,8.353372E-3,1.5093553E-3,-4.2204354E-3,1.7792116E-3,1.47176E5,1.1362371E6,5E0,2.8846003E5,5.1604336E7,-4.505623E-3,6.009E4,1.4623275E-2,4.7639236E-3,-2.4074948E-4,4.0995292E-3,1.6568195E-2,1.2153308E-2,2.5447248E-2,1.6693758E-2,7.683984E-3,-3.5270027E-6,9.820194E-3,6.968415E6,2.1871414E-2,-9.265563E-3,1.345672E6,5.201964E-3,1.6007428E7,3.5538576E-3,1E0,-2.043926E-2,-3.1079069E-2,-5.291665E-3,-1.4257282E-2,-1.2106815E-2,-4.519799E-3,1.220963E-3,-7.2071347E-3,5.0935005E-3,-8.90217E-4,-9.215974E-3,-1.9180061E-3,-1.2051613E-2,-2.2234317E-2,-4.1192146E-3,4.457565E-3,9.330464E-3,2.3287753E-4,5.383695E-4,-7.138954E-3,-2.3208114E-3,5.4996517E-3,3.7311022E-3,1.22307185E-2,4.3693925E-3,9.825735E-3,1.3281742E-2,5.4156007E-3,-3.6526588E-3,5.0203376E-3,2.8532118E-2,1.5245063E-2,7.564245E-3,1.6637865E-2],"split_indices":[2,43,17,3,2,70,7,0,9,52,43,0,0,1,43,68,67,43,8,3,43,0,0,2,53,53,0,4,0,70,60,71,0,44,43,70,0,9,8,4,0,9,2,0,44,7,68,67,44,43,12,0,0,9,46,0,12,10,4,8,9,4,66,62,62,68,0,43,48,0,0,0,0,0,43,2,12,2,67,0,0,0,9,0,0,60,43,73,0,0,68,71,60,4,71,0,9,43,69,48,53,71,0,0,10,0,0,0,0,0,6,0,0,0,0,3,0,0,53,0,0,44,4,5,2,48,9,48,0,2,79,0,0,0,0,0,0,7,44,46,43,0,0,60,43,0,0,69,67,0,0,0,0,48,0,0,0,71,0,68,0,0,0,0,0,73,60,67,0,48,0,0,0,0,0,1,43,8,43,7,0,1,0,0,0,0,0,0,0,0,0,0,0,9,0,0,9,0,66,0,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,7.43E2,2.72E2,2.3E2,5.13E2,1.6E1,2.56E2,5E0,2.25E2,2.26E2,2.87E2,9E0,7E0,1.7E1,2.39E2,1.66E2,5.9E1,2.03E2,2.3E1,2.05E2,8.2E1,1E1,7E0,1.56E2,8.3E1,1.46E2,2E1,5.5E1,4E0,1.63E2,4E1,1.9E1,4E0,1.83E2,2.2E1,6.9E1,1.3E1,7.4E1,8.2E1,7.9E1,4E0,1.11E2,3.5E1,6E0,1.4E1,2E1,3.5E1,3.4E1,1.29E2,1.2E1,2.8E1,1.4E1,5E0,1.68E2,1.5E1,7E0,1.5E1,3.6E1,3.3E1,2.8E1,4.6E1,5.5E1,2.7E1,3.2E1,4.7E1,6.6E1,4.5E1,2.2E1,1.3E1,7E0,7E0,1.6E1,4E0,7E0,2.8E1,2.5E1,9E0,9.2E1,3.7E1,5E0,7E0,4E0,2.4E1,1E1,4E0,5.6E1,1.12E2,1E1,5E0,4E0,1.1E1,1.1E1,2.5E1,2.4E1,9E0,9E0,1.9E1,8E0,3.8E1,3.9E1,1.6E1,2.2E1,5E0,8E0,2.4E1,2.5E1,2.2E1,6.2E1,4E0,6E0,3.9E1,4E0,1.8E1,7E0,6E0,2.1E1,7E0,5E0,2E1,4E0,5E0,3.3E1,5.9E1,1.8E1,1.9E1,1E1,1.4E1,5.2E1,4E0,2.9E1,8.3E1,5E0,5E0,6E0,5E0,7E0,4E0,1.5E1,1E1,9E0,1.5E1,5E0,4E0,9E0,1E1,4E0,4E0,1.3E1,2.5E1,7E0,3.2E1,9E0,7E0,1.5E1,7E0,1.9E1,5E0,3.5E1,4E0,1E1,1.1E1,6E0,1.4E1,4E0,2.9E1,2.5E1,3.4E1,1.4E1,4E0,1.2E1,7E0,6E0,4E0,5E0,9E0,2.6E1,2.6E1,1.5E1,1.4E1,7.9E1,4E0,1.1E1,4E0,5E0,5E0,5E0,4E0,5E0,1E1,5E0,4E0,5E0,5E0,9E0,4E0,4E0,2.1E1,4E0,2.8E1,4E0,1.1E1,1.4E1,2.1E1,4E0,6E0,1.5E1,1.4E1,1E1,1.5E1,1.2E1,2.2E1,9E0,5E0,8E0,4E0,2E1,6E0,1.6E1,1E1,7E0,8E0,4E0,1E1,1.5E1,6.4E1,5E0,6E0,5E0,4E0,4E0,1.7E1,2.2E1,6E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"235","size_leaf_vector":"1"}},{"base_weights":[1.4377204E-2,-1.5660624E-1,4.9529484E-1,-4.8663077E-1,-2.1084787E-3,-5.829388E-1,5.4487056E-1,2.9609978E-1,-5.1911163E-1,-1.6130061E-1,1.134766E-1,-3.1639814E-2,-1.4011223E-2,1.3125441E0,4.939846E-1,3.288987E-2,-4.6090093E-3,-6.983173E-1,-4.0818495E-1,-2.4788819E-1,6.9673453E-3,7.799139E-2,4.5886874E-1,3.4380168E-2,6.706123E-2,3.5069853E-1,7.50795E-1,-6.149155E-1,-8.846754E-1,-4.7176003E-1,3.2110855E-3,-3.9059E-1,-1.7390017E-1,-2.7103266E-1,5.5396438E-2,1.4779969E-1,-1.6382823E-2,2.3503827E-1,2.5766423E-2,9.7157516E-2,4.5448008E-1,5.896566E-1,8.7109065E-1,-6.482583E-1,-9.039893E-3,-2.1030493E-2,-9.478067E-1,-4.1875008E-1,-8.000923E-1,1.7956136E-2,-1.5118612E-1,-2.8482133E-1,-4.650639E-1,-2.5084972E-1,-5.3899504E-2,-5.529362E-4,-2.2403058E-2,-4.924974E-3,1.7627378E-1,9.2411615E-2,2.7075124E-1,2.0955662E-1,-4.5018412E-2,2.8565787E-3,1.8289194E-2,2.7418974E-1,-6.251715E-2,4.8526943E-1,-7.1584913E-3,1.265036E-2,2.9511178E-2,9.343002E-1,1.7807515E-2,-3.1096052E-2,-1.731282E-2,-4.7187775E-2,-2.4150196E-2,-5.386295E-1,-2.7420583E-1,-1.7646331E-2,-1.0127735E0,-1.3990566E-2,-4.50798E-2,-3.202222E-1,-7.5404365E-3,-1.1275629E-2,-2.33795E-2,-1.4055581E-1,-3.3882025E-1,-1.10522784E-1,5.1063456E-2,5.1522065E-2,-7.241517E-2,2.7589345E-1,9.3746796E-2,1.4835468E-1,-1.3073278E-2,3.2628033E-1,-1.2055591E-2,1.520202E-2,-2.1922132E-3,-1.2874891E-1,4.624318E-3,1.860571E-2,1.3351272E-1,3.8614534E-3,-1.13053136E-1,4.25396E-1,8.7166077E-1,2.323293E-2,4.4732958E-2,-3.0387348E-2,-4.7319612E-1,5.3308133E-2,-3.6560488E-1,-6.0503263E-2,-2.7562568E-2,2.1564204E-3,-6.7529925E-3,-8.010528E-3,-1.6387561E-2,3.213173E-2,-2.4402682E-1,-4.5196897E-1,-2.0449527E-1,-5.7164866E-2,-1.247199E-2,-3.1442575E-3,1.055183E-1,-3.3468108E-3,1.0682799E-1,3.0929053E-3,-1.1144002E-1,1.7358132E-2,5.5846944E-3,9.972215E-3,1.0431361E-2,2.3153735E-2,2.0525607E-1,1.2611057E-1,-1.9242454E-1,1.7359488E-2,8.3915464E-4,3.5793418E-3,-2.3604223E-1,4.1813046E-2,-1.4734697E-1,4.511797E-4,8.970255E-3,-7.2414996E-3,-2.0755947E-1,-3.7772248E-3,4.4937763E-1,4.681434E-2,2.4832327E-2,-2.2883609E-2,-1.3626187E-2,1.165859E-2,-3.5924413E-3,-2.906079E-1,-2.3053942E-2,5.409874E-3,-1.53148E-3,-1.3219251E-2,-3.960894E-3,-7.624876E-3,-5.33857E-1,-1.5479107E-3,-2.5869983E-1,-4.8279176E-3,3.266586E-3,1.8617671E-3,6.5732696E-3,5.6622062E-2,8.751726E-3,-6.577063E-3,-2.3720202E-3,-2.4738437E-3,3.303116E-3,9.3819596E-2,-5.593421E-3,4.828695E-2,2.562378E-1,-3.7309334E-3,1.9170468E-1,-1.9731954E-2,-2.5587803E-2,9.149446E-2,-1.1240177E-2,-2.6808238E-1,-3.6911797E-3,8.559756E-3,1.2190492E-2,-1.0530989E-2,1.2568238E-3,1.8117186E-3,-2.0673494E-3,-2.913659E-3,-1.35537E-2,3.4046194E-1,5.7156485E-1,-1.7561594E-2,-8.670689E-3,-2.8586531E-2,-1.3454394E-2,-1.5883882E-2,-8.000104E-3,6.2799673E-3,-1.1510374E-6,2.0641368E-3,5.8707697E-3,7.5796368E-3,-5.2727587E-3,8.692511E-3,1.3919663E-2,1.2261971E-2,4.467113E-3,-5.3287987E-3,3.1615566E-3,6.265921E-4,8.923013E-3,-1.4147436E-2,-6.7963875E-3,-5.1409528E-3,2.3398283E-3,6.366647E-4,1.7056411E-2,1.5334917E-2,2.7908405E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,-1,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,-1,75,77,79,-1,81,83,85,87,89,-1,-1,91,93,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,109,-1,-1,-1,-1,-1,111,113,-1,115,-1,117,119,-1,-1,-1,121,123,125,127,129,131,133,135,137,139,141,-1,-1,-1,143,145,-1,147,-1,149,151,153,-1,-1,-1,155,157,159,-1,-1,-1,-1,-1,-1,161,163,165,167,169,-1,-1,171,-1,173,-1,175,-1,-1,177,-1,179,181,183,185,-1,-1,187,189,191,193,-1,-1,195,197,-1,199,-1,-1,-1,-1,-1,-1,201,-1,-1,-1,-1,-1,-1,203,-1,205,-1,-1,-1,-1,207,-1,-1,-1,-1,-1,209,-1,211,213,-1,215,-1,217,219,-1,221,-1,-1,223,-1,-1,-1,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.214206E1,3.757717E1,1.4322632E1,6.128292E0,9.273622E0,1.3433027E-1,9.156639E0,1.7170925E0,4.2075424E0,3.09055E0,3.5430548E0,0E0,0E0,1.0506058E-1,8.457668E0,0E0,0E0,7.771225E-1,3.715458E0,1.4102278E0,9.978342E-1,1.7548567E0,4.6635056E-1,0E0,0E0,4.008028E0,1.1146317E0,7.3983574E-1,2.192421E-1,1.8057613E0,1.2490338E0,2.3775864E-1,8.5447526E-1,6.037253E-1,4.6023214E-1,1.016917E0,7.470035E-1,2.5410682E-1,0E0,1.3174495E0,2.1204605E0,4.3020344E-1,1.0519333E0,1.3487625E-1,0E0,0E0,9.171867E-2,1.7237167E0,1.0360098E0,0E0,2.2770771E-1,4.2442918E-2,1.1205006E-1,5.0542593E-1,2.3149088E-1,0E0,0E0,1.6754472E-1,1.4678335E-1,6.3336235E-1,1.4728086E0,4.148702E-1,4.2456967E-1,0E0,0E0,3.440467E-1,1.9403213E-1,2.0351944E0,0E0,0E0,0E0,9.919739E-2,0E0,0E0,0E0,0E0,0E0,1.4082909E-1,1.5461471E0,0E0,4.5406246E-1,0E0,9.9581204E-2,1.9532084E-2,0E0,0E0,0E0,4.9028122E-1,3.7873387E-1,1.9725138E-1,9.799653E-2,1.7228341E-1,1.1623266E-1,9.385514E-2,1.4155445E-1,4.9393535E-1,9.728532E-1,6.4865494E-1,0E0,0E0,0E0,5.403665E-1,3.7351626E-1,0E0,8.817053E-2,0E0,1.884945E-1,1.1556816E0,7.0693016E-2,0E0,0E0,0E0,1.5640259E-2,3.3268875E-1,2.551918E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.489652E-2,8.87841E-2,2.8782153E-1,1.3962722E-1,1.2845194E-1,0E0,0E0,1.8242985E-2,0E0,5.9779093E-2,0E0,2.2424966E-2,0E0,0E0,3.9001126E-2,0E0,2.4577151E-1,3.7174964E-1,3.1400722E-1,6.546844E-1,0E0,0E0,4.1293845E-1,8.3642006E-2,2.2298366E-1,1.8909898E-1,0E0,0E0,1.9179584E-2,1.0549575E-1,0E0,9.765167E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.8665218E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.702423E-2,0E0,2.7824879E-2,0E0,0E0,0E0,0E0,5.3915963E-2,0E0,0E0,0E0,0E0,0E0,2.0469725E-2,0E0,2.631826E-1,5.37529E-2,0E0,9.3395054E-2,0E0,1.0072256E-1,1.0828302E-1,0E0,3.3397198E-2,0E0,0E0,2.1930218E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.5483685E-1,1.11252785E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,71,71,77,77,78,78,80,80,82,82,83,83,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,97,97,101,101,102,102,104,104,106,106,107,107,108,108,112,112,113,113,114,114,121,121,122,122,123,123,124,124,125,125,128,128,130,130,132,132,135,135,137,137,138,138,139,139,140,140,143,143,144,144,145,145,146,146,149,149,150,150,152,152,159,159,166,166,168,168,173,173,179,179,181,181,182,182,184,184,186,186,187,187,189,189,192,192,199,199,200,200],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,-1,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,-1,76,78,80,-1,82,84,86,88,90,-1,-1,92,94,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,110,-1,-1,-1,-1,-1,112,114,-1,116,-1,118,120,-1,-1,-1,122,124,126,128,130,132,134,136,138,140,142,-1,-1,-1,144,146,-1,148,-1,150,152,154,-1,-1,-1,156,158,160,-1,-1,-1,-1,-1,-1,162,164,166,168,170,-1,-1,172,-1,174,-1,176,-1,-1,178,-1,180,182,184,186,-1,-1,188,190,192,194,-1,-1,196,198,-1,200,-1,-1,-1,-1,-1,-1,202,-1,-1,-1,-1,-1,-1,204,-1,206,-1,-1,-1,-1,208,-1,-1,-1,-1,-1,210,-1,212,214,-1,216,-1,218,220,-1,222,-1,-1,224,-1,-1,-1,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,6E0,8.1E2,5.4051904E8,2.4481E4,1.89991E5,7.7E1,1.5859042E6,5.1167645E6,-3.1639814E-2,-1.4011223E-2,1.2328386E3,2.8045622E6,3.288987E-2,-4.6090093E-3,1.4504054E8,3.513913E2,3.79E2,1.0203835E-5,2.0342858E0,3.7788504E7,3.4380168E-2,6.706123E-2,7.075161E2,7.459E3,4.56E2,1.4E1,4.9735293E0,2.2E0,3.1E1,1.559733E6,6.83E2,1.65281E8,5.372237E2,9E0,1.2E1,2.5766423E-2,1.124641E0,4.1325716E7,3.532346E-1,4.9036694E0,5.159652E-7,-9.039893E-3,-2.1030493E-2,1E0,3.95E2,3.89E2,1.7956136E-2,2.87834E5,1E0,1E1,4.9E1,3.0283334E2,-5.529362E-4,-2.2403058E-2,2.4915715E-3,2.1111E4,5.0149255E0,1.4676277E7,8.4683485E-2,1.244E3,2.8565787E-3,1.8289194E-2,2.276E4,9.47E4,4.907764E3,-7.1584913E-3,1.265036E-2,2.9511178E-2,1E0,1.7807515E-2,-3.1096052E-2,-1.731282E-2,-4.7187775E-2,-2.4150196E-2,1.5827801E4,2.3961E4,-1.7646331E-2,7.3E1,-1.3990566E-2,3.01E2,1.9E1,-7.5404365E-3,-1.1275629E-2,-2.33795E-2,1.664557E0,1E0,2.4067588E0,1.944578E6,5.08E2,1.2368386E0,3.52E2,2.6138635E2,5.201557E7,2.2842104E0,5.2409735E-7,-1.2055591E-2,1.520202E-2,-2.1922132E-3,1E0,9.932432E0,1.860571E-2,3.0714026E8,3.8614534E-3,1.3E1,1.2825651E0,2.3385885E0,2.323293E-2,4.4732958E-2,-3.0387348E-2,1.901875E2,1.610025E6,1.0885427E9,-6.0503263E-2,-2.7562568E-2,2.1564204E-3,-6.7529925E-3,-8.010528E-3,-1.6387561E-2,1.2916666E0,1.1320151E7,1.8181819E0,1.8738E4,1.802361E6,-1.247199E-2,-3.1442575E-3,3.875E0,-3.3468108E-3,6.23E2,3.0929053E-3,2.8266037E2,1.7358132E-2,5.5846944E-3,1.4E1,1.0431361E-2,4.1221875E-1,2.560356E5,9.7E2,2.5751933E-1,1.7359488E-2,8.3915464E-4,3.792E3,5.821732E2,1.0439024E1,2.402121E4,4.511797E-4,8.970255E-3,2.6845297E5,4.6847186E2,-3.7772248E-3,1.4839433E5,4.681434E-2,2.4832327E-2,-2.2883609E-2,-1.3626187E-2,1.165859E-2,-3.5924413E-3,6.76E2,-2.3053942E-2,5.409874E-3,-1.53148E-3,-1.3219251E-2,-3.960894E-3,-7.624876E-3,5E0,-1.5479107E-3,4.0663E4,-4.8279176E-3,3.266586E-3,1.8617671E-3,6.5732696E-3,5.51E2,8.751726E-3,-6.577063E-3,-2.3720202E-3,-2.4738437E-3,3.303116E-3,1.75E0,-5.593421E-3,2.992648E-2,7.6937294E-1,-3.7309334E-3,8.051603E7,-1.9731954E-2,6.6414773E-1,3E0,-1.1240177E-2,1E0,-3.6911797E-3,8.559756E-3,2.9427118E2,-1.0530989E-2,1.2568238E-3,1.8117186E-3,-2.0673494E-3,-2.913659E-3,-1.35537E-2,2.7726634E5,1.0476191E0,-1.7561594E-2,-8.670689E-3,-2.8586531E-2,-1.3454394E-2,-1.5883882E-2,-8.000104E-3,6.2799673E-3,-1.1510374E-6,2.0641368E-3,5.8707697E-3,7.5796368E-3,-5.2727587E-3,8.692511E-3,1.3919663E-2,1.2261971E-2,4.467113E-3,-5.3287987E-3,3.1615566E-3,6.265921E-4,8.923013E-3,-1.4147436E-2,-6.7963875E-3,-5.1409528E-3,2.3398283E-3,6.366647E-4,1.7056411E-2,1.5334917E-2,2.7908405E-2],"split_indices":[2,43,17,3,2,7,12,9,44,43,43,0,0,67,43,0,0,46,67,2,53,68,7,0,0,67,2,10,3,68,73,10,9,2,5,4,3,3,0,53,66,53,53,52,0,0,113,2,2,0,12,29,3,0,4,0,0,53,9,71,62,57,2,0,0,10,9,4,0,0,0,29,0,0,0,0,0,43,1,0,0,0,0,67,0,0,0,68,29,68,9,2,68,0,4,7,69,52,0,0,0,26,69,0,5,0,3,69,53,0,0,0,4,5,46,0,0,0,0,0,0,68,5,71,9,12,0,0,69,0,2,0,4,0,0,3,0,53,62,2,57,0,0,44,4,73,48,0,0,43,4,0,48,0,0,0,0,0,0,2,0,0,0,0,0,0,6,0,9,0,0,0,0,2,0,0,0,0,0,68,0,53,53,0,7,0,53,8,0,30,0,0,67,0,0,0,0,0,0,43,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,7.36E2,2.61E2,2.34E2,5.02E2,1.1E1,2.5E2,9E0,2.25E2,2.11E2,2.91E2,7E0,4E0,1.4E1,2.36E2,4E0,5E0,8.4E1,1.41E2,1.39E2,7.2E1,2.65E2,2.6E1,4E0,1E1,1.53E2,8.3E1,6.1E1,2.3E1,1.22E2,1.9E1,4.6E1,9.3E1,1E1,6.2E1,1.52E2,1.13E2,9E0,1.7E1,4.5E1,1.08E2,3.8E1,4.5E1,5.6E1,5E0,4E0,1.9E1,1.07E2,1.5E1,5E0,1.4E1,2.1E1,2.5E1,5.6E1,3.7E1,5E0,5E0,4.2E1,2E1,1.06E2,4.6E1,1.2E1,1.01E2,5E0,4E0,2.1E1,2.4E1,1.03E2,5E0,6E0,3.2E1,3.9E1,6E0,5E1,6E0,1.5E1,4E0,5.7E1,5E1,6E0,9E0,5E0,9E0,1.5E1,6E0,5E0,2E1,2.6E1,3E1,2.4E1,1.3E1,2.3E1,1.9E1,8E0,1.2E1,6.9E1,3.7E1,4.2E1,4E0,8E0,4E0,3.7E1,6.4E1,1E1,1.1E1,6E0,1.8E1,9.1E1,1.2E1,4E0,3.5E1,1.7E1,4E1,1.1E1,3.9E1,4E0,5E0,5E0,4E0,4E0,1.1E1,1E1,1.6E1,1.5E1,1.5E1,1.9E1,5E0,4E0,9E0,7E0,1.6E1,4E0,1.5E1,4E0,4E0,8E0,4E0,2.2E1,4.7E1,2.1E1,1.6E1,3.6E1,6E0,1.7E1,2E1,5.2E1,1.2E1,4E0,7E0,9E0,9E0,4E0,8.7E1,7E0,5E0,3.4E1,6E0,4E0,7E0,2.7E1,1.2E1,4E0,6E0,1.2E1,4E0,4E0,1.1E1,4E0,1.1E1,1.4E1,5E0,4E0,5E0,1.1E1,5E0,9E0,6E0,4E0,4E0,1.5E1,7E0,1.2E1,3.5E1,5E0,1.6E1,6E0,1E1,1.3E1,4E0,1.6E1,4E0,8E0,4.4E1,8E0,4E0,4E0,5E0,4E0,5E0,4.8E1,3.9E1,1.3E1,1.4E1,7E0,4E0,4E0,7E0,4E0,7E0,7E0,8E0,7E0,5E0,1.6E1,1.9E1,8E0,8E0,5E0,5E0,8E0,5E0,1.1E1,5E0,1E1,3.4E1,4E0,4.4E1,6E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[4.282539E-3,-1.5980986E-1,4.428412E-1,-4.5826387E-1,-9.64763E-3,-2.8706862E-2,4.8953718E-1,-6.6484565E-1,-3.5655844E-1,-1.7110102E-1,1.2145714E-1,2.0425431E-1,5.975026E-1,-6.287699E-1,-4.9905527E-2,-4.7406977E-1,-1.5504657E-1,-3.5407957E-1,-9.038243E-2,-1.948989E-2,1.9048887E-1,4.245525E-1,6.0930364E-2,1.1161122E0,5.404194E-1,-4.0212203E-2,-5.7448566E-1,-4.203092E-1,-8.556885E-1,5.480729E-1,-2.8238264E-1,-2.4379371E-1,-4.819723E-1,-6.863111E-2,-2.8434414E-2,1.8420845E-2,-2.9343253E-1,1.5160462E-1,4.2861778E-1,2.7033815E-1,2.7421888E-2,1.9839756E-1,-5.2444313E-2,3.0397184E-2,5.9613295E-2,4.2763776E-1,7.372566E-1,-2.7440354E-2,-1.3120598E-2,-2.3259267E-1,-4.647628E-1,-1.4245516E-2,-5.5558696E-2,1.17730005E-2,3.238991E-2,-2.2586961E-1,-2.9085081E-2,-3.053415E-1,-6.4963534E-2,-1.2324744E-2,-5.143408E-1,-1.0789787E-1,1.8881796E-2,1.4778185E-1,-3.637012E-2,-2.1648295E-2,4.8374466E-4,8.579984E-2,2.879833E-1,5.6159967E-3,4.9930394E-1,1.02200955E-1,1.8869437E-2,2.6119836E-3,2.6528913E-1,4.206135E-2,-2.078595E-1,4.638437E-1,-4.2280406E-2,7.809948E-1,9.187501E-3,-1.7436896E-2,-5.001263E-2,-5.210282E-1,-3.6299708E-1,-1.5876178E-1,-4.3716303E-1,-1.6737146E-2,-1.0280319E-2,3.6121877E-3,-9.114485E-3,-1.5027487E-2,-5.5692536E-1,-1.6829827E-1,2.5324164E-2,5.002059E-2,-1.1673954E-2,-1.5445804E-3,2.1004426E-1,-1.4001645E-1,6.1578788E-2,1.0641734E-1,-1.6641088E-2,-5.280974E-3,3.2465306E-1,2.5540506E-2,1.0539849E-2,1.3102842E-3,7.188572E-3,2.316542E-3,1.6000278E-2,9.190085E-3,-2.150581E-2,-1.4457218E-2,-1.857162E-3,1.0171475E-1,4.9673375E-1,-1.2682702E-2,9.166736E-3,8.893733E-3,8.217765E-1,-7.426931E-3,2.338216E-3,-2.4966089E-2,-1.1738144E-2,-1.8443297E-1,-4.3074203E-1,-3.051808E-1,-8.7806605E-2,-2.4272174E-2,-1.04512945E-2,-3.1522036E-2,-1.8650971E-2,-1.01339296E-1,-3.750304E-1,-1.3413919E-2,6.840111E-3,-5.4356554E-3,7.517972E-2,1.0871677E-2,4.493147E-3,1.9456498E-2,-2.0477846E-1,9.0122856E-2,-4.58138E-3,-1.2808391E-2,1.5958446E-1,3.6497775E-1,1.9443567E-3,-3.8519052E-3,2.0067447E-3,1.3277823E-2,-6.5351385E-3,2.1206862E-1,5.427489E-1,1.8452225E-2,3.918162E-2,-2.7686933E-3,-1.1893801E-2,-2.5489086E-2,-1.1202827E-2,-6.0975584E-3,-1.7337618E-2,-1.3765106E-2,-3.830511E-2,-3.1301405E-2,-1.841327E-1,-2.5823444E-1,-2.7890079E-2,6.411128E-2,-6.587412E-2,1.1746271E-1,-1.5169332E-2,-3.6964463E-3,5.314416E-3,-5.311406E-3,-1.0859397E-2,-2.8596302E-3,1.2057656E-1,4.6179496E-2,-1.349823E-1,2.1995944E-1,8.5637145E-2,2.7764356E-1,4.8408473E-1,3.9302735E-3,1.2337889E-2,5.0258005E-1,3.4941804E-2,-1.0424447E-4,-7.744306E-3,3.7727735E-3,-2.8495346E-3,-2.9315157E-3,-1.20281475E-2,-5.2722883E-3,-1.5432335E-2,-1.3650125E-3,5.629392E-3,-8.359306E-4,-7.7323965E-3,6.391512E-3,9.6006435E-5,-3.9546434E-3,3.7544388E-3,-2.256193E-4,6.871089E-3,-9.2264666E-4,3.528156E-3,-1.1525731E-2,-1.5668486E-3,1.1844793E-2,7.766568E-4,5.55371E-3,-2.998259E-3,8.021801E-3,1.7764298E-2,1.1768102E-2,2.491563E-2,2.4774328E-2,1.3376156E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,-1,73,75,-1,-1,77,79,-1,-1,81,83,-1,-1,-1,-1,85,-1,87,89,-1,91,93,95,97,99,-1,-1,101,103,-1,105,107,-1,-1,109,111,113,115,117,119,-1,-1,121,123,125,127,129,-1,-1,-1,-1,-1,131,133,135,137,-1,-1,139,141,143,145,-1,-1,147,-1,-1,-1,-1,-1,-1,-1,149,-1,-1,151,153,-1,-1,-1,155,-1,-1,-1,-1,157,159,161,163,-1,-1,-1,-1,165,167,169,-1,-1,171,-1,-1,173,175,177,-1,179,181,183,-1,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,189,191,193,195,-1,197,199,201,203,-1,-1,-1,-1,-1,205,207,209,211,213,215,217,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.268259E1,3.2935722E1,1.39701805E1,4.931526E0,1.0392763E1,0E0,8.009335E0,5.196037E-1,3.8810444E0,3.2173467E0,2.640493E0,2.3106086E0,5.051178E0,4.9687958E-1,0E0,1.7945328E0,5.795312E0,8.275175E-1,1.7166666E0,9.400883E-1,1.6202416E0,5.953741E-1,7.2732866E-1,5.8794785E-1,3.547451E0,0E0,1.9900513E-1,6.7629814E-1,1.9076719E0,2.7006102E-1,9.172845E-1,3.984151E-1,7.1755886E-2,5.1405317E-1,0E0,5.7240003E-1,5.9043854E-1,1.3932416E0,4.819212E-1,3.5299194E-1,0E0,1.8427515E-1,3.906209E-1,0E0,0E0,1.9421749E0,1.3046684E0,0E0,0E0,5.1001084E-1,2.4031067E-1,0E0,0E0,0E0,0E0,6.112082E-1,0E0,5.1428556E-2,2.237905E-1,0E0,2.0124912E-2,8.330685E-1,4.1008863E-1,2.7781165E-1,5.872886E-1,0E0,0E0,1.0314445E0,7.8946114E-1,0E0,1.6419601E-1,3.1073406E-2,0E0,0E0,2.267437E-1,1.7379001E-1,1.6358727E-1,1.2000694E0,5.5757487E-1,1.209774E0,0E0,0E0,1.19574584E-1,1.0060692E-1,2.834618E-1,3.6799222E-1,8.584547E-2,0E0,0E0,0E0,0E0,0E0,8.955145E-2,9.448173E-1,1.5888289E-1,1.902465E-1,0E0,0E0,2.8432906E-2,2.931623E-1,1.4658654E-1,6.612884E-1,0E0,0E0,5.201764E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.58561E-2,0E0,0E0,4.9219903E-1,1.1423969E0,0E0,0E0,0E0,2.5062943E-1,0E0,0E0,0E0,0E0,7.547602E-2,3.4448504E-1,9.6161604E-2,2.694549E-1,0E0,0E0,0E0,0E0,3.1316352E-1,2.9812407E-1,1.09964944E-1,0E0,0E0,1.4695047E-1,0E0,0E0,9.476501E-2,2.7362287E-2,1.2783307E-1,0E0,2.4405715E-1,3.026122E-1,2.9208374E-1,0E0,0E0,0E0,0E0,0E0,8.203459E-2,3.2339096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0684687E-1,1.1141836E-1,2.1343923E-1,1.0389882E-1,0E0,6.308546E-2,7.4564174E-2,6.2366217E-2,9.525315E-2,0E0,0E0,0E0,0E0,0E0,7.842755E-2,4.7456782E-2,1.1883013E-1,2.842903E-1,1.847666E-1,2.2224092E-1,6.528759E-2,0E0,0E0,3.612442E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,41,41,42,42,45,45,46,46,49,49,50,50,55,55,57,57,58,58,60,60,61,61,62,62,63,63,64,64,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,82,82,83,83,84,84,85,85,86,86,92,92,93,93,94,94,95,95,98,98,99,99,100,100,101,101,104,104,112,112,115,115,116,116,120,120,125,125,126,126,127,127,128,128,133,133,134,134,135,135,138,138,141,141,142,142,143,143,145,145,146,146,147,147,153,153,154,154,164,164,165,165,166,166,167,167,169,169,170,170,171,171,172,172,178,178,179,179,180,180,181,181,182,182,183,183,184,184,187,187],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,-1,74,76,-1,-1,78,80,-1,-1,82,84,-1,-1,-1,-1,86,-1,88,90,-1,92,94,96,98,100,-1,-1,102,104,-1,106,108,-1,-1,110,112,114,116,118,120,-1,-1,122,124,126,128,130,-1,-1,-1,-1,-1,132,134,136,138,-1,-1,140,142,144,146,-1,-1,148,-1,-1,-1,-1,-1,-1,-1,150,-1,-1,152,154,-1,-1,-1,156,-1,-1,-1,-1,158,160,162,164,-1,-1,-1,-1,166,168,170,-1,-1,172,-1,-1,174,176,178,-1,180,182,184,-1,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,190,192,194,196,-1,198,200,202,204,-1,-1,-1,-1,-1,206,208,210,212,214,216,218,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,2.16577E5,1E0,7.7E1,8.1E2,-2.8706862E-2,7.445455E2,1.05039425E1,6.76E2,4.58E2,7.502167E5,9E0,1.059448E7,8.8324326E-1,-4.9905527E-2,8.719101E0,2.74614E5,3.875E0,4.6E1,2.3E1,5.405423E6,5.766206E6,7.44488E5,1.767635E3,2.9251662E6,-4.0212203E-2,6.1454544E0,2E0,3.89E2,1.9E1,2.5E1,1.000501E6,3.808001E5,2.331083E6,-2.8434414E-2,1.88E2,2.22E3,5.163891E2,3.3852024E7,5.404E2,2.7421888E-2,2.3625128E6,1.6E1,3.0397184E-2,5.9613295E-2,3.0070068E7,2E0,-2.7440354E-2,-1.3120598E-2,3.6E2,4.60349E5,-1.4245516E-2,-5.5558696E-2,1.17730005E-2,3.238991E-2,1.7479E4,-2.9085081E-2,4.3827028E5,6.642857E0,-1.2324744E-2,1.9913E4,7.06E2,7.89E2,1.4473684E-1,1.445E3,-2.1648295E-2,4.8374466E-4,2.41E3,1.071E3,5.6159967E-3,1.639E3,6.185E3,1.8869437E-2,2.6119836E-3,1E0,1.124641E0,1.453785E6,1.0476191E0,9.4578236E-1,1E0,9.187501E-3,-1.7436896E-2,1.2E1,3.0588236E0,1.4338E4,1.6580646E2,5.25E0,-1.6737146E-2,-1.0280319E-2,3.6121877E-3,-9.114485E-3,-1.5027487E-2,4.795239E7,2.5493E4,1.6802971E6,5.02E2,-1.1673954E-2,-1.5445804E-3,4.957196E2,3.6086957E0,1.6533886E8,5.4007E7,-1.6641088E-2,-5.280974E-3,5.929839E8,2.5540506E-2,1.0539849E-2,1.3102842E-3,7.188572E-3,2.316542E-3,1.6000278E-2,9.190085E-3,2.8003533E0,-1.4457218E-2,-1.857162E-3,2.6402525E5,9.143923E4,-1.2682702E-2,9.166736E-3,8.893733E-3,1.06314E5,-7.426931E-3,2.338216E-3,-2.4966089E-2,-1.1738144E-2,9.9E1,1.8224286E2,2.9157894E0,3.741267E7,-2.4272174E-2,-1.04512945E-2,-3.1522036E-2,-1.8650971E-2,1E0,4.1361522E6,5.539245E6,6.840111E-3,-5.4356554E-3,1.6E1,1.0871677E-2,4.493147E-3,1.3E1,9.31E2,3.9960475E0,-4.58138E-3,3.774648E0,3.0416667E0,1.8062708E6,1.9443567E-3,-3.8519052E-3,2.0067447E-3,1.3277823E-2,-6.5351385E-3,3.1E2,4.907764E3,1.8452225E-2,3.918162E-2,-2.7686933E-3,-1.1893801E-2,-2.5489086E-2,-1.1202827E-2,-6.0975584E-3,-1.7337618E-2,-1.3765106E-2,4.7220547E1,7.784E3,1.6808511E0,2.28E0,-2.7890079E-2,2.0671213E2,1.3E1,1E0,3.5510652E6,-3.6964463E-3,5.314416E-3,-5.311406E-3,-1.0859397E-2,-2.8596302E-3,3.2892792E2,2.0467092E6,5.092769E7,9.51E3,1.7500242E7,1.724907E8,2.02E2,3.9302735E-3,1.2337889E-2,2.2040408E7,3.4941804E-2,-1.0424447E-4,-7.744306E-3,3.7727735E-3,-2.8495346E-3,-2.9315157E-3,-1.20281475E-2,-5.2722883E-3,-1.5432335E-2,-1.3650125E-3,5.629392E-3,-8.359306E-4,-7.7323965E-3,6.391512E-3,9.6006435E-5,-3.9546434E-3,3.7544388E-3,-2.256193E-4,6.871089E-3,-9.2264666E-4,3.528156E-3,-1.1525731E-2,-1.5668486E-3,1.1844793E-2,7.766568E-4,5.55371E-3,-2.998259E-3,8.021801E-3,1.7764298E-2,1.1768102E-2,2.491563E-2,2.4774328E-2,1.3376156E-2],"split_indices":[2,43,17,44,2,0,67,69,2,2,43,3,12,68,0,69,5,69,3,3,43,60,9,67,43,0,69,32,2,0,3,43,43,43,0,10,10,70,7,4,0,60,3,0,0,58,6,0,0,2,9,0,0,0,0,44,0,43,73,0,9,2,2,71,2,0,0,2,2,0,2,44,0,0,68,53,9,68,49,27,0,0,3,68,12,67,69,0,0,0,0,0,7,1,43,2,0,0,4,69,5,7,0,0,5,0,0,0,0,0,0,0,68,0,0,48,48,0,0,0,1,0,0,0,0,0,4,69,7,0,0,0,0,26,60,5,0,0,3,0,0,3,2,69,0,69,69,43,0,0,0,0,0,8,4,0,0,0,0,0,0,0,0,0,73,1,68,71,0,4,3,30,43,0,0,0,0,0,4,43,5,44,12,7,10,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,7.34E2,2.74E2,2.45E2,4.89E2,1.1E1,2.63E2,7.9E1,1.66E2,2.19E2,2.7E2,7.3E1,1.9E2,7.5E1,4E0,1.04E2,6.2E1,6.6E1,1.53E2,8.9E1,1.81E2,2.8E1,4.5E1,1.7E1,1.73E2,1.1E1,6.4E1,9.3E1,1.1E1,9E0,5.3E1,3.7E1,2.9E1,1.48E2,5E0,7.9E1,1E1,1.57E2,2.4E1,1.6E1,1.2E1,2E1,2.5E1,6E0,1.1E1,1.12E2,6.1E1,5.9E1,5E0,1.9E1,7.4E1,5E0,6E0,4E0,5E0,4.7E1,6E0,2.7E1,1E1,5E0,2.4E1,1.02E2,4.6E1,2.3E1,5.6E1,6E0,4E0,1.07E2,5E1,5E0,1.9E1,8E0,8E0,7E0,1.3E1,1.6E1,9E0,1.04E2,8E0,5.6E1,5E0,1E1,9E0,4.5E1,2.9E1,3.7E1,1E1,1.4E1,1.3E1,5E0,5E0,6E0,1.8E1,7E1,3.2E1,4.2E1,4E0,6E0,1.7E1,2.7E1,2.9E1,1.03E2,4E0,4E0,4.6E1,1.5E1,4E0,4E0,4E0,4E0,9E0,4E0,1.2E1,5E0,4E0,9E0,9.5E1,4E0,4E0,4E0,5.2E1,4E0,5E0,4.1E1,4E0,9E0,2E1,1.1E1,2.6E1,6E0,4E0,8E0,1E1,5.4E1,1.6E1,2.5E1,7E0,5E0,3.7E1,1.3E1,4E0,8E0,1.9E1,2.5E1,4E0,3.2E1,7.1E1,4E1,6E0,6E0,6E0,5E0,4E0,1.4E1,8.1E1,4E0,4.8E1,4E0,5E0,1.1E1,9E0,4E0,7E0,4E0,2.2E1,3E1,2.4E1,1.2E1,4E0,1E1,1.5E1,2.5E1,1.2E1,4E0,4E0,6E0,1.3E1,4E0,2.1E1,2.2E1,1E1,3.8E1,3.3E1,2.5E1,1.5E1,5E0,9E0,7.1E1,1E1,1.8E1,4E0,6E0,2.4E1,1E1,1.4E1,5E0,7E0,4E0,6E0,1.1E1,4E0,2.1E1,4E0,7E0,5E0,4E0,1.7E1,7E0,1.5E1,4E0,6E0,3.2E1,6E0,2.7E1,6E0,1.4E1,1.1E1,4E0,1.1E1,6E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[1.0388326E-2,-1.4030552E-1,4.7239754E-1,-4.348304E-1,-1.2833244E-2,-5.790869E-1,5.2626246E-1,-5.837202E-1,-3.2643813E-1,-1.360659E-1,1.2067847E-1,-3.274073E-2,-1.4311097E-2,3.134193E-1,6.3941395E-1,-5.323711E-1,-3.7736334E-2,-7.323976E-2,-4.166818E-1,-1.8608682E-1,4.474006E-2,2.010919E-1,1.8798243E-2,-1.12498924E-1,3.7654564E-1,6.8760234E-1,-7.1448244E-2,-3.0266145E-2,-3.587346E-1,-2.4892873E-1,2.1274523E-1,-7.369641E-1,-3.4930846E-1,-3.253191E-1,-1.24986306E-1,-9.495562E-3,1.6182142E-1,1.0867217E-1,3.2406023E-1,3.726113E-2,-1.8752895E-2,-1.0087916E-3,-8.045053E-3,4.3161398E-1,1.1904129E-2,1.1011767E0,6.295695E-1,-1.0426399E-2,7.3351082E-3,-2.4215284E-1,-2.5655964E-2,-3.673265E-1,-7.561075E-2,2.6332855E-2,-2.287433E-2,-1.8815547E-2,-8.797719E-1,-4.0034622E-1,-9.780679E-2,-3.643531E-1,-1.8678468E-1,-4.7936368E-1,-8.720403E-2,8.236923E-2,-7.038707E-2,2.3411028E-1,2.1924276E-3,7.45788E-2,2.2304257E-2,3.87757E-1,2.1035975E-2,6.268567E-2,-1.4882028E-1,5.064681E-1,2.4709468E-1,-6.804852E-3,7.813111E-3,3.4257486E-2,5.9078656E-2,4.0932286E-1,7.4131685E-1,-3.1504092E-1,1.901843E-3,-8.280606E-3,-2.0877583E-2,3.6218555E-3,-8.076267E-3,-3.437439E-3,2.0113823E-3,-5.5872116E-2,-2.396147E-2,-4.272993E-1,-2.299726E-1,-1.4218594E-2,-1.2882934E-2,-5.5001173E-3,-3.8231453E-1,-4.089209E-3,-9.711663E-3,-2.734341E-1,-3.8647167E-2,-4.0184204E-2,-2.0043829E-1,1.1519904E-2,8.872906E-3,-1.0236356E-2,4.9308194E-3,4.8747207E-3,1.436282E-2,1.385777E-1,-3.1050999E-2,3.0358925E-1,2.7466597E-2,-5.1160636E-3,7.912424E-3,2.2566853E-1,3.604424E-2,-1.4750997E-2,8.752503E-4,5.362695E-1,7.1535977E-3,2.5675008E-3,2.8888908E-1,9.555297E-3,4.518391E-1,8.5291106E-1,5.8210063E-1,-1.6858865E-2,-6.3681426E-3,-1.17612695E-2,-4.522976E-1,-5.4745353E-3,-1.2515189E-2,-5.9187347E-3,2.7138807E-3,-2.2638229E-1,-4.078414E-1,-5.9702927E-3,-1.7189099E-2,8.165699E-2,-8.0282755E-2,-2.1667428E-2,-1.5982631E-1,-1.5348726E-3,2.8551018E-3,7.133243E-2,-9.020835E-3,-3.4746742E-3,1.6944005E-1,-2.7185538E-1,1.14984185E-1,5.0500883E-3,3.2894447E-1,1.8673599E-3,1.366226E-2,-7.78244E-2,6.852726E-2,5.714867E-1,1.5875896E-2,5.0973957E-3,3.353413E-1,4.889097E-1,7.976E-3,1.4878836E-2,9.059156E-1,9.089502E-3,6.2051976E-1,-4.75416E-1,-1.33524025E-2,-1.3847336E-2,-4.9782274E-3,-1.9682078E-2,-1.0800449E-2,1.2242195E-1,-2.147313E-3,6.240829E-3,-9.374977E-2,-2.0394827E-1,5.0579747E-3,4.847221E-3,1.6002495E-4,2.4208382E-1,1.147353E-1,6.509094E-4,-2.2088427E-2,-5.4375856E-4,1.7014807E-1,2.3618416E-1,1.8400008E-2,1.3297886E-3,-1.37443E-1,-5.8076844E-2,1.0477056E-1,3.8360858E-1,2.9782811E-2,9.697665E-3,1.9074026E-2,5.2090096E-1,1.1925462E-2,1.968422E-2,4.3577157E-2,1.5370795E-2,2.9887285E-2,-1.8692313E-2,-2.606006E-2,1.0476403E-3,7.5558624E-3,-6.3167918E-3,-1.3731056E-3,-3.2650512E-3,-1.159511E-2,3.3349416E-3,1.3188432E-2,3.732447E-3,1.016734E-2,9.53655E-3,4.0367595E-3,1.3284303E-2,5.6239846E-3,-1.1354617E-2,-1.802359E-3,-5.509594E-3,2.946002E-3,1.3176004E-3,8.583301E-3,8.835795E-3,2.2807637E-2,2.6123086E-2,1.4159386E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,71,-1,-1,-1,73,75,77,79,-1,-1,81,-1,83,85,-1,87,-1,89,91,93,95,97,99,101,103,105,107,-1,109,-1,111,113,115,117,119,121,-1,-1,-1,-1,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,129,131,133,-1,-1,135,-1,-1,137,-1,139,141,143,-1,-1,145,-1,-1,147,149,151,-1,-1,-1,153,155,-1,-1,157,-1,-1,159,-1,161,163,165,-1,-1,-1,167,-1,-1,-1,-1,169,171,-1,-1,173,175,-1,177,-1,-1,179,-1,-1,181,183,185,-1,187,-1,-1,189,191,193,-1,-1,195,197,-1,-1,199,-1,201,203,-1,-1,-1,-1,-1,205,-1,-1,207,209,-1,-1,-1,211,213,-1,-1,-1,215,217,-1,-1,219,221,223,225,-1,-1,-1,227,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.2751595E1,2.9578901E1,1.4860748E1,3.6504135E0,9.082026E0,2.3421955E-1,5.700058E0,7.501259E-1,3.1684208E0,2.6039767E0,2.1672423E0,0E0,0E0,2.3814993E0,5.49572E0,1.5554047E0,0E0,1.9622456E0,1.9651566E0,1.8734517E0,4.0158534E-1,1.6467595E0,9.404141E-1,6.3259274E-2,1.520958E0,2.882866E0,4.264229E-1,0E0,7.135663E-1,4.647174E-1,1.2756351E0,4.4209957E-1,1.0775833E0,2.9965782E-1,2.0736883E0,2.519243E-1,1.5423673E-1,1.0609074E0,1.2078476E0,5.4866683E-1,0E0,0E0,0E0,7.864485E-1,3.0022097E-1,1.3768768E-1,2.969963E0,0E0,0E0,5.292337E-1,0E0,1.4171755E-1,1.8176433E-1,0E0,3.7461735E-2,0E0,6.820288E-1,2.3236275E-1,2.3817374E-1,1.8359327E-1,2.0534039E-2,8.745477E-1,7.5863826E-1,1.376725E-1,3.08537E-1,8.56778E-2,0E0,5.451183E-1,0E0,7.2448397E-1,2.5652924E-1,4.304177E-1,4.0643933E-1,3.9899254E-1,1.520387E-1,0E0,0E0,0E0,0E0,3.1118298E-1,1.0493011E0,1.2646317E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4823723E-1,2.2643149E-2,1.0724579E-1,0E0,0E0,9.87339E-2,0E0,0E0,1.0837275E-1,0E0,5.0977814E-1,3.8313055E-1,2.9049342E-2,0E0,0E0,2.6649553E-1,0E0,0E0,3.4264457E-1,1.1214024E0,1.5615511E-1,0E0,0E0,0E0,1.6973954E-1,3.3082727E-1,0E0,0E0,8.8534355E-2,0E0,0E0,1.0400152E-1,0E0,3.088894E-1,1.0869789E0,4.553089E-1,0E0,0E0,0E0,5.8882713E-2,0E0,0E0,0E0,0E0,4.601291E-2,2.702427E-2,0E0,0E0,1.4150064E-1,2.3889303E-1,0E0,4.7386074E-1,0E0,0E0,3.3830076E-2,0E0,0E0,1.4707863E-1,7.1447974E-1,1.4184761E-1,0E0,1.2459707E-1,0E0,0E0,1.3107294E-1,3.2269472E-1,2.201395E-1,0E0,0E0,3.974712E-2,1.0917282E-1,0E0,0E0,3.8630295E-1,0E0,5.074787E-2,4.80299E-2,0E0,0E0,0E0,0E0,0E0,7.8338176E-2,0E0,0E0,1.9958001E-1,1.8911779E-1,0E0,0E0,0E0,1.120739E-1,7.876623E-2,0E0,0E0,0E0,2.399829E-2,6.4926445E-2,0E0,0E0,1.2903346E-1,1.2687759E-1,3.2829916E-1,1.5965712E-1,0E0,0E0,0E0,9.413195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,43,43,44,44,45,45,46,46,49,49,51,51,52,52,54,54,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,69,69,70,70,71,71,72,72,73,73,74,74,79,79,80,80,81,81,91,91,92,92,93,93,96,96,99,99,101,101,102,102,103,103,106,106,109,109,110,110,111,111,115,115,116,116,119,119,122,122,124,124,125,125,126,126,130,130,135,135,136,136,139,139,140,140,142,142,145,145,148,148,149,149,150,150,152,152,155,155,156,156,157,157,160,160,161,161,164,164,166,166,167,167,173,173,176,176,177,177,181,181,182,182,186,186,187,187,190,190,191,191,192,192,193,193,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,72,-1,-1,-1,74,76,78,80,-1,-1,82,-1,84,86,-1,88,-1,90,92,94,96,98,100,102,104,106,108,-1,110,-1,112,114,116,118,120,122,-1,-1,-1,-1,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,130,132,134,-1,-1,136,-1,-1,138,-1,140,142,144,-1,-1,146,-1,-1,148,150,152,-1,-1,-1,154,156,-1,-1,158,-1,-1,160,-1,162,164,166,-1,-1,-1,168,-1,-1,-1,-1,170,172,-1,-1,174,176,-1,178,-1,-1,180,-1,-1,182,184,186,-1,188,-1,-1,190,192,194,-1,-1,196,198,-1,-1,200,-1,202,204,-1,-1,-1,-1,-1,206,-1,-1,208,210,-1,-1,-1,212,214,-1,-1,-1,216,218,-1,-1,220,222,224,226,-1,-1,-1,228,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,1.2E2,9.48E2,9.3326636E-2,9.9347424E2,5.17259E8,1.5522388E0,2.331083E6,3.887931E0,-3.274073E-2,-1.4311097E-2,2.8204132E7,3.206931E2,3E1,-3.7736334E-2,1.5292561E2,1E0,4.67E2,2.9254214E2,4.371613E2,2.5401869E1,1.1E1,2.2E1,1.059448E7,4.7263342E2,-3.0266145E-2,1.388E1,1.2E1,2.3E1,1.654E3,3.2874417E2,1.8154167E2,2.86152E5,1.4285715E0,9.244374E6,2.1578948E0,1E0,1E0,-1.8752895E-2,-1.0087916E-3,-8.045053E-3,1.835821E1,1.9719212E1,4.626E3,1.3711089E6,-1.0426399E-2,7.3351082E-3,4.7837E4,-2.5655964E-2,1.16E0,1.5077367E3,2.6332855E-2,9E0,-1.8815547E-2,6.27907E0,7.997723E6,3.478E4,1.9E1,1.2707424E0,1.2916666E0,2.712E3,5.405423E6,8.611817E6,4.6E1,2.1924276E-3,2.5E0,2.2304257E-2,1.28738E5,2.9149944E5,9E0,4.265829E2,2E0,3.202146E5,-6.804852E-3,7.813111E-3,3.4257486E-2,5.9078656E-2,9.49372E4,1.7103828E7,3.015873E0,1.901843E-3,-8.280606E-3,-2.0877583E-2,3.6218555E-3,-8.076267E-3,-3.437439E-3,2.0113823E-3,-5.5872116E-2,-2.396147E-2,7.090909E0,2.29E2,4.7280664E3,-1.2882934E-2,-5.5001173E-3,2.6003375E5,-4.089209E-3,-9.711663E-3,1E0,-3.8647167E-2,1.3364486E0,1.19111115E2,1.4E1,8.872906E-3,-1.0236356E-2,1.7544615E0,4.8747207E-3,1.436282E-2,1.007E3,2.026453E6,3.655004E7,2.7466597E-2,-5.1160636E-3,7.912424E-3,1.0542694E1,3.82E2,-1.4750997E-2,8.752503E-4,2.2539758E5,7.1535977E-3,2.5675008E-3,1E0,9.555297E-3,4.4895835E0,1E0,1E0,-1.6858865E-2,-6.3681426E-3,-1.17612695E-2,6.926E3,-5.4745353E-3,-1.2515189E-2,-5.9187347E-3,2.7138807E-3,3.8E1,9.860918E5,-5.9702927E-3,-1.7189099E-2,1E0,1.4E1,-2.1667428E-2,2.0200005E8,-1.5348726E-3,2.8551018E-3,1.6E1,-9.020835E-3,-3.4746742E-3,5.92E2,3.1888113E0,6.642857E0,5.0500883E-3,5.5885565E2,1.8673599E-3,1.366226E-2,1.7161617E2,1E0,4.6847186E2,1.5875896E-2,5.0973957E-3,5.489183E1,1E0,7.976E-3,1.4878836E-2,1.32476E5,9.089502E-3,1.5822886E0,1.5E1,-1.33524025E-2,-1.3847336E-2,-4.9782274E-3,-1.9682078E-2,-1.0800449E-2,8.754E3,-2.147313E-3,6.240829E-3,7.09E2,1.08E2,5.0579747E-3,4.847221E-3,1.6002495E-4,1.124E3,1.25789E5,6.509094E-4,-2.2088427E-2,-5.4375856E-4,2.23099E7,1.3881669E6,1.8400008E-2,1.3297886E-3,1.6869704E6,4.888889E0,4.162742E5,5.941442E6,2.9782811E-2,9.697665E-3,1.9074026E-2,2.9E1,1.1925462E-2,1.968422E-2,4.3577157E-2,1.5370795E-2,2.9887285E-2,-1.8692313E-2,-2.606006E-2,1.0476403E-3,7.5558624E-3,-6.3167918E-3,-1.3731056E-3,-3.2650512E-3,-1.159511E-2,3.3349416E-3,1.3188432E-2,3.732447E-3,1.016734E-2,9.53655E-3,4.0367595E-3,1.3284303E-2,5.6239846E-3,-1.1354617E-2,-1.802359E-3,-5.509594E-3,2.946002E-3,1.3176004E-3,8.583301E-3,8.835795E-3,2.2807637E-2,2.6123086E-2,1.4159386E-2],"split_indices":[2,43,17,44,2,53,67,46,68,43,69,0,0,5,73,70,0,67,29,2,70,67,69,3,3,12,71,0,71,3,0,1,67,67,7,71,60,68,6,19,0,0,0,71,71,2,43,0,0,9,0,69,48,0,8,0,71,9,44,67,68,68,44,43,60,0,0,69,0,1,48,3,67,32,43,0,0,0,0,48,62,68,0,0,0,0,0,0,0,0,0,73,10,48,0,0,43,0,0,8,0,68,67,3,0,0,68,0,0,2,9,7,0,0,0,71,4,0,0,48,0,0,29,0,68,15,26,0,0,0,10,0,0,0,0,0,43,0,0,26,10,0,5,0,0,3,0,0,10,71,73,0,4,0,0,67,29,4,0,0,71,23,0,0,1,0,53,8,0,0,0,0,0,1,0,0,2,10,0,0,0,2,1,0,0,0,9,43,0,0,60,69,62,60,0,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.043E3,7.87E2,2.56E2,2.37E2,5.5E2,1.2E1,2.44E2,9.8E1,1.39E2,2.86E2,2.64E2,7E0,5E0,8.6E1,1.58E2,8.3E1,1.5E1,3.7E1,1.02E2,2.24E2,6.2E1,1.47E2,1.17E2,1.1E1,7.5E1,1.48E2,1E1,4.7E1,3.6E1,2.3E1,1.4E1,1.6E1,8.6E1,6.7E1,1.57E2,4.3E1,1.9E1,8.5E1,6.2E1,1.13E2,4E0,5E0,6E0,6.5E1,1E1,1.6E1,1.32E2,6E0,4E0,2.4E1,1.2E1,1.3E1,1E1,5E0,9E0,6E0,1E1,7.1E1,1.5E1,5.1E1,1.6E1,1.4E1,1.43E2,1.7E1,2.6E1,1.1E1,8E0,7.9E1,6E0,5.1E1,1.1E1,1E2,1.3E1,4.5E1,2E1,5E0,5E0,7E0,9E0,4.6E1,8.6E1,1.9E1,5E0,5E0,8E0,4E0,6E0,5E0,4E0,4E0,6E0,6E1,1.1E1,1.1E1,4E0,4E0,4.7E1,4E0,1.2E1,1E1,4E0,1.02E2,4.1E1,1.1E1,6E0,8E0,1.8E1,5E0,6E0,4.9E1,3E1,3.8E1,1.3E1,6E0,5E0,1.3E1,8.7E1,6E0,7E0,4.1E1,4E0,4E0,1.6E1,9E0,3.7E1,4.8E1,3.8E1,1.4E1,5E0,9E0,5.1E1,4E0,7E0,4E0,7E0,8E0,3.9E1,5E0,5E0,2.5E1,7.7E1,4E0,3.7E1,6E0,5E0,1.4E1,4E0,6E0,4.3E1,1.1E1,1.9E1,5E0,3.3E1,4E0,9E0,1.9E1,6.8E1,3.3E1,8E0,4E0,1.2E1,3.2E1,5E0,5E0,4.3E1,4E0,3.4E1,4.3E1,8E0,4E0,4E0,3.4E1,5E0,1.9E1,6E0,4E0,7.3E1,3.2E1,5E0,9E0,5E0,1.7E1,2.6E1,5E0,6E0,6E0,1.3E1,1.6E1,1.7E1,7E0,1.2E1,1.5E1,5.3E1,1.1E1,2.2E1,6E0,6E0,2.7E1,5E0,4E0,3.9E1,4E0,3E1,2.7E1,1.6E1,6E0,1.3E1,4.3E1,3E1,9E0,2.3E1,4E0,1.3E1,2.1E1,5E0,8E0,5E0,1E1,6E0,5E0,7E0,1E1,5E0,2.8E1,2.5E1,5E0,6E0,2.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"229","size_leaf_vector":"1"}},{"base_weights":[-6.069969E-3,-1.39584E-1,4.168373E-1,-4.165128E-1,-1.692058E-2,4.6412694E-1,-5.636922E-1,-3.6771145E-1,-6.8826175E-1,-1.2340638E-1,9.475499E-2,1.1592902E0,4.1920882E-1,-3.1316992E-2,-1.239372E-2,-2.764794E-2,-2.95202E-1,-5.5115956E-1,-5.0783135E-2,-1.905333E-1,1.5783371E-2,1.8670271E-1,9.308463E-3,3.617117E-2,6.5777734E-2,2.5238645E-1,5.974901E-1,9.19941E-2,-3.5077587E-1,-9.668635E-3,-6.3124573E-1,-2.6704106E-1,-6.037407E-2,-1.902264E-2,1.8447621E-1,1.4883727E-1,4.2964143E-1,2.7907675E-2,-1.4016416E-2,2.1528159E-1,3.7666537E-2,4.4670904E-1,7.004935E-1,1.843852E-2,-9.007192E-2,-3.835936E-1,-1.4497967E-1,-3.1929143E-2,-1.8278925E-2,-1.8729578E-1,-4.479107E-1,8.411741E-3,-1.7532864E-1,-6.9576964E-2,5.9868664E-2,1.3444636E-2,9.1451146E-2,1.2956489E-2,2.4056609E-1,9.602993E-3,2.3317957E-2,-1.4078775E-3,2.4419786E-1,5.267276E-2,2.875718E-1,4.47477E-3,4.7700074E-1,7.6333183E-1,3.698196E-1,-7.621937E-3,3.8831128E-4,-3.9534763E-1,-4.2695575E-3,-4.9766037E-3,-2.3075259E-1,-2.910212E-1,-9.30281E-2,-3.7576923E-1,-3.2020617E-2,7.1031146E-2,-1.0258791E-1,5.824676E-4,-2.4173677E-1,-2.1200903E-1,1.5893976E-3,1.5678743E-2,1.6614638E-1,5.87003E-3,1.405876E-3,1.10222556E-1,-2.0189263E-1,-1.3502848E-3,2.624422E-1,1.11344926E-1,-3.5708643E-2,8.19227E-4,3.783038E-1,1.8270288E-1,-2.8766137E-2,3.2911843E-1,-8.354947E-3,3.5599896E-1,2.6182918E-2,7.9507124E-1,1.4135402E-2,3.64063E-3,2.335258E-2,-3.3707044E-1,-4.7670263E-1,5.5863406E-3,-6.0001863E-3,-1.6127607E-2,-1.5943821E-3,-2.9655686E-3,-3.1444147E-1,2.7288396E-2,-1.230318E-1,-1.0543343E-2,-4.281391E-1,3.4443801E-3,7.352528E-3,1.4278679E-3,-1.4604068E-1,-1.630179E-2,-9.5327705E-2,-7.322823E-2,-1.9916672E-2,3.375031E-2,-5.668018E-3,6.965579E-2,-4.3535763E-3,1.0984553E-2,2.831896E-3,2.071923E-1,-5.757317E-3,-1.9149872E-2,-2.6035389E-2,2.8901154E-1,1.10567905E-1,-4.011358E-2,1.9771707E-1,-4.7556735E-3,-2.2344165E-1,2.1848662E-2,8.736158E-3,1.1240756E-3,1.1254942E-2,2.849864E-2,-9.044208E-3,9.710715E-2,3.6936462E-1,2.4912652E-2,2.4675043E-1,3.7781075E-2,2.0149276E-2,-2.4418607E-1,-3.8531062E-1,-3.404038E-2,-4.1112208E-1,-9.160771E-3,-1.5732905E-2,-1.6468738E-3,4.4976445E-3,-6.2192682E-2,-1.7739557E-1,-2.175964E-2,-1.1202959E-2,-5.877514E-2,4.0916502E-3,-9.827798E-3,-1.5146399E-3,-7.0481887E-3,-3.5027225E-4,-7.862704E-3,2.9995697E-4,8.753806E-2,-3.038841E-3,-7.617977E-4,5.0962986E-3,1.0533791E-1,1.6271347E-2,1.11285366E-1,-6.3159033E-3,5.706902E-3,-9.936016E-3,1.4836109E-1,3.144091E-1,1.5741846E-3,6.902647E-3,-7.4920487E-3,2.439277E-3,3.5132426E-3,1.0489423E-2,-2.4694036E-1,2.5583463E-2,-1.4868317E-2,-4.3025813E-3,4.087016E-3,-2.8321285E-2,5.378837E-4,7.2296844E-3,3.958921E-1,2.7818275E-3,4.9096565E-3,1.3313295E-2,-3.2483886E-3,-1.3930216E-2,-1.887233E-2,-9.289061E-3,-2.327068E-2,-1.4219126E-2,-7.5650658E-3,-1.258577E-4,-1.0275359E-2,-1.9765997E-3,1.2900188E-3,-5.1884297E-3,5.4901233E-4,5.962981E-3,-1.8559955E-3,2.0216622E-3,7.915611E-3,1.5577816E-3,1.4711095E-3,7.7831917E-3,8.856118E-3,3.0809408E-3,1.1301039E-2,1.7829586E-2,-5.616756E-3,-1.4918382E-2,1.1033503E-2,4.401701E-4,-4.265691E-3,1.3740485E-3,2.468877E-2,1.4867516E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,29,-1,31,33,35,37,-1,-1,39,41,43,45,-1,47,49,51,53,55,57,59,61,-1,63,-1,65,67,-1,69,71,73,-1,-1,75,77,79,81,83,85,-1,87,89,91,-1,-1,93,95,97,99,-1,101,103,105,-1,-1,107,-1,109,111,113,115,117,-1,119,121,-1,123,125,127,129,131,-1,-1,133,135,-1,137,139,141,-1,143,145,147,149,-1,151,-1,153,-1,-1,-1,155,157,-1,-1,-1,-1,-1,159,161,163,-1,165,167,-1,-1,169,-1,171,173,-1,175,-1,177,-1,-1,-1,179,181,183,-1,185,187,189,191,193,195,-1,-1,-1,-1,197,-1,199,201,-1,203,-1,-1,205,207,-1,209,-1,-1,-1,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,217,219,-1,-1,221,-1,223,-1,-1,-1,225,227,-1,-1,-1,-1,-1,-1,229,231,-1,-1,-1,233,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.906312E1,2.699833E1,1.1900219E1,2.9827728E0,6.576001E0,7.008114E0,2.2292304E-1,3.320097E0,1.422535E0,2.6471834E0,2.119548E0,2.7886963E-1,6.600483E0,0E0,0E0,0E0,3.496665E0,6.619015E-1,0E0,1.8858919E0,5.49561E-1,1.1319208E0,8.285201E-1,0E0,0E0,2.3100548E0,1.3799553E0,1.2101727E0,8.966675E-1,0E0,5.0992966E-2,1.6469278E0,5.707963E-1,3.152526E-1,1.3195938E-1,1.4178E0,1.7818308E-1,8.4999305E-1,0E0,1.3131709E0,0E0,4.5188427E-1,1.0267811E0,0E0,1.04661435E-1,3.7758636E-1,2.5102952E-1,0E0,0E0,8.040533E-1,3.7633705E-1,3.2702065E-1,3.4002143E-1,4.8688287E-1,1.4298357E-1,0E0,1.786182E-2,1.0044371E0,4.0900922E-1,0E0,0E0,4.6420935E-1,4.7921723E-1,3.86478E-1,1.5774946E0,0E0,2.651682E-1,5.1159286E-1,4.3391085E-1,0E0,0E0,3.8928795E-1,0E0,1.5722403E-1,2.8984672E-1,1.92312E-1,1.6941464E-1,1.3411283E-1,0E0,1.7811963E-1,1.03125855E-1,0E0,2.9175115E-1,4.5493335E-1,1.3603643E-1,1.430191E-1,5.3008273E-2,0E0,0E0,3.723287E-1,9.8055714E-1,0E0,2.2226238E-1,3.7422144E-1,5.3353965E-1,0E0,8.959854E-2,1.291348E-1,2.2553302E-1,6.3643646E-1,0E0,2.873497E-1,0E0,3.472519E-2,0E0,0E0,0E0,2.240467E-1,4.8177433E-1,0E0,0E0,0E0,0E0,0E0,4.1629553E-2,4.756647E-2,1.1134267E-1,0E0,5.2872658E-2,1.0053718E-1,0E0,0E0,9.120992E-2,0E0,4.9063466E-2,8.613539E-2,0E0,5.3693768E-2,0E0,5.7856083E-2,0E0,0E0,0E0,2.2875565E-1,2.6044804E-1,3.3405617E-1,0E0,1.4267969E-1,2.7592897E-2,1.33308E-1,4.6972096E-2,5.941316E-1,1.3670337E-1,0E0,0E0,0E0,0E0,6.383695E-2,0E0,5.907937E-2,4.764223E-1,0E0,5.6659102E-2,0E0,0E0,2.4798572E-1,1.1013079E-1,0E0,2.1811104E-1,0E0,0E0,0E0,0E0,1.1430286E-1,1.05626225E-1,0E0,0E0,5.4187227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.303025E-2,3.1301804E-2,0E0,0E0,5.094424E-2,0E0,3.5522006E-2,0E0,0E0,0E0,2.2607997E-2,1.252265E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.6460927E-2,2.4423552E-1,0E0,0E0,0E0,3.991334E-2,0E0,0E0,4.0260315E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,61,61,62,62,63,63,64,64,66,66,67,67,68,68,71,71,73,73,74,74,75,75,76,76,77,77,79,79,80,80,82,82,83,83,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,96,96,97,97,98,98,99,99,101,101,103,103,107,107,108,108,114,114,115,115,116,116,118,118,119,119,122,122,124,124,125,125,127,127,129,129,133,133,134,134,135,135,137,137,138,138,139,139,140,140,141,141,142,142,147,147,149,149,150,150,152,152,155,155,156,156,158,158,163,163,164,164,167,167,175,175,176,176,179,179,181,181,185,185,186,186,193,193,194,194,198,198,201,201],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,30,-1,32,34,36,38,-1,-1,40,42,44,46,-1,48,50,52,54,56,58,60,62,-1,64,-1,66,68,-1,70,72,74,-1,-1,76,78,80,82,84,86,-1,88,90,92,-1,-1,94,96,98,100,-1,102,104,106,-1,-1,108,-1,110,112,114,116,118,-1,120,122,-1,124,126,128,130,132,-1,-1,134,136,-1,138,140,142,-1,144,146,148,150,-1,152,-1,154,-1,-1,-1,156,158,-1,-1,-1,-1,-1,160,162,164,-1,166,168,-1,-1,170,-1,172,174,-1,176,-1,178,-1,-1,-1,180,182,184,-1,186,188,190,192,194,196,-1,-1,-1,-1,198,-1,200,202,-1,204,-1,-1,206,208,-1,210,-1,-1,-1,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,218,220,-1,-1,222,-1,224,-1,-1,-1,226,228,-1,-1,-1,-1,-1,-1,230,232,-1,-1,-1,234,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.7909248E5,9.7172376E7,1E0,9.31E2,2.4481E4,1.665909E3,3.89E2,3.9935064E0,1.5859042E6,3.7326388E0,6.23694E5,1.8062708E6,-3.1316992E-2,-1.239372E-2,-2.764794E-2,9.83871E-1,1.0013907E1,-5.0783135E-2,2.956111E2,5.5426865E6,5.405423E6,1.2294118E1,3.617117E-2,6.5777734E-2,7.6255713E3,6.355E3,2E0,3.3238492E2,-9.668635E-3,1.6074808E5,3.4698796E0,1.934E3,2.732381E2,5.894636E6,1.084437E6,1E0,2.445E3,-1.4016416E-2,7.075161E2,3.7666537E-2,1E0,5.6926E-3,1.843852E-2,6.923077E-1,1.803948E7,1.1544592E1,-3.1929143E-2,-1.8278925E-2,4.33E2,5.4051723E0,1E0,1.2E1,2.331083E6,1.2397727E1,1.3444636E-2,1.4E1,3.72381E5,3.7760816E5,9.602993E-3,2.3317957E-2,9.777778E0,9.68E2,2.0421052E0,3.0070068E7,4.47477E-3,1.0089981E3,1E0,1.3655363E2,-7.621937E-3,3.8831128E-4,1.2E1,-4.2695575E-3,1E0,1.1939979E8,1.0201707E-3,9E0,7.106795E6,-3.2020617E-2,3.82E2,1.1652361E6,5.824676E-4,3.90225E2,3E0,2.0487332E-3,1E0,1.5539158E3,5.87003E-3,1.405876E-3,1.2562814E0,6.9508715E0,-1.3502848E-3,2.23099E7,1.5026077E7,2.0734E4,8.19227E-4,3.4779343E1,3.9896116E7,1.7E1,6.996357E4,-8.354947E-3,2.5959E4,2.6182918E-2,2.4080592E7,1.4135402E-2,3.64063E-3,2.335258E-2,1.4E1,1.63757E4,5.5863406E-3,-6.0001863E-3,-1.6127607E-2,-1.5943821E-3,-2.9655686E-3,1E0,6.407767E-1,1.6808511E0,-1.0543343E-2,3.1540035E9,2.8E1,7.352528E-3,1.4278679E-3,5.9506125E8,-1.630179E-2,2.335E3,3.39E2,-1.9916672E-2,3.5364E4,-5.668018E-3,5.44E2,-4.3535763E-3,1.0984553E-2,2.831896E-3,2.9080848E6,1.57E2,4.5154482E-1,-2.6035389E-2,5.1604336E7,4.8E2,1.1E1,3.7788504E7,2.1924414E2,2.2503355E1,2.1848662E-2,8.736158E-3,1.1240756E-3,1.1254942E-2,1.1E1,-9.044208E-3,2.829581E10,4.2964826E0,2.4912652E-2,4E0,3.7781075E-2,2.0149276E-2,2E0,4.6463413E0,-3.404038E-2,1.547E3,-9.160771E-3,-1.5732905E-2,-1.6468738E-3,4.4976445E-3,1.407037E2,6.9E2,-2.175964E-2,-1.1202959E-2,1.4473684E-1,4.0916502E-3,-9.827798E-3,-1.5146399E-3,-7.0481887E-3,-3.5027225E-4,-7.862704E-3,2.9995697E-4,1.2E1,6.47E2,-7.617977E-4,5.0962986E-3,7.7983914E4,1.6271347E-2,1.7638037E0,-6.3159033E-3,5.706902E-3,-9.936016E-3,4.6E1,4.797342E2,1.5741846E-3,6.902647E-3,-7.4920487E-3,2.439277E-3,3.5132426E-3,1.0489423E-2,1.196966E6,7E0,-1.4868317E-2,-4.3025813E-3,4.087016E-3,2.8003533E0,5.378837E-4,7.2296844E-3,9.643839E6,2.7818275E-3,4.9096565E-3,1.3313295E-2,-3.2483886E-3,-1.3930216E-2,-1.887233E-2,-9.289061E-3,-2.327068E-2,-1.4219126E-2,-7.5650658E-3,-1.258577E-4,-1.0275359E-2,-1.9765997E-3,1.2900188E-3,-5.1884297E-3,5.4901233E-4,5.962981E-3,-1.8559955E-3,2.0216622E-3,7.915611E-3,1.5577816E-3,1.4711095E-3,7.7831917E-3,8.856118E-3,3.0809408E-3,1.1301039E-2,1.7829586E-2,-5.616756E-3,-1.4918382E-2,1.1033503E-2,4.401701E-4,-4.265691E-3,1.3740485E-3,2.468877E-2,1.4867516E-2],"split_indices":[2,43,60,6,2,12,4,1,68,43,69,1,43,0,0,0,68,71,0,4,43,43,68,0,0,67,2,10,67,0,48,71,44,4,60,9,29,2,0,67,0,15,72,0,68,9,71,0,0,2,69,26,3,43,73,0,3,9,43,0,0,73,10,68,58,0,4,6,71,0,0,8,0,13,7,53,3,5,0,4,60,0,67,8,53,26,48,0,0,68,71,0,9,5,44,0,71,5,3,48,0,9,0,9,0,0,0,3,43,0,0,0,0,0,73,71,68,0,46,0,0,0,46,0,44,44,0,9,0,12,0,0,0,66,0,53,0,7,10,3,7,67,71,0,0,0,0,3,0,46,68,0,8,0,0,32,68,0,10,0,0,0,0,67,2,0,0,71,0,0,0,0,0,0,0,3,2,0,0,48,0,68,0,0,0,0,4,0,0,0,0,0,0,9,3,0,0,0,68,0,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.044E3,7.94E2,2.5E2,2.43E2,5.51E2,2.39E2,1.1E1,2.08E2,3.5E1,2.82E2,2.69E2,1.3E1,2.26E2,7E0,4E0,4.8E1,1.6E2,2.8E1,7E0,1.9E2,9.2E1,1.29E2,1.4E2,7E0,6E0,1.18E2,1.08E2,2E1,1.4E2,6E0,2.2E1,1.19E2,7.1E1,7.7E1,1.5E1,1.13E2,1.6E1,1.33E2,7E0,1.12E2,6E0,4.6E1,6.2E1,7E0,1.3E1,1.2E2,2E1,1.6E1,6E0,8.4E1,3.5E1,4.5E1,2.6E1,4.7E1,3E1,6E0,9E0,4.6E1,6.7E1,5E0,1.1E1,1.18E2,1.5E1,3.5E1,7.7E1,4E0,4.2E1,5.1E1,1.1E1,7E0,6E0,1.15E2,5E0,8E0,1.2E1,3.9E1,4.5E1,2.9E1,6E0,2.9E1,1.6E1,7E0,1.9E1,1.5E1,3.2E1,2.2E1,8E0,5E0,4E0,3.2E1,1.4E1,5E0,6.2E1,2.7E1,9.1E1,6E0,9E0,1.3E1,2.2E1,7.1E1,6E0,2E1,2.2E1,4.7E1,4E0,4E0,7E0,7E1,4.5E1,4E0,4E0,7E0,5E0,4E0,3.5E1,9E0,3.6E1,9E0,2E1,1.7E1,1.2E1,4E0,1.2E1,1E1,9E0,1E1,5E0,2.6E1,6E0,1.5E1,7E0,4E0,4E0,1.7E1,1.5E1,1E1,4E0,5.2E1,1E1,1E1,1.7E1,7.9E1,1.2E1,5E0,4E0,4E0,9E0,1.7E1,5E0,1.1E1,6E1,6E0,1.4E1,4.3E1,4E0,2.6E1,4.4E1,7E0,3.8E1,8E0,2.7E1,5E0,4E0,1.8E1,1.8E1,1.5E1,5E0,1E1,7E0,7E0,5E0,5E0,4E0,4E0,6E0,1E1,1.6E1,5E0,1E1,1.1E1,6E0,8E0,7E0,6E0,4E0,9E0,4.3E1,4E0,6E0,4E0,6E0,4E0,1.3E1,8E0,7.1E1,6E0,6E0,8E0,9E0,5E0,6E0,5.5E1,5E0,4E0,1E1,7E0,1.9E1,3.8E1,6E0,1.8E1,2E1,6E0,1.2E1,1.3E1,5E0,4E0,6E0,4E0,6E0,9E0,7E0,5E0,6E0,4E0,4E0,5E0,4E0,2.4E1,1.9E1,4E0,4E0,4E0,6.7E1,4E0,5E0,1.7E1,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[8.597817E-3,-1.2879427E-1,4.256468E-1,-3.8591677E-1,-3.969808E-3,4.8857808E-1,-3.875296E-2,-5.740737E-1,-3.0354354E-1,-1.351161E-1,9.396026E-2,6.364543E-2,4.5750412E-1,-4.631766E-1,1.5592837E-1,-4.4338825E-1,-7.1076024E-1,-3.5070378E-1,2.542251E-2,-2.0304939E-1,-1.0437911E-2,5.080184E-2,2.8214893E-1,3.692196E-1,6.972939E-1,-2.5734607E-2,-1.1915106E-2,2.7503145E-1,-6.402389E-3,-4.653785E-1,-1.0974521E-2,-6.29406E-1,-4.9571794E-2,1.263199E-2,-3.7785378E-1,1.9859787E-2,-1.10807136E-1,-1.7515352E-1,-4.2941716E-1,1.7590694E-2,-1.39172375E-2,6.307764E-2,-2.044582E-2,8.2467906E-2,3.35956E-1,1.3044524E-1,4.5310163E-1,7.579595E-1,2.5552644E-3,2.8878017E-3,3.6595058E-1,-2.2936359E-2,-1.2272982E-2,-3.387607E-2,-4.0385985E-1,-3.350037E-1,-6.8834525E-1,-1.1423461E-2,-3.9302424E-2,-2.7000123E-1,-1.2789492E-1,-1.2143544E-2,-2.9357424E-2,-1.623632E-2,8.819292E-3,1.4592235E-1,1.4928196E-2,9.02584E-5,5.4531014E-3,3.8234964E-1,3.935705E-3,7.739058E-2,1.6302979E-2,6.589423E-1,3.963303E-1,4.2970426E-2,2.8703295E-2,2.2502005E-2,8.49337E-3,-2.2833815E-2,-1.067643E-2,-4.057083E-1,-2.4064533E-1,-1.3344708E-2,-8.307296E-1,1.5412492E-3,-7.4951584E-3,-3.0325803E-1,-4.192686E-3,-8.940315E-2,-2.3930645E-1,-9.753107E-3,8.43118E-4,8.251604E-2,2.4233465E-1,-2.1061499E-2,1.0581072E-1,1.8188989E-1,4.331278E-1,1.7360204E-1,3.889039E-2,3.307639E-2,1.3818471E-2,5.214149E-4,4.1759622E-1,-2.628421E-3,-4.3016022E-1,-3.1865135E-2,-3.243929E-1,-5.8177676E-2,-2.1207329E-2,-5.5476143E-3,-1.4923227E-2,-1.2677565E-1,-1.2395965E-2,-4.294914E-3,-2.88374E-1,-4.682604E-2,9.375582E-2,1.2210748E-1,-1.5241143E-2,2.8454947E-1,-2.74521E-3,-6.2461957E-2,9.004178E-2,-7.915614E-2,1.718667E-1,2.2793636E-3,1.2846289E-2,1.1451453E-2,2.2317665E-2,2.1162769E-3,1.1603518E-2,-4.8818752E-2,1.1231229E-1,2.7839504E-3,4.4240287E-1,-4.7453263E-1,-3.1379494E-1,-1.2640013E-2,5.6744467E-2,-2.9149225E-1,-2.0662732E-2,-1.7108446E-1,-5.713807E-2,-6.658633E-2,6.471014E-2,-1.6361587E-2,-8.2384255E-3,1.0217967E-2,-9.620748E-2,5.180747E-3,1.4862325E-3,7.362799E-2,3.0866265E-1,1.6457209E-1,3.666162E-1,-8.473663E-2,1.3333295E-2,1.8417571E-1,-6.650711E-2,-5.2396295E-5,-8.692818E-3,1.1907593E-1,1.209602E-2,3.9888298E-4,-5.4185004E-3,8.2136495E-3,5.5828143E-4,4.9335462E-1,1.9502313E-1,-1.8970046E-2,-2.715469E-2,-7.6681697E-3,-1.55885415E-2,-1.7287824E-3,1.0797878E-2,-1.7378371E-2,-1.0187471E-2,7.745435E-5,-9.295376E-3,2.5393679E-3,-4.1901073E-3,-2.0631547E-4,-4.2035575E-3,5.1161516E-3,-1.6028775E-4,-1.9313481E-3,4.868096E-3,-5.922737E-3,1.4695912E-3,-1.0086013E-3,5.485132E-3,6.548525E-3,1.8309431E-2,1.1172498E-2,2.4375792E-3,1.9502638E-2,8.983798E-3,6.6987835E-3,-5.088571E-3,1.7708457E-3,1.0187459E-2,-7.0623914E-3,2.0943263E-3,7.5206594E-3,9.939047E-5,1.6885199E-2,2.5971701E-2,2.066766E-3,1.5694655E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,-1,51,-1,53,-1,-1,55,-1,57,59,61,63,-1,65,-1,67,69,71,73,75,-1,-1,77,-1,-1,-1,79,81,83,-1,85,87,89,-1,-1,91,-1,93,95,-1,-1,97,-1,99,-1,101,103,-1,-1,-1,-1,-1,-1,105,107,-1,109,-1,-1,111,-1,113,115,-1,117,119,121,123,125,127,129,131,133,-1,-1,-1,135,-1,137,139,141,-1,-1,-1,-1,143,145,-1,147,149,151,153,-1,155,-1,157,159,161,163,-1,-1,-1,-1,-1,-1,165,167,-1,169,171,173,-1,175,177,-1,179,181,183,185,-1,-1,187,189,-1,-1,191,193,195,197,199,-1,201,203,-1,-1,205,-1,-1,-1,-1,-1,207,209,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.827084E1,2.4551277E1,7.3837013E0,3.7005157E0,6.6397686E0,5.5028152E0,2.6336706E0,1.0245762E0,2.7394009E0,1.8697062E0,2.3897562E0,0E0,4.3152046E0,4.9002647E-2,8.098125E-1,7.503605E-2,3.5609818E-1,2.712862E0,1.3148947E0,8.277917E-1,6.492217E-1,1.4935653E0,5.6322813E-1,3.1455154E0,2.1780167E0,0E0,0E0,2.8628445E-1,0E0,9.164715E-2,0E0,4.3002796E-1,0E0,0E0,1.7384796E0,0E0,1.6591617E-1,5.436938E-1,3.4774804E-1,4.3105894E-1,0E0,9.4319725E-1,0E0,3.699687E-2,4.6768665E-1,4.8015428E-1,1.1177711E0,6.4554214E-1,0E0,0E0,1.5307367E-1,0E0,0E0,0E0,7.31715E-2,7.830162E-1,7.2702694E-1,0E0,1.2153871E-1,2.2285032E-1,3.559867E-1,0E0,0E0,2.0653522E-1,0E0,5.118475E-1,4.961245E-1,0E0,0E0,2.8766203E-1,0E0,1.2659828E-1,0E0,2.7820778E-1,7.7317715E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.066866E-1,1.0212688E0,0E0,1.3333883E0,0E0,0E0,8.791113E-2,0E0,1.9125122E-1,1.3116431E-1,0E0,2.612763E-1,9.2200714E-1,4.5001972E-1,5.0699437E-1,5.389855E-1,1.0101345E-1,1.3934946E-1,8.761391E-2,1.7935678E-1,0E0,0E0,0E0,7.7112865E-1,0E0,2.0736885E-1,4.0108418E-1,3.9402008E-2,0E0,0E0,0E0,0E0,1.290747E-1,1.004122E-1,0E0,5.2670717E-2,1.10842735E-1,1.903607E-2,4.294787E-1,0E0,2.3504996E-1,0E0,6.5004814E-1,4.5434594E-1,1.018431E-1,1.20788395E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.214985E-2,9.6289545E-2,0E0,9.6255875E-1,1.0608387E-1,2.082491E-2,0E0,2.4206087E-1,1.334076E-1,0E0,1.4082551E-1,7.646449E-2,2.0815466E-2,3.1768225E-2,0E0,0E0,9.845888E-2,9.008677E-2,0E0,0E0,1.791074E-1,9.0132356E-2,1.0818079E-1,8.0421925E-2,4.595884E-1,0E0,9.223825E-2,1.2283397E-1,0E0,0E0,1.12092614E-1,0E0,0E0,0E0,0E0,0E0,4.0167427E-1,3.3273482E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,31,31,34,34,36,36,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,50,50,54,54,55,55,56,56,58,58,59,59,60,60,63,63,65,65,66,66,69,69,71,71,73,73,74,74,81,81,82,82,84,84,87,87,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,104,104,106,106,107,107,108,108,113,113,114,114,116,116,117,117,118,118,119,119,121,121,123,123,124,124,125,125,126,126,133,133,134,134,136,136,137,137,138,138,140,140,141,141,143,143,144,144,145,145,146,146,149,149,150,150,153,153,154,154,155,155,156,156,157,157,159,159,160,160,163,163,169,169,170,170],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,-1,52,-1,54,-1,-1,56,-1,58,60,62,64,-1,66,-1,68,70,72,74,76,-1,-1,78,-1,-1,-1,80,82,84,-1,86,88,90,-1,-1,92,-1,94,96,-1,-1,98,-1,100,-1,102,104,-1,-1,-1,-1,-1,-1,106,108,-1,110,-1,-1,112,-1,114,116,-1,118,120,122,124,126,128,130,132,134,-1,-1,-1,136,-1,138,140,142,-1,-1,-1,-1,144,146,-1,148,150,152,154,-1,156,-1,158,160,162,164,-1,-1,-1,-1,-1,-1,166,168,-1,170,172,174,-1,176,178,-1,180,182,184,186,-1,-1,188,190,-1,-1,192,194,196,198,200,-1,202,204,-1,-1,206,-1,-1,-1,-1,-1,208,210,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,2.16577E5,2.856934E7,7.7E1,8.1E2,3.3817584E7,1.465E4,9.55188E5,3.541824E2,1.5859042E6,2.8530578E6,6.364543E-2,4.1973075E6,1.7026364E3,7.5491136E-1,2.0376764E-7,7.45E0,6E0,1.8729E4,2E0,7.89E2,1.2294118E1,4.6712E4,7.445455E2,6.292039E0,-2.5734607E-2,-1.1915106E-2,1.5896863E3,-6.402389E-3,2.7546012E0,-1.0974521E-2,1.8987958E0,-4.9571794E-2,1.263199E-2,8.719101E0,1.9859787E-2,7.70947E5,4.39E2,1.7913036E6,5.5426865E6,-1.39172375E-2,1.4392524E0,-2.044582E-2,2.524828E2,1.0039E4,6.103178E8,2.6793549E1,3.9914E4,2.5552644E-3,2.8878017E-3,5.2224144E5,-2.2936359E-2,-1.2272982E-2,-3.387607E-2,7E0,8.81059E5,1.7915686E2,-1.1423461E-2,8.215373E6,1.7566667E2,1.36E2,-1.2143544E-2,-2.9357424E-2,3.0502679E-6,8.819292E-3,7.5491136E-1,1.4603988E6,9.02584E-5,5.4531014E-3,3.2911258E6,3.935705E-3,3.56E3,1.6302979E-2,4.8149962E8,1E0,4.2970426E-2,2.8703295E-2,2.2502005E-2,8.49337E-3,-2.2833815E-2,-1.067643E-2,1.4953704E0,1.8E0,-1.3344708E-2,6.8E1,1.5412492E-3,-7.4951584E-3,1.9E1,-4.192686E-3,2.118835E6,9.836066E-1,-9.753107E-3,2.485E2,4.5158855E6,2.0110958E0,9.751103E6,4.7563504E7,1.935E3,6.4258E4,4.3566666E2,1.0039E4,3.307639E-2,1.3818471E-2,5.214149E-4,3.808001E5,-2.628421E-3,2.2470966E-1,3.6E2,1.04618106E6,-5.8177676E-2,-2.1207329E-2,-5.5476143E-3,-1.4923227E-2,5.9565744E5,3.056087E2,-4.294914E-3,4.5172415E0,5.56E2,1.6E1,5.372237E2,-1.5241143E-2,1.387E3,-2.74521E-3,1.0564016E3,2.5417458E1,1.72959E5,6.328E2,2.2793636E-3,1.2846289E-2,1.1451453E-2,2.2317665E-2,2.1162769E-3,1.1603518E-2,2.73E2,1.99E2,2.7839504E-3,1E0,6E0,5.912179E3,-1.2640013E-2,1.4E1,1.7915686E2,-2.0662732E-2,9E0,2.9276064E3,5.13657E6,5.539245E6,-1.6361587E-2,-8.2384255E-3,5.08E2,1.118178E6,5.180747E-3,1.4862325E-3,1.071E3,1.47E3,3.6245735E0,2.277593E5,3.5E1,1.3333295E-2,9.72E2,6.648E3,-5.2396295E-5,-8.692818E-3,4.7185095E2,1.209602E-2,3.9888298E-4,-5.4185004E-3,8.2136495E-3,5.5828143E-4,1.1251919E6,4.797448E8,-1.8970046E-2,-2.715469E-2,-7.6681697E-3,-1.55885415E-2,-1.7287824E-3,1.0797878E-2,-1.7378371E-2,-1.0187471E-2,7.745435E-5,-9.295376E-3,2.5393679E-3,-4.1901073E-3,-2.0631547E-4,-4.2035575E-3,5.1161516E-3,-1.6028775E-4,-1.9313481E-3,4.868096E-3,-5.922737E-3,1.4695912E-3,-1.0086013E-3,5.485132E-3,6.548525E-3,1.8309431E-2,1.1172498E-2,2.4375792E-3,1.9502638E-2,8.983798E-3,6.6987835E-3,-5.088571E-3,1.7708457E-3,1.0187459E-2,-7.0623914E-3,2.0943263E-3,7.5206594E-3,9.939047E-5,1.6885199E-2,2.5971701E-2,2.066766E-3,1.5694655E-2],"split_indices":[2,43,60,44,2,7,44,46,67,43,43,0,43,4,53,52,69,3,1,6,2,68,1,67,53,0,0,4,0,68,0,68,0,0,69,0,9,2,60,43,0,68,0,4,44,7,73,9,0,0,43,0,0,0,6,9,4,0,12,67,0,0,0,53,0,53,43,0,0,43,0,2,0,5,26,0,0,0,0,0,0,69,68,0,0,0,0,67,0,9,71,0,67,62,53,9,7,2,1,67,44,0,0,0,43,0,53,2,60,0,0,0,0,43,4,0,69,2,3,4,0,2,0,4,73,12,4,0,0,0,0,0,0,0,8,0,6,8,48,0,3,4,0,3,62,9,5,0,0,2,9,0,0,2,2,71,48,0,0,44,44,0,0,4,0,0,0,0,0,43,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,7.64E2,2.51E2,2.49E2,5.15E2,2.21E2,3E1,7.4E1,1.75E2,2.2E2,2.95E2,6E0,2.15E2,9E0,2.1E1,4E1,3.4E1,1.53E2,2.2E1,1.42E2,7.8E1,2.41E2,5.4E1,1.59E2,5.6E1,5E0,4E0,1.5E1,6E0,3.5E1,5E0,3E1,4E0,6E0,1.47E2,5E0,1.7E1,1.28E2,1.4E1,7.2E1,6E0,2.36E2,5E0,1.2E1,4.2E1,4.2E1,1.17E2,5.1E1,5E0,5E0,1E1,2.9E1,6E0,1.9E1,1.1E1,1.31E2,1.6E1,5E0,1.2E1,4.1E1,8.7E1,9E0,5E0,6.1E1,1.1E1,8.6E1,1.5E2,4E0,8E0,3.5E1,7E0,3.5E1,7E0,2.3E1,9.4E1,2E1,3.1E1,5E0,5E0,6E0,5E0,7.3E1,5.8E1,5E0,1.1E1,8E0,4E0,3.4E1,7E0,6.6E1,2.1E1,4E0,5.7E1,5.3E1,3.3E1,1.08E2,4.2E1,8E0,2.7E1,9E0,2.6E1,1.9E1,4E0,5E0,8.9E1,5E0,6.8E1,1.7E1,4.1E1,4E0,7E0,4E0,3E1,4.4E1,2.2E1,6E0,1.5E1,3.8E1,1.9E1,4.9E1,4E0,2.9E1,4E0,7.9E1,2.9E1,1.1E1,3.1E1,4E0,4E0,7E0,2E1,4E0,5E0,1.2E1,1.4E1,6E0,8.3E1,4.7E1,2.1E1,4E0,1.3E1,3.5E1,6E0,2.6E1,1.8E1,1.3E1,9E0,8E0,7E0,1.8E1,2E1,1.4E1,5E0,4E1,9E0,1.3E1,1.6E1,7.5E1,4E0,1.8E1,1.1E1,7E0,4E0,2.1E1,1E1,7E0,5E0,8E0,6E0,6.8E1,1.5E1,3.3E1,1.4E1,4E0,1.7E1,9E0,4E0,1.4E1,2.1E1,4E0,2.2E1,4E0,1.4E1,4E0,9E0,5E0,4E0,1.2E1,6E0,1.6E1,4E0,1.3E1,2.7E1,4E0,5E0,7E0,6E0,1.1E1,5E0,7E0,6.8E1,4E0,1.4E1,6E0,5E0,1.5E1,6E0,2.6E1,4.2E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"211","size_leaf_vector":"1"}},{"base_weights":[3.220995E-3,-1.2848191E-1,3.9728412E-1,-2.554451E-1,4.9032506E-2,-3.9518216E-1,4.3124267E-1,-3.9445123E-1,-1.5593712E-1,-6.254721E-2,1.4237507E-1,-2.3489894E-2,-7.279638E-3,1.9914469E-1,5.425681E-1,-3.6553645E-1,-8.639063E-1,-9.621763E-2,-5.5182266E-1,-4.52914E-2,-2.1142263E-2,1.6072036E-1,-7.876543E-2,1.0174252E-1,4.7259173E-1,5.721472E-1,-9.554424E-3,-5.348445E-1,-3.0329442E-1,-1.1962927E-2,-5.6550313E-2,4.4917592E-1,-1.1809183E-1,-7.22109E-1,-2.87986E-1,-9.610479E-2,2.4908427E-2,1.0262005E-1,2.3670018E-1,-7.750425E-3,1.5798523E-3,-8.152097E-2,1.5307474E-1,1.6141608E-2,2.6479976E-2,5.93647E-2,5.4065937E-1,-1.2771418E-2,-2.6180707E-2,-9.120253E-2,-3.320098E-1,1.0742796E-2,2.6609905E-2,-2.6936036E-1,-3.7991833E-2,-1.819198E-2,-3.6015943E-2,-2.7202473E-3,-1.6906269E-2,-6.44165E-2,-2.9478228E-1,-1.7224561E-2,1.4185068E-1,3.2275656E-1,7.220895E-2,-9.732608E-3,2.674715E-1,-8.080454E-3,-1.0206928E-2,3.3646703E-1,6.766417E-2,4.661095E-1,7.7206975E-1,9.574252E-3,-1.9897221E-1,-2.5316775E-1,-3.908813E-1,-3.996865E-1,-1.543419E-1,-1.5883903E-1,1.4510805E-2,-2.4133502E-2,-1.9332422E-1,-1.9171244E-2,-3.1213132E-3,-5.4154865E-2,1.1744698E-1,7.7865735E-2,1.23371985E-2,2.119341E-2,8.293495E-3,9.585977E-2,-1.4595016E-2,3.1908882E-1,1.5934035E-1,-3.2006756E-3,2.351885E-3,8.581722E-3,1.9232757E-2,1.1739641E-1,-6.5108836E-3,7.5008446E-1,4.0807354E-1,8.432998E-1,2.321695E-2,-1.2538147E-2,-1.3313857E-3,-2.7091724E-1,-1.4848002E-3,-2.1639124E-2,-2.8742635E-1,-2.3689473E-1,-2.4222445E-2,-5.7015553E-2,-2.2496103E-1,-7.381873E-2,-2.8952903E-1,4.1781794E-2,-1.5989126E-1,1.8479452E-2,-1.0176079E-1,-2.4506535E-1,-2.662048E-3,-6.7606554E-3,-1.4837989E-1,1.978625E-3,8.477528E-3,5.658173E-3,5.9344375E-4,1.120397E-1,-5.528606E-3,2.5520054E-1,4.1095024E-1,1.9344163E-1,-9.759751E-5,-1.0161004E-3,1.7425226E-1,3.885535E-2,1.9072609E-2,2.0911404E-1,4.7076926E-1,2.3250774E-2,4.1472994E-2,-1.5806496E-2,-2.315277E-1,-2.3873536E-1,-2.232034E-2,-3.4274857E-3,-1.586334E-2,-2.580456E-3,-9.651801E-3,-1.2431198E-2,-1.6153818E-4,-8.438632E-3,-2.1150062E-2,-1.576765E-2,-6.363268E-3,1.2111861E-1,-2.8508376E-2,-1.1337749E-2,1.9923884E-3,-1.4758602E-2,6.5053906E-3,-1.554788E-1,9.3379774E-4,-4.6505737E-3,-1.4076071E-2,-5.6894593E-2,4.938022E-3,-5.731622E-4,-1.5534965E-2,1.4330468E-1,-2.0489521E-2,1.6641114E-2,1.9518864E-1,2.1520257E-2,9.105019E-3,1.6813251E-3,1.0759529E-2,2.3198147E-1,1.0593004E-1,2.789836E-1,7.8698323E-4,5.118178E-1,2.3247096E-1,-4.2572315E-3,-1.1924129E-2,-4.202423E-3,-1.24546895E-2,-4.7498825E-3,3.495662E-3,1.7887024E-3,-3.4026618E-3,6.856046E-3,-6.184847E-4,3.8336935E-3,-2.7734078E-3,-6.407575E-3,4.454952E-4,-3.4182416E-3,-1.1632907E-2,-7.1993507E-3,-8.0827257E-4,2.2610591E-3,8.585639E-3,2.2618072E-3,-6.13617E-3,4.183312E-3,1.0350049E-2,5.4446855E-3,1.3324968E-2,1.1464513E-3,6.871469E-3,1.7427403E-2,4.469738E-3,2.5338389E-2,1.3185349E-2,-2.4528322E-3,1.6875928E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,71,-1,-1,73,75,-1,-1,77,79,-1,-1,-1,-1,81,83,85,87,89,91,-1,93,-1,95,97,99,101,103,-1,105,107,109,111,113,115,117,119,121,-1,-1,123,125,127,-1,-1,-1,129,-1,131,133,-1,-1,-1,-1,135,-1,137,139,141,-1,-1,-1,143,-1,-1,145,147,-1,149,151,153,155,157,159,161,163,165,-1,167,169,-1,-1,-1,-1,171,-1,173,175,177,-1,-1,179,-1,-1,181,183,-1,-1,-1,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,-1,193,195,-1,-1,197,-1,199,-1,-1,-1,201,-1,-1,-1,203,205,-1,207,-1,-1,-1,-1,209,211,213,-1,215,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.283259E1,1.7208862E1,7.0327263E0,6.0938263E0,3.3313334E0,2.1593273E-1,6.2223587E0,2.1954384E0,6.1174884E0,9.7463465E-1,7.185576E-1,0E0,0E0,2.1092985E0,3.775711E0,1.6924877E0,1.8618231E0,2.773095E0,1.3195868E0,5.053537E-1,0E0,6.8643665E-1,1.4648151E-1,5.863254E-1,5.9955597E-2,2.8740883E0,0E0,2.2191334E-1,7.6754856E-1,0E0,0E0,1.1331332E-1,2.6554205E0,7.042885E-2,2.4636161E-1,4.9726284E-1,2.980709E-1,5.9929144E-1,1.0062056E0,0E0,0E0,9.144924E-2,7.2908795E-1,0E0,0E0,0E0,2.2968788E0,0E0,0E0,5.880052E-1,4.3481922E-1,0E0,0E0,1.0775442E0,9.2230046E-1,0E0,0E0,0E0,0E0,3.692152E-1,2.7236444E-1,2.3016608E-1,1.0737461E-1,1.3191128E-1,7.9218E-1,0E0,3.1194973E-1,0E0,3.6018983E-2,1.03150606E-1,3.6925775E-1,1.6551208E0,2.6735115E-1,0E0,1.5237588E-1,1.9529629E-1,3.7856007E-1,5.8584166E-1,2.7584398E-1,4.6752465E-1,4.9180454E-1,1.8712448E-1,1.05243266E-1,0E0,0E0,1.5976277E-1,4.142897E-2,3.3231974E-2,0E0,0E0,0E0,2.8889984E-1,0E0,1.4748669E-1,1.2704086E-1,0E0,0E0,0E0,0E0,2.2573695E-1,0E0,2.356205E-1,1.1614208E0,4.7662735E-2,0E0,0E0,0E0,5.544424E-2,0E0,0E0,1.7053485E-1,2.690333E-1,0E0,1.565683E-1,2.3355317E-1,1.566447E-1,8.6580396E-2,4.9977022E-1,2.5365123E-1,1.52524E-1,1.3418582E-1,7.263613E-2,0E0,1.485198E-1,3.023915E-1,0E0,0E0,0E0,0E0,3.0955702E-1,0E0,1.0613406E-1,9.594011E-2,1.0925019E-1,0E0,0E0,5.4309845E-2,0E0,0E0,3.441807E-1,6.244354E-1,0E0,0E0,0E0,1.0215378E-1,9.401369E-2,0E0,0E0,0E0,1.2542138E-1,0E0,0E0,0E0,0E0,6.57149E-2,0E0,0E0,1.5611923E-1,1.7368001E-1,0E0,0E0,9.206462E-2,0E0,9.1257304E-2,0E0,0E0,0E0,6.743945E-2,0E0,0E0,0E0,2.317313E-1,1.2351875E-1,0E0,4.0075958E-2,0E0,0E0,0E0,0E0,3.04268E-2,3.450325E-2,3.0365694E-1,0E0,3.7445068E-1,5.124518E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,46,46,49,49,50,50,53,53,54,54,59,59,60,60,61,61,62,62,63,63,64,64,66,66,68,68,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,85,85,86,86,87,87,91,91,93,93,94,94,99,99,101,101,102,102,103,103,107,107,110,110,111,111,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,124,124,129,129,131,131,132,132,133,133,136,136,139,139,140,140,144,144,145,145,149,149,154,154,157,157,158,158,161,161,163,163,167,167,171,171,172,172,174,174,179,179,180,180,181,181,183,183,184,184],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,72,-1,-1,74,76,-1,-1,78,80,-1,-1,-1,-1,82,84,86,88,90,92,-1,94,-1,96,98,100,102,104,-1,106,108,110,112,114,116,118,120,122,-1,-1,124,126,128,-1,-1,-1,130,-1,132,134,-1,-1,-1,-1,136,-1,138,140,142,-1,-1,-1,144,-1,-1,146,148,-1,150,152,154,156,158,160,162,164,166,-1,168,170,-1,-1,-1,-1,172,-1,174,176,178,-1,-1,180,-1,-1,182,184,-1,-1,-1,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,-1,194,196,-1,-1,198,-1,200,-1,-1,-1,202,-1,-1,-1,204,206,-1,208,-1,-1,-1,-1,210,212,214,-1,216,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,7.785302E5,1E0,4.97E2,9.48E2,2.3308511E5,8.5023944E2,4.5E1,2.3E1,3.4E1,8.695652E0,-2.3489894E-2,-7.279638E-3,3.1993368E6,3.206931E2,3E1,5.5E1,2.74614E5,6.5139695E-3,2.331083E6,-2.1142263E-2,4.2210345E2,2.0394794E8,1E0,5.941442E6,3.3817584E7,-9.554424E-3,1.957E3,1.2051282E0,-1.1962927E-2,-5.6550313E-2,1.9E1,1.7302156E5,3.9401392E6,2.12791E5,1.4992306E8,2.9254214E2,1.8992E4,1.5043378E0,-7.750425E-3,1.5798523E-3,2.0939393E0,1.1057851E1,1.6141608E-2,2.6479976E-2,5.93647E-2,4.391553E6,-1.2771418E-2,-2.6180707E-2,1.9E1,8.6875E0,1.0742796E-2,2.6609905E-2,1.4912975E-1,7.69E2,-1.819198E-2,-3.6015943E-2,-2.7202473E-3,-1.6906269E-2,4.135135E0,1.01573E5,1E0,2.9816E4,9.8933E4,2.511E3,-9.732608E-3,3.875E0,-8.080454E-3,1.7909248E5,4.3566666E2,1.2544625E5,1.307E3,1.6808511E0,9.574252E-3,1E0,4.58E2,1.463E2,1.01E2,2.3412812E0,2E0,7.955228E9,1E0,4.3460325E6,-1.9171244E-2,-3.1213132E-3,6.83E2,2.0833333E0,1.4067796E0,1.23371985E-2,2.119341E-2,8.293495E-3,1E0,-1.4595016E-2,3.18E3,3.1206896E0,-3.2006756E-3,2.351885E-3,8.581722E-3,1.9232757E-2,5.595208E2,-6.5108836E-3,3.256391E1,3.3580637E8,4.029152E8,2.321695E-2,-1.2538147E-2,-1.3313857E-3,7.2864324E-1,-1.4848002E-3,-2.1639124E-2,4.046E3,1.0130841E1,-2.4222445E-2,1.32476E5,6.5162756E2,1E0,1E0,2.1340206E0,1.82E3,8.051603E7,1.559733E6,4.5650104E2,-2.662048E-3,4.49239E6,6.0052995E6,1.978625E-3,8.477528E-3,5.658173E-3,5.9344375E-4,3.472E3,-5.528606E-3,1.9197379E9,1.9E1,7.755551E7,-9.759751E-5,-1.0161004E-3,3.0588236E0,3.885535E-2,1.9072609E-2,1.260841E2,1E0,2.3250774E-2,4.1472994E-2,-1.5806496E-2,4.4908694E2,2.855464E5,-2.232034E-2,-3.4274857E-3,-1.586334E-2,2.5055911E2,-9.651801E-3,-1.2431198E-2,-1.6153818E-4,-8.438632E-3,1.2707424E0,-1.576765E-2,-6.363268E-3,4.41537E6,6.8808866E0,-1.1337749E-2,1.9923884E-3,4.48E2,6.5053906E-3,8.7601E4,9.3379774E-4,-4.6505737E-3,-1.4076071E-2,5.342944E6,4.938022E-3,-5.731622E-4,-1.5534965E-2,5.4007E7,2.5E0,1.6641114E-2,3.221296E6,2.1520257E-2,9.105019E-3,1.6813251E-3,1.0759529E-2,1.5E1,5.657353E1,2.1298597E0,7.8698323E-4,7.61E2,1.4595818E7,-4.2572315E-3,-1.1924129E-2,-4.202423E-3,-1.24546895E-2,-4.7498825E-3,3.495662E-3,1.7887024E-3,-3.4026618E-3,6.856046E-3,-6.184847E-4,3.8336935E-3,-2.7734078E-3,-6.407575E-3,4.454952E-4,-3.4182416E-3,-1.1632907E-2,-7.1993507E-3,-8.0827257E-4,2.2610591E-3,8.585639E-3,2.2618072E-3,-6.13617E-3,4.183312E-3,1.0350049E-2,5.4446855E-3,1.3324968E-2,1.1464513E-3,6.871469E-3,1.7427403E-2,4.469738E-3,2.5338389E-2,1.3185349E-2,-2.4528322E-3,1.6875928E-2],"split_indices":[2,43,17,2,2,48,67,6,3,3,61,0,0,43,73,70,0,5,53,43,0,67,7,27,60,7,0,9,68,0,0,0,43,60,12,5,70,9,69,0,0,68,71,0,0,0,43,0,0,0,73,0,0,53,2,0,0,0,0,69,1,28,1,1,2,0,69,0,43,67,48,44,68,0,8,2,70,0,68,32,46,26,60,0,0,2,68,68,0,0,0,79,0,44,68,0,0,0,0,67,0,73,7,7,0,0,0,73,0,0,44,71,0,1,4,8,30,68,2,7,9,48,0,43,43,0,0,0,0,10,0,46,3,5,0,0,68,0,0,71,6,0,0,0,66,60,0,0,0,4,0,0,0,0,68,0,0,60,73,0,0,2,0,9,0,0,0,60,0,0,0,7,68,0,60,0,0,0,0,3,71,53,0,8,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,7.62E2,2.54E2,4.44E2,3.18E2,1E1,2.44E2,1.84E2,2.6E2,1.45E2,1.73E2,6E0,4E0,8E1,1.64E2,1.75E2,9E0,2.27E2,3.3E1,1.4E2,5E0,1.6E2,1.3E1,6E1,2E1,1.58E2,6E0,4.5E1,1.3E2,4E0,5E0,8E0,2.19E2,1.9E1,1.4E1,8.1E1,5.9E1,9.2E1,6.8E1,7E0,6E0,1.3E1,4.7E1,1.1E1,9E0,5E0,1.53E2,6E0,3.9E1,1.6E1,1.14E2,4E0,4E0,7.5E1,1.44E2,4E0,1.5E1,4E0,1E1,7.1E1,1E1,4.4E1,1.5E1,1E1,8.2E1,4E0,6.4E1,5E0,8E0,1.4E1,3.3E1,1.18E2,3.5E1,4E0,1.2E1,5.1E1,6.3E1,3.4E1,4.1E1,4.3E1,1.01E2,5.5E1,1.6E1,6E0,4E0,3.5E1,9E0,1.1E1,4E0,4E0,6E0,7.8E1,4E0,4.2E1,2.2E1,4E0,4E0,6E0,8E0,2.7E1,6E0,1.8E1,1E2,2.6E1,9E0,8E0,4E0,4.7E1,4E0,3.4E1,2.9E1,1.6E1,1.8E1,1.8E1,2.3E1,2.7E1,1.6E1,8.8E1,1.3E1,3.6E1,1.9E1,1.1E1,5E0,2.4E1,1.1E1,5E0,4E0,6E0,5E0,7.3E1,5E0,2.7E1,1.5E1,1.8E1,4E0,8E0,1.9E1,1.3E1,5E0,2.5E1,7.5E1,5E0,2.1E1,1.4E1,3.3E1,2.5E1,4E0,7E0,9E0,1.4E1,4E0,1.9E1,4E0,8E0,1.9E1,1.1E1,5E0,4.1E1,4.7E1,9E0,4E0,2.9E1,7E0,1.3E1,6E0,4E0,7E0,1.7E1,7E0,7E0,4E0,5.9E1,1.4E1,8E0,1.9E1,1.1E1,4E0,4E0,1.4E1,9E0,1E1,1.8E1,7E0,6.3E1,1.2E1,6E0,2.7E1,5E0,2E1,6E0,8E0,9E0,1E1,3.4E1,7E0,1E1,3.7E1,4E0,2.5E1,8E0,5E0,4E0,1.3E1,1.9E1,4E1,9E0,5E0,5E0,1.4E1,4E0,5E0,4E0,6E0,1.1E1,7E0,5.3E1,1E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-4.4613914E-3,-1.9807957E-1,2.0014012E-1,-3.745944E-1,-9.339277E-2,2.4397902E-1,-5.112069E-1,-4.272029E-1,-2.056768E-1,-2.4976265E-1,-3.2554865E-2,1.2366045E-1,5.148828E-1,-3.2893173E-2,-3.8525602E-1,-3.9716187E-1,-6.6508645E-1,-2.5036967E-1,-7.9253405E-2,-1.6846828E-1,-4.205584E-1,-6.1155513E-2,7.092352E-2,3.402724E-1,6.586562E-2,6.2717986E-1,2.866561E-1,-8.778473E-3,-1.9504804E-2,-4.8264036E-1,-2.6317227E-1,-7.724392E-3,-8.973108E-1,-2.898917E-1,-5.6628706E-3,1.678695E-3,-1.2912284E-1,-2.0375587E-1,4.246028E-2,-3.2448512E-1,-3.3945143E-2,-4.4609047E-2,-3.218436E-1,1.6104831E-1,2.254725E-2,2.3030083E-1,4.8914957E-1,9.570672E-2,-2.0294215E-1,5.04491E-1,8.890067E-1,3.494736E-1,-3.648233E-3,-2.8487349E-2,-2.0430839E-2,1.521193E-2,-3.3034527E-1,-5.3707164E-2,-1.846122E-2,-1.6118526E-2,-1.0319226E-2,-2.2626396E-3,-8.056722E-3,-2.507658E-1,-8.392392E-2,-2.1121674E-4,4.1767093E-3,-9.395055E-3,-1.6742427E-2,6.4748465E-3,-9.873822E-2,-2.2507835E-2,-7.288339E-3,4.5214454E-3,1.0523641E-2,-6.8514915E-3,6.300965E-2,2.8972468E-1,1.711443E-2,2.4119057E-2,1.30020855E-2,1.7322531E-1,2.6793553E-2,-9.353181E-2,-3.9921367E-1,6.3372624E-1,2.465392E-1,1.0068929E0,1.730918E-2,1.009027E-3,3.93158E-1,-2.7900547E-1,-2.2283504E-2,-2.7296433E-1,-3.8428288E-3,-6.0890834E-3,1.192751E-3,5.172601E-2,-1.398573E-1,-1.1622585E-1,9.600464E-3,2.4009965E-2,7.501921E-3,3.495059E-1,1.7546707E-1,-1.2547317E-3,3.0870188E-3,2.0642166E-1,1.5180469E-2,-1.3186507E-1,6.2306833E-2,-1.759049E-1,1.451697E-3,-2.3611257E-2,-9.586669E-3,4.4171327E-1,7.382635E-1,2.8708488E-1,1.6192159E-3,5.0096635E-2,2.3613272E-2,3.2663527E-1,2.6048169E-2,-3.1818485E-1,-4.0383595E-3,-1.7529752E-2,-2.1945564E-1,7.809879E-2,-1.3863076E-1,-2.4217283E-2,-2.1805292E-1,-1.7754853E-1,-4.014939E-3,8.396162E-2,-1.6245382E-2,6.5664626E-3,4.091542E-1,1.8692205E-3,1.3185545E-2,2.1174159E-2,1.8297236E-1,-4.3492056E-2,7.360973E-3,9.450751E-3,-2.2338599E-1,7.9136506E-2,-1.0171062E-2,-1.1788716E-2,-3.110122E-3,5.527112E-1,1.9669945E-3,3.607859E-2,1.8216664E-2,4.285529E-3,3.4567493E-1,7.658027E-3,1.8598875E-2,-1.1427673E-2,-3.8196558E-1,-6.8128845E-3,-1.3173806E-2,4.3001838E-2,2.0977922E-1,-1.3725284E-3,-1.0155769E-2,1.4527359E-3,-3.980914E-3,-2.9408297E-1,-1.736542E-3,-2.4079856E-1,-1.0260431E-1,6.268127E-2,-1.24915786E-1,5.71508E-3,1.2670245E-3,-3.16025E-3,2.1106717E-3,2.3716556E-2,1.1725698E-2,1.7361093E-2,2.2473401E-1,7.4811797E-3,-6.7882463E-3,-6.691366E-3,6.303831E-3,-1.4504919E-2,-4.063392E-3,9.252638E-3,1.5028353E-1,1.269861E-2,2.9399319E-2,1.801399E-2,8.278784E-3,-1.9761374E-2,-9.464315E-3,-6.204706E-4,5.1875E-3,4.5663933E-3,1.1462093E-2,-6.2363097E-3,-1.8219126E-2,-9.189131E-3,-1.8190295E-2,-5.0416205E-4,-6.594905E-3,6.896549E-4,6.8944483E-3,-2.5300044E-3,-8.050146E-3,5.3206435E-3,-3.3448602E-3,1.1482408E-2,2.964767E-3,-4.5070266E-3,3.7624273E-3,-4.783242E-3,3.7754192E-3,-7.2395295E-4,8.979616E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,-1,-1,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,87,89,-1,-1,-1,-1,91,-1,-1,-1,-1,-1,-1,93,95,-1,-1,-1,-1,97,99,-1,-1,-1,-1,-1,101,103,105,-1,-1,107,109,111,113,115,117,119,-1,-1,121,123,-1,125,-1,-1,-1,127,129,131,-1,133,-1,135,137,-1,-1,139,141,143,145,147,-1,-1,-1,149,151,153,-1,-1,-1,155,-1,157,-1,-1,159,161,163,165,167,169,171,173,175,-1,177,-1,-1,-1,179,181,-1,183,185,187,-1,-1,-1,189,-1,-1,-1,-1,191,-1,-1,-1,193,-1,-1,195,197,-1,-1,-1,-1,199,-1,201,203,205,207,-1,-1,-1,-1,-1,-1,209,211,213,-1,-1,-1,-1,-1,215,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0248436E1,9.610933E0,1.5529947E1,1.636116E0,3.1217172E0,1.5125486E1,4.5907068E-1,7.8635406E-1,2.5136828E-1,1.2065444E0,7.0870996E-1,4.03376E0,3.4851875E0,0E0,6.400156E-2,1.3962612E0,1.5842981E0,1.4124036E-1,8.539321E-2,4.8501372E-1,6.369829E-1,7.92261E-1,2.2308365E-1,1.0006618E0,2.0809536E0,2.6032143E0,1.153008E0,0E0,0E0,1.6039467E-1,2.2577612E0,0E0,8.554096E-1,2.412486E-2,0E0,0E0,2.7740464E-2,2.8878593E-1,2.2963624E-2,3.6514282E-2,0E0,4.9006712E-1,2.2158325E-1,5.360627E-2,2.5130102E-1,5.144441E-1,4.452467E-2,1.2347145E0,5.209882E-1,2.0773048E0,1.3811302E0,5.925355E-1,0E0,0E0,0E0,0E0,2.477827E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2238097E-1,9.2722684E-2,0E0,0E0,0E0,0E0,6.164307E-1,4.851762E-1,0E0,0E0,0E0,0E0,0E0,1.08491085E-1,1.5724778E-1,2.3821253E-2,0E0,0E0,5.687363E-1,7.068926E-1,1.9079934E-1,8.695078E-2,5.5708885E-1,1.9573855E-1,3.2915497E-1,0E0,0E0,2.3590374E-1,2.6494098E-1,0E0,1.10440016E-1,0E0,0E0,0E0,3.6726367E-1,1.9365522E-1,5.635351E-1,0E0,5.4458227E-2,0E0,1.8923283E-1,1.814073E-1,0E0,0E0,4.477644E-1,1.7608398E-1,2.995784E-1,5.0230813E-1,7.7501416E-2,0E0,0E0,0E0,7.739277E-1,8.8378906E-2,1.95642E-1,0E0,0E0,0E0,2.8758144E-1,0E0,5.007553E-2,0E0,0E0,7.618415E-2,2.8213874E-1,7.306486E-2,3.7193272E-2,1.6731554E-1,2.2652495E-1,2.4973874E-1,1.6830124E-2,4.835005E-2,0E0,1.1251593E-1,0E0,0E0,0E0,5.7705927E-1,8.019138E-2,0E0,2.1570726E-1,1.4412886E-1,4.808964E-1,0E0,0E0,0E0,1.5047073E-1,0E0,0E0,0E0,0E0,4.0616393E-2,0E0,0E0,0E0,3.398776E-2,0E0,0E0,2.0091042E-1,3.0746281E-2,0E0,0E0,0E0,0E0,9.035039E-2,0E0,1.0109317E-1,9.414825E-2,8.0435514E-2,2.852711E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.6636644E-1,2.4016619E-1,9.339764E-2,0E0,0E0,0E0,0E0,0E0,4.1713798E-1,3.5421288E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,56,56,63,63,64,64,69,69,70,70,76,76,77,77,78,78,81,81,82,82,83,83,84,84,85,85,86,86,87,87,90,90,91,91,93,93,97,97,98,98,99,99,101,101,103,103,104,104,107,107,108,108,109,109,110,110,111,111,115,115,116,116,117,117,121,121,123,123,126,126,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,136,136,140,140,141,141,143,143,144,144,145,145,149,149,154,154,158,158,161,161,162,162,167,167,169,169,170,170,171,171,172,172,179,179,180,180,181,181,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,-1,-1,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,88,90,-1,-1,-1,-1,92,-1,-1,-1,-1,-1,-1,94,96,-1,-1,-1,-1,98,100,-1,-1,-1,-1,-1,102,104,106,-1,-1,108,110,112,114,116,118,120,-1,-1,122,124,-1,126,-1,-1,-1,128,130,132,-1,134,-1,136,138,-1,-1,140,142,144,146,148,-1,-1,-1,150,152,154,-1,-1,-1,156,-1,158,-1,-1,160,162,164,166,168,170,172,174,176,-1,178,-1,-1,-1,180,182,-1,184,186,188,-1,-1,-1,190,-1,-1,-1,-1,192,-1,-1,-1,194,-1,-1,196,198,-1,-1,-1,-1,200,-1,202,204,206,208,-1,-1,-1,-1,-1,-1,210,212,214,-1,-1,-1,-1,-1,216,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,5.1987316E7,1.339646E6,4.97E2,1.0236667E3,4.4522205E6,4.6463413E0,8.0509944E5,4.325E0,2.331083E6,7E0,1.6007428E7,-3.2893173E-2,1E0,2.55E2,1.3E1,6.54E2,1.0526316E0,2.9652428E6,7.668863E6,2.829581E10,2.1111E4,1.0254199E10,9.932432E0,1.9180963E3,4.24038E8,-8.778473E-3,-1.9504804E-2,8.0833334E-1,1.1E1,-7.724392E-3,7.3E1,8.147158E4,-5.6628706E-3,1.678695E-3,4.03E2,2.0399521E2,1.3E1,1.1967312E3,-3.3945143E-2,1E0,2.2084616E2,5.405423E6,9.693913E3,1.5723623E0,9.244374E6,1E1,4E1,8E0,1.4497429E7,1.2841364E3,-3.648233E-3,-2.8487349E-2,-2.0430839E-2,1.521193E-2,1.3439851E7,-5.3707164E-2,-1.846122E-2,-1.6118526E-2,-1.0319226E-2,-2.2626396E-3,-8.056722E-3,2.5882354E0,6.920598E7,-2.1121674E-4,4.1767093E-3,-9.395055E-3,-1.6742427E-2,3.493E3,3.2E1,-2.2507835E-2,-7.288339E-3,4.5214454E-3,1.0523641E-2,-6.8514915E-3,1.273801E6,1.02E2,8.328548E6,2.4119057E-2,1.30020855E-2,8E0,2.2783158E5,1.8285715E1,6.001692E-7,1.7942307E8,4.9735293E0,1.2816234E7,1.730918E-2,1.009027E-3,3.8181802E6,3.1158695E6,-2.2283504E-2,1.1967312E3,-3.8428288E-3,-6.0890834E-3,1.192751E-3,1E0,3.074398E0,4.39776E6,9.600464E-3,2.0487332E-3,7.501921E-3,1.357E3,1.9539816E7,-1.2547317E-3,3.0870188E-3,9.63381E5,1.94261E5,3.63E2,2.277593E5,2.0312773E8,1.451697E-3,-2.3611257E-2,-9.586669E-3,5.526E3,1.7103828E7,2.9697892E-1,1.6192159E-3,5.0096635E-2,2.3613272E-2,2.337765E3,2.6048169E-2,8E0,-4.0383595E-3,-1.7529752E-2,3.2E1,8.15736E7,1E0,7.579E3,9.61E2,3.4919708E2,1.3E1,1.411E3,2.5384614E0,6.5664626E-3,5.55E2,1.8692205E-3,1.3185545E-2,2.1174159E-2,1.977157E7,6.4356956E0,7.360973E-3,9.0927E4,6.808571E2,4.5960168E2,-1.0171062E-2,-1.1788716E-2,-3.110122E-3,1.25E0,1.9669945E-3,3.607859E-2,1.8216664E-2,4.285529E-3,8.255237E1,7.658027E-3,1.8598875E-2,-1.1427673E-2,2.8998098E1,-6.8128845E-3,-1.3173806E-2,3.388854E2,8.62E2,-1.3725284E-3,-1.0155769E-2,1.4527359E-3,-3.980914E-3,2.866353E2,-1.736542E-3,9.455644E5,1E0,9.31E2,1.5E1,5.71508E-3,1.2670245E-3,-3.16025E-3,2.1106717E-3,2.3716556E-2,1.1725698E-2,2.3600838E6,5.2409735E-7,2.4796897E3,-6.7882463E-3,-6.691366E-3,6.303831E-3,-1.4504919E-2,-4.063392E-3,7.026624E7,4.8709216E5,1.269861E-2,2.9399319E-2,1.801399E-2,8.278784E-3,-1.9761374E-2,-9.464315E-3,-6.204706E-4,5.1875E-3,4.5663933E-3,1.1462093E-2,-6.2363097E-3,-1.8219126E-2,-9.189131E-3,-1.8190295E-2,-5.0416205E-4,-6.594905E-3,6.896549E-4,6.8944483E-3,-2.5300044E-3,-8.050146E-3,5.3206435E-3,-3.3448602E-3,1.1482408E-2,2.964767E-3,-4.5070266E-3,3.7624273E-3,-4.783242E-3,3.7754192E-3,-7.2395295E-4,8.979616E-3],"split_indices":[2,43,60,9,2,67,62,68,60,69,43,3,66,0,15,2,0,2,71,43,5,46,9,46,69,67,47,0,0,68,10,0,0,43,0,0,2,4,3,48,0,26,67,43,62,68,60,8,3,32,62,4,0,0,0,0,60,0,0,0,0,0,0,71,7,0,0,0,0,44,3,0,0,0,0,0,9,0,9,0,0,32,43,69,52,12,68,9,0,0,43,60,0,48,0,0,0,19,71,9,0,53,0,2,12,0,0,60,1,0,48,7,0,0,0,2,62,53,0,0,0,67,0,8,0,0,10,7,8,44,2,70,8,44,69,0,44,0,0,0,5,69,0,1,4,67,0,0,0,68,0,0,0,0,73,0,0,0,73,0,0,4,44,0,0,0,0,67,0,62,8,2,3,0,0,0,0,0,0,62,52,48,0,0,0,0,0,7,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.014E3,5.21E2,4.93E2,1.93E2,3.28E2,4.65E2,2.8E1,1.46E2,4.7E1,9.1E1,2.37E2,3.23E2,1.42E2,9E0,1.9E1,1.32E2,1.4E1,3.4E1,1.3E1,6.3E1,2.8E1,1.86E2,5.1E1,6.7E1,2.56E2,9.4E1,4.8E1,4E0,1.5E1,7.9E1,5.3E1,5E0,9E0,2.5E1,9E0,4E0,9E0,5.4E1,9E0,2.3E1,5E0,1.76E2,1E1,1.7E1,3.4E1,4E1,2.7E1,2.31E2,2.5E1,6.6E1,2.8E1,4.1E1,7E0,1.5E1,6.4E1,5E0,4.8E1,5E0,4E0,1.1E1,1.4E1,4E0,5E0,3.8E1,1.6E1,5E0,4E0,7E0,1.6E1,9.1E1,8.5E1,4E0,6E0,1E1,7E0,6E0,2.8E1,3.1E1,9E0,2.2E1,5E0,1.08E2,1.23E2,1.7E1,8E0,4.3E1,2.3E1,2.2E1,6E0,5E0,3.6E1,3.8E1,1E1,3.3E1,5E0,1.1E1,5E0,7E1,2.1E1,8.1E1,4E0,2.1E1,7E0,1.9E1,1.2E1,5E0,4E0,8.9E1,1.9E1,2.2E1,1.01E2,1E1,7E0,4E0,4E0,1.7E1,2.6E1,1.9E1,4E0,1.8E1,4E0,2.8E1,8E0,3.1E1,7E0,9E0,2.4E1,6.2E1,8E0,9E0,1.2E1,5.2E1,2.9E1,8E0,1.3E1,5E0,1.4E1,6E0,6E0,6E0,8.3E1,1.4E1,5E0,9E0,1.3E1,9.6E1,5E0,5E0,5E0,1.3E1,4E0,2.2E1,4E0,5E0,1.4E1,1E1,1.8E1,1.7E1,1.4E1,1.3E1,1.1E1,5E1,1.2E1,4E0,4E0,5E0,4E0,8E0,4E0,2.7E1,2.5E1,1.9E1,1E1,4E0,4E0,7E0,6E0,7E0,7E0,1.7E1,6.6E1,1E1,4E0,4E0,5E0,7E0,6E0,4.9E1,4.7E1,4E0,9E0,1E1,4E0,1E1,4E0,2.8E1,2.2E1,4E0,8E0,4E0,4E0,2.3E1,4E0,8E0,1.7E1,1.3E1,6E0,5E0,5E0,8E0,9E0,5.7E1,9E0,4E0,6E0,1.9E1,3E1,1E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[3.2505463E-3,-1.1654815E-1,3.6172676E-1,-3.590055E-1,-1.0095838E-2,-4.5593828E-1,4.0589243E-1,-5.265351E-1,-2.8014255E-1,-1.14962645E-1,9.217608E-2,-1.11706415E-2,-2.3904162E-2,1.0189188E0,3.708496E-1,-7.593408E-1,-4.1117904E-1,-4.1346416E-1,-2.1395218E-1,-1.6278714E-1,4.9649157E-2,1.6716285E-1,2.4673175E-2,3.0435188E-2,5.6986243E-2,2.9117757E-1,6.2600684E-1,-1.5495122E-2,-8.820214E-1,-2.0719653E-2,1.6941458E-3,-6.6070645E-3,-4.3224E-1,1.6305804E-2,-2.5471857E-1,-2.3922963E-1,-5.7356335E-2,7.908246E-2,-6.7375004E-2,9.513606E-2,2.5741884E-1,9.834508E-3,6.5021114E-3,9.256604E-2,3.666409E-1,7.5896996E-1,4.5508462E-1,-5.306592E-2,-3.192911E-2,-4.5905703E-1,-9.328848E-3,-3.917255E-1,-1.816627E-1,-2.0540728E-1,-4.8547304E-1,-2.5786358E-1,-2.0155588E-2,5.466478E-2,9.260237E-3,1.2802752E-3,-1.0397415E-2,1.234197E-1,-9.112795E-3,2.9675186E-1,-1.1783756E-3,-1.4289995E-1,3.198535E-2,1.701426E-1,-5.1127076E-2,4.4959348E-1,2.244506E-1,4.4486746E-2,2.9615002E-2,5.400847E-1,2.5657594E-1,-5.59759E-1,-3.9877042E-1,-2.676839E-1,-2.6219243E-2,-2.08072E-1,4.1356627E-3,-3.022815E-1,-1.5142633E-1,-1.0892647E-2,-3.599395E-2,-2.4272904E-3,-1.691322E-2,1.18870055E-2,-3.8883276E-2,-3.799378E-3,7.484267E-2,-6.754106E-2,1.4957097E-1,2.1982755E-1,1.9160535E-2,-1.9103704E-1,1.8349694E-3,4.8519544E-2,-9.21015E-3,1.568868E-2,1.13634735E-1,-1.312766E-1,3.487478E-3,4.7322676E-1,-6.343543E-4,3.0808827E-1,-1.85981E-2,2.7130585E-2,1.3859407E-2,2.9573445E-3,1.7291486E-2,-3.0396149E-2,-1.474656E-2,-1.4725634E-2,-2.3049546E-2,-3.7483564E-1,-1.1671515E-1,-2.5105888E-1,-9.0784915E-2,-1.601454E-1,-3.6172333E-1,-1.152518E-1,-1.4908189E-2,8.4625065E-2,-8.084283E-2,-4.8527718E-3,1.16478406E-1,-6.4006546E-3,7.840552E-4,1.5481961E-2,1.2702042E-1,2.6843554E-1,6.038011E-2,-2.4007928E-1,-1.6228916E-3,-3.0662296E-3,1.591481E-1,1.8448913E-1,6.110338E-2,-9.1805327E-4,-9.60999E-3,4.2353433E-1,6.0785896E-1,2.3549022E-1,3.0307941E-2,5.5388524E-3,-8.0485465E-3,-9.137015E-3,-1.9937396E-2,-1.2191463E-2,2.348425E-4,-4.0918523E-3,-2.6411435E-1,-3.68976E-2,-1.082397E-2,-9.446531E-3,-2.2325153E-3,-4.3337488E-1,-2.4201532E-1,1.13993045E-2,-1.373914E-1,-2.925816E-4,1.09302774E-1,2.5645468E-2,-1.1783019E-1,3.9501796E-3,-2.2889606E-3,7.961968E-3,2.7847271E-3,7.358406E-2,1.816897E-1,1.9170721E-1,1.7719213E-2,5.412028E-3,-3.9092152E-4,-4.9004625E-3,-1.3574363E-2,3.6472507E-2,-1.4243434E-1,8.738414E-2,1.523735E-2,1.0475402E-2,4.085527E-3,-8.1262493E-4,1.0562591E-1,2.566952E-2,3.455461E-1,3.1130448E-2,1.3322173E-2,1.3223937E-3,2.8040037E-1,-1.3813071E-2,-7.877615E-3,-4.636489E-3,1.4402999E-3,-2.222312E-2,-1.117366E-2,-3.419595E-3,-1.4634069E-2,-3.15442E-3,3.506412E-3,-7.409692E-3,-1.1445563E-3,2.3210004E-3,7.836609E-3,-4.517476E-3,3.6690293E-3,1.2596359E-3,-6.8607777E-3,-6.2639493E-4,6.9613336E-3,1.0762418E-2,5.0152917E-3,4.3124678E-3,1.0390745E-2,-2.8561084E-3,2.8797733E-3,-3.5921042E-4,-1.0899195E-2,1.3185804E-3,1.1227438E-2,7.2503136E-3,5.8515766E-4,1.7867856E-2,4.1635795E-3,1.42089855E-2,8.1980685E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,49,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,-1,-1,75,-1,77,79,81,83,85,87,89,-1,-1,-1,91,-1,93,-1,95,97,99,101,103,105,-1,-1,107,109,111,113,115,-1,117,-1,119,121,-1,-1,-1,-1,-1,123,-1,125,127,129,131,-1,133,-1,135,-1,-1,137,139,-1,141,-1,143,145,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,153,155,157,159,-1,161,163,165,167,-1,-1,-1,169,171,173,175,-1,177,179,181,183,-1,-1,185,187,189,-1,-1,-1,-1,-1,-1,-1,-1,191,193,-1,-1,-1,195,197,199,201,-1,203,205,207,-1,-1,-1,-1,209,211,213,-1,-1,-1,-1,-1,215,217,219,-1,-1,-1,-1,221,-1,223,-1,-1,-1,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5263226E1,2.038639E1,9.7188225E0,3.03656E0,5.909308E0,5.9205532E-2,5.0141563E0,1.728796E0,1.3764238E0,2.1493962E0,1.4081249E0,0E0,0E0,1.9198227E-1,4.6763306E0,9.5239735E-1,9.0197945E-1,2.123928E-1,2.7013812E0,1.6868105E0,2.188814E-1,8.336222E-1,5.0474364E-1,0E0,0E0,2.7238321E0,9.272804E-1,0E0,1.223011E-1,0E0,0E0,0E0,2.1611881E-1,0E0,9.8981E-1,9.1636133E-1,6.661143E-1,1.3720903E-1,2.011526E-1,6.445767E-1,6.5301085E-1,0E0,5.2235645E-1,5.883543E-1,1.4678326E0,1.04400635E-1,3.1700563E-1,0E0,0E0,4.390812E-2,0E0,6.3312626E-1,5.27107E-1,5.2422667E-1,8.427727E-1,2.9187888E-1,4.101042E-1,1.24939755E-1,0E0,0E0,0E0,3.54679E-1,0E0,3.717084E-1,0E0,1.8340248E-1,4.5720655E-1,2.869624E-1,2.0272456E-1,9.153366E-1,1.0424592E0,0E0,0E0,2.767706E-2,2.0269978E-1,1.20646E-1,8.942413E-2,3.283118E-1,0E0,3.0492496E-1,0E0,2.6312733E-1,4.088887E-1,0E0,0E0,0E0,0E0,0E0,3.8515317E-1,0E0,1.2665604E-1,5.6357764E-2,2.1266389E-1,2.3955548E-1,0E0,1.139043E-1,0E0,6.282425E-1,0E0,0E0,8.922315E-2,9.845179E-2,0E0,2.710762E-1,0E0,8.053863E-1,3.226034E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5203686E-2,1.985236E-1,7.631159E-2,1.3828415E-1,5.455941E-2,1.1812568E-1,1.7261052E-1,0E0,4.3340355E-2,2.1965146E-1,6.077222E-2,6.810048E-2,0E0,0E0,0E0,1.5022486E-1,1.4726806E-1,3.6161963E-2,5.5258095E-2,0E0,4.2398834E-1,3.9986098E-1,2.4596423E-2,6.141933E-2,0E0,0E0,4.3641567E-1,2.061801E-1,2.9855704E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.9386225E-2,6.79294E-2,0E0,0E0,0E0,2.0558596E-2,1.2723851E-1,5.6573566E-2,1.306861E-1,0E0,4.4922665E-2,1.0732127E-1,1.865294E-1,0E0,0E0,0E0,0E0,2.0359685E-1,7.5941324E-2,3.4529686E-2,0E0,0E0,0E0,0E0,0E0,1.5637608E-1,2.121692E-1,2.3117812E-1,0E0,0E0,0E0,0E0,5.0480276E-2,0E0,3.967309E-1,0E0,0E0,0E0,2.0917177E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,61,61,63,63,65,65,66,66,67,67,68,68,69,69,70,70,73,73,74,74,75,75,76,76,77,77,79,79,81,81,82,82,88,88,90,90,91,91,92,92,93,93,95,95,97,97,100,100,101,101,103,103,105,105,106,106,115,115,116,116,117,117,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,130,130,131,131,132,132,133,133,135,135,136,136,137,137,138,138,141,141,142,142,143,143,152,152,153,153,157,157,158,158,159,159,160,160,162,162,163,163,164,164,169,169,170,170,171,171,177,177,178,178,179,179,184,184,186,186,190,190],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,50,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,-1,-1,76,-1,78,80,82,84,86,88,90,-1,-1,-1,92,-1,94,-1,96,98,100,102,104,106,-1,-1,108,110,112,114,116,-1,118,-1,120,122,-1,-1,-1,-1,-1,124,-1,126,128,130,132,-1,134,-1,136,-1,-1,138,140,-1,142,-1,144,146,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,154,156,158,160,-1,162,164,166,168,-1,-1,-1,170,172,174,176,-1,178,180,182,184,-1,-1,186,188,190,-1,-1,-1,-1,-1,-1,-1,-1,192,194,-1,-1,-1,196,198,200,202,-1,204,206,208,-1,-1,-1,-1,210,212,214,-1,-1,-1,-1,-1,216,218,220,-1,-1,-1,-1,222,-1,224,-1,-1,-1,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8988005E5,1E0,3.434405E7,9.24E2,3.714144E8,2.4481E4,2E0,2.71E2,2.331083E6,3.7105262E0,-1.11706415E-2,-2.3904162E-2,6.23694E5,4.391553E6,1.5E1,3.7066666E2,2.416E3,1.2E1,2.61E2,1.8E1,4.265829E2,4.0869565E0,3.0435188E-2,5.6986243E-2,7.601393E2,1.6386554E0,-1.5495122E-2,3.6255838E2,-2.0719653E-2,1.6941458E-3,-6.6070645E-3,3.328629E6,1.6305804E-2,2.4360857E4,2.1E1,1E0,6.9307615E6,1.6923077E0,4.5158855E6,7.978062E6,9.834508E-3,2.1924414E2,1.124641E0,6.9E1,2.5959E4,8.81059E5,-5.306592E-2,-3.192911E-2,3.89E2,-9.328848E-3,5.17259E8,8.909296E4,1.5933333E2,2.85E2,3.1275E4,2.3112903E2,1.3020051E-5,9.260237E-3,1.2802752E-3,-1.0397415E-2,4.0449125E5,-9.112795E-3,1.0029973E8,-1.1783756E-3,9.48E0,2.5243637E-6,6E0,3.149243E8,3.2E1,5.712652E8,4.4486746E-2,2.9615002E-2,1.647E4,1.3E1,2.7546012E0,9.183673E-1,3.183206E-8,-2.6219243E-2,7.997723E6,4.1356627E-3,3.1E1,2E0,-1.0892647E-2,-3.599395E-2,-2.4272904E-3,-1.691322E-2,1.18870055E-2,1.3364486E0,-3.799378E-3,9.22E2,1.4327235E7,1.3600995E2,5.4983668E7,1.9160535E-2,2.008E3,1.8349694E-3,2.008E3,-9.21015E-3,1.568868E-2,1E1,1.2E1,3.487478E-3,1.711E4,-6.343543E-4,4.907764E3,6.619098E8,2.7130585E-2,1.3859407E-2,2.9573445E-3,1.7291486E-2,-3.0396149E-2,-1.474656E-2,-1.4725634E-2,-2.3049546E-2,2.56E2,4.9E1,7.090909E0,5.4444447E0,1E0,1.329762E2,6.055E3,-1.4908189E-2,2.2007043E0,3.373913E0,3.863E3,9.166947E6,-6.4006546E-3,7.840552E-4,1.5481961E-2,2.378914E6,1.763E3,1.1090909E1,2.083138E6,-1.6228916E-3,1.3254E4,6.875854E2,3.744E3,1.5E1,-9.1805327E-4,-9.60999E-3,1.798E3,9.2732E4,1.7068776E10,3.0307941E-2,5.5388524E-3,-8.0485465E-3,-9.137015E-3,-1.9937396E-2,-1.2191463E-2,2.348425E-4,-4.0918523E-3,8.01E2,2.132956E8,-1.082397E-2,-9.446531E-3,-2.2325153E-3,4.7563504E7,1.5277778E0,4.0906172E5,1.03860024E8,-2.925816E-4,1.3717398E4,6.07E2,2.57E2,3.9501796E-3,-2.2889606E-3,7.961968E-3,2.7847271E-3,2.2461708E6,9E0,1.2151898E0,1.7719213E-2,5.412028E-3,-3.9092152E-4,-4.9004625E-3,-1.3574363E-2,1.14E3,1.9E1,1.836095E6,1.523735E-2,1.0475402E-2,4.085527E-3,-8.1262493E-4,7.362415E1,2.566952E-2,2.7578741E1,3.1130448E-2,1.3322173E-2,1.3223937E-3,2.5345264E7,-1.3813071E-2,-7.877615E-3,-4.636489E-3,1.4402999E-3,-2.222312E-2,-1.117366E-2,-3.419595E-3,-1.4634069E-2,-3.15442E-3,3.506412E-3,-7.409692E-3,-1.1445563E-3,2.3210004E-3,7.836609E-3,-4.517476E-3,3.6690293E-3,1.2596359E-3,-6.8607777E-3,-6.2639493E-4,6.9613336E-3,1.0762418E-2,5.0152917E-3,4.3124678E-3,1.0390745E-2,-2.8561084E-3,2.8797733E-3,-3.5921042E-4,-1.0899195E-2,1.3185804E-3,1.1227438E-2,7.2503136E-3,5.8515766E-4,1.7867856E-2,4.1635795E-3,1.42089855E-2,8.1980685E-3],"split_indices":[2,43,17,7,2,7,12,32,2,43,69,0,0,1,43,3,4,9,33,70,3,67,73,0,0,67,69,0,4,0,0,0,9,0,43,3,13,43,68,62,62,0,67,53,8,9,9,0,0,1,0,46,48,4,2,1,67,53,0,0,0,43,0,5,0,71,52,8,7,3,47,0,0,44,8,68,71,52,0,9,0,10,6,0,0,0,0,0,68,0,44,12,67,5,0,2,0,2,0,0,3,3,0,44,0,4,7,0,0,0,0,0,0,0,0,10,0,73,69,8,4,12,0,69,69,12,60,0,0,0,9,2,73,12,0,44,67,10,3,0,0,44,44,46,0,0,0,0,0,0,0,0,2,46,0,0,0,7,68,43,7,0,48,2,44,0,0,0,0,43,8,68,0,0,0,0,0,2,8,43,0,0,0,0,73,0,71,0,0,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.052E3,7.89E2,2.63E2,2.4E2,5.49E2,1.3E1,2.5E2,7.5E1,1.65E2,2.71E2,2.78E2,4E0,9E0,1.2E1,2.38E2,2.3E1,5.2E1,5.3E1,1.12E2,2.1E2,6.1E1,1.31E2,1.47E2,6E0,6E0,1.83E2,5.5E1,6E0,1.7E1,4.8E1,4E0,4E0,4.9E1,7E0,1.05E2,1.21E2,8.9E1,4.9E1,1.2E1,7.4E1,5.7E1,1.2E1,1.35E2,5.1E1,1.32E2,2.9E1,2.6E1,5E0,1.2E1,4.3E1,6E0,3.5E1,7E1,1.08E2,1.3E1,1.3E1,7.6E1,4.2E1,7E0,8E0,4E0,6.8E1,6E0,5E1,7E0,1.9E1,1.16E2,3.3E1,1.8E1,8.2E1,5E1,8E0,2.1E1,1.7E1,9E0,1.3E1,3E1,2.2E1,1.3E1,6.4E1,6E0,3.7E1,7.1E1,8E0,5E0,5E0,8E0,4E0,7.2E1,5E0,3.7E1,8E0,6E1,3.2E1,1.8E1,1.5E1,4E0,1.09E2,7E0,7E0,2.6E1,1.1E1,7E0,7.8E1,4E0,3.7E1,1.3E1,1.3E1,4E0,4E0,5E0,8E0,5E0,1.9E1,1.1E1,1.2E1,1E1,4.6E1,1.8E1,1.2E1,2.5E1,6E1,1.1E1,1.8E1,5.4E1,1.3E1,2.4E1,4E0,4E0,5E0,5.5E1,2.4E1,8E0,1.1E1,4E0,7.5E1,3.4E1,1E1,1.6E1,5E0,6E0,6E1,1.8E1,3.2E1,5E0,7E0,6E0,4E0,8E0,4E0,6E0,4E0,4.2E1,1.4E1,4E0,8E0,4E0,1.4E1,1.1E1,9E0,5.1E1,4E0,1.4E1,1.4E1,4E1,4E0,9E0,1.1E1,1.3E1,2.9E1,2.6E1,1.6E1,8E0,4E0,4E0,4E0,7E0,5.9E1,1.6E1,2.5E1,9E0,6E0,4E0,6E0,1E1,2E1,4E1,1.4E1,4E0,6E0,2.6E1,2.9E1,1.3E1,7E0,7E0,1E1,4E0,4E0,7E0,4E0,5E0,4.2E1,9E0,8E0,6E0,4E0,1E1,7E0,3.3E1,1.4E1,1.5E1,1.4E1,1.2E1,5E0,1.1E1,1.2E1,4.7E1,7E0,9E0,1.9E1,6E0,6E0,4E0,3.4E1,6E0,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[9.038484E-3,-1.2883458E-1,2.5897896E-1,-3.4229952E-1,-2.2767497E-2,-4.104476E-1,2.9649198E-1,-2.956752E-1,-5.1426053E-1,-9.994397E-2,8.2556054E-2,-2.1128622E-1,-2.4282899E-2,1.7741084E-1,4.522518E-1,-4.2905805E-1,-1.8625176E-1,-1.9732893E-1,-5.968157E-1,-8.501878E-2,-3.3037852E-2,5.9052687E-2,3.5282513E-1,-2.869962E-3,-1.4700239E-2,3.4714615E-1,1.0899194E-1,7.490896E-1,3.6288962E-1,-4.9013755E-1,-3.4248704E-1,1.7432038E-2,-2.2666878E-1,-4.1610957E-3,-1.1353574E-2,-6.827386E-1,-1.2158992E-2,-1.2904565E-1,5.340388E-2,1.0425414E-1,-3.7144285E-2,6.8349433E-3,1.9120734E-2,2.4442317E-1,4.448084E-1,1.2789638E-1,-1.6036898E-2,4.6931082E-1,9.2146915E-1,5.40026E-1,2.4629833E-1,-1.579187E-2,-5.350738E-1,-5.937914E-3,-3.6521524E-1,-1.9714724E-1,-2.5926823E-2,-2.3892788E-2,-8.3399636E-1,-2.2614989E-1,-8.668468E-2,7.3612556E-2,-8.8562425E-3,4.4637952E-2,1.4767136E-1,-1.4357184E-1,8.642603E-2,1.3238703E-1,1.7590212E-2,4.8416483E-1,7.712279E-3,6.384352E-2,2.3062661E-1,3.0198673E-2,7.6339548E-3,2.4037002E-2,4.5702185E-2,6.253453E-1,3.0034307E-1,2.928491E-1,2.9202359E-2,-2.5938027E-2,-1.2954414E-2,-1.4005636E-2,-2.1072648E-2,-1.6577858E-1,-1.676818E-2,-4.501945E-2,-2.3455385E-2,-1.8234089E-1,-3.7888932E-1,-2.1259548E-2,-1.5575287E-1,-4.1580193E-2,1.1476305E-1,-6.599024E-3,6.762143E-2,1.6652918E-1,-5.867957E-3,-9.1943525E-2,-1.8186273E-2,1.4028837E-1,-5.79676E-3,1.6521081E-1,6.4692704E-4,1.2622806E-2,2.426811E-2,2.0737039E-1,2.8578728E-2,2.7084735E-1,-9.686181E-3,1.3115358E-2,3.0613964E-2,5.8611194E-3,1.9153757E-2,2.5180168E-2,2.3151086E-1,1.2744788E-2,-9.7050145E-2,-1.9552541E-1,2.5068957E-2,-2.1722208E-1,-4.8436772E-2,-8.439384E-3,-2.5162708E-2,-4.962803E-2,5.6526614E-3,1.7392533E-3,-1.7572029E-1,-9.547308E-2,1.9103154E-3,8.470641E-2,1.0887848E-2,-3.8268674E-2,9.727466E-2,1.2087663E-2,1.400573E-1,-1.6351989E-2,-1.6096766E-1,3.9882407E-2,1.1863329E-2,3.417695E-3,1.0366536E-2,2.832335E-1,-1.4482354E-3,1.0993777E-2,1.0036747E-2,3.687134E-1,1.5354046E-1,1.7514554E-1,2.0302527E-2,-9.418493E-3,3.7603674E-4,-2.6285863E-1,-1.3030097E-1,-9.0386387E-4,3.813366E-3,-2.3222806E-1,-4.6654847E-3,-4.7475668E-3,6.450564E-4,-9.573933E-2,4.0566247E-2,-2.4257724E-1,-1.21034555E-1,-1.2344385E-3,-6.3225767E-3,7.3939515E-3,5.7465035E-2,1.8067614E-3,-4.0687923E-3,4.295742E-2,1.4647219E-1,1.6614866E-1,-4.251073E-3,-4.428699E-3,2.5530807E-3,-4.714015E-3,-1.0280934E-2,8.606419E-2,-2.3663742E-3,1.6952554E-2,8.5489005E-3,-5.757856E-2,3.9939586E-2,2.2588288E-2,2.307433E-1,2.1240956E-1,4.2739182E-4,2.3145424E-1,-8.865486E-3,-5.257754E-3,-1.3586165E-2,-7.855505E-3,-6.4492127E-4,-7.4248053E-3,-1.2885934E-2,-7.843881E-4,-6.574897E-3,-3.31523E-3,5.322587E-3,-1.2294298E-2,-4.1806474E-3,-1.6670658E-3,-7.3210024E-3,4.6119912E-4,4.2712283E-3,-2.9135854E-3,3.9420547E-3,5.457052E-4,8.389474E-3,1.0541899E-2,4.8453845E-3,6.2551233E-3,9.018273E-4,-8.8055115E-3,-2.4561705E-5,2.3880196E-4,7.7648554E-3,5.467038E-3,1.5072199E-2,1.2010223E-2,2.1380277E-3,6.979607E-3,1.672586E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,47,49,51,53,-1,55,-1,-1,57,-1,59,61,63,65,-1,-1,67,69,71,-1,73,75,77,79,-1,81,-1,83,85,-1,-1,87,89,91,93,-1,95,97,99,101,103,-1,105,-1,107,109,-1,-1,-1,-1,111,113,115,117,-1,-1,-1,-1,119,-1,-1,-1,121,123,125,127,129,131,-1,133,135,-1,137,-1,139,-1,141,-1,-1,-1,143,145,147,-1,-1,-1,-1,-1,-1,149,-1,151,153,155,157,159,-1,-1,161,-1,-1,163,165,-1,167,-1,169,171,-1,173,175,177,179,-1,-1,-1,181,-1,183,-1,185,187,189,-1,-1,-1,191,193,-1,-1,195,-1,-1,-1,197,199,201,203,-1,-1,-1,205,-1,-1,207,209,211,-1,-1,-1,-1,-1,213,-1,-1,-1,215,217,-1,219,221,-1,223,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5768444E1,1.513944E1,9.3881E0,1.6278458E0,3.6498191E0,3.4768152E-1,6.411585E0,2.503439E0,1.0857916E0,2.3172398E0,1.1860889E0,1.2375435E-1,0E0,2.2847843E0,3.7336025E0,2.4986362E-1,2.3405576E0,3.49994E-2,8.522997E-1,1.5548478E0,0E0,7.6916337E-1,1.2639368E-1,0E0,0E0,4.631114E-1,1.2860588E0,1.2073002E0,2.303996E0,1.1034775E-1,1.2812328E-1,0E0,8.0781317E-1,0E0,0E0,2.698984E-1,0E0,7.742882E-1,3.2120812E-1,3.0401254E-1,7.6203173E-1,0E0,0E0,4.0779436E-1,2.2538614E-1,8.999188E-1,0E0,7.328346E-1,2.910614E-2,7.52862E-1,7.291846E-1,0E0,5.7444572E-2,0E0,4.22554E-2,3.9233136E-1,0E0,0E0,1.0111141E-1,3.1901193E-1,6.112449E-1,2.797057E-1,0E0,2.3372027E-1,3.7857032E-1,3.6405146E-1,3.257645E-1,6.7631155E-2,0E0,6.469536E-2,0E0,4.344194E-1,9.9534893E-1,0E0,0E0,0E0,0E0,2.322731E-1,2.2487509E-1,8.019414E-1,4.6276113E-1,0E0,0E0,0E0,0E0,4.3068504E-1,0E0,0E0,0E0,2.0900309E-1,2.9082143E-1,2.9533422E-1,2.6148736E-1,7.545705E-2,1.3630474E-1,0E0,1.5130182E-1,1.2701488E-1,0E0,1.3824415E-1,0E0,2.470968E-1,0E0,5.898586E-2,0E0,0E0,0E0,3.1674695E-1,2.3317374E-1,5.0592566E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.090804E-1,0E0,1.1402033E-1,2.512145E-1,2.9832404E-2,4.1084528E-2,3.872416E-2,0E0,0E0,2.5342226E-1,0E0,0E0,1.9000721E-1,2.4794668E-2,0E0,6.468722E-2,0E0,4.5833245E-2,9.258211E-2,0E0,3.3620977E-1,8.501391E-2,2.808845E-2,5.910361E-2,0E0,0E0,0E0,3.8953424E-2,0E0,1.3330185E-1,0E0,3.1843972E-1,2.0189536E-1,8.917923E-1,0E0,0E0,0E0,1.10788345E-1,1.5942556E-1,0E0,0E0,6.203127E-2,0E0,0E0,0E0,1.4432275E-1,1.8434021E-1,6.899512E-2,1.05576575E-1,0E0,0E0,0E0,4.5593224E-2,0E0,0E0,9.07336E-2,8.7715864E-2,1.585064E-1,0E0,0E0,0E0,0E0,0E0,2.6900962E-2,0E0,0E0,0E0,1.5166724E-1,2.0853072E-1,0E0,1.1168337E-1,1.2228173E-1,0E0,3.2316256E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,32,32,35,35,37,37,38,38,39,39,40,40,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,54,54,55,55,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,71,71,72,72,77,77,78,78,79,79,80,80,85,85,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,99,99,101,101,103,103,107,107,108,108,109,109,116,116,118,118,119,119,120,120,121,121,122,122,125,125,128,128,129,129,131,131,133,133,134,134,136,136,137,137,138,138,139,139,143,143,145,145,147,147,148,148,149,149,153,153,154,154,157,157,161,161,162,162,163,163,164,164,168,168,171,171,172,172,173,173,179,179,183,183,184,184,186,186,187,187,189,189],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,48,50,52,54,-1,56,-1,-1,58,-1,60,62,64,66,-1,-1,68,70,72,-1,74,76,78,80,-1,82,-1,84,86,-1,-1,88,90,92,94,-1,96,98,100,102,104,-1,106,-1,108,110,-1,-1,-1,-1,112,114,116,118,-1,-1,-1,-1,120,-1,-1,-1,122,124,126,128,130,132,-1,134,136,-1,138,-1,140,-1,142,-1,-1,-1,144,146,148,-1,-1,-1,-1,-1,-1,150,-1,152,154,156,158,160,-1,-1,162,-1,-1,164,166,-1,168,-1,170,172,-1,174,176,178,180,-1,-1,-1,182,-1,184,-1,186,188,190,-1,-1,-1,192,194,-1,-1,196,-1,-1,-1,198,200,202,204,-1,-1,-1,206,-1,-1,208,210,212,-1,-1,-1,-1,-1,214,-1,-1,-1,216,218,-1,220,222,-1,224,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,1E0,2.1E1,9.31E2,1.91E2,1.1459359E3,2.71E2,1E0,1.1290322E1,5.197839E6,1E0,-2.4282899E-2,8E0,3.4583332E1,3.89E2,2E0,4.0869565E0,6.03E2,2.331083E6,-3.3037852E-2,5.194E3,5.45901E6,-2.869962E-3,-1.4700239E-2,5.691698E2,1.0428572E1,1.562752E3,2.2842104E0,2.416E3,4.456432E6,1.7432038E-2,3.2856784E7,-4.1610957E-3,-1.1353574E-2,4.6463413E0,-1.2158992E-2,1.8149019E2,2.7E1,2.378914E6,7.9016544E5,6.8349433E-3,1.9120734E-2,1E0,1.4553613E1,6.9664386E2,-1.6036898E-2,1.934E3,9E0,2.395631E7,2.3876712E7,-1.579187E-2,7.338571E2,-5.937914E-3,1.5272727E0,3.78E2,-2.5926823E-2,-2.3892788E-2,1.746E3,2.4E0,1.7755102E0,3.0479442E9,-8.8562425E-3,5.4603375E5,1E0,1.28636E7,1.2906634E7,3.222E3,1.7590212E-2,1.000501E6,7.712279E-3,2.992648E-2,1.6063418E7,3.0198673E-2,7.6339548E-3,2.4037002E-2,4.5702185E-2,1.4992306E8,5.854E3,3.3382E4,1.2176E4,-2.5938027E-2,-1.2954414E-2,-1.4005636E-2,-2.1072648E-2,1.3022917E8,-1.676818E-2,-4.501945E-2,-2.3455385E-2,5.41E2,4.9963706E5,3.961275E2,1.6580646E2,2.8080197E2,6.7723384E7,-6.599024E-3,1E0,1.880118E-3,-5.867957E-3,3.78E2,-1.8186273E-2,5.4007E7,-5.79676E-3,4.23025E2,6.4692704E-4,1.2622806E-2,2.426811E-2,1.302273E6,6.0450394E8,3.8396227E0,-9.686181E-3,1.3115358E-2,3.0613964E-2,5.8611194E-3,1.9153757E-2,2.5180168E-2,4.435876E2,1.2744788E-2,2.9925186E-2,8.147158E4,7.8918E4,1.3337367E7,8.8790035E-1,-8.439384E-3,-2.5162708E-2,4.39776E6,5.6526614E-3,1.7392533E-3,2.3238889E2,5.2846E4,1.9103154E-3,8.5E1,1.0887848E-2,1.821E3,6.077143E5,1.2087663E-2,1E0,1.445E3,1.1860938E8,1.3062992E1,1.1863329E-2,3.417695E-3,1.0366536E-2,1E0,-1.4482354E-3,3.0805944E7,1.0036747E-2,1.2629019E8,4.771E3,1.8545505E2,2.0302527E-2,-9.418493E-3,3.7603674E-4,1E0,1.605076E6,-9.0386387E-4,3.813366E-3,1.6667E4,-4.6654847E-3,-4.7475668E-3,6.450564E-4,1.4473684E-1,8.856631E0,1.339646E6,1E0,-1.2344385E-3,-6.3225767E-3,7.3939515E-3,1.5539158E3,1.8067614E-3,-4.0687923E-3,2.9432205E6,2.906237E2,8E0,-4.251073E-3,-4.428699E-3,2.5530807E-3,-4.714015E-3,-1.0280934E-2,3.2283451E4,-2.3663742E-3,1.6952554E-2,8.5489005E-3,1.5204346E8,1.3261502E0,2.2588288E-2,1.1342433E3,7.0296685E6,4.2739182E-4,2.5134058E3,-8.865486E-3,-5.257754E-3,-1.3586165E-2,-7.855505E-3,-6.4492127E-4,-7.4248053E-3,-1.2885934E-2,-7.843881E-4,-6.574897E-3,-3.31523E-3,5.322587E-3,-1.2294298E-2,-4.1806474E-3,-1.6670658E-3,-7.3210024E-3,4.6119912E-4,4.2712283E-3,-2.9135854E-3,3.9420547E-3,5.457052E-4,8.389474E-3,1.0541899E-2,4.8453845E-3,6.2551233E-3,9.018273E-4,-8.8055115E-3,-2.4561705E-5,2.3880196E-4,7.7648554E-3,5.467038E-3,1.5072199E-2,1.2010223E-2,2.1380277E-3,6.979607E-3,1.672586E-2],"split_indices":[2,43,17,3,2,0,67,2,20,69,43,15,0,3,73,1,10,73,10,43,0,44,60,0,0,67,68,67,69,9,7,0,60,0,0,68,0,70,3,9,43,0,0,23,71,67,0,44,3,62,60,0,48,0,71,0,0,0,44,71,68,46,0,60,79,60,12,2,0,43,0,53,62,0,0,0,0,5,2,9,9,0,0,0,0,7,0,0,0,2,43,70,67,4,7,0,29,53,0,0,0,7,0,4,0,0,0,9,7,69,0,0,0,0,0,0,71,0,72,43,1,5,71,0,0,9,0,0,67,9,0,10,0,44,62,0,19,2,5,71,0,0,0,23,0,5,0,5,2,73,0,0,0,15,12,0,0,9,0,0,0,71,73,9,26,0,0,0,48,0,0,60,4,8,0,0,0,0,0,48,0,0,0,7,57,0,4,62,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.036E3,6.68E2,3.68E2,2.21E2,4.47E2,1.9E1,3.49E2,1.76E2,4.5E1,2.58E2,1.89E2,8E0,1.1E1,1.99E2,1.5E2,7.8E1,9.8E1,1E1,3.5E1,2.53E2,5E0,1.75E2,1.4E1,4E0,4E0,5.6E1,1.43E2,3.3E1,1.17E2,4.3E1,3.5E1,6E0,9.2E1,4E0,6E0,2.7E1,8E0,1.92E2,6.1E1,1.19E2,5.6E1,4E0,1E1,2.9E1,2.7E1,1.38E2,5E0,1.4E1,1.9E1,4.5E1,7.2E1,1.2E1,3.1E1,4E0,3.1E1,8.6E1,6E0,1.5E1,1.2E1,5.7E1,1.35E2,5.7E1,4E0,5.1E1,6.8E1,3E1,2.6E1,1.7E1,1.2E1,2.3E1,4E0,8.6E1,5.2E1,8E0,6E0,4E0,1.5E1,3.2E1,1.3E1,5.9E1,1.3E1,2.7E1,4E0,2.1E1,1E1,7.4E1,1.2E1,7E0,5E0,4.6E1,1.1E1,7E1,6.5E1,1.5E1,4.2E1,5E0,4.6E1,6.4E1,4E0,2.6E1,4E0,2.1E1,5E0,1.3E1,4E0,5E0,1.8E1,1.6E1,7E1,4.8E1,4E0,4E0,2.8E1,6E0,7E0,1E1,4.9E1,4E0,9E0,6.4E1,1E1,3.6E1,1E1,6E0,5E0,5.9E1,1.1E1,6E0,5.9E1,9E0,6E0,3.5E1,7E0,1E1,3.6E1,1.2E1,5.2E1,1.3E1,1.3E1,1.2E1,9E0,6E0,7E0,1.2E1,4E0,6.5E1,5E0,2.5E1,2.3E1,4E1,9E0,4E0,5E0,3E1,3.4E1,6E0,4E0,3.1E1,5E0,5E0,5E0,3.9E1,2E1,2.5E1,3.4E1,4E0,5E0,8E0,2.7E1,4E0,6E0,1.8E1,1.8E1,4.7E1,5E0,6E0,7E0,8E0,5E0,8E0,4E0,5E0,7E0,1.9E1,4.6E1,1.2E1,1.3E1,1.6E1,7E0,3.5E1,5E0,6E0,2.4E1,2.5E1,9E0,1.4E1,1.7E1,1.5E1,2.4E1,8E0,1.2E1,2.1E1,4E0,1.1E1,2.3E1,1.2E1,1.5E1,5E0,1.3E1,4E0,1.4E1,2.2E1,2.5E1,4E0,4E0,5E0,1.4E1,3.7E1,9E0,7E0,6E0,1.2E1,4E0,2.3E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"225","size_leaf_vector":"1"}},{"base_weights":[3.913325E-3,-9.931895E-2,3.1083712E-1,-3.1763157E-1,-2.1071972E-3,-4.4784325E-1,3.5547167E-1,-4.5457786E-1,-2.5262824E-1,-9.130332E-2,9.1489136E-2,-2.5141533E-2,-9.822276E-3,4.3573078E-2,3.1894782E-1,-4.1667438E-1,-4.2016767E-2,-3.936701E-1,-1.9808681E-1,-6.4099506E-2,-3.05111E-1,1.7501368E-1,3.434601E-2,1.5826014E-1,4.5669937E-1,-3.713674E-1,-2.9297657E-2,-2.4053128E-2,-3.258181E-1,3.4862563E-1,-2.451318E-1,-2.639635E-1,-3.4193274E-2,-2.065337E-1,-2.5996389E-2,2.1157478E-1,2.4137817E-2,1.2612889E-2,1.9797083E-2,1.2478939E-1,3.3153426E-2,3.062011E-1,5.5495554E-1,-2.0340558E-2,-2.4390937E-1,-1.8274166E-1,-3.7385923E-1,8.407582E-3,2.2123247E-2,-3.3229895E-2,-2.209109E-1,-1.82136E-1,-3.5616052E-1,-1.097188E-1,-1.6712171E-3,-2.4444067E-1,-2.4057087E-3,1.7775781E-1,1.7915823E-2,9.569802E-2,-8.437751E-3,3.8218956E-2,-1.368748E-1,-4.61842E-2,1.6069409E-1,3.5291427E-1,-2.0565116E-3,5.8687085E-1,-4.2657557E-4,-1.3189177E-1,-1.6830482E-2,-4.1841697E-3,-1.0584011E-2,-1.9104682E-2,-6.853139E-3,9.133671E-3,-2.3891242E-1,-9.636414E-3,-3.065895E-3,-8.395773E-3,-1.9998362E-2,2.4143733E-2,-1.5036044E-1,-2.6829498E-2,1.2720603E-1,-5.235127E-3,-1.308249E-2,1.508029E-2,1.5215199E-1,1.3449143E-1,-1.1228424E-3,-9.066222E-2,6.833433E-2,-7.916652E-5,-1.1171382E-2,-8.8848695E-3,5.5086453E-2,1.8045875E-1,-9.218476E-3,4.0806955E-1,1.8441735E-3,3.2386098E-2,4.6930048E-1,-8.346902E-3,-3.3596912E-3,-2.7991658E-1,-7.335519E-2,-3.6704876E-3,4.3540094E-3,-1.982777E-1,-1.2981921E-2,-7.813894E-2,3.0807743E-2,1.8298261E-1,-6.121228E-3,1.6421536E-1,-1.1758386E-3,2.7201432E-3,8.056629E-3,-1.0870488E-2,-4.50208E-2,-5.41519E-2,9.450346E-2,-1.7841109E-3,5.891995E-3,3.3544743E-1,1.3843986E-1,4.3896627E-1,5.0901463E-3,4.9769557E-1,1.0852653E-2,-1.2632258E-3,-2.9235902E-1,-3.0671412E-3,-1.3167296E-1,-3.5097697E-1,-1.1094296E-1,1.8554485E-3,-3.3200176E-3,-1.1270994E-1,4.6013794E-3,8.948036E-2,-5.161618E-2,8.999942E-2,1.4914457E-2,2.9274397E-2,1.8403146E-1,-9.048598E-3,1.4358122E-2,3.214653E-3,-7.4449317E-3,2.0508094E-2,1.3625069E-1,1.0811578E-2,1.9417403E-2,9.204352E-2,2.596275E-1,9.556691E-3,2.1355174E-2,2.5890335E-2,1.5765306E-2,-1.0036199E-2,-1.617885E-2,-2.627989E-3,2.3682076E-3,-8.100321E-3,-1.639086E-3,-2.5239557E-2,-9.399405E-3,2.6148575E-4,-8.734189E-3,1.9759787E-3,-5.905007E-3,2.4998833E-3,-3.8167748E-3,3.16947E-3,7.113256E-3,-7.1091275E-4,-5.467072E-3,-3.112851E-4,5.9766853E-3,-1.2287435E-3,3.6631525E-3,3.609185E-3,9.420872E-3,4.633323E-3,-1.0461089E-3,2.9267166E-3,-1.7201228E-3,6.909068E-3,-1.1526215E-3,4.6080537E-3,1.392952E-2,9.913788E-4,6.5795113E-3,6.3021085E-3,1.3699164E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,-1,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,-1,57,59,-1,61,63,-1,65,67,-1,69,71,73,-1,-1,-1,75,77,79,81,83,85,-1,87,-1,89,-1,91,93,95,97,99,-1,101,-1,103,-1,-1,-1,-1,-1,-1,105,-1,-1,-1,-1,107,109,111,113,-1,-1,-1,115,117,-1,119,121,-1,-1,-1,123,125,-1,127,-1,-1,129,-1,-1,131,133,-1,-1,135,137,139,141,143,-1,145,-1,-1,-1,-1,147,149,151,-1,-1,153,155,157,-1,159,-1,-1,161,163,165,167,169,-1,-1,171,173,175,177,179,-1,181,183,-1,185,187,-1,189,191,-1,-1,193,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2982693E1,1.65317E1,9.031397E0,2.0216312E0,4.5165052E0,2.2429752E-1,5.0224094E0,9.314451E-1,1.1913033E0,1.5915806E0,1.2540224E0,0E0,0E0,0E0,5.129442E0,4.7395802E-1,0E0,2.1805143E-1,3.197588E0,1.4673373E0,6.709602E-1,5.8512497E-1,5.432236E-1,1.9698946E0,1.6932831E0,4.3719292E-1,0E0,0E0,1.5163422E-1,1.2988806E-1,1.1273355E0,1.7654634E-1,5.303656E-1,1.2779498E-1,0E0,4.4356394E-1,3.4301212E-1,0E0,4.3815967E-1,6.519908E-1,0E0,8.6762667E-1,1.3558903E0,0E0,2.75622E-1,2.1977395E-2,1.4442635E-1,0E0,0E0,0E0,8.6108685E-1,4.4336677E-2,1.0866916E-1,3.572557E-1,4.9624702E-1,6.441629E-2,0E0,2.3202395E-1,0E0,8.2408905E-2,0E0,5.301715E-1,2.2651586E-1,2.9177466E-1,6.5543747E-1,7.7782726E-1,0E0,6.238079E-1,0E0,2.7269974E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.854744E-1,0E0,0E0,0E0,0E0,1.2367914E-1,3.2656598E-1,3.809461E-1,3.863781E-1,0E0,0E0,0E0,1.4344203E-1,2.8950542E-2,0E0,1.6104124E-1,3.577069E-1,0E0,0E0,0E0,8.6390056E-2,4.9409437E-1,0E0,3.105855E-1,0E0,0E0,1.3599586E-1,0E0,0E0,2.5284433E-1,8.977506E-2,0E0,0E0,4.5449173E-1,4.6694864E-2,1.9502687E-1,2.996163E-1,2.440188E-1,0E0,1.5772676E-1,0E0,0E0,0E0,0E0,1.9090365E-1,1.2327917E-1,2.7831393E-1,0E0,0E0,4.7944784E-2,3.5282135E-1,9.347296E-2,0E0,1.3253593E-1,0E0,0E0,2.6717567E-1,3.5077408E-2,4.4179976E-2,2.6347935E-1,2.287741E-1,0E0,0E0,1.1820942E-1,9.515066E-2,3.7902355E-2,6.1025783E-2,5.4189637E-2,0E0,2.7259042E-2,9.570241E-2,0E0,5.3145107E-2,3.641135E-2,0E0,2.0317085E-1,3.198514E-1,0E0,0E0,1.7472222E-1,4.13841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,38,38,39,39,41,41,42,42,44,44,45,45,46,46,50,50,51,51,52,52,53,53,54,54,55,55,57,57,59,59,61,61,62,62,63,63,64,64,65,65,67,67,69,69,76,76,81,81,82,82,83,83,84,84,88,88,89,89,91,91,92,92,96,96,97,97,99,99,102,102,105,105,106,106,109,109,110,110,111,111,112,112,113,113,115,115,120,120,121,121,122,122,125,125,126,126,127,127,129,129,132,132,133,133,134,134,135,135,136,136,139,139,140,140,141,141,142,142,143,143,145,145,146,146,148,148,149,149,151,151,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,-1,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,-1,58,60,-1,62,64,-1,66,68,-1,70,72,74,-1,-1,-1,76,78,80,82,84,86,-1,88,-1,90,-1,92,94,96,98,100,-1,102,-1,104,-1,-1,-1,-1,-1,-1,106,-1,-1,-1,-1,108,110,112,114,-1,-1,-1,116,118,-1,120,122,-1,-1,-1,124,126,-1,128,-1,-1,130,-1,-1,132,134,-1,-1,136,138,140,142,144,-1,146,-1,-1,-1,-1,148,150,152,-1,-1,154,156,158,-1,160,-1,-1,162,164,166,168,170,-1,-1,172,174,176,178,180,-1,182,184,-1,186,188,-1,190,192,-1,-1,194,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.8988005E5,1E0,3.3832976E7,9.31E2,9.3326636E-2,2.4481E4,6.9934395E-5,2.71E2,2.9802957E0,3.1706784E0,-2.5141533E-2,-9.822276E-3,4.3573078E-2,1.7587205E6,4.1361522E6,-4.2016767E-2,3.89E2,2.5E-1,3.89E2,2.0303884E6,1.0292E4,1.3208092E0,5.428175E3,5.97E3,4.622222E1,-2.9297657E-2,-2.4053128E-2,1.16E2,1.9E1,1.0479E4,3.8275862E0,1E0,1.339646E6,-2.5996389E-2,1.28738E5,4.68418E5,1.2612889E-2,1E0,1E0,3.3153426E-2,1.2673605E7,2.2313573E2,-2.0340558E-2,1E0,1.2E2,3.328629E6,8.407582E-3,2.2123247E-2,-3.3229895E-2,2.2E1,3E0,4.4761734E5,5E-1,5.5593027E9,3.1488764E0,-2.4057087E-3,1.9913E4,1.7915823E-2,1.9851096E7,-8.437751E-3,1E0,6.763312E7,2.0939393E0,3.0070068E7,1.9269184E7,-2.0565116E-3,1.7288135E0,-4.2657557E-4,1.16408E5,-1.6830482E-2,-4.1841697E-3,-1.0584011E-2,-1.9104682E-2,-6.853139E-3,9.133671E-3,7.997723E6,-9.636414E-3,-3.065895E-3,-8.395773E-3,-1.9998362E-2,5.44E2,1.2673605E7,2.956111E2,1.3508157E7,-5.235127E-3,-1.308249E-2,1.508029E-2,3.472E3,1.42946E5,-1.1228424E-3,4.8709216E5,3.7990784E7,-7.916652E-5,-1.1171382E-2,-8.8848695E-3,1.090477E6,1.4007937E0,-9.218476E-3,5.526E3,1.8441735E-3,3.2386098E-2,1.9995576E7,-8.346902E-3,-3.3596912E-3,1E0,1.8E0,-3.6704876E-3,4.3540094E-3,5.4E0,6.346204E-7,3.011152E6,1E0,1.65E2,-6.121228E-3,4.0906172E5,-1.1758386E-3,2.7201432E-3,8.056629E-3,-1.0870488E-2,4.88E2,5.2663404E-1,5.095006E5,-1.7841109E-3,5.891995E-3,1.1772152E0,1.9616238E2,2.0045958E6,5.0901463E-3,1.407E3,1.0852653E-2,-1.2632258E-3,1.6213593E1,1.3823239E2,5.7E3,2.8055556E0,1.5277778E0,1.8554485E-3,-3.3200176E-3,1.631108E6,1E0,6.5E1,4.017544E0,6.078218E2,1.4914457E-2,1.2E1,1.071E3,-9.048598E-3,1.5724638E1,1.15E2,-7.4449317E-3,1.26E2,2.954124E6,1.0811578E-2,1.9417403E-2,5.615327E5,1.7755102E0,9.556691E-3,2.1355174E-2,2.5890335E-2,1.5765306E-2,-1.0036199E-2,-1.617885E-2,-2.627989E-3,2.3682076E-3,-8.100321E-3,-1.639086E-3,-2.5239557E-2,-9.399405E-3,2.6148575E-4,-8.734189E-3,1.9759787E-3,-5.905007E-3,2.4998833E-3,-3.8167748E-3,3.16947E-3,7.113256E-3,-7.1091275E-4,-5.467072E-3,-3.112851E-4,5.9766853E-3,-1.2287435E-3,3.6631525E-3,3.609185E-3,9.420872E-3,4.633323E-3,-1.0461089E-3,2.9267166E-3,-1.7201228E-3,6.909068E-3,-1.1526215E-3,4.6080537E-3,1.392952E-2,9.913788E-4,6.5795113E-3,6.3021085E-3,1.3699164E-2],"split_indices":[2,43,17,7,2,53,12,52,2,68,69,0,0,0,43,60,0,1,73,2,60,44,68,67,2,4,0,0,2,0,9,69,29,9,0,1,9,0,19,27,0,60,73,0,8,44,9,0,0,0,10,8,43,68,46,73,0,9,0,60,0,29,7,68,58,62,0,69,0,7,0,0,0,0,0,0,9,0,0,0,0,12,60,4,60,0,0,0,10,1,0,43,7,0,0,0,60,68,0,2,0,0,9,0,0,13,68,0,0,73,52,9,26,0,0,43,0,0,0,0,4,57,43,0,0,68,71,43,0,0,0,0,73,67,10,71,68,0,0,5,30,0,69,48,0,3,2,0,73,0,0,10,62,0,0,43,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,7.78E2,2.61E2,2.39E2,5.39E2,1.4E1,2.47E2,7.5E1,1.64E2,2.76E2,2.63E2,9E0,5E0,1.3E1,2.34E2,7.1E1,4E0,4.4E1,1.2E2,2.46E2,3E1,1.06E2,1.57E2,1.09E2,1.25E2,6.1E1,1E1,1.3E1,3.1E1,9E0,1.11E2,3.1E1,2.15E2,2.3E1,7E0,8.5E1,2.1E1,8E0,1.49E2,1.04E2,5E0,5.1E1,7.4E1,3.8E1,2.3E1,9E0,2.2E1,5E0,4E0,4E0,1.07E2,1.8E1,1.3E1,6.4E1,1.51E2,1.8E1,5E0,7.3E1,1.2E1,1.6E1,5E0,1.34E2,1.5E1,1.8E1,8.6E1,4.5E1,6E0,7E1,4E0,1.3E1,1E1,4E0,5E0,1.8E1,4E0,4E0,1.03E2,1.4E1,4E0,5E0,8E0,1.5E1,4.9E1,1.27E2,2.4E1,5E0,1.3E1,9E0,6.4E1,1.2E1,4E0,2.5E1,1.09E2,7E0,8E0,7E0,1.1E1,8.2E1,4E0,3.8E1,7E0,3.3E1,3.7E1,6E0,7E0,8.2E1,2.1E1,6E0,9E0,3.6E1,1.3E1,6.7E1,6E1,2E1,4E0,6E1,4E0,5E0,7E0,5E0,2E1,1.9E1,9E1,5E0,6E0,1.6E1,6.6E1,3.4E1,4E0,3.2E1,5E0,4E0,7.8E1,1E1,1.1E1,1.2E1,2.4E1,7E0,6E0,4.7E1,2E1,3.5E1,2.5E1,1.3E1,7E0,8E0,5.2E1,5E0,1.5E1,1.3E1,6E0,3.3E1,5.7E1,9E0,7E0,4.9E1,1.7E1,4E0,3E1,2.1E1,1.1E1,3.6E1,4.2E1,5E0,5E0,7E0,4E0,4E0,8E0,1E1,1.4E1,4E0,4.3E1,1.3E1,7E0,2.8E1,7E0,1.7E1,8E0,4E0,9E0,4E0,4E0,9E0,4.3E1,4E0,1.1E1,5E0,8E0,8E0,2.5E1,4.8E1,9E0,2.1E1,2.8E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-2.1113707E-3,-1.0200969E-1,2.9720458E-1,-3.0300242E-1,-7.6127104E-3,-4.0487608E-1,3.3611038E-1,-4.32657E-1,-2.3474638E-1,-8.9797586E-2,8.5915886E-2,-2.2989823E-2,-6.3940496E-3,1.9984965E-1,4.3687844E-1,-4.8388547E-1,-2.7575174E-1,-3.1926566E-1,-9.4179496E-2,-2.3252317E-1,-6.087035E-2,1.5551583E-1,1.09599745E-2,3.3258677E-2,3.0441844E-1,3.7213E-2,3.862008E-1,-4.2371097E-1,-3.2307148E-2,-1.2989971E-1,-1.8416071E-2,-2.7564287E-1,-5.7377714E-1,3.7168902E-1,-1.6870017E-1,-1.5832473E-1,-2.6235604E-1,-4.6340823E-2,-3.5253155E-1,1.7503917E-1,-7.3555475E-3,3.770238E-2,-1.6522206E-1,-7.3001226E-3,6.3377075E-2,3.429147E-1,-5.881937E-4,4.063643E-1,-8.269427E-3,-4.4448215E-1,-1.0609695E-2,-1.1470621E-2,1.3799048E-3,-3.544665E-1,-2.0092966E-1,-1.1726351E-2,-3.6128372E-2,2.182578E-2,9.083242E-3,-8.356933E-2,-3.2560232E-1,-1.1683327E-2,-4.266479E-3,-3.1641376E-1,-1.7025228E-1,-7.5457096E-2,5.7075273E-2,-4.0173042E-4,-2.9457076E-2,1.3323134E-1,3.30252E-1,9.147734E-3,1.752083E-1,-1.2054948E-4,-2.914087E-1,3.6195736E-2,1.0630331E-2,3.89102E-1,1.699616E-1,3.0364278E-1,5.201724E-1,-1.6423136E-2,-2.2907563E-2,-3.745789E-1,-6.0390932E-3,4.752744E-3,-2.4677978E-1,-1.4514278E-1,1.6414579E-2,-1.7861769E-2,-7.6578357E-3,-8.806472E-3,-1.727421E-2,-1.05719855E-2,-3.6487186E-3,-5.0267972E-2,-2.2558868E-1,8.590074E-2,-6.110608E-3,1.4697424E-1,-4.605972E-3,1.7539717E-2,1.0079776E-2,2.7939972E-2,-9.807906E-3,1.1873181E-2,8.407445E-2,-1.7347636E-2,-6.885414E-3,7.664771E-3,1.2261668E-3,4.2744943E-1,2.1214108E-1,3.9057012E-3,1.0323624E-2,1.3112285E-3,3.2573983E-1,5.5834115E-1,8.002984E-3,-1.814009E-2,-8.166824E-3,-2.0136353E-1,-1.7184913E-2,7.246878E-4,-2.1648586E-1,-3.1306863E-2,4.3064235E-3,-9.680902E-2,2.788872E-3,-6.1936937E-2,-3.1377172E-1,3.7719805E-2,1.6785988E-1,1.073693E-1,2.5471452E-1,2.0379356E-1,6.174806E-3,5.137581E-4,6.045468E-3,3.122814E-2,-6.33057E-3,2.4400657E-1,2.176655E-2,1.4226803E-2,2.5292698E-3,1.7177637E-1,3.6794648E-1,4.9142686E-1,3.2846943E-2,-2.3944591E-1,4.5805506E-4,-1.2775961E-2,-5.629924E-3,-3.0526544E-3,7.704906E-4,1.2867505E-3,-1.3116398E-1,4.4401463E-2,-6.0443643E-2,-6.868443E-3,9.546565E-4,-4.096277E-1,-6.098741E-3,6.5622004E-3,-1.930035E-3,3.8364648E-3,1.1193912E-2,2.1227369E-1,8.599105E-2,1.2040161E-1,3.1434587E-1,1.3900035E-2,3.0470886E-3,-7.5299838E-3,2.3498027E-2,4.6168263E-3,1.1008066E-3,6.1910963E-3,1.4100073E-2,9.821588E-3,2.20022E-3,3.9764947E-1,8.01268E-3,2.4961524E-2,1.704951E-2,-1.1986963E-2,-6.5753767E-3,1.9012865E-3,-6.5830667E-3,-1.111884E-2,-5.067491E-3,6.215874E-4,6.2839515E-3,-4.8681768E-3,1.6382809E-3,-2.540426E-2,-1.0370468E-2,2.1250919E-3,-6.3480353E-3,1.525366E-2,4.5152805E-3,-1.952035E-3,4.439086E-3,8.087964E-3,1.9244282E-3,1.7417217E-2,6.795048E-3,2.045517E-3,-4.0745554E-3,-2.1880982E-3,1.471022E-3,2.2327043E-2,1.1705415E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,-1,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,-1,79,-1,81,-1,-1,-1,83,85,-1,-1,-1,-1,87,89,-1,-1,91,93,95,97,-1,-1,99,101,103,105,-1,107,109,-1,111,113,115,117,-1,-1,119,-1,-1,121,123,125,-1,-1,-1,-1,-1,-1,127,129,131,-1,133,-1,-1,-1,135,-1,-1,137,-1,-1,-1,139,141,143,-1,-1,-1,145,147,-1,-1,-1,149,-1,-1,151,153,-1,155,157,159,161,163,165,167,169,171,173,-1,-1,175,-1,177,-1,-1,-1,179,181,183,-1,185,-1,-1,-1,-1,-1,187,189,191,193,-1,-1,195,-1,-1,197,-1,-1,199,201,203,205,-1,-1,-1,207,-1,209,-1,-1,-1,-1,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0828514E1,1.4663904E1,7.183359E0,2.0843143E0,4.058572E0,2.7714157E-1,3.27672E0,5.4176044E-1,1.9183521E0,1.1459987E0,1.2868767E0,0E0,0E0,1.8364182E0,2.3041878E0,5.073824E-1,3.604071E-1,9.7429657E-1,2.2498646E0,6.2499523E-2,9.7952265E-1,8.143754E-1,5.716131E-1,2.5196993E-1,7.9795694E-1,0E0,1.4972763E0,1.0349655E-1,0E0,2.3439042E-1,0E0,4.5291328E-1,7.342825E-1,6.585169E-2,7.074363E-1,7.646945E-2,1.054554E-1,6.8180233E-1,1.0726861E0,7.4118257E-1,0E0,4.1067266E-1,3.2755244E-1,0E0,1.5934475E-1,3.971858E-1,0E0,1.256609E0,0E0,2.9431343E-2,0E0,0E0,0E0,1.4003944E-1,6.9400346E-1,0E0,0E0,0E0,0E0,2.301676E-1,1.2124705E-1,0E0,0E0,6.614208E-2,5.6786835E-2,6.542363E-1,2.842557E-1,0E0,0E0,3.251629E-1,5.5583E-2,3.7213796E-1,1.1256564E-1,0E0,5.1782787E-2,1.4948344E-1,0E0,2.1469021E-1,4.5086473E-2,3.9219618E-1,6.4485455E-1,0E0,0E0,6.642771E-2,0E0,0E0,1.6390634E-1,2.6729858E-1,5.7893336E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.7560678E-1,3.4134865E-1,1.6741857E-1,0E0,3.6521983E-1,0E0,0E0,0E0,3.1259516E-1,0E0,0E0,3.2674477E-2,0E0,0E0,0E0,1.16272494E-1,1.7361403E-1,1.3393152E-1,0E0,0E0,0E0,3.3319092E-1,1.6131306E-1,0E0,0E0,0E0,2.678914E-1,0E0,0E0,5.722308E-2,1.7785849E-2,0E0,2.7314502E-1,1.9213514E-1,7.471498E-2,2.2469199E-1,1.2022841E-1,8.089721E-2,1.3649082E-1,1.5437448E-1,1.00586504E-1,2.2068545E-1,0E0,0E0,4.749781E-2,0E0,1.9682765E-2,0E0,0E0,0E0,6.5886766E-2,2.0257378E-1,2.7907372E-2,0E0,1.7505407E-2,0E0,0E0,0E0,0E0,0E0,1.319717E-1,1.1470795E-1,1.22922346E-1,1.286989E-1,0E0,0E0,1.3626087E-1,0E0,0E0,1.4917064E-1,0E0,0E0,1.1627385E-1,8.0442876E-2,3.145042E-2,1.1223614E-1,0E0,0E0,0E0,1.6171852E-1,0E0,2.6834093E-2,0E0,0E0,0E0,0E0,3.7173986E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,47,47,49,49,53,53,54,54,59,59,60,60,63,63,64,64,65,65,66,66,69,69,70,70,71,71,72,72,74,74,75,75,77,77,78,78,79,79,80,80,83,83,86,86,87,87,88,88,95,95,96,96,97,97,99,99,103,103,106,106,110,110,111,111,112,112,116,116,117,117,121,121,124,124,125,125,127,127,128,128,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,141,141,145,145,146,146,147,147,149,149,155,155,156,156,157,157,158,158,161,161,164,164,167,167,168,168,169,169,170,170,174,174,176,176,181,181],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,-1,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,-1,80,-1,82,-1,-1,-1,84,86,-1,-1,-1,-1,88,90,-1,-1,92,94,96,98,-1,-1,100,102,104,106,-1,108,110,-1,112,114,116,118,-1,-1,120,-1,-1,122,124,126,-1,-1,-1,-1,-1,-1,128,130,132,-1,134,-1,-1,-1,136,-1,-1,138,-1,-1,-1,140,142,144,-1,-1,-1,146,148,-1,-1,-1,150,-1,-1,152,154,-1,156,158,160,162,164,166,168,170,172,174,-1,-1,176,-1,178,-1,-1,-1,180,182,184,-1,186,-1,-1,-1,-1,-1,188,190,192,194,-1,-1,196,-1,-1,198,-1,-1,200,202,204,206,-1,-1,-1,208,-1,210,-1,-1,-1,-1,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,7.7E1,9.48E2,1.5900173E3,9.9347424E2,1.4956522E1,7.24E2,3.79E2,3.774648E0,-2.2989823E-2,-6.3940496E-3,6.7148806E5,1.059448E7,1.0195397E8,3.2856784E7,7.382199E0,2.31E2,1.4452E4,3.6E1,2.83475E5,9.639872E0,8.147158E4,2E0,3.7213E-2,7.06191E7,5.707547E0,-3.2307148E-2,6.8E1,-1.8416071E-2,2.55E2,2.49E2,1.0101351E4,2.0734E4,4.9E2,4.795239E7,2.331083E6,5.69E2,6.009825E2,-7.3555475E-3,3.1797794E1,5.144203E2,-7.3001226E-3,7.098E3,2E0,-5.881937E-4,2.8530578E6,-8.269427E-3,1E0,-1.0609695E-2,-1.1470621E-2,1.3799048E-3,1.8E1,8.414097E-1,-1.1726351E-2,-3.6128372E-2,2.182578E-2,9.083242E-3,1.3022917E8,2.9994638E2,-1.1683327E-2,-4.266479E-3,3.390625E0,3E0,2.87E2,1.8E1,-4.0173042E-4,-2.9457076E-2,1E0,7.403968E2,5.1034E4,3.093458E0,-1.2054948E-4,1.2218E4,3.3382E4,1.0630331E-2,7.7191065E6,1.8062708E6,1E0,5.489183E1,-1.6423136E-2,-2.2907563E-2,3.328629E6,-6.0390932E-3,4.752744E-3,2.1E1,3.716E3,4.080551E-1,-1.7861769E-2,-7.6578357E-3,-8.806472E-3,-1.727421E-2,-1.05719855E-2,-3.6487186E-3,6.99E2,9.935484E-1,1.4285715E0,-6.110608E-3,2.8802464E8,-4.605972E-3,1.7539717E-2,1.0079776E-2,7.9555137E3,-9.807906E-3,1.1873181E-2,1.909E3,-1.7347636E-2,-6.885414E-3,7.664771E-3,4.539777E5,4.6134964E2,1.8667632E0,3.9057012E-3,1.0323624E-2,1.3112285E-3,1.0168186E5,6.663214E6,8.002984E-3,-1.814009E-2,-8.166824E-3,1.157602E6,-1.7184913E-2,7.246878E-4,7.0093E4,1.1358E4,4.3064235E-3,1.2916666E0,9.6E1,3.57E2,8.49E2,3.5714287E-1,4.5866325E6,7.8571427E-1,5.4007E7,1.3E1,3.257507E5,5.137581E-4,6.045468E-3,1.5998265E6,-6.33057E-3,5.941442E6,2.176655E-2,1.4226803E-2,2.5292698E-3,5.5933E5,2E1,2.9366477E0,3.2846943E-2,6.763314E7,4.5805506E-4,-1.2775961E-2,-5.629924E-3,-3.0526544E-3,7.704906E-4,2.8361E4,2.590909E0,7.1E1,1.432632E6,-6.868443E-3,9.546565E-4,7.6116E4,-6.098741E-3,6.5622004E-3,5.405423E6,3.8364648E-3,1.1193912E-2,3.074398E0,3.6E1,5.5426865E6,8.051603E7,1.3900035E-2,3.0470886E-3,-7.5299838E-3,4.2E2,4.6168263E-3,2.6463525E8,6.1910963E-3,1.4100073E-2,9.821588E-3,2.20022E-3,1.9347133E5,8.01268E-3,2.4961524E-2,1.704951E-2,-1.1986963E-2,-6.5753767E-3,1.9012865E-3,-6.5830667E-3,-1.111884E-2,-5.067491E-3,6.215874E-4,6.2839515E-3,-4.8681768E-3,1.6382809E-3,-2.540426E-2,-1.0370468E-2,2.1250919E-3,-6.3480353E-3,1.525366E-2,4.5152805E-3,-1.952035E-3,4.439086E-3,8.087964E-3,1.9244282E-3,1.7417217E-2,6.795048E-3,2.045517E-3,-4.0745554E-3,-2.1880982E-3,1.471022E-3,2.2327043E-2,1.1705415E-2],"split_indices":[2,43,17,44,2,70,67,73,2,2,69,0,0,43,12,46,60,69,44,9,3,1,69,43,6,0,59,69,0,0,0,2,2,43,44,1,7,43,2,70,0,71,4,0,2,32,0,43,0,20,0,0,0,3,68,0,0,0,0,7,67,0,0,69,8,0,3,0,0,6,4,44,68,0,44,9,0,62,43,26,71,0,0,9,0,0,8,44,53,0,0,0,0,0,0,2,68,68,0,5,0,0,0,62,0,0,2,0,0,0,43,4,57,0,0,0,48,43,0,0,0,12,0,0,1,44,0,68,0,0,2,68,43,68,7,3,43,0,0,62,0,60,0,0,0,9,3,53,0,7,0,0,0,0,0,1,69,0,12,0,0,12,0,0,43,0,0,71,0,43,7,0,0,0,0,0,7,0,0,0,0,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.72E2,2.57E2,2.46E2,5.26E2,1.3E1,2.44E2,8.3E1,1.63E2,2.8E2,2.46E2,9E0,4E0,1.05E2,1.39E2,6.1E1,2.2E1,1.01E2,6.2E1,4.6E1,2.34E2,1.27E2,1.19E2,4.1E1,6.4E1,1.5E1,1.24E2,5E1,1.1E1,1.1E1,1.1E1,8.8E1,1.3E1,8E0,5.4E1,1.5E1,3.1E1,2.24E2,1E1,1.2E2,7E0,1.04E2,1.5E1,5E0,3.6E1,5.7E1,7E0,1.2E2,4E0,4.4E1,6E0,6E0,5E0,4.1E1,4.7E1,6E0,7E0,4E0,4E0,3.6E1,1.8E1,5E0,1E1,1.8E1,1.3E1,1.75E2,4.9E1,5E0,5E0,9.6E1,2.4E1,8.7E1,1.7E1,7E0,8E0,3.2E1,4E0,4.4E1,1.3E1,6.5E1,5.5E1,1.9E1,2.5E1,3.7E1,4E0,6E0,4.1E1,2.2E1,1.4E1,1.2E1,6E0,7E0,1.1E1,7E0,6E0,1.51E2,2.4E1,4.3E1,6E0,9.1E1,5E0,1.5E1,9E0,8.1E1,6E0,8E0,9E0,4E0,4E0,6E0,2.6E1,3.5E1,9E0,6E0,7E0,5E0,6E1,4.9E1,6E0,3.3E1,4E0,3.2E1,9E0,7E0,1.5E1,9E0,5E0,8E1,7.1E1,9E0,1.5E1,2.8E1,1.5E1,6.8E1,2.3E1,8E0,7.3E1,4E0,5E0,2.2E1,4E0,8E0,2.7E1,5E0,4E0,1.4E1,4.6E1,3.7E1,1.2E1,2.7E1,5E0,8E0,7E0,5E0,4E0,2.1E1,5.9E1,4.3E1,2.8E1,4E0,5E0,9E0,6E0,7E0,2.1E1,8E0,7E0,1E1,5.8E1,8E0,1.5E1,4E0,4E0,6E0,6.7E1,6E0,1.6E1,4E0,4E0,1E1,4E0,3.9E1,7E0,2.4E1,1.3E1,2.1E1,6E0,1.7E1,4E0,8E0,5.1E1,3.3E1,1E1,1.9E1,9E0,4E0,5E0,1.6E1,5E0,4E0,6E0,4E0,5.4E1,4E0,4E0,1E1,5E0,5.7E1,1E1,6E0,1E1,2.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[1.4223594E-2,-8.634216E-2,3.0193153E-1,-2.7222386E-1,-6.765204E-3,7.940106E-1,2.5887245E-1,-4.040506E-1,-1.9873267E-1,-1.9419628E-1,3.405146E-2,4.0034797E-2,1.7948333E-2,-3.1415752E-1,2.9465175E-1,-3.5044417E-1,-5.3188396E-1,-2.8547212E-1,-4.5296635E-2,-1.10069335E-1,-2.6626772E-1,-2.4604792E-2,8.945693E-2,-2.5621648E-3,-1.8626895E-2,2.5246102E-1,5.9453166E-1,-3.798945E-1,-3.3814332E-3,-3.2094732E-2,-3.3830726E-1,-2.3854767E-1,-4.3946558E-1,2.438176E-2,-1.398615E-1,-3.6346477E-2,-1.7054267E-1,-1.8699847E-1,-3.5463905E-1,-1.1870257E-1,3.8861327E-3,1.0711176E-1,-1.8679948E-1,7.600477E-2,3.0271718E-1,1.3814948E-2,6.457696E-1,-1.8644042E-2,-2.1673195E-1,-7.077512E-3,-1.9020766E-2,-3.0667937E-1,-1.5621948E-1,-5.764379E-1,-2.6254186E-1,1.785968E-4,-1.918815E-1,-6.8915084E-2,2.8638325E-3,-2.1919231E-1,-1.8370199E-3,-1.3457746E-2,-1.2549819E-1,-2.8971314E-1,-2.4505343E-2,5.7134286E-2,-1.7573352E-1,-1.1868533E-2,2.2696266E-1,5.4524697E-2,1.6540694E-1,-8.068745E-4,-2.527867E-1,1.9602962E-2,1.4412609E-2,3.270215E-1,-1.413914E-1,3.3096164E-2,1.5074941E-2,-4.93375E-3,-1.3089308E-2,-2.1446358E-2,-2.6522526E-1,6.500218E-3,-1.9773133E-1,-1.43390065E-2,-3.1606384E-2,-1.4578584E-2,-6.2741884E-3,-4.3213803E-2,3.0139075E-3,-2.7602658E-1,-1.0649476E-1,-2.5482537E-4,-4.758274E-3,-4.357136E-3,-1.2010289E-2,-3.8744456E-3,-1.0793365E-2,-1.4583387E-2,-7.110835E-3,1.5007585E-4,4.751945E-3,-2.5428542E-1,-5.1831815E-2,3.7173957E-2,-7.588196E-2,5.351729E-3,1.3871613E-2,3.405184E-2,2.109637E-1,1.7860527E-1,-3.3879743E-3,-1.4970619E-2,-5.3922664E-3,7.656629E-2,-1.002134E-1,6.5897727E-1,2.9645064E-1,-1.0526263E-2,-1.2316033E-3,-1.7388396E-1,-1.418976E-2,-2.4303529E-1,-8.914783E-2,2.676173E-4,-3.8612033E-3,-1.4714607E-2,-6.7019206E-3,1.4255417E-3,-1.7226124E-1,-2.7240536E-3,-3.360536E-1,2.2319464E-2,-8.275968E-3,5.644833E-2,-1.0463053E-1,-2.444161E-2,-1.7546345E-1,1.2201927E-1,-1.5779171E-2,1.6101167E-2,5.1329285E-3,5.2408434E-2,2.0906413E-1,-1.5491253E-2,1.2345568E-1,-9.0388E-3,-7.442658E-4,1.6963217E-2,3.7650026E-2,3.2575783E-1,1.3192868E-1,-3.6361762E-3,-9.58352E-3,-8.411567E-3,-1.510531E-2,-9.4763655E-4,-6.603029E-3,-9.883966E-3,-4.064374E-3,-2.4326963E-2,-2.0158608E-1,3.7598812E-3,-1.8694615E-3,2.863704E-2,1.4276624E-1,-8.576235E-4,-9.433755E-3,-7.633371E-2,8.7890975E-2,-1.5007787E-3,-2.0385443E-1,6.7740284E-2,1.0025926E-2,-9.127981E-3,5.693843E-3,1.0643449E-1,-1.175741E-3,2.3600961E-1,2.5935834E-2,2.5258532E-3,-4.4623865E-3,1.1203726E-2,7.2118066E-2,-2.6310363E-3,3.3956462E-1,-1.8976894E-3,1.8226767E-1,-1.234629E-2,-3.2039825E-3,1.9455624E-3,-6.038761E-3,4.492031E-3,8.926657E-3,-6.647862E-3,-3.8139103E-4,9.4821565E-3,1.2269466E-3,-1.0759202E-2,-4.718485E-3,5.160075E-3,1.1655367E-3,5.1182546E-3,-1.3258873E-3,7.995434E-3,5.0633756E-4,1.1686313E-2,-1.1417135E-3,4.007029E-3,-9.792607E-4,1.2326427E-4,5.1006023E-3,9.331165E-3,1.7439645E-2,1.1267652E-2,5.0161807E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,-1,77,-1,79,-1,-1,81,83,85,87,89,91,93,-1,95,-1,-1,97,99,-1,101,103,105,107,109,111,-1,113,115,-1,117,119,-1,-1,-1,-1,-1,121,-1,123,-1,-1,-1,-1,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,131,133,135,137,-1,-1,139,141,143,-1,-1,-1,145,147,149,151,-1,-1,153,-1,155,157,-1,-1,-1,-1,-1,159,-1,161,163,-1,165,167,169,171,173,175,-1,-1,177,179,181,183,-1,-1,-1,-1,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,189,-1,-1,191,193,-1,-1,195,197,-1,199,201,-1,-1,203,205,-1,207,209,-1,-1,-1,211,-1,213,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9827642E1,1.1299171E1,5.4517746E0,2.138277E0,4.107179E0,2.812481E-1,5.1567135E0,3.6867237E-1,1.9701042E0,5.527544E-1,1.435397E0,0E0,0E0,3.2311928E-1,2.7978554E0,4.621029E-1,4.5838213E-1,5.816426E-1,3.0542905E0,1.9888163E-1,2.9094434E-1,5.7686174E-1,1.1271461E0,0E0,0E0,1.8038015E0,2.3083878E-1,1.2103939E-1,0E0,0E0,9.998667E-2,3.7277794E-1,3.575673E-1,0E0,3.4970886E-1,7.473025E-2,1.5034372E-1,1.4946353E-1,1.0883498E-1,5.114477E-1,5.861001E-1,6.4701366E-1,1.5201959E-1,6.193949E-1,1.773242E0,0E0,2.8507996E-1,0E0,3.5632372E-2,0E0,0E0,1.3810015E-1,4.7154558E-1,1.0285854E-1,3.1056046E-2,4.2341985E-2,2.220751E-1,3.6162883E-2,0E0,6.7139804E-2,0E0,0E0,7.044408E-2,1.8370748E-2,0E0,3.1696096E-2,3.56663E-1,4.9256375E-1,5.4809332E-2,3.5793018E-1,3.3134794E-1,0E0,5.913955E-2,2.7385336E-1,0E0,1.347168E0,8.319841E-2,0E0,0E0,0E0,0E0,0E0,7.677555E-2,0E0,1.357503E-1,0E0,0E0,0E0,0E0,1.8286915E-2,0E0,4.8365593E-2,1.7694445E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.4752095E-1,1.5503596E-1,2.4916035E-1,3.4404194E-1,0E0,0E0,4.4904298E-1,1.3984573E-1,3.5804892E-1,0E0,0E0,0E0,1.184586E-1,1.0119216E-1,2.469325E-1,6.368799E-1,0E0,0E0,2.5191784E-2,0E0,5.3030014E-2,3.7670232E-2,0E0,0E0,0E0,0E0,0E0,2.6849806E-2,0E0,3.037399E-1,4.408544E-2,0E0,1.8555996E-1,9.0473436E-2,2.7485105E-1,8.674151E-2,1.7608303E-1,2.5877574E-1,0E0,0E0,8.68358E-2,3.738339E-1,6.2183775E-2,9.25259E-2,0E0,0E0,0E0,0E0,6.4812946E-1,2.077915E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.696691E-2,0E0,0E0,1.364909E-1,2.4864078E-2,0E0,0E0,1.4599709E-1,1.0251513E-1,0E0,3.2327116E-2,4.389397E-2,0E0,0E0,2.2004953E-1,7.444134E-2,0E0,3.0227232E-1,3.3359654E-2,0E0,0E0,0E0,3.693171E-2,0E0,5.2381897E-1,0E0,5.526066E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,82,82,84,84,89,89,91,91,92,92,103,103,104,104,105,105,106,106,109,109,110,110,111,111,115,115,116,116,117,117,118,118,121,121,123,123,124,124,130,130,132,132,133,133,135,135,136,136,137,137,138,138,139,139,140,140,143,143,144,144,145,145,146,146,151,151,152,152,162,162,165,165,166,166,169,169,170,170,172,172,173,173,176,176,177,177,179,179,180,180,184,184,186,186,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,-1,78,-1,80,-1,-1,82,84,86,88,90,92,94,-1,96,-1,-1,98,100,-1,102,104,106,108,110,112,-1,114,116,-1,118,120,-1,-1,-1,-1,-1,122,-1,124,-1,-1,-1,-1,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,132,134,136,138,-1,-1,140,142,144,-1,-1,-1,146,148,150,152,-1,-1,154,-1,156,158,-1,-1,-1,-1,-1,160,-1,162,164,-1,166,168,170,172,174,176,-1,-1,178,180,182,184,-1,-1,-1,-1,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,190,-1,-1,192,194,-1,-1,196,198,-1,200,202,-1,-1,204,206,-1,208,210,-1,-1,-1,212,-1,214,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.6E1,7.7E1,4.97E2,6.7321223E-1,6.829817E4,1.4504054E8,7.24E2,1.6923077E0,1.037E3,4.0034797E-2,1.7948333E-2,1.0322473E8,6.663214E6,4.56E2,1.4956522E1,1.7E1,2.04115E5,8.510249E-3,4.0753E4,1E0,9.639872E0,-2.5621648E-3,-1.8626895E-2,7.075161E2,3.53562E5,3.1530054E0,-3.3814332E-3,-3.2094732E-2,7.042942E7,2.55E2,6.593462E4,2.438176E-2,1.5522388E0,4.49239E6,6.769866E7,1.43E2,7.668863E6,5E-1,2.0200005E8,1.5115256E6,5.737478E7,6.103178E8,7.134432E7,1.3814948E-2,1.4379113E9,-1.8644042E-2,4.233978E0,-7.077512E-3,-1.9020766E-2,3.9E1,1E0,1.6E1,5E0,1.1E3,9.458613E3,2.0409561E6,2.8638325E-3,1.188E0,-1.8370199E-3,-1.3457746E-2,1.31625E2,4.48E2,-2.4505343E-2,7.52E2,8.1E2,1E0,1.6923077E0,3.299862E1,3.1027884E5,-8.068745E-4,3.38E2,1.9347133E5,1.4412609E-2,1.2083E4,9.4578236E-1,3.3096164E-2,1.5074941E-2,-4.93375E-3,-1.3089308E-2,-2.1446358E-2,1.4E1,6.500218E-3,8E0,-1.43390065E-2,-3.1606384E-2,-1.4578584E-2,-6.2741884E-3,2.02E2,3.0139075E-3,1E0,1.2038125E7,-2.5482537E-4,-4.758274E-3,-4.357136E-3,-1.2010289E-2,-3.8744456E-3,-1.0793365E-2,-1.4583387E-2,-7.110835E-3,1.5007585E-4,4.751945E-3,6.07E2,1.5E1,1E0,2E0,5.351729E-3,1.3871613E-2,1.4915254E0,3.13E2,3.7788504E7,-3.3879743E-3,-1.4970619E-2,-5.3922664E-3,3.0990322E5,1.4521204E0,2.8016653E3,1.7172844E7,-1.0526263E-2,-1.2316033E-3,1.00019E5,-1.418976E-2,5.80563E5,2.3266666E0,2.676173E-4,-3.8612033E-3,-1.4714607E-2,-6.7019206E-3,1.4255417E-3,2.58466E5,-2.7240536E-3,2.019537E2,1E1,-8.275968E-3,7.919006E7,6.763314E7,8.7E1,1.1356604E2,8.684966E-1,4.186E4,1.6101167E-2,5.1329285E-3,1.103E3,2.23099E7,3.921E3,5.7777777E0,-9.0388E-3,-7.442658E-4,1.6963217E-2,3.7650026E-2,3.167E3,1.465E4,-3.6361762E-3,-9.58352E-3,-8.411567E-3,-1.510531E-2,-9.4763655E-4,-6.603029E-3,-9.883966E-3,-4.064374E-3,-2.4326963E-2,1.4386049E6,3.7598812E-3,-1.8694615E-3,9.07826E0,2.28E0,-8.576235E-4,-9.433755E-3,2.118835E6,2E0,-1.5007787E-3,5.1042255E2,9.6E1,1.0025926E-2,-9.127981E-3,1.0439024E1,3.412336E7,-1.175741E-3,5.2409735E-7,2.7777777E0,2.5258532E-3,-4.4623865E-3,1.1203726E-2,8.11E2,-2.6310363E-3,8.747289E5,-1.8976894E-3,6.072304E5,-1.234629E-2,-3.2039825E-3,1.9455624E-3,-6.038761E-3,4.492031E-3,8.926657E-3,-6.647862E-3,-3.8139103E-4,9.4821565E-3,1.2269466E-3,-1.0759202E-2,-4.718485E-3,5.160075E-3,1.1655367E-3,5.1182546E-3,-1.3258873E-3,7.995434E-3,5.0633756E-4,1.1686313E-2,-1.1417135E-3,4.007029E-3,-9.792607E-4,1.2326427E-4,5.1006023E-3,9.331165E-3,1.7439645E-2,1.1267652E-2,5.0161807E-3],"split_indices":[2,43,0,44,2,53,43,46,2,68,2,0,0,5,43,10,73,8,5,53,12,29,69,0,0,67,1,68,0,0,60,2,43,0,68,43,7,2,5,68,5,43,5,7,59,0,12,0,68,0,0,2,21,3,69,2,48,60,0,68,0,0,67,2,0,12,2,26,68,71,48,0,0,48,0,9,49,0,0,0,0,0,3,0,8,0,0,0,0,10,0,27,12,0,0,0,0,0,0,0,0,0,0,2,3,19,32,0,0,68,0,7,0,0,0,43,53,4,60,0,0,5,0,12,68,0,0,0,0,0,1,0,4,3,0,7,7,0,67,57,9,0,0,10,9,2,71,0,0,0,0,2,44,0,0,0,0,0,0,0,0,0,43,0,0,71,71,0,0,9,8,0,4,0,0,0,73,7,0,52,69,0,0,0,0,0,43,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.029E3,7.63E2,2.66E2,2.28E2,5.35E2,2E1,2.46E2,8E1,1.48E2,9.5E1,4.4E2,1.6E1,4E0,1.4E1,2.32E2,5.9E1,2.1E1,9.4E1,5.4E1,4.5E1,5E1,2.14E2,2.26E2,4E0,1E1,2.05E2,2.7E1,5.3E1,6E0,1E1,1.1E1,7.4E1,2E1,7E0,4.7E1,2.1E1,2.4E1,2.8E1,2.2E1,4.9E1,1.65E2,2.13E2,1.3E1,4.6E1,1.59E2,5E0,2.2E1,4.5E1,8E0,4E0,7E0,3.9E1,3.5E1,1E1,1E1,1.3E1,3.4E1,1.6E1,5E0,1.7E1,7E0,9E0,1.9E1,1.8E1,4E0,1.2E1,3.7E1,1.55E2,1E1,1.13E2,1E2,4E0,9E0,3.8E1,8E0,1.51E2,8E0,1.7E1,5E0,4E0,4E0,6E0,3.3E1,4E0,3.1E1,4E0,6E0,6E0,4E0,8E0,5E0,1.6E1,1.8E1,6E0,1E1,5E0,1.2E1,1.5E1,4E0,1.4E1,4E0,6E0,6E0,2.2E1,1.5E1,8.8E1,6.7E1,5E0,5E0,1.01E2,1.2E1,9.5E1,5E0,5E0,4E0,2.6E1,1.2E1,1.1E1,1.4E2,4E0,4E0,1.2E1,2.1E1,2.1E1,1E1,4E0,4E0,1.1E1,5E0,6E0,1.2E1,7E0,1.5E1,1E1,5E0,7.8E1,1E1,4.5E1,2.2E1,3.6E1,6.5E1,4E0,8E0,1.9E1,7.6E1,9E0,1.7E1,5E0,7E0,5E0,6E0,1.18E2,2.2E1,4E0,8E0,1.4E1,7E0,5E0,5E0,7E0,5E0,5E0,1E1,5E0,5E0,6E1,1.8E1,6E0,4E0,3.1E1,1.4E1,4E0,1.8E1,2.4E1,1.2E1,6E0,5.9E1,1.1E1,8E0,6.6E1,1E1,5E0,4E0,4E0,1.3E1,4E0,1.14E2,5E0,1.7E1,6E0,4E0,5.6E1,4E0,1.1E1,7E0,1.5E1,1.6E1,4E0,1E1,1.3E1,5E0,1.1E1,1.3E1,1.4E1,4.5E1,6E0,5E0,6.2E1,4E0,4E0,6E0,5E0,8E0,2.6E1,8.8E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[1.1543609E-2,-8.338619E-2,3.051977E-1,-2.646278E-1,-3.4403836E-3,-3.7307516E-1,3.4792015E-1,-4.4438487E-1,-2.0215484E-1,-9.747073E-2,6.0152967E-2,-2.2339892E-2,-9.977077E-3,2.5551268E-1,5.1589066E-1,-3.5952973E-1,-6.9145286E-1,-2.3411669E-1,1.2810075E-1,-5.7790004E-2,-2.326693E-1,3.556915E-2,2.69155E-1,1.2523043E-1,3.551997E-1,6.626197E-1,2.2994138E-1,-3.8405544E-1,-7.7624316E-3,-1.7001208E-2,-3.622165E-2,-4.2762083E-1,-1.9992173E-1,2.8149905E-2,-1.4602683E-1,-8.362085E-2,9.852671E-2,-3.1050396E-1,-2.1463407E-2,1.14119865E-1,-7.978343E-3,1.4414949E-3,3.1053227E-1,2.2622343E-2,9.431877E-2,2.8278008E-1,4.7222808E-1,7.161346E-1,9.639213E-3,3.1366363E-1,-3.6032856E-3,-9.934622E-3,-4.0205657E-1,-8.603879E-3,-2.3185492E-2,-4.4929963E-2,-2.2759324E-1,3.441804E-4,-1.27123995E-2,-5.3285617E-2,-1.8312359E-1,6.300127E-2,1.02064395E-2,-1.6993105E-1,-4.6681356E-1,5.875403E-3,-9.363669E-2,1.3160466E-1,-1.1453167E-2,6.3196644E-3,-2.851087E-1,3.6019534E-1,5.0858995E-3,-2.9385116E-2,1.7119771E-1,2.0045193E-1,3.356893E-1,1.2639584E-2,2.4458908E-2,7.5985986E-1,1.8848816E-2,2.0518926E-1,2.1233212E-2,-2.591244E-1,-2.0399967E-2,-2.0485388E-1,2.0883279E-1,-2.8055418E-1,-1.5207374E-1,-1.4177686E-1,-2.4549743E-2,-1.0495798E-1,-2.6992512E-1,1.15091E-1,-1.3637824E-3,-1.19288955E-2,-2.4217737E-4,-3.022225E-1,-3.5502557E-2,-7.2634085E-3,6.42378E-5,6.72919E-2,2.542933E-1,-3.192825E-2,8.0160305E-2,-2.6347525E-3,-2.1074397E-2,3.8972566E-1,8.401162E-3,8.1129834E-2,-1.5587047E-1,1.6458493E-2,1.1194983E-1,2.489145E-1,-1.547148E-3,3.5823494E-1,6.5739336E-3,3.7956085E-2,2.4137238E-2,2.5682065E-1,2.335932E-3,-1.4580121E-2,-6.446454E-3,-1.4555292E-2,-3.8262142E-3,7.306444E-5,1.7293112E-2,-3.2006103E-1,-1.8519649E-1,-1.9031139E-1,4.326568E-2,-1.7694527E-1,2.2338717E-3,1.4073353E-2,-5.9194125E-2,-1.6171026E-1,7.129953E-4,-1.5398336E-3,-1.6305007E-2,7.289084E-3,2.585521E-3,-1.6314238E-2,-7.40696E-3,9.140162E-2,-1.1101867E-2,2.8632256E-1,8.160982E-4,-6.1204627E-2,5.801139E-2,-4.7625955E-3,1.6524196E-1,2.0019598E-2,8.762965E-3,7.8591466E-4,7.566744E-3,-1.0268122E-2,-2.4226597E-3,6.4916926E-4,1.39918E-1,6.4655896E-3,1.5030578E-2,1.7991068E-2,8.268109E-3,1.4572807E-2,5.700786E-3,-2.4285667E-1,-1.8167546E-2,-9.113013E-2,-1.5634745E-2,-2.2198202E-1,-1.1111426E-1,-1.84157E-3,5.43947E-3,-1.2372878E-2,-9.939814E-2,-2.0517794E-2,6.940438E-2,-9.404668E-2,2.4697002E-2,-3.5157346E-3,-8.8380575E-3,1.3244663E-1,6.370217E-4,3.975059E-3,3.1492576E-1,2.7158529E-2,-1.0737796E-1,1.1556205E-1,-4.6668056E-2,2.1825334E-2,-6.8090423E-3,1.936915E-1,-7.2736363E-4,1.5811813E-1,2.7241095E-3,-3.5474019E-3,-1.2960464E-2,-9.305543E-4,-8.974389E-3,-6.4456435E-3,-1.2947898E-2,-2.2215578E-3,-6.504209E-3,-5.7108663E-3,-1.4361461E-3,3.832691E-3,-2.9108948E-3,9.4427145E-4,5.979993E-3,-6.3294666E-3,-2.2214286E-3,2.6789808E-3,-2.1632232E-3,1.1095829E-2,4.425329E-3,-2.0961347E-3,1.9590857E-3,8.213827E-3,1.655673E-2,4.260382E-3,-3.8593335E-3,-7.925799E-3,-2.474342E-3,6.598231E-4,6.542611E-3,-5.9609422E-3,3.6020759E-3,-3.8036013E-3,2.5642654E-3,1.0251868E-2,1.4232753E-3,3.6959965E-3,8.329804E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,-1,73,75,77,79,-1,81,-1,-1,83,-1,-1,85,87,-1,-1,89,91,93,-1,95,97,-1,99,101,-1,103,105,107,-1,109,111,113,115,-1,-1,117,-1,119,-1,121,-1,123,125,127,129,131,133,135,137,139,-1,-1,-1,141,-1,-1,-1,143,145,147,149,-1,-1,151,-1,153,155,-1,157,159,-1,161,-1,-1,-1,163,-1,-1,-1,-1,-1,-1,-1,165,167,169,171,173,-1,175,177,179,-1,-1,-1,-1,-1,-1,-1,181,-1,183,-1,185,187,189,191,-1,-1,-1,-1,-1,-1,-1,193,-1,-1,-1,-1,-1,-1,195,-1,197,-1,199,201,-1,-1,-1,203,205,207,209,211,-1,-1,213,215,-1,217,219,221,223,225,227,-1,229,-1,231,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7957724E1,1.0982136E1,7.229288E0,2.5155144E0,3.1571677E0,1.5595078E-1,3.4537048E0,9.8833656E-1,1.8666615E0,1.1292517E0,1.6089178E0,0E0,0E0,1.9166231E0,3.2461567E0,1.5152693E-1,1.3071775E-1,9.6659184E-1,2.1996148E0,6.785125E-1,7.8241396E-1,9.689647E-1,3.1522393E-1,7.08601E-1,5.868511E-1,1.1381836E0,7.721162E-1,3.5241127E-2,0E0,0E0,0E0,3.1870747E-1,5.7734776E-1,0E0,2.2006813E-1,4.2187893E-1,9.043807E-2,6.897249E-1,1.6209728E-1,6.719564E-1,7.269231E-1,0E0,2.3907995E-1,0E0,6.0417473E-1,1.7228603E-1,2.0735884E-1,3.23246E-1,0E0,2.6979947E-1,0E0,0E0,8.323383E-2,0E0,0E0,9.356003E-1,4.2128706E-1,0E0,0E0,2.7877834E-1,1.955707E-1,9.9785306E-2,0E0,2.8940648E-1,5.621488E-1,0E0,6.0639888E-2,7.482611E-1,0E0,4.9680316E-1,3.2511312E-1,4.499674E-2,0E0,3.6210883E-1,3.8771605E-1,2.8043252E-1,8.855057E-2,0E0,0E0,1.568203E-1,0E0,1.0676861E-1,0E0,2.3493469E-2,0E0,1.621843E-1,3.080803E-1,1.9313383E-1,3.8169992E-1,1.8909329E-1,1.1456929E-1,1.3231261E-1,2.828126E-1,2.3979247E-2,0E0,0E0,0E0,3.006506E-2,0E0,0E0,0E0,5.0400937E-1,2.4505019E-1,3.0899096E-1,4.343098E-1,0E0,0E0,7.4831486E-2,0E0,7.141037E-2,7.074347E-2,0E0,8.291304E-2,1.2102866E-1,0E0,8.686447E-2,0E0,0E0,0E0,4.7103286E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6440296E-1,2.8625238E-1,8.068001E-2,6.0410865E-2,1.3545632E-1,0E0,7.989654E-2,1.3389459E-1,1.8377662E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2716707E-1,0E0,1.3903475E-1,0E0,3.619581E-1,1.8009982E-1,1.2076971E-1,1.5751868E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6037365E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.3489258E-1,0E0,1.0589628E-1,0E0,1.0027349E-1,1.9412532E-2,0E0,0E0,0E0,1.9316971E-2,1.1954331E-1,4.438863E-2,5.659473E-2,3.6049075E-2,0E0,0E0,1.4285707E-1,4.035822E-2,0E0,8.525276E-2,2.3095919E-1,1.9115108E-1,4.811704E-2,1.2297499E-1,9.9283375E-2,0E0,1.12653494E-1,0E0,1.8501848E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,49,49,52,52,55,55,56,56,59,59,60,60,61,61,63,63,64,64,66,66,67,67,69,69,70,70,71,71,73,73,74,74,75,75,76,76,79,79,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,97,97,101,101,102,102,103,103,104,104,107,107,109,109,110,110,112,112,113,113,115,115,119,119,127,127,128,128,129,129,130,130,131,131,133,133,134,134,135,135,143,143,145,145,147,147,148,148,149,149,150,150,158,158,165,165,167,167,169,169,170,170,174,174,175,175,176,176,177,177,178,178,181,181,182,182,184,184,185,185,186,186,187,187,188,188,189,189,191,191,193,193],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,-1,74,76,78,80,-1,82,-1,-1,84,-1,-1,86,88,-1,-1,90,92,94,-1,96,98,-1,100,102,-1,104,106,108,-1,110,112,114,116,-1,-1,118,-1,120,-1,122,-1,124,126,128,130,132,134,136,138,140,-1,-1,-1,142,-1,-1,-1,144,146,148,150,-1,-1,152,-1,154,156,-1,158,160,-1,162,-1,-1,-1,164,-1,-1,-1,-1,-1,-1,-1,166,168,170,172,174,-1,176,178,180,-1,-1,-1,-1,-1,-1,-1,182,-1,184,-1,186,188,190,192,-1,-1,-1,-1,-1,-1,-1,194,-1,-1,-1,-1,-1,-1,196,-1,198,-1,200,202,-1,-1,-1,204,206,208,210,212,-1,-1,214,216,-1,218,220,222,224,226,228,-1,230,-1,232,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,1E0,3.3817584E7,8.1E2,9.3326636E-2,1.6975454E3,2.2E1,2.41E3,4.5709968E0,3.7951445E6,-2.2339892E-2,-9.977077E-3,1.6802971E6,1.2792593E2,2.46875E0,4.869229E7,1.0479E4,2.3E1,4.1973075E6,2.6801266E2,1.5132743E0,3.3875784E7,5E1,4.391553E6,4.262058E0,6.747343E8,2E0,-7.7624316E-3,-1.7001208E-2,-3.622165E-2,4E0,9.935484E-1,2.8149905E-2,2E1,3.6245735E0,1.6341463E0,2.6742857E0,3.1316666E2,1.4676277E7,1.2294118E1,1.4414949E-3,8.931E3,2.2622343E-2,8.190129E9,1.0089981E3,1.7068776E10,8.913649E-3,9.639213E-3,3.8771296E3,-3.6032856E-3,-9.934622E-3,1.863E3,-8.603879E-3,-2.3185492E-2,1.8236995E0,1.559733E6,3.441804E-4,-1.27123995E-2,1.3920766E2,9.6E1,6.34E2,1.02064395E-2,8.39777E5,1.1742609E7,5.875403E-3,7.09E2,4.797342E2,-1.1453167E-2,1.4603988E6,4E1,1.8E1,5.0858995E-3,1.2E1,1.3864E4,9.7944025E2,1E0,1.2639584E-2,2.4458908E-2,1.557718E7,1.8848816E-2,5.501002E9,2.1233212E-2,8.209663E3,-2.0399967E-2,8.579633E-2,4.13E2,3.9682608E2,6.4522544E5,2.331083E6,1.4007937E0,3.0948E2,1.45064E5,7.318515E6,-1.3637824E-3,-1.19288955E-2,-2.4217737E-4,1.7E1,-3.5502557E-2,-7.2634085E-3,6.42378E-5,2.511E3,2.0319266E-7,9.751103E6,6.763312E7,-2.6347525E-3,-2.1074397E-2,1E0,8.401162E-3,3.1720988E6,2.0312773E8,1.6458493E-2,8.620714E2,1.2544625E5,-1.547148E-3,1.8039524E7,6.5739336E-3,3.7956085E-2,2.4137238E-2,5.6516E4,2.335932E-3,-1.4580121E-2,-6.446454E-3,-1.4555292E-2,-3.8262142E-3,7.306444E-5,1.7293112E-2,3.962617E0,8.9535345E2,1.5152774E2,9.593451E4,2.6818182E0,2.2338717E-3,2.732381E2,2.347364E7,2.227328E4,7.129953E-4,-1.5398336E-3,-1.6305007E-2,7.289084E-3,2.585521E-3,-1.6314238E-2,-7.40696E-3,5.2767E4,-1.1101867E-2,3.292496E-2,8.160982E-4,2.67637E5,4.3E1,2.1651245E6,2.03353E-6,2.0019598E-2,8.762965E-3,7.8591466E-4,7.566744E-3,-1.0268122E-2,-2.4226597E-3,6.4916926E-4,1.2792593E2,6.4655896E-3,1.5030578E-2,1.7991068E-2,8.268109E-3,1.4572807E-2,5.700786E-3,2.90099E0,-1.8167546E-2,6.185E3,-1.5634745E-2,1.1967312E3,1.3E1,-1.84157E-3,5.43947E-3,-1.2372878E-2,2.707673E6,1E1,2.6818182E0,1.084437E6,1E0,-3.5157346E-3,-8.8380575E-3,7.8571427E-1,1.047E3,3.975059E-3,5.281628E2,7.755551E7,1.244E3,3.0536E4,1.7022566E7,1.0738284E4,-6.8090423E-3,1.7022608E7,-7.2736363E-4,1.4E1,2.7241095E-3,-3.5474019E-3,-1.2960464E-2,-9.305543E-4,-8.974389E-3,-6.4456435E-3,-1.2947898E-2,-2.2215578E-3,-6.504209E-3,-5.7108663E-3,-1.4361461E-3,3.832691E-3,-2.9108948E-3,9.4427145E-4,5.979993E-3,-6.3294666E-3,-2.2214286E-3,2.6789808E-3,-2.1632232E-3,1.1095829E-2,4.425329E-3,-2.0961347E-3,1.9590857E-3,8.213827E-3,1.655673E-2,4.260382E-3,-3.8593335E-3,-7.925799E-3,-2.474342E-3,6.598231E-4,6.542611E-3,-5.9609422E-3,3.6020759E-3,-3.8036013E-3,2.5642654E-3,1.0251868E-2,1.4232753E-3,3.6959965E-3,8.329804E-3],"split_indices":[2,43,17,7,2,53,67,3,2,69,43,0,0,43,73,68,46,9,0,43,70,68,7,0,43,53,47,32,0,0,0,10,68,0,3,71,68,71,4,62,68,0,44,0,46,4,46,72,0,67,0,0,9,0,0,69,9,0,0,70,0,2,0,9,5,0,2,4,0,43,3,3,0,3,10,70,6,0,0,66,0,5,0,43,0,53,2,4,62,43,68,4,7,60,0,0,0,3,0,0,0,2,52,9,7,0,0,31,0,60,7,0,4,48,0,9,0,0,0,9,0,0,0,0,0,0,0,71,4,67,43,69,0,4,5,66,0,0,0,0,0,0,0,1,0,73,0,9,8,62,52,0,0,0,0,0,0,0,73,0,0,0,0,0,0,69,0,44,0,48,3,0,0,0,5,3,69,9,30,0,0,68,2,0,4,5,2,1,9,62,0,9,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.001E3,7.57E2,2.44E2,2.31E2,5.26E2,1.4E1,2.3E2,5.8E1,1.73E2,2.12E2,3.14E2,7E0,7E0,1.5E2,8E1,4.5E1,1.3E1,1.58E2,1.5E1,1.65E2,4.7E1,2.82E2,3.2E1,6.6E1,8.4E1,5.2E1,2.8E1,3.9E1,6E0,4E0,9E0,2.2E1,1.36E2,5E0,1E1,1.42E2,2.3E1,3.4E1,1.3E1,1E2,1.82E2,5E0,2.7E1,4E0,6.2E1,5.4E1,3E1,4.6E1,6E0,2.2E1,6E0,5E0,3.4E1,6E0,1.6E1,2.1E1,1.15E2,5E0,5E0,1.1E2,3.2E1,1.9E1,4E0,1.9E1,1.5E1,4E0,9E0,9.6E1,4E0,1.74E2,8E0,2.1E1,6E0,2.4E1,3.8E1,2.3E1,3.1E1,8E0,2.2E1,3.9E1,7E0,1.4E1,8E0,9E0,2.5E1,1.3E1,8E0,6.6E1,4.9E1,2.6E1,8.4E1,1.8E1,1.4E1,1.2E1,7E0,1.2E1,7E0,1.1E1,4E0,5E0,4E0,6.4E1,3.2E1,1.15E2,5.9E1,4E0,4E0,1.7E1,4E0,1.3E1,1.1E1,8E0,3E1,1.9E1,4E0,2.7E1,4E0,2.9E1,1E1,1E1,4E0,5E0,4E0,6E0,7E0,4E0,4E0,4.5E1,2.1E1,4.1E1,8E0,2.2E1,4E0,4E1,4.4E1,1.2E1,6E0,4E0,1E1,6E0,6E0,7E0,4E0,6E1,4E0,2.8E1,4E0,8.7E1,2.8E1,3E1,2.9E1,1.3E1,4E0,8E0,5E0,6E0,5E0,7E0,2.3E1,9E0,1E1,2.2E1,5E0,6E0,4E0,2.4E1,2.1E1,1.4E1,7E0,2.8E1,1.3E1,4E0,4E0,9E0,1.3E1,2.5E1,1.5E1,3.1E1,1.3E1,4E0,8E0,4.1E1,1.9E1,4E0,2.4E1,3E1,5.7E1,1.8E1,1E1,2.6E1,4E0,2.5E1,4E0,1.8E1,5E0,5E0,1.9E1,9E0,5E0,1.3E1,1.5E1,5E0,8E0,9E0,4E0,7E0,1.8E1,9E0,6E0,1.5E1,1.6E1,9E0,4E0,9E0,3.2E1,9E0,1E1,7E0,1.7E1,1.9E1,1.1E1,2.5E1,3.2E1,4E0,1.4E1,6E0,4E0,6E0,2E1,2.1E1,4E0,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"233","size_leaf_vector":"1"}},{"base_weights":[-3.9656907E-3,-9.501214E-2,2.7065217E-1,-2.784495E-1,-9.365859E-3,-3.6793208E-1,3.017125E-1,-4.4126916E-1,-2.2017272E-1,-1.7643312E-1,2.1908537E-2,-2.1849465E-2,-8.505184E-3,1.5604289E-1,4.175075E-1,-3.507889E-1,-7.0138806E-1,-2.3586765E-1,9.097859E-3,-1.4819905E-1,-2.6608398E-2,3.0838398E-2,-2.1082714E-2,2.8791332E-1,1.0092565E-1,5.0469875E-1,2.5951615E-1,-2.0058792E-2,-2.6352626E-1,-1.7305136E-2,-4.054622E-2,2.1098584E-3,-2.636969E-1,-1.8473676E-1,-5.438283E-2,-5.381478E-2,6.6572115E-2,1.5014201E-1,1.7512841E-2,1.2969783E-1,-7.551079E-3,5.437801E-1,1.9590639E-1,2.812694E-2,1.86803E-1,-1.603637E-1,-2.1972109E-2,-2.7934167E-1,2.3219657E-1,-3.302879E-1,-1.7906928E-1,-1.1146918E-1,-2.4207743E-1,1.619824E-2,-1.09921984E-1,-1.9888683E-1,-3.408119E-2,2.1942863E-2,1.4924604E-1,1.7885772E-3,1.0573727E-2,2.3934389E-2,1.9963692E-1,4.638127E-1,6.910533E-1,1.28993485E-2,1.8524277E-3,1.1050021E-1,3.6835158E-1,-2.061758E-1,4.2250715E-4,-6.0287667E-3,-1.7200807E-2,1.6376795E-2,6.724597E-4,-4.2167246E-1,-2.4743077E-1,-2.2192113E-1,-6.0843613E-2,-9.97553E-3,-6.9336176E-2,-2.721887E-1,-1.7132155E-3,-3.5586585E-3,3.717867E-3,-6.844297E-3,-1.8772351E-3,-1.376032E-2,-3.4678967E-3,7.466982E-3,-4.6565972E-2,3.2273047E-2,-1.7515205E-2,2.2839847E-1,7.858436E-2,-3.4470063E-2,1.3834985E-1,1.7988099E-2,1.5056755E-1,1.0686118E-2,2.2362962E-2,3.714944E-2,2.0898255E-2,1.2367412E-2,4.418352E-2,1.1051776E-2,2.1141248E-2,-1.143979E-2,-5.7026832E-3,-3.279505E-1,-5.030666E-1,-2.6878613E-1,-6.6011306E-4,-1.2509137E-1,-2.6261213E-1,-9.1879135E-3,1.2518891E-2,1.7837885E-4,-4.059825E-3,-3.0420086E-1,-6.5529323E-3,-9.235642E-2,3.6423183E-3,1.7449673E-2,2.3503724E-1,2.5325418E-1,-1.8933563E-3,-2.1648366E-2,1.376548E-1,-4.706644E-3,9.66299E-3,1.0267954E-3,9.797057E-3,2.8059423E-2,1.9189501E-1,9.446828E-2,-3.8922029E-3,-1.7883098E-2,-5.340247E-3,-2.667475E-2,-1.5185724E-2,-2.9505622E-1,-2.6615907E-3,-7.375062E-3,-4.7378655E-4,-2.841051E-1,-3.5496268E-3,6.878219E-2,-4.735933E-3,-7.836087E-3,-1.5603953E-2,-3.6249865E-2,-1.4716834E-1,7.715605E-2,-4.8646215E-2,-1.1873774E-2,7.4849084E-2,1.8090488E-3,1.6288273E-2,1.6469273E-1,3.308046E-1,3.084686E-2,-1.0090223E-1,1.0215925E-2,5.1329482E-2,4.0874323E-3,-3.519478E-2,-4.97195E-3,8.558977E-3,1.2989512E-3,2.2955103E-1,-6.5646076E-4,1.4093477E-1,-1.5428957E-2,-8.987475E-3,-1.4099805E-2,-6.715339E-3,4.6868604E-3,7.3111086E-4,1.4541056E-3,-5.130876E-3,-7.5847246E-3,-1.6679307E-3,4.994398E-3,-2.1293836E-4,-6.8203514E-3,1.4341802E-3,-4.3626237E-3,7.148556E-4,4.195675E-3,-5.6436253E-3,1.6728593E-3,8.770242E-3,9.296893E-3,1.732066E-2,3.204331E-3,-2.4183947E-3,-6.6490406E-3,-1.7418164E-3,5.538608E-3,-2.6759E-3,6.589656E-4,-3.5857034E-3,1.4160664E-2,4.2908005E-3,3.2838914E-3,8.34704E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,-1,35,-1,37,39,41,43,-1,45,-1,-1,47,49,51,53,55,57,59,-1,61,-1,63,65,-1,67,69,-1,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,95,97,99,101,-1,-1,103,105,107,-1,-1,-1,-1,-1,109,111,113,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,121,123,-1,125,127,129,131,-1,133,-1,-1,-1,-1,-1,135,-1,-1,-1,-1,137,139,141,-1,143,145,-1,147,-1,-1,149,-1,151,153,155,157,159,-1,161,163,-1,165,-1,-1,167,169,171,-1,-1,-1,-1,-1,173,-1,-1,-1,175,-1,177,-1,-1,-1,179,181,183,185,187,189,-1,-1,191,193,195,197,-1,199,-1,201,-1,-1,-1,203,-1,205,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5003695E1,1.1796147E1,5.049837E0,2.1706238E0,2.6842532E0,1.4458358E-1,3.9493904E0,1.21317E0,1.2116995E0,8.2832646E-1,1.8633032E0,0E0,0E0,7.486539E-1,1.6873856E0,2.3488188E-1,5.352783E-1,1.1396675E0,0E0,2.5389636E-1,0E0,1.2922211E0,0E0,3.3242273E-1,6.069044E-1,9.071808E-1,1.1308231E0,0E0,4.9255216E-1,0E0,0E0,1.295269E0,8.087139E-1,2.0408261E-1,9.194082E-2,3.5679632E-1,1.1036274E0,1.1395872E-1,0E0,5.130117E-1,0E0,5.509834E-1,1.315833E-1,0E0,5.4110885E-1,1.4943129E-1,0E0,7.596862E-2,2.8760093E-1,5.3927326E-1,3.4113073E-1,9.9455565E-2,1.7508519E-1,7.211452E-2,2.8412238E-2,1.614477E-1,2.836265E-1,8.2428366E-1,5.6932116E-1,0E0,0E0,1.9855772E-1,3.3237934E-1,1.2808514E-1,2.839775E-1,0E0,0E0,3.1134298E-1,2.8845072E-2,2.0869553E-2,0E0,0E0,0E0,0E0,0E0,1.3252687E-1,2.30165E-1,1.680944E-1,2.0501871E-1,0E0,2.748178E-2,6.534195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4633583E-1,5.7298344E-1,0E0,3.3846617E-1,3.4046504E-1,6.0482375E-2,8.088446E-2,0E0,1.7181867E-1,0E0,0E0,0E0,0E0,0E0,1.5649688E-1,0E0,0E0,0E0,0E0,2.1898055E-1,5.6430817E-2,2.2154713E-1,0E0,6.5802634E-2,1.1792421E-1,0E0,9.765879E-2,0E0,0E0,3.302896E-2,0E0,1.6688243E-1,2.0363745E-1,3.0385548E-1,2.8547662E-1,2.5457764E-1,0E0,9.464241E-2,2.4806172E-1,0E0,4.907278E-2,0E0,0E0,2.3323132E-1,1.5389436E-1,8.785039E-2,0E0,0E0,0E0,0E0,0E0,8.36575E-2,0E0,0E0,0E0,4.6357393E-2,0E0,1.5694037E-2,0E0,0E0,0E0,1.5042421E-1,4.6960175E-2,5.617225E-2,2.4910985E-1,2.7238593E-1,2.029925E-1,0E0,0E0,6.742704E-2,5.6003332E-2,4.8486833E-2,1.904314E-2,0E0,1.4925034E-1,0E0,1.998904E-2,0E0,0E0,0E0,1.924293E-1,0E0,2.046971E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,21,21,23,23,24,24,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,41,41,42,42,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,69,69,75,75,76,76,77,77,78,78,80,80,81,81,90,90,91,91,93,93,94,94,95,95,96,96,98,98,104,104,109,109,110,110,111,111,113,113,114,114,116,116,119,119,121,121,122,122,123,123,124,124,125,125,127,127,128,128,130,130,133,133,134,134,135,135,141,141,145,145,147,147,151,151,152,152,153,153,154,154,155,155,156,156,159,159,160,160,161,161,162,162,164,164,166,166,170,170,172,172],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,-1,36,-1,38,40,42,44,-1,46,-1,-1,48,50,52,54,56,58,60,-1,62,-1,64,66,-1,68,70,-1,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,96,98,100,102,-1,-1,104,106,108,-1,-1,-1,-1,-1,110,112,114,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,122,124,-1,126,128,130,132,-1,134,-1,-1,-1,-1,-1,136,-1,-1,-1,-1,138,140,142,-1,144,146,-1,148,-1,-1,150,-1,152,154,156,158,160,-1,162,164,-1,166,-1,-1,168,170,172,-1,-1,-1,-1,-1,174,-1,-1,-1,176,-1,178,-1,-1,-1,180,182,184,186,188,190,-1,-1,192,194,196,198,-1,200,-1,202,-1,-1,-1,204,-1,206,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.972052E5,1E0,3.3817584E7,4.97E2,9.3326636E-2,1.087521E3,2.4545455E0,2.8366232E0,1E0,4.6E1,-2.1849465E-2,-8.505184E-3,7E0,8E0,3.6E1,2.8914285E0,5.597832E-4,9.097859E-3,8.599521E5,-2.6608398E-2,6.374563E5,-2.1082714E-2,1.2774637E10,1.2118524E7,4.9036694E0,1.2083E4,-2.0058792E-2,3.7509E4,-1.7305136E-2,-4.054622E-2,4.1142857E1,5.100098E4,3.2E1,3.5364E4,1E0,3.9742767E2,6.4348956E2,1.7512841E-2,3.3193566E5,-7.551079E-3,1.9617874E3,2.0003653E0,2.812694E-2,9.0577506E-2,2.6412E5,-2.1972109E-2,6.737099E7,2E0,8.3925985E-2,7.9E2,8.3E1,7.026624E7,8.754E3,1E0,5.0200376E7,1.5E1,2.483E3,2.9375E0,1.7885772E-3,1.0573727E-2,2.3625128E6,2.08E3,1.9675E4,1.7493458E7,1.28993485E-2,1.8524277E-3,4.229649E6,4.907764E3,5.6E1,4.2250715E-4,-6.0287667E-3,-1.7200807E-2,1.6376795E-2,6.724597E-4,3.6E2,1.3091103E0,9.117022E0,1.7356666E3,-9.97553E-3,1.2E1,2.241E2,-1.7132155E-3,-3.5586585E-3,3.717867E-3,-6.844297E-3,-1.8772351E-3,-1.376032E-2,-3.4678967E-3,7.466982E-3,4.220534E5,7.4711205E6,-1.7515205E-2,3.0561172E5,1.574E3,1.5204346E8,5.130175E2,1.7988099E-2,4.032683E6,1.0686118E-2,2.2362962E-2,3.714944E-2,2.0898255E-2,1.2367412E-2,2.2807484E7,1.1051776E-2,2.1141248E-2,-1.143979E-2,-5.7026832E-3,4.1142857E1,1.9885094E-8,5.13657E6,-6.6011306E-4,1.4276666E2,7.997723E6,-9.1879135E-3,3.3594444E1,1.7837885E-4,-4.059825E-3,1.2608696E0,-6.5529323E-3,1E0,6.933837E-1,2.378914E6,1E0,1.0641107E8,-1.8933563E-3,5.5586E4,4.857143E0,-4.706644E-3,1.4195632E6,1.0267954E-3,9.797057E-3,2.0094643E0,2.7662E4,2.6776E4,-3.8922029E-3,-1.7883098E-2,-5.340247E-3,-2.667475E-2,-1.5185724E-2,1.0433326E8,-2.6615907E-3,-7.375062E-3,-4.7378655E-4,1.547E3,-3.5496268E-3,1.364E3,-4.735933E-3,-7.836087E-3,-1.5603953E-2,4.2609915E6,4.032683E6,1.5998265E6,9.558022E7,3.2883008E6,8.561815E7,1.8090488E-3,1.6288273E-2,4.371613E2,4.7185095E2,2.141096E1,4.7563504E7,1.0215925E-2,1.3736341E8,4.0874323E-3,2.9355192E1,-4.97195E-3,8.558977E-3,1.2989512E-3,4.205298E0,-6.5646076E-4,1.562752E3,-1.5428957E-2,-8.987475E-3,-1.4099805E-2,-6.715339E-3,4.6868604E-3,7.3111086E-4,1.4541056E-3,-5.130876E-3,-7.5847246E-3,-1.6679307E-3,4.994398E-3,-2.1293836E-4,-6.8203514E-3,1.4341802E-3,-4.3626237E-3,7.148556E-4,4.195675E-3,-5.6436253E-3,1.6728593E-3,8.770242E-3,9.296893E-3,1.732066E-2,3.204331E-3,-2.4183947E-3,-6.6490406E-3,-1.7418164E-3,5.538608E-3,-2.6759E-3,6.589656E-4,-3.5857034E-3,1.4160664E-2,4.2908005E-3,3.2838914E-3,8.34704E-3],"split_indices":[2,43,17,7,2,53,67,73,53,113,3,0,0,3,32,2,68,57,0,43,0,43,0,46,62,53,9,0,1,0,0,67,43,10,9,29,67,4,0,43,0,4,68,0,72,7,0,7,10,53,2,12,7,1,8,7,10,2,69,0,0,60,44,9,66,0,0,60,4,44,0,0,0,0,0,2,57,73,48,0,3,4,0,0,0,0,0,0,0,0,66,43,0,48,2,7,67,0,60,0,0,0,0,0,62,0,0,0,0,67,52,9,0,67,9,0,73,0,0,68,0,26,57,9,29,5,0,1,69,0,60,0,0,57,9,44,0,0,0,0,0,7,0,0,0,10,0,2,0,0,0,60,60,62,7,60,12,0,0,67,4,61,7,0,7,0,73,0,0,0,69,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,7.5E2,2.48E2,2.38E2,5.12E2,1.1E1,2.37E2,6.1E1,1.77E2,8E1,4.32E2,6E0,5E0,1.06E2,1.31E2,4.7E1,1.4E1,1.71E2,6E0,7.6E1,4E0,4.25E2,7E0,3E1,7.6E1,8.3E1,4.8E1,2.2E1,2.5E1,6E0,8E0,1.8E1,1.53E2,5.4E1,2.2E1,1.26E2,2.99E2,1.3E1,1.7E1,6.9E1,7E0,7.3E1,1E1,7E0,4.1E1,1.8E1,7E0,8E0,1E1,8.4E1,6.9E1,2.5E1,2.9E1,1E1,1.2E1,1.4E1,1.12E2,1.95E2,1.04E2,6E0,7E0,2.8E1,4.1E1,5E1,2.3E1,6E0,4E0,3E1,1.1E1,1.4E1,4E0,4E0,4E0,6E0,4E0,3.8E1,4.6E1,5E1,1.9E1,6E0,1.9E1,2.5E1,4E0,4E0,6E0,7E0,5E0,7E0,7E0,6E0,1.06E2,1.91E2,4E0,4.8E1,5.6E1,1.9E1,9E0,7E0,3.4E1,5E0,4.5E1,1.4E1,9E0,8E0,2.2E1,6E0,5E0,8E0,6E0,2E1,1.8E1,4.2E1,4E0,1.6E1,3.4E1,6E0,1.3E1,4E0,1.5E1,1.9E1,6E0,5.5E1,5.1E1,1.79E2,1.2E1,4.4E1,4E0,2.1E1,3.5E1,7E0,1.2E1,4E0,5E0,9E0,2.5E1,1.6E1,6E0,1.5E1,5E0,1.1E1,7E0,3.7E1,5E0,1.2E1,4E0,3E1,4E0,9E0,4E0,5E0,1.4E1,2.8E1,2.7E1,2.1E1,3E1,1.19E2,6E1,5E0,7E0,2.2E1,2.2E1,1.3E1,8E0,1.7E1,1.8E1,4E0,8E0,5E0,4E0,5E0,2E1,5E0,1.1E1,2.5E1,1.2E1,2.5E1,5E0,5E0,4E0,1.5E1,1.3E1,2.3E1,4E0,1.5E1,6E0,1.3E1,1.7E1,2.9E1,9E1,5.6E1,4E0,4E0,1.8E1,7E0,1.5E1,9E0,4E0,4E0,4E0,1.1E1,7E0,4E0,4E0,1.2E1,8E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"207","size_leaf_vector":"1"}},{"base_weights":[3.6584307E-3,-8.717372E-2,2.6933727E-1,-1.9764858E-1,1.747211E-2,3.1734458E-1,-1.3559663E-1,-3.3104765E-1,-1.5139028E-1,-4.751485E-2,6.8291664E-2,3.481294E-2,2.7624777E-1,-4.2068493E-1,1.372734E-2,-2.1810003E-1,-3.7600914E-1,-1.3114409E-1,-4.616602E-1,-8.735668E-2,7.129579E-2,4.550107E-2,2.2154155E-1,1.398515E-1,3.7319E-1,-2.5613237E-2,-8.134605E-3,1.4498642E-1,-1.6281003E-1,-1.39393695E-2,-1.4558814E-1,-4.174836E-1,-2.3760873E-1,-2.134523E-1,-5.4366227E-2,-3.4546994E-2,-3.1609958E-1,-1.8754998E-1,-2.8570468E-2,1.05273254E-1,-7.7200104E-3,5.653491E-2,-1.9249128E-1,1.4698611E-1,1.6100287E-2,1.1064811E-1,2.9183999E-2,3.960429E-1,6.3921496E-2,1.2552225E-2,-4.0077963E-3,-1.1661417E-2,-1.8775862E-3,-1.7426285E-1,-1.6715435E-3,-2.9512233E-1,-5.237878E-1,-1.0580909E-1,-1.5359684E-2,-2.5049242E-1,-1.1815903E-1,2.0022953E-2,-7.944116E-2,-5.7165525E-3,-1.7918082E-2,-1.4690425E-1,-3.51875E-1,-5.6904454E-2,7.284781E-2,8.705399E-2,9.951149E-3,-3.8742732E-2,8.637103E-2,-1.6535815E-2,5.2856794E-4,6.1813336E-2,1.1282641E-2,-8.473943E-2,1.3368845E-1,2.6736724E-1,4.7632608E-1,1.1844745E-2,-4.948767E-3,-3.588991E-3,-1.1999852E-2,-1.8307015E-1,-1.8160485E-2,-5.6335413E-1,-1.131592E-2,-4.64755E-4,-8.334148E-3,-1.8143399E-1,-3.07392E-1,-9.175192E-3,-1.6222003E-1,-3.5944086E-2,-2.6765063E-1,-1.9269049E-1,-3.3465356E-2,-2.6381325E-2,-2.879998E-3,-8.2045004E-2,2.8358858E-2,-5.3661563E-3,8.364929E-3,1.1922198E-1,4.408848E-2,7.3120794E-3,-6.1274584E-2,1.4336166E-1,4.445573E-2,-2.9693093E-4,5.1500276E-3,-6.913837E-3,4.6684203E-4,5.759979E-2,1.7818967E-1,3.1480128E-1,5.5637907E-2,5.3607506E-1,3.2572708E-1,-1.31378705E-2,-1.9006245E-3,-2.814211E-2,-1.3204679E-2,-2.0752932E-1,-6.4006355E-4,-3.2659554E-1,-5.3283176E-3,-3.0754057E-3,2.740203E-3,-5.1902775E-3,-9.2632305E-3,-7.218088E-2,3.7500963E-2,-3.2925078E-1,-2.8742468E-3,-1.2656488E-1,-2.4597009E-1,1.3971746E-2,-5.1816422E-3,-1.8467259E-2,-1.2120399E-1,-1.27165755E-2,4.570811E-3,1.8031854E-3,-2.599539E-3,2.1877536E-3,1.3422905E-1,5.5799777E-5,4.85894E-3,-3.0685423E-2,-1.10736955E-2,1.6237536E-1,-1.2248257E-3,-5.7272278E-2,1.0098505E-1,2.0620754E-2,9.620597E-3,2.0337267E-1,-4.1178144E-3,3.6593717E-1,6.0018757E-3,-9.654969E-3,1.2329838E-2,5.579276E-1,1.0524922E-2,7.307919E-3,3.6347908E-1,-1.02076165E-1,-2.4671112E-1,-3.789711E-1,-2.112526E-1,1.8101072E-2,-1.036983E-1,1.4309485E-1,-6.9076917E-3,-1.6559511E-2,-8.123538E-3,-7.3194234E-3,-1.4387994E-3,-1.3026931E-2,-5.4315035E-3,-1.9422622E-3,3.1041286E-3,-5.6895424E-2,3.0730215E-3,-1.4101261E-1,-2.5369686E-3,1.8590864E-3,-2.9361583E-3,4.1070725E-3,8.30705E-3,-5.9224144E-2,4.2153742E-2,1.2688813E-1,2.28272E-1,2.3988217E-2,-1.8472953E-2,1.5314375E-2,7.130514E-2,5.5312112E-2,-3.171402E-2,1.4755993E-1,2.9879907E-1,9.34993E-3,1.8799089E-2,2.1985715E-2,3.28682E-2,8.785764E-3,1.919603E-2,-9.236898E-3,-1.669248E-4,-1.3185396E-2,-7.040399E-3,-9.734367E-3,-1.9539097E-2,-1.2630345E-2,-2.8701501E-3,-4.281962E-3,2.3834284E-3,3.5186842E-4,-6.084418E-3,2.3185445E-3,8.732369E-3,4.232548E-3,-2.45839E-3,2.5006486E-3,-4.720996E-3,-1.9990192E-3,-7.3178234E-3,-9.091291E-4,-6.9615114E-3,4.0126187E-3,-1.3332478E-3,7.975032E-3,2.7829132E-3,1.187323E-2,4.720372E-3,-1.2999049E-3,5.8013718E-3,2.0875838E-3,8.559218E-3,4.879857E-3,6.726875E-4,-3.108075E-3,4.2195295E-4,8.844434E-3,2.327079E-3,1.599773E-2,7.4821156E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,51,-1,53,55,57,59,61,-1,63,65,67,69,-1,71,73,75,-1,77,-1,79,81,-1,-1,-1,-1,83,-1,85,87,89,-1,91,93,-1,95,-1,-1,97,99,101,103,105,-1,107,109,-1,-1,111,-1,113,115,117,119,-1,-1,-1,-1,121,-1,123,-1,-1,-1,125,127,129,131,133,135,137,139,-1,-1,141,143,145,-1,147,149,-1,151,153,155,-1,-1,-1,-1,157,159,161,163,165,167,-1,-1,-1,-1,169,-1,171,-1,-1,-1,-1,-1,173,175,177,-1,179,181,183,-1,185,187,189,-1,-1,-1,-1,191,-1,-1,193,-1,195,-1,197,199,201,-1,203,-1,205,-1,-1,-1,207,-1,-1,209,211,213,215,217,219,221,223,225,-1,-1,-1,-1,-1,-1,-1,-1,227,-1,229,-1,-1,-1,-1,-1,231,233,235,237,239,-1,-1,241,243,245,247,249,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5289871E1,9.032387E0,5.2334538E0,2.2947922E0,1.3308935E0,4.076231E0,1.24032E0,4.1058636E-1,1.7234836E0,8.440813E-1,7.792599E-1,0E0,2.8567734E0,2.2948349E-1,4.8704785E-1,1.3798487E-1,2.911272E-1,1.6820073E0,4.7165155E-1,7.7800965E-1,3.821002E-1,5.308244E-1,2.2326982E-1,1.2659254E0,8.7934685E-1,0E0,0E0,3.7593743E-1,8.826588E-2,0E0,5.016467E-2,5.088024E-1,1.8986428E-1,4.2573023E-1,1.7467504E0,0E0,1.3142407E-1,2.7558446E-1,2.484E-1,6.1503947E-2,0E0,5.434621E-1,3.0773142E-1,1.5053433E-1,0E0,4.119885E-1,0E0,1.1019726E0,3.5665295E-1,0E0,0E0,0E0,0E0,9.539741E-2,0E0,2.1345782E-1,1.4887285E-1,6.2460452E-2,0E0,3.0573225E-1,1.8098527E-1,0E0,1.0873787E0,0E0,0E0,2.0650327E-1,5.3549767E-1,1.4594564E-1,1.6178617E-1,4.4499785E-2,0E0,2.1389848E-1,3.399999E-1,0E0,0E0,4.138354E-2,0E0,6.307938E-2,2.6061928E-1,4.6455765E-1,4.7325325E-1,0E0,0E0,0E0,0E0,1.895966E-1,0E0,1.0550308E-1,0E0,0E0,0E0,1.8953872E-1,1.4305449E-1,5.11176E-2,2.9031038E-2,2.9546228E-1,2.9588842E-1,7.1040034E-2,5.009608E-2,0E0,0E0,1.2722299E-1,4.7642812E-2,2.9314958E-2,0E0,1.5213877E-2,4.440259E-2,0E0,2.2186998E-1,2.0133007E-1,4.9458593E-1,0E0,0E0,0E0,0E0,1.6839492E-1,3.5472524E-1,3.21182E-1,6.147865E-1,2.0970535E-1,8.7980986E-2,0E0,0E0,0E0,0E0,1.3182676E-1,0E0,1.8575239E-1,0E0,0E0,0E0,0E0,0E0,2.1244073E-1,1.7449191E-1,2.205038E-2,0E0,4.067844E-2,3.6194026E-2,2.9653106E-2,0E0,7.236607E-2,3.4276187E-2,3.2172166E-2,0E0,0E0,0E0,0E0,1.7073601E-2,0E0,0E0,7.940978E-2,0E0,9.965205E-2,0E0,8.7258524E-1,3.5263324E-1,4.895884E-2,0E0,2.0090568E-1,0E0,1.18709564E-1,0E0,0E0,0E0,2.4076462E-1,0E0,0E0,8.3284855E-2,1.1490561E-1,5.504179E-2,1.2540674E-1,1.3127655E-1,7.898476E-2,1.7243844E-1,3.744553E-2,1.2805308E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.095936E-2,0E0,3.3928603E-2,0E0,0E0,0E0,0E0,0E0,9.469057E-2,3.7424363E-2,1.05436265E-1,3.8950086E-2,1.4120728E-1,0E0,0E0,1.4045513E-1,3.1268362E-2,1.6593087E-2,1.22567415E-1,4.7875762E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,47,47,48,48,53,53,55,55,56,56,57,57,59,59,60,60,62,62,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,77,77,78,78,79,79,80,80,85,85,87,87,91,91,92,92,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,105,105,106,106,108,108,109,109,110,110,115,115,116,116,117,117,118,118,119,119,120,120,125,125,127,127,133,133,134,134,135,135,137,137,138,138,139,139,141,141,142,142,143,143,148,148,151,151,153,153,155,155,156,156,157,157,159,159,161,161,165,165,168,168,169,169,170,170,171,171,172,172,173,173,174,174,175,175,176,176,185,185,187,187,193,193,194,194,195,195,196,196,197,197,200,200,201,201,202,202,203,203,204,204],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,52,-1,54,56,58,60,62,-1,64,66,68,70,-1,72,74,76,-1,78,-1,80,82,-1,-1,-1,-1,84,-1,86,88,90,-1,92,94,-1,96,-1,-1,98,100,102,104,106,-1,108,110,-1,-1,112,-1,114,116,118,120,-1,-1,-1,-1,122,-1,124,-1,-1,-1,126,128,130,132,134,136,138,140,-1,-1,142,144,146,-1,148,150,-1,152,154,156,-1,-1,-1,-1,158,160,162,164,166,168,-1,-1,-1,-1,170,-1,172,-1,-1,-1,-1,-1,174,176,178,-1,180,182,184,-1,186,188,190,-1,-1,-1,-1,192,-1,-1,194,-1,196,-1,198,200,202,-1,204,-1,206,-1,-1,-1,208,-1,-1,210,212,214,216,218,220,222,224,226,-1,-1,-1,-1,-1,-1,-1,-1,228,-1,230,-1,-1,-1,-1,-1,232,234,236,238,240,-1,-1,242,244,246,248,250,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,4.4761734E5,1E0,1.16E2,8.39E2,3.6E1,1.0201538E-1,1E0,3E1,3.3486558E6,3.7951445E6,3.481294E-2,1.3026532E6,7E0,1.1E1,4.2E1,9.558022E7,6.07E2,1E0,1E0,7.89E2,1.7500046E8,3.962617E0,1.0534078E4,1.8545505E2,-2.5613237E-2,-8.134605E-3,4.433E3,2.8E1,-1.39393695E-2,1.5832983E-8,6.0921145E4,7.080933E2,1.852364E6,2.00087E5,-3.4546994E-2,2.1095E4,2.43E2,5.751617E6,6.7723384E7,-7.7200104E-3,4.7563504E7,1.7694E4,3.7788504E7,1.6100287E-2,6.4258E4,2.9183999E-2,1.1293673E3,2.4344263E0,1.2552225E-2,-4.0077963E-3,-1.1661417E-2,-1.8775862E-3,6.101E3,-1.6715435E-3,5E0,1.318849E6,1.7E1,-1.5359684E-2,1E0,1.3809524E0,2.0022953E-2,2.0734E4,-5.7165525E-3,-1.7918082E-2,1.1362371E6,3.304531E3,6.887749E7,4.9E1,1.9668E4,9.951149E-3,3.7974394E-6,4.9390244E0,-1.6535815E-2,5.2856794E-4,3E0,1.1282641E-2,2.0939393E0,8.072743E4,7.7191065E6,1.853776E7,1.1844745E-2,-4.948767E-3,-3.588991E-3,-1.1999852E-2,3.6E1,-1.8160485E-2,7.7E1,-1.131592E-2,-4.64755E-4,-8.334148E-3,3.0416667E0,5.57E2,7.1428573E-1,3.478747E4,1.158E3,1.609E3,9.836066E-1,2.141096E1,-2.6381325E-2,-2.879998E-3,3.6E1,4.1226246E4,1.3529412E0,8.364929E-3,4.33E2,1.3935602E1,7.3120794E-3,1.764442E7,1E0,4.326193E2,-2.9693093E-4,5.1500276E-3,-6.913837E-3,4.6684203E-4,2.4693803E2,4.1325716E7,2E0,1E0,2.53E2,7.15244E5,-1.31378705E-2,-1.9006245E-3,-2.814211E-2,-1.3204679E-2,3.8575E4,-6.4006355E-4,1.7909248E5,-5.3283176E-3,-3.0754057E-3,2.740203E-3,-5.1902775E-3,-9.2632305E-3,1.3867E4,1.3520833E1,1.1939979E8,-2.8742468E-3,1.7171982E6,1.605076E6,1.512E4,-5.1816422E-3,2.6530768E2,8.1407714E-1,1E0,4.570811E-3,1.8031854E-3,-2.599539E-3,2.1877536E-3,2.0518206E3,5.5799777E-5,4.85894E-3,4.699871E2,-1.10736955E-2,1.7642576E8,-1.2248257E-3,1.18033766E5,1.684E0,4.5179688E1,9.620597E-3,1.767635E3,-4.1178144E-3,4.6134964E2,6.0018757E-3,-9.654969E-3,1.2329838E-2,1.6975454E3,1.0524922E-2,7.307919E-3,8E0,3.103E3,1.86619E5,4.0869565E0,4.58E2,1.5152774E2,2.978142E0,1.04E2,2.3266666E0,-1.6559511E-2,-8.123538E-3,-7.3194234E-3,-1.4387994E-3,-1.3026931E-2,-5.4315035E-3,-1.9422622E-3,3.1041286E-3,1.7676923E2,3.0730215E-3,1E1,-2.5369686E-3,1.8590864E-3,-2.9361583E-3,4.1070725E-3,8.30705E-3,1.2E1,2.75E0,4.05E2,1.57E2,1E0,-1.8472953E-2,1.5314375E-2,2.445E3,5.117647E0,2.3266666E0,3.776793E8,5.501002E9,9.34993E-3,1.8799089E-2,2.1985715E-2,3.28682E-2,8.785764E-3,1.919603E-2,-9.236898E-3,-1.669248E-4,-1.3185396E-2,-7.040399E-3,-9.734367E-3,-1.9539097E-2,-1.2630345E-2,-2.8701501E-3,-4.281962E-3,2.3834284E-3,3.5186842E-4,-6.084418E-3,2.3185445E-3,8.732369E-3,4.232548E-3,-2.45839E-3,2.5006486E-3,-4.720996E-3,-1.9990192E-3,-7.3178234E-3,-9.091291E-4,-6.9615114E-3,4.0126187E-3,-1.3332478E-3,7.975032E-3,2.7829132E-3,1.187323E-2,4.720372E-3,-1.2999049E-3,5.8013718E-3,2.0875838E-3,8.559218E-3,4.879857E-3,6.726875E-4,-3.108075E-3,4.2195295E-4,8.844434E-3,2.327079E-3,1.599773E-2,7.4821156E-3],"split_indices":[2,43,6,44,2,0,53,24,3,43,43,0,43,6,8,2,7,2,13,29,2,12,71,4,73,0,0,10,3,0,52,60,4,9,5,0,1,0,60,7,0,7,10,7,0,1,0,4,69,0,0,0,0,9,0,12,9,3,0,8,68,0,44,0,0,43,48,7,0,12,0,53,71,0,0,8,0,68,48,62,62,0,0,0,0,2,0,44,0,0,0,71,10,71,62,2,2,71,61,0,0,10,48,68,0,2,73,0,60,19,70,0,0,0,0,71,66,32,8,8,1,0,0,0,0,9,0,43,0,0,0,0,0,1,73,7,0,60,12,9,0,67,53,26,0,0,0,0,48,0,0,67,0,7,0,48,69,71,0,67,0,4,0,0,0,67,0,0,3,1,12,73,2,67,69,10,68,0,0,0,0,0,0,0,0,67,0,3,0,0,0,0,0,8,68,10,0,23,0,0,2,69,68,12,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.046E3,7.8E2,2.66E2,3.79E2,4.01E2,2.38E2,2.8E1,9.6E1,2.83E2,1.76E2,2.25E2,1.9E1,2.19E2,9E0,1.9E1,2.9E1,6.7E1,2.67E2,1.6E1,1.32E2,4.4E1,1.97E2,2.8E1,9.2E1,1.27E2,5E0,4E0,1.1E1,8E0,1.2E1,1.7E1,5E1,1.7E1,1.28E2,1.39E2,4E0,1.2E1,4.8E1,8.4E1,3.9E1,5E0,1.89E2,8E0,1.9E1,9E0,8.8E1,4E0,1.18E2,9E0,7E0,4E0,4E0,4E0,1.3E1,4E0,2.5E1,2.5E1,8E0,9E0,9.1E1,3.7E1,6E0,1.33E2,4E0,8E0,4E1,8E0,6.6E1,1.8E1,3.5E1,4E0,4.5E1,1.44E2,4E0,4E0,1.1E1,8E0,9E0,7.9E1,4.7E1,7.1E1,4E0,5E0,7E0,6E0,1.3E1,1.2E1,2.1E1,4E0,4E0,4E0,4.3E1,4.8E1,1.1E1,2.6E1,1.09E2,2.4E1,2.8E1,1.2E1,4E0,4E0,5.1E1,1.5E1,1.1E1,7E0,1.9E1,1.6E1,4E0,4.1E1,6E1,8.4E1,5E0,6E0,5E0,4E0,3E1,4.9E1,3.8E1,9E0,4.9E1,2.2E1,7E0,6E0,1.7E1,4E0,3.7E1,6E0,4.3E1,5E0,6E0,5E0,1.3E1,1.3E1,7.3E1,3.6E1,1.8E1,6E0,1.4E1,1.4E1,8E0,4E0,2E1,3.1E1,1E1,5E0,6E0,5E0,4E0,1.5E1,1E1,6E0,3.6E1,5E0,5.4E1,6E0,3E1,5.4E1,2.5E1,5E0,4.5E1,4E0,2.9E1,9E0,4E0,5E0,4.5E1,4E0,5E0,1.7E1,1.1E1,2.6E1,2.8E1,1.5E1,1.9E1,5.4E1,1E1,2.6E1,1.4E1,4E0,1E1,4E0,1E1,4E0,4E0,4E0,1.4E1,6E0,2.3E1,8E0,5E0,5E0,9E0,6E0,2.6E1,1E1,3.7E1,1.7E1,2.5E1,5E0,5E0,4.9E1,1.5E1,1E1,3E1,1.5E1,7E0,2.2E1,3.2E1,1.3E1,5E0,1.2E1,5E0,6E0,1.7E1,9E0,7E0,2.1E1,1E1,5E0,4E0,1.5E1,1.1E1,4.3E1,4E0,6E0,8E0,1.8E1,4E0,1E1,4E0,1.9E1,1.9E1,7E0,6E0,4E0,2.1E1,1.6E1,1.3E1,4E0,1.7E1,8E0,4.1E1,8E0,6E0,9E0,5E0,5E0,2E1,1E1,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"251","size_leaf_vector":"1"}},{"base_weights":[-2.99391E-3,-8.831429E-2,2.4912943E-1,-2.6406372E-1,-1.5137249E-2,6.127956E-1,2.1382841E-1,-3.5943136E-1,-2.1115749E-1,-9.3182884E-2,4.57518E-2,3.1338148E-2,1.6913528E-2,-2.1640071E-1,2.4496906E-1,-2.95355E-1,-4.3520728E-1,-4.5271154E-2,-2.502184E-1,-1.7284678E-1,-4.950818E-2,-7.196651E-2,8.058637E-2,-1.0896794E-3,-1.2654556E-2,1.4805791E-1,3.2795694E-1,-4.214891E-3,-3.1222135E-1,-5.516985E-1,-2.7941078E-1,-2.5595522E-1,7.2868645E-2,-2.76405E-1,1.2859181E-2,1.576933E-2,-2.2300647E-1,-1.21104665E-1,3.9490927E-3,-3.0619962E-2,-2.7614754E-1,6.265724E-2,2.4083194E-1,9.1958314E-2,2.658024E-1,4.0736428E-1,2.054631E-1,-1.0115258E-2,-1.6603213E-2,-1.9671071E-2,-3.2649584E-2,-3.5516988E-3,-1.9103112E-2,-1.4849484E-2,-5.225477E-3,1.8012885E-2,-3.0757098E-2,-2.4766478E-1,-4.3157107E-1,1.0021293E-2,-5.3721853E-3,-6.6432714E-2,3.7694715E-3,-1.8739763E-1,-2.1558696E-2,-1.5851676E-1,4.5616136E-4,-1.1833065E-2,9.114149E-3,8.394522E-3,-1.2428759E-1,-6.4939214E-3,-1.507793E-2,1.2589181E-2,1.1591291E-1,3.2262672E-3,3.0713496E-1,-1.8852325E-2,1.2636074E-1,3.2381943E-1,-6.2581357E-3,3.4137793E-2,3.666432E-1,2.4937008E-1,-4.614155E-3,-8.232715E-2,2.1855973E-3,-2.0621872E-1,-3.7024492E-1,-8.3336495E-3,-5.002674E-1,-5.276657E-3,-2.4777374E-4,-1.8875582E-2,-1.6470881E-1,-1.4460988E-2,-1.3912079E-1,-4.388345E-2,5.3428244E-3,1.0040181E-2,-1.6188158E-1,2.9081505E-2,-5.888991E-3,-1.8223038E-1,1.0149796E-3,2.704253E-2,-1.4224796E-2,1.3864143E-1,-2.0600121E-2,9.547849E-3,1.7254243E-2,3.7968703E-2,2.1366796E-1,3.6912245E-1,5.6471373E-3,1.1089947E-1,3.9944065E-1,1.7773433E-1,2.3478601E-2,-1.4606287E-4,-7.4315965E-3,-2.5133523E-1,-8.815539E-2,-8.5149985E-3,-4.116537E-1,-3.2499995E-2,-1.4355317E-2,-8.8783694E-5,-1.8382254E-1,-4.267908E-4,-1.574555E-1,-3.9355992E-3,1.8129069E-3,-5.0451376E-2,7.8654446E-2,-1.37755E-2,2.8323957E-3,-1.898887E-2,8.068132E-2,-3.8668283E-3,-1.0012353E-2,4.0165115E-2,-9.568277E-3,2.094816E-1,5.4692373E-2,4.9777194E-3,-8.8733844E-2,9.831434E-2,-9.091543E-2,2.5220698E-1,-3.1418859E-3,2.0643573E-2,2.5416845E-1,-2.5801815E-3,1.1802392E-2,3.1811407E-1,4.6263516E-1,7.929093E-2,1.1666393E-2,-6.084328E-3,-2.6380596E-1,1.1817312E-3,-1.1485079E-1,-2.1868676E-2,-1.1414294E-2,-2.099745E-1,-6.963592E-2,-1.1265667E-1,-1.863606E-1,-1.1348648E-1,2.8509617E-2,1.24131896E-1,6.7105573E-3,4.246519E-2,-4.8807436E-3,1.2765835E-1,6.033308E-6,3.4689752E-3,1.201243E-1,1.6168655E-1,1.4664508E-2,7.527092E-2,-5.394145E-3,-6.124588E-4,-6.9031436E-3,5.178071E-2,1.7528737E-1,-7.75603E-4,-8.839531E-3,4.5169266E-3,2.8536078E-1,1.4764176E-2,7.474719E-3,8.617172E-3,1.5907837E-2,1.2407149E-2,2.2620369E-2,-9.172108E-3,7.343055E-3,-1.341973E-2,-8.324688E-3,-6.6140885E-3,-5.8214075E-4,-5.578171E-3,-1.2115863E-2,-6.068335E-4,-5.2912077E-3,-6.5090233E-3,-2.5052743E-4,-5.3579533E-3,-9.900675E-3,-2.3328223E-4,-6.28536E-3,-6.032966E-4,2.768627E-3,2.5625895E-3,8.164715E-3,-1.3723923E-3,3.949327E-3,-2.6384208E-3,5.1668244E-3,3.1676488E-3,8.099799E-3,9.294547E-4,-1.048663E-2,4.782991E-4,7.733759E-3,1.066699E-2,4.6780817E-3,5.370828E-3,-7.2246086E-4,3.9856415E-3,-4.5347452E-4,1.1775024E-2,2.8015731E-3,9.146807E-3,1.6794136E-2,-2.6498754E-3,1.8871386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,87,89,-1,-1,91,-1,93,-1,95,97,99,-1,101,103,-1,-1,105,107,-1,109,-1,111,113,-1,-1,115,117,-1,119,-1,121,123,-1,125,-1,-1,-1,127,-1,129,131,-1,133,135,137,-1,139,-1,141,-1,143,145,-1,-1,147,149,151,-1,153,155,157,-1,-1,-1,159,161,-1,163,-1,-1,-1,165,-1,167,-1,-1,169,171,-1,-1,173,175,-1,-1,177,-1,179,181,-1,183,185,187,189,-1,-1,191,-1,-1,193,195,197,-1,-1,199,-1,201,-1,-1,203,205,207,209,211,213,215,217,219,-1,221,-1,223,225,227,-1,229,-1,-1,-1,231,233,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,237,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2630335E1,1.0104609E1,3.2838192E0,1.0896244E0,2.6464446E0,1.2349701E-1,3.322029E0,2.6475143E-1,9.67103E-1,8.4039116E-1,1.2893082E0,0E0,0E0,1.8205786E-1,1.7851229E0,1.3332558E-1,4.8106194E-1,7.653563E-1,8.440256E-1,8.171563E-1,6.090274E-1,5.993813E-1,6.8019164E-1,0E0,0E0,6.823368E-1,1.1029844E0,0E0,1.1046934E-1,6.993055E-2,4.298731E-1,6.6516876E-2,6.690387E-1,3.8812065E-1,3.4145224E-1,1.0822474E-1,5.025053E-1,3.0932593E-1,2.830429E-1,2.2396141E-1,3.583449E-2,5.824842E-1,2.5065565E-1,1.3245192E0,8.3538723E-1,7.146969E-1,6.9523096E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.8308115E-2,4.1137314E-1,1.858468E-1,0E0,0E0,2.5650207E-2,0E0,2.3881245E-1,0E0,1.1227834E-1,9.1903515E-2,2.8388146E-1,0E0,1.2737457E-1,1.5654552E-1,0E0,0E0,5.3639454E-1,3.3174634E-1,0E0,3.6121607E-2,0E0,5.3209555E-1,2.288518E-1,0E0,0E0,5.176811E-1,7.2551584E-1,0E0,6.1662704E-2,0E0,3.7201214E-1,1.0121131E-1,0E0,2.4360752E-1,0E0,0E0,0E0,1.770004E-1,0E0,1.12850785E-1,4.79773E-2,0E0,3.194126E-1,3.5289887E-1,9.826852E-2,0E0,2.604106E-2,0E0,3.4631383E-1,0E0,5.297166E-1,1.494748E-1,0E0,0E0,2.8893167E-1,3.970946E-1,1.0084152E-1,0E0,2.3122764E-1,1.5605354E-1,2.4503076E-1,0E0,0E0,0E0,4.4331312E-2,6.973393E-2,0E0,6.584978E-2,0E0,0E0,0E0,1.3767469E-1,0E0,3.3391714E-2,0E0,0E0,2.0730536E-1,1.1695267E-1,0E0,0E0,1.16320334E-1,7.1879975E-2,0E0,0E0,3.0767363E-1,0E0,2.0146489E-1,1.6050184E-1,0E0,4.8405014E-2,8.0469385E-2,8.611426E-2,1.3082421E-1,0E0,0E0,2.0800054E-2,0E0,0E0,3.5827875E-2,3.0908108E-2,1.2124563E-1,0E0,0E0,5.2469492E-2,0E0,5.0308466E-2,0E0,0E0,1.5959918E-1,2.6398048E-2,5.6761548E-2,2.235037E-2,5.5586487E-2,2.5596896E-2,6.96502E-2,4.5569938E-2,9.6188895E-2,0E0,2.1779373E-2,0E0,2.8466728E-1,1.6952386E-1,1.271683E-1,0E0,1.471216E-1,0E0,0E0,0E0,3.6962986E-2,6.666455E-2,0E0,0E0,0E0,9.8748684E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.4035225E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,56,56,57,57,58,58,61,61,63,63,65,65,66,66,67,67,69,69,70,70,73,73,74,74,76,76,78,78,79,79,82,82,83,83,85,85,87,87,88,88,90,90,94,94,96,96,97,97,99,99,100,100,101,101,103,103,105,105,107,107,108,108,111,111,112,112,113,113,115,115,116,116,117,117,121,121,122,122,124,124,128,128,130,130,133,133,134,134,137,137,138,138,141,141,143,143,144,144,146,146,147,147,148,148,149,149,152,152,155,155,156,156,157,157,160,160,162,162,165,165,166,166,167,167,168,168,169,169,170,170,171,171,172,172,173,173,175,175,177,177,178,178,179,179,181,181,185,185,186,186,190,190,197,197],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,88,90,-1,-1,92,-1,94,-1,96,98,100,-1,102,104,-1,-1,106,108,-1,110,-1,112,114,-1,-1,116,118,-1,120,-1,122,124,-1,126,-1,-1,-1,128,-1,130,132,-1,134,136,138,-1,140,-1,142,-1,144,146,-1,-1,148,150,152,-1,154,156,158,-1,-1,-1,160,162,-1,164,-1,-1,-1,166,-1,168,-1,-1,170,172,-1,-1,174,176,-1,-1,178,-1,180,182,-1,184,186,188,190,-1,-1,192,-1,-1,194,196,198,-1,-1,200,-1,202,-1,-1,204,206,208,210,212,214,216,218,220,-1,222,-1,224,226,228,-1,230,-1,-1,-1,232,234,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,238,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,5.6E1,7.7E1,8.1E2,6.7321223E-1,9.183432E4,1.139605E6,9.935484E-1,1E0,4.539777E5,3.1338148E-2,1.6913528E-2,3.63E2,5.97E3,5E0,1E0,3.06E2,4.2210345E2,5.714286E-1,6.072304E5,1.9E1,5.405423E6,-1.0896794E-3,-1.2654556E-2,2.8530578E6,8.992806E-4,-4.214891E-3,1E0,2.3060504E7,7.042942E7,5.4E1,2.00087E5,4.6463413E0,9.566785E5,5.44E2,3.7568388E0,4.39776E6,1.9E1,2E1,2.9E1,4.5432812E2,1E0,1.4598765E0,1E0,1.0918E4,3.3571344E7,-1.0115258E-2,-1.6603213E-2,-1.9671071E-2,-3.2649584E-2,-3.5516988E-3,-1.9103112E-2,-1.4849484E-2,-5.225477E-3,1.8012885E-2,9.14E2,8.499432E5,2.49E2,1.0021293E-2,-5.3721853E-3,1.3026532E6,3.7694715E-3,1E1,-2.1558696E-2,8.2765434E1,1.185E3,3.6507E4,9.114149E-3,2.3719013E0,1.445E3,-6.4939214E-3,-1.507793E-2,2.41E3,1E0,3.2262672E-3,2.47E2,-1.8852325E-2,5.723202E5,1.6864318E11,-6.2581357E-3,3.4137793E-2,1.0322473E8,2.9622424E3,-4.614155E-3,6.2478176E7,2.1855973E-3,1.19111115E2,2E0,-8.3336495E-3,7.3E1,-5.276657E-3,-2.4777374E-4,-1.8875582E-2,1.6666666E0,-1.4460988E-2,4.1719616E5,6.07E2,5.3428244E-3,1.2463E4,3.6977E4,2.6369516E5,-5.888991E-3,3.074398E0,1.0149796E-3,1.81E2,-1.4224796E-2,1.9545455E0,1.3358E4,9.547849E-3,1.7254243E-2,4.4522205E6,9.4163445E1,4.026E3,5.6471373E-3,1.034073E6,3.7951445E6,2.234478E3,2.3478601E-2,-1.4606287E-4,-7.4315965E-3,6.48334E5,1.9166666E0,-8.5149985E-3,3.3025316E5,-3.2499995E-2,-1.4355317E-2,-8.8783694E-5,2.866353E2,-4.267908E-4,9.966666E0,-3.9355992E-3,1.8129069E-3,2.3112903E2,4.9E4,-1.37755E-2,2.8323957E-3,1.281479E5,2.3625128E6,-3.8668283E-3,-1.0012353E-2,9.750085E6,-9.568277E-3,1.11791E5,1.481649E8,4.9777194E-3,2E0,8.620714E2,1.1E1,2.8608696E1,-3.1418859E-3,2.0643573E-2,9.6732026E-1,-2.5801815E-3,1.1802392E-2,1.1074919E0,1.3586957E1,1.5E1,1.1666393E-2,-6.084328E-3,2.026453E6,1.1817312E-3,3.221198E0,-2.1868676E-2,-1.1414294E-2,1.3556826E6,1.9184E4,7.7E1,1.625E0,5.4E0,2.4E1,5.75E2,3.5834332E0,1.1497368E-2,-4.8807436E-3,3.9123364E2,6.033308E-6,9.575663E6,1.088E3,1.9197379E9,1.4664508E-2,1E0,-5.394145E-3,-6.124588E-4,-6.9031436E-3,3.0594551E1,1.7172844E7,-7.75603E-4,-8.839531E-3,4.5169266E-3,1.3748E4,1.4764176E-2,7.474719E-3,8.617172E-3,1.5907837E-2,1.2407149E-2,2.2620369E-2,5.32E2,7.343055E-3,-1.341973E-2,-8.324688E-3,-6.6140885E-3,-5.8214075E-4,-5.578171E-3,-1.2115863E-2,-6.068335E-4,-5.2912077E-3,-6.5090233E-3,-2.5052743E-4,-5.3579533E-3,-9.900675E-3,-2.3328223E-4,-6.28536E-3,-6.032966E-4,2.768627E-3,2.5625895E-3,8.164715E-3,-1.3723923E-3,3.949327E-3,-2.6384208E-3,5.1668244E-3,3.1676488E-3,8.099799E-3,9.294547E-4,-1.048663E-2,4.782991E-4,7.733759E-3,1.066699E-2,4.6780817E-3,5.370828E-3,-7.2246086E-4,3.9856415E-3,-4.5347452E-4,1.1775024E-2,2.8015731E-3,9.146807E-3,1.6794136E-2,-2.6498754E-3,1.8871386E-3],"split_indices":[2,43,0,44,2,53,43,46,68,29,43,0,0,0,2,0,26,2,67,68,43,3,43,0,0,43,72,0,24,7,60,2,5,68,60,12,68,9,3,8,0,4,29,69,6,9,66,0,0,0,0,0,0,0,0,0,2,60,2,0,0,43,0,3,0,4,44,1,0,53,2,0,0,2,6,0,0,0,43,46,0,0,5,67,0,7,0,67,8,0,0,0,0,0,69,0,60,2,0,1,12,66,0,71,0,8,0,68,9,0,0,62,73,2,0,9,43,4,0,0,0,46,68,0,62,0,0,0,67,0,73,0,0,67,12,0,0,62,60,0,0,9,0,1,12,0,8,4,3,73,0,0,68,0,0,68,73,3,0,0,9,0,68,0,0,60,1,0,68,73,0,2,69,53,0,4,0,9,44,46,0,30,0,0,0,73,60,0,0,0,44,0,0,0,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E3,7.85E2,2.65E2,2.3E2,5.55E2,2.2E1,2.43E2,8E1,1.5E2,2.43E2,3.12E2,1.6E1,6E0,1.6E1,2.27E2,4.6E1,3.4E1,2.9E1,1.21E2,8.5E1,1.58E2,7.1E1,2.41E2,4E0,1.2E1,1.06E2,1.21E2,4E0,4.2E1,1.8E1,1.6E1,1E1,1.9E1,1.1E2,1.1E1,1.8E1,6.7E1,6.7E1,9.1E1,6E1,1.1E1,2.18E2,2.3E1,7.3E1,3.3E1,7.2E1,4.9E1,1.6E1,2.6E1,1.2E1,6E0,7E0,9E0,6E0,4E0,4E0,1.5E1,9.5E1,1.5E1,4E0,7E0,8E0,1E1,6E1,7E0,5.1E1,1.6E1,8.5E1,6E0,4.3E1,1.7E1,4E0,7E0,1.13E2,1.05E2,7E0,1.6E1,4E0,6.9E1,2.9E1,4E0,6E0,6.6E1,4.3E1,6E0,9E0,6E0,7.3E1,2.2E1,4E0,1.1E1,4E0,4E0,4E0,5.6E1,4E0,4.7E1,1.2E1,4E0,7.5E1,1E1,3.8E1,5E0,1.2E1,5E0,1.09E2,4E0,9E1,1.5E1,8E0,8E0,3.5E1,3.4E1,2.3E1,6E0,8E0,5.8E1,3.5E1,8E0,5E0,4E0,5.2E1,2.1E1,5E0,1.7E1,4E0,7E0,6E0,5E1,6E0,4.1E1,8E0,4E0,4E1,3.5E1,6E0,4E0,2E1,1.8E1,4E0,8E0,1.04E2,5E0,4.8E1,4.2E1,5E0,1E1,2.4E1,1.1E1,3E1,4E0,1.2E1,1.1E1,4E0,4E0,2.8E1,3E1,1.6E1,1.9E1,6E0,4.6E1,4E0,1.7E1,1.1E1,6E0,4E1,1E1,1.8E1,2.3E1,2.2E1,1.8E1,2.1E1,1.4E1,1.2E1,8E0,1.1E1,7E0,7.2E1,3.2E1,3.5E1,1.3E1,3.8E1,4E0,5E0,5E0,1.6E1,8E0,7E0,4E0,6E0,2.4E1,5E0,6E0,6E0,2.2E1,5E0,2.5E1,8E0,8E0,3.3E1,1.3E1,1.3E1,4E0,1.6E1,2.4E1,5E0,5E0,1.4E1,4E0,8E0,1.5E1,4E0,1.8E1,8E0,1E1,1E1,1.1E1,1E1,4E0,5E0,7E0,6E0,5E0,6.8E1,4E0,1E1,2.2E1,1.5E1,2E1,2.6E1,1.2E1,1E1,6E0,4E0,4E0,1.3E1,1.1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"239","size_leaf_vector":"1"}},{"base_weights":[8.385376E-3,-8.538872E-2,2.0839147E-1,-2.301312E-1,-1.4239848E-2,-3.2906845E-1,2.3747429E-1,-3.666555E-1,-1.7427713E-1,-7.8797385E-2,6.739256E-2,-3.815999E-1,-5.852169E-3,1.3782695E-1,3.538121E-1,-2.868886E-1,-4.448873E-1,-1.8602254E-2,-1.5831576E-1,-6.749313E-2,-2.9546818E-2,4.5568712E-2,2.4685593E-1,-2.1202961E-2,-1.15376385E-2,2.5929764E-1,8.518831E-2,7.612947E-1,3.0647954E-1,-3.0748403E-1,-4.703944E-3,-2.2583077E-2,-1.0368227E-2,-3.2264754E-2,-2.0078294E-1,-1.0796738E-1,3.296368E-2,5.540572E-4,9.257787E-2,3.8242887E-3,2.7871877E-1,1.3354308E-3,2.7870327E-1,6.6096395E-2,1.9891214E-2,2.4160905E-2,4.2729717E-2,3.4492496E-1,3.5163887E-2,-1.5100135E-2,-6.533194E-3,-1.4553766E-1,2.7664334E-2,-2.3006064E-1,-9.171461E-2,-1.8874885E-1,-7.331491E-2,-2.0556806E-1,6.499129E-2,-4.755107E-2,5.7392977E-2,1.0381096E-1,-5.5044745E-3,1.6947904E-1,1.705334E-2,2.934425E-1,4.5807837E-3,8.473108E-2,-1.0542178E-2,3.176181E-1,2.441365E-2,-9.526243E-3,1.2529123E-1,-4.9157947E-4,-1.9843249E-1,1.08595295E-2,-1.8624386E-2,-1.8568096E-1,-3.2673618E-1,3.4390914E-3,-1.4528327E-1,-1.2332242E-1,-2.2360787E-1,-2.8357996E-2,-1.4377968E-1,-5.0849398E-3,-1.2009749E-2,-4.880094E-2,8.863702E-2,-1.558001E-2,-1.5282294E-1,-2.546018E-2,9.690646E-2,1.2702686E-1,-2.4658425E-2,3.6861175E-3,9.977854E-3,2.113888E-1,1.599295E-2,-6.0455836E-2,1.0545327E-1,4.6006906E-1,2.690929E-1,2.036041E-1,-1.8816956E-3,-4.8382836E-3,-1.1247085E-2,-6.8514557E-3,2.563649E-2,-2.2101967E-1,-9.688479E-2,-2.7479225E-1,-2.13782E-2,-1.68762E-1,-1.8985339E-3,-1.1184934E-3,-1.7114916E-1,-1.5763903E-1,-2.7348912E-1,-4.3729156E-2,7.7849017E-3,-9.073622E-2,-2.73696E-1,-4.9986714E-3,1.3226691E-3,6.71694E-2,9.53378E-3,-8.465152E-2,2.3370089E-2,-9.30596E-3,-2.0551227E-3,-6.669334E-2,2.1946914E-3,1.2639599E-1,-3.5028288E-3,1.0315654E-1,1.0591707E-2,-8.797705E-2,4.939164E-3,1.6794829E-2,1.4658184E-1,8.65815E-2,-1.5820373E-2,1.8924217E-1,4.5217186E-2,1.019785E-2,2.3627738E-2,2.0017476E-1,3.3111122E-1,1.4480535E-2,2.450834E-3,7.2311927E-3,-2.2939708E-2,-1.0854046E-1,-2.4612446E-1,-9.645308E-6,-1.2104075E-1,-4.1258195E-3,-3.24094E-1,-1.0412486E-2,-4.6051564E-3,-1.02068875E-2,-4.148843E-3,-2.1493214E-3,-1.8824986E-1,-1.4288767E-2,-5.4772226E-3,-1.3185384E-2,-1.3305894E-1,-1.4188997E-1,-1.4728785E-2,-2.001718E-2,-1.665338E-1,9.54931E-2,2.9355342E-2,-6.8681487E-3,-4.646384E-4,-4.583179E-3,5.57374E-2,-3.7209212E-4,-5.7158275E-3,4.124596E-2,1.7703688E-1,1.1909425E-1,-1.7003579E-3,-7.177065E-4,-6.1760047E-3,1.02022635E-2,8.7373115E-2,-1.6423877E-3,8.035295E-3,2.1612059E-1,2.490472E-4,7.457233E-3,9.858267E-3,2.7113795E-1,1.2643786E-1,2.2591102E-1,4.1052514E-1,-4.1065407E-3,5.6956057E-4,-2.134921E-3,-7.4670273E-3,-1.2298635E-2,-7.2204466E-3,-6.5089376E-3,-2.2728448E-3,-1.7132578E-2,-9.439976E-3,-1.0521625E-2,-4.81459E-3,-2.1480166E-3,1.7590827E-3,-6.1685685E-5,-1.1617987E-2,-8.336323E-3,-3.0561646E-3,3.0857178E-3,-2.6728727E-3,-3.6108214E-3,-9.81382E-3,5.9812306E-4,5.0489143E-3,-4.0784515E-3,3.1945214E-3,1.0648882E-3,5.0846254E-3,5.369802E-3,-8.4117777E-4,2.8404922E-3,9.7217085E-3,6.401373E-3,-5.817069E-4,6.1227125E-3,7.2585413E-4,4.5768563E-3,1.3541119E-2,-2.2696569E-3,2.8209276E-3,1.4939151E-2,4.808647E-3,8.884772E-3,1.6050225E-3,1.2984555E-2,6.1633885E-3,2.1035686E-2,8.135332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,35,-1,37,39,-1,-1,41,43,45,47,49,-1,-1,-1,51,53,55,57,59,61,-1,63,-1,65,67,-1,-1,-1,69,71,-1,-1,73,75,77,79,81,83,85,87,89,91,93,-1,95,-1,97,-1,99,-1,101,-1,-1,103,-1,105,-1,107,109,111,-1,113,115,117,119,121,-1,-1,123,125,127,129,131,133,135,137,-1,-1,139,-1,141,143,145,147,149,-1,-1,-1,-1,151,153,155,157,-1,159,-1,-1,161,163,165,167,-1,169,171,-1,-1,173,-1,175,177,-1,-1,179,-1,181,-1,183,-1,185,-1,-1,187,189,-1,191,193,-1,-1,195,197,-1,-1,-1,199,201,203,-1,205,-1,207,-1,-1,-1,-1,-1,209,-1,-1,211,213,215,217,-1,219,221,223,-1,-1,-1,225,-1,-1,227,229,231,-1,-1,-1,-1,233,-1,-1,235,-1,-1,237,239,241,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8979494E1,7.0922275E0,5.126911E0,1.6670742E0,2.4453144E0,1.324079E-1,3.51021E0,2.7798653E-1,5.3042316E-1,1.5986202E0,7.9165006E-1,1.7974257E-2,0E0,1.0457177E0,2.4861736E0,1.0209775E-1,1.905551E-1,0E0,8.1788826E-1,1.0403669E0,0E0,3.8931125E-1,9.19435E-2,0E0,0E0,2.1548891E-1,7.495988E-1,6.2181473E-2,1.32302E0,5.347538E-2,0E0,0E0,0E0,2.755599E-1,3.4612656E-1,4.956305E-1,5.7869065E-1,2.624822E-1,2.264179E-1,0E0,1.1521256E-1,0E0,9.263992E-2,6.3901997E-1,0E0,0E0,0E0,3.9342785E-1,3.956443E-1,0E0,0E0,9.7783685E-2,2.635795E-1,3.254695E-1,2.4223363E-1,9.558189E-2,4.0457904E-1,1.8135667E-2,1.8191183E-1,1.72633E-1,1.4651519E-1,2.5963002E-1,0E0,2.4470925E-2,0E0,1.0855198E-1,0E0,3.2829434E-1,0E0,5.6744385E-1,0E0,0E0,1.7731379E-1,0E0,1.6686827E-2,0E0,1.3581477E-1,1.7941499E-1,6.358719E-2,0E0,4.1989148E-2,9.540084E-2,6.8768024E-2,2.4945322E-1,3.1972635E-1,0E0,0E0,5.8288127E-2,1.2514013E-1,1.12282336E-1,5.4234266E-2,4.8297122E-2,1.6239232E-1,1.5449905E-1,1.2603638E-1,0E0,0E0,1.340515E-1,0E0,6.101183E-1,4.658718E-1,1.8214226E-1,2.6283216E-1,1.3219628E-1,0E0,0E0,0E0,0E0,1.1803363E-1,1.0035491E-1,4.6680763E-2,1.72243E-1,0E0,4.0116966E-2,0E0,0E0,3.6854923E-2,4.9065024E-2,6.316447E-2,2.0288068E-1,0E0,1.4176857E-1,1.6726506E-1,0E0,0E0,4.860638E-2,0E0,7.012955E-2,1.1264998E-1,0E0,0E0,3.244455E-2,0E0,1.0481188E-1,0E0,1.4045393E-1,0E0,3.1033158E-2,0E0,0E0,4.6649694E-2,1.12860635E-1,0E0,1.9155884E-1,2.2849253E-1,0E0,0E0,1.6978943E-1,2.33078E-1,0E0,0E0,0E0,3.3658046E-2,2.5619522E-2,2.3350716E-2,0E0,1.5884325E-2,0E0,1.9402266E-2,0E0,0E0,0E0,0E0,0E0,1.9413322E-2,0E0,0E0,9.909747E-2,2.9603454E-1,5.230266E-2,6.011582E-2,0E0,2.3953825E-2,2.932635E-2,1.09083965E-1,0E0,0E0,0E0,3.652709E-2,0E0,0E0,5.1156364E-2,4.8802495E-2,1.4273459E-1,0E0,0E0,0E0,0E0,3.0277222E-2,0E0,0E0,2.7733946E-1,0E0,0E0,1.358888E-1,1.3211071E-1,1.20173424E-1,5.94247E-2,1.4038372E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,33,33,34,34,35,35,36,36,37,37,38,38,40,40,42,42,43,43,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,65,65,67,67,69,69,72,72,74,74,76,76,77,77,78,78,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,93,93,94,94,97,97,99,99,100,100,101,101,102,102,103,103,108,108,109,109,110,110,111,111,113,113,116,116,117,117,118,118,119,119,121,121,122,122,125,125,127,127,128,128,131,131,133,133,135,135,137,137,140,140,141,141,143,143,144,144,147,147,148,148,152,152,153,153,154,154,156,156,158,158,164,164,167,167,168,168,169,169,170,170,172,172,173,173,174,174,178,178,181,181,182,182,183,183,188,188,191,191,194,194,195,195,196,196,197,197,198,198],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,36,-1,38,40,-1,-1,42,44,46,48,50,-1,-1,-1,52,54,56,58,60,62,-1,64,-1,66,68,-1,-1,-1,70,72,-1,-1,74,76,78,80,82,84,86,88,90,92,94,-1,96,-1,98,-1,100,-1,102,-1,-1,104,-1,106,-1,108,110,112,-1,114,116,118,120,122,-1,-1,124,126,128,130,132,134,136,138,-1,-1,140,-1,142,144,146,148,150,-1,-1,-1,-1,152,154,156,158,-1,160,-1,-1,162,164,166,168,-1,170,172,-1,-1,174,-1,176,178,-1,-1,180,-1,182,-1,184,-1,186,-1,-1,188,190,-1,192,194,-1,-1,196,198,-1,-1,-1,200,202,204,-1,206,-1,208,-1,-1,-1,-1,-1,210,-1,-1,212,214,216,218,-1,220,222,224,-1,-1,-1,226,-1,-1,228,230,232,-1,-1,-1,-1,234,-1,-1,236,-1,-1,238,240,242,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.194E3,2.0976269E5,1E0,7.7E1,9.31E2,1.02006085E-1,1.087521E3,1.139605E6,3.07E2,5.433518E0,3.7951445E6,2E1,-5.852169E-3,8E0,2.670351E6,2.0376764E-7,7.7983914E4,-1.8602254E-2,1.4339622E0,2.0766992E6,-2.9546818E-2,2.437372E6,5.2692295E6,-2.1202961E-2,-1.15376385E-2,1.6869704E6,3.3778846E0,2.480065E6,2.3876712E7,3.1530054E0,-4.703944E-3,-2.2583077E-2,-1.0368227E-2,1.20267E5,8.57E2,4.33E2,1.1127184E-5,4.4970587E2,1E0,3.8242887E-3,7.82261E6,1.3354308E-3,2.8827406E5,1.0428572E1,1.9891214E-2,2.4160905E-2,4.2729717E-2,6.701152E0,2.9925186E-2,-1.5100135E-2,-6.533194E-3,5.445545E-1,1.4806053E4,7.485264E5,9.237895E0,1.6923077E0,4.066351E0,5E0,4.92E2,3E0,1E0,2.9789968E1,-5.5044745E-3,3.5364E4,1.705334E-2,5.691698E2,4.5807837E-3,1.4598765E0,-1.0542178E-2,1.0717949E1,2.441365E-2,-9.526243E-3,6.747343E8,-4.9157947E-4,1.0795E4,1.08595295E-2,3.06E2,1.7915686E2,1.3999298E5,3.4390914E-3,1.363176E7,2.8E1,1.7277137E4,1.97525E1,1.9197379E9,-5.0849398E-3,-1.2009749E-2,4.1973075E6,6.2521395E6,2.5084135E6,1.1362371E6,4.285714E0,3.7241504E0,7.955228E9,3.6966666E2,3.6861175E-3,9.977854E-3,1.2375429E8,1.599295E-2,1.9495455E6,4.159558E0,3.0238653E8,1.6466942E3,6.072304E5,-1.8816956E-3,-4.8382836E-3,-1.1247085E-2,-6.8514557E-3,9.384615E0,1.00019E5,2.11429E5,1.26E2,-2.13782E-2,6.829817E4,-1.8985339E-3,-1.1184934E-3,6.5998E4,2.6845297E5,1.7907799E2,2.3695534E9,7.7849017E-3,1.345672E6,3.4402E4,-4.9986714E-3,1.3226691E-3,1.5016073E4,9.53378E-3,5.3871745E-1,1E0,-9.30596E-3,-2.0551227E-3,1.01E2,2.1946914E-3,1.98E2,-3.5028288E-3,3.8435754E8,1.0591707E-2,2.4545455E0,4.939164E-3,1.6794829E-2,4.1290735E2,9.8933E4,-1.5820373E-2,2.4911E4,3.3382E4,1.019785E-2,2.3627738E-2,4.7837E4,1.5416006E5,1.4480535E-2,2.450834E-3,7.2311927E-3,1.8711711E2,8.657E2,1.02166664E2,-9.645308E-6,6.07E2,-4.1258195E-3,2.0262831E2,-1.0412486E-2,-4.6051564E-3,-1.02068875E-2,-4.148843E-3,-2.1493214E-3,9.917037E2,-1.4288767E-2,-5.4772226E-3,6.83E2,1.5043378E0,5.2083335E0,7.8E1,-2.001718E-2,8E0,1.09333336E2,1.3478261E1,-6.8681487E-3,-4.646384E-4,-4.583179E-3,4.4091418E4,-3.7209212E-4,-5.7158275E-3,2.28E4,4.6991256E7,1E0,-1.7003579E-3,-7.177065E-4,-6.1760047E-3,1.02022635E-2,3.1993368E6,-1.6423877E-3,8.035295E-3,5.6440186E2,2.490472E-4,7.457233E-3,7.1E1,1E0,2.72E0,3.09627E5,8.847283E8,-4.1065407E-3,5.6956057E-4,-2.134921E-3,-7.4670273E-3,-1.2298635E-2,-7.2204466E-3,-6.5089376E-3,-2.2728448E-3,-1.7132578E-2,-9.439976E-3,-1.0521625E-2,-4.81459E-3,-2.1480166E-3,1.7590827E-3,-6.1685685E-5,-1.1617987E-2,-8.336323E-3,-3.0561646E-3,3.0857178E-3,-2.6728727E-3,-3.6108214E-3,-9.81382E-3,5.9812306E-4,5.0489143E-3,-4.0784515E-3,3.1945214E-3,1.0648882E-3,5.0846254E-3,5.369802E-3,-8.4117777E-4,2.8404922E-3,9.7217085E-3,6.401373E-3,-5.817069E-4,6.1227125E-3,7.2585413E-4,4.5768563E-3,1.3541119E-2,-2.2696569E-3,2.8209276E-3,1.4939151E-2,4.808647E-3,8.884772E-3,1.6050225E-3,1.2984555E-2,6.1633885E-3,2.1035686E-2,8.135332E-3],"split_indices":[2,43,17,44,2,57,67,46,1,68,43,6,0,3,66,52,48,0,68,43,0,9,60,0,0,60,57,1,60,68,0,0,0,9,2,2,57,67,79,0,43,0,48,68,0,0,0,50,72,0,0,68,43,60,71,68,69,8,2,68,29,73,0,9,0,67,0,69,0,71,0,0,47,0,9,0,2,4,43,0,9,10,66,73,46,0,0,43,43,60,43,69,68,46,67,0,0,7,0,43,69,7,67,43,0,0,0,0,73,5,9,10,0,43,0,0,12,43,67,46,0,9,9,0,0,62,0,57,29,0,0,0,0,10,0,7,0,68,0,0,70,1,0,10,9,0,0,9,48,0,0,0,4,48,67,0,2,0,67,0,0,0,0,0,48,0,0,2,69,69,10,0,8,4,73,0,0,0,48,0,0,1,7,19,0,0,0,0,43,0,0,4,0,0,8,6,69,9,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,6.88E2,3.22E2,2.26E2,4.62E2,1.6E1,3.06E2,6.4E1,1.62E2,2.58E2,2.04E2,1.2E1,4E0,1.66E2,1.4E2,3.4E1,3E1,9E0,1.53E2,2.54E2,4E0,1.83E2,2.1E1,6E0,6E0,4.9E1,1.17E2,1.3E1,1.27E2,3E1,4E0,2.4E1,6E0,3.9E1,1.14E2,1.81E2,7.3E1,9.4E1,8.9E1,4E0,1.7E1,4E0,4.5E1,1.12E2,5E0,7E0,6E0,1.11E2,1.6E1,2.6E1,4E0,1.3E1,2.6E1,8.9E1,2.5E1,5.3E1,1.28E2,8E0,6.5E1,5.1E1,4.3E1,8.5E1,4E0,9E0,8E0,4.1E1,4E0,1.06E2,6E0,9.9E1,1.2E1,4E0,1.2E1,4E0,9E0,4E0,2.2E1,6.3E1,2.6E1,6E0,1.9E1,2E1,3.3E1,7.9E1,4.9E1,4E0,4E0,1.1E1,5.4E1,4E1,1.1E1,1.4E1,2.9E1,7.2E1,1.3E1,4E0,5E0,1.8E1,2.3E1,1.3E1,9.3E1,2.3E1,7.6E1,8E0,4E0,4E0,5E0,5E0,1.7E1,4.4E1,1.9E1,2.1E1,5E0,1.5E1,4E0,7E0,1.3E1,1.6E1,1.7E1,7.4E1,5E0,3.6E1,1.3E1,6E0,5E0,4.7E1,7E0,1.4E1,2.6E1,7E0,4E0,9E0,5E0,2.5E1,4E0,6E1,1.2E1,9E0,4E0,4E0,1.4E1,9E0,4E0,3.8E1,5.5E1,5E0,1.8E1,3.8E1,3.8E1,4E0,4E0,4E0,1.3E1,9E0,3.5E1,4E0,1.5E1,5E0,1.6E1,7E0,8E0,7E0,6E0,4E0,1.2E1,1.3E1,4E0,5.6E1,1.8E1,2.1E1,1.5E1,4E0,9E0,2.6E1,2.1E1,7E0,7E0,5E0,2.1E1,5E0,4E0,1E1,1.5E1,5.4E1,6E0,4E0,5E0,5E0,9E0,4E0,5E0,3.3E1,5E0,1.2E1,4.3E1,1.8E1,2E1,1.8E1,2E1,4E0,9E0,5E0,4E0,2.7E1,8E0,1.1E1,4E0,1E1,6E0,7E0,5E0,3.4E1,2.2E1,9E0,9E0,1.3E1,8E0,5E0,1E1,4E0,5E0,4E0,2.2E1,5E0,1.6E1,1.4E1,7E0,4E0,6E0,4E0,1.1E1,4.7E1,7E0,5E0,4E0,1.4E1,1.9E1,2E1,2.3E1,1.3E1,5E0,1.1E1,9E0,1E1,8E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"247","size_leaf_vector":"1"}},{"base_weights":[4.4282814E-3,-7.089756E-2,2.4218602E-1,-2.0437197E-1,-8.261404E-3,2.8415385E-1,-8.598903E-2,-1.8150964E-1,-4.5935646E-1,-9.294011E-2,5.2365724E-2,1.0600522E-1,3.4112406E-1,-1.5499134E-1,9.440856E-3,-1.9990972E-1,1.657073E-2,-2.907893E-2,-1.5777435E-2,-2.0898332E-1,-5.925382E-2,2.0580392E-2,1.360249E-1,3.554532E-2,2.1839052E-1,4.0324628E-1,1.7871171E-1,-1.398593E-2,-2.9483128E-1,-3.4354174E-1,-1.600544E-1,-1.3231076E-1,-2.5994238E-1,-8.98192E-2,2.8414747E-2,7.378349E-2,-6.5065715E-3,8.728703E-2,2.7384022E-1,9.920785E-2,-1.16205495E-2,4.738623E-3,2.4817392E-1,4.8966122E-1,3.1450874E-1,2.5352243E-1,-4.2203434E-2,-6.1995494E-3,4.9995813E-3,-1.6558258E-2,-7.9238815E-3,-4.6093622E-1,-2.9160646E-1,2.0438117E-1,-1.8067156E-1,-7.6184296E-3,-1.7975752E-3,-1.8328145E-1,-1.4750147E-2,-1.8464176E-1,-5.0108705E-2,9.0700425E-2,-3.2459646E-2,9.518787E-2,-3.891413E-2,-1.58949E-2,7.5983666E-3,-6.7246966E-2,1.1015578E-1,1.1786991E-1,1.8202102E-2,1.3118407E-1,1.3573053E-3,-4.3895687E-3,5.8842584E-2,5.5208523E-3,1.2986845E-2,3.9819297E-1,5.9044045E-1,4.2621452E-1,2.0997998E-1,1.5474588E-1,2.6790693E-2,-6.1836056E-3,2.5626253E-3,-1.3171322E-2,-2.4725582E-2,-7.5487415E-3,-3.0701473E-1,7.7513396E-5,1.8791415E-2,-2.0266195E-1,-3.2269064E-2,-1.1578427E-2,-5.7540867E-3,-7.8102365E-2,-3.1165498E-1,-6.324879E-2,5.5606193E-3,-2.0883323E-3,1.3096264E-1,2.9990165E-3,-7.7523096E-3,1.1098491E-1,-2.3818745E-3,-7.921826E-3,4.5830626E-3,-7.313378E-2,3.0233769E-2,4.5355887E-4,-6.0457014E-3,-5.4546865E-3,1.3486981E-1,-7.790265E-5,8.612825E-3,8.616152E-3,2.9208383E-3,-1.06385596E-4,4.1441983E-3,2.0027258E-2,1.0591589E-2,1.7179692E-2,2.9448232E-2,4.8606926E-1,1.9264043E-3,1.5037008E-1,1.3138838E-2,-4.5302436E-3,9.1990195E-3,-1.5037169E-2,-8.121841E-3,-1.8360811E-1,-3.5809505E-1,2.2791203E-2,-1.2045073E-1,-1.07284226E-1,1.9668234E-3,-1.685911E-3,-4.2326507E-1,-4.998092E-2,-9.597459E-3,6.3006915E-2,9.610854E-3,-3.9417487E-2,2.7359598E-3,1.3972326E-1,5.6301165E-2,-3.9379608E-2,-1.8409701E-2,7.110269E-2,-8.992044E-2,4.881099E-3,-7.3879417E-3,9.77548E-2,1.2968812E-2,1.2038146E-2,2.3867035E-2,8.8894E-3,-1.0467523E-3,-1.5443225E-1,-2.4996966E-1,-2.0567698E-2,-1.04516465E-2,-1.8106676E-3,3.784915E-3,-7.8015993E-3,-2.2148767E-3,-1.0186115E-3,-1.4806482E-1,-3.0158894E-2,-7.6714233E-3,-6.956638E-2,2.2432987E-2,7.837078E-5,4.518229E-3,-3.0988848E-3,5.8769487E-4,7.572765E-2,1.9353291E-1,-1.0267993E-3,1.1372247E-1,-6.88002E-2,6.220237E-2,4.0727045E-2,1.3610534E-1,-7.429002E-3,-2.3357257E-2,6.493661E-2,1.459348E-1,-7.599928E-3,2.7608543E-3,-1.2748771E-2,-2.8213009E-3,-9.0355445E-3,-3.98174E-3,-2.4131336E-3,-9.389699E-3,-3.1999135E-3,3.4211902E-3,-9.884589E-4,5.171651E-3,1.1303496E-2,5.980526E-3,2.4330695E-3,-2.166809E-3,2.1575398E-3,6.714982E-3,6.1021145E-3,-4.183727E-3,7.989451E-3,2.8440674E-5,4.2089876E-3,-1.0054508E-3,-6.8957923E-4,8.829163E-3,-5.13425E-3,2.5509289E-3,4.0809037E-3,-1.1029723E-3,8.457262E-3,2.759687E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,-1,75,77,79,81,83,-1,-1,-1,-1,85,87,89,91,-1,-1,93,-1,95,97,99,101,103,105,107,-1,109,111,113,-1,115,-1,-1,117,-1,-1,119,121,123,125,127,-1,-1,-1,-1,-1,-1,129,-1,-1,131,133,-1,-1,135,137,139,-1,-1,141,143,-1,145,-1,-1,-1,147,149,-1,-1,151,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,155,-1,157,-1,-1,-1,-1,-1,159,161,163,165,167,-1,-1,169,171,-1,173,-1,175,-1,177,179,181,-1,183,185,-1,-1,187,-1,-1,-1,-1,-1,189,191,-1,-1,-1,-1,-1,-1,-1,193,-1,-1,195,197,-1,-1,-1,-1,199,201,203,205,207,209,211,213,-1,215,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8535387E1,6.5695257E0,3.4538612E0,1.379837E0,2.756598E0,2.2068024E0,6.126642E-1,2.3776798E0,1.8588877E-1,8.632474E-1,8.2950985E-1,4.277634E-1,1.6128674E0,4.6911293E-1,0E0,1.2330275E0,0E0,0E0,0E0,1.6096497E-1,4.7148627E-1,3.2905322E-1,5.537443E-1,1.0624728E-1,4.698515E-2,7.6618767E-1,8.039868E-1,2.0548818E-1,3.261125E-2,1.5823984E-1,1.374351E0,6.05582E-2,7.338679E-2,4.8025644E-1,1.7735785E-1,1.8944547E-1,2.4660249E-1,2.3711762E-1,3.6584604E-1,2.9638574E-2,1.2899071E-1,0E0,3.4207225E-2,3.0875874E-1,6.3444424E-1,1.070837E0,1.2378834E-1,0E0,0E0,0E0,0E0,1.7166376E-2,2.095294E-2,4.0952381E-1,5.449796E-1,0E0,0E0,2.8109133E-2,0E0,4.8987174E-1,2.1593298E-1,1.317706E-1,1.1568097E-1,1.5640038E-1,2.550196E-1,3.850623E-1,0E0,4.5406885E-2,1.635961E-1,9.047711E-2,0E0,2.5802672E-2,0E0,0E0,2.2850301E-2,0E0,0E0,1.0340643E-1,2.9957771E-2,6.47171E-1,1.0805762E-1,3.4482598E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7915487E-2,0E0,0E0,3.6960554E-1,1.1441724E-1,0E0,0E0,8.2040966E-2,5.066397E-1,1.5640551E-1,0E0,0E0,8.116335E-2,5.0029255E-2,0E0,8.485317E-2,0E0,0E0,0E0,6.9813865E-1,4.0415004E-1,0E0,0E0,2.0567635E-1,2.2697365E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.7575245E-2,0E0,1.5713403E-1,0E0,0E0,0E0,0E0,0E0,2.1890545E-1,7.156801E-2,5.819227E-2,2.350095E-2,5.9302866E-2,0E0,0E0,6.0008526E-1,1.1643958E-1,0E0,2.329855E-2,0E0,1.815864E-2,0E0,1.1620033E-1,7.25374E-2,1.8359569E-1,0E0,1.14901185E-1,9.532726E-2,0E0,0E0,5.218953E-2,0E0,0E0,0E0,0E0,0E0,2.1477342E-1,1.7457223E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.842454E-2,0E0,0E0,1.3767433E-1,9.033047E-2,0E0,0E0,0E0,0E0,6.922447E-2,3.386545E-2,3.202723E-2,1.6528234E-2,2.1739471E-1,9.536918E-2,1.3735428E-1,1.606979E-1,0E0,8.8898525E-2,5.4055437E-2,3.706959E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,51,51,52,52,53,53,54,54,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,71,71,74,74,77,77,78,78,79,79,80,80,81,81,88,88,91,91,92,92,95,95,96,96,97,97,100,100,101,101,103,103,107,107,108,108,111,111,112,112,123,123,125,125,131,131,132,132,133,133,134,134,135,135,138,138,139,139,141,141,143,143,145,145,146,146,147,147,149,149,150,150,153,153,159,159,160,160,168,168,171,171,172,172,177,177,178,178,179,179,180,180,181,181,182,182,183,183,184,184,186,186,187,187,188,188],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,-1,76,78,80,82,84,-1,-1,-1,-1,86,88,90,92,-1,-1,94,-1,96,98,100,102,104,106,108,-1,110,112,114,-1,116,-1,-1,118,-1,-1,120,122,124,126,128,-1,-1,-1,-1,-1,-1,130,-1,-1,132,134,-1,-1,136,138,140,-1,-1,142,144,-1,146,-1,-1,-1,148,150,-1,-1,152,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,156,-1,158,-1,-1,-1,-1,-1,160,162,164,166,168,-1,-1,170,172,-1,174,-1,176,-1,178,180,182,-1,184,186,-1,-1,188,-1,-1,-1,-1,-1,190,192,-1,-1,-1,-1,-1,-1,-1,194,-1,-1,196,198,-1,-1,-1,-1,200,202,204,206,208,210,212,214,-1,216,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.0976269E5,2E0,4.217427E7,7.9E2,7.601393E2,1.465E4,6.68411E2,1.0479E4,3.95E2,2.0766992E6,5.4051904E8,6.9E1,3.91157E5,9.440856E-3,4.9E2,1.657073E-2,-2.907893E-2,-1.5777435E-2,3.1E1,2.8045622E6,1.592E3,2.831224E10,7.6937294E-1,1.2003246E6,2.1298597E0,5.712652E8,5.071E3,7.650379E8,8.0833334E-1,4E0,1E0,1.7142857E0,1E0,3.5364E4,2.3719013E0,2.0240436E0,2.9077E4,1E0,1.6673345E7,2.89196E6,4.738623E-3,6.1236826E8,1.6975454E3,1.684E0,3.5675005E3,4.7837E4,-6.1995494E-3,4.9995813E-3,-1.6558258E-2,-7.9238815E-3,1.00019E5,2.88E0,2.123E3,1.1939979E8,-7.6184296E-3,-1.7975752E-3,2.55E2,-1.4750147E-2,2.3514E4,2E1,8.612209E-6,7.45E2,1E0,4.6712E4,1.056178E6,7.5983666E-3,3.693316E7,1.9376624E0,1.87627E5,1.8202102E-2,4.4971E4,1.3573053E-3,-4.3895687E-3,1.5822886E0,5.5208523E-3,1.2986845E-2,8E0,1.1E1,3.2952412E5,1.5E1,5.94E2,2.6790693E-2,-6.1836056E-3,2.5626253E-3,-1.3171322E-2,-2.4725582E-2,-7.5487415E-3,5.513889E0,7.7513396E-5,1.8791415E-2,3.1E1,2.3412812E0,-1.1578427E-2,-5.7540867E-3,1.3E1,1.0865825E0,5.44E2,5.5606193E-3,-2.0883323E-3,2.6732E4,5.69E2,-7.7523096E-3,8E0,-2.3818745E-3,-7.921826E-3,4.5830626E-3,8.81059E5,3.84E2,4.5355887E-4,-6.0457014E-3,2.3670635E1,2.483E3,-7.790265E-5,8.612825E-3,8.616152E-3,2.9208383E-3,-1.06385596E-4,4.1441983E-3,2.0027258E-2,1.0591589E-2,1.7179692E-2,2.9448232E-2,2.0659652E0,1.9264043E-3,1.4E1,1.3138838E-2,-4.5302436E-3,9.1990195E-3,-1.5037169E-2,-8.121841E-3,1.2E1,4.6E1,4.306E4,1.862E3,3.3272727E0,1.9668234E-3,-1.685911E-3,2.6801266E2,6.953366E7,-9.597459E-3,7.597862E6,9.610854E-3,1.4162437E0,2.7359598E-3,1.047E3,4.0119403E2,2.8E1,-1.8409701E-2,9.751103E6,4.6816683E-1,4.881099E-3,-7.3879417E-3,5.144203E2,1.2968812E-2,1.2038146E-2,2.3867035E-2,8.8894E-3,-1.0467523E-3,4.7866177E2,7.997723E6,-2.0567698E-2,-1.04516465E-2,-1.8106676E-3,3.784915E-3,-7.8015993E-3,-2.2148767E-3,-1.0186115E-3,5.0149255E0,-3.0158894E-2,-7.6714233E-3,6.763314E7,4.358214E6,7.837078E-5,4.518229E-3,-3.0988848E-3,5.8769487E-4,2.7777777E0,3.3253515E6,2.12E2,3.1706784E0,8.118812E-1,1.9545455E0,1.7787506E4,1.2E2,-7.429002E-3,4.99E2,4.498404E2,1.2183674E1,-7.599928E-3,2.7608543E-3,-1.2748771E-2,-2.8213009E-3,-9.0355445E-3,-3.98174E-3,-2.4131336E-3,-9.389699E-3,-3.1999135E-3,3.4211902E-3,-9.884589E-4,5.171651E-3,1.1303496E-2,5.980526E-3,2.4330695E-3,-2.166809E-3,2.1575398E-3,6.714982E-3,6.1021145E-3,-4.183727E-3,7.989451E-3,2.8440674E-5,4.2089876E-3,-1.0054508E-3,-6.8957923E-4,8.829163E-3,-5.13425E-3,2.5509289E-3,4.0809037E-3,-1.1029723E-3,8.457262E-3,2.759687E-3],"split_indices":[2,43,6,60,2,67,44,67,9,2,43,7,8,1,0,1,0,0,0,10,43,44,46,53,66,53,47,2,5,68,10,8,68,29,9,53,53,1,29,47,62,0,7,67,69,67,9,0,0,0,0,5,69,1,7,0,0,2,0,1,3,53,2,19,1,9,0,7,69,1,0,9,0,0,57,0,0,32,3,48,3,0,0,0,0,0,0,0,69,0,0,3,68,0,0,8,71,10,0,0,1,2,0,8,0,0,0,9,0,0,0,73,2,0,0,0,0,0,0,0,0,0,0,57,0,3,0,0,0,0,0,8,3,1,2,69,0,0,70,7,0,60,0,68,0,2,4,71,0,9,53,0,0,4,0,0,0,0,0,67,9,0,0,0,0,0,0,0,71,0,0,7,5,0,0,0,0,69,60,10,69,68,68,48,0,0,0,4,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.033E3,7.85E2,2.48E2,2.5E2,5.35E2,2.2E2,2.8E1,2.31E2,1.9E1,2.23E2,3.12E2,5.4E1,1.66E2,2.3E1,5E0,2.24E2,7E0,6E0,1.3E1,4.9E1,1.74E2,2.27E2,8.5E1,3.4E1,2E1,1.19E2,4.7E1,1.2E1,1.1E1,4.7E1,1.77E2,2.1E1,2.8E1,1.29E2,4.5E1,7.6E1,1.51E2,6.4E1,2.1E1,1.4E1,2E1,5E0,1.5E1,5.8E1,6.1E1,3.5E1,1.2E1,6E0,6E0,6E0,5E0,1.2E1,3.5E1,9E0,1.68E2,1.5E1,6E0,1.4E1,1.4E1,3.7E1,9.2E1,2.2E1,2.3E1,6.4E1,1.2E1,1.44E2,7E0,8E0,5.6E1,1E1,1.1E1,9E0,5E0,9E0,1.1E1,4E0,1.1E1,3.3E1,2.5E1,2.8E1,3.3E1,2.8E1,7E0,6E0,6E0,5E0,7E0,5E0,3E1,5E0,4E0,1.46E2,2.2E1,5E0,9E0,2.1E1,1.6E1,8.6E1,6E0,5E0,1.7E1,1.9E1,4E0,5.8E1,6E0,6E0,6E0,6.4E1,8E1,4E0,4E0,1E1,4.6E1,4E0,6E0,4E0,5E0,4E0,7E0,2.6E1,7E0,6E0,1.9E1,2.4E1,4E0,2E1,1.3E1,4E0,2.4E1,2.5E1,5E0,1.32E2,1.4E1,1.4E1,8E0,1.7E1,4E0,5E0,1.1E1,8E1,6E0,1E1,7E0,1.1E1,8E0,3.7E1,2.1E1,5.9E1,5E0,6E1,2E1,6E0,4E0,3.8E1,8E0,4E0,2E1,1.6E1,4E0,9.4E1,3.8E1,7E0,7E0,7E0,7E0,4E0,4E0,6E0,1.1E1,5E0,6E0,6.3E1,1.7E1,4E0,6E0,7E0,4E0,1.8E1,1.9E1,1.1E1,1E1,4.6E1,1.3E1,4.2E1,1.8E1,9E0,1.1E1,2.4E1,1.4E1,9E1,4E0,3.3E1,5E0,5E0,6E0,5.7E1,6E0,6E0,1.1E1,5E0,1.3E1,9E0,1E1,5E0,6E0,4E0,6E0,4E0,4.2E1,4E0,9E0,2.3E1,1.9E1,5E0,1.3E1,5E0,6E0,1.9E1,5E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[2.5721124E-4,-6.941781E-2,2.1714778E-1,-2.0841785E-1,-1.1861273E-2,2.6116636E-1,-3.2985624E-2,-1.874136E-1,-4.045349E-1,-1.5475622E-1,1.889853E-2,2.1016075E-1,4.4465017E-1,-1.5201495E-2,7.024072E-2,-2.6033053E-1,-1.2859589E-1,-2.1156715E-2,-8.994043E-3,-1.3139848E-1,-2.250427E-2,2.5277276E-2,-1.5591447E-2,2.643263E-1,8.177779E-2,5.2652365E-1,2.0261356E-1,1.6935204E-1,-8.427964E-2,-2.9067558E-1,-1.4533873E-1,1.6115719E-1,-1.5745144E-1,-1.7316738E-1,-8.1896804E-2,-5.4016E-3,9.334349E-2,1.1552466E-1,3.166208E-1,4.711955E-2,1.3531619E-2,1.5834536E-2,6.438996E-1,3.1449094E-3,1.1773628E-2,3.99937E-2,1.5063236E-2,2.5682794E-3,-7.4456334E-3,-3.2147348E-1,-2.2857998E-1,-6.641187E-2,-2.1105298E-1,3.803326E-3,1.1055024E-2,-1.277601E-1,-3.1648016E-1,-1.4948812E-1,-1.2743434E-2,-2.2059928E-3,-1.08393565E-1,-6.1240386E-2,4.302475E-2,1.03435725E-1,-9.045641E-3,1.8546598E-1,-5.427583E-3,4.648768E-1,2.517334E-1,6.544048E-2,-5.1990584E-3,3.3385865E-2,1.8741027E-2,-6.1183185E-3,7.2737103E-3,-1.8258354E-1,-1.601116E-2,-4.6916767E-3,-2.5283355E-1,-8.951615E-4,-4.7298083E-3,-1.2539495E-2,-5.336325E-3,-1.424571E-1,3.1856492E-3,-1.9143803E-2,-6.0713263E-3,-2.0149945E-1,-1.1628604E-1,2.1152708E-3,-2.6716453E-3,-1.2655258E-1,8.596487E-4,-2.5652793E-1,-4.3023027E-2,6.0954146E-2,-9.001108E-2,3.0621938E-2,1.603076E-1,-1.2320047E-4,2.3263249E-1,2.3992082E-2,1.4339581E-2,1.7017592E-1,3.059089E-1,-2.7941069E-2,1.0504462E-1,-1.1340461E-2,-3.842874E-3,-2.7481338E-1,-5.688928E-3,-1.0213446E-1,-2.1679962E-1,-3.421872E-3,-1.1138808E-2,-3.179752E-3,-1.3882832E-1,-8.746116E-2,-7.4929795E-3,-3.431194E-3,-1.7375572E-2,-1.5040724E-2,-3.1650174E-2,7.369837E-2,-5.5258647E-2,-1.3014556E-1,2.6066182E-3,-1.0831684E-1,6.0576033E-2,1.7547688E-1,-2.0043545E-3,4.7190785E-3,1.2993889E-2,2.186632E-1,6.588124E-2,3.39173E-1,5.5275876E-3,2.6303676E-3,-6.3040005E-3,9.76267E-3,7.409952E-2,-8.616283E-3,-1.5098357E-2,-1.1394966E-1,7.986102E-4,-4.629351E-3,-2.4934334E-1,-7.442143E-3,-2.2514497E-3,-5.6348983E-3,-1.1858534E-3,-4.725372E-2,9.8782554E-2,1.3226067E-2,1.069804E-1,4.2094857E-2,-9.62348E-3,-2.9536115E-3,-8.063637E-3,1.2176153E-3,-1.2713758E-2,8.9630626E-2,-6.4605415E-2,1.0361151E-1,2.2378138E-1,1.197937E-2,5.4405904E-3,9.944889E-3,-5.8465656E-3,3.6956805E-1,5.233609E-3,9.348919E-2,6.3083536E-4,-2.6966045E-3,-6.9168294E-3,-1.2859753E-2,-5.801923E-3,-1.6738634E-3,-1.1154813E-2,8.751863E-3,3.2852514E-4,2.3848298E-3,-1.5936765E-3,5.5873496E-3,-4.5351812E-4,-2.6152588E-3,5.4206196E-3,2.7820508E-3,7.192777E-3,-8.781809E-3,2.34366E-3,2.8736636E-3,9.688542E-3,5.5319746E-3,1.1517445E-2,1.0970537E-2,1.8744828E-2,7.1072887E-4,5.65228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,-1,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,-1,73,-1,-1,-1,75,77,79,81,-1,-1,83,85,87,-1,89,91,93,95,97,-1,99,-1,101,103,105,-1,-1,-1,-1,-1,107,-1,-1,109,-1,-1,-1,-1,111,-1,-1,-1,113,115,-1,-1,117,-1,119,121,123,125,127,129,-1,131,-1,-1,133,135,137,139,-1,-1,141,-1,143,145,-1,-1,-1,147,149,-1,-1,-1,-1,151,153,155,157,-1,159,161,163,-1,-1,-1,165,167,169,-1,-1,-1,-1,171,-1,-1,173,-1,-1,175,-1,-1,-1,-1,177,179,181,183,185,-1,-1,-1,-1,-1,187,189,191,193,-1,-1,-1,-1,195,-1,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5414054E1,6.1738076E0,2.7392235E0,8.5514355E-1,2.4072196E0,1.881237E0,1.184429E0,8.500166E-1,1.4623117E-1,6.890199E-1,1.0337775E0,1.1396914E0,7.850561E-1,0E0,4.5839968E-1,2.7529144E-1,9.952483E-1,0E0,0E0,1.7603993E-1,0E0,9.276626E-1,0E0,8.738594E-1,3.593689E-1,4.911518E-1,7.400721E-2,3.5182512E-1,1.3800666E-1,6.229925E-2,9.4902545E-2,4.8054814E-2,4.6007466E-1,7.78923E-2,9.266901E-2,8.327839E-1,4.229468E-1,5.432877E-1,7.102909E-1,1.389912E-1,0E0,0E0,6.333256E-2,0E0,0E0,2.4361694E-1,0E0,0E0,0E0,9.357071E-2,5.750668E-2,1.6625367E-2,3.2097965E-2,0E0,0E0,2.729149E-1,2.3541677E-1,4.9097538E-2,0E0,3.468129E-2,7.845992E-2,4.977035E-1,3.9905316E-1,5.499928E-1,0E0,2.224595E-1,0E0,4.9082756E-2,2.218833E-1,1.5456519E-1,0E0,0E0,0E0,0E0,0E0,3.5841018E-2,0E0,0E0,2.887392E-2,0E0,0E0,0E0,0E0,2.3266053E-1,0E0,0E0,0E0,5.784613E-2,2.0400047E-2,0E0,0E0,2.6214063E-2,0E0,2.3358792E-1,4.1385034E-1,2.2030461E-1,1.2488291E-1,2.562685E-1,2.3935354E-1,0E0,9.817135E-2,0E0,0E0,1.221323E-1,1.8246865E-1,1.2750067E-1,7.9711616E-2,0E0,0E0,2.1702766E-2,0E0,8.239561E-2,8.505356E-2,0E0,0E0,0E0,2.8249264E-2,3.0987598E-2,0E0,0E0,0E0,0E0,2.6517516E-1,2.6425362E-1,2.3226213E-1,3.5874456E-2,0E0,2.5446662E-1,1.8764544E-1,2.1981311E-1,0E0,0E0,0E0,4.3266058E-2,3.1469896E-1,1.6411138E-1,0E0,0E0,0E0,0E0,2.685155E-2,0E0,0E0,9.463805E-2,0E0,0E0,4.403293E-2,0E0,0E0,0E0,0E0,2.3740658E-1,1.1500035E-1,8.972961E-2,1.3978243E-1,8.090342E-2,0E0,0E0,0E0,0E0,0E0,7.028574E-2,1.543665E-1,1.1876455E-1,7.900262E-2,0E0,0E0,0E0,0E0,2.7895212E-2,0E0,3.868772E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,42,42,45,45,49,49,50,50,51,51,52,52,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,65,65,67,67,68,68,69,69,75,75,78,78,83,83,87,87,88,88,91,91,93,93,94,94,95,95,96,96,97,97,98,98,100,100,103,103,104,104,105,105,106,106,109,109,111,111,112,112,116,116,117,117,122,122,123,123,124,124,125,125,127,127,128,128,129,129,133,133,134,134,135,135,140,140,143,143,146,146,151,151,152,152,153,153,154,154,155,155,161,161,162,162,163,163,164,164,169,169,171,171],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,-1,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,-1,74,-1,-1,-1,76,78,80,82,-1,-1,84,86,88,-1,90,92,94,96,98,-1,100,-1,102,104,106,-1,-1,-1,-1,-1,108,-1,-1,110,-1,-1,-1,-1,112,-1,-1,-1,114,116,-1,-1,118,-1,120,122,124,126,128,130,-1,132,-1,-1,134,136,138,140,-1,-1,142,-1,144,146,-1,-1,-1,148,150,-1,-1,-1,-1,152,154,156,158,-1,160,162,164,-1,-1,-1,166,168,170,-1,-1,-1,-1,172,-1,-1,174,-1,-1,176,-1,-1,-1,-1,178,180,182,184,186,-1,-1,-1,-1,-1,188,190,192,194,-1,-1,-1,-1,196,-1,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.3876712E7,4.217427E7,4.97E2,1.4176E4,1.4598765E0,2.71E2,1.6074808E5,1E0,4.4E1,8E0,1.2792593E2,-1.5201495E-2,7.7175174E8,1.9E1,1.1E1,-2.1156715E-2,-8.994043E-3,3.56E2,-2.250427E-2,2.0766992E6,-1.5591447E-2,6.9664386E2,2.8016653E3,1.9705E4,4.988E3,6.355E3,3.592428E7,1.16E2,1E0,1.07882355E2,3.6349E4,1.9530877E5,1.2051282E0,1.212945E6,1.0089981E3,4.75356E6,1.5282027E5,1.6677892E2,1.3531619E-2,1.5834536E-2,1.61E9,3.1449094E-3,1.1773628E-2,1.7431098E8,1.5063236E-2,2.5682794E-3,-7.4456334E-3,1.863E3,2.0395E4,3.83E2,6.44E2,3.803326E-3,1.1055024E-2,5.185192E2,1.775373E1,1.0538846E3,-1.2743434E-2,4.33E2,1E0,2.83408E5,7.1E1,4.6066E4,-9.045641E-3,1.02E2,-5.427583E-3,1.2909952E1,5.831829E6,1.0901037E10,-5.1990584E-3,3.3385865E-2,1.8741027E-2,-6.1183185E-3,7.2737103E-3,1.1E1,-1.601116E-2,-4.6916767E-3,3.328629E6,-8.951615E-4,-4.7298083E-3,-1.2539495E-2,-5.336325E-3,1.7E1,3.1856492E-3,-1.9143803E-2,-6.0713263E-3,2.4636364E1,3.806E3,2.1152708E-3,-2.6716453E-3,9.695652E0,8.596487E-4,4.5801528E-2,7E0,1E0,2.0316172E6,2.2007043E0,2.5345264E7,-1.2320047E-4,1.9376624E0,2.3992082E-2,1.4339581E-2,1.934E3,2.6912618E7,1.2473E4,1.0092541E8,-1.1340461E-2,-3.842874E-3,2.1E1,-5.688928E-3,2.0263722E8,5.236559E0,-3.421872E-3,-1.1138808E-2,-3.179752E-3,2E0,1.2123E4,-7.4929795E-3,-3.431194E-3,-1.7375572E-2,-1.5040724E-2,4.888E3,8.69E2,5.4007E7,1.4E1,2.6066182E-3,1E0,1.4E1,2.485E2,-2.0043545E-3,4.7190785E-3,1.2993889E-2,3.7951445E6,1.8667632E0,2.057143E0,5.5275876E-3,2.6303676E-3,-6.3040005E-3,9.76267E-3,1.213E5,-8.616283E-3,-1.5098357E-2,2.7777777E0,7.986102E-4,-4.629351E-3,8.43E2,-7.442143E-3,-2.2514497E-3,-5.6348983E-3,-1.1858534E-3,3.607317E1,2.6595745E0,1E0,3.471E3,1.14E3,-9.62348E-3,-2.9536115E-3,-8.063637E-3,1.2176153E-3,-1.2713758E-2,6.7723384E7,2.8266037E2,6.887749E7,1.0029973E8,1.197937E-2,5.4405904E-3,9.944889E-3,-5.8465656E-3,1.0775862E0,5.233609E-3,2.8802464E8,6.3083536E-4,-2.6966045E-3,-6.9168294E-3,-1.2859753E-2,-5.801923E-3,-1.6738634E-3,-1.1154813E-2,8.751863E-3,3.2852514E-4,2.3848298E-3,-1.5936765E-3,5.5873496E-3,-4.5351812E-4,-2.6152588E-3,5.4206196E-3,2.7820508E-3,7.192777E-3,-8.781809E-3,2.34366E-3,2.8736636E-3,9.688542E-3,5.5319746E-3,1.1517445E-2,1.0970537E-2,1.8744828E-2,7.1072887E-4,5.65228E-3],"split_indices":[2,43,60,60,2,2,69,2,48,113,3,32,73,0,7,3,10,0,0,2,0,43,0,67,4,2,44,2,60,44,20,67,44,66,68,9,4,62,48,73,0,0,5,0,0,5,0,0,0,9,9,12,1,0,0,67,71,48,0,2,28,7,8,1,0,0,0,71,60,46,0,0,0,0,0,5,0,0,9,0,0,0,0,3,0,0,0,67,1,0,0,73,0,73,3,19,62,69,60,0,69,0,0,44,62,44,7,0,0,0,0,7,69,0,0,0,32,1,0,0,0,0,10,2,7,3,0,8,8,67,0,0,0,43,57,68,0,0,0,0,10,0,0,71,0,0,10,0,0,0,0,73,68,26,10,2,0,0,0,0,0,7,4,7,5,0,0,0,0,68,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.71E2,2.47E2,2.25E2,5.46E2,2.1E2,3.7E1,2.05E2,2E1,9.6E1,4.5E2,1.66E2,4.4E1,9E0,2.8E1,9E1,1.15E2,1.5E1,5E0,9.1E1,5E0,4.43E2,7E0,1.16E2,5E1,3.2E1,1.2E1,1.7E1,1.1E1,7E1,2E1,1E1,1.05E2,4.8E1,4.3E1,3.06E2,1.37E2,3.1E1,8.5E1,4.4E1,6E0,1.4E1,1.8E1,4E0,8E0,1E1,7E0,4E0,7E0,4.4E1,2.6E1,1E1,1E1,6E0,4E0,9E1,1.5E1,4.1E1,7E0,1.1E1,3.2E1,1.42E2,1.64E2,1.33E2,4E0,2.4E1,7E0,2.4E1,6.1E1,4E1,4E0,1.2E1,6E0,4E0,6E0,8E0,3.6E1,5E0,2.1E1,5E0,5E0,5E0,5E0,8.4E1,6E0,9E0,6E0,1.4E1,2.7E1,6E0,5E0,2.8E1,4E0,1.1E1,1.31E2,1.45E2,1.9E1,5.9E1,7.4E1,5E0,1.9E1,1.6E1,8E0,2.6E1,3.5E1,1.2E1,2.8E1,4E0,4E0,1.7E1,4E0,5.6E1,2.8E1,4E0,1E1,1E1,1.7E1,1.5E1,1.3E1,5E0,6E0,4E0,1.27E2,1.31E2,1.4E1,1.5E1,4E0,1E1,4.9E1,6.9E1,5E0,6E0,1.3E1,1.7E1,9E0,2.9E1,6E0,7E0,5E0,5E0,2.3E1,8E0,9E0,5.1E1,5E0,7E0,2.1E1,1.3E1,4E0,9E0,6E0,1.14E2,1.3E1,4.7E1,8.4E1,9E0,5E0,7E0,8E0,6E0,4E0,4E1,9E0,2.9E1,4E1,1.1E1,6E0,5E0,4E0,2.5E1,4E0,1.7E1,6E0,2.1E1,3E1,1.6E1,5E0,1.09E2,5E0,6E0,7E0,2.6E1,2.1E1,7.5E1,9E0,4E0,5E0,2.9E1,1.1E1,4E0,5E0,2.2E1,7E0,9E0,3.1E1,7E0,1.8E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[-5.3168205E-3,-7.74734E-2,2.0169191E-1,-2.1571895E-1,-1.1063464E-2,1.3282856E-1,3.6131305E-1,-1.9371118E-1,-3.9438784E-1,-1.2515052E-1,2.0699512E-2,9.7009264E-2,5.8387464E-1,3.8022965E-1,-2.7943772E-4,-3.0554298E-1,-1.5399313E-1,-1.7895313E-1,-5.313147E-1,-1.519736E-1,3.9994195E-2,-6.532466E-2,4.1314524E-2,1.2865941E-1,-2.1402809E-1,3.1359896E-2,1.2808675E-2,4.9003658E-1,3.2540062E-1,-2.1329568E-1,-3.4083948E-1,1.1649526E-1,-1.7514001E-1,-1.3769596E-2,-2.3702849E-3,-3.6308873E-2,-2.9995766E-1,-1.2228492E-1,-2.4604942E-1,-2.210123E-2,4.511144E-3,-1.6732469E-1,-1.944176E-2,3.3107933E-2,1.5556166E-2,4.0220708E-1,1.0441745E-1,-1.5473763E-2,-6.933846E-2,1.6368799E-2,2.7545583E-2,2.344124E-1,3.8828143E-1,-1.144036E-2,-3.1243395E-3,-1.645332E-2,-7.3663923E-3,-4.727472E-3,2.2512001E-1,-1.5317227E-1,-3.496251E-1,-7.40218E-3,-1.6927782E-2,-1.6040571E-1,-8.918237E-2,-2.9165658E-1,-1.0871704E-3,2.1256988E-3,-3.9635994E-3,-6.320663E-2,-2.9409254E-1,-9.239928E-3,5.7223067E-3,1.0242157E-1,2.3927146E-3,2.2257049E-2,1.0584036E-2,-6.5492444E-2,1.2629864E-1,-4.2622528E-4,-5.3398423E-3,1.4539092E-2,8.228761E-3,1.890475E-2,9.131394E-3,4.570507E-3,1.4150088E-2,-1.6706988E-1,7.2310865E-2,-1.8060554E-2,-9.643882E-3,-1.01115964E-1,-2.0501201E-1,-1.04588315E-1,2.3978092E-3,-1.7310146E-2,-8.290944E-3,1.4051679E-3,-5.22323E-3,-4.314174E-3,-2.3851106E-2,9.6280344E-2,-2.7791766E-2,7.13368E-2,2.1080156E-1,-1.7589917E-2,1.1751634E-1,9.905234E-4,-1.2276243E-1,1.5467829E-1,3.0117992E-2,-2.1173723E-1,-1.0027598E-1,-1.6374715E-3,7.6507195E-3,-1.4358059E-1,-1.6352473E-3,-1.0641862E-2,-4.065583E-3,-7.0854135E-2,-7.4924002E-3,1.7874026E-3,7.697704E-3,-6.9604285E-2,9.6243724E-2,9.594987E-2,-7.492316E-3,1.5364879E-2,1.3179964E-1,-7.489912E-2,7.6373997E-3,4.0003918E-2,1.7660488E-1,-7.77869E-3,-1.5871506E-3,1.354796E-1,2.0589838E-2,-8.870381E-3,6.870691E-2,-1.8367597E-1,-3.554447E-1,-1.4009155E-1,-6.9369463E-4,-9.562683E-3,-3.0865753E-3,-5.7975124E-3,-2.4362074E-3,-6.7767436E-3,-3.841082E-2,1.8019229E-3,6.7304294E-3,7.2633E-2,1.812663E-1,1.791936E-1,5.4190175E-5,-4.928792E-2,-1.372968E-2,2.3710981E-2,-1.0721794E-1,7.766926E-3,-1.6710917E-2,1.1481164E-2,8.703412E-2,1.904918E-1,7.306057E-2,1.22991525E-1,-3.8058117E-3,-9.950286E-3,-3.0121743E-3,-2.0771096E-2,-8.609469E-3,-8.783466E-3,-5.0339014E-3,-2.618099E-3,1.2636482E-3,-3.939506E-3,2.731606E-5,2.0682104E-3,4.7070077E-3,1.0218954E-2,2.6412944E-3,1.0034836E-2,4.164094E-3,8.341585E-4,-4.223221E-3,-3.3481237E-3,2.1724629E-3,-8.852959E-3,-2.0215022E-3,1.9006011E-3,-3.761425E-3,7.8010594E-4,7.10566E-3,6.305115E-3,1.4021178E-2,-5.677469E-3,5.6100124E-3,3.424116E-3,9.708883E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,-1,-1,49,51,53,55,57,59,-1,-1,-1,61,63,65,67,-1,69,71,73,-1,75,77,-1,79,-1,-1,81,83,-1,-1,-1,-1,-1,85,87,89,-1,-1,91,93,95,-1,-1,-1,97,99,-1,101,103,105,-1,-1,107,109,-1,-1,-1,-1,-1,-1,-1,-1,111,113,-1,-1,115,117,119,-1,-1,-1,-1,-1,-1,-1,121,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,143,-1,-1,-1,145,-1,-1,-1,147,149,151,-1,-1,153,155,157,159,161,-1,-1,163,-1,-1,165,167,169,171,173,-1,-1,-1,-1,-1,175,-1,-1,177,179,181,-1,183,-1,185,187,-1,189,-1,191,193,195,197,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5833953E1,7.2137837E0,2.9638271E0,9.2628956E-1,1.9304225E0,3.0614972E0,5.7213783E-1,9.701071E-1,7.013869E-1,5.199224E-1,7.422021E-1,1.799525E0,2.1929169E-1,3.101492E-1,0E0,1.1940527E-1,9.985213E-1,1.6713586E-1,7.067418E-1,2.492106E-1,6.301643E-2,3.758315E-1,8.058327E-1,1.0455413E0,2.7417797E-1,0E0,0E0,1.346879E-1,2.2365904E-1,7.828534E-2,5.7002544E-2,3.320417E-1,5.5590105E-1,0E0,0E0,0E0,3.4148574E-2,8.412242E-2,2.2654927E-1,4.299224E-2,0E0,3.11382E-1,2.6013216E-1,7.001904E-1,0E0,6.866813E-2,5.7338154E-1,0E0,2.3952786E-2,0E0,0E0,5.6952596E-2,4.275036E-2,0E0,0E0,0E0,0E0,0E0,6.187865E-2,4.6019626E-1,1.7315626E-2,0E0,0E0,7.1834326E-2,1.0106379E-1,1.0975349E-1,0E0,0E0,0E0,7.344839E-2,4.4231218E-1,0E0,1.574592E-1,3.2522774E-1,5.288754E-1,0E0,0E0,9.224757E-2,3.6383367E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.81541E-1,9.6343994E-2,0E0,0E0,4.2361245E-2,3.605622E-2,6.621677E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.0267115E-2,2.0377596E-1,4.753556E-1,1.6782218E-1,2.8394052E-1,1.486584E-1,0E0,3.8571596E-2,5.206454E-1,2.8747773E-1,2.573223E-1,2.2193867E-1,0E0,0E0,3.48496E-2,0E0,0E0,0E0,2.0114452E-2,0E0,0E0,0E0,6.409441E-2,2.2617184E-2,1.3205391E-1,0E0,0E0,9.265354E-2,3.2693344E-1,2.5546297E-1,1.1906559E-1,1.00750804E-1,0E0,0E0,3.2602966E-1,0E0,0E0,2.3977716E-1,2.4512148E-1,1.0726166E-1,4.158306E-2,2.8238486E-2,0E0,0E0,0E0,0E0,0E0,4.0259857E-2,0E0,0E0,4.3531835E-2,6.1675876E-2,1.7663985E-2,0E0,1.5767989E-1,0E0,2.7472693E-1,8.2678944E-2,0E0,4.8269242E-2,0E0,4.4241786E-2,2.7597857E-1,4.698837E-1,7.3884726E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,48,48,51,51,52,52,58,58,59,59,60,60,63,63,64,64,65,65,69,69,70,70,72,72,73,73,74,74,77,77,78,78,87,87,88,88,91,91,92,92,93,93,101,101,102,102,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,112,112,115,115,119,119,123,123,124,124,125,125,128,128,129,129,130,130,131,131,132,132,135,135,138,138,139,139,140,140,141,141,142,142,148,148,151,151,152,152,153,153,155,155,157,157,158,158,160,160,162,162,163,163,164,164,165,165],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,-1,-1,50,52,54,56,58,60,-1,-1,-1,62,64,66,68,-1,70,72,74,-1,76,78,-1,80,-1,-1,82,84,-1,-1,-1,-1,-1,86,88,90,-1,-1,92,94,96,-1,-1,-1,98,100,-1,102,104,106,-1,-1,108,110,-1,-1,-1,-1,-1,-1,-1,-1,112,114,-1,-1,116,118,120,-1,-1,-1,-1,-1,-1,-1,122,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,144,-1,-1,-1,146,-1,-1,-1,148,150,152,-1,-1,154,156,158,160,162,-1,-1,164,-1,-1,166,168,170,172,174,-1,-1,-1,-1,-1,176,-1,-1,178,180,182,-1,184,-1,186,188,-1,190,-1,192,194,196,198,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,2.9251662E6,4.4895835E0,5.47E2,5.428175E3,2E0,4.54E2,3.89E2,2.9652428E6,1E0,3.9661028E7,1.3342042E0,3.3382E4,-2.7943772E-4,2.416E3,2.6333334E1,1.317484E1,1.753E3,6.2233735E-6,4.49239E6,8.1E2,7.168071E6,5.1E1,1.5900173E3,3.1359896E-2,1.2808675E-2,5.3700186E8,6.426937E6,3.07E2,2.7546012E0,1.3E2,4.217427E7,-1.3769596E-2,-2.3702849E-3,-3.6308873E-2,6.185E3,4.3827028E5,9E0,1.2E1,4.511144E-3,6.83E2,4.539777E5,1.5197045E0,1.5556166E-2,6.7321223E-1,2.8204132E7,-1.5473763E-2,2.6654E4,1.6368799E-2,2.7545583E-2,9.4658756E-1,3.3765998E0,-1.144036E-2,-3.1243395E-3,-1.645332E-2,-7.3663923E-3,-4.727472E-3,2.638E3,4.4970587E2,1.5416006E5,-7.40218E-3,-1.6927782E-2,3.1E1,7.9559E5,3.9595376E7,-1.0871704E-3,2.1256988E-3,-3.9635994E-3,1.4598765E0,6.278306E7,-9.239928E-3,1.3E1,4.797342E2,5.862504E2,2.2257049E-2,1.0584036E-2,1.3E1,1.6007428E7,-4.2622528E-4,-5.3398423E-3,1.4539092E-2,8.228761E-3,1.890475E-2,9.131394E-3,4.570507E-3,1.4150088E-2,1.339646E6,1.7500242E7,-1.8060554E-2,-9.643882E-3,3.3820656E7,1.9470909E2,5.1E1,2.3978092E-3,-1.7310146E-2,-8.290944E-3,1.4051679E-3,-5.22323E-3,-4.314174E-3,-2.3851106E-2,1.295313E6,2.364E3,4.5620965E6,2E0,4.4761734E5,2.8209653E-2,9.905234E-4,2.1340163E8,3.4915986E3,1.102794E3,2.0734E4,1.1995476E6,-1.6374715E-3,7.6507195E-3,4.017544E0,-1.6352473E-3,-1.0641862E-2,-4.065583E-3,6.14E2,-7.4924002E-3,1.7874026E-3,7.697704E-3,3.6966666E2,6.33E2,7.6569915E-1,-7.492316E-3,1.5364879E-2,9.07826E0,5.465798E7,6.4E1,1.125584E-3,9E0,-7.77869E-3,-1.5871506E-3,2.9308079E1,2.0589838E-2,-8.870381E-3,3.85374E8,5.57E2,6.160415E6,1.631108E6,1.030671E6,-9.562683E-3,-3.0865753E-3,-5.7975124E-3,-2.4362074E-3,-6.7767436E-3,1.6E1,1.8019229E-3,6.7304294E-3,8.7E1,3.7509E4,1.124641E0,5.4190175E-5,7.9146667E0,-1.372968E-2,2.5437157E2,9.884021E0,7.766926E-3,3.774872E7,1.1481164E-2,2.6662E4,7.177158E6,8.237181E9,3.2952412E5,-3.8058117E-3,-9.950286E-3,-3.0121743E-3,-2.0771096E-2,-8.609469E-3,-8.783466E-3,-5.0339014E-3,-2.618099E-3,1.2636482E-3,-3.939506E-3,2.731606E-5,2.0682104E-3,4.7070077E-3,1.0218954E-2,2.6412944E-3,1.0034836E-2,4.164094E-3,8.341585E-4,-4.223221E-3,-3.3481237E-3,2.1724629E-3,-8.852959E-3,-2.0215022E-3,1.9006011E-3,-3.761425E-3,7.8010594E-4,7.10566E-3,6.305115E-3,1.4021178E-2,-5.677469E-3,5.6100124E-3,3.424116E-3,9.708883E-3],"split_indices":[2,43,43,68,2,67,6,1,2,43,29,60,53,9,0,9,62,69,44,52,43,2,43,0,70,0,0,7,60,1,68,2,60,0,0,0,44,43,8,3,0,2,43,68,0,53,5,0,9,0,0,68,53,0,0,0,0,0,1,67,48,0,0,10,12,7,0,0,0,69,5,0,3,4,67,0,0,3,66,0,0,0,0,0,0,0,0,9,12,0,0,7,4,10,0,0,0,0,0,0,0,9,2,62,8,43,53,0,7,4,70,44,60,0,0,69,0,0,0,1,0,0,0,67,0,57,0,0,71,5,8,53,8,0,0,71,0,0,47,10,12,5,12,0,0,0,0,0,3,0,0,0,1,53,0,73,0,4,71,0,7,0,10,60,46,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.058E3,7.85E2,2.73E2,2.54E2,5.31E2,1.92E2,8.1E1,2.28E2,2.6E1,1.15E2,4.16E2,1.79E2,1.3E1,7.7E1,4E0,5.8E1,1.7E2,1.1E1,1.5E1,9.9E1,1.6E1,8E1,3.36E2,1.63E2,1.6E1,9E0,4E0,2.3E1,5.4E1,1.8E1,4E1,1.2E1,1.58E2,5E0,6E0,6E0,9E0,7.7E1,2.2E1,8E0,8E0,2.4E1,5.6E1,3.28E2,8E0,1.2E1,1.51E2,8E0,8E0,1.2E1,1.1E1,2.4E1,3E1,1.4E1,4E0,3.6E1,4E0,4E0,8E0,1.42E2,1.6E1,4E0,5E0,3.4E1,4.3E1,1.8E1,4E0,4E0,4E0,1.4E1,1E1,6E0,5E1,1E2,2.28E2,7E0,5E0,1.7E1,1.34E2,4E0,4E0,8E0,1.6E1,2.6E1,4E0,4E0,4E0,1.34E2,8E0,1.1E1,5E0,1.6E1,1.8E1,3.9E1,4E0,9E0,9E0,5E0,9E0,6E0,4E0,1.3E1,3.7E1,7.9E1,2.1E1,1.95E2,3.3E1,7E0,1E1,1.03E2,3.1E1,7.9E1,5.5E1,4E0,4E0,9E0,7E0,1.4E1,4E0,2.6E1,1.3E1,8E0,5E0,2.8E1,9E0,7.2E1,7E0,7E0,1.4E1,5.9E1,1.36E2,1.5E1,1.8E1,6E0,4E0,9.8E1,5E0,4E0,2.7E1,6.8E1,1.1E1,3.9E1,1.6E1,4E0,5E0,5E0,2.1E1,7E0,2.1E1,5E0,4E0,5.8E1,1.4E1,1E1,4E0,5.4E1,5E0,1.2E2,1.6E1,4E0,1.1E1,9E0,9E0,5.1E1,4.7E1,2E1,7E0,5.3E1,1.5E1,6E0,5E0,1.3E1,2.6E1,5E0,1.1E1,9E0,1.2E1,3.1E1,2.7E1,1E1,4E0,6E0,4E0,2.1E1,3.3E1,2.3E1,9.7E1,6E0,1E1,6E0,5E0,5E0,4E0,3.6E1,1.5E1,9E0,3.8E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"199","size_leaf_vector":"1"}},{"base_weights":[-9.581912E-3,-7.257247E-2,1.7942704E-1,-1.9418085E-1,-1.7906478E-2,-3.5965043E-1,2.0962237E-1,-1.6565062E-1,-3.8026592E-1,-1.2550124E-2,-2.6842918E-2,-2.3116974E-2,-1.0435031E-2,1.0108888E-1,2.8178123E-1,-1.5956992E-2,-1.3240612E-1,-5.664697E-1,-2.6462272E-1,-6.357436E-2,4.211078E-2,7.2013184E-2,2.8789636E-1,3.04593E-1,-9.930838E-2,6.466385E-2,-1.5498063E-1,-1.1142762E-2,-3.6836233E-2,-6.2297476E-3,-3.341127E-1,-4.581332E-2,-1.7267416E-1,2.0968283E-2,1.3817504E-1,5.1410656E-2,1.3865546E-2,1.6300699E-2,8.400301E-3,3.9637175E-1,2.3511447E-1,-8.551965E-3,2.9365002E-4,-8.7408796E-2,1.5608201E-2,-2.111568E-1,-1.0661881E-1,-1.8770915E-2,-8.416108E-3,-1.9438569E-2,-1.268761E-1,-2.324132E-1,-3.9913446E-2,-2.0647852E-2,7.2986744E-2,3.5533722E-2,1.8669394E-1,7.659787E-2,-8.622999E-2,3.3156115E-1,3.1152323E-2,3.555974E-1,1.6828786E-1,-1.3954026E-3,-6.8889037E-3,-2.4515393E-1,-9.3656994E-2,-7.471838E-2,-1.39719015E-2,-4.5223065E-2,3.9984792E-2,-9.4256885E-2,-2.0000146E-1,-1.6699068E-1,-1.783897E-2,1.1126087E-3,-7.103217E-2,3.2815162E-2,-7.65202E-2,1.4088078E-1,3.0322812E-2,3.80353E-3,-1.6813305E-3,1.0851359E-3,2.1000262E-1,9.989559E-3,1.5330155E-1,1.4128158E-4,-6.5795444E-3,3.895956E-1,1.8751533E-1,3.4678753E-3,3.9922535E-1,9.4451435E-2,2.5016007E-1,-1.6030985E-1,-3.1861877E-1,-6.0339727E-2,-7.7713244E-3,-1.06611826E-1,1.06922075E-1,1.0290782E-2,-7.21214E-2,-5.059808E-3,6.0526878E-2,-1.173335E-2,-7.076105E-2,-1.16590075E-2,-5.06098E-3,-1.0104442E-2,-2.8673087E-3,-8.059999E-4,-5.1009133E-3,-5.1831216E-2,1.4132656E-1,-2.2734904E-1,-8.436309E-3,2.059346E-3,1.6258979E-1,-2.3621519E-3,1.5853576E-1,1.6545305E-1,1.363455E-2,-4.0741548E-2,9.986728E-2,2.0863858E-1,7.6677024E-2,2.731075E-1,2.0024195E-2,3.3940226E-4,2.6874238E-1,1.0049854E-2,2.06835E-2,1.3586044E-1,-6.457043E-3,2.5187032E-3,2.84829E-1,-1.9281705E-1,-8.251519E-2,-6.3519846E-3,-3.5062477E-1,-3.810116E-3,-1.8515399E-4,-2.0755851E-1,-7.2851844E-2,8.5797915E-3,-1.87239E-3,3.1531107E-2,-4.5437623E-2,5.6683127E-2,-9.2034005E-2,4.04152E-2,9.143933E-3,-9.0005174E-2,1.2483862E-3,-1.189828E-1,1.0979742E-2,1.9397499E-1,-1.7005035E-3,-8.496999E-2,-2.6820421E-2,6.3791305E-2,-5.7213634E-2,1.8103935E-3,1.8076679E-1,3.0098017E-2,-1.0225083E-1,9.999024E-3,2.9828758E-3,9.143727E-3,3.5519006E-3,-1.002226E-1,1.0105338E-3,9.627098E-3,3.5622675E-2,5.269908E-3,1.091391E-2,2.9007126E-2,7.678993E-3,1.4638257E-2,6.860326E-3,5.6368955E-3,1.5483941E-2,1.6306031E-1,-5.0777383E-4,8.077706E-3,1.46527635E-2,-1.1176726E-2,-5.7484065E-3,-6.5746172E-3,-8.746909E-4,-1.8007085E-2,-9.23389E-3,-4.8850356E-3,-1.2704869E-2,-2.1737895E-3,-9.500405E-3,1.6851719E-4,4.5868917E-3,-5.4463306E-3,1.0696977E-3,-1.7372688E-3,5.013853E-3,-3.3590717E-3,-1.5900875E-2,2.7876692E-3,-2.6236782E-3,-1.9486265E-3,-6.2377723E-3,-2.3766218E-3,2.4804354E-3,-1.2117583E-2,-2.527635E-3,-2.295875E-3,3.259657E-3,4.09689E-3,1.1639896E-2,-8.153879E-3,1.467654E-3,-5.6455034E-4,4.4573303E-3,-7.0261434E-3,-1.6735867E-4,3.938377E-3,9.00426E-3,2.5920656E-3,-3.2891848E-3,-1.8424853E-3,-8.389362E-3,-2.5470501E-3,-6.6360096E-3,4.0666E-3,-1.5884526E-3,3.1150458E-3,-6.5819285E-4,1.2904538E-2,5.1943446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,49,51,53,55,57,-1,-1,-1,59,61,-1,-1,63,-1,65,67,-1,-1,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,-1,101,103,105,107,109,-1,-1,111,113,115,117,119,-1,-1,-1,121,123,125,-1,-1,127,129,-1,131,133,135,137,139,141,-1,143,145,147,149,-1,151,-1,153,-1,-1,-1,-1,-1,-1,155,157,159,161,-1,163,165,167,169,-1,171,173,175,177,179,-1,-1,181,-1,-1,183,-1,-1,185,187,189,-1,191,-1,-1,193,195,-1,-1,197,199,201,203,205,-1,207,209,211,213,215,-1,217,-1,219,221,-1,223,225,227,-1,-1,-1,-1,229,-1,-1,231,-1,-1,233,-1,-1,-1,-1,-1,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2240258E1,5.1220093E0,4.2587376E0,1.2058716E0,1.6003832E0,1.505487E-1,1.8824387E0,1.2031603E0,5.1762867E-1,1.4780744E0,0E0,0E0,0E0,5.169921E-1,1.2994747E0,0E0,8.0190086E-1,6.3375187E-1,1.4432836E-1,5.230262E-1,5.170484E-1,3.934447E-1,2.5388718E-2,7.867861E-1,8.277353E-2,8.188601E-1,4.121716E-1,0E0,0E0,0E0,6.858122E-2,5.042422E-1,2.921015E-1,4.5828137E-1,2.2244811E-1,2.8741452E-1,0E0,0E0,0E0,8.21187E-1,5.9232616E-1,0E0,0E0,4.1383535E-2,0E0,2.6864958E-1,5.27087E-1,0E0,0E0,2.7760446E-1,1.1949545E-1,1.969943E-1,2.7599968E-2,3.5498926E-1,2.6801097E-1,5.684987E-2,1.1299467E-1,3.509967E-1,6.56547E-2,3.3656597E-1,0E0,3.127277E-1,3.0281007E-1,0E0,0E0,2.9797363E-1,3.4562618E-2,4.574708E-1,0E0,1.8888918E-1,1.7554389E-1,1.3964123E-1,5.2874148E-2,9.497136E-2,0E0,0E0,1.706541E-2,5.6761605E-1,5.9368473E-1,6.801742E-2,2.4762201E-1,0E0,0E0,0E0,5.3560495E-2,1.7733341E-1,1.19687736E-1,0E0,0E0,5.5248737E-2,2.3217899E-1,0E0,1.1400533E-1,3.0994967E-1,1.5403879E-1,5.489856E-2,1.1990929E-1,1.7123844E-2,0E0,2.098043E-1,1.4694121E-1,5.1147655E-2,2.2322768E-1,0E0,1.2740256E-1,0E0,5.2771196E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.4893135E-1,2.6035976E-1,8.5583895E-1,1.4809561E-1,0E0,5.7966113E-2,1.5871291E-1,5.13137E-2,4.2295992E-2,0E0,9.53024E-2,9.017858E-2,2.0031273E-2,5.9202842E-2,1.8362343E-2,0E0,0E0,6.630117E-2,0E0,0E0,1.05645E-1,0E0,0E0,1.9231796E-2,3.3710003E-2,3.532484E-2,0E0,6.3732624E-2,0E0,0E0,8.0796E-2,1.6504192E-1,0E0,0E0,5.820383E-2,6.2363863E-2,6.2739775E-2,3.3556515E-1,8.864807E-2,0E0,5.889103E-2,2.7631111E-2,1.4222875E-1,7.218829E-2,1.0585451E-1,0E0,1.5518844E-1,0E0,4.3420322E-2,1.265544E-1,0E0,1.7138124E-2,1.0147102E-1,5.213912E-2,0E0,0E0,0E0,0E0,1.7530844E-2,0E0,0E0,3.9709907E-2,0E0,0E0,1.9240392E-2,0E0,0E0,0E0,0E0,0E0,9.808725E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,43,43,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,76,76,77,77,78,78,79,79,80,80,84,84,85,85,86,86,89,89,90,90,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,104,104,106,106,113,113,114,114,115,115,116,116,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,127,127,130,130,133,133,136,136,137,137,138,138,140,140,143,143,144,144,147,147,148,148,149,149,150,150,151,151,153,153,154,154,155,155,156,156,157,157,159,159,161,161,162,162,164,164,165,165,166,166,171,171,174,174,177,177,183,183],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,50,52,54,56,58,-1,-1,-1,60,62,-1,-1,64,-1,66,68,-1,-1,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,-1,102,104,106,108,110,-1,-1,112,114,116,118,120,-1,-1,-1,122,124,126,-1,-1,128,130,-1,132,134,136,138,140,142,-1,144,146,148,150,-1,152,-1,154,-1,-1,-1,-1,-1,-1,156,158,160,162,-1,164,166,168,170,-1,172,174,176,178,180,-1,-1,182,-1,-1,184,-1,-1,186,188,190,-1,192,-1,-1,194,196,-1,-1,198,200,202,204,206,-1,208,210,212,214,216,-1,218,-1,220,222,-1,224,226,228,-1,-1,-1,-1,230,-1,-1,232,-1,-1,234,-1,-1,-1,-1,-1,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,2.6E1,5.9E1,2.5330253E-2,9.616803E2,3.9E1,3.355864E0,9.48E2,-2.6842918E-2,-2.3116974E-2,-1.0435031E-2,5.6530495E6,3.206931E2,-1.5956992E-2,2.6333334E1,4E0,1.0772152E1,1.137832E-7,2.8971518E6,3.1639907E0,2.6846153E-1,5.932344E1,3.6086957E0,3.25E2,1.3823239E2,-1.1142762E-2,-3.6836233E-2,-6.2297476E-3,8.215373E6,2E0,2.8266037E2,2.437372E6,5.1942E4,2E0,1.3865546E-2,1.6300699E-2,8.400301E-3,1.9705E4,1.5277778E0,-8.551965E-3,2.9365002E-4,1.31306E5,1.5608201E-2,4.39776E6,4.217427E7,-1.8770915E-2,-8.416108E-3,2.732439E2,1.9E1,5.4E1,8.303459E5,3.72381E5,3.64E2,4.284608E6,4.745397E6,4.1032645E6,1.6E1,9.44363E5,3.1152323E-2,3.714144E8,5.620512E8,-1.3954026E-3,-6.8889037E-3,1E0,5.6842103E0,4.7866177E2,-1.39719015E-2,1.1772152E0,1.0909091E0,1.43E2,5.0710382E0,2.1578947E2,-1.783897E-2,1.1126087E-3,1.7352032E1,8.583682E4,1.048796E6,1.4979E4,5.7420593E2,3.80353E-3,-1.6813305E-3,1.0851359E-3,2.1111112E0,4.8709216E5,3.0977E4,1.4128158E-4,-6.5795444E-3,1.2562814E0,5.2141205E6,3.4678753E-3,7.659164E1,1.28636E7,1E0,2.590909E0,6.642857E0,7.2140925E5,-7.7713244E-3,1.9595902E3,6.743651E7,7.09E2,7.9634375E2,-5.059808E-3,3.7029E4,-1.173335E-2,4.629112E6,-1.16590075E-2,-5.06098E-3,-1.0104442E-2,-2.8673087E-3,-8.059999E-4,-5.1009133E-3,3.693316E7,8.695652E0,9.706078E4,3.1226995E0,2.059346E-3,5.4674416E7,6.4E1,4.857143E0,7E0,1.363455E-2,2.1340163E8,1.3333334E0,6.978874E5,6.624E3,1.0609756E0,2.0024195E-2,3.3940226E-4,2.5205562E5,1.0049854E-2,2.06835E-2,1.9407146E7,-6.457043E-3,2.5187032E-3,2.4344263E0,6.44E2,3.62E2,-6.3519846E-3,4.39E2,-3.810116E-3,-1.8515399E-4,9.6E1,4.76E2,8.5797915E-3,-1.87239E-3,5.57E2,1.994E3,1.1117E4,2.8302418E10,1E0,9.143933E-3,3.440772E7,2.08E2,2.3257812E2,2.3818183E0,1.609E3,-1.7005035E-3,1.373E3,-2.6820421E-2,1.071E3,1.058E3,1.8103935E-3,8.7578294E5,2.923409E7,8.7692904E7,9.999024E-3,2.9828758E-3,9.143727E-3,3.5519006E-3,1.5E1,1.0105338E-3,9.627098E-3,2.0394794E8,5.269908E-3,1.091391E-2,4.771E3,7.678993E-3,1.4638257E-2,6.860326E-3,5.6368955E-3,1.5483941E-2,2E0,-5.0777383E-4,8.077706E-3,1.46527635E-2,-1.1176726E-2,-5.7484065E-3,-6.5746172E-3,-8.746909E-4,-1.8007085E-2,-9.23389E-3,-4.8850356E-3,-1.2704869E-2,-2.1737895E-3,-9.500405E-3,1.6851719E-4,4.5868917E-3,-5.4463306E-3,1.0696977E-3,-1.7372688E-3,5.013853E-3,-3.3590717E-3,-1.5900875E-2,2.7876692E-3,-2.6236782E-3,-1.9486265E-3,-6.2377723E-3,-2.3766218E-3,2.4804354E-3,-1.2117583E-2,-2.527635E-3,-2.295875E-3,3.259657E-3,4.09689E-3,1.1639896E-2,-8.153879E-3,1.467654E-3,-5.6455034E-4,4.4573303E-3,-7.0261434E-3,-1.6735867E-4,3.938377E-3,9.00426E-3,2.5920656E-3,-3.2891848E-3,-1.8424853E-3,-8.389362E-3,-2.5470501E-3,-6.6360096E-3,4.0666E-3,-1.5884526E-3,3.1150458E-3,-6.5819285E-4,1.2904538E-2,5.1943446E-3],"split_indices":[2,43,17,3,3,53,67,2,73,2,0,0,0,43,73,0,62,8,73,52,43,57,53,73,69,2,67,0,0,0,12,32,4,9,1,6,0,0,0,2,68,0,0,9,0,9,60,0,0,67,8,0,43,9,10,43,60,60,3,9,0,7,7,0,0,8,71,67,0,68,68,2,69,70,0,0,73,48,9,1,67,0,0,0,68,43,44,0,0,68,60,0,73,60,14,68,73,60,0,48,7,2,48,0,1,0,9,0,0,0,0,0,0,7,61,48,69,0,7,8,69,8,0,7,68,43,2,68,0,0,48,0,0,9,0,0,69,1,1,0,2,0,0,10,0,0,0,10,12,1,46,19,0,7,0,48,68,2,0,2,0,2,10,0,60,9,5,0,0,0,0,3,0,0,7,0,0,2,0,0,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.026E3,7.7E2,2.56E2,2.38E2,5.32E2,1.3E1,2.43E2,2.08E2,3E1,5.28E2,4E0,5E0,8E0,9.8E1,1.45E2,3.1E1,1.77E2,1E1,2E1,2.73E2,2.55E2,8.6E1,1.2E1,1.37E2,8E0,1.8E1,1.59E2,5E0,5E0,8E0,1.2E1,2.36E2,3.7E1,2.1E2,4.5E1,8E1,6E0,6E0,6E0,5.7E1,8E1,4E0,4E0,1.2E1,6E0,7.2E1,8.7E1,7E0,5E0,1.79E2,5.7E1,2.5E1,1.2E1,1.17E2,9.3E1,1.5E1,3E1,6.8E1,1.2E1,4.8E1,9E0,2.7E1,5.3E1,7E0,5E0,5.5E1,1.7E1,7.6E1,1.1E1,1.25E2,5.4E1,4.1E1,1.6E1,1.9E1,6E0,4E0,8E0,6E1,5.7E1,3.5E1,5.8E1,9E0,6E0,4E0,2.6E1,3.7E1,3.1E1,5E0,7E0,3.3E1,1.5E1,4E0,2.3E1,2.9E1,2.4E1,2.7E1,2.8E1,1.3E1,4E0,6.5E1,1.1E1,4.1E1,8.4E1,6E0,4.8E1,4E0,3.7E1,9E0,7E0,1.2E1,7E0,4E0,4E0,3.4E1,2.6E1,1.7E1,4E1,7E0,2.8E1,4.7E1,1.1E1,1.9E1,7E0,2.4E1,1.3E1,1.7E1,1.4E1,1.1E1,2.2E1,5E0,1E1,6E0,1.7E1,2.5E1,4E0,4E0,2E1,1.8E1,9E0,5E0,2.3E1,9E0,4E0,1.5E1,5E1,7E0,4E0,3E1,1.1E1,1.1E1,7.3E1,4.3E1,5E0,2.9E1,8E0,1.6E1,1.8E1,2E1,6E0,1.3E1,4E0,1.6E1,2.4E1,4E0,2.4E1,3.6E1,1.1E1,6E0,5E0,1.3E1,6E0,1.2E1,1.2E1,4E0,9E0,5E0,1.2E1,1E1,4E0,7E0,4E0,4E0,6E0,2.1E1,4E0,6E0,1.4E1,9E0,9E0,4E0,5E0,1.7E1,6E0,7E0,8E0,4.3E1,7E0,2.2E1,8E0,5E0,6E0,4E0,7E0,6.9E1,4E0,3.6E1,7E0,1.5E1,1.4E1,4E0,4E0,4E0,1.2E1,9E0,9E0,8E0,1.2E1,7E0,6E0,5E0,1.1E1,8E0,1.6E1,4E0,2E1,2.9E1,7E0,7E0,4E0,7E0,5E0,5E0,4E0,5E0,5E0,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[8.72172E-3,-5.2378815E-2,1.8862124E-1,-1.4497288E-1,2.8107746E-2,2.2104456E-1,-4.3223172E-2,-2.8720894E-1,-1.0349912E-1,-5.3254906E-2,5.86618E-2,8.073839E-2,2.7825445E-1,-2.4189971E-1,8.807774E-2,-2.4866153E-1,-2.9690595E-2,-1.7638516E-1,-4.863835E-2,-9.128427E-2,8.682048E-2,4.8905227E-2,1.303921E-2,3.1544287E-2,2.4589539E-1,5.186504E-1,2.4941117E-1,-3.5997012E-3,-1.4142607E-2,1.5876757E-2,2.7034823E-3,-1.761185E-2,-2.2367401E-1,-1.62861E-1,-2.0514648E-2,2.0292308E-2,-7.1081586E-2,-2.1198758E-1,-4.1937176E-2,4.346774E-2,9.684896E-3,-1.4100128E-1,5.9390966E-2,1.0537662E-1,-4.0721823E-2,4.8275758E-3,1.4143654E-2,1.3774401E-2,3.0353699E-2,2.1816213E-2,2.1961127E-1,-5.3911596E-3,3.973649E-3,-1.5521282E-1,-2.4970925E-1,-1.4594304E-2,-1.430656E-1,-7.994017E-3,-1.3078807E-1,-8.0597945E-2,-3.4760985E-1,5.8276732E-3,-6.1842825E-2,3.9057387E-3,-4.0861764E-3,-5.4247882E-2,-1.40287345E-2,6.439775E-2,-9.971807E-3,-1.0422347E-3,1.5369838E-1,2.283647E-3,-1.0644382E-2,2.4693705E-1,9.715158E-2,-1.4571238E-3,-8.978063E-3,-1.2164489E-2,-4.2037857E-3,-8.9428425E-2,-2.0278858E-1,5.5312263E-3,-2.3124851E-2,-9.876157E-2,-2.8085992E-1,-5.663785E-3,9.016345E-4,-2.270514E-2,-8.068586E-3,-2.459085E-2,-1.1584179E-1,8.762837E-4,-4.564232E-3,4.2571846E-2,1.850129E-1,2.9195794E-3,8.805893E-3,4.2759046E-2,-4.4508157E-3,2.1421364E-1,4.018031E-1,1.703014E-2,1.22557245E-2,4.4476306E-3,-1.0588076E-1,-1.3681443E-1,-2.779559E-1,3.590126E-2,-6.642312E-2,-1.7129269E-1,-6.507932E-2,-1.5911153E-2,-8.186132E-3,-4.507132E-2,2.9461032E-3,-9.514544E-3,-5.4056033E-2,2.4281483E-2,1.2361709E-1,2.6249638E-1,8.5557684E-2,4.2141047E-3,-2.826333E-4,1.0423103E-1,2.4100588E-1,2.1852339E-2,1.0741991E-2,6.1002336E-3,-8.818392E-2,-2.001791E-1,-7.799874E-2,-1.7800367E-1,-7.733494E-4,-4.945717E-3,-3.118713E-1,5.4359782E-2,-2.9369479E-3,-1.3686177E-1,-5.772263E-3,-1.8939708E-1,-3.654349E-3,-3.6622614E-2,-7.570231E-3,-6.185178E-2,1.3657555E-3,-7.4566645E-3,1.6373991E-3,6.5352987E-3,1.1890642E-1,1.5681271E-1,-1.1835407E-2,1.6850555E-2,2.0135957E-1,-8.26217E-5,6.313497E-3,1.639012E-3,-1.9929293E-3,4.280685E-2,7.74868E-3,2.532417E-1,3.1546345E-3,2.6112532E-3,-8.966066E-3,-2.8425588E-3,-1.2502305E-2,-2.5173726E-3,-6.3366937E-3,-3.0830633E-3,-9.316043E-3,-1.608188E-2,-7.1748444E-3,8.384456E-4,3.765922E-3,-1.7980152E-3,-8.435483E-3,2.7885148E-3,-1.8783283E-3,-1.0050917E-2,-3.877687E-3,-6.5479623E-3,-3.4059043E-4,-1.1249786E-3,-3.8220037E-3,-6.0070277E-4,2.7662155E-3,6.9019375E-3,8.042589E-5,5.2760583E-3,9.539327E-3,2.090719E-3,-3.074931E-3,1.1207516E-2,5.9967865E-3,4.18515E-3,-1.5084115E-3,5.0713434E-3,1.2412042E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,43,45,47,49,-1,-1,-1,51,-1,53,55,-1,-1,57,59,61,63,-1,65,67,69,71,-1,-1,-1,-1,-1,73,-1,-1,75,77,-1,79,81,83,85,87,-1,89,-1,-1,91,-1,93,-1,-1,95,97,-1,99,101,-1,-1,-1,-1,103,105,-1,107,109,111,-1,-1,-1,-1,113,115,-1,-1,117,119,-1,-1,121,-1,123,125,127,-1,-1,129,131,133,135,137,139,141,-1,-1,143,-1,-1,145,147,149,151,153,-1,155,157,159,-1,-1,-1,161,163,165,167,-1,-1,169,171,-1,173,175,177,-1,179,-1,181,-1,-1,-1,183,185,187,189,-1,191,-1,-1,-1,-1,193,-1,195,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.101283E1,5.578603E0,1.9189138E0,2.0231476E0,9.9999356E-1,1.7675323E0,8.560711E-1,9.0894794E-1,1.0750372E0,5.9389293E-1,6.2315285E-1,5.2847695E-1,9.6618366E-1,1.1305791E-1,4.354735E-1,1.5321207E-1,0E0,3.475933E-1,1.7393423E0,5.093129E-1,1.1867106E-1,5.679323E-1,0E0,2.8174105E-1,9.5800996E-2,2.9226875E-1,8.4577847E-1,0E0,0E0,0E0,1.691855E-1,0E0,7.353306E-2,2.941072E-1,0E0,0E0,5.639649E-1,4.1746747E-1,2.1858262E-1,1.10424176E-1,0E0,1.951932E-1,3.8183808E-1,1.6420215E-1,2.2077356E-1,0E0,0E0,0E0,0E0,0E0,4.0433693E-1,0E0,0E0,9.0636015E-2,7.0994616E-2,0E0,3.0540562E-1,1.457744E-1,3.4234512E-1,6.1256036E-2,2.1274018E-1,0E0,1.1181073E-1,0E0,0E0,3.7750762E-2,0E0,6.844804E-1,0E0,0E0,5.142185E-2,9.593152E-2,0E0,4.4097376E-1,3.322513E-1,0E0,0E0,0E0,0E0,1.7822868E-1,1.9504929E-1,0E0,1.7361847E-1,1.476481E-1,2.4611235E-2,0E0,0E0,0E0,0E0,6.6212535E-2,1.1691299E-1,0E0,0E0,3.2904413E-1,2.8578675E-1,0E0,0E0,3.548234E-2,0E0,2.341876E-1,9.4928026E-2,2.2946034E-1,0E0,0E0,1.1847687E-1,1.3055676E-1,8.833504E-2,5.6703277E-2,1.658244E-1,1.840043E-2,1.23305246E-1,0E0,0E0,3.8147766E-2,0E0,0E0,1.4983103E-1,3.0820155E-1,1.8620956E-1,6.407273E-2,8.5050195E-2,0E0,1.6855368E-2,6.69201E-2,1.3064575E-1,0E0,0E0,0E0,1.6419947E-1,9.077996E-2,4.9550608E-2,3.9165437E-2,0E0,0E0,4.23671E-2,2.3103848E-2,0E0,7.351133E-2,5.32076E-2,2.9256344E-2,0E0,1.1271305E-1,0E0,1.6775578E-2,0E0,0E0,0E0,1.6335548E-1,1.03336334E-1,4.6262145E-2,3.1129247E-2,0E0,1.5717804E-2,0E0,0E0,0E0,0E0,4.2618744E-2,0E0,1.1433411E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,30,30,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,50,50,53,53,54,54,56,56,57,57,58,58,59,59,60,60,62,62,65,65,67,67,70,70,71,71,73,73,74,74,79,79,80,80,82,82,83,83,84,84,89,89,90,90,93,93,94,94,97,97,99,99,100,100,101,101,104,104,105,105,106,106,107,107,108,108,109,109,110,110,113,113,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,128,128,129,129,130,130,131,131,134,134,135,135,137,137,138,138,139,139,141,141,143,143,147,147,148,148,149,149,150,150,152,152,157,157,159,159],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,44,46,48,50,-1,-1,-1,52,-1,54,56,-1,-1,58,60,62,64,-1,66,68,70,72,-1,-1,-1,-1,-1,74,-1,-1,76,78,-1,80,82,84,86,88,-1,90,-1,-1,92,-1,94,-1,-1,96,98,-1,100,102,-1,-1,-1,-1,104,106,-1,108,110,112,-1,-1,-1,-1,114,116,-1,-1,118,120,-1,-1,122,-1,124,126,128,-1,-1,130,132,134,136,138,140,142,-1,-1,144,-1,-1,146,148,150,152,154,-1,156,158,160,-1,-1,-1,162,164,166,168,-1,-1,170,172,-1,174,176,178,-1,180,-1,182,-1,-1,-1,184,186,188,190,-1,192,-1,-1,-1,-1,194,-1,196,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.414E3,4.539777E5,2.856934E7,7.7E1,1E0,8.625455E2,1.3655363E2,1.05039425E1,5.41E2,6.33E2,7.82261E6,3.5510652E6,1.059448E7,3.6451373E8,1.2083E4,9.2058825E-1,-2.9690595E-2,3.1E1,2.00087E5,1.9750606E2,9.50283E5,1.6262903E2,1.303921E-2,1.5382514E1,5.2692295E6,5.467E3,7.714286E0,-3.5997012E-3,-1.4142607E-2,1.5876757E-2,1.01E3,-1.761185E-2,3.5675676E0,5.4E1,-2.0514648E-2,2.0292308E-2,4.064E3,1.0526316E0,1E1,2.5818555E2,9.684896E-3,1.5222145E2,2.3544883E8,1.1656439E9,1E0,4.8275758E-3,1.4143654E-2,1.3774401E-2,3.0353699E-2,2.1816213E-2,1.2792593E2,-5.3911596E-3,3.973649E-3,1.2E1,3E0,-1.4594304E-2,8.960784E0,3.2931E4,4.9735293E0,4.39E2,5.4E0,5.8276732E-3,5.524E3,3.9057387E-3,-4.0861764E-3,2.4481E4,-1.40287345E-2,5.817547E2,-9.971807E-3,-1.0422347E-3,1.29443414E5,6.0641945E1,-1.0644382E-2,4.73909E0,1.5E1,-1.4571238E-3,-8.978063E-3,-1.2164489E-2,-4.2037857E-3,7.3333335E-1,4.612245E0,5.5312263E-3,1E0,8.75E2,7.5391846E2,-5.663785E-3,9.016345E-4,-2.270514E-2,-8.068586E-3,1.535E3,1.5E1,8.762837E-4,-4.564232E-3,8.237181E9,3.8275862E0,2.9195794E-3,8.805893E-3,4.6816683E-1,-4.4508157E-3,1.000501E6,1.7655972E0,2.0277777E0,1.22557245E-2,4.4476306E-3,1E0,8E0,4.7721977E4,1E0,8.83425E7,2.026453E6,2.2587704E7,-1.5911153E-2,-8.186132E-3,1.28636E7,2.9461032E-3,-9.514544E-3,1.2218E4,1.427636E5,2.9411764E0,1.500502E6,1.98E2,4.2141047E-3,1.1E1,7.607E3,3.0546486E8,2.1852339E-2,1.0741991E-2,6.1002336E-3,1.6677892E2,1.1E1,3.373828E-1,2.992648E-2,-7.733494E-4,-4.945717E-3,1.9470909E2,1.2748E4,-2.9369479E-3,6.76E2,1.75E0,2.3847478E5,-3.654349E-3,2.4177586E2,-7.570231E-3,1.4E1,1.3657555E-3,-7.4566645E-3,1.6373991E-3,9.750085E6,2.2681375E6,1.7903225E0,3.3E1,1.6850555E-2,6.726722E2,-8.26217E-5,6.313497E-3,1.639012E-3,-1.9929293E-3,3.019898E8,7.74868E-3,1.0089981E3,3.1546345E-3,2.6112532E-3,-8.966066E-3,-2.8425588E-3,-1.2502305E-2,-2.5173726E-3,-6.3366937E-3,-3.0830633E-3,-9.316043E-3,-1.608188E-2,-7.1748444E-3,8.384456E-4,3.765922E-3,-1.7980152E-3,-8.435483E-3,2.7885148E-3,-1.8783283E-3,-1.0050917E-2,-3.877687E-3,-6.5479623E-3,-3.4059043E-4,-1.1249786E-3,-3.8220037E-3,-6.0070277E-4,2.7662155E-3,6.9019375E-3,8.042589E-5,5.2760583E-3,9.539327E-3,2.090719E-3,-3.074931E-3,1.1207516E-2,5.9967865E-3,4.18515E-3,-1.5084115E-3,5.0713434E-3,1.2412042E-2],"split_indices":[2,43,60,44,29,67,71,69,2,0,43,43,12,7,9,68,0,8,5,70,1,4,0,71,60,2,71,0,0,0,8,0,69,2,0,0,44,71,3,4,0,4,12,46,6,0,0,0,0,0,73,0,0,3,8,0,73,9,68,2,73,0,44,0,0,12,0,67,0,0,48,73,0,50,3,0,0,0,0,68,69,0,26,2,70,0,0,0,0,2,3,0,0,46,69,0,0,53,0,43,49,68,0,0,13,8,43,19,7,9,12,0,0,60,0,0,44,48,68,9,10,0,3,2,47,0,0,0,73,3,53,53,0,0,4,1,0,2,68,43,0,4,0,3,0,0,0,9,43,68,8,0,4,0,0,0,0,7,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.47E2,2.53E2,3.47E2,4E2,2.22E2,3.1E1,7.7E1,2.7E2,1.09E2,2.91E2,6.5E1,1.57E2,1.2E1,1.9E1,7.1E1,6E0,1.15E2,1.55E2,8.6E1,2.3E1,2.8E2,1.1E1,5.1E1,1.4E1,1.5E1,1.42E2,4E0,8E0,4E0,1.5E1,9E0,6.2E1,1.11E2,4E0,6E0,1.49E2,2.4E1,6.2E1,1.8E1,5E0,1.4E1,2.66E2,2.5E1,2.6E1,5E0,9E0,7E0,8E0,1.5E1,1.27E2,6E0,9E0,1.9E1,4.3E1,1.1E1,1E2,7.3E1,7.6E1,1.3E1,1.1E1,6E0,5.6E1,1.4E1,4E0,1E1,4E0,2.62E2,4E0,7E0,1.8E1,2.2E1,4E0,1.03E2,2.4E1,5E0,1.4E1,3.9E1,4E0,5.4E1,4.6E1,7E0,6.6E1,6.4E1,1.2E1,9E0,4E0,5E0,6E0,3.4E1,2.2E1,4E0,6E0,2.23E2,3.9E1,6E0,1.2E1,1.6E1,6E0,8.7E1,1.6E1,1.7E1,7E0,4E0,5E1,2.6E1,2E1,2.8E1,3.8E1,1.9E1,4.5E1,6E0,6E0,2.8E1,6E0,8E0,1.4E1,1.83E2,4E1,2.1E1,1.8E1,7E0,9E0,1.8E1,6.9E1,1E1,6E0,8E0,9E0,1E1,4E1,1.9E1,7E0,4E0,1.6E1,2.4E1,4E0,1.7E1,2.1E1,1.5E1,4E0,3.6E1,9E0,2.3E1,5E0,6E0,8E0,1.55E2,2.8E1,3.2E1,8E0,6E0,1.5E1,7E0,1.1E1,5E0,4E0,1E1,8E0,6.4E1,5E0,4E0,5E0,4E0,6E0,3E1,1E1,4E0,1.5E1,1.2E1,4E0,1.1E1,1.3E1,6E0,1.1E1,7E0,1.4E1,1.1E1,4E0,7E0,2.9E1,9E0,1.4E1,1.14E2,4.1E1,2.2E1,6E0,1.9E1,1.3E1,4E0,4E0,8E0,7E0,6E0,4E0,7E0,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"197","size_leaf_vector":"1"}},{"base_weights":[-6.840664E-3,-7.564314E-2,1.9535907E-1,-1.9300523E-1,-2.1521175E-2,-3.0387625E-1,2.1447061E-1,-1.6691664E-1,-3.4692943E-1,-8.0028E-2,2.1416238E-2,-1.721789E-2,-7.416122E-3,5.991051E-1,1.8573333E-1,-2.0868088E-1,-8.339015E-2,-4.9642673E-1,-2.4282059E-1,-5.5640046E-2,-2.6173565E-1,1.0791406E-2,1.7121804E-1,1.8752169E-2,3.5375725E-2,6.153804E-2,2.328578E-1,-1.9223677E-1,-1.9543128E-2,1.2934754E-2,-1.2123926E-1,-1.0904279E-2,-3.5152256E-2,-1.9409508E-3,-2.851105E-1,-1.4052862E-1,-1.5613228E-2,-1.3557872E-1,-2.7016891E-2,-2.4180397E-2,6.0915094E-2,6.5613404E-2,1.1626313E-2,3.2972004E-2,9.303139E-3,2.5004095E-1,-9.377705E-3,-2.119346E-1,-7.990358E-2,-2.077344E-1,-3.6266204E-2,-1.5910942E-2,-6.5945326E-3,-1.8337767E-1,-9.328806E-2,1.5711425E-2,-8.075538E-2,-9.197322E-3,-5.216909E-2,-1.1941902E-1,-7.477994E-3,8.0042176E-2,-1.517489E-1,-5.2269764E-4,5.4877093E-3,1.5570532E-1,6.157E-3,2.2106372E-1,4.368353E-1,-2.1859212E-1,-1.3124169E-3,-1.2622821E-2,-8.669574E-3,-2.4108933E-1,-2.5025285E-3,1.6260386E-2,-1.4757274E-1,-3.908957E-3,-2.3108338E-1,-4.4649086E-4,-1.3085946E-1,-1.741995E-2,6.118278E-2,-1.1001991E-1,6.4516813E-3,-5.695077E-3,2.0137683E-3,-2.7138608E-2,-2.8045768E-1,9.3885595E-3,-1.0671488E-1,1.6980128E-1,3.4680944E-2,-1.2606748E-2,-1.1788013E-3,1.3482728E-3,1.0865502E-2,3.5373196E-2,-9.267539E-2,2.7034536E-1,1.5341043E-1,1.0501004E-2,2.1841403E-2,-2.0876442E-1,-1.7341226E-2,1.9915015E-3,-2.7630979E-3,-1.7403053E-1,-1.408249E-2,-2.2346012E-2,5.0646264E-2,-9.9995565E-3,-3.571021E-3,-1.3162834E-2,-1.562435E-1,-3.3883373E-3,-7.0597418E-3,5.0022574E-3,-3.538772E-2,2.5663525E-4,1.1904978E-1,-8.802719E-3,-3.23352E-2,1.7780726E-3,-2.1295685E-3,1.3409492E-3,-7.749313E-3,-1.9454127E-2,-3.868254E-3,5.086483E-4,8.497947E-3,-1.9072951E-1,-4.0342066E-2,1.0185063E-1,2.6930583E-1,8.930754E-2,-8.0569506E-2,7.236593E-2,-5.8642894E-2,1.1766521E-3,-9.026096E-3,1.7983173E-1,3.1910643E-1,1.029651E-1,3.3951056E-1,-1.3897629E-2,-1.907007E-1,-1.6402863E-3,-1.0218113E-2,7.861163E-4,-2.4435818E-3,4.6941247E-3,2.8159659E-4,-1.0388775E-2,-2.8447777E-3,-6.0307216E-2,7.929756E-2,-3.3707142E-2,5.030587E-3,7.5726593E-3,5.6892954E-4,6.3205147E-3,-6.1954847E-3,-7.278001E-3,8.822312E-3,-4.1713733E-3,-1.1689367E-2,8.8830624E-4,-3.185609E-3,1.4517672E-1,8.4653875E-4,1.6219297E-2,7.581044E-3,3.3748668E-2,1.530615E-1,1.1560535E-2,-2.000307E-2,3.919705E-2,5.767181E-3,-3.920738E-4,-5.4114377E-3,-3.4258162E-5,2.3252983E-1,2.515274E-1,3.9982662E-1,-2.1985779E-2,1.3787624E-1,8.8514695E-3,1.884963E-2,-1.124831E-2,-7.3551587E-3,-4.0056063E-3,1.2728353E-3,6.5676756E-3,2.6573543E-5,-3.5093222E-3,3.5097462E-4,3.1461755E-3,-1.1642853E-3,6.547251E-5,6.554649E-3,2.2026922E-3,8.144722E-3,-1.2287642E-3,3.7268945E-3,8.080138E-3,2.866218E-3,2.8895813E-3,-5.443667E-3,-1.6372912E-3,2.9130038E-3,1.332312E-2,2.5725174E-3,1.2843513E-2,7.203258E-3,2.037287E-2,1.0006106E-2,-6.540608E-3,5.5992655E-3,1.036429E-2,3.2748797E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,47,-1,-1,49,-1,-1,-1,51,53,55,57,-1,59,61,63,-1,65,-1,67,-1,69,71,73,75,-1,-1,77,79,81,83,-1,85,87,89,91,93,-1,-1,95,97,99,101,103,-1,105,-1,107,-1,109,111,-1,113,-1,115,117,119,121,123,-1,-1,125,127,129,131,133,135,-1,-1,-1,-1,137,139,141,143,-1,-1,145,-1,-1,-1,147,-1,149,151,-1,-1,-1,153,-1,-1,-1,155,157,159,-1,161,-1,-1,-1,-1,-1,-1,163,-1,165,167,169,171,173,175,177,179,-1,-1,181,183,185,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,191,193,195,-1,-1,-1,197,-1,-1,199,-1,-1,-1,-1,201,-1,-1,-1,203,205,207,-1,209,-1,-1,-1,-1,211,213,215,217,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4246592E1,4.8486733E0,2.5458717E0,9.098673E-1,1.3180709E0,3.8817823E-2,2.6631165E0,7.081895E-1,4.0863276E-1,9.6922505E-1,4.8041293E-1,0E0,0E0,2.364521E-1,1.3610392E0,4.0507698E-1,1.010421E0,7.156551E-1,1.7484891E-1,6.671763E-1,9.8036885E-1,4.9930102E-1,1.5338236E-1,0E0,0E0,2.569952E-1,1.3239107E0,2.7071905E-1,0E0,0E0,4.7020048E-1,0E0,0E0,0E0,1.0277665E-1,1.1041856E-1,2.7674374E-1,9.554705E-2,0E0,2.663469E-1,4.880603E-1,4.224673E-2,0E0,1.8334258E-1,0E0,7.9112816E-1,0E0,1.2819242E-1,1.4987557E-1,1.508503E-1,2.011824E-1,0E0,0E0,1.3122296E-1,9.9274665E-2,1.3966641E-1,1.1300105E-1,0E0,7.346066E-2,3.6390662E-1,2.4202682E-1,4.3384224E-1,1.4594364E-1,0E0,0E0,9.369987E-2,1.3899104E-1,4.3863106E-1,5.0651073E-2,8.494806E-2,0E0,3.939328E-2,0E0,5.8071494E-2,0E0,3.29921E-2,3.651063E-2,0E0,4.0088654E-2,0E0,1.6570061E-2,1.2578955E-1,1.3746837E-1,2.0025715E-1,2.1974608E-2,0E0,0E0,1.3911098E-1,2.1374452E-1,1.9210884E-1,1.09407604E-1,2.1598136E-1,4.6678448E-1,0E0,0E0,0E0,0E0,1.329265E-1,1.3844222E-1,3.045249E-1,5.56479E-1,0E0,0E0,1.1224699E-1,0E0,0E0,0E0,9.0109795E-2,0E0,1.5328394E-2,2.8894953E-2,0E0,0E0,0E0,5.7616442E-2,0E0,0E0,0E0,1.4196138E-1,7.7461936E-2,9.375256E-2,0E0,7.165874E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5774725E-1,0E0,3.346306E-2,2.3483142E-2,8.057438E-2,6.2831044E-2,1.719392E-1,7.842184E-1,4.209459E-2,3.12761E-2,0E0,0E0,2.9516435E-1,1.8309212E-1,2.2550237E-1,5.3544402E-2,0E0,1.07503414E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.515312E-2,4.4456627E-2,2.8501181E-2,0E0,0E0,0E0,2.898022E-2,0E0,0E0,1.1212287E-1,0E0,0E0,0E0,0E0,3.7106156E-2,0E0,0E0,0E0,8.1428446E-2,3.4864247E-2,1.393122E-1,0E0,3.4284696E-2,0E0,0E0,0E0,0E0,2.2100937E-1,3.512764E-2,7.039118E-2,2.2263767E-1,2.157852E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,30,30,34,34,35,35,36,36,37,37,39,39,40,40,41,41,43,43,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,68,68,69,69,71,71,73,73,75,75,76,76,78,78,80,80,81,81,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,92,92,97,97,98,98,99,99,100,100,103,103,107,107,109,109,110,110,114,114,118,118,119,119,120,120,122,122,129,129,131,131,132,132,133,133,134,134,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144,146,146,155,155,156,156,157,157,161,161,164,164,169,169,173,173,174,174,175,175,177,177,182,182,183,183,184,184,185,185,186,186],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,48,-1,-1,50,-1,-1,-1,52,54,56,58,-1,60,62,64,-1,66,-1,68,-1,70,72,74,76,-1,-1,78,80,82,84,-1,86,88,90,92,94,-1,-1,96,98,100,102,104,-1,106,-1,108,-1,110,112,-1,114,-1,116,118,120,122,124,-1,-1,126,128,130,132,134,136,-1,-1,-1,-1,138,140,142,144,-1,-1,146,-1,-1,-1,148,-1,150,152,-1,-1,-1,154,-1,-1,-1,156,158,160,-1,162,-1,-1,-1,-1,-1,-1,164,-1,166,168,170,172,174,176,178,180,-1,-1,182,184,186,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,192,194,196,-1,-1,-1,198,-1,-1,200,-1,-1,-1,-1,202,-1,-1,-1,204,206,208,-1,210,-1,-1,-1,-1,212,214,216,218,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.972052E5,1E0,2.6E1,7.785302E5,8.76E2,4.70127E5,7.24E2,3.355864E0,2.1E1,6.9307615E6,-1.721789E-2,-7.416122E-3,1.94261E5,8.399457E2,3.9E1,2.7E1,3.292496E-2,7.6255396E2,5.51E2,6.0692043E0,1.158E3,1E0,1.8752169E-2,3.5375725E-2,3.5510652E6,7.134432E7,9.566785E5,-1.9543128E-2,1.2934754E-2,1.4912975E-1,-1.0904279E-2,-3.5152256E-2,-1.9409508E-3,7.919006E7,1.561132E2,1.9E1,3.9E2,-2.7016891E-2,1.5961905E2,4.7220547E1,9.51E3,1.1626313E-2,3.414E3,9.303139E-3,6.663214E6,-9.377705E-3,1.317484E1,2.5493E4,1.7302156E5,3.38E2,-1.5910942E-2,-6.5945326E-3,1.001791E6,9.144571E5,5.5356906E4,1.862E3,-9.197322E-3,2.5E1,4.5418963E-4,6.346204E-7,2.308943E0,1.5714285E0,-5.2269764E-4,5.4877093E-3,3.3193566E5,2.115238E5,2.9308079E1,3.97371E5,4.233978E0,-1.3124169E-3,1.9903773E2,-8.669574E-3,6.0893228E7,-2.5025285E-3,6.0875E4,5.25E0,-3.908957E-3,5.7E1,-4.4649086E-4,2.087E2,2.6E1,9.61E2,3.011152E6,3.62349E5,-5.695077E-3,2.0137683E-3,7.89E2,7.3809524E0,3.2E1,5.5356906E4,1.28738E5,1.2629019E8,-1.2606748E-2,-1.1788013E-3,1.3482728E-3,1.0865502E-2,6.3055553E0,1E1,1.1772152E0,3.8537518E2,1.0501004E-2,2.1841403E-2,8.0833334E-1,-1.7341226E-2,1.9915015E-3,-2.7630979E-3,1.9166666E0,-1.408249E-2,2.29E2,8.910034E0,-9.9995565E-3,-3.571021E-3,-1.3162834E-2,1.1430505E3,-3.3883373E-3,-7.0597418E-3,5.0022574E-3,1.06314E5,1.4381613E7,1.8729467E8,-8.802719E-3,3.1316226E0,1.7780726E-3,-2.1295685E-3,1.3409492E-3,-7.749313E-3,-1.9454127E-2,-3.868254E-3,1.5881818E2,8.497947E-3,3.1E1,3.3272727E0,9.94E2,6E0,5.372237E2,2.364E3,4.924E3,2.5503825E6,1.1766521E-3,-9.026096E-3,3.714144E8,6.355E3,3.2608695E0,4.907764E3,-1.3897629E-2,1.0891155E4,-1.6402863E-3,-1.0218113E-2,7.861163E-4,-2.4435818E-3,4.6941247E-3,2.8159659E-4,-1.0388775E-2,-2.8447777E-3,1.3205625E7,5.342944E6,3.011152E6,5.030587E-3,7.5726593E-3,5.6892954E-4,2.99E2,-6.1954847E-3,-7.278001E-3,1.9650501E6,-4.1713733E-3,-1.1689367E-2,8.8830624E-4,-3.185609E-3,1.6446976E3,8.4653875E-4,1.6219297E-2,7.581044E-3,1.1E1,2.7012987E1,1.1962106E-1,-2.000307E-2,1.1E1,5.767181E-3,-3.920738E-4,-5.4114377E-3,-3.4258162E-5,2E0,1.6007428E7,1.853776E7,1.4342732E2,8.710612E1,8.8514695E-3,1.884963E-2,-1.124831E-2,-7.3551587E-3,-4.0056063E-3,1.2728353E-3,6.5676756E-3,2.6573543E-5,-3.5093222E-3,3.5097462E-4,3.1461755E-3,-1.1642853E-3,6.547251E-5,6.554649E-3,2.2026922E-3,8.144722E-3,-1.2287642E-3,3.7268945E-3,8.080138E-3,2.866218E-3,2.8895813E-3,-5.443667E-3,-1.6372912E-3,2.9130038E-3,1.332312E-2,2.5725174E-3,1.2843513E-2,7.203258E-3,2.037287E-2,1.0006106E-2,-6.540608E-3,5.5992655E-3,1.036429E-2,3.2748797E-3],"split_indices":[2,43,17,3,43,0,12,2,73,3,43,0,0,1,67,8,0,73,48,2,68,2,29,0,0,43,59,60,0,0,53,0,0,0,7,4,8,48,0,70,73,44,0,2,0,43,0,69,1,43,0,0,0,5,60,48,2,0,3,53,52,69,69,0,0,43,48,71,1,68,0,4,0,5,0,1,69,0,0,0,4,0,2,9,1,0,0,2,73,3,48,1,5,0,0,0,0,69,3,68,71,0,0,68,0,0,0,68,0,10,71,0,0,0,48,0,0,0,1,5,7,0,68,0,0,0,0,0,0,48,0,0,69,10,8,4,2,2,47,0,0,7,2,69,4,0,43,0,0,0,0,0,0,0,0,9,60,9,0,0,0,0,0,0,66,0,0,0,0,48,0,0,0,3,73,53,0,8,0,0,0,0,6,66,62,71,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,7.63E2,2.59E2,2.4E2,5.23E2,9E0,2.5E2,2.07E2,3.3E1,2.21E2,3.02E2,5E0,4E0,1.6E1,2.34E2,1.37E2,7E1,1.2E1,2.1E1,1.96E2,2.5E1,2.83E2,1.9E1,9E0,7E0,6.5E1,1.69E2,1.29E2,8E0,6E0,6.4E1,7E0,5E0,4E0,1.7E1,6.2E1,1.34E2,1.9E1,6E0,1.67E2,1.16E2,9E0,1E1,5.5E1,1E1,1.63E2,6E0,1.09E2,2E1,3.1E1,3.3E1,1.1E1,6E0,3.1E1,3.1E1,9.1E1,4.3E1,1E1,9E0,2.4E1,1.43E2,1.07E2,9E0,4E0,5E0,9E0,4.6E1,1.43E2,2E1,1.05E2,4E0,1.3E1,7E0,2.5E1,6E0,2.3E1,1E1,1.1E1,2E1,1E1,2.1E1,5.3E1,3.8E1,3.2E1,1.1E1,5E0,4E0,1.6E1,8E0,1.23E2,2E1,3.5E1,7.2E1,4E0,5E0,4E0,5E0,3.6E1,1E1,8.1E1,6.2E1,4E0,1.6E1,1.01E2,4E0,6E0,7E0,1.4E1,1.1E1,1.1E1,1.2E1,4E0,6E0,1E1,1E1,7E0,1.4E1,6E0,4.7E1,1.9E1,1.9E1,1.5E1,1.7E1,7E0,4E0,1.2E1,4E0,4E0,4E0,1.18E2,5E0,8E0,1.2E1,2.2E1,1.3E1,4.9E1,2.3E1,2.6E1,1E1,5E0,5E0,3E1,5.1E1,5E1,1.2E1,1.4E1,8.7E1,4E0,1E1,5E0,6E0,5E0,7E0,5E0,5E0,3.9E1,8E0,1.5E1,4E0,1.3E1,6E0,1.3E1,4E0,5E0,1.13E2,4E0,4E0,4E0,8E0,1.4E1,8E0,6E0,7E0,2.7E1,2.2E1,1.9E1,4E0,1.7E1,9E0,6E0,4E0,7E0,2.3E1,3E1,2.1E1,1.1E1,3.9E1,5E0,7E0,3E1,5.7E1,3E1,9E0,4E0,4E0,7E0,8E0,4E0,9E0,1.08E2,5E0,4E0,1E1,1.2E1,1.5E1,1.7E1,5E0,1.4E1,5E0,4E0,1.3E1,1.7E1,6E0,2.2E1,8E0,1.6E1,5E0,6E0,5E0,1.6E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-2.8888178E-3,-1.0378601E-1,1.03640884E-1,-1.7752095E-1,-4.3336123E-2,-2.8024974E-1,1.2771633E-1,-1.5946168E-1,-4.2828533E-1,-2.2663085E-2,-2.1770269E-1,-3.737383E-1,-1.400324E-1,7.256638E-2,2.6645783E-1,-2.0144477E-1,-7.7175714E-2,-1.762753E-1,-3.7833456E-2,-1.1101938E-2,-1.3162233E-1,-2.7249513E-2,-1.091418E-1,-1.882733E-2,-9.442346E-3,-2.26086E-3,-9.343133E-3,4.241862E-2,1.814922E-1,3.3945683E-2,2.3605664E-1,-2.6189423E-1,-1.6384618E-1,-5.902484E-2,-1.34531185E-2,-1.0062221E-2,-3.8291423E-3,8.8417195E-3,-1.6937181E-2,-1.0936117E-2,-7.1762465E-2,2.5273997E-3,-1.6103171E-1,5.24376E-2,-2.3033181E-1,1.3372271E-1,2.632323E-1,3.0090496E-1,1.1237551E-1,-1.8231781E-1,-2.9394698E-1,-1.1706685E-1,-2.0916119E-1,3.4688085E-2,-7.7425525E-2,-2.3373311E-2,1.3407475E-1,1.3794715E-4,-9.3332425E-2,-9.410978E-3,-1.2709263E-3,6.12179E-2,-2.0704196E-1,-1.3680625E-2,-4.8654717E-3,8.1075914E-2,2.4052992E-1,7.573041E-3,1.6519506E-2,3.369364E-1,9.153348E-2,4.2898804E-2,2.7307785E-1,-1.0592125E-2,-5.4114023E-3,-1.7695993E-1,-3.335862E-1,-1.3754368E-1,-4.8206348E-2,-2.647402E-3,-2.2166401E-1,6.4088334E-3,-2.2024706E-3,-1.1233871E-1,-1.6010825E-2,-1.1673879E-1,-1.2039128E-2,1.0470365E-3,9.451013E-3,-9.660795E-4,-5.5535776E-3,-7.306476E-3,1.055014E-1,-1.7359061E-2,1.4180505E-4,1.15015484E-1,-1.5274567E-2,4.1195396E-3,1.3099863E-2,1.6887656E-3,3.5382012E-1,-1.1621586E-3,7.3528434E-3,8.465699E-2,-9.989625E-3,6.3697295E-3,1.6504375E-2,-1.1958338E-2,-4.78911E-3,-9.370715E-3,-1.6632874E-2,-3.141064E-4,-1.5272544E-1,2.7652434E-4,-3.8264103E-3,-2.3514044E-1,-3.2117467E-3,-1.4402582E-1,-4.9925137E-2,-3.7213653E-2,3.326454E-3,3.2567137E-3,-1.795748E-1,3.361361E-2,-4.0257577E-2,1.3968155E-2,-1.63348E-2,1.5979484E-1,6.683572E-2,1.5146789E-1,1.3550721E-3,2.5317718E-3,-3.285927E-3,3.7045714E-1,8.346115E-3,-5.6934517E-2,1.3873275E-1,-1.704616E-1,-2.770229E-3,-1.5368074E-1,-2.6456314E-1,-8.46026E-3,-9.743069E-2,1.4720956E-4,-4.4718855E-3,-3.224681E-3,4.7905123E-4,-2.4685125E-1,2.6873744E-3,5.8459133E-2,-4.3845914E-2,-6.0763683E-2,1.2372842E-2,3.586358E-2,-1.3195297E-1,1.19522125E-1,2.7565986E-1,3.6398917E-2,1.5409309E-1,1.7750493E-1,2.2300573E-3,4.180528E-1,2.6491722E-1,2.59703E-3,-7.331607E-3,9.685445E-3,7.351236E-2,-5.6178286E-3,-1.0853061E-2,-8.500792E-3,-3.1583244E-3,-6.0907956E-3,-1.2990744E-2,-5.485076E-4,-6.0274336E-3,-4.691669E-3,-1.535808E-2,7.7819085E-4,4.014255E-3,8.9214253E-4,-3.339842E-3,-4.8270114E-3,-8.9502777E-4,-3.4885944E-3,2.916251E-3,8.705812E-3,1.0193696E-3,-9.336256E-3,9.5539633E-4,7.5618178E-3,1.1169775E-3,8.491481E-3,1.591753E-2,9.2139846E-4,7.552971E-3,9.486959E-3,3.960368E-3,9.777831E-3,3.405903E-3,2.0701755E-2,1.3645905E-2,3.7264237E-3,1.5198209E-2,5.622386E-4,5.097148E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,41,-1,-1,-1,-1,43,45,-1,47,49,51,53,-1,-1,-1,-1,55,-1,57,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,89,-1,-1,91,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,-1,107,109,111,113,-1,115,-1,-1,117,119,121,123,-1,-1,-1,-1,125,127,-1,-1,129,131,-1,-1,-1,133,-1,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,137,-1,-1,139,-1,141,143,145,-1,-1,147,149,151,153,-1,155,157,159,-1,-1,-1,161,-1,163,165,167,-1,169,171,-1,173,-1,-1,-1,-1,175,-1,177,179,181,183,185,187,189,191,193,195,197,-1,199,201,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1490206E1,2.4393477E0,4.8418083E0,1.0502748E0,1.0872861E0,3.5403466E-1,3.7299428E0,7.8592443E-1,1.430757E0,3.4117547E-1,1.2283708E0,1.9706964E-2,6.943387E-2,1.1482234E0,1.8055296E0,3.0483055E-1,2.9358885E-1,2.4608076E-2,0E0,2.9508433E-1,1.4657792E-1,0E0,2.2986373E-1,0E0,0E0,0E0,0E0,7.6903045E-1,2.6155567E-1,0E0,1.0248938E0,9.1795206E-2,1.8113518E-1,1.3201442E-1,0E0,0E0,0E0,0E0,2.3736948E-1,0E0,2.9300451E-2,0E0,1.0835254E-1,6.232615E-1,5.1064074E-2,2.616517E-1,1.943475E-1,6.179395E-1,5.130435E-1,3.129816E-2,1.2182188E-1,6.38718E-2,7.843256E-2,1.1800276E-1,1.332849E-1,2.4410196E-1,7.3905006E-2,0E0,2.473712E-2,0E0,0E0,7.897406E-1,3.2017112E-1,0E0,0E0,1.1621013E-1,7.890695E-2,0E0,0E0,3.5729122E-1,1.1447441E-1,3.8621104E-1,1.1005843E-1,0E0,0E0,4.5190156E-2,2.3774147E-2,7.438916E-2,2.4883984E-2,0E0,7.386875E-2,0E0,0E0,7.191771E-2,4.746671E-2,3.0678138E-1,2.6930907E-1,0E0,0E0,0E0,0E0,7.637111E-1,3.2100832E-1,0E0,0E0,7.676566E-2,4.3035794E-2,0E0,0E0,0E0,1.3185978E-1,0E0,0E0,2.375987E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.571712E-2,0E0,0E0,5.5212498E-2,0E0,2.9527068E-2,3.7649244E-2,3.173459E-2,0E0,0E0,3.1825638E-1,1.5652499E-1,1.3997442E-1,3.1784278E-1,0E0,2.7035546E-1,2.4403962E-1,3.979683E-2,0E0,0E0,0E0,2.0069122E-1,0E0,1.12548366E-1,8.633819E-2,5.961919E-2,0E0,2.1052778E-2,2.4721503E-2,0E0,3.8021833E-2,0E0,0E0,0E0,0E0,1.5276265E-1,0E0,7.070933E-2,3.732783E-2,1.6626453E-1,1.6979533E-1,1.7690632E-1,1.4334238E-1,2.0654905E-1,3.791082E-2,1.4521608E-1,6.650138E-2,3.4002006E-2,0E0,1.8774033E-2,2.3312235E-1,0E0,0E0,0E0,2.8017692E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,27,27,28,28,30,30,31,31,32,32,33,33,38,38,40,40,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,80,80,83,83,84,84,85,85,86,86,91,91,92,92,95,95,96,96,100,100,103,103,112,112,115,115,117,117,118,118,119,119,122,122,123,123,124,124,125,125,127,127,128,128,129,129,133,133,135,135,136,136,137,137,139,139,140,140,142,142,147,147,149,149,150,150,151,151,152,152,153,153,154,154,155,155,156,156,157,157,158,158,159,159,161,161,162,162,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,42,-1,-1,-1,-1,44,46,-1,48,50,52,54,-1,-1,-1,-1,56,-1,58,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,90,-1,-1,92,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,-1,108,110,112,114,-1,116,-1,-1,118,120,122,124,-1,-1,-1,-1,126,128,-1,-1,130,132,-1,-1,-1,134,-1,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,138,-1,-1,140,-1,142,144,146,-1,-1,148,150,152,154,-1,156,158,160,-1,-1,-1,162,-1,164,166,168,-1,170,172,-1,174,-1,-1,-1,-1,176,-1,178,180,182,184,186,188,190,192,194,196,198,-1,200,202,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,4.97E2,1E0,4.4895835E0,3.1316226E0,1.7540792E5,1.1459359E3,3.51E2,1E0,1E0,1E0,1.3141646E3,1.91E2,2.9652428E6,3.3817584E7,3.89E2,2.1E1,8.5714287E-1,-3.7833456E-2,1.2E1,8.314423E6,-2.7249513E-2,2.857143E0,-1.882733E-2,-9.442346E-3,-2.26086E-3,-9.343133E-3,1.2294118E1,2.593592E6,3.3945683E-2,4.1681065E1,5E1,3.1E1,8.638015E4,-1.34531185E-2,-1.0062221E-2,-3.8291423E-3,8.8417195E-3,7.4711205E6,-1.0936117E-2,3.202146E5,2.5273997E-3,2.0134516E6,1.5449402E7,1.594941E3,1E0,1.81E2,2.2040408E7,4.435876E2,3.5070792E2,2.416E3,2E0,3.3305264E2,2.90099E0,8.599521E5,1.6666667E-1,3.3852024E7,1.3794715E-4,2.5242718E-2,-9.410978E-3,-1.2709263E-3,3.850926E2,4.44E2,-1.3680625E-2,-4.8654717E-3,3.744E3,7.156528E7,7.573041E-3,1.6519506E-2,5.80418E7,4.771E3,3.206931E2,1.11436836E5,-1.0592125E-2,-5.4114023E-3,1.2571428E1,8.683527E3,1E1,3.927456E7,-2.647402E-3,1.901875E2,6.4088334E-3,-2.2024706E-3,1.339646E6,9.458613E3,3.151751E-1,1.5357143E0,1.0470365E-3,9.451013E-3,-9.660795E-4,-5.5535776E-3,2.954124E6,3.6828358E0,-1.7359061E-2,1.4180505E-4,2.4E0,7.617426E4,4.1195396E-3,1.3099863E-2,1.6887656E-3,1E0,-1.1621586E-3,7.3528434E-3,9.408E3,-9.989625E-3,6.3697295E-3,1.6504375E-2,-1.1958338E-2,-4.78911E-3,-9.370715E-3,-1.6632874E-2,-3.141064E-4,1.0666667E1,2.7652434E-4,-3.8264103E-3,1.625E0,-3.2117467E-3,1.9225262E2,1.2E1,9.356961E6,3.326454E-3,3.2567137E-3,8.1E2,1.5532663E1,1.6802971E6,6.16E2,-1.63348E-2,3.51E3,7.445455E2,1.5723623E0,1.3550721E-3,2.5317718E-3,-3.285927E-3,9.44363E5,8.346115E-3,3.0238653E8,8.710612E1,3.407316E7,-2.770229E-3,4.162742E5,9.185E3,-8.46026E-3,1.4E1,1.4720956E-4,-4.4718855E-3,-3.224681E-3,4.7905123E-4,2E0,2.6873744E-3,1.8791208E0,1E0,3.4289816E2,2.7857144E0,3.4662805E-4,4.6404468E2,5.556659E6,1E0,1.8953742E6,5.388794E6,5.1291872E8,2.2300573E-3,1.7103828E7,5.831829E6,2.59703E-3,-7.331607E-3,9.685445E-3,2.2365898E3,-5.6178286E-3,-1.0853061E-2,-8.500792E-3,-3.1583244E-3,-6.0907956E-3,-1.2990744E-2,-5.485076E-4,-6.0274336E-3,-4.691669E-3,-1.535808E-2,7.7819085E-4,4.014255E-3,8.9214253E-4,-3.339842E-3,-4.8270114E-3,-8.9502777E-4,-3.4885944E-3,2.916251E-3,8.705812E-3,1.0193696E-3,-9.336256E-3,9.5539633E-4,7.5618178E-3,1.1169775E-3,8.491481E-3,1.591753E-2,9.2139846E-4,7.552971E-3,9.486959E-3,3.960368E-3,9.777831E-3,3.405903E-3,2.0701755E-2,1.3645905E-2,3.7264237E-3,1.5198209E-2,5.622386E-4,5.097148E-3],"split_indices":[2,2,17,68,68,48,67,2,6,6,29,67,0,43,7,1,8,71,0,10,5,0,73,0,0,0,0,68,9,0,71,44,10,43,0,0,0,0,43,0,43,0,60,62,48,23,0,66,71,48,9,8,48,69,43,73,7,0,57,0,0,67,0,0,0,10,7,0,0,7,2,73,48,0,0,70,43,3,7,0,4,0,0,9,48,68,68,0,0,0,0,62,69,0,0,69,48,0,0,0,6,0,0,2,0,0,0,0,0,0,0,0,73,0,0,68,0,4,3,60,0,0,2,73,43,0,0,10,67,68,0,0,0,9,0,7,73,7,0,62,12,0,3,0,0,0,0,8,0,69,26,4,69,53,4,62,29,43,62,7,0,62,60,0,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.067E3,5.48E2,5.19E2,2.46E2,3.02E2,3E1,4.89E2,2.31E2,1.5E1,2.71E2,3.1E1,1.7E1,1.3E1,3.51E2,1.38E2,1.52E2,7.9E1,1E1,5E0,2.46E2,2.5E1,6E0,2.5E1,1.3E1,4E0,6E0,7E0,2.76E2,7.5E1,7E0,1.31E2,5.6E1,9.6E1,7.4E1,5E0,6E0,4E0,6E0,2.4E2,8E0,1.7E1,6E0,1.9E1,2.67E2,9E0,4.9E1,2.6E1,8.5E1,4.6E1,1.8E1,3.8E1,4.9E1,4.7E1,1.2E1,6.2E1,2.31E2,9E0,4E0,1.3E1,1.4E1,5E0,2.59E2,8E0,5E0,4E0,3.4E1,1.5E1,1.4E1,1.2E1,7.2E1,1.3E1,3.3E1,1.3E1,9E0,9E0,1.1E1,2.7E1,3.7E1,1.2E1,4E0,4.3E1,5E0,7E0,3.9E1,2.3E1,2.4E1,2.07E2,4E0,5E0,4E0,9E0,1.02E2,1.57E2,4E0,4E0,2.5E1,9E0,4E0,1.1E1,4E0,6.8E1,5E0,8E0,2.9E1,4E0,6E0,7E0,4E0,7E0,6E0,2.1E1,4E0,3.3E1,5E0,7E0,3.9E1,4E0,2.5E1,1.4E1,1.9E1,4E0,6E0,1.8E1,7.9E1,1.28E2,9.7E1,5E0,6.4E1,9.3E1,1.7E1,8E0,4E0,5E0,6.1E1,7E0,8E0,2.1E1,2.7E1,6E0,1.2E1,2.7E1,1.2E1,1.3E1,7E0,7E0,1.1E1,8E0,1.4E1,4E0,6E1,1.9E1,9.2E1,3.6E1,8.5E1,1.2E1,4.9E1,1.5E1,7E1,2.3E1,1.3E1,4E0,4E1,2.1E1,4E0,4E0,9E0,1.2E1,1.7E1,1E1,8E0,4E0,4E0,2.3E1,4E0,9E0,6E0,8E0,2.5E1,3.5E1,6E0,1.3E1,4.4E1,4.8E1,1.3E1,2.3E1,6E0,7.9E1,8E0,4E0,3.3E1,1.6E1,8E0,7E0,6.3E1,7E0,1.2E1,1.1E1,9E0,4E0,3E1,1E1,6E0,1.5E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"205","size_leaf_vector":"1"}},{"base_weights":[-2.3427939E-3,-5.623861E-2,1.5676703E-1,-1.3198337E-1,1.8639259E-2,-2.9502624E-1,1.8686032E-1,-1.1840211E-1,-3.6169827E-1,3.3162467E-2,-1.0273241E-1,-1.826237E-2,-5.0650304E-3,5.7936394E-1,1.6126613E-1,-2.2093423E-1,-8.746913E-2,-4.4488412E-1,-1.0701462E-3,9.307009E-3,9.7468786E-2,-2.6139313E-1,-4.1132398E-2,1.748581E-2,3.1540528E-2,7.998693E-2,2.140466E-1,-1.7831215E-1,-2.825526E-1,-1.4985313E-1,-3.467525E-2,-5.6049544E-1,-9.986859E-3,1.6116574E-2,-1.0327695E-2,1.4700647E-1,3.0877972E-3,-6.101707E-3,-1.8025063E-2,-6.814604E-3,-6.22091E-3,5.999928E-2,1.6007181E-2,1.5136747E-1,2.8882807E-1,-1.9712232E-1,-3.472857E-4,-3.3322403E-1,-1.8675955E-1,-1.2366861E-1,-2.0969711E-1,1.0904815E-2,-4.9897004E-2,-1.50895035E-2,-3.368131E-2,-1.0956182E-2,7.0743464E-2,1.2300843E-1,1.2319498E-2,4.746086E-2,-6.541598E-2,1.8706367E-3,-3.4743544E-2,7.546723E-2,-7.0990347E-3,2.2647834E-1,9.6260704E-2,3.17631E-1,-4.2286147E-3,-1.1765553E-3,-2.1324803E-1,-9.120772E-3,-3.676226E-1,-1.9430023E-3,-1.1383385E-2,-1.3343324E-2,-1.08061954E-1,-2.5948998E-1,-1.0164127E-1,-2.1878144E-2,-9.547743E-2,-6.876728E-2,1.8321907E-2,-7.173154E-2,9.2431605E-2,9.3851976E-2,2.2049987E-1,-1.6503694E-3,9.50001E-2,1.0518869E-3,-6.6644624E-3,-3.2587184E-3,2.4912076E-4,4.0731695E-2,1.7590047E-1,2.085931E-3,2.5166938E-1,1.1742903E-1,-7.5403583E-4,2.8857383E-1,2.0203322E-2,-4.8413756E-3,-1.0262682E-2,-2.3152672E-2,-1.2201006E-2,3.9538456E-4,-1.1738142E-1,-1.7621845E-1,-3.584483E-1,-1.6830821E-3,-6.368249E-3,-3.3268414E-2,4.4435696E-3,-1.056216E-2,-6.304716E-2,-4.1113973E-2,-1.3080865E-2,4.2379634E-3,1.17666805E-2,6.068921E-4,-8.140293E-3,1.7759284E-1,4.599027E-2,8.028012E-3,6.2466465E-2,6.238012E-3,1.3677929E-2,5.5254367E-3,1.4653923E-3,5.2801795E-2,-4.7400123E-3,2.0351318E-1,2.4189707E-3,1.3605975E-1,1.3544919E-2,1.4673632E-1,4.1701887E-2,3.1746763E-1,4.824167E-3,-1.3392566E-1,-5.2085243E-2,-9.941129E-3,-2.5528905E-3,-2.013959E-2,-8.952433E-3,-6.040955E-2,2.0263355E-2,-6.8422225E-3,-1.0169511E-1,-1.3024366E-1,3.0384844E-2,1.9406306E-2,-1.3725142E-1,5.4798927E-3,1.0472691E-2,-9.15994E-3,1.6057804E-1,8.553412E-2,-2.21314E-3,-2.292266E-3,6.6084415E-2,3.7688268E-3,1.0965808E-2,2.277029E-3,9.037467E-3,1.8887774E-3,1.8174064E-1,-4.352378E-3,5.6101177E-3,1.2731254E-2,2.1059021E-2,-4.447894E-3,-8.290691E-3,-9.5859176E-4,-4.9006348E-3,-3.3029483E-3,1.3655274E-4,-6.0596596E-4,6.691559E-3,1.4183397E-3,-5.456215E-3,-1.0743662E-3,-5.308265E-3,-2.358725E-3,-1.7983975E-2,3.2831766E-3,-2.9287226E-3,-1.443607E-3,2.3336082E-3,-1.4271461E-3,-1.09691955E-2,-2.4505972E-3,4.282196E-3,3.6186844E-3,8.956619E-3,6.3879024E-3,1.4070527E-3,3.7921395E-3,-2.8258574E-4,4.579095E-3,1.1755486E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,39,-1,-1,41,43,45,47,49,51,53,-1,55,-1,57,59,-1,-1,61,-1,63,-1,65,67,69,-1,71,73,75,77,-1,79,-1,-1,81,83,85,-1,87,89,-1,91,93,-1,95,97,99,-1,-1,101,-1,103,-1,-1,-1,105,107,109,111,113,115,117,119,121,123,125,-1,127,-1,-1,-1,-1,129,131,-1,133,135,-1,137,-1,-1,-1,-1,-1,-1,139,141,143,-1,-1,145,-1,-1,147,149,-1,151,-1,-1,-1,153,155,-1,157,-1,-1,-1,-1,159,-1,161,-1,163,-1,165,167,169,-1,171,173,-1,-1,-1,-1,175,177,179,181,183,185,187,189,-1,-1,191,193,195,-1,-1,197,-1,-1,-1,-1,-1,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.472691E0,4.1881056E0,3.4541159E0,1.102148E0,6.5913343E-1,2.5409234E-1,2.267335E0,1.0844002E0,5.5292034E-1,5.099337E-1,3.801884E-1,0E0,0E0,3.2236576E-2,9.3524885E-1,1.5972185E-1,8.807533E-1,3.0190825E-1,0E0,3.904989E-1,4.203334E-1,1.3164288E-1,9.536655E-2,0E0,0E0,4.5328116E-1,5.843706E-1,1.5915251E-1,8.8240385E-2,1.6680622E-1,6.175291E-1,1.5608215E-1,0E0,3.527534E-1,0E0,1.3661587E-1,1.0037583E-1,0E0,0E0,3.179416E-2,0E0,2.892081E-1,0E0,2.8863275E-1,6.8639946E-1,1.191318E-1,0E0,1.617837E-2,1.018261E-1,1.9560921E-1,1.673199E-1,0E0,1.7670676E-1,0E0,0E0,2.720757E-1,2.5074163E-1,1.242733E-1,0E0,8.248366E-2,9.366595E-2,0E0,2.192907E-2,2.6843485E-1,0E0,1.3023949E-1,1.091595E-1,6.8546295E-2,0E0,0E0,1.626277E-2,0E0,6.1178565E-2,0E0,0E0,0E0,9.012538E-2,1.3460457E-1,2.5668189E-2,1.2153788E-1,2.1443698E-1,3.0847922E-1,3.5513106E-1,9.9841386E-2,2.647276E-1,9.19843E-2,3.1072438E-2,0E0,1.6444102E-2,0E0,0E0,0E0,0E0,1.0921896E-1,5.756992E-2,0E0,8.970201E-2,7.8216374E-2,0E0,2.2225118E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.512534E-2,5.880198E-2,4.311502E-2,0E0,0E0,1.1891006E-1,0E0,0E0,9.488122E-2,3.1621808E-1,0E0,2.2157183E-1,0E0,0E0,0E0,4.2602062E-2,2.9216972E-1,0E0,8.302341E-2,0E0,0E0,0E0,0E0,7.946089E-2,0E0,4.6670675E-2,0E0,3.6872208E-2,0E0,9.26131E-2,1.4147733E-1,1.1204529E-1,0E0,8.557236E-2,2.6560873E-2,0E0,0E0,0E0,0E0,3.8138762E-2,1.1833511E-1,8.298921E-2,2.4258286E-2,4.31387E-1,1.11325234E-1,1.4811152E-1,9.921907E-2,0E0,0E0,1.4794476E-1,2.5888652E-2,7.130657E-2,0E0,0E0,5.825652E-2,0E0,0E0,0E0,0E0,0E0,9.493512E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,35,35,36,36,39,39,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,52,52,55,55,56,56,57,57,59,59,60,60,62,62,63,63,65,65,66,66,67,67,70,70,72,72,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,88,88,93,93,94,94,96,96,97,97,99,99,106,106,107,107,108,108,111,111,114,114,115,115,117,117,121,121,122,122,124,124,129,129,131,131,133,133,135,135,136,136,137,137,139,139,140,140,145,145,146,146,147,147,148,148,149,149,150,150,151,151,152,152,155,155,156,156,157,157,160,160,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,40,-1,-1,42,44,46,48,50,52,54,-1,56,-1,58,60,-1,-1,62,-1,64,-1,66,68,70,-1,72,74,76,78,-1,80,-1,-1,82,84,86,-1,88,90,-1,92,94,-1,96,98,100,-1,-1,102,-1,104,-1,-1,-1,106,108,110,112,114,116,118,120,122,124,126,-1,128,-1,-1,-1,-1,130,132,-1,134,136,-1,138,-1,-1,-1,-1,-1,-1,140,142,144,-1,-1,146,-1,-1,148,150,-1,152,-1,-1,-1,154,156,-1,158,-1,-1,-1,-1,160,-1,162,-1,164,-1,166,168,170,-1,172,174,-1,-1,-1,-1,176,178,180,182,184,186,188,190,-1,-1,192,194,196,-1,-1,198,-1,-1,-1,-1,-1,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,4.9963706E5,1E0,4E1,5.1071826E-8,9.3326636E-2,2.2968E4,7.7E1,6.0078123E-6,3.9742767E2,2.3238889E2,-1.826237E-2,-5.0650304E-3,1.63757E4,1.0665628E6,1E0,5.41E2,8E0,-1.0701462E-3,4.5620965E6,2.3818183E0,5.90352E5,2.01E2,1.748581E-2,3.1540528E-2,7.6255713E3,7.098E3,1.6576804E7,1.5389474E1,1.3306123E1,1.1E1,1E1,-9.986859E-3,6.946813E7,-1.0327695E-2,4.284608E6,1.2218E4,-6.101707E-3,-1.8025063E-2,2.1470589E1,-6.22091E-3,3.0070068E7,1.6007181E-2,1.1057851E1,4.1325716E7,5E0,-3.472857E-4,9E0,7.8918E4,3.9E1,1.9470909E2,1.0904815E-2,1.9E1,-1.50895035E-2,-3.368131E-2,2.1578947E2,5.4E1,2.483E3,1.2319498E-2,2.476353E-2,4.9E1,1.8706367E-3,5.1042255E2,1.5142261E3,-7.0990347E-3,1.2157188E6,1.0975045E7,2.1511E4,-4.2286147E-3,-1.1765553E-3,1.798E3,-9.120772E-3,2.1609572E7,-1.9430023E-3,-1.1383385E-2,-1.3343324E-2,8.8E1,3.50986E5,1.84E2,4.233978E0,1.0201538E-1,1.8E1,3.2E1,1.3287E4,9.538462E0,1.6939986E9,6.257143E2,-1.6503694E-3,2.0316172E6,1.0518869E-3,-6.6644624E-3,-3.2587184E-3,2.4912076E-4,1.1290322E1,3.7798166E0,2.085931E-3,1E0,2.9157894E0,-7.5403583E-4,2E0,2.0203322E-2,-4.8413756E-3,-1.0262682E-2,-2.3152672E-2,-1.2201006E-2,3.9538456E-4,3.201807E6,2.511352E6,1.6595395E5,-1.6830821E-3,-6.368249E-3,6.077143E5,4.4435696E-3,-1.056216E-2,3.401282E1,2.9652428E6,-1.3080865E-2,6.485856E7,1.17666805E-2,6.068921E-4,-8.140293E-3,1.14E3,1.2768175E6,8.028012E-3,1.9882522E0,6.238012E-3,1.3677929E-2,5.5254367E-3,1.4653923E-3,4.6712E4,-4.7400123E-3,3.8153846E0,2.4189707E-3,2.1512408E5,1.3544919E-2,3.6522612E6,4.488263E0,1.9717247E9,4.824167E-3,1E0,2.707673E6,-9.941129E-3,-2.5528905E-3,-2.013959E-2,-8.952433E-3,1.3205625E7,4.6134964E2,5.49E2,1.8571428E0,1.925508E6,2.6210527E0,1.7587205E6,3.2075E4,5.4798927E-3,1.0472691E-2,1.4E1,1.281E3,1.0092541E8,-2.21314E-3,-2.292266E-3,1.2820834E7,3.7688268E-3,1.0965808E-2,2.277029E-3,9.037467E-3,1.8887774E-3,5.306E3,-4.352378E-3,5.6101177E-3,1.2731254E-2,2.1059021E-2,-4.447894E-3,-8.290691E-3,-9.5859176E-4,-4.9006348E-3,-3.3029483E-3,1.3655274E-4,-6.0596596E-4,6.691559E-3,1.4183397E-3,-5.456215E-3,-1.0743662E-3,-5.308265E-3,-2.358725E-3,-1.7983975E-2,3.2831766E-3,-2.9287226E-3,-1.443607E-3,2.3336082E-3,-1.4271461E-3,-1.09691955E-2,-2.4505972E-3,4.282196E-3,3.6186844E-3,8.956619E-3,6.3879024E-3,1.4070527E-3,3.7921395E-3,-2.8258574E-4,4.579095E-3,1.1755486E-2],"split_indices":[2,43,17,3,52,53,12,44,52,67,67,0,0,43,43,15,2,32,0,62,68,9,0,0,0,67,2,5,73,73,10,8,0,7,0,43,44,0,0,61,0,58,0,71,66,0,0,0,1,2,4,0,8,0,0,70,0,2,0,53,8,0,4,67,0,62,60,2,0,0,9,0,60,0,0,0,44,60,10,68,53,3,3,1,73,46,4,0,62,0,0,0,0,69,68,0,29,69,0,32,0,0,0,0,0,0,9,5,43,0,0,62,0,0,73,43,0,7,0,0,0,2,62,0,68,0,0,0,0,1,0,69,0,48,0,60,69,5,0,8,5,0,0,0,0,9,4,0,68,43,71,43,1,0,0,3,44,7,0,0,9,0,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.86E2,7.37E2,2.49E2,3.66E2,3.71E2,1.5E1,2.34E2,3.47E2,1.9E1,3.32E2,3.9E1,9E0,6E0,1.3E1,2.21E2,7.9E1,2.68E2,1.5E1,4E0,2.43E2,8.9E1,1E1,2.9E1,6E0,7E0,8.8E1,1.33E2,4.9E1,3E1,1.22E2,1.46E2,9E0,6E0,2.37E2,6E0,5.8E1,3.1E1,6E0,4E0,2.2E1,7E0,8.3E1,5E0,7.4E1,5.9E1,4.4E1,5E0,1.8E1,1.2E1,8.7E1,3.5E1,7E0,1.39E2,5E0,4E0,1.59E2,7.8E1,5E1,8E0,1.9E1,1.2E1,8E0,1.4E1,7.8E1,5E0,3E1,4.4E1,5.5E1,4E0,4E0,4E1,5E0,1.3E1,4E0,8E0,6E0,8.1E1,2.3E1,1.2E1,8.7E1,5.2E1,5.3E1,1.06E2,1E1,6.8E1,4E1,1E1,7E0,1.2E1,6E0,6E0,7E0,7E0,5.9E1,1.9E1,4E0,2.6E1,3.7E1,7E0,4.7E1,8E0,4E0,3.6E1,4E0,9E0,6E0,7.5E1,1.4E1,9E0,5E0,7E0,8E1,7E0,9E0,4.3E1,4.8E1,5E0,1.01E2,5E0,6E0,4E0,2.3E1,4.5E1,1E1,3E1,6E0,4E0,8E0,4E0,5.5E1,4E0,1.5E1,4E0,8E0,1.8E1,2.6E1,1.1E1,4E1,7E0,5.9E1,1.6E1,1E1,4E0,5E0,4E0,5.3E1,2.7E1,1.8E1,2.5E1,2.1E1,2.7E1,9.2E1,9E0,1.2E1,1.1E1,3.1E1,1.4E1,2.5E1,5E0,6E0,4.9E1,4E0,1.1E1,4E0,4E0,7E0,1.9E1,4E0,7E0,3.3E1,7E0,3.4E1,2.5E1,1.1E1,5E0,4.5E1,8E0,2.2E1,5E0,1.4E1,4E0,4E0,2.1E1,1.7E1,4E0,1.9E1,8E0,3.5E1,5.7E1,5E0,4E0,2.2E1,9E0,5E0,9E0,1.2E1,1.3E1,4E1,9E0,1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[-3.7201808E-3,-5.2778203E-2,1.3622658E-1,-1.0779125E-1,2.8416028E-2,1.6271453E-1,-2.0433286E-1,-1.8276516E-1,-6.9586866E-2,-1.660413E-2,7.807986E-2,4.2367348E-1,1.4083019E-1,-1.9697698E-2,-1.0189746E-1,-1.9264838E-1,4.378566E-3,-1.3625574E-1,-3.219716E-2,-8.073628E-3,-9.6092345E-3,-1.9239467E-2,1.0486154E-1,2.2238046E-2,1.0093383E-2,9.120168E-2,2.448266E-1,-6.5098745E-3,1.9902634E-4,4.2371196E-3,-2.030865E-1,-1.2584434E-1,-1.5233027E-2,-1.4324683E-2,-1.6117576E-1,4.5197673E-2,-2.8210383E-2,6.2002182E-2,-7.5146645E-2,1.563233E-1,5.1622476E-2,5.3254947E-2,1.9003272E-1,2.681881E-1,-3.165949E-3,-1.8626359E-1,-3.1143346E-1,-1.5406427E-1,-8.2578465E-2,-1.1710958E-3,-1.3650678E-1,-2.4947971E-1,4.1078646E-3,7.8360856E-2,-5.258918E-3,-4.8107103E-2,3.313864E-2,3.8908428E-4,6.100071E-3,-1.3480869E-1,-2.0037354E-3,1.3065596E-1,1.4026982E-2,1.3842005E-1,1.3194412E-2,-2.2411019E-2,9.1375254E-2,2.3884837E-1,1.7470121E-4,2.2469002E-1,3.813664E-1,-2.4950086E-1,-1.489723E-1,-1.7351644E-2,-7.251859E-3,-1.0222929E-1,-1.8334144E-1,-1.1884007E-1,-1.799592E-2,-4.8711717E-2,2.3742288E-2,-8.246425E-3,-1.3399397E-6,-1.6387108E-1,-2.0294687E-2,2.10715E-2,1.376623E-1,-1.5259614E-2,-3.0737117E-2,1.5241931E-2,4.583055E-3,-1.5907167E-3,-9.827345E-3,2.0346134E-3,-2.6266773E-3,7.951623E-2,1.6847312E-1,1.6358165E-1,1.86106E-3,3.7603915E-2,-8.082834E-3,-1.2975219E-2,1.410646E-2,2.0414834E-1,4.896818E-2,3.3730572E-1,1.6533695E-1,8.374456E-3,-6.9652623E-3,2.3724182E-1,2.6215327E-3,2.0265052E-2,1.1777775E-2,-1.5467033E-2,-2.1312988E-1,-9.513864E-2,-2.3620494E-1,-2.4398442E-3,-1.3165367E-1,-2.6639467E-3,-1.9605726E-1,-1.4719027E-1,-5.896057E-4,-3.652433E-3,1.4035088E-2,3.0709058E-3,-9.1966264E-2,1.1734547E-2,6.3127475E-3,-2.4482403E-3,-2.0292285E-1,-2.0088296E-2,5.6573905E-2,4.427066E-3,1.049974E-2,5.0690905E-3,-8.101064E-2,5.6614406E-2,-3.879958E-2,-1.5586324E-3,4.797947E-3,1.0187592E-2,1.1876606E-1,3.6456757E-3,8.757406E-3,-4.4908606E-2,6.867655E-2,5.0229605E-2,-5.479357E-3,3.3455284E-3,2.4078888E-1,-7.904197E-3,6.843712E-2,1.9115336E-2,7.6878057E-3,9.614157E-2,1.0431385E-2,2.4852222E-1,4.077983E-3,-1.2565714E-1,-1.1456709E-2,1.870128E-3,-1.1665622E-1,-2.6607448E-1,-4.425153E-3,-7.7513563E-3,-3.652908E-3,-2.0956075E-1,-3.3782672E-3,-8.432478E-3,-3.505158E-3,2.7623072E-3,-1.2561356E-3,3.647485E-3,-1.8755503E-2,-2.1379977E-2,-7.443925E-3,7.988813E-2,-5.0174096E-3,-1.187403E-2,-4.500849E-3,1.4127657E-3,-2.7240717E-3,2.1736983E-4,3.951917E-3,3.2079108E-2,-4.9249128E-2,-1.2078163E-1,-3.1795375E-2,3.62021E-3,8.30691E-5,-3.098265E-3,3.940629E-4,8.457033E-2,8.49701E-3,2.0760712E-3,-4.7432054E-3,7.246409E-3,2.5563816E-2,-2.993375E-2,1.2704432E-1,6.986435E-3,1.3566222E-2,-2.964809E-3,9.289515E-2,4.3040756E-4,6.393469E-3,1.6590248E-1,2.6738778E-1,-1.8351442E-3,-8.37886E-3,-3.1916583E-3,-8.34454E-3,-1.3724857E-2,-5.4687564E-3,-5.771147E-3,-1.0581565E-2,-2.7714788E-3,1.506153E-3,-3.0180826E-3,2.8991923E-3,6.0761184E-3,-1.266455E-3,-1.4005963E-3,2.159663E-3,3.5238713E-3,-1.0352954E-3,-4.372605E-3,7.488627E-5,-1.9527575E-3,-6.417076E-3,1.552799E-4,-3.811379E-3,5.5943653E-3,1.0061627E-3,4.105663E-3,1.9326035E-4,-3.472781E-3,-3.0725062E-4,9.633285E-3,2.2896563E-3,5.334121E-3,-1.3267531E-3,3.5266308E-3,1.052502E-2,5.9843413E-3,1.3464372E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,35,-1,37,39,-1,-1,41,43,-1,-1,-1,45,47,-1,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,83,-1,85,-1,87,89,-1,-1,91,93,95,-1,97,99,101,103,105,107,109,111,113,115,-1,-1,117,119,121,123,125,127,-1,-1,129,-1,131,133,-1,135,137,-1,-1,-1,-1,-1,139,141,143,-1,145,-1,-1,147,149,151,153,155,-1,-1,157,-1,-1,-1,-1,159,161,163,-1,165,-1,167,169,-1,-1,171,173,175,177,-1,-1,179,181,183,-1,-1,185,187,189,191,-1,-1,-1,193,-1,-1,195,197,199,-1,-1,201,-1,203,-1,-1,205,-1,207,-1,209,-1,-1,211,213,-1,-1,-1,215,-1,-1,-1,-1,-1,-1,217,219,-1,221,223,-1,-1,-1,-1,-1,-1,225,227,229,231,-1,-1,-1,-1,233,-1,-1,-1,-1,235,237,239,-1,-1,-1,241,-1,-1,243,245,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.167918E0,3.4558551E0,2.4778628E0,1.3077388E0,7.011248E-1,1.3794842E0,4.1143525E-1,4.4342375E-1,7.6088107E-1,2.6771158E-1,3.903674E-1,1.1971378E-1,1.185389E0,0E0,6.275605E-2,4.638586E-1,0E0,1.8007803E-1,4.5411432E-1,1.7183612E-1,0E0,1.5454498E-1,3.1248403E-1,0E0,0E0,5.899303E-1,5.674114E-1,0E0,0E0,0E0,2.0628405E-1,1.1669934E-1,0E0,2.810884E-1,5.5220246E-1,2.4097246E-1,1.4318895E-1,5.1424187E-2,8.7006986E-2,1.8339312E-1,1.9496588E-1,3.3952376E-1,4.0775204E-1,2.5892353E-1,0E0,2.6468086E-1,1.1940634E-1,7.686496E-2,1.0090348E-1,1.8947577E-1,9.7418875E-2,2.204994E-1,0E0,1.228009E-1,0E0,4.2119813E-1,3.2775342E-2,0E0,0E0,7.869843E-2,2.775989E-2,8.716953E-2,0E0,3.7857294E-2,1.9947144E-1,3.814703E-1,3.6302853E-1,1.9755173E-1,3.0067497E-1,9.8263025E-2,1.8528938E-2,7.134223E-2,3.6730015E-1,0E0,0E0,2.9399306E-2,5.102122E-2,8.232668E-2,3.4359895E-2,1.2347327E-1,1.4074244E-1,0E0,0E0,4.986608E-2,0E0,3.0361911E-2,4.570231E-2,0E0,1.5161939E-1,5.585656E-2,0E0,0E0,0E0,0E0,0E0,6.965536E-2,4.882914E-2,1.5833259E-2,0E0,1.0051168E-1,0E0,0E0,1.785375E-1,8.392513E-2,2.6019835E-1,1.1407268E-1,7.032287E-2,0E0,0E0,6.040573E-2,0E0,0E0,0E0,0E0,7.555199E-2,1.5938577E-1,1.0453725E-1,0E0,1.5230447E-2,0E0,4.2776823E-2,4.1592747E-2,0E0,0E0,2.426122E-2,4.4542007E-2,1.4300957E-1,1.1000459E-1,0E0,0E0,3.2031745E-2,2.1500658E-2,1.6124956E-2,0E0,0E0,7.4936666E-2,6.466657E-2,1.6548164E-2,1.528679E-2,0E0,0E0,0E0,2.2322446E-2,0E0,0E0,6.1698616E-2,1.0243173E-1,1.8237695E-1,0E0,0E0,3.1340778E-2,0E0,1.8110633E-1,0E0,0E0,4.2729057E-2,0E0,2.5779963E-2,0E0,5.0241843E-2,0E0,0E0,1.2749565E-1,7.932532E-2,0E0,0E0,0E0,2.250719E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.642689E-2,6.325996E-2,0E0,1.0886927E-1,1.035655E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.358866E-2,3.941317E-2,2.115795E-2,3.0580873E-2,0E0,0E0,0E0,0E0,2.7175508E-2,0E0,0E0,0E0,0E0,2.6698302E-2,1.5298711E-2,8.4357694E-2,0E0,0E0,0E0,1.2954587E-1,0E0,0E0,4.3611705E-2,7.8309774E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,21,21,22,22,25,25,26,26,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,55,55,56,56,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,78,78,79,79,80,80,83,83,85,85,86,86,88,88,89,89,95,95,96,96,97,97,99,99,102,102,103,103,104,104,105,105,106,106,109,109,114,114,115,115,116,116,118,118,120,120,121,121,124,124,125,125,126,126,127,127,130,130,131,131,132,132,135,135,136,136,137,137,138,138,142,142,145,145,146,146,147,147,150,150,152,152,155,155,157,157,159,159,162,162,163,163,167,167,174,174,175,175,177,177,178,178,185,185,186,186,187,187,188,188,193,193,198,198,199,199,200,200,204,204,207,207,208,208],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,36,-1,38,40,-1,-1,42,44,-1,-1,-1,46,48,-1,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,84,-1,86,-1,88,90,-1,-1,92,94,96,-1,98,100,102,104,106,108,110,112,114,116,-1,-1,118,120,122,124,126,128,-1,-1,130,-1,132,134,-1,136,138,-1,-1,-1,-1,-1,140,142,144,-1,146,-1,-1,148,150,152,154,156,-1,-1,158,-1,-1,-1,-1,160,162,164,-1,166,-1,168,170,-1,-1,172,174,176,178,-1,-1,180,182,184,-1,-1,186,188,190,192,-1,-1,-1,194,-1,-1,196,198,200,-1,-1,202,-1,204,-1,-1,206,-1,208,-1,210,-1,-1,212,214,-1,-1,-1,216,-1,-1,-1,-1,-1,-1,218,220,-1,222,224,-1,-1,-1,-1,-1,-1,226,228,230,232,-1,-1,-1,-1,234,-1,-1,-1,-1,236,238,240,-1,-1,-1,242,-1,-1,244,246,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.9016544E5,4.217427E7,8.147158E4,1.071E3,3.6E1,2.5330253E-2,1.482958E0,5.47E2,2.8461537E0,5.036081E7,6.441311E-1,2.9251662E6,-1.9697698E-2,2.3308511E5,7E0,4.378566E-3,1E0,2.2E1,9.917037E2,-9.6092345E-3,3.1842105E0,8E0,2.2238046E-2,1.0093383E-2,1.5142261E3,5.2964807E0,-6.5098745E-3,1.9902634E-4,4.2371196E-3,4.217427E7,3.72E2,-1.5233027E-2,8.233192E9,3.7725E1,3.7103873E2,3.3486558E6,5.1942E4,1.2839025E1,3.549E3,2.1028344E4,6.648E3,2.856934E7,1.6466942E3,-3.165949E-3,3.175E1,1.5416006E5,3.1E1,1.339646E6,2.9211267E2,1.731E3,4E1,4.1078646E-3,1.1015625E-4,-5.258918E-3,1.2324481E2,3.1847827E2,3.8908428E-4,6.100071E-3,1.5115256E6,1.3254E4,2E0,1.4026982E-2,1.5E1,1.2619E4,1.4076087E0,1.5209424E1,5.6516E4,2.9622424E3,9.244374E6,1.5454545E0,9.2058825E-1,1.8797773E-1,-1.7351644E-2,-7.251859E-3,1.6E1,1.0443479E5,8E0,1.5292561E2,1.6808511E0,1.9099288E7,-8.246425E-3,-1.3399397E-6,3.7509E4,-2.0294687E-2,5.8899284E1,3.5589743E0,-1.5259614E-2,1.6153846E0,2.0487332E-3,4.583055E-3,-1.5907167E-3,-9.827345E-3,2.0346134E-3,-2.6266773E-3,5.441268E7,3.44E2,4.1388846E2,1.86106E-3,3.1609525E2,-8.082834E-3,-1.2975219E-2,1.1555496E8,4.6847186E2,3.97371E5,3.7936268E-3,1.3567028E5,8.374456E-3,-6.9652623E-3,1.562E5,2.6215327E-3,2.0265052E-2,1.1777775E-2,-1.5467033E-2,3.7452831E0,9.935484E-1,1.1408248E0,-2.4398442E-3,2.8003533E0,-2.6639467E-3,7.997723E6,1.9545455E0,-5.896057E-4,-3.652433E-3,2.1790265E2,1E1,9.966666E0,3.8E1,6.3127475E-3,-2.4482403E-3,4.88451E5,9.836066E-1,1.623E3,4.427066E-3,1.049974E-2,4.9390244E0,4.1361522E6,5.41E2,1.3809524E0,-1.5586324E-3,4.797947E-3,1.0187592E-2,1.9505818E8,3.6456757E-3,8.757406E-3,1E0,7E0,2.4358286E7,-5.479357E-3,3.3455284E-3,1E0,-7.904197E-3,2.5711747E5,1.9115336E-2,7.6878057E-3,9.67E2,1.0431385E-2,1.5137369E6,4.077983E-3,2.8877066E4,-1.1456709E-2,1.870128E-3,1.31E2,2.8027795E4,-4.425153E-3,-7.7513563E-3,-3.652908E-3,1.2608696E0,-3.3782672E-3,-8.432478E-3,-3.505158E-3,2.7623072E-3,-1.2561356E-3,3.647485E-3,2.0116506E4,1.4381613E7,-7.443925E-3,4.45E2,1.535E3,-1.187403E-2,-4.500849E-3,1.4127657E-3,-2.7240717E-3,2.1736983E-4,3.951917E-3,1E0,4.0119403E2,5.6666665E0,4.267606E0,3.62021E-3,8.30691E-5,-3.098265E-3,3.940629E-4,9.94E2,8.49701E-3,2.0760712E-3,-4.7432054E-3,7.246409E-3,2.2307692E0,1.5204346E8,8.818731E0,6.986435E-3,1.3566222E-2,-2.964809E-3,1.2792593E2,4.3040756E-4,6.393469E-3,6.000379E8,3.0238653E8,-1.8351442E-3,-8.37886E-3,-3.1916583E-3,-8.34454E-3,-1.3724857E-2,-5.4687564E-3,-5.771147E-3,-1.0581565E-2,-2.7714788E-3,1.506153E-3,-3.0180826E-3,2.8991923E-3,6.0761184E-3,-1.266455E-3,-1.4005963E-3,2.159663E-3,3.5238713E-3,-1.0352954E-3,-4.372605E-3,7.488627E-5,-1.9527575E-3,-6.417076E-3,1.552799E-4,-3.811379E-3,5.5943653E-3,1.0061627E-3,4.105663E-3,1.9326035E-4,-3.472781E-3,-3.0725062E-4,9.633285E-3,2.2896563E-3,5.334121E-3,-1.3267531E-3,3.5266308E-3,1.052502E-2,5.9843413E-3,1.3464372E-2],"split_indices":[2,43,60,43,2,0,53,57,2,68,7,53,43,0,48,3,0,113,3,48,0,69,8,0,0,67,53,0,0,0,60,2,0,46,71,70,43,1,73,10,48,44,60,67,0,70,48,10,9,4,2,3,0,53,0,67,4,0,0,43,44,8,0,3,44,69,71,9,67,60,68,68,57,0,0,3,43,8,67,68,9,0,0,1,0,62,69,0,68,53,0,0,0,0,0,7,10,4,0,67,0,0,5,4,1,72,48,0,0,10,0,0,0,0,69,68,53,0,68,0,9,68,0,0,4,3,73,0,0,0,9,71,44,0,0,71,60,2,68,0,0,0,5,0,0,26,3,5,0,0,23,0,43,0,0,0,0,62,0,60,0,0,44,48,0,0,0,68,0,0,0,0,0,0,48,5,0,44,2,0,0,0,0,0,0,26,4,73,71,0,0,0,0,10,0,0,0,0,71,7,71,0,0,0,73,0,0,5,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,7.72E2,2.7E2,4.6E2,3.12E2,2.51E2,1.9E1,1.54E2,3.06E2,1.64E2,1.48E2,1.8E1,2.33E2,5E0,1.4E1,1.49E2,5E0,1.09E2,1.97E2,1.58E2,6E0,3.2E1,1.16E2,1.3E1,5E0,1.59E2,7.4E1,1E1,4E0,5E0,1.44E2,1.05E2,4E0,1.74E2,2.3E1,4.3E1,1.15E2,1.3E1,1.9E1,5.8E1,5.8E1,1.16E2,4.3E1,6.9E1,5E0,1.27E2,1.7E1,6.2E1,4.3E1,1.58E2,1.6E1,1.7E1,6E0,3.6E1,7E0,8.7E1,2.8E1,8E0,5E0,1E1,9E0,5.1E1,7E0,1.7E1,4.1E1,3.9E1,7.7E1,3.4E1,9E0,5.2E1,1.7E1,4.5E1,8.2E1,1.1E1,6E0,2.4E1,3.8E1,2.7E1,1.6E1,5.4E1,1.04E2,1.2E1,4E0,1.3E1,4E0,1.9E1,1.7E1,4E0,8.3E1,2.3E1,5E0,5E0,5E0,5E0,4E0,2.3E1,2.8E1,1.3E1,4E0,3.7E1,4E0,4E0,3.5E1,2E1,5.7E1,1.3E1,2.1E1,4E0,5E0,4.8E1,4E0,1E1,7E0,1.1E1,3.4E1,5.2E1,3E1,1E1,1.4E1,4E0,3.4E1,2.1E1,6E0,5E0,1.1E1,2.5E1,2.9E1,9.5E1,9E0,4E0,9E0,9E0,1E1,1.3E1,4E0,4.9E1,3.4E1,1.3E1,1E1,4E0,1.9E1,1.2E1,1.6E1,4E0,9E0,1E1,2.7E1,2.8E1,7E0,5E0,1.5E1,4E0,5.3E1,8E0,5E0,1.1E1,1E1,4.4E1,4E0,1.1E1,2.3E1,7E0,4.5E1,2.4E1,6E0,7E0,7E0,3E1,4E0,1.3E1,8E0,5E0,6E0,5E0,2E1,1.5E1,1.4E1,1.8E1,7.7E1,5E0,4E0,4E0,5E0,4E0,6E0,3.3E1,1.6E1,1.8E1,1.6E1,9E0,4E0,6E0,4E0,1.2E1,4E0,4E0,6E0,8E0,1.9E1,1.4E1,1.4E1,7E0,8E0,8E0,4.5E1,4E0,7E0,1E1,3.4E1,5E0,6E0,2.7E1,1.8E1,1.9E1,5E0,7E0,2.3E1,1.1E1,9E0,1E1,5E0,1.2E1,6E0,5.2E1,2.5E1,1.8E1,1.5E1,8E0,8E0,4E0,1.4E1,1E1,6E0,7E0,5E0,4E0,1.5E1,4E0,1E1,6E0,8E0,3.8E1,7E0,5E0,5E0,6E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"247","size_leaf_vector":"1"}},{"base_weights":[-5.6042373E-3,-6.0386825E-2,1.2993024E-1,-1.5974328E-1,-1.3175338E-2,-3.243074E-1,1.5108468E-1,1.9640605E-1,-1.7365013E-1,-8.107625E-3,-2.2728685E-2,-2.524669E-2,-1.6484608E-1,6.4293586E-2,1.8931831E-1,1.4133215E-2,2.1995762E-3,-1.455832E-1,-2.8846595E-1,-5.3682692E-2,3.3824727E-2,-1.0733918E-2,-2.974399E-3,-2.7115382E-2,1.1999552E-1,2.8986552E-1,1.5005909E-1,-2.1175928E-1,-9.8635666E-2,-4.4349626E-1,-1.7600514E-1,-8.8702016E-2,5.1713306E-3,8.3594695E-2,-7.110025E-3,7.0060813E-3,-8.400357E-3,1.3772951E-1,-3.594129E-3,1.9723479E-2,2.1709171E-1,2.184235E-1,9.141413E-2,-2.9992566E-1,-1.8575497E-1,4.8802393E-3,-1.14146076E-1,-3.0517232E-2,-1.2244924E-2,-2.369582E-3,-2.1085495E-1,-7.84992E-2,-1.6382184E-2,-1.09717175E-1,3.299895E-2,6.779765E-2,2.2247683E-1,7.53344E-3,-1.04338124E-1,5.3205886E-3,-2.1514324E-2,6.215589E-2,1.7960593E-1,2.1804802E-3,2.429715E-1,1.6468567E-1,2.24881E-2,1.0880122E-1,-8.18845E-3,-6.986421E-3,-1.8994104E-2,-1.3678305E-1,-2.2019397E-1,-1.6975993E-1,-6.656845E-2,-1.4315485E-2,-1.4628866E-1,6.1695995E-3,-8.906914E-2,-2.9593755E-2,-1.0161552E-2,1.6191307E-2,9.682485E-3,8.215516E-3,1.0017252E-1,4.7066435E-3,1.2789598E-2,9.347914E-2,-1.1641701E-2,-4.5176525E-2,-9.938889E-3,-4.856461E-4,-4.486872E-3,1.120196E-1,6.076592E-3,2.1792844E-1,3.5728407E-3,1.1433973E-1,1.3062398E-2,1.870648E-1,-2.1032027E-3,1.7800152E-1,5.5134304E-2,-8.049488E-3,-4.2519826E-2,-5.101213E-3,-2.3152794E-1,-2.1118683E-1,-7.282702E-2,-1.3332805E-1,-1.44184865E-2,-2.095544E-3,-8.413984E-3,-6.916259E-2,-1.6833213E-1,-3.8833516E-3,6.253603E-4,6.3711755E-2,-9.6031455E-3,3.47807E-2,-7.8296244E-2,7.642113E-2,1.8293333E-1,3.4571737E-2,7.424689E-3,5.619764E-3,-2.1548659E-2,-3.5654642E-3,-3.106019E-4,-3.0075978E-3,1.2585211E-3,7.483405E-3,2.3891607E-3,-2.4689632E-3,2.5253573E-3,1.1354262E-2,4.7459104E-3,8.334024E-3,1.1742724E-3,1.5804507E-2,1.4363039E-1,3.170089E-3,2.0704596E-1,-5.1479917E-3,1.760666E-1,7.642351E-4,-4.300111E-3,-1.135412E-2,-5.268347E-3,-1.4717348E-1,-1.21041965E-2,-1.2980774E-1,1.5836431E-3,-2.2744222E-3,-7.625631E-3,3.1777162E-2,-4.873954E-3,-1.0042541E-1,3.3298023E-2,-3.259218E-1,-6.5730415E-2,1.2241098E-3,4.7077537E-3,-5.6773257E-3,1.20887365E-2,1.1478007E-2,5.6452784E-3,-5.6093144E-3,-9.016676E-4,1.0075476E-1,1.3852444E-2,3.4378578E-3,1.0112955E-2,4.5929444E-3,-4.114629E-3,-7.141527E-2,9.774558E-3,7.47523E-2,1.7047983E-1,2.2311167E-1,4.683377E-3,2.7128326E-2,-5.1033003E-3,4.0393667E-3,9.36807E-3,-8.357427E-3,-3.6020183E-3,-2.5368736E-3,-7.880929E-3,-6.4360676E-4,3.2814434E-3,1.7691898E-3,-4.986447E-3,-4.4692855E-4,4.286147E-3,-2.5783906E-2,-3.608875E-3,1.1859142E-4,-5.854548E-3,1.5286996E-3,-3.8179467E-3,-2.0952502E-3,1.3314765E-3,8.042869E-3,3.325956E-3,-1.8886509E-3,2.3460074E-3,-4.840639E-3,1.867129E-4,-3.2826783E-3,2.0799898E-3,-2.2710275E-3,6.25005E-3,6.351626E-3,1.0893004E-2,1.2034015E-2,6.213594E-3,-2.6580174E-3,2.4874485E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,51,53,55,57,59,-1,61,-1,-1,63,65,67,69,71,-1,73,-1,-1,-1,75,77,-1,79,81,83,85,87,89,-1,91,93,95,-1,97,99,-1,101,-1,-1,-1,103,105,107,109,-1,111,-1,113,115,-1,117,-1,119,121,-1,-1,123,125,127,-1,129,-1,131,133,135,-1,137,-1,139,-1,141,143,-1,145,-1,147,149,151,153,155,-1,-1,157,159,-1,-1,161,163,165,167,169,171,173,-1,-1,175,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,177,-1,179,181,183,-1,-1,-1,-1,185,-1,187,-1,-1,-1,189,-1,191,193,195,197,-1,-1,-1,199,201,-1,-1,-1,203,205,-1,-1,-1,-1,207,209,211,213,215,-1,217,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3511014E0,3.3048859E0,2.787603E0,1.1584172E0,1.1582612E0,3.7047112E-1,8.9373446E-1,1.3209131E-1,6.6373825E-1,9.09574E-1,0E0,0E0,4.606831E-2,4.360445E-1,7.025628E-1,0E0,0E0,5.321238E-1,6.542058E-1,4.703731E-1,5.058501E-1,0E0,0E0,1.7548345E-1,1.9661987E-1,4.259796E-1,5.3330636E-1,1.1165476E-1,3.478999E-1,4.8341203E-1,1.0218817E-1,3.601104E-1,2.785689E-1,2.3063636E-1,1.9584063E-1,8.885164E-2,0E0,1.4390928E-1,0E0,0E0,1.4681816E-1,8.283036E-1,3.7429255E-1,1.8588364E-1,6.9387436E-2,0E0,2.5065088E-1,0E0,0E0,0E0,8.2524896E-2,3.2343298E-1,0E0,1.4364082E-1,2.0436262E-1,1.9593135E-1,4.3910086E-2,1.9885078E-1,1.0826635E-1,0E0,3.652046E-2,5.2449413E-2,1.0464311E-1,0E0,1.2625837E-1,2.6254976E-1,0E0,2.575773E-1,0E0,0E0,0E0,8.791059E-2,1.5493631E-2,1.6532314E-1,1.895607E-1,0E0,4.2375714E-2,0E0,1.9821203E-1,2.6976772E-2,0E0,8.028647E-2,0E0,8.7756634E-2,1.1575526E-1,0E0,0E0,8.157377E-2,1.333448E-1,1.5477557E-2,0E0,3.581347E-2,0E0,2.1752939E-2,3.1838868E-2,4.5310736E-2,0E0,4.827781E-2,0E0,2.8081524E-1,0E0,8.42793E-2,3.0708963E-1,0E0,2.8414294E-2,0E0,2.0484447E-2,6.227684E-2,9.599407E-2,5.5920184E-2,1.3799325E-1,0E0,0E0,3.492903E-1,3.9897025E-1,0E0,0E0,2.998861E-2,1.0715246E-1,5.76699E-2,2.0441346E-2,7.929799E-2,3.8353354E-2,1.13076106E-1,0E0,0E0,1.4596242E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.1229825E-2,0E0,1.8127084E-2,1.0157322E-1,1.5413284E-2,0E0,0E0,0E0,0E0,2.324143E-2,0E0,2.1174312E-2,0E0,0E0,0E0,4.045987E-2,0E0,9.058535E-2,6.73502E-2,5.317664E-1,7.1142726E-2,0E0,0E0,0E0,7.634213E-2,2.4961041E-2,0E0,0E0,0E0,6.762332E-2,3.441491E-2,0E0,0E0,0E0,0E0,9.105116E-2,1.6844496E-1,1.0415837E-1,2.7917206E-2,3.1197786E-2,0E0,5.5115484E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,40,40,41,41,42,42,43,43,44,44,46,46,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,64,64,65,65,67,67,71,71,72,72,73,73,74,74,76,76,78,78,79,79,81,81,83,83,84,84,87,87,88,88,89,89,91,91,93,93,94,94,95,95,97,97,99,99,101,101,102,102,104,104,106,106,107,107,108,108,109,109,110,110,113,113,114,114,117,117,118,118,119,119,120,120,121,121,122,122,123,123,126,126,140,140,142,142,143,143,144,144,149,149,151,151,155,155,157,157,158,158,159,159,160,160,164,164,165,165,169,169,170,170,175,175,176,176,177,177,178,178,179,179,181,181],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,52,54,56,58,60,-1,62,-1,-1,64,66,68,70,72,-1,74,-1,-1,-1,76,78,-1,80,82,84,86,88,90,-1,92,94,96,-1,98,100,-1,102,-1,-1,-1,104,106,108,110,-1,112,-1,114,116,-1,118,-1,120,122,-1,-1,124,126,128,-1,130,-1,132,134,136,-1,138,-1,140,-1,142,144,-1,146,-1,148,150,152,154,156,-1,-1,158,160,-1,-1,162,164,166,168,170,172,174,-1,-1,176,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,178,-1,180,182,184,-1,-1,-1,-1,186,-1,188,-1,-1,-1,190,-1,192,194,196,198,-1,-1,-1,200,202,-1,-1,-1,204,206,-1,-1,-1,-1,208,210,212,214,216,-1,218,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1.972052E5,1E0,7E0,5.6E1,1.2877E4,6.9664386E2,1.1634076E4,2.2E1,8.49E2,-2.2728685E-2,-2.524669E-2,4.27E3,4.7567694E5,1.0717949E1,1.4133215E-2,2.1995762E-3,2.71E2,2E0,2.61E2,3.3358974E0,-1.0733918E-2,-2.974399E-3,4.233978E0,1E0,1.4839433E5,5.932344E1,9.2058825E-1,2.2E1,3.428562E2,6.48334E5,1.925673E1,1.727279E-4,1.8857143E0,1E0,2.7662E4,-8.400357E-3,4.6134964E2,-3.594129E-3,1.9723479E-2,8.118812E-1,3.4915986E3,4.023131E2,3.959E3,1.1066655E-2,4.8802393E-3,1.4912975E-1,-3.0517232E-2,-1.2244924E-2,-2.369582E-3,7.336111E4,7E0,-1.6382184E-2,3.4919708E2,8.257819E9,1.295313E6,5.6530495E6,9E0,2.9690266E4,5.3205886E-3,3.257507E5,7.4010696E0,7.403968E2,2.1804802E-3,1.4831979E0,4.2964826E0,2.24881E-2,6.9E1,-8.18845E-3,-6.986421E-3,-1.8994104E-2,1.8187732E4,1.3947369E0,4.8E0,1.5152774E2,-1.4315485E-2,2.234E3,6.1695995E-3,6.83E2,1.1643481E6,-1.0161552E-2,1.4339622E0,9.682485E-3,1.683263E7,1.19216E8,4.7066435E-3,1.2789598E-2,3.1609525E2,3.757994E0,3.6572136E7,-9.938889E-3,1.5204346E8,-4.486872E-3,5E0,8.909296E4,1.4433751E0,3.5728407E-3,9.5139277E-1,1.3062398E-2,2.1412018E6,-2.1032027E-3,1.4320667E3,7.802E-2,-8.049488E-3,3.1416278E2,-5.101213E-3,5.0710382E0,5.449711E8,8.39E2,2.9157894E0,3.38E2,-2.095544E-3,-8.413984E-3,3.1540035E9,1.5077367E3,-3.8833516E-3,6.253603E-4,2.5677419E0,2.2E1,4.07E2,4.1957852E2,1.8454931E8,5.4007E7,3.7066666E2,7.424689E-3,5.619764E-3,3.35E2,-3.5654642E-3,-3.106019E-4,-3.0075978E-3,1.2585211E-3,7.483405E-3,2.3891607E-3,-2.4689632E-3,2.5253573E-3,1.1354262E-2,4.7459104E-3,8.334024E-3,1.1742724E-3,1.5804507E-2,4.186E4,3.170089E-3,2.0666666E0,1.7291568E2,7.555942E6,7.642351E-4,-4.300111E-3,-1.135412E-2,-5.268347E-3,5.15E2,-1.21041965E-2,4.0753E4,1.5836431E-3,-2.2744222E-3,-7.625631E-3,7.89E2,-4.873954E-3,8.414097E-1,4.49239E6,5.1942E4,1.9090909E0,1.2241098E-3,4.7077537E-3,-5.6773257E-3,1.6533333E1,4.0449125E5,5.6452784E-3,-5.6093144E-3,-9.016676E-4,1.8452264E6,3.472499E-1,3.4378578E-3,1.0112955E-2,4.5929444E-3,-4.114629E-3,1.1840488E8,7.56484E5,1.2586393E3,7.120907E6,8.847283E8,4.683377E-3,7.25E2,-5.1033003E-3,4.0393667E-3,9.36807E-3,-8.357427E-3,-3.6020183E-3,-2.5368736E-3,-7.880929E-3,-6.4360676E-4,3.2814434E-3,1.7691898E-3,-4.986447E-3,-4.4692855E-4,4.286147E-3,-2.5783906E-2,-3.608875E-3,1.1859142E-4,-5.854548E-3,1.5286996E-3,-3.8179467E-3,-2.0952502E-3,1.3314765E-3,8.042869E-3,3.325956E-3,-1.8886509E-3,2.3460074E-3,-4.840639E-3,1.867129E-4,-3.2826783E-3,2.0799898E-3,-2.2710275E-3,6.25005E-3,6.351626E-3,1.0893004E-2,1.2034015E-2,6.213594E-3,-2.6580174E-3,2.4874485E-3],"split_indices":[2,43,17,3,3,9,67,43,3,2,0,0,2,43,71,0,0,2,32,70,69,0,0,68,6,48,73,68,10,67,46,73,53,68,19,9,0,4,0,0,68,4,73,9,53,0,53,0,0,0,60,3,0,70,46,9,43,3,48,0,43,71,4,0,53,68,0,8,0,0,0,43,68,69,67,0,12,0,2,43,0,68,0,60,5,0,0,67,73,5,0,7,0,3,48,53,0,53,0,60,0,4,72,0,62,0,69,46,2,69,0,0,0,46,48,0,0,69,0,0,4,7,7,4,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,9,0,68,73,1,0,0,0,0,44,0,12,0,0,0,2,0,68,43,1,68,0,0,0,73,43,0,0,0,60,53,0,0,0,0,7,12,4,60,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.88E2,7.04E2,2.84E2,2.26E2,4.78E2,1.2E1,2.72E2,8E0,2.18E2,4.74E2,4E0,4E0,8E0,8.4E1,1.88E2,4E0,4E0,1.77E2,4.1E1,2.27E2,2.47E2,4E0,4E0,3.2E1,5.2E1,5.1E1,1.37E2,7.2E1,1.05E2,1.6E1,2.5E1,1.42E2,8.5E1,1.11E2,1.36E2,2.7E1,5E0,4.8E1,4E0,1.6E1,3.5E1,6.2E1,7.5E1,1.4E1,5.8E1,7E0,9.8E1,6E0,1E1,6E0,1.9E1,1.38E2,4E0,1.6E1,6.9E1,1.01E2,1E1,1.19E2,1.7E1,5E0,2.2E1,1.8E1,3E1,5E0,3E1,5.3E1,9E0,7.1E1,4E0,7E0,7E0,2.6E1,3.2E1,4.4E1,5.4E1,6E0,1.3E1,6E0,1.32E2,1E1,6E0,6.4E1,5E0,3.6E1,6.5E1,4E0,6E0,2.1E1,9.8E1,1.2E1,5E0,1.8E1,4E0,9E0,9E0,2.1E1,9E0,8E0,2.2E1,4.8E1,5E0,3E1,4.1E1,1.8E1,8E0,4E0,2.8E1,3E1,1.4E1,2.3E1,3.1E1,4E0,9E0,1.07E2,2.5E1,4E0,6E0,2.2E1,4.2E1,2.8E1,8E0,5.2E1,1.3E1,1.2E1,9E0,6E0,9.2E1,6E0,6E0,5E0,1.3E1,4E0,5E0,4E0,5E0,1.6E1,5E0,4E0,4E0,9E0,3.9E1,7E0,2.3E1,2.8E1,1.3E1,4E0,4E0,2.4E1,4E0,1.5E1,1.5E1,9E0,5E0,7E0,1.6E1,2.1E1,1E1,8.2E1,2.5E1,9E0,1.6E1,1.2E1,1E1,6E0,3.6E1,2.3E1,5E0,4E0,4E0,3.7E1,1.5E1,4E0,9E0,8E0,4E0,3.5E1,5.7E1,1.2E1,2.7E1,1.9E1,4E0,2.2E1,6E0,4E0,9E0,9E0,6E0,4E0,5E0,1E1,1.1E1,4E0,7.8E1,1.5E1,1E1,4E0,5E0,8E0,8E0,3E1,6E0,5E0,1.8E1,9E0,2.8E1,6E0,9E0,2.4E1,1.1E1,1.7E1,4E1,4E0,8E0,2E1,7E0,1.2E1,7E0,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"219","size_leaf_vector":"1"}},{"base_weights":[-3.7557508E-3,-5.356728E-2,1.4186162E-1,-1.528055E-1,-2.857376E-3,-2.832127E-1,1.6357172E-1,-1.3140261E-1,-3.302597E-1,-1.1550596E-1,1.8353786E-2,-1.8730268E-2,-8.493633E-3,6.835811E-2,2.1737468E-1,-1.6718604E-1,-8.6870246E-2,-5.949048E-1,-1.648869E-1,-9.126658E-2,-1.8444842E-2,-9.19258E-3,6.997228E-2,1.09520346E-1,-7.306444E-3,2.8212065E-1,1.00879565E-1,-1.3432595E-2,-1.5529728E-1,1.274936E-2,-1.0777171E-1,-3.4556713E-2,-1.3500018E-2,6.179515E-4,-2.1677454E-1,-1.4211644E-1,-4.7929402E-2,-4.268616E-2,3.2748528E-2,7.840632E-2,-8.635644E-3,4.638027E-2,1.6900975E-1,-4.1179787E-2,4.0469444E-3,2.3710096E-1,4.330049E-1,2.6837327E-2,1.6909108E-1,-1.4538892E-1,-1.4599904E-2,-1.7023702E-1,-2.100985E-2,-1.3649008E-2,-5.582979E-3,-8.309649E-2,-1.7927557E-1,1.3527782E-3,-6.69579E-2,-7.4526265E-2,4.766257E-2,5.0661497E-2,-6.89106E-2,2.434058E-2,1.3213399E-1,-3.7332933E-2,9.68107E-2,2.0912513E-1,-3.390654E-3,1.5296387E-3,-7.80872E-2,2.8110296E-1,1.1583066E-1,2.4640094E-2,1.1977588E-2,-8.712043E-3,7.95725E-2,1.3394545E-1,1.4692688E-2,-1.6311975E-1,-8.5737474E-2,-1.1284824E-1,-1.3032554E-2,5.8899667E-2,-6.592208E-2,-6.0413913E-3,-1.1566941E-3,-9.1470005E-3,-3.8590443E-3,-8.554575E-2,1.5727554E-3,-5.64944E-2,-1.8141877E-2,-5.6810346E-2,1.16797656E-1,1.7292814E-2,8.1288956E-2,-2.2312049E-2,-7.676224E-3,-3.63934E-2,6.519692E-2,1.5005349E-1,-9.975056E-3,-3.116995E-3,8.4780186E-4,4.3281596E-2,6.0893246E-3,1.16314115E-2,1.1572087E-1,-2.0492954E-2,-7.273446E-3,1.6446128E-2,2.1358742E-1,-1.2744347E-3,1.4747547E-1,1.4035816E-1,-5.8292816E-3,9.476695E-3,5.8280297E-2,-2.1458541E-4,-1.707936E-1,2.1300022E-2,-6.3551646E-3,-1.4219256E-1,2.1160326E-3,2.1370817E-2,5.446219E-3,1.6521169E-3,-8.4359266E-2,-3.255745E-2,-1.2656389E-1,-4.2470887E-2,-1.78845E-1,-9.1749795E-2,1.0411858E-3,1.5287383E-1,1.01232676E-4,1.2553751E-1,-1.4810339E-2,1.0559643E-1,2.5457121E-2,-3.823277E-3,4.6852478E-4,-9.754159E-2,1.8690983E-3,1.0593614E-1,-5.7380903E-3,1.6477413E-1,3.0704768E-4,2.0880303E-3,-2.917539E-3,3.8176242E-3,-2.184008E-4,1.246536E-3,7.871543E-3,-3.0559883E-3,2.195523E-3,2.5345418E-1,1.3537777E-3,9.006914E-3,1.765918E-3,2.7478794E-3,8.435069E-3,-9.773464E-4,4.2581656E-3,-1.0102244E-1,-1.849062E-1,3.1487294E-3,-9.838736E-4,-7.738598E-2,-1.8368787E-1,2.8991008E-3,-1.0890296E-3,-4.64967E-3,-1.084472E-4,-3.0389505E-3,1.0488047E-3,-7.977407E-3,-3.3038724E-3,-7.425573E-2,1.3656159E-2,-1.2256857E-2,-1.0178325E-3,-6.5754154E-3,-2.2484954E-3,8.95076E-3,8.593619E-2,2.5366854E-3,8.639962E-3,5.5932276E-2,-4.6011843E-2,9.361874E-3,7.714603E-2,-1.8858064E-3,6.199686E-2,-1.0518959E-2,-1.4058007E-2,7.20199E-2,1.143643E-2,-2.843269E-3,2.264298E-2,1.04351275E-1,2.014991E-1,5.483878E-3,1.3614059E-2,-6.9777896E-3,-6.994794E-4,-1.0109652E-2,-7.368951E-3,-2.4036414E-4,-4.76596E-3,-1.0060209E-2,-4.8676725E-3,1.4966434E-3,-4.370702E-3,6.086785E-3,-1.6152324E-3,1.3124244E-3,5.8338824E-3,4.190093E-3,1.1915165E-5,-3.4006445E-3,2.697135E-4,7.1319556E-3,2.7713517E-3,4.112183E-4,4.0392824E-3,2.9411428E-3,-2.9955518E-3,2.2150865E-3,5.711781E-3,2.2634848E-3,-1.1109737E-3,2.9519508E-3,8.189894E-3,5.9749857E-3,1.2125979E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,-1,49,-1,51,-1,-1,-1,53,55,57,59,61,63,-1,65,67,69,-1,71,73,75,77,79,-1,81,83,-1,-1,85,87,-1,89,91,93,95,97,99,101,103,105,107,-1,-1,109,111,113,-1,-1,-1,115,117,-1,119,121,123,-1,125,127,-1,-1,-1,-1,129,-1,131,-1,133,135,137,139,141,-1,143,145,147,149,-1,-1,151,-1,-1,153,155,-1,-1,157,-1,159,161,-1,-1,163,-1,165,167,-1,169,-1,171,-1,-1,173,175,177,179,181,183,-1,185,-1,187,189,191,193,-1,-1,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,-1,-1,-1,203,-1,-1,-1,-1,-1,-1,-1,205,207,-1,-1,209,211,-1,-1,-1,-1,-1,-1,-1,-1,213,215,-1,-1,-1,-1,-1,217,-1,-1,219,221,-1,223,-1,225,227,-1,229,-1,-1,231,233,235,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.398838E0,3.8241575E0,2.4457111E0,9.3269825E-1,1.2063813E0,8.332908E-2,1.2555885E0,3.5357976E-1,1.0835652E0,5.137789E-1,6.049515E-1,0E0,0E0,2.842217E-1,1.163269E0,1.4141941E-1,8.260407E-1,2.4481821E-1,1.6939294E-1,1.5846509E-1,0E0,3.9192057E-1,3.3532184E-1,2.1377504E-1,1.096934E-1,5.8340263E-1,2.8839296E-1,0E0,1.4197063E-1,0E0,5.383841E-1,0E0,0E0,0E0,6.994367E-2,5.9669316E-2,6.366733E-2,4.4944263E-1,2.2929724E-1,4.150787E-1,0E0,1.3026503E-1,3.0706972E-1,7.1131304E-2,0E0,3.943305E-1,2.4526739E-1,3.4444174E-1,1.14438E-1,1.05531216E-1,0E0,3.404932E-1,1.5820612E-1,0E0,0E0,3.8243175E-2,1.9299865E-2,0E0,6.674351E-2,6.375787E-1,3.0276778E-1,1.06926024E-1,8.239066E-2,1.8351261E-1,1.848464E-1,2.137516E-2,2.9887542E-2,7.442236E-2,0E0,0E0,7.446534E-2,2.171936E-1,1.0845107E-1,0E0,0E0,0E0,3.1786966E-1,1.2808108E-1,0E0,1.0379219E-1,1.5824623E-1,1.9273269E-1,0E0,3.2899674E-2,5.550968E-2,0E0,0E0,0E0,0E0,5.8910385E-2,0E0,1.8114063E-1,0E0,4.9197737E-2,1.0302907E-1,1.8255906E-1,7.195404E-2,2.8302189E-2,0E0,1.4492512E-1,1.2776071E-1,1.3401198E-1,2.9258605E-2,0E0,0E0,1.7394174E-2,0E0,0E0,4.390958E-2,3.7301842E-2,0E0,0E0,2.3501801E-1,0E0,9.11127E-2,5.402416E-2,0E0,0E0,3.9638434E-2,0E0,6.2061787E-2,2.1410324E-2,0E0,7.721692E-2,0E0,2.1937333E-2,0E0,0E0,3.1817272E-2,2.7194906E-2,2.8265774E-2,1.7941299E-1,1.4329469E-1,1.9219846E-2,0E0,3.5328925E-2,0E0,4.1837946E-2,9.3071856E-2,8.928457E-2,4.6121582E-2,0E0,0E0,3.026628E-1,0E0,1.16652995E-1,2.8318616E-2,1.0767686E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1502409E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.6439435E-2,2.282834E-2,0E0,0E0,2.7371578E-2,3.0427039E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4196849E-1,2.170464E-1,0E0,0E0,0E0,0E0,0E0,1.7299362E-2,0E0,0E0,2.5137812E-2,4.1925438E-2,0E0,3.181915E-2,0E0,1.5427865E-2,5.6512907E-2,0E0,2.3892798E-2,0E0,0E0,1.5942406E-2,5.9474915E-2,1.2006509E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,55,55,56,56,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,76,76,77,77,79,79,80,80,81,81,83,83,84,84,89,89,91,91,93,93,94,94,95,95,96,96,97,97,99,99,100,100,101,101,102,102,105,105,108,108,109,109,112,112,114,114,115,115,118,118,120,120,121,121,123,123,125,125,128,128,129,129,130,130,131,131,132,132,133,133,135,135,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,157,157,165,165,166,166,169,169,170,170,179,179,180,180,186,186,189,189,190,190,192,192,194,194,195,195,197,197,200,200,201,201,202,202],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,-1,50,-1,52,-1,-1,-1,54,56,58,60,62,64,-1,66,68,70,-1,72,74,76,78,80,-1,82,84,-1,-1,86,88,-1,90,92,94,96,98,100,102,104,106,108,-1,-1,110,112,114,-1,-1,-1,116,118,-1,120,122,124,-1,126,128,-1,-1,-1,-1,130,-1,132,-1,134,136,138,140,142,-1,144,146,148,150,-1,-1,152,-1,-1,154,156,-1,-1,158,-1,160,162,-1,-1,164,-1,166,168,-1,170,-1,172,-1,-1,174,176,178,180,182,184,-1,186,-1,188,190,192,194,-1,-1,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,-1,-1,-1,204,-1,-1,-1,-1,-1,-1,-1,206,208,-1,-1,210,212,-1,-1,-1,-1,-1,-1,-1,-1,214,216,-1,-1,-1,-1,-1,218,-1,-1,220,222,-1,224,-1,226,228,-1,230,-1,-1,232,234,236,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,2.0976269E5,1E0,4.586207E0,4.97E2,3.698E3,9.616803E2,1.3390654E2,2E0,1E0,1.9791039E6,-1.8730268E-2,-8.493633E-3,1.6942337E1,1.6007428E7,9E0,2.04115E5,3.3E1,3.7990784E7,3.51E2,-1.8444842E-2,1.852364E6,1.0358873E3,7.233077E2,4.439357E8,2.2365898E3,1.4E1,-1.3432595E-2,4.306E4,1.274936E-2,1.4912975E-1,-3.4556713E-2,-1.3500018E-2,6.179515E-4,9.237895E0,1.6667E4,1.2051282E0,1.409E3,5.2E1,5.4007E7,-8.635644E-3,1.1656439E9,3.2E1,4.383864E7,4.0469444E-3,2.7578741E1,1.159499E6,3.3580637E8,5.428175E3,2.71E2,-1.4599904E-2,5.884866E7,1E0,-1.3649008E-2,-5.582979E-3,1.4E1,5.0149255E0,1.3527782E-3,7.026624E7,2.511E3,1.594E3,1.6009701E7,1.6869704E6,2.5437157E2,1.3558896E7,3.164034E0,9.1233586E4,7.4390995E6,-3.390654E-3,1.5296387E-3,3.3803125E6,2.115238E5,9.812221E9,2.4640094E-2,1.1977588E-2,-8.712043E-3,3.85374E8,2.88721E5,1.4692688E-2,2.4E1,5.2117184E4,5.595208E2,-1.3032554E-2,3.7439062E2,2.5454E4,-6.0413913E-3,-1.1566941E-3,-9.1470005E-3,-3.8590443E-3,3.429012E7,1.5727554E-3,1.318849E6,-1.8141877E-2,1.6E1,2.453E4,7.9555137E3,1.8729467E8,2.9849844E7,-7.676224E-3,7.271717E6,2.964867E4,1E0,1.09E2,-3.116995E-3,8.4780186E-4,7.4010696E0,6.0893246E-3,1.16314115E-2,2.35184E5,6.5062125E5,-7.273446E-3,1.6446128E-2,2.3385885E0,-1.2744347E-3,8.710612E1,8E0,-5.8292816E-3,9.476695E-3,1.4608225E5,-2.1458541E-4,2.416E3,7.84664E1,-6.3551646E-3,6.23E2,2.1160326E-3,1.7161617E2,5.446219E-3,1.6521169E-3,1.946094E6,7.1E1,1.9903773E2,3.90225E2,1.3881669E6,2.2190831E5,1.0411858E-3,5.3386245E0,1.01232676E-4,1.177E3,3.1226995E0,1.4195632E6,1.6E1,-3.823277E-3,4.6852478E-4,2.6210527E0,1.8690983E-3,3.6977E4,3.5814576E7,1.1860938E8,3.0704768E-4,2.0880303E-3,-2.917539E-3,3.8176242E-3,-2.184008E-4,1.246536E-3,7.871543E-3,-3.0559883E-3,2.195523E-3,2.6402525E5,1.3537777E-3,9.006914E-3,1.765918E-3,2.7478794E-3,8.435069E-3,-9.773464E-4,4.2581656E-3,3.07E2,3.89E2,3.1487294E-3,-9.838736E-4,1E0,9.24E2,2.8991008E-3,-1.0890296E-3,-4.64967E-3,-1.084472E-4,-3.0389505E-3,1.0488047E-3,-7.977407E-3,-3.3038724E-3,1.5714285E0,4.3E1,-1.2256857E-2,-1.0178325E-3,-6.5754154E-3,-2.2484954E-3,8.95076E-3,3.4444444E0,2.5366854E-3,8.639962E-3,3.92E2,1.0000181E0,9.361874E-3,2E0,-1.8858064E-3,1.2783505E0,2.0399521E2,-1.4058007E-2,3.7035952E6,1.143643E-2,-2.843269E-3,3.2892792E2,1.609E3,4.4971E4,5.483878E-3,1.3614059E-2,-6.9777896E-3,-6.994794E-4,-1.0109652E-2,-7.368951E-3,-2.4036414E-4,-4.76596E-3,-1.0060209E-2,-4.8676725E-3,1.4966434E-3,-4.370702E-3,6.086785E-3,-1.6152324E-3,1.3124244E-3,5.8338824E-3,4.190093E-3,1.1915165E-5,-3.4006445E-3,2.697135E-4,7.1319556E-3,2.7713517E-3,4.112183E-4,4.0392824E-3,2.9411428E-3,-2.9955518E-3,2.2150865E-3,5.711781E-3,2.2634848E-3,-1.1109737E-3,2.9519508E-3,8.189894E-3,5.9749857E-3,1.2125979E-2],"split_indices":[2,43,17,68,2,2,67,67,32,113,43,0,0,71,66,70,5,3,7,2,0,9,4,70,12,67,3,0,1,0,53,0,0,0,71,9,68,10,8,7,0,46,3,12,0,71,9,7,67,2,0,5,26,0,0,0,71,0,7,2,2,5,60,4,9,53,48,62,0,0,66,48,46,0,0,0,47,9,0,2,62,67,0,4,9,0,0,0,0,7,0,9,0,3,44,62,7,5,0,60,62,6,0,0,0,71,0,0,1,43,0,0,53,0,73,3,0,0,48,0,9,67,0,2,0,67,0,0,60,0,4,67,43,62,0,69,0,2,69,60,73,0,0,71,0,12,7,5,0,0,0,0,0,0,0,0,0,48,0,0,0,0,0,0,0,1,1,0,0,17,2,0,0,0,0,0,0,0,0,69,0,0,0,0,0,0,68,0,0,10,57,0,8,0,68,4,0,43,0,0,4,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.018E3,7.59E2,2.59E2,2.56E2,5.03E2,1.2E1,2.47E2,2.3E2,2.6E1,7.9E1,4.24E2,4E0,8E0,9E1,1.57E2,1.26E2,1.04E2,9E0,1.7E1,7.4E1,5E0,2.77E2,1.47E2,5.8E1,3.2E1,1E2,5.7E1,9E0,1.17E2,5E0,9.9E1,5E0,4E0,4E0,1.3E1,3.3E1,4.1E1,1.54E2,1.23E2,1.43E2,4E0,2.9E1,2.9E1,2.4E1,8E0,7.9E1,2.1E1,2.8E1,2.9E1,1.12E2,5E0,5.7E1,4.2E1,6E0,7E0,1.4E1,1.9E1,8E0,3.3E1,1.14E2,4E1,1.05E2,1.8E1,7.2E1,7.1E1,1.1E1,1.8E1,2.5E1,4E0,8E0,1.6E1,5.7E1,2.2E1,1.2E1,9E0,5E0,2.3E1,2.5E1,4E0,8.5E1,2.7E1,3.9E1,1.8E1,1.5E1,2.7E1,7E0,7E0,1.5E1,4E0,2.8E1,5E0,1.09E2,5E0,1.6E1,2.4E1,5.1E1,5.4E1,1.3E1,5E0,2.9E1,4.3E1,6.3E1,8E0,7E0,4E0,8E0,1E1,1.6E1,9E0,1E1,6E0,2.5E1,3.2E1,4E0,1.8E1,1.8E1,5E0,1.2E1,1.3E1,4E0,8.1E1,9E0,1.8E1,3.3E1,6E0,1E1,5E0,4E0,2.3E1,1.3E1,1.5E1,9.9E1,1E1,1.1E1,5E0,1.8E1,6E0,1.1E1,4E1,3.7E1,1.7E1,4E0,9E0,1.6E1,1.3E1,2.7E1,1.6E1,5.7E1,6E0,4E0,4E0,4E0,4E0,4E0,5E0,6E0,4E0,2.6E1,6E0,1.2E1,6E0,7E0,1.1E1,4E0,9E0,1.5E1,6.6E1,4E0,5E0,1.4E1,1.9E1,5E0,5E0,1.9E1,4E0,8E0,5E0,7E0,8E0,6.3E1,3.6E1,6E0,4E0,4E0,7E0,1E1,8E0,6E0,5E0,1.2E1,2.8E1,7E0,3E1,6E0,1.1E1,1.2E1,4E0,2.3E1,4E0,5E0,1.1E1,2.3E1,3.4E1,7E0,1.9E1,9E0,6E0,2.5E1,4.1E1,4E0,1E1,1.2E1,7E0,1E1,5.3E1,1E1,2.6E1,4E0,4E0,7E0,5E0,1.8E1,1E1,4E0,2.6E1,4E0,7E0,5E0,7E0,1.7E1,6E0,7E0,4E0,1.6E1,7E0,1.7E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"237","size_leaf_vector":"1"}},{"base_weights":[4.803224E-4,-6.4988546E-2,1.21288955E-1,-1.1059974E-1,1.0745003E-2,-1.976288E-1,1.3855101E-1,-1.4435022E-1,-3.824352E-2,-1.4367324E-1,2.1822821E-2,-1.8122297E-2,-1.1957463E-1,9.712047E-2,2.5266978E-1,-1.3449155E-1,-4.3064338E-1,9.492502E-2,-5.5827163E-2,-4.490818E-2,-1.7173467E-2,7.91889E-3,9.406123E-2,-3.2540635E-4,-8.953963E-3,7.566279E-2,2.1347274E-1,2.8531857E-2,2.1964534E-1,-1.277048E-1,-2.202736E-2,-6.1311997E-3,-2.968037E-2,-5.8277004E-4,6.6967052E-3,-4.308805E-2,-2.0673989E-1,2.084897E-3,-5.9379395E-3,-4.7458272E-2,2.214873E-2,2.39022E-2,2.0004493E-1,9.030686E-2,-1.4208077E-1,4.2823735E-3,2.4661657E-1,2.5455597E-1,6.712854E-3,-1.7614657E-1,-9.713884E-2,-1.0351726E-1,-4.804799E-3,-1.4698131E-2,-2.4940043E-3,2.204983E-2,-9.282545E-2,1.5735382E-1,1.1036502E-2,6.765682E-2,-7.202281E-3,5.1831864E-3,1.0752917E-2,9.983039E-2,-5.824326E-3,-1.7747069E-2,-6.841284E-3,6.5612113E-3,2.7227536E-1,3.7616786E-1,2.0156719E-1,5.2179517E-3,-4.6420004E-3,-7.661795E-2,-2.0191501E-1,1.7577179E-1,-1.1540851E-1,1.4363092E-4,-1.17628835E-1,4.173631E-3,-2.0582218E-2,7.042988E-2,-1.3296179E-3,-1.3204718E-1,2.466316E-3,2.3801855E-3,9.416779E-3,3.913172E-2,-2.7490882E-2,1.97283E-2,6.333131E-3,5.607387E-2,1.4529963E-1,3.4835741E-3,-3.4297463E-3,1.3654399E-2,7.255779E-3,2.1689147E-2,1.1488178E-2,1.5455331E-1,2.8439677E-1,-5.7331333E-3,-2.4864047E-3,-2.1296485E-1,-4.0944275E-3,3.2724605E-3,1.0423297E-2,-9.6584104E-2,-2.0386173E-1,-9.7910434E-2,-1.0542079E-2,-6.9274396E-2,3.6720706E-3,5.900914E-3,-4.408583E-5,-1.0289645E-3,-1.6715394E-1,6.889242E-3,7.121398E-2,-4.984149E-2,4.7353823E-2,3.5678826E-3,-1.1948405E-3,-2.1087103E-2,7.6592214E-2,2.6549757E-1,8.5739076E-2,1.0424673E-2,7.90671E-2,1.5542647E-2,7.141104E-3,1.5845262E-3,-2.1311706E-3,-1.8088901E-1,-2.51622E-1,-8.0336615E-2,-1.1105385E-2,-3.5130077E-3,-2.2945558E-1,-7.057203E-2,-6.3128164E-3,1.1483617E-3,-4.359377E-3,3.632732E-2,-1.8513006E-3,-1.3865606E-3,-2.1429503E-1,4.0211953E-2,-3.510861E-2,7.59381E-3,5.463084E-2,-1.04452804E-1,-2.2676118E-2,-1.439573E-2,8.026256E-3,1.920989E-2,-6.0866047E-3,8.942626E-2,-3.2350274E-3,1.6472772E-2,9.477966E-3,-2.5429567E-2,1.0178783E-1,-4.0402506E-3,1.4478526E-1,-9.441583E-3,-3.7248184E-3,-1.8939136E-2,-9.839627E-3,-5.2954275E-3,3.3181882E-4,-1.1807099E-2,-4.137211E-3,-2.1972875E-3,-5.7290085E-3,4.3375455E-3,-5.3100684E-4,-1.5052858E-2,-4.805458E-3,-1.6188619E-3,2.5970554E-3,1.2557821E-4,-6.8897875E-3,2.012465E-3,5.4968237E-3,-2.4611356E-3,-6.8824627E-3,-3.7292093E-3,3.9702593E-4,2.7175676E-3,-3.9368607E-3,-2.8369667E-3,2.6757447E-3,5.9484793E-3,1.1066947E-3,-5.294183E-3,3.1795034E-3,9.655174E-3,2.7038043E-3,8.366757E-3,3.057394E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,-1,47,49,-1,-1,-1,-1,-1,51,53,-1,-1,55,57,59,61,63,65,-1,67,69,71,73,75,77,79,-1,-1,81,83,85,87,89,-1,-1,-1,91,-1,-1,93,-1,95,97,99,-1,-1,101,103,105,107,-1,109,-1,111,113,-1,115,-1,-1,-1,117,119,121,-1,123,125,-1,-1,-1,-1,-1,-1,127,129,-1,131,133,-1,-1,-1,135,137,139,-1,141,143,-1,-1,-1,145,147,149,151,153,-1,-1,155,157,159,161,-1,163,-1,-1,-1,-1,165,167,169,-1,-1,171,173,-1,-1,-1,175,-1,-1,177,179,181,-1,183,185,187,189,-1,191,-1,193,-1,-1,-1,195,197,-1,199,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.146455E0,2.3086586E0,2.020256E0,1.0128174E0,4.3421087E-1,2.4174702E-1,1.6018939E0,7.4366236E-1,3.1907216E-1,3.6621127E-1,2.3629305E-1,0E0,1.23946235E-1,6.1799717E-1,9.610038E-1,5.802903E-1,4.826603E-1,8.790864E-2,2.1986881E-1,1.0367721E-1,0E0,1.5786476E-1,2.7777562E-1,0E0,0E0,7.015799E-1,1.3326001E-1,0E0,6.3014793E-1,3.8563108E-1,0E0,0E0,0E0,0E0,0E0,2.5629294E-1,1.3598934E-1,0E0,0E0,1.3133563E-1,2.3714969E-1,1.9935629E-1,1.698786E-2,4.3090224E-1,4.5568228E-1,0E0,4.184544E-2,3.9459372E-1,1.5938137E-1,2.5125837E-1,8.6291754E-1,6.506628E-2,1.0543821E-1,0E0,0E0,4.383269E-2,1.5041453E-1,4.85242E-2,1.6127618E-1,6.3846014E-2,0E0,0E0,0E0,3.798195E-1,0E0,0E0,6.1028562E-2,0E0,1.527667E-2,1.3308072E-1,1.6243505E-1,0E0,0E0,8.064334E-2,8.377552E-2,4.155594E-2,2.4505806E-1,0E0,6.0500085E-2,0E0,7.111403E-2,3.6933497E-2,0E0,7.335377E-2,0E0,0E0,0E0,8.8468775E-2,1.0774812E-1,3.6076866E-2,0E0,1.6151989E-1,6.5734696E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7535573E-1,6.958568E-2,0E0,1.7636184E-2,4.624152E-2,0E0,0E0,0E0,2.9142213E-1,7.088065E-2,2.6359677E-2,0E0,4.9573705E-2,6.0023487E-2,0E0,0E0,0E0,9.054324E-2,6.2989004E-2,5.744861E-2,7.032472E-2,1.19558014E-1,0E0,0E0,1.0063494E-1,1.5679595E-1,9.719443E-2,1.1855608E-1,0E0,2.2058652E-1,0E0,0E0,0E0,0E0,8.710039E-2,1.02329135E-1,3.6389846E-1,0E0,0E0,5.7490945E-2,2.1152787E-2,0E0,0E0,0E0,6.714228E-2,0E0,0E0,1.0140324E-1,3.2075748E-2,8.666871E-2,0E0,2.188699E-2,2.8521672E-2,6.2405165E-2,6.2051054E-2,0E0,5.6605503E-2,0E0,1.8747121E-1,0E0,0E0,0E0,8.352239E-2,2.5212848E-1,0E0,2.798155E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,28,28,29,29,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,52,52,55,55,56,56,57,57,58,58,59,59,63,63,66,66,68,68,69,69,70,70,73,73,74,74,75,75,76,76,78,78,80,80,81,81,83,83,87,87,88,88,89,89,91,91,92,92,99,99,100,100,102,102,103,103,107,107,108,108,109,109,111,111,112,112,116,116,117,117,118,118,119,119,120,120,123,123,124,124,125,125,126,126,128,128,133,133,134,134,135,135,138,138,139,139,143,143,146,146,147,147,148,148,150,150,151,151,152,152,153,153,155,155,157,157,161,161,162,162,164,164],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,-1,48,50,-1,-1,-1,-1,-1,52,54,-1,-1,56,58,60,62,64,66,-1,68,70,72,74,76,78,80,-1,-1,82,84,86,88,90,-1,-1,-1,92,-1,-1,94,-1,96,98,100,-1,-1,102,104,106,108,-1,110,-1,112,114,-1,116,-1,-1,-1,118,120,122,-1,124,126,-1,-1,-1,-1,-1,-1,128,130,-1,132,134,-1,-1,-1,136,138,140,-1,142,144,-1,-1,-1,146,148,150,152,154,-1,-1,156,158,160,162,-1,164,-1,-1,-1,-1,166,168,170,-1,-1,172,174,-1,-1,-1,176,-1,-1,178,180,182,-1,184,186,188,190,-1,192,-1,194,-1,-1,-1,196,198,-1,200,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,7.785302E5,1E0,1.380863E6,5.862757E1,7.937523E-3,1.5142261E3,1.5126086E2,7E0,3.4E1,4.9321495E6,-1.8122297E-2,1E0,4.391553E6,3.3817584E7,4.8E1,4.68418E5,1.4979E4,1.2906634E7,1.3881669E6,-1.7173467E-2,2.3043478E2,3.741267E7,-3.2540635E-4,-8.953963E-3,1.7493458E7,4.4631204E2,2.8531857E-2,2.5130852E2,2.71E2,-2.202736E-2,-6.1311997E-3,-2.968037E-2,-5.8277004E-4,6.6967052E-3,1.5292561E2,2.956111E2,2.084897E-3,-5.9379395E-3,1.422162E3,2.1188422E2,1.7544615E0,2.9565217E0,1.6254545E1,3.167E3,4.2823735E-3,1.7068776E10,1.5277778E0,6.6477684E7,1E0,1.1E1,1.1010101E0,7.696402E3,-1.4698131E-2,-2.4940043E-3,1.4735735E2,1.605076E6,2.4568965E2,3.373913E0,3.074398E0,-7.202281E-3,5.1831864E-3,1.0752917E-2,6.9664386E2,-5.824326E-3,-1.7747069E-2,1.5043378E0,6.5612113E-3,2.5400869E5,8.053062E8,3.5675005E3,5.2179517E-3,-4.6420004E-3,3.9125E1,3.3502305E0,1.07882355E2,1.3922E4,1.4363092E-4,8.0509944E5,4.173631E-3,2.87834E5,3.9152692E6,-1.3296179E-3,1.1481482E0,2.466316E-3,2.3801855E-3,9.416779E-3,1.196966E6,6.2136955E0,5.51E2,6.333131E-3,1.6525911E9,1.0717949E1,3.4835741E-3,-3.4297463E-3,1.3654399E-2,7.255779E-3,2.1689147E-2,1.1488178E-2,2.7578741E1,3.2295492E0,-5.7331333E-3,6.44E2,2.4636364E1,-4.0944275E-3,3.2724605E-3,1.0423297E-2,2.1609572E7,1.9470909E2,3.0588236E0,-1.0542079E-2,2.94E2,1.474E3,5.900914E-3,-4.408583E-5,-1.0289645E-3,4.1E1,3.9914E4,1.552356E0,3.0291306E2,6.887749E7,3.5678826E-3,-1.1948405E-3,1.8578552E7,1E0,1.6521566E5,1.9E1,1.0424673E-2,2.453E4,1.5542647E-2,7.141104E-3,1.5845262E-3,-2.1311706E-3,6.1E1,2E0,3.35E2,-1.1105385E-2,-3.5130077E-3,1.96E2,4.3992E4,-6.3128164E-3,1.1483617E-3,-4.359377E-3,3.3924444E5,-1.8513006E-3,-1.3865606E-3,1.1E1,1.2E1,8.69E2,7.59381E-3,1.574E3,2.402121E4,4.8927066E4,1.43E2,8.026256E-3,1.5204346E8,-6.0866047E-3,1.3864E4,-3.2350274E-3,1.6472772E-2,9.477966E-3,1.2E2,1.1662862E8,-4.0402506E-3,1.2820834E7,-9.441583E-3,-3.7248184E-3,-1.8939136E-2,-9.839627E-3,-5.2954275E-3,3.3181882E-4,-1.1807099E-2,-4.137211E-3,-2.1972875E-3,-5.7290085E-3,4.3375455E-3,-5.3100684E-4,-1.5052858E-2,-4.805458E-3,-1.6188619E-3,2.5970554E-3,1.2557821E-4,-6.8897875E-3,2.012465E-3,5.4968237E-3,-2.4611356E-3,-6.8824627E-3,-3.7292093E-3,3.9702593E-4,2.7175676E-3,-3.9368607E-3,-2.8369667E-3,2.6757447E-3,5.9484793E-3,1.1066947E-3,-5.294183E-3,3.1795034E-3,9.655174E-3,2.7038043E-3,8.366757E-3,3.057394E-3],"split_indices":[2,43,17,9,62,53,67,61,3,3,43,0,15,43,7,3,9,1,12,43,0,4,7,0,0,66,4,0,73,2,0,0,0,0,0,67,4,0,0,48,67,68,73,69,2,0,46,68,59,20,10,68,62,0,0,67,12,70,69,71,0,0,0,67,0,0,69,0,48,7,67,0,0,4,68,67,44,0,60,0,12,43,0,68,0,0,0,9,69,2,0,46,71,0,0,0,0,0,0,71,53,0,1,67,0,0,0,60,4,68,0,44,10,0,0,0,10,9,69,67,7,0,0,5,6,48,73,0,44,0,0,0,0,2,10,67,0,0,8,1,0,0,0,43,0,0,3,3,2,0,2,48,62,0,0,7,0,10,0,0,0,0,12,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,6.67E2,3.61E2,4.16E2,2.51E2,1.8E1,3.43E2,2.83E2,1.33E2,1.6E1,2.35E2,4E0,1.4E1,2.53E2,9E1,2.75E2,8E0,1.5E1,1.18E2,1.2E1,4E0,1.98E2,3.7E1,6E0,8E0,2.15E2,3.8E1,6E0,8.4E1,2.71E2,4E0,4E0,4E0,5E0,1E1,1.1E2,8E0,6E0,6E0,4E1,1.58E2,2.3E1,1.4E1,2.02E2,1.3E1,9E0,2.9E1,7.2E1,1.2E1,1.03E2,1.68E2,4.2E1,6.8E1,4E0,4E0,1.6E1,2.4E1,1.1E1,1.47E2,1.9E1,4E0,5E0,9E0,1.94E2,8E0,4E0,9E0,7E0,2.2E1,2E1,5.2E1,6E0,6E0,2.2E1,8.1E1,1E1,1.58E2,5E0,3.7E1,9E0,5.9E1,8E0,8E0,1.9E1,5E0,4E0,7E0,8.5E1,6.2E1,1.2E1,7E0,1E2,9.4E1,4E0,5E0,1.7E1,5E0,1E1,1E1,3.5E1,1.7E1,1.3E1,9E0,7.3E1,8E0,4E0,6E0,1.32E2,2.6E1,3.3E1,4E0,1.9E1,4E1,4E0,4E0,5E0,1.4E1,4.3E1,4.2E1,4.8E1,1.4E1,5E0,7E0,2.1E1,7.9E1,3E1,6.4E1,1.7E1,1.8E1,1.1E1,6E0,5E0,4E0,4.3E1,3E1,1.2E2,1.2E1,5E0,2.1E1,2.1E1,1.2E1,4E0,1.5E1,2.3E1,1.7E1,4E0,1E1,2.4E1,1.9E1,5E0,3.7E1,1.5E1,3.3E1,1E1,4E0,1.6E1,5E0,7.3E1,6E0,1E1,2E1,8E0,5.6E1,5E0,1.3E1,3.4E1,9E0,4E0,2.6E1,8.6E1,3.4E1,1.7E1,4E0,1.6E1,5E0,1E1,1.3E1,4E0,6E0,4E0,2E1,1.5E1,4E0,3.3E1,4E0,8E0,7E0,1.1E1,2.2E1,5E0,5E0,5E0,1.1E1,4.5E1,2.8E1,4E0,4E0,1.5E1,4.1E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"201","size_leaf_vector":"1"}},{"base_weights":[-4.8119277E-3,-8.19449E-2,6.674725E-2,-6.0867727E-2,-2.3381202E-1,2.2733299E-2,1.7460561E-1,-1.2819494E-1,4.277983E-3,-1.6598009E-1,-2.7359715E-2,3.1408943E-2,-1.7334734E-1,1.8926386E-2,1.5694648E-1,-2.1095286E-1,-1.0319144E-1,1.5616835E-2,-1.4997721E-2,-1.1989513E-2,-1.11577906E-1,-1.2657218E-1,3.9631903E-2,-2.2440404E-1,-7.457351E-4,1.2680195E-1,2.6176533E-1,-1.866383E-1,-1.768178E-2,-9.140318E-2,-1.4859333E-2,4.396741E-3,-1.7916892E-1,-5.5355404E-2,-1.9814077E-1,-6.1191046E-3,-1.6930033E-2,1.6500445E-2,7.585727E-2,-5.640702E-3,-1.2937548E-2,6.5104023E-4,1.4319506E-1,6.5657133E-3,2.907409E-1,-1.1193723E-2,-1.2528831E-1,-7.586996E-2,-1.7860232E-1,-4.750737E-3,1.5629835E-2,-2.0091496E-2,-9.818152E-2,-9.136819E-2,3.94363E-3,-5.0382144E-3,-1.1096451E-2,-3.6236395E-3,2.2583473E-3,4.433173E-2,-6.4346627E-3,8.827237E-2,-7.4022366E-3,8.998234E-2,-6.165467E-3,1.2226552E-1,1.475295E-2,1.4521345E-2,6.388002E-3,-1.7084765E-3,-1.6992027E-1,4.4337574E-2,-8.4808104E-2,-1.2674099E-1,-1.5705595E-2,-6.724795E-2,2.1932188E-2,-7.828827E-3,-3.741805E-2,-1.2933938E-1,1.5349538E-3,2.5929652E-2,1.1204304E-1,-3.1998172E-2,3.6123767E-2,6.0704805E-2,1.4671421E-1,1.4191905E-4,7.3408578E-3,1.4411539E-1,1.8676901E-2,-3.9969385E-3,-9.377069E-3,-1.6101873E-3,4.755746E-3,-1.0512735E-1,-4.3142404E-2,-1.4917591E-1,-1.5748345E-3,-8.000582E-3,-4.8192218E-2,-2.1289203E-2,5.2353542E-2,-4.263576E-3,3.28917E-4,-2.9202914E-3,-7.90162E-3,3.888936E-2,-7.663315E-3,3.050958E-2,8.130047E-3,4.236355E-3,-1.1095308E-1,-2.3079947E-2,8.204179E-2,1.319442E-2,1.1738741E-1,2.2552916E-1,1.04881756E-1,1.1253367E-1,2.1508346E-1,-2.669938E-3,7.798438E-2,-6.846816E-2,-1.3848919E-1,6.092308E-2,-6.866946E-2,-8.476732E-3,-3.8694555E-3,-2.4941197E-2,-1.24132805E-1,-9.40167E-2,2.4256876E-2,7.8029774E-2,-1.8609444E-2,2.3689168E-2,8.667296E-2,-9.649653E-4,3.1533272E-3,-1.5281502E-2,4.482122E-3,-1.7335914E-1,1.2810153E-3,-4.263336E-3,2.1281246E-2,4.1914728E-2,8.280619E-3,3.435392E-2,-5.482454E-3,1.6697888E-1,4.7498997E-2,5.488922E-3,1.2000602E-2,3.7214328E-2,1.4040731E-1,1.7796963E-1,5.8243457E-2,1.3032056E-2,1.4699897E-1,6.570476E-3,5.292767E-4,-1.0127713E-2,-2.1635348E-3,-6.853345E-3,-1.5673038E-3,4.7726654E-3,2.9359382E-4,6.57511E-5,-4.13437E-3,-2.3574363E-3,1.3467293E-3,-1.02446E-3,-8.280099E-3,-5.945158E-3,1.9709137E-3,5.0180973E-3,-1.858477E-3,2.5033061E-3,6.4490414E-3,-2.0649957E-3,3.4100832E-3,2.2316587E-3,-5.1489077E-4,5.842822E-3,5.429285E-4,-1.4401516E-3,4.0613585E-3,-3.6001569E-3,-1.3163268E-2,3.9859163E-3,-1.3917804E-3,2.867904E-3,-7.010903E-5,-1.741727E-3,3.0876757E-3,9.491881E-3,5.2937944E-3,-2.8379129E-3,3.8349226E-3,-1.7605657E-3,3.9592516E-3,7.992866E-3,9.720368E-4,1.0105526E-2,3.1918418E-3,-1.1080394E-3,3.8449534E-3,9.017076E-3,2.6915255E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,-1,31,-1,33,35,37,39,-1,41,43,45,-1,47,-1,49,51,53,55,57,-1,59,61,-1,-1,63,65,-1,67,-1,69,71,73,75,-1,-1,77,79,-1,-1,-1,-1,-1,81,83,85,-1,87,-1,89,-1,-1,-1,-1,91,93,95,97,-1,99,101,-1,103,105,-1,107,109,111,113,115,117,-1,-1,119,121,-1,-1,-1,-1,123,125,127,-1,-1,129,131,133,-1,-1,-1,-1,135,-1,137,-1,139,141,143,145,147,149,151,153,155,157,-1,159,161,163,165,167,-1,-1,169,171,173,175,177,179,181,183,-1,-1,185,-1,187,-1,-1,189,191,-1,193,-1,195,197,-1,-1,199,201,203,205,-1,207,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5857563E0,1.5446205E0,2.488349E0,1.8821764E0,1.3360584E0,6.416659E-1,5.6666136E-1,4.1484094E-1,1.413979E0,2.3056722E-1,0E0,4.7138312E-1,1.24800056E-1,0E0,4.186468E-1,1.2904906E-1,3.9291716E-1,0E0,6.630623E-1,0E0,1.5450239E-1,5.178684E-1,2.8532565E-1,3.4111798E-2,0E0,2.3359144E-1,6.615305E-2,1.1987877E-1,0E0,1.9910324E-1,0E0,5.7280076E-1,4.1292566E-1,1.1928689E-1,2.2502065E-2,5.554225E-2,0E0,1.343311E-1,4.0396422E-1,0E0,0E0,1.7972594E-1,3.2960844E-1,0E0,4.2825222E-2,0E0,8.436707E-2,1.4880842E-1,1.4937681E-1,3.0664724E-1,0E0,0E0,7.1617976E-2,8.945426E-2,0E0,0E0,0E0,0E0,0E0,1.154675E-1,1.2735727E-1,1.9570339E-1,0E0,5.3415954E-2,0E0,2.0327759E-1,0E0,0E0,0E0,0E0,2.378866E-2,4.9995825E-2,1.0213816E-1,3.4409463E-2,0E0,1.0267411E-1,1.7073062E-1,0E0,2.7538143E-2,2.7789697E-2,0E0,1.9536744E-1,9.8888785E-2,2.093153E-1,1.2150829E-1,2.3543623E-1,1.08227015E-1,0E0,0E0,1.4389288E-1,8.160494E-2,0E0,0E0,0E0,0E0,9.325147E-2,1.1841242E-1,1.8297434E-2,0E0,0E0,8.16574E-2,1.8115298E-1,1.3992882E-1,0E0,0E0,0E0,0E0,5.0241813E-2,0E0,2.082568E-2,0E0,9.383805E-2,2.0386156E-1,6.336149E-2,8.9992106E-2,1.4079614E-1,1.3070345E-1,1.662612E-2,6.2084675E-2,1.8147123E-1,5.8892608E-2,0E0,4.110597E-2,1.2673208E-1,3.927654E-2,2.015857E-2,5.1206112E-2,0E0,0E0,5.5108212E-2,5.7815403E-2,1.0499832E-1,1.8933123E-1,7.464054E-2,5.393332E-2,4.8557434E-2,4.9202308E-2,0E0,0E0,7.325361E-2,0E0,1.4814809E-1,0E0,0E0,4.603523E-2,1.6732477E-2,0E0,1.03154555E-1,0E0,2.2823393E-2,7.472238E-2,0E0,0E0,4.3438476E-2,6.640881E-2,9.1423035E-2,6.56774E-2,0E0,4.4210047E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,44,44,46,46,47,47,48,48,49,49,52,52,53,53,59,59,60,60,61,61,63,63,65,65,70,70,71,71,72,72,73,73,75,75,76,76,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,89,89,90,90,95,95,96,96,97,97,100,100,101,101,102,102,107,107,109,109,111,111,112,112,113,113,114,114,115,115,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,125,125,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,136,136,139,139,141,141,144,144,145,145,147,147,149,149,150,150,153,153,154,154,155,155,156,156,158,158],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,-1,32,-1,34,36,38,40,-1,42,44,46,-1,48,-1,50,52,54,56,58,-1,60,62,-1,-1,64,66,-1,68,-1,70,72,74,76,-1,-1,78,80,-1,-1,-1,-1,-1,82,84,86,-1,88,-1,90,-1,-1,-1,-1,92,94,96,98,-1,100,102,-1,104,106,-1,108,110,112,114,116,118,-1,-1,120,122,-1,-1,-1,-1,124,126,128,-1,-1,130,132,134,-1,-1,-1,-1,136,-1,138,-1,140,142,144,146,148,150,152,154,156,158,-1,160,162,164,166,168,-1,-1,170,172,174,176,178,180,182,184,-1,-1,186,-1,188,-1,-1,190,192,-1,194,-1,196,198,-1,-1,200,202,204,206,-1,208,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.072304E5,1E0,3.921E3,6.83E2,9.639872E0,2.8088E1,7.9E1,3.89E2,2.00087E5,5E0,-2.7359715E-2,1.45064E5,1.1997242E0,1.8926386E-2,5.1167645E6,6.737403E7,1.2218E4,1.5616835E-2,1.1290322E1,-1.1989513E-2,1.8414415E1,1.1015E4,1E0,1.8E1,-7.457351E-4,1E0,8.0605554E2,3.6E1,-1.768178E-2,1.4E1,-1.4859333E-2,5.428175E3,1.5E1,1E0,1.25E0,2.0467092E6,-1.6930033E-2,1.5723623E0,3.174172E5,-5.640702E-3,-1.2937548E-2,2.4661344E5,4.43837E0,6.5657133E-3,1.6E-3,-1.1193723E-2,2.7444445E1,8.414097E-1,2E1,1.8988005E5,1.5629835E-2,-2.0091496E-2,6.593462E4,1.3E1,3.94363E-3,-5.0382144E-3,-1.1096451E-2,-3.6236395E-3,2.2583473E-3,1.047E3,1.2839025E1,2.308821E6,-7.4022366E-3,1.3388E4,-6.165467E-3,1.4078995E2,1.475295E-2,1.4521345E-2,6.388002E-3,-1.7084765E-3,5.3E1,3.56E2,1.3823239E2,7.997723E6,-1.5705595E-2,8.01E2,4.220534E5,-7.828827E-3,4.727453E7,1.3439851E7,1.5349538E-3,1.3062992E1,1.411E3,8E0,1.6802971E6,7.5491136E-1,7E0,1.4191905E-4,7.3408578E-3,1.6466942E3,3.4915986E3,-3.9969385E-3,-9.377069E-3,-1.6101873E-3,4.755746E-3,1.257E4,5.8114325E5,6.946813E7,-1.5748345E-3,-8.000582E-3,3.154992E7,1.159499E6,2.847E4,-4.263576E-3,3.28917E-4,-2.9202914E-3,-7.90162E-3,3.0291306E2,-7.663315E-3,1.1E1,8.130047E-3,4.39776E6,2.84E2,3.8343322E6,1.670046E1,4.76E2,3.8275862E0,2.379747E0,4.569524E2,4.320314E8,1.0851E4,-2.669938E-3,5.43E2,3.964269E3,2.3E1,1.4865319E5,1.1146532E8,-8.476732E-3,-3.8694555E-3,2.1340163E8,1.670046E1,5.821732E2,2.232008E-2,1.01573E5,8.076923E0,2.4915715E-3,1.500502E6,-9.649653E-4,3.1533272E-3,1.574E3,4.482122E-3,2.9077E4,1.2810153E-3,-4.263336E-3,1.1643481E6,4.646154E0,8.280619E-3,1.158E3,-5.482454E-3,9E0,4.857143E0,5.488922E-3,1.2000602E-2,4.0705118E6,3.43E2,1.3240695E3,7.618E3,1.3032056E-2,1.3091103E0,6.570476E-3,5.292767E-4,-1.0127713E-2,-2.1635348E-3,-6.853345E-3,-1.5673038E-3,4.7726654E-3,2.9359382E-4,6.57511E-5,-4.13437E-3,-2.3574363E-3,1.3467293E-3,-1.02446E-3,-8.280099E-3,-5.945158E-3,1.9709137E-3,5.0180973E-3,-1.858477E-3,2.5033061E-3,6.4490414E-3,-2.0649957E-3,3.4100832E-3,2.2316587E-3,-5.1489077E-4,5.842822E-3,5.429285E-4,-1.4401516E-3,4.0613585E-3,-3.6001569E-3,-1.3163268E-2,3.9859163E-3,-1.3917804E-3,2.867904E-3,-7.010903E-5,-1.741727E-3,3.0876757E-3,9.491881E-3,5.2937944E-3,-2.8379129E-3,3.8349226E-3,-1.7605657E-3,3.9592516E-3,7.992866E-3,9.720368E-4,1.0105526E-2,3.1918418E-3,-1.1080394E-3,3.8449534E-3,9.017076E-3,2.6915255E-3],"split_indices":[43,6,2,2,69,61,0,1,5,6,0,7,53,0,43,7,44,0,69,0,73,9,27,3,0,27,4,2,0,8,0,67,8,16,68,43,0,68,48,0,0,48,50,0,72,0,4,68,3,43,0,0,43,8,0,0,0,0,0,2,73,43,0,2,0,73,0,0,0,0,44,2,67,9,0,2,66,0,7,60,0,71,44,8,43,53,3,0,0,67,4,0,0,0,0,12,60,7,0,0,12,9,10,0,0,0,0,67,0,3,0,9,0,60,71,0,69,69,67,5,2,0,8,43,3,43,46,0,0,7,71,4,53,1,69,53,9,0,0,2,0,1,0,0,43,71,0,2,0,8,69,0,0,43,0,67,10,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,4.86E2,5.24E2,4.28E2,5.8E1,3.73E2,1.51E2,2.1E2,2.18E2,5E1,8E0,3.58E2,1.5E1,9E0,1.42E2,4.7E1,1.63E2,1.1E1,2.07E2,1.7E1,3.3E1,1.7E1,3.41E2,1.1E1,4E0,1.12E2,3E1,4.3E1,4E0,1.56E2,7E0,1.86E2,2.1E1,2.1E1,1.2E1,1.2E1,5E0,2.09E2,1.32E2,5E0,6E0,1.3E1,9.9E1,7E0,2.3E1,2.1E1,2.2E1,1.34E2,2.2E1,1.82E2,4E0,4E0,1.7E1,1.7E1,4E0,5E0,7E0,5E0,7E0,9.4E1,1.15E2,1.26E2,6E0,8E0,5E0,9E1,9E0,1.9E1,4E0,8E0,1.4E1,9E0,1.25E2,1.8E1,4E0,5.4E1,1.28E2,7E0,1E1,1.3E1,4E0,7.5E1,1.9E1,7.2E1,4.3E1,8.7E1,3.9E1,4E0,4E0,7.4E1,1.6E1,5E0,9E0,4E0,5E0,8.3E1,4.2E1,1.4E1,4E0,7E0,4.7E1,5.3E1,7.5E1,4E0,6E0,6E0,7E0,7.1E1,4E0,9E0,1E1,5E1,2.2E1,1.9E1,2.4E1,4.8E1,3.9E1,1.2E1,2.7E1,5.3E1,2.1E1,7E0,9E0,4.1E1,4.2E1,8E0,3.4E1,8E0,6E0,3.7E1,1E1,2E1,3.3E1,5.5E1,2E1,5.5E1,1.6E1,4E0,5E0,4.2E1,8E0,1.5E1,7E0,7E0,1.2E1,1.8E1,6E0,4.2E1,6E0,2.2E1,1.7E1,4E0,8E0,1E1,1.7E1,2.3E1,3E1,9E0,1.2E1,4E0,5E0,4E0,3.7E1,3.8E1,4E0,4E0,4E0,8E0,2.6E1,2.5E1,1.2E1,4E0,6E0,1.6E1,4E0,1.4E1,1.9E1,4.1E1,1.4E1,1.6E1,4E0,3.2E1,2.3E1,1E1,6E0,3.7E1,5E0,9E0,6E0,5E0,7E0,1.2E1,6E0,1.3E1,2.9E1,1.1E1,1.1E1,4E0,1.3E1,4E0,6E0,1.3E1,4E0,1.6E1,7E0,7E0,2.3E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"209","size_leaf_vector":"1"}},{"base_weights":[-5.0273514E-4,-1.2543014E-1,4.433076E-2,-1.0155056E-1,-2.711262E-1,1.177078E-2,2.009337E-1,-1.5354086E-1,-3.6586028E-2,-4.6052542E-1,-1.8392186E-1,-3.558003E-2,4.612676E-2,2.3762597E-2,1.7290747E-1,-1.427859E-1,-1.79971E-2,2.6996106E-1,-8.72707E-2,-6.3834502E-3,-2.8873853E-2,-2.0180507E-1,-2.9971502E-3,-2.6035894E-2,-1.488079E-1,3.1877626E-2,1.5861045E-1,2.0439933E-1,-3.5970823E-3,-1.948772E-1,-1.0245779E-1,5.1168823E-3,1.6806778E-2,-5.2321207E-2,-2.1511768E-1,-4.169671E-3,-2.3586677E-1,-9.047526E-3,-8.3322965E-2,-7.8002766E-2,-1.243493E-2,1.1171669E-1,1.732415E-2,-1.1329408E-3,1.796398E-1,3.4045517E-1,1.64429E-1,5.8520515E-2,-6.834624E-3,-2.168251E-1,-2.0474137E-3,-8.677662E-2,-1.3639349E-2,-8.476217E-2,3.0500285E-2,-1.1320735E-2,-3.8030786E-3,-1.2915283E-2,-6.1682626E-3,-3.2096844E-2,1.8989265E-2,-1.1183121E-2,-6.612635E-2,-5.1507945E-3,2.1135827E-4,7.6993674E-2,9.329905E-3,-9.390459E-2,2.8018365E-2,1.2619777E-2,1.4764638E-1,6.0773E-3,1.7535836E-2,-3.4592901E-3,1.8138115E-1,3.177586E-4,5.0440314E-3,-1.8143508E-1,-2.7184168E-1,-4.1874185E-2,-1.2552673E-1,-1.4925282E-1,-4.6230275E-2,6.256472E-2,-3.9093117E-3,1.5024849E-2,-6.891583E-2,8.406424E-3,8.694847E-3,1.886704E-2,-9.5699534E-2,9.940017E-2,1.499649E-2,3.931263E-2,-2.54917E-1,3.2674097E-2,-9.46087E-3,9.450849E-2,2.1504937E-1,1.3978823E-1,2.4620543E-1,-9.693711E-2,-9.545233E-3,-1.5372083E-2,-7.8240605E-3,3.4368723E-3,-7.603807E-2,-1.5857081E-1,-5.4866362E-2,-2.7530256E-3,-7.81642E-3,-6.396603E-2,3.207615E-3,5.728842E-4,4.4554253E-3,-2.3151629E-2,5.7724264E-2,-1.1677748E-2,-4.284088E-2,7.073717E-2,-1.8669238E-2,5.03168E-3,-1.295545E-3,-5.0552033E-2,-1.3818641E-1,7.60942E-4,1.1310651E-1,2.848938E-3,-1.5787615E-3,-1.7716123E-3,4.0062843E-3,-1.9272873E-2,-2.3181534E-3,9.634987E-3,7.565743E-2,6.3074734E-3,1.7241014E-3,1.1610202E-2,5.267861E-3,1.6217683E-1,4.9737714E-2,2.9040825E-1,1.514912E-1,-7.0272554E-3,-1.0338688E-3,-9.687571E-2,1.4254802E-3,-8.141663E-3,-3.5213497E-3,-8.661685E-4,-4.3371185E-3,-6.5384167E-3,-1.0749666E-1,2.9847318E-2,-6.0474083E-2,1.2997195E-2,6.2078577E-3,-7.10495E-2,2.183724E-3,1.2312038E-4,9.706163E-2,2.4653893E-2,-6.442012E-2,-4.548244E-3,-7.940892E-4,-4.016721E-3,-8.113032E-3,6.0717477E-3,9.642598E-4,6.811439E-2,-1.3134049E-2,-1.2004864E-2,9.938767E-2,1.7437801E-1,3.1434253E-3,5.0107064E-3,-7.980086E-4,8.947797E-3,1.6891176E-2,8.7557975E-3,3.1391324E-3,-5.445959E-3,-1.6181574E-3,-2.544946E-3,1.6982077E-3,-8.188024E-3,-2.9414995E-3,-1.0493287E-3,2.916391E-3,-4.929272E-3,-3.091876E-4,-2.236729E-3,2.2427244E-3,-1.4561897E-3,-5.2550673E-3,1.7367838E-3,-1.5441832E-3,5.4332195E-3,9.2994294E-4,-5.3042206E-3,2.3342576E-3,-1.824139E-3,-6.8961596E-3,1.2160112E-3,5.499531E-3,-4.0633734E-3,3.8748537E-4,-3.1079208E-3,1.5848759E-3,7.897111E-3,2.1629792E-3,8.711605E-3,4.3768445E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,-1,35,-1,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,61,63,-1,65,67,-1,69,71,73,75,-1,77,-1,79,-1,81,83,-1,-1,-1,-1,85,87,-1,89,-1,-1,91,-1,93,95,-1,97,-1,-1,-1,99,-1,-1,101,103,105,107,109,111,113,-1,115,117,-1,119,121,123,125,127,129,131,133,-1,135,137,139,141,143,-1,-1,-1,-1,145,147,149,-1,-1,151,-1,-1,-1,153,155,-1,157,159,161,-1,-1,163,165,-1,167,-1,-1,-1,-1,-1,-1,169,171,-1,-1,-1,-1,173,175,177,179,-1,-1,181,-1,-1,-1,-1,-1,183,185,187,189,191,-1,193,195,-1,197,199,201,-1,-1,-1,-1,-1,-1,203,205,207,209,211,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6625023E0,9.0140295E-1,3.7912822E0,7.7459097E-1,5.264075E-1,1.0053666E0,1.0354099E0,2.6928306E-1,1.6394327E0,4.968083E-1,4.4389427E-2,2.7709788E-1,5.687446E-1,0E0,6.6219187E-1,2.4052572E-1,0E0,1.8377757E-1,3.908816E-1,0E0,0E0,6.610286E-2,0E0,2.337907E-1,1.4943555E-1,3.6872214E-1,1.5845251E-1,4.9327755E-1,1.790143E-1,1.6552782E-1,1.920293E-1,0E0,0E0,1.9603552E-1,5.5333853E-2,0E0,3.8695753E-2,1.214543E-1,1.3413253E-1,3.922581E-2,0E0,1.3797683E-1,3.249867E-1,0E0,6.738305E-2,1.2984681E-1,3.3812308E-1,3.5875898E-2,0E0,4.0241957E-2,0E0,1.13061905E-1,0E0,1.22317314E-1,8.281002E-2,0E0,0E0,0E0,0E0,1.7973764E-1,1.4102226E-1,0E0,1.2999536E-1,0E0,0E0,4.990752E-2,0E0,5.248125E-1,2.7549082E-1,0E0,8.354396E-2,0E0,0E0,0E0,1.6780066E-1,0E0,0E0,4.5918465E-2,4.350257E-2,1.3673691E-1,7.535648E-2,2.3882598E-2,7.418635E-2,2.7692549E-2,0E0,7.632162E-2,2.6722285E-1,0E0,1.3883686E-1,6.219068E-2,6.5905094E-2,2.8826743E-2,2.734955E-2,5.4866306E-2,3.4448367E-1,2.409037E-1,0E0,3.816046E-2,1.5300751E-2,8.870673E-2,7.32317E-2,3.3617347E-2,0E0,0E0,0E0,0E0,6.0901597E-2,2.1469176E-2,1.6557511E-2,0E0,0E0,7.429293E-2,0E0,0E0,0E0,5.1256258E-2,7.367456E-2,0E0,6.619941E-2,4.454039E-2,1.1456628E-1,0E0,0E0,3.026804E-2,2.1168083E-2,0E0,3.61194E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1393777E-1,1.7801422E-1,0E0,0E0,0E0,0E0,3.17809E-2,4.5158178E-2,6.0135722E-2,2.2072852E-2,0E0,0E0,2.4452105E-2,0E0,0E0,0E0,0E0,0E0,3.145268E-2,4.0673092E-2,2.0674776E-2,3.6574565E-2,3.5096187E-2,0E0,5.0426483E-2,2.7735056E-2,0E0,2.608031E-2,1.14049956E-1,5.33489E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.264222E-2,1.87523E-1,5.093741E-2,2.4400109E-1,1.9424796E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,49,49,51,51,53,53,54,54,59,59,60,60,62,62,65,65,67,67,68,68,70,70,74,74,77,77,78,78,79,79,80,80,81,81,82,82,83,83,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,94,94,95,95,97,97,98,98,99,99,100,100,101,101,106,106,107,107,108,108,111,111,115,115,116,116,118,118,119,119,120,120,123,123,124,124,126,126,133,133,134,134,139,139,140,140,141,141,142,142,145,145,151,151,152,152,153,153,154,154,155,155,157,157,158,158,160,160,161,161,162,162,169,169,170,170,171,171,172,172,173,173],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,-1,36,-1,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,62,64,-1,66,68,-1,70,72,74,76,-1,78,-1,80,-1,82,84,-1,-1,-1,-1,86,88,-1,90,-1,-1,92,-1,94,96,-1,98,-1,-1,-1,100,-1,-1,102,104,106,108,110,112,114,-1,116,118,-1,120,122,124,126,128,130,132,134,-1,136,138,140,142,144,-1,-1,-1,-1,146,148,150,-1,-1,152,-1,-1,-1,154,156,-1,158,160,162,-1,-1,164,166,-1,168,-1,-1,-1,-1,-1,-1,170,172,-1,-1,-1,-1,174,176,178,180,-1,-1,182,-1,-1,-1,-1,-1,184,186,188,190,192,-1,194,196,-1,198,200,202,-1,-1,-1,-1,-1,-1,204,206,208,210,212,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,2.6E1,5.97E3,5.41E2,3.355864E0,9.24E2,1.059448E7,1E0,3E1,4E0,4.7721977E4,6.4153387E-6,4.5866325E6,2.3762597E-2,7.3274844E9,3.89E2,-1.79971E-2,7.635E3,4.217427E7,-6.3834502E-3,-2.8873853E-2,4.1665E3,-2.9971502E-3,2E0,1.8949389E6,6E0,4.745397E6,3.3382E4,2.5652106E7,2.135961E4,9.687E3,5.1168823E-3,1.6806778E-2,4.8614563E2,9.643839E6,-4.169671E-3,1E0,2.4568965E2,1.43E2,1.034073E6,-1.243493E-2,5.691698E2,1.4598765E0,-1.1329408E-3,1.4076087E0,7.0296685E6,1.36E2,2.8582963E-1,-6.834624E-3,9.55188E5,-2.0474137E-3,1.9668E4,-1.3639349E-2,7.026624E7,3.4779343E1,-1.1320735E-2,-3.8030786E-3,-1.2915283E-2,-6.1682626E-3,5.3061223E0,2.2871187E2,-1.1183121E-2,9.5E1,-5.1507945E-3,2.1135827E-4,2.2461708E6,9.329905E-3,5.404E2,3.8304348E1,1.2619777E-2,1.9290142E0,6.0773E-3,1.7535836E-2,-3.4592901E-3,1.4176E4,3.177586E-4,5.0440314E-3,1.957E3,1.2E1,9.9E1,1.1244019E2,6.07E2,3.2874417E2,3.88E2,-3.9093117E-3,1.1356604E2,8.860256E3,8.406424E-3,2.7563172E6,1.880118E-3,8E0,1.6964E4,4.577732E2,4.0695653E0,3.596E3,2.445E3,-9.46087E-3,2.041E3,6.7553375E4,5.657353E1,1.2792593E2,3.2E1,-9.545233E-3,-1.5372083E-2,-7.8240605E-3,3.4368723E-3,7.6811076E-3,3.3025316E5,2.241E2,-2.7530256E-3,-7.81642E-3,1.9545455E0,3.207615E-3,5.728842E-4,4.4554253E-3,3.5714287E-1,2.8361E4,-1.1677748E-2,1.6223962E7,2.7777777E0,2.90099E0,5.03168E-3,-1.295545E-3,6.23E2,3.125E0,7.60942E-4,1.4521204E0,2.848938E-3,-1.5787615E-3,-1.7716123E-3,4.0062843E-3,-1.9272873E-2,-2.3181534E-3,9.777778E0,3.7043128E5,6.3074734E-3,1.7241014E-3,1.1610202E-2,5.267861E-3,1.0905187E9,4.1104166E8,4.73909E0,6.2136906E-1,-7.0272554E-3,-1.0338688E-3,2.49E2,1.4254802E-3,-8.141663E-3,-3.5213497E-3,-8.661685E-4,-4.3371185E-3,1.4929079E2,5.25E0,4.49239E6,1.4141E4,1.407037E2,6.2078577E-3,1.5132743E0,1.901875E2,1.2312038E-4,2.4361508E6,3.7974394E-6,1.676E3,-4.548244E-3,-7.940892E-4,-4.016721E-3,-8.113032E-3,6.0717477E-3,9.642598E-4,1.9882522E0,3.0291306E2,1.02E2,7.202917E7,2.6E1,3.1434253E-3,5.0107064E-3,-7.980086E-4,8.947797E-3,1.6891176E-2,8.7557975E-3,3.1391324E-3,-5.445959E-3,-1.6181574E-3,-2.544946E-3,1.6982077E-3,-8.188024E-3,-2.9414995E-3,-1.0493287E-3,2.916391E-3,-4.929272E-3,-3.091876E-4,-2.236729E-3,2.2427244E-3,-1.4561897E-3,-5.2550673E-3,1.7367838E-3,-1.5441832E-3,5.4332195E-3,9.2994294E-4,-5.3042206E-3,2.3342576E-3,-1.824139E-3,-6.8961596E-3,1.2160112E-3,5.499531E-3,-4.0633734E-3,3.8748537E-4,-3.1079208E-3,1.5848759E-3,7.897111E-3,2.1629792E-3,8.711605E-3,4.3768445E-3],"split_indices":[43,3,2,2,73,2,12,6,0,8,43,52,43,0,5,1,0,1,60,0,0,62,0,32,60,3,60,9,58,43,44,0,0,4,62,0,26,70,2,9,0,67,69,0,69,62,0,49,0,46,0,12,0,7,71,0,0,0,0,73,67,0,10,0,0,43,0,4,68,0,68,0,0,0,2,0,0,9,3,44,67,2,67,10,0,67,62,0,60,53,8,1,4,71,2,2,0,2,48,71,73,2,0,0,0,0,53,62,4,0,0,68,0,0,0,68,1,0,5,69,69,0,0,2,71,0,53,0,0,0,0,0,0,73,43,0,0,0,0,7,7,50,49,0,0,2,0,0,0,0,0,67,69,43,9,67,0,68,4,0,60,53,44,0,0,0,0,0,0,68,67,0,12,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.009E3,2.66E2,7.43E2,2.3E2,3.6E1,6.16E2,1.27E2,1.27E2,1.03E2,1E1,2.6E1,2.59E2,3.57E2,9E0,1.18E2,1.23E2,4E0,1.4E1,8.9E1,4E0,6E0,2.2E1,4E0,2.4E2,1.9E1,3.18E2,3.9E1,1E2,1.8E1,5.2E1,7.1E1,6E0,8E0,7.1E1,1.8E1,6E0,1.6E1,1.86E2,5.4E1,1.3E1,6E0,4.8E1,2.7E2,4E0,3.5E1,2.1E1,7.9E1,1.3E1,5E0,4.5E1,7E0,6.7E1,4E0,5.1E1,2E1,1.4E1,4E0,1E1,6E0,1.02E2,8.4E1,4E0,5E1,9E0,4E0,3.6E1,1.2E1,2.3E1,2.47E2,7E0,2.8E1,4E0,1.7E1,5E0,7.4E1,7E0,6E0,3E1,1.5E1,3.2E1,3.5E1,1.8E1,3.3E1,1.6E1,4E0,4.5E1,5.7E1,4E0,8E1,1.3E1,3.7E1,2.6E1,1E1,1.3E1,1E1,2.43E2,4E0,1.7E1,1.1E1,4.7E1,2.7E1,8E0,2.2E1,8E0,7E0,7E0,2.5E1,2.3E1,1.2E1,4E0,1.4E1,2.9E1,4E0,7E0,9E0,2.4E1,2.1E1,6E0,5.1E1,2.4E1,5.6E1,4E0,9E0,1.9E1,1.8E1,4E0,2.2E1,5E0,5E0,5E0,8E0,5E0,5E0,1.59E2,8.4E1,9E0,8E0,7E0,4E0,3.7E1,1E1,1.7E1,1E1,4E0,4E0,2.1E1,4E0,1.8E1,5E0,7E0,5E0,1.3E1,1.6E1,1E1,1.4E1,1.4E1,7E0,3.1E1,2E1,7E0,1.7E1,2.9E1,2.7E1,7E0,1.2E1,9E0,9E0,1.8E1,4E0,4.4E1,1.15E2,1.8E1,6.6E1,3.2E1,5E0,5E0,5E0,9E0,8E0,6E0,4E0,1.5E1,6E0,6E0,7E0,5E0,1.1E1,4E0,6E0,7E0,7E0,5E0,9E0,1.7E1,1.4E1,1E1,1E1,1.3E1,4E0,4E0,2.5E1,2.2E1,5E0,2.5E1,1.9E1,2.5E1,9E1,8E0,1E1,2.7E1,3.9E1,2.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"213","size_leaf_vector":"1"}},{"base_weights":[1.1542375E-3,-5.76815E-2,1.068986E-1,-1.3268349E-1,-2.0550696E-2,-1.4995354E-1,1.2255918E-1,-1.2017352E-1,-2.4729492E-1,-8.86327E-2,1.6061844E-3,1.2959312E-3,-2.1845846E-1,1.8485563E-2,1.1080396E-1,-2.0795041E-1,-1.0434892E-1,-2.0038301E-2,-1.2309187E-1,-6.996615E-2,-1.6445434E-2,-3.311938E-3,1.0969806E-2,-2.9613188E-1,-3.524436E-3,9.177153E-2,2.6950178E-1,-6.725694E-3,-1.2040149E-2,9.711703E-3,-1.2355179E-1,-3.3570593E-4,-8.622217E-3,5.130638E-2,-9.074831E-2,-7.682282E-2,9.8082265E-3,-8.458637E-3,-1.721166E-2,8.068845E-2,2.7543744E-1,3.297408E-1,1.3756038E-1,-7.711062E-2,1.174303E-1,-1.127049E-1,-2.63053E-1,-9.3462586E-4,3.957514E-3,-8.916907E-3,-8.114612E-2,5.2357037E-2,-1.096071E-1,4.1632015E-2,-1.2792008E-2,3.5423167E-2,1.19343095E-1,1.7810117E-2,4.3378366E-4,6.9425576E-3,1.6544815E-2,3.8725317E-3,8.606136E-3,-1.6418867E-3,-7.0751617E-3,1.0048811E-2,6.870214E-4,-1.019315E-1,-1.0123459E-2,-1.6845658E-2,-6.216617E-3,2.5020705E-3,-9.610132E-2,-1.5455208E-3,4.904986E-3,-1.4406097E-1,-2.4317175E-2,1.4849249E-2,8.093883E-2,1.4323044E-3,-9.819756E-2,1.2038257E-1,1.4053195E-2,-4.906671E-2,1.3894041E-1,-1.3363954E-1,-6.323991E-2,-1.4279623E-1,-7.180374E-2,-2.0152305E-1,-1.0812687E-1,1.1029331E-3,-3.9046765E-3,-6.527125E-3,2.6111165E-2,1.0386447E-1,3.69592E-2,7.5321584E-3,-7.894034E-3,-3.9806597E-2,-8.828309E-3,1.9458464E-1,1.1414602E-2,2.3166211E-2,-8.321872E-3,3.370835E-2,-1.1756204E-2,1.6452897E-1,2.0416373E-2,-7.5284176E-2,-1.6244921E-1,-2.5253227E-2,-1.0038481E-1,-1.7571834E-1,-1.5262672E-3,8.3889463E-4,-8.176051E-2,-4.400068E-3,-1.3340265E-2,-7.920805E-4,-6.7401864E-3,3.2237347E-3,7.075076E-2,5.254402E-2,6.6081984E-3,3.900055E-3,8.0245466E-4,-1.1625355E-1,1.6852802E-2,-3.7231839E-3,1.1153442E-3,3.4418919E-3,1.2922096E-2,-5.374962E-3,4.150367E-3,2.9706245E-2,-4.1438416E-3,3.5956511E-3,-1.6613628E-4,2.2103499E-1,1.3144529E-1,-5.0971475E-3,7.439299E-2,-4.1886596E-3,2.020825E-4,-8.350153E-3,-4.717289E-3,3.4931733E-3,-2.1971525E-3,-1.4555298E-3,-8.170462E-3,-9.469792E-3,-2.4391005E-3,-4.382608E-3,-7.343796E-4,-1.5869667E-3,2.291051E-3,1.7959994E-3,5.8052777E-3,3.691448E-3,-8.370355E-5,-8.759674E-3,-2.2999875E-4,1.4840086E-3,-1.7780653E-3,-1.6564471E-3,2.0127238E-3,1.1070752E-2,3.3547985E-3,7.1514794E-3,-3.8565067E-3,5.220347E-3,-2.600876E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,-1,31,33,-1,35,-1,37,-1,39,41,-1,-1,43,45,-1,-1,47,49,51,53,-1,-1,55,57,59,61,63,65,67,69,-1,-1,-1,71,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,85,-1,-1,-1,-1,87,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,119,-1,-1,-1,121,123,125,127,-1,129,-1,131,133,135,-1,137,-1,139,141,143,145,147,149,151,-1,-1,153,-1,-1,-1,-1,155,157,159,-1,-1,-1,161,163,-1,-1,-1,-1,-1,-1,165,-1,-1,-1,167,169,-1,171,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.526348E0,1.8742855E0,1.5267992E0,2.9001594E-1,6.8197757E-1,2.7440012E-1,1.1107273E0,2.602744E-1,4.3290412E-1,5.290116E-1,3.9746946E-1,0E0,1.540944E-1,0E0,1.0052257E0,6.059897E-2,3.8231063E-1,0E0,1.0167748E-1,2.69288E-1,0E0,3.2481703E-1,0E0,1.8177211E-2,0E0,6.006639E-1,2.4076939E-1,0E0,0E0,2.52217E-1,1.9581223E-1,0E0,0E0,4.10563E-2,7.271284E-2,2.2172725E-1,2.0623092E-1,0E0,0E0,5.031893E-1,4.942106E-1,6.841874E-2,1.8422857E-2,4.197003E-2,1.1993627E-1,1.3817811E-1,8.145499E-2,0E0,0E0,0E0,1.7517936E-1,5.4347377E-2,1.1697805E-1,1.2407483E-1,2.0423296E-1,2.4341156E-1,5.216763E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4981043E-1,0E0,0E0,0E0,0E0,7.5974226E-2,0E0,0E0,3.8090587E-2,3.9732635E-2,1.3028987E-1,4.455778E-2,1.5368478E-1,1.2257518E-1,2.1609136E-1,1.9705957E-1,2.9761422E-1,4.2124128E-1,1.0545385E-1,8.212565E-2,8.453268E-2,4.8062652E-2,6.521124E-2,6.622201E-2,0E0,0E0,0E0,6.9246545E-2,5.4805994E-2,1.5268533E-2,1.646397E-1,0E0,4.3674428E-2,0E0,1.427058E-1,1.3068056E-1,8.0166966E-2,0E0,2.1676803E-2,0E0,1.8593025E-1,1.9169639E-1,3.1027839E-2,3.2322764E-2,7.3339805E-2,1.5244117E-1,5.7515204E-2,0E0,0E0,3.9054394E-2,0E0,0E0,0E0,0E0,8.18454E-2,3.5369493E-2,2.2451662E-2,0E0,0E0,0E0,8.0680445E-2,1.12776116E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.243001E-2,0E0,0E0,0E0,9.576225E-2,3.8933206E-1,0E0,1.02219135E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,18,18,19,19,21,21,23,23,25,25,26,26,29,29,30,30,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,50,50,51,51,52,52,53,53,54,54,55,55,56,56,67,67,72,72,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,94,94,95,95,96,96,97,97,99,99,101,101,102,102,103,103,105,105,107,107,108,108,109,109,110,110,111,111,112,112,113,113,116,116,121,121,122,122,123,123,127,127,128,128,135,135,139,139,140,140,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,-1,32,34,-1,36,-1,38,-1,40,42,-1,-1,44,46,-1,-1,48,50,52,54,-1,-1,56,58,60,62,64,66,68,70,-1,-1,-1,72,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,86,-1,-1,-1,-1,88,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,120,-1,-1,-1,122,124,126,128,-1,130,-1,132,134,136,-1,138,-1,140,142,144,146,148,150,152,-1,-1,154,-1,-1,-1,-1,156,158,160,-1,-1,-1,162,164,-1,-1,-1,-1,-1,-1,166,-1,-1,-1,168,170,-1,172,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.972052E5,4.8465605E3,4.9735293E0,1E0,1E0,3.6E1,3.6E1,2E0,3.5531914E0,7.82261E6,1.2959312E-3,7.978062E6,1.8485563E-2,6.0052995E6,6.48334E5,1E0,-2.0038301E-2,4.0869565E0,5E-1,-1.6445434E-2,1.5292561E2,1.0969806E-2,1.1558494E8,-3.524436E-3,5.428175E3,1.9047619E0,-6.725694E-3,-1.2040149E-2,4.03E2,3.0977E4,-3.3570593E-4,-8.622217E-3,7.52E2,1.4076087E0,3.1488764E0,3.0416667E0,-8.458637E-3,-1.721166E-2,1.1806228E6,4.1325716E7,1.4081731E1,5.5177975E6,6.737099E7,2.00087E5,3.2856784E7,5.27E2,-9.3462586E-4,3.957514E-3,-8.916907E-3,1.6386554E0,1.525E4,1.585814E6,1.058E3,1E0,2.5959E4,7.549744E7,1.7810117E-2,4.3378366E-4,6.9425576E-3,1.6544815E-2,3.8725317E-3,8.606136E-3,-1.6418867E-3,-7.0751617E-3,1.0048811E-2,6.870214E-4,1.8224286E2,-1.0123459E-2,-1.6845658E-2,-6.216617E-3,2.5020705E-3,3.3820656E7,-1.5455208E-3,4.904986E-3,1.1430505E3,4.4091418E4,6.620166E-6,1.1939979E8,2.673585E1,6.763312E7,7.362415E1,3.0070068E7,8.620714E2,2E0,2.772925E3,1.2E1,4.8E0,1.2783505E0,8.238114E2,3.2E1,1.1029331E-3,-3.9046765E-3,-6.527125E-3,2.6086957E0,2.378914E6,7.3333335E-1,3.1706784E0,-7.894034E-3,3.3064186E7,-8.828309E-3,4.204547E5,1.5477113E2,1.6254545E1,-8.321872E-3,2.152E3,-1.1756204E-2,5.6516E4,8.399457E2,3.3857143E1,1.8E6,1.9741463E2,8.7578294E5,8.49E2,-1.5262672E-3,8.3889463E-4,1.2673605E7,-4.400068E-3,-1.3340265E-2,-7.920805E-4,-6.7401864E-3,1.24E3,6.7723384E7,2E0,6.6081984E-3,3.900055E-3,8.0245466E-4,9.24E2,4.3E1,-3.7231839E-3,1.1153442E-3,3.4418919E-3,1.2922096E-2,-5.374962E-3,4.150367E-3,8.2103E4,-4.1438416E-3,3.5956511E-3,-1.6613628E-4,1.683263E7,2.9366477E0,-5.0971475E-3,3.5510652E6,-4.1886596E-3,2.020825E-4,-8.350153E-3,-4.717289E-3,3.4931733E-3,-2.1971525E-3,-1.4555298E-3,-8.170462E-3,-9.469792E-3,-2.4391005E-3,-4.382608E-3,-7.343796E-4,-1.5869667E-3,2.291051E-3,1.7959994E-3,5.8052777E-3,3.691448E-3,-8.370355E-5,-8.759674E-3,-2.2999875E-4,1.4840086E-3,-1.7780653E-3,-1.6564471E-3,2.0127238E-3,1.1070752E-2,3.3547985E-3,7.1514794E-3,-3.8565067E-3,5.220347E-3,-2.600876E-3],"split_indices":[2,43,43,68,29,8,10,2,32,68,43,0,62,0,43,46,21,0,73,68,0,67,0,60,0,67,69,0,0,2,44,0,0,12,69,73,69,0,0,43,66,73,60,7,5,60,0,0,0,0,69,1,9,2,19,9,7,0,0,0,0,0,0,0,0,0,0,4,0,0,0,0,7,0,0,48,48,53,7,68,7,73,58,4,6,62,8,69,68,48,10,0,0,0,69,9,68,69,0,5,0,43,71,69,0,2,0,9,67,67,5,4,60,2,0,0,60,0,0,0,0,44,7,8,0,0,0,2,8,0,0,0,0,0,0,1,0,0,0,60,57,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,6.73E2,3.74E2,2.22E2,4.51E2,2.1E1,3.53E2,2.02E2,2E1,1.1E2,3.41E2,6E0,1.5E1,1.3E1,3.4E2,2.9E1,1.73E2,7E0,1.3E1,1.04E2,6E0,3.35E2,6E0,9E0,6E0,3.05E2,3.5E1,1.5E1,1.4E1,2.5E1,1.48E2,5E0,8E0,1.5E1,8.9E1,5E1,2.85E2,5E0,4E0,2.89E2,1.6E1,2.3E1,1.2E1,1.4E1,1.1E1,1.39E2,9E0,5E0,1E1,6E0,8.3E1,1E1,4E1,1.18E2,1.67E2,1.34E2,1.55E2,1.1E1,5E0,4E0,1.9E1,7E0,5E0,1E1,4E0,5E0,6E0,1.28E2,1.1E1,4E0,5E0,8E0,7.5E1,4E0,6E0,2.8E1,1.2E1,7.1E1,4.7E1,1.44E2,2.3E1,2.6E1,1.08E2,1.6E1,1.39E2,6.9E1,5.9E1,2.4E1,5.1E1,9E0,1.9E1,7E0,5E0,4E0,6.7E1,3E1,1.7E1,1.4E2,4E0,1.5E1,8E0,1.5E1,1.1E1,1.04E2,4E0,1.2E1,4E0,1.14E2,2.5E1,2.4E1,4.5E1,3E1,2.9E1,1.8E1,6E0,5E0,4.6E1,5E0,4E0,6E0,1.3E1,4.5E1,2.2E1,1.4E1,1.6E1,4E0,1.3E1,9E0,1.31E2,9E0,6E0,7E0,8E0,4E0,7E0,9.9E1,5E0,5E0,7E0,4E1,7.4E1,7E0,1.8E1,2E1,4E0,3.3E1,1.2E1,5E0,2.5E1,1.6E1,1.3E1,1.4E1,4E0,3.8E1,8E0,2.5E1,2E1,1.5E1,7E0,9E0,5E0,5E0,4E0,1.03E2,2.8E1,1.7E1,8.2E1,3.5E1,5E0,6.7E1,7E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"173","size_leaf_vector":"1"}},{"base_weights":[-1.3648425E-3,-4.643758E-2,1.2990181E-1,-3.1971816E-2,-2.3962098E-1,3.364901E-1,1.0956251E-1,-1.2884802E-1,-8.204965E-3,-1.8504345E-1,-2.939666E-2,1.7244022E-2,5.982114E-3,-7.7699996E-3,1.2363854E-1,-1.8769374E-1,-9.407046E-2,1.3923354E-2,-1.2019227E-2,-1.1522745E-1,-3.8488916E-1,1.03314966E-1,2.783451E-1,-2.3445815E-1,-1.574157E-1,-4.6898846E-2,-1.2055755E-1,-6.422348E-2,1.646271E-2,-1.4091845E-2,-8.297858E-2,-7.904948E-3,-2.1731677E-2,3.8110703E-2,1.3086286E-1,1.5881384E-2,8.615531E-3,-6.019431E-3,-1.22196E-2,-3.614537E-3,-7.7784923E-3,2.7469278E-3,-6.4881206E-2,-1.2851924E-1,-1.4535735E-3,-5.080084E-2,-1.8941809E-1,-6.767619E-3,9.0190314E-2,-1.3917547E-2,-1.12399995E-1,-1.3047696E-2,6.508588E-2,1.4266464E-1,-5.4888823E-3,2.6243364E-3,-2.8953925E-3,-8.2422554E-2,1.0056944E-3,-1.742598E-1,-1.07081674E-1,-4.0266406E-2,-1.3905594E-1,-1.1049805E-2,-4.96888E-3,-1.6899758E-3,-1.3199069E-2,1.02852836E-1,-4.620193E-3,2.962765E-3,-3.126654E-3,-1.696859E-1,-3.4480296E-2,4.01074E-2,1.11497855E-2,7.116663E-2,1.620457E-1,-4.4580832E-2,-6.3913036E-3,-4.2986376E-3,-9.496773E-3,-7.158731E-2,-7.2409557E-3,-6.275233E-2,5.4375557E-3,-1.781616E-1,-2.1513053E-3,-1.170128E-1,7.2911386E-3,1.4994432E-1,4.31407E-2,-1.079468E-2,-4.642897E-3,-4.1099167E-3,1.1894957E-3,-4.169197E-2,6.3070446E-2,1.0758131E-1,2.5152206E-2,2.0418164E-1,1.200258E-1,-3.435467E-3,3.4748582E-4,-6.092643E-4,-3.9851996E-3,-1.6120221E-2,-8.916895E-2,2.2807406E-2,-5.4019717E-3,-9.701979E-3,-3.9367694E-3,-2.4814813E-3,-1.3328407E-2,-2.5004733E-3,8.146435E-2,1.6666955E-1,4.9268856E-4,-2.0551838E-2,1.0149157E-1,1.0272469E-3,-3.5312236E-3,1.4029053E-1,1.312462E-2,2.1578704E-3,6.3079577E-3,-3.3562683E-4,2.9215373E-3,1.421513E-1,2.563523E-1,1.2189953E-2,1.5568084E-1,-3.504771E-3,9.823864E-4,-2.1566E-3,-5.3018993E-3,4.1064695E-3,-8.5137336E-4,-3.4009924E-3,5.8237165E-3,2.6133207E-3,-8.3855167E-4,5.628635E-3,-2.068917E-3,5.780465E-3,1.1389654E-2,2.8093476E-3,-2.9004205E-3,9.170078E-3,2.5837212E-3,8.167031E-3,1.1392484E-3,-3.0959942E-3,2.1333117E-3,8.1001725E-3,1.2493079E-3,1.4434329E-2,7.401501E-3,6.6161314E-3,-4.207273E-3,4.4999705E-3,1.0532091E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,61,63,65,67,69,71,-1,73,75,-1,-1,-1,77,-1,79,81,83,85,-1,-1,87,-1,89,-1,-1,-1,91,93,95,-1,97,99,101,-1,-1,-1,103,-1,105,107,109,-1,111,113,115,117,-1,-1,-1,-1,119,121,123,125,127,129,-1,-1,-1,-1,131,133,135,-1,-1,-1,137,-1,139,141,143,-1,145,147,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.064577E0,2.124577E0,1.0640211E0,1.6359792E0,1.0297062E0,1.2671018E-1,9.590945E-1,2.688551E-1,6.7905635E-1,6.2536657E-1,0E0,0E0,0E0,0E0,6.912625E-1,3.530526E-2,1.0679549E-1,0E0,8.4263045E-1,2.0115706E-1,1.3886237E-1,3.614459E-1,9.212375E-2,2.1862984E-2,1.631403E-2,3.053166E-2,3.6388755E-2,3.264659E-1,6.2917733E-1,0E0,6.5487504E-2,0E0,0E0,5.544626E-1,4.3831038E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.892436E-2,3.9793164E-2,3.232944E-2,0E0,1.6360933E-1,5.167353E-2,3.9608648E-1,2.2222221E-1,5.0494954E-2,9.6814275E-2,0E0,2.4567217E-1,1.7830944E-1,0E0,0E0,0E0,3.8271025E-2,0E0,1.9005895E-2,5.5750042E-2,1.688833E-1,6.0424447E-2,0E0,0E0,2.8671393E-1,0E0,2.2692168E-1,0E0,0E0,0E0,3.405574E-2,3.8384136E-2,1.0030442E-1,0E0,4.9237728E-2,1.6494155E-1,2.2282878E-2,0E0,0E0,0E0,1.9429564E-2,0E0,1.3406757E-1,1.1956794E-1,2.0580828E-2,0E0,3.9209622E-1,1.8685645E-1,1.05379105E-1,1.4331341E-1,0E0,0E0,0E0,0E0,2.8351832E-2,1.5628836E-1,2.3588896E-2,1.8978842E-2,1.3183975E-1,2.1341628E-1,0E0,0E0,0E0,0E0,9.31837E-2,7.07615E-2,1.3515425E-1,0E0,0E0,0E0,1.281458E-1,0E0,2.1179464E-1,1.5835415E-1,1.0520291E-1,0E0,6.909832E-2,7.729587E-2,0E0,0E0,6.314829E-2,7.1961775E-2,0E0,0E0,0E0,0E0,9.4563544E-2,9.55354E-2,2.1597436E-1,1.5473652E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,33,33,34,34,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,57,57,59,59,60,60,61,61,62,62,65,65,67,67,71,71,72,72,73,73,75,75,76,76,77,77,81,81,83,83,84,84,85,85,87,87,88,88,89,89,90,90,95,95,96,96,97,97,98,98,99,99,100,100,105,105,106,106,107,107,111,111,113,113,114,114,115,115,117,117,118,118,121,121,122,122,127,127,128,128,129,129,130,130],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,62,64,66,68,70,72,-1,74,76,-1,-1,-1,78,-1,80,82,84,86,-1,-1,88,-1,90,-1,-1,-1,92,94,96,-1,98,100,102,-1,-1,-1,104,-1,106,108,110,-1,112,114,116,118,-1,-1,-1,-1,120,122,124,126,128,130,-1,-1,-1,-1,132,134,136,-1,-1,-1,138,-1,140,142,144,-1,146,148,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,9.932432E0,5.6E1,3.56E2,2E1,1.84E2,8.147158E4,4.54E2,7.336111E4,4.9E1,-2.939666E-2,1.7244022E-2,5.982114E-3,-7.7699996E-3,6.9307615E6,1.2916666E0,1.9E1,1.3923354E-2,4.3827028E5,3.909839E-5,1.1E1,3.3580637E8,3.3382E4,6.2886596E-1,7E0,1.00019E5,7.997723E6,3.2856784E7,4.265829E2,-1.4091845E-2,4.1665E3,-7.904948E-3,-2.1731677E-2,1.3809524E0,7.134432E7,1.5881384E-2,8.615531E-3,-6.019431E-3,-1.22196E-2,-3.614537E-3,-7.7784923E-3,6.695E3,3E0,1.2139681E3,-1.4535735E-3,3.0084E4,1.731E3,2.511E3,1.1043767E3,1.3E1,3.541824E2,-1.3047696E-2,7.459E3,6.9664386E2,-5.4888823E-3,2.6243364E-3,-2.8953925E-3,1.7544615E0,1.0056944E-3,3.3305264E2,4.325E0,3.839424E2,1E0,-1.1049805E-2,-4.96888E-3,5.862757E1,-1.3199069E-2,3.5419354E0,-4.620193E-3,2.962765E-3,-3.126654E-3,2.56E2,1.0118329E4,6.726722E2,1.11497855E-2,4.569524E2,1.5918367E0,1.52E2,-6.3913036E-3,-4.2986376E-3,-9.496773E-3,3.2E1,-7.2409557E-3,1.5132743E0,1E0,1.266E3,-2.1513053E-3,9.0217394E-1,1.559287E5,3.4052496E8,1.552E3,-1.079468E-2,-4.642897E-3,-4.1099167E-3,1.1894957E-3,6.63651E4,1.6942337E1,2.6862775E6,5.53648E8,1.1618335E0,3.2608695E0,-3.435467E-3,3.4748582E-4,-6.092643E-4,-3.9851996E-3,7.56484E5,1.0142858E1,5.449711E8,-5.4019717E-3,-9.701979E-3,-3.9367694E-3,7.82E2,-1.3328407E-2,4E0,1.3333334E0,1.09652E5,4.9268856E-4,1.798E3,4.111111E0,1.0272469E-3,-3.5312236E-3,1.3205668E7,2.2715E4,2.1578704E-3,6.3079577E-3,-3.3562683E-4,2.9215373E-3,5E0,2.1089442E0,2.7578741E1,2.4077083E3,-3.504771E-3,9.823864E-4,-2.1566E-3,-5.3018993E-3,4.1064695E-3,-8.5137336E-4,-3.4009924E-3,5.8237165E-3,2.6133207E-3,-8.3855167E-4,5.628635E-3,-2.068917E-3,5.780465E-3,1.1389654E-2,2.8093476E-3,-2.9004205E-3,9.170078E-3,2.5837212E-3,8.167031E-3,1.1392484E-3,-3.0959942E-3,2.1333117E-3,8.1001725E-3,1.2493079E-3,1.4434329E-2,7.401501E-3,6.6161314E-3,-4.207273E-3,4.4999705E-3,1.0532091E-2],"split_indices":[2,69,0,2,6,10,43,1,60,3,0,0,0,0,43,68,10,0,43,53,8,7,9,68,0,5,9,60,67,0,62,0,0,69,59,0,0,0,0,0,0,9,8,48,0,44,2,2,67,0,67,0,2,67,0,0,0,68,0,48,69,4,19,0,0,62,0,69,0,0,0,10,48,4,0,67,68,44,0,0,0,10,0,68,19,2,0,68,48,7,2,0,0,0,0,48,71,62,7,68,69,0,0,0,0,12,73,46,0,0,0,2,0,73,68,1,0,44,69,0,0,9,44,0,0,0,0,6,53,71,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.023E3,7.62E2,2.61E2,7.1E2,5.2E1,2.2E1,2.39E2,1.39E2,5.71E2,4.7E1,5E0,1.8E1,4E0,1.1E1,2.28E2,5E1,8.9E1,6E0,5.65E2,3.6E1,1.1E1,2.03E2,2.5E1,1.7E1,3.3E1,3.3E1,5.6E1,1.99E2,3.66E2,4E0,3.2E1,4E0,7E0,6.1E1,1.42E2,1.3E1,1.2E1,5E0,1.2E1,5E0,2.8E1,9E0,2.4E1,5.1E1,5E0,1.81E2,1.8E1,2.79E2,8.7E1,1E1,2.2E1,4E0,5.7E1,1.36E2,6E0,5E0,4E0,2E1,4E0,1.4E1,3.7E1,1.63E2,1.8E1,1E1,8E0,2.75E2,4E0,8.2E1,5E0,4E0,6E0,1.2E1,1E1,5.1E1,6E0,3E1,1.06E2,1.3E1,7E0,5E0,9E0,2.3E1,1.4E1,1.09E2,5.4E1,1.2E1,6E0,1.9E1,2.56E2,4.5E1,3.7E1,5E0,7E0,5E0,5E0,1.1E1,4E1,1.6E1,1.4E1,5.1E1,5.5E1,8E0,5E0,5E0,1.8E1,4E1,6.9E1,4.8E1,6E0,8E0,4E0,1.2E1,7E0,2.27E2,2.9E1,4E1,5E0,1.8E1,1.9E1,4E0,7E0,1.5E1,2.5E1,6E0,1E1,8E0,6E0,2.5E1,2.6E1,1.4E1,4.1E1,1.5E1,2.5E1,2.7E1,4.2E1,1.8E1,3E1,8E0,4E0,4.7E1,1.8E2,2.2E1,7E0,2.8E1,1.2E1,6E0,1.2E1,5E0,1.4E1,1.1E1,4E0,7E0,1.8E1,1.9E1,6E0,1.5E1,1.1E1,6E0,8E0,2.4E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-2.3163706E-3,-4.3742355E-2,1.2968592E-1,-1.1102305E-1,-1.1658012E-2,8.893596E-2,2.077015E-1,-1.0250346E-1,-2.0608572E-2,-6.083685E-3,-2.345919E-1,1.02474086E-1,-8.679268E-3,3.111256E-1,1.2365165E-1,-8.4660165E-2,-1.9096068E-1,-4.335501E-2,2.5062487E-2,-8.14113E-2,-2.1408575E-2,1.307893E-1,2.9094229E-2,2.6379682E-3,3.5844988E-1,1.5902634E-1,4.275372E-3,-9.175615E-2,1.0311958E-2,-1.4437537E-1,-2.2984022E-2,-9.431513E-2,-2.3252202E-2,3.1502713E-2,-1.2919953E-2,1.5573751E-3,-8.327399E-3,-5.9242765E-3,1.4147119E-1,7.192187E-2,-8.044763E-3,1.0673161E-2,1.8483171E-2,2.7062166E-1,1.1890965E-1,4.742078E-3,-2.6675216E-3,-8.077384E-2,-2.2548053E-1,-9.565029E-3,-8.075886E-2,-1.251941E-1,4.3581687E-2,-1.1334203E-1,-1.4552204E-2,8.3757125E-2,7.1165347E-3,1.5431722E-1,-1.1361421E-3,8.1400096E-4,1.1968596E-1,-5.0875306E-2,2.390062E-2,4.9015544E-3,1.760288E-2,3.975848E-2,7.047192E-3,-1.2097469E-1,-2.7006762E-2,-1.4662112E-2,-3.4708255E-3,-4.3994233E-2,-6.594967E-3,-8.983631E-2,-2.1288743E-2,3.6841158E-3,-1.3964837E-3,-4.7106115E-4,-1.6389525E-1,2.4860755E-2,-3.3029135E-2,3.8901668E-2,1.1957992E-1,-3.081001E-3,1.4776576E-1,1.3359898E-2,1.4105667E-1,5.19689E-3,-6.3412455E-3,7.966141E-3,1.012034E-3,-4.2694362E-4,-4.5731384E-3,-1.2872173E-3,2.0908068E-3,-2.7223488E-3,5.3299638E-3,-1.0546174E-2,-1.0783026E-1,2.0469642E-1,-6.1550986E-2,-5.502692E-3,1.097008E-2,-1.0630176E-1,-5.4800945E-5,-3.7516362E-3,-9.49331E-3,-6.4669102E-3,5.0825156E-2,-4.7684547E-2,5.4219753E-2,5.8565225E-2,-2.2882926E-3,1.3279818E-1,1.2547638E-4,-3.92845E-2,2.6936859E-2,9.001348E-3,2.7892513E-3,9.377825E-2,1.853339E-1,-7.85614E-2,-7.4599828E-3,5.0631985E-3,1.3716438E-2,-4.0502585E-2,-1.473619E-1,2.7592587E-3,-1.8470054E-3,-4.9784373E-2,-1.4897035E-1,1.4266383E-3,-1.8684632E-3,3.2081798E-2,4.7341147E-3,-5.59169E-2,2.8386528E-3,4.7767456E-3,6.6411955E-4,-1.1531069E-3,7.7354796E-2,1.02882884E-1,1.962206E-1,-1.3367072E-2,-2.275188E-2,-3.5706526E-3,8.042419E-2,1.3515028E-1,5.812674E-2,1.0124369E-2,8.997898E-2,-4.4659856E-3,-2.0337415E-4,1.0180435E-4,-2.7191702E-3,-8.4482925E-3,-2.1657597E-3,-4.703047E-3,-6.4426457E-4,-1.0158487E-2,-4.609284E-3,-9.525904E-4,2.254283E-3,-8.0058066E-4,-3.372192E-3,9.4284205E-4,5.790955E-3,1.5408425E-4,5.404589E-3,4.3060756E-3,1.0701922E-2,-4.781559E-3,2.3125742E-4,8.74795E-4,-3.619816E-3,1.08540235E-2,2.4840198E-3,3.1417976E-3,9.341854E-3,-3.8126172E-3,4.335523E-3,5.1972517E-3,1.2449378E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,-1,41,43,45,47,-1,49,-1,51,53,55,-1,-1,-1,-1,57,59,61,-1,-1,63,65,-1,-1,67,69,-1,71,73,75,77,79,81,83,85,87,-1,89,91,93,-1,-1,95,-1,97,99,-1,-1,101,-1,103,-1,-1,-1,-1,105,107,109,111,113,115,117,-1,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,121,123,125,-1,127,129,-1,-1,-1,131,133,135,137,139,-1,141,-1,143,145,-1,-1,147,149,151,-1,-1,-1,153,155,-1,-1,157,159,-1,-1,161,-1,163,-1,-1,-1,-1,165,167,169,-1,171,173,175,177,179,-1,181,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.654453E0,1.6975689E0,7.667732E-1,6.793349E-1,6.62178E-1,6.3382363E-1,6.9172263E-1,3.7684703E-1,0E0,6.0705507E-1,4.1790777E-1,3.2240415E-1,0E0,4.243307E-1,2.0201927E-1,4.7706652E-1,5.1366234E-1,2.4162403E-1,5.681142E-1,1.0848279E-1,0E0,3.30796E-1,7.215449E-2,0E0,7.2004795E-2,1.3346338E-1,7.913114E-2,2.8285062E-1,0E0,1.3077933E-1,0E0,2.907496E-1,1.3316101E-1,3.5620385E-1,0E0,0E0,0E0,0E0,1.9985628E-1,5.2359037E-2,3.54219E-2,0E0,0E0,1.23048484E-1,7.11292E-2,0E0,0E0,4.0957606E-1,1.8339062E-1,0E0,3.9302245E-2,6.1125803E-1,3.7109327E-2,7.654405E-2,1.1588156E-1,1.3835764E-1,2.7583617E-1,1.3804603E-1,1.7011037E-1,0E0,5.2954093E-2,2.1119406E-2,1.7842624E-2,0E0,0E0,8.141511E-2,0E0,1.291213E-1,6.7693174E-1,0E0,0E0,5.9106953E-2,0E0,7.43168E-2,0E0,0E0,0E0,0E0,1.7705292E-2,4.181229E-2,1.4068162E-1,7.419587E-2,7.533151E-2,1.9667351E-1,4.1715175E-2,0E0,1.7730188E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4275563E-1,6.391525E-2,1.2588277E-1,0E0,2.4744278E-2,9.627879E-2,0E0,0E0,0E0,3.1703446E-2,2.3817562E-2,8.698554E-2,2.986348E-2,5.529017E-2,0E0,6.329346E-2,0E0,3.3240622E-1,1.6183195E-1,0E0,0E0,6.1468035E-2,1.318196E-1,8.819911E-2,0E0,0E0,0E0,4.716234E-2,4.137495E-2,0E0,0E0,3.5863835E-2,6.179875E-2,0E0,0E0,2.0383798E-2,0E0,5.606109E-2,0E0,0E0,0E0,0E0,7.3967054E-2,4.453203E-2,2.5852442E-2,0E0,1.7534298E-1,1.0919836E-1,1.3564631E-1,8.080357E-2,1.433868E-1,0E0,1.6839348E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,38,38,39,39,40,40,43,43,44,44,47,47,48,48,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,65,65,67,67,68,68,71,71,73,73,78,78,79,79,80,80,81,81,82,82,83,83,84,84,86,86,98,98,99,99,100,100,102,102,103,103,107,107,108,108,109,109,110,110,111,111,113,113,115,115,116,116,119,119,120,120,121,121,125,125,126,126,129,129,130,130,133,133,135,135,140,140,141,141,142,142,144,144,145,145,146,146,147,147,148,148,150,150],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,-1,42,44,46,48,-1,50,-1,52,54,56,-1,-1,-1,-1,58,60,62,-1,-1,64,66,-1,-1,68,70,-1,72,74,76,78,80,82,84,86,88,-1,90,92,94,-1,-1,96,-1,98,100,-1,-1,102,-1,104,-1,-1,-1,-1,106,108,110,112,114,116,118,-1,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,122,124,126,-1,128,130,-1,-1,-1,132,134,136,138,140,-1,142,-1,144,146,-1,-1,148,150,152,-1,-1,-1,154,156,-1,-1,158,160,-1,-1,162,-1,164,-1,-1,-1,-1,166,168,170,-1,172,174,176,178,180,-1,182,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.8439855E5,1.6466942E3,3.8304348E1,1.2674157E1,9.7172376E7,1.7103828E7,1E0,-2.0608572E-2,8.23E2,1E0,8E0,-8.679268E-3,1.5599597E8,2.5130852E2,6.68411E2,7.45E0,1.5933333E2,1.4676277E7,1.4787234E1,-2.1408575E-2,6.6903913E-1,1.4E1,2.6379682E-3,4.48375E5,1.5277778E0,3.85374E8,3.0977E4,1.0311958E-2,5E0,-2.2984022E-2,4.57E2,5.8182236E1,9.538462E0,-1.2919953E-2,1.5573751E-3,-8.327399E-3,-5.9242765E-3,5.2409735E-7,7.601393E2,2.6844707E8,1.0673161E-2,1.8483171E-2,4.420569E1,1.270394E5,4.742078E-3,-2.6675216E-3,3.72E2,1.266E3,-9.565029E-3,1E0,4.5865917E8,3.5714287E-1,5.2224144E5,1.3619632E0,1.188E3,2.779E3,6.229955E4,2.6750835E6,8.1400096E-4,4.4522205E6,1.3342042E0,4.6750406E4,4.9015544E-3,1.760288E-2,8.93E2,7.047192E-3,6.6E0,1.6E1,-1.4662112E-2,-3.4708255E-3,7.026624E7,-6.594967E-3,4.92E2,-2.1288743E-2,3.6841158E-3,-1.3964837E-3,-4.7106115E-4,1.08712E5,2.4346602E2,3.7066666E2,1E0,1.1208894E0,1.273801E6,4.5E0,1.3359898E-2,1.606722E8,5.19689E-3,-6.3412455E-3,7.966141E-3,1.012034E-3,-4.2694362E-4,-4.5731384E-3,-1.2872173E-3,2.0908068E-3,-2.7223488E-3,5.3299638E-3,-1.0546174E-2,6.3623324E-2,1.9E1,4.142857E0,-5.502692E-3,1.6E1,3.1E1,-5.4800945E-5,-3.7516362E-3,-9.49331E-3,1.901875E2,2.12E2,1.6100003E8,9.207547E0,1.4953704E0,-2.2882926E-3,1.2E1,1.2547638E-4,5E0,1.5736016E6,9.001348E-3,2.7892513E-3,2.5054401E1,1.7103828E7,1.99E2,-7.4599828E-3,5.0631985E-3,1.3716438E-2,1.8E0,5.3776E4,2.7592587E-3,-1.8470054E-3,4.9E2,9.502128E1,1.4266383E-3,-1.8684632E-3,5.08E2,4.7341147E-3,1.1074469E6,2.8386528E-3,4.7767456E-3,6.6411955E-4,-1.1531069E-3,1.7143776E7,3.655004E7,2.8655008E2,-1.3367072E-2,3.64E2,1.3705882E1,1.4872362E-1,1.8863014E1,2.9355192E1,1.0124369E-2,1.3695652E0,-4.4659856E-3,-2.0337415E-4,1.0180435E-4,-2.7191702E-3,-8.4482925E-3,-2.1657597E-3,-4.703047E-3,-6.4426457E-4,-1.0158487E-2,-4.609284E-3,-9.525904E-4,2.254283E-3,-8.0058066E-4,-3.372192E-3,9.4284205E-4,5.790955E-3,1.5408425E-4,5.404589E-3,4.3060756E-3,1.0701922E-2,-4.781559E-3,2.3125742E-4,8.74795E-4,-3.619816E-3,1.08540235E-2,2.4840198E-3,3.1417976E-3,9.341854E-3,-3.8126172E-3,4.335523E-3,5.1972517E-3,1.2449378E-3],"split_indices":[2,43,67,68,69,60,62,6,0,2,6,32,0,46,73,67,69,4,62,71,0,68,3,0,1,68,47,44,0,6,0,0,62,73,0,0,0,0,52,67,7,0,0,71,48,0,0,2,2,0,113,5,68,43,68,2,2,48,43,0,62,53,48,0,0,8,0,67,10,0,0,7,0,2,0,0,0,0,7,4,4,19,53,9,69,0,12,0,0,0,0,0,0,0,0,0,0,0,53,0,68,0,3,10,0,0,0,4,10,5,73,69,0,8,0,3,62,0,0,73,62,2,0,0,0,68,1,0,0,1,67,0,0,2,0,60,0,0,0,0,5,7,4,0,10,71,57,73,73,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.032E3,7.86E2,2.46E2,2.53E2,5.33E2,1.63E2,8.3E1,2.48E2,5E0,5.21E2,1.2E1,1.56E2,7E0,3.6E1,4.7E1,2.08E2,4E1,2.37E2,2.84E2,8E0,4E0,1.12E2,4.4E1,6E0,3E1,3.6E1,1.1E1,2.04E2,4E0,3.6E1,4E0,6.6E1,1.71E2,2.79E2,5E0,4E0,4E0,4E0,1.08E2,2E1,2.4E1,9E0,2.1E1,8E0,2.8E1,4E0,7E0,1.9E2,1.4E1,1.7E1,1.9E1,5.4E1,1.2E1,1.4E1,1.57E2,8.8E1,1.91E2,9.9E1,9E0,1E1,1E1,1E1,1.4E1,4E0,4E0,9E0,1.9E1,1.08E2,8.2E1,8E0,6E0,1.3E1,6E0,5E1,4E0,8E0,4E0,5E0,9E0,5E1,1.07E2,4E1,4.8E1,1.79E2,1.2E1,7E0,9.2E1,5E0,4E0,6E0,4E0,6E0,4E0,4E0,1E1,4E0,5E0,1E1,9.8E1,1E1,7.2E1,5E0,8E0,4.2E1,8E0,4E0,5E0,2.3E1,2.7E1,9.2E1,1.5E1,3.3E1,7E0,4.3E1,5E0,8.1E1,9.8E1,7E0,5E0,4.6E1,4.6E1,6.5E1,3.3E1,6E0,4E0,5.9E1,1.3E1,4E0,4E0,1.9E1,2.3E1,1.1E1,1.2E1,2.1E1,6E0,8.6E1,6E0,6E0,9E0,6E0,2.7E1,3.1E1,1.2E1,4E0,7.7E1,6.3E1,3.5E1,2E1,2.6E1,3.3E1,1.3E1,5.2E1,1.3E1,1.8E1,4.1E1,9E0,4E0,7E0,1.2E1,8E0,1.5E1,5E0,1.6E1,2.7E1,5.9E1,1.3E1,1.4E1,4E0,2.7E1,4E0,8E0,1.9E1,5.8E1,4.9E1,1.4E1,4E0,3.1E1,1.1E1,9E0,5E0,2.1E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"183","size_leaf_vector":"1"}},{"base_weights":[1.7595345E-3,-4.604001E-2,9.6235365E-2,-1.1945923E-1,-1.0832919E-2,3.736891E-1,8.147172E-2,-9.8599575E-2,-2.7050975E-1,-6.886885E-3,-1.6953487E-2,8.473743E-3,1.9316973E-2,-1.4762089E-1,9.934878E-2,1.3500729E-1,-1.09373026E-1,-2.1028584E-2,-2.0486724E-1,-4.6014715E-2,1.782059E-2,-1.080661E-1,-1.1470033E-2,5.8647767E-2,1.4758804E-1,1.3014028E-2,-1.7870535E-3,-1.3100278E-1,-5.6821752E-2,-1.0959197E-2,-2.4903528E-3,-1.7599622E-2,-8.317143E-2,4.2048253E-2,-2.7355488E-2,-6.2773395E-3,-1.3967625E-4,6.88182E-2,-5.6689575E-2,1.5992443E-1,-3.3664703E-3,-1.1612602E-1,-2.1794435E-1,-1.2858361E-1,-1.6495831E-2,-9.081926E-2,4.7345733E-4,-6.795864E-2,-1.3194508E-2,1.1827146E-2,9.2539124E-2,-5.6345433E-2,3.4197066E-2,1.7180946E-1,4.736045E-2,2.1479174E-3,-6.4626858E-3,1.9397405E-1,9.4942704E-2,-1.5816578E-1,-7.565414E-2,-2.568005E-1,-3.0287288E-3,-2.9726156E-3,-7.6712733E-3,3.5225672E-3,-4.4709455E-2,-5.1574036E-2,-7.2225532E-3,1.642953E-2,-5.7172E-2,-1.4236178E-1,-3.4530643E-2,-6.482677E-3,2.0018592E-2,1.0792761E-1,-2.0332953E-2,-3.987955E-2,-1.4621301E-1,5.1721134E-3,-4.0996403E-2,1.9274902E-1,2.0934418E-3,-5.3314183E-2,6.4319156E-2,3.2232958E-1,1.6412465E-1,2.9443393E-2,1.5648606E-1,-2.4898688E-3,-1.8300876E-1,3.500623E-2,-1.07007146E-1,-1.3859208E-2,-6.8937135E-3,-6.735104E-2,2.4628008E-3,-4.575952E-3,-1.657172E-4,5.641105E-3,3.8313204E-3,-4.8381225E-3,7.0951093E-4,-7.486937E-2,-1.16012655E-2,-5.3010643E-2,1.7319338E-3,7.063823E-2,-1.6112518E-2,1.2887438E-1,2.989721E-2,-3.2878083E-3,1.5969548E-3,2.4510561E-3,-5.5810045E-2,-8.785496E-3,-2.9672084E-3,-7.760685E-2,1.8044072E-3,4.2414702E-3,9.705485E-3,-8.020313E-3,-5.9973323E-3,1.0236308E-1,2.6608733E-3,5.888042E-3,1.7629962E-2,8.491013E-2,2.1679874E-1,6.883945E-2,-4.6299775E-3,2.3849605E-4,1.8649268E-1,-2.0220388E-1,-2.432959E-3,-2.0935365E-3,4.6620397E-3,-1.6317834E-1,-6.973752E-2,-2.7008492E-2,-4.8713614E-3,3.6684677E-2,-2.1578815E-2,-1.1000487E-1,8.039836E-4,-2.8681986E-2,-5.0548567E-3,6.1379494E-3,4.7287244E-2,-7.388654E-2,2.96053E-2,4.853903E-2,1.5816322E-1,-2.2769286E-3,7.9363555E-2,-4.1126978E-2,-7.236221E-3,-5.611879E-3,-1.3361542E-3,-3.42289E-3,1.9650548E-3,1.2937905E-1,6.474979E-3,-2.7502578E-2,5.5280607E-2,7.678604E-3,4.2154018E-2,2.4456646E-1,6.6431584E-3,4.554645E-3,2.4831984E-2,1.19303465E-2,5.4118815E-3,-6.7189825E-3,-1.1882401E-2,-1.0256936E-2,-4.8764395E-3,1.4483294E-3,-3.9959946E-3,-3.5947496E-3,6.684414E-4,3.0958077E-3,-2.0195788E-3,1.06457E-3,-2.1839691E-3,-6.664084E-3,-1.8509475E-3,-2.122441E-3,7.712547E-4,1.1582002E-3,3.832561E-3,4.7430227E-4,-5.484657E-3,6.6732673E-4,5.716895E-3,-1.0076571E-3,5.6365225E-3,2.6749328E-3,8.013383E-3,6.4799343E-3,1.1980372E-4,-2.738094E-3,9.392785E-4,7.1157697E-3,3.6971763E-4,-5.3268857E-3,3.3406836E-3,-6.0323866E-5,-5.6981924E-3,-4.1920433E-4,5.579428E-3,4.774754E-3,-1.6400062E-3,8.504729E-3,1.454819E-2,-9.786241E-4,3.0436092E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,-1,29,31,33,35,-1,37,39,-1,-1,41,43,-1,-1,45,47,49,51,-1,-1,53,55,57,-1,59,61,63,65,67,69,71,-1,73,75,77,79,81,83,-1,-1,85,87,89,91,93,-1,-1,-1,-1,95,97,-1,99,101,103,105,-1,107,109,111,113,115,-1,117,119,-1,121,123,125,127,129,131,-1,133,135,137,-1,-1,139,-1,-1,-1,141,-1,-1,-1,143,-1,145,-1,147,149,151,153,-1,-1,-1,155,-1,-1,157,-1,-1,-1,159,-1,161,163,-1,-1,165,167,169,-1,-1,171,173,-1,-1,-1,175,177,179,-1,181,183,185,-1,187,-1,-1,189,191,193,195,197,-1,199,201,-1,-1,-1,-1,-1,203,205,207,209,-1,211,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5971026E0,1.7460828E0,1.3704648E0,6.6336656E-1,6.411064E-1,6.704068E-2,1.3498001E0,5.030161E-1,2.219162E-1,4.3972442E-1,0E0,0E0,0E0,6.872165E-2,5.8625054E-1,2.4019821E-1,2.0397711E-1,0E0,8.976817E-2,1.8435901E-1,3.065188E-1,5.509244E-2,0E0,1.9862425E-1,3.8696432E-1,0E0,0E0,1.441276E-1,1.5993524E-1,0E0,0E0,1.3369673E-1,2.146346E-1,2.766175E-1,1.7679854E-1,0E0,0E0,3.29987E-1,1.2668796E-1,2.7143478E-1,0E0,1.8199694E-1,8.9745104E-2,3.7450433E-2,1.0022424E-1,4.3361157E-2,7.636251E-2,1.7415881E-1,0E0,1.4613466E-1,1.2030578E-1,9.331225E-2,1.9168481E-1,6.0133874E-2,2.2196496E-1,0E0,0E0,2.727611E-1,1.848517E-1,1.326096E-1,2.1191806E-1,2.028513E-2,0E0,0E0,0E0,0E0,6.794954E-2,3.1539E-2,0E0,4.616598E-2,6.368832E-2,1.4267936E-1,6.9863915E-2,0E0,2.0232198E-1,9.421611E-2,2.757657E-2,8.882888E-2,2.382189E-2,0E0,5.2854706E-2,1.9199014E-2,0E0,6.479424E-2,2.5775716E-1,1.24649405E-1,2.7784264E-1,1.294945E-1,1.08014405E-1,0E0,9.943998E-2,7.868431E-2,8.70139E-2,0E0,0E0,3.4033477E-2,0E0,0E0,0E0,4.895681E-2,0E0,0E0,0E0,5.0544694E-2,0E0,5.3123437E-2,0E0,6.0551405E-2,1.7387208E-1,1.0291058E-1,5.8747545E-2,0E0,0E0,0E0,6.757656E-2,0E0,0E0,2.2113673E-2,0E0,0E0,0E0,4.6573278E-2,0E0,1.7531699E-1,6.976964E-2,0E0,0E0,9.7271815E-2,4.9973845E-2,2.2425406E-2,0E0,0E0,6.925994E-2,8.123171E-2,0E0,0E0,0E0,3.759569E-2,5.353117E-2,2.8518198E-2,0E0,6.87595E-2,3.6631614E-2,2.0564884E-2,0E0,2.4261696E-2,0E0,0E0,2.5657915E-2,1.10627115E-1,5.1019773E-2,7.501811E-2,4.2080402E-2,0E0,4.171247E-2,5.1374175E-2,0E0,0E0,0E0,0E0,0E0,1.5730798E-1,1.3661493E-1,7.013682E-2,6.852773E-2,0E0,1.03464484E-1,6.53888E-2,0E0,0E0,1.8395005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,27,27,28,28,31,31,32,32,33,33,34,34,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,66,66,67,67,69,69,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78,80,80,81,81,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91,92,92,95,95,99,99,103,103,105,105,107,107,108,108,109,109,110,110,114,114,117,117,121,121,123,123,124,124,127,127,128,128,129,129,132,132,133,133,137,137,138,138,139,139,141,141,142,142,143,143,145,145,148,148,149,149,150,150,151,151,152,152,154,154,155,155,161,161,162,162,163,163,164,164,166,166,167,167,170,170],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,-1,30,32,34,36,-1,38,40,-1,-1,42,44,-1,-1,46,48,50,52,-1,-1,54,56,58,-1,60,62,64,66,68,70,72,-1,74,76,78,80,82,84,-1,-1,86,88,90,92,94,-1,-1,-1,-1,96,98,-1,100,102,104,106,-1,108,110,112,114,116,-1,118,120,-1,122,124,126,128,130,132,-1,134,136,138,-1,-1,140,-1,-1,-1,142,-1,-1,-1,144,-1,146,-1,148,150,152,154,-1,-1,-1,156,-1,-1,158,-1,-1,-1,160,-1,162,164,-1,-1,166,168,170,-1,-1,172,174,-1,-1,-1,176,178,180,-1,182,184,186,-1,188,-1,-1,190,192,194,196,198,-1,200,202,-1,-1,-1,-1,-1,204,206,208,210,-1,212,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.935E3,1.972052E5,3.6E1,2.6E1,5.6E1,2.00033E5,8.147158E4,6E0,5.6190475E1,6.99E2,-1.6953487E-2,8.473743E-3,1.9316973E-2,3.6672198E-8,5.709E3,1.89991E5,8.147158E4,-2.1028584E-2,4.4E1,1.53053E5,2E0,9.234473E-9,-1.1470033E-2,2.0319266E-7,7.06191E7,1.3014028E-2,-1.7870535E-3,2.7E1,1.463E2,-1.0959197E-2,-2.4903528E-3,6.613774E5,1.1949272E7,4.6066E4,9.750085E6,-6.2773395E-3,-1.3967625E-4,2.5959E4,1.1621469E1,2.5503825E6,-3.3664703E-3,3.8714287E1,6.593462E4,6.1E1,9.593451E4,3.1E1,3.3512E4,4.0832064E7,-1.3194508E-2,5.6591835E1,6.1114804E-8,2.8025E4,2.4E1,2.871795E0,5.9754E4,2.1479174E-3,-6.4626858E-3,2.5959E4,1.9616238E2,3.8E1,1.2E1,3.515484E1,-3.0287288E-3,-2.9726156E-3,-7.6712733E-3,3.5225672E-3,6.354E3,4.9E2,-7.2225532E-3,9E0,3.84E2,1.64388E5,1.8E7,-6.482677E-3,1E0,1.0988551E8,5E0,2.4243538E3,1.007E3,5.1721134E-3,1.594E3,2.297E3,2.0934418E-3,1.136E3,1.3922E4,1.4194314E3,8.439E3,1.3475722E2,2.234478E3,-2.4898688E-3,1.16E2,4.1142857E1,1.7467743E-2,-1.3859208E-2,-6.8937135E-3,4.8581E4,2.4628008E-3,-4.575952E-3,-1.657172E-4,7.619497E0,3.8313204E-3,-4.8381225E-3,7.0951093E-4,7.785302E5,-1.16012655E-2,9.776408E5,1.7319338E-3,3.63E2,2.026453E6,2.5912744E2,1.7728971E1,-3.2878083E-3,1.5969548E-3,2.4510561E-3,1.4172E4,-8.785496E-3,-2.9672084E-3,1.4260629E8,1.8044072E-3,4.2414702E-3,9.705485E-3,6.27E2,-5.9973323E-3,2.459733E0,3.2585382E10,5.888042E-3,1.7629962E-2,9.237895E0,1.7103828E7,1.752E3,-4.6299775E-3,2.3849605E-4,8.837831E-2,7.1E1,-2.432959E-3,-2.0935365E-3,4.6620397E-3,4.488263E0,1E0,1.4473684E-1,-4.8713614E-3,3.979933E0,3.6401E4,4.03E2,8.039836E-4,2.4332604E0,-5.0548567E-3,6.1379494E-3,1.0894737E1,2.1525E4,1.8002772E5,3E0,7.619497E0,-2.2769286E-3,1.7790684E5,3.19E2,-7.236221E-3,-5.611879E-3,-1.3361542E-3,-3.42289E-3,1.9650548E-3,4.797448E8,6.9508715E0,8.072743E4,2.3472653E-1,7.678604E-3,2.2007043E0,1.0494994E7,6.6431584E-3,4.554645E-3,2.0805957E-2,1.19303465E-2,5.4118815E-3,-6.7189825E-3,-1.1882401E-2,-1.0256936E-2,-4.8764395E-3,1.4483294E-3,-3.9959946E-3,-3.5947496E-3,6.684414E-4,3.0958077E-3,-2.0195788E-3,1.06457E-3,-2.1839691E-3,-6.664084E-3,-1.8509475E-3,-2.122441E-3,7.712547E-4,1.1582002E-3,3.832561E-3,4.7430227E-4,-5.484657E-3,6.6732673E-4,5.716895E-3,-1.0076571E-3,5.6365225E-3,2.6749328E-3,8.013383E-3,6.4799343E-3,1.1980372E-4,-2.738094E-3,9.392785E-4,7.1157697E-3,3.6971763E-4,-5.3268857E-3,3.3406836E-3,-6.0323866E-5,-5.6981924E-3,-4.1920433E-4,5.579428E-3,4.774754E-3,-1.6400062E-3,8.504729E-3,1.454819E-2,-9.786241E-4,3.0436092E-3],"split_indices":[2,43,10,3,3,5,43,3,62,2,0,0,0,52,2,9,43,0,3,9,32,52,0,52,59,0,0,8,70,0,0,60,60,1,9,0,0,9,71,47,0,70,43,10,43,10,1,7,0,62,52,44,8,68,1,0,0,9,71,44,33,73,0,0,0,0,10,1,0,8,0,62,5,0,26,7,8,62,2,0,2,2,0,44,44,67,2,73,4,0,44,67,53,0,0,1,0,0,0,73,0,0,0,43,0,62,0,44,9,4,73,0,0,0,44,0,0,7,0,0,0,44,0,53,46,0,0,71,62,0,0,0,72,44,0,0,0,69,21,71,0,69,9,2,0,71,0,0,73,9,48,8,73,0,48,0,0,0,0,0,0,7,71,48,53,0,69,62,0,0,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,6.75E2,3.41E2,2.18E2,4.57E2,1.6E1,3.25E2,1.93E2,2.5E1,4.53E2,4E0,4E0,1.2E1,2.3E1,3.02E2,8E0,1.85E2,5E0,2E1,1.75E2,2.78E2,1.8E1,5E0,1.65E2,1.37E2,4E0,4E0,1.3E2,5.5E1,1.6E1,4E0,1E2,7.5E1,1.81E2,9.7E1,1.4E1,4E0,1.52E2,1.3E1,1.3E2,7E0,1.13E2,1.7E1,1.9E1,3.6E1,1.9E1,8.1E1,7.1E1,4E0,1.14E2,6.7E1,6.6E1,3.1E1,2.5E1,1.27E2,6E0,7E0,8.4E1,4.6E1,5.4E1,5.9E1,1.3E1,4E0,8E0,1.1E1,8E0,2.8E1,1.3E1,6E0,6.4E1,1.7E1,2.1E1,5E1,5E0,1.09E2,5.9E1,8E0,5.7E1,9E0,1.5E1,1.6E1,2.1E1,4E0,1.8E1,1.09E2,1.4E1,7E1,2.3E1,2.3E1,1.1E1,4.3E1,1.3E1,4.6E1,8E0,5E0,2.3E1,5E0,6E0,7E0,5.6E1,8E0,1E1,7E0,1.4E1,7E0,4E1,1E1,4.5E1,6.4E1,4.6E1,1.3E1,4E0,4E0,8E0,4.9E1,5E0,4E0,1.1E1,5E0,4E0,1.7E1,1.2E1,6E0,6.7E1,4.2E1,4E0,1E1,2.9E1,4.1E1,1.8E1,5E0,4E0,1.9E1,3.7E1,6E0,6E0,7E0,1.7E1,2.9E1,1.2E1,1.1E1,2.6E1,3E1,1E1,4E0,2.9E1,1.1E1,1.1E1,3.4E1,2.8E1,3.6E1,1.3E1,3.3E1,5E0,8E0,4.4E1,5E0,5E0,6E0,5E0,7E0,5.2E1,1.5E1,2.7E1,1.5E1,9E0,2E1,2.8E1,1.3E1,1E1,8E0,8E0,1.1E1,2E1,1.7E1,7E0,1E1,4E0,2.5E1,5E0,7E0,1.9E1,7E0,1.1E1,1.9E1,6E0,4E0,2.1E1,8E0,2.2E1,1.2E1,1E1,1.8E1,3.2E1,4E0,7E0,6E0,5E0,2.8E1,4E0,4E0,3.4E1,1E1,4.3E1,9E0,5E0,1E1,2.2E1,5E0,8E0,7E0,1.1E1,9E0,1.7E1,1.1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-4.246676E-3,-6.808504E-2,5.8847517E-2,-1.24956235E-1,-3.197511E-2,7.25163E-2,-1.878514E-1,-1.0967272E-1,-2.89086E-1,-7.940078E-2,-1.0550625E-2,3.926635E-2,1.5051953E-1,-2.3990951E-1,-2.2365642E-4,-1.8054748E-1,-8.5415095E-2,-1.2362893E-1,-2.0285517E-2,-6.1006423E-2,-1.2663756E-2,-1.9150887E-2,1.4990428E-1,4.6509113E-2,-1.3270856E-1,1.8738891E-1,8.367987E-2,-1.4224235E-2,-8.497253E-3,-1.12483455E-2,-1.5302731E-1,3.4755883E-3,-9.085308E-2,-1.9021727E-3,-8.378598E-3,3.4661505E-2,-8.086697E-2,-7.2961594E-3,-1.2680723E-2,3.8711128E-3,9.816399E-3,3.8670145E-2,1.7892972E-1,-1.2233176E-3,-1.8878269E-1,1.6144902E-1,1.6803693E-2,1.1860967E-2,6.0304508E-2,-2.1567966E-3,-7.842399E-3,1.1266137E-2,-9.841681E-2,-2.2521977E-4,2.7197797E-3,-6.1837643E-2,-1.74032E-1,-1.715104E-2,7.461865E-3,1.2748428E-2,3.466672E-2,9.915375E-3,1.9959281E-3,-1.8064856E-3,-1.389236E-2,1.9271572E-1,1.03793584E-1,2.5398724E-2,1.1706977E-1,3.3249524E-3,-1.9033191E-3,-1.2271502E-1,-4.8634205E-2,-8.125813E-2,-4.508185E-3,-1.168239E-2,-6.0444773E-4,4.4353185E-3,-4.218284E-2,5.3904797E-3,6.13476E-2,1.6982117E-1,1.570477E-2,-3.8522992E-3,1.3461204E-1,-6.3506454E-2,5.9083413E-2,1.8047484E-3,1.6315807E-1,-9.565868E-2,-1.641413E-1,-9.196156E-2,-5.0656255E-3,-9.1702044E-2,1.1462504E-3,3.364774E-3,-4.2802643E-2,1.6971877E-2,-7.107972E-2,-6.7158826E-2,4.4043483E-3,-1.4942856E-1,1.9011347E-2,7.560417E-2,-8.150623E-2,2.0148331E-1,8.079732E-2,1.5145561E-1,1.3956367E-3,3.846139E-4,-5.665702E-3,1.752344E-2,1.02828525E-1,3.7424928E-3,9.171824E-3,-6.809631E-2,-1.6560681E-1,-2.2617683E-1,-5.223122E-3,-6.6056475E-2,-6.7318887E-3,1.8655114E-2,-4.1323737E-3,-6.2383365E-2,-1.3287278E-1,-1.1996457E-4,-3.8149245E-3,2.5052862E-2,-5.794073E-3,-4.846777E-3,-8.051778E-4,-9.425922E-2,-1.6991132E-3,-2.2909991E-2,5.1837415E-2,-1.8868674E-2,1.4377348E-3,-4.9310926E-2,3.409211E-2,1.0709327E-1,3.241865E-2,2.4693515E-3,-1.7741615E-1,7.040766E-3,1.1711209E-2,-1.824229E-4,5.9281304E-3,2.946607E-3,7.694757E-3,-9.253373E-4,3.7704797E-3,1.7401703E-3,6.9692265E-3,-4.020863E-3,2.170386E-4,-8.869812E-3,-3.6828346E-3,-5.5350126E-3,-1.2828617E-2,-4.4012032E-3,-6.8798754E-4,-7.7901117E-4,1.9768511E-3,-4.1316934E-3,-3.4449282E-4,-9.569709E-3,-4.346737E-3,-2.4460553E-4,2.775861E-3,-1.1375634E-3,-5.2239527E-3,-2.4104996E-3,1.2883787E-3,-2.7757033E-3,1.5397029E-3,5.1759507E-5,4.0610735E-3,-9.3171693E-4,-5.6071896E-3,1.0261538E-3,7.070875E-3,5.9179016E-3,4.5398576E-4,-2.8310283E-3,2.1788187E-3,-2.7921349E-3,-1.1961488E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,41,43,45,47,-1,-1,-1,49,-1,51,-1,-1,53,55,-1,57,-1,-1,59,61,-1,63,65,-1,-1,67,-1,-1,69,71,-1,-1,73,75,77,-1,-1,79,-1,-1,-1,-1,81,83,85,87,-1,-1,89,91,93,95,-1,-1,97,99,101,103,105,-1,-1,107,109,111,-1,113,115,117,119,121,123,-1,-1,125,127,129,131,133,135,137,139,141,143,145,147,-1,-1,-1,149,151,-1,-1,153,155,157,-1,159,-1,161,-1,163,165,-1,-1,167,-1,-1,-1,169,171,173,175,-1,-1,177,179,181,183,-1,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0761423E0,1.0289729E0,1.730008E0,4.5744872E-1,3.1312805E-1,1.2468004E0,2.5607204E-1,2.9404402E-1,3.4282207E-1,3.2711142E-1,2.9818097E-1,4.29667E-1,3.391838E-1,2.3290992E-2,0E0,4.0672302E-2,1.2564307E-1,3.537406E-2,0E0,1.719045E-1,0E0,1.8117213E-1,2.7703434E-2,3.3252007E-1,7.839517E-2,3.6209536E-1,1.9891614E-1,0E0,0E0,0E0,5.2546978E-2,0E0,1.0306585E-1,0E0,0E0,1.5601469E-2,1.2043044E-1,0E0,1.5539777E-1,0E0,0E0,2.8490737E-1,7.79292E-2,0E0,1.3901716E-1,1.2781835E-1,0E0,0E0,9.213552E-2,0E0,0E0,3.481302E-2,1.4281607E-1,0E0,0E0,6.9775745E-2,1.4895931E-1,1.0393341E-1,0E0,0E0,2.3869076E-1,0E0,0E0,0E0,0E0,1.1754084E-1,1.9004968E-1,9.658921E-2,5.8338553E-2,0E0,0E0,7.620764E-2,7.881805E-2,5.434078E-2,5.3466383E-2,0E0,0E0,9.9663034E-2,1.0385272E-1,3.1282997E-1,3.320192E-1,1.1619902E-1,0E0,0E0,4.2904586E-2,3.9247792E-2,4.0013142E-2,0E0,1.5880048E-2,8.897549E-2,7.2669566E-2,1.9616753E-2,4.570368E-2,4.3615013E-2,0E0,0E0,1.8974999E-2,1.0669867E-1,2.5397733E-2,1.027309E-1,4.2646885E-2,5.738407E-1,1.4180899E-1,1.9506961E-1,2.0363793E-1,4.816413E-2,5.6886256E-2,1.9828439E-2,0E0,0E0,0E0,3.2922707E-2,2.8740212E-2,0E0,0E0,5.417095E-2,1.6421348E-2,3.7951946E-2,0E0,2.2493325E-2,0E0,1.6151093E-2,0E0,3.9447434E-2,2.9479772E-2,0E0,0E0,9.174736E-2,0E0,0E0,0E0,5.0936073E-2,2.8337257E-2,4.5901388E-2,2.1851707E-2,0E0,0E0,5.0053746E-2,1.5489137E-1,1.703397E-1,8.9916594E-2,0E0,7.0151925E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26,30,30,32,32,35,35,36,36,38,38,41,41,42,42,44,44,45,45,48,48,51,51,52,52,55,55,56,56,57,57,60,60,65,65,66,66,67,67,68,68,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,81,81,84,84,85,85,86,86,88,88,89,89,90,90,91,91,92,92,93,93,96,96,97,97,98,98,99,99,100,100,101,101,102,102,103,103,104,104,105,105,106,106,107,107,111,111,112,112,115,115,116,116,117,117,119,119,121,121,123,123,124,124,127,127,131,131,132,132,133,133,134,134,137,137,138,138,139,139,140,140,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,42,44,46,48,-1,-1,-1,50,-1,52,-1,-1,54,56,-1,58,-1,-1,60,62,-1,64,66,-1,-1,68,-1,-1,70,72,-1,-1,74,76,78,-1,-1,80,-1,-1,-1,-1,82,84,86,88,-1,-1,90,92,94,96,-1,-1,98,100,102,104,106,-1,-1,108,110,112,-1,114,116,118,120,122,124,-1,-1,126,128,130,132,134,136,138,140,142,144,146,148,-1,-1,-1,150,152,-1,-1,154,156,158,-1,160,-1,162,-1,164,166,-1,-1,168,-1,-1,-1,170,172,174,176,-1,-1,178,180,182,184,-1,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.071E3,1.972052E5,7.042942E7,4.6463413E0,1E0,1.087521E3,2.3308511E5,5.4E1,1.169375E2,3.7568388E0,2.0200005E8,2.1894855E1,6.6933334E1,1.1015E4,-2.2365642E-4,1.7857143E1,7E0,7.6811076E-3,-2.0285517E-2,6.728972E-1,-1.2663756E-2,2.35E2,7.4654E4,7.82261E6,3.4E1,2.1511E4,3.401282E1,-1.4224235E-2,-8.497253E-3,-1.12483455E-2,1.9363636E1,3.4755883E-3,5.7018555E4,-1.9021727E-3,-8.378598E-3,4.48E2,2.87E2,-7.2961594E-3,2.8034655E6,3.8711128E-3,9.816399E-3,2.3E1,1.3E1,-1.2233176E-3,2.4E1,2.3385885E0,1.6803693E-2,1.1860967E-2,7.802E-2,-2.1567966E-3,-7.842399E-3,1.16E2,1.339646E6,-2.2521977E-4,2.7197797E-3,5E0,1.409E3,1E0,7.461865E-3,1.2748428E-2,4.371613E2,9.915375E-3,1.9959281E-3,-1.8064856E-3,-1.389236E-2,3.478E4,1.5484655E6,3.3272727E0,1.14061164E5,3.3249524E-3,-1.9033191E-3,2.23607E5,1.901875E2,4.0119403E2,6.346204E-7,-1.168239E-2,-6.0444773E-4,1E0,3.011152E6,4E0,2.8827406E5,9.557399E6,1.570477E-2,-3.8522992E-3,2.5E1,1.6677892E2,7.073537E5,1.8047484E-3,2.101E3,1.2E1,2.135961E4,1.7559428E2,1.8E1,2.6686933E-2,1.1462504E-3,3.364774E-3,4.612245E0,1.2E1,2.988E4,4.831007E6,8.69E2,1.24E2,2.16577E5,2E0,2.67637E5,1.6E1,1.7026364E3,2.23462E5,1.3956367E-3,3.846139E-4,-5.665702E-3,8.25E2,3.3580637E8,3.7424928E-3,9.171824E-3,4.7181977E4,3.2216358E-1,4.6991256E7,-5.223122E-3,9.72E2,-6.7318887E-3,5.249058E5,-4.1323737E-3,4.2609915E6,1.48404255E1,-1.1996457E-4,-3.8149245E-3,5.4E1,-5.794073E-3,-4.846777E-3,-8.051778E-4,1.5132743E0,1.4E1,2.1818182E0,1.89965E4,-1.8868674E-2,1.4377348E-3,8.998703E-1,2.5176924E1,2E0,7.8461537E0,2.4693515E-3,1.2E2,7.040766E-3,1.1711209E-2,-1.824229E-4,5.9281304E-3,2.946607E-3,7.694757E-3,-9.253373E-4,3.7704797E-3,1.7401703E-3,6.9692265E-3,-4.020863E-3,2.170386E-4,-8.869812E-3,-3.6828346E-3,-5.5350126E-3,-1.2828617E-2,-4.4012032E-3,-6.8798754E-4,-7.7901117E-4,1.9768511E-3,-4.1316934E-3,-3.4449282E-4,-9.569709E-3,-4.346737E-3,-2.4460553E-4,2.775861E-3,-1.1375634E-3,-5.2239527E-3,-2.4104996E-3,1.2883787E-3,-2.7757033E-3,1.5397029E-3,5.1759507E-5,4.0610735E-3,-9.3171693E-4,-5.6071896E-3,1.0261538E-3,7.070875E-3,5.9179016E-3,4.5398576E-4,-2.8310283E-3,2.1788187E-3,-2.7921349E-3,-1.1961488E-2],"split_indices":[2,43,60,68,29,67,48,2,67,68,5,69,71,9,0,4,3,53,0,69,0,2,1,43,3,2,73,0,0,0,4,0,60,0,0,2,0,0,62,0,0,0,3,0,0,53,0,0,72,0,0,44,9,0,0,8,10,26,0,0,67,0,0,0,0,44,43,69,48,0,0,9,4,4,52,0,0,19,9,3,48,60,0,0,3,73,43,0,0,8,43,4,3,57,0,0,69,71,1,60,2,0,43,32,9,3,4,1,0,0,0,8,7,0,0,48,53,7,0,44,0,60,0,60,61,0,0,0,0,0,0,68,3,71,48,0,0,57,71,6,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,5.02E2,5.08E2,1.94E2,3.08E2,4.82E2,2.6E1,1.79E2,1.5E1,9.5E1,2.13E2,3.39E2,1.43E2,2E1,6E0,4.4E1,1.35E2,8E0,7E0,8.8E1,7E0,2.03E2,1E1,3.26E2,1.3E1,9.1E1,5.2E1,7E0,1.3E1,1.1E1,3.3E1,4E0,1.31E2,4E0,4E0,1.5E1,7.3E1,8E0,1.95E2,6E0,4E0,3.09E2,1.7E1,5E0,8E0,8.1E1,1E1,5E0,4.7E1,5E0,2.8E1,9E0,1.22E2,6E0,9E0,6.2E1,1.1E1,1.91E2,4E0,4E0,3.05E2,1.3E1,4E0,4E0,4E0,5.1E1,3E1,3E1,1.7E1,4E0,5E0,8.1E1,4.1E1,4.6E1,1.6E1,7E0,4E0,1.03E2,8.8E1,1.46E2,1.59E2,4.6E1,5E0,4E0,2.6E1,8E0,2.2E1,7E0,1E1,5.1E1,3E1,2E1,2.1E1,4.2E1,4E0,5E0,1.1E1,8.9E1,1.4E1,5.7E1,3.1E1,1.1E1,1.35E2,1.45E2,1.4E1,3.3E1,1.3E1,2.2E1,4E0,4E0,4E0,1.2E1,1E1,4E0,6E0,3.8E1,1.3E1,1.2E1,1.8E1,1.5E1,5E0,1.7E1,4E0,2.6E1,1.6E1,6E0,5E0,8.5E1,4E0,8E0,6E0,4E1,1.7E1,2E1,1.1E1,4E0,7E0,2.4E1,1.11E2,8.3E1,6.2E1,6E0,8E0,1.9E1,1.4E1,5E0,8E0,4E0,1.8E1,8E0,4E0,5E0,5E0,3E1,8E0,9E0,4E0,5E0,7E0,9E0,6E0,7E0,1E1,1.7E1,9E0,4E0,1.2E1,4.6E1,3.9E1,9E0,3.1E1,6E0,1.1E1,1.2E1,8E0,5E0,6E0,1.8E1,6E0,1.02E2,9E0,6.8E1,1.5E1,8E0,5.4E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[3.3297176E-3,-9.517032E-2,4.093917E-2,-7.589505E-2,-2.047397E-1,5.6218677E-3,1.35447E-1,-1.03723176E-1,3.9400604E-2,-1.5781309E-1,-2.3609966E-2,-3.5599433E-2,3.4445647E-2,2.018905E-2,1.2374736E-1,5.7373635E-2,-1.1288989E-1,1.1896653E-2,-1.1942283E-2,-1.7154962E-1,-1.6441952E-3,-2.5915174E-2,-2.3540896E-1,2.0166505E-2,1.0572814E-1,1.0986993E-1,1.3505884E-2,9.171345E-3,-2.3857206E-3,-1.0469012E-1,-1.2850202E-2,-4.5681067E-2,5.904921E-2,-1.9346522E-1,-9.161186E-2,-2.037825E-2,-9.737564E-3,-1.92913E-2,-2.050034E-3,2.4700554E-2,-8.195852E-3,7.24137E-2,9.736071E-3,1.1811017E-1,-5.0577535E-3,-1.2530367E-1,-5.4875832E-2,-4.7120946E-3,-5.9018005E-4,3.9337403E-3,1.2113429E-5,-3.963647E-3,-9.696643E-3,-1.2340919E-3,-6.384194E-3,1.3450192E-2,-3.5481364E-2,6.465759E-2,-4.6955356E-3,3.9165895E-2,1.2743217E-1,-3.98957E-3,1.247095E-1,-1.06488764E-1,-2.0684081E-1,-2.6475715E-2,-9.369074E-2,-2.8757644E-3,2.5149453E-2,-4.293659E-3,2.8145934E-2,-5.223333E-2,7.059656E-3,7.6398745E-2,-2.0471564E-2,-1.5931932E-2,1.3771853E-1,-1.8382893E-3,5.462749E-2,2.4348241E-3,8.603761E-3,1.4945285E-1,6.6638686E-2,-1.3527343E-1,-2.5583832E-2,-2.692773E-1,-4.0070787E-3,-4.8865356E-2,2.766803E-2,-8.288341E-3,-5.7927426E-2,3.6776708E-3,-5.5572303E-4,4.2710274E-2,-3.0675915E-3,-4.1916486E-2,-7.771451E-3,4.3633394E-3,-4.9401806E-3,6.3400075E-2,1.8184727E-1,-5.6452624E-2,2.567968E-3,-3.488993E-2,6.400086E-2,3.202812E-3,9.513486E-3,7.452817E-2,-6.550628E-4,1.7600492E-1,8.962082E-2,3.4909096E-2,1.9641206E-1,-1.5257972E-1,-7.524455E-3,6.721595E-2,-7.30793E-2,-1.6596582E-2,-9.118207E-3,-3.8823816E-3,-9.356743E-4,3.4212247E-3,-1.5489962E-3,-1.2179071E-3,-5.1467298E-3,2.8590102E-2,5.791706E-3,-1.2005941E-2,-8.84788E-2,-3.2254998E-3,5.03941E-3,4.7472272E-2,1.311127E-1,1.0810228E-2,3.8300692E-3,-5.5680884E-4,-4.5479448E-3,2.7110377E-2,-5.971857E-2,1.0920867E-1,6.9148587E-3,1.4481733E-3,5.27718E-3,1.3534814E-1,1.201341E-2,2.2169113E-2,1.1989972E-1,5.615528E-3,8.921606E-2,1.2231885E-2,3.469839E-3,-3.802598E-3,-7.929313E-3,2.7879747E-3,-2.9026833E-3,4.999125E-3,2.1167596E-4,-1.7738995E-3,-7.50784E-3,8.6023175E-4,3.6325655E-3,-2.8788622E-3,3.031062E-4,-8.70642E-3,-1.7986735E-3,2.8219619E-3,-2.2161666E-4,7.038843E-4,3.240193E-3,9.038444E-3,3.8805315E-3,-2.0097329E-3,4.4410056E-3,1.2077744E-3,-3.9940276E-3,1.1817028E-3,6.3428674E-3,-1.050409E-3,2.0927086E-3,5.188983E-3,1.0133847E-2,-1.5733469E-3,4.3168785E-3,6.6986037E-3,1.4355014E-3,-2.237358E-3,1.4948554E-3,9.953913E-4,6.7327907E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,45,-1,47,49,51,53,55,-1,-1,-1,57,-1,59,-1,61,-1,63,65,-1,67,-1,-1,-1,-1,-1,-1,69,71,73,75,77,79,-1,81,83,85,87,89,-1,91,-1,93,95,97,99,101,103,105,-1,107,-1,-1,109,111,113,115,117,-1,119,121,-1,123,-1,-1,125,-1,127,-1,-1,129,131,133,135,-1,137,139,-1,-1,141,-1,143,145,147,149,151,153,155,157,-1,-1,-1,-1,-1,-1,-1,-1,159,-1,161,163,-1,165,167,169,-1,-1,-1,-1,171,173,175,177,-1,-1,179,-1,181,183,185,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7269514E0,5.7159185E-1,2.4291286E0,7.677009E-1,5.086857E-1,6.3213587E-1,6.444812E-1,2.9000854E-1,5.293951E-1,5.6015015E-2,0E0,4.1766602E-1,3.1645578E-1,0E0,4.1709304E-1,1.7950486E-1,2.1750617E-1,0E0,9.610893E-2,4.0088475E-2,0E0,2.1065018E-1,3.2946062E-1,2.39079E-1,1.6665989E-1,3.3646822E-1,0E0,0E0,0E0,1.7282248E-1,0E0,6.8500035E-2,1.9922905E-2,2.6199758E-2,2.351398E-2,1.05214775E-1,0E0,0E0,0E0,3.0217084E-1,0E0,7.000798E-2,0E0,2.4420547E-1,0E0,1.6666985E-1,5.594392E-2,0E0,2.6957976E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0267708E-1,1.0156444E-1,1.10511065E-1,2.4062705E-1,3.51351E-2,5.609648E-2,0E0,2.3143435E-1,2.3407888E-1,1.4162177E-1,4.0144295E-2,5.704689E-2,0E0,2.5252316E-2,0E0,8.132444E-2,1.1500034E-1,4.346033E-2,1.20057166E-1,4.1450188E-2,2.128688E-1,3.7035853E-2,0E0,3.2550618E-2,0E0,0E0,1.7009997E-1,2.0726153E-1,1.6466653E-1,1.2832405E-1,1.9666433E-2,0E0,2.1443143E-2,3.086966E-2,0E0,2.5758862E-2,0E0,0E0,5.47702E-2,0E0,1.3124464E-1,0E0,0E0,2.4307435E-2,8.760643E-2,3.219551E-2,1.7457087E-2,0E0,1.7556931E-1,6.8924636E-2,0E0,0E0,2.727621E-2,0E0,2.3811865E-1,7.495177E-2,6.790532E-2,6.303039E-2,7.424271E-2,4.1328244E-2,2.5179472E-2,5.269234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0486768E-2,0E0,5.5518754E-2,1.7593735E-1,0E0,1.848474E-2,5.1533163E-2,3.2213032E-2,0E0,0E0,0E0,0E0,1.650054E-1,1.8818924E-1,3.172031E-2,1.5825287E-2,0E0,0E0,8.370173E-2,0E0,5.5276446E-2,5.345133E-2,4.347816E-2,5.4229297E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,19,19,21,21,22,22,23,23,24,24,25,25,29,29,31,31,32,32,33,33,34,34,35,35,39,39,41,41,43,43,45,45,46,46,48,48,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,68,68,70,70,71,71,72,72,73,73,74,74,75,75,76,76,78,78,81,81,82,82,83,83,84,84,85,85,87,87,88,88,90,90,93,93,95,95,98,98,99,99,100,100,101,101,103,103,104,104,107,107,109,109,110,110,111,111,112,112,113,113,114,114,115,115,116,116,125,125,127,127,128,128,130,130,131,131,132,132,137,137,138,138,139,139,140,140,143,143,145,145,146,146,147,147,148,148],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,46,-1,48,50,52,54,56,-1,-1,-1,58,-1,60,-1,62,-1,64,66,-1,68,-1,-1,-1,-1,-1,-1,70,72,74,76,78,80,-1,82,84,86,88,90,-1,92,-1,94,96,98,100,102,104,106,-1,108,-1,-1,110,112,114,116,118,-1,120,122,-1,124,-1,-1,126,-1,128,-1,-1,130,132,134,136,-1,138,140,-1,-1,142,-1,144,146,148,150,152,154,156,158,-1,-1,-1,-1,-1,-1,-1,-1,160,-1,162,164,-1,166,168,170,-1,-1,-1,-1,172,174,176,178,-1,-1,180,-1,182,184,186,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0976269E5,1.0657745E7,3.812E3,2.9994638E2,3.7798166E0,8.1E2,2.4481E4,7E0,1.176582E6,1.4916515E3,-2.3609966E-2,3.6E1,2.8530578E6,2.018905E-2,8.2471845E6,1.3050649E4,3.0977E4,1.1896653E-2,6.3351805E4,8.579633E-2,-1.6441952E-3,1E0,2.6676828E2,1.4676277E7,2.0833333E0,7.134432E7,1.3505884E-2,9.171345E-3,-2.3857206E-3,1.339646E6,-1.2850202E-2,8.147158E4,3.1095755E0,1.5933E4,2.4203823E0,2.6654E4,-9.737564E-3,-1.92913E-2,-2.050034E-3,1.7903225E0,-8.195852E-3,6E0,9.736071E-3,1.55E2,-5.0577535E-3,1.2E1,2.41E2,-4.7120946E-3,1.6E1,3.9337403E-3,1.2113429E-5,-3.963647E-3,-9.696643E-3,-1.2340919E-3,-6.384194E-3,1.62E2,6.887749E7,6.1114804E-8,8.7115955E2,2.02E4,2.742234E6,-3.98957E-3,8.7E1,2.55E2,7.709291E7,1.4276666E2,3.72E2,-2.8757644E-3,2.670362E7,-4.293659E-3,1.9E1,6.763314E7,2.4915715E-3,1.47176E5,6.6414773E-1,9.751103E6,1.7E1,-1.8382893E-3,2.682E3,2.4348241E-3,8.603761E-3,1.7103828E7,3.8537518E2,2.4E1,2.855464E5,1.746E3,-4.0070787E-3,1.33162E6,1.8E1,-8.288341E-3,6.642857E0,3.6776708E-3,-5.5572303E-4,5.6530495E6,-3.0675915E-3,1E0,-7.771451E-3,4.3633394E-3,3.000176E5,6.083812E2,1.72797E5,2.0211798E6,2.567968E-3,1.5123151E6,2.5417458E1,3.202812E-3,9.513486E-3,6.009E4,-6.550628E-4,1.4916515E3,5.97E3,1.4176E4,6.880842E-1,2.416E3,1E0,1.5E1,1.441215E5,-1.6596582E-2,-9.118207E-3,-3.8823816E-3,-9.356743E-4,3.4212247E-3,-1.5489962E-3,-1.2179071E-3,-5.1467298E-3,3.0751158E4,5.791706E-3,7.117E3,2.019537E2,-3.2254998E-3,3.617263E6,1.411E3,1.9197379E9,1.0810228E-2,3.8300692E-3,-5.5680884E-4,-4.5479448E-3,4.0911578E2,1E0,6.1786E4,5.460753E9,1.4481733E-3,5.27718E-3,4.9833015E1,1.201341E-2,3.1991906E0,9.0754684E1,2.453E4,1.9428571E0,1.2231885E-2,3.469839E-3,-3.802598E-3,-7.929313E-3,2.7879747E-3,-2.9026833E-3,4.999125E-3,2.1167596E-4,-1.7738995E-3,-7.50784E-3,8.6023175E-4,3.6325655E-3,-2.8788622E-3,3.031062E-4,-8.70642E-3,-1.7986735E-3,2.8219619E-3,-2.2161666E-4,7.038843E-4,3.240193E-3,9.038444E-3,3.8805315E-3,-2.0097329E-3,4.4410056E-3,1.2077744E-3,-3.9940276E-3,1.1817028E-3,6.3428674E-3,-1.050409E-3,2.0927086E-3,5.188983E-3,1.0133847E-2,-1.5733469E-3,4.3168785E-3,6.6986037E-3,1.4355014E-3,-2.237358E-3,1.4948554E-3,9.953913E-4,6.7327907E-3],"split_indices":[43,60,2,67,68,2,12,3,5,4,0,3,43,0,43,43,44,0,48,53,0,113,4,62,68,59,0,0,0,9,0,43,53,1,69,9,0,0,0,68,0,8,0,10,0,8,10,0,8,0,0,0,0,0,0,2,7,52,67,12,9,0,8,2,7,67,2,0,5,0,8,7,53,1,53,9,3,0,10,0,0,62,71,3,60,44,0,5,3,0,69,0,0,43,0,26,0,0,43,4,1,43,0,60,73,0,0,1,0,4,2,2,57,9,20,3,43,0,0,0,0,0,0,0,0,48,0,1,4,0,5,44,46,0,0,0,0,4,8,1,46,0,0,73,0,53,71,44,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.004E3,2.77E2,7.27E2,2.37E2,4E1,5.3E2,1.97E2,1.91E2,4.6E1,3.6E1,4E0,2.18E2,3.12E2,6E0,1.91E2,1E1,1.81E2,8E0,3.8E1,3.2E1,4E0,2.09E2,9E0,2.61E2,5.1E1,1.78E2,1.3E1,4E0,6E0,1.74E2,7E0,2.6E1,1.2E1,2.4E1,8E0,2.04E2,5E0,4E0,5E0,2.56E2,5E0,4E1,1.1E1,1.72E2,6E0,1.22E2,5.2E1,1.1E1,1.5E1,8E0,4E0,4E0,2E1,4E0,4E0,6.3E1,1.41E2,1.08E2,1.48E2,2.6E1,1.4E1,5E0,1.67E2,1.01E2,2.1E1,3.1E1,2.1E1,4E0,1.1E1,7E0,5.6E1,1.01E2,4E1,9.5E1,1.3E1,1.38E2,1E1,4E0,2.2E1,7E0,7E0,1.16E2,5.1E1,7.4E1,2.7E1,1.3E1,8E0,2.2E1,9E0,5E0,1.6E1,4E0,7E0,4.9E1,7E0,9.4E1,7E0,4E0,3.6E1,8.6E1,9E0,9E0,4E0,1.12E2,2.6E1,6E0,4E0,1.7E1,5E0,7.9E1,3.7E1,4.2E1,9E0,6.5E1,9E0,9E0,1.8E1,4E0,9E0,9E0,1.3E1,5E0,4E0,1.1E1,5E0,4.3E1,6E0,5.8E1,3.6E1,4E0,3.2E1,7.1E1,1.5E1,5E0,4E0,5E0,4E0,3.2E1,8E1,1.4E1,1.2E1,9E0,8E0,5.5E1,2.4E1,1.2E1,2.5E1,2.8E1,1.4E1,5E0,4E0,1.5E1,5E1,4E0,5E0,5E0,4E0,1.4E1,4E0,3.7E1,6E0,1.5E1,4.3E1,1.1E1,2.5E1,4E0,2.8E1,3E1,4.1E1,5E0,1E1,1.6E1,1.6E1,1.9E1,6.1E1,4E0,1E1,7E0,5E0,4.5E1,1E1,7E0,5E0,1.9E1,6E0,9E0,1.9E1,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[-2.5421267E-3,-4.54704E-2,7.471239E-2,-1.09337635E-1,-1.33620575E-2,1.0278751E-1,-8.404298E-2,-1.0111751E-1,-1.7944142E-2,-8.394736E-3,-2.2760764E-1,1.9427637E-2,9.310652E-2,-1.9356988E-2,-5.022023E-2,-8.435361E-2,-2.1126117E-1,-1.332153E-2,1.1324026E-1,-1.1398182E-3,-1.966273E-2,2.6305959E-2,1.217016E-1,-7.619313E-2,3.8039568E-3,-1.5917991E-1,-5.500472E-2,-2.5255677E-1,-3.532036E-3,-5.942694E-2,3.542078E-3,1.3865676E-2,9.241316E-3,-6.367484E-3,1.1543242E-1,1.5685134E-1,5.840395E-2,-2.7696842E-2,-1.5158741E-1,-7.2341606E-2,-1.9477642E-1,7.969432E-2,-7.33234E-2,-1.9888097E-1,-1.6335959E-2,-4.9246356E-2,-1.2355577E-2,1.2362886E-2,-8.839086E-2,2.4679608E-3,-1.3149148E-3,1.1127053E-2,-5.1141405E-3,2.3229497E-2,8.101115E-3,1.3353586E-1,1.3239269E-2,7.806513E-2,-8.309853E-2,-6.772005E-2,4.7144773E-3,-9.600239E-3,-6.4190663E-4,7.998607E-4,-1.1114062E-1,-1.215757E-2,-1.6683866E-1,8.65128E-3,-6.619609E-4,-4.2575836E-2,-9.408454E-2,-4.4017024E-3,-1.0416772E-2,-2.1093674E-2,-9.473748E-2,1.7409757E-2,-5.095121E-2,2.4142342E-3,-1.1512344E-1,-5.0133485E-2,3.266131E-2,3.584761E-3,-2.6576682E-3,1.8481413E-1,8.088663E-2,4.8281103E-2,1.9162512E-1,-8.544575E-3,2.5753442E-3,1.9589511E-3,-1.02603264E-1,-1.3831886E-3,-6.837394E-3,-8.4553035E-3,-2.4286485E-3,-2.4226228E-2,-1.0709931E-1,-7.313269E-2,-1.5756802E-1,-6.6714636E-3,-5.3081783E-3,-3.9686467E-2,-1.4527717E-1,7.143194E-3,4.8257496E-2,-9.067044E-2,1.1515733E-3,-2.9145833E-4,-7.0757326E-3,5.572348E-4,-6.090037E-3,8.611992E-2,1.2832938E-2,2.6268777E-1,1.3486584E-1,-7.571906E-3,1.06119685E-1,-3.2952398E-2,6.665692E-2,5.6689526E-3,1.2153334E-2,2.3366521E-4,-1.6178727E-1,-4.07812E-2,2.2711768E-3,-7.0660263E-3,-9.92913E-4,-1.2123521E-1,-4.080731E-2,-8.279315E-3,-3.224055E-3,-5.5296108E-2,2.0928886E-2,9.3635026E-4,-6.501933E-2,-1.8784513E-1,-3.487928E-4,-4.1923635E-2,1.800284E-2,8.1211045E-2,7.104659E-3,-5.814261E-3,-1.7329064E-3,1.084086E-3,6.6925497E-3,2.4810946E-3,-1.8613518E-4,1.3753464E-2,5.530243E-3,-1.822666E-3,1.5653424E-1,1.287163E-1,3.0749848E-2,3.8311942E-4,-3.88558E-3,1.2206044E-1,1.0784502E-2,-3.452554E-3,-1.1610145E-2,2.3218493E-3,-2.5222346E-3,-6.790782E-3,-1.2962726E-3,-5.549409E-3,-1.2334522E-3,-5.8116326E-3,-1.2548242E-3,-2.4443984E-3,2.2274922E-3,-1.0832128E-3,-4.5489715E-3,-1.0776526E-2,-3.3850789E-3,2.2743465E-3,-3.4247932E-3,1.0696541E-3,-4.5897663E-3,2.7066993E-3,7.071643E-3,-1.6484723E-3,2.4148275E-3,9.705331E-3,3.842353E-3,4.059716E-3,8.01699E-3,4.703545E-3,-9.25853E-4,7.4518058E-3,3.941479E-3,-1.1742665E-3,4.7277724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,29,31,-1,-1,33,35,37,-1,39,41,43,-1,45,47,49,-1,51,53,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,-1,-1,79,-1,81,-1,83,-1,85,87,89,-1,-1,-1,-1,91,-1,93,-1,-1,95,97,-1,-1,99,101,103,105,-1,107,109,111,-1,-1,113,115,117,119,-1,-1,-1,121,-1,-1,-1,-1,123,125,127,129,131,-1,133,135,137,139,141,-1,-1,-1,-1,-1,143,145,147,149,-1,151,153,155,-1,-1,-1,157,159,-1,-1,-1,161,163,-1,-1,165,167,-1,169,171,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,183,185,-1,-1,187,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.425925E0,1.3601847E0,1.6532998E0,4.699943E-1,4.6954185E-1,9.2670035E-1,6.133747E-1,3.8285613E-1,0E0,2.61592E-1,3.9131492E-1,0E0,5.8130956E-1,0E0,1.853061E-1,4.0849745E-1,1.3656747E-1,3.25207E-1,1.4385058E-1,0E0,0E0,2.7155736E-1,4.6696687E-1,1.5668058E-1,0E0,1.4973998E-1,3.4635958E-1,4.357028E-2,0E0,2.2401384E-1,2.5004607E-1,1.6588666E-2,0E0,1.2717147E-1,1.3404658E-1,3.7608337E-1,2.2349128E-1,1.523081E-1,1.2869799E-1,6.1059132E-2,2.6779413E-2,1.7461634E-1,7.387358E-2,2.2602916E-2,0E0,1.363169E-1,0E0,9.067723E-2,1.0753298E-1,0E0,0E0,8.078943E-2,0E0,5.2756384E-2,0E0,3.0362272E-1,0E0,2.2412497E-1,1.5172553E-1,8.8862E-2,0E0,0E0,0E0,0E0,3.1483725E-2,0E0,4.3997407E-2,0E0,0E0,5.77884E-2,8.4243536E-2,0E0,0E0,9.122833E-2,1.0830721E-1,8.231406E-2,6.5549836E-2,0E0,9.422311E-2,8.287967E-2,4.6594705E-2,0E0,0E0,1.9406223E-1,3.8799167E-1,8.578949E-2,3.664857E-2,0E0,0E0,0E0,1.09139115E-1,0E0,0E0,0E0,0E0,5.1816482E-2,3.9887086E-2,8.212891E-2,1.9789249E-2,8.1680715E-2,0E0,3.321284E-2,1.2170005E-1,1.0572588E-1,8.757159E-2,2.146358E-2,0E0,0E0,0E0,0E0,0E0,4.0767103E-2,2.3648227E-2,7.4499846E-2,1.4970922E-1,0E0,8.971262E-2,2.3905348E-2,1.41078E-1,0E0,0E0,0E0,6.694397E-2,4.5501724E-2,0E0,0E0,0E0,4.8669428E-2,3.595761E-2,0E0,0E0,4.0220015E-2,8.1127815E-2,0E0,1.8547483E-2,6.6167295E-2,0E0,1.09338894E-1,1.00211866E-1,4.9555868E-2,5.9839338E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1608368E-1,6.1625123E-2,5.256903E-2,0E0,0E0,1.939997E-2,8.214453E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,51,51,53,53,55,55,57,57,58,58,59,59,64,64,66,66,69,69,70,70,73,73,74,74,75,75,76,76,78,78,79,79,80,80,83,83,84,84,85,85,86,86,90,90,95,95,96,96,97,97,98,98,99,99,101,101,102,102,103,103,104,104,105,105,111,111,112,112,113,113,114,114,116,116,117,117,118,118,122,122,123,123,127,127,128,128,131,131,132,132,134,134,135,135,137,137,138,138,139,139,140,140,150,150,151,151,152,152,155,155,156,156],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,30,32,-1,-1,34,36,38,-1,40,42,44,-1,46,48,50,-1,52,54,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,-1,-1,80,-1,82,-1,84,-1,86,88,90,-1,-1,-1,-1,92,-1,94,-1,-1,96,98,-1,-1,100,102,104,106,-1,108,110,112,-1,-1,114,116,118,120,-1,-1,-1,122,-1,-1,-1,-1,124,126,128,130,132,-1,134,136,138,140,142,-1,-1,-1,-1,-1,144,146,148,150,-1,152,154,156,-1,-1,-1,158,160,-1,-1,-1,162,164,-1,-1,166,168,-1,170,172,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,184,186,-1,-1,188,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.8988005E5,1E0,6.9934395E-5,6.0692043E0,3.3817584E7,7.87E3,1.977157E7,-1.7944142E-2,7.82261E6,1.3586957E1,1.9427637E-2,5.817547E2,-1.9356988E-2,3.2462872E5,4.9E2,3.1126543E1,2.1222334E2,1.9186046E0,-1.1398182E-3,-1.966273E-2,7.127857E6,5.489183E1,4.01969E5,3.8039568E-3,5.3E1,9.83871E-1,3.2E1,-3.532036E-3,3.4E1,1.465E4,5.51E2,9.241316E-3,1.8E1,1E0,1.4176E4,3.206931E2,4.27E3,1.3026532E6,1E0,1.7857143E1,1.32E2,1.0142858E1,7.0093E4,-1.6335959E-2,1E0,-1.2355577E-2,2E0,2.5818555E2,2.4679608E-3,-1.3149148E-3,5.9754E4,-5.1141405E-3,1.1544592E1,8.101115E-3,3.72381E5,1.3239269E-2,3.8537518E2,6.9879E4,1E0,4.7144773E-3,-9.600239E-3,-6.4190663E-4,7.998607E-4,2.3E1,-1.215757E-2,1.76421E6,8.65128E-3,-6.619609E-4,2.0303884E6,3.1316226E0,-4.4017024E-3,-1.0416772E-2,1E0,1.703125E0,4.5432812E2,9E0,2.4142342E-3,1.9E1,1.136E3,1.1283241E6,3.584761E-3,-2.6576682E-3,2.124E3,6.27907E0,3.808001E5,4.43837E0,-8.544575E-3,2.5753442E-3,1.9589511E-3,2.511E3,-1.3831886E-3,-6.837394E-3,-8.4553035E-3,-2.4286485E-3,1.6580646E2,9.14E2,3.95E2,2.897656E8,8.038E3,-5.3081783E-3,8.454545E0,9.5E2,3.990487E5,2E0,7.89E2,1.1515733E-3,-2.9145833E-4,-7.0757326E-3,5.572348E-4,-6.090037E-3,5.9653606E5,9.47E4,1.3587301E1,2.9355192E1,-7.571906E-3,2.3922667E1,7.25723E2,8.710612E1,5.6689526E-3,1.2153334E-2,2.3366521E-4,1.3711089E6,2.6412E5,2.2711768E-3,-7.0660263E-3,-9.92913E-4,2.8461537E0,7.7E1,-8.279315E-3,-3.224055E-3,1.3E1,1.3020051E-5,9.3635026E-4,2.9016996E4,1.8149019E2,-3.487928E-4,1.5132743E0,1E0,2.0277777E0,1.404E3,-5.814261E-3,-1.7329064E-3,1.084086E-3,6.6925497E-3,2.4810946E-3,-1.8613518E-4,1.3753464E-2,5.530243E-3,-1.822666E-3,6E0,1.2E1,2.83E3,3.8311942E-4,-3.88558E-3,1.03339244E6,3.002566E0,-3.452554E-3,-1.1610145E-2,2.3218493E-3,-2.5222346E-3,-6.790782E-3,-1.2962726E-3,-5.549409E-3,-1.2334522E-3,-5.8116326E-3,-1.2548242E-3,-2.4443984E-3,2.2274922E-3,-1.0832128E-3,-4.5489715E-3,-1.0776526E-2,-3.3850789E-3,2.2743465E-3,-3.4247932E-3,1.0696541E-3,-4.5897663E-3,2.7066993E-3,7.071643E-3,-1.6484723E-3,2.4148275E-3,9.705331E-3,3.842353E-3,4.059716E-3,8.01699E-3,4.703545E-3,-9.25853E-4,7.4518058E-3,3.941479E-3,-1.1742665E-3,4.7277724E-3],"split_indices":[2,43,6,52,68,7,9,5,0,43,73,0,67,0,48,1,73,4,69,0,0,60,71,1,0,44,68,3,0,3,44,2,0,3,23,2,73,2,43,24,4,12,73,1,0,26,0,6,4,0,0,1,0,71,0,9,0,71,2,8,0,0,0,0,44,0,46,0,0,60,68,0,0,19,68,4,8,0,8,44,60,0,0,44,71,43,50,0,0,0,2,0,0,0,0,67,2,2,46,1,0,73,10,43,32,2,0,0,0,0,0,60,9,71,73,0,71,67,73,0,0,0,43,7,0,0,0,68,10,0,0,3,53,0,48,70,0,68,79,68,2,0,0,0,0,0,0,0,0,0,8,8,2,0,0,43,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.031E3,6.63E2,3.68E2,2.21E2,4.42E2,3.13E2,5.5E1,2.16E2,5E0,4.33E2,9E0,8E0,3.05E2,4E0,5.1E1,1.89E2,2.7E1,4.17E2,1.6E1,5E0,4E0,9.2E1,2.13E2,4.3E1,8E0,5.2E1,1.37E2,2E1,7E0,1.11E2,3.06E2,8E0,8E0,6.8E1,2.4E1,1.36E2,7.7E1,2.7E1,1.6E1,1.6E1,3.6E1,1.6E1,1.21E2,1.5E1,5E0,1.07E2,4E0,2.8E2,2.6E1,4E0,4E0,5.9E1,9E0,1E1,1.4E1,1.17E2,1.9E1,6.8E1,9E0,2.1E1,6E0,1.1E1,5E0,5E0,1.1E1,8E0,2.8E1,7E0,9E0,5E1,7.1E1,4E0,1.1E1,6.7E1,4E1,2.6E2,2E1,4E0,2.2E1,1.5E1,4.4E1,6E0,4E0,5.8E1,5.9E1,5.5E1,1.3E1,5E0,4E0,5E0,1.6E1,4E0,7E0,2.4E1,4E0,4E1,1E1,5.5E1,1.6E1,5.9E1,8E0,2E1,2E1,1.96E2,6.4E1,1.3E1,7E0,6E0,1.6E1,9E0,6E0,1.1E1,3.3E1,2.1E1,3.7E1,5E0,5.4E1,1E1,4.5E1,8E0,5E0,6E0,1E1,3.3E1,7E0,6E0,4E0,2.1E1,3.4E1,1.2E1,4E0,2.1E1,3.8E1,6E0,1.4E1,1.5E1,5E0,3.5E1,1.61E2,3.5E1,2.9E1,7E0,6E0,6E0,5E0,9E0,2.4E1,1.6E1,5E0,4E0,3.3E1,4.1E1,1.3E1,6E0,4E0,2.2E1,2.3E1,6E0,4E0,4E0,2.9E1,1.6E1,5E0,4E0,3E1,5E0,1.6E1,1E1,2.8E1,7E0,7E0,1E1,5E0,9E0,2.6E1,1.55E2,6E0,2.8E1,7E0,1.5E1,1.4E1,1.8E1,1.5E1,2.3E1,1.8E1,5E0,8E0,9E0,1.3E1,1.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"191","size_leaf_vector":"1"}},{"base_weights":[-1.6768986E-3,-3.4966692E-2,9.1516346E-2,-6.739187E-2,1.5456984E-2,1.0763499E-1,-1.07800454E-1,-5.2370828E-2,-1.547599E-1,1.9478336E-2,-1.0464207E-2,2.0494487E-2,9.771774E-2,-8.86112E-3,3.7827522E-3,-1.3803354E-1,-3.8602564E-2,-9.705497E-2,-3.9143035E-1,7.961811E-3,9.7763024E-2,7.964134E-2,2.091503E-1,3.285471E-3,-3.5930462E-3,-7.501039E-2,-2.0544358E-1,1.0062904E-1,-4.6550117E-2,-1.309202E-1,4.237979E-3,-2.515075E-2,-1.7462966E-3,4.1632973E-2,-1.2664159E-2,-2.2505E-3,1.1642825E-1,5.4190245E-2,1.388433E-1,2.5647342E-1,1.0101846E-1,-6.518899E-3,-4.8759773E-2,-3.2511433E-3,-1.0478092E-2,1.1660193E-2,4.3333333E-2,-3.5339423E-2,-1.3117544E-1,-9.181514E-2,-8.221557E-3,-2.4021226E-3,2.194635E-3,1.4204666E-2,7.7023216E-2,8.0237497E-4,-1.4468968E-1,1.4548971E-1,1.7520034E-4,1.1522659E-2,4.4459067E-2,2.0912269E-1,7.068141E-2,1.3075406E-2,4.8180684E-3,2.2034561E-3,6.7632147E-3,6.2657185E-5,-7.07703E-2,-2.8989706E-3,4.047658E-3,-3.1303614E-2,-1.2008373E-2,-1.7737614E-1,2.968251E-3,4.992811E-4,-1.1801108E-1,2.9725919E-2,-5.6677135E-3,8.975173E-3,4.8828438E-2,-2.679227E-2,3.845852E-2,-1.4495105E-2,2.8808778E-3,1.6173092E-1,1.9256801E-3,2.2927333E-2,1.0094499E-1,2.3838468E-1,4.397981E-3,-3.2774562E-3,1.03562936E-1,-6.0479003E-3,-1.4568117E-3,-6.277974E-2,-4.0931082E-3,-2.3537235E-1,-5.9300605E-3,-6.481607E-3,-2.6139272E-3,5.352191E-2,-2.241417E-2,2.4598083E-2,5.0526774E-3,-1.4538056E-1,-1.2765515E-2,5.607442E-3,1.2290839E-2,-2.8369317E-3,3.0765005E-3,3.3858994E-3,8.153802E-3,-1.362983E-1,3.8940895E-2,1.2367919E-1,-1.104112E-3,6.4709918E-3,2.706289E-1,2.3289045E-2,7.1208873E-3,-1.2806566E-2,-8.651352E-2,-2.080394E-2,3.853658E-2,-4.4824574E-3,-1.3216604E-2,2.5242904E-2,1.0797634E-1,3.2110284E-3,-6.6505514E-2,-1.4431196E-3,4.570221E-2,-1.8732371E-3,-1.0216361E-2,1.2648586E-2,-4.126894E-2,-2.6493251E-2,6.1093733E-2,3.4651808E-3,-1.3382404E-2,8.724209E-3,7.902209E-2,1.6011976E-1,6.684649E-2,7.4967253E-3,1.4551862E-2,-1.36519E-3,3.0146171E-3,1.2837243E-3,-2.0682854E-3,-5.330412E-3,-2.7189185E-3,8.006539E-4,-1.9510069E-3,3.610932E-3,-8.1835163E-4,4.666905E-4,3.303524E-3,2.0861109E-3,7.5430125E-3,-5.617559E-3,4.0883533E-4,4.1358406E-3,7.0317305E-4,2.858946E-3,-1.2262574E-4,-2.4527307E-3,1.3239514E-3,7.4568996E-4,-5.464974E-3,4.5384523E-3,-2.5098824E-3,2.7692802E-3,-1.8188662E-3,8.140432E-3,1.7950365E-3,8.773918E-3,2.7231404E-3,6.678367E-3,-4.7211503E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,33,35,37,39,-1,-1,41,43,45,47,49,51,-1,-1,53,55,-1,57,59,61,63,65,-1,67,-1,-1,-1,69,71,73,75,-1,-1,-1,77,79,81,83,85,-1,-1,87,89,91,-1,-1,-1,-1,-1,93,-1,-1,95,-1,97,-1,-1,99,101,-1,-1,103,105,107,-1,109,111,-1,113,115,117,-1,-1,119,-1,-1,121,123,125,-1,-1,-1,127,129,131,-1,133,135,-1,137,-1,-1,-1,-1,139,141,143,-1,-1,145,147,-1,149,151,153,155,-1,-1,157,159,-1,161,-1,163,-1,-1,165,167,169,171,-1,-1,173,175,177,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.229679E0,1.2550843E0,8.9188766E-1,6.0476685E-1,2.972863E-1,8.0297303E-1,1.975645E-1,4.6644235E-1,8.9068925E-1,2.669872E-1,0E0,0E0,4.828894E-1,0E0,6.044476E-2,2.1825123E-1,3.8755083E-1,1.9265217E-1,6.826532E-1,1.8109734E-1,1.0956398E-1,3.164283E-1,1.4807415E-1,0E0,0E0,4.5170188E-2,5.9051156E-2,1.5427318E-1,3.0654085E-1,6.091845E-2,3.8964592E-2,0E0,0E0,9.500469E-2,2.8814012E-1,0E0,1.1074269E-1,2.7767336E-1,2.91803E-1,6.3414335E-2,2.2052176E-2,0E0,2.5466818E-2,0E0,0E0,0E0,7.595815E-2,2.5562325E-1,3.544889E-1,6.848256E-2,0E0,0E0,0E0,1.2475047E-1,1.3266331E-1,1.5481836E-1,3.7513608E-1,3.8779497E-2,0E0,0E0,1.7525658E-1,8.065164E-2,1.6567805E-1,0E0,0E0,0E0,0E0,0E0,3.36269E-2,0E0,0E0,2.4555314E-1,0E0,5.791509E-2,0E0,0E0,1.9443631E-2,6.5717354E-2,0E0,0E0,5.0117612E-2,1.4119728E-1,1.3520245E-1,0E0,4.095189E-2,1.7093003E-2,0E0,2.7813292E-1,1.1648506E-1,3.817749E-2,0E0,0E0,1.1007944E-1,0E0,0E0,1.5663701E-1,1.1122024E-1,6.344092E-2,0E0,0E0,0E0,5.2711368E-2,7.35201E-2,3.3243068E-2,0E0,6.2510654E-2,5.7024326E-2,0E0,9.436458E-2,0E0,0E0,0E0,0E0,3.4580347E-1,1.1800045E-1,5.971104E-2,0E0,0E0,3.0590415E-2,2.8536215E-2,0E0,5.8332376E-2,6.5071344E-2,9.224582E-2,9.945354E-2,0E0,0E0,1.678972E-2,3.431256E-2,0E0,5.089094E-2,0E0,2.5351577E-2,0E0,0E0,3.1808384E-2,3.2107137E-2,1.1149431E-1,9.96535E-2,0E0,0E0,1.4276846E-1,1.5710282E-1,4.9098134E-2,7.9258405E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,38,38,39,39,40,40,42,42,46,46,47,47,48,48,49,49,53,53,54,54,55,55,56,56,57,57,60,60,61,61,62,62,68,68,71,71,73,73,76,76,77,77,80,80,81,81,82,82,84,84,85,85,87,87,88,88,89,89,92,92,95,95,96,96,97,97,101,101,102,102,103,103,105,105,106,106,108,108,113,113,114,114,115,115,118,118,119,119,121,121,122,122,123,123,124,124,127,127,128,128,130,130,132,132,135,135,136,136,137,137,138,138,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,34,36,38,40,-1,-1,42,44,46,48,50,52,-1,-1,54,56,-1,58,60,62,64,66,-1,68,-1,-1,-1,70,72,74,76,-1,-1,-1,78,80,82,84,86,-1,-1,88,90,92,-1,-1,-1,-1,-1,94,-1,-1,96,-1,98,-1,-1,100,102,-1,-1,104,106,108,-1,110,112,-1,114,116,118,-1,-1,120,-1,-1,122,124,126,-1,-1,-1,128,130,132,-1,134,136,-1,138,-1,-1,-1,-1,140,142,144,-1,-1,146,148,-1,150,152,154,156,-1,-1,158,160,-1,162,-1,164,-1,-1,166,168,170,172,-1,-1,174,176,178,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.785302E5,3.5056704E7,1E0,4.4E1,3.3817584E7,1.3655363E2,3.89E2,9.639872E0,5.817547E2,-1.0464207E-2,2.0494487E-2,6.5590655E6,-8.86112E-3,2.342E4,6.7E1,6E0,5.7E1,1.753E3,2.308943E0,1.2895454E3,1.5142261E3,1.8029197E0,3.285471E-3,-3.5930462E-3,8.8324326E-1,1.3E1,1.89991E5,2.0734E4,5.6E1,5E0,-2.515075E-2,-1.7462966E-3,1.047E3,2.954124E6,-2.2505E-3,2.0319266E-7,5.6E1,8.710612E1,1E0,5.941442E6,-6.518899E-3,3.774648E0,-3.2511433E-3,-1.0478092E-2,1.1660193E-2,1.6595395E5,4.6E1,4.5179688E1,1.2E1,-8.221557E-3,-2.4021226E-3,2.194635E-3,1.683263E7,6.923077E-1,6.848509E7,4.1957852E2,3.361317E1,1.7520034E-4,1.1522659E-2,2.9251662E6,3.1095755E0,6.516E3,1.3075406E-2,4.8180684E-3,2.2034561E-3,6.7632147E-3,6.2657185E-5,2.8E1,-2.8989706E-3,4.047658E-3,1.5292561E2,-1.2008373E-2,2.0421052E0,2.968251E-3,4.992811E-4,3.3E1,6E0,-5.6677135E-3,8.975173E-3,1.28738E5,1.901875E2,3.168978E-2,-1.4495105E-2,1.535E3,1.8425926E0,1.9256801E-3,1.4598765E0,1E0,1.990351E1,4.397981E-3,-3.2774562E-3,1.270394E5,-6.0479003E-3,-1.4568117E-3,4.577342E0,7.09617E6,1.0609756E0,-5.9300605E-3,-6.481607E-3,-2.6139272E-3,2.6676828E2,3.5714287E-1,3.1096E4,5.0526774E-3,1.3938298E2,1E0,5.607442E-3,1.2768175E6,-2.8369317E-3,3.0765005E-3,3.3858994E-3,8.153802E-3,1.38132E5,1.0901037E10,1.500502E6,-1.104112E-3,6.4709918E-3,1.670046E1,2.337765E3,7.1208873E-3,1.31E2,2.511352E6,8.454545E0,2.5417458E1,-4.4824574E-3,-1.3216604E-2,4.28649E5,1.5E1,3.2110284E-3,8.1E2,-1.4431196E-3,2.3773398E3,-1.8732371E-3,-1.0216361E-2,3.491E3,1.9E1,1.1E1,1.990351E1,3.4651808E-3,-1.3382404E-2,1.0717949E1,5.8475E5,4.5866325E6,1.552356E0,7.4967253E-3,1.4551862E-2,-1.36519E-3,3.0146171E-3,1.2837243E-3,-2.0682854E-3,-5.330412E-3,-2.7189185E-3,8.006539E-4,-1.9510069E-3,3.610932E-3,-8.1835163E-4,4.666905E-4,3.303524E-3,2.0861109E-3,7.5430125E-3,-5.617559E-3,4.0883533E-4,4.1358406E-3,7.0317305E-4,2.858946E-3,-1.2262574E-4,-2.4527307E-3,1.3239514E-3,7.4568996E-4,-5.464974E-3,4.5384523E-3,-2.5098824E-3,2.7692802E-3,-1.8188662E-3,8.140432E-3,1.7950365E-3,8.773918E-3,2.7231404E-3,6.678367E-3,-4.7211503E-5],"split_indices":[2,43,60,6,3,7,71,1,69,67,0,0,43,0,9,44,3,6,44,69,48,67,69,0,0,68,5,9,44,0,8,0,0,2,62,0,52,0,73,19,60,0,69,0,0,0,43,3,71,3,0,0,0,60,68,7,4,71,0,0,43,53,44,0,0,0,0,0,2,0,0,67,0,68,0,0,3,8,0,0,1,4,53,0,2,69,0,69,6,73,0,0,48,0,0,73,12,68,0,0,0,4,68,9,0,4,26,0,62,0,0,0,0,9,46,9,0,0,71,67,0,44,5,73,73,0,0,9,3,0,2,0,48,0,0,12,3,8,73,0,0,71,1,43,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,7.66E2,2.73E2,4.66E2,3E2,2.53E2,2E1,3.99E2,6.7E1,2.96E2,4E0,6E0,2.47E2,1.1E1,9E0,5.4E1,3.45E2,5.5E1,1.2E1,2.59E2,3.7E1,2.14E2,3.3E1,5E0,4E0,2.9E1,2.5E1,1.8E1,3.27E2,4.1E1,1.4E1,8E0,4E0,9.8E1,1.61E2,4E0,3.3E1,1.51E2,6.3E1,2.2E1,1.1E1,7E0,2.2E1,4E0,2.1E1,4E0,1.4E1,2.9E2,3.7E1,2.4E1,1.7E1,6E0,8E0,5.6E1,4.2E1,1.47E2,1.4E1,2.6E1,7E0,6E0,1.45E2,3E1,3.3E1,1.8E1,4E0,6E0,5E0,7E0,1.5E1,4E0,1E1,2.86E2,4E0,3E1,7E0,5E0,1.9E1,5.1E1,5E0,7E0,3.5E1,8.5E1,6.2E1,6E0,8E0,2.2E1,4E0,1.06E2,3.9E1,2.3E1,7E0,6E0,2.7E1,5E0,1E1,1.32E2,1.54E2,1.2E1,1.8E1,1.3E1,6E0,3.5E1,1.6E1,2.6E1,9E0,8E0,7.7E1,1.4E1,4.8E1,4E0,4E0,4E0,1.8E1,9E0,9.7E1,3.3E1,6E0,7E0,1.6E1,1.1E1,1.6E1,4.3E1,8.9E1,1.11E2,4.3E1,4E0,8E0,2.4E1,1.1E1,5E0,1.1E1,7E0,1.9E1,4E0,4E0,4.1E1,3.6E1,2.7E1,2.1E1,4E0,5E0,5.6E1,4.1E1,1.9E1,1.4E1,6E0,1E1,5E0,6E0,1.9E1,2.4E1,4.2E1,4.7E1,4E1,7.1E1,2.5E1,1.8E1,1.9E1,5E0,6E0,5E0,6E0,5E0,7E0,1.2E1,9E0,3.2E1,3.1E1,5E0,1.9E1,8E0,1.6E1,5E0,2.7E1,2.9E1,1.1E1,3E1,1.4E1,5E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"181","size_leaf_vector":"1"}},{"base_weights":[-2.3257365E-3,-9.361101E-2,2.9138543E-2,-7.504461E-2,-2.1700005E-1,-4.3321406E-3,1.011941E-1,-1.1467243E-1,-3.931716E-2,-1.2133072E-1,-2.167969E-2,-3.6855094E-2,2.0264916E-2,1.1289062E-1,-7.688294E-4,-1.4771907E-1,-7.8278266E-2,1.1763657E-2,-6.197982E-2,-1.60153E-1,-4.319978E-2,-3.0141326E-2,-1.5133677E-2,2.4603236E-2,-9.040706E-3,1.0391565E-1,1.1384624E-2,1.3134174E-1,-1.0640603E-1,-1.0529617E-1,-1.7624575E-1,1.9840815E-3,-9.1524586E-2,-3.8676467E-2,-1.3649903E-1,-7.770432E-2,-1.232758E-2,2.9120783E-4,-3.568988E-3,-4.25794E-2,2.9313734E-2,6.865808E-2,6.948213E-3,6.336656E-2,1.3131529E-1,1.2039306E-2,-1.9101286E-3,-9.710885E-4,-9.285439E-3,-1.8202295E-3,-1.287337E-1,-2.0858306E-1,-3.2538017E-3,-1.2343109E-1,-5.3145684E-2,-1.4649846E-2,-9.6331365E-2,-8.9190565E-3,-3.6211764E-3,-7.5402656E-3,2.6586677E-3,-3.4477558E-2,-1.9628441E-1,6.643207E-2,-8.497266E-2,3.4993973E-2,9.441506E-2,-2.985897E-2,4.1015346E-2,7.423596E-2,-4.2558517E-3,1.493828E-1,2.5508147E-2,-7.6652523E-3,-3.3778017E-3,-2.4586946E-1,-2.575395E-3,-1.5095799E-1,-5.4662734E-2,-1.7122427E-2,-5.6848805E-3,2.5977049E-2,-3.9675407E-2,-7.3686386E-3,-9.345938E-3,-4.9747836E-2,9.4321783E-4,-3.1843367E-3,-1.313834E-2,4.1217147E-3,8.762426E-4,1.9442398E-3,-8.16296E-3,9.34223E-2,8.7435124E-4,8.1993686E-4,1.0738812E-1,-1.584113E-2,-1.3796739E-2,9.281836E-2,-2.1185244E-3,2.3200482E-3,9.659797E-2,1.7556436E-1,8.20661E-2,9.7870804E-2,-3.9834836E-3,-1.3162144E-2,-7.2342553E-3,-8.248221E-3,-2.011797E-3,4.8022768E-5,-4.593682E-3,6.649162E-4,-3.5711122E-3,-1.5925014E-2,6.588088E-2,-9.046201E-3,-9.591206E-2,2.3565071E-3,-3.6128052E-3,-5.725016E-2,4.075345E-2,1.3056016E-2,-3.7999048E-3,6.9344183E-4,6.946276E-3,-4.660225E-3,3.1354763E-2,2.837775E-2,1.262539E-1,3.9971434E-2,-4.8893124E-2,1.17042884E-1,-7.2920835E-4,-5.25034E-2,3.1982567E-2,3.6384843E-3,-2.1711405E-2,1.451414E-1,5.885555E-2,1.4906694E-1,1.3861813E-2,5.108682E-2,1.0761651E-2,9.2859525E-4,6.2019355E-3,-2.261614E-3,2.1580318E-3,4.2798687E-3,2.0968488E-4,-2.5722475E-3,5.3882714E-2,-6.505277E-3,-5.8119214E-4,-2.8470319E-2,-7.3642984E-2,3.9775637E-3,-1.766854E-4,-1.1563249E-2,6.301725E-2,5.4900598E-2,-2.3220961E-3,-8.796674E-4,3.239513E-3,1.4647314E-1,3.1010772E-3,1.08308464E-1,6.893903E-3,-1.3479684E-1,-1.6083207E-2,6.465944E-2,2.0353024E-1,4.8096015E-4,-8.272238E-2,9.167583E-2,-2.3340499E-2,-2.5334295E-3,4.6874394E-4,1.6548668E-1,2.2980636E-3,-1.4143132E-3,8.551146E-2,1.2755826E-2,1.2556845E-1,7.33895E-2,-2.8515377E-3,1.741479E-4,4.775488E-3,-4.848594E-4,-5.516882E-3,-7.39282E-3,-2.9940906E-3,-1.0144135E-3,2.3578054E-3,3.9681285E-3,4.8866356E-4,3.6606938E-3,-6.693271E-4,2.9359742E-3,7.4189096E-3,6.2821656E-3,1.9575302E-3,1.088083E-3,-2.4281596E-3,-2.3489571E-3,-1.0297211E-2,9.076303E-4,-4.2149E-3,5.7357843E-3,1.0329944E-3,1.2627515E-2,5.8501055E-3,-1.3251698E-3,-6.4646406E-3,1.3527403E-3,5.563906E-3,-3.0664308E-3,1.216456E-3,5.0566373E-3,9.165458E-3,6.077624E-3,2.2166981E-3,1.0801344E-2,5.1971795E-3,4.294069E-3,7.1184384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,35,37,39,-1,41,-1,43,-1,45,47,49,51,-1,53,55,57,59,-1,-1,-1,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,-1,77,79,81,83,-1,-1,-1,-1,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,-1,109,111,113,-1,115,117,-1,119,121,123,-1,-1,-1,-1,-1,-1,125,127,-1,129,131,-1,133,135,137,139,141,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,153,-1,-1,155,157,159,-1,-1,-1,-1,161,163,165,167,169,171,-1,173,175,-1,177,179,181,183,-1,185,-1,-1,-1,-1,-1,-1,-1,-1,187,-1,-1,189,191,-1,-1,193,195,197,-1,-1,-1,199,-1,201,203,205,207,209,211,-1,213,215,217,-1,-1,219,-1,-1,221,-1,223,225,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9582477E0,5.885105E-1,1.8478379E0,3.2254136E-1,7.6362646E-1,4.199306E-1,2.8989124E-1,1.1953163E-1,8.351613E-1,7.194519E-2,0E0,4.3123996E-1,2.849426E-1,2.3565936E-1,3.768261E-1,4.9212337E-2,9.064275E-2,0E0,1.9442368E-1,1.287958E-1,1.734406E-2,1.6541861E-1,0E0,2.28193E-1,0E0,2.2055125E-1,0E0,2.7862227E-1,1.1691502E-1,3.3539206E-2,9.480685E-2,0E0,5.344248E-2,1.21876225E-1,7.3616385E-2,1.3660975E-1,0E0,0E0,0E0,2.2236672E-1,1.7108765E-1,6.953129E-2,2.6580077E-1,1.5049243E-1,2.2941351E-1,0E0,0E0,0E0,0E0,0E0,2.5096208E-2,1.2306404E-1,0E0,4.0965676E-2,5.8331028E-2,6.610015E-2,1.4136745E-1,0E0,0E0,0E0,0E0,9.5242694E-2,8.138168E-2,3.0751914E-2,1.2397021E-1,7.56733E-2,4.4840574E-2,3.7957314E-1,2.4598771E-1,1.2843063E-1,0E0,1.6888762E-1,1.6205882E-1,0E0,0E0,2.1597743E-2,0E0,4.5546174E-2,2.2547279E-2,3.3351928E-2,0E0,4.2850472E-2,6.8031736E-2,0E0,5.3856578E-2,8.584648E-2,5.5490278E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.0242854E-2,8.069519E-2,0E0,5.6040943E-2,1.8273656E-1,0E0,1.333895E-1,1.0648812E-1,3.844759E-2,1.03218555E-1,1.9935656E-1,1.3299358E-1,3.1246841E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9181164E-2,2.091834E-2,8.223039E-2,5.0608307E-2,0E0,0E0,5.1686585E-2,2.054212E-2,5.9769113E-2,0E0,0E0,0E0,0E0,4.119456E-2,1.9149698E-2,2.8425872E-2,8.288889E-2,1.7257862E-1,1.7259061E-1,0E0,4.839112E-2,1.2432909E-1,0E0,1.744135E-2,4.2231977E-2,8.847342E-2,1.5339875E-1,0E0,7.246068E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.9368546E-2,0E0,0E0,6.9041744E-2,4.457295E-2,0E0,0E0,2.259264E-2,1.8284276E-2,2.8167114E-2,0E0,0E0,0E0,1.671055E-2,0E0,1.6133487E-2,2.6987493E-2,1.1439049E-1,1.253054E-1,6.517204E-2,4.5903146E-2,0E0,4.85404E-2,2.877155E-2,4.449857E-2,0E0,0E0,1.7003655E-2,0E0,0E0,4.2070195E-2,0E0,5.1519215E-2,2.4649315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,25,25,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,39,39,40,40,41,41,42,42,43,43,44,44,50,50,51,51,53,53,54,54,55,55,56,56,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,75,75,77,77,78,78,79,79,81,81,82,82,84,84,85,85,86,86,93,93,94,94,96,96,97,97,99,99,100,100,101,101,102,102,103,103,104,104,105,105,115,115,116,116,117,117,118,118,121,121,122,122,123,123,128,128,129,129,130,130,131,131,132,132,133,133,135,135,136,136,138,138,139,139,140,140,141,141,143,143,152,152,155,155,156,156,159,159,160,160,161,161,165,165,167,167,168,168,169,169,170,170,171,171,172,172,174,174,175,175,176,176,179,179,182,182,184,184,185,185],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,36,38,40,-1,42,-1,44,-1,46,48,50,52,-1,54,56,58,60,-1,-1,-1,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,-1,78,80,82,84,-1,-1,-1,-1,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,-1,110,112,114,-1,116,118,-1,120,122,124,-1,-1,-1,-1,-1,-1,126,128,-1,130,132,-1,134,136,138,140,142,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,154,-1,-1,156,158,160,-1,-1,-1,-1,162,164,166,168,170,172,-1,174,176,-1,178,180,182,184,-1,186,-1,-1,-1,-1,-1,-1,-1,-1,188,-1,-1,190,192,-1,-1,194,196,198,-1,-1,-1,200,-1,202,204,206,208,210,212,-1,214,216,218,-1,-1,220,-1,-1,222,-1,224,226,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,4.4895835E0,2.652E3,1.19111115E2,1E0,8.1E2,2.3876712E7,1.3E1,7.336111E4,8.237624E0,-2.167969E-2,1E0,1.2294118E1,9.059046E6,5.2224144E5,1.00033E5,2.88E0,1.1763657E-2,4.76E2,2.9E1,6.763314E7,3.2911258E6,-1.5133677E-2,8.960784E0,-9.040706E-3,1.1342433E3,1.1384624E-2,3.827E5,1.3784861E5,2.8282208E0,2.2470966E-1,1.9840815E-3,1.4473684E-1,1.3439851E7,9.3326636E-2,1.6254545E1,-1.232758E-2,2.9120783E-4,-3.568988E-3,8.84E2,8E0,1.071E3,3.850926E2,1.2118524E7,4.9036694E0,1.2039306E-2,-1.9101286E-3,-9.710885E-4,-9.285439E-3,-1.8202295E-3,4E-1,7.2896E5,-3.2538017E-3,6.44E2,7E0,8E0,1.0590052E8,-8.9190565E-3,-3.6211764E-3,-7.5402656E-3,2.6586677E-3,1.559733E6,2.124E3,9.244374E6,5.75E2,1.6580646E2,7.2864324E-1,2.41E3,3.8275862E0,4.9963706E5,-4.2558517E-3,8.7E1,4.88E2,-7.6652523E-3,-3.3778017E-3,2.72E0,-2.575395E-3,5.0406504E0,1.2445087E2,5.6321215E4,-5.6848805E-3,2.7976523E2,1.0885427E9,-7.3686386E-3,1.8729467E8,1.5512196E1,4.888889E0,-3.1843367E-3,-1.313834E-2,4.1217147E-3,8.762426E-4,1.9442398E-3,-8.16296E-3,8.9E2,1E1,8.1993686E-4,2.492E2,1.5522388E0,-1.3796739E-2,3.9888642E0,5.62E2,4.0663E4,9.318287E0,1.9E1,4.91027E0,2.8389828E7,-3.9834836E-3,-1.3162144E-2,-7.2342553E-3,-8.248221E-3,-2.011797E-3,4.8022768E-5,-4.593682E-3,6.649162E-4,-3.5711122E-3,2.0134516E6,1.6595395E5,8.39E2,2.58466E5,2.3565071E-3,-3.6128052E-3,1.4695653E0,1.8645384E1,1.142E3,-3.7999048E-3,6.9344183E-4,6.946276E-3,-4.660225E-3,1E0,4.07E2,1E0,4.05E2,3.49E2,1.95E2,-7.2920835E-4,3.8795E4,9.822E3,3.6384843E-3,2.1340163E8,6.5590655E6,1.5724638E1,1.0586236E5,1.3861813E-2,9.100503E0,1.0761651E-2,9.2859525E-4,6.2019355E-3,-2.261614E-3,2.1580318E-3,4.2798687E-3,2.0968488E-4,-2.5722475E-3,1.24272164E5,-6.505277E-3,-5.8119214E-4,1.137832E-7,2.8E0,3.9775637E-3,-1.766854E-4,5E0,2.7432E4,5.831829E6,-2.3220961E-3,-8.796674E-4,3.239513E-3,4.3827028E5,3.1010772E-3,3.7509E4,3.7253174E2,7.09415E5,4.2E2,3E0,1.5416006E5,4.8096015E-4,1.08E2,1.609E3,1.862E3,-2.5334295E-3,4.6874394E-4,5.893737E2,2.2980636E-3,-1.4143132E-3,3.307E3,1.2755826E-2,3.596E3,3.0546486E8,-2.8515377E-3,1.741479E-4,4.775488E-3,-4.848594E-4,-5.516882E-3,-7.39282E-3,-2.9940906E-3,-1.0144135E-3,2.3578054E-3,3.9681285E-3,4.8866356E-4,3.6606938E-3,-6.693271E-4,2.9359742E-3,7.4189096E-3,6.2821656E-3,1.9575302E-3,1.088083E-3,-2.4281596E-3,-2.3489571E-3,-1.0297211E-2,9.076303E-4,-4.2149E-3,5.7357843E-3,1.0329944E-3,1.2627515E-2,5.8501055E-3,-1.3251698E-3,-6.4646406E-3,1.3527403E-3,5.563906E-3,-3.0664308E-3,1.216456E-3,5.0566373E-3,9.165458E-3,6.077624E-3,2.2166981E-3,1.0801344E-2,5.1971795E-3,4.294069E-3,7.1184384E-4],"split_indices":[43,68,2,67,6,2,60,3,60,71,0,113,68,43,43,5,69,0,0,3,7,43,0,73,0,4,0,10,48,69,53,0,71,60,53,69,0,0,0,10,8,2,67,62,53,0,0,0,0,0,73,9,0,1,8,32,7,0,0,0,0,9,44,60,2,67,73,2,69,43,0,8,0,0,0,69,0,69,4,43,0,4,46,0,7,71,71,0,0,0,0,0,0,2,3,0,4,68,0,61,10,9,71,8,50,62,0,0,0,0,0,0,0,0,0,60,43,2,1,0,0,68,71,44,0,0,0,0,26,0,19,10,10,0,0,1,44,0,7,43,73,48,0,69,0,0,0,0,0,0,0,0,43,0,0,52,69,0,0,69,1,60,0,0,0,43,0,1,4,12,0,8,48,0,10,2,2,0,0,4,0,0,2,0,2,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.028E3,2.63E2,7.65E2,2.3E2,3.3E1,5.23E2,2.42E2,1.08E2,1.22E2,2.5E1,8E0,2.25E2,2.98E2,2.17E2,2.5E1,5.5E1,5.3E1,8E0,1.14E2,1.6E1,9E0,2.21E2,4E0,2.93E2,5E0,2.05E2,1.2E1,1.1E1,1.4E1,2.4E1,3.1E1,5E0,4.8E1,8.8E1,2.6E1,1E1,6E0,4E0,5E0,1.83E2,3.8E1,8.3E1,2.1E2,8.4E1,1.21E2,6E0,5E0,8E0,6E0,7E0,1.7E1,2.3E1,8E0,2.5E1,2.3E1,6.3E1,2.5E1,1.2E1,1.4E1,6E0,4E0,1.75E2,8E0,2.9E1,9E0,3.7E1,4.6E1,1.01E2,1.09E2,7.9E1,5E0,1.03E2,1.8E1,9E0,8E0,1.8E1,5E0,1.7E1,8E0,1.6E1,7E0,2.4E1,3.9E1,1.4E1,1.1E1,1.22E2,5.3E1,4E0,4E0,1.9E1,1E1,4E0,5E0,1.3E1,2.4E1,7E0,3.9E1,9.7E1,4E0,4.9E1,6E1,1.9E1,6E1,7.3E1,3E1,1.1E1,7E0,1.1E1,7E0,1.3E1,4E0,4E0,4E0,1.1E1,5E0,1.2E1,1.2E1,2.6E1,1.3E1,6E0,5E0,1.13E2,9E0,4.7E1,6E0,6E0,7E0,5E0,1.9E1,8E0,3.1E1,3.6E1,6.1E1,4E1,9E0,2.4E1,3.6E1,4E0,1.5E1,2.5E1,3.5E1,6.2E1,1.1E1,2.6E1,4E0,4E0,7E0,8E0,4E0,8E0,4E0,1.5E1,1.1E1,8E0,5E0,4.2E1,7.1E1,4E0,5E0,3.2E1,1.5E1,1.5E1,4E0,4E0,4E0,2.2E1,9E0,1.1E1,2.5E1,1.6E1,4.5E1,2.6E1,1.4E1,8E0,1.6E1,1.7E1,1.9E1,7E0,8E0,2E1,5E0,8E0,2.7E1,8E0,5.4E1,2.2E1,4E0,6E0,5E0,3.6E1,6E0,5E0,6.6E1,2.8E1,4E0,1E1,5E0,1.1E1,4E0,4E0,1.8E1,7E0,4E0,2E1,5E0,9E0,7E0,3.1E1,1.4E1,1E1,1.6E1,6E0,8E0,9E0,7E0,6E0,1.1E1,1E1,9E0,9E0,1.1E1,1.1E1,1.6E1,4E0,5E1,1.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"227","size_leaf_vector":"1"}},{"base_weights":[1.1009438E-3,-3.9112236E-2,7.401193E-2,-6.565131E-2,1.1799968E-2,9.642514E-2,-5.0982617E-2,-5.7125136E-2,-1.785663E-1,4.2089024E-3,1.4745581E-1,2.640706E-1,8.514786E-2,-1.3052501E-1,8.411319E-3,-8.500051E-2,-1.606398E-2,-1.3229026E-1,-1.7716803E-2,-3.3999672E-3,1.2328587E-1,2.1924123E-3,8.84822E-3,1.4566637E-2,4.643914E-3,1.23720154E-1,3.293409E-2,-2.8229157E-2,-2.015771E-1,8.455084E-2,-5.214153E-2,-7.6764576E-2,-2.5022778E-1,1.6954329E-2,-7.8922436E-2,-1.4988591E-1,-1.3125762E-3,7.1650865E-3,-8.685387E-2,2.9813452E-3,9.442373E-3,1.020845E-1,1.8986122E-1,2.0646023E-2,1.0955011E-2,-3.3969164E-3,1.0059561E-3,-1.0332464E-1,-1.4569315E-2,3.2375936E-4,6.2584677E-3,-8.223674E-3,1.9642137E-2,-1.6757555E-1,-6.591733E-2,-9.0988737E-4,-2.4158787E-2,8.109917E-3,6.0118563E-3,-9.401221E-3,-6.02769E-2,-3.3532484E-3,-7.7777305E-3,1.5964994E-2,-5.755993E-2,-1.1471005E-2,-1.5268955E-2,1.02831E-2,8.1624605E-2,2.6887571E-3,2.0287366E-1,2.8626302E-2,-6.2100897E-3,-1.9861078E-3,-6.6061923E-3,-3.379097E-3,5.064358E-3,-8.985815E-3,-4.1306373E-3,-5.707424E-2,-1.684672E-1,-4.080267E-2,2.5809642E-2,-3.4349825E-2,-6.545361E-3,2.1768924E-2,-7.385123E-2,-5.8555426E-3,-7.7722454E-4,2.6048787E-3,-4.7550974E-3,-2.1161705E-2,9.5475465E-2,2.1868771E-1,2.8222546E-3,-3.5680735E-3,3.522247E-2,3.0007742E-2,-6.511559E-2,-3.6720086E-3,-9.882433E-3,-1.5785502E-2,-9.1760166E-2,3.4780245E-2,-4.1845776E-2,-1.70479E-2,-5.985214E-3,-9.127189E-3,3.700356E-2,-5.6874775E-3,1.2595323E-6,-3.8020848E-3,5.323314E-4,1.0407894E-1,-3.943781E-3,1.0890828E-2,4.8748157E-3,-4.9988687E-2,4.4427574E-2,-1.7183678E-2,5.515672E-3,-6.930566E-2,2.0799518E-2,-1.9589649E-3,9.4224914E-4,-7.7101826E-3,-6.4023543E-4,4.5027938E-2,-2.0264322E-3,-3.6868136E-3,2.0699066E-4,-3.1238016E-2,1.605347E-3,6.6782977E-3,-7.2012E-3,-2.4768012E-2,5.0535183E-2,8.743726E-2,8.303924E-3,2.0616E-4,-5.36937E-3,5.4291193E-3,3.662511E-2,-2.3333067E-3,1.8280034E-3,-4.64554E-3,-2.6723002E-3,3.8047738E-3,-2.0751213E-3,-1.8756782E-3,2.4080144E-3,-3.35718E-3,-5.0970225E-4,1.0832227E-3,-1.5949009E-3,1.5376982E-3,-3.7117896E-3,6.5935426E-3,1.9816665E-3,5.313785E-3,1.0745503E-3,9.09989E-4,3.8047836E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,41,43,45,47,49,51,53,55,57,59,61,-1,63,65,-1,-1,67,69,71,-1,-1,-1,73,-1,-1,-1,-1,75,77,79,-1,-1,-1,81,-1,83,-1,-1,85,87,-1,89,-1,91,-1,93,95,-1,-1,-1,-1,-1,-1,-1,97,99,101,103,105,-1,107,109,-1,-1,-1,-1,111,113,115,-1,-1,117,119,121,-1,-1,123,125,127,129,131,-1,133,135,-1,-1,-1,-1,137,-1,-1,-1,139,141,143,-1,145,147,-1,-1,-1,-1,149,-1,-1,-1,151,-1,153,-1,155,157,159,-1,-1,-1,-1,161,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0023127E0,8.925421E-1,1.0246727E0,4.0762532E-1,2.331377E-1,5.617058E-1,2.656607E-1,4.6219552E-1,2.3375797E-1,1.9612405E-1,4.3792754E-2,1.2750483E-1,5.8200073E-1,1.6874972E-1,1.5655395E-1,3.087443E-1,3.4379223E-1,4.3264776E-2,0E0,1.8048808E-1,4.8753113E-2,0E0,0E0,0E0,0E0,2.1959591E-1,3.087452E-1,2.6517067E-2,1.23927295E-1,5.8273993E-2,1.7516066E-1,2.1650028E-1,6.812315E-1,1.8809797E-1,1.2012792E-1,1.9186378E-2,0E0,1.0458145E-1,2.6201448E-1,0E0,0E0,2.974E-1,5.672252E-2,1.5110719E-1,0E0,0E0,0E0,1.5393563E-2,0E0,0E0,0E0,0E0,1.1654924E-1,3.3123553E-2,1.7901105E-1,0E0,0E0,0E0,9.651597E-2,0E0,1.02767706E-1,0E0,0E0,8.5534886E-2,5.98725E-2,0E0,1.1257855E-1,0E0,1.6004872E-1,0E0,6.735933E-2,8.2322314E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3710368E-1,4.7697335E-2,3.8070694E-2,4.5759827E-2,6.3679166E-2,0E0,7.188278E-2,3.6393583E-2,0E0,0E0,0E0,0E0,2.8922964E-2,1.6135865E-1,3.686881E-2,0E0,0E0,8.693366E-2,7.3758274E-2,6.515962E-2,0E0,0E0,2.2051597E-2,5.6215197E-2,5.43594E-2,1.6183186E-2,2.6810357E-2,0E0,1.1933007E-1,8.6510345E-2,0E0,0E0,0E0,0E0,1.026454E-1,0E0,0E0,0E0,4.01902E-2,5.1582277E-2,2.4691237E-2,0E0,5.4718196E-2,4.0106725E-2,0E0,0E0,0E0,0E0,3.7351184E-2,0E0,0E0,0E0,2.263704E-2,0E0,3.328061E-2,0E0,6.397338E-2,5.30473E-2,1.3716805E-1,0E0,0E0,0E0,0E0,6.805012E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,42,42,43,43,47,47,52,52,53,53,54,54,58,58,60,60,63,63,64,64,66,66,68,68,70,70,71,71,79,79,80,80,81,81,82,82,83,83,85,85,86,86,91,91,92,92,93,93,96,96,97,97,98,98,101,101,102,102,103,103,104,104,105,105,107,107,108,108,113,113,117,117,118,118,119,119,121,121,122,122,127,127,131,131,133,133,135,135,136,136,137,137,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,42,44,46,48,50,52,54,56,58,60,62,-1,64,66,-1,-1,68,70,72,-1,-1,-1,74,-1,-1,-1,-1,76,78,80,-1,-1,-1,82,-1,84,-1,-1,86,88,-1,90,-1,92,-1,94,96,-1,-1,-1,-1,-1,-1,-1,98,100,102,104,106,-1,108,110,-1,-1,-1,-1,112,114,116,-1,-1,118,120,122,-1,-1,124,126,128,130,132,-1,134,136,-1,-1,-1,-1,138,-1,-1,-1,140,142,144,-1,146,148,-1,-1,-1,-1,150,-1,-1,-1,152,-1,154,-1,156,158,160,-1,-1,-1,-1,162,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,9.860918E5,1E0,1.0657745E7,7.82261E6,1.2083E4,1.9674084E5,6.99E2,7.627907E0,4.1E1,5.51E2,1.562E5,8E0,2.5345264E7,1.7234043E1,1.335955E2,7.453E3,5.1485147E0,-1.7716803E-2,2.8302418E10,1.0717949E1,2.1924123E-3,8.84822E-3,1.4566637E-2,4.643914E-3,6.46E2,4.91027E0,6E0,5E0,2.652E3,2.0900433E1,3.4E1,5.5E1,1.1634076E4,3.4402E4,1.4979E4,-1.3125762E-3,1E0,1.3254E4,2.9813452E-3,9.442373E-3,5.6E1,3.307E3,3.0392378E7,1.0955011E-2,-3.3969164E-3,1.0059561E-3,5.1917776E-2,-1.4569315E-2,3.2375936E-4,6.2584677E-3,-8.223674E-3,3.7827366E8,5.0363636E1,3.1E1,-9.0988737E-4,-2.4158787E-2,8.109917E-3,6.613774E5,-9.401221E-3,8.998703E-1,-3.3532484E-3,-7.7777305E-3,1E0,7.694314E-7,-1.1471005E-2,2.0217392E0,1.02831E-2,3.7043128E5,2.6887571E-3,4.7E1,8.147158E4,-6.2100897E-3,-1.9861078E-3,-6.6061923E-3,-3.379097E-3,5.064358E-3,-8.985815E-3,-4.1306373E-3,8.414097E-1,1.6E1,3.3E0,2.6789763E9,1.6533886E8,-6.545361E-3,1.7733E4,5.4814816E0,-5.8555426E-3,-7.7722454E-4,2.6048787E-3,-4.7550974E-3,1.6869566E1,1E0,2.8146256E5,2.8222546E-3,-3.5680735E-3,5.4814816E0,1.1E1,1.073125E1,-3.6720086E-3,-9.882433E-3,3.2638438E5,6.74502E0,1E0,3.57E2,6.5062125E5,-5.985214E-3,2.72E2,6.078218E2,-5.6874775E-3,1.2595323E-6,-3.8020848E-3,5.323314E-4,8.83E3,-3.943781E-3,1.0890828E-2,4.8748157E-3,6.1454544E0,3.8197617E9,9.183432E4,5.515672E-3,3.3820656E7,1.279012E7,-1.9589649E-3,9.4224914E-4,-7.7101826E-3,-6.4023543E-4,8.48E3,-2.0264322E-3,-3.6868136E-3,2.0699066E-4,8.818731E0,1.605347E-3,1E0,-7.2012E-3,3.6507E4,8.691756E-1,4.643602E8,8.303924E-3,2.0616E-4,-5.36937E-3,5.4291193E-3,1.836095E6,-2.3333067E-3,1.8280034E-3,-4.64554E-3,-2.6723002E-3,3.8047738E-3,-2.0751213E-3,-1.8756782E-3,2.4080144E-3,-3.35718E-3,-5.0970225E-4,1.0832227E-3,-1.5949009E-3,1.5376982E-3,-3.7117896E-3,6.5935426E-3,1.9816665E-3,5.313785E-3,1.0745503E-3,9.09989E-4,3.8047836E-3],"split_indices":[2,43,6,60,43,9,48,2,69,8,2,10,32,60,71,61,44,69,0,46,71,0,0,0,0,0,50,8,8,2,71,2,0,43,9,1,0,6,44,0,0,0,2,58,0,0,0,53,0,0,0,0,7,62,8,0,0,0,60,0,57,0,0,79,52,0,71,0,43,0,8,43,0,0,0,0,0,0,0,68,3,68,46,5,0,1,71,0,0,0,0,73,31,48,0,0,71,3,69,0,0,60,69,19,0,43,0,10,48,0,0,0,0,2,0,0,0,69,46,43,0,7,7,0,0,0,0,1,0,0,0,71,0,30,0,1,68,7,0,0,0,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,6.59E2,3.63E2,4.33E2,2.26E2,3.08E2,5.5E1,4.04E2,2.9E1,2.15E2,1.1E1,1.8E1,2.9E2,2.3E1,3.2E1,2.4E2,1.64E2,2.5E1,4E0,2.03E2,1.2E1,4E0,7E0,1.3E1,5E0,1.66E2,1.24E2,1E1,1.3E1,1.4E1,1.8E1,2.3E2,1E1,1.08E2,5.6E1,2.1E1,4E0,1.81E2,2.2E1,8E0,4E0,1.27E2,3.9E1,1.18E2,6E0,5E0,5E0,8E0,5E0,6E0,8E0,6E0,1.2E1,2.3E1,2.07E2,6E0,4E0,6E0,1.02E2,6E0,5E1,5E0,1.6E1,1.6E2,2.1E1,6E0,1.6E1,1.7E1,1.1E2,4E0,3.5E1,1.13E2,5E0,4E0,4E0,6E0,6E0,1.6E1,7E0,1.92E2,1.5E1,3E1,7.2E1,3.9E1,1.1E1,1.51E2,9E0,7E0,1.4E1,9E0,7E0,1.3E1,9.7E1,3.1E1,4E0,6E0,1.07E2,1.6E1,1.76E2,6E0,9E0,2.1E1,9E0,6.4E1,8E0,3.4E1,5E0,5E1,1.01E2,5E0,4E0,4E0,9E0,9.3E1,4E0,2.6E1,5E0,1E1,9.7E1,1.1E1,5E0,1.68E2,8E0,1.2E1,9E0,4E0,5E0,5.7E1,7E0,4E0,4E0,2.7E1,7E0,4.6E1,4E0,1.8E1,8.3E1,7.8E1,1.5E1,6E0,4E0,8E0,8.9E1,7E0,4E0,4.3E1,1.25E2,4E0,4E0,4E0,5.3E1,8E0,1.9E1,3.3E1,1.3E1,9E0,9E0,5E0,7.8E1,5.4E1,2.4E1,6.6E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"163","size_leaf_vector":"1"}},{"base_weights":[-1.7823981E-3,-4.0926073E-2,6.908415E-2,-8.466238E-2,-1.5719987E-2,3.4781877E-2,1.426687E-1,-6.65194E-2,-2.2224681E-1,-7.045499E-2,2.00722E-3,4.7250196E-2,-1.661988E-1,1.8625151E-2,1.224276E-1,8.938055E-2,-7.689234E-2,-3.3350907E-2,-4.3763816E-1,-5.9620846E-2,-9.797345E-3,-1.9559527E-3,8.344906E-3,1.6832297E-1,3.7220646E-2,-1.7415055E-3,-1.265517E-2,1.3706036E-1,-3.760608E-2,1.0930074E-2,6.8394765E-3,-1.0896457E-1,-5.3991854E-2,-5.618569E-3,1.782589E-2,-2.4994403E-2,-7.578095E-3,-9.8991886E-2,-2.288893E-2,1.421071E-2,-3.6013566E-2,9.800655E-3,3.1952346E-3,-6.388553E-2,5.154758E-2,-1.0179935E-3,1.491423E-1,-4.9310275E-3,2.442503E-3,-2.3401529E-3,3.440138E-3,-1.8716906E-1,-8.632807E-2,-3.3540778E-2,-9.6131526E-2,3.0979766E-3,-1.5881547E-3,4.572157E-4,-1.1829479E-1,-1.0720574E-2,-6.2556736E-2,2.6304418E-2,-2.0596504E-2,4.977003E-3,-4.773916E-2,1.3387327E-2,-1.9496838E-2,7.179611E-2,1.1792234E-2,1.7904058E-1,1.0037008E-1,-2.2123496E-1,-2.5109504E-3,1.3077036E-3,-9.602929E-2,-4.3801606E-2,2.5729572E-3,-2.8782912E-2,-1.6550991E-1,-8.080106E-3,-8.779007E-2,-5.083314E-4,-2.7835057E-3,-1.0223103E-3,-4.587428E-3,4.0390793E-2,-1.4233276E-2,-1.2336766E-1,1.3342861E-2,-1.1219512E-1,-2.3116386E-2,6.9491334E-2,-4.966726E-3,8.692444E-2,-1.0145965E-2,-1.0988743E-2,5.948917E-3,1.8852025E-1,3.0973414E-3,-1.1658761E-2,1.4079677E-1,-7.1715E-3,-1.4275366E-2,-8.131578E-2,-7.2465115E-3,-2.5789065E-2,-8.602322E-2,-1.4941373E-1,5.6944806E-2,-9.179282E-3,-2.7952767E-3,-5.4082677E-2,-5.434599E-3,-1.3082469E-2,3.5828743E-3,1.1836862E-2,6.163217E-2,-6.3834814E-3,2.4491977E-3,-1.09247165E-2,-2.5168979E-3,-1.6950825E-2,6.552288E-2,-6.0261615E-2,-1.6776241E-1,-8.509088E-3,-1.0564108E-2,5.828922E-4,6.093556E-3,5.5383097E-2,1.2170904E-1,6.0001258E-2,-9.330804E-2,6.4652073E-3,-9.366883E-2,1.6613105E-1,1.3439045E-2,-2.1326744E-3,1.8007722E-3,9.645941E-2,9.746061E-3,-9.264131E-2,2.0176284E-3,-3.802059E-2,9.327427E-4,-6.6219675E-3,-1.6317331E-3,-1.1597686E-2,-8.2730426E-4,-3.2230516E-4,4.5167725E-3,-3.4073547E-3,4.7528482E-7,1.1356301E-3,-1.9241748E-3,-4.3642693E-3,8.031804E-2,7.39972E-2,-8.077768E-4,-4.180844E-3,1.604562E-2,-1.5501073E-3,1.0083345E-3,4.472581E-3,-1.0474909E-3,-3.4120228E-4,-8.596113E-2,-3.4949712E-3,-1.029082E-2,2.2867149E-2,-5.1443856E-2,9.102348E-2,-4.608941E-2,5.3078935E-2,1.488305E-1,-1.2253443E-3,4.923842E-3,2.323397E-3,-9.120735E-3,-3.668458E-3,1.829681E-2,-5.8822064E-3,-1.5961746E-3,1.465743E-1,1.3548806E-2,1.9197954E-3,5.775997E-3,-6.1476664E-3,-3.3300633E-3,-7.5694895E-4,-5.4485486E-3,-1.4037599E-3,1.0096745E-3,6.1842E-3,1.0308623E-3,4.1973614E-3,1.0633453E-3,2.3505064E-3,-1.4537458E-3,-5.8272686E-3,-9.499235E-4,-3.1516328E-4,5.358671E-3,-3.3846407E-3,1.4188082E-3,5.24261E-4,5.2724076E-3,2.1776587E-3,-6.70844E-3,-4.4813997E-4,3.811271E-3,7.7153766E-3,1.392189E-3,1.4253506E-3,-2.3414674E-3,1.0679259E-2,5.7965866E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,-1,41,43,-1,-1,45,47,-1,49,51,53,-1,55,-1,-1,57,59,61,63,-1,-1,65,67,-1,69,-1,-1,-1,-1,71,73,75,77,-1,-1,-1,79,81,83,85,87,-1,89,91,-1,93,95,97,99,101,-1,-1,103,105,-1,107,109,-1,111,113,-1,-1,-1,115,117,119,121,123,125,127,-1,129,131,133,-1,135,-1,137,139,-1,-1,141,-1,143,145,147,149,-1,-1,151,-1,153,-1,155,157,-1,159,-1,-1,161,163,165,167,-1,169,-1,-1,171,173,175,177,179,181,183,-1,-1,-1,185,-1,187,-1,189,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,191,193,195,-1,-1,197,-1,-1,-1,-1,-1,199,-1,-1,201,203,205,207,209,211,-1,-1,-1,-1,-1,213,-1,-1,215,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8766673E0,7.3508644E-1,9.26437E-1,5.942868E-1,4.125184E-1,6.4184344E-1,5.675175E-1,3.5779214E-1,1.1246276E0,1.4691907E-1,2.283005E-1,2.8395385E-1,1.9732544E-1,0E0,2.6274514E-1,1.6848947E-1,1.4394867E-1,7.444237E-2,2.7895808E-1,1.3945916E-1,0E0,1.7450075E-1,0E0,6.353611E-2,3.2486933E-1,0E0,0E0,1.9742656E-1,6.818693E-2,0E0,4.2650983E-2,1.3254523E-1,1.01053625E-1,0E0,3.0574983E-2,0E0,0E0,9.995845E-2,2.4429344E-2,9.104566E-2,1.7706119E-1,0E0,0E0,7.805198E-1,1.561259E-1,0E0,1.1910081E-1,0E0,0E0,0E0,0E0,6.941813E-2,7.722819E-2,7.843513E-2,1.7746517E-1,0E0,0E0,0E0,5.371189E-2,2.0707555E-2,1.5626907E-2,9.193957E-2,1.969734E-1,0E0,1.4817952E-1,1.7025469E-1,0E0,1.6079628E-1,1.7995669E-1,4.787779E-2,1.7374176E-1,3.3131897E-2,0E0,0E0,4.273194E-2,5.482045E-2,0E0,2.2567782E-1,5.4175317E-2,0E0,2.1805331E-2,3.5368964E-2,0E0,0E0,0E0,7.144691E-2,8.71788E-2,9.6197665E-2,6.908185E-2,6.492126E-2,1.3957712E-1,5.867447E-2,0E0,1.1284906E-1,1.2810396E-1,8.286195E-2,0E0,7.086134E-2,0E0,2.101843E-2,6.675249E-2,0E0,0E0,7.765496E-2,0E0,3.1136673E-2,5.8868155E-2,1.157819E-1,3.4604877E-2,0E0,0E0,1.5961781E-2,0E0,3.437814E-2,0E0,5.80431E-2,6.781703E-2,0E0,4.9473673E-2,0E0,0E0,1.8757867E-2,4.6347134E-2,1.9748509E-2,4.3928772E-2,0E0,9.137602E-2,0E0,0E0,2.1756987E-1,8.702904E-2,5.1187996E-2,1.5943608E-1,5.0220292E-2,1.5497029E-2,6.8903446E-2,0E0,0E0,0E0,2.460298E-2,0E0,3.0306697E-2,0E0,7.241744E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.000048E-2,2.8069824E-2,4.8611194E-2,0E0,0E0,5.7869907E-2,0E0,0E0,0E0,0E0,0E0,2.3679346E-2,0E0,0E0,1.0308711E-1,5.6024097E-2,7.968804E-2,1.5469018E-1,3.0349419E-2,7.26192E-2,0E0,0E0,0E0,0E0,0E0,3.880467E-2,0E0,0E0,4.418516E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,27,27,28,28,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,43,43,44,44,46,46,51,51,52,52,53,53,54,54,58,58,59,59,60,60,61,61,62,62,64,64,65,65,67,67,68,68,69,69,70,70,71,71,74,74,75,75,77,77,78,78,80,80,81,81,85,85,86,86,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,97,97,99,99,100,100,103,103,105,105,106,106,107,107,108,108,111,111,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124,126,126,129,129,130,130,131,131,132,132,133,133,134,134,135,135,139,139,141,141,143,143,155,155,156,156,157,157,160,160,166,166,169,169,170,170,171,171,172,172,173,173,174,174,180,180,183,183],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,-1,42,44,-1,-1,46,48,-1,50,52,54,-1,56,-1,-1,58,60,62,64,-1,-1,66,68,-1,70,-1,-1,-1,-1,72,74,76,78,-1,-1,-1,80,82,84,86,88,-1,90,92,-1,94,96,98,100,102,-1,-1,104,106,-1,108,110,-1,112,114,-1,-1,-1,116,118,120,122,124,126,128,-1,130,132,134,-1,136,-1,138,140,-1,-1,142,-1,144,146,148,150,-1,-1,152,-1,154,-1,156,158,-1,160,-1,-1,162,164,166,168,-1,170,-1,-1,172,174,176,178,180,182,184,-1,-1,-1,186,-1,188,-1,190,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,192,194,196,-1,-1,198,-1,-1,-1,-1,-1,200,-1,-1,202,204,206,208,210,212,-1,-1,-1,-1,-1,214,-1,-1,216,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1E0,7.098E3,3.7568388E0,5.02E2,4.217427E7,1.2646534E4,7E0,8.317E3,4.519E3,7.82261E6,4.125E0,1.7E1,1.8625151E-2,3.206931E2,1.3050649E4,9.9E1,3.3857143E1,2.277451E2,1.463E2,-9.797345E-3,1.4E1,8.344906E-3,1.94261E5,5.9754E4,-1.7415055E-3,-1.265517E-2,2.0778275E-1,6.9879E4,1.0930074E-2,6.228E1,9.2058825E-1,1.1891E4,-5.618569E-3,3.6E1,-2.4994403E-2,-7.578095E-3,2.824856E-3,2.1481E4,1E0,8.6E1,9.800655E-3,3.1952346E-3,2.466E3,4.6603775E0,-1.0179935E-3,7.153514E1,-4.9310275E-3,2.442503E-3,-2.3401529E-3,3.440138E-3,3.89E2,5E0,1.3E1,1.7717391E0,3.0979766E-3,-1.5881547E-3,4.572157E-4,9.917037E2,4.5726547E5,4.5087484E5,4.9390244E0,2.0217392E0,4.977003E-3,2.7534653E2,1.1E1,-1.9496838E-2,2E0,1.836095E6,1.7655972E0,1.3388E4,4.456432E6,-2.5109504E-3,1.3077036E-3,3.2856784E7,6.4153387E-6,2.5729572E-3,1.169375E2,3.8575E4,-8.080106E-3,4.325E0,3.8767453E4,-2.7835057E-3,-1.0223103E-3,-4.587428E-3,1.6494E4,5.633663E0,3.2962964E0,4.4970587E2,1.9578822E5,1.5E1,1E0,-4.966726E-3,1.3457517E10,1.8645384E1,2.2E1,5.948917E-3,2.5045E4,3.0973414E-3,3.8923203E8,3.8537518E2,-7.1715E-3,-1.4275366E-2,2.28E4,-7.2465115E-3,2.866353E2,2.0549193E2,1E1,1.4386049E6,-9.179282E-3,-2.7952767E-3,1.6341463E0,-5.434599E-3,8.201515E5,3.5828743E-3,4.0351807E2,4.519E3,-6.3834814E-3,7.5E-1,-1.09247165E-2,-2.5168979E-3,1.0003492E7,2.978142E0,2.0261577E2,3.9939122E5,-8.509088E-3,8.931E3,5.828922E-4,6.093556E-3,1.4872598E8,5.5177975E6,3.72381E5,1E1,2.5367088E2,9.525663E2,1.752E3,1.3439045E-2,-2.1326744E-3,1.8007722E-3,1E0,9.746061E-3,4.294737E1,2.0176284E-3,2.8387096E0,9.327427E-4,-6.6219675E-3,-1.6317331E-3,-1.1597686E-2,-8.2730426E-4,-3.2230516E-4,4.5167725E-3,-3.4073547E-3,4.7528482E-7,1.1356301E-3,-1.9241748E-3,4.5E1,8.82E2,1.5321098E8,-8.077768E-4,-4.180844E-3,1.6643229E1,-1.5501073E-3,1.0083345E-3,4.472581E-3,-1.0474909E-3,-3.4120228E-4,1.09924164E5,-3.4949712E-3,-1.029082E-2,6.1986052E7,2.593592E6,4.9963706E5,1.5822886E0,1.1481482E0,2.1E1,-1.2253443E-3,4.923842E-3,2.323397E-3,-9.120735E-3,-3.668458E-3,5.388794E6,-5.8822064E-3,-1.5961746E-3,7.0285716E0,1.3548806E-2,1.9197954E-3,5.775997E-3,-6.1476664E-3,-3.3300633E-3,-7.5694895E-4,-5.4485486E-3,-1.4037599E-3,1.0096745E-3,6.1842E-3,1.0308623E-3,4.1973614E-3,1.0633453E-3,2.3505064E-3,-1.4537458E-3,-5.8272686E-3,-9.499235E-4,-3.1516328E-4,5.358671E-3,-3.3846407E-3,1.4188082E-3,5.24261E-4,5.2724076E-3,2.1776587E-3,-6.70844E-3,-4.4813997E-4,3.811271E-3,7.7153766E-3,1.392189E-3,1.4253506E-3,-2.3414674E-3,1.0679259E-2,5.7965866E-3],"split_indices":[2,29,2,68,2,60,48,3,9,44,43,73,3,0,73,43,44,67,67,70,0,8,0,1,1,0,0,53,2,0,62,68,44,0,3,0,0,53,1,30,10,0,0,44,69,0,71,0,0,0,0,1,0,8,69,0,0,0,48,62,60,71,71,0,70,8,0,6,43,49,2,7,0,0,60,52,0,67,9,0,69,48,0,0,0,1,71,69,67,62,8,27,0,46,71,3,0,2,0,47,71,0,0,1,0,67,4,3,43,0,0,68,0,60,0,4,44,0,68,0,0,60,69,70,60,0,44,0,0,12,60,9,8,67,4,0,0,0,0,29,0,62,0,71,0,0,0,0,0,0,0,0,0,0,0,0,2,7,0,0,73,0,0,0,0,0,66,0,0,5,9,43,57,68,3,0,0,0,0,0,62,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.035E3,6.67E2,3.68E2,2.43E2,4.24E2,2.52E2,1.16E2,2.16E2,2.7E1,1.03E2,3.21E2,2.38E2,1.4E1,7E0,1.09E2,1.3E1,2.03E2,1.5E1,1.2E1,9.7E1,6E0,3.15E2,6E0,1.7E1,2.21E2,7E0,7E0,1E2,9E0,4E0,9E0,8.3E1,1.2E2,5E0,1E1,8E0,4E0,4.6E1,5.1E1,2.14E2,1.01E2,1.1E1,6E0,2.7E1,1.94E2,7E0,9.3E1,5E0,4E0,5E0,4E0,1.7E1,6.6E1,8.2E1,3.8E1,5E0,5E0,7E0,3.9E1,4E1,1.1E1,1.59E2,5.5E1,7E0,9.4E1,2.3E1,4E0,1.28E2,6.6E1,5.6E1,3.7E1,1.3E1,4E0,5E0,6.1E1,7.4E1,8E0,2E1,1.8E1,1.2E1,2.7E1,3.4E1,6E0,6E0,5E0,1.18E2,4.1E1,1.3E1,4.2E1,2.5E1,6.9E1,1.6E1,7E0,1.08E2,2E1,5.6E1,1E1,5.1E1,5E0,1E1,2.7E1,9E0,4E0,5.1E1,1E1,5.3E1,2.1E1,8E0,1.2E1,1.3E1,5E0,1.4E1,1.3E1,3E1,4E0,5.1E1,6.7E1,4E0,3.7E1,4E0,9E0,2.7E1,1.5E1,1.4E1,1.1E1,4E0,6.5E1,9E0,7E0,5.8E1,5E1,1.1E1,9E0,4.7E1,9E0,4.4E1,7E0,6E0,4E0,1.8E1,9E0,4.7E1,4E0,4.2E1,1.1E1,9E0,1.2E1,4E0,4E0,5E0,7E0,1E1,4E0,1.3E1,1.7E1,4.2E1,9E0,5.8E1,9E0,4E0,3.3E1,1.9E1,8E0,1.1E1,4E0,5E0,9E0,5E0,6E0,3.6E1,2.9E1,4.3E1,1.5E1,1.5E1,3.5E1,4E0,7E0,4E0,5E0,5E0,4.2E1,5E0,4E0,4E1,4E0,7E0,1.1E1,1.4E1,3.3E1,3.4E1,8E0,2.1E1,2.1E1,4E0,5E0,4.3E1,1.5E1,1.9E1,1.4E1,5E0,4E0,2.8E1,8E0,2.3E1,6E0,1E1,3.3E1,8E0,7E0,5E0,1E1,3E1,5E0,3.6E1,6E0,6E0,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"217","size_leaf_vector":"1"}},{"base_weights":[-2.246716E-3,-7.778354E-2,2.6095033E-2,-6.0222115E-2,-2.0953187E-1,-1.3702271E-3,8.0813855E-2,-7.349078E-2,4.4343524E-2,-3.8790664E-1,-1.0271237E-1,1.966771E-3,-9.652232E-3,1.6765498E-2,7.482498E-2,-6.616697E-2,-1.587768E-1,1.0149643E-2,-3.2918174E-2,-2.2362106E-2,-7.230836E-3,-1.4324442E-1,2.1276942E-4,-5.9582265E-3,7.996801E-2,6.428155E-2,1.7563073E-1,-9.900151E-2,-4.2671546E-2,-1.3761507E-2,-9.30642E-2,2.6838355E-2,-4.6815434E-3,-1.2896681E-3,-1.793611E-1,-6.2180858E-2,2.1290486E-3,1.01124294E-1,-2.418476E-3,6.837701E-2,-6.06279E-3,1.0513901E-2,1.12644E-3,-8.052767E-2,-2.0621286E-1,6.570158E-3,-5.5054422E-2,-5.5885073E-3,-1.1193283E-3,-2.6713351E-3,5.596371E-3,-1.2341322E-2,-4.7161425E-3,-5.3587213E-2,-6.50655E-3,5.279459E-3,-8.045472E-3,6.594445E-2,1.4484148E-1,7.723972E-2,-3.0845776E-2,-9.442909E-2,3.2384848E-4,-1.3958449E-2,-4.3673394E-3,-4.132916E-2,-1.2295029E-1,-6.0812492E-2,1.2907062E-3,9.902494E-3,-9.700982E-2,9.194517E-2,5.7258084E-4,7.790057E-3,2.4970826E-3,6.985779E-2,1.8089138E-1,2.043353E-2,-8.247202E-3,-2.6567988E-2,-1.09578416E-1,-2.1340686E-3,1.85485E-3,-8.3295904E-2,-2.5017614E-2,-6.638835E-3,-1.6682023E-3,-9.239915E-4,-7.999419E-2,1.4523398E-2,-5.5371094E-2,-1.5578517E-2,2.8349573E-3,2.590712E-3,6.577774E-3,3.1344898E-2,8.6996354E-2,1.0408034E-2,2.9937655E-3,8.240165E-2,-2.8716796E-3,-3.4051596E-3,1.8808118E-3,-1.2041012E-1,-4.4448044E-2,-7.4528984E-4,-4.51952E-3,-4.939115E-2,3.6760774E-3,-6.321094E-3,-5.6336038E-2,7.0906878E-3,6.129466E-2,9.6387195E-4,-8.617695E-2,-2.5871943E-3,1.6265125E-3,-3.3372953E-3,3.9881147E-2,1.7698576E-1,6.657143E-2,-6.3252065E-4,7.4849008E-3,-3.7093188E-3,-6.952874E-3,9.4599277E-4,-4.642216E-3,1.5494056E-3,-2.7639477E-3,-2.230725E-3,1.4149114E-3,1.1848741E-3,-3.3805314E-3,1.6459491E-3,-2.0856343E-4,4.851957E-3,1.080785E-3,-5.902269E-3,-8.4228953E-4,5.3548994E-3,8.861318E-4,8.991867E-3,4.2320495E-3,4.317499E-3,-2.3936133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,31,-1,-1,33,-1,35,37,39,41,43,45,-1,47,49,-1,-1,51,53,55,57,-1,59,-1,-1,-1,61,63,-1,65,-1,-1,-1,-1,-1,-1,67,-1,69,-1,71,73,75,77,79,81,-1,-1,83,85,87,-1,89,91,93,-1,-1,-1,95,97,99,-1,101,103,-1,-1,105,107,-1,-1,-1,109,111,113,-1,115,-1,-1,117,119,-1,-1,121,-1,-1,-1,123,125,-1,-1,127,129,-1,131,133,135,-1,137,-1,-1,-1,139,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2285092E0,6.444689E-1,1.1377809E0,3.528955E-1,5.88936E-1,3.5018238E-1,4.002546E-1,1.2940884E-1,4.0011975E-1,1.911695E-1,9.635559E-2,3.0832E-1,0E0,0E0,2.533568E-1,1.5623945E-1,1.2766632E-1,0E0,8.857402E-2,0E0,0E0,6.163782E-2,0E0,2.06106E-1,1.3406324E-1,1.9073576E-1,1.7516172E-1,1.5372622E-1,2.8942782E-1,0E0,2.203741E-2,1.0280403E-1,0E0,0E0,5.4124832E-2,3.0981809E-2,2.213914E-1,5.1967084E-2,0E0,1.984849E-1,0E0,0E0,0E0,8.432683E-2,9.999162E-2,0E0,1.0301763E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.2918215E-2,0E0,1.8585946E-1,0E0,3.2277003E-2,2.7586013E-2,1.442188E-1,1.4752237E-1,6.288397E-2,2.4085097E-2,0E0,0E0,6.5001294E-2,2.904895E-2,3.6759675E-2,0E0,1.139128E-1,4.0865272E-1,2.0959139E-2,0E0,0E0,0E0,1.2466395E-1,5.2401185E-2,8.189341E-2,0E0,4.4229094E-2,3.1332552E-2,0E0,0E0,2.4773046E-2,5.0535973E-2,0E0,0E0,0E0,3.6659285E-2,1.2158606E-1,6.0509697E-2,0E0,2.6731653E-2,0E0,0E0,5.6817323E-2,2.3267531E-1,0E0,0E0,7.106991E-2,0E0,0E0,0E0,4.1030645E-2,3.479987E-2,0E0,0E0,3.5758115E-2,4.9091406E-2,0E0,3.6385737E-2,1.0083707E-1,7.70946E-2,0E0,4.8524186E-2,0E0,0E0,0E0,8.7484166E-2,1.833278E-2,3.549119E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,34,34,35,35,36,36,37,37,39,39,43,43,44,44,46,46,53,53,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,69,69,70,70,71,71,75,75,76,76,77,77,79,79,80,80,83,83,84,84,88,88,89,89,90,90,92,92,95,95,96,96,99,99,103,103,104,104,107,107,108,108,110,110,111,111,112,112,114,114,118,118,119,119,120,120],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,32,-1,-1,34,-1,36,38,40,42,44,46,-1,48,50,-1,-1,52,54,56,58,-1,60,-1,-1,-1,62,64,-1,66,-1,-1,-1,-1,-1,-1,68,-1,70,-1,72,74,76,78,80,82,-1,-1,84,86,88,-1,90,92,94,-1,-1,-1,96,98,100,-1,102,104,-1,-1,106,108,-1,-1,-1,110,112,114,-1,116,-1,-1,118,120,-1,-1,122,-1,-1,-1,124,126,-1,-1,128,130,-1,132,134,136,-1,138,-1,-1,-1,140,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0162934E5,4.6463413E0,5.862504E2,2.83E3,2E0,1E0,1.4E1,2.8025E4,1.11791E5,1.2E1,8.499432E5,4.5866325E6,-9.652232E-3,1.6765498E-2,6.663214E6,2.71E2,6.15E2,1.0149643E-2,4.36853E5,-2.2362106E-2,-7.230836E-3,1.00011E5,2.1276942E-4,4.33E2,1.9E1,5.619497E7,1.213274E9,6.769866E7,2.00087E5,-1.3761507E-2,2.2E1,2.35184E5,-4.6815434E-3,-1.2896681E-3,8.2E1,3.310366E4,5.6E1,3.7788504E7,-2.418476E-3,7.34E0,-6.06279E-3,1.0513901E-2,1.12644E-3,1.9E1,1.76421E6,6.570158E-3,3.2856784E7,-5.5885073E-3,-1.1193283E-3,-2.6713351E-3,5.596371E-3,-1.2341322E-2,-4.7161425E-3,5.30577E-1,-6.50655E-3,4.75356E6,-8.045472E-3,7.6116E4,1.3922E4,4.907764E3,6.70019E1,1.957E3,7.701384E3,-1.3958449E-2,-4.3673394E-3,1.8401923E2,1.7443357E8,2.57E2,1.2907062E-3,3.0084E4,2.1512408E5,2.3043478E2,5.7258084E-4,7.790057E-3,2.4970826E-3,8.1407714E-1,3.2295492E0,2.7664675E5,-8.247202E-3,3.4E1,2.7546012E0,-2.1340686E-3,1.85485E-3,3.1E1,1.453785E6,-6.638835E-3,-1.6682023E-3,-9.239915E-4,3.01E2,1.812513E8,1.5215946E0,-1.5578517E-2,8.599521E5,2.590712E-3,6.577774E-3,1.0775862E0,3.3382E4,1.0408034E-2,2.9937655E-3,3E0,-2.8716796E-3,-3.4051596E-3,1.8808118E-3,1.5661134E0,7.9634375E2,-7.4528984E-4,-4.51952E-3,2.019537E2,1.5152774E2,-6.321094E-3,1.5310282E8,1.3364486E0,1.2E1,9.6387195E-4,1.3092E4,-2.5871943E-3,1.6265125E-3,-3.3372953E-3,2E0,8.7E1,2.9366477E0,-6.3252065E-4,7.4849008E-3,-3.7093188E-3,-6.952874E-3,9.4599277E-4,-4.642216E-3,1.5494056E-3,-2.7639477E-3,-2.230725E-3,1.4149114E-3,1.1848741E-3,-3.3805314E-3,1.6459491E-3,-2.0856343E-4,4.851957E-3,1.080785E-3,-5.902269E-3,-8.4228953E-4,5.3548994E-3,8.861318E-4,8.991867E-3,4.2320495E-3,4.317499E-3,-2.3936133E-3],"split_indices":[43,68,67,2,32,113,0,44,1,8,60,43,0,0,43,2,2,0,1,0,0,5,0,2,8,62,7,7,5,0,71,1,0,0,0,48,3,7,0,61,0,0,0,3,46,0,60,0,0,0,0,0,0,53,0,62,0,12,44,4,73,9,43,0,0,70,7,44,0,44,48,4,0,0,0,53,53,48,0,2,68,0,0,0,9,0,0,0,44,5,69,0,43,0,0,68,9,0,0,8,0,0,0,68,48,0,0,4,67,0,46,68,3,0,10,0,0,0,32,8,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.039E3,2.83E2,7.56E2,2.51E2,3.2E1,5.04E2,2.52E2,2.23E2,2.8E1,1.1E1,2.1E1,4.97E2,7E0,4E0,2.48E2,2.07E2,1.6E1,8E0,2E1,7E0,4E0,1.5E1,6E0,4.52E2,4.5E1,2.26E2,2.2E1,8.5E1,1.22E2,4E0,1.2E1,1.1E1,9E0,4E0,1.1E1,5.6E1,3.96E2,3.9E1,6E0,2.22E2,4E0,1.6E1,6E0,7.4E1,1.1E1,7E0,1.15E2,8E0,4E0,6E0,5E0,4E0,7E0,5.2E1,4E0,3.9E2,6E0,2.3E1,1.6E1,2.04E2,1.8E1,6.3E1,1.1E1,5E0,6E0,9.7E1,1.8E1,4.8E1,4E0,3.74E2,1.6E1,1.5E1,8E0,1.2E1,4E0,1.92E2,1.2E1,1.4E1,4E0,1.2E1,5.1E1,5E0,6E0,2.6E1,7.1E1,1.4E1,4E0,1.6E1,3.2E1,3.5E2,2.4E1,4E0,1.2E1,1E1,5E0,6E1,1.32E2,8E0,4E0,8E0,6E0,7E0,5E0,4.3E1,8E0,5E0,2.1E1,3.8E1,3.3E1,8E0,2.4E1,3.03E2,4.7E1,7E0,1.7E1,4E0,8E0,4E0,5.6E1,2.3E1,1.09E2,4E0,4E0,2E1,2.3E1,4E0,4E0,4E0,3.4E1,1.1E1,2.2E1,4E0,2E1,8.7E1,2.16E2,2.1E1,2.6E1,1E1,7E0,1.1E1,4.5E1,1.8E1,5E0,8.9E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[-2.567656E-3,-5.573658E-2,3.8479526E-2,-4.8533235E-2,-2.565958E-1,2.383949E-2,1.2916835E-1,-8.2375884E-2,-2.506622E-2,-1.01484045E-1,-2.524882E-2,3.0910918E-2,-1.4459175E-1,1.7562997E-1,2.1786032E-2,-7.585837E-2,-1.6404082E-1,7.0511973E-3,-3.0742949E-2,5.20484E-4,-9.116005E-3,1.9763673E-2,1.1887469E-1,-1.1419385E-2,-4.3389145E-3,1.33511E-1,3.1543034E-1,-2.4449607E-2,4.9294084E-3,-1.03909306E-1,-5.7610177E-2,-1.1580421E-2,-4.130033E-3,-2.3796668E-2,-1.12480275E-1,9.005744E-3,1.6761377E-2,-1.0729339E-3,1.3416049E-1,1.5866417E-1,-1.0213535E-3,1.8529927E-2,7.5269165E-3,-5.9708287E-3,4.4458758E-2,-8.619171E-2,-1.0779947E-2,-8.496841E-3,-7.923898E-2,-2.7775029E-2,4.8866454E-3,-8.242472E-3,-5.9152335E-2,2.1102395E-2,-8.611082E-2,7.2912045E-2,1.5482672E-1,1.2456751E-2,1.09403E-1,4.7205263E-3,-9.54947E-4,-2.3603722E-2,-9.9006094E-2,-5.842033E-2,1.3578452E-2,-8.600494E-2,3.3565337E-4,-4.5528298E-4,-4.440114E-2,-4.649631E-3,-4.0993546E-5,-2.8081415E-2,2.806836E-2,-2.7918817E-2,-8.692029E-3,5.204056E-3,-9.764041E-4,2.9035744E-3,7.7642733E-3,1.5350669E-3,1.2866788E-1,-2.9587694E-3,1.743244E-3,-1.1882344E-1,-2.7939728E-3,-4.856889E-3,4.3033942E-4,3.2742673E-3,-2.8316148E-2,-6.6414364E-3,-7.1578465E-2,1.208197E-2,-5.976793E-2,-3.6334645E-2,-1.1628438E-2,3.4833124E-3,-8.955902E-2,3.29158E-2,-3.3604197E-2,-4.1238335E-3,3.502559E-3,1.5145591E-1,9.654395E-4,-8.666436E-2,-1.5984003E-1,6.828456E-4,-2.4901757E-3,-5.582565E-2,-6.2511154E-3,-3.733035E-2,3.219132E-2,-6.5755304E-3,1.3982398E-3,-2.4311658E-2,-9.987207E-2,-3.6536732E-3,2.0446392E-2,-6.3733445E-3,-2.338618E-3,1.0011721E-2,5.3484317E-2,-5.435336E-3,-1.2975327E-2,7.96133E-3,2.8979066E-3,-1.6586041E-3,-4.958629E-3,-9.009088E-3,-3.3081567E-3,1.12197835E-4,-3.0778085E-3,-5.7873293E-3,-9.332417E-5,2.0442274E-3,-1.7622969E-3,4.8791593E-5,-2.178759E-3,-7.192644E-3,-1.6158909E-3,3.5146428E-3,-2.0182246E-4,1.2474451E-3,-2.26424E-3,9.338563E-4,3.910048E-3,2.6906333E-3,-1.7253913E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,-1,33,-1,-1,35,37,-1,-1,39,41,43,-1,45,47,-1,-1,49,51,-1,53,-1,55,57,-1,-1,-1,-1,59,61,-1,63,65,67,-1,-1,69,71,73,75,77,-1,79,-1,-1,81,83,85,87,89,-1,91,93,-1,-1,95,97,99,-1,-1,-1,-1,-1,-1,101,-1,-1,103,-1,-1,-1,-1,105,-1,107,109,111,113,-1,115,117,119,121,-1,-1,123,-1,125,127,-1,-1,129,-1,131,133,-1,-1,135,137,-1,139,-1,-1,141,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.204221E0,6.241642E-1,7.537026E-1,3.362584E-1,6.191022E-1,5.9046566E-1,3.9031708E-1,8.016801E-2,2.5989658E-1,1.21084E-1,0E0,4.6145254E-1,8.075446E-2,2.817402E-1,1.0075929E-1,7.745004E-2,5.5182368E-2,0E0,1.3699488E-1,0E0,0E0,2.1814148E-1,1.1823678E-1,0E0,0E0,1.7573696E-1,9.1415405E-2,1.2850341E-1,0E0,1.2435657E-1,1.06615275E-1,0E0,0E0,1.204668E-1,5.625072E-2,0E0,1.8714371E-1,0E0,4.9468398E-2,1.782251E-1,0E0,0E0,0E0,0E0,4.3463226E-2,4.337862E-2,0E0,3.6059678E-2,4.1594774E-2,1.00816876E-1,0E0,0E0,2.892045E-2,1.3749069E-1,9.568407E-2,5.4694273E-2,3.2042444E-2,0E0,3.6815196E-2,0E0,0E0,2.9701915E-2,2.870065E-2,3.2617927E-2,5.7214536E-2,4.3865412E-2,0E0,6.392962E-2,2.2161794E-1,0E0,0E0,9.7338155E-2,1.0574001E-1,8.312775E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.086428E-2,0E0,0E0,2.5719225E-2,0E0,0E0,0E0,0E0,1.6233398E-2,0E0,4.6658218E-2,7.175084E-2,1.1580853E-1,9.9935815E-2,0E0,4.9238116E-2,2.5052026E-2,1.5257299E-1,4.355167E-2,0E0,0E0,2.252385E-2,0E0,1.6080424E-2,2.3269981E-2,0E0,0E0,2.8460115E-2,0E0,6.4955235E-2,4.49384E-2,0E0,0E0,6.612628E-2,7.0620835E-2,0E0,4.036531E-2,0E0,0E0,1.567538E-1,1.7551258E-1,0E0,4.0358942E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,25,25,26,26,27,27,29,29,30,30,33,33,34,34,36,36,38,38,39,39,44,44,45,45,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,64,64,65,65,67,67,68,68,71,71,72,72,73,73,80,80,83,83,88,88,90,90,91,91,92,92,93,93,95,95,96,96,97,97,98,98,101,101,103,103,104,104,107,107,109,109,110,110,113,113,114,114,116,116,119,119,120,120,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,-1,34,-1,-1,36,38,-1,-1,40,42,44,-1,46,48,-1,-1,50,52,-1,54,-1,56,58,-1,-1,-1,-1,60,62,-1,64,66,68,-1,-1,70,72,74,76,78,-1,80,-1,-1,82,84,86,88,90,-1,92,94,-1,-1,96,98,100,-1,-1,-1,-1,-1,-1,102,-1,-1,104,-1,-1,-1,-1,106,-1,108,110,112,114,-1,116,118,120,122,-1,-1,124,-1,126,128,-1,-1,130,-1,132,134,-1,-1,136,138,-1,140,-1,-1,142,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.31E2,6.0692043E0,1.6466942E3,4.13E2,1.6493898E6,4.217427E7,2.2040408E7,3.578882E1,1.2E1,4.6683E7,-2.524882E-2,4.7429975E6,1.0479E4,3.8771296E3,2.6108465E0,3.9125E1,3.3817584E7,7.0511973E-3,8.7E1,5.20484E-4,-9.116005E-3,2.04115E5,1.0378262E10,-1.1419385E-2,-4.3389145E-3,1E1,1.159499E6,1.3655363E2,4.9294084E-3,2.161111E1,4.577342E0,-1.1580421E-2,-4.130033E-3,3.5E1,1.31306E5,9.005744E-3,1.2294118E1,-1.0729339E-3,1E0,1.6386554E0,-1.0213535E-3,1.8529927E-2,7.5269165E-3,-5.9708287E-3,2.5130852E2,1.957E3,-1.0779947E-2,5.684E3,3.95E2,7.9227E4,4.8866454E-3,-8.242472E-3,6.354E3,1.8439855E5,3.3924444E5,3.82E2,1.28738E5,1.2456751E-2,4.1032645E6,4.7205263E-3,-9.54947E-4,2.125E1,3.9E1,1.8187732E4,1.2E1,3.3817584E7,3.3565337E-4,1.1E1,2.8302418E10,-4.649631E-3,-4.0993546E-5,3.63E2,1.6673345E7,5.0563266E2,-8.692029E-3,5.204056E-3,-9.764041E-4,2.9035744E-3,7.7642733E-3,1.5350669E-3,1.0905187E9,-2.9587694E-3,1.743244E-3,2.6412E5,-2.7939728E-3,-4.856889E-3,4.3033942E-4,3.2742673E-3,2.3412812E0,-6.6414364E-3,1.5E1,7.5793734E8,1.994E3,9.07826E0,-1.1628438E-2,5.1E1,1.6E1,1.862E3,4.48375E5,-4.1238335E-3,3.502559E-3,1.2792593E2,9.654395E-4,2.88E0,4.631579E0,6.828456E-4,-2.4901757E-3,2.0563E4,-6.2511154E-3,3.3817584E7,1E0,-6.5755304E-3,1.3982398E-3,1E0,1.95E2,-3.6536732E-3,1.5853742E1,-6.3733445E-3,-2.338618E-3,8.009E3,5.2692295E6,-5.435336E-3,5.003246E1,7.96133E-3,2.8979066E-3,-1.6586041E-3,-4.958629E-3,-9.009088E-3,-3.3081567E-3,1.12197835E-4,-3.0778085E-3,-5.7873293E-3,-9.332417E-5,2.0442274E-3,-1.7622969E-3,4.8791593E-5,-2.178759E-3,-7.192644E-3,-1.6158909E-3,3.5146428E-3,-2.0182246E-4,1.2474451E-3,-2.26424E-3,9.338563E-4,3.910048E-3,2.6906333E-3,-1.7253913E-3],"split_indices":[2,68,67,2,60,60,66,61,10,7,0,43,9,67,50,4,7,0,8,0,0,5,46,0,0,6,9,71,0,67,73,0,0,3,9,0,68,0,27,69,0,0,0,0,73,9,0,9,2,9,0,0,10,43,43,0,1,0,60,0,0,4,2,43,33,7,0,8,46,0,0,0,47,67,0,0,0,0,0,0,7,0,0,7,0,0,0,0,68,0,8,46,12,71,0,0,3,2,1,0,0,73,0,69,69,0,0,12,0,7,19,0,0,26,10,0,73,0,0,44,60,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,4.39E2,5.69E2,4.25E2,1.4E1,4.91E2,7.8E1,1.73E2,2.52E2,1E1,4E0,4.72E2,1.9E1,5.4E1,2.4E1,1.62E2,1.1E1,7E0,2.45E2,5E0,5E0,4.2E2,5.2E1,5E0,1.4E1,4.3E1,1.1E1,1.6E1,8E0,6.2E1,1E2,4E0,7E0,2.27E2,1.8E1,6E0,4.14E2,5E0,4.7E1,3.7E1,6E0,6E0,5E0,6E0,1E1,5.6E1,6E0,3.1E1,6.9E1,2.21E2,6E0,7E0,1.1E1,3.98E2,1.6E1,1.3E1,3.4E1,1E1,2.7E1,5E0,5E0,1E1,4.6E1,9E0,2.2E1,6.4E1,5E0,8.4E1,1.37E2,6E0,5E0,4.9E1,3.49E2,1.1E1,5E0,9E0,4E0,5E0,2.9E1,6E0,2.1E1,6E0,4E0,2.9E1,1.7E1,5E0,4E0,9E0,1.3E1,1.1E1,5.3E1,7E1,1.4E1,1.33E2,4E0,3.3E1,1.6E1,3.24E2,2.5E1,7E0,4E0,1.7E1,4E0,1.8E1,1.1E1,5E0,8E0,4.4E1,9E0,2E1,5E1,7E0,7E0,1.13E2,2E1,5E0,2.8E1,6E0,1E1,1.54E2,1.7E2,4E0,2.1E1,1.3E1,4E0,6E0,1.2E1,7E0,4E0,7E0,3.7E1,5E0,1.5E1,4.3E1,7E0,5.4E1,5.9E1,1E1,1E1,8E0,2E1,1.2E2,3.4E1,8.3E1,8.7E1,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[3.7052329E-3,-1.3392239E-2,1.1293301E-1,-3.8338415E-2,2.3857282E-2,1.631091E-2,9.9645585E-2,-7.9138584E-2,-1.8889999E-2,1.0220474E-2,8.719398E-2,8.8257894E-2,1.1515891E-2,-6.900184E-2,-1.884737E-2,-1.07211374E-1,-7.146279E-3,-5.1901132E-2,3.0515952E-2,7.0185095E-2,1.627654E-1,-3.3053036E-3,9.6773274E-2,-6.064249E-2,-9.086762E-3,-7.579222E-3,-7.711027E-2,1.5004916E-1,-1.685171E-2,-1.7882802E-2,-1.104559E-1,4.8900127E-2,2.192085E-3,5.4561675E-2,6.166278E-3,9.617119E-3,3.0791692E-3,1.1367391E-1,3.3723872E-2,-6.82102E-2,5.340218E-3,-3.0035583E-2,-1.4521934E-1,3.5626607E-3,8.600763E-3,-8.644546E-3,-9.2343025E-2,-8.9038186E-2,3.185411E-3,-1.1556455E-2,-5.86331E-2,5.569412E-2,-6.5417853E-3,-7.152324E-2,2.7092975E-2,1.15076E-3,7.5273104E-2,1.3409376E-1,5.43253E-2,2.963826E-4,5.224055E-3,-4.7544133E-2,-1.0290723E-1,-7.5654134E-2,1.858653E-2,-1.0400868E-2,-3.5629508E-3,1.4883763E-3,-7.268112E-2,3.9379247E-3,-1.2294161E-1,-2.9599204E-4,-8.635267E-3,-5.6373373E-2,3.2842286E-2,-7.337421E-3,2.8057268E-3,4.40668E-2,1.69264E-1,1.3119429E-2,-1.071209E-2,4.7601566E-2,-3.7232894E-2,5.6640417E-3,1.9868102E-3,1.4637332E-1,1.2322719E-4,7.3623896E-2,7.856037E-5,1.1585838E-3,-3.6526981E-3,-8.506636E-2,-2.4752598E-2,-7.048921E-3,-5.9334535E-2,-5.853912E-3,-9.4698905E-4,3.216998E-3,-1.2496985E-3,-2.4807671E-2,1.7891357E-2,-9.8883994E-2,2.6558195E-3,-1.5565075E-1,2.1051473E-3,-8.9713087E-4,-4.4466844E-3,3.2608781E-3,4.6004853E-4,-2.9323524E-2,3.0723794E-3,1.0482689E-1,3.0395623E-2,9.968748E-3,2.8176096E-3,2.7873942E-3,-3.0307194E-2,7.570677E-3,8.164338E-2,-7.4500754E-2,2.0100423E-3,1.2345921E-1,1.1908433E-2,7.607614E-4,4.2900825E-3,3.0274796E-5,-9.848085E-2,-3.4731673E-4,-9.076999E-2,-5.730533E-3,-3.259769E-2,1.2830986E-3,-7.012057E-2,6.547583E-2,6.3389516E-3,-1.2570807E-1,-7.560943E-4,-5.411486E-3,-1.21586835E-2,1.4184571E-3,-1.7781866E-3,7.824094E-4,-3.6483293E-3,1.7556569E-3,1.2224585E-1,5.5783026E-2,3.3848137E-3,9.5232425E-4,-3.4726185E-3,1.0377073E-3,-2.0016988E-3,9.488311E-4,9.4034225E-2,-1.3008444E-3,-6.1791227E-3,1.1894656E-3,1.371726E-1,-7.4322456E-3,-3.5986681E-3,-2.763639E-3,1.4912166E-3,-1.2982892E-3,-5.291629E-3,-2.5039813E-3,1.416145E-4,-7.4322213E-4,1.4791973E-3,4.4492108E-4,-5.0209644E-3,3.8492386E-3,-1.7016523E-3,6.380402E-4,-2.895762E-3,-9.252627E-3,-4.145214E-3,2.6918112E-3,6.654921E-3,1.6192447E-3,7.643047E-3,-3.6111416E-3,9.59977E-4,5.3370306E-3,2.2070282E-3,3.2869226E-3,7.3763076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,35,-1,37,39,-1,-1,41,43,45,47,49,51,53,55,-1,-1,-1,57,59,61,-1,63,65,-1,-1,67,69,71,73,-1,75,77,-1,79,81,-1,83,85,87,89,-1,91,93,95,97,-1,-1,99,101,-1,103,-1,-1,105,107,-1,109,111,113,115,-1,117,119,-1,-1,121,-1,123,-1,-1,-1,125,127,-1,129,-1,-1,-1,-1,131,133,135,-1,137,-1,-1,-1,-1,139,141,-1,143,145,-1,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,157,159,161,-1,163,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,175,177,179,-1,-1,-1,-1,-1,181,-1,-1,-1,183,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9455996E0,8.381537E-1,4.1469884E-1,4.2699432E-1,3.1177196E-1,0E0,2.0801866E-1,5.5182314E-1,3.7989414E-1,3.7855935E-1,7.0841014E-2,1.7974281E-1,0E0,1.6952473E-1,0E0,6.092608E-2,4.9836463E-1,1.4542796E-1,1.17573485E-1,4.611045E-2,3.4127384E-2,0E0,1.2510109E-1,2.2260833E-1,0E0,0E0,9.1824666E-2,3.569433E-2,1.8940955E-1,7.224123E-2,1.8078822E-1,1.8519932E-1,1.6713972E-1,2.5761992E-2,0E0,0E0,0E0,1.0784495E-1,7.158613E-2,1.06250286E-1,0E0,4.3549493E-2,4.793948E-2,0E0,0E0,1.8049617E-1,1.7349139E-1,8.776646E-2,6.897188E-2,0E0,1.2971076E-1,1.6816932E-1,0E0,3.1519854E-1,9.1480196E-2,0E0,3.443353E-2,1.125797E-1,2.6280023E-2,4.131667E-2,0E0,8.255811E-2,1.1430943E-1,2.5563471E-2,2.5179796E-2,0E0,0E0,1.04395226E-1,1.3529351E-1,0E0,1.5039843E-1,0E0,0E0,1.7288294E-2,3.1743065E-2,0E0,3.0601783E-2,9.8154455E-2,4.72762E-2,3.4634795E-2,0E0,7.024355E-2,5.439528E-2,0E0,0E0,1.3153076E-1,0E0,1.9361801E-2,0E0,0E0,0E0,4.26113E-2,1.0147237E-1,0E0,5.1317275E-2,0E0,0E0,0E0,0E0,1.10039085E-1,8.14925E-2,6.877366E-2,0E0,6.0289502E-2,0E0,0E0,0E0,0E0,1.8851176E-2,2.4303338E-2,0E0,2.0962834E-2,6.907034E-2,0E0,0E0,0E0,2.2107825E-2,1.9921387E-2,1.92689E-2,2.9183663E-2,0E0,7.016677E-2,0E0,0E0,0E0,0E0,2.8570592E-2,9.3126304E-2,2.2104442E-2,0E0,1.8655686E-2,3.2547813E-2,1.0560502E-1,5.6007534E-2,6.3264325E-2,4.6427578E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5321314E-2,1.12425655E-1,7.254931E-2,0E0,0E0,0E0,0E0,0E0,1.7345369E-2,0E0,0E0,0E0,5.9919596E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,37,37,38,38,39,39,41,41,42,42,45,45,46,46,47,47,48,48,50,50,51,51,53,53,54,54,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,67,67,68,68,70,70,73,73,74,74,76,76,77,77,78,78,79,79,81,81,82,82,85,85,87,87,91,91,92,92,94,94,99,99,100,100,101,101,103,103,108,108,109,109,111,111,112,112,116,116,117,117,118,118,119,119,121,121,126,126,127,127,128,128,130,130,131,131,132,132,133,133,134,134,135,135,144,144,145,145,146,146,152,152,156,156],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,36,-1,38,40,-1,-1,42,44,46,48,50,52,54,56,-1,-1,-1,58,60,62,-1,64,66,-1,-1,68,70,72,74,-1,76,78,-1,80,82,-1,84,86,88,90,-1,92,94,96,98,-1,-1,100,102,-1,104,-1,-1,106,108,-1,110,112,114,116,-1,118,120,-1,-1,122,-1,124,-1,-1,-1,126,128,-1,130,-1,-1,-1,-1,132,134,136,-1,138,-1,-1,-1,-1,140,142,-1,144,146,-1,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,158,160,162,-1,164,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,176,178,180,-1,-1,-1,-1,-1,182,-1,-1,-1,184,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,8.995735E5,1.2646534E4,3.9595376E7,4.5866325E6,1.631091E-2,9.059046E6,5E0,2.35E2,2.485E2,6.459696E6,9.9688476E-1,1.1515891E-2,2.707256E7,-1.884737E-2,5.4E1,1.2E1,2.6210527E0,9.113598E6,2.712766E0,4.026E3,-3.3053036E-3,1.0907809E9,5E1,-9.086762E-3,-7.579222E-3,3.14403E5,1.4E1,1.7172844E7,1.241995E2,2.0671213E2,1.853776E7,1.0909091E0,2.8387096E0,6.166278E-3,9.617119E-3,3.0791692E-3,8.7E1,4.435876E2,8E0,5.340218E-3,1.0101351E4,8.960784E0,3.5626607E-3,8.600763E-3,2.2715E4,1.2E1,1.2131707E2,9.971819E8,-1.1556455E-2,9.582126E7,8.118189E6,-6.5417853E-3,2.41E3,6.346204E-7,1.15076E-3,1E0,4.6E1,5.712652E8,3.5056704E7,5.224055E-3,1.16408E5,4.9948E4,4.9E2,1.26E2,-1.0400868E-2,-3.5629508E-3,7.69E2,9.0754684E1,3.9379247E-3,2.3E1,-2.9599204E-4,-8.635267E-3,1.68991E6,1.7676923E2,-7.337421E-3,1.7E1,1.1149426E0,1E0,1.2E1,-1.071209E-2,1.157602E6,1.056178E6,5.6640417E-3,1.9868102E-3,1.3439851E7,1.2322719E-4,2.8646374E8,7.856037E-5,1.1585838E-3,-3.6526981E-3,1.11E2,1E0,-7.048921E-3,4.312981E5,-5.853912E-3,-9.4698905E-4,3.216998E-3,-1.2496985E-3,7.547528E6,2.23607E5,1E0,2.6558195E-3,7E0,2.1051473E-3,-8.9713087E-4,-4.4466844E-3,3.2608781E-3,9.695652E0,2.7005264E2,3.0723794E-3,1.1090909E1,1.676E3,9.968748E-3,2.8176096E-3,2.7873942E-3,1.3809524E0,2.6699028E0,1.4956522E1,7E0,2.0100423E-3,7.785302E5,1.1908433E-2,7.607614E-4,4.2900825E-3,3.0274796E-5,1E1,6.44E2,1.5365228E5,-5.730533E-3,8.315271E0,1.6507974E8,1.625E0,2E1,4.9469455E6,4.080551E-1,-7.560943E-4,-5.411486E-3,-1.21586835E-2,1.4184571E-3,-1.7781866E-3,7.824094E-4,-3.6483293E-3,1.7556569E-3,1.142E3,2.364E3,4.646154E0,9.5232425E-4,-3.4726185E-3,1.0377073E-3,-2.0016988E-3,9.488311E-4,2.295413E6,-1.3008444E-3,-6.1791227E-3,1.1894656E-3,1.2697166E3,-7.4322456E-3,-3.5986681E-3,-2.763639E-3,1.4912166E-3,-1.2982892E-3,-5.291629E-3,-2.5039813E-3,1.416145E-4,-7.4322213E-4,1.4791973E-3,4.4492108E-4,-5.0209644E-3,3.8492386E-3,-1.7016523E-3,6.380402E-4,-2.895762E-3,-9.252627E-3,-4.145214E-3,2.6918112E-3,6.654921E-3,1.6192447E-3,7.643047E-3,-3.6111416E-3,9.59977E-4,5.3370306E-3,2.2070282E-3,3.2869226E-3,7.3763076E-3],"split_indices":[2,43,48,7,43,0,43,6,2,67,9,69,0,60,0,2,33,71,60,68,2,0,7,3,0,0,9,3,60,67,4,62,68,71,0,0,0,8,71,32,0,43,73,0,0,44,3,70,46,0,5,60,0,2,52,0,26,6,47,60,0,7,9,1,44,0,0,2,71,0,8,0,0,43,67,0,3,68,19,3,0,12,9,0,0,60,0,12,0,0,0,1,26,0,60,0,0,0,0,5,9,28,0,8,0,0,0,0,73,70,0,73,44,0,0,0,69,68,73,6,0,43,0,0,0,0,3,1,43,0,71,46,68,3,62,53,0,0,0,0,0,0,0,0,44,2,71,0,0,0,0,0,43,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,9E2,1.4E2,5.39E2,3.61E2,6E0,1.34E2,1.73E2,3.66E2,2.98E2,6.3E1,1.26E2,8E0,1.69E2,4E0,4.2E1,3.24E2,7.3E1,2.25E2,5.3E1,1E1,6E0,1.2E2,1.6E2,9E0,1.3E1,2.9E1,1.8E1,3.06E2,4.7E1,2.6E1,1.36E2,8.9E1,4.4E1,9E0,6E0,4E0,9.4E1,2.6E1,1.54E2,6E0,1.8E1,1.1E1,7E0,1.1E1,2.77E2,2.9E1,1E1,3.7E1,6E0,2E1,1.32E2,4E0,2.2E1,6.7E1,1.9E1,2.5E1,6.9E1,2.5E1,1.9E1,7E0,9.8E1,5.6E1,9E0,9E0,4E0,7E0,2.4E2,3.7E1,4E0,2.5E1,6E0,4E0,1.2E1,2.5E1,7E0,1.3E1,1.21E2,1.1E1,1.5E1,7E0,5.1E1,1.6E1,9E0,1.6E1,6.3E1,6E0,1.8E1,7E0,1.5E1,4E0,3.6E1,6.2E1,2.5E1,3.1E1,4E0,5E0,4E0,5E0,9.2E1,1.48E2,3.1E1,6E0,2.1E1,4E0,7E0,5E0,1.1E1,1.4E1,9E0,4E0,2.1E1,1E2,7E0,4E0,7E0,8E0,2.4E1,2.7E1,1.1E1,5E0,5.4E1,9E0,5E0,1.3E1,5E0,3.1E1,4.6E1,1.6E1,8E0,2.3E1,5.9E1,3.3E1,2.8E1,1.2E2,2.3E1,8E0,1.7E1,4E0,8E0,6E0,5E0,4E0,5E0,1.6E1,5.1E1,4.9E1,4E0,4E0,1.9E1,5E0,5E0,2.2E1,7E0,4E0,7E0,4.7E1,6E0,2.5E1,1.6E1,3E1,5E0,1.1E1,1.4E1,9E0,3.8E1,2.1E1,1.1E1,2.2E1,2.4E1,4E0,1.09E2,1.1E1,6E0,1.7E1,5E0,1.1E1,4.4E1,7E0,8E0,4.1E1,1.4E1,8E0,1.3E1,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"185","size_leaf_vector":"1"}},{"base_weights":[-1.6896375E-3,-1.7278235E-2,1.0297271E-1,-4.4850104E-2,1.9811345E-2,1.6026318E-2,8.9046225E-2,-3.1389132E-2,-1.2141681E-1,2.8260003E-3,4.4210114E-2,4.547129E-2,1.4953184E-1,-6.733951E-2,-3.556275E-3,-3.775534E-2,-1.601674E-1,9.572094E-3,-1.3296606E-1,3.524323E-2,1.4289385E-1,2.8366694E-2,1.3575728E-1,1.662894E-1,-2.215249E-3,-1.4110914E-1,-5.950135E-2,1.08520485E-1,-2.2649748E-2,2.023681E-3,-6.4413175E-2,-2.904859E-1,-1.0108153E-1,3.0521434E-3,8.763196E-2,1.1556856E-3,-1.5132353E-2,-7.901481E-2,4.6831153E-2,9.348502E-3,3.716158E-3,4.7347613E-2,-3.2728433E-3,1.16381E-3,8.680404E-3,1.7969016E-3,1.8612354E-1,-1.0776713E-3,-7.996828E-3,3.7161887E-2,-6.4268805E-2,1.9950835E-1,6.2352218E-2,-8.378911E-2,-4.31869E-3,1.7515651E-3,-9.236442E-2,-1.1113867E-1,-2.1064032E-2,1.8901301E-3,-1.23835586E-1,7.6817484E-3,-9.8623894E-2,-6.587486E-4,6.6759977E-3,5.7972938E-2,-1.5577735E-2,-5.913482E-2,6.056969E-2,5.4327203E-3,2.1132376E-2,2.0330381E-1,3.952182E-3,4.524027E-3,-1.4337078E-3,-5.832092E-2,-1.6418974E-1,1.1305392E-2,4.032403E-3,1.1760663E-2,4.3074563E-3,-6.4046737E-3,-4.8607647E-2,1.0349239E-1,-1.2531519E-2,-6.113991E-3,-1.6560567E-3,-8.242859E-3,-9.99243E-4,-1.9602416E-1,-7.325049E-2,-1.1980067E-2,2.502334E-2,-6.9456054E-3,-1.255789E-3,6.4543122E-3,-2.3885854E-3,-2.346451E-2,-6.1406204E-3,7.270009E-2,-1.2208145E-2,-4.8326543E-3,3.712066E-2,3.1616518E-3,1.007249E-2,-7.388208E-2,-3.203063E-2,-2.8101031E-3,-1.0843636E-2,2.15063E-3,-1.1545226E-3,-1.7161662E-2,-7.3208436E-3,-5.3513714E-4,7.5063356E-3,1.9968376E-2,-3.025174E-2,-4.1439617E-3,-1.4096334E-2,-9.6212946E-2,-1.1046136E-3,-1.6949673E-3,-8.9006545E-3,3.688281E-2,-3.0597564E-2,-2.748403E-3,1.7957533E-3,8.150835E-2,-1.9443603E-3,4.0046684E-3,-5.6911085E-2,5.9090937E-3,2.2678224E-2,-5.28683E-2,-1.1962713E-1,3.2593027E-2,-4.6608146E-2,1.9286551E-3,-4.566239E-2,-1.0608669E-2,7.143529E-2,2.7843105E-2,-4.3032102E-2,-5.5495957E-3,-1.3466601E-3,-3.2579906E-2,2.3046885E-2,2.549738E-2,4.60747E-3,1.9420171E-3,-5.276887E-2,7.102301E-2,7.2105103E-3,-9.555584E-2,1.1102412E-3,1.7279412E-3,-2.23772E-2,8.9306E-4,-3.5921354E-3,-7.922401E-3,-4.008574E-3,-1.8745323E-3,4.1880566E-3,-9.312093E-4,-3.3869282E-3,5.7790417E-4,-3.8722993E-3,-1.973576E-3,1.102998E-3,1.9556483E-4,4.2582313E-3,2.9506234E-3,-2.8114258E-3,-3.4112425E-3,-7.950707E-4,1.059894E-3,-2.6423233E-3,-2.0099138E-3,2.0885812E-3,6.782996E-4,4.7667264E-3,-3.8253152E-4,-3.7655951E-3,3.587164E-3,-2.3929263E-3,-1.5564652E-3,-6.389795E-3,-3.2880423E-3,1.4271927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,-1,55,57,59,61,63,-1,-1,65,67,-1,-1,69,-1,-1,-1,-1,71,-1,-1,73,75,77,79,81,83,-1,85,87,-1,-1,89,91,93,-1,-1,95,-1,97,99,-1,101,103,-1,-1,-1,105,107,-1,-1,109,-1,-1,111,113,115,-1,-1,-1,-1,117,119,121,123,-1,-1,-1,-1,125,-1,127,129,-1,131,-1,-1,133,135,-1,-1,-1,-1,137,-1,-1,-1,139,141,-1,-1,143,-1,145,-1,147,149,-1,-1,151,-1,-1,153,-1,155,157,159,161,163,-1,165,167,169,171,173,-1,-1,175,177,179,-1,-1,181,183,-1,185,-1,-1,187,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7116323E0,9.3454885E-1,4.311037E-1,5.362557E-1,1.6149944E-1,0E0,3.3583617E-1,4.470295E-1,2.4682355E-1,2.1288401E-1,1.3654563E-1,1.1506051E-1,1.866765E-1,1.0435647E-1,5.438614E-1,5.935027E-2,3.7873638E-1,1.122097E-1,3.4633535E-1,1.9965431E-1,3.199938E-2,1.2763725E-1,6.4438E-2,1.190989E-1,0E0,6.377515E-2,8.441746E-2,1.4298806E-1,2.4284074E-1,0E0,6.0704984E-2,4.2410934E-1,1.2784246E-1,9.725519E-2,1.0057299E-1,0E0,0E0,5.1467836E-1,2.0011708E-1,0E0,0E0,1.0064004E-1,0E0,0E0,0E0,0E0,5.6909323E-2,0E0,0E0,4.0332474E-2,9.145838E-2,3.951463E-2,3.9528564E-2,8.9734316E-2,1.500498E-1,0E0,3.140591E-2,5.034226E-2,0E0,0E0,1.061461E-1,6.747031E-2,2.9166162E-2,0E0,0E0,9.686451E-2,0E0,3.8990267E-2,1.06826484E-1,0E0,8.831669E-2,5.6548595E-2,0E0,0E0,0E0,6.3908935E-2,5.133152E-2,0E0,0E0,1.5227805E-2,0E0,0E0,1.0782492E-1,8.333595E-2,9.102121E-2,0E0,0E0,0E0,0E0,1.2343022E-1,2.1492176E-2,1.7249525E-1,7.0390135E-2,0E0,0E0,0E0,0E0,2.9260274E-2,0E0,1.0757071E-1,8.45062E-2,0E0,4.6922967E-2,0E0,0E0,9.076446E-2,5.9926346E-2,0E0,0E0,0E0,0E0,4.5558415E-2,0E0,0E0,0E0,8.888538E-2,7.693039E-2,0E0,0E0,1.8948063E-2,0E0,6.8760276E-2,0E0,5.9710108E-2,3.2815367E-2,0E0,0E0,6.397331E-2,0E0,0E0,4.296054E-2,0E0,2.339222E-2,1.2906748E-1,3.7944794E-2,5.4655436E-2,3.4546264E-2,0E0,4.1000366E-2,4.0881924E-2,2.8873593E-2,6.459721E-2,6.554972E-2,0E0,0E0,5.5977616E-2,7.5711645E-2,6.1495405E-2,0E0,0E0,1.8503688E-2,7.397497E-2,0E0,1.9145362E-2,0E0,0E0,2.5591712E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,37,37,38,38,41,41,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,75,75,76,76,79,79,82,82,83,83,84,84,89,89,90,90,91,91,92,92,97,97,99,99,100,100,102,102,105,105,106,106,111,111,115,115,116,116,119,119,121,121,123,123,124,124,127,127,130,130,132,132,133,133,134,134,135,135,136,136,138,138,139,139,140,140,141,141,142,142,145,145,146,146,147,147,150,150,151,151,153,153,156,156],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,-1,56,58,60,62,64,-1,-1,66,68,-1,-1,70,-1,-1,-1,-1,72,-1,-1,74,76,78,80,82,84,-1,86,88,-1,-1,90,92,94,-1,-1,96,-1,98,100,-1,102,104,-1,-1,-1,106,108,-1,-1,110,-1,-1,112,114,116,-1,-1,-1,-1,118,120,122,124,-1,-1,-1,-1,126,-1,128,130,-1,132,-1,-1,134,136,-1,-1,-1,-1,138,-1,-1,-1,140,142,-1,-1,144,-1,146,-1,148,150,-1,-1,152,-1,-1,154,-1,156,158,160,162,164,-1,166,168,170,172,174,-1,-1,176,178,180,-1,-1,182,184,-1,186,-1,-1,188,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,7.785302E5,3.3817584E7,2.2E1,4.6134964E2,1.6026318E-2,2.36433E6,6.15E2,2.8266037E2,3.0812828E6,4.9321495E6,4.7263342E2,7.153514E1,7.785714E0,3.5E1,6.123E3,4.5E0,7.82261E6,3.9123364E2,1.4598765E0,1.4615384E0,3.592428E7,1.3E1,8.901939E2,-2.215249E-3,5.3E1,5.445545E-1,7.336111E4,1E0,2.023681E-3,6.5E1,1.08712E5,9.207547E0,6.5590655E6,1E0,1.1556856E-3,-1.5132353E-2,2.7664675E5,4.727453E7,9.348502E-3,3.716158E-3,3.0594551E1,-3.2728433E-3,1.16381E-3,8.680404E-3,1.7969016E-3,2.395631E7,-1.0776713E-3,-7.996828E-3,3.8575E4,5.178788E6,2.4595E5,8.754E3,1.7467743E-2,3.3382E4,1.7515651E-3,2.94E2,3E1,-2.1064032E-2,1.8901301E-3,3.88E2,3.1540035E9,7.9E2,-6.587486E-4,6.6759977E-3,1.65915E5,-1.5577735E-2,2.75E0,3.1530054E0,5.4327203E-3,2.0528813E8,1E0,3.952182E-3,4.524027E-3,-1.4337078E-3,1.6547934E2,2.241E2,1.1305392E-2,4.032403E-3,1.834412E6,4.3074563E-3,-6.4046737E-3,1.606722E8,6.999242E7,1.48E1,-6.113991E-3,-1.6560567E-3,-8.242859E-3,-9.99243E-4,1.7144266E4,1.3050649E4,1.857E3,3.9682608E2,-6.9456054E-3,-1.255789E-3,6.4543122E-3,-2.3885854E-3,1.835821E1,-6.1406204E-3,6.1657715E8,1.925673E1,-4.8326543E-3,5.7E3,3.1616518E-3,1.007249E-2,6.737099E7,3.1096E4,-2.8101031E-3,-1.0843636E-2,2.15063E-3,-1.1545226E-3,1.4915254E0,-7.3208436E-3,-5.3513714E-4,7.5063356E-3,5.0857143E0,4.5683857E-2,-4.1439617E-3,-1.4096334E-2,2E1,-1.1046136E-3,2.485E2,-8.9006545E-3,5.603014E6,4.6712E4,-2.748403E-3,1.7957533E-3,1E0,-1.9443603E-3,4.0046684E-3,3.1E2,5.9090937E-3,1.7291568E2,6.0921145E4,1.76421E6,7.9547346E-4,1.7161617E2,1.9286551E-3,9.75458E5,2.2190831E5,2.16577E5,4.2964826E0,1.3631483E8,-5.5495957E-3,-1.3466601E-3,6.4722223E0,1.1E1,1.3177083E1,4.60747E-3,1.9420171E-3,1.1262003E0,5.526E3,7.2105103E-3,2.427869E8,1.1102412E-3,1.7279412E-3,3.4915986E3,8.9306E-4,-3.5921354E-3,-7.922401E-3,-4.008574E-3,-1.8745323E-3,4.1880566E-3,-9.312093E-4,-3.3869282E-3,5.7790417E-4,-3.8722993E-3,-1.973576E-3,1.102998E-3,1.9556483E-4,4.2582313E-3,2.9506234E-3,-2.8114258E-3,-3.4112425E-3,-7.950707E-4,1.059894E-3,-2.6423233E-3,-2.0099138E-3,2.0885812E-3,6.782996E-4,4.7667264E-3,-3.8253152E-4,-3.7655951E-3,3.587164E-3,-2.3929263E-3,-1.5564652E-3,-6.389795E-3,-3.2880423E-3,1.4271927E-3],"split_indices":[2,43,7,3,4,0,43,2,4,66,43,71,71,67,0,43,73,43,4,69,68,60,3,67,0,44,68,60,29,0,44,7,73,43,29,0,0,48,7,0,0,73,0,0,0,0,62,0,0,9,12,9,1,53,9,0,44,3,0,0,10,46,2,0,0,1,0,68,68,0,7,29,0,0,0,4,4,0,0,5,0,0,12,7,73,0,0,0,0,48,43,44,4,0,0,0,0,71,0,7,73,0,10,0,0,7,9,0,0,0,0,68,0,0,0,69,53,0,0,6,0,67,0,9,1,0,0,23,0,0,8,0,73,60,46,53,67,0,66,62,43,68,7,0,0,73,3,71,0,0,68,2,0,7,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.047E3,9.12E2,1.35E2,5.23E2,3.89E2,6E0,1.29E2,4.46E2,7.7E1,2.3E2,1.59E2,7.6E1,5.3E1,1.94E2,2.52E2,2.5E1,5.2E1,2.2E2,1E1,1.47E2,1.2E1,6.5E1,1.1E1,4.9E1,4E0,1.7E1,1.77E2,3.6E1,2.16E2,6E0,1.9E1,1.5E1,3.7E1,2.04E2,1.6E1,6E0,4E0,1.3E1,1.34E2,5E0,7E0,5.5E1,1E1,4E0,7E0,7E0,4.2E1,4E0,1.3E1,8E0,1.69E2,1.1E1,2.5E1,4.9E1,1.67E2,4E0,1.5E1,8E0,7E0,5E0,3.2E1,1.96E2,8E0,6E0,1E1,9E0,4E0,1.5E1,1.19E2,1.4E1,4.1E1,3.5E1,7E0,4E0,4E0,1.61E2,8E0,7E0,4E0,1E1,1.5E1,1.8E1,3.1E1,1.1E1,1.56E2,8E0,7E0,4E0,4E0,1.2E1,2E1,9.2E1,1.04E2,4E0,4E0,5E0,4E0,1.1E1,4E0,1.02E2,1.7E1,4E0,3.7E1,4E0,3.1E1,1E2,6.1E1,4E0,4E0,5E0,5E0,2.5E1,6E0,4E0,7E0,5.5E1,1.01E2,7E0,5E0,1.3E1,7E0,8.8E1,4E0,8.6E1,1.8E1,7E0,4E0,9.5E1,7E0,5E0,1.2E1,4E0,3.3E1,7E1,3E1,1.1E1,5E1,8E0,1.7E1,3.5E1,2E1,1.8E1,8.3E1,9E0,4E0,3.9E1,4.9E1,7.4E1,1.2E1,4E0,1.4E1,8.5E1,1E1,8E0,4E0,2.5E1,8E0,1.8E1,5.2E1,1E1,2E1,5E0,6E0,2.6E1,2.4E1,7E0,1E1,1.8E1,1.7E1,5E0,1.5E1,1.3E1,5E0,3.7E1,4.6E1,1.2E1,2.7E1,1.2E1,3.7E1,6.6E1,8E0,6E0,8E0,8.1E1,4E0,4E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"189","size_leaf_vector":"1"}},{"base_weights":[3.7375215E-4,-2.2854786E-2,7.1063854E-2,-5.8360323E-2,1.0228556E-2,8.425252E-2,-3.8396563E-2,-4.3050762E-2,-1.4425461E-1,-3.8919474E-3,4.4126812E-2,2.1412544E-1,7.600787E-2,-8.458971E-3,1.855634E-2,-7.499242E-2,-9.817644E-3,-2.7764913E-1,-1.09219946E-1,1.0057027E-3,-1.2814783E-2,5.195271E-2,-6.6124233E-3,5.608803E-3,1.2762782E-2,4.6425205E-2,1.12194456E-1,-7.7419385E-2,1.3187157E-1,-7.061742E-2,-9.158675E-3,8.57492E-3,-1.88121E-2,-1.6626748E-3,-1.8969288E-2,-6.0778484E-2,-1.8828017E-1,-2.7380306E-2,2.4074294E-2,9.161428E-2,1.839152E-2,6.436518E-2,-5.232756E-2,1.2440169E-1,-9.069431E-4,-7.1217644E-4,-7.444602E-3,8.472259E-3,2.0179723E-3,-8.366807E-2,-3.0026326E-2,-2.7148295E-2,6.470269E-2,-8.010336E-2,2.396186E-3,-1.0680201E-1,-1.6745647E-2,-1.869584E-2,-7.0748855E-3,-5.630425E-2,3.18294E-2,5.9690524E-2,1.5382847E-1,4.316089E-2,-2.6562203E-2,1.940227E-2,1.1073856E-1,-8.700417E-3,-3.5067042E-3,8.8989146E-2,1.7802781E-1,-6.796249E-2,-1.1799029E-1,-3.81711E-3,-3.888446E-3,-1.6301822E-2,-7.552228E-2,-3.822054E-4,6.415212E-3,1.0924598E-3,-1.0593182E-1,-6.579887E-3,-9.84997E-4,2.7597444E-2,-3.753783E-2,-5.691611E-3,4.2805477E-4,-6.407973E-3,5.142304E-2,6.270709E-3,3.995773E-2,8.671582E-3,2.792569E-3,-4.433734E-3,6.541674E-2,-8.136432E-2,4.0618684E-2,-5.8471568E-2,3.6659908E-2,6.982494E-2,1.5573172E-1,1.5408626E-3,-2.328266E-3,1.165052E-1,3.0721609E-2,1.0859114E-2,1.315351E-1,-1.24975234E-1,-5.3757183E-2,-9.391959E-2,-8.30506E-3,4.103508E-3,-2.2957949E-2,-6.3306004E-2,-3.725974E-3,-4.7688512E-4,-1.0559738E-1,-7.248426E-3,-7.508661E-2,6.478108E-2,-4.0311858E-2,3.4639683E-2,-4.8735023E-2,1.5622701E-2,-5.602467E-3,6.097619E-2,4.8066482E-5,2.9032072E-3,5.1991437E-3,1.7157947E-3,-3.0516386E-3,-3.9130453E-5,4.044495E-3,-1.6181631E-3,-5.9868502E-3,-1.0374942E-3,3.6899229E-3,-6.05112E-3,5.4038176E-4,5.6653224E-2,-2.7900375E-3,4.9900594E-3,-9.71328E-3,2.879077E-3,1.7211762E-1,2.4420454E-3,1.4494091E-1,3.4271793E-3,-8.127193E-4,8.26875E-3,3.332406E-3,-1.8985095E-3,-7.909023E-3,4.1781426E-3,-8.0964744E-2,-1.172697E-1,5.1939E-4,2.1786975E-2,-2.8872078E-3,-1.2262E-2,-7.5702346E-3,-4.35586E-2,1.9688427E-2,-1.4707293E-1,-1.8735223E-3,-9.2844013E-4,-5.05943E-3,2.044092E-2,5.1416717E-3,-4.255829E-3,-3.2574005E-4,-7.035108E-4,3.3464157E-3,-1.23647325E-1,-3.5097986E-2,2.1621373E-3,5.303579E-3,4.0756516E-2,1.0062565E-1,2.6816274E-3,-1.142844E-3,4.214352E-3,2.893743E-2,-3.059964E-3,1.8020627E-3,9.515024E-3,4.612682E-3,7.3838206E-3,2.4686127E-3,-2.1000549E-3,1.2562141E-3,2.9393926E-3,-9.4918493E-4,-5.96852E-3,-2.1517708E-3,-7.7835103E-3,-3.8962588E-3,-8.35593E-4,2.9879336E-3,2.017854E-3,-2.5773207E-3,3.8914723E-4,-3.3399707E-3,-1.8202577E-3,1.6612908E-3,-3.5519698E-3,-9.326895E-3,-1.0680539E-3,2.821849E-3,-2.077275E-3,-9.6590705E-3,1.1493452E-3,-2.2182593E-3,6.698775E-4,-1.8111728E-3,8.8162813E-4,3.5410144E-3,1.6282144E-3,5.675965E-3,3.232429E-4,3.0703444E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,41,43,45,47,49,-1,-1,51,-1,-1,53,55,57,59,61,63,65,67,69,-1,-1,-1,-1,-1,71,73,75,77,79,-1,81,-1,83,-1,85,87,89,91,93,95,97,99,-1,101,103,105,107,109,111,-1,113,115,-1,-1,-1,117,-1,-1,119,121,-1,-1,123,125,-1,127,-1,-1,129,131,133,135,137,139,141,143,-1,-1,145,147,-1,149,151,153,155,-1,-1,157,159,161,-1,163,-1,165,167,169,171,173,175,-1,177,-1,-1,179,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,181,-1,-1,183,-1,185,-1,187,-1,189,-1,-1,-1,-1,191,193,195,-1,197,-1,199,-1,201,203,205,-1,-1,-1,207,-1,-1,-1,-1,-1,209,211,213,-1,215,217,-1,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6863446E0,9.08595E-1,3.6991894E-1,4.842862E-1,1.9211456E-1,2.2678661E-1,2.3415089E-1,3.368274E-1,2.3138726E-1,3.8068515E-1,1.8035841E-1,4.1351438E-2,2.2494793E-1,0E0,2.3859516E-1,7.335484E-2,2.7945817E-1,3.38638E-1,1.6472709E-1,1.8401024E-1,0E0,1.5005928E-1,0E0,0E0,0E0,2.1596271E-1,1.5692794E-1,5.98095E-2,3.7421122E-2,8.118659E-2,0E0,0E0,1.06882095E-1,0E0,0E0,6.946021E-2,2.0255941E-1,1.3474771E-1,9.8410495E-2,9.409565E-2,7.1295395E-2,2.105298E-1,1.2548997E-1,1.5127277E-1,0E0,0E0,0E0,0E0,0E0,5.6128144E-2,5.643835E-2,7.117568E-2,7.691259E-2,7.217732E-2,0E0,3.6549374E-2,0E0,1.04834594E-1,0E0,6.254794E-2,1.06717214E-1,4.859513E-2,4.2425722E-2,4.371965E-2,8.798298E-2,7.337125E-2,8.11677E-2,0E0,2.7605744E-2,8.460185E-2,5.9221506E-2,6.13119E-2,3.664863E-2,5.1688693E-2,0E0,6.715151E-2,4.7932282E-2,0E0,0E0,0E0,2.1230996E-2,0E0,0E0,9.0990454E-2,7.0383936E-2,0E0,0E0,1.2588629E-1,4.517995E-2,0E0,2.353527E-2,0E0,0E0,3.8317557E-2,4.1130565E-2,2.4511077E-2,2.902758E-2,5.240955E-2,8.879821E-2,8.609478E-2,2.4478793E-2,0E0,0E0,5.9301853E-2,2.6100218E-2,0E0,4.6277404E-2,5.499567E-2,1.0910152E-1,7.007365E-2,0E0,0E0,4.381951E-2,1.2117654E-1,8.572939E-2,0E0,3.9897308E-2,0E0,2.6196413E-2,4.5159906E-2,2.1751745E-2,2.3719955E-2,7.125728E-2,5.582016E-2,0E0,5.92331E-2,0E0,0E0,2.2002196E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.280963E-2,0E0,0E0,3.0120505E-2,0E0,2.5905073E-2,0E0,2.5512457E-2,0E0,1.6238572E-2,0E0,0E0,0E0,0E0,3.511108E-2,7.192993E-2,2.4579614E-2,0E0,2.1691162E-2,0E0,4.382789E-2,0E0,5.1397614E-2,5.717989E-2,2.2950247E-2,0E0,0E0,0E0,2.4425022E-2,0E0,0E0,0E0,0E0,0E0,6.453876E-2,5.0777078E-2,1.9962048E-2,0E0,4.0541872E-2,3.3465147E-2,0E0,0E0,0E0,1.7646499E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,25,25,26,26,27,27,28,28,29,29,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,50,50,51,51,52,52,53,53,55,55,57,57,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,80,80,83,83,84,84,87,87,88,88,90,90,93,93,94,94,95,95,96,96,97,97,98,98,99,99,100,100,103,103,104,104,106,106,107,107,108,108,109,109,112,112,113,113,114,114,116,116,118,118,119,119,120,120,121,121,122,122,123,123,125,125,128,128,139,139,142,142,144,144,146,146,148,148,153,153,154,154,155,155,157,157,159,159,161,161,162,162,163,163,167,167,173,173,174,174,175,175,177,177,178,178,182,182],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,42,44,46,48,50,-1,-1,52,-1,-1,54,56,58,60,62,64,66,68,70,-1,-1,-1,-1,-1,72,74,76,78,80,-1,82,-1,84,-1,86,88,90,92,94,96,98,100,-1,102,104,106,108,110,112,-1,114,116,-1,-1,-1,118,-1,-1,120,122,-1,-1,124,126,-1,128,-1,-1,130,132,134,136,138,140,142,144,-1,-1,146,148,-1,150,152,154,156,-1,-1,158,160,162,-1,164,-1,166,168,170,172,174,176,-1,178,-1,-1,180,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,182,-1,-1,184,-1,186,-1,188,-1,190,-1,-1,-1,-1,192,194,196,-1,198,-1,200,-1,202,204,206,-1,-1,-1,208,-1,-1,-1,-1,-1,210,212,214,-1,216,218,-1,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.984E3,4.539777E5,1E0,2.2E1,4.577732E2,3.6E1,3.812E3,5.51E2,3.355864E0,2.511E3,1.1043767E3,1.0952997E3,2.839532E10,-8.458971E-3,2.2539758E5,5.0363636E1,2.04115E5,8E0,2.1772727E4,1.5859042E6,-1.2814783E-2,3.875E0,-6.6124233E-3,5.608803E-3,1.2762782E-2,4.88E2,3.7715748E7,4.9627705E0,3.08564E5,1.6840201E2,-9.158675E-3,8.57492E-3,8.13362E0,-1.6626748E-3,-1.8969288E-2,6.0078123E-6,3.7798166E0,1.70595E5,8.612209E-6,1.09652E5,9.822E3,1.1043767E3,5.11E2,1.6975454E3,-9.069431E-4,-7.1217644E-4,-7.444602E-3,8.472259E-3,2.0179723E-3,6.737099E7,2.732381E2,3.82E2,2.0480016E7,6.123E3,2.396186E-3,5.159652E-7,-1.6745647E-2,1.3902439E0,-7.0748855E-3,2.2715E4,1.185E3,9.46E2,3.43E2,4.4E0,3.4779343E1,3.64E2,1.4992306E8,-8.700417E-3,3.3266933E0,1.6673345E7,8.710612E1,7.785714E0,1.2380953E0,2.90099E0,-3.888446E-3,5.4E1,8.910034E0,-3.822054E-4,6.415212E-3,1.0924598E-3,7.336111E4,-6.579887E-3,-9.84997E-4,2.988E4,5.095006E5,-5.691611E-3,4.2805477E-4,1.2E1,3.7439062E2,6.270709E-3,1.0410659E5,8.671582E-3,2.792569E-3,1.01E2,4E0,2.2E1,3.53562E5,2.0939393E0,9.46898E2,2.059E3,8.822294E6,1.5408626E-3,-2.328266E-3,6.149315E6,1.752E3,1.0859114E-2,2.01E0,5.6E1,6.666982E4,4E0,-8.30506E-3,4.103508E-3,2.992648E-2,1.0439024E1,8.579633E-2,-4.7688512E-4,5.4051723E0,-7.248426E-3,9.99277E4,2.2084616E2,6.23E2,1.8736842E0,6.072304E5,7.6569915E-1,-5.602467E-3,2.8282208E0,4.8066482E-5,2.9032072E-3,3.588785E0,1.7157947E-3,-3.0516386E-3,-3.9130453E-5,4.044495E-3,-1.6181631E-3,-5.9868502E-3,-1.0374942E-3,3.6899229E-3,-6.05112E-3,5.4038176E-4,2.0588236E0,-2.7900375E-3,4.9900594E-3,1.3795834E3,2.879077E-3,1.853776E7,2.4420454E-3,1.4379113E9,3.4271793E-3,3.08E2,8.26875E-3,3.332406E-3,-1.8985095E-3,-7.909023E-3,3.5419354E0,3.774648E0,3.89E2,5.1939E-4,1.4392524E0,-2.8872078E-3,1.397998E6,-7.5702346E-3,5.3776E4,1.4929079E2,1.9067797E1,-1.8735223E-3,-9.2844013E-4,-5.05943E-3,3.0751158E4,5.1416717E-3,-4.255829E-3,-3.2574005E-4,-7.035108E-4,3.3464157E-3,3.2296E4,4E0,1E0,5.303579E-3,2.6138635E2,7.09E2,2.6816274E-3,-1.142844E-3,4.214352E-3,2.1111112E0,-3.059964E-3,1.8020627E-3,9.515024E-3,4.612682E-3,7.3838206E-3,2.4686127E-3,-2.1000549E-3,1.2562141E-3,2.9393926E-3,-9.4918493E-4,-5.96852E-3,-2.1517708E-3,-7.7835103E-3,-3.8962588E-3,-8.35593E-4,2.9879336E-3,2.017854E-3,-2.5773207E-3,3.8914723E-4,-3.3399707E-3,-1.8202577E-3,1.6612908E-3,-3.5519698E-3,-9.326895E-3,-1.0680539E-3,2.821849E-3,-2.077275E-3,-9.6590705E-3,1.1493452E-3,-2.2182593E-3,6.698775E-4,-1.8111728E-3,8.8162813E-4,3.5410144E-3,1.6282144E-3,5.675965E-3,3.232429E-4,3.0703444E-3],"split_indices":[2,43,6,3,4,10,2,2,73,2,67,48,46,0,48,61,5,0,48,43,0,69,0,0,0,0,62,61,1,4,0,0,69,0,0,52,68,1,53,1,44,67,0,67,0,0,0,0,0,7,4,0,5,43,0,52,0,68,0,44,44,44,0,69,71,10,5,0,68,47,73,67,71,69,0,0,71,0,0,0,60,0,0,1,43,0,0,8,4,0,48,0,0,0,8,71,1,68,67,44,62,0,0,60,0,0,68,44,60,8,0,0,53,73,53,0,69,0,43,67,2,68,43,57,0,69,0,0,71,0,0,0,0,0,0,0,0,0,0,69,0,0,67,0,62,0,12,0,8,0,0,0,0,69,69,1,0,68,0,60,0,1,67,73,0,0,0,48,0,0,0,0,0,12,73,30,0,4,2,0,0,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,7.72E2,2.53E2,3.72E2,4E2,2.26E2,2.7E1,3.17E2,5.5E1,2.83E2,1.17E2,1.2E1,2.14E2,7E0,2E1,1.61E2,1.56E2,1E1,4.5E1,2.79E2,4E0,1.13E2,4E0,6E0,6E0,1.19E2,9.5E1,1.1E1,9E0,1.57E2,4E0,6E0,1.5E2,4E0,6E0,2.9E1,1.6E1,1.25E2,1.54E2,5.1E1,6.2E1,1.01E2,1.8E1,8.7E1,8E0,7E0,4E0,5E0,4E0,1.18E2,3.9E1,1.37E2,1.3E1,2.5E1,4E0,1.2E1,4E0,1.18E2,7E0,1.3E1,1.41E2,3.5E1,1.6E1,4E1,2.2E1,5.2E1,4.9E1,4E0,1.4E1,5.4E1,3.3E1,8.3E1,3.5E1,2.7E1,1.2E1,1.13E2,2.4E1,7E0,6E0,5E0,2E1,8E0,4E0,3.4E1,8.4E1,6E0,7E0,4.8E1,9.3E1,6E0,2.9E1,1.1E1,5E0,1.3E1,2.7E1,1.2E1,1E1,9E0,4.3E1,2.7E1,2.2E1,8E0,6E0,3.6E1,1.8E1,1.3E1,2E1,1.5E1,6.8E1,2.7E1,8E0,4E0,2.3E1,2.3E1,9E1,8E0,1.6E1,6E0,1.4E1,2.2E1,1.2E1,1.1E1,7.3E1,4.1E1,7E0,7.8E1,1.5E1,1.7E1,1.2E1,8E0,5E0,7E0,2E1,7E0,5E0,4E0,6E0,4E0,5E0,3.6E1,7E0,1.8E1,9E0,4E0,1.8E1,1.2E1,2.4E1,7E0,1.1E1,1E1,1E1,6E0,9E0,2.2E1,4.6E1,2.2E1,5E0,1.1E1,1.2E1,1.6E1,7E0,3.3E1,5.7E1,9E0,7E0,6E0,8E0,1.2E1,1E1,4E0,8E0,5E0,6E0,1E1,6.3E1,3.7E1,4E0,5.3E1,2.5E1,4E0,8E0,1.5E1,2.1E1,4E0,5E0,1.1E1,7E0,2E1,4E0,4E0,7E0,6E0,1.6E1,1.8E1,2.8E1,7E0,1.5E1,6E0,5E0,7E0,9E0,1.2E1,2.1E1,1.2E1,4.5E1,5E0,4E0,6E0,6E0,6E0,4E0,1.1E1,5.2E1,2.9E1,8E0,3.4E1,1.9E1,7E0,1.8E1,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[-4.676902E-3,-2.7015233E-2,6.269562E-2,-1.4433283E-1,-2.026201E-2,7.7048E-2,-5.1342774E-2,-9.494207E-2,-1.610266E-2,-6.3832276E-2,-1.9735934E-3,4.491648E-2,1.1286066E-1,-1.2745735E-1,-5.681625E-4,-1.1262371E-1,-2.938342E-2,-5.8491174E-2,-1.0141822E-2,8.9425035E-3,-5.341162E-3,1.5622101E-2,9.0033844E-2,1.543186E-1,4.7166493E-2,-1.5672048E-3,-8.751095E-3,4.3330923E-2,-3.0651113E-3,-1.3539714E-3,-5.7882154E-3,7.4794714E-4,-3.191422E-3,-9.907242E-2,-4.4012662E-2,-1.892396E-2,3.1855647E-2,1.0822931E-1,3.3164932E-3,1.4114897E-1,3.090772E-2,1.3716716E-1,1.3015919E-2,6.0195044E-2,-3.072016E-3,-5.882701E-4,3.93597E-3,-7.8674644E-2,-9.776861E-3,2.4785649E-2,-5.5658653E-2,-4.1122038E-2,1.3640688E-2,-2.4848082E-3,7.637008E-2,6.878172E-3,2.121992E-3,3.5483663E-3,-1.5115118E-2,9.114572E-3,1.0523726E-1,-2.7630284E-2,8.2565285E-2,1.4738992E-1,8.7286456E-4,1.8635593E-2,3.5730659E-3,-1.3134022E-1,-4.6591673E-2,3.951455E-3,-2.1455933E-4,-6.1301332E-2,2.0187856E-3,-1.1591104E-2,-3.529352E-2,2.9082855E-2,-5.7990752E-2,-4.078118E-2,2.527182E-2,8.7465055E-2,-3.0207979E-3,-4.6664895E-3,-6.565478E-3,5.4957136E-4,6.054689E-3,1.6593332E-3,-3.6105593E-3,1.4986608E-3,7.219949E-3,8.436715E-2,1.6638876E-1,-1.131794E-3,3.3744434E-3,-2.4045259E-3,-7.69905E-3,2.8641973E-2,-7.286297E-2,1.9043525E-3,-1.4012039E-3,-5.3395648E-2,-1.3524973E-1,-7.0584394E-2,-4.4139833E-3,3.4816034E-2,-5.546807E-3,-2.6899775E-2,-1.1126453E-1,-1.0510528E-2,-8.301122E-3,7.641999E-2,1.2568398E-3,8.1047835E-3,6.731646E-2,8.0370903E-4,-3.5109697E-3,2.2227303E-3,6.039655E-3,1.818477E-1,2.915891E-3,3.3756718E-3,-9.938544E-4,-5.5721467E-3,-5.0246067E-2,-5.949569E-2,5.75592E-4,-2.1886944E-3,-8.00468E-3,-1.6279727E-2,-5.578866E-2,4.6154104E-2,-2.8410831E-2,7.7605054E-2,2.4406698E-2,-2.8837048E-3,1.9822894E-3,-2.5136247E-3,-7.2642816E-3,2.9133026E-2,-6.688009E-2,6.576708E-4,4.515419E-3,2.1913093E-2,-5.6261286E-2,3.962466E-2,1.2559676E-1,-1.2319154E-2,4.6064172E-2,6.051044E-3,1.041517E-2,-1.1298243E-4,-3.4160744E-3,-1.8129412E-3,-3.7574493E-3,-2.947179E-3,5.0381785E-3,2.8153555E-3,-1.392568E-3,-2.459323E-3,2.901479E-3,4.643642E-3,-3.637462E-4,2.4517943E-4,2.3325812E-3,-6.185418E-4,2.5433982E-3,3.1915025E-3,-6.629384E-3,1.583631E-3,-1.5894301E-3,-5.4530525E-3,7.744604E-4,3.6098657E-3,-1.6096738E-4,7.241138E-4,7.5832903E-3,1.0984136E-4,-2.4812461E-3,4.3974826E-3,2.0313096E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,47,49,51,53,55,57,59,61,63,-1,65,-1,-1,-1,67,-1,69,71,73,75,77,79,-1,-1,-1,81,-1,83,85,87,89,-1,91,-1,93,95,-1,97,99,-1,-1,101,103,105,107,109,111,-1,-1,113,-1,-1,-1,-1,-1,-1,115,117,-1,-1,-1,-1,119,121,-1,-1,123,125,127,129,131,-1,133,135,-1,137,139,141,-1,143,145,-1,-1,-1,147,-1,-1,-1,-1,149,151,-1,-1,-1,-1,153,155,157,159,161,-1,-1,-1,-1,163,165,-1,-1,167,169,171,173,175,177,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5894581E0,6.2534726E-1,4.3441367E-1,4.0062398E-1,5.9829426E-1,2.6419806E-1,1.15868226E-1,3.8421363E-2,0E0,1.7176777E-1,3.5003278E-1,1.6380414E-1,2.923113E-1,6.312376E-2,5.773878E-2,2.5873601E-2,1.7400166E-2,1.2255162E-1,0E0,0E0,2.6433644E-1,8.738933E-2,1.437271E-1,1.0777593E-1,6.915995E-2,0E0,0E0,2.9250408E-2,0E0,0E0,0E0,0E0,0E0,1.11916184E-1,1.303767E-1,2.7742112E-1,2.1420023E-1,1.5954398E-2,9.462356E-2,3.1656146E-2,7.485564E-2,7.096934E-2,0E0,2.7061254E-2,0E0,0E0,0E0,7.667437E-2,0E0,3.6465917E-2,8.0046386E-2,2.6991242E-1,1.7424512E-1,8.606886E-2,1.0165897E-1,0E0,0E0,0E0,4.0460773E-2,0E0,4.182124E-2,4.133342E-2,4.2965256E-2,5.3014398E-2,0E0,3.2296866E-2,0E0,4.1722536E-2,6.533447E-2,0E0,2.371108E-2,6.936258E-2,0E0,0E0,2.427063E-1,1.1448563E-1,4.338435E-2,2.04315E-1,5.76351E-2,8.921772E-2,0E0,0E0,2.6666917E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.8924244E-2,5.5575132E-2,0E0,0E0,0E0,0E0,2.1625318E-2,1.9858018E-2,0E0,0E0,4.8947155E-2,3.1878382E-2,4.139595E-1,1.4692824E-1,5.3781584E-2,0E0,4.9789E-2,1.7583951E-2,0E0,6.773403E-2,1.8493675E-2,4.0440455E-2,0E0,7.248983E-2,2.907692E-2,0E0,0E0,0E0,5.0305963E-2,0E0,0E0,0E0,0E0,2.0207822E-2,4.5457184E-2,0E0,0E0,0E0,0E0,1.3890344E-1,4.5832537E-2,1.8900472E-1,4.7788307E-2,5.0546292E-2,0E0,0E0,0E0,0E0,2.1678159E-2,1.3491501E-1,0E0,0E0,1.8588802E-2,4.2577066E-2,5.6720186E-2,6.098543E-2,2.3242194E-2,2.2095801E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,27,27,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,47,47,49,49,50,50,51,51,52,52,53,53,54,54,58,58,60,60,61,61,62,62,63,63,65,65,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,82,82,89,89,90,90,95,95,96,96,99,99,100,100,101,101,102,102,103,103,105,105,106,106,108,108,109,109,110,110,112,112,113,113,117,117,122,122,123,123,128,128,129,129,130,130,131,131,132,132,137,137,138,138,141,141,142,142,143,143,144,144,145,145,146,146],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,48,50,52,54,56,58,60,62,64,-1,66,-1,-1,-1,68,-1,70,72,74,76,78,80,-1,-1,-1,82,-1,84,86,88,90,-1,92,-1,94,96,-1,98,100,-1,-1,102,104,106,108,110,112,-1,-1,114,-1,-1,-1,-1,-1,-1,116,118,-1,-1,-1,-1,120,122,-1,-1,124,126,128,130,132,-1,134,136,-1,138,140,142,-1,144,146,-1,-1,-1,148,-1,-1,-1,-1,150,152,-1,-1,-1,-1,154,156,158,160,162,-1,-1,-1,-1,164,166,-1,-1,168,170,172,174,176,178,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,3.964269E3,2.856934E7,4.6463413E0,5.02E2,7.098E3,1.3655363E2,5.7E1,-1.610266E-2,4.5E1,2.04115E5,5.9671835E6,8.710612E1,4.797448E8,2.342E4,1.2E1,1.86E2,3.89E2,-1.0141822E-2,8.9425035E-3,2.0766992E6,5.617715E6,1.1621469E1,8.604875E3,1.0534078E4,-1.5672048E-3,-8.751095E-3,3.592428E7,-3.0651113E-3,-1.3539714E-3,-5.7882154E-3,7.4794714E-4,-3.191422E-3,2.161111E1,8E0,2.026453E6,5.441268E7,7.6825094E0,3.9914E4,4.27E3,2.95E2,4.931361E-2,1.3015919E-2,1.6326721E0,-3.072016E-3,-5.882701E-4,3.93597E-3,1.2571428E1,-9.776861E-3,3.2226672E7,1.321E3,7E0,3.84E2,2.524828E2,1.8393531E0,6.878172E-3,2.121992E-3,3.5483663E-3,8.295515E7,9.114572E-3,3.0984934E8,1.0657745E7,1.4916515E3,6.7321223E-1,8.7286456E-4,5.063952E-2,3.5730659E-3,1.11E2,2.88E0,3.951455E-3,2.1277844E-4,2.7E1,2.0187856E-3,-1.1591104E-2,3.7439062E2,1E0,5.94E2,2.5247778E6,1.125584E-3,9.4658756E-1,-3.0207979E-3,-4.6664895E-3,1.7E1,5.4957136E-4,6.054689E-3,1.6593332E-3,-3.6105593E-3,1.4986608E-3,7.219949E-3,1.7379E4,4.262058E0,-1.131794E-3,3.3744434E-3,-2.4045259E-3,-7.69905E-3,1.7217412E4,3.887931E0,1.9043525E-3,-1.4012039E-3,1.605076E6,1.2E1,1.16408E5,1.798E3,6E0,-5.546807E-3,8.172E3,2.7012987E1,-1.0510528E-2,6.2521395E6,3.5070792E2,7.2744107E0,8.1047835E-3,6.1319604E2,9.643839E6,-3.5109697E-3,2.2227303E-3,6.039655E-3,1.562752E3,2.915891E-3,3.3756718E-3,-9.938544E-4,-5.5721467E-3,1.8E1,5.217E4,5.75592E-4,-2.1886944E-3,-8.00468E-3,-1.6279727E-2,2.4E1,3.807421E6,3.299862E1,1.6566709E6,1.5026077E7,-2.8837048E-3,1.9822894E-3,-2.5136247E-3,-7.2642816E-3,4.0705118E6,1.1626786E2,6.576708E-4,4.515419E-3,3.82E2,2.33E2,3.839424E2,9.68E2,4.5620965E6,4.6779457E1,6.051044E-3,1.041517E-2,-1.1298243E-4,-3.4160744E-3,-1.8129412E-3,-3.7574493E-3,-2.947179E-3,5.0381785E-3,2.8153555E-3,-1.392568E-3,-2.459323E-3,2.901479E-3,4.643642E-3,-3.637462E-4,2.4517943E-4,2.3325812E-3,-6.185418E-4,2.5433982E-3,3.1915025E-3,-6.629384E-3,1.583631E-3,-1.5894301E-3,-5.4530525E-3,7.744604E-4,3.6098657E-3,-1.6096738E-4,7.241138E-4,7.5832903E-3,1.0984136E-4,-2.4812461E-3,4.3974826E-3,2.0313096E-4],"split_indices":[2,43,60,68,2,2,71,6,0,6,5,60,73,7,9,3,6,1,0,0,43,12,71,4,4,0,0,60,0,0,0,0,0,67,10,9,7,71,9,2,0,72,0,53,0,0,0,70,0,7,10,3,0,4,53,0,0,0,7,0,7,60,4,53,0,72,0,1,69,0,53,8,0,0,4,79,0,43,53,68,0,0,3,0,0,0,0,0,0,2,53,0,0,0,0,43,69,0,0,12,3,7,44,3,0,44,73,0,43,48,71,0,67,62,0,0,0,67,0,0,0,0,70,12,0,0,0,0,3,62,71,43,5,0,0,0,0,43,4,0,0,4,0,4,10,62,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.054E3,7.92E2,2.62E2,4.2E1,7.5E2,2.33E2,2.9E1,3.5E1,7E0,2.21E2,5.29E2,1.24E2,1.09E2,1.1E1,1.8E1,2.7E1,8E0,2.15E2,6E0,8E0,5.21E2,7.6E1,4.8E1,6.6E1,4.3E1,5E0,6E0,1.1E1,7E0,4E0,2.3E1,4E0,4E0,5.5E1,1.6E2,3.82E2,1.39E2,8E0,6.8E1,2.5E1,2.3E1,6E1,6E0,3.9E1,4E0,5E0,6E0,4.8E1,7E0,2.3E1,1.37E2,2.27E2,1.55E2,7.9E1,6E1,4E0,4E0,1.3E1,5.5E1,8E0,1.7E1,1.1E1,1.2E1,5.5E1,5E0,1.2E1,2.7E1,1.7E1,3.1E1,6E0,1.7E1,1.3E2,7E0,5E0,2.22E2,1.28E2,2.7E1,3.3E1,4.6E1,5.6E1,4E0,4E0,5.1E1,4E0,1.3E1,5E0,6E0,8E0,4E0,1.4E1,4.1E1,7E0,5E0,6E0,1.1E1,8E0,2.3E1,7E0,1E1,1.19E2,1.1E1,1.03E2,1.19E2,1.24E2,4E0,1.8E1,9E0,4E0,2.9E1,1.4E1,3.2E1,9E0,4.7E1,4.7E1,4E0,9E0,5E0,3.5E1,6E0,4E0,4E0,6E0,1.7E1,1.09E2,1E1,4E0,7E0,4E0,9.9E1,3.8E1,8.1E1,2.3E1,1.01E2,1.2E1,6E0,5E0,4E0,1.8E1,1.1E1,4E0,1E1,2.4E1,8E0,3.3E1,1.4E1,3.7E1,1E1,1.8E1,1.7E1,6E0,1.1E1,5.8E1,5.1E1,9.5E1,4E0,3.2E1,6E0,6.4E1,1.7E1,1.8E1,5E0,5.9E1,4.2E1,7E0,1.1E1,4E0,7E0,2E1,4E0,4E0,4E0,1.7E1,1.6E1,4E0,1E1,2.8E1,9E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"179","size_leaf_vector":"1"}},{"base_weights":[1.8820602E-3,-2.3254154E-2,6.0664844E-2,-5.817044E-2,-4.1928925E-3,-9.252778E-2,6.8936184E-2,-4.7223926E-2,-2.159959E-1,-5.2362785E-2,8.787443E-3,-6.736836E-4,-5.850275E-3,4.4177786E-2,1.0824084E-1,-5.3769454E-2,4.6137717E-2,9.752715E-4,-3.6037058E-1,-3.6191095E-2,-1.0993559E-1,5.4022055E-3,6.5471455E-3,5.4822262E-2,-1.4119607E-1,1.394868E-1,-4.8247185E-3,7.382396E-2,-5.8908287E-2,3.6359744E-3,5.573894E-4,-2.163606E-2,-7.334285E-3,-6.334675E-2,-1.2614475E-2,-6.461285E-3,-2.5178462E-3,1.9779388E-2,-1.5173376E-2,6.0100798E-2,-3.5424551E-3,2.4684942E-3,-1.6008442E-2,1.2143163E-1,1.2305972E-2,3.0801974E-2,-7.091179E-3,-1.17766685E-4,6.256841E-3,-7.7221796E-2,-3.4447607E-2,-8.518524E-2,-2.7579462E-4,-6.2074084E-2,1.088716E-2,-6.835149E-4,3.4545116E-2,5.4946378E-2,-2.8622575E-2,6.372312E-2,-3.1338425E-3,1.4537615E-1,3.9426208E-2,-4.7660664E-2,8.2752876E-2,-6.494196E-2,-1.8335184E-1,-4.438151E-2,1.3905023E-3,-4.4974156E-2,-4.966688E-3,-4.166776E-3,3.0381273E-4,-7.781972E-3,3.3882523E-3,6.3535824E-2,-1.562934E-2,4.8185106E-2,-8.890355E-3,-1.5526025E-2,6.4513925E-3,-1.07028455E-1,-1.6015884E-2,9.540378E-2,4.4773288E-2,2.086556E-1,1.1762326E-1,-2.1872087E-3,7.379692E-2,7.913354E-4,-4.7547147E-3,5.4156943E-3,1.0624671E-3,-5.5279616E-2,-1.22451484E-1,-1.0283447E-2,-3.8761199E-3,-5.5431057E-2,1.4309211E-3,-3.989234E-3,-4.1560328E-4,9.28315E-4,-2.2553199E-3,1.5722326E-3,5.8921976E-3,-4.038363E-2,2.5596613E-2,3.7701253E-2,1.1580935E-1,-3.2595225E-2,1.9301457E-3,1.375967E-3,-3.0071088E-3,-2.1518843E-2,-1.8064602E-1,-5.7524736E-3,-4.8969863E-3,5.4504536E-2,1.3583387E-1,-3.12618E-3,5.37707E-2,4.7856667E-3,1.1263395E-2,9.045327E-3,1.049118E-1,6.038466E-3,1.2777855E-3,-1.8153297E-2,-7.628401E-2,-1.0370394E-2,-4.8994023E-2,-6.645577E-2,-1.1319116E-3,-4.728548E-3,-6.529911E-2,-1.1599367E-3,6.519381E-2,4.3444466E-2,-2.745755E-3,2.424189E-3,7.0940354E-3,-2.5433872E-3,9.325106E-4,1.3196211E-3,-3.1090914E-3,-4.4090957E-3,-1.0613119E-2,-3.6048897E-2,1.5763143E-2,-1.1187796E-3,6.706382E-2,7.248702E-3,1.537885E-4,1.026793E-1,3.6721017E-2,1.1499044E-1,1.8928207E-3,-5.172863E-5,-2.879527E-3,-2.4938239E-3,-5.6707766E-3,-4.5329537E-3,9.1259036E-4,-1.2672386E-3,-3.8949156E-3,-3.1517833E-3,1.6754561E-3,-1.8356799E-3,1.2716099E-3,4.4266082E-4,-4.026241E-3,4.5792023E-3,-1.3520306E-3,2.477007E-3,1.7522495E-4,-3.183424E-5,-3.7510216E-3,1.4656065E-3,-1.9369116E-3,7.3491996E-3,1.7981133E-3,2.4840212E-4,5.6303227E-3,4.980934E-4,3.1788892E-3,9.120172E-3,4.500747E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,31,33,35,37,-1,39,41,43,45,47,49,-1,-1,-1,-1,51,53,-1,-1,55,57,59,-1,-1,-1,61,-1,63,-1,-1,-1,65,67,69,-1,71,73,75,77,79,81,83,-1,85,87,89,91,93,95,97,-1,99,-1,-1,-1,101,-1,103,105,107,109,111,-1,113,115,117,119,121,123,-1,125,-1,-1,-1,-1,127,129,-1,-1,131,-1,-1,-1,-1,-1,-1,-1,133,135,137,139,141,-1,-1,-1,143,145,147,-1,149,151,-1,153,-1,-1,-1,155,-1,-1,157,159,-1,161,163,165,167,169,-1,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,175,177,-1,179,-1,-1,181,183,185,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4996736E0,4.7296107E-1,3.9157927E-1,4.2343718E-1,2.8870562E-1,4.0685713E-2,2.7645004E-1,1.467008E-1,5.553214E-1,8.726418E-2,1.6303253E-1,0E0,0E0,3.609448E-1,3.9281845E-1,1.4925438E-1,1.680354E-2,0E0,1.4330113E-1,4.913953E-2,2.7211338E-2,1.05611645E-1,0E0,1.2261045E-1,4.1811293E-1,1.6696012E-1,1.3714084E-1,4.2692162E-2,9.2755556E-2,0E0,0E0,0E0,0E0,4.460047E-2,5.065822E-2,0E0,0E0,6.3475035E-2,1.4020173E-1,8.032316E-2,0E0,0E0,0E0,1.4742768E-1,0E0,8.954436E-2,0E0,0E0,0E0,1.4578718E-1,6.096653E-2,1.8501371E-2,0E0,2.7617157E-2,3.5549108E-2,8.64162E-2,7.270843E-2,1.4541936E-1,1.20852254E-1,9.1783404E-2,0E0,8.082855E-2,6.0157947E-2,3.3983726E-2,2.408287E-2,5.4355353E-2,2.8152108E-2,6.916009E-2,0E0,1.5597679E-2,0E0,0E0,0E0,2.8434437E-2,0E0,2.7761877E-2,7.553158E-2,6.156738E-2,3.7395038E-2,3.3378836E-2,0E0,1.0312788E-1,9.913029E-2,9.0156436E-2,1.0682958E-1,3.7520826E-2,2.3178458E-2,0E0,3.314446E-2,0E0,0E0,0E0,0E0,7.347754E-2,1.0141474E-1,0E0,0E0,4.242766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.0555954E-2,5.7853438E-2,4.8326865E-2,2.0629838E-2,2.6002686E-2,0E0,0E0,0E0,2.2556836E-2,1.5714139E-2,6.4449176E-2,0E0,3.2273136E-2,8.043331E-2,0E0,7.544774E-2,0E0,0E0,0E0,2.1066189E-2,0E0,0E0,2.7019009E-2,5.658865E-2,0E0,3.5709593E-2,3.879094E-2,3.5091456E-2,2.3674473E-2,4.5012444E-2,0E0,5.417341E-2,3.125538E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.539129E-2,5.47331E-2,0E0,6.372522E-2,0E0,0E0,4.4162273E-2,5.9186332E-2,3.000325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,33,33,34,34,37,37,38,38,39,39,43,43,45,45,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,73,73,75,75,76,76,77,77,78,78,79,79,81,81,82,82,83,83,84,84,85,85,86,86,88,88,93,93,94,94,97,97,105,105,106,106,107,107,108,108,109,109,113,113,114,114,115,115,117,117,118,118,120,120,124,124,127,127,128,128,130,130,131,131,132,132,133,133,134,134,136,136,137,137,147,147,148,148,150,150,153,153,154,154,155,155],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,32,34,36,38,-1,40,42,44,46,48,50,-1,-1,-1,-1,52,54,-1,-1,56,58,60,-1,-1,-1,62,-1,64,-1,-1,-1,66,68,70,-1,72,74,76,78,80,82,84,-1,86,88,90,92,94,96,98,-1,100,-1,-1,-1,102,-1,104,106,108,110,112,-1,114,116,118,120,122,124,-1,126,-1,-1,-1,-1,128,130,-1,-1,132,-1,-1,-1,-1,-1,-1,-1,134,136,138,140,142,-1,-1,-1,144,146,148,-1,150,152,-1,154,-1,-1,-1,156,-1,-1,158,160,-1,162,164,166,168,170,-1,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,176,178,-1,180,-1,-1,182,184,186,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1E0,1E0,9.369565E0,5.02E2,1.4E1,1.3627213E3,9.454259E10,1.654E3,1.706875E1,7.82261E6,-6.736836E-4,-5.850275E-3,3.3386944E5,1.8545505E2,7E0,8.238114E2,9.752715E-4,8.01E2,1.631108E6,2.1478824E2,9E0,6.5471455E-3,1.0428572E1,1.52179E5,6.5026445E3,8.127911E7,1.9E1,3.168978E-2,3.6359744E-3,5.573894E-4,-2.163606E-2,-7.334285E-3,1.6496063E2,2.6086957E0,-6.461285E-3,-2.5178462E-3,1.8E7,4.273336E-3,2E1,-3.5424551E-3,2.4684942E-3,-1.6008442E-2,1.2820834E7,1.2305972E-2,2.8181964E-1,-7.091179E-3,-1.17766685E-4,6.256841E-3,2.4E1,1.559733E6,1.5365228E5,-2.7579462E-4,4.0705118E6,6.763314E7,2.63E2,1E0,5.72202E2,4.727453E7,3.596E3,-3.1338425E-3,1.6666666E0,1.762852E7,1.8421053E0,7.184408E-1,7.24E2,3.8E1,1.3745962E0,1.3905023E-3,1.9666272E5,-4.966688E-3,-4.166776E-3,3.0381273E-4,2.3412812E0,3.3882523E-3,4.0351807E2,7.058982E-1,1.24147E5,1.1046409E7,2.732439E2,6.4513925E-3,3.14403E5,1.1251582E10,5.928425E2,1.7302156E5,1.7265024E3,1E0,-2.1872087E-3,3.3765998E0,7.913354E-4,-4.7547147E-3,5.4156943E-3,1.0624671E-3,2E0,1.5132743E0,-1.0283447E-2,-3.8761199E-3,1E2,1.4309211E-3,-3.989234E-3,-4.1560328E-4,9.28315E-4,-2.2553199E-3,1.5722326E-3,5.8921976E-3,3.3358974E0,8.15736E7,5.6818056E7,1.2E1,5.217E4,1.9301457E-3,1.375967E-3,-3.0071088E-3,1.16E2,2.75E0,6.0875E4,-4.8969863E-3,2.1924414E2,1.2979348E3,-3.12618E-3,1.552356E0,4.7856667E-3,1.1263395E-2,9.045327E-3,8.847283E8,6.038466E-3,1.2777855E-3,6.843911E6,2.0007E5,-1.0370394E-2,5E0,6.81531E-2,4.9390244E0,2.6043478E2,2.0162934E5,-1.1599367E-3,3.588785E0,1.14E3,-2.745755E-3,2.424189E-3,7.0940354E-3,-2.5433872E-3,9.325106E-4,1.3196211E-3,-3.1090914E-3,-4.4090957E-3,-1.0613119E-2,8E1,4.7220547E1,-1.1187796E-3,3.1157124E-1,7.248702E-3,1.537885E-4,2.5454E4,7.177158E6,1.3358E4,1.8928207E-3,-5.172863E-5,-2.879527E-3,-2.4938239E-3,-5.6707766E-3,-4.5329537E-3,9.1259036E-4,-1.2672386E-3,-3.8949156E-3,-3.1517833E-3,1.6754561E-3,-1.8356799E-3,1.2716099E-3,4.4266082E-4,-4.026241E-3,4.5792023E-3,-1.3520306E-3,2.477007E-3,1.7522495E-4,-3.183424E-5,-3.7510216E-3,1.4656065E-3,-1.9369116E-3,7.3491996E-3,1.7981133E-3,2.4840212E-4,5.6303227E-3,4.980934E-4,3.1788892E-3,9.120172E-3,4.500747E-3],"split_indices":[2,29,17,69,2,3,67,46,1,73,43,0,0,48,73,3,48,0,2,5,4,8,0,68,1,4,59,0,53,0,0,0,0,4,69,0,0,5,53,6,0,0,0,9,0,49,0,0,0,3,9,43,0,43,7,44,30,48,7,2,0,69,62,68,49,10,3,53,0,60,0,0,0,68,0,4,53,1,60,67,0,9,46,67,43,4,13,0,53,0,0,0,0,32,68,0,0,6,0,0,0,0,0,0,0,69,7,12,3,12,0,0,0,0,71,1,0,67,4,0,69,0,0,0,7,0,0,60,5,0,8,53,71,67,43,0,71,10,0,0,0,0,0,0,0,0,0,0,73,0,53,0,0,9,60,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.013E3,7.1E2,3.03E2,2.5E2,4.6E2,1.5E1,2.88E2,2.35E2,1.5E1,9.7E1,3.63E2,5E0,1E1,1.78E2,1.1E2,2.2E2,1.5E1,6E0,9E0,7.7E1,2E1,3.55E2,8E0,1.69E2,9E0,8.6E1,2.4E1,8E0,2.12E2,7E0,8E0,5E0,4E0,3.5E1,4.2E1,1.2E1,8E0,2.09E2,1.46E2,1.63E2,6E0,5E0,4E0,7.7E1,9E0,2E1,4E0,4E0,4E0,1.2E2,9.2E1,2.5E1,1E1,1.3E1,2.9E1,8.8E1,1.21E2,2.3E1,1.23E2,1.59E2,4E0,5.9E1,1.8E1,8E0,1.2E1,1.09E2,1.1E1,8E1,1.2E1,1E1,1.5E1,9E0,4E0,2.3E1,6E0,1.6E1,7.2E1,9.2E1,2.9E1,1.3E1,1E1,1.6E1,1.07E2,5.8E1,1.01E2,1.6E1,4.3E1,5E0,1.3E1,4E0,4E0,7E0,5E0,9.5E1,1.4E1,7E0,4E0,7E1,1E1,4E0,6E0,1.4E1,9E0,1.2E1,4E0,4.5E1,2.7E1,8.1E1,1.1E1,2E1,9E0,7E0,6E0,8E0,8E0,9.7E1,1E1,3E1,2.8E1,7E0,9.4E1,5E0,1.1E1,4E0,3.9E1,5E0,8E0,3.5E1,6E1,5E0,9E0,5.8E1,1.2E1,1.9E1,2.6E1,1.2E1,1.5E1,7.7E1,4E0,5E0,6E0,1.4E1,6E0,4E0,4E0,4E0,4E0,4E1,5.7E1,4E0,2.6E1,2.4E1,4E0,2.3E1,7.1E1,3.3E1,6E0,2.6E1,9E0,4.2E1,1.8E1,5E0,4E0,1.9E1,3.9E1,4E0,8E0,9E0,1E1,6E0,2E1,1.1E1,4E0,6.1E1,1.6E1,2.3E1,1.7E1,4.5E1,1.2E1,5E0,2.1E1,4E0,1.9E1,4E1,3.1E1,4E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"187","size_leaf_vector":"1"}},{"base_weights":[1.423794E-3,-5.0470393E-2,2.6536688E-2,-4.087611E-2,-1.5317702E-1,1.2591555E-2,1.1108404E-1,-8.011143E-2,-2.2470837E-2,-1.158281E-1,-1.3943634E-2,2.1700311E-2,-1.2029618E-1,1.2241098E-1,-5.627696E-3,-6.958135E-2,-1.2009375E-2,-1.1587812E-2,-5.7770766E-2,-7.2058383E-4,-1.4138368E-1,1.3850019E-2,8.25383E-2,-8.737344E-2,-1.478885E-2,9.8302804E-2,1.1615753E-2,-1.4650612E-1,-5.2909844E-2,2.0234497E-2,-2.6525741E-2,-2.550336E-2,-1.1192556E-1,-8.094936E-3,-3.8955284E-3,8.154739E-3,1.0790355E-2,1.0820098E-2,1.04724824E-1,-1.111358E-3,-4.6237097E-3,5.5580497E-2,1.286719E-1,-3.4692339E-3,-7.952699E-3,-6.1069567E-2,2.5494979E-3,1.702953E-3,6.182404E-2,-5.810492E-2,-8.361054E-3,-6.5425164E-3,6.7385836E-3,-6.0565653E-3,-1.656692E-3,-1.1868992E-2,2.1842804E-2,2.7411855E-3,-1.6625805E-2,7.854613E-2,7.5161792E-3,-1.8854549E-3,7.300622E-2,1.0140893E-2,9.734331E-2,-9.645782E-2,-2.5936764E-2,-2.9446915E-2,3.2763194E-2,4.7156163E-3,4.0735537E-4,5.032486E-4,-8.1467345E-2,3.6263962E-3,-1.4718059E-2,2.7741652E-2,-3.1366963E-3,-1.6080143E-2,4.536533E-3,1.0692011E-1,1.5913406E-2,-2.6327455E-3,1.2245688E-3,5.2073663E-3,4.0012702E-2,1.3121872E-1,4.1809496E-2,7.0301397E-3,7.624368E-2,1.1526459E-3,-1.2662338E-1,3.2327892E-3,-7.617263E-2,-4.1114476E-5,-4.2120614E-3,2.458413E-3,-9.431402E-4,-2.3647617E-3,-6.7651845E-3,-2.1264525E-2,3.266782E-3,2.0308178E-3,-1.5219009E-3,-6.8575954E-3,-8.124784E-2,1.862808E-5,1.319592E-1,6.750579E-2,7.871137E-3,8.38086E-4,4.308135E-3,3.1677843E-3,8.323019E-3,1.1964786E-2,3.3929597E-3,8.863961E-2,6.9045724E-5,-1.4481018E-1,-4.78835E-4,-2.9059453E-3,3.244578E-2,-5.766223E-3,-1.1129848E-3,-4.5741063E-2,9.578449E-4,-2.897477E-4,-5.906653E-3,-2.2751654E-2,-6.0806843E-3,7.307735E-3,1.7276135E-3,-1.7481922E-5,9.903697E-2,-6.351451E-2,1.8653069E-2,-1.6403464E-3,1.854359E-3,3.013373E-3,6.8379147E-3,-4.1314885E-3,-9.117624E-3,5.18E-3,7.723988E-5,-7.7845005E-4,-3.3960405E-3,-5.3273607E-4,2.3458803E-3,-2.8938593E-4,4.877667E-3,-2.8835372E-3,1.3580353E-3,5.1651024E-3,8.2982413E-4,-1.53484745E-2,-8.3737E-4,-7.1175564E-5,2.4650353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,-1,29,31,-1,33,35,37,39,-1,41,-1,43,45,47,49,51,53,-1,-1,-1,55,57,59,-1,-1,61,63,-1,-1,65,-1,67,69,71,73,-1,75,-1,-1,77,79,-1,81,83,-1,-1,85,-1,87,89,91,93,95,-1,-1,-1,97,-1,99,101,-1,103,-1,105,107,-1,-1,-1,109,111,113,-1,115,-1,117,119,121,-1,-1,-1,-1,-1,-1,123,-1,-1,-1,125,127,-1,129,131,133,-1,-1,-1,-1,135,-1,137,-1,139,-1,-1,141,-1,-1,143,145,147,-1,149,-1,-1,-1,-1,151,153,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3267074E0,3.2089025E-1,8.074118E-1,2.1845156E-1,1.20483816E-1,7.167713E-1,2.708E-1,1.6560113E-1,7.962212E-2,5.9269696E-2,0E0,2.629674E-1,2.2201258E-1,2.6454866E-1,0E0,1.11516416E-1,0E0,7.706961E-2,8.2769796E-2,0E0,2.1909356E-2,2.4246335E-1,9.911105E-2,2.4952143E-2,0E0,9.664345E-2,0E0,1.6362786E-2,7.216802E-2,3.9972074E-2,6.261303E-2,1.2075242E-1,2.1548346E-2,0E0,0E0,0E0,1.2122238E-1,2.229071E-2,6.00605E-2,0E0,0E0,6.1769046E-2,1.09532654E-1,0E0,0E0,8.825824E-2,0E0,3.67631E-2,3.3234306E-2,6.536582E-2,4.0357567E-2,0E0,4.2639554E-2,0E0,0E0,7.527511E-2,1.6224712E-1,0E0,2.0660046E-2,4.157366E-2,0E0,0E0,4.899499E-2,0E0,3.0044079E-2,1.3500276E-1,5.592984E-2,3.3409826E-2,2.1748917E-2,0E0,0E0,0E0,5.1523507E-2,0E0,3.8973965E-2,2.2113571E-2,0E0,9.199871E-2,0E0,5.5955574E-2,1.260192E-1,0E0,0E0,0E0,1.926822E-2,1.8306315E-2,1.8946722E-2,0E0,2.581188E-2,0E0,5.9816033E-2,5.033967E-2,3.245856E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.4311194E-2,0E0,0E0,0E0,1.08038686E-1,5.2989602E-2,0E0,3.6236137E-2,8.779669E-2,2.0514865E-1,0E0,0E0,0E0,0E0,1.7474303E-2,0E0,2.3707837E-2,0E0,5.510235E-2,0E0,0E0,4.3948423E-2,0E0,0E0,2.2828713E-2,2.174061E-2,8.2883425E-2,0E0,2.2518221E-2,0E0,0E0,0E0,0E0,2.6967585E-2,4.2085797E-1,1.6196641E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,17,17,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,41,41,42,42,45,45,47,47,48,48,49,49,50,50,52,52,55,55,56,56,58,58,59,59,62,62,64,64,65,65,66,66,67,67,68,68,72,72,74,74,75,75,77,77,79,79,80,80,84,84,85,85,86,86,88,88,90,90,91,91,92,92,99,99,103,103,104,104,106,106,107,107,108,108,113,113,115,115,117,117,120,120,123,123,124,124,125,125,127,127,132,132,133,133,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,-1,30,32,-1,34,36,38,40,-1,42,-1,44,46,48,50,52,54,-1,-1,-1,56,58,60,-1,-1,62,64,-1,-1,66,-1,68,70,72,74,-1,76,-1,-1,78,80,-1,82,84,-1,-1,86,-1,88,90,92,94,96,-1,-1,-1,98,-1,100,102,-1,104,-1,106,108,-1,-1,-1,110,112,114,-1,116,-1,118,120,122,-1,-1,-1,-1,-1,-1,124,-1,-1,-1,126,128,-1,130,132,134,-1,-1,-1,-1,136,-1,138,-1,140,-1,-1,142,-1,-1,144,146,148,-1,150,-1,-1,-1,-1,152,154,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.09E2,1.925673E1,1.5142261E3,8.147158E4,3.7377258E4,1E0,7.134432E7,1.2E1,2E0,1.375E0,-1.3943634E-2,4.5866325E6,3.7798166E0,6.5026445E3,-5.627696E-3,1.2571428E1,-1.2009375E-2,3.5364E4,1.89E2,-7.2058383E-4,2.5428572E0,2.04115E5,1.91E2,1.2E1,-1.478885E-2,1.5822886E0,1.1615753E-2,5.3E1,1.0428572E1,2.732381E2,1.5422421E2,1.43E2,1.162E3,-8.094936E-3,-3.8955284E-3,8.154739E-3,7.122041E7,2.0317461E0,3.201807E6,-1.111358E-3,-4.6237097E-3,1.1481482E0,2.5959E4,-3.4692339E-3,-7.952699E-3,7.7E1,2.5494979E-3,6.38E2,5.5177975E6,3.806E3,1.6379E2,-6.5425164E-3,3.4741783E0,-6.0565653E-3,-1.656692E-3,5E1,1.7039736E4,2.7411855E-3,3.6401E4,3.9914E4,7.5161792E-3,-1.8854549E-3,1.9514073E3,1.0140893E-2,1.6038339E0,1.2714286E1,2.5524476E0,4.878788E0,4.8064E4,4.7156163E-3,4.0735537E-4,5.032486E-4,1.703125E0,3.6263962E-3,3.7568388E0,2.1222334E2,-3.1366963E-3,9.6E1,4.536533E-3,1.2131076E2,3.3382E4,-2.6327455E-3,1.2245688E-3,5.2073663E-3,6.103178E8,1.599E3,1E0,7.0301397E-3,7.0993075E0,1.1526459E-3,2.1478824E2,5.4E1,9.876542E-2,-4.1114476E-5,-4.2120614E-3,2.458413E-3,-9.431402E-4,-2.3647617E-3,-6.7651845E-3,1.7594937E0,3.266782E-3,2.0308178E-3,-1.5219009E-3,2.511E3,1.2951E4,1.862808E-5,1.5E1,2.426015E-1,1E0,8.38086E-4,4.308135E-3,3.1677843E-3,8.323019E-3,7.3835544E5,3.3929597E-3,1.6327E4,6.9045724E-5,5.853E3,-4.78835E-4,-2.9059453E-3,9.2E1,-5.766223E-3,-1.1129848E-3,1.5357143E0,1E1,3.5E1,-5.906653E-3,1.2739792E1,-6.0806843E-3,7.307735E-3,1.7276135E-3,-1.7481922E-5,4.27E3,3.1157124E-1,1.1643481E6,-1.6403464E-3,1.854359E-3,3.013373E-3,6.8379147E-3,-4.1314885E-3,-9.117624E-3,5.18E-3,7.723988E-5,-7.7845005E-4,-3.3960405E-3,-5.3273607E-4,2.3458803E-3,-2.8938593E-4,4.877667E-3,-2.8835372E-3,1.3580353E-3,5.1651024E-3,8.2982413E-4,-1.53484745E-2,-8.3737E-4,-7.1175564E-5,2.4650353E-3],"split_indices":[2,73,67,43,48,83,59,8,32,68,0,43,68,4,0,70,0,9,10,0,68,5,0,3,0,53,0,44,68,4,67,2,10,0,0,0,7,69,9,0,0,68,9,0,0,44,0,44,60,1,67,0,68,0,0,3,62,0,9,9,0,0,70,0,57,67,68,73,44,0,0,0,68,0,68,4,0,8,0,48,9,0,0,0,7,10,28,0,50,0,4,2,53,0,0,0,0,0,0,71,0,0,0,2,44,0,3,53,8,0,0,0,0,43,0,2,0,9,0,0,44,0,0,68,8,3,0,71,0,0,0,0,2,53,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.016E3,3.31E2,6.85E2,3.04E2,2.7E1,5.89E2,9.6E1,9.6E1,2.08E2,2.3E1,4E0,5.52E2,3.7E1,9.2E1,4E0,9.2E1,4E0,1.6E2,4.8E1,5E0,1.8E1,4.9E2,6.2E1,3.3E1,4E0,7.9E1,1.3E1,1.5E1,7.7E1,5.1E1,1.09E2,3.1E1,1.7E1,1E1,8E0,8E0,4.82E2,1.5E1,4.7E1,6E0,2.7E1,3.4E1,4.5E1,5E0,1E1,7.2E1,5E0,3.6E1,1.5E1,3.9E1,7E1,6E0,2.5E1,1.3E1,4E0,1.58E2,3.24E2,5E0,1E1,3.4E1,1.3E1,5E0,2.9E1,1E1,3.5E1,3.5E1,3.7E1,1.8E1,1.8E1,8E0,7E0,1E1,2.9E1,4E0,6.6E1,2E1,5E0,1.53E2,5E0,2E1,3.04E2,5E0,5E0,1.7E1,1.7E1,9E0,2E1,8E0,2.7E1,7E0,2.8E1,2.4E1,1.3E1,1.3E1,5E0,1.3E1,5E0,2.1E1,8E0,6.2E1,4E0,1.6E1,4E0,1.35E2,1.8E1,4E0,1.6E1,4E1,2.64E2,1.3E1,4E0,5E0,4E0,1.1E1,9E0,2.3E1,4E0,2.4E1,4E0,7E0,1.7E1,6E0,7E0,2.9E1,3.3E1,1.29E2,6E0,9E0,9E0,1.2E1,4E0,1.3E1,2.7E1,3.4E1,2.3E2,4E0,7E0,1.8E1,5E0,1.3E1,1.1E1,4E0,1.3E1,1.5E1,1.4E1,2.7E1,6E0,1.23E2,6E0,5E0,4E0,2.3E1,4E0,4E0,3E1,1.46E2,8.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-2.3690518E-3,-4.6454236E-2,2.4270676E-2,-3.9487075E-2,-1.7790103E-1,1.2806815E-2,9.6692406E-2,-6.5399684E-2,-1.0367591E-2,-1.0324124E-2,-3.8157013E-1,2.2095663E-2,-4.8716985E-2,2.1900548E-1,7.713679E-2,-5.4802455E-2,-1.7812215E-1,-1.51273925E-2,8.280719E-2,-3.3877736E-3,2.086152E-3,-2.3489974E-2,-8.240744E-3,4.320459E-2,-9.208643E-3,-3.247114E-2,-1.4176326E-1,4.8005264E-3,1.323014E-2,9.144171E-2,-3.214184E-2,-1.17009215E-1,-4.7070682E-2,-1.5294679E-2,-6.460483E-2,-4.4045746E-3,-6.2572256E-2,-1.9115207E-5,6.90522E-3,1.4513594E-2,7.912101E-2,-3.743456E-2,5.928565E-3,-1.01075694E-1,-4.4577237E-4,-1.600472E-3,-9.148606E-3,1.02016166E-1,-3.1666807E-3,-4.510205E-3,2.4424011E-3,-1.4236003E-1,-5.767855E-4,-2.887059E-2,-7.6194726E-2,-3.0261488E-4,-4.4740615E-3,2.4722386E-2,-2.5758069E-2,-7.7478774E-2,6.9002045E-4,2.3605159E-2,-9.570162E-2,1.0650416E-1,3.300488E-2,-7.3764555E-2,-1.34646855E-2,1.05447076E-1,-4.812503E-3,-6.7789786E-2,-8.964123E-3,-3.090284E-2,8.842449E-2,1.1827256E-1,4.2806864E-2,-7.7682324E-3,-2.4023156E-3,-6.4046825E-3,-6.521758E-2,-1.3017711E-2,-5.7793178E-2,3.5074838E-2,-2.2289632E-3,2.9889024E-3,-3.2385856E-2,-1.9217745E-3,-5.2168015E-3,3.1509407E-2,-8.345172E-2,-1.2850322E-2,7.5271144E-4,7.9006106E-2,1.4786023E-1,-4.6273023E-2,6.134398E-2,-1.7573074E-3,-5.4621366E-3,-3.3920713E-2,3.7924357E-2,1.4922781E-1,2.5651226E-4,6.2303375E-3,-4.5400973E-2,-5.406925E-3,-3.0288946E-2,4.670073E-3,-4.595578E-3,7.395668E-4,6.3691614E-3,9.071165E-3,9.961424E-2,-4.0681256E-4,3.8774093E-3,-5.3135406E-2,1.0892414E-2,-7.025822E-3,-2.7026778E-2,-8.574363E-2,-2.4958434E-2,-1.1691692E-3,5.6429226E-2,-5.2172947E-3,-2.5322085E-2,4.2205244E-2,-3.798462E-2,-1.3420903E-3,-6.6029313E-3,3.759445E-2,1.22226916E-1,8.221263E-3,4.456291E-3,-5.069393E-3,6.9462217E-4,1.0558689E-1,1.6246017E-2,2.7053368E-3,-4.9223363E-2,-6.691448E-4,4.688343E-3,2.4992712E-3,9.910259E-3,-4.065815E-3,6.682679E-2,-4.6778736E-3,-6.907592E-4,9.347339E-4,-3.500155E-3,-1.4445928E-3,1.9964813E-3,5.2844435E-2,1.2996086E-1,-3.3647951E-3,-7.4026546E-5,5.5946827E-2,-6.9313142E-3,-4.1447632E-2,2.3322594E-3,1.15156574E-4,-1.0217599E-1,-3.503672E-2,1.4055921E-3,1.5535273E-3,-1.1627731E-3,5.110874E-3,4.086439E-2,-4.5741526E-3,-1.6006589E-2,4.6745043E-2,-3.5995934E-3,7.6965173E-3,-6.7384406E-3,6.0235422E-2,-1.2260028E-3,1.5047327E-1,-9.633956E-4,-3.4186582E-4,6.292773E-3,-1.760535E-3,5.5657398E-2,-8.316235E-2,9.3243754E-4,1.0042654E-2,-4.619761E-2,5.8167763E-3,3.131555E-4,1.087767E-4,7.070837E-2,8.7557705E-3,3.7370187E-3,6.297011E-4,4.4497224E-3,-2.2760548E-3,7.525329E-4,-3.0182349E-3,-2.3613703E-4,-6.422422E-3,-3.1489157E-3,-7.412458E-4,-4.9800836E-3,8.622268E-4,3.6478778E-3,-1.3873229E-3,3.78451E-4,3.99316E-3,1.4709192E-3,1.8237171E-3,-1.1570201E-3,4.405428E-3,9.0722676E-4,5.3884136E-3,9.9619245E-3,7.995641E-4,4.8918626E-3,-4.680241E-3,-1.2458394E-3,5.7625137E-5,4.3145483E-3,-3.5074928E-3,-2.730087E-4,4.252089E-3,6.424683E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,39,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,-1,-1,61,63,65,67,69,71,-1,-1,73,-1,-1,-1,75,-1,77,79,-1,-1,81,83,85,-1,87,89,91,93,95,97,99,101,103,-1,105,107,109,111,-1,-1,113,115,-1,117,119,-1,-1,121,-1,-1,123,125,-1,-1,127,129,131,133,-1,-1,135,137,139,-1,141,143,-1,145,147,-1,-1,-1,-1,149,-1,-1,151,153,-1,155,157,159,161,163,-1,165,167,169,-1,-1,171,173,-1,-1,-1,-1,175,177,-1,179,-1,-1,-1,-1,181,183,-1,-1,-1,-1,-1,-1,185,187,-1,-1,189,191,193,-1,-1,195,197,-1,-1,-1,-1,199,-1,201,203,-1,205,-1,207,-1,209,-1,-1,-1,-1,211,213,-1,215,217,-1,-1,-1,219,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2565796E0,3.6204702E-1,5.5252683E-1,2.890737E-1,6.7866534E-1,3.3082342E-1,2.0077026E-1,2.3273116E-1,8.1968464E-2,4.5257922E-2,1.4134037E-1,3.321197E-1,1.10279575E-1,6.328726E-2,1.2799692E-1,8.443338E-2,2.7375138E-1,8.8245615E-2,4.9989052E-2,0E0,0E0,0E0,0E0,3.0799502E-1,8.695603E-2,1.4496261E-1,5.9435293E-2,0E0,0E0,1.2710792E-1,6.0827993E-2,5.121824E-2,8.6850375E-2,0E0,1.9953046E-2,8.957742E-2,3.8129225E-2,0E0,0E0,1.7059988E-1,1.6443783E-1,6.0835384E-2,1.4267583E-1,5.1823854E-2,1.2724204E-1,0E0,0E0,5.9426606E-2,0E0,0E0,0E0,2.8538644E-2,0E0,8.532194E-2,2.2582644E-1,0E0,0E0,4.777235E-2,5.185471E-2,2.9637948E-2,0E0,1.3463739E-1,2.7016523E-1,8.270675E-2,1.1741844E-1,3.9984316E-2,4.7539912E-2,5.602649E-2,5.451651E-2,2.784992E-2,0E0,8.58385E-2,4.0755257E-2,5.619371E-2,3.469716E-2,0E0,0E0,5.392696E-2,1.2857936E-1,0E0,5.2924722E-2,4.1860282E-2,0E0,0E0,4.1518524E-2,0E0,0E0,1.1012448E-1,3.1436995E-2,0E0,0E0,8.8476926E-2,3.2333255E-2,5.6115575E-2,7.41047E-2,0E0,0E0,4.8167996E-2,4.4771757E-2,4.4289827E-2,0E0,6.01671E-2,4.270429E-2,0E0,2.6804287E-2,3.459743E-2,0E0,0E0,0E0,0E0,5.558172E-2,0E0,0E0,1.8154286E-2,3.973663E-2,0E0,3.4507282E-2,4.6652466E-2,1.7252829E-2,1.8383233E-2,2.4161115E-2,0E0,4.852505E-2,7.289794E-2,9.9676974E-2,0E0,0E0,4.225469E-2,1.0417509E-1,0E0,0E0,0E0,0E0,6.674807E-2,4.4963993E-2,0E0,6.763481E-2,0E0,0E0,0E0,0E0,4.9800407E-2,4.8813827E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.697053E-2,6.0661912E-2,0E0,0E0,2.2603411E-2,3.61915E-2,2.1362014E-2,0E0,0E0,2.436024E-2,3.2816865E-2,0E0,0E0,0E0,0E0,2.2207748E-2,0E0,2.264145E-2,7.001647E-2,0E0,1.6595505E-2,0E0,2.8071135E-2,0E0,2.1151334E-2,0E0,0E0,0E0,0E0,2.1283206E-2,1.664035E-2,0E0,4.6066996E-2,2.4699166E-2,0E0,0E0,0E0,1.5682295E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,47,47,51,51,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,77,77,78,78,80,80,81,81,84,84,87,87,88,88,91,91,92,92,93,93,94,94,97,97,98,98,99,99,101,101,102,102,104,104,105,105,110,110,113,113,114,114,116,116,117,117,118,118,119,119,120,120,122,122,123,123,124,124,127,127,128,128,133,133,134,134,136,136,141,141,142,142,149,149,150,150,153,153,154,154,155,155,158,158,159,159,164,164,166,166,167,167,169,169,171,171,173,173,178,178,179,179,181,181,182,182,186,186],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,40,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,-1,-1,62,64,66,68,70,72,-1,-1,74,-1,-1,-1,76,-1,78,80,-1,-1,82,84,86,-1,88,90,92,94,96,98,100,102,104,-1,106,108,110,112,-1,-1,114,116,-1,118,120,-1,-1,122,-1,-1,124,126,-1,-1,128,130,132,134,-1,-1,136,138,140,-1,142,144,-1,146,148,-1,-1,-1,-1,150,-1,-1,152,154,-1,156,158,160,162,164,-1,166,168,170,-1,-1,172,174,-1,-1,-1,-1,176,178,-1,180,-1,-1,-1,-1,182,184,-1,-1,-1,-1,-1,-1,186,188,-1,-1,190,192,194,-1,-1,196,198,-1,-1,-1,-1,200,-1,202,204,-1,206,-1,208,-1,210,-1,-1,-1,-1,212,214,-1,216,218,-1,-1,-1,220,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.01E2,9.639872E0,1.6466942E3,1.5422421E2,3.89E2,1E0,8.025477E0,6.9E2,1.5512196E1,1.317484E1,1.2916E4,8E0,2.614094E0,1.400229E6,3.206931E2,7.785714E0,1.5077367E3,8E0,6E0,-3.3877736E-3,2.086152E-3,-2.3489974E-2,-8.240744E-3,4.716184E2,6.953366E7,1.4598765E0,3.3512E4,4.8005264E-3,1.323014E-2,1.625E0,4.4615383E0,5.0363636E1,3.53969E5,-1.5294679E-2,3.8795E4,1.4392524E0,7.31E2,-1.9115207E-5,6.90522E-3,4.5620965E6,1.6485592E0,8.315271E0,3.168978E-2,2.060658E5,2.10377E5,-1.600472E-3,-9.148606E-3,1.3948464E7,-3.1666807E-3,-4.510205E-3,2.4424011E-3,1.4806053E4,-5.767855E-4,3.5318289E3,1.76421E6,-3.0261488E-4,-4.4740615E-3,3.0923106E-10,1.556431E6,5.524E3,6.9002045E-4,1.2300693E0,1.065548E0,1.0378262E10,1E0,6.185E3,1.935E3,5.1212654E0,5.1034E4,1.7851852E1,-8.964123E-3,8.625455E2,1.1772152E0,1.3208092E0,9.837502E1,-7.7682324E-3,-2.4023156E-3,1.0891155E4,1.2139681E3,-1.3017711E-2,2.240836E6,5.14E2,-2.2289632E-3,2.9889024E-3,1.952653E2,-1.9217745E-3,-5.2168015E-3,1E0,1.7544615E0,-1.2850322E-2,7.5271144E-4,6.4610266E2,1.6521566E5,6.9508715E0,4.7E3,-1.7573074E-3,-5.4621366E-3,1.06E2,6.0147805E0,2.6043478E2,2.5651226E-4,5.8165E4,1.9505818E8,-5.406925E-3,3.6451373E8,1.317E3,-4.595578E-3,7.395668E-4,6.3691614E-3,9.071165E-3,2.337765E3,-4.0681256E-4,3.8774093E-3,3.2E1,1.870878E6,-7.025822E-3,6E0,3.466E3,2.022131E6,8.6875E0,1.6877083E2,-5.2172947E-3,2.9726138E5,5.857143E0,3.3272727E0,-1.3420903E-3,-6.6029313E-3,2.297E3,1.4523518E8,8.221263E-3,4.456291E-3,-5.069393E-3,6.9462217E-4,4E0,9.812221E9,2.7053368E-3,2.558E3,-6.691448E-4,4.688343E-3,2.4992712E-3,9.910259E-3,6.627033E7,7.973882E7,-4.6778736E-3,-6.907592E-4,9.347339E-4,-3.500155E-3,-1.4445928E-3,1.9964813E-3,9.643839E6,1.4676277E7,-3.3647951E-3,-7.4026546E-5,5.4E1,1.43E2,5.6516E4,2.3322594E-3,1.15156574E-4,1.178E4,1.3935602E1,1.4055921E-3,1.5535273E-3,-1.1627731E-3,5.110874E-3,1.4E1,-4.5741526E-3,1.084437E6,3.0989232E-3,-3.5995934E-3,4.0903897E2,-6.7384406E-3,1.2526851E6,-1.2260028E-3,2.4545455E0,-9.633956E-4,-3.4186582E-4,6.292773E-3,-1.760535E-3,1.6438356E1,2.0130434E1,9.3243754E-4,6.63651E4,3.8104E4,5.8167763E-3,3.131555E-4,1.087767E-4,1.1863237E2,8.7557705E-3,3.7370187E-3,6.297011E-4,4.4497224E-3,-2.2760548E-3,7.525329E-4,-3.0182349E-3,-2.3613703E-4,-6.422422E-3,-3.1489157E-3,-7.412458E-4,-4.9800836E-3,8.622268E-4,3.6478778E-3,-1.3873229E-3,3.78451E-4,3.99316E-3,1.4709192E-3,1.8237171E-3,-1.1570201E-3,4.405428E-3,9.0722676E-4,5.3884136E-3,9.9619245E-3,7.995641E-4,4.8918626E-3,-4.680241E-3,-1.2458394E-3,5.7625137E-5,4.3145483E-3,-3.5074928E-3,-2.730087E-4,4.252089E-3,6.424683E-4],"split_indices":[2,69,67,67,2,6,71,2,71,69,1,32,68,1,73,67,48,32,8,0,0,0,0,70,7,69,1,0,0,61,69,62,9,0,1,68,2,0,0,62,53,71,53,48,1,0,0,9,0,0,0,43,0,62,46,0,0,52,5,44,0,53,53,46,27,44,2,68,44,73,0,67,68,68,73,0,0,43,48,0,9,2,0,0,70,0,0,19,68,0,0,70,48,71,2,0,0,0,69,67,0,10,5,0,7,2,0,0,0,0,67,0,0,4,46,0,8,1,12,73,67,0,43,68,69,0,0,2,12,0,0,0,0,3,46,0,10,0,0,0,0,12,47,0,0,0,0,0,0,62,62,0,0,2,2,9,0,0,1,73,0,0,0,0,3,0,9,53,0,67,0,43,0,68,0,0,0,0,71,71,0,48,10,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.068E3,4.02E2,6.66E2,3.83E2,1.9E1,5.76E2,9E1,2.02E2,1.81E2,1.1E1,8E0,5.01E2,7.5E1,1.1E1,7.9E1,1.86E2,1.6E1,1.73E2,8E0,5E0,6E0,4E0,4E0,2.99E2,2.02E2,6.5E1,1E1,5E0,6E0,7E1,9E0,1.9E1,1.67E2,6E0,1E1,1.42E2,3.1E1,4E0,4E0,1.67E2,1.32E2,7E1,1.32E2,2E1,4.5E1,4E0,6E0,6.6E1,4E0,5E0,4E0,1.5E1,4E0,1.04E2,6.3E1,4E0,6E0,6E1,8.2E1,2.6E1,5E0,1.55E2,1.2E1,8.2E1,5E1,2.7E1,4.3E1,1.2E1,1.2E2,1.6E1,4E0,3.4E1,1.1E1,5.1E1,1.5E1,1.1E1,4E0,6.5E1,3.9E1,4E0,5.9E1,5.3E1,7E0,5E0,7.7E1,1.4E1,1.2E1,1.45E2,1E1,4E0,8E0,5.1E1,3.1E1,1.3E1,3.7E1,1.6E1,1.1E1,3.1E1,1.2E1,8E0,4E0,9.5E1,2.5E1,6E0,1E1,2.3E1,1.1E1,5E0,6E0,8E0,4.3E1,7E0,8E0,1.7E1,4.8E1,1.1E1,2.8E1,3.1E1,2.8E1,2E1,3.3E1,5E0,7.2E1,1.26E2,1.9E1,6E0,4E0,2.7E1,2.4E1,1.8E1,1.3E1,6E0,7E0,1.8E1,1.9E1,4E0,2.7E1,7E0,5E0,4E0,4E0,8.2E1,1.3E1,8E0,1.7E1,5E0,5E0,1.2E1,1.1E1,1.8E1,2.5E1,1.2E1,5E0,1.3E1,3.5E1,2.4E1,4E0,5E0,2.6E1,2.4E1,4E0,8E0,1.2E1,6E0,2.7E1,7E0,6.5E1,1.22E2,4E0,1.4E1,5E0,2E1,7E0,2E1,4E0,4E0,1.4E1,8E0,1.1E1,1.8E1,9E0,6.2E1,2E1,6E0,7E0,5E0,1.3E1,1E1,1.5E1,7E0,6E0,1.2E1,2.3E1,1.4E1,1E1,1.1E1,1.5E1,2E1,4E0,1.8E1,9E0,4.1E1,2.4E1,3.2E1,9E1,7E0,7E0,1E1,1E1,1.5E1,5E0,7E0,4E0,1.3E1,5E0,5.7E1,5E0,1.1E1,9E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"221","size_leaf_vector":"1"}},{"base_weights":[3.1734907E-4,-2.5441412E-2,4.4444777E-2,-1.930243E-2,-1.1689011E-1,5.1820323E-2,-8.1651054E-2,-5.8866374E-2,-8.555098E-3,-2.7244818E-1,-2.264929E-2,3.028332E-2,8.010094E-2,-1.4328314E-1,-2.3486719E-2,-4.4790212E-2,-1.3263816E-1,-1.4765708E-2,3.70305E-2,6.172962E-4,-3.7759373E-1,-4.6482105E-2,1.6994231E-3,3.5040934E-2,-7.584718E-3,8.7630995E-2,-5.4392726E-3,-8.991346E-3,-2.4494752E-3,1.3180748E-3,-3.707882E-3,-6.523228E-3,-3.1407785E-2,-9.862959E-3,-9.145552E-2,-2.7298667E-2,1.1616464E-2,5.6367002E-2,-1.1782863E-2,-2.0760357E-2,-9.313337E-3,-4.9811127E-3,-1.7176581E-3,7.7325464E-3,6.706582E-2,1.6933034E-1,7.343165E-2,4.8997873E-3,-3.846071E-2,-2.1971301E-3,-5.725041E-3,-1.9333543E-2,-7.130554E-2,1.8790262E-2,-9.090453E-2,8.776974E-2,2.5256924E-2,1.4223348E-3,-5.5488846E-3,2.6358298E-3,-1.7664249E-3,2.3178311E-2,-1.1952087E-2,1.0394326E-2,5.3271286E-2,8.88285E-3,3.6925517E-3,6.51994E-2,1.0560282E-2,-4.358769E-2,1.4208389E-3,-2.5222566E-2,5.9746668E-2,-9.945379E-3,-5.1019914E-2,2.466904E-2,-3.960379E-2,-7.2662435E-3,-2.9322098E-4,5.6587663E-3,1.8851146E-3,2.463497E-3,-1.37553755E-2,7.4817287E-3,1.4445846E-2,4.443088E-3,7.0251875E-2,1.3766012E-1,5.296325E-2,-5.094637E-2,1.1885586E-2,-3.6514066E-3,-5.364996E-2,7.324804E-3,7.9231156E-4,1.415675E-3,-8.419614E-2,-1.2184613E-3,3.587004E-2,1.4462165E-3,-4.3761693E-3,-2.8119825E-3,2.1034083E-3,2.6040884E-2,-7.082199E-2,1.4593147E-3,-2.8696032E-3,7.726795E-2,-1.2283513E-3,4.527957E-3,9.850517E-3,-7.6740836E-3,6.893581E-2,-3.321744E-2,-8.0785654E-2,2.3800759E-3,-2.12408E-3,-3.5158604E-2,1.4836361E-2,-6.842919E-2,-5.845265E-3,-1.14167064E-1,-8.700106E-4,1.5398306E-2,-4.5961887E-2,6.464244E-2,1.8580774E-2,-4.6848338E-2,3.5051797E-2,-1.5942835E-3,-5.301727E-3,5.421723E-2,1.0862454E-1,5.122702E-3,-3.846949E-2,7.789783E-2,-2.7975119E-3,-4.4645816E-3,-1.0336055E-3,-4.408084E-3,-5.1937375E-4,-3.1197525E-3,-7.035668E-4,-9.4183546E-4,1.5935844E-3,-7.452188E-4,-3.982482E-3,-1.4736438E-3,1.5623561E-3,-1.8472258E-3,-6.0651773E-3,-5.055231E-4,2.2746518E-3,-4.2294003E-3,8.284892E-4,8.9585455E-4,3.8397154E-3,-6.036367E-4,1.4129496E-3,1.7207733E-4,-3.7506458E-3,2.0657287E-3,-1.007813E-3,1.0732767E-3,3.7291192E-3,6.259252E-3,1.1105543E-3,1.8018075E-3,-2.8517907E-3,4.9809343E-3,2.2626228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,47,-1,49,51,53,55,57,-1,-1,-1,59,61,63,65,67,-1,69,-1,-1,71,73,75,77,79,81,-1,-1,-1,-1,83,-1,-1,85,-1,-1,87,-1,89,-1,91,93,-1,95,97,99,-1,-1,-1,-1,-1,101,-1,103,105,107,109,111,113,115,117,119,-1,-1,-1,121,123,125,-1,-1,-1,-1,127,129,-1,-1,131,-1,-1,-1,133,135,137,139,-1,-1,141,143,145,147,149,-1,151,153,155,157,159,161,-1,-1,163,165,-1,167,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1503019E0,3.564796E-1,3.5113066E-1,2.546205E-1,5.8022606E-1,2.1257943E-1,7.191533E-2,1.2697732E-1,1.3445985E-1,4.5581985E-1,3.853836E-2,1.9263433E-1,2.3675478E-1,3.6021084E-2,3.7727233E-2,1.3619429E-1,4.706779E-2,1.3819677E-1,5.446531E-2,0E0,4.8367143E-2,5.1982917E-2,0E0,1.7264192E-1,0E0,1.5809727E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.812418E-2,0E0,1.548405E-2,9.781864E-2,1.00995876E-1,3.7837803E-2,8.209168E-2,0E0,0E0,0E0,2.7935635E-2,4.5045808E-1,1.8625724E-1,2.77071E-2,1.4730597E-1,0E0,3.4343094E-2,0E0,0E0,1.13741904E-1,1.145965E-1,4.4574272E-2,4.9515665E-2,2.7612E-2,2.482224E-2,0E0,0E0,0E0,0E0,1.22473866E-1,0E0,0E0,7.016404E-2,0E0,0E0,1.0087919E-1,0E0,3.611979E-2,0E0,1.3777676E-1,6.882788E-2,0E0,1.0824139E-1,3.367976E-2,4.98669E-2,0E0,0E0,0E0,0E0,0E0,3.0700976E-2,0E0,9.8735265E-2,4.3515056E-2,4.4859022E-2,3.003341E-2,1.0319358E-1,3.805095E-2,2.7593968E-2,7.566383E-2,6.81321E-2,0E0,0E0,0E0,5.270286E-2,2.7473414E-2,3.9092645E-2,0E0,0E0,0E0,0E0,5.8783587E-2,1.5185334E-2,0E0,0E0,3.6755413E-2,0E0,0E0,0E0,8.8239126E-2,1.02197796E-1,3.2226328E-2,2.741395E-2,0E0,0E0,2.9937278E-2,5.747947E-2,6.7394614E-2,2.587847E-2,1.8876106E-2,0E0,2.4603913E-2,3.0768512E-2,2.3462072E-2,2.015243E-2,1.7667213E-2,4.3831363E-2,0E0,0E0,2.7686536E-2,5.1270485E-2,0E0,3.664307E-2,6.307554E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,25,25,32,32,34,34,35,35,36,36,37,37,38,38,42,42,43,43,44,44,45,45,46,46,48,48,51,51,52,52,53,53,54,54,55,55,56,56,61,61,64,64,67,67,69,69,71,71,72,72,74,74,75,75,76,76,82,82,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,96,96,97,97,98,98,103,103,104,104,107,107,111,111,112,112,113,113,114,114,117,117,118,118,119,119,120,120,121,121,123,123,124,124,125,125,126,126,127,127,128,128,131,131,132,132,134,134,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,48,-1,50,52,54,56,58,-1,-1,-1,60,62,64,66,68,-1,70,-1,-1,72,74,76,78,80,82,-1,-1,-1,-1,84,-1,-1,86,-1,-1,88,-1,90,-1,92,94,-1,96,98,100,-1,-1,-1,-1,-1,102,-1,104,106,108,110,112,114,116,118,120,-1,-1,-1,122,124,126,-1,-1,-1,-1,128,130,-1,-1,132,-1,-1,-1,134,136,138,140,-1,-1,142,144,146,148,150,-1,152,154,156,158,160,162,-1,-1,164,166,-1,168,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.82E3,9.639872E0,4.217427E7,8.147158E4,2E0,1.2456025E3,1.2376862E10,5.17259E8,3.422592E6,8.317E3,1.6254545E1,1.5449402E7,4.1325716E7,4.5576923E1,2.5427E4,1.7857143E1,6.15E2,3.422351E6,1E0,6.172962E-4,5.75E2,6.76E2,1.6994231E-3,1.8062708E6,-7.584718E-3,9.318287E0,-5.4392726E-3,-8.991346E-3,-2.4494752E-3,1.3180748E-3,-3.707882E-3,-6.523228E-3,6E0,-9.862959E-3,1.5757076E9,2.87E2,1.6438356E1,2.1111E4,6.2521395E6,-2.0760357E-2,-9.313337E-3,-4.9811127E-3,3.37231E7,1.68991E6,1.1057693E0,5.1578946E0,8.363565E0,4.8997873E-3,1.3745962E0,-2.1971301E-3,-5.725041E-3,1.187E3,5.1942E4,5.4444447E0,1.23E2,1.67227E5,2.0487332E-3,1.4223348E-3,-5.5488846E-3,2.6358298E-3,-1.7664249E-3,4E0,-1.1952087E-2,1.0394326E-2,2.483E3,8.88285E-3,3.6925517E-3,1.524E3,1.0560282E-2,7.163082E7,1.4208389E-3,1E0,6.904904E3,-9.945379E-3,6E0,8.8790035E-1,5E0,-7.2662435E-3,-2.9322098E-4,5.6587663E-3,1.8851146E-3,2.463497E-3,1.3478261E1,7.4817287E-3,2.2E1,2.6844707E8,8.82E4,7.008608E7,1.0901037E10,2.04115E5,6.03E2,4.33E2,4E0,7.324804E-3,7.9231156E-4,1.415675E-3,1.034073E6,2.6950342E2,1E0,1.4462165E-3,-4.3761693E-3,-2.8119825E-3,2.1034083E-3,2.29E2,5.943433E4,1.4593147E-3,-2.8696032E-3,2.01E0,-1.2283513E-3,4.527957E-3,9.850517E-3,1.3358E4,1.514526E6,4.91271E5,3.257732E0,2.3800759E-3,-2.12408E-3,1.1430505E3,3.3832976E7,3.0052083E0,9E0,1.027972E0,-8.700106E-4,7.09E2,1.6153846E0,1.7733E4,2.9417648E2,2.6845297E5,2.831224E10,-1.5942835E-3,-5.301727E-3,6.9664386E2,5.8475E5,5.122702E-3,7.5456814E-4,9.4163445E1,-2.7975119E-3,-4.4645816E-3,-1.0336055E-3,-4.408084E-3,-5.1937375E-4,-3.1197525E-3,-7.035668E-4,-9.4183546E-4,1.5935844E-3,-7.452188E-4,-3.982482E-3,-1.4736438E-3,1.5623561E-3,-1.8472258E-3,-6.0651773E-3,-5.055231E-4,2.2746518E-3,-4.2294003E-3,8.284892E-4,8.9585455E-4,3.8397154E-3,-6.036367E-4,1.4129496E-3,1.7207733E-4,-3.7506458E-3,2.0657287E-3,-1.007813E-3,1.0732767E-3,3.7291192E-3,6.259252E-3,1.1105543E-3,1.8018075E-3,-2.8517907E-3,4.9809343E-3,2.2626228E-3],"split_indices":[2,69,60,43,32,70,46,46,43,9,69,62,66,73,9,4,2,9,30,0,2,2,0,43,0,71,0,0,0,0,0,0,3,0,46,0,71,9,43,0,0,0,5,43,69,69,50,0,53,0,0,10,1,69,0,1,53,0,0,0,0,73,0,0,2,0,0,44,0,7,0,26,48,0,8,71,8,0,0,0,0,0,73,0,3,7,10,5,46,5,10,2,6,0,0,0,9,67,26,0,0,0,0,10,48,0,0,68,0,0,0,9,10,46,68,0,0,48,7,69,8,68,0,2,68,1,4,43,46,0,0,67,1,0,53,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E3,6.38E2,3.72E2,5.99E2,3.9E1,3.52E2,2E1,1.27E2,4.72E2,1.4E1,2.5E1,2.01E2,1.51E2,9E0,1.1E1,1.08E2,1.9E1,4.16E2,5.6E1,4E0,1E1,1.8E1,7E0,1.97E2,4E0,1.46E2,5E0,5E0,4E0,6E0,5E0,1.2E1,9.6E1,5E0,1.4E1,2.82E2,1.34E2,4E1,1.6E1,6E0,4E0,7E0,1.1E1,1.07E2,9E1,2E1,1.26E2,4E0,9.2E1,7E0,7E0,2.4E2,4.2E1,1.26E2,8E0,1.9E1,2.1E1,1.2E1,4E0,4E0,7E0,1.02E2,5E0,6E0,8.4E1,1.5E1,5E0,1.21E2,5E0,8.6E1,6E0,2.24E2,1.6E1,4E0,3.8E1,1.15E2,1.1E1,4E0,4E0,1E1,9E0,1.2E1,9E0,5E0,9.7E1,2.2E1,6.2E1,1.6E1,1.05E2,7.6E1,1E1,1.28E2,9.6E1,4E0,1.2E1,1.1E1,2.7E1,3.5E1,8E1,5E0,6E0,5E0,4E0,8.6E1,1.1E1,1.6E1,6E0,5.8E1,4E0,1.2E1,4E0,2.2E1,8.3E1,4.9E1,2.7E1,6E0,4E0,4.7E1,8.1E1,7.3E1,2.3E1,1.8E1,9E0,2.6E1,9E0,2.9E1,5.1E1,9E0,7.7E1,7E0,4E0,3.5E1,2.3E1,4E0,1.8E1,7.8E1,5E0,6E0,4.3E1,2.2E1,5E0,1.7E1,3E1,2.9E1,5.2E1,1.9E1,5.4E1,1.4E1,9E0,4E0,1.4E1,1.5E1,1.1E1,5E0,4E0,9E0,2E1,1.4E1,3.7E1,4E0,5E0,6.6E1,1.1E1,1.7E1,1.8E1,1.7E1,6E0,4E0,1.4E1,3.7E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[2.8358696E-3,-2.1731937E-2,4.7262184E-2,-1.914325E-2,-1.0804873E-2,-1.07442856E-1,5.4537464E-2,-4.850239E-2,-4.782098E-3,-1.0055192E-2,-5.679092E-2,1.2139469E-2,4.8979595E-2,-4.3503724E-2,-1.5974875E-1,-3.4631126E-2,4.1432115E-3,2.755532E-4,-4.4701556E-3,3.7976187E-2,9.047904E-2,-3.45927E-2,-1.11122794E-1,-1.1393424E-2,-1.8910074E-3,-2.544626E-2,-8.264027E-3,1.8162321E-3,5.3703445E-3,6.246678E-2,1.574164E-2,1.1415972E-1,-6.775713E-2,-1.3162982E-1,-2.8366704E-2,-1.2663113E-2,-4.1569676E-2,2.0889657E-2,-4.246881E-2,-1.015258E-3,4.3923184E-3,1.04059055E-1,3.7159696E-2,-8.334721E-3,2.2537494E-2,1.2962312E-1,3.7638366E-2,-7.364217E-3,2.576431E-3,-8.075818E-3,-2.0724342E-3,6.308548E-2,-3.364144E-2,-8.0281794E-2,1.4198109E-3,-1.0233842E-2,1.9831716E-3,-6.898858E-2,-3.0415172E-3,-9.7341975E-3,3.1508435E-2,7.90356E-2,8.067667E-3,4.8366394E-2,-6.035828E-2,5.7871625E-2,-4.7866236E-3,9.6582316E-2,1.7765927E-1,-5.571634E-4,5.0647175E-3,7.6603927E-3,-1.5261787E-3,-5.3869475E-2,-1.464016E-2,-5.2134483E-3,-1.3355284E-3,-2.9182574E-3,1.6908598E-3,-4.7223542E-2,-1.0392852E-2,1.789234E-2,-4.146781E-3,-6.584291E-3,-5.6866505E-3,-2.8734442E-2,4.7082268E-2,9.949901E-2,1.976186E-2,1.5820231E-2,7.8003354E-2,-7.739879E-3,2.7205874E-3,1.496291E-2,1.1670819E-1,4.5652285E-2,-5.2709714E-2,4.4692498E-2,1.2049575E-1,9.118442E-3,3.6517426E-3,-4.1236207E-2,-1.503538E-1,8.923282E-2,-2.6142867E-2,-5.873108E-4,-2.8685401E-3,3.1307526E-3,-1.987494E-2,-5.7339385E-2,-1.0628119E-3,1.4283924E-3,-2.8482913E-3,3.0428344E-2,5.573799E-3,1.1780362E-1,1.2823052E-3,2.5670915E-3,-1.585374E-3,-1.5597975E-2,5.80315E-2,5.9674694E-3,4.745268E-2,-4.6728486E-3,4.6099126E-3,1.4337245E-1,1.7921563E-3,2.496411E-2,7.332925E-3,-3.266591E-2,-8.985747E-3,2.9854354E-4,3.7812009E-3,1.3690108E-1,1.7249534E-3,-1.3511529E-3,-4.6082414E-3,-9.808591E-3,-2.6945747E-3,6.3687614E-3,1.0516869E-3,-3.0566845E-3,-3.6080249E-4,1.376148E-3,-3.0977794E-3,-9.797035E-4,-9.130371E-3,-2.3079491E-3,2.611778E-4,-1.131352E-3,1.8780392E-3,1.5895964E-3,6.2102866E-3,-1.4444111E-3,2.0291891E-3,5.431144E-3,9.371942E-4,1.0213414E-3,6.905895E-3,-1.8031822E-3,1.8403545E-3,4.0617585E-3,9.6385805E-3,-8.982889E-4,2.262778E-3,1.116433E-3,-2.138018E-3,7.4168895E-3,2.904172E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,-1,39,-1,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,-1,65,67,69,-1,-1,-1,-1,71,73,75,-1,77,-1,79,81,83,85,87,-1,89,91,93,95,97,99,-1,-1,-1,-1,101,103,-1,-1,-1,-1,105,-1,107,-1,109,-1,111,113,115,117,119,121,-1,-1,123,125,127,129,131,133,-1,-1,135,137,139,141,-1,-1,-1,143,145,147,-1,-1,149,-1,151,-1,-1,-1,153,155,-1,157,159,-1,161,-1,163,-1,165,-1,-1,-1,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1372341E0,3.6402002E-1,4.2342842E-1,2.7973107E-1,0E0,8.25364E-2,3.979709E-1,1.1427647E-1,1.1922956E-1,0E0,3.3183564E-2,0E0,1.5501636E-1,1.22692555E-1,8.279735E-2,1.3191229E-1,8.978397E-2,0E0,0E0,1.4930898E-1,2.775293E-1,1.0924263E-1,2.6065782E-1,0E0,0E0,7.8185186E-2,0E0,8.9612134E-2,0E0,1.3402104E-1,1.9868553E-1,6.9667816E-2,1.22093916E-1,3.336823E-2,8.750458E-2,0E0,5.2848704E-2,1.8957775E-2,7.507357E-2,9.38855E-2,0E0,7.272667E-2,9.362622E-2,0E0,1.3737431E-1,6.537312E-2,4.4255488E-2,0E0,0E0,0E0,0E0,1.04219414E-1,6.37836E-2,1.6814038E-2,0E0,3.206031E-2,0E0,1.3534802E-1,5.6065947E-2,9.238241E-2,6.702777E-2,4.4125527E-2,0E0,7.1108654E-2,1.2488814E-1,1.5477902E-1,1.9819859E-1,3.556326E-2,2.3280442E-2,0E0,0E0,0E0,0E0,9.2531875E-2,1.0747701E-1,0E0,0E0,0E0,0E0,1.9221015E-2,0E0,4.8437238E-2,0E0,7.1401075E-2,0E0,3.136102E-2,6.493725E-2,3.350061E-2,2.3205133E-2,4.997466E-2,5.483286E-2,0E0,0E0,6.2090542E-2,4.845524E-2,8.960797E-2,1.13011725E-1,1.6515747E-2,2.4901718E-2,0E0,0E0,4.8295423E-2,3.893374E-2,2.5948353E-2,5.8179356E-2,0E0,0E0,0E0,3.7141807E-2,1.2253891E-1,7.613755E-2,0E0,0E0,2.7381256E-2,0E0,2.58694E-2,0E0,0E0,0E0,2.188846E-2,3.3565894E-2,0E0,6.3929886E-2,4.89602E-2,0E0,5.0415844E-2,0E0,3.827802E-2,0E0,3.0515894E-2,0E0,0E0,0E0,2.141428E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,27,27,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,51,51,52,52,53,53,55,55,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,73,73,74,74,79,79,81,81,83,83,85,85,86,86,87,87,88,88,89,89,90,90,93,93,94,94,95,95,96,96,97,97,98,98,101,101,102,102,103,103,104,104,108,108,109,109,110,110,113,113,115,115,119,119,120,120,122,122,123,123,125,125,127,127,129,129,133,133],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,-1,40,-1,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,-1,66,68,70,-1,-1,-1,-1,72,74,76,-1,78,-1,80,82,84,86,88,-1,90,92,94,96,98,100,-1,-1,-1,-1,102,104,-1,-1,-1,-1,106,-1,108,-1,110,-1,112,114,116,118,120,122,-1,-1,124,126,128,130,132,134,-1,-1,136,138,140,142,-1,-1,-1,144,146,148,-1,-1,150,-1,152,-1,-1,-1,154,156,-1,158,160,-1,162,-1,164,-1,166,-1,-1,-1,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,5.8E1,1E0,1.972052E5,-1.0804873E-2,1.6427984E1,3.3817584E7,3.0977E4,1E0,-1.0055192E-2,1E1,1.2139469E-2,6.7947706E8,4.6463413E0,3.3064186E7,3.7568388E0,3.7E1,2.755532E-4,-4.4701556E-3,1.1736916E0,3.206931E2,6E0,2E0,-1.1393424E-2,-1.8910074E-3,1E0,-8.264027E-3,6.663214E6,5.3703445E-3,6E0,1.0280637E0,7.3274844E9,6.9879E4,1.4806053E4,6E0,-1.2663113E-2,1.0428572E1,1.1E1,5E0,9.751103E6,4.3923184E-3,7.8114265E-1,2.3308511E5,-8.334721E-3,2.67637E5,1.3E1,1.2567214E7,-7.364217E-3,2.576431E-3,-8.075818E-3,-2.0724342E-3,1.3050649E4,2.135961E4,8.579633E-2,1.4198109E-3,7.54E2,1.9831716E-3,1.339646E6,2E0,9.750042E6,8.0509944E5,2.936348E10,8.067667E-3,1.3026532E6,9.525663E2,1.3758875E3,4.519E3,1.4176E4,2.8389828E7,-5.571634E-4,5.0647175E-3,7.6603927E-3,-1.5261787E-3,1.965084E4,1.813506E-5,-5.2134483E-3,-1.3355284E-3,-2.9182574E-3,1.6908598E-3,1.4452E4,-1.0392852E-2,6.346204E-7,-4.146781E-3,1.5933333E2,-5.6866505E-3,1.6121496E1,1E0,6.441311E-1,1E0,5.691698E2,2.0778275E-1,-7.739879E-3,2.7205874E-3,3.13173E0,1.016E4,6.2868685E6,2.8146256E5,1.1223777E7,5.0554064E7,9.118442E-3,3.6517426E-3,3.2856784E7,2E0,6.0921145E4,1.9595902E3,-5.873108E-4,-2.8685401E-3,3.1307526E-3,6.76E2,1.5222145E2,3.202146E5,1.4283924E-3,-2.8482913E-3,5.55E2,5.573799E-3,3.7253174E2,1.2823052E-3,2.5670915E-3,-1.585374E-3,6.4356956E0,2.0421052E0,5.9674694E-3,6.2850784E1,4.1032645E6,4.6099126E-3,5E0,1.7921563E-3,7.0093E4,7.332925E-3,8E0,-8.985747E-3,2.9854354E-4,3.7812009E-3,1E0,1.7249534E-3,-1.3511529E-3,-4.6082414E-3,-9.808591E-3,-2.6945747E-3,6.3687614E-3,1.0516869E-3,-3.0566845E-3,-3.6080249E-4,1.376148E-3,-3.0977794E-3,-9.797035E-4,-9.130371E-3,-2.3079491E-3,2.611778E-4,-1.131352E-3,1.8780392E-3,1.5895964E-3,6.2102866E-3,-1.4444111E-3,2.0291891E-3,5.431144E-3,9.371942E-4,1.0213414E-3,6.905895E-3,-1.8031822E-3,1.8403545E-3,4.0617585E-3,9.6385805E-3,-8.982889E-4,2.262778E-3,1.116433E-3,-2.138018E-3,7.4168895E-3,2.904172E-3],"split_indices":[2,3,17,43,0,73,7,44,29,0,6,0,7,68,5,68,3,0,0,53,73,67,32,0,0,68,0,43,0,8,69,5,2,43,3,0,68,3,8,9,0,53,48,0,9,3,58,0,0,0,0,43,43,53,0,2,0,9,32,9,60,46,0,43,4,70,44,2,62,0,0,0,0,43,53,0,0,0,0,9,0,52,0,4,0,73,28,53,8,67,57,0,0,53,2,60,48,59,59,0,0,60,10,60,48,0,0,0,2,4,43,0,0,44,0,4,0,0,0,69,68,0,73,60,0,8,0,1,0,3,0,0,0,19,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E3,6.7E2,3.7E2,6.63E2,7E0,1.6E1,3.54E2,2.17E2,4.46E2,4E0,1.2E1,8E0,3.46E2,2.09E2,8E0,1.02E2,3.44E2,5E0,7E0,2.75E2,7.1E1,1.86E2,2.3E1,4E0,4E0,9.7E1,5E0,3.38E2,6E0,1.3E2,1.45E2,6.2E1,9E0,1E1,1.76E2,6E0,1.7E1,2.6E1,7.1E1,3.29E2,9E0,4.8E1,8.2E1,4E0,1.41E2,5.1E1,1.1E1,5E0,4E0,6E0,4E0,9E0,1.67E2,1.1E1,6E0,1.1E1,1.5E1,4.2E1,2.9E1,2.6E2,6.9E1,3.7E1,1.1E1,7.4E1,8E0,6.1E1,8E1,3.2E1,1.9E1,7E0,4E0,4E0,5E0,8E1,8.7E1,6E0,5E0,5E0,6E0,3.8E1,4E0,2.4E1,5E0,2.54E2,6E0,1.4E1,5.5E1,2.7E1,1E1,3.6E1,3.8E1,4E0,4E0,3.6E1,2.5E1,3.9E1,4.1E1,1.1E1,2.1E1,1.5E1,4E0,7.2E1,8E0,8E0,7.9E1,1.2E1,2.6E1,1E1,1.4E1,2.4E1,2.3E2,5E0,9E0,4.6E1,9E0,2.1E1,6E0,6E0,4E0,2.1E1,1.5E1,1.3E1,2.5E1,3E1,6E0,1.8E1,7E0,3.4E1,5E0,3.7E1,4E0,6E0,5E0,1.7E1,4E0,6.1E1,1.1E1,4E0,4E0,4E0,4E0,2.4E1,5.5E1,7E0,7E0,2E1,4E0,2.7E1,2.03E2,7E0,3.9E1,4E0,1.7E1,1.7E1,4E0,5E0,1E1,2.1E1,4E0,1.7E1,1.3E1,1.1E1,7E0,1.2E1,2.2E1,7E0,3E1,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[2.9454422E-3,-9.563474E-3,7.428527E-2,-2.9429536E-2,2.0322178E-2,4.1506413E-2,1.13783166E-1,-1.7095927E-2,-9.8724216E-2,2.3066541E-2,-6.2965574E-3,-4.1470453E-3,4.9289316E-2,1.3534188E-1,-4.0188786E-2,-5.0270803E-2,3.7640538E-3,-6.295235E-2,-1.7997329E-1,2.6324706E-2,-7.5294375E-2,6.173823E-2,-5.4395396E-2,1.6529757E-1,8.126788E-2,-6.065235E-3,2.7232047E-3,-4.616113E-2,-8.085477E-3,1.0416386E-1,-7.4531315E-3,-7.370393E-2,2.7397985E-3,-3.2464108E-1,-4.371797E-2,4.2655105E-3,4.3851E-2,-1.3487922E-2,4.167971E-3,-1.3443623E-3,7.2128266E-2,-1.4028888E-5,-4.509399E-3,1.3111286E-1,1.1876941E-2,9.881935E-2,-7.767354E-5,-1.0347914E-1,-3.748169E-2,1.3016593E-1,1.7415573E-3,-6.1022893E-2,-1.5484404E-3,1.5956135E-3,-8.622015E-2,-5.3663175E-3,-1.9735374E-2,-5.6289393E-3,9.3964546E-4,-8.858226E-3,5.2812744E-2,6.9785826E-3,3.736328E-2,8.584616E-2,5.409155E-4,1.5262698E-1,-1.2382794E-4,1.960861E-3,6.2812814E-3,-1.2023802E-3,-5.5518528E-3,-1.0085321E-2,-6.550774E-2,8.8766284E-2,9.474076E-3,-5.375932E-3,4.015815E-3,2.9827037E-3,-5.020529E-3,-1.0165612E-1,-8.88543E-6,-8.319991E-3,-1.0652757E-3,1.7654244E-3,5.5940794E-3,-1.7961536E-2,4.559733E-2,1.0166748E-1,-1.0409621E-3,9.179849E-3,4.7230236E-3,-1.6026724E-2,3.4700744E-3,2.086195E-4,-7.323711E-2,1.1003499E-3,5.1911105E-3,-2.7842843E-3,3.6193852E-3,1.4941422E-2,-1.7769432E-2,-1.1799691E-1,-2.7672232E-3,1.0791896E-2,-4.690106E-2,4.2978145E-2,-1.0148094E-1,5.5032045E-2,-1.847993E-2,9.680315E-3,8.564236E-2,3.5728882E-3,-3.6594528E-3,1.4066939E-2,-2.9211087E-2,-5.983574E-2,-6.85991E-3,5.559923E-3,4.667273E-3,1.2511676E-2,-3.4337644E-2,-3.7651986E-3,-6.74811E-3,3.1818952E-3,5.3375266E-3,-4.6710996E-3,-1.1250079E-2,-4.1055446E-4,4.440304E-3,-4.0270337E-5,-7.7803126E-3,3.386677E-2,8.028684E-2,-5.821384E-3,1.627714E-2,3.8258277E-2,4.8316563E-3,-8.3033217E-4,3.3440406E-3,-5.3379415E-3,-6.6886307E-4,-3.6172136E-3,-7.629177E-4,1.7429817E-3,-9.52301E-4,-1.1119111E-3,1.9207463E-3,-2.42766E-3,-2.3811366E-4,-1.1721759E-3,1.2622371E-3,7.354085E-4,-2.525038E-3,1.9855238E-3,-2.1034812E-3,1.6549384E-3,4.8858863E-3,-1.2219751E-3,3.4645093E-3,2.8443632E-3,-5.243642E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,-1,49,51,53,-1,55,57,59,61,-1,-1,-1,63,-1,-1,65,-1,67,-1,69,71,73,-1,75,77,-1,79,-1,-1,-1,-1,81,83,-1,85,87,-1,89,-1,-1,-1,-1,-1,91,93,95,-1,-1,97,-1,99,101,-1,-1,103,-1,-1,105,107,109,111,-1,-1,113,-1,-1,115,-1,-1,-1,-1,117,119,121,-1,123,125,127,129,131,133,-1,135,-1,-1,137,139,141,-1,143,-1,145,147,-1,-1,149,-1,-1,151,-1,-1,-1,-1,153,155,-1,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9130956E-1,5.0525177E-1,1.8818694E-1,4.3504015E-1,1.4927793E-1,8.922307E-2,2.2784162E-1,3.012211E-1,2.124654E-1,1.0934158E-1,0E0,0E0,1.05328545E-1,8.2136154E-2,8.9014895E-2,7.804695E-2,3.025988E-1,7.6665744E-2,4.3917787E-1,1.2546313E-1,4.2079395E-1,6.910908E-2,2.100739E-2,8.238542E-2,3.3651367E-2,0E0,0E0,7.7709496E-2,0E0,4.175678E-2,7.6453224E-2,7.207152E-2,0E0,1.8553889E-1,6.753443E-2,9.285038E-2,1.2024295E-1,0E0,0E0,0E0,5.204025E-2,0E0,0E0,8.388391E-2,0E0,3.401336E-2,0E0,2.4024278E-2,1.0990426E-1,4.071629E-2,0E0,8.685171E-2,5.059369E-2,0E0,6.106487E-2,0E0,0E0,0E0,0E0,1.5326431E-1,2.6272096E-2,0E0,7.917313E-2,7.1588576E-2,0E0,3.7735105E-2,0E0,0E0,0E0,0E0,0E0,3.8661696E-2,3.8758546E-2,1.8780746E-2,0E0,0E0,6.2057383E-2,0E0,5.3454533E-2,1.9544423E-2,0E0,0E0,6.0832735E-2,0E0,0E0,1.2142332E-1,9.16996E-2,5.811286E-2,6.126675E-2,0E0,0E0,2.818452E-2,0E0,0E0,5.565524E-2,0E0,0E0,0E0,0E0,6.566116E-2,6.4647704E-2,1.5755832E-2,0E0,7.057266E-2,4.331084E-2,3.943531E-2,6.718852E-2,6.730369E-2,7.679725E-2,0E0,3.1899065E-2,0E0,0E0,4.240682E-2,6.065211E-2,4.349293E-2,0E0,6.392194E-2,0E0,4.990501E-2,4.3838553E-2,0E0,0E0,5.8593817E-2,0E0,0E0,1.8558875E-2,0E0,0E0,0E0,0E0,5.4763347E-2,6.3031524E-2,0E0,4.2122483E-2,1.573117E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,40,40,43,43,45,45,47,47,48,48,49,49,51,51,52,52,54,54,59,59,60,60,62,62,63,63,65,65,71,71,72,72,73,73,76,76,78,78,79,79,82,82,85,85,86,86,87,87,88,88,91,91,94,94,99,99,100,100,101,101,103,103,104,104,105,105,106,106,107,107,108,108,110,110,113,113,114,114,115,115,117,117,119,119,120,120,123,123,126,126,131,131,132,132,134,134,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,-1,50,52,54,-1,56,58,60,62,-1,-1,-1,64,-1,-1,66,-1,68,-1,70,72,74,-1,76,78,-1,80,-1,-1,-1,-1,82,84,-1,86,88,-1,90,-1,-1,-1,-1,-1,92,94,96,-1,-1,98,-1,100,102,-1,-1,104,-1,-1,106,108,110,112,-1,-1,114,-1,-1,116,-1,-1,-1,-1,118,120,122,-1,124,126,128,130,132,134,-1,136,-1,-1,138,140,142,-1,144,-1,146,148,-1,-1,150,-1,-1,152,-1,-1,-1,-1,154,156,-1,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.467E3,8.995735E5,1.767635E3,2.2E1,4.9E1,1.1840488E8,2.5130852E2,5.51E2,6.0692043E0,1.9269184E7,-6.2965574E-3,-4.1470453E-3,6.364486E0,1.159499E6,4.591837E0,6.651E4,2.6E1,6.2233735E-6,1.746E3,1.158E3,2.395631E7,1.3935602E1,1.172766E3,3.5675005E3,1.7655972E0,-6.065235E-3,2.7232047E-3,3.4E1,-8.085477E-3,3.66025E5,1.84E2,1E0,2.7397985E-3,1.04109E5,2.6530768E2,4.5866325E6,6.923077E-1,-1.3487922E-2,4.167971E-3,-1.3443623E-3,1.4379113E9,-1.4028888E-5,-4.509399E-3,1.8776652E7,1.1876941E-2,1.2816234E7,-7.767354E-5,1.3888889E1,3.9961785E-2,6.743651E7,1.7415573E-3,1.1151079E0,5E0,1.5956135E-3,5.095006E5,-5.3663175E-3,-1.9735374E-2,-5.6289393E-3,9.3964546E-4,1.4110284E2,2.712766E0,6.9785826E-3,2.4294034E2,3.0561172E5,5.409155E-4,1.3947369E0,-1.2382794E-4,1.960861E-3,6.2812814E-3,-1.2023802E-3,-5.5518528E-3,7.2132964E0,1.2368386E0,1.7467743E-2,9.474076E-3,-5.375932E-3,6.819212E6,2.9827037E-3,1.4195632E6,2.8998098E1,-8.88543E-6,-8.319991E-3,9.03E2,1.7654244E-3,5.5940794E-3,1.0439024E1,2E0,1.43E2,5.3700186E8,9.179849E-3,4.7230236E-3,2.00033E5,3.4700744E-3,2.086195E-4,7.547528E6,1.1003499E-3,5.1911105E-3,-2.7842843E-3,3.6193852E-3,4.586207E0,3.14403E5,9.874E3,-2.7672232E-3,2.0097298E1,9.99E2,5.80418E7,1.1E1,6.5718125E6,1.4598765E0,9.680315E-3,1.606722E8,3.5728882E-3,-3.6594528E-3,4.9E2,4.9069305E1,1.3600995E2,-6.85991E-3,2.3412812E0,4.667273E-3,5.860227E5,1.3631483E8,-3.7651986E-3,-6.74811E-3,2.485E2,5.3375266E-3,-4.6710996E-3,3.0416667E0,-4.1055446E-4,4.440304E-3,-4.0270337E-5,-7.7803126E-3,2.23099E7,6.763312E7,-5.821384E-3,8.134772E2,2.4180895E6,4.8316563E-3,-8.3033217E-4,3.3440406E-3,-5.3379415E-3,-6.6886307E-4,-3.6172136E-3,-7.629177E-4,1.7429817E-3,-9.52301E-4,-1.1119111E-3,1.9207463E-3,-2.42766E-3,-2.3811366E-4,-1.1721759E-3,1.2622371E-3,7.354085E-4,-2.525038E-3,1.9855238E-3,-2.1034812E-3,1.6549384E-3,4.8858863E-3,-1.2219751E-3,3.4645093E-3,2.8443632E-3,-5.243642E-4],"split_indices":[2,43,67,3,3,7,73,2,68,62,0,0,69,9,69,1,0,52,44,2,62,73,4,67,49,0,0,2,0,9,44,20,0,5,67,43,68,0,0,0,12,0,0,60,0,9,0,4,53,7,0,61,3,0,43,0,0,0,0,4,68,0,67,48,0,68,0,0,0,0,0,69,68,53,0,0,12,0,60,73,0,0,2,0,0,73,6,0,7,0,0,5,0,0,5,0,0,0,0,68,9,1,0,73,2,7,8,60,69,0,12,0,0,1,62,67,0,68,0,62,7,0,0,67,0,0,69,0,0,0,0,9,7,0,70,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,8.49E2,1.48E2,5.1E2,3.39E2,8.2E1,6.6E1,4.34E2,7.6E1,3.34E2,5E0,4E0,7.8E1,5.8E1,8E0,1.67E2,2.67E2,5.4E1,2.2E1,3.24E2,1E1,7E1,8E0,3.6E1,2.2E1,4E0,4E0,1.63E2,4E0,2.6E1,2.41E2,5E1,4E0,1E1,1.2E1,1.44E2,1.8E2,4E0,6E0,7E0,6.3E1,4E0,4E0,2.8E1,8E0,1.8E1,4E0,2E1,1.43E2,1.8E1,8E0,2.3E1,2.18E2,5E0,4.5E1,4E0,6E0,5E0,7E0,1.14E2,3E1,9E0,1.71E2,5.1E1,1.2E1,2.4E1,4E0,8E0,1E1,4E0,1.6E1,7.3E1,7E1,1.3E1,5E0,1.2E1,1.1E1,1E1,2.08E2,3.8E1,7E0,4E0,1.1E2,2.6E1,4E0,2.2E1,1.49E2,4.3E1,8E0,1.1E1,1.3E1,6.9E1,4E0,7E0,6.3E1,4E0,9E0,6E0,5E0,8.1E1,1.27E2,2.6E1,1.2E1,8.8E1,2.2E1,1.3E1,9E0,1.3E2,1.9E1,4E0,3.9E1,4E0,4E0,2.1E1,4.8E1,5.5E1,8E0,7.4E1,7E0,4.5E1,8.2E1,1.3E1,1.3E1,8.3E1,5E0,8E0,1.4E1,7E0,6E0,4E0,5E0,7.2E1,5.8E1,4E0,1.5E1,1.2E1,2.7E1,1.4E1,7E0,6E0,4.2E1,3.8E1,1.7E1,3.3E1,4.1E1,2E1,2.5E1,5E1,3.2E1,3.8E1,4.5E1,9E0,5E0,6.5E1,7E0,2.2E1,3.6E1,9E0,6E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[2.0819542E-3,-1.7877074E-2,3.76012E-2,9.933179E-4,-4.062655E-2,5.258583E-2,-5.2450586E-2,-1.16662E-2,3.8404495E-2,-7.575788E-2,-2.2136638E-2,1.5701999E-1,4.148614E-2,-1.0884336E-2,-2.4057562E-2,-1.7698543E-2,5.362315E-2,2.4370957E-2,4.53019E-3,-4.9488153E-2,-2.3734702E-1,6.727523E-4,-4.7405884E-2,1.7948905E-1,4.0548347E-4,4.580467E-2,-9.699776E-2,-8.577646E-2,1.3775699E-2,-1.4497971E-2,-5.791575E-3,4.737488E-3,2.0695077E-2,-1.1189776E-2,3.9343145E-2,-5.96843E-2,2.8277691E-3,-1.9832907E-2,-5.4348363E-3,-2.0492876E-2,4.6831917E-2,-8.729253E-3,-3.721653E-2,3.3127645E-3,9.046999E-3,1.2169534E-2,6.482141E-2,-8.060399E-3,-5.7684047E-6,-4.000148E-4,-1.236139E-1,8.4833086E-2,-4.555248E-2,-6.140423E-2,-9.027213E-3,3.7792719E-3,-4.1408776E-4,8.329023E-4,-3.505223E-2,5.7367004E-2,1.1096895E-3,-4.2713508E-2,-1.390629E-1,-6.9562527E-3,-4.7919643E-3,4.4867317E-3,1.7274745E-2,2.6133945E-3,-4.476868E-2,5.975969E-3,4.7956347E-3,5.7432503E-2,7.495726E-3,-7.3635615E-3,-1.9243296E-3,1.2752093E-1,-8.089096E-4,2.5987994E-2,-6.9975727E-3,1.3696582E-3,-9.171758E-2,5.0598115E-2,-1.3074896E-2,-3.5596162E-3,5.521968E-4,4.8075696E-3,3.7947826E-2,-1.4810581E-3,1.161024E-3,-8.900546E-2,-1.6046567E-2,-1.6721735E-3,-9.779852E-3,-4.091469E-2,1.6461527E-2,-9.887124E-4,2.1111395E-3,-6.1900683E-2,-9.517646E-3,1.3114582E-2,-3.6036347E-3,8.496984E-2,2.7851483E-2,2.9190194E-3,8.279868E-3,3.5521688E-3,-2.331671E-3,-8.6891875E-3,-5.348176E-2,-2.4155518E-3,4.4803945E-3,-1.7102772E-2,5.2796353E-2,4.4994867E-5,6.330651E-2,-1.5464537E-3,-1.087415E-1,2.415385E-2,-4.13166E-2,-3.5429904E-3,3.5088528E-5,2.952218E-2,-1.672692E-3,-3.0730983E-2,-7.503631E-2,3.6828525E-2,-5.2139565E-2,-1.5952941E-2,3.6106307E-2,1.0857082E-1,-3.3979516E-2,3.496704E-2,-3.146467E-3,-8.7539974E-4,-4.7180164E-3,1.7066026E-3,-2.1535968E-2,3.934041E-4,5.2244924E-3,4.645524E-3,6.231866E-4,-1.5208583E-3,-1.2805347E-1,2.1235573E-3,-1.9287746E-3,-6.4249575E-2,2.951061E-4,-3.7301495E-4,5.7123598E-2,-2.2669241E-3,6.5244216E-4,-1.1241767E-1,-4.8546452E-2,-1.3279982E-3,4.1395235E-3,-1.6305559E-4,-4.3534413E-3,1.7959678E-3,-3.5524596E-2,5.1395256E-2,-3.9253917E-2,1.193095E-1,4.493497E-2,-9.086982E-3,4.3069743E-2,-2.7071675E-2,4.799165E-2,-5.2263984E-4,-3.4198577E-3,-7.490021E-3,-2.5688484E-3,-4.786176E-3,-1.2110481E-3,3.5636595E-3,1.9590536E-4,-6.549292E-3,-2.0684865E-3,-3.534229E-3,-8.781887E-4,5.2576273E-4,-2.7509732E-3,-4.5676634E-4,2.684451E-3,-3.661148E-3,3.968595E-4,5.820149E-3,1.7171294E-3,-1.5692565E-3,5.982705E-3,4.145085E-3,-4.971527E-4,-3.1300636E-3,1.7768291E-3,3.223041E-3,8.920452E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,-1,45,47,49,51,53,-1,-1,55,57,59,61,-1,-1,-1,63,65,-1,67,-1,-1,69,71,-1,-1,-1,73,75,77,79,81,-1,-1,-1,83,85,87,89,91,93,-1,-1,95,-1,97,-1,99,101,-1,-1,-1,103,-1,105,-1,-1,107,109,111,-1,-1,-1,113,-1,-1,115,117,-1,-1,119,121,-1,-1,123,125,127,-1,129,131,-1,-1,-1,-1,-1,133,-1,-1,135,137,-1,139,-1,141,143,145,-1,-1,147,-1,149,151,153,155,157,159,161,163,165,-1,-1,-1,-1,167,-1,-1,-1,-1,-1,169,-1,-1,171,-1,-1,173,-1,-1,175,177,-1,-1,-1,-1,-1,179,181,183,185,187,-1,189,191,193,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.251983E-1,2.8122628E-1,4.9933475E-1,1.7047313E-1,1.9124687E-1,3.5978878E-1,2.699357E-1,1.06894106E-1,7.379095E-2,4.212342E-1,1.1300161E-1,9.835112E-2,1.7608231E-1,0E0,1.109279E-1,8.329445E-2,3.492868E-2,4.037189E-2,0E0,1.0410555E-1,2.711805E-1,1.02566436E-1,1.2709537E-1,3.174132E-2,0E0,1.776681E-1,6.752383E-2,5.1452726E-2,1.3033552E-1,6.1289374E-2,0E0,0E0,2.7918976E-2,1.6668027E-2,3.645941E-2,1.0452375E-1,0E0,0E0,0E0,8.0398895E-2,4.7370106E-2,0E0,6.486076E-2,0E0,0E0,8.7517515E-2,1.1882007E-1,0E0,0E0,0E0,2.8545663E-2,6.3395604E-2,1.3274671E-1,7.172075E-2,5.30333E-2,0E0,0E0,0E0,2.658392E-2,2.8695159E-2,1.5146844E-2,8.3749846E-2,9.69311E-2,5.077308E-2,0E0,0E0,2.516125E-2,0E0,4.893236E-2,0E0,6.78358E-2,1.3366872E-1,0E0,0E0,0E0,2.1939024E-2,0E0,4.668976E-2,0E0,0E0,5.9947595E-2,7.1878284E-2,5.5028502E-2,0E0,0E0,0E0,2.5253791E-2,0E0,0E0,2.3759425E-2,4.6711914E-2,0E0,0E0,3.8821492E-2,2.7208135E-2,0E0,0E0,2.0212904E-2,5.723038E-2,6.005582E-2,0E0,2.451188E-1,5.837079E-2,0E0,0E0,0E0,0E0,0E0,2.326142E-2,0E0,0E0,4.7035083E-2,3.1073507E-2,0E0,2.8333165E-2,0E0,2.2141352E-2,2.7793076E-2,3.1042457E-2,0E0,0E0,3.2630235E-2,0E0,1.5498072E-2,3.2171145E-2,5.1023245E-2,3.0359399E-2,4.4303596E-2,5.9626386E-2,4.3841183E-2,1.9652459E-1,6.351751E-2,0E0,0E0,0E0,0E0,9.5213816E-2,0E0,0E0,0E0,0E0,0E0,2.5543764E-2,0E0,0E0,2.5721401E-2,0E0,0E0,1.8723667E-2,0E0,0E0,2.3427978E-2,1.836222E-2,0E0,0E0,0E0,0E0,0E0,3.4479495E-2,1.7847449E-2,1.7901774E-2,2.8740168E-2,8.44657E-2,0E0,2.8590323E-2,3.9927527E-2,3.862101E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,35,35,39,39,40,40,42,42,45,45,46,46,50,50,51,51,52,52,53,53,54,54,58,58,59,59,60,60,61,61,62,62,63,63,66,66,68,68,70,70,71,71,75,75,77,77,80,80,81,81,82,82,86,86,89,89,90,90,93,93,94,94,97,97,98,98,99,99,101,101,102,102,108,108,111,111,112,112,114,114,116,116,117,117,118,118,121,121,123,123,124,124,125,125,126,126,127,127,128,128,129,129,130,130,131,131,136,136,142,142,145,145,148,148,151,151,152,152,158,158,159,159,160,160,161,161,162,162,164,164,165,165,166,166],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,-1,46,48,50,52,54,-1,-1,56,58,60,62,-1,-1,-1,64,66,-1,68,-1,-1,70,72,-1,-1,-1,74,76,78,80,82,-1,-1,-1,84,86,88,90,92,94,-1,-1,96,-1,98,-1,100,102,-1,-1,-1,104,-1,106,-1,-1,108,110,112,-1,-1,-1,114,-1,-1,116,118,-1,-1,120,122,-1,-1,124,126,128,-1,130,132,-1,-1,-1,-1,-1,134,-1,-1,136,138,-1,140,-1,142,144,146,-1,-1,148,-1,150,152,154,156,158,160,162,164,166,-1,-1,-1,-1,168,-1,-1,-1,-1,-1,170,-1,-1,172,-1,-1,174,-1,-1,176,178,-1,-1,-1,-1,-1,180,182,184,186,188,-1,190,192,194,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,3.875E0,1E0,3.422351E6,1E0,1.8992E4,4.5E0,2.0277777E0,6.965855E9,1.8256016E7,1.45625E1,8.837831E-2,3.206931E2,-1.0884336E-2,1.4598765E0,5.2E1,3.1842105E0,3.011015E8,4.53019E-3,4.5E1,3.6255838E2,1.6223962E7,6.8E1,4.5496914E9,4.0548347E-4,5.817547E2,3.82E3,1.7234043E1,2.4E0,7.7E1,-5.791575E-3,4.737488E-3,1.4473684E-1,3.72E2,1.5321098E8,6.18826E5,2.8277691E-3,-1.9832907E-2,-5.4348363E-3,6.6358675E8,1.0486312E9,-8.729253E-3,2E0,3.3127645E-3,9.046999E-3,9.9688476E-1,4.907764E3,-8.060399E-3,-5.7684047E-6,-4.000148E-4,5.467E3,1.7E1,2E1,4.6874816E4,1.6E1,3.7792719E-3,-4.1408776E-4,8.329023E-4,8E0,1.5123151E6,4E0,2.4636364E1,5.3E1,4.1719616E5,-4.7919643E-3,4.4867317E-3,1.3881669E6,2.6133945E-3,3.885813E2,5.975969E-3,2E1,1.7951853E1,7.495726E-3,-7.3635615E-3,-1.9243296E-3,1.2697166E3,-8.089096E-4,8.0274E4,-6.9975727E-3,1.3696582E-3,7E0,2.66E2,1.775373E1,-3.5596162E-3,5.521968E-4,4.8075696E-3,1.5849056E0,-1.4810581E-3,1.161024E-3,3.8E1,8E0,-1.6721735E-3,-9.779852E-3,2.240836E6,1E0,-9.887124E-4,2.1111395E-3,7.106795E6,7.836E3,6.185E3,-3.6036347E-3,3.174172E5,1.514526E6,2.9190194E-3,8.279868E-3,3.5521688E-3,-2.331671E-3,-8.6891875E-3,7.042942E7,-2.4155518E-3,4.4803945E-3,3.5511714E-1,2.23607E5,4.4994867E-5,2.0467092E6,-1.5464537E-3,1.863E3,2.9E1,1.802361E6,-3.5429904E-3,3.5088528E-5,2.1651703E2,-1.672692E-3,1.7E1,1.0041E5,2.2166292E1,1.763E3,3.819455E-1,1.1108876E11,1.6063418E7,1.4598765E0,1.4407171E8,-3.146467E-3,-8.7539974E-4,-4.7180164E-3,1.7066026E-3,1.0292E4,3.934041E-4,5.2244924E-3,4.645524E-3,6.231866E-4,-1.5208583E-3,1.00011E5,2.1235573E-3,-1.9287746E-3,4.6748266E0,2.951061E-4,-3.7301495E-4,1.946094E6,-2.2669241E-3,6.5244216E-4,2.0518E2,5.4051723E0,-1.3279982E-3,4.1395235E-3,-1.6305559E-4,-4.3534413E-3,1.7959678E-3,1.136E3,5.036081E7,1.2193182E1,4.7E1,8.710612E1,-9.086982E-3,1.0236775E1,2.5205562E5,4.857143E0,-5.2263984E-4,-3.4198577E-3,-7.490021E-3,-2.5688484E-3,-4.786176E-3,-1.2110481E-3,3.5636595E-3,1.9590536E-4,-6.549292E-3,-2.0684865E-3,-3.534229E-3,-8.781887E-4,5.2576273E-4,-2.7509732E-3,-4.5676634E-4,2.684451E-3,-3.661148E-3,3.968595E-4,5.820149E-3,1.7171294E-3,-1.5692565E-3,5.982705E-3,4.145085E-3,-4.971527E-4,-3.1300636E-3,1.7768291E-3,3.223041E-3,8.920452E-4],"split_indices":[2,69,6,9,29,9,73,68,46,5,73,72,73,0,69,8,69,46,0,3,4,5,0,46,0,67,0,71,69,44,0,0,71,44,7,9,0,0,0,46,46,0,32,0,0,69,4,0,0,0,2,8,6,60,0,0,0,0,8,60,3,67,0,60,0,0,43,0,67,0,3,71,0,0,0,4,0,1,0,0,12,1,71,0,0,0,71,0,0,44,10,0,0,9,30,0,0,5,44,44,0,48,10,0,0,0,0,0,60,0,0,69,9,0,43,0,9,3,12,0,0,67,0,3,1,73,2,57,46,62,69,5,0,0,0,0,44,0,0,0,0,0,5,0,0,69,0,0,60,0,0,67,69,0,0,0,0,0,44,7,73,8,73,0,71,48,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.021E3,6.54E2,3.67E2,3.58E2,2.96E2,3.15E2,5.2E1,2.68E2,9E1,1.01E2,1.95E2,2.9E1,2.86E2,6E0,4.6E1,2.46E2,2.2E1,7.4E1,1.6E1,8.8E1,1.3E1,1.03E2,9.2E1,2.5E1,4E0,2.78E2,8E0,1.7E1,2.9E1,2.4E2,6E0,8E0,1.4E1,2.2E1,5.2E1,8.1E1,7E0,4E0,9E0,7.1E1,3.2E1,5E0,8.7E1,4E0,2.1E1,1.01E2,1.77E2,4E0,4E0,6E0,1.1E1,1.3E1,1.6E1,2.4E1,2.16E2,4E0,1E1,1E1,1.2E1,3.5E1,1.7E1,6.8E1,1.3E1,6.2E1,9E0,1.1E1,2.1E1,6E0,8.1E1,5E0,9.6E1,1.66E2,1.1E1,7E0,4E0,9E0,4E0,1E1,6E0,6E0,1.8E1,1.3E1,2.03E2,6E0,6E0,9E0,2.6E1,7E0,1E1,2.4E1,4.4E1,6E0,7E0,2.5E1,3.7E1,9E0,1.2E1,5.4E1,2.7E1,8.8E1,8E0,8.5E1,8.1E1,5E0,4E0,6E0,4E0,4E0,1.4E1,4E0,9E0,1.92E2,1.1E1,1.1E1,1.5E1,7E0,1.7E1,1.7E1,2.7E1,1.3E1,1.2E1,3E1,7E0,1.7E1,3.7E1,1.3E1,1.4E1,3.9E1,4.9E1,7.1E1,1.4E1,7.6E1,5E0,9E0,5E0,1.4E1,1.78E2,7E0,4E0,8E0,7E0,4E0,1.3E1,1.3E1,4E0,1.8E1,9E0,1.3E1,1.7E1,1.2E1,5E0,1.4E1,2.3E1,6E0,7E0,7E0,7E0,1E1,2.9E1,4.1E1,8E0,6E1,1.1E1,4E0,1E1,1.3E1,6.3E1,1.5E2,2.8E1,8E0,5E0,8E0,1E1,1.2E1,5E0,9E0,5E0,1.1E1,1.2E1,1E1,1.9E1,4E0,3.7E1,4E0,4E0,5.5E1,5E0,6E0,5E0,5E0,5E0,8E0,5E0,3.5E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"195","size_leaf_vector":"1"}},{"base_weights":[-5.098481E-4,-1.4738046E-2,4.0954E-2,-1.9372182E-2,5.9637774E-2,-1.23512626E-1,4.744676E-2,-1.7254204E-2,-1.8077473E-1,4.593562E-2,5.7980376E-3,-2.4887423E-3,-8.425841E-3,1.241756E-2,4.2697247E-2,-2.8662639E-2,8.007986E-3,-1.3674575E-2,-1.3583421E-3,3.2477606E-2,5.1880954E-3,5.212447E-2,-3.798862E-2,-1.542429E-2,-5.676903E-2,1.2686985E-2,-6.1987303E-3,-5.8552832E-5,4.940772E-2,3.7044518E-2,8.7050445E-2,3.6837216E-3,-7.729276E-2,-1.194912E-2,-1.3688593E-1,-1.1681E-2,-4.563655E-2,-2.3688074E-2,2.2412676E-2,3.8328255E-4,6.8039946E-2,1.10149E-2,5.3999268E-2,9.8913275E-2,2.2469626E-3,-2.5181672E-2,-1.3818614E-1,-3.8508173E-2,3.4171776E-3,-8.849732E-3,-2.5334677E-3,-8.2773045E-2,-1.645134E-2,-1.2277105E-2,-4.426478E-3,2.6620286E-2,-3.583389E-3,4.267272E-3,7.463321E-4,-3.0195527E-3,1.8861081E-2,6.5743993E-3,3.9159935E-2,6.2475264E-2,1.2962498E-1,2.8180457E-3,-2.631192E-3,-2.4598902E-3,1.1437379E-3,-1.0430779E-2,-1.0605438E-3,-3.1210838E-2,-9.042371E-3,1.0638501E-2,-2.299947E-3,-9.830764E-3,-7.055862E-2,-5.971306E-3,-4.361153E-3,-2.1943724E-4,-2.9575764E-3,3.4335148E-2,-2.6447143E-2,2.4948966E-2,-2.2842064E-3,9.353032E-2,2.4337528E-2,7.833544E-2,1.7111906E-4,7.3710205E-3,3.300594E-3,-4.6692163E-2,-4.9228165E-3,7.3657366E-3,-6.4146735E-2,-1.0125736E-1,-3.6484636E-2,-1.7468985E-2,5.0467397E-3,4.2490862E-2,-9.65095E-5,1.3811332E-3,-1.0489206E-1,-1.7991523E-3,3.0959442E-2,5.7429476E-3,-2.4022316E-4,4.5577753E-3,1.3667204E-2,4.3751905E-3,8.733776E-4,-1.5174905E-3,-5.508142E-3,2.0170852E-3,-1.6322191E-3,8.7473524E-4,-1.7178691E-3,1.8310734E-3,-4.1364287E-3,-5.4006767E-3,-7.667031E-4,-4.4859826E-4,-5.6857583E-3,-2.0381626E-4,-3.969211E-3,8.557374E-4,2.5274237E-3,2.0049165E-3,-1.0617899E-3,-6.7551206E-3,-1.9675244E-3,1.9234428E-3,-6.853169E-4,-3.649284E-3,1.1041055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,-1,-1,27,-1,29,31,33,35,37,-1,-1,39,41,43,-1,45,47,49,-1,51,53,55,-1,57,59,61,63,65,67,69,71,73,-1,-1,75,77,79,-1,81,-1,-1,-1,-1,83,-1,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,-1,93,-1,95,-1,97,-1,-1,99,101,103,-1,105,107,109,-1,-1,-1,111,113,115,117,119,121,123,-1,125,127,-1,129,-1,131,-1,-1,-1,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9114933E-1,2.5827742E-1,2.79228E-1,2.3719943E-1,3.3943772E-2,3.1172812E-2,2.4818105E-1,2.0052996E-1,1.4826483E-1,3.02912E-2,0E0,0E0,0E0,0E0,1.8681666E-1,1.7735177E-1,1.4606614E-1,0E0,0E0,1.8915746E-2,0E0,1.11591935E-1,1.2646657E-1,1.3639097E-1,3.224242E-1,7.517694E-2,0E0,0E0,1.6287457E-2,6.736934E-2,6.516829E-2,0E0,5.9984647E-2,1.3032006E-1,2.9865876E-2,0E0,1.5687838E-1,3.5859473E-2,7.231554E-2,0E0,1.7441131E-2,3.838615E-2,1.16295636E-1,5.56888E-2,3.4776725E-2,1.8135723E-2,8.5657164E-2,1.2812224E-1,2.6381576E-1,0E0,0E0,8.6624116E-2,1.1262984E-1,1.52747E-2,0E0,6.6822745E-2,0E0,0E0,0E0,0E0,2.4935605E-2,0E0,6.3001595E-2,2.580931E-2,4.0832877E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.5758143E-2,0E0,0E0,1.19433E-1,0E0,5.906737E-2,0E0,1.1507625E-1,0E0,0E0,3.959897E-2,9.595109E-2,2.1558948E-2,0E0,5.230528E-2,5.019623E-2,1.9609943E-2,0E0,0E0,0E0,6.738642E-2,6.506628E-2,8.986241E-2,7.468872E-2,3.9114714E-2,6.8354115E-2,6.1287977E-2,0E0,3.236623E-2,2.8869137E-2,0E0,1.694569E-2,0E0,2.45696E-2,0E0,0E0,0E0,5.6874275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,19,19,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,32,32,33,33,34,34,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,51,51,52,52,53,53,55,55,60,60,62,62,63,63,64,64,71,71,74,74,76,76,78,78,81,81,82,82,83,83,85,85,86,86,87,87,91,91,92,92,93,93,94,94,95,95,96,96,97,97,99,99,100,100,102,102,104,104,108,108],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,-1,-1,28,-1,30,32,34,36,38,-1,-1,40,42,44,-1,46,48,50,-1,52,54,56,-1,58,60,62,64,66,68,70,72,74,-1,-1,76,78,80,-1,82,-1,-1,-1,-1,84,-1,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,-1,94,-1,96,-1,98,-1,-1,100,102,104,-1,106,108,110,-1,-1,-1,112,114,116,118,120,122,124,-1,126,128,-1,130,-1,132,-1,-1,-1,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.984E3,4.5866325E6,1E0,5.9E1,1.339646E6,7.472532E9,3.3817584E7,2.026453E6,4.0568292E2,2.712766E0,5.7980376E-3,-2.4887423E-3,-8.425841E-3,1.241756E-2,3.1879792E8,3.39498E5,1E0,-1.3674575E-2,-1.3583421E-3,1.2E1,5.1880954E-3,1.4817301E3,7.556899E-1,4.217427E7,9E0,5.88E2,-6.1987303E-3,-5.8552832E-5,5.294E3,1.1251919E6,1.2820834E7,3.6837216E-3,1.3475722E2,1.7667647E2,1.0479E4,-1.1681E-2,1.084437E6,2.8608696E1,4.643602E8,3.8328255E-4,2.138772E4,3.167E3,3.443E3,1.6326721E0,3.359873E6,1.1863237E2,2.0120485E0,7.45E2,7.336111E4,-8.849732E-3,-2.5334677E-3,3.72424E5,1.318E2,1.0066326E8,-4.426478E-3,1E0,-3.583389E-3,4.267272E-3,7.463321E-4,-3.0195527E-3,5.433518E0,6.5743993E-3,7.5052085E0,1.4052098E0,1E0,2.8180457E-3,-2.631192E-3,-2.4598902E-3,1.1437379E-3,-1.0430779E-2,-1.0605438E-3,8.2765434E1,-9.042371E-3,1.0638501E-2,1.647E4,-9.830764E-3,3.1275E4,-5.971306E-3,1.9791039E6,-2.1943724E-4,-2.9575764E-3,1.7E1,5.036784E6,2.9585715E2,-2.2842064E-3,8.76E2,1.6208625E6,5.501002E9,1.7111906E-4,7.3710205E-3,3.300594E-3,4.577342E0,4.8E1,6.2222223E0,1.3E1,9.393264E-6,2.0734E4,2.4E1,5.0467397E-3,1.2200055E7,5.44E2,1.3811332E-3,1.11E2,-1.7991523E-3,7.973882E7,5.7429476E-3,-2.4022316E-4,4.5577753E-3,3.921E3,4.3751905E-3,8.733776E-4,-1.5174905E-3,-5.508142E-3,2.0170852E-3,-1.6322191E-3,8.7473524E-4,-1.7178691E-3,1.8310734E-3,-4.1364287E-3,-5.4006767E-3,-7.667031E-4,-4.4859826E-4,-5.6857583E-3,-2.0381626E-4,-3.969211E-3,8.557374E-4,2.5274237E-3,2.0049165E-3,-1.0617899E-3,-6.7551206E-3,-1.9675244E-3,1.9234428E-3,-6.853169E-4,-3.649284E-3,1.1041055E-3],"split_indices":[2,43,17,3,9,46,7,9,4,68,0,0,0,0,47,9,79,0,0,3,0,67,57,60,3,2,0,0,12,43,9,0,73,70,9,0,9,73,7,0,62,2,2,53,1,73,50,2,60,0,0,9,70,7,0,19,0,0,0,0,68,0,71,53,30,0,0,0,0,0,0,4,0,0,44,0,1,0,43,0,0,3,9,67,0,0,43,5,0,0,0,73,10,61,3,52,44,8,0,5,10,0,0,0,47,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E3,7.45E2,2.55E2,7.02E2,4.3E1,9E0,2.46E2,6.94E2,8E0,3.7E1,6E0,5E0,4E0,4E0,2.42E2,4.78E2,2.16E2,4E0,4E0,3.2E1,5E0,2.17E2,2.5E1,3.26E2,1.52E2,2.1E2,6E0,1.1E1,2.1E1,1.53E2,6.4E1,6E0,1.9E1,3.18E2,8E0,7E0,1.45E2,4.4E1,1.66E2,7E0,1.4E1,6.1E1,9.2E1,5.6E1,8E0,1.1E1,8E0,1.16E2,2.02E2,4E0,4E0,6.3E1,8.2E1,3.9E1,5E0,1.6E2,6E0,9E0,5E0,5E0,5.6E1,1.2E1,8E1,2.7E1,2.9E1,4E0,4E0,7E0,4E0,4E0,4E0,1.12E2,4E0,4E0,1.98E2,4E0,5.9E1,7E0,7.5E1,3.5E1,4E0,1.4E2,2E1,5.2E1,4E0,1.6E1,6.4E1,2.1E1,6E0,1.8E1,1.1E1,7E1,4.2E1,1.72E2,2.6E1,3E1,2.9E1,6.8E1,7E0,1.13E2,2.7E1,1.2E1,8E0,4E0,4.8E1,1.2E1,4E0,7E0,5.7E1,1.6E1,5E0,6E1,1E1,1.6E1,2.6E1,1.37E2,3.5E1,5E0,2.1E1,2.5E1,5E0,2.3E1,6E0,5.8E1,1E1,3.9E1,7.4E1,9E0,1.8E1,4E0,4E0,3.9E1,9E0,5E0,5.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-2.3366245E-3,-3.3935077E-2,2.497764E-2,-2.9708486E-2,-1.417064E-1,1.0916311E-2,7.048774E-2,-5.9631854E-2,-1.710505E-2,-5.928723E-2,-1.5898887E-2,1.8216379E-2,-7.442907E-2,1.065097E-2,6.237094E-2,-6.490355E-3,-5.35755E-2,2.1833261E-2,-2.692233E-2,2.0630606E-3,-6.598452E-3,1.22842915E-1,1.4626896E-2,2.0293012E-2,-1.0934075E-1,7.504802E-2,1.263924E-2,2.874033E-3,-5.7744954E-2,4.9296217E-3,1.3267819E-2,-9.8109305E-2,-2.0045375E-2,3.0784048E-3,8.191784E-3,-1.9137459E-2,2.371617E-2,2.562346E-3,-1.1997796E-3,-6.3487045E-2,-1.717726E-1,5.0587215E-2,9.540118E-2,4.512991E-2,-5.0172056E-3,-4.9768772E-2,-1.0947064E-1,-1.1092251E-2,3.1169321E-2,-1.1857892E-1,-3.2473574E-5,-1.3697813E-1,-1.5462298E-2,-4.862707E-3,-1.2178419E-2,7.0062275E-3,2.062097E-2,-3.93082E-3,4.8236318E-5,-1.1115208E-2,-2.5332111E-3,-1.5583293E-2,7.068069E-2,1.5074437E-4,1.04736455E-1,5.898369E-3,6.16405E-4,-5.9721004E-2,-9.094102E-3,-1.3707307E-1,-9.85457E-4,2.3244622E-3,-1.6215943E-3,7.4759037E-3,2.9494262E-3,-1.9061627E-3,-1.6024344E-1,-2.5501123E-4,-1.1135857E-2,-1.8156148E-2,5.153377E-3,9.026356E-4,-4.2929705E-2,5.9875626E-2,1.4100203E-2,-3.0645975E-3,8.354735E-4,3.4172386E-2,4.3436084E-3,1.3397242E-1,6.27663E-2,1.3335259E-3,-6.647257E-2,2.3376613E-3,-5.1083006E-2,-1.4053211E-3,-8.620643E-3,-3.4271897E-4,2.7534587E-3,-4.703675E-3,-1.0241547E-2,-2.724116E-2,2.0867791E-2,-3.3161156E-2,2.4616713E-2,-5.1386394E-3,-1.7778542E-2,3.0733211E-2,1.102831E-1,-1.4536855E-2,2.4328878E-2,2.9816085E-3,-1.9684598E-4,1.5591888E-1,2.7542224E-3,1.4732647E-2,4.383693E-3,-7.45126E-2,-1.2692969E-2,-5.083454E-3,-6.1066314E-3,-8.612028E-3,-2.2743389E-2,1.0033327E-2,3.5435753E-3,1.8349178E-3,-2.3374162E-3,-1.2451997E-2,4.3717068E-2,-2.0938818E-3,1.480186E-2,4.4817934E-3,8.248015E-3,6.5334155E-3,1.8484099E-3,2.1140661E-2,-8.525985E-2,1.2984869E-2,7.387153E-2,9.635364E-3,4.96547E-3,2.6512677E-3,-1.0749991E-3,-4.4982047E-3,-2.5811733E-3,1.6035568E-3,-3.0436222E-3,2.394242E-3,-2.902059E-3,-6.9392077E-4,-2.77782E-3,-8.605957E-4,1.2717951E-3,-1.9096925E-3,1.6748591E-3,2.8841703E-3,-7.6629693E-4,3.143531E-3,-1.9126337E-3,-8.562794E-4,3.1547972E-3,-1.0556738E-4,4.88332E-3,1.106909E-4,-1.8539676E-2,8.1247033E-4,-4.814637E-3,5.3307763E-3,5.0010026E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,-1,27,29,31,-1,-1,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,-1,53,55,-1,-1,57,59,61,63,65,-1,67,69,71,73,75,-1,77,79,-1,81,-1,83,-1,-1,-1,-1,85,87,-1,89,-1,-1,91,93,95,-1,-1,-1,97,-1,-1,99,-1,-1,101,-1,103,105,107,109,-1,-1,111,-1,113,115,-1,117,-1,119,-1,-1,-1,-1,-1,-1,121,123,125,127,-1,129,131,133,135,137,-1,-1,139,-1,141,-1,143,145,-1,147,-1,149,151,-1,-1,-1,153,155,-1,157,-1,159,-1,-1,161,163,165,167,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0104896E-1,2.1625483E-1,3.575926E-1,1.7510304E-1,2.7991465E-1,2.6863444E-1,1.5710902E-1,6.005004E-2,1.2673634E-1,1.2794484E-1,0E0,1.4692304E-1,1.1465365E-1,0E0,7.9119E-2,0E0,6.6351265E-2,4.729026E-2,1.2727444E-1,0E0,0E0,2.7988851E-2,1.1830098E-1,1.771107E-2,6.0662955E-2,4.5457244E-2,1.11257784E-1,0E0,4.653603E-2,0E0,2.7382812E-2,4.5775697E-2,1.2757272E-1,0E0,0E0,4.7860697E-2,1.1685783E-1,0E0,0E0,2.2388108E-2,6.7287296E-2,6.492203E-2,4.6105564E-2,5.6021012E-2,0E0,4.4390082E-2,3.600219E-2,4.1570447E-2,2.7391069E-2,5.4596215E-2,0E0,1.21788606E-1,8.208712E-2,0E0,3.1012716E-2,0E0,7.517898E-2,0E0,0E0,0E0,0E0,2.1703668E-2,2.8313294E-2,0E0,5.171591E-2,0E0,0E0,5.514896E-2,6.0233835E-2,5.7708204E-2,0E0,0E0,0E0,1.7521743E-2,0E0,0E0,2.0508885E-2,0E0,0E0,8.211104E-2,0E0,4.5244843E-2,3.7531644E-2,5.8308467E-2,7.5340986E-2,0E0,0E0,1.8947946E-2,0E0,3.6396503E-2,3.254538E-2,0E0,3.4942567E-2,0E0,3.644753E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2963577E-1,2.5847668E-2,3.123142E-2,2.3909206E-2,0E0,1.6824268E-2,4.0927082E-2,2.7162075E-2,1.7338146E-1,1.0526112E-1,0E0,0E0,3.1133235E-2,0E0,1.7545E-2,0E0,2.5233984E-2,3.254157E-2,0E0,3.282278E-2,0E0,5.1932782E-2,1.9567434E-2,0E0,0E0,0E0,1.8380256E-2,2.5860652E-2,0E0,2.9747263E-2,0E0,3.6641683E-2,0E0,0E0,9.136312E-2,6.381247E-1,8.542231E-2,9.036061E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,35,35,36,36,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,52,52,54,54,56,56,61,61,62,62,64,64,67,67,68,68,69,69,73,73,76,76,79,79,81,81,82,82,83,83,84,84,87,87,89,89,90,90,92,92,94,94,101,101,102,102,103,103,104,104,106,106,107,107,108,108,109,109,110,110,113,113,115,115,117,117,118,118,120,120,122,122,123,123,127,127,128,128,130,130,132,132,135,135,136,136,137,137,138,138],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,-1,28,30,32,-1,-1,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,-1,54,56,-1,-1,58,60,62,64,66,-1,68,70,72,74,76,-1,78,80,-1,82,-1,84,-1,-1,-1,-1,86,88,-1,90,-1,-1,92,94,96,-1,-1,-1,98,-1,-1,100,-1,-1,102,-1,104,106,108,110,-1,-1,112,-1,114,116,-1,118,-1,120,-1,-1,-1,-1,-1,-1,122,124,126,128,-1,130,132,134,136,138,-1,-1,140,-1,142,-1,144,146,-1,148,-1,150,152,-1,-1,-1,154,156,-1,158,-1,160,-1,-1,162,164,166,168,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.48E2,6.0692043E0,5.97E3,3.06E2,1.6493898E6,2.856934E7,1.2646534E4,6E0,3.4E1,3.3817584E7,-1.5898887E-2,2.04115E5,8.620714E2,1.065097E-2,2.7021693E8,-6.490355E-3,2.4E1,2.00087E5,6.593462E4,2.0630606E-3,-6.598452E-3,1.5E1,3.3193566E5,2.476353E-2,1.9885094E-8,1.4817301E3,8.81059E5,2.874033E-3,6.769866E7,4.9296217E-3,2.4E1,9.7199225E-1,1.65E2,3.0784048E-3,8.191784E-3,2.5454E4,9.9688476E-1,2.562346E-3,-1.1997796E-3,1.6E1,1.7308458E1,1E0,1.990351E1,6.9E1,-5.0172056E-3,2E1,4E0,1.8949389E6,1.1913043E0,1.1732674E1,-3.2473574E-5,1.01E2,2.2E1,-4.862707E-3,2.3600838E6,7.0062275E-3,2.28E0,-3.93082E-3,4.8236318E-5,-1.1115208E-2,-2.5332111E-3,1.0292E4,8.83E3,1.5074437E-4,2.3385885E0,5.898369E-3,6.16405E-4,8E0,1E0,3.14403E5,-9.85457E-4,2.3244622E-3,-1.6215943E-3,3.1857144E2,2.9494262E-3,-1.9061627E-3,1.5E1,-2.5501123E-4,-1.1135857E-2,2.9652428E6,5.153377E-3,2.7491847E5,8.35684E4,8.7151864E7,3E0,-3.0645975E-3,8.354735E-4,1.925508E6,4.3436084E-3,2.4508352E9,1.4176E4,1.3335259E-3,5.2083335E0,2.3376613E-3,6.44E2,-1.4053211E-3,-8.620643E-3,-3.4271897E-4,2.7534587E-3,-4.703675E-3,-1.0241547E-2,1.2954546E2,2.8198E2,9.99E2,6E0,-5.1386394E-3,2.0227273E0,1.6446976E3,2.8543878E6,4.5620965E6,1.0213677E3,2.9816085E-3,-1.9684598E-4,1.3009709E0,2.7542224E-3,2.0669324E3,4.383693E-3,3.3817584E7,1.3085094E2,-5.083454E-3,1.0146154E2,-8.612028E-3,2.4E1,7.318515E6,3.5435753E-3,1.8349178E-3,-2.3374162E-3,8.6363555E4,3.9416668E1,-2.0938818E-3,3.164034E0,4.4817934E-3,1.5272727E0,6.5334155E-3,1.8484099E-3,2.517059E6,3.289E3,1E0,6.797565E1,9.635364E-3,4.96547E-3,2.6512677E-3,-1.0749991E-3,-4.4982047E-3,-2.5811733E-3,1.6035568E-3,-3.0436222E-3,2.394242E-3,-2.902059E-3,-6.9392077E-4,-2.77782E-3,-8.605957E-4,1.2717951E-3,-1.9096925E-3,1.6748591E-3,2.8841703E-3,-7.6629693E-4,3.143531E-3,-1.9126337E-3,-8.562794E-4,3.1547972E-3,-1.0556738E-4,4.88332E-3,1.106909E-4,-1.8539676E-2,8.1247033E-4,-4.814637E-3,5.3307763E-3,5.0010026E-4],"split_indices":[2,68,2,2,60,60,48,67,0,7,0,5,4,0,47,0,2,5,43,0,0,0,43,53,52,67,9,0,7,0,0,53,12,0,0,9,69,0,0,8,61,29,73,8,0,3,8,60,71,73,0,0,71,0,62,0,71,0,0,0,0,44,2,0,53,0,0,3,20,9,0,0,0,67,0,0,3,0,0,43,0,66,48,7,8,0,0,43,0,12,2,0,69,0,1,0,0,0,0,0,0,4,67,2,8,0,68,48,60,62,70,0,0,68,0,4,0,7,4,0,4,0,8,60,0,0,0,48,71,0,53,0,71,0,0,62,44,79,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,4.83E2,5.59E2,4.66E2,1.7E1,4.28E2,1.31E2,1.37E2,3.29E2,1.3E1,4E0,3.95E2,3.3E1,5E0,1.26E2,8E0,1.29E2,6.6E1,2.63E2,6E0,7E0,1.2E1,3.83E2,9E0,2.4E1,1E2,2.6E1,4E0,1.25E2,5E0,6.1E1,2.2E1,2.41E2,7E0,5E0,8.1E1,3.02E2,5E0,4E0,1.5E1,9E0,4.7E1,5.3E1,2.1E1,5E0,1.1E2,1.5E1,2.6E1,3.5E1,1.8E1,4E0,8E0,2.33E2,5E0,7.6E1,6E0,2.96E2,1.1E1,4E0,5E0,4E0,1.1E1,3.6E1,5E0,4.8E1,5E0,1.6E1,8.8E1,2.2E1,1.1E1,4E0,7E0,1.9E1,2.1E1,1.4E1,7E0,1.1E1,4E0,4E0,2.29E2,4E0,5.4E1,2.2E1,4.1E1,2.55E2,4E0,7E0,1.5E1,2.1E1,2.7E1,2.1E1,6E0,8.2E1,9E0,1.3E1,4E0,7E0,1.7E1,4E0,7E0,4E0,1.86E2,4.3E1,2.2E1,3.2E1,5E0,1.7E1,2.7E1,1.4E1,6.7E1,1.88E2,8E0,7E0,2E1,7E0,9E0,1.2E1,7.1E1,1.1E1,5E0,8E0,4E0,1.82E2,3.7E1,6E0,4E0,1.8E1,1.1E1,2.1E1,9E0,8E0,6E0,2.1E1,9E0,5E0,4.5E1,2.2E1,1.54E2,3.4E1,8E0,1.2E1,4E0,5E0,3E1,4.1E1,6E0,5E0,4E0,4E0,1.52E2,3E1,1.4E1,2.3E1,7E0,4E0,1.6E1,5E0,4E0,4E0,1.5E1,6E0,3.6E1,9E0,1.8E1,4E0,1.49E2,5E0,2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"169","size_leaf_vector":"1"}},{"base_weights":[-4.5186665E-4,-1.232754E-2,6.791436E-2,-7.3019736E-3,-1.1986783E-1,5.622875E-2,9.821498E-3,-3.862031E-2,4.5574345E-3,-5.202672E-2,-2.2494516E-1,6.8715096E-2,-1.8988553E-2,-3.50905E-2,-8.798329E-3,6.2457863E-3,2.3689324E-3,-1.2038623E-2,-9.681999E-3,-1.6154714E-2,-1.4987651E-3,3.7945434E-2,1.0940608E-1,1.04084895E-4,-3.3479459E-3,-4.1402604E-2,3.1994343E-2,-1.7647162E-2,1.6856538E-2,2.3580734E-3,-1.8790762E-3,5.4157972E-2,-7.975848E-2,1.3890329E-1,5.924813E-2,-3.4592424E-2,-9.539265E-3,9.611605E-3,3.8169099E-3,-7.954144E-3,-5.5693094E-2,7.216566E-4,3.537732E-2,7.475748E-3,3.7811138E-2,-6.551623E-3,-8.095593E-5,1.509483E-3,1.5291207E-1,3.152177E-2,4.6366863E-3,-1.8541634E-2,-5.3614337E-2,2.7443261E-3,-1.911738E-3,-2.6344648E-2,1.2626461E-2,1.1205216E-3,-6.4910285E-2,7.618614E-3,-8.210358E-3,4.5564607E-2,-4.3693557E-2,4.064799E-4,5.8580212E-2,7.876785E-3,2.772825E-3,2.7803134E-3,-7.7249296E-4,-5.1793877E-2,7.0877206E-3,-7.6312795E-2,-3.074104E-2,-3.449613E-2,6.624884E-2,4.4400066E-2,-7.1777026E-3,-3.344104E-2,-1.2182505E-1,1.3140672E-2,-7.1245424E-2,6.1933473E-2,-9.346008E-5,-1.1465894E-2,2.7383037E-2,1.8956016E-3,-1.2820043E-3,6.8836E-2,-8.67289E-4,-2.6966687E-2,-8.826871E-2,1.7107772E-2,-5.2004624E-3,5.1355433E-5,-8.601864E-2,-4.171948E-2,8.981079E-3,1.6906897E-2,-4.8552163E-2,4.93053E-4,5.016087E-3,6.374375E-2,-6.5613067E-4,-4.958542E-2,1.5265792E-2,1.6842806E-3,-2.0857288E-3,-7.179622E-3,-2.607858E-3,1.9787349E-2,-5.6860816E-2,-1.5366844E-4,-4.8411638E-3,3.7294164E-2,9.580491E-2,-3.755522E-2,4.569833E-2,-2.7183776E-3,7.637889E-2,3.7102813E-3,7.568201E-4,-3.0836621E-3,1.0747998E-3,-5.392878E-3,-3.3006503E-4,3.1681757E-3,4.949889E-5,-1.0641998E-3,-4.9084323E-3,-2.3102786E-3,4.358501E-4,2.782167E-3,-1.3352493E-3,2.3087817E-3,-2.8223842E-3,-3.4552247E-3,-1.4551475E-3,2.0838044E-4,3.847151E-3,-6.8767555E-4,-3.985468E-3,4.282039E-3,1.8890017E-4,-3.0996418E-4,1.7690883E-3,-5.667811E-3,-1.0058801E-3,5.650749E-4,3.2458662E-3,5.065372E-3,-4.668861E-4,-2.5756788E-3,9.3603204E-4,3.759572E-3,-1.0161304E-3,8.382565E-4,5.182506E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,-1,-1,27,29,-1,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,43,45,47,49,51,-1,53,-1,55,57,59,61,-1,63,-1,-1,-1,65,67,-1,69,71,-1,-1,73,75,-1,77,79,-1,81,83,85,87,-1,-1,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,-1,117,-1,-1,119,-1,121,123,125,-1,-1,127,129,131,133,135,-1,-1,137,-1,139,141,-1,-1,-1,-1,143,145,-1,-1,147,149,151,153,-1,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.338413E-1,4.7201926E-1,2.4502254E-1,3.1105202E-1,2.6631075E-1,1.3466966E-1,0E0,1.1657637E-1,1.7350456E-1,1.5497485E-1,3.4882414E-1,1.4859837E-1,2.401888E-2,9.7023636E-2,0E0,0E0,1.7399865E-1,4.00082E-2,0E0,0E0,0E0,1.3980736E-1,6.870532E-2,0E0,0E0,2.2557074E-1,2.1868177E-2,9.248954E-2,1.0396752E-1,0E0,0E0,1.047533E-1,4.3317165E-2,4.2624593E-2,2.0775795E-2,6.0069248E-2,0E0,4.0541988E-2,0E0,7.678715E-2,3.915572E-2,2.3149186E-1,1.3248476E-1,0E0,4.3480344E-2,0E0,0E0,0E0,3.550458E-2,2.0362806E-2,0E0,9.4112806E-2,4.490024E-2,0E0,0E0,8.304616E-2,6.0726658E-2,0E0,7.831499E-2,7.976023E-2,0E0,1.0762718E-1,2.829521E-1,2.5159061E-2,2.9851675E-2,0E0,0E0,0E0,0E0,4.1086927E-2,7.745904E-2,3.3911705E-2,2.0870771E-2,7.235059E-2,2.0012569E-2,4.2847946E-2,5.78636E-2,2.7009934E-2,2.6354477E-2,8.0222264E-2,2.6990686E-2,8.489132E-2,6.861842E-2,0E0,6.7863144E-2,0E0,0E0,1.7945185E-2,0E0,6.149973E-2,4.303725E-2,4.862351E-2,0E0,0E0,4.851055E-2,1.6157545E-2,2.3093084E-2,5.9796385E-2,3.2547817E-2,0E0,0E0,3.201489E-2,0E0,2.5446285E-2,3.4234248E-2,0E0,0E0,0E0,0E0,7.6748595E-2,3.197217E-2,0E0,0E0,5.118967E-2,6.559163E-2,2.4274735E-2,4.526421E-2,0E0,1.8811878E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,21,21,22,22,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,37,37,39,39,40,40,41,41,42,42,44,44,48,48,49,49,51,51,52,52,55,55,56,56,58,58,59,59,61,61,62,62,63,63,64,64,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,82,82,84,84,87,87,89,89,90,90,91,91,94,94,95,95,96,96,97,97,98,98,101,101,103,103,104,104,109,109,110,110,113,113,114,114,115,115,116,116,118,118],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,-1,-1,28,30,-1,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,44,46,48,50,52,-1,54,-1,56,58,60,62,-1,64,-1,-1,-1,66,68,-1,70,72,-1,-1,74,76,-1,78,80,-1,82,84,86,88,-1,-1,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,-1,118,-1,-1,120,-1,122,124,126,-1,-1,128,130,132,134,136,-1,-1,138,-1,140,142,-1,-1,-1,-1,144,146,-1,-1,148,150,152,154,-1,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.467E3,6.0692043E0,1.0534078E4,1.5152774E2,4.9E1,7.3274844E9,9.821498E-3,4.5E1,7.336111E4,2.0778275E-1,2.8E1,1.6975454E3,3.9661028E7,8.8474586E8,-8.798329E-3,6.2457863E-3,6.285983E5,2.857143E0,-9.681999E-3,-1.6154714E-2,-1.4987651E-3,3.0561172E5,1.7493458E7,1.04084895E-4,-3.3479459E-3,4.5865917E8,1.9290142E0,2.4218928E7,4.4935583E2,2.3580734E-3,-1.8790762E-3,1.81E2,1.36E2,6.6358675E8,1.2E1,4.201202E-2,-9.539265E-3,3.5714287E-1,3.8169099E-3,1.7161779E-1,6.613774E5,2.511E3,2.9366477E0,7.475748E-3,1.3543621E6,-6.551623E-3,-8.095593E-5,1.509483E-3,2.53E2,3.01E2,4.6366863E-3,3.54E1,3.1157124E-1,2.7443261E-3,-1.911738E-3,5.7E3,1.3586957E1,1.1205216E-3,9.6203804E-1,1.70595E5,-8.210358E-3,2E0,6.9508715E0,4.9538185E6,1E0,7.876785E-3,2.772825E-3,2.7803134E-3,-7.7249296E-4,6.0921145E4,1E0,2.6412E5,1.547E3,5.539245E6,2.5393645E5,5.9908E6,1.9E1,8.7961E4,2.909091E0,6.346204E-7,1.162E3,6.965855E9,2.9308079E1,-1.1465894E-2,1.3E1,1.8956016E-3,-1.2820043E-3,2.4262331E5,-8.67289E-4,7.785714E0,3.5070792E2,2.3603575E5,-5.2004624E-3,5.1355433E-5,1.1010101E0,9.29023E5,1.5E1,4.313463E-2,1.8987958E0,4.93053E-4,5.016087E-3,5.4603375E5,-6.5613067E-4,9.783615E4,1.8916E2,1.6842806E-3,-2.0857288E-3,-7.179622E-3,-2.607858E-3,1.71E2,1.4E1,-1.5366844E-4,-4.8411638E-3,2.62E3,2.1199985E-7,3.1566668E1,1.8729467E8,-2.7183776E-3,3.307E3,3.7102813E-3,7.568201E-4,-3.0836621E-3,1.0747998E-3,-5.392878E-3,-3.3006503E-4,3.1681757E-3,4.949889E-5,-1.0641998E-3,-4.9084323E-3,-2.3102786E-3,4.358501E-4,2.782167E-3,-1.3352493E-3,2.3087817E-3,-2.8223842E-3,-3.4552247E-3,-1.4551475E-3,2.0838044E-4,3.847151E-3,-6.8767555E-4,-3.985468E-3,4.282039E-3,1.8890017E-4,-3.0996418E-4,1.7690883E-3,-5.667811E-3,-1.0058801E-3,5.650749E-4,3.2458662E-3,5.065372E-3,-4.668861E-4,-2.5756788E-3,9.3603204E-4,3.759572E-3,-1.0161304E-3,8.382565E-4,5.182506E-3],"split_indices":[2,68,4,67,3,5,0,6,60,53,8,67,60,5,0,0,43,73,0,0,0,48,66,0,0,5,68,12,70,0,0,0,0,46,8,57,0,68,0,53,60,2,57,0,43,0,0,0,8,0,0,70,53,0,0,10,73,0,53,1,0,32,71,60,6,0,0,0,0,60,30,7,10,5,43,9,73,1,68,52,10,46,71,0,3,0,0,48,0,67,48,60,0,0,68,62,3,53,68,0,0,60,0,48,67,0,0,0,0,10,3,0,0,2,52,73,7,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,8.74E2,1.51E2,8.36E2,3.8E1,1.41E2,1E1,2.29E2,6.07E2,2.4E1,1.4E1,1.21E2,2E1,2.25E2,4E0,9E0,5.98E2,2E1,4E0,8E0,6E0,7E1,5.1E1,1.5E1,5E0,2.06E2,1.9E1,2.51E2,3.47E2,6E0,1.4E1,6.2E1,8E0,3.1E1,2E1,1.99E2,7E0,1.4E1,5E0,2.01E2,5E1,1.86E2,1.61E2,7E0,5.5E1,4E0,4E0,4E0,2.7E1,1.3E1,7E0,1.09E2,9E1,7E0,7E0,1.06E2,9.5E1,5E0,4.5E1,1.8E2,6E0,1.43E2,1.8E1,2E1,3.5E1,2.2E1,5E0,8E0,5E0,4.7E1,6.2E1,4.4E1,4.6E1,9.8E1,8E0,3.6E1,5.9E1,3E1,1.5E1,1.69E2,1.1E1,1.05E2,3.8E1,4E0,1.4E1,8E0,1.2E1,3.1E1,4E0,2.9E1,1.8E1,5.8E1,4E0,5E0,3.9E1,3.6E1,1E1,2.1E1,7.7E1,4E0,4E0,2.7E1,9E0,2E1,3.9E1,4E0,2.6E1,9E0,6E0,1.55E2,1.4E1,4E0,7E0,6.2E1,4.3E1,2.1E1,1.7E1,5E0,9E0,2.5E1,6E0,1.6E1,1.3E1,1.3E1,5E0,1.3E1,4.5E1,1E1,2.9E1,3.1E1,5E0,4E0,6E0,1.5E1,6E0,2.9E1,4.8E1,7E0,2E1,1.1E1,9E0,4E0,3.5E1,6.4E1,9.1E1,4E0,1E1,3.6E1,2.6E1,3.8E1,5E0,1.6E1,5E0,1.1E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-5.610202E-3,-2.4882287E-2,1.9601159E-2,-2.1088654E-2,-1.0734692E-1,3.3337165E-2,-3.329409E-2,-1.920235E-2,-6.76749E-3,-5.0205804E-2,-1.2932322E-2,7.034963E-2,1.8605804E-2,-7.561385E-2,-4.219084E-3,-3.5982504E-2,-8.757486E-3,-8.6721204E-2,2.284558E-3,3.6221486E-2,1.01760015E-1,6.303687E-3,7.63384E-2,-1.0180549E-1,7.938385E-4,-3.498649E-2,5.13732E-2,-3.1398974E-2,-9.969479E-3,-4.586434E-3,-5.447526E-2,1.0931337E-3,-1.2901136E-1,5.254168E-2,-1.9473212E-3,1.251664E-1,2.3713157E-2,-1.3147849E-1,1.3926416E-2,1.0949343E-1,1.1473775E-2,-6.844725E-2,-1.0083909E-2,1.8404823E-2,-7.399956E-2,6.8098265E-3,1.8153418E-2,-2.3674041E-2,-1.0536943E-1,-1.2631728E-3,-6.1594028E-2,-7.3459116E-3,-1.8739425E-2,-8.61497E-3,-3.212482E-3,2.9867974E-3,-2.1978188E-3,8.739188E-3,9.657545E-2,-1.7832507E-3,3.1176666E-3,-1.6504169E-2,2.2435505E-3,1.8965283E-2,-5.8336486E-3,6.9539277E-3,7.605873E-2,-3.957373E-3,5.09897E-2,-4.846418E-3,1.8332672E-4,-4.652249E-4,3.1203397E-3,-9.6827786E-4,-1.2031306E-1,3.271095E-3,-1.2215601E-2,-3.4606993E-2,3.2107074E-2,-9.164775E-3,-4.7183715E-2,-4.5385556E-3,7.5813584E-2,-7.9530117E-4,-5.052743E-3,1.1326865E-3,-4.520894E-2,5.964977E-3,3.0496719E-3,1.5158699E-2,6.2111383E-3,2.0621126E-3,6.073613E-3,3.6040123E-3,-1.1279836E-4,-2.0216068E-3,-8.1687E-3,-2.8147774E-3,1.1237209E-3,-4.480446E-3,-3.0128418E-2,-1.9423816E-3,2.563411E-3,-6.5087766E-4,-4.4960114E-3,-8.701281E-3,5.6998145E-2,4.872609E-3,6.0986367E-4,-1.0892138E-2,-4.841695E-3,4.283715E-2,2.9996873E-3,-2.5197765E-2,-3.8575483E-3,4.241282E-2,-1.3967468E-2,-1.735478E-4,1.181702E-1,-2.3454702E-3,1.1158811E-3,7.434839E-2,2.5124997E-2,-8.852257E-3,4.6914063E-2,-1.4146487E-3,1.3151615E-3,5.5539194E-3,1.0102954E-3,-3.1213704E-3,-4.4343178E-4,7.643544E-3,2.1832879E-3,5.292591E-4,4.1281376E-3,-1.8285833E-3,1.8947485E-3,-2.8258418E-3,2.3657916E-4,6.076838E-3,8.2495087E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,29,31,-1,33,35,37,39,41,-1,43,45,47,-1,49,51,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,81,83,-1,85,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,89,-1,-1,91,-1,93,-1,-1,-1,-1,-1,95,-1,97,99,101,-1,103,105,107,-1,-1,-1,109,-1,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,-1,115,117,-1,-1,119,-1,121,123,125,-1,127,129,-1,131,-1,-1,133,135,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.921937E-1,1.7730671E-1,3.200739E-1,1.2728222E-1,2.3248035E-1,1.8898124E-1,1.1165167E-1,9.485452E-2,0E0,7.706805E-2,0E0,1.0243991E-1,1.777873E-1,9.1677904E-2,9.587104E-2,1.6465202E-1,6.3822396E-2,7.307405E-2,0E0,6.488806E-2,9.019852E-2,2.2028844E-1,9.33696E-2,9.772739E-2,0E0,7.634847E-2,6.1125174E-2,1.140745E-1,0E0,5.8462597E-2,1.0161383E-1,0E0,2.4627939E-2,5.2779123E-2,0E0,5.3091526E-2,3.8061373E-2,4.6445948E-1,1.4227024E-1,2.8332144E-2,6.578051E-2,6.3775264E-2,0E0,2.3066547E-2,4.9064934E-2,0E0,2.644825E-2,1.14543654E-1,9.346111E-2,7.431366E-2,3.4161016E-2,0E0,2.6298203E-2,0E0,0E0,0E0,0E0,0E0,2.031818E-2,0E0,0E0,0E0,0E0,8.236056E-2,0E0,0E0,2.5471538E-2,0E0,1.7515557E-2,0E0,0E0,0E0,0E0,0E0,3.984329E-2,0E0,2.1060588E-2,4.0656433E-2,5.607393E-2,0E0,2.0198671E-2,7.273318E-2,2.089008E-2,0E0,0E0,0E0,2.7488498E-2,0E0,0E0,6.2804975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.657894E-2,0E0,0E0,0E0,0E0,7.192415E-2,6.734069E-2,0E0,0E0,1.5201393E-2,0E0,2.9945523E-2,6.859696E-2,4.0450916E-2,0E0,3.7332352E-2,5.5786673E-2,0E0,2.2354305E-2,0E0,0E0,1.7852925E-2,4.0688187E-2,7.633678E-2,6.650075E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,52,52,58,58,63,63,66,66,68,68,74,74,76,76,77,77,78,78,80,80,81,81,82,82,86,86,89,89,100,100,105,105,106,106,109,109,111,111,112,112,113,113,115,115,116,116,118,118,121,121,122,122,123,123,124,124],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,30,32,-1,34,36,38,40,42,-1,44,46,48,-1,50,52,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,82,84,-1,86,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,90,-1,-1,92,-1,94,-1,-1,-1,-1,-1,96,-1,98,100,102,-1,104,106,108,-1,-1,-1,110,-1,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,-1,116,118,-1,-1,120,-1,122,124,126,-1,128,130,-1,132,-1,-1,134,136,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.850926E2,6.0692043E0,1E0,4.093403E6,1.6493898E6,2.165E3,5E0,1.8149019E2,-6.76749E-3,7.86127E6,-1.2932322E-2,7.257846E2,3.9152692E6,1.7308458E1,2.3751075E5,3.4E1,1E0,6.48334E5,2.284558E-3,4.1104166E8,1.9719212E1,1.2825651E0,1.5277778E0,4E0,7.938385E-4,8.7390656E2,3.2888E4,1E0,-9.969479E-3,4.2438595E1,2.1410204E2,1.0931337E-3,1.70698E6,2.4E1,-1.9473212E-3,1.55E2,1.10198E7,3.0984934E8,7.1E1,6.624E3,1E0,1.3026532E6,-1.0083909E-2,1.28636E7,4.3400474E0,6.8098265E-3,2.8827406E5,4.436827E7,3.7936746E3,2.89196E6,1.7181714E7,-7.3459116E-3,3.3272727E0,-8.61497E-3,-3.212482E-3,2.9867974E-3,-2.1978188E-3,8.739188E-3,6.9508715E0,-1.7832507E-3,3.1176666E-3,-1.6504169E-2,2.2435505E-3,4.1E1,-5.8336486E-3,6.9539277E-3,7.858646E-3,-3.957373E-3,2.7578741E1,-4.846418E-3,1.8332672E-4,-4.652249E-4,3.1203397E-3,-9.6827786E-4,4E0,3.271095E-3,1.332E3,9E0,1.45064E5,-9.164775E-3,6.666667E-1,1.9719212E1,1.7903225E0,-7.9530117E-4,-5.052743E-3,1.1326865E-3,1.1111689E8,5.964977E-3,3.0496719E-3,2E0,6.2111383E-3,2.0621126E-3,6.073613E-3,3.6040123E-3,-1.1279836E-4,-2.0216068E-3,-8.1687E-3,-2.8147774E-3,1.1237209E-3,-4.480446E-3,6.0147805E0,-1.9423816E-3,2.563411E-3,-6.5087766E-4,-4.4960114E-3,4.8679228E5,3.3E1,4.872609E-3,6.0986367E-4,4.7272725E0,-4.841695E-3,9.237895E0,1.9616238E2,5.25E0,-3.8575483E-3,1.9095E4,9.5679015E-2,-1.735478E-4,4.44E2,-2.3454702E-3,1.1158811E-3,1.0315458E8,5.658231E9,1E0,3.6828618E8,-1.4146487E-3,1.3151615E-3,5.5539194E-3,1.0102954E-3,-3.1213704E-3,-4.4343178E-4,7.643544E-3,2.1832879E-3,5.292591E-4,4.1281376E-3,-1.8285833E-3,1.8947485E-3,-2.8258418E-3,2.3657916E-4,6.076838E-3,8.2495087E-4],"split_indices":[67,68,6,66,60,44,6,70,0,5,0,4,43,61,48,3,19,46,0,7,71,69,68,6,0,4,9,30,0,73,67,0,5,3,0,10,62,7,3,2,27,43,0,60,61,0,48,5,48,62,12,0,69,0,0,0,0,0,71,0,0,0,0,3,0,0,49,0,71,0,0,0,0,0,8,0,44,70,7,0,71,71,68,0,0,0,7,0,0,32,0,0,0,0,0,0,0,0,0,0,69,0,0,0,0,60,8,0,0,69,0,71,71,69,0,12,73,0,0,0,0,7,46,29,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.011E3,5.73E2,4.38E2,5.49E2,2.4E1,3.48E2,9E1,5.42E2,7E0,1.9E1,5E0,9.8E1,2.5E2,3.6E1,5.4E1,2.07E2,3.35E2,1.4E1,5E0,4.8E1,5E1,2.07E2,4.3E1,2.8E1,8E0,3.5E1,1.9E1,2.03E2,4E0,3.08E2,2.7E1,4E0,1E1,4E1,8E0,3.8E1,1.2E1,1E1,1.97E2,2.8E1,1.5E1,2.3E1,5E0,1.5E1,2E1,4E0,1.5E1,1.85E2,1.8E1,2.92E2,1.6E1,6E0,2.1E1,4E0,6E0,3.6E1,4E0,1E1,2.8E1,5E0,7E0,4E0,6E0,1.91E2,6E0,1.1E1,1.7E1,4E0,1.1E1,1.5E1,8E0,1E1,5E0,1E1,1E1,5E0,1E1,1.55E2,3E1,6E0,1.2E1,2.81E2,1.1E1,9E0,7E0,8E0,1.3E1,1.2E1,1.6E1,1.86E2,5E0,1.2E1,5E0,7E0,4E0,5E0,5E0,4E0,6E0,9E0,1.46E2,7E0,2.3E1,8E0,4E0,2.64E2,1.7E1,7E0,4E0,9E0,4E0,5.6E1,1.3E2,1.35E2,1.1E1,2.4E1,2.4E2,9E0,8E0,4E0,5E0,1.9E1,3.7E1,1.03E2,2.7E1,1.23E2,1.2E1,4E0,2E1,1.7E1,2.23E2,4E0,4E0,4E0,1.5E1,7E0,3E1,2.1E1,8.2E1,6E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[-2.3832999E-3,-1.616509E-2,3.9556604E-2,-9.288826E-3,-1.17473E-1,4.8942212E-2,-3.1051794E-2,-7.5559868E-3,-1.0151071E-2,-1.9056906E-1,-1.5779432E-2,3.2307252E-2,8.266856E-2,4.129138E-3,-5.220338E-2,-5.2623376E-2,1.3472438E-4,4.1708484E-4,-2.4303037E-1,2.2713726E-3,-4.936446E-2,4.6477012E-2,-1.5822701E-2,7.083084E-2,9.563075E-3,-8.976092E-2,4.8905646E-4,-1.0384677E-1,-4.247589E-2,1.3308501E-1,-2.6712634E-3,-1.6859977E-2,-1.3859129E-1,-7.630779E-2,9.139906E-4,2.0981997E-2,7.079216E-2,-7.521709E-3,7.638667E-3,8.336396E-2,-1.6964907E-3,-4.3454796E-2,-8.879673E-3,-6.0306154E-2,-9.06809E-3,-1.346596E-2,-5.7981852E-2,2.0907472E-3,8.423091E-3,2.7398719E-5,-3.8388066E-3,-4.4804253E-2,-1.3787973E-2,-1.5046317E-3,-5.6492793E-3,5.1904593E-2,-3.101977E-2,8.074858E-2,-2.431322E-4,-1.4040481E-3,3.8905915E-2,1.3170636E-1,5.2274413E-2,-3.6403323E-3,1.3329152E-3,-4.2736847E-3,4.4880612E-4,3.5509577E-3,-2.8575424E-2,-8.383783E-2,-2.6361382E-2,-9.664212E-3,2.5248775E-2,1.3585996E-3,-4.5820232E-3,7.898426E-2,5.1168203E-3,9.873965E-4,-6.765529E-3,5.6036357E-3,5.527306E-2,3.8422006E-3,-2.226313E-4,9.6920095E-2,1.0159718E-2,3.888192E-2,6.386809E-3,-5.034832E-3,-1.1351237E-2,-5.5202052E-2,-6.970725E-3,1.6563621E-3,-6.29166E-2,-6.9045634E-3,-8.141564E-3,7.029964E-2,7.913334E-3,4.4405475E-2,6.5875757E-3,-4.0224628E-3,4.6553884E-2,1.0834249E-3,-3.0769308E-3,-1.620746E-3,7.15016E-2,5.501881E-3,1.3413954E-3,4.9138326E-2,-1.5955053E-3,1.1279745E-2,-2.8113066E-3,-3.6164382E-3,-2.0812389E-2,-1.0118706E-3,-4.020491E-3,1.1471406E-2,-2.001307E-2,4.5745578E-2,9.993264E-2,2.6743922E-3,4.2896364E-3,4.2437166E-3,9.256104E-5,3.5194184E-3,-1.9554637E-4,4.8606703E-3,4.9386084E-2,-1.0677816E-3,6.028534E-2,2.2145114E-3,-7.578226E-4,1.1115515E-3,-2.8743246E-3,7.0295564E-4,-4.546169E-3,-7.7078314E-3,-7.253777E-4,8.60132E-4,4.157949E-3,5.8099045E-3,1.5710271E-3,-8.8851004E-5,4.814243E-3,1.20445875E-5,3.7936168E-3,1.7642291E-3,4.20249E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,29,-1,31,-1,33,35,37,39,-1,41,-1,43,45,47,49,-1,51,53,-1,55,57,-1,59,61,-1,63,-1,65,-1,67,69,-1,-1,71,-1,73,-1,-1,-1,75,77,79,-1,-1,81,83,85,-1,-1,-1,-1,-1,87,89,91,93,95,-1,-1,97,99,101,-1,-1,103,-1,-1,105,-1,107,-1,-1,109,111,-1,-1,113,115,-1,117,119,121,-1,-1,123,-1,-1,-1,125,-1,-1,127,-1,129,-1,-1,131,-1,-1,133,135,137,139,141,-1,-1,-1,-1,-1,-1,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.052348E-1,5.478455E-1,1.7315972E-1,2.6781717E-1,3.6866283E-1,1.2580657E-1,8.363092E-2,2.5441167E-1,0E0,3.058567E-1,5.0412472E-2,1.0644382E-1,9.572089E-2,0E0,6.462185E-2,5.120641E-2,2.3463522E-1,0E0,2.4226606E-1,0E0,2.9455896E-2,7.290444E-2,1.2539126E-1,9.7418725E-2,0E0,7.229191E-2,0E0,5.6583375E-2,4.0153474E-2,4.7966853E-2,1.3369302E-1,0E0,1.9652194E-1,1.8136837E-2,0E0,9.809184E-2,4.6616286E-2,0E0,3.9233334E-2,8.780053E-2,0E0,3.549418E-2,0E0,3.0209124E-2,0E0,4.723507E-2,4.6027035E-2,0E0,0E0,1.4617217E-1,0E0,4.3269873E-2,0E0,0E0,0E0,4.7741756E-2,8.511515E-2,4.9072236E-2,0E0,0E0,3.523575E-2,5.232632E-2,4.0679403E-2,0E0,0E0,0E0,0E0,0E0,3.8925767E-2,5.310884E-2,6.593473E-2,1.9735524E-1,1.2908582E-1,0E0,0E0,4.645495E-2,6.1351433E-2,3.0409753E-2,0E0,0E0,5.382988E-2,0E0,0E0,2.5138259E-2,0E0,2.8804958E-2,0E0,0E0,2.869269E-2,1.7417833E-2,0E0,0E0,1.5892394E-2,1.0284511E-1,0E0,2.8898597E-2,5.3594798E-2,3.402076E-2,0E0,0E0,1.6872836E-2,0E0,0E0,0E0,1.7169341E-2,0E0,0E0,2.71656E-2,0E0,1.9051056E-2,0E0,0E0,2.1933664E-2,0E0,0E0,7.472952E-2,1.5502375E-1,3.0738868E-2,2.977541E-2,5.3883057E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.2134607E-2,0E0,1.5508845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,25,25,27,27,28,28,29,29,30,30,32,32,33,33,35,35,36,36,38,38,39,39,41,41,43,43,45,45,46,46,49,49,51,51,55,55,56,56,57,57,60,60,61,61,62,62,68,68,69,69,70,70,71,71,72,72,75,75,76,76,77,77,80,80,83,83,85,85,88,88,89,89,92,92,93,93,95,95,96,96,97,97,100,100,104,104,107,107,109,109,112,112,115,115,116,116,117,117,118,118,119,119,126,126,128,128],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,30,-1,32,-1,34,36,38,40,-1,42,-1,44,46,48,50,-1,52,54,-1,56,58,-1,60,62,-1,64,-1,66,-1,68,70,-1,-1,72,-1,74,-1,-1,-1,76,78,80,-1,-1,82,84,86,-1,-1,-1,-1,-1,88,90,92,94,96,-1,-1,98,100,102,-1,-1,104,-1,-1,106,-1,108,-1,-1,110,112,-1,-1,114,116,-1,118,120,122,-1,-1,124,-1,-1,-1,126,-1,-1,128,-1,130,-1,-1,132,-1,-1,134,136,138,140,142,-1,-1,-1,-1,-1,-1,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,9.932432E0,2.856934E7,1.6776881E7,9.4E1,7.650379E8,2.0263722E8,2.71E2,-1.0151071E-2,1E0,8.303E3,4.88E2,3.8771296E3,4.129138E-3,1.3655363E2,8.8324326E-1,2.04115E5,4.1708484E-4,3.2214062E2,2.2713726E-3,1.23E4,2.35184E5,5.11E2,2E1,9.563075E-3,1.7026364E3,4.8905646E-4,6.737099E7,3.8275862E0,1.766298E2,3.713753E7,-1.6859977E-2,5.7420593E2,8.620714E2,9.139906E-4,2.08E3,2.9E1,-7.521709E-3,2.6172416E10,1.4388715E0,-1.6964907E-3,9.3326636E-2,-8.879673E-3,2.66E2,-9.06809E-3,2.4E1,7.336111E4,2.0907472E-3,8.423091E-3,4.4935583E2,-3.8388066E-3,2.5144437E2,-1.3787973E-2,-1.5046317E-3,-5.6492793E-3,2.8827406E5,9.427676E2,4.029152E8,-2.431322E-4,-1.4040481E-3,3.0977E4,1.188E0,9.059046E6,-3.6403323E-3,1.3329152E-3,-4.2736847E-3,4.4880612E-4,3.5509577E-3,5.4E1,6.48334E5,3.466E3,2.483E3,2.8E0,1.3585996E-3,-4.5820232E-3,1.57922E5,3.2952412E5,1E0,-6.765529E-3,5.6036357E-3,8.484036E5,3.8422006E-3,-2.226313E-4,1.2376238E0,1.0159718E-2,3.116962E7,6.386809E-3,-5.034832E-3,8E0,2.2970297E0,-6.970725E-3,1.6563621E-3,4.8121494E2,1.5357143E0,-8.141564E-3,1.10566504E8,1.0358873E3,2.4434292E7,6.5875757E-3,-4.0224628E-3,1.83154E6,1.0834249E-3,-3.0769308E-3,-1.620746E-3,1.1621469E1,5.501881E-3,1.3413954E-3,4.6750406E4,-1.5955053E-3,2.855464E5,-2.8113066E-3,-3.6164382E-3,2.6412E5,-1.0118706E-3,-4.020491E-3,1.6942337E1,1.16408E5,6.083812E2,2.1609572E7,3.2585382E10,4.2896364E-3,4.2437166E-3,9.256104E-5,3.5194184E-3,-1.9554637E-4,4.8606703E-3,2.0619047E1,-1.0677816E-3,8.072373E1,2.2145114E-3,-7.578226E-4,1.1115515E-3,-2.8743246E-3,7.0295564E-4,-4.546169E-3,-7.7078314E-3,-7.253777E-4,8.60132E-4,4.157949E-3,5.8099045E-3,1.5710271E-3,-8.8851004E-5,4.814243E-3,1.20445875E-5,3.7936168E-3,1.7642291E-3,4.20249E-3],"split_indices":[2,69,60,62,0,5,7,2,0,8,44,0,67,0,71,68,5,0,70,0,10,1,0,3,0,4,0,7,69,4,60,0,67,4,0,44,8,0,46,69,0,53,0,1,0,2,60,0,0,70,0,48,0,0,0,48,4,7,0,0,44,68,43,0,0,0,0,0,2,46,1,2,69,0,0,1,48,6,0,0,43,0,0,69,0,62,0,0,32,68,0,0,48,68,0,5,4,12,0,0,9,0,0,0,71,0,0,48,0,60,0,0,7,0,0,71,7,4,60,46,0,0,0,0,0,0,71,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.045E3,7.87E2,2.58E2,7.38E2,4.9E1,2.28E2,3E1,7.33E2,5E0,2.8E1,2.1E1,1.54E2,7.4E1,4E0,2.6E1,1.06E2,6.27E2,6E0,2.2E1,7E0,1.4E1,1.19E2,3.5E1,6.9E1,5E0,1.6E1,1E1,1.6E1,9E1,1.2E1,6.15E2,9E0,1.3E1,1E1,4E0,5.9E1,6E1,4E0,3.1E1,6.2E1,7E0,1.2E1,4E0,1.2E1,4E0,3.2E1,5.8E1,5E0,7E0,5.96E2,1.9E1,9E0,4E0,6E0,4E0,3.7E1,2.2E1,5.3E1,7E0,1.4E1,1.7E1,2.3E1,3.9E1,8E0,4E0,8E0,4E0,4E0,2.8E1,3.1E1,2.7E1,4.31E2,1.65E2,4E0,5E0,2.3E1,1.4E1,1.8E1,4E0,1.9E1,3.4E1,8E0,9E0,1.8E1,5E0,3.5E1,4E0,4E0,2.4E1,2.3E1,8E0,1E1,1.7E1,4.25E2,6E0,4.5E1,1.2E2,1.6E1,7E0,4E0,1E1,1.4E1,4E0,5E0,2.9E1,1.3E1,5E0,3.1E1,4E0,1.7E1,7E0,1.3E1,1E1,7E0,1E1,1.77E2,2.48E2,2.6E1,1.9E1,1.14E2,6E0,7E0,9E0,6E0,4E0,1E1,1.9E1,4E0,2.7E1,7E0,1E1,5E0,5E0,1.72E2,5E0,6E0,2.42E2,1.7E1,9E0,1.3E1,6E0,1.1E2,4E0,8E0,1.1E1,1.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-2.1796166E-3,-1.5955565E-2,3.6903016E-2,-4.0135726E-2,-3.23034E-3,4.6634834E-2,-2.1685509E-2,-2.7762825E-2,-1.353065E-1,-1.2487384E-3,-9.219274E-3,3.0976562E-2,1.04161955E-1,-7.263966E-2,4.1796535E-2,1.394602E-3,-3.4222674E-2,1.6698814E-3,-2.2671686E-1,-4.4431672E-3,8.920827E-2,8.803175E-2,1.8302405E-2,9.845702E-3,8.400307E-2,-2.9315378E-3,-1.4208128E-1,5.1902547E-3,-1.3311481E-2,-5.879757E-2,-1.7828032E-2,-3.740043E-2,3.3107377E-3,-2.015563E-2,-8.557978E-2,-1.0228534E-2,5.048864E-2,1.3515306E-1,1.9705143E-4,4.662847E-3,-2.0242562E-4,3.177532E-2,-1.5183443E-2,5.7543978E-2,1.295571E-1,-2.3803602E-3,2.5062182E-3,-9.56011E-3,-1.0566904E-3,-3.3320778E-3,1.413662E-3,-4.298241E-2,-9.409306E-3,3.229396E-2,-2.9114697E-2,4.1061343E-4,-3.520769E-3,-1.0646119E-2,7.2326744E-4,-7.701555E-3,-6.67388E-3,8.639728E-2,7.388428E-3,2.4377177E-3,8.070683E-3,1.8092034E-2,9.4918504E-2,8.549523E-4,-3.324707E-2,4.28422E-3,-8.469955E-3,2.0201996E-3,7.243657E-3,-5.9466425E-2,-1.6953155E-3,-1.6933965E-3,5.7313085E-2,-3.6026187E-2,3.3629553E-3,-3.6745653E-3,-7.550389E-2,1.251894E-3,9.9867254E-2,1.8392664E-3,-1.8120952E-2,-4.08743E-3,2.4077382E-2,1.3038072E-1,1.3795211E-3,-3.1572036E-3,-1.1180756E-2,-2.708137E-3,1.3194776E-3,-4.7482144E-2,-6.2304595E-3,1.9251611E-2,-2.003988E-3,4.5912773E-3,1.2934699E-3,-5.9667155E-2,-9.664591E-3,-4.4623144E-2,3.0047023E-3,-8.255444E-4,-9.818465E-2,6.4058197E-3,4.7211863E-2,1.3401976E-3,-1.953801E-3,4.7398186E-3,1.7963966E-2,8.694866E-3,1.9029209E-3,9.350864E-4,-2.6294128E-3,1.8676422E-3,-5.6194555E-2,-1.5724283E-3,1.8410534E-3,-1.9769514E-2,-8.605686E-2,2.4962673E-4,-3.1029787E-2,-3.1112049E-2,-7.1276873E-3,7.8498935E-3,-3.1770267E-2,-5.567701E-3,-9.4650546E-4,-1.8824694E-4,3.7921108E-3,1.5570229E-3,4.896436E-2,-2.8982072E-3,4.0008436E-4,1.7682387E-3,-2.2826153E-3,-4.811528E-3,-9.714286E-4,9.5939456E-4,-3.1087901E-3,1.1182362E-3,-2.1502655E-3,-3.5281477E-3,4.648629E-4,-3.7354912E-4,-5.5599217E-3,-3.7694263E-4,4.5463024E-3,4.512656E-3,7.610985E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,-1,43,45,47,-1,49,51,53,55,-1,-1,57,59,61,63,-1,-1,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,73,-1,75,77,-1,-1,-1,-1,79,-1,81,83,-1,-1,85,87,-1,89,-1,91,-1,-1,93,95,-1,97,99,-1,101,103,-1,105,-1,107,-1,109,111,-1,-1,113,-1,-1,115,-1,117,-1,-1,-1,119,121,123,125,-1,127,-1,129,-1,-1,-1,131,-1,-1,-1,-1,-1,133,-1,-1,135,137,-1,139,141,-1,143,145,-1,-1,-1,-1,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5567956E-1,2.3464392E-1,1.5438774E-1,3.0593616E-1,1.9465898E-1,2.0485252E-1,1.2918834E-1,8.863005E-2,3.7601048E-1,1.4400873E-1,0E0,1.3072094E-1,8.971977E-2,1.0607766E-1,7.147612E-2,0E0,8.396293E-2,3.823042E-2,4.9504668E-1,1.5337855E-1,6.576924E-2,3.977576E-2,6.860643E-2,0E0,4.4373453E-2,3.613718E-2,8.030677E-2,0E0,3.061473E-2,1.8370345E-1,7.335113E-2,1.6844671E-2,0E0,0E0,1.7958742E-1,1.467779E-1,7.055449E-2,2.6616395E-2,0E0,0E0,0E0,9.159539E-2,2.739149E-2,6.8153255E-2,2.8207362E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.2303612E-2,0E0,4.3501407E-2,7.6577306E-2,0E0,0E0,0E0,0E0,1.1685641E-1,0E0,1.7262861E-2,1.8917616E-2,0E0,0E0,5.9535533E-2,3.9941788E-2,0E0,2.1692153E-2,0E0,2.2101728E-2,0E0,0E0,4.3291867E-2,2.091658E-2,0E0,1.9415442E-2,6.112148E-2,0E0,1.1117773E-1,2.9618219E-2,0E0,3.4303486E-2,0E0,1.603558E-2,0E0,3.9438218E-2,5.4136246E-2,0E0,0E0,2.8455315E-2,0E0,0E0,3.9800413E-2,0E0,1.8918943E-2,0E0,0E0,0E0,5.304885E-2,1.5620215E-2,7.965191E-2,5.9221376E-2,0E0,2.7026638E-2,0E0,1.8218156E-2,0E0,0E0,0E0,4.112212E-2,0E0,0E0,0E0,0E0,0E0,1.9625574E-2,0E0,0E0,3.973873E-2,3.4279972E-2,0E0,3.8961586E-2,4.551473E-2,0E0,5.815331E-2,8.9237005E-2,0E0,0E0,0E0,0E0,5.161716E-2,4.2707123E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,41,41,42,42,43,43,44,44,51,51,53,53,54,54,59,59,61,61,62,62,65,65,66,66,68,68,70,70,73,73,74,74,76,76,77,77,79,79,80,80,82,82,84,84,86,86,87,87,90,90,93,93,95,95,99,99,100,100,101,101,102,102,104,104,106,106,110,110,116,116,119,119,120,120,122,122,123,123,125,125,126,126,131,131,132,132],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,-1,44,46,48,-1,50,52,54,56,-1,-1,58,60,62,64,-1,-1,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,74,-1,76,78,-1,-1,-1,-1,80,-1,82,84,-1,-1,86,88,-1,90,-1,92,-1,-1,94,96,-1,98,100,-1,102,104,-1,106,-1,108,-1,110,112,-1,-1,114,-1,-1,116,-1,118,-1,-1,-1,120,122,124,126,-1,128,-1,130,-1,-1,-1,132,-1,-1,-1,-1,-1,134,-1,-1,136,138,-1,140,142,-1,144,146,-1,-1,-1,-1,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.167E3,1E0,2.7555334E7,3.7798166E0,8.432552E6,1.4176E4,3.416447E-2,4.2175066E-1,3.95E2,5.5303917E8,-9.219274E-3,3.9914E4,8.025477E0,1.480315E0,5.735198E-1,1.394602E-3,1.5222145E2,1.3E2,2.277451E2,5.817547E2,8.051603E7,9.033567E6,8E0,9.845702E-3,4.73909E0,8.969567E-9,1.3655363E2,5.1902547E-3,1.01E3,1.1244019E2,1.2E1,8E1,3.3107377E-3,-2.015563E-2,4.5E0,4.093403E6,4.612245E0,8.7568E4,1.9705143E-4,4.662847E-3,-2.0242562E-4,7.659164E1,3.9041237E1,1.4E1,4.29E2,-2.3803602E-3,2.5062182E-3,-9.56011E-3,-1.0566904E-3,-3.3320778E-3,1.413662E-3,3.3527696E-1,-9.409306E-3,3.7788504E7,4.4E1,4.1061343E-4,-3.520769E-3,-1.0646119E-2,7.2326744E-4,2.2968E4,-6.67388E-3,4.7773536E7,1.763E3,2.4377177E-3,8.070683E-3,8.118812E-1,1.3208092E0,8.549523E-4,2.4199E4,4.28422E-3,3.6349E4,2.0201996E-3,7.243657E-3,6.737509E7,4.08E2,-1.6933965E-3,4E0,7.5982756E5,3.3629553E-3,1.5933333E2,1.9067797E1,1.251894E-3,1.8729467E8,1.8392664E-3,6.232639E2,-4.08743E-3,2.3636363E0,4.2904656E8,1.3795211E-3,-3.1572036E-3,1.800554E-2,-2.708137E-3,1.3194776E-3,2.1126761E0,-6.2304595E-3,2.3857143E1,-2.003988E-3,4.5912773E-3,1.2934699E-3,3.7509E4,8.0274E4,1.4209677E2,1E0,-8.255444E-4,2.9308079E1,6.4058197E-3,1.935E3,1.3401976E-3,-1.953801E-3,4.7398186E-3,7.618E3,8.694866E-3,1.9029209E-3,9.350864E-4,-2.6294128E-3,1.8676422E-3,5.0406504E0,-1.5724283E-3,1.8410534E-3,1E0,1.8383686E5,2.4962673E-4,1.3358E4,3.757994E0,-7.1276873E-3,3.7974394E-6,1.2375429E8,-5.567701E-3,-9.4650546E-4,-1.8824694E-4,3.7921108E-3,2.8E1,6.03235E5,-2.8982072E-3,4.0008436E-4,1.7682387E-3,-2.2826153E-3,-4.811528E-3,-9.714286E-4,9.5939456E-4,-3.1087901E-3,1.1182362E-3,-2.1502655E-3,-3.5281477E-3,4.648629E-4,-3.7354912E-4,-5.5599217E-3,-3.7694263E-4,4.5463024E-3,4.512656E-3,7.610985E-4],"split_indices":[2,29,60,68,62,2,72,68,2,5,0,9,71,68,53,0,4,2,67,67,7,60,32,0,50,52,71,0,8,67,3,44,0,0,73,66,69,12,0,0,0,73,73,3,0,0,0,0,0,0,0,71,0,7,3,0,0,0,0,44,0,7,2,0,0,68,68,0,44,0,44,0,0,7,44,0,8,43,0,4,73,0,7,0,67,0,73,7,0,0,72,0,0,69,0,4,0,0,0,1,1,70,19,0,71,0,2,0,0,0,10,0,0,0,0,0,69,0,0,6,48,0,9,73,0,53,7,0,0,0,0,3,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,7.62E2,2.68E2,2.62E2,5E2,2.3E2,3.8E1,2.33E2,2.9E1,4.96E2,4E0,1.82E2,4.8E1,2.1E1,1.7E1,2.3E1,2.1E2,1.2E1,1.7E1,4.8E2,1.6E1,3.2E1,1.5E2,6E0,4.2E1,1.1E1,1E1,7E0,1E1,8.3E1,1.27E2,8E0,4E0,6E0,1.1E1,4.35E2,4.5E1,1E1,6E0,2.8E1,4E0,1.07E2,4.3E1,2.8E1,1.4E1,6E0,5E0,6E0,4E0,4E0,6E0,7.6E1,7E0,2.3E1,1.04E2,4E0,4E0,4E0,7E0,4.28E2,7E0,2.4E1,2.1E1,4E0,6E0,8.9E1,1.8E1,1.5E1,2.8E1,1.8E1,1E1,4E0,1E1,5.4E1,2.2E1,6E0,1.7E1,9.8E1,6E0,4.05E2,2.3E1,5E0,1.9E1,9E0,1.2E1,4E0,8.5E1,1.1E1,7E0,1E1,1.8E1,4E0,6E0,4.8E1,6E0,1.5E1,7E0,6E0,1.1E1,5.1E1,4.7E1,5.6E1,3.49E2,7E0,1.6E1,1E1,9E0,4E0,8E0,5E0,8E1,6E0,5E0,1.1E1,7E0,4E0,4.4E1,4E0,1.1E1,2.1E1,3E1,2.8E1,1.9E1,5.1E1,5E0,3.07E2,4.2E1,1.2E1,4E0,4E0,5E0,5.3E1,2.7E1,4E1,4E0,7E0,1.4E1,2.3E1,7E0,8E0,1.1E1,1.1E1,4E1,7E0,3E2,3.4E1,8E0,4.9E1,4E0,1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-4.394324E-3,-1.4657774E-2,2.9410226E-2,-4.259483E-2,-6.3885613E-3,3.9567206E-2,-2.9179737E-2,-2.3801986E-2,-9.635634E-2,-4.623298E-3,-8.238197E-3,2.7196303E-2,7.688229E-2,3.4070916E-2,-8.423665E-2,-4.785559E-3,-1.7770644E-2,-1.668953E-1,-4.314906E-2,-2.6686054E-2,7.5233844E-3,-4.5937024E-2,3.2708857E-2,1.1982903E-1,4.0392622E-2,4.835181E-3,-4.0303577E-2,-1.1631969E-1,2.1543658E-4,9.7151855E-3,-3.6102008E-2,-2.4053605E-1,-2.906722E-4,-7.035766E-2,2.0623659E-3,-1.682742E-2,-8.981411E-2,3.362829E-2,-2.304895E-3,-6.3896207E-3,1.2290553E-3,7.2906494E-2,1.9254409E-2,1.597043E-1,2.5875287E-3,3.2092799E-3,1.5024359E-3,-4.0444606E-3,6.928843E-4,-8.983178E-3,-3.0318138E-3,2.989017E-2,-2.617782E-3,-4.8118174E-2,3.9548497E-3,-4.6595233E-3,-1.4563231E-2,-6.685542E-3,-1.3469676E-2,-2.8876303E-2,1.8395469E-2,-9.334916E-3,-5.6627978E-2,-3.545487E-3,3.8810607E-2,-1.0196628E-2,5.080911E-2,4.8319746E-2,5.6435647E-3,2.8097006E-2,-3.703017E-3,3.3276656E-3,8.808678E-3,1.952945E-3,-3.3348324E-3,4.6422314E-2,-2.417001E-3,-5.433554E-2,1.2662942E-3,-2.1104075E-3,1.5051365E-3,1.993359E-3,-3.1490335E-3,-2.381475E-2,-6.217197E-3,7.877063E-2,-2.1794997E-2,-7.714783E-2,5.9857586E-5,5.0976045E-2,-4.1335197E-3,-5.2364026E-3,-1.1284767E-2,5.1628323E-3,3.2245267E-2,2.8856678E-3,1.4006702E-4,6.374135E-3,2.2018397E-2,5.0710775E-3,3.3900246E-2,-4.018954E-2,-8.749277E-2,-3.4528847E-3,-1.6585376E-2,4.3282816E-3,8.438426E-4,-3.298905E-3,-5.078036E-3,1.1687091E-4,-4.7057704E-3,7.995242E-2,1.9949615E-2,-5.5630016E-3,2.66841E-2,-2.8890943E-2,1.3977782E-2,8.5900427E-4,3.764203E-3,-1.6856841E-3,3.0916678E-2,1.0237182E-3,3.965133E-3,-3.1024155E-3,-7.5311546E-4,-5.815104E-3,-1.7638017E-3,-7.032845E-3,-3.0021719E-3,-1.7917902E-3,8.817341E-4,8.689986E-2,6.4403075E-4,-3.6498033E-3,4.0334754E-2,-2.3064863E-4,2.5729307E-3,-1.16037935E-1,-2.0676663E-2,-5.7259955E-2,2.5728125E-2,6.1875656E-2,5.8869147E-3,-6.147545E-5,-3.2808736E-3,4.558375E-3,1.8771517E-3,5.738897E-4,3.572522E-3,-7.2325376E-4,-8.926262E-3,-2.0561318E-4,-2.5929569E-3,-4.9445713E-3,4.587254E-4,2.0109846E-3,-2.6904224E-4,1.1480399E-3,5.290877E-3,-2.572983E-3,1.0134846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,53,55,-1,57,-1,59,61,63,65,-1,-1,67,69,71,-1,-1,73,-1,-1,-1,-1,75,-1,77,79,-1,-1,-1,81,83,85,-1,87,-1,89,91,93,95,-1,97,-1,-1,-1,-1,-1,99,-1,101,-1,-1,-1,-1,-1,103,-1,105,107,109,-1,111,113,115,-1,-1,117,-1,-1,-1,119,-1,121,123,125,-1,127,-1,-1,129,-1,-1,-1,131,133,-1,135,137,139,-1,-1,-1,141,-1,-1,-1,-1,-1,-1,143,-1,-1,-1,145,-1,-1,147,-1,-1,149,151,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4703547E-1,1.7691827E-1,1.3981211E-1,1.7451459E-1,1.791532E-1,8.977169E-2,1.2475352E-1,6.1196752E-2,1.61692E-1,1.5777847E-1,0E0,6.237842E-2,7.194832E-2,9.346831E-2,5.4733828E-2,0E0,6.232463E-2,2.2319901E-1,6.763412E-2,1.285612E-1,9.75727E-2,7.73851E-2,7.485424E-2,4.70424E-2,3.1554762E-2,0E0,2.4825234E-2,4.2687103E-2,0E0,6.865272E-2,3.5808697E-2,1.03833914E-1,0E0,8.687762E-2,0E0,7.777135E-2,9.285672E-2,6.2464677E-2,1.1662583E-1,0E0,0E0,3.6479726E-2,9.630768E-2,2.1558136E-2,0E0,0E0,4.2105917E-2,0E0,0E0,0E0,0E0,5.5432133E-2,0E0,2.8255582E-2,2.7062578E-2,0E0,0E0,0E0,4.3195836E-2,6.955447E-2,1.1576258E-1,0E0,2.7437381E-2,0E0,5.245413E-2,2.7968952E-1,3.7124515E-2,1.5822046E-2,0E0,6.308238E-2,0E0,0E0,0E0,0E0,0E0,2.245424E-2,0E0,2.1342948E-2,0E0,0E0,0E0,0E0,0E0,4.6988428E-2,0E0,1.5437402E-2,4.6904568E-2,3.3728853E-2,0E0,6.8529576E-2,8.563318E-2,1.0857547E-1,0E0,0E0,1.797624E-2,0E0,0E0,0E0,5.058517E-2,0E0,1.577929E-2,2.4025545E-2,2.3583218E-2,0E0,5.3172532E-2,0E0,0E0,2.059815E-2,0E0,0E0,0E0,1.7133206E-2,8.185482E-2,0E0,1.7736774E-2,7.419665E-2,1.1196615E-1,0E0,0E0,0E0,6.39521E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.5660185E-2,0E0,0E0,0E0,1.55928135E-2,0E0,0E0,3.3115365E-2,0E0,0E0,6.669875E-2,5.6694962E-2,6.459583E-2,6.428022E-2,6.927925E-2,4.7676142E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,46,46,51,51,53,53,54,54,58,58,59,59,60,60,62,62,64,64,65,65,66,66,67,67,69,69,75,75,77,77,83,83,85,85,86,86,87,87,89,89,90,90,91,91,94,94,98,98,100,100,101,101,102,102,104,104,107,107,111,111,112,112,114,114,115,115,116,116,120,120,127,127,131,131,134,134,137,137,138,138,139,139,140,140,141,141,142,142],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,54,56,-1,58,-1,60,62,64,66,-1,-1,68,70,72,-1,-1,74,-1,-1,-1,-1,76,-1,78,80,-1,-1,-1,82,84,86,-1,88,-1,90,92,94,96,-1,98,-1,-1,-1,-1,-1,100,-1,102,-1,-1,-1,-1,-1,104,-1,106,108,110,-1,112,114,116,-1,-1,118,-1,-1,-1,120,-1,122,124,126,-1,128,-1,-1,130,-1,-1,-1,132,134,-1,136,138,140,-1,-1,-1,142,-1,-1,-1,-1,-1,-1,144,-1,-1,-1,146,-1,-1,148,-1,-1,150,152,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,3.3832976E7,1E0,6E0,1.6776881E7,1.9851073E3,1.8645384E1,9E0,9.66E2,7.09E2,-8.238197E-3,7.549744E7,1.0284041E2,5.7E1,4.6E1,-4.785559E-3,3.373913E0,1.9874651E-5,4.83929E9,3.477E4,9.777778E0,2.0342858E0,1.684E0,1.137863E6,2.0666666E0,4.835181E-3,4.6779457E1,4.433E3,2.1543658E-4,2.4796897E3,2.7777777E0,2.7E1,-2.906722E-4,1.8154167E2,2.0623659E-3,5.51E2,1E1,6.2886596E-1,2.1512408E5,-6.3896207E-3,1.2290553E-3,4.455844E1,1.6553869E7,5.36763E5,2.5875287E-3,3.2092799E-3,8.72419E6,-4.0444606E-3,6.928843E-4,-8.983178E-3,-3.0318138E-3,8E0,-2.617782E-3,2.153533E6,4.36875E1,-4.6595233E-3,-1.4563231E-2,-6.685542E-3,4.9094446E2,1.0828667E5,2.1924414E2,-9.334916E-3,2.73E3,-3.545487E-3,1E0,2.0232125E5,5E0,1.0130841E1,5.6435647E-3,5.617715E6,-3.703017E-3,3.3276656E-3,8.808678E-3,1.952945E-3,-3.3348324E-3,2.6E1,-2.417001E-3,4.5E0,1.2662942E-3,-2.1104075E-3,1.5051365E-3,1.993359E-3,-3.1490335E-3,5.4E1,-6.217197E-3,6.69E2,1.7214285E1,3E0,5.9857586E-5,3.7936746E3,7.9E2,5.338167E5,-1.1284767E-2,5.1628323E-3,3.264E3,2.8856678E-3,1.4006702E-4,6.374135E-3,1.1724138E0,5.0710775E-3,3.1026666E2,1.8187732E4,4.353644E6,-3.4528847E-3,2.49E2,4.3282816E-3,8.438426E-4,1.7594937E0,-5.078036E-3,1.1687091E-4,-4.7057704E-3,1.8E1,2.53692E5,-5.5630016E-3,9.166947E6,5.4E1,1.2397727E1,8.5900427E-4,3.764203E-3,-1.6856841E-3,1.775373E1,1.0237182E-3,3.965133E-3,-3.1024155E-3,-7.5311546E-4,-5.815104E-3,-1.7638017E-3,1E0,-3.0021719E-3,-1.7917902E-3,8.817341E-4,1.2673605E7,6.4403075E-4,-3.6498033E-3,5.603014E6,-2.3064863E-4,2.5729307E-3,1.17E2,3.401282E1,3.270662E6,2E0,9.293198E2,1.4390917E8,-6.147545E-5,-3.2808736E-3,4.558375E-3,1.8771517E-3,5.738897E-4,3.572522E-3,-7.2325376E-4,-8.926262E-3,-2.0561318E-4,-2.5929569E-3,-4.9445713E-3,4.587254E-4,2.0109846E-3,-2.6904224E-4,1.1480399E-3,5.290877E-3,-2.572983E-3,1.0134846E-3],"split_indices":[2,7,6,8,62,67,71,70,12,2,0,7,73,6,6,0,69,57,5,1,73,68,69,9,68,0,73,10,0,48,68,3,0,67,0,2,3,68,48,0,0,73,66,1,0,0,58,0,0,0,0,32,0,43,70,0,0,0,4,48,67,0,10,0,30,48,3,71,0,12,0,0,0,0,0,4,0,73,0,0,0,0,0,2,0,2,73,8,0,48,2,43,0,0,10,0,0,0,68,0,67,43,60,0,10,0,0,71,0,0,0,3,12,0,60,0,73,0,0,0,71,0,0,0,0,0,0,19,0,0,0,60,0,0,9,0,0,10,73,12,32,4,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.98E2,7.66E2,2.32E2,1.74E2,5.92E2,1.98E2,3.4E1,1.3E2,4.4E1,5.87E2,5E0,1.5E2,4.8E1,1.6E1,1.8E1,8E0,1.22E2,1.8E1,2.6E1,2.08E2,3.79E2,1E1,1.4E2,2.1E1,2.7E1,8E0,8E0,1.3E1,5E0,4.9E1,7.3E1,1.2E1,6E0,2E1,6E0,1.81E2,2.7E1,1.03E2,2.76E2,4E0,6E0,3.4E1,1.06E2,1.2E1,9E0,1.5E1,1.2E1,4E0,4E0,4E0,9E0,3.8E1,1.1E1,5.6E1,1.7E1,5E0,7E0,8E0,1.2E1,1.35E2,4.6E1,5E0,2.2E1,4E0,9.9E1,2.41E2,3.5E1,2.4E1,1E1,9.8E1,8E0,4E0,8E0,8E0,4E0,3.2E1,6E0,5.2E1,4E0,6E0,1.1E1,6E0,6E0,1.3E2,5E0,1.8E1,2.8E1,1.6E1,6E0,7.7E1,2.2E1,2.37E2,4E0,7E0,2.8E1,1.8E1,6E0,4E0,9.4E1,4E0,2.8E1,3.8E1,1.4E1,1.5E1,1.15E2,1.4E1,4E0,2.4E1,4E0,4E0,1.2E1,3.9E1,3.8E1,4E0,1.8E1,1.06E2,1.31E2,2.3E1,5E0,1.2E1,8.2E1,2.4E1,4E0,1.7E1,2.1E1,7E0,7E0,9.7E1,1.8E1,9E0,1.5E1,3.5E1,4E0,6E0,3.2E1,9E0,9E0,8E0,9.8E1,1.8E1,1.13E2,3.6E1,4.6E1,9E1,7E0,2.7E1,8E0,1.9E1,1.3E1,4E0,4E0,6.8E1,3E1,1E1,8E0,7.2E1,4.1E1,2.2E1,1.4E1,9E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-3.5064819E-3,-1.7043624E-2,2.9958205E-2,-1.5313692E-2,-8.929629E-3,-8.798362E-3,3.3642247E-2,-4.0011212E-2,-2.3347524E-3,4.154353E-2,-2.1721793E-2,-2.9777834E-2,-1.2304945E-1,-7.003079E-4,-6.0048187E-3,3.3120025E-2,9.920983E-2,-4.1052856E-2,5.0914856E-3,-3.6685344E-2,2.6397053E-2,9.714439E-3,-2.299082E-1,-9.401194E-3,2.0300383E-2,5.3411988E-3,2.8317055E-2,6.0303295E-3,2.5415237E-3,3.367115E-3,-5.9281316E-2,-4.758875E-2,-1.1238481E-2,2.5520409E-3,-7.0842216E-4,-3.6390233E-3,2.669943E-3,-1.8471073E-2,-1.4443144E-3,-1.1500854E-2,4.9408553E-3,4.9678985E-2,8.2272326E-4,6.7663215E-2,1.9386671E-2,-2.7409794E-2,-1.270614E-1,-1.4830986E-2,-6.260465E-2,2.6524141E-2,-2.7662931E-2,-4.8351116E-3,-5.0283622E-2,6.231547E-2,-2.119309E-3,-1.5919986E-3,1.63569E-2,2.8714927E-2,1.0322834E-1,3.582367E-2,-7.643471E-3,-5.8292434E-2,9.3525334E-4,-2.2455424E-3,-8.320666E-3,-3.2136608E-2,3.8670804E-3,-1.07570544E-1,-5.0696123E-2,-4.5756376E-5,2.978936E-3,1.4405181E-3,-4.415626E-2,-5.1902514E-2,3.7245703E-4,-9.0679474E-2,-2.0690499E-2,6.923772E-2,5.0315005E-4,4.1536473E-2,-1.7199343E-2,2.7676343E-3,-1.04027036E-4,5.687251E-3,8.7700016E-4,-1.0285859E-2,5.2666947E-2,4.4412473E-3,-1.5758364E-2,-3.8619095E-3,-5.050631E-5,-3.2476212E-3,-7.955341E-3,-5.44112E-2,-7.4191424E-3,-2.3419643E-2,-7.66727E-2,-5.782504E-3,-2.918293E-2,-4.1498486E-3,-3.494274E-4,7.58996E-3,-3.7308473E-2,-7.7146096E-3,-5.4111432E-2,5.2095467E-4,-4.085223E-2,3.0023122E-2,3.7781356E-3,2.2920173E-2,4.1903337E-3,1.3029128E-3,-4.396736E-3,7.978218E-3,-3.9117597E-3,3.2966908E-2,9.231447E-2,5.1219616E-2,-2.9362127E-2,-1.3554444E-3,1.2498882E-3,-5.1819924E-3,5.734586E-4,-1.5715151E-3,2.5612765E-3,-4.2935116E-3,-9.4207666E-5,-1.8896698E-3,1.7369419E-3,7.4920733E-4,-1.1212428E-3,-4.456104E-3,-9.261613E-4,2.2897439E-4,-3.5130295E-3,-4.555337E-3,-7.9966645E-4,3.103407E-3,-1.6365559E-3,2.1641464E-3,-2.1283876E-4,2.514252E-3,-8.094858E-4,-1.3055975E-3,1.777273E-3,3.8738707E-3,7.031666E-4,2.6839764E-3,5.639245E-3,3.8731173E-3,-2.1650958E-4,-5.8842753E-4,-4.6003815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,33,35,37,39,41,-1,43,-1,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1,51,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,-1,-1,79,81,83,85,87,89,-1,-1,-1,91,-1,93,95,-1,-1,-1,97,99,101,103,105,107,-1,109,111,-1,-1,-1,-1,113,115,-1,117,-1,-1,-1,119,121,-1,123,125,-1,127,-1,-1,129,131,-1,133,-1,135,137,-1,139,-1,141,-1,143,-1,145,147,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.639462E-1,2.1930282E-1,2.4593905E-1,2.3166256E-1,0E0,0E0,1.2819946E-1,2.0792654E-1,9.9005535E-2,1.2058103E-1,9.888673E-2,8.749877E-2,3.8528195E-1,8.607142E-2,0E0,8.547725E-2,3.711492E-2,7.2857335E-2,0E0,5.4679185E-2,3.0938441E-2,5.9854865E-2,4.937685E-1,8.257921E-2,7.8926384E-2,0E0,7.3531374E-2,0E0,0E0,0E0,5.916466E-2,6.744385E-2,3.856778E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.455011E-2,0E0,6.92682E-2,4.660246E-2,5.105865E-2,7.802534E-2,3.208524E-2,2.869974E-2,7.9424985E-2,4.541698E-2,1.990654E-2,4.310949E-2,6.889432E-2,5.550488E-2,1.668863E-2,0E0,0E0,5.056537E-2,1.9453004E-2,3.1150386E-2,8.519499E-2,5.7937868E-2,1.8721722E-2,0E0,0E0,0E0,3.5427988E-2,0E0,4.4813946E-2,5.2762672E-2,0E0,0E0,0E0,3.758832E-2,4.6401568E-2,6.9361575E-2,4.6771973E-2,1.9151341E-2,1.869303E-2,0E0,2.9040456E-2,3.7632167E-2,0E0,0E0,0E0,0E0,4.1433286E-2,5.9441313E-2,0E0,5.890653E-2,0E0,0E0,0E0,1.950987E-2,4.384624E-2,0E0,3.447579E-2,4.7145486E-2,0E0,2.5017329E-2,0E0,0E0,5.9212293E-2,3.99247E-2,0E0,1.9806955E-2,0E0,2.276638E-2,3.1557165E-2,0E0,1.7338216E-2,0E0,2.2952452E-2,0E0,2.871148E-2,0E0,4.839037E-2,1.9376278E-2,2.0450532E-2,6.06014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,26,26,30,30,31,31,32,32,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,56,56,57,57,58,58,59,59,60,60,61,61,65,65,67,67,68,68,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,85,85,86,86,88,88,92,92,93,93,95,95,96,96,98,98,101,101,102,102,104,104,106,106,107,107,109,109,111,111,113,113,115,115,116,116,117,117,118,118],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,34,36,38,40,42,-1,44,-1,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1,52,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,-1,-1,80,82,84,86,88,90,-1,-1,-1,92,-1,94,96,-1,-1,-1,98,100,102,104,106,108,-1,110,112,-1,-1,-1,-1,114,116,-1,118,-1,-1,-1,120,122,-1,124,126,-1,128,-1,-1,130,132,-1,134,-1,136,138,-1,140,-1,142,-1,144,-1,146,148,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.56E3,2.483E3,2.7920364E2,1E0,-8.929629E-3,-8.798362E-3,2.5345264E7,3.7798166E0,4.4E1,1.7379E4,4.7263342E2,7.6937294E-1,4.327E3,3.6698947E2,-6.0048187E-3,1.0918E4,5.4123135E1,2.94857E5,5.0914856E-3,7.3131656E5,1.5E1,3.3857143E1,2.2596774E2,3.5E1,1E0,5.3411988E-3,3.222E3,6.0303295E-3,2.5415237E-3,3.367115E-3,1.4839433E5,6.0921145E4,5.51E2,2.5520409E-3,-7.0842216E-4,-3.6390233E-3,2.669943E-3,-1.8471073E-2,-1.4443144E-3,1.8E1,4.9408553E-3,1.59E2,1.14E3,6.232639E2,1.8645384E1,1.8987958E0,4.797448E8,2.135961E4,1E0,6E0,1.1015E4,8.234307E7,2.3266666E0,1.7172844E7,-2.119309E-3,-1.5919986E-3,1.688E3,4.1228E2,2.1199985E-7,7.653112E2,1.0092541E8,1.514526E6,9.3525334E-4,-2.2455424E-3,-8.320666E-3,5.62201E5,3.8670804E-3,1.3050649E4,6.76E2,-4.5756376E-5,2.978936E-3,1.4405181E-3,1.6716E5,2.026453E6,1E0,1.8149019E2,5.80563E5,2.2116136E5,5.0315005E-4,2.682E3,2.0766992E6,2.7676343E-3,-1.04027036E-4,5.687251E-3,8.7700016E-4,4.4522205E6,1.5E1,4.4412473E-3,2E0,-3.8619095E-3,-5.050631E-5,-3.2476212E-3,1.1E1,3.964269E3,-7.4191424E-3,1.5141701E1,5.7E1,-5.782504E-3,2.0130434E1,-4.1498486E-3,-3.494274E-4,3.5014236E2,1.2E1,-7.7146096E-3,1E0,5.2095467E-4,1.079E3,1.2E1,3.7781356E-3,1.5747747E1,4.1903337E-3,2.7662E4,-4.396736E-3,2.2E0,-3.9117597E-3,7.5052085E0,1.1772152E0,1.2707424E0,1.0975045E7,-1.3554444E-3,1.2498882E-3,-5.1819924E-3,5.734586E-4,-1.5715151E-3,2.5612765E-3,-4.2935116E-3,-9.4207666E-5,-1.8896698E-3,1.7369419E-3,7.4920733E-4,-1.1212428E-3,-4.456104E-3,-9.261613E-4,2.2897439E-4,-3.5130295E-3,-4.555337E-3,-7.9966645E-4,3.103407E-3,-1.6365559E-3,2.1641464E-3,-2.1283876E-4,2.514252E-3,-8.094858E-4,-1.3055975E-3,1.777273E-3,3.8738707E-3,7.031666E-4,2.6839764E-3,5.639245E-3,3.8731173E-3,-2.1650958E-4,-5.8842753E-4,-4.6003815E-3],"split_indices":[2,2,48,29,0,0,60,68,3,2,71,53,1,67,0,9,73,1,0,43,3,67,67,3,26,0,2,0,0,0,48,60,2,0,0,0,0,0,0,3,0,0,2,67,71,68,7,43,18,8,9,46,68,60,0,0,2,4,52,4,7,10,0,0,0,46,0,43,2,0,0,0,7,9,30,70,12,43,0,10,43,0,0,0,0,62,8,0,8,0,0,0,3,43,0,73,6,0,71,0,0,4,3,0,8,0,10,71,0,73,0,9,0,68,0,71,68,68,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,7.28E2,2.94E2,7.22E2,6E0,4E0,2.9E2,2.48E2,4.74E2,2.54E2,3.6E1,2.22E2,2.6E1,4.69E2,5E0,2.23E2,3.1E1,3.2E1,4E0,1.98E2,2.4E1,1.2E1,1.4E1,3.32E2,1.37E2,1.1E1,2.12E2,1.7E1,1.4E1,4E0,2.8E1,1.38E2,6E1,1.4E1,1E1,4E0,8E0,7E0,7E0,3.27E2,5E0,5.4E1,8.3E1,3.8E1,1.74E2,2E1,8E0,4.4E1,9.4E1,1.8E1,4.2E1,2.8E2,4.7E1,4.8E1,6E0,2.5E1,5.8E1,1.9E1,1.9E1,1.08E2,6.6E1,1.2E1,8E0,4E0,4E0,3.8E1,6E0,1.8E1,7.6E1,1.1E1,7E0,9E0,3.3E1,2.7E1,2.53E2,1.9E1,2.8E1,4.2E1,6E0,3.3E1,2.5E1,9E0,1E1,1.5E1,4E0,2.9E1,7.9E1,4E0,6.2E1,8E0,4E0,1.4E1,2.4E1,1E1,8E0,3.8E1,3.8E1,4E0,2.9E1,1.4E1,1.3E1,2.13E2,4E1,5E0,1.4E1,1.1E1,1.7E1,1.1E1,3.1E1,2.5E1,8E0,2.1E1,4E0,2.4E1,5E0,5.4E1,2.5E1,1E1,5.2E1,1.5E1,9E0,5E0,5E0,3.4E1,4E0,3.1E1,7E0,2.5E1,4E0,1.68E2,4.5E1,8E0,3.2E1,4E0,1E1,4E0,1.3E1,7E0,4E0,1.3E1,1.2E1,5E0,1.6E1,1.1E1,1.3E1,1.3E1,4.1E1,1.3E1,1.2E1,6E0,4E0,4.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[-7.3198804E-5,-2.2783266E-2,1.5895527E-2,-1.5977498E-2,-7.454215E-2,-5.250131E-2,2.058859E-2,-1.09443525E-2,-6.320875E-2,-3.5612783E-3,-1.4833216E-1,-6.835955E-2,1.3748431E-3,7.889349E-3,1.8671151E-2,-1.6785571E-2,1.4599447E-2,9.362277E-4,-8.7782174E-2,-3.6571737E-2,3.799857E-2,-1.7136347E-2,-5.3527907E-2,-5.5754366E-3,-4.963083E-2,5.5908125E-2,1.0829558E-2,-1.8994963E-2,2.7791401E-3,2.3128772E-2,-3.3127256E-3,-6.019664E-3,-3.5958882E-2,9.1654784E-4,-3.084298E-3,3.637656E-3,-1.6070552E-3,-3.8415352E-3,2.2650431E-4,3.671403E-4,-2.7932285E-3,3.9474778E-2,1.01657175E-1,8.21374E-3,1.0845626E-1,-2.699716E-2,2.6730103E-3,4.0224258E-2,-5.378261E-3,-3.255703E-3,1.0072666E-3,4.0847105E-3,1.9338377E-2,1.0643075E-3,5.551289E-3,2.5339709E-3,5.7102513E-2,6.8270857E-3,2.0562187E-3,-1.9593183E-2,-7.7448964E-2,1.0233871E-2,-4.7914716E-3,1.7784748E-2,4.5752055E-3,-2.1229796E-2,1.9899188E-3,-3.2505472E-3,9.678808E-2,7.0530996E-3,-5.872616E-2,4.3609206E-2,7.471516E-3,-1.4337468E-2,-6.728788E-2,-6.964446E-3,-1.87423E-3,-4.020971E-2,2.0320127E-2,2.4518825E-3,2.578389E-3,5.384609E-4,-3.600521E-3,-4.117588E-2,4.4826217E-2,7.0764334E-3,6.87196E-4,5.2980117E-2,1.7797659E-3,-2.6662592E-2,-1.2345341E-1,5.576743E-2,-1.8785038E-3,-2.5240244E-2,1.54081015E-2,-1.2521219E-1,-8.718359E-5,-3.304146E-3,8.282728E-4,2.6197698E-2,-2.2359039E-3,1.3245983E-3,-2.0207013E-3,1.2220579E-3,-1.7750418E-3,-2.1131195E-3,-7.300003E-3,-1.9109527E-4,3.406367E-3,2.5541404E-2,1.05892815E-1,-1.014036E-1,6.1785076E-3,1.6401062E-3,-4.9241115E-3,-1.4431817E-3,-8.822992E-3,5.106643E-3,7.6217875E-2,-1.3711936E-3,2.223794E-3,-5.828585E-4,2.2298857E-3,-7.821379E-3,-2.591053E-3,1.6174281E-3,-5.652077E-4,-8.688377E-4,2.2355414E-3,-1.2461173E-3,2.4622704E-3,6.725709E-3,1.6320407E-3,3.4086595E-4,-1.3799031E-2,2.3074378E-3,-3.403599E-6,1.5976295E-3,-1.9899944E-3,4.2402097E-3,1.2835037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,31,33,35,-1,37,-1,39,41,43,45,-1,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,51,53,55,57,59,61,63,65,-1,-1,-1,67,-1,-1,69,71,-1,-1,73,75,77,-1,79,-1,81,-1,83,85,87,89,91,-1,93,95,-1,-1,97,99,101,-1,103,-1,105,107,-1,-1,109,111,113,115,117,-1,119,121,123,-1,-1,-1,125,-1,-1,-1,-1,-1,127,-1,-1,-1,129,131,133,135,-1,-1,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.724419E-1,1.4816678E-1,1.9463833E-1,8.8691615E-2,2.5611803E-1,5.2910015E-2,1.5950966E-1,5.112463E-2,7.562348E-2,3.705932E-2,4.8660952E-1,2.6348755E-2,0E0,0E0,1.6262098E-1,4.8400886E-2,4.8770323E-2,0E0,5.7007954E-2,2.6869562E-2,3.8321793E-2,0E0,3.156471E-2,0E0,1.6693085E-2,6.8912685E-2,1.1734719E-1,4.7061235E-2,0E0,2.8969493E-2,0E0,0E0,2.9445304E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.023091E-2,3.3599466E-2,1.2536526E-1,2.3595676E-2,7.2444245E-2,6.058511E-2,4.7064945E-2,1.8578941E-2,0E0,0E0,0E0,9.3038395E-2,0E0,0E0,1.1281223E-1,5.929704E-2,0E0,0E0,4.2651787E-2,6.146738E-2,3.643215E-2,0E0,1.6188353E-2,0E0,2.226067E-2,0E0,7.8425504E-2,5.314546E-2,9.1695145E-2,5.4819234E-2,4.6324387E-2,0E0,5.1629774E-2,6.416471E-2,0E0,0E0,2.3125919E-2,2.509746E-2,2.6687378E-2,0E0,1.5150642E-2,0E0,1.0920067E-1,2.7196083E-2,0E0,0E0,5.3917736E-2,1.5539856E-1,1.0314853E-1,5.0075084E-2,3.898015E-2,0E0,3.9218865E-2,4.020536E-2,1.7932579E-2,0E0,0E0,0E0,1.9055326E-2,0E0,0E0,0E0,0E0,0E0,1.692801E-2,0E0,0E0,0E0,4.079175E-2,3.126581E-2,3.0265477E-1,8.975745E-2,0E0,0E0,0E0,0E0,1.8531075E-2,1.780349E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,18,18,19,19,20,20,22,22,24,24,25,25,26,26,27,27,29,29,32,32,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,52,52,55,55,56,56,59,59,60,60,61,61,63,63,65,65,67,67,68,68,69,69,70,70,71,71,73,73,74,74,77,77,78,78,79,79,81,81,83,83,84,84,87,87,88,88,89,89,90,90,91,91,93,93,94,94,95,95,99,99,105,105,109,109,110,110,111,111,112,112,117,117,118,118],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,32,34,36,-1,38,-1,40,42,44,46,-1,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,52,54,56,58,60,62,64,66,-1,-1,-1,68,-1,-1,70,72,-1,-1,74,76,78,-1,80,-1,82,-1,84,86,88,90,92,-1,94,96,-1,-1,98,100,102,-1,104,-1,106,108,-1,-1,110,112,114,116,118,-1,120,122,124,-1,-1,-1,126,-1,-1,-1,-1,-1,128,-1,-1,-1,130,132,134,136,-1,-1,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,3.5531914E0,1E0,2.8E1,8.62E2,1.1805571E-1,1.4E1,2.7224753E2,1.1E1,8.5714287E-1,1E0,3.421907E6,1.3748431E-3,7.889349E-3,1.6954315E0,1.3062992E1,5.0394736E2,9.362277E-4,1.084437E6,8.75477E5,6.763314E7,-1.7136347E-2,9.822E3,-5.5754366E-3,9.82E2,1.6E1,4.435876E2,8.93004E5,2.7791401E-3,1E0,-3.3127256E-3,-6.019664E-3,4.5172415E0,9.1654784E-4,-3.084298E-3,3.637656E-3,-1.6070552E-3,-3.8415352E-3,2.2650431E-4,3.671403E-4,-2.7932285E-3,3.887E3,2.5454E4,2.680152E5,2.4704508E7,4.28649E5,8.416667E0,3.541824E2,7.1E1,-3.255703E-3,1.0072666E-3,4.0847105E-3,1.9001543E1,1.0643075E-3,5.551289E-3,1.4127854E7,1.9E1,6.8270857E-3,2.0562187E-3,1E0,1.2E1,2.35E2,-4.7914716E-3,9.008784E7,4.5752055E-3,4.0119403E2,1.9899188E-3,3.659E3,2E0,1.4814814E0,1.7540792E5,1.4E1,7.471516E-3,5.5E1,2.0217392E0,-6.964446E-3,-1.87423E-3,7.773675E1,2.6043478E2,9.22E2,2.578389E-3,1.7638037E0,-3.600521E-3,2.62E3,1.7790684E5,7.0764334E-3,6.87196E-4,4.6604937E-1,1.027972E0,4.2619047E0,6.9756895E6,9.3411E4,-1.8785038E-3,3.1316226E0,1.6808511E0,1.2269693E4,-8.718359E-5,-3.304146E-3,8.282728E-4,1.0115696E0,-2.2359039E-3,1.3245983E-3,-2.0207013E-3,1.2220579E-3,-1.7750418E-3,4.080551E-1,-7.300003E-3,-1.9109527E-4,3.406367E-3,1.1E1,7E0,4.822581E0,6E0,1.6401062E-3,-4.9241115E-3,-1.4431817E-3,-8.822992E-3,4.8E2,1.4780379E7,-1.3711936E-3,2.223794E-3,-5.828585E-4,2.2298857E-3,-7.821379E-3,-2.591053E-3,1.6174281E-3,-5.652077E-4,-8.688377E-4,2.2355414E-3,-1.2461173E-3,2.4622704E-3,6.725709E-3,1.6320407E-3,3.4086595E-4,-1.3799031E-2,2.3074378E-3,-3.403599E-6,1.5976295E-3,-1.9899944E-3,4.2402097E-3,1.2835037E-3],"split_indices":[2,68,17,8,44,57,0,67,3,71,29,12,0,0,69,71,70,0,9,46,7,0,44,0,2,8,71,9,0,26,0,0,69,0,0,0,0,0,0,0,0,44,9,48,1,9,71,67,0,0,0,0,71,0,0,66,8,0,0,30,3,2,0,5,0,4,0,2,8,71,48,8,0,0,71,0,0,67,67,44,0,68,0,2,48,0,0,71,68,69,60,1,0,68,68,62,0,0,0,53,0,0,0,0,0,53,0,0,0,3,3,71,3,0,0,0,0,10,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.025E3,4.23E2,6.02E2,3.75E2,4.8E1,3.8E1,5.64E2,3.4E2,3.5E1,2.5E1,2.3E1,3.2E1,6E0,6E0,5.58E2,2.77E2,6.3E1,8E0,2.7E1,1.4E1,1.1E1,6E0,1.7E1,7E0,2.5E1,9.6E1,4.62E2,2.7E2,7E0,5.8E1,5E0,1.4E1,1.3E1,5E0,9E0,7E0,4E0,1.1E1,6E0,4E0,2.1E1,7.2E1,2.4E1,4.51E2,1.1E1,1.97E2,7.3E1,3.6E1,2.2E1,8E0,5E0,2E1,5.2E1,5E0,1.9E1,4.05E2,4.6E1,6E0,5E0,1.73E2,2.4E1,6.9E1,4E0,2.7E1,9E0,1.7E1,5E0,4.1E1,1.1E1,3.78E2,2.7E1,4.2E1,4E0,1.57E2,1.6E1,7E0,1.7E1,1.1E1,5.8E1,2E1,7E0,1.3E1,4E0,2.3E1,1.8E1,6E0,5E0,3.8E1,3.4E2,1.9E1,8E0,3.7E1,5E0,1.15E2,4.2E1,8E0,8E0,7E0,4E0,5.4E1,4E0,1.3E1,7E0,8E0,5E0,1.8E1,5E0,7E0,1.1E1,2.6E1,1.2E1,1.3E1,3.27E2,1.1E1,8E0,4E0,4E0,1.1E1,2.6E1,1.09E2,6E0,2.3E1,1.9E1,4E0,4E0,4.4E1,1E1,1.4E1,4E0,9E0,1.7E1,7E0,5E0,9E0,4E0,4E1,2.87E2,7E0,4E0,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[-5.0379704E-3,-2.6458643E-2,1.7478172E-2,-2.170328E-2,-1.7119959E-1,-6.794838E-2,2.3113383E-2,-2.4321122E-2,3.901759E-2,-9.5182745E-4,-2.5147036E-1,-4.120725E-2,-9.429642E-3,1.1287728E-2,5.8693375E-2,3.0864634E-2,-2.6794646E-2,3.650673E-3,-7.802085E-4,-1.5879592E-2,-5.4195095E-3,-2.7823346E-3,1.7815157E-3,1.353236E-2,-4.8417076E-3,1.2800871E-1,4.09381E-2,-9.640937E-4,2.3824265E-3,-1.214309E-1,-2.4114335E-2,2.5119218E-3,3.229817E-2,8.202278E-3,7.9946116E-2,1.2456085E-2,6.5346465E-2,-1.1591618E-3,-1.2499488E-2,-3.165019E-2,-5.995764E-3,9.0057375E-3,-1.3227522E-2,3.7086245E-2,-2.1506944E-3,5.17834E-3,8.505047E-4,-4.042029E-3,2.6778769E-2,8.8830836E-2,2.928456E-2,-2.4758635E-2,-6.505052E-2,1.4850303E-2,-2.3982411E-2,5.9431978E-2,2.3047836E-3,6.3128434E-2,1.946525E-2,6.9702476E-2,-9.84424E-3,1.0425684E-1,3.316943E-4,-2.2041834E-4,4.133811E-3,-2.7720671E-2,5.227569E-3,-8.053179E-2,3.5749776E-3,2.2346413E-2,-2.3308652E-3,5.5097076E-3,-3.8838666E-2,9.825463E-2,5.950998E-3,-3.4965403E-2,9.2094E-3,3.2930467E-2,8.763298E-2,4.5370262E-2,-1.9602356E-2,8.9224346E-2,-1.295253E-4,-3.2192608E-3,1.5374078E-2,6.2608216E-3,7.2297126E-2,-1.6361516E-2,-5.2813537E-2,-1.7768513E-2,-1.0871591E-1,-5.1212655E-3,1.9724157E-3,-8.45056E-4,2.4254594E-3,-3.6917245E-3,-1.1644814E-3,3.0869665E-4,6.0495976E-3,-2.4311212E-3,2.5551277E-3,-6.585039E-2,1.2863841E-2,6.7187324E-2,5.4816236E-3,7.04264E-2,-1.0434325E-2,6.5210555E-3,6.5620534E-2,2.6777036E-2,9.0508685E-2,-4.5372076E-2,1.4495606E-3,5.55659E-3,1.6888417E-3,-4.9947044E-3,3.0148623E-3,4.5743897E-3,7.261632E-4,-1.3532762E-3,1.0650757E-3,-1.0867582E-3,-3.4820798E-3,7.823806E-4,-3.8906117E-3,-1.1792188E-2,-3.2153344E-3,2.120461E-3,-1.4019826E-3,-4.938263E-3,1.8210933E-4,1.8892603E-3,-1.8554955E-3,5.717072E-3,-6.5250543E-4,-2.7998232E-3,5.6951516E-4,1.3544664E-3,5.243582E-3,1.0600457E-3,-2.2008317E-3,3.6425916E-3,3.4793493E-4,3.9226608E-4,3.5401762E-3,9.350862E-4,5.5203293E-3,-4.794298E-3,-6.972665E-4,2.1345895E-3,-1.5191894E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,31,-1,33,35,-1,-1,37,39,41,43,-1,45,47,49,-1,-1,51,53,55,-1,57,-1,-1,-1,-1,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,87,-1,89,-1,91,-1,93,95,97,99,101,103,105,107,109,111,113,-1,-1,115,-1,117,119,121,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,129,131,133,135,137,139,-1,141,143,145,147,-1,-1,-1,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.073713E-1,3.6727583E-1,2.4852696E-1,8.408375E-2,2.0032391E-1,1.08104065E-1,2.0214224E-1,6.93517E-2,5.053501E-2,0E0,9.582144E-2,4.4798974E-2,0E0,9.600747E-2,1.4244509E-1,2.4639979E-2,1.1907962E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.374912E-2,0E0,4.294312E-2,6.6620395E-2,0E0,0E0,1.7998101E-1,6.3846946E-2,4.2646283E-1,5.1562324E-2,0E0,2.5626764E-2,6.8192795E-2,4.143694E-2,0E0,0E0,7.457909E-2,5.2474316E-2,7.4877694E-2,0E0,5.613233E-2,0E0,0E0,0E0,0E0,6.5137975E-2,3.8297564E-2,4.552545E-2,1.1588268E-1,1.2971382E-1,3.294612E-2,3.3074666E-2,5.339838E-2,5.1017273E-2,3.411357E-2,7.780492E-2,2.7143799E-2,3.5778712E-2,1.6179293E-2,0E0,0E0,0E0,7.638003E-2,0E0,8.82836E-2,0E0,3.319358E-2,0E0,3.0039666E-2,2.5981337E-2,4.4030115E-2,3.7601948E-2,4.664046E-2,3.5725202E-2,3.9758928E-2,2.2545427E-2,3.6182806E-2,4.2148583E-2,2.0134136E-2,0E0,0E0,1.759925E-2,0E0,2.1118551E-2,9.6494034E-2,5.3727373E-2,4.062638E-2,1.8285868E-1,3.485449E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.484336E-2,2.0964954E-2,4.7738485E-2,7.222712E-2,1.9691184E-2,1.5995158E-2,0E0,1.5539132E-2,3.0343806E-2,2.4055652E-2,3.5570186E-2,0E0,0E0,0E0,2.008178E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,23,23,25,25,26,26,29,29,30,30,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,43,43,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,65,65,67,67,69,69,71,71,72,72,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,84,84,86,86,87,87,88,88,89,89,90,90,91,91,101,101,102,102,103,103,104,104,105,105,106,106,108,108,109,109,110,110,111,111,115,115],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,32,-1,34,36,-1,-1,38,40,42,44,-1,46,48,50,-1,-1,52,54,56,-1,58,-1,-1,-1,-1,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,88,-1,90,-1,92,-1,94,96,98,100,102,104,106,108,110,112,114,-1,-1,116,-1,118,120,122,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,130,132,134,136,138,140,-1,142,144,146,148,-1,-1,-1,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,6.0692043E0,3.964269E3,1.6521566E5,3.89E2,3.5E0,1.3141646E3,3.5714287E-1,4E0,-9.5182745E-4,6.6E1,1.24141544E-1,-9.429642E-3,1.853776E7,1.3587301E1,9.147E3,1.342376E0,3.650673E-3,-7.802085E-4,-1.5879592E-2,-5.4195095E-3,-2.7823346E-3,1.7815157E-3,1.8062708E6,-4.8417076E-3,2.5400869E5,1.6876252E0,-9.640937E-4,2.3824265E-3,1.0280637E0,1.559733E6,1.7587205E6,3.9935064E0,8.202278E-3,3.9813398E6,5.97E3,1.034073E6,-1.1591618E-3,-1.2499488E-2,4.064E3,1E0,2.3514E4,-1.3227522E-2,2.8045622E6,-2.1506944E-3,5.17834E-3,8.505047E-4,-4.042029E-3,6.6858566E-1,2.8827406E5,1.1863237E2,3.653E3,2.0130434E1,1E0,5E1,1.6869704E6,3.77E2,2.152E3,7.4010696E0,1.5859042E6,5.369854E5,5.932344E1,3.316943E-4,-2.2041834E-4,4.133811E-3,1.82407E5,5.227569E-3,1.480315E0,3.5749776E-3,6.83E2,-2.3308652E-3,5.195011E7,5.75E2,9.7573663E2,1.365243E7,1.4787234E1,1.0918E4,1.0092541E8,7.6825094E0,5.235714E2,1E0,5.801167E-1,-1.295253E-4,-3.2192608E-3,5.48352E5,6.2608216E-3,3.6666667E0,5.75E2,5.680271E-2,2.732439E2,1.8149019E2,1.2E1,1.9724157E-3,-8.45056E-4,2.4254594E-3,-3.6917245E-3,-1.1644814E-3,3.0869665E-4,6.0495976E-3,-2.4311212E-3,2.5551277E-3,1.7443357E8,2.3666666E0,1.6673345E7,7.9559E5,6.347708E7,4.3024124E2,6.5210555E-3,5.4567E5,2.01E0,1.0641107E8,3.921E3,1.4495606E-3,5.55659E-3,1.6888417E-3,9.408E3,3.0148623E-3,4.5743897E-3,7.261632E-4,-1.3532762E-3,1.0650757E-3,-1.0867582E-3,-3.4820798E-3,7.823806E-4,-3.8906117E-3,-1.1792188E-2,-3.2153344E-3,2.120461E-3,-1.4019826E-3,-4.938263E-3,1.8210933E-4,1.8892603E-3,-1.8554955E-3,5.717072E-3,-6.5250543E-4,-2.7998232E-3,5.6951516E-4,1.3544664E-3,5.243582E-3,1.0600457E-3,-2.2008317E-3,3.6425916E-3,3.4793493E-4,3.9226608E-4,3.5401762E-3,9.350862E-4,5.5203293E-3,-4.794298E-3,-6.972665E-4,2.1345895E-3,-1.5191894E-3],"split_indices":[2,68,43,48,2,68,67,68,8,0,10,53,0,62,71,44,69,0,0,0,0,0,0,43,0,48,53,0,0,69,9,43,68,0,43,2,9,0,0,44,26,1,0,43,0,0,0,0,57,48,73,44,71,19,0,60,10,2,71,43,43,73,0,0,0,9,0,68,0,2,0,7,2,48,5,71,9,7,71,4,28,57,0,0,44,0,69,2,53,67,70,3,0,0,0,0,0,0,0,0,0,7,68,47,12,7,4,0,1,68,5,2,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E3,5.38E2,5.12E2,5.22E2,1.6E1,3.1E1,4.81E2,5.01E2,2.1E1,6E0,1E1,2.7E1,4E0,3.62E2,1.19E2,2.1E1,4.8E2,1.2E1,9E0,5E0,5E0,2.2E1,5E0,3.56E2,6E0,2.3E1,9.6E1,6E0,1.5E1,1.2E1,4.68E2,2.25E2,1.31E2,1E1,1.3E1,4.5E1,5.1E1,8E0,4E0,3.3E2,1.38E2,2.21E2,4E0,1.24E2,7E0,8E0,5E0,5E0,4E1,3E1,2.1E1,2.75E2,5.5E1,6.4E1,7.4E1,2.5E1,1.96E2,4.9E1,7.5E1,1.8E1,2.2E1,2.5E1,5E0,1.4E1,7E0,2.7E2,5E0,5E1,5E0,5.8E1,6E0,2.5E1,4.9E1,1.4E1,1.1E1,3E1,1.66E2,2.3E1,2.6E1,4.5E1,3E1,1.4E1,4E0,6E0,1.6E1,1.1E1,1.4E1,1.87E2,8.3E1,1.6E1,3.4E1,2.5E1,3.3E1,1.7E1,8E0,1.1E1,3.8E1,4E0,1E1,5E0,6E0,1.8E1,1.2E1,9E0,1.57E2,1.2E1,1.1E1,6E0,2E1,3.3E1,1.2E1,2E1,1E1,8E0,6E0,1.2E1,4E0,9E0,5E0,1.41E2,4.6E1,3.7E1,4.6E1,1.1E1,5E0,6E0,2.8E1,8E0,1.7E1,1.1E1,7E0,8E0,4E0,5E0,4E0,1.4E1,1.43E2,7E0,5E0,6E0,5E0,1.6E1,4E0,2.5E1,8E0,4E0,8E0,6E0,1.4E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-6.0611346E-4,-2.3694906E-2,1.5831003E-2,-1.8948995E-2,-6.957293E-2,-5.7419207E-2,2.126269E-2,-1.0498687E-2,-4.1141726E-2,-2.0615581E-2,-2.0948768E-1,-3.4817286E-2,-9.849156E-3,1.8930182E-2,8.173714E-3,-4.462574E-2,-2.4489863E-3,-5.6143887E-2,1.47852795E-2,1.8199385E-3,-6.1517353E-3,-1.5598287E-2,-6.3831935E-4,-5.3048957E-2,1.7511126E-3,5.199987E-3,4.3018747E-2,-1.6522886E-2,-9.01693E-2,4.6123814E-2,-6.120625E-3,-5.1253475E-3,-8.202377E-2,-7.4939905E-3,3.0851678E-3,-1.9282785E-3,3.165261E-2,-2.6445054E-2,-3.4498938E-3,6.368992E-2,-2.6283106E-3,3.3134618E-3,5.5787474E-2,-3.9011112E-3,1.046837E-3,-5.0963473E-4,-5.047909E-3,2.9819328E-3,-3.3783424E-4,-4.590922E-3,-2.9065527E-3,-1.271066E-3,1.8554521E-3,-5.5347722E-2,-2.1221828E-1,3.139617E-2,-4.5551486E-2,-1.1024267E-3,5.2146208E-2,-2.4895126E-3,5.827955E-4,8.370852E-2,2.341045E-2,2.4086763E-3,-8.8695064E-2,-1.5617399E-2,3.2399737E-3,6.247584E-2,-4.3428443E-2,2.6933176E-3,-7.9541194E-4,-3.6056782E-3,-2.4883013E-4,3.695911E-2,-8.129441E-2,-2.8430296E-3,-1.480468E-2,3.1989787E-3,-5.8812666E-4,-3.7955975E-3,7.6137817E-6,3.7556288E-3,-2.2715343E-4,5.608295E-2,6.1319177E-3,-1.1956391E-3,2.3396944E-3,3.8379703E-2,-1.0034807E-2,-2.112811E-1,4.374598E-2,2.8258592E-3,-3.1431753E-2,3.160368E-2,7.7247284E-2,7.787996E-4,-3.992921E-3,-5.1907557E-3,4.4240944E-2,-6.8492896E-4,3.8153015E-3,-6.227962E-4,-9.9536784E-2,5.9769296E-5,3.754632E-3,2.757065E-2,5.2936114E-3,-7.1155108E-3,-4.5019267E-3,-1.4979242E-2,-2.590533E-3,3.8339172E-3,-1.960762E-4,-1.5056144E-2,-3.7293495E-3,7.981172E-2,8.733244E-3,-1.1164962E-3,8.3216265E-2,1.2045125E-3,-3.596816E-2,-8.078828E-4,2.8073252E-3,-8.479045E-3,-7.439039E-2,3.626753E-2,-2.4095168E-3,-8.627255E-3,-5.679496E-5,-3.1907673E-3,8.477124E-3,4.756977E-3,1.2897695E-3,-1.1739854E-3,2.9774604E-3,8.793451E-2,-6.30025E-4,-1.1709518E-3,3.593029E-4,8.3589846E-5,-3.1039051E-3,-2.1760524E-3,-5.29617E-3,5.036642E-4,3.1325482E-3,2.3443638E-3,-2.8171093E-4,1.148037E-3,-1.9558978E-3,1.1586294E-3,-9.785327E-4,4.343763E-3,3.077762E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,-1,-1,-1,37,-1,39,41,43,45,47,49,51,53,55,-1,-1,57,59,-1,61,63,65,67,-1,69,-1,-1,-1,-1,-1,71,-1,-1,73,75,77,79,-1,81,-1,-1,83,85,87,89,91,-1,93,95,-1,-1,-1,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,103,-1,-1,-1,105,107,109,111,-1,113,115,117,-1,-1,119,121,-1,-1,-1,123,-1,-1,125,-1,-1,127,-1,-1,-1,-1,129,-1,131,133,-1,135,137,139,-1,-1,-1,141,143,-1,-1,145,-1,147,-1,-1,149,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.833039E-1,9.0129495E-2,2.3596549E-1,7.12066E-2,2.6282564E-1,1.3692643E-1,1.9634533E-1,7.62608E-2,8.892782E-2,7.5990215E-2,2.4607664E-1,5.1066257E-2,0E0,1.7936848E-1,0E0,6.634506E-2,4.058755E-2,1.0916898E-1,2.7143696E-2,3.5107207E-2,0E0,0E0,0E0,1.5284985E-2,0E0,1.5905058E-1,9.969139E-2,4.0660903E-2,2.9017553E-2,1.6222056E-2,6.2951826E-2,3.0507263E-2,1.7927337E-1,2.65912E-2,0E0,0E0,1.9447623E-2,1.6757783E-2,0E0,3.1100214E-2,1.3330339E-1,6.307578E-2,1.0158855E-1,0E0,3.0281406E-2,0E0,0E0,0E0,0E0,0E0,4.0919363E-2,0E0,0E0,1.15572244E-1,1.2706017E-1,1.5912052E-2,1.507495E-2,0E0,2.0357706E-2,0E0,0E0,2.9119536E-2,2.1445882E-2,1.3063815E-1,2.8524488E-1,4.9280144E-2,0E0,6.140238E-2,2.7380034E-2,0E0,0E0,0E0,4.398281E-2,2.7213737E-2,4.443024E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.6115246E-2,0E0,0E0,0E0,5.8617055E-2,1.7217854E-1,1.395979E-1,1.730082E-2,0E0,2.5421642E-2,5.1122315E-2,5.8913052E-2,0E0,0E0,3.547268E-2,2.1463603E-2,0E0,0E0,0E0,4.7600687E-2,0E0,0E0,4.8347026E-2,0E0,0E0,1.7010549E-1,0E0,0E0,0E0,0E0,3.3632E-2,0E0,1.514972E-2,1.8291134E-2,0E0,4.1870236E-2,2.6373472E-2,3.6680236E-2,0E0,0E0,0E0,2.095975E-2,4.8059255E-2,0E0,0E0,6.352902E-2,0E0,1.6859137E-2,0E0,0E0,1.5763158E-2,0E0,4.143244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,36,36,37,37,39,39,40,40,41,41,42,42,44,44,50,50,53,53,54,54,55,55,56,56,58,58,61,61,62,62,63,63,64,64,65,65,67,67,68,68,72,72,73,73,74,74,83,83,87,87,88,88,89,89,90,90,92,92,93,93,94,94,97,97,98,98,102,102,105,105,108,108,113,113,115,115,116,116,118,118,119,119,120,120,124,124,125,125,128,128,130,130,133,133,135,135],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,-1,-1,-1,38,-1,40,42,44,46,48,50,52,54,56,-1,-1,58,60,-1,62,64,66,68,-1,70,-1,-1,-1,-1,-1,72,-1,-1,74,76,78,80,-1,82,-1,-1,84,86,88,90,92,-1,94,96,-1,-1,-1,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,104,-1,-1,-1,106,108,110,112,-1,114,116,118,-1,-1,120,122,-1,-1,-1,124,-1,-1,126,-1,-1,128,-1,-1,-1,-1,130,-1,132,134,-1,136,138,140,-1,-1,-1,142,144,-1,-1,146,-1,148,-1,-1,150,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.69E2,3.5964913E0,1E0,2.56E2,2.0303884E6,3.5E0,1.0534078E4,4.9E2,2.7224753E2,6.5199727E-1,3.0536E4,1.24141544E-1,-9.849156E-3,1.7068776E10,8.173714E-3,5.853E3,1.2E1,1.1244019E2,1.9495455E6,4.586207E0,-6.1517353E-3,-1.5598287E-2,-6.3831935E-4,1.618692E6,1.7511126E-3,4.2E1,1.1806228E6,9E0,2E0,5.912179E3,1.43E2,1.4702564E11,2.8302418E10,1.946094E6,3.0851678E-3,-1.9282785E-3,7.8625E1,7.980211E5,-3.4498938E-3,8.295515E7,3.2462872E5,8.93E2,3.1E1,-3.9011112E-3,1.9363636E1,-5.0963473E-4,-5.047909E-3,2.9819328E-3,-3.3783424E-4,-4.590922E-3,5.0363636E1,-1.271066E-3,1.8554521E-3,1.7934177E2,1.4386049E6,7.547528E6,2E0,-1.1024267E-3,5.788E3,-2.4895126E-3,5.827955E-4,6.737099E7,2.8470706E2,2.379747E0,3.524172E5,1.9E1,3.2399737E-3,6.255748E6,2.67637E5,2.6933176E-3,-7.9541194E-4,-3.6056782E-3,4.3297894E9,3.6507E4,2.4203823E0,-2.8430296E-3,-1.480468E-2,3.1989787E-3,-5.8812666E-4,-3.7955975E-3,7.6137817E-6,3.7556288E-3,-2.2715343E-4,2.2116136E5,6.1319177E-3,-1.1956391E-3,2.3396944E-3,4.9833015E1,1.027972E0,6.27907E0,8.7692904E7,2.8258592E-3,3.7481603E8,1.5994397E0,1E0,7.787996E-4,-3.992921E-3,1E0,5.08E2,-6.8492896E-4,3.8153015E-3,-6.227962E-4,3.1706784E0,5.9769296E-5,3.754632E-3,2.19724E5,5.2936114E-3,-7.1155108E-3,1E0,-1.4979242E-2,-2.590533E-3,3.8339172E-3,-1.960762E-4,3.4779343E1,-3.7293495E-3,4.911E3,1.3388E4,-1.1164962E-3,1E0,2.590909E0,1.6808511E0,-8.078828E-4,2.8073252E-3,-8.479045E-3,1.6E1,1.915148E7,-2.4095168E-3,-8.627255E-3,1.7142857E0,-3.1907673E-3,2.2092846E7,4.756977E-3,1.2897695E-3,5.929839E8,2.9774604E-3,1.7655972E0,-6.30025E-4,-1.1709518E-3,3.593029E-4,8.3589846E-5,-3.1039051E-3,-2.1760524E-3,-5.29617E-3,5.036642E-4,3.1325482E-3,2.3443638E-3,-2.8171093E-4,1.148037E-3,-1.9558978E-3,1.1586294E-3,-9.785327E-4,4.343763E-3,3.077762E-4],"split_indices":[2,68,17,10,60,68,4,1,67,57,1,53,0,46,0,9,33,67,43,68,0,0,0,62,0,0,43,70,32,48,2,46,46,60,0,0,4,66,0,7,48,8,3,0,4,0,0,0,0,0,62,0,0,70,43,5,8,0,12,0,0,7,67,69,48,8,0,60,9,0,0,0,46,1,69,0,0,0,0,0,0,0,0,43,0,0,0,73,68,71,5,0,47,69,14,0,0,30,2,0,0,0,69,0,0,1,0,0,15,0,0,0,0,71,0,44,2,0,113,69,68,0,0,0,3,12,0,0,71,0,60,0,0,5,0,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.008E3,4.19E2,5.89E2,3.81E2,3.8E1,4E1,5.49E2,2.77E2,1.04E2,2.9E1,9E0,3.6E1,4E0,5.42E2,7E0,5.2E1,2.25E2,8.2E1,2.2E1,2.5E1,4E0,5E0,4E0,2.9E1,7E0,3.46E2,1.96E2,3.3E1,1.9E1,1.5E1,2.1E2,2.8E1,5.4E1,1.6E1,6E0,1E1,1.5E1,1.4E1,1.5E1,4E1,3.06E2,4.8E1,1.48E2,6E0,2.7E1,4E0,1.5E1,1.1E1,4E0,6E0,2.04E2,1.9E1,9E0,4.6E1,8E0,8E0,8E0,4E0,1.1E1,8E0,6E0,2.6E1,1.4E1,2.9E2,1.6E1,3.8E1,1E1,1.39E2,9E0,6E0,2.1E1,6E0,1.98E2,1E1,3.6E1,4E0,4E0,4E0,4E0,4E0,4E0,7E0,4E0,1.8E1,8E0,5E0,9E0,7.4E1,2.16E2,8E0,8E0,6E0,3.2E1,4.6E1,9.3E1,4E0,5E0,1.79E2,1.9E1,5E0,5E0,8E0,2.8E1,6E0,1.2E1,6.6E1,8E0,7E0,2.09E2,4E0,4E0,4E0,4E0,2.5E1,7E0,1.4E1,3.2E1,5E0,8.8E1,1.49E2,3E1,4E0,1.5E1,5E0,2.3E1,6E1,6E0,4E0,2.05E2,7E0,1.8E1,9E0,5E0,2.8E1,4E0,8.4E1,4E0,2.9E1,1.2E2,1.4E1,1.6E1,1.5E1,8E0,3.4E1,2.6E1,2.1E1,1.84E2,1.4E1,4E0,1.2E1,1.6E1,7.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[5.9554825E-4,-2.1876628E-2,1.7934639E-2,-1.5649695E-2,-8.1401184E-2,-6.374116E-3,3.300077E-2,-4.8421673E-2,-6.5315063E-3,-9.275872E-3,-4.74066E-2,-4.124915E-3,-4.7660996E-3,3.0714719E-2,7.466498E-3,1.7306872E-2,-5.5210866E-2,7.44051E-2,-1.2798433E-2,-2.7949918E-2,-7.1453936E-3,-8.725113E-3,5.8712777E-2,6.656207E-2,2.2821276E-2,-1.1501417E-3,2.5893515E-3,-8.200504E-2,-3.4653146E-2,1.3743595E-3,9.208301E-2,-1.5352372E-2,4.38133E-3,-4.439506E-2,2.9176925E-3,-5.7045175E-3,-3.083982E-3,-1.1341189E-4,3.7503797E-3,5.5552263E-2,7.4794665E-3,4.1974522E-2,1.0176706E-2,-4.349663E-3,-4.4510656E-4,2.5233554E-5,-5.1169906E-2,7.236533E-3,5.751145E-2,-7.110355E-3,-4.8197847E-2,-4.808201E-3,-1.6929764E-2,-1.8330231E-2,8.926301E-3,7.0742674E-2,1.1256013E-2,1.344279E-2,6.1497293E-2,1.8943131E-2,-6.7825414E-2,-6.1883503E-3,-3.0879207E-2,4.7989172E-4,4.1434774E-3,-1.0886417E-2,7.818445E-2,-9.160255E-2,-1.4865039E-2,2.2064399E-3,-1.7261893E-3,-2.8859276E-2,1.7495886E-3,1.5174884E-2,-2.2956925E-3,5.894271E-2,5.6828447E-3,-2.3596713E-3,1.6765526E-3,5.973904E-2,-9.411233E-3,1.3350567E-1,4.8250157E-2,-5.6302366E-3,2.3363248E-2,-1.2650138E-2,4.891592E-4,4.9221294E-4,-2.413337E-3,1.4825744E-3,-2.4918726E-2,5.2794244E-4,5.5801272E-3,-1.0495287E-1,-1.4707184E-3,2.0835115E-2,-1.8893981E-3,-1.6419897E-2,-1.0731031E-1,2.0827616E-2,-1.8675007E-3,6.684442E-2,-2.308612E-4,3.9988137E-3,2.8540035E-5,-2.1667664E-3,8.927682E-3,7.96291E-3,2.7801772E-3,5.6005865E-2,-1.3671643E-3,5.9227683E-2,9.508964E-3,-1.1720821E-2,3.2616075E-2,-4.370065E-3,-1.9740622E-2,-2.96228E-3,-6.4389375E-3,-3.0324302E-4,3.5548224E-3,-3.1264443E-2,9.441818E-4,-8.904431E-3,4.7323437E-4,2.8400443E-2,-2.19275E-2,3.49129E-3,1.322988E-4,4.7796655E-2,-1.1387231E-3,4.0509272E-2,4.937208E-3,-2.9512115E-2,8.0354266E-2,2.7870711E-2,-1.593654E-2,2.0722274E-4,-1.6170022E-3,2.2537686E-4,2.6415347E-3,-1.3797373E-3,1.1044816E-4,-2.77567E-4,-2.601295E-3,9.765567E-4,3.2912276E-3,-2.532683E-3,9.287375E-4,3.7208193E-3,-3.5119473E-4,7.6728576E-4,2.7136812E-3,-4.1455464E-3,1.6913696E-3,2.2544197E-3,5.7380674E-3,6.4408325E-4,3.4326196E-3,-1.1569837E-4,-5.0608525E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,45,-1,47,49,-1,51,-1,53,-1,-1,-1,55,-1,57,59,-1,-1,-1,61,-1,63,65,67,-1,69,71,73,75,77,79,81,83,85,-1,87,-1,-1,89,91,93,95,-1,-1,97,-1,99,-1,101,-1,-1,-1,103,105,107,109,-1,111,-1,-1,-1,-1,113,115,-1,-1,117,-1,119,-1,121,123,125,-1,127,-1,-1,-1,-1,129,-1,-1,131,-1,133,135,137,139,-1,141,-1,-1,-1,-1,143,-1,-1,-1,145,147,-1,-1,149,-1,151,-1,153,155,157,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9900133E-1,1.640384E-1,2.1189645E-1,1.2070628E-1,1.6092828E-1,4.7904048E-2,1.00001514E-1,4.022497E-2,1.6224943E-1,0E0,6.663172E-2,6.352863E-2,0E0,9.808254E-2,0E0,1.610988E-2,4.122442E-2,1.5417591E-2,8.282385E-2,4.7750674E-2,0E0,3.5376836E-2,2.0812105E-2,5.7131976E-2,6.980637E-2,0E0,0E0,2.8977945E-2,2.7352761E-2,0E0,2.6699856E-2,7.816404E-2,0E0,4.0855873E-2,0E0,3.6184903E-2,0E0,0E0,0E0,3.8230658E-2,0E0,6.3230455E-2,1.21633574E-1,0E0,0E0,0E0,4.9612872E-2,0E0,1.7263532E-2,7.589542E-2,8.255276E-2,0E0,2.6900316E-2,6.3225806E-2,3.4085285E-2,1.926884E-2,2.6820078E-2,5.134888E-2,5.8423817E-2,1.0180987E-1,2.9364616E-1,0E0,2.4508104E-2,0E0,0E0,3.891562E-2,2.7058162E-2,1.6607001E-2,3.2449704E-2,0E0,0E0,8.5091494E-2,0E0,2.6927112E-2,0E0,1.902768E-2,0E0,0E0,0E0,2.5313951E-2,2.310673E-2,1.8119186E-2,3.7277356E-2,0E0,7.645259E-2,0E0,0E0,0E0,0E0,4.971044E-2,3.601417E-2,0E0,0E0,1.9173503E-2,0E0,2.3565665E-2,0E0,4.331491E-2,1.2248604E-1,2.5229808E-2,0E0,1.8207744E-2,0E0,0E0,0E0,0E0,3.119448E-2,0E0,0E0,3.8233235E-2,0E0,8.298649E-2,5.324046E-2,3.2225855E-2,2.4246797E-2,0E0,2.2254046E-2,0E0,0E0,0E0,0E0,3.507976E-2,0E0,0E0,0E0,1.8716916E-2,1.7692465E-2,0E0,0E0,2.0564776E-2,0E0,1.775983E-2,0E0,3.912237E-2,4.236895E-2,4.164854E-2,6.0078975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,33,33,35,35,39,39,41,41,42,42,46,46,48,48,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,65,65,66,66,67,67,68,68,71,71,73,73,75,75,79,79,80,80,81,81,82,82,84,84,89,89,90,90,93,93,95,95,97,97,98,98,99,99,101,101,106,106,109,109,111,111,112,112,113,113,114,114,116,116,121,121,125,125,126,126,129,129,131,131,133,133,134,134,135,135,136,136],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,46,-1,48,50,-1,52,-1,54,-1,-1,-1,56,-1,58,60,-1,-1,-1,62,-1,64,66,68,-1,70,72,74,76,78,80,82,84,86,-1,88,-1,-1,90,92,94,96,-1,-1,98,-1,100,-1,102,-1,-1,-1,104,106,108,110,-1,112,-1,-1,-1,-1,114,116,-1,-1,118,-1,120,-1,122,124,126,-1,128,-1,-1,-1,-1,130,-1,-1,132,-1,134,136,138,140,-1,142,-1,-1,-1,-1,144,-1,-1,-1,146,148,-1,-1,150,-1,152,-1,154,156,158,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9236734E5,5.27439E0,1.194E3,2.08E2,3.964269E3,1.25E2,2.88755E5,1.863E3,1.1E1,-9.275872E-3,3.6440647E5,1.5512196E1,-4.7660996E-3,5E0,7.466498E-3,1.1E1,1.63757E4,3.806E3,4.435876E2,2.5393645E5,-7.1453936E-3,1.9E1,4.745397E6,6.439711E1,9.47E4,-1.1501417E-3,2.5893515E-3,5.619529E3,6.0921145E4,1.3743595E-3,4.1957852E2,3.9E2,4.38133E-3,1.317484E1,2.9176925E-3,2.5367088E2,-3.083982E-3,-1.1341189E-4,3.7503797E-3,4.643602E8,7.4794665E-3,6.5199727E-1,3.3386944E5,-4.349663E-3,-4.4510656E-4,2.5233554E-5,2.512195E0,7.236533E-3,9.7573663E2,5.202E3,4.5454545E0,-4.808201E-3,3.3817584E7,8.8474586E8,2E0,2.4545455E0,2.58466E5,2.16948E3,1.2877E4,1.8434782E2,6.27907E0,-6.1883503E-3,4E0,4.7989172E-4,4.1434774E-3,3.0751158E4,1.1481482E0,1.2152382E5,1.4E1,2.2064399E-3,-1.7261893E-3,2.8302418E10,1.7495886E-3,1.45752E8,-2.2956925E-3,2.0342858E0,5.6828447E-3,-2.3596713E-3,1.6765526E-3,1E0,1.8729467E8,1.8383686E5,3.1790426E8,-5.6302366E-3,1.3208092E0,-1.2650138E-2,4.891592E-4,4.9221294E-4,-2.413337E-3,5.117647E0,2.560356E5,5.2794244E-4,5.5801272E-3,2.1251968E1,-1.4707184E-3,6.364486E0,-1.8893981E-3,1.449E3,1.3254E4,3.9444446E2,-1.8675007E-3,1E0,-2.308612E-4,3.9988137E-3,2.8540035E-5,-2.1667664E-3,1.9495455E6,7.96291E-3,2.7801772E-3,9.356961E6,-1.3671643E-3,5.5933E5,8.998703E-1,4.0695653E0,1.2860047E5,-4.370065E-3,1.3631483E8,-2.96228E-3,-6.4389375E-3,-3.0324302E-4,3.5548224E-3,1.703125E0,9.441818E-4,-8.904431E-3,4.7323437E-4,1.978531E6,4.797342E2,3.49129E-3,1.322988E-4,5.94E2,-1.1387231E-3,2.0094643E0,4.937208E-3,2.2145481E5,2.7748251E1,2.6862775E6,3.7035952E6,2.0722274E-4,-1.6170022E-3,2.2537686E-4,2.6415347E-3,-1.3797373E-3,1.1044816E-4,-2.77567E-4,-2.601295E-3,9.765567E-4,3.2912276E-3,-2.532683E-3,9.287375E-4,3.7208193E-3,-3.5119473E-4,7.6728576E-4,2.7136812E-3,-4.1455464E-3,1.6913696E-3,2.2544197E-3,5.7380674E-3,6.4408325E-4,3.4326196E-3,-1.1569837E-4,-5.0608525E-3],"split_indices":[43,68,2,2,43,8,2,9,10,0,43,71,0,3,0,5,43,1,71,43,0,73,60,73,9,0,0,48,60,0,4,0,0,69,0,67,0,0,0,7,0,57,48,0,0,0,69,0,48,2,69,0,7,5,32,68,1,48,9,67,71,0,10,0,0,48,68,48,3,0,0,46,0,7,0,68,0,0,0,19,7,48,47,0,68,0,0,0,0,69,62,0,0,73,0,69,0,44,44,67,0,19,0,0,0,0,43,0,0,60,0,9,57,71,43,0,7,0,0,0,0,68,0,0,0,62,4,0,0,0,0,57,0,48,73,62,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.022E3,4.45E2,5.77E2,4.04E2,4.1E1,2.21E2,3.56E2,8.7E1,3.17E2,8E0,3.3E1,2.17E2,4E0,3.51E2,5E0,8E0,7.9E1,2.2E1,2.95E2,2.9E1,4E0,2.03E2,1.4E1,6.2E1,2.89E2,4E0,4E0,3.3E1,4.6E1,7E0,1.5E1,2.89E2,6E0,2.5E1,4E0,1.94E2,9E0,4E0,1E1,5.7E1,5E0,1.14E2,1.75E2,2.8E1,5E0,1.5E1,3.1E1,4E0,1.1E1,2.32E2,5.7E1,7E0,1.8E1,1.04E2,9E1,4.2E1,1.5E1,4.7E1,6.7E1,1.58E2,1.7E1,5E0,2.6E1,5E0,6E0,2.23E2,9E0,2.4E1,3.3E1,4E0,1.4E1,8.8E1,1.6E1,8.2E1,8E0,3.6E1,6E0,4E0,1.1E1,1.5E1,3.2E1,9E0,5.8E1,4E0,1.54E2,4E0,1.3E1,9E0,1.7E1,1.19E2,1.04E2,4E0,5E0,1.9E1,5E0,1.4E1,1.9E1,7.7E1,1.1E1,7.5E1,7E0,3.2E1,4E0,1E1,5E0,1E1,2.2E1,5E0,4E0,5.3E1,5E0,4.2E1,1.12E2,8.4E1,3.5E1,6E0,9.8E1,1E1,9E0,1E1,4E0,5.5E1,2.2E1,6E0,5E0,6.4E1,1.1E1,2.8E1,4E0,1E1,1.2E1,4.2E1,1.1E1,8E0,3.4E1,6.5E1,4.7E1,5E1,3.4E1,1.7E1,1.8E1,6.7E1,3.1E1,2.8E1,2.7E1,5.6E1,8E0,6E0,5E0,6E0,4E0,1.9E1,2.3E1,4E0,4E0,2.1E1,1.3E1,5.1E1,1.4E1,4.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[5.650882E-4,-6.382714E-3,4.63556E-2,-4.511776E-3,-1.3412294E-1,3.9423298E-2,7.0782066E-3,-1.4508722E-2,1.11286E-2,-5.4246124E-2,-1.1599501E-2,-4.332077E-3,4.4395857E-2,-1.2030394E-2,-1.2337878E-1,1.491511E-2,-8.759417E-2,-6.0621193E-3,1.5511045E-3,5.4352574E-2,-1.1092375E-2,5.0111458E-2,-1.4803842E-2,-1.0342491E-2,-2.085917E-3,-3.6253892E-3,2.6633024E-2,-1.04349945E-2,-1.0413991E-3,4.1141868E-2,1.173865E-1,2.0090997E-2,-2.3763739E-3,-2.0018022E-3,8.5639596E-2,-3.736683E-2,-8.735509E-3,-1.0657948E-2,4.758725E-2,7.7458583E-3,3.7162602E-2,-2.2213517E-3,2.1347508E-3,9.1278896E-2,3.0705065E-2,1.3925987E-3,7.3223677E-3,-4.681101E-4,2.97662E-3,5.3506605E-3,1.7191686E-3,-2.4529317E-2,-1.0019165E-1,-3.1257533E-3,-3.690509E-2,-3.4193262E-3,-4.987276E-3,3.6360817E-3,-1.6929218E-4,1.741027E-2,-6.506097E-2,4.6195902E-2,-4.227491E-2,2.1418915E-3,6.282564E-3,1.4381922E-2,3.4968413E-3,-3.7633654E-2,1.8281268E-2,-8.855379E-3,-5.0259758E-2,-6.129496E-3,7.898671E-2,-8.247789E-2,-1.7147567E-2,-1.028234E-2,3.904352E-3,8.705057E-3,4.8260647E-3,1.933137E-5,-5.4296907E-3,3.6354393E-2,9.309365E-2,-3.188885E-3,2.2618618E-4,-1.3390383E-3,2.2053603E-2,2.0223854E-3,-4.3230806E-2,1.3954345E-4,3.213783E-3,-3.52644E-3,3.105137E-4,-3.7718036E-3,-4.6047056E-3,6.1938735E-3,1.602496E-3,-6.2720566E-3,-2.571711E-3,-3.7010428E-2,9.81731E-4,2.1557739E-2,-2.5822636E-2,1.77644E-2,-3.0035512E-3,5.429795E-3,2.80181E-2,1.8132549E-3,5.5008247E-3,1.5818675E-3,-5.221393E-3,-2.0724467E-4,-2.56587E-3,-3.340695E-4,2.2396864E-3,-2.6626028E-3,-2.946884E-4,1.8733478E-4,2.2877564E-3,-4.240829E-3,-6.0451834E-4,2.439537E-3,1.6893951E-4,5.124633E-3,1.0620741E-3,-2.3530629E-3,1.6627699E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,39,-1,41,43,45,47,-1,-1,49,51,53,55,57,59,61,-1,-1,63,65,-1,-1,-1,-1,-1,-1,67,69,71,73,75,-1,-1,-1,77,79,81,83,-1,-1,85,-1,87,89,-1,91,93,95,97,99,101,-1,103,-1,-1,-1,105,107,-1,-1,-1,109,-1,111,-1,-1,-1,-1,113,-1,-1,-1,-1,-1,115,-1,117,119,121,-1,-1,123,-1,-1,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3084688E-1,2.1520646E-1,9.544331E-2,1.3947466E-1,1.07824564E-1,9.026338E-2,0E0,1.4529555E-1,1.3139401E-1,6.5242454E-2,0E0,0E0,7.0215434E-2,9.260291E-2,8.422525E-2,7.309455E-2,1.5535119E-1,0E0,0E0,8.427182E-2,2.6195515E-2,8.0066144E-2,6.963955E-2,0E0,0E0,4.7687605E-2,4.068017E-2,0E0,2.2225477E-2,4.42916E-2,5.9779823E-2,1.6733892E-2,0E0,0E0,2.0534642E-2,8.464593E-2,6.372653E-2,8.13267E-2,2.5805533E-2,5.3905368E-2,9.6673846E-2,0E0,0E0,2.3818687E-2,5.471883E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.173938E-2,7.259257E-2,8.385006E-2,5.891289E-2,6.664728E-2,0E0,0E0,0E0,4.9863756E-2,3.0968875E-2,5.153945E-2,1.8003406E-2,0E0,0E0,1.9602552E-2,0E0,3.378859E-2,1.7007353E-2,0E0,1.9825704E-2,7.1741335E-2,2.4631418E-2,2.1015555E-2,3.7395407E-2,5.102671E-2,0E0,4.256098E-2,0E0,0E0,0E0,6.437777E-2,2.3298353E-2,0E0,0E0,0E0,1.6465941E-2,0E0,3.1226076E-2,0E0,0E0,0E0,0E0,5.819079E-2,0E0,0E0,0E0,0E0,0E0,1.9847237E-2,0E0,1.6439136E-2,5.6781143E-2,2.748986E-2,0E0,0E0,3.580842E-2,0E0,0E0,0E0,3.1975724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,51,51,52,52,53,53,54,54,55,55,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,72,72,73,73,74,74,75,75,77,77,81,81,82,82,86,86,88,88,93,93,99,99,101,101,102,102,103,103,106,106,110,110],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,40,-1,42,44,46,48,-1,-1,50,52,54,56,58,60,62,-1,-1,64,66,-1,-1,-1,-1,-1,-1,68,70,72,74,76,-1,-1,-1,78,80,82,84,-1,-1,86,-1,88,90,-1,92,94,96,98,100,102,-1,104,-1,-1,-1,106,108,-1,-1,-1,110,-1,112,-1,-1,-1,-1,114,-1,-1,-1,-1,-1,116,-1,118,120,122,-1,-1,124,-1,-1,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,3.8304348E1,9.62385E3,9.860918E5,6.9756895E6,1.1770261E8,7.0782066E-3,6.425E1,1.9269184E7,2.0619047E1,-1.1599501E-2,-4.332077E-3,7.3274844E9,6E0,1.75E2,3.49E2,2.395631E7,-6.0621193E-3,1.5511045E-3,3.558194E6,2.948485E8,3.3527696E-1,3.3832976E7,-1.0342491E-2,-2.085917E-3,9.750085E6,1E0,-1.04349945E-2,1.9E1,2.01E2,1.0912606E5,4.435876E2,-2.3763739E-3,-2.0018022E-3,2.0649375E6,7E0,3.9E2,1.9E1,1.6108533E8,2.1391304E1,5.0741018E8,-2.2213517E-3,2.1347508E-3,6E0,5.1167645E6,1.3925987E-3,7.3223677E-3,-4.681101E-4,2.97662E-3,5.3506605E-3,1.7191686E-3,4.7659574E0,5.764631E1,1.23E4,8.638015E4,1.3E1,-4.987276E-3,3.6360817E-3,-1.6929218E-4,5.306E3,2.1557376E0,3.6451373E8,2.9251662E6,2.1418915E-3,6.282564E-3,3.532346E-1,3.4968413E-3,4.5054945E-1,5.667774E0,-8.855379E-3,2.7E1,3.53562E5,2.3266666E0,1.775373E1,2.364E3,1.0865825E0,3.904352E-3,1.0046621E7,4.8260647E-3,1.933137E-5,-5.4296907E-3,2.0115254E2,5.4204545E0,-3.188885E-3,2.2618618E-4,-1.3390383E-3,1.116241E2,2.0223854E-3,6.0921145E4,1.3954345E-4,3.213783E-3,-3.52644E-3,3.105137E-4,1.0564016E3,-4.6047056E-3,6.1938735E-3,1.602496E-3,-6.2720566E-3,-2.571711E-3,1.58E2,9.81731E-4,6.23E2,2.0217392E0,1.422162E3,-3.0035512E-3,5.429795E-3,1.1160929E0,1.8132549E-3,5.5008247E-3,1.5818675E-3,1.5477113E2,-2.0724467E-4,-2.56587E-3,-3.340695E-4,2.2396864E-3,-2.6626028E-3,-2.946884E-4,1.8733478E-4,2.2877564E-3,-4.240829E-3,-6.0451834E-4,2.439537E-3,1.6893951E-4,5.124633E-3,1.0620741E-3,-2.3530629E-3,1.6627699E-3],"split_indices":[2,68,67,43,60,7,0,61,62,71,0,0,5,3,6,10,62,0,0,1,47,71,7,0,0,9,29,0,3,0,48,71,0,0,62,8,0,8,7,71,7,0,0,8,43,0,0,0,0,0,0,69,62,10,43,8,0,0,0,2,69,7,43,0,0,53,0,68,68,0,3,1,68,71,2,71,0,62,0,0,0,67,71,0,0,0,71,0,60,0,0,0,0,4,0,0,0,0,0,8,0,2,71,48,0,0,69,0,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.038E3,9.02E2,1.36E2,8.9E2,1.2E1,1.29E2,7E0,5.43E2,3.47E2,8E0,4E0,4E0,1.25E2,5.32E2,1.1E1,3.35E2,1.2E1,4E0,4E0,1.06E2,1.9E1,2.2E1,5.1E2,4E0,7E0,1.3E2,2.05E2,4E0,8E0,8.9E1,1.7E1,1.1E1,8E0,6E0,1.6E1,1.07E2,4.03E2,1.15E2,1.5E1,7.4E1,1.31E2,4E0,4E0,1.4E1,7.5E1,6E0,1.1E1,7E0,4E0,9E0,7E0,9E1,1.7E1,3.37E2,6.6E1,1.08E2,7E0,9E0,6E0,6.6E1,8E0,1.18E2,1.3E1,8E0,6E0,5.6E1,1.9E1,6.9E1,2.1E1,5E0,1.2E1,3.26E2,1.1E1,1.9E1,4.7E1,1.01E2,7E0,6.1E1,5E0,4E0,4E0,9.9E1,1.9E1,8E0,5E0,8E0,4.8E1,4E0,6.5E1,1.7E1,4E0,8E0,4E0,3.19E2,7E0,4E0,7E0,5E0,1.4E1,3.1E1,1.6E1,3.3E1,6.8E1,5.5E1,6E0,8E0,9.1E1,7E0,1.2E1,3.3E1,1.5E1,1.6E1,4.9E1,3E2,1.9E1,1.8E1,1.3E1,2.1E1,1.2E1,1E1,5.8E1,1.5E1,4E1,4E0,8.7E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[1.4329132E-3,-3.3973034E-2,9.006472E-3,-2.6577396E-2,-1.0070072E-1,-5.4913033E-3,2.2044191E-2,-2.1321176E-2,-6.6375877E-3,-7.2451485E-3,-5.072872E-2,-8.932805E-3,5.2504815E-2,1.892582E-2,8.861062E-2,-4.7090568E-2,-9.810872E-4,1.3020491E-5,-3.6921296E-3,-1.193331E-2,5.3073812E-2,6.366194E-3,2.1156404E-2,2.0924678E-2,-5.8740634E-3,8.931953E-3,2.9918449E-2,-3.435887E-2,-1.1341266E-1,8.072683E-2,-2.2234092E-2,-9.192986E-2,-8.753858E-3,7.4712727E-3,-1.47254765E-2,2.4482051E-3,-1.1671192E-3,1.5946405E-2,6.4122155E-2,2.4216874E-3,-9.7320374E-4,-8.411323E-2,-1.7748777E-2,-9.182542E-3,-1.6746321E-3,5.583645E-3,2.1826164E-3,-2.9036723E-2,1.6939746E-3,1.4866677E-5,-8.511134E-3,-6.1108726E-3,-8.401018E-2,-2.6792507E-3,1.1302668E-3,-6.3615977E-3,1.8324176E-2,7.6316044E-2,-1.3391825E-5,-5.040966E-4,-5.0698896E-3,1.5586565E-3,-3.842713E-2,-1.1178046E-2,-5.4418586E-2,-6.673053E-2,-3.536216E-3,-7.7013066E-3,-1.0084688E-3,4.0267454E-3,1.546076E-2,2.6335793E-3,5.395029E-3,-4.196974E-3,-1.2800932E-2,-3.2464698E-2,3.552997E-2,-3.2223521E-3,-4.291627E-4,-4.294266E-3,-7.520036E-4,2.0293554E-2,-1.1433555E-2,3.2213912E-3,1.3270903E-2,1.3108187E-2,-5.2610193E-3,2.492809E-5,-3.026385E-3,-5.1812316E-5,3.100221E-3,3.5256784E-2,-2.2797983E-2,-6.3956957E-3,-7.7431314E-3,3.137669E-2,4.9798666E-3,-1.7102346E-3,1.9841224E-3,-2.089834E-3,1.9213211E-3,-2.2421593E-3,3.395017E-4,-3.8993752E-3,-2.3666656E-4,2.5502827E-6,2.2970787E-3,-2.6004745E-3,4.136652E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,21,23,25,27,29,-1,-1,31,33,-1,35,37,-1,-1,39,41,43,45,47,49,51,-1,53,-1,-1,55,57,-1,-1,59,61,-1,-1,-1,-1,63,-1,-1,-1,65,67,-1,-1,-1,69,71,-1,-1,-1,-1,73,75,77,79,81,-1,-1,-1,83,-1,-1,-1,85,87,89,-1,-1,-1,-1,91,93,-1,95,97,-1,-1,-1,-1,-1,99,101,-1,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7998543E-1,8.812645E-2,1.6267644E-1,9.972252E-2,4.252772E-2,8.190326E-2,9.235619E-2,8.4342234E-2,0E0,0E0,1.6401028E-2,7.2449096E-2,5.8583103E-2,1.293792E-1,1.1714721E-1,5.6055173E-2,1.5981992E-1,0E0,0E0,9.304864E-2,1.3497083E-1,0E0,2.7939316E-2,9.123343E-2,0E0,0E0,1.7164223E-2,4.8918992E-2,6.5239236E-2,1.982233E-2,3.0314434E-2,1.1926381E-1,7.029548E-2,0E0,2.1767298E-2,0E0,0E0,1.4354266E-1,3.45812E-2,0E0,0E0,2.694162E-2,5.13354E-2,0E0,0E0,0E0,0E0,2.9308613E-2,0E0,0E0,0E0,5.3609833E-2,5.806119E-2,0E0,0E0,0E0,7.3750466E-2,2.225335E-2,0E0,0E0,0E0,0E0,4.4508692E-2,4.0923756E-2,1.8032372E-2,1.7007913E-2,6.27117E-2,0E0,0E0,0E0,4.2688422E-2,0E0,0E0,0E0,6.434387E-2,3.0570865E-2,1.5120134E-2,0E0,0E0,0E0,0E0,5.4321237E-2,1.1610294E-1,0E0,5.298472E-2,3.1504724E-2,0E0,0E0,0E0,0E0,0E0,3.3476762E-2,1.7424518E-2,0E0,4.858224E-2,6.377583E-2,5.9914872E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,22,22,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,37,37,38,38,41,41,42,42,47,47,51,51,52,52,56,56,57,57,62,62,63,63,64,64,65,65,66,66,70,70,74,74,75,75,76,76,81,81,82,82,84,84,85,85,91,91,92,92,94,94,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,22,24,26,28,30,-1,-1,32,34,-1,36,38,-1,-1,40,42,44,46,48,50,52,-1,54,-1,-1,56,58,-1,-1,60,62,-1,-1,-1,-1,64,-1,-1,-1,66,68,-1,-1,-1,70,72,-1,-1,-1,-1,74,76,78,80,82,-1,-1,-1,84,-1,-1,-1,86,88,90,-1,-1,-1,-1,92,94,-1,96,98,-1,-1,-1,-1,-1,100,102,-1,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.147158E4,9.51E3,3.4734247E2,3.8304348E1,3.3025316E5,1.559287E5,4.435876E2,2.71E2,-6.6375877E-3,-7.2451485E-3,4.3E1,2.2503355E1,8.691756E-1,7.134432E7,4.8647133E-1,1.3E2,2.04115E5,1.3020491E-5,-3.6921296E-3,1.16408E5,1.68574E5,6.366194E-3,5.323779E6,5.1167645E6,-5.8740634E-3,8.931953E-3,2.7021693E8,7.785714E0,1.1626786E2,3.8275862E0,1.0652307E7,6.23E2,2.2051188E6,7.4712727E-3,4.232328E3,2.4482051E-3,-1.1671192E-3,1.0280637E0,1.213274E9,2.4216874E-3,-9.7320374E-4,1.11E2,1.1E1,-9.182542E-3,-1.6746321E-3,5.583645E-3,2.1826164E-3,4.217427E7,1.6939746E-3,1.4866677E-5,-8.511134E-3,1.88E2,6.763314E7,-2.6792507E-3,1.1302668E-3,-6.3615977E-3,3.63E2,1.400229E6,-1.3391825E-5,-5.040966E-4,-5.0698896E-3,1.5586565E-3,1.8857143E0,1E0,1.3141646E3,4.3827028E5,5.3061223E0,-7.7013066E-3,-1.0084688E-3,4.0267454E-3,8.118812E-1,2.6335793E-3,5.395029E-3,-4.196974E-3,3.5318289E3,1.26375E1,5.831829E6,-3.2223521E-3,-4.291627E-4,-4.294266E-3,-7.520036E-4,2.277451E2,5.475771E0,3.2213912E-3,5.6516E4,3.6E1,-5.2610193E-3,2.492809E-5,-3.026385E-3,-5.1812316E-5,3.100221E-3,8.612209E-6,3.8575E4,-6.3956957E-3,4.9558692E7,1E0,1.4598765E0,-1.7102346E-3,1.9841224E-3,-2.089834E-3,1.9213211E-3,-2.2421593E-3,3.395017E-4,-3.8993752E-3,-2.3666656E-4,2.5502827E-6,2.2970787E-3,-2.6004745E-3,4.136652E-4],"split_indices":[43,44,67,68,62,48,71,2,0,0,8,71,68,59,57,2,5,0,0,7,9,0,12,43,0,0,47,67,4,69,66,2,66,0,48,0,0,69,7,0,0,1,3,0,0,0,0,60,0,0,0,2,7,0,0,0,44,1,0,0,0,0,68,27,67,43,73,0,0,0,68,0,0,0,62,73,60,0,0,0,0,67,73,0,9,2,0,0,0,0,0,53,9,0,46,29,69,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.042E3,1.83E2,8.59E2,1.66E2,1.7E1,4.07E2,4.52E2,1.6E2,6E0,7E0,1E1,3.85E2,2.2E1,4.33E2,1.9E1,7E1,9E1,4E0,6E0,3.68E2,1.7E1,5E0,1.7E1,4.28E2,5E0,6E0,1.3E1,6E1,1E1,1.8E1,7.2E1,1.3E1,3.55E2,6E0,1.1E1,1E1,7E0,3.85E2,4.3E1,9E0,4E0,1.4E1,4.6E1,4E0,6E0,7E0,1.1E1,6.5E1,7E0,7E0,6E0,3.44E2,1.1E1,5E0,6E0,5E0,3.8E2,3.6E1,7E0,4E0,1E1,1.3E1,3.3E1,3.9E1,2.6E1,1.3E1,3.31E2,4E0,7E0,1.4E1,3.66E2,2.6E1,1E1,1E1,2.3E1,2.7E1,1.2E1,1.9E1,7E0,8E0,5E0,8.2E1,2.49E2,1.3E1,3.53E2,1.9E1,4E0,1.4E1,1.3E1,6E0,6E0,6.1E1,2.1E1,6E0,2.43E2,1.1E2,2.43E2,7E0,1.2E1,4E0,5.7E1,1.1E1,1E1,7E0,2.36E2,4.1E1,6.9E1,1.4E1,2.29E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[1.1671787E-3,-1.6318643E-2,1.8855238E-2,-1.17567815E-2,-8.3146155E-2,-5.6021255E-2,2.3787173E-2,-3.5236605E-2,-5.9766294E-3,-1.2162982E-1,1.7529247E-2,-7.91766E-3,-3.3420462E-2,1.1393533E-1,2.0647649E-2,8.585303E-4,-4.0340494E-2,7.558665E-3,-2.1797104E-2,-1.9651461E-2,-2.0640755E-1,-1.4655216E-3,3.378295E-3,8.368036E-4,-2.2313385E-3,2.0753755E-3,6.153532E-3,-8.023789E-4,3.193607E-2,-5.0421815E-2,-1.2918692E-2,1.0862727E-2,-3.6059632E-3,-2.4322916E-2,3.3252272E-3,-2.1773067E-3,1.3047814E-3,-1.3436436E-2,-2.7457245E-3,-8.843282E-3,6.401614E-2,4.073136E-2,5.2496777E-3,-9.2098214E-2,-4.2823434E-2,-3.801867E-2,1.6598296E-3,1.6413512E-2,-4.13027E-2,1.0997318E-2,-3.1897742E-2,-1.3973577E-3,-5.8147974E-2,9.946798E-2,-1.0840854E-3,2.2004254E-2,6.711193E-2,2.9541608E-2,-8.2224295E-2,-6.1208676E-3,-1.5378622E-3,-2.4805384E-2,-6.9542E-2,2.1254164E-3,-3.1143334E-3,-2.6794795E-2,2.237313E-2,3.6854463E-4,-4.22169E-3,3.7591243E-3,-1.9616357E-3,-5.253208E-2,-5.3759953E-3,-3.91986E-2,6.461366E-3,-5.898438E-3,-2.4197485E-2,6.4986926E-3,2.0354849E-3,2.6862154E-2,-4.052917E-3,2.3195792E-2,7.851363E-2,7.5571276E-2,3.486751E-3,-1.1895893E-2,-8.488627E-3,-3.164914E-3,-5.282204E-4,-6.216524E-3,-2.1999117E-3,-8.763408E-6,-3.5414186E-3,3.0253574E-2,-1.69838E-2,-1.9466795E-2,2.043597E-3,-6.993608E-2,-2.1643689E-2,-1.2486806E-2,3.809162E-3,-6.079649E-2,2.1433367E-3,-6.6817887E-3,3.449045E-2,3.1877044E-4,-3.7676631E-3,-2.6951255E-3,3.0706078E-2,2.968216E-3,-8.8688095E-5,5.6172274E-3,6.477177E-2,9.613334E-2,-6.267547E-4,2.070991E-2,-3.141287E-3,-2.6898896E-3,2.7368255E-3,3.3460625E-3,2.524807E-2,-4.9086116E-2,3.7091244E-2,2.0651363E-3,-3.719452E-2,-3.9838914E-2,-1.09166816E-1,-2.0034993E-3,6.615832E-5,-1.4908453E-3,-4.9295325E-2,-2.780346E-2,-5.6431512E-3,3.7475287E-3,-3.3615E-3,-2.1279707E-4,4.874945E-2,1.934768E-2,6.0871515E-2,7.938859E-2,5.598643E-4,2.6723088E-3,6.3184304E-3,-2.9879278E-3,3.471733E-2,-7.803605E-4,1.5939849E-3,-4.884244E-4,-3.6423355E-3,-5.622856E-4,4.5570764E-3,-4.549525E-4,-3.9587575E-3,5.481957E-4,-2.3209727E-3,-6.2672775E-3,-1.5254904E-3,8.0606254E-4,-1.446732E-3,-9.0561016E-4,-4.565492E-3,-2.7453857E-3,8.774453E-4,9.408216E-4,-1.5628253E-3,3.05456E-3,-2.4745456E-4,2.711664E-3,5.094988E-4,1.1849916E-3,4.369715E-3,4.298175E-3,9.6806703E-4,6.52018E-5,2.6862174E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,45,47,-1,49,-1,-1,-1,-1,-1,51,53,55,57,59,61,63,-1,65,67,69,71,73,75,77,-1,79,81,83,85,-1,-1,87,89,-1,-1,91,93,-1,-1,-1,95,97,99,101,103,-1,105,-1,-1,107,-1,109,111,113,115,-1,117,-1,-1,-1,-1,-1,-1,119,121,123,-1,125,127,129,-1,131,-1,133,135,-1,-1,-1,137,-1,-1,-1,139,141,-1,143,-1,-1,-1,-1,145,147,149,-1,151,153,155,-1,-1,157,159,161,-1,163,-1,-1,165,167,169,171,-1,-1,-1,-1,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1918597E-1,1.5724371E-1,1.907763E-1,6.5883555E-2,1.299859E-1,7.770377E-2,1.3406426E-1,2.7120002E-2,8.410559E-2,2.034356E-1,2.9701091E-2,0E0,2.23427E-2,1.528433E-2,1.131036E-1,0E0,2.3676038E-2,6.0815345E-2,4.4621155E-2,1.7123068E-2,1.4403385E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.500042E-2,7.160881E-2,1.632312E-2,3.2017063E-2,6.0037516E-2,0E0,4.7706194E-2,0E0,0E0,0E0,0E0,0E0,5.3151E-2,5.815901E-2,1.1215052E-1,1.6604725E-1,1.6968362E-2,2.5234304E-2,4.5085967E-2,0E0,4.849542E-2,5.008562E-2,2.9341387E-2,7.961738E-2,3.7981108E-2,4.1759305E-2,2.3478836E-2,0E0,7.485337E-2,4.574713E-2,7.28936E-2,2.1751365E-1,0E0,0E0,1.966922E-2,2.4952158E-2,0E0,0E0,3.0496193E-2,5.1378436E-2,0E0,0E0,0E0,2.3503939E-2,4.2568E-2,4.1623466E-2,4.3489974E-2,3.927986E-2,0E0,2.5533155E-2,0E0,0E0,4.469379E-2,0E0,2.1775529E-2,3.6518157E-2,4.162696E-2,5.0653566E-2,0E0,4.7239628E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7092025E-2,5.0426826E-2,2.55511E-2,0E0,5.7049096E-2,1.5601579E-2,2.4615955E-2,0E0,3.3039324E-2,0E0,5.0581615E-2,1.9178785E-2,0E0,0E0,0E0,4.1993797E-2,0E0,0E0,0E0,4.444571E-2,2.0093426E-2,0E0,4.1714117E-2,0E0,0E0,0E0,0E0,4.894001E-2,1.9793436E-2,3.455285E-2,0E0,2.1701159E-2,1.7265175E-2,3.9899647E-2,0E0,0E0,2.765806E-2,1.8651254E-2,2.0365711E-2,0E0,4.065009E-2,0E0,0E0,2.384675E-2,2.9795833E-2,3.794667E-2,3.4734488E-2,0E0,0E0,0E0,0E0,2.2545572E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,27,27,28,28,29,29,30,30,31,31,33,33,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,65,65,66,66,70,70,71,71,72,72,73,73,74,74,76,76,79,79,81,81,82,82,83,83,84,84,86,86,93,93,94,94,95,95,97,97,98,98,99,99,101,101,103,103,104,104,108,108,112,112,113,113,115,115,120,120,121,121,122,122,124,124,125,125,126,126,129,129,130,130,131,131,133,133,136,136,137,137,138,138,139,139,144,144],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,46,48,-1,50,-1,-1,-1,-1,-1,52,54,56,58,60,62,64,-1,66,68,70,72,74,76,78,-1,80,82,84,86,-1,-1,88,90,-1,-1,92,94,-1,-1,-1,96,98,100,102,104,-1,106,-1,-1,108,-1,110,112,114,116,-1,118,-1,-1,-1,-1,-1,-1,120,122,124,-1,126,128,130,-1,132,-1,134,136,-1,-1,-1,138,-1,-1,-1,140,142,-1,144,-1,-1,-1,-1,146,148,150,-1,152,154,156,-1,-1,158,160,162,-1,164,-1,-1,166,168,170,172,-1,-1,-1,-1,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,8.719101E0,5.3E1,2.71E2,3.535931E2,9.7573663E2,2.82E2,1.957E3,1E0,3.89E2,1.2674157E1,-7.91766E-3,1.2E1,4.814389E4,8.484036E5,8.585303E-4,1.6E1,1.6942337E1,6.048589E-4,1.0428572E1,1E0,-1.4655216E-3,3.378295E-3,8.368036E-4,-2.2313385E-3,2.0753755E-3,6.153532E-3,2.4693803E2,2.1356704E0,8E0,1.43E2,1E0,-3.6059632E-3,3.2931E4,3.3252272E-3,-2.1773067E-3,1.3047814E-3,-1.3436436E-2,-2.7457245E-3,4.4744192E8,1.3745962E0,7.956885E-1,3.0561172E5,1.8736842E0,1.62E2,1.7387315E2,1.6598296E-3,1.1379E4,6.763314E7,4.97E2,1.559733E6,1E0,1.48E2,8.036266E-2,-1.0840854E-3,3.49E2,2.297E3,1.684E0,1.2825651E0,-6.1208676E-3,-1.5378622E-3,6.48334E5,1.84E2,2.1254164E-3,-3.1143334E-3,8E0,6.764151E0,3.6854463E-4,-4.22169E-3,3.7591243E-3,1.7697E4,4.4E2,1.6957658E6,1.8383686E5,5.4275E2,-5.898438E-3,1.559733E6,6.4986926E-3,2.0354849E-3,1.1283241E6,-4.052917E-3,1.0586236E5,7.4010696E0,2.596275E3,3.1790426E8,-1.1895893E-2,1.4817301E3,-3.164914E-3,-5.282204E-4,-6.216524E-3,-2.1999117E-3,-8.763408E-6,-3.5414186E-3,1.84E2,1.0722478E5,1.3529412E0,2.043597E-3,1.17E2,2.8266037E2,3.4E1,3.809162E-3,7.601393E2,2.1433367E-3,4.8064E4,5.619529E3,3.1877044E-4,-3.7676631E-3,-2.6951255E-3,2.7777777E0,2.968216E-3,-8.8688095E-5,5.6172274E-3,4.1973075E6,6.46E2,-6.267547E-4,1.7717391E0,-3.141287E-3,-2.6898896E-3,2.7368255E-3,3.3460625E-3,8.754E3,1.2894E4,1.4E1,2.0651363E-3,9.489462E6,1.625E0,7.694314E-7,-2.0034993E-3,6.615832E-5,1.2751362E8,3.9939122E5,2.067E3,-5.6431512E-3,2.1068998E6,-3.3615E-3,-2.1279707E-4,6.0091515E6,1.125584E-3,1E0,9.837502E1,5.598643E-4,2.6723088E-3,6.3184304E-3,-2.9879278E-3,4.3948618E8,-7.803605E-4,1.5939849E-3,-4.884244E-4,-3.6423355E-3,-5.622856E-4,4.5570764E-3,-4.549525E-4,-3.9587575E-3,5.481957E-4,-2.3209727E-3,-6.2672775E-3,-1.5254904E-3,8.0606254E-4,-1.446732E-3,-9.0561016E-4,-4.565492E-3,-2.7453857E-3,8.774453E-4,9.408216E-4,-1.5628253E-3,3.05456E-3,-2.4745456E-4,2.711664E-3,5.094988E-4,1.1849916E-3,4.369715E-3,4.298175E-3,9.6806703E-4,6.52018E-5,2.6862174E-3],"split_indices":[2,69,44,2,70,48,44,9,26,2,69,0,3,60,43,0,3,71,52,68,29,0,0,0,0,0,0,71,53,0,2,19,0,9,0,0,0,0,0,7,53,53,48,68,2,48,0,9,7,2,9,29,0,72,0,8,2,69,69,0,0,46,44,0,0,8,71,0,0,0,9,10,66,48,67,0,9,0,0,60,0,48,71,67,47,0,67,0,0,0,0,0,0,44,66,68,0,10,4,8,0,67,0,44,48,0,0,0,68,0,0,0,43,0,0,69,0,0,0,0,1,12,3,0,12,68,52,0,0,7,60,2,0,66,0,0,62,53,29,73,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E3,5.18E2,5.12E2,4.86E2,3.2E1,3.1E1,4.81E2,9.5E1,3.91E2,2.3E1,9E0,4E0,2.7E1,1.5E1,4.66E2,8E0,8.7E1,2.11E2,1.8E2,1.1E1,1.2E1,5E0,4E0,6E0,2.1E1,4E0,1.1E1,1.61E2,3.05E2,6.3E1,2.4E1,2.04E2,7E0,1.76E2,4E0,7E0,4E0,7E0,5E0,1.44E2,1.7E1,2.29E2,7.6E1,8E0,5.5E1,1.6E1,8E0,1.85E2,1.9E1,3.1E1,1.45E2,1.26E2,1.8E1,1.2E1,5E0,1.35E2,9.4E1,6E1,1.6E1,4E0,4E0,3.4E1,2.1E1,4E0,1.2E1,2.2E1,1.63E2,1E1,9E0,4E0,2.7E1,8.1E1,6.4E1,2.1E1,1.05E2,5E0,1.3E1,6E0,6E0,1.3E2,5E0,2E1,7.4E1,2.1E1,3.9E1,4E0,1.2E1,7E0,2.7E1,4E0,1.7E1,1.5E1,7E0,1.36E2,2.7E1,2E1,7E0,5.1E1,3E1,6E1,4E0,1.7E1,4E0,7.2E1,3.3E1,9E0,4E0,5E0,1.25E2,7E0,1.3E1,1.6E1,5.8E1,1.7E1,4E0,3.2E1,7E0,7E0,5E0,1.3E1,1.23E2,1.7E1,1E1,4E0,1.6E1,3E1,2.1E1,1.5E1,1.5E1,4.7E1,1.3E1,1.2E1,5E0,6.3E1,9E0,9E0,2.4E1,9.2E1,3.3E1,4.5E1,1.3E1,1E1,7E0,4E0,2.8E1,2.2E1,1.01E2,8E0,9E0,6E0,4E0,1.1E1,5E0,5E0,2.5E1,1.5E1,6E0,2.9E1,1.8E1,9E0,4E0,7E0,5E0,4.4E1,1.9E1,1.8E1,6E0,1.5E1,7.7E1,1.7E1,1.6E1,3.6E1,9E0,1.2E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[-3.4122604E-3,-2.430848E-2,9.615928E-3,-1.9493194E-2,-1.2938137E-1,1.4941398E-2,-4.113631E-2,-1.576189E-2,-6.713998E-2,-1.2116654E-2,-5.2154955E-2,3.404663E-2,3.869755E-3,-9.680642E-3,-2.5365511E-2,-4.5177612E-2,-1.0360505E-2,-2.1487218E-1,6.372106E-3,9.4395614E-4,-4.204237E-3,2.5462193E-2,6.391187E-2,-6.043369E-2,1.0937199E-2,-3.910601E-2,3.5298183E-3,-8.04002E-3,-6.352307E-2,2.92759E-4,-2.0525085E-2,-3.4781634E-3,-1.43902395E-2,-1.47886565E-2,2.5264043E-3,6.745667E-2,1.3960125E-2,7.677619E-2,2.1064932E-2,-1.634125E-1,2.0904155E-2,2.8469943E-2,-4.453927E-3,-5.89141E-2,5.100592E-4,-1.5182093E-3,1.8113337E-3,9.6618006E-4,-3.4525094E-3,-7.189631E-3,3.8627807E-2,-6.6283636E-2,-1.05567835E-2,1.8204711E-3,-2.276437E-3,3.5901018E-3,-4.4359217E-4,2.3128465E-3,4.5508087E-2,8.90509E-2,1.1188345E-3,-1.0882863E-3,4.0768897E-3,-9.455465E-3,-1.6631957E-2,-2.6570057E-4,4.088667E-3,7.813883E-3,4.2457435E-2,6.957198E-4,-8.301094E-3,-4.130163E-2,-7.3849675E-3,1.4438928E-2,-1.6662836E-2,-1.058075E-3,6.0410284E-2,-4.1354834E-3,-1.5883277E-3,-6.850375E-3,-4.5682453E-3,-6.2522357E-3,3.1907817E-3,6.395616E-2,-3.237498E-4,5.1193666E-2,1.23251446E-1,-2.5711127E-3,1.4145382E-3,1.4943029E-2,-3.59765E-3,4.8511443E-3,2.9721396E-2,-5.66817E-3,6.7258045E-2,-3.552426E-3,-1.2119684E-3,1.5243745E-4,2.5778625E-3,-3.6472242E-2,-2.3530026E-3,3.5207032E-3,1.8926304E-4,-1.917663E-2,1.7084794E-2,-3.6043897E-2,6.5483446E-3,-1.00886034E-4,3.6870416E-3,3.405424E-3,-3.9663675E-4,6.8591866E-3,2.225105E-3,-1.7099573E-3,5.296983E-2,1.3106582E-2,6.0004137E-2,1.35541065E-2,-1.9758547E-2,4.5102823E-4,4.032157E-3,-3.6705396E-4,-2.844864E-3,2.3753118E-2,-1.996189E-2,9.818433E-4,-3.0247677E-2,-1.1644831E-2,1.8071781E-3,-8.0444245E-4,-4.400291E-3,3.0311547E-2,-1.7426096E-2,-2.5337454E-2,3.1201009E-2,3.9751194E-3,-1.2712923E-3,5.095153E-2,-2.8290223E-2,3.9076055E-3,1.6565343E-2,-9.108564E-4,5.9718102E-2,-5.727447E-2,-6.9668833E-3,3.9136843E-3,-2.7331631E-5,-2.6043411E-3,2.8058012E-6,-5.1617913E-5,-1.8449372E-3,4.8178105E-4,-2.8277594E-3,6.156484E-4,2.9867042E-3,-2.7970145E-3,9.3909475E-6,2.2863483E-3,-2.1491125E-3,-3.2629122E-4,2.5960503E-3,2.5885573E-4,3.5710714E-3,-1.2076759E-4,-3.81259E-3,-1.8508105E-3,2.5246267E-3,-2.178779E-3,8.830343E-4,3.728812E-3,-1.2060668E-4,-6.442816E-4,-3.8868415E-3,2.7928483E-3,-7.7613455E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,29,31,33,-1,-1,35,37,39,41,43,-1,45,47,49,51,-1,-1,53,-1,55,57,59,61,63,65,67,69,71,-1,-1,-1,-1,-1,73,75,77,79,-1,-1,-1,-1,81,83,85,-1,-1,-1,87,-1,-1,-1,89,91,93,-1,95,-1,97,99,-1,101,-1,-1,103,-1,105,-1,107,-1,109,111,-1,-1,113,-1,-1,115,117,119,-1,-1,-1,-1,121,123,-1,-1,125,127,129,131,-1,-1,-1,-1,-1,-1,133,135,137,139,141,143,-1,-1,-1,-1,145,147,-1,149,151,-1,-1,-1,153,155,157,159,-1,-1,161,163,-1,165,167,169,171,173,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.779184E-1,1.9581272E-1,1.7062047E-1,6.578879E-2,1.6078484E-1,1.2043826E-1,1.5403841E-1,5.5141896E-2,2.9459193E-1,0E0,3.5697617E-2,5.1955268E-2,1.6517586E-1,0E0,8.129669E-2,3.6279805E-2,3.216694E-2,9.7919494E-2,2.0233218E-2,0E0,0E0,7.827414E-2,2.3737773E-2,3.049547E-1,8.836733E-2,5.0315738E-2,0E0,2.3571357E-2,3.5632312E-2,4.2156037E-2,6.863279E-2,0E0,0E0,2.8106462E-2,0E0,2.839829E-2,4.7662977E-2,2.0423293E-2,3.805142E-2,4.8086843E-1,3.8347248E-2,4.3873988E-2,1.5803763E-1,5.739686E-2,0E0,0E0,0E0,0E0,0E0,2.544228E-2,3.3550337E-2,1.6933404E-2,4.057107E-2,0E0,0E0,0E0,0E0,5.531943E-2,3.435672E-2,2.9942974E-2,0E0,0E0,0E0,2.0217083E-2,0E0,0E0,0E0,3.9648652E-2,6.937079E-2,7.281602E-2,0E0,1.5144557E-2,0E0,1.7323423E-2,2.4292538E-2,0E0,1.570034E-2,0E0,0E0,3.6321416E-2,0E0,3.3032175E-2,0E0,2.7433239E-2,0E0,2.0824704E-2,1.936005E-2,0E0,0E0,3.7452903E-2,0E0,0E0,3.7955493E-2,4.279112E-2,1.6413674E-2,0E0,0E0,0E0,0E0,2.5294248E-2,2.3918843E-2,0E0,0E0,3.6897816E-2,2.7040808E-2,2.7020108E-2,3.5280492E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.344991E-2,5.0283797E-2,8.134108E-2,2.7424872E-2,4.4794347E-2,4.327106E-2,0E0,0E0,0E0,0E0,3.1215465E-2,2.2603339E-2,0E0,1.8015005E-2,2.1424718E-2,0E0,0E0,0E0,1.7218739E-2,2.357597E-2,4.1924838E-2,1.7609848E-2,0E0,0E0,3.1825155E-2,3.459797E-2,0E0,2.5910014E-2,4.906027E-2,2.1146942E-2,2.559235E-2,4.6757687E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,50,50,51,51,52,52,57,57,58,58,59,59,63,63,67,67,68,68,69,69,71,71,73,73,74,74,76,76,79,79,81,81,83,83,85,85,86,86,89,89,92,92,93,93,94,94,99,99,100,100,103,103,104,104,105,105,106,106,113,113,114,114,115,115,116,116,117,117,118,118,123,123,124,124,126,126,127,127,131,131,132,132,133,133,134,134,137,137,138,138,140,140,141,141,142,142,143,143,144,144],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,30,32,34,-1,-1,36,38,40,42,44,-1,46,48,50,52,-1,-1,54,-1,56,58,60,62,64,66,68,70,72,-1,-1,-1,-1,-1,74,76,78,80,-1,-1,-1,-1,82,84,86,-1,-1,-1,88,-1,-1,-1,90,92,94,-1,96,-1,98,100,-1,102,-1,-1,104,-1,106,-1,108,-1,110,112,-1,-1,114,-1,-1,116,118,120,-1,-1,-1,-1,122,124,-1,-1,126,128,130,132,-1,-1,-1,-1,-1,-1,134,136,138,140,142,144,-1,-1,-1,-1,146,148,-1,150,152,-1,-1,-1,154,156,158,160,-1,-1,162,164,-1,166,168,170,172,174,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,6.0692043E0,2.856934E7,1.6631816E8,2E0,3.72381E5,2.7920364E2,1.63757E4,5.1942E4,-1.2116654E-2,3.3817584E7,8.260895E8,1E0,-9.680642E-3,2.3751075E5,5.4009717E3,3.66025E5,3E0,2.906237E2,9.4395614E-4,-4.204237E-3,1.6869704E6,2.8277853E11,1.048796E6,1.8497453E0,2.0618556E-2,3.5298183E-3,2.5192308E0,8E0,2E1,8.39777E5,-3.4781634E-3,-1.43902395E-2,3.5714287E-1,2.5264043E-3,3.89E2,1.38132E5,1.0284041E2,4.7083335E0,1.8669039E1,1.5971001E1,5.1191E4,1.9500381E2,1.6322028E3,5.100592E-4,-1.5182093E-3,1.8113337E-3,9.6618006E-4,-3.4525094E-3,1.2E1,3.2E1,1.1732674E1,7.814751E6,1.8204711E-3,-2.276437E-3,3.5901018E-3,-4.4359217E-4,1.6322028E3,1.3E1,1.0717949E1,1.1188345E-3,-1.0882863E-3,4.0768897E-3,1.03860024E8,-1.6631957E-2,-2.6570057E-4,4.088667E-3,2.152E3,1.5037551E4,1.3388E4,-8.301094E-3,1.775373E1,-7.3849675E-3,2.1924414E2,1.4473684E-1,-1.058075E-3,6.763314E7,-4.1354834E-3,-1.5883277E-3,3.11626E8,-4.5682453E-3,3.6522612E6,3.1907817E-3,3.741267E7,-3.237498E-4,9E0,2.5503825E6,-2.5711127E-3,1.4145382E-3,9.7199225E-1,-3.59765E-3,4.8511443E-3,1.7E1,1.9E1,3.8396227E0,-3.552426E-3,-1.2119684E-3,1.5243745E-4,2.5778625E-3,4.0869565E0,2.0625E0,3.5207032E-3,1.8926304E-4,2.8E1,1.7391304E0,5.556659E6,2.308943E0,-1.00886034E-4,3.6870416E-3,3.405424E-3,-3.9663675E-4,6.8591866E-3,2.225105E-3,2.732002E7,2.884462E0,5.812E3,2.1818182E0,1.0901037E10,7.919006E7,4.5102823E-4,4.032157E-3,-3.6705396E-4,-2.844864E-3,1.5365228E5,2.5437157E2,9.818433E-4,3.088104E5,2.49E2,1.8071781E-3,-8.0444245E-4,-4.400291E-3,1.225E0,3.5589743E0,3.53E2,5.036784E6,3.9751194E-3,-1.2712923E-3,9.8933E4,9E0,3.9076055E-3,1.0130841E1,2.3266666E0,1.7022608E7,3.0588236E0,1.4872362E-1,3.9136843E-3,-2.7331631E-5,-2.6043411E-3,2.8058012E-6,-5.1617913E-5,-1.8449372E-3,4.8178105E-4,-2.8277594E-3,6.156484E-4,2.9867042E-3,-2.7970145E-3,9.3909475E-6,2.2863483E-3,-2.1491125E-3,-3.2629122E-4,2.5960503E-3,2.5885573E-4,3.5710714E-3,-1.2076759E-4,-3.81259E-3,-1.8508105E-3,2.5246267E-3,-2.178779E-3,8.830343E-4,3.728812E-3,-1.2060668E-4,-6.442816E-4,-3.8868415E-3,2.7928483E-3,-7.7613455E-4],"split_indices":[2,68,60,5,32,9,48,43,1,0,7,5,8,0,48,43,9,8,4,0,0,60,46,9,68,72,0,68,3,3,9,0,0,68,0,8,9,73,69,73,71,1,73,4,0,0,0,0,0,3,12,73,60,0,0,0,0,4,8,71,0,0,0,7,0,0,0,2,48,2,0,71,0,67,71,0,7,0,0,46,0,60,0,7,0,8,47,0,0,53,0,0,8,8,69,0,0,0,0,73,71,0,0,10,71,62,69,0,0,0,0,0,0,5,71,44,69,46,7,0,0,0,0,43,4,0,60,10,0,0,0,68,69,44,9,0,0,1,8,0,71,68,9,68,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.019E3,3.91E2,6.28E2,3.75E2,1.6E1,5.69E2,5.9E1,3.49E2,2.6E1,5E0,1.1E1,2.08E2,3.61E2,4E0,5.5E1,5.3E1,2.96E2,8E0,1.8E1,4E0,7E0,1.63E2,4.5E1,3.5E1,3.26E2,4.9E1,6E0,1.8E1,3.5E1,1.45E2,1.51E2,4E0,4E0,1.3E1,5E0,3.4E1,1.29E2,3.4E1,1.1E1,1.5E1,2E1,1.52E2,1.74E2,3.5E1,1.4E1,1.2E1,6E0,4E0,3.1E1,1.22E2,2.3E1,2.6E1,1.25E2,5E0,8E0,3E1,4E0,9.5E1,3.4E1,2.7E1,7E0,7E0,4E0,9E0,6E0,1.5E1,5E0,6.2E1,9E1,1.7E2,4E0,3.1E1,4E0,3.7E1,8.5E1,6E0,1.7E1,1.4E1,1.2E1,1.21E2,4E0,8.5E1,1E1,2.5E1,9E0,1.4E1,1.3E1,4E0,5E0,5.8E1,4E0,1.4E1,7.6E1,1.56E2,1.4E1,8E0,2.3E1,3E1,7E0,3.5E1,5E1,1.3E1,4E0,8E1,4.1E1,2.5E1,6E1,5E0,2E1,1E1,4E0,9E0,4E0,4.1E1,1.7E1,5E1,2.6E1,6.6E1,9E1,4E0,1E1,1.7E1,1.8E1,2E1,3E1,1.7E1,6.3E1,1.8E1,2.3E1,2E1,5E0,3E1,3E1,2.4E1,1.7E1,1.2E1,5E0,2.6E1,2.4E1,1.6E1,1E1,5.1E1,1.5E1,2.2E1,6.8E1,5E0,1.5E1,1E1,2E1,1.6E1,4.7E1,1.3E1,5E0,2.1E1,9E0,8E0,2.2E1,5E0,1.9E1,7E0,1E1,1E1,1.6E1,1.7E1,7E0,4E0,6E0,1.5E1,3.6E1,1.1E1,4E0,9E0,1.3E1,8E0,6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"175","size_leaf_vector":"1"}},{"base_weights":[1.1045056E-4,-1.2815726E-2,1.8037228E-2,-1.0047119E-2,-9.884234E-2,2.6030619E-2,-2.7895272E-2,-1.3516349E-2,2.8885858E-2,1.9730919E-3,-1.849727E-1,5.351257E-3,2.3015238E-2,-3.895394E-2,2.6402446E-3,-2.3300089E-2,3.4412355E-3,-1.0403417E-2,5.4115847E-2,-1.2781561E-2,-2.1793044E-3,8.731328E-2,1.904624E-2,-5.320839E-2,1.5577832E-2,-8.889742E-3,-2.0766817E-2,8.538131E-3,-6.032947E-2,-1.9747985E-3,2.0074777E-3,1.0895453E-3,3.8380194E-3,1.1194588E-1,-6.421764E-4,1.0966069E-2,3.8813796E-2,-6.912194E-2,-6.271259E-3,4.667697E-3,-1.5536258E-3,-4.356134E-3,-1.850333E-2,-3.2385606E-3,1.1985731E-2,-8.4086595E-4,-6.511587E-3,2.1493388E-4,7.2192405E-3,1.346729E-2,-5.2106287E-3,5.8112144E-3,2.9883713E-2,-5.0095234E-2,-8.004593E-3,1.1779047E-3,-2.8735166E-3,1.4696375E-3,-2.8318046E-2,1.5294258E-2,-3.3248083E-3,5.098607E-4,3.7546866E-2,-3.442261E-2,3.904164E-2,-1.0026759E-2,-6.8698E-2,-3.5556927E-3,2.7173567E-3,-1.8700685E-2,-4.8779387E-2,-4.7632554E-3,2.4359833E-2,9.096348E-3,-1.4360896E-1,4.2818524E-2,-2.557734E-3,8.6203177E-4,-2.9242078E-3,-1.238015E-3,4.5631785E-2,-2.2741118E-3,1.4248629E-3,-5.178553E-3,-2.0266369E-3,7.542003E-3,-3.213402E-2,-8.339582E-3,-5.7941783E-2,-6.421848E-2,-6.1987992E-3,9.651417E-3,-5.2076153E-2,2.788856E-2,-2.1867906E-3,-4.310182E-2,1.693625E-2,-1.04953805E-2,-1.4468948E-3,3.3015613E-2,4.7215084E-3,2.8121676E-2,6.901285E-2,1.2956869E-3,-6.594403E-4,-7.989502E-4,-3.979578E-3,-8.8121585E-4,1.4106815E-3,-5.0460147E-3,7.296363E-4,-8.736107E-4,-3.9585796E-3,-2.6647928E-3,4.7477442E-4,8.5820846E-4,-2.8594034E-3,-3.5427369E-3,1.21667406E-4,1.0155893E-3,2.461328E-3,-3.6172033E-4,-3.9427355E-3,1.8423343E-4,1.8254595E-3,2.1788385E-3,1.546392E-4,4.2802566E-3,8.0725714E-4,3.6999774E-3,-2.1224305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,27,29,31,-1,-1,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,47,-1,49,51,53,55,-1,-1,-1,57,-1,59,-1,-1,-1,-1,61,-1,-1,63,65,-1,-1,-1,67,69,71,-1,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,-1,-1,-1,-1,101,-1,-1,-1,-1,103,105,107,109,111,113,115,117,119,-1,121,123,-1,-1,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.44469E-1,1.4500752E-1,1.6300927E-1,8.075556E-2,2.3954457E-1,9.941837E-2,6.4297885E-2,9.0832196E-2,4.9135122E-2,0E0,1.3856879E-1,0E0,9.173985E-2,4.6690777E-2,0E0,1.4467044E-1,6.584665E-2,3.6693778E-2,2.456697E-2,0E0,0E0,5.3912237E-2,5.4839283E-2,3.467253E-2,5.8063302E-2,0E0,5.533734E-2,5.1538415E-2,4.8378706E-2,0E0,0E0,0E0,0E0,8.010849E-2,0E0,7.797346E-2,7.390836E-2,6.0671598E-2,2.4682611E-2,0E0,0E0,0E0,6.55153E-2,0E0,5.1097475E-2,0E0,0E0,0E0,0E0,7.579915E-2,0E0,0E0,5.5523872E-2,2.2418097E-2,0E0,0E0,0E0,3.219898E-2,4.3167427E-2,3.1754952E-2,0E0,1.9806474E-1,4.400278E-2,1.9572716E-2,3.6367826E-2,1.9139552E-2,1.7710283E-2,3.2909174E-2,0E0,6.213027E-2,4.6277687E-2,3.8012605E-2,3.16787E-2,6.252349E-2,7.734038E-2,4.3776244E-2,0E0,0E0,0E0,0E0,2.8355211E-2,0E0,0E0,0E0,0E0,3.3948395E-2,2.1752847E-2,5.1911224E-2,1.2272502E-1,4.895675E-2,1.7556578E-2,2.860646E-2,1.7703831E-2,1.6071111E-2,0E0,2.9130146E-2,3.8430322E-2,0E0,0E0,2.9640816E-2,0E0,2.8613165E-2,2.5638789E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,28,28,33,33,35,35,36,36,37,37,38,38,42,42,44,44,49,49,52,52,53,53,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,75,75,80,80,85,85,86,86,87,87,88,88,89,89,90,90,91,91,92,92,93,93,95,95,96,96,99,99,101,101,102,102],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,28,30,32,-1,-1,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,48,-1,50,52,54,56,-1,-1,-1,58,-1,60,-1,-1,-1,-1,62,-1,-1,64,66,-1,-1,-1,68,70,72,-1,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,-1,-1,-1,-1,102,-1,-1,-1,-1,104,106,108,110,112,114,116,118,120,-1,122,124,-1,-1,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,6.0692043E0,2.856934E7,4.5866325E6,1.901875E2,1.2E1,4.435876E2,1.380863E6,1E0,1.9730919E-3,2.6235E4,5.351257E-3,1.0918E4,2.3308511E5,2.6402446E-3,4E0,1.2E1,1E0,1.81E2,-1.2781561E-2,-2.1793044E-3,1.4791E4,9.033567E6,2.0618556E-2,1E0,-8.889742E-3,6E0,3.7788504E7,1.3994015E7,-1.9747985E-3,2.0074777E-3,1.0895453E-3,3.8380194E-3,3.7509E4,-6.421764E-4,6.5026445E3,5E0,4.9E1,3.463702E2,4.667697E-3,-1.5536258E-3,-4.356134E-3,1.4615384E0,-3.2385606E-3,4.0903897E2,-8.4086595E-4,-6.511587E-3,2.1493388E-4,7.2192405E-3,1.7068776E10,-5.2106287E-3,5.8112144E-3,1.2895454E3,6.5E2,-8.004593E-3,1.1779047E-3,-2.8735166E-3,1.3E1,3.14403E5,1.0140845E0,-3.3248083E-3,2.1373269E8,2.5842668E7,1.194E3,1E1,5.17259E8,1.33E2,1E0,2.7173567E-3,1.65E2,9.5E2,1.2751362E8,5.123862E1,1E0,4.626E3,1.7743401E6,-2.557734E-3,8.6203177E-4,-2.9242078E-3,-1.238015E-3,1.9001543E1,-2.2741118E-3,1.4248629E-3,-5.178553E-3,-2.0266369E-3,3.883E4,8E0,5.5E1,2.2084616E2,3.56E2,2.8266037E2,1.8E1,1.3E1,7.884708E0,-2.1867906E-3,1.2585366E1,7.445455E2,-1.04953805E-2,-1.4468948E-3,4.076E4,4.7215084E-3,9.5596474E-1,2.7561485E6,1.2956869E-3,-6.594403E-4,-7.989502E-4,-3.979578E-3,-8.8121585E-4,1.4106815E-3,-5.0460147E-3,7.296363E-4,-8.736107E-4,-3.9585796E-3,-2.6647928E-3,4.7477442E-4,8.5820846E-4,-2.8594034E-3,-3.5427369E-3,1.21667406E-4,1.0155893E-3,2.461328E-3,-3.6172033E-4,-3.9427355E-3,1.8423343E-4,1.8254595E-3,2.1788385E-3,1.546392E-4,4.2802566E-3,8.0725714E-4,3.6999774E-3,-2.1224305E-4],"split_indices":[4,68,60,43,4,33,71,9,29,0,1,0,9,48,0,3,71,16,0,0,0,2,60,72,15,0,67,7,5,0,0,0,0,1,0,4,3,3,71,0,0,0,68,0,67,0,0,0,0,46,0,0,48,10,0,0,0,8,9,71,0,5,62,2,3,46,0,8,0,0,10,7,73,29,2,58,0,0,0,0,71,0,0,0,0,9,32,0,67,2,4,3,8,71,0,71,67,0,0,44,0,68,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.053E3,6.12E2,4.41E2,5.94E2,1.8E1,3.76E2,6.5E1,5.46E2,4.8E1,7E0,1.1E1,1.1E1,3.65E2,5.8E1,7E0,3.46E2,2E2,1.9E1,2.9E1,6E0,5E0,2E1,3.45E2,4.6E1,1.2E1,4E0,3.42E2,1.86E2,1.4E1,1.2E1,7E0,1.5E1,1.4E1,1.6E1,4E0,2.46E2,9.9E1,3.4E1,1.2E1,4E0,8E0,9E0,3.33E2,7E0,1.79E2,1E1,4E0,5E0,1.1E1,2.42E2,4E0,8E0,9.1E1,3E1,4E0,8E0,4E0,1.1E2,2.23E2,1.73E2,6E0,1.58E2,8.4E1,1.1E1,8E1,1E1,2E1,1.02E2,8E0,1.53E2,7E1,5.4E1,1.19E2,1.5E2,8E0,8E1,4E0,4E0,7E0,7E0,7.3E1,5E0,5E0,6E0,1.4E1,7.4E1,2.8E1,1.22E2,3.1E1,5.1E1,1.9E1,4.2E1,1.2E1,1.14E2,5E0,1.9E1,1.31E2,4E0,4E0,7E1,1E1,4.3E1,3E1,3.8E1,3.6E1,2.3E1,5E0,9.6E1,2.6E1,1.8E1,1.3E1,1.7E1,3.4E1,4E0,1.5E1,3.8E1,4E0,8E0,4E0,9.4E1,2E1,1.1E1,8E0,8.4E1,4.7E1,4.7E1,2.3E1,5E0,3.8E1,2.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[3.93045E-4,-1.3073961E-2,1.5106187E-2,6.039252E-2,-1.6067546E-2,1.2840821E-2,4.183979E-3,-1.3171644E-3,4.937781E-3,-1.2578861E-2,-6.315427E-2,1.4488584E-2,-3.915205E-3,-1.6600374E-2,2.839642E-2,-9.160539E-3,-1.0814054E-1,-2.6114536E-3,2.2298638E-2,2.7369767E-2,-2.0396769E-2,-5.7654246E-3,4.7308274E-2,2.9032074E-3,-3.614889E-2,-1.28113E-2,-4.454249E-2,9.089443E-3,-2.072299E-2,4.0063714E-3,1.9067867E-2,6.4171925E-2,-5.060151E-4,-1.4067255E-2,-5.613506E-2,5.745423E-2,-3.3637436E-4,5.193359E-4,-3.1033242E-3,-9.091091E-2,3.0665323E-3,-1.3860628E-2,3.2347526E-2,-2.3520426E-3,5.395782E-3,1.8976177E-3,3.028788E-2,1.9093285E-3,5.326736E-3,-5.4896926E-3,-1.1842894E-2,-1.08501256E-1,-6.9239717E-3,3.7535783E-3,1.9359307E-2,-6.4186123E-4,-7.0581343E-3,2.6896414E-3,-2.4070736E-2,3.5155264E-3,1.4133363E-2,2.9097486E-3,-7.587823E-4,7.510281E-3,-6.1715238E-2,4.9648867E-3,2.5730597E-2,-1.9521604E-3,-2.673824E-2,-1.3320321E-1,-5.468816E-4,-3.5595693E-2,3.7718985E-2,-1.2059099E-3,2.620339E-3,-3.5133367E-4,-3.1547733E-3,3.8654448E-3,-1.2719702E-3,-2.8051767E-3,1.2588836E-2,-1.2615975E-4,-5.550979E-3,5.3099122E-2,1.11167105E-2,-2.171132E-2,1.1895519E-2,-3.152733E-2,2.457812E-2,-1.7551357E-1,-4.7885578E-2,-2.3510227E-4,-3.729177E-3,5.113411E-3,-7.007122E-3,-1.6715389E-2,3.377651E-3,3.6446556E-2,-1.2793975E-3,6.6027895E-2,1.8545505E-2,3.801265E-2,-1.643462E-2,-6.074711E-4,-5.2675437E-3,8.450591E-4,-1.9476142E-3,-4.3062828E-4,-1.945756E-3,-1.6807514E-3,3.387191E-3,-1.25386E-2,-5.3577637E-3,-3.9051617E-3,-7.6920966E-5,1.7336402E-3,-2.3163406E-3,5.540255E-4,-2.39E-3,8.3220925E-4,4.0701777E-3,-1.7803643E-3,8.6021045E-4,4.569807E-3,1.8227851E-3,-1.7364054E-3,1.7217251E-3,2.5233924E-3,-2.8783127E-4,-6.54822E-3,1.6369525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,-1,19,21,23,25,27,29,31,33,-1,35,-1,37,-1,39,41,43,-1,45,47,-1,49,51,53,-1,-1,-1,55,-1,57,59,-1,61,63,65,-1,-1,-1,67,69,71,-1,73,-1,-1,-1,75,-1,77,-1,-1,79,81,-1,83,85,87,89,-1,91,93,-1,-1,-1,-1,-1,95,-1,97,-1,-1,99,101,103,105,107,109,111,113,-1,-1,-1,115,117,-1,119,121,123,125,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0150974E-1,1.1775222E-1,8.191388E-2,8.983888E-2,8.314797E-2,7.728656E-2,0E0,0E0,0E0,7.9072714E-2,8.402355E-2,6.2293567E-2,0E0,7.3171936E-2,1.3048084E-1,3.5306223E-2,1.9206207E-1,3.134078E-2,6.494801E-2,5.0065406E-2,8.9918986E-2,0E0,2.2018872E-2,0E0,1.9868134E-2,0E0,8.3181426E-2,4.848808E-2,4.6006266E-2,0E0,5.889356E-2,1.6257837E-2,0E0,7.828672E-2,1.5386857E-1,2.8338954E-2,0E0,0E0,0E0,5.028961E-2,0E0,3.5254423E-2,3.4781747E-2,0E0,4.1244015E-2,4.402208E-2,6.2304556E-2,0E0,0E0,0E0,4.9449306E-2,6.7668915E-2,4.230233E-2,0E0,2.5258288E-2,0E0,0E0,0E0,2.9330969E-2,0E0,3.5456855E-2,0E0,0E0,3.991979E-2,3.4141395E-2,0E0,6.980276E-2,5.580282E-2,3.3706546E-2,7.462129E-2,0E0,2.7121047E-2,4.3199234E-2,0E0,0E0,0E0,0E0,0E0,3.3386372E-2,0E0,3.511826E-2,0E0,0E0,2.6026368E-2,8.6582914E-2,6.48913E-2,4.2258702E-2,2.853109E-2,3.832032E-2,5.4167837E-2,1.5102407E-2,0E0,0E0,0E0,1.9168131E-2,2.4746615E-2,0E0,3.5546616E-2,5.1131885E-2,3.4549788E-2,2.0316612E-2,4.3690994E-2,1.4569485E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,26,26,27,27,28,28,30,30,31,31,33,33,34,34,35,35,39,39,41,41,42,42,44,44,45,45,46,46,50,50,51,51,52,52,54,54,58,58,60,60,63,63,64,64,66,66,67,67,68,68,69,69,71,71,72,72,78,78,80,80,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,94,94,95,95,97,97,98,98,99,99,100,100,101,101,102,102],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,-1,20,22,24,26,28,30,32,34,-1,36,-1,38,-1,40,42,44,-1,46,48,-1,50,52,54,-1,-1,-1,56,-1,58,60,-1,62,64,66,-1,-1,-1,68,70,72,-1,74,-1,-1,-1,76,-1,78,-1,-1,80,82,-1,84,86,88,90,-1,92,94,-1,-1,-1,-1,-1,96,-1,98,-1,-1,100,102,104,106,108,110,112,114,-1,-1,-1,116,118,-1,120,122,124,126,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.969174E7,7E0,9.059046E6,3.5443038E-1,6.0692043E0,1E0,4.183979E-3,-1.3171644E-3,4.937781E-3,2.008E3,1.2999985E6,3.808001E5,-3.915205E-3,7.3333335E-1,2.1126761E0,5.004138E-3,1.088E3,3.38E2,1.0164831E-2,4.31903E6,6.648E3,-5.7654246E-3,4.5620965E6,2.9032074E-3,1.1634076E4,-1.28113E-2,1.764442E7,4.062891E8,1.19E2,4.0063714E-3,8.1407714E-1,6.347708E7,-5.060151E-4,6E0,7.714286E0,3.015873E0,-3.3637436E-4,5.193359E-4,-3.1033242E-3,1.8471494E7,3.0665323E-3,7.807738E6,1.3209776E6,-2.3520426E-3,5.49E2,1.2769117E7,8.998703E-1,1.9093285E-3,5.326736E-3,-5.4896926E-3,1E0,2.378914E6,1.1362371E6,3.7535783E-3,5.6115704E0,-6.4186123E-4,-7.0581343E-3,2.6896414E-3,1.4195632E6,3.5155264E-3,3.9961785E-2,2.9097486E-3,-7.587823E-4,2.87834E5,7.3835544E5,4.9648867E-3,5.8606E4,4.1277572E5,2.3E1,8.39E2,-5.468816E-4,1.34214E5,7.82E2,-1.2059099E-3,2.620339E-3,-3.5133367E-4,-3.1547733E-3,3.8654448E-3,5.467E3,-2.8051767E-3,1.4327235E7,-1.2615975E-4,-5.550979E-3,1.016E4,1.5153E4,2.923526E5,1.3177083E1,1.7142857E0,7E0,2E0,5.095006E5,-2.3510227E-4,-3.729177E-3,5.113411E-3,1.3152658E7,1.393875E5,3.377651E-3,6.819212E6,2.194E3,2.1298597E0,1.3388E4,1.1E1,5.668E3,-6.074711E-4,-5.2675437E-3,8.450591E-4,-1.9476142E-3,-4.3062828E-4,-1.945756E-3,-1.6807514E-3,3.387191E-3,-1.25386E-2,-5.3577637E-3,-3.9051617E-3,-7.6920966E-5,1.7336402E-3,-2.3163406E-3,5.540255E-4,-2.39E-3,8.3220925E-4,4.0701777E-3,-1.7803643E-3,8.6021045E-4,4.569807E-3,1.8227851E-3,-1.7364054E-3,1.7217251E-3,2.5233924E-3,-2.8783127E-4,-6.54822E-3,1.6369525E-4],"split_indices":[7,3,43,71,68,79,0,0,0,2,60,43,0,68,69,53,44,0,53,60,44,0,62,0,43,0,60,46,8,0,53,7,0,67,71,68,0,0,0,5,0,46,60,0,0,62,57,0,0,0,26,9,43,0,69,0,0,0,60,0,53,0,0,12,43,0,9,60,3,2,0,1,2,0,0,0,0,0,2,0,12,0,0,2,10,60,71,68,8,32,43,0,0,0,60,48,0,12,2,53,2,8,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.015E3,5.3E2,4.85E2,2E1,5.1E2,4.72E2,1.3E1,7E0,1.3E1,4.76E2,3.4E1,4.65E2,7E0,4.34E2,4.2E1,1.6E1,1.8E1,1.46E2,3.19E2,3.4E1,4E2,4E0,3.8E1,4E0,1.2E1,4E0,1.4E1,8.9E1,5.7E1,1.4E1,3.05E2,1.7E1,1.7E1,3.41E2,5.9E1,3.2E1,6E0,5E0,7E0,1E1,4E0,4.5E1,4.4E1,2.6E1,3.1E1,1.21E2,1.84E2,1.3E1,4E0,6E0,3.35E2,2.8E1,3.1E1,1.9E1,1.3E1,5E0,5E0,5E0,4E1,1.2E1,3.2E1,8E0,2.3E1,1.12E2,9E0,9E0,1.75E2,2.02E2,1.33E2,2.2E1,6E0,1.9E1,1.2E1,6E0,7E0,3E1,1E1,5E0,2.7E1,7E0,1.05E2,5E0,4E0,6E1,1.15E2,8.3E1,1.19E2,1.22E2,1.1E1,1.4E1,8E0,1.2E1,7E0,4E0,8E0,2.3E1,4E0,3.8E1,6.7E1,4.3E1,1.7E1,5.8E1,5.7E1,7.7E1,6E0,1.07E2,1.2E1,4E1,8.2E1,5E0,6E0,4E0,1E1,4E0,4E0,4E0,4E0,1.3E1,1E1,2.9E1,9E0,2.3E1,4.4E1,1.8E1,2.5E1,4E0,1.3E1,4.2E1,1.6E1,7E0,5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-1.554028E-3,-1.8304912E-2,1.6564656E-2,-1.5394103E-2,-1.174105E-1,4.5295143E-3,1.3435578E-2,-1.7329253E-2,2.893446E-2,7.616974E-4,-2.0850763E-1,-2.7874842E-2,1.9149173E-2,-1.4193781E-2,-5.370661E-2,6.4994946E-2,-9.626782E-4,-1.3356135E-2,-3.9830017E-3,-2.8058556E-3,-1.0309225E-2,4.101179E-3,1.6330482E-2,-2.546953E-2,-5.6612757E-3,-5.8101793E-3,-3.6794227E-2,1.7421166E-4,4.7487365E-3,6.8761827E-3,-2.5261184E-3,1.7881889E-2,-3.4079968E-3,-2.1883378E-2,-9.79488E-2,4.765879E-3,-2.6690217E-2,1.0881517E-3,-6.842333E-2,2.1666206E-2,-2.4684125E-3,1.5434419E-2,1.02140196E-1,-1.8405905E-2,-2.817911E-3,-6.939745E-3,-1.2055098E-3,9.4584463E-4,3.9208382E-3,-8.255553E-2,-9.86674E-3,-1.238414E-3,2.4044833E-3,-4.386652E-3,-6.0855004E-4,-5.4762646E-4,5.094096E-2,1.0483627E-2,4.634253E-2,7.2312662E-3,2.2500358E-3,-4.120051E-2,-9.466208E-3,5.538158E-3,-5.959844E-2,-6.128115E-4,-1.1181261E-2,-3.3496052E-2,3.211552E-2,3.636293E-3,2.5069484E-4,1.4981445E-2,-2.6025381E-2,2.7629407E-2,3.5036304E-3,-2.5828788E-2,-8.644425E-2,3.2633454E-2,-2.5208378E-2,2.6760949E-2,-1.4966975E-2,-8.099305E-4,-5.3122505E-3,-4.056157E-3,-2.0849917E-2,5.952784E-3,7.554004E-3,1.8604428E-2,-1.9266376E-2,3.207527E-3,-3.969943E-2,5.221785E-2,-1.4170104E-6,-1.921377E-3,1.0375849E-3,-6.2345397E-3,-2.0938758E-3,2.3278622E-3,-2.1058281E-3,-1.5122887E-4,-1.7460182E-3,1.591008E-3,-2.065975E-3,-1.4087139E-3,5.7482865E-4,1.2719618E-3,-1.5271936E-3,-1.1451181E-3,2.4683645E-3,-6.922327E-5,1.1849086E-3,3.1147618E-4,-2.618261E-3,3.176464E-4,-3.248578E-3,3.0440209E-4,5.0093806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,-1,29,-1,31,33,35,-1,37,-1,-1,39,-1,41,-1,43,45,47,49,51,53,55,-1,57,59,61,-1,-1,-1,63,-1,65,67,-1,-1,-1,-1,-1,69,71,73,-1,-1,75,77,79,81,-1,-1,83,85,-1,-1,87,89,91,-1,93,95,97,99,101,103,-1,-1,-1,105,-1,107,109,111,-1,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.101808E-1,1.5137413E-1,1.2386489E-1,4.48509E-2,1.864033E-1,0E0,1.12412475E-1,5.5811554E-2,4.084006E-2,0E0,6.377882E-2,3.285103E-2,8.038448E-2,4.392018E-2,4.2900458E-2,2.9692862E-2,0E0,0E0,0E0,0E0,3.0113984E-2,0E0,5.7510197E-2,4.906112E-2,5.757647E-2,0E0,3.961458E-2,0E0,0E0,2.7023563E-2,0E0,8.0086514E-2,0E0,2.4583153E-2,2.9879995E-2,5.3774714E-2,8.09523E-2,2.413808E-2,2.5131643E-2,2.4154678E-2,0E0,5.8579735E-2,2.4344124E-2,3.5270933E-2,0E0,0E0,0E0,4.731854E-2,0E0,2.1778351E-1,6.853256E-2,0E0,0E0,0E0,0E0,0E0,1.600309E-2,5.523633E-2,2.7157843E-2,0E0,0E0,3.1845495E-2,8.499063E-2,6.915836E-2,2.5944151E-2,0E0,0E0,2.8384332E-2,5.8425784E-2,0E0,0E0,3.7334062E-2,5.1050145E-2,2.312562E-2,0E0,2.9853227E-2,1.7818898E-2,5.0899398E-2,2.5549069E-2,4.434356E-2,3.484807E-2,0E0,0E0,0E0,2.2762176E-2,0E0,3.24804E-2,3.826867E-2,2.8596187E-2,0E0,4.7541924E-2,4.5454074E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,20,20,22,22,23,23,24,24,26,26,29,29,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,47,47,49,49,50,50,56,56,57,57,58,58,61,61,62,62,63,63,64,64,67,67,68,68,71,71,72,72,73,73,75,75,76,76,77,77,78,78,79,79,80,80,84,84,86,86,87,87,88,88,90,90,91,91],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,-1,30,-1,32,34,36,-1,38,-1,-1,40,-1,42,-1,44,46,48,50,52,54,56,-1,58,60,62,-1,-1,-1,64,-1,66,68,-1,-1,-1,-1,-1,70,72,74,-1,-1,76,78,80,82,-1,-1,84,86,-1,-1,88,90,92,-1,94,96,98,100,102,104,-1,-1,-1,106,-1,108,110,112,-1,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.194E3,6.27957E0,1.2E1,1.559287E5,9E0,4.5295143E-3,1.8439855E5,1.0700264E6,4E0,7.616974E-4,6.6E1,1.3631483E8,6.923077E-1,5.02E2,2.3238889E2,5.4674416E7,-9.626782E-4,-1.3356135E-2,-3.9830017E-3,-2.8058556E-3,5.16175E5,4.101179E-3,5.619497E7,3.578882E1,4.064E3,-5.8101793E-3,8.39E2,1.7421166E-4,4.7487365E-3,4.7220547E1,-2.5261184E-3,4.435876E2,-3.4079968E-3,2.3364584E1,3.3817584E7,3.653E3,5.1942E4,1.7171982E6,3.8192307E2,1.896E3,-2.4684125E-3,4.9321495E6,1.213274E9,3.89E2,-2.817911E-3,-6.939745E-3,-1.2055098E-3,1.683263E7,3.9208382E-3,1.7431098E8,1.925508E6,-1.238414E-3,2.4044833E-3,-4.386652E-3,-6.0855004E-4,-5.4762646E-4,2.0734E4,2.3E1,1.3E1,7.2312662E-3,2.2500358E-3,1.9E1,1.0443479E5,1E0,1E0,-6.128115E-4,-1.1181261E-2,6.15E2,2.331083E6,3.636293E-3,2.5069484E-4,3.7568388E0,1.592E3,1.3784861E5,3.5036304E-3,1.00037E5,7.336111E4,2.952381E0,3.2E1,1E0,1E0,-8.099305E-4,-5.3122505E-3,-4.056157E-3,7.89E2,5.952784E-3,2.2715E4,1.1772152E0,1.0743623E0,3.207527E-3,5.7420593E2,6.203939E2,-1.4170104E-6,-1.921377E-3,1.0375849E-3,-6.2345397E-3,-2.0938758E-3,2.3278622E-3,-2.1058281E-3,-1.5122887E-4,-1.7460182E-3,1.591008E-3,-2.065975E-3,-1.4087139E-3,5.7482865E-4,1.2719618E-3,-1.5271936E-3,-1.1451181E-3,2.4683645E-3,-6.922327E-5,1.1849086E-3,3.1147618E-4,-2.618261E-3,3.176464E-4,-3.248578E-3,3.0440209E-4,5.0093806E-3],"split_indices":[2,68,33,48,0,0,43,66,8,0,10,7,68,2,67,7,0,0,0,0,1,0,62,61,44,0,2,0,0,73,0,71,0,73,7,44,1,60,67,10,0,43,7,1,0,0,0,60,0,5,43,0,0,0,0,0,44,3,3,0,0,67,43,26,8,0,0,2,43,0,0,68,44,48,0,5,60,68,10,23,27,0,0,0,2,0,44,68,53,0,67,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E3,5.3E2,4.9E2,5.16E2,1.4E1,1.7E1,4.73E2,4.95E2,2.1E1,6E0,8E0,5.7E1,4.16E2,4.57E2,3.8E1,1.2E1,9E0,4E0,4E0,1.9E1,3.8E1,1.5E1,4.01E2,1.96E2,2.61E2,6E0,3.2E1,5E0,7E0,2.8E1,1E1,3.95E2,6E0,1.88E2,8E0,1.75E2,8.6E1,1.5E1,1.7E1,2.3E1,5E0,3.85E2,1E1,1.74E2,1.4E1,4E0,4E0,1.68E2,7E0,1.9E1,6.7E1,1E1,5E0,1.1E1,6E0,1.1E1,1.2E1,3.33E2,5.2E1,4E0,6E0,4.8E1,1.26E2,1.57E2,1.1E1,1.4E1,5E0,4.3E1,2.4E1,7E0,5E0,2.97E2,3.6E1,3.3E1,1.9E1,3.7E1,1.1E1,3.4E1,9.2E1,7.7E1,8E1,7E0,4E0,7E0,3.6E1,4E0,2E1,2.69E2,2.8E1,4E0,3.2E1,1.7E1,1.6E1,2.8E1,9E0,4E0,7E0,2.8E1,6E0,3.4E1,5.8E1,7E1,7E0,5.1E1,2.9E1,7E0,2.9E1,1.2E1,8E0,7E1,1.99E2,1.7E1,1.1E1,1.3E1,1.9E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"117","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics index 14ec21817..39973212f 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics @@ -56,54 +56,61 @@ 54,sqlOp_BroadcastExchange,9.207968e-05,0.897503285151117,0.3034000972146485,0.0,1.0,1.0,1.0,1.0 55,sqlOp_Exchange,7.649039e-05,0.9579500657030223,0.20076907391845303,0.0,1.0,1.0,1.0,1.0 56,sqlOp_BroadcastNestedLoopJoin,6.7585526e-05,0.03942181340341656,0.19466029547643685,0.0,0.0,0.0,0.0,1.0 -57,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_Execute InsertIntoHadoopFsRelationCommand,0.0,0.13009198423127463,0.3365151758705616,0.0,0.0,0.0,0.0,1.0 -62,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan JDBCRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_MapElements,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 -65,sqlOp_Scan csv ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_SerializeFromObject,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 -69,sqlOp_Scan text ,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 -70,sqlOp_Scan orc ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan json ,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,sr_remoteBytesReadRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,sr_remoteBlocksFetched_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,sqlOp_LocalTableScan,0.0,0.0019710906701708277,0.044367767973722515,0.0,0.0,0.0,0.0,1.0 -86,sqlOp_Scan parquet ,0.0,0.9500657030223391,0.21788082416070467,0.0,1.0,1.0,1.0,1.0 -87,maxMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 -88,maxOnHeapMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 -89,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,executorMemory,0.0,16262.896189224704,1358.9340452651124,1024.0,16384.0,16384.0,16384.0,16384.0 -91,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -94,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -95,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -98,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sr_totalBytesReadRatio,0.0,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 -100,sr_remoteBytesRead_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sr_totalBytesRead_mean,0.0,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 -103,platform_onprem,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -104,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,maxOnHeapMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 +58,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,sqlOp_MapElements,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 +60,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +61,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_SerializeFromObject,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 +73,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sr_totalBytesReadRatio,0.0,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 +88,executorMemory,0.0,16262.896189224704,1358.9340452651124,1024.0,16384.0,16384.0,16384.0,16384.0 +89,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +92,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,maxMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 +95,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +97,sqlOp_Scan parquet,0.0,0.9500657030223391,0.21788082416070467,0.0,1.0,1.0,1.0,1.0 +98,sr_remoteBytesReadRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_LocalTableScan,0.0,0.0019710906701708277,0.044367767973722515,0.0,0.0,0.0,0.0,1.0 +101,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sr_remoteBytesRead_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sr_totalBytesRead_mean,0.0,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 +105,sr_remoteBlocksFetched_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,platform_onprem,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +107,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 From ab4cf6abb7125fd6a31b7621ba96d7bb267c6697 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 5 Dec 2024 17:08:57 -0600 Subject: [PATCH 13/52] Count expressions per Exec in SQLPlanParser (#1449) * Count expressions per Exec in SQLPlanParser Signed-off-by: Ahmed Hussein (amahussein) Fixes #1447 Improves the operators-stats by counting the occurence of each expression within the Exec node. * ckeanup overriding methods Signed-off-by: Ahmed Hussein (amahussein) --------- Signed-off-by: Ahmed Hussein (amahussein) --- .../tool/planparser/SQLPlanParser.scala | 49 ++++----- .../planparser/WholeStageExecParser.scala | 2 +- .../planparser/WindowGroupLimitParser.scala | 2 +- .../tool/planparser/ops/ExprOpRef.scala | 31 ++++++ .../planparser/ops/OpRefWrapperBase.scala | 50 ++++++++++ .../tool/planparser/ops/OperatorCounter.scala | 61 ++++++------ .../planparser/ops/UnsupportedExprOpRef.scala | 2 +- .../qualification/PluginTypeChecker.scala | 2 +- .../tool/qualification/QualOutputWriter.scala | 44 +++++---- .../tool/planparser/SqlPlanParserSuite.scala | 99 +++++++++++++------ 10 files changed, 232 insertions(+), 110 deletions(-) create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/ExprOpRef.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRefWrapperBase.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index 17159842b..4d9c59dd8 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.{ArrayBuffer, WeakHashMap} import scala.util.control.NonFatal import scala.util.matching.Regex -import com.nvidia.spark.rapids.tool.planparser.ops.{OperatorRefBase, OpRef, UnsupportedExprOpRef} +import com.nvidia.spark.rapids.tool.planparser.ops.{ExprOpRef, OperatorRefTrait, OpRef, UnsupportedExprOpRef} import com.nvidia.spark.rapids.tool.planparser.photon.{PhotonPlanParser, PhotonStageExecParser} import com.nvidia.spark.rapids.tool.qualification.PluginTypeChecker @@ -75,7 +75,7 @@ object UnsupportedReasons extends Enumeration { case class UnsupportedExecSummary( sqlId: Long, execId: Long, - execRef: OperatorRefBase, + execRef: OperatorRefTrait, opType: OpTypes.OpType, reason: UnsupportedReasons.UnsupportedReason, opAction: OpActions.OpAction) { @@ -89,8 +89,6 @@ case class UnsupportedExecSummary( val unsupportedOperatorCSVFormat: String = execRef.getOpNameCSV val details: String = UnsupportedReasons.reportUnsupportedReason(reason) - - def isExpression: Boolean = execRef.isInstanceOf[UnsupportedExprOpRef] } case class ExecInfo( @@ -110,7 +108,7 @@ case class ExecInfo( dataSet: Boolean, udf: Boolean, shouldIgnore: Boolean, - expressions: Seq[OpRef]) { + expressions: Seq[ExprOpRef]) { private def childrenToString = { val str = children.map { c => @@ -141,10 +139,6 @@ case class ExecInfo( stages = stageIDs } - def appendToStages(stageIDs: Set[Int]): Unit = { - stages ++= stageIDs - } - def setShouldRemove(value: Boolean): Unit = { shouldRemove ||= value } @@ -286,7 +280,7 @@ object ExecInfo { udf, shouldIgnore, // convert array of string expressions to OpRefs - expressions = expressions.map(OpRef.fromExpr) + expressions = ExprOpRef.fromRawExprSeq(expressions) ) } @@ -351,7 +345,7 @@ case class PlanInfo( sqlID: Long, sqlDesc: String, execInfo: Seq[ExecInfo]) { - def getUnsupportedExpressions: Seq[OperatorRefBase] = { + def getUnsupportedExpressions: Seq[OperatorRefTrait] = { execInfo.flatMap { e => if (e.isClusterNode) { // wholeStageCodeGen does not have expressions/unsupported-expressions @@ -727,21 +721,21 @@ object SQLPlanParser extends Logging { } private def getAllFunctionNames(regPattern: Regex, expr: String, - groupInd: Int = 1, isAggr: Boolean = true): Set[String] = { + groupInd: Int = 1, isAggr: Boolean = true): Array[String] = { // Returns all matches in an expression. This can be used when the SQL expression is not // tokenized. val newExpr = processSpecialFunctions(expr) // first get all the functionNames val exprss = - regPattern.findAllMatchIn(newExpr).map(_.group(groupInd)).toSet + regPattern.findAllMatchIn(newExpr).map(_.group(groupInd)).toSeq // For aggregate expressions we want to process the results to remove the prefix // DB: remove the "^partial_" and "^finalmerge_" prefixes // TODO: // for performance sake, we can turn off the aggregate processing by enabling it only // when needed. However, for now, we always do this processing until we are confident we know - // the correct place to turn on/off that flag.we can use the argument isAgg only when needed + // the correct place to turn on/off that flag. We can use the argument isAgg only when needed val results = if (isAggr) { exprss.collect { case func => @@ -750,7 +744,7 @@ object SQLPlanParser extends Logging { } else { exprss } - results.filterNot(ignoreExpression(_)) + results.filterNot(ignoreExpression(_)).toArray } def parseProjectExpressions(exprStr: String): Array[String] = { @@ -758,7 +752,7 @@ object SQLPlanParser extends Logging { // This is to split the string such that only function names are extracted. The pattern is // such that function name is succeeded by `(`. We use regex to extract all the function names // below: - getAllFunctionNames(functionPrefixPattern, exprStr).toArray + getAllFunctionNames(functionPrefixPattern, exprStr) } // This parser is used for SortAggregateExec, HashAggregateExec and ObjectHashAggregateExec @@ -792,7 +786,7 @@ object SQLPlanParser extends Logging { } } } - parsedExpressions.distinct.toArray + parsedExpressions.toArray } def parseWindowExpressions(exprStr:String): Array[String] = { @@ -828,7 +822,7 @@ object SQLPlanParser extends Logging { } } } - parsedExpressions.distinct.toArray + parsedExpressions.toArray } def parseWindowGroupLimitExpressions(exprStr: String): Array[String] = { @@ -858,10 +852,10 @@ object SQLPlanParser extends Logging { // - Some values can be NULLs. That's why we cannot limit the extract to the first row. // - Nested brackets/parenthesis makes it challenging to use regex that contains // brackets/parenthesis to extract expressions. - // The implementation Use regex to extract all function names and return distinct set of + // The implementation Use regex to extract all function names and return a list of // function names. // This implementation is 1 line implementation, but it can be a memory/time bottleneck. - getAllFunctionNames(functionPrefixPattern, exprStr).toArray + getAllFunctionNames(functionPrefixPattern, exprStr) } def parseTakeOrderedExpressions(exprStr: String): Array[String] = { @@ -889,7 +883,7 @@ object SQLPlanParser extends Logging { } } } - parsedExpressions.distinct.toArray + parsedExpressions.toArray } def parseGenerateExpressions(exprStr: String): Array[String] = { @@ -897,11 +891,11 @@ object SQLPlanParser extends Logging { // 1. Generate explode(arrays#1306), [id#1304], true, [col#1426] // 2. Generate json_tuple(values#1305, Zipcode, ZipCodeType, City), [id#1304], // false, [c0#1407, c1#1408, c2#1409] - getAllFunctionNames(functionPrefixPattern, exprStr).toArray + getAllFunctionNames(functionPrefixPattern, exprStr) } private def addFunctionNames(exprs: String, parsedExpressions: ArrayBuffer[String]): Unit = { - val functionNames = getAllFunctionNames(functionPrefixPattern, exprs).toArray + val functionNames = getAllFunctionNames(functionPrefixPattern, exprs) functionNames.foreach(parsedExpressions += _) } @@ -946,7 +940,7 @@ object SQLPlanParser extends Logging { val isSortMergeSupported = !(isSortMergeJoin && joinCondition.nonEmpty && isSMJConditionUnsupported(joinCondition)) - (parsedExpressions.distinct.toArray, equiJoinSupportedTypes(buildSide, joinType) + (parsedExpressions.toArray, equiJoinSupportedTypes(buildSide, joinType) && isSortMergeSupported) } @@ -1054,7 +1048,7 @@ object SQLPlanParser extends Logging { } } } - parsedExpressions.distinct.toArray + parsedExpressions.toArray } def parseFilterExpressions(exprStr: String): Array[String] = { @@ -1109,7 +1103,7 @@ object SQLPlanParser extends Logging { processedExpr = nonBinaryOperatorsRegEx.replaceAllIn(processedExpr, " ") // Step-4: remove remaining parentheses '(', ')' and commas if we had functionCalls - if (!functionMatches.isEmpty) { + if (functionMatches.nonEmpty) { // remove "," processedExpr = processedExpr.replaceAll(",", " ") } @@ -1147,7 +1141,6 @@ object SQLPlanParser extends Logging { logDebug(s"Unrecognized Token - $token") } } - - parsedExpressions.distinct.toArray + parsedExpressions.toArray } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala index 5b8730938..8754b69c6 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WholeStageExecParser.scala @@ -59,7 +59,7 @@ abstract class WholeStageExecParserBase( // The node should be marked as shouldRemove when all the children of the // wholeStageCodeGen are marked as shouldRemove. val removeNode = isDupNode || childNodes.forall(_.shouldRemove) - // Remove any suffix in order to get the node label without any trailing number. + // Remove any suffix to get the node label without any trailing number. val nodeLabel = nodeNameRegeX.findFirstMatchIn(node.name) match { case Some(m) => m.group(1) // in case not found, use the full exec name diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala index 853c1b767..1dd50680b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/WindowGroupLimitParser.scala @@ -37,7 +37,7 @@ case class WindowGroupLimitParser( override def getUnsupportedExprReasonsForExec( expressions: Array[String]): Seq[UnsupportedExprOpRef] = { - expressions.flatMap { expr => + expressions.distinct.flatMap { expr => if (!supportedRankingExprs.contains(expr)) { Some(UnsupportedExprOpRef(expr, s"Ranking function $expr is not supported in $fullExecName")) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/ExprOpRef.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/ExprOpRef.scala new file mode 100644 index 000000000..48be98c50 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/ExprOpRef.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + +/** + * Represents a reference to an expression operator that is stored in the ExecInfo expressions + * @param opRef the opRef to wrap + * @param count the count of that expression within the exec. + */ +case class ExprOpRef(opRef: OpRef, count: Int = 1) extends OpRefWrapperBase(opRef) + +object ExprOpRef extends OpRefWrapperBaseTrait[ExprOpRef] { + def fromRawExprSeq(exprArr: Seq[String]): Seq[ExprOpRef] = { + exprArr.groupBy(identity) + .mapValues(expr => ExprOpRef(OpRef.fromExpr(expr.head), expr.size)).values.toSeq + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRefWrapperBase.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRefWrapperBase.scala new file mode 100644 index 000000000..09cd6bb83 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OpRefWrapperBase.scala @@ -0,0 +1,50 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.planparser.ops + +/** + * An instance that wraps OpRef and exposes the OpRef methods. + * This is used to provide common interface for all classes that wrap OpRef along with other + * metadata. + * @param opRef the opRef to wrap + */ +class OpRefWrapperBase(opRef: OpRef) extends OperatorRefTrait { + override def getOpName: String = opRef.getOpName + + override def getOpNameCSV: String = opRef.getOpNameCSV + + override def getOpType: String = opRef.getOpType + + override def getOpTypeCSV: String = opRef.getOpTypeCSV + + def getOpRef: OpRef = opRef +} + +/** + * A trait that provides a factory method to create instances of OpRefWrapperBase from a sequence of + * @tparam R the type of the OpRefWrapperBase + */ +trait OpRefWrapperBaseTrait[R <: OpRefWrapperBase] { + /** + * Create instances of OpRefWrapperBase from a sequence of expressions. + * The expressions are grouped by their value and the count of each expression is stored in the + * OpRefWrapperBase entry. + * @param exprArr the sequence of expressions + * @return a sequence of OpRefWrapperBase instances + */ + def fromRawExprSeq(exprArr: Seq[String]): Seq[R] +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala index fab9422aa..f91bedc49 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/OperatorCounter.scala @@ -38,21 +38,21 @@ case class OperatorCounter(planInfo: PlanInfo) { * @param stages The set of stages where the operator appears. */ case class OperatorData( - opRef: OperatorRefBase, + opRef: OpRef, var count: Int = 0, - var stages: Set[Int] = Set()) + var stages: Set[Int] = Set()) extends OpRefWrapperBase(opRef) // Summarizes the count information for an exec or expression, including whether it is supported. case class OperatorCountSummary( opData: OperatorData, isSupported: Boolean) - private val supportedMap: mutable.Map[OperatorRefBase, OperatorData] = mutable.Map() - private val unsupportedMap: mutable.Map[OperatorRefBase, OperatorData] = mutable.Map() + private val supportedMap: mutable.Map[OpRef, OperatorData] = mutable.Map() + private val unsupportedMap: mutable.Map[OpRef, OperatorData] = mutable.Map() // Returns a sequence of `OperatorCountSummary`, combining both supported and // unsupported operators. - def getOpsCountSummary(): Seq[OperatorCountSummary] = { + def getOpsCountSummary: Seq[OperatorCountSummary] = { supportedMap.values.map(OperatorCountSummary(_, isSupported = true)).toSeq ++ unsupportedMap.values.map(OperatorCountSummary(_, isSupported = false)).toSeq } @@ -60,44 +60,49 @@ case class OperatorCounter(planInfo: PlanInfo) { // Updates the operator data in the given map (supported or unsupported). // Increments the count and updates the stages where the operator appears. - private def updateOpRefEntry(opRef: OperatorRefBase, stages: Set[Int], - targetMap: mutable.Map[OperatorRefBase, OperatorData]): Unit = { + private def updateOpRefEntry(opRef: OpRef, stages: Set[Int], + targetMap: mutable.Map[OpRef, OperatorData], incrValue: Int = 1): Unit = { val operatorData = targetMap.getOrElseUpdate(opRef, OperatorData(opRef)) - operatorData.count += 1 + operatorData.count += incrValue operatorData.stages ++= stages } // Processes an `ExecInfo` node to update exec and expression counts. // Separates supported and unsupported execs and expressions into their respective maps. private def processExecInfo(execInfo: ExecInfo): Unit = { - val opMap = execInfo.isSupported match { - case true => supportedMap - case false => unsupportedMap + val opMap = if (execInfo.isSupported) { + supportedMap + } else { + unsupportedMap } updateOpRefEntry(execInfo.execRef, execInfo.stages, opMap) - // update the map for supported expressions. We should exclude the unsupported expressions. - execInfo.expressions.filterNot( - e => execInfo.unsupportedExprs.exists(exp => exp.opRef.equals(e))).foreach { expr => - updateOpRefEntry(expr, execInfo.stages, supportedMap) - } - // update the map for unsupported expressions - execInfo.unsupportedExprs.foreach { expr => - updateOpRefEntry(expr, execInfo.stages, unsupportedMap) + // Update the map for supported expressions. For unsupported expressions, + // we use the count stored in the supported expressions. + execInfo.expressions.foreach { expr => + val exprMap = + if (execInfo.unsupportedExprs.exists(unsupExec => + unsupExec.getOpRef.equals(expr.getOpRef))) { + // The expression skips because it exists in the unsupported expressions. + unsupportedMap + } else { + supportedMap + } + updateOpRefEntry(expr.getOpRef, execInfo.stages, exprMap, expr.count) } } - // Counts the execs and expressions in the execution plan. + // Counts the execs and expressions in the execution plan excluding clusterNodes + // (i.e., WholeStageCodeGen). private def countOperators(): Unit = { planInfo.execInfo.foreach { exec => - exec.isClusterNode match { - // we do not want to count the cluster nodes in that aggregation - case true => - if (exec.children.nonEmpty) { - exec.children.get.foreach { child => - processExecInfo(child) - } + if (exec.isClusterNode) { + if (exec.children.nonEmpty) { + exec.children.get.foreach { child => + processExecInfo(child) } - case false => processExecInfo(exec) + } + } else { + processExecInfo(exec) } } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala index 0e17d8f0d..e40cfd825 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ops/UnsupportedExprOpRef.scala @@ -25,7 +25,7 @@ package com.nvidia.spark.rapids.tool.planparser.ops * @param unsupportedReason A string describing why the expression is unsupported. */ case class UnsupportedExprOpRef(opRef: OpRef, - unsupportedReason: String) extends OperatorRefBase(opRef.value, opRef.opType) + unsupportedReason: String) extends OpRefWrapperBase(opRef) // Provides a factory method to create an instance from an expression name and unsupported reason. object UnsupportedExprOpRef { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala index 44bbf9fa3..9e58ba967 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala @@ -388,7 +388,7 @@ class PluginTypeChecker(platform: Platform = PlatformFactory.createInstance(), } def getNotSupportedExprs(exprs: Seq[String]): Seq[UnsupportedExprOpRef] = { - exprs.collect { + exprs.distinct.collect { case expr if !isExprSupported(expr) => val reason = unsupportedOpsReasons.getOrElse(expr, "") UnsupportedExprOpRef(expr, reason) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala index 9dbe7c920..f82124feb 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala @@ -166,7 +166,7 @@ class QualOutputWriter(outputDir: String, reportReadSchema: Boolean, sums.foreach { sum => QualOutputWriter.constructUnsupportedDetailedStagesDurationInfo(csvFileWriter, sum, headersAndSizes, - QualOutputWriter.CSV_DELIMITER, false) + QualOutputWriter.CSV_DELIMITER, prettyPrint = false) } } finally { csvFileWriter.close() @@ -888,15 +888,15 @@ object QualOutputWriter { planInfos.foreach { planInfo => val sqlIDCSVStr = planInfo.sqlID.toString val allOpsCount = OperatorCounter(planInfo) - .getOpsCountSummary().sortBy(oInfo => (-oInfo.opData.count, oInfo.opData.opRef.getOpName)) + .getOpsCountSummary.sortBy(oInfo => (-oInfo.opData.count, oInfo.opData.opRef.getOpName)) if (allOpsCount.nonEmpty) { val planBuffer = allOpsCount.map { opInfo => val supportFlag = if (opInfo.isSupported) supportedCSVStr else unsupportedCSVStr val stageStr = StringUtils.reformatCSVString(opInfo.opData.stages.mkString(":")) s"$appIDCSVStr$delimiter" + s"$sqlIDCSVStr$delimiter" + - s"${opInfo.opData.opRef.getOpTypeCSV}$delimiter" + - s"${opInfo.opData.opRef.getOpNameCSV}$delimiter${opInfo.opData.count}$delimiter" + + s"${opInfo.opData.getOpTypeCSV}$delimiter" + + s"${opInfo.opData.getOpNameCSV}$delimiter${opInfo.opData.count}$delimiter" + s"$supportFlag$delimiter" + s"$stageStr" } @@ -1102,32 +1102,36 @@ object QualOutputWriter { reformatCSV: Boolean = true): Unit = { val reformatCSVFunc = getReformatCSVFunc(reformatCSV) val appId = sumInfo.appId + val appIDStr = reformatCSVFunc(appId) val appDuration = sumInfo.estimatedInfo.appDur val dummyStageID = -1 val dummyStageDur = 0 val execIdGenerator = new AtomicLong(0) - def constructDetailedUnsupportedRow(unSupExecInfo: UnsupportedExecSummary, - stageId: Int, stageAppDuration: Long): String = { - val data = ListBuffer[(String, Int)]( - reformatCSVFunc(appId) -> headersAndSizes(APP_ID_STR), - unSupExecInfo.sqlId.toString -> headersAndSizes(SQL_ID_STR), - stageId.toString -> headersAndSizes(STAGE_ID_STR), - reformatCSVFunc(unSupExecInfo.execId.toString) -> headersAndSizes(EXEC_ID), - reformatCSVFunc(unSupExecInfo.finalOpType) -> headersAndSizes(UNSUPPORTED_TYPE), - unSupExecInfo.unsupportedOperatorCSVFormat -> headersAndSizes(UNSUPPORTED_OPERATOR), - reformatCSVFunc(unSupExecInfo.details) -> headersAndSizes(DETAILS), - stageAppDuration.toString -> headersAndSizes(STAGE_WALLCLOCK_DUR_STR), - appDuration.toString -> headersAndSizes(APP_DUR_STR), - reformatCSVFunc(unSupExecInfo.opAction.toString) -> headersAndSizes(EXEC_ACTION) - ) - constructOutputRow(data, delimiter, prettyPrint) + def constructDetailedUnsupportedRow( + appID: String, + unSupExecInfo: UnsupportedExecSummary, + stageId: String, + stageAppDuration: String): String = { + val reformatCSVFunc = getReformatCSVFunc(reformatCSV) + s"$appID" + delimiter + + s"${unSupExecInfo.sqlId}" + delimiter + + s"$stageId" + delimiter + + s"${unSupExecInfo.execId.toString}" + delimiter + + s"${reformatCSVFunc(unSupExecInfo.finalOpType)}" + delimiter + + s"${unSupExecInfo.unsupportedOperatorCSVFormat}" + delimiter + + s"${reformatCSVFunc(unSupExecInfo.details)}" + delimiter + + s"$stageAppDuration" + delimiter + + s"$appDuration" + delimiter + + s"${unSupExecInfo.opAction}\n" } def getUnsupportedRows(execI: ExecInfo, stageId: Int, stageDur: Long): String = { val results = execI.getUnsupportedExecSummaryRecord(execIdGenerator.getAndIncrement()) + val stageIDStr = stageId.toString + val stageDurStr = stageDur.toString results.map { unsupportedExecSummary => - constructDetailedUnsupportedRow(unsupportedExecSummary, stageId, stageDur) + constructDetailedUnsupportedRow(appIDStr, unsupportedExecSummary, stageIDStr, stageDurStr) }.mkString } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index e31064851..771484c14 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import scala.util.control.NonFatal import com.nvidia.spark.rapids.tool.ToolTestUtils +import com.nvidia.spark.rapids.tool.planparser.ops.{ExprOpRef, OpRef} import com.nvidia.spark.rapids.tool.qualification._ import org.scalatest.Matchers.{be, contain, convertToAnyShouldWrapper} import org.scalatest.exceptions.TestFailedException @@ -1319,46 +1320,69 @@ class SQLPlanParserSuite extends BasePlanParserSuite { test("Parsing Conditional Expressions") { // scalastyle:off line.size.limit - val expressionsMap: mutable.HashMap[String, Array[String]] = mutable.HashMap( - "(((lower(partition_act#90) = moduleview) && (isnotnull(productarr#22) && NOT (productarr#22=[]))) || (lower(moduletype#13) = saveforlater))" -> - Array("lower", "isnotnull", "EqualTo", "And", "Not", "Or"), + val expressionsMap: mutable.HashMap[String, Map[String, Int]] = mutable.HashMap( + "(((lower(partition_act#90) = moduleview) && (isnotnull(productarr#22) && NOT (productarr#22 = []))) || (lower(moduletype#13) = saveforlater))" -> + Map("lower" -> 2, + "isnotnull" -> 1, + "EqualTo" -> 3, + "And" -> 2, + "Not" -> 1, + "Or" -> 1), "(IsNotNull(c_customer_id))" -> - Array("IsNotNull"), + Map("IsNotNull" -> 1), "(isnotnull(names#15) AND StartsWith(names#15, OR))" -> - Array("isnotnull", "And", "StartsWith"), + Map("isnotnull" -> 1, + "StartsWith" -> 1, + "And" -> 1), "((isnotnull(s_state#68) AND (s_state#68 = TN)) OR (hex(cast(value#0 as bigint)) = B))" -> - Array("isnotnull", "And", "Or", "hex", "EqualTo"), + Map("isnotnull" -> 1, + "And" -> 1, + "Or" -> 1, + "hex" -> 1, + "EqualTo" -> 2), // Test that AND followed by '(' without space can be parsed "((isnotnull(s_state#68) AND(s_state#68 = TN)) OR (hex(cast(value#0 as bigint)) = B))" -> - Array("isnotnull", "And", "Or", "hex", "EqualTo"), + Map("isnotnull" -> 1, + "And" -> 1, + "Or" -> 1, + "hex" -> 1, + "EqualTo" -> 2), "(((isnotnull(d_year#498) AND isnotnull(d_moy#500)) AND (d_year#498 = 1998)) AND (d_moy#500 = 12))" -> - Array("isnotnull", "And", "EqualTo"), + Map("isnotnull" -> 2, + "And" -> 3, + "EqualTo" -> 2), "IsNotNull(d_year) AND IsNotNull(d_moy) AND EqualTo(d_year,1998) AND EqualTo(d_moy,12)" -> - Array("IsNotNull", "And", "EqualTo"), + Map("IsNotNull" -> 2, + "EqualTo" -> 2, + "And" -> 3), // check that a predicate with a single variable name is fine "flagVariable" -> - Array(), + Map(), // check that a predicate with a single function call "isnotnull(c_customer_sk#412)" -> - Array("isnotnull"), + Map("isnotnull" -> 1), "((substr(ca_zip#457, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#456 IN (CA,WA,GA)) OR (cs_sales_price#20 > 500.00))" -> - Array("substr", "In", "Or", "GreaterThan"), + Map("substr" -> 1, + "In" -> 2, + "Or" -> 2, + "GreaterThan" -> 1), // test the operator is at the beginning of expression and not followed by space "NOT(isnotnull(d_moy))" -> - Array("Not", "isnotnull"), + Map("isnotnull" -> 1, + "Not" -> 1), // test the shiftright operator(since spark-4.0) "((isnotnull(d_year#498) AND isnotnull(d_moy#500)) AND (d_year#498 >> 1) >= 100)" -> - Array("isnotnull", "And", "GreaterThanOrEqual", "ShiftRight") + Map("isnotnull" -> 2, + "And" -> 2, + "ShiftRight" -> 1, + "GreaterThanOrEqual" -> 1) ) // scalastyle:on line.size.limit - for ((condExpr, expectedExpression) <- expressionsMap) { - val parsedExpressionsMine = SQLPlanParser.parseConditionalExpressions(condExpr) - val currOutput = parsedExpressionsMine.sorted - val expectedOutput = expectedExpression.sorted - assert(currOutput sameElements expectedOutput, - s"The parsed expressions are not as expected. Expression: ${condExpr}, " + - s"Expected: ${expectedOutput.mkString}, " + - s"Output: ${currOutput.mkString}") + for ((condExpr, expectedExpressionCounts) <- expressionsMap) { + val rawExpressions = SQLPlanParser.parseConditionalExpressions(condExpr) + val expected = expectedExpressionCounts.map(e => ExprOpRef(OpRef.fromExpr(e._1), e._2)) + val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) + actualExpressions should ===(expected) } } @@ -1403,10 +1427,18 @@ class SQLPlanParserSuite extends BasePlanParserSuite { "AND (content_name_16#197L = 1)) AND NOT (split(split(split(replace(replace(replace" + "(replace(trim(replace(cast(unbase64(content#192) as string), , ), Some( )), *., ), *, ), " + "https://, ), http://, ), /, -1)[0], :, -1)[0], \\?, -1)[0] = ))" - val expected = Array("isnotnull", "split", "replace", "trim", "unbase64", "And", - "EqualTo", "Not") - val expressions = SQLPlanParser.parseFilterExpressions(exprString) - expressions should ===(expected) + val expected = Map( + "isnotnull" -> 3, + "split" -> 3, + "replace" -> 5, + "trim" -> 1, + "unbase64" -> 1, + "And" -> 6, + "EqualTo" -> 4, // EqualTo comes from the = operator + "Not" -> 2).map(e => ExprOpRef(OpRef.fromExpr(e._1), e._2)) + val rawExpressions = SQLPlanParser.parseFilterExpressions(exprString) + val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) + actualExpressions should ===(expected) } @@ -1418,9 +1450,16 @@ class SQLPlanParserSuite extends BasePlanParserSuite { "THEN concat(replace(cast(unbase64(content#192) as string), , ), %) " + "ELSE replace(replace(replace(cast(unbase64(content#192) as string), , ), %, " + "\\%), *, %) END#200])" - val expected = Array("replace", "concat", "instr", "split", "trim", "unbase64") - val expressions = SQLPlanParser.parseAggregateExpressions(exprString) - expressions should ===(expected) + val expected = Map( + "replace" -> 10, + "concat" -> 1, + "instr" -> 1, + "split" -> 3, + "trim" -> 1, + "unbase64" -> 4).map(e => ExprOpRef(OpRef.fromExpr(e._1), e._2)) + val rawExpressions = SQLPlanParser.parseAggregateExpressions(exprString) + val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) + actualExpressions should ===(expected) } runConditionalTest("promote_precision is supported for Spark LT 3.4.0: issue-517", @@ -1522,7 +1561,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { "gid#1296," + "CAST((IF((supersql_t12.`ret_type` = 2), 1, 0)) AS BIGINT)#1300L]]" // Only "IF" should be picked up as a function name - val expected = Array("IF") + val expected = Array("IF", "IF", "IF") val expressions = SQLPlanParser.parseExpandExpressions(exprString) expressions should ===(expected) } From 96041332f9a7d01146ca91d31294433ae472b502 Mon Sep 17 00:00:00 2001 From: spark-rapids automation <70000568+nvauto@users.noreply.github.com> Date: Fri, 6 Dec 2024 16:35:24 +0000 Subject: [PATCH 14/52] Update dev-version by jenkins-spark-rapids-tools-auto-release-96 Signed-off-by: spark-rapids automation <70000568+nvauto@users.noreply.github.com> --- core/pom.xml | 2 +- user_tools/src/spark_rapids_pytools/__init__.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index fd9057ddf..cde7561fb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -23,7 +23,7 @@ rapids-4-spark-tools_2.12 RAPIDS Accelerator for Apache Spark tools RAPIDS Accelerator for Apache Spark tools - 24.10.2-SNAPSHOT + 24.10.3-SNAPSHOT jar http://github.com/NVIDIA/spark-rapids-tools diff --git a/user_tools/src/spark_rapids_pytools/__init__.py b/user_tools/src/spark_rapids_pytools/__init__.py index 84452ec1c..f4ec6f064 100644 --- a/user_tools/src/spark_rapids_pytools/__init__.py +++ b/user_tools/src/spark_rapids_pytools/__init__.py @@ -16,7 +16,7 @@ from spark_rapids_pytools.build import get_version, get_spark_dep_version -VERSION = '24.10.2' +VERSION = '24.10.3' # defines the default runtime build version for the user tools environment SPARK_DEP_VERSION = '350' __version__ = get_version(VERSION) From 8df1407d6fa3f3193890b26b342ccafcba7b4e1e Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 12 Dec 2024 14:54:50 -0600 Subject: [PATCH 15/52] Fix dataframe handling of column-types (#1458) Signed-off-by: Ahmed Hussein (amahussein) Fixes #1456 --- .../tools/qualification_stats_report.py | 15 ++++++++++++--- .../spark_rapids_tools/tools/qualx/preprocess.py | 4 ++-- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/user_tools/src/spark_rapids_tools/tools/qualification_stats_report.py b/user_tools/src/spark_rapids_tools/tools/qualification_stats_report.py index de762013d..aedda60f7 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualification_stats_report.py +++ b/user_tools/src/spark_rapids_tools/tools/qualification_stats_report.py @@ -75,7 +75,10 @@ def _read_csv_files(self) -> None: 'toolOutput', 'csv', 'unsupportedOperatorsReport', 'fileName') rapids_unsupported_operators_file = FSUtil.build_path( qual_output_dir, unsupported_operator_report_file) - self.unsupported_operators_df = pd.read_csv(rapids_unsupported_operators_file) + # load the unsupported operators and drop operators that have no names. + self.unsupported_operators_df = ( + pd.read_csv(rapids_unsupported_operators_file, + dtype={'Unsupported Operator': str})).dropna(subset=['Unsupported Operator']) stages_report_file = self.ctxt.get_value('toolOutput', 'csv', 'stagesInformation', 'fileName') @@ -84,7 +87,14 @@ def _read_csv_files(self) -> None: rapids_execs_file = self.ctxt.get_value('toolOutput', 'csv', 'execsInformation', 'fileName') - self.execs_df = pd.read_csv(FSUtil.build_path(qual_output_dir, rapids_execs_file)) + # Load the execs CSV file and drop execs that have no stages or name + self.execs_df = ( + pd.read_csv(FSUtil.build_path(qual_output_dir, rapids_execs_file), + dtype={'Exec Name': str, + 'Exec Stages': str, + 'Exec Children': str, + 'Exec Children Node Ids': str}) + .dropna(subset=['Exec Stages', 'Exec Name'])) self.logger.info('Reading CSV files completed.') def _convert_durations(self) -> None: @@ -103,7 +113,6 @@ def _preprocess_dataframes(self) -> None: # from this dataframe can be matched with the stageID of stages dataframe self.execs_df['Exec Stages'] = self.execs_df['Exec Stages'].str.split(':') self.execs_df = (self.execs_df.explode('Exec Stages'). - dropna(subset=['Exec Stages']). rename(columns={'Exec Stages': 'Stage ID'})) self.execs_df['Stage ID'] = self.execs_df['Stage ID'].astype(int) diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py index 1bdbd76b7..a47b45d73 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py @@ -461,7 +461,7 @@ def combine_tables(table_name: str) -> pd.DataFrame: # normalize WholeStageCodegen labels ops_tbl.loc[ - ops_tbl['nodeName'].str.startswith('WholeStageCodegen'), 'nodeName' + ops_tbl['nodeName'].astype(str).str.startswith('WholeStageCodegen'), 'nodeName' ] = 'WholeStageCodegen' # format WholeStageCodegen for merging @@ -1140,7 +1140,7 @@ def _is_ignore_no_perf(action: str) -> bool: node_level_supp['Exec Is Supported'] = ( node_level_supp['Exec Is Supported'] | node_level_supp['Action'].apply(_is_ignore_no_perf) - | node_level_supp['Exec Name'].apply( + | node_level_supp['Exec Name'].astype(str).apply( lambda x: x.startswith('WholeStageCodegen') ) ) From 75760a9044387027988c73731cc17f61122a3ea7 Mon Sep 17 00:00:00 2001 From: spark-rapids automation <70000568+nvauto@users.noreply.github.com> Date: Fri, 13 Dec 2024 01:57:42 +0000 Subject: [PATCH 16/52] Update dev-version by jenkins-spark-rapids-tools-auto-release-99 Signed-off-by: spark-rapids automation <70000568+nvauto@users.noreply.github.com> --- core/pom.xml | 2 +- user_tools/src/spark_rapids_pytools/__init__.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index cde7561fb..7ae043493 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -23,7 +23,7 @@ rapids-4-spark-tools_2.12 RAPIDS Accelerator for Apache Spark tools RAPIDS Accelerator for Apache Spark tools - 24.10.3-SNAPSHOT + 24.10.4-SNAPSHOT jar http://github.com/NVIDIA/spark-rapids-tools diff --git a/user_tools/src/spark_rapids_pytools/__init__.py b/user_tools/src/spark_rapids_pytools/__init__.py index f4ec6f064..13d08a4dd 100644 --- a/user_tools/src/spark_rapids_pytools/__init__.py +++ b/user_tools/src/spark_rapids_pytools/__init__.py @@ -16,7 +16,7 @@ from spark_rapids_pytools.build import get_version, get_spark_dep_version -VERSION = '24.10.3' +VERSION = '24.10.4' # defines the default runtime build version for the user tools environment SPARK_DEP_VERSION = '350' __version__ = get_version(VERSION) From a1f866fc4361124aae8ea9e1b3542cb2e4b8feb1 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Mon, 16 Dec 2024 09:20:02 -0600 Subject: [PATCH 17/52] Deduplicate calls to aggregateSparkMetricsBySql (#1464) Signed-off-by: Ahmed Hussein (amahussein) Contributes to #1461 AppSparkMetricsAnalyzer was calling `aggregateSparkMetricsBySql` twice. This code change eleiminates this redundancy to save CPU time and memory allocations. --- .../spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala index 0f43ae8b2..30fb10ac9 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAggTrait.scala @@ -35,12 +35,13 @@ trait AppSparkMetricsAggTrait extends AppIndexMapperTrait { def getAggRawMetrics(app: AppBase, index: Int, sqlAnalyzer: Option[AppSQLPlanAnalyzer] = None): AggRawMetricsResult = { val analysisObj = new AppSparkMetricsAnalyzer(app) + val sqlMetricsAgg = analysisObj.aggregateSparkMetricsBySql(index) AggRawMetricsResult( analysisObj.aggregateSparkMetricsByJob(index), analysisObj.aggregateSparkMetricsByStage(index), analysisObj.shuffleSkewCheck(index), - analysisObj.aggregateSparkMetricsBySql(index), - analysisObj.aggregateIOMetricsBySql(analysisObj.aggregateSparkMetricsBySql(index)), + sqlMetricsAgg, + analysisObj.aggregateIOMetricsBySql(sqlMetricsAgg), analysisObj.aggregateDurationAndCPUTimeBySql(index), Seq(analysisObj.maxTaskInputSizeBytesPerSQL(index)), analysisObj.aggregateDiagnosticMetricsByStage(index, sqlAnalyzer)) From 9564d0b400ff49b1c39ce52ab6f508641c2b4d74 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Mon, 16 Dec 2024 09:23:19 -0600 Subject: [PATCH 18/52] Mark RunningWindowFunction as supported in Qual tool (#1465) Signed-off-by: Ahmed Hussein (amahussein) Fix #1460 `RunningWindowFunction` should be marked as supported by the qualification tool. --- .../src/main/resources/operatorsScore-databricks-aws-a10G.csv | 1 + core/src/main/resources/operatorsScore-databricks-aws-t4.csv | 1 + .../src/main/resources/operatorsScore-databricks-azure-t4.csv | 1 + core/src/main/resources/operatorsScore-dataproc-gke-l4.csv | 1 + core/src/main/resources/operatorsScore-dataproc-gke-t4.csv | 1 + core/src/main/resources/operatorsScore-dataproc-l4.csv | 1 + .../main/resources/operatorsScore-dataproc-serverless-l4.csv | 1 + core/src/main/resources/operatorsScore-dataproc-t4.csv | 1 + core/src/main/resources/operatorsScore-emr-a10.csv | 1 + core/src/main/resources/operatorsScore-emr-a10G.csv | 1 + core/src/main/resources/operatorsScore-emr-t4.csv | 1 + core/src/main/resources/operatorsScore-onprem-a100.csv | 1 + core/src/main/resources/supportedExecs.csv | 1 + .../spark/rapids/tool/planparser/GenericExecParser.scala | 3 ++- .../nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala | 4 ++-- 15 files changed, 17 insertions(+), 3 deletions(-) diff --git a/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv b/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv index e5096e1a9..52eb193f2 100644 --- a/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv +++ b/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv @@ -305,3 +305,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-databricks-aws-t4.csv b/core/src/main/resources/operatorsScore-databricks-aws-t4.csv index e5096e1a9..52eb193f2 100644 --- a/core/src/main/resources/operatorsScore-databricks-aws-t4.csv +++ b/core/src/main/resources/operatorsScore-databricks-aws-t4.csv @@ -305,3 +305,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-databricks-azure-t4.csv b/core/src/main/resources/operatorsScore-databricks-azure-t4.csv index 66c738016..5ad387036 100644 --- a/core/src/main/resources/operatorsScore-databricks-azure-t4.csv +++ b/core/src/main/resources/operatorsScore-databricks-azure-t4.csv @@ -293,3 +293,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv b/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv index 57fc5b44d..902e598a1 100644 --- a/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv @@ -287,3 +287,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv b/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv index 3459e64cb..e30f156f4 100644 --- a/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv @@ -287,3 +287,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-l4.csv b/core/src/main/resources/operatorsScore-dataproc-l4.csv index 422020970..0660dbdee 100644 --- a/core/src/main/resources/operatorsScore-dataproc-l4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-l4.csv @@ -293,3 +293,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv b/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv index 61d9e3f1a..8dc9faa90 100644 --- a/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv @@ -287,3 +287,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-t4.csv b/core/src/main/resources/operatorsScore-dataproc-t4.csv index 10ef53900..e2eb69f60 100644 --- a/core/src/main/resources/operatorsScore-dataproc-t4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-t4.csv @@ -293,3 +293,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-emr-a10.csv b/core/src/main/resources/operatorsScore-emr-a10.csv index 77befd12e..0d350be80 100644 --- a/core/src/main/resources/operatorsScore-emr-a10.csv +++ b/core/src/main/resources/operatorsScore-emr-a10.csv @@ -293,3 +293,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-emr-a10G.csv b/core/src/main/resources/operatorsScore-emr-a10G.csv index 77befd12e..0d350be80 100644 --- a/core/src/main/resources/operatorsScore-emr-a10G.csv +++ b/core/src/main/resources/operatorsScore-emr-a10G.csv @@ -293,3 +293,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-emr-t4.csv b/core/src/main/resources/operatorsScore-emr-t4.csv index 3f1296c38..c651cf976 100644 --- a/core/src/main/resources/operatorsScore-emr-t4.csv +++ b/core/src/main/resources/operatorsScore-emr-t4.csv @@ -293,3 +293,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/operatorsScore-onprem-a100.csv b/core/src/main/resources/operatorsScore-onprem-a100.csv index 4dc58f0c0..7cdd59978 100644 --- a/core/src/main/resources/operatorsScore-onprem-a100.csv +++ b/core/src/main/resources/operatorsScore-onprem-a100.csv @@ -305,3 +305,4 @@ DecimalSum,1.5 MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 +RunningWindowFunctionExec,1.5 diff --git a/core/src/main/resources/supportedExecs.csv b/core/src/main/resources/supportedExecs.csv index 06e35b026..50e47f42f 100644 --- a/core/src/main/resources/supportedExecs.csv +++ b/core/src/main/resources/supportedExecs.csv @@ -57,3 +57,4 @@ WriteFilesExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S CustomShuffleReaderExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS WindowGroupLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS MapInArrowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +RunningWindowFunctionExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala index 6295c5533..cc60904be 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/GenericExecParser.scala @@ -91,7 +91,8 @@ class GenericExecParser( ExecInfo( node, sqlID, - node.name, + // Remove trailing spaces from node name if any + node.name.trim, "", speedupFactor, duration, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index 4d9c59dd8..8471e8a57 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -483,8 +483,8 @@ object SQLPlanParser extends Logging { case "AggregateInPandas" | "ArrowEvalPython" | "AQEShuffleRead" | "CartesianProduct" | "Coalesce" | "CollectLimit" | "CustomShuffleReader" | "FlatMapGroupsInPandas" | "GlobalLimit" | "LocalLimit" | "InMemoryTableScan" | "MapInPandas" - | "PythonMapInArrow" | "MapInArrow" | "Range" | "Sample" | "Union" - | "WindowInPandas" => + | "PythonMapInArrow" | "MapInArrow" | "Range" | "RunningWindowFunction" + | "Sample" | "Union" | "WindowInPandas" => GenericExecParser(node, checker, sqlID, app = Some(app)).parse case "BatchScan" => BatchScanExecParser(node, checker, sqlID, app).parse From aa59d200bcf5638b7712b745a54fce59b2cf58b2 Mon Sep 17 00:00:00 2001 From: Matt Ahrens Date: Mon, 16 Dec 2024 15:33:30 -0600 Subject: [PATCH 19/52] Adding Spark 3.5.2 support in auto tuner for EMR (#1466) Signed-off-by: mattahrens --- .../scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala index 783726cd9..44ff839ba 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala @@ -760,6 +760,7 @@ class AutoTuner( } } else if (sparkVersion.contains("amzn")) { sparkVersion match { + case ver if ver.contains("3.5.2") => "352" case ver if ver.contains("3.5.1") => "351" case ver if ver.contains("3.5.0") => "350" case ver if ver.contains("3.4.1") => "341" From 4143cccefc06d6d5585b30b93e0c14f533b4e824 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Tue, 17 Dec 2024 11:17:23 -0800 Subject: [PATCH 20/52] Make platform mandatory for qualification and profiling CLI (#1463) Signed-off-by: Partho Sarthi --- .../spark_rapids_tools/cmdli/argprocessor.py | 9 +++ .../test_tool_argprocessor.py | 65 +++++++------------ 2 files changed, 32 insertions(+), 42 deletions(-) diff --git a/user_tools/src/spark_rapids_tools/cmdli/argprocessor.py b/user_tools/src/spark_rapids_tools/cmdli/argprocessor.py index 978f683c1..87b39c40e 100644 --- a/user_tools/src/spark_rapids_tools/cmdli/argprocessor.py +++ b/user_tools/src/spark_rapids_tools/cmdli/argprocessor.py @@ -400,6 +400,15 @@ def init_extra_arg_cases(self) -> list: def define_invalid_arg_cases(self) -> None: super().define_invalid_arg_cases() self.define_rejected_missing_eventlogs() + self.rejected['Missing Platform argument'] = { + 'valid': False, + 'callable': partial(self.raise_validation_exception, + 'Cannot run tool cmd without platform argument. Re-run the command ' + 'providing the platform argument.'), + 'cases': [ + [ArgValueCase.UNDEFINED, ArgValueCase.IGNORE, ArgValueCase.IGNORE] + ] + } self.rejected['Cluster By Name Without Platform Hints'] = { 'valid': False, 'callable': partial(self.raise_validation_exception, diff --git a/user_tools/tests/spark_rapids_tools_ut/test_tool_argprocessor.py b/user_tools/tests/spark_rapids_tools_ut/test_tool_argprocessor.py index 300751242..ff4bfff35 100644 --- a/user_tools/tests/spark_rapids_tools_ut/test_tool_argprocessor.py +++ b/user_tools/tests/spark_rapids_tools_ut/test_tool_argprocessor.py @@ -131,13 +131,9 @@ def test_with_platform_with_eventlogs(self, get_ut_data_dir, tool_name, csp): cost_savings_enabled=False, expected_platform=csp) - # should pass: platform not provided; event logs are provided - tool_args = self.create_tool_args_should_pass(tool_name, - eventlogs=f'{get_ut_data_dir}/eventlogs') - # for qualification, cost savings should be disabled because cluster is not provided - self.validate_tool_args(tool_name=tool_name, tool_args=tool_args, - cost_savings_enabled=False, - expected_platform=CspEnv.ONPREM) + # should fail: platform must be provided + self.create_tool_args_should_fail(tool_name, + eventlogs=f'{get_ut_data_dir}/eventlogs') @pytest.mark.parametrize('tool_name', ['qualification', 'profiling']) @pytest.mark.parametrize('csp', all_csps) @@ -150,17 +146,19 @@ def test_with_platform_with_eventlogs_with_jar_files(self, get_ut_data_dir, tool tools_jar=f'{get_ut_data_dir}/tools_mock.jar') assert tool_args['toolsJar'] == f'{get_ut_data_dir}/tools_mock.jar' - # should pass: tools_jar is correct - tool_args = self.create_tool_args_should_pass(tool_name, eventlogs=f'{get_ut_data_dir}/eventlogs', - tools_jar=f'{get_ut_data_dir}/tools_mock.jar') - assert tool_args['toolsJar'] == f'{get_ut_data_dir}/tools_mock.jar' + # should fail: platform must be provided + self.create_tool_args_should_fail(tool_name, + eventlogs=f'{get_ut_data_dir}/eventlogs', + tools_jar=f'{get_ut_data_dir}/tools_mock.jar') # should fail: tools_jar does not exist - self.create_tool_args_should_fail(tool_name, eventlogs=f'{get_ut_data_dir}/eventlogs', + self.create_tool_args_should_fail(tool_name, platform=csp, + eventlogs=f'{get_ut_data_dir}/eventlogs', tools_jar=f'{get_ut_data_dir}/tools_mock.txt') # should fail: tools_jar is not .jar extension - self.create_tool_args_should_fail(tool_name, eventlogs=f'{get_ut_data_dir}/eventlogs', + self.create_tool_args_should_fail(tool_name, platform=csp, + eventlogs=f'{get_ut_data_dir}/eventlogs', tools_jar=f'{get_ut_data_dir}/worker_info.yaml') @pytest.mark.parametrize('tool_name', ['qualification', 'profiling']) @@ -230,25 +228,15 @@ def test_with_platform_with_cluster_props(self, get_ut_data_dir, tool_name, csp, self.validate_tool_args(tool_name=tool_name, tool_args=tool_args, cost_savings_enabled=True, expected_platform=csp) - - # should pass: platform not provided; missing eventlogs should be accepted for all CSPs (except onPrem) - # because the eventlogs can be retrieved from the cluster properties - tool_args = self.create_tool_args_should_pass(tool_name, - cluster=cluster_prop_file) - # for qualification, cost savings should be enabled because cluster is provided - self.validate_tool_args(tool_name=tool_name, tool_args=tool_args, - cost_savings_enabled=True, - expected_platform=csp) else: # should fail: onprem platform cannot retrieve eventlogs from cluster properties self.create_tool_args_should_fail(tool_name, platform=csp, cluster=cluster_prop_file) - # should fail: platform not provided; defaults platform to onprem, cannot retrieve eventlogs from - # cluster properties - self.create_tool_args_should_fail(tool_name, - cluster=cluster_prop_file) + # should fail: platform must be provided for all CSPs as well as onprem + self.create_tool_args_should_fail(tool_name, + cluster=cluster_prop_file) @pytest.mark.parametrize('tool_name', ['qualification', 'profiling']) @pytest.mark.parametrize('csp,prop_path', all_cpu_cluster_props) @@ -266,14 +254,10 @@ def test_with_platform_with_cluster_props_with_eventlogs(self, get_ut_data_dir, cost_savings_enabled=CspEnv(csp) != CspEnv.ONPREM, expected_platform=csp) - # should pass: platform not provided; cluster properties and eventlogs are provided - tool_args = self.create_tool_args_should_pass(tool_name, - cluster=cluster_prop_file, - eventlogs=f'{get_ut_data_dir}/eventlogs') - # for qualification, cost savings should be enabled because cluster is provided (except for onprem) - self.validate_tool_args(tool_name=tool_name, tool_args=tool_args, - cost_savings_enabled=CspEnv(csp) != CspEnv.ONPREM, - expected_platform=csp) + # should fail: platform must be provided + self.create_tool_args_should_fail(tool_name, + cluster=cluster_prop_file, + eventlogs=f'{get_ut_data_dir}/eventlogs') @pytest.mark.parametrize('tool_name', ['profiling']) @pytest.mark.parametrize('csp', all_csps) @@ -308,18 +292,15 @@ def test_with_platform_with_autotuner_with_eventlogs(self, get_ut_data_dir, tool cost_savings_enabled=False, expected_platform=csp) - # should pass: platform not provided; autotuner properties and eventlogs are provided - tool_args = self.create_tool_args_should_pass(tool_name, - cluster=autotuner_prop_file, - eventlogs=f'{get_ut_data_dir}/eventlogs') - # cost savings should be disabled for profiling - self.validate_tool_args(tool_name=tool_name, tool_args=tool_args, - cost_savings_enabled=False, - expected_platform=CspEnv.ONPREM) + # should fail: platform must be provided + self.create_tool_args_should_fail(tool_name, + cluster=autotuner_prop_file, + eventlogs=f'{get_ut_data_dir}/eventlogs') @pytest.mark.parametrize('prop_path', [autotuner_prop_path]) def test_profiler_with_driverlog(self, get_ut_data_dir, prop_path): prof_args = AbsToolUserArgModel.create_tool_args('profiling', + platform=CspEnv.get_default(), driverlog=f'{get_ut_data_dir}/{prop_path}') assert not prof_args['requiresEventlogs'] assert prof_args['rapidOptions']['driverlog'] == f'{get_ut_data_dir}/{prop_path}' From 18b0472af9ffcf77e43fe5726e91b74ecab0637f Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Wed, 18 Dec 2024 11:31:32 -0600 Subject: [PATCH 21/52] Optimize implementation of getAggregateRawMetrics in core-tools (#1468) * Optimize implementation of getAggregateRawMetrics in core-tools * address reviews and fix issues in aggregateDiagnostic Contributes to #1461 This commit improves the implementation of aggregation accross raw metrics by replacing the builtin scala collections with accumulators. --------- Signed-off-by: Ahmed Hussein (amahussein) --- .../analysis/AppSparkMetricsAnalyzer.scala | 293 ++++++++---------- .../tool/analysis/util/AggAccumHelper.scala | 59 ++++ .../analysis/util/AggAccumPhotonHelper.scala | 31 ++ .../tool/analysis/util/JobAggAccum.scala | 31 ++ .../tool/analysis/util/SQLAggAccum.scala | 42 +++ .../tool/analysis/util/StageAggAccum.scala | 31 ++ .../tool/analysis/util/StageAggPhoton.scala | 54 ++++ .../analysis/util/TaskMetricsAccumRec.scala | 157 ++++++++++ .../sql/rapids/tool/store/AccumMetaRef.scala | 1 + .../sql/rapids/tool/store/AccumNameRef.scala | 2 +- 10 files changed, 529 insertions(+), 172 deletions(-) create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumPhotonHelper.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/JobAggAccum.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/SQLAggAccum.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggAccum.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/TaskMetricsAccumRec.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 33194644e..3a862097b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -16,17 +16,16 @@ package com.nvidia.spark.rapids.tool.analysis -import java.util.concurrent.TimeUnit - import scala.collection.mutable.{ArrayBuffer, HashMap, LinkedHashMap} import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics._ +import com.nvidia.spark.rapids.tool.analysis.util.{AggAccumHelper, AggAccumPhotonHelper} import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper -import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IOAnalysisProfileResult, JobAggTaskMetricsProfileResult, ShuffleSkewProfileResult, SQLDurationExecutorTimeProfileResult, SQLMaxTaskInputSizes, SQLTaskAggMetricsProfileResult, StageAggTaskMetricsProfileResult, StageDiagnosticResult} +import com.nvidia.spark.rapids.tool.profiling._ import org.apache.spark.sql.rapids.tool.{AppBase, ToolUtils} import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo -import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef, AccumNameRef, TaskModel} +import org.apache.spark.sql.rapids.tool.store.{AccumInfo, AccumMetaRef} /** * Does analysis on the DataFrames from object of AppBase. @@ -84,52 +83,47 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { if (jc.stageIds.isEmpty) { None } else { - val profResultsInJob = stageLevelSparkMetrics(index).filterKeys(jc.stageIds.contains).values - if (profResultsInJob.isEmpty) { + val jobAggAccumulator = new AggAccumHelper() + val perJobRec = jobAggAccumulator.accumPerJob( + jc.stageIds.collect { + case stageId if stageLevelSparkMetrics(index).contains(stageId) => + stageLevelSparkMetrics(index)(stageId) + }) + if (perJobRec.isEmptyAggregates) { None } else { - // Recalculate the duration sum, max, min, avg for the job based on the cached - // stage Profiling results - val tasksInJob = profResultsInJob.map(_.numTasks).sum - val durSum = profResultsInJob.map(_.durationSum).sum - val durMax = - AppSparkMetricsAnalyzer.maxWithEmptyHandling(profResultsInJob.map(_.durationMax)) - val durMin = - AppSparkMetricsAnalyzer.minWithEmptyHandling(profResultsInJob.map(_.durationMin)) - val durAvg = ToolUtils.calculateAverage(durSum, tasksInJob, 1) Some(JobAggTaskMetricsProfileResult(index, id, - tasksInJob, + perJobRec.numTasks, jc.duration, - profResultsInJob.map(_.diskBytesSpilledSum).sum, - durSum, - durMax, - durMin, - durAvg, - profResultsInJob.map(_.executorCPUTimeSum).sum, - profResultsInJob.map(_.executorDeserializeCpuTimeSum).sum, - profResultsInJob.map(_.executorDeserializeTimeSum).sum, - profResultsInJob.map(_.executorRunTimeSum).sum, - profResultsInJob.map(_.inputBytesReadSum).sum, - profResultsInJob.map(_.inputRecordsReadSum).sum, - profResultsInJob.map(_.jvmGCTimeSum).sum, - profResultsInJob.map(_.memoryBytesSpilledSum).sum, - profResultsInJob.map(_.outputBytesWrittenSum).sum, - profResultsInJob.map(_.outputRecordsWrittenSum).sum, - AppSparkMetricsAnalyzer.maxWithEmptyHandling( - profResultsInJob.map(_.peakExecutionMemoryMax)), - profResultsInJob.map(_.resultSerializationTimeSum).sum, - AppSparkMetricsAnalyzer.maxWithEmptyHandling(profResultsInJob.map(_.resultSizeMax)), - profResultsInJob.map(_.srFetchWaitTimeSum).sum, - profResultsInJob.map(_.srLocalBlocksFetchedSum).sum, - profResultsInJob.map(_.srcLocalBytesReadSum).sum, - profResultsInJob.map(_.srRemoteBlocksFetchSum).sum, - profResultsInJob.map(_.srRemoteBytesReadSum).sum, - profResultsInJob.map(_.srRemoteBytesReadToDiskSum).sum, - profResultsInJob.map(_.srTotalBytesReadSum).sum, - profResultsInJob.map(_.swBytesWrittenSum).sum, - profResultsInJob.map(_.swRecordsWrittenSum).sum, - profResultsInJob.map(_.swWriteTimeSum).sum)) + perJobRec.diskBytesSpilledSum, + perJobRec.durationSum, + perJobRec.durationMax, + perJobRec.durationMin, + perJobRec.durationAvg, + perJobRec.executorCPUTimeSum, + perJobRec.executorDeserializeCpuTimeSum, + perJobRec.executorDeserializeTimeSum, + perJobRec.executorRunTimeSum, + perJobRec.inputBytesReadSum, + perJobRec.inputRecordsReadSum, + perJobRec.jvmGCTimeSum, + perJobRec.memoryBytesSpilledSum, + perJobRec.outputBytesWrittenSum, + perJobRec.outputRecordsWrittenSum, + perJobRec.peakExecutionMemoryMax, + perJobRec.resultSerializationTimeSum, + perJobRec.resultSizeMax, + perJobRec.srFetchWaitTimeSum, + perJobRec.srLocalBlocksFetchedSum, + perJobRec.srLocalBytesReadSum, + perJobRec.srRemoteBlocksFetchSum, + perJobRec.srRemoteBytesReadSum, + perJobRec.srRemoteBytesReadToDiskSum, + perJobRec.srTotalBytesReadSum, + perJobRec.swBytesWrittenSum, + perJobRec.swRecordsWrittenSum, + perJobRec.swWriteTimeSum)) } } } @@ -182,66 +176,55 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { if (app.sqlIdToStages.contains(sqlId)) { val stagesInSQL = app.sqlIdToStages(sqlId) // TODO: Should we only consider successful tasks? - val cachedResBySQL = stageLevelSparkMetrics(index).filterKeys(stagesInSQL.contains).values - if (cachedResBySQL.isEmpty) { + val sqlAggAccumulator = new AggAccumHelper() + val preSqlRec = sqlAggAccumulator.accumPerSQL( + stagesInSQL.collect { + case stageId if stageLevelSparkMetrics(index).contains(stageId) => + stageLevelSparkMetrics(index)(stageId) + }) + if (preSqlRec.isEmptyAggregates) { None } else { - // Recalculate the duration sum, max, min, avg for the job based on the cached - // stage Profiling results - val tasksInSql = cachedResBySQL.map(_.numTasks).sum - val durSum = cachedResBySQL.map(_.durationSum).sum - val durMax = - AppSparkMetricsAnalyzer.maxWithEmptyHandling(cachedResBySQL.map(_.durationMax)) - val durMin = - AppSparkMetricsAnalyzer.minWithEmptyHandling(cachedResBySQL.map(_.durationMin)) - val durAvg = ToolUtils.calculateAverage(durSum, tasksInSql, 1) - val diskBytes = cachedResBySQL.map(_.diskBytesSpilledSum).sum - val execCpuTime = cachedResBySQL.map(_.executorCPUTimeSum).sum - val execRunTime = cachedResBySQL.map(_.executorRunTimeSum).sum - val execCPURatio = ToolUtils.calculateDurationPercent(execCpuTime, execRunTime) - val inputBytesRead = cachedResBySQL.map(_.inputBytesReadSum).sum // set this here, so make sure we don't get it again until later - sqlCase.sqlCpuTimePercent = execCPURatio - + sqlCase.sqlCpuTimePercent = preSqlRec.executorCpuRatio Some(SQLTaskAggMetricsProfileResult(index, app.appId, sqlId, sqlCase.description, - tasksInSql, + preSqlRec.numTasks, sqlCase.duration, - execCpuTime, - execRunTime, - execCPURatio, - diskBytes, - durSum, - durMax, - durMin, - durAvg, - execCpuTime, - cachedResBySQL.map(_.executorDeserializeCpuTimeSum).sum, - cachedResBySQL.map(_.executorDeserializeTimeSum).sum, - execRunTime, - inputBytesRead, - inputBytesRead * 1.0 / tasksInSql, - cachedResBySQL.map(_.inputRecordsReadSum).sum, - cachedResBySQL.map(_.jvmGCTimeSum).sum, - cachedResBySQL.map(_.memoryBytesSpilledSum).sum, - cachedResBySQL.map(_.outputBytesWrittenSum).sum, - cachedResBySQL.map(_.outputRecordsWrittenSum).sum, - AppSparkMetricsAnalyzer.maxWithEmptyHandling( - cachedResBySQL.map(_.peakExecutionMemoryMax)), - cachedResBySQL.map(_.resultSerializationTimeSum).sum, - AppSparkMetricsAnalyzer.maxWithEmptyHandling(cachedResBySQL.map(_.resultSizeMax)), - cachedResBySQL.map(_.srFetchWaitTimeSum).sum, - cachedResBySQL.map(_.srLocalBlocksFetchedSum).sum, - cachedResBySQL.map(_.srcLocalBytesReadSum).sum, - cachedResBySQL.map(_.srRemoteBlocksFetchSum).sum, - cachedResBySQL.map(_.srRemoteBytesReadSum).sum, - cachedResBySQL.map(_.srRemoteBytesReadToDiskSum).sum, - cachedResBySQL.map(_.srTotalBytesReadSum).sum, - cachedResBySQL.map(_.swBytesWrittenSum).sum, - cachedResBySQL.map(_.swRecordsWrittenSum).sum, - cachedResBySQL.map(_.swWriteTimeSum).sum)) + preSqlRec.executorCPUTimeSum, + preSqlRec.executorRunTimeSum, + preSqlRec.executorCpuRatio, + preSqlRec.diskBytesSpilledSum, + preSqlRec.durationSum, + preSqlRec.durationMax, + preSqlRec.durationMin, + preSqlRec.durationAvg, + preSqlRec.executorCPUTimeSum, + preSqlRec.executorDeserializeCpuTimeSum, + preSqlRec.executorDeserializeTimeSum, + preSqlRec.executorRunTimeSum, + preSqlRec.inputBytesReadSum, + preSqlRec.inputBytesReadAvg, + preSqlRec.inputRecordsReadSum, + preSqlRec.jvmGCTimeSum, + preSqlRec.memoryBytesSpilledSum, + preSqlRec.outputBytesWrittenSum, + preSqlRec.outputRecordsWrittenSum, + preSqlRec.peakExecutionMemoryMax, + preSqlRec.resultSerializationTimeSum, + preSqlRec.resultSizeMax, + preSqlRec.srFetchWaitTimeSum, + preSqlRec.srLocalBlocksFetchedSum, + preSqlRec.srLocalBytesReadSum, + preSqlRec.srRemoteBlocksFetchSum, + preSqlRec.srRemoteBytesReadSum, + preSqlRec.srRemoteBytesReadToDiskSum, + preSqlRec.srTotalBytesReadSum, + preSqlRec.swBytesWrittenSum, + preSqlRec.swRecordsWrittenSum, + preSqlRec.swWriteTimeSum)) } } else { None @@ -339,8 +322,9 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { app.asInstanceOf[ApplicationInfo].planMetricProcessor } val zeroAccumProfileResults = - AccumProfileResults(0, 0, AccumMetaRef(0L, AccumNameRef("")), 0L, 0L, 0L, 0L) - + AccumProfileResults(0, 0, AccumMetaRef.EMPTY_ACCUM_META_REF, 0L, 0L, 0L, 0L) + val emptyNodeNames = Seq.empty[String] + val emptyDiagnosticMetrics = HashMap.empty[String, AccumProfileResults] // TODO: this has stage attempts. we should handle different attempts app.stageManager.getAllStages.map { sm => // TODO: Should we only consider successful tasks? @@ -348,11 +332,11 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { sm.stageInfo.attemptNumber()) // count duplicate task attempts val numTasks = tasksInStage.size - val nodeNames = sqlAnalyzer.stageToNodeNames. - getOrElse(sm.stageInfo.stageId, Seq.empty[String]) - val diagnosticMetricsMap = sqlAnalyzer.stageToDiagnosticMetrics. - getOrElse(sm.stageInfo.stageId, HashMap.empty[String, AccumProfileResults]). - withDefaultValue(zeroAccumProfileResults) + val nodeNames = sqlAnalyzer.stageToNodeNames.getOrElse(sm.stageInfo.stageId, emptyNodeNames) + val diagnosticMetricsMap = + sqlAnalyzer.stageToDiagnosticMetrics + .getOrElse(sm.stageInfo.stageId, emptyDiagnosticMetrics) + .withDefaultValue(zeroAccumProfileResults) val srTotalBytesMetrics = AppSparkMetricsAnalyzer.getStatistics(tasksInStage.map(_.sr_totalBytesRead)) @@ -417,10 +401,8 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { // TODO: Should we only consider successful tasks? val tasksInStage = app.taskManager.getTasks(sm.stageInfo.stageId, sm.stageInfo.attemptNumber()) - // count duplicate task attempts - val numAttempts = tasksInStage.size - val (peakMemoryMax, shuffleWriteTimeSum) = if (app.isPhoton) { + val accumHelperObj = if (app.isPhoton) { // If this a photon app, use the photonHelper // For max peak memory, we need to look at the accumulators at the task level. val peakMemoryValues = tasksInStage.flatMap { taskModel => photonPeakMemoryAccumInfos.flatMap { accumInfo => @@ -431,50 +413,45 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { val shuffleWriteValues = photonShuffleWriteTimeAccumInfos.flatMap { accumInfo => accumInfo.stageValuesMap.get(sm.stageInfo.stageId) } - (AppSparkMetricsAnalyzer.maxWithEmptyHandling(peakMemoryValues), - TimeUnit.NANOSECONDS.toMillis(shuffleWriteValues.sum)) + new AggAccumPhotonHelper(shuffleWriteValues, peakMemoryValues) } else { // For non-Photon apps, use the task metrics directly. - val peakMemoryValues = tasksInStage.map(_.peakExecutionMemory) - val shuffleWriteTime = tasksInStage.map(_.sw_writeTime) - (AppSparkMetricsAnalyzer.maxWithEmptyHandling(peakMemoryValues), - shuffleWriteTime.sum) + new AggAccumHelper() } - - val (durSum, durMax, durMin, durAvg) = AppSparkMetricsAnalyzer.getDurations(tasksInStage) + val perStageRec = accumHelperObj.accumPerStage(tasksInStage) val stageRow = StageAggTaskMetricsProfileResult(index, sm.stageInfo.stageId, - numAttempts, // TODO: why is this numAttempts and not numTasks? + // numTasks includes duplicate task attempts + perStageRec.numTasks, sm.duration, - tasksInStage.map(_.diskBytesSpilled).sum, - durSum, - durMax, - durMin, - durAvg, - tasksInStage.map(_.executorCPUTime).sum, - tasksInStage.map(_.executorDeserializeCPUTime).sum, - tasksInStage.map(_.executorDeserializeTime).sum, - tasksInStage.map(_.executorRunTime).sum, - tasksInStage.map(_.input_bytesRead).sum, - tasksInStage.map(_.input_recordsRead).sum, - tasksInStage.map(_.jvmGCTime).sum, - tasksInStage.map(_.memoryBytesSpilled).sum, - tasksInStage.map(_.output_bytesWritten).sum, - tasksInStage.map(_.output_recordsWritten).sum, - peakMemoryMax, - tasksInStage.map(_.resultSerializationTime).sum, - AppSparkMetricsAnalyzer.maxWithEmptyHandling(tasksInStage.map(_.resultSize)), - tasksInStage.map(_.sr_fetchWaitTime).sum, - tasksInStage.map(_.sr_localBlocksFetched).sum, - tasksInStage.map(_.sr_localBytesRead).sum, - tasksInStage.map(_.sr_remoteBlocksFetched).sum, - tasksInStage.map(_.sr_remoteBytesRead).sum, - tasksInStage.map(_.sr_remoteBytesReadToDisk).sum, - tasksInStage.map(_.sr_totalBytesRead).sum, - tasksInStage.map(_.sw_bytesWritten).sum, - tasksInStage.map(_.sw_recordsWritten).sum, - shuffleWriteTimeSum - ) + perStageRec.diskBytesSpilledSum, + perStageRec.durationSum, + perStageRec.durationMax, + perStageRec.durationMin, + perStageRec.durationAvg, + perStageRec.executorCPUTimeSum, + perStageRec.executorDeserializeCpuTimeSum, + perStageRec.executorDeserializeTimeSum, + perStageRec.executorRunTimeSum, + perStageRec.inputBytesReadSum, + perStageRec.inputRecordsReadSum, + perStageRec.jvmGCTimeSum, + perStageRec.memoryBytesSpilledSum, + perStageRec.outputBytesWrittenSum, + perStageRec.outputRecordsWrittenSum, + perStageRec.peakExecutionMemoryMax, + perStageRec.resultSerializationTimeSum, + perStageRec.resultSizeMax, + perStageRec.srFetchWaitTimeSum, + perStageRec.srLocalBlocksFetchedSum, + perStageRec.srLocalBytesReadSum, + perStageRec.srRemoteBlocksFetchSum, + perStageRec.srRemoteBytesReadSum, + perStageRec.srRemoteBytesReadToDiskSum, + perStageRec.srTotalBytesReadSum, + perStageRec.swBytesWrittenSum, + perStageRec.swRecordsWrittenSum, + perStageRec.swWriteTimeSum) stageLevelSparkMetrics(index).put(sm.stageInfo.stageId, stageRow) } } @@ -482,16 +459,6 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { object AppSparkMetricsAnalyzer { - def getDurations(tcs: Iterable[TaskModel]): (Long, Long, Long, Double) = { - val durations = tcs.map(_.duration) - if (durations.nonEmpty) { - (durations.sum, durations.max, durations.min, - ToolUtils.calculateAverage(durations.sum, durations.size, 1)) - } else { - (0L, 0L, 0L, 0.toDouble) - } - } - /** * Given an input iterable, returns its min, median, max and sum. */ @@ -509,20 +476,4 @@ object AppSparkMetricsAnalyzer { StatisticsMetrics(sortedArr.head, med, sortedArr(len - 1), sortedArr.sum) } } - - def maxWithEmptyHandling(arr: Iterable[Long]): Long = { - if (arr.isEmpty) { - 0L - } else { - arr.max - } - } - - def minWithEmptyHandling(arr: Iterable[Long]): Long = { - if (arr.isEmpty) { - 0L - } else { - arr.min - } - } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala new file mode 100644 index 000000000..b42ac08b4 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala @@ -0,0 +1,59 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +import com.nvidia.spark.rapids.tool.profiling.StageAggTaskMetricsProfileResult + +import org.apache.spark.sql.rapids.tool.store.TaskModel + +/** + * A helper class to facilitate the accumulation of aggregate metrics. + * This is a separate class to allow further customization in the future. For example, + * a parellel processor can be used to split the iterables without changing the caller side. + */ +class AggAccumHelper { + + private def accumCachedRecords[R <: TaskMetricsAccumRec]( + stageRecords: Iterable[StageAggTaskMetricsProfileResult], + rec: R): Unit = { + stageRecords.foreach(rec.addRecord) + rec.finalizeAggregation() + } + + protected def createStageAccumRecord(): TaskMetricsAccumRec = { + StageAggAccum() + } + + def accumPerStage(taskRecords: Iterable[TaskModel]): TaskMetricsAccumRec = { + val resRec = createStageAccumRecord() + taskRecords.foreach(resRec.addRecord) + resRec.finalizeAggregation() + resRec + } + + def accumPerSQL(stageRecords: Iterable[StageAggTaskMetricsProfileResult]): SQLAggAccum = { + val resRec = SQLAggAccum() + accumCachedRecords(stageRecords, resRec) + resRec + } + + def accumPerJob(stageRecords: Iterable[StageAggTaskMetricsProfileResult]): JobAggAccum = { + val resRec = JobAggAccum() + accumCachedRecords(stageRecords, resRec) + resRec + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumPhotonHelper.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumPhotonHelper.scala new file mode 100644 index 000000000..4f1356960 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumPhotonHelper.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +/** + * Implementation of AggAccumHelper for Photon. + * It takes the shuffleWriteValues and peakMemValues Accumulables as an argument because those + * values are not available in the TaskModel. + */ +class AggAccumPhotonHelper( + shuffleWriteValues: Iterable[Long], + peakMemValues: Iterable[Long]) extends AggAccumHelper { + + override def createStageAccumRecord(): TaskMetricsAccumRec = { + StageAggPhoton(shuffleWriteValues, peakMemValues) + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/JobAggAccum.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/JobAggAccum.scala new file mode 100644 index 000000000..a8e5b78db --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/JobAggAccum.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +import org.apache.spark.sql.rapids.tool.store.TaskModel + +/** + * Accumulator for Job Aggregates. + * This is an optimization to avoid using the Scala collections API on each field for the entire + * number of tasks/stages in a Job. + */ +case class JobAggAccum() extends TaskMetricsAccumRec { + override def addRecord(rec: TaskModel): Unit = { + throw new UnsupportedOperationException( + "Not implemented: JobAggAccum accepts only cached records") + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/SQLAggAccum.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/SQLAggAccum.scala new file mode 100644 index 000000000..b8222679f --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/SQLAggAccum.scala @@ -0,0 +1,42 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +import org.apache.spark.sql.rapids.tool.ToolUtils +import org.apache.spark.sql.rapids.tool.store.TaskModel + +/** + * Accumulator for SQL Aggregates. + * This is an optimization to avoid using the Scala collections API on each field for the entire + * number of tasks/stages in a SQL. + */ +case class SQLAggAccum( + var executorCpuRatio: Double = 0, + // Not added to the output since it is used only by the AutoTuner + var inputBytesReadAvg: Double = 0) extends TaskMetricsAccumRec { + + override def finalizeAggregation(): Unit = { + super.finalizeAggregation() + executorCpuRatio = ToolUtils.calculateDurationPercent(executorCPUTimeSum, executorRunTimeSum) + inputBytesReadAvg = ToolUtils.calculateAverage(inputBytesReadSum, numTasks, 1) + } + + override def addRecord(rec: TaskModel): Unit = { + throw new UnsupportedOperationException( + "Not implemented: SQLAggAccum accepts only cached records") + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggAccum.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggAccum.scala new file mode 100644 index 000000000..c88f1a77d --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggAccum.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +import com.nvidia.spark.rapids.tool.profiling.StageAggTaskMetricsProfileResult + +/** + * Accumulator for Stage Aggregates. + * This is an optimization to avoid using the Scala collections API on each field for the entire + * number of tasks in a Stage. + */ +case class StageAggAccum() extends TaskMetricsAccumRec { + override def addRecord(rec: StageAggTaskMetricsProfileResult): Unit = { + throw new UnsupportedOperationException("Not implemented: Cannot use cached results to" + + "calculate stage aggregates") + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala new file mode 100644 index 000000000..ed7127050 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +import java.util.concurrent.TimeUnit + +import com.nvidia.spark.rapids.tool.profiling.StageAggTaskMetricsProfileResult + +/** + * Implementation of Accumulator object for Photon. + * It takes the shuffleWriteValues and peakMemValues Accumulables as an argument because those + * values are not available in the TaskModel. + */ +case class StageAggPhoton( + shuffleWriteValues: Iterable[Long], + peakMemValues: Iterable[Long]) extends TaskMetricsAccumRec { + + override def addRecord(rec: StageAggTaskMetricsProfileResult): Unit = { + throw new UnsupportedOperationException("Not implemented: Cannot use cached results to" + + "calculate stage aggregates") + } + + override def finalizeAggregation(): Unit = { + // Fix the shuffleWriteTimes and the peakMemoryValues to use the shuffleWriteValues and + // the peakMemValues. + swWriteTimeSum = 0 + peakExecutionMemoryMax = 0 + if (!isEmptyAggregates) { + // Re-calculate the photon specific fields only if the accumulator has tasks. + // Otherwise, leave it as 0. + if (shuffleWriteValues.nonEmpty) { + swWriteTimeSum = TimeUnit.NANOSECONDS.toMillis(shuffleWriteValues.sum) + } + if (peakMemValues.nonEmpty) { + peakExecutionMemoryMax = peakMemValues.max + } + } + super.finalizeAggregation() + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/TaskMetricsAccumRec.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/TaskMetricsAccumRec.scala new file mode 100644 index 000000000..b5d98b9ac --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/TaskMetricsAccumRec.scala @@ -0,0 +1,157 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +import com.nvidia.spark.rapids.tool.profiling.StageAggTaskMetricsProfileResult + +import org.apache.spark.sql.rapids.tool.ToolUtils +import org.apache.spark.sql.rapids.tool.store.TaskModel + +/** + * Accumulator used for task metrics. + * This is an optimization decision to avoid using Scala builtin collections on every field in the + * taskModel. + */ +class TaskMetricsAccumRec { + var numTasks: Int = 0 + var diskBytesSpilledSum: Long = 0 + var durationSum: Long = 0 + var durationMax: Long = Long.MinValue + var durationMin: Long = Long.MaxValue + var durationAvg: Double = 0.0 + var executorCPUTimeSum: Long = 0 + var executorDeserializeCpuTimeSum: Long = 0 + var executorDeserializeTimeSum: Long = 0 + var executorRunTimeSum: Long = 0 + var inputBytesReadSum: Long = 0 + var inputRecordsReadSum: Long = 0 + var jvmGCTimeSum: Long = 0 + var memoryBytesSpilledSum: Long = 0 + var outputBytesWrittenSum: Long = 0 + var outputRecordsWrittenSum: Long = 0 + var peakExecutionMemoryMax: Long = Long.MinValue + var resultSerializationTimeSum: Long = 0 + var resultSizeMax: Long = Long.MinValue + var srFetchWaitTimeSum: Long = 0 + var srLocalBlocksFetchedSum: Long = 0 + var srLocalBytesReadSum: Long = 0 + var srRemoteBlocksFetchSum: Long = 0 + var srRemoteBytesReadSum: Long = 0 + var srRemoteBytesReadToDiskSum: Long = 0 + var srTotalBytesReadSum: Long = 0 + var swBytesWrittenSum: Long = 0 + var swRecordsWrittenSum: Long = 0 + var swWriteTimeSum: Long = 0 + + /** + * Assumption that 0-tasks implies no aggregations on metrics. This means that metrics on + * job/SQL levels won't be accumulated as long as no tasks are accounted for. + */ + def isEmptyAggregates: Boolean = numTasks == 0 + + /** + * Reset all fields to 0. This is used to reset the fields when the Task iterator is empty. + * When the iterator is empty, then fields such as "max" should be reset to 0. + */ + def resetFields(): Unit = { + durationMax = 0 + durationMin = 0 + peakExecutionMemoryMax = 0 + resultSizeMax = 0 + } + + def addRecord(rec: TaskModel): Unit = { + numTasks += 1 + // SumFields + diskBytesSpilledSum += rec.diskBytesSpilled + durationSum += rec.duration + executorCPUTimeSum += rec.executorCPUTime + executorDeserializeCpuTimeSum += rec.executorDeserializeCPUTime + executorDeserializeTimeSum += rec.executorDeserializeTime + executorRunTimeSum += rec.executorRunTime + inputBytesReadSum += rec.input_bytesRead + inputRecordsReadSum += rec.input_recordsRead + jvmGCTimeSum += rec.jvmGCTime + memoryBytesSpilledSum += rec.memoryBytesSpilled + outputBytesWrittenSum += rec.output_bytesWritten + outputRecordsWrittenSum += rec.output_recordsWritten + resultSerializationTimeSum += rec.resultSerializationTime + srFetchWaitTimeSum += rec.sr_fetchWaitTime + srLocalBlocksFetchedSum += rec.sr_localBlocksFetched + srLocalBytesReadSum += rec.sr_localBytesRead + srRemoteBlocksFetchSum += rec.sr_remoteBlocksFetched + srRemoteBytesReadSum += rec.sr_remoteBytesRead + srRemoteBytesReadToDiskSum += rec.sr_remoteBytesReadToDisk + srTotalBytesReadSum += rec.sr_totalBytesRead + swBytesWrittenSum += rec.sw_bytesWritten + swRecordsWrittenSum += rec.sw_recordsWritten + swWriteTimeSum += rec.sw_writeTime + // Max fields + durationMax = math.max(durationMax, rec.duration) + peakExecutionMemoryMax = math.max(peakExecutionMemoryMax, rec.peakExecutionMemory) + resultSizeMax = math.max(resultSizeMax, rec.resultSize) + // Min Fields + durationMin = math.min(durationMin, rec.duration) + } + + def addRecord(rec: StageAggTaskMetricsProfileResult): Unit = { + // Sums + numTasks += rec.numTasks + durationSum += rec.durationSum + diskBytesSpilledSum += rec.diskBytesSpilledSum + executorCPUTimeSum += rec.executorCPUTimeSum + executorRunTimeSum += rec.executorRunTimeSum + inputBytesReadSum += rec.inputBytesReadSum + executorDeserializeCpuTimeSum += rec.executorDeserializeCpuTimeSum + executorDeserializeTimeSum += rec.executorDeserializeTimeSum + inputRecordsReadSum += rec.inputRecordsReadSum + jvmGCTimeSum += rec.jvmGCTimeSum + memoryBytesSpilledSum += rec.memoryBytesSpilledSum + outputBytesWrittenSum += rec.outputBytesWrittenSum + outputRecordsWrittenSum += rec.outputRecordsWrittenSum + resultSerializationTimeSum += rec.resultSerializationTimeSum + srFetchWaitTimeSum += rec.srFetchWaitTimeSum + srLocalBlocksFetchedSum += rec.srLocalBlocksFetchedSum + srLocalBytesReadSum += rec.srcLocalBytesReadSum + srRemoteBlocksFetchSum += rec.srRemoteBlocksFetchSum + srRemoteBytesReadSum += rec.srRemoteBytesReadSum + srRemoteBytesReadToDiskSum += rec.srRemoteBytesReadToDiskSum + srTotalBytesReadSum += rec.srTotalBytesReadSum + swBytesWrittenSum += rec.swBytesWrittenSum + swRecordsWrittenSum += rec.swRecordsWrittenSum + swWriteTimeSum += rec.swWriteTimeSum + // Max + durationMax = math.max(durationMax, rec.durationMax) + peakExecutionMemoryMax = math.max(peakExecutionMemoryMax, rec.peakExecutionMemoryMax) + resultSizeMax = math.max(resultSizeMax, rec.resultSizeMax) + // Min + durationMin = math.min(durationMin, rec.durationMin) + } + + /** + * This method should be called to finalize the accumulations of all the metrics. + * For example, calculating averages and doing any last transformations on a field before the + * results are consumed. + */ + def finalizeAggregation(): Unit = { + durationAvg = ToolUtils.calculateAverage(durationSum, numTasks, 1) + if (numTasks < 1) { + // number of tasks is 0, then we should reset fields such as (max, min) to 0. + resetFields() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumMetaRef.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumMetaRef.scala index 7b70bedb2..35c9c19e1 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumMetaRef.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumMetaRef.scala @@ -27,6 +27,7 @@ case class AccumMetaRef(id: Long, name: AccumNameRef) { } object AccumMetaRef { + val EMPTY_ACCUM_META_REF: AccumMetaRef = new AccumMetaRef(0L, AccumNameRef.EMPTY_ACC_NAME_REF) def apply(id: Long, name: Option[String]): AccumMetaRef = new AccumMetaRef(id, AccumNameRef.getOrCreateAccumNameRef(name)) } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala index 0172f5229..4ce41e4a5 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala @@ -42,7 +42,7 @@ case class AccumNameRef(value: String) { object AccumNameRef { // Dummy AccNameRef to represent None accumulator names. This is an optimization to avoid // storing an option[string] for all accumulable names which leads to "get-or-else" everywhere. - private val EMPTY_ACC_NAME_REF: AccumNameRef = new AccumNameRef("N/A") + val EMPTY_ACC_NAME_REF: AccumNameRef = new AccumNameRef("N/A") // A global table to store reference to all accumulator names. The map is accessible by all // threads (different applications) running in parallel. This avoids duplicate work across // different threads. From 3db52ef3a4c56abce9d1d3337daa05ae176d9622 Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 19 Dec 2024 19:56:25 -0600 Subject: [PATCH 22/52] Improve implementation of finding median in StatisticsMetrics (#1474) * Improve implementation of finding median in StatisticsMetrics Signed-off-by: Ahmed Hussein (amahussein) Fixes #1461 Adds an InPlace median finding to improve the performance of the metric aggregates. We used to sort a sequence to create StatisticsMetrics which turned out to be very expensive in large eventlogs. Signed-off-by: Ahmed Hussein (amahussein) * remove empty lines Signed-off-by: Ahmed Hussein (amahussein) --------- Signed-off-by: Ahmed Hussein (amahussein) --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 82 ++++------ .../analysis/AppSparkMetricsAnalyzer.scala | 48 ++---- .../tool/analysis/StatisticsMetrics.scala | 29 ++++ .../sql/rapids/tool/store/AccumInfo.scala | 20 +-- .../tool/util/InPlaceMedianArrView.scala | 150 ++++++++++++++++++ .../rapids/tool/util/ToolUtilsSuite.scala | 25 ++- 6 files changed, 249 insertions(+), 105 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/sql/rapids/tool/util/InPlaceMedianArrView.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 9580aa470..7ca4bbb5b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -16,6 +16,7 @@ package com.nvidia.spark.rapids.tool.analysis +import scala.collection.breakOut import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} @@ -265,7 +266,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val jobsWithSQL = app.jobIdToInfo.filter { case (_, j) => j.sqlID.nonEmpty } - val sqlToStages = jobsWithSQL.flatMap { case (jobId, j) => + jobsWithSQL.flatMap { case (jobId, j) => val stages = j.stageIds val stagesInJob = app.stageManager.getStagesByIds(stages) stagesInJob.map { sModel => @@ -283,8 +284,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap SQLStageInfoProfileResult(appIndex, j.sqlID.get, jobId, sModel.stageInfo.stageId, sModel.stageInfo.attemptNumber(), sModel.duration, nodeNames) } - } - sqlToStages.toSeq + }(breakOut) } def generateSQLAccums(): Seq[SQLAccumProfileResults] = { @@ -294,20 +294,11 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val driverAccumsOpt = app.driverAccumMap.get(metric.accumulatorId) val driverMax = driverAccumsOpt match { case Some(accums) => - val filtered = accums.filter { a => - a.sqlID == metric.sqlID - } - val accumValues = filtered.map(_.value).sortWith(_ < _) - if (accumValues.isEmpty) { - None - } else if (accumValues.length <= 1) { - Some(StatisticsMetrics(0L, 0L, 0L, accumValues.sum)) - } else { - Some(StatisticsMetrics(accumValues(0), accumValues(accumValues.size / 2), - accumValues(accumValues.size - 1), accumValues.sum)) - } - case None => - None + StatisticsMetrics.createOptionalFromArr(accums.collect { + case a if a.sqlID == metric.sqlID => + a.value + }(breakOut)) + case _ => None } if (accumTaskStats.isDefined || driverMax.isDefined) { @@ -325,7 +316,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap } else { None } - } + }(breakOut) } /** @@ -341,40 +332,31 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap def generateStageLevelAccums(): Seq[AccumProfileResults] = { app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 - accumInfo.stageValuesMap.keySet.flatMap( stageId => { - val stageTaskIds = app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId).toSet - // get the task updates that belong to that stage - val taskUpatesSubset = - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds.contains).values.toSeq.sorted - if (taskUpatesSubset.isEmpty) { - None - } else { - val min = taskUpatesSubset.head - val max = taskUpatesSubset.last - val sum = taskUpatesSubset.sum - val median = if (taskUpatesSubset.size % 2 == 0) { - val mid = taskUpatesSubset.size / 2 - (taskUpatesSubset(mid) + taskUpatesSubset(mid - 1)) / 2 - } else { - taskUpatesSubset(taskUpatesSubset.size / 2) - } - // reuse AccumProfileResults to avoid generating extra memory from allocating new objects - val accumProfileResults = AccumProfileResults( - appIndex, - stageId, - accumInfo.infoRef, - min = min, - median = median, - max = max, - total = sum) - if (accumInfo.infoRef.name.isDiagnosticMetrics()) { - updateStageDiagnosticMetrics(accumProfileResults) - } - Some(accumProfileResults) + accumInfo.stageValuesMap.keys.flatMap( stageId => { + val stageTaskIds: Set[Long] = + app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId)(breakOut) + // Get the task updates that belong to that stage + StatisticsMetrics.createOptionalFromArr( + accumInfo.taskUpdatesMap.filterKeys(stageTaskIds).map(_._2)(breakOut)) match { + case Some(stat) => + // Reuse AccumProfileResults to avoid generating allocating new objects + val accumProfileResults = AccumProfileResults( + appIndex, + stageId, + accumInfo.infoRef, + min = stat.min, + median = stat.med, + max = stat.max, + total = stat.total) + if (accumInfo.infoRef.name.isDiagnosticMetrics()) { + updateStageDiagnosticMetrics(accumProfileResults) + } + Some(accumProfileResults) + case _ => None } }) - } - }.toSeq + }(breakOut) + } } object AppSQLPlanAnalyzer { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 3a862097b..6b8c3d5e5 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -16,6 +16,7 @@ package com.nvidia.spark.rapids.tool.analysis +import scala.collection.breakOut import scala.collection.mutable.{ArrayBuffer, HashMap, LinkedHashMap} import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics._ @@ -79,7 +80,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { * @return sequence of JobAggTaskMetricsProfileResult that contains only Job Ids */ def aggregateSparkMetricsByJob(index: Int): Seq[JobAggTaskMetricsProfileResult] = { - val jobRows = app.jobIdToInfo.flatMap { case (id, jc) => + app.jobIdToInfo.flatMap { case (id, jc) => if (jc.stageIds.isEmpty) { None } else { @@ -126,8 +127,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { perJobRec.swWriteTimeSum)) } } - } - jobRows.toSeq + }(breakOut) } private case class AverageStageInfo(avgDuration: Double, avgShuffleReadBytes: Double) @@ -163,7 +163,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { tc.taskId, tc.attempt, tc.duration, avg.avgDuration, tc.sr_totalBytesRead, avg.avgShuffleReadBytes, tc.peakExecutionMemory, tc.successful, tc.endReason) } - }.toSeq + }(breakOut) } /** @@ -172,7 +172,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { * @return sequence of SQLTaskAggMetricsProfileResult */ def aggregateSparkMetricsBySql(index: Int): Seq[SQLTaskAggMetricsProfileResult] = { - val sqlRows = app.sqlIdToInfo.flatMap { case (sqlId, sqlCase) => + app.sqlIdToInfo.flatMap { case (sqlId, sqlCase) => if (app.sqlIdToStages.contains(sqlId)) { val stagesInSQL = app.sqlIdToStages(sqlId) // TODO: Should we only consider successful tasks? @@ -229,8 +229,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { } else { None } - } - sqlRows.toSeq + }(breakOut) } /** @@ -241,7 +240,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { */ def aggregateIOMetricsBySql( sqlMetricsAggs: Seq[SQLTaskAggMetricsProfileResult]): Seq[IOAnalysisProfileResult] = { - val sqlIORows = sqlMetricsAggs.map { sqlAgg => + sqlMetricsAggs.map { sqlAgg => IOAnalysisProfileResult(sqlAgg.appIndex, app.appId, sqlAgg.sqlId, @@ -253,8 +252,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { sqlAgg.memoryBytesSpilledSum, sqlAgg.srTotalBytesReadSum, sqlAgg.swBytesWrittenSum) - } - sqlIORows + }(breakOut) } /** @@ -289,7 +287,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { * @return a sequence of SQLDurationExecutorTimeProfileResult or Empty if None. */ def aggregateDurationAndCPUTimeBySql(index: Int): Seq[SQLDurationExecutorTimeProfileResult] = { - val sqlRows = app.sqlIdToInfo.map { case (sqlId, sqlCase) => + app.sqlIdToInfo.map { case (sqlId, sqlCase) => // First, build the SQLIssues string by retrieving the potential issues from the // app.sqlIDtoProblematic map. val sqlIssues = if (app.sqlIDtoProblematic.contains(sqlId)) { @@ -301,8 +299,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { SQLDurationExecutorTimeProfileResult(index, app.appId, sqlCase.rootExecutionID, sqlId, sqlCase.duration, sqlCase.hasDatasetOrRDD, app.getAppDuration.orElse(Option(0L)), sqlIssues, sqlCase.sqlCpuTimePercent) - } - sqlRows.toSeq + }(breakOut) } /** @@ -338,7 +335,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { .getOrElse(sm.stageInfo.stageId, emptyDiagnosticMetrics) .withDefaultValue(zeroAccumProfileResults) val srTotalBytesMetrics = - AppSparkMetricsAnalyzer.getStatistics(tasksInStage.map(_.sr_totalBytesRead)) + StatisticsMetrics.createFromArr(tasksInStage.map(_.sr_totalBytesRead)(breakOut)) StageDiagnosticResult(index, app.getAppName, @@ -359,7 +356,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { diagnosticMetricsMap(SW_WRITE_TIME_METRIC), diagnosticMetricsMap(GPU_SEMAPHORE_WAIT_METRIC), nodeNames) - }.toSeq + }(breakOut) } /** @@ -456,24 +453,3 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { } } } - - -object AppSparkMetricsAnalyzer { - /** - * Given an input iterable, returns its min, median, max and sum. - */ - def getStatistics(arr: Iterable[Long]): StatisticsMetrics = { - if (arr.isEmpty) { - StatisticsMetrics(0L, 0L, 0L, 0L) - } else { - val sortedArr = arr.toSeq.sorted - val len = sortedArr.size - val med = if (len % 2 == 0) { - (sortedArr(len / 2) + sortedArr(len / 2 - 1)) / 2 - } else { - sortedArr(len / 2) - } - StatisticsMetrics(sortedArr.head, med, sortedArr(len - 1), sortedArr.sum) - } - } -} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/StatisticsMetrics.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/StatisticsMetrics.scala index 1b88d2d4c..d0a21a6c0 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/StatisticsMetrics.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/StatisticsMetrics.scala @@ -16,6 +16,8 @@ package com.nvidia.spark.rapids.tool.analysis +import org.apache.spark.sql.rapids.tool.util.InPlaceMedianArrView.{chooseMidpointPivotInPlace, findMedianInPlace} + // Store (min, median, max, total) for a given metric case class StatisticsMetrics(min: Long, med: Long, max: Long, total: Long) @@ -23,4 +25,31 @@ object StatisticsMetrics { // a static variable used to represent zero-statistics instead of allocating a dummy record // on every calculation. val ZERO_RECORD: StatisticsMetrics = StatisticsMetrics(0L, 0L, 0L, 0L) + + def createFromArr(arr: Array[Long]): StatisticsMetrics = { + if (arr.isEmpty) { + return ZERO_RECORD + } + val medV = findMedianInPlace(arr)(chooseMidpointPivotInPlace) + var minV = Long.MaxValue + var maxV = Long.MinValue + var totalV = 0L + arr.foreach { v => + if (v < minV) { + minV = v + } + if (v > maxV) { + maxV = v + } + totalV += v + } + StatisticsMetrics(minV, medV, maxV, totalV) + } + + def createOptionalFromArr(arr: Array[Long]): Option[StatisticsMetrics] = { + if (arr.isEmpty) { + return None + } + Some(createFromArr(arr)) + } } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumInfo.scala index 0f8e520c6..080a34df3 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumInfo.scala @@ -16,7 +16,7 @@ package org.apache.spark.sql.rapids.tool.store -import scala.collection.mutable +import scala.collection.{breakOut, mutable} import com.nvidia.spark.rapids.tool.analysis.StatisticsMetrics @@ -98,22 +98,8 @@ class AccumInfo(val infoRef: AccumMetaRef) { } def calculateAccStats(): StatisticsMetrics = { - val sortedTaskUpdates = taskUpdatesMap.values.toSeq.sorted - if (sortedTaskUpdates.isEmpty) { - // do not check stage values because the stats is only meant for task updates - StatisticsMetrics.ZERO_RECORD - } else { - val min = sortedTaskUpdates.head - val max = sortedTaskUpdates.last - val sum = sortedTaskUpdates.sum - val median = if (sortedTaskUpdates.size % 2 == 0) { - val mid = sortedTaskUpdates.size / 2 - (sortedTaskUpdates(mid) + sortedTaskUpdates(mid - 1)) / 2 - } else { - sortedTaskUpdates(sortedTaskUpdates.size / 2) - } - StatisticsMetrics(min, median, max, sum) - } + // do not check stage values because the stats is only meant for task updates + StatisticsMetrics.createFromArr(taskUpdatesMap.map(_._2)(breakOut)) } def getMaxStageValue: Option[Long] = { diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/InPlaceMedianArrView.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/InPlaceMedianArrView.scala new file mode 100644 index 000000000..1be48a6a7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/InPlaceMedianArrView.scala @@ -0,0 +1,150 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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.spark.sql.rapids.tool.util + +import scala.annotation.tailrec +import scala.language.postfixOps + +/** + * Allows for in-place partitioning and finding the median. + * The tools used to find the median of a sequence by sorting the entire sequence, then returning + * the elements in the middle. As we started to capture all the accumulators in Spark plans, + * sorting is inefficient for large eventlogs that contain huge number of tasks and + * Accumulables. Thus, this class is an optimized version to get the median in a linear + * complexity while doing it in place to avoid allocating new array to store the sorted elements. + * The code is copied from a Stackoverflow thread: + * https://stackoverflow.com/questions/4662292/scala-median-implementation + * + * Notes: + * - The implementation assumes that the array is not empty. + */ +case class InPlaceMedianArrView(arr: Array[Long], from: Int, until: Int) { + def apply(n: Int): Long = { + if (from + n < until) { + arr(from + n) + } else { + throw new ArrayIndexOutOfBoundsException(n) + } + } + + /** + * Returns a new view of the array with the same elements but a different range. + * @param p a predicate to apply on the elements to proceed with the partitioning. + * @return a tuple of 2 views, the first one contains the elements that satisfy the predicate, + * and the second one contains the rest. + */ + def partitionInPlace(p: Long => Boolean): (InPlaceMedianArrView, InPlaceMedianArrView) = { + var upper = until - 1 + var lower = from + while (lower < upper) { + while (lower < until && p(arr(lower))) lower += 1 + while (upper >= from && !p(arr(upper))) upper -= 1 + if (lower < upper) { val tmp = arr(lower); arr(lower) = arr(upper); arr(upper) = tmp } + } + (copy(until = lower), copy(from = lower)) + } + + def size: Int = { + until - from + } + + def isEmpty: Boolean = { + size <= 0 + } + + override def toString = { + arr mkString ("ArraySize(", ", ", ")") + } +} + +/** + * Companion object for InPlaceMedianArrView. + */ +object InPlaceMedianArrView { + + def apply(arr: Array[Long]): InPlaceMedianArrView = { + InPlaceMedianArrView(arr, 0, arr.size) + } + + /** + * Finds the median of the array in place. + * @param arr the Array[Long] to be processed + * @param k the index of the median + * @param choosePivot a function to choose the pivot index. This useful to choose different + * strategies. For example, choosing the midpoint works better for sorted + * arrays. + * @return the median of the array. + */ + @tailrec + def findKMedianInPlace(arr: InPlaceMedianArrView, k: Int) + (implicit choosePivot: InPlaceMedianArrView => Long): Long = { + val a = choosePivot(arr) + val (s, b) = arr partitionInPlace (a >) + if (s.size == k) { + a + } else if (s.isEmpty) { + val (s, b) = arr partitionInPlace (a ==) + if (s.size > k) { + a + } else { + findKMedianInPlace(b, k - s.size) + } + } else if (s.size < k) { + findKMedianInPlace(b, k - s.size) + } else { + findKMedianInPlace(s, k) + } + } + + /** + * Choose a random pivot in the array. This can lead to worst case for sorted arrays. + * @param arr the array to choose the pivot from. + * @return a random element from the array. + */ + def chooseRandomPivotInPlace(arr: InPlaceMedianArrView): Long = { + arr(scala.util.Random.nextInt(arr.size)) + } + + /** + * Choose the element in the middle as a pivot. This works better to find median of sorted arrays. + * @param arr the array to choose the pivot from. + * @return the element in the middle of the array. + */ + def chooseMidpointPivotInPlace(arr: InPlaceMedianArrView): Long = { + arr((arr.size - 1) / 2) + } + + /** + * Finds the median of the array in place. + * @param arr the Array[Long] to be processed. + * @param choosePivot a function to choose the pivot index. + * @return the median of the array. + */ + def findMedianInPlace( + arr: Array[Long])(implicit choosePivot: InPlaceMedianArrView => Long): Long = { + val midIndex = (arr.size - 1) / 2 + if (arr.size % 2 == 0) { + // For even-length arrays, find the two middle elements and compute their average + val mid1 = findKMedianInPlace(InPlaceMedianArrView(arr), midIndex) + val mid2 = findKMedianInPlace(InPlaceMedianArrView(arr), midIndex + 1) + (mid1 + mid2) / 2 + } else { + // For odd-length arrays, return the middle element + findKMedianInPlace(InPlaceMedianArrView(arr), midIndex) + } + } +} diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/util/ToolUtilsSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/util/ToolUtilsSuite.scala index baba6eb79..5e1b6558b 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/util/ToolUtilsSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/util/ToolUtilsSuite.scala @@ -24,13 +24,13 @@ import scala.concurrent.duration._ import scala.xml.XML import com.nvidia.spark.rapids.tool.profiling.{ProfileOutputWriter, ProfileResult} +import org.scalatest.AppendedClues.convertToClueful import org.scalatest.FunSuite import org.scalatest.Matchers.{contain, convertToAnyShouldWrapper, equal, not} import org.apache.spark.internal.Logging import org.apache.spark.sql.TrampolineUtil -import org.apache.spark.sql.rapids.tool.util.{FSUtils, RapidsToolsConfUtil, StringUtils, WebCrawlerUtil} - +import org.apache.spark.sql.rapids.tool.util.{FSUtils, InPlaceMedianArrView, RapidsToolsConfUtil, StringUtils, WebCrawlerUtil} class ToolUtilsSuite extends FunSuite with Logging { test("get page links of a url") { @@ -210,6 +210,27 @@ class ToolUtilsSuite extends FunSuite with Logging { } } + test("Finding median of arrays") { + val testSet: Map[String, (Array[Long], Long)] = Map( + "All same values" -> (Array[Long](5, 5, 5, 5) -> 5L), + "Odd number of values [9, 7, 5, 3, 1]" -> (Array[Long](9, 7, 5, 3, 1) -> 5L), + "Even number of values [11, 9, 7, 5, 3, 1]" -> (Array[Long](11, 9, 7, 5, 3, 1) -> 6), + "Even number of values(2) [15, 13, 11, 9, 7, 5, 3, 1]" -> + (Array[Long](15, 13, 11, 9, 7, 5, 3, 1) -> 8), + "Even number of values(3) [3, 13, 11, 9, 7, 5, 15, 1]" -> + (Array[Long](3, 13, 11, 9, 7, 5, 15, 1) -> 8), + "Single element" -> (Array[Long](1) -> 1), + "Two elements" -> (Array[Long](1, 2).reverse -> 1) + ) + for ((desc, (arr, expectedMedian)) <- testSet) { + val actualMedian = + InPlaceMedianArrView.findMedianInPlace(arr)(InPlaceMedianArrView.chooseMidpointPivotInPlace) + actualMedian shouldBe expectedMedian withClue s"Failed for $desc. " + + s"Expected: $expectedMedian, " + + s"Actual: $actualMedian" + } + } + case class MockProfileResults(appID: String, appIndex: Int, nonEnglishField: String, parentIDs: String) extends ProfileResult { override val outputHeaders: Seq[String] = Seq("appID", "appIndex", "nonEnglishField", From f0058c015eaca8cb9c891eed0ae4ffb513c6bdf0 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Fri, 20 Dec 2024 09:48:12 -0800 Subject: [PATCH 23/52] Follow Up: Make '--platform' argument mandatory in CLI (#1473) * Followup: Make '--platform' argument mandatory in CLI Signed-off-by: Partho Sarthi * Add comments Signed-off-by: Partho Sarthi --------- Signed-off-by: Partho Sarthi --- .../src/spark_rapids_tools/cmdli/tools_cli.py | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py b/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py index 0d46e5025..e50659b46 100644 --- a/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py +++ b/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py @@ -35,9 +35,10 @@ class ToolsCLI(object): # pylint: disable=too-few-public-methods """ def qualification(self, + *, # force named arguments + platform: str, eventlogs: str = None, cluster: str = None, - platform: str = None, output_folder: str = None, filter_apps: str = None, custom_model_file: str = None, @@ -55,6 +56,8 @@ def qualification(self, The cmd will process each app individually, but will group apps with the same name into the same output row after averaging duration metrics accordingly. + :param platform: Defines one of the following: "onprem", "emr", "dataproc", "dataproc-gke", + "databricks-aws", and "databricks-azure". :param eventlogs: Event log filenames or CSP storage directories containing event logs (comma separated). @@ -62,8 +65,6 @@ def qualification(self, cluster name on the CSP. :param cluster: The CPU cluster on which the Spark application(s) were executed. Name or ID (for databricks platforms) of cluster or path to cluster-properties. - :param platform: Defines one of the following: "onprem", "emr", "dataproc", "dataproc-gke", - "databricks-aws", and "databricks-azure". :param output_folder: Local path to store the output. :param tools_jar: Path to a bundled jar including Rapids tool. The path is a local filesystem, or remote cloud storage url. If missing, the wrapper downloads the latest rapids-4-spark-tools_*.jar @@ -89,8 +90,8 @@ def qualification(self, For more details on Qualification tool options, please visit https://docs.nvidia.com/spark-rapids/user-guide/latest/qualification/jar-usage.html#running-the-qualification-tool-standalone-on-spark-event-logs """ - eventlogs = Utils.get_value_or_pop(eventlogs, rapids_options, 'e') platform = Utils.get_value_or_pop(platform, rapids_options, 'p') + eventlogs = Utils.get_value_or_pop(eventlogs, rapids_options, 'e') tools_jar = Utils.get_value_or_pop(tools_jar, rapids_options, 't') output_folder = Utils.get_value_or_pop(output_folder, rapids_options, 'o') filter_apps = Utils.get_value_or_pop(filter_apps, rapids_options, 'f') @@ -108,9 +109,9 @@ def qualification(self, if estimation_model_args is None: return None qual_args = AbsToolUserArgModel.create_tool_args('qualification', + platform=platform, eventlogs=eventlogs, cluster=cluster, - platform=platform, output_folder=output_folder, tools_jar=tools_jar, jvm_heap_size=jvm_heap_size, @@ -127,9 +128,10 @@ def qualification(self, return None def profiling(self, + *, # force named arguments + platform: str, eventlogs: str = None, cluster: str = None, - platform: str = None, driverlog: str = None, output_folder: str = None, tools_jar: str = None, @@ -146,14 +148,14 @@ def profiling(self, The tool also will recommend setting for the application assuming that the job will be able to use all the cluster resources (CPU and GPU) when it is running. + :param platform: defines one of the following "onprem", "emr", "dataproc", "databricks-aws", + and "databricks-azure". :param eventlogs: Event log filenames or cloud storage directories containing event logs (comma separated). If missing, the wrapper reads the Spark's property `spark.eventLog.dir` defined in the `cluster`. :param cluster: The cluster on which the Spark applications were executed. The argument can be a cluster name or ID (for databricks platforms) or a valid path to the cluster's properties file (json format) generated by the CSP SDK. - :param platform: defines one of the following "onprem", "emr", "dataproc", "databricks-aws", - and "databricks-azure". :param driverlog: Valid path to the GPU driver log file. :param output_folder: path to store the output. :param tools_jar: Path to a bundled jar including Rapids tool. The path is a local filesystem, @@ -173,9 +175,9 @@ def profiling(self, For more details on Profiling tool options, please visit https://docs.nvidia.com/spark-rapids/user-guide/latest/profiling/jar-usage.html#prof-tool-title-options """ + platform = Utils.get_value_or_pop(platform, rapids_options, 'p') eventlogs = Utils.get_value_or_pop(eventlogs, rapids_options, 'e') cluster = Utils.get_value_or_pop(cluster, rapids_options, 'c') - platform = Utils.get_value_or_pop(platform, rapids_options, 'p') driverlog = Utils.get_value_or_pop(driverlog, rapids_options, 'd') output_folder = Utils.get_value_or_pop(output_folder, rapids_options, 'o') tools_jar = Utils.get_value_or_pop(tools_jar, rapids_options, 't') @@ -184,9 +186,9 @@ def profiling(self, ToolLogging.enable_debug_mode() init_environment('prof') prof_args = AbsToolUserArgModel.create_tool_args('profiling', + platform=platform, eventlogs=eventlogs, cluster=cluster, - platform=platform, driverlog=driverlog, jvm_heap_size=jvm_heap_size, jvm_threads=jvm_threads, From 7308c12ab64c5016cbbe25ec8fcdcd4a153e6a8a Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Fri, 20 Dec 2024 09:49:42 -0800 Subject: [PATCH 24/52] Skip processing apps with invalid platform and spark runtime configurations (#1421) * Add platform specific runtime check Signed-off-by: Partho Sarthi * Refactor comments Signed-off-by: Partho Sarthi * Update behavior to fail on unsupported Spark Runtime Signed-off-by: Partho Sarthi * Fix trailing comma Signed-off-by: Partho Sarthi --------- Signed-off-by: Partho Sarthi --- .../nvidia/spark/rapids/tool/Platform.scala | 19 +++++- .../rapids/tool/profiling/Profiler.scala | 8 ++- .../spark/sql/rapids/tool/AppBase.scala | 20 ++++++- .../spark/sql/rapids/tool/ToolUtils.scala | 9 ++- .../tool/profiling/ApplicationInfo.scala | 7 ++- .../qualification/QualificationAppInfo.scala | 2 +- .../spark/rapids/tool/ToolTestUtils.scala | 5 +- .../tool/planparser/BasePlanParserSuite.scala | 7 ++- .../planparser/PhotonPlanParserSuite.scala | 3 +- .../rapids/tool/profiling/AnalysisSuite.scala | 20 ++++--- .../tool/profiling/ApplicationInfoSuite.scala | 60 +++++++++++++++---- .../qualification/QualificationSuite.scala | 8 ++- .../features/event_log_processing.feature | 14 +++-- 13 files changed, 140 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala index 5ac20b05e..39429085e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala @@ -22,7 +22,7 @@ import com.nvidia.spark.rapids.tool.profiling.ClusterProperties import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.tool.{ExistingClusterInfo, RecommendedClusterInfo} -import org.apache.spark.sql.rapids.tool.util.StringUtils +import org.apache.spark.sql.rapids.tool.util.{SparkRuntime, StringUtils} /** * Utility object containing constants for various platform names. @@ -132,6 +132,19 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], var recommendedClusterInfo: Option[RecommendedClusterInfo] = None // the number of GPUs to use, this might be updated as we handle different cases var numGpus: Int = 1 + // Default runtime for the platform + val defaultRuntime: SparkRuntime.SparkRuntime = SparkRuntime.SPARK + // Set of supported runtimes for the platform + protected val supportedRuntimes: Set[SparkRuntime.SparkRuntime] = Set( + SparkRuntime.SPARK, SparkRuntime.SPARK_RAPIDS + ) + + /** + * Checks if the given runtime is supported by the platform. + */ + def isRuntimeSupported(runtime: SparkRuntime.SparkRuntime): Boolean = { + supportedRuntimes.contains(runtime) + } // This function allow us to have one gpu type used by the auto // tuner recommendations but have a different GPU used for speedup @@ -511,6 +524,10 @@ abstract class DatabricksPlatform(gpuDevice: Option[GpuDevice], override val defaultGpuDevice: GpuDevice = T4Gpu override def isPlatformCSP: Boolean = true + override val supportedRuntimes: Set[SparkRuntime.SparkRuntime] = Set( + SparkRuntime.SPARK, SparkRuntime.SPARK_RAPIDS, SparkRuntime.PHOTON + ) + // note that Databricks generally sets the spark.executor.memory for the user. Our // auto tuner heuristics generally sets it lower then Databricks so go ahead and // allow our auto tuner to take affect for this in anticipation that we will use more diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 75b4c4590..5f92cf32d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal -import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, EventLogInfo, EventLogPathProcessor, FailedEventLog, PlatformFactory, ToolBase} +import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, EventLogInfo, EventLogPathProcessor, FailedEventLog, Platform, PlatformFactory, ToolBase} import com.nvidia.spark.rapids.tool.profiling.AutoTuner.loadClusterProps import com.nvidia.spark.rapids.tool.views._ import org.apache.hadoop.conf.Configuration @@ -43,6 +43,8 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea private val outputCombined: Boolean = appArgs.combined() private val useAutoTuner: Boolean = appArgs.autoTuner() private val outputAlignedSQLIds: Boolean = appArgs.outputSqlIdsAligned() + // Unlike qualification tool, profiler tool does not require platform per app + private val platform: Platform = PlatformFactory.createInstance(appArgs.platform()) override def getNumThreads: Int = appArgs.numThreads.getOrElse( Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) @@ -295,9 +297,9 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea private def createApp(path: EventLogInfo, index: Int, hadoopConf: Configuration): Either[FailureApp, ApplicationInfo] = { try { - // This apps only contains 1 app in each loop. + // These apps only contains 1 app in each loop. val startTime = System.currentTimeMillis() - val app = new ApplicationInfo(hadoopConf, path, index) + val app = new ApplicationInfo(hadoopConf, path, index, platform) EventLogPathProcessor.logApplicationInfo(app) val endTime = System.currentTimeMillis() if (!app.isAppMetaDefined) { diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index b90917cd8..971a8711f 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -23,7 +23,7 @@ import scala.collection.immutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, LinkedHashSet, Map} import com.nvidia.spark.rapids.SparkRapidsBuildInfoEvent -import com.nvidia.spark.rapids.tool.{DatabricksEventLog, DatabricksRollingEventLogFilesFileReader, EventLogInfo} +import com.nvidia.spark.rapids.tool.{DatabricksEventLog, DatabricksRollingEventLogFilesFileReader, EventLogInfo, Platform} import com.nvidia.spark.rapids.tool.planparser.{HiveParseHelper, ReadParser} import com.nvidia.spark.rapids.tool.planparser.HiveParseHelper.isHiveTableScanNode import com.nvidia.spark.rapids.tool.profiling.{BlockManagerRemovedCase, DriverAccumCase, JobInfoClass, ResourceProfileInfoCase, SQLExecutionInfoClass, SQLPlanMetricsCase} @@ -42,7 +42,8 @@ import org.apache.spark.util.Utils abstract class AppBase( val eventLogInfo: Option[EventLogInfo], - val hadoopConf: Option[Configuration]) extends Logging + val hadoopConf: Option[Configuration], + val platform: Option[Platform] = None) extends Logging with ClusterTagPropHandler with AccumToStageRetriever { @@ -481,6 +482,7 @@ abstract class AppBase( protected def postCompletion(): Unit = { registerAttemptId() calculateAppDuration() + validateSparkRuntime() } /** @@ -491,6 +493,20 @@ abstract class AppBase( processEventsInternal() postCompletion() } + + /** + * Validates if the spark runtime (parsed from event log) is supported by the platform. + * If the runtime is not supported, an `UnsupportedSparkRuntimeException` + * is thrown. + */ + private def validateSparkRuntime(): Unit = { + val parsedRuntime = getSparkRuntime + platform.foreach { p => + require(p.isRuntimeSupported(parsedRuntime), + throw UnsupportedSparkRuntimeException(p, parsedRuntime) + ) + } + } } object AppBase { diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala index 021337495..455f19147 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.rapids.tool import scala.collection.mutable import scala.util.{Failure, Success, Try} +import com.nvidia.spark.rapids.tool.Platform import com.nvidia.spark.rapids.tool.planparser.SubqueryExecParser import com.nvidia.spark.rapids.tool.profiling.ProfileUtils.replaceDelimiter import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter @@ -28,7 +29,7 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphNode} -import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph +import org.apache.spark.sql.rapids.tool.util.{SparkRuntime, ToolsPlanGraph} object ToolUtils extends Logging { // List of recommended file-encodings on the GPUs. @@ -441,6 +442,12 @@ case class UnsupportedMetricNameException(metricName: String) extends AppEventlogProcessException( s"Unsupported metric name found in the event log: $metricName") +case class UnsupportedSparkRuntimeException( + platform: Platform, + sparkRuntime: SparkRuntime.SparkRuntime) + extends AppEventlogProcessException( + s"Platform '${platform.platformName}' does not support the runtime '$sparkRuntime'") + // Class used a container to hold the information of the Tuple // to simplify arguments of methods and caching. case class SqlPlanInfoGraphEntry( diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala index 83a3cbc0b..6fbf2bb68 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.rapids.tool.profiling import scala.collection.Map -import com.nvidia.spark.rapids.tool.EventLogInfo +import com.nvidia.spark.rapids.tool.{EventLogInfo, Platform, PlatformFactory} import com.nvidia.spark.rapids.tool.analysis.AppSQLPlanAnalyzer import org.apache.hadoop.conf.Configuration @@ -184,8 +184,9 @@ object SparkPlanInfoWithStage { class ApplicationInfo( hadoopConf: Configuration, eLogInfo: EventLogInfo, - val index: Int) - extends AppBase(Some(eLogInfo), Some(hadoopConf)) with Logging { + val index: Int, + platform: Platform = PlatformFactory.createInstance()) + extends AppBase(Some(eLogInfo), Some(hadoopConf), Some(platform)) with Logging { private lazy val eventProcessor = new EventsProcessor(this) diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index 1ef8b7315..e3c33203f 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -41,7 +41,7 @@ class QualificationAppInfo( mlOpsEnabled: Boolean = false, penalizeTransitions: Boolean = true, platform: Platform) - extends AppBase(eventLogInfo, hadoopConf) with Logging { + extends AppBase(eventLogInfo, hadoopConf, Some(platform)) with Logging { var lastJobEndTime: Option[Long] = None var lastSQLEndTime: Option[Long] = None diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/ToolTestUtils.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/ToolTestUtils.scala index bd5e7bf25..011e5010e 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/ToolTestUtils.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/ToolTestUtils.scala @@ -144,12 +144,13 @@ object ToolTestUtils extends Logging { val apps: ArrayBuffer[ApplicationInfo] = ArrayBuffer[ApplicationInfo]() val appArgs = new ProfileArgs(logs) var index: Int = 1 + val platform = PlatformFactory.createInstance(appArgs.platform()) for (path <- appArgs.eventlog()) { val eventLogInfo = EventLogPathProcessor .getEventLogInfo(path, RapidsToolsConfUtil.newHadoopConf()) - assert(eventLogInfo.size >= 1, s"event log not parsed as expected $path") + assert(eventLogInfo.nonEmpty, s"event log not parsed as expected $path") apps += new ApplicationInfo(RapidsToolsConfUtil.newHadoopConf(), - eventLogInfo.head._1, index) + eventLogInfo.head._1, index, platform) index += 1 } apps diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala index b7966d4d2..6fe3cd2cd 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/BasePlanParserSuite.scala @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids.tool.planparser import com.nvidia.spark.rapids.BaseTestSuite -import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, PlatformFactory, ToolTestUtils} +import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, PlatformFactory, PlatformNames, ToolTestUtils} import com.nvidia.spark.rapids.tool.qualification._ import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo @@ -59,7 +59,8 @@ class BasePlanParserSuite extends BaseTestSuite { } } - def createAppFromEventlog(eventLog: String): QualificationAppInfo = { + def createAppFromEventlog(eventLog: String, + platformName: String = PlatformNames.DEFAULT): QualificationAppInfo = { val hadoopConf = RapidsToolsConfUtil.newHadoopConf() val (_, allEventLogs) = EventLogPathProcessor.processAllPaths( None, None, List(eventLog), hadoopConf) @@ -67,7 +68,7 @@ class BasePlanParserSuite extends BaseTestSuite { assert(allEventLogs.size == 1) val appResult = QualificationAppInfo.createApp(allEventLogs.head, hadoopConf, pluginTypeChecker, reportSqlLevel = false, mlOpsEnabled = false, penalizeTransitions = true, - PlatformFactory.createInstance()) + PlatformFactory.createInstance(platformName)) appResult match { case Right(app) => app case Left(_) => throw new AssertionError("Cannot create application") diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/PhotonPlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/PhotonPlanParserSuite.scala index edf8095bc..74f237178 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/PhotonPlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/PhotonPlanParserSuite.scala @@ -16,6 +16,7 @@ package com.nvidia.spark.rapids.tool.planparser +import com.nvidia.spark.rapids.tool.PlatformNames import com.nvidia.spark.rapids.tool.qualification.PluginTypeChecker @@ -34,7 +35,7 @@ class PhotonPlanParserSuite extends BasePlanParserSuite { test(s"$photonName is parsed as Spark $sparkName") { val eventLog = s"$qualLogDir/nds_q88_photon_db_13_3.zstd" val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) + val app = createAppFromEventlog(eventLog, platformName = PlatformNames.DATABRICKS_AWS) assert(app.sqlPlans.nonEmpty) val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index 2b8c3bf12..b7d8b315f 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.tool.profiling import java.io.File -import com.nvidia.spark.rapids.tool.ToolTestUtils +import com.nvidia.spark.rapids.tool.{PlatformNames, ToolTestUtils} import com.nvidia.spark.rapids.tool.views.{ProfDataSourceView, RawMetricProfilerView} import org.scalatest.FunSuite @@ -139,7 +139,8 @@ class AnalysisSuite extends FunSuite { s"${fileName}_${metric}_metrics_agg_expectation.csv" } testSqlMetricsAggregation(Array(s"${qualLogDir}/${fileName}.zstd"), - expectFile("sql"), expectFile("job"), expectFile("stage")) + expectFile("sql"), expectFile("job"), expectFile("stage"), + platformName = PlatformNames.DATABRICKS_AWS) } test("test stage-level diagnostic aggregation simple") { @@ -163,8 +164,10 @@ class AnalysisSuite extends FunSuite { } private def testSqlMetricsAggregation(logs: Array[String], expectFileSQL: String, - expectFileJob: String, expectFileStage: String): Unit = { - val apps = ToolTestUtils.processProfileApps(logs, sparkSession) + expectFileJob: String, expectFileStage: String, + platformName: String = PlatformNames.DEFAULT): Unit = { + val args = Array("--platform", platformName) ++ logs + val apps = ToolTestUtils.processProfileApps(args, sparkSession) assert(apps.size == logs.size) val aggResults = RawMetricProfilerView.getAggMetrics(apps) import sparkSession.implicits._ @@ -256,9 +259,12 @@ class AnalysisSuite extends FunSuite { } test("test photon scan metrics") { - val fileName = "nds_q88_photon_db_13_3" - val logs = Array(s"${qualLogDir}/${fileName}.zstd") - val apps = ToolTestUtils.processProfileApps(logs, sparkSession) + val args = Array( + "--platform", + PlatformNames.DATABRICKS_AWS, + s"$qualLogDir/nds_q88_photon_db_13_3.zstd" + ) + val apps = ToolTestUtils.processProfileApps(args, sparkSession) val dataSourceResults = ProfDataSourceView.getRawView(apps) assert(dataSourceResults.exists(_.scan_time > 0)) } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala index 7ff03a943..1d40472c9 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala @@ -22,7 +22,7 @@ import java.nio.file.{Files, Paths, StandardOpenOption} import scala.collection.mutable.ArrayBuffer -import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, StatusReportCounts, ToolTestUtils} +import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, PlatformNames, StatusReportCounts, ToolTestUtils} import com.nvidia.spark.rapids.tool.views.RawMetricProfilerView import org.apache.hadoop.io.IOUtils import org.scalatest.FunSuite @@ -30,6 +30,7 @@ import org.scalatest.FunSuite import org.apache.spark.internal.Logging import org.apache.spark.resource.ResourceProfile import org.apache.spark.sql.{SparkSession, TrampolineUtil} +import org.apache.spark.sql.rapids.tool.UnsupportedSparkRuntimeException import org.apache.spark.sql.rapids.tool.profiling._ import org.apache.spark.sql.rapids.tool.util.{FSUtils, SparkRuntime} @@ -1116,17 +1117,56 @@ class ApplicationInfoSuite extends FunSuite with Logging { } } - val sparkRuntimeTestCases: Seq[(SparkRuntime.Value, String)] = Seq( - SparkRuntime.SPARK -> s"$qualLogDir/nds_q86_test", - SparkRuntime.SPARK_RAPIDS -> s"$logDir/nds_q66_gpu.zstd", - SparkRuntime.PHOTON -> s"$qualLogDir/nds_q88_photon_db_13_3.zstd" + // scalastyle:off line.size.limit + val supportedSparkRuntimeTestCases: Map[String, Seq[(String, SparkRuntime.SparkRuntime)]] = Map( + // tests for standard Spark runtime + s"$qualLogDir/nds_q86_test" -> Seq( + (PlatformNames.DATABRICKS_AWS, SparkRuntime.SPARK), // Expected: SPARK on Databricks AWS + (PlatformNames.ONPREM, SparkRuntime.SPARK) // Expected: SPARK on Onprem + ), + // tests for Spark Rapids runtime + s"$logDir/nds_q66_gpu.zstd" -> Seq( + (PlatformNames.DATABRICKS_AWS, SparkRuntime.SPARK_RAPIDS), // Expected: SPARK_RAPIDS on Databricks AWS + (PlatformNames.ONPREM, SparkRuntime.SPARK_RAPIDS) // Expected: SPARK_RAPIDS on Onprem + ), + // tests for Photon runtime with fallback to SPARK for unsupported platforms + s"$qualLogDir/nds_q88_photon_db_13_3.zstd" -> Seq( + (PlatformNames.DATABRICKS_AWS, SparkRuntime.PHOTON), // Expected: PHOTON on Databricks AWS + (PlatformNames.DATABRICKS_AZURE, SparkRuntime.PHOTON) // Expected: PHOTON on Databricks Azure + ) ) + // scalastyle:on line.size.limit + + supportedSparkRuntimeTestCases.foreach { case (logPath, platformRuntimeCases) => + val baseFileName = logPath.split("/").last + platformRuntimeCases.foreach { case (platform, expectedRuntime) => + test(s"test eventlog $baseFileName on $platform has supported runtime: $expectedRuntime") { + val args = Array("--platform", platform, logPath) + val apps = ToolTestUtils.processProfileApps(args, sparkSession) + assert(apps.size == 1) + assert(apps.head.getSparkRuntime == expectedRuntime) + } + } + } - sparkRuntimeTestCases.foreach { case (expectedSparkRuntime, eventLog) => - test(s"test spark runtime property for ${expectedSparkRuntime.toString} eventlog") { - val apps = ToolTestUtils.processProfileApps(Array(eventLog), sparkSession) - assert(apps.size == 1) - assert(apps.head.getSparkRuntime == expectedSparkRuntime) + // scalastyle:off line.size.limit + val unsupportedSparkRuntimeTestCases: Map[String, Seq[String]] = Map( + s"$qualLogDir/nds_q88_photon_db_13_3.zstd" -> Seq( + PlatformNames.ONPREM, // Expected: PHOTON runtime on Onprem is not supported + PlatformNames.DATAPROC // Expected: PHOTON runtime on Dataproc is not supported + ) + ) + // scalastyle:on line.size.limit + + unsupportedSparkRuntimeTestCases.foreach { case (logPath, platformNames) => + val baseFileName = logPath.split("/").last + platformNames.foreach { platform => + test(s"test eventlog $baseFileName on $platform has unsupported runtime") { + val args = Array("--platform", platform, logPath) + intercept[UnsupportedSparkRuntimeException] { + ToolTestUtils.processProfileApps(args, sparkSession) + } + } } } } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 03943d463..6de463db1 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -136,12 +136,15 @@ class QualificationSuite extends BaseTestSuite { } } - private def runQualificationTest(eventLogs: Array[String], expectFileName: String = "", + private def runQualificationTest(eventLogs: Array[String], + expectFileName: String = "", platformName: String = PlatformNames.DEFAULT, shouldReturnEmpty: Boolean = false, expectPerSqlFileName: Option[String] = None, expectedStatus: Option[StatusReportCounts] = None): Unit = { TrampolineUtil.withTempDir { outpath => val qualOutputPrefix = "rapids_4_spark_qualification_output" val outputArgs = Array( + "--platform", + platformName, "--output-directory", outpath.getAbsolutePath()) @@ -1762,7 +1765,8 @@ class QualificationSuite extends BaseTestSuite { val logFiles = Array(s"$logDir/nds_q88_photon_db_13_3.zstd") // photon event log // Status counts: 1 SUCCESS, 0 FAILURE, 0 SKIPPED, 0 UNKNOWN val expectedStatus = Some(StatusReportCounts(1, 0, 0, 0)) - runQualificationTest(logFiles, expectedStatus = expectedStatus) + runQualificationTest(logFiles, platformName = PlatformNames.DATABRICKS_AWS, + expectedStatus = expectedStatus) } test("process multiple attempts of the same app ID and skip lower attempts") { diff --git a/user_tools/tests/spark_rapids_tools_e2e/features/event_log_processing.feature b/user_tools/tests/spark_rapids_tools_e2e/features/event_log_processing.feature index cd66b0bb6..fc7ec2a52 100644 --- a/user_tools/tests/spark_rapids_tools_e2e/features/event_log_processing.feature +++ b/user_tools/tests/spark_rapids_tools_e2e/features/event_log_processing.feature @@ -16,6 +16,7 @@ Feature: Event Log Processing @test_id_ELP_0001 Scenario Outline: Tool spark_rapids runs with different types of event logs + Given platform is "" When spark-rapids tool is executed with "" eventlogs Then stderr contains the following """ @@ -25,12 +26,13 @@ Feature: Event Log Processing And return code is "0" Examples: - | event_logs | expected_stderr | processed_apps_count | - | invalid_path_eventlog | process.failure.count = 1;invalid_path_eventlog not found, skipping! | 0 | - | gpu_eventlog.zstd | process.skipped.count = 1;GpuEventLogException: Cannot parse event logs from GPU run: skipping this file | 0 | - | photon_eventlog.zstd | process.success.count = 1; | 1 | - | streaming_eventlog.zstd | process.skipped.count = 1;StreamingEventLogException: Encountered Spark Structured Streaming Job: skipping this file! | 0 | - | incorrect_app_status_eventlog.zstd | process.NA.count = 1;IncorrectAppStatusException: Application status is incorrect. Missing AppInfo | 0 | + | platform | event_logs | expected_stderr | processed_apps_count | + | onprem | invalid_path_eventlog | process.failure.count = 1;invalid_path_eventlog not found, skipping! | 0 | + | onprem | gpu_eventlog.zstd | process.skipped.count = 1;GpuEventLogException: Cannot parse event logs from GPU run: skipping this file | 0 | + | onprem | streaming_eventlog.zstd | process.skipped.count = 1;StreamingEventLogException: Encountered Spark Structured Streaming Job: skipping this file! | 0 | + | onprem | incorrect_app_status_eventlog.zstd | process.NA.count = 1;IncorrectAppStatusException: Application status is incorrect. Missing AppInfo | 0 | + | onprem | photon_eventlog.zstd | process.skipped.count = 1;UnsupportedSparkRuntimeException: Platform 'onprem' does not support the runtime 'PHOTON' | 0 | + | databricks-aws | photon_eventlog.zstd | process.success.count = 1; | 1 | @test_id_ELP_0002 Scenario: Qualification tool JAR crashes From 8a78f5d6a29c68652d0556b735c598af7f497f33 Mon Sep 17 00:00:00 2001 From: spark-rapids automation <70000568+nvauto@users.noreply.github.com> Date: Fri, 20 Dec 2024 20:23:39 +0000 Subject: [PATCH 25/52] Update dev-version by jenkins-spark-rapids-tools-auto-release-102 Signed-off-by: spark-rapids automation <70000568+nvauto@users.noreply.github.com> --- core/pom.xml | 2 +- user_tools/src/spark_rapids_pytools/__init__.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 7ae043493..094e6ce3e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -23,7 +23,7 @@ rapids-4-spark-tools_2.12 RAPIDS Accelerator for Apache Spark tools RAPIDS Accelerator for Apache Spark tools - 24.10.4-SNAPSHOT + 24.12.1-SNAPSHOT jar http://github.com/NVIDIA/spark-rapids-tools diff --git a/user_tools/src/spark_rapids_pytools/__init__.py b/user_tools/src/spark_rapids_pytools/__init__.py index 13d08a4dd..103ac0793 100644 --- a/user_tools/src/spark_rapids_pytools/__init__.py +++ b/user_tools/src/spark_rapids_pytools/__init__.py @@ -16,7 +16,7 @@ from spark_rapids_pytools.build import get_version, get_spark_dep_version -VERSION = '24.10.4' +VERSION = '24.12.1' # defines the default runtime build version for the user tools environment SPARK_DEP_VERSION = '350' __version__ = get_version(VERSION) From 53803429f91aafe476dd5999ee552504b170c913 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Mon, 30 Dec 2024 16:29:13 -0800 Subject: [PATCH 26/52] Split AutoTuner for Profiling and Qualification and Override BATCH_SIZE_BYTES (#1471) * Split AutoTuner for Profiling and Qualification and override batch size Signed-off-by: Partho Sarthi * Refactor AutoTuner and move it to tuning package Signed-off-by: Partho Sarthi * Address feedbacks Signed-off-by: Partho Sarthi --------- Signed-off-by: Partho Sarthi --- .../nvidia/spark/rapids/tool/Platform.scala | 2 +- .../rapids/tool/profiling/ProfileArgs.scala | 3 +- .../rapids/tool/profiling/Profiler.scala | 9 +- .../tool/qualification/Qualification.scala | 5 +- .../qualification/QualificationArgs.scala | 4 +- .../qualification/QualificationMain.scala | 6 +- .../{profiling => tuning}/AutoTuner.scala | 233 ++++--- .../tool/tuning/QualificationAutoTuner.scala | 106 +-- .../tuning/QualificationAutoTunerRunner.scala | 105 +++ .../rapids/tool/tuning/TunerContext.scala | 2 +- .../tool/tuning/BaseAutoTunerSuite.scala | 139 ++++ .../ProfilingAutoTunerSuite.scala} | 656 +++++++++--------- .../tuning/QualificationAutoTunerSuite.scala | 66 ++ 13 files changed, 813 insertions(+), 523 deletions(-) rename core/src/main/scala/com/nvidia/spark/rapids/tool/{profiling => tuning}/AutoTuner.scala (87%) create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerRunner.scala create mode 100644 core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala rename core/src/test/scala/com/nvidia/spark/rapids/tool/{profiling/AutoTunerSuite.scala => tuning/ProfilingAutoTunerSuite.scala} (85%) create mode 100644 core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala index 39429085e..866e1fbbd 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids.tool import scala.annotation.tailrec import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper -import com.nvidia.spark.rapids.tool.profiling.ClusterProperties +import com.nvidia.spark.rapids.tool.tuning.ClusterProperties import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.tool.{ExistingClusterInfo, RecommendedClusterInfo} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala index ed4fbb27d..7dcc36e7c 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileArgs.scala @@ -16,6 +16,7 @@ package com.nvidia.spark.rapids.tool.profiling import com.nvidia.spark.rapids.tool.PlatformNames +import com.nvidia.spark.rapids.tool.tuning.ProfilingAutoTunerConfigsProvider import org.rogach.scallop.{ScallopConf, ScallopOption} import org.rogach.scallop.exceptions.ScallopException @@ -117,7 +118,7 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* descr = "File path containing the system information of a worker node. It is assumed " + "that all workers are homogenous. It requires the AutoTuner to be enabled. Default is " + "./worker_info.yaml", - default = Some(AutoTuner.DEFAULT_WORKER_INFO_PATH)) + default = Some(ProfilingAutoTunerConfigsProvider.DEFAULT_WORKER_INFO_PATH)) validate(filterCriteria) { case crit if (crit.endsWith("-newest-filesystem") || diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 5f92cf32d..592fa2cbb 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, EventLogInfo, EventLogPathProcessor, FailedEventLog, Platform, PlatformFactory, ToolBase} -import com.nvidia.spark.rapids.tool.profiling.AutoTuner.loadClusterProps +import com.nvidia.spark.rapids.tool.tuning.{AutoTuner, ProfilingAutoTunerConfigsProvider} import com.nvidia.spark.rapids.tool.views._ import org.apache.hadoop.conf.Configuration @@ -421,9 +421,10 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea // assumptions made in the code if (appInfo.isDefined && appInfo.get.appInfo.head.pluginEnabled) { val appInfoProvider = AppSummaryInfoBaseProvider.fromAppInfo(appInfo) - val workerInfoPath = appArgs.workerInfo.getOrElse(AutoTuner.DEFAULT_WORKER_INFO_PATH) - val clusterPropsOpt = loadClusterProps(workerInfoPath) - val autoTuner: AutoTuner = AutoTuner.buildAutoTuner(appInfoProvider, + val workerInfoPath = appArgs.workerInfo + .getOrElse(ProfilingAutoTunerConfigsProvider.DEFAULT_WORKER_INFO_PATH) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider.loadClusterProps(workerInfoPath) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider.buildAutoTuner(appInfoProvider, PlatformFactory.createInstance(appArgs.platform(), clusterPropsOpt), driverInfoProvider) // The autotuner allows skipping some properties, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index fa59e8b3e..81e6c8c91 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -21,9 +21,8 @@ import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.JavaConverters._ import com.nvidia.spark.rapids.tool.{EventLogInfo, FailedEventLog, PlatformFactory, ToolBase} -import com.nvidia.spark.rapids.tool.profiling.AutoTuner import com.nvidia.spark.rapids.tool.qualification.QualOutputWriter.DEFAULT_JOB_FREQUENCY -import com.nvidia.spark.rapids.tool.tuning.TunerContext +import com.nvidia.spark.rapids.tool.tuning.{QualificationAutoTunerConfigsProvider, TunerContext} import com.nvidia.spark.rapids.tool.views.QualRawReportGenerator import org.apache.hadoop.conf.Configuration @@ -147,7 +146,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, // we need a platform per application because it's storing cluster information which could // vary between applications, especially when using dynamic allocation val platform = { - val clusterPropsOpt = AutoTuner.loadClusterProps(workerInfoPath) + val clusterPropsOpt = QualificationAutoTunerConfigsProvider.loadClusterProps(workerInfoPath) PlatformFactory.createInstance(platformArg, clusterPropsOpt) } val appResult = QualificationAppInfo.createApp(path, hadoopConf, pluginTypeChecker, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationArgs.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationArgs.scala index ce322a0dc..4d5de33a4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationArgs.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationArgs.scala @@ -16,7 +16,7 @@ package com.nvidia.spark.rapids.tool.qualification import com.nvidia.spark.rapids.tool.PlatformNames -import com.nvidia.spark.rapids.tool.profiling.AutoTuner +import com.nvidia.spark.rapids.tool.tuning.QualificationAutoTunerConfigsProvider import org.rogach.scallop.{ScallopConf, ScallopOption} import org.rogach.scallop.exceptions.ScallopException @@ -195,7 +195,7 @@ Usage: java -cp rapids-4-spark-tools_2.12-.jar:$SPARK_HOME/jars/* descr = "File path containing the system information of a worker node. It is assumed " + "that all workers are homogenous. It requires the AutoTuner to be enabled. Default is " + "./worker_info.yaml", - default = Some(AutoTuner.DEFAULT_WORKER_INFO_PATH)) + default = Some(QualificationAutoTunerConfigsProvider.DEFAULT_WORKER_INFO_PATH)) val clusterReport: ScallopOption[Boolean] = toggle("cluster-report", default = Some(true), diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala index 743f95de8..04406ed0d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala @@ -19,8 +19,7 @@ package com.nvidia.spark.rapids.tool.qualification import scala.util.control.NonFatal import com.nvidia.spark.rapids.tool.{EventLogPathProcessor, PlatformFactory} -import com.nvidia.spark.rapids.tool.profiling.AutoTuner.loadClusterProps -import com.nvidia.spark.rapids.tool.tuning.TunerContext +import com.nvidia.spark.rapids.tool.tuning.{QualificationAutoTunerConfigsProvider, TunerContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.tool.AppFilterImpl @@ -75,7 +74,8 @@ object QualificationMain extends Logging { // This platform instance should not be used for anything other then referencing the // files for this particular Platform. val referencePlatform = try { - val clusterPropsOpt = loadClusterProps(appArgs.workerInfo()) + val clusterPropsOpt = + QualificationAutoTunerConfigsProvider.loadClusterProps(appArgs.workerInfo()) PlatformFactory.createInstance(appArgs.platform(), clusterPropsOpt) } catch { case NonFatal(e) => diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala similarity index 87% rename from core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala rename to core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index 44ff839ba..9b022c826 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.tool.profiling +package com.nvidia.spark.rapids.tool.tuning import java.io.{BufferedReader, InputStreamReader, IOException} import java.util @@ -28,6 +28,7 @@ import scala.util.matching.Regex import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, GpuDevice, Platform, PlatformFactory} import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper +import com.nvidia.spark.rapids.tool.profiling._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path} import org.yaml.snakeyaml.{DumperOptions, LoaderOptions, Yaml} @@ -63,10 +64,10 @@ class GpuWorkerProps( * * @return true if the value has been updated. */ - def setDefaultGpuCountIfMissing(): Boolean = { + def setDefaultGpuCountIfMissing(autoTunerConfigsProvider: AutoTunerConfigsProvider): Boolean = { // TODO - do we want to recommend 1 or base it on core count? 32 cores to 1 gpu may be to much. if (count == 0) { - count = AutoTuner.DEF_WORKER_GPU_COUNT + count = autoTunerConfigsProvider.DEF_WORKER_GPU_COUNT true } else { false @@ -105,9 +106,10 @@ class GpuWorkerProps( * @return a list containing information of what was missing and the default value that has been * used to initialize the field. */ - def setMissingFields(platform: Platform): Seq[String] = { + def setMissingFields(platform: Platform, + autoTunerConfigsProvider: AutoTunerConfigsProvider): Seq[String] = { val res = new ListBuffer[String]() - if (setDefaultGpuCountIfMissing()) { + if (setDefaultGpuCountIfMissing(autoTunerConfigsProvider)) { res += s"GPU count is missing. Setting default to $getCount." } if (setDefaultGpuNameIfMissing(platform)) { @@ -143,9 +145,9 @@ class SystemClusterProps( // consider the object incorrect if either numCores or memory are not set. memory == null || memory.isEmpty || numCores <= 0 || memory.startsWith("0") } - def setDefaultNumWorkersIfMissing(): Boolean = { + def setDefaultNumWorkersIfMissing(autoTunerConfigsProvider: AutoTunerConfigsProvider): Boolean = { if (numWorkers <= 0) { - numWorkers = AutoTuner.DEF_NUM_WORKERS + numWorkers = autoTunerConfigsProvider.DEF_NUM_WORKERS true } else { false @@ -156,9 +158,9 @@ class SystemClusterProps( * @return a list containing information of what was missing and the default value that has been * used to initialize the field. */ - def setMissingFields(): Seq[String] = { + def setMissingFields(autoTunerConfigsProvider: AutoTunerConfigsProvider): Seq[String] = { val res = new ListBuffer[String]() - if (setDefaultNumWorkersIfMissing()) { + if (setDefaultNumWorkersIfMissing(autoTunerConfigsProvider)) { res += s"Number of workers is missing. Setting default to $getNumWorkers." } res @@ -184,17 +186,12 @@ class ClusterProperties( @BeanProperty var gpu: GpuWorkerProps, @BeanProperty var softwareProperties: util.LinkedHashMap[String, String]) { - import AutoTuner._ - def this() { this(new SystemClusterProps(), new GpuWorkerProps(), new util.LinkedHashMap[String, String]()) } def isEmpty: Boolean = { system.isEmpty && gpu.isEmpty } - def getTargetProperties: mutable.Map[String, String] = { - softwareProperties.asScala.filter(entry => recommendationsTarget.contains(entry._1)) - } override def toString: String = s"{${system.toString}, ${gpu.toString}, $softwareProperties}" } @@ -337,11 +334,10 @@ class AutoTuner( val clusterProps: ClusterProperties, val appInfoProvider: AppSummaryInfoBaseProvider, val platform: Platform, - val driverInfoProvider: DriverLogInfoProvider) + val driverInfoProvider: DriverLogInfoProvider, + val autoTunerConfigsProvider: AutoTunerConfigsProvider) extends Logging { - import AutoTuner._ - var comments = new ListBuffer[String]() var recommendations: mutable.LinkedHashMap[String, RecommendationEntry] = mutable.LinkedHashMap[String, RecommendationEntry]() @@ -369,7 +365,7 @@ class AutoTuner( } def initRecommendations(): Unit = { - recommendationsTarget.foreach { key => + autoTunerConfigsProvider.recommendationsTarget.foreach { key => // no need to add new records if they are missing from props getPropertyValue(key).foreach { propVal => val recommendationVal = new RecommendationEntry(key, Option(propVal), None) @@ -455,7 +451,7 @@ class AutoTuner( * Assumption - cluster properties were updated to have a default values if missing. */ def calcGpuConcTasks(): Long = { - Math.min(MAX_CONC_GPU_TASKS, platform.getGpuOrDefault.getGpuConcTasks) + Math.min(autoTunerConfigsProvider.MAX_CONC_GPU_TASKS, platform.getGpuOrDefault.getGpuConcTasks) } /** @@ -478,7 +474,7 @@ class AutoTuner( val maxExecutorHeap = Math.max(0, executorContainerMemCalculator()).toInt // give up to 2GB of heap to each executor core // TODO - revisit this in future as we could let heap be bigger - Math.min(maxExecutorHeap, DEF_HEAP_PER_CORE_MB * numExecCores) + Math.min(maxExecutorHeap, autoTunerConfigsProvider.DEF_HEAP_PER_CORE_MB * numExecCores) } /** @@ -497,10 +493,14 @@ class AutoTuner( val containerMem = containerMemCalculator.apply() var setMaxBytesInFlight = false // reserve 10% of heap as memory overhead - var executorMemOverhead = (executorHeap * DEF_HEAP_OVERHEAD_FRACTION).toLong - executorMemOverhead += DEF_PAGEABLE_POOL_MB + var executorMemOverhead = ( + executorHeap * autoTunerConfigsProvider.DEF_HEAP_OVERHEAD_FRACTION + ).toLong + executorMemOverhead += autoTunerConfigsProvider.DEF_PAGEABLE_POOL_MB val containerMemLeftOverOffHeap = containerMem - executorHeap - val minOverhead = executorMemOverhead + (MIN_PINNED_MEMORY_MB + MIN_SPILL_MEMORY_MB) + val minOverhead = executorMemOverhead + ( + autoTunerConfigsProvider.MIN_PINNED_MEMORY_MB + autoTunerConfigsProvider.MIN_SPILL_MEMORY_MB + ) logDebug("containerMem " + containerMem + " executorHeap: " + executorHeap + " executorMemOverhead: " + executorMemOverhead + " minOverhead " + minOverhead) if (containerMemLeftOverOffHeap >= minOverhead) { @@ -508,14 +508,15 @@ class AutoTuner( // memory to core ratio if (numExecutorCores >= 16 && platform.isPlatformCSP && containerMemLeftOverOffHeap > - executorMemOverhead + 4096L + MIN_PINNED_MEMORY_MB + MIN_SPILL_MEMORY_MB) { + executorMemOverhead + 4096L + autoTunerConfigsProvider.MIN_PINNED_MEMORY_MB + + autoTunerConfigsProvider.MIN_SPILL_MEMORY_MB) { // Account for the setting of: // appendRecommendation("spark.rapids.shuffle.multiThreaded.maxBytesInFlight", "4g") executorMemOverhead += 4096L setMaxBytesInFlight = true } // Pinned memory uses any unused space up to 4GB. Spill memory is same size as pinned. - val pinnedMem = Math.min(MAX_PINNED_MEMORY_MB, + val pinnedMem = Math.min(autoTunerConfigsProvider.MAX_PINNED_MEMORY_MB, (containerMemLeftOverOffHeap - executorMemOverhead) / 2).toLong // Spill storage is set to the pinned size by default. Its not guaranteed to use just pinned // memory though so the size worst case would be doesn't use any pinned memory and uses @@ -525,7 +526,8 @@ class AutoTuner( executorMemOverhead += pinnedMem + spillMem } else { // use min pinned and spill mem - executorMemOverhead += MIN_PINNED_MEMORY_MB + MIN_SPILL_MEMORY_MB + executorMemOverhead += autoTunerConfigsProvider.MIN_PINNED_MEMORY_MB + + autoTunerConfigsProvider.MIN_SPILL_MEMORY_MB } (pinnedMem, executorMemOverhead, executorHeap, setMaxBytesInFlight) } else { @@ -534,7 +536,7 @@ class AutoTuner( // first calculate what we think min overhead is and make sure we have enough // for that // calculate minimum heap size - val minExecHeapMem = MIN_HEAP_PER_CORE_MB * numExecutorCores + val minExecHeapMem = autoTunerConfigsProvider.MIN_HEAP_PER_CORE_MB * numExecutorCores if ((containerMem - minOverhead) < minExecHeapMem) { // For now just throw so we don't get any tunings and its obvious to user this isn't a good // setup. In the future we may just recommend them to use larger nodes. This would be more @@ -547,7 +549,8 @@ class AutoTuner( warnNotEnoughMem(minExecHeapMem + minOverhead) } // Pinned memory uses any unused space up to 4GB. Spill memory is same size as pinned. - val pinnedMem = Math.min(MAX_PINNED_MEMORY_MB, (leftOverMemUsingMinHeap / 2)).toLong + val pinnedMem = Math.min(autoTunerConfigsProvider.MAX_PINNED_MEMORY_MB, + leftOverMemUsingMinHeap / 2).toLong val spillMem = pinnedMem // spill memory is by default same size as pinned memory executorMemOverhead += pinnedMem + spillMem @@ -604,7 +607,8 @@ class AutoTuner( * else recommend "spark.kubernetes.memoryOverheadFactor" and add comment if missing */ def addRecommendationForMemoryOverhead(recomValue: String): Unit = { - if (enableMemoryOverheadRecommendation(getPropertyValue("spark.master"))) { + if (autoTunerConfigsProvider + .enableMemoryOverheadRecommendation(getPropertyValue("spark.master"))) { val memOverheadLookup = memoryOverheadLabel appendRecommendationForMemoryMB(memOverheadLookup, recomValue) getPropertyValue("spark.rapids.memory.pinnedPool.size").foreach { lookup => @@ -705,7 +709,8 @@ class AutoTuner( } else { addDefaultComments() } - appendRecommendation("spark.rapids.sql.batchSizeBytes", BATCH_SIZE_BYTES) + appendRecommendation("spark.rapids.sql.batchSizeBytes", + autoTunerConfigsProvider.BATCH_SIZE_BYTES) appendRecommendation("spark.locality.wait", 0) } @@ -716,7 +721,7 @@ class AutoTuner( case Some(smClassName) => appendRecommendation("spark.shuffle.manager", smClassName) case None => appendComment("Could not define the Spark Version") } - appendComment(classPathComments("rapids.shuffle.jars")) + appendComment(autoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")) recommendFileCache() recommendMaxPartitionBytes() recommendShufflePartitions() @@ -781,10 +786,12 @@ class AutoTuner( def configureClusterPropDefaults: Unit = { if (!clusterProps.system.isEmpty) { if (clusterProps.system.isMissingInfo) { - clusterProps.system.setMissingFields().foreach(m => appendComment(m)) + clusterProps.system.setMissingFields(autoTunerConfigsProvider) + .foreach(m => appendComment(m)) } if (clusterProps.gpu.isMissingInfo) { - clusterProps.gpu.setMissingFields(platform).foreach(m => appendComment(m)) + clusterProps.gpu.setMissingFields(platform, autoTunerConfigsProvider) + .foreach(m => appendComment(m)) } } } @@ -792,7 +799,8 @@ class AutoTuner( private def recommendGCProperty(): Unit = { val jvmGCFraction = appInfoProvider.getJvmGCFractions if (jvmGCFraction.nonEmpty) { // avoid zero division - if ((jvmGCFraction.sum / jvmGCFraction.size) > MAX_JVM_GCTIME_FRACTION) { + if ((jvmGCFraction.sum / jvmGCFraction.size) > + autoTunerConfigsProvider.MAX_JVM_GCTIME_FRACTION) { // TODO - or other cores/memory ratio appendComment("Average JVM GC time is very high. " + "Other Garbage Collectors can be used for better performance.") @@ -804,7 +812,7 @@ class AutoTuner( val aqeEnabled = getPropertyValue("spark.sql.adaptive.enabled") .getOrElse("false").toLowerCase if (aqeEnabled == "false") { - appendComment(commentsForMissingProps("spark.sql.adaptive.enabled")) + appendComment(autoTunerConfigsProvider.commentsForMissingProps("spark.sql.adaptive.enabled")) } appInfoProvider.getSparkVersion match { case Some(version) => @@ -833,7 +841,8 @@ class AutoTuner( val advisoryPartitionSizeProperty = getPropertyValue("spark.sql.adaptive.advisoryPartitionSizeInBytes") - if (appInfoProvider.getMeanInput < AQE_INPUT_SIZE_BYTES_THRESHOLD) { + if (appInfoProvider.getMeanInput < + autoTunerConfigsProvider.AQE_INPUT_SIZE_BYTES_THRESHOLD) { if(advisoryPartitionSizeProperty.isEmpty) { // The default is 64m, but 128m is slightly better for the GPU as the GPU has sub-linear // scaling until it is full and 128m makes the GPU more full, but too large can be @@ -841,15 +850,17 @@ class AutoTuner( appendRecommendation("spark.sql.adaptive.advisoryPartitionSizeInBytes", "128m") } } - if (appInfoProvider.getMeanInput > AQE_INPUT_SIZE_BYTES_THRESHOLD && - appInfoProvider.getMeanShuffleRead > AQE_SHUFFLE_READ_BYTES_THRESHOLD) { + if (appInfoProvider.getMeanInput > autoTunerConfigsProvider.AQE_INPUT_SIZE_BYTES_THRESHOLD && + appInfoProvider.getMeanShuffleRead > + autoTunerConfigsProvider.AQE_SHUFFLE_READ_BYTES_THRESHOLD) { // AQE Recommendations for large input and large shuffle reads platform.getGpuOrDefault.getAdvisoryPartitionSizeInBytes.foreach { size => appendRecommendation("spark.sql.adaptive.advisoryPartitionSizeInBytes", size) } val initialPartitionNumProperty = getPropertyValue("spark.sql.adaptive.coalescePartitions.initialPartitionNum").map(_.toInt) - if (initialPartitionNumProperty.getOrElse(0) <= AQE_MIN_INITIAL_PARTITION_NUM) { + if (initialPartitionNumProperty.getOrElse(0) <= + autoTunerConfigsProvider.AQE_MIN_INITIAL_PARTITION_NUM) { platform.getGpuOrDefault.getInitialPartitionNum.foreach { initialPartitionNum => appendRecommendation( "spark.sql.adaptive.coalescePartitions.initialPartitionNum", initialPartitionNum) @@ -867,9 +878,10 @@ class AutoTuner( if (autoBroadcastJoinThresholdProperty.isEmpty) { appendComment("'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set.") } else if (autoBroadcastJoinThresholdProperty.get > - StringUtils.convertToMB(AQE_AUTOBROADCAST_JOIN_THRESHOLD)) { + StringUtils.convertToMB(autoTunerConfigsProvider.AQE_AUTOBROADCAST_JOIN_THRESHOLD)) { appendComment("Setting 'spark.sql.adaptive.autoBroadcastJoinThreshold' > " + - s"$AQE_AUTOBROADCAST_JOIN_THRESHOLD could lead to performance\n" + + s"${autoTunerConfigsProvider.AQE_AUTOBROADCAST_JOIN_THRESHOLD} could " + + s"lead to performance\n" + " regression. Should be set to a lower number.") } } @@ -896,36 +908,37 @@ class AutoTuner( * 4- If there is a new release recommend that to the user */ private def recommendClassPathEntries(): Unit = { - val missingRapidsJarsEntry = classPathComments("rapids.jars.missing") - val multipleRapidsJarsEntry = classPathComments("rapids.jars.multiple") + val missingRapidsJarsEntry = autoTunerConfigsProvider.classPathComments("rapids.jars.missing") + val multipleRapidsJarsEntry = autoTunerConfigsProvider.classPathComments("rapids.jars.multiple") appInfoProvider.getRapidsJars match { case Seq() => // No rapids jars appendComment(missingRapidsJarsEntry) case s: Seq[String] => - s.flatMap(e => pluginJarRegEx.findAllMatchIn(e).map(_.group(1))) match { - case Seq() => appendComment(missingRapidsJarsEntry) - case v: Seq[String] if v.length > 1 => - val comment = s"$multipleRapidsJarsEntry [${v.mkString(", ")}]" - appendComment(comment) - case Seq(jarVer) => - // compare jarVersion to the latest release - val latestPluginVersion = WebCrawlerUtil.getLatestPluginRelease - latestPluginVersion match { - case Some(ver) => - if (ToolUtils.compareVersions(jarVer, ver) < 0) { - val jarURL = WebCrawlerUtil.getPluginMvnDownloadLink(ver) + s.flatMap(e => + autoTunerConfigsProvider.pluginJarRegEx.findAllMatchIn(e).map(_.group(1))) match { + case Seq() => appendComment(missingRapidsJarsEntry) + case v: Seq[String] if v.length > 1 => + val comment = s"$multipleRapidsJarsEntry [${v.mkString(", ")}]" + appendComment(comment) + case Seq(jarVer) => + // compare jarVersion to the latest release + val latestPluginVersion = WebCrawlerUtil.getLatestPluginRelease + latestPluginVersion match { + case Some(ver) => + if (ToolUtils.compareVersions(jarVer, ver) < 0) { + val jarURL = WebCrawlerUtil.getPluginMvnDownloadLink(ver) + appendComment( + "A newer RAPIDS Accelerator for Apache Spark plugin is available:\n" + + s" $jarURL\n" + + s" Version used in application is $jarVer.") + } + case None => + logError("Could not pull the latest release of RAPIDS-plugin jar.") + val pluginRepoUrl = WebCrawlerUtil.getMVNArtifactURL("rapids.plugin") appendComment( - "A newer RAPIDS Accelerator for Apache Spark plugin is available:\n" + - s" $jarURL\n" + - s" Version used in application is $jarVer.") - } - case None => - logError("Could not pull the latest release of RAPIDS-plugin jar.") - val pluginRepoUrl = WebCrawlerUtil.getMVNArtifactURL("rapids.plugin") - appendComment( - "Failed to validate the latest release of Apache Spark plugin.\n" + + "Failed to validate the latest release of Apache Spark plugin.\n" + s" Verify that the version used in application ($jarVer) is the latest on:\n" + s" $pluginRepoUrl") @@ -956,19 +969,19 @@ class AutoTuner( maxPartitionBytesNum.toString } else { if (inputBytesMax > 0 && - inputBytesMax < MIN_PARTITION_BYTES_RANGE_MB) { + inputBytesMax < autoTunerConfigsProvider.MIN_PARTITION_BYTES_RANGE_MB) { // Increase partition size val calculatedMaxPartitionBytes = Math.min( maxPartitionBytesNum * - (MIN_PARTITION_BYTES_RANGE_MB / inputBytesMax), - MAX_PARTITION_BYTES_BOUND_MB) + (autoTunerConfigsProvider.MIN_PARTITION_BYTES_RANGE_MB / inputBytesMax), + autoTunerConfigsProvider.MAX_PARTITION_BYTES_BOUND_MB) calculatedMaxPartitionBytes.toLong.toString - } else if (inputBytesMax > MAX_PARTITION_BYTES_RANGE_MB) { + } else if (inputBytesMax > autoTunerConfigsProvider.MAX_PARTITION_BYTES_RANGE_MB) { // Decrease partition size val calculatedMaxPartitionBytes = Math.min( maxPartitionBytesNum / - (inputBytesMax / MAX_PARTITION_BYTES_RANGE_MB), - MAX_PARTITION_BYTES_BOUND_MB) + (inputBytesMax / autoTunerConfigsProvider.MAX_PARTITION_BYTES_RANGE_MB), + autoTunerConfigsProvider.MAX_PARTITION_BYTES_BOUND_MB) calculatedMaxPartitionBytes.toLong.toString } else { // Do not recommend maxPartitionBytes @@ -981,8 +994,10 @@ class AutoTuner( * Recommendation for 'spark.rapids.file.cache' based on read characteristics of job. */ private def recommendFileCache() { - if (appInfoProvider.getDistinctLocationPct < DEF_DISTINCT_READ_THRESHOLD - && appInfoProvider.getRedundantReadSize > DEF_READ_SIZE_THRESHOLD) { + if (appInfoProvider.getDistinctLocationPct < + autoTunerConfigsProvider.DEF_DISTINCT_READ_THRESHOLD && + appInfoProvider.getRedundantReadSize > + autoTunerConfigsProvider.DEF_READ_SIZE_THRESHOLD) { appendRecommendation("spark.rapids.filecache.enabled", "true") appendComment("Enable file cache only if Spark local disks bandwidth is > 1 GB/s" + " and you have sufficient disk space available to fit both cache and normal Spark" + @@ -997,7 +1012,8 @@ class AutoTuner( */ private def recommendMaxPartitionBytes(): Unit = { val maxPartitionProp = - getPropertyValue("spark.sql.files.maxPartitionBytes").getOrElse(MAX_PARTITION_BYTES) + getPropertyValue("spark.sql.files.maxPartitionBytes") + .getOrElse(autoTunerConfigsProvider.MAX_PARTITION_BYTES) val recommended = if (isCalculationEnabled("spark.sql.files.maxPartitionBytes")) { calculateMaxPartitionBytes(maxPartitionProp) @@ -1015,7 +1031,7 @@ class AutoTuner( def recommendShufflePartitions(): Unit = { val lookup = "spark.sql.shuffle.partitions" var shufflePartitions = - getPropertyValue(lookup).getOrElse(DEF_SHUFFLE_PARTITIONS).toInt + getPropertyValue(lookup).getOrElse(autoTunerConfigsProvider.DEF_SHUFFLE_PARTITIONS).toInt val shuffleStagesWithPosSpilling = appInfoProvider.getShuffleStagesWithPosSpilling // TODO: Need to look at other metrics for GPU spills (DEBUG mode), and batch sizes metric @@ -1028,7 +1044,7 @@ class AutoTuner( s" stages with spilling. Increasing shuffle partitions is not recommended in this\n" + s" case since keys will still hash to the same task.") } else { - shufflePartitions *= DEF_SHUFFLE_PARTITION_MULTIPLIER + shufflePartitions *= autoTunerConfigsProvider.DEF_SHUFFLE_PARTITION_MULTIPLIER // Could be memory instead of partitions appendOptionalComment(lookup, s"'$lookup' should be increased since spilling occurred in shuffle stages.") @@ -1050,10 +1066,10 @@ class AutoTuner( private def recommendFromDriverLogs(): Unit = { // Iterate through unsupported operators' reasons and check for matching properties driverInfoProvider.getUnsupportedOperators.map(_.reason).foreach { operatorReason => - recommendationsFromDriverLogs.collect { + autoTunerConfigsProvider.recommendationsFromDriverLogs.collect { case (config, recommendedValue) if operatorReason.contains(config) => appendRecommendation(config, recommendedValue) - appendComment(commentForExperimentalConfig(config)) + appendComment(autoTunerConfigsProvider.commentForExperimentalConfig(config)) } } } @@ -1098,7 +1114,7 @@ class AutoTuner( private def addDefaultComments(): Unit = { appendComment("Could not infer the cluster configuration, recommendations " + "are generated using default values!") - commentsForMissingProps.foreach { + autoTunerConfigsProvider.commentsForMissingProps.foreach { case (key, value) => if (!skippedRecommendations.contains(key)) { appendComment(value) @@ -1107,7 +1123,7 @@ class AutoTuner( } private def addMissingMemoryComments(): Unit = { - commentsForMissingMemoryProps.foreach { + autoTunerConfigsProvider.commentsForMissingMemoryProps.foreach { case (key, value) => if (!skippedRecommendations.contains(key)) { appendComment(value) @@ -1188,7 +1204,7 @@ class AutoTuner( // - make sure that we exclude the skipped list private def processPropKeys( srcMap: collection.Map[String, String]): collection.Map[String, String] = { - (srcMap -- skippedRecommendations) -- filteredPropKeys + (srcMap -- skippedRecommendations) -- autoTunerConfigsProvider.filteredPropKeys } // Combines the original Spark properties with the recommended ones. @@ -1206,7 +1222,10 @@ class AutoTuner( } } -object AutoTuner extends Logging { +/** + * Trait defining configuration defaults and parameters for the AutoTuner. + */ +trait AutoTunerConfigsProvider extends Logging { // Maximum number of concurrent tasks to run on the GPU val MAX_CONC_GPU_TASKS = 4L // Amount of CPU memory to reserve for system overhead (kernel, buffers, etc.) in megabytes @@ -1251,13 +1270,13 @@ object AutoTuner extends Logging { private val DOC_URL: String = "https://nvidia.github.io/spark-rapids/docs/" + "additional-functionality/advanced_configs.html#advanced-configuration" // Value of batchSizeBytes that performs best overall - private val BATCH_SIZE_BYTES = 2147483647 - private val AQE_INPUT_SIZE_BYTES_THRESHOLD = 35000 - private val AQE_SHUFFLE_READ_BYTES_THRESHOLD = 50000 - private val AQE_MIN_INITIAL_PARTITION_NUM = 200 - private val AQE_AUTOBROADCAST_JOIN_THRESHOLD = "100m" + val BATCH_SIZE_BYTES = 2147483647 + val AQE_INPUT_SIZE_BYTES_THRESHOLD = 35000 + val AQE_SHUFFLE_READ_BYTES_THRESHOLD = 50000 + val AQE_MIN_INITIAL_PARTITION_NUM = 200 + val AQE_AUTOBROADCAST_JOIN_THRESHOLD = "100m" // Set of spark properties to be filtered out from the combined Spark properties. - private val filteredPropKeys: Set[String] = Set( + val filteredPropKeys: Set[String] = Set( "spark.app.id" ) @@ -1313,7 +1332,7 @@ object AutoTuner extends Logging { ) // Recommended values for specific unsupported configurations - private val recommendationsFromDriverLogs: Map[String, String] = Map( + val recommendationsFromDriverLogs: Map[String, String] = Map( "spark.rapids.sql.incompatibleDateFormats.enabled" -> "true" ) @@ -1325,13 +1344,23 @@ object AutoTuner extends Logging { // the plugin jar is in the form of rapids-4-spark_scala_binary-(version)-*.jar val pluginJarRegEx: Regex = "rapids-4-spark_\\d\\.\\d+-(\\d{2}\\.\\d{2}\\.\\d+).*\\.jar".r - private def handleException( + /** + * Abstract method to create an instance of the AutoTuner. + */ + def createAutoTunerInstance( + clusterProps: ClusterProperties, + appInfoProvider: AppSummaryInfoBaseProvider, + platform: Platform, + driverInfoProvider: DriverLogInfoProvider): AutoTuner + + def handleException( ex: Throwable, appInfo: AppSummaryInfoBaseProvider, platform: Platform, driverInfoProvider: DriverLogInfoProvider): AutoTuner = { logError("Exception: " + ex.getStackTrace.mkString("Array(", ", ", ")")) - val tuning = new AutoTuner(new ClusterProperties(), appInfo, platform, driverInfoProvider) + val tuning = createAutoTunerInstance(new ClusterProperties(), appInfo, + platform, driverInfoProvider) val msg = ex match { case cEx: ConstructorException => cEx.getContext case _ => if (ex.getCause != null) ex.getCause.toString else ex.toString @@ -1394,8 +1423,8 @@ object AutoTuner extends Logging { ): AutoTuner = { try { val clusterPropsOpt = loadClusterPropertiesFromContent(clusterProps) - new AutoTuner(clusterPropsOpt.getOrElse(new ClusterProperties()), singleAppProvider, platform, - driverInfoProvider) + createAutoTunerInstance(clusterPropsOpt.getOrElse(new ClusterProperties()), + singleAppProvider, platform, driverInfoProvider) } catch { case NonFatal(e) => handleException(e, singleAppProvider, platform, driverInfoProvider) @@ -1408,7 +1437,8 @@ object AutoTuner extends Logging { driverInfoProvider: DriverLogInfoProvider = BaseDriverLogInfoProvider.noneDriverLog ): AutoTuner = { try { - val autoT = new AutoTuner(platform.clusterProperties.getOrElse(new ClusterProperties()), + val autoT = createAutoTunerInstance( + platform.clusterProperties.getOrElse(new ClusterProperties()), singleAppProvider, platform, driverInfoProvider) autoT } catch { @@ -1431,3 +1461,18 @@ object AutoTuner extends Logging { } } } + +/** + * Provides configuration settings for the Profiling Tool's AutoTuner. This object is as a concrete + * implementation of the `AutoTunerConfigsProvider` interface. + */ +object ProfilingAutoTunerConfigsProvider extends AutoTunerConfigsProvider { + def createAutoTunerInstance( + clusterProps: ClusterProperties, + appInfoProvider: AppSummaryInfoBaseProvider, + platform: Platform, + driverInfoProvider: DriverLogInfoProvider): AutoTuner = { + new AutoTuner(clusterProps, appInfoProvider, platform, driverInfoProvider, + ProfilingAutoTunerConfigsProvider) + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala index 4344f0a5d..21c253f5e 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala @@ -16,89 +16,35 @@ package com.nvidia.spark.rapids.tool.tuning -import scala.util.{Failure, Success, Try} - -import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, Platform, ToolTextFileWriter} -import com.nvidia.spark.rapids.tool.analysis.AggRawMetricsResult -import com.nvidia.spark.rapids.tool.profiling.{AutoTuner, DataSourceProfileResult, Profiler} -import org.apache.hadoop.conf.Configuration - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.rapids.tool.qualification.{QualificationAppInfo, QualificationSummaryInfo} +import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, Platform} +import com.nvidia.spark.rapids.tool.profiling.DriverLogInfoProvider /** - * Implementation of the AutoTuner for Qualification. - * @param appInfoProvider Provider of the qualification analysis data - * @param tunerContext Container which holds the arguments passed to the AutoTuner execution + * Implementation of the `AutoTuner` designed the Qualification Tool. This class can be used to + * implement the logic to recommend AutoTuner configurations by the Qualification Tool. */ -class QualificationAutoTuner(val appInfoProvider: QualAppSummaryInfoProvider, - val tunerContext: TunerContext) { - - // When enabled, the profiler recommendations should only include updated settings. - private val filterByUpdatedPropsEnabled: Boolean = false - - private def writeTuningReport(tuningResult: TuningResult, - outputDir: String, hadoopConf: Configuration): Unit = { - // First, write down the recommendations and the comments - val textFileWriter = new ToolTextFileWriter(outputDir, - s"${tuningResult.appID}.log", s"Tuning Qual App - ${tuningResult.appID}", Option(hadoopConf)) - try { - textFileWriter.write( - s"### Recommended SPARK Configuration on GPU Cluster for App: ${tuningResult.appID} ###\n") - textFileWriter.write(Profiler.getAutoTunerResultsAsString( - tuningResult.recommendations, tuningResult.comments)) - } finally { - textFileWriter.close() - } - // Write down the combined configurations - tuningResult.combinedProps.collect { - case combinedProps => - val textFileWriter = new ToolTextFileWriter(outputDir, - s"${tuningResult.appID}.conf", - s"Qual combined configurations for App - ${tuningResult.appID}", Option(hadoopConf)) - try { - textFileWriter.write(combinedProps.map(_.toString).reduce(_ + "\n" + _)) - } finally { - textFileWriter.close() - } - } - } +class QualificationAutoTuner( + clusterProps: ClusterProperties, + appInfoProvider: AppSummaryInfoBaseProvider, + platform: Platform, + driverInfoProvider: DriverLogInfoProvider) + extends AutoTuner(clusterProps, appInfoProvider, platform, driverInfoProvider, + QualificationAutoTunerConfigsProvider) - def runAutoTuner(platform: Platform): TuningResult = { - val autoTuner: AutoTuner = AutoTuner.buildAutoTuner(appInfoProvider, platform) - val (recommendations, comments) = - autoTuner.getRecommendedProperties(showOnlyUpdatedProps = filterByUpdatedPropsEnabled) - // Combine the GPU recommendations with all others. - // There are two ways we can that: - // 1- Combine them from the beginning; Or - // 2- At the end, get the union of the two properties. - // The 2nd needs more effort but it favourite because it keeps two separate lists. - // Otherwise, it is difficult to do separate them logically. - val combinedProps = autoTuner.combineSparkProperties(recommendations) - val resultRecord = TuningResult(appInfoProvider.getAppID, recommendations, - comments, Option(combinedProps)) - writeTuningReport(resultRecord, tunerContext.getOutputPath, tunerContext.hadoopConf) - resultRecord - } -} - -object QualificationAutoTuner extends Logging { - def apply(appInfo: QualificationAppInfo, - appAggStats: Option[QualificationSummaryInfo], - tunerContext: TunerContext, - rawAggMetrics: AggRawMetricsResult, - dsInfo: Seq[DataSourceProfileResult]): Option[QualificationAutoTuner] = { - Try { - val qualInfoProvider: QualAppSummaryInfoProvider = - AppSummaryInfoBaseProvider.fromQualAppInfo(appInfo, appAggStats, rawAggMetrics, dsInfo) - .asInstanceOf[QualAppSummaryInfoProvider] - new QualificationAutoTuner(qualInfoProvider, tunerContext) - } match { - case Success(q) => Some(q) - case Failure(e) => - logError( - s"Failed to create Qualification tuning object for application ${appInfo.appId}", e) - None - } +/** + * Provides configuration settings for the Qualification Tool's AutoTuner + */ +object QualificationAutoTunerConfigsProvider extends AutoTunerConfigsProvider { + + // For qualification tool's auto-tuner, the batch size to be recommended is 1GB + // See https://github.com/NVIDIA/spark-rapids-tools/issues/1399 + override val BATCH_SIZE_BYTES = 1073741824 + + override def createAutoTunerInstance( + clusterProps: ClusterProperties, + appInfoProvider: AppSummaryInfoBaseProvider, + platform: Platform, + driverInfoProvider: DriverLogInfoProvider): AutoTuner = { + new QualificationAutoTuner(clusterProps, appInfoProvider, platform, driverInfoProvider) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerRunner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerRunner.scala new file mode 100644 index 000000000..cd4c74c17 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerRunner.scala @@ -0,0 +1,105 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.tuning + +import scala.util.{Failure, Success, Try} + +import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, Platform, ToolTextFileWriter} +import com.nvidia.spark.rapids.tool.analysis.AggRawMetricsResult +import com.nvidia.spark.rapids.tool.profiling.{DataSourceProfileResult, Profiler} +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.rapids.tool.qualification.{QualificationAppInfo, QualificationSummaryInfo} + +/** + * A wrapper class to run the AutoTuner for Qualification Tool. + * @param appInfoProvider Provider of the qualification analysis data + * @param tunerContext Container which holds the arguments passed to the AutoTuner execution + */ +class QualificationAutoTunerRunner(val appInfoProvider: QualAppSummaryInfoProvider, + val tunerContext: TunerContext) { + + // When enabled, the profiler recommendations should only include updated settings. + private val filterByUpdatedPropsEnabled: Boolean = false + + private def writeTuningReport(tuningResult: TuningResult, + outputDir: String, hadoopConf: Configuration): Unit = { + // First, write down the recommendations and the comments + val textFileWriter = new ToolTextFileWriter(outputDir, + s"${tuningResult.appID}.log", s"Tuning Qual App - ${tuningResult.appID}", Option(hadoopConf)) + try { + textFileWriter.write( + s"### Recommended SPARK Configuration on GPU Cluster for App: ${tuningResult.appID} ###\n") + textFileWriter.write(Profiler.getAutoTunerResultsAsString( + tuningResult.recommendations, tuningResult.comments)) + } finally { + textFileWriter.close() + } + // Write down the combined configurations + tuningResult.combinedProps.collect { + case combinedProps => + val textFileWriter = new ToolTextFileWriter(outputDir, + s"${tuningResult.appID}.conf", + s"Qual combined configurations for App - ${tuningResult.appID}", Option(hadoopConf)) + try { + textFileWriter.write(combinedProps.map(_.toString).reduce(_ + "\n" + _)) + } finally { + textFileWriter.close() + } + } + } + + def runAutoTuner(platform: Platform): TuningResult = { + val autoTuner: AutoTuner = + QualificationAutoTunerConfigsProvider.buildAutoTuner(appInfoProvider, platform) + val (recommendations, comments) = + autoTuner.getRecommendedProperties(showOnlyUpdatedProps = filterByUpdatedPropsEnabled) + // Combine the GPU recommendations with all others. + // There are two ways we can do that: + // 1- Combine them from the beginning; Or + // 2- At the end, get the union of the two properties. + // The 2nd needs more effort but it favourite because it keeps two separate lists. + // Otherwise, it is difficult to separate them logically. + val combinedProps = autoTuner.combineSparkProperties(recommendations) + val resultRecord = TuningResult(appInfoProvider.getAppID, recommendations, + comments, Option(combinedProps)) + writeTuningReport(resultRecord, tunerContext.getOutputPath, tunerContext.hadoopConf) + resultRecord + } +} + +object QualificationAutoTunerRunner extends Logging { + def apply(appInfo: QualificationAppInfo, + appAggStats: Option[QualificationSummaryInfo], + tunerContext: TunerContext, + rawAggMetrics: AggRawMetricsResult, + dsInfo: Seq[DataSourceProfileResult]): Option[QualificationAutoTunerRunner] = { + Try { + val qualInfoProvider: QualAppSummaryInfoProvider = + AppSummaryInfoBaseProvider.fromQualAppInfo(appInfo, appAggStats, rawAggMetrics, dsInfo) + .asInstanceOf[QualAppSummaryInfoProvider] + new QualificationAutoTunerRunner(qualInfoProvider, tunerContext) + } match { + case Success(q) => Some(q) + case Failure(e) => + logError( + s"Failed to create Qualification tuning object for application ${appInfo.appId}", e) + None + } + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala index b9b23ec07..bbf7fb9b6 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/TunerContext.scala @@ -56,7 +56,7 @@ case class TunerContext ( val sqlAnalyzer = AppSQLPlanAnalyzer(appInfo, appIndex) val rawAggMetrics = QualSparkMetricsAnalyzer.getAggRawMetrics(appInfo, appIndex, Some(sqlAnalyzer)) - QualificationAutoTuner(appInfo, appAggStats, this, rawAggMetrics, dsInfo).collect { + QualificationAutoTunerRunner(appInfo, appAggStats, this, rawAggMetrics, dsInfo).collect { case qualTuner => Try { qualTuner.runAutoTuner(platform) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala new file mode 100644 index 000000000..7b5e11d97 --- /dev/null +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala @@ -0,0 +1,139 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.tuning + +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import com.nvidia.spark.rapids.tool.AppSummaryInfoBaseProvider +import com.nvidia.spark.rapids.tool.profiling._ +import org.scalatest.{BeforeAndAfterEach, FunSuite} +import org.yaml.snakeyaml.{DumperOptions, Yaml} + +import org.apache.spark.internal.Logging + + +case class DriverInfoProviderMockTest(unsupportedOps: Seq[DriverLogUnsupportedOperators]) + extends BaseDriverLogInfoProvider(None) { + override def getUnsupportedOperators: Seq[DriverLogUnsupportedOperators] = unsupportedOps +} + +class AppInfoProviderMockTest(val maxInput: Double, + val spilledMetrics: Seq[Long], + val jvmGCFractions: Seq[Double], + val propsFromLog: mutable.Map[String, String], + val sparkVersion: Option[String], + val rapidsJars: Seq[String], + val distinctLocationPct: Double, + val redundantReadSize: Long, + val meanInput: Double, + val meanShuffleRead: Double, + val shuffleStagesWithPosSpilling: Set[Long], + val shuffleSkewStages: Set[Long]) extends AppSummaryInfoBaseProvider { + override def isAppInfoAvailable = true + override def getMaxInput: Double = maxInput + override def getMeanInput: Double = meanInput + override def getMeanShuffleRead: Double = meanShuffleRead + override def getSpilledMetrics: Seq[Long] = spilledMetrics + override def getJvmGCFractions: Seq[Double] = jvmGCFractions + override def getRapidsProperty(propKey: String): Option[String] = propsFromLog.get(propKey) + override def getSparkProperty(propKey: String): Option[String] = propsFromLog.get(propKey) + override def getSystemProperty(propKey: String): Option[String] = propsFromLog.get(propKey) + override def getSparkVersion: Option[String] = sparkVersion + override def getRapidsJars: Seq[String] = rapidsJars + override def getDistinctLocationPct: Double = distinctLocationPct + override def getRedundantReadSize: Long = redundantReadSize + override def getShuffleStagesWithPosSpilling: Set[Long] = shuffleStagesWithPosSpilling + override def getShuffleSkewStages: Set[Long] = shuffleSkewStages +} + +/** + * Base class for AutoTuner test suites + */ +abstract class BaseAutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { + + val defaultSparkVersion = "3.1.1" + + val defaultDataprocProps: mutable.Map[String, String] = { + mutable.LinkedHashMap[String, String]( + "spark.dynamicAllocation.enabled" -> "true", + "spark.driver.maxResultSize" -> "7680m", + "spark.driver.memory" -> "15360m", + "spark.executor.cores" -> "16", + "spark.executor.instances" -> "2", + "spark.executor.resource.gpu.amount" -> "1", + "spark.executor.memory" -> "26742m", + "spark.executor.memoryOverhead" -> "7372m", + "spark.executorEnv.OPENBLAS_NUM_THREADS" -> "1", + "spark.extraListeners" -> "com.google.cloud.spark.performance.DataprocMetricsListener", + "spark.rapids.memory.pinnedPool.size" -> "2048m", + "spark.scheduler.mode" -> "FAIR", + "spark.sql.cbo.enabled" -> "true", + "spark.sql.adaptive.enabled" -> "true", + "spark.ui.port" -> "0", + "spark.yarn.am.memory" -> "640m" + ) + } + + protected final def buildWorkerInfoAsString( + customProps: Option[mutable.Map[String, String]] = None, + numCores: Option[Int] = Some(32), + systemMemory: Option[String] = Some("122880MiB"), + numWorkers: Option[Int] = Some(4), + gpuCount: Option[Int] = None, + gpuMemory: Option[String] = None, + gpuDevice: Option[String] = None): String = { + val gpuWorkerProps = new GpuWorkerProps( + gpuMemory.getOrElse(""), gpuCount.getOrElse(0), gpuDevice.getOrElse("")) + val cpuSystem = new SystemClusterProps( + numCores.getOrElse(0), systemMemory.getOrElse(""), numWorkers.getOrElse(0)) + val systemProperties = customProps match { + case None => mutable.Map[String, String]() + case Some(newProps) => newProps + } + val convertedMap = new util.LinkedHashMap[String, String](systemProperties.asJava) + val clusterProps = new ClusterProperties(cpuSystem, gpuWorkerProps, convertedMap) + // set the options to convert the object into formatted yaml content + val options = new DumperOptions() + options.setIndent(2) + options.setPrettyFlow(true) + options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK) + val yaml = new Yaml(options) + val rawString = yaml.dump(clusterProps) + // Skip the first line as it contains "the name of the class" + rawString.split("\n").drop(1).mkString("\n") + } + + protected def getMockInfoProvider(maxInput: Double, + spilledMetrics: Seq[Long], + jvmGCFractions: Seq[Double], + propsFromLog: mutable.Map[String, String], + sparkVersion: Option[String], + rapidsJars: Seq[String] = Seq(), + distinctLocationPct: Double = 0.0, + redundantReadSize: Long = 0, + meanInput: Double = 0.0, + meanShuffleRead: Double = 0.0, + shuffleStagesWithPosSpilling: Set[Long] = Set(), + shuffleSkewStages: Set[Long] = Set()): AppSummaryInfoBaseProvider = { + new AppInfoProviderMockTest(maxInput, spilledMetrics, jvmGCFractions, propsFromLog, + sparkVersion, rapidsJars, distinctLocationPct, redundantReadSize, meanInput, meanShuffleRead, + shuffleStagesWithPosSpilling, shuffleSkewStages) + } +} diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala similarity index 85% rename from core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala rename to core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala index d61440195..24da7ff50 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2024, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,84 +14,27 @@ * limitations under the License. */ -package com.nvidia.spark.rapids.tool.profiling +package com.nvidia.spark.rapids.tool.tuning -import java.util - -import scala.collection.JavaConverters._ import scala.collection.mutable import com.nvidia.spark.rapids.tool.{A100Gpu, AppSummaryInfoBaseProvider, GpuDevice, PlatformFactory, PlatformNames, T4Gpu} import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper -import com.nvidia.spark.rapids.tool.profiling.AutoTuner.loadClusterPropertiesFromContent -import org.scalatest.{BeforeAndAfterEach, FunSuite} +import com.nvidia.spark.rapids.tool.profiling.{DriverLogUnsupportedOperators, Profiler} import org.scalatest.prop.TableDrivenPropertyChecks._ import org.scalatest.prop.TableFor4 -import org.yaml.snakeyaml.{DumperOptions, Yaml} -import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.tool.util.WebCrawlerUtil -case class DriverInfoProviderMockTest(unsupportedOps: Seq[DriverLogUnsupportedOperators]) - extends BaseDriverLogInfoProvider(None) { - override def getUnsupportedOperators: Seq[DriverLogUnsupportedOperators] = unsupportedOps -} - -class AppInfoProviderMockTest(val maxInput: Double, - val spilledMetrics: Seq[Long], - val jvmGCFractions: Seq[Double], - val propsFromLog: mutable.Map[String, String], - val sparkVersion: Option[String], - val rapidsJars: Seq[String], - val distinctLocationPct: Double, - val redundantReadSize: Long, - val meanInput: Double, - val meanShuffleRead: Double, - val shuffleStagesWithPosSpilling: Set[Long], - val shuffleSkewStages: Set[Long]) extends AppSummaryInfoBaseProvider { - override def isAppInfoAvailable = true - override def getMaxInput: Double = maxInput - override def getMeanInput: Double = meanInput - override def getMeanShuffleRead: Double = meanShuffleRead - override def getSpilledMetrics: Seq[Long] = spilledMetrics - override def getJvmGCFractions: Seq[Double] = jvmGCFractions - override def getRapidsProperty(propKey: String): Option[String] = propsFromLog.get(propKey) - override def getSparkProperty(propKey: String): Option[String] = propsFromLog.get(propKey) - override def getSystemProperty(propKey: String): Option[String] = propsFromLog.get(propKey) - override def getSparkVersion: Option[String] = sparkVersion - override def getRapidsJars: Seq[String] = rapidsJars - override def getDistinctLocationPct: Double = distinctLocationPct - override def getRedundantReadSize: Long = redundantReadSize - override def getShuffleStagesWithPosSpilling: Set[Long] = shuffleStagesWithPosSpilling - override def getShuffleSkewStages: Set[Long] = shuffleSkewStages -} - -class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { - - val defaultSparkVersion = "3.1.1" - - val defaultDataprocProps: mutable.Map[String, String] = { - mutable.LinkedHashMap[String, String]( - "spark.dynamicAllocation.enabled" -> "true", - "spark.driver.maxResultSize" -> "7680m", - "spark.driver.memory" -> "15360m", - "spark.executor.cores" -> "16", - "spark.executor.instances" -> "2", - "spark.executor.resource.gpu.amount" -> "1", - "spark.executor.memory" -> "26742m", - "spark.executor.memoryOverhead" -> "7372m", - "spark.executorEnv.OPENBLAS_NUM_THREADS" -> "1", - "spark.extraListeners" -> "com.google.cloud.spark.performance.DataprocMetricsListener", - "spark.rapids.memory.pinnedPool.size" -> "2048m", - "spark.scheduler.mode" -> "FAIR", - "spark.sql.cbo.enabled" -> "true", - "spark.sql.adaptive.enabled" -> "true", - "spark.ui.port" -> "0", - "spark.yarn.am.memory" -> "640m" - ) - } +/** + * Suite to test the Profiling Tool's AutoTuner + */ +class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { - private def buildWorkerInfoAsString( + /** + * Helper method to build a worker info string with GPU properties + */ + protected def buildGpuWorkerInfoAsString( customProps: Option[mutable.Map[String, String]] = None, numCores: Option[Int] = Some(32), systemMemory: Option[String] = Some("122880MiB"), @@ -99,25 +42,8 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { gpuCount: Option[Int] = Some(2), gpuMemory: Option[String] = Some(GpuDevice.DEFAULT.getMemory), gpuDevice: Option[String] = Some(GpuDevice.DEFAULT.toString)): String = { - val gpuWorkerProps = new GpuWorkerProps( - gpuMemory.getOrElse(""), gpuCount.getOrElse(0), gpuDevice.getOrElse("")) - val cpuSystem = new SystemClusterProps( - numCores.getOrElse(0), systemMemory.getOrElse(""), numWorkers.getOrElse(0)) - val systemProperties = customProps match { - case None => mutable.Map[String, String]() - case Some(newProps) => newProps - } - val convertedMap = new util.LinkedHashMap[String, String](systemProperties.asJava) - val clusterProps = new ClusterProperties(cpuSystem, gpuWorkerProps, convertedMap) - // set the options to convert the object into formatted yaml content - val options = new DumperOptions() - options.setIndent(2) - options.setPrettyFlow(true) - options.setDefaultFlowStyle(DumperOptions.FlowStyle.BLOCK) - val yaml = new Yaml(options) - val rawString = yaml.dump(clusterProps) - // Skip the first line as it contains "the name of the class" - rawString.split("\n").drop(1).mkString("\n") + buildWorkerInfoAsString(customProps, numCores, systemMemory, numWorkers, + gpuCount, gpuMemory,gpuDevice) } private def getGpuAppMockInfoProvider: AppSummaryInfoBaseProvider = { @@ -134,32 +60,17 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { Some(defaultSparkVersion), rapidsJars) } - private def getMockInfoProvider(maxInput: Double, - spilledMetrics: Seq[Long], - jvmGCFractions: Seq[Double], - propsFromLog: mutable.Map[String, String], - sparkVersion: Option[String], - rapidsJars: Seq[String] = Seq(), - distinctLocationPct: Double = 0.0, - redundantReadSize: Long = 0, - meanInput: Double = 0.0, - meanShuffleRead: Double = 0.0, - shuffleStagesWithPosSpilling: Set[Long] = Set(), - shuffleSkewStages: Set[Long] = Set()): AppSummaryInfoBaseProvider = { - new AppInfoProviderMockTest(maxInput, spilledMetrics, jvmGCFractions, propsFromLog, - sparkVersion, rapidsJars, distinctLocationPct, redundantReadSize, meanInput, meanShuffleRead, - shuffleStagesWithPosSpilling, shuffleSkewStages) - } - test("verify 3.2.0+ auto conf setting") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), Some("122880MiB"), Some(0)) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("122880MiB"), Some(0)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), mutable.Map("spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.AnotherPlugin, com.nvidia.spark.SQLPlugin"), Some("3.2.0"), Seq()) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -223,7 +134,8 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { test("Load non-existing cluster properties") { val platform = PlatformFactory.createInstance(clusterProperties = None) - val autoTuner = AutoTuner.buildAutoTuner(getGpuAppMockInfoProvider, platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTuner(getGpuAppMockInfoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -263,8 +175,9 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { } test("Load cluster properties with CPU cores 0") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(0)) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(0)) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -304,11 +217,13 @@ class AutoTunerSuite extends FunSuite with BeforeAndAfterEach with Logging { } test("Load cluster properties with memory to cores ratio to small") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(8), Some("14000MiB")) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(8), Some("14000MiB")) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, - getGpuAppMockInfoProvider, platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, + getGpuAppMockInfoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -321,8 +236,9 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." } test("Load cluster properties with CPU memory missing") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), None) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), None) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -362,11 +278,13 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." } test("Load cluster properties with CPU memory 0") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), Some("0m")) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("0m")) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -424,11 +342,13 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." } test("Load cluster properties with number of workers 0") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), Some("122880MiB"), Some(0)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("122880MiB"), Some(0)) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -477,8 +397,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- Number of workers is missing. Setting default to 1. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } @@ -494,12 +414,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps), Some(32), Some("122880MiB"), Some(4), Some(0)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -536,8 +458,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.shuffle.partitions' was not set. |- GPU count is missing. Setting default to 1. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } @@ -557,12 +479,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = - buildWorkerInfoAsString(Some(sparkProps), Some(32), Some("122880MiB"), Some(4), Some(2), None) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps), Some(32), + Some("122880MiB"), Some(4), Some(2), None) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -591,8 +515,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.shuffle.partitions' was not set. |- GPU memory is missing. Setting default to 15109m. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } @@ -614,12 +538,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.adaptive.advisoryPartitionSizeInBytes" -> "64m", "spark.sql.adaptive.coalescePartitions.minPartitionNum" -> "1") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps), Some(32), Some("122880MiB"), Some(4), Some(2), Some("0M")) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -644,8 +570,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.shuffle.partitions' was not set. |- GPU memory is missing. Setting default to 15109m. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } @@ -664,12 +590,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps), Some(32), Some("122880MiB"), Some(4), Some(2), Some("0MiB"), None) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -699,8 +627,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- GPU device is missing. Setting default to $T4Gpu. |- GPU memory is missing. Setting default to ${T4Gpu.getMemory}. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } @@ -720,12 +648,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps), Some(32), Some("122880MiB"), Some(4), Some(2), Some("0MiB"), Some("GPU-X")) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -755,19 +685,21 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- GPU device is missing. Setting default to $T4Gpu. |- GPU memory is missing. Setting default to ${T4Gpu.getMemory}. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } test("Test executor memory on CSP where executor/cpu ratio is small") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(8), Some("15360MiB"), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(8), Some("15360MiB"), Some(4), Some(1)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = @@ -809,8 +741,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.files.maxPartitionBytes' was not set. |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin assert(expectedResults == autoTunerOutput) } @@ -829,7 +761,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps)) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps)) val expectedResults = s"""| |Spark Properties: @@ -855,13 +787,15 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.shuffle.partitions' was not set. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) assert(expectedResults == autoTunerOutput) @@ -884,7 +818,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps)) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps)) val expectedResults = s"""| |Spark Properties: @@ -910,20 +844,22 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.shuffle.partitions' was not set. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) assert(expectedResults == autoTunerOutput) } test("test AutoTuner with empty sparkProperties") { - val dataprocWorkerInfo = buildWorkerInfoAsString(None) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None) val expectedResults = s"""| |Spark Properties: @@ -969,13 +905,15 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.files.maxPartitionBytes' was not set. |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, - platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) assert(expectedResults == autoTunerOutput) @@ -1011,14 +949,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "4") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1048,8 +988,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |- file.encoding should be [UTF-8] because GPU only supports the charset when using some expressions. |""".stripMargin // scalastyle:on line.size.limit @@ -1084,14 +1024,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "4") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1121,8 +1063,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1148,14 +1090,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.plugins" -> "com.nvidia.spark.WrongPlugin0, com.nvidia.spark.WrongPlugin1", "spark.rapids.memory.pinnedPool.size" -> "5g", "spark.rapids.sql.concurrentGpuTasks" -> "4") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion)) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1191,8 +1135,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1221,14 +1165,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "false", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "4") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1264,8 +1210,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1299,13 +1245,15 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "1") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, - getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.01, 0.0), logEventsProps, - Some(defaultSparkVersion)), platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, + getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.01, 0.0), logEventsProps, + Some(defaultSparkVersion)), platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1333,8 +1281,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(autoTunerOutput == expectedResults) @@ -1370,14 +1318,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "1") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1406,8 +1356,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1437,14 +1387,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "1") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1473,8 +1425,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1494,11 +1446,13 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.sql.files.maxPartitionBytes" -> "512m", "spark.task.resource.gpu.amount" -> "0.0625") val sparkProps = defaultDataprocProps.++(customProps) - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(sparkProps)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(sparkProps)) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, - getGpuAppMockInfoWithJars(rapidsJars), platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, + getGpuAppMockInfoWithJars(rapidsJars), platform) val (properties, comments) = autoTuner.getRecommendedProperties() Profiler.getAutoTunerResultsAsString(properties, comments) } @@ -1506,6 +1460,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." test("Multiple RAPIDS jars trigger a comment") { // 1. The Autotuner should warn the users that they have multiple jars defined in the classPath // 2. Compare the output + // scalastyle:off line.size.limit val expectedResults = s"""| |Spark Properties: @@ -1531,9 +1486,10 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.shuffle.partitions' was not set. - |- ${AutoTuner.classPathComments("rapids.jars.multiple")} [23.06.0, 23.02.1] - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.multiple")} [23.06.0, 23.02.1] + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin + // scalastyle:on line.size.limit val rapidsJarsArr = Seq("rapids-4-spark_2.12-23.06.0-SNAPSHOT.jar", "rapids-4-spark_2.12-23.02.1.jar") val autoTunerOutput = generateRecommendationsForRapidsJars(rapidsJarsArr) @@ -1579,7 +1535,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- A newer RAPIDS Accelerator for Apache Spark plugin is available: | $pluginJarMvnURl | Version used in application is $jarVer. - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin val rapidsJarsArr = Seq(s"rapids-4-spark_2.12-$jarVer.jar") val autoTunerOutput = generateRecommendationsForRapidsJars(rapidsJarsArr) @@ -1619,7 +1575,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.shuffle.partitions' was not set. - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin val rapidsJarsArr = Seq(s"rapids-4-spark_2.12-$latestRelease.jar") val autoTunerOutput = generateRecommendationsForRapidsJars(rapidsJarsArr) @@ -1649,14 +1605,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "1") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion), Seq(), 40.0, 200000000000L) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1688,8 +1646,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. |- Enable file cache only if Spark local disks bandwidth is > 1 GB/s and you have sufficient disk space available to fit both cache and normal Spark temporary data. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1718,14 +1676,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "1") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion), Seq(), 40.0, 2000000L) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1754,19 +1714,21 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) } test("test recommendations for databricks-aws platform argument") { - val databricksWorkerInfo = buildWorkerInfoAsString() - val clusterPropsOpt = loadClusterPropertiesFromContent(databricksWorkerInfo) + val databricksWorkerInfo = buildGpuWorkerInfoAsString() + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(databricksWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATABRICKS_AWS, clusterPropsOpt) - val autoTuner = AutoTuner.buildAutoTunerFromProps(databricksWorkerInfo, - getGpuAppMockInfoProvider, platform) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(databricksWorkerInfo, + getGpuAppMockInfoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() // Assert recommendations are excluded in properties @@ -1806,14 +1768,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "4") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1840,8 +1804,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1862,10 +1826,11 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "to force onto GPU.") ) val driverInfoProvider = DriverInfoProviderMockTest(unsupportedDriverOperators) - val workerInfo = buildWorkerInfoAsString(Some(customProps)) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(workerInfo, - AppSummaryInfoBaseProvider.fromAppInfo(None), - PlatformFactory.createInstance(), driverInfoProvider) + val workerInfo = buildGpuWorkerInfoAsString(Some(customProps)) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(workerInfo, + AppSummaryInfoBaseProvider.fromAppInfo(None), + PlatformFactory.createInstance(), driverInfoProvider) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1876,7 +1841,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." | |Comments: |- 'spark.rapids.sql.incompatibleDateFormats.enabled' was not set. - |- ${AutoTuner.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} + |- ${ProfilingAutoTunerConfigsProvider.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1897,9 +1862,10 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "to force onto GPU.") ) val driverInfoProvider = DriverInfoProviderMockTest(unsupportedDriverOperators) - val workerInfo = buildWorkerInfoAsString(Some(customProps)) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(workerInfo, - getGpuAppMockInfoProvider, PlatformFactory.createInstance(), driverInfoProvider) + val workerInfo = buildGpuWorkerInfoAsString(Some(customProps)) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(workerInfo, + getGpuAppMockInfoProvider, PlatformFactory.createInstance(), driverInfoProvider) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1926,9 +1892,9 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' should be set to Min(1, (gpuCount / numCores)). |- Could not infer the cluster configuration, recommendations are generated using default values! - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} - |- ${AutoTuner.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -1940,10 +1906,11 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.executor.memory" -> "47222m", "spark.rapids.sql.concurrentGpuTasks" -> "2", "spark.task.resource.gpu.amount" -> "0.0625") - val workerInfo = buildWorkerInfoAsString(Some(customProps)) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(workerInfo, - AppSummaryInfoBaseProvider.fromAppInfo(None), - PlatformFactory.createInstance()) + val workerInfo = buildGpuWorkerInfoAsString(Some(customProps)) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(workerInfo, + AppSummaryInfoBaseProvider.fromAppInfo(None), + PlatformFactory.createInstance()) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1967,10 +1934,11 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "expression Literal 1700518632630000 produces an unsupported type TimestampType") ) val driverInfoProvider = DriverInfoProviderMockTest(unsupportedDriverOperators) - val workerInfo = buildWorkerInfoAsString(Some(customProps)) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(workerInfo, - AppSummaryInfoBaseProvider.fromAppInfo(None), - PlatformFactory.createInstance(), driverInfoProvider) + val workerInfo = buildGpuWorkerInfoAsString(Some(customProps)) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(workerInfo, + AppSummaryInfoBaseProvider.fromAppInfo(None), + PlatformFactory.createInstance(), driverInfoProvider) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2005,14 +1973,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.concurrentGpuTasks" -> "4", "spark.sql.adaptive.enabled" -> "true", "spark.sql.adaptive.autoBroadcastJoinThreshold" -> "500mb") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2043,19 +2013,19 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionNum' was not set. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- Setting 'spark.sql.adaptive.autoBroadcastJoinThreshold' > 100m could lead to performance\n regression. Should be set to a lower number. - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) } private def testPartitionConfigurations( - inputSize: Double, - shuffleRead: Double, - gpuDevice: GpuDevice, - expectedLines: Seq[String]): Unit = { + inputSize: Double, + shuffleRead: Double, + gpuDevice: GpuDevice, + expectedLines: Seq[String]): Unit = { val customProps = mutable.LinkedHashMap( "spark.executor.cores" -> "8", "spark.executor.memory" -> "47222m", @@ -2076,14 +2046,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "4") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(gpuDevice.getMemory), Some(gpuDevice.toString)) val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(defaultSparkVersion), meanInput = inputSize, meanShuffleRead = shuffleRead) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) assert(expectedLines.forall(line => autoTunerOutput.contains(line)), @@ -2141,14 +2113,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.SQLPlugin", "spark.rapids.sql.concurrentGpuTasks" -> "1") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(0, 0), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2178,36 +2152,38 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) } test("test shuffle manager version for databricks") { - val databricksWorkerInfo = buildWorkerInfoAsString(None) + val databricksWorkerInfo = buildGpuWorkerInfoAsString(None) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), mutable.Map("spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.AnotherPlugin, com.nvidia.spark.SQLPlugin", DatabricksParseHelper.PROP_TAG_CLUSTER_SPARK_VERSION_KEY -> "11.3.x-gpu-ml-scala2.12"), Some("3.3.0"), Seq()) // Do not set the platform as DB to see if it can work correctly irrespective - val autoTuner = AutoTuner.buildAutoTunerFromProps(databricksWorkerInfo, - infoProvider, PlatformFactory.createInstance()) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(databricksWorkerInfo, + infoProvider, PlatformFactory.createInstance()) val smVersion = autoTuner.getShuffleManagerClassName() // Assert shuffle manager string for DB 11.3 tag assert(smVersion.get == "com.nvidia.spark.rapids.spark330db.RapidsShuffleManager") } test("test shuffle manager version for non-databricks") { - val databricksWorkerInfo = buildWorkerInfoAsString(None) + val databricksWorkerInfo = buildGpuWorkerInfoAsString(None) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), mutable.Map("spark.rapids.sql.enabled" -> "true", "spark.plugins" -> "com.nvidia.spark.AnotherPlugin, com.nvidia.spark.SQLPlugin"), Some("3.3.0"), Seq()) - val autoTuner = AutoTuner.buildAutoTunerFromProps(databricksWorkerInfo, - infoProvider, PlatformFactory.createInstance()) + val autoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(databricksWorkerInfo, + infoProvider, PlatformFactory.createInstance()) val smVersion = autoTuner.getShuffleManagerClassName() assert(smVersion.get == "com.nvidia.spark.rapids.spark330.RapidsShuffleManager") } @@ -2235,14 +2211,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.shuffle.multiThreaded.writer.threads" -> "8", "spark.sql.adaptive.coalescePartitions.minPartitionNum" -> "1", "spark.shuffle.manager" -> "com.nvidia.spark.rapids.spark311.RapidsShuffleManager") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(1000L, 1000L), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion), shuffleStagesWithPosSpilling = Set(1)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2275,8 +2253,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' should be increased since spilling occurred in shuffle stages. |- 'spark.sql.shuffle.partitions' was not set. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -2305,15 +2283,17 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.rapids.shuffle.multiThreaded.writer.threads" -> "8", "spark.sql.adaptive.coalescePartitions.minPartitionNum" -> "1", "spark.shuffle.manager" -> "com.nvidia.spark.rapids.spark311.RapidsShuffleManager") - val dataprocWorkerInfo = buildWorkerInfoAsString(Some(customProps), Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(Some(customProps), Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(3.7449728E7, Seq(1000L, 1000L), Seq(0.4, 0.4), logEventsProps, Some(defaultSparkVersion), shuffleStagesWithPosSpilling = Set(1, 5), shuffleSkewStages = Set(1)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2345,11 +2325,11 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.adaptive.enabled' should be enabled for better performance. |- 'spark.sql.shuffle.partitions' was not set. |- Average JVM GC time is very high. Other Garbage Collectors can be used for better performance. - |- ${AutoTuner.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- Shuffle skew exists (when task's Shuffle Read Size > 3 * Avg Stage-level size) in | stages with spilling. Increasing shuffle partitions is not recommended in this | case since keys will still hash to the same task. - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -2366,14 +2346,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.executor.instances" -> "1", "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer" ) - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2416,8 +2398,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -2435,14 +2417,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer", "spark.kryo.registrator" -> "org.apache.SomeRegistrator,org.apache.SomeOtherRegistrator" ) - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2484,8 +2468,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -2503,14 +2487,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.serializer" -> "org.apache.spark.serializer.KryoSerializer", "spark.kryo.registrator" -> "" ) - val dataprocWorkerInfo = buildWorkerInfoAsString(None, Some(32), + val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), logEventsProps, Some(defaultSparkVersion)) - val clusterPropsOpt = loadClusterPropertiesFromContent(dataprocWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2552,8 +2538,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html - |- ${AutoTuner.classPathComments("rapids.jars.missing")} - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) @@ -2569,14 +2555,16 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." "spark.executor.resource.gpu.amount" -> "1", "spark.executor.instances" -> "1" ) - val emrWorkerInfo = buildWorkerInfoAsString(None, Some(32), + val emrWorkerInfo = buildGpuWorkerInfoAsString(None, Some(32), Some("212992MiB"), Some(5), Some(4), Some(T4Gpu.getMemory), Some(T4Gpu.toString)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), logEventsProps, Some("3.4.1-amzn-1")) - val clusterPropsOpt = loadClusterPropertiesFromContent(emrWorkerInfo) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(emrWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.EMR, clusterPropsOpt) - val autoTuner: AutoTuner = AutoTuner.buildAutoTunerFromProps(emrWorkerInfo, infoProvider, - platform) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(emrWorkerInfo, infoProvider, + platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2620,7 +2608,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. |- RAPIDS Accelerator for Apache Spark jar is missing in "spark.plugins". Please refer to https://docs.nvidia.com/spark-rapids/user-guide/latest/getting-started/overview.html - |- ${AutoTuner.classPathComments("rapids.shuffle.jars")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit assert(expectedResults == autoTunerOutput) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala new file mode 100644 index 000000000..336319387 --- /dev/null +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2024, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.tuning + +import scala.collection.mutable + +import com.nvidia.spark.rapids.tool.{PlatformFactory, PlatformNames} +import com.nvidia.spark.rapids.tool.profiling.Profiler + +/** + * Suite to test the Qualification Tool's AutoTuner + */ +class QualificationAutoTunerSuite extends BaseAutoTunerSuite { + + /** + * Helper method to build a worker info string with CPU properties + */ + protected def buildCpuWorkerInfoAsString( + customProps: Option[mutable.Map[String, String]] = None, + numCores: Option[Int] = Some(32), + systemMemory: Option[String] = Some("122880MiB"), + numWorkers: Option[Int] = Some(4)): String = { + buildWorkerInfoAsString(customProps, numCores, systemMemory, numWorkers) + } + + test("test AutoTuner for Qualification sets batch size to 1GB") { + // mock the properties loaded from eventLog + val logEventsProps: mutable.Map[String, String] = + mutable.LinkedHashMap[String, String]( + "spark.executor.cores" -> "32", + "spark.executor.instances" -> "1", + "spark.executor.memory" -> "80g", + "spark.executor.instances" -> "1" + ) + val workerInfo = buildCpuWorkerInfoAsString(None, Some(32), + Some("212992MiB"), Some(5)) + val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), + logEventsProps, Some(defaultSparkVersion)) + val clusterPropsOpt = QualificationAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(workerInfo) + val platform = PlatformFactory.createInstance(PlatformNames.EMR, clusterPropsOpt) + val autoTuner = QualificationAutoTunerConfigsProvider.buildAutoTunerFromProps( + workerInfo, infoProvider, platform) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = Seq( + "--conf spark.rapids.sql.batchSizeBytes=1073741824", + "- 'spark.rapids.sql.batchSizeBytes' was not set." + ) + assert(expectedResults.forall(autoTunerOutput.contains)) + } +} From 891b3b5d28eb47e01152adec78d866253def0b84 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Tue, 31 Dec 2024 13:55:30 -0800 Subject: [PATCH 27/52] Add shuffle partition conf to limited logic recommendation (#1479) Signed-off-by: Partho Sarthi --- .../spark/rapids/tool/tuning/AutoTuner.scala | 4 +- .../tool/tuning/QualificationAutoTuner.scala | 13 +++++- .../tuning/QualificationAutoTunerSuite.scala | 43 ++++++++++++++----- 3 files changed, 47 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index 9b022c826..7471491e4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -345,7 +345,7 @@ class AutoTuner( // Note that the recommendations will be computed anyway to avoid breaking dependencies. private val skippedRecommendations: mutable.HashSet[String] = mutable.HashSet[String]() // list of recommendations having the calculations disabled, and only depend on default values - private val limitedLogicRecommendations: mutable.HashSet[String] = mutable.HashSet[String]() + protected val limitedLogicRecommendations: mutable.HashSet[String] = mutable.HashSet[String]() // When enabled, the profiler recommendations should only include updated settings. private var filterByUpdatedPropertiesEnabled: Boolean = true @@ -1032,10 +1032,10 @@ class AutoTuner( val lookup = "spark.sql.shuffle.partitions" var shufflePartitions = getPropertyValue(lookup).getOrElse(autoTunerConfigsProvider.DEF_SHUFFLE_PARTITIONS).toInt - val shuffleStagesWithPosSpilling = appInfoProvider.getShuffleStagesWithPosSpilling // TODO: Need to look at other metrics for GPU spills (DEBUG mode), and batch sizes metric if (isCalculationEnabled(lookup)) { + val shuffleStagesWithPosSpilling = appInfoProvider.getShuffleStagesWithPosSpilling if (shuffleStagesWithPosSpilling.nonEmpty) { val shuffleSkewStages = appInfoProvider.getShuffleSkewStages if (shuffleSkewStages.exists(id => shuffleStagesWithPosSpilling.contains(id))) { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala index 21c253f5e..f10a7330f 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTuner.scala @@ -16,6 +16,8 @@ package com.nvidia.spark.rapids.tool.tuning +import scala.collection.mutable + import com.nvidia.spark.rapids.tool.{AppSummaryInfoBaseProvider, Platform} import com.nvidia.spark.rapids.tool.profiling.DriverLogInfoProvider @@ -29,7 +31,16 @@ class QualificationAutoTuner( platform: Platform, driverInfoProvider: DriverLogInfoProvider) extends AutoTuner(clusterProps, appInfoProvider, platform, driverInfoProvider, - QualificationAutoTunerConfigsProvider) + QualificationAutoTunerConfigsProvider) { + + /** + * List of recommendations for which the Qualification AutoTuner skips calculations and only + * depend on default values. + */ + override protected val limitedLogicRecommendations: mutable.HashSet[String] = mutable.HashSet( + "spark.sql.shuffle.partitions" + ) +} /** * Provides configuration settings for the Qualification Tool's AutoTuner diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala index 336319387..3410d2a75 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala @@ -26,6 +26,18 @@ import com.nvidia.spark.rapids.tool.profiling.Profiler */ class QualificationAutoTunerSuite extends BaseAutoTunerSuite { + /** + * Default Spark properties to be used when building the Qualification AutoTuner + */ + private val defaultSparkProps: mutable.Map[String, String] = { + mutable.LinkedHashMap[String, String]( + "spark.executor.cores" -> "32", + "spark.executor.instances" -> "1", + "spark.executor.memory" -> "80g", + "spark.executor.instances" -> "1" + ) + } + /** * Helper method to build a worker info string with CPU properties */ @@ -37,15 +49,11 @@ class QualificationAutoTunerSuite extends BaseAutoTunerSuite { buildWorkerInfoAsString(customProps, numCores, systemMemory, numWorkers) } - test("test AutoTuner for Qualification sets batch size to 1GB") { - // mock the properties loaded from eventLog - val logEventsProps: mutable.Map[String, String] = - mutable.LinkedHashMap[String, String]( - "spark.executor.cores" -> "32", - "spark.executor.instances" -> "1", - "spark.executor.memory" -> "80g", - "spark.executor.instances" -> "1" - ) + /** + * Helper method to return an instance of the Qualification AutoTuner with default properties + */ + private def buildDefaultAutoTuner( + logEventsProps: mutable.Map[String, String] = defaultSparkProps): AutoTuner = { val workerInfo = buildCpuWorkerInfoAsString(None, Some(32), Some("212992MiB"), Some(5)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), @@ -53,8 +61,12 @@ class QualificationAutoTunerSuite extends BaseAutoTunerSuite { val clusterPropsOpt = QualificationAutoTunerConfigsProvider .loadClusterPropertiesFromContent(workerInfo) val platform = PlatformFactory.createInstance(PlatformNames.EMR, clusterPropsOpt) - val autoTuner = QualificationAutoTunerConfigsProvider.buildAutoTunerFromProps( + QualificationAutoTunerConfigsProvider.buildAutoTunerFromProps( workerInfo, infoProvider, platform) + } + + test("test AutoTuner for Qualification sets batch size to 1GB") { + val autoTuner = buildDefaultAutoTuner() val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) val expectedResults = Seq( @@ -63,4 +75,15 @@ class QualificationAutoTunerSuite extends BaseAutoTunerSuite { ) assert(expectedResults.forall(autoTunerOutput.contains)) } + + test("test AutoTuner for Qualification sets shuffle partitions to 200") { + val autoTuner = buildDefaultAutoTuner() + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val expectedResults = Seq( + "--conf spark.sql.shuffle.partitions=200", + "- 'spark.sql.shuffle.partitions' was not set." + ) + assert(expectedResults.forall(autoTunerOutput.contains)) + } } From 5755cfc21c8cf9d3ce96897aff62d8cc8f30f3c7 Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Thu, 2 Jan 2025 14:14:02 -0800 Subject: [PATCH 28/52] Fix inconsistent shuffle write time sum results in Profiler output (#1450) * keep nano sec unit for shuffle write time in taskmodel and convert to ms only in output * refactored executorCpuTime and executorDeserializeCpuTime * cap cpu percent at 100 --------- Signed-off-by: cindyyuanjiang --- .../analysis/AppSparkMetricsAnalyzer.scala | 6 +- .../tool/analysis/util/AggAccumHelper.scala | 8 ++ .../tool/analysis/util/StageAggPhoton.scala | 4 +- .../tool/profiling/GenerateTimeline.scala | 2 +- .../profiling/ProfileClassWarehouse.scala | 121 +++++++++++++----- .../spark/sql/rapids/tool/ToolUtils.scala | 33 +++-- .../sql/rapids/tool/store/TaskModel.scala | 18 ++- ...on_db_13_3_job_metrics_agg_expectation.csv | 30 ++--- ...on_db_13_3_sql_metrics_agg_expectation.csv | 2 +- ..._db_13_3_stage_metrics_agg_expectation.csv | 30 ++--- .../rapids_duration_and_cpu_expectation.csv | 10 +- ...in_eventlog_jobmetricsagg2_expectation.csv | 2 +- ...oin_eventlog_jobmetricsagg_expectation.csv | 2 +- ...ventlog_jobmetricsaggmulti_expectation.csv | 4 +- ...in_eventlog_sqlmetricsagg2_expectation.csv | 2 +- ...oin_eventlog_sqlmetricsagg_expectation.csv | 2 +- ...ventlog_sqlmetricsaggmulti_expectation.csv | 4 +- ..._eventlog_stagemetricsagg2_expectation.csv | 6 +- ...n_eventlog_stagemetricsagg_expectation.csv | 6 +- ...ntlog_stagemetricsaggmulti_expectation.csv | 12 +- 20 files changed, 190 insertions(+), 114 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 6b8c3d5e5..b68f18899 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -426,8 +426,8 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { perStageRec.durationMax, perStageRec.durationMin, perStageRec.durationAvg, - perStageRec.executorCPUTimeSum, - perStageRec.executorDeserializeCpuTimeSum, + perStageRec.executorCPUTimeSum, // converted to milliseconds by the aggregator + perStageRec.executorDeserializeCpuTimeSum, // converted to milliseconds by the aggregator perStageRec.executorDeserializeTimeSum, perStageRec.executorRunTimeSum, perStageRec.inputBytesReadSum, @@ -448,7 +448,7 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { perStageRec.srTotalBytesReadSum, perStageRec.swBytesWrittenSum, perStageRec.swRecordsWrittenSum, - perStageRec.swWriteTimeSum) + perStageRec.swWriteTimeSum) // converted to milliseconds by the aggregator stageLevelSparkMetrics(index).put(sm.stageInfo.stageId, stageRow) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala index b42ac08b4..af76a853d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/AggAccumHelper.scala @@ -16,6 +16,8 @@ package com.nvidia.spark.rapids.tool.analysis.util +import java.util.concurrent.TimeUnit + import com.nvidia.spark.rapids.tool.profiling.StageAggTaskMetricsProfileResult import org.apache.spark.sql.rapids.tool.store.TaskModel @@ -42,6 +44,12 @@ class AggAccumHelper { val resRec = createStageAccumRecord() taskRecords.foreach(resRec.addRecord) resRec.finalizeAggregation() + // convert the nanoseconds units to milliseconds for stage level. + // This helps to avoid overflow when aggregating across multiple stages on the level of SQL/Job. + resRec.executorCPUTimeSum = TimeUnit.NANOSECONDS.toMillis(resRec.executorCPUTimeSum) + resRec.executorDeserializeCpuTimeSum = + TimeUnit.NANOSECONDS.toMillis(resRec.executorDeserializeCpuTimeSum) + resRec.swWriteTimeSum = TimeUnit.NANOSECONDS.toMillis(resRec.swWriteTimeSum) resRec } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala index ed7127050..02003941b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/StageAggPhoton.scala @@ -16,8 +16,6 @@ package com.nvidia.spark.rapids.tool.analysis.util -import java.util.concurrent.TimeUnit - import com.nvidia.spark.rapids.tool.profiling.StageAggTaskMetricsProfileResult /** @@ -43,7 +41,7 @@ case class StageAggPhoton( // Re-calculate the photon specific fields only if the accumulator has tasks. // Otherwise, leave it as 0. if (shuffleWriteValues.nonEmpty) { - swWriteTimeSum = TimeUnit.NANOSECONDS.toMillis(shuffleWriteValues.sum) + swWriteTimeSum = shuffleWriteValues.sum } if (peakMemValues.nonEmpty) { peakExecutionMemoryMax = peakMemValues.max diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala index 047657ed2..7943e8de8 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala @@ -304,7 +304,7 @@ object GenerateTimeline { tc.sr_fetchWaitTime val opTimeMs = opMetrics.flatMap(_.taskUpdatesMap.get(taskId)).sum / 1000000 val writeTimeMs = writeMetrics.flatMap(_.taskUpdatesMap.get(taskId)).sum / 1000000 + - tc.sw_writeTime + tc.sw_writeTime / 1000000 val taskInfo = new TimelineTaskInfo(stageId, taskId, launchTime, finishTime, duration, tc.executorDeserializeTime, readTimeMs, semTimeMs, opTimeMs, writeTimeMs) val execHost = s"$execId/$host" diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index ab5abdd2b..2b1131fa4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -421,8 +421,8 @@ trait BaseJobStageAggTaskMetricsProfileResult extends ProfileResult { def durationMax: Long def durationMin: Long def durationAvg: Double - def executorCPUTimeSum: Long - def executorDeserializeCpuTimeSum: Long + def executorCPUTimeSum: Long // milliseconds + def executorDeserializeCpuTimeSum: Long // milliseconds def executorDeserializeTimeSum: Long def executorRunTimeSum: Long def inputBytesReadSum: Long @@ -443,20 +443,44 @@ trait BaseJobStageAggTaskMetricsProfileResult extends ProfileResult { def srTotalBytesReadSum: Long def swBytesWrittenSum: Long def swRecordsWrittenSum: Long - def swWriteTimeSum: Long + def swWriteTimeSum: Long // milliseconds def idHeader: String - override val outputHeaders = Seq("appIndex", idHeader, "numTasks", "Duration", - "diskBytesSpilled_sum", "duration_sum", "duration_max", "duration_min", - "duration_avg", "executorCPUTime_sum", "executorDeserializeCPUTime_sum", - "executorDeserializeTime_sum", "executorRunTime_sum", "input_bytesRead_sum", - "input_recordsRead_sum", "jvmGCTime_sum", "memoryBytesSpilled_sum", - "output_bytesWritten_sum", "output_recordsWritten_sum", "peakExecutionMemory_max", - "resultSerializationTime_sum", "resultSize_max", "sr_fetchWaitTime_sum", - "sr_localBlocksFetched_sum", "sr_localBytesRead_sum", "sr_remoteBlocksFetched_sum", - "sr_remoteBytesRead_sum", "sr_remoteBytesReadToDisk_sum", "sr_totalBytesRead_sum", - "sw_bytesWritten_sum", "sw_recordsWritten_sum", "sw_writeTime_sum") + override val outputHeaders = { + Seq("appIndex", + idHeader, + "numTasks", + "Duration", + "diskBytesSpilled_sum", + "duration_sum", + "duration_max", + "duration_min", + "duration_avg", + "executorCPUTime_sum", + "executorDeserializeCPUTime_sum", + "executorDeserializeTime_sum", + "executorRunTime_sum", + "input_bytesRead_sum", + "input_recordsRead_sum", + "jvmGCTime_sum", + "memoryBytesSpilled_sum", + "output_bytesWritten_sum", + "output_recordsWritten_sum", + "peakExecutionMemory_max", + "resultSerializationTime_sum", + "resultSize_max", + "sr_fetchWaitTime_sum", + "sr_localBlocksFetched_sum", + "sr_localBytesRead_sum", + "sr_remoteBlocksFetched_sum", + "sr_remoteBytesRead_sum", + "sr_remoteBytesReadToDisk_sum", + "sr_totalBytesRead_sum", + "sw_bytesWritten_sum", + "sw_recordsWritten_sum", + "sw_writeTime_sum") + } val durStr = duration match { case Some(dur) => dur.toString @@ -511,8 +535,8 @@ case class JobAggTaskMetricsProfileResult( durationMax: Long, durationMin: Long, durationAvg: Double, - executorCPUTimeSum: Long, - executorDeserializeCpuTimeSum: Long, + executorCPUTimeSum: Long, // milliseconds + executorDeserializeCpuTimeSum: Long, // milliseconds executorDeserializeTimeSum: Long, executorRunTimeSum: Long, inputBytesReadSum: Long, @@ -533,7 +557,8 @@ case class JobAggTaskMetricsProfileResult( srTotalBytesReadSum: Long, swBytesWrittenSum: Long, swRecordsWrittenSum: Long, - swWriteTimeSum: Long) extends BaseJobStageAggTaskMetricsProfileResult { + swWriteTimeSum: Long // milliseconds + ) extends BaseJobStageAggTaskMetricsProfileResult { override def idHeader = "jobId" } @@ -547,8 +572,8 @@ case class StageAggTaskMetricsProfileResult( durationMax: Long, durationMin: Long, durationAvg: Double, - executorCPUTimeSum: Long, - executorDeserializeCpuTimeSum: Long, + executorCPUTimeSum: Long, // milliseconds + executorDeserializeCpuTimeSum: Long, // milliseconds executorDeserializeTimeSum: Long, executorRunTimeSum: Long, inputBytesReadSum: Long, @@ -569,7 +594,8 @@ case class StageAggTaskMetricsProfileResult( srTotalBytesReadSum: Long, swBytesWrittenSum: Long, swRecordsWrittenSum: Long, - swWriteTimeSum: Long) extends BaseJobStageAggTaskMetricsProfileResult { + swWriteTimeSum: Long // milliseconds + ) extends BaseJobStageAggTaskMetricsProfileResult { override def idHeader = "stageId" } @@ -747,7 +773,7 @@ case class SQLTaskAggMetricsProfileResult( description: String, numTasks: Int, duration: Option[Long], - executorCpuTime: Long, + executorCpuTime: Long, // milliseconds executorRunTime: Long, executorCpuRatio: Double, diskBytesSpilledSum: Long, @@ -755,8 +781,8 @@ case class SQLTaskAggMetricsProfileResult( durationMax: Long, durationMin: Long, durationAvg: Double, - executorCPUTimeSum: Long, - executorDeserializeCpuTimeSum: Long, + executorCPUTimeSum: Long, // milliseconds + executorDeserializeCpuTimeSum: Long, // milliseconds executorDeserializeTimeSum: Long, executorRunTimeSum: Long, inputBytesReadSum: Long, @@ -779,7 +805,8 @@ case class SQLTaskAggMetricsProfileResult( srTotalBytesReadSum: Long, swBytesWrittenSum: Long, swRecordsWrittenSum: Long, - swWriteTimeSum: Long) extends ProfileResult { + swWriteTimeSum: Long // milliseconds + ) extends ProfileResult { override val outputHeaders = Seq("appIndex", "appID", "sqlID", "description", "numTasks", "Duration", "executorCPUTime", "executorRunTime", "executorCPURatio", @@ -924,12 +951,27 @@ case class IOAnalysisProfileResult( } } -case class SQLDurationExecutorTimeProfileResult(appIndex: Int, appId: String, - rootsqlID: Option[Long], sqlID: Long, duration: Option[Long], containsDataset: Boolean, - appDuration: Option[Long], potentialProbs: String, +case class SQLDurationExecutorTimeProfileResult( + appIndex: Int, + appId: String, + rootsqlID: Option[Long], + sqlID: Long, + duration: Option[Long], + containsDataset: Boolean, + appDuration: Option[Long], + potentialProbs: String, executorCpuRatio: Double) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "App ID", "RootSqlID", "sqlID", "SQL Duration", - "Contains Dataset or RDD Op", "App Duration", "Potential Problems", "Executor CPU Time Percent") + override val outputHeaders = { + Seq("appIndex", + "App ID", + "RootSqlID", + "sqlID", + "SQL Duration", + "Contains Dataset or RDD Op", + "App Duration", + "Potential Problems", + "Executor CPU Time Percent") + } val durStr = duration match { case Some(dur) => dur.toString case None => "" @@ -950,14 +992,27 @@ case class SQLDurationExecutorTimeProfileResult(appIndex: Int, appId: String, } override def convertToSeq: Seq[String] = { - Seq(appIndex.toString, rootsqlID.getOrElse("").toString, appId, sqlID.toString, durStr, - containsDataset.toString, appDurStr, potentialStr, execCpuTimePercent) + Seq(appIndex.toString, + rootsqlID.getOrElse("").toString, + appId, + sqlID.toString, + durStr, + containsDataset.toString, + appDurStr, + potentialStr, + execCpuTimePercent) } override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, StringUtils.reformatCSVString(appId), rootsqlID.getOrElse("").toString, - sqlID.toString, durStr, containsDataset.toString, appDurStr, - StringUtils.reformatCSVString(potentialStr), execCpuTimePercent) + Seq(appIndex.toString, + StringUtils.reformatCSVString(appId), + rootsqlID.getOrElse("").toString, + sqlID.toString, + durStr, + containsDataset.toString, + appDurStr, + StringUtils.reformatCSVString(potentialStr), + execCpuTimePercent) } } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala index 455f19147..b088762f2 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ToolUtils.scala @@ -113,16 +113,33 @@ object ToolUtils extends Logging { df.showString(numRows, 0) } - // given to duration values, calculate a human readable percent - // rounded to 2 decimal places. ie 39.12% - def calculateDurationPercent(first: Long, total: Long): Double = { - val firstDec = BigDecimal.decimal(first) - val totalDec = BigDecimal.decimal(total) - if (firstDec == 0 || totalDec == 0) { + /** + * Calculate the duration percent given the numerator and total values. + * This is used to calculate the CPURatio which represents the percentage of CPU time to + * the runTime. + * There is an implicit check to ensure that the denominator is not zero. If it is, then the + * ratio will be set to 0. + * There is an option to force the cap to 100% if the calculated value is greater + * than the total. This is possible to happen because the tasks CPUTime is measured in + * nanoseconds, while the runtTime is measured in milliseconds. This leads to a loss of precision + * causing the total percentage to exceed 100%. + * @param numerator the numerator value. + * @param total the total value. + * @param forceCap if true, then the value is capped at 100%. + * @return the calculated percentage. + */ + def calculateDurationPercent(numerator: Long, total: Long, forceCap: Boolean = true): Double = { + if (numerator == 0 || total == 0) { 0.toDouble } else { - val res = (firstDec / totalDec) * 100 - formatDoubleValue(res, 2) + val numeratorDec = BigDecimal.decimal(numerator) + val totalDec = BigDecimal.decimal(total) + val res = formatDoubleValue((numeratorDec / totalDec) * 100, 2) + if (forceCap) { + math.min(res, 100) + } else { + res + } } } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala index 39b396bfc..9ae2f3579 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/TaskModel.scala @@ -16,8 +16,6 @@ package org.apache.spark.sql.rapids.tool.store -import java.util.concurrent.TimeUnit - import org.apache.spark.TaskFailedReason import org.apache.spark.scheduler.SparkListenerTaskEnd import org.apache.spark.sql.rapids.tool.annotation.Since @@ -40,9 +38,9 @@ case class TaskModel( speculative: Boolean, gettingResultTime: Long, executorDeserializeTime: Long, - executorDeserializeCPUTime: Long, - executorRunTime: Long, - executorCPUTime: Long, + executorDeserializeCPUTime: Long, // nanoseconds + executorRunTime: Long, // milliseconds + executorCPUTime: Long, // nanoseconds peakExecutionMemory: Long, resultSize: Long, jvmGCTime: Long, @@ -59,7 +57,7 @@ case class TaskModel( sr_totalBytesRead: Long, // Note: sw stands for ShuffleWrite sw_bytesWritten: Long, - sw_writeTime: Long, + sw_writeTime: Long, // nanoseconds sw_recordsWritten: Long, input_bytesRead: Long, input_recordsRead: Long, @@ -92,9 +90,9 @@ object TaskModel { event.taskInfo.speculative, event.taskInfo.gettingResultTime, event.taskMetrics.executorDeserializeTime, - TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.executorDeserializeCpuTime), - event.taskMetrics.executorRunTime, - TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.executorCpuTime), + event.taskMetrics.executorDeserializeCpuTime, // nanoseconds + event.taskMetrics.executorRunTime, // milliseconds + event.taskMetrics.executorCpuTime, // nanoseconds event.taskMetrics.peakExecutionMemory, event.taskMetrics.resultSize, event.taskMetrics.jvmGCTime, @@ -109,7 +107,7 @@ object TaskModel { event.taskMetrics.shuffleReadMetrics.localBytesRead, event.taskMetrics.shuffleReadMetrics.totalBytesRead, event.taskMetrics.shuffleWriteMetrics.bytesWritten, - TimeUnit.NANOSECONDS.toMillis(event.taskMetrics.shuffleWriteMetrics.writeTime), + event.taskMetrics.shuffleWriteMetrics.writeTime, // nanoseconds event.taskMetrics.shuffleWriteMetrics.recordsWritten, event.taskMetrics.inputMetrics.bytesRead, event.taskMetrics.inputMetrics.recordsRead, diff --git a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_job_metrics_agg_expectation.csv b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_job_metrics_agg_expectation.csv index 97de9ad08..4df39b334 100644 --- a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_job_metrics_agg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_job_metrics_agg_expectation.csv @@ -1,16 +1,16 @@ appIndex,jobId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,48,431,237976,0,371230,1032,333,861.3,343288,1261,1286,367785,190131859,8639936081,160,0,0,0,169667947,21,15767,8,3,15657,3,15657,0,31314,18964,431,16 -1,47,432,214657,0,376777,1133,499,872.2,346230,1272,1310,373271,14007280,8639936081,144,0,0,0,159530057,11,15577,4,3,15657,3,15657,0,31314,19008,432,20 -1,46,433,191440,0,457364,3323,391,1056.3,352763,1508,2509,451358,5242628040,8639936081,1912,0,0,0,250840493,9,16203,551,3,15657,3,15657,0,31314,19052,433,16 +1,48,431,237976,0,371230,1032,333,861.3,343507,1365,1286,367785,190131859,8639936081,160,0,0,0,169667947,21,15767,8,3,15657,3,15657,0,31314,18964,431,16 +1,47,432,214657,0,376777,1133,499,872.2,346447,1388,1310,373271,14007280,8639936081,144,0,0,0,159530057,11,15577,4,3,15657,3,15657,0,31314,19008,432,20 +1,46,433,191440,0,457364,3323,391,1056.3,352977,1639,2509,451358,5242628040,8639936081,1912,0,0,0,250840493,9,16203,551,3,15657,3,15657,0,31314,19052,433,16 1,49,1,186241,0,266,266,266,266.0,86,1,1,261,0,0,0,0,0,0,138414192,0,5344,10,209,9196,222,9768,0,18964,44,1,0 -1,45,433,166081,0,415849,1448,339,960.4,349795,1302,1375,412139,2276478144,8639936081,568,0,0,0,195992906,2,15780,7,3,15657,3,15657,0,31314,19052,433,34 -1,44,431,139667,0,398973,1403,365,925.7,354119,1281,1327,395265,1075691986,8639936081,328,0,0,0,188587155,0,15767,10,3,15657,3,15657,0,31314,18964,431,17 +1,45,433,166081,0,415849,1448,339,960.4,350015,1415,1375,412139,2276478144,8639936081,568,0,0,0,195992906,2,15780,7,3,15657,3,15657,0,31314,19052,433,34 +1,44,431,139667,0,398973,1403,365,925.7,354332,1420,1327,395265,1075691986,8639936081,328,0,0,0,188587155,0,15767,10,3,15657,3,15657,0,31314,18964,431,17 1,50,1,122711,0,267,267,267,267.0,71,1,1,262,0,0,0,0,0,0,138414192,0,5343,58,219,9636,213,9372,0,19008,44,1,0 -1,43,432,114755,0,403652,1369,329,934.4,353325,1290,1326,399766,1395949742,8639936081,624,0,0,0,201771890,13,15767,14,3,15657,3,15657,0,31314,19008,432,16 +1,43,432,114755,0,403652,1369,329,934.4,353529,1424,1326,399766,1395949742,8639936081,624,0,0,0,201771890,13,15767,14,3,15657,3,15657,0,31314,19008,432,16 1,51,1,97958,0,386,386,386,386.0,60,1,1,381,0,0,0,0,0,0,138414192,0,5343,154,221,9724,210,9240,0,18964,44,1,0 -1,42,431,89634,0,616500,1899,589,1430.4,378074,1330,1515,612098,16461920726,8639936081,4132,0,0,0,216740322,23,15805,10,3,15657,3,15657,0,31314,18964,431,16 +1,42,431,89634,0,616500,1899,589,1430.4,378287,1521,1515,612098,16461920726,8639936081,4132,0,0,0,216740322,23,15805,10,3,15657,3,15657,0,31314,18964,431,16 1,52,1,71718,0,384,384,384,384.0,54,1,1,379,0,0,0,0,0,0,138414192,0,5343,170,223,9812,210,9240,0,19052,44,1,0 -1,41,431,51085,0,759623,2321,918,1762.5,394996,1460,2027,754015,26337468742,8639936081,7772,0,0,0,250648581,87,16157,170,3,15657,3,15657,0,31314,18964,431,19 +1,41,431,51085,0,759623,2321,918,1762.5,395214,1706,2027,754015,26337468742,8639936081,7772,0,0,0,250648581,87,16157,170,3,15657,3,15657,0,31314,18964,431,19 1,53,1,46297,0,136,136,136,136.0,57,1,1,131,0,0,0,0,0,0,138414192,0,5344,0,214,9416,219,9636,0,19052,44,1,0 1,54,1,23051,0,340,340,340,340.0,36,1,1,334,0,0,0,0,0,0,138414192,0,5343,223,215,9460,217,9548,0,19008,44,1,0 1,31,1,6979,0,6738,6738,6738,6738.0,5104,128,688,6035,349526,86400,53,0,0,0,155563380,1,10759,0,0,0,0,0,0,0,7239,1800,0 @@ -19,20 +19,20 @@ appIndex,jobId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max, 1,35,1,6925,0,6729,6729,6729,6729.0,157,136,681,6035,12261,1350,53,0,0,0,155199546,1,9839,0,0,0,0,0,0,0,699,165,0 1,38,1,6855,0,6743,6743,6743,6743.0,187,256,688,6035,349526,86400,53,0,0,0,155563380,1,9927,0,0,0,0,0,0,0,7239,1800,0 1,0,1,6033,0,5699,5699,5699,5699.0,422,948,1114,4382,0,0,37,0,0,0,0,8,2794,0,0,0,0,0,0,0,0,0,0 -1,13,200,5707,0,87661,966,349,438.3,9821,427,951,84265,0,0,144,0,0,0,0,9,6258,0,0,0,0,0,0,0,0,0,0 -1,23,200,5479,0,84240,490,355,421.2,5290,200,214,82784,0,0,136,0,0,0,0,0,6214,0,0,0,0,0,0,0,0,0,0 -1,21,200,5271,0,80904,485,353,404.5,6004,203,220,79384,0,0,136,0,0,0,0,1,6302,0,0,0,0,0,0,0,0,0,0 -1,27,200,4728,0,70760,442,309,353.8,4042,200,209,69494,0,0,152,0,0,0,0,10,5788,0,0,0,0,0,0,0,0,0,0 +1,13,200,5707,0,87661,966,349,438.3,9924,528,951,84265,0,0,144,0,0,0,0,9,6258,0,0,0,0,0,0,0,0,0,0 +1,23,200,5479,0,84240,490,355,421.2,5394,292,214,82784,0,0,136,0,0,0,0,0,6214,0,0,0,0,0,0,0,0,0,0 +1,21,200,5271,0,80904,485,353,404.5,6100,304,220,79384,0,0,136,0,0,0,0,1,6302,0,0,0,0,0,0,0,0,0,0 +1,27,200,4728,0,70760,442,309,353.8,4145,287,209,69494,0,0,152,0,0,0,0,10,5788,0,0,0,0,0,0,0,0,0,0 1,3,1,4708,0,4693,4693,4693,4693.0,280,701,804,3796,0,0,26,0,0,0,0,7,2834,0,0,0,0,0,0,0,0,0,0 -1,25,200,4603,0,70379,569,314,351.9,4106,200,216,69040,0,0,168,0,0,0,0,14,5708,0,0,0,0,0,0,0,0,0,0 +1,25,200,4603,0,70379,569,314,351.9,4200,294,216,69040,0,0,168,0,0,0,0,14,5708,0,0,0,0,0,0,0,0,0,0 1,36,1,4556,0,4332,4332,4332,4332.0,3359,95,401,3907,30328,7200,39,0,0,0,155245068,1,10552,0,0,0,0,0,0,0,7719,1920,0 -1,29,200,4555,0,69682,423,310,348.4,3730,200,218,68521,0,0,168,0,0,0,0,9,5748,0,0,0,0,0,0,0,0,0,0 +1,29,200,4555,0,69682,423,310,348.4,3830,272,218,68521,0,0,168,0,0,0,0,9,5748,0,0,0,0,0,0,0,0,0,0 1,32,1,4515,0,4334,4334,4334,4334.0,260,130,404,3907,349526,86400,39,0,0,0,155563380,1,9851,0,0,0,0,0,0,0,7239,1800,0 1,39,1,4488,0,4322,4322,4322,4322.0,112,124,392,3907,349526,86400,39,0,0,0,155563380,1,9926,0,0,0,0,0,0,0,7239,1800,0 1,37,1,4481,0,4334,4334,4334,4334.0,136,144,405,3907,349526,86400,39,0,0,0,155563380,1,9851,0,0,0,0,0,0,0,7239,1800,0 1,40,1,4476,0,4327,4327,4327,4327.0,98,147,394,3907,349526,86400,39,0,0,0,155563380,1,9895,0,0,0,0,0,0,0,7239,1800,0 1,56,1,1055,0,1022,1022,1022,1022.0,758,77,95,901,0,0,0,0,0,0,134218344,6,10091,5,218,9592,220,9680,0,19272,0,0,0 -1,19,200,803,0,11895,145,38,59.5,943,209,252,10017,0,0,56,0,0,0,0,22,2739,0,0,0,0,0,0,0,0,0,0 +1,19,200,803,0,11895,145,38,59.5,1050,321,252,10017,0,0,56,0,0,0,0,22,2739,0,0,0,0,0,0,0,0,0,0 1,26,1,316,0,312,312,312,312.0,2,1,1,306,0,0,0,0,0,0,0,0,3777,0,0,0,0,0,0,0,0,0,0 1,2,1,280,0,267,267,267,267.0,6,4,4,124,0,0,0,0,0,0,0,1,3342,0,0,0,0,0,0,0,0,0,0 1,11,1,264,0,254,254,254,254.0,5,3,3,241,0,0,0,0,0,0,0,0,2913,0,0,0,0,0,0,0,0,0,0 diff --git a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv index 63aaf7a1b..5e1116c98 100644 --- a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv @@ -1,2 +1,2 @@ appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"app-20240919162642-0000",26,"query88",3472,250542,2883837,3818106,75.53,0,3858136,6743,54,1111.2,2883837,12349,18186,3818106,52997115316,69120188398,16100,0,0,0,250840493,181,16203,1394,1759,201596,1750,201200,0,402796,218614,19946,154 +1,"app-20240919162642-0000",26,"query88",3472,250542,2885555,3818106,75.58,0,3858136,6743,54,1111.2,2885555,13523,18186,3818106,52997115316,69120188398,16100,0,0,0,250840493,181,16203,1394,1759,201596,1750,201200,0,402796,218614,19946,154 diff --git a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_stage_metrics_agg_expectation.csv b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_stage_metrics_agg_expectation.csv index ae34c1ac0..d50fa6860 100644 --- a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_stage_metrics_agg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_stage_metrics_agg_expectation.csv @@ -1,16 +1,16 @@ appIndex,stageId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,58,431,237799,0,371230,1032,333,861.3,343288,1261,1286,367785,190131859,8639936081,160,0,0,0,169667947,21,15767,8,3,15657,3,15657,0,31314,18964,431,16 -1,54,432,214633,0,376777,1133,499,872.2,346230,1272,1310,373271,14007280,8639936081,144,0,0,0,159530057,11,15577,4,3,15657,3,15657,0,31314,19008,432,20 -1,44,433,191384,0,457364,3323,391,1056.3,352763,1508,2509,451358,5242628040,8639936081,1912,0,0,0,250840493,9,16203,551,3,15657,3,15657,0,31314,19052,433,16 +1,58,431,237799,0,371230,1032,333,861.3,343507,1365,1286,367785,190131859,8639936081,160,0,0,0,169667947,21,15767,8,3,15657,3,15657,0,31314,18964,431,16 +1,54,432,214633,0,376777,1133,499,872.2,346447,1388,1310,373271,14007280,8639936081,144,0,0,0,159530057,11,15577,4,3,15657,3,15657,0,31314,19008,432,20 +1,44,433,191384,0,457364,3323,391,1056.3,352977,1639,2509,451358,5242628040,8639936081,1912,0,0,0,250840493,9,16203,551,3,15657,3,15657,0,31314,19052,433,16 1,61,1,186240,0,266,266,266,266.0,86,1,1,261,0,0,0,0,0,0,138414192,0,5344,10,209,9196,222,9768,0,18964,44,1,0 -1,46,433,166015,0,415849,1448,339,960.4,349795,1302,1375,412139,2276478144,8639936081,568,0,0,0,195992906,2,15780,7,3,15657,3,15657,0,31314,19052,433,34 -1,50,431,139628,0,398973,1403,365,925.7,354119,1281,1327,395265,1075691986,8639936081,328,0,0,0,188587155,0,15767,10,3,15657,3,15657,0,31314,18964,431,17 +1,46,433,166015,0,415849,1448,339,960.4,350015,1415,1375,412139,2276478144,8639936081,568,0,0,0,195992906,2,15780,7,3,15657,3,15657,0,31314,19052,433,34 +1,50,431,139628,0,398973,1403,365,925.7,354332,1420,1327,395265,1075691986,8639936081,328,0,0,0,188587155,0,15767,10,3,15657,3,15657,0,31314,18964,431,17 1,64,1,122708,0,267,267,267,267.0,71,1,1,262,0,0,0,0,0,0,138414192,0,5343,58,219,9636,213,9372,0,19008,44,1,0 -1,48,432,114722,0,403652,1369,329,934.4,353325,1290,1326,399766,1395949742,8639936081,624,0,0,0,201771890,13,15767,14,3,15657,3,15657,0,31314,19008,432,16 +1,48,432,114722,0,403652,1369,329,934.4,353529,1424,1326,399766,1395949742,8639936081,624,0,0,0,201771890,13,15767,14,3,15657,3,15657,0,31314,19008,432,16 1,67,1,97957,0,386,386,386,386.0,60,1,1,381,0,0,0,0,0,0,138414192,0,5343,154,221,9724,210,9240,0,18964,44,1,0 -1,56,431,89600,0,616500,1899,589,1430.4,378074,1330,1515,612098,16461920726,8639936081,4132,0,0,0,216740322,23,15805,10,3,15657,3,15657,0,31314,18964,431,16 +1,56,431,89600,0,616500,1899,589,1430.4,378287,1521,1515,612098,16461920726,8639936081,4132,0,0,0,216740322,23,15805,10,3,15657,3,15657,0,31314,18964,431,16 1,70,1,71716,0,384,384,384,384.0,54,1,1,379,0,0,0,0,0,0,138414192,0,5343,170,223,9812,210,9240,0,19052,44,1,0 -1,52,431,51060,0,759623,2321,918,1762.5,394996,1460,2027,754015,26337468742,8639936081,7772,0,0,0,250648581,87,16157,170,3,15657,3,15657,0,31314,18964,431,19 +1,52,431,51060,0,759623,2321,918,1762.5,395214,1706,2027,754015,26337468742,8639936081,7772,0,0,0,250648581,87,16157,170,3,15657,3,15657,0,31314,18964,431,19 1,73,1,46297,0,136,136,136,136.0,57,1,1,131,0,0,0,0,0,0,138414192,0,5344,0,214,9416,219,9636,0,19052,44,1,0 1,76,1,23048,0,340,340,340,340.0,36,1,1,334,0,0,0,0,0,0,138414192,0,5343,223,215,9460,217,9548,0,19008,44,1,0 1,31,1,6956,0,6738,6738,6738,6738.0,5104,128,688,6035,349526,86400,53,0,0,0,155563380,1,10759,0,0,0,0,0,0,0,7239,1800,0 @@ -19,20 +19,20 @@ appIndex,stageId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_ma 1,34,1,6907,0,6729,6729,6729,6729.0,157,136,681,6035,12261,1350,53,0,0,0,155199546,1,9839,0,0,0,0,0,0,0,699,165,0 1,38,1,6842,0,6743,6743,6743,6743.0,187,256,688,6035,349526,86400,53,0,0,0,155563380,1,9927,0,0,0,0,0,0,0,7239,1800,0 1,0,1,5904,0,5699,5699,5699,5699.0,422,948,1114,4382,0,0,37,0,0,0,0,8,2794,0,0,0,0,0,0,0,0,0,0 -1,13,200,5697,0,87661,966,349,438.3,9821,427,951,84265,0,0,144,0,0,0,0,9,6258,0,0,0,0,0,0,0,0,0,0 -1,23,200,5476,0,84240,490,355,421.2,5290,200,214,82784,0,0,136,0,0,0,0,0,6214,0,0,0,0,0,0,0,0,0,0 -1,21,200,5265,0,80904,485,353,404.5,6004,203,220,79384,0,0,136,0,0,0,0,1,6302,0,0,0,0,0,0,0,0,0,0 -1,27,200,4719,0,70760,442,309,353.8,4042,200,209,69494,0,0,152,0,0,0,0,10,5788,0,0,0,0,0,0,0,0,0,0 +1,13,200,5697,0,87661,966,349,438.3,9924,528,951,84265,0,0,144,0,0,0,0,9,6258,0,0,0,0,0,0,0,0,0,0 +1,23,200,5476,0,84240,490,355,421.2,5394,292,214,82784,0,0,136,0,0,0,0,0,6214,0,0,0,0,0,0,0,0,0,0 +1,21,200,5265,0,80904,485,353,404.5,6100,304,220,79384,0,0,136,0,0,0,0,1,6302,0,0,0,0,0,0,0,0,0,0 +1,27,200,4719,0,70760,442,309,353.8,4145,287,209,69494,0,0,152,0,0,0,0,10,5788,0,0,0,0,0,0,0,0,0,0 1,3,1,4701,0,4693,4693,4693,4693.0,280,701,804,3796,0,0,26,0,0,0,0,7,2834,0,0,0,0,0,0,0,0,0,0 -1,25,200,4599,0,70379,569,314,351.9,4106,200,216,69040,0,0,168,0,0,0,0,14,5708,0,0,0,0,0,0,0,0,0,0 -1,29,200,4552,0,69682,423,310,348.4,3730,200,218,68521,0,0,168,0,0,0,0,9,5748,0,0,0,0,0,0,0,0,0,0 +1,25,200,4599,0,70379,569,314,351.9,4200,294,216,69040,0,0,168,0,0,0,0,14,5708,0,0,0,0,0,0,0,0,0,0 +1,29,200,4552,0,69682,423,310,348.4,3830,272,218,68521,0,0,168,0,0,0,0,9,5748,0,0,0,0,0,0,0,0,0,0 1,35,1,4525,0,4332,4332,4332,4332.0,3359,95,401,3907,30328,7200,39,0,0,0,155245068,1,10552,0,0,0,0,0,0,0,7719,1920,0 1,36,1,4509,0,4334,4334,4334,4334.0,260,130,404,3907,349526,86400,39,0,0,0,155563380,1,9851,0,0,0,0,0,0,0,7239,1800,0 1,39,1,4474,0,4322,4322,4322,4322.0,112,124,392,3907,349526,86400,39,0,0,0,155563380,1,9926,0,0,0,0,0,0,0,7239,1800,0 1,40,1,4469,0,4327,4327,4327,4327.0,98,147,394,3907,349526,86400,39,0,0,0,155563380,1,9895,0,0,0,0,0,0,0,7239,1800,0 1,37,1,4464,0,4334,4334,4334,4334.0,136,144,405,3907,349526,86400,39,0,0,0,155563380,1,9851,0,0,0,0,0,0,0,7239,1800,0 1,107,1,1052,0,1022,1022,1022,1022.0,758,77,95,901,0,0,0,0,0,0,134218344,6,10091,5,218,9592,220,9680,0,19272,0,0,0 -1,19,200,794,0,11895,145,38,59.5,943,209,252,10017,0,0,56,0,0,0,0,22,2739,0,0,0,0,0,0,0,0,0,0 +1,19,200,794,0,11895,145,38,59.5,1050,321,252,10017,0,0,56,0,0,0,0,22,2739,0,0,0,0,0,0,0,0,0,0 1,26,1,315,0,312,312,312,312.0,2,1,1,306,0,0,0,0,0,0,0,0,3777,0,0,0,0,0,0,0,0,0,0 1,2,1,276,0,267,267,267,267.0,6,4,4,124,0,0,0,0,0,0,0,1,3342,0,0,0,0,0,0,0,0,0,0 1,11,1,260,0,254,254,254,254.0,5,3,3,241,0,0,0,0,0,0,0,0,2913,0,0,0,0,0,0,0,0,0,0 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_duration_and_cpu_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_duration_and_cpu_expectation.csv index ebde48473..e2ae98c14 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_duration_and_cpu_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_duration_and_cpu_expectation.csv @@ -1,9 +1,9 @@ appIndex,App ID,rootSqlId,sqlID,SQL Duration,Contains Dataset or RDD Op,App Duration,Potential Problems,Executor CPU Time Percent -1,"local-1626104300434","",0,1260,false,131104,"NESTED COMPLEX TYPE",92.65 -1,"local-1626104300434","",1,259,false,131104,"NESTED COMPLEX TYPE",76.79 -1,"local-1626104300434","",2,130,false,131104,"NESTED COMPLEX TYPE",90.48 -1,"local-1626104300434","",3,76,false,131104,"NESTED COMPLEX TYPE",97.56 +1,"local-1626104300434","",0,1260,false,131104,"NESTED COMPLEX TYPE",93.01 +1,"local-1626104300434","",1,259,false,131104,"NESTED COMPLEX TYPE",77.38 +1,"local-1626104300434","",2,130,false,131104,"NESTED COMPLEX TYPE",92.06 +1,"local-1626104300434","",3,76,false,131104,"NESTED COMPLEX TYPE",100.0 1,"local-1626104300434","",4,65,false,131104,"NESTED COMPLEX TYPE",100.0 -1,"local-1626104300434","",5,479,false,131104,"NESTED COMPLEX TYPE",87.32 +1,"local-1626104300434","",5,479,false,131104,"NESTED COMPLEX TYPE",87.8 1,"local-1626104300434","",6,95,false,131104,"",96.3 1,"local-1626104300434","",7,65,false,131104,"",95.24 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg2_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg2_expectation.csv index 9db36ced3..655179961 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg2_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg2_expectation.csv @@ -1,2 +1,2 @@ appIndex,jobId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,0,213,2515,0,25761,1624,9,120.9,7045,3021,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,849 +1,0,213,2515,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg_expectation.csv index c53dac282..b4cd7175c 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsagg_expectation.csv @@ -1,2 +1,2 @@ appIndex,jobId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,0,213,2569,0,26735,1598,10,125.5,6500,3433,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,944 +1,0,213,2569,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsaggmulti_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsaggmulti_expectation.csv index ee224c58f..d9202e3ad 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsaggmulti_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_jobmetricsaggmulti_expectation.csv @@ -1,3 +1,3 @@ appIndex,jobId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,0,213,2569,0,26735,1598,10,125.5,6500,3433,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,944 -2,0,213,2515,0,25761,1624,9,120.9,7045,3021,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,849 +1,0,213,2569,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 +2,0,213,2515,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv index e55b4efb5..b4fa1dff0 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv @@ -1,2 +1,2 @@ appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"local-1622821994212",0,"count at :28",213,3041,7045,13522,52.1,0,25761,1624,9,120.9,7045,3021,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,849 +1,"local-1622821994212",0,"count at :28",213,3041,7151,13522,52.88,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv index 9b6da3b5b..1cf88cdb6 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv @@ -1,2 +1,2 @@ appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"local-1622814619968",0,"count at :28",213,3087,6500,13414,48.46,0,26735,1598,10,125.5,6500,3433,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,944 +1,"local-1622814619968",0,"count at :28",213,3087,6608,13414,49.26,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv index f639dab93..fab4b6a71 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv @@ -1,3 +1,3 @@ appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"local-1622814619968",0,"count at :28",213,3087,6500,13414,48.46,0,26735,1598,10,125.5,6500,3433,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,944 -2,"local-1622821994212",0,"count at :28",213,3041,7045,13522,52.1,0,25761,1624,9,120.9,7045,3021,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,849 +1,"local-1622814619968",0,"count at :28",213,3087,6608,13414,49.26,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 +2,"local-1622821994212",0,"count at :28",213,3041,7151,13522,52.88,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg2_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg2_expectation.csv index 997834dfa..4aab14955 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg2_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg2_expectation.csv @@ -1,5 +1,5 @@ appIndex,stageId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,0,6,1761,0,9455,1624,1540,1575.8,2914,1283,5056,4248,0,0,228,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132263,1200,373 -1,1,6,1666,0,9274,1621,1528,1545.7,2568,1004,5016,4099,0,0,196,0,0,0,0,4,2951,0,0,0,0,0,0,0,40132257,1200,473 -1,2,200,592,0,6937,221,9,34.7,1518,696,1065,5125,0,0,0,0,0,0,0,3,7402,0,2400,80264520,0,0,0,80264520,15400,200,3 +1,0,6,1761,0,9455,1624,1540,1575.8,2917,1287,5056,4248,0,0,228,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132263,1200,376 +1,1,6,1666,0,9274,1621,1528,1545.7,2570,1007,5016,4099,0,0,196,0,0,0,0,4,2951,0,0,0,0,0,0,0,40132257,1200,475 +1,2,200,592,0,6937,221,9,34.7,1619,802,1065,5125,0,0,0,0,0,0,0,3,7402,0,2400,80264520,0,0,0,80264520,15400,200,50 1,3,1,101,0,95,95,95,95.0,45,38,41,50,0,0,0,0,0,0,0,0,8075,0,200,15400,0,0,0,15400,0,0,0 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg_expectation.csv index af9fa58db..b99aebc5a 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsagg_expectation.csv @@ -1,5 +1,5 @@ appIndex,stageId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,0,6,1743,0,9518,1598,1580,1586.3,2509,1391,5309,4043,0,0,168,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132250,1200,397 -1,1,6,1631,0,9434,1582,1568,1572.3,2405,1065,5273,3998,0,0,168,0,0,0,0,5,2951,0,0,0,0,0,0,0,40132258,1200,505 -1,2,200,688,0,7705,237,10,38.5,1556,940,1474,5337,0,0,0,0,0,0,0,0,7359,0,2400,80264508,0,0,0,80264508,15400,200,42 +1,0,6,1743,0,9518,1598,1580,1586.3,2512,1393,5309,4043,0,0,168,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132250,1200,400 +1,1,6,1631,0,9434,1582,1568,1572.3,2406,1067,5273,3998,0,0,168,0,0,0,0,5,2951,0,0,0,0,0,0,0,40132258,1200,508 +1,2,200,688,0,7705,237,10,38.5,1660,1034,1474,5337,0,0,0,0,0,0,0,0,7359,0,2400,80264508,0,0,0,80264508,15400,200,93 1,3,1,83,0,78,78,78,78.0,30,37,39,36,0,0,0,0,0,0,0,0,8075,0,200,15400,0,0,0,15400,0,0,0 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsaggmulti_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsaggmulti_expectation.csv index b0931558b..4704cafa9 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsaggmulti_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagemetricsaggmulti_expectation.csv @@ -1,9 +1,9 @@ appIndex,stageId,numTasks,Duration,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,0,6,1743,0,9518,1598,1580,1586.3,2509,1391,5309,4043,0,0,168,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132250,1200,397 -1,1,6,1631,0,9434,1582,1568,1572.3,2405,1065,5273,3998,0,0,168,0,0,0,0,5,2951,0,0,0,0,0,0,0,40132258,1200,505 -1,2,200,688,0,7705,237,10,38.5,1556,940,1474,5337,0,0,0,0,0,0,0,0,7359,0,2400,80264508,0,0,0,80264508,15400,200,42 +1,0,6,1743,0,9518,1598,1580,1586.3,2512,1393,5309,4043,0,0,168,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132250,1200,400 +1,1,6,1631,0,9434,1582,1568,1572.3,2406,1067,5273,3998,0,0,168,0,0,0,0,5,2951,0,0,0,0,0,0,0,40132258,1200,508 +1,2,200,688,0,7705,237,10,38.5,1660,1034,1474,5337,0,0,0,0,0,0,0,0,7359,0,2400,80264508,0,0,0,80264508,15400,200,93 1,3,1,83,0,78,78,78,78.0,30,37,39,36,0,0,0,0,0,0,0,0,8075,0,200,15400,0,0,0,15400,0,0,0 -2,0,6,1761,0,9455,1624,1540,1575.8,2914,1283,5056,4248,0,0,228,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132263,1200,373 -2,1,6,1666,0,9274,1621,1528,1545.7,2568,1004,5016,4099,0,0,196,0,0,0,0,4,2951,0,0,0,0,0,0,0,40132257,1200,473 -2,2,200,592,0,6937,221,9,34.7,1518,696,1065,5125,0,0,0,0,0,0,0,3,7402,0,2400,80264520,0,0,0,80264520,15400,200,3 +2,0,6,1761,0,9455,1624,1540,1575.8,2917,1287,5056,4248,0,0,228,0,0,0,0,3,2951,0,0,0,0,0,0,0,40132263,1200,376 +2,1,6,1666,0,9274,1621,1528,1545.7,2570,1007,5016,4099,0,0,196,0,0,0,0,4,2951,0,0,0,0,0,0,0,40132257,1200,475 +2,2,200,592,0,6937,221,9,34.7,1619,802,1065,5125,0,0,0,0,0,0,0,3,7402,0,2400,80264520,0,0,0,80264520,15400,200,50 2,3,1,101,0,95,95,95,95.0,45,38,41,50,0,0,0,0,0,0,0,0,8075,0,200,15400,0,0,0,15400,0,0,0 From 1857fe25e821262a8af4172aba347bcbea83e0fa Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Thu, 2 Jan 2025 14:33:48 -0800 Subject: [PATCH 29/52] Bump minimum Spark version to 3.2.0 and improve AutoTuner unit tests for multiple Spark versions (#1482) * Update AutoTuner unit tests to Spark version 3.2.0 Signed-off-by: Partho Sarthi * Bump default Spark version to 3.5.1 Signed-off-by: Partho Sarthi * Fix licence header Signed-off-by: Partho Sarthi * Remove Spark runtime versions 3.1.1, 3.1.3 and 3.1.4 Signed-off-by: Partho Sarthi * Remove Spark 4.0.0 from github workflows Signed-off-by: Partho Sarthi --------- Signed-off-by: Partho Sarthi --- .github/workflows/mvn-verify-check.yml | 4 +- core/pom.xml | 62 +---- .../spark/sql/rapids/tool/ToolUtils.scala | 15 +- .../tool/tuning/BaseAutoTunerSuite.scala | 8 +- .../tool/tuning/ProfilingAutoTunerSuite.scala | 242 ++++++++++-------- .../tuning/QualificationAutoTunerSuite.scala | 4 +- 6 files changed, 152 insertions(+), 183 deletions(-) diff --git a/.github/workflows/mvn-verify-check.yml b/.github/workflows/mvn-verify-check.yml index 5e4bbf7e6..df4f1436b 100644 --- a/.github/workflows/mvn-verify-check.yml +++ b/.github/workflows/mvn-verify-check.yml @@ -1,4 +1,4 @@ -# Copyright (c) 2023-2024, NVIDIA CORPORATION. +# Copyright (c) 2023-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -25,7 +25,7 @@ jobs: strategy: matrix: java-version: [8, 11] - spark-version: ['313', '324', '334', '350'] + spark-version: ['324', '334', '350'] steps: - uses: actions/checkout@v4 diff --git a/core/pom.xml b/core/pom.xml index 094e6ce3e..79cb3830a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -1,6 +1,6 @@ + + + ^println$ + + + @@ -119,4 +168,12 @@ You can also disable only one rule, by specifying its rule id, as specified in: + + + + + + + + diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/AppSummaryInfoBaseProvider.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/AppSummaryInfoBaseProvider.scala index fb886a149..aa3c8e162 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/AppSummaryInfoBaseProvider.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/AppSummaryInfoBaseProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -40,7 +40,7 @@ class AppSummaryInfoBaseProvider extends AppInfoPropertyGetter override def getProperty(propKey: String): Option[String] = { if (propKey.startsWith(ToolUtils.PROPS_RAPIDS_KEY_PREFIX)) { getRapidsProperty(propKey) - } else if (propKey.startsWith("spark")){ + } else if (propKey.startsWith("spark")) { getSparkProperty(propKey) } else { getSystemProperty(propKey) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala index 24cfbfeba..7473c7698 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala @@ -432,7 +432,7 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], /** * Attempts to get the instance type based on the core and gpu requirements. */ - def getInstanceByResources(cores:Int, numGpus: Int): Option[InstanceInfo] = None + def getInstanceByResources(cores: Int, numGpus: Int): Option[InstanceInfo] = None /** * Recommend a GPU Instance type to use for this application. @@ -452,7 +452,7 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], // we could put on a node. if (origClusterNumExecsPerNode == -1) { maxGpusSupported - } else { + } else { origClusterNumExecsPerNode } } else { @@ -611,7 +611,7 @@ class DatabricksAwsPlatform(gpuDevice: Option[GpuDevice], clusterProperties: Option[ClusterProperties]) extends DatabricksPlatform(gpuDevice, clusterProperties) with Logging { - override val platformName: String = PlatformNames.DATABRICKS_AWS + override val platformName: String = PlatformNames.DATABRICKS_AWS override val defaultGpuDevice: GpuDevice = A10GGpu override def getInstanceByResources( @@ -665,7 +665,7 @@ class DatabricksAzurePlatform(gpuDevice: Option[GpuDevice], class DataprocPlatform(gpuDevice: Option[GpuDevice], clusterProperties: Option[ClusterProperties]) extends Platform(gpuDevice, clusterProperties) { - override val platformName: String = PlatformNames.DATAPROC + override val platformName: String = PlatformNames.DATAPROC override val defaultGpuDevice: GpuDevice = T4Gpu override def isPlatformCSP: Boolean = true override def maxGpusSupported: Int = 4 @@ -694,7 +694,7 @@ class DataprocPlatform(gpuDevice: Option[GpuDevice], class DataprocServerlessPlatform(gpuDevice: Option[GpuDevice], clusterProperties: Option[ClusterProperties]) extends DataprocPlatform(gpuDevice, clusterProperties) { - override val platformName: String = PlatformNames.DATAPROC_SL + override val platformName: String = PlatformNames.DATAPROC_SL override val defaultGpuDevice: GpuDevice = L4Gpu override def isPlatformCSP: Boolean = true } @@ -702,13 +702,13 @@ class DataprocServerlessPlatform(gpuDevice: Option[GpuDevice], class DataprocGkePlatform(gpuDevice: Option[GpuDevice], clusterProperties: Option[ClusterProperties]) extends DataprocPlatform(gpuDevice, clusterProperties) { - override val platformName: String = PlatformNames.DATAPROC_GKE + override val platformName: String = PlatformNames.DATAPROC_GKE override def isPlatformCSP: Boolean = true } class EmrPlatform(gpuDevice: Option[GpuDevice], clusterProperties: Option[ClusterProperties]) extends Platform(gpuDevice, clusterProperties) { - override val platformName: String = PlatformNames.EMR + override val platformName: String = PlatformNames.EMR override val defaultGpuDevice: GpuDevice = A10GGpu override def isPlatformCSP: Boolean = true @@ -755,7 +755,7 @@ class EmrPlatform(gpuDevice: Option[GpuDevice], class OnPremPlatform(gpuDevice: Option[GpuDevice], clusterProperties: Option[ClusterProperties]) extends Platform(gpuDevice, clusterProperties) { - override val platformName: String = PlatformNames.ONPREM + override val platformName: String = PlatformNames.ONPREM // Note we don't have an speedup factor file for onprem l4's but we want auto tuner // to use L4. override val defaultGpuDevice: GpuDevice = L4Gpu diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala index 884ad8907..3822706ca 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/DeltaLakeHelper.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -55,7 +55,7 @@ class DLWriteWithFormatAndSchemaParser(node: SparkPlanGraphNode, object DeltaLakeHelper { // we look for the serdeLibrary which is part of the node description: // Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - //private val serdeRegex = "Serde Library: ([\\w.]+)".r + // private val serdeRegex = "Serde Library: ([\\w.]+)".r private val serdeRegex = "Serde Library: ([\\w.]+)".r // We look for the schema in the node description. It is in the following format // Schema: root diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala index ac43bf783..0efb36a45 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/ReadParser.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -86,7 +86,7 @@ object ReadParser extends Logging { } // This tries to get just the field specified by tag in a string that - // may contain multiple fields. It looks for a comma to delimit fields. + // may contain multiple fields. It looks for a comma to delimit fields. private def getFieldWithoutTag(str: String, tag: String): String = { val index = str.indexOf(tag) // remove the tag from the final string returned @@ -187,7 +187,6 @@ object ReadParser extends Logging { } ReadMetaData(schema, location, fileFormat, tags = extractReadTags(node.desc)) } - } // For the read score we look at the read format and datatypes for each diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala index 8471e8a57..ecfafa5c7 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/planparser/SQLPlanParser.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -67,7 +67,7 @@ object UnsupportedReasons extends Enumeration { case IS_UNSUPPORTED => "Unsupported" case CONTAINS_UNSUPPORTED_EXPR => "Contains unsupported expr" case UNSUPPORTED_IO_FORMAT => "Unsupported IO format" - case customReason @ _ => customReason.toString + case customReason @ _ => customReason.toString } } } @@ -163,7 +163,7 @@ case class ExecInfo( OpActions.IgnoreNoPerf } else if (shouldIgnore) { OpActions.IgnorePerf - } else { + } else { OpActions.Triage } } @@ -296,7 +296,7 @@ object ExecInfo { children: Option[Seq[ExecInfo]], // only one level deep stages: Set[Int] = Set.empty, shouldRemove: Boolean = false, - unsupportedExecReason:String = "", + unsupportedExecReason: String = "", unsupportedExprs: Seq[UnsupportedExprOpRef] = Seq.empty, dataSet: Boolean = false, udf: Boolean = false, @@ -660,7 +660,7 @@ object SQLPlanParser extends Logging { maxDuration } - private def ignoreExpression(expr:String): Boolean = { + private def ignoreExpression(expr: String): Boolean = { ignoreExpressions.contains(expr.toLowerCase) } @@ -789,7 +789,7 @@ object SQLPlanParser extends Logging { parsedExpressions.toArray } - def parseWindowExpressions(exprStr:String): Array[String] = { + def parseWindowExpressions(exprStr: String): Array[String] = { val parsedExpressions = ArrayBuffer[String]() // [sum(cast(level#30 as bigint)) windowspecdefinition(device#29, id#28 ASC NULLS FIRST, // specifiedwindowframe(RangeFrame, unboundedpreceding$(), currentrow$())) AS sum#35L, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DriverLogProcessor.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DriverLogProcessor.scala index 4a7afca25..fa4f757d7 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DriverLogProcessor.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DriverLogProcessor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -64,7 +64,6 @@ class DriverLogProcessor(logPath: String) } } - object BaseDriverLogInfoProvider { def noneDriverLog: BaseDriverLogInfoProvider = new BaseDriverLogInfoProvider() -} \ No newline at end of file +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala index 7943e8de8..aa861d04b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimeline.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,7 +35,7 @@ class TimelineTaskInfo(val stageId: Int, val taskId: Long, class TimelineStageInfo(val stageId: Int, startTime: Long, - endTime:Long, + endTime: Long, val duration: Long) extends TimelineTiming(startTime, endTime) class TimelineJobInfo(val jobId: Int, @@ -279,7 +279,7 @@ object GenerateTimeline { .flatMap(_.taskUpdatesMap.values).sum val semMetricsMs = app.accumManager.accumInfoMap.flatMap { - case (_,accumInfo: AccumInfo) + case (_, accumInfo: AccumInfo) if accumInfo.infoRef.name == AccumNameRef.NAMES_TABLE.get("gpuSemaphoreWait") => Some(accumInfo.taskUpdatesMap.values.sum) case _ => None diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala index 2da4b59ec..444479af3 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/HealthCheck.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -47,7 +47,7 @@ class HealthCheck(apps: Seq[ApplicationInfo]) { ProfRemovedExecutorView.getRawView(apps) } - //Function to list all *possible* not-supported plan nodes if GPU Mode=on + // Function to list all *possible* not-supported plan nodes if GPU Mode=on def getPossibleUnsupportedSQLPlan: Seq[UnsupportedOpsProfileResult] = { val res = apps.flatMap { app => app.planMetricProcessor.unsupportedSQLPlan.map { unsup => diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 2b1131fa4..ed03cea46 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -203,7 +203,7 @@ class SQLExecutionInfoClass( } case class SQLAccumProfileResults(appIndex: Int, sqlID: Long, nodeID: Long, - nodeName: String, accumulatorId: Long, name: String, min: Long, median:Long, + nodeName: String, accumulatorId: Long, name: String, min: Long, median: Long, max: Long, total: Long, metricType: String, stageIds: String) extends ProfileResult { override val outputHeaders = Seq("appIndex", "sqlID", "nodeID", "nodeName", "accumulatorId", "name", "min", "median", "max", "total", "metricType", "stageIds") @@ -310,7 +310,7 @@ case class AppInfoProfileResults(appIndex: Int, appName: String, override def convertToSeq: Seq[String] = { Seq(appIndex.toString, appName, appId.getOrElse(""), - sparkUser, startTime.toString, endTimeToStr, durToStr, + sparkUser, startTime.toString, endTimeToStr, durToStr, durationStr, sparkRuntime.toString, sparkVersion, pluginEnabled.toString) } override def convertToCSVSeq: Seq[String] = { @@ -1105,7 +1105,7 @@ case class WholeStageCodeGenResults( } } -case class RecommendedPropertyResult(property: String, value: String){ +case class RecommendedPropertyResult(property: String, value: String) { override def toString: String = "--conf %s=%s".format(property, value) } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala index 9e58ba967..b4eeb04af 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -162,7 +162,7 @@ class PluginTypeChecker(platform: Platform = PlatformFactory.createInstance(), // Some SQL function names have backquotes(`) around their names, // so we remove them before saving. readOperators(source, "exprs", true).map( - x => (x._1.toLowerCase.replaceAll("\\`", "").replaceAll(" ",""), x._2)) + x => (x._1.toLowerCase.replaceAll("\\`", "").replaceAll(" ", ""), x._2)) } def readUnsupportedOpsByDefaultReasons: Map[String, String] = { @@ -170,7 +170,7 @@ class PluginTypeChecker(platform: Platform = PlatformFactory.createInstance(), val unsupportedExecsBydefault = readOperators(execsSource, "execs", false) val exprsSource = UTF8Source.fromResource(SUPPORTED_EXPRS_FILE) val unsupportedExprsByDefault = readOperators(exprsSource, "exprs", false).map( - x => (x._1.toLowerCase.replaceAll("\\`", "").replaceAll(" ",""), x._2)) + x => (x._1.toLowerCase.replaceAll("\\`", "").replaceAll(" ", ""), x._2)) unsupportedExecsBydefault ++ unsupportedExprsByDefault } @@ -222,13 +222,13 @@ class PluginTypeChecker(platform: Platform = PlatformFactory.createInstance(), // In the notes section of the supported csv files, it specifies reason for why it is not // supported by default. We use this information to propagate it unsupported operators // csv file. - private def processOperatorLine(cols: Array[String], operatorType:String, + private def processOperatorLine(cols: Array[String], operatorType: String, isSupported: Boolean): Seq[(String, String)] = { operatorType match { case "exprs" if isSupported => // Logic for supported expressions val exprName = Seq((cols(0), cols(1))) - val sqlFuncNames = if (cols(2).nonEmpty && cols(2) != NONE ){ + val sqlFuncNames = if (cols(2).nonEmpty && cols(2) != NONE) { // There are addidtional checks for Expressions. In physical plan, SQL function name is // printed instead of expression name. We have to save both expression name and // SQL function name(if there is one) so that we don't miss the expression while diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala index 81e6c8c91..8962974ea 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/Qualification.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -35,7 +35,7 @@ class Qualification(outputPath: String, numRows: Int, hadoopConf: Configuration, timeout: Option[Long], nThreads: Int, order: String, pluginTypeChecker: PluginTypeChecker, reportReadSchema: Boolean, printStdout: Boolean, enablePB: Boolean, - reportSqlLevel: Boolean, maxSQLDescLength: Int, mlOpsEnabled:Boolean, + reportSqlLevel: Boolean, maxSQLDescLength: Int, mlOpsEnabled: Boolean, penalizeTransitions: Boolean, tunerContext: Option[TunerContext], clusterReport: Boolean, platformArg: String, workerInfoPath: String) extends ToolBase(timeout) { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningAppMetadata.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningAppMetadata.scala index 5026e42b6..58c9f2500 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningAppMetadata.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningAppMetadata.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -43,4 +43,4 @@ object RunningAppMetadata { rStartTime: Long): RunningAppMetadata = { new RunningAppMetadata(rName, rId, "", rStartTime) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala index 8e2350334..4e4e6efcf 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualOutputWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -39,7 +39,7 @@ class RunningQualOutputWriter( outputDir: String, hadoopConf: Option[Configuration] = None, fileNameSuffix: String = "") - extends QualOutputWriter(outputDir, reportReadSchema=false, printStdout=false, + extends QualOutputWriter(outputDir, reportReadSchema = false, printStdout = false, prettyPrintOrder = "desc", hadoopConf) { private lazy val csvPerSQLFileWriter = new ToolTextFileWriter(outputDir, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala index 92b05aeb1..c47ce84aa 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/RunningQualificationApp.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -74,7 +74,7 @@ class RunningQualificationApp( perSqlOnly: Boolean = false, pluginTypeChecker: PluginTypeChecker = new PluginTypeChecker(), platform: Platform = PlatformFactory.createInstance()) - extends QualificationAppInfo(None, None, pluginTypeChecker, reportSqlLevel=false, + extends QualificationAppInfo(None, None, pluginTypeChecker, reportSqlLevel = false, perSqlOnly, platform = platform) { // note we don't use the per sql reporting providing by QualificationAppInfo so we always // send down false for it diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index 048e71ada..f7aca2ec4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -843,7 +843,7 @@ class AutoTuner( getPropertyValue("spark.sql.adaptive.advisoryPartitionSizeInBytes") if (appInfoProvider.getMeanInput < autoTunerConfigsProvider.AQE_INPUT_SIZE_BYTES_THRESHOLD) { - if(advisoryPartitionSizeProperty.isEmpty) { + if (advisoryPartitionSizeProperty.isEmpty) { // The default is 64m, but 128m is slightly better for the GPU as the GPU has sub-linear // scaling until it is full and 128m makes the GPU more full, but too large can be // slightly problematic because this is the compressed shuffle size @@ -1470,7 +1470,7 @@ trait AutoTunerConfigsProvider extends Logging { def shuffleManagerCommentForUnsupportedVersion( sparkVersion: String, platform: Platform): String = { - val (latestSparkVersion, latestSmVersion) = platform.latestSupportedShuffleManagerInfo + val (latestSparkVersion, latestSmVersion) = platform.latestSupportedShuffleManagerInfo // scalastyle:off line.size.limit s""" |Cannot recommend RAPIDS Shuffle Manager for unsupported ${platform.sparkVersionLabel}: '$sparkVersion'. diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala index 3a2cb8b48..0fd30525a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/QualRawReportGenerator.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,13 +19,14 @@ package com.nvidia.spark.rapids.tool.views import com.nvidia.spark.rapids.tool.analysis.{AggRawMetricsResult, AppSQLPlanAnalyzer, QualSparkMetricsAnalyzer} import com.nvidia.spark.rapids.tool.profiling.{DataSourceProfileResult, ProfileOutputWriter, ProfileResult, SQLAccumProfileResults} +import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo /** * This object generates the raw metrics view for the qualification tool. It is used to generate * the CSV files without applying any heuristics or estimation. */ -object QualRawReportGenerator { +object QualRawReportGenerator extends Logging { private def constructLabelsMaps( aggRawResult: AggRawMetricsResult): Map[String, Seq[ProfileResult]] = { @@ -110,7 +111,7 @@ object QualRawReportGenerator { pWriter.write(QualRemovedExecutorView.getLabel, QualRemovedExecutorView.getRawView(Seq(app))) } catch { case e: Exception => - println(s"Error generating raw metrics for ${app.appId}: ${e.getMessage}") + logError(s"Error generating raw metrics for ${app.appId}: ${e.getMessage}") } finally { pWriter.close() } diff --git a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/Benchmark.scala b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/Benchmark.scala index 45ad3bdd4..0a4032e19 100644 --- a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/Benchmark.scala +++ b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/Benchmark.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -76,6 +76,7 @@ class Benchmark( def run(): Seq[Result] = { require(benchmarks.nonEmpty) val separator = "-" * 80 + // scalastyle:off println println(separator) println("Running benchmark: " + name) println(separator) @@ -85,6 +86,7 @@ class Benchmark( measure(c.name, c.numIters)(c.fn) } println + // scalastyle:on println results } @@ -93,6 +95,7 @@ class Benchmark( * the rate of the function. */ def measure(name: String, overrideNumIters: Int)(f: ToolsTimer => Unit): Result = { + // scalastyle:off println System.gc() // ensures garbage from previous cases don't impact this one val separator = "-" * 80 for (wi <- 0 until warmUpIterations) { @@ -102,7 +105,7 @@ class Benchmark( val runTimes = ArrayBuffer[Long]() val gcCounts = ArrayBuffer[Long]() val gcTimes = ArrayBuffer[Long]() - //For tracking maximum GC over iterations + // For tracking maximum GC over iterations for (i <- 0 until minIters) { System.gc() // ensures GC for a consistent state across different iterations val timer = new ToolsTimer(i) @@ -144,14 +147,15 @@ class Benchmark( bestRuntime / 1000000.0, stdevRunTime / 1000000.0, JVMMemoryParams(avgGcTime, avgGcCount, stdevGcCount, maxGcCount, maxGcTime)) + // scalastyle:on println } } object Benchmark { case class Case(name: String, fn: ToolsTimer => Unit, numIters: Int) - case class JVMMemoryParams( avgGCTime:Double, avgGCCount:Double, - stdDevGCCount: Double, maxGCCount: Long, maxGcTime:Long) + case class JVMMemoryParams(avgGCTime: Double, avgGCCount: Double, + stdDevGCCount: Double, maxGCCount: Long, maxGcTime: Long) case class Result(caseName: String, avgMs: Double, bestMs: Double, stdevMs: Double, memoryParams: JVMMemoryParams) } diff --git a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkArgs.scala b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkArgs.scala index c5c0847f4..5e9f6d2da 100644 --- a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkArgs.scala +++ b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkArgs.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,7 +27,7 @@ Benchmarker class for running various benchmarks. val iterations: ScallopOption[Int] = opt[Int](short = 'i', default = Some(5), descr = "Total iterations to run excluding warmup (for avg time calculation)." + " Default is 5 iterations", validate = _ > 0) - val warmupIterations: ScallopOption[Int] = opt[Int](short = 'w' , + val warmupIterations: ScallopOption[Int] = opt[Int](short = 'w', default = Some(3), descr = "Total number of warmup iterations to run. Can take " + "any input >=0. Warm up is important for benchmarking to ensure initial " + "JVM operations do not skew the result ( classloading etc. )", validate = _ >= 0) diff --git a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkBase.scala b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkBase.scala index 7e37e26f5..0592752cc 100644 --- a/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkBase.scala +++ b/core/src/main/scala/org/apache/spark/rapids/tool/benchmarks/BenchmarkBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -71,8 +71,9 @@ abstract class BenchmarkBase { val firstBest = results.head.bestMs val nameLen = Math.max(40, results.map(_.caseName.length).max) printStream.printf(s"%-${nameLen}s %14s %14s %11s %20s %18s %18s %18s %18s %10s\n", - "Benchmark :", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)","Avg GC Time(ms)", - "Avg GC Count", "Stdev GC Count","Max GC Time(ms)","Max GC Count", "Relative") + "Benchmark :", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)", "Avg GC Time(ms)", + "Avg GC Count", "Stdev GC Count", "Max GC Time(ms)", "Max GC Count", "Relative") + // scalastyle:off println printStream.println("-" * (nameLen + 160)) results.foreach { result => printStream.printf(s"%-${nameLen}s %14s %14s %11s %20s %18s %18s %18s %18s %10s\n", @@ -88,6 +89,7 @@ abstract class BenchmarkBase { "%3.2fX" format (firstBest / result.bestMs)) } printStream.println() + // scalastyle:on println } /** @@ -99,17 +101,17 @@ abstract class BenchmarkBase { private def printSystemInformation(warmUpIterations: Int, iterations: Int, inputArgs: String ): Unit = { val jvmInfo = RuntimeUtil.getJVMOSInfo - output.get.printf(s"%-26s : %s \n","JVM Name", jvmInfo("jvm.name")) - output.get.printf(s"%-26s : %s \n","Java Version", jvmInfo("jvm.version")) - output.get.printf(s"%-26s : %s \n","OS Name", jvmInfo("os.name")) - output.get.printf(s"%-26s : %s \n","OS Version", jvmInfo("os.version")) - output.get.printf(s"%-26s : %s MB \n","MaxHeapMemory", + output.get.printf(s"%-26s : %s \n", "JVM Name", jvmInfo("jvm.name")) + output.get.printf(s"%-26s : %s \n", "Java Version", jvmInfo("jvm.version")) + output.get.printf(s"%-26s : %s \n", "OS Name", jvmInfo("os.name")) + output.get.printf(s"%-26s : %s \n", "OS Version", jvmInfo("os.version")) + output.get.printf(s"%-26s : %s MB \n", "MaxHeapMemory", (Runtime.getRuntime.maxMemory()/1024/1024).toString) - output.get.printf(s"%-26s : %s \n","Total Warm Up Iterations", + output.get.printf(s"%-26s : %s \n", "Total Warm Up Iterations", warmUpIterations.toString) - output.get.printf(s"%-26s : %s \n","Total Runtime Iterations", + output.get.printf(s"%-26s : %s \n", "Total Runtime Iterations", iterations.toString) - output.get.printf(s"%-26s : %s \n \n","Input Arguments", inputArgs) + output.get.printf(s"%-26s : %s \n \n", "Input Arguments", inputArgs) } /** diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala index 971a8711f..432f3b1c9 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -355,7 +355,7 @@ abstract class AppBase( ".*second\\(.*\\).*" -> "TIMEZONE second()" ) - def findPotentialIssues(desc: String): Set[String] = { + def findPotentialIssues(desc: String): Set[String] = { val potentialIssuesRegexs = potentialIssuesRegexMap val issues = potentialIssuesRegexs.filterKeys(desc.matches(_)) issues.values.toSet diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppFilterImpl.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppFilterImpl.scala index 09429a097..cb756cd9c 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppFilterImpl.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/AppFilterImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -77,7 +77,7 @@ class AppFilterImpl( val apps: Seq[AppFilterReturnParameters] = appsForFiltering.asScala.toSeq appArgs match { - case profileArgs:ProfileArgs => + case profileArgs: ProfileArgs => filterEventLogsInternal(apps, profileArgs) case qualificationArgs: QualificationArgs => filterEventLogsInternal(apps, qualificationArgs) @@ -274,7 +274,7 @@ class AppFilterImpl( val startAppInfo = new FilterAppInfo(path, hadoopConf) val appInfo = AppFilterReturnParameters(startAppInfo, path) if (!startAppInfo.isAppMetaDefined) { - logWarning("Cannot process file due to missing start event: " + path) + logWarning("Cannot process file due to missing start event: " + path) } else { appsForFiltering.add(appInfo) } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala index b4dc8ce3f..7ddc42792 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/EventProcessorBase.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -131,7 +131,7 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi def doSparkListenerLogStart( app: T, - event: SparkListenerLogStart): Unit = { + event: SparkListenerLogStart): Unit = { logDebug("Processing event: " + event.getClass) app.handleLogStartForCachedProps(event) } @@ -174,7 +174,7 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi val SparkListenerDriverAccumUpdates(sqlID, accumUpdates) = event accumUpdates.foreach { accum => val driverAccum = DriverAccumCase(sqlID, accum._1, accum._2) - val arrBuf = app.driverAccumMap.getOrElseUpdate(accum._1, + val arrBuf = app.driverAccumMap.getOrElseUpdate(accum._1, ArrayBuffer[DriverAccumCase]()) arrBuf += driverAccum } @@ -202,7 +202,7 @@ abstract class EventProcessorBase[T <: AppBase](app: T) extends SparkListener wi def doSparkRapidsBuildInfoEvent( app: T, - event: SparkRapidsBuildInfoEvent): Unit = { + event: SparkRapidsBuildInfoEvent): Unit = { logDebug("Processing event: " + event.getClass) app.sparkRapidsBuildInfo = event } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala index 6fbf2bb68..a3eef2d7e 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/profiling/ApplicationInfo.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -41,6 +41,7 @@ class SparkPlanInfoWithStage( import SparkPlanInfoWithStage._ def debugEquals(other: Any, depth: Int = 0): Boolean = { + // scalastyle:off println System.err.println(s"${" " * depth}DOES $this == $other?") other match { case o: SparkPlanInfo => @@ -68,6 +69,7 @@ class SparkPlanInfoWithStage( System.err.println(s"${" " * depth}NOT EQUAL WRONG TYPE") false } + // scalastyle:on println } override def toString: String = { @@ -157,7 +159,7 @@ object SparkPlanInfoWithStage { "HashAggregate" -> "Aggregate", "SortAggregate" -> "Aggregate", "GpuHashAggregate" -> "Aggregate", - "RunningWindow" -> "Window", //GpuWindow and Window are already covered + "RunningWindow" -> "Window", // GpuWindow and Window are already covered "GpuRunningWindow" -> "Window") private def isShuffledTopN(info: SparkPlanInfoWithStage): Boolean = { @@ -188,7 +190,7 @@ class ApplicationInfo( platform: Platform = PlatformFactory.createInstance()) extends AppBase(Some(eLogInfo), Some(hadoopConf), Some(platform)) with Logging { - private lazy val eventProcessor = new EventsProcessor(this) + private lazy val eventProcessor = new EventsProcessor(this) // Process all events processEvents() diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index e3c33203f..b1842ec23 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -60,7 +60,7 @@ class QualificationAppInfo( val notSupportFormatAndTypes: HashMap[String, Set[String]] = HashMap[String, Set[String]]() - private lazy val eventProcessor = new QualificationEventProcessor(this, perSqlOnly) + private lazy val eventProcessor = new QualificationEventProcessor(this, perSqlOnly) /** * Important system properties that should be retained. We also include @@ -485,7 +485,7 @@ class QualificationAppInfo( def aggregateStats(): Option[QualificationSummaryInfo] = { appMetaData.map { info => val appDuration = calculateAppDuration().getOrElse(0L) - //calculateAppDuration(info.startTime).getOrElse(0L) + // calculateAppDuration(info.startTime).getOrElse(0L) // if either job or stage failures then we mark as N/A // TODO - what about incomplete, do we want to change those? @@ -754,9 +754,9 @@ class QualificationAppInfo( } // Consider stageInfo to have below string as an example - //org.apache.spark.rdd.RDD.first(RDD.scala:1463) - //org.apache.spark.mllib.feature.PCA.fit(PCA.scala:44) - //org.apache.spark.ml.feature.PCA.fit(PCA.scala:93) + // org.apache.spark.rdd.RDD.first(RDD.scala:1463) + // org.apache.spark.mllib.feature.PCA.fit(PCA.scala:44) + // org.apache.spark.ml.feature.PCA.fit(PCA.scala:93) val splitString = stageInfoDetails.split("\n") // filteredString = org.apache.spark.ml.feature.PCA.fit diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/DataSourceRecord.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/DataSourceRecord.scala index 7804d2072..e1b8bea31 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/DataSourceRecord.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/DataSourceRecord.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -42,7 +42,7 @@ case class NonFinalDataSourceCase( partitionFilters: String) extends DataSourceRecord { override def isFromFinalPlan: Boolean = false - override def comments: String = DataSourceRecord.COMMENT_NON_FINAL_PLAN + override def comments: String = DataSourceRecord.COMMENT_NON_FINAL_PLAN } case class FinalDataSourceCase( @@ -57,12 +57,12 @@ case class FinalDataSourceCase( partitionFilters: String) extends DataSourceRecord { override def isFromFinalPlan: Boolean = true - override def comments: String = DataSourceRecord.COMMENT_FINAL_PLAN + override def comments: String = DataSourceRecord.COMMENT_FINAL_PLAN } object DataSourceRecord { - val COMMENT_NON_FINAL_PLAN="isFinalPlan=false" - val COMMENT_FINAL_PLAN="isFinalPlan=true" + val COMMENT_NON_FINAL_PLAN = "isFinalPlan=false" + val COMMENT_FINAL_PLAN = "isFinalPlan=true" def apply( sqlID: Long, diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanModelManager.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanModelManager.scala index 03b77c1cc..ff2e01011 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanModelManager.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanModelManager.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -66,8 +66,8 @@ class SQLPlanModelManager { * @param physicalDescription String representation of the physical plan for the new version. */ def addNewExecution(id: Long, planInfo: SparkPlanInfo, physicalDescription: String): Unit = { - //TODO: in future we should pass more arguments to this method to capture the common information - // of an SqlPlan (i.e., startTime,..etc)) + // TODO: in future we should pass more arguments to this method to capture the common + // information of an SqlPlan (i.e., startTime,..etc)) val planModel = sqlPlans.getOrElseUpdate(id, new SQLPlanModelWithDSCaching(id)) planModel.addPlan(planInfo, physicalDescription) } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanVersion.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanVersion.scala index a79fafda9..e175a8c82 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanVersion.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/SQLPlanVersion.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -90,7 +90,7 @@ class SQLPlanVersion( readSchema, ReadParser.extractTagFromV1ReadMeta(ReadParser.METAFIELD_TAG_DATA_FILTERS, meta), ReadParser.extractTagFromV1ReadMeta(ReadParser.METAFIELD_TAG_PARTITION_FILTERS, meta), - fromFinalPlan=isFinal)) + fromFinalPlan = isFinal)) } else { None } @@ -116,7 +116,7 @@ class SQLPlanVersion( res.schema, res.dataFilters, res.partitionFilters, - fromFinalPlan=isFinal) + fromFinalPlan = isFinal) } } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala index 67d3a83e1..178bb2bee 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ui/ConsoleProgressBar.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022-2024, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -186,11 +186,13 @@ class ConsoleProgressBar( "" } val consoleLine = header + bar + tailer + // scalastyle:off println if (inlinePBEnabled) { System.out.print(CR + consoleLine + CR) } else { System.out.println(consoleLine) } + // scalastyle:on println lastUpdateTime = now lastUpdatedCount = currentCount lastProgressBar = consoleLine diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/EventUtils.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/EventUtils.scala index 53311ca43..be432f820 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/EventUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/EventUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2023-2024, NVIDIA CORPORATION. + * Copyright (c) 2023-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -47,8 +47,8 @@ object EventUtils extends Logging { // from pull https://github.com/apache/spark/pull/23551/files "number of files" -> "number of files read", // type sum "metadata time (ms)" -> "metadata time", // type sum spark2.x, but it was fixed to be timing - "time to build (ms)" -> "time to build", //type timing - "time to broadcast (ms)" -> "time to broadcast", //type timing + "time to build (ms)" -> "time to build", // type timing + "time to broadcast (ms)" -> "time to broadcast", // type timing "total time to update rows" -> "time to update", "total time to remove rows" -> "time to remove", "bytes of written output" -> "written output", // type sum diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/MemoryMetricsTracker.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/MemoryMetricsTracker.scala index de2a9a98d..c8a295a90 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/MemoryMetricsTracker.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/MemoryMetricsTracker.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -37,12 +37,12 @@ class MemoryMetricsTracker { } def getTotalGCCount: Long = { - val (newGcCount:Long, _) = getCurrentGCMetrics + val (newGcCount: Long, _) = getCurrentGCMetrics newGcCount - startGCMetrics._1 } def getTotalGCTime: Long = { - val (_, newGcTime:Long) = getCurrentGCMetrics + val (_, newGcTime: Long) = getCurrentGCMetrics newGcTime - startGCMetrics._2 } } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala index 129872a0a..37307ec65 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/OperationResult.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -30,7 +30,7 @@ class AppResult(path: String, message: String) extends Logging { val messageToLog = s"File: $path, Message: $message" exp match { case Some(e) => logWarning(messageToLog, e) - case None => logWarning(messageToLog) + case None => logWarning(messageToLog) } } } diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/stubs/db/DBGraphSQLMetricStub.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/stubs/db/DBGraphSQLMetricStub.scala index b2591ad7d..a8f679d34 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/stubs/db/DBGraphSQLMetricStub.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/util/stubs/db/DBGraphSQLMetricStub.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -25,7 +25,7 @@ case class DBGraphSQLMetricStub(m: Mirror) extends GraphReflectionEntry[org.apache.spark.sql.execution.ui.SQLPlanMetric]( m, "org.apache.spark.sql.execution.ui.SQLPlanMetric") { // DataBricks has different constructor of the sparkPlanGraphNode - //Array(final java.lang.String name, final long accumulatorId, + // Array(final java.lang.String name, final long accumulatorId, // final java.lang.String metricType, final boolean experimental) // for 10.4 it is only one constructor with 3 arguments. diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index ec27c601c..4d2c93c89 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -69,7 +69,11 @@ class SQLPlanParserSuite extends BasePlanParserSuite { } else { l } - ).foreach(modifiedLine => pWriter.println(modifiedLine)) + ).foreach { modifiedLine => + // scalastyle:off println + pWriter.println(modifiedLine) + // scalastyle:on println + } } finally { bufferedSource.close() pWriter.close() @@ -169,14 +173,14 @@ class SQLPlanParserSuite extends BasePlanParserSuite { "spark-events-qualification/db_subExecution_id.zstd") val app = createAppFromEventlog(eventlog) // Get sum of durations of all the sqlIds. It contains duplicate values - val totalSqlDuration = app.sqlIdToInfo.values.map(x=> x.duration.getOrElse(0L)).sum + val totalSqlDuration = app.sqlIdToInfo.values.map(x => x.duration.getOrElse(0L)).sum // This is to group the sqlIds based on the rootExecutionId. So that we can verify the // subExecutionId to rootExecutionId mapping. val rootIdToSqlId = app.sqlIdToInfo.groupBy { case (_, info) => info.rootExecutionID } - assert(rootIdToSqlId(Some(5L)).keySet == Set(5,6,7,8,9,10)) + assert(rootIdToSqlId(Some(5L)).keySet == Set(5, 6, 7, 8, 9, 10)) TrampolineUtil.withTempDir { outpath => val allArgs = Array( @@ -1381,7 +1385,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { for ((condExpr, expectedExpressionCounts) <- expressionsMap) { val rawExpressions = SQLPlanParser.parseConditionalExpressions(condExpr) val expected = expectedExpressionCounts.map(e => ExprOpRef(OpRef.fromExpr(e._1), e._2)) - val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) + val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) actualExpressions should ===(expected) } } @@ -1437,7 +1441,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { "EqualTo" -> 4, // EqualTo comes from the = operator "Not" -> 2).map(e => ExprOpRef(OpRef.fromExpr(e._1), e._2)) val rawExpressions = SQLPlanParser.parseFilterExpressions(exprString) - val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) + val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) actualExpressions should ===(expected) } @@ -1458,7 +1462,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { "trim" -> 1, "unbase64" -> 4).map(e => ExprOpRef(OpRef.fromExpr(e._1), e._2)) val rawExpressions = SQLPlanParser.parseAggregateExpressions(exprString) - val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) + val actualExpressions = ExprOpRef.fromRawExprSeq(rawExpressions) actualExpressions should ===(expected) } @@ -1528,7 +1532,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { // +- FileScan parquet [] Batched: true, DataFilters: [], Format: Parquet, Location: // InMemoryFileIndex(1 paths)[file:/tmp_folder/T/toolTest..., PartitionFilters: [] // PushedFilters: [], ReadSchema: struct<> - df2.selectExpr("current_database()","current_database() as my_db") + df2.selectExpr("current_database()", "current_database() as my_db") } val pluginTypeChecker = new PluginTypeChecker() val app = createAppFromEventlog(eventLog) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index b7d8b315f..b33afb21a 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -196,15 +196,15 @@ class AnalysisSuite extends FunSuite { val sqlAggDurCpu = aggResults.sqlDurAggs val resultExpectation = new File(expRoot, expectFile) val schema = new StructType() - .add("appIndex",IntegerType,true) - .add("appID",StringType,true) - .add("rootsqlID",LongType,true) - .add("sqlID",LongType,true) - .add("sqlDuration",LongType,true) - .add("containsDataset",BooleanType,true) - .add("appDuration",LongType,true) - .add("potentialProbs",StringType,true) - .add("executorCpuTime",DoubleType,true) + .add("appIndex", IntegerType, true) + .add("appID", StringType, true) + .add("rootsqlID", LongType, true) + .add("sqlID", LongType, true) + .add("sqlDuration", LongType, true) + .add("containsDataset", BooleanType, true) + .add("appDuration", LongType, true) + .add("potentialProbs", StringType, true) + .add("executorCpuTime", DoubleType, true) val actualDf = sqlAggDurCpu.toDF val dfExpect = sparkSession.read.option("header", "true").option("nullValue", "-") diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AppFilterSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AppFilterSuite.scala index fac5a6ef3..0c9db7615 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AppFilterSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AppFilterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -62,23 +62,23 @@ class AppFilterSuite extends BaseTestSuite { } test("time period minute parsing fail") { - testTimePeriod(msMinAgo(16), "10min", failFilter=true) + testTimePeriod(msMinAgo(16), "10min", failFilter = true) } test("time period hour parsing fail") { - testTimePeriod(msHoursAgo(10), "8h", failFilter=true) + testTimePeriod(msHoursAgo(10), "8h", failFilter = true) } test("time period day parsing fail") { - testTimePeriod(msDaysAgo(40), "38d", failFilter=true) + testTimePeriod(msDaysAgo(40), "38d", failFilter = true) } test("time period week parsing fail") { - testTimePeriod(msWeeksAgo(2), "1w", failFilter=true) + testTimePeriod(msWeeksAgo(2), "1w", failFilter = true) } test("time period month parsing fail") { - testTimePeriod(msMonthsAgo(8), "7m", failFilter=true) + testTimePeriod(msMonthsAgo(8), "7m", failFilter = true) } private def testTimePeriod(eventLogTime: Long, startTimePeriod: String, @@ -124,7 +124,7 @@ class AppFilterSuite extends BaseTestSuite { appTime: Long, uniqueId: Int) private val appsWithFsAndStartTimeToTest = Array( - TestEventLogFSAndAppStartInfo("app-ndshours18", msHoursAgo(16),msHoursAgo(18), 1), + TestEventLogFSAndAppStartInfo("app-ndshours18", msHoursAgo(16), msHoursAgo(18), 1), TestEventLogFSAndAppStartInfo("app-ndsweeks2", msWeeksAgo(2), msWeeksAgo(2), 1), TestEventLogFSAndAppStartInfo("app-nds86-1", msDaysAgo(3), msDaysAgo(4), 1), TestEventLogFSAndAppStartInfo("app-nds86-2", msDaysAgo(13), msWeeksAgo(2), 2)) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala index 1d40472c9..3791aee89 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -479,7 +479,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { val eventlogPaths = appArgs.eventlog() for (path <- eventlogPaths) { apps += new ApplicationInfo(hadoopConf, - EventLogPathProcessor.getEventLogInfo(path,hadoopConf).head._1, index) + EventLogPathProcessor.getEventLogInfo(path, hadoopConf).head._1, index) index += 1 } assert(apps.size == 1) @@ -558,7 +558,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { test("test multiple resource profile in single app") { - val apps :ArrayBuffer[ApplicationInfo] = ArrayBuffer[ApplicationInfo]() + val apps: ArrayBuffer[ApplicationInfo] = ArrayBuffer[ApplicationInfo]() val appArgs = new ProfileArgs(Array(s"$logDir/rp_nosql_eventlog")) var index: Int = 1 val eventlogPaths = appArgs.eventlog() @@ -753,7 +753,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { // verify ucx parameters are captured. assert(rows.contains("spark.executorEnv.UCX_RNDV_SCHEME")) - //verify gds parameters are captured. + // verify gds parameters are captured. assert(rows.contains("spark.rapids.memory.gpu.direct.storage.spill.alignedIO")) val sparkProps = collect.getSparkProperties diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimelineSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimelineSuite.scala index 6cc27c914..d1e1eba3c 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimelineSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateTimelineSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -57,7 +57,7 @@ class GenerateTimelineSuite extends FunSuite with BeforeAndAfterAll with Logging val tempSubDir = new File(dotFileDir, s"${Profiler.SUBDIR}/$appId") // assert that a file was generated - val outputDirs = ToolTestUtils.listFilesMatching(tempSubDir, { f => + val outputDirs = ToolTestUtils.listFilesMatching(tempSubDir, { f => f.endsWith("timeline.svg") }) assert(outputDirs.length === 1) diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/QualificationInfoUtils.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/QualificationInfoUtils.scala index ffcf4c2a5..7700bdfc6 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/QualificationInfoUtils.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/QualificationInfoUtils.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -59,7 +59,7 @@ object QualificationInfoUtils extends Logging { def generateFriendsDataset(spark: SparkSession): Dataset[RapidsFriends] = { import spark.implicits._ val df = spark.sparkContext.parallelize( - Seq.fill(1000){(randomAlpha(10), randomAlpha(5), randomInt)}) + Seq.fill(1000) {(randomAlpha(10), randomAlpha(5), randomInt)}) .toDF("name", "friend", "age") df.as[RapidsFriends] } @@ -96,7 +96,7 @@ object QualificationInfoUtils extends Logging { TrampolineUtil.withTempPath { jsonOutFile => val ageFunc = udf(parseAge) val ds = generateFriendsDataset(spark) - ds.withColumn("ageCategory",ageFunc(col("age"))) + ds.withColumn("ageCategory", ageFunc(col("age"))) val dsAge = ds.filter(d => d.age > 25).map(d => (d.friend, d.age)) dsAge.write.json(jsonOutFile.getCanonicalPath) } @@ -147,13 +147,17 @@ object QualificationInfoUtils extends Logging { */ def main(args: Array[String]): Unit = { if (args.length == 0) { + // scalastyle:off println println(s"ERROR: must specify a logType dataset, udfds, dsAndDf or udffunc") + // scalastyle:on println System.exit(1) } val logType = args(0) if (logType != "dataset" && logType != "udfds" && logType != "udffunc" && logType != "dsAndDf") { + // scalastyle:off println println(s"ERROR: logType must be one of: dataset, udfds, dsAndDf or udffunc") + // scalastyle:on println System.exit(1) } val eventDir = if (args.length > 1) args(1) else "/tmp/spark-eventLogTest" @@ -179,7 +183,9 @@ object QualificationInfoUtils extends Logging { genjoinDataFrameOpEventLog(spark) genjoinDataFrameOpEventLog(spark) } else { + // scalastyle:off println println(s"ERROR: Invalid log type specified: $logType") + // scalastyle:on println System.exit(1) } spark.stop() diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala index 63e3265f3..943e08b70 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/AppFilterSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -62,23 +62,23 @@ class AppFilterSuite extends BaseTestSuite { } test("time period minute parsing fail") { - testTimePeriod(msMinAgo(16), "10min", failFilter=true) + testTimePeriod(msMinAgo(16), "10min", failFilter = true) } test("time period hour parsing fail") { - testTimePeriod(msHoursAgo(10), "8h", failFilter=true) + testTimePeriod(msHoursAgo(10), "8h", failFilter = true) } test("time period day parsing fail") { - testTimePeriod(msDaysAgo(40), "38d", failFilter=true) + testTimePeriod(msDaysAgo(40), "38d", failFilter = true) } test("time period week parsing fail") { - testTimePeriod(msWeeksAgo(2), "1w", failFilter=true) + testTimePeriod(msWeeksAgo(2), "1w", failFilter = true) } test("time period month parsing fail") { - testTimePeriod(msMonthsAgo(8), "7m", failFilter=true) + testTimePeriod(msMonthsAgo(8), "7m", failFilter = true) } private def testTimePeriod(eventLogTime: Long, startTimePeriod: String, @@ -559,7 +559,7 @@ class AppFilterSuite extends BaseTestSuite { test("App Name Regex match with all user name") { testAppNameRegexAndUserName(appsWithAppNameRegexAndUserNameToTest, - "10-newest", "[Nn].*", "user", "all" ,7) + "10-newest", "[Nn].*", "user", "all", 7) } test("App Name Regex match with user name match") { @@ -569,7 +569,7 @@ class AppFilterSuite extends BaseTestSuite { test("App Name Regex exclude with user name match") { testAppNameRegexAndUserName(appsWithAppNameRegexAndUserNameToTest, - "10-newest", "[^Nn].*", "user3", "all",0) + "10-newest", "[^Nn].*", "user3", "all", 0) } test("App Name partial with username match") { @@ -657,7 +657,7 @@ class AppFilterSuite extends BaseTestSuite { test("Test disjunction no appName") { testConjunctionAndDisjunction(appsNameConjunctionAndDisjunctionToTest, filterCriteria("10-newest") ++ - startTimePeriod("2w") ++ userName("user3"), 6, "any") + startTimePeriod("2w") ++ userName("user3"), 6, "any") } test("Test disjunction no startTime") { @@ -832,7 +832,7 @@ class AppFilterSuite extends BaseTestSuite { Array("--application-name", appName) } - def matchFileName(appName:String): Array[String] = { + def matchFileName(appName: String): Array[String] = { Array("--match-event-logs", appName) } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala index 6b7821d3f..c75f3087a 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeCheckerSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -133,7 +133,7 @@ class PluginTypeCheckerSuite extends FunSuite with Logging { assert(result.contains("isnull")) } - test("write data format"){ + test("write data format") { val inputString = Array("Execute InsertIntoHadoopFsRelationCommand " + "file:/home/ubuntu/eventlogs/complex_nested_decimal, false," + " Parquet, Map(path -> complex_nested_decimal), Append, [name, subject]", @@ -178,7 +178,7 @@ class PluginTypeCheckerSuite extends FunSuite with Logging { // Using databricks azure speedup factor as custom file val platform = PlatformFactory.createInstance(PlatformNames.DATABRICKS_AZURE) val speedupFactorFile = ToolTestUtils.getTestResourcePath(platform.getOperatorScoreFile) - val checker = new PluginTypeChecker(speedupFactorFile=Some(speedupFactorFile)) + val checker = new PluginTypeChecker(speedupFactorFile = Some(speedupFactorFile)) assert(checker.getSpeedupFactor("SortExec") == 13.11) assert(checker.getSpeedupFactor("FilterExec") == 3.14) } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 6de463db1..0b493b0cf 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -485,20 +485,24 @@ class QualificationSuite extends BaseTestSuite { try { val allEventLines = bufferedSource.getLines.toList // the following val will contain the last two lines of the eventlog - //59 = "{"Event":"SparkListenerTaskEnd", - //60 = "{"Event":"SparkListenerStageCompleted" - //61 = "{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1718401564645," - //62 = "{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd"," - //63 = "{"Event":"SparkListenerApplicationEnd","Timestamp":1718401564663}" + // 59 = "{"Event":"SparkListenerTaskEnd", + // 60 = "{"Event":"SparkListenerStageCompleted" + // 61 = "{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1718401564645," + // 62 = "{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd"," + // 63 = "{"Event":"SparkListenerApplicationEnd","Timestamp":1718401564663}" val tailLines = allEventLines.takeRight(5) val selectedLines: List[String] = allEventLines.dropRight(5) + // scalastyle:off println selectedLines.foreach { line => pwList.foreach(pw => pw.println(line)) } + // scalastyle:on println for (i <- 0 to tailLines.length - 1) { if (i == 0) { // add truncatedTaskEvent to the brokenEventlog + // scalastyle:off println pwList(2).println(tailLines(i).substring(0, 59)) + // scalastyle:on println } // Write all the lines to the unfinishedLog and incompleteLog. // We do not want to ApplicationEnd in the incompleteLog @@ -508,7 +512,9 @@ class QualificationSuite extends BaseTestSuite { 0 // index of incomplete } for (lIndex <- startListInd to 1) { + // scalastyle:off println pwList(lIndex).println(tailLines(i)) + // scalastyle:on println } } // For the first two eventlogs, add a random incomplete line @@ -610,12 +616,12 @@ class QualificationSuite extends BaseTestSuite { test("test eventlog with no jobs") { val logFiles = Array(s"$logDir/empty_eventlog") - runQualificationTest(logFiles, shouldReturnEmpty=true) + runQualificationTest(logFiles, shouldReturnEmpty = true) } test("test eventlog with rdd only jobs") { val logFiles = Array(s"$logDir/rdd_only_eventlog") - runQualificationTest(logFiles, shouldReturnEmpty=true) + runQualificationTest(logFiles, shouldReturnEmpty = true) } test("test truncated log file 1") { @@ -721,7 +727,7 @@ class QualificationSuite extends BaseTestSuite { dfGen.write.parquet(dir) } - private def createIntFile(spark:SparkSession, dir:String): Unit = { + private def createIntFile(spark: SparkSession, dir: String): Unit = { import spark.implicits._ val t1 = Seq((1, 2), (3, 4), (1, 6)).toDF("a", "b") t1.write.parquet(dir) @@ -791,7 +797,7 @@ class QualificationSuite extends BaseTestSuite { // Spark3.2.+ generates a plan with 6 stages. StageID 3 and 4 are both // "isEmpty at RowMatrix.scala:441" val expStageCount = if (ToolUtils.isSpark320OrLater()) 6 else 5 - assert(mlOpsRes.mlFunctions.get.map(x=> x.stageId).size == expStageCount) + assert(mlOpsRes.mlFunctions.get.map(x => x.stageId).size == expStageCount) assert(mlOpsRes.mlFunctions.get.head.mlOps.mkString.contains( "org.apache.spark.ml.feature.PCA.fit")) assert(mlOpsRes.mlFunctionsStageDurations.get.head.mlFuncName.equals("PCA")) @@ -1056,7 +1062,7 @@ class QualificationSuite extends BaseTestSuite { val df1 = spark.sparkContext.parallelize(List(10, 20, 30, 40)).toDF df1.filter(hex($"value") === "A") // hex is not supported in GPU yet. } - //stdout output tests + // stdout output tests val sumOut = qualApp.getSummary() val detailedOut = qualApp.getDetailed() assert(sumOut.nonEmpty) @@ -1092,7 +1098,7 @@ class QualificationSuite extends BaseTestSuite { // so create a new one to read in the csv file createSparkSession() - //csv output tests + // csv output tests val outputResults = s"$outpath/rapids_4_spark_qualification_output/" + s"rapids_4_spark_qualification_output.csv" val outputActual = readExpectedFile(new File(outputResults), "\"") @@ -1100,7 +1106,7 @@ class QualificationSuite extends BaseTestSuite { assert(rows.size == 1) val expectedExecs = "Scan unknown;Filter;SerializeFromObject" // Unsupported Execs - val expectedExprs = "hex" //Unsupported Exprs + val expectedExprs = "hex" // Unsupported Exprs val unsupportedExecs = outputActual.select(QualOutputWriter.UNSUPPORTED_EXECS).first.getString(0) val unsupportedExprs = @@ -1223,7 +1229,7 @@ class QualificationSuite extends BaseTestSuite { val expr = ".*to_json.*" val matches = lines.filter(_.matches(expr)) assert(matches.length == 1) - //get line number containing to_json + // get line number containing to_json val lineNum = lines.indexOf(matches(0)) // check if lineNum has the expected value "This is disabled by default" assert(lines(lineNum).contains("This is disabled by default")) @@ -1467,12 +1473,12 @@ class QualificationSuite extends BaseTestSuite { } test("test frequency of repeated job") { - val logFiles = Array(s"$logDir/empty_eventlog", s"$logDir/nested_type_eventlog") + val logFiles = Array(s"$logDir/empty_eventlog", s"$logDir/nested_type_eventlog") runQualificationTest(logFiles, "multi_run_freq_test_expectation.csv") } test("test CSV qual output with escaped characters") { - val jobNames = List("test,name", "\"test\"name\"", "\"", ",", ",\"") + val jobNames = List("test,name", "\"test\"name\"", "\"", ",", ",\"") jobNames.foreach { jobName => TrampolineUtil.withTempDir { eventLogDir => val (eventLog, _) = diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala index 8b1007a9c..146038a94 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala @@ -43,7 +43,7 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { gpuMemory: Option[String] = Some(GpuDevice.DEFAULT.getMemory), gpuDevice: Option[String] = Some(GpuDevice.DEFAULT.toString)): String = { buildWorkerInfoAsString(customProps, numCores, systemMemory, numWorkers, - gpuCount, gpuMemory,gpuDevice) + gpuCount, gpuMemory, gpuDevice) } private def getGpuAppMockInfoProvider: AppSummaryInfoBaseProvider = { @@ -2212,7 +2212,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." .buildAutoTunerFromProps(databricksWorkerInfo, infoProvider, PlatformFactory.createInstance(PlatformNames.DATABRICKS_AWS)) // Assert shuffle manager string for DB 11.3 tag - verifyRecommendedShuffleManagerVersion(autoTuner, expectedSmVersion="330db") + verifyRecommendedShuffleManagerVersion(autoTuner, expectedSmVersion = "330db") } test("test shuffle manager version for supported spark version") { @@ -2226,7 +2226,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." .buildAutoTunerFromProps(workerInfo, infoProvider, PlatformFactory.createInstance()) // Assert shuffle manager string for supported Spark v3.3.0 - verifyRecommendedShuffleManagerVersion(autoTuner, expectedSmVersion="330") + verifyRecommendedShuffleManagerVersion(autoTuner, expectedSmVersion = "330") } test("test shuffle manager version for supported custom spark version") { @@ -2240,7 +2240,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." .buildAutoTunerFromProps(workerInfo, infoProvider, PlatformFactory.createInstance()) // Assert shuffle manager string for supported custom Spark v3.3.0 - verifyRecommendedShuffleManagerVersion(autoTuner, expectedSmVersion="330") + verifyRecommendedShuffleManagerVersion(autoTuner, expectedSmVersion = "330") } /** diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala index 0ca9714ca..c6891d636 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/QualificationAutoTunerSuite.scala @@ -57,7 +57,7 @@ class QualificationAutoTunerSuite extends BaseAutoTunerSuite { val workerInfo = buildCpuWorkerInfoAsString(None, Some(32), Some("212992MiB"), Some(5)) val infoProvider = getMockInfoProvider(0, Seq(0), Seq(0.0), - logEventsProps, Some(testSparkVersion)) + logEventsProps, Some(testSparkVersion)) val clusterPropsOpt = QualificationAutoTunerConfigsProvider .loadClusterPropertiesFromContent(workerInfo) val platform = PlatformFactory.createInstance(PlatformNames.EMR, clusterPropsOpt) From dd0b336cc262939ebc7d8921006a23a9c75a95b4 Mon Sep 17 00:00:00 2001 From: Jenkins Automation <70000568+nvauto@users.noreply.github.com> Date: Sat, 11 Jan 2025 07:16:47 +0800 Subject: [PATCH 38/52] Qualx model updates from weekly KPI run 2025-01-10 (#1495) Description: The latest /ssd0/qual/spark-rapids-tools-private/qual-kpis/kpi_summary_xgboost-2025-01-10.csv: platform,tp_count,fp_count,tn_count,fn_count,precision,recall databricks-aws,15,8,26,1,65.22,93.75 databricks-aws_photon,10,18,18,4,35.71,71.43 databricks-azure,18,1,21,8,94.74,69.23 databricks-azure_photon,20,7,6,15,74.07,57.14 dataproc,34,20,29,2,62.96,94.44 emr,13,9,25,4,59.09,76.47 onprem,27,19,28,2,58.7,93.1 Signed-off-by: nvauto <70000568+nvauto@users.noreply.github.com> --- .../qualx/models/xgboost/databricks-aws.cfg | 2 +- .../qualx/models/xgboost/databricks-aws.json | 2 +- .../models/xgboost/databricks-aws.metrics | 228 +++++++++--------- .../models/xgboost/databricks-aws_photon.cfg | 2 +- .../models/xgboost/databricks-aws_photon.json | 2 +- .../xgboost/databricks-aws_photon.metrics | 228 +++++++++--------- .../qualx/models/xgboost/databricks-azure.cfg | 2 +- .../models/xgboost/databricks-azure.json | 2 +- .../models/xgboost/databricks-azure.metrics | 226 ++++++++--------- .../xgboost/databricks-azure_photon.cfg | 2 +- .../xgboost/databricks-azure_photon.json | 2 +- .../xgboost/databricks-azure_photon.metrics | 226 ++++++++--------- .../qualx/models/xgboost/dataproc.cfg | 2 +- .../qualx/models/xgboost/dataproc.json | 2 +- .../qualx/models/xgboost/dataproc.metrics | 228 +++++++++--------- .../resources/qualx/models/xgboost/emr.cfg | 2 +- .../resources/qualx/models/xgboost/emr.json | 2 +- .../qualx/models/xgboost/emr.metrics | 162 ++++++------- .../resources/qualx/models/xgboost/onprem.cfg | 2 +- .../qualx/models/xgboost/onprem.json | 2 +- .../qualx/models/xgboost/onprem.metrics | 228 +++++++++--------- 21 files changed, 777 insertions(+), 777 deletions(-) diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg index f496f1297..a57381700 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"71"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.041366484","gamma":"0.0151119828","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.041366484","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"6","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.0151119828","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.706577659"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2358004E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0460410565","gamma":"0.00246754545","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0460410565","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"5","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00246754545","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.871091902"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2358004E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json index 17b951baf..bf93c39cb 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"71"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.1171235E-2,-2.4103911E-1,4.065256E-1,-2.0538169E-1,-2.8038349E-2,5.943318E-1,-1.8922163E-2,-3.320172E-1,-1.09247014E-1,3.9093933E-1,8.55308E-1,1.362929E-1,-9.27845E-3,-3.7015015E-1,-8.555055E-2,-9.870883E-3,-3.4725618E-1,4.7909573E-1,8.3032414E-2,5.1944204E-2,6.627762E-1,-8.4822625E-2,3.6801127E-1,-4.499109E-1,-2.6516694E-1,-1.0717573E-2,2.911777E-3,-8.021954E-2,1.9111152E-1,-2.3070526E-1,-2.6919978E-2,5.2855456E-1,9.013643E-3,6.2227827E-3,-1.6059658E-3,2.9637573E-2,1.1679127E-2,4.874526E-3,-8.512905E-3,1.9948091E-2,5.511552E-3,-2.2632578E-2,-1.1120226E-2,-9.298919E-4,-1.2590552E-2,2.041491E-3,-7.065667E-3,-4.5306037E-6,1.0279107E-2,-1.2121515E-2,-1.4474523E-3,1.1785626E-2,2.3449425E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,39,41,43,-1,-1,45,47,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1216166E1,1.807528E0,6.802891E0,1.388094E0,0E0,2.7973824E0,8.91264E-1,4.3825865E-1,1.6049374E0,9.102583E-1,1.0949898E0,8.510002E-1,0E0,2.4076033E-1,2.3472942E-1,7.081341E-1,5.573559E-1,2.3363113E-1,7.975551E-2,0E0,1.8579197E-1,2.4489176E-1,1.5027094E-1,2.5781584E-1,1.929338E-1,0E0,0E0,4.4271475E-1,1.4265987E-1,1.8490434E-1,0E0,4.387474E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,40,42,44,-1,-1,46,48,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,2.9432115E6,6.627576E7,3.2420947E2,-2.8038349E-2,3.7488223E3,1.739E3,1E0,8.19162E5,3.4264328E6,6.677E3,1.7543759E3,-9.27845E-3,2.1146445E4,1.661E3,8.564393E5,5.6252275E5,2.4025E4,1.4456249E7,5.1944204E-2,7.5E0,8.805777E7,1.1034263E10,6.5258E4,4.4289045E-2,-1.0717573E-2,2.911777E-3,6.290837E0,5.524659E8,2.6876314E8,-2.6919978E-2,1.9652087E-2,9.013643E-3,6.2227827E-3,-1.6059658E-3,2.9637573E-2,1.1679127E-2,4.874526E-3,-8.512905E-3,1.9948091E-2,5.511552E-3,-2.2632578E-2,-1.1120226E-2,-9.298919E-4,-1.2590552E-2,2.041491E-3,-7.065667E-3,-4.5306037E-6,1.0279107E-2,-1.2121515E-2,-1.4474523E-3,1.1785626E-2,2.3449425E-2],"split_indices":[52,28,45,52,0,4,0,112,29,47,9,52,0,33,2,28,28,9,32,0,53,45,5,29,57,0,0,53,5,7,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.24E2,8.4E1,1.16E2,8E0,5.8E1,2.6E1,4.9E1,6.7E1,3.4E1,2.4E1,1.5E1,1.1E1,4.2E1,7E0,4.8E1,1.9E1,2.6E1,8E0,6E0,1.8E1,8E0,7E0,2.2E1,2E1,3E0,4E0,3.6E1,1.2E1,1.5E1,4E0,2.1E1,5E0,5E0,3E0,1.5E1,3E0,3E0,5E0,4E0,3E0,1.3E1,9E0,3E0,1.7E1,1.5E1,2.1E1,3E0,9E0,1.1E1,4E0,4E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-8.553654E-4,-1.9865622E-1,4.0335876E-1,-1.6423997E-1,-3.1037115E-2,6.6852355E-1,3.636044E-2,-2.8224054E-1,1.1587561E-4,4.7430065E-2,5.131782E-1,-3.794686E-2,1.7187208E-2,-4.5945197E-1,-2.2141957E-1,1.7822178E-1,-2.913354E-1,2.4559808E-1,6.416441E-1,2.2156839E-1,-1.2687986E-1,-1.2192507E-2,-2.1964356E-2,-1.9571562E-1,-2.2021282E-2,2.592726E-1,4.7465768E-2,-1.1086058E-1,-5.397034E-1,1.5914414E-2,1.7522422E-3,1.2458134E-2,2.8197845E-2,1.2319113E-2,3.7205212E-3,-2.5782287E-3,-8.667633E-3,-2.979309E-3,-9.852544E-3,1.2353586E-2,3.0904694E-4,-2.387797E-3,8.888392E-3,-8.947042E-3,-1.9668583E-4,-2.6268244E-2,-1.083031E-2,3.0333425E-3,-2.1822068E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6790495E1,2.5537086E0,6.6904306E0,2.59884E0,0E0,2.382164E0,8.60921E-1,7.389655E-1,3.0107462E0,0E0,9.4241047E-1,6.281788E-1,0E0,6.731224E-2,3.7533402E-1,3.6921728E-1,9.2630005E-1,3.22998E-1,8.081913E-2,4.345867E-2,2.049376E-1,0E0,0E0,2.8222084E-1,0E0,2.158364E-1,2.765376E-1,1.5558097E-1,7.211447E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.814574E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,25,25,26,26,27,27,28,28,35,35],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3389954E3,3.013037E6,5.9337012E7,4.653861E2,-3.1037115E-2,6.677E3,8.129243E5,1.6414569E2,2.6519718E1,4.7430065E-2,3.8126804E1,9.893967E4,1.7187208E-2,2.2304833E0,4.991491E8,1.88832E5,2.7814416E11,2.1637352E7,2.608036E6,1.181E3,7.632764E7,-1.2192507E-2,-2.1964356E-2,1.8465776E8,-2.2021282E-2,7.5746506E1,2.0832575E1,7.1568984E7,1.9662491E3,1.5914414E-2,1.7522422E-3,1.2458134E-2,2.8197845E-2,1.2319113E-2,3.7205212E-3,2.1893E4,-8.667633E-3,-2.979309E-3,-9.852544E-3,1.2353586E-2,3.0904694E-4,-2.387797E-3,8.888392E-3,-8.947042E-3,-1.9668583E-4,-2.6268244E-2,-1.083031E-2,3.0333425E-3,-2.1822068E-3],"split_indices":[52,28,45,52,0,9,28,52,56,0,56,28,0,56,7,29,31,45,1,0,45,0,0,5,0,58,56,7,4,0,0,0,0,0,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.4E2,6.8E1,1.33E2,7E0,3.9E1,2.9E1,7.7E1,5.6E1,8E0,3.1E1,2.5E1,4E0,1.8E1,5.9E1,3.5E1,2.1E1,1.1E1,2E1,6E0,1.9E1,7E0,1.1E1,5.6E1,3E0,2.1E1,1.4E1,1.3E1,8E0,6E0,5E0,3E0,1.7E1,3E0,3E0,8E0,1.1E1,1.5E1,4.1E1,1.8E1,3E0,9E0,5E0,6E0,7E0,5E0,3E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-7.347498E-3,-2.2601315E-1,3.1835404E-1,-3.527297E-1,-1.2269206E-1,4.3737894E-1,-1.3200453E-1,-3.8988176E-1,-2.0015627E-1,-2.715264E-2,-4.852391E-1,3.0641013E-1,7.9347926E-1,-2.33868E-1,5.0024208E-2,-4.3395093E-1,-2.0219906E-1,-2.559801E-3,-1.0311766E-2,-1.3845548E-1,8.1520505E-2,-5.6147436E-3,-6.317668E-1,4.3674996E-1,1.1985658E-1,4.7746334E-2,5.8219576E-1,-1.2273007E-3,-2.921608E-1,8.4003275E-3,-3.4093412E-3,-2.2785701E-1,-5.203075E-1,3.1084285E-4,-1.1656283E-2,1.0952737E-2,-3.2066208E-1,1.1701825E-1,-5.5253627E-3,-3.0230232E-2,-1.5172415E-2,4.754004E-1,1.9585714E-3,1.8015878E-1,-2.3267668E-2,6.541322E-1,9.87921E-3,-5.9164306E-3,-1.388442E-2,-1.3524461E-2,-1.6264337E-4,-2.657537E-2,-1.5294696E-2,9.783577E-3,-2.0652625E-3,-5.9122425E-3,-1.6692424E-2,2.7343268E-3,9.764611E-3,2.1568023E-2,6.2206206E-3,2.3381577E-3,1.0029636E-2,2.549788E-3,-3.579833E-3,3.0248458E-2,1.3915745E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,-1,39,41,43,-1,45,-1,47,-1,-1,49,51,-1,-1,53,55,57,-1,-1,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6310053E1,1.7503943E0,4.989293E0,2.6773548E-1,2.6539145E0,3.1614475E0,3.8254064E-1,3.0926466E-1,7.966733E-2,7.612159E-1,7.336495E-1,1.284997E0,7.7619743E-1,1.4607519E-1,1.7889534E-1,5.577512E-1,1.8570057E-1,0E0,0E0,8.474302E-1,2.5995874E-1,0E0,2.0599365E-2,4.5911074E-1,2.1132883E-1,0E0,1.8959427E-1,0E0,1.6844034E-2,0E0,0E0,2.871551E-1,2.4058676E-1,0E0,0E0,2.5897723E-1,1.485188E-1,1.5410349E-1,0E0,0E0,0E0,3.5506487E-1,0E0,1.17236614E-1,4.8311766E-2,2.34766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,22,22,23,23,24,24,26,26,28,28,31,31,32,32,35,35,36,36,37,37,41,41,43,43,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,-1,40,42,44,-1,46,-1,48,-1,-1,50,52,-1,-1,54,56,58,-1,-1,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,2.8070276E2,9.298568E7,1.06623E5,3.8293724E7,9.467578E6,2.68442E9,4.565017E4,4.57E2,6.046394E-1,7.341E3,3.8279E5,6.677E3,1.13808104E8,1.739E3,3.7248906E3,1.1156E4,-2.559801E-3,-1.0311766E-2,6.8144927E0,9.63855E4,-5.6147436E-3,1.2728E4,7.9940905E6,2.0662125E9,4.7746334E-2,4.6E1,-1.2273007E-3,9.123462E-1,8.4003275E-3,-3.4093412E-3,4.1749332E4,1.9977521E4,3.1084285E-4,-1.1656283E-2,9.4390506E5,1.3839568E1,8.4569194E10,-5.5253627E-3,-3.0230232E-2,-1.5172415E-2,1.4E1,1.9585714E-3,2.692487E6,3E0,5.9283892E7,9.87921E-3,-5.9164306E-3,-1.388442E-2,-1.3524461E-2,-1.6264337E-4,-2.657537E-2,-1.5294696E-2,9.783577E-3,-2.0652625E-3,-5.9122425E-3,-1.6692424E-2,2.7343268E-3,9.764611E-3,2.1568023E-2,6.2206206E-3,2.3381577E-3,1.0029636E-2,2.549788E-3,-3.579833E-3,3.0248458E-2,1.3915745E-2],"split_indices":[52,52,45,29,45,1,7,33,0,27,9,29,9,45,0,47,9,0,0,53,33,0,9,47,12,0,3,0,27,0,0,32,33,0,0,32,56,31,0,0,0,8,0,1,8,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.27E2,1.36E2,9.1E1,6E1,7.6E1,7.2E1,1.9E1,4.7E1,1.3E1,6.1E1,1.5E1,5.4E1,1.8E1,1.2E1,7E0,3.7E1,1E1,4E0,9E0,3E1,3.1E1,5E0,1E1,3.1E1,2.3E1,5E0,1.3E1,3E0,9E0,3E0,4E0,1.2E1,2.5E1,3E0,7E0,1.7E1,1.3E1,2.7E1,4E0,6E0,4E0,2.8E1,3E0,1.6E1,7E0,1E1,3E0,3E0,6E0,8E0,4E0,1.2E1,1.3E1,3E0,1.4E1,5E0,8E0,2E1,7E0,2.4E1,4E0,6E0,1E1,3E0,4E0,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-1.1603539E-2,-1.9039239E-1,2.8651124E-1,-3.022029E-1,-8.490384E-2,3.9048842E-1,-1.3376175E-1,-3.921927E-1,-2.4349318E-1,-3.2986805E-2,-3.590458E-1,2.9148918E-1,7.34053E-1,-9.947233E-3,6.826865E-2,-6.7597195E-3,-1.7164156E-2,-2.1043219E-2,-2.9102173E-1,-8.873158E-2,8.0002144E-2,-3.73709E-3,-4.385762E-1,-2.2566938E-3,3.2648465E-1,8.373989E-1,9.948731E-3,-4.36609E-3,9.30815E-3,-5.8100214E-3,6.756443E-3,-1.3761938E-1,-3.2859415E-1,2.0449389E-2,-1.7252709E-1,1.4391337E-1,-4.732879E-3,-2.2213612E-2,-7.4998797E-3,1.607391E-1,4.1202042E-1,3.894348E-2,1.6297908E-2,-1.2397586E-2,2.150494E-3,-1.5162226E-3,-1.4927917E-2,-5.1027318E-3,5.2369144E-3,1.5184376E-3,-9.950003E-3,4.8321055E-4,8.192081E-3,1.3620295E-2,-9.867933E-4,4.213712E-3,1.8668206E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,29,31,33,35,-1,37,-1,39,41,-1,-1,-1,-1,-1,43,45,47,49,51,-1,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2366757E1,1.6845775E0,3.8540282E0,2.74333E-1,1.0698367E0,2.166916E0,4.0445128E-1,6.7439795E-2,4.696486E-1,4.1650403E-1,2.1682084E-1,6.959667E-1,5.636997E-1,0E0,2.1446611E-1,0E0,0E0,2.2108437E-1,1.6803598E-1,4.0607855E-1,2.8982297E-1,0E0,1.3052642E-1,0E0,6.568074E-1,1.8260956E-1,0E0,0E0,0E0,0E0,0E0,2.921653E-1,2.6002836E-1,3.2090756E-1,3.6455774E-1,1.19353E-1,0E0,0E0,0E0,5.989026E-1,3.5594416E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,22,22,24,24,25,25,31,31,32,32,33,33,34,34,35,35,39,39,40,40],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,30,32,34,36,-1,38,-1,40,42,-1,-1,-1,-1,-1,44,46,48,50,52,-1,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0207629E3,3.4008475E2,9.820072E7,1.7398648E2,6.0671224E7,5.087614E3,2.137881E5,1.0604926E3,8.1446184E7,8.129243E5,3.8E1,1.0625348E8,4.718769E6,-9.947233E-3,2.8109E4,-6.7597195E-3,-1.7164156E-2,4.7669297E-1,2.1573034E1,7.11E2,9.997036E0,-3.73709E-3,1.9662491E3,-2.2566938E-3,1E0,2.0216698E7,9.948731E-3,-4.36609E-3,9.30815E-3,-5.8100214E-3,6.756443E-3,4.18972E5,7.40087E2,2.8314105E4,4.410199E4,5.319824E2,-4.732879E-3,-2.2213612E-2,-7.4998797E-3,1.096E3,1.4964847E3,3.894348E-2,1.6297908E-2,-1.2397586E-2,2.150494E-3,-1.5162226E-3,-1.4927917E-2,-5.1027318E-3,5.2369144E-3,1.5184376E-3,-9.950003E-3,4.8321055E-4,8.192081E-3,1.3620295E-2,-9.867933E-4,4.213712E-3,1.8668206E-2],"split_indices":[52,52,45,52,45,52,33,48,5,28,3,7,29,0,2,0,0,27,58,0,53,0,4,0,109,47,0,0,0,0,0,12,33,33,33,52,0,0,0,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.3E2,1.44E2,8.6E1,6.9E1,7.5E1,6.9E1,1.7E1,2.5E1,4.4E1,6.4E1,1.1E1,5.5E1,1.4E1,1.1E1,6E0,3E0,2.2E1,8E0,3.6E1,4.3E1,2.1E1,3E0,8E0,5E0,5E1,1.1E1,3E0,3E0,3E0,5E0,3E0,8E0,2.8E1,1.9E1,2.4E1,1.6E1,5E0,5E0,3E0,1.8E1,3.2E1,8E0,3E0,4E0,4E0,3E0,2.5E1,8E0,1.1E1,6E0,1.8E1,5E0,1.1E1,9E0,9E0,4E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.1280696E-2,-2.2122468E-1,2.4967276E-1,-1.9044587E-1,-2.6468938E-2,1.2769474E-1,5.8723813E-1,-3.0301473E-1,-3.579468E-2,2.0369034E-1,-1.6051993E-1,3.7404045E-2,4.687106E-1,-3.9734557E-1,-2.3350383E-1,2.9052706E-2,-2.0462029E-1,1.08049214E-1,3.847881E-1,-2.6058173E-1,5.356912E-3,7.605375E-3,2.093031E-2,-3.7281E-3,-4.4685704E-1,1.4107327E-3,-2.716188E-1,6.328506E-2,-8.384629E-3,-1.5111546E-4,-3.1586283E-1,-8.918278E-2,1.7676596E-1,2.4253564E-1,5.678498E-1,-1.3902878E-2,-4.8752446E-3,-4.3381285E-3,4.7259224E-3,-1.0902009E-2,-2.0519925E-2,-1.2489184E-2,-9.678136E-4,-2.2452318E-3,4.013535E-3,-3.8383745E-3,-1.7040286E-2,-7.018613E-3,3.891987E-3,4.1182125E-3,1.2454669E-2,1.1643518E-3,1.3845027E-2,1.2027553E-2,2.7961876E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,-1,-1,-1,39,-1,41,43,-1,-1,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2162852E1,1.4630237E0,4.0013924E0,2.008587E0,0E0,1.6535215E0,5.530081E-1,3.465395E-1,5.533397E-1,9.803271E-1,2.6696286E-1,0E0,1.2987709E-1,3.7042427E-1,4.2745638E-1,3.0635375E-1,3.118316E-1,5.578902E-1,3.830955E-1,6.816125E-2,9.599837E-2,0E0,0E0,0E0,7.667923E-2,0E0,2.636683E-1,1.3615684E-1,0E0,0E0,1.5465623E-1,1.7821985E-1,2.5798345E-1,2.4742347E-1,4.310417E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,24,24,26,26,27,27,30,30,31,31,32,32,33,33,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,-1,-1,-1,40,-1,42,44,-1,-1,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,4.462437E7,1.5915463E0,3.4008475E2,-2.6468938E-2,9.996903E7,6.677E3,1.4300815E6,1.00149225E3,8.601996E-1,2.137881E5,3.7404045E-2,1.5262272E3,7.31E3,5.988024E-3,2.4025E4,9.222491E0,4.2906433E-2,1.6507974E3,5.535E3,2.8109E4,7.605375E-3,2.093031E-2,-3.7281E-3,4.7001828E7,1.4107327E-3,2.5653692E6,6.0570435E2,-8.384629E-3,-1.5111546E-4,1.6391889E5,7.588E3,3.4521E4,8.1350375E5,2.88446E5,-1.3902878E-2,-4.8752446E-3,-4.3381285E-3,4.7259224E-3,-1.0902009E-2,-2.0519925E-2,-1.2489184E-2,-9.678136E-4,-2.2452318E-3,4.013535E-3,-3.8383745E-3,-1.7040286E-2,-7.018613E-3,3.891987E-3,4.1182125E-3,1.2454669E-2,1.1643518E-3,1.3845027E-2,1.2027553E-2,2.7961876E-2],"split_indices":[52,45,39,52,0,45,9,32,55,27,33,0,4,9,57,9,54,38,52,9,2,0,0,0,5,0,51,4,0,0,28,2,10,28,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.2E2,1.22E2,9.8E1,1.15E2,7E0,7.3E1,2.5E1,6.6E1,4.9E1,5.8E1,1.5E1,5E0,2E1,2.6E1,4E1,3.6E1,1.3E1,3.9E1,1.9E1,9E0,6E0,3E0,1.7E1,4E0,2.2E1,5E0,3.5E1,3.2E1,4E0,5E0,8E0,1E1,2.9E1,1.2E1,7E0,5E0,4E0,3E0,3E0,6E0,1.6E1,3.1E1,4E0,7E0,2.5E1,3E0,5E0,7E0,3E0,1.9E1,1E1,4E0,8E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[8.6019235E-3,-2.0270345E-1,3.315331E-1,-2.8880727E-1,-1.16151154E-1,4.693588E-1,2.435183E-2,-4.2320988E-1,-2.4097885E-1,-1.8518299E-2,-4.079604E-1,2.7938792E-1,7.2414255E-1,1.3411233E-1,-1.12611234E-1,-1.0534867E-2,-1.9718008E-2,-4.1503924E-1,-1.8273626E-1,-8.714817E-2,1.5657859E-1,-6.220105E-3,-5.02761E-1,1.2081792E-2,3.9651933E-1,4.452663E-2,5.5625296E-1,-3.2796886E-2,1.4618715E-2,3.7465964E-3,-1.8262294E-1,-2.0970708E-2,-9.871603E-3,-2.0517841E-1,6.3681445E-4,4.6926767E-2,-1.3037579E-1,-3.577175E-3,2.245483E-1,-2.6383292E-2,-3.151955E-1,6.253592E-3,-3.1588231E-3,4.833053E-1,1.0015967E-2,2.6171366E-2,1.1799208E-2,3.8909896E-3,-1.0060207E-1,-9.596588E-3,-3.5930318E-3,-3.494992E-3,-1.0544417E-2,-1.1407773E-3,6.8493485E-3,-1.016242E-2,-1.4288822E-3,3.8390688E-3,1.1533743E-2,-6.706412E-3,-1.6111014E-2,9.728999E-3,2.1747801E-2,-4.2809086E-4,-6.854628E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,37,-1,39,41,43,-1,45,47,-1,-1,49,-1,-1,51,-1,53,55,-1,57,-1,59,-1,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5011125E1,9.576349E-1,3.6719313E0,3.2228088E-1,1.9297769E0,2.7017717E0,4.360303E-1,2.2758007E-2,4.4118142E-1,6.385231E-1,3.3954382E-1,1.1223671E0,8.797369E-1,5.971955E-1,2.0146689E-1,0E0,0E0,7.357955E-2,1.8101776E-1,2.263563E-1,2.6722226E-1,0E0,1.02900505E-1,1.5916738E-1,2.1795464E-1,0E0,2.130394E-1,9.5479265E-2,0E0,0E0,2.7125299E-2,0E0,0E0,1.945064E-1,0E0,9.296565E-2,3.119317E-1,0E0,5.9919953E-2,0E0,1.6444862E-2,0E0,0E0,3.4263372E-2,0E0,0E0,0E0,0E0,3.941539E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,22,22,23,23,24,24,26,26,27,27,30,30,33,33,35,35,36,36,38,38,40,40,43,43,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,38,-1,40,42,44,-1,46,48,-1,-1,50,-1,-1,52,-1,54,56,-1,58,-1,60,-1,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,2.98492E2,6.627576E7,7.056876E7,3.8293724E7,3.5078225E3,1.739E3,6.3323677E3,1.0893594E6,8.564393E5,2.7060036E11,1E0,6.677E3,6.805265E8,2.7350403E5,-1.0534867E-2,-1.9718008E-2,2.4830703E4,2.491016E-1,5.15E2,5.990846E2,-6.220105E-3,1.270805E6,2.8244882E6,1.7328871E6,4.452663E-2,5.7025972E7,8.805777E7,1.4618715E-2,3.7465964E-3,1.4688343E-1,-2.0970708E-2,-9.871603E-3,4.914413E-3,6.3681445E-4,2.1286093E8,1.5181E4,-3.577175E-3,1.127938E6,-2.6383292E-2,9.214292E7,6.253592E-3,-3.1588231E-3,1.138E3,1.0015967E-2,2.6171366E-2,1.1799208E-2,3.8909896E-3,2.6112175E6,-9.596588E-3,-3.5930318E-3,-3.494992E-3,-1.0544417E-2,-1.1407773E-3,6.8493485E-3,-1.016242E-2,-1.4288822E-3,3.8390688E-3,1.1533743E-2,-6.706412E-3,-1.6111014E-2,9.728999E-3,2.1747801E-2,-4.2809086E-4,-6.854628E-3],"split_indices":[52,52,45,7,45,52,0,50,45,28,31,109,9,7,28,0,0,33,38,0,4,0,29,32,47,0,45,45,0,0,38,0,0,38,0,7,10,0,28,0,7,0,0,10,0,0,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.32E2,8.6E1,6.5E1,6.7E1,5.9E1,2.7E1,1.5E1,5E1,5.1E1,1.6E1,3.5E1,2.4E1,1.5E1,1.2E1,5E0,1E1,1.1E1,3.9E1,3.7E1,1.4E1,5E0,1.1E1,1.1E1,2.4E1,6E0,1.8E1,9E0,6E0,3E0,9E0,6E0,5E0,3.5E1,4E0,9E0,2.8E1,3E0,1.1E1,5E0,6E0,4E0,7E0,1.4E1,1E1,1.3E1,5E0,3E0,6E0,5E0,4E0,1.1E1,2.4E1,6E0,3E0,1.2E1,1.6E1,4E0,7E0,3E0,3E0,3E0,1.1E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-6.0438593E-3,-1.3383913E-1,3.4762377E-1,-2.9424053E-1,-6.6097535E-2,5.5394006E-1,7.494315E-2,-3.28116E-1,-4.4113874E-2,2.462567E-2,-3.2150218E-1,4.1054186E-1,3.377369E-2,1.9597766E-1,-1.1892319E-1,-4.0224078E-1,-2.1331415E-1,-7.50506E-3,4.3115974E-3,-4.3919887E-2,1.7307758E-1,-4.9192923E-1,-7.9417266E-2,2.004593E-3,4.5996034E-1,-3.581405E-3,2.627093E-1,-9.304342E-3,-2.6218828E-2,-4.6104938E-1,-9.08579E-3,8.6144486E-4,-2.7713948E-1,5.3600144E-2,-1.3397242E-1,2.0990464E-1,-4.15158E-3,-2.7982152E-1,-2.621118E-2,3.383303E-2,-1.6858412E-1,2.7092492E-2,1.4813177E-2,3.220139E-1,1.4041518E-3,1.063504E-3,-2.9615203E-3,-1.0951606E-2,-2.324391E-2,-1.3424358E-2,-2.7179988E-3,-2.5864372E-3,5.1344596E-3,-9.216604E-3,-1.5910748E-4,1.4902383E-2,4.2532245E-3,-3.5278162E-3,-1.5010971E-2,-1.6895394E-3,4.138758E-3,-1.1344556E-2,-2.0823267E-3,7.2589032E-3,1.5747415E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,-1,33,35,37,39,-1,41,-1,43,-1,45,47,-1,-1,49,51,53,55,-1,57,-1,59,61,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0215861E1,1.7842817E0,3.333992E0,4.0071058E-1,2.7421105E0,9.446945E-1,6.5684825E-1,2.801118E-1,1.6149794E-1,9.0306485E-1,1.2399242E0,4.07156E-1,0E0,3.432858E-1,1.0219753E-1,1.6719484E-1,2.8970712E-1,0E0,0E0,5.4287845E-1,3.0302447E-1,3.637128E-1,1.4707811E-1,0E0,1.8255615E-1,0E0,1.790052E-1,0E0,1.8333696E-2,2.0759773E-1,0E0,0E0,1.2895322E-1,2.5343877E-1,3.6903948E-1,3.6546552E-1,0E0,1.1447233E-1,0E0,3.870076E-2,8.614653E-2,0E0,0E0,2.7763486E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,24,24,26,26,28,28,29,29,32,32,33,33,34,34,35,35,37,37,39,39,40,40,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,-1,34,36,38,40,-1,42,-1,44,-1,46,48,-1,-1,50,52,54,56,-1,58,-1,60,62,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5699575E3,2.7022223E2,5.9283892E7,1E0,3.8293724E7,1.12648186E2,1.739E3,2.1146445E4,1.661E3,7.761685E2,1.643614E6,7.441983E-3,3.377369E-2,5.34E2,2.68442E9,6.5258E4,5.754737E6,-7.50506E-3,4.3115974E-3,4.95036E5,3.1821228E7,9.214292E7,9.39E2,2.004593E-3,6.668107E0,-3.581405E-3,7.5214844E0,-9.304342E-3,4.0492815E-1,2.8434807E3,-9.08579E-3,8.6144486E-4,3.7607914E1,3.41043E2,1.1334876E0,2.6277386E8,-4.15158E-3,1.57727E3,-2.621118E-2,3.97E2,2.6876314E8,2.7092492E-2,1.4813177E-2,7.294459E7,1.4041518E-3,1.063504E-3,-2.9615203E-3,-1.0951606E-2,-2.324391E-2,-1.3424358E-2,-2.7179988E-3,-2.5864372E-3,5.1344596E-3,-9.216604E-3,-1.5910748E-4,1.4902383E-2,4.2532245E-3,-3.5278162E-3,-1.5010971E-2,-1.6895394E-3,4.138758E-3,-1.1344556E-2,-2.0823267E-3,7.2589032E-3,1.5747415E-2],"split_indices":[52,52,45,112,45,56,0,33,2,52,32,39,0,0,7,29,12,0,0,1,45,7,0,0,54,0,54,0,27,47,0,0,58,52,39,12,0,4,0,0,7,0,0,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E2,1.65E2,5.9E1,4.8E1,1.17E2,3.3E1,2.6E1,4.2E1,6E0,8.7E1,3E1,2.3E1,1E1,1.6E1,1E1,2.4E1,1.8E1,3E0,3E0,6E1,2.7E1,1.7E1,1.3E1,3E0,2E1,3E0,1.3E1,4E0,6E0,1.7E1,7E0,4E0,1.4E1,2.9E1,3.1E1,2.4E1,3E0,8E0,9E0,6E0,7E0,5E0,1.5E1,1E1,3E0,3E0,3E0,7E0,1E1,1.1E1,3E0,1.1E1,1.8E1,1.8E1,1.3E1,9E0,1.5E1,3E0,5E0,3E0,3E0,3E0,4E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-2.007114E-3,-1.7659411E-1,2.6283842E-1,-1.4297768E-1,-4.5692348E-1,1.2653993E-1,5.137665E-1,-2.1811382E-1,3.5527352E-2,-5.89053E-3,-2.547503E-2,2.2936018E-1,-5.6810487E-2,6.5413374E-1,2.9659685E-1,-3.6286622E-1,-1.7446074E-1,1.1563214E-1,-1.6432326E-1,7.4859746E-2,3.508672E-1,1.3606146E-1,-1.3764131E-1,1.158168E-2,2.9382605E-2,3.425197E-3,1.6801909E-2,-5.1176283E-3,-3.9997506E-1,-1.3136911E-1,-3.4666616E-1,1.8501252E-1,-2.081576E-2,-7.285474E-2,-1.2665568E-2,1.4509353E-1,-2.3262554E-3,2.0689514E-1,2.020947E-2,-6.0147613E-5,9.90982E-3,4.846175E-3,-2.0545152E-1,-7.7218423E-3,-1.8843314E-2,4.645063E-3,-6.2989406E-3,-2.1571636E-3,-1.9515429E-2,9.004615E-3,6.7262206E-4,1.9204525E-3,-5.513479E-3,6.823964E-4,-5.3679086E-3,-4.8593528E-4,8.21865E-3,1.1214584E-2,4.427608E-4,-1.0584135E-2,-5.1993644E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,43,45,47,49,51,53,-1,55,-1,57,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.035762E1,1.2068663E0,2.9788885E0,1.6440346E0,6.1205935E-1,1.125306E0,7.509737E-1,4.8011827E-1,6.1133367E-1,0E0,0E0,6.871041E-1,3.6170763E-1,1.7940044E-1,2.9434216E-1,1.1913562E-1,4.590602E-1,2.59455E-1,1.2042251E-1,1.7389074E-1,3.151846E-1,9.99784E-2,3.0051073E-1,0E0,0E0,0E0,0E0,0E0,1.0454178E-1,3.0231166E-1,4.5438492E-1,9.5686376E-2,8.181241E-2,4.2820364E-2,0E0,1.0318637E-1,0E0,1.4826328E-1,0E0,0E0,0E0,0E0,2.011156E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,28,28,29,29,30,30,31,31,32,32,33,33,35,35,37,37,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,44,46,48,50,52,54,-1,56,-1,58,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,4.56567E7,1.5915463E0,4.561433E2,9.180692E7,6.627576E7,8.812601E-1,1.5977058E2,1.1424464E3,-5.89053E-3,-2.547503E-2,6.2711465E-1,1.0937031E5,8.81832E5,2.40912E5,1E0,1.996866E1,5.9122612E1,9.24E2,3.8E1,4.8759037E1,3.76978E7,4.4292E5,1.158168E-2,2.9382605E-2,3.425197E-3,1.6801909E-2,-5.1176283E-3,2.5E3,4.570007E7,3.3E1,2.2198856E1,1.873E3,4.084859E8,-1.2665568E-2,1E0,-2.3262554E-3,1.204378E6,2.020947E-2,-6.0147613E-5,9.90982E-3,4.846175E-3,7.381975E8,-7.7218423E-3,-1.8843314E-2,4.645063E-3,-6.2989406E-3,-2.1571636E-3,-1.9515429E-2,9.004615E-3,6.7262206E-4,1.9204525E-3,-5.513479E-3,6.823964E-4,-5.3679086E-3,-4.8593528E-4,8.21865E-3,1.1214584E-2,4.427608E-4,-1.0584135E-2,-5.1993644E-3],"split_indices":[52,45,39,52,7,45,57,52,4,0,0,27,28,1,29,109,56,58,0,3,56,32,29,0,0,0,0,0,29,5,3,56,0,7,0,109,0,29,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.34E2,8.8E1,1.21E2,1.3E1,5.8E1,3E1,8.5E1,3.6E1,5E0,8E0,3.7E1,2.1E1,1.7E1,1.3E1,1.8E1,6.7E1,2.6E1,1E1,1.7E1,2E1,6E0,1.5E1,3E0,1.4E1,5E0,8E0,3E0,1.5E1,5.5E1,1.2E1,1.7E1,9E0,7E0,3E0,1.1E1,6E0,1.1E1,9E0,3E0,3E0,3E0,1.2E1,4E0,1.1E1,4E0,5.1E1,4E0,8E0,1.4E1,3E0,6E0,3E0,3E0,4E0,3E0,8E0,8E0,3E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.3388702E-2,-1.9777314E-1,2.371269E-1,-1.6924144E-1,-6.0815984E-1,3.6889765E-1,2.976296E-2,-2.5455114E-1,-4.950625E-2,-2.9741485E-2,-1.3138013E-2,2.257671E-1,6.2955725E-1,-1.545455E-1,9.229129E-2,-3.5589874E-1,-2.1341614E-1,-1.04812786E-1,1.4358327E-1,3.131542E-1,9.4257124E-2,7.6592475E-1,2.6688552E-1,-1.0526053E-2,-1.883296E-3,3.289743E-4,1.3801979E-2,-4.40885E-1,-7.608622E-3,-9.870368E-2,-2.91641E-1,7.4017285E-3,-1.9295466E-1,3.4258413E-4,8.695187E-3,3.4728333E-1,2.65655E-3,7.1692807E-3,4.7299E-3,3.445149E-2,1.6151061E-2,5.097591E-3,1.4222611E-2,1.1202241E-2,-5.6630615E-2,-9.222197E-3,-2.159292E-2,-1.108244E-2,-2.0261768E-3,-4.947472E-3,-1.4324425E-2,-4.3994677E-3,4.981216E-3,-1.0608423E-2,-1.7309912E-3,1.5789269E-2,4.725057E-3,3.927415E-3,-3.5752277E-3,-4.185337E-3,3.6540958E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,-1,45,-1,47,49,51,53,-1,-1,55,-1,-1,57,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.034753E1,1.4075389E0,2.584848E0,1.233576E0,2.9241323E-2,2.0217505E0,4.521942E-1,2.2003937E-1,5.703633E-1,0E0,0E0,4.1868162E-1,7.84719E-1,9.7338766E-2,6.4357483E-1,1.9658875E-1,4.4680238E-1,4.0695086E-1,1.0641752E-1,1.7310357E-1,1.19497806E-1,2.656746E-2,3.49952E-2,0E0,0E0,3.5427204E-1,0E0,9.574795E-2,0E0,1.8038353E-1,2.4999261E-1,2.5707144E-1,2.0994681E-1,0E0,0E0,1.1678767E-1,0E0,0E0,8.221763E-2,0E0,0E0,0E0,0E0,0E0,1.3163339E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,27,27,29,29,30,30,31,31,32,32,35,35,38,38,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,-1,46,-1,48,50,52,54,-1,-1,56,-1,-1,58,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.969686E2,6.8652206E9,5.9283892E7,3.4008475E2,1.1992621E3,3.7488223E3,1.08017064E8,1.056E3,6.98247E5,-2.9741485E-2,-1.3138013E-2,4.24975E5,3.4912622E0,1.8159722E-1,1.0526E4,6.974669E7,1.2623029E7,7.11E2,6.6962665E2,7.8582473E0,3.5E1,1.2003011E8,5.087614E3,-1.0526053E-2,-1.883296E-3,3.0429645E0,1.3801979E-2,6.3323677E3,-7.608622E-3,5.815166E0,6.338275E0,2.8314105E4,6.041926E1,3.4258413E-4,8.695187E-3,1.5583563E8,2.65655E-3,7.1692807E-3,1.0731037E-1,3.445149E-2,1.6151061E-2,5.097591E-3,1.4222611E-2,1.1202241E-2,1.1192292E1,-9.222197E-3,-2.159292E-2,-1.108244E-2,-2.0261768E-3,-4.947472E-3,-1.4324425E-2,-4.3994677E-3,4.981216E-3,-1.0608423E-2,-1.7309912E-3,1.5789269E-2,4.725057E-3,3.927415E-3,-3.5752277E-3,-4.185337E-3,3.6540958E-3],"split_indices":[52,5,45,52,4,4,7,2,28,0,0,29,57,27,9,7,12,0,4,53,3,32,52,0,0,53,0,50,0,54,53,33,58,0,0,32,0,0,27,0,0,0,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.28E2,9.4E1,1.21E2,7E0,5.7E1,3.7E1,7E1,5.1E1,4E0,3E0,3.8E1,1.9E1,9E0,2.8E1,1.8E1,5.2E1,4E1,1.1E1,2.2E1,1.6E1,1.3E1,6E0,4E0,5E0,2.1E1,7E0,1.1E1,7E0,2.2E1,3E1,1.8E1,2.2E1,4E0,7E0,1.9E1,3E0,8E0,8E0,1E1,3E0,3E0,3E0,3E0,1.8E1,4E0,7E0,4E0,1.8E1,8E0,2.2E1,9E0,9E0,1.5E1,7E0,1.6E1,3E0,4E0,4E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[3.5809882E-2,-1.55102E-1,2.3709244E-1,-2.4052006E-1,-4.2664845E-2,3.4317666E-1,-7.7007536E-4,-2.626203E-1,-2.3043858E-2,1.4003679E-2,-3.4192505E-1,3.7578505E-2,2.608972E-1,-1.0431229E-2,6.2106464E-2,-2.3319556E-1,-1.8353764E-2,-5.6942045E-3,4.026028E-3,1.0880717E-1,-6.5582715E-2,-1.7154338E-2,-6.845553E-3,1.7776258E-1,4.7271374E-1,1.3991291E-2,1.544281E-2,-1.0568507E-1,-2.610389E-1,1.4655255E-1,-3.2601939E-3,-1.243839E-1,5.441822E-3,2.7573773E-1,8.048696E-2,2.1685172E-2,6.675872E-3,-5.489506E-2,1.4658736E-1,-8.042627E-3,1.6416745E-3,-6.2837875E-3,-1.3616752E-2,3.2179637E-3,8.860067E-3,-7.0829494E-3,-2.6931667E-3,1.3673418E-2,6.864993E-3,-2.3791706E-3,5.5578626E-3,3.7806117E-4,-7.1901E-3,1.1292391E-2,-3.7754464E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,-1,-1,-1,29,31,-1,-1,33,35,-1,37,39,41,43,-1,45,-1,47,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.452336E0,1.0750086E0,2.700245E0,3.0154943E-1,8.5271996E-1,3.2046328E0,5.530964E-1,2.0554876E-1,1.099651E-1,3.315738E-1,3.408581E-2,0E0,1.0781822E0,0E0,3.555519E-1,1.5702844E-1,0E0,0E0,0E0,1.5318562E-1,2.9456425E-1,0E0,0E0,4.4480574E-1,2.0404243E-1,0E0,2.3865762E-1,1.4961642E-1,2.6229954E-1,6.23959E-2,0E0,4.160878E-2,0E0,8.954513E-2,2.0101391E-1,0E0,0E0,1.3104254E-1,1.7962734E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,19,19,20,20,23,23,24,24,26,26,27,27,28,28,29,29,31,31,33,33,34,34,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,-1,-1,-1,30,32,-1,-1,34,36,-1,38,40,42,44,-1,46,-1,48,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3350134E2,3.4008475E2,6.627576E7,7.06836E7,3.2271808E7,6.677E3,7.056125E7,9.278236E-1,5.3153326E2,7.47E2,1.2728E4,3.7578505E-2,1.1946838E7,-1.0431229E-2,3.566963E0,8.6E1,-1.8353764E-2,-5.6942045E-3,4.026028E-3,3.8935616E8,8.129243E5,-1.7154338E-2,-6.845553E-3,1.259779E5,5E1,1.3991291E-2,1.0377E4,1.295E3,6.5833335E0,2.9464627E10,-3.2601939E-3,5.0423752E2,5.441822E-3,6.7759786E0,6.4866364E-1,2.1685172E-2,6.675872E-3,2.2980049E10,9.820072E7,-8.042627E-3,1.6416745E-3,-6.2837875E-3,-1.3616752E-2,3.2179637E-3,8.860067E-3,-7.0829494E-3,-2.6931667E-3,1.3673418E-2,6.864993E-3,-2.3791706E-3,5.5578626E-3,3.7806117E-4,-7.1901E-3,1.1292391E-2,-3.7754464E-4],"split_indices":[52,52,45,32,45,9,7,27,4,0,9,0,1,0,54,10,0,0,0,7,28,0,0,33,3,0,9,2,53,31,0,52,0,53,39,0,0,5,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.12E2,1.06E2,6.3E1,4.9E1,7.3E1,3.3E1,5.7E1,6E0,4.2E1,7E0,8E0,6.5E1,6E0,2.7E1,5.1E1,6E0,3E0,3E0,1.9E1,2.3E1,4E0,3E0,4.8E1,1.7E1,3E0,2.4E1,1E1,4.1E1,1.6E1,3E0,1.8E1,5E0,2.3E1,2.5E1,1.4E1,3E0,1.6E1,8E0,6E0,4E0,1.7E1,2.4E1,9E0,7E0,9E0,9E0,1.4E1,9E0,7E0,1.8E1,1.1E1,5E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[9.248212E-3,-1.3050836E-1,2.1051846E-1,-1.02454215E-1,-4.071877E-1,3.1866005E-1,-5.5845864E-3,-1.7783755E-1,5.8412157E-2,-4.6271384E-3,-5.4087085E-1,2.2802614E-1,6.1311734E-1,-1.299225E-1,1.1176783E-1,-3.1064636E-1,-1.3691022E-1,-2.6133299E-2,1.8686368E-1,-1.1795772E-2,-2.636166E-2,1.0598886E-1,3.4094238E-1,3.0578686E-2,4.677387E-3,-9.83251E-3,-7.4191384E-2,1.2505171E-2,-5.1836304E-2,-4.082903E-3,-1.4239062E-2,-7.169311E-2,-1.999548E-1,7.229739E-2,-9.6573584E-2,2.2518018E-1,6.461066E-4,-3.239553E-3,1.8201274E-1,4.3119013E-1,2.2446829E-1,-1.7100219E-2,-5.3128013E-3,-5.8099926E-3,4.8067733E-3,-6.2107206E-3,-7.6946875E-5,-4.8364336E-3,-1.4701738E-2,5.5158E-3,-1.1426042E-3,-1.6730177E-4,-6.528059E-3,1.1338382E-2,5.064468E-3,1.867177E-3,9.563309E-3,8.708599E-3,1.9704426E-2,1.4175667E-3,1.1504076E-2,2.1954793E-3,-3.4333873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,39,-1,-1,-1,41,-1,43,-1,-1,45,47,49,51,53,-1,-1,55,57,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.3845925E0,9.9839044E-1,2.1751642E0,1.5019652E0,4.132682E-1,1.5148373E0,4.8144445E-1,4.140525E-1,4.3928307E-1,0E0,1.54902935E-2,6.390834E-1,7.9917336E-1,7.846367E-2,5.4559726E-1,1.01214886E-1,2.5702906E-1,1.8031916E-1,1.01466894E-1,0E0,0E0,3.6453846E-1,1.6599965E-1,0E0,0E0,0E0,3.496404E-2,0E0,1.652262E-1,0E0,0E0,1.8597637E-1,3.8441324E-1,7.0775256E-2,8.435458E-2,3.178227E-2,0E0,0E0,1.10713184E-1,2.923417E-2,1.23082936E-1,3.677592E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,26,26,28,28,31,31,32,32,33,33,34,34,35,35,38,38,39,39,40,40,41,41],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,40,-1,-1,-1,42,-1,44,-1,-1,46,48,50,52,54,-1,-1,56,58,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.156076E7,6.1108776E7,4.653861E2,5.328E3,5.206759E3,2.4654E4,1.5977058E2,5.370622E5,-4.6271384E-3,5.673844E5,5.8319894E5,1.095857E10,1.8159722E-1,1.739E3,1.0712912E2,1.3736346E8,6.290837E0,8.542832E0,-1.1795772E-2,-2.636166E-2,1E0,1.2951E4,3.0578686E-2,4.677387E-3,-9.83251E-3,2E0,1.2505171E-2,5.7E1,-4.082903E-3,-1.4239062E-2,7.484013E5,1.3386886E1,6.59182E1,3.119E3,7E0,6.461066E-4,-3.239553E-3,1.0688497E3,7.07E2,1.57561E5,2.6112175E6,-5.3128013E-3,-5.8099926E-3,4.8067733E-3,-6.2107206E-3,-7.6946875E-5,-4.8364336E-3,-1.4701738E-2,5.5158E-3,-1.1426042E-3,-1.6730177E-4,-6.528059E-3,1.1338382E-2,5.064468E-3,1.867177E-3,9.563309E-3,8.708599E-3,1.9704426E-2,1.4175667E-3,1.1504076E-2,2.1954793E-3,-3.4333873E-3],"split_indices":[52,45,45,52,9,52,2,52,28,0,28,28,12,27,0,52,7,53,53,0,0,109,9,0,0,0,8,0,3,0,0,32,56,58,2,8,0,0,52,0,29,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.25E2,1.33E2,9.2E1,1.22E2,1.1E1,6.1E1,3.1E1,8.3E1,3.9E1,4E0,7E0,4.8E1,1.3E1,1.5E1,1.6E1,1.8E1,6.5E1,2.4E1,1.5E1,3E0,4E0,2.4E1,2.4E1,1E1,3E0,4E0,1.1E1,7E0,9E0,3E0,1.5E1,3.3E1,3.2E1,1E1,1.4E1,1.2E1,3E0,7E0,1.7E1,1.2E1,1.2E1,6E0,5E0,6E0,3E0,1.5E1,1.8E1,2.2E1,1E1,6E0,4E0,6E0,8E0,7E0,5E0,5E0,1.2E1,3E0,9E0,3E0,9E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.3522198E-3,-1.468343E-1,2.4363932E-1,-1.2032509E-1,-4.5400622E-1,3.7953445E-1,2.2118522E-2,-2.2337422E-1,-1.2330142E-2,-2.3117172E-2,-5.4124207E-3,3.95635E-2,3.1153584E-1,-4.004426E-2,2.5872946E-1,-2.5278705E-1,-6.50488E-2,1.18920185E-1,-1.0061586E-1,1.9131999E-1,4.7027203E-1,-1.5256189E-1,1.1331626E-1,1.4115592E-2,4.6141823E-3,-3.3048287E-1,-1.605134E-1,3.8816864E-3,-6.462314E-3,1.5910237E-1,-1.9952918E-3,-1.6709223E-1,9.541955E-3,2.9373124E-1,6.0612854E-2,6.515015E-3,5.162036E-1,-2.8945394E-3,-8.64654E-3,1.7123988E-1,-1.8755962E-3,-1.4913493E-2,-1.8657832E-3,-1.5058262E-4,-8.795802E-3,8.3095655E-3,5.8841833E-4,-8.867523E-3,7.583334E-4,-6.058609E-3,2.3499932E-3,1.3755127E-2,3.8983526E-3,4.1834726E-3,-2.7288578E-3,1.1711527E-2,2.4395065E-2,9.662788E-3,4.1835294E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,43,-1,-1,45,-1,47,49,51,53,-1,55,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.068124E0,1.1098042E0,2.520308E0,1.4773743E0,2.9954028E-1,1.7418485E0,4.9573764E-1,3.0021882E-1,7.7674216E-1,0E0,0E0,0E0,8.259926E-1,4.831933E-1,4.6940416E-2,3.556018E-1,1.8849035E-1,1.890235E-1,2.9451007E-1,3.703866E-1,2.0602703E-1,5.9982747E-2,1.1804418E-1,0E0,0E0,2.4090815E-1,2.2815049E-1,0E0,0E0,1.3029075E-1,0E0,2.2306728E-1,1.2630202E-1,9.246802E-2,7.8476526E-2,0E0,1.24385834E-1,0E0,0E0,1.5716523E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,29,29,31,31,32,32,33,33,34,34,36,36,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,44,-1,-1,46,-1,48,50,52,54,-1,56,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,6.0671224E7,5.9337012E7,3.41043E2,1.8611923E3,4.9695907E0,8.129243E5,1E0,7.25E2,-2.3117172E-2,-5.4124207E-3,3.95635E-2,3.614485E3,9.5891304E1,9.040637E7,2.1146445E4,3E1,2.6921875E1,3.4288502E8,1.7747324E7,4.747723E6,1.650269E10,9.744113E0,1.4115592E-2,4.6141823E-3,1E0,7.9008045E0,3.8816864E-3,-6.462314E-3,1E0,-1.9952918E-3,8.595345E-1,1E0,7.8582473E0,1.2931905E7,6.515015E-3,5.0777936E-1,-2.8945394E-3,-8.64654E-3,2.2946306E5,-1.8755962E-3,-1.4913493E-2,-1.8657832E-3,-1.5058262E-4,-8.795802E-3,8.3095655E-3,5.8841833E-4,-8.867523E-3,7.583334E-4,-6.058609E-3,2.3499932E-3,1.3755127E-2,3.8983526E-3,4.1834726E-3,-2.7288578E-3,1.1711527E-2,2.4395065E-2,9.662788E-3,4.1835294E-3],"split_indices":[52,45,45,52,4,54,28,112,0,0,0,0,4,56,45,33,3,56,7,45,32,5,54,0,0,79,54,0,0,8,0,27,109,53,32,0,57,0,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.25E2,1.42E2,8.3E1,1.32E2,1E1,5.1E1,3.2E1,6.7E1,6.5E1,7E0,3E0,4E0,4.7E1,2.6E1,6E0,5.6E1,1.1E1,2.6E1,3.9E1,2.8E1,1.9E1,1.5E1,1.1E1,3E0,3E0,2.9E1,2.7E1,4E0,7E0,2.1E1,5E0,2.4E1,1.5E1,1.5E1,1.3E1,3E0,1.6E1,7E0,8E0,8E0,3E0,2.6E1,3E0,7E0,2E1,1.6E1,5E0,1.9E1,5E0,3E0,1.2E1,1.2E1,3E0,1E1,3E0,5E0,1.1E1,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-6.943856E-3,-1.7918782E-1,1.6981454E-1,-2.3636249E-1,-9.5559515E-2,2.4460469E-1,-3.4153458E-2,-2.1320203E-1,-1.7140169E-2,-1.3503878E-2,-3.5427704E-1,1.6963747E-1,5.344949E-1,-9.89848E-3,5.652016E-2,-2.8281295E-1,-1.6436514E-1,2.13063E-2,-8.625363E-3,-1.7681492E-2,-4.182232E-3,2.804068E-2,2.3100188E-1,6.385621E-1,9.815356E-3,1.7885469E-1,-5.1341377E-2,-2.99512E-1,-4.9749445E-3,-8.24991E-2,-2.4210623E-1,-7.538815E-2,7.082921E-2,-7.51579E-2,1.8545617E-1,9.57974E-2,2.8955552E-1,3.1005299E-2,1.3787691E-2,1.0306439E-2,1.8478817E-3,-4.5971605E-3,4.590512E-3,-1.3400131E-2,-6.2744617E-3,-7.185726E-3,6.7057565E-4,-1.1971729E-2,8.044256E-4,9.748252E-4,-5.287079E-3,7.3981383E-3,9.177652E-4,-7.981589E-4,-5.3058546E-3,1.1328667E-2,2.4803376E-3,8.511057E-3,1.12168E-3,1.402007E-2,4.9743024E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,29,31,-1,-1,-1,33,35,37,-1,39,41,43,-1,45,47,49,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1546917E0,5.464022E-1,1.7779107E0,2.0021772E-1,1.0566775E0,1.7565713E0,6.087322E-1,1.710887E-1,0E0,2.6918355E-1,1.7904723E-1,5.9306383E-1,3.743024E-1,0E0,3.1267202E-1,3.1839013E-2,2.3460066E-1,1.6815177E-1,0E0,0E0,0E0,3.7132967E-1,3.522637E-1,1.5665102E-1,0E0,9.262687E-2,1.3849607E-1,3.0302763E-2,0E0,1.9188502E-1,2.5305784E-1,6.526083E-2,1.1687768E-1,3.905899E-2,8.3429456E-2,1.1524619E-1,2.3506784E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,21,21,22,22,23,23,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,30,32,-1,-1,-1,34,36,38,-1,40,42,44,-1,46,48,50,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.26031E2,3.4008475E2,6.627576E7,1.3865231E6,3.8293724E7,5.087614E3,7.340097E7,1.775107E2,-1.7140169E-2,2.6519718E1,1.6491606E3,1E0,9.299267E9,-9.89848E-3,1.256E3,9.314573E8,9.360022E5,4.3062625E2,-8.625363E-3,-1.7681492E-2,-4.182232E-3,1.1223402E6,1.0866135E3,3.0291426E-1,9.815356E-3,1E0,8.3E1,8.625696E-1,-4.9749445E-3,1.549E3,2.491016E-1,4.61E2,3.7568363E4,5E0,2E0,7.47E2,3.858136E6,3.1005299E-2,1.3787691E-2,1.0306439E-2,1.8478817E-3,-4.5971605E-3,4.590512E-3,-1.3400131E-2,-6.2744617E-3,-7.185726E-3,6.7057565E-4,-1.1971729E-2,8.044256E-4,9.748252E-4,-5.287079E-3,7.3981383E-3,9.177652E-4,-7.981589E-4,-5.3058546E-3,1.1328667E-2,2.4803376E-3,8.511057E-3,1.12168E-3,1.402007E-2,4.9743024E-3],"split_indices":[52,52,45,28,45,52,7,52,0,56,4,109,12,0,0,5,48,52,0,0,0,28,52,38,0,8,3,27,0,2,38,0,33,8,8,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.33E2,1.18E2,1.15E2,6.9E1,4.9E1,8.4E1,3.1E1,6.3E1,6E0,3.8E1,1.1E1,6.8E1,1.6E1,9E0,2.2E1,2.4E1,3.9E1,3.3E1,5E0,8E0,3E0,2.1E1,4.7E1,1.1E1,5E0,1E1,1.2E1,2.1E1,3E0,2E1,1.9E1,1.1E1,2.2E1,1.3E1,8E0,1.5E1,3.2E1,7E0,4E0,6E0,4E0,9E0,3E0,1.7E1,4E0,1E1,1E1,1.6E1,3E0,4E0,7E0,6E0,1.6E1,7E0,6E0,4E0,4E0,5E0,1E1,2.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-4.115728E-3,-1.434476E-1,2.1213621E-1,-1.1797723E-1,-2.4502922E-2,1.1908382E-1,4.3814275E-1,-2.0818311E-1,-3.139592E-2,1.7242208E-1,-9.932659E-2,5.656861E-1,2.2438888E-1,-2.8645778E-1,-1.4914826E-1,1.3182755E-2,-2.3681341E-1,8.837354E-2,3.093263E-1,-5.6628585E-3,8.0357434E-4,1.0948562E-2,2.5601033E-2,1.2714021E-2,2.621163E-3,-3.7767955E-3,-3.0794343E-1,3.2413313E-3,-1.7821811E-1,8.300433E-2,-6.488502E-2,-1.6474312E-2,-5.739601E-3,3.1465787E-2,1.6578192E-2,2.1502174E-1,1.7047843E-2,-1.496793E-2,-8.945751E-3,-1.2205193E-2,-3.3789414E-3,4.8460388E-3,-1.8995742E-3,-4.581377E-3,3.3913078E-3,-6.9818427E-3,2.906258E-3,1.0762618E-2,2.4681897E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,37,-1,39,41,43,-1,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.689334E0,1.4736259E0,1.7774732E0,1.0009725E0,0E0,7.510529E-1,5.546384E-1,2.4839401E-1,6.166569E-1,5.595002E-1,6.0656443E-2,7.639599E-2,1.2747282E-1,8.0904245E-2,2.6527876E-1,3.106314E-1,1.3471776E-1,5.740448E-1,9.556532E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.540206E-2,0E0,3.59365E-1,1.3744223E-1,1.9095674E-1,0E0,0E0,2.3711982E-1,0E0,6.8657935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,26,26,28,28,29,29,30,30,33,33,35,35],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,38,-1,40,42,44,-1,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.013037E6,1.8009317E0,3.067147E2,-2.4502922E-2,9.996903E7,8.812601E-1,1.8452284E2,8.0019E5,6.3901825E5,5.7E1,2.6207143E-1,8.2731545E-1,1.0604926E3,4.6376264E9,7.25E2,9.32784E5,7.773148E3,9.011E3,-5.6628585E-3,8.0357434E-4,1.0948562E-2,2.5601033E-2,1.2714021E-2,2.621163E-3,-3.7767955E-3,3.4E1,3.2413313E-3,1.5662762E10,2.2026947E1,8.129243E5,-1.6474312E-2,-5.739601E-3,2.4146196E-3,1.6578192E-2,1.3755225E6,1.7047843E-2,-1.496793E-2,-8.945751E-3,-1.2205193E-2,-3.3789414E-3,4.8460388E-3,-1.8995742E-3,-4.581377E-3,3.3913078E-3,-6.9818427E-3,2.906258E-3,1.0762618E-2,2.4681897E-3],"split_indices":[52,28,42,52,0,45,57,52,29,28,3,38,27,48,31,0,29,52,2,0,0,0,0,0,0,0,3,0,31,56,28,0,0,39,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.2E2,1.34E2,8.6E1,1.28E2,6E0,6.2E1,2.4E1,6.2E1,6.6E1,5E1,1.2E1,1.4E1,1E1,2.5E1,3.7E1,5.5E1,1.1E1,3.2E1,1.8E1,9E0,3E0,3E0,1.1E1,6E0,4E0,3E0,2.2E1,4E0,3.3E1,2.9E1,2.6E1,3E0,8E0,2.8E1,4E0,1.1E1,7E0,1.2E1,1E1,1.4E1,1.9E1,2.3E1,6E0,2E1,6E0,4E0,2.4E1,8E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.3341741E-2,-1.7014104E-1,1.6686265E-1,-3.492495E-1,-1.3114434E-1,2.5062627E-1,1.2936709E-2,-2.0166756E-1,-2.0348776E-2,-1.9719896E-1,-4.793483E-2,2.0039032E-1,6.454162E-1,-2.4376918E-1,7.048524E-2,-1.2408572E-2,-1.5301732E-3,-2.1884671E-1,1.56006E-3,-3.712567E-3,-2.0438536E-1,5.020855E-2,2.5386077E-1,3.2763653E-2,1.3958895E-2,-4.3612346E-3,-1.3285545E-2,1.4351502E-1,-5.7104815E-2,-1.0387484E-1,-2.5118995E-1,7.333009E-2,-6.38454E-2,-9.319741E-4,-1.115923E-2,8.569152E-3,-1.4016532E-2,7.544073E-2,2.8557697E-1,2.49849E-1,3.3800934E-2,-5.0429786E-3,4.2600255E-3,-8.84934E-4,-8.698785E-3,-1.1033951E-2,-1.3620659E-3,7.3464783E-3,2.6466616E-4,-4.8270784E-3,2.0611102E-3,5.2592426E-4,-3.1992083E-3,5.972178E-3,2.1540573E-4,1.272429E-2,3.4382232E-3,2.0381298E-3,1.3600175E-2,-3.108034E-3,5.6714476E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,-1,31,33,35,37,-1,-1,-1,-1,39,41,43,45,47,49,-1,-1,-1,51,53,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5341864E0,8.761699E-1,1.3118837E0,3.8505697E-1,5.951482E-1,1.1796188E0,5.637849E-1,1.9934303E-1,0E0,3.1845784E-1,3.45171E-1,4.6907115E-1,4.8811197E-2,4.109001E-2,2.9704946E-1,0E0,0E0,1.8214726E-1,0E0,1.8999352E-1,1.2470859E-1,1.742586E-1,2.2744322E-1,0E0,0E0,0E0,0E0,2.2488162E-1,1.37048E-1,1.1837977E-1,1.3712645E-1,1.2447955E-1,1.4397435E-1,0E0,0E0,0E0,2.2987152E-2,3.7978698E-2,1.3940167E-1,1.4210582E-1,1.3408582E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,20,20,21,21,22,22,27,27,28,28,29,29,30,30,31,31,32,32,36,36,37,37,38,38,39,39,40,40],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,-1,32,34,36,38,-1,-1,-1,-1,40,42,44,46,48,50,-1,-1,-1,52,54,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.869239E2,6.5214685E2,6.627576E7,1.2347529E6,3.41043E2,1.02089375E4,7.056125E7,8.351808E-1,-2.0348776E-2,7.06836E7,2.325945E9,1E0,1.4364E4,2.5415527E5,1.739E3,-1.2408572E-2,-1.5301732E-3,1.814433E1,1.56006E-3,7.47E2,3.5753168E4,6.985E3,1.0866135E3,3.2763653E-2,1.3958895E-2,-4.3612346E-3,-1.3285545E-2,1E0,8.9E1,1.3448397E8,1.307581E6,6.540508E-1,3.03E4,-9.319741E-4,-1.115923E-2,8.569152E-3,2.9E1,1.4606427E5,6.126689E0,3.0455479E3,7.6126127E0,-5.0429786E-3,4.2600255E-3,-8.84934E-4,-8.698785E-3,-1.1033951E-2,-1.3620659E-3,7.3464783E-3,2.6466616E-4,-4.8270784E-3,2.0611102E-3,5.2592426E-4,-3.1992083E-3,5.972178E-3,2.1540573E-4,1.272429E-2,3.4382232E-3,2.0381298E-3,1.3600175E-2,-3.108034E-3,5.6714476E-3],"split_indices":[52,33,45,28,52,4,7,27,0,32,5,109,10,28,0,0,0,58,0,0,33,9,52,0,0,0,0,8,3,7,9,27,10,0,0,0,8,33,57,4,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.31E2,1.01E2,2.2E1,1.09E2,6.5E1,3.6E1,1.2E1,1E1,6E1,4.9E1,5.9E1,6E0,6E0,3E1,7E0,5E0,5.5E1,5E0,3.9E1,1E1,1.6E1,4.3E1,3E0,3E0,3E0,3E0,1.9E1,1.1E1,1.3E1,4.2E1,1.7E1,2.2E1,3E0,7E0,4E0,1.2E1,7E0,3.6E1,9E0,1E1,8E0,3E0,8E0,5E0,3.9E1,3E0,6E0,1.1E1,1.5E1,7E0,9E0,3E0,3E0,4E0,3.2E1,4E0,3E0,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-3.7735621E-3,-8.607484E-2,2.166904E-1,-2.7328885E-1,-5.6962013E-2,4.0329966E-1,2.285816E-2,-7.610052E-3,-1.5386035E-2,-1.4498587E-2,-1.7248383E-1,3.302702E-1,2.3282995E-2,-6.323617E-2,1.2576854E-1,-1.0099793E-1,7.398111E-2,-3.5660896E-1,-8.086901E-2,5.35031E-3,1.46413725E-2,3.0339774E-2,-5.4509984E-3,1.949446E-1,4.314614E-5,-1.0811796E-2,-1.6677403E-1,1.7448589E-1,3.5107299E-3,-1.8303985E-2,-6.1176056E-3,1.2743416E-1,-1.6431935E-1,6.475496E-3,-3.1723173E-3,1.1221086E-2,3.2944253E-3,2.0661876E-3,-6.487407E-3,-5.226553E-3,-1.1646913E-2,5.1983055E-3,1.2640808E-2,1.9958061E-3,-5.424666E-3,4.820794E-4,9.940407E-3,-8.826897E-3,6.508984E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,21,23,25,27,29,31,-1,-1,33,-1,35,-1,37,39,41,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9378562E0,8.4558487E-1,2.1248636E0,1.1662972E-1,6.7445153E-1,1.6365767E-1,2.7375314E-1,0E0,0E0,7.881223E-1,5.9529924E-1,6.3622475E-2,0E0,1.1302624E-1,1.2058896E-1,3.0666572E-1,3.6060056E-1,1.4964783E-1,4.728971E-1,0E0,0E0,1.2005999E-1,0E0,5.759284E-2,0E0,2.1188796E-1,1.0363126E-1,9.600395E-2,1.9305907E-1,0E0,0E0,1.0174161E-1,1.7120522E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,21,21,23,23,25,25,26,26,27,27,28,28,31,31,32,32],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,22,24,26,28,30,32,-1,-1,34,-1,36,-1,38,40,42,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5699575E3,1.142E3,5.9283892E7,5.1510445E6,3.8293724E7,1.0946131E2,9.5891304E1,-7.610052E-3,-1.5386035E-2,4.561433E2,6.9080975E2,2.7898628E5,2.3282995E-2,6.232522E1,6.543368E6,5.1E2,1.05163E5,1.2067E4,1.4834309E3,5.35031E-3,1.46413725E-2,5.9241706E-1,-5.4509984E-3,6E1,4.314614E-5,7.6228375E0,9.901698E-1,3.654E3,1.0476415E1,-1.8303985E-2,-6.1176056E-3,1.1003439E3,7.3376025E6,6.475496E-3,-3.1723173E-3,1.1221086E-2,3.2944253E-3,2.0661876E-3,-6.487407E-3,-5.226553E-3,-1.1646913E-2,5.1983055E-3,1.2640808E-2,1.9958061E-3,-5.424666E-3,4.820794E-4,9.940407E-3,-8.826897E-3,6.508984E-4],"split_indices":[52,2,45,45,45,56,56,0,0,52,52,28,0,56,29,0,29,9,4,0,0,57,0,3,0,53,57,2,54,0,0,33,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.57E2,5.8E1,2E1,1.37E2,2.9E1,2.9E1,1.2E1,8E0,1.01E2,3.6E1,2.2E1,7E0,1.6E1,1.3E1,5.1E1,5E1,1.1E1,2.5E1,3E0,1.9E1,7E0,9E0,8E0,5E0,2.2E1,2.9E1,2E1,3E1,7E0,4E0,7E0,1.8E1,3E0,4E0,4E0,4E0,1.6E1,6E0,2.3E1,6E0,1.6E1,4E0,2.3E1,7E0,4E0,3E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[5.652759E-3,-1.1231643E-1,2.220944E-1,-4.080663E-1,-9.143333E-2,3.3261192E-1,8.391902E-3,-9.369005E-3,-2.0326275E-2,-6.44327E-2,-2.699411E-1,3.589703E-2,2.6312912E-1,-4.4202954E-2,1.1591169E-2,-1.3055745E-1,1.1420416E-2,-4.060641E-1,-3.2013975E-2,1.3205391E-1,3.3062303E-1,-1.0522992E-1,7.311882E-3,-7.1627885E-2,-1.9645639E-1,-1.4376114E-1,3.755652E-2,-6.3191773E-3,-1.9588068E-2,-5.284102E-3,3.9565167E-3,1.3542398E-2,3.8911987E-2,3.327991E-3,3.6235526E-1,1.8389575E-3,-1.3264719E-1,-7.871921E-3,-6.7150756E-4,-5.082379E-3,-1.1677824E-2,-4.331782E-4,-1.0271229E-2,-7.181908E-4,4.328564E-3,3.5875784E-3,-8.8078564E-4,8.023326E-3,1.6841134E-2,-3.5735166E-3,-7.432497E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,21,-1,23,25,27,29,31,33,35,-1,37,39,41,43,-1,-1,-1,-1,-1,45,-1,47,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1529717E0,9.26172E-1,2.0047278E0,3.997326E-2,6.9112074E-1,1.8540225E0,4.4126785E-1,0E0,0E0,6.4852524E-1,6.023599E-1,0E0,4.0752554E-1,3.6920026E-1,0E0,2.5388253E-1,2.5280553E-1,1.327213E-1,1.09978974E-1,3.3292216E-1,2.3281908E-1,9.207998E-2,0E0,2.4670963E-1,1.6748226E-1,1.228032E-1,1.9707239E-1,0E0,0E0,0E0,0E0,0E0,4.214768E-2,0E0,1.3964772E-1,0E0,2.3637444E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,32,32,34,34,36,36],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,22,-1,24,26,28,30,32,34,36,-1,38,40,42,44,-1,-1,-1,-1,-1,46,-1,48,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,5.178571E0,6.627576E7,3.804992E5,1E0,5.0655737E0,8.129243E5,-9.369005E-3,-2.0326275E-2,3.7812952E2,2.4875E4,3.589703E-2,1.8366416E3,1.4975258E4,1.1591169E-2,1.4338E4,3.8396814E-5,8.3949E4,5.739E3,4.1367622E6,3.0903362E5,6.971004E7,7.311882E-3,1.292E3,7.739782E0,1.2187347E3,5.2259412E5,-6.3191773E-3,-1.9588068E-2,-5.284102E-3,3.9565167E-3,1.3542398E-2,2.1845242E3,3.327991E-3,8.119423E8,1.8389575E-3,1.0129378E12,-7.871921E-3,-6.7150756E-4,-5.082379E-3,-1.1677824E-2,-4.331782E-4,-1.0271229E-2,-7.181908E-4,4.328564E-3,3.5875784E-3,-8.8078564E-4,8.023326E-3,1.6841134E-2,-3.5735166E-3,-7.432497E-3],"split_indices":[52,54,45,28,89,54,28,0,0,52,9,0,52,4,0,9,38,12,2,45,28,45,0,2,53,4,28,0,0,0,0,0,4,0,7,0,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.39E2,1.55E2,8.4E1,9E0,1.46E2,5.5E1,2.9E1,4E0,5E0,1.28E2,1.8E1,5E0,5E1,2.5E1,4E0,6.8E1,6E1,1.1E1,7E0,1.8E1,3.2E1,2E1,5E0,3.7E1,3.1E1,8E0,5.2E1,3E0,8E0,4E0,3E0,5E0,1.3E1,4E0,2.8E1,3E0,1.7E1,1.1E1,2.6E1,1.8E1,1.3E1,4E0,4E0,2.9E1,2.3E1,7E0,6E0,7E0,2.1E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-6.164037E-3,-1.2189265E-1,1.4899972E-1,-1.0062367E-1,-1.793927E-2,2.52898E-1,-3.407512E-2,-1.6518994E-1,-3.6509924E-2,6.9129294E-1,1.8453333E-1,-2.1478888E-1,2.5228422E-2,-2.2467922E-1,-1.23270646E-1,3.137578E-2,-2.2204629E-1,1.4094607E-2,3.4478486E-2,1.2261085E-1,2.7184382E-1,-3.3890342E-3,-1.1068236E-2,8.590712E-3,-2.346277E-2,-2.4110875E-1,-3.4025742E-3,-6.486591E-2,-2.0850019E-1,1.337033E-1,-2.0768479E-2,-2.8184906E-3,-2.8049824E-1,2.002334E-1,-2.580894E-3,2.1576213E-3,3.0378577E-1,-9.736188E-2,6.5920935E-3,-1.0722972E-2,-4.1091866E-3,-5.0826655E-3,1.1841396E-3,-1.9890403E-3,-1.0118136E-2,3.2494848E-3,9.8999115E-3,-7.385176E-3,3.3693132E-4,-1.48629965E-2,-5.866289E-3,1.0330554E-2,1.4421879E-3,1.570448E-3,-4.692401E-3,8.25837E-3,1.4949011E-2,-1.5418101E-3,-5.243195E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,33,35,-1,-1,-1,37,39,-1,41,43,45,47,-1,49,51,53,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0764184E0,8.184085E-1,1.8536704E0,5.0313795E-1,0E0,1.7306185E0,3.912296E-1,1.2442219E-1,8.004686E-1,1.148026E-1,2.640257E-1,4.11869E-2,2.5360936E-1,3.6327124E-2,1.7563224E-1,2.5355908E-1,1.3382775E-1,0E0,0E0,3.3091867E-1,1.3853252E-1,0E0,0E0,0E0,3.2625267E-1,2.8168082E-2,0E0,1.3366468E-1,7.509798E-2,7.439792E-2,1.4742059E-1,0E0,8.019161E-2,1.6319191E-1,6.7230746E-2,0E0,4.1820765E-2,2.5296435E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,24,24,25,25,27,27,28,28,29,29,30,30,32,32,33,33,34,34,36,36,37,37],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,34,36,-1,-1,-1,38,40,-1,42,44,46,48,-1,50,52,54,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.013037E6,6.627576E7,3.2420947E2,-1.793927E-2,6.677E3,1.2721619E6,1.7398648E2,4.91101E5,3.559158E6,2.077646E3,4.1E1,1E0,9.4100406E5,7.7524557E0,8.9517656E5,1.1181545E3,1.4094607E-2,3.4478486E-2,4.24975E5,3.0903362E5,-3.3890342E-3,-1.1068236E-2,8.590712E-3,1.0377E4,9.030266E-1,-3.4025742E-3,2.8490282E8,5.4E1,8.909818E10,1.7196646E8,-2.8184906E-3,1.5441E4,1.8905495E6,1.4456249E7,2.1576213E-3,2.2764034E5,9.32784E5,6.5920935E-3,-1.0722972E-2,-4.1091866E-3,-5.0826655E-3,1.1841396E-3,-1.9890403E-3,-1.0118136E-2,3.2494848E-3,9.8999115E-3,-7.385176E-3,3.3693132E-4,-1.48629965E-2,-5.866289E-3,1.0330554E-2,1.4421879E-3,1.570448E-3,-4.692401E-3,8.25837E-3,1.4949011E-2,-1.5418101E-3,-5.243195E-3],"split_indices":[52,28,45,52,0,9,32,52,29,1,52,3,67,28,53,48,4,0,0,29,28,0,0,0,9,27,0,5,0,31,7,0,9,47,32,0,33,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.25E2,1.29E2,9.6E1,1.22E2,7E0,6.1E1,3.5E1,6E1,6.2E1,7E0,5.4E1,8E0,2.7E1,2.3E1,3.7E1,4.6E1,1.6E1,3E0,4E0,3.3E1,2.1E1,3E0,5E0,5E0,2.2E1,2E1,3E0,2.3E1,1.4E1,1.5E1,3.1E1,5E0,1.1E1,2E1,1.3E1,3E0,1.8E1,1.6E1,6E0,1.7E1,3E0,1.4E1,9E0,3E0,1.1E1,1.1E1,4E0,4E0,2.7E1,6E0,5E0,1.5E1,5E0,1E1,3E0,8E0,1E1,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-5.418824E-3,-1.2467135E-1,1.4608607E-1,-1.0577735E-1,-2.0189771E-2,5.0469935E-2,3.007632E-1,-1.8854174E-1,-4.954448E-2,-5.095566E-2,1.5681106E-1,1.4604185E-3,3.3095527E-1,-2.4402238E-1,-8.809744E-2,-5.354239E-3,-2.0939946E-1,-1.0776761E-1,1.1251706E-1,-2.4093537E-2,2.441034E-1,4.0832695E-1,2.0572935E-1,-2.755938E-1,1.5712284E-3,-1.4364408E-1,-1.0418488E-2,5.009713E-2,-1.08080395E-1,-1.1224315E-3,-2.4937356E-1,-1.1868691E-2,-1.7090899E-1,8.640888E-3,1.2210602E-3,3.7470215E-3,-1.02562055E-1,1.0952275E-1,3.035707E-1,8.380788E-3,1.8494816E-2,-1.9450836E-4,2.5872904E-1,-5.5917124E-3,-1.273035E-2,-1.5799075E-3,-7.3803663E-3,3.8129776E-3,-4.588735E-3,-9.689423E-4,5.6743375E-3,-9.430174E-3,8.9480815E-4,-3.7316722E-3,-1.2827314E-2,2.78654E-3,-2.7619037E-3,-9.841085E-3,-4.1289954E-3,-6.0628806E-3,-1.3617243E-3,7.718133E-3,-5.865233E-4,6.4654085E-3,1.5223852E-2,4.4254684E-3,1.2303407E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,-1,53,55,57,-1,-1,-1,59,61,63,-1,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.101432E0,8.2036364E-1,1.4637835E0,5.5845654E-1,0E0,6.874902E-1,2.9287648E-1,2.5156534E-1,5.216171E-1,3.1860346E-1,4.954204E-1,0E0,2.4541926E-1,2.8647065E-1,8.048475E-2,3.414966E-1,1.0980797E-1,1.4935124E-1,6.5820925E-2,1.0827201E-1,1.3481832E-1,6.893945E-2,1.6850859E-1,8.4812164E-2,0E0,3.3517823E-2,1.0303074E-1,2.5288373E-1,3.314944E-1,0E0,9.765577E-2,5.2343395E-2,4.769385E-2,0E0,0E0,0E0,1.6626962E-2,7.8901574E-2,7.537198E-2,0E0,0E0,0E0,3.8644016E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,30,30,31,31,32,32,36,36,37,37,38,38,42,42],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,-1,54,56,58,-1,-1,-1,60,62,64,-1,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.0978662E6,8.1350375E5,2.8070276E2,-2.0189771E-2,1E0,7.340097E7,2.037744E10,4.91101E5,1.0946131E2,7.278827E1,1.4604185E-3,1.2728E4,5.841047E4,2.576E3,9.34789E0,2.0173512E-1,5.1160636E11,1.181E3,3.1749196E7,4.0827688E5,9.717001E5,3.108307E6,2.8434807E3,1.5712284E-3,3.1E1,7.629E3,1.4202555E1,5.8392155E-1,-1.1224315E-3,1.9561611E1,1.9026287E9,5.535E3,8.640888E-3,1.2210602E-3,3.7470215E-3,7.056125E7,1.5E1,1.5677256E5,8.380788E-3,1.8494816E-2,-1.9450836E-4,6.3494425E0,-5.5917124E-3,-1.273035E-2,-1.5799075E-3,-7.3803663E-3,3.8129776E-3,-4.588735E-3,-9.689423E-4,5.6743375E-3,-9.430174E-3,8.9480815E-4,-3.7316722E-3,-1.2827314E-2,2.78654E-3,-2.7619037E-3,-9.841085E-3,-4.1289954E-3,-6.0628806E-3,-1.3617243E-3,7.718133E-3,-5.865233E-4,6.4654085E-3,1.5223852E-2,4.4254684E-3,1.2303407E-2],"split_indices":[52,28,28,52,0,109,7,31,29,56,58,0,9,33,10,54,27,31,0,45,28,28,1,47,0,3,10,56,27,0,56,12,9,0,0,0,7,8,33,0,0,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.25E2,1.26E2,9.9E1,1.21E2,5E0,6.2E1,3.7E1,4.8E1,7.3E1,3.2E1,3E1,4E0,3.3E1,3E1,1.8E1,5.8E1,1.5E1,2.4E1,8E0,1E1,2E1,1.9E1,1.4E1,2.7E1,3E0,1E1,8E0,3.8E1,2E1,3E0,1.2E1,1E1,1.4E1,3E0,5E0,4E0,6E0,7E0,1.3E1,4E0,1.5E1,3E0,1.1E1,6E0,2.1E1,3E0,7E0,4E0,4E0,2.1E1,1.7E1,1E1,1E1,4E0,8E0,4E0,6E0,6E0,8E0,3E0,3E0,4E0,3E0,5E0,8E0,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-2.278424E-4,-1.1472517E-1,1.5243772E-1,-6.843444E-2,-2.2172137E-1,2.0412943E-1,-9.066224E-2,-1.0816341E-1,4.2016275E-2,-1.7017871E-1,-4.1830721E-1,4.5188108E-1,1.4674422E-1,-1.7024067E-1,4.734132E-2,-1.948947E-1,-8.0533355E-2,1.19330026E-1,-1.0009886E-1,-4.9945943E-2,-2.6625592E-1,-2.1549437E-2,-8.732386E-3,2.343658E-1,3.294987E-2,4.205396E-2,1.8679577E-1,-9.141564E-3,-1.08308E-3,4.706314E-3,-1.279212E-3,-1.0015745E-2,-4.398561E-3,-1.3333501E-1,-2.4634259E-2,7.5073163E-3,1.4267964E-3,-5.9913676E-3,-3.2960973E-4,-9.167787E-2,3.629343E-3,-1.3357606E-2,-2.7180687E-3,-1.7483892E-3,1.4848921E-2,-3.0078242E-2,7.0325527E-3,8.896106E-2,2.1870916E-1,-6.317784E-3,-8.828731E-4,1.6912807E-3,-6.3264337E-3,-6.261644E-4,-8.099875E-3,-2.898396E-3,3.2022393E-3,7.976322E-3,-2.2317074E-6,9.816801E-3,3.8197571E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,-1,45,47,-1,-1,-1,-1,-1,-1,49,51,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,55,-1,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.793125E0,5.989901E-1,1.1871684E0,3.9069685E-1,3.037176E-1,1.0284197E0,1.947552E-1,1.4006984E-1,2.756437E-1,3.4480214E-1,3.889501E-2,8.6286545E-1,2.6049685E-1,7.4632764E-2,3.991227E-2,3.699404E-2,1.4724517E-1,7.810782E-2,3.5941184E-2,9.422965E-2,1.7193127E-1,0E0,0E0,3.5983926E-1,0E0,1.8047334E-1,1.2416625E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.3206325E-2,2.248163E-1,0E0,0E0,0E0,0E0,9.267831E-2,0E0,0E0,0E0,0E0,0E0,6.1301872E-2,0E0,1.201965E-1,5.801916E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,25,25,26,26,33,33,34,34,39,39,45,45,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,-1,46,48,-1,-1,-1,-1,-1,-1,50,52,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,56,-1,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,1.996866E1,9.996903E7,4.561433E2,6.8652206E9,6.677E3,2.68442E9,1.053E3,2.144E5,1.412805E-1,1.1992621E3,9.467578E6,1E0,6.139E3,1.739E3,1.0437426E8,6.741443E-1,3.302999E8,2.9836262E8,1.1859864E8,9.50634E5,-2.1549437E-2,-8.732386E-3,9.0853137E-1,3.294987E-2,8.1350375E5,2.262586E6,-9.141564E-3,-1.08308E-3,4.706314E-3,-1.279212E-3,-1.0015745E-2,-4.398561E-3,3.366E3,2.414E3,7.5073163E-3,1.4267964E-3,-5.9913676E-3,-3.2960973E-4,1.3083E4,3.629343E-3,-1.3357606E-2,-2.7180687E-3,-1.7483892E-3,1.4848921E-2,5.1737704E7,7.0325527E-3,8.81832E5,2.616253E6,-6.317784E-3,-8.828731E-4,1.6912807E-3,-6.3264337E-3,-6.261644E-4,-8.099875E-3,-2.898396E-3,3.2022393E-3,7.976322E-3,-2.2317074E-6,9.816801E-3,3.8197571E-3],"split_indices":[52,56,45,52,5,9,7,2,29,41,4,1,109,9,0,7,27,7,7,12,1,0,0,27,0,28,1,0,0,0,0,0,0,2,2,0,0,0,0,9,0,0,0,0,0,32,0,1,29,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.23E2,9.2E1,8.7E1,3.6E1,7.6E1,1.6E1,6.4E1,2.3E1,3E1,6E0,1.3E1,6.3E1,1E1,6E0,1.4E1,5E1,1.5E1,8E0,1.4E1,1.6E1,3E0,3E0,9E0,4E0,1.8E1,4.5E1,7E0,3E0,3E0,3E0,8E0,6E0,2.5E1,2.5E1,8E0,7E0,5E0,3E0,1.1E1,3E0,1.2E1,4E0,3E0,6E0,1.2E1,6E0,1.2E1,3.3E1,2.1E1,4E0,1.7E1,8E0,7E0,4E0,9E0,3E0,5E0,7E0,2.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-9.971588E-3,-1.0775053E-1,1.4774236E-1,-3.2231566E-1,-8.8317715E-2,2.159064E-1,-2.4487544E-2,-7.4487044E-3,-1.8904561E-2,3.222781E-2,-1.21187285E-1,1.5505923E-1,3.2442197E-1,-8.6691365E-2,1.18532E-1,-1.2268896E-1,1.0032897E-1,-3.1635517E-1,-8.8388E-2,-1.264292E-2,1.9609742E-1,4.873943E-3,1.6513769E-2,-1.4372414E-1,2.1232672E-3,7.503265E-3,4.2742238E-4,1.1185043E-5,-7.620273E-3,4.8903357E-2,8.776789E-3,-7.2058607E-3,-1.7937731E-2,-2.1457622E-2,-1.5330957E-1,-3.3837322E-3,1.4713317E-3,1.5961364E-1,1.380284E-2,-3.5693075E-3,-8.122541E-3,-8.7174773E-4,4.6647973E-3,-3.6783216E-3,1.8203265E-3,-1.3561438E-2,-4.09795E-3,8.427846E-3,9.861974E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,-1,-1,-1,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.423983E0,5.427697E-1,1.0031774E0,1.2845886E-1,5.0711524E-1,3.5246992E-1,2.3268475E-1,0E0,0E0,3.0770954E-1,6.121744E-1,2.8464913E-1,2.774217E-1,1.4906658E-1,5.2531205E-2,6.813519E-2,1.04655504E-1,1.5786707E-1,3.7474495E-1,3.291622E-2,1.1237645E-1,0E0,0E0,2.2357821E-2,0E0,0E0,0E0,0E0,0E0,6.941195E-2,0E0,0E0,0E0,1.982291E-1,3.8408065E-1,0E0,0E0,1.6288084E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,29,29,33,33,34,34,37,37],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,-1,-1,-1,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,5.178571E0,6.627576E7,9.372475E-1,8.637E3,1.1392155E6,9.5891304E1,-7.4487044E-3,-1.8904561E-2,5.3153326E2,8.27725E3,1E0,3.8657358E6,8.129243E5,5.0415534E9,7.110474E7,9.400184E-1,8.3949E4,8.513227E0,4.1808826E-1,2.4864366E9,4.873943E-3,1.6513769E-2,4.6E1,2.1232672E-3,7.503265E-3,4.2742238E-4,1.1185043E-5,-7.620273E-3,1.4267187E8,8.776789E-3,-7.2058607E-3,-1.7937731E-2,2.6960093E10,2.1424272E6,-3.3837322E-3,1.4713317E-3,3.895276E7,1.380284E-2,-3.5693075E-3,-8.122541E-3,-8.7174773E-4,4.6647973E-3,-3.6783216E-3,1.8203265E-3,-1.3561438E-2,-4.09795E-3,8.427846E-3,9.861974E-4],"split_indices":[52,54,45,27,9,28,56,0,0,4,48,109,32,28,7,5,27,12,54,27,7,0,0,3,0,0,0,0,0,7,0,0,0,31,45,0,0,45,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.2E2,1.36E2,8.4E1,1E1,1.26E2,6E1,2.4E1,6E0,4E0,2.7E1,9.9E1,4E1,2E1,1.7E1,7E0,8E0,1.9E1,1.3E1,8.6E1,8E0,3.2E1,6E0,1.4E1,1.2E1,5E0,4E0,3E0,3E0,5E0,1.4E1,5E0,7E0,6E0,4.3E1,4.3E1,3E0,5E0,2.7E1,5E0,7E0,5E0,7E0,7E0,2.1E1,2.2E1,9E0,3.4E1,2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.3090101E-3,-1.1154435E-1,1.5403748E-1,-9.462563E-2,-2.0488191E-2,1.0158473E-1,4.025352E-1,-1.5411146E-1,-1.9066796E-2,1.4409514E-1,-3.8675178E-2,2.5178146E-1,2.4269808E-2,-2.2684596E-1,-1.13511354E-1,5.5894017E-2,-8.561557E-2,9.901282E-2,3.4608078E-1,-1.01604104E-1,6.00893E-3,4.582966E-3,1.2260182E-2,-1.6390938E-1,-1.2090288E-2,-2.6130727E-2,-1.8237063E-1,1.1644083E-1,-2.4447331E-2,-1.3630849E-2,-2.0216957E-1,1.243908E-1,-2.7880913E-3,1.9064682E-2,5.7546734E-3,-6.9707558E-3,-3.035903E-2,-2.2126222E-3,-7.824953E-3,1.1716362E-3,-8.523495E-3,-2.1617839E-3,-1.0436434E-2,2.7057898E-4,6.445302E-3,-4.142984E-3,2.8108805E-3,3.1285612E-3,-2.9784862E-3,-2.0603093E-3,-1.1670277E-2,9.344758E-3,2.2021055E-3,1.7520732E-3,-3.4110125E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,-1,-1,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.630488E0,7.581909E-1,1.1061347E0,5.364232E-1,0E0,4.4679838E-1,2.6873732E-1,1.7210329E-1,2.741671E-1,4.847747E-1,2.2310726E-1,3.0044138E-2,0E0,4.6801567E-2,2.65998E-1,1.293214E-1,2.4055734E-1,2.1109825E-1,1.7346656E-1,6.1620295E-2,0E0,0E0,0E0,2.8919429E-2,0E0,2.1157804E-1,2.1425462E-1,6.3880414E-2,9.0746276E-2,1.0444154E-1,1.1994395E-1,2.9134798E-1,0E0,0E0,0E0,0E0,3.3799294E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,-1,-1,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.869239E2,6.8652206E9,1.6049542E6,3.4008475E2,-2.0488191E-2,9.996903E7,7.3765434E1,1.292E3,1E0,7.773148E3,1.4975258E4,7.44905E0,2.4269808E-2,5.1510445E6,1.2117E4,2.2906403E-1,1.7175853E9,1.0369473E7,8.812601E-1,5.535E3,6.00893E-3,4.582966E-3,1.2260182E-2,4.1195285E-3,-1.2090288E-2,1E0,6.5833335E0,6.725E3,1.2534044E3,3.8390288E5,3.3E1,6.6108694E0,-2.7880913E-3,1.9064682E-2,5.7546734E-3,-6.9707558E-3,2.565E3,-2.2126222E-3,-7.824953E-3,1.1716362E-3,-8.523495E-3,-2.1617839E-3,-1.0436434E-2,2.7057898E-4,6.445302E-3,-4.142984E-3,2.8108805E-3,3.1285612E-3,-2.9784862E-3,-2.0603093E-3,-1.1670277E-2,9.344758E-3,2.2021055E-3,1.7520732E-3,-3.4110125E-3],"split_indices":[52,5,28,52,0,45,58,2,8,52,4,54,0,45,9,57,5,47,57,9,0,0,0,41,0,108,53,9,4,47,3,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.23E2,8.7E1,1.19E2,4E0,7.3E1,1.4E1,6.6E1,5.3E1,5.6E1,1.7E1,9E0,5E0,2.2E1,4.4E1,2.5E1,2.8E1,4.7E1,9E0,1.3E1,4E0,3E0,6E0,1.3E1,9E0,2E1,2.4E1,1.4E1,1.1E1,1.8E1,1E1,4.1E1,6E0,5E0,4E0,6E0,7E0,3E0,1E1,1.6E1,4E0,9E0,1.5E1,4E0,1E1,6E0,5E0,7E0,1.1E1,4E0,6E0,1.6E1,2.5E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.0646043E-3,-1.0428354E-1,1.8365698E-1,-8.663862E-2,-3.757724E-1,2.6195666E-1,4.7724497E-2,-3.0080345E-1,-7.188487E-2,-1.8778889E-2,-7.6151537E-3,1.722011E-1,4.6763438E-1,1.4858373E-1,-5.0913417E-3,-6.880043E-3,-1.6286308E-2,1.8667443E-2,-1.2140914E-1,-3.1564727E-3,1.9788307E-1,1.1288855E-2,2.3974933E-2,-1.3267764E-4,8.856367E-3,-9.3422726E-2,1.1324954E-1,-1.0788169E-1,6.411963E-2,-1.507389E-1,1.3780389E-2,2.2869554E-1,1.0857238E-3,1.0037475E-4,-5.1973704E-3,7.6748286E-3,7.576944E-4,-6.454054E-3,-3.8396072E-4,6.133389E-3,3.7120294E-4,-3.2529964E-3,-8.54778E-3,4.9807685E-3,-9.476984E-4,1.0327911E-2,5.506114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,-1,-1,27,29,-1,31,-1,-1,-1,-1,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.042239E0,6.1893773E-1,7.964022E-1,3.8598245E-1,3.6333323E-2,8.0558705E-1,1.5523848E-1,3.4472167E-2,5.4938E-1,0E0,0E0,2.3847413E-1,1.4701581E-1,1.00127354E-1,2.1964435E-1,0E0,0E0,2.6000246E-1,3.153181E-1,0E0,1.6323102E-1,0E0,0E0,0E0,0E0,3.7363224E-2,5.8360003E-2,5.5510506E-2,1.5180732E-1,2.4571097E-1,6.1440144E-2,2.0897627E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,17,17,18,18,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,-1,-1,28,30,-1,32,-1,-1,-1,-1,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,3.0978662E6,5.6759316E7,5.178571E0,4.2993605E6,5.978742E1,3.7E1,3.512733E-1,6.500938E0,-1.8778889E-2,-7.6151537E-3,5.6865673E0,2.4934822E7,9.329E3,2.1312436E5,-6.880043E-3,-1.6286308E-2,2.2487996E2,1.662926E6,-3.1564727E-3,3.2395E4,1.1288855E-2,2.3974933E-2,-1.3267764E-4,8.856367E-3,9.32784E5,1.467E3,6.0127387E0,1.1253E4,1.3083E4,3.931E3,4.24975E5,1.0857238E-3,1.0037475E-4,-5.1973704E-3,7.6748286E-3,7.576944E-4,-6.454054E-3,-3.8396072E-4,6.133389E-3,3.7120294E-4,-3.2529964E-3,-8.54778E-3,4.9807685E-3,-9.476984E-4,1.0327911E-2,5.506114E-3],"split_indices":[52,28,45,54,28,56,3,27,53,0,0,54,45,9,33,0,0,52,1,0,9,0,0,0,0,29,0,53,9,9,2,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.35E2,7.5E1,1.28E2,7E0,4.7E1,2.8E1,7E0,1.21E2,4E0,3E0,3.4E1,1.3E1,9E0,1.9E1,4E0,3E0,4.3E1,7.8E1,3E0,3.1E1,6E0,7E0,3E0,6E0,1.1E1,8E0,1.1E1,3.2E1,6.4E1,1.4E1,2.6E1,5E0,3E0,8E0,4E0,4E0,7E0,4E0,1.2E1,2E1,2.9E1,3.5E1,3E0,1.1E1,2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.4466142E-4,-1.0034551E-1,1.3342483E-1,-8.146963E-2,-3.6858097E-1,1.01107165E-1,4.722411E-1,-1.2135577E-1,1.3861078E-2,-5.766629E-3,-1.9781737E-2,4.237585E-2,2.5595835E-1,2.5623398E-2,7.040664E-3,-9.49885E-2,-2.7164367E-1,6.207619E-2,-1.376816E-1,-2.0854406E-2,1.6691019E-1,5.976857E-3,2.85089E-1,-2.0230108E-1,-6.37473E-2,-1.3688562E-2,-2.911565E-3,1.0209163E-2,1.7043889E-1,-1.1725465E-3,-9.079181E-3,-1.7329372E-1,1.2669081E-2,1.9690457E-1,-7.152547E-4,3.1183675E-1,4.6956143E-3,-1.0148968E-2,-2.4216303E-3,2.3832954E-3,-3.6077357E-3,1.993729E-3,-2.7808372E-3,8.537589E-3,3.0571464E-3,-9.097502E-3,-2.9652277E-3,2.2412187E-3,-3.2400747E-3,9.608728E-3,2.5502552E-4,1.4130402E-2,5.982257E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,33,-1,35,37,39,-1,-1,41,43,-1,-1,45,47,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.959925E0,6.0793245E-1,9.9247944E-1,4.541645E-1,1.343267E-1,7.894794E-1,2.5021267E-1,3.0706227E-1,2.7158603E-1,0E0,0E0,5.153128E-1,3.6743402E-2,0E0,0E0,2.3184186E-1,1.1984247E-1,1.5563262E-1,7.427205E-2,2.2704856E-1,1.249544E-1,0E0,4.720974E-2,8.4640205E-2,1.702059E-1,0E0,0E0,6.207025E-2,1.5981019E-2,0E0,0E0,2.2141278E-2,1.440021E-1,1.2678093E-1,0E0,2.5232673E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,27,27,28,28,31,31,32,32,33,33,35,35],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,34,-1,36,38,40,-1,-1,42,44,-1,-1,46,48,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.869239E2,6.8652206E9,2.9028345E2,4.653861E2,1E0,1.4977078E0,1.703939E4,1.996866E1,1.009673E1,-5.766629E-3,-1.9781737E-2,7.9522475E5,3.8657358E6,2.5623398E-2,7.040664E-3,1.142E3,5.747E3,8.129243E5,1.65E9,2.4146196E-3,1E0,5.976857E-3,2.603E3,1.0437422E8,2.8E2,-1.3688562E-2,-2.911565E-3,7.363373E1,2.5371E4,-1.1725465E-3,-9.079181E-3,8.513227E0,1.2199979E10,5.6688416E-1,-7.152547E-4,4.0370885E-1,4.6956143E-3,-1.0148968E-2,-2.4216303E-3,2.3832954E-3,-3.6077357E-3,1.993729E-3,-2.7808372E-3,8.537589E-3,3.0571464E-3,-9.097502E-3,-2.9652277E-3,2.2412187E-3,-3.2400747E-3,9.608728E-3,2.5502552E-4,1.4130402E-2,5.982257E-3],"split_indices":[52,5,56,52,64,39,4,56,54,0,0,28,32,0,0,2,2,28,5,39,112,0,0,7,0,0,0,58,10,0,0,54,5,42,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.25E2,9.4E1,1.18E2,7E0,8.7E1,7E0,8.3E1,3.5E1,3E0,4E0,6.4E1,2.3E1,4E0,3E0,7.2E1,1.1E1,2.7E1,8E0,4.3E1,2.1E1,6E0,1.7E1,1.5E1,5.7E1,8E0,3E0,1.9E1,8E0,4E0,4E0,7E0,3.6E1,1.8E1,3E0,1.4E1,3E0,1.1E1,4E0,9E0,4.8E1,1.3E1,6E0,5E0,3E0,4E0,3E0,2.5E1,1.1E1,1.5E1,3E0,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.619632E-2,-1.21097565E-1,1.60031E-1,-2.0247827E-1,-8.400708E-2,2.444088E-1,2.768661E-2,-2.5072935E-1,-1.1617226E-1,-2.6067043E-2,-1.6957477E-1,1.7757043E-1,4.8346713E-1,-3.321651E-2,2.0731969E-1,-2.6487967E-1,-4.120412E-3,4.807209E-4,-6.875831E-3,-7.9792194E-2,4.2020366E-2,-2.0659377E-1,3.3424832E-3,2.03318E-1,-2.431306E-3,8.98893E-3,2.3004621E-2,-8.174499E-2,6.2156175E-3,1.0930128E-2,2.901002E-3,-1.6977511E-1,-1.3111702E-2,-3.331049E-2,-1.3954711E-1,-3.4496922E-2,8.6446464E-2,-2.6101355E-2,-2.4926086E-1,2.5764868E-1,1.17851906E-1,-1.3285989E-1,-7.121143E-3,-1.0432672E-2,-2.4428277E-3,-3.5792314E-3,1.9548123E-3,-7.6963836E-3,1.4420018E-3,-2.499128E-3,1.0739608E-3,4.9385065E-3,-2.4274623E-3,1.1808744E-3,-2.6722536E-3,-1.4769694E-2,-5.765288E-3,1.1495331E-2,3.8069626E-3,6.7478595E-3,-2.898253E-3,-6.935494E-3,-1.8055707E-3,-2.6812335E-3,3.9557167E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,33,35,37,-1,39,-1,-1,-1,41,-1,-1,-1,43,-1,45,47,49,51,53,55,57,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3082395E0,4.2368436E-1,9.64756E-1,1.5826833E-1,5.0127465E-1,7.609987E-1,3.89794E-1,3.349173E-2,1.1704953E-1,2.3014928E-1,3.9670944E-1,2.7361858E-1,9.186411E-2,2.5268286E-1,5.173382E-2,7.002473E-2,0E0,0E0,0E0,9.256445E-2,9.816435E-2,2.70733E-1,0E0,1.5072954E-1,0E0,0E0,0E0,8.2970455E-2,0E0,0E0,0E0,8.5496604E-2,0E0,9.412205E-2,1.2815106E-1,1.8804826E-2,9.304941E-2,1.8674865E-2,2.9372513E-1,5.1529408E-2,1.560524E-1,3.3041194E-2,6.547862E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,23,23,27,27,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,34,36,38,-1,40,-1,-1,-1,42,-1,-1,-1,44,-1,46,48,50,52,54,56,58,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,4.8081778E2,5.7025972E7,3.254141E4,1.9284746E1,9.8940155E1,7.2596356E5,2.1034482E-1,2.2010857E1,4.3062625E2,6.034607E6,1.2562557E7,7.65E2,1.4975258E4,9.298568E7,3.7248906E3,-4.120412E-3,4.807209E-4,-6.875831E-3,1.5445154E8,3.0972834E5,4.61E2,3.3424832E-3,6.6108694E0,-2.431306E-3,8.98893E-3,2.3004621E-2,2.137881E5,6.2156175E-3,1.0930128E-2,2.901002E-3,6.0127387E0,-1.3111702E-2,2.13E3,4.305809E7,7.26031E2,1.1710234E3,3.2432194E1,6E0,2.603E3,3.291238E7,7.381975E8,5.7481343E3,-1.0432672E-2,-2.4428277E-3,-3.5792314E-3,1.9548123E-3,-7.6963836E-3,1.4420018E-3,-2.499128E-3,1.0739608E-3,4.9385065E-3,-2.4274623E-3,1.1808744E-3,-2.6722536E-3,-1.4769694E-2,-5.765288E-3,1.1495331E-2,3.8069626E-3,6.7478595E-3,-2.898253E-3,-6.935494E-3,-1.8055707E-3,-2.6812335E-3,3.9557167E-3],"split_indices":[52,4,45,33,56,56,28,57,58,52,48,47,0,4,45,47,0,0,0,7,28,0,0,53,0,0,0,33,0,0,0,53,0,2,32,52,4,56,8,0,45,7,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,1.45E2,8.6E1,4.4E1,1.01E2,5.2E1,3.4E1,2.7E1,1.7E1,6.1E1,4E1,4.2E1,1E1,2.6E1,8E0,2.4E1,3E0,5E0,1.2E1,3.4E1,2.7E1,3.5E1,5E0,3.8E1,4E0,3E0,7E0,2.1E1,5E0,5E0,3E0,1E1,1.4E1,2E1,1.4E1,1E1,1.7E1,7E0,2.8E1,2.2E1,1.6E1,1.2E1,9E0,5E0,5E0,1.2E1,8E0,1.1E1,3E0,7E0,3E0,1.4E1,3E0,3E0,4E0,1.3E1,1.5E1,1.9E1,3E0,1.3E1,3E0,8E0,4E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.2189808E-2,-1.2103275E-1,1.181144E-1,-9.994131E-2,-1.6922742E-2,1.9106759E-1,-2.6997935E-2,-1.5098614E-1,-3.519497E-2,4.6359536E-1,1.4700396E-1,-1.2055884E-1,6.806286E-2,-9.252345E-2,-2.0655747E-1,-9.157013E-2,9.737228E-2,2.738002E-2,1.0512617E-2,9.8038815E-2,2.5580937E-1,-4.7558926E-2,-7.7882516E-3,1.0572099E-2,-3.766179E-3,-1.8041351E-1,-2.6374228E-2,-8.754935E-2,-2.4931937E-1,2.286132E-2,-1.2916815E-1,1.3975376E-1,1.4247381E-4,1.903581E-1,5.1688522E-2,2.8849388E-3,3.0399027E-1,1.1822915E-3,-3.4860887E-3,-3.3136797E-3,6.350143E-2,-9.718854E-3,-3.8188836E-3,3.3640612E-3,-4.3413946E-3,-4.711994E-3,-8.0803316E-4,-4.1563413E-3,-1.179287E-2,-1.9792865E-3,3.2255596E-3,-1.054878E-2,-3.752949E-3,6.97457E-3,1.9489455E-3,9.502724E-3,1.4280697E-3,1.111394E-3,8.534384E-3,1.4057099E-2,5.35551E-3,4.400834E-3,-2.473808E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,33,35,37,-1,-1,39,41,43,45,47,49,51,53,-1,55,57,-1,59,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1770363E0,7.0033884E-1,1.084972E0,3.7475705E-1,0E0,7.484317E-1,3.194947E-1,1.88815E-1,3.975384E-1,2.0560598E-1,2.945813E-1,8.002406E-2,2.4339682E-1,1.870991E-1,1.4102066E-1,1.6214162E-1,6.321165E-2,0E0,0E0,1.7415875E-1,1.4016831E-1,3.036284E-2,0E0,0E0,7.697903E-2,4.5565903E-2,1.7778233E-1,1.5703358E-2,9.378815E-2,4.2701345E-2,1.12321675E-1,2.1456167E-2,0E0,7.794613E-2,1.0959631E-1,0E0,4.3546677E-2,0E0,0E0,0E0,2.4474002E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,36,36,40,40],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,34,36,38,-1,-1,40,42,44,46,48,50,52,54,-1,56,58,-1,60,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.8652206E9,6.627576E7,3.3078247E2,-1.6922742E-2,6.677E3,3.0692886E3,1.4084E4,8.129243E5,4.9695907E0,5.936149E1,4.4E1,1E0,1.295E3,6.5833335E0,5.62E2,6.621114E4,2.738002E-2,1.0512617E-2,1.1740399E5,3.5935396E-1,2.7864855E11,-7.7882516E-3,1.0572099E-2,2.1312436E5,6.2734294E0,1.015214E5,5.124411E2,2.2304833E0,2.3799521E4,8.15E2,3.7E1,1.4247381E-4,1.775894E10,4.9E1,2.8849388E-3,5.9283892E7,1.1822915E-3,-3.4860887E-3,-3.3136797E-3,1.7837838E0,-9.718854E-3,-3.8188836E-3,3.3640612E-3,-4.3413946E-3,-4.711994E-3,-8.0803316E-4,-4.1563413E-3,-1.179287E-2,-1.9792865E-3,3.2255596E-3,-1.054878E-2,-3.752949E-3,6.97457E-3,1.9489455E-3,9.502724E-3,1.4280697E-3,1.111394E-3,8.534384E-3,1.4057099E-2,5.35551E-3,4.400834E-3,-2.473808E-4],"split_indices":[52,5,45,52,0,9,52,9,28,54,56,3,8,2,53,0,33,0,0,33,57,31,0,0,33,54,47,4,56,33,0,3,0,5,3,0,45,0,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.21E2,1.01E2,1.14E2,7E0,6.7E1,3.4E1,6.3E1,5.1E1,8E0,5.9E1,1.7E1,1.7E1,3.2E1,3.1E1,3.6E1,1.5E1,3E0,5E0,4.2E1,1.7E1,9E0,8E0,4E0,1.3E1,1.3E1,1.9E1,9E0,2.2E1,9E0,2.7E1,1E1,5E0,1.3E1,2.9E1,4E0,1.3E1,3E0,6E0,6E0,7E0,7E0,6E0,8E0,1.1E1,6E0,3E0,5E0,1.7E1,4E0,5E0,5E0,2.2E1,7E0,3E0,1E1,3E0,2.6E1,3E0,1E1,3E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.7250547E-2,-8.736647E-2,1.6200902E-1,-6.314989E-2,-3.0604646E-1,8.191456E-2,2.674549E-1,-1.120947E-1,1.3622315E-2,-3.34862E-3,-1.6426416E-2,-1.7943203E-2,1.2968779E-1,3.0538894E-2,1.9660513E-1,-1.2514353E-1,2.3433766E-3,6.99523E-2,-5.064734E-2,-3.8109464E-3,9.5574535E-2,-2.6552968E-3,1.6100828E-1,3.098893E-1,7.297333E-2,-1.4563937E-1,1.0345934E-2,9.490535E-2,-2.2268218E-3,-1.06346905E-2,-7.965775E-3,5.166747E-4,7.2613214E-3,2.180467E-1,1.0136258E-1,1.6116448E-2,2.1649839E-1,-4.2251937E-2,8.5089225E-3,-7.594391E-3,-2.505266E-3,4.9912315E-3,-2.2407395E-3,1.8075621E-3,7.4070473E-3,1.3415067E-3,-3.0570694E-3,1.03309285E-2,2.503781E-3,2.4151716E-3,7.3461686E-3,4.59737E-3,1.07895285E-2,1.5307172E-3,-3.8399342E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,-1,27,29,-1,31,-1,33,35,37,39,41,43,-1,45,-1,-1,-1,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4670317E0,6.8499887E-1,7.8887177E-1,4.5619678E-1,2.3428571E-1,2.694766E-1,1.2224467E0,1.700306E-1,1.7731772E-1,0E0,0E0,1.6880608E-1,2.410838E-1,0E0,4.955908E-1,1.9311774E-1,0E0,8.5417286E-2,1.2855852E-1,0E0,5.095613E-2,0E0,9.01674E-2,5.7095528E-2,2.992583E-1,1.8259406E-1,7.770304E-2,8.698337E-2,0E0,5.4163616E-2,0E0,0E0,0E0,6.476498E-2,4.859923E-2,0E0,2.2388637E-2,4.7527127E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,17,17,18,18,20,20,22,22,23,23,24,24,25,25,26,26,27,27,29,29,33,33,34,34,36,36,37,37],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,-1,28,30,-1,32,-1,34,36,38,40,42,44,-1,46,-1,-1,-1,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,4.56567E7,3.4573945E3,3.7812952E2,9.180692E7,1E0,1.4164306E-3,7.06836E7,1.9887656E8,-3.34862E-3,-1.6426416E-2,5.324658E5,7.340097E7,3.0538894E-2,2.4352698E2,1E0,2.3433766E-3,8.542832E0,9.604621E0,-3.8109464E-3,9.3405694E-1,-2.6552968E-3,1.3766816E6,3.9247762E6,1.3978E4,2.495E3,2.8E1,5.0798903E5,-2.2268218E-3,7.398674E1,-7.965775E-3,5.166747E-4,7.2613214E-3,7.8582473E0,7.5812045E8,1.6116448E-2,1E0,8.0060637E-1,8.5089225E-3,-7.594391E-3,-2.505266E-3,4.9912315E-3,-2.2407395E-3,1.8075621E-3,7.4070473E-3,1.3415067E-3,-3.0570694E-3,1.03309285E-2,2.503781E-3,2.4151716E-3,7.3461686E-3,4.59737E-3,1.07895285E-2,1.5307172E-3,-3.8399342E-3],"split_indices":[52,45,52,52,7,109,57,32,12,0,0,28,7,0,58,112,0,53,54,0,27,0,47,47,9,2,3,28,0,58,0,0,0,53,7,0,64,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.27E2,1.32E2,9.5E1,1.2E2,1.2E1,5.5E1,4E1,7.3E1,4.7E1,4E0,8E0,1.8E1,3.7E1,4E0,3.6E1,6.8E1,5E0,2.5E1,2.2E1,1.1E1,7E0,5E0,3.2E1,1.8E1,1.8E1,5.9E1,9E0,2.1E1,4E0,1.8E1,4E0,4E0,3E0,1.5E1,1.7E1,8E0,1E1,1E1,8E0,4E1,1.9E1,3E0,6E0,1.4E1,7E0,1.1E1,7E0,1.2E1,3E0,1.2E1,5E0,4E0,6E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.7918464E-2,-9.6864514E-2,9.182216E-2,-8.310484E-2,-3.3856347E-1,9.078988E-3,1.9744532E-1,-1.559874E-1,-1.6411755E-2,-1.775069E-2,-6.758374E-3,-1.5715651E-2,1.3871763E-2,1.3840581E-1,3.2304373E-1,-1.7491092E-1,1.7684717E-2,1.4617068E-2,-1.2909E-1,-6.818595E-2,1.3532203E-1,1.6683257E-1,-3.24496E-4,1.5258407E-2,5.2843303E-3,-1.887925E-1,-1.1485704E-3,3.8243148E-3,-2.5440943E-3,7.428971E-2,-4.1444253E-2,-9.299879E-3,-1.7226957E-2,-8.452681E-3,-3.8149495E-2,6.7209722E-3,2.231887E-3,2.1075337E-1,4.1666612E-2,-9.376055E-3,-5.1255273E-3,6.520926E-5,7.4422956E-3,2.27908E-3,-2.8552718E-3,2.6568468E-3,-4.7462955E-3,5.398376E-4,-2.9149274E-3,6.30302E-3,1.4219818E-2,-1.3578255E-3,5.1444843E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,29,31,33,35,37,-1,-1,-1,39,-1,-1,-1,41,43,-1,45,-1,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0275106E0,4.241587E-1,8.546877E-1,6.293391E-1,4.0927768E-2,4.5828092E-1,2.7414536E-1,2.0685565E-1,2.4312976E-1,0E0,0E0,4.2925167E-1,0E0,1.301887E-1,6.920087E-2,1.0914397E-1,4.7128126E-2,1.8707186E-1,1.5675142E-1,1.5578383E-1,2.2163957E-2,1.3415074E-1,0E0,0E0,0E0,9.7222924E-2,0E0,0E0,0E0,2.0707892E-1,8.101548E-2,0E0,7.0910394E-2,0E0,5.717561E-2,0E0,0E0,9.515518E-2,5.3363524E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,29,29,30,30,32,32,34,34,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,30,32,34,36,38,-1,-1,-1,40,-1,-1,-1,42,44,-1,46,-1,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.8652206E9,7.2596356E5,2.8070276E2,1.1992621E3,2.6014774E2,1.9894526E6,5.0558603E5,2.6519718E1,-1.775069E-2,-6.758374E-3,1.5585917E0,1.3871763E-2,9.298568E7,9E0,1E0,4.35E2,3.7568363E4,3.9761555E1,3.4488773E-4,1.851E4,3.072278E-1,-3.24496E-4,1.5258407E-2,5.2843303E-3,2.4830703E4,-1.1485704E-3,3.8243148E-3,-2.5440943E-3,4.249921E2,4.0079948E-1,-9.299879E-3,7.34E2,-8.452681E-3,3.170081E6,6.7209722E-3,2.231887E-3,1.5677256E5,3.108307E6,-9.376055E-3,-5.1255273E-3,6.520926E-5,7.4422956E-3,2.27908E-3,-2.8552718E-3,2.6568468E-3,-4.7462955E-3,5.398376E-4,-2.9149274E-3,6.30302E-3,1.4219818E-2,-1.3578255E-3,5.1444843E-3],"split_indices":[52,5,28,52,4,56,28,47,56,0,0,41,0,45,8,68,0,33,56,38,9,38,0,0,0,33,0,0,0,52,27,0,0,0,1,0,0,33,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.35E2,9.7E1,1.29E2,6E0,5.5E1,4.2E1,6.1E1,6.8E1,3E0,3E0,5.2E1,3E0,3E1,1.2E1,5.5E1,6E0,5.4E1,1.4E1,3.9E1,1.3E1,2.5E1,5E0,9E0,3E0,5E1,5E0,3E0,3E0,2.6E1,2.8E1,7E0,7E0,6E0,3.3E1,9E0,4E0,1.8E1,7E0,3E1,2E1,1.6E1,1E1,6E0,2.2E1,4E0,3E0,1.3E1,2E1,1.4E1,4E0,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.4046455E-2,-9.391392E-2,1.03800826E-1,-8.082653E-2,-2.9975256E-1,5.3877965E-2,2.4924438E-1,-1.4922632E-1,-2.9143222E-2,-1.6667625E-2,-6.505436E-3,3.358095E-2,1.6154932E-2,1.933141E-2,1.8226309E-1,-1.8456236E-1,-1.5430292E-2,9.4091584E-4,-1.1493006E-1,-2.8191913E-2,1.4342888E-1,4.7473636E-4,2.1172684E-1,-7.924947E-2,-2.0868921E-1,6.490077E-2,-3.8701254E-3,3.1963002E-2,-9.011993E-2,1.0474192E-2,-1.9940574E-1,-4.184787E-4,-6.47765E-3,1.815583E-1,-3.1331024E-4,1.1336666E-2,1.4069214E-1,7.421079E-4,-5.1073073E-3,-1.2885723E-2,-5.6686294E-3,-4.04596E-4,5.10285E-3,3.7648326E-3,-1.0717239E-3,-9.742122E-3,2.5335603E-4,-2.7053184E-3,2.4534662E-3,-1.0020842E-2,-2.1992493E-3,-1.2658308E-3,4.8228675E-3,8.842286E-3,-2.757196E-5,2.0694528E-3,7.1316552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,21,23,25,27,29,31,33,-1,35,37,39,41,-1,43,45,47,49,51,-1,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2121947E0,3.5291684E-1,6.7636216E-1,4.65823E-1,5.424303E-2,4.7886646E-1,2.6539338E-1,2.6663804E-1,1.988315E-1,0E0,0E0,4.722514E-1,0E0,0E0,9.821057E-2,9.780061E-2,8.765997E-2,1.6672455E-1,2.1396086E-1,1.6040917E-1,1.4531642E-1,0E0,3.670883E-2,4.5187578E-2,2.2252572E-1,3.1765796E-2,0E0,1.5271504E-1,2.1133411E-1,3.7227035E-2,6.2299877E-2,1.3776477E-1,0E0,1.1748266E-1,0E0,0E0,2.0124674E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,33,33,36,36],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,22,24,26,28,30,32,34,-1,36,38,40,42,-1,44,46,48,50,52,-1,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.969686E2,6.8652206E9,1.5915463E0,2.8070276E2,1.613773E6,2.9028345E2,6.677E3,1.0137E5,7.739782E0,-1.6667625E-2,-6.505436E-3,7.9522475E5,1.6154932E-2,1.933141E-2,5.4907064E0,2.8434807E3,1.0879989E8,3.2432194E1,9.952575E5,1.4146589E8,9.040637E7,4.7473636E-4,8.0426145E-1,4.611731E5,1.9977521E4,3.25077E5,-3.8701254E-3,5.5067914E4,3.9761555E1,6.7378756E5,2.4536678E8,6.2019708E7,-6.47765E-3,7.1273893E-1,-3.1331024E-4,1.1336666E-2,7.4245725E0,7.421079E-4,-5.1073073E-3,-1.2885723E-2,-5.6686294E-3,-4.04596E-4,5.10285E-3,3.7648326E-3,-1.0717239E-3,-9.742122E-3,2.5335603E-4,-2.7053184E-3,2.4534662E-3,-1.0020842E-2,-2.1992493E-3,-1.2658308E-3,4.8228675E-3,8.842286E-3,-2.757196E-5,2.0694528E-3,7.1316552E-3],"split_indices":[52,5,39,52,1,56,9,29,53,0,0,28,0,0,54,47,7,56,51,45,45,0,27,28,33,1,0,33,56,51,7,32,0,42,0,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.33E2,1.39E2,9.4E1,1.32E2,7E0,7.1E1,2.3E1,5.6E1,7.6E1,3E0,4E0,6.8E1,3E0,4E0,1.9E1,4.4E1,1.2E1,5.7E1,1.9E1,4.4E1,2.4E1,3E0,1.6E1,9E0,3.5E1,6E0,6E0,4.3E1,1.4E1,8E0,1.1E1,3.7E1,7E0,1.9E1,5E0,7E0,9E0,3E0,6E0,1.3E1,2.2E1,3E0,3E0,2.1E1,2.2E1,5E0,9E0,3E0,5E0,8E0,3E0,3E1,7E0,1.6E1,3E0,3E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.080086E-2,-7.972779E-2,1.5150319E-1,-6.0212612E-2,-2.3834008E-1,1.957967E-1,-1.3313913E-2,-1.6798964E-1,-2.6258763E-2,-2.2953784E-3,-1.4646858E-2,1.1032531E-1,3.1577852E-1,-7.03512E-2,5.439622E-3,-8.84642E-2,-2.2359942E-1,-1.0002504E-2,-1.1916165E-2,2.2951801E-1,6.7251205E-2,4.2089403E-1,7.843546E-3,-4.51044E-3,2.6181384E-3,-7.4851373E-3,-1.6979715E-2,-4.257136E-3,-1.0974921E-2,6.8024225E-3,-1.3727362E-1,1.1585974E-2,4.6672137E-3,3.603094E-2,1.4650182E-1,4.8070705E-1,8.485365E-3,-2.59568E-3,2.3131403E-3,-1.2596803E-3,2.5611245E-3,3.3259296E-4,-1.0019697E-2,3.966078E-3,-7.3793624E-4,7.7389623E-3,2.0271528E-3,2.42658E-2,1.2190328E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,-1,29,31,33,35,-1,-1,-1,-1,37,-1,-1,39,41,-1,-1,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9289875E0,4.189976E-1,6.596365E-1,4.5667887E-1,2.9778808E-1,6.932676E-1,1.7444496E-1,1.128912E-1,2.9698947E-1,0E0,0E0,2.0533168E-1,3.0479598E-1,8.445024E-2,0E0,8.948268E-2,5.8271766E-2,0E0,2.1681313E-1,3.330314E-2,7.637085E-2,9.498E-2,0E0,0E0,0E0,0E0,3.353676E-2,0E0,0E0,1.6818629E-1,1.8494542E-1,0E0,0E0,8.23074E-2,2.6440904E-2,4.3819427E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,26,26,29,29,30,30,33,33,34,34,35,35],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,-1,30,32,34,36,-1,-1,-1,-1,38,-1,-1,40,42,-1,-1,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.512144E2,4.56567E7,9.298568E7,1.8452284E2,9.214292E7,3.5078225E3,1.4975258E4,2.2304833E0,5.160874E0,-2.2953784E-3,-1.4646858E-2,1.10597E5,1.2951E4,8.129243E5,5.439622E-3,1.0135136E0,9.021944E0,-1.0002504E-2,2.6921875E1,6.7179665E1,3.0455479E3,5.9337012E7,7.843546E-3,-4.51044E-3,2.6181384E-3,-7.4851373E-3,1.4653994E6,-4.257136E-3,-1.0974921E-2,4.3062625E2,1.176491E-1,1.1585974E-2,4.6672137E-3,2.5218935E1,5.8622135E6,8.315045E0,8.485365E-3,-2.59568E-3,2.3131403E-3,-1.2596803E-3,2.5611245E-3,3.3259296E-4,-1.0019697E-2,3.966078E-3,-7.3793624E-4,7.7389623E-3,2.0271528E-3,2.42658E-2,1.2190328E-2],"split_indices":[52,45,45,52,7,52,4,56,54,0,0,29,9,28,0,56,54,0,56,58,4,45,0,0,0,0,50,0,0,52,38,0,0,56,32,54,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.39E2,8.9E1,1.25E2,1.4E1,7E1,1.9E1,2.9E1,9.6E1,6E0,8E0,4.2E1,2.8E1,1.4E1,5E0,1.3E1,1.6E1,5E0,9.1E1,1E1,3.2E1,1.4E1,1.4E1,1.1E1,3E0,5E0,8E0,5E0,1.1E1,8E1,1.1E1,6E0,4E0,2.4E1,8E0,1E1,4E0,5E0,3E0,4.8E1,3.2E1,5E0,6E0,1.1E1,1.3E1,5E0,3E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.0230091E-2,-2.5522126E-2,2.1995468E-1,-1.155817E-1,1.0690755E-2,1.9756163E-2,1.2871362E-1,-1.6644476E-1,-4.5067675E-2,-1.5287995E-1,3.396816E-2,-1.3564095E-2,1.8479739E-1,-8.172417E-2,-2.6348233E-1,-1.3286221E-1,2.7132863E-2,-1.2862638E-2,-1.4522415E-2,4.2346553E-3,1.0817738E-1,4.695651E-3,-4.654279E-3,8.835622E-3,1.2663028E-3,-1.2280439E-1,8.647118E-4,-3.0039278E-1,-5.6657577E-3,-1.2185518E-3,-6.9477824E-3,-3.0343838E-3,7.466285E-2,5.4001324E-3,-8.295796E-2,-2.096201E-2,9.4189316E-2,-1.2449856E-3,1.3305551E-1,-8.816239E-3,-3.0774858E-3,-5.274038E-3,-1.4737968E-2,4.5912643E-3,-3.1336397E-4,-4.9065757E-3,-7.301385E-4,3.531328E-4,-3.6149928E-3,5.928841E-3,3.6040565E-4,-5.8537124E-5,6.3963924E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,-1,-1,-1,-1,39,-1,41,-1,-1,-1,-1,43,-1,45,47,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6477019E0,6.1436313E-1,6.9074535E-1,1.8617296E-1,5.192058E-1,0E0,2.0199344E-1,2.333206E-1,1.5573642E-1,4.6722826E-1,2.6211545E-1,1.13365516E-1,7.3381126E-2,7.845461E-2,2.4234116E-2,3.497085E-2,7.234252E-2,1.2140285E-1,0E0,1.9689833E-1,1.2096432E-1,0E0,0E0,0E0,0E0,4.0984526E-2,0E0,5.120033E-2,0E0,0E0,0E0,0E0,3.0715536E-2,0E0,1.6534418E-2,1.339935E-1,7.8550145E-2,0E0,8.4341645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,25,25,27,27,32,32,34,34,35,35,36,36,38,38],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,-1,-1,-1,-1,40,-1,42,-1,-1,-1,-1,44,-1,46,48,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.4788145E3,2.8070276E2,3.9247762E6,2.037744E10,6.794276E-5,1.9756163E-2,1.8790482E9,8.1446184E7,1.661E3,8.8284516E-1,8.564393E5,3.733336E2,6E1,5.841047E4,8.346531E5,2.44412E3,1.5761888E2,3.347E3,-1.4522415E-2,1.537424E0,1.988E3,4.695651E-3,-4.654279E-3,8.835622E-3,1.2663028E-3,2.6336688E-1,8.647118E-4,3.2217744E5,-5.6657577E-3,-1.2185518E-3,-6.9477824E-3,-3.0343838E-3,9.129683E7,5.4001324E-3,5.128017E-1,1.0076795E1,1.286E3,-1.2449856E-3,7.340097E7,-8.816239E-3,-3.0774858E-3,-5.274038E-3,-1.4737968E-2,4.5912643E-3,-3.1336397E-4,-4.9065757E-3,-7.301385E-4,3.531328E-4,-3.6149928E-3,5.928841E-3,3.6040565E-4,-5.8537124E-5,6.3963924E-3],"split_indices":[52,52,47,31,42,0,7,5,2,58,28,58,3,33,51,47,52,2,0,42,2,0,0,0,0,27,0,32,0,0,0,0,7,0,27,54,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.87E2,3.1E1,5.3E1,1.34E2,7E0,2.4E1,3E1,2.3E1,1.6E1,1.18E2,7E0,1.7E1,1.7E1,1.3E1,1E1,1.3E1,1E1,6E0,8.5E1,3.3E1,3E0,4E0,1.4E1,3E0,1.2E1,5E0,9E0,4E0,3E0,7E0,4E0,9E0,3E0,7E0,6.7E1,1.8E1,5E0,2.8E1,3E0,9E0,3E0,6E0,6E0,3E0,4E0,3E0,4.7E1,2E1,1.1E1,7E0,4E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.9052262E-3,-7.8282915E-2,1.0157419E-1,-6.240629E-2,-2.609741E-1,1.700144E-1,-7.709844E-3,-1.0424082E-1,-1.10702E-2,-5.3585274E-3,-1.3527196E-2,2.6971335E-2,1.304296E-1,-6.039091E-2,1.3530001E-1,-1.4925286E-1,-3.782259E-2,2.5234343E-2,-1.0113967E-1,7.973008E-2,1.9824572E-1,-9.422665E-3,-2.9975604E-2,7.821637E-3,3.043493E-4,-2.5947267E-1,-7.715086E-2,7.39501E-2,-8.622917E-2,4.9034975E-2,-3.8873593E-3,-1.5859585E-1,8.7089936E-4,3.260227E-3,1.5599616E-1,3.0813077E-3,2.216092E-1,4.835901E-3,-5.652523E-2,-3.165558E-3,-1.2361654E-2,6.417292E-5,-5.754041E-3,5.7299165E-3,-1.7119998E-3,-1.8898451E-3,-7.165379E-3,2.6216237E-3,-3.0761447E-3,-2.1999762E-3,-7.920772E-3,2.7847588E-3,-1.5636929E-3,7.9047205E-3,1.21707424E-4,1.0030249E-2,3.6389364E-3,-3.1546312E-3,1.9200273E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,39,41,43,45,47,-1,49,-1,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7260755E0,3.5788065E-1,6.819382E-1,2.5656062E-1,4.4433296E-2,9.834206E-1,2.7939174E-1,1.916697E-1,1.823984E-1,0E0,0E0,0E0,1.6642082E-1,1.3063827E-1,6.741831E-2,2.8991705E-1,1.5787159E-1,1.1809701E-1,1.15670785E-1,1.8611722E-1,4.7103763E-2,0E0,1.00198284E-1,0E0,0E0,8.514428E-2,1.22102335E-1,7.275367E-2,6.2622786E-2,6.4879246E-2,0E0,2.7945071E-2,0E0,4.7260474E-2,8.537778E-2,0E0,2.8858423E-2,0E0,4.62024E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,31,31,33,33,34,34,36,36,38,38],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,40,42,44,46,48,-1,50,-1,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.969686E2,4.56567E7,5.9337012E7,3.4008475E2,5.6252275E5,4.783784E0,8.129243E5,3.34226E8,9.607843E-1,-5.3585274E-3,-1.3527196E-2,2.6971335E-2,1.4977078E0,2.19E2,9.4721675E-1,1.9977521E4,2.7172614E10,1.5350326E1,1.1596293E3,6.2711465E-1,4.361E3,-9.422665E-3,1.0866135E3,7.821637E-3,3.043493E-4,6.3323677E3,6.358E3,1.4338E4,8.87E2,7.5264E1,-3.8873593E-3,1.7086E4,8.7089936E-4,1.1956835E11,1.3965513E-1,3.0813077E-3,2.768E3,4.835901E-3,1.8357493E-1,-3.165558E-3,-1.2361654E-2,6.417292E-5,-5.754041E-3,5.7299165E-3,-1.7119998E-3,-1.8898451E-3,-7.165379E-3,2.6216237E-3,-3.0761447E-3,-2.1999762E-3,-7.920772E-3,2.7847588E-3,-1.5636929E-3,7.9047205E-3,1.21707424E-4,1.0030249E-2,3.6389364E-3,-3.1546312E-3,1.9200273E-3],"split_indices":[52,45,45,52,28,54,28,5,57,0,0,0,39,10,27,33,31,54,55,27,2,0,52,0,0,50,29,9,0,58,0,9,0,31,38,0,0,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.28E2,9E1,1.19E2,9E0,5.5E1,3.5E1,6.5E1,5.4E1,4E0,5E0,3E0,5.2E1,2.6E1,9E0,3.8E1,2.7E1,3.9E1,1.5E1,3.1E1,2.1E1,3E0,2.3E1,6E0,3E0,1.4E1,2.4E1,8E0,1.9E1,3.3E1,6E0,1E1,5E0,1.6E1,1.5E1,4E0,1.7E1,3E0,2E1,3E0,1.1E1,1.1E1,1.3E1,5E0,3E0,1.4E1,5E0,3E1,3E0,3E0,7E0,6E0,1E1,1.2E1,3E0,1.4E1,3E0,1.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[5.7941223E-3,-3.35162E-2,2.1918818E-1,-1.1002138E-1,7.308644E-3,2.0075614E-2,1.6244051E-1,-1.600644E-1,-5.83123E-2,4.0810894E-2,-1.4515194E-1,2.4543485E-1,3.6637038E-3,-2.3414701E-1,-6.785064E-2,1.6549198E-2,-1.0670368E-1,-1.1374123E-2,9.04034E-2,-2.2909635E-1,-2.5117967E-2,2.708543E-1,2.8827726E-3,-3.1452833E-3,5.3961845E-3,-3.258463E-3,-1.1748582E-2,-1.2477338E-1,1.1784121E-2,8.603898E-2,-3.1090623E-3,-1.3511382E-1,2.2718662E-3,7.128497E-2,-6.613552E-2,1.6084148E-1,3.2548547E-2,-1.36088375E-2,-1.13059185E-1,3.8042986E-3,-3.3308319E-3,2.9954085E-1,3.953238E-3,-6.927635E-3,-2.00971E-3,-1.6956693E-3,2.1364838E-3,-6.397984E-4,7.046232E-3,-3.5511188E-3,-7.4591315E-3,4.0700287E-3,-1.6773461E-3,-6.056745E-3,-8.0001156E-4,2.085359E-3,8.314071E-3,-1.6938116E-3,2.6097184E-3,-6.389233E-3,-1.3671814E-3,5.202807E-3,1.3754421E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,43,45,47,-1,49,-1,51,53,55,57,-1,59,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9954422E0,6.285898E-1,4.7993445E-1,1.7180377E-1,6.8014985E-1,0E0,4.2078316E-1,2.2321326E-1,1.3218126E-1,2.8311872E-1,2.3607385E-1,7.493949E-2,1.3117269E-1,1.2483752E-1,7.853183E-2,1.020583E-1,1.08439386E-1,2.4914694E-1,2.2297516E-1,1.2507218E-1,7.875858E-2,6.1885238E-2,0E0,0E0,0E0,0E0,0E0,2.4394333E-2,1.8947674E-2,7.9644755E-2,0E0,2.6836425E-2,0E0,7.0675835E-2,1.2114382E-1,1.0069573E-1,7.440655E-2,0E0,2.139099E-2,0E0,0E0,4.4092655E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,27,27,28,28,29,29,31,31,33,33,34,34,35,35,36,36,38,38,41,41],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,44,46,48,-1,50,-1,52,54,56,58,-1,60,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,3.4008475E2,2.2779044E-3,2.1146445E4,6.156076E7,2.0075614E-2,3.09483E2,2.8490282E8,9.360022E5,7.608277E2,2.2270758E1,7.794012E7,1.3978E4,4.54E2,5.2682075E5,7.7524557E0,5.9066307E1,5.88E2,1.3766816E6,1.9662491E3,2.7864855E11,6.997599E0,2.8827726E-3,-3.1452833E-3,5.3961845E-3,-3.258463E-3,-1.1748582E-2,5.377697E2,7.683833E-1,1.3776097E-1,-3.1090623E-3,8.110325E0,2.2718662E-3,3.314203E8,9.50634E5,1.0625348E8,9.281264E-2,-1.36088375E-2,8.8519E4,3.8042986E-3,-3.3308319E-3,5.0777936E-1,3.953238E-3,-6.927635E-3,-2.00971E-3,-1.6956693E-3,2.1364838E-3,-6.397984E-4,7.046232E-3,-3.5511188E-3,-7.4591315E-3,4.0700287E-3,-1.6773461E-3,-6.056745E-3,-8.0001156E-4,2.085359E-3,8.314071E-3,-1.6938116E-3,2.6097184E-3,-6.389233E-3,-1.3671814E-3,5.202807E-3,1.3754421E-2],"split_indices":[52,52,57,33,45,0,58,5,48,52,58,45,9,2,32,53,58,0,47,4,31,57,0,0,0,0,0,4,27,39,0,54,0,7,1,7,38,0,12,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2E2,3.6E1,6.9E1,1.31E2,5E0,3.1E1,3.4E1,3.5E1,1.08E2,2.3E1,2E1,1.1E1,1.8E1,1.6E1,1.4E1,2.1E1,5.3E1,5.5E1,1.3E1,1E1,1.7E1,3E0,7E0,4E0,5E0,1.3E1,9E0,7E0,8E0,6E0,1.8E1,3E0,2.1E1,3.2E1,2.4E1,3.1E1,6E0,7E0,3E0,7E0,1.4E1,3E0,5E0,4E0,3E0,4E0,4E0,4E0,1E1,8E0,1.7E1,4E0,1.1E1,2.1E1,7E0,1.7E1,9E0,2.2E1,4E0,3E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.805905E-4,-7.292913E-2,1.0869555E-1,-5.8432646E-2,-2.35245E-1,7.008175E-2,3.1419542E-1,-1.1916666E-1,-2.6537867E-2,-3.7548665E-3,-1.3089709E-2,5.1821303E-2,1.2592267E-2,6.860142E-3,1.7277442E-2,-1.4983591E-1,-1.2175307E-2,-7.1539275E-2,2.7950974E-2,1.5710209E-2,1.5125711E-1,4.824629E-4,-1.6755591E-1,-3.5927196E-3,5.028135E-3,3.5199514E-3,-1.2075184E-1,2.0368602E-3,1.1839577E-1,-1.000615E-2,5.660245E-3,1.9870687E-1,-4.903437E-4,-3.2040915E-3,-7.8020426E-3,2.1015538E-3,-4.116542E-3,-6.3393973E-3,1.9189368E-3,2.5695274E-3,-1.5772695E-3,7.301813E-3,6.679669E-4,-2.9067714E-3,1.6807043E-3,2.8399634E-3,9.428855E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,27,29,31,-1,33,-1,-1,35,37,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6625711E0,2.8273314E-1,6.445318E-1,2.2347805E-1,8.857256E-2,2.9230544E-1,1.3029575E-1,1.2942946E-1,1.9326141E-1,0E0,0E0,2.4179375E-1,0E0,0E0,0E0,9.066892E-2,1.104178E-1,1.592609E-1,8.413552E-2,1.6022411E-1,1.416676E-1,0E0,3.8500726E-2,0E0,0E0,9.2747815E-2,1.4833304E-1,7.23497E-2,4.4690594E-2,1.3406706E-1,0E0,3.7567794E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,15,15,16,16,17,17,18,18,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,31,31],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,28,30,32,-1,34,-1,-1,36,38,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.579211E9,1.5988196E6,4.8081778E2,1E0,2.9028345E2,2.567762E6,1E0,4.4589956E5,-3.7548665E-3,-1.3089709E-2,1.5438564E0,1.2592267E-2,6.860142E-3,1.7277442E-2,2.7362996E7,1.278E3,4.79348E5,4.5E1,8.1350375E5,8.413669E-1,4.824629E-4,2.8434807E3,-3.5927196E-3,5.028135E-3,2.2198856E1,1.4849977E5,5.588992E5,3E0,1.8436606E5,5.660245E-3,1.5262272E3,-4.903437E-4,-3.2040915E-3,-7.8020426E-3,2.1015538E-3,-4.116542E-3,-6.3393973E-3,1.9189368E-3,2.5695274E-3,-1.5772695E-3,7.301813E-3,6.679669E-4,-2.9067714E-3,1.6807043E-3,2.8399634E-3,9.428855E-3],"split_indices":[52,5,28,4,64,56,47,112,28,0,0,39,0,0,0,5,2,1,3,28,27,0,47,0,0,56,33,48,8,33,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.25E2,8.3E1,1.16E2,9E0,7.1E1,1.2E1,3.9E1,7.7E1,4E0,5E0,6.7E1,4E0,6E0,6E0,3E1,9E0,4.2E1,3.5E1,5E1,1.7E1,3E0,2.7E1,6E0,3E0,1.7E1,2.5E1,2.8E1,7E0,4.2E1,8E0,1.3E1,4E0,6E0,2.1E1,1.2E1,5E0,2.1E1,4E0,1.1E1,1.7E1,4E0,3E0,1.9E1,2.3E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-5.601678E-3,-4.349248E-2,2.1157904E-1,-7.882008E-2,2.3539558E-2,3.1851465E-1,4.370111E-2,-5.7658486E-2,-2.2875044E-1,1.0060298E-1,-4.3474484E-2,4.348777E-1,1.8377416E-1,-8.779634E-3,7.325916E-3,-1.4358938E-1,-2.9351432E-2,-3.2818024E-3,-1.2224821E-2,1.6533555E-1,2.0532537E-2,-6.647325E-2,4.6501113E-3,2.4079993E-2,1.1939091E-2,1.0115378E-2,2.563072E-3,2.685951E-3,-3.3517857E-3,-1.0306747E-2,-1.08271435E-1,2.2894539E-2,-7.300886E-2,8.880545E-3,3.4329372E-3,-1.6191399E-2,5.194439E-3,-1.4670074E-1,-2.87016E-2,-5.3557325E-3,-4.261196E-4,-7.715279E-3,1.8832419E-3,-6.099774E-3,3.2474234E-4,6.3572626E-4,-2.624637E-3,-2.1328486E-3,-7.450506E-3,2.695351E-3,-2.716176E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,39,41,43,-1,-1,45,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8278446E0,4.4942272E-1,5.7850075E-1,3.802336E-1,3.4521952E-1,2.3365688E-1,9.9565506E-2,2.6335496E-1,1.2621146E-1,1.572834E-1,1.3653997E-1,4.733956E-2,6.625992E-2,6.383987E-2,0E0,7.369101E-2,1.9326918E-1,0E0,0E0,5.1261067E-2,5.9894666E-2,9.275985E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.43573E-2,1.9331576E-1,2.7806896E-1,0E0,0E0,1.8871995E-2,0E0,2.2499174E-2,8.406568E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,30,30,31,31,32,32,35,35,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,40,42,44,-1,-1,46,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,8.3813794E2,1.0528413E10,2.9076396E9,3.291238E7,1.0847E4,1.739E4,1.7398648E2,9.214292E7,1.8905495E6,1.4961406E6,1.4164306E-3,4.636564E7,3.733336E2,7.325916E-3,1.2338027E5,9.360022E5,-3.2818024E-3,-1.2224821E-2,2.5218935E1,4.8E1,1.9991961E-3,4.6501113E-3,2.4079993E-2,1.1939091E-2,1.0115378E-2,2.563072E-3,2.685951E-3,-3.3517857E-3,-1.0306747E-2,2.465E3,1.0135136E0,1.0903429E0,8.880545E-3,3.4329372E-3,1.9540394E7,5.194439E-3,6.83964E5,2.815322E1,-5.3557325E-3,-4.261196E-4,-7.715279E-3,1.8832419E-3,-6.099774E-3,3.2474234E-4,6.3572626E-4,-2.624637E-3,-2.1328486E-3,-7.450506E-3,2.695351E-3,-2.716176E-3],"split_indices":[52,52,12,5,45,9,9,52,7,47,28,57,45,58,0,28,48,0,0,56,3,41,0,0,0,0,0,0,0,0,2,56,42,0,0,45,0,1,58,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.2E2,1.88E2,3.2E1,1.23E2,6.5E1,1.9E1,1.3E1,1.09E2,1.4E1,3E1,3.5E1,9E0,1E1,1E1,3E0,2.6E1,8.3E1,5E0,9E0,1.6E1,1.4E1,3.1E1,4E0,3E0,6E0,6E0,4E0,5E0,5E0,5E0,2.1E1,3.8E1,4.5E1,9E0,7E0,1.1E1,3E0,9E0,2.2E1,1.7E1,4E0,3E0,3.5E1,2.3E1,2.2E1,7E0,4E0,3E0,6E0,6E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[7.2624795E-3,-6.2133513E-2,1.2256904E-1,-2.1971588E-1,-4.9909953E-2,1.6682304E-1,-4.226469E-2,-1.3954355E-2,-3.5194934E-3,-1.9169983E-1,-3.567102E-2,4.024967E-1,1.2405085E-1,-3.1567519E-3,2.7003514E-2,-4.7448245E-3,-1.311399E-2,7.8682356E-2,-5.350594E-2,2.3569817E-2,5.0159656E-3,-1.7676305E-2,1.6448843E-1,2.8469518E-3,-8.92131E-4,1.1039696E-2,7.186915E-3,-3.6202572E-2,-2.0122603E-1,-2.5335334E-3,4.4079935E-3,1.7487533E-1,4.8428576E-4,-1.4838781E-3,4.223608E-3,-3.3969618E-3,3.5579174E-4,-1.5153328E-4,-1.1369751E-2,7.7874134E-3,1.4703831E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,-1,-1,25,27,-1,-1,29,31,-1,-1,33,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8320291E0,2.644831E-1,6.34382E-1,1.2965572E-1,2.6369938E-1,6.3552916E-1,4.6869975E-2,0E0,0E0,7.942924E-2,2.550664E-1,4.0137935E-1,3.4104258E-1,0E0,1.5702367E-2,0E0,0E0,1.07388444E-1,2.6621303E-1,0E0,0E0,8.230727E-2,7.095122E-2,0E0,0E0,5.0652914E-2,0E0,2.002552E-1,1.5900123E-1,0E0,0E0,7.346368E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,17,17,18,18,21,21,22,22,25,25,27,27,28,28,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,-1,-1,26,28,-1,-1,30,32,-1,-1,34,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0688497E3,7.784166E5,9.040637E7,1.9977521E4,5.178571E0,5.0655737E0,1.0252E4,-1.3954355E-2,-3.5194934E-3,9.372475E-1,7.31E3,5.9337012E7,1E0,-3.1567519E-3,1E0,-4.7448245E-3,-1.311399E-2,9.255088E-1,4.462437E7,2.3569817E-2,5.0159656E-3,1.2739197E9,1E0,2.8469518E-3,-8.92131E-4,4.7904646E-1,7.186915E-3,3.6913754E2,9.214292E7,-2.5335334E-3,4.4079935E-3,4.679E3,4.8428576E-4,-1.4838781E-3,4.223608E-3,-3.3969618E-3,3.5579174E-4,-1.5153328E-4,-1.1369751E-2,7.7874134E-3,1.4703831E-3],"split_indices":[52,45,45,33,54,54,9,0,0,27,9,45,109,0,8,0,0,27,45,0,0,7,89,0,0,41,0,52,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.27E2,1.42E2,8.5E1,9E0,1.33E2,6.7E1,1.8E1,4E0,5E0,1.1E1,1.22E2,9E0,5.8E1,1.2E1,6E0,8E0,3E0,1.6E1,1.06E2,5E0,4E0,1.3E1,4.5E1,3E0,3E0,1E1,6E0,9.6E1,1E1,1E1,3E0,4.2E1,3E0,7E0,3E0,4.7E1,4.9E1,3E0,7E0,3.8E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[5.7318574E-3,-4.2113416E-2,1.5979381E-1,-1.1411153E-1,-1.8340442E-2,2.1039871E-2,1.1381915E-1,-6.499676E-3,-1.7764078E-1,-4.3626256E-2,7.297215E-2,1.693369E-1,8.797656E-3,8.149027E-2,-1.15674645E-1,-1.2274005E-1,-1.2107454E-2,2.1383192E-2,-8.980314E-2,7.0661684E-3,3.4074668E-2,6.3767895E-2,2.1442427E-1,-3.2075727E-3,5.229251E-3,8.4274844E-4,6.952605E-3,-1.4359726E-3,-6.5073078E-3,-1.5632911E-1,-3.802765E-2,-2.2927731E-2,1.414563E-1,-1.2672357E-1,-1.3444024E-2,-1.3059534E-3,4.2437753E-3,7.225663E-3,-2.389598E-3,2.3096722E-1,3.4019432E-3,-1.9684774E-3,-7.5146444E-3,2.2662394E-3,-5.0191125E-3,1.5580177E-4,-5.943493E-3,1.7474726E-3,7.0254123E-3,-5.7933987E-3,1.3056691E-3,4.0963534E-4,-4.763052E-3,1.0408467E-2,3.7490702E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,-1,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,41,43,45,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6799774E0,2.957272E-1,8.0811894E-1,2.928149E-1,3.0796003E-1,0E0,2.8420752E-1,1.7273346E-1,1.346116E-1,3.1412137E-1,1.04552954E-1,1.3864118E-1,1.9258587E-1,4.9493358E-2,2.1505222E-2,5.0963372E-2,0E0,2.3734748E-1,1.7026827E-1,0E0,1.021875E-1,1.5835875E-1,2.911365E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.9919535E-2,6.0769238E-2,1.0691158E-1,2.6610047E-2,9.090936E-2,5.1000964E-2,0E0,0E0,0E0,0E0,3.2253504E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,20,20,21,21,22,22,29,29,30,30,31,31,32,32,33,33,34,34,39,39],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,-1,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,42,44,46,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.077646E3,2.3877826E4,1.4164306E-3,3E1,8.1350375E5,2.1039871E-2,3.09483E2,1E0,7.340097E7,7.34E2,8.976933E5,3.383436E5,2.8159826E0,1.4532935E6,8.231047E0,8.8519E4,-1.2107454E-2,2.9110428E6,3.4741312E8,7.0661684E-3,2.5149E4,5.4E9,5.1E1,-3.2075727E-3,5.229251E-3,8.4274844E-4,6.952605E-3,-1.4359726E-3,-6.5073078E-3,5.58E2,1.33511E5,4.3977472E5,7.83E2,7.2838904E7,9.3405694E-1,-1.3059534E-3,4.2437753E-3,7.225663E-3,-2.389598E-3,1E0,3.4019432E-3,-1.9684774E-3,-7.5146444E-3,2.2662394E-3,-5.0191125E-3,1.5580177E-4,-5.943493E-3,1.7474726E-3,7.0254123E-3,-5.7933987E-3,1.3056691E-3,4.0963534E-4,-4.763052E-3,1.0408467E-2,3.7490702E-3],"split_indices":[52,48,57,3,28,0,58,89,7,0,50,28,35,28,53,12,0,50,7,0,10,5,3,0,0,0,0,0,0,12,12,28,10,32,27,0,0,0,0,89,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.73E2,5.3E1,4.2E1,1.31E2,5E0,4.8E1,1.6E1,2.6E1,1.03E2,2.8E1,3.1E1,1.7E1,9E0,7E0,1.9E1,7E0,4.3E1,6E1,7E0,2.1E1,1E1,2.1E1,1E1,7E0,6E0,3E0,3E0,4E0,1.3E1,6E0,3.2E1,1.1E1,4E1,2E1,1.1E1,1E1,5E0,5E0,1.8E1,3E0,3E0,1E1,3E0,3E0,2.7E1,5E0,3E0,8E0,3.7E1,3E0,1.7E1,3E0,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[7.60741E-3,-2.1384846E-2,1.5557875E-1,-7.452437E-2,1.2663214E-2,2.4859248E-1,4.8754316E-2,-6.0562816E-2,-8.087792E-3,6.8502E-2,-7.015637E-2,1.9230604E-2,1.7898265E-1,1.0757301E-1,-1.5808482E-3,-1.2135244E-1,-3.1494446E-2,1.08919874E-1,-1.53120905E-2,-1.8886879E-1,-3.6175817E-2,9.562018E-3,4.6628863E-3,8.0694E-3,1.2288698E-3,-3.8268577E-2,-1.5203474E-1,9.403876E-2,-5.2675396E-2,4.927657E-2,1.529041E-1,3.9119907E-2,-1.22262076E-1,-5.6601108E-5,-1.1128854E-2,-1.19225485E-2,-1.22603595E-1,-3.2714661E-3,5.0115224E-4,-7.5434274E-3,-2.271889E-3,-7.4751844E-4,7.5551113E-3,-1.5277453E-4,-4.1975784E-3,3.5508918E-3,-1.493858E-3,7.842019E-3,2.5884858E-3,6.0323523E-3,1.3558751E-4,-7.845501E-3,-1.8156825E-3,2.107368E-3,-2.6058217E-3,-7.5310823E-3,-7.2950666E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,-1,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.4284207E-1,3.3385456E-1,3.4558094E-1,1.0978326E-1,5.2737945E-1,2.0285785E-1,9.618929E-2,1.1280374E-1,0E0,2.3202375E-1,1.7872755E-1,0E0,2.9254854E-2,6.9149435E-2,0E0,4.7715127E-2,1.272635E-1,1.13140166E-1,1.3867742E-1,1.4993584E-1,7.609603E-2,0E0,0E0,0E0,0E0,1.5353414E-2,3.4153998E-2,7.28304E-2,9.5233664E-2,6.866427E-2,7.425988E-2,6.0715906E-2,3.3929907E-2,0E0,0E0,9.9528626E-2,4.67157E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,-1,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,4.561433E2,5.9337012E7,1.3865231E6,3.203027E7,5.841377E0,9.820072E7,1.8452284E2,-8.087792E-3,2.0688E4,4.994744E-5,1.9230604E-2,6.8734935E6,6.972111E6,-1.5808482E-3,1.153E3,4.6376264E9,6.8686145E2,2.2198856E1,1.9637405E-6,9.362201E-1,9.562018E-3,4.6628863E-3,8.0694E-3,1.2288698E-3,1.1475E2,4.7095413E2,1.0347391E5,1.4573639E6,9.2413794E2,8.47598E1,4.45006E8,2.4277832E6,-5.6601108E-5,-1.1128854E-2,6.1093975E6,1.2876742E3,-3.2714661E-3,5.0115224E-4,-7.5434274E-3,-2.271889E-3,-7.4751844E-4,7.5551113E-3,-1.5277453E-4,-4.1975784E-3,3.5508918E-3,-1.493858E-3,7.842019E-3,2.5884858E-3,6.0323523E-3,1.3558751E-4,-7.845501E-3,-1.8156825E-3,2.107368E-3,-2.6058217E-3,-7.5310823E-3,-7.2950666E-4],"split_indices":[52,52,45,28,45,54,45,52,0,9,42,0,47,47,0,29,31,52,56,42,27,0,0,0,0,52,4,28,51,4,58,12,48,0,0,32,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.83E2,3.5E1,7.1E1,1.12E2,1.8E1,1.7E1,6.5E1,6E0,6.7E1,4.5E1,3E0,1.5E1,1E1,7E0,2E1,4.5E1,4.5E1,2.2E1,9E0,3.6E1,7E0,8E0,4E0,6E0,6E0,1.4E1,6E0,3.9E1,2E1,2.5E1,1.5E1,7E0,3E0,6E0,2.9E1,7E0,3E0,3E0,1E1,4E0,3E0,3E0,2E1,1.9E1,1.4E1,6E0,1.7E1,8E0,3E0,1.2E1,3E0,4E0,1.3E1,1.6E1,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.397775E-2,-1.9844405E-2,1.4658162E-1,-4.689803E-2,4.9330316E-2,2.234487E-2,1.035584E-1,-1.454698E-1,-3.0646916E-2,1.3694298E-1,5.356589E-3,1.8206939E-1,3.1046253E-2,4.4797356E-3,-2.07583E-1,-5.1510543E-2,8.161033E-2,8.306715E-3,1.0814922E-3,-5.0183315E-2,1.0673337E-1,1.2717286E-1,1.0672974E-2,-2.0269565E-2,5.952818E-3,-1.9072363E-3,-1.0235341E-2,-1.5633011E-2,-1.19289294E-1,-1.5754747E-3,1.5036146E-1,-4.287235E-3,-7.581388E-4,6.268131E-3,5.674318E-4,6.6862553E-3,1.6425442E-3,-2.3432078E-3,2.9405607E-3,-1.5278872E-3,3.9121285E-3,2.1959312E-4,-7.053085E-3,7.669923E-3,2.9459426E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,29,-1,-1,31,33,35,-1,37,-1,-1,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.49233E-1,3.1858462E-1,6.7219615E-1,1.909737E-1,1.8419391E-1,0E0,2.2206742E-1,2.9043517E-1,2.524657E-1,1.0796845E-1,1.911613E-1,4.639697E-2,1.3032785E-1,0E0,7.867515E-2,2.169912E-1,1.4562374E-1,0E0,0E0,3.556485E-2,4.791966E-2,3.2766834E-2,0E0,5.7196103E-2,0E0,0E0,0E0,1.4435135E-1,1.9871733E-1,0E0,1.7312437E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,20,20,21,21,23,23,27,27,28,28,30,30],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,30,-1,-1,32,34,36,-1,38,-1,-1,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.077646E3,4.4243875E6,1.4164306E-3,2.5333811E-5,7.11E2,2.234487E-2,5.7025972E7,2.1E1,1.3900659E6,3.541162E-1,4.375139E5,1.182193E6,8.129243E5,4.4797356E-3,4.327473E5,1.7086E4,1.62521E5,8.306715E-3,1.0814922E-3,3.4288502E8,1.3766816E6,5.501913E9,1.0672974E-2,1.4975258E4,5.952818E-3,-1.9072363E-3,-1.0235341E-2,2.1816934E8,6.425382E0,-1.5754747E-3,9.14E2,-4.287235E-3,-7.581388E-4,6.268131E-3,5.674318E-4,6.6862553E-3,1.6425442E-3,-2.3432078E-3,2.9405607E-3,-1.5278872E-3,3.9121285E-3,2.1959312E-4,-7.053085E-3,7.669923E-3,2.9459426E-3],"split_indices":[52,51,57,38,0,0,45,3,28,38,28,29,28,0,28,9,1,0,0,7,47,5,0,4,0,0,0,7,53,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.68E2,4.2E1,1.21E2,4.7E1,3E0,3.9E1,1.6E1,1.05E2,1.5E1,3.2E1,1.8E1,2.1E1,3E0,1.3E1,8.9E1,1.6E1,9E0,6E0,2.1E1,1.1E1,1.2E1,6E0,1.5E1,6E0,3E0,1E1,5.9E1,3E1,6E0,1E1,7E0,1.4E1,7E0,4E0,8E0,4E0,1.1E1,4E0,5E1,9E0,9E0,2.1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-4.6013114E-3,-5.0133355E-2,8.716888E-2,-1.1859211E-1,-3.012609E-2,9.943325E-2,-2.5757807E-3,-8.423254E-3,-9.1137394E-2,-7.782088E-3,-1.0018009E-1,1.9706339E-1,7.848031E-2,-5.2291133E-2,-1.3935582E-1,2.9553868E-2,-5.1457286E-2,-2.1768269E-1,-3.924833E-2,2.7249023E-1,2.153255E-3,1.3021936E-1,4.3716323E-2,-2.9183635E-3,8.215174E-5,-7.557838E-3,-2.1012798E-3,-4.5663923E-2,6.623921E-2,-1.1414997E-2,-1.0496067E-1,-1.5557045E-3,-1.2470014E-2,-8.2406074E-2,2.8126917E-3,5.209279E-3,1.4516655E-2,1.4822267E-1,5.0255365E-4,-2.2924345E-3,1.0521416E-1,-2.68167E-3,1.1596039E-3,-2.496731E-4,4.318035E-3,-5.636199E-3,5.339065E-4,-5.658442E-3,-5.035442E-5,-2.5167348E-4,-5.134872E-3,7.0418417E-3,9.570219E-4,-1.623727E-3,5.122386E-3,-1.1678145E-3,5.8296965E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,41,43,45,47,-1,-1,49,-1,-1,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9438725E-1,1.9229048E-1,1.3631606E-1,5.6147933E-2,1.7453247E-1,1.1954653E-1,0E0,0E0,4.1413695E-2,1.4175712E-1,1.8476662E-1,1.0613036E-1,9.5032066E-2,1.5995841E-2,3.2947257E-2,1.3271606E-1,8.449779E-2,1.2442595E-1,9.3541935E-2,3.6278665E-2,0E0,4.4963777E-2,9.9271566E-2,0E0,0E0,0E0,0E0,2.4413392E-2,8.885312E-2,7.462767E-2,5.5965513E-2,0E0,0E0,4.3828495E-2,0E0,0E0,0E0,4.8367143E-2,0E0,1.0275076E-1,7.54661E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,27,27,28,28,29,29,30,30,33,33,37,37,39,39,40,40],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,42,44,46,48,-1,-1,50,-1,-1,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,1.8452284E2,1.3917024E8,1.2338027E5,2.6519718E1,4.9695907E0,-2.5757807E-3,-8.423254E-3,5.734444E6,7.25E2,1.643614E6,1.2537655E-1,2.1508584E7,2.203E3,4.7095413E2,1.438E3,8.0391425E6,3.26E2,8.969686E2,5.256E3,2.153255E-3,2.709E4,5.506676E11,-2.9183635E-3,8.215174E-5,-7.557838E-3,-2.1012798E-3,1E0,5.3153326E2,8.27E2,1.4446067E10,-1.5557045E-3,-1.2470014E-2,1E0,2.8126917E-3,5.209279E-3,1.4516655E-2,2.603E3,5.0255365E-4,1E0,1.8159722E-1,-2.68167E-3,1.1596039E-3,-2.496731E-4,4.318035E-3,-5.636199E-3,5.339065E-4,-5.658442E-3,-5.035442E-5,-2.5167348E-4,-5.134872E-3,7.0418417E-3,9.570219E-4,-1.623727E-3,5.122386E-3,-1.1678145E-3,5.8296965E-3],"split_indices":[52,52,45,28,56,54,0,0,45,0,32,34,45,2,4,2,45,0,52,9,0,9,31,0,0,0,0,68,4,0,5,0,0,8,0,0,0,0,0,79,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.42E2,7E1,3.1E1,1.11E2,6.5E1,5E0,6E0,2.5E1,8.5E1,2.6E1,1E1,5.5E1,1.5E1,1E1,4.6E1,3.9E1,8E0,1.8E1,6E0,4E0,2.1E1,3.4E1,1.1E1,4E0,6E0,4E0,1.5E1,3.1E1,2.3E1,1.6E1,3E0,5E0,1.3E1,5E0,3E0,3E0,1.8E1,3E0,2E1,1.4E1,1.2E1,3E0,1.1E1,2E1,3E0,2E1,1.2E1,4E0,5E0,8E0,1.5E1,3E0,1.6E1,4E0,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[7.9259975E-3,-4.800474E-2,8.5787706E-2,-1.9326498E-1,-3.570468E-2,6.6163234E-2,2.8848034E-1,-3.7188223E-3,-1.1526799E-2,2.4574345E-2,-7.790642E-2,4.2776443E-2,8.635915E-3,1.6283346E-2,3.512651E-3,-2.9307206E-3,8.492485E-2,-2.2575046E-1,-5.8511674E-2,9.6929826E-2,9.167601E-3,-4.6246868E-2,4.2954694E-2,5.6940187E-3,6.203133E-4,-2.6373777E-3,-1.2831702E-2,3.323444E-2,-7.385006E-2,5.6153633E-2,7.49992E-3,4.1277073E-2,-7.176091E-2,-3.4579919E-3,6.434458E-4,5.1141374E-6,4.560365E-3,5.4544015E-3,-1.1528125E-3,9.150392E-4,-3.640728E-3,3.448781E-3,-1.969766E-3,-2.3554934E-3,4.2154077E-3,-3.7572468E-3,-5.730936E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,-1,-1,21,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.796074E-1,2.2715849E-1,3.5482037E-1,6.320888E-2,3.1226784E-1,2.8121656E-1,1.3782263E-1,0E0,0E0,8.490606E-2,1.9324362E-1,1.3743386E-1,0E0,0E0,0E0,7.3539376E-2,5.692631E-2,8.966002E-2,9.392476E-2,8.9592695E-2,1.2769352E-1,4.515449E-2,5.1862903E-2,0E0,0E0,0E0,0E0,6.393478E-2,7.80555E-2,6.328265E-2,0E0,2.142178E-1,1.8660426E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,27,27,28,28,29,29,31,31,32,32],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,-1,-1,22,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,5.178571E0,2.9028345E2,3.804992E5,6.611028E0,1.9894526E6,1.703939E4,-3.7188223E-3,-1.1526799E-2,5.438556E2,7.784166E5,4.541179E1,8.635915E-3,1.6283346E-2,3.512651E-3,7.2013897E-1,4.997555E8,5.84E2,8.6E1,3.0850107E1,8.419477E-1,7.555544E2,1.988E3,5.6940187E-3,6.203133E-4,-2.6373777E-3,-1.2831702E-2,7.31E3,2.0133628E6,2.2971932E7,7.49992E-3,3.192845E-1,7.157224E0,-3.4579919E-3,6.434458E-4,5.1141374E-6,4.560365E-3,5.4544015E-3,-1.1528125E-3,9.150392E-4,-3.640728E-3,3.448781E-3,-1.969766E-3,-2.3554934E-3,4.2154077E-3,-3.7572468E-3,-5.730936E-5],"split_indices":[52,54,56,28,53,28,4,0,0,52,45,56,0,0,0,27,12,10,10,56,27,4,2,0,0,0,0,9,45,45,0,27,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.23E2,1.3E2,9.3E1,9E0,1.21E2,8.6E1,7E0,5E0,4E0,5E1,7.1E1,7.5E1,1.1E1,4E0,3E0,3.5E1,1.5E1,7E0,6.4E1,2.8E1,4.7E1,1.8E1,1.7E1,8E0,7E0,3E0,4E0,9E0,5.5E1,2E1,8E0,3.4E1,1.3E1,1.1E1,7E0,1.1E1,6E0,3E0,6E0,7E0,4.8E1,1.6E1,4E0,1.3E1,2.1E1,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-9.496782E-4,-3.8115274E-2,1.0724571E-1,-2.7029138E-2,-1.5417278E-1,1.2327098E-2,8.68418E-2,-6.915812E-2,2.9126583E-2,-1.071029E-2,-6.721616E-2,-2.4017293E-3,1.3025728E-1,-4.4792943E-2,-1.2681744E-1,9.887964E-2,-1.4116176E-2,7.747441E-4,-7.418232E-3,-3.9393872E-2,4.2051766E-3,-3.89786E-5,1.5215528E-1,-5.4665316E-2,4.63399E-3,-2.1896085E-1,-7.057577E-2,1.1849135E-1,-1.3508077E-4,-8.037257E-3,5.2213822E-3,-2.7477073E-3,1.8526439E-3,1.8237442E-1,9.1628715E-2,-2.684636E-3,2.847514E-3,-3.9714174E-3,-1.0938849E-2,-4.232905E-3,-4.3843818E-4,5.6028725E-3,1.5368316E-3,-3.573848E-4,5.1790243E-3,3.317087E-3,8.816932E-3,-1.4268134E-3,5.6879497E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,-1,-1,31,-1,-1,33,35,-1,37,39,41,-1,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.806975E-1,2.0555343E-1,1.920777E-1,3.5688183E-1,1.1010602E-1,0E0,2.017895E-1,1.1513913E-1,1.9745626E-1,0E0,9.007883E-2,7.3292784E-2,1.0069865E-1,1.0210869E-1,1.1410409E-1,5.046794E-2,1.4386661E-1,0E0,0E0,3.4829818E-2,0E0,0E0,3.816098E-2,7.9034746E-2,0E0,2.4936795E-2,3.1289153E-2,2.3926646E-2,0E0,0E0,6.4275146E-2,0E0,0E0,4.2645156E-2,7.416866E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,22,22,23,23,25,25,26,26,27,27,30,30,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,-1,-1,32,-1,-1,34,36,-1,38,40,42,-1,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6252404E3,7.294459E7,1.4164306E-3,4.561433E2,8.657441E7,1.2327098E-2,1E0,2.0749E4,7.16E2,-1.071029E-2,1.4792529E10,1.3179107E4,2.6063783E5,2.0662694E-1,3.180155E-1,4.2946205E1,9.36105E5,7.747441E-4,-7.418232E-3,5.6487894E0,4.2051766E-3,-3.89786E-5,6.972111E6,5.7563637E1,4.63399E-3,8.741716E0,3.6E1,4.45006E8,-1.3508077E-4,-8.037257E-3,1.6974416E6,-2.7477073E-3,1.8526439E-3,1.5098364E5,5.318116E3,-2.684636E-3,2.847514E-3,-3.9714174E-3,-1.0938849E-2,-4.232905E-3,-4.3843818E-4,5.6028725E-3,1.5368316E-3,-3.573848E-4,5.1790243E-3,3.317087E-3,8.816932E-3,-1.4268134E-3,5.6879497E-3],"split_indices":[52,45,57,52,45,0,109,9,0,0,5,4,28,38,27,56,1,0,0,57,0,0,47,58,0,54,3,12,0,0,28,0,0,33,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.62E2,5.5E1,1.49E2,1.3E1,4E0,5.1E1,8.5E1,6.4E1,5E0,8E0,1.7E1,3.4E1,6.1E1,2.4E1,2.4E1,4E1,5E0,3E0,1.3E1,4E0,5E0,2.9E1,5.8E1,3E0,8E0,1.6E1,2E1,4E0,3E0,3.7E1,1E1,3E0,1.8E1,1.1E1,5.4E1,4E0,3E0,5E0,1E1,6E0,1.6E1,4E0,3.4E1,3E0,5E0,1.3E1,3E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.7470933E-3,-2.2789543E-2,1.8049288E-1,-7.472804E-2,1.3627979E-2,2.986159E-1,5.7172354E-2,-5.9161063E-2,-1.5399939E-1,5.153737E-2,-4.3520022E-2,2.0520555E-2,6.724672E-3,-2.6732462E-3,4.341468E-3,-1.19589984E-1,-3.1389114E-2,-1.1563775E-3,-2.0691332E-1,1.16864175E-1,1.8488783E-2,-1.8934958E-1,-1.4004011E-2,-1.5783414E-1,-2.3956813E-3,2.228505E-2,-8.179162E-2,-1.0340871E-2,-3.7470656E-3,6.552793E-2,6.7319716E-3,-3.8274344E-2,6.242807E-2,-9.81231E-4,-1.1747378E-2,6.102512E-2,-6.446366E-2,-8.098195E-3,-2.3987603E-3,-2.7728104E-3,2.5127132E-3,3.2421027E-3,-4.3885927E-3,4.518601E-3,5.154818E-4,2.2705756E-6,-6.9745844E-3,3.997342E-4,3.9712484E-3,4.4815158E-4,6.256148E-3,3.489927E-4,-3.4379915E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,23,25,-1,27,29,31,33,35,37,-1,39,41,-1,-1,43,-1,45,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.1942564E-1,3.8120416E-1,3.034013E-1,9.331253E-2,2.6004997E-1,2.1485913E-1,7.6613724E-2,1.1540043E-1,8.091977E-2,1.5328433E-1,2.0395508E-1,0E0,0E0,0E0,0E0,4.2751998E-2,1.3708834E-1,0E0,2.2448033E-2,4.5857936E-2,1.24601826E-1,1.1865622E-1,1.5915759E-1,3.78848E-2,0E0,8.9943185E-2,1.09503716E-1,0E0,0E0,2.87897E-2,0E0,1.09909445E-1,4.8674345E-2,0E0,0E0,7.533533E-2,3.4685247E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,29,29,31,31,32,32,35,35,36,36],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,24,26,-1,28,30,32,34,36,38,-1,40,42,-1,-1,44,-1,46,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.74915E1,4.249921E2,5.9337012E7,1.996866E1,3.3624732E7,6.677E3,6.695E3,1.295E3,3.3E1,1.0837189E-1,8.607021E2,2.0520555E-2,6.724672E-3,-2.6732462E-3,4.341468E-3,4.7335387E-3,1.358226E8,-1.1563775E-3,7.57046E5,1.6726906E11,1.1596293E3,2.5055168E9,4.541179E1,1.6687299E7,-2.3956813E-3,2.2662702E-1,1.0044E4,-1.0340871E-2,-3.7470656E-3,1.1253E4,6.7319716E-3,2.281387E1,1.235105E0,-9.81231E-4,-1.1747378E-2,3.2432194E1,2.6E1,-8.098195E-3,-2.3987603E-3,-2.7728104E-3,2.5127132E-3,3.2421027E-3,-4.3885927E-3,4.518601E-3,5.154818E-4,2.2705756E-6,-6.9745844E-3,3.997342E-4,3.9712484E-3,4.4815158E-4,6.256148E-3,3.489927E-4,-3.4379915E-3],"split_indices":[56,52,45,56,45,9,2,2,3,38,52,0,0,0,0,39,7,0,1,31,55,5,56,45,0,27,29,0,0,9,0,56,39,0,0,56,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.21E2,2E2,2.1E1,8.2E1,1.18E2,1E1,1.1E1,7E1,1.2E1,7.1E1,4.7E1,3E0,7E0,3E0,8E0,2.1E1,4.9E1,4E0,8E0,2.3E1,4.8E1,7E0,4E1,1.2E1,9E0,2.4E1,2.5E1,5E0,3E0,1.2E1,1.1E1,2.1E1,2.7E1,3E0,4E0,1.6E1,2.4E1,8E0,4E0,7E0,1.7E1,3E0,2.2E1,6E0,6E0,1.7E1,4E0,1.1E1,1.6E1,1.1E1,5E0,5E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[9.517996E-4,-5.0611947E-2,8.6051576E-2,-3.539822E-2,-1.6345337E-1,1.3207358E-1,-9.0657E-3,-7.499884E-2,2.3126654E-2,-5.4488357E-2,-1.0016881E-2,1.657986E-2,9.180982E-2,-3.7068095E-2,4.504542E-2,-5.0559454E-2,-1.950191E-1,9.745878E-2,-2.1730753E-2,1.2282317E-3,-6.043273E-3,6.399411E-2,8.135681E-3,3.0352664E-3,-6.2362116E-2,-7.942792E-4,3.1158319E-3,1.1567295E-2,-8.0419034E-2,-1.19569115E-2,-4.627812E-3,1.3222662E-1,-1.0894468E-3,-6.1310357E-3,-3.5356386E-3,1.1368823E-1,1.12526445E-2,5.1738846E-4,-3.3342135E-3,-8.0884533E-4,4.759767E-3,6.2941597E-4,-4.1358494E-3,2.5935827E-3,6.6484753E-3,6.2252075E-4,-3.960894E-3,1.9134841E-3,5.960904E-3,-1.2240034E-3,3.363897E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,27,29,31,33,-1,-1,35,-1,-1,37,-1,-1,39,41,-1,-1,43,-1,-1,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.6534103E-1,2.2940725E-1,3.6452878E-1,2.8448427E-1,1.2400803E-1,5.6912816E-1,4.4096496E-2,2.0426512E-1,1.6883516E-1,6.602646E-2,0E0,0E0,1.3325658E-1,5.7653286E-2,2.0898301E-2,1.1594513E-1,6.1480552E-2,8.526012E-2,7.311925E-2,0E0,0E0,1.0617927E-1,0E0,0E0,2.2857688E-2,0E0,0E0,6.988758E-2,8.009365E-2,0E0,0E0,1.964122E-2,0E0,0E0,5.114077E-2,3.6399007E-2,6.250101E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,24,24,27,27,28,28,31,31,34,34,35,35,36,36],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,28,30,32,34,-1,-1,36,-1,-1,38,-1,-1,40,42,-1,-1,44,-1,-1,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,4.56567E7,6.627576E7,3.9407016E2,9.214292E7,5.0655737E0,1.0252E4,1.1921976E0,8.97956E5,9.1002154E-1,-1.0016881E-2,1.657986E-2,5.206759E3,9.893967E4,5.534958E-1,4.4823305E6,1.9977521E4,1.1424464E3,2.3404005E6,1.2282317E-3,-6.043273E-3,2.1508584E7,8.135681E-3,3.0352664E-3,5.635113E1,-7.942792E-4,3.1158319E-3,7.629506E7,1.3868161E5,-1.19569115E-2,-4.627812E-3,3.5905025E5,-1.0894468E-3,-6.1310357E-3,1.0476415E1,6.9979796E0,5.506676E11,5.1738846E-4,-3.3342135E-3,-8.0884533E-4,4.759767E-3,6.2941597E-4,-4.1358494E-3,2.5935827E-3,6.6484753E-3,6.2252075E-4,-3.960894E-3,1.9134841E-3,5.960904E-3,-1.2240034E-3,3.363897E-3],"split_indices":[52,45,45,52,7,54,9,39,48,27,0,0,52,28,27,45,33,4,48,0,0,45,0,0,56,0,0,12,28,0,0,28,0,0,54,54,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.36E2,8.2E1,1.21E2,1.5E1,5.5E1,2.7E1,7.2E1,4.9E1,7E0,8E0,6E0,4.9E1,1.8E1,9E0,6.1E1,1.1E1,1.8E1,3.1E1,4E0,3E0,4E1,9E0,3E0,1.5E1,3E0,6E0,2E1,4.1E1,4E0,7E0,1.4E1,4E0,3E0,2.8E1,2E1,2E1,3E0,1.2E1,1.6E1,4E0,7E0,3.4E1,5E0,9E0,2.4E1,4E0,7E0,1.3E1,1.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[3.7125556E-3,-2.7559236E-2,1.409128E-1,-1.2975019E-1,-1.3246279E-2,1.6147085E-2,8.700084E-2,-7.824451E-2,-1.0509483E-2,3.0558249E-2,-3.8024873E-2,1.3990024E-1,7.141632E-3,-4.523257E-3,-1.223917E-3,4.5021445E-2,-2.863578E-2,-1.8173282E-1,-2.386108E-2,1.6907175E-1,1.6047229E-3,-2.7064746E-3,4.112805E-3,1.42816E-2,1.0964767E-1,2.0409583E-3,-3.0524414E-3,-1.0719027E-2,-8.3616667E-4,-3.972385E-2,5.595267E-2,2.3854189E-3,1.9034508E-1,2.7028671E-3,-1.8222426E-3,6.0842154E-3,2.2194261E-4,-2.161455E-3,3.9957967E-3,4.6389955E-3,-1.3367017E-3,3.1174882E-3,9.6414825E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,21,-1,-1,23,25,27,29,31,-1,-1,-1,33,35,-1,-1,-1,-1,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.477929E-1,2.615591E-1,5.1737624E-1,1.2097755E-1,1.737429E-1,0E0,1.4685851E-1,2.2491083E-2,0E0,5.0945032E-2,2.032795E-1,5.5379838E-2,1.0695627E-1,0E0,0E0,9.180426E-2,4.5891277E-2,1.0726285E-1,1.2123299E-1,2.6943386E-2,0E0,0E0,0E0,1.011639E-1,5.7765096E-2,0E0,0E0,0E0,0E0,1.3995838E-1,8.298035E-2,0E0,4.6305954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,23,23,24,24,29,29,30,30,32,32],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,22,-1,-1,24,26,28,30,32,-1,-1,-1,34,36,-1,-1,-1,-1,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0692886E3,5.4E0,2.2779044E-3,9.372475E-1,6.4613514E0,1.6147085E-2,3.09483E2,2.3833251E8,-1.0509483E-2,2.4918722E1,7.784166E5,3.2944346E9,1.3978E4,-4.523257E-3,-1.223917E-3,3.215E3,2.325945E9,3.6050353E5,1.0386934E0,8.119423E8,1.6047229E-3,-2.7064746E-3,4.112805E-3,9.360022E5,2.081E3,2.0409583E-3,-3.0524414E-3,-1.0719027E-2,-8.3616667E-4,1E0,2.4025E4,2.3854189E-3,5.0777936E-1,2.7028671E-3,-1.8222426E-3,6.0842154E-3,2.2194261E-4,-2.161455E-3,3.9957967E-3,4.6389955E-3,-1.3367017E-3,3.1174882E-3,9.6414825E-3],"split_indices":[52,54,57,27,53,0,58,7,0,56,45,7,9,0,0,2,5,28,42,7,0,0,0,48,0,0,0,0,0,94,9,0,57,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.79E2,4E1,2.1E1,1.58E2,6E0,3.4E1,1.6E1,5E0,5.7E1,1.01E2,2E1,1.4E1,9E0,7E0,4.6E1,1.1E1,8E0,9.3E1,1.5E1,5E0,8E0,6E0,3.2E1,1.4E1,4E0,7E0,5E0,3E0,7.8E1,1.5E1,3E0,1.2E1,1.7E1,1.5E1,1E1,4E0,7.2E1,6E0,9E0,6E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.3108605E-3,-1.926569E-2,1.2934735E-1,-1.325124E-1,-7.507367E-3,1.902015E-1,-2.3107207E-3,-6.2085565E-2,-9.692738E-3,-2.5938304E-2,5.3362966E-2,2.3364641E-1,6.324467E-2,-3.2529987E-3,4.1302275E-3,-4.871275E-3,-2.2735582E-4,-7.4991793E-3,-1.3158831E-1,9.111744E-2,5.9063237E-5,2.6999816E-1,3.22087E-3,4.15593E-3,3.750631E-5,-7.900317E-2,1.4035341E-2,-9.574572E-3,-6.0008332E-2,1.18279636E-1,-4.170743E-4,2.8726037E-3,-3.1968735E-2,1.24419015E-2,4.5575756E-3,-1.8139066E-3,-7.163744E-3,2.103647E-3,-4.7373326E-4,-8.407879E-4,-4.257339E-3,4.1791273E-4,5.7057366E-3,1.6118748E-4,-3.0801906E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,-1,25,27,29,31,33,-1,-1,-1,35,37,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0057315E-1,2.5467077E-1,3.048932E-1,1.145044E-1,1.9771616E-1,1.286295E-1,1.092166E-1,3.71591E-2,0E0,2.620058E-1,8.25055E-2,8.657527E-2,1.847139E-2,0E0,0E0,0E0,0E0,1.7972612E-1,1.2908295E-1,6.716399E-2,4.2219736E-2,4.0638804E-2,0E0,0E0,0E0,8.179127E-2,8.503938E-2,0E0,2.0044155E-2,3.8998008E-2,0E0,0E0,2.0102128E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,17,17,18,18,19,19,20,20,21,21,25,25,26,26,28,28,29,29,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,-1,26,28,30,32,34,-1,-1,-1,36,38,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4573945E3,5.178571E0,3.09483E2,9.372475E-1,5.819E3,7.794012E7,7.644773E-1,8.591E3,-9.692738E-3,2.9277092E1,1.8905495E6,5.4831944E0,1E1,-3.2529987E-3,4.1302275E-3,-4.871275E-3,-2.2735582E-4,1.8452284E2,1.1328599E3,4.9E1,3.673362E6,2.5501368E9,3.22087E-3,4.15593E-3,3.750631E-5,2.8961487E1,5.1E2,-9.574572E-3,1.6672E4,1.7096153E1,-4.170743E-4,2.8726037E-3,3.1393208E7,1.24419015E-2,4.5575756E-3,-1.8139066E-3,-7.163744E-3,2.103647E-3,-4.7373326E-4,-8.407879E-4,-4.257339E-3,4.1791273E-4,5.7057366E-3,1.6118748E-4,-3.0801906E-3],"split_indices":[52,54,58,27,2,45,34,2,0,56,47,57,8,0,0,0,0,52,4,3,32,7,0,0,0,58,0,0,9,58,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.91E2,3.7E1,1.7E1,1.74E2,2.5E1,1.2E1,1.1E1,6E0,1.34E2,4E1,1.8E1,7E0,7E0,5E0,5E0,6E0,1.15E2,1.9E1,2.3E1,1.7E1,1.4E1,4E0,4E0,3E0,2.6E1,8.9E1,7E0,1.2E1,1.8E1,5E0,5E0,1.2E1,1.1E1,3E0,2E1,6E0,3.6E1,5.3E1,7E0,5E0,3E0,1.5E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.5091602E-3,-5.2964598E-2,5.5549197E-2,-1.20238274E-1,-3.5740044E-2,8.9050606E-2,-1.7482173E-2,-1.614951E-3,-1.4182362E-1,-7.96712E-3,-2.6092608E-2,2.6797643E-1,6.511564E-2,-1.21168844E-1,3.713418E-2,-8.351395E-3,-2.7302336E-3,1.6331729E-2,-5.9134945E-2,1.6332889E-2,5.7392716E-3,-7.6225447E-3,8.996299E-2,1.1716436E-3,-1.688489E-1,1.4182182E-1,-1.1456664E-2,-2.235058E-2,5.6190386E-2,-9.359478E-2,-6.21144E-3,-2.5418536E-3,7.406722E-2,4.9793966E-2,1.2976183E-1,-3.6005883E-3,-9.648664E-3,8.899322E-3,6.091865E-4,-5.7291847E-2,6.8754606E-2,-1.8257499E-3,2.1441397E-3,-2.9282813E-4,5.264965E-3,-4.969079E-3,1.1631629E-3,3.800175E-3,-1.862664E-3,4.957958E-3,-6.964678E-5,5.5435053E-3,3.8914444E-4,-7.655302E-4,6.201073E-3,-2.1177977E-4,-3.4225814E-3,4.8164963E-3,1.6074211E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,19,21,23,25,-1,-1,27,29,-1,-1,31,33,-1,35,37,39,41,43,45,47,-1,49,51,53,-1,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.620241E-1,1.2316617E-1,2.8292203E-1,3.3022404E-2,1.2965661E-1,3.208748E-1,2.1403006E-1,0E0,6.409711E-2,0E0,1.2011872E-1,9.276897E-2,1.2900773E-1,9.7443804E-2,1.2879632E-1,0E0,0E0,5.9845235E-2,8.695792E-2,0E0,0E0,8.809274E-2,7.826865E-2,0E0,3.2381505E-2,7.137248E-2,7.011604E-2,3.4609955E-2,8.635424E-2,9.701505E-2,8.027784E-2,0E0,2.7949188E-2,9.397275E-2,8.042818E-2,0E0,0E0,0E0,0E0,1.5507426E-2,2.1198232E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,11,11,12,12,13,13,14,14,17,17,18,18,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,39,39,40,40],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,20,22,24,26,-1,-1,28,30,-1,-1,32,34,-1,36,38,40,42,44,46,48,-1,50,52,54,-1,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.26031E2,1.142E3,6.627576E7,6.3323677E3,9.224633E-6,6.677E3,1.643614E6,-1.614951E-3,5.95081E4,-7.96712E-3,9.360022E5,4.9695907E0,1E0,3.18947E5,1E0,-8.351395E-3,-2.7302336E-3,7.027618E-1,1.1565725E0,1.6332889E-2,5.7392716E-3,1.1223402E6,1.5061628E11,1.1716436E-3,3.7375356E5,1.2199979E10,1.0377E4,3E0,9.101559E7,4.187737E7,4.0079948E-1,-2.5418536E-3,1E0,3.1808215E10,2.6877522E5,-3.6005883E-3,-9.648664E-3,8.899322E-3,6.091865E-4,2.7864855E11,6.543368E6,-1.8257499E-3,2.1441397E-3,-2.9282813E-4,5.264965E-3,-4.969079E-3,1.1631629E-3,3.800175E-3,-1.862664E-3,4.957958E-3,-6.964678E-5,5.5435053E-3,3.8914444E-4,-7.655302E-4,6.201073E-3,-2.1177977E-4,-3.4225814E-3,4.8164963E-3,1.6074211E-4],"split_indices":[52,2,45,50,38,9,32,0,48,0,48,54,109,29,67,0,0,27,42,0,0,28,31,0,28,5,9,8,7,32,27,0,8,31,28,0,0,0,0,31,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.23E2,1.09E2,1.14E2,2.1E1,8.8E1,7.8E1,3.6E1,5E0,1.6E1,4E0,8.4E1,8E0,7E1,1.2E1,2.4E1,8E0,8E0,3.7E1,4.7E1,3E0,5E0,1.8E1,5.2E1,3E0,9E0,7E0,1.7E1,1.9E1,1.8E1,2.8E1,1.9E1,1.1E1,7E0,2.7E1,2.5E1,5E0,4E0,4E0,3E0,1.1E1,6E0,1.5E1,4E0,1E1,8E0,2.3E1,5E0,5E0,1.4E1,4E0,3E0,8E0,1.9E1,3E0,2.2E1,4E0,7E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[3.3319353E-3,-3.138492E-2,7.22715E-2,-1.5437375E-1,-2.1704765E-2,5.3617395E-2,2.5115296E-1,-3.597831E-3,-1.036556E-2,-1.1395114E-1,-8.3544515E-3,2.1667369E-2,1.5383117E-1,3.7373262E-3,1.4443975E-2,-3.03824E-4,-1.6664231E-1,5.5139232E-3,-1.0404415E-1,-1.9524172E-2,1.0074209E-1,2.0771952E-1,8.445345E-4,-8.3061E-3,-1.991494E-3,8.802538E-2,-5.3766114E-3,-6.5774955E-3,-1.8047873E-3,3.3369195E-3,-3.0737154E-2,3.2725554E-2,6.0905367E-3,4.3428307E-3,1.017467E-2,1.7358974E-3,6.342767E-3,-2.3134856E-3,6.769404E-4,-1.8242977E-3,3.3306594E-3,3.6528073E-3,-1.2160736E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,-1,-1,23,25,27,29,31,33,-1,-1,-1,35,37,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.503488E-1,1.7880362E-1,2.4347314E-1,4.953155E-2,1.7515507E-1,2.2399701E-1,7.878688E-2,0E0,0E0,9.9886E-2,1.6778332E-1,1.8111342E-1,1.17981166E-1,0E0,0E0,0E0,3.88954E-2,1.0021721E-1,4.6190426E-2,4.404693E-2,5.526334E-2,2.1326423E-2,0E0,0E0,0E0,3.267066E-2,1.09717265E-1,0E0,0E0,0E0,5.4100435E-2,3.3670016E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,16,16,17,17,18,18,19,19,20,20,21,21,25,25,26,26,30,30,31,31],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,-1,-1,24,26,28,30,32,34,-1,-1,-1,36,38,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2286531E3,5.178571E0,2.9028345E2,6.3004294E5,2.750917E-4,1.9107767E0,7.65E2,-3.597831E-3,-1.036556E-2,1.57525425E1,1.4100806E1,7.9522475E5,1.2562557E7,3.7373262E-3,1.4443975E-2,-3.03824E-4,1.662926E6,7.751E3,3.0743053E4,9.024829E4,5.5326223E9,1.6837959E3,8.445345E-4,-8.3061E-3,-1.991494E-3,3.7709497E-2,2.037744E10,-6.5774955E-3,-1.8047873E-3,3.3369195E-3,5.11442E5,1.1233E4,6.0905367E-3,4.3428307E-3,1.017467E-2,1.7358974E-3,6.342767E-3,-2.3134856E-3,6.769404E-4,-1.8242977E-3,3.3306594E-3,3.6528073E-3,-1.2160736E-3],"split_indices":[52,54,56,28,39,42,0,0,0,56,54,28,47,0,0,0,1,9,33,28,5,4,0,0,0,57,31,0,0,0,9,9,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,1.52E2,7.6E1,1E1,1.42E2,7E1,6E0,7E0,3E0,1.7E1,1.25E2,5.4E1,1.6E1,3E0,3E0,6E0,1.1E1,1.1E2,1.5E1,3.6E1,1.8E1,1.1E1,5E0,8E0,3E0,1.2E1,9.8E1,7E0,8E0,3E0,3.3E1,8E0,1E1,4E0,7E0,8E0,4E0,2.9E1,6.9E1,3E1,3E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-8.841635E-3,-3.0673463E-2,9.940114E-2,-9.821085E-2,-1.5124879E-2,2.2260913E-1,5.3487994E-2,-1.3850419E-3,-1.1614962E-1,4.171513E-3,-9.502235E-2,4.2785425E-3,1.1782143E-2,-6.1098295E-3,4.5304555E-3,-1.3092218E-1,-1.0327521E-3,-1.958529E-2,4.4014405E-2,-2.0100197E-1,-4.7347855E-2,-2.332966E-3,3.3681672E-2,-1.5297404E-3,-1.5496272E-1,2.0418392E-2,-6.328449E-2,9.305969E-2,2.0387327E-2,-1.1334491E-2,-1.7064379E-3,1.3775668E-3,-7.302717E-2,3.330824E-3,-1.3061083E-4,-7.995631E-3,-3.4290992E-3,-9.801557E-4,3.1051838E-3,-8.8427094E-4,-4.94627E-3,6.5663736E-3,1.3087405E-3,-2.2162867E-3,1.6053565E-3,-5.6506437E-3,-2.0537386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,-1,21,-1,23,-1,25,27,29,31,-1,33,-1,35,37,39,41,43,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3475493E-1,1.959354E-1,2.0305395E-1,3.675139E-2,2.3782323E-1,4.4687033E-2,9.7511254E-2,0E0,3.3100873E-2,1.1917952E-1,1.4205873E-1,0E0,0E0,3.1914886E-2,0E0,4.621303E-2,0E0,1.3939434E-1,5.2566193E-2,9.3652666E-2,4.7994383E-2,0E0,1.5783519E-2,0E0,3.3692718E-2,1.03024915E-1,8.7081894E-2,5.548802E-2,4.7041092E-2,0E0,0E0,0E0,1.8485412E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,13,13,15,15,17,17,18,18,19,19,20,20,22,22,24,24,25,25,26,26,27,27,28,28,32,32],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,-1,22,-1,24,-1,26,28,30,32,-1,34,-1,36,38,40,42,44,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,4.3724646E2,4.0642028E6,1.153E3,5.2829284E7,7.6309943E-1,2.3816228E-1,-1.3850419E-3,1E0,7.608277E2,8.3813794E2,4.2785425E-3,1.1782143E-2,2.1312436E5,4.5304555E-3,1.5133574E1,-1.0327521E-3,5.99E2,8.2852056E5,1.270805E6,6.2947E2,-2.332966E-3,1.0456502E-1,-1.5297404E-3,7.385222E5,3.41043E2,9.222491E0,2.32789E1,8.144481E-2,-1.1334491E-2,-1.7064379E-3,1.3775668E-3,3.05E2,3.330824E-3,-1.3061083E-4,-7.995631E-3,-3.4290992E-3,-9.801557E-4,3.1051838E-3,-8.8427094E-4,-4.94627E-3,6.5663736E-3,1.3087405E-3,-2.2162867E-3,1.6053565E-3,-5.6506437E-3,-2.0537386E-3],"split_indices":[52,55,47,29,45,27,38,0,112,52,52,0,0,33,0,58,0,0,47,29,33,0,38,0,51,52,54,56,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E2,1.87E2,3.7E1,3.4E1,1.53E2,9E0,2.8E1,8E0,2.6E1,1.24E2,2.9E1,4E0,5E0,1.4E1,1.4E1,2.2E1,4E0,7.8E1,4.6E1,8E0,2.1E1,6E0,8E0,5E0,1.7E1,4.1E1,3.7E1,1.4E1,3.2E1,5E0,3E0,5E0,1.6E1,3E0,5E0,1E1,7E0,2.3E1,1.8E1,2.2E1,1.5E1,6E0,8E0,6E0,2.6E1,3E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[7.1345344E-3,-3.5421792E-2,5.3077098E-2,-2.8964793E-2,-8.911729E-3,-2.390216E-3,9.384826E-2,-8.262742E-2,-5.987002E-3,6.284614E-2,-4.36269E-2,-4.0996682E-2,1.138439E-1,-8.1849955E-3,-6.137525E-2,1.2069097E-2,-9.583986E-2,-4.4093298E-4,4.851195E-3,-7.4983224E-2,1.0996893E-3,-5.504462E-3,1.1258086E-3,2.3964514E-1,9.395485E-2,1.6921503E-2,-1.1886299E-1,-1.350116E-2,4.661281E-2,-8.561863E-3,-1.1100321E-2,-4.7043577E-2,-5.6668664E-3,1.3349842E-2,3.9983927E-3,7.196686E-2,1.5100017E-1,-1.297399E-3,3.5793923E-3,-5.598809E-5,-5.8854027E-3,-1.1874547E-3,5.55624E-3,5.6063533E-3,7.7507395E-4,2.0671254E-3,-4.133846E-3,-3.7139882E-3,-8.047736E-4,3.6459381E-3,-2.1670596E-3,7.9802545E-3,3.620102E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,-1,25,27,29,-1,-1,31,-1,-1,-1,33,35,37,39,41,43,-1,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5156386E-1,1.3324031E-1,2.512471E-1,1.4377199E-1,0E0,1.3186806E-1,1.7723113E-1,7.357296E-2,1.3581525E-1,7.726345E-2,6.804009E-2,6.014351E-2,1.17599964E-1,0E0,1.4154553E-1,6.2459745E-2,1.2954906E-1,0E0,0E0,2.9937826E-2,0E0,0E0,0E0,5.1958293E-2,5.184859E-2,4.976772E-2,4.9327135E-2,9.600385E-2,7.18195E-2,0E0,5.381948E-2,1.711921E-2,0E0,0E0,0E0,8.103064E-2,1.7711908E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,35,35,36,36],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,-1,26,28,30,-1,-1,32,-1,-1,-1,34,36,38,40,42,44,-1,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3021045E2,6.0073395E8,1E0,4.3724646E2,-8.911729E-3,1.096122E6,1.08017064E8,5.657236E0,3.8293724E7,5.4732365E9,9.4908775E1,1.2286531E3,6.677E3,-8.1849955E-3,8.110325E0,6.212876E-1,3.864E3,-4.4093298E-4,4.851195E-3,1.4233672E8,1.0996893E-3,-5.504462E-3,1.1258086E-3,1.884E3,5.318116E3,1.729367E6,1.614724E2,1E0,3.04144E2,-8.561863E-3,1.09137096E8,4.0492815E-1,-5.6668664E-3,1.3349842E-2,3.9983927E-3,5.8229775E3,1.373141E7,-1.297399E-3,3.5793923E-3,-5.598809E-5,-5.8854027E-3,-1.1874547E-3,5.55624E-3,5.6063533E-3,7.7507395E-4,2.0671254E-3,-4.133846E-3,-3.7139882E-3,-8.047736E-4,3.6459381E-3,-2.1670596E-3,7.9802545E-3,3.620102E-3],"split_indices":[52,7,109,55,0,29,7,53,45,5,56,52,9,0,54,27,2,0,0,45,0,0,0,0,52,45,55,79,52,0,7,27,0,0,0,4,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.29E2,1.19E2,1.1E2,1.16E2,3E0,4.7E1,6.3E1,3.4E1,8.2E1,1.8E1,2.9E1,8E0,5.5E1,4E0,3E1,6.9E1,1.3E1,8E0,1E1,2E1,9E0,3E0,5E0,6E0,4.9E1,1.3E1,1.7E1,4E1,2.9E1,5E0,8E0,1.5E1,5E0,3E0,3E0,3.7E1,1.2E1,8E0,5E0,3E0,1.4E1,3.7E1,3E0,6E0,2.3E1,5E0,3E0,5E0,1E1,3.3E1,4E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.0294736E-2,-1.5248241E-2,1.0214141E-1,-6.9946777E-3,-8.685983E-3,1.46258995E-2,7.124872E-2,-1.4353992E-2,1.09981954E-1,1.5286076E-1,3.2093324E-2,-8.611879E-2,-4.5119557E-3,1.0143571E-3,5.9197485E-3,1.8230141E-3,7.94113E-3,-2.6876405E-2,1.0561916E-1,-5.0785523E-3,-4.085596E-2,9.752067E-3,-5.944501E-2,1.6302239E-3,-1.7688652E-3,6.5497877E-3,2.4298427E-3,-3.7026124E-3,2.6873115E-4,1.4332411E-3,-1.3471033E-3,1.3979974E-3,-3.525995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,-1,-1,-1,-1,23,25,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.176857E-1,2.76577E-1,3.476376E-1,1.4534362E-1,0E0,0E0,1.356985E-1,1.112087E-1,2.4798036E-2,5.1106334E-2,1.3724843E-1,2.6503712E-2,1.1027899E-1,0E0,0E0,0E0,0E0,2.0637559E-2,2.5298133E-2,0E0,2.361912E-2,1.19120255E-1,7.262408E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,-1,-1,-1,-1,24,26,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3526E4,3.0532868E6,2.2779044E-3,2.00347E6,-8.685983E-3,1.46258995E-2,3.0437586E7,1.7398648E2,9.1797715E-1,2.8244882E6,1.7595624E0,5.0515676E-1,3.2271808E7,1.0143571E-3,5.9197485E-3,1.8230141E-3,7.94113E-3,1E0,5.6431256E5,-5.0785523E-3,3.2217744E5,9.357682E0,2.325945E9,1.6302239E-3,-1.7688652E-3,6.5497877E-3,2.4298427E-3,-3.7026124E-3,2.6873115E-4,1.4332411E-3,-1.3471033E-3,1.3979974E-3,-3.525995E-3],"split_indices":[2,28,57,28,0,0,50,52,27,32,41,27,45,0,0,0,0,67,28,0,32,54,5,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.72E2,4.7E1,1.66E2,6E0,4E0,4.3E1,1.57E2,9E0,1.3E1,3E1,1.8E1,1.39E2,3E0,6E0,4E0,9E0,1.7E1,1.3E1,9E0,9E0,1.11E2,2.8E1,3E0,1.4E1,5E0,8E0,4E0,5E0,7E1,4.1E1,6E0,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[6.977331E-3,-3.3416852E-2,7.1899265E-2,-2.3109617E-2,-1.6509996E-1,9.428345E-2,-3.332324E-2,-3.568124E-2,6.32931E-2,-9.987793E-3,-1.673859E-3,1.986663E-1,6.56623E-2,-3.6460035E-3,3.250998E-2,-5.1937334E-2,6.2865512E-3,1.0017583E-1,-8.695741E-4,5.6440453E-3,1.3885838E-2,4.6908118E-2,1.5099156E-1,3.4154162E-3,-5.8061525E-4,-3.8493954E-2,-9.019757E-3,3.14585E-2,-4.642573E-3,6.75163E-4,5.793906E-3,8.849409E-2,-8.151666E-3,8.432195E-3,2.3733475E-3,-7.6468557E-4,-3.3164409E-3,3.9113034E-3,-9.0061047E-4,1.6430327E-3,5.0712423E-3,-2.2521857E-3,2.6791194E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,-1,-1,-1,31,33,-1,-1,35,-1,37,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9508055E-1,1.8620504E-1,2.0805463E-1,1.4455348E-1,8.53852E-2,2.0205921E-1,6.060464E-2,7.8783214E-2,5.497144E-2,0E0,0E0,8.2086265E-2,8.573353E-2,0E0,1.9797634E-2,1.7694965E-1,1.0197952E-1,3.785032E-2,0E0,0E0,0E0,1.1291738E-1,3.7781626E-2,0E0,0E0,6.356305E-2,0E0,9.609632E-2,0E0,0E0,0E0,4.169981E-2,7.7788346E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,21,21,22,22,25,25,27,27,31,31,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,-1,-1,-1,32,34,-1,-1,36,-1,38,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,9.996903E7,9.179278E-1,9.32784E5,6.677E3,2.0814263E9,1.1565725E0,2.5933685E-2,-9.987793E-3,-1.673859E-3,9.98127E6,5.2086E4,-3.6460035E-3,2.012E3,9.3026364E-1,3.0530536E0,3.1496124E2,-8.695741E-4,5.6440453E-3,1.3885838E-2,3.291238E7,8.0426145E-1,3.4154162E-3,-5.8061525E-4,1.4573639E6,-9.019757E-3,1.1592E4,-4.642573E-3,6.75163E-4,5.793906E-3,7.582143E0,1.1392155E6,8.432195E-3,2.3733475E-3,-7.6468557E-4,-3.3164409E-3,3.9113034E-3,-9.0061047E-4,1.6430327E-3,5.0712423E-3,-2.2521857E-3,2.6791194E-3],"split_indices":[52,5,45,27,29,9,7,42,38,0,0,1,2,0,0,39,42,33,0,0,0,45,27,0,0,51,0,10,0,0,0,54,28,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.25E2,1.39E2,8.6E1,1.3E2,9E0,7.1E1,1.5E1,1.14E2,1.6E1,5E0,4E0,1.4E1,5.7E1,8E0,7E0,8.2E1,3.2E1,1.1E1,5E0,1.1E1,3E0,4.8E1,9E0,3E0,4E0,7.7E1,5E0,2.7E1,5E0,4E0,7E0,2.7E1,2.1E1,5E0,4E0,5.3E1,2.4E1,1.2E1,1.5E1,1.2E1,1.5E1,1.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.8733817E-3,-4.0591124E-2,5.0054505E-2,-3.3112444E-2,-8.644342E-3,3.723186E-2,9.3282815E-3,-6.288404E-2,1.0094166E-2,6.437276E-2,-2.1287613E-2,-1.4725652E-1,-4.2888757E-2,6.0631927E-2,-2.6850218E-2,2.1883616E-2,1.13919765E-1,-4.8870735E-2,2.2246495E-3,-9.462059E-3,-8.5627586E-2,-1.3005011E-1,-2.4225444E-2,-5.0990615E-4,7.347648E-2,2.3241812E-2,-6.9002606E-2,4.271089E-3,-2.3492752E-2,4.702195E-2,1.3961242E-1,-5.1400047E-3,-2.9403567E-2,-1.4145136E-3,-5.8479817E-3,-7.907182E-3,-2.376875E-3,7.4882957E-4,-2.4565565E-3,4.202733E-3,1.0600353E-3,2.780319E-3,-1.9271369E-3,2.000493E-3,-4.1286377E-3,-2.8840103E-4,-3.2377315E-3,3.3950456E-3,-7.1601965E-4,3.211028E-3,7.348271E-3,7.410126E-4,-2.8562914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,21,23,25,27,29,31,-1,-1,33,35,37,-1,39,41,43,-1,45,47,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2826423E-1,1.4670894E-1,1.9632854E-1,1.5242165E-1,0E0,1.3796169E-1,0E0,1.1177799E-1,9.311521E-2,1.2171438E-1,6.0604252E-2,4.5694232E-2,9.119921E-2,2.0585217E-2,6.283721E-2,1.2426527E-1,3.9842308E-2,2.7164456E-2,0E0,0E0,2.0968862E-2,3.3369496E-2,7.399846E-2,0E0,2.1617703E-2,4.6116423E-2,6.267011E-2,0E0,1.9363433E-2,2.1714104E-2,3.0997157E-2,0E0,3.5002615E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,24,24,25,25,26,26,28,28,29,29,30,30,32,32],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,22,24,26,28,30,32,-1,-1,34,36,38,-1,40,42,44,-1,46,48,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,7.511677E7,9.472401E8,7.555544E2,-8.644342E-3,6.971004E7,9.3282815E-3,2.1293039E5,1.00149225E3,3.7808718E11,3.511154E7,1.9977521E4,5.657236E0,2.85E2,6.611028E0,1.20171E5,5.0777936E-1,1.2286531E3,2.2246495E-3,-9.462059E-3,6.0274365E2,1.8243903E0,1.4149E4,-5.0990615E-4,2.0971161E6,9.01799E5,2.0200117E-1,4.271089E-3,3.6511216E7,2.3E1,3.7697E4,-5.1400047E-3,1.256E3,-1.4145136E-3,-5.8479817E-3,-7.907182E-3,-2.376875E-3,7.4882957E-4,-2.4565565E-3,4.202733E-3,1.0600353E-3,2.780319E-3,-1.9271369E-3,2.000493E-3,-4.1286377E-3,-2.8840103E-4,-3.2377315E-3,3.3950456E-3,-7.1601965E-4,3.211028E-3,7.348271E-3,7.410126E-4,-2.8562914E-3],"split_indices":[52,45,32,4,0,45,0,28,55,31,32,33,53,0,53,29,57,52,0,0,4,56,9,0,48,29,27,0,50,8,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.21E2,9E1,1.17E2,4E0,8.5E1,5E0,6.9E1,4.8E1,5.8E1,2.7E1,1.2E1,5.7E1,2E1,2.8E1,3.2E1,2.6E1,2E1,7E0,4E0,8E0,9E0,4.8E1,3E0,1.7E1,1.3E1,1.5E1,1.1E1,2.1E1,8E0,1.8E1,3E0,1.7E1,5E0,3E0,4E0,5E0,2.2E1,2.6E1,1E1,7E0,8E0,5E0,3E0,1.2E1,1.7E1,4E0,5E0,3E0,8E0,1E1,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[2.65665E-3,-2.0276988E-2,1.154926E-1,-2.8532773E-2,9.559364E-2,1.624391E-2,7.2594196E-2,-6.202233E-2,-2.3150512E-3,1.3323977E-3,6.0114376E-3,1.447959E-1,-3.640516E-3,-4.4701416E-2,-1.917737E-1,6.0712315E-2,-3.4721565E-2,6.895913E-3,1.0939657E-3,-1.9573853E-3,2.9682724E-3,-4.8907483E-3,-2.4567625E-2,-1.1047363E-2,-3.2320432E-3,1.5749253E-2,1.3559216E-1,-9.153247E-2,7.334384E-3,-2.9608398E-3,4.8767406E-4,1.8591287E-3,-2.3744125E-3,1.2082297E-3,6.808371E-3,-6.5324837E-3,-2.2240041E-3,-1.8039207E-3,1.6399455E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,29,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.975218E-1,1.8639532E-1,4.352433E-1,1.5789998E-1,3.6294594E-2,0E0,1.9317678E-1,1.6897303E-1,2.1044274E-1,0E0,0E0,4.225999E-2,6.2671475E-2,1.0220404E-1,5.6135803E-2,1.1570349E-1,1.6313942E-1,0E0,0E0,0E0,0E0,0E0,9.826048E-2,0E0,0E0,5.1695615E-2,3.5291985E-2,6.422126E-2,6.7566946E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,22,22,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,30,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,1E0,1.4164306E-3,3.0743053E4,9.596726E2,1.624391E-2,3.077965E2,2.1843149E8,7.34E2,1.3323977E-3,6.0114376E-3,1.3424457E0,2.8159826E0,2.8246236E-1,1.5484602E5,3.464E3,3.4741312E8,6.895913E-3,1.0939657E-3,-1.9573853E-3,2.9682724E-3,-4.8907483E-3,4.1749332E4,-1.1047363E-2,-3.2320432E-3,8.445123E-1,2.9E1,1.23428725E-1,1E0,-2.9608398E-3,4.8767406E-4,1.8591287E-3,-2.3744125E-3,1.2082297E-3,6.808371E-3,-6.5324837E-3,-2.2240041E-3,-1.8039207E-3,1.6399455E-3],"split_indices":[52,94,57,33,4,0,58,7,0,0,0,35,35,27,47,2,7,0,0,0,0,0,32,0,0,27,3,39,109,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.29E2,1.91E2,3.8E1,1.79E2,1.2E1,4E0,3.4E1,7.8E1,1.01E2,6E0,6E0,1.7E1,1.7E1,7E1,8E0,3.4E1,6.7E1,1.4E1,3E0,1.1E1,6E0,1.4E1,5.6E1,4E0,4E0,2.2E1,1.2E1,2.8E1,3.9E1,2.4E1,3.2E1,1.6E1,6E0,3E0,9E0,9E0,1.9E1,1.5E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.440677E-3,-1.8581178E-2,7.800658E-2,-1.3833836E-1,-1.231606E-2,1.0023959E-2,5.6294505E-2,-2.2783056E-3,-8.022323E-3,-7.252014E-2,-2.2669341E-3,1.0176534E-1,4.6810898E-4,-1.1043568E-1,-3.9880738E-2,-1.9851184E-2,4.5394495E-2,-1.5621152E-3,1.2077981E-1,-2.4189243E-2,3.0761007E-3,-5.867465E-3,-1.8358874E-3,1.2749977E-3,-2.8291706E-3,1.335456E-1,-3.3062056E-2,-2.3804004E-3,6.1289985E-2,1.6487682E-1,6.294307E-2,-1.8002669E-3,2.2481934E-3,1.6048751E-3,7.216552E-3,-5.3965114E-3,-9.2451036E-4,2.9776935E-3,-8.027103E-4,8.108114E-3,3.1383673E-3,3.7427014E-3,2.6063828E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,-1,-1,-1,33,35,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8898027E-1,1.3559982E-1,1.7900926E-1,3.0979276E-2,1.0571032E-1,0E0,1.24337524E-1,0E0,0E0,2.653426E-2,1.2745279E-1,7.619348E-2,4.3705486E-2,1.6524121E-2,3.2346234E-2,2.2935745E-1,7.0613556E-2,0E0,5.083236E-2,2.969253E-2,0E0,0E0,0E0,0E0,0E0,2.8116167E-2,1.0455079E-1,0E0,3.24609E-2,2.1150738E-2,1.814475E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,25,25,26,26,28,28,29,29,30,30],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,-1,-1,-1,34,36,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.23072E3,2.3754893E2,5.491E3,6.974669E7,1.292E3,1.0023959E-2,5.7025972E7,-2.2783056E-3,-8.022323E-3,1.51E2,8.129243E5,5.22E2,1.08747E5,6.5273724E0,7.89E2,1.994E4,3.0117606E8,-1.5621152E-3,6.4613514E0,8.129243E5,3.0761007E-3,-5.867465E-3,-1.8358874E-3,1.2749977E-3,-2.8291706E-3,1.9777294E5,2.1424272E6,-2.3804004E-3,4.24975E5,3.6E1,1.5156659E7,-1.8002669E-3,2.2481934E-3,1.6048751E-3,7.216552E-3,-5.3965114E-3,-9.2451036E-4,2.9776935E-3,-8.027103E-4,8.108114E-3,3.1383673E-3,3.7427014E-3,2.6063828E-4],"split_indices":[52,33,9,7,2,0,45,0,0,0,28,0,2,54,2,29,5,0,53,28,0,0,0,0,0,47,45,0,29,3,1,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,1.82E2,5.3E1,8E0,1.74E2,5E0,4.8E1,4E0,4E0,2.4E1,1.5E2,2.6E1,2.2E1,1E1,1.4E1,1.1E2,4E1,3E0,2.3E1,1.7E1,5E0,6E0,4E0,4E0,1E1,8E0,1.02E2,5E0,3.5E1,1.2E1,1.1E1,1.4E1,3E0,3E0,5E0,9E0,9.3E1,3.1E1,4E0,8E0,4E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.5668424E-3,-4.0050495E-2,4.4704918E-2,-2.6343586E-2,-1.1224846E-1,2.1334768E-4,8.737615E-2,-9.338947E-2,-1.1046934E-2,-8.264241E-3,-1.3490978E-2,-2.8898086E-2,8.995582E-2,1.9985068E-1,6.0555495E-2,-1.0261289E-3,-1.3132946E-1,4.8071142E-2,-4.948862E-2,-3.1322013E-3,2.642493E-3,2.1593561E-3,-4.956114E-2,1.2656973E-1,-6.1695784E-4,2.2980438E-3,1.25827715E-2,-3.14866E-3,7.3348425E-2,-7.610914E-3,-2.7894531E-3,7.603607E-2,-5.2319404E-2,-7.1339846E-2,2.1552747E-2,-6.466943E-2,-2.9465224E-4,2.250835E-3,7.173506E-3,2.5992317E-2,8.933302E-2,5.226086E-3,1.9941868E-3,4.4996844E-4,-3.8228065E-3,-6.8649654E-3,-2.1958102E-3,-1.3436867E-3,3.7232984E-3,2.2404728E-4,-3.1447064E-3,2.1042582E-3,-1.2516868E-3,2.423883E-3,4.9760677E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,-1,29,31,33,-1,-1,-1,35,37,-1,-1,-1,-1,39,-1,-1,41,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.845248E-1,1.167859E-1,1.8223833E-1,1.0460282E-1,1.6155027E-1,1.280011E-1,1.3571331E-1,4.6181485E-2,1.9554985E-1,0E0,5.2982837E-2,6.4589754E-2,4.7964238E-2,1.1858928E-1,7.726872E-2,0E0,2.7969345E-2,9.943028E-2,8.225933E-2,0E0,0E0,0E0,1.8774405E-2,2.098544E-2,0E0,0E0,0E0,0E0,2.6442826E-2,0E0,0E0,3.1703323E-2,2.127567E-2,6.0289904E-2,5.1010355E-2,1.7913975E-2,0E0,0E0,0E0,1.6931672E-2,1.9596666E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,22,22,23,23,28,28,31,31,32,32,33,33,34,34,35,35,39,39,40,40],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,-1,30,32,34,-1,-1,-1,36,38,-1,-1,-1,-1,40,-1,-1,42,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.744699E2,3.2271808E7,5.0798903E5,1.295E3,1.2534044E3,6.2019708E7,5.0655737E0,5.322348E7,1.2117E4,-8.264241E-3,7.343501E-1,2.1637352E7,2.0127554E10,9.717001E5,5.4907064E0,-1.0261289E-3,8.95E2,8.712121E0,6.3004294E5,-3.1322013E-3,2.642493E-3,2.1593561E-3,5.864617E3,4.5510876E9,-6.1695784E-4,2.2980438E-3,1.25827715E-2,-3.14866E-3,2.030662E1,-7.610914E-3,-2.7894531E-3,2.0565E4,3E1,7.784166E5,1.4202555E1,1.6958537E9,-2.9465224E-4,2.250835E-3,7.173506E-3,2E0,5.0777936E-1,5.226086E-3,1.9941868E-3,4.4996844E-4,-3.8228065E-3,-6.8649654E-3,-2.1958102E-3,-1.3436867E-3,3.7232984E-3,2.2404728E-4,-3.1447064E-3,2.1042582E-3,-1.2516868E-3,2.423883E-3,4.9760677E-3],"split_indices":[52,45,28,2,4,32,54,5,9,0,27,45,12,28,54,0,2,53,28,0,0,0,4,5,0,0,0,0,56,0,0,29,3,45,56,5,0,0,0,8,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.2E2,9.5E1,1.02E2,1.8E1,4.7E1,4.8E1,1.8E1,8.4E1,9E0,9E0,3.6E1,1.1E1,8E0,4E1,7E0,1.1E1,3.3E1,5.1E1,5E0,4E0,7E0,2.9E1,8E0,3E0,4E0,4E0,3E0,3.7E1,5E0,6E0,2.6E1,7E0,3.9E1,1.2E1,2.1E1,8E0,4E0,4E0,1E1,2.7E1,8E0,1.8E1,3E0,4E0,5E0,3.4E1,7E0,5E0,3E0,1.8E1,7E0,3E0,1.5E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[7.0114112E-3,-1.5631918E-2,1.01542346E-1,-4.4183638E-2,9.3284855E-3,1.3882309E-2,7.623148E-2,-5.9544537E-2,6.59629E-2,9.387126E-2,-3.4115668E-3,2.4972217E-2,1.1936574E-1,-7.456421E-2,1.9459507E-3,2.1721087E-2,5.706747E-3,-6.0496636E-4,5.945049E-3,-3.7527204E-2,2.7811658E-2,4.622265E-3,-4.7155554E-4,1.4745439E-3,1.4189242E-1,1.809964E-2,-8.743557E-2,3.330273E-3,-1.2265785E-3,2.2348673E-3,-5.135943E-2,5.6775462E-2,-4.2446714E-2,6.534976E-3,2.0733434E-3,3.2709858E-3,-3.221862E-3,-4.89699E-3,-2.0247693E-3,-2.7805413E-3,1.7281079E-3,1.0920535E-3,4.7624195E-3,1.3577103E-3,-2.7284583E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,-1,-1,-1,29,31,-1,-1,-1,33,35,37,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8747966E-1,1.3154206E-1,2.3507458E-1,1.4943543E-1,1.069511E-1,0E0,8.7671414E-2,1.2517533E-1,3.203951E-2,7.2405875E-2,9.371612E-2,6.43911E-2,3.6432594E-2,8.149409E-2,0E0,2.6546836E-2,0E0,0E0,0E0,5.5962916E-2,9.601762E-2,0E0,0E0,0E0,1.7176509E-2,5.8831967E-2,6.447628E-2,0E0,0E0,0E0,5.7454027E-2,5.5453204E-2,2.694018E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,19,19,20,20,24,24,25,25,26,26,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,-1,-1,-1,30,32,-1,-1,-1,34,36,38,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2737E4,3.1433902E4,1.4164306E-3,1E0,4.6376264E9,1.3882309E-2,4.0827688E5,1E0,7.1299434E0,1.153E3,3.103785E8,2.7778377E11,5.0777936E-1,1.347836E2,1.9459507E-3,1.2E2,5.706747E-3,-6.0496636E-4,5.945049E-3,7.4500206E5,1.8905495E6,4.622265E-3,-4.7155554E-4,1.4745439E-3,2.936137E7,1.746849E6,1.01978E5,3.330273E-3,-1.2265785E-3,2.2348673E-3,7.2838904E7,1.0527294E6,2.7350403E5,6.534976E-3,2.0733434E-3,3.2709858E-3,-3.221862E-3,-4.89699E-3,-2.0247693E-3,-2.7805413E-3,1.7281079E-3,1.0920535E-3,4.7624195E-3,1.3577103E-3,-2.7284583E-3],"split_indices":[2,33,57,79,31,0,28,112,54,29,7,31,57,51,0,10,0,0,0,48,47,0,0,0,47,1,29,0,0,0,32,28,28,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.83E2,4.3E1,8.5E1,9.8E1,3E0,4E1,7.5E1,1E1,1.2E1,8.6E1,1.9E1,2.1E1,6.6E1,9E0,7E0,3E0,4E0,8E0,4.1E1,4.5E1,5E0,1.4E1,5E0,1.6E1,8E0,5.8E1,3E0,4E0,5E0,3.6E1,3.2E1,1.3E1,1.3E1,3E0,5E0,3E0,3.1E1,2.7E1,3.1E1,5E0,2.2E1,1E1,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.1998726E-3,-1.2652648E-2,1.0993669E-1,-5.596024E-2,2.9656107E-3,1.5855557E-1,4.7277976E-4,-1.05173446E-1,-2.6812853E-2,-7.0219305E-3,1.0346495E-2,1.3947075E-2,1.0035766E-1,-3.8118362E-2,-7.25749E-3,5.943483E-4,-9.264957E-2,-2.071006E-2,4.6499595E-2,5.994588E-3,1.789368E-3,3.8944127E-5,-3.5235318E-3,3.0719542E-3,-2.1535639E-2,-6.1006024E-3,-1.3038092E-3,9.1751866E-2,-4.1153643E-2,-1.7017646E-2,7.118743E-2,5.2157213E-4,-2.0938904E-3,5.1705586E-3,-2.4740022E-4,-6.0827243E-3,-1.0688449E-3,-3.0358834E-3,1.2679735E-3,4.459029E-3,6.775775E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,19,21,-1,23,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.566273E-1,1.3273825E-1,1.2749979E-1,7.177068E-2,1.8660066E-1,1.5326154E-1,0E0,8.283998E-2,6.1404668E-2,0E0,1.5815371E-1,0E0,3.1894937E-2,2.029991E-2,0E0,4.2355835E-2,2.8868325E-2,1.7850941E-1,1.03138134E-1,0E0,0E0,0E0,0E0,0E0,2.0498134E-2,0E0,0E0,3.9733507E-2,1.01614036E-1,5.330746E-2,9.252939E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,12,12,13,13,15,15,16,16,17,17,18,18,24,24,27,27,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,20,22,-1,24,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.4788145E3,8.882435E7,6.627576E7,4.543028E2,4.994744E-5,5.870237E0,4.7277976E-4,2.237415E1,1.7129352E3,-7.0219305E-3,4.6048853E5,1.3947075E-2,2.7113E4,6.357E3,-7.25749E-3,7.275E3,1.8020761E1,3.7844E4,1.3012535E1,5.994588E-3,1.789368E-3,3.8944127E-5,-3.5235318E-3,3.0719542E-3,1.3585858E1,-6.1006024E-3,-1.3038092E-3,1.5489E4,2.9540545E6,1.8347148E6,5.9122612E1,5.2157213E-4,-2.0938904E-3,5.1705586E-3,-2.4740022E-4,-6.0827243E-3,-1.0688449E-3,-3.0358834E-3,1.2679735E-3,4.459029E-3,6.775775E-4],"split_indices":[52,7,45,4,42,53,0,58,4,0,28,0,2,29,0,9,58,29,56,0,0,0,0,0,56,0,0,9,45,32,58,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.21E2,1.95E2,2.6E1,5.1E1,1.44E2,1.7E1,9E0,1.8E1,3.3E1,5E0,1.39E2,3E0,1.4E1,1E1,8E0,2.4E1,9E0,7.5E1,6.4E1,7E0,7E0,6E0,4E0,5E0,1.9E1,4E0,5E0,1.1E1,6.4E1,1.8E1,4.6E1,9E0,1E1,8E0,3E0,7E0,5.7E1,8E0,1E1,2.7E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.2736395E-3,-1.532366E-2,1.0377453E-1,-9.288727E-2,-8.348644E-3,1.46567505E-2,7.026136E-2,-1.5684154E-3,-8.076746E-3,2.622898E-2,-2.5716571E-2,1.1663213E-1,2.513948E-2,-5.7890904E-3,4.1411016E-2,-9.1588266E-2,-7.865047E-3,7.1898825E-3,1.5607044E-3,-1.2677099E-3,3.8905889E-3,6.0656533E-2,-9.295811E-4,-3.8414698E-2,-7.2781257E-3,5.7987638E-2,-1.7768428E-2,9.005109E-4,-2.6599471E-3,7.113441E-5,3.5155274E-3,-4.330562E-3,1.0243556E-3,-1.6776705E-3,4.6466608E-3,2.4000012E-4,-2.252464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,19,-1,21,23,25,-1,-1,27,-1,29,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2667904E-1,9.82289E-2,2.7737564E-1,7.7002406E-2,1.0216606E-1,0E0,6.611504E-2,0E0,0E0,1.4959952E-1,1.321575E-1,6.5626055E-2,3.2875247E-2,0E0,6.652436E-2,1.0089129E-1,5.9725136E-2,0E0,0E0,2.1729311E-2,0E0,5.864133E-2,0E0,6.9583915E-2,0E0,6.881592E-2,6.862143E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,21,21,23,23,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,20,-1,22,24,26,-1,-1,28,-1,30,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,5.178571E0,1.4164306E-3,3.804992E5,6.3058443E0,1.46567505E-2,7.9684106E9,-1.5684154E-3,-8.076746E-3,3.8396814E-5,3.0727175E6,8.315045E0,1.08747E5,-5.7890904E-3,1.3237829E7,1.555368E8,1.0937031E5,7.1898825E-3,1.5607044E-3,1.0129378E12,3.8905889E-3,1.725E3,-9.295811E-4,2.8246236E-1,-7.2781257E-3,2.4065792E6,4.91606E6,9.005109E-4,-2.6599471E-3,7.113441E-5,3.5155274E-3,-4.330562E-3,1.0243556E-3,-1.6776705E-3,4.6466608E-3,2.4000012E-4,-2.252464E-3],"split_indices":[52,54,57,28,53,0,12,0,0,38,45,54,2,0,50,5,28,0,0,31,0,2,0,27,0,32,32,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.82E2,3.5E1,1.4E1,1.68E2,3E0,3.2E1,1E1,4E0,5.6E1,1.12E2,1.5E1,1.7E1,4E0,5.2E1,2.3E1,8.9E1,8E0,7E0,1.3E1,4E0,4E1,1.2E1,1.5E1,8E0,1.1E1,7.8E1,1E1,3E0,1.2E1,2.8E1,7E0,8E0,4E0,7E0,4.8E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.568711E-3,-2.1280082E-2,9.298501E-2,-1.00754015E-1,-1.2219225E-2,1.3955258E-2,5.5418063E-2,-2.0836093E-3,-7.443147E-3,1.9015184E-2,-2.9586928E-2,1.0140864E-1,7.5694744E-4,-3.0091228E-3,3.423799E-2,-6.8711704E-3,-6.709425E-2,1.224652E-1,-3.3795467E-4,-1.9191427E-3,2.465467E-3,-3.717263E-2,4.775927E-2,2.3133606E-2,-6.0604922E-2,-1.1506553E-1,-5.4924856E-3,2.1019075E-3,6.222166E-3,-3.057035E-3,1.1257252E-3,3.2183127E-3,-2.6044628E-4,-1.5514655E-3,2.5353355E-3,1.2344114E-3,-3.753484E-3,-6.9364663E-3,-2.1865922E-3,7.580766E-4,-4.1115293E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,19,-1,21,23,25,27,-1,-1,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2133E-1,1.3342792E-1,3.3933583E-1,6.67343E-2,9.224002E-2,0E0,8.790961E-2,0E0,0E0,8.769642E-2,9.240779E-2,4.4840068E-2,4.993182E-2,0E0,5.295017E-2,1.12648584E-1,1.20527744E-1,2.4400994E-2,0E0,0E0,0E0,2.3294423E-2,7.3563635E-2,1.0641703E-1,7.084138E-2,6.6419184E-2,4.431591E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,20,-1,22,24,26,28,-1,-1,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,5.178571E0,1.4164306E-3,9.372475E-1,6.4613514E0,1.3955258E-2,3.09483E2,-2.0836093E-3,-7.443147E-3,6.324075E-5,1.7186E4,7.251039E0,2.0390862E-1,-3.0091228E-3,1.661E3,3.203027E7,3E0,5.0777936E-1,-3.3795467E-4,-1.9191427E-3,2.465467E-3,4.9658173E2,1.552E3,5.156382E2,2.1434378E5,5.246288E-1,2.1637352E7,2.1019075E-3,6.222166E-3,-3.057035E-3,1.1257252E-3,3.2183127E-3,-2.6044628E-4,-1.5514655E-3,2.5353355E-3,1.2344114E-3,-3.753484E-3,-6.9364663E-3,-2.1865922E-3,7.580766E-4,-4.1115293E-3],"split_indices":[52,54,57,27,53,0,58,0,0,38,9,57,38,0,2,45,8,57,0,0,0,4,0,4,28,27,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E2,1.86E2,3.8E1,1.8E1,1.68E2,4E0,3.4E1,1.2E1,6E0,6E1,1.08E2,1.8E1,1.6E1,8E0,5.2E1,6.8E1,4E1,1.5E1,3E0,9E0,7E0,8E0,4.4E1,4.4E1,2.4E1,2.2E1,1.8E1,5E0,1E1,5E0,3E0,2.8E1,1.6E1,1.7E1,2.7E1,6E0,1.8E1,1.1E1,1.1E1,1.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.6589627E-3,-2.1119064E-2,6.474576E-2,-3.1263284E-2,4.4856135E-2,2.5053513E-1,4.0365916E-2,-9.097914E-2,-1.8236281E-2,8.3813064E-2,-8.574259E-4,4.3675085E-3,1.3769066E-2,-4.7966177E-3,5.140204E-2,-2.4254324E-3,-5.5280253E-3,3.84755E-2,-3.3285294E-2,4.913043E-3,-1.1834134E-4,4.708339E-3,7.73571E-2,4.8790043E-3,7.1719303E-3,-8.193875E-2,-1.5174835E-2,-3.264524E-2,7.874288E-2,9.6890956E-2,-1.1829342E-2,1.2553375E-3,-1.5260734E-3,-1.7028786E-3,-7.483322E-3,-4.104721E-5,-3.717414E-3,9.716918E-4,-2.7484458E-3,4.984086E-3,7.1621663E-4,4.6354714E-3,1.4635278E-3,-2.613698E-3,1.7573558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,-1,-1,23,25,-1,-1,27,29,-1,31,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3058262E-1,1.0982361E-1,2.7071688E-1,1.080274E-1,5.7945896E-2,4.838577E-2,1.025697E-1,2.6658922E-2,1.0113506E-1,4.2756125E-2,0E0,0E0,0E0,0E0,6.386848E-2,0E0,0E0,6.1346725E-2,8.07023E-2,0E0,0E0,5.7879675E-2,6.0379446E-2,0E0,2.0521617E-2,9.343E-2,7.3002696E-2,2.8120771E-2,1.5863605E-2,2.215901E-2,2.2208398E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,17,17,18,18,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,-1,-1,24,26,-1,-1,28,30,-1,32,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.667E3,1E0,1.4164306E-3,6.126268E0,3.1240498E1,3.907438E6,1.2721619E6,8.351808E-1,6.3274317E0,2.6519718E1,-8.574259E-4,4.3675085E-3,1.3769066E-2,-4.7966177E-3,5.6865673E0,-2.4254324E-3,-5.5280253E-3,5.492906E8,3.7103668E6,4.913043E-3,-1.1834134E-4,5E1,3.138649E6,4.8790043E-3,1.7747324E7,1.555368E8,6.1108776E7,8.286144E-1,1.096E3,1E0,3.167407E7,1.2553375E-3,-1.5260734E-3,-1.7028786E-3,-7.483322E-3,-4.104721E-5,-3.717414E-3,9.716918E-4,-2.7484458E-3,4.984086E-3,7.1621663E-4,4.6354714E-3,1.4635278E-3,-2.613698E-3,1.7573558E-3],"split_indices":[2,79,57,54,58,1,32,27,53,56,0,0,0,0,54,0,0,5,45,0,0,3,29,0,45,5,45,27,0,89,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.61E2,6.1E1,1.4E2,2.1E1,6E0,5.5E1,2.4E1,1.16E2,1.3E1,8E0,3E0,3E0,3E0,5.2E1,1.5E1,9E0,2.4E1,9.2E1,9E0,4E0,1.9E1,3.3E1,6E0,1.8E1,2.4E1,6.8E1,1.3E1,6E0,2.7E1,6E0,1.2E1,6E0,1.8E1,6E0,5.8E1,1E1,5E0,8E0,3E0,3E0,2.1E1,6E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.8445231E-3,-1.6023695E-2,9.7475566E-2,-8.1381686E-2,-7.292331E-3,1.3341884E-2,5.7568017E-2,-2.4789309E-3,-6.962572E-3,-2.4714854E-2,2.0900577E-2,1.1176834E-1,-1.04630245E-2,1.3923284E-2,-5.4713473E-2,1.027107E-1,1.1836274E-3,1.9672303E-3,6.3697337E-3,-1.6967354E-3,1.6760122E-3,5.0293733E-3,-4.3511544E-3,-8.786125E-2,-1.510692E-2,6.009691E-3,1.8809016E-3,-6.2729223E-3,1.291673E-2,3.910284E-4,-3.6712855E-3,-4.444348E-3,-5.482267E-4,8.4690854E-4,-1.5714215E-3,-3.1372635E-3,1.4832029E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,29,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1560764E-1,1.11256786E-1,2.3290971E-1,3.2813028E-2,8.5983895E-2,0E0,9.335205E-2,0E0,0E0,1.259436E-1,1.0795251E-1,3.0626878E-2,2.0353362E-2,9.496336E-2,7.87577E-2,2.5071174E-2,1.0058065E-1,0E0,0E0,0E0,0E0,0E0,4.9770612E-2,4.6776503E-2,2.438683E-2,0E0,0E0,0E0,1.0858913E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,30,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.206759E3,1.292E3,1.4164306E-3,9.2664665E-1,5.324658E5,1.3341884E-2,3.09483E2,-2.4789309E-3,-6.962572E-3,7.34E2,6.891E3,3.9880952E-1,2.3816228E-1,1.994E4,3.4288502E8,5.294574E0,5.050378E0,1.9672303E-3,6.3697337E-3,-1.6967354E-3,1.6760122E-3,5.0293733E-3,2.325945E9,7.257E3,2.1637352E7,6.009691E-3,1.8809016E-3,-6.2729223E-3,4.1749332E4,3.910284E-4,-3.6712855E-3,-4.444348E-3,-5.482267E-4,8.4690854E-4,-1.5714215E-3,-3.1372635E-3,1.4832029E-3],"split_indices":[52,2,57,27,28,0,58,0,0,0,9,57,38,29,7,53,53,0,0,0,0,0,5,2,45,0,0,0,32,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,1.95E2,2.7E1,2.2E1,1.73E2,3E0,2.4E1,1.9E1,3E0,1.07E2,6.6E1,1.3E1,1.1E1,4.7E1,6E1,1.2E1,5.4E1,6E0,7E0,7E0,4E0,6E0,4.1E1,3.2E1,2.8E1,6E0,6E0,3E0,5.1E1,3.6E1,5E0,2.5E1,7E0,1.1E1,1.7E1,1E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.553252E-3,-1.6703485E-2,7.270691E-2,-3.2296632E-3,-7.918737E-2,1.2927953E-2,4.9774267E-2,-2.601383E-2,4.6371344E-2,1.8034107E-3,-9.389973E-2,6.55655E-3,8.408618E-2,-1.704064E-2,-7.987716E-3,9.372017E-2,2.8495318E-3,-5.8800397E-3,-5.4320708E-2,3.452301E-3,-6.8034447E-4,1.05185755E-1,6.123724E-4,-5.0504366E-3,-8.680789E-3,1.7311171E-3,4.762819E-3,-3.9009284E-2,2.8833196E-2,-7.9006046E-2,1.8887174E-3,6.564539E-4,1.35951E-1,2.1330388E-3,-9.26758E-4,8.621999E-4,-3.4079612E-3,2.311046E-3,-1.0563099E-3,3.5761532E-4,-4.289572E-3,6.6089663E-3,2.528749E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,-1,-1,31,-1,-1,33,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0112103E-1,1.4707617E-1,2.484254E-1,1.6507336E-1,6.009303E-2,0E0,6.617839E-2,1.4663586E-1,9.452546E-2,0E0,4.5744613E-2,3.852059E-2,3.4988716E-2,8.324712E-2,0E0,1.962234E-2,2.8315896E-2,0E0,4.5563478E-2,0E0,0E0,5.0051644E-2,0E0,0E0,7.556125E-2,0E0,0E0,2.7678244E-2,2.4943762E-2,3.1194866E-2,0E0,0E0,1.5179604E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,18,18,21,21,24,24,27,27,28,28,29,29,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,-1,-1,32,-1,-1,34,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2737E4,3.203027E7,1.4164306E-3,5.6477E2,2.6E1,1.2927953E-2,3.7988422E5,5.138372E8,1.0397707E6,1.8034107E-3,5.36E3,2.7778377E11,6.126689E0,1.0135136E0,-7.987716E-3,2.757E3,1.2936785E3,-5.8800397E-3,1.1392155E6,3.452301E-3,-6.8034447E-4,3.9880952E-1,6.123724E-4,-5.0504366E-3,1.1E2,1.7311171E-3,4.762819E-3,7.4809E4,2.6391107E-1,1.448907E6,1.8887174E-3,6.564539E-4,7.5253453E9,2.1330388E-3,-9.26758E-4,8.621999E-4,-3.4079612E-3,2.311046E-3,-1.0563099E-3,3.5761532E-4,-4.289572E-3,6.6089663E-3,2.528749E-3],"split_indices":[2,45,57,52,3,0,28,7,47,0,2,31,57,56,0,2,4,0,28,0,0,57,0,0,10,0,0,29,38,1,0,0,12,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.21E2,1.74E2,4.7E1,1.44E2,3E1,3E0,4.4E1,9.9E1,4.5E1,3E0,2.7E1,2E1,2.4E1,9.5E1,4E0,2.1E1,2.4E1,1.1E1,1.6E1,4E0,1.6E1,1.8E1,6E0,6E0,8.9E1,7E0,1.4E1,9E0,1.5E1,1.3E1,3E0,5E0,1.3E1,1.6E1,7.3E1,4E0,5E0,1E1,5E0,3E0,1E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-4.7051087E-3,-3.8700417E-2,3.766359E-2,-3.2101016E-2,-7.7233785E-3,2.5987374E-2,1.657726E-1,-2.1549348E-2,-1.23511635E-1,1.665832E-2,9.2962785E-3,2.026071E-3,9.351031E-3,-3.0286148E-2,7.694984E-2,1.4953944E-3,-7.4769426E-3,6.024506E-2,-1.7672503E-2,-5.7858497E-2,-1.2161433E-2,3.8097968E-4,4.52073E-3,7.078176E-3,4.268607E-2,-3.5685524E-2,2.6736919E-2,1.1334919E-3,-2.8288616E-3,1.6459684E-3,-1.0087847E-3,3.3213529E-3,2.8100255E-4,-5.4452666E-3,-9.503511E-4,2.1430857E-3,-1.3967515E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,21,-1,-1,23,25,27,29,-1,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1689137E-1,1.1316995E-1,1.4228322E-1,1.1115347E-1,0E0,1.6497748E-1,4.5251653E-2,9.441127E-2,1.1619696E-1,1.3274512E-1,0E0,0E0,0E0,4.877071E-2,1.8706858E-2,0E0,0E0,6.861569E-2,4.095152E-2,3.687702E-2,3.971204E-2,0E0,0E0,0E0,4.6695948E-2,4.088383E-2,2.4501488E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,14,14,17,17,18,18,19,19,20,20,24,24,25,25,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,22,-1,-1,24,26,28,30,-1,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.0073395E8,2.7553815E2,4.462437E7,-7.7233785E-3,1.4964736E4,7.65E2,1E0,9.214292E7,4.22372E5,9.2962785E-3,2.026071E-3,9.351031E-3,4.543028E2,1.338E3,1.4953944E-3,-7.4769426E-3,5.0655737E0,9.5891304E1,3.327841E2,7.751E3,3.8097968E-4,4.52073E-3,7.078176E-3,8.479005E6,2.19E2,3.632724E6,1.1334919E-3,-2.8288616E-3,1.6459684E-3,-1.0087847E-3,3.3213529E-3,2.8100255E-4,-5.4452666E-3,-9.503511E-4,2.1430857E-3,-1.3967515E-3],"split_indices":[52,7,56,45,0,55,0,79,7,29,0,0,0,4,2,0,0,54,56,33,9,0,0,0,48,10,29,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,1.21E2,9.7E1,1.17E2,4E0,9E1,7E0,1.06E2,1.1E1,8.7E1,3E0,3E0,4E0,9.8E1,8E0,3E0,8E0,3.8E1,4.9E1,3.8E1,6E1,3E0,5E0,4E0,3.4E1,3.5E1,1.4E1,4E0,3.4E1,1.1E1,4.9E1,1.6E1,1.8E1,3E0,3.2E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[5.6398376E-3,-7.75812E-3,6.661004E-2,4.543229E-3,-5.7751246E-2,8.350648E-3,4.221645E-2,-6.3478515E-2,1.7897563E-2,-1.5242553E-1,-2.6378147E-2,5.3319256E-3,9.349932E-2,-1.7699042E-2,-9.281823E-2,3.5737775E-2,-2.179834E-2,-2.188771E-3,-7.998782E-3,-1.660155E-3,1.9719824E-3,3.0307057E-3,-3.8265454E-4,5.4542804E-3,1.7977287E-3,-3.208759E-3,5.9767725E-4,-6.6104685E-3,-2.2310102E-3,7.888394E-2,1.3193322E-2,4.3991946E-2,-5.156532E-2,-4.0010092E-4,4.1956347E-3,-2.7085035E-3,1.1856991E-3,3.52371E-3,-7.5539656E-4,-3.0737668E-3,1.1281059E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8594766E-1,1.1526418E-1,1.2675619E-1,1.3830227E-1,1.0587763E-1,0E0,6.759706E-2,3.193613E-2,9.07183E-2,2.6434138E-2,3.145039E-2,2.2471266E-2,2.3058295E-2,2.2292055E-2,3.0452102E-2,8.455743E-2,8.060358E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.086795E-2,7.368022E-2,3.4890782E-2,5.2223958E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,29,29,30,30,31,31,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,3.607011E1,2.2779044E-3,1.295E3,4.2845196E1,8.350648E-3,5.324658E5,3.0799932E3,7.739782E0,4.1733156E7,7.929336E0,4.5510876E9,8.795753E3,7.5E0,5.95081E4,9.360022E5,4.573327E5,-2.188771E-3,-7.998782E-3,-1.660155E-3,1.9719824E-3,3.0307057E-3,-3.8265454E-4,5.4542804E-3,1.7977287E-3,-3.208759E-3,5.9767725E-4,-6.6104685E-3,-2.2310102E-3,3.2834557E-1,1.385431E8,1E0,5.7563637E1,-4.0010092E-4,4.1956347E-3,-2.7085035E-3,1.1856991E-3,3.52371E-3,-7.5539656E-4,-3.0737668E-3,1.1281059E-3],"split_indices":[52,56,57,2,56,0,28,47,53,45,53,5,4,58,48,48,48,0,0,0,0,0,0,0,0,0,0,0,0,27,7,68,58,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.26E2,1.86E2,4E1,1.5E2,3.6E1,5E0,3.5E1,2.4E1,1.26E2,8E0,2.8E1,2.1E1,1.4E1,1E1,1.4E1,8.7E1,3.9E1,3E0,5E0,2.4E1,4E0,3E0,1.8E1,7E0,7E0,3E0,7E0,4E0,1E1,2.9E1,5.8E1,1.2E1,2.7E1,6E0,2.3E1,9E0,4.9E1,7E0,5E0,2.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.0979734E-3,-2.5215901E-2,3.8677156E-2,-1.6001917E-2,-1.473148E-1,1.4230476E-2,1.0264718E-1,2.7897488E-4,-4.5919813E-2,-1.5711497E-3,-8.09341E-3,-9.718648E-3,6.046874E-2,9.357595E-3,2.826945E-3,-6.2386498E-2,1.8178968E-2,-1.1109469E-1,-1.0625226E-2,-2.002652E-2,4.175685E-3,-9.6861814E-4,7.9360984E-2,-9.204167E-2,2.6647064E-3,4.3590378E-2,-3.116004E-2,-1.5383307E-3,-6.4628487E-3,-3.5537314E-2,4.8683558E-2,-3.0095829E-3,-3.4415894E-3,4.709629E-3,1.1743217E-3,-2.2454285E-3,-6.482379E-3,9.819221E-4,4.705359E-3,-3.4042988E-3,1.6293573E-4,-3.058479E-3,-2.386159E-5,3.8313786E-3,-1.2158657E-3,2.1469744E-3,-1.5527981E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,-1,-1,33,35,-1,37,39,-1,-1,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1852314E-1,1.4389493E-1,1.4085005E-1,5.9437796E-2,4.0132254E-2,7.4781194E-2,9.0522915E-2,9.0871036E-2,9.7733356E-2,0E0,0E0,5.335274E-2,3.83078E-2,0E0,0E0,7.361592E-2,8.034602E-2,4.4231474E-2,4.4577744E-2,3.647793E-2,0E0,0E0,3.0171245E-2,2.8120011E-2,0E0,5.5799976E-2,3.981012E-2,0E0,0E0,2.8148634E-2,3.359607E-2,0E0,6.42183E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,22,22,23,23,25,25,26,26,29,29,30,30,32,32],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,-1,-1,34,36,-1,38,40,-1,-1,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,1.5995066E0,2.0749E4,1E0,7.9522475E5,6.677E3,6.010101E0,2.9751791E4,-1.5711497E-3,-8.09341E-3,2.3274304E2,2.030662E1,9.357595E-3,2.826945E-3,1E0,1.6533424E6,2.8671232E1,3.37353E5,5.806435E3,4.175685E-3,-9.6861814E-4,1.1233E4,8.721346E5,2.6647064E-3,2.0526677E8,3.2378372E2,-1.5383307E-3,-6.4628487E-3,2.495E3,1.9E1,-3.0095829E-3,3.748732E6,4.709629E-3,1.1743217E-3,-2.2454285E-3,-6.482379E-3,9.819221E-4,4.705359E-3,-3.4042988E-3,1.6293573E-4,-3.058479E-3,-2.386159E-5,3.8313786E-3,-1.2158657E-3,2.1469744E-3,-1.5527981E-3],"split_indices":[52,5,42,9,64,28,9,54,33,0,0,56,56,0,0,79,50,58,9,33,0,0,9,28,0,7,52,0,0,2,8,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.29E2,9E1,1.21E2,8E0,6.6E1,2.4E1,7.9E1,4.2E1,3E0,5E0,4.4E1,2.2E1,4E0,2E1,1.7E1,6.2E1,1.4E1,2.8E1,4.1E1,3E0,4E0,1.8E1,1.4E1,3E0,4.1E1,2.1E1,6E0,8E0,2E1,8E0,9E0,3.2E1,1E1,8E0,1E1,4E0,3.3E1,8E0,8E0,1.3E1,9E0,1.1E1,5E0,3E0,1.2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.8196059E-3,-1.698803E-2,6.39607E-2,-4.2101737E-2,-2.9341197E-3,8.520983E-3,4.5997523E-2,-6.638161E-2,-8.9270686E-4,3.37814E-3,-5.737205E-3,7.97034E-2,7.755276E-3,-1.2712072E-1,-3.1122798E-2,-6.7898706E-2,4.3837473E-2,3.4373194E-2,-2.0172598E-2,5.4663416E-2,4.5981244E-3,-6.108972E-4,5.7463072E-2,-7.566134E-3,-3.1358576E-3,-4.3154396E-2,6.690998E-4,-3.8722353E-3,-2.454148E-4,-3.0672504E-4,7.8829534E-2,1.664577E-2,7.9456784E-2,5.188494E-2,-3.6731772E-2,3.4943498E-3,4.4557538E-7,3.8003507E-3,3.6522213E-6,-5.764978E-4,-3.0570084E-3,4.5266584E-3,5.470394E-4,2.1260905E-3,-7.871907E-4,4.5588445E-3,-7.148562E-4,-6.217921E-4,3.6080666E-3,-9.5848157E-4,-5.5670873E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,-1,-1,39,-1,-1,-1,-1,41,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.772084E-1,6.0936097E-2,1.3339536E-1,6.1974846E-2,9.644576E-2,0E0,6.5120295E-2,7.956563E-2,7.490907E-2,7.952869E-2,0E0,1.5188158E-2,2.883808E-2,2.6209727E-2,1.5493764E-2,1.53757E-2,2.7485857E-2,3.624052E-2,7.588506E-2,2.7695E-2,0E0,0E0,1.5784737E-2,0E0,0E0,1.7835326E-2,0E0,0E0,0E0,0E0,1.6619794E-2,4.4369165E-2,4.0574744E-2,2.9686522E-2,6.455581E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,25,25,30,30,31,31,32,32,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,-1,-1,40,-1,-1,-1,-1,42,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7545184E3,4.3117376E8,2.2779044E-3,3.6E1,3.0532868E6,8.520983E-3,5.7025972E7,2.6239506E5,6.2734294E0,7.47E2,-5.737205E-3,7.327557E9,4.078932E5,7.4517044E5,3.9743054E-1,1.5217391E-2,1.7114774E6,9.1487586E-1,2.2E1,4.5510876E9,4.5981244E-3,-6.108972E-4,7.424212E-1,-7.566134E-3,-3.1358576E-3,1.4507717E5,6.690998E-4,-3.8722353E-3,-2.454148E-4,-3.0672504E-4,1.8465776E8,1.9540394E7,2.7577372E11,8.017964E0,7.511677E7,3.4943498E-3,4.4557538E-7,3.8003507E-3,3.6522213E-6,-5.764978E-4,-3.0570084E-3,4.5266584E-3,5.470394E-4,2.1260905E-3,-7.871907E-4,4.5588445E-3,-7.148562E-4,-6.217921E-4,3.6080666E-3,-9.5848157E-4,-5.5670873E-3],"split_indices":[52,5,57,3,28,0,45,28,54,0,0,5,28,51,38,57,45,27,3,5,0,0,42,0,0,47,0,0,0,0,5,45,31,54,45,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.27E2,1.72E2,5.5E1,6.1E1,1.11E2,5E0,5E1,3.8E1,2.3E1,1.07E2,4E0,2.6E1,2.4E1,1.3E1,2.5E1,9E0,1.4E1,4.6E1,6.1E1,1.6E1,1E1,1.7E1,7E0,5E0,8E0,2E1,5E0,6E0,3E0,6E0,8E0,3.4E1,1.2E1,1.1E1,5E1,1E1,6E0,4E0,3E0,1.1E1,9E0,5E0,3E0,1.7E1,1.7E1,9E0,3E0,4E0,7E0,4.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[2.4328413E-3,-2.6882904E-2,3.378449E-2,-1.9866189E-2,-1.243878E-1,2.4863081E-2,1.24080844E-1,8.1230905E-3,-4.949995E-2,-1.2541146E-3,-7.229486E-3,3.825448E-2,-3.052416E-2,2.5274062E-3,8.409009E-3,-6.374502E-2,3.1381756E-2,2.550993E-3,-6.160796E-2,2.9172627E-2,6.327835E-3,-5.6614753E-2,-6.861234E-5,-5.3805397E-3,-1.5026209E-3,4.5848332E-2,-2.3069053E-3,-1.3556094E-2,-9.4792396E-2,-3.871335E-2,4.2922344E-2,-7.1197405E-4,-4.1555334E-3,7.870943E-4,3.9008448E-3,2.7850852E-3,-1.235644E-3,-4.689615E-3,-1.2219618E-3,-2.5295615E-3,5.7510444E-4,3.5400738E-3,9.4936317E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,-1,27,29,-1,31,-1,-1,-1,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.16896E-1,8.099689E-2,8.8750035E-2,9.5786266E-2,3.2615E-2,7.904752E-2,3.7462384E-2,1.02293804E-1,7.917659E-2,0E0,0E0,8.409303E-2,1.5721582E-2,0E0,0E0,2.2555154E-2,6.0551412E-2,0E0,8.0038264E-2,7.6334715E-2,0E0,1.727366E-2,0E0,0E0,0E0,5.0186202E-2,0E0,3.1041726E-2,3.3013284E-2,1.7377546E-2,5.511576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,18,18,19,19,21,21,25,25,27,27,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,-1,28,30,-1,32,-1,-1,-1,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.9080975E2,4.462437E7,1.4964736E4,1.5225E4,5.673844E5,6.995407E6,2.9997022E5,5.8930116E0,1.5E1,-1.2541146E-3,-7.229486E-3,5.804289E3,1.8790482E9,2.5274062E-3,8.409009E-3,8.95E2,9.104811E0,2.550993E-3,6.5833335E0,1.09141096E8,6.327835E-3,3.733336E2,-6.861234E-5,-5.3805397E-3,-1.5026209E-3,2.746E3,-2.3069053E-3,3.06123E5,1.5E1,1.4362E4,8.41E2,-7.1197405E-4,-4.1555334E-3,7.870943E-4,3.9008448E-3,2.7850852E-3,-1.235644E-3,-4.689615E-3,-1.2219618E-3,-2.5295615E-3,5.7510444E-4,3.5400738E-3,9.4936317E-4],"split_indices":[52,45,55,9,28,47,33,54,3,0,0,52,7,0,0,2,53,0,53,7,0,58,0,0,0,2,0,12,8,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,1.21E2,1.13E2,1.14E2,7E0,1.04E2,9E0,5.9E1,5.5E1,3E0,4E0,8.4E1,2E1,6E0,3E0,1.4E1,4.5E1,5E0,5E1,7.9E1,5E0,1E1,1E1,3E0,1.1E1,3.9E1,6E0,2.1E1,2.9E1,1.3E1,6.6E1,6E0,4E0,2.6E1,1.3E1,3E0,1.8E1,2.2E1,7E0,9E0,4E0,2E1,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.464617E-4,-1.2114681E-2,4.788821E-2,-1.00445114E-1,-8.121581E-3,9.1912635E-3,3.2035027E-2,-7.1648066E-3,-9.1625267E-4,2.530969E-3,-4.5468777E-2,4.015971E-3,1.4074184E-2,-3.815242E-2,1.6842917E-2,2.3403207E-3,-6.166294E-2,-4.1339782E-4,4.13713E-2,-1.8435845E-2,-3.7041034E-3,6.717887E-2,1.9050402E-3,-1.6912986E-2,-9.437975E-2,3.9210073E-3,-1.2375496E-3,-2.3582424E-3,7.774576E-4,3.4638292E-3,-1.511381E-3,-1.0686071E-3,9.138175E-4,-1.608253E-3,2.428299E-3,-5.219763E-3,-7.152132E-4,-2.6056117E-3,1.2324154E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,17,19,21,-1,23,-1,25,27,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4203872E-1,6.514143E-2,1.3104485E-1,4.1736633E-2,7.17268E-2,0E0,5.3544268E-2,0E0,0E0,8.2741044E-2,6.958933E-2,0E0,2.5378672E-2,3.5839725E-2,7.88228E-2,0E0,4.9646407E-2,0E0,4.108502E-2,4.1281402E-2,0E0,4.4270724E-2,4.648296E-2,2.8856607E-2,4.6853215E-2,0E0,2.2954138E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,12,12,13,13,14,14,16,16,18,18,19,19,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,18,20,22,-1,24,-1,26,28,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2737E4,7.784166E5,4.1749332E4,1.9977521E4,3.203027E7,9.1912635E-3,1.3888E4,-7.1648066E-3,-9.1625267E-4,5.156382E2,1.6592433E9,4.015971E-3,4.0827688E5,1.5195648E6,1.0949861E1,2.3403207E-3,1E0,-4.1339782E-4,2.1958594E5,1.7398648E2,-3.7041034E-3,1.2429E4,5.6477E2,1.4332E4,7.6336784E1,3.9210073E-3,1.7330067E-1,-2.3582424E-3,7.774576E-4,3.4638292E-3,-1.511381E-3,-1.0686071E-3,9.138175E-4,-1.608253E-3,2.428299E-3,-5.219763E-3,-7.152132E-4,-2.6056117E-3,1.2324154E-3],"split_indices":[2,45,32,33,45,0,10,0,0,4,5,0,28,50,56,0,109,0,33,52,0,10,52,9,56,0,38,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,1.86E2,4.9E1,7E0,1.79E2,3E0,4.6E1,3E0,4E0,1.4E2,3.9E1,9E0,3.7E1,3.6E1,1.04E2,5E0,3.4E1,2E1,1.7E1,2.7E1,9E0,2.3E1,8.1E1,1.5E1,1.9E1,7E0,1E1,1.3E1,1.4E1,2E1,3E0,3.4E1,4.7E1,1.2E1,3E0,1.3E1,6E0,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-6.4535546E-3,-1.3688807E-2,7.4542195E-2,-5.7845503E-2,-5.4392144E-3,1.11458436E-1,2.1657532E-4,-8.4732495E-2,3.0981213E-5,-1.6500955E-2,2.4021547E-2,-6.938286E-4,1.5160505E-1,-4.7745116E-2,-1.3174032E-1,-3.0335406E-2,1.5842915E-2,-7.809355E-2,3.9038595E-2,7.946699E-3,3.0066974E-3,8.618174E-4,-7.457685E-2,-6.70451E-3,-1.8911918E-3,-1.2534401E-1,-2.0386945E-2,-4.648951E-2,4.866947E-2,-4.927276E-3,-7.2602154E-4,4.7370974E-2,-2.4941564E-3,-1.6192833E-3,-5.2451333E-3,-7.830434E-3,7.930791E-5,6.761512E-4,-1.5890244E-3,-5.334968E-3,-6.255618E-4,2.463014E-3,-1.3008446E-3,2.591851E-3,-7.170276E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,33,-1,-1,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4283837E-1,8.091864E-2,5.0587296E-2,5.5203468E-2,6.198908E-2,6.9467455E-2,0E0,3.543876E-2,0E0,6.2230032E-2,8.278895E-2,0E0,1.799962E-2,2.9155038E-2,1.868561E-2,8.9127965E-2,8.8201284E-2,1.5293319E-2,4.0929377E-2,0E0,0E0,0E0,1.5389383E-2,0E0,0E0,7.360828E-2,5.9559334E-2,3.6856502E-2,2.6260681E-2,0E0,0E0,4.3612994E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,26,26,27,27,28,28,31,31],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,34,-1,-1,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.8940155E1,1.4164306E-3,7.588906E-2,1.2153278E-3,8.373168E5,6.695E3,2.1657532E-4,3.8E1,3.0981213E-5,2.1360543E-1,1.38E2,-6.938286E-4,1.5213319E0,6.804E3,1.2286531E3,2.4790902E6,1.5675428E5,1.3581458E8,8.039419E6,7.946699E-3,3.0066974E-3,8.618174E-4,5.8E2,-6.70451E-3,-1.8911918E-3,6.09E2,1.0851249E5,3.5623631E0,8.445123E-1,-4.927276E-3,-7.2602154E-4,2.709E4,-2.4941564E-3,-1.6192833E-3,-5.2451333E-3,-7.830434E-3,7.930791E-5,6.761512E-4,-1.5890244E-3,-5.334968E-3,-6.255618E-4,2.463014E-3,-1.3008446E-3,2.591851E-3,-7.170276E-4],"split_indices":[56,57,38,39,28,2,0,3,0,38,0,0,57,9,52,45,28,7,47,0,0,0,0,0,0,0,47,56,27,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.22E2,1.9E1,3.4E1,1.88E2,1.2E1,7E0,2.3E1,1.1E1,1.37E2,5.1E1,3E0,9E0,1.4E1,9E0,9.6E1,4.1E1,6E0,4.5E1,5E0,4E0,4E0,1E1,6E0,3E0,8E0,8.8E1,1.4E1,2.7E1,3E0,3E0,4.2E1,3E0,7E0,3E0,5E0,3E0,2.9E1,5.9E1,3E0,1.1E1,2.4E1,3E0,3.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.671373E-3,-1.2260128E-3,7.1498905E-3,-4.627279E-3,7.5596776E-3,-6.741428E-2,1.0751997E-3,-3.5879017E-5,-5.5757337E-3,-4.271804E-2,1.0038423E-2,1.9977854E-3,-2.400311E-3,-5.7447303E-2,9.2253776E-4,2.947432E-3,8.122457E-2,-2.9972002E-3,1.0787596E-3,1.8954923E-3,-2.0898628E-4,-8.520098E-5,4.6205116E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,-1,7,9,11,-1,13,15,-1,-1,17,-1,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4811061E-1,1.3788432E-1,0E0,7.7409446E-2,0E0,8.17076E-2,7.854202E-2,3.0829096E-2,0E0,3.3842653E-2,8.3554134E-2,0E0,0E0,3.6987677E-2,0E0,5.2363627E-2,3.829813E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,9,9,10,10,13,13,15,15,16,16],"right_children":[2,4,-1,6,-1,8,10,12,-1,14,16,-1,-1,18,-1,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[6.5772717E8,3.8375797E2,7.1498905E-3,6.794276E-5,7.5596776E-3,8.8284516E-1,8.882435E7,1.57727E3,-5.5757337E-3,4.2E1,2.00347E6,1.9977854E-3,-2.400311E-3,1E0,9.2253776E-4,1.26747945E5,9.1797715E-1,-2.9972002E-3,1.0787596E-3,1.8954923E-3,-2.0898628E-4,-8.520098E-5,4.6205116E-3],"split_indices":[32,56,0,42,0,58,7,4,0,3,28,0,0,112,0,47,27,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,2.18E2,4E0,2.15E2,3E0,1.7E1,1.98E2,9E0,8E0,3.3E1,1.65E2,5E0,4E0,2.7E1,6E0,1.51E2,1.4E1,2.3E1,4E0,2.3E1,1.28E2,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2358004E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.4967669E-2,-2.1997929E-1,3.5349327E-1,-3.603856E-1,-1.3046402E-1,5.2479666E-1,-2.9239764E-3,-4.866211E-1,-2.7781805E-1,-3.759615E-2,-4.5592237E-1,3.4534794E-1,8.2649803E-1,-1.4859952E-1,1.35614E-1,-2.4728978E-2,-1.3890923E-2,-9.2154704E-2,-3.822819E-1,1.3083331E-1,-1.0551934E-1,-7.749887E-3,-5.876975E-1,2.822971E-2,2.716092E-1,5.78141E-2,6.523633E-1,1.7711538E-3,-2.0104906E-1,2.1510456E-2,-3.9866533E-2,-1.1387159E-2,-6.097911E-4,-7.3999115E-3,-1.9270817E-2,-1.5337057E-3,1.0451775E-2,-2.9124357E-3,-1.7506724E-2,-1.2541317E-2,-3.3729676E-2,4.347807E-3,1.6137244E-2,3.2968387E-2,1.6719718E-2,-2.5100373E-3,-1.148159E-2,4.681266E-3,-7.308228E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,37,-1,39,-1,41,-1,43,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1483139E1,1.8759646E0,6.9618587E0,4.7976208E-1,2.8541985E0,3.8833447E0,7.870527E-1,4.333353E-2,7.017851E-1,8.7182546E-1,6.9693565E-1,8.208046E-1,1.27355E0,1.941557E-1,1.1823692E0,0E0,0E0,1.8039349E-1,1.2561774E-1,3.5487393E-1,6.029036E-1,0E0,4.400792E-1,0E0,5.3995824E-1,0E0,1.4527988E-1,0E0,9.26286E-2,0E0,2.5168407E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,22,22,24,24,26,26,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,38,-1,40,-1,42,-1,44,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,2.704891E2,6.627576E7,1.142E3,3.8293724E7,3.5078225E3,3.4573945E3,1.0437422E8,4.57E2,5.2183847E5,2.7060036E11,6.518218E6,6.677E3,1.0897255E3,1E0,-2.4728978E-2,-1.3890923E-2,4.1749332E4,1.3567E4,4.3062625E2,2.4079119E1,-7.749887E-3,9.214292E7,2.822971E-2,5.0123274E-1,5.78141E-2,5.7025972E7,1.7711538E-3,7.632764E7,2.1510456E-2,3.733336E2,-1.1387159E-2,-6.097911E-4,-7.3999115E-3,-1.9270817E-2,-1.5337057E-3,1.0451775E-2,-2.9124357E-3,-1.7506724E-2,-1.2541317E-2,-3.3729676E-2,4.347807E-3,1.6137244E-2,3.2968387E-2,1.6719718E-2,-2.5100373E-3,-1.148159E-2,4.681266E-3,-7.308228E-3],"split_indices":[60,60,53,2,53,60,60,7,0,56,39,53,9,60,8,0,0,40,9,60,64,0,7,0,35,0,53,0,53,0,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.52E2,1.13E2,5.8E1,9.4E1,7.6E1,3.7E1,2.1E1,3.7E1,7.4E1,2E1,4.9E1,2.7E1,1.8E1,1.9E1,1.5E1,6E0,1.4E1,2.3E1,2.1E1,5.3E1,7E0,1.3E1,9E0,4E1,6E0,2.1E1,4E0,1.4E1,6E0,1.3E1,4E0,1E1,4E0,1.9E1,8E0,1.3E1,4.7E1,6E0,5E0,8E0,1.3E1,2.7E1,1.6E1,5E0,4E0,1E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.0015358E-2,-2.1834044E-1,3.3261546E-1,-3.3468074E-1,-1.17166E-1,5.093784E-1,-3.041369E-2,-4.739779E-1,-2.7942786E-1,-2.7011435E-3,-3.6968493E-1,1.0281904E0,4.0577322E-1,-1.8742266E-1,8.318384E-2,-1.2573218E-2,-5.196883E-1,-3.1347278E-1,-8.126041E-2,-7.867527E-2,1.721744E-1,-4.9198493E-1,-4.3242212E-2,2.6400184E-2,5.5532537E-2,1.2569474E-1,4.805674E-1,5.5687913E-5,-2.5612614E-1,2.8417742E-1,-9.124279E-2,-2.9213479E-2,-1.656935E-2,-1.9510692E-2,-1.0396879E-2,2.5700482E-3,-1.0566688E-2,3.3224258E-3,-6.45877E-3,-1.3054486E-3,1.1091635E-2,-3.2223266E-2,-1.4282481E-2,4.072164E-3,-7.655815E-3,-9.4241457E-4,1.808876E-2,1.9628037E-2,3.1849075E-2,-5.7932525E-3,-1.4392748E-2,1.8789975E-2,3.6196883E-3,-8.223446E-3,1.1921913E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,-1,-1,45,47,-1,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.963036E1,1.9735699E0,6.689183E0,4.7558594E-1,2.687124E0,3.3453999E0,6.398446E-1,5.091858E-2,3.725767E-1,8.771369E-1,1.1365199E0,1.9866467E-1,1.1964893E0,1.9470513E-1,7.621605E-1,0E0,6.523752E-2,3.876996E-1,2.1602772E-1,4.3758884E-1,2.8248936E-1,5.764775E-1,1.6053389E-1,0E0,0E0,5.5478776E-1,2.4358177E-1,0E0,4.161793E-2,2.2267592E-1,1.2743366E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,-1,-1,46,48,-1,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,3.4008475E2,6.627576E7,1.5761888E2,3.8293724E7,6.677E3,7.381975E8,8.324524E3,1E0,6.98247E5,1.0528782E1,3.559158E6,1E0,2.1845242E3,2.012E3,-1.2573218E-2,3.529412E-2,2.1146445E4,1.0414141E8,6.290837E0,5.990846E2,1.1992621E3,1.1710234E3,2.6400184E-2,5.5532537E-2,1.1223402E6,9.198947E-1,5.5687913E-5,2.7350403E5,4E0,9.851E3,-2.9213479E-2,-1.656935E-2,-1.9510692E-2,-1.0396879E-2,2.5700482E-3,-1.0566688E-2,3.3224258E-3,-6.45877E-3,-1.3054486E-3,1.1091635E-2,-3.2223266E-2,-1.4282481E-2,4.072164E-3,-7.655815E-3,-9.4241457E-4,1.808876E-2,1.9628037E-2,3.1849075E-2,-5.7932525E-3,-1.4392748E-2,1.8789975E-2,3.6196883E-3,-8.223446E-3,1.1921913E-3],"split_indices":[60,60,53,60,53,9,7,58,112,36,62,1,109,4,0,0,65,41,7,61,4,4,4,0,0,36,35,0,36,8,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.7E2,1.03E2,7.8E1,9.2E1,6.9E1,3.4E1,2E1,5.8E1,6.4E1,2.8E1,1E1,5.9E1,1.4E1,2E1,5E0,1.5E1,4.9E1,9E0,4.5E1,1.9E1,2E1,8E0,4E0,6E0,1.3E1,4.6E1,4E0,1E1,9E0,1.1E1,7E0,8E0,2E1,2.9E1,5E0,4E0,1.3E1,3.2E1,5E0,1.4E1,8E0,1.2E1,4E0,4E0,9E0,4E0,3.9E1,7E0,4E0,6E0,5E0,4E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-9.927471E-3,-2.3480533E-1,3.1141526E-1,-2.0868525E-1,-3.2459006E-2,4.7390327E-1,3.965768E-2,-3.2612428E-1,-5.307506E-2,3.332751E-1,4.271199E-2,1.2569241E-1,-2.568262E-1,-4.3542677E-1,-2.750277E-1,-1.18524805E-1,1.0665808E-1,5.040586E-1,2.2913775E-1,-7.23089E-3,2.9454398E-1,-1.6151818E-2,-4.266918E-3,-2.405812E-2,-1.4186274E-2,-6.8748193E-3,-1.6651638E-2,1.5434107E-3,-1.0096709E-2,-5.982133E-3,8.9403875E-3,1.5871864E-2,3.0409496E-2,2.7030183E-3,1.3065208E-2,-3.939026E-3,5.5686226E-3,2.0144671E-2,6.933783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9656359E1,1.8179264E0,4.9268284E0,2.7787228E0,0E0,4.157922E0,1.1291213E0,3.661394E-1,7.130927E-1,8.626733E-1,0E0,7.652364E-1,1.2176758E-1,1.2731981E-1,6.212735E-1,7.4282086E-1,4.3937773E-1,2.7076864E-1,3.1855106E-1,2.1113843E-1,2.4285209E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.969686E2,6.8652206E9,5.9337012E7,3.4008475E2,-3.2459006E-2,5.206759E3,1.3917024E8,1.142E3,8.0251825E5,1.2951E4,4.271199E-2,3.5078225E3,6.139E3,3.5523141E-3,1.2623029E7,7.11E2,4.249921E2,2.555441E6,1E0,5.8194575E5,1.0456502E-1,-1.6151818E-2,-4.266918E-3,-2.405812E-2,-1.4186274E-2,-6.8748193E-3,-1.6651638E-2,1.5434107E-3,-1.0096709E-2,-5.982133E-3,8.9403875E-3,1.5871864E-2,3.0409496E-2,2.7030183E-3,1.3065208E-2,-3.939026E-3,5.5686226E-3,2.0144671E-2,6.933783E-3],"split_indices":[60,5,53,60,0,60,53,2,36,9,0,60,9,47,12,0,60,1,109,36,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.59E2,1.11E2,1.52E2,7E0,6.9E1,4.2E1,8.6E1,6.6E1,5.4E1,1.5E1,3.3E1,9E0,2.5E1,6.1E1,4.7E1,1.9E1,1.9E1,3.5E1,1.9E1,1.4E1,5E0,4E0,1.3E1,1.2E1,2.6E1,3.5E1,1.9E1,2.8E1,5E0,1.4E1,1.1E1,8E0,9E0,2.6E1,1.2E1,7E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.759327E-3,-2.0167859E-1,3.1294468E-1,-3.2415283E-1,-1.10569246E-1,4.5694566E-1,-2.8177107E-2,-3.8362142E-1,-2.1083036E-1,-2.3774587E-2,-3.6653045E-1,9.328397E-1,3.6205837E-1,-1.00822516E-1,1.1788901E-2,-4.485415E-1,-1.05054945E-1,-1.2628736E-1,-1.4840014E-2,-1.23516396E-1,4.930524E-2,-4.4165522E-1,1.0267001E-3,2.5444364E-2,5.1817276E-2,2.6474923E-1,5.677665E-1,8.258048E-2,-2.135643E-1,-2.2032904E-2,-1.1582716E-2,7.655454E-4,-1.0592336E-2,-1.2646011E-2,-1.9140664E-3,-2.4140307E-3,-1.2645571E-2,4.7160164E-3,-5.496516E-3,-9.8639745E-3,-2.8956E-2,3.4272068E-3,1.5537227E-2,2.9718794E-2,9.462133E-3,-5.4317648E-3,9.854549E-3,-1.2522672E-2,-6.050255E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,37,39,-1,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7490677E1,1.8611121E0,5.318039E0,4.0433788E-1,2.1906762E0,3.0941362E0,7.071976E-1,7.847686E-1,2.137909E-1,5.527736E-1,7.401178E-1,2.568102E-1,1.1508083E0,5.759202E-1,0E0,9.829903E-2,1.5593794E-1,2.022708E-1,0E0,3.2940164E-1,4.0654022E-1,7.595568E-1,0E0,0E0,0E0,6.114528E-1,4.3041277E-1,3.1526417E-1,4.5855463E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,25,25,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,38,40,-1,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,2.8070276E2,6.627576E7,2.037744E10,3.8293724E7,6.677E3,8.129243E5,4.565017E4,7.8908873E0,4.561433E2,9.432957E4,3.559158E6,1.1946838E7,2.7350403E5,1.1788901E-2,1.2765501E6,9.360022E5,5.7772512E0,-1.4840014E-2,7.6373405E0,2.6921875E1,9.214292E7,1.0267001E-3,2.5444364E-2,5.1817276E-2,1E0,4.679E3,1.1484E4,1.8671203E-1,-2.2032904E-2,-1.1582716E-2,7.655454E-4,-1.0592336E-2,-1.2646011E-2,-1.9140664E-3,-2.4140307E-3,-1.2645571E-2,4.7160164E-3,-5.496516E-3,-9.8639745E-3,-2.8956E-2,3.4272068E-3,1.5537227E-2,2.9718794E-2,9.462133E-3,-5.4317648E-3,9.854549E-3,-1.2522672E-2,-6.050255E-3],"split_indices":[60,60,53,39,53,9,36,41,61,60,41,1,1,36,0,56,56,61,0,61,64,7,0,0,0,109,0,2,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.69E2,1.07E2,7.1E1,9.8E1,7.5E1,3.2E1,4.5E1,2.6E1,7.4E1,2.4E1,1.1E1,6.4E1,2.6E1,6E0,3.6E1,9E0,1.6E1,1E1,3.1E1,4.3E1,2E1,4E0,5E0,6E0,4.5E1,1.9E1,1E1,1.6E1,3E1,6E0,5E0,4E0,5E0,1.1E1,2.2E1,9E0,3.3E1,1E1,1E1,1E1,1.3E1,3.2E1,1.5E1,4E0,4E0,6E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.1863986E-3,-1.8802468E-1,3.11628E-1,-1.5824486E-1,-2.9728679E-2,4.689817E-1,6.223996E-2,-2.9270655E-1,-4.4904567E-2,5.1499162E-2,3.9369923E-1,-9.5457677E-4,2.5823846E-1,-3.1809422E-1,-1.20694704E-1,3.2574505E-2,-2.528727E-1,2.782563E-1,5.572752E-1,-1.21552065E-1,1.4382605E-1,1.9448243E-2,4.4125243E-4,-1.8592108E-2,-1.116891E-2,-1.1282862E-2,3.5708013E-3,-1.8263417E-3,8.083806E-3,-1.5935542E-2,-2.7168829E-3,2.228115E-2,8.829341E-3,1.2311567E-2,2.7819239E-2,1.3436385E-3,-9.178367E-3,9.2738345E-3,-5.367261E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5976862E1,2.201202E0,4.0127497E0,2.4252439E0,0E0,2.6403923E0,5.080739E-1,2.8039694E-1,1.4261968E0,0E0,9.401884E-1,5.7620806E-1,4.041739E-1,3.0982018E-1,2.902199E-1,6.7847323E-1,4.13409E-1,5.4758716E-1,1.514554E-1,2.1690711E-1,1.3668427E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,3.0978662E6,5.9337012E7,3.2420947E2,-2.9728679E-2,5.0655737E0,8.129243E5,1E0,3.203027E7,5.1499162E-2,3.6028174E3,9.5891304E1,9.298568E7,2.1146445E4,1.661E3,8.129243E5,1.0528782E1,6.377402E6,4.747723E6,7.294459E7,9.744113E0,1.9448243E-2,4.4125243E-4,-1.8592108E-2,-1.116891E-2,-1.1282862E-2,3.5708013E-3,-1.8263417E-3,8.083806E-3,-1.5935542E-2,-2.7168829E-3,2.228115E-2,8.829341E-3,1.2311567E-2,2.7819239E-2,1.3436385E-3,-9.178367E-3,9.2738345E-3,-5.367261E-4],"split_indices":[60,36,53,60,0,62,36,112,53,0,4,64,53,41,2,36,62,53,40,53,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.68E2,1.02E2,1.59E2,9E0,6.2E1,4E1,7.2E1,8.7E1,5E0,5.7E1,3.1E1,9E0,6.2E1,1E1,6.4E1,2.3E1,3.5E1,2.2E1,1.7E1,1.4E1,5E0,4E0,2.7E1,3.5E1,6E0,4E0,4.3E1,2.1E1,1.5E1,8E0,9E0,2.6E1,4E0,1.8E1,6E0,1.1E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.3005435E-3,-1.8461594E-1,2.9949173E-1,-2.7731577E-1,-8.048137E-2,4.373191E-1,6.920876E-3,-3.6863548E-1,-2.2463396E-1,1.1306074E-2,-2.9894763E-1,2.5973356E-1,6.7943203E-1,-5.861273E-2,1.0461987E-2,-9.018392E-3,-3.943245E-1,-1.0619816E-2,-2.629441E-1,-6.8178535E-2,1.7804703E-1,-4.315931E-1,9.3428174E-4,7.762069E-2,3.480178E-1,4.8051517E-2,5.3903747E-1,-1.1478728E-1,9.738269E-3,-1.4252192E-2,-2.394373E-2,-5.5227336E-3,5.6493855E-3,-9.587753E-3,-1.8536665E-2,3.0924731E-3,-7.0016016E-3,1.160058E-2,7.0635E-4,-2.3942377E-2,-5.3112223E-3,-2.248859E-3,1.2682445E-2,2.368114E-2,1.2342758E-2,2.7342923E-2,1.3106342E-2,3.4157245E-3,-7.296419E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,31,33,35,37,39,-1,41,43,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4998191E1,1.6090822E0,4.1503744E0,3.4536028E-1,1.6276294E0,2.833232E0,5.0420254E-1,6.0338497E-2,4.8094344E-1,7.8083265E-1,1.0250556E0,6.482501E-1,9.2038536E-1,4.3398172E-1,0E0,0E0,1.1612272E-1,1.6048399E-1,2.981534E-1,4.6440652E-1,2.2463417E-1,4.1540575E-1,0E0,3.887509E-1,2.3439622E-1,0E0,1.5180731E-1,2.0164937E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,26,26,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,32,34,36,38,40,-1,42,44,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,3.4008475E2,6.627576E7,1.7398648E2,3.8293724E7,3.5078225E3,1.3179107E4,2.2290532E3,8.1446184E7,8.564393E5,1.0209414E1,1E0,6.677E3,8.129243E5,1.0461987E-2,-9.018392E-3,1.193419E1,4.7669297E-1,1.260567E1,7.25E2,6.621114E4,3.9778934E1,9.3428174E-4,8.1350375E5,6.518218E6,4.8051517E-2,5.7025972E7,1.23421266E5,9.738269E-3,-1.4252192E-2,-2.394373E-2,-5.5227336E-3,5.6493855E-3,-9.587753E-3,-1.8536665E-2,3.0924731E-3,-7.0016016E-3,1.160058E-2,7.0635E-4,-2.3942377E-2,-5.3112223E-3,-2.248859E-3,1.2682445E-2,2.368114E-2,1.2342758E-2,2.7342923E-2,1.3106342E-2,3.4157245E-3,-7.296419E-3],"split_indices":[60,60,53,60,53,60,4,56,5,36,62,109,9,36,0,0,62,35,64,0,41,64,0,36,53,0,53,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.68E2,1.02E2,8.8E1,8E1,6.9E1,3.3E1,3E1,5.8E1,5.7E1,2.3E1,4.1E1,2.8E1,2.6E1,7E0,5E0,2.5E1,9E0,4.9E1,3.9E1,1.8E1,1.6E1,7E0,1.4E1,2.7E1,6E0,2.2E1,2.2E1,4E0,1.7E1,8E0,5E0,4E0,3.7E1,1.2E1,1.5E1,2.4E1,1.2E1,6E0,1.2E1,4E0,9E0,5E0,7E0,2E1,1.7E1,5E0,4E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.6427932E-3,-1.7457105E-1,2.597662E-1,-1.4849645E-1,-5.308566E-1,3.9527416E-1,-8.0541E-4,-2.5437012E-1,-4.3413408E-2,-3.1859066E-2,-9.167951E-3,3.006834E-1,6.78107E-1,-1.4072418E-1,1.4614958E-1,-3.3622533E-1,-2.0180668E-1,1.9132216E-2,-2.320069E-1,1.1971327E-1,3.6677408E-1,7.996617E-1,9.30639E-3,-1.5936796E-2,-7.202895E-2,1.82709E-2,1.0365452E-2,-1.3219944E-2,-2.0695174E-2,4.482791E-4,-1.1157242E-2,-2.0333538E-3,5.7840743E-3,-4.860817E-3,-1.4157459E-2,-2.7983438E-3,1.1608609E-2,2.2803929E-2,1.3113608E-2,4.2312603E-2,2.243806E-2,2.04105E-3,-8.305151E-3,-2.9724997E-3,5.5723013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2781124E1,1.4726005E0,4.0256042E0,1.7356646E0,4.5012283E-1,1.8026075E0,8.430372E-1,2.677493E-1,9.480946E-1,0E0,0E0,6.4979744E-1,8.7978745E-1,2.71405E-1,6.7668027E-1,3.2190323E-2,4.310069E-1,4.1673774E-1,1.5842032E-1,4.2164713E-1,3.0128527E-1,7.3991776E-2,0E0,0E0,2.2233956E-1,0E0,1.23898044E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.0978662E6,6.627576E7,3.2420947E2,6.622819E5,5.206759E3,3.4573945E3,1.7398648E2,4.91101E5,-3.1859066E-2,-9.167951E-3,1E0,9.299267E9,8.8519E4,1E0,7.44E2,4.6376264E9,8.129243E5,6.4525446E2,5.8194575E5,1.3047E4,3.0291426E-1,9.30639E-3,-1.5936796E-2,3.005574E6,1.82709E-2,3.522085E6,-1.3219944E-2,-2.0695174E-2,4.482791E-4,-1.1157242E-2,-2.0333538E-3,5.7840743E-3,-4.860817E-3,-1.4157459E-2,-2.7983438E-3,1.1608609E-2,2.2803929E-2,1.3113608E-2,4.2312603E-2,2.243806E-2,2.04105E-3,-8.305151E-3,-2.9724997E-3,5.5723013E-3],"split_indices":[60,36,53,60,40,60,60,60,37,0,0,109,12,12,8,0,39,36,60,36,9,46,0,0,1,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.66E2,1.13E2,1.56E2,1E1,7.4E1,3.9E1,7.7E1,7.9E1,6E0,4E0,5.7E1,1.7E1,2E1,1.9E1,2.8E1,4.9E1,6E1,1.9E1,1.6E1,4.1E1,1.3E1,4E0,4E0,1.6E1,6E0,1.3E1,2.2E1,6E0,8E0,4.1E1,3.8E1,2.2E1,8E0,1.1E1,7E0,9E0,1.4E1,2.7E1,8E0,5E0,8E0,8E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[8.207698E-3,-1.703527E-1,2.6948056E-1,-1.4524621E-1,-5.349361E-1,3.9087102E-1,2.9191557E-2,-2.1848282E-1,1.3158984E-2,-1.0194312E-2,-3.2553107E-2,8.101997E-1,3.0340624E-1,-4.1320737E-2,2.3075806E-1,-3.4741083E-1,-1.7965567E-1,9.473352E-2,-1.6100934E-1,2.2152586E-2,4.4959128E-2,1.8299711E-1,4.430817E-1,8.134741E-3,-1.0587361E-1,1.587327E-2,2.200766E-3,-1.7860359E-2,-5.7083843E-3,-5.9087956E-3,-1.2865683E-2,-1.9281011E-3,7.880035E-3,5.8135204E-3,-1.2306354E-2,1.2863129E-2,8.4996165E-4,8.718803E-3,2.1983525E-2,-1.7060858E-5,-1.0172042E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,-1,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2642214E1,1.3960013E0,3.197157E0,1.7662365E0,3.830409E-1,2.4315643E0,5.4962593E-1,4.5819998E-1,7.114303E-1,0E0,0E0,1.8681717E-1,9.616461E-1,4.2681926E-1,1.9210213E-1,1.6008472E-1,3.8403583E-1,3.621167E-1,5.2226704E-1,0E0,0E0,5.4991126E-1,1.5391827E-1,0E0,2.7912062E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,-1,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,5.9337012E7,4.653861E2,1E0,6.677E3,1.0526E4,1.142E3,9.222491E0,-1.0194312E-2,-3.2553107E-2,3.559158E6,2.077646E3,9.893967E4,3.9E1,1.3448806E8,1.3585858E1,3.0972834E5,9.2413794E2,2.2152586E-2,4.4959128E-2,4.22372E5,3.145611E6,8.134741E-3,1.970492E6,1.587327E-2,2.200766E-3,-1.7860359E-2,-5.7083843E-3,-5.9087956E-3,-1.2865683E-2,-1.9281011E-3,7.880035E-3,5.8135204E-3,-1.2306354E-2,1.2863129E-2,8.4996165E-4,8.718803E-3,2.1983525E-2,-1.7060858E-5,-1.0172042E-2],"split_indices":[60,5,53,60,13,9,9,2,62,0,0,1,60,36,3,7,64,36,4,0,0,37,1,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,1.6E2,1.09E2,1.51E2,9E0,7.2E1,3.7E1,1.03E2,4.8E1,4E0,5E0,1.1E1,6.1E1,2.8E1,9E0,2.2E1,8.1E1,3.3E1,1.5E1,5E0,6E0,3.4E1,2.7E1,6E0,2.2E1,5E0,4E0,1.8E1,4E0,5.5E1,2.6E1,1.2E1,2.1E1,4E0,1.1E1,2.1E1,1.3E1,4E0,2.3E1,1.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-7.399762E-3,-1.6680072E-1,2.255286E-1,-1.4401753E-1,-4.788865E-1,3.4329113E-1,-2.2129497E-2,-2.4152003E-1,-2.4201764E-2,-7.3801083E-3,-2.9376045E-2,4.499266E-2,2.8648403E-1,-1.3267483E-2,3.6757614E-2,-1.7806576E-1,-3.1505182E-1,-8.850304E-2,1.4740671E-1,1.2596594E-1,3.3672696E-1,1.5700015E-1,-9.3297444E-2,-1.713944E-2,-4.83529E-3,-6.9210753E-3,-1.7237661E-2,1.4886531E-3,-7.2275307E-3,1.20065715E-2,3.6920863E-3,-2.539884E-3,1.2095586E-2,8.6868545E-3,1.832411E-2,5.7714853E-5,1.359603E-2,2.84162E-3,-6.9113593E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.05079975E1,1.1255641E0,3.3615537E0,1.8399644E0,4.5548248E-1,2.5289907E0,6.0590565E-1,3.498335E-1,8.0754024E-1,0E0,0E0,0E0,5.463209E-1,0E0,5.1506853E-1,6.559758E-1,3.2497978E-1,4.436248E-1,1.3149628E-1,4.851079E-1,4.2380047E-1,3.6581132E-1,1.5018249E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,6.627576E7,3.41043E2,9.214292E7,5.0655737E0,7.056125E7,1.5225E4,8.129243E5,-7.3801083E-3,-2.9376045E-2,4.499266E-2,1E0,-1.3267483E-2,1.739E3,5.8930116E0,6.394554E0,5.88E2,3.235813E-3,8.1350375E5,1.3389954E3,2.4654E4,2.7350403E5,-1.713944E-2,-4.83529E-3,-6.9210753E-3,-1.7237661E-2,1.4886531E-3,-7.2275307E-3,1.20065715E-2,3.6920863E-3,-2.539884E-3,1.2095586E-2,8.6868545E-3,1.832411E-2,5.7714853E-5,1.359603E-2,2.84162E-3,-6.9113593E-3],"split_indices":[60,5,53,60,7,62,7,9,36,0,0,0,109,0,0,62,61,0,50,36,60,2,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.67E2,1.14E2,1.57E2,1E1,7.7E1,3.7E1,8.6E1,7.1E1,4E0,6E0,5E0,7.2E1,6E0,3.1E1,4.8E1,3.8E1,5.2E1,1.9E1,1.8E1,5.4E1,1.6E1,1.5E1,1.2E1,3.6E1,1.1E1,2.7E1,1.9E1,3.3E1,6E0,1.3E1,8E0,1E1,1.7E1,3.7E1,8E0,8E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[4.40071E-5,-1.4946079E-1,2.222434E-1,-2.3184995E-1,-6.625369E-2,2.8721228E-1,-1.2216664E-1,-2.017797E-1,-4.0486252E-1,8.820409E-3,-3.156478E-1,1.8824671E-1,4.6689E-1,-2.1251608E-1,2.3099026E-3,-3.2959077E-1,-1.2515213E-1,-1.0341123E-2,-2.1905383E-2,-4.832723E-2,1.281383E-1,-5.4313596E-3,-3.9781034E-1,3.080345E-1,4.5445174E-2,3.0990122E-2,3.4256396E-1,-1.3518575E-2,-3.7972599E-3,-1.8775048E-2,-1.0130131E-2,-8.639053E-3,-8.0698344E-4,4.672374E-3,-4.665761E-3,-1.385157E-3,8.262688E-3,-8.849337E-3,-2.312586E-2,1.7013434E-2,5.9691337E-3,6.9562676E-3,-4.4423525E-4,6.2737493E-3,2.0751458E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,35,-1,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.002583E0,1.0952084E0,2.4740648E0,3.3971167E-1,1.5380733E0,1.5480447E0,2.9093218E-1,6.541145E-1,3.3751726E-2,4.4259167E-1,2.6166713E-1,1.0294602E0,5.898037E-1,1.0234511E-1,0E0,1.3304377E-1,3.0606252E-1,0E0,0E0,3.5971954E-1,1.7593715E-1,0E0,1.8271184E-1,3.1408525E-1,1.7024621E-1,0E0,4.294436E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,36,-1,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.4008475E2,9.996903E7,1.996866E1,3.8293724E7,3.5078225E3,2.4494735E9,1.142E3,1.2414683E5,8.129243E5,2.5927516E11,4.24975E5,4.0642028E6,5.535E3,2.3099026E-3,3.5523141E-3,6.439228E-1,-1.0341123E-2,-2.1905383E-2,5.15E2,5.990846E2,-5.4313596E-3,9.214292E7,2.4025E4,2.7E1,3.0990122E-2,2.3818828E-1,-1.3518575E-2,-3.7972599E-3,-1.8775048E-2,-1.0130131E-2,-8.639053E-3,-8.0698344E-4,4.672374E-3,-4.665761E-3,-1.385157E-3,8.262688E-3,-8.849337E-3,-2.312586E-2,1.7013434E-2,5.9691337E-3,6.9562676E-3,-4.4423525E-4,6.2737493E-3,2.0751458E-2],"split_indices":[60,60,53,64,53,60,7,2,36,36,39,37,55,9,0,47,35,0,0,0,4,0,7,9,3,0,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,1.61E2,1.08E2,8E1,8.1E1,9.1E1,1.7E1,7E1,1E1,6.3E1,1.8E1,6E1,3.1E1,1.1E1,6E0,2.5E1,4.5E1,4E0,6E0,4.3E1,2E1,6E0,1.2E1,3.2E1,2.8E1,1E1,2.1E1,6E0,5E0,1.3E1,1.2E1,2.8E1,1.7E1,1.1E1,3.2E1,5E0,1.5E1,5E0,7E0,2.3E1,9E0,9E0,1.9E1,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.2337968E-2,-1.3216752E-1,2.3817983E-1,-1.11230545E-1,-4.6022338E-1,3.5090888E-1,6.284482E-3,-2.071963E-1,-4.8700158E-2,-2.8827472E-2,-7.785376E-3,7.298882E-1,2.7315235E-1,-1.0517588E-1,9.621504E-2,-2.3516178E-1,-6.6415735E-2,3.8507052E-3,-1.9092475E-1,4.22632E-2,2.1382669E-2,5.794121E-2,3.2794842E-1,-4.4985306E-2,-1.1079146E-2,1.6097037E-2,-2.8456582E-2,-1.462926E-2,-6.316034E-3,1.7051813E-3,-8.10508E-3,4.0121595E-3,-3.182652E-3,-1.266148E-2,-3.6475356E-3,-9.4060664E-4,9.350702E-3,1.0868638E-2,1.9989276E-2,1.7137254E-3,-5.5601858E-3,-6.9325077E-3,3.2083322E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.202024E0,1.1301906E0,2.8733416E0,9.6803594E-1,3.7710977E-1,1.9812698E0,3.8099614E-1,2.363491E-1,7.481377E-1,0E0,0E0,1.7274857E-1,7.239399E-1,1.2778331E-1,6.7339885E-1,3.852508E-1,1.4261015E-1,4.5582816E-1,2.317909E-1,0E0,0E0,1.634108E-1,3.8547802E-1,8.548105E-2,0E0,0E0,1.9026041E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,3.0978662E6,6.627576E7,2.8070276E2,4.2993605E6,6.677E3,3.4573945E3,4.565017E4,2.6921875E1,-2.8827472E-2,-7.785376E-3,5.0655737E0,1E0,4.6E1,1E0,2.037744E10,9.360022E5,7.25E2,4.2845196E1,4.22632E-2,2.1382669E-2,1.1223402E6,1.2737E4,9.39E2,-1.1079146E-2,1.6097037E-2,2.1312436E5,-1.462926E-2,-6.316034E-3,1.7051813E-3,-8.10508E-3,4.0121595E-3,-3.182652E-3,-1.266148E-2,-3.6475356E-3,-9.4060664E-4,9.350702E-3,1.0868638E-2,1.9989276E-2,1.7137254E-3,-5.5601858E-3,-6.9325077E-3,3.2083322E-3],"split_indices":[60,36,53,60,36,9,60,41,64,0,0,62,109,3,8,39,56,0,64,0,0,36,2,0,0,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.71E2,1.09E2,1.62E2,9E0,7.3E1,3.6E1,6.3E1,9.9E1,5E0,4E0,1.1E1,6.2E1,1.6E1,2E1,5.2E1,1.1E1,7.3E1,2.6E1,5E0,6E0,1.3E1,4.9E1,1.2E1,4E0,6E0,1.4E1,2.7E1,2.5E1,6E0,5E0,3.4E1,3.9E1,1.4E1,1.2E1,9E0,4E0,2.8E1,2.1E1,6E0,6E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.8854748E-4,-1.1591903E-1,2.51188E-1,-9.589218E-2,-4.6681312E-1,3.9303067E-1,4.505268E-2,-2.1316467E-1,-3.147412E-2,-2.7721431E-2,-9.71942E-3,7.354785E-1,2.939841E-1,-7.904436E-3,1.2575699E-2,-2.9868853E-1,-1.6125484E-1,3.0771967E-2,-1.5835138E-1,1.9601075E-2,3.9211273E-2,1.756131E-5,3.4290725E-1,-9.284672E-2,1.0187168E-1,-5.983123E-3,-1.5072573E-2,-8.618735E-3,-2.2113627E-4,3.646088E-3,-3.353574E-3,-4.573961E-3,-1.7246146E-2,1.2207134E-2,2.2815982E-2,2.9365083E-3,-6.544875E-3,7.6538976E-3,-5.368602E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,29,31,-1,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.105446E0,1.2904241E0,2.555306E0,1.3606492E0,2.1884227E-1,1.5475321E0,4.4907826E-1,2.3596501E-1,9.367912E-1,0E0,0E0,3.327608E-2,6.031563E-1,2.9852214E-1,0E0,6.87778E-2,1.6956449E-1,4.07116E-1,4.6038514E-1,0E0,0E0,0E0,2.9599524E-1,1.48075E-1,1.0424979E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,17,17,18,18,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,30,32,-1,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3457836E3,3.0978662E6,5.9337012E7,2.8070276E2,4.2993605E6,6.677E3,8.129243E5,1.5977058E2,2.6921875E1,-2.7721431E-2,-9.71942E-3,3.559158E6,1E0,9.5891304E1,1.2575699E-2,5.2907334E3,1E0,7.739782E0,5.506676E11,1.9601075E-2,3.9211273E-2,1.756131E-5,5.206759E3,6.822099E7,6.543368E6,-5.983123E-3,-1.5072573E-2,-8.618735E-3,-2.2113627E-4,3.646088E-3,-3.353574E-3,-4.573961E-3,-1.7246146E-2,1.2207134E-2,2.2815982E-2,2.9365083E-3,-6.544875E-3,7.6538976E-3,-5.368602E-4],"split_indices":[60,36,53,60,36,9,36,60,64,0,0,1,109,64,0,58,112,61,39,0,0,0,60,53,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.89E2,8.7E1,1.8E2,9E0,5.1E1,3.6E1,6.3E1,1.17E2,5E0,4E0,1E1,4.1E1,3E1,6E0,2.2E1,4.1E1,7.9E1,3.8E1,4E0,6E0,6E0,3.5E1,1.7E1,1.3E1,4E0,1.8E1,3.5E1,6E0,5.4E1,2.5E1,3.1E1,7E0,2.5E1,1E1,4E0,1.3E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.0537671E-3,-1.417877E-1,2.0085134E-1,-1.2087798E-1,-4.3113822E-1,2.9949564E-1,5.1759694E-2,-1.9829324E-1,-3.5860524E-2,-2.6095783E-2,-7.136033E-3,1.8505573E-1,5.355692E-1,-3.0728258E-2,1.8636407E-1,-1.4325173E-1,-2.6285157E-1,1.6099203E-2,-1.7518643E-1,3.774602E-1,1.17652774E-1,6.295684E-1,1.124341E-2,-1.2911564E-1,7.626683E-2,8.668663E-2,1.5221921E-2,-1.1639838E-2,-3.8671647E-3,-3.8122996E-3,-1.592851E-2,5.205228E-3,-2.4316378E-3,-9.949354E-3,-1.6985746E-3,9.798566E-3,1.9943899E-2,1.8627851E-3,9.488545E-3,3.192539E-2,1.6134242E-2,-1.0687691E-3,-8.063756E-3,-1.9850149E-3,7.396243E-3,8.284642E-3,-2.5916056E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.03414E0,9.7037506E-1,1.6758075E0,1.0393713E0,3.2421827E-1,1.7650099E0,5.2757883E-1,2.572775E-1,5.6024915E-1,0E0,0E0,5.7808506E-1,4.4879532E-1,3.2565627E-1,2.3015636E-1,2.8391027E-1,5.2502847E-1,3.8682637E-1,1.108402E-1,1.7886281E-2,2.4424934E-1,4.2141438E-2,0E0,7.39243E-2,1.5802222E-1,1.04318045E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.0978662E6,5.7025972E7,3.2420947E2,6.622819E5,3.614485E3,3.9174525E5,1.6212E4,4.66667E5,-2.6095783E-2,-7.136033E-3,6.518218E6,3.3784313E0,1.0946131E2,2.982115E0,1.295E3,6.5833335E0,8.9517656E5,2.770862E8,4.558E3,8.1350375E5,1.7173885E8,1.124341E-2,7.361366E7,1.9330153E5,2.9E1,1.5221921E-2,-1.1639838E-2,-3.8671647E-3,-3.8122996E-3,-1.592851E-2,5.205228E-3,-2.4316378E-3,-9.949354E-3,-1.6985746E-3,9.798566E-3,1.9943899E-2,1.8627851E-3,9.488545E-3,3.192539E-2,1.6134242E-2,-1.0687691E-3,-8.063756E-3,-1.9850149E-3,7.396243E-3,8.284642E-3,-2.5916056E-4],"split_indices":[60,36,53,60,40,4,36,9,37,0,0,53,65,64,65,2,61,56,7,2,36,40,0,53,41,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.68E2,1.14E2,1.58E2,1E1,6.8E1,4.6E1,8.2E1,7.6E1,6E0,4E0,4.7E1,2.1E1,2.9E1,1.7E1,4.6E1,3.6E1,5.6E1,2E1,1.1E1,3.6E1,1.5E1,6E0,1.5E1,1.4E1,1.1E1,6E0,1.5E1,3.1E1,1.2E1,2.4E1,2.3E1,3.3E1,1.5E1,5E0,4E0,7E0,2E1,1.6E1,1.1E1,4E0,5E0,1E1,6E0,8E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[8.960068E-4,-1.283783E-1,2.343721E-1,-4.0135494E-1,-1.0787169E-1,3.374839E-1,4.741607E-2,-1.2455327E-2,-2.442061E-2,-1.6627894E-1,-1.4557668E-2,1.8362103E-1,5.1154315E-1,-1.557104E-2,2.381075E-1,-2.818251E-1,-1.2907152E-1,7.1364276E-2,-1.9798289E-1,2.8131488E-1,6.363258E-2,3.9468527E-2,4.2194077E-1,-1.05112106E-1,1.1257639E-1,1.4977125E-2,4.7557727E-3,-6.098714E-3,-1.4928978E-2,3.235818E-4,-7.4162786E-3,5.3160707E-3,-8.4958365E-3,4.477936E-3,-1.4851182E-2,1.6741678E-2,4.063832E-3,-8.6110266E-4,8.468174E-3,1.19166495E-2,2.1685097E-2,-1.5878804E-3,-7.5533446E-3,-1.241755E-3,8.5508E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.300082E0,9.475205E-1,1.8731813E0,5.911815E-2,9.023149E-1,1.5932884E0,4.3610722E-1,0E0,0E0,4.0214038E-1,1.0385454E0,3.923738E-1,5.3659153E-1,3.3311793E-1,7.2190106E-2,1.09141946E-1,3.452798E-1,5.2911663E-1,8.03786E-1,2.6981306E-1,1.7181593E-1,0E0,7.812977E-2,6.511107E-2,1.274947E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,5.178571E0,5.9337012E7,6.3004294E5,4.653861E2,2.4881973E3,8.129243E5,-1.2455327E-2,-2.442061E-2,1.5977058E2,3.203027E7,2.1356926E3,5.0655737E0,6.2150537E3,9.040637E7,3.0353E4,1.06423064E8,2.6921875E1,3.1E1,1.5562991E6,2.077646E3,3.9468527E-2,5.125467E-1,7.632764E7,1.6117271E9,1.4977125E-2,4.7557727E-3,-6.098714E-3,-1.4928978E-2,3.235818E-4,-7.4162786E-3,5.3160707E-3,-8.4958365E-3,4.477936E-3,-1.4851182E-2,1.6741678E-2,4.063832E-3,-8.6110266E-4,8.468174E-3,1.19166495E-2,2.1685097E-2,-1.5878804E-3,-7.5533446E-3,-1.241755E-3,8.5508E-3],"split_indices":[60,62,53,36,60,60,36,0,0,60,53,4,62,4,53,12,5,64,3,55,60,0,65,53,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.76E2,9.7E1,1.1E1,1.65E2,6.2E1,3.5E1,7E0,4E0,1.01E2,6.4E1,3.4E1,2.8E1,2.7E1,8E0,2.3E1,7.8E1,4.4E1,2E1,1.8E1,1.6E1,4E0,2.4E1,1.6E1,1.1E1,4E0,4E0,6E0,1.7E1,1.5E1,6.3E1,3.8E1,6E0,6E0,1.4E1,1.2E1,6E0,1E1,6E0,7E0,1.7E1,8E0,8E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-1.1500401E-2,-1.4667022E-1,1.7548658E-1,-1.2700802E-1,-4.2693627E-1,2.9200697E-1,2.8702097E-2,-2.0632994E-1,-5.3106915E-2,-8.583368E-3,-2.5608188E-2,1.8033624E-1,4.832315E-1,-1.1183286E-1,8.869824E-2,-2.3190364E-1,-8.381722E-2,5.117077E-3,-1.3031858E-1,2.7337742E-1,6.432205E-2,2.9645752E-2,3.1011114E-1,1.9856822E-3,-1.8619062E-1,3.3303965E-2,1.4795043E-2,-1.3423955E-2,-6.111909E-3,3.2723765E-5,-7.755466E-3,2.0474936E-3,-5.71567E-3,-1.5172915E-3,-1.0393467E-2,1.4915967E-2,1.2176175E-3,5.8514317E-3,-1.2208973E-3,7.5534536E-3,1.8962543E-2,-1.302488E-2,-4.167449E-3,5.4072877E-3,-2.9593369E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,-1,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9258122E0,8.219693E-1,1.9585085E0,8.6833906E-1,2.0720494E-1,1.2795238E0,4.4836023E-1,2.06146E-1,3.5581702E-1,0E0,0E0,4.35768E-1,4.4005728E-1,1.923914E-1,4.66127E-1,3.1032014E-1,1.0026956E-1,2.3951778E-1,3.0845082E-1,2.787739E-1,1.1740941E-1,0E0,1.2562156E-1,0E0,7.616925E-2,2.6207358E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,-1,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.869239E2,6.8652206E9,5.7025972E7,2.8070276E2,6.4044494E5,3.7488223E3,1.08017064E8,4.565017E4,1.7812634E1,-8.583368E-3,-2.5608188E-2,3.30441E5,4.0642028E6,6.0671224E7,8.129243E5,2.037744E10,9.360022E5,2.3238E4,9.222491E0,2.4025E4,8.369446E0,2.9645752E-2,5.318116E3,1.9856822E-3,8.8519E4,8.2731545E-1,1.4795043E-2,-1.3423955E-2,-6.111909E-3,3.2723765E-5,-7.755466E-3,2.0474936E-3,-5.71567E-3,-1.5172915E-3,-1.0393467E-2,1.4915967E-2,1.2176175E-3,5.8514317E-3,-1.2208973E-3,7.5534536E-3,1.8962543E-2,-1.302488E-2,-4.167449E-3,5.4072877E-3,-2.9593369E-3],"split_indices":[60,5,53,60,36,4,7,41,64,0,0,37,55,53,36,39,56,9,62,9,62,0,60,0,12,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.58E2,1.14E2,1.49E2,9E0,6.3E1,5.1E1,7.1E1,7.8E1,4E0,5E0,4.1E1,2.2E1,1.5E1,3.6E1,5.8E1,1.3E1,4.5E1,3.3E1,2.2E1,1.9E1,1E1,1.2E1,5E0,1E1,3E1,6E0,3.5E1,2.3E1,7E0,6E0,3.5E1,1E1,1.7E1,1.6E1,1.8E1,4E0,1.1E1,8E0,6E0,6E0,4E0,6E0,1.6E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.4089692E-3,-1.0964801E-1,1.7794436E-1,-9.0975806E-2,-2.1073733E-2,2.879495E-1,1.7538292E-2,-3.325498E-1,-7.202208E-2,2.0718306E-1,5.3797436E-1,-6.757084E-2,9.482424E-2,-9.920918E-3,-1.9794794E-2,-1.3355981E-1,1.7382357E-2,3.334804E-1,1.2571955E-1,3.055514E-2,1.7516403E-2,-1.0464695E-1,3.6564579E-3,1.3871488E-2,-7.911749E-3,-4.2408076E-3,-9.754046E-3,5.6378627E-3,-2.359205E-3,4.651405E-3,1.7897664E-2,7.6517533E-3,-1.9601765E-3,-9.256225E-3,-2.888849E-3,-3.4618692E-3,7.7623585E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.359538E0,1.0112294E0,1.9006629E0,6.857519E-1,0E0,1.1809325E0,3.0240566E-1,3.2770395E-2,8.0007464E-1,4.7250605E-1,4.4463634E-2,1.2820528E-1,5.114469E-1,0E0,0E0,2.5959003E-1,4.3889454E-1,2.047553E-1,2.2504789E-1,0E0,0E0,6.0090616E-2,0E0,0E0,2.1455744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,21,21,24,24],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0528038E3,3.0978662E6,5.7025972E7,5.178571E0,-2.1073733E-2,5.087614E3,3.5078225E3,9.372475E-1,4.653861E2,1.3047E4,2.4816266E5,2.4E1,5.8282824E7,-9.920918E-3,-1.9794794E-2,7.34E2,6.4613514E0,6.47258E-2,5.805676E6,3.055514E-2,1.7516403E-2,7.056125E7,3.6564579E-3,1.3871488E-2,2.0390862E-1,-4.2408076E-3,-9.754046E-3,5.6378627E-3,-2.359205E-3,4.651405E-3,1.7897664E-2,7.6517533E-3,-1.9601765E-3,-9.256225E-3,-2.888849E-3,-3.4618692E-3,7.7623585E-3],"split_indices":[60,36,53,62,0,60,60,35,60,9,41,8,59,0,0,0,61,46,55,0,0,7,0,0,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.61E2,1.07E2,1.54E2,7E0,6.3E1,4.4E1,1E1,1.44E2,4.9E1,1.4E1,2.1E1,2.3E1,6E0,4E0,8.5E1,5.9E1,1.8E1,3.1E1,6E0,8E0,1.7E1,4E0,7E0,1.6E1,5.7E1,2.8E1,2.3E1,3.6E1,4E0,1.4E1,2.5E1,6E0,4E0,1.3E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.6095605E-3,-9.386973E-2,2.1417654E-1,-1.9919157E-1,-5.1520295E-2,3.1748548E-1,1.3979899E-2,-2.2986412E-1,-3.8178697E-2,-3.2804115E-3,-1.7512819E-1,6.2267107E-1,2.4093284E-1,1.2039544E-1,-1.07556365E-1,-2.8751E-1,-1.0380873E-1,3.2182182E-3,-7.377437E-3,9.18401E-2,-5.5114627E-2,-3.1764752E-1,-8.2611464E-2,3.53799E-2,1.7178891E-2,3.074274E-2,2.8864998E-1,2.352576E-1,-7.844001E-3,-1.4757924E-1,9.23227E-5,-8.577905E-3,-1.5371016E-2,-8.6082835E-3,1.3730602E-3,5.7569225E-3,-2.9263885E-3,-4.6512485E-3,2.9359246E-3,-4.6785655E-3,-1.9130101E-2,-6.226972E-3,7.1356874E-3,6.4412537E-3,-4.898648E-3,3.4223488E-3,1.5341295E-2,1.3421899E-2,6.0748174E-3,-3.063128E-3,2.4130652E-3,-3.230754E-3,-8.369711E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,-1,-1,43,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.735417E0,8.505795E-1,1.815587E0,2.6285148E-1,8.282589E-1,1.205018E0,4.1388303E-1,3.0441618E-1,1.4311695E-1,5.03042E-1,4.8799038E-1,1.1342764E-1,4.725504E-1,2.5225294E-1,6.607047E-2,8.319187E-2,1.8338242E-1,0E0,0E0,1.9367543E-1,3.6665654E-1,2.748673E-1,3.3361974E-1,0E0,0E0,1.645869E-1,3.4767318E-1,1.3850361E-2,3.531234E-2,1.6372114E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,-1,-1,44,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3457836E3,4.9054803E2,6.627576E7,1E0,2.6921875E1,6.677E3,1.739E3,3.2658758E4,1.0414141E8,8.285E4,1.643614E6,5.0655737E0,3.108307E6,1E0,9.851E3,2.8434807E3,3.6E1,3.2182182E-3,-7.377437E-3,9.6494156E-1,7.608277E2,3.26E2,9.313E3,3.53799E-2,1.7178891E-2,3.8343488E6,3.0903362E5,3.566963E0,9.64E2,1.0129378E12,9.23227E-5,-8.577905E-3,-1.5371016E-2,-8.6082835E-3,1.3730602E-3,5.7569225E-3,-2.9263885E-3,-4.6512485E-3,2.9359246E-3,-4.6785655E-3,-1.9130101E-2,-6.226972E-3,7.1356874E-3,6.4412537E-3,-4.898648E-3,3.4223488E-3,1.5341295E-2,1.3421899E-2,6.0748174E-3,-3.063128E-3,2.4130652E-3,-3.230754E-3,-8.369711E-3],"split_indices":[60,63,53,112,64,9,0,41,7,37,40,62,1,8,9,55,3,0,0,65,60,0,2,0,0,40,36,62,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.92E2,8.7E1,5.4E1,1.38E2,5.7E1,3E1,4.5E1,9E0,1E2,3.8E1,1E1,4.7E1,1.6E1,1.4E1,3E1,1.5E1,5E0,4E0,3.5E1,6.5E1,1.4E1,2.4E1,5E0,5E0,9E0,3.8E1,8E0,8E0,1E1,4E0,1.1E1,1.9E1,9E0,6E0,2.9E1,6E0,4.7E1,1.8E1,5E0,9E0,2E1,4E0,5E0,4E0,7E0,3.1E1,4E0,4E0,4E0,4E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.2330455E-2,-1.2586787E-1,1.3361461E-1,-1.0565757E-1,-3.6295515E-1,2.017965E-1,-3.0316154E-2,-1.500177E-1,-4.9969153E-3,-2.1769613E-2,-5.274631E-3,5.2080405E-1,1.5360326E-1,-1.1521176E-1,6.569653E-2,-2.309792E-1,-1.20731756E-1,6.207541E-2,-1.23065904E-1,3.0505467E-2,1.5890675E-2,1.0006322E-1,2.60964E-1,-3.4297675E-2,-9.682246E-3,1.35305645E-2,-4.4672683E-2,-7.1921283E-3,-1.4674876E-2,-9.759924E-4,-7.373923E-3,4.477846E-3,-4.367604E-3,-9.266192E-3,4.8274132E-3,8.3935745E-3,1.4242023E-3,1.6463958E-2,7.951528E-3,3.0511094E-3,-4.275595E-3,3.68992E-3,-5.0208564E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,-1,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6564817E0,7.1872497E-1,1.380902E0,6.5565217E-1,2.733358E-1,1.2575417E0,3.0915803E-1,2.1395898E-1,3.7192082E-1,0E0,0E0,4.5253992E-2,4.1649032E-1,1.4666027E-1,4.66449E-1,1.20803595E-1,2.9878366E-1,1.7486987E-1,3.2406825E-1,0E0,0E0,2.9568988E-1,1.5145099E-1,7.6833166E-2,0E0,0E0,1.1320245E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,-1,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,4.56567E7,6.627576E7,4.2708307E2,5.752E3,6.661E3,3.4573945E3,1.142E3,1.0141194E3,-2.1769613E-2,-5.274631E-3,5.0655737E0,2.4881973E3,4.4E1,1E0,5.1510445E6,4.999413E5,8.6273805E4,7.26031E2,3.0505467E-2,1.5890675E-2,3.17308E5,3.895276E7,2.815322E1,-9.682246E-3,1.35305645E-2,8.0756325E-1,-7.1921283E-3,-1.4674876E-2,-9.759924E-4,-7.373923E-3,4.477846E-3,-4.367604E-3,-9.266192E-3,4.8274132E-3,8.3935745E-3,1.4242023E-3,1.6463958E-2,7.951528E-3,3.0511094E-3,-4.275595E-3,3.68992E-3,-5.0208564E-3],"split_indices":[60,53,53,60,2,9,60,2,4,0,0,62,60,3,8,53,56,41,60,0,0,37,53,66,0,0,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.57E2,1.22E2,1.46E2,1.1E1,8.6E1,3.6E1,1.01E2,4.5E1,7E0,4E0,1E1,7.6E1,1.9E1,1.7E1,2.5E1,7.6E1,2.9E1,1.6E1,4E0,6E0,5.2E1,2.4E1,1.1E1,8E0,5E0,1.2E1,1.5E1,1E1,2.2E1,5.4E1,2.4E1,5E0,1.2E1,4E0,2.3E1,2.9E1,1E1,1.4E1,4E0,7E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.2409782E-2,-1.0491553E-1,1.765573E-1,-8.621271E-2,-3.5211644E-1,2.6689523E-1,2.4718937E-2,-1.4401272E-1,-2.2481013E-2,-2.192349E-2,-4.9731005E-3,1.5709448E-1,4.242862E-1,6.9676384E-2,-8.577409E-3,-1.2122116E-1,-2.7388594E-1,1.7108649E-2,-1.539004E-1,2.4236442E-1,2.0462083E-2,3.2397553E-2,3.5143867E-1,-4.4150446E-3,1.1150585E-1,-8.812143E-3,-3.8812784E-3,-3.325648E-3,-1.7440248E-2,5.14033E-3,-1.3530091E-3,-5.278268E-4,-9.774215E-3,1.3888022E-2,7.632355E-3,3.7284568E-3,-2.258538E-3,9.363582E-3,1.8311474E-2,7.942032E-3,2.5973388E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,29,31,33,35,-1,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.314655E0,7.1400297E-1,1.5708303E0,5.530002E-1,2.8167248E-1,1.1776257E0,4.3040136E-1,2.0064032E-1,3.82441E-1,0E0,0E0,5.0766873E-1,3.4291697E-1,2.669438E-1,0E0,1.6264164E-1,2.0535296E-1,2.5383058E-1,1.3895997E-1,7.258475E-2,7.9700306E-2,0E0,9.0430975E-2,0E0,1.927368E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,30,32,34,36,-1,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.0978662E6,5.9337012E7,3.2420947E2,6.622819E5,2.4881973E3,1.4146589E8,1.996866E1,4.91101E5,-2.192349E-2,-4.9731005E-3,1.878369E6,5.841377E0,7.056125E7,-8.577409E-3,1.7398648E2,3.589166E-2,1.5342362E4,5.721356E2,1.20171E5,2.1637352E7,3.2397553E-2,5.125467E-1,-4.4150446E-3,2.3910058E5,-8.812143E-3,-3.8812784E-3,-3.325648E-3,-1.7440248E-2,5.14033E-3,-1.3530091E-3,-5.278268E-4,-9.774215E-3,1.3888022E-2,7.632355E-3,3.7284568E-3,-2.258538E-3,9.363582E-3,1.8311474E-2,7.942032E-3,2.5973388E-4],"split_indices":[60,36,53,60,40,60,53,64,37,0,0,55,62,7,0,60,46,41,60,37,53,0,65,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.6E2,1.14E2,1.5E2,1E1,7.1E1,4.3E1,7.8E1,7.2E1,6E0,4E0,4.3E1,2.8E1,3.6E1,7E0,6.8E1,1E1,5.6E1,1.6E1,2.6E1,1.7E1,4E0,2.4E1,7E0,2.9E1,2.2E1,4.6E1,4E0,6E0,1.8E1,3.8E1,5E0,1.1E1,1.3E1,1.3E1,9E0,8E0,7E0,1.7E1,1.8E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.550195E-3,-1.1021256E-1,1.5620065E-1,-9.424292E-2,-3.2915407E-1,2.2009261E-1,-1.4400415E-2,-1.4033689E-1,3.8730584E-3,-2.298281E-2,-6.105328E-3,5.264841E-1,1.7090906E-1,-1.5274532E-1,3.807894E-2,-1.5605004E-1,1.3543896E-2,1.4180818E-1,-3.7239358E-2,1.1102556E-2,3.0160844E-2,1.3105033E-1,3.162928E-1,-9.031743E-3,-3.6268583E-3,1.0092572E-2,-3.4475673E-2,-7.81845E-3,2.0923033E-4,-3.9679594E-3,4.3459255E-3,1.1095349E-3,1.0860296E-2,-5.121825E-3,2.5953155E-3,1.158076E-3,7.820948E-3,1.8073812E-2,7.967501E-3,1.5437994E-3,-4.8899255E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,-1,-1,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6365542E0,5.039313E-1,1.2457731E0,6.5179133E-1,2.6798952E-1,1.1607804E0,2.3814443E-1,2.390728E-1,2.7181062E-1,0E0,0E0,2.4567652E-1,3.8157082E-1,1.3454601E-2,3.2295945E-1,1.9750953E-1,8.876278E-2,1.1612368E-1,2.6259074E-1,0E0,0E0,2.3826385E-1,9.593248E-2,0E0,0E0,0E0,9.137005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,-1,-1,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,6.627576E7,4.653861E2,9.32784E5,6.677E3,1.08017064E8,9.2664665E-1,8.920345E3,-2.298281E-2,-6.105328E-3,3.559158E6,5.318116E3,2.04556E5,9.39E2,1E0,2.781252E2,6.948309E7,1.1334876E0,1.1102556E-2,3.0160844E-2,1E0,3.09483E2,-9.031743E-3,-3.6268583E-3,1.0092572E-2,1.2300734E8,-7.81845E-3,2.0923033E-4,-3.9679594E-3,4.3459255E-3,1.1095349E-3,1.0860296E-2,-5.121825E-3,2.5953155E-3,1.158076E-3,7.820948E-3,1.8073812E-2,7.967501E-3,1.5437994E-3,-4.8899255E-3],"split_indices":[60,5,53,60,37,9,7,35,41,0,0,1,60,12,0,84,4,7,47,0,0,109,66,0,0,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.52E2,1.13E2,1.43E2,9E0,8.2E1,3.1E1,9.7E1,4.6E1,4E0,5E0,1E1,7.2E1,8E0,2.3E1,8.8E1,9E0,1E1,3.6E1,4E0,6E0,5.8E1,1.4E1,4E0,4E0,6E0,1.7E1,8.1E1,7E0,4E0,5E0,5E0,5E0,2E1,1.6E1,1.6E1,4.2E1,8E0,6E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.3434655E-3,-1.02087945E-1,1.6788118E-1,-8.6028844E-2,-3.3077395E-1,2.4404608E-1,-8.736109E-3,-1.4223734E-1,-3.2775905E-2,-2.0610558E-2,-4.649419E-3,5.3486025E-1,1.9401135E-1,-1.1171175E-1,7.756226E-2,-1.9348161E-1,-9.2486866E-2,-1.2546526E-2,-1.1118994E-2,3.157098E-2,1.4733403E-2,2.9882329E-2,2.3608725E-1,-8.440286E-3,-6.900654E-2,1.14197675E-2,-7.083528E-3,-1.9777538E-2,-6.6618933E-3,-2.625365E-3,-8.621129E-3,-2.073713E-3,3.6287673E-3,-2.9041627E-3,9.660604E-3,8.152232E-3,1.3705341E-2,-2.8794128E-4,-4.786967E-3,-2.5894914E-3,3.8131385E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,-1,31,-1,-1,33,35,-1,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7054625E0,6.0688484E-1,1.3876917E0,4.832405E-1,2.5023532E-1,9.4388866E-1,2.9303676E-1,1.835103E-1,4.382766E-1,0E0,0E0,1.0468316E-1,4.2801547E-1,3.3220887E-2,2.614925E-1,3.7551725E-1,1.3047174E-1,0E0,2.445575E-1,0E0,0E0,2.4742104E-1,1.2905073E-1,0E0,2.3485433E-2,0E0,6.2113363E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,18,18,21,21,22,22,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,-1,32,-1,-1,34,36,-1,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,3.013037E6,6.627576E7,3.2420947E2,6.622819E5,6.677E3,3.4573945E3,2.1146445E4,5.178571E0,-2.0610558E-2,-4.649419E-3,4.9695907E0,1E0,1.5374186E3,1E0,1.0893594E6,1.06403265E1,-1.2546526E-2,8.564393E5,3.157098E-2,1.4733403E-2,1.1223402E6,1.2737E4,-8.440286E-3,3.0455479E3,1.14197675E-2,8.9E1,-1.9777538E-2,-6.6618933E-3,-2.625365E-3,-8.621129E-3,-2.073713E-3,3.6287673E-3,-2.9041627E-3,9.660604E-3,8.152232E-3,1.3705341E-2,-2.8794128E-4,-4.786967E-3,-2.5894914E-3,3.8131385E-3],"split_indices":[60,36,53,60,40,9,60,41,62,0,0,62,109,60,8,53,62,0,36,0,0,36,2,0,4,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.72E2,1.02E2,1.62E2,1E1,7.1E1,3.1E1,7.8E1,8.4E1,6E0,4E0,9E0,6.2E1,1.4E1,1.7E1,3.7E1,4.1E1,6E0,7.8E1,4E0,5E0,1.3E1,4.9E1,4E0,1E1,5E0,1.2E1,5E0,3.2E1,3.1E1,1E1,5.7E1,2.1E1,9E0,4E0,2.7E1,2.2E1,4E0,6E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.5111145E-3,-1.00758635E-1,1.4078632E-1,-8.7795235E-2,-3.1138134E-1,1.9300216E-1,-9.715943E-2,-1.4021263E-1,-3.625705E-2,-1.927791E-2,-6.527477E-3,1.2909508E-1,3.818892E-1,-1.8718022E-1,1.6940399E-3,-8.275625E-2,-1.8724231E-1,7.0627574E-3,-1.7342818E-1,8.7704934E-2,1.4976596E-2,3.0386819E-2,2.886043E-1,-4.788227E-3,-1.0122371E-2,4.2756526E-3,-2.9314735E-3,-9.469563E-3,-1.5730014E-3,-1.2773798E-3,-1.1615457E-2,-1.5101638E-3,4.1631334E-3,-4.6815965E-3,-1.29786795E-2,1.1693222E-2,2.635436E-3,1.7574273E-2,7.6722535E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.883796E0,4.149847E-1,1.4227252E0,4.1094697E-1,1.04471505E-1,1.0733232E0,1.8719327E-1,1.8986821E-1,4.706608E-1,0E0,0E0,5.480224E-1,4.0621138E-1,7.032752E-3,7.1466826E-2,2.0799468E-1,4.183402E-1,2.0566694E-1,1.1919159E-1,2.8937453E-1,0E0,0E0,1.5214634E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.0978662E6,9.996903E7,3.2420947E2,4.2993605E6,5.206759E3,1.8671203E-1,1.4084E4,4.91101E5,-1.927791E-2,-6.527477E-3,1.9894526E6,6.677E3,5.34E2,2.012E3,6.2734294E0,6.5833335E0,8.129243E5,6.156076E7,6.518218E6,1.4976596E-2,3.0386819E-2,7.9684106E9,-4.788227E-3,-1.0122371E-2,4.2756526E-3,-2.9314735E-3,-9.469563E-3,-1.5730014E-3,-1.2773798E-3,-1.1615457E-2,-1.5101638E-3,4.1631334E-3,-4.6815965E-3,-1.29786795E-2,1.1693222E-2,2.635436E-3,1.7574273E-2,7.6722535E-3],"split_indices":[60,36,53,60,36,60,42,9,37,0,0,36,9,0,0,62,61,36,53,53,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.61E2,1.12E2,1.53E2,8E0,9.2E1,2E1,7.5E1,7.8E1,4E0,4E0,7E1,2.2E1,1E1,1E1,3.5E1,4E1,6E1,1.8E1,5.9E1,1.1E1,4E0,1.8E1,4E0,6E0,4E0,6E0,9E0,2.6E1,1.2E1,2.8E1,4.1E1,1.9E1,1.2E1,6E0,8E0,5.1E1,9E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.2639436E-3,-8.636061E-2,1.439204E-1,-7.071299E-2,-3.0611143E-1,2.259547E-1,1.8230418E-2,-2.5667608E-1,-5.3900603E-2,-1.9120274E-2,-4.237741E-3,1.6117018E-1,4.1805103E-1,-3.241513E-2,8.777754E-3,-7.6393327E-3,-1.6975058E-2,2.7235942E-2,-9.813327E-2,1.8854268E-1,-4.397764E-3,2.4255758E-2,1.2459357E-2,-9.8366976E-2,8.020588E-2,-1.1373969E-3,3.7810619E-3,-1.2103636E-3,-7.293788E-3,4.5848847E-3,1.1390872E-2,-1.8290838E-3,-6.5715956E-3,8.717477E-3,9.0721835E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,-1,-1,25,27,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2535193E0,5.629121E-1,9.8572624E-1,4.8640227E-1,2.1885872E-1,6.5442514E-1,3.467402E-1,7.098025E-2,5.338279E-1,0E0,0E0,3.3418858E-1,8.1962824E-2,2.3814407E-1,0E0,0E0,0E0,1.5312463E-1,4.1163188E-1,1.9006169E-1,0E0,0E0,0E0,4.636231E-2,1.0208607E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,17,17,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,-1,-1,26,28,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1080475E3,3.0978662E6,5.7025972E7,5.178571E0,6.622819E5,5.206759E3,6.3901825E5,9.372475E-1,6.500994E0,-1.9120274E-2,-4.237741E-3,6.8734935E6,2.4816266E5,6.2150537E3,8.777754E-3,-7.6393327E-3,-1.6975058E-2,4.653861E2,1.2117E4,6.2711465E-1,-4.397764E-3,2.4255758E-2,1.2459357E-2,7.632764E7,1.3424457E0,-1.1373969E-3,3.7810619E-3,-1.2103636E-3,-7.293788E-3,4.5848847E-3,1.1390872E-2,-1.8290838E-3,-6.5715956E-3,8.717477E-3,9.0721835E-5],"split_indices":[60,36,53,62,40,60,36,35,61,0,0,55,41,4,0,0,0,60,9,35,0,0,0,53,43,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,1.69E2,9.5E1,1.59E2,1E1,5.7E1,3.8E1,1.2E1,1.47E2,6E0,4E0,4.4E1,1.3E1,3E1,8E0,8E0,4E0,5.2E1,9.5E1,4E1,4E0,6E0,7E0,1.9E1,1.1E1,2.7E1,2.5E1,4.4E1,5.1E1,1.7E1,2.3E1,9E0,1E1,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.7045098E-3,-7.839583E-2,1.3465284E-1,-2.5165224E-1,-6.418407E-2,5.9031375E-2,2.5977445E-1,-7.3266905E-3,-1.6936429E-2,-1.10465445E-1,-3.5125155E-3,2.6152054E-2,2.0037118E-2,2.1607919E-2,2.0407821E-1,-8.7018885E-2,-1.2038729E-2,4.0760163E-2,-1.1297283E-1,-5.1036235E-2,1.9133984E-1,1.529384E-1,1.4369114E-2,-2.0081457E-3,-6.6324915E-3,7.325742E-3,-5.304471E-4,-8.1181675E-3,5.755938E-3,6.0415505E-3,-3.4202163E-3,2.5032037E-3,1.1527527E-2,1.3497812E-3,8.97087E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,-1,21,23,-1,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8758905E0,4.1720927E-1,9.21391E-1,8.122331E-2,4.6011794E-1,7.678155E-1,3.360424E-1,0E0,0E0,3.0653405E-1,3.535241E-1,7.620484E-1,0E0,0E0,1.2350929E-1,1.9671214E-1,0E0,3.2359135E-1,3.3866256E-1,1.8348446E-1,1.4002305E-1,1.1344051E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,14,14,15,15,17,17,18,18,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,-1,22,24,-1,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,5.178571E0,1.4797393E0,9.372475E-1,4.2708307E2,2.7553815E2,6.677E3,-7.3266905E-3,-1.6936429E-2,1E0,3.2271808E7,7.2596356E5,2.0037118E-2,2.1607919E-2,1E0,9.360022E5,-1.2038729E-2,5.2183847E5,8.491E3,8.587809E4,3.1386855E11,2.40912E5,1.4369114E-2,-2.0081457E-3,-6.6324915E-3,7.325742E-3,-5.304471E-4,-8.1181675E-3,5.755938E-3,6.0415505E-3,-3.4202163E-3,2.5032037E-3,1.1527527E-2,1.3497812E-3,8.97087E-3],"split_indices":[60,62,47,35,60,64,9,0,0,92,53,36,0,0,84,56,0,56,2,36,39,37,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.75E2,9.8E1,1.2E1,1.63E2,6.2E1,3.6E1,8E0,4E0,9.2E1,7.1E1,5.8E1,4E0,6E0,3E1,8.1E1,1.1E1,5.1E1,2E1,4E1,1.8E1,2.2E1,8E0,4.7E1,3.4E1,1.5E1,3.6E1,1.6E1,4E0,4E0,3.6E1,6E0,1.2E1,6E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.3146847E-4,-1.0252857E-1,1.2483631E-1,-8.719631E-2,-3.1459346E-1,9.428457E-2,3.466703E-1,-1.5371905E-1,-2.9674547E-2,-5.894221E-3,-1.9228509E-2,6.649059E-2,4.0807164E-1,1.0666719E-2,2.0487934E-2,-1.8945529E-1,-6.0266133E-2,4.185615E-2,-7.432306E-2,1.9170286E-2,1.8984139E-1,2.3671616E-2,1.0146873E-2,-5.0432915E-3,-1.1397712E-2,1.4888474E-3,-4.295161E-3,-5.410418E-3,3.9045226E-3,-6.1355595E-4,-8.672138E-3,-1.5425166E-3,4.9498295E-3,1.0531037E-2,-1.9602953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5980308E0,4.7328937E-1,8.2804585E-1,5.5028176E-1,1.3878685E-1,9.583844E-1,6.6498995E-2,2.123394E-1,2.5541627E-1,0E0,0E0,6.0924554E-1,6.5858126E-2,0E0,0E0,1.9575393E-1,6.4178914E-2,2.2215313E-1,3.3831826E-1,3.618905E-1,2.7186286E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.8652206E9,1.9894526E6,2.8070276E2,6.4044494E5,2.7553815E2,7.3765434E1,9.4312E4,1.6533424E6,-5.894221E-3,-1.9228509E-2,1.5995066E0,1.088737E4,1.0666719E-2,2.0487934E-2,2.328E0,1.0215407E1,4.1749332E4,9.50996E0,5.8194575E5,8.419477E-1,2.3671616E-2,1.0146873E-2,-5.0432915E-3,-1.1397712E-2,1.4888474E-3,-4.295161E-3,-5.410418E-3,3.9045226E-3,-6.1355595E-4,-8.672138E-3,-1.5425166E-3,4.9498295E-3,1.0531037E-2,-1.9602953E-3],"split_indices":[60,5,36,60,36,64,66,37,58,0,0,50,4,0,0,64,64,40,62,36,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.53E2,1.26E2,1.44E2,9E0,1.12E2,1.4E1,6.6E1,7.8E1,4E0,5E0,1.04E2,8E0,8E0,6E0,4.7E1,1.9E1,3E1,4.8E1,7.6E1,2.8E1,4E0,4E0,2.1E1,2.6E1,5E0,1.4E1,6E0,2.4E1,3.2E1,1.6E1,4.8E1,2.8E1,2.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.8348909E-3,-1.0244059E-1,1.08923584E-1,-8.661916E-2,-3.0262154E-1,1.4962576E-1,-7.5382814E-2,-1.955423E-1,-5.8324352E-2,-5.6771203E-3,-1.8490758E-2,1.0443854E-1,3.2755315E-1,-1.2664315E-1,-3.2894178E-3,-1.06303155E-1,-2.5984678E-1,1.0225266E-2,-9.778994E-2,3.5171982E-2,1.9521934E-1,2.693649E-2,2.3254985E-1,-6.384929E-2,-9.1956295E-3,3.3041695E-3,-2.5981113E-3,-7.1285125E-3,-1.9609192E-3,-6.6446103E-3,-1.4771335E-2,-9.641608E-3,3.3001797E-3,-7.4244654E-3,5.035398E-4,-7.637554E-4,7.230864E-3,1.0246149E-2,-1.112614E-3,1.5002102E-2,5.2219317E-3,-2.480024E-4,-5.0434377E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0197504E0,4.2352223E-1,9.789946E-1,3.9849567E-1,1.27989E-1,8.180356E-1,8.827616E-2,1.2731636E-1,2.904725E-1,0E0,0E0,5.33908E-1,3.776772E-1,5.1496387E-2,4.7923233E-2,3.359823E-2,5.937612E-2,5.549154E-1,4.710055E-1,3.1803548E-1,2.271769E-1,0E0,1.5198052E-1,2.3451928E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.8652206E9,9.996903E7,1.775107E2,6.4044494E5,5.206759E3,1.8671203E-1,2.2304833E0,7.4500206E5,-5.6771203E-3,-1.8490758E-2,6.3901825E5,6.677E3,9.222717E-1,2.012E3,1.51E2,1.2027512E1,1.295E3,1.1565725E0,1.6864376E0,6.8734935E6,2.693649E-2,7.5253453E9,4E1,-9.1956295E-3,3.3041695E-3,-2.5981113E-3,-7.1285125E-3,-1.9609192E-3,-6.6446103E-3,-1.4771335E-2,-9.641608E-3,3.3001797E-3,-7.4244654E-3,5.035398E-4,-7.637554E-4,7.230864E-3,1.0246149E-2,-1.112614E-3,1.5002102E-2,5.2219317E-3,-2.480024E-4,-5.0434377E-3],"split_indices":[60,5,53,60,36,60,42,64,56,0,0,36,9,35,0,0,62,2,50,49,55,0,12,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,1.41E2,1.28E2,1.32E2,9E0,1.05E2,2.3E1,2.6E1,1.06E2,4E0,5E0,8.5E1,2E1,1.3E1,1E1,1.2E1,1.4E1,3.9E1,6.7E1,4.9E1,3.6E1,4E0,1.6E1,8E0,5E0,4E0,6E0,6E0,6E0,6E0,8E0,8E0,3.1E1,4.2E1,2.5E1,3.5E1,1.4E1,3.2E1,4E0,8E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.0181858E-3,-9.5756255E-2,1.0255134E-1,-7.354858E-2,-2.3915727E-1,1.5723592E-1,-2.4419857E-2,-1.3163959E-2,-5.686898E-2,-2.8265128E-1,-2.8013124E-3,4.1690564E-1,1.16320014E-1,-1.4931783E-1,2.2190062E-2,-7.038067E-6,-1.1278311E-1,-6.5108603E-3,-1.4786287E-2,2.5971679E-2,1.0643894E-2,8.064929E-2,2.1929973E-1,-1.3829821E-3,-9.815335E-3,9.49927E-2,-5.694542E-2,-2.2295967E-3,3.951583E-3,-1.9330584E-3,-8.610391E-3,5.7624495E-3,7.6884106E-5,1.2885571E-2,1.4420432E-3,7.9328794E-4,7.85468E-3,1.7045075E-3,-4.3499954E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.695608E0,4.44178E-1,9.064771E-1,4.336037E-1,1.3015819E-1,9.1834617E-1,2.3617277E-1,0E0,3.7834612E-1,3.6155105E-2,0E0,1.9763494E-1,2.7618253E-1,7.739705E-2,1.7824756E-1,2.5351846E-1,3.0747646E-1,0E0,0E0,0E0,0E0,2.1422702E-1,2.1895778E-1,0E0,0E0,9.1133386E-2,5.6404043E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,1E0,6.627576E7,5.178571E0,2.4875E4,6.677E3,8.647907E-4,-1.3163959E-2,8.623692E0,8.35E2,-2.8013124E-3,5.0655737E0,1.1946838E7,1.15E2,1.739E3,4.653861E2,1.5078E4,-6.5108603E-3,-1.4786287E-2,2.5971679E-2,1.0643894E-2,4.24975E5,4.679E3,-1.3829821E-3,-9.815335E-3,2.3329E4,2.7350403E5,-2.2295967E-3,3.951583E-3,-1.9330584E-3,-8.610391E-3,5.7624495E-3,7.6884106E-5,1.2885571E-2,1.4420432E-3,7.9328794E-4,7.85468E-3,1.7045075E-3,-4.3499954E-3],"split_indices":[60,92,53,62,9,9,50,0,62,2,0,62,1,10,0,60,9,0,0,0,0,37,0,0,0,2,36,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.44E2,1.29E2,1.26E2,1.8E1,9E1,3.9E1,8E0,1.18E2,1.4E1,4E0,1.1E1,7.9E1,1E1,2.9E1,5.9E1,5.9E1,4E0,1E1,5E0,6E0,6E1,1.9E1,4E0,6E0,1.5E1,1.4E1,3.8E1,2.1E1,3.1E1,2.8E1,3.8E1,2.2E1,1.4E1,5E0,8E0,7E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.432283E-3,-8.456559E-2,9.9877775E-2,-2.4925354E-1,-7.219812E-2,1.3183694E-1,-4.3459274E-2,-6.05118E-3,-1.5690375E-2,-4.351316E-2,-1.428227E-1,8.273477E-2,2.4740356E-1,4.842598E-3,-8.587809E-2,-2.0182057E-1,-2.1922028E-2,-8.746046E-2,-2.1257982E-1,5.0105184E-2,1.814211E-1,1.805791E-2,1.5287405E-1,-1.6790965E-1,-3.4968392E-3,-5.2506486E-3,-1.389984E-2,2.514117E-3,-2.9032193E-3,-6.4028725E-3,1.5257885E-3,-1.5437902E-2,-6.1465316E-3,5.42169E-3,-9.886223E-4,1.0336134E-2,7.3231314E-4,3.360595E-3,1.1568934E-2,-4.1112513E-3,-9.458521E-3,2.9497885E-3,-2.3599889E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3906376E0,2.8548336E-1,6.1499345E-1,6.306243E-2,2.7740616E-1,5.980594E-1,1.666602E-1,0E0,0E0,3.4079736E-1,1.358841E-1,2.4207723E-1,3.7808192E-1,0E0,1.3457009E-1,7.098845E-2,2.8676254E-1,1.53984E-1,1.2022334E-1,2.9307383E-1,1.3189769E-1,0E0,1.4672938E-1,1.1157066E-2,3.878139E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.26031E2,5.178571E0,9.996903E7,3.804992E5,1.06403265E1,3.5078225E3,2.7864855E11,-6.05118E-3,-1.5690375E-2,4.059859E-5,1.5225E4,1.1223402E6,4.0642028E6,4.842598E-3,7.381975E8,1.8965986E6,1.2117E4,1.1085E4,2.0289855E-2,2.1637352E7,1.7328871E6,1.805791E-2,7.773148E3,1.1810204E8,2.012E3,-5.2506486E-3,-1.389984E-2,2.514117E-3,-2.9032193E-3,-6.4028725E-3,1.5257885E-3,-1.5437902E-2,-6.1465316E-3,5.42169E-3,-9.886223E-4,1.0336134E-2,7.3231314E-4,3.360595E-3,1.1568934E-2,-4.1112513E-3,-9.458521E-3,2.9497885E-3,-2.3599889E-3],"split_indices":[60,62,53,36,62,60,39,0,0,46,9,36,55,0,7,36,9,9,65,53,55,0,60,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.48E2,1.32E2,9E0,1.39E2,1.08E2,2.4E1,5E0,4E0,1E2,3.9E1,7.7E1,3.1E1,5E0,1.9E1,1.1E1,8.9E1,2.3E1,1.6E1,5.9E1,1.8E1,1.1E1,2E1,9E0,1E1,7E0,4E0,3.1E1,5.8E1,1.6E1,7E0,5E0,1.1E1,3E1,2.9E1,1.4E1,4E0,1.2E1,8E0,4E0,5E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-5.6453827E-3,-7.5557195E-2,9.299645E-2,-6.2294193E-2,-2.555761E-1,6.579071E-2,3.0558148E-1,-1.362432E-1,-2.3637965E-2,-3.064689E-3,-1.6301926E-2,9.826401E-2,-6.118873E-2,9.891103E-3,1.877618E-2,-2.4510026E-1,-1.1144415E-1,-5.516808E-2,6.715674E-2,6.2532365E-2,2.3541038E-1,-1.26065E-1,2.9269299E-2,-1.4705287E-2,-5.6071277E-3,-5.9346617E-3,1.0818621E-3,1.8403296E-3,-4.1092397E-3,-1.6074609E-3,5.155277E-3,9.08301E-3,1.9004786E-3,1.4356905E-2,3.7613234E-3,-8.322513E-3,-2.4566525E-3,-2.3055845E-3,4.167303E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9448668E0,3.780315E-1,6.5656877E-1,4.3986082E-1,1.8122071E-1,4.38974E-1,3.299606E-2,1.1576122E-1,2.9856998E-1,0E0,0E0,3.969183E-1,1.3279858E-1,0E0,0E0,4.3557286E-2,1.0902351E-1,2.5325504E-1,1.2741378E-1,1.8660876E-1,1.7421758E-1,4.2054906E-2,5.312682E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,1.9894526E6,4.9054803E2,9.214292E7,9.996903E7,7.3765434E1,2.6239506E5,7.0712406E5,-3.064689E-3,-1.6301926E-2,5.318116E3,2.137881E5,9.891103E-3,1.877618E-2,7.385222E5,2.850657E0,1.0949861E1,4.249921E2,6.518218E6,5.7025972E7,5.535E3,2.8109E4,-1.4705287E-2,-5.6071277E-3,-5.9346617E-3,1.0818621E-3,1.8403296E-3,-4.1092397E-3,-1.6074609E-3,5.155277E-3,9.08301E-3,1.9004786E-3,1.4356905E-2,3.7613234E-3,-8.322513E-3,-2.4566525E-3,-2.3055845E-3,4.167303E-3],"split_indices":[60,5,36,63,7,53,66,36,36,0,0,60,41,0,0,59,49,64,60,53,53,9,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.64E2,1.16E2,1.54E2,1E1,1.04E2,1.2E1,5.2E1,1.02E2,4E0,6E0,8.3E1,2.1E1,8E0,4E0,8E0,4.4E1,7.6E1,2.6E1,6.7E1,1.6E1,1.2E1,9E0,4E0,4E0,3.9E1,5E0,2E1,5.6E1,8E0,1.8E1,8E0,5.9E1,1E1,6E0,6E0,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[6.4275595E-3,-7.3580384E-2,9.763641E-2,-1.1390833E-2,-6.304668E-2,1.4751208E-1,-1.1795181E-2,-1.4650843E-1,-4.067337E-2,2.3813225E-2,1.1900915E-1,-1.5596089E-1,4.6264276E-2,-5.7398994E-2,-1.9777535E-1,-9.026477E-3,-1.1520159E-1,8.7947436E-2,2.4358338E-1,-8.971938E-3,-2.8783502E-3,1.5727854E-1,-3.6482923E-2,-3.95767E-3,-1.1380718E-3,-1.069602E-2,-2.8319738E-3,-9.68799E-4,7.965687E-3,-7.3141707E-3,4.960778E-3,5.841153E-3,3.685954E-5,1.296835E-2,6.1309217E-3,1.2145145E-3,1.0856178E-2,-3.320184E-3,3.2495512E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9993832E0,2.5173193E-1,7.0036066E-1,0E0,2.5359786E-1,8.6576176E-1,3.5039294E-1,1.21130586E-1,2.607633E-1,0E0,2.9974508E-1,3.1445205E-2,2.8132153E-1,9.075645E-3,7.042807E-2,1.7643787E-1,3.383388E-1,2.3054326E-1,2.9780269E-2,0E0,0E0,1.2738836E-1,7.37543E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22],"right_children":[2,4,6,-1,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.26031E2,5.178571E0,6.627576E7,-1.1390833E-2,1.775107E2,5.0655737E0,1.08017064E8,2.2304833E0,1.7810251E1,2.3813225E-2,5.206759E3,1.0805848E3,1.2199979E10,2.1934064E7,4.9658173E2,9.39616E-1,1.0903429E0,3.291238E7,3.09483E2,-8.971938E-3,-2.8783502E-3,7.632764E7,1.4975258E4,-3.95767E-3,-1.1380718E-3,-1.069602E-2,-2.8319738E-3,-9.68799E-4,7.965687E-3,-7.3141707E-3,4.960778E-3,5.841153E-3,3.685954E-5,1.296835E-2,6.1309217E-3,1.2145145E-3,1.0856178E-2,-3.320184E-3,3.2495512E-3],"split_indices":[60,62,53,0,60,62,7,64,64,0,60,41,5,5,4,35,50,53,66,0,0,53,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.45E2,1.27E2,7E0,1.38E2,8.7E1,4E1,2.8E1,1.1E2,5E0,8.2E1,1.1E1,2.9E1,1.1E1,1.7E1,7.8E1,3.2E1,6.7E1,1.5E1,7E0,4E0,1.2E1,1.7E1,5E0,6E0,1.3E1,4E0,7.4E1,4E0,2.7E1,5E0,4.6E1,2.1E1,1E1,5E0,5E0,7E0,1.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-5.0064377E-3,-5.719735E-2,8.771015E-2,-2.2641023E-1,-4.326547E-2,1.4464538E-1,8.35257E-3,-5.737707E-3,-1.4935417E-2,-1.633894E-1,-2.6162364E-2,2.4779609E-1,8.654907E-2,-3.1976372E-2,1.597783E-1,-7.96044E-2,-1.1013664E-2,-3.8377542E-2,9.8941535E-2,1.3367528E-1,1.43987425E-2,3.5176575E-2,1.721036E-1,-5.7992034E-2,6.008951E-3,1.0503422E-2,2.7380285E-3,-5.0951242E-3,-9.299817E-4,-1.1901899E-3,-1.1813678E-2,8.53343E-3,1.0024084E-3,7.953358E-3,3.1248338E-3,-5.5490003E-4,6.309671E-3,2.3905926E-3,9.898068E-3,-5.0797705E-3,7.8392644E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,-1,29,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3163759E0,4.0040118E-1,4.4157946E-1,8.923024E-2,3.285531E-1,3.2013595E-1,2.6159638E-1,0E0,0E0,1.0801679E-1,2.2232491E-1,1.0641694E-1,1.6096166E-1,1.5129574E-1,4.8141286E-2,1.80614E-2,0E0,3.5145956E-1,8.264384E-2,1.1414364E-2,0E0,1.2192928E-1,6.0984045E-2,9.3919195E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,-1,30,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1080475E3,5.178571E0,5.7025972E7,9.372475E-1,3.0344595E2,1.1233E4,8.129243E5,-5.737707E-3,-1.4935417E-2,2.5E3,6.816E3,1.613246E3,1.9107767E0,1.4964736E4,9.040637E7,2.55E2,-1.1013664E-2,6.0671224E7,9.39E2,8.68379E5,1.43987425E-2,5.506676E11,3.108307E6,2.137881E5,6.008951E-3,1.0503422E-2,2.7380285E-3,-5.0951242E-3,-9.299817E-4,-1.1901899E-3,-1.1813678E-2,8.53343E-3,1.0024084E-3,7.953358E-3,3.1248338E-3,-5.5490003E-4,6.309671E-3,2.3905926E-3,9.898068E-3,-5.0797705E-3,7.8392644E-5],"split_indices":[60,62,53,35,4,9,36,0,0,37,2,60,50,63,53,0,0,53,0,55,0,39,1,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.73E2,9.7E1,1.2E1,1.61E2,5.6E1,4.1E1,7E0,5E0,1.9E1,1.42E2,1.9E1,3.7E1,3.3E1,8E0,1E1,9E0,1.3E2,1.2E1,8E0,1.1E1,2.4E1,1.3E1,2.9E1,4E0,4E0,4E0,6E0,4E0,1.24E2,6E0,5E0,7E0,4E0,4E0,1.7E1,7E0,4E0,9E0,1.5E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.2627862E-3,-6.600195E-2,8.184687E-2,-3.9167482E-2,-1.3655624E-1,1.341317E-1,-3.0523774E-3,-9.227897E-3,-2.5603727E-2,-1.6323091E-1,3.5671548E-3,9.334165E-2,3.1393322E-1,-3.937748E-2,1.1223148E-1,5.456276E-3,-1.0748211E-1,-2.4187009E-1,-9.928717E-2,6.75176E-2,1.0526004E-2,1.7683364E-2,7.2005293E-3,-6.2496264E-2,5.7876254E-3,8.057512E-3,-4.9060467E-4,2.2440886E-3,-3.5851572E-3,-1.6750878E-3,-7.0944303E-3,-6.8804375E-3,-1.82709E-2,3.8046967E-3,-7.4265744E-3,4.692352E-3,-4.956195E-4,-2.3510116E-4,-4.951041E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4641546E0,2.653914E-1,5.5971617E-1,2.2453403E-1,2.3530889E-1,5.4275787E-1,2.0961659E-1,0E0,2.4981448E-1,1.5198225E-1,0E0,2.1164078E-1,9.464109E-2,1.5288404E-1,9.443675E-2,2.634779E-1,8.386502E-2,1.5554994E-1,2.5025657E-1,1.5189618E-1,0E0,0E0,0E0,8.7333724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,23,23],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3350134E2,2.0020384E1,5.709406E7,5.160874E0,1.0472412E5,5.206759E3,8.129243E5,-9.227897E-3,7.8908873E0,2.368447E6,3.5671548E-3,1.9894526E6,1.7173885E8,1.4964736E4,9.4721675E-1,2.3233E4,1.5133574E1,4.0727316E7,7.4500206E5,1.031599E7,1.0526004E-2,1.7683364E-2,7.2005293E-3,5.1160636E11,5.7876254E-3,8.057512E-3,-4.9060467E-4,2.2440886E-3,-3.5851572E-3,-1.6750878E-3,-7.0944303E-3,-6.8804375E-3,-1.82709E-2,3.8046967E-3,-7.4265744E-3,4.692352E-3,-4.956195E-4,-2.3510116E-4,-4.951041E-3],"split_indices":[60,64,53,62,41,60,36,0,61,40,0,36,40,63,35,9,66,12,56,40,0,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,1.42E2,1.25E2,1.04E2,3.8E1,7.7E1,4.8E1,7E0,9.7E1,3.4E1,4E0,6.4E1,1.3E1,3.7E1,1.1E1,7.1E1,2.6E1,1.4E1,2E1,5.5E1,9E0,8E0,5E0,3.3E1,4E0,7E0,4E0,4.7E1,2.4E1,1.1E1,1.5E1,1E1,4E0,5E0,1.5E1,3.8E1,1.7E1,1.5E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[7.831876E-3,-3.6079604E-2,1.5142351E-1,-1.1763128E-1,-9.6816727E-4,2.3261422E-1,4.6240848E-2,-1.5202983E-1,-5.6512415E-2,3.8338907E-2,-8.693223E-2,2.273927E-2,1.8597789E-1,1.3915485E-1,6.1287177E-3,-2.1336113E-1,-8.3459E-2,-8.887115E-3,-1.2706165E-1,1.2001167E-1,4.069648E-3,-1.2874354E-1,6.540982E-2,8.023059E-2,2.1660781E-1,2.6422993E-3,8.890006E-3,-3.2723814E-2,4.3305745E-3,-3.0844326E-3,-1.3733147E-2,-5.2853674E-3,1.8315795E-3,-3.4914915E-3,4.0525943E-3,-7.558825E-3,-2.6295157E-3,3.0748749E-3,9.701547E-3,-3.5738873E-3,1.7162656E-3,-8.1500625E-3,-4.3274098E-4,7.834436E-3,-2.3132854E-3,6.3156486E-3,3.3337335E-4,1.0758531E-2,4.6784277E-3,-2.396552E-3,9.304801E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7772584E0,6.183797E-1,5.5287445E-1,1.2806451E-1,5.1694024E-1,3.589921E-1,1.1095651E-1,1.5462005E-1,8.2869E-2,2.9339245E-1,3.1416294E-1,0E0,8.950937E-2,2.8607607E-2,7.823686E-2,2.4103624E-1,8.567065E-2,1.10479E-1,1.658456E-2,1.3558906E-1,2.0635466E-1,2.1726203E-1,1.4181441E-1,3.6413524E-2,2.5313973E-2,0E0,0E0,1.7422162E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.077646E3,2.8070276E2,5.9337012E7,2.037744E10,3.2271808E7,5.0655737E0,5.0993966E9,2.4830703E4,7.8908873E0,5.6818184E-2,1.0207E4,2.273927E-2,8.182337E8,4.6E1,1.0526E4,2.8434807E3,5.5971736E-1,7.0120897E0,3.3E1,1.2347529E6,4.561433E2,1.0528782E1,3.005574E6,3.8228506E9,4.6E1,2.6422993E-3,8.890006E-3,1.3988934E2,4.3305745E-3,-3.0844326E-3,-1.3733147E-2,-5.2853674E-3,1.8315795E-3,-3.4914915E-3,4.0525943E-3,-7.558825E-3,-2.6295157E-3,3.0748749E-3,9.701547E-3,-3.5738873E-3,1.7162656E-3,-8.1500625E-3,-4.3274098E-4,7.834436E-3,-2.3132854E-3,6.3156486E-3,3.3337335E-4,1.0758531E-2,4.6784277E-3,-2.396552E-3,9.304801E-4],"split_indices":[60,60,53,39,53,62,12,41,61,65,2,0,7,3,9,55,46,61,3,36,60,62,1,5,3,0,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.15E2,6.5E1,6.4E1,1.51E2,3.6E1,2.9E1,4E1,2.4E1,1.04E2,4.7E1,4E0,3.2E1,8E0,2.1E1,2E1,2E1,1.5E1,9E0,3E1,7.4E1,3.7E1,1E1,8E0,2.4E1,4E0,4E0,1.5E1,6E0,8E0,1.2E1,1.6E1,4E0,9E0,6E0,5E0,4E0,2E1,1E1,2.1E1,5.3E1,2.6E1,1.1E1,5E0,5E0,4E0,4E0,2E1,4E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-4.1842987E-3,-6.007171E-2,9.166539E-2,-4.2257406E-2,-1.5573992E-1,1.2626387E-1,-3.35959E-2,-7.4384585E-2,2.0892376E-2,-2.7882376E-1,-6.138604E-2,2.9120243E-1,8.4988594E-2,-7.0757866E-2,4.9220813E-3,-3.372021E-2,-1.13499284E-1,1.0770947E-1,-6.005003E-2,-7.1096616E-3,-1.6425462E-2,2.2495787E-3,-1.525621E-1,4.3560127E-3,3.5408902E-1,-1.056873E-2,1.12506375E-1,-9.01926E-2,-7.7608105E-4,-4.645028E-3,4.5782863E-4,-6.6552414E-3,2.7543495E-3,2.6170076E-3,8.571696E-3,-5.51714E-3,1.4900278E-3,-1.1440844E-2,-1.2025724E-3,2.0929834E-2,6.5640244E-3,-3.7446364E-3,3.771828E-3,-1.5073045E-3,5.807827E-3,-5.9731863E-3,-2.6797093E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,-1,-1,-1,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4250561E0,2.797349E-1,4.2895925E-1,2.9167593E-1,2.848335E-1,4.994583E-1,1.2243393E-1,1.4727741E-1,3.508E-1,5.1556468E-2,1.6828342E-1,1.5194309E-1,1.6684914E-1,1.7335825E-2,0E0,1.4178048E-1,2.6697534E-1,8.521232E-2,1.475159E-1,0E0,0E0,0E0,1.0267627E-1,0E0,1.6902876E-1,1.0483919E-1,1.01192355E-1,8.893974E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,22,22,24,24,25,25,26,26,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,-1,-1,-1,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0688497E3,3.3207447E1,9.298568E7,4.561433E2,1.643614E6,6.677E3,1.21440664E8,8.110325E0,3.347E3,2.336E3,1.2117E4,5.314E3,1E0,6.597786E0,4.9220813E-3,1.295E3,3.4597E5,7.2929355E6,3.4288502E8,-7.1096616E-3,-1.6425462E-2,2.2495787E-3,4.888E3,4.3560127E-3,2.0004E4,5.4656273E-1,1.0625348E8,3.9952212E3,-7.7608105E-4,-4.645028E-3,4.5782863E-4,-6.6552414E-3,2.7543495E-3,2.6170076E-3,8.571696E-3,-5.51714E-3,1.4900278E-3,-1.1440844E-2,-1.2025724E-3,2.0929834E-2,6.5640244E-3,-3.7446364E-3,3.771828E-3,-1.5073045E-3,5.807827E-3,-5.9731863E-3,-2.6797093E-3],"split_indices":[60,64,53,60,40,9,40,62,2,11,9,9,109,61,0,2,37,53,7,0,0,0,2,0,10,35,7,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,1.67E2,9.7E1,1.42E2,2.5E1,7.6E1,2.1E1,9.4E1,4.8E1,1E1,1.5E1,1.4E1,6.2E1,1.7E1,4E0,4.7E1,4.7E1,2.3E1,2.5E1,5E0,5E0,7E0,8E0,4E0,1E1,1.4E1,4.8E1,1.2E1,5E0,1.8E1,2.9E1,4E1,7E0,1.5E1,8E0,1.5E1,1E1,4E0,4E0,6E0,4E0,8E0,6E0,4E0,4.4E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[6.2517715E-3,-3.0565914E-2,1.3407166E-1,-8.579644E-2,2.638087E-3,2.2572698E-1,3.1472143E-2,-1.3706839E-1,-4.4227287E-2,4.7415234E-2,-7.918623E-2,2.1130813E-2,1.7417617E-1,1.281923E-1,-1.1478024E-2,-9.8461375E-2,-1.5713664E-2,6.0675863E-2,-6.922454E-2,1.00784935E-1,5.288278E-3,-1.4788327E-1,-2.9742306E-2,6.438349E-2,2.2549565E-1,2.399839E-3,8.223956E-3,-5.4493994E-2,3.0034734E-3,-2.4171136E-3,-9.202386E-3,-1.3142215E-3,6.342667E-3,-4.852924E-3,5.487463E-4,6.7652096E-3,-3.9084023E-3,-4.280236E-3,2.046007E-3,-1.9437581E-3,-1.325503E-2,3.1304013E-3,-3.3257948E-3,-9.339301E-4,5.7187486E-3,4.169241E-3,1.1859346E-2,-5.9316686E-4,-4.340017E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,-1,31,33,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2417846E0,3.7617058E-1,5.477196E-1,1.580503E-1,4.7639254E-1,2.9277802E-1,1.2194165E-1,2.3447347E-1,1.1958103E-1,1.8832448E-1,1.5184754E-1,0E0,1.3761336E-1,2.5214642E-2,7.285219E-2,1.2934077E-1,0E0,6.583053E-2,1.0694286E-1,3.2954723E-1,1.8877226E-1,2.6894072E-1,1.2119578E-1,5.3173397E-2,5.4610312E-2,0E0,0E0,2.1953527E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,-1,32,34,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.077646E3,3.41043E2,5.9337012E7,2.1146445E4,3.1821228E7,5.0655737E0,5.0993966E9,4.7080725E-2,2.1087719E1,7.47E2,8.607021E2,2.1130813E-2,5.0777936E-1,4.6E1,1.0252E4,1.3431159E6,-1.5713664E-2,8.5539725E8,1.5064244E0,2.32789E1,2.9836262E8,5.6252275E5,1.7129352E3,1.8192074E0,2.7E1,2.399839E-3,8.223956E-3,1.0129378E12,3.0034734E-3,-2.4171136E-3,-9.202386E-3,-1.3142215E-3,6.342667E-3,-4.852924E-3,5.487463E-4,6.7652096E-3,-3.9084023E-3,-4.280236E-3,2.046007E-3,-1.9437581E-3,-1.325503E-2,3.1304013E-3,-3.3257948E-3,-9.339301E-4,5.7187486E-3,4.169241E-3,1.1859346E-2,-5.9316686E-4,-4.340017E-3],"split_indices":[60,60,53,41,53,62,12,46,66,0,60,0,65,3,9,36,0,39,50,64,7,36,4,50,3,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.62E2,2.04E2,5.8E1,7.6E1,1.28E2,3E1,2.8E1,3.3E1,4.3E1,8.3E1,4.5E1,4E0,2.6E1,8E0,2E1,2.9E1,4E0,8E0,3.5E1,3.6E1,4.7E1,1.8E1,2.7E1,9E0,1.7E1,4E0,4E0,1.3E1,7E0,2.1E1,8E0,4E0,4E0,2.4E1,1.1E1,2.9E1,7E0,1.3E1,3.4E1,1.1E1,7E0,8E0,1.9E1,4E0,5E0,4E0,1.3E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-1.953781E-3,-6.4126745E-2,8.011017E-2,-4.943846E-2,-1.9414945E-1,1.1030366E-1,-5.4648962E-2,-8.9543276E-2,5.301558E-4,-7.272177E-4,-1.3793101E-2,7.35192E-2,2.3737954E-1,9.104583E-3,-1.2502597E-1,-1.4310448E-1,-4.351422E-2,8.0192074E-2,-4.394464E-2,1.3486598E-1,2.4171267E-2,2.0098818E-2,1.6886911E-1,-3.8662307E-2,4.2939764E-3,-7.377444E-3,-3.175824E-3,-1.4455997E-2,-4.9076634E-3,4.948754E-3,-2.9961837E-3,5.3742696E-3,-3.7748646E-3,-6.8272976E-4,-7.011615E-3,7.397385E-3,-2.398179E-3,-4.1795714E-4,4.9132127E-3,1.0157785E-2,3.526597E-3,-2.9849417E-3,-2.1915448E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4439461E0,2.9788107E-1,5.0242764E-1,2.9262888E-1,2.9613978E-1,4.499898E-1,1.0433432E-1,1.9346803E-1,2.3735854E-1,0E0,0E0,2.3589975E-1,2.2646403E-1,5.5866282E-2,1.0655269E-2,1.973713E-1,1.5350932E-1,1.5178001E-1,1.3338909E-1,1.7720997E-1,1.24469884E-1,0E0,6.906152E-2,7.676374E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,4.56567E7,9.996903E7,3.41043E2,9.214292E7,5.206759E3,9.106645E-1,2.1146445E4,6.9789816E7,-7.272177E-4,-1.3793101E-2,3.36425E5,6.677E3,1.3174513E4,9.55288E-1,1.0893594E6,2.4385988E4,8.542832E0,2.4079119E1,6.8734935E6,5.506676E11,2.0098818E-2,3.09483E2,5.534958E-1,4.2939764E-3,-7.377444E-3,-3.175824E-3,-1.4455997E-2,-4.9076634E-3,4.948754E-3,-2.9961837E-3,5.3742696E-3,-3.7748646E-3,-6.8272976E-4,-7.011615E-3,7.397385E-3,-2.398179E-3,-4.1795714E-4,4.9132127E-3,1.0157785E-2,3.526597E-3,-2.9849417E-3,-2.1915448E-4],"split_indices":[60,53,53,60,7,60,35,41,12,0,0,37,9,4,35,53,41,61,64,55,39,0,66,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.6E2,1.21E2,1.45E2,1.5E1,9.9E1,2.2E1,8E1,6.5E1,6E0,9E0,7.8E1,2.1E1,1.2E1,1E1,3.6E1,4.4E1,2.3E1,4.2E1,3.4E1,4.4E1,4E0,1.7E1,8E0,4E0,5E0,5E0,5E0,3.1E1,5E0,3.9E1,1.9E1,4E0,3.4E1,8E0,3E1,4E0,3.2E1,1.2E1,1E1,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[7.016428E-3,-4.800191E-2,8.506017E-2,-3.635335E-2,-2.1793014E-1,5.8147267E-2,2.3203538E-1,-1.038279E-2,-8.967261E-2,-1.3564201E-2,-3.7904256E-3,3.639033E-2,2.4161789E-1,1.226481E-2,5.7393173E-3,-1.119294E-1,8.41179E-3,-2.1381732E-2,-1.3221078E-1,6.8183662E-3,1.1373264E-1,1.540821E-2,3.7588316E-3,-6.2858947E-3,-1.404565E-3,2.8912688E-3,-9.813211E-4,5.9950695E-4,-5.856172E-3,-8.433799E-3,-1.2670646E-3,-1.0130635E-3,3.7613984E-3,6.6356543E-3,-1.4080358E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.163419E0,3.0680516E-1,4.295271E-1,2.067786E-1,7.972714E-2,3.7441626E-1,2.950257E-2,1.9561118E-1,1.3951656E-1,0E0,0E0,1.9820783E-1,1.2152767E-1,0E0,0E0,2.7881116E-2,1.4198825E-1,7.446153E-2,1.5503854E-1,1.399179E-1,1.0967231E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,3.0978662E6,1.9894526E6,7.739782E0,4.2993605E6,2.7553815E2,9E0,1.7398648E2,1.5133574E1,-1.3564201E-2,-3.7904256E-3,1.5438564E0,1.8454938E9,1.226481E-2,5.7393173E-3,7.9178885E-2,6.216734E-2,9.3392765E11,2.1102592E8,7.9522475E5,8.413669E-1,1.540821E-2,3.7588316E-3,-6.2858947E-3,-1.404565E-3,2.8912688E-3,-9.813211E-4,5.9950695E-4,-5.856172E-3,-8.433799E-3,-1.2670646E-3,-1.0130635E-3,3.7613984E-3,6.6356543E-3,-1.4080358E-3],"split_indices":[60,36,36,61,36,64,8,60,66,0,0,47,7,0,0,65,50,39,7,36,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,1.58E2,1.11E2,1.49E2,9E0,9.5E1,1.6E1,1.01E2,4.8E1,5E0,4E0,8.6E1,9E0,1.1E1,5E0,1.5E1,8.6E1,1.9E1,2.9E1,6.3E1,2.3E1,5E0,4E0,1.1E1,4E0,3E1,5.6E1,1.5E1,4E0,1.9E1,1E1,4.6E1,1.7E1,1.9E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[6.3871974E-3,-3.173963E-2,1.1379496E-1,-1.1066346E-1,-1.3481528E-2,1.7172344E-1,2.6970962E-2,-1.3960779E-1,7.0575083E-4,1.0895795E-2,-7.731141E-2,1.8755512E-2,1.2999786E-1,6.845645E-3,2.5269429E-3,-6.26594E-2,-1.7879643E-1,-3.146239E-2,5.1711846E-2,-1.6461305E-1,-9.780136E-3,-2.2101612E-3,1.5235116E-1,-3.2645687E-2,3.9478396E-3,-4.6856166E-3,5.732149E-4,-1.2688062E-2,-4.85654E-3,-8.810449E-5,-5.184312E-3,5.2174567E-3,3.232417E-4,-1.3451915E-3,-1.01544885E-2,3.634753E-3,-2.694739E-3,3.8449098E-3,8.793388E-3,1.8859581E-3,-2.8481104E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,-1,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.125674E0,2.9103094E-1,3.574984E-1,1.4071685E-1,2.5913864E-1,3.6983132E-1,8.888018E-2,8.22773E-2,0E0,2.1082315E-1,2.6958722E-1,0E0,1.5851295E-1,0E0,7.890216E-2,3.651833E-2,1.1284411E-1,1.4363901E-1,1.6991636E-1,1.4503652E-1,1.21403664E-1,0E0,7.4488044E-2,4.339213E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,-1,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5699575E3,4.5135025E2,5.9337012E7,1E0,2.9277092E1,5.0655737E0,2.7828033E0,1.5133574E1,7.0575083E-4,4.561433E2,5.754E3,1.8755512E-2,3.108307E6,6.845645E-3,1.0526E4,2.9615752E-4,1.831115E4,1.3585858E1,7.47E2,2.2766894E5,2.7778377E11,-2.2101612E-3,1.1576279E9,2.7350403E5,3.9478396E-3,-4.6856166E-3,5.732149E-4,-1.2688062E-2,-4.85654E-3,-8.810449E-5,-5.184312E-3,5.2174567E-3,3.232417E-4,-1.3451915E-3,-1.01544885E-2,3.634753E-3,-2.694739E-3,3.8449098E-3,8.793388E-3,1.8859581E-3,-2.8481104E-3],"split_indices":[60,4,53,112,64,62,61,66,0,60,2,0,1,0,9,46,41,64,0,36,39,0,7,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.02E2,7.1E1,3.7E1,1.65E2,4.2E1,2.9E1,3E1,7E0,1.2E2,4.5E1,5E0,3.7E1,4E0,2.5E1,1.1E1,1.9E1,5.9E1,6.1E1,1.9E1,2.6E1,4E0,3.3E1,1.8E1,7E0,7E0,4E0,7E0,1.2E1,4.4E1,1.5E1,2.5E1,3.6E1,6E0,1.3E1,9E0,1.7E1,1.3E1,2E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.3117217E-3,-5.8269E-2,6.331777E-2,-3.3478305E-2,-1.2034082E-1,4.3454796E-2,2.3472422E-1,-8.520905E-2,2.9101465E-2,1.9773014E-2,-1.5713707E-1,2.736222E-2,2.2417396E-1,7.750722E-3,1.3590137E-2,-1.4670075E-1,-5.3839058E-2,6.7136936E-2,-2.370149E-2,5.8030672E-3,-4.1644005E-3,-2.0135084E-1,1.0689548E-3,1.0159254E-1,-5.050416E-3,1.4364742E-2,3.4047228E-3,-3.480695E-3,-9.641076E-3,-4.8113265E-3,6.8555935E-4,-2.4583484E-3,4.7936635E-3,-2.710201E-3,4.307677E-3,-1.2818551E-2,-5.5669663E-3,6.6788946E-3,1.563546E-3,-2.715408E-3,2.2560263E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0163361E0,2.0832613E-1,4.56236E-1,3.2620952E-1,2.0417798E-1,3.5448754E-1,6.4871907E-3,9.819645E-2,9.3976974E-2,1.1681886E-1,2.518736E-1,2.7720314E-1,1.08861566E-1,0E0,0E0,6.441507E-2,1.338698E-1,1.2590602E-1,8.797488E-2,0E0,0E0,1.2130892E-1,0E0,9.7976625E-2,2.3899066E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,21,21,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.26031E2,1.7810251E1,1.9894526E6,5.377697E2,1.4218562E5,2.7553815E2,7.3765434E1,7.056912E7,7.25E2,6.62E2,1E0,2.1508584E7,1.8454938E9,7.750722E-3,1.3590137E-2,3.7248906E3,3.6E1,3.4013605E-3,8.129243E5,5.8030672E-3,-4.1644005E-3,2.368447E6,1.0689548E-3,5E0,1.3526E4,1.4364742E-2,3.4047228E-3,-3.480695E-3,-9.641076E-3,-4.8113265E-3,6.8555935E-4,-2.4583484E-3,4.7936635E-3,-2.710201E-3,4.307677E-3,-1.2818551E-2,-5.5669663E-3,6.6788946E-3,1.563546E-3,-2.715408E-3,2.2560263E-3],"split_indices":[60,64,36,4,36,64,66,7,0,0,84,53,7,0,0,55,3,65,36,0,0,40,0,8,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.37E2,1.36E2,9.9E1,3.8E1,1.23E2,1.3E1,5.4E1,4.5E1,8E0,3E1,1.14E2,9E0,8E0,5E0,1.7E1,3.7E1,2.6E1,1.9E1,4E0,4E0,2.4E1,6E0,3.4E1,8E1,5E0,4E0,9E0,8E0,2.1E1,1.6E1,6E0,2E1,1.5E1,4E0,1.1E1,1.3E1,2E1,1.4E1,4E1,4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.1381755E-3,-4.015542E-2,6.9048055E-2,-1.9872302E-2,-1.2941483E-1,1.1655842E-1,4.385897E-5,-8.7272234E-2,8.715801E-3,-1.6556093E-1,1.9688567E-3,7.560037E-2,2.1020263E-1,-3.4717426E-2,1.1822785E-1,3.3699423E-3,-1.1759863E-1,3.933789E-2,-4.7380824E-2,-3.663656E-2,-2.2407612E-1,1.516232E-1,1.1758094E-2,2.4584042E-1,3.148117E-3,4.0325113E-3,-5.3943936E-2,7.038063E-3,2.4409944E-3,-3.0580126E-3,4.62256E-3,-1.1010333E-2,-2.878998E-3,-2.7127643E-4,3.6493966E-3,-4.833684E-3,-8.322444E-5,5.4913457E-4,-3.5853493E-3,-1.2539799E-2,-4.1510537E-3,7.6184473E-3,3.5445376E-3,-4.3345224E-3,2.3092385E-3,7.889781E-3,1.4575751E-2,-3.18395E-3,1.6547258E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.526853E-1,2.92533E-1,3.3767158E-1,2.5895673E-1,1.9378668E-1,2.1807307E-1,1.8075833E-1,1.1035031E-1,1.6505446E-1,1.8039149E-1,0E0,2.1176991E-1,7.41691E-2,8.4674016E-2,1.4482319E-2,8.115732E-2,1.8583834E-1,1.1242996E-1,8.906941E-2,1.8952057E-2,8.536881E-2,7.9316795E-3,1.0638328E-1,1.9508958E-2,0E0,0E0,4.3501325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0897255E3,2.6921875E1,5.7025972E7,4.3724646E2,1.1484905E5,5.936149E1,8.129243E5,3.7248906E3,9.22E2,4.61E2,1.9688567E-3,1.8905495E6,2.88E3,7.272321E4,9.040637E7,8.27725E3,1.831115E4,2.427E3,1.251E3,7.856E3,4.9230335E1,8.47598E1,3.108307E6,3.4E1,3.148117E-3,4.0325113E-3,1.00152E5,7.038063E-3,2.4409944E-3,-3.0580126E-3,4.62256E-3,-1.1010333E-2,-2.878998E-3,-2.7127643E-4,3.6493966E-3,-4.833684E-3,-8.322444E-5,5.4913457E-4,-3.5853493E-3,-1.2539799E-2,-4.1510537E-3,7.6184473E-3,3.5445376E-3,-4.3345224E-3,2.3092385E-3,7.889781E-3,1.4575751E-2,-3.18395E-3,1.6547258E-3],"split_indices":[60,64,53,63,41,64,36,55,0,0,0,55,0,36,53,56,41,2,0,9,64,66,1,3,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,1.62E2,1.02E2,1.33E2,2.9E1,6E1,4.2E1,3.9E1,9.4E1,2.4E1,5E0,4.3E1,1.7E1,3.3E1,9E0,1E1,2.9E1,6.1E1,3.3E1,8E0,1.6E1,1.9E1,2.4E1,1.3E1,4E0,4E0,2.9E1,5E0,4E0,6E0,4E0,8E0,2.1E1,2.9E1,3.2E1,1.4E1,1.9E1,4E0,4E0,1.1E1,5E0,1.5E1,4E0,6E0,1.8E1,8E0,5E0,2.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-3.4266562E-4,-4.982749E-2,5.8398206E-2,-4.131914E-2,-9.801049E-3,8.410596E-2,-5.703143E-2,-1.7833007E-2,-9.5695525E-2,5.607249E-2,1.8823002E-1,-1.0588339E-1,-1.1308122E-2,-2.7576159E-3,-1.3818544E-1,-6.929023E-3,-1.16377965E-1,7.0869334E-2,-7.017664E-2,1.7329078E-2,1.2181851E-1,-1.8145917E-3,-7.122877E-3,2.127167E-3,-1.9337921E-3,-6.630691E-3,4.2307322E-4,-1.2237696E-2,-1.2565367E-3,-1.5753851E-3,1.00115E-3,-1.577692E-4,-6.0462914E-3,-2.0782233E-3,4.244037E-3,-8.8484574E-4,-4.9309665E-3,1.0381237E-3,6.935114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8192735E-1,1.937465E-1,3.6979145E-1,1.7708619E-1,0E0,2.8426033E-1,5.0177135E-2,1.7924058E-1,7.630175E-2,1.5628853E-1,2.1655577E-1,2.9601522E-2,2.488766E-2,1.5151767E-1,1.484137E-1,7.786065E-3,5.694756E-2,1.8462667E-1,1.4875263E-2,0E0,4.522416E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,3.0978662E6,9.996903E7,7.7524557E0,-9.801049E-3,5.206759E3,7.381975E8,4.347012E8,5.2646E4,6.995407E6,6.677E3,1.3509979E10,2.012E3,5.178571E0,2.8314105E4,3.607307E7,8.623692E0,7.157903E7,1.4456249E7,1.7329078E-2,2.881356E-1,-1.8145917E-3,-7.122877E-3,2.127167E-3,-1.9337921E-3,-6.630691E-3,4.2307322E-4,-1.2237696E-2,-1.2565367E-3,-1.5753851E-3,1.00115E-3,-1.577692E-4,-6.0462914E-3,-2.0782233E-3,4.244037E-3,-8.8484574E-4,-4.9309665E-3,1.0381237E-3,6.935114E-3],"split_indices":[60,36,53,61,0,60,7,7,1,55,9,5,0,62,41,5,62,7,40,0,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,1.45E2,1.22E2,1.39E2,6E0,1E2,2.2E1,9.8E1,4.1E1,8E1,2E1,1E1,1.2E1,8.8E1,1E1,8E0,3.3E1,7.2E1,8E0,4E0,1.6E1,5E0,5E0,4E0,8E0,6E0,8.2E1,4E0,6E0,4E0,4E0,4E0,2.9E1,1.1E1,6.1E1,4E0,4E0,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.3863375E-3,-5.80328E-2,5.989253E-2,-4.805752E-2,-1.0375174E-2,8.3406106E-2,-3.937908E-2,-4.0058862E-2,-1.1786947E-2,6.9134004E-2,2.247887E-1,-6.524929E-2,3.5521416E-3,-1.00423634E-1,-2.4184803E-2,5.0895337E-2,8.299492E-3,1.360742E-2,5.0216955E-3,-2.1059461E-2,-9.66367E-2,-6.668194E-3,-2.4070912E-3,-4.7883054E-4,-5.7634055E-3,1.8001967E-3,1.045526E-2,7.0072996E-4,-2.1999413E-3,-6.0307942E-3,-1.0289423E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,21,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.475006E-1,2.3451304E-1,2.997317E-1,2.2261164E-1,0E0,1.9229561E-1,8.0400914E-2,1.2789112E-1,0E0,1.8337539E-1,4.145977E-2,2.7091973E-2,0E0,5.228898E-2,1.5044644E-1,1.636163E-1,0E0,0E0,0E0,1.0421995E-2,2.7694859E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,22,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,3.013037E6,9.996903E7,6.0073395E8,-1.0375174E-2,2.7553815E2,1.4975258E4,1.7398648E2,-1.1786947E-2,1.9894526E6,1.088737E4,9.106645E-1,3.5521416E-3,5.0515676E-1,8.903381E0,5.4349E4,8.299492E-3,1.360742E-2,5.0216955E-3,8.66076E1,7.381975E8,-6.668194E-3,-2.4070912E-3,-4.7883054E-4,-5.7634055E-3,1.8001967E-3,1.045526E-2,7.0072996E-4,-2.1999413E-3,-6.0307942E-3,-1.0289423E-3],"split_indices":[60,36,53,7,0,64,4,60,0,36,4,35,0,35,61,2,0,0,0,64,7,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.46E2,1.26E2,1.39E2,7E0,1.02E2,2.4E1,1.35E2,4E0,9.4E1,8E0,2E1,4E0,2.7E1,1.08E2,8.2E1,1.2E1,4E0,4E0,9E0,1.1E1,1.3E1,1.4E1,9.6E1,1.2E1,7.8E1,4E0,4E0,5E0,7E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[4.4954307E-3,-4.5983575E-2,6.8287104E-2,-5.7552572E-2,4.9953848E-2,3.766518E-2,1.455354E-1,-4.8612874E-2,-1.7929353E-1,-1.9123511E-2,6.96913E-3,2.4612535E-2,1.1483405E-2,7.745452E-2,1.952069E-1,-1.0655844E-1,-2.5903793E-2,-1.2820261E-2,-2.0384935E-3,1.4403615E-3,-2.4124202E-3,-4.6253926E-3,9.790008E-2,1.2366532E-1,1.7227685E-5,9.879947E-3,5.407309E-3,-6.409315E-3,-1.3654503E-3,-7.469502E-4,-7.6049003E-3,7.6444144E-4,-2.787665E-3,5.686722E-3,1.4379848E-3,2.8919247E-3,7.0795324E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,-1,-1,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.822384E-1,1.7268962E-1,2.8074282E-1,1.4038491E-1,1.2198654E-1,2.370859E-1,9.985745E-2,1.6698962E-1,1.0816589E-1,2.0088963E-2,0E0,1.8044598E-1,0E0,5.6944832E-2,3.435254E-3,8.626181E-2,1.2405021E-1,0E0,0E0,0E0,0E0,7.342207E-2,3.7125304E-2,8.658767E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,-1,-1,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,1E0,1.0527294E6,6.156076E7,1.725E3,2.9028345E2,3.8657358E6,4.3724646E2,3.9E1,2.1857248E4,6.96913E-3,1.5995066E0,1.1483405E-2,7.208905E5,2.4716E4,7.774228E-2,6.0073395E8,-1.2820261E-2,-2.0384935E-3,1.4403615E-3,-2.4124202E-3,1.4539318E-1,5.6431256E5,1.9894526E6,1.7227685E-5,9.879947E-3,5.407309E-3,-6.409315E-3,-1.3654503E-3,-7.469502E-4,-7.6049003E-3,7.6444144E-4,-2.787665E-3,5.686722E-3,1.4379848E-3,2.8919247E-3,7.0795324E-3],"split_indices":[60,112,36,53,2,64,40,63,3,41,0,50,0,40,2,65,7,0,0,0,0,46,36,36,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.52E2,1.2E2,1.36E2,1.6E1,8.7E1,3.3E1,1.28E2,8E0,1E1,6E0,8.3E1,4E0,1.5E1,1.8E1,3.5E1,9.3E1,4E0,4E0,4E0,6E0,6E1,2.3E1,9E0,6E0,1.3E1,5E0,2.4E1,1.1E1,8.8E1,5E0,4.4E1,1.6E1,1.6E1,7E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-8.471147E-4,-4.6182446E-2,5.6405522E-2,-3.9163478E-2,-1.1072691E-2,3.637769E-2,2.0258531E-1,-2.8460281E-2,-1.808106E-1,2.0979797E-2,1.9829078E-1,6.599342E-3,1.1864551E-2,-1.0405181E-1,-9.647264E-3,-2.1189952E-3,-1.2108688E-2,7.431334E-2,-2.109879E-2,1.269802E-2,3.7351109E-3,-2.7744253E-3,-7.0797415E-3,1.4456391E-3,-2.7408542E-3,4.443432E-3,-9.3301607E-4,-1.988888E-3,2.8402908E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.930258E-1,1.9379956E-1,3.380948E-1,2.1468526E-1,0E0,2.5761753E-1,8.7759495E-3,1.9240244E-1,9.867284E-2,2.1925537E-1,5.9355736E-2,0E0,0E0,4.9520046E-2,2.2709666E-1,0E0,0E0,9.2227444E-2,1.032922E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17,18,18],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.0073395E8,1.9894526E6,6.8652206E9,-1.1072691E-2,2.6014774E2,2.567762E6,1.295E3,6.4044494E5,3.3624732E7,1.088737E4,6.599342E-3,1.1864551E-2,5.734444E6,7.34E2,-2.1189952E-3,-1.2108688E-2,3.2395E4,3.76978E7,1.269802E-2,3.7351109E-3,-2.7744253E-3,-7.0797415E-3,1.4456391E-3,-2.7408542E-3,4.443432E-3,-9.3301607E-4,-1.988888E-3,2.8402908E-3],"split_indices":[60,7,36,5,0,64,55,2,36,53,4,0,0,53,0,0,0,9,40,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.48E2,1.17E2,1.44E2,4E0,1.04E2,1.3E1,1.35E2,9E0,9.6E1,8E0,8E0,5E0,2.6E1,1.09E2,4E0,5E0,4.2E1,5.4E1,4E0,4E0,1.5E1,1.1E1,6E1,4.9E1,3.4E1,8E0,4.3E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[3.9934046E-3,-1.5935697E-2,1.0143383E-1,-4.7182538E-2,8.463276E-3,1.4049073E-2,6.4180344E-2,-8.3365805E-2,-1.7789032E-2,5.374204E-2,-5.833308E-2,1.183576E-1,-1.9267414E-2,-1.5468733E-1,-2.0121098E-2,3.6068797E-2,-7.485853E-2,9.824209E-2,-9.916882E-3,-1.2681662E-1,-3.0497536E-2,9.9147175E-5,1.4730096E-1,-7.4534826E-2,3.2382451E-3,-1.456931E-3,-1.0496073E-2,-4.2700055E-3,2.005725E-3,-2.3663838E-3,4.8793377E-3,3.0447405E-3,-4.655314E-3,5.1976973E-3,-2.047495E-3,3.8514717E-3,-1.3634319E-3,-1.0788369E-2,-1.3403309E-3,3.756699E-4,-3.5014527E-3,8.040364E-3,1.398404E-3,-4.985774E-3,-1.3056063E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.375809E-1,1.7574921E-1,3.3709177E-1,1.0593584E-1,3.9631423E-1,0E0,1.8764025E-1,1.998694E-1,1.778975E-1,2.2193466E-1,9.716506E-2,8.376625E-2,8.9426845E-2,1.8214679E-1,1.19663954E-1,1.8888734E-1,1.103286E-1,1.01198524E-1,6.329624E-2,1.507337E-1,6.9297455E-2,0E0,5.8617294E-2,1.4075033E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,4.2708307E2,2.2779044E-3,2.1857248E4,3.291238E7,1.4049073E-2,3.09483E2,2.8490282E8,1.446662E6,6.7842064E1,8.607021E2,4.747723E6,7.644773E-1,2.8434807E3,1.3324517E5,1.8452284E2,8.587809E4,4.8539555E6,1.4154823E-1,3.6820934E1,5.1160636E11,9.9147175E-5,7.794012E7,1.8790482E9,3.2382451E-3,-1.456931E-3,-1.0496073E-2,-4.2700055E-3,2.005725E-3,-2.3663838E-3,4.8793377E-3,3.0447405E-3,-4.655314E-3,5.1976973E-3,-2.047495E-3,3.8514717E-3,-1.3634319E-3,-1.0788369E-2,-1.3403309E-3,3.756699E-4,-3.5014527E-3,8.040364E-3,1.398404E-3,-4.985774E-3,-1.3056063E-3],"split_indices":[60,60,65,41,53,0,66,5,56,66,60,40,42,55,40,60,36,55,46,64,39,0,53,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.29E2,4.6E1,1E2,1.29E2,6E0,4E1,4.4E1,5.6E1,7.7E1,5.2E1,2.4E1,1.6E1,2E1,2.4E1,2.9E1,2.7E1,4.5E1,3.2E1,1.4E1,3.8E1,5E0,1.9E1,1E1,6E0,8E0,1.2E1,1.1E1,1.3E1,1.3E1,1.6E1,4E0,2.3E1,4.1E1,4E0,5E0,2.7E1,6E0,8E0,2.1E1,1.7E1,1.5E1,4E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-2.4034858E-3,-1.6533416E-2,1.19489126E-1,-5.86549E-2,4.3750172E-3,1.2983275E-2,6.483612E-2,-1.0214734E-1,-1.9658009E-2,4.078952E-2,-5.395549E-2,3.3678438E-3,1.2090138E-1,-1.1460367E-2,-7.324261E-2,6.5880954E-2,-4.2872775E-2,1.7498422E-2,1.2680641E-1,-9.584041E-2,-1.5315839E-2,1.6810031E-3,-2.3174633E-3,1.5335907E-3,7.3911473E-3,-4.9569393E-3,-1.0343269E-3,6.273022E-3,-1.7213741E-4,-3.9433557E-4,-4.7382023E-3,2.2751438E-3,-2.162123E-3,9.548526E-4,7.6099155E-3,2.7134952E-3,-5.5688103E-3,3.9852727E-3,-2.0200962E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8078442E-1,2.2043017E-1,2.392757E-1,1.3876578E-1,3.5904565E-1,0E0,7.885623E-2,1.47219E-1,9.2184015E-2,2.0613757E-1,1.0372934E-1,2.3196207E-2,3.6310658E-2,0E0,5.7006434E-2,4.949906E-2,7.327278E-2,1.7308077E-1,8.636209E-2,1.2762147E-1,1.05632536E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.5891304E1,3.41043E2,1.6064256E-2,2.1146445E4,3.3624732E7,1.2983275E-2,3.9174525E5,1.0893594E6,2.1087719E1,1.1223402E6,1.2286531E3,5.3868E4,3.5935396E-1,-1.1460367E-2,1.369E3,5.225893E4,9.258696E0,2.2287E4,3.993E4,2.325945E9,4.541179E1,1.6810031E-3,-2.3174633E-3,1.5335907E-3,7.3911473E-3,-4.9569393E-3,-1.0343269E-3,6.273022E-3,-1.7213741E-4,-3.9433557E-4,-4.7382023E-3,2.2751438E-3,-2.162123E-3,9.548526E-4,7.6099155E-3,2.7134952E-3,-5.5688103E-3,3.9852727E-3,-2.0200962E-3],"split_indices":[64,60,65,41,53,0,36,53,66,36,60,10,65,0,2,41,62,9,37,5,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.49E2,2.8E1,8.2E1,1.67E2,6E0,2.2E1,3.8E1,4.4E1,1.03E2,6.4E1,1.1E1,1.1E1,5E0,3.3E1,9E0,3.5E1,8.2E1,2.1E1,3E1,3.4E1,7E0,4E0,4E0,7E0,1.9E1,1.4E1,4E0,5E0,2.3E1,1.2E1,5.5E1,2.7E1,6E0,1.5E1,4E0,2.6E1,7E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[6.7138695E-4,-4.4880837E-2,5.540211E-2,-3.461925E-2,-1.019413E-2,7.6242924E-2,-3.9431144E-2,-9.797035E-2,-2.0245334E-2,3.8959254E-2,1.4650139E-1,-6.5063104E-2,3.0321046E-3,-4.526886E-2,-1.3360019E-1,-6.416324E-3,-1.0121502E-1,1.3799776E-1,1.7425545E-2,1.3406477E-2,1.1268125E-1,-8.623541E-2,-2.217891E-4,-3.7635218E-3,-7.7413814E-4,-2.4725331E-3,-7.2639766E-3,-7.9039007E-4,4.8521585E-3,-6.6503366E-3,1.4467175E-3,7.4778143E-3,3.2840476E-3,-1.4626401E-3,1.9732462E-3,6.767003E-3,1.3531849E-3,-5.8705756E-3,-1.8113933E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.806113E-1,2.6224977E-1,2.4808022E-1,1.2732223E-1,0E0,2.6148075E-1,6.6355616E-2,4.277344E-2,1.3035905E-1,1.4261663E-1,1.3845193E-1,2.3819938E-2,0E0,1.10798385E-2,2.0495862E-2,1.23208776E-1,1.0201211E-1,7.950544E-3,7.2676346E-2,0E0,8.137688E-2,2.2078276E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,6.8652206E9,9.996903E7,1.142E3,-1.019413E-2,3.5078225E3,1.4975258E4,5.322348E7,2.5356756E1,6.518218E6,1.4164306E-3,7.2596356E5,3.0321046E-3,1.0893594E6,4.3301904E2,2.2009405E1,6.013E3,4E1,5.4656273E-1,1.3406477E-2,3.09483E2,7.381975E8,-2.217891E-4,-3.7635218E-3,-7.7413814E-4,-2.4725331E-3,-7.2639766E-3,-7.9039007E-4,4.8521585E-3,-6.6503366E-3,1.4467175E-3,7.4778143E-3,3.2840476E-3,-1.4626401E-3,1.9732462E-3,6.767003E-3,1.3531849E-3,-5.8705756E-3,-1.8113933E-3],"split_indices":[60,5,53,2,0,60,4,5,64,53,65,36,0,53,41,64,2,3,35,0,66,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.48E2,1.23E2,1.41E2,7E0,1.01E2,2.2E1,2.5E1,1.16E2,6.7E1,3.4E1,1.8E1,4E0,1.1E1,1.4E1,1E2,1.6E1,1.1E1,5.6E1,5E0,2.9E1,1.3E1,5E0,4E0,7E0,4E0,1E1,9.2E1,8E0,1.2E1,4E0,7E0,4E0,1.9E1,3.7E1,2E1,9E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.0204675E-3,-2.3778036E-2,7.860227E-2,-1.3984196E-1,-1.587799E-2,1.1393086E-1,1.4193154E-2,-2.1340703E-3,-9.823091E-3,2.862562E-2,-3.5539757E-2,1.42730875E-2,8.320723E-2,5.3815157E-3,-1.4458896E-2,-6.826148E-3,4.740201E-2,-4.2260922E-2,4.300178E-3,-1.9202358E-3,1.051848E-1,1.5509196E-2,-4.5836914E-2,3.0882326E-3,-6.9083174E-4,1.9408354E-4,-2.9809852E-3,-1.3207413E-4,5.468808E-3,2.824044E-3,-6.93938E-4,-3.152014E-3,-7.170159E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,-1,21,-1,23,25,-1,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.649522E-1,1.8666312E-1,1.6449332E-1,7.945675E-2,1.7077115E-1,2.5734228E-1,8.11331E-2,0E0,0E0,2.0630676E-1,1.209404E-1,0E0,1.1969006E-1,0E0,2.1398362E-2,0E0,6.853353E-2,1.3508257E-1,0E0,0E0,5.322939E-2,1.7742392E-2,6.4653363E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,14,14,16,16,17,17,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,-1,22,-1,24,26,-1,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5472196E3,5.178571E0,6.627576E7,9.1002154E-1,6.3274317E0,5.0655737E0,3.3855326E0,-2.1340703E-3,-9.823091E-3,3.8396814E-5,1E0,1.42730875E-2,3.108307E6,5.3815157E-3,1.1256308E8,-6.826148E-3,6.7842064E1,4.61E2,4.300178E-3,-1.9202358E-3,2.7898628E5,2.7350403E5,3.312524E6,3.0882326E-3,-6.9083174E-4,1.9408354E-4,-2.9809852E-3,-1.3207413E-4,5.468808E-3,2.824044E-3,-6.93938E-4,-3.152014E-3,-7.170159E-4],"split_indices":[60,62,53,35,61,62,62,0,0,46,96,0,1,0,53,0,66,0,0,0,36,36,37,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.05E2,7.2E1,1.2E1,1.93E2,4.6E1,2.6E1,6E0,6E0,5.9E1,1.34E2,5E0,4.1E1,5E0,2.1E1,5E0,5.4E1,1.28E2,6E0,6E0,3.5E1,1.1E1,1E1,4.1E1,1.3E1,4.2E1,8.6E1,4E0,3.1E1,4E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.6312275E-3,-1.9263467E-2,8.05169E-2,-7.379985E-2,-4.363461E-3,1.1229304E-2,6.087243E-2,7.60158E-2,-1.0824691E-1,-1.4688718E-2,6.0471345E-2,9.467849E-2,-1.1999603E-2,6.386699E-3,-8.697324E-5,-7.1025565E-2,-9.28381E-3,-8.884909E-2,-4.062141E-3,1.5255253E-1,-2.2095151E-2,1.1114886E-1,-4.117553E-4,-6.696552E-2,2.996282E-3,-4.296606E-3,2.311195E-3,-7.850506E-6,-5.256675E-3,3.532819E-3,-7.940268E-4,2.9588789E-3,8.923726E-3,2.992545E-3,-3.8974956E-3,6.04132E-3,1.5905177E-3,-5.022803E-3,-1.2572547E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,-1,25,-1,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.585668E-1,1.7032239E-1,1.745154E-1,2.4046034E-1,1.1201463E-1,0E0,1.3416596E-1,4.4224918E-2,1.14300996E-1,1.131213E-1,1.7823556E-1,6.414148E-2,8.061804E-2,0E0,0E0,8.03621E-2,0E0,4.0403947E-2,1.366015E-1,2.7621835E-2,7.632987E-2,4.465753E-2,0E0,1.5399106E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,-1,26,-1,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.077646E3,2.070165E4,1.4164306E-3,2.6E1,4.5E1,1.1229304E-2,3.09483E2,8.6E1,1.33511E5,7.157903E7,1.5176752E8,1E0,2.3329864E0,6.386699E-3,-8.697324E-5,1.3655606E6,-9.28381E-3,4.2631797E4,4.573327E5,8.70029E6,9.14595E5,7.794012E7,-4.117553E-4,1.8790482E9,2.996282E-3,-4.296606E-3,2.311195E-3,-7.850506E-6,-5.256675E-3,3.532819E-3,-7.940268E-4,2.9588789E-3,8.923726E-3,2.992545E-3,-3.8974956E-3,6.04132E-3,1.5905177E-3,-5.022803E-3,-1.2572547E-3],"split_indices":[60,56,65,3,3,0,66,10,12,7,12,92,43,0,0,40,0,56,56,12,1,53,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,2.09E2,5.8E1,4.4E1,1.65E2,5E0,5.3E1,8E0,3.6E1,1.43E2,2.2E1,3.6E1,1.7E1,4E0,4E0,2.7E1,9E0,1.7E1,1.26E2,1E1,1.2E1,3.1E1,5E0,1E1,7E0,2.3E1,4E0,4E0,1.3E1,1.7E1,1.09E2,4E0,6E0,5E0,7E0,2.4E1,7E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.6052106E-3,-2.8001808E-2,5.2021828E-2,-1.9939292E-2,-1.3466665E-1,8.2869925E-2,3.608008E-4,3.5979718E-2,-3.2609727E-2,-1.02487365E-2,-1.1182874E-3,1.33868875E-2,6.3016064E-2,5.5341246E-3,-1.4628091E-2,6.8102535E-3,-8.376362E-3,-1.1197135E-1,-2.4524122E-2,-1.1280433E-2,8.285439E-2,3.6387683E-3,-2.7230257E-2,-1.7425773E-3,4.305005E-3,-2.205656E-3,-7.245689E-3,1.7701143E-3,-2.026211E-3,-2.2581194E-3,3.409773E-3,4.2736544E-3,-1.4459548E-3,-1.8721637E-3,2.955137E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,-1,23,25,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.27316E-1,1.3405278E-1,1.8323919E-1,1.067425E-1,1.0135794E-1,2.7885148E-1,8.079344E-2,1.4026174E-1,7.553977E-2,0E0,0E0,0E0,9.968713E-2,0E0,4.936546E-2,0E0,6.659364E-2,2.4457112E-2,1.3911149E-1,5.197562E-2,6.328106E-2,0E0,4.681606E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,16,16,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,-1,24,26,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.216184E2,6.8652206E9,5.9337012E7,8.588E3,9.32784E5,5.0655737E0,2.7828033E0,3.131995E6,8.27725E3,-1.02487365E-2,-1.1182874E-3,1.33868875E-2,1E0,5.5341246E-3,1.0866135E3,6.8102535E-3,9.2664665E-1,8.3949E4,4.573327E5,9.3405694E-1,3.09483E2,3.6387683E-3,1.36332E5,-1.7425773E-3,4.305005E-3,-2.205656E-3,-7.245689E-3,1.7701143E-3,-2.026211E-3,-2.2581194E-3,3.409773E-3,4.2736544E-3,-1.4459548E-3,-1.8721637E-3,2.955137E-3],"split_indices":[60,5,53,9,37,62,61,53,56,0,0,0,109,0,60,0,35,12,56,35,66,0,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.58E2,1.14E2,1.48E2,1E1,7.1E1,4.3E1,2.7E1,1.21E2,5E0,5E0,5E0,6.6E1,4E0,3.9E1,7E0,2E1,1E1,1.11E2,1.4E1,5.2E1,4E0,3.5E1,1.6E1,4E0,5E0,5E0,2.6E1,8.5E1,1E1,4E0,4.8E1,4E0,3.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.2135604E-3,-1.2528309E-2,8.1777684E-2,-1.9828515E-2,8.758971E-2,1.1493084E-2,5.1125932E-2,-5.543397E-2,4.3585373E-4,-6.7939505E-4,1.2614793E-1,1.00502774E-1,-1.7550187E-2,-3.836494E-2,-8.814922E-3,8.7022506E-2,-9.026854E-3,7.0923604E-3,2.848259E-3,-3.7969707E-4,1.2955517E-1,-6.1605636E-2,2.7845914E-3,-6.448574E-3,-1.7099059E-4,5.2958E-5,6.4710192E-3,5.5862986E-3,-7.2785E-4,1.7371568E-3,7.7824756E-3,-4.4473363E-3,-1.0503777E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,-1,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4384662E-1,1.610691E-1,2.2904655E-1,1.4721465E-1,6.1055645E-2,0E0,1.4079525E-1,1.6446762E-1,1.0811964E-1,0E0,1.0196611E-2,7.689498E-2,6.5604225E-2,2.3558047E-1,0E0,5.9591956E-2,1.0712391E-1,0E0,0E0,0E0,6.249857E-2,1.4083851E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,20,20,21,21],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,-1,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,1E0,2.2779044E-3,2.1146445E4,4.2864978E2,1.1493084E-2,3.09483E2,2.9432115E6,1.0644666E6,-6.7939505E-4,8.63332E5,4.747723E6,2.8159826E0,4.4669402E-1,-8.814922E-3,6.740874E5,2.4385988E4,7.0923604E-3,2.848259E-3,-3.7969707E-4,5.0777936E-1,1.8790482E9,2.7845914E-3,-6.448574E-3,-1.7099059E-4,5.2958E-5,6.4710192E-3,5.5862986E-3,-7.2785E-4,1.7371568E-3,7.7824756E-3,-4.4473363E-3,-1.0503777E-3],"split_indices":[60,96,65,41,60,0,66,36,40,0,1,40,43,35,0,55,41,0,0,0,65,7,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.63E2,2.17E2,4.6E1,2.03E2,1.4E1,6E0,4E1,7.3E1,1.3E2,4E0,1E1,2.3E1,1.7E1,6.6E1,7E0,1.2E1,1.18E2,6E0,4E0,5E0,1.8E1,1.1E1,6E0,1.6E1,5E1,5E0,7E0,5E0,1.13E2,6E0,1.2E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.7926455E-3,-1.6193103E-2,7.56889E-2,-1.8467335E-4,-7.639471E-2,1.6740227E-1,2.9067282E-2,-2.343478E-2,7.580798E-2,-9.964643E-2,6.6467943E-3,1.2044126E-2,1.1507652E-1,-2.2768581E-2,8.4829696E-2,-4.8113924E-2,6.8880315E-3,1.290908E-1,1.8915268E-2,-1.5963526E-1,-5.4215565E-2,3.4081668E-3,-1.9535078E-3,7.23217E-3,2.3311516E-3,-4.127319E-2,2.0580727E-3,2.0147912E-4,1.2162551E-1,7.0316746E-4,-3.165824E-3,-5.4461225E-3,9.2896074E-4,7.935401E-3,4.3807006E-3,1.9897174E-3,-3.1073478E-3,-3.2989837E-3,-9.045274E-3,2.3942238E-3,-3.715902E-3,-2.1142884E-4,-2.4036386E-3,2.4643836E-3,6.719451E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,-1,-1,-1,-1,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8182288E-1,2.1104345E-1,2.3885614E-1,3.0920625E-1,8.948177E-2,6.354225E-2,1.1466068E-1,1.00609116E-1,1.2184577E-1,8.945769E-2,3.951411E-2,0E0,2.9944003E-2,2.8063338E-2,5.5714533E-2,9.789693E-2,1.0385649E-1,1.4439672E-2,4.7011323E-2,3.452313E-2,6.610515E-2,0E0,0E0,0E0,0E0,6.5779667E-3,0E0,0E0,1.3717949E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,-1,-1,-1,-1,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2737E4,3.2271808E7,1.096E3,7.608277E2,1.07431E1,1.6064256E-2,4.0827688E5,6.9163686E-1,9.686559E6,5.36E3,1.1710234E3,1.2044126E-2,7.357764E0,1.4861412E0,5.0777936E-1,1.0781069E5,1.0135136E0,1.5429025E3,1.9540394E7,7.5773336E7,3.607011E1,3.4081668E-3,-1.9535078E-3,7.23217E-3,2.3311516E-3,1.1740655E5,2.0580727E-3,2.0147912E-4,3.7264E4,7.0316746E-4,-3.165824E-3,-5.4461225E-3,9.2896074E-4,7.935401E-3,4.3807006E-3,1.9897174E-3,-3.1073478E-3,-3.2989837E-3,-9.045274E-3,2.3942238E-3,-3.715902E-3,-2.1142884E-4,-2.4036386E-3,2.4643836E-3,6.719451E-3],"split_indices":[2,53,0,60,62,65,36,35,59,2,4,0,62,50,65,36,64,4,53,7,64,0,0,0,0,36,0,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.18E2,5.6E1,1.73E2,4.5E1,1.8E1,3.8E1,1.33E2,4E1,3.5E1,1E1,5E0,1.3E1,2E1,1.8E1,7.3E1,6E1,2E1,2E1,1.4E1,2.1E1,4E0,6E0,7E0,6E0,1.6E1,4E0,6E0,1.2E1,1.8E1,5.5E1,5E0,5.5E1,7E0,1.3E1,1.6E1,4E0,5E0,9E0,4E0,1.7E1,4E0,1.2E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-2.367116E-3,-1.6322367E-2,8.3630145E-2,-2.9698187E-2,5.3450234E-2,4.1501503E-2,1.6145435E-1,-2.1217855E-2,-9.976107E-2,1.2356894E-1,2.3480835E-3,8.085947E-2,-4.1825708E-2,1.1786614E-2,4.1892007E-3,-5.409311E-2,5.8355955E-3,-1.6677882E-1,-2.3674134E-2,1.4658443E-3,1.7542386E-1,2.9661786E-3,-1.8474553E-2,1.1740359E-1,9.440202E-4,-3.918089E-3,4.5182908E-4,-1.6052758E-3,-4.6145357E-3,2.1811472E-3,-9.2710246E-4,-1.0271478E-2,-2.5130103E-3,1.780277E-3,-4.8903483E-3,4.54651E-3,9.672408E-3,-3.6985984E-3,8.816339E-4,7.2126784E-3,1.7940976E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,-1,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2659578E-1,2.1997155E-1,1.1788723E-1,1.1493212E-1,1.3587207E-1,8.8651344E-2,6.33181E-2,1.5773752E-1,1.03134334E-1,7.052231E-2,3.0919962E-2,3.7294306E-2,2.0946981E-2,0E0,0E0,6.746583E-2,1.0667981E-1,5.7325274E-2,6.0711317E-2,0E0,5.8288276E-3,0E0,4.3429893E-2,2.7763724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,-1,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3396984E7,2.077646E3,1.7227725E0,7.294459E7,3.9247762E6,7.269805E6,3.541162E-1,3.41043E2,2.2270758E1,5.031977E6,3.291238E7,1.2117E4,1.8790482E9,1.1786614E-2,4.1892007E-3,1.358226E8,7.25E2,2.7123457E3,2.083122E6,1.4658443E-3,1.0630312E2,2.9661786E-3,5.9145586E3,1.096E3,9.440202E-4,-3.918089E-3,4.5182908E-4,-1.6052758E-3,-4.6145357E-3,2.1811472E-3,-9.2710246E-4,-1.0271478E-2,-2.5130103E-3,1.780277E-3,-4.8903483E-3,4.54651E-3,9.672408E-3,-3.6985984E-3,8.816339E-4,7.2126784E-3,1.7940976E-3],"split_indices":[40,60,47,53,55,55,46,60,66,1,53,9,7,0,0,7,0,4,1,0,64,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,2.33E2,3.7E1,1.96E2,3.7E1,2.5E1,1.2E1,1.76E2,2E1,1.5E1,2.2E1,1.7E1,8E0,4E0,8E0,7.9E1,9.7E1,1E1,1E1,6E0,9E0,5E0,1.7E1,1E1,7E0,4E0,4E0,5.7E1,2.2E1,3.7E1,6E1,6E0,4E0,6E0,4E0,4E0,5E0,6E0,1.1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.124094E-3,-1.7050944E-2,5.7664324E-2,-1.1686295E-1,-1.0239508E-2,7.736976E-2,-1.56030115E-2,-2.2798583E-3,-8.617925E-3,2.939551E-2,-2.9673366E-2,1.5795603E-1,5.3730976E-2,-3.722471E-2,2.212657E-3,-5.155899E-3,4.5951832E-2,3.1935915E-3,-7.316173E-2,1.175767E-2,2.4388488E-3,-2.1831356E-2,7.269329E-2,-1.0541495E-2,-3.2710058E-3,-1.491601E-3,2.7226664E-3,-5.179797E-4,5.7812924E-3,-4.5385403E-3,-1.5954225E-4,-3.999825E-3,2.157071E-3,4.059184E-3,-9.879339E-5,-1.4049724E-3,5.313575E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,-1,25,27,29,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2409242E-1,1.381753E-1,1.1815211E-1,5.2292064E-2,1.4953269E-1,1.1343041E-1,2.6565354E-2,0E0,0E0,1.5495315E-1,1.8604657E-1,1.2965569E-1,7.466164E-2,1.1885304E-2,0E0,0E0,6.182617E-2,1.339809E-1,9.885019E-2,0E0,0E0,5.3211205E-2,4.7660887E-2,4.3218806E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,-1,26,28,30,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5472196E3,5.178571E0,9.820072E7,9.372475E-1,6.3274317E0,6.677E3,1.4975258E4,-2.2798583E-3,-8.617925E-3,3.8396814E-5,1.6672E4,1.4364E4,1E0,2.2980049E10,2.212657E-3,-5.155899E-3,1.292E3,2.0662694E-1,1.9274853E0,1.175767E-2,2.4388488E-3,5.6942026E3,3.09483E2,8.92851E6,-3.2710058E-3,-1.491601E-3,2.7226664E-3,-5.179797E-4,5.7812924E-3,-4.5385403E-3,-1.5954225E-4,-3.999825E-3,2.157071E-3,4.059184E-3,-9.879339E-5,-1.4049724E-3,5.313575E-4],"split_indices":[60,62,53,35,61,9,4,0,0,46,9,10,109,5,0,0,2,46,65,0,0,4,66,1,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.04E2,8E1,1.2E1,1.92E2,6.3E1,1.7E1,7E0,5E0,6.3E1,1.29E2,1.3E1,5E1,1.3E1,4E0,6E0,5.7E1,7.4E1,5.5E1,6E0,7E0,1E1,4E1,8E0,5E0,8E0,4.9E1,6.7E1,7E0,4E1,1.5E1,5E0,5E0,3.3E1,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.3679695E-4,-2.5444321E-2,5.295069E-2,-1.271998E-1,-1.7912112E-2,2.9637601E-2,1.23662174E-1,-2.2623679E-3,-9.672405E-3,-2.6111715E-2,8.0591194E-2,7.0667945E-2,-2.293727E-2,1.666666E-1,4.302615E-2,-1.6905438E-2,-7.736697E-3,6.638943E-3,6.843058E-4,-4.2178933E-2,1.0115033E-1,-7.16907E-2,3.692495E-3,5.52181E-3,1.0442198E-2,4.961053E-3,-1.395308E-3,-1.0715444E-3,4.162229E-3,-3.4974533E-4,-3.1457874E-3,2.4681466E-3,7.4980245E-3,-1.666438E-3,-4.3848744E-3,3.8664632E-3,-1.5663406E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,-1,-1,-1,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7596402E-1,1.4411232E-1,1.4470023E-1,7.3786214E-2,1.4668052E-1,1.5046531E-1,6.911856E-2,0E0,0E0,2.1405858E-1,5.6385823E-2,1.3842025E-1,4.0511783E-2,1.2144446E-2,4.5984264E-2,1.0939185E-1,0E0,0E0,0E0,7.618971E-3,8.221206E-2,5.747229E-3,6.644143E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,-1,-1,-1,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2286531E3,5.178571E0,1.7948325E0,9.1002154E-1,6.816E3,3.043758E7,8.812601E-1,-2.2623679E-3,-9.672405E-3,6.0671224E7,1.027E3,3.3478114E-1,3.402369E7,5.501913E9,1.265917E7,1E0,-7.736697E-3,6.638943E-3,6.843058E-4,1.7099724E3,3.0110518E3,7.9766E0,5.8282824E7,5.52181E-3,1.0442198E-2,4.961053E-3,-1.395308E-3,-1.0715444E-3,4.162229E-3,-3.4974533E-4,-3.1457874E-3,2.4681466E-3,7.4980245E-3,-1.666438E-3,-4.3848744E-3,3.8664632E-3,-1.5663406E-3],"split_indices":[60,62,47,35,2,59,65,0,0,53,0,35,56,5,40,96,0,0,0,60,60,62,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.9E2,8.9E1,1.2E1,1.78E2,6.8E1,2.1E1,7E0,5E0,1.65E2,1.3E1,3.8E1,3E1,1.3E1,8E0,1.56E2,9E0,6E0,7E0,8E0,3E1,1E1,2E1,9E0,4E0,4E0,4E0,1.48E2,8E0,4E0,4E0,1.8E1,1.2E1,5E0,5E0,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[5.8174073E-3,-1.20211085E-2,6.1136775E-2,-1.989166E-2,6.607439E-2,8.418778E-2,-1.0256596E-2,-1.2168811E-2,-1.4235102E-1,-2.346111E-3,1.00679874E-1,5.7937596E-2,1.4754988E-1,2.634672E-3,-3.5421856E-2,-1.0253095E-1,-3.1690393E-3,-8.013718E-3,-3.1995755E-3,1.4013088E-1,-2.876792E-4,8.324431E-2,-2.8735045E-3,2.0090261E-1,1.9765026E-3,-2.5488655E-3,-4.79874E-4,-2.1532269E-3,-8.640772E-3,4.0559933E-4,-3.333961E-3,8.40455E-3,3.4237024E-3,4.6418086E-3,9.261153E-4,1.2760412E-2,3.889171E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,29,-1,-1,31,-1,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.671751E-1,1.2731856E-1,1.09478936E-1,1.7512025E-1,8.2078904E-2,7.46147E-2,3.0896002E-2,1.4361832E-1,1.3313338E-2,0E0,6.415197E-2,1.1788436E-1,6.952065E-2,0E0,5.9028883E-3,6.718998E-2,1.3500717E-1,0E0,0E0,1.7110243E-2,0E0,3.2365665E-2,0E0,5.648935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,19,19,21,21,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,30,-1,-1,32,-1,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.077646E3,7.2838904E7,9.820072E7,3.013037E6,5.3153326E2,9.821139E3,3.3855326E0,5.178571E0,6.622819E5,-2.346111E-3,6.7842064E1,2.4029039E2,8.812601E-1,2.634672E-3,3.312524E6,3.804992E5,1.9975014E7,-8.013718E-3,-3.1995755E-3,9.39E2,-2.876792E-4,5.1E1,-2.8735045E-3,3.0291426E-1,1.9765026E-3,-2.5488655E-3,-4.79874E-4,-2.1532269E-3,-8.640772E-3,4.0559933E-4,-3.333961E-3,8.40455E-3,3.4237024E-3,4.6418086E-3,9.261153E-4,1.2760412E-2,3.889171E-3],"split_indices":[60,40,53,36,4,4,62,62,40,0,66,66,65,0,37,36,40,0,0,0,0,3,0,46,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,2.04E2,6.5E1,1.86E2,1.8E1,4.9E1,1.6E1,1.76E2,1E1,4E0,1.4E1,3.6E1,1.3E1,4E0,1.2E1,1.5E1,1.61E2,6E0,4E0,1E1,4E0,3E1,6E0,8E0,5E0,6E0,6E0,1E1,5E0,1.38E2,2.3E1,5E0,5E0,2.3E1,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.8575594E-3,-1.6758442E-2,5.8173157E-2,-1.0051013E-1,-9.6548535E-3,7.819564E-2,-1.8044144E-2,-3.7592974E-2,-8.104755E-3,2.2000287E-2,-3.2179464E-2,4.9936526E-2,1.382194E-1,-5.0878465E-2,9.7376766E-4,-2.852864E-3,-3.820347E-5,-3.6369964E-2,3.832829E-2,-1.7440373E-3,-8.097715E-2,9.270007E-2,-9.855781E-3,1.8876037E-1,6.670507E-2,-3.19964E-3,-1.016857E-3,4.638762E-4,-3.8644413E-3,2.56507E-3,-1.8507597E-3,-5.7409382E-3,8.489929E-4,-5.030469E-3,3.687372E-3,2.503035E-3,5.5689076E-3,2.69946E-3,-1.8742022E-3,1.1578678E-2,3.487039E-3,1.0116816E-3,4.5164274E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.026246E-1,1.21586725E-1,1.1281328E-1,6.940955E-2,1.3689472E-1,9.147844E-2,2.1666681E-2,8.9079635E-3,0E0,7.763957E-2,1.6619873E-1,1.05747804E-1,5.2467972E-2,3.2893922E-3,0E0,0E0,0E0,4.061044E-2,8.824281E-2,1.7600915E-1,2.0463642E-1,1.9419983E-2,4.04195E-2,5.184725E-2,1.0481857E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5699575E3,5.178571E0,9.820072E7,9.372475E-1,6.5142856E0,5.804289E3,9.5891304E1,8.591E3,-8.104755E-3,1.46E3,1.643E4,3.858136E6,8.812601E-1,5.576E3,9.7376766E-4,-2.852864E-3,-3.820347E-5,1.1826522E3,1.3237829E7,7.6077003E0,1.251E3,5.5326223E9,2.1637352E7,2.0867E4,1.625E3,-3.19964E-3,-1.016857E-3,4.638762E-4,-3.8644413E-3,2.56507E-3,-1.8507597E-3,-5.7409382E-3,8.489929E-4,-5.030469E-3,3.687372E-3,2.503035E-3,5.5689076E-3,2.69946E-3,-1.8742022E-3,1.1578678E-2,3.487039E-3,1.0116816E-3,4.5164274E-3],"split_indices":[60,62,53,35,61,60,64,2,0,2,9,55,65,9,0,0,0,41,58,62,0,5,53,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.05E2,7.2E1,1.5E1,1.9E2,5.7E1,1.5E1,9E0,6E0,7.9E1,1.11E2,4E1,1.7E1,8E0,7E0,5E0,4E0,1.7E1,6.2E1,6.9E1,4.2E1,2.3E1,1.7E1,9E0,8E0,4E0,4E0,9E0,8E0,5.1E1,1.1E1,9E0,6E1,3.6E1,6E0,1.1E1,1.2E1,5E0,1.2E1,5E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.7676535E-3,-9.7376695E-3,9.949977E-2,-2.1826744E-2,3.8569637E-2,1.1199317E-2,5.76746E-2,-7.512769E-3,-1.714976E-2,1.0362506E-1,5.858835E-3,9.151322E-2,4.5330795E-3,-7.010881E-2,-6.1056353E-3,1.3407801E-1,2.0864548E-3,-3.539202E-2,5.7719328E-2,2.2001425E-3,5.9122816E-3,8.794043E-4,-6.378697E-4,-1.8450024E-3,-7.903174E-3,-1.2493156E-3,1.8388721E-3,7.5166835E-3,3.326167E-3,2.0048604E-3,-3.4610967E-3,9.816614E-4,4.001074E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.989454E-1,1.398316E-1,1.5431312E-1,1.234689E-1,1.0153761E-1,0E0,4.0944576E-2,0E0,1.0836079E-1,2.2334248E-2,7.263936E-2,1.6452804E-2,2.9431896E-3,9.072195E-2,1.5117002E-1,6.250322E-3,0E0,6.2938854E-2,1.4080096E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.74915E1,1.6394367E0,1.6064256E-2,1.0893594E6,1.1244E4,1.1199317E-2,6.971004E7,-7.512769E-3,1.4164306E-3,6.7179665E1,2.144E5,5.2442E4,1.7E1,6.8652206E9,7.54579E5,6.93E2,2.0864548E-3,2.06513E5,7.702E3,2.2001425E-3,5.9122816E-3,8.794043E-4,-6.378697E-4,-1.8450024E-3,-7.903174E-3,-1.2493156E-3,1.8388721E-3,7.5166835E-3,3.326167E-3,2.0048604E-3,-3.4610967E-3,9.816614E-4,4.001074E-3],"split_indices":[64,50,65,53,9,0,53,0,65,66,37,10,8,5,36,0,0,1,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,2.37E2,2.7E1,1.9E2,4.7E1,5E0,2.2E1,5E0,1.85E2,1.5E1,3.2E1,1.3E1,9E0,3.1E1,1.54E2,9E0,6E0,1.8E1,1.4E1,7E0,6E0,5E0,4E0,2.5E1,6E0,1.06E2,4.8E1,5E0,4E0,6E0,1.2E1,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-8.3088217E-4,-1.6782394E-2,4.1292187E-2,-1.2228039E-1,-9.922723E-3,6.10166E-2,-2.4558274E-2,-1.6495052E-3,-8.237426E-3,3.1736314E-2,-2.8899161E-2,3.9505616E-2,1.4933576E-1,-2.2555043E-3,-4.557921E-3,-4.382254E-3,4.269727E-2,9.201845E-3,-5.4749217E-2,5.571024E-2,-2.9174287E-3,9.506563E-3,1.8170804E-3,1.1722066E-3,-1.1670571E-3,2.6481466E-3,-2.3738164E-3,1.2676194E-3,-4.4820467E-3,-1.0462032E-3,-4.170952E-3,3.2330146E-3,-3.0252622E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,-1,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8345702E-1,1.4207555E-1,9.89131E-2,5.2345008E-2,1.4874817E-1,1.04290664E-1,8.571918E-3,0E0,0E0,8.614123E-2,1.2755759E-1,8.3545804E-2,6.0913414E-2,0E0,7.5102565E-3,0E0,7.979251E-2,1.0619621E-1,8.632974E-2,7.860875E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,16,16,17,17,18,18,19,19],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,-1,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5699575E3,5.178571E0,9.820072E7,3.0855316E5,6.3274317E0,9.821139E3,2.0814263E9,-1.6495052E-3,-8.237426E-3,2.50391E-5,1.2117E4,1.0369473E7,8.812601E-1,-2.2555043E-3,2.012E3,-4.382254E-3,1.3237829E7,1E0,1.7810251E1,7.904E0,-2.9174287E-3,9.506563E-3,1.8170804E-3,1.1722066E-3,-1.1670571E-3,2.6481466E-3,-2.3738164E-3,1.2676194E-3,-4.4820467E-3,-1.0462032E-3,-4.170952E-3,3.2330146E-3,-3.0252622E-3],"split_indices":[60,62,53,36,61,4,7,0,0,46,9,55,65,0,0,0,58,92,64,61,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.97E2,7.4E1,1.1E1,1.86E2,5.7E1,1.7E1,5E0,6E0,5.8E1,1.28E2,4.7E1,1E1,7E0,1E1,4E0,5.4E1,5.2E1,7.6E1,4.1E1,6E0,6E0,4E0,4E0,6E0,4.7E1,7E0,4.5E1,7E0,4.1E1,3.5E1,3.7E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.4331366E-3,-1.0724672E-2,7.418579E-2,-5.347445E-2,-7.8892155E-4,-1.1419505E-2,1.1141756E-1,6.789295E-2,-8.0812134E-2,-1.3984216E-2,3.989023E-2,1.4397213E-3,-2.7791976E-3,1.0555903E-2,7.020065E-2,5.4551656E-3,1.7138795E-4,-5.6447465E-2,-7.046519E-3,6.4858315E-3,-1.9156257E-2,-3.693515E-3,5.5362605E-2,9.217523E-2,-4.4814602E-5,-1.6793209E-3,-5.9973337E-3,1.5944134E-3,-1.4024997E-3,4.9813255E-3,1.2478014E-3,2.3620017E-3,5.5194246E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,-1,-1,25,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9260977E-1,1.0359709E-1,9.688942E-2,1.5811017E-1,1.0737574E-1,2.2781707E-2,8.5914105E-2,2.8777454E-2,5.824144E-2,1.2363675E-1,9.5504105E-2,0E0,0E0,0E0,2.667804E-2,0E0,0E0,3.9476976E-2,0E0,0E0,9.044458E-2,0E0,6.2760904E-2,8.571103E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,17,17,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,-1,-1,26,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.74915E1,2.3877826E4,3.4587786E-1,2.6E1,8.373168E5,1.6630856E2,2.2779044E-3,8.6E1,3.607011E1,2.8683594E-3,1.29E2,1.4397213E-3,-2.7791976E-3,1.0555903E-2,6.971004E7,5.4551656E-3,1.7138795E-4,2.6692104E3,-7.046519E-3,6.4858315E-3,9.905223E6,-3.693515E-3,5.1270317E-2,4.6673E4,-4.4814602E-5,-1.6793209E-3,-5.9973337E-3,1.5944134E-3,-1.4024997E-3,4.9813255E-3,1.2478014E-3,2.3620017E-3,5.5194246E-3],"split_indices":[64,56,35,3,36,64,65,10,64,35,0,0,0,0,53,0,0,55,0,0,12,0,46,10,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2.43E2,2.9E1,4.5E1,1.98E2,9E0,2E1,8E0,3.7E1,1.5E2,4.8E1,5E0,4E0,4E0,1.6E1,4E0,4E0,2.9E1,8E0,4E0,1.46E2,5E0,4.3E1,1.2E1,4E0,2.4E1,5E0,2.5E1,1.21E2,1.4E1,2.9E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.596322E-3,-2.1797339E-2,3.9045244E-2,-1.1601291E-2,-7.141663E-2,2.3966122E-2,1.2026854E-1,-6.090971E-2,1.427158E-3,-1.5301901E-1,-1.8884372E-2,-7.664027E-3,5.1260155E-2,2.4782952E-3,6.9613666E-3,-2.322099E-2,-1.02075525E-1,3.798861E-2,-2.7206838E-2,-4.0753353E-3,-8.584267E-3,3.0076155E-3,-5.9639994E-2,5.8104645E-2,-3.772403E-2,5.588325E-3,3.620243E-2,1.8867791E-3,-2.0484312E-3,-7.1567027E-3,-3.1555875E-3,-6.605586E-4,3.5581077E-3,-4.110577E-3,-8.440079E-5,1.0158559E-3,-5.6941784E-3,-7.83924E-4,6.2649767E-3,-4.1366566E-5,-3.1871747E-3,8.551304E-4,4.9072606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,-1,-1,-1,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2498435E-1,9.175496E-2,1.0857223E-1,9.837599E-2,1.299998E-1,6.773138E-2,2.0631894E-2,4.7277287E-2,1.287554E-1,9.372115E-3,7.2535776E-2,7.534117E-2,3.994947E-2,0E0,0E0,2.6403878E-2,1.6192377E-2,1.1215479E-1,1.08514205E-1,0E0,0E0,0E0,7.597701E-2,7.261184E-2,3.0098502E-2,0E0,4.268846E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,-1,-1,-1,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2286531E3,3.3207447E1,1.9894526E6,1.295E3,1.643614E6,1E0,9.1797715E-1,5.734444E6,9.360022E5,6.6932364E7,6.077547E0,5.491E3,2.0469713E1,2.4782952E-3,6.9613666E-3,4.1195285E-3,1.8243903E0,6.212876E-1,1.7196646E8,-4.0753353E-3,-8.584267E-3,3.0076155E-3,5.67E2,3.0110518E3,3.2441288E5,5.588325E-3,5.9165E4,1.8867791E-3,-2.0484312E-3,-7.1567027E-3,-3.1555875E-3,-6.605586E-4,3.5581077E-3,-4.110577E-3,-8.440079E-5,1.0158559E-3,-5.6941784E-3,-7.83924E-4,6.2649767E-3,-4.1366566E-5,-3.1871747E-3,8.551304E-4,4.9072606E-3],"split_indices":[60,64,36,2,40,109,35,53,56,53,61,9,64,0,0,49,64,35,7,0,0,0,0,60,36,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.82E2,9E1,1.52E2,3E1,7.7E1,1.3E1,3.1E1,1.21E2,1.1E1,1.9E1,3.6E1,4.1E1,5E0,8E0,1.7E1,1.4E1,5.3E1,6.8E1,5E0,6E0,6E0,1.3E1,1.1E1,2.5E1,6E0,3.5E1,4E0,1.3E1,4E0,1E1,2.3E1,3E1,1.9E1,4.9E1,6E0,7E0,6E0,5E0,1.2E1,1.3E1,2.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.6495953E-3,-2.9520188E-2,1.9088933E-2,-9.961801E-3,-7.9084285E-2,3.4187816E-2,-2.0956941E-2,-2.9960997E-2,8.772367E-2,9.050867E-4,-9.566942E-2,1.21284656E-1,1.9627234E-2,-8.05147E-3,-4.1366946E-3,-1.8604444E-2,-6.2039173E-3,1.5001443E-1,-1.9311662E-3,-1.2906276E-1,-2.1963187E-2,1.0258493E-2,7.935662E-2,-1.9376582E-2,3.0555496E-2,-5.6501422E-2,1.9790808E-2,-2.440112E-3,6.013606E-4,3.5895056E-3,8.8427765E-3,-8.257364E-3,-4.2894143E-3,4.6782512E-5,-1.8669575E-3,4.6123164E-3,1.8991561E-3,-2.0337047E-3,1.930664E-3,2.5090302E-4,2.392462E-3,1.3331962E-3,-4.1533415E-3,5.417737E-3,-5.10486E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,-1,29,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.560275E-1,9.905974E-2,1.1197172E-1,1.491506E-1,4.9372792E-2,1.6755576E-1,1.3131326E-1,7.246396E-2,1.112949E-1,0E0,5.8697775E-2,6.31983E-2,5.0036356E-2,0E0,6.0007196E-2,6.38612E-2,0E0,1.2292951E-2,0E0,1.39781535E-2,3.8851765E-3,0E0,7.7696517E-3,4.1376997E-2,4.862681E-2,4.5821626E-2,1.01044364E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,17,17,19,19,20,20,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,-1,30,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2864978E2,7.34E2,6.627576E7,2.734E3,2.7375658E1,7.275E3,1.1992621E3,2.1843149E8,6.58542E5,9.050867E-4,1.177E3,4.9695907E0,1E0,-8.05147E-3,1.09141096E8,1.295E3,-6.2039173E-3,2.0368612E1,-1.9311662E-3,1.772278E5,1.627E3,1.0258493E-2,1E0,1.27718E6,4.485647E5,3.607011E1,7.6499896E9,-2.440112E-3,6.013606E-4,3.5895056E-3,8.8427765E-3,-8.257364E-3,-4.2894143E-3,4.6782512E-5,-1.8669575E-3,4.6123164E-3,1.8991561E-3,-2.0337047E-3,1.930664E-3,2.5090302E-4,2.392462E-3,1.3331962E-3,-4.1533415E-3,5.417737E-3,-5.10486E-4],"split_indices":[60,0,53,2,66,9,4,7,1,0,0,62,109,0,7,2,0,66,0,36,0,0,13,36,36,64,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,1.02E2,1.83E2,7.4E1,2.8E1,1.33E2,5E1,6.2E1,1.2E1,4E0,2.4E1,1.8E1,1.15E2,4E0,4.6E1,5.7E1,5E0,8E0,4E0,1.6E1,8E0,4E0,1.4E1,2.5E1,9E1,1.4E1,3.2E1,2.7E1,3E1,4E0,4E0,5E0,1.1E1,4E0,4E0,8E0,6E0,1.8E1,7E0,4.2E1,4.8E1,4E0,1E1,7E0,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[8.826905E-4,-2.7767442E-2,1.823144E-2,-2.0007031E-2,-1.09184414E-1,3.3425257E-2,-2.3858633E-2,1.1583539E-2,-4.1993346E-2,-8.280537E-3,-7.680021E-4,6.738293E-3,2.7691402E-2,-7.754911E-3,-6.4926115E-3,4.6712544E-2,-6.767221E-2,2.6011616E-3,-5.6619346E-2,-1.9180741E-2,4.0936027E-2,-9.3517415E-2,1.6070697E-2,-1.4821391E-3,4.1407202E-3,1.2767317E-3,-6.020561E-3,-3.1053836E-3,1.650017E-3,1.614121E-3,-2.3601209E-3,6.3513557E-4,2.8274318E-3,-9.98486E-4,-6.7516672E-3,4.1978527E-3,-6.1852345E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,-1,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3967857E-1,6.478944E-2,1.1271152E-1,6.8553336E-2,5.583228E-2,7.947692E-2,1.16909824E-1,1.1731234E-1,8.6748585E-2,0E0,0E0,0E0,7.776487E-2,0E0,8.593629E-2,1.0179864E-1,8.187595E-2,0E0,5.3478435E-2,5.0517537E-2,5.305718E-2,3.1165063E-2,7.878419E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,-1,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.249921E2,3.0597075E8,6.627576E7,4.6008915E6,4.94326E5,1.1816E4,1.1992621E3,1.0268356E1,1.4218562E5,-8.280537E-3,-7.680021E-4,6.738293E-3,1E0,-7.754911E-3,3.1663745E-5,1.278E3,5.2646E4,2.6011616E-3,1E0,2E0,4.485647E5,8.8284516E-1,3.219208E6,-1.4821391E-3,4.1407202E-3,1.2767317E-3,-6.020561E-3,-3.1053836E-3,1.650017E-3,1.614121E-3,-2.3601209E-3,6.3513557E-4,2.8274318E-3,-9.98486E-4,-6.7516672E-3,4.1978527E-3,-6.1852345E-4],"split_indices":[60,7,53,53,1,12,4,62,36,0,0,0,109,0,46,2,1,0,84,8,36,66,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.05E2,1.74E2,9.7E1,8E0,1.28E2,4.6E1,4E1,5.7E1,4E0,4E0,5E0,1.23E2,4E0,4.2E1,2.8E1,1.2E1,7E0,5E1,2.7E1,9.6E1,8E0,3.4E1,1E1,1.8E1,5E0,7E0,4.5E1,5E0,1E1,1.7E1,4.2E1,5.4E1,4E0,4E0,9E0,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.0917053E-3,-2.6654562E-2,2.1541292E-2,-1.5578272E-2,-1.06529966E-1,-2.7263047E-2,3.6968496E-2,-2.9502202E-2,4.129754E-2,-4.3888513E-2,-7.2438577E-3,-5.1796526E-2,4.2252042E-3,9.666481E-2,1.6595768E-2,-3.785679E-2,4.056037E-3,6.886479E-2,-1.536255E-2,7.635906E-6,-3.6448482E-3,-3.4083389E-3,-8.290604E-2,1.3448574E-1,-2.4139341E-4,-1.245896E-2,4.179646E-2,1.1824578E-3,-2.4898814E-3,1.4056774E-3,4.5832484E-3,-1.891649E-3,6.1849464E-4,-3.3077893E-3,1.831983E-3,-5.5680093E-3,3.2095503E-4,1.1248362E-3,7.468085E-3,8.3044247E-4,-1.6829332E-3,1.150101E-3,6.3781827E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,-1,29,31,-1,-1,33,35,37,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6019072E-1,1.2308642E-1,1.0265131E-1,1.00210235E-1,4.7197074E-2,1.01123385E-1,1.2366198E-1,1.0256604E-1,4.0107008E-2,1.3999946E-2,0E0,4.186675E-2,0E0,1.00915566E-1,5.7632275E-2,9.931375E-2,0E0,1.6955554E-2,7.2185816E-3,0E0,0E0,3.8334724E-2,5.8949046E-2,5.3998888E-2,0E0,2.7845692E-2,6.486371E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,17,17,18,18,21,21,22,22,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,-1,30,32,-1,-1,34,36,38,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2707859E3,2.7846207E1,1.060931E-2,1.6E1,1.4267187E8,2.7082097E3,8.41E2,1E0,3.7E1,7.856E3,-7.2438577E-3,2.7864855E11,4.2252042E-3,3.072278E-1,1E0,7.751E3,4.056037E-3,2.8244882E6,2.0526677E8,7.635906E-6,-3.6448482E-3,1.8159722E-1,1.4362E4,5.1344125E4,-2.4139341E-4,1.671E3,5.4349E4,1.1824578E-3,-2.4898814E-3,1.4056774E-3,4.5832484E-3,-1.891649E-3,6.1849464E-4,-3.3077893E-3,1.831983E-3,-5.5680093E-3,3.2095503E-4,1.1248362E-3,7.468085E-3,8.3044247E-4,-1.6829332E-3,1.150101E-3,6.3781827E-3],"split_indices":[4,64,47,8,7,60,0,84,3,9,0,39,0,46,109,9,0,40,7,0,0,35,9,41,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.4E2,1.34E2,1.24E2,1.6E1,3.2E1,1.02E2,1E2,2.4E1,8E0,8E0,2.7E1,5E0,2.5E1,7.7E1,9.4E1,6E0,1.6E1,8E0,4E0,4E0,1.1E1,1.6E1,1.8E1,7E0,3.6E1,4.1E1,1.9E1,7.5E1,8E0,8E0,4E0,4E0,4E0,7E0,1.1E1,5E0,4E0,1.4E1,1.6E1,2E1,3.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.4883691E-3,-5.209672E-3,5.743935E-3,-3.3300497E-2,8.253384E-3,-6.645189E-2,-1.2201443E-2,1.9960258E-2,-2.7301153E-2,-1.0048314E-1,-1.0658451E-2,3.736288E-2,-4.391361E-2,6.022446E-3,6.637661E-2,-7.2793798E-3,-1.107909E-2,-7.2930506E-3,-2.0185704E-3,-3.0482612E-3,2.5022742E-3,4.6751234E-3,-1.6596818E-3,-5.278676E-3,-2.3469888E-4,1.254431E-3,-1.6316975E-3,1.9675796E-3,6.0755117E-3,-2.8388575E-3,7.5567706E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3125336E-1,1.02763884E-1,0E0,6.0741946E-2,7.713171E-2,6.34855E-2,8.80893E-2,8.952524E-2,9.585706E-2,6.194794E-2,5.4153495E-2,1.0731278E-1,9.277149E-2,9.600325E-2,4.3974847E-2,0E0,5.941555E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4702875E4,3.4008475E2,5.743935E-3,5.37009E-1,6.627576E7,2.3194496E8,1.5637306E8,8.9639384E-1,1.1992621E3,7.4517044E5,9.735178E0,8.5959184E-1,3.34226E8,2.2971932E7,5.978742E1,-7.2793798E-3,1.09141096E8,-7.2930506E-3,-2.0185704E-3,-3.0482612E-3,2.5022742E-3,4.6751234E-3,-1.6596818E-3,-5.278676E-3,-2.3469888E-4,1.254431E-3,-1.6316975E-3,1.9675796E-3,6.0755117E-3,-2.8388575E-3,7.5567706E-4],"split_indices":[60,60,0,35,53,5,5,35,4,59,62,35,5,53,64,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.7E2,7E0,8.7E1,1.83E2,3.3E1,5.4E1,1.38E2,4.5E1,2E1,1.3E1,2.1E1,3.3E1,1.07E2,3.1E1,4E0,4.1E1,9E0,1.1E1,7E0,6E0,1.1E1,1E1,1.1E1,2.2E1,7.1E1,3.6E1,2.4E1,7E0,1.4E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.9759689E-4,-7.8622E-3,6.588338E-2,-3.931674E-3,-1.0180166E-1,7.829333E-3,3.1315666E-2,-8.607116E-3,7.8730464E-2,-6.6379183E-3,-2.2801615E-3,1.5322073E-2,3.9516203E-3,-1.3414576E-2,4.849225E-2,7.556855E-4,4.8160455E-3,3.2230087E-2,-3.243928E-4,-2.0354504E-3,-4.4560715E-5,-1.2704153E-3,4.090027E-3,2.3850605E-3,-7.4488846E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4132069E-1,9.190762E-2,9.9099115E-2,9.38198E-2,1.5010849E-2,0E0,1.8737504E-2,6.351167E-2,1.9243285E-2,0E0,0E0,7.412792E-3,0E0,8.121587E-2,5.7701178E-2,0E0,0E0,7.371312E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.74915E1,3.0532868E6,1.6064256E-2,2.00347E6,2E0,7.829333E-3,1.3860551E6,1E0,1.44404335E1,-6.6379183E-3,-2.2801615E-3,6.4613514E0,3.9516203E-3,2.495E3,2.8558376E1,7.556855E-4,4.8160455E-3,9.820072E7,-3.243928E-4,-2.0354504E-3,-4.4560715E-5,-1.2704153E-3,4.090027E-3,2.3850605E-3,-7.4488846E-5],"split_indices":[64,36,65,36,8,0,36,112,66,0,0,61,0,2,66,0,0,53,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.49E2,2.8E1,2.4E2,9E0,6E0,2.2E1,2.28E2,1.2E1,4E0,5E0,1.8E1,4E0,2.11E2,1.7E1,4E0,8E0,1E1,8E0,6E1,1.51E2,6E0,1.1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-5.5375796E-5,-2.0781755E-2,2.6767464E-2,-1.0496193E-2,-1.03053875E-1,1.8658997E-2,1.1328384E-1,-1.8076194E-2,7.060017E-2,-6.3195513E-3,-8.042405E-4,2.7525991E-2,-5.6048613E-2,7.2529577E-3,1.7278662E-3,-1.3770757E-3,-5.183924E-2,5.5129793E-3,-1.0195514E-3,5.5927247E-2,9.6748015E-3,-4.927246E-3,-7.9127745E-4,-5.4266775E-4,4.097279E-3,-4.835175E-3,-1.2461842E-3,3.1257528E-3,-1.6855027E-3,-1.8987608E-3,1.4475822E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5177205E-1,1.2923545E-1,8.136653E-2,8.610623E-2,4.7369584E-2,7.4410304E-2,2.7608514E-2,7.153288E-2,5.7341572E-2,0E0,0E0,4.965155E-2,2.1930423E-2,0E0,0E0,8.194895E-2,5.2487716E-2,0E0,0E0,4.455273E-2,7.000757E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.3813794E2,3.8293724E7,1.4964736E4,1.02942106E11,5.752E3,3.806384E2,2.2047469E4,7.34E2,3.7568363E4,-6.3195513E-3,-8.042405E-4,3.36425E5,1.8790482E9,7.2529577E-3,1.7278662E-3,1.88832E5,8.4E2,5.5129793E-3,-1.0195514E-3,9.4721675E-1,7.588E3,-4.927246E-3,-7.9127745E-4,-5.4266775E-4,4.097279E-3,-4.835175E-3,-1.2461842E-3,3.1257528E-3,-1.6855027E-3,-1.8987608E-3,1.4475822E-3],"split_indices":[60,53,63,39,2,66,4,0,41,0,0,37,7,0,0,37,0,0,0,35,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.53E2,1.18E2,1.37E2,1.6E1,1.09E2,9E0,1.26E2,1.1E1,1.1E1,5E0,9.8E1,1.1E1,5E0,4E0,8.5E1,4.1E1,7E0,4E0,3.7E1,6.1E1,4E0,7E0,7.7E1,8E0,1.2E1,2.9E1,3.3E1,4E0,1.8E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-6.618102E-4,-3.4341544E-2,1.5117969E-2,-1.0702268E-1,-1.5694294E-2,3.7332814E-2,-7.4898507E-3,-8.678086E-3,-7.0284225E-2,-2.728105E-2,4.410471E-3,6.5384097E-3,2.7503924E-2,-4.139819E-2,2.2576133E-2,-6.618204E-4,-5.083837E-3,3.5782788E-2,-4.8177134E-2,7.339499E-3,1.7754488E-2,1.8972921E-3,-5.769969E-2,8.266253E-2,-4.7560483E-3,-6.1939564E-4,4.9743527E-3,-4.2731E-3,-8.2078425E-4,2.1705976E-4,4.401601E-3,-2.7026032E-4,-3.7475266E-3,7.6813955E-4,6.747584E-3,-9.004919E-4,3.3387006E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,-1,-1,25,27,-1,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4826472E-1,1.1894236E-1,9.570317E-2,4.0623352E-2,9.562234E-2,9.486714E-2,9.778594E-2,0E0,2.9828168E-2,8.869788E-2,0E0,0E0,1.1162637E-1,6.297545E-2,8.415005E-2,0E0,0E0,6.1934922E-2,6.6076584E-2,0E0,8.481999E-2,0E0,4.6191454E-2,6.500444E-2,4.259539E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,12,12,13,13,14,14,17,17,18,18,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,-1,-1,26,28,-1,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.555544E2,2.4242616E5,4.66667E5,1.9977521E4,5.0738132E7,1.0781069E5,1.1937E4,-8.678086E-3,1.0781069E5,8.842206E7,4.410471E-3,6.5384097E-3,5.0655737E0,2.7E1,2.431031E7,-6.618204E-4,-5.083837E-3,1.5202844E6,3.0117606E8,7.339499E-3,1E0,1.8972921E-3,2.0201733E9,2.849513E3,2.0749E4,-6.1939564E-4,4.9743527E-3,-4.2731E-3,-8.2078425E-4,2.1705976E-4,4.401601E-3,-2.7026032E-4,-3.7475266E-3,7.6813955E-4,6.747584E-3,-9.004919E-4,3.3387006E-3],"split_indices":[4,36,37,41,40,36,2,0,36,5,0,0,62,3,59,0,0,58,5,0,96,0,5,4,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,8.8E1,1.89E2,1.7E1,7.1E1,9.5E1,9.4E1,4E0,1.3E1,6.5E1,6E0,7E0,8.8E1,4.4E1,5E1,6E0,7E0,1.6E1,4.9E1,5E0,8.3E1,7E0,3.7E1,1.5E1,3.5E1,1E1,6E0,1.9E1,3E1,7.2E1,1.1E1,1.2E1,2.5E1,8E0,7E0,3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.3602284E-3,-9.5443E-3,5.2941076E-2,-1.4588556E-2,3.1910546E-2,8.4940875E-3,2.7559547E-2,-3.4351066E-2,1.5368425E-4,7.404272E-2,-2.6924903E-2,6.781122E-2,-6.8674106E-3,-2.1002324E-2,-1.4438027E-1,7.671377E-2,-8.502271E-3,1.04231395E-1,2.9265415E-4,8.212369E-4,-2.8290332E-3,6.001671E-4,4.1762744E-3,-2.317543E-3,1.4979353E-3,-1.5362641E-3,4.704527E-3,-8.703767E-3,-3.8258948E-3,1.6395202E-3,4.9198773E-3,2.1510143E-3,-1.0160267E-3,5.7246173E-3,2.5431674E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.105542E-1,5.1906314E-2,1.0176028E-1,6.410089E-2,6.892296E-2,0E0,3.9266482E-2,1.3503033E-1,8.481523E-2,3.203123E-2,1.9638734E-2,1.51217505E-2,2.9042078E-2,1.3287775E-1,1.1661917E-2,1.2302585E-2,8.7179154E-2,3.9678067E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.206759E3,1E0,2.2779044E-3,3.1433902E4,5.8E2,8.4940875E-3,3.09483E2,2.237806E8,9.12314E4,2.5356756E1,5.380323E-1,5.142232E-1,2.0390862E-1,2.1050163E8,1.2051519E6,1.2236692E3,5.225893E4,3.5E1,2.9265415E-4,8.212369E-4,-2.8290332E-3,6.001671E-4,4.1762744E-3,-2.317543E-3,1.4979353E-3,-1.5362641E-3,4.704527E-3,-8.703767E-3,-3.8258948E-3,1.6395202E-3,4.9198773E-3,2.1510143E-3,-1.0160267E-3,5.7246173E-3,2.5431674E-3],"split_indices":[60,84,65,41,0,0,66,7,36,64,35,65,46,7,40,4,41,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.45E2,3.1E1,2.19E2,2.6E1,4E0,2.7E1,9.3E1,1.26E2,1.5E1,1.1E1,1.2E1,1.5E1,8.4E1,9E0,1.2E1,1.14E2,1E1,5E0,5E0,6E0,4E0,8E0,7E0,8E0,7.7E1,7E0,4E0,5E0,6E0,6E0,2.2E1,9.2E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[8.8592366E-5,-7.1994634E-3,6.702745E-2,7.3937245E-2,-1.0212092E-2,8.1812255E-3,4.005524E-2,-5.4564606E-4,6.673114E-3,-1.4455275E-2,5.439933E-2,6.603743E-2,-7.788322E-4,-2.0125499E-2,3.410768E-2,-5.3874095E-4,8.861978E-2,8.224671E-2,6.4124377E-4,-1.4276436E-3,2.0949359E-4,2.7707212E-3,-1.1767214E-3,5.6987065E-3,2.0513253E-3,1.0749275E-3,5.008259E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,-1,19,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3415459E-1,6.137488E-2,7.347539E-2,5.6537423E-2,6.658429E-2,0E0,3.516257E-2,0E0,0E0,6.2918454E-2,3.496688E-2,1.2369096E-2,0E0,5.4653503E-2,3.8872153E-2,0E0,9.976417E-3,1.6212583E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,13,13,14,14,16,16,17,17],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,-1,20,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.5891304E1,1.5E1,2.2779044E-3,3E0,1E0,8.1812255E-3,8.805777E7,-5.4564606E-4,6.673114E-3,7.06836E7,4.22631E5,1E0,-7.788322E-4,5.754E3,6.332424E1,-5.3874095E-4,8.63332E5,9.886E3,6.4124377E-4,-1.4276436E-3,2.0949359E-4,2.7707212E-3,-1.1767214E-3,5.6987065E-3,2.0513253E-3,1.0749275E-3,5.008259E-3],"split_indices":[64,3,65,8,96,0,53,0,0,40,36,84,0,2,64,0,1,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.47E2,2.6E1,8E0,2.39E2,4E0,2.2E1,4E0,4E0,2.25E2,1.4E1,1.5E1,7E0,2.02E2,2.3E1,5E0,9E0,1.1E1,4E0,1.4E2,6.2E1,1.6E1,7E0,4E0,5E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[5.5505815E-3,-7.601315E-4,5.957039E-2,-2.9660264E-2,1.5537662E-2,8.747815E-2,-2.4257414E-3,-7.7807516E-2,-2.858848E-3,-9.638513E-3,3.4607235E-2,1.0371688E-1,1.2379859E-3,-4.5530804E-2,-1.2242836E-1,-3.5700914E-2,3.51458E-2,4.98334E-2,-4.3392975E-2,-2.0936362E-3,6.4324826E-2,2.3539749E-3,5.587929E-3,-3.0675048E-3,1.2366262E-3,-2.7952315E-3,-7.2670546E-3,-2.4932106E-3,3.1024476E-3,3.9660966E-4,6.669296E-3,-2.507601E-4,5.3250743E-3,-7.7712716E-4,-4.984926E-3,1.5290948E-3,-1.7170617E-3,5.611041E-3,1.6675391E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.799838E-2,1.097348E-1,8.919557E-2,1.0817238E-1,7.18683E-2,1.8856868E-2,0E0,3.7632152E-2,6.990305E-2,1.3264616E-1,9.28409E-2,9.575427E-3,0E0,3.0803256E-2,1.6642153E-2,6.070973E-2,7.443206E-2,8.7962165E-2,7.0420146E-2,4.970625E-2,7.160351E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.1433902E4,4.636564E7,5.37009E-1,1.2707859E3,1.3277E4,-2.4257414E-3,1.35157555E-2,7.0120897E0,2.8945708E6,1E0,8.118562E2,1.2379859E-3,3.6977453E-3,8.146263E0,7.0229135E6,2.0526677E8,6.6352534E-1,3.615E3,1.671E3,1.2951E4,2.3539749E-3,5.587929E-3,-3.0675048E-3,1.2366262E-3,-2.7952315E-3,-7.2670546E-3,-2.4932106E-3,3.1024476E-3,3.9660966E-4,6.669296E-3,-2.507601E-4,5.3250743E-3,-7.7712716E-4,-4.984926E-3,1.5290948E-3,-1.7170617E-3,5.611041E-3,1.6675391E-3],"split_indices":[84,41,53,35,4,10,0,46,61,53,109,4,0,49,62,40,7,35,2,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,2.31E2,2.6E1,8.3E1,1.48E2,2.1E1,5E0,2.9E1,5.4E1,6.4E1,8.4E1,1.6E1,5E0,1.8E1,1.1E1,2.9E1,2.5E1,2.3E1,4.1E1,3.8E1,4.6E1,5E0,1.1E1,1.4E1,4E0,5E0,6E0,2.5E1,4E0,2.1E1,4E0,1.3E1,1E1,3E1,1.1E1,1.9E1,1.9E1,1.4E1,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.2472441E-3,-9.431977E-3,5.2670226E-2,-9.316273E-4,-5.846452E-2,6.6181314E-3,3.3343982E-2,-1.8622305E-2,2.3846282E-2,-1.0726859E-1,-2.8337678E-2,-1.4962945E-3,5.3902354E-2,-1.1937866E-2,-7.7643613E-3,3.7208654E-2,-3.1791948E-2,-2.4608872E-3,-5.766603E-3,-3.1858957E-3,-6.136991E-3,8.958368E-2,1.0755718E-2,-1.0044026E-4,-4.7874046E-3,3.5380488E-3,7.836287E-4,-2.488433E-3,8.296172E-4,-1.4772696E-3,5.82003E-4,5.200081E-3,2.4597554E-3,-9.3983405E-4,2.0868822E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,-1,25,27,-1,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2053041E-1,9.872565E-2,5.8571182E-2,8.942669E-2,4.8421204E-2,0E0,4.3714218E-2,1.1731808E-1,6.437692E-2,5.885452E-3,2.0438673E-2,0E0,3.5985164E-2,1.0304376E-1,0E0,5.3906307E-2,1.9820154E-2,0E0,0E0,0E0,8.201877E-3,4.9468577E-3,1.3855573E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,20,20,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,-1,26,28,-1,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2838904E7,1.9989124E7,4.514673E-3,8.3813794E2,7.856E3,6.6181314E-3,1.2371134E-2,6.0073395E8,7.794012E7,9.257E3,2.2872795E-1,-1.4962945E-3,1.942E3,3.013037E6,-7.7643613E-3,1E0,1.0702152E-2,-2.4608872E-3,-5.766603E-3,-3.1858957E-3,3.3E1,8.269529E-1,2.4864366E9,-1.0044026E-4,-4.7874046E-3,3.5380488E-3,7.836287E-4,-2.488433E-3,8.296172E-4,-1.4772696E-3,5.82003E-4,5.200081E-3,2.4597554E-3,-9.3983405E-4,2.0868822E-3],"split_indices":[40,40,65,60,10,0,65,7,53,9,47,0,0,36,0,8,46,0,0,0,3,35,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.36E2,3.5E1,2.02E2,3.4E1,5E0,3E1,1.18E2,8.4E1,1.2E1,2.2E1,7E0,2.3E1,1.14E2,4E0,6.8E1,1.6E1,4E0,8E0,7E0,1.5E1,1.2E1,1.1E1,1.04E2,1E1,2.2E1,4.6E1,1.1E1,5E0,6E0,9E0,6E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.4035108E-4,-3.0717986E-2,1.4525555E-2,-1.9585114E-2,-6.235052E-3,3.471791E-2,-4.3727085E-3,-2.9714486E-2,3.8602635E-2,7.2844387E-3,2.7221253E-2,-3.517487E-2,2.0348523E-2,-1.2347729E-2,-6.990898E-2,-7.1392447E-4,4.387524E-3,1.9574665E-2,6.8467828E-3,2.285768E-3,-5.2563906E-2,7.904985E-2,-1.6238795E-3,-1.880669E-3,1.9348994E-3,-4.858251E-3,-1.0932897E-3,1.700951E-3,-6.465358E-4,-1.444496E-4,-3.377316E-3,4.8867757E-3,1.6770973E-4,-8.569901E-4,2.8017003E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,25,-1,-1,27,-1,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.19819246E-1,9.643015E-2,7.189269E-2,4.7120675E-2,0E0,7.994313E-2,7.533955E-2,4.590599E-2,3.8289003E-2,0E0,7.791007E-2,6.750779E-2,7.106787E-2,7.6099046E-2,3.0372545E-2,0E0,0E0,4.8983894E-2,0E0,0E0,3.7791185E-2,3.0254722E-2,4.4652205E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,17,17,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,26,-1,-1,28,-1,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.555544E2,1.996866E1,4.66667E5,1.7E1,-6.235052E-3,4.9695907E0,1.0207E4,1.6306231E6,2.4734788E7,7.2844387E-3,4.19688E5,2.7E1,2.0967E4,4.3724646E2,3.719267E7,-7.1392447E-4,4.387524E-3,9.6494156E-1,6.8467828E-3,2.285768E-3,2.0201733E9,5.347626E5,2.4435181E12,-1.880669E-3,1.9348994E-3,-4.858251E-3,-1.0932897E-3,1.700951E-3,-6.465358E-4,-1.444496E-4,-3.377316E-3,4.8867757E-3,1.6770973E-4,-8.569901E-4,2.8017003E-3],"split_indices":[4,64,37,8,0,62,2,59,12,0,37,3,10,63,40,0,0,65,0,0,5,36,39,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,8.4E1,1.87E2,7.7E1,7E0,9E1,9.7E1,6.6E1,1.1E1,4E0,8.6E1,4.3E1,5.4E1,4.7E1,1.9E1,6E0,5E0,8.2E1,4E0,7E0,3.6E1,1.4E1,4E1,3.1E1,1.6E1,1E1,9E0,5.4E1,2.8E1,1.1E1,2.5E1,1E1,4E0,3.2E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.1215687E-3,-5.5902754E-3,5.0307773E-2,1.8846302E-3,-4.985285E-2,3.3973295E-2,6.214907E-3,7.175894E-3,-8.513734E-2,-8.260854E-2,-1.0311028E-2,4.611518E-3,1.6010819E-2,-6.0443534E-3,3.8200274E-2,-5.2155266E-3,-1.0627084E-3,-7.767475E-4,-1.0488127E-1,4.8827947E-4,-1.6191519E-3,-2.2769042E-3,3.157643E-2,2.6978096E-3,-7.4330973E-4,-4.112915E-3,2.3786216E-3,-5.9437198E-3,-2.3517245E-3,-4.8918556E-4,2.1340903E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0736896E-2,8.016173E-2,4.1058928E-2,9.638583E-2,4.4480905E-2,3.2874294E-2,0E0,8.103044E-2,1.8342137E-2,2.6049614E-2,9.211391E-3,0E0,2.6017483E-2,9.169551E-2,1.0591414E-1,0E0,0E0,0E0,1.1645332E-2,0E0,0E0,0E0,1.2963403E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,18,18,22,22],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3396984E7,1.9989124E7,1.1038146E0,3.013037E6,1.164E3,5.5104347E1,6.214907E-3,7.54579E5,1.0422E4,2.351E3,1.929E3,4.611518E-3,1.9078356E6,1.0781069E5,1.38E2,-5.2155266E-3,-1.0627084E-3,-7.767475E-4,3.719267E7,4.8827947E-4,-1.6191519E-3,-2.2769042E-3,1E0,2.6978096E-3,-7.4330973E-4,-4.112915E-3,2.3786216E-3,-5.9437198E-3,-2.3517245E-3,-4.8918556E-4,2.1340903E-3],"split_indices":[40,40,42,36,0,66,0,36,9,2,0,0,56,36,0,0,0,0,40,0,0,0,109,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.41E2,3.2E1,2.07E2,3.4E1,2.8E1,4E0,1.96E2,1.1E1,1.8E1,1.6E1,5E0,2.3E1,1.38E2,5.8E1,7E0,4E0,5E0,1.3E1,9E0,7E0,4E0,1.9E1,1.8E1,1.2E2,5E0,5.3E1,8E0,5E0,5E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[6.512211E-4,1.3021781E-2,-3.1955674E-2,9.566487E-3,5.5373185E-3,-9.308747E-2,-9.144919E-3,1.3615808E-2,-5.7564832E-2,-1.3241665E-1,-5.206784E-5,-3.6041975E-2,2.7098862E-2,1.8132918E-3,4.1121434E-2,-7.3329447E-4,-3.6410473E-3,-3.3225205E-3,-7.7618496E-3,7.910024E-3,-5.6379702E-2,3.7477505E-3,-4.8753092E-5,3.174642E-4,-5.689078E-3,-2.381727E-3,2.282909E-3,-6.289654E-4,1.2966394E-3,-1.0164743E-3,-4.389302E-3,1.5308287E-3,-1.9183055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,-1,-1,-1,-1,27,29,-1,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0851809E-1,7.111084E-2,1.0233896E-1,5.2432694E-2,0E0,7.218179E-2,5.4593503E-2,5.835985E-2,8.595955E-3,1.8343687E-2,0E0,2.905025E-2,3.5383604E-2,8.177231E-2,4.4997364E-2,0E0,0E0,0E0,0E0,5.1902975E-3,2.7292378E-2,0E0,2.4942905E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,19,19,20,20,22,22],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,-1,-1,-1,-1,28,30,-1,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0867E4,1.4702875E4,3.7693813E-1,1.4146589E8,5.5373185E-3,8.51E2,1.5660624E1,8.6593616E-1,3.8E1,5.43E2,-5.206784E-5,1.6126542E8,1.2052342E5,8.601996E-1,1.85E2,-7.3329447E-4,-3.6410473E-3,-3.3225205E-3,-7.7618496E-3,3.0345E4,6.8992513E-1,3.7477505E-3,7.348312E-1,3.174642E-4,-5.689078E-3,-2.381727E-3,2.282909E-3,-6.289654E-4,1.2966394E-3,-1.0164743E-3,-4.389302E-3,1.5308287E-3,-1.9183055E-3],"split_indices":[9,60,35,53,0,0,64,35,3,10,0,5,41,35,0,0,0,0,0,9,35,0,35,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,1.94E2,7.3E1,1.89E2,5E0,1.9E1,5.4E1,1.79E2,1E1,1.3E1,6E0,3.1E1,2.3E1,1.26E2,5.3E1,4E0,6E0,6E0,7E0,1E1,2.1E1,7E0,1.6E1,1.22E2,4E0,4E0,4.9E1,5E0,5E0,1.2E1,9E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.0217762E-3,-1.4682543E-2,2.3604771E-2,-1.8996791E-3,-4.747971E-2,6.230071E-2,7.15362E-3,1.2531688E-2,-3.2060686E-2,-6.450792E-2,3.1662493E-3,8.5270464E-2,7.834151E-3,-7.30817E-3,5.4371655E-2,-3.810464E-3,7.380028E-2,-1.7339656E-2,-4.8477435E-3,-6.8170107E-3,-1.1038795E-1,1.2012501E-1,2.545052E-2,2.3812223E-3,-1.3831976E-3,-2.7878452E-2,2.1280475E-2,5.3382884E-3,-1.3244248E-2,-6.2017527E-4,3.4599842E-3,7.514997E-3,1.3906453E-3,-1.6398169E-3,1.6376437E-3,-2.5383513E-3,2.3869106E-3,-6.301069E-3,-1.7050263E-3,2.7656597E-3,6.4522857E-3,-1.294503E-3,3.4036867E-3,-2.8047636E-3,-5.002114E-4,5.176438E-6,3.3459198E-3,4.7023894E-4,-1.5678415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.521632E-2,7.62118E-2,6.3850194E-2,5.7856698E-2,1.05392046E-1,3.686449E-2,4.9563088E-2,9.0394676E-2,4.4529863E-2,1.1835623E-1,0E0,4.0727675E-2,1.8176196E-2,3.352555E-2,7.23138E-2,5.592848E-2,6.914126E-2,3.704592E-2,0E0,6.2376063E-2,4.347399E-2,7.209465E-3,2.5449228E-2,0E0,0E0,1.7889012E-2,2.6100688E-2,0E0,4.740958E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.816E3,2.0020384E1,5.39594E5,2.0688E4,2.6493E4,3.218279E-1,1.12648186E2,1.9453941E-1,1.2027512E1,5.67E2,3.1662493E-3,2E0,1.2126853E9,1.479E4,7.588906E-2,1E0,2.356535E9,3.37353E5,-4.8477435E-3,1.2392865E6,8.607021E2,7.019E3,4.3201213E9,2.3812223E-3,-1.3831976E-3,7.528358E-2,3.5632E4,5.3382884E-3,3.7E1,-6.2017527E-4,3.4599842E-3,7.514997E-3,1.3906453E-3,-1.6398169E-3,1.6376437E-3,-2.5383513E-3,2.3869106E-3,-6.301069E-3,-1.7050263E-3,2.7656597E-3,6.4522857E-3,-1.294503E-3,3.4036867E-3,-2.8047636E-3,-5.002114E-4,5.176438E-6,3.3459198E-3,4.7023894E-4,-1.5678415E-3],"split_indices":[2,64,37,9,10,46,64,46,62,0,0,8,7,9,46,84,39,9,0,40,60,10,5,0,0,47,2,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,1.81E2,1E2,1.31E2,5E1,2.9E1,7.1E1,8.9E1,4.2E1,4.4E1,6E0,2E1,9E0,5.5E1,1.6E1,7.1E1,1.8E1,3.6E1,6E0,2E1,2.4E1,1.2E1,8E0,4E0,5E0,3.2E1,2.3E1,8E0,8E0,6.4E1,7E0,5E0,1.3E1,2.7E1,9E0,1.1E1,9E0,1.7E1,7E0,4E0,8E0,4E0,4E0,1E1,2.2E1,1.7E1,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.0194498E-3,-1.4171326E-2,1.9251512E-2,3.525826E-3,-1.7906092E-2,4.227702E-2,2.526437E-3,-4.6297973E-3,-5.049819E-2,6.670167E-3,3.199436E-2,-3.4727532E-2,1.9777806E-2,-1.5786763E-2,6.662733E-2,-6.662294E-2,2.516913E-3,4.3409977E-2,-1.798901E-3,-5.55475E-2,1.0771017E-2,4.8477133E-3,7.410391E-3,1.8525576E-4,-2.8281412E-3,4.844778E-3,-1.100296E-3,-5.365136E-3,-1.6949708E-3,2.9365034E-3,-1.9987651E-6,8.058412E-5,-3.31695E-3,-4.0831967E-4,1.3009558E-3,-9.0251364E-5,2.61749E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,-1,7,9,11,13,15,-1,17,19,21,23,25,27,-1,29,-1,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.950895E-2,5.1194716E-2,5.3037442E-2,0E0,6.120994E-2,5.6553297E-2,5.2744076E-2,8.21432E-2,7.316814E-2,0E0,4.5504272E-2,2.522754E-2,5.8444146E-2,8.071605E-2,5.1434714E-2,4.9265146E-2,0E0,4.160489E-2,0E0,1.7139204E-2,3.3412557E-3,0E0,2.3276018E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,22,22],"right_children":[2,4,6,-1,8,10,12,14,16,-1,18,20,22,24,26,28,-1,30,-1,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.26031E2,1.5E1,4.58572E5,3.525826E-3,2.0688E4,4.9695907E0,1.0207E4,2.0662694E-1,2.2847826E0,6.670167E-3,9.5096946E-1,3.3580646E8,2.0584858E8,1.6533424E6,5.680776E1,3.677609E-1,2.516913E-3,1.9975014E7,-1.798901E-3,2.0529972E1,1.5E1,4.8477133E-3,1.4964736E4,1.8525576E-4,-2.8281412E-3,4.844778E-3,-1.100296E-3,-5.365136E-3,-1.6949708E-3,2.9365034E-3,-1.9987651E-6,8.058412E-5,-3.31695E-3,-4.0831967E-4,1.3009558E-3,-9.0251364E-5,2.61749E-3],"split_indices":[60,3,37,0,9,62,2,46,65,0,35,7,7,58,66,35,0,40,0,64,8,0,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.46E2,1.37E2,5E0,1.41E2,5.7E1,8E1,1.01E2,4E1,4E0,5.3E1,2.5E1,5.5E1,8.8E1,1.3E1,3.5E1,5E0,4.6E1,7E0,1.7E1,8E0,6E0,4.9E1,6.2E1,2.6E1,9E0,4E0,1.2E1,2.3E1,3.1E1,1.5E1,4E0,1.3E1,4E0,4E0,4.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.5678196E-3,-7.7246227E-3,3.3255238E-2,-3.3770535E-2,5.082167E-3,6.1120387E-2,-1.7863303E-2,2.479463E-2,-6.0571115E-2,1.3874428E-3,4.9842647E-3,8.630632E-2,1.637323E-2,-4.6005435E-3,1.0482209E-2,-2.2585797E-3,4.8033953E-2,-7.178352E-2,1.4617289E-3,-1.2499112E-2,2.399169E-2,1.3703762E-1,2.5033655E-2,-1.0630119E-3,4.2359013E-2,-9.06949E-3,1.4291895E-3,4.605111E-3,-6.93722E-4,-4.9283197E-3,-1.8741249E-3,6.7484175E-4,-1.6314599E-3,-2.40294E-3,2.0577633E-3,8.452708E-3,4.179416E-3,3.029089E-4,2.4514615E-3,8.624348E-4,2.8655857E-3,5.6654576E-4,-1.5149929E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,-1,27,29,-1,31,33,35,37,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.182051E-2,7.188801E-2,9.083063E-2,1.1294205E-1,5.512415E-2,4.4793844E-2,5.454572E-2,4.1829713E-2,5.3000063E-2,4.4559084E-2,0E0,7.738893E-2,1.738504E-2,0E0,7.51702E-3,0E0,6.0556237E-2,4.3856174E-2,0E0,5.5330683E-2,8.749983E-2,1.3485163E-2,6.4178826E-3,0E0,3.5313927E-3,5.499735E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,16,16,17,17,19,19,20,20,21,21,22,22,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,-1,28,30,-1,32,34,36,38,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5642393E0,5.37009E-1,9.6494156E-1,1.1732847E5,1.3988934E2,3.0001976E0,1.23356E5,2.9751791E4,1E0,8.721346E5,4.9842647E-3,1.4364E4,7.595543E-1,-4.6005435E-3,2.1845242E3,-2.2585797E-3,1.9850808E5,3.864E3,1.4617289E-3,1.2742E4,1.142E3,6.668107E0,8.220298E-1,-1.0630119E-3,1.6696589E-1,4.9255515E6,1.4291895E-3,4.605111E-3,-6.93722E-4,-4.9283197E-3,-1.8741249E-3,6.7484175E-4,-1.6314599E-3,-2.40294E-3,2.0577633E-3,8.452708E-3,4.179416E-3,3.029089E-4,2.4514615E-3,8.624348E-4,2.8655857E-3,5.6654576E-4,-1.5149929E-3],"split_indices":[50,35,65,36,64,50,37,41,84,36,0,10,35,0,4,0,41,2,0,10,2,62,35,0,65,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.14E2,6.2E1,7E1,1.44E2,4E1,2.2E1,2.2E1,4.8E1,1.4E2,4E0,2.5E1,1.5E1,5E0,1.7E1,5E0,1.7E1,4.3E1,5E0,8.7E1,5.3E1,1.3E1,1.2E1,6E0,9E0,9E0,8E0,9E0,8E0,1.9E1,2.4E1,4E1,4.7E1,1.1E1,4.2E1,5E0,8E0,8E0,4E0,5E0,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.2584315E-3,-5.290618E-2,4.1832686E-3,-1.9553734E-2,-7.252935E-2,4.652813E-3,1.7641317E-3,3.1456206E-5,-1.6819597E-3,-5.5128825E-3,-4.6814803E-2,-4.594641E-3,4.217559E-2,-3.5064782E-3,-6.894237E-4,5.2174176E-3,-2.5161255E-2,6.8163928E-3,2.2466997E-2,2.0540282E-3,-3.6661304E-4,-4.020804E-3,-3.4189975E-4,2.357747E-3,-3.5789862E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,13,15,17,-1,-1,19,21,-1,23,-1,-1,-1,-1,-1,-1],"loss_changes":[7.666734E-2,1.5598275E-2,5.7706915E-2,3.8043722E-3,1.3817824E-2,0E0,6.237986E-2,0E0,0E0,0E0,1.0071961E-2,4.2509448E-2,6.553292E-2,0E0,0E0,7.4606515E-2,7.4327946E-2,0E0,2.5604628E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,10,10,11,11,12,12,15,15,16,16,18,18],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,14,16,18,-1,-1,20,22,-1,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.292E3,8.137731E7,1.352E3,5.53839E5,1.8243903E0,4.652813E-3,5.206759E3,3.1456206E-5,-1.6819597E-3,-5.5128825E-3,3.1E1,3.37915E1,2.2779044E-3,-3.5064782E-3,-6.894237E-4,4.573327E5,3.9778934E1,6.8163928E-3,3.09483E2,2.0540282E-3,-3.6661304E-4,-4.020804E-3,-3.4189975E-4,2.357747E-3,-3.5789862E-4],"split_indices":[2,5,2,36,64,0,60,0,0,0,3,64,65,0,0,56,64,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.5E1,2.46E2,1E1,1.5E1,5E0,2.41E2,5E0,5E0,4E0,1.1E1,2.09E2,3.2E1,5E0,6E0,1.42E2,6.7E1,4E0,2.8E1,3.5E1,1.07E2,1.4E1,5.3E1,1.4E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.979719E-3,-5.528506E-3,2.9729804E-2,-5.2271783E-3,-2.892994E-3,8.659096E-2,1.4437471E-2,-5.3110835E-3,3.6371595E-3,6.292962E-3,2.434954E-3,7.2509035E-2,1.2118459E-3,-4.0779583E-2,2.3666236E-3,5.4573123E-3,1.0162275E-3,-2.850327E-2,3.6529277E-2,-2.8710482E-3,3.4874494E-4,3.0246503E-3,-1.5680827E-4,-2.3870366E-3,8.635119E-4,2.7443604E-3,2.3864504E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,7,9,11,13,-1,-1,-1,15,17,19,21,-1,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.320838E-2,5.973208E-2,5.700902E-2,0E0,4.1409742E-2,1.6407177E-2,4.1385897E-2,5.5104494E-2,0E0,0E0,0E0,2.0595994E-2,4.8266973E-2,3.803578E-2,6.1271984E-2,0E0,0E0,2.8550753E-2,1.4882183E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,11,11,12,12,13,13,14,14,17,17,18,18],"right_children":[2,4,6,-1,8,10,12,14,-1,-1,-1,16,18,20,22,-1,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5642393E0,1.0893594E6,7.751E3,-5.2271783E-3,2.7553815E2,2E0,5.1E2,1.8761726E-3,3.6371595E-3,6.292962E-3,2.434954E-3,5.377468E1,3.108307E6,1.2153278E-3,8.6E1,5.4573123E-3,1.0162275E-3,5E0,5.6431256E5,-2.8710482E-3,3.4874494E-4,3.0246503E-3,-1.5680827E-4,-2.3870366E-3,8.635119E-4,2.7443604E-3,2.3864504E-4],"split_indices":[50,53,9,0,64,8,0,65,0,0,0,66,1,47,10,0,0,8,36,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.1E2,6.6E1,4E0,2.06E2,1.3E1,5.3E1,2.01E2,5E0,4E0,9E0,9E0,4.4E1,3.5E1,1.66E2,4E0,5E0,2.4E1,2E1,2.4E1,1.1E1,1.3E1,1.53E2,1.6E1,8E0,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.5278658E-3,-5.5954815E-3,3.5566084E-2,-5.01612E-3,-2.7383324E-3,7.726234E-2,1.6478842E-2,-2.2740835E-2,7.011117E-3,9.9976234E-2,8.728227E-4,4.5946147E-2,-1.0966758E-2,2.2829724E-2,-4.459701E-2,3.093491E-3,4.567857E-3,5.9645823E-3,1.5038966E-3,3.6933392E-3,3.2593834E-4,1.5452945E-2,-2.5704904E-3,-2.1522208E-3,2.0073599E-3,-1.5597509E-3,-5.33039E-3,-1.5168267E-3,6.9419364E-4,-5.6260364E-4,1.4818E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,21,23,25,27,-1,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.48089E-2,6.584796E-2,4.2499468E-2,0E0,4.286737E-2,2.0618312E-2,3.2045048E-2,7.2730005E-2,5.315236E-2,1.9655757E-2,0E0,2.4741124E-2,2.5753707E-2,3.6719393E-2,3.3464737E-2,6.2257607E-2,0E0,0E0,0E0,0E0,0E0,6.8752775E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,21,21],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,22,24,26,28,-1,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5642393E0,1.0893594E6,1.1244E4,-5.01612E-3,5.37009E-1,2.6807916E0,3.218279E-1,1.0937031E5,1.3988934E2,8.04E2,8.728227E-4,1.286E3,1.1450704E0,2.9751791E4,5.128017E-1,6.1137104E0,4.567857E-3,5.9645823E-3,1.5038966E-3,3.6933392E-3,3.2593834E-4,5.5566853E-1,-2.5704904E-3,-2.1522208E-3,2.0073599E-3,-1.5597509E-3,-5.33039E-3,-1.5168267E-3,6.9419364E-4,-5.6260364E-4,1.4818E-3],"split_indices":[50,53,9,0,35,47,46,36,64,0,0,0,65,41,35,62,0,0,0,0,0,35,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.23E2,5.4E1,5E0,2.18E2,1.6E1,3.8E1,7.1E1,1.47E2,1.1E1,5E0,1.8E1,2E1,2.3E1,4.8E1,1.42E2,5E0,7E0,4E0,9E0,9E0,1.3E1,7E0,5E0,1.8E1,4.3E1,5E0,3.5E1,1.07E2,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[6.395134E-4,-1.5665783E-2,1.4751886E-2,-8.575736E-3,-7.3292896E-2,5.8230027E-3,1.0797658E-2,-3.1852767E-2,1.0834551E-2,-2.5615277E-4,-4.596199E-3,4.204454E-3,6.765846E-2,2.0771537E-2,-4.7761623E-2,2.6661098E-2,-3.4489498E-2,6.5769635E-2,-3.1355205E-3,3.8448356E-2,5.450755E-3,-9.982388E-4,3.403063E-3,-1.4416008E-3,-6.3768644E-3,-1.609256E-3,2.4429143E-3,-4.0556826E-3,1.056254E-3,4.857906E-3,5.0431397E-4,-8.9291524E-4,8.2289474E-4,2.333194E-3,6.279747E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.397057E-2,5.207814E-2,6.445889E-2,5.2755516E-2,2.4605714E-2,0E0,5.40753E-2,4.5363285E-2,4.6828214E-2,0E0,0E0,5.9403397E-2,1.7675802E-2,3.093113E-2,5.6865945E-2,7.9926684E-2,5.4350995E-2,2.9343918E-2,4.0651336E-2,2.6458073E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1028036E3,1E0,1.1424464E3,6.748543E-1,3.1067E4,5.8230027E-3,1.9894526E6,4.573327E5,5.5038383E4,-2.5615277E-4,-4.596199E-3,9.12314E4,9.5096946E-1,1.691E3,2.0301949E1,1.308E3,9.049849E0,7.34E2,1.2737E4,1.884E3,5.450755E-3,-9.982388E-4,3.403063E-3,-1.4416008E-3,-6.3768644E-3,-1.609256E-3,2.4429143E-3,-4.0556826E-3,1.056254E-3,4.857906E-3,5.0431397E-4,-8.9291524E-4,8.2289474E-4,2.333194E-3,6.279747E-4],"split_indices":[63,92,4,35,37,0,36,56,41,0,0,36,35,2,64,2,64,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.28E2,1.48E2,1.15E2,1.3E1,4E0,1.44E2,5.2E1,6.3E1,4E0,9E0,1.3E2,1.4E1,1.2E1,4E1,4.7E1,1.6E1,1.3E1,1.17E2,1E1,4E0,7E0,5E0,3.5E1,5E0,1.4E1,3.3E1,8E0,8E0,7E0,6E0,6.6E1,5.1E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.2647817E-3,-3.960561E-3,7.495663E-2,-7.399385E-3,5.1324658E-2,5.1992256E-3,1.0127233E-3,-2.1609852E-2,4.0029977E-3,-3.563624E-4,8.163105E-2,1.9337832E-3,-4.0084593E-2,7.900048E-2,-2.0288248E-3,5.5451132E-3,1.6430384E-3,-8.620075E-4,1.4635911E-3,-5.9360935E-4,-3.4598422E-3,4.5086982E-4,6.733487E-3,-2.7584003E-3,3.5162084E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4131616E-2,4.8669267E-2,1.561391E-2,3.8974468E-2,2.7482428E-2,0E0,0E0,4.6618514E-2,6.0778063E-2,0E0,1.3531923E-2,3.0196609E-2,5.6235023E-2,4.5109395E-2,7.039416E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,14,14],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4702875E4,1E0,2.2047469E4,6.540508E-1,4.22631E5,5.1992256E-3,1.0127233E-3,6.5142856E0,8.1446184E7,-3.563624E-4,8.63332E5,2.9836262E8,1.7237E4,1.888492E5,1.8716404E10,5.5451132E-3,1.6430384E-3,-8.620075E-4,1.4635911E-3,-5.9360935E-4,-3.4598422E-3,4.5086982E-4,6.733487E-3,-2.7584003E-3,3.5162084E-4],"split_indices":[60,96,4,35,36,0,0,61,5,0,1,7,9,55,39,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.61E2,2.53E2,8E0,2.39E2,1.4E1,4E0,4E0,1.06E2,1.33E2,5E0,9E0,4.7E1,5.9E1,9E0,1.24E2,4E0,5E0,2.8E1,1.9E1,3.4E1,2.5E1,5E0,4E0,1.7E1,1.07E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[3.0469906E-3,-2.0171374E-3,4.724929E-2,6.9704913E-3,-2.5043124E-2,6.1464934E-3,2.7311616E-2,-9.808193E-3,3.4009974E-2,2.138086E-3,-3.386266E-2,-3.961915E-3,4.9932845E-2,1.793693E-2,-3.4095284E-2,7.153649E-2,9.076023E-3,-9.3063265E-3,-6.750581E-2,8.3550665E-4,-8.8343595E-4,3.2279033E-3,1.1987665E-3,5.8258458E-5,5.0402256E-3,8.837738E-4,-2.5839314E-3,1.5898412E-3,4.5506456E-3,-8.278152E-4,2.71529E-3,3.7066573E-3,-1.6970146E-3,-4.585936E-3,-2.8480956E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.098145E-2,5.0877072E-2,4.450392E-2,8.0636166E-2,4.507265E-2,0E0,1.7176528E-2,7.473488E-2,6.2977284E-2,0E0,5.059322E-2,4.0511587E-3,4.9244463E-3,7.921557E-2,7.033354E-2,2.3866162E-2,5.7763547E-2,9.4527245E-2,5.0560646E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.318116E3,3.291238E7,2.2779044E-3,5.4757043E2,1.6592433E9,6.1464934E-3,1.0526E4,4.61E2,1.0397707E6,2.138086E-3,1E0,1.6792546E7,2.7445236E-1,3.267E3,1.4135E4,9.440229E10,1.6468374E3,8.19162E5,5.9862564E1,8.3550665E-4,-8.8343595E-4,3.2279033E-3,1.1987665E-3,5.8258458E-5,5.0402256E-3,8.837738E-4,-2.5839314E-3,1.5898412E-3,4.5506456E-3,-8.278152E-4,2.71529E-3,3.7066573E-3,-1.6970146E-3,-4.585936E-3,-2.8480956E-4],"split_indices":[60,53,65,60,5,0,9,0,55,0,109,55,46,10,9,39,60,37,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.44E2,2.7E1,1.76E2,6.8E1,4E0,2.3E1,1.09E2,6.7E1,7E0,6.1E1,1E1,1.3E1,5.1E1,5.8E1,2.6E1,4.1E1,3.6E1,2.5E1,4E0,6E0,6E0,7E0,4.4E1,7E0,1.7E1,4.1E1,1.2E1,1.4E1,2.7E1,1.4E1,8E0,2.8E1,1.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-3.2690374E-4,2.5380948E-3,-5.3306025E-2,-1.3533867E-2,1.5267319E-2,-4.1693854E-3,1.4668268E-3,-4.208316E-2,2.2535312E-3,8.885616E-3,8.525739E-2,-9.8786615E-2,-1.7999005E-2,2.3381934E-2,-2.5345705E-2,1.3360155E-2,-2.9124834E-3,5.706513E-3,1.1930815E-3,-1.7312444E-3,-5.7403324E-3,1.1150218E-3,-1.8563017E-3,6.358493E-5,3.170228E-3,2.0775189E-4,-2.830427E-3,4.2657657E-3,3.66613E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.142084E-2,5.319378E-2,4.7301054E-2,5.1927526E-2,6.400752E-2,0E0,0E0,5.4213785E-2,4.4327214E-2,4.4101834E-2,2.4338E-2,1.4322013E-2,2.9244166E-2,4.2925626E-2,3.5877265E-2,5.3614102E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.0532868E6,6.2711465E-1,3.7491336E-1,2.9751791E4,1E0,-4.1693854E-3,1.4668268E-3,4.899767E6,1.7328871E6,1.402511E8,3.0143394E7,8.146263E0,1.2109598E7,1.3238162E3,6.5142856E0,1E0,-2.9124834E-3,5.706513E-3,1.1930815E-3,-1.7312444E-3,-5.7403324E-3,1.1150218E-3,-1.8563017E-3,6.358493E-5,3.170228E-3,2.0775189E-4,-2.830427E-3,4.2657657E-3,3.66613E-4],"split_indices":[36,35,65,41,112,0,0,53,55,53,53,62,53,4,61,14,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.58E2,1.3E1,1.14E2,1.44E2,9E0,4E0,4E1,7.4E1,1.33E2,1.1E1,1.1E1,2.9E1,4.2E1,3.2E1,1.26E2,7E0,6E0,5E0,4E0,7E0,1E1,1.9E1,2.9E1,1.3E1,1.8E1,1.4E1,7E0,1.19E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.5013305E-3,-6.439301E-3,2.2351691E-2,9.364468E-3,-2.557249E-2,1.5864156E-2,4.491678E-3,1.5972052E-2,-6.287079E-3,-4.6014555E-2,9.617631E-3,4.351542E-2,-4.439413E-3,-1.0871909E-3,4.2181287E-2,-5.6330703E-2,3.529783E-3,6.375022E-2,-1.1572506E-2,-4.1745935E-4,5.8814574E-2,-1.6839083E-2,4.4274792E-2,-3.7138636E-4,3.278707E-3,2.4759406E-3,-2.6105272E-3,-3.049351E-3,1.6372788E-3,6.099213E-4,4.383612E-3,6.705015E-5,-2.1705818E-3,1.555909E-3,-1.5905057E-3,1.9632935E-4,3.2263005E-3,-1.6822564E-4,-2.8031003E-3,3.1572017E-3,5.1202305E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6833266E-2,6.2226653E-2,3.650886E-2,1.11131884E-1,6.734292E-2,4.1176796E-2,0E0,4.8780818E-2,0E0,7.89885E-2,4.088545E-2,2.0860769E-2,2.671927E-2,3.438244E-2,5.235204E-2,5.2392587E-2,0E0,1.4627274E-2,1.2118639E-2,1.1675502E-2,1.3828449E-2,2.0102946E-2,6.4877737E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5699575E3,7.34E2,2.4612432E6,5.138372E8,3.4741312E8,9.789218E6,4.491678E-3,2.734E3,-6.287079E-3,7.2838904E7,5.1446883E4,6.2711465E-1,1.4964736E4,5.275742E5,9.3235035E0,1E0,3.529783E-3,1.311311E6,1.7473396E7,2.4394053E9,2.692487E6,4.2847183E2,5.870237E0,-3.7138636E-4,3.278707E-3,2.4759406E-3,-2.6105272E-3,-3.049351E-3,1.6372788E-3,6.099213E-4,4.383612E-3,6.705015E-5,-2.1705818E-3,1.555909E-3,-1.5905057E-3,1.9632935E-4,3.2263005E-3,-1.6822564E-4,-2.8031003E-3,3.1572017E-3,5.1202305E-4],"split_indices":[60,0,36,7,7,40,0,2,0,40,41,35,63,55,61,96,0,1,58,5,1,66,61,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,2.04E2,7.7E1,1.12E2,9.2E1,7.2E1,5E0,1.08E2,4E0,5.8E1,3.4E1,3E1,4.2E1,6.6E1,4.2E1,5.4E1,4E0,9E0,2.5E1,8E0,2.2E1,3.4E1,8E0,6.1E1,5E0,3.8E1,4E0,4.9E1,5E0,4E0,5E0,1.9E1,6E0,4E0,4E0,4E0,1.8E1,2.7E1,7E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.7906722E-3,-1.5498284E-2,1.4790369E-2,4.0610865E-2,-2.8967202E-2,3.594864E-2,-5.3219153E-3,2.1599634E-3,8.309409E-2,-3.627489E-2,2.317513E-3,5.6384904E-3,2.9941741E-2,-2.3184327E-2,2.3204327E-2,-1.4963073E-3,1.2250647E-3,6.019632E-4,6.284368E-3,-1.0753777E-2,-6.149729E-2,3.4877505E-2,-2.2604284E-3,3.242662E-2,-3.3560723E-2,6.1136607E-2,-6.4051016E-3,-2.1352149E-3,5.960417E-4,-3.469228E-3,1.816978E-3,1.9638513E-3,-6.234077E-4,3.0155184E-3,-3.281977E-4,7.321637E-4,-2.2818241E-3,9.976447E-4,3.3848807E-3,-1.0498902E-3,1.6417335E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,27,29,31,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.603064E-2,7.3726416E-2,7.168224E-2,3.0857366E-2,4.7028616E-2,3.894891E-2,4.4868607E-2,1.0185675E-2,3.186731E-2,4.568535E-2,0E0,0E0,3.214103E-2,3.224849E-2,3.8578793E-2,0E0,0E0,0E0,0E0,3.1837333E-2,5.3325683E-2,2.844131E-2,0E0,1.2239515E-2,3.7192795E-2,5.7374276E-3,1.4608355E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,19,19,20,20,21,21,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,28,30,32,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.249921E2,4.7001828E7,4.66667E5,1.4719672E2,7.2838904E7,4.9695907E0,9.467578E6,4.3109778E-1,6.1E1,1.25615805E5,2.317513E-3,5.6384904E-3,4.5E1,2.3E1,7.9684106E9,-1.4963073E-3,1.2250647E-3,6.019632E-4,6.284368E-3,1.295E3,3.37353E5,4.080045E7,-2.2604284E-3,2.692487E6,7.11E2,5.121875E1,1.36332E5,-2.1352149E-3,5.960417E-4,-3.469228E-3,1.816978E-3,1.9638513E-3,-6.234077E-4,3.0155184E-3,-3.281977E-4,7.321637E-4,-2.2818241E-3,9.976447E-4,3.3848807E-3,-1.0498902E-3,1.6417335E-3],"split_indices":[60,5,37,60,40,62,1,35,0,55,0,0,8,3,12,0,0,0,0,2,9,53,0,1,0,64,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.62E2,9.5E1,1.67E2,1.8E1,7.7E1,8.1E1,8.6E1,1E1,8E0,7.1E1,6E0,4E0,7.7E1,5.3E1,3.3E1,4E0,6E0,4E0,4E0,3.6E1,3.5E1,7.3E1,4E0,8E0,4.5E1,1.4E1,1.9E1,1.4E1,2.2E1,3.1E1,4E0,6.3E1,1E1,4E0,4E0,1.1E1,3.4E1,4E0,1E1,1.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.080507E-3,-3.4665603E-3,3.1659607E-2,5.705969E-3,-3.1556968E-2,4.6864273E-3,1.7908067E-2,-5.1713935E-3,3.2162104E-2,2.6248991E-3,-4.0988244E-2,5.4682408E-2,9.006538E-4,-7.2483567E-4,-3.7428867E-3,9.222945E-2,9.7747985E-3,-9.373637E-2,-1.5644828E-2,9.159267E-4,3.2039997E-3,-2.7110936E-2,2.5117321E-2,4.929693E-4,-1.7991839E-3,2.8139332E-3,6.2618894E-3,2.7060956E-3,-6.528741E-4,-5.302714E-3,-8.863971E-4,1.1260612E-3,-1.5447692E-3,-4.0384877E-4,-2.5184343E-3,1.9948452E-3,3.7583386E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,-1,25,27,29,31,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.5723226E-2,6.0730185E-2,4.060961E-2,5.138991E-2,5.078538E-2,0E0,2.3717336E-2,4.29288E-2,6.902527E-2,0E0,6.938485E-2,4.8390105E-3,1.8996334E-2,5.3471025E-2,0E0,1.0755226E-2,4.661241E-2,2.492708E-2,2.7391184E-2,0E0,0E0,6.0977275E-3,4.277438E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,-1,26,28,30,32,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5078225E3,3.291238E7,2.2779044E-3,8.4569194E10,1.6592433E9,4.6864273E-3,6.972111E6,5.138372E8,2E0,2.6248991E-3,8.607021E2,5.7625353E-1,1.4861412E0,7.8908873E0,-3.7428867E-3,1.6726906E11,8.501634E6,4.2845196E1,1.7129352E3,9.159267E-4,3.2039997E-3,1.03232314E12,5.6431256E5,4.929693E-4,-1.7991839E-3,2.8139332E-3,6.2618894E-3,2.7060956E-3,-6.528741E-4,-5.302714E-3,-8.863971E-4,1.1260612E-3,-1.5447692E-3,-4.0384877E-4,-2.5184343E-3,1.9948452E-3,3.7583386E-4],"split_indices":[60,53,65,39,5,0,55,7,8,0,60,35,50,61,0,39,53,64,4,0,0,39,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.34E2,4.3E1,1.77E2,5.7E1,6E0,3.7E1,1.26E2,5.1E1,5E0,5.2E1,1.1E1,2.6E1,1.2E2,6E0,1.3E1,3.8E1,1.6E1,3.6E1,4E0,7E0,1.2E1,1.4E1,9.3E1,2.7E1,9E0,4E0,1.2E1,2.6E1,1.2E1,4E0,1.1E1,2.5E1,8E0,4E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-8.971966E-4,2.5821688E-2,-6.2839137E-3,6.095389E-2,-2.7311116E-2,-4.472092E-2,-1.1151853E-3,-1.2642875E-3,8.1368215E-2,2.536046E-3,-3.1615458E-3,4.0364813E-4,-6.0420703E-2,5.179319E-2,-5.028424E-3,5.1332302E-3,3.1057447E-2,1.2815379E-3,-6.2581786E-4,-7.410622E-2,-6.379398E-4,8.492626E-2,-3.0104496E-4,-6.0083155E-2,-6.204996E-4,4.0090195E-4,2.0491113E-3,-3.8474575E-3,-1.6842731E-3,2.0022807E-3,5.0358903E-3,-1.617959E-4,-4.1952566E-3,2.0535956E-3,-2.641055E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,-1,25,-1,-1,27,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.945435E-2,8.753165E-2,4.524699E-2,5.277183E-2,2.3627454E-2,2.3202986E-2,4.208314E-2,0E0,3.139554E-2,5.274739E-3,0E0,0E0,1.23557225E-2,2.7613271E-2,4.582869E-2,0E0,2.6182476E-3,0E0,0E0,2.622418E-3,0E0,4.3622553E-3,0E0,2.4259835E-2,4.0974785E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,12,12,13,13,14,14,16,16,19,19,21,21,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,-1,26,-1,-1,28,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.275E3,8.93182E5,1.3324517E5,1.657E3,3.2441288E5,9.242357E9,5.6338026E-3,-1.2642875E-3,1.006E3,1.096E3,-3.1615458E-3,4.0364813E-4,1.6E1,8.87E2,1.4336918E-2,5.1332302E-3,3E0,1.2815379E-3,-6.2581786E-4,2.479E3,-6.379398E-4,1.3474202E8,-3.0104496E-4,6.4745765E0,2.356535E9,4.0090195E-4,2.0491113E-3,-3.8474575E-3,-1.6842731E-3,2.0022807E-3,5.0358903E-3,-1.617959E-4,-4.1952566E-3,2.0535956E-3,-2.641055E-4],"split_indices":[9,37,40,2,36,39,65,0,0,0,0,0,8,0,65,0,8,0,0,11,0,7,0,62,39,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,4.5E1,2.27E2,2.7E1,1.8E1,2.6E1,2.01E2,5E0,2.2E1,1.1E1,7E0,6E0,2E1,1.3E1,1.88E2,1.3E1,9E0,4E0,7E0,1.5E1,5E0,8E0,5E0,1.3E1,1.75E2,4E0,5E0,1.1E1,4E0,4E0,4E0,5E0,8E0,1.7E1,1.58E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.139894E-4,-1.9236285E-2,9.573939E-3,-7.843109E-3,-9.2638336E-2,6.80766E-3,4.2306553E-3,-1.905686E-2,3.750209E-2,-7.736995E-4,-6.6485433E-3,9.546042E-3,-2.4837034E-3,1.3368953E-2,-4.0185433E-2,-2.4214982E-5,4.6487567E-3,1.254292E-2,-2.1696093E-3,-9.020735E-4,2.766974E-3,5.608188E-4,-2.700897E-3,3.272947E-4,1.8626725E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,19,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.0430957E-2,7.453885E-2,4.1955434E-2,4.0885366E-2,4.4680975E-2,3.079342E-2,0E0,4.4384345E-2,3.8669944E-2,0E0,0E0,3.0311538E-2,0E0,4.1224524E-2,3.84506E-2,0E0,0E0,2.5044367E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,13,13,14,14,17,17],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,20,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.449939E2,5.9952423E2,1E0,5.3153326E2,1.4084E4,1.4623123E8,4.2306553E-3,8.1446184E7,7.0120897E0,-7.736995E-4,-6.6485433E-3,9.997036E0,-2.4837034E-3,1.6414569E2,2.8434807E3,-2.4214982E-5,4.6487567E-3,1.07431E1,-2.1696093E-3,-9.020735E-4,2.766974E-3,5.608188E-4,-2.700897E-3,3.272947E-4,1.8626725E-3],"split_indices":[4,63,112,4,9,53,0,5,61,0,0,61,0,60,55,0,0,62,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,8.9E1,1.86E2,7.8E1,1.1E1,1.81E2,5E0,6.3E1,1.5E1,5E0,6E0,1.74E2,7E0,2.5E1,3.8E1,1E1,5E0,1.66E2,8E0,1.5E1,1E1,1E1,2.8E1,1.4E2,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-6.3497963E-4,-5.608508E-3,3.037021E-2,-2.7667964E-3,-5.5672113E-2,5.0042765E-3,1.5655546E-2,3.0379961E-3,-3.232898E-2,-4.0431046E-3,-1.6279973E-4,2.9133854E-3,9.653557E-4,3.548692E-4,3.437661E-3,-1.0129516E-2,-6.222015E-2,-2.8002685E-2,2.9864443E-2,-1.3515035E-3,2.846092E-4,3.1144363E-3,-1.384146E-3,-3.5099054E-3,-1.2043237E-3,-9.5130265E-5,-2.8656088E-3,2.5788697E-3,6.3555187E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,-1,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.350199E-2,3.4416683E-2,4.3244824E-2,3.9738357E-2,2.147511E-2,0E0,2.3724727E-2,3.725733E-2,2.4585322E-2,0E0,0E0,0E0,2.3439333E-2,3.272587E-2,0E0,3.769134E-2,6.0920417E-3,1.2303441E-2,5.1060608E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,-1,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2838904E7,3.013037E6,4.514673E-3,1.9989124E7,1.0422E4,5.0042765E-3,1.2003011E8,1.3988934E2,7.1423984E0,-4.0431046E-3,-1.6279973E-4,2.9133854E-3,1.6394367E0,7.056912E7,3.437661E-3,5.033623E1,1.4505113E1,1.1947E4,2.2100918E1,-1.3515035E-3,2.846092E-4,3.1144363E-3,-1.384146E-3,-3.5099054E-3,-1.2043237E-3,-9.5130265E-5,-2.8656088E-3,2.5788697E-3,6.3555187E-4],"split_indices":[40,36,65,40,9,0,40,64,61,0,0,0,50,7,0,66,62,9,64,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.42E2,3.8E1,2.3E2,1.2E1,5E0,3.3E1,1.93E2,3.7E1,7E0,5E0,7E0,2.6E1,1.87E2,6E0,2.2E1,1.5E1,1.3E1,1.3E1,3E1,1.57E2,4E0,1.8E1,1E1,5E0,8E0,5E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.7268125E-3,-1.9453868E-2,1.1271757E-2,1.2110878E-2,-4.7706958E-2,2.0286476E-2,-1.702452E-2,-4.499717E-2,3.2646798E-2,-1.1752361E-1,-2.0873422E-2,2.4276817E-2,-2.237077E-3,-5.267868E-3,-4.732189E-3,2.745423E-4,-3.4516572E-3,2.7040383E-3,6.240929E-2,-2.6029856E-3,-6.489501E-3,-1.6352928E-3,-3.7195722E-3,3.843955E-2,2.1738345E-3,-2.9256833E-3,7.1655978E-3,2.0402842E-3,-9.3179604E-4,4.0387018E-3,4.5230516E-4,-1.9271593E-3,8.5482653E-4,1.5110847E-3,5.4000146E-3,-2.4257798E-3,8.985287E-4,2.1321608E-3,-1.2999908E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,-1,-1,27,29,-1,-1,31,-1,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.399608E-2,7.4597314E-2,4.7280647E-2,4.837967E-2,7.99777E-2,3.9656565E-2,5.3331546E-2,1.7248167E-2,2.6566379E-2,9.176016E-3,3.7905253E-2,4.1550577E-2,0E0,0E0,2.9144814E-2,0E0,0E0,1.6171144E-2,1.9102484E-2,0E0,0E0,2.1903798E-2,0E0,3.2038227E-2,5.145249E-2,0E0,4.980895E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,17,17,18,18,21,21,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,-1,-1,28,30,-1,-1,32,-1,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.555544E2,1.6126542E8,6.971004E7,2.8246236E-1,1.5662762E10,9.997036E0,1.1992621E3,1.51E2,1.278E3,1.2371134E-2,3.0686172E6,9.6494156E-1,-2.237077E-3,-5.267868E-3,5.36E3,2.745423E-4,-3.4516572E-3,1.1826522E3,1.7128E4,-2.6029856E-3,-6.489501E-3,1.3324517E5,-3.7195722E-3,2.9997022E5,1.2936785E3,-2.9256833E-3,1.4792529E10,2.0402842E-3,-9.3179604E-4,4.0387018E-3,4.5230516E-4,-1.9271593E-3,8.5482653E-4,1.5110847E-3,5.4000146E-3,-2.4257798E-3,8.985287E-4,2.1321608E-3,-1.2999908E-3],"split_indices":[4,5,53,35,39,61,4,0,2,65,58,65,0,0,2,0,0,41,37,0,0,40,0,41,4,0,5,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,8.2E1,1.83E2,3.9E1,4.3E1,1.39E2,4.4E1,1E1,2.9E1,1.1E1,3.2E1,1.32E2,7E0,4E0,4E1,4E0,6E0,1.5E1,1.4E1,4E0,7E0,2.5E1,7E0,8E1,5.2E1,6E0,3.4E1,5E0,1E1,9E0,5E0,8E0,1.7E1,7.6E1,4E0,1.2E1,4E1,1.6E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.271051E-3,-2.974091E-2,9.385245E-3,7.798032E-3,-4.7932934E-2,7.8460254E-2,6.3144756E-3,-1.3582414E-3,1.7302507E-3,-4.324173E-3,-1.353994E-3,5.4279757E-3,1.4973416E-3,-3.28834E-3,9.0998905E-3,1.3513592E-2,-3.357887E-2,3.336227E-3,4.626162E-4,1.4868195E-3,-2.3470726E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,-1,-1,-1,15,17,19,-1,-1,-1,-1],"loss_changes":[3.8307264E-2,1.9495247E-2,4.9775444E-2,1.2217159E-2,1.342383E-2,1.4281195E-2,5.0018333E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.208383E-2,4.0335536E-2,2.57145E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,-1,-1,-1,16,18,20,-1,-1,-1,-1],"split_conditions":[1.292E3,2.8434807E3,3.0404909E0,6.8E1,8.35E2,8.110325E0,7.784166E5,-1.3582414E-3,1.7302507E-3,-4.324173E-3,-1.353994E-3,5.4279757E-3,1.4973416E-3,-3.28834E-3,8.903381E0,2.0133628E6,2.5E1,3.336227E-3,4.626162E-4,1.4868195E-3,-2.3470726E-3],"split_indices":[2,55,64,0,2,62,53,0,0,0,0,0,0,0,61,53,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.63E2,2.7E1,2.36E2,9E0,1.8E1,9E0,2.27E2,4E0,5E0,4E0,1.4E1,4E0,5E0,7E0,2.2E2,2E2,2E1,1E1,1.9E2,4E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.4887268E-3,7.616793E-3,-2.5408603E-2,5.008144E-2,1.9031183E-3,-7.8078486E-2,8.94396E-4,3.381211E-2,5.2903052E-3,-3.2127E-3,3.6656838E-2,-1.4846979E-3,-5.4928344E-3,-1.3047775E-2,2.6019649E-3,6.505302E-2,1.2813416E-2,-7.920782E-3,4.4657424E-2,6.627418E-2,1.0519174E-2,-2.0572215E-2,9.051461E-4,4.460861E-3,9.3033735E-4,-9.6851354E-4,2.074657E-3,-1.4830867E-3,2.565779E-4,-7.488059E-4,4.9545565E-3,9.552589E-4,4.412064E-3,2.4401243E-3,-4.487194E-4,-1.7299151E-4,-1.350723E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.86765E-2,5.5224147E-2,5.825562E-2,2.458962E-2,3.6032096E-2,2.2603966E-2,2.322567E-2,1.4254874E-2,0E0,4.0245354E-2,1.9430183E-2,0E0,0E0,6.310943E-3,0E0,1.0891642E-2,1.7321274E-2,5.3279754E-2,6.3114285E-2,1.415791E-2,1.3429454E-2,2.837468E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.294459E7,7.31E3,1.2286531E3,9.4047743E-1,1E0,6.3901825E5,8.991292E5,3.296197E5,5.2903052E-3,1E0,2.8582031E1,-1.4846979E-3,-5.4928344E-3,1.4964736E4,2.6019649E-3,8.93182E5,2.4853547E-1,2.9751791E4,1.661E3,9.006097E5,3.835264E-2,2.7819494E5,9.051461E-4,4.460861E-3,9.3033735E-4,-9.6851354E-4,2.074657E-3,-1.4830867E-3,2.565779E-4,-7.488059E-4,4.9545565E-3,9.552589E-4,4.412064E-3,2.4401243E-3,-4.487194E-4,-1.7299151E-4,-1.350723E-3],"split_indices":[53,9,60,35,84,36,36,36,0,112,66,0,0,63,0,37,47,41,2,40,35,36,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,2.27E2,4.1E1,2.6E1,2.01E2,1.3E1,2.8E1,2.2E1,4E0,1.76E2,2.5E1,7E0,6E0,2.3E1,5E0,8E0,1.4E1,1.61E2,1.5E1,1.1E1,1.4E1,1.9E1,4E0,4E0,4E0,7E0,7E0,5.7E1,1.04E2,8E0,7E0,5E0,6E0,4E0,1E1,7E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.6098587E-5,-1.7561223E-2,7.4125715E-3,1.7497944E-2,-2.7539115E-2,3.734725E-2,5.29882E-4,-1.0141945E-2,2.6137754E-3,-5.4140903E-2,-3.1906974E-3,7.35691E-2,-8.4024353E-4,-1.6252685E-2,1.2843363E-2,-1.171523E-3,7.537666E-4,9.953302E-4,-7.318785E-2,-2.0434013E-2,2.7562145E-2,9.73377E-2,8.7098335E-4,-3.5712834E-2,2.7768262E-2,-2.490967E-2,2.9457987E-3,6.781373E-3,3.552624E-3,-4.5413766E-3,-1.7218234E-3,1.62434E-5,-3.13066E-3,2.4953E-3,-1.1921651E-3,2.2259215E-3,5.6130458E-3,-5.465999E-4,-2.413062E-3,2.3638853E-3,3.205508E-4,-1.666251E-3,1.7724901E-3,1.0845466E-3,-5.4109195E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,-1,29,31,33,35,-1,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.64759E-2,2.9516654E-2,4.0774368E-2,2.1200074E-2,4.19285E-2,5.1241014E-2,3.3686336E-2,5.2854987E-3,0E0,4.6314627E-2,1.9123042E-2,2.2839442E-2,1.9947086E-2,4.9520507E-2,3.6127288E-2,0E0,0E0,0E0,1.9082487E-2,2.2763764E-2,1.9912742E-2,9.758145E-3,0E0,2.9606763E-3,5.037982E-3,4.6456825E-2,0E0,2.7324976E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,19,19,20,20,21,21,23,23,24,24,25,25,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,-1,30,32,34,36,-1,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.593E3,8.1446184E7,1.4265206E1,4.6966617E4,3.34226E8,5.9E2,7.036E3,7.2315273E0,2.6137754E-3,5.244755E-3,3.8053186E2,1.5363803E8,1.6391889E5,1E0,2.4612432E6,-1.171523E-3,7.537666E-4,9.953302E-4,1.872E3,7.25E2,1.9561611E1,9.815951E-3,8.7098335E-4,2.7953E4,9.180692E7,2.6493E4,2.9457987E-3,3.0703132E7,3.552624E-3,-4.5413766E-3,-1.7218234E-3,1.62434E-5,-3.13066E-3,2.4953E-3,-1.1921651E-3,2.2259215E-3,5.6130458E-3,-5.465999E-4,-2.413062E-3,2.3638853E-3,3.205508E-4,-1.666251E-3,1.7724901E-3,1.0845466E-3,-5.4109195E-4],"split_indices":[2,5,64,41,5,0,2,61,0,65,60,12,36,96,36,0,0,0,2,0,64,65,0,9,7,10,0,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,8.2E1,1.97E2,1.8E1,6.4E1,3.6E1,1.61E2,1.1E1,7E0,3E1,3.4E1,1.8E1,1.8E1,6.8E1,9.3E1,7E0,4E0,6E0,2.4E1,2.2E1,1.2E1,1.2E1,6E0,8E0,1E1,6.2E1,6E0,8.6E1,7E0,1.3E1,1.1E1,1.6E1,6E0,8E0,4E0,5E0,7E0,4E0,4E0,4E0,6E0,5.3E1,9E0,4.5E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.3701476E-3,-1.6134817E-3,3.993964E-2,3.9691504E-2,-4.8631965E-3,7.238194E-2,-3.570256E-4,-1.709447E-3,8.58825E-2,-5.426412E-2,-1.8084765E-3,4.658503E-3,5.4449425E-4,-1.2236601E-3,7.50364E-4,5.8280155E-3,1.2894021E-3,-1.1635666E-5,-7.584341E-2,1.4991308E-2,-1.2673024E-2,-1.263519E-3,-4.7669187E-3,1.6066218E-3,-1.1057389E-3,-1.833664E-3,3.6145255E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,21,23,25,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2055866E-2,3.495845E-2,3.150744E-2,3.6481485E-2,3.627021E-2,1.9731112E-2,0E0,5.3590047E-3,1.7655604E-2,1.629822E-2,4.180501E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0561984E-2,7.071524E-2,7.777649E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,22,24,26,-1,-1,-1,-1,-1,-1],"split_conditions":[4.8008E4,2.356535E9,8.805777E7,1.4068772E6,8.8916E4,3.0291426E-1,-3.570256E-4,1.2940955E6,1.5489E4,6.554877E-4,9.360022E5,4.658503E-3,5.4449425E-4,-1.2236601E-3,7.50364E-4,5.8280155E-3,1.2894021E-3,-1.1635666E-5,7.2929355E6,3.607011E1,1.2707859E3,-1.263519E-3,-4.7669187E-3,1.6066218E-3,-1.1057389E-3,-1.833664E-3,3.6145255E-4],"split_indices":[2,39,53,58,1,46,0,53,9,47,56,0,0,0,0,0,0,0,53,64,4,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.58E2,1.9E1,1.8E1,2.4E2,1.1E1,8E0,1E1,8E0,1.3E1,2.27E2,7E0,4E0,4E0,6E0,4E0,4E0,4E0,9E0,8.9E1,1.38E2,4E0,5E0,5.9E1,3E1,5.9E1,7.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.7704475E-3,-6.4291754E-3,1.4794891E-2,-6.417873E-4,-5.1476102E-2,4.3763544E-2,3.0619022E-4,1.35106975E-2,-2.2459317E-2,-1.4876264E-2,-4.1662403E-3,6.647959E-2,-1.1842349E-2,-1.5634539E-2,1.8113293E-2,2.2769894E-2,-3.904131E-2,-5.3597588E-2,-3.8408635E-3,1.0982683E-3,-2.353953E-3,9.294685E-2,1.748403E-2,-2.3985736E-3,8.564701E-4,-3.1558073E-3,-5.0259404E-2,3.5847737E-3,2.529619E-3,-1.6237571E-4,2.089981E-3,-1.2581017E-4,-2.7122428E-3,-3.9609196E-3,-6.1437604E-4,5.582869E-4,-2.607603E-3,2.248377E-3,4.998152E-3,-1.1156884E-3,2.2713789E-3,-7.1940746E-4,2.1753625E-3,-1.007463E-3,-3.0171075E-3,-1.6065388E-3,8.8599534E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,-1,-1,37,39,-1,-1,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9036706E-2,4.350454E-2,4.4057E-2,4.6310507E-2,2.5783774E-2,4.4997394E-2,2.0437121E-2,4.5168605E-2,3.3999175E-2,1.6663752E-2,0E0,3.0794367E-2,1.4449844E-2,1.6260218E-2,3.145973E-2,4.656498E-2,9.938328E-3,2.7573548E-2,3.2719713E-2,0E0,0E0,5.7174265E-3,1.44820735E-2,0E0,0E0,1.8965242E-2,2.8997175E-3,0E0,1.8185223E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,-1,-1,38,40,-1,-1,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.821E3,4.462437E7,4.91101E5,1.6672E4,9.180692E7,3.218279E-1,9.467578E6,8.712121E0,2.4242616E5,1.0363E4,-4.1662403E-3,2E0,3.5286108E-1,2.173481E9,3.5736008E6,7.6077003E0,1.1E2,7.555544E2,4.1969254E8,1.0982683E-3,-2.353953E-3,2.687E3,1.3312868E6,-2.3985736E-3,8.564701E-4,8.513227E0,6.331E3,3.5847737E-3,1.8790482E9,-1.6237571E-4,2.089981E-3,-1.2581017E-4,-2.7122428E-3,-3.9609196E-3,-6.1437604E-4,5.582869E-4,-2.607603E-3,2.248377E-3,4.998152E-3,-1.1156884E-3,2.2713789E-3,-7.1940746E-4,2.1753625E-3,-1.007463E-3,-3.0171075E-3,-1.6065388E-3,8.8599534E-4],"split_indices":[2,53,37,9,7,46,1,61,36,9,0,8,46,12,55,62,10,4,7,0,0,10,36,0,0,62,9,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.66E2,1.04E2,1.48E2,1.8E1,3.4E1,7E1,9E1,5.8E1,1E1,8E0,2.4E1,1E1,3.7E1,3.3E1,7.7E1,1.3E1,2.1E1,3.7E1,5E0,5E0,1.5E1,9E0,4E0,6E0,2.8E1,9E0,6E0,2.7E1,3.6E1,4.1E1,5E0,8E0,1.1E1,1E1,2.9E1,8E0,5E0,1E1,4E0,5E0,2.3E1,5E0,4E0,5E0,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2358004E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics index 26a7fe0d8..a85c44374 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws.metrics @@ -1,116 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,executorCPUTime_mean,0.26506853,2152.2279055967347,4768.119051696857,70.88447653429603,284.2316451435637,672.5167513645774,1630.6395926865093,61271.6827309237 -1,input_bytesRead_mean,0.08203817,32744094.213089865,39629389.010754265,157930.11165644173,4825140.119812995,15745821.867247352,49695222.62577245,178342334.9422222 -2,scan_bw,0.02758161,852348.8649740079,1256066.2869357998,25664.476385554077,244592.89003173803,453737.53107269213,984211.7582137551,14656983.122448979 -3,resultSize_max,0.018899895,99931.8717948718,292860.4886001664,5158.0,8628.0,13086.5,22400.5,2030358.0 -4,numTasks_sum,0.016852418,1183.2788461538462,1204.703444102634,4.0,419.25,807.5,1624.25,11722.0 -5,jvmGCTime_mean,0.016368486,51.26736194273545,136.7085736361219,0.0,9.827855037169956,20.31146613289841,49.76661765200461,2002.3373493975903 -6,input_recordsRead_sum,0.01436129,6899154505.211538,16894449776.068132,600000.0,362916825.5,1646770012.0,6151825905.25,230401504404.0 -7,executorDeserializeCPUTime_mean,0.013984824,6.809404488739635,3.052904823341863,2.46474358974359,5.713835885304287,6.525625811651963,7.75061394891945,37.37243401759531 -8,duration_max,0.013980187,13206.358974358975,28086.892904146353,268.0,2215.75,4042.5,9516.75,214099.0 -9,duration_mean,0.010543714,2933.22180477384,5516.967032475489,144.4834355828221,597.3303784602791,1217.3476729944887,2645.471320224719,66753.3641231593 -10,scan_time,0.010355052,1400805.6185897435,5314404.8107528975,75.0,45099.75,213395.5,913930.75,71049280.0 -11,shuffle_write_bw,0.010250016,97597.46220339966,101684.3639886883,89.38862559241706,5279.766135993155,58133.42077217826,166003.50456482448,363609.36008282914 -12,executorDeserializeTime_mean,0.009677426,10.25826375949487,13.985580724022466,2.33974358974359,6.198198926905551,7.915370390246956,10.337483478096676,167.07771260997066 -13,sr_remoteBytesReadRatio,0.00962542,0.8157941158244443,1.1548408409516235,4.494264550895562e-07,0.0065696072316052765,0.28397315656730227,1.3175949154660336,10.726449601853131 -14,sqlOp_SubqueryBroadcast,0.0095524,0.7467948717948718,0.43554598500503106,0.0,0.0,1.0,1.0,1.0 -15,sr_localBytesRead_mean,0.009114479,3015731.6640652833,6376318.262818828,0.0,41219.35598607367,386142.11036059354,2336170.0426780568,39251927.01077199 -16,sr_fetchWaitTime_mean,0.009097806,2.1910402912994535,9.923627614482589,0.0,0.00936196319018405,0.2188722371459541,1.442273795534665,164.9670855774985 -17,peakExecutionMemory_max,0.007944924,808883609.4615384,1738902255.3593984,0.0,98950280.0,216700784.0,679936172.0,13432783808.0 -18,duration_sum,0.0066407644,4829029.923076923,10325780.420673069,604.0,294242.0,1060120.5,3679291.0,68898339.0 -19,cache_hit_ratio,0.006391222,0.6285631138115668,0.30548501366690567,1.187845066996456e-05,0.40065365426082494,0.7015237671310255,0.9068745980835918,1.0 -20,shuffle_read_bw,0.006100891,581925830.5982301,6398696547.126322,0.0,955032.840866873,4561425.729699758,23080427.753544167,107487374955.0 -21,sw_bytesWrittenRatio,0.0052421656,0.8968440930204622,1.2304635979404275,5.157297693553143e-07,0.006672492716992155,0.36986430596766073,1.4866485306622774,12.23950555496033 -22,sw_writeTime_mean,0.0049655978,118.89261641821152,322.16134532828437,0.23129251700680273,22.18550110537951,45.54013792957777,83.09396412465505,3121.572710951526 -23,sr_remoteBytesRead_mean,0.0048047174,17015457.156458072,38838528.23309634,68.60897435897436,165447.60836938542,1539625.8261178578,11668314.370339088,274787384.05385995 -24,data_size,0.0040127886,567279237847.8557,1789756116215.4756,57333908.0,21732999116.25,90621097739.5,337866257513.75,21680680059621.0 -25,sr_localBytesReadRatio,0.0037421721,0.17264467224546515,0.28254638538639176,0.0,0.0015028110961492072,0.0691738614719501,0.23132092359082565,2.0961972120502077 -26,sw_recordsWritten_sum,0.0037404716,2241520607.669872,5416061058.183992,264.0,1538364.5,136380867.5,1589319942.5,40948506792.0 -27,duration_min,0.0033926074,40.23717948717949,34.89232167158378,8.0,28.0,35.0,41.25,413.0 -28,resultSerializationTime_sum,0.0027190405,19.634615384615383,94.37842595989993,0.0,0.0,1.0,17.0,1511.0 -29,sqlOp_Window,0.0024652344,0.0673076923076923,0.2509566164678633,0.0,0.0,0.0,0.0,1.0 -30,executorRunTime_mean,0.0021331045,2913.129180877565,5517.281917094228,114.91288343558283,573.4333323657474,1179.9774741118122,2628.774232209738,66732.86345381525 -31,sr_localBlocksFetched_sum,0.0020611403,18225.53846153846,25193.523582463084,0.0,434.0,8839.5,25910.5,147330.0 -32,sqlOp_Expand,0.0018685365,0.10576923076923077,0.3080360743511338,0.0,0.0,0.0,0.0,1.0 -33,sw_bytesWritten_mean,0.001802557,18910510.05056233,43366105.94795357,65.83893533526702,103045.92562708561,1998947.246616053,13751745.095315164,314037255.60981447 -34,sqlOp_ObjectHashAggregate,0.0017800746,0.11217948717948718,0.31609411857312486,0.0,0.0,0.0,0.0,1.0 -35,sr_totalBytesRead_mean,0.0016529755,20031188.820523355,44488391.851112396,78.73076923076923,209696.66105015675,2247593.5513412068,14743721.973818542,314039311.06463194 -36,sr_totalBytesReadRatio,0.0012469853,0.9884387880699097,1.3285035445450863,5.157297693553143e-07,0.008813453447385317,0.41119710033966017,1.664552629012729,12.262128623858096 -37,sqlOp_RunningWindowFunction,0.0007885995,0.0673076923076923,0.2509566164678633,0.0,0.0,0.0,0.0,1.0 -38,sqlOp_AQEShuffleRead,0.0006590924,0.7692307692307693,0.42200187205134587,0.0,1.0,1.0,1.0,1.0 -39,sqlOp_BroadcastHashJoin,0.00056949805,0.9294871794871795,0.2564205620816261,0.0,1.0,1.0,1.0,1.0 -40,memoryBytesSpilledRatio,0.00028866128,0.3993333357850152,1.7678606626748894,0.0,0.0,0.0,0.0,13.887281849453222 -41,sqlOp_Subquery,0.00027296058,0.21474358974358973,0.411304017893851,0.0,0.0,0.0,0.0,1.0 -42,diskBytesSpilledRatio,0.00021575802,0.08881273658791966,0.3656079230955561,0.0,0.0,0.0,0.0,3.1632320652698986 -43,sqlOp_BroadcastNestedLoopJoin,0.00020984092,0.038461538461538464,0.19261662052660916,0.0,0.0,0.0,0.0,1.0 -44,diskBytesSpilled_mean,5.9101258e-05,7069993.416079486,31523322.991665952,0.0,0.0,0.0,0.0,374126772.9802514 -45,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -47,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -48,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_HashAggregate,0.0,0.9903846153846154,0.09774225976795364,0.0,1.0,1.0,1.0,1.0 -53,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -57,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_WindowGroupLimit,0.0,0.019230769230769232,0.13755578096385773,0.0,0.0,0.0,0.0,1.0 -62,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_SortMergeJoin,0.0,0.5833333333333334,0.49379862765412535,0.0,0.0,1.0,1.0,1.0 -65,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,memoryBytesSpilled_mean,0.0,26248908.39940863,107040263.16755028,0.0,0.0,0.0,0.0,724966162.8487283 -80,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -81,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -83,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -84,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -86,sqlOp_Sort,0.0,0.7435897435897436,0.43735238019288775,0.0,0.0,1.0,1.0,1.0 -87,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -89,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -90,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -92,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -93,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -94,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 -95,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sr_remoteBlocksFetched_sum,0.0,127050.61538461539,175988.1980301241,3.0,2757.25,61665.0,181167.5,1003606.0 -99,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -106,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -108,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -109,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -111,platform_databricks-aws,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -112,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -113,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,executorCPUTime_mean,0.28860697,2152.2279055967347,4768.119051696857,70.88447653429603,284.2316451435637,672.5167513645774,1630.6395926865093,61271.6827309237 +1,input_bytesRead_mean,0.09818622,32744094.213089865,39629389.010754265,157930.11165644173,4825140.119812995,15745821.867247352,49695222.62577245,178342334.9422222 +2,scan_bw,0.028898515,852348.8649740079,1256066.2869358,25664.476385554077,244592.89003173803,453737.53107269213,984211.7582137551,14656983.122448979 +3,resultSize_max,0.025798483,99931.8717948718,292860.48860016646,5158.0,8628.0,13086.5,22400.5,2030358.0 +4,jvmGCTime_mean,0.021706052,51.26736194273545,136.7085736361219,0.0,9.827855037169956,20.31146613289841,49.76661765200461,2002.3373493975903 +5,duration_max,0.016436096,13206.358974358975,28086.892904146353,268.0,2215.75,4042.5,9516.75,214099.0 +6,input_recordsRead_sum,0.016311798,6899154505.211538,16894449776.068132,600000.0,362916825.5,1646770012.0,6151825905.25,230401504404.0 +7,numTasks_sum,0.01528645,1183.2788461538462,1204.7034441026337,4.0,419.25,807.5,1624.25,11722.0 +8,executorDeserializeCPUTime_mean,0.011849796,6.809404488739633,3.052904823341863,2.46474358974359,5.713835885304287,6.525625811651963,7.75061394891945,37.37243401759531 +9,executorDeserializeTime_mean,0.01173742,10.25826375949487,13.985580724022466,2.33974358974359,6.198198926905551,7.915370390246956,10.337483478096676,167.07771260997066 +10,sqlOp_SubqueryBroadcast,0.011515717,0.7467948717948718,0.4355459850050311,0.0,0.0,1.0,1.0,1.0 +11,scan_time,0.0110211205,1400805.6185897435,5314404.8107528975,75.0,45099.75,213395.5,913930.75,71049280.0 +12,shuffle_write_bw,0.010796894,97597.46220339967,101684.36398868829,89.38862559241706,5279.766135993155,58133.42077217826,166003.50456482448,363609.36008282914 +13,duration_mean,0.010728625,2933.22180477384,5516.967032475489,144.4834355828221,597.3303784602791,1217.3476729944887,2645.471320224719,66753.3641231593 +14,cache_hit_ratio,0.008865399,0.6285631138115669,0.30548501366690567,1.187845066996456e-05,0.40065365426082494,0.7015237671310255,0.9068745980835918,1.0 +15,shuffle_read_bw,0.0080945045,581925830.5982302,6398696547.126323,0.0,955032.840866873,4561425.729699758,23080427.753544167,107487374955.0 +16,sr_remoteBytesRead_mean,0.0074725673,17015457.156458072,38838528.23309634,68.60897435897436,165447.60836938542,1539625.8261178578,11668314.370339088,274787384.05385995 +17,duration_sum,0.007466309,4829029.923076923,10325780.420673069,604.0,294242.0,1060120.5,3679291.0,68898339.0 +18,sr_localBytesRead_mean,0.0072316476,3015731.6640652833,6376318.262818829,0.0,41219.35598607367,386142.11036059354,2336170.0426780568,39251927.01077199 +19,peakExecutionMemory_max,0.0069952277,808883609.4615384,1738902255.3593981,0.0,98950280.0,216700784.0,679936172.0,13432783808.0 +20,sw_bytesWrittenRatio,0.006182557,0.8968440930204622,1.2304635979404275,5.157297693553143e-07,0.006672492716992155,0.36986430596766073,1.4866485306622774,12.23950555496033 +21,data_size,0.00501867,567279237847.8557,1789756116215.4756,57333908.0,21732999116.25,90621097739.5,337866257513.75,21680680059621.0 +22,executorRunTime_mean,0.0046263966,2913.1291808775654,5517.281917094229,114.91288343558283,573.4333323657474,1179.9774741118122,2628.774232209738,66732.86345381525 +23,sw_writeTime_mean,0.0043949042,118.89261641821152,322.16134532828437,0.23129251700680273,22.18550110537951,45.54013792957777,83.09396412465505,3121.572710951526 +24,sr_fetchWaitTime_mean,0.0043467274,2.191040291299454,9.923627614482589,0.0,0.00936196319018405,0.2188722371459541,1.442273795534665,164.9670855774985 +25,resultSerializationTime_sum,0.0037066948,19.634615384615383,94.37842595989993,0.0,0.0,1.0,17.0,1511.0 +26,sw_recordsWritten_sum,0.0036701674,2241520607.669872,5416061058.183992,264.0,1538364.5,136380867.5,1589319942.5,40948506792.0 +27,sr_remoteBytesReadRatio,0.0035212438,0.8157941158244444,1.1548408409516235,4.494264550895562e-07,0.0065696072316052765,0.28397315656730227,1.3175949154660336,10.726449601853131 +28,sqlOp_Window,0.003480682,0.0673076923076923,0.2509566164678633,0.0,0.0,0.0,0.0,1.0 +29,sr_localBytesReadRatio,0.0031731676,0.17264467224546515,0.2825463853863918,0.0,0.0015028110961492072,0.0691738614719501,0.23132092359082565,2.0961972120502077 +30,sqlOp_Expand,0.0026263234,0.10576923076923077,0.3080360743511337,0.0,0.0,0.0,0.0,1.0 +31,duration_min,0.0025240614,40.23717948717949,34.89232167158379,8.0,28.0,35.0,41.25,413.0 +32,sqlOp_ObjectHashAggregate,0.0020888746,0.11217948717948718,0.31609411857312486,0.0,0.0,0.0,0.0,1.0 +33,sqlOp_RunningWindowFunction,0.0020100132,0.0673076923076923,0.2509566164678633,0.0,0.0,0.0,0.0,1.0 +34,sw_bytesWritten_mean,0.0019917141,18910510.05056233,43366105.94795357,65.83893533526702,103045.92562708561,1998947.246616053,13751745.095315164,314037255.60981447 +35,sr_totalBytesRead_mean,0.0019821683,20031188.820523355,44488391.851112396,78.73076923076923,209696.66105015675,2247593.5513412068,14743721.973818542,314039311.06463194 +36,sr_localBlocksFetched_sum,0.0013500017,18225.53846153846,25193.523582463087,0.0,434.0,8839.5,25910.5,147330.0 +37,diskBytesSpilledRatio,0.00085794536,0.08881273658791965,0.3656079230955562,0.0,0.0,0.0,0.0,3.1632320652698986 +38,sr_totalBytesReadRatio,0.00073587405,0.9884387880699097,1.3285035445450863,5.157297693553143e-07,0.008813453447385317,0.41119710033966017,1.664552629012729,12.262128623858096 +39,sqlOp_AQEShuffleRead,0.0006215946,0.7692307692307693,0.42200187205134587,0.0,1.0,1.0,1.0,1.0 +40,sqlOp_BroadcastHashJoin,0.00022135998,0.9294871794871795,0.2564205620816261,0.0,1.0,1.0,1.0,1.0 +41,memoryBytesSpilledRatio,0.00020403732,0.3993333357850152,1.7678606626748896,0.0,0.0,0.0,0.0,13.887281849453222 +42,sr_remoteBlocksFetched_sum,0.0001505047,127050.61538461539,175988.1980301241,3.0,2757.25,61665.0,181167.5,1003606.0 +43,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +44,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +45,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +47,sqlOp_HashAggregate,0.0,0.9903846153846154,0.09774225976795364,0.0,1.0,1.0,1.0,1.0 +48,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +49,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +50,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +51,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,sqlOp_SortMergeJoin,0.0,0.5833333333333334,0.49379862765412535,0.0,0.0,1.0,1.0,1.0 +56,sqlOp_Subquery,0.0,0.21474358974358973,0.41130401789385107,0.0,0.0,0.0,0.0,1.0 +57,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +58,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +59,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +60,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +61,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +62,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 +63,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +65,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +66,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_Sort,0.0,0.7435897435897436,0.4373523801928878,0.0,0.0,1.0,1.0,1.0 +68,platform_databricks-aws,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +69,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_BroadcastNestedLoopJoin,0.0,0.038461538461538464,0.19261662052660916,0.0,0.0,0.0,0.0,1.0 +75,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,diskBytesSpilled_mean,0.0,7069993.416079485,31523322.991665952,0.0,0.0,0.0,0.0,374126772.9802514 +81,memoryBytesSpilled_mean,0.0,26248908.399408635,107040263.1675503,0.0,0.0,0.0,0.0,724966162.8487283 +82,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_WindowGroupLimit,0.0,0.019230769230769232,0.13755578096385773,0.0,0.0,0.0,0.0,1.0 114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg index 1dbc84ae3..a748c6db8 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0450923033","gamma":"0.00857468415","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0450923033","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"8","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.00857468415","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.774576485"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-4.1710395E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"73"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0488800742","gamma":"0.00704098027","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0488800742","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.00704098027","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.66913265"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-4.1710395E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json index 41f64da58..28654c253 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-9.146741E-3,5.5504434E-2,-3.9678478E-1,-1.2542758E-2,4.836776E-1,-2.2794344E-1,-5.1708266E-2,1.3460074E-2,-2.5411095E-2,4.703392E-2,2.0777218E-1,8.804978E-3,-3.7147656E-1,-7.1534336E-2,1.3590257E-1,1.808181E-2,1.32262595E-2,-2.199478E-2,-2.3713817E-3,1.8940192E-1,-1.1924882E-1,1.6652683E-2,2.6910236E-1,6.126256E-3,-5.0328514E-3,2.6899594E-1,5.371657E-4,-9.229226E-2,-2.0890705E-2,1.3701685E-2,-4.669401E-2,3.5692504E-1,-2.2527727E-3,1.5577266E-2,6.6604195E-3,-1.393904E-1,1.0775583E-1,2.2464486E-2,-9.067573E-3,2.0835906E-1,2.3127336E-2,-1.5900154E-3,-1.1224628E-2,9.2085935E-3,-3.9251978E-3,3.0340948E-3,-3.1558857E-3,1.080914E-2,5.47324E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,-1,17,19,21,-1,23,-1,-1,25,27,29,31,-1,-1,33,-1,35,-1,-1,37,39,-1,-1,-1,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.96154E0,5.949807E0,4.043698E0,2.543528E0,4.1046968E0,1.8341279E0,0E0,1.7788993E0,0E0,0E0,7.4651253E-1,0E0,7.9022336E-1,1.2800803E0,1.1074032E0,0E0,1.8356764E-1,0E0,0E0,2.2282904E-1,7.5481486E-1,7.055431E-1,9.425192E-1,0E0,0E0,5.0982118E-2,0E0,7.781935E-1,0E0,0E0,3.462049E-1,4.9557853E-1,0E0,0E0,0E0,7.4400496E-1,3.1843132E-1,1.0001786E-1,0E0,1.18098855E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,12,12,13,13,14,14,16,16,19,19,20,20,21,21,22,22,25,25,27,27,30,30,31,31,35,35,36,36,37,37,39,39],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,-1,18,20,22,-1,24,-1,-1,26,28,30,32,-1,-1,34,-1,36,-1,-1,38,40,-1,-1,-1,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5965E5,5.246E3,8.756827E-1,4.966421E6,4.083812E7,1.9024893E8,-5.1708266E-2,2.1649122E0,-2.5411095E-2,4.703392E-2,9E0,8.804978E-3,1.1698093E7,1.8130071E-1,7.6696295E-1,1.808181E-2,7.3240237E8,-2.199478E-2,-2.3713817E-3,1.7930752E10,7.804584E7,2.94E2,2.7395833E0,6.126256E-3,-5.0328514E-3,2.9164E4,5.371657E-4,2.2870504E1,-2.0890705E-2,1.3701685E-2,7.4034056E5,4.0100845E2,-2.2527727E-3,1.5577266E-2,6.6604195E-3,6.4865E4,3.7017515E6,1.4697656E6,-9.067573E-3,2.1845032E7,2.3127336E-2,-1.5900154E-3,-1.1224628E-2,9.2085935E-3,-3.9251978E-3,3.0340948E-3,-3.1558857E-3,1.080914E-2,5.47324E-3],"split_indices":[9,2,26,27,44,7,0,52,0,0,3,0,49,26,26,0,7,0,0,30,44,0,53,0,0,28,0,57,0,0,27,4,0,0,0,11,46,31,0,44,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2.03E2,3.3E1,1.76E2,2.7E1,2.8E1,5E0,1.69E2,7E0,8E0,1.9E1,7E0,2.1E1,1E2,6.9E1,9E0,1E1,1.5E1,6E0,1.5E1,8.5E1,3.7E1,3.2E1,5E0,5E0,1E1,5E0,8E1,5E0,6E0,3.1E1,2.5E1,7E0,5E0,5E0,6.5E1,1.5E1,2.2E1,9E0,1.4E1,1.1E1,3.4E1,3.1E1,1E1,5E0,1.5E1,7E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[3.420386E-3,-4.427893E-2,3.6722183E-1,6.1322926E-3,-3.4022918E-1,8.2723105E-1,4.7967497E-2,-6.621938E-2,1.5571968E-1,-1.0516482E-1,-3.6066692E-2,1.9575827E-2,4.881072E-2,1.7727974E-1,-7.30901E-3,3.984273E-2,-2.056822E-1,3.6347535E-1,7.479386E-2,-2.9202357E-1,5.843159E-3,1.2745788E-2,1.90979E-3,-2.4377154E-1,1.1003482E-1,-2.9188657E-1,1.4216012E-1,6.1366605E-3,2.0492645E-2,1.685999E-1,-1.11550756E-1,-2.2963744E-2,-2.890533E-3,-4.8202956E-3,-1.8418072E-2,2.0385817E-1,-6.9040105E-2,-3.985426E-1,-1.5463778E-1,9.49975E-3,2.2525159E-3,2.7679864E-1,1.4140054E-2,1.824595E-3,-9.802597E-3,3.167029E-1,5.2470043E-2,1.7436784E-2,-9.060623E-3,-1.1056772E-2,-2.7498458E-2,-1.7620891E-2,-1.3071924E-2,5.3371997E-3,3.3295056E-1,6.4069265E-3,-5.2227983E-3,1.8502489E-2,7.786826E-3,8.366043E-3,-5.429814E-3,-2.541485E-3,5.0922204E-3,2.2221776E-3,-3.678883E-3,9.217101E-3,1.7837068E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,23,25,27,29,31,-1,-1,-1,33,35,37,39,-1,-1,41,43,-1,-1,-1,-1,45,47,49,51,-1,-1,53,55,-1,-1,57,59,61,-1,-1,-1,63,-1,-1,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.042175E0,3.0754445E0,3.925538E0,1.9259647E0,3.1570392E0,6.3372755E-1,4.9050626E-1,1.7832015E0,9.4795656E-1,9.5899624E-1,0E0,0E0,0E0,1.4443335E-1,0E0,1.4004395E0,1.6007421E0,2.6916742E-1,7.704583E-1,5.615102E-1,0E0,0E0,0E0,2.7190298E-1,9.549532E-1,5.5318356E-1,5.896738E-2,0E0,0E0,4.8074436E-1,2.501316E-1,0E0,0E0,0E0,0E0,6.154666E-1,2.3162091E-1,5.956247E-1,3.6553574E-1,0E0,0E0,1.1183262E-1,2.3262458E-1,0E0,0E0,2.1228766E-1,4.1341585E-1,9.797821E-2,0E0,0E0,0E0,5.1093303E-2,0E0,0E0,1.5734076E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,18,18,19,19,23,23,24,24,25,25,26,26,29,29,30,30,35,35,36,36,37,37,38,38,41,41,42,42,45,45,46,46,47,47,51,51,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,24,26,28,30,32,-1,-1,-1,34,36,38,40,-1,-1,42,44,-1,-1,-1,-1,46,48,50,52,-1,-1,54,56,-1,-1,58,60,62,-1,-1,-1,64,-1,-1,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,6.5023036E7,5.25191E5,1.9851093E2,3.0711978E5,1.4753E4,8.294758E-1,5.58E2,1.02848E5,6.633555E2,-3.6066692E-2,1.9575827E-2,4.881072E-2,2.2762294E0,-7.30901E-3,5.64E2,2.3900106E0,3.1565277E2,4.213523E0,5.580861E4,5.843159E-3,1.2745788E-2,1.90979E-3,1.8242699E8,5.606407E5,1.06199684E2,1.9760254E-1,6.1366605E-3,2.0492645E-2,3.619228E-2,6.78E2,-2.2963744E-2,-2.890533E-3,-4.8202956E-3,-1.8418072E-2,8.2147594E5,5.696862E8,2.5110843E2,1.059E3,9.49975E-3,2.2525159E-3,6.6704494E5,1E0,1.824595E-3,-9.802597E-3,2.71E2,1.2694E4,1.5789307E8,-9.060623E-3,-1.1056772E-2,-2.7498458E-2,2.47173E5,-1.3071924E-2,5.3371997E-3,3.61791E2,6.4069265E-3,-5.2227983E-3,1.8502489E-2,7.786826E-3,8.366043E-3,-5.429814E-3,-2.541485E-3,5.0922204E-3,2.2221776E-3,-3.678883E-3,9.217101E-3,1.7837068E-2],"split_indices":[2,44,28,51,32,9,26,0,28,51,0,0,0,52,0,2,52,54,55,32,0,0,0,7,27,51,38,0,0,37,0,0,0,0,0,47,5,4,2,0,0,27,80,0,0,0,9,5,0,0,0,1,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,2.05E2,2.6E1,1.76E2,2.9E1,1E1,1.6E1,1.19E2,5.7E1,2E1,9E0,5E0,5E0,1E1,6E0,6.8E1,5.1E1,1.5E1,4.2E1,1.1E1,9E0,5E0,5E0,1.3E1,5.5E1,4.1E1,1E1,5E0,1E1,2.8E1,1.4E1,5E0,6E0,8E0,5E0,3.6E1,1.9E1,2.2E1,1.9E1,5E0,5E0,1.6E1,1.2E1,6E0,8E0,2E1,1.6E1,1.2E1,7E0,1.4E1,8E0,1E1,9E0,5E0,1.1E1,6E0,6E0,1.1E1,9E0,9E0,7E0,7E0,5E0,5E0,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[1.7136386E-2,-3.6134012E-2,3.5399628E-1,-1.2527469E-2,-3.2266326E-2,4.0958006E-2,1.0828039E-1,1.0329245E-1,-9.683243E-2,3.3645758E-1,-7.8736015E-2,1.4251041E-1,-1.9992374E-2,-1.5262055E-1,6.4657465E-2,7.502324E-3,2.0312363E-2,-1.3737618E-2,3.6355862E-3,7.591972E-2,3.4181607E-1,-4.5995528E-1,-9.321646E-2,-6.5103665E-2,2.0740959E-1,1.3175325E-1,-1.2492049E-1,3.9013994E-1,7.3968056E-3,-2.8336741E-2,-9.096038E-3,-1.7584637E-2,-5.1962778E-2,-1.2624022E-1,3.0671814E-3,3.5092183E-3,1.7532744E-2,-5.5823443E-3,1.8463641E-1,1.7539627E-3,-1.3270433E-2,9.163098E-3,2.0890605E-2,7.067853E-2,-1.4182521E-1,-7.612881E-3,-3.2331808E-3,5.78584E-3,1.6713355E-2,-2.8458666E-3,9.007512E-3,-8.177828E-3,4.7720564E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,25,27,29,31,33,35,37,39,41,-1,-1,-1,-1,43,45,-1,-1,-1,-1,47,-1,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.337454E0,3.3431742E0,4.396408E0,1.9921333E0,0E0,0E0,1.0506405E0,1.9069225E0,1.0729557E0,1.383425E-1,5.2860296E-1,1.0490735E0,0E0,1.5637352E0,5.877371E-1,0E0,0E0,0E0,0E0,7.1170306E-1,1.0782218E-1,4.8227072E-1,8.978987E-1,1.4694522E-1,3.2236427E-1,6.8506145E-1,3.9989847E-1,1.1095548E-1,0E0,0E0,0E0,0E0,7.484208E-1,1.576753E-2,0E0,0E0,0E0,0E0,3.820051E-1,0E0,0E0,0E0,0E0,5.134227E-1,4.0101546E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,32,32,33,33,38,38,43,43,44,44],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,26,28,30,32,34,36,38,40,42,-1,-1,-1,-1,44,46,-1,-1,-1,-1,48,-1,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.3575526E8,4.083812E7,2.5542571E8,-3.2266326E-2,4.0958006E-2,1E1,3.7578388E6,2.3641207E0,2.4010162E5,2.1374558E3,2.3082722E2,-1.9992374E-2,8.8109635E-2,1.2106E4,7.502324E-3,2.0312363E-2,-1.3737618E-2,3.6355862E-3,1.02848E5,1.479E3,1.5566E4,5.6380646E1,2.9450394E5,7.375037E5,1.4025452E0,2.8367348E0,3.63E2,7.3968056E-3,-2.8336741E-2,-9.096038E-3,-1.7584637E-2,9.14E2,8E0,3.0671814E-3,3.5092183E-3,1.7532744E-2,-5.5823443E-3,1.136E3,1.7539627E-3,-1.3270433E-2,9.163098E-3,2.0890605E-2,2.8374527E2,2.4646638E6,-7.612881E-3,-3.2331808E-3,5.78584E-3,1.6713355E-2,-2.8458666E-3,9.007512E-3,-8.177828E-3,4.7720564E-3],"split_indices":[2,44,44,7,0,0,3,27,52,32,4,51,0,40,11,0,0,0,0,28,0,9,51,32,46,52,55,0,0,0,0,0,0,3,0,0,0,0,2,0,0,0,0,4,27,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,2.08E2,3.2E1,2.02E2,6E0,9E0,2.3E1,8.5E1,1.17E2,1E1,1.3E1,8E1,5E0,8.7E1,3E1,5E0,5E0,5E0,8E0,6.1E1,1.9E1,1.3E1,7.4E1,1.6E1,1.4E1,4.8E1,1.3E1,1.4E1,5E0,7E0,6E0,8E0,6.6E1,1.1E1,5E0,9E0,5E0,8E0,4E1,7E0,6E0,5E0,9E0,2.8E1,3.8E1,5E0,6E0,3.2E1,8E0,1.4E1,1.4E1,3.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.4562023E-2,-4.195406E-2,3.5639378E-1,5.5492617E-2,-1.5749606E-1,8.3843744E-1,1.10757075E-1,9.352175E-2,-1.6610973E-2,-8.927811E-2,-6.07164E-1,1.5566784E-2,5.146917E-2,1.8370828E-1,-6.803217E-3,-1.3313587E-2,1.1878996E-1,-2.2735979E-1,6.197708E-3,-1.0032668E-2,-3.696544E-2,2.8900158E-1,2.2327371E-3,2.2751452E-1,6.950262E-2,4.079556E-3,-3.071973E-1,1.4859755E-1,-4.8730377E-2,1.6227687E-2,7.6638507E-3,2.9843885E-1,-5.8423406E-3,1.9468985E-1,2.448984E-4,8.842548E-4,-3.7609923E-1,5.055878E-4,1.20580485E-2,-1.5189368E-1,5.6869436E-2,5.1648435E-3,3.455273E-1,3.075222E-1,1.2870967E-2,-9.875245E-2,1.2173051E-1,-4.5592415E-1,-8.675733E-3,-1.2172739E-2,-9.454426E-2,-3.5897493E-3,1.7577071E-1,8.631941E-3,1.7212588E-2,1.7944004E-2,9.626213E-3,-6.737057E-3,7.80109E-3,-1.1215366E-2,1.8462873E-4,1.2316052E-3,1.27857905E-2,-2.5440289E-2,-1.5085671E-2,-2.0960844E-3,-7.164573E-3,4.2942353E-3,1.0236594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,-1,23,25,27,-1,-1,29,-1,31,33,-1,35,37,39,-1,-1,41,-1,43,45,-1,47,-1,-1,49,51,-1,53,55,57,59,61,63,-1,-1,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.902657E0,2.463242E0,4.169811E0,1.9526596E0,3.0133364E0,1.3991823E0,5.048726E-1,1.1156704E0,0E0,1.1615537E0,8.808079E-1,0E0,0E0,2.6828367E-1,0E0,0E0,5.4149663E-1,9.4128656E-1,4.2179397E-1,0E0,0E0,3.1644642E-2,0E0,8.490846E-1,6.328756E-1,0E0,6.6038704E-1,2.4184409E-1,4.3350503E-1,0E0,0E0,1.9423127E-1,0E0,5.2943325E-1,5.8930534E-1,0E0,2.6421213E-1,0E0,0E0,1.0988864E-1,3.3854192E-1,0E0,4.4047356E-2,5.1100254E-2,3.116901E-1,4.174587E-1,3.2837856E-1,2.148676E-2,0E0,0E0,3.8998768E-2,0E0,2.3778856E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,16,16,17,17,18,18,21,21,23,23,24,24,26,26,27,27,28,28,31,31,33,33,34,34,36,36,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,50,50,52,52],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,-1,24,26,28,-1,-1,30,-1,32,34,-1,36,38,40,-1,-1,42,-1,44,46,-1,48,-1,-1,50,52,-1,54,56,58,60,62,64,-1,-1,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,3.5540915E0,4.083812E7,3.7578388E6,5.33233E5,1.4781E4,2.1661E4,1.736E3,-1.6610973E-2,9.130081E0,3.6550702E2,1.5566784E-2,5.146917E-2,3.5481934E1,-6.803217E-3,-1.3313587E-2,1.5316E4,2.0218372E0,1.3261E4,-1.0032668E-2,-3.696544E-2,2.035294E0,2.2327371E-3,1E1,5.606407E5,4.079556E-3,2.0304577E-1,3.0954462E2,7.330627E5,1.6227687E-2,7.6638507E-3,7.24E2,-5.8423406E-3,7.251407E6,8.9341575E-1,8.842548E-4,6.124014E5,5.055878E-4,1.20580485E-2,1E0,5.932798E6,5.1648435E-3,2.758609E2,6.748347E5,9.402985E-1,1.2399164E5,2.49328E5,8.3E2,-8.675733E-3,-1.2172739E-2,1.9123037E0,-3.5897493E-3,1.9944866E0,8.631941E-3,1.7212588E-2,1.7944004E-2,9.626213E-3,-6.737057E-3,7.80109E-3,-1.1215366E-2,1.8462873E-4,1.2316052E-3,1.27857905E-2,-2.5440289E-2,-1.5085671E-2,-2.0960844E-3,-7.164573E-3,4.2942353E-3,1.0236594E-2],"split_indices":[2,56,44,27,9,9,9,12,0,56,4,0,0,56,0,0,9,57,10,0,0,53,0,3,27,0,26,4,27,0,0,2,0,12,26,0,27,0,0,8,50,0,4,31,55,32,1,0,0,0,53,0,52,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.17E2,3.5E1,1.18E2,9.9E1,1.1E1,2.4E1,1.09E2,9E0,8.7E1,1.2E1,5E0,6E0,1.9E1,5E0,6E0,1.03E2,3.5E1,5.2E1,5E0,7E0,1E1,9E0,3.1E1,7.2E1,7E0,2.8E1,1.4E1,3.8E1,5E0,5E0,2.6E1,5E0,2.5E1,4.7E1,5E0,2.3E1,7E0,7E0,1.9E1,1.9E1,6E0,2E1,1.5E1,1E1,2.6E1,2.1E1,1.5E1,8E0,5E0,1.4E1,9E0,1E1,5E0,1.5E1,6E0,9E0,5E0,5E0,1E1,1.6E1,1.4E1,7E0,6E0,9E0,9E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[1.2451561E-2,-3.4080334E-2,3.3931082E-1,1.2782627E-2,-2.7009928E-1,3.5760056E-2,1.0709108E-1,-1.7679188E-2,3.4267012E-2,-5.3319346E-2,-6.328562E-1,1.33418795E-2,-3.184624E-2,1.5391849E-1,-3.8298666E-2,-2.0221601E-1,1.000206E-1,-3.7107483E-2,-1.2353401E-2,-7.712321E-3,5.0454265E-3,1.02074176E-1,3.030718E-1,3.4286954E-2,-1.8038979E-1,-1.136262E-2,-4.980382E-3,-2.0114433E-3,1.1367002E-2,1.4314554E-2,5.8619864E-2,2.8057767E-3,1.8347694E-2,-2.2424337E-1,1.0171972E-1,-2.863859E-1,-7.047823E-2,-3.5089068E-2,1.1197998E-1,-2.6248507E-3,-1.35925645E-2,2.871915E-1,2.2335814E-2,-3.884657E-3,-3.7932524E-1,3.6191947E-3,-1.462246E-1,-5.129402E-3,5.184578E-3,7.095768E-3,-6.97576E-4,4.9632513E-3,1.5864389E-2,9.163734E-3,-1.2478634E-3,-2.2276899E-2,-8.219583E-3,-1.1007939E-2,-2.458652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,-1,19,21,23,25,27,-1,-1,-1,-1,29,31,33,35,-1,-1,-1,-1,-1,37,-1,-1,39,41,43,45,47,49,-1,-1,51,53,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6618154E0,2.337228E0,3.0760276E0,1.55297E0,2.7186015E0,0E0,5.6024694E-1,0E0,1.4732528E0,5.4535735E-1,6.6134214E-1,0E0,2.7922153E-1,4.6396017E-1,1.0994449E0,2.6975423E-2,2.7515417E-1,0E0,0E0,0E0,0E0,4.399827E-1,3.7476122E-1,1.2605824E0,3.992231E-1,0E0,0E0,0E0,0E0,0E0,2.1394269E-1,0E0,0E0,1.747073E-1,8.328235E-1,3.022995E-1,2.2867863E-1,2.0195653E-1,1.5859705E-1,0E0,0E0,1.5587866E-1,3.7650922E-1,0E0,1.8116343E-1,0E0,1.0339028E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,21,21,22,22,23,23,24,24,30,30,33,33,34,34,35,35,36,36,37,37,38,38,41,41,42,42,44,44,46,46],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,-1,20,22,24,26,28,-1,-1,-1,-1,30,32,34,36,-1,-1,-1,-1,-1,38,-1,-1,40,42,44,46,48,50,-1,-1,52,54,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,5.0826204E7,4.083812E7,4.26E2,1E0,3.5760056E-2,3.5481934E1,-1.7679188E-2,1.3582685E7,5.289323E2,2.0788991E0,1.33418795E-2,1.4638443E2,4.0972165E2,2.2448614E0,1E0,2.3772728E0,-3.7107483E-2,-1.2353401E-2,-7.712321E-3,5.0454265E-3,1.5681747E9,2.6569772E-1,8.92E2,3.677233E5,-1.136262E-2,-4.980382E-3,-2.0114433E-3,1.1367002E-2,1.4314554E-2,2.1282723E0,2.8057767E-3,1.8347694E-2,1.8939584E8,1.3255E4,2.9166096E-1,2.5319636E0,4.2905E4,2.430046E0,-2.6248507E-3,-1.35925645E-2,5.1264524E-1,6.4675875E5,-3.884657E-3,2.7573213E8,3.6191947E-3,3.2033653E5,-5.129402E-3,5.184578E-3,7.095768E-3,-6.97576E-4,4.9632513E-3,1.5864389E-2,9.163734E-3,-1.2478634E-3,-2.2276899E-2,-8.219583E-3,-1.1007939E-2,-2.458652E-3],"split_indices":[2,44,44,2,108,0,56,0,12,51,53,0,56,4,55,63,52,0,0,0,0,30,26,2,27,0,0,0,0,0,52,0,0,12,9,26,56,28,52,0,0,55,47,0,7,0,32,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.39E2,2.1E2,2.9E1,1.76E2,3.4E1,9E0,2E1,8E0,1.68E2,2.2E1,1.2E1,8E0,1.2E1,6.3E1,1.05E2,1.1E1,1.1E1,7E0,5E0,6E0,6E0,4.8E1,1.5E1,7E1,3.5E1,6E0,5E0,6E0,5E0,7E0,4.1E1,5E0,1E1,1.4E1,5.6E1,1.7E1,1.8E1,1.5E1,2.6E1,5E0,9E0,1.6E1,4E1,6E0,1.1E1,6E0,1.2E1,1E1,5E0,1.9E1,7E0,5E0,1.1E1,8E0,3.2E1,6E0,5E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.0679025E-3,-5.3443313E-2,3.0539253E-1,5.4728355E-2,-1.771571E-1,7.7343667E-1,6.851565E-2,8.920684E-2,-2.0539127E-2,-1.1292429E-1,-3.9506528E-1,1.3638078E-2,4.809772E-2,2.606026E-1,-7.6915815E-2,-9.9122755E-2,1.363497E-1,-2.2062394E-1,3.5326004E-2,-6.4500153E-1,-1.09214015E-1,1.5085313E-2,6.458501E-3,4.4206297E-3,-1.3115762E-2,-2.666869E-1,7.745253E-2,1.6674829E-1,-4.4729295E-3,-1.552655E-1,-2.41051E-2,-8.819784E-2,1.2588069E-2,-1.809604E-2,-3.5225738E-2,-9.899266E-3,8.706279E-4,-5.979701E-3,-1.6067097E-2,8.2757035E-3,-1.8727626E-3,1.8815629E-1,-4.6745893E-3,-7.287307E-2,-2.8541926E-1,4.4494655E-2,-2.0248282E-1,1.4505601E-1,3.7720293E-1,5.9311586E-3,-1.3358717E-1,-1.6404336E-2,-5.4239896E-3,5.3976704E-3,-1.7193321E-3,-1.2498239E-2,-3.6795535E-3,8.858546E-3,1.4393966E-3,2.0307079E-2,1.0326307E-2,-1.11869965E-2,-2.079611E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,25,27,29,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,47,-1,49,51,53,55,57,59,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9313502E0,2.6995435E0,3.938285E0,1.9405234E0,1.2710917E0,1.3345971E0,7.184316E-1,9.191117E-1,0E0,1.1893153E0,1.429893E0,0E0,0E0,4.7549486E-2,5.8997345E-1,6.4199686E-1,6.0367906E-1,7.9932404E-1,9.884484E-1,5.156994E-2,1.6020145E-1,0E0,0E0,0E0,0E0,8.493745E-2,1.464568E-1,4.421065E-1,0E0,3.7597698E-1,0E0,3.426341E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.9934602E-1,0E0,3.1542015E-1,1.3743341E-1,7.2917916E-2,8.572331E-2,3.2517624E-1,2.6943922E-2,0E0,1.7885175E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,25,25,26,26,27,27,29,29,31,31,41,41,43,43,44,44,45,45,46,46,47,47,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,26,28,30,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,48,-1,50,52,54,56,58,60,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.013E3,7.07E2,4.083812E7,3.26989E6,2.5965E5,1.4781E4,1E1,2.377416E6,-2.0539127E-2,3.1236285E2,5.5157606E-2,1.3638078E-2,4.809772E-2,3.5481934E1,2.01E3,2.6727284E5,2.46604E5,4.3625324E7,1.1779541E6,6.3171947E-1,9E0,1.5085313E-2,6.458501E-3,4.4206297E-3,-1.3115762E-2,3.09E2,1.8267011E5,9.770401E0,-4.4729295E-3,1.321E3,-2.41051E-2,8.581225E2,1.2588069E-2,-1.809604E-2,-3.5225738E-2,-9.899266E-3,8.706279E-4,-5.979701E-3,-1.6067097E-2,8.2757035E-3,-1.8727626E-3,1.496E3,-4.6745893E-3,5.965282E5,1.993E3,1.794E1,6.526919E0,1.4559748E0,2.9807162E0,5.9311586E-3,9.7846725E1,-1.6404336E-2,-5.4239896E-3,5.3976704E-3,-1.7193321E-3,-1.2498239E-2,-3.6795535E-3,8.858546E-3,1.4393966E-3,2.0307079E-2,1.0326307E-2,-1.11869965E-2,-2.079611E-3],"split_indices":[2,0,44,27,9,9,3,44,0,51,37,0,0,56,0,32,28,44,27,26,3,0,0,0,0,10,46,56,0,2,0,4,0,0,0,0,0,0,0,0,0,2,0,50,2,56,55,55,55,0,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,2E2,3.5E1,1.07E2,9.3E1,1.1E1,2.4E1,1.01E2,6E0,7.3E1,2E1,5E0,6E0,1E1,1.4E1,2E1,8.1E1,4.2E1,3.1E1,1E1,1E1,5E0,5E0,8E0,6E0,1E1,1E1,7.2E1,9E0,3.6E1,6E0,2.1E1,1E1,5E0,5E0,5E0,5E0,5E0,5E0,5E0,5E0,6.7E1,5E0,2.3E1,1.3E1,1E1,1.1E1,5.6E1,1.1E1,5E0,1.8E1,8E0,5E0,5E0,5E0,6E0,5E0,3.8E1,1.8E1,6E0,5E0,7E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.03896195E-2,-3.566386E-2,2.8772947E-1,-1.6775822E-2,-2.6321225E-2,3.5430714E-2,1.11070156E-1,4.090312E-2,-1.8727162E-1,2.768882E-1,-6.2681474E-2,1.4567497E-1,-4.8140973E-2,-3.1046E-1,9.361715E-3,6.6168853E-3,1.9207569E-2,-1.09600555E-2,2.9100005E-3,-8.484298E-3,1.9105253E-1,1.5536518E-1,-1.0599667E-1,-4.780735E-1,-1.1296693E-1,1.1650843E-1,-6.6035856E-3,1.4753772E-1,1.8349284E-2,4.148842E-2,1.5541312E-2,-2.467188E-1,-5.6031216E-2,-2.80134E-1,-3.2503664E-2,4.4069514E-3,-2.1762379E-1,4.6474364E-4,1.0763214E-2,2.279624E-1,8.051418E-2,-6.3529634E-3,7.804786E-3,-2.8074333E-3,-3.1838843E-1,8.000011E-2,-1.0109374E-1,-1.6374841E-2,-6.7836186E-3,-4.941098E-3,-1.3049692E-2,1.1736842E-2,3.950735E-3,-5.0775317E-4,1.0225227E-2,-8.239085E-3,-1.910147E-2,6.13334E-3,1.2620027E-3,-1.19304145E-2,-2.509083E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,27,29,31,33,35,37,-1,39,-1,41,-1,43,45,47,-1,-1,49,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1395156E0,2.1744127E0,2.9919496E0,2.0236337E0,0E0,0E0,7.95257E-1,1.4437274E0,1.2624997E0,2.091142E-1,3.3694038E-1,1.1114794E0,1.0054834E0,1.0052738E0,3.6764207E-1,0E0,0E0,0E0,0E0,0E0,5.3176904E-1,3.952605E-1,4.502383E-1,6.1492443E-1,3.7408578E-1,1.662309E-1,0E0,2.727245E-1,0E0,3.3638617E-1,0E0,2.0491922E-1,3.1458324E-1,6.379098E-2,0E0,0E0,5.359453E-2,0E0,0E0,8.606851E-2,4.1863015E-1,0E0,0E0,0E0,9.390521E-2,3.4070462E-2,2.6763403E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,31,31,32,32,33,33,36,36,39,39,40,40,44,44,45,45,46,46],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,28,30,32,34,36,38,-1,40,-1,42,-1,44,46,48,-1,-1,50,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.774E3,4.966421E6,2.13281E5,3.00037E5,-2.6321225E-2,3.5430714E-2,1.1E1,1.3582685E7,2.225675E0,2.6823762E5,1.99607E3,9.23253E1,1.3257E4,3.49917E2,1.2E1,6.6168853E-3,1.9207569E-2,-1.09600555E-2,2.9100005E-3,-8.484298E-3,1.496E3,7.2123044E5,3.5280538E5,7.01684E5,6.567405E2,3.6141206E8,-6.6035856E-3,5.606407E5,1.8349284E-2,1.2262E4,1.5541312E-2,2.4294901E-1,9.1565394E5,6.435986E0,-3.2503664E-2,4.4069514E-3,7.45226E5,4.6474364E-4,1.0763214E-2,3.1808215E10,1.6971204E2,-6.3529634E-3,7.804786E-3,-2.8074333E-3,1E0,8.3735475E5,1.573075E5,-1.6374841E-2,-6.7836186E-3,-4.941098E-3,-1.3049692E-2,1.1736842E-2,3.950735E-3,-5.0775317E-4,1.0225227E-2,-8.239085E-3,-1.910147E-2,6.13334E-3,1.2620027E-3,-1.19304145E-2,-2.509083E-3],"split_indices":[2,27,28,28,0,0,3,12,53,32,4,4,9,51,3,0,0,0,0,0,2,46,27,27,4,12,0,27,0,9,0,26,47,56,0,0,28,0,0,30,51,0,0,0,80,31,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,2.1E2,3.4E1,2.04E2,6E0,8E0,2.6E1,1.53E2,5.1E1,1.3E1,1.3E1,7E1,8.3E1,3.1E1,2E1,8E0,5E0,5E0,8E0,8E0,6.2E1,1.8E1,6.5E1,1.6E1,1.5E1,1.2E1,8E0,5.3E1,9E0,1.2E1,6E0,1.6E1,4.9E1,1E1,6E0,5E0,1E1,7E0,5E0,2.3E1,3E1,5E0,7E0,5E0,1.1E1,1.2E1,3.7E1,5E0,5E0,5E0,5E0,1.8E1,5E0,1.9E1,1.1E1,6E0,5E0,5E0,7E0,7E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[6.627681E-3,-1.6370922E-2,4.9526817E-1,2.7064756E-2,-2.4836802E-1,3.204374E-2,8.899696E-3,-2.068062E-1,5.3044386E-2,7.962529E-2,-4.514781E-1,-1.92861E-2,-5.496172E-2,1.3602394E-1,-5.371485E-2,-7.925875E-3,1.0141256E-2,-3.1292412E-1,-3.5708312E-2,3.8182107E-3,-8.420861E-3,2.6129324E-2,2.7220103E-1,-2.8459153E-1,2.937202E-2,-1.6530083E-2,-9.019691E-3,5.927106E-2,-1.19699305E-2,2.1324632E-1,2.7951175E-2,-3.7574807E-1,2.671477E-4,-7.134973E-2,2.1502803E-1,1.3780536E-1,-3.1058202E-2,1.0436083E-3,2.600487E-1,-2.0447334E-2,-9.517169E-3,1.4315677E-2,-2.4378176E-1,3.050837E-3,2.694657E-1,8.8750925E-3,3.3159752E-3,-5.2069887E-3,3.1571412E-3,1.4732783E-2,7.336541E-3,3.4491913E-3,-4.4656214E-3,-1.5344217E-2,-3.3585187E-3,8.487387E-3,1.6231429E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,-1,19,21,23,-1,-1,25,-1,-1,-1,27,29,31,33,-1,-1,35,-1,37,-1,39,-1,41,43,45,47,-1,49,-1,-1,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.750192E0,2.3647287E0,5.654557E-1,1.2139945E0,2.5008428E0,0E0,0E0,6.4733106E-1,1.5930378E0,5.9606767E-1,8.370137E-1,0E0,2.5504088E-1,1.5043659E0,1.5238018E0,0E0,0E0,3.5716414E-2,0E0,0E0,0E0,5.665576E-1,7.9015756E-1,5.58357E-1,1.1186185E0,0E0,0E0,3.731652E-1,0E0,3.4987152E-1,0E0,1.09053135E-1,0E0,5.795768E-1,1.5024114E-1,9.17241E-2,2.2138949E-1,0E0,1.5454769E-1,0E0,0E0,1.9785973E-1,1.8704945E-1,0E0,4.252827E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,21,21,22,22,23,23,24,24,27,27,29,29,31,31,33,33,34,34,35,35,36,36,38,38,41,41,42,42,44,44],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,-1,20,22,24,-1,-1,26,-1,-1,-1,28,30,32,34,-1,-1,36,-1,38,-1,40,-1,42,44,46,48,-1,50,-1,-1,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,7.8337E4,4.083812E7,2.3135895E6,3.4345872E0,3.204374E-2,8.899696E-3,1.340973E-1,2.0223E5,3.328384E5,2.1897722E10,-1.92861E-2,9.462E3,3.189603E2,2.8598264E2,-7.925875E-3,1.0141256E-2,2.5831E4,-3.5708312E-2,3.8182107E-3,-8.420861E-3,4.476758E6,2.798E3,2.148789E0,1.0083192E6,-1.6530083E-2,-9.019691E-3,3.58E2,-1.19699305E-2,2.6569772E-1,2.7951175E-2,2.7573213E8,2.671477E-4,5.941498E11,9.62589E5,6.3080795E6,8.15E1,1.0436083E-3,2.6786362E8,-2.0447334E-2,-9.517169E-3,3.5293162E5,6.3161764E0,3.050837E-3,2.7932866E5,8.8750925E-3,3.3159752E-3,-5.2069887E-3,3.1571412E-3,1.4732783E-2,7.336541E-3,3.4491913E-3,-4.4656214E-3,-1.5344217E-2,-3.3585187E-3,8.487387E-3,1.6231429E-2],"split_indices":[2,9,44,44,56,0,0,37,28,32,5,0,28,4,51,0,0,10,0,0,0,27,2,53,27,0,0,10,0,26,0,7,0,30,1,44,51,0,7,0,0,32,55,0,32,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.33E2,1E1,1.97E2,3.6E1,5E0,5E0,1.9E1,1.78E2,1.4E1,2.2E1,7E0,1.2E1,1E2,7.8E1,5E0,9E0,1.7E1,5E0,6E0,6E0,5.6E1,4.4E1,2E1,5.8E1,1E1,7E0,5.1E1,5E0,3.9E1,5E0,1.5E1,5E0,3.8E1,2E1,2.7E1,2.4E1,8E0,3.1E1,9E0,6E0,2.6E1,1.2E1,6E0,1.4E1,1.3E1,1.4E1,1.3E1,1.1E1,1.7E1,1.4E1,1.7E1,9E0,7E0,5E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-2.4749802E-3,6.028506E-2,-1.9273971E-1,2.416087E-2,2.6584613E-1,-3.9913288E-1,2.6528727E-2,1.0921541E-1,-5.3172786E-2,5.146048E-1,7.461377E-2,-2.5779906E-1,-4.367699E-2,-4.5589685E-2,9.3739545E-3,-3.8185623E-4,1.8308274E-1,-2.578282E-1,-1.2640161E-2,1.0061208E-2,3.467136E-2,-1.1956031E-2,1.0611019E-2,9.1901724E-4,-3.2194814E-1,8.1162885E-2,-1.4933495E-1,6.2441017E-2,-1.259929E-1,2.2891174E-1,-7.2416407E-3,-2.220043E-2,-3.2847081E-3,1.1766189E-1,-1.0851453E-1,3.174673E-3,-4.7820355E-3,-1.7059918E-2,-4.8939707E-3,6.726328E-3,-1.6655968E-5,-1.0801831E-2,-2.841144E-3,-4.0609846E-3,1.12591684E-1,-2.0555654E-3,-8.360169E-3,1.5584342E-1,1.5031104E-2,1.8905737E-1,-1.1135726E-2,-2.5953692E-1,-2.373416E-2,8.586909E-3,5.2822866E-2,2.698795E-1,3.1139827E-4,4.7859894E-3,1.0618895E-2,-5.9709293E-3,4.2571356E-3,-3.087822E-3,-1.8855523E-2,3.471864E-2,-7.855349E-3,-9.713057E-4,5.338131E-3,6.7645204E-3,1.7380232E-2,4.745281E-3,-4.0433123E-3,-5.385553E-3,4.705019E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,-1,-1,35,-1,-1,37,39,41,43,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,53,-1,-1,55,-1,57,59,61,63,-1,65,67,69,-1,-1,-1,-1,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.984934E0,1.3896139E0,2.8164303E0,1.0647974E0,1.2935834E0,2.3258705E0,4.1492736E-1,6.234354E-1,7.0183486E-1,7.1789503E-1,2.3931935E-1,4.883759E-1,0E0,3.1456998E-1,0E0,2.6038697E-1,7.612833E-1,5.714518E-1,9.1223717E-1,0E0,0E0,1.0046111E-1,0E0,0E0,2.2367549E-1,6.104581E-2,8.615062E-2,1.7839587E-1,4.4093788E-2,2.6089954E-1,0E0,0E0,0E0,2.8716788E-1,5.315969E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.902972E-2,0E0,0E0,4.6105868E-1,0E0,5.201304E-2,1.6610736E-1,4.259441E-1,2.523222E-1,0E0,5.6177355E-2,1.4994705E-1,1.2272683E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.378469E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,21,21,24,24,25,25,26,26,27,27,28,28,29,29,33,33,34,34,44,44,47,47,49,49,50,50,51,51,52,52,54,54,55,55,56,56,63,63],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,-1,-1,36,-1,-1,38,40,42,44,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,54,-1,-1,56,-1,58,60,62,64,-1,66,68,70,-1,-1,-1,-1,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.136E4,5.246E3,2.6056657E0,2.5542571E8,5.6044944E7,1.1E1,3.6796716E5,2.4246153E2,9.27E2,1.4753E4,1.0083192E6,4.6994E4,-4.367699E-2,5.913995E5,9.3739545E-3,7.65244E5,8.385435E0,2.1036612E5,1.48767E5,1.0061208E-2,3.467136E-2,1.1484E4,1.0611019E-2,9.1901724E-4,1.0587665E7,8.509488E2,3.2785368E7,4.761905E-2,8E0,2.3082722E2,-7.2416407E-3,-2.220043E-2,-3.2847081E-3,3.73802E5,3.348166E2,3.174673E-3,-4.7820355E-3,-1.7059918E-2,-4.8939707E-3,6.726328E-3,-1.6655968E-5,-1.0801831E-2,-2.841144E-3,-4.0609846E-3,9.6E1,-2.0555654E-3,-8.360169E-3,7.0516006E8,1.5031104E-2,6.8915665E-1,1.4650456E0,2.9166096E-1,1.722239E6,8.586909E-3,7.12E2,1.113E3,3.5598325E2,4.7859894E-3,1.0618895E-2,-5.9709293E-3,4.2571356E-3,-3.087822E-3,-1.8855523E-2,8.60198E5,-7.855349E-3,-9.713057E-4,5.338131E-3,6.7645204E-3,1.7380232E-2,4.745281E-3,-4.0433123E-3,-5.385553E-3,4.705019E-3],"split_indices":[9,2,52,7,44,3,32,4,2,9,27,28,0,31,0,47,55,46,28,0,0,9,0,0,47,4,12,57,3,51,0,0,0,32,51,0,0,0,0,0,0,0,0,0,10,0,0,5,0,55,53,26,1,0,2,2,4,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,1.87E2,6.1E1,1.6E2,2.7E1,3.1E1,3E1,7.6E1,8.4E1,1.1E1,1.6E1,2.6E1,5E0,2.2E1,8E0,3.1E1,4.5E1,1.3E1,7.1E1,6E0,5E0,1.1E1,5E0,5E0,2.1E1,1E1,1.2E1,2.1E1,1E1,4E1,5E0,5E0,8E0,3E1,4.1E1,6E0,5E0,1.6E1,5E0,5E0,5E0,5E0,7E0,5E0,1.6E1,5E0,5E0,2.5E1,1.5E1,1.9E1,1.1E1,1.4E1,2.7E1,6E0,1E1,1.4E1,1.1E1,8E0,1.1E1,5E0,6E0,7E0,7E0,2E1,7E0,5E0,5E0,8E0,6E0,5E0,6E0,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-2.36425E-3,-2.7555551E-2,4.6177256E-1,2.222548E-2,-2.6043692E-1,3.0943565E-2,5.543953E-3,-4.1164644E-2,1.630732E-1,-1.2894706E-1,-3.3992458E-2,-1.9863173E-2,-2.1077665E-2,3.1838906E-1,2.253767E-2,-3.3447793E-1,-2.0320404E-2,2.7342835E-2,-1.7219082E-1,2.0458352E-2,2.2552836E-1,1.5794787E-1,-2.0502412E-1,-1.9827195E-2,-7.823835E-3,4.9264026E-3,-8.8070735E-2,-2.1345982E-1,7.215155E-2,2.916857E-3,-2.5050113E-1,1.2448578E-2,4.8099635E-3,6.1991975E-2,1.6752375E-2,-1.6111188E-3,-1.446763E-2,-8.605447E-3,7.4271455E-5,-4.2269367E-4,-1.5713563E-2,3.685266E-2,2.2265653E-1,-3.0102223E-1,-5.6122504E-3,7.058188E-3,-8.3076674E-4,6.085451E-3,-1.757926E-3,5.8149924E-3,1.3010188E-2,-6.4567607E-3,-1.6486622E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,-1,-1,11,13,15,-1,17,-1,19,21,23,25,27,29,-1,31,33,35,-1,-1,-1,37,39,41,-1,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7253172E0,2.567858E0,8.282428E-1,1.6406018E0,2.4330251E0,0E0,0E0,1.1438954E0,1.2369859E0,7.0763946E-1,0E0,8.8198334E-1,0E0,2.3693323E-1,9.8762286E-1,1.10027194E-1,2.0274977E-1,1.0296669E0,5.541518E-1,0E0,7.868141E-2,3.8682574E-1,2.238304E-1,0E0,0E0,0E0,1.386188E-1,4.1005528E-1,4.1680717E-1,0E0,1.02623105E-1,0E0,0E0,1.16915934E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.9629396E-1,5.536771E-2,1.0057986E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,26,26,27,27,28,28,30,30,33,33,41,41,42,42,43,43],"right_children":[2,4,6,8,10,-1,-1,12,14,16,-1,18,-1,20,22,24,26,28,30,-1,32,34,36,-1,-1,-1,38,40,42,-1,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,6.627576E7,3.0861E4,4.6253732E2,3.5753E4,3.0943565E-2,5.543953E-3,5.3159845E6,1E0,4.705219E2,-3.3992458E-2,2.1740167E0,-2.1077665E-2,1.0932244E9,4.213523E0,2.126386E0,1E1,1.1744206E6,1.3986014E-2,2.0458352E-2,3.1665432E0,7.399212E2,2.7E1,-1.9827195E-2,-7.823835E-3,4.9264026E-3,1.6741641E3,1.338983E0,3.0954462E2,2.916857E-3,7.265892E0,1.2448578E-2,4.8099635E-3,3.701903E8,1.6752375E-2,-1.6111188E-3,-1.446763E-2,-8.605447E-3,7.4271455E-5,-4.2269367E-4,-1.5713563E-2,6.6123275E5,2.697E3,2.1283955E-3,-5.6122504E-3,7.058188E-3,-8.3076674E-4,6.085451E-3,-1.757926E-3,5.8149924E-3,1.3010188E-2,-6.4567607E-3,-1.6486622E-2],"split_indices":[2,44,10,4,9,0,0,27,107,51,0,55,0,5,55,53,3,44,56,0,55,51,8,0,0,0,4,53,4,0,56,0,0,7,0,0,0,0,0,0,0,47,10,37,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,2.2E2,1.1E1,1.82E2,3.8E1,6E0,5E0,1.26E2,5.6E1,3.1E1,7E0,1.21E2,5E0,2.6E1,3E1,1E1,2.1E1,9.3E1,2.8E1,9E0,1.7E1,1.9E1,1.1E1,5E0,5E0,7E0,1.4E1,1.4E1,7.9E1,7E0,2.1E1,1.1E1,6E0,1.4E1,5E0,5E0,6E0,6E0,8E0,6E0,8E0,6.5E1,1.4E1,1.4E1,7E0,6E0,8E0,2.8E1,3.7E1,7E0,7E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-7.9117104E-4,-4.77925E-2,2.5110805E-1,-8.438953E-5,-3.5618836E-1,5.275657E-1,7.518911E-2,7.388206E-2,-8.725813E-2,-1.7247774E-1,-3.0194694E-2,4.0162418E-2,8.411154E-3,-2.1459295E-3,1.0812879E-2,-1.2295871E-2,9.917311E-2,-3.8596354E-2,-1.6955264E-2,-1.7616877E-2,-6.664059E-2,6.450884E-2,-6.592808E-3,-6.6671264E-3,1.2099931E-1,-1.6920203E-1,5.1330157E-2,-5.234576E-3,8.402492E-4,4.9003405E-3,1.0516298E-3,1.775933E-1,3.746337E-2,-2.3671949E-1,3.8985973E-3,1.15757756E-1,-8.0850255E-3,5.4698315E-2,2.6179653E-1,1.24493666E-1,-6.712894E-2,-5.616567E-3,-1.6140865E-2,1.6944991E-1,-1.5970622E-3,4.400625E-3,-1.4683703E-3,9.735415E-3,1.8460812E-2,1.0041351E-2,1.8251634E-3,-1.0957811E-2,1.1906571E-3,4.9265926E-3,1.1491098E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,-1,23,25,-1,-1,27,29,-1,-1,31,33,35,-1,-1,-1,-1,37,39,41,-1,43,-1,45,47,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.936382E0,3.0749836E0,1.857064E0,1.1799704E0,1.496764E0,1.6913304E0,3.1875837E-1,8.947755E-1,1.165796E0,4.1275835E-1,0E0,0E0,0E0,1.8227403E-1,0E0,0E0,5.19317E-1,8.6606383E-1,0E0,0E0,6.1193008E-2,2.111277E-2,0E0,0E0,3.999232E-1,5.383122E-1,6.7454743E-1,0E0,0E0,0E0,0E0,5.134361E-1,3.3605433E-1,2.8175914E-1,0E0,2.9009417E-1,0E0,8.552267E-2,1.1484718E-1,1.5598139E-1,2.860782E-1,0E0,0E0,1.0578966E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,16,16,17,17,20,20,21,21,24,24,25,25,26,26,31,31,32,32,33,33,35,35,37,37,38,38,39,39,40,40,43,43],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,-1,24,26,-1,-1,28,30,-1,-1,32,34,36,-1,-1,-1,-1,38,40,42,-1,44,-1,46,48,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,6.140638E7,5.25191E5,7.42E2,3.2032586E8,1E0,2.7546022E5,4.305949E-2,5.2726665E0,1.5566E4,-3.0194694E-2,4.0162418E-2,8.411154E-3,1.5971836E5,1.0812879E-2,-1.2295871E-2,7.038094E5,3.70278E5,-1.6955264E-2,-1.7616877E-2,4.7071097E2,4.9308786E-1,-6.592808E-3,-6.6671264E-3,1.7315E4,1E0,5.5180666E8,-5.234576E-3,8.402492E-4,4.9003405E-3,1.0516298E-3,2.3114078E2,6.5229E4,1E0,3.8985973E-3,2.5492957E0,-8.0850255E-3,3.891871E5,2.4539418E0,1.9645177E5,9.73219E6,-5.616567E-3,-1.6140865E-2,5.506776E11,-1.5970622E-3,4.400625E-3,-1.4683703E-3,9.735415E-3,1.8460812E-2,1.0041351E-2,1.8251634E-3,-1.0957811E-2,1.1906571E-3,4.9265926E-3,1.1491098E-2],"split_indices":[51,44,28,0,7,63,32,57,55,9,0,0,0,32,0,0,44,1,0,0,51,41,0,0,9,15,7,0,0,0,0,4,28,80,0,55,0,32,52,46,44,0,0,30,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.08E2,3.8E1,1.81E2,2.7E1,1.4E1,2.4E1,9.8E1,8.3E1,1.8E1,9E0,6E0,8E0,1.7E1,7E0,6E0,9.2E1,7.2E1,1.1E1,5E0,1.3E1,1.2E1,5E0,7E0,8.5E1,2.9E1,4.3E1,8E0,5E0,5E0,7E0,5E1,3.5E1,2.3E1,6E0,3.4E1,9E0,2.1E1,2.9E1,1.9E1,1.6E1,1.3E1,1E1,2.5E1,9E0,1.4E1,7E0,2.4E1,5E0,8E0,1.1E1,5E0,1.1E1,1.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[8.790642E-3,-3.349942E-2,2.9397386E-1,7.45116E-3,-2.8764138E-1,6.035772E-1,5.0372742E-2,-2.23001E-1,3.0318035E-2,-5.691856E-1,-5.7160575E-3,4.1129112E-2,1.1641229E-2,1.2409126E-2,-4.957723E-2,-2.6371707E-3,-1.4984832E-2,8.908825E-2,-1.02209784E-1,-1.3652963E-2,-4.140979E-2,-1.0181847E-1,7.7299206E-3,-8.32796E-3,2.074449E-3,2.1077147E-2,2.2805779E-1,9.388957E-3,-2.2757593E-1,-4.3355575E-4,-7.9836985E-3,1.1350661E-1,-5.171715E-2,1.5616673E-1,1.789516E-2,8.233617E-2,-7.2988397E-3,-1.2348782E-1,-1.6745446E-2,2.018346E-1,3.275248E-2,1.2834552E-1,-1.1197529E-1,1.9225189E-1,6.8953657E-4,-1.8240253E-3,1.6100907E-1,-8.587717E-3,-1.4755188E-3,1.2637511E-2,6.0234186E-3,-5.8851363E-3,4.3232343E-3,2.6397316E-3,8.495104E-3,-6.859271E-3,4.8425477E-3,1.0513133E-2,2.952224E-3,1.1004311E-2,3.5447055E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,-1,-1,25,27,-1,-1,29,-1,-1,-1,31,33,35,37,-1,-1,39,41,43,-1,45,-1,47,-1,49,51,53,55,57,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9887705E0,2.2501116E0,2.383027E0,9.921987E-1,2.3779795E0,1.2565188E0,4.4058982E-1,2.8629243E-1,1.3409715E0,1.1171846E0,2.8983217E-1,0E0,0E0,0E0,1.8929273E-1,0E0,0E0,1.1199185E0,7.43644E-1,0E0,0E0,7.4602865E-2,0E0,0E0,0E0,5.508656E-1,4.1129303E-1,3.756086E-1,3.2829523E-1,0E0,0E0,2.50172E-1,5.1348263E-1,1.444816E-1,0E0,2.0937334E-1,0E0,9.101038E-2,0E0,5.356121E-2,2.1862839E-1,3.9271146E-2,3.2453212E-1,1.03984416E-1,0E0,0E0,6.975743E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,17,17,18,18,21,21,25,25,26,26,27,27,28,28,31,31,32,32,33,33,35,35,37,37,39,39,40,40,41,41,42,42,43,43,46,46],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,-1,-1,26,28,-1,-1,30,-1,-1,-1,32,34,36,38,-1,-1,40,42,44,-1,46,-1,48,-1,50,52,54,56,58,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,6.5023036E7,5.25191E5,1.0585154E6,9.4014386E2,1E0,5.6115907E9,7.543738E1,2.2448614E0,2.585E3,1.118307E6,4.1129112E-2,1.1641229E-2,1.2409126E-2,1E0,-2.6371707E-3,-1.4984832E-2,3.200388E2,7.42E2,-1.3652963E-2,-4.140979E-2,2.0519513E-1,7.7299206E-3,-8.32796E-3,2.074449E-3,8.968965E5,2.3484848E0,1E0,5.137097E0,-4.3355575E-4,-7.9836985E-3,5.282343E5,1.3257E4,2.493493E7,1.789516E-2,3.322259E-2,-7.2988397E-3,4.6253732E2,-1.6745446E-2,3.8315E4,3.19E2,8.3299077E-1,1.067509E6,1.4103535E0,6.8953657E-4,-1.8240253E-3,2.0242085E8,-8.587717E-3,-1.4755188E-3,1.2637511E-2,6.0234186E-3,-5.8851363E-3,4.3232343E-3,2.6397316E-3,8.495104E-3,-6.859271E-3,4.8425477E-3,1.0513133E-2,2.952224E-3,1.1004311E-2,3.5447055E-3],"split_indices":[2,44,28,44,4,80,12,54,55,2,27,0,0,0,111,0,0,4,0,0,0,26,0,0,0,47,52,107,55,0,0,27,9,44,0,56,0,4,0,1,11,55,9,55,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.15E2,3.1E1,1.86E2,2.9E1,1.3E1,1.8E1,1.6E1,1.7E2,1.4E1,1.5E1,6E0,7E0,5E0,1.3E1,7E0,9E0,1.18E2,5.2E1,9E0,5E0,1E1,5E0,5E0,8E0,8E1,3.8E1,2.8E1,2.4E1,5E0,5E0,3.5E1,4.5E1,2.8E1,1E1,2E1,8E0,1.5E1,9E0,1.6E1,1.9E1,1.1E1,3.4E1,2.2E1,6E0,8E0,1.2E1,8E0,7E0,6E0,1E1,5E0,1.4E1,6E0,5E0,2.9E1,5E0,1.6E1,6E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[9.908398E-4,-3.6378037E-2,2.02758E-1,-2.0660823E-2,-2.266779E-2,3.0155038E-2,6.536429E-2,5.403817E-2,-1.06582515E-1,2.1773885E-1,-3.69474E-2,-1.1984075E-1,9.27201E-2,-1.4127442E-1,1.5381564E-1,1.5011296E-2,4.6963417E-3,-1.0448802E-2,4.3424927E-2,-1.3869903E-2,1.9357435E-2,1.4133112E-1,1.8642802E-2,-1.00266315E-1,-3.479405E-1,1.0610357E-2,1.4930537E-3,4.241584E-3,-2.1739816E-3,-2.210221E-3,3.8312674E-3,2.0528585E-1,3.316082E-2,-3.8840823E-2,1.464274E-1,-1.7029756E-1,2.7947258E-2,-2.6005475E-2,-4.701727E-3,1.4996031E-1,1.8269826E-2,-3.7082482E-2,1.0220473E-2,-9.041769E-2,4.4399865E-2,1.8983957E-3,9.6918E-3,-2.2469378E-1,-6.5621044E-3,-4.708189E-2,9.560773E-3,1.0157748E-3,8.216921E-3,3.7967428E-3,-5.880023E-3,-5.5764085E-5,-8.594221E-3,6.297872E-3,-2.6273695E-3,-6.2223105E-3,-1.2948563E-2,-7.03599E-3,6.4864606E-3,1.547417E-3,-9.160637E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,45,47,49,-1,-1,51,-1,53,-1,55,57,-1,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9300281E0,1.5727301E0,2.5172098E0,1.353595E0,0E0,0E0,5.069152E-1,7.711464E-1,9.0703344E-1,1.3538468E-1,3.2314947E-1,5.5477315E-1,3.3225566E-1,7.024369E-1,1.10239774E-1,0E0,0E0,0E0,7.414167E-2,0E0,6.2479742E-2,3.806007E-1,2.8481716E-1,6.692023E-1,7.2031295E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.262862E-1,3.0742437E-1,1.19933456E-1,7.6714605E-2,4.2604172E-1,3.828884E-1,0E0,0E0,1.1512989E-1,0E0,2.0338018E-1,0E0,1.5163593E-1,1.15724996E-1,0E0,0E0,1.553421E-1,3.1471598E-1,2.6105076E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,18,18,20,20,21,21,22,22,23,23,24,24,31,31,32,32,33,33,34,34,35,35,36,36,39,39,41,41,43,43,44,44,47,47,48,48,49,49],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,46,48,50,-1,-1,52,-1,54,-1,56,58,-1,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.796E3,4.966421E6,2.13281E5,7.11E2,-2.266779E-2,3.0155038E-2,1E1,6.52E2,2.7485228E6,9.982716E0,3.3176926E-1,2.1152425E6,2.6786362E8,7.713233E7,3.7578388E6,1.5011296E-2,4.6963417E-3,-1.0448802E-2,8.096533E-1,-1.3869903E-2,4.39E2,2.9807162E0,3E0,3.090766E2,9.4014386E2,1.0610357E-2,1.4930537E-3,4.241584E-3,-2.1739816E-3,-2.210221E-3,3.8312674E-3,1.222E3,1E0,1.5368506E6,1.9627686E5,1E0,1.3143399E6,-2.6005475E-2,-4.701727E-3,7.38E2,1.8269826E-2,1.217E2,1.0220473E-2,1.1406399E6,5.516019E5,1.8983957E-3,9.6918E-3,1.136E3,2.4109095E-1,1.722239E6,9.560773E-3,1.0157748E-3,8.216921E-3,3.7967428E-3,-5.880023E-3,-5.5764085E-5,-8.594221E-3,6.297872E-3,-2.6273695E-3,-6.2223105E-3,-1.2948563E-2,-7.03599E-3,6.4864606E-3,1.547417E-3,-9.160637E-3],"split_indices":[2,27,28,0,0,0,3,2,27,56,38,44,7,44,27,0,0,0,26,0,2,55,8,51,4,0,0,0,0,0,0,2,15,49,46,15,27,0,0,2,0,51,0,49,27,0,0,2,40,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.54E2,2.15E2,3.9E1,2.09E2,6E0,8E0,3.1E1,1.12E2,9.7E1,1.2E1,1.9E1,2E1,9.2E1,8.6E1,1.1E1,5E0,7E0,5E0,1.4E1,8E0,1.2E1,5.5E1,3.7E1,7.3E1,1.3E1,6E0,5E0,9E0,5E0,6E0,6E0,3.4E1,2.1E1,2.6E1,1.1E1,4.7E1,2.6E1,6E0,7E0,2.8E1,6E0,1.6E1,5E0,1.6E1,1E1,5E0,6E0,3.5E1,1.2E1,1.9E1,7E0,6E0,2.2E1,7E0,9E0,9E0,7E0,5E0,5E0,1.6E1,1.9E1,6E0,6E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-1.4351153E-2,-4.114671E-2,3.616836E-1,-2.2000525E-2,-2.236954E-2,2.8743088E-2,3.4446272E-3,-6.2842383E-3,-1.5562599E-2,8.3613195E-2,-7.892925E-2,6.0290918E-2,1.6302267E-2,6.7444734E-2,-1.2460587E-1,-6.1838124E-3,8.20195E-2,-8.308786E-3,1.6012126E-1,-1.9114202E-1,-2.194434E-3,1.1856282E-1,-5.8230404E-2,1.1249048E-3,2.0080413E-1,4.220463E-3,-2.2032061E-1,5.9565548E-2,-1.1009153E-1,7.2394894E-3,-6.6752796E-4,-1.2089339E-2,2.3248668E-3,1.1695284E-2,3.9889556E-3,-1.1871479E-2,-2.4897547E-3,6.2147523E-3,-3.8252037E-3,-1.127211E-2,1.1516135E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,-1,-1,-1,9,-1,11,13,15,-1,17,19,-1,21,-1,23,25,27,29,31,-1,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.423061E0,1.9401932E0,1.2099929E0,1.0947565E0,0E0,0E0,0E0,1.3584338E0,0E0,5.809989E-1,7.7727896E-1,3.844626E-1,0E0,6.8266803E-1,7.163216E-1,0E0,4.1179448E-1,0E0,1.1047572E-1,4.9419165E-1,2.1979679E-1,3.5514152E-1,4.0552768E-1,0E0,8.231455E-2,0E0,3.5421157E-1,2.4899274E-1,2.3405972E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,9,9,10,10,11,11,13,13,14,14,16,16,18,18,19,19,20,20,21,21,22,22,24,24,26,26,27,27,28,28],"right_children":[2,4,6,8,-1,-1,-1,10,-1,12,14,16,-1,18,20,-1,22,-1,24,26,28,30,32,-1,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,1.2754595E8,5.939257E7,4.476758E6,-2.236954E-2,2.8743088E-2,3.4446272E-3,1.8949672E0,-1.5562599E-2,2.902E3,1.3257E4,1.68128E8,1.6302267E-2,3.3712113E-1,1.1583171E6,-6.1838124E-3,2.80617E5,-8.308786E-3,3.6488026E-1,2.460664E7,3.328384E5,1.0159599E0,2.8598264E2,1.1249048E-3,2.160042E9,4.220463E-3,2.38502E0,2.1159E4,1.9270934E-1,7.2394894E-3,-6.6752796E-4,-1.2089339E-2,2.3248668E-3,1.1695284E-2,3.9889556E-3,-1.1871479E-2,-2.4897547E-3,6.2147523E-3,-3.8252037E-3,-1.127211E-2,1.1516135E-3],"split_indices":[51,44,44,27,0,0,0,57,0,2,9,7,0,38,46,0,28,0,55,12,32,57,51,0,5,0,52,9,37,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.38E2,2.23E2,1.5E1,2.15E2,8E0,7E0,8E0,2.06E2,9E0,9.2E1,1.14E2,8.6E1,6E0,2.7E1,8.7E1,8E0,7.8E1,7E0,2E1,5.6E1,3.1E1,6.2E1,1.6E1,5E0,1.5E1,5E0,5.1E1,2E1,1.1E1,4.7E1,1.5E1,5E0,1.1E1,9E0,6E0,4E1,1.1E1,1.3E1,7E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.1352962E-2,-2.2608943E-2,2.541978E-1,8.6810015E-2,-7.679951E-2,6.484933E-1,4.910648E-3,-1.0420439E-1,1.2546127E-1,-1.6311277E-2,-5.8067724E-2,3.7768774E-2,1.4420541E-2,-6.603861E-3,1.1802346E-1,-8.047382E-3,5.49076E-4,1.8207686E-1,2.8233357E-2,-8.898784E-2,1.4515555E-1,7.7978624E-3,1.5463931E-3,1.04365036E-1,2.9945812E-1,-4.2547382E-2,1.1167867E-2,-5.9358206E-2,-2.8342617E-1,2.9196832E-4,9.409138E-3,7.821029E-3,5.4667316E-2,1.714317E-2,9.968983E-3,4.6979524E-2,-7.7342014E-3,1.0869365E-1,-8.763209E-2,-3.3690943E-3,-1.8090175E-2,4.621629E-3,-9.678693E-4,7.3459614E-3,-2.6649698E-3,8.551741E-3,-1.0139223E-3,-2.7644566E-1,-5.349432E-2,-1.8002354E-2,-4.3570227E-3,1.7230856E-3,-4.6720007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,21,-1,-1,23,25,27,29,-1,-1,31,33,35,-1,37,39,-1,-1,-1,41,-1,-1,43,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,49,51,-1,-1,-1,-1],"loss_changes":[2.0841105E0,1.3230103E0,3.043892E0,5.5941147E-1,7.742487E-1,4.7798824E-1,3.59706E-1,1.1452551E-1,3.3614558E-1,0E0,8.989171E-1,0E0,0E0,0E0,4.9238205E-2,0E0,0E0,3.2450163E-1,3.8329744E-1,6.917962E-1,1.6598928E-1,0E0,0E0,7.634598E-2,1.0852575E-2,2.2744063E-1,0E0,5.226848E-1,3.6325252E-1,0E0,0E0,0E0,6.045613E-2,0E0,0E0,1.5720193E-1,0E0,1.7418022E-1,5.846595E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.705419E-1,3.716327E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,27,27,28,28,32,32,35,35,37,37,38,38,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,22,-1,-1,24,26,28,30,-1,-1,32,34,36,-1,38,40,-1,-1,-1,42,-1,-1,44,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,50,52,-1,-1,-1,-1],"split_conditions":[5.666E3,2.2064762E8,5.939257E7,1.07314285E2,1.156884E4,1E0,9.1481786E8,1.8723404E0,2.6986768E0,-1.6311277E-2,1.9002409E6,3.7768774E-2,1.4420541E-2,-6.603861E-3,1.1430797E9,-8.047382E-3,5.49076E-4,1.002E3,1E0,7.85999E9,4.653159E9,7.7978624E-3,1.5463931E-3,8.2E1,1.2345679E-2,4.06E2,1.1167867E-2,1.2414E4,2.6823762E5,2.9196832E-4,9.409138E-3,7.821029E-3,9.157509E-1,1.714317E-2,9.968983E-3,4.720179E10,-7.7342014E-3,9.537158E10,2.1152425E6,-3.3690943E-3,-1.8090175E-2,4.621629E-3,-9.678693E-4,7.3459614E-3,-2.6649698E-3,8.551741E-3,-1.0139223E-3,6.124014E5,1.30727E5,-1.8002354E-2,-4.3570227E-3,1.7230856E-3,-4.6720007E-3],"split_indices":[2,7,44,4,31,80,7,52,55,0,27,0,0,0,7,0,0,2,15,5,5,0,0,10,56,0,0,9,32,0,0,0,55,0,0,30,0,30,44,0,0,0,0,0,0,0,0,27,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.21E2,3E1,7.3E1,1.48E2,1.1E1,1.9E1,1.2E1,6.1E1,8E0,1.4E2,6E0,5E0,8E0,1.1E1,7E0,5E0,3.8E1,2.3E1,1.22E2,1.8E1,6E0,5E0,2.4E1,1.4E1,1.8E1,5E0,1.07E2,1.5E1,6E0,1.2E1,9E0,1.5E1,5E0,9E0,1.1E1,7E0,1.5E1,9.2E1,6E0,9E0,9E0,6E0,5E0,6E0,9E0,6E0,1.3E1,7.9E1,7E0,6E0,2.8E1,5.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[9.213805E-3,4.4507742E-2,-2.259994E-1,2.184267E-2,3.6511537E-1,-3.6820874E-2,-8.639186E-2,6.17963E-2,-6.602151E-2,2.6717333E-2,2.3936885E-3,-1.8491228E-1,5.9158755E-3,-1.6343816E-1,8.259171E-2,2.712654E-2,-1.4626916E-1,1.3308286E-3,-2.63601E-1,-1.4189821E-2,-4.712443E-4,4.1814275E-2,1.9290015E-1,9.459915E-2,-6.838373E-2,-1.2455979E-2,-9.565597E-2,-7.676948E-3,-1.4083829E-2,8.754771E-2,-7.49149E-2,1.7923956E-3,2.3849462E-1,2.4907311E-2,8.0535095E-3,-5.501186E-3,6.538253E-4,-3.9110888E-2,-8.933583E-3,2.6301373E-2,1.9873677E-1,-1.6082545E-1,6.4666376E-2,1.817269E-3,2.8376445E-1,3.713431E-3,-1.654363E-3,-6.3320138E-3,2.324731E-2,3.6394969E-3,-4.5067407E-3,1.3122572E-2,2.8509076E-3,-3.7924363E-3,-1.0669933E-2,5.1607736E-3,1.8514508E-4,1.4402611E-2,6.3776094E-3,-1.2253541E-3,3.5261293E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,23,25,-1,27,-1,-1,29,31,33,35,-1,37,-1,-1,39,41,-1,43,45,-1,-1,-1,47,-1,49,51,53,55,-1,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.035649E0,1.5398096E0,2.56778E0,7.061784E-1,9.6187687E-1,0E0,6.030512E-1,6.59869E-1,4.7488722E-1,0E0,0E0,3.2836455E-1,0E0,2.7437577E-1,5.631922E-1,1.9979885E-1,1.9722462E-1,0E0,1.0514557E-2,0E0,0E0,5.0837076E-1,2.2793591E-1,1.0092732E-1,5.9537865E-2,0E0,1.4288782E-1,0E0,0E0,4.5784765E-1,3.3417058E-1,0E0,2.2184098E-1,4.194299E-2,0E0,0E0,0E0,1.1698244E-1,0E0,3.171786E-1,2.8172082E-1,7.8960866E-2,3.26936E-2,0E0,6.1346054E-2,0E0,0E0,0E0,3.5373464E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,26,26,29,29,30,30,32,32,33,33,37,37,39,39,40,40,41,41,42,42,44,44,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,24,26,-1,28,-1,-1,30,32,34,36,-1,38,-1,-1,40,42,-1,44,46,-1,-1,-1,48,-1,50,52,54,56,-1,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,6.811E3,1.4893761E-2,3.870239E5,5.939257E7,-3.6820874E-2,2.8815672E0,3.814714E-2,6.96E2,2.6717333E-2,2.3936885E-3,1.0904851E0,5.9158755E-3,2.6453333E2,1.546644E6,1.0402472E6,7.2764084E-2,1.3308286E-3,1.978142E0,-1.4189821E-2,-4.712443E-4,5.4732365E9,8.04E2,1E0,5.6617576E7,-1.2455979E-2,4.616371E8,-7.676948E-3,-1.4083829E-2,1.436E3,7.723729E2,1.7923956E-3,3.7787056E-1,5.033303E-1,8.0535095E-3,-5.501186E-3,6.538253E-4,2.2542E5,-8.933583E-3,7.688848E-2,1.6507992E9,2.545E3,4.557268E9,1.817269E-3,3.4863315E6,3.713431E-3,-1.654363E-3,-6.3320138E-3,5.936784E-1,3.6394969E-3,-4.5067407E-3,1.3122572E-2,2.8509076E-3,-3.7924363E-3,-1.0669933E-2,5.1607736E-3,1.8514508E-4,1.4402611E-2,6.3776094E-3,-1.2253541E-3,3.5261293E-3],"split_indices":[9,2,37,32,44,0,52,57,0,0,0,55,0,4,27,50,38,0,53,0,0,5,2,15,12,0,7,0,0,2,51,0,55,26,0,0,0,1,0,37,5,2,12,0,27,0,0,0,26,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.12E2,3.1E1,1.99E2,1.3E1,5E0,2.6E1,1.37E2,6.2E1,7E0,6E0,1.8E1,8E0,1.1E1,1.26E2,2.9E1,3.3E1,5E0,1.3E1,5E0,6E0,9.3E1,3.3E1,1.7E1,1.2E1,8E0,2.5E1,6E0,7E0,6.7E1,2.6E1,8E0,2.5E1,1E1,7E0,7E0,5E0,1.7E1,8E0,4.4E1,2.3E1,1.6E1,1E1,5E0,2E1,5E0,5E0,6E0,1.1E1,3.1E1,1.3E1,1.3E1,1E1,9E0,7E0,5E0,5E0,1.5E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[1.44841615E-2,5.3431246E-2,-2.0728043E-1,1.9405155E-2,3.5259748E-1,-3.5651445E-1,4.5312706E-2,3.139211E-2,-1.1936561E-2,2.94376E-2,1.3440543E-1,-3.543726E-2,-2.0216058E-1,7.4497247E-3,-5.199815E-3,9.770642E-2,-2.195831E-2,1.0290858E-4,1.2003691E-2,-1.4202975E-2,-9.1479175E-2,1.3878885E-1,7.2297227E-4,-1.1690133E-1,3.8266648E-2,-6.721029E-3,-8.4148283E-4,9.4017506E-2,1.4682399E-2,6.261108E-3,-5.747422E-2,2.8576907E-3,-1.5588084E-1,-1.0074635E-2,6.866593E-2,1.20636255E-1,-1.6996224E-3,-3.0476626E-3,-5.949018E-3,-2.0380047E-1,1.1423226E-3,9.660602E-2,-6.885142E-3,7.675017E-3,5.8071094E-4,-3.8578825E-3,3.0711687E-3,-1.71019E-2,-6.614276E-3,7.457361E-3,2.0880012E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,-1,19,-1,-1,21,23,-1,-1,-1,25,27,29,31,33,-1,-1,35,-1,-1,37,-1,39,-1,41,43,-1,45,-1,47,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1962576E0,2.1919205E0,1.4485835E0,6.724259E-1,1.3534389E0,1.4317172E0,3.0793488E-1,6.6727E-1,0E0,0E0,2.4318117E-1,0E0,2.0843029E-1,0E0,0E0,3.3458215E-1,6.048571E-1,0E0,0E0,0E0,4.3332912E-2,4.6287072E-1,2.169855E-1,2.9798806E-1,5.32427E-1,0E0,0E0,1.7633829E-1,0E0,0E0,7.6592766E-2,0E0,3.0040854E-1,0E0,3.7964842E-1,2.1681458E-1,0E0,7.627809E-2,0E0,2.1496356E-1,0E0,1.7568997E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,12,12,15,15,16,16,20,20,21,21,22,22,23,23,24,24,27,27,30,30,32,32,34,34,35,35,37,37,39,39,41,41],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,-1,20,-1,-1,22,24,-1,-1,-1,26,28,30,32,34,-1,-1,36,-1,-1,38,-1,40,-1,42,44,-1,46,-1,48,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,5.666E3,2.3357933E0,4.476758E6,4.083812E7,1.4893761E-2,1.363E3,2.0064912E7,-1.1936561E-2,2.94376E-2,1.0083192E6,-3.543726E-2,5.5157606E-2,7.4497247E-3,-5.199815E-3,1.7266E4,1.9104764E6,1.0290858E-4,1.2003691E-2,-1.4202975E-2,5.436773E5,1.676E4,4.311919E5,6.8064E4,1.0589394E-2,-6.721029E-3,-8.4148283E-4,6.0095956E7,1.4682399E-2,6.261108E-3,1.8295E4,2.8576907E-3,2.38502E0,-1.0074635E-2,6.3018E4,1.6003E4,-1.6996224E-3,1.7899E4,-5.949018E-3,5.0238598E1,1.1423226E-3,1.3255E4,-6.885142E-3,7.675017E-3,5.8071094E-4,-3.8578825E-3,3.0711687E-3,-1.71019E-2,-6.614276E-3,7.457361E-3,2.0880012E-3],"split_indices":[9,2,53,27,44,37,2,12,0,0,27,0,37,0,0,9,50,0,0,0,31,9,31,1,37,0,0,44,0,0,9,0,52,0,10,9,0,9,0,51,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.15E2,3.7E1,1.94E2,2.1E1,2.3E1,1.4E1,1.87E2,7E0,8E0,1.3E1,5E0,1.8E1,8E0,6E0,8.3E1,1.04E2,7E0,6E0,8E0,1E1,5.8E1,2.5E1,4E1,6.4E1,5E0,5E0,4.8E1,1E1,7E0,1.8E1,7E0,3.3E1,6E0,5.8E1,4E1,8E0,1.1E1,7E0,2.6E1,7E0,5.2E1,6E0,2.7E1,1.3E1,5E0,6E0,5E0,2.1E1,2.1E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[4.3579773E-3,-3.3221886E-2,1.8778619E-1,1.2110272E-2,-3.056185E-1,4.2237204E-1,3.3128936E-2,1.03401706E-1,-4.5055E-2,-1.8466797E-1,-3.051154E-2,2.91008E-2,9.097099E-3,9.247945E-2,-3.472779E-3,3.4149673E-2,3.5504916E-1,1.0101753E-1,-7.790791E-2,-1.6605191E-2,-1.0301605E-1,1.5966678E-3,7.187333E-3,-1.4786795E-1,9.261704E-2,2.1063466E-2,8.955256E-3,-3.2886902E-3,1.5993516E-1,-5.067349E-2,-2.1433306E-1,-1.5006544E-1,-3.4465687E-4,9.532726E-4,-1.2502599E-2,-6.0260207E-3,1.3291879E-1,2.698223E-3,9.018198E-3,-7.843489E-2,6.1804123E-2,-1.2862932E-2,-4.1721677E-3,-8.605438E-3,-3.4939148E-3,2.0376778E-1,8.093384E-2,4.138963E-2,-1.1041706E-1,3.2070908E-5,6.324219E-3,4.9407952E-3,1.147109E-2,6.008731E-3,-1.5251702E-3,4.461961E-3,-1.3666768E-3,-9.5364265E-3,-1.8780464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,23,25,27,29,-1,31,-1,-1,33,35,-1,-1,-1,37,39,41,43,-1,-1,-1,-1,45,-1,-1,47,49,-1,-1,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7433267E0,2.59685E0,1.5460647E0,9.4924337E-1,1.2213252E0,7.06161E-1,1.8362382E-1,1.2077723E0,5.453477E-1,3.2181817E-1,0E0,0E0,0E0,6.309298E-2,0E0,6.096522E-1,1.7023194E-1,2.2688755E-1,3.3096135E-1,0E0,7.962307E-2,0E0,0E0,3.1203452E-1,4.0985626E-1,0E0,0E0,0E0,5.218962E-2,2.4840543E-1,1.03194594E-1,2.0727336E-2,0E0,0E0,0E0,0E0,1.1978227E-1,0E0,0E0,2.4600169E-1,7.6580524E-2,0E0,0E0,0E0,0E0,4.365498E-2,1.4260176E-1,6.0778327E-2,3.387106E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,18,18,20,20,23,23,24,24,28,28,29,29,30,30,31,31,36,36,39,39,40,40,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,24,26,28,30,-1,32,-1,-1,34,36,-1,-1,-1,38,40,42,44,-1,-1,-1,-1,46,-1,-1,48,50,-1,-1,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,6.1061732E7,5.25191E5,9.0996725E5,3.5753E4,1E0,1E0,1.518E3,1.3255E4,3.1681168E2,-3.051154E-2,2.91008E-2,9.097099E-3,1.6089E4,-3.472779E-3,1.704596E2,2.9807162E0,2.0260053E5,5.058997E0,-1.6605191E-2,2.6600266E0,1.5966678E-3,7.187333E-3,1.4168001E-3,3.814714E-2,2.1063466E-2,8.955256E-3,-3.2886902E-3,5.1264524E-1,2.3040292E0,1.1E1,6.3609333E0,-3.4465687E-4,9.532726E-4,-1.2502599E-2,-6.0260207E-3,1.81E2,2.698223E-3,9.018198E-3,1.0166397E6,2.5965E5,-1.2862932E-2,-4.1721677E-3,-8.605438E-3,-3.4939148E-3,4.761905E-2,2.3005404E0,1.1285714E0,1.0469056E2,3.2070908E-5,6.324219E-3,4.9407952E-3,1.147109E-2,6.008731E-3,-1.5251702E-3,4.461961E-3,-1.3666768E-3,-9.5364265E-3,-1.8780464E-3],"split_indices":[51,44,28,49,9,63,15,2,9,51,0,0,0,9,0,4,55,46,55,0,52,0,0,41,57,0,0,0,55,52,3,56,0,0,0,0,10,0,0,50,9,0,0,0,0,55,55,57,51,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.09E2,4.2E1,1.8E2,2.9E1,1.6E1,2.6E1,6.9E1,1.11E2,2.3E1,6E0,7E0,9E0,1.7E1,9E0,5.5E1,1.4E1,2E1,9.1E1,6E0,1.7E1,1E1,7E0,1.3E1,4.2E1,7E0,7E0,5E0,1.5E1,7.7E1,1.4E1,1.1E1,6E0,6E0,7E0,6E0,3.6E1,5E0,1E1,6.2E1,1.5E1,8E0,6E0,6E0,5E0,1.4E1,2.2E1,1.3E1,4.9E1,9E0,6E0,6E0,8E0,1.5E1,7E0,7E0,6E0,1.9E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-2.7923079E-3,-4.2400643E-2,2.5970018E-1,-8.905785E-2,7.7281915E-2,5.3701663E-1,8.886694E-2,-6.47928E-2,-2.3482393E-2,-3.8487643E-2,1.2781382E-1,3.4419846E-2,1.2009245E-2,2.1313076E-1,-4.3475688E-2,-5.077403E-2,-1.728997E-2,4.652725E-3,-1.2972693E-1,1.6133398E-1,-2.833361E-3,1.3473948E-2,4.1454057E-3,2.0323272E-3,-5.6264284E-3,2.8683547E-2,-1.17189206E-1,-8.716661E-3,-2.5566102E-3,7.356044E-2,2.5510672E-1,-2.4410037E-2,1.5851066E-1,-2.4236318E-1,-5.2348576E-2,-8.949603E-4,1.374862E-1,5.5467985E-3,1.618175E-2,-1.0134966E-2,2.2450568E-2,4.2016683E-3,9.378792E-3,-1.4390288E-1,-3.2975405E-1,-1.3177505E-1,1.5188806E-2,8.770615E-3,2.1667546E-3,2.4959962E-3,-2.5338607E-3,-2.339862E-3,-9.308217E-3,-1.8308135E-2,-7.8061204E-3,-5.3862523E-4,-9.148181E-3,3.593611E-3,-3.1714872E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,23,25,-1,-1,27,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,49,-1,-1,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5374556E0,1.1907021E0,1.4682724E0,1.5689857E0,3.5592252E-1,5.3180695E-1,3.546194E-1,6.327094E-1,0E0,2.5897688E-1,2.7854133E-1,0E0,0E0,8.675444E-2,8.4810555E-2,7.47445E-1,0E0,0E0,4.475881E-2,2.7754182E-1,0E0,0E0,0E0,0E0,0E0,4.5191336E-1,6.122724E-1,0E0,0E0,1.2215195E-1,1.8884647E-1,4.4630745E-1,4.2039245E-2,1.7626536E-1,2.8094286E-1,0E0,5.184579E-2,0E0,0E0,0E0,1.0402529E-1,0E0,0E0,6.98289E-2,8.500373E-2,2.0205975E-1,1.655884E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,18,18,19,19,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,36,36,40,40,43,43,44,44,45,45,46,46],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,24,26,-1,-1,28,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,50,-1,-1,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,2.3900106E0,1.9824325E0,1.0104842E12,3.7384987E-1,8.3360725E5,5.287008E5,4.966421E6,-2.3482393E-2,9.115134E5,1.1E1,3.4419846E-2,1.2009245E-2,3.6E1,2.402801E6,5.87E2,-1.728997E-2,4.652725E-3,9E0,4.0972165E2,-2.833361E-3,1.3473948E-2,4.1454057E-3,2.0323272E-3,-5.6264284E-3,1.159E3,2.123667E-1,-8.716661E-3,-2.5566102E-3,2.123E3,1.6262975E-1,1.1744206E6,1.436E3,2.4063775E5,3.200388E2,-8.949603E-4,8.73E2,5.5467985E-3,1.618175E-2,-1.0134966E-2,1.262303E7,4.2016683E-3,9.378792E-3,2.2760513E0,7.476915E5,3.3897146E8,2.493493E7,8.770615E-3,2.1667546E-3,2.4959962E-3,-2.5338607E-3,-2.339862E-3,-9.308217E-3,-1.8308135E-2,-7.8061204E-3,-5.3862523E-4,-9.148181E-3,3.593611E-3,-3.1714872E-3],"split_indices":[2,52,53,30,26,27,31,27,0,31,3,0,0,8,28,0,0,0,3,4,0,0,0,0,0,2,40,0,0,11,57,44,2,46,4,0,0,0,0,0,12,0,0,55,46,7,44,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,2.11E2,3.1E1,1.52E2,5.9E1,1.1E1,2E1,1.45E2,7E0,1.8E1,4.1E1,5E0,6E0,1E1,1E1,1.4E2,5E0,7E0,1.1E1,3.5E1,6E0,5E0,5E0,5E0,5E0,6.4E1,7.6E1,5E0,6E0,1.9E1,1.6E1,4.6E1,1.8E1,2.5E1,5.1E1,8E0,1.1E1,8E0,8E0,8E0,3.8E1,9E0,9E0,1.3E1,1.2E1,2.3E1,2.8E1,6E0,5E0,2.7E1,1.1E1,6E0,7E0,7E0,5E0,9E0,1.4E1,1.6E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.8816907E-3,-2.4278399E-2,2.9888695E-1,-7.0945686E-4,-2.8742552E-1,2.4037756E-2,4.939831E-3,-1.1689686E-2,9.900523E-3,-5.035354E-2,-2.9435074E-2,5.3824916E-2,-6.4244695E-2,-7.7549573E-3,2.7547232E-3,1.441727E-1,1.3277931E-2,-2.4218352E-3,-1.5277673E-1,1.7956981E-1,1.2221748E-3,-6.800343E-2,9.241022E-2,1.0790477E-1,-1.07846335E-1,-2.0467201E-1,-1.2859874E-3,1.0510705E-2,1.1407518E-1,7.250806E-2,-1.12779886E-1,1.101082E-1,-1.7147857E-3,2.7661111E-2,9.953622E-3,-1.2394037E-2,-3.2732412E-2,-2.442192E-1,-4.089383E-3,1.8382233E-3,8.65924E-3,4.8516583E-4,5.5090194E-3,-2.52077E-3,-1.0732318E-2,7.3431195E-3,2.5769994E-3,5.1525715E-3,-1.4947979E-3,4.262048E-3,-4.993272E-3,-6.008546E-3,-1.36806965E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,-1,-1,-1,11,13,-1,15,17,-1,-1,19,21,23,25,27,-1,29,31,33,35,37,-1,-1,39,41,43,45,-1,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3742262E0,1.3580558E0,5.6238043E-1,5.566513E-1,1.5261797E0,0E0,0E0,0E0,6.3853085E-1,1.7316076E-1,0E0,4.4867536E-1,3.9918122E-1,0E0,0E0,1.5260106E-1,5.5943453E-1,5.2340674E-1,1.8761456E-1,7.506913E-2,0E0,2.7871656E-1,1.0576117E-1,1.9308689E-1,2.7953807E-1,7.251376E-2,0E0,0E0,7.789795E-2,3.241904E-2,2.2174194E-1,9.9618435E-2,0E0,7.7520184E-2,0E0,0E0,1.7920531E-1,5.8070183E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,8,8,9,9,11,11,12,12,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,33,33,36,36,37,37],"right_children":[2,4,6,8,10,-1,-1,-1,12,14,-1,16,18,-1,-1,20,22,24,26,28,-1,30,32,34,36,38,-1,-1,40,42,44,46,-1,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,1.0388931E8,1.1408508E9,4.26E2,2.7135178E5,2.4037756E-2,4.939831E-3,-1.1689686E-2,2.3005404E0,5.293399E2,-2.9435074E-2,2.1203648E8,1.8733E4,-7.7549573E-3,2.7547232E-3,4.031725E6,2.8003845E2,1.9776594E9,2.3950179E0,1.2092575E7,1.2221748E-3,1.3257E4,6.5023036E7,4.010345E0,6.15175E5,4.1789412E6,-1.2859874E-3,1.0510705E-2,2.053764E2,1E0,2.1485985E2,9E0,-1.7147857E-3,1.2557897E7,9.953622E-3,-1.2394037E-2,2.0600267E5,2.260846E-2,-4.089383E-3,1.8382233E-3,8.65924E-3,4.8516583E-4,5.5090194E-3,-2.52077E-3,-1.0732318E-2,7.3431195E-3,2.5769994E-3,5.1525715E-3,-1.4947979E-3,4.262048E-3,-4.993272E-3,-6.008546E-3,-1.36806965E-2],"split_indices":[2,44,7,2,32,0,0,0,55,51,0,7,9,0,0,12,4,5,53,44,0,9,44,55,1,47,0,0,51,80,54,3,0,44,0,0,32,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,2.18E2,1.3E1,2.01E2,1.7E1,5E0,8E0,7E0,1.94E2,1.1E1,6E0,1.22E2,7.2E1,5E0,6E0,3.7E1,8.5E1,4.3E1,2.9E1,2.8E1,9E0,4.2E1,4.3E1,2.1E1,2.2E1,2E1,9E0,1.4E1,1.4E1,1E1,3.2E1,3.8E1,5E0,1.3E1,8E0,6E0,1.6E1,1.4E1,6E0,8E0,6E0,5E0,5E0,2.3E1,9E0,1.8E1,2E1,5E0,8E0,6E0,1E1,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[6.4480496E-3,3.6821157E-2,-1.716358E-1,4.1867048E-3,2.3473416E-1,2.4548653E-3,-2.7461916E-1,-1.7710816E-2,1.4452136E-1,3.9700696E-1,7.8074574E-2,5.966465E-3,-8.691408E-3,-3.8287452E-1,-4.798961E-3,-1.0037099E-2,-4.9599656E-3,9.385788E-4,2.050738E-1,2.8959526E-2,8.54114E-3,1.4705986E-1,-2.7691568E-3,-1.0017087E-2,-2.4050722E-2,1.8801461E-1,-2.0531734E-2,4.1358364E-3,1.2763257E-2,1.1042952E-2,1.1143715E-3,1.2252835E-2,3.2901866E-3,-5.6005836E-2,7.242794E-2,-1.3623777E-1,-1.0000279E-2,2.226803E-1,2.9665998E-3,-7.950242E-3,-6.501379E-4,-1.4936882E-3,8.253434E-3,5.8507835E-3,1.3256422E-2,-4.2072055E-3,1.8634726E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,-1,25,-1,27,-1,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,-1,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2990614E0,1.3230678E0,6.2816954E-1,5.464772E-1,7.062297E-1,3.803936E-1,3.598262E-1,4.0120322E-1,1.7550933E-1,5.910361E-1,1.6299012E-1,0E0,0E0,1.9596255E-1,0E0,0E0,4.4258377E-1,0E0,1.15457535E-1,0E0,0E0,1.2561794E-1,0E0,0E0,0E0,8.61156E-2,4.5253825E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.6405796E-1,3.9593524E-1,1.7319691E-1,2.9175222E-1,4.1369557E-2,1.0356092E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,16,16,18,18,21,21,25,25,26,26,33,33,34,34,35,35,36,36,37,37,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,-1,26,-1,28,-1,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,-1,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,4.796E3,1.15318E5,1E0,9E0,6.47825E-1,5.5157606E-2,9.235376E5,1.4840448E2,6.4554214E8,1.5152593E10,5.966465E-3,-8.691408E-3,6.3171947E-1,-4.798961E-3,-1.0037099E-2,2.0080225E5,9.385788E-4,3.1060094E4,2.8959526E-2,8.54114E-3,8.389881E0,-2.7691568E-3,-1.0017087E-2,-2.4050722E-2,1.92E2,6.142153E-1,4.1358364E-3,1.2763257E-2,1.1042952E-2,1.1143715E-3,1.2252835E-2,3.2901866E-3,1.4733E4,1.3117E4,1E0,9.190713E-1,8.309148E5,3.0828518E6,-7.950242E-3,-6.501379E-4,-1.4936882E-3,8.253434E-3,5.8507835E-3,1.3256422E-2,-4.2072055E-3,1.8634726E-3],"split_indices":[9,2,28,112,3,38,37,44,51,7,5,0,0,26,0,0,27,0,50,0,0,56,0,0,0,0,40,0,0,0,0,0,0,9,9,108,26,46,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.38E2,2.04E2,3.4E1,1.76E2,2.8E1,1.3E1,2.1E1,1.53E2,2.3E1,1.3E1,1.5E1,8E0,5E0,1.2E1,9E0,8E0,1.45E2,8E0,1.5E1,5E0,8E0,1E1,5E0,7E0,5E0,1E1,1.35E2,7E0,8E0,5E0,5E0,5E0,5E0,9.8E1,3.7E1,3.5E1,6.3E1,1.1E1,2.6E1,2.6E1,9E0,5.7E1,6E0,6E0,5E0,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-3.2143816E-3,9.900955E-3,-1.9555975E-2,3.853463E-2,-1.5629713E-1,1.2253068E-2,2.3682761E-1,-3.541957E-1,3.929027E-2,4.1591465E-2,-6.623793E-2,2.8112395E-2,9.4799355E-2,-3.069922E-2,-1.6137308E-1,7.0784558E-3,-3.1815702E-3,-1.5622856E-1,5.9679843E-2,-1.7631534E-1,-1.8529667E-4,1.2837138E-2,9.999816E-3,-2.914119E-3,-9.976499E-3,1.5430594E-5,-1.2930724E-2,3.4904435E-2,1.7933412E-1,-9.340049E-2,-1.4726168E-2,6.0929954E-3,-6.1680526E-2,-1.4714221E-3,2.2535438E-3,7.224802E-2,-2.4399746E-2,3.4998704E-3,2.4067841E-1,-7.4651865E-3,-1.2450306E-3,-1.18121296E-1,3.4332587E-3,-4.266253E-3,4.474036E-3,4.3271007E-3,-2.87268E-3,1.4342808E-2,6.3108685E-3,-1.1420288E-3,-7.7227587E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,21,-1,23,-1,-1,25,27,29,31,33,-1,-1,-1,-1,-1,35,37,39,-1,-1,41,-1,-1,43,45,-1,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3367032E0,1.1005611E0,0E0,1.0237536E0,1.3299363E0,4.057449E-1,1.203829E0,9.6076727E-1,2.4377167E-1,4.6733892E-1,3.4897065E-1,0E0,1.8446556E-1,0E0,5.5212617E-2,0E0,0E0,2.2491172E-1,3.4354943E-1,1.9375998E-1,2.6627916E-1,2.1894695E-2,0E0,0E0,0E0,0E0,0E0,2.2101127E-1,1.0611647E-1,5.7139E-2,0E0,0E0,1.80123E-1,0E0,0E0,2.8391472E-1,1.9062264E-1,0E0,4.903406E-2,0E0,0E0,7.456665E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,17,17,18,18,19,19,20,20,21,21,27,27,28,28,29,29,32,32,35,35,36,36,38,38,41,41],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,22,-1,24,-1,-1,26,28,30,32,34,-1,-1,-1,-1,-1,36,38,40,-1,-1,42,-1,-1,44,46,-1,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,7.8337E4,-1.9555975E-2,5.666E3,2.2862453E0,3.00037E5,3.0451374E9,1.4893761E-2,2.2193549E0,9.541066E5,3.1236285E2,2.8112395E-2,1.0083192E6,-3.069922E-2,9.33E2,7.0784558E-3,-3.1815702E-3,6.8883444E5,2.201776E6,4.0491594E5,7.0522516E2,2.1492538E0,9.999816E-3,-2.914119E-3,-9.976499E-3,1.5430594E-5,-1.2930724E-2,3.9418628E5,6.872321E-1,1.616E3,-1.4726168E-2,6.0929954E-3,9.2379E2,-1.4714221E-3,2.2535438E-3,1.6819989E8,1.1285714E0,3.4998704E-3,2.674421E9,-7.4651865E-3,-1.2450306E-3,3.3658516E11,3.4332587E-3,-4.266253E-3,4.474036E-3,4.3271007E-3,-2.87268E-3,1.4342808E-2,6.3108685E-3,-1.1420288E-3,-7.7227587E-3],"split_indices":[27,9,0,2,52,28,12,37,57,44,51,0,27,0,0,0,0,44,31,46,4,55,0,0,0,0,0,32,26,2,0,0,51,0,0,7,57,0,5,0,0,30,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,2.29E2,6E0,1.96E2,3.3E1,1.74E2,2.2E1,1.6E1,1.7E1,1.27E2,4.7E1,5E0,1.7E1,5E0,1.1E1,8E0,9E0,1E1,1.17E2,1.7E1,3E1,1.1E1,6E0,5E0,6E0,5E0,5E0,9.8E1,1.9E1,1.2E1,5E0,9E0,2.1E1,5E0,6E0,6E1,3.8E1,8E0,1.1E1,5E0,7E0,1.5E1,6E0,8E0,5.2E1,9E0,2.9E1,5E0,6E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[7.66354E-3,-6.9291014E-3,2.7212456E-1,3.927668E-3,-1.5094389E-2,2.7548177E-3,2.1327492E-2,5.0123893E-2,-6.433801E-2,3.7694994E-2,1.3279273E-2,6.4003535E-2,-1.0945547E-1,5.069662E-3,1.292953E-1,-2.0017274E-2,1.0102692E-2,-1.3803177E-2,-7.171012E-2,4.627883E-2,-6.876097E-2,2.9338393E-2,9.729422E-3,-4.8548565E-3,2.9610254E-2,-1.0527952E-1,7.2138354E-2,1.1489791E-2,1.3905627E-1,2.127545E-3,-1.14205495E-1,5.0207935E-3,-4.148709E-3,3.1824987E-3,-7.346423E-4,-5.6407508E-2,-1.4621704E-2,-2.058709E-3,8.022331E-3,3.6259915E-3,-3.392667E-3,3.168279E-3,8.976421E-3,-7.849406E-3,2.2173062E-4,1.5335291E-3,-4.2613717E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,-1,15,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,35,37,39,41,-1,43,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.856389E-1,7.771797E-1,4.7973216E-1,6.717916E-1,0E0,0E0,0E0,3.7462527E-1,5.049838E-1,3.639404E-1,0E0,3.137609E-1,4.5310628E-1,2.8004956E-1,2.7025926E-1,7.278371E-2,0E0,0E0,2.731846E-1,1.8883294E-1,1.7926456E-1,1.6945899E-1,0E0,0E0,2.1835174E-2,4.6330354E-1,1.4517301E-1,2.6906213E-1,5.2068084E-2,0E0,1.7201737E-1,0E0,0E0,0E0,0E0,1.3410403E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,18,18,19,19,20,20,21,21,24,24,25,25,26,26,27,27,28,28,30,30,35,35],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,-1,16,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,36,38,40,42,-1,44,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.013055E3,4.966421E6,2.3E1,3.452399E8,-1.5094389E-2,2.7548177E-3,2.1327492E-2,4.774E3,1.3117E4,1E0,1.3279273E-2,1.2262E4,2.1152425E6,2.6986768E0,1.4840448E2,2.1996392E-1,1.0102692E-2,-1.3803177E-2,1.714168E6,2.2023554E0,1.1890752E-1,2.854988E5,9.729422E-3,-4.8548565E-3,1.1007E4,4.9442764E7,1.738E3,8.575148E-1,4.3804785E2,2.127545E-3,5.3157896E-1,5.0207935E-3,-4.148709E-3,3.1824987E-3,-7.346423E-4,7.6618755E6,-1.4621704E-2,-2.058709E-3,8.022331E-3,3.6259915E-3,-3.392667E-3,3.168279E-3,8.976421E-3,-7.849406E-3,2.2173062E-4,1.5335291E-3,-4.2613717E-3],"split_indices":[51,27,8,7,0,0,0,2,9,15,0,9,44,55,51,38,0,0,27,52,26,46,0,0,9,44,0,26,4,0,26,0,0,0,0,44,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.28E2,2.17E2,1.1E1,2.11E2,6E0,6E0,5E0,1.26E2,8.5E1,1.21E2,5E0,2.2E1,6.3E1,9E1,3.1E1,1.5E1,7E0,9E0,5.4E1,5.8E1,3.2E1,1.5E1,1.6E1,5E0,1E1,4.4E1,1E1,4.3E1,1.5E1,9E0,2.3E1,9E0,6E0,5E0,5E0,3.7E1,7E0,5E0,5E0,2.4E1,1.9E1,8E0,7E0,1.5E1,8E0,1.1E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-3.8230412E-3,-3.6651343E-2,1.5480688E-1,-1.5809039E-2,-2.1309864E-2,3.181688E-1,-1.5926767E-2,6.843989E-2,-5.4275658E-2,1.8063225E-1,2.5587052E-2,-7.828384E-2,2.7278317E-3,-7.331226E-3,9.711986E-2,-1.2547378E-1,-1.0200266E-2,2.3979351E-3,1.0778925E-2,-7.0239007E-3,-3.0940017E-5,6.947403E-2,1.3830149E-2,-1.704042E-2,-7.5592615E-2,6.742552E-2,-7.5524256E-2,1.6718656E-1,1.7031992E-2,-3.1889077E-2,-1.9983119E-1,3.3137105E-2,1.1037434E-2,1.7404685E-2,-1.4410329E-1,9.331242E-3,2.697525E-3,1.24137565E-1,-4.2163413E-2,9.634735E-3,-7.343522E-3,-1.3122007E-2,-3.3978822E-3,5.9924447E-3,-4.100449E-3,4.6460405E-3,-6.20224E-2,-1.9041133E-1,1.8935205E-3,3.2695339E-3,7.380842E-3,-4.99372E-3,3.1205202E-3,5.135381E-3,-2.8245898E-3,-3.775574E-3,2.0708297E-3,-6.199578E-3,1.072234E-3,-1.7481825E-3,-1.1880167E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,-1,21,23,25,-1,-1,-1,-1,27,-1,-1,29,31,33,35,37,39,41,43,-1,45,47,-1,-1,49,51,53,-1,-1,-1,-1,55,-1,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.245071E0,1.791522E0,1.1486137E0,6.1946124E-1,0E0,6.1745644E-1,1.0480915E-1,4.133331E-1,4.0980938E-1,9.8955184E-2,0E0,7.2043456E-2,0E0,0E0,2.9157996E-1,6.0100144E-1,4.2090002E-1,0E0,0E0,0E0,0E0,2.4824043E-1,0E0,0E0,2.2710319E-1,2.2296716E-1,2.89292E-1,6.0171783E-2,2.1445929E-1,1.8103026E-1,1.0290912E-1,1.2312871E-1,0E0,1.4241616E-1,2.3205632E-1,0E0,0E0,1.2632474E-2,1.7569137E-1,2.0303734E-1,0E0,0E0,0E0,0E0,1.034312E-1,0E0,7.449302E-2,2.2241902E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,14,14,15,15,16,16,21,21,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,37,37,38,38,39,39,44,44,46,46,47,47],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,-1,22,24,26,-1,-1,-1,-1,28,-1,-1,30,32,34,36,38,40,42,44,-1,46,48,-1,-1,50,52,54,-1,-1,-1,-1,56,-1,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,1.00068504E8,1.4521701E2,2.1203648E8,-2.1309864E-2,6.1377846E-2,1.8327797E3,4.75E2,1.06199684E2,5.3447795E-1,2.5587052E-2,2.189944E0,2.7278317E-3,-7.331226E-3,3.1515045E6,1.573075E5,1.7884366E7,2.3979351E-3,1.0778925E-2,-7.0239007E-3,-3.0940017E-5,2.94E2,1.3830149E-2,-1.704042E-2,1.28831E5,2.596E3,3.91014E5,6.805556E5,1.6003E4,1.9795657E0,1.1968433E-1,1.3176E4,1.1037434E-2,3.3897146E8,2.6056657E0,9.331242E-3,2.697525E-3,4E0,2.1857262E0,6.557827E-1,-7.343522E-3,-1.3122007E-2,-3.3978822E-3,5.9924447E-3,5.5064187E0,4.6460405E-3,5.6790997E1,1.2106E4,1.8935205E-3,3.2695339E-3,7.380842E-3,-4.99372E-3,3.1205202E-3,5.135381E-3,-2.8245898E-3,-3.775574E-3,2.0708297E-3,-6.199578E-3,1.072234E-3,-1.7481825E-3,-1.1880167E-2],"split_indices":[51,44,57,7,0,37,51,2,51,26,0,52,0,0,31,32,44,0,0,0,0,0,0,0,28,2,28,27,9,40,40,9,0,7,52,0,0,8,53,26,0,0,0,0,56,0,57,9,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,1.97E2,4E1,1.89E2,8E0,2E1,2E1,5.9E1,1.3E2,1.4E1,6E0,1.1E1,9E0,6E0,5.3E1,4.9E1,8.1E1,5E0,9E0,5E0,6E0,4.8E1,5E0,7E0,4.2E1,3.7E1,4.4E1,1.6E1,3.2E1,3.2E1,1E1,3.2E1,5E0,1.9E1,2.5E1,1.1E1,5E0,1.1E1,2.1E1,2.5E1,7E0,5E0,5E0,8E0,2.4E1,9E0,1E1,2E1,5E0,6E0,5E0,1.3E1,8E0,1E1,1.5E1,9E0,1.5E1,5E0,5E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[1.6482933E-3,-2.972496E-2,1.8722759E-1,-6.532279E-4,-2.2997269E-1,2.0680085E-2,1.0590549E-1,5.4273E-2,-5.7960454E-2,-3.6444846E-1,-7.907031E-2,1.07753975E-2,4.3837674E-2,3.3781806E-3,1.8421243E-1,-2.0662054E-1,-3.088195E-2,-7.352619E-3,-2.4464648E-2,-7.533032E-4,-6.2700063E-3,6.4246883E-3,4.1939355E-3,3.202434E-2,-1.3866705E-2,3.1924757E-1,8.600072E-2,-1.2571415E-2,-2.8825365E-3,8.910095E-2,-6.7747414E-2,2.1186348E-3,-1.1182071E-3,5.100752E-2,-7.356578E-3,1.7736625E-2,8.655325E-3,8.956289E-3,-2.9546744E-3,-5.8006606E-4,1.3715121E-1,-9.209885E-2,5.1313546E-2,-2.9720578E-3,7.0262425E-2,8.40927E-3,3.7428022E-3,-1.3454959E-1,-1.3521312E-2,5.655657E-3,-1.413606E-3,4.8198763E-4,4.9725026E-3,-3.6683434E-3,-9.06065E-3,-3.533504E-3,1.0115457E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,27,29,-1,-1,-1,-1,31,-1,33,-1,35,37,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,45,47,49,-1,51,-1,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4494773E0,1.2398396E0,7.459476E-1,5.9176E-1,5.190861E-1,0E0,2.290237E-1,6.342491E-1,3.6396444E-1,4.1367638E-1,5.0044023E-2,0E0,3.803408E-2,6.332672E-1,3.3061934E-1,1.2635934E-1,3.5547113E-1,0E0,0E0,0E0,0E0,1.7628204E-2,0E0,2.4914224E-1,0E0,2.8252363E-2,3.031181E-1,0E0,0E0,9.48537E-2,1.815835E-1,0E0,0E0,1.397058E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.9251645E-2,1.6830012E-1,7.13198E-2,0E0,1.22769654E-1,0E0,0E0,1.00195765E-1,4.605205E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,21,21,23,23,25,25,26,26,29,29,30,30,33,33,40,40,41,41,42,42,44,44,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,28,30,-1,-1,-1,-1,32,-1,34,-1,36,38,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,46,48,50,-1,52,-1,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.723729E2,6.140638E7,4.083812E7,2.3378747E0,2.3641207E0,2.0680085E-2,1.6565532E3,4.1698615E2,1.398214E6,1E0,3.338528E-3,1.07753975E-2,2.013055E3,3.26989E6,2.1261516E0,5.428733E5,1.3255E4,-7.352619E-3,-2.4464648E-2,-7.533032E-4,-6.2700063E-3,2.4269E4,4.1939355E-3,3.9128714E0,-1.3866705E-2,4.8090588E7,1.6507992E9,-1.2571415E-2,-2.8825365E-3,3.1667283E8,3.9556694E-1,2.1186348E-3,-1.1182071E-3,1.4221454E0,-7.356578E-3,1.7736625E-2,8.655325E-3,8.956289E-3,-2.9546744E-3,-5.8006606E-4,3.1506528E8,1.4580708E1,1.1797377E0,-2.9720578E-3,3.2456957E-3,8.40927E-3,3.7428022E-3,6.001792E5,3.005376E5,5.655657E-3,-1.413606E-3,4.8198763E-4,4.9725026E-3,-3.6683434E-3,-9.06065E-3,-3.533504E-3,1.0115457E-3],"split_indices":[51,44,44,57,52,0,4,4,49,63,38,0,51,27,55,27,9,0,0,0,0,10,0,55,0,44,5,0,0,12,37,0,0,52,0,0,0,0,0,0,7,56,41,0,41,0,0,46,32,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.47E2,2.12E2,3.5E1,1.86E2,2.6E1,7E0,2.8E1,9.5E1,9.1E1,1.3E1,1.3E1,8E0,2E1,6.9E1,2.6E1,1.3E1,7.8E1,7E0,6E0,7E0,6E0,1.2E1,8E0,6.4E1,5E0,1E1,1.6E1,8E0,5E0,1.8E1,6E1,5E0,7E0,5.9E1,5E0,5E0,5E0,9E0,7E0,6E0,1.2E1,5E1,1E1,8E0,5.1E1,5E0,7E0,3.2E1,1.8E1,5E0,5E0,2.1E1,3E1,1.9E1,1.3E1,6E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-3.5081918E-3,-3.6444664E-2,1.4746839E-1,-4.530097E-3,-3.5356563E-1,3.449253E-1,5.990683E-2,5.251018E-2,-5.72745E-2,-1.6479579E-1,-3.172872E-2,2.918248E-2,3.38754E-3,-8.403556E-3,1.6876523E-1,3.423092E-3,1.889515E-1,-2.2287215E-1,-3.0322744E-2,-1.1210288E-2,-1.1534971E-3,-6.3598417E-3,4.5211464E-2,4.355598E-3,1.0138495E-2,2.641272E-2,-1.0237924E-2,2.9566535E-1,6.649169E-2,-1.3539568E-2,-3.7218514E-3,4.0474508E-2,-8.3905265E-2,5.3153504E-3,-1.3117655E-4,4.4476807E-2,-6.4742113E-3,1.8172048E-2,7.2792135E-3,7.594671E-3,-2.8639224E-3,-6.1608784E-2,8.785551E-2,-3.781795E-2,-1.7970367E-1,-5.8951035E-2,7.7893786E-2,-5.866151E-3,1.287679E-3,7.166081E-3,2.9884312E-2,-7.649343E-2,4.6486955E-2,-1.0288415E-2,-4.905183E-3,-5.240338E-3,1.6579519E-3,4.5579383E-3,-3.5204142E-3,3.9237994E-3,-3.170784E-3,-1.2871347E-3,-8.2074E-3,4.1521643E-3,-3.091238E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,-1,-1,-1,33,-1,-1,35,-1,37,39,-1,-1,41,43,-1,-1,45,-1,-1,-1,-1,-1,47,49,51,53,55,57,-1,-1,-1,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1938344E0,1.9921914E0,7.2279096E-1,5.4454744E-1,1.0735438E0,1.0114846E0,2.3193917E-1,5.820363E-1,4.1277778E-1,1.4532256E-1,0E0,0E0,0E0,1.4860332E-1,2.6845574E-2,3.5038933E-1,2.809084E-1,1.16404414E-1,3.144626E-1,0E0,0E0,0E0,5.279356E-2,0E0,0E0,1.9062825E-1,0E0,1.0783756E-1,1.697181E-1,0E0,0E0,1.7953777E-1,2.0071325E-1,0E0,0E0,1.9468543E-1,0E0,0E0,0E0,0E0,0E0,7.781277E-2,9.8243535E-2,1.1115352E-1,2.6733637E-2,8.100851E-2,1.5977472E-1,0E0,0E0,0E0,9.199101E-2,1.1117469E-1,2.7384518E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,27,27,28,28,31,31,32,32,35,35,41,41,42,42,43,43,44,44,45,45,46,46,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,-1,-1,-1,34,-1,-1,36,-1,38,40,-1,-1,42,44,-1,-1,46,-1,-1,-1,-1,-1,48,50,52,54,56,58,-1,-1,-1,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,7.526755E7,1E0,2.459517E0,9.355891E0,4.27928E5,1.1533942E6,4.4526025E2,1.4065148E6,2.4192488E0,-3.172872E-2,2.918248E-2,3.38754E-3,2.1374558E3,2.5324342E5,3.6543632E6,2.9807162E0,1.6066E4,1.0994765E0,-1.1210288E-2,-1.1534971E-3,-6.3598417E-3,4.9308786E-1,4.355598E-3,1.0138495E-2,3.7304838E0,-1.0237924E-2,1.6506441E9,1.0800359E-1,-1.3539568E-2,-3.7218514E-3,1.901916E6,2.5086E4,5.3153504E-3,-1.3117655E-4,1.4150975E4,-6.4742113E-3,1.8172048E-2,7.2792135E-3,7.594671E-3,-2.8639224E-3,1E0,1E0,2.1712038E6,7.6132205E6,1.514963E6,1.1486098E6,-5.866151E-3,1.287679E-3,7.166081E-3,6E0,6.727519E2,3.6777365E6,-1.0288415E-2,-4.905183E-3,-5.240338E-3,1.6579519E-3,4.5579383E-3,-3.5204142E-3,3.9237994E-3,-3.170784E-3,-1.2871347E-3,-8.2074E-3,4.1521643E-3,-3.091238E-4],"split_indices":[51,44,63,57,56,28,27,4,49,52,0,0,0,4,32,27,55,9,55,0,0,0,41,0,0,55,0,5,26,0,0,50,9,0,0,50,0,0,0,0,0,80,111,46,49,27,50,0,0,0,8,4,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.38E2,1.96E2,4.2E1,1.79E2,1.7E1,1.2E1,3E1,8.6E1,9.3E1,1.2E1,5E0,5E0,7E0,1.9E1,1.1E1,6.4E1,2.2E1,1.2E1,8.1E1,7E0,5E0,5E0,1.4E1,6E0,5E0,5.9E1,5E0,1.1E1,1.1E1,7E0,5E0,3.5E1,4.6E1,5E0,9E0,5.4E1,5E0,5E0,6E0,6E0,5E0,1.1E1,2.4E1,3.2E1,1.4E1,1.3E1,4.1E1,6E0,5E0,1E1,1.4E1,2.2E1,1E1,7E0,7E0,8E0,5E0,3.6E1,5E0,9E0,5E0,1.6E1,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-5.2889116E-4,-2.5684608E-2,1.4552872E-1,8.697321E-4,-1.7818597E-1,2.0899376E-2,5.2799817E-2,-2.4423653E-2,8.659927E-2,-7.395148E-2,-2.7063096E-2,1.6614425E-1,-2.5696456E-2,-1.2490281E-2,-1.2932284E-2,-4.65127E-2,1.4328396E-1,-1.2047326E-2,1.12960115E-2,3.0645751E-3,1.1408316E-2,-6.631078E-3,3.229213E-2,6.853372E-2,-5.9198584E-2,1.0037571E-3,-4.1610356E-3,1.2162369E-3,1.9427566E-1,3.916241E-3,-5.1547345E-2,4.5144293E-3,-1.0196161E-3,2.7234688E-2,1.6032921E-1,6.9473185E-2,-1.0183828E-1,2.9359383E-3,1.0506075E-2,5.0031736E-5,-4.3114107E-3,4.294594E-3,3.5575386E-3,1.0481151E-2,3.8973491E-3,-3.1604462E-3,1.2618929E-1,-1.4309837E-1,5.5860453E-3,-1.2327714E-3,3.6233382E-3,1.1041986E-3,7.6743145E-3,-1.277989E-2,-4.8270468E-3,3.2411292E-3,-2.7625426E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,-1,23,25,27,-1,29,-1,-1,-1,31,33,35,-1,-1,-1,37,-1,39,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,49,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.002387E-1,8.465158E-1,1.0369257E0,3.902569E-1,1.319164E0,0E0,2.622857E-1,3.9984792E-1,3.1602642E-1,4.3127498E-1,0E0,8.513668E-2,1.3142283E-1,0E0,5.0912064E-1,4.297055E-2,1.6675717E-1,0E0,9.468944E-2,0E0,0E0,0E0,5.0672602E-2,1.8139568E-1,4.7891623E-1,0E0,0E0,0E0,8.48428E-2,0E0,2.5630055E-2,0E0,0E0,5.6017257E-2,5.9841156E-2,1.8298295E-1,2.8890127E-1,0E0,0E0,0E0,0E0,0E0,6.6260405E-2,0E0,0E0,0E0,6.7433655E-2,2.0740986E-1,8.860409E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,18,18,22,22,23,23,24,24,28,28,30,30,33,33,34,34,35,35,36,36,42,42,46,46,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,-1,24,26,28,-1,30,-1,-1,-1,32,34,36,-1,-1,-1,38,-1,40,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,50,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,6.5023036E7,2.13281E5,2.421233E0,2.9119E4,2.0899376E-2,1E1,3.0734711E-6,3.7384987E-1,3.348166E2,-2.7063096E-2,2.6823762E5,1.99607E3,-1.2490281E-2,2.0064912E7,1.4435129E6,2.0752347E2,-1.2047326E-2,2.7831267E8,3.0645751E-3,1.1408316E-2,-6.631078E-3,4.9308786E-1,1.135E3,1.3257E4,1.0037571E-3,-4.1610356E-3,1.2162369E-3,1.569966E-1,3.916241E-3,2.1053917E5,4.5144293E-3,-1.0196161E-3,2.836764E5,1.9185424E0,3.6452372E5,2.5245471E2,2.9359383E-3,1.0506075E-2,5.0031736E-5,-4.3114107E-3,4.294594E-3,1E0,1.0481151E-2,3.8973491E-3,-3.1604462E-3,5.1264524E-1,2.8290488E5,2.493493E7,-1.2327714E-3,3.6233382E-3,1.1041986E-3,7.6743145E-3,-1.277989E-2,-4.8270468E-3,3.2411292E-3,-2.7625426E-3],"split_indices":[2,44,28,52,9,0,3,37,26,51,0,32,4,0,12,31,4,0,7,0,0,0,41,2,9,0,0,0,57,0,32,0,0,27,53,27,51,0,0,0,0,0,8,0,0,0,55,32,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.08E2,3.5E1,1.78E2,3E1,7E0,2.8E1,1.38E2,4E1,2.5E1,5E0,1.1E1,1.7E1,5E0,1.33E2,1.2E1,2.8E1,7E0,1.8E1,6E0,5E0,5E0,1.2E1,4.8E1,8.5E1,5E0,7E0,9E0,1.9E1,8E0,1E1,5E0,7E0,3.4E1,1.4E1,2.1E1,6.4E1,5E0,1.4E1,5E0,5E0,8E0,2.6E1,6E0,8E0,6E0,1.5E1,4.6E1,1.8E1,1.9E1,7E0,5E0,1E1,8E0,3.8E1,9E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[4.0719085E-3,-1.7674431E-2,2.0397536E-1,-4.9054384E-4,-2.0848021E-1,2.1254865E-2,5.04184E-2,-2.839942E-2,6.306337E-2,-6.1903875E-2,-2.3256127E-2,-2.0048523E-3,1.0621276E-1,2.4865146E-3,-1.5522401E-1,9.554495E-3,1.2144231E-1,-8.154225E-3,1.09399E-3,8.13037E-3,6.501559E-4,1.6526705E-2,-1.02144545E-2,6.069664E-4,-1.9092748E-1,8.07648E-3,-2.4675077E-2,1.5288489E-1,-1.5804496E-3,-7.9016164E-2,4.0232785E-2,-1.1141777E-2,-1.0209311E-1,-8.856093E-2,2.9388279E-2,1.014768E-1,1.07641565E-2,-1.7739917E-1,1.0309162E-2,1.0046856E-1,-2.218709E-2,-6.654065E-3,-1.7858932E-3,-2.9735942E-4,-7.6469933E-3,4.508898E-3,-8.362922E-3,1.1613623E-3,6.643083E-3,-1.0541449E-2,-4.124003E-3,-3.5765527E-3,3.4378185E-3,2.3975754E-3,7.289738E-3,2.4587081E-3,-4.885872E-3,1.5656125E-3,-2.5807547E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,19,21,23,25,27,-1,-1,-1,-1,29,-1,-1,31,-1,33,35,-1,37,39,-1,41,43,45,47,-1,49,51,53,55,-1,-1,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1077859E0,7.540183E-1,1.0027173E0,3.7781924E-1,8.237905E-1,0E0,9.47159E-2,5.801162E-1,2.0188615E-1,1.4785391E-1,0E0,0E0,7.2214164E-2,3.8958457E-1,1.7722607E-1,2.0755291E-1,1.578939E-1,0E0,0E0,0E0,0E0,2.61383E-1,0E0,0E0,9.451014E-2,0E0,1.0621916E-1,9.0495884E-2,0E0,2.0395504E-1,3.4814078E-1,0E0,2.541177E-2,9.185966E-2,4.614802E-2,5.869201E-2,0E0,3.1915277E-2,8.2864575E-2,1.2743953E-1,3.0996984E-1,0E0,0E0,0E0,0E0,0E0,2.7252669E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,16,16,21,21,24,24,26,26,27,27,29,29,30,30,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,46,46],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,20,22,24,26,28,-1,-1,-1,-1,30,-1,-1,32,-1,34,36,-1,38,40,-1,42,44,46,48,-1,50,52,54,56,-1,-1,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,1.8E1,4.083812E7,2.3040292E0,1.1647451E6,2.1254865E-2,9.1481786E8,2.1159E4,4.397E3,3E0,-2.3256127E-2,-2.0048523E-3,1.1430797E9,4.476758E6,2.161E4,1.5779E4,2.6308E6,-8.154225E-3,1.09399E-3,8.13037E-3,6.501559E-4,7.23E2,-1.02144545E-2,6.069664E-4,1.9805251E-1,8.07648E-3,5.3157896E-1,7.3972464E-1,-1.5804496E-3,7.912113E-1,7.07E2,-1.1141777E-2,5.932798E6,4.607055E5,9.29E2,3.645971E5,1.07641565E-2,3.0050538E5,3.202E3,1E0,3.300363E5,-6.654065E-3,-1.7858932E-3,-2.9735942E-4,-7.6469933E-3,4.508898E-3,2.33948E8,1.1613623E-3,6.643083E-3,-1.0541449E-2,-4.124003E-3,-3.5765527E-3,3.4378185E-3,2.3975754E-3,7.289738E-3,2.4587081E-3,-4.885872E-3,1.5656125E-3,-2.5807547E-3],"split_indices":[2,3,44,52,27,0,7,9,11,8,0,0,7,27,28,9,1,0,0,0,0,2,0,0,38,0,26,26,0,26,0,0,50,27,11,32,0,32,28,80,32,0,0,0,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,2.29E2,2.4E1,2.11E2,1.8E1,8E0,1.6E1,1.47E2,6.4E1,1.3E1,5E0,6E0,1E1,1.19E2,2.8E1,3.4E1,3E1,5E0,8E0,5E0,5E0,1.13E2,6E0,5E0,2.3E1,5E0,2.9E1,2.5E1,5E0,2.2E1,9.1E1,1.3E1,1E1,1.3E1,1.6E1,1.7E1,8E0,1E1,1.2E1,4.6E1,4.5E1,5E0,5E0,7E0,6E0,5E0,1.1E1,7E0,1E1,5E0,5E0,5E0,7E0,2.7E1,1.9E1,2.4E1,2.1E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-6.4496947E-3,-2.8472792E-2,1.3531163E-1,-1.8054172E-2,-1.39254555E-2,1.8431708E-2,2.4936793E-2,-7.6272488E-3,-1.16943745E-2,-5.5176023E-2,9.754606E-2,4.1358378E-2,-6.511935E-2,2.0632336E-3,-8.141675E-3,9.6384046E-4,6.8541486E-3,1.9305032E-2,1.2777357E-1,-4.3999042E-2,-1.7764378E-1,-8.592917E-2,3.770976E-2,4.2536788E-2,2.0236255E-1,-1.1195725E-2,-2.7720679E-2,-1.1193316E-2,-4.644699E-3,-6.1423234E-3,1.7241485E-4,9.061834E-2,-1.3537657E-2,4.691314E-3,-1.6370359E-3,1.1603358E-2,4.7127362E-3,-1.0862571E-2,-1.0290403E-2,4.134209E-4,6.6231843E-3,-1.6897144E-3,5.966918E-3,-1.5485843E-3,5.6185517E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,-1,-1,9,11,-1,13,15,17,19,-1,-1,-1,-1,21,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,39,41,-1,-1,-1,-1,43,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.124569E-1,6.5161407E-1,1.0461516E0,5.464882E-1,0E0,0E0,1.5662247E-1,5.941134E-1,0E0,1.727739E-1,5.4823294E-2,2.1478814E-1,2.2270924E-1,0E0,0E0,0E0,0E0,1.8142664E-1,1.3761982E-1,2.691285E-1,5.346024E-2,6.370902E-2,2.1542957E-1,6.196274E-2,3.764221E-2,0E0,2.6114774E-1,0E0,0E0,0E0,0E0,1.7737329E-1,1.4809857E-1,0E0,0E0,0E0,0E0,2.36469E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,9,9,10,10,11,11,12,12,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,31,31,32,32,37,37],"right_children":[2,4,6,8,-1,-1,10,12,-1,14,16,18,20,-1,-1,-1,-1,22,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,40,42,-1,-1,-1,-1,44,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.3575526E8,4.083812E7,4.476758E6,-1.39254555E-2,1.8431708E-2,9.1481786E8,3.5540915E0,-1.16943745E-2,5.809724E6,1.4062E4,1.9002409E6,4.291834E0,2.0632336E-3,-8.141675E-3,9.6384046E-4,6.8541486E-3,4.855866E-5,2.0116584E8,6.87E2,4.653159E9,1E0,3.74E2,6.916487E4,1E0,-1.1195725E-2,9.49181E-1,-1.1193316E-2,-4.644699E-3,-6.1423234E-3,1.7241485E-4,7.24E2,9.55465E5,4.691314E-3,-1.6370359E-3,1.1603358E-2,4.7127362E-3,4.2687164E2,-1.0290403E-2,4.134209E-4,6.6231843E-3,-1.6897144E-3,5.966918E-3,-1.5485843E-3,5.6185517E-3],"split_indices":[2,44,44,27,0,0,7,56,0,1,9,27,55,0,0,0,0,37,7,2,5,8,0,31,80,0,26,0,0,0,0,2,9,0,0,0,0,51,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.58E2,2.24E2,3.4E1,2.17E2,7E0,9E0,2.5E1,2.09E2,8E0,1.2E1,1.3E1,1.13E2,9.6E1,7E0,5E0,6E0,7E0,9.1E1,2.2E1,8.2E1,1.4E1,1.3E1,7.8E1,1.1E1,1.1E1,5E0,7.7E1,6E0,8E0,8E0,5E0,3.8E1,4E1,6E0,5E0,6E0,5E0,7.2E1,5E0,1.6E1,2.2E1,3.5E1,5E0,6.2E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.3645057E-3,2.3109589E-2,-1.14086695E-1,1.4443128E-2,1.4167706E-2,-4.5108505E-2,-1.9072758E-2,-4.3497987E-2,5.0792076E-2,5.0624018E-3,-1.0880233E-1,-2.2932634E-2,-1.3071703E-2,1.389724E-1,1.9904304E-2,-9.807948E-3,-4.085511E-2,2.438312E-2,-8.5796535E-2,9.061612E-2,1.0655069E-2,-7.655822E-2,6.0526222E-2,-6.1561055E-3,1.5041094E-2,-6.511357E-2,5.3476468E-2,-1.2792079E-1,-4.929371E-3,1.4028372E-1,-1.0063125E-4,-9.616188E-3,-2.0837378E-2,-8.036546E-2,8.712055E-2,-2.0301228E-3,3.2735583E-3,-5.138223E-3,-2.446651E-4,5.311035E-3,2.8187629E-2,-1.8113546E-1,-6.3076966E-2,3.2294416E-3,-3.1491385E-3,3.5899251E-3,7.7506965E-3,-6.3288338E-3,4.8945565E-2,-6.5064486E-3,-1.3730148E-4,9.462005E-3,1.5909705E-1,-1.5116412E-3,3.565416E-3,-1.079599E-2,-4.178338E-3,-5.682249E-3,4.6772536E-4,-2.9746105E-4,5.5960175E-3,1.9570214E-3,-4.443984E-3,9.8967245E-3,1.0113036E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,7,9,-1,11,13,-1,15,17,-1,19,21,-1,23,25,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,-1,47,49,51,-1,-1,-1,-1,-1,53,55,57,-1,-1,-1,-1,-1,59,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1770797E-1,5.4555523E-1,8.118667E-1,0E0,4.2882723E-1,3.430031E-1,0E0,3.9365226E-1,3.3497187E-1,0E0,1.6838402E-1,2.2220151E-1,0E0,1.2918437E-1,3.690536E-1,0E0,8.761237E-2,1.1544365E-1,1.0837643E-1,1.0637818E-1,0E0,2.0882936E-1,2.542981E-1,0E0,4.129483E-2,3.144535E-2,5.0344534E-2,6.1038077E-2,6.462469E-2,1.5075058E-2,0E0,0E0,1.8201262E-1,5.3931646E-2,3.1151173E-1,0E0,0E0,0E0,0E0,0E0,8.114947E-2,3.4537733E-2,5.2156553E-2,0E0,0E0,0E0,0E0,0E0,5.925922E-2,0E0,0E0,1.0692108E-1,2.3438889E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,32,32,33,33,34,34,40,40,41,41,42,42,48,48,51,51,52,52],"right_children":[2,4,6,-1,8,10,-1,12,14,-1,16,18,-1,20,22,-1,24,26,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,-1,48,50,52,-1,-1,-1,-1,-1,54,56,58,-1,-1,-1,-1,-1,60,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,6.747104E4,1.3E1,1.4443128E-2,1.0926206E2,2.2193549E0,-1.9072758E-2,5.3159845E6,1.02848E5,5.0624018E-3,1.2114334E2,7.9716E4,-1.3071703E-2,4.0480963E2,3.1236285E2,-9.807948E-3,5.9378143E-2,1.4221454E0,2.2E1,1.5182E4,1.0655069E-2,1.3501E4,2.7784336E-1,-6.1561055E-3,1.569E3,2.894E3,3.823572E-4,6.930967E8,6.3042126E0,2.758609E2,-1.0063125E-4,-9.616188E-3,1.282E3,2.5522516E0,1.4041E4,-2.0301228E-3,3.2735583E-3,-5.138223E-3,-2.446651E-4,5.311035E-3,7.99E2,8E0,7.7863485E6,3.2294416E-3,-3.1491385E-3,3.5899251E-3,7.7506965E-3,-6.3288338E-3,1.978142E0,-6.5064486E-3,-1.3730148E-4,4.3413E4,6.627576E7,-1.5116412E-3,3.565416E-3,-1.079599E-2,-4.178338E-3,-5.682249E-3,4.6772536E-4,-2.9746105E-4,5.5960175E-3,1.9570214E-3,-4.443984E-3,9.8967245E-3,1.0113036E-3],"split_indices":[9,27,3,0,51,57,0,27,28,0,51,1,0,4,51,0,37,52,8,9,0,9,26,0,0,11,37,5,56,4,0,0,10,53,9,0,0,0,0,0,2,3,44,0,0,0,0,0,53,0,0,10,44,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,2.06E2,3.8E1,5E0,2.01E2,3.2E1,6E0,7.8E1,1.23E2,9E0,2.3E1,7.3E1,5E0,3.1E1,9.2E1,8E0,1.5E1,4.2E1,3.1E1,2.2E1,9E0,2.7E1,6.5E1,5E0,1E1,1E1,3.2E1,2E1,1.1E1,1.4E1,8E0,7E0,2E1,1E1,5.5E1,5E0,5E0,5E0,5E0,8E0,2.4E1,1E1,1E1,5E0,6E0,6E0,8E0,7E0,1.3E1,5E0,5E0,2.7E1,2.8E1,1.1E1,1.3E1,5E0,5E0,5E0,5E0,8E0,5E0,2.1E1,6E0,1.9E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[1.2433085E-2,-1.0432522E-2,1.3245119E-1,-3.4019053E-2,6.7072965E-2,2.38113E-1,1.4257132E-2,-2.3314144E-1,-1.857207E-2,-2.6874965E-2,1.0378468E-1,2.1305252E-2,1.03920974E-1,5.85508E-2,-4.124636E-3,-4.435905E-3,-1.4837717E-2,6.1370675E-2,-6.0451705E-2,-3.663474E-3,2.6675514E-3,1.2659383E-1,6.2365894E-4,7.1046655E-3,2.7708572E-4,3.8515183E-3,3.8316697E-4,2.4545588E-2,1.2809615E-2,-1.2105089E-1,-1.970652E-2,5.4577366E-2,1.7269406E-1,9.312023E-2,-3.129595E-2,1.0463907E-3,-1.4882414E-1,7.702244E-2,-5.8389515E-2,5.115733E-3,-1.0463168E-3,1.0944528E-2,4.798307E-3,1.2871517E-1,-5.134727E-4,-1.0709369E-1,3.5908967E-2,-6.7813076E-2,-2.2559687E-1,4.892198E-4,6.071117E-3,1.2187439E-2,-1.172221E-1,2.9213517E-3,6.953284E-3,-8.179483E-3,-1.2674741E-3,3.7936964E-3,-1.0972169E-3,-5.257219E-3,8.3514204E-4,-1.3595137E-2,-5.0508375E-3,3.3440904E-3,-2.0408828E-3,-1.0279739E-2,-1.9370719E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,27,29,-1,-1,31,-1,-1,-1,-1,-1,33,-1,35,37,39,41,43,45,-1,47,49,51,-1,-1,-1,-1,53,-1,55,57,59,61,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5177786E-1,3.6789888E-1,4.711718E-1,4.6934938E-1,1.6489172E-1,5.799055E-1,9.43343E-2,1.0981196E-1,4.858181E-1,7.029115E-2,6.800941E-2,0E0,7.2454095E-2,1.8098596E-2,0E0,0E0,0E0,4.030833E-1,2.3218143E-1,0E0,0E0,8.021417E-2,0E0,0E0,0E0,0E0,0E0,1.714041E-1,0E0,1.5569931E-1,2.2151972E-1,5.7582963E-2,4.843089E-2,7.5864375E-2,1.3119516E-1,0E0,1.8372375E-1,6.3353E-2,1.7582208E-1,0E0,0E0,0E0,0E0,1.4449343E-2,0E0,6.532504E-2,4.271747E-2,7.374491E-2,1.04165554E-1,0E0,0E0,7.456112E-2,1.7937446E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,17,17,18,18,21,21,27,27,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,43,43,45,45,46,46,47,47,48,48,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,28,30,-1,-1,32,-1,-1,-1,-1,-1,34,-1,36,38,40,42,44,46,-1,48,50,52,-1,-1,-1,-1,54,-1,56,58,60,62,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.723729E2,2.421233E0,1E1,1.9898948E-6,3.7384987E-1,1E0,8.493112E-1,3.1E2,1.262303E7,1E0,2.4767955E-1,2.1305252E-2,7.069E3,4.9308786E-1,-4.124636E-3,-4.435905E-3,-1.4837717E-2,1.491E3,2.6477316E-1,-3.663474E-3,2.6675514E-3,1.3222336E-3,6.2365894E-4,7.1046655E-3,2.7708572E-4,3.8515183E-3,3.8316697E-4,6.4781194E5,1.2809615E-2,1.0989007E5,1.3257E4,2.4772727E0,2.6486957E0,4.45E2,1.06E2,1.0463907E-3,1.8271865E0,1.2293E4,1E0,5.115733E-3,-1.0463168E-3,1.0944528E-2,4.798307E-3,7.24E2,-5.134727E-4,2.71E2,1.5539E4,1.343E4,8.87885E5,4.892198E-4,6.071117E-3,3.2033653E5,3.38061E5,2.9213517E-3,6.953284E-3,-8.179483E-3,-1.2674741E-3,3.7936964E-3,-1.0972169E-3,-5.257219E-3,8.3514204E-4,-1.3595137E-2,-5.0508375E-3,3.3440904E-3,-2.0408828E-3,-1.0279739E-2,-1.9370719E-3],"split_indices":[51,52,3,37,26,63,26,0,12,108,38,0,2,41,0,0,0,2,40,0,0,41,0,0,0,0,0,27,0,46,9,52,53,10,10,0,53,9,80,0,0,0,0,2,0,0,9,10,1,0,0,32,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.99E2,3.7E1,1.53E2,4.6E1,1.9E1,1.8E1,1E1,1.43E2,1.3E1,3.3E1,6E0,1.3E1,1.3E1,5E0,5E0,5E0,4.9E1,9.4E1,8E0,5E0,2.6E1,7E0,8E0,5E0,8E0,5E0,4.3E1,6E0,3.7E1,5.7E1,1.1E1,1.5E1,1.9E1,2.4E1,6E0,3.1E1,1.6E1,4.1E1,6E0,5E0,6E0,9E0,1.4E1,5E0,1.1E1,1.3E1,1.6E1,1.5E1,8E0,8E0,1.9E1,2.2E1,5E0,9E0,5E0,6E0,7E0,6E0,1E1,6E0,8E0,7E0,9E0,1E1,8E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-5.6029614E-3,1.4672604E-2,-1.2593767E-1,2.044438E-1,3.0384574E-3,-4.476175E-2,-2.3034396E-2,1.484744E-2,3.0773524E-3,-6.2628806E-2,3.8139254E-2,4.6042506E-2,-1.2709314E-1,-4.0489882E-2,-1.300188E-2,1.7265314E-2,1.3887772E-1,-2.2499468E-3,5.210229E-3,-8.684484E-3,-1.6917987E-3,8.502645E-3,-1.3440755E-1,3.954147E-3,9.413539E-3,1.3742951E-3,1.7762302E-1,-1.3783681E-2,4.2489106E-3,-4.8516E-2,-1.1412071E-2,3.3504017E-2,-7.158532E-2,1.0420758E-2,3.420473E-3,2.8667882E-2,-7.229696E-2,-5.0830944E-3,1.4338542E-3,6.442282E-2,-4.2505115E-2,-1.1752104E-1,-6.008962E-3,-2.8266835E-3,3.5542988E-3,-5.047006E-3,-1.0655622E-3,3.9987625E-3,-1.6348343E-3,-4.064654E-3,-3.9889256E-4,-8.4463265E-3,-7.030922E-4,-2.1875086E-3,1.684301E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,29,31,-1,-1,33,35,-1,37,-1,39,41,-1,-1,43,45,-1,-1,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9229255E-1,4.586066E-1,1.0706673E0,1.815384E-1,4.5646837E-1,2.3013288E-1,0E0,0E0,0E0,3.3311722E-1,2.6886457E-1,1.05786555E-1,8.664882E-2,2.956237E-1,0E0,2.7447066E-1,8.69866E-2,0E0,0E0,0E0,0E0,8.344943E-2,2.1197125E-1,2.3008089E-1,0E0,0E0,6.954342E-2,8.901191E-2,0E0,7.578263E-2,0E0,1.7664659E-1,8.665043E-2,0E0,0E0,1.0116044E-1,2.628415E-2,0E0,0E0,1.326472E-1,3.4476675E-2,1.18012264E-1,2.5770716E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,15,15,16,16,21,21,22,22,23,23,26,26,27,27,29,29,31,31,32,32,35,35,36,36,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,30,32,-1,-1,34,36,-1,38,-1,40,42,-1,-1,44,46,-1,-1,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,1.3835684E5,1.9E1,1.1280869E9,1.06199684E2,2.5979605E0,-2.3034396E-2,1.484744E-2,3.0773524E-3,4.674595E6,1E0,3.6437822E5,1.4456801E6,6.072E3,-1.300188E-2,1.3717E4,2.2913464E2,-2.2499468E-3,5.210229E-3,-8.684484E-3,-1.6917987E-3,1E0,1.8558352E0,4.7564573E0,9.413539E-3,1.3742951E-3,2.6786362E8,2.7958122E8,4.2489106E-3,7.112949E-1,-1.1412071E-2,3.7861453E5,6.6774924E7,1.0420758E-2,3.420473E-3,1.5E0,4.717241E10,-5.0830944E-3,1.4338542E-3,3.6025445E6,5.760101E0,1.6371E4,4.710155E2,-2.8266835E-3,3.5542988E-3,-5.047006E-3,-1.0655622E-3,3.9987625E-3,-1.6348343E-3,-4.064654E-3,-3.9889256E-4,-8.4463265E-3,-7.030922E-4,-2.1875086E-3,1.684301E-3],"split_indices":[9,27,3,30,51,55,0,0,0,27,112,32,46,10,0,2,51,0,0,0,0,8,52,55,0,0,7,5,0,26,0,32,44,0,0,52,30,0,0,46,56,9,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.07E2,3.4E1,1.1E1,1.96E2,2.9E1,5E0,5E0,6E0,6.8E1,1.28E2,1.4E1,1.5E1,6.3E1,5E0,1.07E2,2.1E1,6E0,8E0,8E0,7E0,4.2E1,2.1E1,1.01E2,6E0,6E0,1.5E1,3.4E1,8E0,1.3E1,8E0,7.3E1,2.8E1,9E0,6E0,2E1,1.4E1,7E0,6E0,5.2E1,2.1E1,1.6E1,1.2E1,7E0,1.3E1,7E0,7E0,4.2E1,1E1,8E0,1.3E1,9E0,7E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.1937552E-2,-9.133552E-3,1.4283936E-1,4.2133372E-2,-4.833697E-2,3.2759446E-1,4.069559E-3,-7.003491E-2,5.7655822E-2,-2.1825977E-1,-2.8182212E-2,2.6223226E-2,6.4240494E-3,6.9168165E-2,-6.1329505E-3,-7.5814733E-4,-5.431566E-3,7.773457E-2,-4.8290353E-2,-4.8420273E-4,-1.562983E-2,6.2138055E-2,-4.9708188E-2,6.557134E-3,7.441474E-4,5.954805E-2,9.627651E-3,-7.3897303E-3,4.090359E-3,6.4747515E-3,5.7847355E-3,-1.7887156E-1,-2.9712329E-2,9.447352E-2,-2.597458E-3,-2.0925268E-3,3.4226188E-3,-1.2696828E-2,-2.9439714E-3,-2.5585308E-3,-1.2309994E-1,4.6264965E-2,1.4785895E-1,4.4644587E-3,-4.1966043E-2,-3.0556172E-2,9.76158E-2,-8.607557E-3,-7.329887E-2,5.2551557E-3,6.5860985E-4,7.76246E-3,3.000711E-3,-5.2558975E-3,-1.9437184E-4,2.114742E-3,-2.7948206E-3,1.8780874E-3,7.4535515E-3,-3.617507E-4,-6.1883875E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,-1,-1,25,27,-1,-1,29,31,-1,-1,33,-1,-1,-1,35,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,51,-1,53,55,57,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.963825E-1,4.4018143E-1,8.960118E-1,1.6944498E-1,4.185184E-1,6.2236E-1,2.0090452E-1,3.0175682E-2,1.8244416E-1,3.425622E-1,2.208427E-1,0E0,0E0,5.783444E-2,0E0,0E0,0E0,1.6260475E-1,2.3980515E-1,0E0,0E0,8.0174334E-2,2.2971395E-1,0E0,0E0,1.3915488E-1,0E0,0E0,0E0,5.124985E-2,0E0,1.21600956E-1,2.0255132E-1,9.867832E-2,1.0015561E-1,0E0,0E0,0E0,0E0,1.7967539E-1,4.677424E-2,4.9470905E-2,2.6066601E-2,0E0,5.0056368E-2,1.24602154E-1,4.6144053E-2,0E0,4.898379E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,17,17,18,18,21,21,22,22,25,25,29,29,31,31,32,32,33,33,34,34,39,39,40,40,41,41,42,42,44,44,45,45,46,46,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,-1,-1,26,28,-1,-1,30,32,-1,-1,34,-1,-1,-1,36,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,52,-1,54,56,58,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,2.5542571E8,9E0,9.23253E1,8.230099E4,6.234292E-1,1.0129377E12,7.281633E1,2.46604E5,1E0,7.65244E5,2.6223226E-2,6.4240494E-3,8.728938E0,-6.1329505E-3,-7.5814733E-4,-5.431566E-3,1.496E3,7.3155574E9,-4.8420273E-4,-1.562983E-2,7.238806E-1,1.3362592E-2,6.557134E-3,7.441474E-4,6.4596E4,9.627651E-3,-7.3897303E-3,4.090359E-3,4.710155E2,5.7847355E-3,1.3501E4,2.68225E5,2.0787193E2,1.2618296E0,-2.0925268E-3,3.4226188E-3,-1.2696828E-2,-2.9439714E-3,1.6018559E1,4.9196738E5,1.6714E4,2.0142368E8,4.4644587E-3,1.2282595E7,1.3257E4,4.9960384E8,-8.607557E-3,5.82516E5,5.2551557E-3,6.5860985E-4,7.76246E-3,3.000711E-3,-5.2558975E-3,-1.9437184E-4,2.114742E-3,-2.7948206E-3,1.8780874E-3,7.4535515E-3,-3.617507E-4,-6.1883875E-3],"split_indices":[2,7,3,4,31,26,30,4,28,63,47,0,0,56,0,0,0,2,5,0,0,56,37,0,0,28,0,0,0,51,0,9,9,4,55,0,0,0,0,56,27,1,7,0,44,9,7,0,1,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.17E2,3.4E1,9.4E1,1.23E2,1.4E1,2E1,1.1E1,8.3E1,1.2E1,1.11E2,5E0,9E0,1.4E1,6E0,6E0,5E0,7E1,1.3E1,5E0,7E0,2.1E1,9E1,5E0,9E0,6.3E1,7E0,7E0,6E0,1.2E1,9E0,1.1E1,7.9E1,4E1,2.3E1,7E0,5E0,5E0,6E0,6.2E1,1.7E1,2.2E1,1.8E1,6E0,1.7E1,4.9E1,1.3E1,6E0,1.1E1,6E0,1.6E1,1.3E1,5E0,5E0,1.2E1,1.4E1,3.5E1,8E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[7.453887E-3,-6.44516E-3,1.7305619E-1,-2.9338803E-2,5.6031402E-2,1.9102383E-2,4.4095427E-2,-1.345524E-2,-1.6028745E-1,7.689904E-2,-3.7879185E-3,4.215491E-3,6.947388E-5,-2.259566E-3,-1.0328348E-2,-9.225253E-3,-2.0280648E-3,1.3066868E-1,-6.137069E-3,3.6750574E-2,-3.9565563E-2,5.2768756E-2,1.708752E-1,5.6066066E-3,-5.4680984E-2,1.7228305E-3,1.4829382E-1,-8.70964E-2,1.9599067E-2,4.217837E-3,-1.6188003E-4,2.0821877E-1,3.236605E-3,-7.210912E-3,3.4676428E-4,-4.149423E-2,8.566029E-2,2.2109242E-3,1.8895589E-1,-2.0975411E-2,-1.5559347E-1,5.8847565E-2,-1.8493585E-2,5.7821395E-3,1.2169969E-2,1.0718303E-3,-4.1318373E-3,2.8241338E-4,6.1152596E-3,1.1865496E-2,3.7553413E-3,1.4023944E-3,-4.9369927E-3,-1.6848668E-3,-8.799721E-3,1.668406E-5,4.234037E-3,7.9950417E-4,-3.961777E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,19,-1,-1,-1,21,23,25,27,29,31,-1,33,35,37,39,41,-1,-1,43,-1,-1,-1,45,47,-1,49,51,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.35113E-1,3.0916184E-1,5.629717E-1,3.264724E-1,1.7594738E-1,0E0,2.7179295E-2,3.422304E-1,7.763681E-2,2.2850785E-1,0E0,0E0,0E0,1.9937651E-1,0E0,0E0,0E0,8.8077724E-2,1.3980618E-1,2.6152077E-1,1.9855708E-1,2.790812E-2,5.9068024E-2,0E0,1.0624619E-1,1.92186E-1,5.5314958E-2,1.7337102E-1,4.8930846E-2,0E0,0E0,3.445542E-2,0E0,0E0,0E0,1.1670154E-1,7.0545554E-2,0E0,6.431636E-2,1.00200206E-1,8.156371E-2,3.275905E-2,4.4273693E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,13,13,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,31,31,35,35,36,36,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,20,-1,-1,-1,22,24,26,28,30,32,-1,34,36,38,40,42,-1,-1,44,-1,-1,-1,46,48,-1,50,52,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,2.3900106E0,1E0,3.10364E5,1.472353E0,1.9102383E-2,7.946966E-1,4.966421E6,8.039736E5,2.9806678E0,-3.7879185E-3,4.215491E-3,6.947388E-5,7.07E2,-1.0328348E-2,-9.225253E-3,-2.0280648E-3,2.2694193E2,5.524659E8,2.501667E6,9.479827E0,3.908917E8,5.709408E7,5.6066066E-3,1.496E3,1E0,9.06E2,3.300363E5,9E0,4.217837E-3,-1.6188003E-4,1.2557897E7,3.236605E-3,-7.210912E-3,3.4676428E-4,4.459525E5,4.8749522E5,2.2109242E-3,2.5542571E8,5.709408E7,2.980289E-1,2.0822755E9,1.9231517E0,5.7821395E-3,1.2169969E-2,1.0718303E-3,-4.1318373E-3,2.8241338E-4,6.1152596E-3,1.1865496E-2,3.7553413E-3,1.4023944E-3,-4.9369927E-3,-1.6848668E-3,-8.799721E-3,1.668406E-5,4.234037E-3,7.9950417E-4,-3.961777E-3],"split_indices":[51,52,63,9,40,0,26,27,27,55,0,0,0,0,0,0,0,4,5,31,56,5,44,0,2,80,2,32,3,0,0,44,0,0,0,27,47,0,7,44,26,5,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,2.14E2,1.7E1,1.57E2,5.7E1,5E0,1.2E1,1.41E2,1.6E1,5E1,7E0,5E0,7E0,1.35E2,6E0,1.1E1,5E0,3E1,2E1,6.6E1,6.9E1,1.1E1,1.9E1,5E0,1.5E1,5.1E1,1.5E1,3.8E1,3.1E1,6E0,5E0,1.3E1,6E0,5E0,1E1,3.4E1,1.7E1,5E0,1E1,2E1,1.8E1,1.5E1,1.6E1,7E0,6E0,1.5E1,1.9E1,7E0,1E1,5E0,5E0,1.3E1,7E0,5E0,1.3E1,6E0,9E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.2507507E-3,1.2934335E-2,-1.7034183E-1,-8.625857E-3,1.6497368E-1,3.7468534E-2,-3.6266658E-1,3.6218677E-2,-5.3380538E-2,1.4221139E-2,7.2250284E-2,4.5771003E-3,-1.4796059E-3,-5.5690855E-3,-2.441228E-2,1.131962E-2,1.8124747E-1,-1.6689068E-1,-3.0299764E-2,3.4306434E-4,5.5555194E-3,3.22195E-2,-9.929152E-3,1.5633231E-3,1.4559076E-2,-1.0435669E-2,-9.359008E-4,-1.0473738E-2,-1.3319945E-1,6.777604E-2,-1.9899882E-2,1.4045561E-2,-8.630016E-2,-2.94834E-3,-7.5489343E-3,-3.2348717E-3,1.0951467E-1,1.7007956E-2,-6.9506764E-3,-8.063925E-2,3.7940618E-2,-1.4382233E-1,1.9447202E-3,2.9330645E-3,-3.5506964E-3,7.7064373E-3,1.8408822E-3,-4.1332124E-3,2.1304477E-3,-5.0506135E-4,-6.161323E-3,-4.3506436E-3,2.6228826E-3,-8.107432E-3,-3.511889E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,-1,21,23,25,27,-1,-1,29,-1,-1,-1,-1,-1,31,33,35,37,39,41,-1,-1,43,45,47,-1,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4314216E-1,7.1274847E-1,8.528973E-1,3.8527086E-1,3.537568E-1,5.2837044E-2,4.033059E-1,3.44612E-1,2.4645385E-1,0E0,5.834955E-2,0E0,0E0,0E0,0E0,4.087027E-1,2.7943447E-1,1.4609817E-1,1.6385233E-1,0E0,0E0,1.4404397E-1,0E0,0E0,0E0,0E0,0E0,1.2949768E-1,1.9214332E-2,1.3669357E-1,1.6116995E-1,1.22971945E-1,1.3276707E-1,0E0,0E0,9.792511E-2,1.1531013E-1,8.976618E-2,0E0,4.124283E-2,1.2252487E-1,1.4462635E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,15,15,16,16,17,17,18,18,21,21,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,39,39,40,40,41,41],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,-1,22,24,26,28,-1,-1,30,-1,-1,-1,-1,-1,32,34,36,38,40,42,-1,-1,44,46,48,-1,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.00068504E8,7.737766E2,8.493112E-1,2.459517E0,1E0,3.660755E-1,1.0232127E12,1.993E3,1.4065148E6,1.4221139E-2,6.0004652E7,4.5771003E-3,-1.4796059E-3,-5.5690855E-3,-2.441228E-2,4.966421E6,2.0519513E-1,8.294758E-1,5.217E4,3.4306434E-4,5.5555194E-3,6.063759E8,-9.929152E-3,1.5633231E-3,1.4559076E-2,-1.0435669E-2,-9.359008E-4,2.68225E5,5.928644E5,7.23E2,3.9128714E0,3.677233E5,2.2541766E0,-2.94834E-3,-7.5489343E-3,1E0,4.031725E6,6.2652588E1,-6.9506764E-3,1.84052E10,8.47E2,9.637789E5,1.9447202E-3,2.9330645E-3,-3.5506964E-3,7.7064373E-3,1.8408822E-3,-4.1332124E-3,2.1304477E-3,-5.0506135E-4,-6.161323E-3,-4.3506436E-3,2.6228826E-3,-8.107432E-3,-3.511889E-3],"split_indices":[44,51,26,57,63,26,30,2,49,0,44,0,0,0,0,27,26,26,10,0,0,5,0,0,0,0,0,9,46,2,55,27,52,0,0,111,12,51,0,30,2,46,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,2.16E2,2E1,1.9E2,2.6E1,1E1,1E1,9.5E1,9.5E1,9E0,1.7E1,5E0,5E0,5E0,5E0,8.2E1,1.3E1,1.5E1,8E1,8E0,9E0,7.6E1,6E0,7E0,6E0,1E1,5E0,6.8E1,1.2E1,4.5E1,3.1E1,5.2E1,1.6E1,5E0,7E0,1.7E1,2.8E1,2.5E1,6E0,1E1,4.2E1,1.1E1,5E0,9E0,8E0,1.4E1,1.4E1,5E0,2E1,5E0,5E0,5E0,3.7E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.6109952E-4,-1.5919607E-2,1.397809E-1,-2.5378396E-3,-2.3086205E-1,1.6492663E-2,1.1926002E-2,5.7261656E-3,-9.555949E-3,-2.274584E-2,1.4296571E-4,3.1614613E-3,-2.8090635E-3,5.5354822E-2,-2.5041781E-2,6.986838E-2,-2.8714302E-3,-4.1859932E-2,8.525459E-2,4.368739E-2,1.4892162E-1,-7.718456E-3,-2.8642487E-2,-6.920874E-4,1.2845145E-1,6.0037144E-2,-4.3792226E-3,8.109355E-3,3.5651163E-3,-1.4140648E-2,-7.1784756E-3,2.0607682E-3,8.163065E-3,-1.5608493E-4,3.54017E-3,7.616566E-5,-5.9007225E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,15,17,19,-1,21,23,25,27,-1,29,-1,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4472345E-1,6.3795495E-1,7.179303E-1,3.6458343E-1,8.339042E-1,0E0,7.778365E-2,3.1136146E-1,0E0,0E0,0E0,0E0,0E0,1.3920219E-1,2.3729637E-1,1.3837248E-1,0E0,1.8310384E-1,7.584885E-2,1.3014483E-1,2.2499025E-2,0E0,1.8896747E-1,0E0,4.3937564E-2,5.7746947E-2,0E0,0E0,0E0,1.7020199E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22,24,24,25,25,29,29],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,16,18,20,-1,22,24,26,28,-1,30,-1,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,1.0388931E8,5.939257E7,4.966421E6,2.2862453E0,1.6492663E-2,1.0881593E10,1.411682E7,-9.555949E-3,-2.274584E-2,1.4296571E-4,3.1614613E-3,-2.8090635E-3,8.385435E0,1.8656379E6,2.2913464E2,-2.8714302E-3,8.180048E5,4.3326885E-1,8.5057585E10,1.1034263E10,-7.718456E-3,5.08616E5,-6.920874E-4,3.727302E6,4.75E2,-4.3792226E-3,8.109355E-3,3.5651163E-3,4.0312E4,-7.1784756E-3,2.0607682E-3,8.163065E-3,-1.5608493E-4,3.54017E-3,7.616566E-5,-5.9007225E-3],"split_indices":[2,44,44,27,52,0,12,12,0,0,0,0,0,55,27,51,0,44,55,30,5,0,32,0,50,2,0,0,0,10,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,2.21E2,2.4E1,2.09E2,1.2E1,8E0,1.6E1,2.02E2,7E0,5E0,7E0,9E0,7E0,7.7E1,1.25E2,6.9E1,8E0,1.09E2,1.6E1,5.3E1,1.6E1,9E0,1E2,5E0,1.1E1,4.8E1,5E0,1E1,6E0,9.1E1,9E0,5E0,6E0,1.1E1,3.7E1,8.1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4816744E-2,-2.3565311E-2,8.550929E-3,-4.6543158E-3,-1.2987421E-1,-2.3349924E-2,5.5907108E-2,-3.1774202E-1,-4.418615E-3,-1.1033524E-2,-1.4156893E-2,9.571456E-2,-1.9971615E-2,-4.6377447E-3,-2.6474727E-2,3.8283344E-2,-4.5719924E-3,5.7334783E-3,-1.2164122E-1,3.281434E-2,1.455845E-1,-7.494104E-2,4.2069242E-3,-4.4258533E-4,5.4148305E-3,1.4919405E-2,-5.1549803E-3,-1.5093522E-1,-6.081017E-4,-1.3636178E-4,3.926455E-3,8.229857E-3,3.0103065E-3,-4.874079E-3,-1.0721029E-3,-1.753598E-2,6.5394804E-2,-8.70687E-3,-2.5751223E-3,1.7182612E-4,-4.7907755E-3,3.9506494E-3,-2.3038017E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,-1,5,7,9,11,13,15,-1,17,19,21,-1,-1,23,-1,25,27,29,31,33,-1,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1],"loss_changes":[4.355287E-1,4.6581E-1,0E0,2.2548749E-1,8.235133E-1,3.0582002E-1,1.438769E-1,7.5004387E-1,9.498247E-2,0E0,3.149692E-1,9.246448E-2,1.12837896E-1,0E0,0E0,6.41298E-2,0E0,1.3955663E-1,7.083556E-2,2.9433059E-2,3.7299156E-2,1.7783672E-2,0E0,0E0,0E0,1.9281599E-1,0E0,6.016794E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3674423E-1,1.235261E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,25,25,27,27,35,35,36,36],"right_children":[2,4,-1,6,8,10,12,14,16,-1,18,20,22,-1,-1,24,-1,26,28,30,32,34,-1,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0086E4,7.3365464E7,8.550929E-3,2.421233E0,9.4014386E2,1.9898948E-6,8.5913794E5,1.2714615E6,2.4269E4,-1.1033524E-2,2.1159E4,4.0972165E2,7.05571E5,-4.6377447E-3,-2.6474727E-2,1E0,-4.5719924E-3,4.32016E5,1.5452969E6,2.8994E4,1.43626E6,2.9656984E7,4.2069242E-3,-4.4258533E-4,5.4148305E-3,1.281E3,-5.1549803E-3,1.4852E4,-6.081017E-4,-1.3636178E-4,3.926455E-3,8.229857E-3,3.0103065E-3,-4.874079E-3,-1.0721029E-3,3.3897146E8,4.213523E0,-8.70687E-3,-2.5751223E-3,1.7182612E-4,-4.7907755E-3,3.9506494E-3,-2.3038017E-3],"split_indices":[2,44,0,52,4,37,31,27,10,0,9,4,9,0,0,15,0,11,27,9,1,12,0,0,0,2,0,10,0,0,0,0,0,0,0,7,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,2.31E2,9E0,1.97E2,3.4E1,1.51E2,4.6E1,1.3E1,2.1E1,5E0,1.46E2,3E1,1.6E1,8E0,5E0,1.5E1,6E0,1.24E2,2.2E1,1.4E1,1.6E1,1.1E1,5E0,1E1,5E0,1.16E2,8E0,1.7E1,5E0,9E0,5E0,1E1,6E0,6E0,5E0,7.1E1,4.5E1,1.1E1,6E0,5.8E1,1.3E1,3.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.0168829E-2,9.285844E-3,-1.195891E-1,-2.7290273E-3,2.1924804E-1,-1.7785098E-2,-1.3372354E-2,-2.9872485E-2,5.7936843E-2,1.670563E-2,1.419434E-3,1.5067345E-2,-4.3640835E-3,1.2400608E-2,-7.361447E-2,8.317885E-2,-4.254155E-3,-2.0255318E-3,4.8269406E-2,-1.5737692E-2,9.9396445E-2,-1.7222978E-1,-3.8612973E-2,9.884374E-2,-1.4269268E-3,3.6742964E-3,5.9400336E-4,2.2155266E-2,-1.3403566E-1,1.4653413E-1,-5.160821E-4,-3.2391958E-3,-1.0611246E-2,-8.098834E-3,-7.2084395E-3,6.4060904E-2,1.6645755E-1,-3.9644814E-3,5.2066273E-3,-9.471174E-3,-1.7533536E-3,3.566534E-3,8.27022E-3,8.899159E-3,-4.903201E-3,1.16716176E-1,1.0734709E-2,2.7955896E-3,1.0335576E-2,2.1016125E-3,-1.5819188E-3,-2.163618E-3,1.9450809E-3,6.8640234E-3,2.2239694E-3,1.6379722E-3,-1.3983997E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,23,-1,-1,25,27,29,31,33,35,-1,-1,-1,37,39,41,-1,-1,-1,43,-1,45,47,49,-1,-1,-1,-1,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.0774956E-1,5.127001E-1,1.0456073E0,3.1954023E-1,3.0069488E-1,0E0,6.5505475E-2,2.4869344E-1,2.3847862E-1,0E0,0E0,4.39938E-2,0E0,1.704704E-1,2.211608E-1,9.666011E-2,0E0,0E0,1.5246514E-2,2.4055056E-1,9.049885E-2,9.07774E-2,1.8369222E-1,9.75194E-2,0E0,0E0,0E0,1.003851E-1,8.584806E-2,1.5333712E-2,0E0,0E0,0E0,7.1104184E-2,0E0,8.8600054E-2,8.411592E-2,5.3565353E-2,0E0,0E0,0E0,0E0,0E0,7.217629E-2,0E0,3.0779049E-2,1.9287668E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,15,15,18,18,19,19,20,20,21,21,22,22,23,23,27,27,28,28,29,29,33,33,35,35,36,36,37,37,43,43,45,45,46,46],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,24,-1,-1,26,28,30,32,34,36,-1,-1,-1,38,40,42,-1,-1,-1,44,-1,46,48,50,-1,-1,-1,-1,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3365464E7,1.2568668E3,8.947112E2,2.183847E0,3.6007138E2,-1.7785098E-2,3.945346E1,7.11E2,1.472353E0,1.670563E-2,1.419434E-3,6.2026776E2,-4.3640835E-3,1.6947379E2,1.06199684E2,1.3E1,-4.254155E-3,-2.0255318E-3,2.4274172E-2,4.7031E4,7.785208E2,4.527094E6,5.2726665E0,2.035E3,-1.4269268E-3,3.6742964E-3,5.9400336E-4,1.191E3,1.6601E4,5.56E2,-5.160821E-4,-3.2391958E-3,-1.0611246E-2,5.5180666E8,-7.2084395E-3,7.0918E4,3.9553946E-1,2.956215E5,5.2066273E-3,-9.471174E-3,-1.7533536E-3,3.566534E-3,8.27022E-3,1.8425703E2,-4.903201E-3,1.6256208E6,2.3005404E0,2.7955896E-3,1.0335576E-2,2.1016125E-3,-1.5819188E-3,-2.163618E-3,1.9450809E-3,6.8640234E-3,2.2239694E-3,1.6379722E-3,-1.3983997E-3],"split_indices":[44,51,4,53,57,0,56,0,40,0,0,51,0,51,51,3,0,0,37,28,4,44,55,2,0,0,0,2,9,0,0,0,0,7,0,28,26,27,0,0,0,0,0,51,0,27,55,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,2.02E2,3.5E1,1.92E2,1E1,9E0,2.6E1,1.33E2,5.9E1,5E0,5E0,2E1,6E0,6.8E1,6.5E1,5.1E1,8E0,7E0,1.3E1,5.2E1,1.6E1,1.6E1,4.9E1,4.5E1,6E0,6E0,7E0,4E1,1.2E1,1.1E1,5E0,7E0,9E0,4E1,9E0,3.1E1,1.4E1,3.2E1,8E0,6E0,6E0,5E0,6E0,3.5E1,5E0,1.5E1,1.6E1,6E0,8E0,1.2E1,2E1,1.3E1,2.2E1,9E0,6E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[7.6448144E-3,-9.1022225E-3,1.2576543E-1,3.1572092E-2,-5.653113E-2,1.5554419E-2,4.0346716E-2,-6.794609E-2,4.782958E-2,-3.644752E-2,-1.2192258E-2,-1.2215876E-2,6.9769504E-3,3.8848724E-3,-8.9000575E-3,6.314565E-2,-1.7564299E-2,-1.0286054E-1,-3.7404394E-3,-3.171909E-3,2.4667135E-3,7.6156825E-2,-7.768388E-3,-4.436391E-3,3.7779633E-2,7.791237E-4,-1.4945054E-1,2.3096573E-2,-6.764328E-2,3.8809709E-3,8.899392E-2,-5.1361234E-3,3.2828639E-3,5.246363E-3,-2.713616E-3,-1.2922805E-3,-1.8540688E-1,2.8145378E-3,5.0268294E-3,-1.7595045E-4,-5.9048817E-3,3.2369017E-3,-2.916065E-3,1.12040244E-1,3.8217384E-2,-1.0508637E-2,-3.0286263E-3,3.3427775E-2,-3.612915E-3,3.3024661E-3,6.628025E-3,-7.219179E-4,3.5672863E-3,2.5287035E-3,1.9899922E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,-1,-1,-1,21,23,25,27,-1,-1,29,31,-1,33,-1,35,37,39,41,43,-1,-1,-1,-1,-1,45,47,-1,-1,-1,-1,-1,49,51,-1,-1,53,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.620584E-1,3.9725816E-1,5.290245E-1,1.8283844E-1,3.981651E-1,0E0,1.3415565E-1,3.361222E-1,9.7260565E-2,1.9076672E-1,0E0,6.608497E-2,0E0,0E0,0E0,7.256022E-2,8.870234E-2,1.6490322E-1,1.0446E-1,0E0,0E0,6.0890198E-2,1.19460516E-1,0E0,9.935826E-2,0E0,8.5894465E-2,7.6766685E-2,7.212469E-2,5.5844214E-2,6.125E-2,0E0,0E0,0E0,0E0,0E0,7.447672E-2,9.408146E-2,0E0,0E0,0E0,0E0,0E0,4.076487E-2,4.3399647E-2,0E0,0E0,2.0148663E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,15,15,16,16,17,17,18,18,21,21,22,22,24,24,26,26,27,27,28,28,29,29,30,30,36,36,37,37,43,43,44,44,47,47],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,-1,-1,-1,22,24,26,28,-1,-1,30,32,-1,34,-1,36,38,40,42,44,-1,-1,-1,-1,-1,46,48,-1,-1,-1,-1,-1,50,52,-1,-1,54,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.013E3,2.9722316E0,2.13281E5,1.4827905E2,7.960309E7,1.5554419E-2,2.7584056E5,3.2590533E-6,3.3897146E8,1.0469056E2,-1.2192258E-2,9.248443E8,6.9769504E-3,3.8848724E-3,-8.9000575E-3,7.42E2,1.35767E5,1.6428572E0,3.34486E5,-3.171909E-3,2.4667135E-3,4.75E2,2.1127937E0,-4.436391E-3,9.5E2,7.791237E-4,2.8893906E-1,1E0,2.0957246E0,1E0,3.4577703E0,-5.1361234E-3,3.2828639E-3,5.246363E-3,-2.713616E-3,-1.2922805E-3,1.4852E4,7.4456794E5,5.0268294E-3,-1.7595045E-4,-5.9048817E-3,3.2369017E-3,-2.916065E-3,8.3599795E3,1.043E5,-1.0508637E-2,-3.0286263E-3,1.2007376E0,-3.612915E-3,3.3024661E-3,6.628025E-3,-7.219179E-4,3.5672863E-3,2.5287035E-3,1.9899922E-5],"split_indices":[2,56,28,46,44,0,32,41,7,51,0,7,0,0,0,0,1,52,28,0,0,2,52,0,0,0,55,15,52,111,55,0,0,0,0,0,10,31,0,0,0,0,0,46,28,0,0,55,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,2.04E2,2.8E1,1.1E2,9.4E1,7E0,2.1E1,1.5E1,9.5E1,8.7E1,7E0,1.5E1,6E0,7E0,8E0,7.7E1,1.8E1,2.8E1,5.9E1,8E0,7E0,6.5E1,1.2E1,7E0,1.1E1,8E0,2E1,4.2E1,1.7E1,1E1,5.5E1,5E0,7E0,6E0,5E0,5E0,1.5E1,3.5E1,7E0,9E0,8E0,5E0,5E0,3.7E1,1.8E1,1E1,5E0,2.6E1,9E0,1.9E1,1.8E1,8E0,1E1,1.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-7.588025E-5,-1.9513115E-2,1.17727846E-1,3.3290633E-3,-1.4414722E-1,2.6325643E-1,3.2997355E-2,-1.5212038E-2,8.376294E-2,-6.4474314E-2,-2.0984694E-2,9.064948E-4,1.9573014E-2,-2.6465226E-3,5.793855E-3,4.4939436E-2,-3.9969597E-2,1.2823275E-1,-1.2127087E-3,-1.0430381E-2,-1.3736258E-2,-2.8856283E-3,3.6633253E-2,-4.3897587E-3,8.8000685E-2,7.0714526E-2,-6.0576458E-2,1.6817318E-1,2.138154E-3,-6.435019E-2,3.812642E-2,-7.860189E-4,3.8144616E-3,1.2169522E-1,-5.2438973E-4,5.8140988E-3,5.00678E-4,-1.3949981E-1,-4.4160917E-2,3.2526115E-3,9.423407E-3,-5.8120827E-4,-4.7385716E-3,3.0731778E-3,7.870356E-5,7.01686E-3,2.6447522E-3,-1.9420454E-3,-1.0361255E-2,-5.907177E-2,2.3764754E-3,-8.238916E-4,-3.8845167E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,23,25,27,-1,-1,29,-1,31,-1,33,35,37,39,-1,41,43,-1,-1,45,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,51,-1,-1,-1],"loss_changes":[5.3352594E-1,5.698628E-1,3.982266E-1,2.5480074E-1,7.6337564E-1,4.8966748E-1,7.52211E-2,2.0879552E-1,1.609171E-1,2.1691287E-1,0E0,0E0,0E0,4.3309335E-2,0E0,2.5978142E-1,2.3008801E-1,6.6265225E-2,0E0,0E0,5.7577893E-2,0E0,2.9996306E-2,0E0,1.0883872E-1,5.410035E-2,1.0266739E-1,4.3713123E-2,0E0,2.1704849E-2,1.1897445E-2,0E0,0E0,4.218903E-2,0E0,0E0,0E0,1.1198428E-1,1.05659634E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.691362E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,20,20,22,22,24,24,25,25,26,26,27,27,29,29,30,30,33,33,37,37,38,38,49,49],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,24,26,28,-1,-1,30,-1,32,-1,34,36,38,40,-1,42,44,-1,-1,46,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,52,-1,-1,-1],"split_conditions":[5.246E3,6.5023036E7,4.083812E7,3.360145E2,3.5753E4,1.4781E4,2.7584056E5,3.08E2,4.213523E0,3.1236285E2,-2.0984694E-2,9.064948E-4,1.9573014E-2,9.1481786E8,5.793855E-3,4.75E2,1.3257E4,3.6E1,-1.2127087E-3,-1.0430381E-2,5.293399E2,-2.8856283E-3,2.3E1,-4.3897587E-3,1.7810288E5,2.88002E5,2.4256398E8,1.7848537E0,2.138154E-3,4.71118E5,1.307446E3,-7.860189E-4,3.8144616E-3,2.73E2,-5.2438973E-4,5.8140988E-3,5.00678E-4,1E0,4.0963454E0,3.2526115E-3,9.423407E-3,-5.8120827E-4,-4.7385716E-3,3.0731778E-3,7.870356E-5,7.01686E-3,2.6447522E-3,-1.9420454E-3,-1.0361255E-2,7.11E2,2.3764754E-3,-8.238916E-4,-3.8845167E-3],"split_indices":[2,44,44,51,9,9,32,0,55,51,0,0,0,7,0,2,9,8,0,0,51,0,8,0,46,1,5,53,0,28,4,0,0,0,0,0,0,8,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,1.99E2,3.2E1,1.69E2,3E1,1.1E1,2.1E1,1.38E2,3.1E1,2.5E1,5E0,5E0,6E0,1.6E1,5E0,4E1,9.8E1,2.2E1,9E0,5E0,2E1,6E0,1E1,9E0,3.1E1,1.5E1,8.3E1,1.4E1,8E0,1E1,1E1,5E0,5E0,2.3E1,8E0,7E0,8E0,1.3E1,7E1,5E0,9E0,5E0,5E0,5E0,5E0,1.4E1,9E0,7E0,6E0,6.1E1,9E0,2.5E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[8.883511E-3,2.7863357E-2,-7.271635E-2,8.232676E-3,1.9303745E-2,-1.7092153E-2,-1.5869197E-2,-2.1351142E-2,4.6233095E-2,-1.08273014E-1,4.020786E-2,-3.639383E-2,4.0243015E-2,-3.29059E-2,7.2062366E-2,-1.4229644E-3,-7.73131E-3,-1.0421438E-2,5.284545E-3,2.1643437E-2,-5.622055E-2,-1.2133861E-3,3.450007E-3,1.5059459E-2,-9.811467E-2,4.5328576E-2,1.4099422E-1,-2.9030559E-3,1.7450689E-3,-2.7158011E-3,6.433278E-2,-1.7998686E-2,-1.5983078E-1,-2.0180896E-3,4.371155E-2,-6.427971E-3,-2.0746812E-3,8.6892836E-2,-1.903061E-3,4.0421177E-2,2.1524878E-1,4.852566E-3,4.6577488E-4,-5.1417816E-2,2.7760068E-2,-1.0951999E-2,-1.6369451E-3,3.6517107E-3,-3.1822E-4,6.012741E-2,8.013697E-3,1.900807E-2,-3.306461E-3,-1.3113733E-3,4.6529607E-3,1.1820328E-2,5.269365E-3,-2.3443512E-4,-3.68842E-3,2.8550355E-3,-5.0797546E-4,5.4642158E-3,7.704871E-4,-2.9842143E-3,2.0250583E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,-1,7,-1,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,-1,-1,33,35,37,39,-1,-1,-1,41,43,45,-1,47,-1,-1,49,51,53,55,-1,-1,57,59,-1,-1,-1,-1,61,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6908048E-1,2.5205815E-1,7.776188E-1,0E0,2.024321E-1,0E0,2.0644106E-1,7.002868E-2,2.2926667E-1,6.7296E-2,9.866512E-2,7.0258126E-2,3.8589664E-2,8.905489E-2,1.4840493E-1,0E0,0E0,4.4900034E-2,0E0,5.979494E-2,1.753312E-1,0E0,0E0,3.1090874E-2,2.122578E-2,1.2188052E-1,1.6306353E-1,0E0,0E0,0E0,2.459912E-2,5.3420767E-2,1.1428985E-1,0E0,2.3278156E-2,0E0,0E0,6.9536135E-2,4.620015E-2,5.09879E-2,2.9340804E-2,0E0,0E0,2.7656637E-2,2.1526607E-2,0E0,0E0,0E0,0E0,6.8876505E-2,0E0,5.5927347E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,20,20,23,23,24,24,25,25,26,26,30,30,31,31,32,32,34,34,37,37,38,38,39,39,40,40,43,43,44,44,49,49,51,51],"right_children":[2,4,6,-1,8,-1,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,-1,-1,34,36,38,40,-1,-1,-1,42,44,46,-1,48,-1,-1,50,52,54,56,-1,-1,58,60,-1,-1,-1,-1,62,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3701E4,9.558929E4,2.1696895E5,8.232676E-3,3.200388E2,-1.7092153E-2,2.3040292E0,1E0,3.3892578E-1,4.3E1,3.328384E5,1.5023475E-2,1.1728099E2,1.0016339E6,1.6747E4,-1.4229644E-3,-7.73131E-3,2.9656984E7,5.284545E-3,8.842206E7,2.0006285E0,-1.2133861E-3,3.450007E-3,2.1891162E0,2.1368143E-1,2.21076E5,2.081E3,-2.9030559E-3,1.7450689E-3,-2.7158011E-3,1.8117242E8,1.8638788E6,3.4449153E5,-2.0180896E-3,9.05E3,-6.427971E-3,-2.0746812E-3,9.630793E2,3.4231E4,2.2673786E5,6.5023036E7,4.852566E-3,4.6577488E-4,4.283054E6,1.2247E4,-1.0951999E-2,-1.6369451E-3,3.6517107E-3,-3.1822E-4,1.00473E5,8.013697E-3,2.874086E2,-3.306461E-3,-1.3113733E-3,4.6529607E-3,1.1820328E-2,5.269365E-3,-2.3443512E-4,-3.68842E-3,2.8550355E-3,-5.0797546E-4,5.4642158E-3,7.704871E-4,-2.9842143E-3,2.0250583E-3],"split_indices":[9,27,31,0,4,0,52,15,26,8,32,56,4,31,9,0,0,12,0,5,52,0,0,52,40,28,2,0,0,0,7,50,32,0,10,0,0,51,10,31,44,0,0,44,10,0,0,0,0,28,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.36E2,1.92E2,4.4E1,9E0,1.83E2,6E0,3.8E1,7.3E1,1.1E2,1.4E1,2.4E1,5.9E1,1.4E1,2.7E1,8.3E1,7E0,7E0,1.5E1,9E0,1.5E1,4.4E1,5E0,9E0,1.6E1,1.1E1,6.1E1,2.2E1,7E0,8E0,5E0,1E1,3.3E1,1.1E1,5E0,1.1E1,5E0,6E0,3.2E1,2.9E1,1E1,1.2E1,5E0,5E0,1.9E1,1.4E1,6E0,5E0,6E0,5E0,2.6E1,6E0,2.3E1,6E0,5E0,5E0,7E0,5E0,8E0,1.1E1,7E0,7E0,1E1,1.6E1,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[8.814241E-3,-6.322562E-3,1.4278612E-1,1.5346397E-3,-9.372669E-3,1.6971244E-2,2.4466619E-2,7.3286146E-3,-8.694666E-3,-2.1319103E-3,4.2158384E-3,-1.0166416E-1,1.6664937E-2,-2.6997487E-4,-8.864821E-3,4.2043727E-2,-2.3195984E-2,2.0467697E-2,1.0758197E-1,6.676241E-2,-4.761216E-2,6.211083E-3,1.0080233E-2,9.90543E-4,1.407296E-1,5.613732E-3,8.8714063E-4,-1.0563998E-1,-6.4613093E-3,3.5082737E-3,-6.727369E-4,3.985483E-3,8.78785E-3,-5.6037786E-3,-1.308901E-3,5.2777845E-3,-1.3954714E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,-1,-1,9,11,-1,-1,-1,13,15,-1,-1,17,19,21,23,25,27,-1,29,-1,31,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7948855E-1,3.3760077E-1,6.54083E-1,2.3338597E-1,0E0,0E0,8.861047E-2,2.0623013E-1,0E0,0E0,0E0,1.4414272E-1,1.8913692E-1,0E0,0E0,1.5900035E-1,1.6351883E-1,1.04592994E-1,7.6370716E-2,4.1046828E-2,1.3781533E-1,0E0,1.382938E-1,0E0,3.9470732E-2,0E0,0E0,3.0654311E-2,1.0946612E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,22,22,24,24,27,27,28,28],"right_children":[2,4,6,8,-1,-1,10,12,-1,-1,-1,14,16,-1,-1,18,20,22,24,26,28,-1,30,-1,32,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.818E3,4.966421E6,4.083812E7,1.3575526E8,-9.372669E-3,1.6971244E-2,1.0865541E7,1.1744206E6,-8.694666E-3,-2.1319103E-3,4.2158384E-3,7.543738E1,2.2725725E0,-2.6997487E-4,-8.864821E-3,2.3097014E0,3.89E2,7E1,2.0676692E-1,1.5316E4,1.723E3,6.211083E-3,1.3257E4,9.90543E-4,4.9345682E2,5.613732E-3,8.8714063E-4,4.9487215E6,5.2091E5,3.5082737E-3,-6.727369E-4,3.985483E-3,8.78785E-3,-5.6037786E-3,-1.308901E-3,5.2777845E-3,-1.3954714E-3],"split_indices":[2,27,44,44,0,0,46,44,0,0,0,54,55,0,0,52,0,0,55,9,2,0,9,0,4,0,0,49,1,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.35E2,2.12E2,2.3E1,2.05E2,7E0,7E0,1.6E1,2E2,5E0,8E0,8E0,1.5E1,1.85E2,8E0,7E0,1.13E2,7.2E1,8.6E1,2.7E1,1.5E1,5.7E1,6E0,8E1,8E0,1.9E1,6E0,9E0,2.3E1,3.4E1,2.1E1,5.9E1,1.1E1,8E0,1.8E1,5E0,5E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.2257328E-3,6.718793E-3,-1.9604647E-1,-1.2129969E-2,1.3972837E-1,-1.6568784E-2,-9.527912E-4,4.3517746E-2,-3.946268E-2,2.703746E-1,4.4868186E-2,2.095494E-2,1.0693319E-2,-9.0272715E-3,-2.9345704E-2,3.9759004E-3,1.7492337E-2,-4.2474377E-3,6.484604E-3,-8.76818E-2,4.353281E-2,7.0669085E-2,-4.4880092E-2,-1.5179951E-3,1.1623027E-3,-8.0800377E-4,-6.440582E-3,8.513562E-2,1.018623E-2,-1.7721459E-4,1.0829615E-1,-1.2443929E-1,-3.0621048E-2,3.56363E-2,1.275403E-1,6.68654E-2,-5.095063E-2,6.5422663E-3,2.1340013E-3,-9.053989E-3,-1.5450463E-3,2.364931E-2,-5.8537528E-2,3.4068602E-3,-7.608245E-4,7.129699E-3,3.18419E-3,4.3765465E-3,1.6637601E-5,-4.185666E-3,-9.348224E-4,-1.4908946E-3,2.5290588E-3,-5.09028E-3,-1.3542571E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,21,-1,-1,23,-1,25,27,29,31,-1,-1,-1,-1,33,35,-1,37,39,41,43,45,47,49,-1,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6918982E-1,5.8584124E-1,3.5199422E-1,3.1352296E-1,3.4726936E-1,0E0,0E0,2.9210407E-1,2.1853103E-1,2.2280651E-1,8.808106E-2,1.5588976E-1,0E0,0E0,2.0670222E-1,0E0,0E0,1.234925E-2,0E0,3.9761744E-2,7.127299E-2,5.0950646E-2,1.255155E-1,0E0,0E0,0E0,0E0,4.3731987E-2,1.07362434E-1,0E0,2.0051196E-2,1.1016101E-1,1.4985803E-1,2.642658E-2,9.719476E-3,3.2086864E-2,1.705658E-2,0E0,0E0,0E0,0E0,6.4400114E-2,9.730585E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,14,14,17,17,19,19,20,20,21,21,22,22,27,27,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,41,41,42,42],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,22,-1,-1,24,-1,26,28,30,32,-1,-1,-1,-1,34,36,-1,38,40,42,44,46,48,50,-1,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,7.737766E2,2.126386E0,2.202021E8,3.0451374E9,-1.6568784E-2,-9.527912E-4,1.514E3,7.7586204E-2,3.758E3,1.7E1,9.23253E1,1.0693319E-2,-9.0272715E-3,3.5273347E5,3.9759004E-3,1.7492337E-2,4.3085775E11,6.484604E-3,8.756827E-1,1.6003E4,4.43E2,1.8417827E5,-1.5179951E-3,1.1623027E-3,-8.0800377E-4,-6.440582E-3,9.079337E1,5.5043226E8,-1.7721459E-4,3.61791E2,1.4058E4,7.07E2,2.53E2,9.525163E-1,7.251407E6,5.2857965E-1,6.5422663E-3,2.1340013E-3,-9.053989E-3,-1.5450463E-3,6.8656494E5,2.1368143E-1,3.4068602E-3,-7.608245E-4,7.129699E-3,3.18419E-3,4.3765465E-3,1.6637601E-5,-4.185666E-3,-9.348224E-4,-1.4908946E-3,2.5290588E-3,-5.09028E-3,-1.3542571E-3],"split_indices":[44,51,53,7,12,0,0,2,57,2,8,4,0,0,47,0,0,30,0,26,9,0,32,0,0,0,0,51,5,0,51,10,0,11,26,12,26,0,0,0,0,50,40,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.32E2,1.1E1,2.04E2,2.8E1,5E0,6E0,6.7E1,1.37E2,1.1E1,1.7E1,6.1E1,6E0,7E0,1.3E2,5E0,6E0,1.2E1,5E0,1E1,5.1E1,1.7E1,1.13E2,6E0,6E0,5E0,5E0,2.2E1,2.9E1,6E0,1.1E1,1.6E1,9.7E1,1.1E1,1.1E1,1.5E1,1.4E1,6E0,5E0,8E0,8E0,3.3E1,6.4E1,6E0,5E0,6E0,5E0,1E1,5E0,5E0,9E0,1.2E1,2.1E1,2.1E1,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-7.867916E-3,-1.2997443E-4,-1.40199E-2,-1.5223897E-2,8.999345E-2,-3.494468E-3,-1.863852E-1,1.8158327E-1,1.6801406E-2,-1.0416838E-1,6.3052517E-3,-1.5114389E-2,-4.940383E-4,2.793778E-3,1.39537295E-2,4.7536376E-3,-3.902562E-2,-1.2692951E-3,-8.394914E-3,-1.1859483E-2,4.834218E-2,1.203649E-4,-3.3884817E-3,5.8611133E-3,-8.916039E-2,8.5410856E-2,-7.927262E-6,1.6499301E-2,-7.4549643E-3,1.2362938E-3,-1.2306615E-1,1.0931209E-1,-1.2811957E-3,-2.899571E-3,3.6158554E-2,2.6814798E-3,-3.1344645E-4,-7.696958E-3,-1.9026889E-3,2.6880698E-3,6.4979647E-3,3.7235334E-3,-4.111895E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,23,25,-1,-1,27,29,31,33,35,-1,-1,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.650962E-1,3.224046E-1,0E0,4.068964E-1,2.2487313E-1,1.8909284E-1,3.3568352E-1,2.1026778E-1,1.0306051E-1,1.0073514E-1,1.3420148E-1,0E0,0E0,0E0,0E0,0E0,1.9778762E-2,0E0,0E0,1.6901022E-1,9.499123E-2,0E0,0E0,1.866618E-1,9.428489E-2,8.4721565E-2,5.8132958E-2,9.714318E-2,0E0,0E0,6.2127203E-2,3.463766E-2,0E0,0E0,3.437943E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,16,16,19,19,20,20,23,23,24,24,25,25,26,26,27,27,30,30,31,31,34,34],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,24,26,-1,-1,28,30,32,34,36,-1,-1,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4099883E8,7.015278E2,-1.40199E-2,7.526755E7,3.0451374E9,1.1744206E6,2.0651913E0,1.0376885E0,1.3626402E1,7.543738E1,2.3040292E0,-1.5114389E-2,-4.940383E-4,2.793778E-3,1.39537295E-2,4.7536376E-3,7.720737E5,-1.2692951E-3,-8.394914E-3,1.9577E4,2.5492957E0,1.203649E-4,-3.3884817E-3,3.7578388E6,2.161E4,1.2141942E6,1.3407147E4,8.968965E5,-7.4549643E-3,1.2362938E-3,1.0986164E6,2.0116584E8,-1.2811957E-3,-2.899571E-3,6.16E2,2.6814798E-3,-3.1344645E-4,-7.696958E-3,-1.9026889E-3,2.6880698E-3,6.4979647E-3,3.7235334E-3,-4.111895E-4],"split_indices":[44,51,0,44,12,44,53,56,56,54,52,0,0,0,0,0,27,0,0,9,55,0,0,27,28,50,46,47,0,0,46,7,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.4E2,2.35E2,5E0,2.02E2,3.3E1,1.9E2,1.2E1,1.4E1,1.9E1,1.6E1,1.74E2,6E0,6E0,8E0,6E0,7E0,1.2E1,9E0,7E0,1.22E2,5.2E1,6E0,6E0,1E2,2.2E1,2.9E1,2.3E1,9.5E1,5E0,5E0,1.7E1,2.4E1,5E0,8E0,1.5E1,3.3E1,6.2E1,1E1,7E0,1.1E1,1.3E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.2782744E-3,2.1829158E-2,-6.643703E-2,8.367616E-3,1.8639956E-1,-5.573927E-4,-2.0255113E-1,-8.34553E-3,5.2444227E-2,2.5777577E-3,1.67167E-2,-4.3707434E-2,3.8686201E-3,-2.4473602E-3,-1.6425172E-2,4.7128078E-2,-2.5334086E-2,9.5038414E-2,1.6854208E-2,-7.4930415E-2,4.3820794E-4,-2.7244713E-3,7.96583E-2,-9.687733E-3,-1.3592843E-2,4.612773E-2,6.502404E-3,-3.3532623E-2,7.323529E-2,-1.6351013E-3,-5.4311794E-3,1.2134342E-1,2.252936E-4,-2.5624968E-2,7.5775474E-2,-2.3171541E-4,3.5537956E-3,1.7434899E-3,-6.9864675E-2,4.908672E-3,9.947849E-4,3.7546307E-3,7.14288E-3,5.013072E-2,-3.8246006E-2,-2.0283625E-4,6.031387E-3,-4.594202E-3,-1.1814561E-3,-1.4123028E-4,4.3393197E-3,-7.03477E-3,-1.2515637E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,21,23,25,27,29,-1,-1,31,-1,33,35,-1,37,39,-1,-1,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9106647E-1,4.3929917E-1,4.1237587E-1,1.3671955E-1,3.3611596E-1,1.2298305E-1,3.3988756E-1,1.2842019E-1,7.5710535E-2,0E0,0E0,3.7426416E-2,0E0,0E0,0E0,1.1676681E-1,2.2848612E-1,4.8654735E-2,8.484083E-2,2.0273112E-2,0E0,0E0,7.720113E-2,0E0,1.0839553E-1,2.218758E-2,0E0,4.4670664E-2,2.312018E-2,0E0,0E0,9.390399E-3,0E0,8.605547E-2,5.6453854E-2,0E0,0E0,0E0,1.2709685E-2,0E0,0E0,0E0,0E0,3.2222603E-2,8.955834E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,15,15,16,16,17,17,18,18,19,19,22,22,24,24,25,25,27,27,28,28,31,31,33,33,34,34,38,38,43,43,44,44],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,22,24,26,28,30,-1,-1,32,-1,34,36,-1,38,40,-1,-1,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.32661E5,5.666E3,9.149094E-1,1E0,1.4781E4,9.933566E5,7.526755E7,2.86E2,1.108055E0,2.5777577E-3,1.67167E-2,2.225675E0,3.8686201E-3,-2.4473602E-3,-1.6425172E-2,1.6819989E8,7.12E2,2.98E2,5E0,4.32016E5,4.3820794E-4,-2.7244713E-3,2.202021E8,-9.687733E-3,9.42673E-1,3.1060094E4,6.502404E-3,3.09E2,4.2993274E8,-1.6351013E-3,-5.4311794E-3,1.112E3,2.252936E-4,1.2694E4,2.3484848E0,-2.3171541E-4,3.5537956E-3,1.7434899E-3,3.936726E7,4.908672E-3,9.947849E-4,3.7546307E-3,7.14288E-3,1.2262E4,1.8567E4,-2.0283625E-4,6.031387E-3,-4.594202E-3,-1.1814561E-3,-1.4123028E-4,4.3393197E-3,-7.03477E-3,-1.2515637E-3],"split_indices":[11,2,26,15,9,27,44,0,57,0,0,53,0,0,0,7,2,0,8,11,0,0,7,0,26,50,0,10,7,0,0,2,0,9,52,0,0,0,12,0,0,0,0,9,12,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,1.98E2,4.5E1,1.84E2,1.4E1,3.1E1,1.4E1,1.34E2,5E1,9E0,5E0,2.1E1,1E1,8E0,6E0,3.1E1,1.03E2,2.2E1,2.8E1,1.3E1,8E0,7E0,2.4E1,5E0,9.8E1,1.2E1,1E1,1.5E1,1.3E1,8E0,5E0,1.5E1,9E0,8.7E1,1.1E1,5E0,7E0,5E0,1E1,7E0,6E0,9E0,6E0,1.2E1,7.5E1,5E0,6E0,5E0,5E0,6E0,6E0,5E0,7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[5.199954E-3,8.403238E-3,-8.3427425E-5,-5.1779617E-3,5.3262175E-3,-2.003578E-2,3.5884373E-2,-7.202602E-3,-1.3987921E-2,1.0960703E-1,-7.4032643E-3,-2.6551636E-2,4.1176252E-2,6.9039814E-2,7.4420567E-3,-5.684427E-3,1.314976E-2,-4.0648714E-2,5.191102E-2,5.3202966E-3,9.329678E-3,4.113692E-3,9.267571E-4,3.250838E-2,-3.1081578E-3,8.396527E-3,-6.2210966E-2,9.722498E-2,-1.5276052E-3,-1.6603056E-2,3.8586804E-3,-2.5404978E-3,6.903822E-2,2.4505274E-3,-1.4014347E-3,-9.465346E-3,-2.1373169E-3,5.8396286E-3,1.4701197E-3,9.51641E-4,-1.678129E-3,9.106634E-4,4.689025E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,-1,3,5,-1,7,9,-1,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,-1,-1,31,-1,33,35,37,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.364238E-1,0E0,1.4573742E-1,1.4226142E-1,0E0,1.4223538E-1,1.986708E-1,0E0,1.15449354E-1,4.032764E-2,9.920955E-2,1.5143284E-1,7.4729964E-2,1.4262922E-2,0E0,0E0,5.7853226E-2,1.22137845E-1,8.4929295E-2,0E0,4.689628E-2,0E0,0E0,9.801243E-2,0E0,6.698721E-2,1.6507158E-1,2.4981081E-2,0E0,1.47756785E-2,0E0,0E0,3.3747032E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,20,20,23,23,25,25,26,26,27,27,29,29,32,32],"right_children":[2,-1,4,6,-1,8,10,-1,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,-1,-1,32,-1,34,36,38,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.747104E4,8.403238E-3,1.0086E4,1E0,5.3262175E-3,9.235376E5,9.4152045E-1,-7.202602E-3,2.3357933E0,2.2913464E2,1.13634065E-1,9.9472994E-1,6.063759E8,1.052641E7,7.4420567E-3,-5.684427E-3,7.6499896E9,4.49E2,3.706683E5,5.3202966E-3,1.118307E6,4.113692E-3,9.267571E-4,2.3135895E6,-3.1081578E-3,6.423972E1,5.050505E-3,1.306E3,-1.5276052E-3,2.5121107E0,3.8586804E-3,-2.5404978E-3,3.455983E-1,2.4505274E-3,-1.4014347E-3,-9.465346E-3,-2.1373169E-3,5.8396286E-3,1.4701197E-3,9.51641E-4,-1.678129E-3,9.106634E-4,4.689025E-3],"split_indices":[27,0,2,15,0,44,57,0,53,51,40,38,5,44,0,0,5,0,32,0,27,0,0,44,0,51,56,0,0,52,0,0,41,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,6E0,2.4E2,2.31E2,9E0,1.7E2,6.1E1,6E0,1.64E2,2.2E1,3.9E1,1.34E2,3E1,1.4E1,8E0,5E0,3.4E1,1.14E2,2E1,8E0,2.2E1,9E0,5E0,2.8E1,6E0,3.5E1,7.9E1,1.3E1,7E0,1.7E1,5E0,8E0,2E1,1.6E1,1.9E1,6E0,7.3E1,8E0,5E0,6E0,1.1E1,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.5410116E-3,-9.534064E-3,9.1156326E-2,-4.304543E-3,-8.336538E-3,1.27257025E-2,1.6822513E-4,4.4332672E-2,-2.5021784E-2,-2.688783E-2,3.4630443E-3,-1.8034318E-3,5.969683E-2,-3.75133E-2,5.8476184E-2,9.900216E-3,-4.328077E-3,8.232695E-2,-2.2597464E-2,-2.4516653E-2,-1.1340912E-1,-1.1958429E-3,9.0726204E-2,-9.4699045E-4,1.5770036E-3,6.5899864E-2,7.5380034E-3,-3.25435E-3,1.7587984E-3,-6.3495964E-2,2.2465214E-3,-1.005742E-2,-5.408614E-2,1.4109221E-4,6.2697995E-3,9.865095E-2,3.476645E-2,-8.737255E-3,-3.3613846E-2,5.130403E-2,-2.2333292E-2,-2.722078E-4,-4.921272E-3,2.4928087E-3,5.5680303E-3,3.0144637E-3,-1.5497865E-3,6.079436E-4,-4.67097E-3,-1.1260117E-4,4.0016295E-3,-1.5510139E-3,1.2806532E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,-1,17,19,21,23,-1,25,27,29,31,-1,33,-1,-1,35,-1,-1,-1,37,39,-1,41,-1,-1,43,45,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.735164E-1,1.8993197E-1,5.3885716E-1,2.0565796E-1,0E0,0E0,4.7678154E-2,8.1419855E-2,1.5134495E-1,4.416191E-2,0E0,0E0,9.870149E-2,1.20518506E-1,5.448743E-2,1.0031791E-2,0E0,4.6125174E-2,3.946074E-2,1.1280934E-1,1.05002716E-1,0E0,5.881962E-2,0E0,0E0,3.3277765E-2,0E0,0E0,0E0,1.6476473E-1,7.952437E-2,0E0,3.3728767E-2,0E0,0E0,1.1479855E-2,4.666129E-2,0E0,1.2332832E-1,4.4535402E-2,2.7905356E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22,25,25,29,29,30,30,32,32,35,35,36,36,38,38,39,39,40,40],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,-1,18,20,22,24,-1,26,28,30,32,-1,34,-1,-1,36,-1,-1,-1,38,40,-1,42,-1,-1,44,46,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,5.3159845E6,3.0451374E9,2.202021E8,-8.336538E-3,1.27257025E-2,3.988886E6,4.39E2,1.9002409E6,6.346443E6,3.4630443E-3,-1.8034318E-3,1.9042E4,5.1160636E11,4.3326885E-1,1.1989736E6,-4.328077E-3,1.491E3,3.51679E5,1.0469056E2,1.4400111E10,-1.1958429E-3,1.4943152E5,-9.4699045E-4,1.5770036E-3,7.0516006E8,7.5380034E-3,-3.25435E-3,1.7587984E-3,2.6727284E5,1.9167768E0,-1.005742E-2,2.2476077E0,1.4109221E-4,6.2697995E-3,2.557E4,1.6003E4,-8.737255E-3,7.07E2,5.2873194E-1,1.9270934E-1,-2.722078E-4,-4.921272E-3,2.4928087E-3,5.5680303E-3,3.0144637E-3,-1.5497865E-3,6.079436E-4,-4.67097E-3,-1.1260117E-4,4.0016295E-3,-1.5510139E-3,1.2806532E-3],"split_indices":[2,27,12,7,0,0,28,2,27,1,0,0,9,30,55,31,0,2,9,51,5,0,32,0,0,5,0,0,0,32,52,0,52,0,0,28,9,0,0,26,37,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,2.07E2,3E1,2.02E2,5E0,9E0,2.1E1,6E1,1.42E2,1.6E1,5E0,9E0,5.1E1,1.24E2,1.8E1,1.1E1,5E0,4E1,1.1E1,1.07E2,1.7E1,5E0,1.3E1,5E0,6E0,3.5E1,5E0,6E0,5E0,4.3E1,6.4E1,5E0,1.2E1,5E0,8E0,1.6E1,1.9E1,7E0,3.6E1,2.1E1,4.3E1,7E0,5E0,7E0,9E0,1.3E1,6E0,2.2E1,1.4E1,9E0,1.2E1,3.5E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-4.065063E-3,1.1936535E-2,-9.012328E-2,-3.6237247E-2,4.186069E-2,-1.7116604E-2,-3.6390167E-2,-1.157802E-2,-1.8350542E-2,8.911788E-2,-9.811525E-3,-8.395608E-2,5.7810348E-2,-3.6684614E-2,2.8318757E-2,1.1110175E-1,-1.2437104E-2,-2.2953732E-2,4.4463365E-3,-3.7479925E-4,-1.16530955E-1,4.529068E-4,4.080597E-3,-4.750863E-2,1.0784671E-3,4.537862E-3,1.129586E-3,6.4951956E-2,1.6185336E-1,1.635233E-3,-3.3954158E-3,3.7427425E-2,-5.660375E-2,-2.053131E-3,-6.5232357E-3,-6.977551E-2,4.645741E-3,-9.983227E-4,1.3561416E-3,1.74385E-2,9.829842E-2,2.585556E-1,6.0497463E-2,4.8670163E-3,8.648049E-3,-1.2804696E-1,-2.1452887E-2,-4.8279405E-2,-7.1453904E-3,-1.5380424E-3,1.5454006E-3,2.4174114E-3,-1.1900718E-3,6.390959E-3,1.7639392E-3,1.4985068E-2,6.6671143E-3,7.2610164E-3,-5.9716887E-4,-2.340019E-3,2.26195E-3,-3.0092553E-3,-7.318808E-3,7.1995775E-4,-3.607924E-2,-4.8768553E-3,-6.4148795E-4,-3.5238554E-4,-4.3296046E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,21,23,25,27,29,31,-1,-1,33,-1,-1,35,-1,-1,37,39,41,-1,-1,43,45,-1,-1,47,49,-1,-1,51,53,55,57,-1,59,61,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,67,-1,-1,-1,-1],"loss_changes":[3.4258887E-1,3.042444E-1,5.9279144E-1,3.1479055E-1,3.1831926E-1,0E0,1.5719846E-1,0E0,6.606737E-2,1.5319663E-1,9.2027985E-2,5.5705756E-2,1.7825406E-2,3.72136E-2,4.314311E-2,1.2064916E-1,4.2529684E-2,1.1793267E-1,0E0,0E0,2.5372058E-2,0E0,0E0,5.490596E-2,0E0,0E0,1.21158445E-2,4.7402397E-2,2.3919296E-1,0E0,0E0,4.167874E-2,9.017078E-2,0E0,0E0,5.5260837E-2,1.8391198E-2,0E0,0E0,2.3610815E-2,4.0721416E-2,5.701977E-2,1.0591477E-1,0E0,4.28172E-2,1.4374971E-2,1.4806017E-2,5.5550158E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.143746E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,23,23,26,26,27,27,28,28,31,31,32,32,35,35,36,36,39,39,40,40,41,41,42,42,44,44,45,45,46,46,47,47,64,64],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,22,24,26,28,30,32,-1,-1,34,-1,-1,36,-1,-1,38,40,42,-1,-1,44,46,-1,-1,48,50,-1,-1,52,54,56,58,-1,60,62,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,68,-1,-1,-1,-1],"split_conditions":[2.68225E5,3.200388E2,1.5615072E5,3.9134244E-5,3.1665432E0,-1.7116604E-2,2.6056657E0,-1.157802E-2,1.1633914E0,1.2E1,2.4494185E3,1.36623E5,5E1,1.137E3,1.6128859E2,2.001E3,1.00068504E8,2.47538E5,4.4463365E-3,-3.7479925E-4,1.6908462E5,4.529068E-4,4.080597E-3,8.0563555E6,1.0784671E-3,4.537862E-3,2.2E1,6.1895325E5,1.119543E6,1.635233E-3,-3.3954158E-3,5.5043226E8,2.7573213E8,-2.053131E-3,-6.5232357E-3,6.310469E6,8.558452E1,-9.983227E-4,1.3561416E-3,5.1454154E8,2.67302E5,2.674421E9,5.913995E5,4.8670163E-3,1.33358E5,5.638132E0,4.7564573E0,6.888215E-1,-7.1453904E-3,-1.5380424E-3,1.5454006E-3,2.4174114E-3,-1.1900718E-3,6.390959E-3,1.7639392E-3,1.4985068E-2,6.6671143E-3,7.2610164E-3,-5.9716887E-4,-2.340019E-3,2.26195E-3,-3.0092553E-3,-7.318808E-3,7.1995775E-4,3.7276E4,-4.8768553E-3,-6.4148795E-4,-3.5238554E-4,-4.3296046E-3],"split_indices":[9,4,46,41,55,0,52,0,55,3,4,28,8,2,54,2,44,28,0,0,32,0,0,44,0,0,8,27,1,0,0,5,7,0,0,44,51,0,0,5,28,5,31,0,28,55,55,26,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.47E2,2.09E2,3.8E1,8E1,1.29E2,5E0,3.3E1,5E0,7.5E1,6.7E1,6.2E1,2.2E1,1.1E1,5.4E1,2.1E1,5.5E1,1.2E1,5.6E1,6E0,7E0,1.5E1,5E0,6E0,4.6E1,8E0,5E0,1.6E1,3E1,2.5E1,7E0,5E0,2E1,3.6E1,5E0,1E1,3.2E1,1.4E1,9E0,7E0,1.3E1,1.7E1,1.2E1,1.3E1,5E0,1.5E1,1.1E1,2.5E1,2.7E1,5E0,6E0,8E0,7E0,6E0,9E0,8E0,6E0,6E0,5E0,8E0,6E0,9E0,5E0,6E0,7E0,1.8E1,9E0,1.8E1,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[-1.4966164E-2,-2.8582516E-3,-1.223701E-1,-1.041183E-2,1.4366633E-1,-2.946632E-1,1.2136173E-2,3.3412647E-2,-3.0315422E-2,1.1954483E-2,-7.7699115E-5,-4.0375646E-3,-2.0322012E-2,-1.9689968E-3,3.7042415E-3,-4.7937687E-3,4.889866E-2,-1.0470393E-1,-1.6385412E-2,6.411595E-2,-6.767648E-3,-8.946408E-3,-3.0297829E-2,5.720856E-2,-2.7713047E-2,3.333007E-2,9.898425E-2,-3.1251514E-3,1.6087273E-3,1.268526E-3,-3.5008085E-3,-4.8320653E-4,4.771997E-3,-3.63899E-4,-6.2461674E-2,5.6412134E-2,-1.2869945E-3,1.5102178E-3,1.1627666E-1,1.9897485E-2,-9.611191E-2,-1.7571754E-3,-1.0217299E-1,5.3982288E-3,9.3481847E-4,6.4446325E-3,2.6061633E-3,2.2619027E-3,-6.671064E-4,-6.265919E-3,-1.6795775E-3,1.961105E-3,-1.9268864E-3,-6.2451133E-4,-7.3818E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,-1,-1,19,21,23,25,27,-1,29,31,33,35,37,-1,-1,-1,-1,-1,-1,39,41,43,-1,-1,45,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.208583E-1,2.4830887E-1,5.829386E-1,1.8680528E-1,1.9468193E-1,3.1166655E-1,6.2189143E-2,1.5016958E-1,1.5061058E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.2106395E-2,1.543383E-1,1.0579112E-1,4.822649E-2,3.9633617E-2,0E0,4.0907755E-2,5.727491E-2,1.0356492E-1,4.01693E-2,2.1021247E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.22191586E-1,1.1553194E-1,4.2692304E-2,0E0,0E0,1.822856E-2,5.5337682E-2,2.2567116E-2,3.8939137E-2,1.5533748E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,33,33,34,34,35,35,38,38,39,39,40,40,41,41,42,42],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,-1,-1,20,22,24,26,28,-1,30,32,34,36,38,-1,-1,-1,-1,-1,-1,40,42,44,-1,-1,46,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0232127E12,2.013055E3,2.095716E0,4.15E2,1E0,9E0,2.421233E0,3.9E2,1.4773677E5,1.1954483E-2,-7.7699115E-5,-4.0375646E-3,-2.0322012E-2,-1.9689968E-3,3.7042415E-3,-4.7937687E-3,2.3112622E5,1.6994E5,6.4266656E5,8.364481E3,1.4395604E0,-8.946408E-3,2.024E3,1E0,3.8024444E5,1.81692E5,4.959E3,-3.1251514E-3,1.6087273E-3,1.268526E-3,-3.5008085E-3,-4.8320653E-4,4.771997E-3,5.33107E5,1E0,2.73E2,-1.2869945E-3,1.5102178E-3,5.0381964E2,1.5202525E6,2.4629184E2,2.5055168E9,1.0914128E0,5.3982288E-3,9.3481847E-4,6.4446325E-3,2.6061633E-3,2.2619027E-3,-6.671064E-4,-6.265919E-3,-1.6795775E-3,1.961105E-3,-1.9268864E-3,-6.2451133E-4,-7.3818E-3],"split_indices":[30,51,53,0,15,3,52,2,32,0,0,0,0,0,0,0,46,1,50,46,52,0,2,80,32,12,28,0,0,0,0,0,0,9,63,0,0,0,4,46,51,5,55,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.22E2,2.4E1,2.12E2,1E1,1E1,1.4E1,6.6E1,1.46E2,5E0,5E0,5E0,5E0,8E0,6E0,6E0,6E1,2.2E1,1.24E2,4.7E1,1.3E1,9E0,1.3E1,1.6E1,1.08E2,2.6E1,2.1E1,5E0,8E0,6E0,7E0,7E0,9E0,6.1E1,4.7E1,1.9E1,7E0,5E0,1.6E1,5.1E1,1E1,1.9E1,2.8E1,6E0,1.3E1,1E1,6E0,2.7E1,2.4E1,5E0,5E0,9E0,1E1,1.2E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.4514912E-2,-8.332141E-3,-1.0159535E-2,7.407785E-3,-9.5314644E-2,-3.835213E-3,7.063736E-2,-3.7867192E-2,-1.73973E-2,-9.836819E-3,6.3854577E-3,-2.4458074E-3,1.0555534E-1,2.325461E-3,-7.510702E-2,2.5783194E-2,-2.724808E-2,-1.5755851E-3,1.3733921E-3,6.5306732E-3,3.1675645E-3,-1.0385045E-1,2.4476138E-4,3.7351858E-2,-3.4652383E-3,-8.820748E-3,-1.6083088E-2,-6.7795003E-3,-2.065894E-3,1.0863369E-2,8.1392296E-2,-2.8963527E-2,4.577599E-2,2.542925E-3,-9.717879E-4,4.711924E-3,1.2111634E-3,-4.9368956E-4,-3.6509563E-3,3.184256E-3,-3.1093962E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,-1,5,7,9,11,13,-1,15,-1,17,19,-1,21,23,25,-1,-1,-1,-1,27,-1,29,-1,-1,31,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2003006E-1,3.3065873E-1,0E0,1.4600337E-1,6.028655E-1,1.5414985E-1,7.936758E-2,1.1045472E-1,0E0,1.0551454E-1,0E0,1.2825478E-2,1.8947378E-2,0E0,5.3805307E-2,6.935932E-2,2.1115333E-1,0E0,0E0,0E0,0E0,3.8985863E-2,0E0,5.8610596E-2,0E0,0E0,8.737721E-2,0E0,0E0,4.9918164E-2,2.1130562E-2,8.342871E-2,2.5346242E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,14,14,15,15,16,16,21,21,23,23,26,26,29,29,30,30,31,31,32,32],"right_children":[2,4,-1,6,8,10,12,14,-1,16,-1,18,20,-1,22,24,26,-1,-1,-1,-1,28,-1,30,-1,-1,32,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,2.68225E5,-1.0159535E-2,1.9002409E6,1.9E1,1.7951E4,4.3326885E-1,2.2193549E0,-1.73973E-2,4.86E2,6.3854577E-3,9.5082015E-1,1.479E3,2.325461E-3,2.6180778E6,1.6958537E9,7.66E2,-1.5755851E-3,1.3733921E-3,6.5306732E-3,3.1675645E-3,6.069098E5,2.4476138E-4,1.135E3,-3.4652383E-3,-8.820748E-3,1E0,-6.7795003E-3,-2.065894E-3,1.8117242E8,1.2983751E2,1.876E3,1E0,2.542925E-3,-9.717879E-4,4.711924E-3,1.2111634E-3,-4.9368956E-4,-3.6509563E-3,3.184256E-3,-3.1093962E-4],"split_indices":[27,9,0,27,3,2,55,57,0,0,0,26,0,0,46,5,2,0,0,0,0,27,0,2,0,0,108,0,0,7,51,0,111,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.4E2,6E0,2.04E2,3.6E1,1.74E2,3E1,3.1E1,5E0,1.68E2,6E0,1E1,2E1,9E0,2.2E1,5.5E1,1.13E2,5E0,5E0,8E0,1.2E1,1.6E1,6E0,5E1,5E0,6E0,1.07E2,8E0,8E0,3.2E1,1.8E1,8.9E1,1.8E1,1.3E1,1.9E1,1.2E1,6E0,6.7E1,2.2E1,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-7.5745984E-4,-1.608656E-2,9.5420755E-2,6.415564E-6,-1.2811868E-1,1.218457E-2,2.8678373E-2,-1.7062267E-2,4.6683908E-2,-1.1809043E-3,-1.5971141E-2,-8.290177E-3,5.633141E-3,-7.05645E-3,-1.0445037E-2,9.394006E-2,1.6214533E-2,-3.8156763E-3,2.5014432E-2,5.1165104E-2,-2.649386E-2,1.988045E-3,1.19346194E-1,4.024073E-3,-4.881403E-3,2.8053313E-3,-1.5760942E-3,9.912661E-3,8.8762894E-2,-7.521639E-3,-1.7633013E-2,3.0990834E-3,7.1475925E-3,-2.182029E-3,2.2616731E-2,2.9639627E-3,-2.4934164E-3,2.3656378E-3,5.790765E-3,-7.49054E-2,-1.9874598E-3,-7.6807668E-3,3.3545378E-3,4.7820093E-4,-4.951881E-3,4.1428205E-4,-3.5575756E-3,-2.3397182E-3,1.7047552E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,-1,19,21,23,-1,25,27,29,-1,31,-1,33,-1,-1,35,37,-1,39,-1,-1,-1,41,-1,-1,-1,-1,43,45,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.538988E-1,3.732073E-1,3.772325E-1,1.4580017E-1,5.898485E-1,0E0,8.998779E-2,1.219964E-1,6.903021E-2,0E0,0E0,5.0416782E-2,0E0,0E0,1.2890692E-1,1.8802479E-2,4.8192903E-2,0E0,3.340422E-2,4.0997148E-2,1.2511647E-1,0E0,1.2895152E-2,0E0,3.1004751E-2,0E0,0E0,5.4592274E-2,1.3417102E-2,0E0,8.766484E-2,0E0,0E0,0E0,2.5670301E-2,0E0,0E0,0E0,0E0,6.3855104E-2,6.776095E-2,2.408054E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,14,14,15,15,16,16,18,18,19,19,20,20,22,22,24,24,27,27,28,28,30,30,34,34,39,39,40,40,41,41],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,-1,20,22,24,-1,26,28,30,-1,32,-1,34,-1,-1,36,38,-1,40,-1,-1,-1,42,-1,-1,-1,-1,44,46,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,6.5023036E7,4.083812E7,1E0,1.2714615E6,1.218457E-2,1.1533942E6,9.235376E5,4.8162012E5,-1.1809043E-3,-1.5971141E-2,1.9884044E3,5.633141E-3,-7.05645E-3,3.01E2,1.6262975E-1,1.672E4,-3.8156763E-3,8.096533E-1,1.8126168E8,4.305949E-2,1.988045E-3,2.758609E2,4.024073E-3,1.8435853E0,2.8053313E-3,-1.5760942E-3,8.0224875E4,1.321E3,-7.521639E-3,3.6452372E5,3.0990834E-3,7.1475925E-3,-2.182029E-3,3.645971E5,2.9639627E-3,-2.4934164E-3,2.3656378E-3,5.790765E-3,1E0,5.2726665E0,8E0,3.3545378E-3,4.7820093E-4,-4.951881E-3,4.1428205E-4,-3.5575756E-3,-2.3397182E-3,1.7047552E-3],"split_indices":[2,44,44,15,27,0,27,44,50,0,0,54,0,0,0,57,9,0,26,7,57,0,4,0,53,0,0,32,2,0,27,0,0,0,32,0,0,0,0,63,55,3,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.38E2,2.06E2,3.2E1,1.81E2,2.5E1,8E0,2.4E1,1.33E2,4.8E1,1.8E1,7E0,1.8E1,6E0,5E0,1.28E2,1.8E1,3E1,5E0,1.3E1,2.6E1,1.02E2,7E0,1.1E1,6E0,2.4E1,8E0,5E0,1.3E1,1.3E1,5E0,9.7E1,6E0,5E0,9E0,1.5E1,7E0,6E0,8E0,5E0,2E1,7.7E1,1E1,5E0,6E0,1.4E1,6.8E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-8.718829E-3,1.5471022E-3,-1.2820925E-1,7.614226E-3,-6.4431275E-3,-2.5626497E-2,-1.5610991E-2,9.673455E-4,1.10310815E-1,-3.9523784E-3,1.436105E-3,3.337736E-2,-3.1134408E-2,9.320704E-3,-8.296253E-5,2.530905E-2,7.0753163E-3,1.791191E-2,-6.178073E-2,-2.3578696E-3,3.3937555E-2,-1.7188374E-2,7.755827E-2,-7.563152E-2,3.7159226E-3,8.533436E-2,1.9292964E-2,1.6840523E-2,-5.829213E-2,5.411996E-3,1.998724E-3,-1.1902719E-1,-1.764349E-2,1.6439431E-3,4.738687E-3,1.511773E-3,-2.011041E-3,1.8380372E-3,-7.3595205E-4,-7.8402815E-4,-4.342353E-3,-8.982146E-3,-2.8738114E-3,1.2640422E-3,-2.9524476E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,-1,15,17,-1,-1,19,-1,21,23,-1,25,27,29,31,-1,33,35,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9460165E-1,1.9562718E-1,4.1600928E-1,1.4597224E-1,0E0,5.2699074E-2,0E0,2.1120368E-1,1.409158E-1,0E0,0E0,9.669335E-2,1.5442172E-1,0E0,0E0,6.6052586E-2,0E0,8.507726E-2,1.320508E-1,0E0,6.383727E-2,3.734827E-2,1.5847288E-2,1.4385286E-1,0E0,1.451382E-2,6.424235E-2,1.2564197E-2,1.6981762E-2,0E0,0E0,1.3359049E-1,5.8639526E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,11,11,12,12,15,15,17,17,18,18,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,31,31,32,32],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,-1,16,18,-1,-1,20,-1,22,24,-1,26,28,30,32,-1,34,36,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2E1,4.966421E6,1.0860942E6,2.6732678E3,-6.4431275E-3,8.87885E5,-1.5610991E-2,2.4626505E0,1.0915004E7,-3.9523784E-3,1.436105E-3,2.902E3,1.9167768E0,9.320704E-3,-8.296253E-5,7.361644E1,7.0753163E-3,1.5423752E2,1.8656379E6,-2.3578696E-3,2.572438E4,5.0795E4,7.091913E8,1.6797491E6,3.7159226E-3,2.0787193E2,2.72126E5,1.388952E0,5.5719394E5,5.411996E-3,1.998724E-3,1.6066E4,7.224059E-1,1.6439431E-3,4.738687E-3,1.511773E-3,-2.011041E-3,1.8380372E-3,-7.3595205E-4,-7.8402815E-4,-4.342353E-3,-8.982146E-3,-2.8738114E-3,1.2640422E-3,-2.9524476E-3],"split_indices":[3,27,27,4,0,1,0,57,46,0,0,2,52,0,0,54,0,51,27,0,31,28,12,46,0,4,28,40,46,0,0,9,26,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.39E2,2.21E2,1.8E1,2.13E2,8E0,1.3E1,5E0,2.01E2,1.2E1,6E0,7E0,1E2,1.01E2,6E0,6E0,9.5E1,5E0,3.9E1,6.2E1,9E0,8.6E1,2.5E1,1.4E1,5.7E1,5E0,1.8E1,6.8E1,1.4E1,1.1E1,5E0,9E0,3.2E1,2.5E1,6E0,1.2E1,5.6E1,1.2E1,8E0,6E0,6E0,5E0,1.2E1,2E1,1.3E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-5.3743357E-3,8.2446E-3,-9.526168E-2,-6.902669E-2,2.0353317E-2,-1.42063415E-2,-4.087732E-2,-2.4457375E-2,-6.6198087E-3,7.9261556E-2,2.1452925E-3,-7.923436E-2,3.3618044E-3,-6.199333E-3,2.5728261E-2,4.401993E-2,1.5449135E-1,2.6807047E-2,-3.868271E-2,-4.546867E-3,-5.423648E-4,3.5703038E-3,-1.0937593E-3,1.04780324E-1,3.3281073E-3,3.9984463E-3,9.762541E-3,-6.290719E-2,4.2205397E-2,-6.976362E-2,3.5394696E-3,2.5127921E-3,5.9458115E-3,-1.7944471E-3,2.629222E-2,1.5061715E-3,-8.15426E-3,6.1662123E-2,-1.7362077E-2,-4.1452486E-2,-7.940729E-3,-5.328261E-2,6.0066596E-2,2.1701166E-3,-3.247393E-4,8.812658E-2,1.6756767E-2,8.2783E-4,-2.3153324E-3,-4.258096E-3,-5.750347E-3,-3.1976166E-4,-3.8447224E-3,4.150051E-3,5.75356E-4,2.1790697E-3,5.745984E-3,2.4737627E-3,-6.241847E-4,-1.9257984E-3,1.6479304E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,-1,21,23,25,27,29,-1,-1,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,-1,43,-1,-1,45,47,49,-1,51,53,-1,-1,55,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8846532E-1,1.9321448E-1,3.576664E-1,9.347315E-2,1.9075325E-1,0E0,1.2102477E-1,1.113082E-1,0E0,1.0455054E-1,1.3898094E-1,2.782078E-2,0E0,0E0,3.9323356E-2,7.382481E-2,3.3860862E-2,1.21459834E-1,6.8486735E-2,0E0,0E0,0E0,0E0,8.591995E-3,1.986168E-2,0E0,0E0,1.5368792E-1,8.683571E-2,8.301489E-2,7.707624E-2,0E0,0E0,0E0,9.853505E-3,0E0,0E0,6.5073416E-2,2.400762E-2,4.6742853E-2,0E0,1.7122194E-2,1.6973425E-2,0E0,0E0,4.859054E-2,2.640976E-2,0E0,0E0,0E0,2.6571343E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,23,23,24,24,27,27,28,28,29,29,30,30,34,34,37,37,38,38,39,39,41,41,42,42,45,45,46,46,50,50],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,-1,22,24,26,28,30,-1,-1,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,-1,44,-1,-1,46,48,50,-1,52,54,-1,-1,56,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8337E4,7.23E2,2.4586095E5,3.31E2,2.1203648E8,-1.42063415E-2,4.525301E0,1.4693E4,-6.6198087E-3,4.4526025E2,3.7861453E5,1.3143399E6,3.3618044E-3,-6.199333E-3,5.84E2,2.92E2,2.4507043E-1,1.062E3,1.6497E4,-4.546867E-3,-5.423648E-4,3.5703038E-3,-1.0937593E-3,4.9605957E1,1.26E2,3.9984463E-3,9.762541E-3,6.6318585E6,4.8958065E6,1.0220919E6,2E0,2.5127921E-3,5.9458115E-3,-1.7944471E-3,1.6740066E5,1.5061715E-3,-8.15426E-3,2.7438753E0,2.3898147E5,7.265892E0,-7.940729E-3,5.63E2,1.7266E4,2.1701166E-3,-3.247393E-4,1.895E3,7.57E2,8.2783E-4,-2.3153324E-3,-4.258096E-3,2.1201453E-1,-3.1976166E-4,-3.8447224E-3,4.150051E-3,5.75356E-4,2.1790697E-3,5.745984E-3,2.4737627E-3,-6.241847E-4,-1.9257984E-3,1.6479304E-3],"split_indices":[9,2,31,10,7,0,52,9,0,4,32,27,0,0,2,0,57,2,9,0,0,0,0,51,10,0,0,44,46,27,8,0,0,0,31,0,0,55,32,56,0,0,9,0,0,2,0,0,0,0,38,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,2.04E2,3E1,2.7E1,1.77E2,5E0,2.5E1,1.8E1,9E0,4.1E1,1.36E2,1.9E1,6E0,5E0,1.3E1,2.9E1,1.2E1,8.5E1,5.1E1,1.4E1,5E0,6E0,7E0,1.1E1,1.8E1,7E0,5E0,1.2E1,7.3E1,2.9E1,2.2E1,5E0,6E0,6E0,1.2E1,7E0,5E0,5.5E1,1.8E1,2.4E1,5E0,1.1E1,1.1E1,7E0,5E0,3.4E1,2.1E1,9E0,9E0,9E0,1.5E1,5E0,6E0,6E0,5E0,1.8E1,1.6E1,9E0,1.2E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.1502168E-3,1.7106434E-2,-8.3482444E-2,8.612298E-3,1.4058903E-1,-2.3123294E-1,-9.29728E-3,-8.0588005E-2,1.5579669E-2,-2.6367258E-3,1.4410051E-2,-2.7681931E-3,-1.8784093E-2,-3.425862E-2,2.0989291E-3,-5.884405E-4,-7.596437E-3,-1.1914592E-3,6.1026298E-2,-6.4536505E-3,-4.8397384E-3,1.0597506E-2,-6.7150086E-2,8.180088E-2,4.730827E-3,1.3138361E-3,-1.0691822E-3,6.0366318E-2,-2.1941721E-4,-4.8686685E-3,-2.372245E-4,4.9181603E-2,6.3229636E-3,2.2981942E-3,-1.6376032E-3,3.407042E-4,8.8221595E-2,-6.561685E-2,1.2200595E-2,6.6167906E-2,6.096075E-4,1.716088E-3,5.6718397E-3,-5.639705E-3,-1.0543998E-3,-3.9094347E-3,9.702482E-4,4.170757E-3,1.2752255E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,21,23,25,-1,27,29,31,33,-1,-1,35,37,-1,-1,39,-1,-1,-1,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7269643E-1,2.1171142E-1,4.910905E-1,1.19960435E-1,4.8185122E-1,4.4660085E-1,4.480734E-2,8.08917E-2,1.3574275E-1,0E0,0E0,0E0,0E0,4.4005524E-2,0E0,0E0,0E0,1.02575116E-1,5.574973E-2,1.1132122E-2,0E0,6.0307927E-2,4.9959667E-2,5.980231E-2,2.8420933E-2,0E0,0E0,2.875451E-2,7.63386E-2,0E0,0E0,1.3644043E-2,0E0,0E0,0E0,0E0,1.970815E-2,3.47395E-2,7.4654326E-2,1.3343014E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,13,13,17,17,18,18,19,19,21,21,22,22,23,23,24,24,27,27,28,28,31,31,36,36,37,37,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,22,24,26,-1,28,30,32,34,-1,-1,36,38,-1,-1,40,-1,-1,-1,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,5.818E3,9.4014386E2,9.235376E5,2.605017E5,1.3143399E6,8.3360725E5,6.8883444E5,2.1799042E0,-2.6367258E-3,1.4410051E-2,-2.7681931E-3,-1.8784093E-2,1E0,2.0989291E-3,-5.884405E-4,-7.596437E-3,1.9577E4,3.3564355E0,2.0968523E0,-4.8397384E-3,6.2085975E9,2.3566537E-1,3.9401126E2,5.8142107E-2,1.3138361E-3,-1.0691822E-3,1.4615384E0,3.6452372E5,-4.8686685E-3,-2.372245E-4,1.052641E7,6.3229636E-3,2.2981942E-3,-1.6376032E-3,3.407042E-4,1.1924399E0,1.3802E4,4.305949E-2,1.5698539E-1,6.096075E-4,1.716088E-3,5.6718397E-3,-5.639705E-3,-1.0543998E-3,-3.9094347E-3,9.702482E-4,4.170757E-3,1.2752255E-3],"split_indices":[44,2,4,44,32,27,27,44,53,0,0,0,0,108,0,0,0,9,55,52,0,30,41,4,26,0,0,52,27,0,0,44,0,0,0,0,57,9,57,41,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.02E2,4.4E1,1.9E2,1.2E1,1.4E1,3E1,1.3E1,1.77E2,6E0,6E0,8E0,6E0,2.1E1,9E0,8E0,5E0,1.3E2,4.7E1,1.6E1,5E0,1.11E2,1.9E1,3.4E1,1.3E1,5E0,1.1E1,1.9E1,9.2E1,1.1E1,8E0,2.3E1,1.1E1,6E0,7E0,7E0,1.2E1,1.4E1,7.8E1,1.5E1,8E0,6E0,6E0,5E0,9E0,6E0,7.2E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.2944112E-3,-9.220463E-3,1.01152055E-1,2.2466786E-3,-9.284702E-2,1.45256845E-2,2.2675235E-2,-8.09318E-2,8.300074E-3,-8.791214E-3,-1.3612809E-2,6.1933745E-2,-1.2040772E-3,-5.2438414E-4,-6.699506E-3,4.564134E-2,-8.608016E-3,-5.4788206E-2,2.643567E-3,4.978066E-4,5.004696E-3,6.431431E-2,-1.4665207E-2,-6.352768E-3,9.846933E-4,-4.697871E-3,-4.9120234E-4,4.804869E-2,7.428673E-3,-4.4574128E-3,2.3647298E-3,7.6737985E-2,-6.997975E-3,4.537791E-3,1.7055389E-2,5.43722E-4,5.4658824E-3,-5.4456968E-2,6.2628346E-3,-2.0148724E-3,1.4933653E-3,-9.40203E-4,-7.097616E-3,1.7175864E-3,-1.0091605E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,-1,-1,-1,21,23,25,-1,-1,-1,27,29,-1,31,-1,-1,33,-1,-1,-1,35,37,-1,39,-1,-1,41,43,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9462266E-1,2.2268677E-1,4.6092474E-1,1.03891745E-1,4.893362E-1,0E0,4.2362206E-2,6.390041E-2,1.2161583E-1,6.8332456E-2,0E0,2.8733432E-2,0E0,0E0,0E0,6.850934E-2,1.6889998E-1,2.7051706E-2,0E0,0E0,0E0,6.722745E-2,8.992587E-2,0E0,7.612618E-2,0E0,0E0,6.5104365E-2,0E0,0E0,0E0,3.3059984E-2,7.208635E-2,0E0,2.8255235E-2,0E0,0E0,8.320815E-2,8.2899116E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,15,15,16,16,17,17,21,21,22,22,24,24,27,27,31,31,32,32,34,34,37,37,38,38],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,-1,-1,-1,22,24,26,-1,-1,-1,28,30,-1,32,-1,-1,34,-1,-1,-1,36,38,-1,40,-1,-1,42,44,-1,-1,-1,-1,-1,-1],"split_conditions":[5.818E3,7.526755E7,3.0451374E9,4.75E2,1.0055311E1,1.45256845E-2,3.945346E1,1.7046307E0,2.1203648E8,5.293399E2,-1.3612809E-2,4E0,-1.2040772E-3,-5.2438414E-4,-6.699506E-3,3.3564355E0,2.2315563E8,1E1,2.643567E-3,4.978066E-4,5.004696E-3,5.0003815E2,3.322259E-2,-6.352768E-3,2.583702E-2,-4.697871E-3,-4.9120234E-4,2.73E2,7.428673E-3,-4.4574128E-3,2.3647298E-3,1.631E3,1.4065148E6,4.537791E-3,6.71E2,5.43722E-4,5.4658824E-3,1.1593482E6,1.4103535E0,-2.0148724E-3,1.4933653E-3,-9.40203E-4,-7.097616E-3,1.7175864E-3,-1.0091605E-3],"split_indices":[2,44,12,2,56,0,56,53,7,51,0,8,0,0,0,55,7,3,0,0,0,4,56,0,40,0,0,0,0,0,0,2,49,0,2,0,0,50,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.57E2,2.31E2,2.6E1,2.04E2,2.7E1,6E0,2E1,1.3E1,1.91E2,2E1,7E0,1.1E1,9E0,7E0,6E0,5.9E1,1.32E2,1.2E1,8E0,6E0,5E0,4.5E1,1.4E1,8E0,1.24E2,5E0,7E0,4E1,5E0,6E0,8E0,1.1E1,1.13E2,1.4E1,2.6E1,5E0,6E0,2.4E1,8.9E1,5E0,2.1E1,1.9E1,5E0,4.2E1,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[7.059997E-3,-4.6984586E-3,1.12101994E-1,7.726425E-4,-5.862529E-3,1.5400032E-2,3.375335E-2,1.8375661E-2,-4.587347E-2,-5.7817358E-3,4.615366E-3,7.44324E-3,7.317733E-2,-9.4524056E-2,1.1683626E-2,1.4275579E-3,-1.494325E-3,1.624408E-2,-6.47524E-2,8.299366E-6,1.14224546E-1,-8.117734E-3,-5.1363513E-2,3.0617183E-2,-2.6914102E-3,5.3485376E-3,6.156414E-2,-7.0339693E-3,1.4734761E-4,2.7051365E-3,6.7521413E-3,8.7301875E-4,-7.909514E-2,1.0122261E-2,3.2957438E-3,5.628928E-2,-1.2193267E-2,9.936205E-3,5.524891E-3,-4.429584E-3,-1.3899816E-3,1.3581467E-3,-8.9692714E-4,1.6242579E-4,3.5069243E-3,-2.605049E-3,3.84153E-4,-1.1644291E-3,2.5233394E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,17,19,21,23,-1,-1,25,27,-1,29,-1,31,33,-1,35,37,-1,-1,-1,-1,-1,39,41,-1,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9448807E-1,1.475048E-1,4.1986713E-1,1.7081322E-1,0E0,0E0,5.212211E-2,8.9921676E-2,1.60715E-1,1.4364699E-2,0E0,8.174539E-2,7.488325E-2,1.05151325E-1,3.831237E-2,0E0,0E0,5.5962395E-2,8.739403E-2,0E0,2.1835312E-2,0E0,4.4679884E-2,1.8410144E-2,0E0,8.3790466E-2,6.802172E-2,0E0,0E0,0E0,0E0,0E0,1.1752978E-2,1.0201873E-2,0E0,2.6884764E-2,6.657496E-2,2.2839993E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,20,20,22,22,23,23,25,25,26,26,32,32,33,33,35,35,36,36,37,37],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,18,20,22,24,-1,-1,26,28,-1,30,-1,32,34,-1,36,38,-1,-1,-1,-1,-1,40,42,-1,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.818E3,4.966421E6,3.0451374E9,1.8733E4,-5.862529E-3,1.5400032E-2,2.7E1,1E0,2.6056657E0,7.3240237E8,4.615366E-3,7.822023E7,4.1133E4,1.1825188E6,1.515891E0,1.4275579E-3,-1.494325E-3,1.8656379E6,2.1564245E0,8.299366E-6,1.2555326E-3,-8.117734E-3,4.6994E4,4.1170962E5,-2.6914102E-3,7.3328E4,2.1807466E0,-7.0339693E-3,1.4734761E-4,2.7051365E-3,6.7521413E-3,8.7301875E-4,4.4419664E8,8.264212E-2,3.2957438E-3,1.4654956E2,9.9E1,6.8915665E-1,5.524891E-3,-4.429584E-3,-1.3899816E-3,1.3581467E-3,-8.9692714E-4,1.6242579E-4,3.5069243E-3,-2.605049E-3,3.84153E-4,-1.1644291E-3,2.5233394E-3],"split_indices":[2,27,12,9,0,0,8,112,52,7,0,44,28,49,40,0,0,27,53,0,41,0,28,32,0,1,52,0,0,0,0,0,7,37,0,4,51,55,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,2.14E2,2.3E1,2.06E2,8E0,5E0,1.8E1,1.5E2,5.6E1,1.2E1,6E0,1.26E2,2.4E1,3E1,2.6E1,5E0,7E0,1.13E2,1.3E1,9E0,1.5E1,9E0,2.1E1,2.1E1,5E0,9.2E1,2.1E1,5E0,8E0,7E0,8E0,6E0,1.5E1,1.5E1,6E0,2.3E1,6.9E1,1.2E1,9E0,1E1,5E0,9E0,6E0,7E0,1.6E1,2.1E1,4.8E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[3.2002467E-4,-7.902102E-3,1.1952086E-1,-8.5471856E-4,-8.225991E-3,1.1958742E-2,2.5203422E-4,3.1530454E-3,-6.6283722E-3,-9.27496E-3,5.141411E-2,-1.5067287E-3,-8.637758E-2,3.4637094E-2,6.9754114E-3,3.9455924E-2,-1.28732845E-2,-4.1137252E-4,-6.5587163E-3,1.056626E-2,7.0082955E-2,-2.5946598E-3,6.1508443E-2,2.4495268E-2,-2.3799967E-2,3.0748725E-3,-7.249364E-3,1.9443278E-3,4.8626037E-3,3.8877963E-3,5.9682125E-4,4.5820732E-5,3.4521844E-3,-2.8095043E-3,-3.027248E-4,2.2284123E-3,-1.327219E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,-1,-1,-1,9,-1,11,13,15,17,19,-1,21,23,-1,-1,25,27,-1,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4302527E-1,2.8518882E-1,2.6408753E-1,1.3165992E-1,0E0,0E0,0E0,1.3183059E-1,0E0,1.04686275E-1,7.261307E-2,7.5004615E-2,7.169211E-2,3.3387937E-2,0E0,7.840526E-2,5.203059E-2,0E0,0E0,2.6159693E-2,1.1638135E-2,0E0,3.345096E-2,3.5371542E-2,6.405863E-2,0E0,2.659716E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,-1,-1,-1,10,-1,12,14,16,18,20,-1,22,24,-1,-1,26,28,-1,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,1.2754595E8,1E0,5.3159845E6,-8.225991E-3,1.1958742E-2,2.5203422E-4,1.5811304E6,-6.6283722E-3,1.2360571E6,7.737766E2,2.0242085E8,7.57E2,1.9E1,6.9754114E-3,4.855866E-5,1.2694E4,-4.1137252E-4,-6.5587163E-3,1.3257E4,9.4655323E-1,-2.5946598E-3,8.869983E5,1.2262E4,2.9044975E5,3.0748725E-3,1.12E2,1.9443278E-3,4.8626037E-3,3.8877963E-3,5.9682125E-4,4.5820732E-5,3.4521844E-3,-2.8095043E-3,-3.027248E-4,2.2284123E-3,-1.327219E-3],"split_indices":[51,44,63,27,0,0,0,27,0,27,51,7,0,8,0,37,9,0,0,9,26,0,31,9,27,0,10,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.31E2,1.5E1,2.23E2,8E0,6E0,9E0,2.18E2,5E0,1.74E2,4.4E1,1.59E2,1.5E1,3.9E1,5E0,3.4E1,1.25E2,7E0,8E0,2.4E1,1.5E1,6E0,2.8E1,2.8E1,9.7E1,5E0,1.9E1,1E1,5E0,1.8E1,1E1,2E1,8E0,2.9E1,6.8E1,5E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-6.312341E-3,3.4359295E-2,-2.2686789E-2,1.123344E-1,2.4820685E-3,-1.0179292E-1,-5.1337616E-3,5.5208296E-2,1.066105E-2,-5.0879912E-3,1.7802525E-2,-3.6083136E-2,-1.6664755E-2,-1.8530576E-2,4.107822E-2,4.4475202E-3,2.2023467E-4,3.285679E-2,-2.0771902E-3,-7.011204E-3,2.5622718E-2,5.0786324E-2,-3.0009942E-2,-3.1930596E-2,8.084144E-2,6.0303997E-2,9.073014E-3,4.4155736E-3,-1.868767E-2,6.912706E-4,4.2469217E-3,-4.015726E-2,2.7171706E-3,1.6865939E-3,-4.847341E-3,8.07053E-4,1.1103484E-1,7.713592E-4,4.005631E-3,1.3457953E-3,-9.893061E-4,-2.49381E-3,1.2241047E-3,-7.7736184E-2,-2.520406E-2,6.962537E-3,2.6697095E-3,-1.18278764E-1,1.3423382E-4,3.193095E-2,-3.6586724E-2,-2.7819541E-3,-6.2403334E-3,-5.752616E-4,3.2149684E-3,-3.8455848E-3,-1.1132379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,-1,-1,17,19,-1,21,23,-1,-1,25,-1,-1,27,29,31,33,35,37,39,-1,41,-1,-1,43,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,47,49,-1,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5994768E-1,1.7122966E-1,2.3691984E-1,1.287261E-1,9.034427E-2,5.321272E-1,8.805336E-2,3.2296825E-2,0E0,0E0,4.4780295E-2,1.9554758E-1,0E0,8.905265E-2,9.5479414E-2,0E0,0E0,2.3606393E-2,0E0,0E0,5.9495717E-2,2.3174983E-2,8.9437634E-2,6.6893384E-2,3.792286E-2,1.9430168E-2,1.4183074E-2,0E0,2.1641051E-2,0E0,0E0,4.636629E-2,0E0,0E0,0E0,0E0,2.2329152E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.888767E-2,4.2140204E-2,0E0,0E0,9.669885E-3,0E0,2.0261072E-2,2.8603725E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,10,10,11,11,13,13,14,14,17,17,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,31,31,36,36,43,43,44,44,47,47,49,49,50,50],"right_children":[2,4,6,8,10,12,14,16,-1,-1,18,20,-1,22,24,-1,-1,26,-1,-1,28,30,32,34,36,38,40,-1,42,-1,-1,44,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,48,50,-1,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1],"split_conditions":[4.15E2,2.585911E5,8.7722336E4,1.544E3,1.4205997E-6,3.2647366E8,1.3335946E6,3.5211E4,1.066105E-2,-5.0879912E-3,1.4963856E0,1.6771895E2,-1.6664755E-2,1.8836116E5,3E0,4.4475202E-3,2.2023467E-4,1.2557897E7,-2.0771902E-3,-7.011204E-3,1.7863992E0,2.035E3,4.6328125E0,2.0229886E0,1.6914554E7,6.1895325E5,9.123214E-1,4.4155736E-3,1.306E3,6.912706E-4,4.2469217E-3,1.2555331E6,2.7171706E-3,1.6865939E-3,-4.847341E-3,8.07053E-4,5.25191E5,7.713592E-4,4.005631E-3,1.3457953E-3,-9.893061E-4,-2.49381E-3,1.2241047E-3,6.124014E5,8.609E4,6.962537E-3,2.6697095E-3,8.033374E5,1.3423382E-4,3.8372688E8,1.7371938E0,-2.7819541E-3,-6.2403334E-3,-5.752616E-4,3.2149684E-3,-3.8455848E-3,-1.1132379E-3],"split_indices":[0,27,31,2,37,7,27,1,0,0,56,51,0,31,8,0,0,44,0,0,55,2,52,53,44,27,26,0,0,0,0,47,0,0,0,0,28,0,0,0,0,0,0,27,28,0,0,47,0,5,53,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.38E2,6.8E1,1.7E2,1.9E1,4.9E1,3E1,1.4E2,1.4E1,5E0,5E0,4.4E1,2.5E1,5E0,1.09E2,3.1E1,7E0,7E0,3.6E1,8E0,8E0,1.7E1,1.5E1,9.4E1,1.1E1,2E1,1.6E1,2E1,6E0,1.1E1,9E0,6E0,8.5E1,9E0,6E0,5E0,7E0,1.3E1,7E0,9E0,1.2E1,8E0,6E0,5E0,2.3E1,6.2E1,6E0,7E0,1.5E1,8E0,1E1,5.2E1,5E0,1E1,5E0,5E0,9E0,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.2509951E-2,-3.340307E-3,-8.5033275E-2,-1.0213723E-1,2.1932283E-3,-2.1143728E-1,3.8415678E-2,-8.891299E-4,-7.5544757E-3,-2.6588186E-3,4.2022597E-3,-2.6379628E-3,-1.7140137E-2,4.602451E-3,-9.957048E-4,1.8454585E-2,-2.3968548E-2,-5.2199166E-2,2.9298492E-2,2.0693133E-2,-4.227539E-2,1.2881645E-3,-6.8174107E-3,2.1647932E-2,4.927198E-3,5.7054763E-3,-7.815879E-3,-6.7026056E-3,-2.3207454E-2,3.585799E-2,-1.32738575E-2,1.2173683E-3,-3.4479902E-3,-5.9873138E-2,4.435179E-3,2.687229E-3,7.6365826E-4,7.9083705E-4,-3.8872645E-3,-1.7270558E-4,-3.6729886E-3,-4.8157133E-4,3.460885E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,-1,-1,-1,17,19,21,23,25,27,-1,-1,29,-1,-1,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5399902E-1,1.1339013E-1,4.1383803E-1,5.598568E-2,8.722963E-2,3.1311685E-1,5.616422E-2,0E0,0E0,8.5027464E-2,0E0,0E0,0E0,0E0,0E0,7.431482E-2,7.7730566E-2,1.0825363E-1,4.841564E-2,8.547252E-2,1.3243914E-1,0E0,0E0,3.8686506E-2,0E0,0E0,5.7031963E-2,0E0,5.9050635E-2,2.3693725E-2,5.3197827E-2,0E0,0E0,2.9925734E-2,3.7983216E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,15,15,16,16,17,17,18,18,19,19,20,20,23,23,26,26,28,28,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,-1,-1,-1,18,20,22,24,26,28,-1,-1,30,-1,-1,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0232127E12,9.235376E5,2.095716E0,3.461353E1,2.013055E3,9E0,2.7438753E0,-8.891299E-4,-7.5544757E-3,1.2141942E6,4.2022597E-3,-2.6379628E-3,-1.7140137E-2,4.602451E-3,-9.957048E-4,4.945055E-2,1.3257E4,3.1E2,6.007572E2,3.5540915E0,1.3802E4,1.2881645E-3,-6.8174107E-3,3.4572864E0,4.927198E-3,5.7054763E-3,2.0968523E0,-6.7026056E-3,1.2779E4,1E0,8.5913794E5,1.2173683E-3,-3.4479902E-3,1.7284313E0,1.5506504E6,2.687229E-3,7.6365826E-4,7.9083705E-4,-3.8872645E-3,-1.7270558E-4,-3.6729886E-3,-4.8157133E-4,3.460885E-3],"split_indices":[30,44,53,51,51,3,55,0,0,50,0,0,0,0,0,57,9,0,51,56,9,0,0,55,0,0,52,0,10,111,31,0,0,52,31,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.31E2,2.06E2,2.5E1,1E1,1.96E2,1.2E1,1.3E1,5E0,5E0,1.87E2,9E0,7E0,5E0,6E0,7E0,9.4E1,9.3E1,1.2E1,8.2E1,2.7E1,6.6E1,7E0,5E0,7.6E1,6E0,5E0,2.2E1,9E0,5.7E1,5.4E1,2.2E1,1.5E1,7E0,2.4E1,3.3E1,2.3E1,3.1E1,1.6E1,6E0,7E0,1.7E1,2.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.9159109E-3,-1.1571262E-2,1.3337876E-1,-4.812708E-3,-8.723134E-3,1.1563936E-2,1.528213E-3,-2.0623505E-4,-7.520162E-3,2.1120919E-2,-3.0498145E-2,3.6733758E-2,-2.9544197E-2,-9.558645E-3,-1.7011562E-2,2.0840747E-2,1.1489627E-1,2.0216875E-2,-6.38414E-2,4.974289E-2,-3.495957E-2,1.2137694E-2,3.6713546E-3,6.3568894E-3,2.1615361E-3,-1.3310902E-3,3.024109E-3,-5.2396795E-3,6.43544E-5,3.7073633E-3,7.0081156E-4,-7.215711E-2,-5.6519876E-3,1.0369484E-3,-1.5552258E-3,-6.1262515E-3,-2.4000276E-3,3.284273E-3,-1.308106E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,-1,-1,9,-1,11,13,15,17,-1,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1375325E-1,2.744191E-1,1.87215E-1,1.6186148E-1,0E0,0E0,0E0,1.376047E-1,0E0,9.996355E-2,2.117795E-1,1.168603E-1,5.2490644E-2,0E0,1.0118206E-1,4.1716002E-2,2.118218E-2,3.2270182E-2,6.1677314E-2,1.8714078E-2,7.1543366E-2,3.7246868E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7744293E-2,7.1776375E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,31,31,32,32],"right_children":[2,4,6,8,-1,-1,-1,10,-1,12,14,16,18,-1,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,1.2754595E8,5.939257E7,4.966421E6,-8.723134E-3,1.1563936E-2,1.528213E-3,3.544243E8,-7.520162E-3,3.4572864E0,6.523944E-2,1.518E3,1.16E2,-9.558645E-3,1.2414E4,2.5245471E2,9.0763354E-1,1E0,2.3950179E0,7.57E2,1.6601E4,9.7409576E-1,3.6713546E-3,6.3568894E-3,2.1615361E-3,-1.3310902E-3,3.024109E-3,-5.2396795E-3,6.43544E-5,3.7073633E-3,7.0081156E-4,2.0762905E6,2.1159E4,1.0369484E-3,-1.5552258E-3,-6.1262515E-3,-2.4000276E-3,3.284273E-3,-1.308106E-3],"split_indices":[51,44,44,27,0,0,0,7,0,55,40,2,10,0,9,51,26,111,53,0,9,40,0,0,0,0,0,0,0,0,0,44,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.38E2,2.23E2,1.5E1,2.16E2,7E0,6E0,9E0,2.11E2,5E0,1.24E2,8.7E1,9.5E1,2.9E1,5E0,8.2E1,8E1,1.5E1,1.2E1,1.7E1,1.7E1,6.5E1,7.1E1,9E0,1E1,5E0,6E0,6E0,9E0,8E0,8E0,9E0,2.8E1,3.7E1,5.8E1,1.3E1,5E0,2.3E1,8E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.1661485E-3,-5.8759823E-3,8.468231E-2,8.062369E-3,-6.059263E-2,1.0962028E-2,1.0081059E-2,-4.359651E-2,1.9875474E-2,-1.922805E-1,3.029579E-3,3.112532E-3,-1.0365401E-3,-7.38619E-2,1.6425601E-3,4.758938E-2,-7.0059802E-3,-1.437118E-2,-3.2730326E-3,-3.922784E-2,8.7269E-2,-9.943903E-3,-6.958107E-3,3.194335E-2,1.11244544E-1,3.8032603E-2,-1.9650146E-2,-6.396126E-3,-5.668692E-3,5.519115E-3,1.6953457E-3,1.5776231E-3,-2.9892158E-3,5.657085E-2,-4.7979834E-3,5.113497E-4,7.462191E-3,6.0175906E-4,3.6555852E-3,-7.26242E-2,1.263697E-3,-1.9519359E-3,1.4071501E-3,-1.3625382E-3,7.154357E-2,1.969662E-2,-3.8620203E-3,-4.725409E-3,-1.0521729E-3,2.8822131E-2,-3.324753E-2,4.5776768E-3,2.1314838E-3,-2.4045749E-4,3.0185317E-3,2.5969502E-3,2.6166355E-4,-2.7522163E-3,1.2209822E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,-1,21,-1,23,25,-1,-1,27,29,31,-1,33,35,37,39,41,-1,-1,-1,-1,-1,43,45,-1,-1,-1,-1,47,49,-1,-1,-1,51,53,-1,-1,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6440822E-1,1.7284966E-1,2.57552E-1,1.1132207E-1,3.8598308E-1,0E0,3.2773778E-2,8.503621E-2,1.10591955E-1,2.038545E-1,1.1733508E-1,0E0,0E0,1.2701453E-1,0E0,6.9150835E-2,4.402747E-2,0E0,0E0,6.1664E-2,1.4591239E-2,4.029481E-2,0E0,5.456226E-2,7.39903E-2,1.6979981E-2,6.6565245E-2,2.4933167E-2,0E0,0E0,0E0,0E0,0E0,4.89418E-2,5.119606E-2,0E0,0E0,0E0,0E0,2.4511158E-2,4.2805575E-2,0E0,0E0,0E0,1.7757043E-2,2.4006817E-2,0E0,0E0,0E0,1.622228E-2,3.5178423E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,33,33,34,34,39,39,40,40,44,44,45,45,49,49,50,50],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,-1,22,-1,24,26,-1,-1,28,30,32,-1,34,36,38,40,42,-1,-1,-1,-1,-1,44,46,-1,-1,-1,-1,48,50,-1,-1,-1,52,54,-1,-1,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,4.3951338E11,1E0,7.23E2,8.87885E5,1.0962028E-2,4.9308786E-1,2.2901595E0,3.5540915E0,2.0651913E0,1.3143399E6,3.112532E-3,-1.0365401E-3,1.7113637E0,1.6425601E-3,2.039E3,1.2694E4,-1.437118E-2,-3.2730326E-3,7.24E4,3.2433453E5,4.71E2,-6.958107E-3,1.0661045E6,3.1924096E-1,1.2268E4,1.5064E4,1.00531176E8,-5.668692E-3,5.519115E-3,1.6953457E-3,1.5776231E-3,-2.9892158E-3,2.8684334E4,1.0284E5,5.113497E-4,7.462191E-3,6.0175906E-4,3.6555852E-3,2.2745E4,1.8733E4,-1.9519359E-3,1.4071501E-3,-1.3625382E-3,2.92E2,3.6529312E5,-3.8620203E-3,-4.725409E-3,-1.0521729E-3,9E0,2.38502E0,4.5776768E-3,2.1314838E-3,-2.4045749E-4,3.0185317E-3,2.5969502E-3,2.6166355E-4,-2.7522163E-3,1.2209822E-3],"split_indices":[51,30,63,2,1,0,41,52,56,53,27,0,0,53,0,2,9,0,0,10,31,2,0,31,26,9,9,44,0,0,0,0,0,32,28,0,0,0,0,10,9,0,0,0,0,32,0,0,0,3,52,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.46E2,2.25E2,2.1E1,1.8E2,4.5E1,6E0,1.5E1,3.3E1,1.47E2,1.4E1,3.1E1,5E0,1E1,2.4E1,9E0,7.2E1,7.5E1,6E0,8E0,2.1E1,1E1,1.4E1,1E1,5.9E1,1.3E1,1.6E1,5.9E1,1.6E1,5E0,5E0,5E0,8E0,6E0,3.5E1,2.4E1,5E0,8E0,1.1E1,5E0,1.6E1,4.3E1,8E0,8E0,5E0,3E1,1.9E1,5E0,9E0,7E0,2.4E1,1.9E1,1.2E1,1.8E1,1.3E1,6E0,1E1,1.4E1,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.3499351E-3,1.3085562E-2,-6.47757E-2,-4.2568754E-2,2.2390988E-2,-2.4430903E-2,-1.2439889E-2,-8.909966E-3,-1.02956556E-1,-5.1876232E-2,2.8812265E-2,-5.6739908E-2,3.6838185E-2,-3.3177938E-3,1.5897356E-2,-1.7984286E-3,-6.71291E-3,2.5484748E-3,-6.93451E-3,3.3241395E-2,-2.9324777E-3,3.403951E-4,-7.609242E-2,3.0612063E-3,-2.491705E-4,2.891955E-3,-1.2962293E-3,5.88844E-2,2.2026189E-2,-4.933205E-2,-5.272692E-3,3.939422E-2,1.0982689E-1,1.6024359E-2,5.0655184E-3,-3.9498494E-3,1.5916668E-4,7.354477E-2,-7.394126E-4,2.4832012E-3,7.4191513E-3,4.1124425E-3,5.8041044E-2,4.370288E-3,1.5425117E-3,1.0200192E-3,-1.8343405E-3,1.0243985E-3,-6.0170976E-4,3.2634332E-3,1.3461232E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,25,-1,-1,-1,-1,27,-1,-1,29,-1,-1,-1,-1,31,33,35,-1,37,39,41,-1,-1,-1,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9563007E-1,1.1178099E-1,3.1689864E-1,6.2092636E-2,8.924253E-2,6.741512E-2,0E0,3.4858663E-2,2.591788E-2,1.7438415E-1,7.2822586E-2,2.794569E-2,1.6159631E-2,0E0,3.629892E-2,0E0,0E0,0E0,0E0,4.558043E-2,0E0,0E0,1.2810133E-2,0E0,0E0,0E0,0E0,4.4324487E-2,6.05331E-2,2.4580844E-2,0E0,5.0766055E-2,2.9881462E-2,5.431573E-2,0E0,0E0,0E0,1.5073784E-2,1.7714364E-2,0E0,0E0,2.8234467E-2,1.8720381E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,14,14,19,19,22,22,27,27,28,28,29,29,31,31,32,32,33,33,37,37,38,38,41,41,42,42],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,26,-1,-1,-1,-1,28,-1,-1,30,-1,-1,-1,-1,32,34,36,-1,38,40,42,-1,-1,-1,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,7.23E2,2.363013E6,3.31E2,1.4827905E2,2.6056657E0,-1.2439889E-2,1.4693E4,1.7280551E0,2.790375E3,7.1656E4,1.04671E5,1.1243519E-1,-3.3177938E-3,1.5773E4,-1.7984286E-3,-6.71291E-3,2.5484748E-3,-6.93451E-3,2.5542571E8,-2.9324777E-3,3.403951E-4,1.067509E6,3.0612063E-3,-2.491705E-4,2.891955E-3,-1.2962293E-3,2.3082722E2,1.7951E4,1.4456801E6,-5.272692E-3,7.0516006E8,1.8013916E-3,1E0,5.0655184E-3,-3.9498494E-3,1.5916668E-4,4.3161097E5,3.503662E2,2.4832012E-3,7.4191513E-3,8E2,2.9646394E0,4.370288E-3,1.5425117E-3,1.0200192E-3,-1.8343405E-3,1.0243985E-3,-6.0170976E-4,3.2634332E-3,1.3461232E-4],"split_indices":[9,2,27,10,46,52,0,9,52,32,10,28,37,0,9,0,0,0,0,7,0,0,9,0,0,0,0,51,2,46,0,5,41,108,0,0,0,31,4,0,0,0,55,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.5E2,2.13E2,3.7E1,3E1,1.83E2,3.2E1,5E0,2E1,1E1,1.4E1,1.69E2,2.1E1,1.1E1,5E0,1.5E1,5E0,5E0,7E0,7E0,1.62E2,7E0,5E0,1.6E1,6E0,5E0,7E0,8E0,4.8E1,1.14E2,1.1E1,5E0,3.6E1,1.2E1,1.08E2,6E0,6E0,5E0,1.9E1,1.7E1,7E0,5E0,8.5E1,2.3E1,1.1E1,8E0,1.1E1,6E0,4.1E1,4.4E1,1.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[2.450434E-3,-4.3544434E-3,8.888746E-2,9.684755E-4,-7.814738E-3,1.0431126E-2,-1.6103103E-3,-1.1662639E-2,2.7620709E-2,-1.9504932E-3,2.13035E-3,-5.093219E-3,-7.695887E-3,1.0112113E-2,7.592826E-2,2.5133584E-2,-2.9359996E-2,2.6755821E-2,-1.731637E-2,2.6887283E-5,1.10651E-1,4.500072E-2,-5.9324797E-2,4.0618666E-2,-4.967616E-2,-6.067724E-3,6.406154E-2,-2.0932888E-3,8.721932E-4,6.437832E-3,2.8283983E-3,1.6302554E-2,9.700774E-2,-2.6113426E-4,-4.1511734E-3,2.6653612E-3,2.1500151E-5,-8.725127E-3,-8.1284605E-2,2.043128E-3,-3.326585E-2,7.6320325E-4,4.0876498E-3,1.8148433E-3,-8.9748274E-4,5.6146323E-3,7.511476E-4,-3.1666667E-3,6.328949E-4,-7.1963165E-3,-2.5648493E-3,-3.386099E-3,1.0201903E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,15,-1,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,-1,-1,-1,-1,43,45,-1,-1,-1,-1,47,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4508262E-1,2.0611031E-1,2.324029E-1,7.5394504E-2,0E0,0E0,2.6458282E-2,1.5790746E-1,6.012961E-2,0E0,0E0,1.0856347E-1,0E0,2.5114944E-2,4.963296E-2,1.1341411E-1,1.1859703E-1,4.2021908E-2,2.3304664E-2,0E0,1.1056349E-2,7.9013854E-2,2.2248168E-2,1.4074314E-2,8.2132265E-2,2.8057408E-2,1.8518701E-2,0E0,0E0,0E0,0E0,3.2010823E-2,3.9918467E-2,0E0,0E0,0E0,0E0,4.1580107E-2,5.971968E-2,0E0,1.948833E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,37,37,38,38,40,40],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,16,-1,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,-1,-1,-1,-1,44,46,-1,-1,-1,-1,48,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3908939E3,1.2754595E8,1E0,2.148789E0,-7.814738E-3,1.0431126E-2,1.4062E4,4.966421E6,1.5452969E6,-1.9504932E-3,2.13035E-3,2.747292E8,-7.695887E-3,1.9776594E9,2.0116584E8,1.8344E4,1.2694E4,7.864689E6,7.7573473E9,2.6887283E-5,3.6E1,1.8520895E2,5.1454154E8,1.4515E4,1.9011765E0,4.8162012E5,1.569966E-1,-2.0932888E-3,8.721932E-4,6.437832E-3,2.8283983E-3,1.7046307E0,1.813E3,-2.6113426E-4,-4.1511734E-3,2.6653612E-3,2.1500151E-5,1.4832215E0,3.6327022E5,2.043128E-3,1.6999166E5,7.6320325E-4,4.0876498E-3,1.8148433E-3,-8.9748274E-4,5.6146323E-3,7.511476E-4,-3.1666667E-3,6.328949E-4,-7.1963165E-3,-2.5648493E-3,-3.386099E-3,1.0201903E-4],"split_indices":[51,44,63,53,0,0,9,27,27,0,0,7,0,5,7,9,9,44,5,0,8,51,5,10,52,50,57,0,0,0,0,53,0,0,0,0,0,52,31,0,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,2.28E2,1.7E1,2.22E2,6E0,6E0,1.1E1,1.51E2,7.1E1,6E0,5E0,1.46E2,5E0,5.3E1,1.8E1,6.5E1,8.1E1,3.3E1,2E1,6E0,1.2E1,5.3E1,1.2E1,1.8E1,6.3E1,1.8E1,1.5E1,1.1E1,9E0,6E0,6E0,3.5E1,1.8E1,5E0,7E0,1.2E1,6E0,2.8E1,3.5E1,6E0,1.2E1,6E0,9E0,2.1E1,1.4E1,1.3E1,5E0,7E0,2.1E1,7E0,2.8E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-3.210202E-3,-1.8959306E-2,2.665822E-2,-5.9659095E-3,-1.2867008E-1,3.5084236E-2,-3.2576355E-3,5.6370376E-3,-7.474698E-2,-1.1674524E-2,7.151345E-4,2.6633656E-2,4.8137815E-3,1.6179152E-2,-5.792881E-2,5.973094E-4,-1.10827416E-1,1.854495E-2,3.6612265E-3,-3.6545295E-2,2.8577268E-2,-7.752752E-2,-2.6199222E-4,-2.4066367E-3,-8.482597E-3,5.72364E-2,5.532539E-3,2.2463478E-2,-4.67613E-3,6.7710444E-2,8.194683E-3,-1.5587468E-3,-4.511769E-3,3.4943428E-3,4.7616096E-4,-3.9574895E-2,2.123157E-2,-1.3225283E-3,2.870047E-3,3.1525582E-2,1.273456E-1,-3.7367619E-3,3.8204337E-3,-3.2334384E-3,4.4478424E-4,2.0982996E-3,1.8651445E-3,4.185614E-3,2.2116915E-4,8.314305E-3,2.2132564E-3,1.7032529E-3,-9.029145E-4,-8.7661104E-4,1.1245103E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,19,21,-1,23,25,-1,27,29,31,-1,-1,-1,33,35,37,-1,39,41,-1,-1,-1,-1,43,45,-1,-1,47,49,51,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.15753636E-1,2.2874036E-1,7.336932E-2,1.1614051E-1,3.2408792E-1,4.500316E-2,0E0,8.4702045E-2,6.8139665E-2,0E0,0E0,3.0269511E-2,0E0,7.164222E-2,1.793582E-2,0E0,5.657068E-2,3.1905353E-2,0E0,8.554815E-2,6.963702E-2,9.12302E-3,0E0,0E0,0E0,1.4309958E-2,3.5539243E-2,2.7463691E-2,0E0,6.072204E-2,5.419033E-2,0E0,0E0,0E0,0E0,2.1358805E-2,1.8043453E-2,0E0,0E0,3.218054E-2,4.005386E-2,3.591146E-2,0E0,0E0,0E0,0E0,1.1299527E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,13,13,14,14,16,16,17,17,19,19,20,20,21,21,25,25,26,26,27,27,29,29,30,30,35,35,36,36,39,39,40,40,41,41,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,20,22,-1,24,26,-1,28,30,32,-1,-1,-1,34,36,38,-1,40,42,-1,-1,-1,-1,44,46,-1,-1,48,50,52,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.148789E0,2.6146525E6,1.515891E0,3.6235E4,3.6170732E-4,9.5082015E-1,-3.2576355E-3,1.9577E4,1.3318E4,-1.1674524E-2,7.151345E-4,2.7608912E1,4.8137815E-3,7.23E2,1.2153666E6,5.973094E-4,8.5637856E-1,3.01E2,3.6612265E-3,1.3579175E0,2.4589229E8,5.489421E8,-2.6199222E-4,-2.4066367E-3,-8.482597E-3,2.2243262E5,1.059E3,1.2917E4,-4.67613E-3,7.689625E-1,2.7034E4,-1.5587468E-3,-4.511769E-3,3.4943428E-3,4.7616096E-4,7.864689E6,1.9776594E9,-1.3225283E-3,2.870047E-3,8.014E3,1.6403488E5,4.0485874E-1,3.8204337E-3,-3.2334384E-3,4.4478424E-4,2.0982996E-3,5.941498E11,4.185614E-3,2.2116915E-4,8.314305E-3,2.2132564E-3,1.7032529E-3,-9.029145E-4,-8.7661104E-4,1.1245103E-3],"split_indices":[53,27,40,10,37,26,0,9,9,0,0,56,0,2,46,0,26,0,0,56,7,5,0,0,0,32,2,1,0,57,10,0,0,0,0,44,5,0,0,12,46,26,0,0,0,0,30,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.44E2,1.6E2,8.4E1,1.44E2,1.6E1,7.8E1,6E0,1.24E2,2E1,8E0,8E0,7.1E1,7E0,1.07E2,1.7E1,6E0,1.4E1,6.3E1,8E0,2E1,8.7E1,1.2E1,5E0,9E0,5E0,1.5E1,4.8E1,1.1E1,9E0,2.9E1,5.8E1,5E0,7E0,1E1,5E0,1.2E1,3.6E1,5E0,6E0,1.9E1,1E1,5.1E1,7E0,7E0,5E0,1.5E1,2.1E1,5E0,1.4E1,5E0,5E0,1.4E1,3.7E1,1.1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.0243137E-3,-8.909894E-3,1.0433163E-1,1.0304671E-4,-1.2777449E-1,1.0653329E-2,1.0895023E-2,-6.501491E-2,5.391292E-3,-1.4298201E-2,-1.4182368E-2,-1.1761932E-3,2.0768761E-3,-4.735524E-3,6.406927E-5,1.0293759E-2,-7.844077E-2,1.9874754E-3,-3.1694965E-3,-8.871692E-3,2.4089878E-2,-6.2149414E-3,-2.696961E-4,-2.3865808E-2,3.367709E-2,4.2648554E-2,-1.9943694E-2,7.376088E-3,-4.870415E-2,5.0186113E-2,-2.9356868E-4,3.2186136E-2,4.9388763E-3,1.5762333E-2,-5.8866493E-2,3.2201034E-3,-8.390249E-4,-5.277763E-4,-3.3977565E-3,3.0681007E-3,5.440172E-4,1.9643942E-3,-1.7207663E-3,-5.6944403E-4,2.3821953E-3,-4.4769915E-3,-2.7223595E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,-1,27,29,31,33,35,37,39,-1,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0281222E-1,2.432027E-1,2.0697768E-1,7.335478E-2,3.3455938E-1,0E0,1.550479E-2,4.2671874E-2,8.1771284E-2,3.9050516E-2,0E0,0E0,0E0,0E0,0E0,4.965817E-2,4.6509996E-2,0E0,0E0,5.1204257E-2,8.992589E-2,0E0,0E0,4.6143066E-2,1.4259309E-2,5.035332E-2,4.6790563E-2,4.6251982E-2,3.165386E-2,9.388659E-3,0E0,5.6088902E-2,0E0,1.9609611E-2,3.3565063E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,15,15,16,16,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,-1,28,30,32,34,36,38,40,-1,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,1.9E1,3.6007138E2,1.1744206E6,1.2916E4,1.0653329E-2,1.019415E3,5.514485E5,8.836406E0,2.7060036E11,-1.4182368E-2,-1.1761932E-3,2.0768761E-3,-4.735524E-3,6.406927E-5,1.0926206E2,7.35721E5,1.9874754E-3,-3.1694965E-3,2.170343E0,1.037239E7,-6.2149414E-3,-2.696961E-4,2.7958122E8,2.9420671E0,2.5932198E6,1E0,1.9781647E5,1.6999155E0,1.052641E7,-2.9356868E-4,1.0830879E6,4.9388763E-3,2.2737896E0,1.6362E4,3.2201034E-3,-8.390249E-4,-5.277763E-4,-3.3977565E-3,3.0681007E-3,5.440172E-4,1.9643942E-3,-1.7207663E-3,-5.6944403E-4,2.3821953E-3,-4.4769915E-3,-2.7223595E-4],"split_indices":[51,3,57,44,10,0,57,27,55,30,0,0,0,0,0,51,1,0,0,53,49,0,0,5,56,31,111,32,53,44,0,31,0,52,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,2.26E2,1.6E1,2.11E2,1.5E1,6E0,1E1,1.5E1,1.96E2,1E1,5E0,5E0,5E0,9E0,6E0,1.86E2,1E1,5E0,5E0,7.8E1,1.08E2,5E0,5E0,5.8E1,2E1,7.6E1,3.2E1,2.6E1,3.2E1,1.4E1,6E0,6.7E1,9E0,1.7E1,1.5E1,7E0,1.9E1,1.4E1,1.8E1,9E0,5E0,5.8E1,9E0,1E1,7E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.791674E-3,-4.058127E-3,8.4842354E-2,1.73319E-3,-6.5461732E-3,9.705196E-3,8.107527E-3,5.2509503E-3,-4.782371E-3,2.677344E-3,-1.413929E-3,1.4895666E-2,-3.7075955E-2,9.957954E-4,5.625674E-2,-6.935337E-2,7.068111E-4,-4.8804913E-2,7.3179547E-3,7.68314E-3,3.7255768E-2,-9.834465E-2,-3.1051264E-4,-1.286569E-4,-3.2238946E-3,3.604323E-3,2.0185746E-3,6.0613696E-2,-4.157835E-3,-2.6335232E-3,-5.7429182E-3,-1.1411713E-4,2.428614E-3,1.1871709E-3,3.68476E-3,-3.1165835E-3,1.5887205E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,-1,-1,9,11,-1,-1,-1,13,15,17,19,21,-1,23,25,-1,27,29,-1,-1,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3860388E-1,1.865822E-1,1.8835454E-1,8.39768E-2,0E0,0E0,2.8163398E-2,8.79137E-2,0E0,0E0,0E0,1.0055387E-1,6.919338E-2,4.191709E-2,8.906984E-2,4.459788E-2,0E0,1.5436053E-2,4.524623E-2,0E0,3.7978124E-2,1.226297E-2,0E0,0E0,0E0,0E0,2.6314259E-2,1.5936479E-2,4.0815894E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,11,11,12,12,13,13,14,14,15,15,17,17,18,18,20,20,21,21,26,26,27,27,28,28],"right_children":[2,4,6,8,-1,-1,10,12,-1,-1,-1,14,16,18,20,22,-1,24,26,-1,28,30,-1,-1,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,1.2754595E8,1E0,4.966421E6,-6.5461732E-3,9.705196E-3,1.2307982E-1,4.767898E0,-4.782371E-3,2.677344E-3,-1.413929E-3,4.6253732E2,2.3950179E0,1.4760101E0,1.25077E5,1E0,7.068111E-4,4.4358948E-1,1.3846154E0,7.68314E-3,2.0868962E7,1.6908462E5,-3.1051264E-4,-1.286569E-4,-3.2238946E-3,3.604323E-3,2.4646638E6,1.777E3,1.5917099E0,-2.6335232E-3,-5.7429182E-3,-1.1411713E-4,2.428614E-3,1.1871709E-3,3.68476E-3,-3.1165835E-3,1.5887205E-3],"split_indices":[51,44,63,27,0,0,37,55,0,0,0,4,53,52,28,15,0,26,53,0,49,32,0,0,0,0,27,2,55,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,2.27E2,1.8E1,2.19E2,8E0,6E0,1.2E1,2.13E2,6E0,5E0,7E0,1.74E2,3.9E1,1.31E2,4.3E1,2.4E1,1.5E1,1.4E1,1.17E2,5E0,3.8E1,1.6E1,8E0,5E0,9E0,7E0,1.1E2,2.4E1,1.4E1,8E0,8E0,1.02E2,8E0,1E1,1.4E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-8.036546E-4,-9.588475E-3,4.293042E-2,-2.4495784E-3,-9.0071924E-2,1.0864085E-1,-1.6852606E-3,2.0992425E-3,-4.3144743E-3,-2.1814976E-2,-9.540351E-3,4.390645E-2,8.786065E-3,2.3032464E-3,-3.2623623E-2,6.028712E-3,-3.4005716E-3,7.2302745E-4,-2.3060592E-3,3.4796104E-3,1.501016E-4,-3.124559E-3,2.0810745E-3,-1.2411478E-2,2.6133522E-2,5.140567E-3,-5.550687E-2,3.2541754E-3,2.0413592E-2,1.9434143E-2,-3.8042545E-2,-1.26432525E-2,-1.00695126E-1,1.0249201E-2,4.5358703E-2,-2.6624145E-3,1.318855E-3,-5.898998E-4,-3.6805742E-3,1.4004109E-3,-2.4693736E-3,-6.5518958E-3,-1.1020518E-3,1.584133E-3,-2.0516191E-4,4.255019E-3,9.5075544E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,-1,21,23,-1,-1,-1,-1,-1,-1,-1,25,27,29,31,-1,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.646064E-2,1.1986304E-1,1.2331514E-1,8.205077E-2,1.3637146E-1,8.631268E-2,4.410939E-2,5.6802724E-2,0E0,1.4139606E-2,0E0,1.4588859E-2,0E0,0E0,5.230405E-2,6.5944724E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.063168E-2,2.0908002E-2,4.2094477E-2,5.102405E-2,0E0,1.90586E-2,4.100515E-2,1.7424677E-2,2.9309867E-2,4.0666148E-2,2.0780422E-2,2.4179827E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,14,14,15,15,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,-1,22,24,-1,-1,-1,-1,-1,-1,-1,26,28,30,32,-1,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,1.959242E6,1.3770975E2,4.966421E6,5.80663E2,2.7608912E1,1.3626402E1,8.836406E0,-4.3144743E-3,5.127848E-2,-9.540351E-3,2.6786362E8,8.786065E-3,2.3032464E-3,2.2385964E0,1.0926206E2,-3.4005716E-3,7.2302745E-4,-2.3060592E-3,3.4796104E-3,1.501016E-4,-3.124559E-3,2.0810745E-3,7.089E3,6.8617E4,3.2004956E5,1.8558352E0,3.2541754E-3,1E1,2.4397528E8,1.1767652E6,8E0,6.700036E8,1.5678E4,1.2E1,-2.6624145E-3,1.318855E-3,-5.898998E-4,-3.6805742E-3,1.4004109E-3,-2.4693736E-3,-6.5518958E-3,-1.1020518E-3,1.584133E-3,-2.0516191E-4,4.255019E-3,9.5075544E-4],"split_indices":[51,1,57,27,51,56,56,55,0,38,0,7,0,0,53,51,0,0,0,0,0,0,0,10,12,46,52,0,3,30,47,3,5,9,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.49E2,2.08E2,4.1E1,1.92E2,1.6E1,1.6E1,2.5E1,1.84E2,8E0,1.1E1,5E0,1E1,6E0,9E0,1.6E1,1.76E2,8E0,5E0,6E0,5E0,5E0,1.1E1,5E0,9.2E1,8.4E1,6.6E1,2.6E1,8E0,7.6E1,5E1,1.6E1,1.4E1,1.2E1,5.5E1,2.1E1,5E0,4.5E1,1.1E1,5E0,7E0,7E0,7E0,5E0,2E1,3.5E1,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-5.9086527E-3,-6.42546E-2,6.961309E-3,-2.5547134E-2,-1.24572385E-2,7.1318215E-4,6.597045E-2,9.0013845E-3,-7.281874E-2,5.7801727E-2,-5.2307886E-3,8.570201E-3,-7.860352E-4,3.0040938E-2,-2.0916623E-3,-6.7833005E-3,-1.5541984E-4,8.190528E-4,4.302767E-3,-1.8469986E-2,1.8769545E-2,-1.3175976E-3,1.8936935E-3,-1.054413E-3,2.3622732E-3,-1.0704688E-2,-4.786552E-3,-2.0521814E-2,3.6608096E-2,-6.944427E-2,-3.7170276E-3,7.5017905E-4,-5.2436497E-2,2.1659838E-2,4.977766E-3,-4.769943E-3,-9.7096036E-4,7.634848E-3,-5.2071325E-2,-3.7401696E-3,-5.947149E-4,3.5241038E-2,-1.8749262E-3,8.963225E-4,-1.388106E-3,-3.5570858E-3,-1.0250517E-3,-7.4563234E-4,2.1792606E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,-1,29,-1,31,33,35,37,-1,39,41,-1,-1,-1,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9117844E-1,3.7042862E-1,7.7010535E-2,6.598203E-2,0E0,6.477555E-2,1.6582376E-1,2.9361427E-2,9.101208E-2,2.510763E-2,5.5316582E-2,0E0,1.8257583E-2,2.271432E-2,0E0,0E0,0E0,0E0,0E0,7.512025E-2,4.420711E-2,0E0,0E0,0E0,0E0,4.2428903E-2,0E0,2.4590295E-2,4.503451E-2,1.6873065E-2,5.1995277E-2,0E0,1.2077207E-2,3.3243477E-2,0E0,0E0,0E0,3.6828436E-2,1.2366902E-2,0E0,0E0,2.1806166E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,19,19,20,20,25,25,27,27,28,28,29,29,30,30,32,32,33,33,37,37,38,38,41,41],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,-1,30,-1,32,34,36,38,-1,40,42,-1,-1,-1,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.7722336E4,8.0522484E11,1.406168E3,2.1600779E8,-1.24572385E-2,1.8357558E5,6.234292E-1,1.222893E5,1.041E3,1E0,2.148789E0,8.570201E-3,4.0289428E6,6.0429E-1,-2.0916623E-3,-6.7833005E-3,-1.5541984E-4,8.190528E-4,4.302767E-3,6.6E1,3.7384987E-1,-1.3175976E-3,1.8936935E-3,-1.054413E-3,2.3622732E-3,7.23E2,-4.786552E-3,7.686371E5,3.159399E6,3.6408416E5,2.1159E4,7.5017905E-4,3.2785368E7,1.0391348E7,4.977766E-3,-4.769943E-3,-9.7096036E-4,4.357202E5,1.2114334E2,-3.7401696E-3,-5.947149E-4,1.3316584E-1,-1.8749262E-3,8.963225E-4,-1.388106E-3,-3.5570858E-3,-1.0250517E-3,-7.4563234E-4,2.1792606E-3],"split_indices":[31,30,51,7,0,31,26,32,2,80,53,0,31,26,0,0,0,0,0,8,26,0,0,0,0,2,0,31,31,46,9,0,12,50,0,0,0,32,51,0,0,56,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.53E2,4.5E1,2.08E2,3.9E1,6E0,1.89E2,1.9E1,2.3E1,1.6E1,1.7E1,1.72E2,6E0,1.3E1,1.7E1,6E0,7E0,9E0,9E0,8E0,1.11E2,6.1E1,8E0,5E0,5E0,1.2E1,1.03E2,8E0,1.9E1,4.2E1,1E1,9.3E1,9E0,1E1,3.6E1,6E0,5E0,5E0,7.6E1,1.7E1,5E0,5E0,3E1,6E0,5.8E1,1.8E1,8E0,9E0,6E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[7.609607E-4,1.0074492E-2,-7.5845145E-2,-7.674439E-2,1.4964312E-2,-1.639886E-1,4.7999904E-2,-5.8352067E-3,-9.3081966E-4,5.4724417E-3,1.0718543E-2,-3.541639E-2,-1.6791152E-2,4.8275227E-3,-4.2743172E-4,1.9051038E-2,-3.150426E-2,-4.8561683E-3,1.9283231E-3,1.2182701E-2,1.178466E-1,-3.6840085E-3,-7.2471485E-3,3.3351906E-2,-1.411668E-3,6.6302404E-5,9.675984E-3,2.696209E-2,-3.8444977E-2,5.8072094E-2,5.9609977E-3,-2.2025298E-2,3.6911123E-2,-8.4974614E-5,3.397874E-3,-2.6680145E-3,-4.2980973E-5,3.7570517E-3,1.6907612E-3,1.4335562E-3,-1.6948729E-3,-2.1282036E-3,4.0519E-5,2.5032316E-3,2.2736905E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,17,-1,-1,-1,19,21,-1,-1,23,25,27,-1,29,31,-1,-1,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7770548E-1,9.539382E-2,3.0260694E-1,3.2781594E-2,9.401533E-2,4.1549078E-1,4.1750446E-2,0E0,0E0,0E0,7.237173E-2,6.676292E-2,0E0,0E0,0E0,1.1459828E-1,1.21629775E-1,0E0,0E0,4.6379983E-2,1.2351884E-1,3.1929914E-2,0E0,4.231167E-2,7.9020195E-2,0E0,0E0,2.2476923E-2,1.0820728E-2,1.4019288E-2,3.6065727E-2,3.754896E-2,2.0469338E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,11,11,15,15,16,16,19,19,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,18,-1,-1,-1,20,22,-1,-1,24,26,28,-1,30,32,-1,-1,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6146525E6,9.541066E5,9.265399E-4,1.6654063E0,9.3699955E8,1.5316E4,3.2235E4,-5.8352067E-3,-9.3081966E-4,5.4724417E-3,3.4231E4,9.27E2,-1.6791152E-2,4.8275227E-3,-4.2743172E-4,6.811E3,9.4151026E-1,-4.8561683E-3,1.9283231E-3,1.6003E4,1.3841E4,6.1037578E-2,-7.2471485E-3,4.0500336E0,2.3357933E0,6.6302404E-5,9.675984E-3,8.081268E-1,4.7563137E1,4.6741E4,2.0210526E0,1.23844246E2,1.8413795E9,-8.4974614E-5,3.397874E-3,-2.6680145E-3,-4.2980973E-5,3.7570517E-3,1.6907612E-3,1.4335562E-3,-1.6948729E-3,-2.1282036E-3,4.0519E-5,2.5032316E-3,2.2736905E-4],"split_indices":[27,44,37,38,30,9,10,0,0,0,10,2,0,0,0,2,26,0,0,9,9,37,0,56,53,0,0,26,56,28,55,51,5,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.47E2,2.21E2,2.6E1,1.1E1,2.1E2,1.5E1,1.1E1,5E0,6E0,7E0,2.03E2,1E1,5E0,5E0,6E0,1.7E2,3.3E1,5E0,5E0,1.6E2,1E1,2.8E1,5E0,6.2E1,9.8E1,5E0,5E0,1.5E1,1.3E1,3.2E1,3E1,6.4E1,3.4E1,1E1,5E0,8E0,5E0,1.3E1,1.9E1,1.9E1,1.1E1,3E1,3.4E1,2.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-2.7934907E-3,1.8350483E-3,-5.7935594E-3,-4.242357E-3,1.1061969E-1,-2.0301484E-2,1.5872344E-2,1.0891946E-2,-6.33031E-5,-1.1886412E-2,-8.668178E-3,5.6054417E-2,-8.112099E-3,9.32226E-3,-3.0385297E-2,5.9335907E-3,7.8110576E-2,-3.1182358E-2,2.4311194E-2,-2.4825667E-3,2.0500202E-2,-1.19385265E-1,-1.26767885E-2,1.669908E-3,-8.1764715E-4,6.131867E-3,4.65854E-2,1.6038964E-3,-4.2044915E-2,-1.1397366E-3,4.912999E-2,5.832097E-2,3.1215209E-3,-2.7616073E-3,-7.1078795E-3,-4.7095884E-3,4.4369935E-3,6.4754635E-2,-1.6114661E-4,-3.61759E-3,-1.7572034E-2,2.9080855E-3,2.2985124E-4,3.752032E-3,1.3414616E-4,-1.4222248E-3,7.1442535E-4,6.535714E-4,-3.1589174E-3,1.1136429E-3,3.7996932E-3,-2.9350459E-3,4.0322464E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,21,23,25,27,29,-1,31,33,35,-1,-1,-1,37,-1,39,-1,41,43,45,-1,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4741983E-1,1.6244572E-1,0E0,7.5586945E-2,1.9101049E-1,1.8579993E-1,1.0068416E-1,0E0,0E0,4.9325354E-2,0E0,4.2649977E-2,5.0126992E-2,4.35855E-2,1.0408237E-1,1.0401377E-2,4.2926446E-2,2.9774103E-2,3.5700984E-2,0E0,3.333898E-2,1.4805734E-2,8.998032E-2,0E0,0E0,0E0,1.7547116E-2,0E0,3.0360177E-2,0E0,1.25232935E-2,2.179068E-2,1.6374674E-2,0E0,0E0,0E0,3.7724555E-2,8.860353E-3,0E0,0E0,2.829975E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,26,26,28,28,30,30,31,31,32,32,36,36,37,37,40,40],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,22,24,26,28,30,-1,32,34,36,-1,-1,-1,38,-1,40,-1,42,44,46,-1,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4089414E8,1.3402396E3,-5.7935594E-3,1.6634E4,1E0,4.966421E6,1.8344E4,1.0891946E-2,-6.33031E-5,2.3378747E0,-8.668178E-3,8.92E2,2.6600266E0,3.720508E-2,1.5839127E-1,1.9356645E8,2.1203648E8,2.161E4,3.7384987E-1,-2.4825667E-3,8.0224875E4,3.2647366E8,1.2295239E6,1.669908E-3,-8.1764715E-4,6.131867E-3,3.007712E6,1.6038964E-3,2.8290488E5,-1.1397366E-3,4.530014E8,7.61739E5,3.568E4,-2.7616073E-3,-7.1078795E-3,-4.7095884E-3,2.1908425E6,4.1689847E5,-1.6114661E-4,-3.61759E-3,1.0164216E2,2.9080855E-3,2.2985124E-4,3.752032E-3,1.3414616E-4,-1.4222248E-3,7.1442535E-4,6.535714E-4,-3.1589174E-3,1.1136429E-3,3.7996932E-3,-2.9350459E-3,4.0322464E-4],"split_indices":[44,51,0,9,63,27,9,0,0,57,0,2,52,57,41,7,7,28,26,0,32,7,47,0,0,0,31,0,32,0,7,28,1,0,0,0,31,27,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.52E2,2.44E2,8E0,2.32E2,1.2E1,1.29E2,1.03E2,5E0,7E0,1.24E2,5E0,3.8E1,6.5E1,5.8E1,6.6E1,1.2E1,2.6E1,3.8E1,2.7E1,8E0,5E1,1E1,5.6E1,5E0,7E0,8E0,1.8E1,5E0,3.3E1,9E0,1.8E1,1.5E1,3.5E1,5E0,5E0,8E0,4.8E1,1.3E1,5E0,1.2E1,2.1E1,1.3E1,5E0,1E1,5E0,9E0,2.6E1,4.3E1,5E0,5E0,8E0,7E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-6.960007E-4,1.5277823E-2,-5.0286718E-2,1.08618215E-2,4.3515423E-3,-2.9574048E-2,-9.98413E-3,4.4629127E-2,6.2871017E-3,-9.659663E-3,-8.328692E-4,4.645649E-3,8.2498055E-4,1.5758967E-2,-1.688964E-2,-2.0523932E-2,5.6668803E-2,-1.2944505E-3,2.0135472E-2,-4.346292E-2,7.0657153E-3,-4.4818223E-2,4.9105776E-3,7.3809107E-4,3.7479107E-3,3.65037E-3,1.3829963E-2,-6.479507E-4,-3.6370384E-3,-2.2713965E-2,2.9959342E-2,4.6882257E-4,-4.0596626E-3,2.0410316E-3,-8.5845793E-4,2.1899153E-2,-1.5507158E-2,3.6126105E-4,-2.2390068E-3,-2.0649155E-4,2.2391165E-3,-1.8599863E-3,1.2789418E-3,1.6682656E-3,-2.1023995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,15,-1,-1,17,19,21,23,-1,25,27,29,31,33,-1,-1,-1,35,-1,-1,37,39,-1,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8534E-1,6.2440027E-2,1.9549017E-1,2.5825845E-2,0E0,2.757823E-1,0E0,2.8870925E-2,3.319521E-2,0E0,5.3247444E-2,0E0,0E0,2.1400668E-2,2.831628E-2,2.1847054E-2,1.1429694E-2,0E0,3.646217E-2,2.1565601E-2,1.7042851E-2,4.5870863E-2,1.8256638E-2,0E0,0E0,0E0,2.1645006E-2,0E0,0E0,9.502981E-3,9.751677E-3,0E0,0E0,0E0,0E0,3.0149318E-2,3.4400396E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,26,26,29,29,30,30,35,35,36,36],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,16,-1,-1,18,20,22,24,-1,26,28,30,32,34,-1,-1,-1,36,-1,-1,38,40,-1,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.3951338E11,6.071E3,3.7861453E5,1.3176E4,4.3515423E-3,6.31064E5,-9.98413E-3,4.0500336E0,3.452399E8,-9.659663E-3,1.805781E6,4.645649E-3,8.2498055E-4,1.6819989E8,1.6382E4,2.1564245E0,3.3E1,-1.2944505E-3,4.1932972E9,8.797907E-2,2.2726698E2,1.9844419E0,4.8100915E8,7.3809107E-4,3.7479107E-3,3.65037E-3,1.9042E4,-6.479507E-4,-3.6370384E-3,2.2736814E5,8E0,4.6882257E-4,-4.0596626E-3,2.0410316E-3,-8.5845793E-4,1.4176E4,1.9439217E0,3.6126105E-4,-2.2390068E-3,-2.0649155E-4,2.2391165E-3,-1.8599863E-3,1.2789418E-3,1.6682656E-3,-2.1023995E-3],"split_indices":[30,2,32,9,0,1,0,56,7,0,27,0,0,7,9,53,8,0,30,37,4,53,12,0,0,0,9,0,0,46,3,0,0,0,0,28,55,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,1.76E2,5.6E1,1.68E2,8E0,5.1E1,5E0,1.9E1,1.49E2,6E0,4.5E1,5E0,1.4E1,1.06E2,4.3E1,3.4E1,1.1E1,9E0,9.7E1,2E1,2.3E1,1.7E1,1.7E1,5E0,6E0,8E0,8.9E1,1.2E1,8E0,1E1,1.3E1,8E0,9E0,6E0,1.1E1,7E1,1.9E1,5E0,5E0,5E0,8E0,6E0,6.4E1,7E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-2.0100076E-3,-7.144386E-2,4.0364577E-3,-2.6547687E-2,-7.7458196E-3,5.106713E-3,6.055078E-4,-2.4302853E-3,1.0577347E-3,8.073159E-3,-3.7354596E-2,-4.539791E-2,1.5068909E-2,-1.0990203E-2,-5.931971E-3,6.4254436E-4,-7.418336E-2,7.877587E-3,7.805837E-2,-4.2932704E-2,2.3814477E-2,-5.1672678E-3,2.4936636E-4,1.3705369E-2,-4.2150907E-2,1.5049544E-3,9.939704E-2,-3.1758535E-3,1.1194008E-4,-7.2187476E-4,2.0438994E-3,1.8077351E-2,-2.6381712E-3,-3.1232834E-3,-2.8766904E-4,6.3338364E-3,1.8832397E-3,1.3915523E-3,-4.5594847E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,-1,-1,9,-1,-1,11,13,15,17,19,-1,-1,21,23,25,27,29,-1,-1,31,33,-1,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.048259E-1,8.5531175E-2,8.6114034E-2,2.177728E-2,0E0,0E0,6.379948E-2,0E0,0E0,7.1009316E-2,9.044189E-2,3.88306E-2,7.503232E-2,3.4374874E-2,0E0,0E0,4.8951745E-2,4.460252E-2,1.2891859E-2,2.0155586E-2,1.3576134E-2,0E0,0E0,4.399377E-2,1.5076203E-2,0E0,2.0168446E-2,0E0,0E0,0E0,0E0,4.67946E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,19,19,20,20,23,23,24,24,26,26,31,31],"right_children":[2,4,6,8,-1,-1,10,-1,-1,12,14,16,18,20,-1,-1,22,24,26,28,30,-1,-1,32,34,-1,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9894668E4,2.268511E6,6.747104E4,8.53386E3,-7.7458196E-3,5.106713E-3,2.68225E5,-2.4302853E-3,1.0577347E-3,7.23E2,8.756827E-1,4.1174043E3,2.7349632E6,2.2827177E0,-5.931971E-3,6.4254436E-4,3.0050538E5,6.6E1,9.770216E4,1.0986164E6,3.7384987E-1,-5.1672678E-3,2.4936636E-4,9.49181E-1,2.1564245E0,1.5049544E-3,1.9898E4,-3.1758535E-3,1.1194008E-4,-7.2187476E-4,2.0438994E-3,3.4577703E0,-2.6381712E-3,-3.1232834E-3,-2.8766904E-4,6.3338364E-3,1.8832397E-3,1.3915523E-3,-4.5594847E-4],"split_indices":[32,27,27,32,0,0,9,0,0,2,26,46,27,52,0,0,32,8,32,46,26,0,0,26,53,0,10,0,0,0,0,55,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,1.9E1,2.29E2,1.4E1,5E0,6E0,2.23E2,9E0,5E0,1.87E2,3.6E1,2.1E1,1.66E2,2.9E1,7E0,7E0,1.4E1,1.5E2,1.6E1,1.5E1,1.4E1,9E0,5E0,1.35E2,1.5E1,6E0,1E1,9E0,6E0,5E0,9E0,1.28E2,7E0,8E0,7E0,5E0,5E0,8.8E1,4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[8.6430536E-4,8.911805E-3,-4.3529052E-2,-6.842594E-2,1.3674194E-2,-5.1465835E-3,-1.0866376E-2,-5.849845E-4,-5.488484E-3,1.0758232E-2,4.0411083E-3,-3.3786427E-2,2.371899E-3,3.8408823E-2,3.212409E-3,1.0392083E-3,-2.3383778E-3,6.2148184E-2,5.941368E-3,-6.523888E-2,1.568199E-2,1.2996233E-4,8.789694E-2,-1.7218997E-3,1.4063065E-3,-2.4932703E-2,-7.903972E-3,4.3341513E-3,5.34891E-2,5.681801E-3,2.6658913E-3,-4.1990777E-5,-3.281839E-3,4.7528163E-2,-4.67356E-3,1.2966915E-2,8.133024E-2,1.5230173E-4,2.9599846E-3,-1.3697027E-3,3.5180646E-4,2.5167977E-3,-1.4309115E-3,4.5361826E-3,1.7781781E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,-1,17,19,-1,-1,21,23,25,27,-1,29,-1,-1,31,-1,33,35,-1,-1,-1,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.6129256E-2,7.607141E-2,2.79171E-1,3.3882458E-2,4.1775897E-2,5.4701395E-2,0E0,0E0,0E0,3.8893506E-2,0E0,2.3789825E-2,0E0,3.0461024E-2,1.273498E-1,0E0,0E0,3.4845293E-2,2.1243617E-2,9.764641E-2,5.3825825E-2,0E0,1.0095671E-2,0E0,0E0,2.0603739E-2,0E0,3.835189E-2,3.1662725E-2,0E0,0E0,0E0,0E0,1.3374403E-2,2.6529828E-2,2.6669724E-2,9.755932E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,13,13,14,14,17,17,18,18,19,19,20,20,22,22,25,25,27,27,28,28,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,-1,18,20,-1,-1,22,24,26,28,-1,30,-1,-1,32,-1,34,36,-1,-1,-1,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8337E4,4.75E2,2.363013E6,1.7039711E0,1.7951E4,3.3640351E0,-1.0866376E-2,-5.849845E-4,-5.488484E-3,1.3298E4,4.0411083E-3,4.7943E4,2.371899E-3,3.91014E5,1.4693E4,1.0392083E-3,-2.3383778E-3,6.271094E-1,5.95425E5,3.623473E7,1.5811304E6,1.2996233E-4,4.4592E4,-1.7218997E-3,1.4063065E-3,1.2826048E6,-7.903972E-3,2.73E2,1E0,5.681801E-3,2.6658913E-3,-4.1990777E-5,-3.281839E-3,2.26E2,9.9844826E1,1E0,2.447E3,1.5230173E-4,2.9599846E-3,-1.3697027E-3,3.5180646E-4,2.5167977E-3,-1.4309115E-3,4.5361826E-3,1.7781781E-3],"split_indices":[9,2,27,53,2,53,0,0,0,9,0,28,0,28,9,0,0,55,28,44,27,0,28,0,0,31,0,0,8,0,0,0,0,11,51,80,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.39E2,2.03E2,3.6E1,1.1E1,1.92E2,3.1E1,5E0,6E0,5E0,1.86E2,6E0,2.1E1,1E1,3.9E1,1.47E2,5E0,1.6E1,2.2E1,1.7E1,2.2E1,1.25E2,7E0,1.5E1,6E0,1.1E1,1.7E1,5E0,9.7E1,2.8E1,5E0,1E1,1.2E1,5E0,1.6E1,8.1E1,1.2E1,1.6E1,5E0,1.1E1,2.6E1,5.5E1,6E0,6E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.1473002E-3,5.2632345E-3,-6.2659527E-3,-8.343244E-4,7.558954E-2,1.8295921E-2,-1.6069526E-2,8.256688E-3,1.188128E-2,1.140413E-2,5.5043898E-3,-6.604897E-3,-9.564678E-2,-6.202328E-4,1.9877779E-3,-3.263438E-3,1.8013688E-2,-7.446332E-2,8.5101975E-4,-9.6642E-3,2.3963276E-4,2.5881758E-2,-2.036307E-3,-5.018796E-3,-1.1370293E-3,1.019354E-2,-4.941821E-2,4.1209042E-2,-8.260443E-3,-2.0323632E-2,2.6076375E-2,-4.566213E-3,-1.6646296E-2,9.821637E-4,2.7638702E-3,-1.4767974E-3,1.7548329E-3,1.4817811E-4,-2.5257138E-3,1.5522705E-3,-7.4858917E-4,9.2749274E-4,-2.3036292E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,19,-1,-1,-1,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4143778E-1,1.0212193E-1,0E0,6.441371E-2,1.2796894E-1,6.869605E-2,9.2008084E-2,0E0,1.133796E-2,5.280019E-2,0E0,5.6223463E-2,1.5689734E-1,0E0,0E0,0E0,4.4411648E-2,1.714911E-2,4.7937516E-2,0E0,0E0,4.1158356E-2,0E0,0E0,0E0,4.2214103E-2,2.5499545E-2,1.9735768E-2,3.0158713E-2,2.546191E-2,2.087884E-2,0E0,1.5149622E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,16,16,17,17,18,18,21,21,25,25,26,26,27,27,28,28,29,29,30,30,32,32],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,20,-1,-1,-1,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,1.406168E3,-6.2659527E-3,2.9722316E0,1E0,3.383E3,7.960309E7,8.256688E-3,1.9E1,1.736E3,5.5043898E-3,7.66E2,6.9795944E5,-6.202328E-4,1.9877779E-3,-3.263438E-3,4.530014E8,9.681E3,5.058997E0,-9.6642E-3,2.3963276E-4,2.5605617E0,-2.036307E-3,-5.018796E-3,-1.1370293E-3,1.0469056E2,5.90314E5,9.06E2,6.567405E2,5.4335805E6,2.447E3,-4.566213E-3,2.7E1,9.821637E-4,2.7638702E-3,-1.4767974E-3,1.7548329E-3,1.4817811E-4,-2.5257138E-3,1.5522705E-3,-7.4858917E-4,9.2749274E-4,-2.3036292E-3],"split_indices":[27,51,0,56,63,2,44,0,3,12,0,2,31,0,0,0,7,10,55,0,0,55,0,0,0,51,28,2,4,44,0,0,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.37E2,6E0,2.19E2,1.8E1,9.7E1,1.22E2,6E0,1.2E1,9.2E1,5E0,1.1E2,1.2E1,7E0,5E0,6E0,8.6E1,1E1,1E2,5E0,7E0,7.7E1,9E0,5E0,5E0,8.5E1,1.5E1,5.3E1,2.4E1,2.9E1,5.6E1,5E0,1E1,2.8E1,2.5E1,1.6E1,8E0,1.8E1,1.1E1,4.7E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.0006378E-3,4.6754335E-3,-9.711255E-2,-2.8986267E-3,7.090808E-2,3.4929713E-4,-9.832729E-3,-1.48711065E-2,2.0364646E-2,1.0152029E-2,1.2302628E-2,-5.1886523E-3,-7.778662E-2,1.2551728E-2,3.4017332E-3,1.9678718E-3,-2.363941E-3,-2.5347821E-2,1.4707875E-2,-8.544682E-3,-1.5451151E-2,2.4422137E-2,-2.3896283E-2,2.004502E-2,-4.244386E-2,2.4413189E-2,-2.154908E-3,1.0595456E-3,-2.1025727E-3,4.9106427E-2,9.109894E-3,-3.4062604E-3,4.2744871E-4,-1.2615604E-3,3.2270076E-3,-1.6615963E-2,-8.176146E-2,3.436508E-2,-1.44881E-3,1.0082709E-3,3.7932133E-3,2.0543436E-2,-1.7199372E-3,-1.904134E-3,9.9869E-4,-5.435063E-3,-2.0943556E-3,1.9317237E-3,-1.1287791E-3,1.7381752E-3,-3.840208E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,-1,-1,-1,23,25,-1,27,29,31,33,35,37,-1,-1,-1,39,41,-1,-1,-1,-1,43,45,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.452281E-1,1.07558645E-1,1.919218E-1,5.405702E-2,1.9608304E-1,0E0,0E0,7.7571E-2,2.7452514E-2,0E0,3.6771342E-2,4.529573E-2,1.1544782E-1,2.6110113E-2,0E0,0E0,0E0,4.4308905E-2,3.574003E-2,0E0,1.5667189E-2,1.6561376E-2,2.6570614E-2,4.158752E-2,4.027658E-2,2.9239666E-2,0E0,0E0,0E0,1.4039736E-2,1.6451214E-2,0E0,0E0,0E0,0E0,2.6747925E-2,1.6308956E-2,2.2716455E-2,0E0,0E0,0E0,1.2884616E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,29,29,30,30,35,35,36,36,37,37,41,41],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,-1,-1,-1,24,26,-1,28,30,32,34,36,38,-1,-1,-1,40,42,-1,-1,-1,-1,44,46,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.5E1,5.818E3,7.4211675E5,2.1799042E0,3.0451374E9,3.4929713E-4,-9.832729E-3,4.9873673E11,2.3802902E6,1.0152029E-2,3.945346E1,2.6439633E2,4.53328E5,3.6075268E0,3.4017332E-3,1.9678718E-3,-2.363941E-3,9.462E3,2.8E1,-8.544682E-3,4.699E3,6.5229E4,1E0,4.75E2,1.7184795E0,1.7476E4,-2.154908E-3,1.0595456E-3,-2.1025727E-3,9.44E2,9.115134E5,-3.4062604E-3,4.2744871E-4,-1.2615604E-3,3.2270076E-3,4.1100688E8,7.625902E4,3.7017515E6,-1.44881E-3,1.0082709E-3,3.7932133E-3,9.05E3,-1.7199372E-3,-1.904134E-3,9.9869E-4,-5.435063E-3,-2.0943556E-3,1.9317237E-3,-1.1287791E-3,1.7381752E-3,-3.840208E-4],"split_indices":[3,2,27,53,12,0,0,30,27,0,56,54,28,57,0,0,0,28,8,0,10,28,108,2,53,9,0,0,0,2,31,0,0,0,0,5,31,46,0,0,0,10,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.27E2,2.13E2,1.4E1,1.92E2,2.1E1,8E0,6E0,1.27E2,6.5E1,5E0,1.6E1,1.11E2,1.6E1,5.8E1,7E0,1.1E1,5E0,5.5E1,5.6E1,5E0,1.1E1,4.4E1,1.4E1,1.5E1,4E1,4.9E1,7E0,5E0,6E0,1.6E1,2.8E1,5E0,9E0,8E0,7E0,2.5E1,1.5E1,4.2E1,7E0,1E1,6E0,2.3E1,5E0,1.5E1,1E1,6E0,9E0,3.7E1,5E0,1.4E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-5.86956E-3,6.439433E-3,-9.577432E-3,-2.6484996E-2,2.0810269E-2,-1.4024516E-2,-1.727653E-1,1.4518344E-2,4.61217E-3,-4.2329427E-2,6.3602375E-3,-1.7182234E-3,-1.3576177E-2,3.6544107E-2,2.1925808E-3,-3.0179191E-2,-5.862575E-3,-6.461434E-4,3.2918113E-3,5.7674426E-4,3.3481582E-3,-6.5781362E-3,2.0653422E-3,-5.3097498E-2,2.282521E-2,2.925385E-3,-8.125098E-3,1.9559354E-2,-2.029886E-2,-1.5833039E-2,-8.9528635E-2,-2.1569541E-4,2.456757E-3,2.9074656E-2,-2.466121E-2,1.8316141E-3,-2.966132E-4,-1.5979656E-3,1.0028542E-4,-2.4961748E-3,1.3178958E-3,-6.5455874E-3,-2.37576E-3,1.8986849E-3,-5.724486E-4,-2.477659E-3,9.2468355E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,-1,3,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,-1,25,-1,-1,-1,27,-1,29,31,-1,33,35,37,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.358772E-1,0E0,1.2334505E-1,2.7827635E-1,4.2600296E-2,8.277861E-2,1.9586733E-1,2.1910103E-2,0E0,5.999873E-2,3.9268028E-2,0E0,0E0,2.502523E-2,2.0573301E-2,6.6809595E-2,0E0,3.8259678E-2,0E0,0E0,0E0,1.653517E-2,0E0,5.0170787E-2,1.511902E-2,0E0,4.377677E-2,9.074344E-3,1.0307885E-2,3.723663E-2,3.129372E-2,0E0,0E0,1.2510058E-2,3.975236E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,17,17,21,21,23,23,24,24,26,26,27,27,28,28,29,29,30,30,33,33,34,34],"right_children":[2,-1,4,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,-1,26,-1,-1,-1,28,-1,30,32,-1,34,36,38,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.550401E4,6.439433E-3,2.1577182E0,1.0232127E12,3.26989E6,1.9665255E-1,9E0,1.5544465E0,4.61217E-3,2.491E3,2.6732678E3,-1.7182234E-3,-1.3576177E-2,1.6509379E6,1.1510987E1,1.9851093E2,-5.862575E-3,4.45E2,3.2918113E-3,5.7674426E-4,3.3481582E-3,1.8357558E5,2.0653422E-3,9.62689E6,5.501913E9,2.925385E-3,1.3155E4,5.69E2,2.83596E0,1.5060241E0,1.4693E4,-2.1569541E-4,2.456757E-3,1.655E3,9.130081E0,1.8316141E-3,-2.966132E-4,-1.5979656E-3,1.0028542E-4,-2.4961748E-3,1.3178958E-3,-6.5455874E-3,-2.37576E-3,1.8986849E-3,-5.724486E-4,-2.477659E-3,9.2468355E-5],"split_indices":[27,0,53,30,27,41,3,55,0,2,4,0,0,49,55,51,0,10,0,0,0,31,0,44,5,0,9,0,53,53,9,0,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,5E0,2.38E2,1.53E2,8.5E1,1.42E2,1.1E1,8E1,5E0,5.9E1,8.3E1,6E0,5E0,2.8E1,5.2E1,5.3E1,6E0,7.6E1,7E0,1.8E1,1E1,4.4E1,8E0,3.7E1,1.6E1,7E0,6.9E1,1.5E1,2.9E1,1.9E1,1.8E1,9E0,7E0,2.1E1,4.8E1,8E0,7E0,1.7E1,1.2E1,1E1,9E0,6E0,1.2E1,1.6E1,5E0,2.2E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.2768342E-3,8.627293E-3,-4.646357E-2,2.8187397E-3,1.1345305E-1,-1.5352927E-2,-1.1332103E-2,-5.8597955E-3,2.6999956E-2,9.123146E-3,2.5593012E-4,-4.79367E-3,2.3271216E-4,-1.0144815E-2,2.7707932E-3,1.0305919E-2,3.2813053E-3,-9.813232E-3,2.3492177E-3,-5.394426E-3,-5.2036257E-3,2.3994409E-2,-2.5997204E-3,3.5659545E-3,-2.9830441E-3,2.7727156E-3,-9.888956E-3,2.5701167E-3,1.9342542E-3,1.7745323E-3,-3.0792708E-4,-4.9006264E-3,-3.0555606E-3,-5.9202366E-4,1.5124094E-3,-6.176643E-4,1.3584336E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,17,19,-1,21,-1,23,-1,-1,25,27,-1,29,-1,-1,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[1.1452233E-1,1.2754628E-1,2.892939E-1,4.212972E-2,1.0420984E-1,5.8146007E-2,0E0,4.3308273E-2,3.9745454E-2,0E0,0E0,0E0,1.926853E-2,7.511618E-2,0E0,3.8593724E-2,0E0,2.3603579E-2,0E0,0E0,4.2792954E-2,1.3657758E-2,0E0,9.940777E-3,0E0,0E0,3.6331758E-2,8.705913E-3,0E0,0E0,0E0,3.6772132E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,12,12,13,13,15,15,17,17,20,20,21,21,23,23,26,26,27,27,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,18,20,-1,22,-1,24,-1,-1,26,28,-1,30,-1,-1,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,1.14157E3,2.0459295E6,1E0,7.946966E-1,1.5917099E0,-1.1332103E-2,1.3943323E12,1.8520895E2,9.123146E-3,2.5593012E-4,-4.79367E-3,5.111236E6,1.9898948E-6,2.7707932E-3,1.491E3,3.2813053E-3,3.7E1,2.3492177E-3,-5.394426E-3,5.322348E7,7.86E2,-2.5997204E-3,3.708E5,-2.9830441E-3,2.7727156E-3,6.4554214E8,1.1652305E6,1.9342542E-3,1.7745323E-3,-3.0792708E-4,2.5245471E2,-3.0555606E-3,-5.9202366E-4,1.5124094E-3,-6.176643E-4,1.3584336E-3],"split_indices":[44,51,27,15,26,55,0,30,51,0,0,0,28,37,0,2,0,3,0,0,5,2,0,28,0,0,7,31,0,0,0,51,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.54E2,2.09E2,4.5E1,1.99E2,1E1,4E1,5E0,1.47E2,5.2E1,5E0,5E0,5E0,3.5E1,1.39E2,8E0,3.9E1,1.3E1,3E1,5E0,5E0,1.34E2,3.3E1,6E0,2.5E1,5E0,8E0,1.26E2,1.6E1,1.7E1,5E0,2E1,1.17E2,9E0,1.1E1,5E0,9.4E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.6509421E-3,-4.7521222E-2,7.6105413E-3,-1.9028641E-2,-1.0096124E-2,1.666313E-2,-2.879008E-2,-4.264859E-3,-5.1532574E-3,7.6713343E-3,1.0307241E-1,-1.4635986E-2,-4.804914E-3,-3.1747934E-3,9.477985E-3,-4.213385E-2,1.4561427E-2,5.984405E-4,1.0622029E-2,1.666302E-4,-2.4786105E-3,1.9260906E-2,-1.7759964E-3,-9.031835E-3,-3.7148113E-3,-3.5307266E-2,2.0454533E-2,3.655088E-2,-8.27913E-3,1.4404028E-3,-2.1870576E-3,-2.8208157E-3,5.16356E-4,6.1681524E-2,1.1223061E-2,2.4886678E-3,-4.4762372E-4,-1.6072195E-3,7.376309E-4,7.938928E-4,3.3528279E-3,-2.8880244E-2,2.149375E-2,1.5541476E-3,-2.358028E-3,2.76495E-3,3.33269E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,-1,21,23,25,-1,-1,-1,-1,27,-1,29,-1,31,33,35,37,-1,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1,43,45,-1,-1,-1,-1],"loss_changes":[1.2337872E-1,2.5203794E-1,6.605419E-2,6.2796384E-2,0E0,1.2352523E-1,4.2469412E-2,3.7026312E-2,0E0,5.0715223E-2,1.7533903E-1,2.6079625E-2,0E0,0E0,1.7011646E-2,2.3237474E-2,3.8632907E-2,0E0,0E0,0E0,0E0,1.4111437E-2,0E0,1.9339528E-2,0E0,1.8554138E-2,4.3455705E-2,1.6141774E-2,8.673104E-3,0E0,0E0,0E0,0E0,1.0421097E-2,4.0162046E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.1464957E-2,4.279252E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,14,14,15,15,16,16,21,21,23,23,25,25,26,26,27,27,28,28,33,33,34,34,41,41,42,42],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,-1,22,24,26,-1,-1,-1,-1,28,-1,30,-1,32,34,36,38,-1,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1,44,46,-1,-1,-1,-1],"split_conditions":[8.7722336E4,3.2647366E8,3.6235E4,3.6543632E6,-1.0096124E-2,5.474E3,9.512301E-1,1.8723404E0,-5.1532574E-3,3.183024E-2,1.4781E4,1.3595536E6,-4.804914E-3,-3.1747934E-3,2.539E3,1.5023475E-2,1.222893E5,5.984405E-4,1.0622029E-2,1.666302E-4,-2.4786105E-3,9.8504E4,-1.7759964E-3,2.3641207E0,-3.7148113E-3,9.35E2,9.0996725E5,1.5779E4,1E0,1.4404028E-3,-2.1870576E-3,-2.8208157E-3,5.16356E-4,3.1060094E4,2.6054564E5,2.4886678E-3,-4.4762372E-4,-1.6072195E-3,7.376309E-4,7.938928E-4,3.3528279E-3,3.08E2,1.3155E4,1.5541476E-3,-2.358028E-3,2.76495E-3,3.33269E-4],"split_indices":[31,7,10,27,0,2,26,52,0,56,9,31,0,0,2,56,32,0,0,0,0,12,0,52,0,0,49,9,8,0,0,0,0,50,27,0,0,0,0,0,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,5E1,1.98E2,4.4E1,6E0,1.59E2,3.9E1,3.9E1,5E0,1.45E2,1.4E1,3.4E1,5E0,6E0,3.3E1,1.7E1,1.28E2,9E0,5E0,2.4E1,1E1,2.8E1,5E0,1E1,7E0,1.3E1,1.15E2,1.7E1,1.1E1,5E0,5E0,8E0,5E0,2E1,9.5E1,1.2E1,5E0,5E0,6E0,5E0,1.5E1,1.9E1,7.6E1,5E0,1.4E1,1.9E1,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.6462183E-3,-5.361753E-3,4.7592483E-2,4.3452485E-3,-6.79191E-2,1.1664575E-1,4.885038E-4,-3.585599E-3,9.19569E-3,-1.669529E-2,-7.8552235E-3,9.194063E-3,2.373301E-3,1.7980249E-3,-2.7679114E-2,1.245547E-2,-3.1303682E-3,-2.4613866E-3,3.4423408E-3,-2.5653963E-3,9.3582633E-4,-3.2537002E-3,4.100013E-2,-1.5121644E-3,1.2495676E-3,1.43884625E-2,-2.2529654E-2,6.9634125E-2,1.9787478E-3,-1.7048601E-3,2.2452164E-2,-4.4485773E-3,-8.495616E-3,3.482823E-2,9.5279396E-2,3.969185E-2,-2.832365E-2,1.39991E-3,-1.196892E-3,-1.1823962E-3,7.7293615E-4,2.7012054E-3,1.0211408E-4,7.1277106E-3,2.7760512E-3,4.38623E-4,3.067867E-3,-4.9847964E-5,-2.3448619E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,21,-1,-1,23,-1,-1,25,27,-1,-1,29,31,33,35,-1,37,-1,39,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.807771E-2,1.2660192E-1,1.2029232E-1,7.435628E-2,1.4953133E-1,7.3045135E-2,2.6619988E-2,0E0,4.4943847E-2,1.5448289E-2,0E0,0E0,0E0,0E0,2.098881E-2,7.4569955E-2,0E0,0E0,1.3492887E-2,0E0,0E0,3.7051678E-2,6.578516E-2,0E0,0E0,2.4840094E-2,5.471579E-2,2.70302E-2,3.0836951E-2,0E0,2.187553E-2,0E0,2.089927E-2,1.292925E-2,3.049174E-2,9.529803E-3,9.609488E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,14,14,15,15,18,18,21,21,22,22,25,25,26,26,27,27,28,28,30,30,32,32,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,22,-1,-1,24,-1,-1,26,28,-1,-1,30,32,34,36,-1,38,-1,40,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,6.0095956E7,1.3770975E2,4.39E2,1.2319056E6,6.5023036E7,3.945346E1,-3.585599E-3,8.836406E0,4.7090962E5,-7.8552235E-3,9.194063E-3,2.373301E-3,1.7980249E-3,2.2161171E0,1.6947379E2,-3.1303682E-3,-2.4613866E-3,1.306E3,-2.5653963E-3,9.3582633E-4,5.37E2,3.4480013E8,-1.5121644E-3,1.2495676E-3,1.1216729E2,1.2399164E5,1.8013916E-3,7.4456794E5,-1.7048601E-3,1.8134046E6,-4.4485773E-3,2.5E1,8.606702E-2,2.2156863E0,2.6858338E5,7.561909E-1,1.39991E-3,-1.196892E-3,-1.1823962E-3,7.7293615E-4,2.7012054E-3,1.0211408E-4,7.1277106E-3,2.7760512E-3,4.38623E-4,3.067867E-3,-4.9847964E-5,-2.3448619E-3],"split_indices":[51,44,57,2,27,44,56,0,55,27,0,0,0,0,53,51,0,0,0,0,0,0,7,0,0,4,32,41,31,0,47,0,8,57,56,32,26,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.07E2,3.6E1,1.8E2,2.7E1,1.4E1,2.2E1,9E0,1.71E2,1.9E1,8E0,5E0,9E0,9E0,1.3E1,1.65E2,6E0,6E0,1.3E1,8E0,5E0,1.07E2,5.8E1,5E0,8E0,5.6E1,5.1E1,3.3E1,2.5E1,7E0,4.9E1,7E0,4.4E1,1.5E1,1.8E1,1.1E1,1.4E1,4.2E1,7E0,2.6E1,1.8E1,8E0,7E0,5E0,1.3E1,6E0,5E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.0957947E-3,-4.0036337E-3,6.506108E-2,-7.351873E-3,2.318714E-3,8.824674E-3,-2.3965079E-4,-2.9994554E-3,-8.036761E-2,1.1636418E-3,-1.5749363E-3,-5.3762242E-2,4.7130726E-4,-7.4644466E-3,1.8559325E-4,-5.116429E-3,-1.02106E-4,-3.849693E-3,4.2214792E-2,-1.497234E-3,-2.6946599E-3,-3.1244094E-4,2.779905E-3,-8.899145E-3,1.956495E-2,1.5310032E-3,-6.989672E-4,1.7669548E-3,-3.7488213E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,15,17,-1,-1,-1,-1,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[7.952377E-2,4.2318355E-2,1.5360448E-1,6.776707E-2,0E0,0E0,1.2646105E-2,3.579101E-2,8.702624E-2,0E0,0E0,3.9712872E-2,3.4613874E-2,0E0,0E0,0E0,0E0,2.2795664E-2,1.7618597E-2,2.6365383E-2,0E0,0E0,0E0,3.5887934E-2,2.439323E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,11,11,12,12,17,17,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,16,18,-1,-1,-1,-1,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[1.3908939E3,2.1907706E12,1E0,1.0129377E12,2.318714E-3,8.824674E-3,2.3898147E5,3.9894668E4,2.1030048E10,1.1636418E-3,-1.5749363E-3,3.1546597E2,2.7349632E6,-7.4644466E-3,1.8559325E-4,-5.116429E-3,-1.02106E-4,2.1049242E0,1E0,2.035E3,-2.6946599E-3,-3.1244094E-4,2.779905E-3,8.1446184E7,3.3487454E11,1.5310032E-3,-6.989672E-4,1.7669548E-3,-3.7488213E-4],"split_indices":[51,30,63,30,0,0,32,32,5,0,0,4,27,0,0,0,0,41,8,2,0,0,0,5,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.42E2,2.25E2,1.7E1,2.13E2,1.2E1,5E0,1.2E1,2.02E2,1.1E1,7E0,5E0,1.2E1,1.9E2,5E0,6E0,5E0,7E0,1.73E2,1.7E1,1.67E2,6E0,5E0,1.2E1,1.24E2,4.3E1,1.6E1,1.08E2,2.5E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.2498172E-3,-6.40194E-3,5.3274993E-2,-5.301227E-4,-3.8854044E-2,7.6884236E-3,5.984384E-3,-1.1237327E-2,1.9594474E-2,-1.5860189E-2,-5.032371E-3,-1.0264154E-3,2.6013607E-3,-3.898313E-3,-6.985769E-3,1.0215776E-2,3.8920548E-3,-5.8899168E-2,1.741031E-2,2.285881E-2,-1.3174408E-2,-1.5907022E-3,1.9109933E-2,-4.006052E-3,-6.3807104E-4,2.1749926E-3,-8.7672466E-4,-4.7111185E-3,4.584058E-2,7.9084316E-4,-8.212237E-2,3.298469E-3,7.075076E-3,-1.2670029E-3,1.5805835E-3,3.0665214E-3,8.3792035E-4,-3.0949395E-3,7.82265E-3,-1.183937E-3,-6.412981E-3,-2.1444529E-3,1.9360254E-2,7.834009E-4,-9.1098336E-4,-6.2987914E-5,1.9884189E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,19,-1,21,-1,23,25,27,29,-1,31,-1,-1,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,43,-1,-1,-1,45,-1,-1,-1,-1],"loss_changes":[6.8323344E-2,4.2282127E-2,1.15400225E-1,4.0945563E-2,5.510333E-2,0E0,2.5095057E-2,1.3015412E-1,4.0416658E-2,3.968752E-2,0E0,0E0,0E0,2.9837143E-2,0E0,2.4546484E-2,0E0,1.4821291E-2,1.9113174E-2,1.985774E-2,8.576088E-2,0E0,3.2000244E-2,0E0,0E0,0E0,0E0,1.4933981E-2,9.007271E-3,3.7151434E-2,4.6626747E-2,0E0,2.9109616E-2,0E0,0E0,0E0,0E0,0E0,1.8819455E-2,0E0,0E0,0E0,1.8032562E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,13,13,15,15,17,17,18,18,19,19,20,20,22,22,27,27,28,28,29,29,30,30,32,32,38,38,42,42],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,20,-1,22,-1,24,26,28,30,-1,32,-1,-1,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,44,-1,-1,-1,46,-1,-1,-1,-1],"split_conditions":[1.3908939E3,4.6808E4,1.3770975E2,1.6634E4,8.756827E-1,7.6884236E-3,2.3E1,4.966421E6,9.1648686E-1,4.4818586E-1,-5.032371E-3,-1.0264154E-3,2.6013607E-3,1.2414E4,-6.985769E-3,7.61E2,3.8920548E-3,2.3283134E7,6.557827E-1,2.3898147E5,4.2858253E8,-1.5907022E-3,2.0242085E8,-4.006052E-3,-6.3807104E-4,2.1749926E-3,-8.7672466E-4,8.941473E5,6.1061732E7,1.3226E4,2E0,3.298469E-3,1.4285715E-1,-1.2670029E-3,1.5805835E-3,3.0665214E-3,8.3792035E-4,-3.0949395E-3,1.5779E4,-1.183937E-3,-6.412981E-3,-2.1444529E-3,2.035E3,7.834009E-4,-9.1098336E-4,-6.2987914E-5,1.9884189E-3],"split_indices":[51,9,57,9,26,0,8,27,26,26,0,0,0,9,0,2,0,44,26,32,7,0,7,0,0,0,0,27,44,9,8,0,56,0,0,0,0,0,9,0,0,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.41E2,2.21E2,2E1,1.88E2,3.3E1,5E0,1.5E1,1.23E2,6.5E1,2.6E1,7E0,1E1,5E0,1.18E2,5E0,5.8E1,7E0,1.1E1,1.5E1,3E1,8.8E1,9E0,4.9E1,6E0,5E0,8E0,7E0,1.4E1,1.6E1,7.4E1,1.4E1,8E0,4.1E1,9E0,5E0,8E0,8E0,6E0,6.8E1,8E0,6E0,7E0,3.4E1,5.1E1,1.7E1,1.9E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.0295589E-3,1.416853E-3,-5.162461E-3,6.4194E-3,-8.219619E-2,2.1229885E-3,6.856573E-2,2.4420875E-5,-8.063128E-3,1.3463427E-2,-8.283423E-3,6.6221864E-3,3.946547E-4,2.3027187E-2,-1.0085764E-2,-4.7506753E-2,-2.353529E-3,1.1027484E-2,7.023832E-2,1.3438091E-3,-3.6116816E-2,5.900491E-4,-5.8835144E-3,2.6698047E-2,-1.1601206E-2,-1.9616012E-3,3.1781103E-2,4.6401587E-3,1.4824744E-3,1.1229172E-3,-2.773241E-3,3.4026317E-3,8.961087E-3,1.846231E-2,-2.351914E-2,1.4802533E-3,-9.344836E-4,2.8601508E-3,5.764414E-4,-4.5169986E-4,1.5302586E-3,-7.034258E-4,2.2927593E-3,-1.1310723E-4,-1.8685713E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,23,25,27,-1,29,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.124187E-2,9.631335E-2,0E0,5.7794776E-2,1.0401792E-1,2.4186231E-2,6.4276226E-2,0E0,0E0,2.230369E-2,2.477534E-2,0E0,0E0,3.8881816E-2,3.1187091E-2,7.209025E-2,2.5563734E-2,1.5427862E-2,1.3702981E-2,0E0,2.9412733E-2,0E0,0E0,1.921594E-2,2.623146E-2,2.4178004E-2,1.241616E-2,0E0,0E0,0E0,0E0,0E0,8.871663E-3,2.3957234E-2,1.9474389E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,32,32,33,33,34,34],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,24,26,28,-1,30,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,1.2754595E8,-5.162461E-3,6.811E3,9.0292144E-1,2.3378747E0,6.4554214E8,2.4420875E-5,-8.063128E-3,3.3564355E0,1.3362592E-2,6.6221864E-3,3.946547E-4,9E0,2.0519513E-1,6.0106687E-2,1.3255E4,3.5351078E5,3.33386E5,1.3438091E-3,8E0,5.900491E-4,-5.8835144E-3,4.0500336E0,4.2342335E-2,4.5198887E3,1.647139E0,4.6401587E-3,1.4824744E-3,1.1229172E-3,-2.773241E-3,3.4026317E-3,1.2699157E0,5.936784E-1,3.005376E5,1.4802533E-3,-9.344836E-4,2.8601508E-3,5.764414E-4,-4.5169986E-4,1.5302586E-3,-7.034258E-4,2.2927593E-3,-1.1310723E-4,-1.8685713E-3],"split_indices":[27,44,0,2,26,57,7,0,0,55,37,0,0,3,26,41,9,32,12,0,3,0,0,56,37,50,52,0,0,0,0,0,55,26,32,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.34E2,2.28E2,6E0,2.16E2,1.2E1,2.03E2,1.3E1,7E0,5E0,9.7E1,1.06E2,5E0,8E0,6.9E1,2.8E1,1.3E1,9.3E1,5.6E1,1.3E1,1.1E1,1.7E1,8E0,5E0,2.2E1,7.1E1,3.5E1,2.1E1,6E0,7E0,5E0,1.2E1,5E0,1.7E1,2E1,5.1E1,1.2E1,2.3E1,7E0,1.4E1,1E1,7E0,1E1,1E1,2.4E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-3.4057721E-3,7.2377394E-3,-4.7559794E-2,8.8035944E-4,9.2261106E-2,-1.3550756E-2,-8.210871E-3,-5.5851735E-2,5.399604E-3,1.5836512E-4,8.802756E-3,-4.9228355E-2,5.610238E-2,-4.5384915E-4,-4.223334E-3,4.8855245E-2,1.3077467E-3,-5.3791665E-3,-2.5747743E-2,3.9617033E-3,1.1396242E-3,2.1678627E-4,3.6655543E-3,-1.275764E-2,2.0933878E-2,9.0736407E-4,-1.9109789E-3,-2.3135861E-2,1.5881611E-2,3.135276E-2,-2.0319966E-3,8.403147E-3,-3.4480993E-2,1.6314941E-3,-7.43309E-3,3.7267904E-3,2.086318E-2,1.1536906E-3,-1.3629112E-3,-2.1906095E-3,1.4568713E-4,8.1574835E-4,-1.6220568E-3,-1.3646558E-3,1.2477159E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,19,-1,-1,21,23,-1,25,-1,-1,-1,-1,27,29,-1,-1,31,33,35,-1,37,39,-1,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0890356E-1,1.0099711E-1,2.0341891E-1,4.5169163E-2,1.1809781E-1,9.470893E-2,0E0,2.1561757E-2,2.8919838E-2,0E0,0E0,3.8057126E-2,1.050644E-2,0E0,0E0,1.961064E-2,4.1674435E-2,0E0,1.6110577E-2,0E0,0E0,0E0,0E0,2.6523188E-2,4.4732545E-2,0E0,0E0,2.3547672E-2,1.1627635E-2,2.7869333E-2,0E0,1.272E-2,2.5679559E-2,0E0,9.391761E-3,0E0,1.7194545E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,12,12,15,15,16,16,18,18,23,23,24,24,27,27,28,28,29,29,31,31,32,32,34,34,36,36],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,20,-1,-1,22,24,-1,26,-1,-1,-1,-1,28,30,-1,-1,32,34,36,-1,38,40,-1,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8639234E11,5.818E3,3.2656612E5,4.75E2,2.6998322E5,2.421233E0,-8.210871E-3,1.8928572E0,1.1568E4,1.5836512E-4,8.802756E-3,8.87885E5,2.396E3,-4.5384915E-4,-4.223334E-3,7.86E2,1.6971204E2,-5.3791665E-3,2.1602634E9,3.9617033E-3,1.1396242E-3,2.1678627E-4,3.6655543E-3,2.195572E0,9.159633E0,9.0736407E-4,-1.9109789E-3,2.0154546E2,5.56E2,1.0932244E9,-2.0319966E-3,2.4714162E5,1.4852E4,1.6314941E-3,3.1605058E0,3.7267904E-3,1.6416188E9,1.1536906E-3,-1.3629112E-3,-2.1906095E-3,1.4568713E-4,8.1574835E-4,-1.6220568E-3,-1.3646558E-3,1.2477159E-3],"split_indices":[30,2,32,2,32,52,0,52,28,0,0,1,0,0,0,2,51,0,12,0,0,0,0,53,55,0,0,4,0,5,0,46,10,0,52,0,5,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.3E2,1.86E2,4.4E1,1.74E2,1.2E1,3.6E1,8E0,1.2E1,1.62E2,7E0,5E0,2.4E1,1.2E1,6E0,6E0,1.3E1,1.49E2,5E0,1.9E1,5E0,7E0,6E0,7E0,8.7E1,6.2E1,5E0,1.4E1,6.4E1,2.3E1,5.4E1,8E0,1.7E1,4.7E1,1.2E1,1.1E1,8E0,4.6E1,1.2E1,5E0,3.4E1,1.3E1,6E0,5E0,5E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.6346957E-3,5.1758504E-3,-4.350773E-3,-8.0914574E-4,3.6902025E-2,4.4245776E-3,-4.7348823E-2,1.1199674E-1,-1.3677226E-2,6.982787E-3,-1.7699461E-3,3.989366E-3,-7.566616E-3,8.364819E-3,1.9110148E-3,-3.2526106E-2,1.5355061E-3,4.409876E-4,3.4976084E-2,-1.4979984E-3,1.1686336E-3,-2.0076174E-4,-2.3327414E-3,2.2806298E-2,-8.599956E-3,9.237417E-4,2.9222814E-3,1.3162939E-2,3.3935644E-3,-1.5599059E-3,-4.502272E-2,1.6365551E-3,-2.705365E-4,-3.0779545E-4,2.2239636E-3,-4.0433533E-4,-3.605479E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,7,9,11,13,15,17,-1,19,-1,-1,-1,21,-1,23,25,-1,-1,-1,-1,27,29,-1,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1],"loss_changes":[6.044704E-2,4.4272143E-2,0E0,4.819393E-2,1.4206648E-1,2.0162538E-2,1.2434725E-1,6.889866E-2,2.1799592E-2,3.0932963E-2,0E0,1.3099604E-2,0E0,0E0,0E0,8.936077E-3,0E0,2.8100215E-2,1.2504723E-2,0E0,0E0,0E0,0E0,1.9241873E-2,2.5315158E-2,0E0,0E0,1.5731132E-2,0E0,2.284315E-2,1.9436844E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,15,15,17,17,18,18,23,23,24,24,27,27,29,29,30,30],"right_children":[2,4,-1,6,8,10,12,14,16,18,-1,20,-1,-1,-1,22,-1,24,26,-1,-1,-1,-1,28,30,-1,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,7.737766E2,-4.350773E-3,7.960309E7,1.3770975E2,6.2235913E2,6.540767E0,6.482745E7,9.295463E6,2.5245471E2,-1.7699461E-3,2.6119404E-2,-7.566616E-3,8.364819E-3,1.9110148E-3,6.6704494E5,1.5355061E-3,2.92E2,1.6155814E6,-1.4979984E-3,1.1686336E-3,-2.0076174E-4,-2.3327414E-3,9E0,3.7352297E0,9.237417E-4,2.9222814E-3,1.9195E4,3.3935644E-3,7.05571E5,4.066361E5,1.6365551E-3,-2.705365E-4,-3.0779545E-4,2.2239636E-3,-4.0433533E-4,-3.605479E-3],"split_indices":[27,51,0,44,57,51,56,44,31,51,0,56,0,0,0,27,0,0,31,0,0,0,0,3,55,0,0,1,0,9,31,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.37E2,2.32E2,5E0,1.96E2,3.6E1,1.77E2,1.9E1,1.4E1,2.2E1,1.68E2,9E0,1.4E1,5E0,6E0,8E0,1.6E1,6E0,1.37E2,3.1E1,5E0,9E0,7E0,9E0,3.9E1,9.8E1,2.2E1,9E0,3.4E1,5E0,8.3E1,1.5E1,1.5E1,1.9E1,7.6E1,7E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.9786546E-3,7.96384E-3,-6.605213E-2,2.6303518E-3,6.974543E-2,-1.0396843E-2,4.1576973E-3,-6.6839494E-3,2.260989E-2,1.8284008E-3,7.94576E-3,1.3801141E-3,-1.3726893E-3,-4.91693E-4,-5.763022E-3,-1.520536E-2,4.2722527E-2,-1.3267429E-3,1.2785455E-3,-4.7758196E-2,4.8060887E-3,6.908538E-4,-3.1426035E-2,5.928061E-2,-5.564784E-4,-3.6851377E-3,5.0281023E-4,1.3939907E-2,-1.8161958E-2,-2.6611073E-3,1.5960106E-4,4.172763E-2,3.9673913E-3,4.6292283E-3,6.0184684E-2,-5.92119E-2,6.3034723E-4,2.6128453E-3,1.8389497E-4,9.292383E-4,-3.966406E-4,3.6156748E-3,7.585402E-4,-1.2185522E-3,-3.9691306E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,19,-1,21,23,-1,-1,25,27,-1,29,31,-1,-1,-1,33,35,-1,-1,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.00691535E-1,7.461563E-2,2.3197126E-1,3.9231163E-2,1.2983266E-1,0E0,1.465839E-2,1.0793899E-1,5.1607434E-2,1.0781729E-2,0E0,0E0,0E0,3.4785785E-2,0E0,1.2365222E-2,4.0848322E-2,0E0,0E0,2.892397E-2,2.647642E-2,0E0,1.5643269E-2,1.3458759E-2,0E0,0E0,0E0,3.847271E-2,4.8325628E-2,0E0,0E0,1.3806369E-2,0E0,1.6484378E-2,1.165944E-2,1.2459122E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,13,13,15,15,16,16,19,19,20,20,22,22,23,23,27,27,28,28,31,31,33,33,34,34,35,35],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,20,-1,22,24,-1,-1,26,28,-1,30,32,-1,-1,-1,34,36,-1,-1,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,5.818E3,9.4014386E2,2.5245471E2,2.6998322E5,-1.0396843E-2,5.158987E6,4.360773E7,5.936784E-1,4.3085775E11,7.94576E-3,1.3801141E-3,-1.3726893E-3,9.541066E5,-5.763022E-3,2.0519513E-1,3.4231E4,-1.3267429E-3,1.2785455E-3,4.8444237E8,2.69011E5,6.908538E-4,2.5324342E5,2.38502E0,-5.564784E-4,-3.6851377E-3,5.0281023E-4,1.24E3,1.7768E4,-2.6611073E-3,1.5960106E-4,1.9185424E0,3.9673913E-3,1.9413766E8,3.474114E5,1.1767652E6,6.3034723E-4,2.6128453E-3,1.8389497E-4,9.292383E-4,-3.966406E-4,3.6156748E-3,7.585402E-4,-1.2185522E-3,-3.9691306E-3],"split_indices":[44,2,4,51,32,0,1,44,26,30,0,0,0,44,0,26,10,0,0,7,1,0,32,52,0,0,0,2,10,0,0,53,0,7,32,47,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,2.26E2,1.9E1,2.09E2,1.7E1,5E0,1.4E1,1.43E2,6.6E1,1.1E1,6E0,8E0,6E0,1.37E2,6E0,2.3E1,4.3E1,5E0,6E0,1.3E1,1.24E2,8E0,1.5E1,3.3E1,1E1,8E0,5E0,8.9E1,3.5E1,8E0,7E0,2.2E1,1.1E1,7.5E1,1.4E1,1.5E1,2E1,1.5E1,7E0,3.4E1,4.1E1,9E0,5E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.199706E-3,-2.5044498E-3,6.0019325E-2,7.509839E-4,-5.6967856E-3,7.728519E-3,8.000521E-3,-1.0056097E-2,1.3386144E-2,1.303322E-3,-1.1132052E-3,-3.5715837E-3,-6.1771763E-3,2.7373314E-2,-1.3437854E-2,2.1227475E-2,-1.6396772E-2,6.442794E-2,-2.028358E-3,7.3942065E-4,-3.4525797E-2,-8.7538076E-4,3.4963563E-2,1.4101951E-2,-4.097762E-2,4.6121466E-4,8.283531E-2,2.2085852E-3,-1.0714558E-2,-2.9205552E-3,-3.880846E-4,2.629374E-3,6.701655E-4,2.9452197E-2,-1.9527418E-3,-1.8435733E-2,-8.796482E-2,5.664252E-3,5.626845E-2,-2.547334E-2,1.2954896E-3,-3.0480823E-4,1.8614737E-3,-2.5522558E-3,3.8993297E-4,-1.304542E-3,-5.300473E-3,5.463012E-4,3.392598E-3,-2.892282E-3,-3.5403593E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,15,-1,17,19,21,23,25,27,-1,29,-1,31,33,35,-1,37,-1,39,-1,-1,-1,-1,41,-1,43,45,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.869242E-2,9.442629E-2,1.08706035E-1,3.1404376E-2,0E0,0E0,1.0279317E-2,1.0157537E-1,4.0138166E-2,0E0,0E0,3.819632E-2,0E0,7.639344E-2,2.3845643E-2,2.3566684E-2,5.9802935E-2,3.0080631E-2,1.8247265E-2,0E0,1.6687667E-2,0E0,1.3682194E-2,3.299927E-2,4.4982232E-2,0E0,1.9071639E-2,0E0,2.1107078E-2,0E0,0E0,0E0,0E0,1.27014145E-2,0E0,3.2531932E-2,2.1048553E-2,0E0,1.2488686E-2,1.7212994E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,22,22,23,23,24,24,26,26,28,28,33,33,35,35,36,36,38,38,39,39],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,16,-1,18,20,22,24,26,28,-1,30,-1,32,34,36,-1,38,-1,40,-1,-1,-1,-1,42,-1,44,46,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.406168E3,1.3575526E8,1E0,1.662E4,-5.6967856E-3,7.728519E-3,2.631789E3,4.966421E6,3.4577703E0,1.303322E-3,-1.1132052E-3,1.3579175E0,-6.1771763E-3,2.2575532E-1,1.8733E4,3.8E1,1.3318E4,7.72E2,6.093073E5,7.3942065E-4,2.7573213E8,-8.7538076E-4,1E0,2.6490524E0,4.2858253E8,4.6121466E-4,3.77E2,2.2085852E-3,5.5811644E5,-2.9205552E-3,-3.880846E-4,2.629374E-3,6.701655E-4,2.3898147E5,-1.9527418E-3,1.041E3,9.62689E6,5.664252E-3,1.222E3,4.761905E-2,1.2954896E-3,-3.0480823E-4,1.8614737E-3,-2.5522558E-3,3.8993297E-4,-1.304542E-3,-5.300473E-3,5.463012E-4,3.392598E-3,-2.892282E-3,-3.5403593E-4],"split_indices":[51,44,63,9,0,0,51,27,55,0,0,56,0,40,9,10,9,2,47,0,7,0,111,55,7,0,0,0,32,0,0,0,0,32,0,2,44,0,2,55,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,2.33E2,1.8E1,2.28E2,5E0,5E0,1.3E1,1.23E2,1.05E2,8E0,5E0,1.18E2,5E0,6.9E1,3.6E1,4E1,7.8E1,3E1,3.9E1,1.5E1,2.1E1,1E1,3E1,3.5E1,4.3E1,8E0,2.2E1,5E0,3.4E1,9E0,1.2E1,1.3E1,1.7E1,2.8E1,7E0,3E1,1.3E1,7E0,1.5E1,2.5E1,9E0,7E0,2.1E1,1.2E1,1.8E1,5E0,8E0,5E0,1E1,7E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[1.04801824E-4,-8.741257E-3,3.1234108E-2,-2.0599829E-3,-7.834831E-2,8.112104E-2,-2.4802582E-3,-1.7493699E-2,1.7311076E-2,7.373956E-4,-9.128635E-3,7.131212E-3,3.131185E-2,1.8086615E-3,-1.8348338E-2,-5.563643E-3,-8.871861E-3,4.7474064E-2,-1.8517314E-2,2.4423166E-3,3.2628017E-5,-1.6597036E-3,1.4327446E-4,2.7197744E-3,-1.4669924E-2,3.0371567E-4,5.566333E-2,-3.6650065E-2,8.69889E-4,-4.3010954E-2,2.4963103E-5,6.940482E-2,9.298754E-4,-1.7762065E-2,-4.0742573E-3,-4.370314E-3,-1.2374411E-2,2.1226246E-2,-2.274771E-2,3.346003E-2,4.385037E-3,1.2651855E-5,-2.565807E-3,1.6412078E-3,-1.460648E-3,-1.043934E-3,1.417528E-3,-1.6611467E-4,-4.006773E-3,2.7416367E-3,2.3652239E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,-1,23,25,27,-1,-1,-1,-1,-1,29,-1,31,33,-1,35,37,39,-1,41,-1,-1,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.471E-2,8.494528E-2,8.766034E-2,5.0529357E-2,1.9134197E-1,7.561672E-2,2.236248E-2,8.472399E-2,8.1930935E-2,0E0,0E0,0E0,9.74632E-3,0E0,9.6529005E-3,0E0,3.618637E-2,1.33034885E-2,2.4702117E-2,0E0,0E0,0E0,0E0,0E0,3.415147E-2,0E0,1.4940634E-2,2.2739641E-2,0E0,4.5044497E-2,2.7037393E-2,2.076386E-2,0E0,1.3433266E-2,0E0,0E0,1.9728638E-2,1.3867318E-2,3.3700727E-2,8.937948E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,16,16,17,17,18,18,24,24,26,26,27,27,29,29,30,30,31,31,33,33,36,36,37,37,38,38,39,39],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,-1,24,26,28,-1,-1,-1,-1,-1,30,-1,32,34,-1,36,38,40,-1,42,-1,-1,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.383E3,2.697E3,9E0,3.200388E2,3.7808718E11,4.083812E7,4.44E3,2.2253042E-5,2.9807162E0,7.373956E-4,-9.128635E-3,7.131212E-3,3.1665432E0,1.8086615E-3,1.9316398E3,-5.563643E-3,4.741633E3,5.1958606E5,2.3950179E0,2.4423166E-3,3.2628017E-5,-1.6597036E-3,1.4327446E-4,2.7197744E-3,2.9219403E5,3.0371567E-4,6.0004652E7,7.631266E0,8.69889E-4,1.6407117E2,1.2141942E6,1.4943152E5,9.298754E-4,3.9218522E5,-4.0742573E-3,-4.370314E-3,1.8190146E8,2.32E2,8.869983E5,1.156884E4,4.385037E-3,1.2651855E-5,-2.565807E-3,1.6412078E-3,-1.460648E-3,-1.043934E-3,1.417528E-3,-1.6611467E-4,-4.006773E-3,2.7416367E-3,2.3652239E-4],"split_indices":[2,2,3,4,30,44,2,37,55,0,0,0,55,0,4,0,47,27,53,0,0,0,0,0,32,0,44,55,0,4,50,32,0,32,0,0,7,11,31,31,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.33E2,1.82E2,5.1E1,1.67E2,1.5E1,2E1,3.1E1,9.3E1,7.4E1,9E0,6E0,7E0,1.3E1,8E0,2.3E1,6E0,8.7E1,4E1,3.4E1,7E0,6E0,1.2E1,1.1E1,6E0,8.1E1,7E0,3.3E1,2.3E1,1.1E1,2.7E1,5.4E1,2.3E1,1E1,1.8E1,5E0,9E0,1.8E1,2.8E1,2.6E1,1.1E1,1.2E1,1.3E1,5E0,5E0,1.3E1,5E0,2.3E1,2.1E1,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-4.0591103E-3,-3.9435085E-2,4.457957E-3,-1.2278499E-2,-9.046441E-3,-1.795109E-5,5.7051003E-2,-3.6522787E-2,4.0550582E-2,-5.179408E-3,2.8637653E-2,5.980876E-3,-7.4289215E-4,-1.5222893E-2,-5.374336E-3,3.3163216E-3,6.334822E-4,-9.6270227E-4,-5.3110927E-2,2.971857E-3,3.4768635E-3,-5.4661635E-2,8.869248E-4,-4.5793448E-3,2.8567344E-3,-5.636919E-3,3.3598623E-4,1.0137091E-3,-1.053817E-3,-6.4411707E-4,-3.6733039E-3,-1.801534E-2,8.73364E-3,-4.9150623E-2,-2.0723774E-3,1.350328E-2,-2.0734766E-3,-1.2020682E-3,-3.7731628E-3,1.615913E-3,-9.1939775E-4,8.243826E-4,-1.3559201E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,-1,-1,-1,23,25,27,-1,29,-1,31,-1,-1,-1,-1,-1,-1,-1,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.6103285E-2,2.1202174E-1,4.802296E-2,5.6555565E-2,0E0,2.8102309E-2,9.110482E-2,5.1006682E-2,1.1208722E-2,3.2489084E-2,3.5845842E-2,0E0,0E0,3.547755E-2,0E0,0E0,0E0,3.4948565E-2,5.75368E-2,1.080348E-2,0E0,1.1821799E-2,0E0,2.5557335E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.5141736E-2,1.941136E-2,1.6407225E-2,3.4059774E-2,1.4501075E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,17,17,18,18,19,19,21,21,23,23,31,31,32,32,33,33,34,34,35,35],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,-1,-1,-1,24,26,28,-1,30,-1,32,-1,-1,-1,-1,-1,-1,-1,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.7722336E4,3.2647366E8,1.3908939E3,2.1E1,-9.046441E-3,1E0,6.234292E-1,4.3951338E11,1.81E2,9.512301E-1,2.2913464E2,5.980876E-3,-7.4289215E-4,2.567751E2,-5.374336E-3,3.3163216E-3,6.334822E-4,1.0007368E7,2.1807466E0,8.856264E6,3.4768635E-3,2.41E2,8.869248E-4,5.885039E-1,2.8567344E-3,-5.636919E-3,3.3598623E-4,1.0137091E-3,-1.053817E-3,-6.4411707E-4,-3.6733039E-3,2.6727284E5,5.5910575E6,1.6384877E5,7.65244E5,6.6E1,-2.0734766E-3,-1.2020682E-3,-3.7731628E-3,1.615913E-3,-9.1939775E-4,8.243826E-4,-1.3559201E-3],"split_indices":[31,7,51,8,0,112,26,30,10,26,51,0,0,4,0,0,0,31,52,44,0,11,0,26,0,0,0,0,0,0,0,32,46,32,47,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.51E2,4.8E1,2.03E2,4.2E1,6E0,1.88E2,1.5E1,2.9E1,1.3E1,1.6E2,2.8E1,7E0,8E0,2.4E1,5E0,5E0,8E0,1.48E2,1.2E1,1.9E1,9E0,1.1E1,1.3E1,1.41E2,7E0,5E0,7E0,1.1E1,8E0,5E0,6E0,7E1,7.1E1,2.3E1,4.7E1,6.6E1,5E0,1.5E1,8E0,1.5E1,3.2E1,6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.0708933E-3,7.3688296E-3,-6.485702E-2,-2.1866364E-3,9.897471E-3,2.4113261E-4,-7.933647E-3,-2.6286608E-3,1.2102428E-2,1.7229348E-2,-9.700811E-3,4.069004E-3,3.545109E-2,2.564389E-3,-3.6829233E-3,2.1746494E-3,-2.426704E-3,-4.651197E-3,5.0964493E-2,7.7362853E-4,-1.1080196E-3,-1.222941E-2,1.2618123E-3,8.4099907E-4,-1.0678656E-3,6.0542386E-2,-1.4680435E-3,4.2306856E-4,-1.3793769E-3,2.1534443E-3,6.3108564E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,-1,7,-1,-1,-1,9,11,13,15,17,19,-1,-1,21,23,25,-1,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1],"loss_changes":[8.699539E-2,3.2544427E-2,1.4131744E-1,0E0,3.348435E-2,0E0,0E0,0E0,2.3865238E-2,4.1102104E-2,3.640344E-2,2.9087365E-2,4.50299E-2,1.4676942E-2,0E0,0E0,2.687277E-2,9.754062E-3,4.356867E-2,0E0,0E0,2.726974E-2,0E0,0E0,0E0,4.103826E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,21,21,25,25],"right_children":[2,4,6,-1,8,-1,-1,-1,10,12,14,16,18,20,-1,-1,22,24,26,-1,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1],"split_conditions":[2.5E1,8.180048E5,7.475293E5,-2.1866364E-3,3.814714E-2,2.4113261E-4,-7.933647E-3,-2.6286608E-3,3.4231E4,1.9851093E2,9.512301E-1,7.110474E7,4.2378342E-1,1E1,-3.6829233E-3,2.1746494E-3,2.1577182E0,1.8733E4,5.182959E1,7.7362853E-4,-1.1080196E-3,2.818264E5,1.2618123E-3,8.4099907E-4,-1.0678656E-3,5.818E3,-1.4680435E-3,4.2306856E-4,-1.3793769E-3,2.1534443E-3,6.3108564E-3],"split_indices":[3,44,27,0,57,0,0,0,10,51,26,5,26,3,0,0,53,9,56,0,0,31,0,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.43E2,2.26E2,1.7E1,9E0,2.17E2,1.1E1,6E0,6E0,2.11E2,1.71E2,4E1,1E2,7.1E1,3.5E1,5E0,1.2E1,8.8E1,2E1,5.1E1,2.3E1,1.2E1,6.7E1,2.1E1,9E0,1.1E1,4.6E1,5E0,3.1E1,3.6E1,4.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.4341812E-3,1.0399644E-2,-3.1646784E-2,-2.1918627E-2,1.6249845E-2,-9.163613E-3,-2.1723341E-3,-5.7170745E-2,-7.3081325E-4,3.685801E-3,1.2206822E-2,1.7010485E-3,-1.6868185E-2,-5.962702E-4,-3.9082724E-3,1.631002E-3,-2.622152E-2,7.232925E-3,5.5376098E-2,-3.285993E-3,-8.2752074E-4,6.8927405E-4,-2.438337E-3,1.1146255E-2,-1.7450374E-3,8.991906E-4,3.5256087E-3,-1.0756545E-3,1.3162583E-3,3.2085113E-2,2.3620182E-3,1.9736575E-2,4.5466046E-3,2.5483664E-2,-6.22218E-3,-4.4085118E-5,1.7280192E-3,-1.6957257E-4,2.3857271E-3,-1.359074E-3,8.055122E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,19,-1,-1,-1,21,23,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1],"loss_changes":[6.044048E-2,3.9304253E-2,2.0689541E-1,2.3859512E-2,4.5337714E-2,0E0,2.1743352E-2,1.4412325E-2,2.0701783E-2,0E0,3.5242908E-2,0E0,2.4195088E-2,0E0,0E0,0E0,1.5855843E-2,2.7393505E-2,1.2181733E-2,0E0,1.5903162E-2,0E0,0E0,2.5490748E-2,0E0,0E0,0E0,0E0,0E0,3.281425E-2,1.9808162E-2,1.3896718E-2,0E0,2.1852769E-2,4.2591933E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,12,12,16,16,17,17,18,18,20,20,23,23,29,29,30,30,31,31,33,33,34,34],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,20,-1,-1,-1,22,24,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,7.38E2,2.426363E5,2.1152425E6,1.1280869E9,-9.163613E-3,1.09E3,1.5539E4,5.1373015E6,3.685801E-3,2.7319932E6,1.7010485E-3,1.0164216E2,-5.962702E-4,-3.9082724E-3,1.631002E-3,1.3986014E-2,3.4867987E-1,1.4E3,-3.285993E-3,5.096271E-1,6.8927405E-4,-2.438337E-3,1.9818894E0,-1.7450374E-3,8.991906E-4,3.5256087E-3,-1.0756545E-3,1.3162583E-3,2.539E3,2.29924E5,5.69E2,4.5466046E-3,9.079337E1,9.2485E5,-4.4085118E-5,1.7280192E-3,-1.6957257E-4,2.3857271E-3,-1.359074E-3,8.055122E-4],"split_indices":[9,2,31,44,30,0,2,9,44,0,27,0,51,0,0,0,56,37,2,0,40,0,0,52,0,0,0,0,0,2,1,0,0,51,1,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.45E2,2.05E2,4E1,3.1E1,1.74E2,5E0,3.5E1,1.1E1,2E1,9E0,1.65E2,9E0,2.6E1,5E0,6E0,8E0,1.2E1,1.49E2,1.6E1,5E0,2.1E1,5E0,7E0,1.38E2,1.1E1,7E0,9E0,1.2E1,9E0,4E1,9.8E1,3.5E1,5E0,2.6E1,7.2E1,1.7E1,1.8E1,1.3E1,1.3E1,3.6E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.575293E-4,-5.2696434E-3,4.6820566E-2,-7.22002E-4,-6.8282135E-2,7.010593E-3,8.373848E-3,2.4539739E-2,-5.9444197E-3,4.7728312E-4,-8.492969E-3,2.55593E-2,-1.3369139E-3,-1.507529E-2,4.758629E-2,-1.7465979E-2,1.0620374E-2,9.6856646E-5,2.3451722E-3,9.702391E-4,-3.041511E-3,2.9588758E-3,7.928572E-4,-1.1798453E-2,-4.830495E-3,3.6794837E-3,3.8940322E-3,-6.708696E-3,-3.07618E-3,9.594117E-3,-1.8551067E-3,-4.3985885E-2,4.4114403E-3,1.7186642E-3,-1.6431147E-3,-3.3107292E-4,-3.1950995E-3,-1.8190907E-3,4.8405203E-4,-1.4014044E-3,5.920342E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,-1,19,21,23,25,-1,-1,-1,-1,-1,-1,27,-1,29,-1,31,-1,33,-1,35,37,-1,39,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5349154E-2,6.385318E-2,1.1148501E-1,2.7718691E-2,1.4402956E-1,0E0,1.4479807E-2,3.3585005E-2,3.3393785E-2,0E0,0E0,9.176372E-3,0E0,2.8282674E-2,1.128963E-2,5.1074784E-2,3.753114E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7684998E-2,0E0,1.8155873E-2,0E0,3.7916034E-2,0E0,1.9291941E-2,0E0,2.0157497E-2,2.0524923E-2,0E0,1.9540794E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,23,23,25,25,27,27,29,29,31,31,32,32,34,34],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,-1,20,22,24,26,-1,-1,-1,-1,-1,-1,28,-1,30,-1,32,-1,34,-1,36,38,-1,40,-1,-1,-1,-1,-1,-1],"split_conditions":[5.818E3,1.0388931E8,3.0451374E9,1.647139E0,1E0,7.010593E-3,1E0,1.4221454E0,2.148789E0,4.7728312E-4,-8.492969E-3,2.3E1,-1.3369139E-3,2.0260053E5,1.4582623E0,4.476758E6,3.26989E6,9.6856646E-5,2.3451722E-3,9.702391E-4,-3.041511E-3,2.9588758E-3,7.928572E-4,9.5067096E-1,-4.830495E-3,1.4991511E0,3.8940322E-3,3.6452372E5,-3.07618E-3,6.5229E4,-1.8551067E-3,2.1161814E7,6.71E2,1.7186642E-3,2.2652755E8,-3.3107292E-4,-3.1950995E-3,-1.8190907E-3,4.8405203E-4,-1.4014044E-3,5.920342E-4],"split_indices":[2,44,12,52,80,0,15,52,53,0,0,8,0,46,53,27,27,0,0,0,0,0,0,26,0,40,0,27,0,28,0,12,2,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.48E2,2.22E2,2.6E1,2.08E2,1.4E1,6E0,2E1,3.5E1,1.73E2,9E0,5E0,1.4E1,6E0,1.3E1,2.2E1,1.02E2,7.1E1,8E0,6E0,8E0,5E0,1.3E1,9E0,9.7E1,5E0,6.6E1,5E0,9E1,7E0,5.9E1,7E0,2E1,7E1,1.6E1,4.3E1,9E0,1.1E1,8E0,6.2E1,1.4E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.0726674E-3,7.5854454E-3,-4.432919E-2,1.5386986E-3,6.931116E-2,-8.984074E-3,4.744149E-3,-2.4539526E-3,4.7094645E-3,3.6406026E-3,6.8064244E-3,2.8080898E-3,-1.4696119E-2,-3.2905678E-3,2.2369364E-2,-1.1614038E-3,1.4623696E-3,-2.146363E-3,7.306956E-5,6.337729E-3,-2.9449845E-2,4.7340814E-2,5.5534435E-3,-2.2722607E-2,1.7249405E-2,-4.7287783E-3,-5.1113097E-3,3.1109015E-3,3.789911E-4,1.4888023E-3,-1.0927316E-2,2.1062496E-3,-4.7107015E-2,2.1910746E-2,-1.4887579E-3,2.2716084E-2,-1.3791905E-3,1.7406947E-4,-2.2999023E-3,-3.632827E-3,-1.3258678E-3,1.3840379E-3,-5.8073015E-4,2.3227693E-3,-7.7555975E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,-1,11,-1,13,15,-1,-1,17,19,21,-1,-1,-1,-1,23,25,27,29,31,33,-1,35,-1,-1,-1,37,-1,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.8345465E-2,7.528778E-2,2.2139926E-1,3.308158E-2,9.594651E-2,0E0,2.6638411E-2,0E0,2.4938844E-2,1.0144935E-2,0E0,0E0,9.740741E-3,3.0914698E-2,2.2832578E-2,0E0,0E0,0E0,0E0,2.893068E-2,6.0038388E-2,1.7964378E-2,1.5749745E-2,4.452702E-2,1.6187409E-2,0E0,1.9126E-2,0E0,0E0,0E0,1.323504E-2,0E0,8.885089E-3,1.9033536E-2,0E0,9.235965E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,12,12,13,13,14,14,19,19,20,20,21,21,22,22,23,23,24,24,26,26,30,30,32,32,33,33,35,35],"right_children":[2,4,6,8,10,-1,12,-1,14,16,-1,-1,18,20,22,-1,-1,-1,-1,24,26,28,30,32,34,-1,36,-1,-1,-1,38,-1,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.581563E11,5.818E3,1.1219874E5,9.541066E5,1.4781E4,-8.984074E-3,1.5544465E0,-2.4539526E-3,2.148789E0,3.4349198E6,6.8064244E-3,2.8080898E-3,4.259865E5,1.9185424E0,2.6021504E0,-1.1614038E-3,1.4623696E-3,-2.146363E-3,7.306956E-5,3.6452372E5,1.4693E4,1.5544807E11,1.1502146E0,1.1280869E9,5.7880085E6,-4.7287783E-3,2.0754318E8,3.1109015E-3,3.789911E-4,1.4888023E-3,1.2121498E6,2.1062496E-3,1.8357308E-1,1.7707E4,-1.4887579E-3,8.271E3,-1.3791905E-3,1.7406947E-4,-2.2999023E-3,-3.632827E-3,-1.3258678E-3,1.3840379E-3,-5.8073015E-4,2.3227693E-3,-7.7555975E-5],"split_indices":[30,2,31,44,9,0,55,0,53,31,0,0,27,53,53,0,0,0,0,27,9,30,55,30,46,0,7,0,0,0,47,0,38,9,0,12,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.29E2,2.01E2,2.8E1,1.84E2,1.7E1,6E0,2.2E1,9E0,1.75E2,1E1,7E0,5E0,1.7E1,1.21E2,5.4E1,5E0,5E0,5E0,1.2E1,8.9E1,3.2E1,2.1E1,3.3E1,2.4E1,6.5E1,7E0,2.5E1,1.3E1,8E0,1.2E1,2.1E1,6E0,1.8E1,6E1,5E0,1.2E1,1.3E1,1.6E1,5E0,5E0,1.3E1,4.8E1,1.2E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-4.1710395E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_TakeOrderedAndProject","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"73"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-4.3699045E-2,-1.0177426E-1,3.2357824E-1,-4.2337015E-2,-4.4481683E-1,4.1763127E-2,5.5105966E-2,-9.241501E-2,2.2948945E-1,-2.2530514E-1,-5.3884722E-2,1.4220427E-2,-7.611233E-2,4.5663677E-2,-1.6117272E-1,2.1345997E-2,6.50888E-2,-3.314552E-1,3.7941143E-3,5.082868E-3,-1.4837972E-2,-1.5580411E-2,1.2765303E-1,-2.180035E-1,4.1731097E-2,8.318949E-3,-3.1442787E-3,-9.702286E-3,-2.224325E-2,2.2592081E-1,-3.5885837E-2,-1.6852038E-1,-3.306741E-2,7.666271E-3,-3.790615E-3,1.0998978E-1,2.2417948E-2,3.763718E-3,-7.809596E-3,-2.576462E-1,-2.8645573E-3,1.0208027E-2,-2.2941923E-3,-1.6525443E-1,-4.517565E-1,9.699791E-3,-9.075653E-2,-1.18612945E-2,1.2279708E-3,-2.6795909E-2,-1.2901888E-2,-2.9398213E-4,-1.1290862E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,19,21,23,-1,25,27,-1,-1,-1,-1,29,31,33,-1,-1,-1,-1,35,37,39,-1,-1,-1,41,-1,-1,-1,43,45,-1,-1,47,49,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[4.213555E0,3.4473839E0,3.8007114E0,2.008637E0,3.360231E0,0E0,6.100741E-1,1.1846125E0,7.5508857E-1,6.6884005E-1,0E0,0E0,5.640828E-1,1.2955588E0,9.6618557E-1,0E0,2.0137328E-1,1.5619421E-1,0E0,0E0,0E0,0E0,5.705784E-1,1.3607078E0,2.7303344E-1,0E0,0E0,0E0,0E0,5.430801E-1,2.0808809E-1,8.8511455E-1,0E0,0E0,0E0,2.5799036E-1,0E0,0E0,0E0,6.24779E-1,4.0725818E-1,0E0,0E0,4.1871607E-1,7.264948E-2,0E0,1.8875124E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,12,12,13,13,14,14,16,16,17,17,22,22,23,23,24,24,29,29,30,30,31,31,35,35,39,39,40,40,43,43,44,44,46,46],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,20,22,24,-1,26,28,-1,-1,-1,-1,30,32,34,-1,-1,-1,-1,36,38,40,-1,-1,-1,42,-1,-1,-1,44,46,-1,-1,48,50,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.1034263E10,5.939257E7,2.763847E6,2.9119E4,4.1763127E-2,1E1,4.49E2,1.6813E4,1.6022055E12,-5.3884722E-2,1.4220427E-2,1.0129378E12,4.18E2,3.1681168E2,2.1345997E-2,1.8918378E2,1E0,3.7941143E-3,5.082868E-3,-1.4837972E-2,-1.5580411E-2,2.202021E8,4.520813E7,1.7851065E1,8.318949E-3,-3.1442787E-3,-9.702286E-3,-2.224325E-2,4.0354648E2,2.0495868E0,1.23844246E2,-3.306741E-2,7.666271E-3,-3.790615E-3,9.157509E-1,2.2417948E-2,3.763718E-3,-7.809596E-3,2.6006784E2,2.08215E5,1.0208027E-2,-2.2941923E-3,1.9759825E0,3.5000316E5,9.699791E-3,4.0190366E5,-1.18612945E-2,1.2279708E-3,-2.6795909E-2,-1.2901888E-2,-2.9398213E-4,-1.1290862E-2],"split_indices":[2,5,54,41,9,0,3,0,9,40,0,0,40,2,61,0,61,18,0,0,0,0,7,54,66,0,0,0,0,4,63,61,0,0,0,65,0,0,0,4,1,0,0,62,41,0,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.95E2,1.69E2,2.6E1,1.45E2,2.4E1,8E0,1.8E1,1.23E2,2.2E1,1.9E1,5E0,6E0,1.2E1,4.1E1,8.2E1,9E0,1.3E1,1.4E1,5E0,7E0,5E0,7E0,3.4E1,6.4E1,1.8E1,7E0,6E0,8E0,6E0,2.1E1,1.3E1,5.9E1,5E0,9E0,9E0,1.5E1,6E0,7E0,6E0,3.8E1,2.1E1,9E0,6E0,2.7E1,1.1E1,6E0,1.5E1,1.9E1,8E0,6E0,5E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[6.8937447E-3,6.9295816E-2,-2.648259E-1,1.7660331E-2,5.7070386E-1,-6.345505E-1,8.116753E-2,1.1005546E-1,-8.011314E-2,4.3043982E-2,4.4349777E-3,-8.895128E-3,-9.060073E-1,1.8557851E-1,-4.821837E-3,-2.5018847E-1,1.7486887E-1,-1.9403672E-2,-3.4304105E-2,-5.6007277E-2,-2.5183178E-2,5.160234E-3,1.277372E-2,-1.8099958E-2,-3.3418448E-3,2.1866676E-1,-4.295884E-3,1.7445055E-1,-9.624442E-2,3.394073E-1,1.5733352E-1,1.2749216E-2,6.919538E-5,-1.906068E-1,3.2517105E-2,6.7456225E-3,1.9857757E-2,6.2894486E-2,2.5343278E-1,-2.8077334E-1,-1.1044374E-1,-5.5138E-2,8.077749E-3,1.2672926E-1,-5.7587563E-3,1.4906493E-1,1.8491602E-2,-8.042325E-3,-1.9012714E-2,7.7475474E-4,-1.8077222E-1,2.8977597E-3,-6.124295E-3,9.6804724E-4,8.482208E-3,3.8507914E-3,9.19013E-3,-3.355594E-3,-1.2844032E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,25,-1,27,-1,-1,-1,-1,-1,-1,29,-1,31,33,35,37,-1,-1,39,41,-1,-1,43,45,47,49,51,-1,53,-1,55,-1,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3586574E0,4.1582828E0,4.7846293E0,1.3367844E0,2.1512518E0,2.0464935E0,3.9379734E-1,1.8182468E0,1.0314388E0,0E0,0E0,0E0,4.405489E-1,5.120012E-2,0E0,2.3673326E-1,7.672453E-1,0E0,8.4433085E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.6130285E-1,0E0,2.2382152E-1,6.137102E-1,1.8627477E-1,3.3800578E-1,0E0,0E0,1.7860305E-1,2.6508817E-1,0E0,0E0,2.5717565E-1,1.9122922E-1,9.9497795E-2,1.5386017E-1,1.1980742E-1,0E0,7.663149E-2,0E0,1.8038243E-2,0E0,0E0,0E0,0E0,8.308849E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,16,16,18,18,25,25,27,27,28,28,29,29,30,30,33,33,34,34,37,37,38,38,39,39,40,40,41,41,43,43,45,45,50,50],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,26,-1,28,-1,-1,-1,-1,-1,-1,30,-1,32,34,36,38,-1,-1,40,42,-1,-1,44,46,48,50,52,-1,54,-1,56,-1,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.482745E7,3.847E3,6.0317725E2,2.459517E0,9.5E2,7.6499896E9,1.0881593E10,4.305949E-2,1.4065148E6,4.3043982E-2,4.4349777E-3,-8.895128E-3,2.395E3,1.8434525E6,-4.821837E-3,2.567751E2,3.2004956E5,-1.9403672E-2,1.3257E4,-5.6007277E-2,-2.5183178E-2,5.160234E-3,1.277372E-2,-1.8099958E-2,-3.3418448E-3,1.15E2,-4.295884E-3,2.160042E9,1.182134E6,8.988764E-2,1E0,1.2749216E-2,6.919538E-5,1.0469056E2,1.068875E6,6.7456225E-3,1.9857757E-2,3.9741936E0,2.758609E2,2.228762E2,3.619228E-2,6.84E2,8.077749E-3,3.7866578E6,-5.7587563E-3,2.3084E4,1.8491602E-2,-8.042325E-3,-1.9012714E-2,7.7475474E-4,8E0,2.8977597E-3,-6.124295E-3,9.6804724E-4,8.482208E-3,3.8507914E-3,9.19013E-3,-3.355594E-3,-1.2844032E-2],"split_indices":[54,2,61,67,0,5,12,67,59,0,0,0,0,41,0,4,56,0,9,0,0,0,0,0,0,10,0,5,56,67,22,0,0,61,1,0,0,65,4,4,47,0,0,54,0,9,0,0,0,0,8,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.6E2,3.6E1,1.46E2,1.4E1,1.7E1,1.9E1,7.5E1,7.1E1,8E0,6E0,7E0,1E1,1.2E1,7E0,1.1E1,6.4E1,8E0,6.3E1,5E0,5E0,7E0,5E0,6E0,5E0,5.5E1,9E0,1.4E1,4.9E1,1.7E1,3.8E1,9E0,5E0,2.8E1,2.1E1,5E0,1.2E1,2E1,1.8E1,1.2E1,1.6E1,1.3E1,8E0,1.5E1,5E0,1.1E1,7E0,7E0,5E0,6E0,1E1,5E0,8E0,5E0,1E1,5E0,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[2.392537E-2,-2.9119618E-2,3.608476E-1,7.959821E-2,-1.3753411E-1,4.182639E-2,1.08023204E-1,1.2434806E-1,-2.0930136E-2,-6.9097705E-2,-4.753976E-1,3.265759E-1,-1.203498E-1,1.738204E-1,-1.3130353E-1,-1.0849088E-1,1.4530437E-1,-3.158617E-2,-1.3234462E-2,7.322924E-3,2.1942677E-2,-1.5489598E-2,4.704634E-3,-7.4166976E-2,2.3195139E-1,2.1972982E-3,-1.4231287E-2,-2.9782148E-3,-2.1701926E-1,1.6390735E-3,1.1551262E-2,-9.540935E-3,6.83834E-3,1.6813073E-1,3.3744574E-1,-1.1321749E-1,1.2689024E-1,5.639313E-3,-3.2373032E-1,2.0979914E-1,5.3421162E-2,9.4881635E-3,2.0614967E-2,-2.752812E-2,-1.1648636E-2,1.1049423E-2,-2.3003973E-3,-4.6991932E-1,-1.4139369E-1,1.4095788E-1,2.9557776E-1,1.0270009E-2,-6.7592156E-3,3.5153867E-3,-4.823102E-3,-2.8017256E-2,-1.1570062E-2,-1.7475042E-3,-9.92029E-3,1.9818612E-3,8.625339E-3,8.418644E-3,1.9073974E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,33,-1,-1,35,37,-1,-1,-1,-1,39,41,43,45,-1,47,49,51,-1,-1,53,-1,-1,-1,55,57,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9423556E0,2.2621858E0,3.6608431E0,2.2347112E0,2.1868792E0,0E0,1.0874457E0,1.1484673E0,0E0,6.9988847E-1,3.8428974E-1,1.7060816E-1,4.9877274E-1,1.0983846E0,4.3568933E-1,7.894809E-1,1.2432212E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.1591555E-1,3.53961E-1,0E0,0E0,5.2976394E-1,1.2433171E0,0E0,0E0,0E0,0E0,1.799829E-1,1.8396306E-1,2.0782265E-1,3.0387765E-1,0E0,6.266022E-1,1.2958407E-1,3.894974E-1,0E0,0E0,9.90719E-2,0E0,0E0,0E0,2.0148659E-1,7.728529E-2,5.7011366E-2,7.287824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,23,23,24,24,27,27,28,28,33,33,34,34,35,35,36,36,38,38,39,39,40,40,43,43,47,47,48,48,49,49,50,50],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,34,-1,-1,36,38,-1,-1,-1,-1,40,42,44,46,-1,48,50,52,-1,-1,54,-1,-1,-1,56,58,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,3.5540915E0,4.083812E7,3.7578388E6,2.68225E5,4.182639E-2,1E1,4.767898E0,-2.0930136E-2,1E0,1.0455943E1,2.4010162E5,2.1374558E3,7.999278E1,6.84E2,1.7678572E0,7.2234793E0,-3.158617E-2,-1.3234462E-2,7.322924E-3,2.1942677E-2,-1.5489598E-2,4.704634E-3,2.6727284E5,4.541294E2,2.1972982E-3,-1.4231287E-2,2.5110843E2,1.6449704E0,1.6390735E-3,1.1551262E-2,-9.540935E-3,6.83834E-3,2.747292E8,2.014519E0,1.6571E4,1.8793418E0,5.639313E-3,1.3643599E1,3.2894747E5,6.557827E-1,9.4881635E-3,2.0614967E-2,4.1226995E-1,-1.1648636E-2,1.1049423E-2,-2.3003973E-3,1.5784E4,1.9123037E0,2.0043669E0,6.514E1,1.0270009E-2,-6.7592156E-3,3.5153867E-3,-4.823102E-3,-2.8017256E-2,-1.1570062E-2,-1.7475042E-3,-9.92029E-3,1.9818612E-3,8.625339E-3,8.418644E-3,1.9073974E-2],"split_indices":[2,66,54,37,9,0,3,65,0,24,66,42,4,64,0,63,66,0,0,0,0,0,0,42,4,0,0,4,67,0,0,0,0,7,63,10,62,0,67,57,36,0,0,65,0,0,0,9,63,63,61,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,1.9E2,2.9E1,9.5E1,9.5E1,9E0,2E1,8.8E1,7E0,8E1,1.5E1,1E1,1E1,7.4E1,1.4E1,6.8E1,1.2E1,7E0,8E0,5E0,5E0,5E0,5E0,1.4E1,6E1,7E0,7E0,3.5E1,3.3E1,6E0,6E0,9E0,5E0,3.9E1,2.1E1,1.9E1,1.6E1,8E0,2.5E1,2.8E1,1.1E1,9E0,1.2E1,1.2E1,7E0,1E1,6E0,1.3E1,1.2E1,1.7E1,1.1E1,6E0,5E0,5E0,7E0,8E0,5E0,5E0,7E0,5E0,1.2E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[2.1368094E-2,-4.159329E-2,3.71778E-1,-1.6208742E-2,-4.3934953E-1,8.156339E-1,7.9306334E-2,1.22388095E-1,-1.2494086E-1,-3.2855444E-2,-6.51619E-3,5.845976E-2,2.0941086E-2,2.0862225E-1,-1.0222226E-2,-8.204306E-2,1.691195E-1,-9.404816E-2,-2.8494848E-2,2.8148424E-3,1.4806462E-2,6.1701965E-3,-1.6526548E-2,1.9925216E-1,-1.652904E-3,-1.4374304E-1,1.2099955E-1,1.313897E-1,4.0839317E-1,-2.32424E-1,1.6582353E-2,-1.1960499E-3,1.3157879E-2,1.4398927E-2,8.567336E-2,2.4077643E-2,1.1148103E-2,-3.9262316E-1,-1.5020017E-1,-1.0315901E-1,1.0013024E-1,-3.666745E-3,1.1495528E-1,-2.7275786E-2,-2.6671672E-1,8.412159E-3,-2.254104E-1,1.4892471E-3,-1.1822295E-2,1.1099987E-2,-5.5955484E-4,2.0105646E-3,7.6023107E-3,-1.7194444E-2,-6.7069675E-3,-6.6853473E-3,-1.9422136E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,19,-1,21,23,25,-1,-1,-1,-1,-1,27,-1,29,31,33,35,37,39,-1,-1,-1,41,-1,-1,43,45,47,49,-1,51,-1,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7809186E0,1.8507015E0,4.219219E0,2.6374586E0,6.941507E-1,1.4022274E0,8.26913E-1,7.494836E-1,1.3318028E0,0E0,0E0,0E0,0E0,1.9617754E-1,0E0,8.4264326E-1,3.919809E-1,1.0112624E0,0E0,0E0,0E0,0E0,0E0,7.269466E-1,0E0,1.0844076E0,3.9460585E-1,2.9555595E-1,8.5020065E-2,5.8646536E-1,2.9044518E-1,0E0,0E0,0E0,1.7330772E-1,0E0,0E0,1.843493E-1,8.6295664E-1,2.2978354E-1,2.4341226E-1,0E0,8.346239E-2,0E0,7.289225E-2,0E0,3.723774E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,13,13,15,15,16,16,17,17,23,23,25,25,26,26,27,27,28,28,29,29,30,30,34,34,37,37,38,38,39,39,40,40,42,42,44,44,46,46],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,20,-1,22,24,26,-1,-1,-1,-1,-1,28,-1,30,32,34,36,38,40,-1,-1,-1,42,-1,-1,44,46,48,50,-1,52,-1,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.796E3,4.476758E6,4.083812E7,1.3582685E7,4.5971E4,1E0,6.084144E0,1.8302504E2,2E1,-3.2855444E-2,-6.51619E-3,5.845976E-2,2.0941086E-2,8.899272E5,-1.0222226E-2,4.8206668E2,2.80617E5,1.3473596E6,-2.8494848E-2,2.8148424E-3,1.4806462E-2,6.1701965E-3,-1.6526548E-2,3.9401126E2,-1.652904E-3,1.340973E-1,3.200388E2,1.5681747E9,1.6506441E9,9.9844826E1,1.137E3,-1.1960499E-3,1.3157879E-2,1.4398927E-2,4.5071664E-1,2.4077643E-2,1.1148103E-2,8.194808E5,8.8209605E-1,3.891871E5,2.7933811E1,-3.666745E-3,1.5016E4,-2.7275786E-2,6.109438E-1,8.412159E-3,4.236455E11,1.4892471E-3,-1.1822295E-2,1.1099987E-2,-5.5955484E-4,2.0105646E-3,7.6023107E-3,-1.7194444E-2,-6.7069675E-3,-6.6853473E-3,-1.9422136E-2],"split_indices":[2,37,54,12,12,18,65,56,3,0,0,0,0,37,0,57,38,37,0,0,0,0,0,4,0,47,4,40,5,61,2,0,0,0,36,0,0,57,65,42,67,0,38,0,36,0,40,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.83E2,3.2E1,1.73E2,1E1,1.2E1,2E1,7.6E1,9.7E1,5E0,5E0,5E0,7E0,1.4E1,6E0,1.4E1,6.2E1,9.2E1,5E0,6E0,8E0,8E0,6E0,5.4E1,8E0,7.5E1,1.7E1,4.2E1,1.2E1,4.8E1,2.7E1,9E0,8E0,8E0,3.4E1,7E0,5E0,1.5E1,3.3E1,1.1E1,1.6E1,5E0,2.9E1,5E0,1E1,6E0,2.7E1,6E0,5E0,7E0,9E0,1.1E1,1.8E1,5E0,5E0,1.9E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.3541933E-2,-5.5656597E-2,2.4100128E-1,-1.6598722E-2,-3.5245E-1,3.65279E-2,2.2925988E-2,8.148147E-2,-1.0975218E-1,-3.1426505E-3,-5.6320995E-1,1.152002E-1,-9.348883E-3,-1.2783587E-2,1.2931904E-1,-3.4623486E-1,-3.8937975E-2,-1.131712E-2,-3.7493456E-2,3.7977442E-2,1.1612695E-2,1.802157E-1,-1.0770603E-1,1.1920666E-3,-4.8034197E-1,1.088399E-1,-1.6871443E-1,-2.7144034E-3,6.1176936E-3,2.3537134E-1,8.604969E-2,-1.2619998E-2,2.8427374E-3,-2.9912017E-2,-9.916659E-3,1.9290088E-1,-5.7715606E-3,-2.0470764E-2,-1.1307437E-1,6.571247E-2,2.9809734E-1,-2.0023733E-2,2.0358649E-1,2.4025229E-1,2.4609836E-3,2.0745462E-3,-1.6225058E-1,-1.975555E-4,6.0862773E-3,7.402685E-3,3.2580265E-1,1.7403455E-3,-4.441111E-3,6.0816365E-3,1.2162454E-2,4.7659506E-3,1.5116245E-2,-2.2524146E-3,-2.10195E-1,1.7845295E-2,9.535855E-3,-1.3843753E-2,-4.974473E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,-1,-1,21,23,25,-1,-1,27,-1,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,-1,43,-1,-1,45,47,49,51,53,55,-1,-1,57,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,-1],"loss_changes":[2.2646139E0,2.0923235E0,3.2951803E0,1.4799207E0,1.2391601E0,0E0,4.5684126E-1,1.3301172E0,1.3764637E0,0E0,6.337261E-1,1.421797E-1,0E0,0E0,8.638755E-1,9.560735E-1,1.2649387E0,0E0,0E0,9.6623555E-2,0E0,2.8098595E-1,3.394771E-1,0E0,3.8707042E-1,6.1409485E-1,4.396537E-1,0E0,0E0,3.6352432E-1,2.9082975E-1,0E0,0E0,0E0,0E0,1.4315635E-1,0E0,0E0,2.3631665E-1,4.562175E-2,5.6297302E-2,5.4459307E-2,8.434683E-3,1.3729054E-1,0E0,0E0,1.184175E-1,0E0,0E0,0E0,3.660345E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.785992E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,14,14,15,15,16,16,19,19,21,21,22,22,24,24,25,25,26,26,29,29,30,30,35,35,38,38,39,39,40,40,41,41,42,42,43,43,46,46,50,50,58,58],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,-1,-1,22,24,26,-1,-1,28,-1,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,-1,44,-1,-1,46,48,50,52,54,56,-1,-1,58,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,-1],"split_conditions":[3.847E3,6.5023036E7,4.083812E7,1E0,8.955553E-1,3.65279E-2,2.41E3,6.519271E1,1.8417827E5,-3.1426505E-3,7.7573473E9,1.0083192E6,-9.348883E-3,-1.2783587E-2,3.1840762E7,1.4733E4,3.891871E5,-1.131712E-2,-3.7493456E-2,2.176116E3,1.1612695E-2,1.7087E4,3.5280538E5,1.1920666E-3,1.4058E4,2.1448E4,1.04790576E-1,-2.7144034E-3,6.1176936E-3,1.9216243E2,2.3950179E0,-1.2619998E-2,2.8427374E-3,-2.9912017E-2,-9.916659E-3,1.4103535E0,-5.7715606E-3,-2.0470764E-2,1.1486098E6,1.4176E4,1.4782E4,1.8541E4,2.550714E0,2.5176196E2,2.4609836E-3,2.0745462E-3,4.357202E5,-1.975555E-4,6.0862773E-3,7.402685E-3,2.1203648E8,1.7403455E-3,-4.441111E-3,6.0816365E-3,1.2162454E-2,4.7659506E-3,1.5116245E-2,-2.2524146E-3,1.4916388E0,1.7845295E-2,9.535855E-3,-1.3843753E-2,-4.974473E-3],"split_indices":[2,54,54,23,36,0,0,64,42,0,5,37,0,0,12,9,42,0,0,4,0,9,37,0,10,9,48,0,0,4,63,0,0,0,0,65,0,0,60,38,9,9,62,4,0,0,42,0,0,0,7,0,0,0,0,0,0,0,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.8E2,2.9E1,1.6E2,2E1,8E0,2.1E1,7.8E1,8.2E1,9E0,1.1E1,1.5E1,6E0,9E0,6.9E1,1.8E1,6.4E1,5E0,6E0,1E1,5E0,5.7E1,1.2E1,5E0,1.3E1,3E1,3.4E1,5E0,5E0,3.5E1,2.2E1,6E0,6E0,8E0,5E0,2.2E1,8E0,5E0,2.9E1,1E1,2.5E1,1.2E1,1E1,1.6E1,6E0,7E0,2.2E1,5E0,5E0,5E0,2E1,7E0,5E0,5E0,5E0,6E0,1E1,7E0,1.5E1,1.4E1,6E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.09259635E-2,-5.047245E-2,3.230142E-1,1.4589432E-2,-2.4155363E-1,7.850976E-1,6.7883104E-2,1.11676246E-1,-6.2747896E-2,-3.7825006E-1,1.1351093E-1,1.4624119E-2,5.3251836E-2,-4.0739E-2,1.5738506E-2,-9.36034E-3,1.5704295E-1,-1.4817749E-1,1.2941113E-1,-2.5543773E-1,-3.2442775E-2,1.5003176E-2,-7.982653E-3,6.3715116E-3,-1.0132903E-2,2.4758178E-1,6.046401E-2,2.7870344E-2,-2.0306069E-1,2.564932E-1,-3.254144E-2,-2.4529338E-2,-1.0087896E-1,3.1297922E-1,1.299508E-1,1.3658783E-1,-4.814778E-3,-4.013862E-3,5.2241413E-3,-3.0241132E-1,-9.355701E-2,1.7306963E-2,5.9767948E-3,-4.8438283E-3,1.9276767E-3,-2.0458351E-1,5.184181E-3,1.810483E-2,7.6036714E-3,8.884383E-3,2.7609593E-3,3.355232E-2,1.647585E-2,-1.9863326E-2,-7.714117E-3,-6.2915933E-4,-1.0121381E-2,-1.3088822E-2,-5.2446146E-3,-2.8490238E-3,7.352118E-3,-4.22471E-3,4.867322E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,-1,25,27,29,31,-1,-1,-1,-1,-1,33,35,37,39,41,43,-1,45,47,49,51,-1,-1,-1,53,55,-1,-1,-1,-1,57,-1,-1,-1,-1,-1,59,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.963618E0,2.1542778E0,3.955028E0,9.8310393E-1,2.1785219E0,1.4486604E0,6.4793473E-1,8.2780087E-1,1.2175488E0,9.5236254E-1,7.4621123E-1,0E0,0E0,5.1146996E-1,0E0,0E0,4.3098164E-1,4.984249E-1,4.8171595E-1,8.63335E-1,0E0,0E0,0E0,0E0,0E0,1.593399E-1,3.2808244E-1,1.2174195E-1,3.9599848E-1,1.2696195E-1,5.6603502E-2,0E0,3.6506474E-1,8.699286E-2,3.16011E-2,3.6050683E-1,0E0,0E0,0E0,2.3650026E-1,2.1083185E-1,0E0,0E0,0E0,0E0,3.889364E-2,0E0,0E0,0E0,0E0,0E0,1.4828466E-1,0E0,0E0,0E0,1.11779645E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,16,16,17,17,18,18,19,19,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,39,39,40,40,45,45,51,51,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,-1,26,28,30,32,-1,-1,-1,-1,-1,34,36,38,40,42,44,-1,46,48,50,52,-1,-1,-1,54,56,-1,-1,-1,-1,58,-1,-1,-1,-1,-1,60,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,3.3286803E11,4.083812E7,1.3582685E7,2.421233E0,1.4781E4,1.0083192E6,1.68128E8,4.6762675E6,2.2661412E0,3.2875034E5,1.4624119E-2,5.3251836E-2,3.1844E4,1.5738506E-2,-9.36034E-3,6.50596E5,2.0218372E0,2.2448614E0,3.1236285E2,-3.2442775E-2,1.5003176E-2,-7.982653E-3,6.3715116E-3,-1.0132903E-2,3.068497E6,1.7079E4,1E0,4.1108723E8,9.5E2,3E0,-2.4529338E-2,1.1668722E6,2.73E2,2.1848371E8,2.4507043E-1,-4.814778E-3,-4.013862E-3,5.2241413E-3,7.938338E0,8.3268556E5,1.7306963E-2,5.9767948E-3,-4.8438283E-3,1.9276767E-3,1.4400111E10,5.184181E-3,1.810483E-2,7.6036714E-3,8.884383E-3,2.7609593E-3,2.098154E6,1.647585E-2,-1.9863326E-2,-7.714117E-3,7.552092E-2,-1.0121381E-2,-1.3088822E-2,-5.2446146E-3,-2.8490238E-3,7.352118E-3,-4.22471E-3,4.867322E-3],"split_indices":[2,40,54,12,62,9,37,7,60,62,41,0,0,10,0,0,37,67,65,61,0,0,0,0,0,12,9,23,7,0,8,0,37,0,7,67,0,0,0,66,41,0,0,0,0,5,0,0,0,0,0,37,0,0,0,47,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.72E2,3.3E1,1.29E2,4.3E1,1.1E1,2.2E1,5.7E1,7.2E1,3.1E1,1.2E1,5E0,6E0,1.6E1,6E0,7E0,5E1,5E1,2.2E1,2.3E1,8E0,7E0,5E0,8E0,8E0,2.5E1,2.5E1,1.2E1,3.8E1,1.2E1,1E1,8E0,1.5E1,1.5E1,1E1,1.7E1,8E0,5E0,7E0,1.9E1,1.9E1,6E0,6E0,5E0,5E0,1E1,5E0,1E1,5E0,5E0,5E0,1.2E1,5E0,1E1,9E0,1.1E1,8E0,5E0,5E0,7E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-7.843381E-3,-4.7946E-2,2.3302533E-1,-2.908308E-2,-2.9767297E-2,3.0474447E-2,1.1438445E-1,-6.140812E-3,-2.281981E-2,-1.3528261E-2,1.6779942E-2,9.857244E-2,-8.920039E-2,-1.0018477E-2,8.331514E-2,-9.57538E-3,1.3081773E-1,1.2176276E-2,-1.7716436E-1,1.511489E-3,6.381497E-3,1.9194622E-1,-7.105031E-2,-8.4962286E-2,1.8288103E-1,-9.129509E-2,-4.03717E-1,1.0372595E-1,2.8296322E-1,5.340047E-3,-1.5527858E-1,-1.36349425E-2,-3.5007507E-2,1.2337932E-1,1.3266259E-2,-2.3718682E-1,-1.7669631E-2,-2.362236E-2,-1.06118005E-2,1.0334458E-2,6.643822E-2,3.464968E-1,6.567092E-4,-1.32903205E-2,-1.6197735E-3,-9.3418464E-2,3.8719876E-3,9.767127E-3,6.665988E-4,-1.4615181E-2,-5.6327954E-3,6.3962536E-3,-6.633678E-2,6.274457E-3,-6.250293E-4,1.9292556E-2,9.469177E-3,3.4468833E-4,-7.276359E-3,-5.613682E-3,1.3122583E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,-1,-1,9,11,-1,13,-1,15,17,-1,19,-1,21,23,25,-1,-1,27,29,31,33,35,37,39,41,-1,43,-1,45,47,-1,49,51,-1,-1,-1,53,55,-1,-1,-1,57,-1,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0976171E0,1.9507002E0,1.3646291E0,1.8149949E0,0E0,0E0,7.366359E-1,1.5134495E0,0E0,3.322382E-1,0E0,7.5675684E-1,8.671541E-1,0E0,2.5662728E-2,0E0,8.7742996E-1,7.783106E-1,9.74746E-1,0E0,0E0,4.0424502E-1,2.7512938E-1,2.8094846E-1,5.605811E-2,4.1472155E-1,1.029315E-1,1.0241279E-1,4.4057083E-1,0E0,1.6191575E-1,0E0,1.741948E-1,9.793757E-2,0E0,6.354064E-2,2.0384526E-1,0E0,0E0,0E0,1.1448187E-1,7.816076E-2,0E0,0E0,0E0,9.574595E-2,0E0,0E0,0E0,0E0,0E0,0E0,9.800845E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,9,9,11,11,12,12,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,32,32,33,33,35,35,36,36,40,40,41,41,45,45,52,52],"right_children":[2,4,6,8,-1,-1,10,12,-1,14,-1,16,18,-1,20,-1,22,24,26,-1,-1,28,30,32,34,36,38,40,42,-1,44,-1,46,48,-1,50,52,-1,-1,-1,54,56,-1,-1,-1,58,-1,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.3575526E8,1.79838E5,4.476758E6,-2.9767297E-2,3.0474447E-2,2.7546022E5,1.411682E7,-2.281981E-2,6.1377846E-2,1.6779942E-2,9.23253E1,1.2007376E0,-1.0018477E-2,9.772E3,-9.57538E-3,4.767898E0,2.9789917E2,2.5086E4,1.511489E-3,6.381497E-3,1.002E3,2.46604E5,7.99E2,1.6105E4,1.14296764E-1,5.5157606E-2,6.067944E1,6.0004652E7,5.340047E-3,2.2652755E8,-1.36349425E-2,1.9759825E0,1.8793418E0,1.3266259E-2,1.996E3,1.7284313E0,-2.362236E-2,-1.06118005E-2,1.0334458E-2,1.5773139E5,2.202021E8,6.567092E-4,-1.32903205E-2,-1.6197735E-3,5.1264524E-1,3.8719876E-3,9.767127E-3,6.665988E-4,-1.4615181E-2,-5.6327954E-3,6.3962536E-3,1.662E4,6.274457E-3,-6.250293E-4,1.9292556E-2,9.469177E-3,3.4468833E-4,-7.276359E-3,-5.613682E-3,1.3122583E-3],"split_indices":[2,54,38,37,0,0,42,12,0,47,0,4,65,0,2,0,65,4,9,0,0,2,38,2,9,51,47,61,54,0,7,0,62,62,0,2,62,0,0,0,42,7,0,0,0,65,0,0,0,0,0,0,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.85E2,3E1,1.8E2,5E0,6E0,2.4E1,1.72E2,8E0,1.6E1,8E0,7.6E1,9.6E1,5E0,1.1E1,7E0,6.9E1,4.5E1,5.1E1,6E0,5E0,5.3E1,1.6E1,2.9E1,1.6E1,3.8E1,1.3E1,2.8E1,2.5E1,5E0,1.1E1,5E0,2.4E1,1.1E1,5E0,1.2E1,2.6E1,8E0,5E0,6E0,2.2E1,2E1,5E0,5E0,6E0,1.6E1,8E0,6E0,5E0,7E0,5E0,6E0,2E1,1.2E1,1E1,1.4E1,6E0,6E0,1E1,1.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[2.2333339E-2,-2.6890216E-2,2.8227177E-1,9.072414E-2,-1.2188894E-1,5.555146E-1,-5.9846044E-2,2.7850151E-2,2.690379E-1,-4.2001417E-1,-8.032275E-2,4.5173913E-2,3.1427133E-1,-9.158853E-3,2.940707E-3,-7.638726E-2,1.1860877E-1,1.1013746E-3,3.446627E-1,-8.625991E-3,-3.0996991E-2,9.5595255E-2,-1.2018964E-1,2.1443896E-2,4.69162E-3,2.5419265E-2,-2.4529058E-1,2.156427E-1,1.45978555E-2,9.808546E-3,1.9154811E-2,1.7480741E-1,-3.8498736E-3,-1.8472705E-2,-7.804147E-2,-8.66632E-3,1.1048226E-1,-5.4051424E-3,-1.6576197E-2,5.705522E-3,1.2659281E-2,8.9809865E-2,-5.1655415E-3,1.1079771E-2,4.1627986E-3,1.2750423E-1,-1.3293555E-1,1.5940777E-3,1.0209773E-2,5.735698E-3,2.3124756E-3,1.0659285E-3,9.97404E-3,-3.0944794E-2,-2.032976E-1,-1.0355128E-1,3.2523882E-3,-7.671205E-4,-2.5971204E-1,-1.3181991E-3,-7.236441E-3,-1.9047558E-2,-9.332128E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,-1,29,-1,-1,31,33,-1,-1,35,37,39,41,-1,-1,43,-1,-1,45,-1,47,-1,-1,-1,-1,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,-1,-1,61,-1,-1,-1,-1],"loss_changes":[2.7329452E0,2.0222368E0,3.2116072E0,9.0062237E-1,1.2054563E0,1.3993726E0,2.5614208E-1,5.861472E-1,3.837148E-1,5.138843E-1,6.3567364E-1,0E0,3.1100273E-1,0E0,0E0,5.0490344E-1,3.299075E-1,0E0,3.0136228E-2,0E0,0E0,2.4855825E-1,7.634922E-1,0E0,0E0,3.4721863E-1,1.0153818E-1,4.26569E-2,1.6327594E-1,0E0,0E0,3.6488533E-2,0E0,0E0,7.3907745E-1,0E0,1.0045314E-1,0E0,0E0,0E0,0E0,7.32024E-3,0E0,0E0,0E0,1.0882157E-1,3.5969913E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6260372E-1,3.134308E-1,4.0304497E-2,0E0,0E0,1.3124287E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,18,18,21,21,22,22,25,25,26,26,27,27,28,28,31,31,34,34,36,36,41,41,45,45,46,46,53,53,54,54,55,55,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,-1,30,-1,-1,32,34,-1,-1,36,38,40,42,-1,-1,44,-1,-1,46,-1,48,-1,-1,-1,-1,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,-1,-1,62,-1,-1,-1,-1],"split_conditions":[3.973E3,2.5537605E8,6.627576E7,2.3082722E2,8.230099E4,1E0,1.99607E3,2.142857E0,4.7722706E-1,1E0,7.65244E5,4.5173913E-2,9.982716E0,-9.158853E-3,2.940707E-3,6.946372E6,5.696862E8,1.1013746E-3,8E0,-8.625991E-3,-3.0996991E-2,2.297E3,1.4893761E-2,2.1443896E-2,4.69162E-3,3.202E3,1.796E4,2.3301888E0,2.8367348E0,9.808546E-3,1.9154811E-2,3.2004956E5,-3.8498736E-3,-1.8472705E-2,1.3257E4,-8.66632E-3,1E0,-5.4051424E-3,-1.6576197E-2,5.705522E-3,1.2659281E-2,2.3103847E2,-5.1655415E-3,1.1079771E-2,4.1627986E-3,1.131E3,1E0,1.5940777E-3,1.0209773E-2,5.735698E-3,2.3124756E-3,1.0659285E-3,9.97404E-3,2.7709558E2,1.127E3,1.8508342E5,3.2523882E-3,-7.671205E-4,4E0,-1.3181991E-3,-7.236441E-3,-1.9047558E-2,-9.332128E-3],"split_indices":[2,7,54,61,41,18,4,62,36,13,57,0,66,0,0,54,5,0,3,0,0,2,47,0,0,38,9,62,65,0,0,56,0,0,9,0,18,0,0,0,0,64,0,0,0,2,18,0,0,0,0,0,0,4,2,41,0,0,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.79E2,3.3E1,8E1,9.9E1,1.8E1,1.5E1,6E1,2E1,1.1E1,8.8E1,6E0,1.2E1,7E0,8E0,2.8E1,3.2E1,5E0,1.5E1,6E0,5E0,1.6E1,7.2E1,7E0,5E0,1.8E1,1E1,1.6E1,1.6E1,5E0,1E1,1.1E1,5E0,9E0,6.3E1,5E0,1.3E1,5E0,5E0,6E0,1E1,1E1,6E0,6E0,5E0,1.3E1,5E1,8E0,5E0,5E0,5E0,6E0,7E0,2.1E1,2.9E1,1.2E1,9E0,7E0,2.2E1,5E0,7E0,6E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.3438757E-2,4.4017404E-2,-2.2162563E-1,-1.4114233E-3,3.7883043E-1,-4.6727613E-1,4.1331384E-2,-3.603107E-2,2.2277953E-1,4.6761267E-2,1.0413288E-1,-3.096246E-1,-4.8573475E-2,-1.7722752E-2,8.984172E-3,5.403454E-3,-2.2649883E-1,2.1278765E-3,1.5823092E-2,-1.068604E-3,9.842568E-3,-1.9629551E-2,-7.3279203E-3,5.8452287E-3,-1.1922521E-1,-1.6182797E-2,2.616063E-2,-3.2889965E-1,1.9511554E-3,-8.374827E-3,-2.8119842E-3,7.7747973E-3,1.4845692E-2,-8.172453E-3,-2.295927E-2,7.549869E-2,-8.215373E-2,2.6521182E-2,1.385505E-2,-1.6606769E-1,4.1145593E-2,-7.5606774E-3,2.625738E-3,-1.7722175E-5,-1.081631E-2,5.8932207E-3,-6.088042E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,35,-1,-1,-1,37,39,41,-1,43,45,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8439677E0,2.4189506E0,3.507471E0,1.1023862E0,2.9161832E0,2.016202E0,2.3041865E-1,9.719326E-1,3.2965457E-1,0E0,1.759062E-1,2.7759624E-1,0E0,2.9299486E-1,0E0,7.2105646E-1,6.133156E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8723031E-2,0E0,4.9263912E-1,2.8577518E-1,0E0,0E0,0E0,5.665111E-1,0E0,0E0,0E0,5.322839E-1,4.2069432E-1,2.2514264E-1,0E0,2.1950513E-1,2.3872748E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,15,15,16,16,24,24,26,26,27,27,31,31,35,35,36,36,37,37,39,39,40,40],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,36,-1,-1,-1,38,40,42,-1,44,46,-1,-1,-1,-1,-1,-1],"split_conditions":[2.136E4,5.666E3,2.6056657E0,1E0,3.0451374E9,1.1E1,1.01081E6,6.5023036E7,2.0676692E-1,4.6761267E-2,1E0,2.23226E-1,-4.8573475E-2,5.0797403E5,8.984172E-3,1.505656E6,6.633555E2,2.1278765E-3,1.5823092E-2,-1.068604E-3,9.842568E-3,-1.9629551E-2,-7.3279203E-3,5.8452287E-3,2.9656984E7,-1.6182797E-2,1.1206417E12,3.3658516E11,1.9511554E-3,-8.374827E-3,-2.8119842E-3,7.07E2,1.4845692E-2,-8.172453E-3,-2.295927E-2,1.496E3,3.4107404E2,4.5454547E-2,1.385505E-2,2.1548604E2,1.898E3,-7.5606774E-3,2.625738E-3,-1.7722175E-5,-1.081631E-2,5.8932207E-3,-6.088042E-3],"split_indices":[9,2,62,112,12,3,9,54,65,0,18,48,0,41,0,54,61,0,0,0,0,0,0,0,12,0,40,40,0,0,0,0,0,0,0,2,61,67,0,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.58E2,5.3E1,1.4E2,1.8E1,2.7E1,2.6E1,1.22E2,1.8E1,5E0,1.3E1,2.2E1,5E0,1.9E1,7E0,1.01E2,2.1E1,7E0,1.1E1,6E0,7E0,1.3E1,9E0,8E0,1.1E1,5E0,9.6E1,1.5E1,6E0,5E0,6E0,9.1E1,5E0,8E0,7E0,5.2E1,3.9E1,4.3E1,9E0,2.3E1,1.6E1,5E0,3.8E1,6E0,1.7E1,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-9.448418E-3,-6.510653E-2,3.2098758E-1,-2.0309906E-2,-3.276983E-1,7.5080323E-1,5.5252865E-2,-5.951824E-2,1.8474641E-1,-1.7416205E-1,-3.8364477E-2,4.9715124E-2,1.7566955E-2,1.9909689E-1,-6.1928947E-3,-4.0980116E-3,-2.8103524E-1,2.7006394E-1,6.131252E-4,4.070958E-4,-2.9651642E-1,1.3908617E-2,3.933146E-3,-1.3111224E-2,2.4656802E-2,-3.4524632E-1,-9.853789E-4,1.6798697E-2,6.1749923E-3,-7.85445E-3,-1.9823965E-2,4.9982402E-2,-1.6945444E-2,-1.9813832E-2,-6.717638E-3,8.96234E-2,-1.6311873E-1,1.356115E-2,6.5860204E-2,-1.1072545E-3,-1.1664673E-2,1.2513567E-3,6.4752614E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,23,25,27,-1,-1,29,-1,-1,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,37,39,-1,41,-1,-1,-1,-1],"loss_changes":[3.6991875E0,2.016619E0,3.2708073E0,1.2009034E0,1.6181145E0,7.809572E-1,5.224968E-1,1.5334136E0,3.492188E-1,4.4903982E-1,0E0,0E0,0E0,8.861029E-2,0E0,7.7372265E-1,4.118184E-1,1.2586367E-1,0E0,0E0,1.12570524E-1,0E0,0E0,0E0,8.825114E-1,1.9412589E-1,0E0,0E0,0E0,0E0,0E0,7.495553E-1,0E0,0E0,0E0,3.1199396E-1,1.4310622E-1,0E0,1.7628822E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,20,20,24,24,25,25,31,31,35,35,36,36,38,38],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,24,26,28,-1,-1,30,-1,-1,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,38,40,-1,42,-1,-1,-1,-1],"split_conditions":[4.796E3,6.5023036E7,5.939257E7,3.1036722E2,2.9119E4,1E0,6.084144E0,2.00472E5,2.9807162E0,8.3636254E-1,-3.8364477E-2,4.9715124E-2,1.7566955E-2,6.487E3,-6.1928947E-3,9.235376E5,2.195572E0,3.6E1,6.131252E-4,4.070958E-4,5.1160636E11,1.3908617E-2,3.933146E-3,-1.3111224E-2,4.476758E6,1.4638E4,-9.853789E-4,1.6798697E-2,6.1749923E-3,-7.85445E-3,-1.9823965E-2,4.356558E6,-1.6945444E-2,-1.9813832E-2,-6.717638E-3,6.8102196E-2,1.0686696E0,1.356115E-2,2.1935484E0,-1.1072545E-3,-1.1664673E-2,1.2513567E-3,6.4752614E-3],"split_indices":[2,54,54,61,9,18,65,38,65,36,0,0,0,2,0,54,63,8,0,0,40,0,0,0,37,10,0,0,0,0,0,59,0,0,0,36,65,0,62,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.99E2,1.71E2,2.8E1,1.47E2,2.4E1,1E1,1.8E1,1.24E2,2.3E1,1.9E1,5E0,5E0,5E0,1E1,8E0,1E2,2.4E1,1.5E1,8E0,8E0,1.1E1,5E0,5E0,9E0,9.1E1,1.9E1,5E0,9E0,6E0,6E0,5E0,8.6E1,5E0,1.4E1,5E0,7.3E1,1.3E1,7E0,6.6E1,5E0,8E0,4.2E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.0720788E-4,5.442093E-2,-2.6575488E-1,2.9720576E-2,2.768204E-2,-1.3303486E-1,-3.4883052E-2,-2.2515777E-1,5.4353397E-2,-2.1797292E-1,5.7274825E-3,-1.9018833E-2,4.183735E-3,1.3023104E-1,-2.943636E-2,1.0679181E-3,-2.8979358E-1,7.4276894E-2,2.3349148E-1,-9.485231E-2,1.9547904E-1,-1.7471826E-2,-9.284533E-3,1.16838105E-1,-3.9190636E-3,1.8854337E-2,1.2748727E-1,-3.5287734E-2,-2.3643231E-1,1.2246566E-2,2.5794513E-3,1.4629696E-1,-2.4405443E-3,-7.4999467E-3,7.111917E-2,2.0257664E-1,-2.7594867E-3,-1.5028246E-1,2.1443775E-2,-4.9709696E-3,-2.8409714E-1,-1.4787755E-3,1.8281712E-1,6.409314E-3,-1.50258E-3,1.3820925E-2,4.5684325E-3,-9.540195E-3,-2.8299494E-3,9.243767E-3,-2.1366559E-2,-1.8435188E-2,-8.004164E-3,1.152909E-2,3.0453696E-3,-2.7916026E-3,1.6808464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,-1,-1,17,19,-1,21,23,25,27,29,-1,-1,31,33,-1,35,37,39,-1,-1,41,-1,-1,43,45,-1,47,49,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0608013E0,2.1959083E0,2.0450468E0,1.0670047E0,0E0,6.4185673E-1,0E0,7.9743946E-1,9.8292994E-1,3.8526082E-1,0E0,0E0,0E0,4.5066285E-1,1.1073078E0,0E0,4.6934724E-2,1.8017608E-1,4.02676E-1,4.8077965E-1,1.203661E-1,0E0,0E0,1.7925164E-1,2.3530145E-1,0E0,2.6005325E-1,2.7722323E-1,8.028424E-2,0E0,0E0,1.9895983E-1,0E0,0E0,8.959836E-2,8.73009E-2,0E0,4.6765506E-2,2.1219964E-1,0E0,7.262564E-2,0E0,1.4513004E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.9793486E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,13,13,14,14,16,16,17,17,18,18,19,19,20,20,23,23,24,24,26,26,27,27,28,28,31,31,34,34,35,35,37,37,38,38,40,40,42,42,50,50],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,-1,-1,18,20,-1,22,24,26,28,30,-1,-1,32,34,-1,36,38,40,-1,-1,42,-1,-1,44,46,-1,48,50,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,1.0086E4,1.0104842E12,3.9894668E4,2.768204E-2,4.6328125E0,-3.4883052E-2,1.119E3,7.42E2,2.4589229E8,5.7274825E-3,-1.9018833E-2,4.183735E-3,1.436E3,1E0,1.0679181E-3,1.2114334E2,2.6246939E8,3.1665432E0,4.2719266E5,3.0513447E5,-1.7471826E-2,-9.284533E-3,1.815E4,9.06E2,1.8854337E-2,1.586E3,1.5839127E-1,1.9303116E5,1.2246566E-2,2.5794513E-3,4.39E2,-2.4405443E-3,-7.4999467E-3,4.9332215E6,1.6773E4,-2.7594867E-3,9.001196E0,1.6324973E0,-4.9709696E-3,2.0700347E8,-1.4787755E-3,1.2557897E7,6.409314E-3,-1.50258E-3,1.3820925E-2,4.5684325E-3,-9.540195E-3,-2.8299494E-3,9.243767E-3,1.086593E6,-1.8435188E-2,-8.004164E-3,1.152909E-2,3.0453696E-3,-2.7916026E-3,1.6808464E-3],"split_indices":[9,2,40,42,0,62,0,0,0,7,0,0,0,2,24,0,61,7,65,42,42,0,0,9,2,0,10,51,41,0,0,2,0,0,54,9,0,66,63,0,12,0,54,0,0,0,0,0,0,0,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.74E2,3.5E1,1.67E2,7E0,2.8E1,7E0,1.4E1,1.53E2,2.1E1,7E0,9E0,5E0,8E1,7.3E1,5E0,1.6E1,5.3E1,2.7E1,5.7E1,1.6E1,8E0,8E0,3.4E1,1.9E1,1E1,1.7E1,4.1E1,1.6E1,1.1E1,5E0,2.9E1,5E0,6E0,1.3E1,1.2E1,5E0,1.3E1,2.8E1,5E0,1.1E1,5E0,2.4E1,8E0,5E0,6E0,6E0,8E0,5E0,5E0,2.3E1,5E0,6E0,1.6E1,8E0,1.4E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[5.668402E-3,-4.6778165E-2,2.947474E-1,-1.8701993E-2,-2.856486E-2,5.476071E-1,1.0300823E-1,8.6863875E-2,-8.8901125E-2,4.1537076E-2,1.0497445E-2,3.187859E-2,9.1113625E-3,-8.956639E-3,1.2467104E-1,-4.8530642E-2,-3.2788864E-1,5.2645854E-3,-1.8412538E-3,1.5726016E-1,-5.619405E-3,9.9382214E-2,-9.4324246E-2,-1.8926926E-2,-8.523176E-3,-2.3944778E-3,1.9867776E-1,-3.60336E-2,2.1523154E-1,-2.3521149E-1,-5.490223E-2,2.2953694E-1,1.1440169E-3,2.3038783E-3,-5.5329744E-3,5.948589E-3,1.2936409E-2,-1.4598335E-2,-4.412328E-3,6.519827E-2,-9.648807E-2,1.5801202E-1,3.6043203E-1,-1.5676728E-3,5.7209483E-3,-6.0966298E-2,-1.0751688E-2,4.881416E-3,1.3863742E-2,2.1757036E-2,9.162829E-3,-6.4190333E-3,8.3654927E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,-1,19,21,23,-1,-1,25,-1,27,29,-1,-1,-1,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,43,45,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2582848E0,2.7361765E0,1.5435348E0,1.2971724E0,0E0,1.2255669E0,1.1269368E-1,7.376667E-1,9.9713165E-1,0E0,0E0,6.7338705E-2,0E0,0E0,5.0226617E-1,6.257593E-1,6.9090486E-2,0E0,0E0,4.7955644E-1,0E0,3.528878E-1,3.7587053E-1,0E0,0E0,0E0,2.4343204E-1,7.592514E-2,2.3266435E-2,1.1097872E-1,2.8598177E-1,3.0817533E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.9394184E-2,1.7080018E-1,1.6888773E-1,1.0697806E-1,0E0,0E0,1.890239E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,14,14,15,15,16,16,19,19,21,21,22,22,26,26,27,27,28,28,29,29,30,30,31,31,39,39,40,40,41,41,42,42,45,45],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,-1,20,22,24,-1,-1,26,-1,28,30,-1,-1,-1,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,44,46,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,1.00068504E8,5.25191E5,7.701429E5,-2.856486E-2,1E0,1E0,3.814714E-2,5.058997E0,4.1537076E-2,1.0497445E-2,1.183E3,9.1113625E-3,-8.956639E-3,6.0095956E7,1.3252E4,2.2827177E0,5.2645854E-3,-1.8412538E-3,1.2960748E2,-5.619405E-3,1.2262E4,2.897327E6,-1.8926926E-2,-8.523176E-3,-2.3944778E-3,3.203272E8,2.649435E5,2.6006784E2,1.7325E4,6.8915665E-1,1.518E3,1.1440169E-3,2.3038783E-3,-5.5329744E-3,5.948589E-3,1.2936409E-2,-1.4598335E-2,-4.412328E-3,1.5756359E0,8.563462E-1,2.3082722E2,1.1E1,-1.5676728E-3,5.7209483E-3,3.7721686E2,-1.0751688E-2,4.881416E-3,1.3863742E-2,2.1757036E-2,9.162829E-3,-6.4190333E-3,8.3654927E-4],"split_indices":[61,54,38,57,0,13,18,67,65,0,0,0,0,0,54,9,62,0,0,64,0,9,54,0,0,0,7,42,4,9,65,2,0,0,0,0,0,0,0,63,36,61,8,0,0,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.81E2,3.2E1,1.73E2,8E0,1.3E1,1.9E1,6.9E1,1.04E2,6E0,7E0,1.1E1,8E0,8E0,6.1E1,9E1,1.4E1,5E0,6E0,5.4E1,7E0,2.1E1,6.9E1,9E0,5E0,9E0,4.5E1,1E1,1.1E1,1.4E1,5.5E1,3.8E1,7E0,5E0,5E0,5E0,6E0,9E0,5E0,1.4E1,4.1E1,2.6E1,1.2E1,5E0,9E0,3.3E1,8E0,1.9E1,7E0,7E0,5E0,1.7E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[2.466624E-3,-4.1586377E-2,2.263342E-1,-2.1589363E-2,-2.6458016E-2,3.2982565E-2,7.4112356E-2,-4.612117E-3,-1.6520755E-2,-8.745273E-2,1.6872922E-1,8.277429E-2,-8.2237676E-2,2.5782771E-3,-1.041522E-2,7.2236076E-2,1.515487E-2,-1.5497446E-2,1.3312966E-1,8.92089E-2,-1.4955112E-1,5.7418966E-3,1.1313538E-3,-8.145772E-3,2.4606962E-2,1.7324635E-1,-3.579284E-2,-8.512654E-3,1.7692544E-1,-2.0666607E-1,5.460979E-2,-2.4718557E-2,4.45796E-3,2.3523606E-1,9.710302E-2,-6.2692985E-3,3.0617784E-3,9.5129263E-4,2.802794E-1,-1.3206959E-1,-3.253512E-1,-4.491205E-3,7.942044E-3,1.6642589E-3,-5.315626E-3,5.257021E-3,1.5894998E-2,1.6894442E-1,5.782703E-5,1.7130604E-2,8.802473E-3,-1.99459E-1,-2.6365465E-2,-1.1203785E-2,-1.9879889E-2,1.1231079E-2,3.4131065E-3,-1.3742553E-2,-5.2332627E-3,-6.4275023E-3,3.368759E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,-1,-1,9,11,-1,13,15,17,19,-1,-1,21,-1,23,25,27,29,-1,-1,-1,31,33,35,-1,37,39,41,43,-1,45,47,-1,-1,-1,49,51,53,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1593368E0,1.8169353E0,2.4073863E0,9.492027E-1,0E0,0E0,4.4278574E-1,1.1532049E0,0E0,2.0497653E-1,2.2304636E-1,3.9794397E-1,1.0520006E0,0E0,0E0,2.3927279E-2,0E0,1.738313E-1,3.6536145E-1,6.3144517E-1,7.6922774E-1,0E0,0E0,0E0,7.780557E-2,1.8012226E-1,1.0815104E-1,0E0,3.2003516E-1,4.0855908E-1,2.5196114E-1,7.283636E-2,0E0,2.2918391E-1,1.445117E-1,0E0,0E0,0E0,2.1272182E-2,2.2981131E-1,5.539131E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.6300938E-2,0E0,0E0,0E0,1.2085426E-1,1.4930372E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,38,38,39,39,40,40,47,47,51,51,52,52],"right_children":[2,4,6,8,-1,-1,10,12,-1,14,16,18,20,-1,-1,22,-1,24,26,28,30,-1,-1,-1,32,34,36,-1,38,40,42,44,-1,46,48,-1,-1,-1,50,52,54,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,1.3575526E8,1.5190906E9,4.476758E6,-2.6458016E-2,3.2982565E-2,9.1481786E8,3.1840762E7,-1.6520755E-2,5.6115907E9,2.6014878E5,2.0787193E2,1.3255E4,2.5782771E-3,-1.041522E-2,7.365962E5,1.515487E-2,1.635904E8,6.0095956E7,3.3712113E-1,2.6600266E0,5.7418966E-3,1.1313538E-3,-8.145772E-3,7.53E2,2.1526105E0,5.34075E5,-8.512654E-3,6.271094E-1,1.9577E4,1.2106E4,1.4210526E0,4.45796E-3,8.47E2,3.74E2,-6.2692985E-3,3.0617784E-3,9.5129263E-4,4.4592E4,1.6371E4,2.67302E5,-4.491205E-3,7.942044E-3,1.6642589E-3,-5.315626E-3,5.257021E-3,1.5894998E-2,2.2E2,5.782703E-5,1.7130604E-2,8.802473E-3,7.938338E0,1.1532972E-1,-1.1203785E-2,-1.9879889E-2,1.1231079E-2,3.4131065E-3,-1.3742553E-2,-5.2332627E-3,-6.4275023E-3,3.368759E-3],"split_indices":[2,54,12,37,0,0,7,12,0,12,42,4,9,0,0,41,0,7,54,48,62,0,0,0,2,65,1,0,65,9,11,67,0,2,0,0,0,0,38,9,38,0,0,0,0,0,0,10,0,0,0,66,48,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.82E2,3.5E1,1.76E2,6E0,8E0,2.7E1,1.68E2,8E0,1E1,1.7E1,7.9E1,8.9E1,5E0,5E0,1.1E1,6E0,2.7E1,5.2E1,2.5E1,6.4E1,5E0,6E0,5E0,2.2E1,4.2E1,1E1,6E0,1.9E1,5E1,1.4E1,1.3E1,9E0,2.2E1,2E1,5E0,5E0,8E0,1.1E1,3.2E1,1.8E1,6E0,8E0,8E0,5E0,1E1,1.2E1,1.1E1,9E0,5E0,6E0,1.9E1,1.3E1,1E1,8E0,6E0,5E0,9E0,1E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.0979192E-2,6.8404065E-3,-2.5719672E-2,6.0118455E-2,-1.712765E-1,2.5332047E-2,2.7242908E-1,-3.641021E-1,1.2974103E-2,1.0007102E-1,-6.2858686E-2,3.8156304E-1,3.5386833E-3,-2.6340553E-1,-2.959728E-2,7.53846E-2,-6.309014E-3,1.280752E-1,-2.9095152E-2,-1.4219402E-1,4.8188202E-2,1.0266353E-2,2.5568685E-2,-1.4652732E-2,-7.9889E-3,-1.0813405E-3,6.81611E-3,5.9516E-2,2.7649468E-1,4.3707034E-3,-7.839436E-3,2.398405E-3,-1.9664767E-1,9.298703E-2,-4.550359E-3,1.0049099E-2,2.54026E-2,1.6484328E-2,9.194327E-3,2.83204E-3,-2.521006E-1,-1.3186311E-3,1.6888611E-1,-4.675063E-3,5.7918683E-2,-3.105041E-1,-5.8783614E-3,1.0629987E-2,3.712876E-3,4.0603373E-3,-2.5578479E-3,-7.745084E-3,-1.8862385E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,-1,23,-1,25,-1,27,29,31,33,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,39,41,-1,-1,43,-1,-1,-1,45,-1,47,-1,49,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9223561E0,1.9367102E0,0E0,1.1526262E0,1.6745505E0,9.02705E-1,4.5264745E-1,3.9960742E-1,2.3135805E-1,2.7139866E-1,5.618784E-1,2.3000383E-1,0E0,1.6438484E-2,0E0,1.1551985E-1,0E0,6.04251E-1,2.3216613E-1,3.9499897E-1,1.7954484E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.090454E-1,3.858757E-2,0E0,0E0,0E0,4.2401528E-1,1.9576417E-1,0E0,0E0,1.472336E-1,0E0,0E0,0E0,1.4745617E-1,0E0,4.217404E-2,0E0,8.475172E-2,1.2226069E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,17,17,18,18,19,19,20,20,27,27,28,28,32,32,33,33,36,36,40,40,42,42,44,44,45,45],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,-1,24,-1,26,-1,28,30,32,34,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,40,42,-1,-1,44,-1,-1,-1,46,-1,48,-1,50,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,2.136E4,-2.5719672E-2,4.796E3,2.225675E0,2.747292E8,4.948817E2,7.18771E7,1.1032838E6,2.46604E5,9.479827E0,2.8488008E11,3.5386833E-3,4.1698615E2,-2.959728E-2,3.5753E4,-6.309014E-3,4.0354648E2,6.0095956E7,2.0519513E-1,2.1934114E0,1.0266353E-2,2.5568685E-2,-1.4652732E-2,-7.9889E-3,-1.0813405E-3,6.81611E-3,2.585911E5,1.6506441E9,4.3707034E-3,-7.839436E-3,2.398405E-3,2.76E2,1.062E3,-4.550359E-3,1.0049099E-2,1.1192772E8,1.6484328E-2,9.194327E-3,2.83204E-3,9.933566E5,-1.3186311E-3,6.425227E5,-4.675063E-3,6.96E2,1.8976982E0,-5.8783614E-3,1.0629987E-2,3.712876E-3,4.0603373E-3,-2.5578479E-3,-7.745084E-3,-1.8862385E-2],"split_indices":[37,9,0,2,63,7,67,54,41,38,66,40,0,4,0,9,0,4,54,36,65,0,0,0,0,0,0,37,5,0,0,0,0,2,0,0,5,0,0,0,37,0,41,0,0,62,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,2.02E2,6E0,1.56E2,4.6E1,1.35E2,2.1E1,2.2E1,2.4E1,7.3E1,6.2E1,1.3E1,8E0,1.7E1,5E0,1.7E1,7E0,6E1,1.3E1,3.6E1,2.6E1,7E0,6E0,1.1E1,6E0,7E0,1E1,4.2E1,1.8E1,7E0,6E0,8E0,2.8E1,2E1,6E0,7E0,3.5E1,9E0,9E0,5E0,2.3E1,8E0,1.2E1,7E0,2.8E1,1.5E1,8E0,7E0,5E0,2.3E1,5E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.5833683E-2,-1.9138376E-3,3.2361686E-1,5.8927342E-2,-7.011037E-2,2.8021274E-2,3.09905E-3,-1.1600979E-2,1.665864E-1,-1.0442029E-1,1.01526886E-1,1.4989709E-2,-1.41199725E-2,2.3624092E-1,1.61716E-2,8.065262E-3,-1.2380629E-1,-4.0319E-3,1.8507254E-1,-1.4068042E-1,5.4376412E-2,2.5879904E-3,3.004871E-1,-6.237192E-3,7.719321E-3,-9.16857E-2,-2.526004E-1,5.0410973E-3,1.1187104E-2,-1.2251716E-2,1.4366054E-3,9.687537E-2,-7.730327E-2,3.8531065E-1,9.66318E-3,-2.104138E-1,-6.307395E-2,-1.857508E-2,-4.5757815E-3,7.280481E-2,1.0613299E-2,1.2156348E-3,-9.80779E-3,2.354229E-2,1.0986732E-2,-4.7945655E-3,-1.3521873E-2,-2.8348582E-2,-1.7935655E-1,1.0946776E-1,-1.2773861E-3,5.2860193E-3,-9.642901E-3,-1.1792304E-2,-3.7762357E-3,7.581083E-3,2.302887E-3,-2.0025584E-3,6.33708E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,21,23,-1,25,-1,27,29,31,-1,33,-1,-1,35,37,-1,-1,-1,-1,39,41,43,-1,45,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2023449E0,8.7130547E-1,7.4320555E-1,8.44757E-1,5.935986E-1,0E0,0E0,5.056781E-1,4.571706E-1,4.539371E-1,2.7661422E-1,3.9392364E-1,0E0,3.37371E-1,3.258543E-1,0E0,2.995218E-1,0E0,1.946026E-2,2.5049737E-1,2.928975E-1,0E0,1.156559E-1,0E0,0E0,2.0413762E-1,2.6828283E-1,0E0,0E0,0E0,0E0,9.708059E-2,1.6882537E-1,6.1846018E-2,0E0,6.2123537E-2,2.0892717E-1,0E0,0E0,1.2619081E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.3967971E-1,5.719477E-2,6.382075E-2,0E0,2.1239151E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,18,18,19,19,20,20,22,22,25,25,26,26,31,31,32,32,33,33,35,35,36,36,39,39,47,47,48,48,49,49,51,51],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,22,24,-1,26,-1,28,30,32,-1,34,-1,-1,36,38,-1,-1,-1,-1,40,42,44,-1,46,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3717E4,7.11E2,6.234292E-1,1.135E3,1.8656379E6,2.8021274E-2,3.09905E-3,3.26989E6,1.6507992E9,2.0505953E5,1.351E3,2.0762905E6,-1.41199725E-2,2.3873117E2,4.83E9,8.065262E-3,5.591566E0,-4.0319E-3,1.88556E5,5.64E2,2.1740167E0,2.5879904E-3,2.202021E8,-6.237192E-3,7.719321E-3,7.639199E1,2.38502E0,5.0410973E-3,1.1187104E-2,-1.2251716E-2,1.4366054E-3,1.64935E5,1.8029E4,2.1941175E0,9.66318E-3,1.6999166E5,6.627576E7,-1.857508E-2,-4.5757815E-3,1.4963856E0,1.0613299E-2,1.2156348E-3,-9.80779E-3,2.354229E-2,1.0986732E-2,-4.7945655E-3,-1.3521873E-2,4.31759E5,2.0983605E0,1.2776401E7,-1.2773861E-3,4.0868604E2,-9.642901E-3,-1.1792304E-2,-3.7762357E-3,7.581083E-3,2.302887E-3,-2.0025584E-3,6.33708E-3],"split_indices":[2,0,36,2,37,0,0,37,5,37,0,54,0,4,5,0,65,0,38,2,65,0,7,0,0,61,62,0,0,0,0,1,9,62,0,56,54,0,0,66,0,0,0,0,0,0,0,11,65,54,0,61,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.19E2,2.08E2,1.1E1,1.1E2,9.8E1,5E0,6E0,6.7E1,4.3E1,8.2E1,1.6E1,6.2E1,5E0,2.9E1,1.4E1,5E0,7.7E1,5E0,1.1E1,1.2E1,5E1,8E0,2.1E1,7E0,7E0,6.3E1,1.4E1,5E0,6E0,7E0,5E0,3.8E1,1.2E1,1E1,1.1E1,1.1E1,5.2E1,7E0,7E0,3.3E1,5E0,7E0,5E0,5E0,5E0,5E0,6E0,4.1E1,1.1E1,2.4E1,9E0,3.5E1,6E0,6E0,5E0,1.3E1,1.1E1,2.6E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-8.43606E-3,3.71997E-2,-3.0560115E-1,-1.4044805E-2,2.596848E-1,-4.023793E-2,-1.5054767E-1,2.0399839E-2,-1.8711802E-1,3.1431075E-2,1.6826618E-1,5.0197304E-3,-2.3107162E-1,8.96792E-2,-3.8691733E-2,-2.595279E-1,-4.8182358E-4,5.6596965E-2,3.1244203E-1,-1.5302841E-2,-3.3829014E-3,-7.4457545E-3,1.392618E-1,1.13985434E-1,-7.829152E-2,-8.345302E-3,-1.6525576E-2,-5.169662E-5,5.2772397E-3,7.585373E-3,1.967915E-2,3.1599782E-2,1.6750559E-1,1.9908005E-3,7.3397583E-3,-1.5973382E-1,-1.53410565E-2,5.573878E-3,-2.7421121E-3,2.1997245E-1,8.4894575E-2,-2.8131062E-3,-2.0084177E-1,4.112116E-2,-1.2173791E-1,4.575672E-3,2.5163135E-1,1.5097127E-3,6.6008796E-3,-5.5176835E-3,-1.455682E-2,-2.102875E-3,9.621033E-2,-1.0006194E-2,-9.0316223E-4,8.553877E-3,1.5444642E-2,1.8905154E-3,6.224111E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,19,21,23,25,-1,27,29,-1,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,43,-1,-1,45,47,-1,49,51,53,-1,55,-1,-1,-1,-1,-1,57,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7507617E0,2.0102477E0,2.0429559E0,8.622652E-1,1.048265E0,0E0,4.7235575E-1,4.9882278E-1,3.054921E-1,0E0,4.331169E-1,0E0,2.0396388E-1,6.9317406E-1,4.0795898E-1,6.086433E-2,0E0,5.045947E-2,1.07661724E-1,0E0,0E0,0E0,1.3762379E-1,3.0983418E-2,2.692715E-1,0E0,0E0,0E0,0E0,0E0,0E0,8.591779E-2,1.416961E-1,0E0,0E0,8.5053265E-2,1.9123547E-1,0E0,0E0,6.445551E-2,3.9163575E-2,0E0,1.0284728E-1,1.0148096E-1,9.0461954E-2,0E0,2.8525352E-2,0E0,0E0,0E0,0E0,0E0,1.8353976E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,17,17,18,18,22,22,23,23,24,24,31,31,32,32,35,35,36,36,39,39,40,40,42,42,43,43,44,44,46,46,52,52],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,20,22,24,26,-1,28,30,-1,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,44,-1,-1,46,48,-1,50,52,54,-1,56,-1,-1,-1,-1,-1,58,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,7.015278E2,1.4893761E-2,6.1061732E7,1.6620063E7,-4.023793E-2,1.09E3,8.2147594E5,2.4192488E0,3.1431075E-2,1.4062E4,5.0197304E-3,1.12160355E-1,4.305949E-2,1E0,5.506776E11,-4.8182358E-4,2.1908425E6,1E0,-1.5302841E-2,-3.3829014E-3,-7.4457545E-3,1.4738E4,3.0698562E5,1.8638788E6,-8.345302E-3,-1.6525576E-2,-5.169662E-5,5.2772397E-3,7.585373E-3,1.967915E-2,1.3683E4,1.7315E4,1.9908005E-3,7.3397583E-3,1.3542666E6,1.5E1,5.573878E-3,-2.7421121E-3,7.52E2,1E0,-2.8131062E-3,5.2015275E-1,2.991523E8,5.5157606E-2,4.575672E-3,7.106593E4,1.5097127E-3,6.6008796E-3,-5.5176835E-3,-1.455682E-2,-2.102875E-3,3.8796097E5,-1.0006194E-2,-9.0316223E-4,8.553877E-3,1.5444642E-2,1.8905154E-3,6.224111E-3],"split_indices":[9,61,47,54,54,0,2,57,62,0,9,0,47,67,25,40,0,41,8,0,0,0,9,41,60,0,0,0,0,0,0,9,9,0,0,59,8,0,0,2,8,0,48,7,47,0,56,0,0,0,0,0,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.01E2,1.75E2,2.6E1,1.43E2,3.2E1,5E0,2.1E1,1.2E2,2.3E1,5E0,2.7E1,5E0,1.6E1,5.5E1,6.5E1,1.6E1,7E0,1.6E1,1.1E1,1E1,6E0,9E0,4.6E1,1.3E1,5.2E1,9E0,7E0,8E0,8E0,5E0,6E0,1E1,3.6E1,5E0,8E0,2.2E1,3E1,5E0,5E0,2.1E1,1.5E1,7E0,1.5E1,2E1,1E1,5E0,1.6E1,8E0,7E0,9E0,6E0,8E0,1.2E1,5E0,5E0,9E0,7E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-6.0576308E-3,-3.6307327E-2,1.6939083E-1,1.0613647E-3,-2.61892E-1,2.6215884E-1,-4.841128E-2,-6.5833986E-2,8.0807604E-2,-4.113608E-1,-5.7784818E-2,3.2964116E-1,2.5017625E-3,-8.413544E-3,4.0752413E-3,-4.6553023E-2,-1.6504316E-2,2.062071E-1,-3.1587377E-2,-5.2295006E-3,-2.6394594E-2,1.678E-3,-8.357139E-3,9.209611E-3,2.1992868E-2,3.5924714E-2,-1.3774113E-1,2.7868965E-1,5.095548E-2,1.1078301E-1,-1.4298113E-1,6.0408857E-2,-6.914172E-3,2.233242E-3,-1.891518E-1,9.631673E-3,3.6468512E-1,6.4078798E-3,-1.7822799E-3,-5.3791597E-2,1.4645474E-2,-2.2203243E-1,6.570129E-4,8.92632E-2,-2.7034858E-3,-2.850004E-1,-8.217662E-2,2.21708E-2,1.0509898E-2,2.5790872E-3,-7.3995395E-3,-2.8229058E-1,-3.644194E-3,4.4835012E-2,1.5758298E-1,-1.9207742E-2,-1.9039197E-1,1.1422348E-3,-1.2810093E-1,-1.7803779E-2,-7.4932585E-3,6.5725977E-3,-2.911288E-4,9.496388E-3,4.027263E-3,-4.283944E-3,-1.277774E-2,-9.018813E-3,-3.0037311E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,-1,27,29,-1,-1,-1,-1,-1,-1,31,33,35,37,39,41,43,-1,-1,45,-1,47,-1,-1,49,-1,51,-1,53,-1,55,57,-1,-1,-1,-1,59,-1,61,63,-1,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2268207E0,1.6609492E0,6.9966376E-1,9.122108E-1,8.30444E-1,3.2482207E-1,1.9369008E-1,4.716813E-1,1.1058669E0,5.686364E-1,1.4140892E-1,2.2102821E-1,0E0,0E0,0E0,6.667954E-1,0E0,4.021616E-1,6.8192506E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.1266174E-1,4.0471143E-1,1.0366213E-1,9.5851794E-2,6.065964E-1,2.99754E-1,1.4517294E-1,0E0,0E0,3.1494248E-1,0E0,4.8822522E-2,0E0,0E0,1.2237322E-1,0E0,1.21142805E-1,0E0,9.6135765E-2,0E0,9.8814726E-2,8.5393645E-2,0E0,0E0,0E0,0E0,6.0433447E-2,0E0,1.0095218E-1,1.9863844E-2,0E0,5.73577E-2,0E0,3.377682E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,17,17,18,18,25,25,26,26,27,27,28,28,29,29,30,30,31,31,34,34,36,36,39,39,41,41,43,43,45,45,46,46,51,51,53,53,54,54,56,56,58,58],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,-1,28,30,-1,-1,-1,-1,-1,-1,32,34,36,38,40,42,44,-1,-1,46,-1,48,-1,-1,50,-1,52,-1,54,-1,56,58,-1,-1,-1,-1,60,-1,62,64,-1,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.723729E2,6.0095956E7,5.2587685E1,3.0954462E2,2.3641207E0,1.6E1,2.6732678E3,4.476758E6,2.1934114E0,7.7573473E9,3.338528E-3,1.4762E4,2.5017625E-3,-8.413544E-3,4.0752413E-3,2.4837341E0,-1.6504316E-2,7.685243E6,9.0996725E5,-5.2295006E-3,-2.6394594E-2,1.678E-3,-8.357139E-3,9.209611E-3,2.1992868E-2,1.2141942E6,1.3257E4,4.6253732E2,2.841527E5,1.4285715E-1,3.0147552E2,6.3236547E10,-6.914172E-3,2.233242E-3,6.124014E5,9.631673E-3,1.6506441E9,6.4078798E-3,-1.7822799E-3,3.8491228E0,1.4645474E-2,1.4638E4,6.570129E-4,2.1343284E0,-2.7034858E-3,1.3131311E6,4.2809364E-1,2.21708E-2,1.0509898E-2,2.5790872E-3,-7.3995395E-3,3.3897146E8,-3.644194E-3,4.795446E-1,8.7678794E5,-1.9207742E-2,1.6634E4,1.1422348E-3,1.5661E4,-1.7803779E-2,-7.4932585E-3,6.5725977E-3,-2.911288E-4,9.496388E-3,4.027263E-3,-4.283944E-3,-1.277774E-2,-9.018813E-3,-3.0037311E-3],"split_indices":[61,54,66,4,62,3,4,37,65,5,48,10,0,0,0,67,0,57,59,0,0,0,0,0,0,60,9,4,42,66,61,40,0,0,37,0,5,0,0,65,0,10,0,63,0,60,65,0,0,0,0,7,0,36,59,0,9,0,9,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.29E2,1.96E2,3.3E1,1.69E2,2.7E1,2.3E1,1E1,9.2E1,7.7E1,1.5E1,1.2E1,1.7E1,6E0,5E0,5E0,8.7E1,5E0,3.6E1,4.1E1,5E0,1E1,7E0,5E0,9E0,8E0,4.6E1,4.1E1,2.4E1,1.2E1,1.8E1,2.3E1,4.1E1,5E0,9E0,3.2E1,1.4E1,1E1,6E0,6E0,1E1,8E0,1.5E1,8E0,3.3E1,8E0,1.6E1,1.6E1,5E0,5E0,5E0,5E0,1E1,5E0,2.1E1,1.2E1,6E0,1E1,5E0,1.1E1,5E0,5E0,7E0,1.4E1,7E0,5E0,5E0,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[4.9883774E-3,4.6291176E-2,-2.0520838E-1,7.5404225E-3,2.6272908E-1,-9.769675E-2,-3.4634717E-2,-2.1027847E-1,2.6024723E-2,5.38507E-1,6.2063127E-3,9.60623E-2,-1.993794E-1,-3.257938E-3,-1.6755927E-2,1.2842016E-1,-1.7241606E-2,3.8207572E-2,1.0676628E-2,-3.8994143E-3,4.468224E-3,8.641754E-3,-3.3278247E-5,-1.9560251E-3,-2.648193E-1,1.6151422E-1,-4.74446E-3,-8.665914E-2,4.113018E-2,-1.5194068E-2,-7.412468E-3,1.1020013E-1,1.631386E-2,-2.7215123E-2,-2.7332512E-1,-2.006159E-2,2.2176762E-1,1.1575388E-2,4.4661142E-2,5.3335936E-3,-7.0063844E-2,-1.700087E-2,-7.4927406E-3,1.014186E-1,-8.442378E-2,1.2667459E-2,6.292186E-3,5.033976E-3,8.142425E-3,-1.349635E-1,4.2505026E-2,8.878536E-3,-7.923698E-4,-1.3646664E-1,-1.3876887E-2,-2.3599262E-3,2.1923988E-3,-8.973601E-3,-3.4827477E-4,-3.951358E-5,3.8485366E-3,-2.6061018E-3,-1.0409134E-2,3.2861547E-3,-4.545861E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,29,31,-1,33,35,-1,-1,37,-1,39,41,43,45,-1,47,-1,49,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8500229E0,1.4909284E0,1.8248602E0,6.1745936E-1,1.906727E0,6.102108E-1,0E0,2.0555007E-1,6.2688285E-1,8.410394E-1,1.1718398E-1,8.6034074E-2,1.9797623E-1,0E0,0E0,3.2908797E-1,4.0876687E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5784194E-2,2.9023623E-1,0E0,5.029855E-1,6.123929E-1,0E0,0E0,2.383641E-1,0E0,2.1794304E-1,4.503244E-2,3.369488E-1,1.5338063E-2,0E0,4.389057E-2,0E0,2.1029568E-1,0E0,0E0,1.4448853E-1,1.0090724E-1,0E0,0E0,0E0,3.1163326E-2,1.1057225E-1,1.732497E-2,0E0,0E0,9.040347E-2,8.967808E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,24,24,25,25,27,27,28,28,31,31,33,33,34,34,35,35,36,36,38,38,40,40,43,43,44,44,48,48,49,49,50,50,53,53,54,54],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,30,32,-1,34,36,-1,-1,38,-1,40,42,44,46,-1,48,-1,50,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.8337E4,5.246E3,8.231275E7,5.64E2,5.25191E5,1.0914128E0,-3.4634717E-2,1.8928572E0,2.1203648E8,1E0,5.269822E5,6.557827E-1,1E0,-3.257938E-3,-1.6755927E-2,7.631266E0,1.6771895E2,3.8207572E-2,1.0676628E-2,-3.8994143E-3,4.468224E-3,8.641754E-3,-3.3278247E-5,-1.9560251E-3,1.4456801E6,4.0972165E2,-4.74446E-3,1.4642024E7,1.9002409E6,-1.5194068E-2,-7.412468E-3,2.73E2,1.631386E-2,8.194808E5,8.36E2,1.9366747E11,1.06748E6,1.1575388E-2,1.5596E4,5.3335936E-3,1.7325802E6,-1.700087E-2,-7.4927406E-3,1.337503E6,1E0,1.2667459E-2,6.292186E-3,5.033976E-3,1.6289E4,9.13242E-1,1.081E3,8.878536E-3,-7.923698E-4,5.6627544E5,1.27047E6,-2.3599262E-3,2.1923988E-3,-8.973601E-3,-3.4827477E-4,-3.951358E-5,3.8485366E-3,-2.6061018E-3,-1.0409134E-2,3.2861547E-3,-4.545861E-3],"split_indices":[9,2,54,2,38,65,0,62,7,18,37,36,23,0,0,65,61,0,0,0,0,0,0,0,56,4,0,54,37,0,0,0,0,57,0,40,1,0,9,0,57,0,0,56,13,0,0,0,9,65,2,0,0,37,1,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.77E2,3.4E1,1.51E2,2.6E1,2.9E1,5E0,1.1E1,1.4E2,1.2E1,1.4E1,1E1,1.9E1,6E0,5E0,4.1E1,9.9E1,6E0,6E0,7E0,7E0,5E0,5E0,6E0,1.3E1,3.6E1,5E0,4.5E1,5.4E1,8E0,5E0,2.9E1,7E0,3.5E1,1E1,4.1E1,1.3E1,9E0,2E1,8E0,2.7E1,5E0,5E0,1.4E1,2.7E1,8E0,5E0,7E0,1.3E1,1.7E1,1E1,8E0,6E0,1.5E1,1.2E1,5E0,8E0,1.2E1,5E0,5E0,5E0,8E0,7E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-6.120535E-4,-3.157735E-2,2.6329273E-1,-7.2816303E-3,-2.3596834E-2,2.7455933E-2,3.1338107E-2,6.946064E-3,-1.4148122E-2,-3.6868812E-3,4.840735E-3,-2.3490578E-1,2.573126E-2,-2.9161605E-3,-1.6471474E-2,6.081072E-2,-6.786494E-2,3.3607185E-2,1.7049152E-1,9.7865835E-2,-1.16619766E-1,1.0627338E-1,-1.0828982E-3,2.5938007E-1,8.847255E-2,-2.3488908E-3,1.1118988E-2,-2.2569573E-1,-4.5155272E-2,-2.6733384E-3,1.535313E-1,1.4573656E-1,-2.8909298E-2,1.6700171E-2,6.543777E-3,6.7003355E-3,1.7051605E-3,-5.1072687E-3,-1.37560805E-2,-9.578486E-2,3.018262E-3,8.77472E-3,2.9851205E-3,2.921744E-3,1.01382155E-2,4.5164762E-4,-7.902687E-3,-5.3824706E-4,-7.114381E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,-1,-1,9,11,-1,-1,-1,13,15,-1,-1,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,-1,41,43,45,-1,-1,-1,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.789786E0,2.1449492E0,1.5743867E0,7.5272995E-1,0E0,0E0,1.0865399E-1,8.1928337E-1,0E0,0E0,0E0,2.1243554E-1,5.5243874E-1,0E0,0E0,3.5828233E-1,3.8310108E-1,2.4967393E-1,1.520251E-1,2.1896914E-1,2.7043077E-1,2.5181702E-1,2.820242E-1,6.7848444E-2,3.168276E-2,0E0,0E0,6.516445E-2,1.304017E-1,0E0,4.9099624E-2,4.5920163E-2,2.9525846E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.589419E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,30,30,31,31,32,32,39,39],"right_children":[2,4,6,8,-1,-1,10,12,-1,-1,-1,14,16,-1,-1,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,-1,42,44,46,-1,-1,-1,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,1.0388931E8,5.939257E7,4.476758E6,-2.3596834E-2,2.7455933E-2,7.3365464E7,9.541066E5,-1.4148122E-2,-3.6868812E-3,4.840735E-3,3.461353E1,3.00037E5,-2.9161605E-3,-1.6471474E-2,4.6253732E2,1.2920592E0,2.92E2,2.5745831E5,1.5539E4,2.8598264E2,4.75E2,1.3257E4,2.2448614E0,3.4480013E8,-2.3488908E-3,1.1118988E-2,1.8508342E5,8.54374E5,-2.6733384E-3,1.988805E8,8.3299077E-1,5.787692E-1,1.6700171E-2,6.543777E-3,6.7003355E-3,1.7051605E-3,-5.1072687E-3,-1.37560805E-2,4.2687164E2,3.018262E-3,8.77472E-3,2.9851205E-3,2.921744E-3,1.01382155E-2,4.5164762E-4,-7.902687E-3,-5.3824706E-4,-7.114381E-3],"split_indices":[2,54,54,37,0,0,54,54,0,0,0,61,38,0,0,4,65,0,42,9,61,2,9,65,7,0,0,41,38,0,7,65,48,0,0,0,0,0,0,61,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.95E2,2.2E1,1.86E2,9E0,9E0,1.3E1,1.78E2,8E0,5E0,8E0,1.2E1,1.66E2,5E0,7E0,1.21E2,4.5E1,9.8E1,2.3E1,1E1,3.5E1,3.1E1,6.7E1,1E1,1.3E1,5E0,5E0,1.3E1,2.2E1,7E0,2.4E1,1E1,5.7E1,5E0,5E0,6E0,7E0,5E0,8E0,1.5E1,7E0,1.8E1,6E0,5E0,5E0,4.5E1,1.2E1,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-6.7648804E-3,-3.4554504E-2,3.2365972E-1,1.307511E-2,-2.2541705E-1,3.4141086E-2,8.979617E-2,-2.723189E-2,1.1041707E-1,-1.2579621E-1,-2.8510313E-2,6.8615894E-3,1.1853566E-3,9.627364E-3,-1.734311E-1,1.6959761E-1,1.2224153E-2,-1.7796133E-2,-6.596992E-2,-1.3065073E-1,4.756531E-2,-1.34643465E-2,-6.5454975E-2,2.0233584E-3,1.9566813E-1,-8.789049E-3,9.991426E-2,-1.2032315E-1,4.8268726E-3,4.100528E-3,-2.1326436E-1,1.4664331E-1,1.0449017E-2,4.665192E-4,-5.8846343E-3,2.6530203E-1,9.5585525E-2,2.633629E-4,1.0230452E-2,-1.6598694E-1,-5.771041E-4,-2.9462366E-3,-1.5664704E-2,3.8109065E-3,9.808171E-3,8.695549E-3,-2.4038892E-2,7.958053E-3,1.51044335E-2,7.4943425E-3,1.071408E-3,-4.5683193E-3,-1.0201266E-2,-7.7659234E-2,3.8386203E-2,2.390561E-3,-6.4840135E-3,3.9413758E-3,-2.8168438E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,-1,-1,19,21,23,25,-1,27,29,31,-1,33,-1,35,-1,37,39,-1,-1,41,43,45,-1,-1,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,55,57,-1,-1,-1,-1],"loss_changes":[1.8935709E0,1.7301822E0,1.3397379E0,6.0331184E-1,1.2813141E0,0E0,3.3064134E-2,5.8904815E-1,2.5942677E-1,4.1790184E-1,0E0,0E0,0E0,4.7453836E-1,2.2432941E-1,8.54857E-2,3.210739E-1,0E0,2.4466035E-1,3.5464582E-1,2.5588626E-1,0E0,5.058928E-2,0E0,1.349312E-1,0E0,1.3328643E-1,9.7145885E-2,0E0,0E0,2.1030712E-1,5.4839373E-2,3.045703E-1,0E0,0E0,7.928431E-3,4.3424174E-2,0E0,0E0,2.3864836E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5026025E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7353089E-1,8.982974E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,18,18,19,19,20,20,22,22,24,24,26,26,27,27,30,30,31,31,32,32,35,35,36,36,39,39,46,46,53,53,54,54],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,-1,-1,20,22,24,26,-1,28,30,32,-1,34,-1,36,-1,38,40,-1,-1,42,44,46,-1,-1,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,56,58,-1,-1,-1,-1],"split_conditions":[1.406168E3,6.0095956E7,1E0,2.3900106E0,3.5753E4,3.4141086E-2,8.932868E6,2.1247E4,8.5913794E5,4.760484E2,-2.8510313E-2,6.8615894E-3,1.1853566E-3,6.97E2,2.23226E-1,2.0618556E-2,2.6569772E-1,-1.7796133E-2,1.1668722E6,4.741633E3,1.3257E4,-1.34643465E-2,3.649047E8,2.0233584E-3,3.4863906E5,-8.789049E-3,4.397E3,2.4192488E0,4.8268726E-3,4.100528E-3,1.7039711E0,4.0190366E5,2.0255389E5,4.665192E-4,-5.8846343E-3,7.977422E-4,2.9420671E0,2.633629E-4,1.0230452E-2,1.9231517E0,-5.771041E-4,-2.9462366E-3,-1.5664704E-2,3.8109065E-3,9.808171E-3,8.695549E-3,3.7721686E2,7.958053E-3,1.51044335E-2,7.4943425E-3,1.071408E-3,-4.5683193E-3,-1.0201266E-2,8.6E2,2.297E3,2.390561E-3,-6.4840135E-3,3.9413758E-3,-2.8168438E-3],"split_indices":[61,54,13,62,9,0,1,9,41,4,0,0,0,2,48,66,36,0,37,57,9,0,7,0,57,0,11,62,0,0,63,41,37,0,0,47,66,0,0,63,0,0,0,0,0,0,4,0,0,0,0,0,0,2,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.89E2,1.5E1,1.52E2,3.7E1,5E0,1E1,1.08E2,4.4E1,3E1,7E0,5E0,5E0,8.7E1,2.1E1,2.7E1,1.7E1,5E0,2.5E1,1.8E1,6.9E1,1E1,1.1E1,5E0,2.2E1,5E0,1.2E1,1.9E1,6E0,5E0,1.3E1,1.8E1,5.1E1,5E0,6E0,1.2E1,1E1,7E0,5E0,1.3E1,6E0,6E0,7E0,9E0,9E0,8E0,4.3E1,5E0,7E0,5E0,5E0,6E0,7E0,2.3E1,2E1,7E0,1.6E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.2954922E-2,2.4751952E-2,-2.1366568E-1,-6.4353556E-3,1.8982898E-1,-1.0397963E-1,-2.9631918E-2,-3.3376355E-2,1.3527302E-1,2.2487383E-2,7.784734E-2,-1.6090518E-1,5.967148E-3,-1.1745218E-2,-1.6251065E-1,-3.6068086E-3,1.9417295E-1,1.281655E-2,-1.33755775E-2,-1.4049971E-2,-9.549068E-2,-2.6180657E-2,1.1752069E-2,-2.498307E-1,-1.0818018E-3,2.415712E-1,2.503387E-3,-2.993908E-3,3.3553487E-3,1.6827921E-3,-8.646101E-3,-1.2880336E-2,-1.1532675E-2,-1.6060045E-2,-6.32815E-3,6.8526575E-3,1.3799591E-2,8.9053795E-2,-5.2454855E-2,-5.9822374E-3,1.3820317E-1,-1.2680641E-2,-3.2844752E-2,1.0323908E-2,2.3265285E-3,1.2083269E-3,-4.821282E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,-1,21,23,-1,25,-1,27,-1,29,31,-1,33,-1,35,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,39,41,-1,43,-1,45,-1,-1,-1,-1],"loss_changes":[1.5794691E0,9.073237E-1,1.357859E0,5.7336193E-1,8.114672E-1,3.670901E-1,0E0,3.5004976E-1,3.098575E-1,0E0,3.5667568E-1,1.5464753E-1,0E0,4.0307745E-1,2.1511286E-1,0E0,1.16371214E-1,0E0,6.3104525E-2,0E0,1.7528033E-1,3.5850453E-1,0E0,6.1705172E-2,0E0,1.8248975E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.1217685E-1,0E0,0E0,0E0,0E0,3.18286E-1,2.7964532E-1,0E0,1.487023E-1,0E0,2.523941E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,13,13,14,14,16,16,18,18,20,20,21,21,23,23,25,25,32,32,37,37,38,38,40,40,42,42],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,-1,22,24,-1,26,-1,28,-1,30,32,-1,34,-1,36,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,40,42,-1,44,-1,46,-1,-1,-1,-1],"split_conditions":[2.68225E5,5.246E3,1.0104842E12,1E0,1.79838E5,4.6328125E0,-2.9631918E-2,6.5023036E7,6.71E2,2.2487383E-2,4.834403E-2,2.5256923E5,5.967148E-3,5.4883756E7,2.223956E0,-3.6068086E-3,2.7078925E8,1.281655E-2,1.919023E-1,-1.4049971E-2,9.73219E6,8.026791E-6,1.1752069E-2,8.87885E5,-1.0818018E-3,4.259865E5,2.503387E-3,-2.993908E-3,3.3553487E-3,1.6827921E-3,-8.646101E-3,-1.2880336E-2,9.39E2,-1.6060045E-2,-6.32815E-3,6.8526575E-3,1.3799591E-2,3.086E3,6.87E2,-5.9822374E-3,3.068497E6,-1.2680641E-2,3.891871E5,1.0323908E-2,2.3265285E-3,1.2083269E-3,-4.821282E-3],"split_indices":[9,2,40,112,38,62,0,54,2,0,47,37,0,54,62,0,7,0,47,0,54,47,0,1,0,37,0,0,0,0,0,0,11,0,0,0,0,38,2,0,12,0,42,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.75E2,3.2E1,1.48E2,2.7E1,2.6E1,6E0,1.25E2,2.3E1,7E0,2E1,2.1E1,5E0,1.08E2,1.7E1,5E0,1.8E1,6E0,1.4E1,6E0,1.5E1,1.03E2,5E0,1E1,7E0,1.3E1,5E0,9E0,5E0,6E0,9E0,5E0,9.8E1,5E0,5E0,5E0,8E0,2.8E1,7E1,5E0,2.3E1,5E0,6.5E1,1.2E1,1.1E1,3.5E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.874389E-2,-3.8119353E-2,1.669629E-2,-2.2960523E-2,-2.378167E-2,4.8027255E-2,-1.2529473E-1,1.835563E-2,1.9371718E-1,-2.5639197E-2,-7.5754456E-2,3.690511E-4,1.4749843E-1,2.8370604E-1,5.756464E-4,-4.9856562E-2,-1.4205237E-2,2.1864083E-2,-1.3645107E-1,1.0622915E-2,2.5949308E-3,5.9493263E-3,1.8673558E-2,-1.4765154E-1,9.095772E-3,4.109614E-2,-8.502569E-2,-9.812589E-3,-2.4152652E-3,-2.4018027E-1,-4.281851E-2,1.1640082E-1,-7.9112716E-2,1.128733E-2,1.22364916E-1,1.2857743E-4,-7.748025E-3,-1.6413901E-2,-4.330508E-3,-5.2493927E-3,9.115278E-4,9.118415E-3,-1.0009622E-3,-1.2936033E-1,3.7364413E-3,-4.2991675E-3,2.4459644E-3,3.2090906E-3,1.006655E-2,-9.969301E-3,-1.9744134E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,19,21,-1,23,-1,25,27,-1,-1,-1,-1,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3674076E0,1.2463471E0,0E0,1.3136237E0,0E0,4.589899E-1,1.4238627E0,2.0900089E-1,2.9136592E-1,0E0,3.583266E-1,2.3825963E-1,6.0979947E-2,1.446324E-1,0E0,3.5302275E-1,0E0,1.4739396E-1,5.164057E-2,0E0,0E0,0E0,0E0,2.1281737E-1,3.7835222E-1,1.4397036E-1,7.12727E-2,0E0,0E0,1.44184E-1,4.963313E-2,1.7664666E-1,1.8184492E-1,1.7746459E-1,6.2318936E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0458088E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,17,17,18,18,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,43,43],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,20,22,-1,24,-1,26,28,-1,-1,-1,-1,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.013055E3,4.739033E6,1.669629E-2,3.452399E8,-2.378167E-2,2.840343E6,6.523944E-2,4.710155E2,2E0,-2.5639197E-2,1.062872E6,3.91014E5,2.4589229E8,8E0,5.756464E-4,2.6727284E5,-1.4205237E-2,7.9410756E-1,1.5566E4,1.0622915E-2,2.5949308E-3,5.9493263E-3,1.8673558E-2,6.124014E5,3.7490938E5,1.5803707E-1,4.0007344E-1,-9.812589E-3,-2.4152652E-3,2.6574177E9,7.335273E-1,2.7438753E0,2.126386E0,8.0563555E6,2.47173E5,1.2857743E-4,-7.748025E-3,-1.6413901E-2,-4.330508E-3,-5.2493927E-3,9.115278E-4,9.118415E-3,-1.0009622E-3,7.2234793E0,3.7364413E-3,-4.2991675E-3,2.4459644E-3,3.2090906E-3,1.006655E-2,-9.969301E-3,-1.9744134E-3],"split_indices":[61,37,0,7,0,41,50,61,8,0,9,38,7,3,0,42,0,48,9,0,0,0,0,37,42,51,36,0,0,12,51,65,63,54,1,0,0,0,0,0,0,0,0,66,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.84E2,9E0,1.79E2,5E0,1.06E2,7.3E1,8.9E1,1.7E1,7E0,6.6E1,7.9E1,1E1,1.1E1,6E0,6E1,6E0,6.9E1,1E1,5E0,5E0,5E0,6E0,2.2E1,3.8E1,5.9E1,1E1,5E0,5E0,1.1E1,1.1E1,1.7E1,2.1E1,4.4E1,1.5E1,5E0,5E0,6E0,5E0,5E0,6E0,1.1E1,6E0,1.6E1,5E0,1.2E1,3.2E1,1E1,5E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[9.505872E-3,-2.2868535E-2,2.0688151E-1,6.38456E-4,-1.7415868E-1,4.7761366E-1,2.9940657E-2,-2.8048113E-2,9.8491505E-2,-2.1289228E-3,-2.409109E-1,3.539001E-2,7.4106096E-3,-3.1162687E-3,9.535988E-2,7.595091E-3,-4.3074265E-2,1.4490497E-1,-2.274651E-3,-1.8296352E-2,-6.685803E-3,1.5511174E-3,6.994412E-3,-5.998781E-2,7.3144156E-3,2.76469E-3,9.206483E-3,-4.3750305E-2,-1.5010274E-2,-1.6205994E-2,-1.4623116E-1,-1.1140883E-2,1.9138195E-3,-1.3569589E-3,-2.0788734E-1,-7.0072006E-4,8.814901E-3,-1.314868E-2,-5.722767E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,21,-1,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,29,-1,31,33,-1,35,-1,37,-1,-1,-1,-1],"loss_changes":[1.2699349E0,6.0825145E-1,1.3270003E0,4.2101568E-1,1.9002491E-1,7.7385116E-1,1.1640851E-1,3.264181E-1,2.3560396E-1,0E0,1.5716851E-1,0E0,0E0,0E0,2.886758E-2,0E0,3.610006E-1,8.9134336E-2,0E0,0E0,0E0,0E0,0E0,3.8778707E-1,0E0,0E0,0E0,2.657928E-1,0E0,2.9199266E-1,1.4031723E-1,0E0,2.0843264E-1,0E0,4.064983E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,14,14,16,16,17,17,23,23,27,27,29,29,30,30,32,32,34,34],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,22,-1,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,30,-1,32,34,-1,36,-1,38,-1,-1,-1,-1],"split_conditions":[5.246E3,3.10364E5,3.0451374E9,1E0,1.0904851E0,9E0,8.853438E8,1.7959783E5,2.82952E5,-2.1289228E-3,4.6922117E-2,3.539001E-2,7.4106096E-3,-3.1162687E-3,1.200998E10,7.595091E-3,2.1907706E12,1.9216243E2,-2.274651E-3,-1.8296352E-2,-6.685803E-3,1.5511174E-3,6.994412E-3,4.674595E6,7.3144156E-3,2.76469E-3,9.206483E-3,3.9741936E0,-1.5010274E-2,9.541066E5,2.0519513E-1,-1.1140883E-2,2.4209485E0,-1.3569589E-3,6.3189E5,-7.0072006E-4,8.814901E-3,-1.314868E-2,-5.722767E-3],"split_indices":[2,9,12,22,65,3,7,37,38,0,47,0,0,0,5,0,40,4,0,0,0,0,0,37,0,0,0,65,0,54,36,0,62,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.97E2,1.7E2,2.7E1,1.48E2,2.2E1,1E1,1.7E1,1.15E2,3.3E1,8E0,1.4E1,5E0,5E0,7E0,1E1,8E0,1.07E2,2.5E1,8E0,5E0,9E0,5E0,5E0,9.9E1,8E0,9E0,1.6E1,9.4E1,5E0,7.5E1,1.9E1,5E0,7E1,7E0,1.2E1,6.5E1,5E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-5.9074727E-3,-4.0919274E-2,1.7781825E-1,-1.4641843E-3,-2.7443483E-1,3.536454E-1,2.9648477E-2,7.021346E-2,-6.0886327E-2,-1.1309023E-1,-2.9482303E-2,6.5705036E-3,3.2264695E-2,5.1792813E-3,-3.5482686E-2,-1.4613068E-1,1.14527784E-1,-2.2916773E-2,-1.8557923E-1,-1.0463648E-2,-5.328639E-4,-3.813157E-3,6.334307E-4,1.5466532E-3,-1.3570637E-2,3.7285738E-2,1.9897132E-1,1.15119115E-1,-6.752203E-2,-2.4229383E-1,-2.626853E-3,-6.216597E-2,8.760398E-2,3.8144905E-3,2.2290906E-1,-4.1558454E-4,8.411455E-3,-1.3276467E-1,5.97091E-3,-1.4840559E-2,-5.873159E-3,2.1342777E-3,-7.705186E-3,9.554621E-3,4.1678715E-2,6.5380284E-3,2.7434108E-1,-3.2559354E-2,-1.9335802E-1,-3.7259797E-3,4.4301033E-2,4.0347306E-3,-1.1073095E-3,1.5754202E-2,8.048992E-3,2.3100814E-3,-5.2278377E-3,-6.5108496E-3,-1.2301882E-2,-1.2629715E-3,8.49077E-2,1.615142E-3,5.993732E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,33,35,37,39,-1,41,43,-1,45,-1,-1,47,49,-1,-1,-1,-1,-1,51,-1,53,55,57,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,-1],"loss_changes":[1.3259528E0,1.5935125E0,8.4923995E-1,6.3890165E-1,1.2577227E0,9.1893697E-1,9.819332E-2,6.6866183E-1,3.8397887E-1,1.8340419E-1,0E0,0E0,0E0,0E0,2.3672156E-2,2.8931102E-1,3.6436892E-1,4.0182912E-1,1.2904513E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6057728E-1,6.076634E-2,1.14135236E-1,2.3574151E-1,6.08809E-2,0E0,1.1802018E-1,9.588295E-2,0E0,6.4181566E-2,0E0,0E0,1.5157056E-1,7.8992635E-2,0E0,0E0,0E0,0E0,0E0,4.3932684E-2,0E0,1.5308738E-2,7.037273E-2,2.2607863E-2,0E0,5.0611895E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7464228E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,17,17,18,18,25,25,26,26,27,27,28,28,29,29,31,31,32,32,34,34,37,37,38,38,44,44,46,46,47,47,48,48,50,50,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,34,36,38,40,-1,42,44,-1,46,-1,-1,48,50,-1,-1,-1,-1,-1,52,-1,54,56,58,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,-1],"split_conditions":[7.737766E2,6.0004652E7,5.8816573E9,7.701429E5,1.0055311E1,1E0,2.0657775E0,4.305949E-2,2.294901E0,2.38502E0,-2.9482303E-2,6.5705036E-3,3.2264695E-2,5.1792813E-3,1.9E1,2.87275E3,1.002E3,1.3257E4,3.0147552E2,-1.0463648E-2,-5.328639E-4,-3.813157E-3,6.334307E-4,1.5466532E-3,-1.3570637E-2,2.1602914E0,2.2694193E2,2.0581586E5,7.2234793E0,6.2775606E5,-2.626853E-3,1.8558352E0,2.73E2,3.8144905E-3,1E0,-4.1558454E-4,8.411455E-3,3.819848E8,1.7598796E0,-1.4840559E-2,-5.873159E-3,2.1342777E-3,-7.705186E-3,9.554621E-3,2.430046E0,6.5380284E-3,3.0963843E8,1.0402472E6,5.520198E-1,-3.7259797E-3,1.6060533E-1,4.0347306E-3,-1.1073095E-3,1.5754202E-2,8.048992E-3,2.3100814E-3,-5.2278377E-3,-6.5108496E-3,-1.2301882E-2,-1.2629715E-3,1.9944866E0,1.615142E-3,5.993732E-3],"split_indices":[61,54,5,57,66,8,63,67,65,62,0,0,0,0,3,42,2,9,61,0,0,0,0,0,0,62,4,41,66,56,0,62,0,0,18,0,0,7,62,0,0,0,0,0,62,0,7,60,48,0,51,0,0,0,0,0,0,0,0,0,62,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.72E2,3.2E1,1.48E2,2.4E1,1.4E1,1.8E1,6.7E1,8.1E1,1.7E1,7E0,9E0,5E0,8E0,1E1,1.1E1,5.6E1,6.3E1,1.8E1,8E0,9E0,5E0,5E0,5E0,6E0,3E1,2.6E1,1.5E1,4.8E1,1.2E1,6E0,1E1,2E1,5E0,2.1E1,5E0,1E1,2.5E1,2.3E1,7E0,5E0,5E0,5E0,5E0,1.5E1,9E0,1.2E1,1E1,1.5E1,7E0,1.6E1,9E0,6E0,7E0,5E0,5E0,5E0,9E0,6E0,6E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.4113026E-3,-3.6776196E-2,1.9633609E-1,-7.5939216E-4,-2.6725337E-1,3.6574346E-1,7.64045E-2,8.7626435E-2,-3.687569E-2,-1.5034501E-1,-2.9934434E-2,2.6890999E-2,2.8800634E-3,1.1789254E-2,8.400222E-3,5.4605924E-2,1.1818014E-2,-7.453918E-2,1.0903748E-1,-2.0487392E-1,-9.682808E-4,-2.7833784E-3,3.8535749E-3,9.6058294E-2,-3.887657E-2,-2.5205103E-1,-3.944665E-2,1.5640594E-1,6.2134484E-4,-5.3765257E-3,-1.38839055E-2,1.21690884E-1,-8.9008955E-4,-4.783394E-3,1.2542846E-3,-1.5987882E-2,-4.15421E-3,1.1821422E-1,-6.990339E-2,9.512716E-3,4.588463E-3,7.37696E-3,6.7078037E-4,1.7392452E-4,9.259326E-3,-1.5715963E-1,-1.2865197E-2,-1.2011218E-2,-9.581573E-2,6.52676E-3,-4.1775856E-2,-6.790576E-3,3.1129192E-4,6.2908715E-4,-5.6454046E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,23,-1,25,27,29,-1,-1,-1,31,33,35,37,39,-1,-1,-1,41,-1,-1,-1,-1,-1,43,45,-1,-1,-1,-1,-1,-1,47,49,-1,51,-1,53,-1,-1,-1,-1],"loss_changes":[1.4824861E0,1.5354915E0,6.2220895E-1,5.171704E-1,9.167075E-1,7.0309556E-1,1.2132346E-1,2.2603175E-1,6.401214E-1,1.3830096E-1,0E0,0E0,0E0,6.4398E-2,0E0,1.5873727E-1,0E0,5.631567E-1,1.0751921E-1,7.3915064E-2,0E0,0E0,0E0,8.42275E-2,5.1996946E-2,1.6023028E-1,3.8280678E-1,1.995954E-2,0E0,0E0,0E0,7.05114E-2,0E0,0E0,0E0,0E0,0E0,1.0547434E-1,3.26456E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1771721E-1,1.7910103E-1,0E0,7.576808E-2,0E0,1.4248207E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,15,15,17,17,18,18,19,19,23,23,24,24,25,25,26,26,27,27,31,31,37,37,38,38,45,45,46,46,48,48,50,50],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,24,-1,26,28,30,-1,-1,-1,32,34,36,38,40,-1,-1,-1,42,-1,-1,-1,-1,-1,44,46,-1,-1,-1,-1,-1,-1,48,50,-1,52,-1,54,-1,-1,-1,-1],"split_conditions":[7.737766E2,6.0095956E7,5.25191E5,3.77E2,2.9119E4,1.3770975E2,1.6089E4,6.204429E6,9.015486E-1,2.38502E0,-2.9934434E-2,2.6890999E-2,2.8800634E-3,1.7E1,8.400222E-3,8.575148E-1,1.1818014E-2,1.4773677E5,2.126E4,2.014519E0,-9.682808E-4,-2.7833784E-3,3.8535749E-3,1.4175246E6,1.4937E4,3.6327022E5,9.619212E5,1.479E3,6.2134484E-4,-5.3765257E-3,-1.38839055E-2,1.0661045E6,-8.9008955E-4,-4.783394E-3,1.2542846E-3,-1.5987882E-2,-4.15421E-3,1E0,3.6452372E5,9.512716E-3,4.588463E-3,7.37696E-3,6.7078037E-4,1.7392452E-4,9.259326E-3,1.9679782E-1,1.3255E4,-1.2011218E-2,2.5256923E5,6.52676E-3,1E0,-6.790576E-3,3.1129192E-4,6.2908715E-4,-5.6454046E-3],"split_indices":[61,54,38,0,9,67,9,41,36,62,0,0,0,3,0,36,0,42,9,63,0,0,0,59,9,41,59,0,0,0,0,41,0,0,0,0,0,18,37,0,0,0,0,0,0,50,9,0,37,0,18,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,1.84E2,3.1E1,1.6E2,2.4E1,1.2E1,1.9E1,4.6E1,1.14E2,1.9E1,5E0,7E0,5E0,1.2E1,7E0,3.9E1,7E0,9.1E1,2.3E1,1.3E1,6E0,6E0,6E0,2.7E1,1.2E1,1.4E1,7.7E1,1.5E1,8E0,7E0,6E0,2.2E1,5E0,6E0,6E0,9E0,5E0,1.2E1,6.5E1,8E0,7E0,1.7E1,5E0,5E0,7E0,2.5E1,4E1,9E0,1.6E1,6E0,3.4E1,1.1E1,5E0,2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-7.2399722E-3,1.97451E-2,-1.7070809E-1,-1.4477426E-1,3.6474522E-2,-3.241367E-2,-2.3941144E-2,1.1121975E-3,-2.3822159E-1,2.466752E-2,1.1716569E-2,-1.1494423E-1,6.177603E-3,-6.1941557E-3,-1.5153707E-2,-3.8153403E-2,6.351863E-2,-9.526542E-3,-1.9546042E-3,4.488392E-2,-1.0541493E-1,4.5627907E-2,1.6142078E-2,1.353615E-1,-6.9925696E-2,-2.3379382E-1,-3.3851374E-2,1.397906E-1,1.4481856E-2,2.546165E-3,8.804052E-3,-5.307041E-3,-1.0406233E-3,-6.3540344E-3,-1.544268E-2,-9.763366E-4,-5.3208326E-3,9.679696E-2,1.0960555E-2,-3.727233E-2,7.872203E-2,3.6497288E-3,-3.8716644E-2,1.8688258E-3,7.068344E-3,-9.207874E-3,-9.520353E-3,3.163374E-2,1.6574207E-1,-3.4349354E-3,2.2247965E-4,-2.6969654E-3,4.6973885E-3,3.8737096E-3,-1.6340516E-3,2.1894122E-3,1.266331E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,-1,-1,13,15,-1,17,-1,-1,-1,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,41,-1,43,-1,45,47,-1,49,-1,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.3413365E-1,5.062017E-1,1.307956E0,2.7155587E-1,3.924021E-1,3.028494E-1,0E0,0E0,4.8772633E-2,3.8804814E-1,0E0,8.210038E-2,0E0,0E0,0E0,3.4542966E-1,4.5256922E-1,0E0,0E0,3.003207E-1,3.0445224E-1,2.7072555E-1,0E0,5.0245285E-2,2.2124007E-2,6.1241746E-2,5.660578E-2,6.1796606E-2,2.389877E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.1422507E-2,0E0,4.2069316E-2,0E0,1.7500961E-1,1.2588096E-1,0E0,1.6704332E-2,0E0,0E0,1.7472784E-1,0E0,7.080604E-2,1.1256322E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,11,11,15,15,16,16,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,35,35,37,37,39,39,40,40,42,42,45,45,47,47,48,48],"right_children":[2,4,6,8,10,12,-1,-1,14,16,-1,18,-1,-1,-1,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,42,-1,44,-1,46,48,-1,50,-1,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.68225E5,2.3135895E6,7.224059E-1,1.3226E4,1.3919099E7,4.6328125E0,-2.3941144E-2,1.1121975E-3,4.27E2,1.4733E4,1.1716569E-2,4.4818586E-1,6.177603E-3,-6.1941557E-3,-1.5153707E-2,1.7923728E0,1.00904926E3,-9.526542E-3,-1.9546042E-3,3.706683E5,5.88998E5,1.6003E4,1.6142078E-2,6.8915665E-1,1.6868132E0,3.3E2,1.1467657E8,4.4893765E2,2.195572E0,2.546165E-3,8.804052E-3,-5.307041E-3,-1.0406233E-3,-6.3540344E-3,-1.544268E-2,9.298268E4,-5.3208326E-3,2.0043669E0,1.0960555E-2,4.213523E0,7.3972464E-1,3.6497288E-3,5.666E3,1.8688258E-3,7.068344E-3,1.436E3,-9.520353E-3,1.9776594E9,2.6944466E7,-3.4349354E-3,2.2247965E-4,-2.6969654E-3,4.6973885E-3,3.8737096E-3,-1.6340516E-3,2.1894122E-3,1.266331E-2],"split_indices":[9,54,36,9,41,62,0,0,0,9,0,36,0,0,0,63,61,0,0,42,1,9,0,65,63,10,54,64,63,0,0,0,0,0,0,42,0,63,0,65,36,0,2,0,0,2,0,5,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.81E2,2.9E1,1.6E1,1.65E2,2.1E1,8E0,6E0,1E1,1.57E2,8E0,1.4E1,7E0,5E0,5E0,6E1,9.7E1,6E0,8E0,2.7E1,3.3E1,9.2E1,5E0,1.5E1,1.2E1,1.1E1,2.2E1,2.2E1,7E1,6E0,9E0,6E0,6E0,6E0,5E0,1.6E1,6E0,1.6E1,6E0,3.9E1,3.1E1,5E0,1.1E1,8E0,8E0,3.4E1,5E0,2.1E1,1E1,6E0,5E0,2.4E1,1E1,1.2E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.0464338E-2,-4.442531E-4,-1.7487004E-2,7.634471E-2,-4.4897687E-2,5.8408406E-2,1.2091161E-2,-2.743118E-1,-2.4015952E-2,7.963508E-3,4.0609732E-2,-5.434861E-3,-1.9147173E-2,-4.187414E-2,1.616196E-1,-1.2487171E-2,1.0146094E-1,-1.1732448E-2,-1.5930979E-1,1.2026224E-2,2.4570697E-3,-3.741369E-2,5.3803376E-3,4.134274E-2,1.3590303E-1,-4.0736023E-2,1.0031985E-1,-2.2459684E-1,1.0755105E-3,-4.456786E-3,-5.3844363E-3,-8.662233E-4,4.2067673E-3,3.2502434E-3,8.06845E-3,-1.7618014E-1,-8.453462E-3,3.5672414E-4,1.3302067E-1,-1.4784077E-2,-5.952732E-3,2.7359195E-2,-4.3286537E-3,-2.3447168E-3,-1.45437205E-2,2.6357116E-2,-8.109932E-3,3.5144843E-3,7.771326E-3,2.5852525E-3,-1.1734442E-3,3.258402E-3,-2.395021E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,15,-1,-1,17,19,21,23,25,27,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,43,45,-1,47,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.44532E-1,7.168561E-1,0E0,2.2174564E-1,6.305572E-1,1.2545492E-1,0E0,1.6711438E-1,4.1452676E-1,0E0,2.0158961E-1,0E0,0E0,3.9811796E-1,9.1031164E-2,1.0798748E-1,5.4428697E-2,2.998291E-1,2.772016E-1,0E0,0E0,6.9032595E-2,0E0,3.322155E-2,2.545935E-2,3.1770194E-1,5.682519E-2,9.9252164E-2,0E0,5.8612924E-2,0E0,0E0,0E0,0E0,0E0,2.0356134E-1,3.332524E-1,0E0,1.0788351E-2,0E0,0E0,2.2252012E-2,0E0,0E0,0E0,1.5512183E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,21,21,23,23,24,24,25,25,26,26,27,27,29,29,35,35,36,36,38,38,41,41,45,45],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,16,-1,-1,18,20,22,24,26,28,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,44,46,-1,48,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,2.0136692E7,-1.7487004E-2,2.738E3,8.7722336E4,2.0080225E5,1.2091161E-2,1.4157E4,1.0086E4,7.963508E-3,1E0,-5.434861E-3,-1.9147173E-2,2.68225E5,1E1,9.42673E-1,2.0433267E2,1E0,2.6056657E0,1.2026224E-2,2.4570697E-3,1.3843815E6,5.3803376E-3,2.0967135E0,1.0695187E-2,1.3371641E6,1.9759825E0,1E0,1.0755105E-3,7.4811506E5,-5.3844363E-3,-8.662233E-4,4.2067673E-3,3.2502434E-3,8.06845E-3,1.7340133E5,2.09E3,3.5672414E-4,4.110594E8,-1.4784077E-2,-5.952732E-3,8.909818E10,-4.3286537E-3,-2.3447168E-3,-1.45437205E-2,1.7946554E0,-8.109932E-3,3.5144843E-3,7.771326E-3,2.5852525E-3,-1.1734442E-3,3.258402E-3,-2.395021E-3],"split_indices":[37,12,0,2,41,37,0,9,2,0,22,0,0,9,3,36,4,24,62,0,0,37,0,63,66,60,62,25,0,41,0,0,0,0,0,56,0,0,7,0,0,40,0,0,0,63,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,2.08E2,5E0,7.6E1,1.32E2,7E1,6E0,1E1,1.22E2,9E0,6.1E1,5E0,5E0,1.12E2,1E1,3.3E1,2.8E1,9E1,2.2E1,5E0,5E0,2.8E1,5E0,1.1E1,1.7E1,7.2E1,1.8E1,1.6E1,6E0,2E1,8E0,5E0,6E0,6E0,1.1E1,1.3E1,5.9E1,5E0,1.3E1,8E0,8E0,1.5E1,5E0,7E0,6E0,4.9E1,1E1,5E0,8E0,1E1,5E0,3.2E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.0472162E-2,2.934161E-2,-1.9367042E-1,1.6231367E-2,2.5908303E-1,-4.3677952E-2,-2.499602E-2,2.5613489E-2,-1.4303443E-1,1.544507E-3,2.1672823E-2,-7.7033634E-3,1.8144933E-3,-1.345117E-2,8.573257E-2,-9.962586E-3,-2.8552248E-3,1.6495258E-2,-1.3135003E-1,1.3078292E-1,-1.7889181E-2,3.5149373E-2,-1.03809275E-2,6.121363E-4,-1.9507729E-1,1.5289715E-1,5.1786465E-4,-6.707827E-2,4.043611E-3,-7.50564E-2,5.983713E-2,-1.2591068E-2,-2.344087E-3,1.6905531E-1,1.2258741E-3,-1.1069692E-1,1.1764925E-3,-8.872362E-3,3.4063405E-4,1.0425993E-1,5.871509E-4,2.9379649E-3,1.8374148E-1,-1.8922279E-3,-8.027706E-3,7.6216E-3,3.791823E-4,2.9213736E-3,-1.9520083E-3,1.3124342E-2,7.574533E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,-1,17,19,-1,-1,21,23,25,27,29,-1,-1,31,33,-1,35,-1,37,39,-1,-1,41,-1,43,-1,-1,-1,45,47,-1,49,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.646915E-1,6.1324775E-1,8.830751E-1,2.9504257E-1,4.4718206E-1,1.3471454E-1,0E0,4.3563148E-1,4.467307E-2,0E0,0E0,0E0,0E0,4.0113008E-1,3.431179E-1,0E0,0E0,3.9597848E-1,2.133219E-1,1.3393158E-1,1.1937898E-1,2.3575345E-1,0E0,0E0,1.3480079E-1,8.456838E-2,0E0,6.6275865E-2,0E0,1.4097866E-1,1.8418232E-1,0E0,0E0,4.9758077E-2,0E0,3.603396E-2,0E0,0E0,0E0,1.9822073E-1,7.6725945E-2,0E0,3.8891792E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,13,13,14,14,17,17,18,18,19,19,20,20,21,21,24,24,25,25,27,27,29,29,30,30,33,33,35,35,39,39,40,40,42,42],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,-1,18,20,-1,-1,22,24,26,28,30,-1,-1,32,34,-1,36,-1,38,40,-1,-1,42,-1,44,-1,-1,-1,46,48,-1,50,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2E1,6.811E3,1.0083192E6,6.3018E4,2.6014878E5,4E0,-2.499602E-2,1.6971204E2,8.0961E4,1.544507E-3,2.1672823E-2,-7.7033634E-3,1.8144933E-3,1.15318E5,4.213523E0,-9.962586E-3,-2.8552248E-3,3.241538E6,1.9303116E5,6.627576E7,1E0,1.1744206E6,-1.03809275E-2,6.121363E-4,1.23844246E2,2.1282E4,5.1786465E-4,1.2E1,4.043611E-3,1.1256753E0,6.4781194E5,-1.2591068E-2,-2.344087E-3,2.5686172E-1,1.2258741E-3,8.8317425E5,1.1764925E-3,-8.872362E-3,3.4063405E-4,4.0500336E0,1.4848E4,2.9379649E-3,1.6506441E9,-1.8922279E-3,-8.027706E-3,7.6216E-3,3.791823E-4,2.9213736E-3,-1.9520083E-3,1.3124342E-2,7.574533E-3],"split_indices":[3,2,37,10,42,8,0,61,10,0,0,0,0,38,65,0,0,37,41,54,22,54,0,0,61,9,0,3,0,48,37,0,0,65,0,37,0,0,0,66,38,0,5,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,2.04E2,1.8E1,1.94E2,1E1,1.3E1,5E0,1.84E2,1E1,5E0,5E0,5E0,8E0,1.12E2,7.2E1,5E0,5E0,9E1,2.2E1,5E1,2.2E1,8.4E1,6E0,7E0,1.5E1,4.2E1,8E0,1.5E1,7E0,1.5E1,6.9E1,1E1,5E0,3.7E1,5E0,1E1,5E0,6E0,9E0,3.9E1,3E1,5E0,3.2E1,5E0,5E0,2.5E1,1.4E1,1.2E1,1.8E1,6E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-3.719925E-3,1.847095E-2,-1.1642022E-1,-8.259407E-4,1.95405E-1,1.1690613E-2,-2.7728942E-1,-3.1212017E-2,8.681823E-2,1.9222945E-2,-1.0811003E-4,-3.852253E-3,7.344581E-2,-6.535609E-3,-2.1643821E-2,-1.451615E-2,-1.8583044E-1,1.11574255E-1,-2.758676E-3,9.1190357E-4,6.5625412E-3,-1.16575904E-1,1.4315423E-2,-1.4151355E-2,-3.4418197E-3,1.4570087E-1,2.8335406E-2,3.477154E-3,-1.670608E-1,8.731958E-2,-3.9309066E-2,8.9445494E-2,1.1830084E-2,6.3542402E-3,-4.6432074E-3,-2.2398639E-1,-2.6235955E-3,4.5165658E-2,1.4245836E-1,-1.8002889E-1,2.835981E-3,1.5279662E-3,1.0942015E-1,-5.072533E-3,-1.3986866E-2,7.414421E-2,-9.012092E-4,1.7429148E-1,2.9501382E-3,-1.1179438E-2,-5.5030393E-3,4.072813E-3,-2.2404697E-2,7.069936E-3,2.1617597E-3,6.14034E-3,3.404664E-4,4.5951284E-3,1.1023735E-2,-2.5666563E-3,2.398851E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,23,25,-1,-1,-1,27,29,-1,-1,31,33,-1,35,37,39,41,-1,-1,-1,43,-1,45,47,49,51,-1,53,-1,-1,55,-1,57,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.473506E-1,6.249742E-1,7.451688E-1,4.4479477E-1,7.0468676E-1,1.2315521E-1,3.2101285E-1,3.1740707E-1,1.5879413E-1,0E0,0E0,0E0,4.0809885E-2,0E0,0E0,3.3417624E-1,1.2321329E-1,1.0097596E-1,0E0,0E0,0E0,2.4880007E-1,3.5185057E-1,0E0,0E0,1.1923623E-1,1.6279964E-1,0E0,1.1706978E-1,8.188981E-2,3.0890405E-1,1.7499998E-2,0E0,0E0,0E0,6.7445815E-2,0E0,4.3758705E-2,3.1298548E-2,1.3652742E-2,8.515977E-2,0E0,2.3451656E-2,0E0,0E0,5.445493E-2,0E0,2.4044901E-2,0E0,0E0,0E0,0E0,7.1437836E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,16,16,17,17,21,21,22,22,25,25,26,26,28,28,29,29,30,30,31,31,35,35,37,37,38,38,39,39,40,40,42,42,45,45,47,47,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,24,26,-1,-1,-1,28,30,-1,-1,32,34,-1,36,38,40,42,-1,-1,-1,44,-1,46,48,50,52,-1,54,-1,-1,56,-1,58,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,3.973E3,8.493112E-1,2.3025806E0,1.5743027E7,9.197156E2,1.1668722E6,5.058997E0,1.1797377E0,1.9222945E-2,-1.0811003E-4,-3.852253E-3,1.4062E4,-6.535609E-3,-2.1643821E-2,7.23E2,1.993E3,2.1526105E0,-2.758676E-3,9.1190357E-4,6.5625412E-3,5.4472397E3,7.07E2,-1.4151355E-2,-3.4418197E-3,3.3488025E2,1.6671585E9,3.477154E-3,3.0050538E5,1.104E3,1.0469056E2,1.9098653E5,1.1830084E-2,6.3542402E-3,-4.6432074E-3,1.7887033E4,-2.6235955E-3,6.1910925E0,1.9557953E0,6.109438E-1,1.6692607E0,1.5279662E-3,1.9760254E-1,-5.072533E-3,-1.3986866E-2,9.201837E6,-9.012092E-4,1.7829868E2,2.9501382E-3,-1.1179438E-2,-5.5030393E-3,4.072813E-3,2.1723518E0,7.069936E-3,2.1617597E-3,6.14034E-3,3.404664E-4,4.5951284E-3,1.1023735E-2,-2.5666563E-3,2.398851E-3],"split_indices":[54,2,36,62,54,61,37,65,51,0,0,0,9,0,0,2,2,65,0,0,0,59,0,0,0,4,5,0,42,2,61,42,0,0,0,57,0,66,63,36,63,0,48,0,0,54,0,4,0,0,0,0,62,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.17E2,1.82E2,3.5E1,1.65E2,1.7E1,2E1,1.5E1,1.23E2,4.2E1,8E0,9E0,8E0,1.2E1,9E0,6E0,1.12E2,1.1E1,3.6E1,6E0,7E0,5E0,2.4E1,8.8E1,5E0,6E0,2.5E1,1.1E1,5E0,1.9E1,3.7E1,5.1E1,1.7E1,8E0,6E0,5E0,1.2E1,7E0,2.2E1,1.5E1,1.1E1,4E1,5E0,1.2E1,5E0,7E0,1.5E1,7E0,1E1,5E0,5E0,6E0,9E0,3.1E1,7E0,5E0,8E0,7E0,5E0,5E0,2.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-1.0759566E-2,-2.5174906E-2,1.3971433E-2,-9.609711E-3,-2.5432163E-1,-8.2856865E-4,-1.384042E-2,-3.462074E-3,-2.23183E-2,-2.7887626E-2,9.686823E-2,4.1767113E-2,-6.95304E-2,1.32453125E-2,1.6173637E-1,-6.6621536E-3,6.998623E-2,-5.3341404E-2,-1.2748097E-2,5.8277357E-2,-4.5965356E-3,1.04965776E-1,1.1000048E-2,-1.7902972E-3,1.3354972E-1,6.30305E-3,-6.6029236E-2,-1.4714632E-4,5.1138056E-3,8.4239215E-4,9.298736E-3,8.115957E-2,-6.683732E-2,9.0861665E-3,8.922104E-2,-9.190564E-3,-4.8413295E-2,6.694404E-3,5.7858665E-4,-5.7639624E-3,-6.7380536E-4,7.677994E-3,1.6032133E-3,-1.6948568E-4,-5.509772E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,13,15,17,19,21,-1,23,25,-1,27,-1,29,-1,31,33,-1,35,-1,-1,-1,-1,37,39,-1,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.221425E-1,7.2873235E-1,0E0,4.6335164E-1,4.501623E-1,4.9969506E-1,0E0,0E0,0E0,4.3219528E-1,2.202608E-1,2.9092664E-1,2.7406478E-1,9.727192E-2,5.9037447E-2,0E0,2.2379501E-1,2.0973448E-1,0E0,4.0077742E-2,0E0,1.01454064E-1,0E0,1.3491988E-1,4.373601E-2,0E0,1.6711766E-1,0E0,0E0,0E0,0E0,4.0926196E-2,3.631571E-2,0E0,5.610633E-2,0E0,2.106034E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,19,19,21,21,23,23,24,24,26,26,31,31,32,32,34,34,36,36],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,14,16,18,20,22,-1,24,26,-1,28,-1,30,-1,32,34,-1,36,-1,-1,-1,-1,38,40,-1,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3717E4,2.5E1,1.3971433E-2,5.3159845E6,7.475293E5,1.5452969E6,-1.384042E-2,-3.462074E-3,-2.23183E-2,2.0136692E7,2.874086E2,1.4221454E0,7.3798E4,6.96E2,2.114345E0,-6.6621536E-3,1.2971756E5,1.002E4,-1.2748097E-2,4.6524E4,-4.5965356E-3,5.0826204E7,1.1000048E-2,3.12E2,4.898012E-1,6.30305E-3,1.573075E5,-1.4714632E-4,5.1138056E-3,8.4239215E-4,9.298736E-3,1.9321119E3,1.289E3,9.0861665E-3,1.8691002E8,-9.190564E-3,1.8981702E0,6.694404E-3,5.7858665E-4,-5.7639624E-3,-6.7380536E-4,7.677994E-3,1.6032133E-3,-1.6948568E-4,-5.509772E-3],"split_indices":[2,3,0,37,37,37,0,0,0,12,61,62,10,0,63,0,59,9,0,1,0,54,0,11,36,0,42,0,0,0,0,56,11,0,7,0,65,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,2.04E2,9E0,1.92E2,1.2E1,1.87E2,5E0,7E0,5E0,1.47E2,4E1,5.5E1,9.2E1,1.8E1,2.2E1,7E0,4.8E1,8.6E1,6E0,1.3E1,5E0,1.3E1,9E0,2.3E1,2.5E1,5E0,8.1E1,6E0,7E0,7E0,6E0,1E1,1.3E1,1E1,1.5E1,9E0,7.2E1,5E0,5E0,6E0,7E0,6E0,9E0,4.3E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[9.168584E-3,-1.425132E-2,1.5016305E-1,5.890259E-3,-1.4017038E-1,2.0255508E-2,5.1728528E-2,1.5233301E-2,-1.1329513E-2,5.00593E-4,-1.882078E-1,-2.9404997E-3,1.2795217E-1,-9.932106E-3,9.9641435E-2,-1.5657976E-2,-1.0258512E-1,2.4334332E-3,8.106641E-3,4.6279956E-2,-5.052154E-2,1.7180847E-2,1.6817766E-1,-8.412362E-3,-1.8390764E-3,-4.58536E-2,7.0459686E-2,-1.2762372E-1,-1.6446695E-2,-1.7891317E-3,2.8589673E-3,3.2762445E-3,1.0831688E-2,8.8755484E-4,-4.9966546E-3,1.09391175E-1,1.1352887E-2,-1.0438406E-2,-5.831815E-2,2.1567825E-2,-7.264614E-2,1.5956362E-1,6.2156323E-2,2.3330664E-3,-1.2848653E-3,-4.679209E-3,6.265522E-5,5.544526E-3,-1.3421011E-3,-6.5641697E-3,-4.314499E-2,4.663722E-3,9.635326E-3,4.806839E-3,-1.2111189E-4,1.1728182E-3,-2.397551E-3,1.1797537E-3,-4.43247E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,15,-1,17,19,21,-1,23,-1,-1,25,27,29,31,-1,-1,33,35,37,39,-1,-1,-1,-1,-1,-1,41,43,-1,45,47,49,51,53,-1,-1,-1,-1,-1,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.955761E-1,4.5987943E-1,7.5884134E-1,3.5216185E-1,1.8256223E-1,0E0,2.0384946E-1,3.2386273E-1,0E0,0E0,1.8208623E-1,0E0,3.3214614E-2,2.715688E-1,1.9491446E-1,0E0,5.2231252E-2,0E0,0E0,1.1461905E-1,1.791809E-1,3.9909966E-2,8.781296E-2,0E0,0E0,4.1547082E-2,9.080273E-2,1.12606645E-1,1.0625017E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.695943E-2,2.453126E-2,0E0,2.910811E-2,6.328974E-2,3.0578077E-2,7.696092E-3,3.298492E-2,0E0,0E0,0E0,0E0,0E0,3.1394728E-2,0E0,5.0161954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,10,10,12,12,13,13,14,14,16,16,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,35,35,36,36,38,38,39,39,40,40,41,41,42,42,48,48,50,50],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,16,-1,18,20,22,-1,24,-1,-1,26,28,30,32,-1,-1,34,36,38,40,-1,-1,-1,-1,-1,-1,42,44,-1,46,48,50,52,54,-1,-1,-1,-1,-1,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.246E3,2.68225E5,3.0451374E9,4.966421E6,1E0,2.0255508E-2,9.248443E8,1.5811304E6,-1.1329513E-2,5.00593E-4,4.6922117E-2,-2.9404997E-3,2.3E1,5.29E2,6.8915665E-1,-1.5657976E-2,2.8003845E2,2.4334332E-3,8.106641E-3,1.4221454E0,9.9E1,2.1602914E0,9.123214E-1,-8.412362E-3,-1.8390764E-3,1.5681747E9,1.2557897E7,5.759701E-1,2.7814416E11,-1.7891317E-3,2.8589673E-3,3.2762445E-3,1.0831688E-2,8.8755484E-4,-4.9966546E-3,2.1203648E8,4.607055E5,-1.0438406E-2,8.983957E-1,1.3318E4,2E0,3.37825E5,2.0174673E0,2.3330664E-3,-1.2848653E-3,-4.679209E-3,6.265522E-5,5.544526E-3,4.938111E5,-6.5641697E-3,2.872E4,4.663722E-3,9.635326E-3,4.806839E-3,-1.2111189E-4,1.1728182E-3,-2.397551E-3,1.1797537E-3,-4.43247E-3],"split_indices":[2,9,12,37,24,0,7,37,0,0,47,0,8,0,65,0,4,0,0,62,61,62,36,0,0,40,54,36,40,0,0,0,0,0,0,7,37,0,65,9,8,12,62,0,0,0,0,0,41,0,10,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,1.8E2,2.9E1,1.56E2,2.4E1,7E0,2.2E1,1.51E2,5E0,6E0,1.8E1,9E0,1.3E1,1.17E2,3.4E1,6E0,1.2E1,5E0,8E0,4.9E1,6.8E1,1.6E1,1.8E1,5E0,7E0,1E1,3.9E1,2E1,4.8E1,7E0,9E0,7E0,1.1E1,5E0,5E0,2.3E1,1.6E1,8E0,1.2E1,2.9E1,1.9E1,1E1,1.3E1,8E0,8E0,7E0,5E0,5E0,2.4E1,5E0,1.4E1,5E0,5E0,8E0,5E0,1.6E1,8E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-3.0483364E-3,1.8253405E-2,-1.9664758E-1,1.4306073E-3,2.6907426E-1,-3.3235466E-1,3.197772E-3,4.492458E-2,-3.974036E-2,2.3746382E-2,2.1928768E-3,-3.3603942E-3,-2.5489317E-2,-9.636486E-3,6.188236E-2,-2.0972298E-1,-1.4380218E-2,-5.188022E-3,7.401362E-2,-1.3366703E-2,-4.9080625E-3,-1.0019511E-1,9.605139E-3,9.373284E-2,-4.2253155E-2,-7.4644303E-3,-1.5012765E-3,9.011987E-2,-2.5610073E-2,1.1820235E-1,3.8308743E-2,3.7830765E-3,-6.7832153E-3,7.4405423E-3,4.4971887E-2,-2.5402256E-3,-9.076295E-2,9.741266E-2,1.1758828E-2,8.470141E-2,-4.86464E-3,4.152712E-3,4.5758853E-4,1.512331E-2,-4.4204467E-3,-1.6948156E-3,-6.152729E-3,6.1157714E-3,2.2633122E-3,5.6014425E-4,6.590244E-3,-4.581938E-4,3.5841241E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,-1,-1,17,19,21,-1,23,-1,-1,25,27,29,31,-1,-1,33,35,37,39,-1,-1,-1,41,43,45,47,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.737537E-1,8.0554116E-1,7.685981E-1,3.2410935E-1,5.613397E-1,6.620668E-1,0E0,3.7343735E-1,3.97887E-1,0E0,0E0,0E0,0E0,0E0,1.770342E-1,5.6147933E-2,1.6974305E-1,0E0,1.8279034E-1,0E0,0E0,6.098737E-2,1.8660529E-1,8.633721E-2,1.493209E-1,0E0,0E0,4.9228594E-2,6.8910524E-2,9.358436E-2,1.4910498E-1,0E0,0E0,0E0,1.7715283E-2,5.5706017E-2,1.9268118E-2,5.2262396E-2,0E0,6.0864113E-2,0E0,0E0,0E0,4.346049E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,14,14,15,15,16,16,18,18,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,34,34,35,35,36,36,37,37,39,39,43,43],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,-1,-1,18,20,22,-1,24,-1,-1,26,28,30,32,-1,-1,34,36,38,40,-1,-1,-1,42,44,46,48,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0129377E12,1.406168E3,2.326087E0,2.459517E0,6.234292E-1,1.5566E4,3.197772E-3,1.68128E8,1.4065148E6,2.3746382E-2,2.1928768E-3,-3.3603942E-3,-2.5489317E-2,-9.636486E-3,1.3105267E-5,1.6066E4,2.5696233E-1,-5.188022E-3,5.4116106E5,-1.3366703E-2,-4.9080625E-3,1.0455943E1,1.3257E4,3.3564355E0,4.398424E9,-7.4644303E-3,-1.5012765E-3,8.77E2,4.3350944E5,1.777E3,3.33231E5,3.7830765E-3,-6.7832153E-3,7.4405423E-3,1.9863014E0,5.33107E5,5.001725E5,6.5229E4,1.1758828E-2,1.0469056E2,-4.86464E-3,4.152712E-3,4.5758853E-4,1.546644E6,-4.4204467E-3,-1.6948156E-3,-6.152729E-3,6.1157714E-3,2.2633122E-3,5.6014425E-4,6.590244E-3,-4.581938E-4,3.5841241E-3],"split_indices":[40,61,62,67,36,9,0,7,59,0,0,0,0,0,51,9,50,0,42,0,0,66,9,65,40,0,0,0,42,2,38,0,0,0,62,9,56,38,0,61,0,0,0,37,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.9E2,2E1,1.79E2,1.1E1,1.3E1,7E0,8.7E1,9.2E1,5E0,6E0,6E0,7E0,5E0,8.2E1,1.1E1,8.1E1,5E0,7.7E1,6E0,5E0,1.7E1,6.4E1,6.6E1,1.1E1,9E0,8E0,1.9E1,4.5E1,4.5E1,2.1E1,5E0,6E0,7E0,1.2E1,3.4E1,1.1E1,4E1,5E0,1.6E1,5E0,5E0,7E0,2.9E1,5E0,5E0,6E0,2.5E1,1.5E1,7E0,9E0,2.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[5.686055E-3,2.1918137E-2,-1.617887E-1,1.2842843E-2,1.1972017E-2,-2.2409197E-2,-3.2337233E-2,-1.327504E-1,2.4578279E-2,-6.7981193E-3,1.7079021E-3,-3.8606557E-4,-1.0514475E-2,7.266771E-2,-2.4823729E-2,4.6628214E-2,1.6096558E-1,1.0162161E-2,-1.4359327E-1,3.0782916E-2,7.7057555E-3,1.3811482E-2,4.622432E-2,-1.4274719E-2,7.972256E-3,-1.0243393E-2,-4.98544E-4,4.445563E-2,-5.540727E-3,4.8347074E-3,-1.1215954E-3,-8.015244E-2,1.5994124E-2,-2.560489E-3,6.0061153E-2,-1.0953023E-1,3.25091E-4,4.8099536E-2,-7.362861E-2,5.0610458E-3,7.018089E-4,-2.2225245E-3,-7.815936E-3,4.3335385E-3,6.6229055E-4,-1.5912742E-3,-4.8057167E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,-1,-1,15,17,19,21,23,25,27,-1,-1,29,31,-1,-1,-1,33,-1,-1,-1,35,37,-1,39,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.152005E-1,4.1377062E-1,7.532463E-1,3.4327424E-1,0E0,0E0,1.12578385E-1,1.5253818E-1,4.4131106E-1,0E0,0E0,0E0,0E0,2.0862228E-1,3.8374296E-1,1.253215E-1,2.7992517E-1,2.7141246E-1,1.8265826E-1,1.3510776E-1,0E0,0E0,4.6000548E-2,1.2690663E-1,0E0,0E0,0E0,9.5330544E-2,0E0,0E0,0E0,5.17302E-2,1.3014601E-1,0E0,1.0383144E-1,4.113105E-2,0E0,4.504016E-2,8.967675E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,23,23,27,27,31,31,32,32,34,34,35,35,37,37,38,38],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,-1,-1,16,18,20,22,24,26,28,-1,-1,30,32,-1,-1,-1,34,-1,-1,-1,36,38,-1,40,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2E1,9.772E3,3.3E1,9.541066E5,1.1972017E-2,-2.2409197E-2,3E0,7.543738E1,7.34E2,-6.7981193E-3,1.7079021E-3,-3.8606557E-4,-1.0514475E-2,1.496E3,5.2726665E0,2.3082722E2,3.1665432E0,1.2241036E12,2.1835346E0,1.3927366E9,7.7057555E-3,1.3811482E-2,2.46604E5,3.86569E5,7.972256E-3,-1.0243393E-2,-4.98544E-4,1.4221454E0,-5.540727E-3,4.8347074E-3,-1.1215954E-3,1.4676493E9,2.395E3,-2.560489E-3,7.024907E1,6.791045E-1,3.25091E-4,8.893143E5,5.692891E-1,5.0610458E-3,7.018089E-4,-2.2225245E-3,-7.815936E-3,4.3335385E-3,6.6229055E-4,-1.5912742E-3,-4.8057167E-3],"split_indices":[3,2,3,54,0,0,8,64,0,0,0,0,0,2,65,61,65,40,63,5,0,0,38,1,0,0,0,62,0,0,0,5,0,0,61,65,0,56,36,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.24E2,2.05E2,1.9E1,1.98E2,7E0,5E0,1.4E1,1.4E1,1.84E2,5E0,9E0,6E0,8E0,9.3E1,9.1E1,7.3E1,2E1,7.1E1,2E1,6.5E1,8E0,9E0,1.1E1,6.2E1,9E0,1.3E1,7E0,6E1,5E0,6E0,5E0,1.9E1,4.3E1,8E0,5.2E1,1.4E1,5E0,3.2E1,1.1E1,2.6E1,2.6E1,7E0,7E0,1.4E1,1.8E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.1281592E-3,-2.0496879E-2,2.1598919E-1,-7.094735E-4,-1.9634551E-1,2.162715E-2,2.8600602E-3,9.261069E-3,-1.2700282E-2,-4.657455E-2,-2.3859587E-2,-1.9386686E-2,7.221406E-2,-3.963225E-3,3.5173685E-4,5.460904E-3,-1.4392802E-1,9.275428E-2,-3.1419073E-3,-1.0247414E-1,3.1641576E-2,-1.0071705E-2,-1.9806332E-3,1.1048649E-1,-1.7112321E-3,3.220437E-3,-1.7229548E-1,8.785332E-2,-2.392358E-2,8.109416E-2,1.2402337E-2,-1.1396639E-2,-3.0517497E-3,5.3130176E-2,1.11972885E-2,-9.6691225E-4,-6.132039E-3,1.1635751E-1,-1.0482289E-5,5.180586E-4,6.9109923E-3,2.122427E-3,-2.3496042E-3,1.822769E-3,7.264489E-3,-4.206991E-3,4.2060516E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,-1,-1,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,33,35,37,-1,-1,-1,39,-1,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.7368206E-1,6.1922735E-1,5.0905603E-1,4.1835395E-1,7.638674E-1,0E0,0E0,2.8274006E-1,0E0,2.4703717E-2,0E0,3.349976E-1,1.4333719E-1,0E0,0E0,2.6039135E-1,1.0727963E-1,1.0128543E-1,0E0,2.2294201E-1,2.3323442E-1,0E0,0E0,1.3941377E-1,0E0,0E0,7.2363466E-2,1.6926679E-1,8.8446215E-2,9.43819E-2,0E0,0E0,0E0,1.1486598E-1,0E0,6.899143E-2,0E0,5.3125948E-2,8.887175E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,15,15,16,16,17,17,19,19,20,20,23,23,26,26,27,27,28,28,29,29,33,33,35,35,37,37,38,38],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,-1,-1,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,34,36,38,-1,-1,-1,40,-1,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3908939E3,7.960309E7,4.083812E7,4.966421E6,9.479827E0,2.162715E-2,2.8600602E-3,2.326087E0,-1.2700282E-2,1.443266E6,-2.3859587E-2,2.1159E4,1.472353E0,-3.963225E-3,3.5173685E-4,7.23E2,7.476915E5,2.168E4,-3.1419073E-3,1E0,7.57E2,-1.0071705E-2,-1.9806332E-3,2.799E3,-1.7112321E-3,3.220437E-3,2.1152425E6,2.501667E6,5.1E1,2.3005404E0,1.2402337E-2,-1.1396639E-2,-3.0517497E-3,1E0,1.11972885E-2,3.300363E5,-6.132039E-3,4.761905E-2,2.3950179E0,5.180586E-4,6.9109923E-3,2.122427E-3,-2.3496042E-3,1.822769E-3,7.264489E-3,-4.206991E-3,4.2060516E-3],"split_indices":[61,54,54,37,66,0,0,62,0,38,0,9,50,0,0,2,56,10,0,25,0,0,0,2,0,0,54,41,8,65,0,0,0,18,0,42,0,65,63,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.91E2,1.77E2,1.4E1,1.6E2,1.7E1,5E0,9E0,1.55E2,5E0,1.2E1,5E0,1.07E2,4.8E1,7E0,5E0,9E1,1.7E1,4.2E1,6E0,1.7E1,7.3E1,1E1,7E0,3.7E1,5E0,5E0,1.2E1,3.6E1,3.7E1,3.2E1,5E0,7E0,5E0,3E1,6E0,3.1E1,6E0,2.2E1,1E1,2.1E1,9E0,1.6E1,1.5E1,7E0,1.5E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.3012133E-2,-3.9525285E-2,1.2617472E-1,-1.7720887E-2,-2.4498333E-1,1.7124562E-2,3.1462304E-2,7.002048E-2,-3.8192566E-2,-3.7074557E-1,2.315531E-3,8.7872125E-2,-3.84126E-3,-4.272341E-3,1.1957797E-1,-9.327309E-3,-3.068108E-2,-2.5415432E-2,-6.592802E-3,1.2941206E-1,4.908508E-4,1.5437049E-1,1.5004537E-3,-5.1211994E-3,-7.557736E-2,8.249079E-3,3.3479834E-3,2.4478638E-3,9.465726E-3,-5.4423674E-3,9.499643E-3,3.946655E-3,-9.660996E-2,-1.2019656E-2,6.335789E-2,-1.7325725E-1,-4.572663E-2,1.2241742E-2,-9.593291E-2,9.372243E-2,-1.3771409E-3,-9.500572E-4,-1.18000815E-2,-7.640166E-2,-1.3976486E-2,1.5805515E-3,-5.558094E-3,-6.118774E-4,-7.5141755E-3,2.036738E-3,5.9041386E-3,-7.633293E-4,-6.1722053E-3,3.22238E-4,-2.0774242E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,-1,-1,21,-1,23,-1,-1,25,-1,27,-1,29,31,-1,-1,-1,-1,-1,33,-1,35,37,39,41,43,45,47,49,-1,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.836366E-1,7.930787E-1,7.108339E-1,2.936608E-1,6.4260566E-1,0E0,1.6209976E-1,2.5230092E-1,1.454783E-1,3.5219514E-1,0E0,5.3662464E-2,0E0,0E0,6.947905E-2,0E0,1.4535102E-1,0E0,0E0,1.4809027E-2,0E0,5.9946567E-2,0E0,1.2840585E-1,1.590389E-1,0E0,0E0,0E0,0E0,0E0,8.536793E-2,0E0,1.519804E-1,1.0907459E-1,6.1006248E-2,1.6303787E-1,2.4254523E-2,1.0997014E-1,5.592613E-2,1.7509297E-2,0E0,0E0,0E0,3.7436403E-2,8.494069E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,14,14,16,16,19,19,21,21,23,23,24,24,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,43,43,44,44],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,-1,-1,22,-1,24,-1,-1,26,-1,28,-1,30,32,-1,-1,-1,-1,-1,34,-1,36,38,40,42,44,46,48,50,-1,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,9.7155967E11,1E0,2.94E2,4.0530096E10,1.7124562E-2,5.2587685E1,9.23253E1,6.56E2,2.095716E0,2.315531E-3,3.2916805E6,-3.84126E-3,-4.272341E-3,4.031725E6,-9.327309E-3,2.294901E0,-2.5415432E-2,-6.592802E-3,1.5140376E3,4.908508E-4,1.5322963E-3,1.5004537E-3,2.5256923E5,7.9209E4,8.249079E-3,3.3479834E-3,2.4478638E-3,9.465726E-3,-5.4423674E-3,2.173763E0,3.946655E-3,1.9851093E2,2E1,1.3528182E-1,2.980289E-1,1E0,2.0428267E0,1E0,1.8508342E5,-1.3771409E-3,-9.500572E-4,-1.18000815E-2,6.801082E2,3.546902E-2,1.5805515E-3,-5.558094E-3,-6.118774E-4,-7.5141755E-3,2.036738E-3,5.9041386E-3,-7.633293E-4,-6.1722053E-3,3.22238E-4,-2.0774242E-3],"split_indices":[61,40,13,0,5,0,66,4,2,63,0,41,0,0,12,0,65,0,0,4,0,48,0,37,38,0,0,0,0,0,63,0,61,8,47,36,13,63,18,41,0,0,0,4,47,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.1E2,1.77E2,3.3E1,1.61E2,1.6E1,9E0,2.4E1,3E1,1.31E2,1.1E1,5E0,1.6E1,8E0,7E0,2.3E1,5E0,1.26E2,6E0,5E0,1E1,6E0,1.6E1,7E0,8.1E1,4.5E1,5E0,5E0,5E0,1.1E1,9E0,7.2E1,5E0,4E1,5.2E1,2E1,1.5E1,2.5E1,4.1E1,1.1E1,1.5E1,5E0,5E0,1E1,1.2E1,1.3E1,3.6E1,5E0,5E0,6E0,6E0,9E0,6E0,6E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.3837313E-2,-3.0209785E-2,2.1168783E-1,-1.4708077E-2,-2.2625236E-1,2.3482488E-2,4.408334E-4,-5.1762694E-3,-1.4110658E-2,-1.4725437E-3,-2.359606E-2,-2.841783E-2,8.1648365E-2,-1.5009186E-2,-9.705018E-3,2.3062615E-2,9.863028E-3,5.0499015E-2,-4.9513422E-2,8.362105E-2,-3.4817547E-2,-6.003291E-3,8.0312125E-2,-1.6527607E-1,-3.0072E-2,1.1058209E-3,5.8126645E-3,-5.707869E-3,3.1195106E-3,1.07214965E-1,2.085347E-3,-3.094195E-3,-1.1197044E-2,7.258363E-2,-5.7401117E-2,3.7150236E-3,8.218233E-3,2.0472927E-3,-1.8604177E-3,2.2775827E-3,5.2854796E-3,-4.408215E-3,3.038044E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,13,15,17,-1,19,-1,21,23,25,27,-1,29,31,33,-1,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.6333785E-1,5.8015925E-1,7.5813186E-1,4.6595335E-1,6.8969715E-1,0E0,0E0,3.534418E-1,0E0,0E0,0E0,3.1182677E-1,2.557528E-1,2.940804E-1,0E0,9.404491E-2,0E0,2.4238233E-1,1.8632999E-1,2.5298342E-2,1.20626315E-1,0E0,8.185282E-2,6.356561E-2,2.1177247E-1,0E0,0E0,0E0,0E0,4.235944E-2,1.9169552E-2,0E0,0E0,9.742878E-3,1.2360783E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,22,22,23,23,24,24,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,14,16,18,-1,20,-1,22,24,26,28,-1,30,32,34,-1,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3402396E3,1.0388931E8,4.27928E5,4.476758E6,2.9119E4,2.3482488E-2,4.408334E-4,1.546644E6,-1.4110658E-2,-1.4725437E-3,-2.359606E-2,1.2727361E6,4.1822775E2,1.108055E0,-9.705018E-3,7.11E2,9.863028E-3,8.53386E3,4.135593E1,2.5823686E-2,2.7319932E6,-6.003291E-3,6.805556E5,3.461353E1,1.6686391E0,1.1058209E-3,5.8126645E-3,-5.707869E-3,3.1195106E-3,3.9401126E2,2.0261486E8,-3.094195E-3,-1.1197044E-2,5.5719394E5,3.8739338E1,3.7150236E-3,8.218233E-3,2.0472927E-3,-1.8604177E-3,2.2775827E-3,5.2854796E-3,-4.408215E-3,3.038044E-4],"split_indices":[61,54,38,37,9,0,0,37,0,0,0,37,61,67,0,0,0,42,61,66,37,0,37,61,62,0,0,0,0,4,7,0,0,56,67,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.91E2,1.3E1,1.78E2,1.3E1,5E0,8E0,1.73E2,5E0,8E0,5E0,1.37E2,3.6E1,1.28E2,9E0,2.5E1,1.1E1,4.4E1,8.4E1,1.2E1,1.3E1,6E0,3.8E1,1.1E1,7.3E1,5E0,7E0,7E0,6E0,2.8E1,1E1,5E0,6E0,1.5E1,5.8E1,2E1,8E0,5E0,5E0,1E1,5E0,3.8E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-3.3833364E-3,-1.4876087E-2,1.4250561E-2,-7.3089707E-3,-1.2935327E-2,4.028505E-3,-1.3794172E-1,2.0075222E-2,-6.294472E-2,-1.898366E-2,-1.6279079E-2,-1.018485E-1,3.1415317E-2,3.4179222E-3,-9.742783E-2,-4.490363E-3,2.7891714E-3,-4.9197994E-4,-8.753549E-3,8.464311E-3,7.29291E-2,-1.4800349E-1,-1.3445352E-2,-4.010771E-3,6.688517E-3,1.0571275E-1,1.0595441E-2,-1.951625E-1,-1.0342237E-3,9.749194E-4,-1.9622906E-3,9.180636E-2,-1.8394027E-2,8.232933E-2,7.279383E-3,4.6704803E-3,-5.2152753E-2,-5.364839E-3,-1.1478152E-2,6.999977E-3,1.2271092E-3,1.1812466E-3,-2.5515985E-3,6.557099E-4,5.457024E-3,-5.1706987E-3,4.971097E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,-1,5,-1,7,9,11,13,15,-1,17,19,-1,21,-1,-1,-1,-1,23,25,27,29,31,-1,33,35,37,-1,-1,-1,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.229101E-1,3.8550493E-1,0E0,2.9676485E-1,0E0,2.000832E-1,3.650162E-1,2.105936E-1,1.7175557E-1,6.620689E-2,0E0,9.035157E-2,1.3086674E-1,0E0,1.21186644E-1,0E0,0E0,0E0,0E0,1.4467824E-1,9.901264E-2,1.0354471E-1,1.14988675E-2,1.1607616E-1,0E0,2.00724E-2,1.00936346E-1,1.8261552E-2,0E0,0E0,0E0,3.4118697E-2,1.0629694E-1,4.4962585E-2,0E0,0E0,3.784229E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,31,31,32,32,33,33,36,36],"right_children":[2,4,-1,6,-1,8,10,12,14,16,-1,18,20,-1,22,-1,-1,-1,-1,24,26,28,30,32,-1,34,36,38,-1,-1,-1,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.434E3,4.476758E6,1.4250561E-2,1.0388931E8,-1.2935327E-2,4.7564573E0,1.0129377E12,1.1744206E6,4.86E2,3.1805055E0,-1.6279079E-2,4.5341E4,2.1843657E0,3.4179222E-3,9.729738E6,-4.490363E-3,2.7891714E-3,-4.9197994E-4,-8.753549E-3,1E0,1.5544465E0,1.3E1,2.1844962E0,1.2414E4,6.688517E-3,4.541294E2,2.326087E0,4.0830806E-2,-1.0342237E-3,9.749194E-4,-1.9622906E-3,6.15175E5,7.07E2,3.2647732E-3,7.279383E-3,4.6704803E-3,2E0,-5.364839E-3,-1.1478152E-2,6.999977E-3,1.2271092E-3,1.1812466E-3,-2.5515985E-3,6.557099E-4,5.457024E-3,-5.1706987E-3,4.971097E-4],"split_indices":[2,37,0,54,0,65,40,54,0,65,0,1,62,0,60,0,0,0,0,112,65,3,62,9,0,4,62,48,0,0,0,1,0,51,0,0,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.04E2,7E0,1.99E2,5E0,1.84E2,1.5E1,1.49E2,3.5E1,1E1,5E0,1.2E1,1.37E2,7E0,2.8E1,5E0,5E0,6E0,6E0,8.9E1,4.8E1,1.7E1,1.1E1,8.2E1,7E0,3.1E1,1.7E1,1.2E1,5E0,5E0,6E0,1E1,7.2E1,2.2E1,9E0,7E0,1E1,5E0,7E0,5E0,5E0,3.2E1,4E1,7E0,1.5E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.4636147E-2,4.8411472E-4,-1.9232947E-1,-1.5243398E-2,8.113542E-2,-7.094381E-2,-1.8712023E-2,-5.729771E-3,-7.6459907E-3,2.271111E-1,8.0837857E-4,-5.404797E-3,-9.5272454E-4,-2.4134172E-2,7.1893215E-2,3.6796588E-3,1.6672555E-2,4.818885E-3,-4.4875015E-2,-1.5123193E-2,-9.5719835E-3,9.7935395E-3,3.6781333E-2,2.2210919E-3,-4.622896E-3,-1.4915803E-3,-9.00142E-2,8.34565E-2,-2.51925E-3,-8.4606014E-2,1.53469555E-2,-2.8333634E-2,-8.939831E-3,5.574628E-3,1.9041987E-3,9.929271E-4,-1.7767199E-3,-7.0946547E-3,-7.169371E-4,-4.8676287E-3,2.7088428E-2,1.2139792E-3,-4.1862093E-3,-2.1511034E-4,3.495299E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,-1,19,21,-1,-1,-1,23,25,-1,-1,27,-1,-1,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,43,-1,-1,-1,-1],"loss_changes":[5.481941E-1,2.4097277E-1,3.4279692E-1,2.126621E-1,3.63316E-1,2.0273935E-2,0E0,2.1602577E-1,0E0,1.6468644E-1,9.822701E-2,0E0,0E0,1.8578592E-1,1.2343964E-1,0E0,0E0,0E0,7.1628794E-2,1.193093E-1,0E0,0E0,4.423492E-2,0E0,0E0,1.4125098E-1,9.793684E-2,1.053118E-2,1.1552208E-2,6.984842E-2,1.1605118E-1,3.8691983E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.08016595E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,13,13,14,14,18,18,19,19,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,40,40],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,-1,20,22,-1,-1,-1,24,26,-1,-1,28,-1,-1,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,44,-1,-1,-1,-1],"split_conditions":[2.4E1,7.737766E2,1E0,5.217E4,2.46604E5,3E0,-1.8712023E-2,2.128E3,-7.6459907E-3,1E0,1.14416916E2,-5.404797E-3,-9.5272454E-4,4.17027E6,2.7675075E8,3.6796588E-3,1.6672555E-2,4.818885E-3,1.4740609E1,3.9183674E0,-9.5719835E-3,9.7935395E-3,3.824118E7,2.2210919E-3,-4.622896E-3,2.1152425E6,2.75639E5,7.785208E2,4.3318925E8,2.6727284E5,7E0,7.686371E5,-8.939831E-3,5.574628E-3,1.9041987E-3,9.929271E-4,-1.7767199E-3,-7.0946547E-3,-7.169371E-4,-4.8676287E-3,2.1723518E0,1.2139792E-3,-4.1862093E-3,-2.1511034E-4,3.495299E-3],"split_indices":[3,61,24,10,38,8,0,2,0,8,67,0,0,37,7,0,0,0,66,65,0,0,54,0,0,54,38,4,7,42,3,41,0,0,0,0,0,0,0,0,62,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.88E2,1.5E1,1.58E2,3E1,1E1,5E0,1.49E2,9E0,1E1,2E1,5E0,5E0,1.21E2,2.8E1,5E0,5E0,6E0,1.4E1,1.16E2,5E0,5E0,2.3E1,5E0,9E0,9.9E1,1.7E1,1E1,1.3E1,1.6E1,8.3E1,1.1E1,6E0,5E0,5E0,8E0,5E0,8E0,8E0,7E0,7.6E1,6E0,5E0,4.5E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-3.017712E-3,-2.6434649E-2,1.3926433E-1,-7.8038746E-3,-2.184324E-1,1.6295437E-2,4.495092E-2,5.4144062E-2,-3.5012104E-2,-1.3289434E-3,-2.4477575E-2,5.373098E-3,-1.5159994E-3,1.5253884E-1,1.710861E-2,-9.202811E-2,8.079396E-3,3.0987435E-3,1.2023247E-2,4.2247612E-2,-3.7226502E-3,3.4156892E-2,-1.4659621E-1,2.6924131E-2,-7.162143E-3,-2.4386117E-2,9.7911656E-2,-4.994621E-4,3.1711229E-3,-1.327041E-2,-1.0252929E-1,4.3492906E-2,-3.239537E-3,-3.1337377E-3,6.56029E-4,2.3008655E-3,6.04688E-3,-1.4741763E-2,-1.6592401E-1,2.6126796E-2,5.308769E-3,1.1883241E-3,-2.253841E-3,-4.6499823E-3,-1.1400834E-2,5.0406314E-2,-3.0844964E-3,-5.4782344E-4,4.2982684E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,-1,25,-1,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,-1,-1,-1,-1,41,43,45,-1,-1,-1,-1,-1,47,-1,-1,-1],"loss_changes":[6.6004795E-1,6.063967E-1,4.9761933E-1,2.648327E-1,7.963123E-1,0E0,1.0212379E-1,1.7230684E-1,2.6854622E-1,0E0,0E0,0E0,0E0,9.2686355E-2,8.762497E-2,3.2863393E-1,1.8749398E-1,0E0,0E0,1.0995167E-1,0E0,2.111003E-2,1.5378481E-1,9.090182E-2,0E0,2.188678E-2,1.5161023E-2,0E0,0E0,0E0,1.422509E-1,5.2571453E-2,0E0,0E0,0E0,0E0,0E0,1.5820986E-2,4.9295753E-2,8.98394E-2,0E0,0E0,0E0,0E0,0E0,7.497861E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,19,19,21,21,22,22,23,23,25,25,26,26,30,30,31,31,37,37,38,38,39,39,45,45],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,-1,26,-1,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,-1,-1,-1,-1,42,44,46,-1,-1,-1,-1,-1,48,-1,-1,-1],"split_conditions":[7.737766E2,7.713233E7,2.13281E5,2.1203648E8,6.540767E0,1.6295437E-2,5.287008E5,2.585911E5,1.06199684E2,-1.3289434E-3,-2.4477575E-2,5.373098E-3,-1.5159994E-3,1.127E3,3.3564355E0,7.2419E4,4.8995E4,3.0987435E-3,1.2023247E-2,4.2905E4,-3.7226502E-3,5.18914E5,1.4773677E5,6.239726E0,-7.162143E-3,1.8723404E0,1.1042428E2,-4.994621E-4,3.1711229E-3,-1.327041E-2,6.3228765E6,2.3829322E0,-3.239537E-3,-3.1337377E-3,6.56029E-4,2.3008655E-3,6.04688E-3,2.4837341E0,4.884925E5,1.9231517E0,5.308769E-3,1.1883241E-3,-2.253841E-3,-4.6499823E-3,-1.1400834E-2,1.7729497E2,-3.0844964E-3,-5.4782344E-4,4.2982684E-3],"split_indices":[61,54,38,7,66,0,41,37,61,0,0,0,0,2,65,1,10,0,0,38,0,60,42,65,0,62,61,0,0,0,54,62,0,0,0,0,0,67,42,63,0,0,0,0,0,61,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.96E2,1.69E2,2.7E1,1.55E2,1.4E1,8E0,1.9E1,4.7E1,1.08E2,9E0,5E0,1E1,9E0,1.2E1,3.5E1,4.6E1,6.2E1,7E0,5E0,2.8E1,7E0,1.4E1,3.2E1,5.6E1,6E0,1.3E1,1.5E1,6E0,8E0,7E0,2.5E1,4.8E1,8E0,6E0,7E0,6E0,9E0,1.1E1,1.4E1,3.9E1,9E0,5E0,6E0,8E0,6E0,3.1E1,8E0,1.2E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[4.200003E-3,-1.0830721E-2,1.7209372E-1,2.9232295E-3,-2.0248346E-1,1.982164E-2,2.912774E-2,-3.7934523E-2,3.3961695E-2,1.0377131E-3,-1.7856453E-2,-1.231639E-3,3.8418768E-3,2.1101786E-2,-1.06273286E-1,6.7275693E-3,9.986144E-2,-4.591234E-3,5.752764E-2,-1.5747613E-1,1.5103223E-2,4.7273573E-2,-3.1752758E-2,1.2323454E-3,1.284608E-1,7.6085135E-2,-1.5874346E-3,-7.689571E-2,-2.4571702E-1,3.3885054E-3,-2.0350532E-3,9.781059E-3,6.661254E-3,-1.1890712E-2,-4.877428E-3,1.690263E-1,2.7553298E-4,1.07750036E-1,2.1025391E-4,-8.016173E-3,4.3648246E-4,-1.4550202E-2,-7.469353E-3,7.214022E-2,-4.4368297E-2,2.4949252E-3,-2.7128385E-2,4.588357E-3,1.0201308E-2,7.5620553E-3,3.1187723E-3,5.355567E-3,1.4544612E-3,6.573698E-4,-4.3704687E-3,-5.6231376E-2,6.660556E-3,-9.561769E-4,-4.1483543E-3,-1.0180915E-3,1.8389103E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,17,19,21,23,-1,25,27,29,31,33,-1,35,37,-1,39,41,-1,-1,43,-1,45,-1,47,-1,49,-1,-1,-1,-1,-1,51,53,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1],"loss_changes":[4.9143532E-1,4.7243273E-1,5.221307E-1,2.1433952E-1,4.4487685E-1,0E0,3.154259E-2,2.9675874E-1,1.716215E-1,0E0,0E0,0E0,0E0,1.730066E-1,2.1368235E-1,1.09156474E-1,5.6539506E-2,0E0,5.4248504E-2,1.5175718E-1,3.672489E-2,1.1212443E-1,4.7457818E-2,0E0,9.869546E-2,5.863516E-2,0E0,1.0612111E-1,7.609248E-3,0E0,0E0,8.761845E-2,0E0,2.9194068E-2,0E0,2.5136292E-2,0E0,2.7272284E-2,0E0,0E0,0E0,0E0,0E0,1.5774813E-2,3.7662562E-2,0E0,2.3975156E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1991087E-2,1.0996359E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,31,31,33,33,35,35,37,37,43,43,44,44,46,46,55,55,56,56],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,18,20,22,24,-1,26,28,30,32,34,-1,36,38,-1,40,42,-1,-1,44,-1,46,-1,48,-1,50,-1,-1,-1,-1,-1,52,54,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1],"split_conditions":[6.811E3,1.0388931E8,9E0,1.06199684E2,1E0,1.982164E-2,1.2047E4,7.3328E4,1.546644E6,1.0377131E-3,-1.7856453E-2,-1.231639E-3,3.8418768E-3,1.4221454E0,2.170343E0,1.1956835E11,8.3299077E-1,-4.591234E-3,3.237835E-1,1.6999155E0,9.314573E8,1E1,6.3018E4,1.2323454E-3,1.7393E4,8.595345E-1,-1.5874346E-3,6.5674934E1,1.623E4,3.3885054E-3,-2.0350532E-3,1.6448E4,6.661254E-3,1.9167768E0,-4.877428E-3,9.0770394E-1,2.7553298E-4,5.971677E-3,2.1025391E-4,-8.016173E-3,4.3648246E-4,-1.4550202E-2,-7.469353E-3,5.549974E8,7.686371E5,2.4949252E-3,2.1564245E0,4.588357E-3,1.0201308E-2,7.5620553E-3,3.1187723E-3,5.355567E-3,1.4544612E-3,6.573698E-4,-4.3704687E-3,1.5768844E9,1E0,-9.561769E-4,-4.1483543E-3,-1.0180915E-3,1.8389103E-3],"split_indices":[2,54,3,61,8,0,9,1,37,0,0,0,0,62,63,40,65,0,47,63,5,3,10,0,9,36,0,61,9,0,0,9,0,62,0,36,0,66,0,0,0,0,0,5,41,0,63,0,0,0,0,0,0,0,0,12,24,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.93E2,1.78E2,1.5E1,1.67E2,1.1E1,5E0,1E1,7.2E1,9.5E1,5E0,6E0,5E0,5E0,3.9E1,3.3E1,6.8E1,2.7E1,9E0,3E1,2.3E1,1E1,3.3E1,3.5E1,8E0,1.9E1,2.5E1,5E0,1.3E1,1E1,5E0,5E0,2.4E1,9E0,2.8E1,7E0,1.4E1,5E0,1.7E1,8E0,6E0,7E0,5E0,5E0,1.1E1,1.3E1,5E0,2.3E1,6E0,8E0,7E0,1E1,5E0,6E0,6E0,7E0,1.2E1,1.1E1,6E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[7.127429E-3,3.1775493E-2,-8.559813E-2,1.7054535E-2,1.9353934E-1,-2.5171076E-3,-2.3602517E-1,-2.412755E-3,6.6662475E-2,1.7383242E-2,-4.001022E-3,-7.4389935E-2,4.9530886E-2,-7.221461E-2,-2.4293737E-2,1.1248308E-2,-7.513772E-2,-5.5276154E-4,8.430349E-2,1.7638982E-3,-7.231725E-3,4.5182853E-4,3.9512864E-3,-4.81081E-5,-6.4232936E-3,-9.572982E-3,1.02005735E-1,4.7234778E-4,-1.2998056E-1,1.1204295E-1,1.9487659E-2,7.6314755E-2,-2.6094887E-2,6.023532E-3,1.9071582E-3,-8.008759E-3,-3.6392498E-3,2.245051E-3,7.1541364E-3,2.307409E-3,-3.448221E-4,-2.1984396E-4,8.375379E-3,-6.333872E-2,3.1427037E-2,6.603215E-4,-7.689858E-2,5.354051E-2,-2.6832E-3,-4.712946E-3,-4.7875117E-4,2.393798E-4,3.4937342E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,-1,29,-1,-1,-1,-1,-1,-1,31,33,-1,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,45,47,-1,49,51,-1,-1,-1,-1,-1],"loss_changes":[4.895326E-1,3.9914215E-1,5.6179935E-1,1.5081504E-1,6.540526E-1,1.15909696E-1,6.481341E-1,1.1311071E-1,6.14748E-2,0E0,0E0,1.1098276E-1,2.1954581E-2,4.6251703E-2,0E0,1.8238643E-1,8.4969275E-2,0E0,6.2542945E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.1409473E-1,1.9256786E-2,0E0,8.485988E-3,4.9999535E-2,9.161292E-3,1.0060619E-1,1.4552712E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.376267E-2,5.4477908E-2,0E0,4.4901118E-2,1.8843256E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,18,18,25,25,26,26,28,28,29,29,30,30,31,31,32,32,43,43,44,44,46,46,47,47],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,-1,30,-1,-1,-1,-1,-1,-1,32,34,-1,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,46,48,-1,50,52,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,5.666E3,3.0711978E5,1E0,3.93672E6,2.2802818E0,9.150449E5,4.291834E0,6.71E2,1.7383242E-2,-4.001022E-3,8.3636254E-1,6.344626E2,1.5678E4,-2.4293737E-2,2.126386E0,7.381306E-2,-5.5276154E-4,4.2993274E8,1.7638982E-3,-7.231725E-3,4.5182853E-4,3.9512864E-3,-4.81081E-5,-6.4232936E-3,1.3846154E0,2.4274447E0,4.7234778E-4,6.15175E5,2.942161E5,8.3735475E5,1.4615384E0,1.23844246E2,6.023532E-3,1.9071582E-3,-8.008759E-3,-3.6392498E-3,2.245051E-3,7.1541364E-3,2.307409E-3,-3.448221E-4,-2.1984396E-4,8.375379E-3,2.3242424E1,4.4101E4,6.603215E-4,2.8003845E2,5.936784E-1,-2.6832E-3,-4.712946E-3,-4.7875117E-4,2.393798E-4,3.4937342E-3],"split_indices":[54,2,42,22,1,62,37,65,2,0,0,36,61,9,0,63,36,0,7,0,0,0,0,0,0,63,63,0,1,41,41,62,61,0,0,0,0,0,0,0,0,0,0,61,10,0,4,36,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.68E2,4.4E1,1.55E2,1.3E1,2.9E1,1.5E1,1.12E2,4.3E1,8E0,5E0,1.2E1,1.7E1,1E1,5E0,9.5E1,1.7E1,8E0,3.5E1,5E0,7E0,8E0,9E0,5E0,5E0,7.8E1,1.7E1,7E0,1E1,2.4E1,1.1E1,1.2E1,6.6E1,1.2E1,5E0,5E0,5E0,9E0,1.5E1,5E0,6E0,7E0,5E0,4E1,2.6E1,6E0,3.4E1,2.1E1,5E0,2.6E1,8E0,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.5085545E-2,-1.0247435E-4,1.3461311E-1,-7.805591E-2,1.0003233E-2,1.8675208E-2,2.4804063E-2,-2.1850498E-2,-7.359806E-3,3.622174E-2,-3.517192E-2,-2.7005391E-2,6.0752537E-3,-4.954181E-3,1.980048E-3,1.7180014E-2,1.5604556E-1,5.1397104E-3,-5.5688877E-2,1.1768076E-3,-4.012993E-3,3.469668E-2,-7.271643E-2,4.818244E-3,9.748473E-3,-8.930088E-3,-3.8653072E-2,1.3749432E-3,6.978243E-2,-1.0951993E-3,-5.0721522E-3,1.9188158E-2,-7.099143E-2,5.196248E-3,-2.81799E-2,1.3374843E-1,2.0089777E-2,-2.1220928E-3,4.233718E-3,-3.9940044E-2,-7.58969E-3,1.1102274E-2,-6.349372E-2,8.49833E-3,4.0683877E-3,5.059707E-2,-2.5667488E-3,-2.4565784E-4,-6.827563E-2,1.6974978E-3,-1.1845856E-3,1.7188694E-3,-6.651178E-3,4.546422E-3,-1.683479E-4,1.6950353E-3,-1.7170495E-3,-5.356764E-3,-1.1531379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,-1,-1,-1,21,23,-1,25,-1,-1,27,29,-1,-1,-1,31,33,35,-1,-1,37,39,-1,41,43,45,-1,-1,47,-1,49,51,-1,-1,53,-1,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7035275E-1,1.4415582E-1,6.1548156E-1,8.229774E-2,1.9315083E-1,0E0,9.097886E-2,6.8556145E-2,0E0,2.3232931E-1,1.785936E-1,3.574712E-2,0E0,0E0,0E0,1.4424475E-1,1.5254229E-2,0E0,1.0761081E-1,0E0,0E0,8.875703E-2,2.1107815E-2,0E0,0E0,0E0,9.114573E-2,1.270446E-1,1.149227E-1,0E0,0E0,7.973913E-2,7.4925974E-2,0E0,4.5095436E-2,1.7950565E-2,5.1383782E-2,0E0,0E0,2.697755E-2,0E0,1.4199226E-2,1.2651208E-1,0E0,0E0,3.6994588E-2,0E0,1.4618253E-2,2.3260161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,15,15,16,16,18,18,21,21,22,22,26,26,27,27,28,28,31,31,32,32,34,34,35,35,36,36,39,39,41,41,42,42,45,45,47,47,48,48],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,-1,-1,-1,22,24,-1,26,-1,-1,28,30,-1,-1,-1,32,34,36,-1,-1,38,40,-1,42,44,46,-1,-1,48,-1,50,52,-1,-1,54,-1,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.474E3,2.377416E6,1.5190906E9,4.5585E4,2.2448614E0,1.8675208E-2,2.7E1,4.75E2,-7.359806E-3,2.6056657E0,7.2591E4,3.945346E1,6.0752537E-3,-4.954181E-3,1.980048E-3,1.7315E4,2.758609E2,5.1397104E-3,2.0973E4,1.1768076E-3,-4.012993E-3,3.200388E2,6.74E2,4.818244E-3,9.748473E-3,-8.930088E-3,1.7476E4,1.2627974E8,1.36623E5,-1.0951993E-3,-5.0721522E-3,1.6362E4,8.988928E5,5.196248E-3,5.088063E-1,1.3176E4,6.1061732E7,-2.1220928E-3,4.233718E-3,8.2841175E5,-7.58969E-3,2.087324E0,5.0756625E6,8.49833E-3,4.0683877E-3,1.813E3,-2.5667488E-3,3.9553946E-1,7.476915E5,1.6974978E-3,-1.1845856E-3,1.7188694E-3,-6.651178E-3,4.546422E-3,-1.683479E-4,1.6950353E-3,-1.7170495E-3,-5.356764E-3,-1.1531379E-3],"split_indices":[2,54,12,1,65,0,8,2,0,62,38,66,0,0,0,9,4,0,12,0,0,4,0,0,0,0,9,5,38,0,0,9,41,0,65,9,54,0,0,57,0,63,54,0,0,0,0,36,56,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.03E2,1.81E2,2.2E1,2E1,1.61E2,6E0,1.6E1,1.2E1,8E0,1.02E2,5.9E1,1.1E1,5E0,5E0,7E0,8.9E1,1.3E1,7E0,5.2E1,6E0,5E0,7.5E1,1.4E1,7E0,6E0,5E0,4.7E1,3.9E1,3.6E1,6E0,8E0,1.7E1,3E1,8E0,3.1E1,1.5E1,2.1E1,9E0,8E0,2.3E1,7E0,1.5E1,1.6E1,7E0,8E0,1.5E1,6E0,1E1,1.3E1,9E0,6E0,7E0,9E0,8E0,7E0,5E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[4.1982117E-3,2.0554883E-2,-8.896155E-2,9.097273E-3,1.1720698E-2,-2.4552247E-1,1.7251126E-2,-8.07497E-3,1.739235E-2,-1.8028715E-3,-1.8149726E-2,-2.5104368E-3,4.3545146E-2,5.2370097E-2,-1.3114944E-2,-7.992496E-4,4.169122E-3,6.2060706E-2,-2.6779526E-3,-1.1162676E-1,4.966619E-3,2.9280886E-2,1.3522935E-1,-1.6261502E-3,-8.125686E-3,6.682743E-2,-1.7911341E-2,5.8035728E-2,-1.4561067E-2,1.584609E-1,2.2909301E-3,6.0294135E-3,4.2284955E-3,-2.9568288E-2,4.2086123E-3,5.5563273E-3,3.0108232E-2,-6.165483E-2,4.3226746E-3,3.4039498E-3,1.0444327E-2,3.300008E-3,-2.921078E-3,-5.0751816E-2,1.7030254E-2,-2.2953148E-3,2.7146544E-3,-5.3712446E-3,4.4797297E-4,2.0430225E-3,-3.2762976E-3,-1.4331789E-3,1.8958795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,-1,-1,15,17,19,-1,-1,21,-1,23,25,27,29,-1,-1,31,33,35,37,39,-1,-1,41,43,-1,-1,45,47,-1,-1,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2639453E-1,4.5497543E-1,5.363557E-1,2.544435E-1,0E0,3.2748258E-1,3.8317215E-2,0E0,1.7888859E-1,0E0,0E0,0E0,3.8902707E-2,8.439602E-2,1.6086668E-1,0E0,0E0,1.6872907E-1,0E0,5.4378346E-2,1.1017115E-1,6.513916E-2,3.7734002E-2,0E0,0E0,7.378323E-2,7.1656294E-2,4.4745304E-2,1.07311614E-1,6.848827E-2,0E0,0E0,4.8578426E-2,5.2194636E-2,0E0,0E0,4.572141E-2,5.224292E-2,0E0,0E0,0E0,0E0,0E0,5.702713E-2,1.8280152E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,12,12,13,13,14,14,17,17,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,32,32,33,33,36,36,37,37,43,43,44,44],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,-1,-1,16,18,20,-1,-1,22,-1,24,26,28,30,-1,-1,32,34,36,38,40,-1,-1,42,44,-1,-1,46,48,-1,-1,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2898104E7,6.811E3,2.126386E0,4.26E2,1.1720698E-2,2E0,8.790647E7,-8.07497E-3,1.933162E0,-1.8028715E-3,-1.8149726E-2,-2.5104368E-3,1.1334E4,8.613281E0,1.4065148E6,-7.992496E-4,4.169122E-3,4.0972165E2,-2.6779526E-3,1.7070305E5,1.3257E4,1.9413766E8,1.06748E6,-1.6261502E-3,-8.125686E-3,9.537158E10,4.0048194E0,2.7958122E8,1.8048836E7,4.743991E5,2.2909301E-3,6.0294135E-3,1E0,1.6259075E1,4.2086123E-3,5.5563273E-3,1.4654956E2,8.4040934E-1,4.3226746E-3,3.4039498E-3,1.0444327E-2,3.300008E-3,-2.921078E-3,4.777882E5,2.8766456E5,-2.2953148E-3,2.7146544E-3,-5.3712446E-3,4.4797297E-4,2.0430225E-3,-3.2762976E-3,-1.4331789E-3,1.8958795E-3],"split_indices":[54,2,63,2,0,8,54,0,67,0,0,0,9,65,59,0,0,4,0,56,9,7,1,0,0,40,63,5,12,37,0,0,13,66,0,0,4,36,0,0,0,0,0,57,42,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.81E2,3.1E1,1.73E2,8E0,1.2E1,1.9E1,7E0,1.66E2,5E0,7E0,5E0,1.4E1,7.7E1,8.9E1,6E0,8E0,7.1E1,6E0,1.3E1,7.6E1,5E1,2.1E1,6E0,7E0,2E1,5.6E1,3E1,2E1,1.6E1,5E0,1E1,1E1,5.1E1,5E0,9E0,2.1E1,1.4E1,6E0,7E0,9E0,5E0,5E0,3.5E1,1.6E1,5E0,1.6E1,8E0,6E0,5E0,3E1,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.5447753E-5,1.8455232E-2,-1.3496439E-1,6.5168594E-3,1.2148637E-2,-1.7138083E-2,-1.285739E-2,-1.1397129E-1,1.5293329E-2,2.7701724E-3,-3.9572828E-3,-1.5785718E-4,-9.414852E-3,2.3602156E-2,-6.375213E-3,6.7343235E-2,4.1234964E-3,9.249967E-2,1.12491315E-2,-2.3007732E-2,5.542152E-2,1.1548517E-1,-4.5814973E-4,-9.681363E-4,2.6398709E-3,5.982845E-2,-3.8926195E-2,7.5581305E-2,-3.162259E-3,3.3741524E-3,1.536922E-1,6.365148E-3,-4.4255122E-4,3.982621E-3,-5.354254E-2,9.8775975E-2,-1.2064179E-3,9.181549E-3,4.531774E-3,6.3300243E-4,-3.8518005E-3,6.585149E-3,2.7263353E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,-1,-1,9,11,13,-1,-1,-1,-1,15,-1,17,19,21,23,25,27,29,-1,-1,-1,31,33,35,-1,-1,37,-1,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.177136E-1,5.010509E-1,6.5380293E-1,1.867986E-1,0E0,0E0,8.508524E-2,1.03882715E-1,2.0164187E-1,0E0,0E0,0E0,0E0,1.3263705E-1,0E0,6.6693574E-2,1.5297896E-1,7.8355014E-2,2.2315597E-2,9.726988E-2,9.6906975E-2,3.7314236E-2,0E0,0E0,0E0,5.9363473E-2,1.11713976E-1,7.9181224E-2,0E0,0E0,1.1736393E-2,0E0,0E0,0E0,9.339324E-2,3.4217417E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,26,26,27,27,30,30,34,34,35,35],"right_children":[2,4,6,8,-1,-1,10,12,14,-1,-1,-1,-1,16,-1,18,20,22,24,26,28,30,-1,-1,-1,32,34,36,-1,-1,38,-1,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3365464E7,1.3908939E3,3.248512E5,9.235376E5,1.2148637E-2,-1.7138083E-2,1.371407E6,9.09357E1,8.613281E0,2.7701724E-3,-3.9572828E-3,-1.5785718E-4,-9.414852E-3,1E0,-6.375213E-3,2.5492957E0,3.7597818E2,2.5831E4,1.1625969E6,1.6012292E-1,1.813E3,7.2419E4,-4.5814973E-4,-9.681363E-4,2.6398709E-3,7.1784E4,1.3176E4,1.8495E4,-3.162259E-3,3.3741524E-3,1.8E1,6.365148E-3,-4.4255122E-4,3.982621E-3,3.050899E4,3.178989E8,-1.2064179E-3,9.181549E-3,4.531774E-3,6.3300243E-4,-3.8518005E-3,6.585149E-3,2.7263353E-3],"split_indices":[54,61,41,54,0,0,38,4,65,0,0,0,0,25,0,65,64,10,37,36,0,1,0,0,0,1,9,9,0,0,8,0,0,0,57,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.06E2,1.82E2,2.4E1,1.74E2,8E0,8E0,1.6E1,1.1E1,1.63E2,8E0,8E0,5E0,6E0,1.55E2,8E0,4.7E1,1.08E2,3.2E1,1.5E1,7.1E1,3.7E1,2.6E1,6E0,9E0,6E0,1.1E1,6E1,3.2E1,5E0,1.3E1,1.3E1,5E0,6E0,6E0,5.4E1,2.6E1,6E0,7E0,6E0,1.5E1,3.9E1,1.3E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.7988496E-4,8.993286E-3,-5.6223474E-3,-3.2640636E-2,2.0410838E-2,1.7439321E-4,-7.796078E-2,5.9659056E-2,-3.665429E-2,-2.429905E-2,5.72694E-2,-2.6693983E-2,-1.3861564E-1,4.528261E-2,9.105974E-3,-8.13943E-2,3.000652E-2,8.97295E-3,-8.478685E-2,-5.380993E-4,1.017198E-1,2.2645278E-3,-4.8348688E-2,-1.0353298E-2,-6.612776E-2,3.334647E-2,5.1439856E-3,-1.19000554E-1,1.8417955E-3,-7.095255E-4,2.9161072E-3,-4.9913716E-2,4.446075E-2,-8.041561E-3,-1.3916446E-3,6.5413695E-3,2.5740946E-3,-8.466731E-2,2.1728605E-3,-6.545095E-4,-5.2714283E-3,8.010714E-2,1.4235473E-2,-1.6278861E-1,-2.3166311E-3,2.4558775E-4,-4.718529E-3,4.0668948E-3,-1.7228229E-4,-1.6696653E-3,-5.3246254E-3,1.5185479E-3,5.5236463E-3,-2.786262E-3,3.6624823E-2,-9.981885E-3,-4.2687077E-3,-4.260727E-4,5.2153744E-2,3.5941466E-3,-6.619151E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,-1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,-1,37,-1,39,41,-1,43,-1,-1,-1,45,47,-1,-1,-1,-1,49,-1,-1,-1,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,57,-1,-1,-1,59,-1,-1],"loss_changes":[2.0951697E-1,0E0,1.4345075E-1,1.4873178E-1,2.3511603E-1,8.3832964E-2,1.2690216E-1,1.0443731E-1,1.3068579E-1,8.528789E-2,5.5749476E-2,4.01905E-2,8.6802125E-2,3.7281193E-2,0E0,1.209102E-1,2.4628967E-2,6.073243E-2,6.266693E-2,0E0,1.02778375E-2,0E0,6.7801796E-2,0E0,2.2756204E-2,4.2648148E-2,0E0,5.499485E-2,0E0,0E0,0E0,2.8657747E-2,3.375502E-2,0E0,0E0,0E0,0E0,1.343751E-2,0E0,0E0,0E0,1.907517E-2,5.9506275E-2,1.9674808E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0169538E-2,0E0,0E0,0E0,3.04863E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,20,20,22,22,24,24,25,25,27,27,31,31,32,32,37,37,41,41,42,42,43,43,54,54,58,58],"right_children":[2,-1,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,-1,38,-1,40,42,-1,44,-1,-1,-1,46,48,-1,-1,-1,-1,50,-1,-1,-1,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,58,-1,-1,-1,60,-1,-1],"split_conditions":[9.558929E4,8.993286E-3,2.1114638E0,1.9011765E0,2.7438753E0,1.5423752E2,6.019841E0,6.811E3,2.3950179E0,7.07E2,2E0,2.7661E4,9.637789E5,9.6403706E-1,9.105974E-3,7.723729E2,7.5644026E0,1.4632207E0,5.7724595E0,-5.380993E-4,1.3255E4,2.2645278E-3,1.7135462E9,-1.0353298E-2,1.3724E4,5.097052E8,5.1439856E-3,5.3157896E-1,1.8417955E-3,-7.095255E-4,2.9161072E-3,4.4358948E-1,3.8951958E6,-8.041561E-3,-1.3916446E-3,6.5413695E-3,2.5740946E-3,2.86354E5,2.1728605E-3,-6.545095E-4,-5.2714283E-3,6.7491784E9,1.21816E5,8.9655056E5,-2.3166311E-3,2.4558775E-4,-4.718529E-3,4.0668948E-3,-1.7228229E-4,-1.6696653E-3,-5.3246254E-3,1.5185479E-3,5.5236463E-3,-2.786262E-3,5.1958606E5,-9.981885E-3,-4.2687077E-3,-4.260727E-4,2.491E3,3.5941466E-3,-6.619151E-4],"split_indices":[37,0,62,62,65,61,66,2,63,0,8,38,56,36,0,61,65,62,66,0,9,0,12,0,9,5,0,36,0,0,0,36,54,0,0,0,0,1,0,0,0,40,1,37,0,0,0,0,0,0,0,0,0,0,37,0,0,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,5E0,2.02E2,9.9E1,1.03E2,5.8E1,4.1E1,6.1E1,4.2E1,4.1E1,1.7E1,2.3E1,1.8E1,5.6E1,5E0,2.5E1,1.7E1,2.7E1,1.4E1,7E0,1E1,5E0,1.8E1,8E0,1E1,4.8E1,8E0,1.9E1,6E0,7E0,1E1,1E1,1.7E1,5E0,9E0,5E0,5E0,1.3E1,5E0,5E0,5E0,1.3E1,3.5E1,1.1E1,8E0,5E0,5E0,9E0,8E0,5E0,8E0,6E0,7E0,8E0,2.7E1,6E0,5E0,7E0,2E1,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-3.4231495E-4,6.555855E-2,-1.6339391E-2,3.953591E-2,1.0514354E-2,-4.331337E-2,3.0277437E-2,-3.9476827E-3,6.0856387E-2,-1.9442169E-2,-1.510595E-2,-1.1737225E-2,1.0568345E-1,9.179086E-2,1.6496951E-3,-7.654101E-2,3.1971354E-3,3.8907662E-2,-6.423485E-2,9.24969E-3,7.196253E-2,1.6614896E-3,1.2474521E-1,-1.1217616E-3,1.4699963E-3,1.3275059E-2,-1.3216612E-1,-4.3596506E-2,3.262797E-2,6.369619E-2,-6.0778944E-4,-1.0775351E-1,9.7526197E-4,6.4083014E-4,5.7548136E-3,7.669747E-3,2.98504E-3,1.981413E-3,-5.8597827E-4,-9.31982E-3,-3.7933313E-3,3.3633807E-3,-8.248951E-2,7.236643E-2,-6.8142214E-3,2.58187E-4,4.5153E-3,-7.6251402E-3,-3.109683E-3,-1.0002601E-3,-1.2385644E-1,1.0028269E-1,3.6601952E-4,2.0733662E-2,-3.7480348E-3,-9.020573E-3,-3.0725023E-3,1.7285172E-3,7.1778963E-3,-1.6680622E-3,5.1007096E-2,4.283524E-3,6.025175E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,-1,13,15,-1,17,19,21,23,25,27,29,31,-1,33,-1,35,-1,-1,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,49,51,53,-1,-1,-1,-1,-1,55,57,-1,59,-1,-1,-1,-1,-1,-1,61,-1,-1],"loss_changes":[2.2667454E-1,1.5494269E-1,2.1884876E-1,9.981334E-2,0E0,6.9174784E-1,2.0400256E-1,0E0,5.8457457E-2,1.3209787E-1,0E0,1.1416616E-1,5.1181287E-2,3.575957E-2,9.080525E-3,1.4663932E-1,1.0333348E-1,2.8791238E-2,7.9087704E-2,0E0,4.694335E-2,0E0,1.7043203E-2,0E0,0E0,8.750379E-3,4.2657226E-2,1.3245767E-1,7.2592236E-2,2.4640836E-2,0E0,1.988472E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.391358E-2,4.090575E-2,4.7780313E-2,0E0,0E0,0E0,0E0,0E0,3.652467E-2,4.31747E-2,0E0,3.1634666E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5920514E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,22,22,25,25,26,26,27,27,28,28,29,29,31,31,42,42,43,43,44,44,50,50,51,51,53,53,60,60],"right_children":[2,4,6,8,-1,10,12,-1,14,16,-1,18,20,22,24,26,28,30,32,-1,34,-1,36,-1,-1,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,50,52,54,-1,-1,-1,-1,-1,56,58,-1,60,-1,-1,-1,-1,-1,-1,62,-1,-1],"split_conditions":[3.01E2,5.2015275E-1,3.49917E2,1.3226E4,1.0514354E-2,7.3155574E9,1.0361217E6,-3.9476827E-3,3.068497E6,2.00689E5,-1.510595E-2,1.74E3,5.054757E11,7.12E2,2.972438E2,1.5139E4,3.6452372E5,6.50596E5,2.2161171E0,9.24969E-3,1.650269E10,1.6614896E-3,1.2092575E7,-1.1217616E-3,1.4699963E-3,1.43499E5,2.8374527E2,7.65244E5,3.7490938E5,1E1,-6.0778944E-4,3.650307E9,9.7526197E-4,6.4083014E-4,5.7548136E-3,7.669747E-3,2.98504E-3,1.981413E-3,-5.8597827E-4,-9.31982E-3,-3.7933313E-3,3.3633807E-3,2.9166096E-1,4.6808E4,1.131E3,2.58187E-4,4.5153E-3,-7.6251402E-3,-3.109683E-3,-1.0002601E-3,2.7573213E8,4.5042853E5,3.6601952E-4,1.5208125E2,-3.7480348E-3,-9.020573E-3,-3.0725023E-3,1.7285172E-3,7.1778963E-3,-1.6680622E-3,7.7579245E8,4.283524E-3,6.025175E-4],"split_indices":[0,48,61,9,0,5,37,0,12,42,0,0,40,2,4,9,37,37,63,0,5,0,54,0,0,1,4,57,42,3,0,12,0,0,0,0,0,0,0,0,0,0,36,9,2,0,0,0,0,0,7,56,0,4,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,4.1E1,1.72E2,3.6E1,5E0,1.09E2,6.3E1,5E0,3.1E1,1.01E2,8E0,4.1E1,2.2E1,2E1,1.1E1,2.8E1,7.3E1,2.1E1,2E1,5E0,1.7E1,8E0,1.2E1,6E0,5E0,1.1E1,1.7E1,2.8E1,4.5E1,1.4E1,7E0,1.3E1,7E0,8E0,9E0,7E0,5E0,5E0,6E0,7E0,1E1,7E0,2.1E1,2.2E1,2.3E1,5E0,9E0,5E0,8E0,9E0,1.2E1,1.5E1,7E0,1.7E1,6E0,5E0,7E0,7E0,8E0,6E0,1.1E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-3.1251067E-4,-1.8252023E-2,6.326255E-2,-2.09639E-3,-9.95509E-2,1.5408469E-2,2.4846267E-2,-7.368729E-2,2.0423574E-2,-1.4390868E-2,-1.6629018E-2,8.106126E-3,-9.058804E-3,-1.9104395E-2,-1.4639948E-1,4.520959E-2,-1.1125329E-2,-6.934741E-2,4.457048E-3,1.1140207E-2,-3.7196814E-3,-7.0439973E-3,3.217057E-2,-8.977729E-3,-3.2307818E-3,3.0715372E-3,7.831619E-2,3.4575798E-2,-8.66758E-2,-1.8566545E-3,-5.1243333E-3,3.2786306E-2,-2.3325598E-2,4.3274257E-3,-2.3770555E-4,-2.1913549E-2,2.3334145E-3,9.526513E-2,-6.760876E-4,6.8223864E-2,-2.158163E-2,-5.740916E-3,-1.0528516E-3,1.1836957E-2,3.3835089E-3,5.863059E-4,-2.6765938E-3,1.6212903E-3,-4.5650486E-2,6.6764764E-2,7.2349487E-3,3.090245E-2,6.246326E-3,-3.5719913E-3,1.253287E-3,2.0708023E-3,-7.8310247E-4,-3.7310156E-3,-8.277618E-4,1.6815731E-3,4.6772026E-3,3.333138E-3,-5.2789773E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,-1,19,21,23,25,27,29,-1,31,-1,-1,33,-1,-1,35,37,39,41,-1,-1,43,45,-1,-1,47,-1,49,-1,51,53,-1,-1,55,-1,-1,-1,-1,57,59,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3837984E-1,2.1388118E-1,4.3743196E-1,2.2237973E-1,4.3137637E-1,0E0,1.9749469E-1,1.2817495E-1,8.2609996E-2,0E0,1.19997196E-1,0E0,4.6888098E-2,1.3282686E-1,2.9759794E-2,8.206726E-2,1.6527574E-1,1.1600114E-2,0E0,2.0908115E-2,0E0,0E0,3.1739254E-2,0E0,0E0,3.1177662E-2,5.2857265E-2,5.8160048E-2,3.3550367E-2,0E0,0E0,1.2156315E-2,1.2869211E-2,0E0,0E0,2.5048893E-2,0E0,3.2212228E-2,0E0,3.8288914E-2,3.1053847E-2,0E0,0E0,1.0884059E-2,0E0,0E0,0E0,0E0,1.0160271E-2,1.5428424E-2,0E0,2.0951208E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,22,22,25,25,26,26,27,27,28,28,31,31,32,32,35,35,37,37,39,39,40,40,43,43,48,48,49,49,51,51],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,-1,20,22,24,26,28,30,-1,32,-1,-1,34,-1,-1,36,38,40,42,-1,-1,44,46,-1,-1,48,-1,50,-1,52,54,-1,-1,56,-1,-1,-1,-1,58,60,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.923E3,5.0826204E7,1.9122807E0,5.6380646E1,5.8866205E2,1.5408469E-2,2.5233118E11,4.5585E4,1.4103535E0,-1.4390868E-2,1.9002409E6,8.106126E-3,7.427366E6,1.19E3,8.953319E5,2.4246153E2,1.8733E4,2.545E3,4.457048E-3,2.9436884E7,-3.7196814E-3,-7.0439973E-3,2.760273E5,-8.977729E-3,-3.2307818E-3,2.1577182E0,3.4514523E1,5.75E2,2.3950179E0,-1.8566545E-3,-5.1243333E-3,1.1954182E7,3.3178E4,4.3274257E-3,-2.3770555E-4,1.8836116E5,2.3334145E-3,1.494E3,-6.760876E-4,2.2555719E2,2.749412E5,-5.740916E-3,-1.0528516E-3,9.115134E5,3.3835089E-3,5.863059E-4,-2.6765938E-3,1.6212903E-3,2.23226E-1,3.595E3,7.2349487E-3,5.549974E8,6.246326E-3,-3.5719913E-3,1.253287E-3,2.0708023E-3,-7.8310247E-4,-3.7310156E-3,-8.277618E-4,1.6815731E-3,4.6772026E-3,3.333138E-3,-5.2789773E-4],"split_indices":[2,54,62,61,4,0,40,1,65,0,37,0,1,38,60,4,9,2,0,60,0,0,37,0,0,63,67,0,63,0,0,60,10,0,0,41,0,2,0,61,37,0,0,41,0,0,0,0,48,10,0,5,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.62E2,4.5E1,1.36E2,2.6E1,5E0,4E1,3.2E1,1.04E2,7E0,1.9E1,7E0,3.3E1,1.9E1,1.3E1,5.8E1,4.6E1,1.3E1,6E0,2.6E1,7E0,5E0,1.4E1,8E0,5E0,2.6E1,3.2E1,2.9E1,1.7E1,8E0,5E0,1.6E1,1E1,5E0,9E0,1.7E1,9E0,2.7E1,5E0,1.8E1,1.1E1,1.1E1,6E0,1.1E1,5E0,5E0,5E0,5E0,1.2E1,1.9E1,8E0,1.2E1,6E0,5E0,6E0,5E0,6E0,5E0,7E0,1E1,9E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-8.9086965E-4,1.9812249E-2,-9.457628E-2,3.6172261E-3,2.3544444E-1,-2.5725057E-2,-1.7086161E-2,-1.3132003E-2,4.452242E-2,-6.0433264E-5,2.308759E-2,-9.458659E-2,7.689756E-2,-2.4082445E-2,6.1476436E-2,-2.7355612E-3,9.3954034E-2,-1.6539009E-1,1.1458462E-3,1.7143245E-3,5.8582113E-3,-1.5353069E-2,-5.455513E-3,6.255799E-3,-1.6039433E-3,2.6635002E-2,-4.8968797E-3,2.464737E-4,1.1837157E-1,-1.985723E-3,-1.2156717E-2,-6.061755E-3,-6.734964E-2,-1.1181075E-2,3.6218665E-3,7.336507E-3,1.4622683E-3,-3.7744857E-2,1.172424E-2,-5.677268E-3,-9.580498E-5,1.252545E-3,-2.5543661E-3,6.9179437E-3,-9.324661E-2,2.9931776E-3,1.3305519E-3,-1.7218998E-3,3.527063E-3,-6.1765793E-3,-1.6400131E-3,9.0688694E-4,-2.0122102E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,19,21,23,25,27,29,-1,-1,-1,31,-1,-1,-1,33,-1,-1,35,-1,-1,37,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,47,49,-1,51,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0724596E-1,5.998012E-1,6.581143E-1,1.1089068E-1,6.733828E-1,2.2634274E-1,0E0,9.5434956E-2,1.1005083E-1,0E0,0E0,1.6265695E-1,1.915054E-2,7.545672E-2,9.8262906E-2,7.4219406E-2,4.9336165E-2,1.1463457E-1,0E0,0E0,0E0,4.4521578E-2,0E0,0E0,0E0,3.672501E-2,0E0,0E0,4.60155E-2,0E0,0E0,4.552377E-2,4.4444025E-2,1.9481517E-2,0E0,0E0,0E0,7.2532125E-2,2.6677763E-2,0E0,0E0,0E0,0E0,4.9283713E-2,2.1458887E-2,0E0,3.2967906E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,25,25,28,28,31,31,32,32,33,33,37,37,38,38,43,43,44,44,46,46],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,20,22,24,26,28,30,-1,-1,-1,32,-1,-1,-1,34,-1,-1,36,-1,-1,38,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,48,50,-1,52,-1,-1,-1,-1,-1,-1],"split_conditions":[7.85999E9,5.818E3,3.0513447E5,1E0,1.4781E4,1.944213E12,-1.7086161E-2,2.596E3,1.4840448E2,-6.0433264E-5,2.308759E-2,7.08827E-2,2.6823762E5,4.520813E7,2.4274447E0,1.491E3,6.6958696E-1,2.0004E4,1.1458462E-3,1.7143245E-3,5.8582113E-3,3.10364E5,-5.455513E-3,6.255799E-3,-1.6039433E-3,5E0,-4.8968797E-3,2.464737E-4,3.1506528E8,-1.985723E-3,-1.2156717E-2,8.82E2,4.9196738E5,4.45E2,3.6218665E-3,7.336507E-3,1.4622683E-3,5.696862E8,1.3117E4,-5.677268E-3,-9.580498E-5,1.252545E-3,-2.5543661E-3,2.1282723E0,7.9119E4,2.9931776E-3,4.2179213E8,-1.7218998E-3,3.527063E-3,-6.1765793E-3,-1.6400131E-3,9.0688694E-4,-2.0122102E-3],"split_indices":[5,2,42,22,9,40,0,2,61,0,0,47,42,54,63,2,65,12,0,0,0,9,0,0,0,8,0,0,7,0,0,2,37,10,0,0,0,5,9,0,0,0,0,62,38,0,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,1.71E2,3.7E1,1.6E2,1.1E1,3E1,7E0,1.14E2,4.6E1,6E0,5E0,1.8E1,1.2E1,1E2,1.4E1,2.4E1,2.2E1,1.1E1,7E0,7E0,5E0,9.2E1,8E0,8E0,6E0,1.9E1,5E0,5E0,1.7E1,5E0,6E0,7.9E1,1.3E1,1.1E1,8E0,1.2E1,5E0,2.8E1,5.1E1,7E0,6E0,6E0,5E0,1.6E1,1.2E1,8E0,4.3E1,1E1,6E0,7E0,5E0,3.1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.388919E-2,1.8475325E-3,-1.0246599E-1,-5.9671043E-3,1.1280003E-2,-2.661528E-2,-1.596869E-2,-7.573806E-3,1.1204927E-3,2.3984194E-2,-9.319499E-2,1.4440366E-2,-5.2191954E-2,-5.275009E-4,3.8100411E-3,1.1683551E-3,-9.519882E-3,5.1045742E-2,-2.458268E-3,-1.1284872E-1,1.6343696E-3,2.5022026E-2,8.862321E-3,5.785412E-2,-2.1690583E-2,-8.86443E-3,-1.9268246E-3,3.045079E-3,-2.8932919E-3,4.690892E-3,2.1416775E-3,7.321399E-4,8.9683354E-2,-5.98533E-3,-5.352844E-3,-4.5094334E-2,3.351337E-2,2.1906937E-3,5.950517E-3,-1.8721826E-2,5.9547964E-2,-4.175964E-3,-1.9925572E-4,2.531051E-3,-5.695356E-4,-2.5104552E-3,5.989375E-4,4.9743075E-3,3.6199228E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,9,-1,-1,11,13,15,17,19,-1,-1,-1,-1,21,23,25,27,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,37,-1,39,41,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.956979E-1,3.2517475E-1,5.355495E-1,1.860133E-1,0E0,8.645766E-2,0E0,0E0,1.20763116E-1,2.8990133E-2,1.3547826E-1,8.425248E-2,1.1119208E-1,0E0,0E0,0E0,0E0,1.4122874E-1,1.10289894E-1,7.333194E-2,7.379484E-2,5.9854873E-2,0E0,3.0044906E-2,1.1786874E-1,0E0,0E0,0E0,0E0,0E0,4.44856E-2,0E0,1.3239451E-2,0E0,5.577729E-2,1.950696E-2,1.5454253E-2,0E0,0E0,5.4209784E-2,2.3460861E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,20,20,21,21,23,23,24,24,30,30,32,32,34,34,35,35,36,36,39,39,40,40],"right_children":[2,4,6,8,-1,10,-1,-1,12,14,16,18,20,-1,-1,-1,-1,22,24,26,28,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,38,-1,40,42,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.526755E7,1.3717E4,2.7584056E5,4.26E2,1.1280003E-2,8.096533E-1,-1.596869E-2,-7.573806E-3,4.4085712E0,1E0,1E0,2.202021E8,2.326087E0,-5.275009E-4,3.8100411E-3,1.1683551E-3,-9.519882E-3,4.4450412E2,1.3257E4,7.5167036E0,1.6507992E9,6.416E3,8.862321E-3,1.2262E4,2.1152425E6,-8.86443E-3,-1.9268246E-3,3.045079E-3,-2.8932919E-3,4.690892E-3,7.131518E1,7.321399E-4,2E0,-5.98533E-3,2.3866532E0,8.0563555E6,4.3161097E5,2.1906937E-3,5.950517E-3,1.436E3,4.520508E0,-4.175964E-3,-1.9925572E-4,2.531051E-3,-5.695356E-4,-2.5104552E-3,5.989375E-4,4.9743075E-3,3.6199228E-4],"split_indices":[54,2,42,2,0,36,0,0,65,24,8,7,62,0,0,0,0,4,9,66,5,38,0,9,54,0,0,0,0,0,61,0,8,0,62,54,41,0,0,2,63,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.8E2,3.1E1,1.75E2,5E0,2.4E1,7E0,7E0,1.68E2,1.4E1,1E1,1.35E2,3.3E1,9E0,5E0,5E0,5E0,4.2E1,9.3E1,1.5E1,1.8E1,3.6E1,6E0,2.2E1,7.1E1,7E0,8E0,9E0,9E0,8E0,2.8E1,1E1,1.2E1,9E0,6.2E1,1.1E1,1.7E1,6E0,6E0,5.2E1,1E1,5E0,6E0,1.2E1,5E0,2.5E1,2.7E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-8.379541E-3,-9.319503E-2,8.536133E-3,-4.4189997E-2,-1.6172227E-2,3.4849116E-5,7.5228685E-3,-5.0686356E-3,-1.0086447E-2,-2.2657143E-2,3.682285E-2,-5.0765526E-2,4.4533294E-2,8.362366E-3,-3.3831272E-2,1.4628185E-2,1.1130951E-1,9.7392494E-4,-4.509274E-3,3.3064038E-3,7.362078E-4,-1.8096833E-2,-1.503315E-1,5.387427E-2,-2.3706125E-2,1.720375E-3,9.446983E-3,-3.9491476E-3,-1.2063007E-1,-9.840456E-3,-4.1622636E-3,5.5222088E-3,-4.852809E-3,-4.975546E-3,8.713089E-3,-3.875131E-2,3.094934E-2,-1.0098947E-2,-7.1113225E-4,-3.1804135E-3,3.160347E-3,3.959258E-3,-2.0782318E-2,-7.497516E-3,-7.827019E-2,5.1687956E-3,6.608707E-3,1.8793356E-4,-2.8528257E-3,6.008832E-4,-3.3687132E-3,-5.028788E-3,-1.2171281E-3,1.5029737E-3,-1.5171064E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,31,33,-1,-1,35,37,-1,-1,-1,39,-1,41,43,45,-1,-1,-1,-1,-1,47,49,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1069314E-1,4.0465504E-1,2.2367781E-1,1.9561666E-1,0E0,1.444187E-1,0E0,6.1193757E-2,0E0,2.3742521E-1,1.0748338E-1,4.2895567E-2,7.835355E-3,0E0,1.842522E-1,7.950967E-2,8.677086E-2,0E0,0E0,0E0,0E0,1.3152894E-1,2.273646E-2,9.078884E-2,6.923597E-2,0E0,0E0,9.9577315E-2,9.7319975E-2,0E0,0E0,0E0,6.28006E-2,0E0,4.4325653E-2,5.005289E-2,7.394411E-2,0E0,0E0,0E0,0E0,0E0,1.4107235E-2,3.0020222E-2,2.1079972E-2,0E0,3.0034518E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,21,21,22,22,23,23,24,24,27,27,28,28,32,32,34,34,35,35,36,36,42,42,43,43,44,44,46,46],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,32,34,-1,-1,36,38,-1,-1,-1,40,-1,42,44,46,-1,-1,-1,-1,-1,48,50,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.7722336E4,3.2032586E8,1.3717E4,3.6543632E6,-1.6172227E-2,2.148789E0,7.5228685E-3,8.9341575E-1,-1.0086447E-2,1.3541759E9,1.5452969E6,2.4704047E-4,5.99999E9,8.362366E-3,6.239726E0,7.65244E5,3.607934E2,9.7392494E-4,-4.509274E-3,3.3064038E-3,7.362078E-4,9.534638E-1,7.5167036E0,5.696862E8,1.211648E6,1.720375E-3,9.446983E-3,1.5423752E2,9.76592E-3,-9.840456E-3,-4.1622636E-3,5.5222088E-3,1.3316584E-1,-4.975546E-3,1.2007376E0,2.854988E5,1.3235667E9,-1.0098947E-2,-7.1113225E-4,-3.1804135E-3,3.160347E-3,3.959258E-3,6.557827E-1,1.6790401E0,2.8003845E2,5.1687956E-3,8.493112E-1,1.8793356E-4,-2.8528257E-3,6.008832E-4,-3.3687132E-3,-5.028788E-3,-1.2171281E-3,1.5029737E-3,-1.5171064E-3],"split_indices":[41,7,2,37,0,63,0,36,0,40,37,48,5,0,65,57,61,0,0,0,0,36,66,5,57,0,0,61,47,0,0,0,66,0,65,56,5,0,0,0,0,0,36,65,4,0,36,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,3.5E1,1.8E2,3E1,5E0,1.71E2,9E0,2.5E1,5E0,1.06E2,6.5E1,1.3E1,1.2E1,5E0,1.01E2,5.1E1,1.4E1,5E0,8E0,6E0,6E0,9E1,1.1E1,2.5E1,2.6E1,8E0,6E0,8E1,1E1,5E0,6E0,1.2E1,1.3E1,7E0,1.9E1,4E1,4E1,5E0,5E0,7E0,6E0,5E0,1.4E1,2.3E1,1.7E1,9E0,3.1E1,9E0,5E0,1.8E1,5E0,1.1E1,6E0,1.9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[4.165697E-3,4.8959002E-2,-1.136638E-2,2.94351E-2,7.7006673E-3,-2.9021911E-2,5.0500505E-2,4.0579744E-2,-2.614266E-3,-1.0273094E-1,6.239744E-4,3.2280062E-3,1.0643485E-1,1.724145E-2,7.073961E-2,-1.4543228E-1,2.9115484E-3,2.0024508E-2,-6.2226515E-2,2.864619E-3,-2.3173168E-3,6.663225E-3,2.9089584E-3,5.256832E-2,-3.1478372E-3,4.653869E-3,1.3808848E-3,-3.1420306E-4,-1.8305035E-1,7.462591E-3,7.2629545E-3,-1.782795E-2,-7.2575565E-3,-7.663325E-4,8.002281E-2,-2.596947E-1,-9.753884E-2,-6.1649658E-2,2.4933115E-2,-2.5929667E-3,2.1430322E-3,1.7923164E-3,5.0119846E-3,-1.5390843E-2,-7.8813E-3,-1.0629417E-3,-8.2953125E-3,-5.1594205E-4,-5.841197E-3,7.989621E-2,7.820857E-3,2.0983485E-3,5.362592E-3,2.7573299E-2,-4.676043E-2,1.7225656E-4,2.1219007E-3,-4.2381724E-3,4.780813E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,27,-1,29,31,-1,-1,-1,-1,33,-1,-1,-1,-1,35,37,-1,39,-1,-1,41,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,51,53,-1,-1,55,57,-1,-1,-1,-1],"loss_changes":[1.4596471E-1,1.09840825E-1,1.7186329E-1,4.5602847E-2,0E0,2.666548E-1,9.2201896E-2,2.8343298E-2,0E0,2.5122184E-1,1.0854736E-1,5.886004E-2,1.4322475E-2,7.548806E-2,1.602529E-2,1.4523733E-1,0E0,1.0871077E-1,7.7770114E-2,0E0,0E0,0E0,0E0,3.4980517E-2,0E0,0E0,0E0,0E0,1.18857086E-1,7.760503E-2,0E0,3.4905925E-2,0E0,0E0,8.92926E-3,8.987129E-3,6.1947986E-2,3.716479E-2,4.734282E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.515745E-3,4.4413883E-2,0E0,0E0,1.1273082E-2,2.106091E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,23,23,28,28,29,29,31,31,34,34,35,35,36,36,37,37,38,38,49,49,50,50,53,53,54,54],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,28,-1,30,32,-1,-1,-1,-1,34,-1,-1,-1,-1,36,38,-1,40,-1,-1,42,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,52,54,-1,-1,56,58,-1,-1,-1,-1],"split_conditions":[3.74E2,1.564E3,2.430046E0,9.4801545E-1,7.7006673E-3,1.4087999E6,1E0,1.9590893E4,-2.614266E-3,3.749911E2,6.7135654E8,8.3735475E5,2.5492957E0,1.9781647E5,1.8267011E5,1.8866123E8,2.9115484E-3,5.364E3,7.363421E6,2.864619E-3,-2.3173168E-3,6.663225E-3,2.9089584E-3,1.7288483E-5,-3.1478372E-3,4.653869E-3,1.3808848E-3,-3.1420306E-4,7.0652356E5,3.6452372E5,7.2629545E-3,1.4400111E10,-7.2575565E-3,-7.663325E-4,8E0,2.0206897E0,1.3612218E6,8.26E2,1.2247E4,-2.5929667E-3,2.1430322E-3,1.7923164E-3,5.0119846E-3,-1.5390843E-2,-7.8813E-3,-1.0629417E-3,-8.2953125E-3,-5.1594205E-4,-5.841197E-3,1.64935E5,1.9231517E0,2.0983485E-3,5.362592E-3,1E0,2.1723518E0,1.7225656E-4,2.1219007E-3,-4.2381724E-3,4.780813E-5],"split_indices":[0,2,62,36,0,60,24,56,0,61,7,41,65,42,56,7,0,2,1,0,0,0,0,47,0,0,0,0,60,37,0,5,0,0,3,63,59,0,10,0,0,0,0,0,0,0,0,0,0,1,63,0,0,25,62,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.08E2,5.3E1,1.55E2,4.6E1,7E0,1.21E2,3.4E1,4.1E1,5E0,3.4E1,8.7E1,1.9E1,1.5E1,2.4E1,1.7E1,2.7E1,7E0,6.7E1,2E1,9E0,1E1,8E0,7E0,1.7E1,7E0,1E1,7E0,6E0,2.1E1,6.2E1,5E0,1.4E1,6E0,5E0,1.2E1,1E1,1.1E1,1.2E1,5E1,9E0,5E0,5E0,7E0,5E0,5E0,6E0,5E0,7E0,5E0,1.1E1,3.9E1,6E0,5E0,2.9E1,1E1,1.2E1,1.7E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[9.461092E-4,7.08676E-3,-8.965025E-3,9.264328E-4,1.0865495E-1,2.418765E-2,-2.9297063E-2,-1.5497577E-3,1.2430175E-2,-1.0058915E-3,8.275517E-2,-1.2816241E-2,-1.0755059E-1,-8.382247E-2,1.2022512E-2,1.5363199E-1,4.6821836E-2,-4.399291E-4,-4.4355253E-3,-1.2463918E-3,-1.10653825E-2,-7.569108E-3,5.7486348E-5,4.289917E-2,-2.598412E-2,3.6150336E-3,1.0152461E-2,6.1538022E-2,-3.7066085E-4,-4.228082E-3,1.1646268E-2,1.8060164E-3,7.8035876E-2,-6.670334E-2,1.3868332E-2,8.0431186E-2,3.5182148E-4,-6.7471624E-3,7.441997E-2,3.996974E-3,-4.139099E-2,5.478153E-3,5.30119E-2,-9.477826E-2,-2.011629E-4,1.9499446E-3,-1.9792148E-3,1.6385713E-3,5.0232583E-3,3.1762414E-3,-2.4012024E-2,4.7449707E-3,1.5549553E-3,-3.3312964E-3,-4.2606497E-4,3.9359224E-3,1.0907128E-3,-6.2358E-3,-2.2576083E-3,-3.5570525E-3,6.1298127E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,-1,37,39,41,43,45,47,-1,49,51,-1,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4582423E-1,1.3129492E-1,0E0,1.4061266E-1,2.53375E-1,1.6681737E-1,1.1082399E-1,0E0,0E0,8.732517E-2,7.939926E-2,7.036463E-2,1.4047751E-1,6.659227E-2,8.326094E-2,3.202668E-2,1.9681834E-2,6.757559E-2,0E0,0E0,0E0,0E0,0E0,5.6170315E-2,5.2853413E-2,0E0,0E0,1.8927641E-2,0E0,0E0,6.631567E-2,6.901531E-2,1.19455755E-2,2.7724348E-2,2.566449E-2,1.1223502E-2,0E0,5.728671E-2,9.460017E-3,0E0,1.0773353E-2,0E0,1.002625E-2,1.1637077E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.6856384E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,23,23,24,24,27,27,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,42,42,43,43,50,50],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,-1,38,40,42,44,46,48,-1,50,52,-1,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,1.8327797E3,-8.965025E-3,2.2725725E0,3E0,3.1274893E2,3.4847E4,-1.5497577E-3,1.2430175E-2,9.235376E5,6.96E2,3.3604588E6,9.3392765E11,5.01E2,1.2141942E6,1.12E2,6.1061732E7,7.7586204E-2,-4.4355253E-3,-1.2463918E-3,-1.10653825E-2,-7.569108E-3,5.7486348E-5,2.0117114E5,1E0,3.6150336E-3,1.0152461E-2,1.4137893E0,-3.7066085E-4,-4.228082E-3,9.3941006E5,5.4472397E3,4.031725E6,7.328832E-1,1.886E3,5.0003815E2,3.5182148E-4,5.549974E8,2.2519704E-2,3.996974E-3,2.0787193E2,5.478153E-3,6.557583E5,1.2519281E6,-2.011629E-4,1.9499446E-3,-1.9792148E-3,1.6385713E-3,5.0232583E-3,3.1762414E-3,1.897E3,4.7449707E-3,1.5549553E-3,-3.3312964E-3,-4.2606497E-4,3.9359224E-3,1.0907128E-3,-6.2358E-3,-2.2576083E-3,-3.5570525E-3,6.1298127E-4],"split_indices":[37,61,0,65,8,4,9,0,0,54,0,41,40,0,60,10,54,67,0,0,0,0,0,60,8,0,0,48,0,0,37,59,12,36,0,4,0,5,47,0,4,0,57,57,0,0,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.15E2,2.09E2,6E0,1.98E2,1.1E1,1.12E2,8.6E1,6E0,5E0,7.9E1,3.3E1,7.2E1,1.4E1,1E1,6.9E1,1E1,2.3E1,6.3E1,9E0,9E0,5E0,5E0,5E0,3.8E1,3.1E1,5E0,5E0,1.8E1,5E0,7E0,5.6E1,1.8E1,2E1,1.5E1,1.6E1,1.3E1,5E0,4.4E1,1.2E1,6E0,1.2E1,7E0,1.3E1,1E1,5E0,1.1E1,5E0,5E0,8E0,8E0,3.6E1,7E0,5E0,6E0,6E0,6E0,7E0,5E0,5E0,1.5E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[-2.6831527E-3,4.2517986E-3,-1.11826705E-2,-1.0841423E-2,5.6638416E-2,1.3522399E-4,-1.398283E-1,1.3177198E-1,1.32323615E-2,-2.2647262E-2,3.552703E-2,1.756486E-3,-1.322239E-2,1.33518325E-2,4.2674758E-2,-1.414849E-2,3.6978144E-3,-4.5952413E-2,7.2142687E-3,4.943908E-2,-1.633197E-3,2.0177035E-4,3.6224627E-3,1.3640294E-3,-3.7168812E-2,9.308639E-4,-6.105266E-2,8.538854E-2,-2.268681E-2,2.62793E-2,8.876344E-2,-3.8567258E-4,-3.6607243E-3,-2.708172E-2,-1.1031647E-1,5.6049163E-3,2.047047E-3,1.4960996E-3,-4.2209877E-3,4.2318128E-2,-2.3267977E-3,5.3309486E-3,2.159792E-3,-5.4837823E-2,7.5572096E-3,-2.5550905E-3,-8.675075E-3,-2.1673854E-2,3.2698305E-3,-3.483602E-4,6.090979E-2,-1.2435998E-3,-4.389046E-3,-1.2442028E-3,1.6997115E-3,-2.5141449E-3,5.1301887E-4,4.4275727E-3,1.5200063E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,-1,25,27,29,-1,-1,-1,-1,31,-1,33,35,37,39,41,-1,-1,43,45,-1,-1,47,-1,49,-1,-1,-1,51,53,-1,-1,55,-1,-1,57,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1613162E-1,1.5558787E-1,0E0,2.1664295E-1,1.4175144E-1,1.15301065E-1,2.8534156E-1,1.8989244E-1,5.0633334E-2,6.0877208E-2,5.5360794E-2,0E0,0E0,0E0,1.3022786E-2,2.1367308E-2,0E0,4.9254186E-2,9.30995E-2,4.04723E-2,0E0,0E0,0E0,0E0,1.4871838E-2,0E0,6.3953966E-2,9.210557E-3,4.477924E-2,3.8748574E-2,1.0563627E-2,0E0,0E0,2.445046E-2,5.3094625E-2,0E0,0E0,3.4786213E-2,0E0,2.4334956E-2,0E0,0E0,0E0,1.2100741E-2,1.1666337E-2,0E0,0E0,1.681554E-2,0E0,0E0,1.3990119E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,17,17,18,18,19,19,24,24,26,26,27,27,28,28,29,29,30,30,33,33,34,34,37,37,39,39,43,43,44,44,47,47,50,50],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,-1,26,28,30,-1,-1,-1,-1,32,-1,34,36,38,40,42,-1,-1,44,46,-1,-1,48,-1,50,-1,-1,-1,52,54,-1,-1,56,-1,-1,58,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7379535E6,3.313E3,-1.11826705E-2,2.697E3,5.2340768E8,2.1839464E0,7.6599895E9,2.5492957E0,6.0481E4,1.159E3,1E1,1.756486E-3,-1.322239E-2,1.33518325E-2,1E0,6.3171947E-1,3.6978144E-3,1E0,3.8372688E8,1.9002409E6,-1.633197E-3,2.0177035E-4,3.6224627E-3,1.3640294E-3,2.4010162E5,9.308639E-4,7.113821E-1,3.4076E4,1.9231517E0,2.8799667E9,2.13E3,-3.8567258E-4,-3.6607243E-3,3.0050538E5,1E0,5.6049163E-3,2.047047E-3,1.8435853E0,-4.2209877E-3,1.6386789E2,-2.3267977E-3,5.3309486E-3,2.159792E-3,1.5539E4,1.1556956E6,-2.5550905E-3,-8.675075E-3,1.2916E4,3.2698305E-3,-3.483602E-4,1.2092575E7,-1.2435998E-3,-4.389046E-3,-1.2442028E-3,1.6997115E-3,-2.5141449E-3,5.1301887E-4,4.4275727E-3,1.5200063E-3],"split_indices":[37,2,0,2,7,62,5,65,10,2,3,0,0,0,22,36,0,13,5,37,0,0,0,0,42,0,65,38,63,5,0,0,0,42,8,0,0,63,0,4,0,0,0,9,57,0,0,10,0,0,54,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2E2,1.95E2,5E0,1.52E2,4.3E1,1.41E2,1.1E1,1.5E1,2.8E1,8.6E1,5.5E1,5E0,6E0,5E0,1E1,2E1,8E0,4.8E1,3.8E1,4.6E1,9E0,5E0,5E0,7E0,1.3E1,9E0,3.9E1,1E1,2.8E1,3E1,1.6E1,8E0,5E0,2.4E1,1.5E1,5E0,5E0,2.1E1,7E0,2.5E1,5E0,1E1,6E0,1.3E1,1.1E1,9E0,6E0,1.6E1,5E0,7E0,1.8E1,8E0,5E0,5E0,6E0,8E0,8E0,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.5001265E-3,1.507229E-2,-8.046338E-2,3.4259765E-3,1.8416177E-1,-2.4731968E-2,-1.3074295E-2,3.2154683E-2,-1.3506852E-2,-1.1318385E-3,1.7635213E-2,-6.7145586E-2,4.3022595E-3,-3.209946E-3,4.224182E-2,8.244476E-3,-7.4232556E-2,-8.588018E-3,-2.5795769E-2,3.1527746E-2,6.0848584E-3,-1.8482603E-2,7.0636556E-2,-9.789423E-2,1.2170653E-3,2.0760395E-2,-5.632108E-3,4.1772593E-3,1.7604727E-2,-5.0093373E-3,-4.169096E-3,2.8558688E-2,1.0682804E-1,-5.475589E-2,-7.047756E-3,-6.703455E-4,2.0254275E-3,5.168266E-2,-4.0434115E-3,5.2311506E-2,-2.9175473E-2,-1.1821856E-4,2.4943887E-3,2.013652E-3,8.094264E-3,-3.8417068E-3,-6.7674665E-4,4.6604425E-3,-8.8348426E-4,-2.8114324E-2,2.6214595E-3,3.7035292E-3,4.1101716E-4,-6.1711468E-2,5.0767264E-3,-2.2982836E-3,5.381877E-4,-3.9023403E-3,-7.4195955E-4,1.8511118E-3,-1.3823826E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,-1,19,21,23,-1,25,27,-1,29,31,33,-1,35,-1,-1,37,39,-1,41,43,45,-1,-1,-1,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,55,-1,-1,-1,57,59,-1,-1,-1,-1,-1,-1],"loss_changes":[3.193846E-1,3.3611408E-1,4.5197615E-1,7.87541E-2,4.111422E-1,1.732826E-1,0E0,6.198015E-2,1.3534021E-1,0E0,0E0,1.121695E-1,0E0,0E0,4.4539303E-2,1.2803994E-1,6.576918E-2,0E0,8.565983E-2,3.604004E-2,0E0,4.818058E-2,3.1974934E-2,3.603667E-2,0E0,1.0547519E-2,0E0,0E0,3.0455649E-2,6.5246485E-2,0E0,8.476901E-3,3.946182E-2,1.15902275E-2,0E0,0E0,0E0,5.045486E-2,3.76026E-2,1.37799755E-2,3.7089467E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4570718E-2,0E0,0E0,0E0,1.3124764E-2,1.9667802E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,14,14,15,15,16,16,18,18,19,19,21,21,22,22,23,23,25,25,28,28,29,29,31,31,32,32,33,33,37,37,38,38,39,39,40,40,49,49,53,53,54,54],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,-1,20,22,24,-1,26,28,-1,30,32,34,-1,36,-1,-1,38,40,-1,42,44,46,-1,-1,-1,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,56,-1,-1,-1,58,60,-1,-1,-1,-1,-1,-1],"split_conditions":[7.6599895E9,5.666E3,3.2033653E5,9.0996725E5,1.3841E4,1.944213E12,-1.3074295E-2,8.53386E3,2.294901E0,-1.1318385E-3,1.7635213E-2,5.050505E-3,4.3022595E-3,-3.209946E-3,5.3719894E5,3.0332904E2,7.45226E5,-8.588018E-3,5.578E4,4.5198887E3,6.0848584E-3,3.34231E-1,9.350463E5,4.3232112E5,1.2170653E-3,1.1334E4,-5.632108E-3,4.1772593E-3,1.2557897E7,6.2085975E9,-4.169096E-3,1E0,2.0174673E0,6.069098E5,-7.047756E-3,-6.703455E-4,2.0254275E-3,4.38E2,1.496E3,2.4714162E5,6.700036E8,-1.1821856E-4,2.4943887E-3,2.013652E-3,8.094264E-3,-3.8417068E-3,-6.7674665E-4,4.6604425E-3,-8.8348426E-4,3.0893688E-3,2.6214595E-3,3.7035292E-3,4.1101716E-4,4.091156E8,3.3897146E8,-2.2982836E-3,5.381877E-4,-3.9023403E-3,-7.4195955E-4,1.8511118E-3,-1.3823826E-3],"split_indices":[5,2,42,59,9,40,0,42,65,0,0,66,0,0,57,4,38,0,10,60,0,47,41,42,0,9,0,0,54,40,0,8,62,37,0,0,0,0,2,56,5,0,0,0,0,0,0,0,0,48,0,0,0,7,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.13E2,1.7E2,4.3E1,1.6E2,1E1,3.4E1,9E0,5.9E1,1.01E2,5E0,5E0,2.5E1,9E0,5E0,5.4E1,7.5E1,2.6E1,6E0,1.9E1,4.9E1,5E0,5.3E1,2.2E1,2.1E1,5E0,1.3E1,6E0,9E0,4E1,4.5E1,8E0,1.1E1,1.1E1,1.2E1,9E0,5E0,8E0,1.5E1,2.5E1,1.3E1,3.2E1,5E0,6E0,6E0,5E0,7E0,5E0,9E0,6E0,1.8E1,7E0,8E0,5E0,1.6E1,1.6E1,1.2E1,6E0,1.1E1,5E0,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[6.450778E-3,-6.534406E-3,9.758044E-2,4.4795603E-2,-1.713615E-2,2.4635807E-1,1.2396394E-3,-2.4423467E-3,7.587482E-2,-1.1235684E-1,-8.194256E-3,1.8615818E-2,3.4611837E-3,4.1313786E-3,-4.7444288E-2,5.5878935E-3,5.243775E-2,-1.6464455E-3,-8.170379E-3,4.5726597E-4,-6.488459E-2,9.3011255E-4,-5.723292E-3,3.3860393E-3,3.5306418E-4,-1.7516136E-2,2.9346712E-2,8.443873E-3,-1.16037205E-2,-5.491768E-3,-7.1151853E-3,7.88409E-2,-3.923357E-5,3.1621417E-3,-1.2781793E-3,-1.3719528E-2,3.233863E-3,1.17366314E-1,5.4734264E-4,3.5295726E-3,-1.876876E-2,2.3043882E-3,-2.247444E-2,3.0916522E-3,7.1846535E-3,1.2899665E-2,-4.909656E-2,-1.5691123E-3,2.215047E-3,1.8709934E-3,-8.772049E-4,-4.9546926E-4,-3.4027768E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,-1,-1,21,-1,23,-1,-1,25,27,-1,-1,-1,-1,29,31,33,-1,35,-1,37,39,-1,-1,41,-1,43,-1,-1,45,-1,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6570475E-1,1.0797257E-1,4.010285E-1,1.0392525E-1,1.3871765E-1,2.3273432E-1,7.699011E-2,0E0,1.7431542E-2,5.472222E-2,7.408877E-2,0E0,0E0,0E0,5.778127E-2,0E0,1.3200749E-2,0E0,0E0,6.905394E-2,2.549981E-1,0E0,0E0,0E0,0E0,1.2545568E-1,7.41792E-2,3.0878568E-2,0E0,4.5711678E-2,0E0,4.819946E-2,4.6009943E-2,0E0,0E0,3.7826337E-2,0E0,9.937778E-3,0E0,0E0,2.6565034E-2,0E0,4.126369E-2,0E0,0E0,1.1645973E-2,1.0485865E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,14,14,16,16,19,19,20,20,25,25,26,26,27,27,29,29,31,31,32,32,35,35,37,37,40,40,42,42,45,45,46,46],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,-1,-1,22,-1,24,-1,-1,26,28,-1,-1,-1,-1,30,32,34,-1,36,-1,38,40,-1,-1,42,-1,44,-1,-1,46,-1,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1],"split_conditions":[5.666E3,2.86E2,1.3770975E2,4.39E2,5.5456768E1,1E0,1.1920863E1,-2.4423467E-3,4.4E1,8.0623E4,7.804584E7,1.8615818E-2,3.4611837E-3,4.1313786E-3,8.493112E-1,5.5878935E-3,4.4067E4,-1.6464455E-3,-8.170379E-3,2.5245471E2,3.2032586E8,9.3011255E-4,-5.723292E-3,3.3860393E-3,3.5306418E-4,4.360773E7,3.2875034E5,2.7078925E8,-1.16037205E-2,4.15874E5,-7.1151853E-3,2.9807162E0,1.3255E4,3.1621417E-3,-1.2781793E-3,1.4892368E0,3.233863E-3,8E0,5.4734264E-4,3.5295726E-3,5.1128894E2,2.3043882E-3,9.4010997E-1,3.0916522E-3,7.1846535E-3,2.1661E4,6.425227E5,-1.5691123E-3,2.215047E-3,1.8709934E-3,-8.772049E-4,-4.9546926E-4,-3.4027768E-3],"split_indices":[2,0,67,2,61,18,66,0,10,1,54,0,0,0,36,0,38,0,0,61,7,0,0,0,0,54,41,7,0,9,0,65,9,0,0,63,0,3,0,0,61,0,36,0,0,9,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.23E2,1.96E2,2.7E1,3.3E1,1.63E2,1E1,1.7E1,8E0,2.5E1,1.3E1,1.5E2,5E0,5E0,6E0,1.1E1,8E0,1.7E1,6E0,7E0,1.31E2,1.9E1,6E0,5E0,1.2E1,5E0,8.1E1,5E1,1.4E1,5E0,7.5E1,6E0,1.8E1,3.2E1,5E0,9E0,6.8E1,7E0,1.1E1,7E0,6E0,2.6E1,8E0,6E1,5E0,6E0,1.3E1,1.3E1,5.3E1,7E0,7E0,6E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.2584595E-4,-1.482275E-2,8.856671E-2,-1.663405E-4,-1.2229517E-1,1.2080705E-2,3.645154E-2,5.516158E-3,-5.9193093E-3,-1.6314216E-2,-1.8795783E-2,8.451733E-2,-1.6856645E-3,3.1127397E-2,-2.2305965E-2,-4.03969E-3,2.134928E-2,5.364951E-3,1.5920716E-3,1.4683748E-2,8.51238E-2,-6.3820537E-3,-5.884966E-3,-8.3266076E-4,2.7458437E-3,4.001674E-3,2.9981877E-3,7.089551E-3,1.1089478E-3,-4.305864E-3,3.7642682E-3,-1.1108112E-2,4.3087743E-2,3.2325894E-2,-2.9168127E-2,-6.002045E-2,9.514514E-3,7.269334E-5,3.6221247E-3,1.3603571E-2,3.1835493E-3,-4.6149394E-3,-1.0048107E-2,-5.503461E-4,-5.225538E-3,-1.7593055E-3,1.0017862E-3,-1.3738874E-3,1.3879462E-3,6.217642E-4,-2.5164764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,17,-1,19,21,-1,23,-1,-1,25,27,29,-1,-1,-1,-1,31,-1,-1,-1,33,35,37,39,41,43,45,-1,-1,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.659245E-1,2.7563262E-1,2.3066409E-1,1.0718359E-1,5.773533E-1,0E0,8.1336714E-2,1.0686563E-1,0E0,4.1736346E-2,0E0,1.4780924E-2,0E0,6.8006106E-2,1.11693986E-1,0E0,1.5661253E-2,0E0,0E0,4.7644183E-2,6.404658E-2,5.2561052E-2,0E0,0E0,0E0,0E0,3.0450936E-2,0E0,0E0,0E0,5.4557238E-2,4.0919095E-2,1.7952904E-2,1.8153124E-2,3.2733638E-2,2.6230346E-2,1.5228275E-2,0E0,0E0,1.3754276E-2,0E0,0E0,2.1409554E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,11,11,13,13,14,14,16,16,19,19,20,20,21,21,26,26,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,42,42],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,18,-1,20,22,-1,24,-1,-1,26,28,30,-1,-1,-1,-1,32,-1,-1,-1,34,36,38,40,42,44,46,-1,-1,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.737766E2,7.526755E7,1E0,5.3159845E6,9.479827E0,1.2080705E-2,4.9308786E-1,7.11E2,-5.9193093E-3,2.0651913E0,-1.8795783E-2,1.7266E4,-1.6856645E-3,1.6971204E2,5.591566E0,-4.03969E-3,7.475293E5,5.364951E-3,1.5920716E-3,1.5668E4,1.854111E7,1.58128E5,-5.884966E-3,-8.3266076E-4,2.7458437E-3,4.001674E-3,1.222E3,7.089551E-3,1.1089478E-3,-4.305864E-3,9.044871E5,4.4818586E-1,9E0,3.5E1,3.200388E2,1.3840085E-1,3.6285E4,7.269334E-5,3.6221247E-3,9.7846725E1,3.1835493E-3,-4.6149394E-3,1.722239E6,-5.503461E-4,-5.225538E-3,-1.7593055E-3,1.0017862E-3,-1.3738874E-3,1.3879462E-3,6.217642E-4,-2.5164764E-3],"split_indices":[61,54,13,37,66,0,51,0,0,63,0,9,0,61,65,0,37,0,0,1,54,1,0,0,0,0,2,0,0,0,56,36,3,8,4,36,1,0,0,61,0,0,1,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.02E2,1.74E2,2.8E1,1.54E2,2E1,6E0,2.2E1,1.48E2,6E0,1.5E1,5E0,1.3E1,9E0,7.7E1,7.1E1,5E0,1E1,8E0,5E0,6E1,1.7E1,6.2E1,9E0,5E0,5E0,8E0,5.2E1,8E0,9E0,6E0,5.6E1,3.9E1,1.3E1,3E1,2.6E1,1.1E1,2.8E1,6E0,7E0,2E1,1E1,5E0,2.1E1,6E0,5E0,5E0,2.3E1,5E0,1.5E1,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[8.200903E-3,1.9639758E-3,1.0400256E-1,-1.5028749E-2,3.6173735E-2,1.236032E-2,-1.0093045E-3,3.2951966E-2,-2.6776E-2,7.417766E-2,-7.711109E-3,-1.8926915E-3,5.9973605E-2,1.8924162E-2,-4.437004E-2,-1.1199134E-3,9.057627E-2,3.780224E-3,-2.6325027E-2,1.0701937E-3,7.879369E-2,4.505228E-2,-4.7634467E-3,-6.614119E-2,5.301282E-3,2.3298925E-3,1.15169115E-1,-4.7746938E-2,2.1660092E-4,5.1585413E-3,1.4667343E-3,1.3801488E-2,3.928072E-3,-4.6716142E-2,-8.525849E-3,3.0981109E-2,-3.184949E-3,7.6308614E-3,2.557838E-3,-8.5083296E-4,-4.663803E-3,-3.0008474E-4,2.1866856E-3,-7.096338E-3,-3.189078E-2,-2.5976764E-4,2.5709134E-3,-8.7345816E-2,-1.2439382E-2,-1.879856E-3,-5.9475056E-3,1.2267763E-3,-1.4591214E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,-1,19,21,23,-1,25,-1,27,-1,29,31,-1,33,35,-1,37,39,-1,-1,-1,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,47,-1,-1,49,51,-1,-1,-1,-1],"loss_changes":[1.2706493E-1,1.1738836E-1,2.464564E-1,7.6973945E-2,1.12314336E-1,0E0,0E0,5.4613806E-2,8.848116E-2,5.9391797E-2,5.2694224E-2,0E0,1.2607992E-2,9.865157E-2,8.5781366E-2,0E0,2.722463E-2,0E0,1.8000765E-2,0E0,1.3793156E-2,2.7580287E-2,0E0,1.075383E-1,4.725351E-2,0E0,4.0717453E-2,2.147859E-2,0E0,0E0,0E0,9.527395E-3,0E0,6.5437675E-2,0E0,1.5185699E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.5296624E-2,0E0,0E0,1.3781652E-2,2.2317417E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,18,18,20,20,21,21,23,23,24,24,26,26,27,27,31,31,33,33,35,35,44,44,47,47,48,48],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,-1,20,22,24,-1,26,-1,28,-1,30,32,-1,34,36,-1,38,40,-1,-1,-1,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,48,-1,-1,50,52,-1,-1,-1,-1],"split_conditions":[1.406168E3,2.326087E0,6.234292E-1,1.9356645E8,2.5605617E0,1.236032E-2,-1.0093045E-3,4.39E2,1.3257E4,2.0676692E-1,5.5043226E8,-1.8926915E-3,5.033303E-1,2.1127937E0,3.1236285E2,-1.1199134E-3,2.758609E2,3.780224E-3,7.35721E5,1.0701937E-3,8.5164154E-1,3E0,-4.7634467E-3,4.4428455E6,5.5180666E8,2.3298925E-3,1.330112E6,4.08648E5,2.1660092E-4,5.1585413E-3,1.4667343E-3,7.335273E-1,3.928072E-3,1.4329503E5,-8.525849E-3,1E0,-3.184949E-3,7.6308614E-3,2.557838E-3,-8.5083296E-4,-4.663803E-3,-3.0008474E-4,2.1866856E-3,-7.096338E-3,5.5456768E1,-2.5976764E-4,2.5709134E-3,1.6382E4,6.16E2,-1.879856E-3,-5.9475056E-3,1.2267763E-3,-1.4591214E-3],"split_indices":[61,62,36,7,65,0,0,2,9,65,5,0,36,62,61,0,4,0,1,0,36,8,0,60,7,0,1,1,0,0,0,51,0,42,0,13,0,0,0,0,0,0,0,0,61,0,0,9,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,2E2,1.2E1,1.34E2,6.6E1,5E0,7E0,2.6E1,1.08E2,3.5E1,3.1E1,7E0,1.9E1,3E1,7.8E1,5E0,3E1,5E0,2.6E1,7E0,1.2E1,2.5E1,5E0,5.4E1,2.4E1,1.2E1,1.8E1,1.5E1,1.1E1,7E0,5E0,1.4E1,1.1E1,4.7E1,7E0,1.8E1,6E0,1E1,8E0,1E1,5E0,9E0,5E0,5E0,4.2E1,7E0,1.1E1,1E1,3.2E1,5E0,5E0,1E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.5406217E-4,-1.0859226E-2,8.875204E-2,-6.580997E-3,-6.925059E-3,1.4726101E-2,1.2982037E-2,3.4978956E-2,-1.748814E-2,-3.1096343E-2,4.237433E-3,-1.7001329E-3,5.1326033E-2,-4.741576E-2,-3.9526196E-3,-3.6298786E-3,1.1948752E-3,3.682555E-2,5.280215E-3,-7.530477E-2,-1.35684395E-2,1.4355735E-2,-2.695114E-2,3.989266E-3,2.0142691E-2,-4.2019784E-2,-7.5711277E-3,-4.244401E-3,2.641053E-2,-2.9166483E-3,2.4044596E-2,-7.2197705E-3,-1.4903179E-3,1.9446951E-3,-8.1870373E-4,-7.697129E-2,2.6236838E-4,3.099286E-3,-6.787573E-4,-1.1111045E-3,3.306591E-2,1.268721E-2,-2.8357354E-3,-1.3588537E-3,-5.538811E-3,1.5026601E-2,6.369554E-2,3.7588984E-2,-1.4563876E-2,1.6354062E-3,-1.1407583E-3,4.4959127E-3,6.0007234E-5,1.9306155E-4,3.277493E-3,2.071474E-3,-2.1653615E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,-1,17,19,21,-1,-1,23,-1,25,27,29,31,-1,33,35,-1,-1,37,-1,39,-1,41,-1,-1,43,-1,-1,-1,-1,45,47,-1,-1,-1,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0368749E-1,1.0596225E-1,3.6644614E-1,8.492559E-2,0E0,0E0,6.1176762E-2,4.626014E-2,5.97653E-2,3.0584339E-2,0E0,0E0,2.2330634E-2,4.240077E-2,4.3756604E-2,0E0,0E0,1.8530175E-2,0E0,5.971788E-2,6.6732176E-2,4.303348E-2,1.4120507E-1,0E0,1.5883882E-2,3.1882104E-2,0E0,0E0,2.3242347E-2,0E0,2.2694567E-2,0E0,3.1989392E-2,0E0,0E0,1.650735E-2,0E0,0E0,0E0,0E0,2.335452E-2,2.2262655E-2,0E0,0E0,0E0,2.1286529E-2,2.8155446E-2,1.6583957E-2,2.8968899E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,12,12,13,13,14,14,17,17,19,19,20,20,21,21,22,22,24,24,25,25,28,28,30,30,32,32,35,35,40,40,41,41,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,-1,18,20,22,-1,-1,24,-1,26,28,30,32,-1,34,36,-1,-1,38,-1,40,-1,42,-1,-1,44,-1,-1,-1,-1,46,48,-1,-1,-1,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.474E3,4.966421E6,2.13281E5,1.9413766E8,-6.925059E-3,1.4726101E-2,1.6362E4,8.53386E3,4.9196738E5,4.9196738E5,4.237433E-3,-1.7001329E-3,4.0480963E2,2.123667E-1,5.1020538E8,-3.6298786E-3,1.1948752E-3,2.04E2,5.280215E-3,4.0491594E5,4.135593E1,2.047771E8,7.9208275E8,3.989266E-3,1.6826E4,2.758759E8,-7.5711277E-3,-4.244401E-3,1.8134046E6,-2.9166483E-3,2.0676692E-1,-7.2197705E-3,2.4019E6,1.9446951E-3,-8.1870373E-4,4.43E2,2.6236838E-4,3.099286E-3,-6.787573E-4,-1.1111045E-3,1E0,2.289485E1,-2.8357354E-3,-1.3588537E-3,-5.538811E-3,3.0200253E8,4E1,1.2557897E7,2.1745205E6,1.6354062E-3,-1.1407583E-3,4.4959127E-3,6.0007234E-5,1.9306155E-4,3.277493E-3,2.071474E-3,-2.1653615E-3],"split_indices":[2,37,38,7,0,0,9,42,37,37,0,0,4,50,12,0,0,0,0,56,61,7,12,0,9,7,0,0,57,0,65,0,41,0,0,0,0,0,0,0,18,66,0,0,0,7,8,54,56,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,1.9E2,2.2E1,1.85E2,5E0,5E0,1.7E1,3.8E1,1.47E2,1.1E1,6E0,7E0,3.1E1,4.5E1,1.02E2,6E0,5E0,2.6E1,5E0,2.4E1,2.1E1,5.7E1,4.5E1,6E0,2E1,1.8E1,6E0,7E0,1.4E1,6E0,5.1E1,7E0,3.8E1,1.3E1,7E0,1E1,8E0,7E0,7E0,8E0,4.3E1,3.1E1,7E0,5E0,5E0,2.8E1,1.5E1,1.6E1,1.5E1,1.9E1,9E0,1E1,5E0,8E0,8E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-1.0318555E-2,-2.2158952E-2,2.517337E-2,-9.59378E-3,-8.4132545E-2,-3.053319E-3,4.0194206E-2,-5.667584E-3,-7.248247E-4,-9.906752E-3,-4.6257056E-2,5.297412E-3,9.108735E-3,1.9484036E-2,-3.2957584E-2,-4.519701E-3,-1.9041714E-4,3.7709095E-3,-1.6784826E-2,3.8753882E-2,-3.9350975E-2,3.127276E-2,-6.404284E-2,-4.552524E-2,1.3688514E-3,7.068286E-2,1.6664727E-2,-5.3162873E-3,1.0511073E-3,2.7872836E-3,4.946855E-4,-8.4896505E-2,1.9782975E-3,-3.5782964E-3,-9.2567597E-4,4.417882E-2,5.2302955E-3,-5.915275E-3,6.646793E-2,-2.2775847E-3,2.3730013E-3,-4.921323E-2,-9.853526E-3,3.129759E-3,1.8239487E-4,3.2236197E-3,-2.7178964E-2,5.4657366E-3,4.906855E-4,-4.348204E-3,-1.4855795E-2,4.132258E-4,-2.524102E-3,1.1785494E-3,-2.0639095E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,15,-1,17,19,21,-1,-1,-1,23,25,27,29,31,33,-1,35,37,-1,39,-1,-1,41,-1,-1,-1,43,-1,45,47,-1,-1,49,-1,-1,-1,-1,51,-1,-1,-1,53,-1,-1,-1,-1],"loss_changes":[8.712693E-2,1.1989272E-1,7.0964046E-2,1.2253944E-1,1.07360736E-1,0E0,9.438826E-2,0E0,7.9460025E-2,0E0,4.0746037E-2,0E0,5.7670273E-2,8.655562E-2,9.584376E-2,0E0,0E0,0E0,3.216908E-2,3.9023288E-2,5.339685E-2,8.137729E-3,7.318084E-2,9.652954E-3,0E0,1.7413013E-2,3.9752655E-2,0E0,3.1681497E-2,0E0,0E0,1.0250336E-1,0E0,0E0,0E0,1.1804592E-2,0E0,3.9995234E-2,2.702815E-2,0E0,0E0,2.8719027E-2,0E0,0E0,0E0,0E0,1.7867925E-2,0E0,0E0,0E0,1.4881916E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,10,10,12,12,13,13,14,14,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,31,31,35,35,37,37,38,38,41,41,46,46,50,50],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,16,-1,18,20,22,-1,-1,-1,24,26,28,30,32,34,-1,36,38,-1,40,-1,-1,42,-1,-1,-1,44,-1,46,48,-1,-1,50,-1,-1,-1,-1,52,-1,-1,-1,54,-1,-1,-1,-1],"split_conditions":[2.3900106E0,2.1159E4,4.761905E-2,1.9898948E-6,1.1825188E6,-3.053319E-3,1.2920592E0,-5.667584E-3,3.7861453E5,-9.906752E-3,6.069098E5,5.297412E-3,5.696862E8,2.0740488E0,1.03817875E6,-4.519701E-3,-1.9041714E-4,3.7709095E-3,8.639937E9,6.6828716E-1,4.762529E-1,1.0932717E8,4.83512E5,2.83596E0,1.3688514E-3,6.001792E5,1.5227739E6,-5.3162873E-3,2.262069E0,2.7872836E-3,4.946855E-4,1.8271865E0,1.9782975E-3,-3.5782964E-3,-9.2567597E-4,1.0004987E2,5.2302955E-3,8.444269E4,1.203E3,-2.2775847E-3,2.3730013E-3,1.3407901E6,-9.853526E-3,3.129759E-3,1.8239487E-4,3.2236197E-3,1.3176E4,5.4657366E-3,4.906855E-4,-4.348204E-3,1.113E3,4.132258E-4,-2.524102E-3,1.1785494E-3,-2.0639095E-3],"split_indices":[62,9,65,47,59,0,65,0,42,0,37,0,5,63,60,0,0,0,5,36,36,5,38,63,0,56,37,0,62,0,0,63,0,0,0,61,0,42,0,0,0,57,0,0,0,0,9,0,0,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.05E2,1.54E2,5.1E1,1.29E2,2.5E1,7E0,4.4E1,9E0,1.2E2,5E0,2E1,1.3E1,3.1E1,7.4E1,4.6E1,9E0,1.1E1,8E0,2.3E1,5.6E1,1.8E1,1.5E1,3.1E1,1.4E1,9E0,2.2E1,3.4E1,6E0,1.2E1,6E0,9E0,2.6E1,5E0,6E0,8E0,1.4E1,8E0,2.4E1,1E1,6E0,6E0,2.1E1,5E0,9E0,5E0,5E0,1.9E1,5E0,5E0,9E0,1.2E1,8E0,1.1E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.034873E-3,1.14956E-2,-6.64241E-2,3.613801E-3,8.485873E-3,-1.2230804E-2,-2.3735315E-2,-5.758158E-2,1.4872426E-2,1.9947972E-2,-1.0145482E-2,-6.3443426E-3,-1.1716512E-2,7.5707875E-2,4.1912952E-3,-1.2252112E-2,3.385879E-3,-3.2599804E-3,1.6343876E-3,5.0007855E-3,9.627179E-4,2.1315912E-2,-3.236087E-2,8.936506E-3,-2.9157063E-3,-3.9323596E-3,2.8870711E-2,-6.192918E-2,5.576002E-3,2.55515E-3,-1.0239402E-3,3.6857393E-2,-1.7819608E-3,-4.1655526E-3,7.510534E-6,-1.6518019E-3,1.9772213E-3,2.3784418E-2,7.621523E-2,2.2222088E-3,4.2093784E-4,7.526915E-4,4.9013128E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,-1,9,11,13,15,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,-1,-1,37,-1,-1,-1,-1,-1,39,41,-1,-1,-1,-1],"loss_changes":[2.0908447E-1,2.2041842E-1,3.3144116E-1,1.1522637E-1,0E0,0E0,2.9322064E-1,8.459203E-2,9.130326E-2,4.849303E-2,0E0,0E0,4.4430535E-2,2.9272765E-2,7.642877E-2,1.9614765E-2,0E0,0E0,0E0,0E0,0E0,6.613423E-2,4.4092756E-2,1.9226654E-2,0E0,0E0,4.203953E-2,3.1824008E-2,2.6080413E-2,0E0,0E0,3.4203954E-2,0E0,0E0,0E0,0E0,0E0,1.7369969E-2,2.3327969E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,12,12,13,13,14,14,15,15,21,21,22,22,23,23,26,26,27,27,28,28,31,31,37,37,38,38],"right_children":[2,4,6,8,-1,-1,10,12,14,16,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,-1,-1,38,-1,-1,-1,-1,-1,40,42,-1,-1,-1,-1],"split_conditions":[5.8639234E11,1.3908939E3,8.87885E5,6.97E2,8.485873E-3,-1.2230804E-2,1.01081E6,4.0573265E1,2.93E2,2.326087E0,-1.0145482E-2,-6.3443426E-3,2.2918523E5,3.068497E6,1.8495E4,4.7564573E0,3.385879E-3,-3.2599804E-3,1.6343876E-3,5.0007855E-3,9.627179E-4,4.945055E-2,3.3640351E0,8.018171E-1,-2.9157063E-3,-3.9323596E-3,3.7017515E6,6.6608225E6,1.3225807E0,2.55515E-3,-1.0239402E-3,1E0,-1.7819608E-3,-4.1655526E-3,7.510534E-6,-1.6518019E-3,1.9772213E-3,6.6318585E6,7.518072E-1,2.2222088E-3,4.2093784E-4,7.526915E-4,4.9013128E-3],"split_indices":[40,61,1,2,0,0,9,61,0,62,0,0,42,12,9,65,0,0,0,0,0,67,63,36,0,0,56,57,66,0,0,22,0,0,0,0,0,54,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.14E2,1.72E2,4.2E1,1.65E2,7E0,7E0,3.5E1,2.5E1,1.4E2,2.9E1,6E0,9E0,1.6E1,2E1,1.2E2,1.8E1,1.1E1,7E0,9E0,1.3E1,7E0,8.2E1,3.8E1,1.3E1,5E0,5E0,7.7E1,2.1E1,1.7E1,5E0,8E0,6.9E1,8E0,1.5E1,6E0,8E0,9E0,5.3E1,1.6E1,2.1E1,3.2E1,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.5406855E-3,6.394711E-3,-1.0168172E-1,1.3119299E-2,-7.175802E-2,-2.1945554E-1,3.5666574E-3,4.549842E-3,1.1746411E-1,1.16556985E-2,-1.039794E-2,-3.1731364E-3,-1.6493037E-2,-6.0406663E-3,9.925556E-3,1.3793446E-2,-2.641676E-4,-1.0075414E-3,2.052049E-3,1.344487E-2,-4.211684E-3,1.7840797E-2,-3.3986714E-2,-1.3506499E-2,2.6261931E-2,-5.1921634E-3,8.7723986E-4,1.4374932E-2,-5.0380833E-2,6.466198E-2,1.3454478E-2,-1.5123643E-3,2.847395E-3,-4.9998285E-3,6.5267086E-5,1.6801287E-3,7.5738323E-3,2.5993243E-3,5.7182104E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,-1,21,-1,23,25,27,29,-1,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9288279E-1,1.060504E-1,3.862571E-1,1.6488856E-1,1.9061472E-1,1.786241E-1,0E0,1.19699664E-1,2.84879E-1,1.1629422E-2,0E0,0E0,0E0,0E0,5.7294328E-2,0E0,0E0,0E0,0E0,3.417578E-2,0E0,3.942372E-2,5.4426745E-2,3.3623718E-2,5.667141E-2,0E0,0E0,3.958062E-2,3.7718434E-2,7.398028E-2,4.3368354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,14,14,19,19,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,-1,22,-1,24,26,28,30,-1,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6146525E6,9.525163E-1,9.265399E-4,1.3908939E3,4.8012204E0,2.1113134E8,3.5666574E-3,9.235376E5,1.8380427E0,2.3097014E0,-1.039794E-2,-3.1731364E-3,-1.6493037E-2,-6.0406663E-3,3.988886E6,1.3793446E-2,-2.641676E-4,-1.0075414E-3,2.052049E-3,5.199797E5,-4.211684E-3,8.82E2,1.9665255E-1,3.264022E10,2.0233E5,-5.1921634E-3,8.7723986E-4,2.1282723E0,8.0563555E6,1.544E3,1.3255E4,-1.5123643E-3,2.847395E-3,-4.9998285E-3,6.5267086E-5,1.6801287E-3,7.5738323E-3,2.5993243E-3,5.7182104E-5],"split_indices":[37,36,47,61,66,7,0,54,63,62,0,0,0,0,38,0,0,0,0,42,0,2,51,40,1,0,0,62,54,2,9,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.16E2,1.99E2,1.7E1,1.84E2,1.5E1,1E1,7E0,1.71E2,1.3E1,1E1,5E0,5E0,5E0,6E0,1.65E2,5E0,8E0,5E0,5E0,1.6E2,5E0,1.47E2,1.3E1,3.1E1,1.16E2,5E0,8E0,1.8E1,1.3E1,2.8E1,8.8E1,9E0,9E0,6E0,7E0,2.2E1,6E0,2E1,6.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[8.684522E-4,1.1194223E-2,-1.0702673E-1,6.036891E-3,7.023366E-3,-5.0551696E-3,-1.5159046E-2,-5.37176E-3,3.390647E-2,1.4759967E-3,-2.5033725E-3,3.6414447E-3,-5.5671646E-3,-3.659967E-3,7.305854E-2,3.9512753E-3,-1.8076677E-3,2.7197214E-2,-2.5226213E-2,9.064615E-2,6.0470984E-4,4.376922E-2,-8.626125E-3,2.8959715E-3,-2.0328669E-4,-2.0710677E-3,2.5996254E-4,2.6862277E-3,5.6139575E-3,2.455422E-4,3.3747607E-3,1.44204525E-2,-3.828587E-2,4.958544E-4,2.4474862E-3,-5.5786863E-2,1.928865E-2,-2.154011E-3,6.8357814E-4,3.364069E-5,-3.878479E-3,2.623448E-3,-8.9492503E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,-1,15,-1,17,19,-1,21,23,25,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,33,35,37,-1,39,41,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2958876E-1,1.2759343E-1,3.71221E-1,5.804982E-2,0E0,2.2699911E-2,0E0,1.2757248E-1,7.8219995E-2,0E0,0E0,5.104386E-2,0E0,1.9319344E-2,2.6629537E-2,0E0,3.580466E-2,1.2305672E-2,9.127591E-3,1.0966495E-2,0E0,1.43412035E-2,6.890766E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.8271284E-2,4.5410953E-2,2.5862873E-2,0E0,4.529336E-2,1.5202183E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,11,11,13,13,14,14,16,16,17,17,18,18,19,19,21,21,22,22,31,31,32,32,33,33,35,35,36,36],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,-1,16,-1,18,20,-1,22,24,26,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,34,36,38,-1,40,42,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0388931E8,9.772E3,2.9119E4,3.1236285E2,7.023366E-3,9.2790455E-1,-1.5159046E-2,4.3625324E7,1.0361217E6,1.4759967E-3,-2.5033725E-3,1.23E2,-5.5671646E-3,6.1061732E7,3.2016622E6,3.9512753E-3,1.3257E4,2E0,8E0,2.001E3,6.0470984E-4,6.271094E-1,4.741634E-2,2.8959715E-3,-2.0328669E-4,-2.0710677E-3,2.5996254E-4,2.6862277E-3,5.6139575E-3,2.455422E-4,3.3747607E-3,2.195572E0,3.80384E5,1.5786376E-4,2.4474862E-3,4.41E2,1.09E3,-2.154011E-3,6.8357814E-4,3.364069E-5,-3.878479E-3,2.623448E-3,-8.9492503E-4],"split_indices":[54,2,9,61,0,36,0,54,37,0,0,0,0,54,41,0,9,8,8,2,0,65,47,0,0,0,0,0,0,0,0,63,9,47,0,10,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.04E2,1.87E2,1.7E1,1.81E2,6E0,1.2E1,5E0,1.29E2,5.2E1,7E0,5E0,1.2E2,9E0,2.7E1,2.5E1,7E0,1.13E2,1.1E1,1.6E1,1.9E1,6E0,1.4E1,9.9E1,5E0,6E0,1E1,6E0,9E0,1E1,6E0,8E0,5.6E1,4.3E1,4.1E1,1.5E1,3.3E1,1E1,9E0,3.2E1,1E1,2.3E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.5647176E-3,-5.369337E-3,1.0019719E-2,2.9589843E-3,6.606995E-2,2.9795494E-2,-7.5557334E-3,1.3597844E-2,-2.5159184E-2,8.579646E-3,5.6172885E-2,-2.37572E-3,-3.8449476E-3,1.587225E-3,-4.5973198E-3,3.186879E-2,-2.905876E-3,7.6782905E-2,3.876795E-4,3.5230637E-2,-1.0308068E-2,3.0568168E-3,8.4885315E-4,5.579643E-3,2.0992963E-3,9.744363E-3,3.8426376E-3,-7.450373E-2,5.057434E-4,-1.0803331E-3,2.4096537E-3,-1.5162934E-4,-8.851654E-3,6.9044353E-3,-3.732033E-3,-1.25803575E-2,2.5775526E-2,5.327018E-4,-2.7161774E-3,5.8997E-4,3.4117133E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,-1,3,5,7,9,11,-1,13,15,17,19,-1,-1,-1,21,-1,23,-1,25,27,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,35,-1,37,39,-1,-1,-1,-1],"loss_changes":[1.1061672E-1,0E0,8.133019E-2,5.2145943E-2,4.5207962E-1,2.8618224E-2,4.8800398E-2,0E0,7.055125E-2,4.9424496E-2,2.225902E-2,3.7651498E-2,0E0,0E0,0E0,9.143794E-3,0E0,1.5326373E-2,0E0,2.355869E-2,7.223446E-2,0E0,0E0,0E0,0E0,1.998362E-2,0E0,1.13593966E-1,4.4787485E-2,0E0,0E0,0E0,0E0,3.1211002E-2,0E0,4.299575E-2,2.5101924E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,15,15,17,17,19,19,20,20,25,25,27,27,28,28,33,33,35,35,36,36],"right_children":[2,-1,4,6,8,10,12,-1,14,16,18,20,-1,-1,-1,22,-1,24,-1,26,28,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,36,-1,38,40,-1,-1,-1,-1],"split_conditions":[4.26E2,-5.369337E-3,5.818E3,2.2064762E8,4.083812E7,1.1543406E-2,5.08616E5,1.3597844E-2,7.427366E6,2.886375E4,9.90627E6,1.2414E4,-3.8449476E-3,1.587225E-3,-4.5973198E-3,4.8102058E8,-2.905876E-3,1.0734139E-1,3.876795E-4,1.1231081E1,1.3598E4,3.0568168E-3,8.4885315E-4,5.579643E-3,2.0992963E-3,1E0,3.8426376E-3,1.9863014E0,1.5745124E0,-1.0803331E-3,2.4096537E-3,-1.5162934E-4,-8.851654E-3,3.91193E5,-3.732033E-3,1.1001313E9,1.5819151E6,5.327018E-4,-2.7161774E-3,5.8997E-4,3.4117133E-3],"split_indices":[2,0,2,7,54,48,42,0,1,60,12,9,0,0,0,5,0,48,0,66,9,0,0,0,0,24,0,62,48,0,0,0,0,1,0,5,37,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.12E2,7E0,2.05E2,1.83E2,2.2E1,5.1E1,1.32E2,6E0,1.6E1,2.9E1,2.2E1,1.24E2,8E0,9E0,7E0,2.2E1,7E0,1.5E1,7E0,2.1E1,1.03E2,6E0,1.6E1,6E0,9E0,1.4E1,7E0,1.4E1,8.9E1,8E0,6E0,9E0,5E0,8.3E1,6E0,4.1E1,4.2E1,2.7E1,1.4E1,3.3E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.2066257E-3,4.6008278E-2,-9.665301E-3,2.3845293E-2,9.046634E-3,3.4867905E-2,-2.1344746E-2,-2.712136E-2,4.263121E-2,6.3209916E-3,9.138709E-2,-6.393567E-2,1.769222E-3,2.1840922E-5,-2.9014584E-3,6.4053625E-2,-2.4023878E-3,3.541285E-2,-2.5736755E-3,6.42425E-3,1.7652644E-3,-1.9423462E-2,-1.5793157E-1,-1.1251612E-2,7.866172E-2,7.676084E-2,2.2812399E-4,2.503849E-3,2.5564084E-5,2.4436615E-2,-4.8410702E-2,-3.1728873E-3,-1.3253319E-2,-5.1391307E-2,1.454755E-2,5.982151E-3,7.108056E-4,4.8758355E-3,2.3156423E-3,-2.8032577E-4,2.1512476E-3,-5.606924E-3,-1.6539766E-2,-1.9456953E-3,-5.726626E-3,-2.2379909E-2,3.4419682E-2,7.828735E-4,-2.099828E-3,-2.9014917E-3,3.4767177E-2,-2.2687651E-3,3.6449736E-4,3.324984E-3,1.0105064E-2,3.0368366E-3,7.877065E-5,-1.8824615E-3,1.5527401E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,-1,-1,25,-1,27,-1,-1,-1,29,31,33,35,37,-1,-1,-1,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,47,49,-1,51,53,-1,-1,-1,55,-1,-1,-1,57,-1,-1,-1,-1],"loss_changes":[1.2105754E-1,1.5586133E-1,8.240357E-2,4.5199163E-2,0E0,5.266614E-2,1.2316877E-1,1.1579953E-2,6.999709E-2,4.1324742E-2,1.9599006E-2,1.8102686E-1,8.298384E-2,0E0,0E0,2.0771638E-2,0E0,8.799776E-3,0E0,0E0,0E0,4.059585E-2,1.2982816E-1,7.4273124E-2,3.1862117E-2,1.08134E-2,0E0,0E0,0E0,7.929966E-3,3.8248803E-2,0E0,0E0,9.0822674E-2,3.3058714E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2729388E-2,3.789673E-2,0E0,1.1820194E-2,2.3007333E-2,0E0,0E0,0E0,9.878824E-3,0E0,0E0,0E0,2.0179238E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,21,21,22,22,23,23,24,24,25,25,29,29,30,30,33,33,34,34,42,42,43,43,45,45,46,46,50,50,54,54],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,-1,-1,26,-1,28,-1,-1,-1,30,32,34,36,38,-1,-1,-1,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,48,50,-1,52,54,-1,-1,-1,56,-1,-1,-1,58,-1,-1,-1,-1],"split_conditions":[4.15E2,1.544E3,1.3255E4,1.4037507E0,9.046634E-3,1.673295E6,2.0235784E0,1.69E2,5.657E4,6.567405E2,9.0073E5,8.111963E-1,9.2379E2,2.1840922E-5,-2.9014584E-3,9.1696185E-1,-2.4023878E-3,9.0763354E-1,-2.5736755E-3,6.42425E-3,1.7652644E-3,4.741634E-2,9.857292E-1,2.170343E0,8.526109E-2,1.9781647E5,2.2812399E-4,2.503849E-3,2.5564084E-5,3.9232688E5,1.7761696E8,-3.1728873E-3,-1.3253319E-2,2.5605617E0,8E0,5.982151E-3,7.108056E-4,4.8758355E-3,2.3156423E-3,-2.8032577E-4,2.1512476E-3,-5.606924E-3,4.1401203E5,1.0904851E0,-5.726626E-3,1.7950666E5,1.4559748E0,7.828735E-4,-2.099828E-3,-2.9014917E-3,1E0,-2.2687651E-3,3.6449736E-4,3.324984E-3,2.48042E5,3.0368366E-3,7.877065E-5,-1.8824615E-3,1.5527401E-3],"split_indices":[0,2,9,63,0,41,62,0,38,4,1,36,61,0,0,36,0,36,0,0,0,47,65,63,47,42,0,0,0,37,12,0,0,65,8,0,0,0,0,0,0,0,42,65,0,56,65,0,0,0,25,0,0,0,38,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,5.1E1,1.56E2,4.5E1,6E0,3.2E1,1.24E2,1.2E1,3.3E1,2.2E1,1E1,4.3E1,8.1E1,7E0,5E0,2.7E1,6E0,1.5E1,7E0,5E0,5E0,3E1,1.3E1,7E1,1.1E1,2.2E1,5E0,1E1,5E0,1.2E1,1.8E1,8E0,5E0,2.7E1,4.3E1,6E0,5E0,1.1E1,1.1E1,5E0,7E0,5E0,1.3E1,1.6E1,1.1E1,1.5E1,2.8E1,6E0,7E0,6E0,1E1,8E0,7E0,1.1E1,1.7E1,5E0,5E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-4.153873E-3,2.908881E-3,-1.1629512E-1,-6.3524796E-3,9.4881944E-2,6.499158E-4,-1.1206871E-2,-2.3556747E-3,-4.7162296E-3,-1.0243001E-3,1.2681398E-2,-7.122442E-2,2.4909955E-3,2.0445255E-3,-2.1375087E-3,-5.621955E-3,-4.0396245E-4,7.3982226E-3,-6.955942E-2,2.6438886E-2,-1.1268258E-2,-1.1719954E-3,-5.0614653E-3,1.489076E-2,6.0338728E-2,-3.3492479E-3,-4.674711E-3,4.6660557E-2,-5.566236E-3,8.217553E-2,-2.1065975E-4,-5.385149E-2,1.2114982E-2,2.95227E-3,2.2823652E-4,-1.6700159E-3,1.0695668E-3,2.5064533E-3,5.3847297E-3,-3.727673E-3,-3.78384E-4,1.974481E-3,-1.1377314E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,-1,-1,-1,-1,19,21,23,25,-1,-1,27,29,31,-1,33,35,37,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7379199E-1,1.7692445E-1,1.9338036E-1,6.785902E-2,3.0012468E-1,0E0,0E0,6.0926124E-2,0E0,2.5619045E-2,0E0,3.213437E-2,6.0974695E-2,0E0,0E0,0E0,0E0,5.7520892E-2,1.4559656E-2,3.0419882E-2,5.446484E-2,0E0,0E0,3.9904512E-2,2.8588466E-2,6.000676E-2,0E0,1.3562672E-2,3.0577043E-2,7.3220357E-3,0E0,1.8009774E-2,2.4191253E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12,17,17,18,18,19,19,20,20,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,-1,-1,-1,-1,20,22,24,26,-1,-1,28,30,32,-1,34,36,38,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,5.818E3,7.3972464E-1,4.476758E6,2.6998322E5,6.499158E-4,-1.1206871E-2,9.541066E5,-4.7162296E-3,2.251073E0,1.2681398E-2,8E0,8.385435E0,2.0445255E-3,-2.1375087E-3,-5.621955E-3,-4.0396245E-4,1.2141942E6,2.2952059E-1,2.3082722E2,9.534638E-1,-1.1719954E-3,-5.0614653E-3,5.5141E4,2.6786362E8,1.0469056E2,-4.674711E-3,6.423972E1,5.21E2,1E0,-2.1065975E-4,1.3E1,1.2007376E0,2.95227E-3,2.2823652E-4,-1.6700159E-3,1.0695668E-3,2.5064533E-3,5.3847297E-3,-3.727673E-3,-3.78384E-4,1.974481E-3,-1.1377314E-4],"split_indices":[54,2,36,37,42,0,0,54,0,62,0,3,65,0,0,0,0,60,36,61,36,0,0,1,7,61,0,61,10,112,0,8,65,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,2.06E2,1.2E1,1.88E2,1.8E1,6E0,6E0,1.81E2,7E0,1.2E1,6E0,1.1E1,1.7E2,6E0,6E0,6E0,5E0,1.6E2,1E1,7.9E1,8.1E1,5E0,5E0,6E1,1.9E1,7.5E1,6E0,2.3E1,3.7E1,1.4E1,5E0,1.7E1,5.8E1,1.7E1,6E0,1.8E1,1.9E1,8E0,6E0,1.1E1,6E0,1.9E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-4.030755E-3,2.9854639E-2,-1.6849853E-2,-4.3639015E-2,5.423078E-2,-6.3260524E-3,-1.0566223E-2,1.2950597E-3,-3.9766533E-3,3.6895186E-2,6.8721003E-3,-1.7314557E-2,6.623305E-2,5.54369E-2,1.0729715E-2,-6.010676E-3,-6.447273E-2,7.180455E-3,-6.047166E-4,1.3656793E-3,3.535225E-3,1.993823E-3,-1.3777817E-3,1.2298636E-2,-4.246685E-2,-3.2346252E-2,-8.1223985E-3,-3.7489973E-3,1.985922E-2,-2.3051133E-2,-6.7913746E-3,-5.173811E-2,6.7594426E-4,2.6451943E-2,-2.628429E-3,-6.152672E-2,3.1105555E-3,-3.7975132E-3,-5.366887E-4,-1.4229419E-3,3.2701734E-2,-8.0241193E-4,-5.446217E-3,-1.633934E-2,1.6779462E-3,1.1026344E-3,3.616765E-3,6.655607E-4,-1.7970062E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,27,29,31,-1,-1,33,35,-1,37,-1,39,-1,41,43,-1,-1,-1,45,-1,-1,47,-1,-1,-1,-1,-1],"loss_changes":[9.172288E-2,1.0627309E-1,1.0685681E-1,4.1833192E-2,6.068556E-2,0E0,7.6813795E-2,0E0,0E0,1.7840989E-2,0E0,7.157697E-2,7.7906236E-2,8.195341E-3,2.1037214E-2,7.394591E-2,7.9571515E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.131522E-2,6.610147E-2,1.951922E-2,0E0,0E0,3.491755E-2,3.2402143E-2,0E0,1.5013602E-2,0E0,2.3176357E-2,0E0,2.743036E-2,1.2704344E-2,0E0,0E0,0E0,2.2623938E-2,0E0,0E0,8.454226E-3,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,11,11,12,12,13,13,14,14,15,15,16,16,23,23,24,24,25,25,28,28,29,29,31,31,33,33,35,35,36,36,40,40,43,43],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,28,30,32,-1,-1,34,36,-1,38,-1,40,-1,42,44,-1,-1,-1,46,-1,-1,48,-1,-1,-1,-1,-1],"split_conditions":[2.2064762E8,4.762529E-1,7.7586204E-2,1.6012292E-1,4.3804785E2,-6.3260524E-3,1.0086E4,1.2950597E-3,-3.9766533E-3,1.1977647E7,6.8721003E-3,7.8337E4,7.950396E-1,1.895933E0,1.24990305E5,3.8024444E5,1.1E1,7.180455E-3,-6.047166E-4,1.3656793E-3,3.535225E-3,1.993823E-3,-1.3777817E-3,7.53E2,1E0,2.2541766E0,-8.1223985E-3,-3.7489973E-3,6.071E3,6.109438E-1,-6.7913746E-3,6.069098E5,6.7594426E-4,2.8893906E-1,-2.628429E-3,2.9166096E-1,9.479827E0,-3.7975132E-3,-5.366887E-4,-1.4229419E-3,1.805781E6,-8.0241193E-4,-5.446217E-3,7.58E5,1.6779462E-3,1.1026344E-3,3.616765E-3,6.655607E-4,-1.7970062E-3],"split_indices":[7,36,67,36,4,0,2,0,0,54,0,9,36,62,42,42,3,0,0,0,0,0,0,2,22,62,0,0,2,36,0,37,0,65,0,36,66,0,0,0,37,0,0,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.09E2,5.7E1,1.52E2,1.4E1,4.3E1,7E0,1.45E2,5E0,9E0,3.7E1,6E0,1.34E2,1.1E1,2.1E1,1.6E1,1.09E2,2.5E1,5E0,6E0,9E0,1.2E1,9E0,7E0,7.3E1,3.6E1,2E1,5E0,5E0,6.8E1,3.1E1,5E0,1.4E1,6E0,6.3E1,5E0,1.2E1,1.9E1,8E0,6E0,6E0,5.7E1,7E0,5E0,1.2E1,7E0,4.7E1,1E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-3.1433979E-3,1.4758463E-3,-5.9202863E-3,8.714467E-3,-3.9434943E-2,3.7415768E-3,4.8890305E-3,-1.0956245E-2,4.7072796E-3,1.0847009E-2,-2.906813E-2,-2.5661962E-2,3.705052E-2,-3.6004651E-3,3.338753E-2,-5.536481E-3,-7.697767E-3,-2.793137E-3,2.488674E-4,1.8157113E-4,3.681808E-3,2.056012E-2,-1.932853E-2,4.5591775E-2,-4.7382913E-3,-4.411669E-2,3.5432458E-2,3.2466598E-2,-5.5049174E-4,-3.95707E-2,1.243709E-2,6.808781E-2,2.092471E-2,7.668938E-4,-1.6907596E-3,-3.1062837E-3,-3.7223753E-4,3.2727767E-3,1.6380475E-4,4.382771E-2,-1.4706995E-4,-1.4876216E-2,-4.517054E-3,1.4968019E-3,-1.0235978E-3,8.6676585E-4,4.0431423E-3,-7.818483E-4,1.6356199E-3,1.223772E-3,3.928487E-3,-1.5493401E-3,1.0081225E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,15,17,19,21,23,-1,25,-1,-1,-1,-1,27,29,31,33,35,37,39,-1,41,43,45,47,-1,-1,-1,-1,-1,-1,49,-1,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1570949E-1,6.1051883E-2,0E0,7.9085745E-2,2.5381455E-1,3.9247535E-2,0E0,0E0,2.648902E-2,4.506041E-2,5.310867E-2,1.3844849E-2,1.6306344E-2,3.269733E-2,2.5342055E-2,0E0,4.0832072E-2,0E0,0E0,0E0,0E0,1.3247915E-2,3.392841E-2,2.1326631E-2,9.079855E-3,9.446375E-3,1.1911121E-2,1.02078635E-2,0E0,4.0380877E-2,1.3387731E-2,1.4001533E-2,1.0255548E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.1034813E-2,0E0,1.4387944E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,39,39,41,41],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,16,18,20,22,24,-1,26,-1,-1,-1,-1,28,30,32,34,36,38,40,-1,42,44,46,48,-1,-1,-1,-1,-1,-1,50,-1,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,7.960309E7,-5.9202863E-3,9.772E3,2.0426695E0,4.213523E0,4.8890305E-3,-1.0956245E-2,1.6384877E5,3.200388E2,2.2346368E-2,2.2385964E0,1.5166805E6,5.282343E5,1.0391348E7,-5.536481E-3,6.4945297E0,-2.793137E-3,2.488674E-4,1.8157113E-4,3.681808E-3,3.2144122E5,2.1602914E0,1.7863992E0,1E0,2.326087E0,2.46604E5,1.2247E4,-5.5049174E-4,1.7640048E0,1.1172265E9,2.0116584E8,8.909818E10,7.668938E-4,-1.6907596E-3,-3.1062837E-3,-3.7223753E-4,3.2727767E-3,1.6380475E-4,3.413613E7,-1.4706995E-4,2.2542E5,-4.517054E-3,1.4968019E-3,-1.0235978E-3,8.6676585E-4,4.0431423E-3,-7.818483E-4,1.6356199E-3,1.223772E-3,3.928487E-3,-1.5493401E-3,1.0081225E-3],"split_indices":[37,54,0,2,62,65,0,0,42,4,66,63,41,37,60,0,65,0,0,0,0,56,62,65,22,62,38,10,0,63,5,7,40,0,0,0,0,0,0,12,0,1,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,2.04E2,7E0,1.74E2,3E1,1.66E2,8E0,5E0,2.5E1,1.37E2,2.9E1,1.3E1,1.2E1,8.4E1,5.3E1,5E0,2.4E1,6E0,7E0,7E0,5E0,3.3E1,5.1E1,4E1,1.3E1,1.3E1,1.1E1,2.4E1,9E0,3.1E1,2E1,2E1,2E1,8E0,5E0,8E0,5E0,5E0,6E0,1.8E1,6E0,2.2E1,9E0,1.3E1,7E0,5E0,1.5E1,5E0,1.5E1,1.3E1,5E0,1.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-5.7126596E-3,4.5989407E-3,-8.4416345E-2,-4.032425E-3,8.203956E-3,-1.7172089E-2,-1.4942729E-2,5.5683374E-2,4.3197852E-4,2.2636035E-3,-6.8642266E-2,2.3717739E-2,7.0204255E-3,-6.6407677E-3,3.877141E-2,-6.384416E-3,7.3800713E-4,4.2031612E-2,-1.421052E-3,-5.636533E-2,1.7489925E-4,6.664301E-2,-2.26806E-3,3.2244406E-3,-4.327857E-4,7.755347E-4,-5.3041684E-3,-3.611011E-3,3.708236E-3,7.8020623E-4,9.910045E-2,-9.32166E-3,5.3823747E-2,6.677631E-3,2.58038E-3,-1.4188694E-2,2.6642876E-3,4.0320894E-3,7.912433E-4,1.0371138E-3,-1.0477108E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,-1,7,9,-1,11,13,-1,15,17,-1,19,21,-1,-1,23,-1,25,27,29,-1,-1,-1,-1,-1,31,-1,-1,33,35,37,-1,-1,39,-1,-1,-1,-1,-1],"loss_changes":[1.8217789E-1,6.315874E-2,3.816351E-1,0E0,7.082567E-2,7.179418E-2,0E0,7.249393E-2,4.5270875E-2,0E0,6.424651E-2,2.1524515E-2,0E0,4.779561E-2,6.481308E-2,0E0,0E0,2.0071026E-2,0E0,6.575671E-2,3.6092944E-2,3.131795E-2,0E0,0E0,0E0,0E0,0E0,3.9860494E-2,0E0,0E0,1.363463E-2,3.494537E-2,1.0532282E-2,0E0,0E0,2.6878392E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,10,10,11,11,13,13,14,14,17,17,19,19,20,20,21,21,27,27,30,30,31,31,32,32,35,35],"right_children":[2,4,6,-1,8,10,-1,12,14,-1,16,18,-1,20,22,-1,-1,24,-1,26,28,30,-1,-1,-1,-1,-1,32,-1,-1,34,36,38,-1,-1,40,-1,-1,-1,-1,-1],"split_conditions":[1.0129377E12,3.4E1,2.0459295E6,-4.032425E-3,2.93E2,1E1,-1.4942729E-2,1E0,9.4010997E-1,2.2636035E-3,2.2161171E0,1.2060649E6,7.0204255E-3,5.0238598E1,2.289485E1,-6.384416E-3,7.3800713E-4,4.6741E4,-1.421052E-3,3.461353E1,1.8327797E3,1E0,-2.26806E-3,3.2244406E-3,-4.327857E-4,7.755347E-4,-5.3041684E-3,1E0,3.708236E-3,7.8020623E-4,1.3226E4,1.067509E6,2.0142368E8,6.677631E-3,2.58038E-3,4.0996128E6,2.6642876E-3,4.0320894E-3,7.912433E-4,1.0371138E-3,-1.0477108E-3],"split_indices":[40,10,37,0,0,3,0,8,36,0,63,37,0,61,66,0,0,38,0,61,61,8,0,0,0,0,0,112,0,0,9,9,7,0,0,54,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.23E2,1.98E2,2.5E1,7E0,1.91E2,2E1,5E0,2.6E1,1.65E2,9E0,1.1E1,2E1,6E0,1.4E2,2.5E1,6E0,5E0,1.5E1,5E0,1.6E1,1.24E2,1.9E1,6E0,1E1,5E0,7E0,9E0,1.19E2,5E0,8E0,1.1E1,1.09E2,1E1,5E0,6E0,1.02E2,7E0,5E0,5E0,1.7E1,8.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-4.1589555E-3,5.774104E-4,-6.428754E-3,-5.1897173E-3,6.0646564E-2,2.2808727E-2,-1.6391983E-2,1.1538279E-2,-1.8859444E-2,3.0692192E-2,-1.1069704E-3,-8.66838E-2,-8.636342E-3,-3.4123939E-3,8.409387E-4,8.492326E-3,2.81312E-3,-1.0944988E-3,-5.8613955E-3,-8.242303E-4,-3.9552625E-2,2.6216177E-2,-2.9750608E-3,1.5867347E-2,-2.0628307E-2,3.1603E-4,-8.6068004E-2,2.7458514E-3,-1.1504516E-3,-1.4245713E-2,3.8199443E-2,-3.8492326E-2,3.0823474E-4,-1.6787702E-3,-6.876829E-3,-1.2030477E-3,1.4822229E-3,1.6165297E-3,-3.8457938E-2,5.11674E-2,-8.4449543E-4,-1.886259E-2,-3.612363E-3,1.02463935E-4,-3.2776769E-3,3.1696784E-3,5.1690254E-4,-2.5344468E-3,3.641606E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,7,9,11,-1,13,15,-1,17,19,-1,-1,21,-1,-1,-1,23,25,27,-1,29,31,-1,33,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3459691E-1,7.514285E-2,0E0,6.2475786E-2,2.6942298E-1,2.112079E-2,7.666381E-2,0E0,2.6926253E-2,2.8728586E-2,0E0,2.7225003E-2,3.112367E-2,0E0,0E0,3.6015186E-2,0E0,0E0,0E0,3.4705687E-2,5.621068E-2,1.8921854E-2,0E0,3.887598E-2,2.3338117E-2,0E0,3.189464E-2,0E0,1.0346037E-2,2.9913133E-2,2.4624698E-2,1.8763725E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.9856298E-2,1.4305986E-2,0E0,1.7746191E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,11,11,12,12,15,15,19,19,20,20,21,21,23,23,24,24,26,26,28,28,29,29,30,30,31,31,38,38,39,39,41,41],"right_children":[2,4,-1,6,8,10,12,-1,14,16,-1,18,20,-1,-1,22,-1,-1,-1,24,26,28,-1,30,32,-1,34,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,6.811E3,-6.428754E-3,3.77E2,5.099834E9,1.549821E6,5.5456768E1,1.1538279E-2,2.0968523E0,7.0940796E3,-1.1069704E-3,3.5035088E1,3.4231E4,-3.4123939E-3,8.409387E-4,4.1174043E3,2.81312E-3,-1.0944988E-3,-5.8613955E-3,2.3005404E0,9E0,1.2557897E7,-2.9750608E-3,2.8374527E2,4.56845E5,3.1603E-4,9.3941006E5,2.7458514E-3,2.8800612E9,1.2038681E6,3.135E3,2.5055168E9,3.0823474E-4,-1.6787702E-3,-6.876829E-3,-1.2030477E-3,1.4822229E-3,1.6165297E-3,4.1898E4,5.1160636E11,-8.4449543E-4,4.323016E2,-3.612363E-3,1.02463935E-4,-3.2776769E-3,3.1696784E-3,5.1690254E-4,-2.5344468E-3,3.641606E-4],"split_indices":[37,2,0,0,12,59,61,0,62,56,0,61,10,0,0,56,0,0,0,65,3,54,0,4,38,0,37,0,5,60,2,5,0,0,0,0,0,0,38,40,0,64,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.22E2,2.15E2,7E0,1.97E2,1.8E1,5.6E1,1.41E2,5E0,1.3E1,4.8E1,8E0,1.3E1,1.28E2,5E0,8E0,2.7E1,2.1E1,5E0,8E0,1.03E2,2.5E1,2.2E1,5E0,5.6E1,4.7E1,1.3E1,1.2E1,1E1,1.2E1,2.4E1,3.2E1,2.8E1,1.9E1,7E0,5E0,7E0,5E0,8E0,1.6E1,2.6E1,6E0,1.9E1,9E0,7E0,9E0,1.9E1,7E0,8E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.5625955E-4,4.9389903E-3,-9.660318E-3,-1.5060834E-3,8.502232E-2,1.5801826E-2,-2.755204E-2,1.29125295E-2,-1.1635475E-2,2.2198979E-2,-5.42587E-2,-8.097969E-3,-1.6848782E-2,-2.5934374E-3,1.2479585E-3,3.278067E-3,5.394601E-2,6.019934E-4,-5.464304E-3,-8.476648E-3,-4.0059066E-3,2.8103678E-3,-4.986312E-3,6.1344015E-3,3.7017085E-2,-4.2423084E-3,-1.3369819E-3,-2.7917542E-3,2.166412E-3,5.931863E-2,1.7200738E-2,5.3845225E-3,-2.9190737E-3,-3.9915066E-2,1.1569062E-2,3.1104524E-2,5.174464E-3,2.6065228E-3,2.249305E-3,2.0705555E-3,-1.829677E-3,-3.1784344E-3,-3.9849285E-4,8.813745E-4,-1.4556815E-3,-2.4375574E-4,3.3251646E-3,-8.940746E-4,1.4225397E-3,-4.1119815E-4,1.4935681E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,-1,19,-1,-1,21,23,-1,-1,25,-1,-1,27,-1,29,-1,31,-1,33,35,37,39,-1,41,43,45,-1,47,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3989938E-1,1.1780865E-1,0E0,9.60052E-2,2.9744202E-1,5.857253E-2,1.2286532E-1,0E0,1.9828586E-2,7.070373E-2,4.350802E-2,0E0,4.2910732E-2,0E0,0E0,3.3891987E-2,4.8652276E-2,0E0,0E0,4.0141482E-2,0E0,0E0,2.4732333E-2,0E0,1.5331276E-2,0E0,2.6613405E-2,0E0,2.3806749E-2,1.9030549E-2,8.7115625E-3,1.634912E-2,0E0,8.243067E-3,1.3630499E-2,1.6330214E-2,0E0,8.8379495E-3,0E0,0E0,1.1342616E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,15,15,16,16,19,19,22,22,24,24,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,37,37,40,40],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,-1,20,-1,-1,22,24,-1,-1,26,-1,-1,28,-1,30,-1,32,-1,34,36,38,40,-1,42,44,46,-1,48,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.966421E6,1.406168E3,-9.660318E-3,3.452399E8,1E0,8.836406E0,9.76592E-3,1.29125295E-2,2.1492538E0,1.7729497E2,2.0519513E-1,-8.097969E-3,6.3645E4,-2.5934374E-3,1.2479585E-3,5.322348E7,1.9356645E8,6.019934E-4,-5.464304E-3,7.53E2,-4.0059066E-3,2.8103678E-3,1.4693E4,6.1344015E-3,1.8352579E0,-4.2423084E-3,3.7553006E-1,-2.7917542E-3,1.4150975E4,9E0,1.5092975E6,2.6855614E0,-2.9190737E-3,2.0787193E2,3.9741936E0,1.514E3,5.174464E-3,2.1283955E-3,2.249305E-3,2.0705555E-3,8.54374E5,-3.1784344E-3,-3.9849285E-4,8.813745E-4,-1.4556815E-3,-2.4375574E-4,3.3251646E-3,-8.940746E-4,1.4225397E-3,-4.1119815E-4,1.4935681E-3],"split_indices":[37,61,0,7,13,65,47,0,65,61,36,0,10,0,0,5,7,0,0,2,0,0,9,0,65,0,47,0,60,3,37,67,0,4,65,2,0,47,0,0,38,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.32E2,2.27E2,5E0,2.11E2,1.6E1,1.27E2,8.4E1,5E0,1.1E1,1.17E2,1E1,5E0,7.9E1,5E0,6E0,7.4E1,4.3E1,5E0,5E0,7.1E1,8E0,9E0,6.5E1,7E0,3.6E1,5E0,6.6E1,7E0,5.8E1,1.6E1,2E1,6E1,6E0,1E1,4.8E1,1.1E1,5E0,1.4E1,6E0,9E0,5.1E1,5E0,5E0,4.2E1,6E0,6E0,5E0,8E0,6E0,4.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[2.6726713E-3,1.7757766E-2,-6.31514E-2,8.6292215E-3,1.18859455E-1,-1.0670232E-2,-1.047401E-2,1.6196389E-2,-2.8405158E-2,1.5059956E-2,-1.0024125E-3,7.2605005E-3,-2.976176E-3,1.346616E-3,4.7631703E-2,-4.007459E-3,-5.5527478E-3,-1.4789698E-2,3.265978E-2,-3.8559195E-2,6.816832E-3,3.027767E-2,5.7483767E-3,2.0679068E-3,-3.457698E-2,8.26122E-4,-1.7943389E-3,2.8126033E-3,1.141528E-4,-3.8239878E-4,-3.073024E-3,3.340083E-2,-7.3561636E-3,7.516622E-2,9.315601E-3,-2.9250942E-3,-2.1370995E-4,2.5978526E-3,-1.9483007E-3,-2.7315912E-3,2.7831406E-5,5.006666E-3,1.7292401E-3,1.7792967E-3,-2.8466273E-2,-1.9879069E-3,1.8133129E-3,-3.2457296E-2,1.20929405E-2,-2.867613E-3,3.166516E-4,2.6519404E-4,-3.211579E-3,1.1627082E-3,-7.6173374E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,19,21,-1,23,25,27,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1,37,39,41,43,-1,-1,-1,45,-1,47,-1,-1,-1,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0765758E-1,1.5586926E-1,3.070379E-1,4.4458587E-2,3.755532E-1,2.745494E-2,0E0,6.108762E-2,3.2604657E-2,0E0,0E0,1.3364388E-2,0E0,1.9909259E-2,4.743824E-2,0E0,2.924391E-2,9.650716E-3,8.165181E-3,7.727092E-3,3.038769E-2,3.223328E-2,0E0,0E0,9.659105E-3,0E0,0E0,0E0,0E0,0E0,0E0,1.9648293E-2,1.8992828E-2,8.308113E-3,2.661989E-2,0E0,0E0,0E0,1.8139286E-2,0E0,1.8813523E-2,0E0,0E0,0E0,1.19885905E-2,0E0,0E0,1.6729265E-2,1.1698488E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,24,24,31,31,32,32,33,33,34,34,38,38,40,40,44,44,47,47,48,48],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,20,22,-1,24,26,28,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1,38,40,42,44,-1,-1,-1,46,-1,48,-1,-1,-1,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1],"split_conditions":[6.627576E7,5.818E3,1.118307E6,4.283035E0,1E0,5.158987E6,-1.047401E-2,1.436E3,1.3316584E-1,1.5059956E-2,-1.0024125E-3,9.517795E7,-2.976176E-3,1.4395604E0,2.2762294E0,-4.007459E-3,2.260846E-2,1.9402143E0,9.3392765E11,4.3243725E5,7.3328E4,3.4480013E8,5.7483767E-3,2.0679068E-3,1.1817E4,8.26122E-4,-1.7943389E-3,2.8126033E-3,1.141528E-4,-3.8239878E-4,-3.073024E-3,5.1373015E6,2.8595803E5,2.951829E9,7.3871275E5,-2.9250942E-3,-2.1370995E-4,2.5978526E-3,1.7E2,-2.7315912E-3,1.21816E5,5.006666E-3,1.7292401E-3,1.7792967E-3,2.6180778E6,-1.9879069E-3,1.8133129E-3,9.8194115E1,3.3897146E8,-2.867613E-3,3.166516E-4,2.6519404E-4,-3.211579E-3,1.1627082E-3,-7.6173374E-4],"split_indices":[54,2,37,65,18,1,0,2,66,0,0,54,0,62,62,0,48,63,40,37,1,7,0,0,10,0,0,0,0,0,0,54,37,5,41,0,0,0,0,0,1,0,0,0,56,0,0,61,7,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.07E2,1.69E2,3.8E1,1.56E2,1.3E1,2.9E1,9E0,1.3E2,2.6E1,5E0,8E0,2.2E1,7E0,8.9E1,4.1E1,7E0,1.9E1,1.2E1,1E1,1E1,7.9E1,3.4E1,7E0,7E0,1.2E1,5E0,7E0,5E0,5E0,5E0,5E0,2.7E1,5.2E1,1E1,2.4E1,6E0,6E0,1.7E1,1E1,6E0,4.6E1,5E0,5E0,1.4E1,1E1,5E0,5E0,1.2E1,3.4E1,5E0,5E0,6E0,6E0,2.4E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-2.8811086E-3,3.3891108E-3,-1.1132482E-1,-5.9205135E-3,5.1206615E-2,-3.0387368E-4,-1.0528612E-2,7.788881E-3,-3.5973538E-2,7.951319E-3,1.6283898E-2,3.0668497E-2,-8.755475E-3,-6.538945E-2,3.3626117E-2,-1.75142E-2,4.8918862E-2,1.3233869E-2,6.7979634E-2,-6.337393E-4,-3.1593537E-3,-2.580304E-2,-1.3628457E-2,-9.653029E-4,6.45348E-2,4.1833118E-4,-2.6250528E-3,-3.707715E-4,3.9667594E-3,-2.3315977E-3,2.6364564E-3,4.3262364E-3,9.2949875E-4,8.68065E-3,-3.2141213E-3,-4.6918E-2,1.8605092E-3,1.1658216E-3,4.617365E-3,5.606643E-4,-2.1215864E-3,2.2911867E-2,-1.4354499E-2,-2.960094E-2,-4.8737368E-3,-9.254988E-4,3.0517327E-2,-1.9591753E-3,5.095543E-4,-4.6889495E-2,1.047762E-3,8.7972777E-4,2.6276393E-3,-3.6770098E-3,-4.472877E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,17,19,21,23,25,27,29,31,33,-1,35,-1,-1,37,-1,-1,-1,-1,39,-1,-1,-1,41,-1,43,-1,-1,-1,-1,-1,45,47,49,-1,-1,51,-1,-1,53,-1,-1,-1,-1,-1],"loss_changes":[1.4928982E-1,9.289707E-2,1.2992667E-1,7.238173E-2,1.2969959E-1,0E0,0E0,4.6070475E-2,1.1480235E-1,0E0,3.0637767E-2,3.227615E-2,3.218148E-2,3.2230574E-1,2.931996E-2,1.3669425E-2,2.6114583E-2,2.2934172E-2,1.43986195E-2,3.8784523E-2,0E0,4.763562E-2,0E0,0E0,1.1140697E-2,0E0,0E0,0E0,0E0,1.5803391E-2,0E0,0E0,0E0,1.868671E-2,0E0,2.0816922E-2,0E0,0E0,0E0,0E0,0E0,1.17168315E-2,1.4442515E-2,1.9534314E-2,0E0,0E0,7.761022E-3,0E0,0E0,1.6421273E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,24,24,29,29,33,33,35,35,41,41,42,42,43,43,46,46,49,49],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,18,20,22,24,26,28,30,32,34,-1,36,-1,-1,38,-1,-1,-1,-1,40,-1,-1,-1,42,-1,44,-1,-1,-1,-1,-1,46,48,50,-1,-1,52,-1,-1,54,-1,-1,-1,-1,-1],"split_conditions":[1.2754595E8,3.625E3,9.0292144E-1,1.2060649E6,1.60756E5,-3.0387368E-4,-1.0528612E-2,7.701429E5,3.49917E2,7.951319E-3,1.6362E4,3.286688E-2,4.4085712E0,7.4999895E9,3.9712732E7,2.2179742E0,2.41E3,7.6696295E-1,1.9760254E-1,2.8267702E-1,-3.1593537E-3,2.1E1,-1.3628457E-2,-9.653029E-4,1.564E3,4.1833118E-4,-2.6250528E-3,-3.707715E-4,3.9667594E-3,5.87E2,2.6364564E-3,4.3262364E-3,9.2949875E-4,1.00473E5,-3.2141213E-3,4.17027E6,1.8605092E-3,1.1658216E-3,4.617365E-3,5.606643E-4,-2.1215864E-3,1.5960511E5,6.540767E0,2.248226E6,-4.8737368E-3,-9.254988E-4,1E0,-1.9591753E-3,5.095543E-4,2.2032669E0,1.047762E-3,8.7972777E-4,2.6276393E-3,-3.6770098E-3,-4.472877E-4],"split_indices":[54,2,36,37,38,0,0,57,61,0,9,48,65,5,54,62,0,36,48,47,0,8,0,0,2,0,0,0,0,0,0,0,0,38,0,37,0,0,0,0,0,42,66,37,0,0,18,0,0,62,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.18E2,2.07E2,1.1E1,1.74E2,3.3E1,6E0,5E0,1.2E2,5.4E1,7E0,2.6E1,5E1,7E1,3.8E1,1.6E1,1.3E1,1.3E1,3.5E1,1.5E1,6.2E1,8E0,3.3E1,5E0,6E0,1E1,8E0,5E0,5E0,8E0,2.6E1,9E0,1E1,5E0,5.5E1,7E0,2.5E1,8E0,5E0,5E0,2E1,6E0,3.4E1,2.1E1,2E1,5E0,5E0,2.9E1,1E1,1.1E1,1.5E1,5E0,2E1,9E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[2.0187846E-3,8.834806E-3,-7.668469E-2,-3.773953E-2,1.5552967E-2,8.04573E-4,-1.069899E-2,2.423526E-3,-6.204873E-2,2.2999737E-2,-3.908808E-2,-1.3054989E-3,1.1864178E-3,-6.660687E-3,-2.0508101E-2,1.7286018E-2,7.77939E-3,-4.913723E-3,-7.811137E-3,1.5264878E-3,-2.5770057E-3,3.7976753E-2,1.7471678E-3,1.0123043E-3,-1.423696E-3,1.6168866E-2,8.762823E-2,7.542004E-3,-3.7055255E-3,4.6175715E-2,-1.7300827E-2,2.0869474E-3,6.4917547E-3,1.6517268E-2,-3.5949767E-2,5.8835354E-2,-7.8652476E-5,2.1015606E-3,-2.8926053E-3,2.1825805E-2,-2.0085059E-3,-3.2504124E-3,7.7542185E-4,1.133971E-3,4.0396284E-3,-7.733872E-4,1.3396376E-3,-1.3097867E-3,1.3448753E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,-1,-1,-1,19,21,-1,-1,23,-1,-1,25,27,-1,-1,29,31,33,-1,35,37,-1,-1,39,41,43,-1,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1440375E-1,6.191416E-2,1.8749571E-1,5.614379E-2,7.08857E-2,8.396113E-3,0E0,0E0,5.886598E-2,1.1549563E-1,3.9568238E-2,0E0,0E0,0E0,2.4978967E-2,4.7195654E-2,0E0,0E0,9.722183E-3,0E0,0E0,6.6799164E-2,3.8772658E-2,0E0,0E0,4.599328E-2,3.290236E-2,3.1821802E-2,0E0,1.46131925E-2,1.799957E-2,0E0,0E0,2.1386292E-2,2.3214051E-2,1.4203116E-2,0E0,7.6905945E-3,0E0,1.7883908E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,14,14,15,15,18,18,21,21,22,22,25,25,26,26,27,27,29,29,30,30,33,33,34,34,35,35,37,37,39,39],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,-1,-1,-1,20,22,-1,-1,24,-1,-1,26,28,-1,-1,30,32,34,-1,36,38,-1,-1,40,42,44,-1,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.4E1,6.97E2,7.4211675E5,4.0171E4,2.9549628E7,6.15175E5,-1.069899E-2,2.423526E-3,4.0573265E1,6.923E3,2.0235784E0,-1.3054989E-3,1.1864178E-3,-6.660687E-3,8.1446184E7,2.9722316E0,7.77939E-3,-4.913723E-3,2.1807466E0,1.5264878E-3,-2.5770057E-3,9.0770394E-1,9.4655323E-1,1.0123043E-3,-1.423696E-3,1.0428423E9,9.846896E4,4.6385403E5,-3.7055255E-3,1.0661045E6,1.3843815E6,2.0869474E-3,6.4917547E-3,2.391641E6,1.4852E4,5.7296395E6,-7.8652476E-5,9.933566E5,-2.8926053E-3,2.8893906E-1,-2.0085059E-3,-3.2504124E-3,7.7542185E-4,1.133971E-3,4.0396284E-3,-7.733872E-4,1.3396376E-3,-1.3097867E-3,1.3448753E-3],"split_indices":[3,2,37,12,60,1,0,0,61,2,62,0,0,0,5,66,0,0,62,0,0,36,36,0,0,5,42,42,0,41,37,0,0,41,10,54,0,37,0,65,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.11E2,1.95E2,1.6E1,2.4E1,1.71E2,1.1E1,5E0,5E0,1.9E1,1.51E2,2E1,5E0,6E0,6E0,1.3E1,1.46E2,5E0,6E0,1.4E1,5E0,8E0,6.2E1,8.4E1,6E0,8E0,4.4E1,1.8E1,7.9E1,5E0,2.3E1,2.1E1,1E1,8E0,6.6E1,1.3E1,1.8E1,5E0,1.5E1,6E0,6.1E1,5E0,8E0,5E0,8E0,1E1,9E0,6E0,6E0,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-4.1710395E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics index 6b55e977b..2ea2429b6 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-aws_photon.metrics @@ -1,116 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,duration_max,0.094851375,3080.1935483870966,5856.877323993372,91.0,923.25,1433.0,2713.25,56110.0 -1,input_bytesRead_mean,0.083778396,34614190.90219704,41416129.188791394,266261.23312883434,5459412.016398072,16533685.465698259,53804870.10884919,215759776.27180278 -2,executorCPUTime_mean,0.06275888,484.9524522296494,994.3035100356893,6.285198555956678,72.97144430612424,172.3301168927889,511.25156722100104,10856.768406961179 -3,resultSize_max,0.051463157,367273.03225806454,2247437.485686325,9691.0,13663.5,16271.0,20763.5,33908394.0 -4,scan_time,0.032323215,702452.7451612903,2614923.258259799,104.0,41115.75,135025.0,427800.75,38566189.0 -5,peakExecutionMemory_max,0.030858058,422880427.65806454,402352807.44425225,157630402.0,210582455.25,304087040.0,453001400.0,3751805616.0 -6,jvmGCTime_mean,0.030563733,2.9618926687032987,4.020258699557162,0.0,0.5477208341866339,1.5502085984742942,3.7139701928907503,26.954509177972866 -7,numTasks_sum,0.0285184,1155.8451612903225,1236.5684255604363,5.0,369.5,745.0,1542.25,11722.0 -8,executorDeserializeCPUTime_mean,0.027397223,2.424006050263326,1.5480948682100073,1.0994152046783625,1.85294797974374,2.145231677576691,2.4152654335815518,16.50413223140496 -9,scan_bw,0.026037991,1157664.506403393,1384794.4128176263,22989.60626746936,363898.5144438856,693542.2221841079,1377425.7566807903,12673979.523529412 -10,duration_mean,0.02582579,859.128933583903,1472.5642216030267,32.6,226.48981733741198,400.2075833084706,850.7909051708319,14227.349397590362 -11,sw_writeTime_mean,0.015917424,74.21249570226658,299.6079575558875,0.0,0.32014410156611994,2.9045399304817834,24.63743722003529,3606.1337732453508 -12,duration_min,0.01565841,12.245161290322581,17.387769363454428,6.0,7.0,8.0,9.0,225.0 -13,sw_recordsWritten_sum,0.015468385,1598284434.735484,4325298137.228029,264.0,1418011.5,80786347.0,926538005.75,40948492132.0 -14,executorDeserializeTime_mean,0.014171251,3.7019762156661162,9.300108094794437,1.0058479532163742,1.7107390104056064,1.9774027165036006,2.272371614762325,97.46767757382283 -15,sr_fetchWaitTime_mean,0.012338699,16.171628372914686,49.85360041382992,0.0,0.06290047980188826,3.6375977178161643,12.382437359981768,442.2783443311338 -16,data_size,0.010960059,568346723573.9452,1721345313838.1055,57333908.0,22262058221.5,90621097739.5,371990718621.0,21680680059621.0 -17,shuffle_write_bw,0.0093427515,294988.3149564088,206444.54891989162,0.0,142335.9011306867,287941.35018557333,411282.75758824224,1258785.8495575222 -18,cache_hit_ratio,0.0082903225,0.6234281162221763,0.3125084127792779,2.3756618577933566e-05,0.4048859664027882,0.7021331510333857,0.911661358433798,1.0 -19,input_recordsRead_sum,0.008157277,6926286337.6967745,16944905504.544098,600000.0,372126082.5,1646770012.0,6110635272.5,230401504404.0 -20,shuffle_read_bw,0.007961464,75761848.45416413,1256128268.2786763,0.0,150817.65029961508,488655.9100812153,1342448.866631469,22113997979.0 -21,sr_localBytesReadRatio,0.007858885,0.11763505588045406,0.2532205881427819,0.0,0.0009792636137592992,0.032991256423365936,0.12053135013917013,2.359115013482412 -22,sqlOp_AQEShuffleRead,0.0077882945,0.7806451612903226,0.41447854308610055,0.0,1.0,1.0,1.0,1.0 -23,sr_remoteBytesRead_mean,0.0071323127,10311397.907227328,25904396.59503362,51.311688311688314,134832.20991280992,1168469.420949517,5610163.5331268115,185061465.67966408 -24,sqlOp_Sort,0.0068246843,0.27419354838709675,0.4468282627569248,0.0,0.0,0.0,1.0,1.0 -25,sr_localBlocksFetched_sum,0.0063881576,16988.609677419354,25018.356493130832,0.0,310.5,7484.5,24031.5,142772.0 -26,sr_totalBytesReadRatio,0.005597536,0.6291743304032147,1.3628151304257985,3.800158290722679e-07,0.006041410216258603,0.2122817799381812,0.7754056426179594,18.862443965831122 -27,duration_sum,0.0042502503,1303312.477419355,2860079.6431399835,163.0,97331.0,328116.0,1057554.75,27072068.0 -28,sr_localBytesRead_mean,0.0041107833,2092517.811769234,4949978.624893702,0.0,23510.989846290275,251194.81541218638,1207058.518057883,30924005.30712166 -29,sw_bytesWritten_mean,0.0034306108,11694414.49800359,28993820.0534498,49.335949496672924,67474.51700024115,1251171.401178395,6400153.625044058,211482650.29994002 -30,resultSerializationTime_sum,0.003300473,20.838709677419356,39.54449116503082,0.0,0.0,1.5,25.0,274.0 -31,sr_totalBytesRead_mean,0.003173616,12403915.718996564,29902823.960300017,58.76623376623377,161124.53418283595,1545843.0242091469,7775367.934015937,211484163.58068386 -32,sr_remoteBlocksFetched_sum,0.002663965,117736.82903225806,172191.79281083873,3.0,1697.75,52133.5,165851.75,999894.0 -33,sqlOp_Window,0.002419913,0.12580645161290321,0.3321672892517517,0.0,0.0,0.0,0.0,1.0 -34,sqlOp_Filter,0.0021850606,0.47096774193548385,0.4999634604040792,0.0,0.0,0.0,1.0,1.0 -35,sr_remoteBytesReadRatio,0.002155173,0.5115392745227606,1.1868052697969418,3.3181050622420564e-07,0.004988142514446131,0.1524366852830112,0.5779331591564784,16.50332895234871 -36,sqlOp_SortMergeJoin,0.0019975656,0.535483870967742,0.4995456752050157,0.0,0.0,1.0,1.0,1.0 -37,sw_bytesWrittenRatio,0.0019910757,0.5648668864092706,1.3161095570909536,3.800158290722679e-07,0.00440722715697505,0.15641603032642426,0.6834818879961551,18.780553045389944 -38,executorRunTime_mean,0.0014312438,843.4557617191201,1472.0381565091477,17.407942238267147,204.14190042775138,382.83865152638316,837.6448190197581,14216.539491298527 -39,sqlOp_Subquery,0.0013303771,0.25806451612903225,0.4382771459540444,0.0,0.0,0.0,1.0,1.0 -40,sqlOp_TakeOrderedAndProject,0.0010956501,0.6387096774193548,0.4811511408013016,0.0,0.0,1.0,1.0,1.0 -41,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -42,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_HashAggregate,0.0,0.9903225806451613,0.09805499648850532,0.0,1.0,1.0,1.0,1.0 -45,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -53,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -57,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -76,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -78,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,diskBytesSpilled_mean,0.0,508837.0986981265,6334276.584243264,0.0,0.0,0.0,0.0,105076448.0334672 -82,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 -84,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -87,diskBytesSpilledRatio,0.0,0.008646349817617971,0.11400204273633267,0.0,0.0,0.0,0.0,1.8994929430148348 -88,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -89,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -90,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -91,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -95,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -96,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_BroadcastHashJoin,0.0,0.9290322580645162,0.2571861028762713,0.0,1.0,1.0,1.0,1.0 -98,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -103,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_BroadcastNestedLoopJoin,0.0,0.03870967741935484,0.1932140342025925,0.0,0.0,0.0,0.0,1.0 -105,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -107,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -108,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -109,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -111,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -112,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -113,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,duration_max,0.09690424,3080.1935483870966,5856.877323993371,91.0,923.25,1433.0,2713.25,56110.0 +1,input_bytesRead_mean,0.08261457,34614190.90219704,41416129.18879139,266261.23312883434,5459412.016398072,16533685.465698259,53804870.10884919,215759776.27180278 +2,executorCPUTime_mean,0.06386489,484.9524522296494,994.3035100356892,6.285198555956678,72.97144430612424,172.3301168927889,511.25156722100104,10856.768406961179 +3,resultSize_max,0.050946232,367273.03225806454,2247437.485686325,9691.0,13663.5,16271.0,20763.5,33908394.0 +4,jvmGCTime_mean,0.038380563,2.9618926687032983,4.020258699557162,0.0,0.5477208341866339,1.5502085984742942,3.7139701928907503,26.954509177972866 +5,numTasks_sum,0.03043986,1155.8451612903225,1236.5684255604363,5.0,369.5,745.0,1542.25,11722.0 +6,executorDeserializeCPUTime_mean,0.026885407,2.4240060502633263,1.5480948682100073,1.0994152046783625,1.85294797974374,2.145231677576691,2.4152654335815518,16.50413223140496 +7,peakExecutionMemory_max,0.02484491,422880427.65806454,402352807.44425225,157630402.0,210582455.25,304087040.0,453001400.0,3751805616.0 +8,scan_bw,0.023939203,1157664.506403393,1384794.4128176263,22989.60626746936,363898.5144438856,693542.2221841079,1377425.7566807903,12673979.523529412 +9,duration_mean,0.023744209,859.128933583903,1472.5642216030267,32.6,226.48981733741198,400.2075833084706,850.7909051708319,14227.349397590362 +10,sw_recordsWritten_sum,0.023438096,1598284434.735484,4325298137.228029,264.0,1418011.5,80786347.0,926538005.75,40948492132.0 +11,scan_time,0.018704882,702452.7451612903,2614923.2582597984,104.0,41115.75,135025.0,427800.75,38566189.0 +12,input_recordsRead_sum,0.016858952,6926286337.6967745,16944905504.544094,600000.0,372126082.5,1646770012.0,6110635272.5,230401504404.0 +13,sw_writeTime_mean,0.015492341,74.21249570226657,299.60795755588754,0.0,0.32014410156611994,2.9045399304817834,24.63743722003529,3606.1337732453508 +14,data_size,0.013629718,568346723573.9452,1721345313838.106,57333908.0,22262058221.5,90621097739.5,371990718621.0,21680680059621.0 +15,shuffle_write_bw,0.0120088365,294988.3149564088,206444.5489198916,0.0,142335.9011306867,287941.35018557333,411282.75758824224,1258785.8495575222 +16,executorDeserializeTime_mean,0.011114314,3.7019762156661153,9.300108094794437,1.0058479532163742,1.7107390104056064,1.9774027165036006,2.272371614762325,97.46767757382283 +17,sr_remoteBytesRead_mean,0.010884746,10311397.907227326,25904396.59503362,51.311688311688314,134832.20991280992,1168469.420949517,5610163.5331268115,185061465.67966408 +18,duration_min,0.010758477,12.245161290322581,17.387769363454428,6.0,7.0,8.0,9.0,225.0 +19,cache_hit_ratio,0.008505135,0.6234281162221763,0.3125084127792779,2.3756618577933566e-05,0.4048859664027882,0.7021331510333857,0.911661358433798,1.0 +20,shuffle_read_bw,0.008313608,75761848.45416412,1256128268.2786763,0.0,150817.65029961508,488655.9100812153,1342448.866631469,22113997979.0 +21,sr_fetchWaitTime_mean,0.007989078,16.171628372914686,49.85360041382992,0.0,0.06290047980188826,3.6375977178161643,12.382437359981768,442.2783443311338 +22,sr_localBytesReadRatio,0.0070928996,0.11763505588045403,0.2532205881427819,0.0,0.0009792636137592992,0.032991256423365936,0.12053135013917013,2.359115013482412 +23,sqlOp_Sort,0.0049014827,0.27419354838709675,0.4468282627569248,0.0,0.0,0.0,1.0,1.0 +24,duration_sum,0.0048065316,1303312.477419355,2860079.6431399826,163.0,97331.0,328116.0,1057554.75,27072068.0 +25,sqlOp_Filter,0.004759093,0.47096774193548385,0.4999634604040792,0.0,0.0,0.0,1.0,1.0 +26,sr_totalBytesRead_mean,0.004322892,12403915.718996564,29902823.960300017,58.76623376623377,161124.53418283595,1545843.0242091469,7775367.934015937,211484163.58068386 +27,sr_localBlocksFetched_sum,0.0043207793,16988.609677419354,25018.356493130836,0.0,310.5,7484.5,24031.5,142772.0 +28,sqlOp_AQEShuffleRead,0.004214536,0.7806451612903226,0.41447854308610055,0.0,1.0,1.0,1.0,1.0 +29,sqlOp_Window,0.004019247,0.12580645161290321,0.3321672892517517,0.0,0.0,0.0,0.0,1.0 +30,sw_bytesWritten_mean,0.0037503697,11694414.498003587,28993820.0534498,49.335949496672924,67474.51700024115,1251171.401178395,6400153.625044058,211482650.29994002 +31,sqlOp_Subquery,0.003677518,0.25806451612903225,0.4382771459540444,0.0,0.0,0.0,1.0,1.0 +32,sr_localBytesRead_mean,0.003562748,2092517.811769234,4949978.624893702,0.0,23510.989846290275,251194.81541218638,1207058.518057883,30924005.30712166 +33,sqlOp_SortMergeJoin,0.0035121734,0.535483870967742,0.4995456752050157,0.0,0.0,1.0,1.0,1.0 +34,sr_totalBytesReadRatio,0.0033915085,0.6291743304032146,1.3628151304257985,3.800158290722679e-07,0.006041410216258603,0.2122817799381812,0.7754056426179594,18.862443965831122 +35,resultSerializationTime_sum,0.0032920027,20.838709677419356,39.54449116503081,0.0,0.0,1.5,25.0,274.0 +36,executorRunTime_mean,0.0030267218,843.4557617191199,1472.038156509148,17.407942238267147,204.14190042775138,382.83865152638316,837.6448190197581,14216.539491298527 +37,sr_remoteBytesReadRatio,0.002803222,0.5115392745227607,1.1868052697969418,3.3181050622420564e-07,0.004988142514446131,0.1524366852830112,0.5779331591564784,16.50332895234871 +38,sw_bytesWrittenRatio,0.0025004845,0.5648668864092706,1.3161095570909536,3.800158290722679e-07,0.00440722715697505,0.15641603032642426,0.6834818879961551,18.780553045389944 +39,sqlOp_TakeOrderedAndProject,0.0021593936,0.6387096774193548,0.4811511408013016,0.0,0.0,1.0,1.0,1.0 +40,sr_remoteBlocksFetched_sum,0.0014324969,117736.82903225806,172191.7928108387,3.0,1697.75,52133.5,165851.75,999894.0 +41,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +42,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +43,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +44,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +46,sqlOp_BroadcastHashJoin,0.0,0.9290322580645162,0.2571861028762713,0.0,1.0,1.0,1.0,1.0 +47,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +49,sqlOp_HashAggregate,0.0,0.9903225806451613,0.09805499648850532,0.0,1.0,1.0,1.0,1.0 +50,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +51,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,diskBytesSpilledRatio,0.0,0.008646349817617971,0.11400204273633266,0.0,0.0,0.0,0.0,1.8994929430148348 +56,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +59,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +60,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +61,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +62,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +63,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 +64,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,diskBytesSpilled_mean,0.0,508837.0986981265,6334276.584243264,0.0,0.0,0.0,0.0,105076448.0334672 +66,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_BroadcastNestedLoopJoin,0.0,0.03870967741935484,0.1932140342025925,0.0,0.0,0.0,0.0,1.0 +76,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg index 1bfd976e9..1a5413a19 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"96"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0484221466","gamma":"0.0615322217","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0484221466","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"6","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.0615322217","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.626046658"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0477568395","gamma":"0.00921756867","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0477568395","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"10","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00921756867","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.704420388"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json index 591118a8d..5e67ebb08 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-azure","platform_databricks-aws","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"96"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[5.502359E-3,-4.183997E-1,4.292986E-1,-5.4472244E-1,2.4702572E-3,-1.474169E-2,5.6962496E-1,-2.095471E-1,-6.9834214E-1,1.2778612E-2,-9.389585E-3,3.1878093E-1,8.291788E-1,-1.9249089E-2,5.197953E-3,-3.7156157E-2,-1.5929278E-2,-2.8126498E-3,4.210136E-1,4.4440117E-2,2.2020187E-2,1.4502673E-2,3.0717304E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,17,19,-1,-1,-1,-1,-1,21,-1,-1,-1,-1],"loss_changes":[1.8683372E1,2.7659492E0,5.5274935E0,1.9132452E0,7.185921E-1,0E0,2.6004057E0,8.766462E-1,4.3567657E-1,0E0,0E0,9.491296E-1,2.7918148E-1,0E0,0E0,0E0,0E0,0E0,3.025608E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,11,11,12,12,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,18,20,-1,-1,-1,-1,-1,22,-1,-1,-1,-1],"split_conditions":[3.3433047E-1,2.561633E3,2.80369E5,4.828443E-1,6.1358623E9,-1.474169E-2,1.3536367E0,1.3564575E5,1.1653038E-2,1.2778612E-2,-9.389585E-3,8.184232E5,2.326E4,-1.9249089E-2,5.197953E-3,-3.7156157E-2,-1.5929278E-2,-2.8126498E-3,1.5791555E9,4.4440117E-2,2.2020187E-2,1.4502673E-2,3.0717304E-2],"split_indices":[39,52,1,27,5,0,39,28,38,0,0,32,9,0,0,0,0,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,5.1E1,5.1E1,3.9E1,1.2E1,8E0,4.3E1,1.3E1,2.6E1,5E0,7E0,2.3E1,2E1,8E0,5E0,2.1E1,5E0,5E0,1.8E1,1.5E1,5E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.9554853E-2,-2.866141E-1,5.9643453E-1,-4.1968364E-1,2.779474E-1,8.0592066E-1,2.2149436E-1,-5.895617E-1,-1.7247047E-1,2.6456462E-2,2.0282199E-3,1.882251E-2,8.9525235E-1,2.223707E-3,1.6145838E-2,-6.751509E-1,-5.830744E-3,-1.472395E-2,-6.614812E-2,2.4594214E-2,4.8291817E-2,-4.1558552E-1,-4.056774E-2,2.741372E-3,-9.539064E-3,-1.1581288E-2,-2.4570683E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,-1,23,-1,-1,25,-1,-1,-1,-1,-1],"loss_changes":[1.8697906E1,4.999835E0,2.8061934E0,2.1241112E0,8.0085194E-1,5.464678E-1,2.7950114E-1,1.1634722E0,3.0171287E-1,0E0,0E0,0E0,2.4453354E-1,0E0,0E0,7.494488E-1,0E0,0E0,2.3604053E-1,0E0,0E0,7.305956E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,18,18,21,21],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,-1,24,-1,-1,26,-1,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,3.5328901E3,1.4046429E1,1.1816701E-2,8.69374E5,2.1146134E5,1.5959719E-1,2.2794224E3,1.2239595E3,2.6456462E-2,2.0282199E-3,1.882251E-2,5.403511E8,2.223707E-3,1.6145838E-2,3.566468E5,-5.830744E-3,-1.472395E-2,4.5843E7,2.4594214E-2,4.8291817E-2,5.189068E-1,-4.056774E-2,2.741372E-3,-9.539064E-3,-1.1581288E-2,-2.4570683E-2],"split_indices":[39,52,54,38,29,28,38,52,4,0,0,0,7,0,0,28,0,0,45,0,0,27,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,6.4E1,3.7E1,5.2E1,1.2E1,2.3E1,1.4E1,3E1,2.2E1,5E0,7E0,5E0,1.8E1,6E0,8E0,2.5E1,5E0,9E0,1.3E1,5E0,1.3E1,1.1E1,1.4E1,7E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.37868E-2,-3.8094634E-1,4.4869643E-1,-4.654821E-1,1.693616E-2,-2.972376E-2,6.430504E-1,-3.0609107E-1,-7.597651E-1,-1.6957164E-2,1.5969442E-1,4.183384E-1,8.478415E-1,-3.2531083E-2,-1.8616864E-1,-1.7008327E-2,-4.2639267E-2,-2.045118E-3,1.5009095E-2,2.2929884E-2,9.307084E-3,2.8417723E-2,5.211278E-2,-2.3338985E-1,2.0150626E-3,-1.682601E-2,-7.7115786E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,25,-1,-1,-1],"loss_changes":[2.080793E1,3.9926014E0,5.505103E0,2.3941832E0,0E0,1.1485775E0,1.5781136E0,1.5415843E0,6.2838554E-1,0E0,3.7721646E-1,1.8360972E-1,6.199684E-1,0E0,3.3239818E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5611625E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,23,23],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,26,-1,-1,-1],"split_conditions":[1.517606E8,7.621429E1,5.1707494E-1,5.070136E7,1.693616E-2,3.0962205E-2,7.3845965E8,1.497E3,5.3323513E-1,-1.6957164E-2,4.349136E6,5.5E1,2.5172096E-1,-3.2531083E-2,2.3514317E8,-1.7008327E-2,-4.2639267E-2,-2.045118E-3,1.5009095E-2,2.2929884E-2,9.307084E-3,2.8417723E-2,5.211278E-2,1E0,2.0150626E-3,-1.682601E-2,-7.7115786E-3],"split_indices":[12,56,39,45,0,38,7,2,27,0,32,3,38,0,7,0,0,0,0,0,0,0,0,16,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,6.1E1,5.8E1,5.5E1,6E0,1.7E1,4.1E1,3.7E1,1.8E1,6E0,1.1E1,2.1E1,2E1,8E0,2.9E1,5E0,1.3E1,5E0,6E0,1.6E1,5E0,1.1E1,9E0,2.4E1,5E0,8E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.6765657E-2,-3.6046076E-1,4.706868E-1,-4.9366096E-1,3.70798E-2,1.3346654E-1,8.021042E-1,-6.9482416E-1,-1.9231817E-1,1.6042082E-2,-1.241968E-1,-7.0611513E-3,3.1076303E-1,2.8137453E-2,5.2361015E-2,-7.524503E-1,-1.5480688E-2,-2.7942953E-1,4.893974E-3,-1.2024528E-2,-7.5415144E-4,4.3494273E-3,2.1696799E-2,-2.1543229E-2,-3.9664254E-2,-2.173256E-2,-7.6185386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.938807E1,3.830884E0,5.0339365E0,3.139804E0,9.427007E-1,1.233734E0,7.911358E-1,4.511776E-1,6.1606055E-1,0E0,1.7141548E-1,0E0,4.1482103E-1,0E0,0E0,1.8929195E-1,0E0,3.016404E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,15,15,17,17],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.095874E6,7.388E3,1.2480776E0,2.667836E6,6.15786E5,8.184232E5,1.5605432E9,3.159013E3,2.2781514E8,1.6042082E-2,1.309E4,-7.0611513E-3,3.042433E6,2.8137453E-2,5.2361015E-2,4.4E1,-1.5480688E-2,2.6326587E-2,4.893974E-3,-1.2024528E-2,-7.5415144E-4,4.3494273E-3,2.1696799E-2,-2.1543229E-2,-3.9664254E-2,-2.173256E-2,-7.6185386E-3],"split_indices":[48,2,39,32,29,32,7,4,7,0,9,0,1,0,0,10,0,38,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.16E2,7.1E1,4.5E1,5.3E1,1.8E1,2.3E1,2.2E1,3.1E1,2.2E1,6E0,1.2E1,9E0,1.4E1,1.4E1,8E0,2.6E1,5E0,1.7E1,5E0,5E0,7E0,6E0,8E0,6E0,2E1,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.5259627E-2,-3.550146E-1,5.242485E-1,-1.8318953E-1,-6.585757E-1,2.6704225E-1,7.9269874E-1,-3.5900474E-1,1.9087869E-1,-1.5058845E-2,-3.72979E-2,2.1803634E-2,9.0904914E-2,2.8770017E-2,5.16127E-2,-6.0800236E-1,-1.6675615E-1,2.526564E-2,-2.8975375E-3,-2.7577023E-3,1.4407447E-2,-3.612426E-2,-1.9912634E-2,-1.1403946E-2,-2.6265395E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1815578E1,3.6869936E0,2.9644823E0,3.2253366E0,7.698164E-1,7.6595235E-1,5.600233E-1,1.4783034E0,1.3586224E0,0E0,0E0,0E0,4.4467366E-1,0E0,0E0,7.344484E-2,2.2056937E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,16,16],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,5.4875424E7,1.5364555E0,1.0668866E3,5.3323513E-1,9.48E2,1.5605432E9,2.667836E6,9.163856E0,-1.5058845E-2,-3.72979E-2,2.1803634E-2,6.9015497E-1,2.8770017E-2,5.16127E-2,2.3E1,2.3514317E8,2.526564E-2,-2.8975375E-3,-2.7577023E-3,1.4407447E-2,-3.612426E-2,-1.9912634E-2,-1.1403946E-2,-2.6265395E-4],"split_indices":[50,45,42,52,27,0,7,32,53,0,0,0,39,0,0,3,7,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,7.2E1,4.5E1,4.7E1,2.5E1,2.4E1,2.1E1,3.2E1,1.5E1,7E0,1.8E1,1.1E1,1.3E1,1.4E1,7E0,1.3E1,1.9E1,6E0,9E0,8E0,5E0,6E0,7E0,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[6.100069E-3,-3.2587263E-1,4.6935153E-1,-5.027516E-1,-9.09635E-2,2.5414613E-1,6.922206E-1,-6.7788434E-1,-1.8902242E-1,1.6482896E-1,-2.7064896E-1,3.3964792E-1,-4.235378E-3,3.7016507E-2,2.0235999E-2,-1.836257E-2,-3.6625154E-2,-1.3720467E-2,-4.5833117E-3,-4.852346E-3,2.3762768E-2,-1.6694881E-2,-3.14397E-3,4.2859972E-1,4.867191E-3,1.273478E-2,2.4684701E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,25,-1,-1,-1],"loss_changes":[1.768465E1,2.7381506E0,2.1262808E0,1.9597464E0,1.4234765E0,7.891319E-1,1.3626385E-1,2.7576733E-1,1.06705725E-1,1.172108E0,2.5211108E-1,4.0361476E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.759071E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,26,-1,-1,-1],"split_conditions":[1.8095805E5,3.3041668E1,1.3536367E0,2.9282052E6,6.15786E5,1.8854474E-1,2.1685E4,4.4E1,2.207872E-1,3.09E2,2.794308E3,3.1293203E1,-4.235378E-3,3.7016507E-2,2.0235999E-2,-1.836257E-2,-3.6625154E-2,-1.3720467E-2,-4.5833117E-3,-4.852346E-3,2.3762768E-2,-1.6694881E-2,-3.14397E-3,4.916426E6,4.867191E-3,1.273478E-2,2.4684701E-2],"split_indices":[33,56,39,32,29,38,9,10,39,0,52,57,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,6.6E1,4.7E1,3.7E1,2.9E1,2.5E1,2.2E1,2.3E1,1.4E1,1.2E1,1.7E1,2E1,5E0,1.6E1,6E0,6E0,1.7E1,6E0,8E0,7E0,5E0,1.2E1,5E0,1.4E1,6E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.2704228E-2,-2.7591306E-1,4.8225084E-1,-3.9350894E-1,3.775211E-2,-1.7772585E-5,5.761034E-1,-2.2078633E-1,-6.1115295E-1,1.4883896E-2,-1.3107546E-1,3.090963E-1,7.521039E-1,-2.0518381E-2,-1.2772426E-1,-1.37782255E-2,-3.3434357E-2,-2.0870275E-3,-1.09690325E-2,1.8194485E-2,6.555726E-3,1.7682757E-2,4.1168228E-2,-1.8156509E-1,6.5790274E-4,-2.7442835E-3,-1.129115E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,25,-1,-1,-1],"loss_changes":[1.5311217E1,2.5968566E0,1.9478931E0,1.78226E0,9.506949E-1,0E0,1.4592886E0,5.1249564E-1,3.7350082E-1,0E0,9.940423E-2,1.3437486E-1,4.8655224E-1,0E0,1.6984963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.9927375E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,14,14,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,26,-1,-1,-1],"split_conditions":[5.093857E-1,2.532835E3,1.1467087E3,4.851587E7,6.15786E5,-1.7772585E-5,1.5083523E0,6.376981E2,5.3323513E-1,1.4883896E-2,7.752181E7,5.5E1,4.1372877E8,-2.0518381E-2,3.095172E5,-1.37782255E-2,-3.3434357E-2,-2.0870275E-3,-1.09690325E-2,1.8194485E-2,6.555726E-3,1.7682757E-2,4.1168228E-2,4.0389168E-1,6.5790274E-4,-2.7442835E-3,-1.129115E-2],"split_indices":[39,52,52,45,29,0,42,4,27,0,45,3,7,0,28,0,0,0,0,0,0,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,6.9E1,4.2E1,5E1,1.9E1,7E0,3.5E1,2.9E1,2.1E1,7E0,1.2E1,1.5E1,2E1,8E0,2.1E1,5E0,1.6E1,7E0,5E0,1E1,5E0,5E0,1.5E1,1.5E1,6E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-4.2783067E-2,-3.5375866E-1,3.6173385E-1,-4.629965E-1,1.2103776E-2,1.3507205E-1,6.131644E-1,-5.891078E-1,-2.4614263E-1,1.5597918E-2,-8.479616E-3,1.6053837E-2,1.155775E-2,3.2351844E-2,1.7202478E-2,-4.1380733E-1,-3.333902E-2,-1.7373992E-2,-3.926934E-3,-6.4212275E-3,7.478346E-3,-1.0111906E-2,-2.7100481E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.358802E1,2.4435306E0,2.5956945E0,1.140338E0,9.270494E-1,6.2709713E-1,7.4459076E-2,2.204113E-1,3.2418668E-1,0E0,0E0,0E0,3.7066334E-1,0E0,0E0,2.7181292E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,2.667836E6,6.15786E5,1.1E1,1.4046429E1,3.31E2,1.841192E-1,1.5597918E-2,-8.479616E-3,1.6053837E-2,2.3272075E5,3.2351844E-2,1.7202478E-2,8.317631E-1,-3.333902E-2,-1.7373992E-2,-3.926934E-3,-6.4212275E-3,7.478346E-3,-1.0111906E-2,-2.7100481E-2],"split_indices":[12,2,39,32,29,8,54,11,39,0,0,0,33,0,0,27,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,6E1,4.6E1,4.6E1,1.4E1,2.5E1,2.1E1,2.8E1,1.8E1,5E0,9E0,9E0,1.6E1,1.6E1,5E0,1.2E1,1.6E1,1E1,8E0,8E0,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.5194857E-2,-3.1753445E-1,2.946798E-1,-3.906241E-1,1.903933E-2,1.6315854E-1,5.2836853E-1,-5.193662E-1,-1.94521E-1,-9.86226E-3,2.288949E-1,2.8586412E-2,1.4316713E-2,-5.792432E-1,-7.910181E-3,-2.8170812E-1,2.1793782E-3,3.1598654E-1,6.9746247E-4,-1.5428794E-2,-6.722046E-1,-1.8941084E-2,-6.8250834E-3,3.7282485E-1,4.952631E-3,-3.8946353E-2,-2.1114703E-2,2.3583597E-2,8.883086E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,21,-1,23,-1,-1,25,-1,-1,27,-1,-1,-1,-1,-1],"loss_changes":[1.0566918E1,3.2184606E0,1.5734844E0,1.2622957E0,0E0,9.1472876E-1,9.953117E-2,5.875082E-1,4.9300718E-1,0E0,5.745405E-1,0E0,0E0,4.3994045E-1,0E0,2.0678592E-1,0E0,2.2909474E-1,0E0,0E0,2.4200726E-1,0E0,0E0,2.783563E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,15,15,17,17,20,20,23,23],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,22,-1,24,-1,-1,26,-1,-1,28,-1,-1,-1,-1,-1],"split_conditions":[2.0020565E8,9.301E3,1.3536367E0,2.9282052E6,1.903933E-2,5.114449E-2,2.0837E4,3.159013E3,2.1816934E8,-9.86226E-3,1.8634315E1,2.8586412E-2,1.4316713E-2,4.4E1,-7.910181E-3,2.1225102E-1,2.1793782E-3,9.780346E9,6.9746247E-4,-1.5428794E-2,1.2887158E1,-1.8941084E-2,-6.8250834E-3,1.4696799E-1,4.952631E-3,-3.8946353E-2,-2.1114703E-2,2.3583597E-2,8.883086E-3],"split_indices":[12,2,39,32,0,38,9,4,7,0,54,0,0,10,0,39,0,5,0,0,53,0,0,38,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,5.8E1,5.3E1,5.3E1,5E0,3.5E1,1.8E1,3.1E1,2.2E1,5E0,3E1,1.3E1,5E0,2.6E1,5E0,1.6E1,6E0,2.1E1,9E0,8E0,1.8E1,8E0,8E0,1.6E1,5E0,1E1,8E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.424292E-2,-3.6985704E-1,3.6925498E-1,-5.5047804E-1,-5.9810847E-2,2.0754582E-1,6.7883694E-1,-3.3445276E-2,-4.3078205E-1,-9.672527E-3,9.324478E-3,3.983875E-1,4.758308E-2,2.1490982E-2,4.0141657E-2,-2.7382657E-1,-3.4165453E-2,2.7849706E-2,9.390367E-3,-5.6986E-3,1.3019743E-2,-6.5874737E-3,-1.5936421E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,-1,15,-1,-1,17,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3407691E1,2.9911203E0,2.1119103E0,2.731228E-1,7.801548E-1,9.2739785E-1,1.6139984E-1,0E0,7.1253014E-1,0E0,0E0,3.944199E-1,6.8996614E-1,0E0,0E0,6.9492936E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,11,11,12,12,15,15],"right_children":[2,4,6,8,10,12,14,-1,16,-1,-1,18,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.168E3,4.852701E6,1.3536367E0,2.0943393E-4,9.637648E2,8.69374E5,1.5605432E9,-3.3445276E-2,9.067368E5,-9.672527E-3,9.324478E-3,1.3420091E6,2.3272075E5,2.1490982E-2,4.0141657E-2,5.661451E-1,-3.4165453E-2,2.7849706E-2,9.390367E-3,-5.6986E-3,1.3019743E-2,-6.5874737E-3,-1.5936421E-2],"split_indices":[2,32,39,42,52,29,7,0,32,0,0,47,33,0,0,27,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,5.3E1,4.4E1,3.3E1,2E1,3E1,1.4E1,1.3E1,2E1,1.3E1,7E0,1.3E1,1.7E1,7E0,7E0,1.4E1,6E0,6E0,7E0,1E1,7E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.7473819E-2,-2.8163576E-1,4.4684654E-1,-3.9458776E-1,5.4208566E-2,-7.656225E-3,5.352609E-1,-2.432687E-1,-5.264101E-1,1.850002E-2,-9.016995E-2,3.9012372E-1,7.6203257E-1,-2.9953343E-3,-2.9215446E-1,-6.1129457E-1,-1.5131371E-2,-8.712638E-3,2.8811037E-3,1.1546276E-2,2.1841163E-2,1.8893635E-2,4.579106E-2,-2.016806E-2,-1.0393683E-2,-1.8168438E-2,-3.4754686E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3863295E1,2.752658E0,2.277608E0,9.4909286E-1,9.3380123E-1,0E0,8.875818E-1,2.2137344E-1,3.1741762E-1,0E0,1.9878861E-1,1.1425328E-1,5.1868916E-1,0E0,1.1299741E-1,2.0818567E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,9.066E3,3.65733E1,7.332772E-1,6.52678E5,-7.656225E-3,1.62739E9,1.19288025E1,2.827343E6,1.850002E-2,6.758564E6,2.0681982E-1,2.0105577E-1,-2.9953343E-3,1.46E2,1.269E3,-1.5131371E-2,-8.712638E-3,2.8811037E-3,1.1546276E-2,2.1841163E-2,1.8893635E-2,4.579106E-2,-2.016806E-2,-1.0393683E-2,-1.8168438E-2,-3.4754686E-2],"split_indices":[42,2,58,27,29,0,7,54,12,0,1,38,38,0,10,2,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,7.1E1,4E1,5.3E1,1.8E1,5E0,3.5E1,2.6E1,2.7E1,5E0,1.3E1,2.3E1,1.2E1,6E0,2E1,1.8E1,9E0,8E0,5E0,8E0,1.5E1,5E0,7E0,6E0,1.4E1,7E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.7632847E-3,-3.1281063E-1,2.899608E-1,-3.7624645E-1,7.641157E-3,9.244564E-2,5.548836E-1,-4.6298006E-1,-1.3873291E-1,-1.103372E-2,1.6657427E-1,3.7857658E-1,4.0009003E-2,-3.1501666E-1,-5.5080366E-1,-1.9985922E-3,-1.0007286E-2,2.523978E-1,3.1904958E-3,2.2117846E-2,9.762718E-3,-1.8523501E-2,-7.2619542E-3,-6.387407E-1,-1.4344731E-2,1.7048644E-2,7.1612215E-3,5.1528504E-3,-4.8696175E-3,-1.9427707E-2,-3.712685E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,23,-1,-1,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0144116E1,1.6416221E0,3.035953E0,9.0398884E-1,0E0,8.690034E-1,8.8331366E-1,2.6830673E-1,8.416134E-2,0E0,4.0583766E-1,1.4314151E-1,0E0,1.098001E-1,2.584839E-1,0E0,0E0,1.4586806E-1,1.2851419E-1,0E0,0E0,0E0,0E0,1.1799383E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,17,17,18,18,23,23],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,24,-1,-1,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4350427E8,2.794308E3,1.3536367E0,3.2316522E7,7.641157E-3,4.739085E-2,1.0861106E4,7.7E2,2.9285714E-1,-1.103372E-2,5E1,1.2271712E1,4.0009003E-2,2E0,7.23E2,-1.9985922E-3,-1.0007286E-2,1.1451835E1,1.308E3,2.2117846E-2,9.762718E-3,-1.8523501E-2,-7.2619542E-3,7.995392E7,-1.4344731E-2,1.7048644E-2,7.1612215E-3,5.1528504E-3,-4.8696175E-3,-1.9427707E-2,-3.712685E-2],"split_indices":[12,52,39,12,0,38,4,11,57,0,3,54,0,8,10,0,0,54,0,0,0,0,0,7,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,5.2E1,5.8E1,4.6E1,6E0,3.4E1,2.4E1,3.3E1,1.3E1,6E0,2.8E1,1.6E1,8E0,1.4E1,1.9E1,6E0,7E0,1.8E1,1E1,1E1,6E0,9E0,5E0,1.3E1,6E0,8E0,1E1,5E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-3.3383213E-2,-2.515718E-1,3.4920552E-1,-3.3402434E-1,2.2288144E-1,9.4566256E-2,4.7765458E-1,-2.1517211E-1,-5.203351E-1,2.1648822E-2,-5.487957E-5,1.1175304E-2,-4.9615104E-4,1.6146105E-2,3.3087887E-2,-2.691387E-1,-8.801471E-2,-5.6400764E-1,-1.3947392E-2,-1.5076236E-2,-6.8149995E-3,1.4912106E-4,-8.694146E-3,-1.5734665E-2,-3.012008E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,-1,19,21,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.010503E1,3.086678E0,1.39399E0,1.3495069E0,6.03206E-1,2.3081072E-1,6.322665E-1,2.6452637E-1,9.117031E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.1335492E-1,1.17288455E-1,7.529402E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,15,15,16,16,17,17],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,-1,20,22,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,3.7254233E3,4.034396E8,8.0073607E-1,9E0,9.581152E0,2.6281825E-1,2.834008E-1,3.188024E1,2.1648822E-2,-5.487957E-5,1.1175304E-2,-4.9615104E-4,1.6146105E-2,3.3087887E-2,7.215E3,9.513889E-1,2.08E2,-1.3947392E-2,-1.5076236E-2,-6.8149995E-3,1.4912106E-4,-8.694146E-3,-1.5734665E-2,-3.012008E-2],"split_indices":[42,52,7,27,17,53,38,57,58,0,0,0,0,0,0,2,57,11,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,7.6E1,4.3E1,6.5E1,1.1E1,1.5E1,2.8E1,4.1E1,2.4E1,5E0,6E0,6E0,9E0,1.8E1,1E1,2.8E1,1.3E1,1.9E1,5E0,2E1,8E0,7E0,6E0,5E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.6532049E-2,-2.3193298E-1,3.8346878E-1,-2.9707325E-1,6.65123E-2,1.7465083E-1,5.5188984E-1,-4.51488E-1,-1.3773926E-1,1.2352455E-2,-4.0307557E-3,-4.6284995E-3,2.597551E-1,1.7808042E-2,4.0595055E-2,-2.6832306E-1,-2.77019E-2,-2.3167126E-1,3.7028678E-2,5.48207E-3,1.987545E-2,-6.5198345E-3,-1.7609613E-2,-1.6387267E-2,-3.653604E-3,6.1567244E-3,-2.8695425E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,-1,21,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0391255E1,1.3417337E0,1.5214019E0,1.315609E0,3.8837722E-1,5.307926E-1,1.010314E0,4.6502447E-1,4.8465127E-1,0E0,0E0,0E0,3.289703E-1,0E0,0E0,1.16725445E-1,0E0,2.9132533E-1,1.0298677E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,15,15,17,17,18,18],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,-1,22,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.095874E6,2.532835E3,1.5364555E0,2.667836E6,6.15786E5,4.1451712E5,1.5605432E9,7.48E2,2.0237808E8,1.2352455E-2,-4.0307557E-3,-4.6284995E-3,2.505442E9,1.7808042E-2,4.0595055E-2,8.317631E-1,-2.77019E-2,2.6326587E-2,4.8049056E5,5.48207E-3,1.987545E-2,-6.5198345E-3,-1.7609613E-2,-1.6387267E-2,-3.653604E-3,6.1567244E-3,-2.8695425E-3],"split_indices":[48,52,42,32,29,32,7,11,7,0,0,0,12,0,0,27,0,38,47,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,6.7E1,4.5E1,5.5E1,1.2E1,2.1E1,2.4E1,2.7E1,2.8E1,5E0,7E0,5E0,1.6E1,1.6E1,8E0,1.2E1,1.5E1,1.8E1,1E1,9E0,7E0,6E0,6E0,1E1,8E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.749493E-3,-2.4109998E-1,3.4021547E-1,-3.3799446E-1,3.6774352E-2,1.6136737E-1,5.0545496E-1,-4.73376E-1,-1.3186613E-1,2.5094625E-1,-1.399789E-2,1.3898559E-2,8.534845E-2,3.5862538E-1,3.741287E-2,-1.5514958E-2,-5.5312216E-1,-1.9979458E-1,5.8024167E-4,1.3419549E-3,2.2737099E-2,-1.9846298E-3,1.1262892E-2,2.0832658E-2,1.0427271E-2,-3.0320244E-2,-1.4066748E-2,-1.2796215E-2,-2.859182E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.740265E0,1.8971438E0,1.3665214E0,1.3886881E0,1.398533E0,2.1431905E-1,7.036772E-1,2.0092487E-1,2.1736598E-1,5.726071E-1,0E0,0E0,3.257789E-1,9.2030525E-2,0E0,0E0,1.8256426E-1,1.2050408E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,16,16,17,17],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.5364555E0,2.667836E6,6.1358623E9,1.7644464E5,1.5605432E9,3.03E2,3.167669E5,1.1653E4,-1.399789E-2,1.3898559E-2,3.9219662E5,2.0837E4,3.741287E-2,-1.5514958E-2,1.365445E1,2.207872E-1,5.8024167E-4,1.3419549E-3,2.2737099E-2,-1.9846298E-3,1.1262892E-2,2.0832658E-2,1.0427271E-2,-3.0320244E-2,-1.4066748E-2,-1.2796215E-2,-2.859182E-3],"split_indices":[50,2,42,32,5,28,7,11,28,2,0,0,28,9,0,0,54,39,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,6.9E1,4.8E1,5.1E1,1.8E1,2.4E1,2.4E1,3E1,2.1E1,1.1E1,7E0,8E0,1.6E1,1.7E1,7E0,1.2E1,1.8E1,1.4E1,7E0,6E0,5E0,9E0,7E0,1E1,7E0,1.3E1,5E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.6655434E-2,-2.1712367E-1,2.1165463E-1,-2.731738E-1,1.5212936E-2,4.3058492E-2,4.708257E-1,-2.061161E-1,-4.2931223E-1,1.153804E-1,-5.1085236E-3,1.3514222E-2,2.581202E-2,-2.0557588E-2,-1.264345E-1,-2.477393E-2,-1.0680668E-2,-4.700494E-4,2.027263E-1,-1.6759275E-1,-5.70276E-4,1.4230793E-2,3.999744E-3,-9.861466E-3,-1.8149178E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,-1,21,23,-1,-1,-1,-1,-1],"loss_changes":[4.988998E0,1.8130207E0,2.2081842E0,4.6643686E-1,0E0,3.5598105E-1,8.988428E-2,6.251216E-1,1.448729E-1,2.423378E-1,0E0,0E0,0E0,0E0,1.3959906E-1,0E0,0E0,0E0,1.1808765E-1,9.559643E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,14,14,18,18,19,19],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,-1,22,24,-1,-1,-1,-1,-1],"split_conditions":[2.3724032E8,1.0565E4,1.3589408E0,2.5233118E11,1.5212936E-2,4.9E1,4.0895187E8,9.7888E4,3.025695E3,3.5849205E-1,-5.1085236E-3,1.3514222E-2,2.581202E-2,-2.0557588E-2,5.5693356E5,-2.477393E-2,-1.0680668E-2,-4.700494E-4,6.52334E-1,3.5983E4,-5.70276E-4,1.4230793E-2,3.999744E-3,-9.861466E-3,-1.8149178E-3],"split_indices":[7,2,42,31,0,3,7,1,4,39,0,0,0,0,28,0,0,0,27,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,5.7E1,5E1,5.2E1,5E0,3.1E1,1.9E1,3.8E1,1.4E1,2.1E1,1E1,6E0,1.3E1,9E0,2.9E1,9E0,5E0,9E0,1.2E1,2.1E1,8E0,6E0,6E0,1.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.1154886E-2,-1.6822976E-1,2.3542058E-1,-2.4267593E-1,2.4636607E-1,-5.6295404E-3,2.834888E-1,-1.3153106E-1,-3.885717E-1,2.511227E-3,1.935966E-2,1.1296564E-4,3.4491876E-1,-1.7288613E-1,3.4615793E-4,-4.455627E-1,-6.480814E-3,2.6269507E-1,2.4188865E-2,-9.733128E-3,-1.9836915E-3,-1.2827846E-2,-2.6337618E-2,1.446352E-2,5.370434E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,17,19,-1,21,-1,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.377455E0,2.1894772E0,7.767625E-1,9.056096E-1,3.0756706E-1,0E0,6.7277956E-1,2.0196497E-1,3.033011E-1,0E0,0E0,0E0,2.7560973E-1,9.192264E-2,0E0,2.1125221E-1,0E0,9.255576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,12,12,13,13,15,15,17,17],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,18,20,-1,22,-1,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,5.188755E3,3.727E3,7.9888564E-1,6.083071E3,-5.6295404E-3,1E0,3.566468E5,2.3724032E8,2.511227E-3,1.935966E-2,1.1296564E-4,1.8027E4,7.446E3,3.4615793E-4,5.895365E5,-6.480814E-3,1.1E1,2.4188865E-2,-9.733128E-3,-1.9836915E-3,-1.2827846E-2,-2.6337618E-2,1.446352E-2,5.370434E-3],"split_indices":[42,4,2,27,52,0,109,28,7,0,0,0,2,2,0,28,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,6.8E1,4.3E1,5.8E1,1E1,5E0,3.8E1,3.4E1,2.4E1,5E0,5E0,7E0,3.1E1,2.6E1,8E0,1.9E1,5E0,2.2E1,9E0,2.1E1,5E0,8E0,1.1E1,1.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.3869952E-2,-1.6843648E-1,3.4316093E-1,-2.730675E-1,6.964187E-2,5.261305E-1,-2.0707445E-2,-4.3624038E-1,-2.0330699E-1,1.6048835E-1,-1.1119935E-2,1.3871801E-2,3.2056764E-2,6.5429895E-3,-8.381271E-3,-2.4617841E-2,-1.1779505E-2,-2.0978171E-2,-1.4837356E-1,1.3156677E-2,2.6584128E-3,-8.8912755E-4,-1.963628E-1,-1.2062463E-2,-5.3931735E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,-1,-1,-1,21,-1,-1,-1,23,-1,-1],"loss_changes":[5.6291413E0,1.9122062E0,2.0082002E0,5.146096E-1,6.893967E-1,4.617076E-1,2.8458986E-1,8.5197926E-2,4.2831814E-1,2.0820758E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0228666E-1,0E0,0E0,0E0,7.984024E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,18,18,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,-1,-1,-1,22,-1,-1,-1,24,-1,-1],"split_conditions":[1.5016E4,2.0020565E8,7.9606894E11,2.0943393E-4,5.9633E7,1.7644464E5,4.1E1,1.4618939E3,1.477E3,9.366E4,-1.1119935E-2,1.3871801E-2,3.2056764E-2,6.5429895E-3,-8.381271E-3,-2.4617841E-2,-1.1779505E-2,-2.0978171E-2,2.8327732E6,1.3156677E-2,2.6584128E-3,-8.8912755E-4,2.942787E6,-1.2062463E-2,-5.3931735E-3],"split_indices":[2,12,31,42,45,28,3,52,2,29,0,0,0,0,0,0,0,0,32,0,0,0,51,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,7.5E1,2.9E1,5.2E1,2.3E1,1.9E1,1E1,1.4E1,3.8E1,1.8E1,5E0,8E0,1.1E1,5E0,5E0,9E0,5E0,6E0,3.2E1,8E0,1E1,9E0,2.3E1,1.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.9966666E-2,-1.3363191E-1,3.5153207E-1,-1.6553815E-2,-3.3895707E-1,4.5107526E-1,-4.008056E-4,-3.530786E-1,9.323059E-2,-7.201763E-3,-4.1712338E-1,2.8919816E-1,6.2806255E-1,-2.3605444E-2,-8.145802E-3,1.8277739E-1,-3.867258E-2,-2.2464693E-2,-1.1164968E-2,1.7711446E-2,7.6183346E-3,3.6317997E-2,1.746897E-2,-6.3472893E-4,1.36327455E-2,-5.107845E-3,4.909964E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,-1,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9546204E0,1.9175153E0,1.3289471E0,1.9540669E0,3.6917543E-1,6.4922094E-1,0E0,2.410022E-1,4.7952878E-1,0E0,6.767726E-2,1.14109516E-1,1.53234E-1,0E0,0E0,4.6799618E-1,1.6979289E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,15,15,16,16],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,-1,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5328901E3,5.070136E7,9.302862E7,7.998101E2,2.3856905E5,1.0825864E4,-4.008056E-4,9.2433E4,1.8831E5,-7.201763E-3,3.0595828E3,1.0494656E1,3.813527E2,-2.3605444E-2,-8.145802E-3,1.4960615E3,6.9015497E-1,-2.2464693E-2,-1.1164968E-2,1.7711446E-2,7.6183346E-3,3.6317997E-2,1.746897E-2,-6.3472893E-4,1.36327455E-2,-5.107845E-3,4.909964E-3],"split_indices":[52,45,45,4,28,52,0,1,29,0,4,53,56,0,0,4,39,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,7.9E1,3.6E1,5.1E1,2.8E1,2.8E1,8E0,1.2E1,3.9E1,9E0,1.9E1,1.6E1,1.2E1,6E0,6E0,2.3E1,1.6E1,1.4E1,5E0,9E0,7E0,7E0,5E0,8E0,1.5E1,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.738458E-3,-2.117457E-1,1.9708106E-1,-2.9382393E-1,1.1397291E-2,2.1339647E-2,2.9983073E-1,-3.6648327E-1,-1.2751782E-1,9.325092E-3,-7.8035705E-2,2.4520174E-1,2.3025395E-2,-8.359324E-3,-4.2287892E-1,-9.412513E-3,-2.888311E-4,-8.186464E-3,1.7986464E-3,6.6199536E-3,1.4423886E-2,-2.7879838E-2,-3.3911625E-1,-8.240872E-3,-2.1867199E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[4.247283E0,1.8306987E0,9.686997E-1,4.4869828E-1,0E0,3.7238458E-1,2.0971417E-1,2.330997E-1,1.1263029E-1,0E0,1.5306666E-1,1.1928916E-1,0E0,0E0,7.367945E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.2715724E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,14,14,22,22],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[2.1915238E8,2.561633E3,6.9015497E-1,7.355655E6,1.1397291E-2,3.5925132E-1,6.866558E5,4.4E1,1.01359664E5,9.325092E-3,4.6947882E-1,1.3536367E0,2.3025395E-2,-8.359324E-3,3E0,-9.412513E-3,-2.888311E-4,-8.186464E-3,1.7986464E-3,6.6199536E-3,1.4423886E-2,-2.7879838E-2,5.501383E9,-8.240872E-3,-2.1867199E-2],"split_indices":[7,52,39,32,0,27,28,10,33,0,39,39,0,0,8,0,0,0,0,0,0,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,4.7E1,5.3E1,4E1,7E0,2E1,3.3E1,2.7E1,1.3E1,7E0,1.3E1,2.7E1,6E0,7E0,2E1,8E0,5E0,7E0,6E0,1E1,1.7E1,5E0,1.5E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-9.310281E-4,-1.4661501E-1,3.46167E-1,-2.189296E-1,7.0423335E-2,2.0039771E-1,2.8894024E-2,-1.97681E-2,-1.7380412E-1,-3.570925E-3,1.3300733E-1,7.9882436E-2,1.4730655E-2,-2.2212946E-1,-4.301468E-2,1.0352374E-2,2.6823212E-3,7.614845E-3,1.0398244E-4,-3.5713832E-3,-2.5429162E-1,-7.4739526E-3,3.5685843E-3,-7.5499457E-3,-1.6573418E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,-1,-1,13,-1,15,17,-1,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,-1],"loss_changes":[5.8153486E0,1.2867161E0,1.1221552E0,4.5016527E-1,1.9928487E-1,2.6345974E-1,0E0,0E0,3.1278777E-1,0E0,8.2208455E-2,7.184123E-2,0E0,1.5781009E-1,2.0628755E-1,0E0,0E0,0E0,0E0,0E0,2.0627093E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,10,10,11,11,13,13,14,14,20,20],"right_children":[2,4,6,8,10,12,-1,-1,14,-1,16,18,-1,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,-1],"split_conditions":[1.1392E4,6.402099E-1,1.3127055E4,8.130303E-5,8.873786E0,7.90905E-2,2.8894024E-2,-1.97681E-2,2.1915238E8,-3.570925E-3,9.366E4,1.470899E6,1.4730655E-2,6.9237E4,3.1982497E5,1.0352374E-2,2.6823212E-3,7.614845E-3,1.0398244E-4,-3.5713832E-3,6.3997424E-1,-7.4739526E-3,3.5685843E-3,-7.5499457E-3,-1.6573418E-2],"split_indices":[2,42,4,42,56,38,0,0,7,0,29,29,0,12,28,0,0,0,0,0,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,8E1,3.3E1,6E1,2E1,2.2E1,1.1E1,1E1,5E1,6E0,1.4E1,1.1E1,1.1E1,3.6E1,1.4E1,6E0,8E0,5E0,6E0,7E0,2.9E1,7E0,7E0,1.5E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[3.5561435E-2,-6.6560164E-2,4.2184544E-1,-1.9294612E-1,9.523856E-2,4.9830553E-1,7.7663143E-3,-1.0212928E-1,-1.7221931E-2,2.8133255E-1,-1.9055102E-2,1.10052945E-2,2.9590355E-2,-1.1517611E-2,-1.8174073E-2,2.3185164E-2,4.748057E-3,-5.4547847E-3,8.478138E-2,-2.8351555E-3,3.8286082E-3,-3.7174276E-4,7.898119E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,19,-1,-1,-1,21,-1,-1,-1,-1],"loss_changes":[3.7756324E0,1.5725068E0,3.198998E-1,5.9910333E-1,7.2851914E-1,3.1963325E-1,0E0,3.2613683E-1,0E0,4.2356765E-1,2.2336051E-1,0E0,0E0,0E0,7.922893E-2,0E0,0E0,0E0,8.091549E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,14,14,18,18],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,20,-1,-1,-1,22,-1,-1,-1,-1],"split_conditions":[1.3536367E0,2.561633E3,2.1383E4,4.851587E7,2.5927516E11,2.5172096E-1,7.7663143E-3,7.196866E2,-1.7221931E-2,8.465346E0,9.250217E5,1.10052945E-2,2.9590355E-2,-1.1517611E-2,1.1467087E3,2.3185164E-2,4.748057E-3,-5.4547847E-3,8.9011444E1,-2.8351555E-3,3.8286082E-3,-3.7174276E-4,7.898119E-3],"split_indices":[39,52,9,45,31,38,0,4,0,53,32,0,0,0,52,0,0,0,56,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E1,7.5E1,1.9E1,4.2E1,3.3E1,1.4E1,5E0,2.8E1,1.4E1,1.2E1,2.1E1,5E0,9E0,1E1,1.8E1,5E0,7E0,1.1E1,1E1,1.3E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-7.927675E-3,-1.6464318E-1,2.615048E-1,-7.2115205E-2,-3.2741767E-1,3.5344264E-1,1.0725403E-1,-1.4310788E-1,1.9434538E-2,-2.016298E-2,-4.7164722E-3,8.984425E-3,4.1166314E-1,-2.0587826E-3,8.677111E-3,-1.7503748E-2,-8.404216E-2,1.1828356E-2,2.3279356E-2,-7.1587274E-3,-1.9585958E-1,-2.6632196E-3,5.5025793E-3,-1.3484452E-2,-2.5692703E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,-1,19,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[4.476601E0,9.854884E-1,5.151696E-1,1.5364096E0,4.5065403E-1,1.5822244E-1,1.800194E-1,4.7265935E-1,0E0,0E0,0E0,0E0,7.917905E-2,0E0,0E0,0E0,2.7369893E-1,0E0,0E0,1.2183121E-1,1.3859472E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,-1,20,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[1.8485416E5,2.5927516E11,1.4046429E1,9.667E3,2.561633E3,5.403511E8,8.413108E-2,9.7888E4,1.9434538E-2,-2.016298E-2,-4.7164722E-3,8.984425E-3,1.2532358E6,-2.0587826E-3,8.677111E-3,-1.7503748E-2,2.8399336E7,1.1828356E-2,2.3279356E-2,1.6987854E1,5.376E3,-2.6632196E-3,5.5025793E-3,-1.3484452E-2,-2.5692703E-3],"split_indices":[33,31,54,2,52,7,38,1,0,0,0,0,32,0,0,0,45,0,0,56,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,6.6E1,3.8E1,4.3E1,2.3E1,2.3E1,1.5E1,3.8E1,5E0,1.6E1,7E0,7E0,1.6E1,5E0,1E1,7E0,3.1E1,6E0,1E1,1.9E1,1.2E1,1.4E1,5E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.640818E-2,-1.7463194E-1,2.1001641E-1,-2.1445304E-1,7.6268227E-3,8.430943E-2,4.3066838E-1,-3.4384993E-1,-8.2018435E-2,-4.650377E-3,1.612504E-1,1.3433492E-2,2.741527E-2,-9.427352E-3,-2.102628E-2,-6.9743264E-3,3.1426982E-3,7.42463E-2,1.3678083E-2,7.6131313E-3,1.3229459E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,-1,-1,-1,-1,19,-1,-1,-1],"loss_changes":[3.6550763E0,8.273461E-1,1.1286652E0,8.971758E-1,0E0,4.040177E-1,1.5440679E-1,2.7944112E-1,2.6445678E-1,0E0,1.899693E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.6713674E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,17,17],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,-1,-1,-1,-1,20,-1,-1,-1],"split_conditions":[3.4707712E8,2.561633E3,1.6957201E0,4.852701E6,7.6268227E-3,8.184232E5,2.463E4,7.1569096E7,2.3514317E8,-4.650377E-3,1.7302E4,1.3433492E-2,2.741527E-2,-9.427352E-3,-2.102628E-2,-6.9743264E-3,3.1426982E-3,3.5258648E-1,1.3678083E-2,7.6131313E-3,1.3229459E-4],"split_indices":[7,52,39,32,0,32,2,7,7,0,9,0,0,0,0,0,0,27,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,5.9E1,4.1E1,5.3E1,6E0,2.7E1,1.4E1,2.6E1,2.7E1,8E0,1.9E1,8E0,6E0,1.1E1,1.5E1,1.9E1,8E0,1.2E1,7E0,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.8131602E-2,-9.984318E-2,2.9588103E-1,-2.0507225E-1,3.3082247E-2,3.183571E-2,2.25442E-1,-1.5370408E-1,-1.9570975E-2,-7.927066E-3,8.109816E-2,1.935579E-2,1.5899627E-1,-1.8228705E-1,-1.6681731E-3,-1.1702166E-3,1.6155745E-1,4.022704E-3,1.1959012E-2,-6.5343203E-3,-1.2940227E-2,1.1974533E-2,2.1745672E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,15,-1,17,19,-1,-1,21,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0837145E0,1.0558559E0,8.7055635E-1,3.6743152E-1,3.3454362E-1,0E0,3.4176886E-1,1.1302179E-1,0E0,0E0,2.4105197E-1,0E0,1.5982413E-1,7.654673E-2,0E0,0E0,1.4891264E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,10,10,12,12,13,13,16,16],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,16,-1,18,20,-1,-1,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.6777216E8,1.6451234E-2,1E0,1.937363E6,3.183571E-2,2.8129198E7,6.939E3,-1.9570975E-2,-7.927066E-3,1.1467087E3,1.935579E-2,8.640846E3,1.6958537E9,-1.6681731E-3,-1.1702166E-3,1.327538E7,4.022704E-3,1.1959012E-2,-6.5343203E-3,-1.2940227E-2,1.1974533E-2,2.1745672E-3],"split_indices":[2,7,57,112,32,0,45,2,0,0,52,0,52,5,0,0,45,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,7.4E1,3.9E1,4.1E1,3.3E1,5E0,3.4E1,3.4E1,7E0,6E0,2.7E1,8E0,2.6E1,2.7E1,7E0,1.2E1,1.5E1,1.5E1,1.1E1,1.9E1,8E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.7084177E-2,-8.990376E-2,2.9355738E-1,-2.1187007E-1,7.298652E-2,4.239255E-1,5.841277E-2,-2.5637326E-1,-7.9247676E-2,-1.1262423E-2,9.538482E-3,8.368315E-3,2.5323955E-2,7.366693E-3,-4.4502723E-3,-2.900163E-1,-3.2858227E-3,9.240367E-4,-7.3703327E-3,7.3930835E-3,-3.684368E-3,-7.329E-3,-1.9274749E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.543844E0,1.465764E0,1.0788302E0,2.2757196E-1,3.3651677E-1,4.817171E-1,2.112162E-1,1.7692947E-1,8.899772E-2,2.2417174E-1,0E0,0E0,0E0,0E0,0E0,3.2925344E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,15,15],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,5.6614784E7,1.4046429E1,3.0595828E3,1.1648228E0,1.7059127E5,1.6981527E1,3.2936362E6,3.6948596E3,2.7186182E-1,9.538482E-3,8.368315E-3,2.5323955E-2,7.366693E-3,-4.4502723E-3,2.9538235E2,-3.2858227E-3,9.240367E-4,-7.3703327E-3,7.3930835E-3,-3.684368E-3,-7.329E-3,-1.9274749E-2],"split_indices":[2,12,54,4,39,28,54,50,4,27,0,0,0,0,0,52,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,7.2E1,3.5E1,4.1E1,3.1E1,2.2E1,1.3E1,3E1,1.1E1,1.9E1,1.2E1,7E0,1.5E1,8E0,5E0,2.5E1,5E0,5E0,6E0,5E0,1.4E1,1.2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.764445E-2,-3.0292595E-2,4.196442E-1,-1.2291454E-1,1.2649943E-1,2.6503902E-2,9.718927E-3,-2.3717202E-1,-4.843075E-2,-4.3095462E-4,1.8199177E-1,-1.5425666E-2,-4.02988E-3,-4.7789165E-3,2.516137E-3,2.1831535E-1,8.9452E-4,1.3171192E-2,4.375082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,15,-1,-1,-1,-1,17,-1,-1,-1],"loss_changes":[2.1710286E0,1.4823092E0,2.2564816E-1,5.3317475E-1,2.8713763E-1,0E0,0E0,2.867502E-1,2.0696715E-1,0E0,1.563353E-1,0E0,0E0,0E0,0E0,1.2657642E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,10,10,15,15],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,16,-1,-1,-1,-1,18,-1,-1,-1],"split_conditions":[9.669896E3,6.402099E-1,6.5566176E7,2.827343E6,1.6511278E1,2.6503902E-2,9.718927E-3,2.8688198E3,3.290356E5,-4.3095462E-4,1.8634315E1,-1.5425666E-2,-4.02988E-3,-4.7789165E-3,2.516137E-3,7.5305285E6,8.9452E-4,1.3171192E-2,4.375082E-3],"split_indices":[52,42,45,12,56,0,0,4,28,0,54,0,0,0,0,47,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,1E2,1.1E1,6.3E1,3.7E1,6E0,5E0,2.4E1,3.9E1,1.1E1,2.6E1,1.5E1,9E0,2.6E1,1.3E1,2.1E1,5E0,1.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.8235078E-2,-1.04151435E-1,2.5031358E-1,-2.3056142E-1,1.5381586E-2,1.3428341E-1,4.159661E-1,-2.963065E-1,-8.639309E-2,-5.7431754E-2,1.5470862E-1,1.608795E-2,6.818118E-2,2.5436262E-2,7.078778E-3,-1.5032612E-1,-2.0312803E-2,-1.3797732E-4,-8.879932E-3,-4.7119535E-3,4.7628037E-3,1.0914808E-2,2.5085812E-3,5.792139E-3,-3.4754165E-3,-1.0944465E-2,-1.1787923E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2924478E0,1.1519191E0,7.1297E-1,3.2510042E-1,4.1467986E-1,3.0448753E-1,3.95149E-1,3.8651323E-1,1.0481798E-1,1.7731392E-1,9.0176016E-2,0E0,1.5256035E-1,0E0,0E0,1.18469685E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5328901E3,3.2316522E7,1.0825864E4,4.6573498E1,3.290356E5,1.8403254E7,1.200998E10,3.91028E5,8.661788E7,1.2940105E0,1.4333148E6,1.608795E-2,5.9633E7,2.5436262E-2,7.078778E-3,1.0376E4,-2.0312803E-2,-1.3797732E-4,-8.879932E-3,-4.7119535E-3,4.7628037E-3,1.0914808E-2,2.5085812E-3,5.792139E-3,-3.4754165E-3,-1.0944465E-2,-1.1787923E-3],"split_indices":[52,12,52,56,28,45,5,12,45,39,47,0,45,0,0,9,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,7.5E1,3.9E1,3.6E1,3.9E1,2.4E1,1.5E1,2.4E1,1.2E1,2.6E1,1.3E1,5E0,1.9E1,1E1,5E0,1.2E1,1.2E1,7E0,5E0,2.1E1,5E0,7E0,6E0,1.4E1,5E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[9.01323E-3,-1.2750037E-1,1.5694301E-1,-1.726097E-1,1.0553641E-1,2.6515102E-2,2.1823339E-1,-1.207953E-1,-1.4491495E-2,-1.6496464E-3,1.1018529E-2,1.1999279E-2,-6.376877E-2,1.3941002E-1,2.9008624E-1,-2.7884167E-1,-5.1231444E-2,1.5766186E-3,-5.577702E-3,1.9734798E-3,8.998096E-3,1.0200279E-2,1.9732581E-2,-6.7719906E-3,-1.9103572E-2,-5.009163E-3,2.82419E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4837303E0,6.909932E-1,4.658581E-1,3.1825328E-1,1.9512445E-1,4.153815E-1,1.8610656E-1,4.2548716E-1,0E0,0E0,0E0,0E0,7.804925E-2,8.858845E-2,1.00705266E-1,1.377626E-1,1.7193095E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,2.794308E3,6.2371206E-1,5.5124454E9,4.12E2,3.5925132E-1,2.6167098E5,1.73E2,-1.4491495E-2,-1.6496464E-3,1.1018529E-2,1.1999279E-2,4.494817E7,2.0545694E5,1.5605432E9,1.7162472E-1,8E0,1.5766186E-3,-5.577702E-3,1.9734798E-3,8.998096E-3,1.0200279E-2,1.9732581E-2,-6.7719906E-3,-1.9103572E-2,-5.009163E-3,2.82419E-3],"split_indices":[7,52,39,5,0,27,28,0,0,0,0,0,45,33,7,57,8,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,6.3E1,5.8E1,5.3E1,1E1,1.9E1,3.9E1,3.9E1,1.4E1,5E0,5E0,5E0,1.4E1,2E1,1.9E1,1.1E1,2.8E1,5E0,9E0,7E0,1.3E1,1.3E1,6E0,6E0,5E0,1.9E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-8.769942E-3,-1.3651514E-1,1.8913977E-1,-1.7459261E-1,9.87452E-3,9.479888E-2,3.7105018E-1,-9.259439E-2,-3.1856108E-1,-1.2080905E-2,1.6078547E-1,1.1364003E-2,2.2946661E-2,-1.6026255E-1,5.506983E-2,-7.616439E-3,-1.7389687E-2,3.0157703E-3,-4.6883654E-3,1.9218223E-3,2.1802633E-1,-2.2993623E-1,-1.8408047E-3,5.8223177E-3,-8.7006134E-4,1.3874889E-2,4.9272343E-3,-1.7591866E-2,-6.9772415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.705714E0,8.782071E-1,6.856115E-1,6.670072E-1,0E0,2.0646879E-1,8.5112214E-2,3.9852017E-1,8.089352E-2,8.164879E-2,1.1611682E-1,0E0,0E0,2.2122806E-1,6.4040035E-2,0E0,0E0,0E0,0E0,0E0,6.643659E-2,1.4227891E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4707712E8,2.794308E3,1.6957201E0,2.5233118E11,9.87452E-3,6.2371206E-1,1.2750684E9,7E0,5.432393E-1,6.113901E8,7.21E2,1.1364003E-2,2.2946661E-2,2.3616585E6,4.3E1,-7.616439E-3,-1.7389687E-2,3.0157703E-3,-4.6883654E-3,1.9218223E-3,9.921247E3,3.2E1,-1.8408047E-3,5.8223177E-3,-8.7006134E-4,1.3874889E-2,4.9272343E-3,-1.7591866E-2,-6.9772415E-3],"split_indices":[7,52,39,31,0,39,7,8,27,7,0,0,0,48,3,0,0,0,0,0,4,3,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,6.4E1,4.1E1,5.8E1,6E0,2.8E1,1.3E1,3.8E1,2E1,1.1E1,1.7E1,7E0,6E0,2.6E1,1.2E1,5E0,1.5E1,6E0,5E0,6E0,1.1E1,1.6E1,1E1,6E0,6E0,6E0,5E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.4697129E-2,-7.1742296E-2,1.9525969E-1,-1.0225194E-1,1.3355836E-2,2.6985106E-1,6.650692E-2,-2.7340995E-2,-1.9688517E-1,6.0965554E-3,1.7307138E-2,-1.6256835E-3,5.5709663E-3,-1.0116136E-1,8.0484584E-2,-4.0866667E-3,-1.263525E-2,-9.046667E-3,-2.5181884E-3,1.1564462E-2,-1.8595456E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8233933E0,7.8108907E-1,3.6620677E-1,4.6249527E-1,0E0,2.701379E-1,8.159365E-2,3.1061143E-1,1.869334E-1,0E0,0E0,0E0,0E0,8.713983E-2,3.103681E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,1.3400168E7,1.343964E1,5.1121724E7,1.3355836E-2,3.077573E5,8.784E3,2.8866995E1,5.91734E-1,6.0965554E-3,1.7307138E-2,-1.6256835E-3,5.5709663E-3,9.7E1,1.80956E5,-4.0866667E-3,-1.263525E-2,-9.046667E-3,-2.5181884E-3,1.1564462E-2,-1.8595456E-3],"split_indices":[39,1,54,45,0,28,2,56,27,0,0,0,0,10,29,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,7E1,3.9E1,6.5E1,5E0,2.4E1,1.5E1,3.7E1,2.8E1,1E1,1.4E1,5E0,1E1,2.2E1,1.5E1,1.1E1,1.7E1,7E0,1.5E1,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.0109035E-2,-1.2695557E-1,1.2677011E-1,-1.5371664E-1,7.1874848E-3,-2.0534785E-2,2.1666807E-1,-9.339817E-2,-2.606359E-1,-7.6831505E-3,1.5761124E-3,2.577246E-1,4.671067E-3,-1.18582025E-2,-5.1290106E-2,-7.2578234E-3,-1.836857E-2,8.9965025E-3,1.8321173E-2,-3.8067035E-3,3.663901E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,-1,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8393935E0,4.79739E-1,7.06297E-1,3.4326196E-1,0E0,1.5915355E-1,1.3799691E-1,2.2984591E-1,2.0607519E-1,0E0,0E0,1.379683E-1,0E0,0E0,1.1348894E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,14,14],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,-1,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5363757E8,2.9853157E3,9.745363E-1,8.297822E-1,7.1874848E-3,6.919127E5,2.5588E4,8.137731E7,2.6078947E1,-7.6831505E-3,1.5761124E-3,3.6742297E5,4.671067E-3,-1.18582025E-2,1E1,-7.2578234E-3,-1.836857E-2,8.9965025E-3,1.8321173E-2,-3.8067035E-3,3.663901E-3],"split_indices":[12,52,42,27,0,32,9,5,56,0,0,28,0,0,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,6.1E1,5.2E1,5.6E1,5E0,2E1,3.2E1,3.7E1,1.9E1,5E0,1.5E1,2.3E1,9E0,7E0,3E1,1.1E1,8E0,1.6E1,7E0,2.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.3385034E-2,-1.17667995E-1,1.6058068E-1,-1.4828087E-1,7.1867215E-3,8.3904706E-2,2.9440218E-1,-3.184585E-2,-2.6773694E-1,1.133509E-1,-2.8320302E-3,8.61967E-3,2.0186186E-2,-1.1346184E-2,1.8021315E-2,-1.5676524E-2,-6.514917E-3,1.7169109E-1,2.3206929E-3,8.287142E-2,-6.301856E-3,1.2913407E-2,4.2624576E-3,6.065048E-3,-1.1025E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,-1,19,-1,-1,21,-1,23,-1,-1,-1,-1,-1],"loss_changes":[2.2951663E0,5.3518426E-1,5.499085E-1,7.8427935E-1,0E0,1.6176572E-1,2.1089602E-1,3.0712417E-1,1.6532564E-1,1.10094935E-1,0E0,0E0,0E0,0E0,2.5099877E-1,0E0,0E0,1.03684396E-1,0E0,8.3442315E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,14,14,17,17,19,19],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,-1,20,-1,-1,22,-1,24,-1,-1,-1,-1,-1],"split_conditions":[3.4707712E8,2.794308E3,1.7618678E0,6.4987373E-1,7.1867215E-3,9.014471E-1,1.2750684E9,1.688E3,3.188024E1,1.2320755E1,-2.8320302E-3,8.61967E-3,2.0186186E-2,-1.1346184E-2,3.925844E7,-1.5676524E-2,-6.514917E-3,1.5E1,2.3206929E-3,4.3E1,-6.301856E-3,1.2913407E-2,4.2624576E-3,6.065048E-3,-1.1025E-3],"split_indices":[7,52,39,27,0,27,7,2,58,54,0,0,0,0,45,0,0,8,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,6.2E1,5.5E1,5.6E1,6E0,3.6E1,1.9E1,2.9E1,2.7E1,3E1,6E0,1.1E1,8E0,5E0,2.4E1,1.8E1,9E0,1.5E1,1.5E1,1.7E1,7E0,6E0,9E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.8588599E-3,-6.784637E-2,2.1626766E-1,-1.7770652E-3,-1.8104458E-1,3.65877E-1,7.3039174E-2,-8.127009E-2,1.1755132E-1,-1.1923789E-2,-2.8032435E-3,9.352298E-3,2.3128042E-2,1.500201E-4,7.462853E-3,-3.4479115E-2,-1.2719891E-2,8.019973E-3,2.231623E-4,-3.7999207E-3,1.3872717E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,-1,-1,19,-1,-1,-1,-1,-1],"loss_changes":[1.4252484E0,5.680767E-1,4.6612787E-1,4.7435635E-1,2.1057928E-1,1.2009752E-1,7.324447E-2,2.4560362E-1,1.07830584E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.168305E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,15,15],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,-1,-1,20,-1,-1,-1,-1,-1],"split_conditions":[4.4925254E3,1.7994973E11,2.5927516E11,1.0668866E3,3.271557E8,7.845511E-1,1.6537456E6,1E0,1.499169E1,-1.1923789E-2,-2.8032435E-3,9.352298E-3,2.3128042E-2,1.500201E-4,7.462853E-3,4E0,-1.2719891E-2,8.019973E-3,2.231623E-4,-3.7999207E-3,1.3872717E-3],"split_indices":[52,31,31,52,7,27,32,112,54,0,0,0,0,0,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,7.5E1,2.2E1,4.8E1,2.7E1,1E1,1.2E1,2.9E1,1.9E1,1.7E1,1E1,5E0,5E0,7E0,5E0,2.4E1,5E0,1.3E1,6E0,1.4E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.864719E-2,-9.048942E-2,1.4460278E-1,-3.711181E-2,-1.8257481E-1,-1.5080041E-3,1.913199E-1,-7.551254E-2,1.2448317E-2,-1.3966735E-1,-1.5331628E-2,1.6237179E-2,6.9412803E-3,-1.13487124E-1,1.8581905E-3,-1.0664896E-2,-2.9411516E-3,-1.5154171E-1,-9.753153E-4,-1.1668047E-2,-4.443686E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,-1,17,-1,-1,-1,19,-1,-1,-1],"loss_changes":[1.3031915E0,3.7170225E-1,2.8510952E-1,5.842698E-1,1.1682618E-1,0E0,1.3844818E-1,1.9897217E-1,0E0,1.3290349E-1,0E0,0E0,0E0,1.1772293E-1,0E0,0E0,0E0,1.0616702E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,13,13,17,17],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,-1,18,-1,-1,-1,20,-1,-1,-1],"split_conditions":[1.0230335E0,6.784095E7,1.6511278E1,4.797364E3,1.414835E8,-1.5080041E-3,8.892E3,3.4288502E8,1.2448317E-2,3.4442917E3,-1.5331628E-2,1.6237179E-2,6.9412803E-3,3.0870352E1,1.8581905E-3,-1.0664896E-2,-2.9411516E-3,1.7214674E1,-9.753153E-4,-1.1668047E-2,-4.443686E-3],"split_indices":[39,45,56,4,45,0,9,7,0,4,0,0,0,56,0,0,0,58,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,7.6E1,3.3E1,4.9E1,2.7E1,7E0,2.6E1,4.4E1,5E0,2.2E1,5E0,5E0,2.1E1,3.3E1,1.1E1,1E1,1.2E1,2.3E1,1E1,8E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[8.5765764E-4,-8.351889E-2,1.720672E-1,-1.7375039E-1,2.6240975E-2,2.6931316E-1,-2.8659835E-2,-2.4761127E-1,-7.792657E-2,-3.3117088E-3,1.0364432E-1,1.5703066E-1,1.82807E-2,3.0318138E-3,-6.3126576E-3,-5.502604E-3,-3.064808E-1,-6.399176E-3,-4.3654794E-4,7.5720805E-3,-7.553063E-5,3.5306977E-3,1.1767221E-2,-1.9102002E-2,-5.974735E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4878701E0,6.879214E-1,6.713947E-1,2.4965441E-1,2.4148665E-1,2.2018027E-1,1.1955889E-1,1.3079762E-1,6.6072166E-2,0E0,1.0010128E-1,7.6302975E-2,0E0,0E0,0E0,0E0,1.7691529E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,16,16],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7254233E3,2.8980099E1,7.9606894E11,1.365445E1,3.8757455E-1,8.792254E3,2.2950048E10,2E0,1.809227E1,-3.3117088E-3,9.581152E0,8.631652E6,1.82807E-2,3.0318138E-3,-6.3126576E-3,-5.502604E-3,4.6573498E1,-6.399176E-3,-4.3654794E-4,7.5720805E-3,-7.553063E-5,3.5306977E-3,1.1767221E-2,-1.9102002E-2,-5.974735E-3],"split_indices":[52,58,31,54,41,52,5,8,58,0,53,32,0,0,0,0,56,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,6.8E1,3.3E1,3.7E1,3.1E1,2.2E1,1.1E1,2E1,1.7E1,1.4E1,1.7E1,1.2E1,1E1,6E0,5E0,7E0,1.3E1,9E0,8E0,1.1E1,6E0,7E0,5E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-7.0797163E-3,-9.242823E-2,8.144338E-2,-1.16811894E-1,6.699955E-3,1.4346854E-1,-3.7375346E-2,-7.975958E-2,-1.2984924E-2,7.9923995E-2,1.3868051E-2,-6.120418E-3,4.138792E-3,-9.927275E-2,1.8188944E-3,5.4661073E-3,-1.4708262E-3,-1.5091473E-3,-6.6678664E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,-1,-1,-1,-1,-1],"loss_changes":[8.612782E-1,3.425633E-1,4.1806826E-1,2.7582657E-1,0E0,3.1316042E-1,2.2952722E-1,1.0446E-1,0E0,1.01587564E-1,0E0,0E0,0E0,9.516677E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,6.099784E3,6.19999E9,1E0,6.699955E-3,2.505442E9,2.3430255E5,8.2091756E5,-1.2984924E-2,2.4E1,1.3868051E-2,-6.120418E-3,4.138792E-3,2.3452184E5,1.8188944E-3,5.4661073E-3,-1.4708262E-3,-1.5091473E-3,-6.6678664E-3],"split_indices":[7,4,5,112,0,12,33,28,0,8,0,0,0,28,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5.7E1,5.5E1,5.2E1,5E0,3.6E1,1.9E1,4.3E1,9E0,2.6E1,1E1,1.1E1,8E0,3.7E1,6E0,2E1,6E0,1.4E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-8.108421E-3,-6.102122E-2,2.0391133E-1,-1.5808132E-1,1.590609E-2,3.14305E-1,-1.3832127E-3,-1.2067669E-1,-1.5099694E-2,-3.962533E-2,8.83836E-2,2.0194309E-2,6.648723E-3,-7.0077507E-3,5.64003E-4,1.6326914E-3,-3.5985238E-3,3.4855E-2,7.836151E-3,5.2389656E-3,-1.6305256E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,-1,-1,-1,19,-1,-1,-1],"loss_changes":[1.2352659E0,6.5957296E-1,5.735863E-1,1.8666095E-1,2.0474118E-1,2.1551228E-1,0E0,1.06684476E-1,0E0,7.6290466E-2,8.085194E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.400381E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,17,17],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,-1,-1,-1,20,-1,-1,-1],"split_conditions":[6.622281E3,3.851071E7,1.370645E7,1E0,1.0889989E0,1.1517007E1,-1.3832127E-3,2.5E-1,-1.5099694E-2,3.5692856E1,3.208219E1,2.0194309E-2,6.648723E-3,-7.0077507E-3,5.64003E-4,1.6326914E-3,-3.5985238E-3,2.6E1,7.836151E-3,5.2389656E-3,-1.6305256E-3],"split_indices":[52,12,47,112,42,54,0,57,0,58,56,0,0,0,0,0,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,8.7E1,2.1E1,3.8E1,4.9E1,1.4E1,7E0,3.2E1,6E0,2.8E1,2.1E1,8E0,6E0,2.7E1,5E0,9E0,1.9E1,1.3E1,8E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.9596266E-2,-4.5835063E-2,1.8588324E-1,3.6538176E-2,-1.678671E-1,2.1027429E-2,1.1604583E-1,-7.083749E-3,6.8416886E-2,-1.3190853E-2,-1.17467724E-1,-1.200638E-3,1.6437036E-1,1.0146673E-2,3.6407143E-2,-2.370108E-3,-8.790506E-3,1.1450184E-2,3.1959054E-3,5.317822E-3,-5.216948E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,15,-1,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1183183E0,7.499944E-1,4.5262492E-1,2.7228564E-1,1.2607259E-1,0E0,1.6742149E-1,0E0,1.6855396E-1,0E0,8.7699145E-2,0E0,1.12048924E-1,0E0,1.15488395E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,10,10,12,12,14,14],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,16,-1,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7254233E3,4.5843E7,6.4417176E-2,1.00441E5,2.2196926E-2,2.1027429E-2,7.94E2,-7.083749E-3,1.82E2,-1.3190853E-2,7.752181E7,-1.200638E-3,8.260109E9,1.0146673E-2,9.163856E0,-2.370108E-3,-8.790506E-3,1.1450184E-2,3.1959054E-3,5.317822E-3,-5.216948E-4],"split_indices":[52,45,57,1,57,0,0,0,10,0,45,0,5,0,53,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.01E2,7.3E1,2.8E1,4.4E1,2.9E1,5E0,2.3E1,6E0,3.8E1,8E0,2.1E1,6E0,1.7E1,6E0,3.2E1,1.1E1,1E1,9E0,8E0,1.2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.6966619E-2,-5.1050477E-2,2.2290866E-1,-1.4720911E-1,4.869862E-2,1.8517025E-2,1.3562423E-1,-1.8672566E-1,-1.5910184E-3,-7.867723E-4,7.3211147E-3,3.7965E-3,1.1937684E-2,-1.1394891E-1,-2.9245976E-1,-8.587008E-3,-1.5895923E-3,-1.9100249E-2,-7.905267E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,-1,-1,-1,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[1.524198E0,7.936912E-1,3.240255E-1,1.8365395E-1,2.75382E-1,0E0,1.0124919E-1,2.0521903E-1,0E0,0E0,0E0,0E0,0E0,9.701967E-2,9.3735695E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,13,13,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,-1,-1,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[4.8360786E3,3.2936362E6,6.758E3,4.6573498E1,9.8309404E-1,1.8517025E-2,8.3899E4,2.4E2,-1.5910184E-3,-7.867723E-4,7.3211147E-3,3.7965E-3,1.1937684E-2,1.7214674E1,1.2775832E1,-8.587008E-3,-1.5895923E-3,-1.9100249E-2,-7.905267E-3],"split_indices":[52,50,9,56,39,0,2,10,0,0,0,0,0,58,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,8.1E1,2.6E1,4.1E1,4E1,8E0,1.8E1,3E1,1.1E1,2.5E1,1.5E1,1.3E1,5E0,1.9E1,1.1E1,1E1,9E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.2794668E-2,-9.069982E-2,7.080198E-2,-1.2457832E-1,3.613196E-3,1.7853205E-1,-2.0149813E-3,-2.7325672E-1,-8.1650496E-2,4.728689E-3,1.5396725E-2,-7.074255E-2,1.3560806E-1,-6.9881477E-3,-1.7269928E-2,-6.7466986E-3,-1.5266618E-1,-4.8650717E-4,-6.85464E-3,1.4851016E-3,1.1400248E-2,-3.427597E-3,6.3647814E-3,-5.0400016E-3,-1.24334805E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,-1,-1,21,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.683971E-1,3.5094112E-1,4.4750065E-1,3.031929E-1,0E0,2.2832316E-1,3.4064066E-1,6.681341E-2,2.1706313E-1,0E0,0E0,1.0173668E-1,1.184866E-1,0E0,0E0,1.9514537E-1,7.948458E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,-1,-1,22,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9517919E3,3.4707712E8,2.5927516E11,2.3E1,3.613196E-3,5.311576E6,1.0230335E0,5.3315993E-2,3.925844E7,4.728689E-3,1.5396725E-2,7.752181E7,2.119303E6,-6.9881477E-3,-1.7269928E-2,1.5426178E3,7.9918444E5,-4.8650717E-4,-6.85464E-3,1.4851016E-3,1.1400248E-2,-3.427597E-3,6.3647814E-3,-5.0400016E-3,-1.24334805E-2],"split_indices":[52,7,31,3,0,1,39,57,45,0,0,45,32,0,0,4,28,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.16E2,6E1,5.6E1,5E1,1E1,2.2E1,3.4E1,1E1,4E1,1.5E1,7E0,2.3E1,1.1E1,5E0,5E0,2E1,2E1,1.3E1,1E1,6E0,5E0,1.4E1,6E0,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.1286551E-2,-8.846079E-2,1.1083588E-1,6.3628512E-3,-1.1212861E-1,6.398126E-2,2.011777E-1,-2.4717405E-1,-6.543562E-2,1.3387153E-1,-4.4103275E-4,1.633765E-3,1.3357793E-2,-1.5331584E-2,-5.490067E-3,-1.7873826E-2,-8.470179E-3,9.723175E-3,3.51153E-3,-4.121209E-3,6.0244218E-2,6.794955E-3,-1.931354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,-1,-1,-1,-1,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[1.1318699E0,3.1134585E-1,2.2840542E-1,0E0,3.117211E-1,2.0044045E-1,2.2828436E-1,8.489698E-2,2.0517132E-1,6.756285E-2,0E0,0E0,0E0,0E0,0E0,1.5744577E-1,0E0,0E0,0E0,0E0,1.1785928E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,15,15,20,20],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,-1,-1,-1,-1,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[2.3514317E8,8.572271E0,1.5605432E9,6.3628512E-3,2.3E1,1.6505978E9,1E0,1.3436171E1,4.352552E9,1.2257767E1,-4.4103275E-4,1.633765E-3,1.3357793E-2,-1.5331584E-2,-5.490067E-3,1.4600837E3,-8.470179E-3,9.723175E-3,3.51153E-3,-4.121209E-3,7.45658E5,6.794955E-3,-1.931354E-3],"split_indices":[7,54,7,0,3,5,109,54,5,54,0,0,0,0,0,4,0,0,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5.6E1,5.6E1,5E0,5.1E1,3.8E1,1.8E1,1.2E1,3.9E1,1.9E1,1.9E1,6E0,1.2E1,7E0,5E0,2.8E1,1.1E1,8E0,1.1E1,1.5E1,1.3E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[4.177792E-3,2.2745775E-2,-1.52659975E-2,-1.0941577E-2,3.0548924E-1,-6.396394E-2,4.703703E-2,1.9955164E-2,7.164318E-3,-1.6449833E-2,-1.448799E-1,5.941495E-3,7.652757E-4,-2.5079015E-3,5.6737545E-3,-4.215188E-3,-1.0813554E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,-1,-1,-1,-1],"loss_changes":[6.47771E-1,9.844051E-1,0E0,2.889058E-1,1.2378335E-1,1.8600461E-1,1.0440602E-1,0E0,0E0,1.571909E-1,6.2208384E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825864E4,-1.52659975E-2,2.0020565E8,3.0376984E1,8.4947616E-1,1.8831E5,1.9955164E-2,7.164318E-3,5.1222866E5,2.0749E4,5.941495E-3,7.652757E-4,-2.5079015E-3,5.6737545E-3,-4.215188E-3,-1.0813554E-2],"split_indices":[45,52,0,12,57,27,29,0,0,28,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,1.02E2,5E0,9.2E1,1E1,4.8E1,4.4E1,5E0,5E0,3.1E1,1.7E1,1.2E1,3.2E1,2.5E1,6E0,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.3136507E-2,-4.6916343E-2,1.2801802E-1,2.0149855E-2,-1.2049279E-1,3.8302538E-3,2.2333612E-1,-2.8236493E-2,1.36994235E-2,-7.391986E-2,-1.4021662E-2,4.4207927E-3,1.4749745E-2,-3.4813436E-3,3.615068E-2,-8.500365E-3,1.0484351E-4,-1.2470947E-3,5.955127E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,-1,-1,-1,17,-1,-1,-1,-1],"loss_changes":[7.441951E-1,3.5277689E-1,1.5485191E-1,4.9113265E-1,2.4632722E-1,0E0,1.0096222E-1,9.536512E-2,0E0,2.1688476E-1,0E0,0E0,0E0,0E0,7.842322E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,14,14],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,-1,-1,-1,18,-1,-1,-1,-1],"split_conditions":[1.0639322E0,2.5927516E11,1.5605432E9,2.5760403E3,1.2511909E8,3.8302538E-3,2.6167098E5,2.4662777E1,1.36994235E-2,2.6385223E-2,-1.4021662E-2,4.4207927E-3,1.4749745E-2,-3.4813436E-3,3.136809E5,-8.500365E-3,1.0484351E-4,-1.2470947E-3,5.955127E-3],"split_indices":[42,31,7,52,45,0,28,56,0,57,0,0,0,0,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,7E1,3.6E1,3.7E1,3.3E1,2.5E1,1.1E1,3.2E1,5E0,2.7E1,6E0,5E0,6E0,1.9E1,1.3E1,1.1E1,1.6E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[9.980609E-3,3.0815419E-2,-1.0443752E-2,6.171741E-3,1.3126763E-2,6.2095284E-2,-5.509752E-2,-4.8584922E-4,1.3779646E-1,-2.5584875E-2,-8.765243E-3,8.886482E-3,2.2408878E-3,-3.0863076E-3,2.7131834E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,-1,-1,-1,-1,-1],"loss_changes":[5.003167E-1,5.726883E-1,0E0,3.0837035E-1,0E0,2.5801405E-1,1.5516633E-1,0E0,8.7753564E-2,1.1567152E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,-1,-1,-1,-1,-1],"split_conditions":[1.12782936E8,1.0825864E4,-1.0443752E-2,2.8399336E7,1.3126763E-2,1.6987854E1,1E0,-4.8584922E-4,7.6363635E-1,1.1855E4,-8.765243E-3,8.886482E-3,2.2408878E-3,-3.0863076E-3,2.7131834E-3],"split_indices":[45,52,0,45,0,56,89,0,57,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.04E2,9.6E1,8E0,8.8E1,8E0,4.6E1,4.2E1,2.4E1,2.2E1,3.5E1,7E0,1.4E1,8E0,2.4E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-5.1047024E-3,-3.1541783E-2,1.2834966E-2,-1.1405763E-1,3.560332E-2,-1.4147604E-1,4.4699004E-3,1.0896586E-1,-1.0218322E-2,-2.1517058E-1,-2.955908E-3,1.3045501E-3,1.086847E-2,-2.913287E-3,1.8224532E-3,-7.064658E-3,-1.611851E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,-1,5,7,9,-1,11,13,15,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.011749E-1,5.691954E-1,0E0,2.7334172E-1,1.927242E-1,2.2988677E-1,0E0,2.023535E-1,8.831058E-2,1.2266344E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9],"right_children":[2,4,-1,6,8,10,-1,12,14,16,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0825864E4,5.6614784E7,1.2834966E-2,1E0,1.5046106E7,1.365445E1,4.4699004E-3,4.524696E5,4.867052E6,1.411E4,-2.955908E-3,1.3045501E-3,1.086847E-2,-2.913287E-3,1.8224532E-3,-7.064658E-3,-1.611851E-2],"split_indices":[52,12,0,94,45,54,0,28,1,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.01E2,9E0,4.5E1,5.6E1,4E1,5E0,2.1E1,3.5E1,2E1,2E1,1.3E1,8E0,1.7E1,1.8E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.2407216E-2,-6.469195E-2,1.2917514E-1,-1.3305482E-1,8.7063685E-3,1.7180746E-2,3.3825461E-3,-1.0367118E-1,-1.4143053E-2,-2.800995E-2,5.2320273E-3,-1.7611243E-2,-1.7594157E-1,-2.5161863E-3,2.8142019E-3,-3.4132097E-3,4.8369654E-3,-2.0164028E-3,-1.055273E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,-1,15,17,-1,-1,-1,-1,-1,-1],"loss_changes":[8.2264227E-1,4.0759376E-1,3.7675595E-1,1.6597271E-1,1.4892823E-1,0E0,0E0,2.2702608E-1,0E0,6.491228E-2,0E0,1.18879534E-1,1.0371035E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,11,11,12,12],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,-1,16,18,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5328901E3,3.3209075E6,1.6451234E-2,1E0,3.9219662E5,1.7180746E-2,3.3825461E-3,1.39323E5,-1.4143053E-2,2.2539987E8,5.2320273E-3,2.848E3,2.5107703E-1,-2.5161863E-3,2.8142019E-3,-3.4132097E-3,4.8369654E-3,-2.0164028E-3,-1.055273E-2],"split_indices":[52,50,57,112,28,0,0,29,0,32,0,2,27,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,8E1,2.9E1,4.1E1,3.9E1,5E0,2.4E1,3.6E1,5E0,2.9E1,1E1,1.7E1,1.9E1,2.3E1,6E0,1.2E1,5E0,5E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.2073816E-2,-3.7920978E-2,1.4537922E-1,-2.1535464E-2,-1.1888381E-2,2.2219266E-3,1.157948E-2,-3.883534E-2,9.278212E-3,-1.6619451E-2,-1.7259702E-1,1.0882753E-2,-3.601114E-3,-1.6120436E-2,-5.0965743E-4,-1.9173332E-3,2.4473774E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,-1,-1,9,-1,11,13,15,-1,-1,-1,-1,-1],"loss_changes":[4.6132094E-1,3.246622E-1,1.3827667E-1,3.429952E-1,0E0,0E0,0E0,2.4972047E-1,0E0,1.1833942E-1,3.0829117E-1,1.0410246E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,9,9,10,10,11,11],"right_children":[2,4,6,8,-1,-1,-1,10,-1,12,14,16,-1,-1,-1,-1,-1],"split_conditions":[1.7618678E0,1.1384717E8,3.840785E1,8.640846E3,-1.1888381E-2,2.2219266E-3,1.157948E-2,2.915E4,9.278212E-3,5.9911316E7,3.5983E4,1.5493506E1,-3.601114E-3,-1.6120436E-2,-5.0965743E-4,-1.9173332E-3,2.4473774E-3],"split_indices":[39,45,56,52,0,0,0,9,0,45,9,56,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,9.6E1,1.5E1,9E1,6E0,8E0,7E0,8.4E1,6E0,7.3E1,1.1E1,5E1,2.3E1,5E0,6E0,2.2E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-8.127822E-3,-6.1608568E-2,1.0338962E-1,-1.3000979E-1,-7.352893E-3,1.7578249E-1,-6.2077377E-2,-9.5138706E-2,-1.2150145E-2,2.216739E-3,-3.0295856E-3,1.0482941E-1,1.56075815E-2,-8.02395E-3,1.7246661E-3,-6.4055678E-3,-9.7735036E-5,1.176723E-2,3.8119894E-2,4.7714347E-3,-1.7613741E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,-1,-1,-1,-1,-1,-1,19,-1,-1],"loss_changes":[6.803058E-1,2.8480294E-1,4.5412147E-1,1.1948234E-1,1.3194299E-1,2.365464E-1,1.2739232E-1,9.6589446E-2,0E0,0E0,0E0,1.658819E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.659675E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,18,18],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,-1,-1,-1,-1,-1,-1,20,-1,-1],"split_conditions":[1.1855E4,1.04856936E8,9.780346E9,9.1181945E4,2.7260774E7,1.1811867E4,2.3430255E5,1.4756987E-3,-1.2150145E-2,2.216739E-3,-3.0295856E-3,2.70482E5,1.56075815E-2,-8.02395E-3,1.7246661E-3,-6.4055678E-3,-9.7735036E-5,1.176723E-2,1.83841E2,4.7714347E-3,-1.7613741E-3],"split_indices":[2,7,5,33,45,52,33,39,0,0,0,29,0,0,0,0,0,0,56,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,7.6E1,3.6E1,3.3E1,4.3E1,2.5E1,1.1E1,2.7E1,6E0,2.2E1,2.1E1,1.8E1,7E0,5E0,6E0,1.9E1,8E0,5E0,1.3E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.4533859E-3,1.0192082E-2,-9.688716E-3,-3.5605602E-2,1.14674665E-1,-2.1512926E-2,-9.27663E-3,1.0250847E-2,7.692662E-4,-6.971884E-3,-7.7970345E-3,6.5022693E-3,-2.0218842E-2,-3.1883225E-3,3.795741E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,-1,3,5,7,9,-1,-1,-1,-1,11,-1,13,-1,-1],"loss_changes":[2.7483955E-1,0E0,3.3606353E-1,1.849128E-1,1.6916129E-1,1.332584E-1,0E0,0E0,0E0,0E0,1.3155204E-1,0E0,8.756603E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,10,10,12,12],"right_children":[2,-1,4,6,8,10,-1,-1,-1,-1,12,-1,14,-1,-1],"split_conditions":[6.241954E0,1.0192082E-2,2.0479662E0,1.1384717E8,2.1383E4,3.9245284E0,-9.27663E-3,1.0250847E-2,7.692662E-4,-6.971884E-3,3.31E2,6.5022693E-3,1.1309524E-1,-3.1883225E-3,3.795741E-4],"split_indices":[54,0,42,45,9,56,0,0,0,0,11,0,57,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,5E0,1.02E2,8.5E1,1.7E1,7.9E1,6E0,8E0,9E0,7E0,7.2E1,5E0,6.7E1,2.5E1,4.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.4753265E-2,-3.883647E-2,1.2301153E-1,-5.7421856E-2,8.632246E-3,1.6591991E-1,2.117629E-3,-4.078806E-2,-7.2545614E-3,9.4013605E-3,2.5936796E-3,1.94082E-4,-3.2700421E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,-1,-1,-1],"loss_changes":[6.313428E-1,3.0479303E-1,1.1520606E-1,9.839952E-2,0E0,6.256282E-2,0E0,7.0954114E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,-1,-1,-1],"split_conditions":[7.90671E-1,5.528599E3,1.33556795E1,1.01949844E6,8.632246E-3,2.5617347E5,2.117629E-3,2.8399336E7,-7.2545614E-3,9.4013605E-3,2.5936796E-3,1.94082E-4,-3.2700421E-3],"split_indices":[39,52,54,28,0,33,0,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,7.2E1,3.5E1,6.7E1,5E0,2.2E1,1.3E1,5.8E1,9E0,1.7E1,5E0,2.2E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-8.875987E-3,-3.778672E-2,7.138095E-3,-2.1789629E-2,-1.16383005E-2,-4.8333608E-2,1.411922E-1,-1.08193895E-2,-3.0880587E-2,1.1033845E-2,1.0133732E-4,3.435711E-2,-8.263579E-2,-1.0933941E-3,4.0219217E-3,-6.4934366E-3,-4.5638974E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,-1,13,15,-1,-1,-1,-1],"loss_changes":[5.20674E-1,3.0908513E-1,0E0,4.0050724E-1,0E0,2.345332E-1,1.5625784E-1,0E0,2.4931562E-1,0E0,0E0,9.3384475E-2,1.5321442E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,11,11,12,12],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,-1,14,16,-1,-1,-1,-1],"split_conditions":[1.7618678E0,1.5081978E8,7.138095E-3,8.517173E3,-1.16383005E-2,3.8864895E6,6.1177564E9,-1.08193895E-2,3.925844E7,1.1033845E-2,1.0133732E-4,2.951724E1,3.159013E3,-1.0933941E-3,4.0219217E-3,-6.4934366E-3,-4.5638974E-4],"split_indices":[39,45,0,4,0,45,12,0,45,0,0,56,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,9.6E1,1.7E1,9E1,6E0,7.8E1,1.2E1,6E0,7.2E1,7E0,5E0,3.2E1,4E1,1.5E1,1.7E1,2.3E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.6003396E-2,3.0611455E-2,-1.1025568E-2,1.0789993E-2,9.972225E-3,-6.5592965E-3,2.6710311E-2,1.1752905E-2,7.921727E-3,3.868227E-2,-3.3727847E-3,5.630519E-4,4.680189E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,5,-1,-1,7,9,-1,11,-1,-1,-1],"loss_changes":[3.7720868E-1,3.418362E-1,0E0,2.1317032E-1,0E0,0E0,1.6597E-1,1.6711162E-1,0E0,8.827382E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,9,9],"right_children":[2,4,-1,6,-1,-1,8,10,-1,12,-1,-1,-1],"split_conditions":[2.8800072E10,1.2769175E4,-1.1025568E-2,8.130303E-5,9.972225E-3,-6.5592965E-3,2.4241872E0,8.5959184E-1,7.921727E-3,3.842849E5,-3.3727847E-3,5.630519E-4,4.680189E-3],"split_indices":[5,52,0,42,0,0,39,27,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.8E1,5E0,8.9E1,9E0,8E0,8.1E1,7.4E1,7E0,5.6E1,1.8E1,3.9E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.4151866E-3,6.4988814E-2,-5.668025E-2,3.17518E-2,1.3620035E-2,-1.4370625E-1,-2.2488402E-4,7.106646E-2,-3.3498844E-3,-1.8989186E-1,-2.2095565E-3,-1.9496244E-3,3.8674185E-3,1.1071375E-1,-3.6220302E-4,-4.0046726E-3,-1.3884686E-2,8.7557005E-4,7.220783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,-1,-1,-1,-1,-1],"loss_changes":[3.881885E-1,3.2175255E-1,2.996658E-1,1.6760986E-1,0E0,1.0004681E-1,1.2507439E-1,9.4252914E-2,0E0,1.4238584E-1,0E0,0E0,0E0,6.845909E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,-1,-1,-1,-1,-1],"split_conditions":[9.581152E0,1.3127055E4,2.667836E6,8.63558E-1,1.3620035E-2,1.2166998E0,3.181762E5,2.8399336E7,-3.3498844E-3,7.158023E7,-2.2095565E-3,-1.9496244E-3,3.8674185E-3,1.6987854E1,-3.6220302E-4,-4.0046726E-3,-1.3884686E-2,8.7557005E-4,7.220783E-3],"split_indices":[53,4,32,27,0,57,28,45,0,7,0,0,0,56,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,4.5E1,6E1,4E1,5E0,2.3E1,3.7E1,2.9E1,1.1E1,1.5E1,8E0,2.5E1,1.2E1,1.9E1,1E1,8E0,7E0,6E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-6.9537484E-3,-9.411539E-3,9.024073E-3,2.2752177E-2,-9.121804E-3,8.571798E-3,1.0164975E-2,3.8311772E-2,-5.9407856E-2,2.4385255E-2,6.5965904E-3,-4.452982E-3,1.8178223E-3,7.1197366E-3,4.990366E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,-1,3,5,-1,7,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[3.431171E-1,0E0,2.9154548E-1,2.6361743E-1,0E0,1.9231705E-1,0E0,8.6680315E-2,9.497809E-2,1.00366935E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,9,9],"right_children":[2,-1,4,6,-1,8,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[1.21719E5,-9.411539E-3,2.8841052E10,1.1811867E4,-9.121804E-3,5.1121724E7,1.0164975E-2,3.200296E9,1.7561E4,9.021898E0,6.5965904E-3,-4.452982E-3,1.8178223E-3,7.1197366E-3,4.990366E-4],"split_indices":[1,0,5,52,0,45,0,12,10,54,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,8E0,1.05E2,9.9E1,6E0,9.3E1,6E0,6.5E1,2.8E1,5.8E1,7E0,2.1E1,7E0,5E0,5.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-5.521874E-3,-2.5739884E-2,8.766152E-3,-9.829054E-3,-1.1013514E-2,5.4145493E-2,-6.372576E-2,-1.433546E-4,4.9988353E-3,-4.518562E-3,2.144538E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,-1,-1,5,7,9,-1,-1,-1,-1],"loss_changes":[3.8561457E-1,2.3790303E-1,0E0,0E0,2.9883274E-1,1.096334E-1,1.5754879E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6],"right_children":[2,4,-1,-1,6,8,10,-1,-1,-1,-1],"split_conditions":[2.8937003E2,9.2433E4,8.766152E-3,-9.829054E-3,2.8399336E7,1.6511278E1,1.6142E4,-1.433546E-4,4.9988353E-3,-4.518562E-3,2.144538E-3],"split_indices":[56,1,0,0,45,56,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1E2,9.1E1,9E0,6E0,8.5E1,3.8E1,4.7E1,1.8E1,2E1,3.7E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[3.9606947E-3,-7.2830827E-3,8.8755535E-3,3.016015E-3,-9.187448E-3,7.1743117E-3,-5.55154E-3,3.3054683E-2,-4.186776E-2,-1.060149E-3,7.359999E-2,-6.557114E-2,1.3968206E-3,7.6131388E-3,2.1765323E-3,-2.190604E-3,-8.388777E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,-1,-1,7,9,11,-1,13,15,-1,-1,-1,-1,-1],"loss_changes":[2.3926999E-1,2.1103205E-1,0E0,1.3388176E-1,0E0,0E0,1.443912E-1,1.13006726E-1,9.072865E-2,0E0,6.241432E-2,7.97271E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,10,10,11,11],"right_children":[2,4,-1,6,-1,-1,8,10,12,-1,14,16,-1,-1,-1,-1,-1],"split_conditions":[1.1276298E9,6.2180454E-1,8.8755535E-3,6.541823E0,-9.187448E-3,7.1743117E-3,3.925844E7,1.632896E3,2.3272075E5,-1.060149E-3,2.2087815E3,1.1384717E8,1.3968206E-3,7.6131388E-3,2.1765323E-3,-2.190604E-3,-8.388777E-3],"split_indices":[32,38,0,54,0,0,45,4,33,0,4,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,1.11E2,6E0,1.06E2,5E0,5E0,1.01E2,4.9E1,5.2E1,2.1E1,2.8E1,3.9E1,1.3E1,6E0,2.2E1,3.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.4218885E-2,3.6395513E-2,-6.158378E-3,1.5431354E-3,1.6701478E-1,2.7241694E-2,-3.7278426E-3,1.1899082E-2,5.1193554E-3,5.4779977E-2,-3.0612508E-3,9.217004E-3,1.2007724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,-1,-1,-1],"loss_changes":[3.464281E-1,4.322195E-1,0E0,1.5547235E-1,7.051891E-2,1.4795054E-1,0E0,0E0,0E0,1.7818512E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,-1,-1,-1],"split_conditions":[1.3509979E10,1.5325E4,-6.158378E-3,3.925844E7,9.275501E-1,8.5154843E-1,-3.7278426E-3,1.1899082E-2,5.1193554E-3,2.1270042E1,-3.0612508E-3,9.217004E-3,1.2007724E-3],"split_indices":[5,2,0,45,57,27,0,0,0,58,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,9.4E1,1.4E1,7.5E1,1.9E1,5.7E1,1.8E1,7E0,1.2E1,4.4E1,1.3E1,7E0,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[5.195425E-4,-8.665807E-3,1.5898872E-2,2.8366486E-2,-9.420261E-3,1.1836274E-2,1.050685E-2,9.092679E-2,-2.7269159E-2,2.5245E-3,9.487622E-3,-5.3165564E-3,-5.698224E-3,4.836021E-3,-9.3669695E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,-1,3,5,-1,7,-1,9,11,-1,-1,-1,13,-1,-1],"loss_changes":[3.147629E-1,0E0,2.8100976E-1,3.092211E-1,0E0,2.901163E-1,0E0,1.16061926E-1,1.11525714E-1,0E0,0E0,0E0,7.54041E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,12,12],"right_children":[2,-1,4,6,-1,8,-1,10,12,-1,-1,-1,14,-1,-1],"split_conditions":[9.7888E4,-8.665807E-3,1.5081978E8,1.1811867E4,-9.420261E-3,1.5046106E7,1.050685E-2,7.090088E10,4E0,2.5245E-3,9.487622E-3,-5.3165564E-3,8.9E1,4.836021E-3,-9.3669695E-4],"split_indices":[1,0,45,52,0,45,0,31,8,0,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,8E0,1.04E2,9.9E1,5E0,9.2E1,7E0,3E1,6.2E1,2.3E1,7E0,1.2E1,5E1,5E0,4.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.0776274E-2,2.0160542E-3,-7.552887E-3,3.540441E-2,-6.988059E-2,8.279115E-3,1.0376339E-2,-7.058785E-3,-3.9763037E-2,-5.6304443E-3,5.6714476E-3,-2.7762694E-4,-7.097125E-3,4.383601E-3,-1.7290143E-2,-5.5909357E-3,-1.6996481E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,-1,5,7,-1,9,-1,11,13,-1,-1,-1,-1,15,-1,-1],"loss_changes":[2.0649628E-1,2.4972667E-1,0E0,2.3913899E-1,6.963436E-2,0E0,1.048904E-1,0E0,9.002261E-2,6.2109422E-2,0E0,0E0,0E0,0E0,6.588295E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,9,9,14,14],"right_children":[2,4,-1,6,8,-1,10,-1,12,14,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1E0,6.19999E9,-7.552887E-3,9.021898E0,1.8879009E3,8.279115E-3,1.2750684E9,-7.058785E-3,1.5939707E7,4.38756E2,5.6714476E-3,-2.7762694E-4,-7.097125E-3,4.383601E-3,6.642599E0,-5.5909357E-3,-1.6996481E-4],"split_indices":[112,5,0,54,52,0,7,0,47,4,0,0,0,0,58,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.02E2,8E0,7E1,3.2E1,1E1,6E1,8E0,2.4E1,5.3E1,7E0,1.9E1,5E0,5E0,4.8E1,5E0,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.3380788E-3,-1.4201705E-1,2.1001155E-2,-2.4022069E-3,-1.2442487E-2,-3.785247E-3,9.35993E-2,-5.043117E-3,2.4247026E-2,1.2288771E-2,2.1003154E-3,5.815997E-2,-1.1447934E-3,6.171314E-4,5.8538276E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,-1,-1,7,9,-1,11,-1,-1,13,-1,-1,-1],"loss_changes":[3.271448E-1,1.3595262E-1,1.8389785E-1,0E0,0E0,2.1915892E-1,1.9816607E-1,0E0,1.00442424E-1,0E0,0E0,1.0102192E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,11,11],"right_children":[2,4,6,-1,-1,8,10,-1,12,-1,-1,14,-1,-1,-1],"split_conditions":[4.46438E0,1.7302E4,1.0098112E2,-2.4022069E-3,-1.2442487E-2,6.7281544E-5,1.6451234E-2,-5.043117E-3,9.275501E-1,1.2288771E-2,2.1003154E-3,3.9219662E5,-1.1447934E-3,6.171314E-4,5.8538276E-3],"split_indices":[56,9,56,0,0,38,57,0,57,0,0,28,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,1.3E1,1.01E2,8E0,5E0,7.6E1,2.5E1,1.6E1,6E1,5E0,2E1,3.5E1,2.5E1,2.1E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-2.8046006E-3,7.185053E-2,-2.3968346E-2,8.606533E-3,-1.1270187E-3,-1.2283069E-2,-8.134298E-3,-3.685124E-2,2.880504E-2,-7.7827787E-4,-7.566328E-3,-2.4500198E-3,2.7295188E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,-1,-1,7,-1,9,11,-1,-1,-1,-1],"loss_changes":[1.644603E-1,2.3580986E-1,1.3425148E-1,0E0,0E0,7.778043E-2,0E0,1.1658042E-1,6.632311E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8],"right_children":[2,4,6,-1,-1,8,-1,10,12,-1,-1,-1,-1],"split_conditions":[7.573E3,2.5927516E11,1.12782936E8,8.606533E-3,-1.1270187E-3,1.9556936E3,-8.134298E-3,1.01949844E6,1E0,-7.7827787E-4,-7.566328E-3,-2.4500198E-3,2.7295188E-3],"split_indices":[9,31,45,0,0,52,0,28,109,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,2.2E1,8E1,1E1,1.2E1,7.5E1,5E0,4.7E1,2.8E1,4.1E1,6E0,7E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-1.0829205E-2,-6.0842793E-2,3.9384924E-2,-4.3133177E-2,-9.458339E-3,5.9318375E-2,-6.2933816E-3,1.0806314E-5,-6.0410625E-3,2.6094636E-2,1.7171904E-1,6.1536888E-3,1.5394527E-6,1.187545E-2,3.368726E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,-1,-1,-1,-1],"loss_changes":[2.7111492E-1,1.2018825E-1,1.9000325E-1,1.7343713E-1,0E0,1.785032E-1,0E0,0E0,0E0,1.0264689E-1,6.555852E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,-1,-1,-1,-1],"split_conditions":[1.8879009E3,9.493188E-1,2.2437828E7,1.6506441E9,-9.458339E-3,9.669896E3,-6.2933816E-3,1.0806314E-5,-6.0410625E-3,2.44168E5,8.29017E0,6.1536888E-3,1.5394527E-6,1.187545E-2,3.368726E-3],"split_indices":[52,57,47,5,0,52,0,0,0,29,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,5.3E1,5.3E1,4.8E1,5E0,4.8E1,5E0,3.2E1,1.6E1,3.8E1,1E1,7E0,3.1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[2.4978723E-2,4.745622E-3,1.2588058E-1,3.872041E-2,-3.763145E-2,8.654188E-3,-4.64517E-4,-2.0561889E-2,8.090342E-2,6.448543E-4,-3.4269015E-3,1.9897146E-3,-3.2154971E-3,5.637019E-4,6.44904E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1989085E-1,1.3247453E-1,1.3057396E-1,1.291995E-1,7.016848E-2,0E0,0E0,6.490928E-2,1.0708068E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,-1,-1,-1,-1],"split_conditions":[6.622281E3,2.8399336E7,9.299267E9,1.3890547E1,1.1727146E6,8.654188E-3,-4.64517E-4,1.5662651E0,3.136809E5,6.448543E-4,-3.4269015E-3,1.9897146E-3,-3.2154971E-3,5.637019E-4,6.44904E-3],"split_indices":[52,45,12,56,32,0,0,56,28,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.07E2,9E1,1.7E1,5E1,4E1,1.2E1,5E0,2.1E1,2.9E1,1.6E1,2.4E1,9E0,1.2E1,1.3E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[7.4695996E-3,1.8530475E-2,-8.714375E-3,-1.6432649E-3,1.5521759E-1,3.844863E-2,-4.1698642E-2,1.0603268E-2,2.1469078E-3,6.3581406E-3,1.1586775E-2,-2.9044056E-3,3.2015156E-3,-1.8966751E-3,2.21128E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,-1,-1,-1,-1],"loss_changes":[2.2309515E-1,2.787616E-1,0E0,1.4452802E-1,8.2194746E-2,1.1072358E-1,9.26658E-2,0E0,0E0,0E0,6.406058E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,-1,-1,-1,-1],"split_conditions":[2.8841052E10,8.351458E3,-8.714375E-3,2.8399336E7,3.813527E2,2.2333334E1,3.3E1,1.0603268E-2,2.1469078E-3,6.3581406E-3,1.0323E4,-2.9044056E-3,3.2015156E-3,-1.8966751E-3,2.21128E-3],"split_indices":[5,52,0,45,56,58,8,0,0,0,11,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,1E2,5E0,8.8E1,1.2E1,4.4E1,4.4E1,7E0,5E0,9E0,3.5E1,3.8E1,6E0,1.4E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.5576296E-4,-1.881275E-2,4.242411E-3,-9.9482145E-3,-4.215033E-3,7.999445E-3,-7.3660742E-3,-2.885891E-4,6.8017202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,-1,5,7,-1,-1,-1],"loss_changes":[1.6754167E-1,2.2526278E-1,0E0,0E0,1.421322E-1,1.358256E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5],"right_children":[2,4,-1,-1,6,8,-1,-1,-1],"split_conditions":[1.7123758E0,7.110474E7,4.242411E-3,-9.9482145E-3,1.4974915E8,7E1,-7.3660742E-3,-2.885891E-4,6.8017202E-3],"split_indices":[39,5,0,0,45,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[9.9E1,8.2E1,1.7E1,5E0,7.7E1,7.2E1,5E0,6.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.0631954E-2,-7.066705E-3,1.445962E-1,-6.737896E-2,2.4110317E-2,1.040328E-2,2.5998012E-3,-1.0397797E-2,-1.6738885E-3,-3.1364914E-3,4.5061883E-2,9.284921E-3,2.6180506E-2,-1.9109672E-3,2.4686567E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,-1,-1,-1,-1,-1,11,-1,13,-1,-1],"loss_changes":[2.642015E-1,1.8777983E-1,7.148379E-2,1.5695703E-1,1.2582822E-1,0E0,0E0,0E0,0E0,0E0,1.4454173E-1,0E0,8.178882E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,10,10,12,12],"right_children":[2,4,6,8,10,-1,-1,-1,-1,-1,12,-1,14,-1,-1],"split_conditions":[1.1803382E4,1.04856936E8,1.0889231E1,5.279518E7,1E0,1.040328E-2,2.5998012E-3,-1.0397797E-2,-1.6738885E-3,-3.1364914E-3,1.8970638E1,9.284921E-3,1.0953223E3,-1.9109672E-3,2.4686567E-3],"split_indices":[4,7,54,5,109,0,0,0,0,0,58,0,52,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,9.8E1,1.2E1,3.3E1,6.5E1,6E0,6E0,5E0,2.8E1,1.2E1,5.3E1,5E0,4.8E1,1.3E1,3.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[4.31502E-3,-6.065101E-3,1.4160887E-2,2.2746312E-2,-6.063851E-3,-3.485448E-3,7.3920935E-2,1.583926E-3,-1.9170303E-3,1.0098455E-2,4.7202837E-2,4.9316734E-3,7.640882E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,-1,3,5,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[1.6251522E-1,0E0,1.4406149E-1,1.506065E-1,0E0,9.930851E-2,1.268434E-1,0E0,0E0,0E0,8.2113385E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,10,10],"right_children":[2,-1,4,6,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[9.2433E4,-6.065101E-3,2.8841052E10,1.1855E4,-6.063851E-3,2.2232678E7,1.6451234E-2,1.583926E-3,-1.9170303E-3,1.0098455E-2,1.5E1,4.9316734E-3,7.640882E-5],"split_indices":[1,0,5,2,0,45,57,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,8E0,1.17E2,1.11E2,6E0,7.4E1,3.7E1,3.7E1,3.7E1,5E0,3.2E1,1.4E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.3005984E-2,3.768742E-2,-4.378033E-2,4.5754178E-4,6.865282E-3,-7.953688E-4,-8.389459E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,-1,-1,-1,-1],"loss_changes":[1.6142872E-1,2.3381251E-1,1.2111621E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2],"right_children":[2,4,6,-1,-1,-1,-1],"split_conditions":[6.5566176E7,1.286E4,1.1717857E1,4.5754178E-4,6.865282E-3,-7.953688E-4,-8.389459E-3],"split_indices":[45,2,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[1.13E2,7.9E1,3.4E1,6.3E1,1.6E1,2.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-8.883006E-3,1.4155968E-3,-8.418813E-3,1.1250664E-2,-8.515109E-3,-1.5136791E-2,7.1680844E-2,2.6128364E-3,-3.596047E-2,4.9439385E-3,7.549525E-5,-2.7429212E-3,6.9260993E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,-1,-1,-1],"loss_changes":[2.0243452E-1,1.989865E-1,0E0,1.7330167E-1,0E0,1.1129223E-1,7.023305E-2,0E0,6.2154815E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,-1,-1,-1],"split_conditions":[6.2640357E-1,2.8800072E10,-8.418813E-3,4.798087E6,-8.515109E-3,6.4604245E6,7.02347E1,2.6128364E-3,2.5E-1,4.9439385E-3,7.549525E-5,-2.7429212E-3,6.9260993E-4],"split_indices":[38,5,0,1,0,45,57,0,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.18E2,1.12E2,6E0,1.07E2,5E0,7.5E1,3.2E1,1.7E1,5.8E1,2.2E1,1E1,4.1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[3.0341905E-3,1.3864015E-2,-8.792423E-3,-4.87888E-2,3.2722287E-2,2.0171732E-3,-3.934401E-3,4.866448E-2,-2.8282436E-2,1.0269172E-2,1.3433584E-3,-4.1424725E-3,1.5556462E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,5,7,-1,-1,9,11,-1,-1,-1,-1],"loss_changes":[2.16445E-1,1.2211192E-1,0E0,7.38537E-2,7.8207724E-2,0E0,0E0,2.0950508E-1,6.1558798E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8],"right_children":[2,4,-1,6,8,-1,-1,10,12,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1E0,-8.792423E-3,6.9237E4,1.1828883E1,2.0171732E-3,-3.934401E-3,2.55E2,4.798087E6,1.0269172E-2,1.3433584E-3,-4.1424725E-3,1.5556462E-3],"split_indices":[45,16,0,12,53,0,0,10,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,1.01E2,5E0,2.3E1,7.8E1,6E0,1.7E1,6.2E1,1.6E1,6E0,5.6E1,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[2.2709784E-3,9.780048E-3,-6.6350596E-3,5.612797E-2,-1.1414202E-2,-5.953896E-4,4.2612064E-3,-3.495079E-2,6.767521E-2,-5.2777114E-3,-1.2151804E-2,7.5596427E-3,1.0482897E-3,-1.6330633E-3,2.5945585E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,7,-1,-1,9,11,-1,13,-1,-1,-1,-1],"loss_changes":[1.2267444E-1,1.09677E-1,0E0,7.741308E-2,1.4570041E-1,0E0,0E0,9.9961795E-2,6.989384E-2,0E0,6.8458974E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,10,10],"right_children":[2,4,-1,6,8,-1,-1,10,12,-1,14,-1,-1,-1,-1],"split_conditions":[6.2640357E-1,1.5046106E7,-6.6350596E-3,2.1816934E8,1.3400168E7,-5.953896E-4,4.2612064E-3,2.2196926E-2,8.3512023E9,-5.2777114E-3,7.434519E5,7.5596427E-3,1.0482897E-3,-1.6330633E-3,2.5945585E-3],"split_indices":[38,45,0,7,1,0,0,57,5,0,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,1.1E2,5E0,3.4E1,7.6E1,1.1E1,2.3E1,5.9E1,1.7E1,1.3E1,4.6E1,5E0,1.2E1,3.5E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-3.619274E-4,9.462665E-3,-8.339842E-3,-9.4632973E-4,5.83957E-3,-9.464961E-3,4.6004215E-3,1.2846318E-2,-5.2342758E-2,-2.0119187E-4,7.463207E-2,-8.769735E-4,-5.927759E-3,4.310353E-4,7.255284E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,-1,7,-1,9,11,-1,13,-1,-1,-1,-1],"loss_changes":[1.8737061E-1,1.2157788E-1,0E0,8.0185786E-2,0E0,8.677168E-2,0E0,6.333628E-2,7.214601E-2,0E0,6.2925994E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,10,10],"right_children":[2,4,-1,6,-1,8,-1,10,12,-1,14,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.2769175E4,-8.339842E-3,1E0,5.83957E-3,7.959073E-1,4.6004215E-3,4.3316594E5,7.9918444E5,-2.0119187E-4,4.349136E6,-8.769735E-4,-5.927759E-3,4.310353E-4,7.255284E-3],"split_indices":[45,52,0,94,0,27,0,28,28,0,32,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,1.04E2,5E0,9.6E1,8E0,8.9E1,7E0,5.9E1,3E1,4.7E1,1.2E1,2.1E1,9E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[8.412533E-4,9.406689E-3,-7.5008017E-3,-1.5037388E-3,5.6930063E-3,4.2274345E-2,-3.13937E-2,-1.1925517E-3,3.5936898E-3,-5.216197E-3,-1.0067043E-2,1.4397729E-3,-2.0397578E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,11,-1,-1],"loss_changes":[1.5353645E-1,1.288089E-1,0E0,1.3217966E-1,0E0,8.9443475E-2,9.658695E-2,0E0,0E0,0E0,6.24963E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,10,10],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,12,-1,-1],"split_conditions":[5.3296334E-1,1.0825864E4,-7.5008017E-3,2.2232678E7,5.6930063E-3,5.682206E2,2.2196926E-2,-1.1925517E-3,3.5936898E-3,-5.216197E-3,7.118011E-1,1.4397729E-3,-2.0397578E-3],"split_indices":[38,52,0,45,0,52,57,0,0,0,27,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,1.08E2,5E0,9.9E1,9E0,4E1,5.9E1,1.3E1,2.7E1,1.2E1,4.7E1,2.1E1,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-2.6660373E-3,7.830731E-3,-7.985858E-3,5.924881E-3,-1.4741883E-3,-5.732011E-4,3.44783E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[1.6820106E-1,9.9288195E-2,0E0,0E0,6.637025E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[1.5081978E8,1.7043712E3,-7.985858E-3,5.924881E-3,2.1729739E0,-5.732011E-4,3.44783E-3],"split_indices":[45,48,0,0,39,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[9.7E1,9.2E1,5E0,6E0,8.6E1,7.6E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-3.559119E-3,-5.6850225E-2,1.4244128E-2,-1.2054802E-3,-8.741216E-3,2.8829167E-2,-4.0171524E-3,3.6188033E-3,7.724843E-2,-1.2470293E-3,2.2360063E-3,8.637192E-3,1.9301953E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,-1,-1,7,-1,9,11,-1,-1,-1,-1],"loss_changes":[1.13711596E-1,1.140629E-1,1.2998843E-1,0E0,0E0,9.615317E-2,0E0,6.744099E-2,9.49685E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8],"right_children":[2,4,6,-1,-1,8,-1,10,12,-1,-1,-1,-1],"split_conditions":[2.3E1,2.915E4,1.3509979E10,-1.2054802E-3,-8.741216E-3,1.1855E4,-4.0171524E-3,3.181762E5,3.2258064E-2,-1.2470293E-3,2.2360063E-3,8.637192E-3,1.9301953E-3],"split_indices":[3,9,5,0,0,2,0,28,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.18E2,2.9E1,8.9E1,2.4E1,5E0,7.8E1,1.1E1,5.2E1,2.6E1,3.1E1,2.1E1,6E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.0284273E-3,-6.824778E-3,1.0574329E-2,2.0316768E-2,-7.3571787E-3,1.00792125E-2,7.4292943E-3,6.2124852E-2,-1.3296215E-2,1.4720943E-3,6.8565332E-3,-4.1991086E-3,5.2386364E-5,1.4444544E-3,-2.1291042E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,-1,3,5,-1,7,-1,9,11,-1,-1,-1,13,-1,-1],"loss_changes":[1.5056768E-1,0E0,1.6774692E-1,1.3336381E-1,0E0,1.1396881E-1,0E0,6.8810254E-2,6.371028E-2,0E0,0E0,0E0,7.472151E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,12,12],"right_children":[2,-1,4,6,-1,8,-1,10,12,-1,-1,-1,14,-1,-1],"split_conditions":[1.458E3,-6.824778E-3,1.2511909E8,1.0825864E4,-7.3571787E-3,1.5046106E7,7.4292943E-3,7.359468E8,2.0547945E-2,1.4720943E-3,6.8565332E-3,-4.1991086E-3,4.349136E6,1.4444544E-3,-2.1291042E-3],"split_indices":[2,0,45,52,0,45,0,5,57,0,0,0,32,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.09E2,6E0,1.03E2,9.8E1,5E0,9.2E1,6E0,2.8E1,6.4E1,2.1E1,7E0,9E0,5.5E1,3.3E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-5.595835E-3,2.0529882E-3,-7.3246066E-3,-2.1712154E-2,4.3876022E-2,-9.445672E-3,-5.894424E-3,7.0291184E-2,-6.76704E-4,1.2165492E-4,-4.742967E-3,5.9981365E-3,5.822534E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,-1,-1,-1],"loss_changes":[1.3401484E-1,1.1525463E-1,0E0,8.963826E-2,6.516448E-2,7.115512E-2,0E0,8.904736E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,5.4218586E1,-7.3246066E-3,1E0,1.0494656E1,8.67241E9,-5.894424E-3,8.69374E5,-6.76704E-4,1.2165492E-4,-4.742967E-3,5.9981365E-3,5.822534E-4],"split_indices":[45,56,0,112,53,5,0,29,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,1.14E2,5E0,7.3E1,4.1E1,6.6E1,7E0,2.8E1,1.3E1,5.9E1,7E0,1.4E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[6.642722E-4,8.91025E-3,-7.027282E-3,-1.2905855E-3,5.762041E-3,-3.0940599E-3,1.4575631E-2,-4.8404644E-4,2.379002E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[1.3473488E-1,1.1918801E-1,0E0,9.827835E-2,0E0,0E0,6.7568585E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[1.54005E8,1.1811867E4,-7.027282E-3,1.13892675E-4,5.762041E-3,-3.0940599E-3,4.2298006E5,-4.8404644E-4,2.379002E-3],"split_indices":[45,52,0,38,0,0,28,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.05E2,5E0,9.7E1,8E0,1.9E1,7.8E1,4.6E1,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[3.3185554E-3,-7.2224075E-3,1.5362815E-2,3.1244617E-2,-1.9298972E-3,6.7653726E-3,1.0666129E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":79,"left_children":[1,-1,3,5,-1,-1,-1],"loss_changes":[2.0062606E-1,0E0,8.981916E-2,7.59075E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3],"right_children":[2,-1,4,6,-1,-1,-1],"split_conditions":[5.279518E7,-7.2224075E-3,6.784095E7,8.270677E0,-1.9298972E-3,6.7653726E-3,1.0666129E-3],"split_indices":[5,0,45,54,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[1.07E2,7E0,1E2,7.8E1,2.2E1,5E0,7.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[1.4239065E-3,-3.1495534E-4,5.5434406E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":80,"left_children":[1,-1,-1],"loss_changes":[9.575788E-2,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[1.0825864E4,-3.1495534E-4,5.5434406E-3],"split_indices":[52,0,0],"split_type":[0,0,0],"sum_hessian":[1.05E2,9.9E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-6.6636875E-3,2.579614E-3,-7.579373E-3,1.146279E-2,-6.5631163E-3,7.279338E-2,-7.4363174E-3,3.6465947E-2,7.921325E-3,-2.8352174E-3,9.36552E-4,-4.0674393E-4,5.8164727E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,-1,5,-1,7,9,11,-1,-1,-1,-1,-1],"loss_changes":[1.4308265E-1,1.2179348E-1,0E0,1.08337E-1,0E0,6.6598654E-2,9.9683434E-2,6.481373E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7],"right_children":[2,4,-1,6,-1,8,10,12,-1,-1,-1,-1,-1],"split_conditions":[1.1971412E8,6.2640357E-1,-7.579373E-3,2.55E2,-6.5631163E-3,6.939E3,2.518E3,2.0979407E-4,7.921325E-3,-2.8352174E-3,9.36552E-4,-4.0674393E-4,5.8164727E-3],"split_indices":[45,38,0,10,0,2,10,38,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.02E2,9.7E1,5E0,9.2E1,5E0,2.1E1,7.1E1,1.6E1,5E0,2.4E1,4.7E1,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-8.889863E-3,-7.887498E-3,2.92419E-3,2.3160731E-2,-5.8792423E-2,-6.18923E-5,7.315669E-2,2.540329E-3,-4.8976457E-3,5.388627E-3,-5.928418E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,-1,3,5,7,-1,9,-1,-1,-1,-1],"loss_changes":[1.787745E-1,0E0,1.1626331E-1,8.5622795E-2,1.1367969E-1,0E0,7.6252714E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6],"right_children":[2,-1,4,6,8,-1,10,-1,-1,-1,-1],"split_conditions":[9.7888E4,-7.887498E-3,1.1748485E1,3.1681887E3,6.948189E7,-6.18923E-5,9.08277E6,2.540329E-3,-4.8976457E-3,5.388627E-3,-5.928418E-4],"split_indices":[1,0,53,52,7,0,47,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.7E1,6E0,9.1E1,6.9E1,2.2E1,4.7E1,2.2E1,6E0,1.6E1,1.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.5367227E-2,7.297797E-3,6.387259E-3,3.2292552E-2,-1.0123362E-3,-1.6494689E-4,7.2241664E-2,5.0720526E-3,-6.773116E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[1.0154271E-1,7.328588E-2,0E0,7.868396E-2,0E0,0E0,7.433988E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[1.2769175E4,2.727403E7,6.387259E-3,3.136809E5,-1.0123362E-3,-1.6494689E-4,1.4922581E1,5.0720526E-3,-6.773116E-4],"split_indices":[52,45,0,28,0,0,54,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,1.02E2,6E0,5.4E1,4.8E1,2.9E1,2.5E1,1.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-7.461209E-3,-9.058897E-4,-6.1733914E-3,-2.0222554E-2,4.9595907E-2,-8.119376E-3,-5.6240303E-3,4.7065825E-3,-7.336239E-4,7.938675E-4,-2.1210227E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,-1],"loss_changes":[9.115726E-2,1.09286375E-1,0E0,9.309606E-2,9.70187E-2,6.457455E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,-1],"split_conditions":[1.4974915E8,1.3137E4,-6.1733914E-3,1E0,8.50999E9,2.8399336E7,-5.6240303E-3,4.7065825E-3,-7.336239E-4,7.938675E-4,-2.1210227E-3],"split_indices":[45,2,0,112,5,45,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,1.1E2,5E0,8E1,3E1,7.2E1,8E0,1.7E1,1.3E1,4.3E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.1646565E-2,-6.9031524E-3,-2.457914E-3,2.9924633E-3,-2.5557801E-2,-1.364295E-2,-5.9868805E-3,-1.5466774E-3,4.3131698E-2,4.587563E-3,-1.2963288E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,-1,3,-1,5,7,-1,-1,9,-1,-1],"loss_changes":[1.2836365E-1,0E0,1.5149763E-1,0E0,8.5960194E-2,7.202258E-2,0E0,0E0,6.3866414E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,8,8],"right_children":[2,-1,4,-1,6,8,-1,-1,10,-1,-1],"split_conditions":[2.1662036E9,-6.9031524E-3,1.4163358E7,2.9924633E-3,1.2511909E8,1.0296329E7,-5.9868805E-3,-1.5466774E-3,7.066745E9,4.587563E-3,-1.2963288E-3],"split_indices":[31,0,45,0,45,1,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.06E2,6E0,1E2,2.6E1,7.4E1,6.7E1,7E0,5.1E1,1.6E1,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-6.526208E-3,3.1855507E-3,-4.305906E-3,1.1215652E-2,-5.639173E-3,-1.3208606E-3,7.315051E-2,-5.5360217E-3,4.0598874E-4,7.8141475E-3,6.2466855E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,-1],"loss_changes":[8.350725E-2,9.161706E-2,0E0,6.920041E-2,0E0,8.3215736E-2,7.765155E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,-1],"split_conditions":[1E0,3.6030095E10,-4.305906E-3,1.0296329E7,-5.639173E-3,2.8283243E2,8.260109E9,-5.5360217E-3,4.0598874E-4,7.8141475E-3,6.2466855E-4],"split_indices":[112,5,0,1,0,33,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,9.3E1,1E1,8.8E1,5E0,7.4E1,1.4E1,5E0,6.9E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.6658073E-3,1.0802038E-2,-6.5668E-3,-2.2322443E-4,2.9200902E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":87,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.4068499E-1,8.0386445E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[6.2180454E-1,9.7752365E1,-6.5668E-3,-2.2322443E-4,2.9200902E-3],"split_indices":[38,56,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[1.1E2,1.04E2,6E0,8E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-1.0004009E-2,-5.4084696E-3,-5.630391E-4,2.657276E-2,-3.8936537E-2,5.4193934E-4,5.8737854E-3,-7.741314E-4,-5.2528814E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,-1,3,5,7,-1,-1,-1,-1],"loss_changes":[1.0170278E-1,0E0,1.03081614E-1,8.3150476E-2,6.370181E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4],"right_children":[2,-1,4,6,8,-1,-1,-1,-1],"split_conditions":[9.7888E4,-5.4084696E-3,1.0636054E1,1.3772013E4,7.526629E7,5.4193934E-4,5.8737854E-3,-7.741314E-4,-5.2528814E-3],"split_indices":[1,0,53,4,45,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[1.05E2,8E0,9.7E1,5.7E1,4E1,5E1,7E0,3.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.7616575E-3,1.549951E-2,-5.213885E-3,2.3215601E-2,-4.5875134E-3,1.0255133E-2,4.5636185E-3,3.6268517E-2,-2.2189187E-2,6.1359955E-5,3.6793645E-3,2.597462E-3,-2.478611E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,-1,7,-1,9,11,-1,-1,-1,-1],"loss_changes":[1.1266183E-1,9.0663716E-2,0E0,8.925169E-2,0E0,7.169964E-2,0E0,6.524007E-2,8.630707E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8],"right_children":[2,4,-1,6,-1,8,-1,10,12,-1,-1,-1,-1],"split_conditions":[1E0,8.2E1,-5.213885E-3,1.3400168E7,-4.5875134E-3,2.727403E7,4.5636185E-3,3.566468E5,1.9612434E5,6.1359955E-5,3.6793645E-3,2.597462E-3,-2.478611E-3],"split_indices":[112,3,0,1,0,45,0,28,28,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,1.03E2,7E0,9.7E1,6E0,8.3E1,1.4E1,4.6E1,3.7E1,2.5E1,2.1E1,1E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.8558659E-3,8.28561E-3,-5.7368297E-3,3.5866885E-3,-2.6510477E-3,-4.4782595E-3,5.9814047E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[9.15044E-2,8.077757E-2,0E0,0E0,1.3188091E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[1.54005E8,8.1837E4,-5.7368297E-3,3.5866885E-3,1.1788061E4,-4.4782595E-3,5.9814047E-4],"split_indices":[45,12,0,0,47,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[1.16E2,1.11E2,5E0,1.5E1,9.6E1,1.3E1,8.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[2.3367477E-3,-5.2595353E-3,1.084929E-2,1.8828567E-2,-4.9589355E-3,-8.538805E-3,2.4187453E-3,3.162563E-2,-3.6990356E-3,6.9756713E-3,1.1561598E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,-1,3,5,-1,7,-1,9,-1,-1,-1],"loss_changes":[1.06019065E-1,0E0,9.591629E-2,8.389756E-2,0E0,1.4685476E-1,0E0,1.11425504E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7],"right_children":[2,-1,4,6,-1,8,-1,10,-1,-1,-1],"split_conditions":[1.458E3,-5.2595353E-3,2.710357E1,5.4218586E1,-4.9589355E-3,2.727403E7,2.4187453E-3,3.42E2,-3.6990356E-3,6.9756713E-3,1.1561598E-4],"split_indices":[2,0,54,56,0,45,0,10,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,7E0,1.03E2,9.7E1,6E0,5.2E1,4.5E1,3.3E1,1.9E1,6E0,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-4.4803033E-3,-6.2135085E-3,5.779433E-3,1.444942E-2,-6.20402E-3,6.70225E-3,5.2085156E-3,-2.9578155E-3,1.961053E-2,5.153887E-3,3.5284073E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,-1,3,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[1.3287966E-1,0E0,1.14462264E-1,6.564605E-2,0E0,7.644816E-2,0E0,0E0,7.733616E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,8,8],"right_children":[2,-1,4,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[9.7888E4,-6.2135085E-3,1.1971412E8,1.3772013E4,-6.20402E-3,5.981675E-4,5.2085156E-3,-2.9578155E-3,2.2333334E1,5.153887E-3,3.5284073E-4],"split_indices":[1,0,45,4,0,42,0,0,58,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.03E2,7E0,9.6E1,9.1E1,5E0,8.5E1,6E0,1.3E1,7.2E1,8E0,6.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[6.2235014E-4,8.624424E-3,-5.6756255E-3,-1.8802976E-4,4.6249893E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":93,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.0378851E-1,1.12308785E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[3.813527E2,6.083071E3,-5.6756255E-3,-1.8802976E-4,4.6249893E-3],"split_indices":[56,52,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[1.08E2,1.02E2,6E0,9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-4.1001616E-3,2.5719465E-3,-6.519312E-3,4.5481796E-4,-6.116468E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":94,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.07472144E-1,1.0492241E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[2.8841052E10,6.2640357E-1,-6.519312E-3,4.5481796E-4,-6.116468E-3],"split_indices":[5,38,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[1.22E2,1.17E2,5E0,1.12E2,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[7.4578915E-3,6.2345788E-2,-9.467462E-3,1.0707365E-1,2.876705E-4,-2.201061E-5,-6.4192307E-3,6.0674245E-4,7.413058E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[1.06495924E-1,6.748139E-2,9.757523E-2,6.334312E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[2.0382088E1,6.15786E5,4.2484435E-1,2.8E1,2.876705E-4,-2.201061E-5,-6.4192307E-3,6.0674245E-4,7.413058E-3],"split_indices":[58,29,38,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,2.6E1,8.7E1,1.4E1,1.2E1,8.2E1,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-azure","platform_databricks-aws","platform_dataproc","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"92"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-3.8167074E-2,-4.4051185E-1,4.0796763E-1,-5.536225E-1,2.0302583E-1,6.895089E-2,7.212263E-1,-4.3062454E-1,-3.994903E-2,-5.9536183E-3,2.6536085E-2,-1.21464774E-1,2.9800472E-1,5.764676E-1,4.91494E-2,-3.6988802E-2,-3.1689176E-1,4.3454603E-3,-1.4175118E-2,4.5323655E-1,-1.9586764E-3,1.3316521E-2,3.092746E-2,-2.0138817E-1,-2.4288664E-2,1.0125012E-2,2.8836248E-2,-2.6427853E-1,2.5175828E-3,-1.5518838E-2,-4.722127E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,21,-1,-1,23,-1,-1,25,-1,-1,-1,27,-1,-1,-1,29,-1,-1,-1],"loss_changes":[2.118121E1,4.6254416E0,5.8974676E0,1.4838848E0,1.2247881E0,1.2574292E0,6.994276E-1,1.2799516E0,0E0,0E0,0E0,6.230878E-1,7.027323E-1,2.8520107E-1,0E0,0E0,5.839739E-1,0E0,0E0,1.9889164E-1,0E0,0E0,0E0,3.495276E-1,0E0,0E0,0E0,1.3847983E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,13,13,16,16,19,19,23,23,27,27],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,22,-1,-1,24,-1,-1,26,-1,-1,-1,28,-1,-1,-1,30,-1,-1,-1],"split_conditions":[2.3514317E8,2.561633E3,1.0639322E0,5.501383E9,6E0,8.182337E8,2.3431041E0,9.2433E4,-3.994903E-2,-5.9536183E-3,2.6536085E-2,4.5843E7,1.3649979E10,2.951724E1,4.91494E-2,-3.6988802E-2,3.925844E7,4.3454603E-3,-1.4175118E-2,2.3E1,-1.9586764E-3,1.3316521E-2,3.092746E-2,1.6987854E1,-2.4288664E-2,1.0125012E-2,2.8836248E-2,7.307441E-1,2.5175828E-3,-1.5518838E-2,-4.722127E-3],"split_indices":[7,60,50,5,8,7,47,1,0,0,0,53,5,64,0,0,53,0,0,3,0,0,0,64,0,0,0,47,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.16E2,6.1E1,5.5E1,5.2E1,9E0,2.7E1,2.8E1,3.8E1,1.4E1,5E0,4E0,1.5E1,1.2E1,2.1E1,7E0,8E0,3E1,7E0,8E0,8E0,4E0,5E0,1.6E1,2E1,1E1,4E0,4E0,1.6E1,4E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.640876E-2,-2.91722E-1,6.0442823E-1,-4.234082E-1,2.7467242E-1,8.1594086E-1,2.080579E-1,-6.867099E-1,-2.55407E-1,2.7463872E-2,9.700277E-2,1.5894843E-2,8.7992567E-1,3.128635E-1,6.514409E-4,-7.895494E-1,-1.34139145E-2,-7.132477E-2,-3.7767556E-1,-2.2447712E-3,1.1958817E-2,3.3107832E-2,6.021614E-2,6.78511E-3,1.8632792E-2,-2.3951197E-2,-4.4939898E-2,-1.2522512E-2,6.943018E-2,-2.5032744E-2,-1.0120403E-2,1.027594E-2,-3.0370075E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,-1],"loss_changes":[2.2116302E1,5.4119287E0,3.6029186E0,2.3902397E0,6.9143546E-1,6.035805E-1,3.421244E-1,6.9641113E-1,8.058386E-1,0E0,2.3268822E-1,0E0,9.0371704E-1,8.978081E-2,0E0,2.6969814E-1,0E0,4.481033E-1,4.2314863E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0755425E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,15,15,17,17,18,18,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,3.5328901E3,1.4046429E1,2.2392191E4,2.5927516E11,1.29240516E5,1.925776E1,3.0387878E3,5.4470426E-1,2.7463872E-2,1.8302105E6,1.5894843E-2,2.097152E9,5.93E3,6.514409E-4,2.8E2,-1.34139145E-2,1.3564575E5,1.9452468E-2,-2.2447712E-3,1.1958817E-2,3.3107832E-2,6.021614E-2,6.78511E-3,1.8632792E-2,-2.3951197E-2,-4.4939898E-2,-1.2522512E-2,1.0428423E9,-2.5032744E-2,-1.0120403E-2,1.027594E-2,-3.0370075E-3],"split_indices":[47,60,62,59,39,36,62,4,35,0,40,0,7,2,0,0,0,36,46,0,0,0,0,0,0,0,0,0,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,7E1,4.4E1,5.7E1,1.3E1,2.8E1,1.6E1,2.1E1,3.6E1,4E0,9E0,4E0,2.4E1,1E1,6E0,1.6E1,5E0,1.5E1,2.1E1,5E0,4E0,1.8E1,6E0,4E0,6E0,7E0,9E0,6E0,9E0,1E1,1.1E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.2588393E-2,-3.7784308E-1,4.6079767E-1,-4.6423241E-1,1.1986721E-1,1.1412355E-2,6.852186E-1,-2.9368487E-1,-7.588573E-1,1.743445E-2,-6.9395727E-3,-1.6569808E-2,1.6748786E-1,4.424824E-1,8.48834E-1,-5.479484E-1,-1.1193895E-1,-2.1450445E-2,-4.210354E-2,1.8626856E-2,3.0872144E-2,2.4709769E-2,9.260484E-3,2.9924264E-2,5.0360303E-2,-3.0574966E-2,-1.0358591E-2,-1.6161716E-1,5.213218E-3,-2.5660654E-3,6.836075E-3,-2.0309184E-1,1.6357923E-4,-1.3456873E-2,-4.259069E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,31,-1,-1,-1,33,-1,-1,-1],"loss_changes":[2.3161043E1,3.0224304E0,6.3458223E0,2.7417402E0,7.682852E-1,1.2886481E0,1.2924519E0,1.7409062E0,4.0243816E-1,0E0,0E0,0E0,4.744146E-1,2.5353503E-1,4.3628693E-1,3.4988117E-1,2.812557E-1,0E0,0E0,0E0,1.1217596E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3759726E-1,0E0,0E0,0E0,1.1828154E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,20,20,27,27,31,31],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,32,-1,-1,-1,34,-1,-1,-1],"split_conditions":[1.5363757E8,2.532835E3,6.9015497E-1,4.851587E7,2.5927516E11,3.0962205E-2,7.3845965E8,2.316E3,3.181762E5,1.743445E-2,-6.9395727E-3,-1.6569808E-2,8E0,5.5E1,2.5172096E-1,4.3012936E2,5.1177016E5,-2.1450445E-2,-4.210354E-2,1.8626856E-2,4.6947882E-1,2.4709769E-2,9.260484E-3,2.9924264E-2,5.0360303E-2,-3.0574966E-2,-1.0358591E-2,8E0,5.213218E-3,-2.5660654E-3,6.836075E-3,1.5151923E8,1.6357923E-4,-1.3456873E-2,-4.259069E-3],"split_indices":[12,60,47,53,39,46,7,2,36,0,0,0,8,3,46,60,36,0,0,0,47,0,0,0,0,0,0,8,0,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.3E2,6.8E1,6.2E1,5.8E1,1E1,2.1E1,4.1E1,3.8E1,2E1,5E0,5E0,6E0,1.5E1,1.8E1,2.3E1,1.5E1,2.3E1,7E0,1.3E1,5E0,1E1,1.3E1,5E0,1.3E1,1E1,1.1E1,4E0,1.9E1,4E0,6E0,4E0,1.5E1,4E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-4.0200684E-2,-3.8164067E-1,4.5016986E-1,-5.002557E-1,2.8355356E-2,1.21841684E-1,8.1302375E-1,-6.962294E-1,-2.2474179E-1,1.7348371E-2,-1.2839934E-1,-1.7188032E-1,2.4192056E-1,2.2311008E-2,4.528747E-2,-1.5838431E-2,-7.625096E-1,-3.0716202E-1,4.8323194E-3,-1.2054547E-2,-9.235028E-4,-7.656168E-4,-1.4009614E-2,3.9499238E-2,4.2234913E-1,-1.9378554E-2,-4.117109E-2,-2.1993054E-2,-1.7736217E-1,-3.6207407E-3,7.079065E-3,2.5583802E-2,8.5568065E-3,-1.1260517E-2,-4.46996E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,-1,25,27,-1,-1,-1,-1,-1,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1603115E1,3.7101927E0,6.215103E0,3.0291433E0,9.916178E-1,1.0644058E0,7.265301E-1,5.687103E-1,7.2951925E-1,0E0,1.709488E-1,1.656794E-1,7.5029194E-1,0E0,0E0,0E0,6.449299E-1,3.3634794E-1,0E0,0E0,0E0,0E0,0E0,1.4916182E-1,2.0724511E-1,0E0,0E0,0E0,4.155284E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,16,16,17,17,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,-1,26,28,-1,-1,-1,-1,-1,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.2480776E0,2.667836E6,6.15786E5,6.919127E5,6.084362E8,2.3452184E5,2.2781514E8,1.7348371E-2,1.309E4,7.6599895E9,1.3137E4,2.2311008E-2,4.528747E-2,-1.5838431E-2,4.5E1,2.6326587E-2,4.8323194E-3,-1.2054547E-2,-9.235028E-4,-7.656168E-4,-1.4009614E-2,4.6947882E-1,6.19999E9,-1.9378554E-2,-4.117109E-2,-2.1993054E-2,1.2103737E5,-3.6207407E-3,7.079065E-3,2.5583802E-2,8.5568065E-3,-1.1260517E-2,-4.46996E-3],"split_indices":[58,2,47,40,37,40,7,36,7,0,9,5,2,0,0,0,10,46,0,0,0,0,0,47,5,0,0,0,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.27E2,7.5E1,5.2E1,5.8E1,1.7E1,2.8E1,2.4E1,3.3E1,2.5E1,5E0,1.2E1,8E0,2E1,8E0,1.6E1,6E0,2.7E1,2E1,5E0,5E0,7E0,4E0,4E0,1E1,1E1,7E0,2E1,8E0,1.2E1,5E0,5E0,6E0,4E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.2837348E-2,-4.149507E-1,4.2947698E-1,-2.7040687E-1,-6.686808E-1,1.6791183E-1,7.550054E-1,-3.5688868E-1,1.151747E-1,-1.4067225E-2,-3.710819E-2,-1.2710556E-2,2.704308E-1,2.7015625E-2,9.2873096E-1,-5.7477045E-1,-1.4295064E-1,-3.7284775E-3,1.3629162E-2,4.1867065E-1,9.228592E-2,5.191283E-2,2.3387283E-2,-3.119763E-2,-1.1463833E-2,-2.149026E-1,2.9014356E-3,1.24234995E-2,2.9280137E-2,-7.3589883E-3,1.0810826E-2,-5.0549787E-3,-1.1697376E-2,-3.0172453E-3,2.3846498E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,-1,-1,27,29,-1,-1,-1,-1,31,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[2.2525385E1,2.3130913E0,4.8968916E0,1.5416648E0,7.439203E-1,1.5849668E0,3.29731E-1,1.6425247E0,3.1831688E-1,0E0,0E0,0E0,7.007911E-1,0E0,3.0158424E-1,3.160901E-1,3.061957E-1,0E0,0E0,3.0286932E-1,1.8872023E-1,0E0,0E0,0E0,0E0,2.3203254E-2,0E0,0E0,0E0,3.1937357E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,15,15,16,16,19,19,20,20,25,25,29,29],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,-1,-1,28,30,-1,-1,-1,-1,32,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,5.709511E7,1.3536367E0,5.1299828E1,5.3323513E-1,4.739085E-2,2.463E4,2.667836E6,7.441E3,-1.4067225E-2,-3.710819E-2,-1.2710556E-2,9.48E2,2.7015625E-2,5.8639234E11,8E0,2.3514317E8,-3.7284775E-3,1.3629162E-2,1.5301518E9,6.9015497E-1,5.191283E-2,2.3387283E-2,-3.119763E-2,-1.1463833E-2,2.5855421E1,2.9014356E-3,1.24234995E-2,2.9280137E-2,1.447E3,1.0810826E-2,-5.0549787E-3,-1.1697376E-2,-3.0172453E-3,2.3846498E-3],"split_indices":[12,53,47,64,35,46,2,40,2,0,0,0,0,0,39,8,7,0,0,12,47,0,0,0,0,66,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,6.7E1,5.8E1,4.4E1,2.3E1,3.3E1,2.5E1,3.6E1,8E0,6E0,1.7E1,6E0,2.7E1,1.4E1,1.1E1,1.7E1,1.9E1,4E0,4E0,1.4E1,1.3E1,7E0,4E0,1.3E1,4E0,1.4E1,5E0,9E0,5E0,8E0,5E0,4E0,1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.9596804E-2,-3.0500537E-1,4.8138127E-1,-4.61424E-1,-2.1424586E-2,2.5968468E-1,6.779679E-1,-6.492738E-1,-1.7536302E-1,-1.651935E-1,2.4322504E-2,3.367569E-1,-7.0099644E-3,1.7377641E-2,3.750871E-2,-3.7882408E-1,-3.5269022E-2,-2.3710065E-1,2.735032E-3,-7.6573814E-4,-1.44298645E-2,1.9350494E-3,4.1679424E-1,-2.2495339E-2,-1.0069255E-2,-1.5064612E-2,-5.698666E-3,2.6556847E-3,-3.798411E-3,1.2499437E-2,2.2565825E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,-1,-1,23,-1,25,-1,27,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9193998E1,3.2281976E0,2.1774025E0,2.4117565E0,2.1442745E0,8.952731E-1,6.59111E-1,3.9600182E-1,3.008229E-1,4.9520677E-1,0E0,5.284445E-1,0E0,0E0,0E0,4.0096402E-2,0E0,1.0998815E-1,0E0,5.326992E-2,0E0,0E0,3.1852722E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,15,15,17,17,19,19,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,-1,-1,24,-1,26,-1,28,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8095805E5,5.1299828E1,1.3536367E0,2.9282052E6,1.1653E4,1.9354674E-1,4.1372877E8,4.4E1,1.0154962E3,5.070136E7,2.4322504E-2,4.2163028E5,-7.0099644E-3,1.7377641E-2,3.750871E-2,1.21E2,-3.5269022E-2,2.207872E-1,2.735032E-3,3.6948596E3,-1.44298645E-2,1.9350494E-3,4.798087E6,-2.2495339E-2,-1.0069255E-2,-1.5064612E-2,-5.698666E-3,2.6556847E-3,-3.798411E-3,1.2499437E-2,2.2565825E-2],"split_indices":[41,64,47,40,2,46,7,10,60,53,0,40,0,0,0,0,0,47,0,4,0,0,1,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,7.2E1,5.3E1,4.6E1,2.6E1,2.6E1,2.7E1,2.7E1,1.9E1,2.1E1,5E0,2.2E1,4E0,8E0,1.9E1,8E0,1.9E1,1.5E1,4E0,1E1,1.1E1,5E0,1.7E1,4E0,4E0,8E0,7E0,6E0,4E0,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-2.012178E-3,-2.8318292E-1,4.37359E-1,-4.0165535E-1,4.0586926E-2,-1.6121443E-1,5.6110406E-1,-2.5354785E-1,-5.966941E-1,2.8911084E-1,-1.3459016E-1,-1.7226646E-2,3.368282E-3,2.849762E-1,7.333153E-1,-2.2175409E-2,-1.4171039E-1,-1.4126659E-2,-3.2736856E-2,2.3790149E-3,2.2473622E-2,-9.755964E-3,8.489729E-4,3.1795574E-3,3.4705427E-1,2.691279E-2,4.7872785E-2,-2.0476798E-2,-2.0500319E-1,1.913214E-2,8.655108E-3,2.768156E-3,-4.528389E-3,-5.910418E-3,-1.24464035E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,27,-1,-1,-1,-1,-1,-1,-1,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5195528E1,2.8931365E0,3.6458416E0,1.43748E0,9.5316106E-1,4.415409E-1,1.6931152E0,7.1215487E-1,4.1174126E-1,3.6739028E-1,1.4167951E-1,0E0,0E0,2.0737517E-1,5.3163433E-1,0E0,1.7228612E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2856574E-2,0E0,0E0,5.798103E-2,3.552264E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,24,24,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,28,-1,-1,-1,-1,-1,-1,-1,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1],"split_conditions":[5.093857E-1,2.532835E3,1.1467087E3,4.851587E7,6.15786E5,3.727E3,1.5083523E0,4.3012936E2,5.3323513E-1,1.0269E4,1.7802E4,-1.7226646E-2,3.368282E-3,7.1004186E0,1.5605432E9,-2.2175409E-2,4.828443E-1,-1.4126659E-2,-3.2736856E-2,2.3790149E-3,2.2473622E-2,-9.755964E-3,8.489729E-4,3.1795574E-3,1.8634315E1,2.691279E-2,4.7872785E-2,1.0154509E5,3.25E2,1.913214E-2,8.655108E-3,2.768156E-3,-4.528389E-3,-5.910418E-3,-1.24464035E-2],"split_indices":[47,60,60,53,37,2,50,60,35,2,10,0,0,61,7,0,35,0,0,0,0,0,0,0,62,0,0,41,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,7.4E1,4.7E1,5.4E1,2E1,8E0,3.9E1,3.2E1,2.2E1,8E0,1.2E1,4E0,4E0,1.6E1,2.3E1,1E1,2.2E1,6E0,1.6E1,4E0,4E0,8E0,4E0,4E0,1.2E1,1.6E1,7E0,8E0,1.4E1,8E0,4E0,4E0,4E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-4.6324845E-2,-3.5229686E-1,3.5654673E-1,-4.453095E-1,-1.6577244E-4,1.459914E-1,6.065275E-1,-5.976683E-1,-2.2059505E-1,1.510882E-2,-1.9007052E-1,-8.314282E-3,2.0416926E-1,3.174942E-2,1.578744E-2,-1.1607024E-2,-6.412036E-1,-3.3240518E-1,-4.200136E-2,-1.5758159E-2,7.5545494E-4,2.67206E-1,-4.8434515E-3,-1.9158855E-2,-3.4440022E-2,-1.973766E-2,-7.434292E-3,4.155909E-3,-6.8063443E-3,1.5372708E-1,1.9171052E-2,1.0298128E-2,5.5132154E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,-1,-1,-1,23,25,27,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,31,-1,-1,-1],"loss_changes":[1.4547899E1,2.1943521E0,2.5958266E0,1.6827307E0,9.6180665E-1,5.755833E-1,1.5015984E-1,3.2275867E-1,4.4531596E-1,0E0,2.9325056E-1,0E0,5.0868225E-1,0E0,0E0,0E0,2.2882843E-1,1.3580775E-1,1.4209636E-1,0E0,0E0,2.5815392E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1194283E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,16,16,17,17,18,18,21,21,29,29],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,-1,-1,-1,24,26,28,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,32,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,2.667836E6,6.15786E5,3.0962205E-2,2.326E4,2.3452184E5,1.6777216E8,1.510882E-2,2.4502628E6,-8.314282E-3,2.68442E9,3.174942E-2,1.578744E-2,-1.1607024E-2,4.4E1,2.5413216E5,7.84E3,-1.5758159E-2,7.5545494E-4,1.3137E4,-4.8434515E-3,-1.9158855E-2,-3.4440022E-2,-1.973766E-2,-7.434292E-3,4.155909E-3,-6.8063443E-3,1.6E1,1.9171052E-2,1.0298128E-2,5.5132154E-4],"split_indices":[12,2,47,40,37,46,9,36,7,0,40,0,7,0,0,0,10,55,9,0,0,2,0,0,0,0,0,0,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.16E2,6.6E1,5E1,5.2E1,1.4E1,2.8E1,2.2E1,3E1,2.2E1,5E0,9E0,4E0,2.4E1,1.7E1,5E0,4E0,2.6E1,1.3E1,9E0,5E0,4E0,2E1,4E0,8E0,1.8E1,8E0,5E0,4E0,5E0,1.2E1,8E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-3.069784E-2,-3.1872278E-1,2.822563E-1,-3.7838045E-1,2.2011777E-2,-1.3567133E-2,3.6395815E-1,-5.0075513E-1,-1.8702564E-1,2.2649586E-1,5.5609316E-1,-5.6470615E-1,-7.409139E-3,-2.9038548E-1,-2.9473305E-3,2.8574333E-1,-1.2379845E-3,2.9431676E-2,1.4465905E-2,-3.208469E-1,-6.387983E-1,-1.8879192E-2,-7.872527E-3,5.785286E-3,-5.0556636E-3,2.142102E-1,2.5948219E-2,-1.8801354E-2,-8.779387E-3,-3.5991564E-2,-4.2562148E-1,1.3231139E-1,1.5177469E-2,-1.0255568E-2,-2.5330922E-2,9.089283E-3,1.1098818E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,-1,21,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,31,-1,-1,-1,-1,33,35,-1,-1,-1,-1,-1],"loss_changes":[1.1086871E1,3.151122E0,2.8334055E0,1.3113651E0,0E0,0E0,1.2474356E0,7.0807076E-1,4.7480172E-1,4.859724E-1,1.3334131E-1,3.3708572E-1,0E0,1.4560616E-1,1.4052954E-1,3.627057E-1,0E0,0E0,0E0,1.7448306E-2,2.1783829E-1,0E0,0E0,0E0,0E0,1.4460886E-1,0E0,0E0,0E0,0E0,1.0707915E-1,8.4162146E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,19,19,20,20,25,25,30,30,31,31],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,-1,22,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,32,-1,-1,-1,-1,34,36,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,1.0269E4,3.0962205E-2,2.9282052E6,2.2011777E-2,-1.3567133E-2,1.3536367E0,3.159013E3,1.6777216E8,2.3152558E9,2.0837E4,4.4E1,-7.409139E-3,1.7301265E6,1.6920574E7,1.62739E9,-1.2379845E-3,2.9431676E-2,1.4465905E-2,1.16E2,1.365445E1,-1.8879192E-2,-7.872527E-3,5.785286E-3,-5.0556636E-3,4.451217E6,2.5948219E-2,-1.8801354E-2,-8.779387E-3,-3.5991564E-2,1.325631E1,1.6E1,1.5177469E-2,-1.0255568E-2,-2.5330922E-2,9.089283E-3,1.1098818E-3],"split_indices":[12,2,46,40,0,0,47,4,7,7,9,10,0,56,40,7,0,0,0,0,62,0,0,0,0,1,0,0,0,0,61,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,6.3E1,5.8E1,5.9E1,4E0,7E0,5.1E1,3.5E1,2.4E1,3.1E1,2E1,2.9E1,6E0,1.5E1,9E0,2.5E1,6E0,1.5E1,5E0,8E0,2.1E1,7E0,8E0,4E0,5E0,2.1E1,4E0,4E0,4E0,1.2E1,9E0,1.3E1,8E0,4E0,5E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.7212866E-2,-3.5431314E-1,3.1184202E-1,-4.4205543E-1,6.874925E-2,1.1574053E-1,5.96699E-1,-5.3439045E-1,-1.1282333E-1,1.779095E-2,-7.5484347E-3,-1.5010878E-2,1.9789441E-1,1.6368346E-2,7.0929235E-1,-3.234037E-2,-4.1181847E-1,-8.611278E-4,-9.771522E-3,-4.709431E-2,2.8912187E-1,3.8335577E-2,1.7282417E-2,-4.9294913E-1,-9.459667E-3,2.3161566E-3,-6.3644927E-3,4.158547E-1,9.5457E-2,-3.3647146E-2,-3.6370242E-1,1.1825533E-2,2.726242E-2,-1.3711009E-3,8.740459E-3,-1.0498324E-2,-2.0777343E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,-1,23,-1,-1,25,27,-1,-1,29,-1,-1,-1,31,33,-1,35,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2864558E1,2.2204986E0,3.1150074E0,1.4292908E0,8.167896E-1,1.2987887E0,4.1454983E-1,3.862362E-1,1.0071799E-1,0E0,0E0,0E0,6.8410707E-1,0E0,1.9639349E-1,0E0,3.0119395E-1,0E0,0E0,8.0602616E-2,5.00265E-1,0E0,0E0,1.8133521E-1,0E0,0E0,0E0,1.9764733E-1,1.13978945E-1,0E0,2.7567029E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,16,16,19,19,20,20,23,23,27,27,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,-1,24,-1,-1,26,28,-1,-1,30,-1,-1,-1,32,34,-1,36,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,7.388E3,1.3536367E0,7.355655E6,6.15786E5,3.030377E-2,3.1E1,2.0943393E-4,1.0783582E1,1.779095E-2,-7.5484347E-3,-1.5010878E-2,8.184232E5,1.6368346E-2,5.8639234E11,-3.234037E-2,1.04856936E8,-8.611278E-4,-9.771522E-3,6.169951E0,1.0494656E1,3.8335577E-2,1.7282417E-2,5.279518E7,-9.459667E-3,2.3161566E-3,-6.3644927E-3,1.5791555E9,7.278853E-2,-3.3647146E-2,8.317631E-1,1.1825533E-2,2.726242E-2,-1.3711009E-3,8.740459E-3,-1.0498324E-2,-2.0777343E-2],"split_indices":[12,2,47,40,37,46,3,50,61,0,0,0,40,0,39,0,7,0,0,61,61,0,0,5,0,0,0,7,46,0,35,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,5.8E1,5.6E1,4.8E1,1E1,3.4E1,2.2E1,3.7E1,1.1E1,4E0,6E0,5E0,2.9E1,8E0,1.4E1,1.5E1,2.2E1,6E0,5E0,8E0,2.1E1,1E1,4E0,1.5E1,7E0,4E0,4E0,1.2E1,9E0,4E0,1.1E1,7E0,5E0,4E0,5E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.7450418E-3,-3.247758E-1,3.479195E-1,-3.8534567E-1,1.2183964E-1,-6.1712086E-2,5.0064665E-1,-4.6790937E-1,-1.4412728E-1,-3.8281013E-3,1.4301718E-2,-1.782348E-1,5.072763E-3,3.3031705E-1,6.5661466E-1,-2.0566784E-1,-5.2775675E-1,-1.9554257E-1,8.776719E-4,-1.3667187E-2,-3.6136152E-3,6.768417E-3,3.609488E-1,2.0660885E-2,4.3197792E-2,-1.4677508E-3,-1.5146951E-2,-3.920547E-1,-6.41678E-1,-5.3751967E-3,-1.1143657E-2,4.0947682E-1,1.0140147E-2,-4.6182323E-1,-4.9828663E-3,-3.5532475E-2,-1.7887363E-2,1.1151733E-2,2.180158E-2,-1.2565535E-2,-2.4363691E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,29,-1,-1,-1,-1,31,-1,-1,-1,-1,33,35,-1,-1,37,-1,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4902257E1,1.9134998E0,3.9722934E0,1.1474304E0,3.469326E-1,3.7115586E-1,1.0051432E0,5.977087E-1,1.4563298E-1,0E0,0E0,1.0013735E-1,0E0,8.748841E-2,8.858967E-1,1.853042E-1,3.0496407E-1,1.4516443E-2,0E0,0E0,0E0,0E0,5.7301044E-2,0E0,0E0,0E0,0E0,3.3321166E-1,2.1345663E-1,0E0,0E0,9.272337E-3,0E0,9.846449E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,22,22,27,27,28,28,31,31,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,30,-1,-1,-1,-1,32,-1,-1,-1,-1,34,36,-1,-1,38,-1,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7761682E8,2.532835E3,5.9176934E-1,1.977643E6,6E0,4.7308203E3,1.6957201E0,4.029937E-1,2.924E4,-3.8281013E-3,1.4301718E-2,9.257E3,5.072763E-3,1.2723797E5,2.463E4,1.0526E4,7.7E2,1.4101E4,8.776719E-4,-1.3667187E-2,-3.6136152E-3,6.768417E-3,1.5625807E1,2.0660885E-2,4.3197792E-2,-1.4677508E-3,-1.5146951E-2,4.831714E-3,1.4710438E1,-5.3751967E-3,-1.1143657E-2,7.401E3,1.0140147E-2,4.4815472E5,-4.9828663E-3,-3.5532475E-2,-1.7887363E-2,1.1151733E-2,2.180158E-2,-1.2565535E-2,-2.4363691E-2],"split_indices":[12,60,49,56,8,4,47,35,9,0,0,9,0,36,2,9,11,9,0,0,0,0,62,0,0,0,0,47,62,0,0,2,0,36,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.3E2,6.8E1,6.2E1,6E1,8E0,1.7E1,4.5E1,4.4E1,1.6E1,4E0,4E0,1E1,7E0,2.3E1,2.2E1,9E0,3.5E1,1.2E1,4E0,4E0,6E0,4E0,1.9E1,1.3E1,9E0,4E0,5E0,1.8E1,1.7E1,5E0,7E0,1.3E1,6E0,1.4E1,4E0,1.1E1,6E0,4E0,9E0,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.581221E-3,-3.1352073E-1,2.798847E-1,-3.5900736E-1,1.1482605E-2,9.417447E-2,5.3518915E-1,-4.4234923E-1,-1.3342364E-1,-1.0582473E-2,1.6786984E-1,3.6862212E-1,3.8117006E-2,-4.7812784E-1,-1.0052143E-2,9.221997E-4,-9.771224E-3,2.6249617E-1,-1.9600937E-2,9.710546E-3,1.972979E-2,-3.7252796E-1,-6.3188356E-1,3.480947E-1,1.24968484E-1,4.917099E-3,-5.819363E-3,-4.3807447E-1,-9.697612E-3,-3.6103655E-2,-1.4658383E-2,2.1396425E-2,7.4898885E-3,2.9470741E-3,8.399711E-3,-2.3400288E-2,-1.1282523E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,-1,-1,-1,23,25,-1,-1,27,29,31,33,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0946998E1,1.5477891E0,3.126247E0,9.562106E-1,0E0,9.6801573E-1,9.3218994E-1,2.2289467E-1,1.7789665E-1,0E0,5.9055436E-1,5.5773973E-2,0E0,3.0041313E-1,0E0,0E0,0E0,2.1548676E-1,1.6293445E-1,0E0,0E0,1.554172E-1,2.5188923E-1,1.7820847E-1,2.1355152E-2,0E0,0E0,4.1402817E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,17,17,18,18,21,21,22,22,23,23,24,24,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,-1,-1,-1,24,26,-1,-1,28,30,32,34,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4350427E8,3.373559E3,1.3536367E0,7.355655E6,1.1482605E-2,4.739085E-2,1.0861106E4,3.2316522E7,1.0865672E1,-1.0582473E-2,5E1,1.601247E1,3.8117006E-2,2.56E2,-1.0052143E-2,9.221997E-4,-9.771224E-3,1E0,1.308E3,9.710546E-3,1.972979E-2,2.7427354E-4,1.365445E1,1.4273524E-1,1.6270023E1,4.917099E-3,-5.819363E-3,6.1E1,-9.697612E-3,-3.6103655E-2,-1.4658383E-2,2.1396425E-2,7.4898885E-3,2.9470741E-3,8.399711E-3,-2.3400288E-2,-1.1282523E-2],"split_indices":[12,60,47,40,0,46,4,12,61,0,3,64,0,10,0,0,0,24,0,0,0,46,62,46,62,0,0,3,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,5.7E1,6.6E1,5.3E1,4E0,3.9E1,2.7E1,3.8E1,1.5E1,7E0,3.2E1,1.8E1,9E0,3.2E1,6E0,5E0,1E1,2.1E1,1.1E1,5E0,1.3E1,2.1E1,1.1E1,1.2E1,9E0,5E0,6E0,1.4E1,7E0,7E0,4E0,7E0,5E0,5E0,4E0,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.1264033E-2,-2.4747981E-1,3.7034374E-1,-3.3529854E-1,2.0929104E-1,8.328023E-2,5.1127297E-1,-2.237452E-1,-5.0467074E-1,5.45943E-2,2.2771718E-2,-2.9058668E-3,1.9025409E-1,3.5307232E-1,7.1532995E-1,-1.7958244E-2,-1.6789423E-1,-5.558915E-1,-1.1115376E-2,7.5221774E-3,-1.9230621E-3,1.3137196E-2,2.407233E-3,3.9272994E-1,7.428741E-3,1.683272E-2,3.999345E-2,-2.1060549E-1,-2.585127E-2,-1.3858868E-2,-2.8597407E-2,2.3053376E-2,1.1576034E-2,-1.1423369E-2,-4.71954E-3,-3.37887E-3,1.1566351E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,27,29,-1,-1,-1,-1,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1608074E1,3.399557E0,1.8992915E0,1.2096639E0,5.533838E-1,2.7368003E-1,7.804594E-1,3.1110954E-1,2.4667358E-1,1.0397051E-1,0E0,0E0,1.04768604E-1,9.5571995E-2,2.8084517E-1,0E0,2.0082784E-1,7.7114105E-2,0E0,0E0,0E0,0E0,0E0,9.945798E-2,0E0,0E0,0E0,5.9461713E-2,2.1947172E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,13,13,14,14,16,16,17,17,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,28,30,-1,-1,-1,-1,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,3.7254233E3,4.034396E8,8.0073607E-1,6.160098E3,3.22115E5,2.6281825E-1,1.1406926E1,2.7240606E5,1.470899E6,2.2771718E-2,-2.9058668E-3,4.5E1,7.6834904E7,3.840785E1,-1.7958244E-2,3.566468E5,2.07E2,-1.1115376E-2,7.5221774E-3,-1.9230621E-3,1.3137196E-2,2.407233E-3,1.2320755E1,7.428741E-3,1.683272E-2,3.999345E-2,7.388E3,9.269114E0,-1.3858868E-2,-2.8597407E-2,2.3053376E-2,1.1576034E-2,-1.1423369E-2,-4.71954E-3,-3.37887E-3,1.1566351E-3],"split_indices":[50,60,7,35,60,1,46,66,55,37,0,0,3,53,64,0,36,11,0,0,0,0,0,62,0,0,0,2,61,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.29E2,8.2E1,4.7E1,6.9E1,1.3E1,1.6E1,3.1E1,4.3E1,2.6E1,9E0,4E0,7E0,9E0,1.9E1,1.2E1,1E1,3.3E1,2.1E1,5E0,4E0,5E0,5E0,4E0,1.5E1,4E0,4E0,8E0,2.5E1,8E0,4E0,1.7E1,8E0,7E0,1.9E1,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.1297039E-2,-2.4140024E-1,3.823768E-1,-3.041635E-1,6.684552E-2,1.2813921E-1,5.391226E-1,-4.632674E-1,-1.2961736E-1,1.3227999E-2,-4.733462E-3,-4.2266445E-3,2.5160092E-1,3.6792758E-1,7.346909E-1,-3.0800638E-1,-5.582889E-1,-2.0469338E-1,1.8667882E-2,2.904528E-3,1.734706E-2,2.0441385E-2,1.1427979E-2,4.1393626E-2,2.0829283E-2,-3.7325227E-3,-1.8726913E-2,-3.0037874E-2,-1.1877556E-2,-1.5123759E-2,-6.103592E-2,4.677518E-3,-3.0430662E-3,-6.3578784E-3,4.400943E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,-1],"loss_changes":[1.1344808E1,1.41328E0,1.8835998E0,1.6062107E0,4.8083255E-1,5.57209E-1,7.340975E-1,2.8539038E-1,3.3780116E-1,0E0,0E0,0E0,2.5478446E-1,3.6735535E-2,1.3451242E-1,2.4005485E-1,2.3746395E-1,3.0243272E-1,7.808675E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.1873975E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,17,17,18,18,30,30],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,-1],"split_conditions":[9.095874E6,2.532835E3,1.3536367E0,2.667836E6,6.15786E5,6.919127E5,1.5605432E9,7.48E2,2.0237808E8,1.3227999E-2,-4.733462E-3,-4.2266445E-3,3.486353E6,2.326E4,3.0376984E1,5.9775025E-1,3.974E3,2.6326587E-2,1.1356572E5,2.904528E-3,1.734706E-2,2.0441385E-2,1.1427979E-2,4.1393626E-2,2.0829283E-2,-3.7325227E-3,-1.8726913E-2,-3.0037874E-2,-1.1877556E-2,-1.5123759E-2,2.68E2,4.677518E-3,-3.0430662E-3,-6.3578784E-3,4.400943E-4],"split_indices":[56,60,47,40,37,40,7,11,7,0,0,0,1,9,65,35,10,46,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,7.1E1,4.8E1,5.9E1,1.2E1,1.9E1,2.9E1,3E1,2.9E1,5E0,7E0,7E0,1.2E1,1.7E1,1.2E1,1.3E1,1.7E1,1.9E1,1E1,5E0,7E0,1E1,7E0,7E0,5E0,4E0,9E0,1.3E1,4E0,1E1,9E0,5E0,5E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.756298E-3,-2.3988245E-1,3.541553E-1,-3.2709724E-1,2.0725606E-2,1.8904401E-1,5.664268E-1,-4.7040287E-1,-1.4651927E-1,2.9959357E-1,-1.7393112E-1,-2.0688921E-2,2.685559E-1,3.4693784E-1,3.793245E-2,-5.07735E-1,-7.6313335E-3,-1.9129723E-1,4.6667415E-3,1.6270757E-3,2.412668E-2,-1.5249654E-2,-2.0606604E-3,-7.7670077E-3,4.8257774E-3,4.7804634E-3,3.1914672E-1,1.9706909E-2,1.0615129E-2,-1.0249746E-2,-5.460479E-1,-2.2995137E-1,7.539574E-5,1.8977562E-2,8.199091E-3,-5.9879327E-1,-1.715019E-2,-6.421046E-3,-1.2943588E-2,-1.7396946E-2,-3.156576E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,-1,-1,-1,-1,-1,-1,-1,-1,33,-1,-1,-1,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1],"loss_changes":[1.1624289E1,1.825089E0,1.8556318E0,1.4837465E0,1.1920626E0,5.558727E-1,9.247203E-1,3.0152225E-1,3.2491487E-1,4.7412437E-1,2.3340502E-1,1.8923825E-1,1.7258894E-1,2.2966385E-2,0E0,2.0847416E-1,0E0,1.7929405E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3943005E-1,0E0,0E0,0E0,3.1460762E-2,4.2020082E-2,0E0,0E0,0E0,2.1674633E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,26,26,30,30,31,31,35,35],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,-1,-1,-1,-1,-1,-1,-1,-1,34,-1,-1,-1,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1],"split_conditions":[1.0397225E7,7.388E3,1.7915075E0,2.667836E6,6.15786E5,8.184232E5,2.463E4,3.0387878E3,2.3514317E8,1.0269E4,1.4260634E8,6.758564E6,2.7113557E3,1.5567E4,3.793245E-2,1.8089891E2,-7.6313335E-3,7.307441E-1,4.6667415E-3,1.6270757E-3,2.412668E-2,-1.5249654E-2,-2.0606604E-3,-7.7670077E-3,4.8257774E-3,4.7804634E-3,1.6135693E1,1.9706909E-2,1.0615129E-2,-1.0249746E-2,1.365445E1,1.087337E1,7.539574E-5,1.8977562E-2,8.199091E-3,3.9E1,-1.715019E-2,-6.421046E-3,-1.2943588E-2,-1.7396946E-2,-3.156576E-2],"split_indices":[58,2,50,40,37,40,2,4,7,2,7,1,63,9,0,60,0,47,0,0,0,0,0,0,0,0,62,0,0,0,62,61,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.34E2,7.9E1,5.5E1,5.9E1,2E1,3.2E1,2.3E1,3.2E1,2.7E1,8E0,1.2E1,9E0,2.3E1,1.3E1,1E1,2.8E1,4E0,2.3E1,4E0,4E0,4E0,5E0,7E0,4E0,5E0,6E0,1.7E1,7E0,6E0,4E0,2.4E1,1.9E1,4E0,1E1,7E0,1.7E1,7E0,7E0,1.2E1,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.9869895E-2,-2.095309E-1,2.5363615E-1,-2.4425489E-1,1.5211087E-2,7.4997656E-2,4.9104825E-1,-4.118143E-1,-1.6403934E-1,-7.266686E-3,1.3099544E-1,1.3584653E-2,2.6708977E-2,-4.5899478E-1,-7.622074E-3,-1.9738102E-2,-1.2632051E-1,-5.2136327E-3,1.8730247E-1,-1.1187677E-2,-2.4187438E-2,-6.8241306E-2,-2.5956774E-1,2.6752395E-3,2.4358627E-1,-1.4217074E-1,2.5023447E-2,-1.4239261E-2,-6.5992805E-3,1.3902868E-2,5.220403E-3,-7.4772496E-4,-8.462474E-3,4.8332247E-3,-3.7009157E-2,-3.8359426E-3,6.5454975E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,-1,21,-1,23,-1,-1,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,35,-1,-1],"loss_changes":[6.1653423E0,1.4346418E0,1.9564867E0,8.733072E-1,0E0,3.7610155E-1,1.2279749E-1,1.8853283E-1,4.0848827E-1,0E0,3.3147752E-1,0E0,0E0,7.340026E-2,0E0,0E0,3.22798E-1,0E0,1.267494E-1,0E0,0E0,2.2419795E-1,1.9698858E-2,0E0,5.1146686E-2,7.699415E-2,7.463148E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0870572E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,16,16,18,18,21,21,22,22,24,24,25,25,26,26,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,-1,22,-1,24,-1,-1,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,36,-1,-1],"split_conditions":[9.379017E6,1.1653E4,1.4695233E0,2.2372459E4,1.5211087E-2,2.199703E-1,5.403511E8,2.4948413E3,9.2433E4,-7.266686E-3,2.891331E5,1.3584653E-2,2.6708977E-2,1.433458E10,-7.622074E-3,-1.9738102E-2,5.9633E7,-5.2136327E-3,4.6772876E3,-1.1187677E-2,-2.4187438E-2,1.2239595E3,6.104948E1,2.6752395E-3,2.3872569E5,2.2333334E1,2.2232678E7,-1.4239261E-2,-6.5992805E-3,1.3902868E-2,5.220403E-3,-7.4772496E-4,-8.462474E-3,4.8332247E-3,9.030129E4,-3.8359426E-3,6.5454975E-4],"split_indices":[59,2,50,59,0,47,7,4,1,0,40,0,0,39,0,0,53,0,4,0,0,4,64,0,41,66,53,0,0,0,0,0,0,0,41,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,7.3E1,4.6E1,6.9E1,4E0,2.7E1,1.9E1,2.1E1,4.8E1,5E0,2.2E1,6E0,1.3E1,1.7E1,4E0,5E0,4.3E1,4E0,1.8E1,4E0,1.3E1,3.1E1,1.2E1,6E0,1.2E1,1.7E1,1.4E1,8E0,4E0,8E0,4E0,4E0,1.3E1,6E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.4717984E-3,-1.7210898E-1,2.7826092E-1,-2.4532215E-1,2.3880486E-1,2.0853232E-1,3.1846277E-2,-1.3115706E-1,-3.91499E-1,2.1151016E-2,3.8874627E-3,-2.2332852E-3,2.594722E-1,-1.4166744E-2,-9.70942E-2,-4.3842274E-1,-4.200944E-3,5.0761834E-2,3.172908E-1,3.6676931E-3,-1.3858163E-1,-4.8048672E-1,-8.714776E-3,-3.5115262E-3,7.8751305E-3,3.4509337E-1,5.748981E-3,-1.6689892E-1,-1.6157277E-3,-1.3366695E-2,-2.5248691E-2,1.210493E-2,2.055133E-2,-9.048E-3,-4.5231227E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,-1,17,-1,19,21,-1,23,25,-1,27,29,-1,-1,-1,31,-1,33,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0427823E0,2.3449917E0,1.1886077E0,1.0333915E0,3.4943295E-1,5.713382E-1,0E0,1.854012E-1,3.6023617E-1,0E0,0E0,0E0,4.1831803E-1,0E0,2.4871793E-1,1.7070055E-1,0E0,1.398029E-1,1.1174703E-1,0E0,7.4441254E-2,4.5422554E-2,0E0,0E0,0E0,7.836127E-2,0E0,1.625675E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,12,12,14,14,15,15,17,17,18,18,20,20,21,21,25,25,27,27],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,-1,18,-1,20,22,-1,24,26,-1,28,30,-1,-1,-1,32,-1,34,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.402099E-1,5.188755E3,9.7156115E8,7.9888564E-1,4.0342972E11,6.7918215E0,3.1846277E-2,8.844156E0,3.271557E8,2.1151016E-2,3.8874627E-3,-2.2332852E-3,1E0,-1.4166744E-2,2.01E2,1.365445E1,-4.200944E-3,1.0889989E0,1.8634315E1,3.6676931E-3,8.156E3,1.44E2,-8.714776E-3,-3.5115262E-3,7.8751305E-3,7.3845965E8,5.748981E-3,3.3320275E-1,-1.6157277E-3,-1.3366695E-2,-2.5248691E-2,1.210493E-2,2.055133E-2,-9.048E-3,-4.5231227E-3],"split_indices":[50,4,40,35,39,64,0,66,7,0,0,0,109,0,0,62,0,50,62,0,2,0,0,0,0,7,0,49,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,7.5E1,4.8E1,6.4E1,1.1E1,4.2E1,6E0,3.7E1,2.7E1,4E0,7E0,7E0,3.5E1,5E0,3.2E1,2.3E1,4E0,8E0,2.7E1,6E0,2.6E1,1.9E1,4E0,4E0,4E0,2.3E1,4E0,2E1,6E0,5E0,1.4E1,1.3E1,1E1,1.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.4564333E-2,-2.1947452E-1,2.1686216E-1,-2.5445196E-1,1.2996142E-2,1.1564947E-1,5.3496087E-1,-4.1877696E-1,-1.9095244E-1,-1.5592268E-1,1.9212218E-1,1.07871555E-2,3.090986E-2,-1.0875875E-2,-2.1970116E-2,-2.0345822E-2,-1.4722644E-1,7.835616E-4,-1.5833024E-2,1.0610515E-1,2.8225976E-1,-8.752353E-2,-2.3877344E-1,1.7907406E-1,2.45062E-4,1.6898358E-2,6.6705393E-3,-1.4296441E-1,3.9243165E-2,-8.1964955E-3,-1.4354869E-2,4.6544713E-3,1.2298166E-2,-9.905731E-3,-7.723112E-2,-8.917439E-4,4.265177E-3,-7.810982E-4,-5.2379947E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,-1,-1,-1,21,-1,-1,23,25,27,29,31,-1,-1,-1,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,-1],"loss_changes":[5.7884865E0,1.2385128E0,1.7088912E0,5.8784485E-1,0E0,9.2297405E-1,3.0492997E-1,2.924776E-2,4.3066823E-1,3.080709E-1,2.3366535E-1,0E0,0E0,0E0,0E0,0E0,2.0865792E-1,0E0,0E0,1.39072E-1,1.13877654E-1,1.9536789E-1,1.481241E-2,4.532227E-2,0E0,0E0,0E0,6.4481616E-2,2.7764734E-2,0E0,0E0,0E0,0E0,0E0,1.993516E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,16,16,19,19,20,20,21,21,22,22,23,23,27,27,28,28,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,-1,-1,-1,22,-1,-1,24,26,28,30,32,-1,-1,-1,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,-1],"split_conditions":[2.0020565E8,6.099784E3,1.8319664E0,2.0943393E-4,1.2996142E-2,6.919127E5,6.084362E8,5.189068E-1,1.477E3,6.169951E0,1.067E4,1.07871555E-2,3.090986E-2,-1.0875875E-2,-2.1970116E-2,-2.0345822E-2,3.925844E7,7.835616E-4,-1.5833024E-2,1.5E1,1.5625807E1,1.4861879E3,6.3997424E-1,2.9290114E7,2.45062E-4,1.6898358E-2,6.6705393E-3,2.207872E-1,3.9343938E-1,-8.1964955E-3,-1.4354869E-2,4.6544713E-3,1.2298166E-2,-9.905731E-3,2.4E1,-8.917439E-4,4.265177E-3,-7.810982E-4,-5.2379947E-3],"split_indices":[12,4,47,50,0,40,7,35,2,61,2,0,0,0,0,0,53,0,0,8,62,4,35,40,0,0,0,47,35,0,0,0,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,6.7E1,5.4E1,6.3E1,4E0,4.2E1,1.2E1,1.6E1,4.7E1,9E0,3.3E1,4E0,8E0,4E0,1.2E1,6E0,4.1E1,5E0,4E0,1.8E1,1.5E1,2.6E1,1.5E1,1E1,8E0,9E0,6E0,1.8E1,8E0,9E0,6E0,6E0,4E0,8E0,1E1,4E0,4E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[6.9164853E-3,-1.8898293E-1,2.484105E-1,-2.3060314E-1,8.749493E-3,5.942672E-2,4.4837844E-1,-2.2070087E-2,-1.5492803E-1,-8.922807E-3,1.01280235E-1,3.2407105E-1,5.764436E-1,-2.5225354E-2,-1.1281744E-1,-5.6280144E-2,1.6711839E-1,1.7330684E-2,9.189025E-3,3.3416606E-2,1.1425831E-2,-1.5669104E-1,3.1860963E-2,2.4975894E-3,-7.3355604E-3,6.497254E-2,1.359848E-2,-1.8890966E-1,-7.777134E-4,6.2452457E-3,-1.6209102E-3,6.6601764E-3,-1.5644393E-3,-2.0381187E-1,-3.5166345E-3,-1.0564648E-2,-4.129286E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,15,17,19,-1,21,23,25,-1,-1,-1,-1,27,29,-1,-1,31,-1,33,-1,-1,-1,-1,-1,35,-1,-1,-1],"loss_changes":[6.33898E0,1.1885617E0,2.2407959E0,1.1023211E0,0E0,3.4874883E-1,2.748556E-1,0E0,7.5779057E-1,0E0,2.9986307E-1,1.7068386E-2,3.7299252E-1,0E0,3.0967534E-1,1.03136666E-1,2.2179908E-1,0E0,0E0,0E0,0E0,1.6430026E-1,8.254068E-2,0E0,0E0,9.192546E-2,0E0,3.652692E-2,0E0,0E0,0E0,0E0,0E0,3.397739E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,10,10,11,11,12,12,14,14,15,15,16,16,21,21,22,22,25,25,27,27,33,33],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,16,18,20,-1,22,24,26,-1,-1,-1,-1,28,30,-1,-1,32,-1,34,-1,-1,-1,-1,-1,36,-1,-1,-1],"split_conditions":[9.379017E6,2.9853157E3,1.3536367E0,1.9483352E4,8.749493E-3,3.1570956E-2,1.5605432E9,-2.2070087E-2,1.477E3,-8.922807E-3,8.184232E5,2.326E4,7.9606894E11,-2.5225354E-2,2.3514317E8,1.381E3,4.451217E6,1.7330684E-2,9.189025E-3,3.3416606E-2,1.1425831E-2,5.5693356E5,2E0,2.4975894E-3,-7.3355604E-3,1.6E1,1.359848E-2,6.1626316E7,-7.777134E-4,6.2452457E-3,-1.6209102E-3,6.6601764E-3,-1.5644393E-3,3.970935E3,-3.5166345E-3,-1.0564648E-2,-4.129286E-3],"split_indices":[59,60,47,59,0,46,7,0,2,0,40,9,39,0,7,0,1,0,0,0,0,36,8,0,0,8,0,40,0,0,0,0,0,63,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.32E2,7.3E1,5.9E1,6.6E1,7E0,3.1E1,2.8E1,1.5E1,5.1E1,4E0,2.7E1,1.6E1,1.2E1,4E0,4.7E1,8E0,1.9E1,1.1E1,5E0,8E0,4E0,3.6E1,1.1E1,4E0,4E0,1.1E1,8E0,2.9E1,7E0,4E0,7E0,6E0,5E0,2.5E1,4E0,2.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4822811E-2,-2.159943E-1,1.6350642E-1,-2.6718292E-1,7.921417E-3,-9.290788E-3,2.1352297E-1,-3.7641695E-1,-1.4226484E-1,2.8162128E-1,4.874304E-2,-6.846225E-3,-4.4247013E-1,-1.970858E-1,2.6288065E-3,2.3540437E-1,2.5768254E-2,-4.280957E-2,7.5021656E-3,-2.3821833E-2,-1.3114962E-2,-1.34078E-2,-5.9641083E-3,2.744063E-1,2.185708E-3,-6.846079E-3,4.126395E-3,1.7356849E-1,3.2078463E-1,1.2639513E-2,3.9974772E-3,7.978527E-3,1.6847232E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,-1,19,21,-1,23,-1,25,-1,-1,-1,-1,-1,27,-1,-1,-1,29,31,-1,-1,-1,-1],"loss_changes":[4.125038E0,1.0998611E0,1.1339536E0,6.0142374E-1,0E0,0E0,5.922196E-1,3.2142997E-1,2.704507E-1,3.260064E-1,1.7535701E-1,0E0,4.2708397E-2,7.971227E-2,0E0,2.389121E-1,0E0,1.4230213E-1,0E0,0E0,0E0,0E0,0E0,7.526922E-2,0E0,0E0,0E0,6.789425E-2,2.9568672E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,17,17,23,23,27,27,28,28],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,-1,20,22,-1,24,-1,26,-1,-1,-1,-1,-1,28,-1,-1,-1,30,32,-1,-1,-1,-1],"split_conditions":[1.4112683E8,2.532835E3,2.4835809E-1,2.667836E6,7.921417E-3,-9.290788E-3,1.0692556E1,4.4E1,2.3514317E8,1.2965194E6,4.1822475E-1,-6.846225E-3,1.365445E1,2.3396162E5,2.6288065E-3,8.912541E-1,2.5768254E-2,1.4634906E-1,7.5021656E-3,-2.3821833E-2,-1.3114962E-2,-1.34078E-2,-5.9641083E-3,2.1263623E5,2.185708E-3,-6.846079E-3,4.126395E-3,2E0,3.8E1,1.2639513E-2,3.9974772E-3,7.978527E-3,1.6847232E-2],"split_indices":[12,60,49,40,0,0,61,10,7,36,35,0,62,55,0,35,0,46,0,0,0,0,0,41,0,0,0,8,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,5.3E1,6E1,4.7E1,6E0,7E0,5.3E1,2.4E1,2.3E1,3.7E1,1.6E1,6E0,1.8E1,1.8E1,5E0,3.3E1,4E0,9E0,7E0,1.2E1,6E0,7E0,1.1E1,2.7E1,6E0,5E0,4E0,1E1,1.7E1,4E0,6E0,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-5.296643E-3,-1.3952996E-1,3.2670507E-1,-2.217752E-1,1.0612055E-1,1.8217778E-1,5.683935E-1,-3.3692312E-1,-1.5392691E-1,-2.8276404E-3,1.6795509E-1,6.6556826E-2,2.7489114E-1,3.3203475E-2,1.4542167E-2,-3.8322484E-1,-4.0274053E-3,-1.9555008E-1,-3.9629318E-2,2.0497733E-1,1.819833E-3,-2.0845491E-6,7.63184E-3,1.5659012E-2,5.946402E-3,-1.0641793E-2,-2.1607524E-2,-9.65765E-2,-2.4279451E-1,4.0903096E-3,-1.0482494E-1,4.5080734E-3,1.1635282E-2,-7.295687E-3,6.038477E-4,-1.6929422E-2,-1.6162488E-1,-6.9525857E-3,-2.058409E-3,-2.7709007E-3,-9.42072E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,25,-1,27,29,31,-1,-1,-1,-1,-1,-1,-1,33,35,-1,37,-1,-1,-1,-1,-1,39,-1,-1,-1,-1],"loss_changes":[5.5717306E0,1.821654E0,1.1539521E0,4.715817E-1,2.4507543E-1,2.3897463E-1,2.2351646E-1,2.4837255E-1,2.015773E-1,0E0,7.391316E-2,7.4532434E-2,6.2781334E-2,0E0,0E0,1.2123871E-1,0E0,1.2618232E-1,1.15156606E-1,3.257537E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.557813E-2,1.3310075E-1,0E0,1.6366534E-2,0E0,0E0,0E0,0E0,0E0,4.0247887E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,15,15,17,17,18,18,19,19,27,27,28,28,30,30,36,36],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,26,-1,28,30,32,-1,-1,-1,-1,-1,-1,-1,34,36,-1,38,-1,-1,-1,-1,-1,40,-1,-1,-1,-1],"split_conditions":[1.1392E4,6.402099E-1,1.3127055E4,2.2372459E4,8.873786E0,7.90905E-2,6.5566176E7,3.0387878E3,2.3514317E8,-2.8276404E-3,9.6E3,6.083071E3,1.0494656E1,3.3203475E-2,1.4542167E-2,1.4688152E9,-4.0274053E-3,2.1212547E4,4.5843E7,1.0829101E0,1.819833E-3,-2.0845491E-6,7.63184E-3,1.5659012E-2,5.946402E-3,-1.0641793E-2,-2.1607524E-2,6.542E3,1.5418326E6,4.0903096E-3,3.9307986E3,4.5080734E-3,1.1635282E-2,-7.295687E-3,6.038477E-4,-1.6929422E-2,3.25E2,-6.9525857E-3,-2.058409E-3,-2.7709007E-3,-9.42072E-3],"split_indices":[2,50,4,59,64,46,53,4,7,0,2,60,61,0,0,5,0,41,53,49,0,0,0,0,0,0,0,2,59,0,4,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,8.8E1,3.5E1,6.6E1,2.2E1,2.3E1,1.2E1,2.3E1,4.3E1,6E0,1.6E1,1.1E1,1.2E1,7E0,5E0,1.9E1,4E0,3.1E1,1.2E1,1.2E1,4E0,7E0,4E0,8E0,4E0,7E0,1.2E1,1.1E1,2E1,4E0,8E0,4E0,8E0,7E0,4E0,7E0,1.3E1,4E0,4E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.6736286E-2,-1.6999894E-1,2.0919646E-1,-2.2654293E-1,1.3160981E-1,9.165079E-2,3.7897483E-1,-1.1296147E-1,-3.4015256E-1,1.4584179E-2,-3.2706966E-3,1.2634239E-1,-4.636346E-3,1.5482965E-1,5.120089E-1,-1.9603819E-1,4.7281746E-2,-1.9022241E-2,-7.575086E-3,-2.7312618E-2,1.9349512E-1,1.0766585E-2,1.8684459E-3,1.1857591E-2,3.0131271E-2,-1.2645209E-2,-1.131495E-1,7.2824713E-3,-3.2180229E-3,-4.216292E-3,1.339636E-3,2.4852361E-1,3.4866505E-3,-2.626302E-3,-7.1002906E-3,6.974969E-3,1.5766012E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,-1,-1,29,31,-1,-1,-1,-1,-1,33,-1,-1,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[3.9482648E0,9.33686E-1,1.1027994E0,5.4225326E-1,3.33859E-1,2.414265E-1,6.0656166E-1,3.2876858E-1,1.6910529E-1,0E0,0E0,3.148356E-1,0E0,7.288657E-2,2.8241205E-1,6.121081E-2,1.1884959E-1,0E0,0E0,3.6234092E-2,1.21089816E-1,0E0,0E0,0E0,0E0,0E0,1.0418586E-2,0E0,0E0,0E0,0E0,6.8858266E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,19,19,20,20,26,26,31,31],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,-1,-1,30,32,-1,-1,-1,-1,-1,34,-1,-1,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[2.2843675E-1,7.618855E1,1.3536367E0,4.851587E7,9E0,1.925776E1,4.2864155E1,2.8866995E1,3.5904072E3,1.4584179E-2,-3.2706966E-3,5.816E3,-4.636346E-3,9.519022E0,2.5172096E-1,9.7E1,2.70482E5,-1.9022241E-2,-7.575086E-3,3.47E3,1.11E3,1.0766585E-2,1.8684459E-3,1.1857591E-2,3.0131271E-2,-1.2645209E-2,1.2126529E7,7.2824713E-3,-3.2180229E-3,-4.216292E-3,1.339636E-3,1.5190904E9,3.4866505E-3,-2.626302E-3,-7.1002906E-3,6.974969E-3,1.5766012E-2],"split_indices":[47,64,47,53,25,62,64,64,4,0,0,2,0,61,46,10,37,0,0,2,0,0,0,0,0,0,53,0,0,0,0,12,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.08E2,5.2E1,5.6E1,4.4E1,8E0,3.4E1,2.2E1,2.3E1,2.1E1,4E0,4E0,2.9E1,5E0,9E0,1.3E1,1.5E1,8E0,1.5E1,6E0,9E0,2E1,5E0,4E0,5E0,8E0,7E0,8E0,4E0,4E0,4E0,5E0,1.3E1,7E0,4E0,4E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-6.4384136E-3,-1.6754752E-1,2.3022045E-1,-2.033914E-1,1.5852047E-2,1.0953059E-1,3.513955E-1,-1.00611E-1,-3.2751152E-1,3.173263E-2,1.0652367E-2,3.8975418E-1,6.46405E-3,-1.7385688E-2,-2.7007952E-2,-1.9973626E-2,-2.0043284E-1,-4.5137294E-2,6.543335E-3,1.2451683E-2,4.432902E-1,-6.7278504E-2,5.488253E-3,-6.64457E-3,-1.35125695E-2,-5.846082E-3,2.7040693E-3,1.0992733E-2,2.4884751E-2,3.6227472E-2,-1.21295705E-1,6.27627E-3,-3.162073E-3,-6.7253605E-2,-9.558862E-3,-4.703872E-3,-4.805824E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,-1,21,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,31,33,-1,-1,35,-1,-1,-1],"loss_changes":[4.423281E0,1.3028404E0,6.488676E-1,7.962613E-1,0E0,2.1371967E-1,1.3784862E-1,7.075839E-1,2.5051665E-1,1.3567142E-1,0E0,1.5661001E-2,0E0,0E0,1.7919812E-1,0E0,3.144437E-2,8.556667E-2,0E0,0E0,7.897067E-2,1.3858071E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.646614E-2,5.4727957E-2,0E0,0E0,1.8663459E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,14,14,16,16,17,17,20,20,21,21,29,29,30,30,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,-1,22,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,32,34,-1,-1,36,-1,-1,-1],"split_conditions":[1.8485416E5,1.8552577E2,1.3536367E0,5.070136E7,1.5852047E-2,8.18961E3,2.1383E4,9.7888E4,4.6573498E1,3.1484194E5,1.0652367E-2,1.9088202E0,6.46405E-3,-1.7385688E-2,5.1299828E1,-1.9973626E-2,1.1384717E8,6.70699E-2,6.543335E-3,1.2451683E-2,2.94E2,4.828443E-1,5.488253E-3,-6.64457E-3,-1.35125695E-2,-5.846082E-3,2.7040693E-3,1.0992733E-2,2.4884751E-2,1.1947E4,1.6546519E7,6.27627E-3,-3.162073E-3,2.7758823E1,-9.558862E-3,-4.703872E-3,-4.805824E-4],"split_indices":[41,64,47,53,0,4,9,1,64,36,0,50,0,0,64,0,53,46,0,0,0,35,0,0,0,0,0,0,0,9,53,0,0,66,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,6.8E1,4.6E1,6.4E1,4E0,2.4E1,2.2E1,3.6E1,2.8E1,1.5E1,9E0,1.8E1,4E0,7E0,2.9E1,1.5E1,1.3E1,9E0,6E0,7E0,1.1E1,2.3E1,6E0,9E0,4E0,5E0,4E0,4E0,7E0,8E0,1.5E1,4E0,4E0,1E1,5E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.1884916E-2,-1.1627673E-1,3.053001E-1,-1.9704488E-1,5.4839313E-2,4.435637E-1,3.872652E-2,-2.2539254E-1,7.634308E-3,-8.7771654E-2,2.1390618E-1,2.5115132E-1,2.8253889E-2,6.769663E-3,-5.408725E-3,-3.0270875E-1,-9.214437E-2,-1.9234231E-4,-1.5245846E-1,4.5244605E-3,1.2039446E-2,1.827181E-2,4.763815E-3,-1.9116788E-1,-3.76806E-1,3.0059505E-3,-1.3010502E-1,-9.920249E-3,-3.1854338E-3,-1.2591282E-1,-1.3321333E-2,-4.4862136E-1,-8.564991E-3,-8.520858E-3,-2.9591883E-3,-7.5686174E-3,-2.6329719E-3,-1.0727299E-2,-2.4630986E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,-1,27,-1,-1,-1,-1,29,31,-1,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8764355E0,1.2270296E0,1.0516877E0,6.4307237E-1,6.766306E-1,3.926823E-1,1.8829376E-1,5.490327E-1,0E0,8.60602E-2,3.983438E-2,1.6084945E-1,0E0,0E0,0E0,2.1723723E-1,1.3770781E-1,0E0,2.8799519E-2,0E0,0E0,0E0,0E0,5.6519806E-2,2.0315504E-1,0E0,4.8206925E-2,0E0,0E0,1.6620398E-2,0E0,9.468818E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,18,18,23,23,24,24,26,26,29,29,31,31],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,-1,28,-1,-1,-1,-1,30,32,-1,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9446938E3,2.3724032E8,2.692756E6,1E0,5.12544E-1,1.0861106E4,3.2944346E9,7.355655E6,7.634308E-3,1.501414E5,1.0812932E3,1.8403254E7,2.8253889E-2,6.769663E-3,-5.408725E-3,7.156922E7,2.5758067E-1,-1.9234231E-4,5.0557896E1,4.5244605E-3,1.2039446E-2,1.827181E-2,4.763815E-3,6.5549725E5,3.2316522E7,3.0059505E-3,9.162401E4,-9.920249E-3,-3.1854338E-3,1.1717857E1,-1.3321333E-2,5.1E1,-8.564991E-3,-8.520858E-3,-2.9591883E-3,-7.5686174E-3,-2.6329719E-3,-1.0727299E-2,-2.4630986E-2],"split_indices":[60,7,37,96,47,4,7,40,0,41,60,53,0,0,0,7,35,0,64,0,0,0,0,36,12,0,41,0,0,61,0,10,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,8.7E1,2.8E1,5.9E1,2.8E1,1.8E1,1E1,5.5E1,4E0,1.5E1,1.3E1,9E0,9E0,6E0,4E0,3.4E1,2.1E1,7E0,8E0,4E0,9E0,4E0,5E0,1.5E1,1.9E1,4E0,1.7E1,4E0,4E0,1E1,5E0,1.3E1,6E0,9E0,8E0,6E0,4E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.8102366E-2,-1.3931242E-1,2.1591575E-1,-1.6406868E-1,1.0992986E-2,1.3657893E-1,4.1563874E-1,-1.05985574E-1,-3.1134078E-1,-4.4315853E-3,1.6893654E-1,1.02697285E-2,2.6159517E-2,-2.7020523E-1,-3.6545567E-2,-1.8002655E-2,-5.0778696E-3,-3.8000997E-3,3.4191506E-3,1.20410785E-1,2.6386443E-1,-6.6961297E-3,-1.7499147E-2,-7.2190754E-2,8.2371764E-2,9.934685E-3,7.478101E-2,5.848272E-3,1.725417E-2,-9.4166845E-2,1.325032E-3,-1.2663793E-3,8.347246E-3,1.9973224E-2,1.2736955E-1,-1.8003656E-3,-5.112024E-3,-1.6110218E-3,3.8409424E-3,8.168405E-3,2.780576E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,-1,-1,-1,25,27,-1,-1,29,31,-1,33,-1,-1,35,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0781174E0,7.05528E-1,8.622649E-1,5.5722976E-1,0E0,1.9694734E-1,3.0627227E-1,5.716994E-1,2.15541E-1,5.7110853E-2,1.294536E-1,0E0,0E0,1.3623393E-1,1.6278322E-1,0E0,0E0,0E0,0E0,8.439058E-2,1.0730082E-1,0E0,0E0,6.630263E-2,9.5201306E-2,0E0,4.9336724E-2,0E0,0E0,1.3414606E-2,0E0,0E0,0E0,3.5181098E-2,1.7219022E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,19,19,20,20,23,23,24,24,26,26,29,29,33,33,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,-1,-1,-1,26,28,-1,-1,30,32,-1,34,-1,-1,36,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8938947E8,6.099784E3,1.7915075E0,5.501383E9,1.0992986E-2,6.919127E5,2.463E4,1.21719E5,3.5904072E3,6.758564E6,1.5791555E9,1.02697285E-2,2.6159517E-2,7.660981E5,3.566468E5,-1.8002655E-2,-5.0778696E-3,-3.8000997E-3,3.4191506E-3,1.7644464E5,2.119303E6,-6.6961297E-3,-1.7499147E-2,7.388E3,1.05190264E8,9.934685E-3,6.206162E5,5.848272E-3,1.725417E-2,1.675502E8,1.325032E-3,-1.2663793E-3,8.347246E-3,1.0639322E0,8E0,-1.8003656E-3,-5.112024E-3,-1.6110218E-3,3.8409424E-3,8.168405E-3,2.780576E-3],"split_indices":[12,4,50,5,0,40,2,1,4,1,7,0,0,59,36,0,0,0,0,36,40,0,0,2,7,0,36,0,0,5,0,0,0,50,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.29E2,7.2E1,5.7E1,6.8E1,4E0,4.2E1,1.5E1,5E1,1.8E1,8E0,3.4E1,7E0,8E0,1.4E1,3.6E1,1.3E1,5E0,4E0,4E0,2.4E1,1E1,7E0,7E0,2.8E1,8E0,7E0,1.7E1,5E0,5E0,2.3E1,5E0,4E0,4E0,9E0,8E0,5E0,1.8E1,5E0,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.0659487E-2,-6.816797E-2,3.0225462E-1,-1.9995402E-1,4.8872195E-2,4.398868E-1,5.667534E-2,-2.4269252E-1,-5.3560894E-2,-3.7946597E-2,1.7665678E-1,9.336904E-3,2.6176034E-2,6.287025E-3,-4.2793937E-3,-2.7279434E-1,-4.3895766E-3,5.7514147E-3,-8.127709E-3,7.3072305E-3,-7.320936E-2,1.8758499E-3,1.1114317E-2,-2.0943896E-1,-3.762341E-1,-1.04862615E-1,1.3222431E-3,-5.093513E-3,-2.2973764E-1,-1.0313018E-2,-2.2028934E-2,-1.5315467E-3,-6.3883793E-3,-1.2599768E-2,-5.1951385E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1,25,-1,-1,27,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1],"loss_changes":[3.322054E0,1.41338E0,1.0904565E0,2.5603604E-1,5.5031455E-1,4.4823647E-1,1.5436815E-1,1.24702215E-1,2.4371305E-1,2.1321042E-1,1.4491516E-1,0E0,0E0,0E0,0E0,9.8145485E-2,0E0,0E0,0E0,0E0,8.593947E-2,0E0,0E0,1.5138447E-2,2.3063302E-2,3.7793547E-2,0E0,0E0,3.315711E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,15,15,20,20,23,23,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1,26,-1,-1,28,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5016E4,3.851071E7,1.4046429E1,6.939E3,9.2954946E-1,1.6872E5,1.782E3,3.0174E4,3.24395E5,2.7186182E-1,1.6511278E1,9.336904E-3,2.6176034E-2,6.287025E-3,-4.2793937E-3,1E0,-4.3895766E-3,5.7514147E-3,-8.127709E-3,7.3072305E-3,1.3627136E8,1.8758499E-3,1.1114317E-2,1.8089891E2,8.62E3,5.4470426E-1,1.3222431E-3,-5.093513E-3,4.757E3,-1.0313018E-2,-2.2028934E-2,-1.5315467E-3,-6.3883793E-3,-1.2599768E-2,-5.1951385E-3],"split_indices":[2,12,62,2,47,36,0,9,37,35,64,0,0,0,0,22,0,0,0,0,40,0,0,60,9,35,0,0,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.22E2,9E1,3.2E1,4.2E1,4.8E1,2E1,1.2E1,3.2E1,1E1,2.9E1,1.9E1,7E0,1.3E1,8E0,4E0,2.6E1,6E0,4E0,6E0,4E0,2.5E1,6E0,1.3E1,1.8E1,8E0,1.9E1,6E0,4E0,1.4E1,4E0,4E0,6E0,1.3E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.078145E-2,-1.3408571E-1,1.6237593E-1,-1.6004753E-1,1.0251378E-2,5.936143E-2,3.1658348E-1,-1.8052956E-1,5.231934E-3,-3.827363E-2,1.21978074E-1,2.1890321E-1,2.2250617E-2,-2.6475564E-1,-8.717399E-2,-1.217262E-1,2.8842608E-3,1.0887961E-3,1.4687796E-1,1.1590431E-2,5.408904E-3,-3.2890874E-1,-1.0355851E-1,1.8017327E-3,-1.0866727E-1,-7.6558255E-3,-2.80804E-3,1.811147E-1,3.6850665E-3,-2.5226018E-1,-2.5011795E-2,-6.5355804E-3,-2.0485572E-3,-1.3637854E-1,8.667208E-5,1.0902161E-2,4.5797443E-3,-1.4084578E-2,-4.340401E-3,-1.5909337E-1,-3.413003E-3,-8.826783E-3,-3.6202632E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,21,23,25,-1,-1,27,-1,-1,29,31,-1,33,-1,-1,35,-1,37,-1,-1,-1,39,-1,-1,-1,-1,-1,41,-1,-1,-1],"loss_changes":[2.8327882E0,6.319201E-1,9.820007E-1,3.6214054E-1,0E0,2.4309814E-1,2.626779E-1,4.3297243E-1,0E0,1.3909768E-1,5.592069E-2,1.5244365E-2,0E0,2.7617764E-1,8.195126E-2,1.2425035E-2,0E0,0E0,3.1373978E-2,0E0,0E0,1.8147492E-1,1.4326021E-2,0E0,7.678753E-2,0E0,0E0,2.6344806E-2,0E0,9.0235114E-2,0E0,0E0,0E0,1.8119603E-2,0E0,0E0,0E0,0E0,0E0,1.5995592E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,18,18,21,21,22,22,24,24,27,27,29,29,33,33,39,39],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,22,24,26,-1,-1,28,-1,-1,30,32,-1,34,-1,-1,36,-1,38,-1,-1,-1,40,-1,-1,-1,-1,-1,42,-1,-1,-1],"split_conditions":[2.5092034E8,4.797364E3,1.5364555E0,1E0,1.0251378E-2,5.9176934E-1,1.5605432E9,2.667836E6,5.231934E-3,4.7308203E3,3.0992308E1,2.5337728E5,2.2250617E-2,2.6631096E3,4.570007E7,1.6843E4,2.8842608E-3,1.0887961E-3,4.7E1,1.1590431E-2,5.408904E-3,8.4477744E5,7.388E3,1.8017327E-3,3.365958E8,-7.6558255E-3,-2.80804E-3,1.4922581E1,3.6850665E-3,1.0478E4,-2.5011795E-2,-6.5355804E-3,-2.0485572E-3,2.9153493E-1,8.667208E-5,1.0902161E-2,4.5797443E-3,-1.4084578E-2,-4.340401E-3,7E0,-3.413003E-3,-8.826783E-3,-3.6202632E-3],"split_indices":[12,4,50,96,0,49,7,40,0,4,64,41,0,63,5,10,0,0,3,0,0,40,2,0,7,0,0,62,0,9,0,0,0,47,0,0,0,0,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.27E2,6.5E1,6.2E1,6.1E1,4E0,3.8E1,2.4E1,5.7E1,4E0,1.5E1,2.3E1,1.6E1,8E0,2.9E1,2.8E1,8E0,7E0,5E0,1.8E1,1.2E1,4E0,2E1,9E0,4E0,2.4E1,4E0,4E0,1.1E1,7E0,1.6E1,4E0,5E0,4E0,1.9E1,5E0,6E0,5E0,1.2E1,4E0,1.3E1,6E0,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.75063E-2,-1.0017915E-1,2.5526243E-1,-2.2926214E-1,9.063744E-3,1.4693028E-1,4.2211902E-1,-2.8325275E-1,-6.60813E-2,-6.791534E-2,1.260811E-1,1.7817196E-2,7.857401E-2,2.4886133E-2,8.132941E-3,-6.1554846E-4,-3.2599533E-1,3.518703E-3,-9.304399E-3,-1.1416246E-1,3.1370427E-2,1.6715394E-1,4.4084058E-4,1.2446486E-1,-3.0825003E-3,-2.21971E-1,-2.0927561E-2,-6.2179924E-3,-1.9105774E-3,5.877931E-3,-4.0572127E-3,3.2020945E-3,2.0049053E-1,1.9097016E-3,7.4658846E-3,-1.2400121E-2,-4.521634E-3,1.1618667E-2,5.2071875E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,-1,25,-1,-1,27,29,31,-1,33,-1,35,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5511072E0,1.200697E0,6.9716716E-1,3.2488155E-1,4.3193358E-1,3.8807464E-1,3.1006694E-1,3.3092332E-1,2.1228707E-1,1.3674004E-1,8.964503E-2,0E0,1.5252717E-1,0E0,0E0,0E0,1.9455814E-1,0E0,0E0,1.857838E-2,1.1713897E-1,3.3276826E-2,0E0,3.9512098E-2,0E0,4.7361135E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.2613505E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,16,16,19,19,20,20,21,21,23,23,25,25,32,32],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,-1,26,-1,-1,28,30,32,-1,34,-1,36,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5328901E3,2.667836E6,1.0825864E4,2.2794224E3,5.093857E-1,1.5046106E7,1.200998E10,4.029937E-1,7.752181E7,3.181762E5,1E0,1.7817196E-2,5.9633E7,2.4886133E-2,8.132941E-3,-6.1554846E-4,2.4E2,3.518703E-3,-9.304399E-3,7.287E3,2E0,5.3149715E6,4.4084058E-4,7.94E2,-3.0825003E-3,1.0376E4,-2.0927561E-2,-6.2179924E-3,-1.9105774E-3,5.877931E-3,-4.0572127E-3,3.2020945E-3,9.163856E0,1.9097016E-3,7.4658846E-3,-1.2400121E-2,-4.521634E-3,1.1618667E-2,5.2071875E-3],"split_indices":[60,40,60,60,47,53,5,35,53,36,24,0,53,0,0,0,10,0,0,2,8,40,0,0,0,9,0,0,0,0,0,0,61,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,8.4E1,4.1E1,3.8E1,4.6E1,2.6E1,1.5E1,2.8E1,1E1,2.8E1,1.8E1,5E0,2.1E1,1E1,5E0,4E0,2.4E1,5E0,5E0,1.9E1,9E0,1.3E1,5E0,1.6E1,5E0,1.4E1,1E1,1.5E1,4E0,5E0,4E0,4E0,9E0,5E0,1.1E1,1E1,4E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.5440443E-3,-1.3100442E-1,1.4000645E-1,-1.6929564E-1,1.19501546E-1,9.251479E-3,2.2922692E-1,-1.2971342E-1,-2.6644194E-1,-2.4260203E-3,1.4325256E-2,7.418763E-2,-7.3662694E-3,1.68105E-1,3.4764895E-1,-2.9709738E-1,-8.096874E-2,-1.527975E-2,-8.755274E-3,5.319043E-3,1.1174838E-2,1.8581017E-3,1.888825E-1,2.1620171E-2,5.2540023E-3,-6.569316E-3,-2.0493684E-2,-2.9603159E-2,-1.4653586E-1,5.0061424E-3,-4.0394787E-2,1.1282606E-2,1.257737E-1,-8.457043E-2,4.788356E-3,-9.334437E-3,-4.6628374E-3,-2.3988938E-4,-3.2968444E-3,2.72611E-3,7.3060123E-3,-5.922518E-4,-5.138317E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,-1,-1,29,-1,-1,31,-1,-1,-1,-1,33,35,-1,37,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4721615E0,7.012757E-1,7.4450445E-1,2.038455E-1,3.225645E-1,2.9809815E-1,2.1660924E-1,3.5126197E-1,1.6943693E-2,0E0,0E0,2.1886882E-1,0E0,6.5129876E-2,2.4979675E-1,1.5160823E-1,1.2027411E-1,0E0,0E0,7.246684E-2,0E0,0E0,3.904146E-2,0E0,0E0,0E0,0E0,1.6552687E-1,1.9191533E-2,0E0,1.0796398E-2,0E0,1.369597E-2,2.5250323E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,19,19,22,22,27,27,28,28,30,30,32,32,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,-1,-1,30,-1,-1,32,-1,-1,-1,-1,34,36,-1,38,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.3514317E8,2.9853157E3,8.8579214E-1,5.5124454E9,6.3997424E-1,5.5E1,1.5605432E9,9.2433E4,1.8879009E3,-2.4260203E-3,1.4325256E-2,8.517173E3,-7.3662694E-3,1.1648228E0,1.767918E6,5.644265E4,3.925844E7,-1.527975E-2,-8.755274E-3,5.816E3,1.1174838E-2,1.8581017E-3,1.2271712E1,2.1620171E-2,5.2540023E-3,-6.569316E-3,-2.0493684E-2,1.4861879E3,2.1750195E3,5.0061424E-3,3.9343938E-1,1.1282606E-2,1.3007676E7,2.1305195E1,4.788356E-3,-9.334437E-3,-4.6628374E-3,-2.3988938E-4,-3.2968444E-3,2.72611E-3,7.3060123E-3,-5.922518E-4,-5.138317E-3],"split_indices":[7,60,50,5,35,3,7,1,60,0,0,4,0,47,37,41,53,0,0,2,0,0,62,0,0,0,0,4,4,0,35,0,56,66,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.33E2,7E1,6.3E1,6.1E1,9E0,2.6E1,3.7E1,4.5E1,1.6E1,5E0,4E0,1.9E1,7E0,2.6E1,1.1E1,9E0,3.6E1,8E0,8E0,1.4E1,5E0,4E0,2.2E1,7E0,4E0,5E0,4E0,2.1E1,1.5E1,4E0,1E1,1.1E1,1.1E1,1.5E1,6E0,6E0,9E0,5E0,5E0,4E0,7E0,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.883241E-3,-1.2720196E-1,1.7627677E-1,-5.8644153E-2,-2.7071014E-1,8.3678745E-2,3.0716124E-1,-9.6917585E-2,1.4575399E-2,-1.5276056E-2,-6.6166394E-3,-2.7684238E-2,1.333011E-1,3.769822E-1,4.9519213E-3,-1.4283857E-2,-5.9113503E-2,-4.3592234E-3,1.935352E-3,5.74661E-2,1.9802766E-1,2.0841066E-2,9.800655E-3,-1.8499732E-2,-1.5233731E-1,4.6839835E-3,2.4151379E-5,4.0784255E-3,1.1636747E-2,-9.146692E-2,6.621275E-2,-1.0374139E-2,-4.428887E-3,-6.4959084E-3,-4.1700758E-2,6.593265E-3,-1.2109376E-3,-4.4512413E-3,8.6654787E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,21,-1,-1,23,-1,-1,25,27,-1,-1,29,31,-1,-1,-1,-1,33,35,-1,-1,-1,37,-1,-1,-1,-1],"loss_changes":[2.8419144E0,6.9250417E-1,6.287577E-1,7.2585803E-1,1.0511911E-1,1.8605141E-1,2.7345467E-1,3.3390436E-1,0E0,0E0,0E0,5.1414885E-2,1.01516545E-1,7.3732615E-2,0E0,0E0,1.486286E-1,0E0,0E0,2.7710889E-2,4.0873438E-2,0E0,0E0,1.8531199E-1,2.6263922E-2,0E0,0E0,0E0,0E0,2.9803991E-2,9.56053E-2,0E0,0E0,0E0,2.9432759E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,13,13,16,16,19,19,20,20,23,23,24,24,29,29,30,30,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,22,-1,-1,24,-1,-1,26,28,-1,-1,30,32,-1,-1,-1,-1,34,36,-1,-1,-1,38,-1,-1,-1,-1],"split_conditions":[4.664004E8,2.5927516E11,1.5364555E0,2.7728145E3,3.5904072E3,8.819445E0,2.326E4,1.269E3,1.4575399E-2,-1.5276056E-2,-6.6166394E-3,3.5849205E-1,5.506832E-1,4.4E1,4.9519213E-3,-1.4283857E-2,3.925844E7,-4.3592234E-3,1.935352E-3,9E0,7.41E2,2.0841066E-2,9.800655E-3,1.2239595E3,4.757E3,4.6839835E-3,2.4151379E-5,4.0784255E-3,1.1636747E-2,1.0629717E1,5.28076E5,-1.0374139E-2,-4.428887E-3,-6.4959084E-3,5.1612902E-2,6.593265E-3,-1.2109376E-3,-4.4512413E-3,8.6654787E-4],"split_indices":[12,39,50,60,4,62,9,2,0,0,0,47,35,3,0,0,53,0,0,25,0,0,0,4,2,0,0,0,0,61,1,0,0,0,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.24E2,7.1E1,5.3E1,4.9E1,2.2E1,3.2E1,2.1E1,4.5E1,4E0,1.5E1,7E0,1E1,2.2E1,1.5E1,6E0,6E0,3.9E1,5E0,5E0,1.1E1,1.1E1,1E1,5E0,2.8E1,1.1E1,6E0,5E0,4E0,7E0,1.5E1,1.3E1,4E0,7E0,7E0,8E0,7E0,6E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.3236227E-2,-8.8224314E-2,1.9697312E-1,-1.1655237E-1,1.2948681E-2,1.2670667E-1,3.1969002E-1,-2.5636396E-1,-7.28094E-2,-2.4389217E-3,1.5719365E-1,7.4459184E-3,1.9654373E-2,-3.0708492E-1,-3.4965978E-3,-4.440945E-2,-1.7560843E-1,3.805768E-3,1.8290651E-1,-7.673758E-3,-1.6920164E-2,-9.114647E-2,5.8256827E-2,-1.1072376E-2,-2.4118053E-3,9.633484E-3,4.472374E-3,-1.4823286E-1,-5.4185368E-2,1.1969492E-1,-1.3876901E-3,-9.02648E-3,-4.3986733E-3,-7.030067E-2,5.46516E-4,5.2257074E-4,9.766681E-3,-4.9247555E-3,-1.7676692E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,19,-1,21,23,-1,25,-1,-1,27,29,-1,-1,-1,-1,31,33,35,-1,-1,-1,37,-1,-1,-1,-1,-1],"loss_changes":[2.2544231E0,8.256437E-1,3.3027697E-1,4.2778587E-1,0E0,1.6520378E-1,1.6150439E-1,1.3543618E-1,1.5861207E-1,0E0,3.5244703E-2,0E0,0E0,3.2926798E-2,0E0,2.260324E-1,7.2724074E-2,0E0,1.13348365E-2,0E0,0E0,5.9487194E-2,8.3944336E-2,0E0,0E0,0E0,0E0,1.0055125E-2,2.301478E-2,8.07755E-2,0E0,0E0,0E0,1.4755517E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,13,13,15,15,16,16,18,18,21,21,22,22,27,27,28,28,29,29,33,33],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,20,-1,22,24,-1,26,-1,-1,28,30,-1,-1,-1,-1,32,34,36,-1,-1,-1,38,-1,-1,-1,-1,-1],"split_conditions":[5.1707494E-1,6.083071E3,1.5605432E9,4.1964554E-4,1.2948681E-2,7.010949E6,2.4190365E6,2.873808E3,8.5154843E-1,-2.4389217E-3,1.218666E0,7.4459184E-3,1.9654373E-2,2.3737179E1,-3.4965978E-3,3.566468E5,8.615362E5,3.805768E-3,2.0837E4,-7.673758E-3,-1.6920164E-2,2E0,7.45658E5,-1.1072376E-2,-2.4118053E-3,9.633484E-3,4.472374E-3,2.91E2,1.0098112E2,2.78876E5,-1.3876901E-3,-9.02648E-3,-4.3986733E-3,4.1E1,5.46516E-4,5.2257074E-4,9.766681E-3,-4.9247555E-3,-1.7676692E-3],"split_indices":[47,60,7,50,0,56,40,4,35,0,47,0,0,64,0,36,36,0,9,0,0,8,1,0,0,0,0,0,64,1,0,0,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,7.7E1,4.2E1,7.2E1,5E0,2.8E1,1.4E1,1.6E1,5.6E1,4E0,2.4E1,6E0,8E0,1.2E1,4E0,4.5E1,1.1E1,7E0,1.7E1,4E0,8E0,3.1E1,1.4E1,7E0,4E0,1.3E1,4E0,1.1E1,2E1,8E0,6E0,5E0,6E0,1.6E1,4E0,4E0,4E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-7.1179676E-3,-1.2301354E-1,1.16369545E-1,-1.4824003E-1,7.352549E-3,-1.5212338E-2,1.9987422E-1,-9.520415E-2,-2.3793073E-1,-8.497125E-2,9.792316E-2,1.6701823E-1,1.8306157E-2,-1.5649525E-2,-6.0269643E-2,-2.7017534E-1,-3.5466137E-3,-1.4475557E-3,-8.16737E-3,2.2399577E-3,6.665052E-3,3.4752695E-3,1.8875775E-1,-7.8929365E-2,2.205596E-3,-5.180519E-3,-1.4739026E-2,3.7238619E-3,2.04634E-1,-6.107331E-3,-5.9322465E-2,1.1246117E-2,6.23108E-3,-3.4066655E-3,2.800615E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,-1,23,25,-1,-1,-1,-1,-1,-1,27,29,-1,-1,-1,-1,31,-1,33,-1,-1,-1,-1],"loss_changes":[1.8318954E0,4.8396623E-1,6.8427813E-1,2.6751423E-1,0E0,2.0562589E-1,1.650157E-1,3.0512387E-1,9.6037865E-2,7.007134E-2,1.4698222E-2,5.592704E-2,0E0,0E0,7.515544E-2,7.843602E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.1531394E-2,2.216886E-2,0E0,0E0,0E0,0E0,2.1899343E-2,0E0,1.731155E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,22,22,23,23,28,28,30,30],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,-1,24,26,-1,-1,-1,-1,-1,-1,28,30,-1,-1,-1,-1,32,-1,34,-1,-1,-1,-1],"split_conditions":[1.5363757E8,2.9853157E3,1.0639322E0,8.297822E-1,7.352549E-3,3.9219662E5,1.1490676E6,9.2433E4,3.184188E1,7.752181E7,9.638E3,1E0,1.8306157E-2,-1.5649525E-2,1E1,2.07E2,-3.5466137E-3,-1.4475557E-3,-8.16737E-3,2.2399577E-3,6.665052E-3,3.4752695E-3,3.3213286E8,1.85E2,2.205596E-3,-5.180519E-3,-1.4739026E-2,3.7238619E-3,1.33556795E1,-6.107331E-3,1.8116838E1,1.1246117E-2,6.23108E-3,-3.4066655E-3,2.800615E-5],"split_indices":[12,60,50,35,0,36,36,1,66,53,2,109,0,0,8,11,0,0,0,0,0,0,7,10,0,0,0,0,62,0,62,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.26E2,6.5E1,6.1E1,6E1,5E0,2.4E1,3.7E1,3.9E1,2.1E1,1.5E1,9E0,3.3E1,4E0,4E0,3.5E1,1.7E1,4E0,1E1,5E0,5E0,4E0,7E0,2.6E1,3E1,5E0,4E0,1.3E1,4E0,2.2E1,7E0,2.3E1,1.4E1,8E0,1.9E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[5.244775E-3,-1.20529205E-1,1.5398991E-1,-7.090418E-2,-2.6270822E-1,5.9535485E-2,2.4645981E-1,-1.5283582E-1,8.8066526E-2,-5.928742E-3,-1.4013311E-2,-2.926479E-2,1.1716012E-1,1.7022452E-1,1.8238716E-2,-2.7170837E-1,-1.0314837E-1,1.4984271E-2,-1.4950478E-2,4.6435273E-3,-9.6289545E-2,9.547756E-3,6.358244E-2,1.9417638E-1,2.6143675E-3,-5.292199E-3,-1.9601226E-2,-1.1387879E-2,-5.3134177E-2,3.5919284E-3,-4.3924158E-3,-6.2100682E-3,-2.0672032E-3,-6.595853E-4,4.752487E-3,4.806027E-3,1.0278086E-2,-8.001394E-2,2.3345377E-3,-5.4085064E-3,-7.8100676E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,-1,29,-1,31,-1,33,35,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,39,-1,-1,-1],"loss_changes":[2.450635E0,4.823233E-1,5.0937843E-1,7.167952E-1,4.0198207E-2,1.6205823E-1,2.4476767E-1,1.8460679E-1,4.4644946E-1,0E0,0E0,1.19582534E-1,7.1540296E-2,4.7455907E-2,0E0,1.7772335E-1,1.7004627E-1,0E0,1.0414419E-1,0E0,1.0469027E-2,0E0,3.7526116E-2,1.1800349E-2,0E0,0E0,0E0,0E0,6.1498E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.411773E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,18,18,20,20,22,22,23,23,28,28,37,37],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,-1,30,-1,32,-1,34,36,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,40,-1,-1,-1],"split_conditions":[9.379017E6,6.721145E7,1.5364555E0,3.3041668E1,2.9844797E-1,4.7308203E3,1.5605432E9,1.477E3,1.0526E4,-5.928742E-3,-1.4013311E-2,1.5779017E7,1.1E1,7.639958E-1,1.8238716E-2,2.16268E5,6.7281544E-5,1.4984271E-2,3.3782916E7,4.6435273E-3,1.6E1,9.547756E-3,8.517173E3,1.601247E1,2.6143675E-3,-5.292199E-3,-1.9601226E-2,-1.1387879E-2,1.0668866E3,3.5919284E-3,-4.3924158E-3,-6.2100682E-3,-2.0672032E-3,-6.595853E-4,4.752487E-3,4.806027E-3,1.0278086E-2,2.207872E-1,2.3345377E-3,-5.4085064E-3,-7.8100676E-4],"split_indices":[59,53,50,64,35,4,7,2,9,0,0,53,8,35,0,12,46,0,53,0,8,0,4,64,0,0,0,0,60,0,0,0,0,0,0,0,0,47,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.29E2,7E1,5.9E1,5.3E1,1.7E1,3E1,2.9E1,3.5E1,1.8E1,4E0,1.3E1,1.2E1,1.8E1,2E1,9E0,9E0,2.6E1,5E0,1.3E1,4E0,8E0,6E0,1.2E1,1.6E1,4E0,5E0,4E0,6E0,2E1,6E0,7E0,4E0,4E0,4E0,8E0,4E0,1.2E1,1.6E1,4E0,1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-7.007901E-4,-8.645468E-2,1.3379283E-1,-3.4035087E-2,-2.1318093E-1,2.2030236E-1,3.7269566E-2,-6.711338E-2,1.3233047E-2,-2.3962589E-1,-4.103311E-3,1.4866668E-1,1.7326321E-2,-4.4932384E-2,5.9006573E-3,-4.024052E-2,-1.3344347E-2,-6.2103686E-3,-1.3962782E-2,8.768215E-3,1.3421358E-3,-4.1483967E-3,1.487446E-3,-6.075677E-2,4.44058E-3,-3.146169E-2,-7.7852593E-3,2.463856E-3,-5.8981918E-2,-7.9067685E-2,-8.482637E-5,-5.627313E-3,-4.0148973E-2,-1.3944578E-5,-3.184026E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,21,-1,23,-1,-1,-1,-1,-1,-1,-1,25,-1,27,-1,-1,29,31,-1,-1,33,-1,-1],"loss_changes":[1.3033024E0,4.5110714E-1,3.5919732E-1,5.331745E-1,4.6718836E-2,1.8085015E-1,1.6159625E-1,2.5120243E-1,0E0,5.446261E-2,0E0,6.644112E-2,0E0,4.0987603E-2,0E0,1.2044583E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0571474E-1,0E0,7.161242E-2,0E0,0E0,2.6287027E-2,2.0916723E-2,0E0,0E0,1.0551661E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,13,13,15,15,23,23,25,25,28,28,29,29,32,32],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,22,-1,24,-1,-1,-1,-1,-1,-1,-1,26,-1,28,-1,-1,30,32,-1,-1,34,-1,-1],"split_conditions":[4.664004E8,2.5927516E11,2.5129596E11,2.7728145E3,4.8E3,1.2750684E9,8.18961E3,6.313997E-1,1.3233047E-2,1.07715E5,-4.103311E-3,1E0,1.7326321E-2,9.851E3,5.9006573E-3,2.9193023E-1,-1.3344347E-2,-6.2103686E-3,-1.3962782E-2,8.768215E-3,1.3421358E-3,-4.1483967E-3,1.487446E-3,8.5154843E-1,4.44058E-3,3.5087322E3,-7.7852593E-3,2.463856E-3,3.365958E8,1.3101191E1,-8.482637E-5,-5.627313E-3,1.087337E1,-1.3944578E-5,-3.184026E-3],"split_indices":[12,39,39,60,10,7,4,46,0,12,0,24,0,9,0,46,0,0,0,0,0,0,0,35,0,41,0,0,7,64,0,0,61,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.11E2,6.8E1,4.3E1,4.9E1,1.9E1,2.2E1,2.1E1,4.5E1,4E0,1.5E1,4E0,1.6E1,6E0,1.1E1,1E1,4.1E1,4E0,6E0,9E0,1.2E1,4E0,7E0,4E0,3.6E1,5E0,2.9E1,7E0,7E0,2.2E1,1.6E1,6E0,7E0,9E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.5418748E-2,-7.891762E-2,1.483046E-1,-1.3548203E-1,3.8727254E-2,5.766494E-3,1.9032454E-1,-7.225901E-2,-2.0275366E-1,1.1007832E-2,-2.965924E-2,3.3533806E-3,-2.8576797E-3,1.6232789E-2,1.4087875E-1,-8.635765E-2,1.1444894E-3,-2.4262498E-1,-3.2361632E-3,-1.0300592E-2,2.0856788E-2,8.195003E-3,8.685521E-2,-9.714324E-2,-7.2390673E-4,-1.2954827E-2,-7.573955E-3,-3.6648393E-2,7.237999E-3,7.118373E-4,6.3200193E-3,-7.296227E-2,-6.305083E-3,-5.0578625E-3,1.3597157E-2,-4.1378257E-3,-1.0887735E-3,-1.3231364E-3,2.4919794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,-1,21,23,-1,25,-1,-1,27,-1,29,31,-1,-1,-1,33,-1,-1,-1,35,-1,-1,37,-1,-1,-1,-1],"loss_changes":[1.3008387E0,6.041528E-1,2.0853299E-1,2.4478936E-1,4.0027124E-1,4.225644E-2,1.5180999E-1,4.6838254E-2,1.3964391E-1,0E0,2.2063988E-1,0E0,0E0,0E0,2.1606773E-2,2.0797387E-2,0E0,9.927511E-3,0E0,0E0,1.483599E-1,0E0,3.0751817E-2,1.1453882E-2,0E0,0E0,0E0,5.0160337E-2,0E0,0E0,0E0,9.691976E-3,0E0,0E0,1.5788147E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,14,14,15,15,17,17,20,20,22,22,23,23,27,27,31,31,34,34],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,-1,22,24,-1,26,-1,-1,28,-1,30,32,-1,-1,-1,34,-1,-1,-1,36,-1,-1,38,-1,-1,-1,-1],"split_conditions":[1.0230335E0,2.532835E3,1.6511278E1,8.0073607E-1,2.5927516E11,1.1180212E1,8.892E3,3.4707712E8,2.4502628E6,1.1007832E-2,2.4719655E5,3.3533806E-3,-2.8576797E-3,1.6232789E-2,2.7987084E7,6.1767822E1,1.1444894E-3,2.2375912E1,-3.2361632E-3,-1.0300592E-2,4.152984E-1,8.195003E-3,3.8737573E11,3.8980476E1,-7.2390673E-4,-1.2954827E-2,-7.573955E-3,3.4922136E11,7.237999E-3,7.118373E-4,6.3200193E-3,6.78E2,-6.305083E-3,-5.0578625E-3,1.0005E4,-4.1378257E-3,-1.0887735E-3,-1.3231364E-3,2.4919794E-3],"split_indices":[47,60,64,35,39,62,9,7,40,0,40,0,0,0,53,64,0,66,0,0,47,0,39,64,0,0,0,39,0,0,0,0,0,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,8.9E1,3.4E1,6E1,2.9E1,8E0,2.6E1,3.2E1,2.8E1,7E0,2.2E1,4E0,4E0,5E0,2.1E1,2.8E1,4E0,2.1E1,7E0,4E0,1.8E1,1.2E1,9E0,2.4E1,4E0,1.4E1,7E0,1.3E1,5E0,4E0,5E0,1.6E1,8E0,5E0,8E0,1.2E1,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.1260364E-3,-6.945903E-2,1.5547651E-1,-1.5852132E-1,1.12510035E-2,2.2038478E-1,-6.289796E-3,-1.295489E-2,-1.3064645E-1,5.4722123E-2,-9.9964224E-2,2.3542495E-2,1.6578855E-1,-1.5847655E-1,2.9830588E-3,-1.5157726E-2,1.2138302E-1,-1.545761E-3,-7.3201926E-3,2.4823765E-1,1.16274565E-1,-1.9303443E-1,-9.055354E-2,3.3067344E-3,-4.6452817E-2,1.686485E-1,1.4216381E-3,6.438302E-3,1.4393159E-2,1.6752116E-3,1.3438135E-1,-6.821379E-3,-1.2353949E-2,-6.068525E-3,-2.2119225E-3,-3.64054E-3,7.850287E-4,1.0487227E-2,5.165597E-3,8.510144E-3,3.2613445E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,-1,19,21,-1,23,25,-1,-1,27,29,31,33,-1,35,37,-1,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3709786E0,6.0559416E-1,7.746428E-1,9.027147E-2,2.2302952E-1,4.093747E-1,0E0,0E0,1.9262421E-1,1.5556224E-1,4.1890934E-2,0E0,8.95254E-2,5.2938223E-2,0E0,4.8117995E-2,6.859332E-2,0E0,0E0,1.9649744E-2,2.469036E-2,2.5666118E-2,1.3499849E-2,0E0,2.5598962E-2,1.0144234E-2,0E0,0E0,0E0,0E0,3.416413E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,12,12,13,13,15,15,16,16,19,19,20,20,21,21,22,22,24,24,25,25,30,30],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,-1,20,22,-1,24,26,-1,-1,28,30,32,34,-1,36,38,-1,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,3.2334878E7,1.4400111E10,2.3E1,2.979291E7,1.6451234E-2,-6.289796E-3,-1.295489E-2,2.834008E-1,1.6511278E1,1.5305152E5,2.3542495E-2,2.8129198E7,1.0034782E-3,2.9830588E-3,2.4E1,8.645E3,-1.545761E-3,-7.3201926E-3,1.5353E4,4.2163028E5,1.07715E5,1.7367E5,3.3067344E-3,4.410513E1,1.792E3,1.4216381E-3,6.438302E-3,1.4393159E-2,1.6752116E-3,2.0086452E5,-6.821379E-3,-1.2353949E-2,-6.068525E-3,-2.2119225E-3,-3.64054E-3,7.850287E-4,1.0487227E-2,5.165597E-3,8.510144E-3,3.2613445E-3],"split_indices":[2,12,5,3,53,65,0,0,65,64,41,0,53,47,0,3,2,0,0,2,40,12,1,0,66,10,0,0,0,0,36,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.22E2,8.3E1,3.9E1,3.9E1,4.4E1,3.2E1,7E0,6E0,3.3E1,3.2E1,1.2E1,4E0,2.8E1,2.9E1,4E0,1.6E1,1.6E1,6E0,6E0,9E0,1.9E1,1.8E1,1.1E1,4E0,1.2E1,1E1,6E0,4E0,5E0,4E0,1.5E1,1.2E1,6E0,5E0,6E0,8E0,4E0,4E0,6E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.6734526E-2,-9.1157906E-2,8.694118E-2,-1.14221506E-1,7.1256E-3,-2.2907281E-2,1.6317785E-1,-8.873149E-2,-2.279048E-1,-8.6769335E-2,5.1424676E-3,1.1956994E-1,1.4414875E-2,-5.2002747E-2,-1.5096772E-1,-1.5742633E-2,-9.334152E-4,-3.9147593E-2,-8.773296E-3,1.3988557E-1,2.6922475E-3,-7.4461035E-2,3.8293798E-3,-3.6135365E-3,-1.7604433E-1,-4.029336E-3,2.487474E-4,4.3808026E-3,8.748949E-3,-9.532337E-2,1.2651632E-4,-5.174897E-3,-9.987282E-3,-7.524829E-3,-6.409915E-2,6.843154E-4,-4.1055903E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,21,23,-1,-1,25,-1,27,-1,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1,35,-1,-1],"loss_changes":[9.955602E-1,4.3281657E-1,4.5595992E-1,1.7972505E-1,0E0,2.0115383E-1,1.5740418E-1,1.2472907E-1,2.4793094E-1,6.667022E-2,0E0,2.5494546E-2,0E0,1.1878162E-1,2.6334465E-2,0E0,0E0,2.451122E-2,0E0,2.2822618E-2,0E0,5.3339586E-2,0E0,0E0,1.0927856E-2,0E0,0E0,0E0,0E0,4.0430173E-2,0E0,0E0,0E0,0E0,3.382042E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,17,17,19,19,21,21,24,24,29,29,34,34],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,22,24,-1,-1,26,-1,28,-1,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1,36,-1,-1],"split_conditions":[9.379017E6,1.2491071E2,1E0,1E0,7.1256E-3,2.3430255E5,1.5435694E9,5.070136E7,2.667836E6,1.799984E7,5.1424676E-3,2.7987084E7,1.4414875E-2,1E1,2.5233118E11,-1.5742633E-2,-9.334152E-4,3.0502517E3,-8.773296E-3,6.5857696E8,2.6922475E-3,6.336E3,3.8293798E-3,-3.6135365E-3,6.948189E7,-4.029336E-3,2.487474E-4,4.3808026E-3,8.748949E-3,9.7E1,1.2651632E-4,-5.174897E-3,-9.987282E-3,-7.524829E-3,3.6967509E0,6.843154E-4,-4.1055903E-3],"split_indices":[59,64,109,112,0,41,7,53,40,55,0,53,0,8,39,0,0,60,0,12,0,2,0,0,7,0,0,0,0,10,0,0,0,0,64,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.27E2,7.4E1,5.3E1,6.8E1,6E0,2.2E1,3.1E1,5.7E1,1.1E1,1.5E1,7E0,2.5E1,6E0,3.7E1,2E1,7E0,4E0,1.1E1,4E0,1.8E1,7E0,3.2E1,5E0,6E0,1.4E1,5E0,6E0,1E1,8E0,2.5E1,7E0,6E0,8E0,7E0,1.8E1,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.4631092E-2,-6.4512454E-2,1.9589595E-1,-1.6322353E-1,8.522948E-3,3.033344E-1,1.5148705E-2,-1.3743965E-1,-1.42944595E-2,-5.3837012E-2,8.974279E-2,4.0235493E-1,6.3367966E-3,5.0054556E-3,-4.559638E-3,-7.7281065E-2,-2.0720726E-1,3.050162E-3,-8.134066E-2,5.812929E-2,1.0071612E-2,1.0958666E-2,2.3628693E-2,-5.766415E-3,-1.304942E-2,-6.0893046E-3,-1.2602316E-2,-7.7528576E-3,-5.6673784E-2,7.8318544E-2,-1.2501735E-3,-3.3132632E-3,2.7295176E-3,-3.910432E-3,-3.4619693E-2,3.7098695E-2,5.979444E-3,3.3983766E-4,-2.7984006E-3,-2.2022233E-4,2.9414275E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,-1,27,29,-1,-1,-1,-1,31,-1,-1,-1,33,35,-1,-1,-1,-1,37,39,-1,-1,-1,-1,-1],"loss_changes":[1.3148084E0,7.2954094E-1,4.6146566E-1,1.0975027E-1,3.0365855E-1,2.0007336E-1,1.09196045E-1,1.4347744E-1,0E0,1.1526446E-1,8.732091E-2,3.0263543E-2,0E0,0E0,0E0,5.984354E-2,4.314649E-2,0E0,4.6743765E-2,3.9496407E-2,0E0,0E0,0E0,0E0,4.350995E-2,0E0,0E0,0E0,9.952046E-3,3.014353E-2,0E0,0E0,0E0,0E0,1.4426822E-2,1.1521685E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,18,18,19,19,24,24,28,28,29,29,34,34,35,35],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,-1,28,30,-1,-1,-1,-1,32,-1,-1,-1,34,36,-1,-1,-1,-1,38,40,-1,-1,-1,-1,-1],"split_conditions":[6.160098E3,3.2334878E7,2.692756E6,1.366892E6,7.307441E-1,1.1517007E1,1.063036E8,4.8861527E9,-1.42944595E-2,2.5758067E-1,1.07355234E11,2.2059325E5,6.3367966E-3,5.0054556E-3,-4.559638E-3,1.56967745E1,6.9533944E7,3.050162E-3,1.5698778E-2,1.2378788E1,1.0071612E-2,1.0958666E-2,2.3628693E-2,-5.766415E-3,7.156922E7,-6.0893046E-3,-1.2602316E-2,-7.7528576E-3,2.513E3,2.3145154E0,-1.2501735E-3,-3.3132632E-3,2.7295176E-3,-3.910432E-3,9.069479E-1,9.871107E5,5.979444E-3,3.3983766E-4,-2.7984006E-3,-2.2022233E-4,2.9414275E-3],"split_indices":[60,12,37,56,47,62,59,5,0,35,39,36,0,0,0,64,7,0,46,61,0,0,0,0,7,0,0,0,10,47,0,0,0,0,65,55,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,1E2,2.3E1,4.2E1,5.8E1,1.4E1,9E0,3.7E1,5E0,3.3E1,2.5E1,8E0,6E0,5E0,4E0,2.1E1,1.6E1,6E0,2.7E1,2.1E1,4E0,4E0,4E0,1.2E1,9E0,8E0,8E0,5E0,2.2E1,1.7E1,4E0,5E0,4E0,9E0,1.3E1,1E1,7E0,5E0,8E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.9961428E-2,-5.4760296E-2,1.4808808E-1,-7.625019E-2,9.623012E-3,8.600513E-2,2.5824258E-1,-2.0302469E-2,-1.4030528E-1,-1.1886213E-3,1.10145114E-1,5.486968E-3,1.6287107E-2,-5.807319E-3,1.5328434E-2,-1.7660755E-1,-1.3328822E-3,2.3861977E-3,1.3198578E-1,8.0766276E-2,-5.11313E-2,-8.435207E-2,-1.038536E-2,7.28232E-3,2.4969296E-3,-1.3643861E-4,7.3685804E-3,-3.8897751E-3,1.231897E-3,-1.652856E-3,-5.5982433E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,-1,-1,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1186754E0,4.2288318E-1,2.7185464E-1,2.4442929E-1,0E0,8.037394E-2,1.3885522E-1,1.3902071E-1,1.2563765E-1,0E0,2.4949938E-2,0E0,0E0,0E0,1.3025014E-1,7.2116494E-2,0E0,0E0,1.980561E-2,9.2669815E-2,3.7085205E-2,1.06589645E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,14,14,15,15,18,18,19,19,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,-1,-1,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.307441E-1,7.3153823E3,1.5605432E9,5.070136E7,9.623012E-3,1.2103737E5,2.4190365E6,2.190008E10,4.0916511E3,-1.1886213E-3,1.3924915E0,5.486968E-3,1.6287107E-2,-5.807319E-3,2.70482E5,2.5233118E11,-1.3328822E-3,2.3861977E-3,7.704923E6,2.4662777E1,4.759E3,1.4048359E1,-1.038536E-2,7.28232E-3,2.4969296E-3,-1.3643861E-4,7.3685804E-3,-3.8897751E-3,1.231897E-3,-1.652856E-3,-5.5982433E-3],"split_indices":[47,4,7,53,0,41,40,39,4,0,50,0,0,0,37,39,0,0,55,64,10,62,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,7.3E1,4.2E1,6.8E1,5E0,2.8E1,1.4E1,3.7E1,3.1E1,5E0,2.3E1,6E0,8E0,9E0,2.8E1,2.3E1,8E0,7E0,1.6E1,1.4E1,1.4E1,8E0,1.5E1,1.2E1,4E0,7E0,7E0,1E1,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[1.3729947E-2,-4.566325E-2,1.6775997E-1,-1.3376005E-1,3.25781E-2,2.5375625E-1,2.708105E-3,-1.0375917E-1,-1.6285796E-2,-2.9816868E-2,1.2454164E-1,3.5035E-3,3.131097E-1,7.481601E-2,-6.095098E-3,-1.7269383E-1,-2.9636558E-2,-6.7627415E-2,4.16402E-2,1.616482E-1,5.2620034E-4,3.5998857E-1,6.1416607E-3,1.307427E-3,5.12393E-3,-9.810178E-3,-1.5966771E-3,2.4488638E-3,-5.3069998E-2,-4.5692315E-3,-6.491774E-4,-1.5378329E-3,4.2234017E-3,4.043761E-3,9.153454E-3,8.535334E-3,2.0090915E-2,-9.826024E-4,-3.632274E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,-1,25,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1512023E0,6.3160354E-1,4.9609727E-1,2.319833E-1,2.8487852E-1,2.2329259E-1,1.3172579E-1,1.9415697E-1,0E0,8.3870925E-2,8.246806E-2,0E0,1.0074568E-1,1.0928459E-2,0E0,8.427948E-2,4.0643327E-2,2.9012077E-2,4.085756E-2,1.8431872E-2,0E0,6.785488E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.0386813E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,21,21,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,-1,26,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9446938E3,2.667836E6,1.4046429E1,1.8302105E6,1.0639322E0,4.8E1,2.990383E6,2.2372459E4,-1.6285796E-2,3.181762E5,7.5710654E-1,3.5035E-3,6.2277466E2,4.3258228E7,-6.095098E-3,3.0387878E3,8.9E1,4.3E1,2.1E1,1.7108389E5,5.2620034E-4,2.0086452E5,6.1416607E-3,1.307427E-3,5.12393E-3,-9.810178E-3,-1.5966771E-3,2.4488638E-3,7.4611723E-1,-4.5692315E-3,-6.491774E-4,-1.5378329E-3,4.2234017E-3,4.043761E-3,9.153454E-3,8.535334E-3,2.0090915E-2,-9.826024E-4,-3.632274E-3],"split_indices":[60,40,62,40,50,64,37,59,0,36,35,0,64,53,0,4,10,3,3,36,0,36,0,0,0,0,0,0,35,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.24E2,9E1,3.4E1,4.2E1,4.8E1,2.2E1,1.2E1,3.8E1,4E0,2.9E1,1.9E1,6E0,1.6E1,8E0,4E0,1.9E1,1.9E1,1.9E1,1E1,1.4E1,5E0,1.2E1,4E0,4E0,4E0,1.5E1,4E0,4E0,1.5E1,1.2E1,7E0,4E0,6E0,5E0,9E0,4E0,8E0,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.3829952E-2,-8.865297E-2,8.845288E-2,-1.09529525E-1,1.077035E-2,3.777133E-2,2.4928781E-1,-8.7324575E-2,-1.3060442E-2,5.8910888E-2,-6.4314837E-3,6.2036696E-3,1.4693194E-2,-3.8164314E-2,-1.7329876E-1,2.6504067E-3,1.0226442E-1,-5.7492286E-2,5.557041E-3,-4.441228E-3,-1.0884548E-2,3.1454272E-3,-2.0114973E-2,3.0835316E-4,1.2319631E-1,-5.4072314E-3,-2.0264309E-2,-2.03112E-3,1.3724826E-3,7.4417097E-3,2.3716968E-3,-4.4007335E-2,2.7841914E-3,1.1782341E-3,-7.760308E-2,-5.5477247E-3,-2.2711544E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,19,21,23,25,-1,-1,-1,-1,27,-1,29,-1,31,-1,-1,-1,-1,33,-1,-1,35,-1,-1],"loss_changes":[9.950928E-1,5.1662284E-1,4.3890902E-1,2.3459178E-1,0E0,1.6468428E-1,5.063951E-2,2.6388308E-1,0E0,9.4853476E-2,0E0,0E0,0E0,1.3212976E-1,7.967973E-2,2.7228057E-2,4.3324143E-2,7.654752E-2,0E0,0E0,0E0,0E0,1.6553463E-2,0E0,3.5449415E-2,0E0,4.7333717E-2,0E0,0E0,0E0,0E0,4.575362E-2,0E0,0E0,9.538382E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13,14,14,15,15,16,16,17,17,22,22,24,24,26,26,31,31,34,34],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,20,22,24,26,-1,-1,-1,-1,28,-1,30,-1,32,-1,-1,-1,-1,34,-1,-1,36,-1,-1],"split_conditions":[3.4707712E8,5.4125615E3,1.8830955E0,1E0,1.077035E-2,2.8799975E10,4.4993465E1,4.352552E9,-1.3060442E-2,6.9015497E-1,-6.4314837E-3,6.2036696E-3,1.4693194E-2,1E0,7.157903E7,5.28E2,1.2738525E3,1E0,5.557041E-3,-4.441228E-3,-1.0884548E-2,3.1454272E-3,1.662E3,3.0835316E-4,4.1649513E1,-5.4072314E-3,1.1058695E0,-2.03112E-3,1.3724826E-3,7.4417097E-3,2.3716968E-3,5.2287582E-2,2.7841914E-3,1.1782341E-3,7.303835E0,-5.5477247E-3,-2.2711544E-3],"split_indices":[7,60,47,112,0,5,64,5,0,47,0,0,0,96,7,0,60,8,0,0,0,0,0,0,65,0,49,0,0,0,0,65,0,0,64,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.28E2,7.4E1,5.4E1,7E1,4E0,4.2E1,1.2E1,6.3E1,7E0,3.8E1,4E0,5E0,7E0,4.1E1,2.2E1,1.7E1,2.1E1,3.7E1,4E0,1E1,1.2E1,4E0,1.3E1,4E0,1.7E1,1.4E1,2.3E1,9E0,4E0,1.1E1,6E0,1.8E1,5E0,6E0,1.2E1,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.0414203E-3,-7.878248E-2,8.054004E-2,7.587486E-3,-9.727443E-2,-1.1415311E-2,1.4471501E-1,3.4136884E-2,-1.2113401E-1,2.3151333E-2,-7.6996195E-3,9.105018E-2,2.4445716E-1,5.4257982E-3,-1.8043827E-3,-2.3732725E-1,-7.513824E-2,-2.255222E-2,8.0250815E-2,3.2193463E-3,6.791306E-3,6.0013365E-3,1.3760916E-2,-2.8465027E-1,-4.5472025E-3,-1.5620251E-1,-2.6736716E-2,1.0156436E-3,-2.5118946E-3,6.030733E-3,4.2817096E-4,-8.079432E-3,-1.7492676E-2,-4.7277887E-3,-1.0314373E-2,5.007895E-3,-5.679665E-2,-3.5121858E-3,-1.2767134E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,-1,7,9,11,13,15,17,-1,19,21,-1,-1,23,25,27,29,-1,-1,-1,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,37,-1,-1],"loss_changes":[7.927815E-1,2.967217E-1,3.616271E-1,0E0,1.9226032E-1,1.3837187E-1,1.704374E-1,6.145151E-2,2.5473183E-1,5.9222117E-2,0E0,2.0645484E-2,2.6068568E-2,0E0,0E0,6.704885E-2,1.4492175E-1,1.8233925E-2,3.1679794E-2,0E0,0E0,0E0,0E0,3.2298803E-2,0E0,2.6490748E-2,1.04852326E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.8837221E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,15,15,16,16,17,17,18,18,23,23,25,25,26,26,36,36],"right_children":[2,4,6,-1,8,10,12,14,16,18,-1,20,22,-1,-1,24,26,28,30,-1,-1,-1,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,38,-1,-1],"split_conditions":[2.3724032E8,7.5E0,8.8579214E-1,7.587486E-3,2.5758067E-1,5.5E1,1.5605432E9,7.45658E5,2.3E1,9.7752365E1,-7.6996195E-3,2.3145154E0,2.2059325E5,5.4257982E-3,-1.8043827E-3,3.2316522E7,5.981675E-4,6.381E3,1.650269E10,3.2193463E-3,6.791306E-3,6.0013365E-3,1.3760916E-2,1.2903E4,-4.5472025E-3,4.5E2,2.1270042E1,1.0156436E-3,-2.5118946E-3,6.030733E-3,4.2817096E-4,-8.079432E-3,-1.7492676E-2,-4.7277887E-3,-1.0314373E-2,5.007895E-3,3.9214565E1,-3.5121858E-3,-1.2767134E-4],"split_indices":[7,61,50,0,35,3,7,1,3,64,0,47,36,0,0,12,50,10,5,0,0,0,0,9,0,0,66,0,0,0,0,0,0,0,0,0,66,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,6.3E1,6E1,4E0,5.9E1,2.5E1,3.5E1,9E0,5E1,2.1E1,4E0,2.4E1,1.1E1,4E0,5E0,1.3E1,3.7E1,1.2E1,9E0,1.8E1,6E0,4E0,7E0,9E0,4E0,1.3E1,2.4E1,5E0,7E0,5E0,4E0,5E0,4E0,8E0,5E0,4E0,2E1,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.1345742E-2,-1.5183492E-2,2.1139954E-1,-1.5795894E-1,1.0086736E-2,3.3098742E-1,-3.34566E-3,-4.169932E-3,-8.698029E-3,-7.7102296E-2,4.246941E-2,1.9742193E-2,9.893028E-3,-9.987715E-2,1.622422E-3,1.1202803E-1,-1.6018717E-3,-7.408665E-2,-8.00354E-3,8.514822E-2,9.928343E-3,-6.072125E-2,3.0126434E-2,-4.5086797E-3,-1.5075676E-3,1.1998349E-1,1.8818754E-3,-4.5665596E-3,-1.6098467E-3,4.422938E-3,1.5852982E-2,1.957132E-3,7.371841E-3,-5.498181E-4,2.109813E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,-1,-1,17,-1,19,21,23,-1,25,-1,27,29,-1,-1,31,-1,-1,-1,-1,33,-1,-1,-1,-1],"loss_changes":[6.3477534E-1,3.840379E-1,4.9440938E-1,1.1419088E-2,2.6020512E-1,1.6406178E-2,0E0,0E0,0E0,6.663594E-2,2.0557047E-1,0E0,0E0,2.425684E-2,0E0,4.929632E-2,8.061114E-2,1.1709295E-2,0E0,3.032823E-2,0E0,1.0917395E-2,2.350525E-2,0E0,0E0,2.6265845E-2,0E0,0E0,0E0,0E0,1.911209E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,13,13,15,15,16,16,17,17,19,19,21,21,22,22,25,25,30,30],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,-1,-1,18,-1,20,22,24,-1,26,-1,28,30,-1,-1,32,-1,-1,-1,-1,34,-1,-1,-1,-1],"split_conditions":[1.0825864E4,5.981675E-4,1.073818E10,1.0032363E0,1.4861879E3,2.5501368E9,-3.34566E-3,-4.169932E-3,-8.698029E-3,3.4653E4,2.70482E5,1.9742193E-2,9.893028E-3,5.895365E5,1.622422E-3,1.1104E4,8.388E3,1.0629717E1,-8.00354E-3,1.2257767E1,9.928343E-3,7.17E2,4.2202312E1,-4.5086797E-3,-1.5075676E-3,2.6078947E1,1.8818754E-3,-4.5665596E-3,-1.6098467E-3,4.422938E-3,7.7628E6,1.957132E-3,7.371841E-3,-5.498181E-4,2.109813E-3],"split_indices":[60,50,12,66,4,7,0,0,0,9,37,0,0,36,0,2,2,61,0,62,0,0,64,0,0,64,0,0,0,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.18E2,1.05E2,1.3E1,1.5E1,9E1,9E0,4E0,5E0,1E1,2.4E1,6.6E1,4E0,5E0,2E1,4E0,2.5E1,4.1E1,1.6E1,4E0,2.1E1,4E0,1.4E1,2.7E1,1E1,6E0,1.1E1,1E1,5E0,9E0,4E0,2.3E1,4E0,7E0,1.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.8874965E-3,-5.4012496E-2,1.19606234E-1,1.0388618E-2,-7.1630165E-2,-1.4794766E-3,1.3823333E-1,-4.994655E-2,-1.2991682E-2,2.7595446E-3,1.7419717E-1,-1.2555449E-1,-3.0685738E-3,1.0813551E-1,1.2652921E-2,-9.0983495E-2,-1.2061275E-2,-3.9176367E-2,7.4524455E-2,6.541325E-3,2.65589E-3,-6.230874E-2,-7.500915E-3,-5.6585707E-2,2.826607E-3,6.2378654E-3,7.702095E-4,-3.4879875E-5,-3.8778435E-3,-9.4374164E-4,-3.300594E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,-1,7,-1,9,11,-1,-1,13,15,17,19,-1,21,-1,23,25,-1,-1,27,-1,29,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.702092E-1,3.8873178E-1,1.10871375E-1,0E0,3.0798477E-1,0E0,8.830631E-2,2.4063033E-1,0E0,0E0,1.092335E-1,9.117633E-2,1.2339325E-1,1.5100792E-2,0E0,3.190045E-2,0E0,5.472267E-2,4.3752305E-2,0E0,0E0,1.971642E-2,0E0,1.0627776E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,10,10,11,11,12,12,13,13,15,15,17,17,18,18,21,21,23,23],"right_children":[2,4,6,-1,8,-1,10,12,-1,-1,14,16,18,20,-1,22,-1,24,26,-1,-1,28,-1,30,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,6.371407E0,1.2103737E5,1.0388618E-2,1.2511909E8,-1.4794766E-3,1.7915075E0,3.572374E6,-1.2991682E-2,2.7595446E-3,1.2750684E9,1.5088414E6,3.9219662E5,9.641E3,1.2652921E-2,3.91028E5,-1.2061275E-2,8.18961E3,8E0,6.541325E-3,2.65589E-3,3.2E1,-7.500915E-3,4.0605398E6,2.826607E-3,6.2378654E-3,7.702095E-4,-3.4879875E-5,-3.8778435E-3,-9.4374164E-4,-3.300594E-3],"split_indices":[47,62,41,0,53,0,50,12,0,0,7,36,36,2,0,12,0,4,8,0,0,3,0,40,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.14E2,7.7E1,3.7E1,4E0,7.3E1,4E0,3.3E1,6.7E1,6E0,1.1E1,2.2E1,2.5E1,4.2E1,1.4E1,8E0,2.1E1,4E0,2.9E1,1.3E1,8E0,6E0,1.6E1,5E0,2.5E1,4E0,6E0,7E0,4E0,1.2E1,7E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.9425756E-3,-1.905174E-2,1.9297548E-1,-9.370549E-2,2.2946702E-2,3.0794173E-1,-3.6081297E-3,-1.582578E-1,-2.0823821E-2,8.688966E-2,-2.1281337E-2,1.8515596E-2,9.080877E-3,-2.1206818E-3,-2.1614687E-1,5.4630887E-2,-9.419615E-2,3.97801E-3,1.5848123E-1,-5.473064E-2,2.4500031E-2,-2.7210742E-1,-3.4474938E-3,2.0243535E-4,4.1796346E-3,-6.0857884E-3,-1.6940742E-3,-3.164799E-2,3.5547523E-3,3.2082796E-3,9.427875E-3,-3.1087719E-2,-5.753654E-3,3.3653707E-3,6.681297E-5,-7.2869197E-3,-1.6104063E-2,-3.221712E-3,8.4323826E-4,-2.0696581E-3,5.6730025E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5529815E-1,3.3136106E-1,4.5546812E-1,1.7548224E-1,1.94158E-1,2.023524E-2,0E0,1.22217536E-1,1.1033547E-1,1.6557126E-1,6.412645E-2,0E0,0E0,0E0,8.5084856E-2,1.6202038E-2,1.4997229E-2,3.749682E-2,4.005751E-2,3.3561707E-2,1.9016087E-2,1.857847E-2,0E0,0E0,0E0,0E0,0E0,1.884859E-2,0E0,0E0,0E0,1.0515049E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,27,27,31,31],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.669896E3,3.2334878E7,1.073818E10,3.0288889E1,1.4163358E7,6.4417176E-2,-3.6081297E-3,6.9237E4,4.171707E9,1.1191123E3,4.867052E6,1.8515596E-2,9.080877E-3,-2.1206818E-3,2.667836E6,3.1749377E-1,1.2775832E1,2.990621E5,2.1146134E5,2.5233118E11,1.11E3,7.444956E7,-3.4474938E-3,2.0243535E-4,4.1796346E-3,-6.0857884E-3,-1.6940742E-3,2.83E2,3.5547523E-3,3.2082796E-3,9.427875E-3,2.2539987E8,-5.753654E-3,3.3653707E-3,6.681297E-5,-7.2869197E-3,-1.6104063E-2,-3.221712E-3,8.4323826E-4,-2.0696581E-3,5.6730025E-4],"split_indices":[60,12,12,64,53,65,0,12,5,60,1,0,0,0,40,35,62,36,36,39,0,7,0,0,0,0,0,0,0,0,0,40,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,1.04E2,1.3E1,3.7E1,6.7E1,9E0,4E0,1.9E1,1.8E1,2.7E1,4E1,4E0,5E0,7E0,1.2E1,9E0,9E0,1.3E1,1.4E1,2.3E1,1.7E1,8E0,4E0,4E0,5E0,5E0,4E0,9E0,4E0,5E0,9E0,1.8E1,5E0,5E0,1.2E1,4E0,4E0,5E0,4E0,1.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.715182E-4,-3.1160966E-2,2.4982548E-1,-1.10963106E-1,2.3648625E-2,1.515712E-2,6.539328E-3,-1.3560817E-1,2.35147E-3,8.515787E-2,-1.7920602E-2,-1.143811E-1,-1.2682635E-2,-1.299645E-2,1.4943399E-1,-6.9780365E-2,1.667696E-2,-5.930143E-2,-1.5352269E-1,2.1469516E-3,-2.904249E-3,3.899332E-3,9.580715E-3,-4.9285023E-3,-1.7224307E-3,-1.890571E-2,4.0097736E-2,-4.631485E-3,-2.3227982E-2,-3.92591E-3,-1.0071082E-2,9.076326E-4,-2.562907E-3,2.9587504E-4,3.1413399E-3,5.571424E-4,-2.3131103E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,-1,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.853867E-1,5.0562525E-1,6.331563E-2,1.9224137E-1,1.7795162E-1,0E0,0E0,8.2728565E-2,0E0,1.7859352E-1,7.6521024E-2,7.0664376E-2,0E0,3.598707E-2,4.2596847E-2,1.5431762E-2,2.2425793E-2,2.0854011E-2,6.8568856E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5515603E-2,1.3522284E-2,0E0,9.360904E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,25,25,26,26,28,28],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,-1,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.669896E3,3.2334878E7,3.0376984E1,1E0,1.5046106E7,1.515712E-2,6.539328E-3,6.828667E-2,2.35147E-3,1.1191123E3,2.074635E-1,6.3997424E-1,-1.2682635E-2,2.4E1,3.077573E5,1.3045E4,4.867052E6,1.46E2,2.3737179E1,2.1469516E-3,-2.904249E-3,3.899332E-3,9.580715E-3,-4.9285023E-3,-1.7224307E-3,1.6E1,2.119303E6,-4.631485E-3,3.863112E7,-3.92591E-3,-1.0071082E-2,9.076326E-4,-2.562907E-3,2.9587504E-4,3.1413399E-3,5.571424E-4,-2.3131103E-3],"split_indices":[60,12,65,96,53,0,0,46,0,60,47,35,0,3,36,9,1,10,64,0,0,0,0,0,0,8,40,0,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.27E2,1.14E2,1.3E1,4.6E1,6.8E1,7E0,6E0,4E1,6E0,2.7E1,4.1E1,3.6E1,4E0,1.1E1,1.6E1,1.6E1,2.5E1,1.6E1,2E1,5E0,6E0,8E0,8E0,7E0,9E0,1E1,1.5E1,7E0,9E0,1E1,1E1,5E0,5E0,7E0,8E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-3.0058022E-3,-5.537523E-2,1.2363715E-1,-1.1897155E-1,1.24668535E-2,2.7526972E-1,6.289778E-2,-9.473306E-2,-1.40373055E-2,-4.4215795E-2,5.1229194E-2,7.5524254E-3,1.7229114E-2,8.587542E-2,-1.1502523E-4,-1.5665269E-2,-1.5008669E-1,1.471115E-4,-5.7949033E-2,1.0016496E-2,1.1616786E-1,4.7866423E-3,1.0872991E-3,3.0996075E-3,-5.1646043E-2,-3.502818E-3,-1.7737088E-1,-4.052554E-3,-9.0343517E-4,2.6172674E-3,-3.5883985E-2,2.2903574E-3,7.3377183E-3,-4.171226E-3,1.0954322E-3,-1.0863473E-2,-5.136688E-3,2.7808145E-4,-3.3627518E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,23,25,-1,27,29,31,-1,-1,-1,33,-1,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.1597716E-1,3.7684533E-1,3.142485E-1,1.6299081E-1,9.6742675E-2,4.3087482E-2,4.0722653E-2,1.7709312E-1,0E0,1.1870053E-2,6.842015E-2,0E0,0E0,1.5835136E-2,0E0,5.5533E-2,3.724116E-2,0E0,1.3098963E-2,3.691467E-2,1.8195778E-2,0E0,0E0,0E0,3.661482E-2,0E0,3.4996152E-2,0E0,0E0,0E0,1.33712655E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,15,15,16,16,18,18,19,19,20,20,24,24,26,26,30,30],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,24,26,-1,28,30,32,-1,-1,-1,34,-1,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3024514E3,3.2936362E6,1.3453537E-1,9.5270414E4,5.3654873E-1,8.328308E3,8.442486E-1,3.925844E7,-1.40373055E-2,8.272727E0,3.208219E1,7.5524254E-3,1.7229114E-2,2.1897722E10,-1.1502523E-4,7.690597E-4,5.432393E-1,1.471115E-4,3.301091E-2,2.6E1,3.077573E5,4.7866423E-3,1.0872991E-3,3.0996075E-3,2.6666668E-1,-3.502818E-3,8.8132757E-1,-4.052554E-3,-9.0343517E-4,2.6172674E-3,2.2659668E-1,2.2903574E-3,7.3377183E-3,-4.171226E-3,1.0954322E-3,-1.0863473E-2,-5.136688E-3,2.7808145E-4,-3.3627518E-3],"split_indices":[60,58,65,41,49,4,35,53,0,62,64,0,0,5,0,47,35,0,46,3,36,0,0,0,65,0,35,0,0,0,46,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,8.6E1,3.5E1,4.4E1,4.2E1,9E0,2.6E1,4E1,4E0,1.7E1,2.5E1,5E0,4E0,1.9E1,7E0,1.7E1,2.3E1,4E0,1.3E1,1.6E1,9E0,1.5E1,4E0,5E0,1.2E1,7E0,1.6E1,7E0,6E0,8E0,8E0,4E0,5E0,8E0,4E0,8E0,8E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.1784698E-2,-4.2694695E-2,1.0313324E-1,-9.150875E-2,7.949537E-3,1.4991245E-1,-6.336271E-3,-1.1589388E-1,2.0059671E-2,4.5299232E-2,-5.5373605E-2,1.5106503E-2,8.611438E-2,-9.570322E-2,-1.17560215E-2,3.7404408E-3,-1.5203897E-3,-2.2941418E-2,9.964982E-2,-7.437559E-2,-9.5269765E-4,4.9280155E-2,8.215481E-3,-5.1991593E-2,-1.5255722E-1,1.751519E-3,-2.2901583E-3,7.2418344E-3,3.0811045E-2,-1.3377668E-3,-4.9423887E-3,3.5527763E-3,-5.1585207E-4,-7.910854E-2,1.0168018E-3,-8.658434E-3,-3.5546059E-3,3.0724222E-3,-4.2383344E-4,-1.0319292E-1,-3.1875865E-5,-2.0308674E-3,-5.9902044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,-1,-1,-1,25,27,29,-1,31,-1,33,35,-1,-1,-1,37,-1,-1,-1,-1,39,-1,-1,-1,-1,-1,41,-1,-1,-1],"loss_changes":[4.5196947E-1,2.4794249E-1,3.1772843E-1,1.4107594E-1,1.20763384E-1,2.1694112E-1,0E0,8.69115E-2,3.272946E-2,1.20971516E-1,1.1305936E-2,0E0,4.848413E-2,8.6505204E-2,0E0,0E0,0E0,2.4126967E-2,5.974841E-2,1.32995695E-2,0E0,2.192714E-2,0E0,4.739007E-2,2.2437394E-2,0E0,0E0,0E0,1.2544697E-2,0E0,0E0,0E0,0E0,3.2047585E-2,0E0,0E0,0E0,0E0,0E0,1.2205049E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,17,17,18,18,19,19,21,21,23,23,24,24,28,28,33,33,39,39],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,-1,-1,-1,26,28,30,-1,32,-1,34,36,-1,-1,-1,38,-1,-1,-1,-1,40,-1,-1,-1,-1,-1,42,-1,-1,-1],"split_conditions":[4.8360786E3,3.851071E7,9.302862E7,1.2887158E1,2.7260774E7,6.691E3,-6.336271E-3,8.8945635E-2,2.8E2,1.1467087E3,1.2663E4,1.5106503E-2,6.9673E4,4.8861527E9,-1.17560215E-2,3.7404408E-3,-1.5203897E-3,5.3149715E6,1.4922581E1,1.7761682E8,-9.5269765E-4,1.5605432E9,8.215481E-3,1.2809688E3,3.0595828E3,1.751519E-3,-2.2901583E-3,7.2418344E-3,8.5731603E8,-1.3377668E-3,-4.9423887E-3,3.5527763E-3,-5.1585207E-4,2.3396162E5,1.0168018E-3,-8.658434E-3,-3.5546059E-3,3.0724222E-3,-4.2383344E-4,2.230075E3,-3.1875865E-5,-2.0308674E-3,-5.9902044E-3],"split_indices":[60,12,53,61,53,9,0,46,0,60,10,0,2,5,0,0,0,40,62,12,0,7,0,60,4,0,0,0,5,0,0,0,0,55,0,0,0,0,0,55,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,9.9E1,2.6E1,5E1,4.9E1,2.2E1,4E0,4.1E1,9E0,3.1E1,1.8E1,5E0,1.7E1,3.7E1,4E0,4E0,5E0,1.4E1,1.7E1,1.1E1,7E0,1.3E1,4E0,2.2E1,1.5E1,4E0,1E1,9E0,8E0,5E0,6E0,9E0,4E0,1.6E1,6E0,1E1,5E0,4E0,4E0,1.2E1,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-4.729807E-3,-5.048992E-2,9.775282E-2,-1.1888584E-1,-4.798555E-3,1.536164E-2,6.194967E-2,-1.1205685E-2,-9.08371E-2,3.931228E-2,-6.2125795E-2,9.015847E-2,-5.716163E-3,-6.662216E-3,-4.257724E-2,-1.3091415E-3,6.482192E-2,-9.364309E-2,-1.9230282E-2,3.465947E-2,1.6006798E-1,-3.885128E-3,-8.384457E-4,1.15984425E-1,2.0479847E-2,-1.9777098E-3,-6.327615E-3,1.1043444E-3,-2.231982E-3,-2.6524996E-3,8.438858E-2,9.423418E-3,2.9131325E-3,2.113784E-3,7.470263E-3,1.9241504E-3,-9.205335E-4,7.3403227E-3,1.4574961E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,-1,-1,21,-1,23,25,27,29,31,-1,-1,33,35,-1,-1,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7706875E-1,2.650633E-1,2.8470814E-1,8.907616E-2,1.3393745E-1,0E0,1.8500474E-1,0E0,6.2169835E-2,5.284081E-2,2.9294074E-2,1.1084935E-1,0E0,0E0,1.4094586E-2,0E0,4.7535002E-2,2.0894647E-2,1.3895858E-2,8.542838E-2,3.5941213E-2,0E0,0E0,2.2080198E-2,1.1015167E-2,0E0,0E0,0E0,0E0,0E0,4.011569E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,14,14,16,16,17,17,18,18,19,19,20,20,23,23,24,24,30,30],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,-1,-1,22,-1,24,26,28,30,32,-1,-1,34,36,-1,-1,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1855E4,1.04856936E8,1.4440433E-2,1.458E3,2.7260774E7,1.536164E-2,2.192772E10,-1.1205685E-2,1.4499603E4,3.33113E5,3.5904072E3,9.669896E3,-5.716163E-3,-6.662216E-3,1.81E2,-1.3091415E-3,1.1247059E1,2.0822755E9,9.513889E-1,1.0768721E1,7.9684106E9,-3.885128E-3,-8.384457E-4,8.2472925E0,2.5E1,-1.9777098E-3,-6.327615E-3,1.1043444E-3,-2.231982E-3,-2.6524996E-3,1.8783619E5,9.423418E-3,2.9131325E-3,2.113784E-3,7.470263E-3,1.9241504E-3,-9.205335E-4,7.3403227E-3,1.4574961E-3],"split_indices":[2,7,65,2,53,0,5,0,59,1,4,60,0,0,10,0,62,5,65,62,12,0,0,61,8,0,0,0,0,0,41,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,8.4E1,3.7E1,3.3E1,5.1E1,4E0,3.3E1,5E0,2.8E1,2.9E1,2.2E1,2.9E1,4E0,1.3E1,1.5E1,8E0,2.1E1,1.2E1,1E1,1.7E1,1.2E1,5E0,1E1,9E0,1.2E1,6E0,6E0,4E0,6E0,6E0,1.1E1,8E0,4E0,4E0,5E0,8E0,4E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.25158E-3,-2.6561212E-2,1.0965885E-1,-4.8485763E-2,5.0385475E-2,1.7374676E-1,-2.6866705E-3,-5.9695866E-2,2.591387E-3,8.717021E-2,2.313918E-2,1.4774739E-2,1.1695447E-1,-1.0114739E-1,-3.0640189E-2,6.0749487E-3,1.4184045E-3,-2.889158E-4,2.5402978E-3,2.325769E-3,8.14681E-3,-6.9224745E-2,-6.587683E-3,-4.920239E-2,2.5985343E-3,-5.085927E-4,-4.1275597E-3,-1.0100856E-3,-8.2504585E-2,1.3182738E-3,-2.0859079E-3,-6.126978E-3,-2.080876E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,21,23,-1,-1,-1,-1,-1,-1,25,-1,27,29,-1,-1,-1,31,-1,-1,-1,-1],"loss_changes":[3.804665E-1,1.660841E-1,2.8623822E-1,9.0777114E-2,2.0032406E-2,1.1016667E-1,0E0,8.000457E-2,0E0,1.6929418E-2,1.2602861E-2,0E0,4.6604425E-2,2.333945E-2,2.604118E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.6210817E-2,0E0,2.3478009E-2,1.9720113E-2,0E0,0E0,0E0,1.5802711E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,14,14,21,21,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,22,24,-1,-1,-1,-1,-1,-1,26,-1,28,30,-1,-1,-1,32,-1,-1,-1,-1],"split_conditions":[4.3977373E3,7.818375E-1,1.0590909E1,1E0,1.1180212E1,1.4440433E-2,-2.6866705E-3,3.572374E6,2.591387E-3,2.2087815E3,7.6781285E6,1.4774739E-2,8.351458E3,7.157903E7,3.181762E5,6.0749487E-3,1.4184045E-3,-2.889158E-4,2.5402978E-3,2.325769E-3,8.14681E-3,1E0,-6.587683E-3,2.1924436E5,8.3956606E5,-5.085927E-4,-4.1275597E-3,-1.0100856E-3,1.9452468E-2,1.3182738E-3,-2.0859079E-3,-6.126978E-3,-2.080876E-3],"split_indices":[60,47,61,96,62,65,0,12,0,4,53,0,60,7,36,0,0,0,0,0,0,8,0,36,36,0,0,0,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,9.6E1,2.5E1,7.5E1,2.1E1,1.8E1,7E0,6.8E1,7E0,8E0,1.3E1,4E0,1.4E1,2.7E1,4.1E1,4E0,4E0,7E0,6E0,7E0,7E0,1.6E1,1.1E1,2.6E1,1.5E1,4E0,1.2E1,1.5E1,1.1E1,1E1,5E0,4E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[5.4570036E-3,-3.5273395E-2,1.11914314E-1,-2.0278567E-2,-1.1429266E-2,1.4550084E-1,2.5569315E-3,-3.9258465E-2,7.8769075E-3,3.1817567E-3,1.6598582E-1,-2.9588932E-2,-7.1038203E-3,9.818055E-3,4.09297E-3,6.516288E-3,-5.7046935E-2,-2.9981267E-2,6.9843985E-2,-9.066072E-2,-2.3895906E-2,-3.5770577E-3,-3.352412E-4,1.4257273E-3,5.7241037E-3,-5.5301483E-3,-1.8376831E-3,-2.3478274E-3,2.7888032E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,15,-1,-1,-1,17,19,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.329455E-1,2.6745796E-1,5.8869153E-2,3.0176777E-1,0E0,2.2872895E-2,0E0,7.712087E-2,0E0,0E0,3.324485E-2,7.1750894E-2,0E0,0E0,0E0,7.60558E-2,4.352118E-2,2.1134365E-2,2.0423163E-2,2.1777749E-2,1.681576E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,11,11,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,16,-1,-1,-1,18,20,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0230335E0,1.4159094E8,2.326E4,5.528599E3,-1.1429266E-2,3.97E2,2.5569315E-3,1.5088414E6,7.8769075E-3,3.1817567E-3,1.890618E6,2.8399336E7,-7.1038203E-3,9.818055E-3,4.09297E-3,3.9219662E5,5.127381E1,2.74E2,7.925698E5,1.3350591E-1,2.9457063E3,-3.5770577E-3,-3.352412E-4,1.4257273E-3,5.7241037E-3,-5.5301483E-3,-1.8376831E-3,-2.3478274E-3,2.7888032E-4],"split_indices":[47,53,9,60,0,0,0,36,0,0,37,53,0,0,0,36,64,0,36,65,60,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,8.8E1,3.3E1,8.3E1,5E0,2E1,1.3E1,7.6E1,7E0,5E0,1.5E1,7.1E1,5E0,9E0,6E0,3.1E1,4E1,2E1,1.1E1,1.9E1,2.1E1,6E0,1.4E1,7E0,4E0,1.2E1,7E0,1.1E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-8.0780545E-3,-3.4752574E-2,1.2969962E-1,-2.0583535E-2,-1.0799269E-2,8.5350506E-2,9.545129E-3,-3.830769E-2,8.66243E-3,5.684579E-3,2.1591166E-3,-1.1354176E-2,-2.6266783E-2,-9.589336E-2,-8.343153E-4,-3.306402E-2,-7.1984637E-3,-4.428214E-2,2.1414205E-2,-3.1378667E-3,5.0277857E-4,1.3138035E-2,-7.80321E-2,3.3515573E-2,-2.1781574E-3,-1.5103592E-3,2.6397351E-3,-4.598028E-3,-1.2383289E-3,-1.5908232E-2,4.769828E-2,-2.2600642E-3,8.925558E-4,1.8520623E-2,1.01283774E-1,-2.6229053E-4,2.0574925E-3,6.189675E-3,2.2463761E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,-1,-1,13,15,17,19,-1,21,23,-1,-1,25,27,29,-1,-1,-1,-1,-1,31,33,-1,-1,35,37,-1,-1,-1,-1],"loss_changes":[4.5287612E-1,2.7464423E-1,4.518038E-2,3.541835E-1,0E0,1.3545938E-2,0E0,2.0988676E-1,0E0,0E0,0E0,0E0,1.522067E-1,7.426156E-2,6.2813856E-2,1.7766364E-2,0E0,4.365961E-2,3.64049E-2,0E0,0E0,1.8723903E-2,1.0814063E-2,2.669445E-2,0E0,0E0,0E0,0E0,0E0,1.0666855E-2,4.3465637E-2,0E0,0E0,1.2032709E-2,9.268343E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,12,12,13,13,14,14,15,15,17,17,18,18,21,21,22,22,23,23,29,29,30,30,33,33,34,34],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,-1,-1,14,16,18,20,-1,22,24,-1,-1,26,28,30,-1,-1,-1,-1,-1,32,34,-1,-1,36,38,-1,-1,-1,-1],"split_conditions":[1.7618678E0,1.5081978E8,1.286E4,8.351458E3,-1.0799269E-2,9.136126E0,9.545129E-3,2.7571955E6,8.66243E-3,5.684579E-3,2.1591166E-3,-1.1354176E-2,2.09197E4,6.9237E4,1.6777216E8,1.3369197E-5,-7.1984637E-3,7.157903E7,9.014471E-1,-3.1378667E-3,5.0277857E-4,6.275E3,2.9290114E7,1.5493506E1,-2.1781574E-3,-1.5103592E-3,2.6397351E-3,-4.598028E-3,-1.2383289E-3,9.866438E0,1.2597537E7,-2.2600642E-3,8.925558E-4,7.331134E-2,9.574368E0,-2.6229053E-4,2.0574925E-3,6.189675E-3,2.2463761E-3],"split_indices":[47,53,2,60,0,61,0,53,0,0,0,0,59,12,7,46,0,7,35,0,0,2,40,64,0,0,0,0,0,61,40,0,0,46,61,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,1.02E2,1.9E1,9.6E1,6E0,1.3E1,6E0,8.9E1,7E0,6E0,7E0,4E0,8.5E1,2.2E1,6.3E1,1.1E1,1.1E1,2.1E1,4.2E1,6E0,5E0,8E0,1.3E1,3.6E1,6E0,4E0,4E0,9E0,4E0,8E0,2.8E1,4E0,4E0,1.9E1,9E0,1E1,9E0,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.2890033E-2,2.3540517E-2,-1.090092E-2,5.6333947E-3,2.0562984E-1,-2.2239566E-2,6.543178E-2,1.5189269E-2,3.709328E-3,-9.300091E-3,-6.6484567E-3,9.519585E-2,1.8237982E-2,-2.0231966E-2,4.019625E-3,5.7347853E-2,8.307297E-3,-7.791098E-4,2.2059507E-3,2.9045835E-2,-4.086931E-2,4.1297023E-4,3.8603463E-3,6.182676E-3,-8.891635E-3,-2.050276E-2,-7.268353E-2,-2.0231307E-3,1.4628766E-3,1.1895241E-3,-3.4663334E-2,-4.705936E-3,-6.288122E-4,-3.6181859E-3,-1.3172754E-2,3.5570405E-4,-2.1655627E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,17,19,-1,21,-1,-1,-1,23,25,-1,-1,-1,27,29,31,-1,-1,-1,33,-1,-1,-1,35,-1,-1],"loss_changes":[3.0330223E-1,3.6403954E-1,0E0,1.7313126E-1,1.1915326E-1,1.061707E-1,4.461825E-2,0E0,0E0,6.811909E-2,0E0,4.963799E-2,1.4275407E-2,6.11879E-2,0E0,1.6457345E-2,0E0,0E0,0E0,6.972248E-2,2.5723517E-2,0E0,0E0,0E0,1.981838E-2,1.8225513E-2,2.3324288E-2,0E0,0E0,0E0,1.7550223E-2,0E0,0E0,0E0,1.02892835E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,12,12,13,13,15,15,19,19,20,20,24,24,25,25,26,26,30,30,34,34],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,18,20,-1,22,-1,-1,-1,24,26,-1,-1,-1,28,30,32,-1,-1,-1,34,-1,-1,-1,36,-1,-1],"split_conditions":[1.5081978E8,1.2769175E4,-1.090092E-2,9.2954946E-1,1.3477259E7,1.5088414E6,1.5046106E7,1.5189269E-2,3.709328E-3,1E0,-6.6484567E-3,4.4993465E1,7.94867E5,4.029937E-1,4.019625E-3,2.4221443E-1,8.307297E-3,-7.791098E-4,2.2059507E-3,3.8051785E1,8.5154843E-1,4.1297023E-4,3.8603463E-3,6.182676E-3,1.76E3,7.7E2,9.4812524E-1,-2.0231307E-3,1.4628766E-3,1.1895241E-3,8.72606E4,-4.705936E-3,-6.288122E-4,-3.6181859E-3,4.9562156E-1,3.5570405E-4,-2.1655627E-3],"split_indices":[53,60,0,47,55,36,53,0,0,96,0,64,37,35,0,46,0,0,0,64,35,0,0,0,10,11,35,0,0,0,41,0,0,0,50,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,1.11E2,4E0,1.02E2,9E0,7E1,3.2E1,4E0,5E0,6.4E1,6E0,1.9E1,1.3E1,5.8E1,6E0,1.4E1,5E0,6E0,7E0,1.7E1,4.1E1,5E0,9E0,4E0,1.3E1,2.6E1,1.5E1,7E0,6E0,6E0,2E1,1E1,5E0,6E0,1.4E1,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.542461E-3,5.5450644E-2,-4.564126E-2,2.6661737E-2,2.252217E-1,-1.1145994E-1,5.4010195E-3,-4.6813056E-2,5.538511E-2,1.5000484E-2,4.360093E-3,-7.6434344E-2,-1.2287396E-2,-2.3671871E-2,7.627075E-2,-3.3610833E-3,9.125409E-4,1.292235E-1,5.913134E-3,-1.24035865E-1,-4.251322E-4,-6.397319E-2,3.285375E-5,4.877542E-3,1.1848309E-3,2.906024E-3,7.391341E-3,3.9209817E-3,-1.7239496E-2,-7.251035E-3,-2.945089E-3,-4.9571693E-3,-1.6171162E-4,2.3275337E-3,-1.7363895E-2,-2.1119022E-3,1.8715081E-3,-2.386954E-3,3.0136635E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,-1,-1,25,27,29,-1,31,33,-1,-1,-1,-1,-1,35,-1,-1,-1,-1,-1,37,-1,-1,-1,-1],"loss_changes":[3.2443085E-1,3.0190915E-1,2.1561226E-1,1.18667156E-1,7.845113E-2,1.23352885E-1,7.803982E-2,2.6200812E-2,1.4535314E-1,0E0,0E0,7.604723E-2,0E0,2.5796056E-2,1.2105927E-2,0E0,0E0,1.8529594E-2,4.5515332E-2,1.4907107E-2,0E0,2.377852E-2,1.6097613E-2,0E0,0E0,0E0,0E0,0E0,3.218422E-2,0E0,0E0,0E0,0E0,0E0,1.1126173E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,17,17,18,18,19,19,21,21,22,22,28,28,34,34],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,-1,-1,26,28,30,-1,32,34,-1,-1,-1,-1,-1,36,-1,-1,-1,-1,-1,38,-1,-1,-1,-1],"split_conditions":[9.909091E0,1.3127055E4,2.667836E6,1.0154962E3,3.0376984E1,1.1755388E-1,1.3137E4,3.0464E4,1.5046106E7,1.5000484E-2,4.360093E-3,8.0168776E-2,-1.2287396E-2,3.1570956E-2,1.6135693E1,-3.3610833E-3,9.125409E-4,2.6446635E8,1.6496868E6,8.670369E-1,-4.251322E-4,6.114E3,2.4E1,4.877542E-3,1.1848309E-3,2.906024E-3,7.391341E-3,3.9209817E-3,1.9649E4,-7.251035E-3,-2.945089E-3,-4.9571693E-3,-1.6171162E-4,2.3275337E-3,1.5151923E8,-2.1119022E-3,1.8715081E-3,-2.386954E-3,3.0136635E-4],"split_indices":[61,4,40,60,65,46,2,9,53,0,0,65,0,46,62,0,0,5,55,35,0,2,3,0,0,0,0,0,2,0,0,0,0,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,6.2E1,6.3E1,5.4E1,8E0,2.7E1,3.6E1,1.5E1,3.9E1,4E0,4E0,2.3E1,4E0,2.6E1,1E1,1.1E1,4E0,1.5E1,2.4E1,1.3E1,1E1,9E0,1.7E1,6E0,4E0,5E0,1E1,5E0,1.9E1,8E0,5E0,5E0,4E0,4E0,1.3E1,1.3E1,6E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.573836E-3,-8.5016815E-3,9.444495E-3,2.1678766E-2,-1.3977551E-1,6.968152E-3,1.843558E-1,-3.1235914E-3,-8.891834E-3,3.1124907E-2,-5.1240332E-2,1.3428955E-2,2.4186955E-3,-7.9291575E-3,5.61577E-2,-8.892325E-2,1.2694318E-2,-3.0584924E-2,2.2307547E-2,-1.4322726E-3,7.3266655E-2,-6.135342E-3,-2.6337032E-3,-1.1365078E-3,2.013419E-3,5.368551E-4,-2.3376665E-3,3.989589E-3,-6.776168E-4,9.94723E-2,1.5594872E-3,1.1406836E-1,1.2909637E-3,6.582775E-3,2.0799527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,-1,3,5,7,9,11,-1,-1,13,15,-1,-1,17,19,21,23,25,27,-1,29,-1,-1,-1,-1,-1,-1,-1,-1,31,-1,33,-1,-1,-1],"loss_changes":[2.6314428E-1,0E0,2.1662514E-1,2.5956097E-1,1.888822E-2,1.4361379E-1,1.02292955E-1,0E0,0E0,7.083482E-2,7.3406145E-2,0E0,0E0,2.0548312E-2,6.704952E-2,1.8293545E-2,1.3906407E-2,1.3608249E-2,3.023713E-2,0E0,3.6129415E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.0178035E-2,0E0,2.416876E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,20,20,29,29,31,31],"right_children":[2,-1,4,6,8,10,12,-1,-1,14,16,-1,-1,18,20,22,24,26,28,-1,30,-1,-1,-1,-1,-1,-1,-1,-1,32,-1,34,-1,-1,-1],"split_conditions":[9.7888E4,-8.5016815E-3,2.8799975E10,1.1811867E4,5.3E1,5.1121724E7,6.1358623E9,-3.1235914E-3,-8.891834E-3,1.1467087E3,4.599408E6,1.3428955E-2,2.4186955E-3,1.5049451E1,1E0,2.4E1,1.10794815E2,3.4135348E7,2.68E2,-1.4322726E-3,1.343964E1,-6.135342E-3,-2.6337032E-3,-1.1365078E-3,2.013419E-3,5.368551E-4,-2.3376665E-3,3.989589E-3,-6.776168E-4,2.3292195E5,1.5594872E-3,4.3E1,1.2909637E-3,6.582775E-3,2.0799527E-3],"split_indices":[1,0,5,60,3,53,5,0,0,60,1,0,0,64,109,3,64,12,0,0,62,0,0,0,0,0,0,0,0,41,0,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,7E0,1.16E2,1.08E2,8E0,1E2,8E0,4E0,4E0,7.1E1,2.9E1,4E0,4E0,2.8E1,4.3E1,1.8E1,1.1E1,1.6E1,1.2E1,7E0,3.6E1,7E0,1.1E1,5E0,6E0,5E0,1.1E1,4E0,8E0,2.1E1,1.5E1,1.7E1,4E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-4.1402844E-3,-2.2836883E-2,1.5979998E-1,-5.9765212E-2,3.626503E-2,1.2094872E-2,3.888017E-3,-4.279397E-2,-1.0483428E-2,1.0008951E-1,3.0576098E-3,-7.462265E-3,-1.0805465E-1,3.212016E-3,6.959852E-3,2.4669648E-3,-9.753375E-3,-2.585924E-2,4.8280796E-3,-1.5644564E-3,-1.2384433E-1,-1.436938E-3,1.2045434E-3,-3.909467E-3,-1.599238E-2,-1.4602482E-1,-3.4049307E-3,7.955247E-4,-3.2333322E-2,-3.9120517E-3,-8.873036E-3,-2.2085065E-3,4.848698E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,-1,-1,11,-1,13,15,17,19,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1,27,29,-1,-1,31,-1,-1,-1,-1],"loss_changes":[3.5920388E-1,2.3133357E-1,6.729379E-2,1.668329E-1,8.6591266E-2,0E0,0E0,1.3753992E-1,0E0,1.1179E-2,1.793672E-2,8.250088E-2,2.0909876E-2,0E0,0E0,0E0,1.7117474E-2,1.8030807E-2,0E0,0E0,9.4026625E-3,0E0,0E0,0E0,1.7078187E-2,1.2826905E-2,0E0,0E0,1.2881501E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,9,9,10,10,11,11,12,12,16,16,17,17,20,20,24,24,25,25,28,28],"right_children":[2,4,6,8,10,-1,-1,12,-1,14,16,18,20,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1,28,30,-1,-1,32,-1,-1,-1,-1],"split_conditions":[9.669896E3,9.379017E6,2.300644E5,9.493188E-1,9.366E4,1.2094872E-2,3.888017E-3,2.4126838E11,-1.0483428E-2,5.815544E1,1.5360503E5,7.9918444E5,2.0161922E-1,3.212016E-3,6.959852E-3,2.4669648E-3,3.3367968E7,5.737924E0,4.8280796E-3,-1.5644564E-3,8.8132757E-1,-1.436938E-3,1.2045434E-3,-3.909467E-3,2.3452184E5,8.180538E-1,-3.4049307E-3,7.955247E-4,4.9E1,-3.9120517E-3,-8.873036E-3,-2.2085065E-3,4.848698E-4],"split_indices":[60,59,41,65,37,0,0,39,0,64,36,36,35,0,0,0,40,66,0,0,35,0,0,0,36,35,0,0,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.15E2,1.04E2,1.1E1,6.4E1,4E1,4E0,7E0,5.9E1,5E0,1.3E1,2.7E1,3.9E1,2E1,9E0,4E0,5E0,2.2E1,3.4E1,5E0,4E0,1.6E1,1.4E1,8E0,4E0,3E1,1E1,6E0,1E1,2E1,5E0,5E0,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[7.5497585E-3,1.615194E-2,-9.981885E-3,-1.3793353E-3,1.448066E-1,-4.048124E-2,3.77472E-2,1.1975793E-2,3.986692E-3,-7.864104E-3,-2.5077084E-2,7.700159E-2,8.0547035E-3,-3.6351946E-3,-3.7067272E-3,1.1413681E-4,1.1415803E-1,-1.0213582E-3,1.3496515E-3,3.4082916E-3,-1.766733E-2,2.2514681E-3,6.701856E-3,-4.0413473E-2,2.2844657E-5,9.521342E-5,-2.7496344E-3,8.127058E-4,-1.8928587E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,15,17,-1,19,-1,21,-1,-1,-1,23,-1,-1,25,27,-1,-1,-1,-1],"loss_changes":[2.4369362E-1,2.8213474E-1,0E0,1.713494E-1,7.653993E-2,1.0299959E-1,6.465118E-2,0E0,0E0,0E0,5.534764E-2,6.626244E-2,2.023603E-2,0E0,4.003995E-2,0E0,2.1449238E-2,0E0,0E0,0E0,1.2877188E-2,0E0,0E0,1.0304026E-2,1.3480124E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10,11,11,12,12,14,14,16,16,20,20,23,23,24,24],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,16,18,-1,20,-1,22,-1,-1,-1,24,-1,-1,26,28,-1,-1,-1,-1],"split_conditions":[1.5081978E8,9.669896E3,-9.981885E-3,2.3514317E8,7.7854166E0,1.477E3,1.5046106E7,1.1975793E-2,3.986692E-3,-7.864104E-3,5.981675E-4,1.1191123E3,3.5328901E3,-3.6351946E-3,1.82E2,1.1413681E-4,5.35E3,-1.0213582E-3,1.3496515E-3,3.4082916E-3,3.7E1,2.2514681E-3,6.701856E-3,3.2785356E7,8.8945635E-2,9.521342E-5,-2.7496344E-3,8.127058E-4,-1.8928587E-3],"split_indices":[53,60,0,7,62,2,53,0,0,0,50,60,60,0,10,0,2,0,0,0,3,0,0,12,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.28E2,1.24E2,4E0,1.1E2,1.4E1,5.5E1,5.5E1,4E0,1E1,5E0,5E1,2.3E1,3.2E1,1.4E1,3.6E1,8E0,1.5E1,1.3E1,1.9E1,5E0,3.1E1,5E0,1E1,1.3E1,1.8E1,4E0,9E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.018283E-2,2.2156123E-2,-9.058825E-3,1.844924E-3,1.7000593E-1,-1.04244046E-1,1.5662562E-2,1.1214366E-2,3.421474E-3,-2.2609835E-3,-6.0544345E-3,-2.3681257E-2,4.256079E-2,2.212112E-2,-5.1321022E-2,9.6712805E-2,6.2694605E-3,-5.7597965E-4,3.601053E-3,-5.259457E-3,-1.2225619E-3,2.0659226E-3,1.4623377E-1,2.2249084E-2,-1.6383722E-3,8.463019E-3,3.5158303E-3,3.270178E-2,-7.78504E-4,1.5180839E-2,3.3933064E-3,2.4064812E-3,-3.5640533E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,15,17,19,21,23,-1,-1,-1,-1,-1,25,27,-1,-1,-1,29,-1,31,-1,-1,-1],"loss_changes":[2.9616007E-1,3.4803516E-1,0E0,1.5262033E-1,7.2433144E-2,9.3830675E-3,9.836955E-2,0E0,0E0,0E0,0E0,4.924202E-2,1.0748235E-1,2.822895E-2,3.3484094E-2,5.190967E-2,2.2808524E-2,0E0,0E0,0E0,0E0,0E0,1.1696875E-2,1.0607E-2,0E0,0E0,0E0,1.2360627E-2,0E0,1.2335172E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,12,12,13,13,14,14,15,15,16,16,22,22,23,23,27,27,29,29],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,16,18,20,22,24,-1,-1,-1,-1,-1,26,28,-1,-1,-1,30,-1,32,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825864E4,-9.058825E-3,4.1964554E-4,3.0376984E1,2.8325282E11,1.1679049E3,1.1214366E-2,3.421474E-3,-2.2609835E-3,-6.0544345E-3,1.1085E4,2.0752128E7,9.513E3,3.508913E5,3.6742297E5,8.650247E-1,-5.7597965E-4,3.601053E-3,-5.259457E-3,-1.2225619E-3,2.0659226E-3,9.275501E-1,8.057621E6,-1.6383722E-3,8.463019E-3,3.5158303E-3,1.2424786E2,-7.78504E-4,2.8399336E7,3.3933064E-3,2.4064812E-3,-3.5640533E-4],"split_indices":[53,60,0,50,65,39,60,0,0,0,0,9,53,9,55,36,35,0,0,0,0,0,65,55,0,0,0,64,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,1.15E2,6E0,1.02E2,1.3E1,1.1E1,9.1E1,7E0,6E0,4E0,7E0,3.7E1,5.4E1,1.4E1,2.3E1,2.1E1,3.3E1,9E0,5E0,6E0,1.7E1,1.1E1,1E1,2.4E1,9E0,6E0,4E0,1.9E1,5E0,1.4E1,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.8978153E-3,-1.487843E-1,1.6062366E-2,-1.5231037E-3,-1.0573193E-2,2.6952952E-2,-8.895926E-3,1.2314923E-2,1.0126764E-2,6.844188E-2,-1.3945706E-2,1.3636653E-2,1.1922113E-1,-4.758874E-2,1.6522948E-2,2.381731E-3,-6.442435E-5,7.1502044E-3,2.5913378E-3,2.5307846E-3,-7.166898E-2,-2.4508685E-4,4.2250272E-2,1.9835639E-3,-1.058434E-3,-4.836734E-3,-1.962897E-3,-2.0854196E-2,2.520834E-3,3.138101E-3,2.8902604E-4,-2.8964449E-3,-4.2159947E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,-1,-1,7,-1,9,-1,11,13,15,17,19,21,-1,-1,-1,-1,23,25,27,29,-1,-1,-1,-1,31,-1,-1,-1,-1,-1],"loss_changes":[2.499964E-1,7.781637E-2,2.5843203E-1,0E0,0E0,2.954263E-1,0E0,1.5276921E-1,0E0,9.021218E-2,7.3565416E-2,9.298468E-3,2.5557667E-2,4.121565E-2,1.640343E-2,0E0,0E0,0E0,0E0,1.2478363E-2,1.6653918E-2,2.7330339E-2,1.2340324E-2,0E0,0E0,0E0,0E0,1.4252398E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,22,22,27,27],"right_children":[2,4,6,-1,-1,8,-1,10,-1,12,14,16,18,20,22,-1,-1,-1,-1,24,26,28,30,-1,-1,-1,-1,32,-1,-1,-1,-1,-1],"split_conditions":[9.7888E4,1.0478E4,1.5081978E8,-1.5231037E-3,-1.0573193E-2,1.1811867E4,-8.895926E-3,1.5046106E7,1.0126764E-2,9.4939746E2,2.532835E3,1.314E3,9.275501E-1,1.8230173E5,6.160098E3,2.381731E-3,-6.442435E-5,7.1502044E-3,2.5913378E-3,4.0605398E6,1.4710438E1,1E0,7.118011E-1,1.9835639E-3,-1.058434E-3,-4.836734E-3,-1.962897E-3,7.94E2,2.520834E-3,3.138101E-3,2.8902604E-4,-2.8964449E-3,-4.2159947E-5],"split_indices":[1,9,53,0,0,60,0,53,0,60,60,11,65,36,60,0,0,0,0,40,62,96,35,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,9E0,1.14E2,4E0,5E0,1.09E2,5E0,1.02E2,7E0,3.2E1,7E1,1.6E1,1.6E1,3.3E1,3.7E1,4E0,1.2E1,1E1,6E0,1.1E1,2.2E1,2.3E1,1.4E1,4E0,7E0,1E1,1.2E1,1.7E1,6E0,8E0,6E0,5E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-5.485467E-3,5.9787272E-3,-1.526367E-1,1.615318E-2,-8.751206E-3,-1.2547667E-2,-5.7333615E-4,2.0716772E-3,1.4397849E-1,3.837911E-2,-3.4276377E-2,1.1837135E-2,2.3499809E-3,-7.2389534E-3,8.6111456E-2,-7.595288E-2,-7.1574296E-3,-2.8361834E-3,7.312558E-3,1.7391649E-3,1.22555815E-1,-2.2134834E-3,-5.9366594E-3,-1.835101E-2,2.720139E-3,2.684344E-3,-1.0375013E-2,6.8072844E-3,2.7734903E-3,7.136905E-4,-2.811082E-2,-2.637169E-3,3.6636324E-4,-3.2282495E-3,-7.440778E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,-1,17,19,21,23,-1,25,-1,27,-1,-1,29,-1,-1,31,-1,-1,-1,33,-1,-1,-1,-1],"loss_changes":[2.100969E-1,2.2628883E-1,1.3620248E-1,1.9949865E-1,0E0,0E0,0E0,1.3460568E-1,9.610093E-2,1.11673675E-1,5.709774E-2,0E0,0E0,2.092316E-2,4.0813863E-2,2.2869565E-2,2.4011072E-2,0E0,1.9609906E-2,0E0,9.76266E-3,0E0,0E0,9.51885E-3,0E0,0E0,1.4181827E-2,0E0,0E0,0E0,9.831963E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,20,20,23,23,26,26,30,30],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,-1,18,20,22,24,-1,26,-1,28,-1,-1,30,-1,-1,32,-1,-1,-1,34,-1,-1,-1,-1],"split_conditions":[1E0,1.5081978E8,2.667836E6,1.0825864E4,-8.751206E-3,-1.2547667E-2,-5.7333615E-4,2.727403E7,6.691E3,1.6511278E1,5.0557896E1,1.1837135E-2,2.3499809E-3,9.7E1,3.1346828E5,1E0,5.2E1,-2.8361834E-3,4.4493478E2,1.7391649E-3,1.2320755E1,-2.2134834E-3,-5.9366594E-3,5.8163033E1,2.720139E-3,2.684344E-3,2.5975E4,6.8072844E-3,2.7734903E-3,7.136905E-4,5.786E3,-2.637169E-3,3.6636324E-4,-3.2282495E-3,-7.440778E-4],"split_indices":[112,53,40,60,0,0,0,53,9,64,64,0,0,10,36,24,8,0,4,0,62,0,0,64,0,0,37,0,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,1.15E2,8E0,1.1E2,5E0,4E0,4E0,1E2,1E1,5E1,5E1,4E0,6E0,2.6E1,2.4E1,1.9E1,3.1E1,5E0,2.1E1,1.1E1,1.3E1,1.3E1,6E0,2.7E1,4E0,5E0,1.6E1,9E0,4E0,6E0,2.1E1,4E0,1.2E1,4E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.1815302E-3,-2.3219846E-2,8.313469E-2,-1.3494164E-1,-9.802931E-3,1.1114979E-2,5.2350737E-2,-1.07406E-2,-1.7901447E-3,2.4362285E-2,-4.3740172E-2,2.6366299E-2,6.6335155E-3,-1.348862E-2,6.110581E-2,-7.9454854E-2,-1.215543E-2,2.051758E-3,-1.196354E-3,8.657432E-4,-1.4065179E-3,5.096483E-3,1.3094642E-3,-5.6299223E-3,-2.1017462E-3,2.2616594E-3,-2.6425969E-2,-2.554793E-3,-1.8727333E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,17,-1,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[2.6367867E-1,1.395796E-1,1.3058557E-1,7.9242215E-2,9.962082E-2,0E0,5.706022E-2,0E0,0E0,6.0627587E-2,4.7539867E-2,1.984965E-2,0E0,1.16372965E-2,3.0876622E-2,2.2056997E-2,2.163434E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1974148E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,11,11,13,13,14,14,15,15,16,16,26,26],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,18,-1,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[1.3137E4,9.7888E4,1.8484E4,2.3E1,2.8399336E7,1.1114979E-2,8.870948E7,-1.07406E-2,-1.7901447E-3,1.1467087E3,1.0363E4,6.2E1,6.6335155E-3,4.0953384E7,1.4117E4,6.2462933E2,1.0803E4,2.051758E-3,-1.196354E-3,8.657432E-4,-1.4065179E-3,5.096483E-3,1.3094642E-3,-5.6299223E-3,-2.1017462E-3,2.2616594E-3,1E0,-2.554793E-3,-1.8727333E-4],"split_indices":[2,1,2,3,53,0,40,0,0,60,9,3,0,12,9,41,9,0,0,0,0,0,0,0,0,0,22,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,9.3E1,3E1,9E0,8.4E1,4E0,2.6E1,4E0,5E0,4.2E1,4.2E1,2.1E1,5E0,2.1E1,2.1E1,1.9E1,2.3E1,1.6E1,5E0,7E0,1.4E1,8E0,1.3E1,8E0,1.1E1,4E0,1.9E1,8E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.1977757E-3,5.4910136E-3,-8.218771E-3,-1.665647E-2,7.648324E-2,-2.0948378E-3,-7.478661E-2,1.339357E-1,4.519686E-4,-2.6983518E-2,2.285046E-2,-7.6502596E-4,-5.2880086E-3,9.509858E-3,3.26402E-3,-4.0344793E-2,1.2652791E-3,7.1224794E-2,-3.550193E-4,-3.0688597E-3,-1.5402632E-2,5.1169596E-3,8.3315576E-4,1.9433018E-2,-1.9050989E-3,4.3566548E-4,-2.7602694E-3,1.02578684E-4,3.053794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,19,-1,21,23,-1,25,-1,-1,27,-1,-1,-1,-1,-1],"loss_changes":[1.7998281E-1,1.8523294E-1,0E0,7.602561E-2,1.0654427E-1,4.593904E-2,3.6245286E-2,5.0531358E-2,0E0,2.7506525E-2,4.155962E-2,0E0,0E0,0E0,0E0,1.6905215E-2,0E0,2.1312505E-2,2.1116374E-2,0E0,1.7163517E-2,0E0,0E0,1.3711615E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10,15,15,17,17,18,18,20,20,23,23],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,20,-1,22,24,-1,26,-1,-1,28,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,4.867052E6,-8.218771E-3,2.8325282E11,4.0342972E11,4.757E3,7.157903E7,9.275501E-1,4.519686E-4,1.8393064E1,1.0526E4,-7.6502596E-4,-5.2880086E-3,9.509858E-3,3.26402E-3,1.4731E4,1.2652791E-3,2.7943066E3,4.6E1,-3.0688597E-3,8.2474226E-1,5.1169596E-3,8.3315576E-4,6.206162E5,-1.9050989E-3,4.3566548E-4,-2.7602694E-3,1.02578684E-4,3.053794E-3],"split_indices":[53,1,0,39,39,2,7,65,0,62,9,0,0,0,0,9,0,4,3,0,65,0,0,36,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,1.16E2,5E0,8.9E1,2.7E1,7.2E1,1.7E1,1.4E1,1.3E1,3.6E1,3.6E1,7E0,1E1,6E0,8E0,2.9E1,7E0,1.1E1,2.5E1,1.4E1,1.5E1,6E0,5E0,1.7E1,8E0,1E1,5E0,1.3E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-5.665865E-4,7.3283655E-3,-8.768927E-3,-8.8798385E-3,1.4853945E-1,-1.0469477E-3,-7.895555E-3,1.1115651E-2,3.6933806E-3,-2.6958E-2,3.1796984E-2,-6.970774E-3,-6.0889877E-2,6.867492E-2,6.9541335E-3,-2.85108E-2,6.32105E-2,-3.6454021E-3,-9.121477E-4,7.946628E-4,4.7888407E-3,-1.3708207E-3,1.3543353E-3,1.2524375E-3,-4.711889E-2,4.544135E-3,8.895858E-4,-4.589625E-3,-8.525654E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,13,15,17,19,21,23,25,-1,-1,-1,-1,-1,-1,-1,27,-1,-1,-1,-1],"loss_changes":[1.7476572E-1,2.6681727E-1,0E0,1.2849846E-1,5.3955555E-2,8.68103E-2,0E0,0E0,0E0,3.8233183E-2,4.0749308E-2,5.773532E-2,1.2094431E-2,2.7929567E-2,2.2185318E-2,3.07732E-2,1.1043835E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.982425E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,24,24],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,14,16,18,20,22,24,26,-1,-1,-1,-1,-1,-1,-1,28,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.0825864E4,-8.768927E-3,1.088501E0,9.9182825E0,1.1565699E5,-7.895555E-3,1.1115651E-2,3.6933806E-3,4.352552E9,1.4163358E7,1.2809688E3,2.09038E3,4.308E3,4.451217E6,2.8017812E3,1.19461E5,-3.6454021E-3,-9.121477E-4,7.946628E-4,4.7888407E-3,-1.3708207E-3,1.3543353E-3,1.2524375E-3,1.6528512E7,4.544135E-3,8.895858E-4,-4.589625E-3,-8.525654E-4],"split_indices":[5,60,0,46,62,41,0,0,0,5,53,60,41,2,1,55,37,0,0,0,0,0,0,0,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.19E2,1.15E2,4E0,1.04E2,1.1E1,1E2,4E0,4E0,7E0,5.6E1,4.4E1,3.6E1,2E1,1.7E1,2.7E1,2.8E1,8E0,1.4E1,6E0,7E0,1E1,1E1,1.7E1,7E0,2.1E1,4E0,4E0,7E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.3377202E-2,7.929139E-4,1.5298352E-1,-2.1819925E-2,3.6235936E-2,1.0857218E-2,1.5833569E-3,-3.1901684E-2,4.8936438E-2,7.145765E-2,9.103303E-3,-4.258162E-3,-1.752084E-2,-1.9603378E-4,4.402723E-3,1.0140312E-1,8.408657E-4,-8.3717087E-4,2.245282E-3,-5.2017353E-2,-2.660602E-3,1.9139096E-3,6.067845E-3,3.0795322E-4,-7.386383E-2,4.313752E-2,-1.1718228E-3,-4.928418E-3,-8.6072076E-4,-1.084473E-3,4.4611683E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,-1,19,-1,-1,21,-1,-1,-1,23,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1],"loss_changes":[2.1245708E-1,9.0555005E-2,8.1567585E-2,5.0782207E-2,4.1398443E-2,0E0,0E0,4.866957E-2,2.102664E-2,2.8853506E-2,2.7076505E-2,0E0,2.549292E-2,0E0,0E0,1.3787426E-2,0E0,0E0,0E0,1.9635346E-2,3.712726E-2,0E0,0E0,0E0,1.6158652E-2,4.1847356E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,12,12,15,15,19,19,20,20,24,24,25,25],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,-1,20,-1,-1,22,-1,-1,-1,24,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1],"split_conditions":[1.3772013E4,3.4707712E8,8.50999E9,1E0,1.5046106E7,1.0857218E-2,1.5833569E-3,1.13892675E-4,5.614829E5,1.343964E1,6.160098E3,-4.258162E-3,1E0,-1.9603378E-4,4.402723E-3,6.471E3,8.408657E-4,-8.3717087E-4,2.245282E-3,1.1085E4,3.42E2,1.9139096E-3,6.067845E-3,3.0795322E-4,2.8734463E1,2.862E3,-1.1718228E-3,-4.928418E-3,-8.6072076E-4,-1.084473E-3,4.4611683E-3],"split_indices":[4,7,5,96,53,0,0,46,36,62,60,0,8,0,0,2,0,0,0,9,10,0,0,0,66,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.2E2,1.11E2,9E0,6.8E1,4.3E1,5E0,4E0,6E1,8E0,1.8E1,2.5E1,1.1E1,4.9E1,4E0,4E0,1.1E1,7E0,1.5E1,1E1,1.4E1,3.5E1,4E0,7E0,4E0,1E1,1.1E1,2.4E1,6E0,4E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[6.022914E-3,1.3852455E-2,-8.295134E-3,-2.382033E-3,1.4016517E-1,-6.543896E-2,1.0999224E-2,9.266423E-3,2.1481006E-3,-3.5416445E-4,-8.1487246E-2,-8.825861E-3,3.695679E-2,-2.0333312E-3,-5.45934E-3,-3.1865537E-2,6.8281186E-3,6.8680264E-2,-1.4149657E-2,-5.9598744E-2,-4.0164316E-4,1.4416062E-3,-3.9561023E-4,4.961875E-3,2.7738446E-2,-3.599489E-3,1.4964637E-2,-6.168194E-4,-4.5064264E-3,-1.631862E-6,3.1818922E-3,2.6700222E-3,-7.841156E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,13,15,17,-1,-1,19,21,23,25,27,-1,-1,-1,-1,29,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6844513E-1,2.3410483E-1,0E0,8.678544E-2,5.7929367E-2,1.6156733E-2,4.4068106E-2,0E0,0E0,0E0,1.3015829E-2,1.789005E-2,6.0958948E-2,0E0,0E0,1.2508612E-2,1.0839736E-2,3.0281886E-2,2.7864225E-2,1.3386846E-2,0E0,0E0,0E0,0E0,1.29627E-2,0E0,1.505263E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,24,24,26,26],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,14,16,18,-1,-1,20,22,24,26,28,-1,-1,-1,-1,30,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,8.351458E3,-8.295134E-3,1.13892675E-4,3.813527E2,2.3737179E1,3.290356E5,9.266423E-3,2.1481006E-3,-3.5416445E-4,6.3997424E-1,1.1679049E3,8.63558E-1,-2.0333312E-3,-5.45934E-3,2.91E2,4.4993465E1,1.4117E4,8.091413E0,1.19288025E1,-4.0164316E-4,1.4416062E-3,-3.9561023E-4,4.961875E-3,7.651737E1,-3.599489E-3,8.8887E4,-6.168194E-4,-4.5064264E-3,-1.631862E-6,3.1818922E-3,2.6700222E-3,-7.841156E-4],"split_indices":[53,60,0,46,64,64,36,0,0,0,35,60,35,0,0,0,64,9,61,62,0,0,0,0,64,0,37,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,1.13E2,4E0,1.01E2,1.2E1,1.7E1,8.4E1,7E0,5E0,4E0,1.3E1,4.8E1,3.6E1,7E0,6E0,1.9E1,2.9E1,2.2E1,1.4E1,8E0,1.1E1,1.1E1,1.8E1,1.1E1,1.1E1,4E0,1E1,4E0,4E0,7E0,4E0,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.6575159E-3,9.468109E-3,-8.068276E-3,-4.8900684E-3,1.3783097E-1,3.440113E-3,-7.745467E-3,2.678895E-3,1.0730765E-2,-1.4037359E-2,3.828755E-2,-2.0424273E-2,3.1239227E-3,3.3800057E-3,1.0744218E-3,-2.6254915E-2,1.7396538E-3,-1.6106837E-3,5.129605E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,13,15,-1,-1,-1,17,-1,-1,-1],"loss_changes":[1.5217282E-1,2.015672E-1,0E0,1.3027757E-1,6.549613E-2,5.839674E-2,0E0,0E0,0E0,3.3812393E-2,1.4822923E-2,2.0845274E-2,0E0,0E0,0E0,1.5565835E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9,10,10,11,11,15,15],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,14,16,-1,-1,-1,18,-1,-1,-1],"split_conditions":[1.5081978E8,1.0825864E4,-8.068276E-3,6.313997E-1,9.48E2,6.402099E-1,-7.745467E-3,2.678895E-3,1.0730765E-2,2.1382671E1,1.3420091E6,1E0,3.1239227E-3,3.3800057E-3,1.0744218E-3,7.621429E1,1.7396538E-3,-1.6106837E-3,5.129605E-4],"split_indices":[53,60,0,46,0,50,0,0,0,62,55,96,0,0,0,64,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,1.08E2,4E0,9.8E1,1E1,9.4E1,4E0,6E0,4E0,6.3E1,3.1E1,5.9E1,4E0,9E0,2.2E1,5.4E1,5E0,4.5E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[9.108477E-3,-5.8931457E-3,1.2847085E-1,8.885746E-4,-7.220819E-3,9.107877E-3,2.7674914E-3,2.3422848E-2,-3.6507018E-2,-1.0990182E-2,6.737903E-2,-2.3531828E-3,6.859089E-4,-4.2775787E-2,3.024727E-3,6.9111586E-3,4.9915113E-2,5.4258053E-5,-3.0981658E-3,2.367243E-3,-6.7979572E-3,7.210635E-2,1.3099209E-2,-1.6031776E-3,4.14814E-4,4.4660256E-3,9.951434E-4,-1.0043894E-3,2.1512804E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,-1,-1,-1,9,11,13,15,-1,-1,17,19,-1,21,-1,-1,-1,23,25,27,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2458579E-1,1.1050232E-1,5.0400242E-2,9.186482E-2,0E0,0E0,0E0,1.0360175E-1,2.7335715E-2,1.7502809E-2,3.3294484E-2,0E0,0E0,1.1718078E-2,1.3138186E-2,0E0,2.0297222E-2,0E0,0E0,0E0,1.02403E-2,1.5613586E-2,1.294156E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,9,9,10,10,13,13,14,14,16,16,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,-1,-1,10,12,14,16,-1,-1,18,20,-1,22,-1,-1,-1,24,26,28,-1,-1,-1,-1,-1,-1],"split_conditions":[9.669896E3,1.088501E0,9.9182825E0,4.494817E7,-7.220819E-3,9.107877E-3,2.7674914E-3,2.951724E1,3.7254233E3,1.92E2,3.42E2,-2.3531828E-3,6.859089E-4,7.286E3,1.2410928E8,6.9111586E-3,1.734241E7,5.4258053E-5,-3.0981658E-3,2.367243E-3,7.3434E4,1.343964E1,7.94867E5,-1.6031776E-3,4.14814E-4,4.4660256E-3,9.951434E-4,-1.0043894E-3,2.1512804E-3],"split_indices":[60,46,62,53,0,0,0,64,60,0,10,0,0,9,5,0,53,0,0,0,37,62,37,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.24E2,1.11E2,1.3E1,1.07E2,4E0,6E0,7E0,6.7E1,4E1,3.8E1,2.9E1,3.2E1,8E0,1.1E1,2.7E1,4E0,2.5E1,4E0,7E0,4E0,2.3E1,1.5E1,1E1,8E0,1.5E1,1E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.8099964E-4,6.0323607E-3,-7.185535E-3,1.2109524E-2,-7.1412744E-3,4.2212394E-4,1.0641339E-1,-7.116359E-3,2.8433348E-3,8.047815E-3,9.6643047E-4,3.3677241E-3,-3.7891667E-2,-2.7728016E-3,3.11227E-3,-4.885169E-3,-1.768705E-2,-9.708262E-3,2.5338577E-3,-5.400019E-6,-2.207829E-3,1.9399375E-2,-2.242925E-2,3.0942594E-3,-2.433403E-6,1.1046147E-3,-1.3103617E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,9,11,-1,-1,-1,13,15,17,-1,-1,19,21,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[1.1794869E-1,1.2565851E-1,0E0,1.3993028E-1,0E0,5.1226895E-2,7.151578E-2,3.3038E-2,0E0,0E0,0E0,2.9401654E-2,3.2021817E-2,2.8006919E-2,0E0,0E0,1.0528881E-2,2.4132898E-2,0E0,0E0,0E0,1.7661318E-2,1.09038185E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,11,11,12,12,13,13,16,16,17,17,21,21,22,22],"right_children":[2,4,-1,6,-1,8,10,12,-1,-1,-1,14,16,18,-1,-1,20,22,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[6.313997E-1,1.5081978E8,-7.185535E-3,1.0825864E4,-7.1412744E-3,2.1729739E0,3.813527E2,6.784095E7,2.8433348E-3,8.047815E-3,9.6643047E-4,1E0,6.319674E2,1.2750684E9,3.11227E-3,-4.885169E-3,8.180538E-1,2.3452184E5,2.5338577E-3,-5.400019E-6,-2.207829E-3,1.1085E4,1.062E3,3.0942594E-3,-2.433403E-6,1.1046147E-3,-1.3103617E-3],"split_indices":[46,53,0,60,0,47,64,53,0,0,0,96,41,7,0,0,35,36,0,0,0,9,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.34E2,1.3E2,4E0,1.26E2,4E0,1.13E2,1.3E1,1.01E2,1.2E1,7E0,6E0,7.6E1,2.5E1,7E1,6E0,5E0,2E1,6.3E1,7E0,1.3E1,7E0,1.9E1,4.4E1,5E0,1.4E1,4E0,4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.262944E-3,1.10188415E-2,-6.7671128E-3,-9.66277E-3,7.147634E-2,5.4580276E-4,-8.201898E-2,7.771204E-3,3.576798E-2,2.5569486E-2,-3.6647834E-2,-8.457942E-3,-5.892371E-4,6.241636E-4,2.8304877E-3,3.2405874E-3,5.3582415E-2,-4.7453768E-3,-2.5349911E-2,-1.165278E-3,9.803233E-4,7.581617E-2,8.164947E-4,-2.8009242E-3,-6.300911E-4,1.3194524E-3,4.7526285E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,-1,-1,-1,-1,19,21,-1,23,-1,-1,25,-1,-1,-1,-1,-1],"loss_changes":[1.336996E-1,1.6198692E-1,0E0,7.1694165E-2,1.0170421E-1,8.097913E-2,7.732221E-2,0E0,1.2561098E-2,3.235225E-2,2.2281323E-2,0E0,0E0,0E0,0E0,1.4835516E-2,1.7361604E-2,0E0,1.1774929E-2,0E0,0E0,1.3239846E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,15,15,16,16,18,18,21,21],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,-1,-1,-1,-1,20,22,-1,24,-1,-1,26,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.3137E4,-6.7671128E-3,1E0,1.3453537E-1,2.8399336E7,6.069723E5,7.771204E-3,1.864698E6,3.136809E5,6.85159E5,-8.457942E-3,-5.892371E-4,6.241636E-4,2.8304877E-3,7.3434E4,8.317631E-1,-4.7453768E-3,1.6055046E-2,-1.165278E-3,9.803233E-4,4.524696E5,8.164947E-4,-2.8009242E-3,-6.300911E-4,1.3194524E-3,4.7526285E-3],"split_indices":[53,2,0,112,65,53,36,0,40,36,1,0,0,0,0,37,35,0,65,0,0,36,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.33E2,1.28E2,5E0,9.6E1,3.2E1,8.5E1,1.1E1,8E0,2.4E1,5.1E1,3.4E1,4E0,7E0,1.3E1,1.1E1,2.9E1,2.2E1,4E0,3E1,1.1E1,1.8E1,1.3E1,9E0,7E0,2.3E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-4.950212E-3,1.1976451E-3,-7.764062E-3,-1.5580554E-2,5.9590764E-2,-7.2098533E-3,-8.441088E-3,7.94697E-3,2.4491223E-2,1.5107892E-2,-3.381464E-2,2.1884867E-3,-2.9486308E-3,-9.336393E-3,4.7754288E-2,-2.623156E-2,-4.406342E-3,-1.8765564E-3,1.1191131E-3,1.1176997E-2,-1.2264231E-3,3.6417262E-3,5.883952E-4,-5.5071335E-2,-7.101645E-3,1.7887924E-3,-9.767781E-4,-1.1367942E-3,-3.8310457E-3,7.6918857E-4,-1.0363637E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,-1,5,7,-1,9,-1,11,13,15,-1,17,19,21,23,-1,-1,-1,25,-1,-1,-1,27,29,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2907887E-1,1.2731381E-1,0E0,9.6353136E-2,1.0464919E-1,0E0,5.8470435E-2,0E0,1.3599615E-2,4.1144762E-2,1.8412117E-2,0E0,1.146837E-2,1.0345269E-2,2.1278396E-2,2.333211E-2,0E0,0E0,0E0,1.2403136E-2,0E0,0E0,0E0,1.1458103E-2,9.497611E-3,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,9,9,10,10,12,12,13,13,14,14,15,15,19,19,23,23,24,24],"right_children":[2,4,-1,6,8,-1,10,-1,12,14,16,-1,18,20,22,24,-1,-1,-1,26,-1,-1,-1,28,30,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.5325E4,-7.764062E-3,1.227133E6,1.3453537E-1,-7.2098533E-3,2.8399336E7,7.94697E-3,2.4092134E5,1.2738525E3,1.2511909E8,2.1884867E-3,1.5791555E9,4.593727E6,9.163856E0,1.3454421E3,-4.406342E-3,-1.8765564E-3,1.1191131E-3,3.8E1,-1.2264231E-3,3.6417262E-3,5.883952E-4,5.0717188E7,4E1,1.7887924E-3,-9.767781E-4,-1.1367942E-3,-3.8310457E-3,7.6918857E-4,-1.0363637E-3],"split_indices":[5,2,0,53,65,0,53,0,41,60,53,0,7,53,61,60,0,0,0,3,0,0,0,53,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.32E2,1.28E2,4E0,1E2,2.8E1,4E0,9.6E1,6E0,2.2E1,5E1,4.6E1,1.2E1,1E1,2.9E1,2.1E1,4.2E1,4E0,4E0,6E0,1.3E1,1.6E1,1.1E1,1E1,1.6E1,2.6E1,7E0,6E0,8E0,8E0,1E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.6468543E-3,1.0593817E-2,-6.2689874E-3,7.2848722E-3,4.1623046E-3,-2.0071745E-2,3.2510664E-2,-2.8500075E-2,1.8782024E-3,4.7142703E-2,1.04150325E-2,-2.1010492E-2,-3.6192886E-3,6.9613785E-2,5.762341E-3,1.7128616E-3,-6.582432E-4,-2.192902E-4,-3.472662E-2,3.3229703E-4,4.051825E-3,1.984804E-3,-8.277156E-4,-5.617373E-2,-7.2847423E-4,-3.912293E-3,-8.0101803E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,-1,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,-1,23,-1,-1,-1,-1,25,-1,-1,-1],"loss_changes":[1.14798434E-1,1.0526548E-1,0E0,0E0,7.759604E-2,3.1660177E-2,1.6320832E-2,1.7531808E-2,0E0,2.8424561E-2,1.4043637E-2,1.0650178E-2,0E0,1.8493377E-2,1.0641121E-2,0E0,0E0,0E0,1.0001656E-2,0E0,0E0,0E0,0E0,1.0799579E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,18,18,23,23],"right_children":[2,4,-1,-1,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,-1,24,-1,-1,-1,-1,26,-1,-1,-1],"split_conditions":[1.5081978E8,6.371407E0,-6.2689874E-3,7.2848722E-3,4.7992307E-1,1E0,1.343964E1,1.1674061E6,1.8782024E-3,4.3E1,1.0875838E7,8.1980075E5,-3.6192886E-3,2.1146134E5,4.9980104E-1,1.7128616E-3,-6.582432E-4,-2.192902E-4,2.3396162E5,3.3229703E-4,4.051825E-3,1.984804E-3,-8.277156E-4,3.0595828E3,-7.2847423E-4,-3.912293E-3,-8.0101803E-4],"split_indices":[53,62,0,0,47,96,62,36,0,3,40,40,0,36,35,0,0,0,55,0,0,0,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.2E2,1.15E2,5E0,4E0,1.11E2,6E1,5.1E1,5.3E1,7E0,3E1,2.1E1,4.7E1,6E0,1.9E1,1.1E1,1E1,1.1E1,2.2E1,2.5E1,4E0,1.5E1,4E0,7E0,1.1E1,1.4E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[3.5711743E-3,1.0021201E-2,-7.81081E-3,1.364426E-3,8.716132E-2,-1.5739003E-2,3.733574E-2,7.92615E-3,8.19529E-4,-9.819102E-3,-4.1603176E-3,5.358079E-2,5.5636663E-4,1.6645093E-2,-2.68649E-2,3.6744925E-3,1.4180092E-3,-1.39823565E-2,8.966273E-2,-3.6901843E-2,-3.6617597E-3,-2.2330303E-3,8.086654E-3,6.093848E-3,1.6137678E-3,-1.73967E-4,-4.7836423E-2,6.8511354E-4,-2.031872E-3,2.0094384E-3,-1.0824208E-3,-1.4418154E-3,-3.7896915E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,15,-1,17,19,-1,-1,21,23,25,27,-1,29,-1,-1,-1,31,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4500773E-1,8.60253E-2,0E0,7.2569855E-2,6.886769E-2,3.294752E-2,1.5231907E-2,0E0,0E0,3.4279913E-2,0E0,1.0550745E-2,0E0,6.834402E-2,1.0577753E-2,0E0,0E0,1.6291454E-2,1.4765434E-2,1.1463314E-2,1.1113614E-2,0E0,1.5587188E-2,0E0,0E0,0E0,1.0951761E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,11,11,13,13,14,14,17,17,18,18,19,19,20,20,22,22,26,26],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,16,-1,18,20,-1,-1,22,24,26,28,-1,30,-1,-1,-1,32,-1,-1,-1,-1,-1,-1],"split_conditions":[6.313997E-1,1.0825864E4,-7.81081E-3,6.402099E-1,1.3309696E7,1.5088414E6,1.5046106E7,7.92615E-3,8.19529E-4,3.0721E5,-4.1603176E-3,1.1225543E1,5.5636663E-4,1.4861879E3,3.0502517E3,3.6744925E-3,1.4180092E-3,9.7E1,2.0472059E3,2.8866995E1,8.44629E7,-2.2330303E-3,1.1085E4,6.093848E-3,1.6137678E-3,-1.73967E-4,6.4987373E-1,6.8511354E-4,-2.031872E-3,2.0094384E-3,-1.0824208E-3,-1.4418154E-3,-3.7896915E-3],"split_indices":[46,60,0,50,55,36,53,0,0,37,0,62,0,4,60,0,0,10,4,64,53,0,9,0,0,0,35,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.32E2,1.28E2,4E0,1.16E2,1.2E1,7.9E1,3.7E1,5E0,7E0,7.4E1,5E0,2.2E1,1.5E1,2.9E1,4.5E1,1E1,1.2E1,2.1E1,8E0,3.1E1,1.4E1,8E0,1.3E1,4E0,4E0,8E0,2.3E1,1E1,4E0,6E0,7E0,1.6E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[3.646897E-3,1.0034246E-2,-5.9048156E-3,6.263937E-4,1.125172E-1,-3.731666E-2,1.0536211E-2,9.351684E-3,1.1627066E-3,7.075695E-4,-7.886197E-2,4.952403E-3,4.865293E-3,-1.2556497E-3,1.747414E-3,-1.0691251E-3,-5.1546507E-3,1.5216328E-2,-2.9018834E-2,4.2378386E-3,4.1572783E-2,-3.3248795E-4,-2.9243268E-3,-3.571387E-3,2.0418072E-3,3.8714865E-3,5.6253496E-4,-5.866425E-4,2.2850267E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,17,-1,-1,-1,-1,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1],"loss_changes":[1.0205853E-1,1.14662714E-1,0E0,4.1754138E-2,6.868015E-2,3.6389552E-2,4.5988187E-2,0E0,0E0,1.3556771E-2,1.5644811E-2,0E0,2.9903367E-2,0E0,0E0,0E0,0E0,1.8631838E-2,1.378477E-2,1.4289505E-2,2.1401748E-2,0E0,0E0,1.8507222E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,12,12,17,17,18,18,19,19,20,20,23,23],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,18,-1,-1,-1,-1,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1],"split_conditions":[1.5081978E8,1.2769175E4,-5.9048156E-3,1.7214674E1,6.5566176E7,6.9237E4,2.0382088E1,9.351684E-3,1.1627066E-3,4.847826E0,5.432393E-1,4.952403E-3,6.721145E7,-1.2556497E-3,1.747414E-3,-1.0691251E-3,-5.1546507E-3,1.1104E4,1.5270755E6,2.7310195E0,1.9593608E5,-3.3248795E-4,-2.9243268E-3,2.3145154E0,2.0418072E-3,3.8714865E-3,5.6253496E-4,-5.866425E-4,2.2850267E-3],"split_indices":[53,60,0,66,53,12,66,0,0,66,35,0,53,0,0,0,0,2,40,65,41,0,0,47,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,1.18E2,5E0,1.09E2,9E0,2.2E1,8.7E1,4E0,5E0,1.2E1,1E1,4E0,8.3E1,7E0,5E0,4E0,6E0,6.4E1,1.9E1,4.6E1,1.8E1,1.2E1,7E0,3.9E1,7E0,7E0,1.1E1,3.4E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[1.3976836E-3,-9.283381E-3,1.0631168E-1,-9.592127E-4,-1.0999173E-1,8.204924E-3,1.670825E-3,2.9575275E-2,-2.336799E-2,-9.419501E-3,-3.5642817E-5,5.0583226E-3,7.2542936E-2,-7.204648E-2,-1.07489815E-2,-1.1539183E-3,1.1836628E-3,4.9413946E-3,1.6025123E-3,-1.1327841E-3,-4.3405932E-3,-4.729018E-4,-3.1877797E-2,3.168229E-4,-2.1902737E-3,-2.379334E-3,4.4525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,-1,11,13,-1,-1,15,17,19,21,-1,-1,-1,-1,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[1.4554344E-1,9.8815456E-2,5.0046444E-2,7.5957574E-2,8.5634835E-2,0E0,0E0,4.914455E-2,3.8539127E-2,0E0,0E0,1.8468292E-2,1.702588E-2,9.682156E-3,1.1275237E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.1917393E-2,1.3032842E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,11,11,12,12,13,13,14,14,21,21,22,22],"right_children":[2,4,6,8,10,-1,-1,12,14,-1,-1,16,18,20,22,-1,-1,-1,-1,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[1.0825864E4,1E0,2.5501368E9,2.7260774E7,2.667836E6,8.204924E-3,1.670825E-3,4.524696E5,2.2196926E-2,-9.419501E-3,-3.5642817E-5,1.0666667E1,8.317631E-1,2.269289E-3,4.349136E6,-1.1539183E-3,1.1836628E-3,4.9413946E-3,1.6025123E-3,-1.1327841E-3,-4.3405932E-3,1.414835E8,7.94E2,3.168229E-4,-2.1902737E-3,-2.379334E-3,4.4525E-4],"split_indices":[60,112,7,53,40,0,0,36,65,0,0,64,35,65,40,0,0,0,0,0,0,53,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.28E2,1.17E2,1.1E1,1.09E2,8E0,5E0,6E0,4.6E1,6.3E1,4E0,4E0,3E1,1.6E1,1.2E1,5.1E1,1.2E1,1.8E1,8E0,8E0,4E0,8E0,3.5E1,1.6E1,3.1E1,4E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.1728502E-3,-5.783183E-3,3.4078385E-3,1.0430581E-2,-5.621802E-3,5.6063716E-4,1.1345437E-1,-9.58631E-3,3.274179E-2,8.627159E-3,1.1256407E-3,4.1695973E-3,-3.060112E-2,5.3958375E-2,-1.1251905E-5,4.029922E-2,-1.1767412E-2,-3.2633485E-3,-1.7787145E-2,1.7149771E-4,3.521885E-3,7.2500465E-4,3.4031405E-3,-2.9043048E-3,-1.2363061E-4,-3.1028083E-3,-2.1176263E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,15,17,19,-1,21,23,-1,25,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.810276E-2,0E0,9.3095906E-2,1.04670815E-1,0E0,3.1335216E-2,5.0098546E-2,2.1230273E-2,1.610507E-2,0E0,0E0,2.638513E-2,1.2805769E-2,1.36884E-2,0E0,9.805551E-3,1.42485155E-2,0E0,1.4202915E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,18,18],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,16,18,20,-1,22,24,-1,26,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.7571955E6,-5.783183E-3,1.5081978E8,1.1811867E4,-5.621802E-3,9.7752365E1,1.3309696E7,3.925844E7,1.0533929E1,8.627159E-3,1.1256407E-3,1.1085E4,9.433962E-3,8.496393E5,-1.1251905E-5,1.0252E4,3.9245284E0,-3.2633485E-3,3.94526E5,1.7149771E-4,3.521885E-3,7.2500465E-4,3.4031405E-3,-2.9043048E-3,-1.2363061E-4,-3.1028083E-3,-2.1176263E-4],"split_indices":[53,0,53,60,0,64,55,53,61,0,0,9,65,40,0,9,64,0,37,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.12E2,5E0,1.07E2,1.02E2,5E0,9.4E1,8E0,7.2E1,2.2E1,4E0,4E0,4.4E1,2.8E1,1.3E1,9E0,1.3E1,3.1E1,6E0,2.2E1,4E0,9E0,8E0,5E0,4E0,2.7E1,4E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-1.6212905E-3,-6.3450118E-3,3.4780803E-3,8.795503E-3,-6.2157004E-3,5.636164E-3,4.2094886E-3,-1.52566815E-2,1.9606013E-2,-7.846746E-3,-4.1216398E-3,5.9319545E-2,9.140642E-3,-1.3899805E-2,2.3597009E-3,8.523632E-4,4.2117815E-3,3.0862226E-3,3.521901E-3,-2.5204184E-2,9.6610276E-4,-1.6625442E-3,1.2565331E-2,1.1428843E-3,-1.5433208E-3,2.74983E-4,2.1862846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,-1,3,5,-1,-1,7,9,11,13,-1,15,17,19,-1,-1,-1,-1,21,23,-1,-1,25,-1,-1,-1,-1],"loss_changes":[8.947087E-2,0E0,9.280873E-2,6.211517E-2,0E0,0E0,3.65642E-2,2.77517E-2,2.7719729E-2,1.8015947E-2,0E0,1.5189473E-2,1.6918367E-2,1.8257223E-2,0E0,0E0,0E0,0E0,1.811792E-2,1.3004256E-2,0E0,0E0,9.232624E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,8,8,9,9,11,11,12,12,13,13,18,18,19,19,22,22],"right_children":[2,-1,4,6,-1,-1,8,10,12,14,-1,16,18,20,-1,-1,-1,-1,22,24,-1,-1,26,-1,-1,-1,-1],"split_conditions":[3.3171368E7,-6.3450118E-3,1.54005E8,6.169213E0,-6.2157004E-3,5.636164E-3,1.4042297E3,1.2028319E6,1.5046106E7,1.4617647E1,-4.1216398E-3,7.359468E8,9.91707E5,2.3145E4,2.3597009E-3,8.523632E-4,4.2117815E-3,3.0862226E-3,2.4727786E3,1.9071268E3,9.6610276E-4,-1.6625442E-3,4.3348173E8,1.1428843E-3,-1.5433208E-3,2.74983E-4,2.1862846E-3],"split_indices":[5,0,53,62,0,0,60,36,53,61,0,5,1,9,0,0,0,0,60,56,0,0,40,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.32E2,4E0,1.28E2,1.24E2,4E0,4E0,1.2E2,5.3E1,6.7E1,4.9E1,4E0,1.3E1,5.4E1,4.5E1,4E0,6E0,7E0,4E0,5E1,3.4E1,1.1E1,9E0,4.1E1,4E0,3E1,3.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.546468E-3,-5.9511987E-3,9.417727E-3,1.6814845E-2,-5.9250556E-3,8.641214E-3,6.320452E-3,3.0665092E-2,-9.6041355E-3,9.667961E-5,5.4642286E-2,-2.2575182E-3,4.0770215E-3,8.053752E-4,8.8217236E-2,2.1869873E-3,-2.7981196E-3,5.204892E-3,1.5849328E-3,-8.344976E-4,9.4994396E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,-1,3,5,-1,7,-1,9,11,-1,13,-1,15,-1,17,-1,19,-1,-1,-1,-1],"loss_changes":[1.1835305E-1,0E0,1.12675354E-1,9.989788E-2,0E0,4.087968E-2,0E0,3.147864E-2,2.9051075E-2,0E0,3.0222334E-2,0E0,1.2174226E-2,0E0,1.1241622E-2,0E0,1.2667991E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,10,10,12,12,14,14,16,16],"right_children":[2,-1,4,6,-1,8,-1,10,12,-1,14,-1,16,-1,18,-1,20,-1,-1,-1,-1],"split_conditions":[1.458E3,-5.9511987E-3,1.2511909E8,1.2769175E4,-5.9250556E-3,2.8129198E7,6.320452E-3,1.6984678E3,6.336E3,9.667961E-5,3.136809E5,-2.2575182E-3,1.013E3,8.053752E-4,2.1440733E5,2.1869873E-3,7.525053E6,5.204892E-3,1.5849328E-3,-8.344976E-4,9.4994396E-4],"split_indices":[2,0,53,60,0,53,0,4,2,0,36,0,11,0,41,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,6E0,1.11E2,1.06E2,5E0,1E2,6E0,4.5E1,5.5E1,2.1E1,2.4E1,1.4E1,4.1E1,1.2E1,1.2E1,5E0,3.6E1,8E0,4E0,2.2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.3937518E-3,-5.932995E-3,1.2092182E-3,6.951291E-3,-4.8865196E-3,5.0825346E-3,2.8175109E-3,1.2500692E-2,-2.3676544E-2,5.863658E-3,2.6339223E-3,-5.9313077E-4,-3.4509995E-3,1.093427E-2,-2.7532373E-3,-3.638461E-3,2.3412053E-2,2.3746528E-2,-7.422519E-4,5.0209E-2,1.6501793E-4,-9.7136636E-4,3.0504747E-3,6.951428E-2,1.5426766E-4,4.6035373E-3,1.2687712E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,-1,3,5,-1,-1,7,9,11,13,-1,-1,-1,15,-1,17,19,21,-1,23,-1,-1,-1,25,-1,-1,-1],"loss_changes":[7.579633E-2,0E0,7.918224E-2,5.1511798E-2,0E0,0E0,3.1594675E-2,2.5154099E-2,1.7150633E-2,2.6073033E-2,0E0,0E0,0E0,1.3541002E-2,0E0,1.177862E-2,2.121763E-2,2.0759583E-2,0E0,1.5193388E-2,0E0,0E0,0E0,1.1292554E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17,19,19,23,23],"right_children":[2,-1,4,6,-1,-1,8,10,12,14,-1,-1,-1,16,-1,18,20,22,-1,24,-1,-1,-1,26,-1,-1,-1],"split_conditions":[3.3171368E7,-5.932995E-3,2.8841052E10,1.8660948E2,-4.8865196E-3,5.0825346E-3,6.5566176E7,1.3519049E9,2.5739892E6,1.2410836E6,2.6339223E-3,-5.9313077E-4,-3.4509995E-3,1.632896E3,-2.7532373E-3,1.2410928E8,1.1663035E9,2.5332516E5,-7.422519E-4,2.8122288E3,1.6501793E-4,-9.7136636E-4,3.0504747E-3,4.37E2,1.5426766E-4,4.6035373E-3,1.2687712E-3],"split_indices":[5,0,5,41,0,0,53,7,40,36,0,0,0,4,0,5,5,36,0,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.35E2,4E0,1.31E2,1.25E2,6E0,4E0,1.21E2,8.9E1,3.2E1,7.8E1,1.1E1,2.7E1,5E0,7.3E1,5E0,3.4E1,3.9E1,1E1,2.4E1,1.6E1,2.3E1,5E0,5E0,1.1E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[3.6839E-3,9.1067115E-3,-4.9897544E-3,1.0596268E-3,9.6713975E-2,1.5612123E-2,-1.9526718E-2,8.021199E-3,1.0135904E-3,-3.0518873E-3,3.748853E-2,-4.268216E-2,-1.4884441E-4,1.4502963E-3,-1.0748212E-2,6.0822573E-2,1.0965554E-4,-2.9233003E-3,-4.561664E-4,1.5463517E-3,-1.8628648E-2,2.4600988E-4,3.908341E-3,-1.9098931E-3,1.2127374E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,17,-1,-1,19,21,-1,-1,-1,-1,23,-1,-1,-1,-1],"loss_changes":[7.368569E-2,8.3790496E-2,0E0,3.325895E-2,5.0218232E-2,2.6654009E-2,1.7346095E-2,0E0,0E0,9.5860725E-3,2.449578E-2,1.1079513E-2,0E0,0E0,1.0799187E-2,2.0637974E-2,0E0,0E0,0E0,0E0,1.1859305E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,11,11,14,14,15,15,20,20],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,18,-1,-1,20,22,-1,-1,-1,-1,24,-1,-1,-1,-1],"split_conditions":[2.8841052E10,1.1811867E4,-4.9897544E-3,3.925844E7,1.3309696E7,2.5227545E1,8.0168776E-2,8.021199E-3,1.0135904E-3,4.56E3,2.70482E5,1.2775832E1,-1.4884441E-4,1.4502963E-3,2.4333723E3,3.0765435E10,1.0965554E-4,-2.9233003E-3,-4.561664E-4,1.5463517E-3,1.0666667E1,2.4600988E-4,3.908341E-3,-1.9098931E-3,1.2127374E-4],"split_indices":[5,60,0,53,55,64,65,0,0,37,37,62,0,0,55,39,0,0,0,0,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.23E2,1.18E2,5E0,1.09E2,9E0,6.4E1,4.5E1,4E0,5E0,3.5E1,2.9E1,1.8E1,2.7E1,6E0,2.9E1,1.7E1,1.2E1,1.1E1,7E0,4E0,2.5E1,5E0,1.2E1,1.2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[4.5447443E-3,-9.351735E-2,1.2049139E-2,4.946382E-4,-8.533605E-3,6.2519372E-3,4.88337E-3,-1.301581E-3,4.5350756E-2,8.041877E-3,-2.2521121E-2,3.0477324E-3,6.824376E-4,2.6526174E-3,1.4645758E-3,-3.3263562E-3,-3.4374333E-4,-1.7921987E-4,1.9464743E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,-1,-1,7,-1,9,11,13,15,-1,-1,-1,17,-1,-1,-1,-1],"loss_changes":[9.3113884E-2,8.1475124E-2,6.054478E-2,0E0,0E0,3.284106E-2,0E0,1.8817918E-2,9.412866E-3,2.0539517E-2,2.0442499E-2,0E0,0E0,0E0,1.2248371E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,8,8,9,9,10,10,14,14],"right_children":[2,4,6,-1,-1,8,-1,10,12,14,16,-1,-1,-1,18,-1,-1,-1,-1],"split_conditions":[5.279518E7,2.815298E8,1.2769175E4,4.946382E-4,-8.533605E-3,1.3536367E0,4.88337E-3,6.721145E7,4.5147678E-1,8.9E1,8.49491E5,3.0477324E-3,6.824376E-4,2.6526174E-3,5.8481636E3,-3.3263562E-3,-3.4374333E-4,-1.7921987E-4,1.9464743E-3],"split_indices":[5,39,60,0,0,47,0,53,65,10,1,0,0,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.24E2,8E0,1.16E2,4E0,4E0,1.1E2,6E0,9.3E1,1.7E1,6.5E1,2.8E1,1E1,7E0,7E0,5.8E1,6E0,2.2E1,5.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.9325301E-3,-2.7220075E-3,5.247521E-3,8.070399E-3,-2.9457018E-2,-3.5691794E-2,1.3283769E-2,-2.0829318E-3,6.2775024E-4,-3.2171733E-3,1.4902394E-4,2.5946481E-2,-7.6040963E-4,3.2422747E-2,-9.70346E-4,3.9455323E-3,2.1295112E-2,6.4962514E-4,2.9292079E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,7,9,11,-1,-1,-1,-1,13,-1,15,-1,-1,17,-1,-1],"loss_changes":[7.226297E-2,3.3130735E-2,0E0,1.9182298E-2,2.0477919E-2,1.1274239E-2,2.778062E-2,0E0,0E0,0E0,0E0,1.6239233E-2,0E0,2.3933712E-2,0E0,0E0,1.117795E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,11,11,13,13,16,16],"right_children":[2,4,-1,6,8,10,12,-1,-1,-1,-1,14,-1,16,-1,-1,18,-1,-1],"split_conditions":[1.2769175E4,6.721145E7,5.247521E-3,1.00441E5,1.9649E4,4E1,4.4E1,-2.0829318E-3,6.2775024E-4,-3.2171733E-3,1.4902394E-4,1E0,-7.6040963E-4,1.82E2,-9.70346E-4,3.9455323E-3,1.3519049E9,6.4962514E-4,2.9292079E-3],"split_indices":[60,53,0,1,2,3,3,0,0,0,0,74,0,10,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.18E2,1.13E2,5E0,8.1E1,3.2E1,8E0,7.3E1,2.4E1,8E0,4E0,4E0,5.1E1,2.2E1,4.5E1,6E0,7E0,3.8E1,3.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-4.137484E-3,-5.8921427E-3,1.4731706E-3,8.143047E-3,-5.3564194E-3,-4.566145E-4,8.730559E-2,3.199449E-2,-8.136394E-3,7.554466E-3,6.5367704E-4,2.7547574E-3,-3.4989088E-4,-2.8808776E-2,6.744774E-3,-1.8226286E-2,-3.4207513E-3,1.5614895E-3,-2.879817E-4,-1.4782844E-3,5.0953304E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,15,17,19,-1,-1,-1,-1,-1],"loss_changes":[7.444268E-2,0E0,8.2010865E-2,6.947852E-2,0E0,2.3437534E-2,5.0015576E-2,1.8604372E-2,2.3574129E-2,0E0,0E0,0E0,0E0,1.3194939E-2,1.51161915E-2,1.0260503E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,15,15],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,16,18,20,-1,-1,-1,-1,-1],"split_conditions":[1.369606E6,-5.8921427E-3,1.1971412E8,9.669896E3,-5.3564194E-3,7.7E2,3.0376984E1,2.1270042E1,2.518E3,7.554466E-3,6.5367704E-4,2.7547574E-3,-3.4989088E-4,1.1674061E6,1.5046106E7,7.2660637E-1,-3.4207513E-3,1.5614895E-3,-2.879817E-4,-1.4782844E-3,5.0953304E-4],"split_indices":[53,0,53,60,0,11,65,66,10,0,0,0,0,36,53,35,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.1E2,4E0,1.06E2,1.01E2,5E0,9.2E1,9E0,1.7E1,7.5E1,4E0,5E0,1E1,7E0,3.1E1,4.4E1,2.6E1,5E0,1.4E1,3E1,1.8E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-5.1197107E-3,-6.7576966E-3,1.126035E-3,5.8272686E-3,-4.7167195E-3,-1.0045878E-4,4.282672E-3,-1.973591E-3,3.93355E-3,1.9544272E-2,-6.9235587E-3,2.6665216E-3,1.099483E-2,-1.5738867E-3,1.0124764E-2,-1.2286664E-3,2.0458747E-2,1.1623286E-3,-4.5377758E-4,1.639128E-3,-4.5568115E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,-1,3,5,-1,7,-1,-1,9,11,13,-1,15,-1,17,-1,19,-1,-1,-1,-1],"loss_changes":[9.72657E-2,0E0,5.2232914E-2,5.2694835E-2,0E0,1.6792906E-2,0E0,0E0,1.5728146E-2,1.1176314E-2,2.4743438E-2,0E0,1.1647554E-2,0E0,9.716131E-3,0E0,1.1141278E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,8,8,9,9,10,10,12,12,14,14,16,16],"right_children":[2,-1,4,6,-1,8,-1,-1,10,12,14,-1,16,-1,18,-1,20,-1,-1,-1,-1],"split_conditions":[2.7571955E6,-6.7576966E-3,2.8841052E10,1.2769175E4,-4.7167195E-3,1.3369197E-5,4.282672E-3,-1.973591E-3,1.5270755E6,4.347826E-2,6.70699E-2,2.6665216E-3,7.482993E-2,-1.5738867E-3,3.8E1,-1.2286664E-3,3.3406117E11,1.1623286E-3,-4.5377758E-4,1.639128E-3,-4.5568115E-4],"split_indices":[53,0,5,60,0,46,0,0,40,65,46,0,65,0,3,0,39,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.13E2,4E0,1.09E2,1.05E2,4E0,9.9E1,6E0,8E0,9.1E1,3.7E1,5.4E1,6E0,3.1E1,2.1E1,3.3E1,6E0,2.5E1,1.9E1,1.4E1,1.7E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[8.999448E-3,5.380328E-3,4.373952E-3,-3.3850435E-2,1.1422045E-2,4.5146068E-4,-5.6290086E-2,-1.6946296E-3,1.5464608E-2,-4.59449E-3,-1.1607988E-3,4.4917352E-3,9.735164E-3,2.419875E-3,4.2208084E-3,-4.0005855E-4,2.3795562E-3,-1.8245191E-4,2.1408878E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,-1,3,5,7,-1,9,-1,11,-1,-1,-1,13,-1,15,17,-1,-1,-1],"loss_changes":[5.6086533E-2,0E0,3.1099267E-2,1.8023113E-2,1.894217E-2,0E0,1.2981415E-2,0E0,3.9609123E-2,0E0,0E0,0E0,1.8973468E-2,0E0,1.603601E-2,1.1000279E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,8,8,12,12,14,14,15,15],"right_children":[2,-1,4,6,8,-1,10,-1,12,-1,-1,-1,14,-1,16,18,-1,-1,-1],"split_conditions":[4.173E3,5.380328E-3,2.0979407E-4,2.8325282E11,1.00441E5,4.5146068E-4,9.07E2,-1.6946296E-3,1.8970638E1,-4.59449E-3,-1.1607988E-3,4.4917352E-3,7.81203E-2,2.419875E-3,1.1276298E9,1.45E2,2.3795562E-3,-1.8245191E-4,2.1408878E-3],"split_indices":[12,0,46,39,1,0,0,0,66,0,0,0,35,0,40,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.17E2,4E0,1.13E2,1.7E1,9.6E1,6E0,1.1E1,7E0,8.9E1,4E0,7E0,5E0,8.4E1,9E0,7.5E1,6.9E1,6E0,6.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-4.0560695E-3,5.599232E-4,-5.5338275E-3,4.815113E-3,-4.774468E-3,-5.811452E-3,4.4578392E-2,7.864682E-3,-3.6736052E-2,6.5374235E-3,2.0476708E-2,-9.946615E-3,2.5977392E-2,-8.278436E-4,-5.9281386E-2,1.9922343E-3,-3.3708516E-4,2.274696E-3,-1.9211296E-2,5.4225504E-2,3.4640162E-4,-2.8689843E-4,-3.9299666E-3,-2.560878E-3,-2.0346462E-4,4.022049E-3,7.872802E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,15,17,19,-1,21,-1,-1,-1,23,25,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5100506E-2,5.2672163E-2,0E0,5.0119616E-2,0E0,4.0044237E-2,5.281845E-2,2.1550702E-2,1.1657368E-2,0E0,1.2575904E-2,1.9051882E-2,1.7060913E-2,0E0,1.5441317E-2,0E0,0E0,0E0,1.5206207E-2,1.3327319E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,18,18,19,19],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,16,18,20,-1,22,-1,-1,-1,24,26,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.088501E0,1.54005E8,-5.5338275E-3,1.0296329E7,-4.774468E-3,2.9929216E9,2.2732E4,1.1679049E3,5.614829E5,6.5374235E-3,2.9492358E2,8.1837E4,9.382285E6,-8.278436E-4,6.737101E7,1.9922343E-3,-3.3708516E-4,2.274696E-3,2.7240606E5,2.7339255E3,3.4640162E-4,-2.8689843E-4,-3.9299666E-3,-2.560878E-3,-2.0346462E-4,4.022049E-3,7.872802E-4],"split_indices":[46,53,0,1,0,5,2,60,36,0,64,12,56,0,7,0,0,0,55,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.25E2,1.21E2,4E0,1.17E2,4E0,9.3E1,2.4E1,6.5E1,2.8E1,4E0,2E1,3.3E1,3.2E1,1.6E1,1.2E1,1.1E1,9E0,4E0,2.9E1,1.2E1,2E1,4E0,8E0,8E0,2.1E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.1998E-3,-6.201716E-3,2.1942859E-3,1.5930537E-2,-4.0511124E-2,6.161256E-3,9.170732E-3,-1.9375365E-2,-6.594632E-3,-3.1733E-2,1.7564973E-2,2.441153E-5,-1.9562014E-3,-5.7286257E-4,-3.2861056E-3,3.186167E-2,1.1381543E-4,-5.214455E-4,4.220773E-2,2.8327992E-3,1.8417178E-2,1.9607854E-3,-4.818052E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,-1,3,5,7,-1,9,11,-1,13,15,-1,-1,-1,-1,17,-1,-1,19,-1,21,-1,-1],"loss_changes":[8.3637476E-2,0E0,6.986561E-2,6.761427E-2,5.713278E-2,0E0,3.008598E-2,1.075287E-2,0E0,1.0151506E-2,1.5551582E-2,0E0,0E0,0E0,0E0,1.6837385E-2,0E0,0E0,1.1118818E-2,0E0,9.449651E-3,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,7,7,9,9,10,10,15,15,18,18,20,20],"right_children":[2,-1,4,6,8,-1,10,12,-1,14,16,-1,-1,-1,-1,18,-1,-1,20,-1,22,-1,-1],"split_conditions":[2.7803582E6,-6.201716E-3,7.150216E7,7.619835E0,1.1748485E1,6.161256E-3,5.981675E-4,8.180538E-1,-6.594632E-3,7.158023E7,1.343964E1,2.441153E-5,-1.9562014E-3,-5.7286257E-4,-3.2861056E-3,1.3101191E1,1.1381543E-4,-5.214455E-4,1.3783542E7,2.8327992E-3,2.5975403E-1,1.9607854E-3,-4.818052E-4],"split_indices":[53,0,53,61,61,0,50,35,0,7,62,0,0,0,0,64,0,0,59,0,46,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,4E0,1.17E2,8.9E1,2.8E1,4E0,8.5E1,2.4E1,4E0,1.4E1,7.1E1,1.3E1,1.1E1,1E1,4E0,3.6E1,3.5E1,7E0,2.9E1,1.6E1,1.3E1,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.6110213E-3,3.103629E-3,-4.434511E-3,8.297294E-3,-5.1893317E-3,2.7135261E-3,4.1872133E-3,-5.572295E-2,8.339491E-3,-4.110763E-3,-6.793106E-4,3.7837443E-3,2.408825E-3,-7.646179E-4,4.2277094E-2,1.9293275E-3,-4.8122313E-3,1.3406549E-4,3.5001712E-3,-3.0309774E-3,-1.7345024E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,-1,7,-1,9,11,-1,-1,13,-1,15,17,-1,19,-1,-1,-1,-1],"loss_changes":[6.0627285E-2,6.5333866E-2,0E0,4.7188997E-2,0E0,3.3669617E-2,0E0,1.0112442E-2,1.7646216E-2,0E0,0E0,1.491427E-2,0E0,1.3023713E-2,1.0811459E-2,0E0,1.8504875E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,11,11,13,13,14,14,16,16],"right_children":[2,4,-1,6,-1,8,-1,10,12,-1,-1,14,-1,16,18,-1,20,-1,-1,-1,-1],"split_conditions":[6.313997E-1,1.5081978E8,-4.434511E-3,9.8E1,-5.1893317E-3,1.4528436E-4,4.1872133E-3,8.76563E5,2.437707E0,-4.110763E-3,-6.793106E-4,1E0,2.408825E-3,2.8017812E3,3.2E1,1.9293275E-3,2.7272415E2,1.3406549E-4,3.5001712E-3,-3.0309774E-3,-1.7345024E-5],"split_indices":[46,53,0,8,0,47,0,37,47,0,0,84,0,55,3,0,60,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.16E2,1.1E2,6E0,1.06E2,4E0,1E2,6E0,8E0,9.2E1,4E0,4E0,8.4E1,8E0,7.6E1,8E0,6E0,7E1,4E0,4E0,4E0,6.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.5181267E-3,-6.0148817E-3,2.4392158E-3,7.860272E-3,-6.8305545E-2,4.66431E-3,3.8169236E-3,-6.1451206E-3,2.7341826E-4,-2.2418797E-2,1.272303E-2,7.5600535E-4,-1.9721463E-3,-1.2072509E-2,2.3255711E-2,1.1791997E-3,-1.6845559E-3,-1.5057779E-3,3.2749612E-4,4.5724242E-4,1.947449E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,-1,3,5,7,-1,9,-1,-1,11,13,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9240315E-2,0E0,4.8475817E-2,4.2112667E-2,4.095959E-2,0E0,2.6688602E-2,0E0,0E0,1.2717722E-2,2.2479657E-2,1.2944501E-2,0E0,9.792901E-3,1.399361E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,6,6,9,9,10,10,11,11,13,13,14,14],"right_children":[2,-1,4,6,8,-1,10,-1,-1,12,14,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.369606E6,-6.0148817E-3,1.2511909E8,1.8660948E2,2.7140412E5,4.66431E-3,1E0,-6.1451206E-3,2.7341826E-4,2.28569E5,1.2542795E3,3.925844E7,-1.9721463E-3,9.866438E0,4.2298006E5,1.1791997E-3,-1.6845559E-3,-1.5057779E-3,3.2749612E-4,4.5724242E-4,1.947449E-3],"split_indices":[53,0,53,41,40,0,22,0,0,12,60,53,0,61,36,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.28E2,4E0,1.24E2,1.16E2,8E0,4E0,1.12E2,4E0,4E0,2.8E1,8.4E1,1.3E1,1.5E1,2.5E1,5.9E1,8E0,5E0,1.2E1,1.3E1,3.4E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.664764E-3,-5.745346E-3,2.473164E-3,7.5181737E-3,-4.3186927E-3,8.393746E-4,5.0465614E-3,-3.0003453E-2,6.7881094E-3,-3.6629703E-4,-2.9042177E-3,-4.082966E-3,2.4174023E-2,5.333209E-4,-9.700271E-4,-7.0076226E-4,3.669149E-2,5.7114303E-2,7.214447E-4,8.7441265E-4,3.7765384E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,-1,3,5,-1,7,-1,9,11,-1,-1,13,15,-1,-1,-1,17,19,-1,-1,-1],"loss_changes":[7.2223835E-2,0E0,5.4580577E-2,7.2021775E-2,0E0,1.928515E-2,0E0,1.12308115E-2,1.6731018E-2,0E0,0E0,1.3839111E-2,1.7071683E-2,0E0,0E0,0E0,1.0598607E-2,9.632148E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,11,11,12,12,16,16,17,17],"right_children":[2,-1,4,6,-1,8,-1,10,12,-1,-1,14,16,-1,-1,-1,18,20,-1,-1,-1],"split_conditions":[4.0748124E7,-5.745346E-3,1.2511909E8,1.2769175E4,-4.3186927E-3,1.7214674E1,5.0465614E-3,7.105352E5,4.2298006E5,-3.6629703E-4,-2.9042177E-3,2.2010338E5,1.1788061E4,5.333209E-4,-9.700271E-4,-7.0076226E-4,2.0752128E7,8.116438E0,7.214447E-4,8.7441265E-4,3.7765384E-3],"split_indices":[5,0,53,60,0,66,0,36,36,0,0,36,55,0,0,0,53,61,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.18E2,4E0,1.14E2,1.09E2,5E0,1.03E2,6E0,1.6E1,8.7E1,1E1,6E0,5.4E1,3.3E1,2.8E1,2.6E1,8E0,2.5E1,1.2E1,1.3E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.6779293E-3,1.0543333E-2,-8.445534E-2,2.0371323E-2,-3.4418687E-2,-6.5921247E-3,-6.6785165E-4,1.4113069E-2,4.138881E-3,-2.947413E-4,-4.96707E-3,-1.6567063E-3,1.8107112E-2,3.0506048E-2,-2.1018961E-4,6.314491E-2,1.596058E-2,6.8217854E-4,3.7365607E-3,-4.0347865E-4,3.612271E-2,2.6513743E-3,4.4107428E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,13,15,-1,17,19,-1,-1,-1,21,-1,-1],"loss_changes":[7.531411E-2,5.1518083E-2,3.2052055E-2,3.8191173E-2,4.0637262E-2,0E0,0E0,1.77814E-2,0E0,0E0,0E0,0E0,2.3018802E-2,2.4153784E-2,0E0,1.06842965E-2,1.900655E-2,0E0,0E0,0E0,1.0438418E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,12,12,13,13,15,15,16,16,20,20],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,14,16,-1,18,20,-1,-1,-1,22,-1,-1],"split_conditions":[1E0,7.752181E7,1.0931163E9,1.3436509E4,1.1717857E1,-6.5921247E-3,-6.6785165E-4,4.9541533E-5,4.138881E-3,-2.947413E-4,-4.96707E-3,-1.6567063E-3,4.3E1,2.2333334E1,-2.1018961E-4,2.7213043E2,6.506745E8,6.8217854E-4,3.7365607E-3,-4.0347865E-4,2.1440733E5,2.6513743E-3,4.4107428E-4],"split_indices":[112,53,5,4,61,0,0,50,0,0,0,0,3,66,0,60,12,0,0,0,41,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.22E2,1.14E2,8E0,9.4E1,2E1,4E0,4E0,8.7E1,7E0,1.5E1,5E0,6E0,8.1E1,5.2E1,2.9E1,1.5E1,3.7E1,4E0,1.1E1,1.7E1,2E1,1.1E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[4.7668585E-4,-5.2989344E-3,5.9531024E-3,1.0563691E-2,-4.0567317E-3,4.9063964E-3,4.1563273E-3,2.2068612E-2,-8.736426E-3,2.507188E-3,4.407252E-2,-1.638572E-3,-2.6954971E-3,2.0018376E-3,-5.133337E-4,3.354283E-3,1.2147353E-3,-1.7683357E-2,4.988786E-4,-1.5185276E-4,-2.5049543E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,-1,3,5,-1,7,-1,9,11,13,15,17,-1,-1,-1,-1,-1,19,-1,-1,-1],"loss_changes":[7.81629E-2,0E0,5.2027106E-2,5.0186995E-2,0E0,2.594832E-2,0E0,2.098046E-2,2.0901214E-2,1.4556015E-2,9.428579E-3,1.0849111E-2,0E0,0E0,0E0,0E0,0E0,1.1935843E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,17,17],"right_children":[2,-1,4,6,-1,8,-1,10,12,14,16,18,-1,-1,-1,-1,-1,20,-1,-1,-1],"split_conditions":[2.1662036E9,-5.2989344E-3,1.3649481E10,1.2769175E4,-4.0567317E-3,2.8399336E7,4.1563273E-3,1.7694729E3,1E0,1.2410928E8,5.572903E1,1.1637191E1,-2.6954971E-3,2.0018376E-3,-5.133337E-4,3.354283E-3,1.2147353E-3,9E0,4.988786E-4,-1.5185276E-4,-2.5049543E-3],"split_indices":[39,0,12,60,0,53,0,4,92,5,66,62,0,0,0,0,0,25,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.26E2,5E0,1.21E2,1.16E2,5E0,1.09E2,7E0,4.8E1,6.1E1,2.6E1,2.2E1,5.4E1,7E0,6E0,2E1,8E0,1.4E1,2.3E1,3.1E1,1.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.1867421E-3,6.944058E-3,-5.278242E-3,5.164519E-3,2.4446403E-3,8.0811065E-3,-4.0529906E-3,3.7519857E-3,2.9478446E-3,7.308753E-3,-3.1519372E-3,-1.964029E-2,1.5950216E-2,-1.04818086E-4,-2.0938723E-3,3.4117114E-3,1.0003551E-2,3.3542234E-5,1.7097516E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,-1,5,7,-1,9,-1,11,-1,13,15,-1,-1,-1,17,-1,-1],"loss_changes":[6.618352E-2,5.346477E-2,0E0,0E0,5.6788683E-2,2.4849867E-2,0E0,2.5539523E-2,0E0,2.2902546E-2,0E0,1.0037835E-2,2.3703469E-2,0E0,0E0,0E0,1.629138E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,7,7,9,9,11,11,12,12,16,16],"right_children":[2,4,-1,-1,6,8,-1,10,-1,12,-1,14,16,-1,-1,-1,18,-1,-1],"split_conditions":[6.313997E-1,6.169213E0,-5.278242E-3,5.164519E-3,1.1971412E8,2.4241872E0,-4.0529906E-3,1.6171268E6,2.9478446E-3,1E0,-3.1519372E-3,1.2903E4,1.8041044E1,-1.04818086E-4,-2.0938723E-3,3.4117114E-3,1.3519049E9,3.3542234E-5,1.7097516E-3],"split_indices":[46,62,0,0,53,47,0,36,0,22,0,9,66,0,0,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.21E2,1.17E2,4E0,4E0,1.13E2,1.07E2,6E0,1E2,7E0,9.6E1,4E0,2.3E1,7.3E1,1.4E1,9E0,6E0,6.7E1,5E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"6.2244874E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics index e5ad65714..37e6261fd 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure.metrics @@ -1,116 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,executorCPUTime_mean,0.11096189,3084.1367948933903,5116.547031434917,79.51624548736461,529.9750264230711,1391.67585446527,3099.3812093299503,46124.06428571429 -1,sr_remoteBytesReadRatio,0.10843087,0.821447639429395,1.190295307104319,4.361993847958805e-07,0.004681915012233263,0.2507003788881129,1.3747557349907953,9.945049508262114 -2,duration_max,0.08590039,13014.565543071161,23588.431561357003,453.0,3392.5,6114.0,11157.5,238607.0 -3,input_bytesRead_mean,0.07575204,35669048.99371199,38136124.990772754,459755.9357142857,6486236.150910443,19828931.08045977,55778735.98521887,172540384.9351621 -4,sw_recordsWritten_sum,0.06681385,1374410117.7790263,3387308853.036538,200.0,1055515.5,70762368.0,1083955333.0,32214909337.0 -5,peakExecutionMemory_max,0.064965226,625983011.6554307,1239878879.6041248,0.0,93695248.0,218169344.0,609353728.0,11609833472.0 -6,sw_bytesWrittenRatio,0.05561869,0.8869646693077999,1.2280184306032964,5.108378528816216e-07,0.003869714259491267,0.35905628936826683,1.4713609319883258,11.3434662691877 -7,shuffle_read_bw,0.04526964,170230419.00411582,868425238.6185787,0.0,672086.8175217048,3162045.938095238,19115831.030837156,12002025920.666666 -8,cache_hit_ratio,0.031706758,0.5818479081056788,0.3103653453688123,1.187602344157431e-05,0.3420876939265072,0.6498737129489489,0.8626540832583185,1.0 -9,sr_totalBytesRead_mean,0.029667305,18229103.83876161,35775424.5056799,78.7411003236246,248444.1857191991,3494626.944915254,17084917.87147192,251271497.82890856 -10,jvmGCTime_mean,0.028404517,85.95293278190968,187.4390674747875,0.0,11.372529335653415,30.870352716873214,71.1075879639193,2096.9339285714286 -11,shuffle_write_bw,0.02362022,117972.64852185854,98184.76955373441,78.48709677419355,4113.728138675776,111748.20965692503,209729.1539859219,362052.3134449244 -12,sr_remoteBytesRead_mean,0.023481352,14993982.41990812,30930608.85769257,67.23624595469256,179231.31632904644,2527588.171821306,13247257.400636984,219808445.37463126 -13,duration_mean,0.021401376,4111.606057920529,5885.389553551322,165.25,1127.7932454448978,2285.394183483127,4375.702988070108,49986.05357142857 -14,input_recordsRead_sum,0.021248274,5493571789.872659,16697912537.18372,600000.0,279581211.0,1323566726.0,5501298942.5,230400506548.0 -15,scan_time,0.020554502,1429634.5767790263,6248258.313808985,157.0,54414.5,231308.0,883806.0,91377236.0 -16,data_size,0.020303203,503941425933.2809,1845124535803.31,57436094.0,17447415178.0,77919859708.0,284879125360.5,22044104226233.0 -17,scan_bw,0.020207189,546795.340357651,924062.3721427768,10051.367316990334,158274.13560895674,299062.0855115105,677896.4525754678,12927459.114 -18,duration_sum,0.019760216,4309284.955056179,8830994.729334807,661.0,276294.5,980699.0,3542482.0,74511844.0 -19,sr_localBytesReadRatio,0.017992433,0.17759885752115132,0.29873276214277433,0.0,0.000931286144792052,0.06185394835231594,0.2359029041051135,2.2948561663133886 -20,executorDeserializeTime_mean,0.016531114,15.251722354571783,17.94773710910694,3.6990291262135924,9.736833142773737,12.014332965821389,15.4860520606015,200.2536443148688 -21,executorDeserializeCPUTime_mean,0.010857376,10.520917481855633,5.154766500551515,3.7119741100323624,8.437218721872187,9.98870056497175,11.321838330755423,60.04373177842566 -22,resultSize_max,0.009270817,103628.91760299625,322863.3680497398,5167.0,9168.5,13045.0,21886.5,2687556.0 -23,sqlOp_Window,0.008974059,0.07865168539325842,0.26969987919421534,0.0,0.0,0.0,0.0,1.0 -24,sr_localBlocksFetched_sum,0.008068115,7093.528089887641,13338.957317489889,0.0,179.0,1760.0,7417.0,94615.0 -25,sr_remoteBlocksFetched_sum,0.007611187,49432.284644194755,93232.8358643141,3.0,1200.5,12270.0,51788.0,661803.0 -26,duration_min,0.007406523,39.47191011235955,25.912558232588758,4.0,22.0,37.0,44.5,321.0 -27,sw_writeTime_mean,0.0063885525,80.64291615355337,132.4893162447752,0.2717391304347826,18.232321339330333,33.43801652892562,71.4602401023634,904.0 -28,sr_fetchWaitTime_mean,0.005851487,24.41398828526225,163.0148145392446,0.0,0.04514407355772622,0.26666666666666666,1.5420825688073396,2439.190265486726 -29,resultSerializationTime_sum,0.0054286686,20.53932584269663,46.60529904197508,0.0,1.0,5.0,17.5,403.0 -30,sqlOp_SubqueryBroadcast,0.003887681,0.7752808988764045,0.4181810501732485,0.0,1.0,1.0,1.0,1.0 -31,sr_localBytesRead_mean,0.0034266065,3235121.4188534864,8073494.560600611,0.0,37941.34757286536,512787.9633507853,2592251.0058443258,96841040.74163242 -32,numTasks_sum,0.0034102306,729.1985018726592,1003.3357980291394,4.0,238.5,404.0,913.5,11794.0 -33,sqlOp_Sort,0.0026515697,0.7265917602996255,0.44654563247974927,0.0,0.0,1.0,1.0,1.0 -34,sqlOp_RunningWindowFunction,0.0014497552,0.06741573033707865,0.25121147708214225,0.0,0.0,0.0,0.0,1.0 -35,sr_totalBytesReadRatio,0.0012670225,0.9990464969505464,1.365559384793236,5.108378528816216e-07,0.005347120858719093,0.3978628497915356,1.6787480002630113,11.365839535188458 -36,numExecutors,0.0010392942,8.239700374531836,0.42770227167428976,8.0,8.0,8.0,8.0,9.0 -37,sqlOp_ObjectHashAggregate,0.00076275755,0.12734082397003746,0.33398039193838686,0.0,0.0,0.0,0.0,1.0 -38,sw_bytesWritten_mean,0.0007160791,16768197.572215984,33741236.27880933,65.79642190944548,81878.14984010742,2995362.978313253,15229625.137337428,251266600.89970502 -39,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -40,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -41,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -42,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_HashAggregate,0.0,0.9887640449438202,0.10560054067615239,0.0,1.0,1.0,1.0,1.0 -48,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -52,sqlOp_Expand,0.0,0.10861423220973783,0.3117389214311049,0.0,0.0,0.0,0.0,1.0 -53,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,sw_recordsWritten_sum,0.11849262,1374410117.7790263,3387308853.036538,200.0,1055515.5,70762368.0,1083955333.0,32214909337.0 +1,sr_remoteBytesReadRatio,0.10528322,0.821447639429395,1.190295307104319,4.361993847958805e-07,0.004681915012233263,0.2507003788881129,1.3747557349907953,9.945049508262114 +2,executorCPUTime_mean,0.099570684,3084.1367948933903,5116.547031434917,79.51624548736461,529.9750264230711,1391.67585446527,3099.3812093299503,46124.06428571429 +3,input_bytesRead_mean,0.06114182,35669048.99371199,38136124.990772754,459755.9357142857,6486236.150910443,19828931.08045977,55778735.98521887,172540384.9351621 +4,duration_max,0.059728656,13014.565543071161,23588.431561357003,453.0,3392.5,6114.0,11157.5,238607.0 +5,shuffle_read_bw,0.055303685,170230419.00411582,868425238.6185787,0.0,672086.8175217048,3162045.938095238,19115831.030837156,12002025920.666666 +6,peakExecutionMemory_max,0.05034364,625983011.6554307,1239878879.6041248,0.0,93695248.0,218169344.0,609353728.0,11609833472.0 +7,sw_bytesWrittenRatio,0.047951605,0.8869646693077999,1.2280184306032964,5.108378528816216e-07,0.003869714259491267,0.35905628936826683,1.4713609319883258,11.3434662691877 +8,jvmGCTime_mean,0.030100452,85.95293278190968,187.4390674747875,0.0,11.372529335653415,30.870352716873214,71.1075879639193,2096.9339285714286 +9,duration_mean,0.025482303,4111.606057920529,5885.389553551322,165.25,1127.7932454448978,2285.394183483127,4375.702988070108,49986.05357142857 +10,cache_hit_ratio,0.024056798,0.5818479081056788,0.3103653453688123,1.187602344157431e-05,0.3420876939265072,0.6498737129489489,0.8626540832583185,1.0 +11,sw_bytesWritten_mean,0.023445426,16768197.572215984,33741236.27880933,65.79642190944548,81878.14984010742,2995362.978313253,15229625.137337428,251266600.89970502 +12,sr_localBytesReadRatio,0.02276076,0.17759885752115132,0.29873276214277433,0.0,0.000931286144792052,0.06185394835231594,0.2359029041051135,2.2948561663133886 +13,sr_totalBytesRead_mean,0.022631863,18229103.83876161,35775424.5056799,78.7411003236246,248444.1857191991,3494626.944915254,17084917.87147192,251271497.82890856 +14,shuffle_write_bw,0.019281115,117972.64852185854,98184.76955373441,78.48709677419355,4113.728138675776,111748.20965692503,209729.1539859219,362052.3134449244 +15,input_recordsRead_sum,0.018100992,5493571789.872659,16697912537.18372,600000.0,279581211.0,1323566726.0,5501298942.5,230400506548.0 +16,data_size,0.015162328,503941425933.2809,1845124535803.31,57436094.0,17447415178.0,77919859708.0,284879125360.5,22044104226233.0 +17,duration_sum,0.014280071,4309284.955056179,8830994.729334807,661.0,276294.5,980699.0,3542482.0,74511844.0 +18,scan_time,0.01394606,1429634.5767790263,6248258.313808985,157.0,54414.5,231308.0,883806.0,91377236.0 +19,scan_bw,0.012751421,546795.340357651,924062.3721427768,10051.367316990334,158274.13560895674,299062.0855115105,677896.4525754678,12927459.114 +20,executorDeserializeTime_mean,0.011850291,15.251722354571783,17.94773710910694,3.6990291262135924,9.736833142773737,12.014332965821389,15.4860520606015,200.2536443148688 +21,sr_remoteBytesRead_mean,0.011608642,14993982.41990812,30930608.85769257,67.23624595469256,179231.31632904644,2527588.171821306,13247257.400636984,219808445.37463126 +22,sr_localBlocksFetched_sum,0.009764083,7093.528089887641,13338.957317489889,0.0,179.0,1760.0,7417.0,94615.0 +23,sr_totalBytesReadRatio,0.009626217,0.9990464969505464,1.365559384793236,5.108378528816216e-07,0.005347120858719093,0.3978628497915356,1.6787480002630113,11.365839535188458 +24,resultSize_max,0.007949857,103628.91760299625,322863.3680497398,5167.0,9168.5,13045.0,21886.5,2687556.0 +25,executorDeserializeCPUTime_mean,0.007629198,10.520917481855633,5.154766500551515,3.7119741100323624,8.437218721872187,9.98870056497175,11.321838330755423,60.04373177842566 +26,duration_min,0.005952773,39.47191011235955,25.912558232588758,4.0,22.0,37.0,44.5,321.0 +27,sr_remoteBlocksFetched_sum,0.0055641793,49432.284644194755,93232.8358643141,3.0,1200.5,12270.0,51788.0,661803.0 +28,sr_fetchWaitTime_mean,0.005506142,24.41398828526225,163.0148145392446,0.0,0.04514407355772622,0.26666666666666666,1.5420825688073396,2439.190265486726 +29,sqlOp_RunningWindowFunction,0.005262384,0.06741573033707865,0.25121147708214225,0.0,0.0,0.0,0.0,1.0 +30,sqlOp_Window,0.004690932,0.07865168539325842,0.26969987919421534,0.0,0.0,0.0,0.0,1.0 +31,resultSerializationTime_sum,0.004453542,20.53932584269663,46.60529904197508,0.0,1.0,5.0,17.5,403.0 +32,sr_localBytesRead_mean,0.0043537756,3235121.4188534864,8073494.560600611,0.0,37941.34757286536,512787.9633507853,2592251.0058443258,96841040.74163242 +33,numTasks_sum,0.0038608178,729.1985018726592,1003.3357980291394,4.0,238.5,404.0,913.5,11794.0 +34,sw_writeTime_mean,0.0035855446,80.64291615355337,132.4893162447752,0.2717391304347826,18.232321339330333,33.43801652892562,71.4602401023634,904.0 +35,sqlOp_SubqueryBroadcast,0.0031898357,0.7752808988764045,0.4181810501732485,0.0,1.0,1.0,1.0,1.0 +36,executorRunTime_mean,0.0020181602,4084.432467240084,5886.28991028309,144.25,1108.6970838377952,2244.473941368078,4357.135471851895,49963.330357142855 +37,sqlOp_Subquery,0.001850056,0.2247191011235955,0.4181810501732485,0.0,0.0,0.0,0.0,1.0 +38,numExecutors,0.0010192364,8.239700374531836,0.42770227167428976,8.0,8.0,8.0,8.0,9.0 +39,sqlOp_Sort,0.001007466,0.7265917602996255,0.44654563247974927,0.0,0.0,1.0,1.0,1.0 +40,sqlOp_Expand,0.00026082725,0.10861423220973783,0.3117389214311049,0.0,0.0,0.0,0.0,1.0 +41,sqlOp_ObjectHashAggregate,0.00025514147,0.12734082397003746,0.33398039193838686,0.0,0.0,0.0,0.0,1.0 +42,sqlOp_BroadcastNestedLoopJoin,0.00019233921,0.04119850187265917,0.19912228909086188,0.0,0.0,0.0,0.0,1.0 +43,sqlOp_Filter,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +44,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +45,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +46,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +47,sqlOp_BroadcastHashJoin,0.0,0.947565543071161,0.22332010811379224,0.0,1.0,1.0,1.0,1.0 +48,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +49,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +50,sqlOp_AQEShuffleRead,0.0,0.9176029962546817,0.2754849853147098,0.0,1.0,1.0,1.0,1.0 +51,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,memoryBytesSpilledRatio,0.0,0.13140823253187295,0.9393631585625732,0.0,0.0,0.0,0.0,9.641198498202698 +53,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 55,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_WindowGroupLimit,0.0,0.0149812734082397,0.1217058407722143,0.0,0.0,0.0,0.0,1.0 -59,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -63,sqlOp_Subquery,0.0,0.2247191011235955,0.4181810501732485,0.0,0.0,0.0,0.0,1.0 -64,sqlOp_SortMergeJoin,0.0,0.5617977528089888,0.4970981094018272,0.0,0.0,1.0,1.0,1.0 -65,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,memoryBytesSpilled_mean,0.0,11069065.0864877,79810143.77222423,0.0,0.0,0.0,0.0,874196883.2566372 -79,maxMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 -80,maxOnHeapMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 -81,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,executorMemory,0.0,47016.0,0.0,47016.0,47016.0,47016.0,47016.0,47016.0 -84,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,diskBytesSpilled_mean,0.0,2880697.370667337,23636965.460199267,0.0,0.0,0.0,0.0,314481379.9144543 -87,diskBytesSpilledRatio,0.0,0.03193087515173895,0.2585787901798907,0.0,0.0,0.0,0.0,3.4683004089982026 -88,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -89,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -91,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -94,memoryBytesSpilledRatio,0.0,0.13140823253187295,0.9393631585625732,0.0,0.0,0.0,0.0,9.641198498202698 -95,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -96,executorRunTime_mean,0.0,4084.432467240084,5886.28991028309,144.25,1108.6970838377952,2244.473941368078,4357.135471851895,49963.330357142855 -97,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -98,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -99,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_BroadcastNestedLoopJoin,0.0,0.04119850187265917,0.19912228909086188,0.0,0.0,0.0,0.0,1.0 -105,sqlOp_BroadcastHashJoin,0.0,0.947565543071161,0.22332010811379224,0.0,1.0,1.0,1.0,1.0 -106,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -108,sqlOp_AQEShuffleRead,0.0,0.9176029962546817,0.2754849853147098,0.0,1.0,1.0,1.0,1.0 -109,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -111,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -112,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -113,platform_databricks-azure,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +56,diskBytesSpilledRatio,0.0,0.03193087515173895,0.2585787901798907,0.0,0.0,0.0,0.0,3.4683004089982026 +57,sqlOp_SortMergeJoin,0.0,0.5617977528089888,0.4970981094018272,0.0,0.0,1.0,1.0,1.0 +58,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +59,maxMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 +60,maxOnHeapMem,0.0,26104509235.0,0.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0,26104509235.0 +61,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +62,executorMemory,0.0,47016.0,0.0,47016.0,47016.0,47016.0,47016.0,47016.0 +63,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +66,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_HashAggregate,0.0,0.9887640449438202,0.10560054067615239,0.0,1.0,1.0,1.0,1.0 +68,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +69,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,platform_databricks-azure,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +71,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,memoryBytesSpilled_mean,0.0,11069065.0864877,79810143.77222423,0.0,0.0,0.0,0.0,874196883.2566372 +82,diskBytesSpilled_mean,0.0,2880697.370667337,23636965.460199267,0.0,0.0,0.0,0.0,314481379.9144543 +83,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_WindowGroupLimit,0.0,0.0149812734082397,0.1217058407722143,0.0,0.0,0.0,0.0,1.0 114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg index b3350955a..e034c76b0 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"95"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0490951464","gamma":"0.109058358","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0490951464","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.109058358","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.600743413"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0450306572","gamma":"0.00794881675","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0450306572","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"9","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.00794881675","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.627399802"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json index 354742c0e..37a8eb9d3 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_AQEShuffleRead","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"95"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.072645E-1,-6.220444E-1,1.9525845E-1,-3.574447E-1,-8.611123E-1,3.9173424E-2,4.314317E-1,-2.3384592E-2,-4.897678E-3,-2.5254954E-2,-4.8926707E-2,-5.2764706E-2,1.2743061E-2,1.1988305E-2,2.5544703E-2,2.7265228E-2,-1.3291139E-2,-2.9731796E-3,6.560979E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,-1,-1,-1,17,-1,-1,-1],"loss_changes":[1.1801414E1,1.4634047E0,1.7525284E0,4.1193867E-1,1.4484501E-1,6.215492E-1,1.8914962E-1,0E0,0E0,0E0,0E0,3.9112976E-1,0E0,0E0,0E0,1.669099E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,15,15],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,-1,-1,-1,18,-1,-1,-1],"split_conditions":[1.0420895E-2,8.28645E-1,5.517496E-1,4.2332153E0,9.29295E5,1.6E1,4.650193E-1,-2.3384592E-2,-4.897678E-3,-2.5254954E-2,-4.8926707E-2,2.5042E4,1.2743061E-2,1.1988305E-2,2.5544703E-2,1.419E3,-1.3291139E-2,-2.9731796E-3,6.560979E-3],"split_indices":[37,26,41,53,27,3,26,0,0,0,0,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.7E1,4.7E1,1.4E1,1.3E1,2.9E1,1.8E1,9E0,5E0,5E0,8E0,2.1E1,8E0,7E0,1.1E1,1.6E1,5E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-5.796369E-2,-4.728356E-1,2.014669E-1,-5.9635687E-1,3.9015252E-3,9.149514E-2,2.4364186E-2,-4.3475255E-1,-4.436388E-2,1.6905628E-1,-1.1381246E-2,-2.4862707E-2,-1.0749935E-2,-3.074874E-3,2.2214615E-1,1.830552E-2,1.470209E-1,8.941129E-4,1.2739913E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1,15,-1,17,-1,-1],"loss_changes":[7.8446546E0,1.9575887E0,1.4134684E0,7.492504E-1,0E0,8.9179873E-1,0E0,1.5201807E-1,0E0,3.5832006E-1,0E0,0E0,0E0,0E0,2.0559418E-1,0E0,2.4156329E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,14,14,16,16],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1,16,-1,18,-1,-1],"split_conditions":[1.0420895E-2,1.962943E3,9.306648E-1,1.2427474E8,3.9015252E-3,1.0232127E12,2.4364186E-2,8.60448E5,-4.436388E-2,7.233348E10,-1.1381246E-2,-2.4862707E-2,-1.0749935E-2,-3.074874E-3,2.325535E7,1.830552E-2,1.381E3,8.941129E-4,1.2739913E-2],"split_indices":[37,4,38,44,0,30,0,1,0,30,0,0,0,0,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.7E1,4.4E1,2.2E1,5E0,3.3E1,1.1E1,1.6E1,6E0,2.7E1,6E0,1.1E1,5E0,5E0,2.2E1,6E0,1.6E1,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[4.2534076E-2,-4.109137E-1,2.1991473E-1,-5.4405105E-1,1.7082373E-3,6.1845727E-2,4.2809653E-1,-3.171518E-2,-1.5029646E-2,-9.706842E-3,1.3933185E-1,7.851151E-3,5.086054E-1,1.6156761E-2,6.5949626E-2,2.9382257E-2,1.5782023E-2,-5.597345E-3,1.5121883E-1,1.2171325E-2,-9.340632E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,-1,9,11,-1,-1,-1,13,-1,15,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[6.283697E0,1.3244188E0,1.8163209E0,2.1456003E-1,0E0,6.9125545E-1,4.368453E-1,0E0,0E0,0E0,3.403449E-1,0E0,1.1035776E-1,0E0,3.2414103E-1,0E0,0E0,0E0,2.3517966E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,10,10,12,12,14,14,18,18],"right_children":[2,4,6,8,-1,10,12,-1,-1,-1,14,-1,16,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[1.6405078E-2,4.3646493E0,5.451147E-1,7.801514E2,1.7082373E-3,2.3428571E0,2.4460206E0,-3.171518E-2,-1.5029646E-2,-9.706842E-3,2.8216465E0,7.851151E-3,3.1198547E0,1.6156761E-2,8.110699E5,2.9382257E-2,1.5782023E-2,-5.597345E-3,2.829E4,1.2171325E-2,-9.340632E-4],"split_indices":[40,53,41,4,0,55,55,0,0,0,52,0,53,0,31,0,0,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.1E1,5.5E1,1.6E1,5E0,3.2E1,2.3E1,1E1,6E0,7E0,2.5E1,6E0,1.7E1,6E0,1.9E1,1E1,7E0,6E0,1.3E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.9800555E-3,-3.698022E-1,2.938747E-1,-2.1407835E-2,-6.300276E-1,2.0628484E-1,3.9258026E-2,-9.047106E-3,1.3783639E-2,-1.57717E-2,-3.6301717E-2,7.225239E-2,3.5119575E-1,1.0640476E-2,1.3556869E-2,2.0926505E-2,8.46258E-3,-3.7834013E-3,8.711106E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,13,15,-1,17,-1,-1,-1,-1],"loss_changes":[8.498739E0,3.1596494E0,1.7399123E0,8.3914095E-1,4.3800068E-1,7.1269035E-1,0E0,0E0,0E0,0E0,0E0,1.751479E-1,1.680963E-1,0E0,2.5127968E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,11,11,12,12,14,14],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,14,16,-1,18,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,5.661358E3,6.9983356E2,5.689896E5,5.596155E-1,3.9258026E-2,-9.047106E-3,1.3783639E-2,-1.57717E-2,-3.6301717E-2,7.9600364E-2,3.3934937E0,1.0640476E-2,2.6597537E3,2.0926505E-2,8.46258E-3,-3.7834013E-3,8.711106E-3],"split_indices":[41,26,51,51,27,38,0,0,0,0,0,38,53,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,3.4E1,4.2E1,1.5E1,1.9E1,3.7E1,5E0,1E1,5E0,6E0,1.3E1,2E1,1.7E1,5E0,1.5E1,1.1E1,6E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[6.6942396E-3,-4.546203E-1,2.0676103E-1,-5.827659E-1,2.74271E-4,9.7575545E-2,3.582547E-2,-2.0036664E-2,-3.9222516E-2,-8.146377E-3,2.3013526E-1,1.1121949E-1,-1.0309598E-1,1.630624E-2,7.3477356E-3,9.844743E-3,-5.648591E-4,-1.1767243E-2,-5.319224E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,15,17,-1,-1,-1,-1,-1,-1],"loss_changes":[7.016027E0,1.3596454E0,2.9550443E0,3.5342503E-1,0E0,6.325258E-1,0E0,0E0,0E0,3.0614358E-1,1.14454865E-1,1.3382536E-1,1.8642808E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,11,11,12,12],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,16,18,-1,-1,-1,-1,-1,-1],"split_conditions":[4.823969E-3,1.962943E3,4.0235266E3,9.1569895E-1,2.74271E-4,5.096104E-1,3.582547E-2,-2.0036664E-2,-3.9222516E-2,1.6547014E5,2.156825E7,1.219233E6,5.7E1,1.630624E-2,7.3477356E-3,9.844743E-3,-5.648591E-4,-1.1767243E-2,-5.319224E-4],"split_indices":[38,4,51,26,0,41,0,0,0,32,44,31,8,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.2E1,5.2E1,1.7E1,5E0,4.4E1,8E0,1.1E1,6E0,2.5E1,1.9E1,1.1E1,1.4E1,7E0,1.2E1,6E0,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[3.1985685E-3,-4.3204668E-1,1.482242E-1,-1.1138079E-2,-2.9163573E-2,-1.9296741E-2,3.1063396E-1,-1.2987757E-1,1.7346075E-1,3.9995566E-1,6.289656E-3,3.035823E-3,-9.9546965E-3,1.4737003E-2,8.758123E-4,2.4306571E-2,1.3001604E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,-1,-1,7,9,11,13,15,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7981687E0,4.966805E-1,1.5567951E0,0E0,0E0,6.406719E-1,4.2155433E-1,2.780279E-1,2.1155488E-1,1.1314678E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9],"right_children":[2,4,6,-1,-1,8,10,12,14,16,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9956966E-1,1.4E1,3.8151306E-1,-1.1138079E-2,-2.9163573E-2,1.1813418E3,3.3027112E0,2.2519132E-1,1.23E2,1.1213404E6,6.289656E-3,3.035823E-3,-9.9546965E-3,1.4737003E-2,8.758123E-4,2.4306571E-2,1.3001604E-2],"split_indices":[56,3,41,0,0,51,53,26,8,31,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,1.8E1,5.6E1,9E0,9E0,2.8E1,2.8E1,1.8E1,1E1,1.8E1,1E1,5E0,1.3E1,5E0,5E0,9E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.4080282E-2,-3.8578752E-1,1.8343948E-1,-1.5024497E-1,-5.236113E-1,1.1118091E-1,2.951148E-2,-1.5254649E-2,1.7314352E-3,-1.6642386E-2,-3.8040355E-2,1.604537E-1,-1.077198E-2,-4.9628825E-3,2.0314433E-1,2.0489473E-2,1.400877E-1,1.1425914E-3,9.232242E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,13,-1,-1,15,-1,17,-1,-1],"loss_changes":[5.5038133E0,7.653775E-1,1.4119945E0,3.3842024E-1,4.7946024E-1,7.356386E-1,0E0,0E0,0E0,0E0,0E0,4.4481814E-1,0E0,0E0,3.8725293E-1,0E0,1.4689004E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,11,11,14,14,16,16],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,14,-1,-1,16,-1,18,-1,-1],"split_conditions":[7.2921924E-2,5.0415697E5,4.0235266E3,1.3E1,1.1686677E8,1.1269586E12,2.951148E-2,-1.5254649E-2,1.7314352E-3,-1.6642386E-2,-3.8040355E-2,2.00809E5,-1.077198E-2,-4.9628825E-3,1.3024019E7,2.0489473E-2,5.316E3,1.1425914E-3,9.232242E-3],"split_indices":[41,27,51,3,44,30,0,0,0,0,0,1,0,0,44,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,2.5E1,4.8E1,1E1,1.5E1,4.2E1,6E0,5E0,5E0,1E1,5E0,3.7E1,5E0,5E0,3.2E1,6E0,2.6E1,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[7.320848E-2,-2.4420697E-2,6.2570214E-1,-4.7607082E-1,6.977877E-2,3.924833E-2,1.7069722E-2,-2.9825568E-2,-1.3024502E-2,-3.675304E-2,2.4739392E-1,1.13071175E-2,-1.2908012E-2,1.3994735E-3,1.5634663E-2,1.4494884E-1,-9.155462E-2,1.34142395E-2,1.5032827E-3,-8.204917E-3,7.045104E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,15,-1,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[3.9528635E0,2.7135673E0,2.5334978E-1,1.4357567E-1,1.0119407E0,0E0,0E0,0E0,0E0,3.7253714E-1,3.0201757E-1,4.1192272E-1,0E0,0E0,0E0,1.822944E-1,1.3844934E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,11,11,15,15,16,16],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,16,-1,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[4.0235266E3,1.5278552E4,8.982776E9,9.408949E2,5.517496E-1,3.924833E-2,1.7069722E-2,-2.9825568E-2,-1.3024502E-2,1.2932927E1,8.69E2,3.5228616E-1,-1.2908012E-2,1.3994735E-3,1.5634663E-2,1.3E1,1.0390377E8,1.34142395E-2,1.5032827E-3,-8.204917E-3,7.045104E-4],"split_indices":[51,50,5,4,41,0,0,0,0,56,10,26,0,0,0,3,12,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.2E1,1E1,1E1,5.2E1,5E0,5E0,5E0,5E0,3.3E1,1.9E1,2.8E1,5E0,5E0,1.4E1,1.2E1,1.6E1,5E0,7E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.7427694E-2,-2.9992643E-1,8.911279E-2,-2.0627676E-2,-5.2185045E-3,1.3426933E-1,-1.1872929E-2,7.572837E-2,1.8342061E-2,1.3145997E-1,-2.8386184E-3,1.133267E-2,6.113696E-2,-1.7572212E-3,9.370952E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,-1,-1,7,-1,9,-1,11,-1,-1,13,-1,-1],"loss_changes":[2.1226983E0,4.2095172E-1,7.3486066E-1,0E0,0E0,5.601606E-1,0E0,2.680986E-1,0E0,1.6058901E-1,0E0,0E0,2.0803542E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,7,7,9,9,12,12],"right_children":[2,4,6,-1,-1,8,-1,10,-1,12,-1,-1,14,-1,-1],"split_conditions":[2.0113895E-2,1.528E3,1.15266744E8,-2.0627676E-2,-5.2185045E-3,3.6591887E3,-1.1872929E-2,1.7444956E5,1.8342061E-2,3.9130908E6,-2.8386184E-3,1.133267E-2,2.4106981E-1,-1.7572212E-3,9.370952E-3],"split_indices":[40,11,44,0,0,4,0,32,0,50,0,0,41,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.5E1,1.9E1,4.6E1,1.1E1,8E0,4.1E1,5E0,3.4E1,7E0,2.4E1,1E1,9E0,1.5E1,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.937342E-2,-1.7052306E-1,3.2243285E-1,7.04902E-3,-3.484476E-1,2.1515913E-1,3.0746972E-2,-4.9066176E-3,6.6146934E-3,-2.999712E-2,-1.600734E-1,1.3851354E-2,5.005458E-3,-1.8943113E-3,-1.2700653E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,13,-1,-1,-1,-1],"loss_changes":[3.7381773E0,1.2662337E0,6.8010354E-1,3.0034244E-1,8.91356E-1,1.22870445E-1,0E0,0E0,0E0,0E0,1.4577648E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,10,10],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,14,-1,-1,-1,-1],"split_conditions":[4.7985215E-2,6.7376137E-1,1.91E4,4.2332153E0,1.0781229E3,2.4832625E6,3.0746972E-2,-4.9066176E-3,6.6146934E-3,-2.999712E-2,1.5368E4,1.3851354E-2,5.005458E-3,-1.8943113E-3,-1.2700653E-2],"split_indices":[37,26,2,53,4,31,0,0,0,0,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.3E1,3.9E1,2.4E1,2E1,1.9E1,1.9E1,5E0,1.1E1,9E0,7E0,1.2E1,1.1E1,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-6.0487296E-3,-3.2768604E-1,1.7005561E-1,-4.9076512E-1,4.381319E-3,2.350399E-1,-9.9433444E-2,-1.6171254E-2,-3.286129E-2,1.4184609E-1,4.1795236E-1,2.136249E-3,-1.1086032E-2,-2.5427341E-4,1.9554526E-1,2.845483E-2,1.2949227E-2,1.5328273E-2,5.2323313E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,-1,17,-1,-1,-1,-1],"loss_changes":[4.643705E0,2.015569E0,9.5153594E-1,3.847084E-1,0E0,6.737063E-1,2.0853591E-1,0E0,0E0,2.3786682E-1,1.9605112E-1,0E0,0E0,0E0,1.950891E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,14,14],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,-1,18,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,1.8E2,8.616169E-1,4.381319E-3,2.8544061E9,1.4181119E12,-1.6171254E-2,-3.286129E-2,3.6696808E2,6.5560097E-1,2.136249E-3,-1.1086032E-2,-2.5427341E-4,2.325535E7,2.845483E-2,1.2949227E-2,1.5328273E-2,5.2323313E-3],"split_indices":[41,4,8,26,0,12,30,0,0,51,26,0,0,0,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.8E1,5.2E1,2E1,8E0,4.2E1,1E1,1.2E1,8E0,2.9E1,1.3E1,5E0,5E0,8E0,2.1E1,5E0,8E0,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-4.3166887E-2,-3.0879998E-1,1.4933993E-1,-4.7703913E-1,-2.6680864E-3,4.0612917E-2,2.6131046E-1,-2.4898025E-1,-3.5746824E-2,5.122726E-3,-6.2384936E-3,6.3061225E-3,-5.0605547E-2,1.734972E-2,7.142151E-3,-5.167443E-3,-1.6525716E-2,3.212682E-3,-7.0128473E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,-1,17,-1,-1,-1,-1,-1,-1],"loss_changes":[3.779057E0,1.5953314E0,5.12879E-1,9.639082E-1,1.7300643E-1,1.9077814E-1,1.7257595E-1,1.157006E-1,0E0,0E0,0E0,0E0,1.3778844E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,12,12],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,-1,18,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1576994E-2,1.7493458E3,5.517496E-1,1.0370839E6,2.805454E-1,3.652811E5,1.7329262E7,9.502012E2,-3.5746824E-2,5.122726E-3,-6.2384936E-3,6.3061225E-3,1.8020144E-1,1.734972E-2,7.142151E-3,-5.167443E-3,-1.6525716E-2,3.212682E-3,-7.0128473E-3],"split_indices":[37,4,41,27,26,27,44,4,0,0,0,0,40,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,3E1,4.2E1,1.9E1,1.1E1,2.2E1,2E1,1.1E1,8E0,6E0,5E0,1.1E1,1.1E1,1E1,1E1,5E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[6.161666E-3,-2.8142956E-1,1.4983879E-1,5.264393E-3,-2.4228318E-2,1.4601785E-2,2.652347E-1,1.2467932E-1,-1.1503882E-1,3.578173E-1,7.91129E-3,-5.6681415E-4,1.5132903E-2,-1.0279862E-2,6.974771E-4,8.275241E-3,2.5676768E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,-1,-1,7,9,11,13,15,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2238305E0,2.1958828E0,8.0762863E-1,0E0,0E0,3.7110594E-1,2.1218133E-1,3.5362798E-1,1.4930281E-1,3.4952545E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,7,7,8,8,9,9],"right_children":[2,4,6,-1,-1,8,10,12,14,16,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,5.517496E-1,5.264393E-3,-2.4228318E-2,3.3406114E11,1.3996E4,3.7057927E0,3.2414E-1,1.2253E4,7.91129E-3,-5.6681415E-4,1.5132903E-2,-1.0279862E-2,6.974771E-4,8.275241E-3,2.5676768E-2],"split_indices":[41,26,41,0,0,30,9,56,26,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.5E1,5.1E1,9E0,1.6E1,2.4E1,2.7E1,1.3E1,1.1E1,1.3E1,1.4E1,8E0,5E0,6E0,5E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[5.2636985E-2,-9.513471E-2,2.4494243E-1,7.337656E-2,-2.2457552E-1,8.028527E-2,3.3972245E-1,-4.407692E-3,1.38059E-1,-2.2352083E-2,-1.2238035E-1,-1.012086E-3,8.332232E-3,9.366022E-3,2.0773357E-2,1.1558115E-2,-1.5217743E-3,-2.3664015E-3,-1.289764E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0977654E0,9.325925E-1,4.721241E-1,2.1290693E-1,5.101299E-1,1.2080502E-1,1.6773319E-1,0E0,2.3773542E-1,0E0,1.7470732E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.839604E-1,5.81609E-1,4.620608E-1,7.348946E4,1.346E3,2.1933217E0,3.6120827E0,-4.407692E-3,1.13219306E-1,-2.2352083E-2,3.6979167E0,-1.012086E-3,8.332232E-3,9.366022E-3,2.0773357E-2,1.1558115E-2,-1.5217743E-3,-2.3664015E-3,-1.289764E-2],"split_indices":[41,26,26,32,11,56,55,0,41,0,53,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,4.1E1,3.1E1,1.8E1,2.3E1,1.2E1,1.9E1,5E0,1.3E1,6E0,1.7E1,6E0,6E0,8E0,1.1E1,8E0,5E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[4.634194E-2,-2.899864E-2,2.7199874E-2,-2.091426E-1,6.1703745E-2,-3.651765E-1,-6.102604E-2,1.2529667E-2,7.7365725E-3,-7.94981E-3,-2.4918912E-2,1.7878206E-3,-8.875272E-3,-4.2854853E-2,9.391212E-2,-4.1592973E-3,4.7552474E-3,9.89459E-3,5.756966E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,-1,5,7,9,11,-1,13,-1,-1,-1,-1,15,17,-1,-1,-1,-1],"loss_changes":[2.9612415E0,1.12399E0,0E0,5.0927246E-1,4.7840765E-1,2.3615372E-1,1.5827627E-1,0E0,1.653351E-1,0E0,0E0,0E0,0E0,1.4858091E-1,1.2147268E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,13,13,14,14],"right_children":[2,4,-1,6,8,10,12,-1,14,-1,-1,-1,-1,16,18,-1,-1,-1,-1],"split_conditions":[3.9514639E3,6.6504064E0,2.7199874E-2,3.3237658E0,1.7329262E7,2.7288842E0,2.7623658E8,1.2529667E-2,2.3239934E-1,-7.94981E-3,-2.4918912E-2,1.7878206E-3,-8.875272E-3,5.0175633E0,3.2449896E0,-4.1592973E-3,4.7552474E-3,9.89459E-3,5.756966E-4],"split_indices":[51,57,0,53,44,53,7,0,38,0,0,0,0,53,53,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.7E1,9E0,2.2E1,4.5E1,1E1,1.2E1,9E0,3.6E1,5E0,5E0,7E0,5E0,2.3E1,1.3E1,1.8E1,5E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-2.5526343E-2,3.7216682E-2,-2.6441526E-2,-1.5603109E-1,1.1673054E-1,1.2331113E-3,-2.4224122E-1,7.5995736E-2,1.6513858E-2,-1.6436853E-2,-4.351622E-3,-6.2105646E-3,9.482339E-3,5.270621E-3,-2.0650811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,-1,5,7,-1,9,11,-1,-1,-1,13,-1,-1,-1],"loss_changes":[2.659654E0,1.1552883E0,0E0,3.4965348E-1,4.48615E-1,0E0,1.835773E-1,4.4418722E-1,0E0,0E0,0E0,1.1852024E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,11,11],"right_children":[2,4,-1,6,8,-1,10,12,-1,-1,-1,14,-1,-1,-1],"split_conditions":[1.5414109E8,2.0113895E-2,-2.6441526E-2,5.6660336E-1,1.0302285E8,1.2331113E-3,4.593E3,6.3104886E-1,1.6513858E-2,-1.6436853E-2,-4.351622E-3,1.7970878E6,9.482339E-3,5.270621E-3,-2.0650811E-3],"split_indices":[44,40,0,26,44,0,2,40,0,0,0,47,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,7.3E1,8E0,2.1E1,5.2E1,7E0,1.4E1,4.5E1,7E0,8E0,6E0,2.7E1,1.8E1,6E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-2.4680318E-2,-2.9581618E-1,9.901199E-2,-3.9835843E-1,4.678472E-3,3.9854974E-2,1.7734792E-2,-1.2945204E-2,-2.7857706E-2,7.2641626E-2,-9.368809E-3,9.721887E-2,-4.2677824E-3,-1.0250885E-3,1.264454E-1,1.1811204E-2,3.8880985E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,-1,13,-1,-1,15,-1,-1],"loss_changes":[2.6787455E0,1.0405939E0,8.38775E-1,3.0578494E-1,0E0,3.6177745E-1,0E0,0E0,0E0,1.6924502E-1,0E0,1.2689805E-1,0E0,0E0,1.373381E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,11,11,14,14],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,-1,14,-1,-1,16,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,1.1997641E0,9.29295E5,4.678472E-3,6.2012E4,1.7734792E-2,-1.2945204E-2,-2.7857706E-2,2.8192934E5,-9.368809E-3,7.095772E2,-4.2677824E-3,-1.0250885E-3,2.2103658E7,1.1811204E-2,3.8880985E-3],"split_indices":[41,4,38,27,0,10,0,0,0,32,0,4,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.4E1,5.4E1,1.9E1,5E0,4.5E1,9E0,1.2E1,7E0,4E1,5E0,3.5E1,5E0,7E0,2.8E1,7E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-2.094883E-2,-3.1796566E-1,1.24956325E-1,-4.4023082E-1,-9.036691E-4,5.833958E-2,2.5348932E-2,-1.1327739E-2,-3.1768367E-2,-1.1450506E-2,1.1192482E-1,1.4096776E-2,6.875149E-2,4.9823998E-3,-2.8462426E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,-1,-1,-1,-1,11,-1,13,-1,-1],"loss_changes":[3.1599443E0,8.709123E-1,1.2473835E0,5.873859E-1,0E0,6.97931E-1,0E0,0E0,0E0,0E0,2.6013714E-1,0E0,1.3447507E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,10,10,12,12],"right_children":[2,4,6,8,-1,10,-1,-1,-1,-1,12,-1,14,-1,-1],"split_conditions":[4.856813E0,1.650581E3,3.3077192E3,1.0789844E6,-9.036691E-4,6.015076E-2,2.5348932E-2,-1.1327739E-2,-3.1768367E-2,-1.1450506E-2,8.403455E6,1.4096776E-2,4.800481E5,4.9823998E-3,-2.8462426E-3],"split_indices":[57,4,51,27,0,38,0,0,0,0,44,0,27,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.3E1,4.8E1,1.6E1,7E0,4.2E1,6E0,9E0,7E0,6E0,3.6E1,6E0,3E1,2.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.5666451E-3,-2.6866388E-1,1.0623693E-1,-1.4039582E-1,-2.9883765E-2,-1.8703036E-2,2.5592315E-1,-1.3464021E-2,1.1745162E-3,9.708145E-3,-6.6109106E-2,3.765722E-1,6.9883196E-3,-5.4734135E-3,-1.0282227E-2,8.484132E-3,2.7077582E-2,3.7645334E-3,-3.9108815E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,-1,9,11,-1,-1,-1,13,15,-1,17,-1,-1,-1,-1,-1],"loss_changes":[2.303271E0,9.17678E-1,1.0707887E0,4.0244764E-1,0E0,3.4141752E-1,3.0268633E-1,0E0,0E0,0E0,2.3350114E-1,3.3249366E-1,0E0,1.27998E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,10,10,11,11,13,13],"right_children":[2,4,6,8,-1,10,12,-1,-1,-1,14,16,-1,18,-1,-1,-1,-1,-1],"split_conditions":[6.6504064E0,7.916317E11,6.012512E-1,5.84957E5,-2.9883765E-2,2.9862975E5,3.827309E6,-1.3464021E-2,1.1745162E-3,9.708145E-3,1.1658537E1,3.5615184E0,6.9883196E-3,6.5957415E-1,-1.0282227E-2,8.484132E-3,2.7077582E-2,3.7645334E-3,-3.9108815E-3],"split_indices":[57,30,41,1,0,31,46,0,0,0,56,55,0,26,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,2.2E1,5.6E1,1.7E1,5E0,3.1E1,2.5E1,9E0,8E0,5E0,2.6E1,1.1E1,1.4E1,1.9E1,7E0,6E0,5E0,9E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.6054535E-2,-1.8598679E-1,1.2345314E-1,5.0387103E-3,-3.659999E-1,8.4036686E-2,1.7951837E-2,-9.659654E-3,-2.6085056E-2,7.698176E-3,5.270578E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,-1,7,9,-1,-1,-1,-1,-1],"loss_changes":[1.4984899E0,1.2844851E0,3.9887303E-1,0E0,3.151493E-1,2.1166578E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5],"right_children":[2,4,6,-1,8,10,-1,-1,-1,-1,-1],"split_conditions":[6.015076E-2,5.8871865E-1,6.5317163E3,5.0387103E-3,9.5728264E7,1.181508E6,1.7951837E-2,-9.659654E-3,-2.6085056E-2,7.698176E-3,5.270578E-4],"split_indices":[38,26,4,0,44,31,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,2.3E1,4.4E1,9E0,1.4E1,3.9E1,5E0,8E0,6E0,1.9E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-2.9041283E-2,-2.5280547E-1,8.97217E-2,4.3057264E-3,-4.194248E-1,1.45868445E-2,5.1703043E-2,8.21035E-3,-6.6750594E-3,-1.1245698E-2,-2.9700289E-2,-2.2421088E-3,5.986851E-3,1.7973605E-3,-5.816663E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,-1,13,-1,-1,-1],"loss_changes":[2.1486742E0,1.2013174E0,4.025311E-1,2.9697874E-1,4.6184087E-1,0E0,1.7457435E-1,0E0,0E0,0E0,0E0,1.2626818E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,11,11],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,-1,14,-1,-1,-1],"split_conditions":[1.4865639E-2,5.0415697E5,2.9862975E5,1.0689756E5,2.5036643E8,1.45868445E-2,6.304023E-1,8.21035E-3,-6.6750594E-3,-1.1245698E-2,-2.9700289E-2,3.9250955E-1,5.986851E-3,1.7973605E-3,-5.816663E-3],"split_indices":[37,27,31,32,7,0,40,0,0,0,0,40,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,2.7E1,5.2E1,1.1E1,1.6E1,7E0,4.5E1,5E0,6E0,9E0,7E0,2.6E1,1.9E1,2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-3.0049921E-3,-2.4526216E-1,1.4126725E-1,-1.2291123E-1,-2.6625287E-2,1.3159894E-3,2.1802947E-1,-1.0123482E-2,-4.3591075E-2,1.3538176E-2,1.4827667E-3,2.998949E-3,-6.9224844E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,-1,-1,9,-1,11,-1,-1,-1,-1],"loss_changes":[2.3414125E0,8.419758E-1,3.6318392E-1,1.165365E-1,0E0,0E0,2.627275E-1,0E0,1.207742E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8],"right_children":[2,4,6,8,-1,-1,10,-1,12,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,1.5135763E8,2.5222173E-1,1.346E3,-2.6625287E-2,1.3159894E-3,3.3934937E0,-1.0123482E-2,9.837914E0,1.3538176E-2,1.4827667E-3,2.998949E-3,-6.9224844E-3],"split_indices":[37,44,38,11,0,0,53,0,55,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.5E1,2.4E1,4.1E1,1.8E1,6E0,1.7E1,2.4E1,8E0,1E1,1.8E1,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-7.7880896E-4,6.1643828E-2,-2.216756E-2,1.4499612E-2,2.5667531E-2,-8.34008E-3,6.911146E-2,1.0648367E-2,2.330869E-2,-5.5685276E-3,4.821752E-2,6.6294437E-3,-1.0623133E-2,-3.4819979E-3,5.4738694E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,11,-1,13,-1,-1],"loss_changes":[2.081806E0,1.4056114E0,0E0,6.1588556E-1,0E0,0E0,3.1307715E-1,0E0,1.3148472E-1,0E0,1.648986E-1,0E0,1.5518305E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8,10,10,12,12],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,12,-1,14,-1,-1],"split_conditions":[1.4899646E8,3.4414336E3,-2.216756E-2,2.2839906E8,2.5667531E-2,-8.34008E-3,1.7329262E7,1.0648367E-2,4.57563E5,-5.5685276E-3,5.7229916E7,6.6294437E-3,9.250027E7,-3.4819979E-3,5.4738694E-3],"split_indices":[44,51,0,7,0,0,44,0,1,0,44,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.4E1,8E0,5.9E1,5E0,1.3E1,4.6E1,1E1,3.6E1,5E0,3.1E1,1.2E1,1.9E1,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-7.0366934E-3,-2.1274422E-1,9.167809E-2,-2.9697755E-1,3.7024175E-3,1.3440922E-1,-8.72939E-3,-7.8110406E-3,-2.206517E-2,-1.9364968E-3,1.7030679E-1,2.4369304E-1,3.8426183E-3,1.811467E-2,7.732006E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,-1,9,-1,-1,-1,-1,11,13,-1,-1,-1],"loss_changes":[1.4409503E0,5.8066237E-1,5.766351E-1,3.0686724E-1,0E0,2.6883525E-1,0E0,0E0,0E0,0E0,2.1732771E-1,1.470772E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,10,10,11,11],"right_children":[2,4,6,8,-1,10,-1,-1,-1,-1,12,14,-1,-1,-1],"split_conditions":[2.0113895E-2,1.7493458E3,3.45E2,1.0370839E6,3.7024175E-3,2.5547945E0,-8.72939E-3,-7.8110406E-3,-2.206517E-2,-1.9364968E-3,3.1419718E0,2.0571364E6,3.8426183E-3,1.811467E-2,7.732006E-3],"split_indices":[40,4,8,27,0,52,0,0,0,0,52,46,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.9E1,2.2E1,4.7E1,1.7E1,5E0,4.1E1,6E0,1E1,7E0,7E0,3.4E1,1.8E1,1.6E1,6E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.9457502E-2,-1.1716236E-1,8.904412E-2,-5.6788504E-2,-1.8000867E-2,1.2668326E-1,-6.520419E-3,-1.0848941E-2,1.7726362E-2,1.2323694E-2,2.144599E-3,7.6829125E-3,-3.4420106E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,-1,9,-1,-1,11,-1,-1,-1,-1],"loss_changes":[8.4792227E-1,6.118115E-1,3.3414298E-1,4.3212637E-1,0E0,3.2958692E-1,0E0,0E0,3.1568074E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8],"right_children":[2,4,6,8,-1,10,-1,-1,12,-1,-1,-1,-1],"split_conditions":[3.2711282E-2,9.1671896E-1,2.292109E7,2.23E2,-1.8000867E-2,1.1667828E6,-6.520419E-3,-1.0848941E-2,2.217E3,1.2323694E-2,2.144599E-3,7.6829125E-3,-3.4420106E-3],"split_indices":[37,26,46,10,0,31,0,0,10,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,4.1E1,3.7E1,3.4E1,7E0,3.2E1,5E0,1E1,2.4E1,1.2E1,2E1,9E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[5.206865E-3,4.1475102E-2,-1.9670278E-2,-1.9523042E-1,1.0768468E-1,-1.3634004E-2,-1.2388254E-3,4.1639794E-2,2.5781098E-1,1.3674739E-1,-6.066503E-2,2.1280596E-2,6.8029743E-3,2.4711913E-3,1.1624263E-2,-6.058208E-3,4.70435E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,-1,5,7,-1,-1,9,11,13,15,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0764529E0,1.0710589E0,0E0,2.0330131E-1,5.147639E-1,0E0,0E0,3.7803447E-1,2.7505088E-1,1.6094676E-1,1.9834189E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,9,9,10,10],"right_children":[2,4,-1,6,8,-1,-1,10,12,14,16,-1,-1,-1,-1,-1,-1],"split_conditions":[3.698723E10,1.648E3,-1.9670278E-2,3.8251555E-1,1.892355E3,-1.3634004E-2,-1.2388254E-3,1.181508E6,1.383E4,3.015899E-2,1E0,2.1280596E-2,6.8029743E-3,2.4711913E-3,1.1624263E-2,-6.058208E-3,4.70435E-3],"split_indices":[5,11,0,57,51,0,0,31,10,37,80,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.6E1,5E0,1.4E1,5.2E1,9E0,5E0,3.7E1,1.5E1,1.9E1,1.8E1,5E0,1E1,1.1E1,8E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.647754E-2,3.7622735E-2,-2.8987297E-1,-9.913354E-3,8.4004186E-2,-2.0809948E-2,-7.5184917E-3,4.4491112E-2,1.186158E-2,1.0351287E-1,-7.2644226E-2,9.4982395E-3,2.526701E-3,-8.224554E-3,9.703081E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,-1,7,-1,-1,9,-1,11,13,-1,-1,-1,-1],"loss_changes":[1.1645857E0,7.502485E-1,1.7326903E-1,0E0,3.3797675E-1,0E0,0E0,3.2554212E-1,0E0,1.3518E-1,1.4359155E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,9,9,10,10],"right_children":[2,4,6,-1,8,-1,-1,10,-1,12,14,-1,-1,-1,-1],"split_conditions":[1.0232127E12,1.543E3,2.277961E6,-9.913354E-3,4.9891987E3,-2.0809948E-2,-7.5184917E-3,6.98494E7,1.186158E-2,1.7329262E7,1.962943E3,9.4982395E-3,2.526701E-3,-8.224554E-3,9.703081E-4],"split_indices":[30,2,1,0,4,0,0,44,0,44,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,6.5E1,1.2E1,1E1,5.5E1,5E0,7E0,4.5E1,1E1,3E1,1.5E1,1E1,2E1,7E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-1.6774865E-2,2.2359641E-2,-2.1500543E-2,-1.2565155E-1,8.171822E-2,-8.9705875E-3,-8.4445666E-5,1.2181568E-2,5.1314753E-2,1.6408395E-2,9.4877565E-3,-1.1270148E-3,4.9488926E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,-1,5,7,-1,-1,-1,9,11,-1,-1,-1],"loss_changes":[1.163528E0,5.8134073E-1,0E0,1.3406095E-1,2.25052E-1,0E0,0E0,0E0,1.9996464E-1,1.1526482E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,8,8,9,9],"right_children":[2,4,-1,6,8,-1,-1,-1,10,12,-1,-1,-1],"split_conditions":[1.5414109E8,1.438E3,-2.1500543E-2,2.741545E8,1.6256282E7,-8.9705875E-3,-8.4445666E-5,1.2181568E-2,1.6132394E2,1E0,9.4877565E-3,-1.1270148E-3,4.9488926E-3],"split_indices":[44,10,0,7,44,0,0,0,56,15,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.9E1,6.4E1,5E0,1.8E1,4.6E1,1.2E1,6E0,6E0,4E1,3.3E1,7E0,2.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[4.0234547E-2,-1.8009594E-1,1.0566312E-1,-1.641945E-3,-1.585929E-2,5.9209943E-2,1.7788097E-2,-4.962642E-3,7.841298E-2,7.261781E-3,2.1453684E-2,-3.9354246E-3,2.9960868E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,-1,-1,7,-1,-1,9,-1,11,-1,-1],"loss_changes":[1.1290545E0,3.6650872E-1,6.938956E-1,0E0,0E0,1.6798708E-1,0E0,0E0,1.8288276E-1,0E0,1.13460645E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,8,8,10,10],"right_children":[2,4,6,-1,-1,8,-1,-1,10,-1,12,-1,-1],"split_conditions":[1.217012E-2,8.0159146E-1,3.9514639E3,-1.641945E-3,-1.585929E-2,2.074E3,1.7788097E-2,-4.962642E-3,4.8178736E7,7.261781E-3,1.0773855E3,-3.9354246E-3,2.9960868E-3],"split_indices":[40,26,51,0,0,2,0,0,44,0,51,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,1.7E1,5.9E1,9E0,8E0,5.1E1,8E0,5E0,4.6E1,2E1,2.6E1,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.3357232E-2,-1.09336255E-2,6.514406E-2,1.07315265E-1,-1.0896942E-1,7.0487596E-2,1.5364354E-2,-3.5821355E-4,-1.0281824E-2,9.455917E-3,2.6307374E-2,-4.530428E-3,3.7667106E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,-1,3,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[8.824686E-1,0E0,4.4490603E-1,3.4144813E-1,1.2103732E-1,2.215284E-1,0E0,0E0,0E0,0E0,1.9839323E-1,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,10,10],"right_children":[2,-1,4,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[1.528E3,-1.09336255E-2,1.96E2,1.0011831E8,4.901466E9,3.5175372E5,1.5364354E-2,-3.5821355E-4,-1.0281824E-2,9.455917E-3,4.46206E5,-4.530428E-3,3.7667106E-3],"split_indices":[11,0,8,44,12,31,0,0,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,1.2E1,5.8E1,4.7E1,1.1E1,4.1E1,6E0,6E0,5E0,1E1,3.1E1,9E0,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[1.1344429E-3,4.542106E-2,-1.589505E-2,-1.4501415E-2,2.3358513E-1,-5.8184072E-2,7.3463065E-3,1.9264063E-2,6.172905E-3,2.2440087E-3,-4.8015625E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,-1],"loss_changes":[1.0794531E0,7.4755424E-1,0E0,3.7436986E-1,2.2468871E-1,1.7322114E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,-1],"split_conditions":[1.2159766E6,3.2311362E3,-1.589505E-2,5.5143005E-1,9.5E2,3.3207992E-1,7.3463065E-3,1.9264063E-2,6.172905E-3,2.2440087E-3,-4.8015625E-3],"split_indices":[27,4,0,38,0,26,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.5E1,8E0,5E1,1.5E1,4E1,1E1,5E0,1E1,1.1E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.9008998E-2,2.0432957E-2,-1.362022E-2,-1.6878393E-1,6.560678E-2,-1.2191049E-2,-1.699292E-3,3.5918865E-2,1.4587329E-2,8.3578415E-3,-2.2672575E-2,-4.1214605E-3,2.7168903E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,-1,5,7,-1,-1,9,-1,-1,11,-1,-1],"loss_changes":[7.70101E-1,5.752183E-1,0E0,1.3012546E-1,3.6048335E-1,0E0,0E0,3.894863E-1,0E0,0E0,1.7195417E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,10,10],"right_children":[2,4,-1,6,8,-1,-1,10,-1,-1,12,-1,-1],"split_conditions":[1.21144904E8,2.3140822E8,-1.362022E-2,1.3440569E1,4.146482E3,-1.2191049E-2,-1.699292E-3,2.325535E7,1.4587329E-2,8.3578415E-3,1.2052782E3,-4.1214605E-3,2.7168903E-3],"split_indices":[44,7,0,55,4,0,0,44,0,0,51,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.5E1,9E0,1.2E1,5.3E1,7E0,5E0,4.8E1,5E0,1.4E1,3.4E1,1.9E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-9.672453E-3,3.2812443E-2,-2.215897E-2,-6.3623213E-3,1.7276714E-2,-1.1557485E-1,5.4741472E-2,-9.425394E-3,-2.3214722E-3,7.2672334E-3,8.8617014E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,-1],"loss_changes":[1.4972064E0,9.1475964E-1,0E0,4.467645E-1,0E0,1.1615011E-1,1.4450203E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,-1],"split_conditions":[1.2427474E8,4.9891987E3,-2.215897E-2,7.0277144E7,1.7276714E-2,3.5618556E0,2.6529046E7,-9.425394E-3,-2.3214722E-3,7.2672334E-3,8.8617014E-4],"split_indices":[44,4,0,12,0,53,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.2E1,6E0,6.5E1,7E0,2.3E1,4.2E1,1E1,1.3E1,1.1E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.5570879E-2,-2.190597E-1,8.2088366E-2,8.5699797E-4,-3.5545832E-1,2.4372004E-3,1.1291955E-2,-1.0046729E-2,-2.342058E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,-1,7,-1,-1,-1,-1],"loss_changes":[1.3494442E0,7.1593356E-1,2.0934469E-1,0E0,1.5910637E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4],"right_children":[2,4,6,-1,8,-1,-1,-1,-1],"split_conditions":[7.940699E-3,3.6588228E5,1.1997641E0,8.5699797E-4,9.3570206E5,2.4372004E-3,1.1291955E-2,-1.0046729E-2,-2.342058E-2],"split_indices":[37,27,38,0,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[6.6E1,2.1E1,4.5E1,8E0,1.3E1,3.8E1,7E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[1.4198847E-3,-4.122868E-2,2.2194607E-2,-1.3280036E-1,7.040858E-2,-8.228565E-2,-2.0313371E-2,8.839084E-3,1.1655054E-3,3.7776062E-3,-1.382122E-1,-1.3436738E-3,-9.907611E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,-1,5,7,9,-1,-1,-1,-1,11,-1,-1],"loss_changes":[1.5562164E0,7.6583827E-1,0E0,5.4784024E-1,1.6968237E-1,3.312218E-1,0E0,0E0,0E0,0E0,1.8479353E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,10,10],"right_children":[2,4,-1,6,8,10,-1,-1,-1,-1,12,-1,-1],"split_conditions":[6.7780703E3,4.283039E1,2.2194607E-2,1.5135763E8,2.156825E7,3.459181E-1,-2.0313371E-2,8.839084E-3,1.1655054E-3,3.7776062E-3,1.5368E4,-1.3436738E-3,-9.907611E-3],"split_indices":[4,57,0,44,44,26,0,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.3E1,6E0,4E1,3.3E1,3.5E1,5E0,9E0,2.4E1,9E0,2.6E1,1E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[2.3551324E-2,-2.4767287E-2,2.7618116E-1,-1.1624122E-1,3.0292E-2,1.9757818E-2,5.100631E-3,-9.066552E-3,2.787641E-3,-2.9648573E-4,5.7301046E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,-1,-1,-1,-1,-1,-1],"loss_changes":[8.3689743E-1,2.9563755E-1,1.9747049E-1,2.7097008E-1,1.1687774E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4],"right_children":[2,4,6,8,10,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1118015E3,1.1576994E-2,9.5E2,2.477407E6,1.1328785E-1,1.9757818E-2,5.100631E-3,-9.066552E-3,2.787641E-3,-2.9648573E-4,5.7301046E-3],"split_indices":[51,37,0,1,37,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,5.7E1,1E1,2.1E1,3.6E1,5E0,5E0,1.5E1,6E0,2.6E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.6161116E-2,-3.6539175E-2,3.452665E-1,-1.606901E-1,3.3245545E-2,2.7661115E-2,3.4155506E-3,-6.1070785E-2,-1.7907323E-2,2.813106E-3,-6.5125767E-3,-8.6820815E-3,1.4079659E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,-1,11,-1,-1,-1,-1,-1],"loss_changes":[1.3827877E0,6.031471E-1,6.223842E-1,4.859256E-1,1.8716986E-1,0E0,0E0,1.9209798E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7],"right_children":[2,4,6,8,10,-1,-1,12,-1,-1,-1,-1,-1],"split_conditions":[4.9891987E3,8.161361E0,8.982776E9,7.916317E11,1.3312784E7,2.7661115E-2,3.4155506E-3,4E0,-1.7907323E-2,2.813106E-3,-6.5125767E-3,-8.6820815E-3,1.4079659E-3],"split_indices":[4,57,5,30,31,0,0,8,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,6.8E1,1E1,2.4E1,4.4E1,5E0,5E0,1.7E1,7E0,3.9E1,5E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-2.9478155E-2,3.134547E-2,-1.05996765E-1,9.182898E-3,-2.9935643E-2,-1.7123075E-1,-8.002812E-4,1.1299612E-2,-7.3593357E-3,-1.4833488E-2,-3.4214563E-3,-3.1115087E-3,5.170479E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,-1,-1,-1],"loss_changes":[3.4772885E-1,4.0633434E-1,1.9029963E-1,0E0,1.550427E-1,2.3149824E-1,0E0,1.751737E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,-1,-1,-1],"split_conditions":[6.7376137E-1,1.5199E4,1E0,9.182898E-3,1.6107566E5,1.4235585E3,-8.002812E-4,8.1637933E2,-7.3593357E-3,-1.4833488E-2,-3.4214563E-3,-3.1115087E-3,5.170479E-3],"split_indices":[26,9,15,0,32,4,0,51,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,4.1E1,3.2E1,1.1E1,3E1,1.8E1,1.4E1,2.3E1,7E0,7E0,1.1E1,1.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-3.0035045E-2,1.0828146E-3,-1.8288529E-2,-9.803892E-2,3.6744982E-2,-7.5814067E-3,1.9866752E-3,4.269513E-3,-6.3606974E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,-1,5,7,-1,-1,-1,-1],"loss_changes":[7.677181E-1,2.4044046E-1,0E0,1.4681849E-1,1.259232E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4],"right_children":[2,4,-1,6,8,-1,-1,-1,-1],"split_conditions":[1.5414109E8,3.068E3,-1.8288529E-2,2.37321E6,6.854598E7,-7.5814067E-3,1.9866752E-3,4.269513E-3,-6.3606974E-4],"split_indices":[44,2,0,47,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.6E1,5E0,1.7E1,4.9E1,1.2E1,5E0,2.4E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.0493905E-2,1.8862784E-2,1.4323112E-2,8.2179636E-2,-4.965037E-2,-1.1853584E-3,7.5804605E-3,-8.0141E-3,2.24567E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,-1,5,7,-1,-1,-1,-1],"loss_changes":[6.298864E-1,2.5134835E-1,0E0,2.3410118E-1,3.1084996E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4],"right_children":[2,4,-1,6,8,-1,-1,-1,-1],"split_conditions":[6.5317163E3,5.7229916E7,1.4323112E-2,3.144E3,5.062801E11,-1.1853584E-3,7.5804605E-3,-8.0141E-3,2.24567E-3],"split_indices":[4,44,0,2,30,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[6.5E1,5.6E1,9E0,2.9E1,2.7E1,1.2E1,1.7E1,1.2E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[7.97047E-3,-3.4158625E-2,1.666469E-2,3.7368038E-3,-1.3044306E-2,-5.386321E-3,4.9033236E-2,6.2351944E-3,-1.2828831E-2,-4.9604382E-3,3.4364306E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[1.1147797E0,6.2395734E-1,0E0,3.2398638E-1,0E0,0E0,2.1867892E-1,0E0,1.9702259E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[4.0235266E3,1.4899646E8,1.666469E-2,7.801514E2,-1.3044306E-2,-5.386321E-3,5.7229916E7,6.2351944E-3,1.0781599E3,-4.9604382E-3,3.4364306E-3],"split_indices":[51,44,0,4,0,0,44,0,51,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7E1,8E0,6.1E1,9E0,1.7E1,4.4E1,1.9E1,2.5E1,1.2E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.2421385E-2,-2.661511E-2,1.5661146E-2,8.515507E-3,-1.186941E-2,-5.6693163E-3,3.174948E-2,-5.650824E-3,4.9444146E-2,8.778609E-3,5.2509847E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,-1,5,-1,-1,7,-1,9,-1,-1],"loss_changes":[9.539298E-1,5.386975E-1,0E0,1.8227701E-1,0E0,0E0,1.4340845E-1,0E0,2.3869595E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,8,8],"right_children":[2,4,-1,6,-1,-1,8,-1,10,-1,-1],"split_conditions":[4.9891987E3,9.1569895E-1,1.5661146E-2,1.648E3,-1.186941E-2,-5.6693163E-3,2.5332227E0,-5.650824E-3,1.6256282E7,8.778609E-3,5.2509847E-4],"split_indices":[4,26,0,11,0,0,52,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7E1,8E0,6.1E1,9E0,9E0,5.2E1,5E0,4.7E1,1E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[9.186646E-3,-2.3591982E-2,1.9150686E-2,7.2749235E-2,-5.1189575E-2,-3.1576736E-3,7.924987E-3,-7.966798E-3,-1.6415874E-2,7.535355E-3,-8.1189135E-3,-4.0703155E-2,3.7512253E-3,-4.729123E-3,2.7750346E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,-1,5,7,-1,-1,-1,9,11,-1,13,-1,-1,-1],"loss_changes":[9.4557226E-1,1.8983902E-1,0E0,2.0484525E-1,2.0978843E-1,0E0,0E0,0E0,1.5465544E-1,1.2935658E-1,0E0,1.298286E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,8,8,9,9,11,11],"right_children":[2,4,-1,6,8,-1,-1,-1,10,12,-1,14,-1,-1,-1],"split_conditions":[4.0235266E3,1.7329262E7,1.9150686E-2,2.1262457E0,9.408949E2,-3.1576736E-3,7.924987E-3,-7.966798E-3,3.64E2,1.2052782E3,-8.1189135E-3,4.3646493E0,3.7512253E-3,-4.729123E-3,2.7750346E-3],"split_indices":[51,44,0,55,4,0,0,0,8,51,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,1.5E1,5.4E1,6E0,9E0,1.2E1,4.2E1,3.7E1,5E0,2.2E1,1.5E1,1.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-4.942342E-3,3.4710273E-2,-1.627301E-2,1.1842767E-2,1.382875E-2,7.971509E-2,-4.0837314E-2,8.768508E-3,1.5332443E-3,-7.4814674E-3,-3.079751E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,-1,-1,5,7,9,-1,-1,-1,-1],"loss_changes":[9.566525E-1,2.8127292E-1,0E0,0E0,2.2296189E-1,1.2866971E-1,1.301358E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6],"right_children":[2,4,-1,-1,6,8,10,-1,-1,-1,-1],"split_conditions":[1.5414109E8,1E0,-1.627301E-2,1.1842767E-2,4.7293822E5,2.9124088E0,2.1836805E6,8.768508E-3,1.5332443E-3,-7.4814674E-3,-3.079751E-4],"split_indices":[44,63,0,0,31,52,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.5E1,7E0,5E0,6E1,2.7E1,3.3E1,8E0,1.9E1,7E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.2136525E-3,7.74439E-2,-7.1138464E-2,5.237987E-3,-1.2470221E-3,-1.5859775E-1,1.9877737E-2,-3.7901737E-3,-1.4666451E-2,4.2365417E-3,-3.4800547E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,-1,-1,7,9,-1,-1,-1,-1],"loss_changes":[4.191657E-1,1.1400004E-1,3.1360084E-1,0E0,0E0,2.0507336E-1,1.2667494E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6],"right_children":[2,4,6,-1,-1,8,10,-1,-1,-1,-1],"split_conditions":[6.585265E-1,6.7584877E8,7.673801E2,5.237987E-3,-1.2470221E-3,1.052686E6,1.3946067E7,-3.7901737E-3,-1.4666451E-2,4.2365417E-3,-3.4800547E-3],"split_indices":[26,7,51,0,0,27,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,3.6E1,3.8E1,2.8E1,8E0,1.9E1,1.9E1,1.3E1,6E0,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[1.25434585E-2,-3.1279963E-2,1.6803306E-2,-2.9869617E-3,-1.586476E-2,-7.0873266E-3,7.082917E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":45,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[1.1088532E0,5.6055737E-1,0E0,1.5717831E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[3.3077192E3,1.5012118E6,1.6803306E-2,2.0435429E8,-1.586476E-2,-7.0873266E-3,7.082917E-4],"split_indices":[51,27,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,6.2E1,5E0,6E0,5.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-3.5548672E-2,-5.60212E-3,-1.8117001E-2,-4.7952808E-3,2.7003406E-2,-5.980847E-3,4.5828324E-2,1.0206075E-1,1.21624966E-4,1.8545174E-3,1.0607216E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,-1,-1,5,-1,7,9,-1,-1,-1],"loss_changes":[7.2692984E-1,2.067753E-1,0E0,0E0,1.484654E-1,0E0,1.11882806E-1,1.3840435E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7],"right_children":[2,4,-1,-1,6,-1,8,10,-1,-1,-1],"split_conditions":[1.5414109E8,1.528E3,-1.8117001E-2,-4.7952808E-3,2.4742014E0,-5.980847E-3,3.6871E4,5.76824E2,1.21624966E-4,1.8545174E-3,1.0607216E-2],"split_indices":[44,11,0,0,52,0,11,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.7E1,5E0,1.7E1,5E1,5E0,4.5E1,1.9E1,2.6E1,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.3936535E-2,3.9373323E-2,-1.855898E-1,-2.2830173E-2,1.4937094E-1,-1.8054586E-2,-7.904547E-2,-2.8332032E-3,3.6145858E-3,1.4737386E-2,4.5075227E-2,9.192151E-4,-8.833915E-3,5.9456374E-3,-3.1327375E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,13,-1,-1,-1,-1],"loss_changes":[6.5701485E-1,3.797332E-1,3.0086392E-1,1.2535831E-1,3.0104432E-1,0E0,1.2173442E-1,0E0,0E0,0E0,1.1534701E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,10,10],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,14,-1,-1,-1,-1],"split_conditions":[1.0232127E12,1.2777201E3,1.65427E6,4.0614333E0,3.6274084E9,-1.8054586E-2,5.303548E9,-2.8332032E-3,3.6145858E-3,1.4737386E-2,3.5010372E5,9.192151E-4,-8.833915E-3,5.9456374E-3,-3.1327375E-3],"split_indices":[30,51,28,53,12,0,12,0,0,0,31,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,5.4E1,1.6E1,3.5E1,1.9E1,5E0,1.1E1,2.6E1,9E0,7E0,1.2E1,6E0,5E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[1.9423576E-2,-5.6491535E-2,1.4887007E-1,-5.6541525E-4,-1.2278832E-2,2.5301895E-1,2.6975197E-3,3.158153E-3,1.781714E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[7.4543077E-1,4.150858E-1,2.6000005E-1,0E0,0E0,2.462179E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.1813418E3,9.635135E7,1.0956273E10,-5.6541525E-4,-1.2278832E-2,1.144E4,2.6975197E-3,3.158153E-3,1.781714E-2],"split_indices":[51,44,5,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,4.7E1,2.7E1,3.9E1,8E0,1.2E1,1.5E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.8203143E-2,3.948176E-2,-7.383702E-2,-4.91742E-3,3.7627732E-3,-2.033799E-3,-1.150748E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,-1,-1,-1,-1],"loss_changes":[2.339801E-1,1.9448869E-1,1.8283024E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2],"right_children":[2,4,6,-1,-1,-1,-1],"split_conditions":[6.585265E-1,2.0435429E8,3E1,-4.91742E-3,3.7627732E-3,-2.033799E-3,-1.150748E-2],"split_indices":[26,7,3,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.1E1,3.5E1,3.6E1,7E0,2.8E1,3.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[2.5208674E-2,-5.284616E-3,1.0246003E-2,-4.221731E-3,3.9235905E-2,6.4257034E-3,4.1273102E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":50,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[3.9276403E-1,2.2255188E-1,0E0,0E0,1.1191535E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[6.7780703E3,2.1647314E8,1.0246003E-2,-4.221731E-3,2.325535E7,6.4257034E-3,4.1273102E-4],"split_indices":[4,12,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[6.9E1,6E1,9E0,2.1E1,3.9E1,9E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[1.4583331E-2,-9.185806E-3,1.3116717E-2,1.629297E-2,-1.4430714E-1,-5.6454637E-3,3.9866414E-2,-1.4945229E-3,-1.242595E-2,1.0730911E-1,2.040496E-4,-1.3348513E-3,7.7219913E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,-1,5,7,-1,9,-1,-1,11,-1,-1,-1],"loss_changes":[4.88758E-1,2.6055562E-1,0E0,2.0265129E-1,1.4094786E-1,0E0,1.3201559E-1,0E0,0E0,1.3199131E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,9,9],"right_children":[2,4,-1,6,8,-1,10,-1,-1,12,-1,-1,-1],"split_conditions":[4.0235266E3,1.0232127E12,1.3116717E-2,1.595E3,4.212135E8,-5.6454637E-3,4.654812E5,-1.4945229E-3,-1.242595E-2,2.4463703E-5,2.040496E-4,-1.3348513E-3,7.7219913E-3],"split_indices":[51,30,0,2,7,0,31,0,0,37,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.4E1,6E0,6.3E1,1.1E1,9E0,5.4E1,6E0,5E0,1.8E1,3.6E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-4.6967324E-3,-3.0794473E-2,1.4045407E-1,-1.2780759E-2,-9.514151E-3,1.5503837E-2,-2.8618781E-3,-6.064544E-3,5.427009E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[2.732809E-1,1.7662871E-1,4.0173334E-1,1.4918314E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[5.744167E3,1.5135763E8,5.1835245E-1,2.1339026E8,-9.514151E-3,1.5503837E-2,-2.8618781E-3,-6.064544E-3,5.427009E-4],"split_indices":[4,44,26,7,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6E1,1E1,5.5E1,5E0,5E0,5E0,9E0,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-1.0094666E-2,-3.5785582E-2,8.388562E-3,-6.339316E-3,-6.482565E-3,2.856376E-3,-6.592214E-2,-1.3867484E-3,-9.17103E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,-1,5,-1,7,-1,-1],"loss_changes":[3.3619183E-1,1.7147234E-1,0E0,0E0,1.9216886E-1,0E0,1.13133766E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6],"right_children":[2,4,-1,-1,6,-1,8,-1,-1],"split_conditions":[4.4845093E3,3.731E3,8.388562E-3,-6.339316E-3,6.2597164E7,2.856376E-3,6.650418E-2,-1.3867484E-3,-9.17103E-3],"split_indices":[51,11,0,0,44,0,37,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.2E1,8E0,1.4E1,4.8E1,2.3E1,2.5E1,2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[9.499023E-3,-1.9454498E-2,1.1628415E-2,4.3340633E-4,-1.3712147E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":54,"left_children":[1,3,-1,-1,-1],"loss_changes":[5.2464485E-1,5.24243E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[4.9891987E3,1.15266744E8,1.1628415E-2,4.3340633E-4,-1.3712147E-2],"split_indices":[4,44,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.8E1,7E1,8E0,6.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[1.2096384E-2,-2.7100151E-2,2.3904929E-1,-2.5640659E-3,-1.2953998E-2,2.0477127E-2,1.0391646E-3,4.003879E-3,-2.577606E-2,-2.6870011E-3,3.6310458E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,-1,-1,-1,9,-1,-1],"loss_changes":[6.6476506E-1,3.7110174E-1,4.1788572E-1,1.1730173E-1,0E0,0E0,0E0,0E0,1.3977358E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,8,8],"right_children":[2,4,6,8,-1,-1,-1,-1,10,-1,-1],"split_conditions":[4.2930435E3,1.4805952E8,8.982776E9,1.6256282E7,-1.2953998E-2,2.0477127E-2,1.0391646E-3,4.003879E-3,5.0175633E0,-2.6870011E-3,3.6310458E-3],"split_indices":[4,44,5,44,0,0,0,0,53,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.3E1,1E1,5.8E1,5E0,5E0,5E0,1.2E1,4.6E1,3.6E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[2.3147348E-2,-2.1615578E-3,1.0398267E-2,3.6789202E-3,-6.499883E-2,-1.4884324E-2,-1.0007113E-2,-3.2078598E-3,3.6143311E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,-1,5,7,-1,-1,-1],"loss_changes":[3.6332348E-1,3.3429703E-1,0E0,0E0,2.6135135E-1,1.3430858E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5],"right_children":[2,4,-1,-1,6,8,-1,-1,-1],"split_conditions":[3.3077192E3,5.090285E7,1.0398267E-2,3.6789202E-3,1.2159766E6,1.775E3,-1.0007113E-2,-3.2078598E-3,3.6143311E-3],"split_indices":[51,44,0,0,27,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.7E1,8E0,3E1,3.7E1,2.8E1,9E0,1.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-3.319726E-2,-1.1683443E-2,-1.075212E-2,-5.882681E-2,5.7229344E-3,-7.3974794E-3,-4.6753042E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":57,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[2.692679E-1,3.8172275E-1,0E0,2.0687792E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[5.673536E2,1.2052782E3,-1.075212E-2,4.0770257E-3,5.7229344E-3,-7.3974794E-3,-4.6753042E-4],"split_indices":[57,51,0,37,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[6.7E1,6.1E1,6E0,4.5E1,1.6E1,1.5E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-2.6736325E-3,-1.1941707E-1,1.442405E-3,-1.4116995E-2,-1.52251115E-2,-5.0588553E-3,4.515846E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":58,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[2.9538652E-1,3.369896E-1,0E0,1.2269549E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[8.872871E-3,9.29295E5,1.442405E-3,2.0672107E8,-1.52251115E-2,-5.0588553E-3,4.515846E-3],"split_indices":[40,27,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.7E1,1.6E1,6.1E1,1.1E1,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[3.865119E-2,1.2201436E-2,1.1065193E-2,5.735236E-3,-2.152015E-2,-4.7150054E-3,1.7552887E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":59,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[3.5779712E-1,2.3182508E-1,0E0,0E0,2.1682681E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[4.9891987E3,1.7329262E7,1.1065193E-2,5.735236E-3,7.4374756E2,-4.7150054E-3,1.7552887E-3],"split_indices":[4,44,0,0,51,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.4E1,8E0,1.5E1,4.9E1,2.1E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-1.0897434E-2,2.6469478E-2,-1.526777E-1,-2.9688086E-3,2.8831253E-3,-3.0205812E-2,-1.7269867E-2,-6.1113243E-3,3.3925665E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,-1,-1,7,-1,-1,-1],"loss_changes":[4.06681E-1,1.746145E-1,3.7949267E-1,0E0,0E0,1.1158453E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5],"right_children":[2,4,6,-1,-1,8,-1,-1,-1],"split_conditions":[1.0232127E12,7.095772E2,3.5715E4,-2.9688086E-3,2.8831253E-3,7.575E3,-1.7269867E-2,-6.1113243E-3,3.3925665E-3],"split_indices":[30,4,9,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6E1,1.5E1,1.6E1,4.4E1,1E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[4.039818E-3,2.2525087E-2,-9.363859E-3,-9.92935E-3,6.717436E-3,1.3164226E-3,-7.1289785E-2,-9.660784E-3,-6.0964015E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[2.4912001E-1,2.3505925E-1,0E0,1.1464001E-1,0E0,0E0,1.3792285E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[1.06102504E8,3.1672727E3,-9.363859E-3,6.1358623E9,6.717436E-3,1.3164226E-3,1.27426E6,-9.660784E-3,-6.0964015E-4],"split_indices":[50,4,0,5,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,6.2E1,5E0,4.9E1,1.3E1,3.1E1,1.8E1,5E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[2.03194E-3,2.5173092E-2,-1.2792875E-2,-1.556973E-3,6.578646E-2,1.3811609E-3,9.443853E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[4.5017925E-1,1.5919018E-1,0E0,0E0,1.852264E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[2.8800072E10,1.3E1,-1.2792875E-2,-1.556973E-3,6.254855E6,1.3811609E-3,9.443853E-3],"split_indices":[5,3,0,0,1,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.7E1,5E0,2.8E1,3.9E1,3.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-1.7721854E-2,9.4442405E-3,-3.8477477E-2,-1.8571196E-2,-1.0554522E-2,-6.5618195E-2,3.3457435E-3,-3.3343695E-2,-1.2455826E-2,-5.8482266E-3,7.5279386E-3,3.6893128E-3,-3.2339357E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,-1,3,5,-1,7,-1,9,-1,-1,11,-1,-1],"loss_changes":[3.3880022E-1,0E0,2.419545E-1,2.655844E-1,0E0,2.4650148E-1,0E0,1.3061237E-1,0E0,0E0,1.3399036E-1,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,7,7,10,10],"right_children":[2,-1,4,6,-1,8,-1,10,-1,-1,12,-1,-1],"split_conditions":[1E0,9.4442405E-3,6.174736E2,1.0781599E3,-1.0554522E-2,9.8788344E5,3.3457435E-3,6.51289E2,-1.2455826E-2,-5.8482266E-3,5.7229916E7,3.6893128E-3,-3.2339357E-3],"split_indices":[63,0,57,51,0,27,0,54,0,0,44,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6E0,6.9E1,6.3E1,6E0,4.1E1,2.2E1,3.6E1,5E0,1.1E1,2.5E1,1.3E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-2.504406E-2,7.6711266E-3,-1.665611E-1,3.0940183E-2,-4.5387764E-3,-1.3064414E-2,-2.983536E-4,-1.1108434E-2,5.959388E-3,-2.228569E-3,6.024879E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,-1,-1,-1,9,-1,-1,-1],"loss_changes":[3.207309E-1,1.3565277E-1,2.060585E-1,1.8009207E-1,0E0,0E0,0E0,1.5698038E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7],"right_children":[2,4,6,8,-1,-1,-1,10,-1,-1,-1],"split_conditions":[1.0232127E12,6.349519E6,4.0950096E10,1.2052782E3,-4.5387764E-3,-1.3064414E-2,-2.983536E-4,9.306648E-1,5.959388E-3,-2.228569E-3,6.024879E-3],"split_indices":[30,31,5,51,0,0,0,38,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,5.6E1,1.2E1,4.6E1,1E1,7E0,5E0,3.2E1,1.4E1,2.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[5.645277E-4,1.9174173E-2,-8.460074E-3,-4.6539283E-3,1.2482908E-2,-2.4433006E-2,5.4074517E-3,-3.4774726E-3,1.823826E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,-1,7,-1,-1,-1],"loss_changes":[2.3173374E-1,3.652897E-1,0E0,1.389249E-1,0E0,1.512104E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5],"right_children":[2,4,-1,6,-1,8,-1,-1,-1],"split_conditions":[7.149144E7,2.1118015E3,-8.460074E-3,1.0379711E0,1.2482908E-2,7.673801E2,5.4074517E-3,-3.4774726E-3,1.823826E-3],"split_indices":[47,51,0,41,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.4E1,6E0,5.9E1,5E0,5.1E1,8E0,2.9E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[-7.0295446E-3,1.3844832E-3,-1.1624204E-1,-5.692552E-4,-1.5887374E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":66,"left_children":[1,-1,3,-1,-1],"loss_changes":[2.84146E-1,0E0,3.868443E-1,0E0,0E0],"parents":[2147483647,0,0,2,2],"right_children":[2,-1,4,-1,-1],"split_conditions":[1.0232127E12,1.3844832E-3,1.0302285E8,-5.692552E-4,-1.5887374E-2],"split_indices":[30,0,44,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.2E1,5.5E1,1.7E1,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[3.235737E-3,1.8166335E-2,-9.641216E-3,-3.755254E-3,1.0601117E-2,-1.1551724E-3,4.3851454E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[2.5689772E-1,3.476345E-1,0E0,1.3645498E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[1.3866357E8,4.9891987E3,-9.641216E-3,5.2518907E0,1.0601117E-2,-1.1551724E-3,4.3851454E-3],"split_indices":[47,4,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.4E1,7.9E1,5E0,7.2E1,7E0,6E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[3.0160226E-2,8.513849E-2,-7.6134055E-4,2.2429074E-3,1.3490965E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":68,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.9966793E-1,2.5681683E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[5.81609E-1,3.3077192E3,-7.6134055E-4,2.2429074E-3,1.3490965E-2],"split_indices":[26,51,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.8E1,3.5E1,4.3E1,3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-2.5746262E-2,-6.5625873E-3,-9.935627E-3,-4.2345333E-3,8.1055466E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":69,"left_children":[1,3,-1,-1,-1],"loss_changes":[2.7651963E-1,1.3932057E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[1.5135763E8,4.96576E5,-9.935627E-3,-4.2345333E-3,8.1055466E-4],"split_indices":[44,1,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[8.1E1,7.4E1,7E0,1.6E1,5.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-2.2393899E-2,-7.88017E-2,3.3593888E-4,-1.7641761E-3,-1.0239334E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":70,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.2092841E-1,1.3156827E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[2.289789E6,5.9481585E-1,3.3593888E-4,-1.7641761E-3,-1.0239334E-2],"split_indices":[50,56,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.2E1,2.4E1,4.8E1,1.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[1.5220827E-2,3.3491213E-2,-1.0739114E-2,1.38611635E-2,1.2774472E-2,4.2835763E-3,-7.183633E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[3.5356608E-1,3.346139E-1,0E0,1.497447E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[2.1050402E7,1.6173E4,-1.0739114E-2,1.7329262E7,1.2774472E-2,4.2835763E-3,-7.183633E-4],"split_indices":[46,2,0,44,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8E1,7.5E1,5E0,7E1,5E0,1.9E1,5.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[9.86926E-3,9.521191E-3,-5.1882323E-3,3.5291424E-4,-7.2000236E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":72,"left_children":[1,-1,3,-1,-1],"loss_changes":[2.1995823E-1,0E0,1.3061874E-1,0E0,0E0],"parents":[2147483647,0,0,2,2],"right_children":[2,-1,4,-1,-1],"split_conditions":[1E0,9.521191E-3,1.5414109E8,3.5291424E-4,-7.2000236E-3],"split_indices":[63,0,44,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.8E1,5E0,7.3E1,6.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-3.761863E-3,1.3387413E-2,-9.419713E-3,-3.7431791E-3,3.6302008E-2,5.9039367E-4,6.2292283E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[2.7031484E-1,1.6091447E-1,0E0,0E0,1.3480818E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[3.698723E10,3.2716873E0,-9.419713E-3,-3.7431791E-3,6.7469894E5,5.9039367E-4,6.2292283E-3],"split_indices":[5,55,0,0,27,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.6E1,6E0,1.5E1,6.1E1,4.9E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[1.5921641E-2,3.533586E-2,-6.188995E-3,9.485208E-4,9.308668E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":74,"left_children":[1,3,-1,-1,-1],"loss_changes":[2.0567802E-1,1.5693486E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[1.5414109E8,4.146482E3,-6.188995E-3,9.485208E-4,9.308668E-3],"split_indices":[44,4,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.2E1,6.4E1,8E0,5.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-4.9867425E-3,7.547285E-4,-9.7099785E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":75,"left_children":[1,-1,-1],"loss_changes":[3.288281E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[1.5414109E8,7.547285E-4,-9.7099785E-3],"split_indices":[44,0,0],"split_type":[0,0,0],"sum_hessian":[8.2E1,7.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[4.638584E-3,-2.2545266E-3,1.822349E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":76,"left_children":[1,-1,-1],"loss_changes":[1.2158522E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[3.21252E8,-2.2545266E-3,1.822349E-3],"split_indices":[7,0,0],"split_type":[0,0,0],"sum_hessian":[7.2E1,2.8E1,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-1.9765327E-2,-5.3712655E-2,1.9641162E-3,-3.0514078E-2,-1.0070168E-2,-7.2246315E-3,-5.080012E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":77,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[1.7259221E-1,1.834068E-1,0E0,1.1138721E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[1.2052782E3,9.8788344E5,1.9641162E-3,2.0672107E8,-1.0070168E-2,-7.2246315E-3,-5.080012E-4],"split_indices":[51,27,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.3E1,5.3E1,3E1,4.7E1,6E0,6E0,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[1.180388E-2,-5.4423185E-3,8.842814E-3,1.5580512E-2,-4.7720824E-3,-1.6885188E-3,3.366301E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[2.1868587E-1,1.346169E-1,0E0,1.532887E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[4.4845093E3,4.501324E7,8.842814E-3,7.281736E2,-4.7720824E-3,-1.6885188E-3,3.366301E-3],"split_indices":[51,49,0,51,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.8E1,6E0,5.6E1,1.2E1,2.9E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[2.5241738E-2,5.121224E-4,9.879386E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":79,"left_children":[1,-1,-1],"loss_changes":[2.012054E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[4.0235266E3,5.121224E-4,9.879386E-3],"split_indices":[51,0,0],"split_type":[0,0,0],"sum_hessian":[7.7E1,7.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[6.71125E-4,3.1779505E-2,-1.5233433E-1,8.267778E-3,1.1688988E-2,1.8855729E-4,-1.2982552E-2,-3.5211432E-3,1.8684041E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,-1,-1,-1,-1],"loss_changes":[3.3802375E-1,2.8422165E-1,2.1110639E-1,1.3139966E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3],"right_children":[2,4,6,8,-1,-1,-1,-1,-1],"split_conditions":[1.0232127E12,6.7780703E3,1.3E1,2.4786325E0,1.1688988E-2,1.8855729E-4,-1.2982552E-2,-3.5211432E-3,1.8684041E-3],"split_indices":[30,4,3,55,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[6.9E1,5.8E1,1.1E1,5.3E1,5E0,5E0,6E0,1.4E1,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.398545E-3,-6.491392E-4,7.619856E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":81,"left_children":[1,-1,-1],"loss_changes":[2.0059113E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[6.7780703E3,-6.491392E-4,7.619856E-3],"split_indices":[4,0,0],"split_type":[0,0,0],"sum_hessian":[6.7E1,6E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[-3.1757005E-3,1.2169152E-2,-6.9138585E-3,-8.774195E-4,4.333356E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":82,"left_children":[1,3,-1,-1,-1],"loss_changes":[1.4745684E-1,1.4559397E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[3.698723E10,1.1778256E3,-6.9138585E-3,-8.774195E-4,4.333356E-3],"split_indices":[5,51,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[6.8E1,6.2E1,6E0,4.5E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-1.725164E-2,7.355011E-5,-7.3685083E-3,1.6620966E-2,-8.797967E-3,-3.1616262E-3,2.0725317E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[1.795191E-1,2.1063726E-1,0E0,1.3750044E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[1.3237324E6,1.89293E7,-7.3685083E-3,6E0,-8.797967E-3,-3.1616262E-3,2.0725317E-3],"split_indices":[27,1,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.7E1,6.9E1,8E0,6.4E1,5E0,1.5E1,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-8.566814E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0],"id":84,"left_children":[-1],"loss_changes":[0E0],"parents":[2147483647],"right_children":[-1],"split_conditions":[-4.2058897E-4],"split_indices":[0],"split_type":[0],"sum_hessian":[7.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"1","size_leaf_vector":"1"}},{"base_weights":[-1.3000159E-2,4.4473674E-4,-9.763806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":85,"left_children":[1,-1,-1],"loss_changes":[3.0931544E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[1.5092398E8,4.4473674E-4,-9.763806E-3],"split_indices":[44,0,0],"split_type":[0,0,0],"sum_hessian":[7.4E1,6.7E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[5.5318503E-3,2.4580687E-2,-1.0060938E-2,3.0368129E-3,1.1758941E-2,2.0726004E-3,-4.4646367E-2,-7.2275703E-3,1.8371874E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,-1,-1,7,-1,-1],"loss_changes":[2.8967643E-1,3.0488443E-1,0E0,1.1584426E-1,0E0,0E0,1.3951956E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6],"right_children":[2,4,-1,6,-1,-1,8,-1,-1],"split_conditions":[1.06102504E8,3.2311362E3,-1.0060938E-2,1.181508E6,1.1758941E-2,2.0726004E-3,7.927819E5,-7.2275703E-3,1.8371874E-4],"split_indices":[50,4,0,31,0,0,46,0,0],"split_type":[0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.5E1,5E0,6E1,5E0,3.3E1,2.7E1,8E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"9","size_leaf_vector":"1"}},{"base_weights":[6.1434195E-3,-5.4629345E-4,9.86094E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0],"id":87,"left_children":[1,-1,-1],"loss_changes":[2.4771747E-1,0E0,0E0],"parents":[2147483647,0,0],"right_children":[2,-1,-1],"split_conditions":[4.0235266E3,-5.4629345E-4,9.86094E-3],"split_indices":[51,0,0],"split_type":[0,0,0],"sum_hessian":[7.2E1,6.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"3","size_leaf_vector":"1"}},{"base_weights":[2.3054387E-2,-2.5247064E-4,9.521352E-2,1.7581216E-1,-3.9072707E-3,1.2977737E-2,2.4313477E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":88,"left_children":[1,-1,3,5,-1,-1,-1],"loss_changes":[1.4495057E-1,0E0,2.9576367E-1,1.4342767E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3],"right_children":[2,-1,4,6,-1,-1,-1],"split_conditions":[1.462134E3,-2.5247064E-4,1.3946067E7,1.174E3,-3.9072707E-3,1.2977737E-2,2.4313477E-3],"split_indices":[51,0,46,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7E1,5.1E1,1.9E1,1.3E1,6E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-1.51158E-3,1.7167503E-2,-8.203774E-3,-5.691452E-4,4.471863E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":89,"left_children":[1,3,-1,-1,-1],"loss_changes":[2.4412507E-1,1.519949E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[1.06102504E8,9.334E3,-8.203774E-3,-5.691452E-4,4.471863E-3],"split_indices":[50,2,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.7E1,7E1,7E0,5.1E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-1.9875844E-3,1.4524038E-2,-8.305851E-3,-3.3125756E-4,1.0308427E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":90,"left_children":[1,3,-1,-1,-1],"loss_changes":[2.1502304E-1,2.965888E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1],"right_children":[2,4,-1,-1,-1],"split_conditions":[9.1561624E7,3.9370352E3,-8.305851E-3,-3.3125756E-4,1.0308427E-2],"split_indices":[47,4,0,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,6.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[-2.2242067E-2,-1.0352011E-2,-8.790527E-3,1.5853534E-3,-4.694518E-2,-4.212157E-3,2.5675911E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,-1,5,-1,-1],"loss_changes":[1.5849048E-1,1.2791313E-1,0E0,0E0,1.741364E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4],"right_children":[2,4,-1,-1,6,-1,-1],"split_conditions":[1.278211E8,1.5326E4,-8.790527E-3,1.5853534E-3,4.1067896E0,-4.212157E-3,2.5675911E-3],"split_indices":[47,9,0,0,52,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8E1,5E0,3.7E1,4.3E1,3.1E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[9.052075E-3,-1.2239711E-3,6.698479E-2,7.781008E-3,3.6058345E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0],"id":92,"left_children":[1,-1,3,-1,-1],"loss_changes":[1.4932309E-1,0E0,1.5164006E-1,0E0,0E0],"parents":[2147483647,0,0,2,2],"right_children":[2,-1,4,-1,-1],"split_conditions":[1.1778256E3,-1.2239711E-3,3.6274084E9,7.781008E-3,3.6058345E-4],"split_indices":[51,0,12,0,0],"split_type":[0,0,0,0,0],"sum_hessian":[7.4E1,4.7E1,2.7E1,1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"5","size_leaf_vector":"1"}},{"base_weights":[5.755616E-3,3.0328378E-2,-8.521617E-2,4.956823E-4,7.759264E-3,7.978966E-4,-1.0702596E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,-1,-1,-1,-1],"loss_changes":[1.6800615E-1,1.5085879E-1,2.191115E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2],"right_children":[2,4,6,-1,-1,-1,-1],"split_conditions":[1.0232127E12,3.3077192E3,3.5715E4,4.956823E-4,7.759264E-3,7.978966E-4,-1.0702596E-2],"split_indices":[30,51,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[7.3E1,5.8E1,1.5E1,5.1E1,7E0,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[3.1948206E-3,-1.7840095E-2,6.711048E-2,-1.2447803E-4,-7.298514E-3,5.3109317E-3,-2.9667115E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,-1,-1,-1,-1],"loss_changes":[1.1950979E-1,1.3376793E-1,1.21761106E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2],"right_children":[2,4,6,-1,-1,-1,-1],"split_conditions":[4.946992E6,7.9606894E11,3.8218057E2,-1.2447803E-4,-7.298514E-3,5.3109317E-3,-2.9667115E-3],"split_indices":[1,30,57,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[8.7E1,6.6E1,2.1E1,6E1,6E0,1.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_CommandResult","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_TakeOrderedAndProject","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"98"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-9.862387E-2,-6.0200715E-1,2.0130958E-1,-3.3766466E-1,-8.611123E-1,3.9173424E-2,4.3444833E-1,-4.4816887E-1,-3.5482426E-3,-2.3164146E-2,-4.4876166E-2,-5.2764706E-2,1.1688088E-2,1.0995816E-2,5.1811427E-1,-1.2322679E-2,-2.4676442E-2,2.7265228E-2,-1.2190793E-2,2.7749313E-2,1.3551473E-2,-2.727036E-3,6.017809E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[1.1746764E1,1.6955147E0,1.8351918E0,4.2238295E-1,1.4484501E-1,6.215492E-1,1.9185519E-1,4.1671753E-2,0E0,0E0,0E0,3.9112976E-1,0E0,0E0,9.1564655E-2,0E0,0E0,1.669099E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,14,14,17,17],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,8.28645E-1,5.517496E-1,1.264233E6,9.29295E5,1.6E1,4.650193E-1,6.3E2,-3.5482426E-3,-2.3164146E-2,-4.4876166E-2,2.5042E4,1.1688088E-2,1.0995816E-2,3.0894554E0,-1.2322679E-2,-2.4676442E-2,1.419E3,-1.2190793E-2,2.7749313E-2,1.3551473E-2,-2.727036E-3,6.017809E-3],"split_indices":[47,36,51,1,37,3,36,0,0,0,0,9,0,0,63,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.8E1,4.8E1,1.5E1,1.3E1,2.9E1,1.9E1,1E1,5E0,5E0,8E0,2.1E1,8E0,7E0,1.2E1,5E0,5E0,1.6E1,5E0,7E0,5E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-5.8449373E-2,-4.8959684E-1,2.088214E-1,-6.102994E-1,3.3688683E-3,2.7109423E-1,-1.179317E-2,-4.3931776E-1,-4.0408168E-2,1.8732502E-1,2.5101626E-2,-2.3040967E-2,-9.969242E-3,-2.4618267E-3,2.3093347E-1,1.5591662E-2,1.1560212E-1,1.6407159E-3,9.554724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,-1,-1,-1,-1,15,-1,17,-1,-1],"loss_changes":[8.744722E0,2.0213022E0,1.4486926E0,7.837E-1,0E0,9.0301013E-1,0E0,1.5478373E-1,0E0,3.7142825E-1,0E0,0E0,0E0,0E0,3.4564698E-1,0E0,1.14604235E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,14,14,16,16],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,-1,-1,-1,-1,16,-1,18,-1,-1],"split_conditions":[1.0420895E-2,1.962943E3,1.4696082E12,1.2427474E8,3.3688683E-3,1.1997641E0,-1.179317E-2,8.60448E5,-4.0408168E-2,2.5547945E0,2.5101626E-2,-2.3040967E-2,-9.969242E-3,-2.4618267E-3,6.1577463E0,1.5591662E-2,1.381E3,1.6407159E-3,9.554724E-3],"split_indices":[47,4,40,54,0,48,0,1,0,62,0,0,0,0,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,2.8E1,4.6E1,2.3E1,5E0,4.1E1,5E0,1.6E1,7E0,3.3E1,8E0,1.1E1,5E0,5E0,2.8E1,1.3E1,1.5E1,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[4.669221E-2,-2.0188038E-1,3.6301574E-1,7.544298E-2,-4.8248386E-1,4.1054767E-3,4.254897E-1,-5.0308714E-3,1.4403771E-1,-6.27041E-1,-2.6778147E-1,2.8479147E-1,2.5971416E-2,1.4681936E-2,-1.4242097E-3,-3.4102205E-2,-1.5708538E-2,-6.046367E-3,-1.6060654E-2,1.5826551E-2,4.6825516E-3,-3.264818E-3,3.147241E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,-1,11,-1,13,15,17,19,-1,-1,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.4422636E0,3.6160235E0,5.726042E-1,3.2421517E-1,5.459347E-1,0E0,4.5292473E-1,0E0,4.770118E-1,2.0695353E-1,8.263826E-2,1.683768E-1,0E0,0E0,6.0825605E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,9,9,10,10,11,11,14,14],"right_children":[2,4,6,8,10,-1,12,-1,14,16,18,20,-1,-1,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1091123E-1,5.81609E-1,2.1267605E0,6.7572955E2,9.2989355E2,4.1054767E-3,1.1328785E-1,-5.0308714E-3,5.4600003E9,6.451613E-2,1.2106538E1,5.405855E6,2.5971416E-2,1.4681936E-2,1E0,-3.4102205E-2,-1.5708538E-2,-6.046367E-3,-1.6060654E-2,1.5826551E-2,4.6825516E-3,-3.264818E-3,3.147241E-3],"split_indices":[48,36,65,4,4,0,47,0,5,66,65,41,0,0,24,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,4.5E1,3.5E1,2.3E1,2.2E1,7E0,2.8E1,6E0,1.7E1,1.2E1,1E1,1.6E1,1.2E1,7E0,1E1,7E0,5E0,5E0,5E0,1.1E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.6343214E-3,-3.7803835E-1,2.8670993E-1,-3.599862E-2,-6.2063825E-1,2.0509537E-1,3.6485046E-2,-1.9930702E-1,1.2131255E-2,-1.1666655E-2,-7.174317E-1,-5.204573E-3,2.5155342E-1,-1.3659414E-2,-2.7946176E-3,-1.9006673E-2,-3.6623884E-2,4.1146338E-1,1.6807367E-1,1.0120536E-2,2.3088338E-2,6.584778E-2,2.5099245E-1,-5.2300544E-4,7.121873E-3,1.41108725E-2,6.477999E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,-1,17,-1,-1,-1,-1,19,21,-1,-1,23,25,-1,-1,-1,-1],"loss_changes":[9.008171E0,2.9648948E0,1.8419766E0,8.5167956E-1,5.4904747E-1,6.547848E-1,0E0,1.3822863E-1,0E0,0E0,1.0979557E-1,0E0,4.2476583E-1,0E0,0E0,0E0,0E0,1.1165166E-1,2.0386034E-1,0E0,0E0,9.4792165E-2,4.846722E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,12,12,17,17,18,18,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,-1,18,-1,-1,-1,-1,20,22,-1,-1,24,26,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,5.661358E3,6.9983356E2,5.0415697E5,3.1790106E8,3.6485046E-2,1.6841E4,1.2131255E-2,-1.1666655E-2,6.1358623E9,-5.204573E-3,1.7329262E7,-1.3659414E-2,-2.7946176E-3,-1.9006673E-2,-3.6623884E-2,1.2566517E5,2.6597537E3,1.0120536E-2,2.3088338E-2,7.8972295E-2,1E0,-5.2300544E-4,7.121873E-3,1.41108725E-2,6.477999E-3],"split_indices":[51,36,61,61,37,7,0,9,0,0,5,0,54,0,0,0,0,42,4,0,0,47,18,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,3.5E1,4.6E1,1.5E1,2E1,4.1E1,5E0,1E1,5E0,5E0,1.5E1,5E0,3.6E1,5E0,5E0,5E0,1E1,1.1E1,2.5E1,5E0,6E0,1.2E1,1.3E1,7E0,5E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-2.9388414E-4,-4.6793285E-1,1.9177736E-1,-5.9709406E-1,-1.1079321E-4,8.596808E-2,3.3087853E-2,-1.9043617E-2,-3.6493186E-2,-2.2214988E-2,2.385351E-1,9.633547E-2,-1.1156446E-1,1.5210139E-2,1.591168E-1,7.5920154E-3,-7.235793E-4,1.9701398E-3,-1.7992519E-1,8.596402E-3,4.062588E-3,-2.7663398E-3,-1.2976903E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,15,17,-1,19,-1,-1,-1,21,-1,-1,-1,-1],"loss_changes":[7.0956993E0,1.3813133E0,3.1543264E0,3.2610655E-1,0E0,7.975472E-1,0E0,0E0,0E0,3.1792784E-1,1.0388076E-1,1.0830088E-1,1.8836914E-1,0E0,1.1245817E-2,0E0,0E0,0E0,1.3622448E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,11,11,12,12,14,14,18,18],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,16,18,-1,20,-1,-1,-1,22,-1,-1,-1,-1],"split_conditions":[4.823969E-3,1.962943E3,4.0235266E3,9.1569895E-1,-1.1079321E-4,5.096104E-1,3.3087853E-2,-1.9043617E-2,-3.6493186E-2,1.6547014E5,2.156825E7,1.219233E6,2.8496484E5,1.5210139E-2,3.7360862E5,7.5920154E-3,-7.235793E-4,1.9701398E-3,6.2597164E7,8.596402E-3,4.062588E-3,-2.7663398E-3,-1.2976903E-2],"split_indices":[48,4,61,36,0,51,0,0,0,42,54,41,37,0,37,0,0,0,54,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.2E1,5.5E1,1.7E1,5E0,4.7E1,8E0,1.1E1,6E0,2.8E1,1.9E1,1.2E1,1.6E1,7E0,1.2E1,7E0,5E0,5E0,1.1E1,7E0,5E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[7.3581715E-3,-4.2380327E-1,1.5338987E-1,-7.832314E-3,-5.449694E-1,-2.4416048E-2,3.147764E-1,-3.0455884E-2,-1.2562869E-2,-1.3516574E-1,1.6909851E-1,3.9420402E-1,1.345258E-1,2.9607571E-3,-2.1161771E-1,1.3388899E-2,5.7123305E-4,2.077857E-2,1.2343373E-2,9.101421E-3,2.004519E-3,-4.279195E-3,-1.2931953E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,-1,7,9,11,-1,-1,13,15,17,19,-1,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9761395E0,5.107279E-1,1.7007189E0,0E0,2.6555395E-1,6.44375E-1,3.9079237E-1,0E0,0E0,3.0576122E-1,2.1685362E-1,4.3062687E-2,5.792564E-2,0E0,9.604782E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14],"right_children":[2,4,6,-1,8,10,12,-1,-1,14,16,18,20,-1,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9956966E-1,5.0415697E5,3.8151306E-1,-7.832314E-3,1E0,1.1813418E3,3.3027112E0,-3.0455884E-2,-1.2562869E-2,2.2519132E-1,1.23E2,1.1213404E6,6.8221856E8,2.9607571E-3,3.67827E2,1.3388899E-2,5.7123305E-4,2.077857E-2,1.2343373E-2,9.101421E-3,2.004519E-3,-4.279195E-3,-1.2931953E-2],"split_indices":[66,37,51,0,22,61,63,0,0,36,8,41,7,0,61,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,1.9E1,5.8E1,7E0,1.2E1,2.8E1,3E1,7E0,5E0,1.8E1,1E1,2E1,1E1,5E0,1.3E1,5E0,5E0,1.1E1,9E0,5E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.4615096E-2,-4.016892E-1,1.7920302E-1,-1.6755293E-1,-5.375523E-1,1.0909804E-1,2.766199E-2,-1.4834054E-2,1.0015196E-3,-3.4972224E-1,-3.567848E-2,1.5552418E-1,-1.01918E-2,-1.9275827E-2,-9.595912E-3,-4.055958E-3,2.0019968E-1,2.9228196E-1,7.2584346E-2,1.60642E-2,5.476309E-3,-2.2153996E-3,1.3241228E-1,2.8945867E-3,8.036869E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,-1,15,-1,-1,-1,-1,17,19,21,-1,-1,-1,23,-1,-1],"loss_changes":[5.8486757E0,7.370043E-1,1.5190822E0,3.452641E-1,4.88544E-1,7.515412E-1,0E0,0E0,0E0,2.6513577E-2,0E0,4.6788782E-1,0E0,0E0,0E0,0E0,3.9007378E-1,1.76633E-1,1.2308974E-1,0E0,0E0,0E0,2.3049667E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,11,11,16,16,17,17,18,18,22,22],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,-1,16,-1,-1,-1,-1,18,20,22,-1,-1,-1,24,-1,-1],"split_conditions":[7.2921924E-2,5.0415697E5,4.0235266E3,1.3E1,1.1686677E8,1.1269586E12,2.766199E-2,-1.4834054E-2,1.0015196E-3,1E0,-3.567848E-2,2.00809E5,-1.01918E-2,-1.9275827E-2,-9.595912E-3,-4.055958E-3,1.0191781E1,5.7588155E6,1.3652755E6,1.60642E-2,5.476309E-3,-2.2153996E-3,1.05E2,2.8945867E-3,8.036869E-3],"split_indices":[51,37,61,3,54,40,0,0,0,25,0,1,0,0,0,0,65,41,56,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,2.5E1,5.1E1,1E1,1.5E1,4.5E1,6E0,5E0,5E0,1E1,5E0,4E1,5E0,5E0,5E0,6E0,3.4E1,1.9E1,1.5E1,1.3E1,6E0,5E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[6.2830575E-2,-4.0447977E-1,1.8437377E-1,-2.4265692E-1,-2.8537806E-2,8.2740255E-2,6.3882244E-1,-1.7959367E-2,-2.0734656E-3,-2.7911909E-2,2.556467E-1,3.6110822E-2,1.6627938E-2,1.9336864E-2,-1.0621805E-2,1.7290233E-3,3.2550365E-1,8.0534145E-2,-7.520902E-3,1.734515E-2,7.735525E-3,1.5011093E-1,-1.3020118E-3,1.0674801E-2,1.8787312E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,15,-1,-1,17,-1,-1,19,21,-1,-1,-1,23,-1,-1,-1],"loss_changes":[4.3912535E0,4.3981576E-1,2.7645643E0,3.1938505E-1,0E0,9.828918E-1,1.8749332E-1,0E0,0E0,3.1899986E-1,2.910303E-1,0E0,0E0,3.2136866E-1,0E0,0E0,7.14457E-2,1.642562E-1,0E0,0E0,0E0,1.12621546E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,10,10,13,13,16,16,17,17,21,21],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,16,-1,-1,18,-1,-1,20,22,-1,-1,-1,24,-1,-1,-1],"split_conditions":[1.6403629E4,9.1671896E-1,4.0235266E3,1.5E2,-2.8537806E-2,5.517496E-1,1.904071E-1,-1.7959367E-2,-2.0734656E-3,3.9272E4,1E0,3.6110822E-2,1.6627938E-2,6.5957415E-1,-1.0621805E-2,1.7290233E-3,3.73524E5,1.5172464E-1,-7.520902E-3,1.734515E-2,7.735525E-3,3.3207992E-1,-1.3020118E-3,1.0674801E-2,1.8787312E-3],"split_indices":[56,36,61,10,0,51,47,0,0,10,23,0,0,36,0,0,38,51,0,0,0,36,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,1.5E1,6E1,1E1,5E0,5E1,1E1,5E0,5E0,3.1E1,1.9E1,5E0,5E0,2.6E1,5E0,5E0,1.4E1,2E1,6E0,9E0,5E0,1.2E1,8E0,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.5214778E-2,-3.1927425E-1,9.047568E-2,-4.673145E-1,-4.2731594E-3,1.3807961E-1,-9.9821305E-3,-1.1691648E-2,-2.542692E-2,2.8045788E-2,2.9545924E-1,1.2193733E-1,-6.4737676E-3,1.816194E-2,6.429515E-3,1.049641E-2,-6.808411E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,15,-1,-1,-1,-1,-1],"loss_changes":[2.5638394E0,6.8643403E-1,7.735865E-1,1.162014E-1,0E0,7.536652E-1,0E0,0E0,0E0,4.5307913E-1,2.3884797E-1,2.7775213E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,11,11],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,16,-1,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,1.528E3,6.2012E4,7.83225E-1,-4.2731594E-3,1.1778256E3,-9.9821305E-3,-1.1691648E-2,-2.542692E-2,1.7444956E5,4.5402945E6,3.9130908E6,-6.4737676E-3,1.816194E-2,6.429515E-3,1.049641E-2,-6.808411E-4],"split_indices":[50,11,10,36,0,61,0,0,0,42,41,60,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,2.1E1,4.9E1,1.2E1,9E0,4.3E1,6E0,5E0,7E0,2.6E1,1.7E1,1.7E1,9E0,9E0,8E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.4561889E-2,-1.7698422E-1,3.1198317E-1,-4.5643693E-1,-6.309981E-2,2.0582327E-1,2.9235655E-2,-9.119022E-3,-3.046845E-2,5.5200704E-3,-1.446337E-1,1.25233E-2,1.082581E-1,-8.4725164E-2,-1.3071001E-2,2.649624E-3,6.2877536E-3,-6.192315E-3,-3.6696967E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,13,-1,15,17,-1,-1,-1,-1,-1],"loss_changes":[3.9291687E0,1.3078616E0,8.330536E-1,5.3390074E-1,4.8705685E-1,1.2504303E-1,0E0,0E0,0E0,0E0,1.6735238E-1,0E0,8.839026E-3,6.7598395E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,10,10,12,12,13,13],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,14,-1,16,18,-1,-1,-1,-1,-1],"split_conditions":[4.7985215E-2,5.382431E0,1.91E4,9.6910744E7,3.459181E-1,2.4832625E6,2.9235655E-2,-9.119022E-3,-3.046845E-2,5.5200704E-3,1.6920523E1,1.25233E-2,3.5265556E5,1.7058623E3,-1.3071001E-2,2.649624E-3,6.2877536E-3,-6.192315E-3,-3.6696967E-4],"split_indices":[47,65,2,54,36,41,0,0,0,0,65,0,37,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,4.1E1,2.6E1,1.1E1,3E1,2.1E1,5E0,6E0,5E0,9E0,2.1E1,1.1E1,1E1,1.6E1,5E0,5E0,5E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.3918804E-2,-3.4741983E-1,1.688255E-1,-5.11773E-1,3.3625802E-3,2.3523538E-1,-1.0612452E-1,-3.4843767E-1,-3.1108893E-2,1.7480367E-1,2.3463449E-2,-1.0634102E-2,1.8728639E-3,-1.8832898E-2,-8.885289E-3,-8.527525E-4,2.2890243E-1,1.5184934E-2,1.633019E-1,2.0294422E-3,2.1075442E-1,5.951426E-3,1.1400721E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,-1,-1,-1,17,-1,19,-1,21,-1,-1],"loss_changes":[4.9952316E0,2.050013E0,9.927124E-1,3.73487E-1,0E0,6.516378E-1,2.2110021E-1,5.4578066E-2,0E0,3.921368E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5098464E-1,0E0,1.0271162E-1,0E0,1.3215184E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,16,16,18,18,20,20],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,-1,-1,-1,18,-1,20,-1,22,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,1.8E2,8.616169E-1,3.3625802E-3,6.7780703E3,7.087626E5,1.500766E3,-3.1108893E-2,3.6696808E2,2.3463449E-2,-1.0634102E-2,1.8728639E-3,-1.8832898E-2,-8.885289E-3,-8.527525E-4,2.325535E7,1.5184934E-2,9.9583336E1,2.0294422E-3,1.3136593E3,5.951426E-3,1.1400721E-2],"split_indices":[51,4,8,36,0,4,41,4,0,61,0,0,0,0,0,0,54,0,67,0,61,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.8E1,5.2E1,2E1,8E0,4.2E1,1E1,1.2E1,8E0,3.6E1,6E0,5E0,5E0,7E0,5E0,8E0,2.8E1,9E0,1.9E1,6E0,1.3E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.8137358E-2,-3.315699E-1,1.901542E-1,-4.451193E-1,3.553454E-3,1.0560054E-1,2.1794267E-2,-5.4865736E-1,-6.4653456E-3,-2.4885505E-3,1.385182E-1,-1.5190209E-2,-3.359215E-2,1.7481126E-1,-4.9030205E-4,2.0598076E-1,2.9375888E-3,1.0641137E-2,4.5429347E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,-1,9,-1,11,-1,-1,13,-1,-1,15,-1,17,-1,-1,-1],"loss_changes":[5.229517E0,1.5732539E0,1.0917451E0,7.329755E-1,0E0,2.0357984E-1,0E0,5.406656E-1,0E0,0E0,1.6999823E-1,0E0,0E0,7.194853E-2,0E0,3.6719143E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,10,10,13,13,15,15],"right_children":[2,4,6,8,-1,10,-1,12,-1,-1,14,-1,-1,16,-1,18,-1,-1,-1],"split_conditions":[1.4865639E-2,1.962943E3,1.0272479E0,1E0,3.553454E-3,4.57563E5,2.1794267E-2,1.0370839E6,-6.4653456E-3,-2.4885505E-3,1.0232127E12,-1.5190209E-2,-3.359215E-2,1.3096068E7,-4.9030205E-4,1.3773196E1,2.9375888E-3,1.0641137E-2,4.5429347E-3],"split_indices":[47,4,48,22,0,1,0,37,0,0,40,0,0,41,0,65,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,3.2E1,4.5E1,2.5E1,7E0,3.6E1,9E0,1.8E1,7E0,6E0,3E1,1E1,8E0,2.4E1,6E0,1.8E1,6E0,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-3.0069263E-3,-3.0329373E-1,1.4446126E-1,4.1629304E-3,-5.062998E-1,4.8412222E-2,3.3920583E-1,-1.5650945E-2,-2.6952373E-2,9.1343336E-2,-8.815994E-3,4.1818333E-1,5.0233356E-3,1.4711976E-1,-8.29332E-3,2.2858871E-2,1.0322155E-2,3.3000495E-2,2.08686E-1,-1.1927236E-3,3.917119E-3,1.23104E-2,4.217053E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,-1,7,9,11,-1,-1,13,-1,15,-1,17,-1,-1,-1,19,21,-1,-1,-1,-1],"loss_changes":[3.6307173E0,2.215923E0,1.012355E0,0E0,5.5510998E-2,4.162499E-1,2.769792E-1,0E0,0E0,5.3421205E-1,0E0,1.0335827E-1,0E0,1.8983823E-1,0E0,0E0,0E0,3.7631277E-2,1.08355045E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,9,9,11,11,13,13,17,17,18,18],"right_children":[2,4,6,-1,8,10,12,-1,-1,14,-1,16,-1,18,-1,-1,-1,20,22,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,5.8871865E-1,9.306648E-1,4.1629304E-3,8.595346E-1,1.1269586E12,3.3662374E0,-1.5650945E-2,-2.6952373E-2,2.69722E5,-8.815994E-3,1.3996E4,5.0233356E-3,3.7057927E0,-8.29332E-3,2.2858871E-2,1.0322155E-2,8.8E1,1.7616E4,-1.1927236E-3,3.917119E-3,1.23104E-2,4.217053E-3],"split_indices":[51,36,48,0,36,40,63,0,0,42,0,9,0,66,0,0,0,8,9,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,2.6E1,5.4E1,9E0,1.7E1,3.7E1,1.7E1,8E0,9E0,3.2E1,5E0,1.2E1,5E0,2.7E1,5E0,7E0,5E0,1E1,1.7E1,5E0,5E0,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[4.4412617E-2,-1.0881378E-1,2.5491932E-1,-3.7308684E-1,-2.7305193E-2,-6.5713446E-4,3.253334E-1,-1.0574069E-2,-2.0226564E-2,8.985541E-2,-1.4302868E-1,4.395818E-3,3.726496E-1,1.5426318E-1,-3.3955511E-3,-8.354933E-2,-1.1616815E-2,4.3615845E-1,1.1384791E-2,9.265699E-4,1.2834208E-2,-2.1655929E-4,-7.277066E-3,2.3288347E-2,1.1503292E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,-1,11,-1,-1,13,15,-1,17,19,-1,21,-1,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.608153E0,1.0022174E0,6.4998126E-1,1.0248423E-2,5.145065E-1,0E0,2.545371E-1,0E0,0E0,2.1386974E-1,1.0834882E-1,0E0,5.7150126E-2,2.3884398E-1,0E0,8.52657E-2,0E0,5.8187485E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,9,9,10,10,12,12,13,13,15,15,17,17],"right_children":[2,4,6,8,10,-1,12,-1,-1,14,16,-1,18,20,-1,22,-1,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.839604E-1,1.346E3,2.045738E0,8.31E2,5.81609E-1,-6.5713446E-4,3.3389777E-1,-1.0574069E-2,-2.0226564E-2,1.6372213E-1,3.6979167E0,4.395818E-3,1.3595E4,1E0,-3.3955511E-3,1.5326E4,-1.1616815E-2,1.0956273E10,1.1384791E-2,9.265699E-4,1.2834208E-2,-2.1655929E-4,-7.277066E-3,2.3288347E-2,1.1503292E-2],"split_indices":[51,11,65,11,36,0,36,0,0,50,63,0,9,18,0,9,0,5,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,4.6E1,3.3E1,1E1,3.6E1,7E0,2.6E1,5E0,5E0,1.8E1,1.8E1,5E0,2.1E1,1.3E1,5E0,1.3E1,5E0,1.2E1,9E0,7E0,6E0,7E0,6E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.7483963E-2,-2.967343E-1,1.4287986E-1,-1.5759493E-1,-2.7564643E-2,5.9607822E-2,2.491453E-2,-2.440191E-1,5.8874703E-4,-6.6002377E-4,1.9148499E-1,-1.4772793E-2,-4.741756E-3,-7.4318857E-3,3.771744E-2,1.1976825E-2,2.5992675E-3,9.368486E-3,-4.6703126E-3,-6.681444E-2,4.8929673E-2,2.1699785E-4,-5.343766E-3,4.7880486E-3,-2.040957E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,-1,-1,17,-1,-1,-1,19,21,23,-1,-1,-1,-1],"loss_changes":[3.3127367E0,9.5674825E-1,2.017508E0,2.6868954E-1,0E0,4.0547198E-1,0E0,1.053527E-1,0E0,2.3333123E-1,1.440618E-1,0E0,0E0,0E0,2.1756805E-1,0E0,0E0,0E0,8.6542316E-2,4.514647E-2,8.0559865E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,14,14,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,-1,-1,18,-1,-1,-1,20,22,24,-1,-1,-1,-1],"split_conditions":[1.6405078E-2,7.916317E11,3.9514639E3,4.593E3,-2.7564643E-2,5.596155E-1,2.491453E-2,5E0,5.8874703E-4,3.8414814E0,5.557455E8,-1.4772793E-2,-4.741756E-3,-7.4318857E-3,2.7014925E0,1.1976825E-2,2.5992675E-3,9.368486E-3,1.4343789E6,7.1E1,6.908864E8,2.1699785E-4,-5.343766E-3,4.7880486E-3,-2.040957E-3],"split_indices":[50,40,61,2,0,48,0,8,0,65,7,0,0,0,62,0,0,0,56,8,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.3E1,5.9E1,1.7E1,6E0,5E1,9E0,1.1E1,6E0,3.5E1,1.5E1,6E0,5E0,6E0,2.9E1,9E0,6E0,5E0,2.4E1,1.1E1,1.3E1,5E0,6E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.597757E-2,3.5215583E-2,-2.4309052E-2,-9.5162354E-2,1.8272336E-1,4.6313256E-2,-2.3210642E-1,1.15995735E-1,3.065123E-1,-8.8702645E-3,8.364464E-3,-1.6641067E-2,-1.3698837E-1,7.197586E-5,1.6973576E-1,1.6434519E-2,8.431293E-3,5.8803847E-3,-8.413107E-2,5.072896E-4,-9.933911E-3,8.905382E-3,3.8452884E-3,-5.5984776E-3,-1.3470642E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,-1,5,7,9,11,13,15,17,-1,-1,19,-1,21,-1,-1,-1,23,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6603448E0,1.479002E0,0E0,8.0509365E-1,2.6181102E-1,1.6323498E-1,2.2391605E-1,1.5342164E-1,1.53285265E-2,1.789161E-1,0E0,0E0,1.7603242E-1,0E0,2.3298383E-2,0E0,0E0,0E0,2.0252414E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,12,12,14,14,18,18],"right_children":[2,4,-1,6,8,10,12,14,16,18,-1,-1,20,-1,22,-1,-1,-1,24,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5414109E8,3.2711282E-2,-2.4309052E-2,5.81609E-1,2.4273067E1,1.04089E3,3.507E3,2.6880343E0,4.5512906E2,1.0191781E1,8.364464E-3,-1.6641067E-2,6.8989144E7,7.197586E-5,5.21286E5,1.6434519E-2,8.431293E-3,5.8803847E-3,1.3E1,5.072896E-4,-9.933911E-3,8.905382E-3,3.8452884E-3,-5.5984776E-3,-1.3470642E-3],"split_indices":[54,47,0,36,66,61,2,63,67,65,0,0,54,0,9,0,0,0,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.5E1,8E0,4E1,3.5E1,2E1,2E1,2.4E1,1.1E1,1.5E1,5E0,7E0,1.3E1,8E0,1.6E1,6E0,5E0,5E0,1E1,5E0,8E0,1.1E1,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.8024659E-2,-2.882975E-1,1.1031554E-1,-4.0481225E-1,4.612586E-3,-1.1184225E-2,2.556108E-1,-1.1859559E-2,-2.5403768E-2,2.950208E-2,-8.664284E-3,3.253581E-1,5.0199875E-3,-4.747502E-3,8.199909E-2,8.440812E-3,3.835424E-1,8.045648E-3,3.7863947E-2,1.0083558E-2,2.1580247E-2,-2.268885E-3,6.5707914E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,-1,15,-1,-1,17,-1,19,-1,21,-1,-1,-1,-1],"loss_changes":[2.9108138E0,1.2706614E0,1.0090992E0,3.246963E-1,0E0,2.4162361E-1,2.2510159E-1,0E0,0E0,1.9989786E-1,0E0,6.452012E-2,0E0,0E0,7.856691E-2,0E0,6.0700536E-2,0E0,1.5038776E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,11,11,14,14,16,16,18,18],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,-1,16,-1,-1,18,-1,20,-1,22,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,1.9315491E3,5.451147E-1,9.29295E5,4.612586E-3,6.2012E4,1.5909592E5,-1.1859559E-2,-2.5403768E-2,7.095772E2,-8.664284E-3,9.3829626E-1,5.0199875E-3,-4.747502E-3,5.090285E7,8.440812E-3,1.3E1,8.045648E-3,3.048E3,1.0083558E-2,2.1580247E-2,-2.268885E-3,6.5707914E-3],"split_indices":[51,4,51,37,0,10,42,0,0,4,0,48,0,0,54,0,3,0,4,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2.6E1,5.6E1,2E1,6E0,3.1E1,2.5E1,1.2E1,8E0,2.6E1,5E0,1.6E1,9E0,7E0,1.9E1,6E0,1E1,5E0,1.4E1,5E0,5E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.4273314E-2,-3.4339988E-1,1.3252896E-1,-4.6473315E-1,-1.9201831E-3,6.792966E-2,2.298457E-2,-1.148426E-2,-3.0115059E-2,-8.119804E-2,1.7854953E-1,4.2215087E-2,-1.1535413E-2,1.337595E-2,1.15967244E-1,-1.3712654E-3,5.401756E-3,1.5555415E-1,1.4201424E-3,3.3471477E-3,9.139082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,15,-1,-1,17,-1,-1,19,-1,-1,-1],"loss_changes":[3.6483738E0,8.559902E-1,1.161492E0,5.568104E-1,0E0,7.238467E-1,0E0,0E0,0E0,4.2109227E-1,1.5093958E-1,7.144435E-2,0E0,0E0,5.5256084E-2,0E0,0E0,3.111288E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,11,11,14,14,17,17],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,16,-1,-1,18,-1,-1,20,-1,-1,-1],"split_conditions":[4.856813E0,1.650581E3,3.3077192E3,1.0789844E6,-1.9201831E-3,2.5222173E-1,2.298457E-2,-1.148426E-2,-3.0115059E-2,6.5957415E-1,4.515E3,4.02451E0,-1.1535413E-2,1.337595E-2,3.2449896E0,-1.3712654E-3,5.401756E-3,1.0584E4,1.4201424E-3,3.3471477E-3,9.139082E-3],"split_indices":[67,4,61,37,0,48,0,0,0,36,10,62,0,0,63,0,0,2,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.3E1,4.8E1,1.6E1,7E0,4.2E1,6E0,9E0,7E0,1.8E1,2.4E1,1.1E1,7E0,7E0,1.7E1,6E0,5E0,1.1E1,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-6.246863E-3,-2.7051753E-1,1.2840451E-1,-3.4403545E-1,2.5391607E-3,-8.167514E-3,2.5113004E-1,-2.4448703E-1,-2.6358308E-2,6.2740326E-2,-1.2014255E-2,3.1817213E-1,3.0078297E-3,-1.5558649E-2,-5.8141574E-3,1.0709817E-1,-3.305113E-3,2.0458477E-2,1.7819846E-1,9.838824E-3,2.6475193E-2,4.368449E-3,1.0011719E-2,-1.4164564E-3,3.6021478E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,17,-1,-1,-1,19,-1,-1,21,-1,23,-1,-1,-1,-1],"loss_changes":[2.952657E0,6.9233346E-1,9.238992E-1,4.0938902E-1,0E0,5.118974E-1,3.3713508E-1,1.65187E-1,0E0,1.407133E-1,0E0,3.1924963E-1,0E0,0E0,0E0,1.4689031E-1,0E0,0E0,2.1428019E-2,0E0,3.6619857E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,11,11,15,15,18,18,20,20],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,18,-1,-1,-1,20,-1,-1,22,-1,24,-1,-1,-1,-1],"split_conditions":[7.2921924E-2,4.1067896E0,5.451147E-1,1.2427474E8,2.5391607E-3,4.6302417E-1,1E0,2.23E2,-2.6358308E-2,5.0999485E9,-1.2014255E-2,3.827309E6,3.0078297E-3,-1.5558649E-2,-5.8141574E-3,3.6130127E-1,-3.305113E-3,2.0458477E-2,3.480396E6,9.838824E-3,1.27426E6,4.368449E-3,1.0011719E-2,-1.4164564E-3,3.6021478E-3],"split_indices":[51,62,51,54,0,50,22,10,0,12,0,56,0,0,0,36,0,0,1,0,1,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,2.7E1,5.4E1,2.2E1,5E0,2.6E1,2.8E1,1.7E1,5E0,2.1E1,5E0,2E1,8E0,8E0,9E0,1.6E1,5E0,9E0,1.1E1,6E0,1E1,5E0,6E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.2830725E-2,-2.0071347E-1,1.2152E-1,5.919548E-3,-3.647612E-1,8.625346E-2,1.620599E-2,-2.2558683E-1,-2.392727E-2,1.1035564E-2,5.9608854E-2,-1.4554952E-2,-4.0686405E-3,-4.085901E-3,1.2687102E-1,6.027212E-2,-6.8072475E-2,7.4713444E-3,1.6315916E-3,-8.3391304E-4,5.8097504E-3,-5.0350577E-3,-5.847473E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,-1,7,9,-1,11,-1,-1,13,-1,-1,15,17,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7187179E0,1.410241E0,3.808726E-1,0E0,2.7428842E-1,1.7157972E-1,0E0,1.1603737E-1,0E0,0E0,1.67604E-1,0E0,0E0,9.058191E-2,6.1230123E-2,6.1971016E-2,2.5053486E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,10,10,13,13,14,14,15,15,16,16],"right_children":[2,4,6,-1,8,10,-1,12,-1,-1,14,-1,-1,16,18,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0420895E-2,5.81609E-1,6.5317163E3,5.919548E-3,1.1686677E8,3.5175372E5,1.620599E-2,9.408949E2,-2.392727E-2,1.1035564E-2,5.517496E-1,-1.4554952E-2,-4.0686405E-3,1.219233E6,6.7584877E8,7.016421E5,4.4233555E6,7.4713444E-3,1.6315916E-3,-8.3391304E-4,5.8097504E-3,-5.0350577E-3,-5.847473E-4],"split_indices":[47,36,4,0,54,41,0,4,0,0,51,0,0,41,7,41,41,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,2.4E1,4.8E1,8E0,1.6E1,4.3E1,5E0,1E1,6E0,5E0,3.8E1,5E0,5E0,2E1,1.8E1,1E1,1E1,1.2E1,6E0,5E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-3.4006756E-2,-2.6129368E-1,9.528785E-2,-2.2818733E-2,-4.423146E-1,1.3030693E-2,5.946229E-2,5.4738317E-3,-5.2476204E-3,-1.1699113E-2,-2.7701257E-2,-8.620615E-3,1.484011E-1,2.3191512E-2,-5.57607E-3,1.6927299E-1,3.2191456E-3,4.5021186E-3,-1.9307902E-2,9.008431E-3,4.0421328E-3,1.7730128E-3,-2.367979E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,-1,13,15,17,-1,19,-1,-1,21,-1,-1,-1,-1],"loss_changes":[2.43463E0,1.2849753E0,3.513148E-1,2.0359059E-1,3.7647772E-1,0E0,2.7981907E-1,0E0,0E0,0E0,0E0,1.018268E-1,2.0005524E-2,7.372567E-2,0E0,1.8749654E-2,0E0,0E0,3.1362485E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,11,11,12,12,13,13,15,15,18,18],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,-1,14,16,18,-1,20,-1,-1,22,-1,-1,-1,-1],"split_conditions":[1.4865639E-2,5.0415697E5,2.9862975E5,1.8144448E-4,2.5036643E8,1.3030693E-2,6.304023E-1,5.4738317E-3,-5.2476204E-3,-1.1699113E-2,-2.7701257E-2,6.855354E-2,7.7591565E8,3.4768486E-1,-5.57607E-3,2.6529046E7,3.2191456E-3,4.5021186E-3,2.26133E-2,9.008431E-3,4.0421328E-3,1.7730128E-3,-2.367979E-3],"split_indices":[47,37,41,47,7,0,50,0,0,0,0,47,7,36,0,54,0,0,47,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,2.9E1,5.2E1,1.3E1,1.6E1,7E0,4.5E1,5E0,8E0,9E0,7E0,2.6E1,1.9E1,2.1E1,5E0,1.4E1,5E0,7E0,1.4E1,9E0,5E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-8.888685E-5,-2.581531E-1,1.4654054E-1,-1.4109373E-1,-2.4271827E-2,1.8015062E-2,2.1934028E-1,-9.91262E-3,-6.360061E-2,4.1484954E-3,-1.9397055E-3,3.1866407E-1,1.2493092E-1,1.3607525E-3,-6.6113775E-3,6.9165006E-3,1.8130811E-2,1.693837E-3,7.258957E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,-1,15,17,-1,-1,-1,-1,-1,-1],"loss_changes":[2.610937E0,7.458632E-1,4.0773988E-1,1.0237083E-1,0E0,8.093516E-2,2.227478E-1,0E0,9.0319216E-2,0E0,0E0,1.183449E-1,4.4606656E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,11,11,12,12],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,-1,16,18,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7489342E-2,1.5135763E8,2.8831577E-1,1.346E3,-2.4271827E-2,4.4605106E5,1.5312917E5,-9.91262E-3,9.837914E0,4.1484954E-3,-1.9397055E-3,3.9976162E-1,1.7325155E5,1.3607525E-3,-6.6113775E-3,6.9165006E-3,1.8130811E-2,1.693837E-3,7.258957E-3],"split_indices":[47,54,50,11,0,37,42,0,65,0,0,36,42,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.7E1,2.4E1,4.3E1,1.8E1,6E0,1.6E1,2.7E1,8E0,1E1,7E0,9E0,1.2E1,1.5E1,5E0,5E0,5E0,7E0,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-9.846738E-3,-2.51504E-1,1.1354416E-1,-1.4695892E-1,-2.3158148E-2,5.6752555E-2,2.3867259E-2,-2.267551E-1,1.5142717E-3,-3.8335912E-2,1.535078E-1,-5.4650675E-3,-1.5627906E-2,7.902697E-2,-1.4372195E-1,1.8419813E-1,1.6144481E-3,-5.234666E-4,7.7279834E-3,-9.152675E-3,-1.81887E-3,1.06144E-2,5.3772526E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,-1,9,-1,11,-1,13,15,-1,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.294748E0,6.3868E-1,1.1761959E0,2.9582742E-1,0E0,4.2909902E-1,0E0,1.3537478E-1,0E0,3.0819973E-1,7.644653E-2,0E0,0E0,1.0271519E-1,7.175952E-2,3.1927824E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,9,9,10,10,13,13,14,14,15,15],"right_children":[2,4,6,8,-1,10,-1,12,-1,14,16,-1,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.856813E0,7.916317E11,3.4414336E3,4.147E3,-2.3158148E-2,2.3239934E-1,2.3867259E-2,1.7E1,1.5142717E-3,3.2760158E6,7.53991E8,-5.4650675E-3,-1.5627906E-2,4.4385012E-2,1.742E3,2.6529046E7,1.6144481E-3,-5.234666E-4,7.7279834E-3,-9.152675E-3,-1.81887E-3,1.06144E-2,5.3772526E-3],"split_indices":[67,40,61,2,0,48,0,3,0,57,7,0,0,48,0,54,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,2.5E1,5E1,1.9E1,6E0,4.5E1,5E0,1.3E1,6E0,2.3E1,2.2E1,8E0,5E0,1.1E1,1.2E1,1.7E1,5E0,6E0,5E0,7E0,5E0,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-8.280011E-3,-2.289028E-1,9.337208E-2,-4.059768E-1,-6.1463047E-2,1.5119068E-1,-1.2536576E-1,-8.651436E-3,-2.486447E-2,-6.652043E-3,2.872661E-3,-9.424092E-4,1.8833493E-1,9.0478186E-4,-1.1254503E-2,2.4542288E-1,9.33282E-2,1.6961582E-2,1.622704E-1,6.4719195E-4,6.105963E-3,1.8430453E-3,1.013797E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,-1,15,-1,-1,17,19,-1,21,-1,-1,-1,-1],"loss_changes":[1.6361003E0,6.569798E-1,6.5131E-1,2.3781323E-1,1.4988303E-1,2.5967038E-1,2.043294E-1,0E0,0E0,0E0,0E0,0E0,1.5607858E-1,0E0,0E0,1.5714443E-1,4.4772968E-2,0E0,9.7578645E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,12,12,15,15,16,16,18,18],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,-1,16,-1,-1,18,20,-1,22,-1,-1,-1,-1],"split_conditions":[2.0113895E-2,3.554517E0,1.55E2,1.0370839E6,1.7493458E3,2.5547945E0,4.9379824E8,-8.651436E-3,-2.486447E-2,-6.652043E-3,2.872661E-3,-9.424092E-4,3.1419718E0,9.0478186E-4,-1.1254503E-2,2.0571364E6,2.931939E5,1.6961582E-2,8.27E2,6.4719195E-4,6.105963E-3,1.8430453E-3,1.013797E-2],"split_indices":[50,63,8,37,4,62,7,0,0,0,0,0,62,0,0,56,37,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,2.2E1,4.9E1,1E1,1.2E1,3.9E1,1E1,5E0,5E0,7E0,5E0,7E0,3.2E1,5E0,5E0,1.9E1,1.3E1,6E0,1.3E1,5E0,8E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.2180732E-2,-8.515637E-2,1.7100401E-1,-1.984918E-1,1.7021919E-2,2.12083E-1,2.900735E-3,-1.2747718E-1,-2.0239852E-2,6.8714045E-2,-8.321382E-3,1.1245141E-2,5.416173E-3,-1.1375131E-2,-3.555775E-2,-1.80051E-2,1.6348891E-1,1.6485279E-3,-5.550655E-3,4.523866E-3,-3.930492E-3,1.0501314E-2,1.949277E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,-1,-1,-1,17,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[9.8783666E-1,7.097843E-1,7.390809E-2,4.5957005E-1,3.5696316E-1,1.8340528E-2,0E0,2.6706603E-1,0E0,2.2485109E-1,0E0,0E0,0E0,0E0,9.945461E-2,1.3187918E-1,9.884241E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,14,14,15,15,16,16],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,-1,-1,-1,18,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[5.517496E-1,1.7742582E3,6.585265E-1,9.1569895E-1,3.45E2,1.6089E4,2.900735E-3,1.648E3,-2.0239852E-2,2.6597537E3,-8.321382E-3,1.1245141E-2,5.416173E-3,-1.1375131E-2,2.8318418E6,7.76264E7,1E0,1.6485279E-3,-5.550655E-3,4.523866E-3,-3.930492E-3,1.0501314E-2,1.949277E-3],"split_indices":[51,4,36,36,8,9,0,11,0,4,0,0,0,0,59,54,18,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,6E1,1.9E1,2.8E1,3.2E1,1.3E1,6E0,2.3E1,5E0,2.6E1,6E0,8E0,5E0,9E0,1.4E1,1.4E1,1.2E1,8E0,6E0,5E0,9E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-2.1408797E-4,3.486389E-2,-1.8436518E-2,-1.9750282E-1,1.0182759E-1,-1.38605945E-2,-2.1941962E-3,5.1483627E-2,2.9852313E-1,6.8675876E-3,1.0320338E-2,1.968277E-2,4.9621663E-3,-6.5697166E-3,6.1180133E-2,8.588861E-2,-1.9692983E-3,1.3364293E-1,3.0976914E-2,3.0628638E-3,7.691309E-3,2.7525255E-3,-4.2145824E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,-1,5,7,-1,-1,9,11,13,-1,-1,-1,-1,15,17,-1,19,21,-1,-1,-1,-1],"loss_changes":[1.0908365E0,1.1093208E0,0E0,2.4756342E-1,5.292648E-1,0E0,0E0,3.5520038E-1,2.3890465E-1,3.1591064E-1,0E0,0E0,0E0,0E0,7.633852E-2,5.617237E-2,0E0,1.764527E-2,1.5165121E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,9,9,14,14,15,15,17,17,18,18],"right_children":[2,4,-1,6,8,-1,-1,10,12,14,-1,-1,-1,-1,16,18,-1,20,22,-1,-1,-1,-1],"split_conditions":[3.698723E10,1.648E3,-1.8436518E-2,3.068E3,4.2930435E3,-1.38605945E-2,-2.1941962E-3,5.6296086E-1,1.3946067E7,5.03E2,1.0320338E-2,1.968277E-2,4.9621663E-3,-6.5697166E-3,8.968064E-1,9.709474E0,-1.9692983E-3,4.0348735E0,2.7252597E-1,3.0628638E-3,7.691309E-3,2.7525255E-3,-4.2145824E-4],"split_indices":[5,11,0,2,4,0,0,48,56,0,0,0,0,0,36,65,0,65,36,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,1.5E1,5.4E1,8E0,7E0,4.4E1,1E1,3.6E1,8E0,5E0,5E0,9E0,2.7E1,2.2E1,5E0,1.1E1,1.1E1,5E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.7703414E-2,3.58037E-2,-3.0034092E-1,-2.011917E-1,8.4219925E-2,-1.937442E-2,-7.446577E-3,-1.1610786E-2,-5.578973E-3,4.9172696E-2,2.2949696E-1,1.1640821E-1,-6.935361E-2,4.921054E-3,1.4025344E-2,2.0327878E-1,6.536133E-2,-6.6710846E-3,1.1662383E-3,5.256341E-3,1.1525583E-2,-2.3985698E-3,1.05761215E-1,1.8354486E-3,5.9261145E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,15,17,-1,-1,19,21,-1,-1,-1,-1,-1,23,-1,-1],"loss_changes":[1.2357259E0,8.086795E-1,1.5679681E-1,2.060321E-2,2.840254E-1,0E0,0E0,0E0,0E0,3.9059386E-1,7.435024E-2,1.24182016E-1,1.3892867E-1,0E0,0E0,2.0269275E-2,1.06275916E-1,0E0,0E0,0E0,0E0,0E0,2.1510616E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,11,11,12,12,15,15,16,16,22,22],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,16,18,-1,-1,20,22,-1,-1,-1,-1,-1,24,-1,-1],"split_conditions":[1.0232127E12,1.885E3,2.277961E6,2.8353047E0,4.9891987E3,-1.937442E-2,-7.446577E-3,-1.1610786E-2,-5.578973E-3,6.2597164E7,1.3773196E1,1.7329262E7,1.962943E3,4.921054E-3,1.4025344E-2,1.117223E5,4.57563E5,-6.6710846E-3,1.1662383E-3,5.256341E-3,1.1525583E-2,-2.3985698E-3,3.0059965E0,1.8354486E-3,5.9261145E-3],"split_indices":[40,2,1,62,4,0,0,0,0,54,65,54,4,0,0,42,1,0,0,0,0,0,62,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,6.8E1,1.2E1,1.1E1,5.7E1,5E0,7E0,5E0,6E0,4.7E1,1E1,3E1,1.7E1,5E0,5E0,1E1,2E1,9E0,8E0,5E0,5E0,5E0,1.5E1,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.6433265E-2,2.5819914E-2,-2.3541788E-2,-1.04581915E-1,1.059301E-1,8.056722E-4,-1.6000858E-1,2.309457E-1,5.411986E-2,-9.261005E-3,-2.6518852E-3,5.974638E-3,1.270683E-2,8.300226E-3,8.463519E-3,4.9844302E-2,-4.4972345E-2,-1.2610152E-3,5.031301E-3,-4.3205665E-3,6.078207E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,-1,5,7,-1,9,11,13,-1,-1,-1,-1,-1,15,17,19,-1,-1,-1,-1],"loss_changes":[1.9068601E0,7.1099484E-1,0E0,1.7935863E-1,2.5953898E-1,0E0,7.0979744E-2,2.297765E-2,1.8272379E-1,0E0,0E0,0E0,0E0,0E0,5.5310845E-2,7.057678E-2,3.408076E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,14,14,15,15,16,16],"right_children":[2,4,-1,6,8,-1,10,12,14,-1,-1,-1,-1,-1,16,18,20,-1,-1,-1,-1],"split_conditions":[1.5414109E8,2.1647314E8,-2.3541788E-2,3.459181E-1,2.6529046E7,8.056722E-4,8.5633373E-1,7.285458E-1,3.5010372E5,-9.261005E-3,-2.6518852E-3,5.974638E-3,1.270683E-2,8.300226E-3,3.125978E0,1.2052782E3,1.775E3,-1.2610152E-3,5.031301E-3,-4.3205665E-3,6.078207E-4],"split_indices":[54,12,0,36,54,0,36,48,41,0,0,0,0,0,63,61,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.6E1,6E0,2.5E1,4.1E1,8E0,1.7E1,1.1E1,3E1,1.1E1,6E0,5E0,6E0,7E0,2.3E1,1.3E1,1E1,6E0,7E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.8151473E-2,-2.1678317E-1,1.09349504E-1,-1.1315897E-5,-3.0337915E-1,6.683769E-2,1.6075673E-2,-1.7571116E-2,-6.682913E-3,-4.5026666E-3,8.5520394E-2,1.5943106E-1,2.1753771E-2,2.3355565E-3,1.8642882E-1,-2.9109651E-3,5.954511E-2,2.222298E-1,5.128941E-3,5.221091E-3,-3.8048907E-4,1.2405388E-2,5.94106E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,-1,7,9,-1,-1,-1,-1,11,13,15,-1,17,-1,19,21,-1,-1,-1,-1,-1],"loss_changes":[1.6744266E0,3.936904E-1,6.393918E-1,0E0,1.4391947E-1,1.7997608E-1,0E0,0E0,0E0,0E0,2.3218343E-1,5.7123065E-2,9.527177E-2,0E0,2.142787E-2,0E0,7.767786E-2,1.6552627E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,10,10,11,11,12,12,14,14,16,16,17,17],"right_children":[2,4,6,-1,8,10,-1,-1,-1,-1,12,14,16,-1,18,-1,20,22,-1,-1,-1,-1,-1],"split_conditions":[1.217012E-2,5.6660336E-1,3.9514639E3,-1.1315897E-5,1.021346E3,2.074E3,1.6075673E-2,-1.7571116E-2,-6.682913E-3,-4.5026666E-3,4.8178736E7,2.5630938E5,1.0781599E3,2.3355565E-3,3.6726675E6,-2.9109651E-3,3.28464E0,2.9677246E0,5.128941E-3,5.221091E-3,-3.8048907E-4,1.2405388E-2,5.94106E-3],"split_indices":[50,36,61,0,4,2,0,0,0,0,54,37,61,0,56,0,63,62,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,2E1,6.2E1,6E0,1.4E1,5.4E1,8E0,8E0,6E0,5E0,4.9E1,2.2E1,2.7E1,5E0,1.7E1,8E0,1.9E1,1E1,7E0,1E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.6061913E-3,-2.3515114E-1,5.1222954E-2,-4.4558733E-3,-1.3865238E-2,1.0394434E-1,-1.5549183E-1,-4.423737E-3,1.3374738E-1,8.7609695E-4,-1.3879624E-2,1.1859989E-2,9.762506E-2,4.2400673E-2,1.5049386E-1,7.848898E-2,-2.2007455E-3,8.536392E-3,3.510345E-3,6.21084E-3,7.507336E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,-1,-1,7,9,-1,11,-1,-1,-1,13,15,17,19,-1,-1,-1,-1,-1],"loss_changes":[8.8133293E-1,9.8352015E-2,6.9185644E-1,0E0,0E0,3.144222E-1,3.5336274E-1,0E0,1.8031716E-1,0E0,0E0,0E0,9.872627E-2,6.641969E-2,3.6242247E-2,4.91388E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,5,5,6,6,8,8,12,12,13,13,14,14,15,15],"right_children":[2,4,6,-1,-1,8,10,-1,12,-1,-1,-1,14,16,18,20,-1,-1,-1,-1,-1],"split_conditions":[1.528E3,5.295928E-1,1.0232127E12,-4.4558733E-3,-1.3865238E-2,1.885E3,3.5715E4,-4.423737E-3,4.48E2,8.7609695E-4,-1.3879624E-2,1.1859989E-2,1.02239975E2,6.2597164E7,3.003632E0,1.1667828E6,-2.2007455E-3,8.536392E-3,3.510345E-3,6.21084E-3,7.507336E-4],"split_indices":[11,36,40,0,0,2,9,0,0,0,0,0,67,54,62,41,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,1.2E1,6.1E1,5E0,7E0,4.9E1,1.2E1,6E0,4.3E1,6E0,6E0,8E0,3.5E1,1.8E1,1.7E1,1.3E1,5E0,1E1,7E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-3.027757E-3,3.9544903E-2,-1.5031775E-2,-8.399993E-3,2.5041097E-1,-6.234184E-2,1.3709372E-1,1.7274795E-2,5.367683E-3,-6.8443577E-4,-1.3321364E-1,1.0160586E-2,3.4373826E-3,-5.1779564E-2,5.472566E-2,-8.637073E-3,-9.614981E-4,4.8476795E-4,-4.1525364E-3,4.867048E-3,-7.495578E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,15,-1,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1116252E0,7.0980006E-1,0E0,4.6374223E-1,1.814962E-1,1.8780896E-1,6.8853706E-2,0E0,0E0,7.078213E-2,1.2698546E-1,0E0,0E0,3.3870716E-2,4.7497183E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,10,10,13,13,14,14],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,16,-1,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2159766E6,3.3795098E3,-1.5031775E-2,5.517496E-1,9.5E2,5.6660336E-1,7.843361E5,1.7274795E-2,5.367683E-3,2.1933217E0,9.014711E-2,1.0160586E-2,3.4373826E-3,2.5722395E3,1.3941433E-1,-8.637073E-3,-9.614981E-4,4.8476795E-4,-4.1525364E-3,4.867048E-3,-7.495578E-4],"split_indices":[37,4,0,50,0,36,41,0,0,66,48,0,0,56,51,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,6.9E1,8E0,5.7E1,1.2E1,4.2E1,1.5E1,5E0,7E0,2.3E1,1.9E1,5E0,1E1,1.2E1,1.1E1,1.2E1,7E0,5E0,7E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.5195565E-2,2.6267525E-2,-1.331219E-2,-1.7902663E-1,7.3456034E-2,-1.1740517E-2,-1.8129664E-3,4.342946E-2,1.4249467E-2,1.8254703E-1,-1.4143322E-2,3.3829687E-3,1.0300538E-2,-5.4511555E-2,6.457584E-2,2.8084859E-3,-9.9749334E-2,5.5854176E-3,-1.8504514E-4,1.7434803E-3,-1.2776096E-3,-1.4996078E-3,-5.987472E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,-1,5,7,-1,-1,9,-1,11,13,-1,-1,15,17,19,21,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0308064E-1,6.7190236E-1,0E0,1.3687941E-1,3.948306E-1,0E0,0E0,4.1106048E-1,0E0,5.725494E-2,1.2109701E-1,0E0,0E0,6.510593E-2,5.360199E-2,1.4534471E-2,2.647075E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,9,9,10,10,13,13,14,14,15,15,16,16],"right_children":[2,4,-1,6,8,-1,-1,10,-1,12,14,-1,-1,16,18,20,22,-1,-1,-1,-1,-1,-1],"split_conditions":[1.21144904E8,2.3140822E8,-1.331219E-2,1.3440569E1,4.146482E3,-1.1740517E-2,-1.8129664E-3,2.325535E7,1.4249467E-2,2.656056E-1,2.4399805E3,3.3829687E-3,1.0300538E-2,6.98494E7,1.6089E4,1.4322224E-1,1.5368E4,5.5854176E-3,-1.8504514E-4,1.7434803E-3,-1.2776096E-3,-1.4996078E-3,-5.987472E-3],"split_indices":[54,7,0,65,4,0,0,54,0,36,4,0,0,54,9,48,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.7E1,9E0,1.2E1,5.5E1,7E0,5E0,5E1,5E0,1.4E1,3.6E1,5E0,9E0,2.4E1,1.2E1,1.1E1,1.3E1,6E0,6E0,5E0,6E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-5.530305E-3,-2.2566777E-1,8.945189E-2,-9.9669784E-2,-1.822381E-2,4.3524362E-2,1.6709002E-2,-8.822796E-3,-1.0051562E-3,-5.7210033E-3,6.48266E-2,1.4726539E-1,2.788813E-2,2.2471342E-3,8.817511E-3,-2.5996447E-3,4.5238987E-2,-6.548126E-4,6.2203728E-2,5.4307366E-3,3.2767244E-2,2.401785E-3,-4.685247E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,-1,9,-1,-1,-1,-1,11,13,15,-1,-1,-1,17,-1,19,-1,21,-1,-1],"loss_changes":[1.7347358E0,5.235995E-1,7.339922E-1,1.1475372E-1,0E0,1.93547E-1,0E0,0E0,0E0,0E0,1.3597053E-1,5.6400627E-2,5.1635064E-2,0E0,0E0,0E0,2.9301021E-2,0E0,3.3866383E-2,0E0,1.4763255E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,10,10,11,11,12,12,16,16,18,18,20,20],"right_children":[2,4,6,8,-1,10,-1,-1,-1,-1,12,14,16,-1,-1,-1,18,-1,20,-1,22,-1,-1],"split_conditions":[4.856813E0,1.0789844E6,3.4414336E3,5E0,-1.822381E-2,2E-1,1.6709002E-2,-8.822796E-3,-1.0051562E-3,-5.7210033E-3,2.6529046E7,2.4460206E0,6.797331E2,2.2471342E-3,8.817511E-3,-2.5996447E-3,3.7818575E0,-6.548126E-4,1.367E4,5.4307366E-3,1.9607E4,2.401785E-3,-4.685247E-4],"split_indices":[67,37,61,8,0,66,0,0,0,0,54,65,61,0,0,0,65,0,9,0,10,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,2.4E1,5.7E1,1.5E1,9E0,5E1,7E0,6E0,9E0,5E0,4.5E1,1.3E1,3.2E1,5E0,8E0,5E0,2.7E1,6E0,2.1E1,6E0,1.5E1,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.43220415E-2,-2.2727926E-1,8.358173E-2,-8.327193E-2,-2.1114115E-2,4.871615E-2,1.1339624E-2,3.0899132E-3,-7.4786325E-3,-2.8805414E-2,7.8100905E-2,1.5322791E-3,-4.3819123E-3,8.105902E-3,3.9259087E-2,1.4154444E-2,4.8646545E-3,1.8713601E-3,-2.1424617E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,-1,-1,-1,15,17,-1,-1,-1],"loss_changes":[1.457599E0,7.2639525E-1,2.6928723E-1,2.0005864E-1,0E0,9.564575E-2,0E0,0E0,0E0,5.4962933E-2,1.1168136E-1,0E0,0E0,0E0,3.817959E-2,3.2426942E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,14,14,15,15],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,-1,-1,-1,16,18,-1,-1,-1],"split_conditions":[7.940699E-3,9.29295E5,1.1997641E0,2.7307262E5,-2.1114115E-2,2.6277744E8,1.1339624E-2,3.0899132E-3,-7.4786325E-3,1.6362794E-1,2.325535E7,1.5322791E-3,-4.3819123E-3,8.105902E-3,3.3017554E2,4.901466E9,4.8646545E-3,1.8713601E-3,-2.1424617E-3],"split_indices":[47,37,48,37,0,12,0,0,0,51,54,0,0,0,66,12,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,2.1E1,4.7E1,1.4E1,7E0,4E1,7E0,5E0,9E0,1.1E1,2.9E1,6E0,5E0,7E0,2.2E1,1.7E1,5E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[6.9914595E-4,-4.0013198E-2,1.9669525E-2,-1.409182E-1,7.933304E-2,3.0725922E-3,-2.0187436E-1,1.8148908E-1,3.1211056E-2,-1.4749558E-1,-1.8594652E-2,3.4023102E-3,1.1580739E-2,-7.0351446E-3,7.3230706E-2,-5.38843E-2,-1.9939138E-1,1.5310943E-3,-2.3834163E-3,1.4864777E-3,4.3789507E-3,6.704714E-4,-5.118955E-3,-1.074739E-2,-3.944941E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,-1,5,7,-1,9,11,13,15,-1,-1,-1,17,19,21,23,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4556174E0,9.143768E-1,0E0,5.3648674E-1,1.6639383E-1,0E0,3.063619E-1,6.876308E-2,4.0692486E-2,1.2042272E-1,0E0,0E0,0E0,2.816589E-2,8.37972E-3,4.692634E-2,5.37315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16],"right_children":[2,4,-1,6,8,-1,10,12,14,16,-1,-1,-1,18,20,22,24,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.7780703E3,4.283039E1,1.9669525E-2,3.459181E-1,2.156825E7,3.0725922E-3,1.5135763E8,7.285458E-1,4.162943E-1,1.5368E4,-1.8594652E-2,3.4023102E-3,1.1580739E-2,7.844101E0,2.8E1,8.918406E-2,6.817E3,1.5310943E-3,-2.3834163E-3,1.4864777E-3,4.3789507E-3,6.704714E-4,-5.118955E-3,-1.074739E-2,-3.944941E-3],"split_indices":[4,67,0,36,54,0,54,48,48,9,0,0,0,66,8,66,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.4E1,6E0,4E1,3.4E1,9E0,3.1E1,1E1,2.4E1,2.6E1,5E0,5E0,5E0,1.3E1,1.1E1,1E1,1.6E1,7E0,6E0,5E0,6E0,5E0,5E0,1.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-3.8960754E-4,-2.6627053E-2,1.3769423E-2,5.7133168E-2,-9.866548E-2,-2.1787107E-2,1.1643164E-1,-1.8714808E-1,-9.6737305E-3,-3.7033658E-3,3.719557E-3,8.794872E-3,6.500319E-2,-2.2868863E-1,-2.6581064E-3,6.664942E-2,-8.955216E-3,-7.234574E-5,4.8108557E-3,-1.3014142E-2,-5.7226936E-3,4.751401E-3,-4.158345E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,-1,5,7,9,11,13,15,-1,-1,-1,17,19,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.106228E-1,4.280139E-1,0E0,1.5697181E-1,2.9741E-1,1.01452745E-1,6.437814E-2,8.762151E-2,3.0205458E-1,0E0,0E0,0E0,3.639505E-2,4.9071193E-2,0E0,4.5213245E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,-1,-1,-1,18,20,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0235266E3,5.460315E9,1.3769423E-2,7.767353E2,2.667437E6,3.4919355E0,8.4087044E5,1E0,2.1446E4,-3.7033658E-3,3.719557E-3,8.794872E-3,5.03E2,1.2439503E3,-2.6581064E-3,3.4478765E0,-8.955216E-3,-7.234574E-5,4.8108557E-3,-1.3014142E-2,-5.7226936E-3,4.751401E-3,-4.158345E-4],"split_indices":[61,5,0,4,1,62,41,22,9,0,0,0,0,4,0,62,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,3.2E1,3.7E1,1.4E1,1.8E1,1.8E1,1.9E1,9E0,5E0,6E0,1.2E1,1.3E1,5E0,1.4E1,5E0,5E0,7E0,7E0,6E0,9E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.7285367E-2,-9.29543E-2,1.6482763E-1,-3.5690673E-2,-1.869116E-2,9.7757205E-2,1.7065296E-2,-1.5202647E-1,6.7410856E-3,1.5750287E-1,3.547744E-2,-9.148189E-3,-3.40254E-3,7.633847E-2,-8.322277E-2,9.893674E-3,4.436929E-3,-2.8775858E-3,4.12291E-3,-3.0092782E-4,1.1840597E-1,-5.261988E-3,-1.481452E-3,6.633099E-3,2.9251813E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,-1,-1,19,21,-1,-1,-1,-1,-1,23,-1,-1,-1,-1],"loss_changes":[1.3327872E0,8.5214174E-1,4.6564597E-1,2.0341444E-1,0E0,9.859869E-2,0E0,2.7654767E-2,2.0045182E-1,2.9709846E-2,8.945009E-2,0E0,0E0,6.3656166E-2,1.9849703E-2,0E0,0E0,0E0,0E0,0E0,8.963808E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,13,13,14,14,20,20],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,-1,-1,20,22,-1,-1,-1,-1,-1,24,-1,-1,-1,-1],"split_conditions":[1.0781599E3,1.052686E6,6.7780703E3,3.731E3,-1.869116E-2,3.2449896E0,1.7065296E-2,2E0,5.7229916E7,3.1453275E6,1.2106538E1,-9.148189E-3,-3.40254E-3,2.727E3,7.4374756E2,9.893674E-3,4.436929E-3,-2.8775858E-3,4.12291E-3,-3.0092782E-4,5.777288E0,-5.261988E-3,-1.481452E-3,6.633099E-3,2.9251813E-3],"split_indices":[61,37,4,11,0,63,0,8,54,56,65,0,0,2,61,0,0,0,0,0,65,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,4.6E1,3.4E1,4E1,6E0,2.7E1,7E0,1E1,3E1,1.3E1,1.4E1,5E0,5E0,1.7E1,1.3E1,5E0,8E0,5E0,9E0,6E0,1.1E1,7E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-2.5164437E-2,3.8037956E-2,-1.0435674E-1,1.6834517E-1,-1.2015933E-2,-1.7837843E-1,-9.933682E-3,9.838598E-3,3.6830225E-3,3.5774168E-2,-7.198511E-3,-2.3790833E-3,-2.3225866E-1,-2.675151E-3,1.5826761E-3,-3.9270874E-3,8.8445984E-2,-1.4047113E-2,-3.9311615E-3,8.652438E-3,3.3781134E-2,-1.7535065E-3,3.7870582E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,-1,-1,15,-1,-1,17,-1,-1,-1,19,-1,-1,-1,21,-1,-1],"loss_changes":[3.843604E-1,2.8248528E-1,2.3586613E-1,3.4210682E-2,2.318112E-1,1.16253555E-1,3.777257E-2,0E0,0E0,1.6965723E-1,0E0,0E0,1.2293464E-1,0E0,0E0,0E0,9.554604E-2,0E0,0E0,0E0,5.0844807E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,12,12,16,16,20,20],"right_children":[2,4,6,8,10,12,14,-1,-1,16,-1,-1,18,-1,-1,-1,20,-1,-1,-1,22,-1,-1],"split_conditions":[6.7376137E-1,1.5199E4,9.014711E-2,3.5175372E5,1.6107566E5,1.0671435E5,2.9097745E0,9.838598E-3,3.6830225E-3,7.348946E4,-7.198511E-3,-2.3790833E-3,3.9497445E0,-2.675151E-3,1.5826761E-3,-3.9270874E-3,8.4087044E5,-1.4047113E-2,-3.9311615E-3,8.652438E-3,6.797331E2,-1.7535065E-3,3.7870582E-3],"split_indices":[36,9,48,41,42,41,62,0,0,42,0,0,63,0,0,0,41,0,0,0,61,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,4.2E1,3.3E1,1.1E1,3.1E1,1.8E1,1.5E1,6E0,5E0,2.4E1,7E0,6E0,1.2E1,7E0,8E0,7E0,1.7E1,7E0,5E0,5E0,1.2E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-3.4873E-2,-3.8185199E-3,-1.7623339E-2,-9.3067415E-2,4.7832362E-2,-1.327999E-1,2.7697228E-3,6.5124795E-2,-1.8140936E-3,-7.181974E-3,-1.7783014E-3,1.23317786E-1,3.9937384E-2,3.0782663E-3,7.1023824E-3,-4.896574E-3,6.923562E-2,1.116422E-3,-1.7434844E-3,1.3684198E-3,4.9146423E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,-1,13,15,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[8.2880163E-1,3.2713652E-1,0E0,1.6785094E-1,7.119355E-2,4.600203E-2,0E0,5.0772786E-2,0E0,0E0,0E0,1.0017723E-2,3.7118588E-2,0E0,0E0,1.30093265E-2,2.3036145E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,11,11,12,12,15,15,16,16],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,-1,14,16,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[1.5414109E8,3.1457312E8,-1.7623339E-2,1.0740735E3,2.5922572E7,4.593E3,2.7697228E-3,2.6529046E7,-1.8140936E-3,-7.181974E-3,-1.7783014E-3,4.4740204E2,2.585186E6,3.0782663E-3,7.1023824E-3,1.329146E5,7.7572956E9,1.116422E-3,-1.7434844E-3,1.3684198E-3,4.9146423E-3],"split_indices":[54,7,0,61,41,2,0,54,0,0,0,61,56,0,0,42,5,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,2.5E1,4.4E1,2E1,5E0,3.7E1,7E0,1.5E1,5E0,1E1,2.7E1,5E0,5E0,1.1E1,1.6E1,6E0,5E0,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[5.2901726E-2,1.3353078E-2,1.2829389E-2,1.3535446E-1,-2.3271868E-2,1.8205917E-3,1.0109524E-2,-1.5937263E-1,2.7944755E-2,-4.1004987E-3,-1.0082087E-2,4.517731E-3,-9.410064E-3,-3.67594E-3,3.936519E-2,3.738484E-3,-1.0081073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,-1,5,7,-1,-1,9,11,-1,-1,-1,13,-1,15,-1,-1],"loss_changes":[6.292951E-1,2.7124757E-1,0E0,1.09396756E-1,3.3207285E-1,0E0,0E0,3.691101E-2,9.557623E-2,0E0,0E0,0E0,8.7756835E-2,0E0,4.2296194E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,12,12,14,14],"right_children":[2,4,-1,6,8,-1,-1,10,12,-1,-1,-1,14,-1,16,-1,-1],"split_conditions":[6.5317163E3,1.7329262E7,1.2829389E-2,4.4740204E2,4.492E3,1.8205917E-3,1.0109524E-2,8.994286E7,5.7229916E7,-4.1004987E-3,-1.0082087E-2,4.517731E-3,6.209776E11,-3.67594E-3,3.45E2,3.738484E-3,-1.0081073E-3],"split_indices":[4,54,0,61,2,0,0,54,54,0,0,0,40,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[6.8E1,5.9E1,9E0,1.3E1,4.6E1,7E0,6E0,1.2E1,3.4E1,7E0,5E0,1.1E1,2.3E1,9E0,1.4E1,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[8.702452E-3,-2.951466E-2,1.4689516E-2,8.899148E-3,-1.2612832E-2,-8.273712E-2,6.67929E-2,-1.1704529E-1,2.3024632E-3,1.6634221E-1,2.9314337E-2,-5.7190154E-2,-7.977943E-3,4.462866E-3,9.269715E-3,4.6634395E-2,-2.279676E-3,-7.542846E-3,2.050479E-3,1.3524514E-2,3.5440899E-3,2.4667475E-3,-1.0926232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,-1,5,-1,7,9,11,-1,13,15,17,-1,-1,-1,19,-1,-1,-1,21,-1,-1,-1],"loss_changes":[1.0167731E0,7.244215E-1,0E0,3.556673E-1,0E0,1.2539667E-1,1.4809336E-1,6.543815E-2,0E0,8.820295E-3,4.5282036E-2,1.4361252E-1,0E0,0E0,0E0,2.6542839E-2,0E0,0E0,0E0,2.3154551E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,9,9,10,10,11,11,15,15,19,19],"right_children":[2,4,-1,6,-1,8,10,12,-1,14,16,18,-1,-1,-1,20,-1,-1,-1,22,-1,-1,-1],"split_conditions":[4.0235266E3,1.4899646E8,1.4689516E-2,5.666351E7,-1.2612832E-2,1.7058623E3,1.7329262E7,5.7269626E0,2.3024632E-3,1.2566517E5,2.5922572E7,1.02E3,-7.977943E-3,4.462866E-3,9.269715E-3,1.02239975E2,-2.279676E-3,-7.542846E-3,2.050479E-3,5.484294E-1,3.5440899E-3,2.4667475E-3,-1.0926232E-3],"split_indices":[61,54,0,12,0,4,54,65,0,42,41,2,0,0,0,67,0,0,0,36,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,6.5E1,9E0,2.5E1,4E1,2E1,5E0,1E1,3E1,1.1E1,9E0,5E0,5E0,2.5E1,5E0,5E0,6E0,1.3E1,1.2E1,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.4582858E-2,-2.3801463E-2,1.4411482E-2,-1.5364677E-1,4.028198E-2,-6.506799E-2,-1.45300785E-2,1.0778716E-2,1.3035215E-1,2.0550864E-3,-1.149984E-1,3.963271E-2,-3.899606E-3,9.020117E-3,1.2162166E-3,-2.363542E-3,-6.8514557E-3,5.5652442E-3,1.3494419E-2,4.4794906E-2,-3.1035275E-3,-1.6568568E-4,3.7799764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,15,17,-1,-1,-1,-1,-1,-1,19,21,-1,-1,-1],"loss_changes":[9.455898E-1,6.0529643E-1,0E0,3.383311E-1,1.2880582E-1,9.921693E-2,0E0,1.1020203E-1,8.1346855E-2,0E0,1.9883275E-2,6.416561E-2,0E0,0E0,0E0,0E0,0E0,0E0,6.49982E-2,3.446641E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,10,10,11,11,18,18,19,19],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,16,18,-1,-1,-1,-1,-1,-1,20,22,-1,-1,-1],"split_conditions":[4.9891987E3,7.940699E-3,1.4411482E-2,1.0370839E6,1.0379711E0,4.4092423E-1,-1.45300785E-2,2.0264975E5,1.9188015E9,2.0550864E-3,8.857896E4,1.3645554E7,-3.899606E-3,9.020117E-3,1.2162166E-3,-2.363542E-3,-6.8514557E-3,5.5652442E-3,5.6064875E5,2.8496484E5,-3.1035275E-3,-1.6568568E-4,3.7799764E-3],"split_indices":[4,47,0,37,51,36,0,42,12,0,41,54,0,0,0,0,0,0,37,37,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.1E1,8E0,2.3E1,4.8E1,1.6E1,7E0,3.7E1,1.1E1,5E0,1.1E1,2.9E1,8E0,6E0,5E0,5E0,6E0,6E0,2.3E1,1.7E1,6E0,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.5988005E-2,-1.4948761E-2,1.5344188E-2,-4.8544694E-2,1.0055199E-1,-2.6615579E-2,-8.20795E-3,1.5320025E-1,1.5555519E-4,-6.861794E-2,5.4200858E-2,3.2856262E-3,9.362006E-3,-1.2802593E-2,-1.3846387E-1,3.9370223E-3,-1.2081146E-3,-5.251479E-2,3.990242E-3,-9.752195E-3,-3.5013612E-3,1.6889483E-4,-4.602849E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,-1,5,7,9,-1,11,-1,13,15,-1,-1,17,19,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[8.088602E-1,2.9202756E-1,0E0,1.6523835E-1,8.637507E-2,1.7683768E-1,0E0,3.3110976E-2,0E0,1.3077547E-1,5.081114E-2,0E0,0E0,8.520109E-2,5.428478E-2,0E0,0E0,4.233021E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,9,9,10,10,13,13,14,14,17,17],"right_children":[2,4,-1,6,8,10,-1,12,-1,14,16,-1,-1,18,20,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[4.0235266E3,8.363552E-1,1.5344188E-2,1.0743855E-1,3.73524E5,1.517E3,-8.20795E-3,1.667541E0,1.5555519E-4,2.8225484E11,3.45E2,3.2856262E-3,9.362006E-3,4.0614333E0,9.29342E5,3.9370223E-3,-1.2081146E-3,1.4075E4,3.990242E-3,-9.752195E-3,-3.5013612E-3,1.6889483E-4,-4.602849E-3],"split_indices":[61,50,0,47,38,0,0,66,0,40,8,0,0,63,1,0,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.3E1,6E0,5.7E1,1.6E1,5E1,7E0,1E1,6E0,3.3E1,1.7E1,5E0,5E0,1.9E1,1.4E1,1.2E1,5E0,1.4E1,5E0,5E0,9E0,7E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-4.3025403E-3,3.0643793E-2,-1.2236917E-2,9.063747E-3,9.928654E-3,8.189314E-2,-2.4246516E-2,-2.4695515E-3,1.3112748E-1,-4.8654727E-3,3.8952573E-4,3.538961E-3,8.863622E-3,5.498268E-3,-2.3965664E-2,-6.0861737E-2,2.8964717E-2,-4.8961304E-4,-3.8253583E-3,2.9127416E-3,-1.0576721E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,13,-1,-1,-1,15,17,19,-1,-1,-1,-1],"loss_changes":[7.2798336E-1,2.806818E-1,0E0,1.5482149E-1,0E0,1.418321E-1,9.088172E-2,0E0,3.631249E-2,0E0,1.06676556E-1,0E0,0E0,0E0,6.0350537E-2,2.075103E-2,2.5875326E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,10,10,14,14,15,15,16,16],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,14,-1,-1,-1,16,18,20,-1,-1,-1,-1],"split_conditions":[1.5414109E8,4.9891987E3,-1.2236917E-2,2.5511668E7,9.928654E-3,1.51E3,4.593E3,-2.4695515E-3,4.189857E8,-4.8654727E-3,2.217E3,3.538961E-3,8.863622E-3,5.498268E-3,1.775E3,3.3E1,3.45E2,-4.8961304E-4,-3.8253583E-3,2.9127416E-3,-1.0576721E-3],"split_indices":[54,4,0,54,0,2,2,0,7,0,10,0,0,0,0,8,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.8E1,8E0,6.2E1,6E0,1.9E1,4.3E1,5E0,1.4E1,9E0,3.4E1,9E0,5E0,5E0,2.9E1,1.7E1,1.2E1,6E0,1.1E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.2702371E-2,2.5087321E-2,-1.8184868E-1,-6.325701E-3,4.8291154E-2,-3.3589893E-3,-1.2538686E-2,2.9009214E-2,8.065917E-3,1.07342616E-1,-7.629176E-3,2.792831E-4,7.5384355E-3,-4.7947136E-3,2.0885428E-2,4.072787E-3,-1.1460536E-2,-2.5753845E-3,3.115521E-2,-2.8895284E-4,2.861009E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,-1,7,-1,-1,9,-1,11,13,-1,-1,-1,15,-1,17,-1,19,-1,-1],"loss_changes":[4.9035314E-1,2.4881022E-1,1.2428033E-1,0E0,1.3607197E-1,0E0,0E0,1.4431943E-1,0E0,9.616043E-2,1.0087469E-1,0E0,0E0,0E0,6.403552E-2,0E0,4.0759265E-2,0E0,1.3789874E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,7,7,9,9,10,10,14,14,16,16,18,18],"right_children":[2,4,6,-1,8,-1,-1,10,-1,12,14,-1,-1,-1,16,-1,18,-1,20,-1,-1],"split_conditions":[1.0129378E12,8.857896E4,1.5E1,-6.325701E-3,6.5317163E3,-3.3589893E-3,-1.2538686E-2,1.6939252E7,8.065917E-3,4.4939005E2,4.83604E5,2.792831E-4,7.5384355E-3,-4.7947136E-3,5.090285E7,4.072787E-3,8.933723E7,-2.5753845E-3,2.6597537E3,-2.8895284E-4,2.861009E-3],"split_indices":[40,41,3,0,4,0,0,54,0,4,1,0,0,0,54,0,54,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.2E1,1.3E1,7E0,5.5E1,7E0,6E0,4.9E1,6E0,1.5E1,3.4E1,6E0,9E0,7E0,2.7E1,8E0,1.9E1,9E0,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.9604716E-4,-3.8168687E-2,1.5337205E-2,-1.0266078E-2,-1.3319967E-2,-8.398671E-2,3.673398E-2,-1.1924455E-1,2.4052472E-3,-1.9370059E-3,5.1634286E-2,-1.7995365E-3,-7.897292E-3,-1.7279105E-3,7.157942E-2,5.333138E-3,3.1539273E-2,1.6918554E-4,2.513479E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,-1,5,-1,7,9,11,-1,-1,13,-1,-1,-1,15,-1,17,-1,-1],"loss_changes":[1.0329244E0,5.145371E-1,0E0,2.3176685E-1,0E0,1.3232465E-1,5.0940886E-2,8.644596E-2,0E0,0E0,6.5578096E-2,0E0,0E0,0E0,5.0669968E-2,0E0,1.1256605E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,10,10,14,14,16,16],"right_children":[2,4,-1,6,-1,8,10,12,-1,-1,14,-1,-1,-1,16,-1,18,-1,-1],"split_conditions":[4.0235266E3,1.5135763E8,1.5337205E-2,5.666351E7,-1.3319967E-2,1.9315491E3,3.8045087E2,1E0,2.4052472E-3,-1.9370059E-3,2.5332227E0,-1.7995365E-3,-7.897292E-3,-1.7279105E-3,4.8178736E7,5.333138E-3,1.6547014E5,1.6918554E-4,2.513479E-3],"split_indices":[61,54,0,12,0,4,61,25,0,0,62,0,0,0,54,0,42,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.1E1,7E0,6.5E1,6E0,2.5E1,4E1,2E1,5E0,6E0,3.4E1,9E0,1.1E1,6E0,2.8E1,1.2E1,1.6E1,8E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-2.442632E-2,4.004274E-3,-1.3928176E-2,-1.02483146E-1,3.942593E-2,-3.6988918E-2,-8.093239E-3,-4.764864E-3,5.640577E-2,-4.507981E-3,1.4543162E-3,1.5190266E-1,2.132833E-2,8.671714E-3,3.2583138E-3,5.692275E-3,9.592186E-4,-1.6865738E-2,3.648008E-3,2.8162733E-2,-4.958298E-2,-5.718773E-4,3.2035627E-3,-3.970659E-3,-7.929457E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,-1,5,7,9,-1,-1,11,-1,-1,13,15,-1,-1,-1,17,19,-1,21,23,-1,-1,-1,-1],"loss_changes":[6.2488085E-1,2.718497E-1,0E0,8.441496E-2,1.3914049E-1,5.133932E-2,0E0,0E0,1.6089971E-1,0E0,0E0,2.8122962E-2,7.9073444E-2,0E0,0E0,0E0,4.7028117E-2,4.1172992E-2,0E0,2.197826E-2,1.8191248E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,11,11,12,12,16,16,17,17,19,19,20,20],"right_children":[2,4,-1,6,8,10,-1,-1,12,-1,-1,14,16,-1,-1,-1,18,20,-1,22,24,-1,-1,-1,-1],"split_conditions":[1.5414109E8,1.528E3,-1.3928176E-2,2.62518E5,2.4742014E0,1.02E3,-8.093239E-3,-4.764864E-3,1.6939252E7,-4.507981E-3,1.4543162E-3,1.5558655E5,3.0014542E7,8.671714E-3,3.2583138E-3,5.692275E-3,5.2518907E0,3.309969E0,3.648008E-3,1.859E3,3.5519714E0,-5.718773E-4,3.2035627E-3,-3.970659E-3,-7.929457E-4],"split_indices":[54,11,0,38,62,2,0,0,54,0,0,42,12,0,0,0,63,63,0,0,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7E1,6E0,1.7E1,5.3E1,1E1,7E0,5E0,4.8E1,5E0,5E0,1.2E1,3.6E1,7E0,5E0,5E0,3.1E1,2.6E1,5E0,1.1E1,1.5E1,6E0,5E0,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-1.7869341E-3,5.008105E-2,-1.7169937E-1,1.2789708E-2,1.28884325E-2,-6.997437E-2,-1.6368058E-2,-6.821575E-2,4.7473427E-2,-7.988277E-3,8.708462E-4,-1.4273654E-1,3.5923205E-3,8.8067435E-2,-5.46467E-3,-8.341768E-3,-3.4420192E-3,1.1111769E-1,1.6535737E-3,-4.8290476E-2,3.6518837E-3,2.5615254E-3,6.0758535E-3,-7.5977856E-5,-3.6626793E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,-1,11,13,-1,-1,15,-1,17,19,-1,-1,21,-1,23,-1,-1,-1,-1,-1],"loss_changes":[6.7838806E-1,5.158779E-1,3.2573795E-1,1.4933611E-1,0E0,1.2815554E-1,0E0,1.8784103E-1,7.9993136E-2,0E0,0E0,1.684241E-2,0E0,2.0774126E-2,6.6937014E-2,0E0,0E0,1.040256E-2,0E0,1.8343916E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,11,11,13,13,14,14,17,17,19,19],"right_children":[2,4,6,8,-1,10,-1,12,14,-1,-1,16,-1,18,20,-1,-1,22,-1,24,-1,-1,-1,-1,-1],"split_conditions":[1.0232127E12,6.7780703E3,1.0302285E8,5.02409E5,1.28884325E-2,4.375793E6,-1.6368058E-2,5.596155E-1,5.7229916E7,-7.988277E-3,8.708462E-4,1.1017415E1,3.5923205E-3,3.6726675E6,7.7572956E9,-8.341768E-3,-3.4420192E-3,5.76824E2,1.6535737E-3,2.4973628E5,3.6518837E-3,2.5615254E-3,6.0758535E-3,-7.5977856E-5,-3.6626793E-3],"split_indices":[40,4,54,1,0,1,0,48,54,0,0,65,0,56,5,0,0,61,0,37,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,5.8E1,1.7E1,5.1E1,7E0,1.2E1,5E0,1.5E1,3.6E1,5E0,7E0,1E1,5E0,2E1,1.6E1,5E0,5E0,1.3E1,7E0,1.1E1,5E0,5E0,8E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.9604618E-2,-5.389469E-2,1.3655002E-1,-1.3469879E-2,-1.3002347E-2,1.3167253E-2,7.640215E-2,-4.7439155E-3,1.4780329E-2,9.188238E-3,2.961029E-2,6.55919E-2,-3.6900673E-2,4.087215E-3,-8.9275034E-4,5.072245E-3,9.6340285E-4,-3.6173211E-3,4.7862693E-4,1.3978237E-3,-1.9508674E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,-1,-1,9,-1,11,-1,13,15,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[6.6920567E-1,4.5181078E-1,2.5889242E-1,1.1057195E-1,0E0,0E0,1.3132802E-1,0E0,8.907816E-2,0E0,3.3658404E-2,3.293997E-2,3.5945E-2,0E0,1.8959627E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,10,10,11,11,12,12,14,14],"right_children":[2,4,6,8,-1,-1,10,-1,12,-1,14,16,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1813418E3,1.3237324E6,5.99999E9,2.97E2,-1.3002347E-2,1.3167253E-2,1.4302E4,-4.7439155E-3,5.7229916E7,9.188238E-3,2.9685414E0,3.5175372E5,1.7742582E3,4.087215E-3,5.902777E-1,5.072245E-3,9.6340285E-4,-3.6173211E-3,4.7862693E-4,1.3978237E-3,-1.9508674E-3],"split_indices":[61,37,5,0,0,0,10,0,54,0,62,41,4,0,36,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,4.7E1,2.9E1,4.1E1,6E0,7E0,2.2E1,9E0,3.2E1,5E0,1.7E1,1.6E1,1.6E1,5E0,1.2E1,7E0,9E0,8E0,8E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.6994454E-2,3.576227E-2,-7.680229E-2,-4.4892957E-3,5.7961598E-2,-4.0707067E-2,-9.698159E-3,8.255149E-2,-3.1937524E-3,-1.01630464E-1,2.17004E-3,7.640248E-3,3.1575173E-2,-3.048507E-3,2.7848429E-3,-6.3640373E-3,-1.7282635E-3,3.493398E-2,-3.1152284E-3,-5.172559E-4,2.830951E-3,3.4119245E-3,-1.1408493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,-1,7,9,-1,11,13,15,17,-1,19,-1,-1,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[2.3623627E-1,1.2606016E-1,1.6628751E-1,0E0,5.2896775E-2,7.5974904E-2,0E0,1.0566427E-1,5.033384E-2,2.4705768E-2,4.4495594E-2,0E0,2.3629552E-2,0E0,0E0,0E0,0E0,3.391362E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,7,7,8,8,9,9,10,10,12,12,17,17],"right_children":[2,4,6,-1,8,10,-1,12,14,16,18,-1,20,-1,-1,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[7.1974045E-1,3.7605792E2,3E1,-4.4892957E-3,1.76E2,7.673801E2,-9.698159E-3,3.5175372E5,1.60079E5,8.611686E-1,1.0743855E-1,7.640248E-3,4.57563E5,-3.048507E-3,2.7848429E-3,-6.3640373E-3,-1.7282635E-3,1.5368E4,-3.1152284E-3,-5.172559E-4,2.830951E-3,3.4119245E-3,-1.1408493E-3],"split_indices":[36,4,3,0,8,61,0,41,41,36,47,0,1,0,0,0,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,3.9E1,3.4E1,5E0,3.4E1,2.8E1,6E0,2.4E1,1E1,1.1E1,1.7E1,8E0,1.6E1,5E0,5E0,6E0,5E0,1.2E1,5E0,7E0,9E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.2425799E-2,-5.646006E-3,9.125461E-3,-7.9395205E-2,4.8844088E-2,8.780739E-4,-1.0174185E-1,5.3642094E-3,2.292553E-2,-6.521211E-3,-5.7467226E-2,6.9175237E-3,3.253383E-3,1.5877801E-3,-6.3936706E-3,1.7344152E-3,-9.31422E-3,-1.684074E-3,1.0557325E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,-1,5,7,-1,9,-1,11,-1,13,15,-1,-1,-1,-1,17,-1,-1],"loss_changes":[3.755262E-1,2.6508704E-1,0E0,6.3862905E-2,6.6487595E-2,0E0,3.554204E-2,0E0,2.2397384E-2,0E0,1.0688843E-1,1.2155534E-2,0E0,0E0,0E0,0E0,1.5596933E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,8,8,10,10,11,11,16,16],"right_children":[2,4,-1,6,8,-1,10,-1,12,-1,14,16,-1,-1,-1,-1,18,-1,-1],"split_conditions":[6.7780703E3,6.0199605E6,9.125461E-3,4.0246898E-1,1.7329262E7,8.780739E-4,1.648E3,5.3642094E-3,1.0193359E8,-6.521211E-3,6.3380525E5,5.4734116E9,3.253383E-3,1.5877801E-3,-6.3936706E-3,1.7344152E-3,1.2204E4,-1.684074E-3,1.0557325E-3],"split_indices":[4,60,0,36,54,0,11,0,54,0,41,5,0,0,0,0,2,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.3E1,6.4E1,9E0,2.7E1,3.7E1,5E0,2.2E1,9E0,2.8E1,1E1,1.2E1,2.2E1,6E0,6E0,6E0,7E0,1.5E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.5541525E-2,-8.398837E-3,1.245842E-2,1.6102877E-2,-1.4245835E-1,-4.662207E-3,3.6809307E-2,-1.4028689E-3,-1.1193306E-2,1.08761825E-1,1.7238881E-2,5.798299E-4,7.8989165E-3,-3.7699826E-3,3.7506446E-2,8.275608E-2,8.856411E-3,1.5576917E-4,5.4963795E-3,4.4390813E-2,-3.3338428E-2,3.3326978E-3,1.8909767E-4,8.160301E-4,-3.273028E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,-1,5,7,-1,9,-1,-1,11,13,-1,-1,-1,15,17,19,-1,-1,21,23,-1,-1,-1,-1],"loss_changes":[5.2118963E-1,2.5521496E-1,0E0,1.6730979E-1,1.339859E-1,0E0,7.8389116E-2,0E0,0E0,7.4430734E-2,9.726517E-2,0E0,0E0,0E0,4.9826816E-2,4.6326317E-2,3.896413E-2,0E0,0E0,1.6355101E-2,2.561415E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,9,9,10,10,14,14,15,15,16,16,19,19,20,20],"right_children":[2,4,-1,6,8,-1,10,-1,-1,12,14,-1,-1,-1,16,18,20,-1,-1,22,24,-1,-1,-1,-1],"split_conditions":[4.0235266E3,1.0232127E12,1.245842E-2,1.595E3,4.212135E8,-4.662207E-3,1.6939252E7,-1.4028689E-3,-1.1193306E-2,2.6326888E5,5.84957E5,5.798299E-4,7.8989165E-3,-3.7699826E-3,5.999298E5,8.236574E2,6.2597164E7,1.5576917E-4,5.4963795E-3,1.1767926E2,1E0,3.3326978E-3,1.8909767E-4,8.160301E-4,-3.273028E-3],"split_indices":[61,40,0,2,7,0,54,0,0,37,1,0,0,0,41,61,54,0,0,67,25,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.6E1,6E0,6.5E1,1.1E1,9E0,5.6E1,6E0,5E0,1.1E1,4.5E1,5E0,6E0,7E0,3.8E1,1.4E1,2.4E1,5E0,9E0,1.3E1,1.1E1,7E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[-4.6844776E-3,-4.3191206E-2,8.479405E-2,-2.041356E-2,-8.260818E-3,5.966196E-3,1.3239342E-2,-1.045157E-1,9.869946E-3,3.8118146E-2,-2.6717621E-3,-6.259497E-3,-2.1100752E-3,3.8505778E-2,-2.8237207E-3,2.779347E-3,1.9039889E-4,6.2049314E-2,-1.3913995E-3,4.472891E-4,3.842868E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,-1,-1,17,-1,-1,-1,19,-1,-1,-1],"loss_changes":[2.517402E-1,1.5918672E-1,3.6106616E-1,1.1564242E-1,0E0,3.795256E-2,0E0,1.7349184E-2,7.3076434E-2,9.337911E-3,0E0,0E0,0E0,4.2768076E-2,0E0,0E0,0E0,2.2213936E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,13,13,17,17],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,-1,-1,18,-1,-1,-1,20,-1,-1,-1],"split_conditions":[1.8166235E1,1.5092398E8,3.3795098E3,3.6561127E2,-8.260818E-3,3.5228616E-1,1.3239342E-2,1.1017415E1,5.3427195E6,2.8547795E0,-2.6717621E-3,-6.259497E-3,-2.1100752E-3,6.2597164E7,-2.8237207E-3,2.779347E-3,1.9039889E-4,2.6880343E0,-1.3913995E-3,4.472891E-4,3.842868E-3],"split_indices":[65,54,4,61,0,36,0,65,56,66,0,0,0,54,0,0,0,63,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,5E1,2.1E1,4.4E1,6E0,1.6E1,5E0,1.1E1,3.3E1,1.1E1,5E0,6E0,5E0,2.4E1,9E0,6E0,5E0,1.8E1,6E0,6E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-9.763694E-3,-3.0758481E-2,1.019068E-2,-1.2616909E-1,-8.0986175E-4,-6.727565E-3,-2.8167358E-3,2.1460386E-2,-6.621375E-3,5.3737096E-2,-1.3152722E-2,-5.75198E-4,3.6868933E-3,-4.1574605E-2,3.3460052E-3,-2.9214055E-3,-4.982575E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,-1,5,7,-1,-1,9,-1,11,13,-1,-1,15,-1,-1,-1],"loss_changes":[3.6380863E-1,1.9134471E-1,0E0,1.4300764E-2,1.7249955E-1,0E0,0E0,5.2097864E-2,0E0,4.5961365E-2,6.026064E-2,0E0,0E0,1.20786745E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,9,9,10,10,13,13],"right_children":[2,4,-1,6,8,-1,-1,10,-1,12,14,-1,-1,16,-1,-1,-1],"split_conditions":[9.55721E3,3.731E3,1.019068E-2,1.6633E4,1.5277152E8,-6.727565E-3,-2.8167358E-3,6.2597164E7,-6.621375E-3,7.767353E2,7.0826065E1,-5.75198E-4,3.6868933E-3,3.5618556E0,3.3460052E-3,-2.9214055E-3,-4.982575E-4],"split_indices":[4,11,0,9,54,0,0,54,0,4,66,0,0,63,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.6E1,5E0,1.5E1,5.1E1,1E1,5E0,4.5E1,6E0,2.3E1,2.2E1,7E0,1.6E1,1.7E1,5E0,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[8.198452E-3,-1.9082282E-2,1.0565407E-2,9.680336E-3,-1.1761073E-2,-4.1967947E-3,2.3013964E-2,5.884042E-2,-2.7477052E-2,-4.0848987E-4,7.554573E-2,-3.6954305E-3,-3.0465894E-3,8.9961616E-4,9.955238E-2,2.0764335E-3,-3.5564803E-2,6.508307E-3,6.995239E-2,-1.5544855E-4,-3.0216563E-3,1.4630989E-3,4.2322916E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,-1,5,-1,-1,7,9,11,-1,13,-1,15,-1,17,-1,19,-1,21,-1,-1,-1,-1],"loss_changes":[5.171794E-1,5.2477896E-1,0E0,9.542384E-2,0E0,0E0,1.1196053E-1,4.152684E-2,3.4423664E-2,0E0,3.6697105E-2,0E0,3.2012112E-2,0E0,1.651685E-2,0E0,1.1921386E-2,0E0,9.551324E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,6,6,7,7,8,8,10,10,12,12,14,14,16,16,18,18],"right_children":[2,4,-1,6,-1,-1,8,10,12,-1,14,-1,16,-1,18,-1,20,-1,22,-1,-1,-1,-1],"split_conditions":[4.9891987E3,1.15266744E8,1.0565407E-2,2.1113522E8,-1.1761073E-2,-4.1967947E-3,6.0487356E7,2.5630938E5,6.715997E2,-4.0848987E-4,6.8177136E2,-3.6954305E-3,4.7293822E5,8.9961616E-4,2.384E3,2.0764335E-3,4.1124467E9,6.508307E-3,7.8972295E-2,-1.5544855E-4,-3.0216563E-3,1.4630989E-3,4.2322916E-3],"split_indices":[4,54,0,7,0,0,54,37,61,0,4,0,41,0,10,0,12,0,47,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,7.4E1,8E0,6.7E1,7E0,7E0,6E1,3.5E1,2.5E1,7E0,2.8E1,7E0,1.8E1,9E0,1.9E1,7E0,1.1E1,6E0,1.3E1,6E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[7.975514E-3,-2.5729625E-2,1.1618152E-2,-1.3111423E-1,2.1904554E-2,-6.0704943E-2,-1.3375004E-2,8.984497E-2,1.1140367E-3,-1.1204812E-1,1.9607118E-3,1.32384E-3,6.09342E-3,-2.0157745E-2,3.1311016E-3,-6.691609E-3,-2.5586588E-3,-4.6875667E-2,2.000434E-3,-1.8227514E-2,-3.447812E-3,7.0806674E-4,-2.4676719E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,-1,5,7,9,-1,11,13,15,-1,-1,-1,17,-1,-1,-1,19,-1,21,-1,-1,-1],"loss_changes":[6.386852E-1,3.3920592E-1,0E0,2.272768E-1,6.628818E-2,9.0516426E-2,0E0,2.6256748E-2,5.52509E-2,1.3762608E-2,0E0,0E0,0E0,5.2121475E-2,0E0,0E0,0E0,1.6466364E-2,0E0,1.5761945E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,13,13,17,17,19,19],"right_children":[2,4,-1,6,8,10,-1,12,14,16,-1,-1,-1,18,-1,-1,-1,20,-1,22,-1,-1,-1],"split_conditions":[3.3077192E3,8.872871E-3,1.1618152E-2,1.5012118E6,1.6256282E7,3.507E3,-1.3375004E-2,2.91E2,5.2518907E0,1.43E2,1.9607118E-3,1.32384E-3,6.09342E-3,2.916545E3,3.1311016E-3,-6.691609E-3,-2.5586588E-3,5.501383E9,2.000434E-3,3.2825522E5,-3.447812E-3,7.0806674E-4,-2.4676719E-3],"split_indices":[61,50,0,37,54,2,0,0,63,10,0,0,0,4,0,0,0,5,0,37,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.6E1,8E0,2E1,4.6E1,1.5E1,5E0,1E1,3.6E1,1E1,5E0,5E0,5E0,2.8E1,8E0,5E0,5E0,2E1,8E0,1.1E1,9E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.3252992E-2,-1.390529E-3,9.685163E-3,1.8058948E-2,-9.686231E-3,7.2502814E-2,-2.9324906E-2,5.6763296E-3,5.1631205E-2,-5.89335E-2,1.9509507E-3,1.3795048E-2,3.7188686E-3,-2.4212258E-2,-6.7418986E-3,-9.734957E-4,2.378145E-3,-3.1011845E-3,-1.547906E-4,1.4900618E-3,-1.0042173E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,-1,15,-1,17,-1,-1,-1,-1,19,-1,-1],"loss_changes":[3.7375385E-1,2.990029E-1,0E0,1.723903E-1,0E0,2.8140843E-2,7.9991445E-2,0E0,2.6969112E-2,7.8330874E-2,0E0,1.7722327E-2,0E0,2.1475563E-2,0E0,0E0,0E0,0E0,1.1045239E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9,11,11,13,13,18,18],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,-1,16,-1,18,-1,-1,-1,-1,20,-1,-1],"split_conditions":[3.3077192E3,1.4805952E8,9.685163E-3,5.090285E7,-9.686231E-3,3.5175372E5,4.946992E6,5.6763296E-3,9.537158E10,9.29295E5,1.9509507E-3,1E0,3.7188686E-3,1.2E1,-6.7418986E-3,-9.734957E-4,2.378145E-3,-3.1011845E-3,1.7888845E3,1.4900618E-3,-1.0042173E-3],"split_indices":[61,54,0,54,0,41,1,0,40,37,0,18,0,3,0,0,0,0,4,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7E1,8E0,6.5E1,5E0,3E1,3.5E1,7E0,2.3E1,2.5E1,1E1,1.1E1,1.2E1,1.9E1,6E0,6E0,5E0,6E0,1.3E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.6253406E-2,-6.2894626E-3,-9.489507E-3,-9.771626E-2,1.7979681E-2,-2.5176909E-3,-6.490661E-3,3.1953603E-2,-4.359341E-3,1.3691914E-2,6.1673503E-3,-1.7599568E-3,4.106917E-3,1.8345142E-2,-3.1130105E-2,3.641132E-2,-1.6165955E-3,3.9430498E-4,-2.6430301E-3,-4.5835742E-4,2.5519817E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,-1,5,7,-1,-1,9,-1,11,-1,13,-1,15,17,19,-1,-1,-1,-1,-1],"loss_changes":[2.6384827E-1,1.48201E-1,0E0,1.9111231E-2,8.810738E-2,0E0,0E0,9.016836E-2,0E0,5.046335E-2,0E0,2.1828717E-2,0E0,2.2867171E-2,1.7005354E-2,1.6624032E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,9,9,11,11,13,13,14,14,15,15],"right_children":[2,4,-1,6,8,-1,-1,10,-1,12,-1,14,-1,16,18,20,-1,-1,-1,-1,-1],"split_conditions":[1.5277152E8,1.648E3,-9.489507E-3,5.86E2,5.673536E2,-2.5176909E-3,-6.490661E-3,1.892355E3,-4.359341E-3,1.0379711E0,6.1673503E-3,4.3511776E8,4.106917E-3,8.773388E7,7.016421E5,2.4389961E5,-1.6165955E-3,3.9430498E-4,-2.6430301E-3,-4.5835742E-4,2.5519817E-3],"split_indices":[54,11,0,0,67,0,0,61,0,51,0,7,0,54,41,37,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.5E1,6E0,1.3E1,5.2E1,8E0,5E0,4.7E1,5E0,4.1E1,6E0,3.5E1,6E0,2.1E1,1.4E1,1.6E1,5E0,6E0,8E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-7.955743E-3,-1.2821051E-1,2.4961557E-2,-2.1930194E-2,-1.3152986E-2,-3.272714E-2,4.4076484E-2,-3.9202506E-3,2.5985641E-3,1.4635415E-2,-4.6146833E-3,6.3257315E-3,2.2369979E-2,2.1916642E-3,-9.834201E-4,-2.2276116E-3,3.341491E-2,4.7837384E-2,-2.2098382E-3,2.4565991E-2,4.348964E-3,2.1735833E-3,-7.9562527E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,-1,-1,13,-1,-1,15,-1,-1,-1,17,19,-1,21,-1,-1,-1],"loss_changes":[3.3175275E-1,3.0710262E-1,7.422169E-2,6.7261994E-2,0E0,5.7661228E-2,1.01482496E-1,0E0,0E0,1.4718378E-2,0E0,0E0,3.4978077E-2,0E0,0E0,0E0,4.6366293E-2,3.3923745E-2,0E0,2.3877766E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,9,9,12,12,16,16,17,17,19,19],"right_children":[2,4,6,8,-1,10,12,-1,-1,14,-1,-1,16,-1,-1,-1,18,20,-1,22,-1,-1,-1],"split_conditions":[8.872871E-3,9.29295E5,2.4786325E0,2.0672107E8,-1.3152986E-2,1.4818296E-1,4.69E2,-3.9202506E-3,2.5985641E-3,1.4498382E0,-4.6146833E-3,6.3257315E-3,3.1790106E8,2.1916642E-3,-9.834201E-4,-2.2276116E-3,1.1696494E7,1.2207E4,-2.2098382E-3,8.4E1,4.348964E-3,2.1735833E-3,-7.9562527E-4],"split_indices":[50,37,65,7,0,47,0,0,0,65,0,0,7,0,0,0,56,2,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.2E1,1.7E1,6.5E1,1.1E1,6E0,1.6E1,4.9E1,6E0,5E0,1E1,6E0,8E0,4.1E1,5E0,5E0,5E0,3.6E1,3.1E1,5E0,2.2E1,9E0,1.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.219638E-2,8.129322E-3,9.35669E-3,-5.464748E-2,4.6797596E-2,-7.0716483E-3,-7.2413413E-3,9.566415E-2,2.1593137E-2,2.4707739E-3,-4.7775913E-2,1.4281532E-3,6.983464E-3,7.490639E-3,2.9200208E-3,-3.9010877E-3,2.485076E-4,3.1133842E-2,-2.2965032E-3,-2.7338174E-4,2.6020668E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,15,-1,-1,17,-1,-1,-1,19,-1,-1,-1],"loss_changes":[3.1525326E-1,1.651979E-1,0E0,1.3018177E-1,4.9663827E-2,5.0524898E-2,0E0,4.82779E-2,1.720316E-2,0E0,2.5327785E-2,0E0,0E0,3.3393897E-2,0E0,0E0,0E0,1.7206784E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,10,10,13,13,17,17],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,16,-1,-1,18,-1,-1,-1,20,-1,-1,-1],"split_conditions":[4.9891987E3,6.857143E-1,9.35669E-3,8.16747E11,1.7329262E7,2.5036643E8,-7.2413413E-3,4.189857E8,1.8925636E5,2.4707739E-3,7.4374756E2,1.4281532E-3,6.983464E-3,1.219233E6,2.9200208E-3,-3.9010877E-3,2.485076E-4,1.0773855E3,-2.2965032E-3,-2.7338174E-4,2.6020668E-3],"split_indices":[4,66,0,40,54,7,0,7,42,0,61,0,0,41,0,0,0,61,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.6E1,8E0,2.5E1,4.1E1,1.8E1,7E0,1.3E1,2.8E1,7E0,1.1E1,7E0,6E0,2.2E1,6E0,6E0,5E0,1.6E1,6E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.2388344E-2,2.3955772E-2,-1.5244752E-1,-5.6741554E-2,5.325792E-2,-3.4671307E-2,-1.5443834E-2,-1.06331706E-1,2.3368963E-3,2.8265541E-2,1.3020954E-1,-5.1073865E-3,2.245055E-3,-6.8175406E-3,-2.3647137E-3,8.592786E-2,2.7828116E-3,8.096554E-3,2.6530498E-3,5.766885E-3,1.3269928E-3,-1.8550579E-2,3.4667244E-3,-4.4027954E-2,1.7424109E-3,-3.609005E-4,-2.7573714E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,-1,19,21,-1,-1,-1,-1,23,-1,25,-1,-1,-1],"loss_changes":[3.9560717E-1,1.4962746E-1,3.4711942E-1,9.710287E-2,8.478692E-2,7.887558E-2,0E0,2.115412E-2,0E0,5.2659035E-2,2.8297484E-2,0E0,0E0,0E0,0E0,2.2395857E-2,4.258621E-2,0E0,0E0,0E0,0E0,3.2330822E-2,0E0,8.803539E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,15,15,16,16,21,21,23,23],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,-1,20,22,-1,-1,-1,-1,24,-1,26,-1,-1,-1],"split_conditions":[1.0232127E12,3.3252203E0,3.5715E4,1E0,1.7916888E3,7.575E3,-1.5443834E-2,2.045738E0,2.3368963E-3,1.6939252E7,2.0276645E5,-5.1073865E-3,2.245055E-3,-6.8175406E-3,-2.3647137E-3,5.7269626E0,3.3487454E11,8.096554E-3,2.6530498E-3,5.766885E-3,1.3269928E-3,4.25247E0,3.4667244E-3,6.1702337E0,1.7424109E-3,-3.609005E-4,-2.7573714E-3],"split_indices":[40,65,9,24,61,2,0,65,0,54,42,0,0,0,0,65,40,0,0,0,0,62,0,65,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,6.1E1,1.5E1,1.6E1,4.5E1,1E1,5E0,1.1E1,5E0,3.5E1,1E1,5E0,5E0,5E0,6E0,1E1,2.5E1,5E0,5E0,5E0,5E0,2E1,5E0,1.4E1,6E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-6.7669246E-3,9.232686E-3,-8.179135E-3,-3.1018687E-2,9.1145374E-2,-8.3016075E-2,2.7115675E-2,7.4187904E-3,5.9805855E-2,1.9738398E-4,-1.1239272E-1,4.288228E-3,1.4806085E-3,3.5942085E-3,8.9233654E-4,-7.915764E-3,-2.5249105E-3,-1.4773344E-3,1.2952111E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,-1,5,7,9,11,-1,13,-1,15,-1,17,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0032208E-1,2.2043568E-1,0E0,1.382786E-1,4.0894985E-2,6.211269E-2,3.827302E-2,0E0,1.2022339E-2,0E0,5.4411247E-2,0E0,1.6846314E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,10,10,12,12],"right_children":[2,4,-1,6,8,10,12,-1,14,-1,16,-1,18,-1,-1,-1,-1,-1,-1],"split_conditions":[1.06102504E8,1.0781599E3,-8.179135E-3,1.4255319E0,5.23E3,4.0246898E-1,9.9948E4,7.4187904E-3,1.55E2,1.9738398E-4,3.2716873E0,4.288228E-3,3.3392856E0,3.5942085E-3,8.9233654E-4,-7.915764E-3,-2.5249105E-3,-1.4773344E-3,1.2952111E-3],"split_indices":[60,61,0,66,10,36,38,0,8,0,65,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7E1,6.5E1,5E0,4.4E1,2.1E1,2.3E1,2.1E1,5E0,1.6E1,6E0,1.7E1,5E0,1.6E1,1E1,6E0,7E0,1E1,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-2.869462E-4,2.3925075E-2,-1.2730738E-2,5.4812174E-2,-2.7295575E-2,2.7661446E-2,8.752678E-3,-5.4652423E-2,3.492674E-3,-1.3113393E-3,4.6287768E-2,-2.5806595E-2,-5.7312106E-3,7.445018E-2,-6.564371E-3,-6.2475026E-2,1.598673E-3,4.6944707E-3,2.1785416E-3,-2.2823955E-3,1.740466E-3,-3.6573605E-4,-4.7919652E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,-1,5,7,9,-1,11,-1,-1,13,15,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1962125E-1,1.12240136E-1,0E0,1.613455E-1,8.169053E-2,4.1538604E-2,0E0,4.2801455E-2,0E0,0E0,4.365358E-2,4.043544E-2,0E0,9.71809E-3,2.3903295E-2,2.5407076E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,10,10,11,11,13,13,14,14,15,15],"right_children":[2,4,-1,6,8,10,-1,12,-1,-1,14,16,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.8841052E10,7.455661E-1,-1.2730738E-2,1.7916888E3,6.3E1,2.2389898E5,8.752678E-3,1.0167108E8,3.492674E-3,-1.3113393E-3,4.800481E5,6.7499106E5,-5.7312106E-3,2.156825E7,8.1637933E2,1.2058E4,1.598673E-3,4.6944707E-3,2.1785416E-3,-2.2823955E-3,1.740466E-3,-3.6573605E-4,-4.7919652E-3],"split_indices":[5,36,0,61,8,37,0,54,0,0,37,37,0,54,61,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,4.3E1,2.6E1,3.7E1,6E0,2.1E1,5E0,9E0,2.8E1,1.6E1,5E0,1.8E1,1E1,1E1,6E0,7E0,1.1E1,5E0,5E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-9.196346E-3,8.908472E-3,-2.8655764E-2,-1.0107999E-2,-1.0680331E-2,-5.395176E-2,5.2226562E-2,-2.1604598E-2,-1.0764253E-2,3.1009655E-2,5.1550334E-3,-8.2257405E-2,2.5635859E-2,2.6999088E-3,-6.020048E-3,-1.8613402E-3,-4.930551E-3,3.0163364E-3,-1.1953647E-3,-1.5782147E-3,1.0812224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,-1,3,5,-1,7,9,11,-1,13,-1,15,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2795995E-1,0E0,2.8380808E-1,1.9139345E-1,0E0,2.4030952E-1,3.4752958E-2,1.0525833E-1,0E0,2.5015019E-2,0E0,1.330699E-2,4.7087915E-2,0E0,1.0430452E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,9,9,11,11,12,12,14,14],"right_children":[2,-1,4,6,-1,8,10,12,-1,14,-1,16,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,8.908472E-3,6.624127E2,1.7742582E3,-1.0680331E-2,9.742515E7,1.1686677E8,6.942296E2,-1.0764253E-2,1.258E3,5.1550334E-3,1.5176E4,1.3097832E3,2.6999088E-3,1.868E3,-1.8613402E-3,-4.930551E-3,3.0163364E-3,-1.1953647E-3,-1.5782147E-3,1.0812224E-3],"split_indices":[13,0,67,4,0,54,54,4,0,0,0,9,4,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,6E0,7.3E1,6.8E1,5E0,4E1,2.8E1,3.5E1,5E0,2.2E1,6E0,1.5E1,2E1,1.2E1,1E1,7E0,8E0,1.1E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.2892136E-2,8.359579E-3,-1.5519291E-1,-5.1887553E-2,3.54152E-2,-9.386763E-4,-1.2904105E-2,-5.0129606E-3,5.735525E-4,1.1041208E-1,5.2417186E-3,1.9634007E-3,7.653223E-3,-3.265909E-2,6.797292E-2,-2.675939E-3,-4.600953E-3,9.979213E-4,4.3918365E-3,-2.1750622E-3,1.4908828E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,-1,-1,-1,11,13,-1,-1,15,17,19,-1,-1,-1,-1,-1],"loss_changes":[3.0312505E-1,9.963901E-2,2.4111554E-1,7.439698E-2,9.446376E-2,0E0,0E0,0E0,0E0,4.032758E-2,7.5924404E-2,0E0,0E0,4.141198E-2,1.4087435E-2,2.6089406E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,9,9,10,10,13,13,14,14,15,15],"right_children":[2,4,6,8,10,-1,-1,-1,-1,12,14,-1,-1,16,18,20,-1,-1,-1,-1,-1],"split_conditions":[1.0232127E12,3.8734267E0,1.5E1,1.9377503E-1,3.0014542E7,-9.386763E-4,-1.2904105E-2,-5.0129606E-3,5.735525E-4,8.236574E2,4.152296E1,1.9634007E-3,7.653223E-3,3.652811E5,4.650193E-1,1.2930398E-1,-4.600953E-3,9.979213E-4,4.3918365E-3,-2.1750622E-3,1.4908828E-3],"split_indices":[40,65,3,48,12,0,0,0,0,61,66,0,0,37,36,50,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,5.9E1,1.3E1,1.8E1,4.1E1,7E0,6E0,9E0,9E0,1.1E1,3E1,6E0,5E0,1.9E1,1.1E1,1.4E1,5E0,5E0,6E0,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[6.7100755E-4,1.816163E-2,-7.512694E-3,-3.2253286E-3,1.0633236E-2,-7.768957E-2,2.0111943E-2,-5.5766973E-4,-4.913017E-3,3.68287E-3,4.093079E-3,-2.768238E-2,3.3379164E-2,-1.4520276E-4,-2.5765202E-3,3.4248976E-3,2.9754406E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,-1,13,15,-1,-1,-1,-1],"loss_changes":[2.1690437E-1,3.1309888E-1,0E0,1.0930562E-1,0E0,2.942162E-2,5.5484753E-2,0E0,0E0,3.8181283E-2,0E0,1.4262579E-2,2.3491066E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9,11,11,12,12],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,-1,14,16,-1,-1,-1,-1],"split_conditions":[7.149144E7,2.1118015E3,-7.512694E-3,4.010025E-1,1.0633236E-2,4.785384E5,1.0379711E0,-5.5766973E-4,-4.913017E-3,7.673801E2,4.093079E-3,7.6351406E5,1.5308358E-1,-1.4520276E-4,-2.5765202E-3,3.4248976E-3,2.9754406E-4],"split_indices":[57,61,0,67,0,37,51,0,0,61,0,56,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.6E1,6E0,6.1E1,5E0,1.4E1,4.7E1,5E0,9E0,3.9E1,8E0,1.9E1,2E1,1.1E1,8E0,7E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-8.525402E-3,2.7367726E-2,-1.1971928E-1,-1.1167354E-2,6.36303E-2,-1.2759647E-2,-1.49282E-2,2.3509348E-2,-6.6895835E-2,7.847965E-3,2.0501578E-2,-3.6274453E-3,1.7868988E-3,-1.5450334E-3,6.342308E-2,-4.6457993E-3,-8.768667E-4,4.486708E-2,-2.3393848E-3,4.733757E-3,5.022118E-4,-1.3415006E-4,3.2077956E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[2.9462644E-1,7.8963906E-2,4.035288E-1,5.5682115E-2,1.3482079E-1,4.9415328E-2,0E0,4.371674E-2,1.6913455E-2,0E0,4.1168407E-2,0E0,0E0,0E0,2.2803988E-2,0E0,0E0,2.1660145E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,14,14,17,17],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[1.0232127E12,8.38261E5,1.0302285E8,2.9876392E7,7.33E2,8.1637933E2,-1.49282E-2,2.4460206E0,5.02409E5,7.847965E-3,2.6954106E3,-3.6274453E-3,1.7868988E-3,-1.5450334E-3,2.384E3,-4.6457993E-3,-8.768667E-4,3.9976162E-1,-2.3393848E-3,4.733757E-3,5.022118E-4,-1.3415006E-4,3.2077956E-3],"split_indices":[40,1,54,54,0,61,0,65,1,0,61,0,0,0,10,0,0,36,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,5.5E1,1.7E1,2.7E1,2.8E1,1.2E1,5E0,1.7E1,1E1,7E0,2.1E1,5E0,7E0,7E0,1E1,5E0,5E0,1.6E1,5E0,5E0,5E0,6E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.6454101E-3,1.6991882E-2,-9.382766E-3,-5.064279E-3,9.980793E-3,-2.0851854E-2,7.18343E-2,-5.9995685E-2,1.6154627E-3,1.769643E-3,4.649092E-3,-6.8384656E-3,-2.6040338E-2,4.325696E-2,-1.9262983E-2,-5.0348826E-2,1.3945143E-3,-4.6268973E-4,3.3385102E-3,-3.7070564E-3,1.6771421E-3,9.8014294E-5,-3.757777E-3,1.7473547E-3,-1.4670536E-2,-2.2592358E-3,2.7140297E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,-1,15,17,19,21,-1,-1,-1,-1,23,-1,-1,-1,25,-1,-1],"loss_changes":[2.8393304E-1,3.7125808E-1,0E0,9.255104E-2,0E0,5.5502705E-2,9.227365E-3,6.7790896E-2,3.647906E-2,0E0,0E0,0E0,2.6503414E-2,2.3905938E-2,3.7111606E-2,2.278366E-2,0E0,0E0,0E0,0E0,1.3921786E-2,0E0,0E0,0E0,1.2058797E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,20,20,24,24],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,-1,16,18,20,22,-1,-1,-1,-1,24,-1,-1,-1,26,-1,-1],"split_conditions":[1.3866357E8,4.9891987E3,-9.382766E-3,5.2518907E0,9.980793E-3,2.8506322E-2,4.7449777E-1,3.2716873E0,6.3380525E5,1.769643E-3,4.649092E-3,-6.8384656E-3,1.7493458E3,3.9785632E8,2.3239934E-1,3.5041518E0,1.3945143E-3,-4.6268973E-4,3.3385102E-3,-3.7070564E-3,4.9098603E-2,9.8014294E-5,-3.757777E-3,1.7473547E-3,1.1343E4,-2.2592358E-3,2.7140297E-4],"split_indices":[57,4,0,63,0,48,36,65,41,0,0,0,4,7,48,62,0,0,0,0,47,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,8.1E1,5E0,7.4E1,7E0,6.2E1,1.2E1,2.2E1,4E1,7E0,5E0,5E0,1.7E1,1.3E1,2.7E1,1.2E1,5E0,5E0,8E0,6E0,2.1E1,5E0,7E0,6E0,1.5E1,5E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.23835765E-2,2.7471717E-2,-7.0928014E-3,7.5631808E-3,1.193824E-2,-4.3597948E-2,3.5117894E-2,-7.973301E-3,-7.2507416E-3,-3.159328E-3,4.8991065E-2,-4.185042E-2,2.355478E-3,4.801095E-3,3.4824137E-2,3.0391084E-4,-4.116431E-3,-9.4128744E-4,4.8996378E-2,2.9364682E-3,1.0818364E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,-1,7,9,11,-1,-1,13,15,-1,-1,17,-1,-1,-1,19,-1,-1],"loss_changes":[2.196515E-1,3.6702093E-1,0E0,1.04441375E-1,0E0,1.07475296E-1,7.3542684E-2,4.507465E-2,0E0,0E0,3.1392388E-2,3.4339562E-2,0E0,0E0,2.9456336E-2,0E0,0E0,0E0,1.017534E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,10,10,11,11,14,14,18,18],"right_children":[2,4,-1,6,-1,8,10,12,-1,-1,14,16,-1,-1,18,-1,-1,-1,20,-1,-1],"split_conditions":[5.673536E2,3.3077192E3,-7.0928014E-3,7.261484E-1,1.193824E-2,1.1686677E8,2.5547945E0,7.4374756E2,-7.2507416E-3,-3.159328E-3,9.9948E4,3.1933124E0,2.355478E-3,4.801095E-3,3.67827E2,3.0391084E-4,-4.116431E-3,-9.4128744E-4,1E0,2.9364682E-3,1.0818364E-3],"split_indices":[67,61,0,66,0,54,62,61,0,0,38,62,0,0,61,0,0,0,25,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.3E1,7.7E1,6E0,7.2E1,5E0,2.5E1,4.7E1,2E1,5E0,5E0,4.2E1,1.3E1,7E0,7E0,3.5E1,7E0,6E0,7E0,2.8E1,1.6E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.652122E-2,2.4381476E-3,-9.188494E-3,-2.4603378E-2,5.3536814E-2,-7.5432345E-2,-2.554249E-3,1.3025004E-1,-3.762923E-3,-8.5963093E-4,-4.8330873E-3,-2.3322064E-2,1.9987393E-3,2.0729147E-3,8.277922E-3,-1.5642496E-3,1.2441834E-3,1.6259595E-3,-4.2021457E-2,-1.9082844E-2,-3.4633684E-3,-2.2799901E-3,1.8328437E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,-1,5,7,9,11,13,15,-1,-1,17,-1,-1,-1,-1,-1,-1,19,21,-1,-1,-1],"loss_changes":[3.0981925E-1,1.1186007E-1,0E0,5.9206247E-2,1.23568244E-1,2.6705109E-2,3.8079493E-2,4.568419E-2,1.7490095E-2,0E0,0E0,3.152248E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.5339151E-2,1.04325E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,11,11,18,18,19,19],"right_children":[2,4,-1,6,8,10,12,14,16,-1,-1,18,-1,-1,-1,-1,-1,-1,20,22,-1,-1,-1],"split_conditions":[1.5135763E8,1.2052782E3,-9.188494E-3,7.97E2,1.383E4,6.1359977E4,4.894405E0,4.0342972E11,1.859E3,-8.5963093E-4,-4.8330873E-3,1.0689672E7,1.9987393E-3,2.0729147E-3,8.277922E-3,-1.5642496E-3,1.2441834E-3,1.6259595E-3,3.330335E0,2.8E1,-3.4633684E-3,-2.2799901E-3,1.8328437E-4],"split_indices":[54,61,0,10,10,41,63,40,0,0,0,54,0,0,0,0,0,0,62,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,5.2E1,2.7E1,1.5E1,3.7E1,1.1E1,1.6E1,6E0,9E0,2.6E1,1.1E1,5E0,6E0,8E0,8E0,6E0,2E1,1.3E1,7E0,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.3265707E-2,2.2304796E-2,-5.035924E-2,-2.9969476E-3,3.994313E-2,-1.04479775E-1,-1.4435608E-2,4.4646156E-3,2.3715835E-2,-1.8847788E-3,-8.620684E-3,1.7468724E-2,-3.131903E-3,2.107575E-3,-2.9376106E-4,-1.041198E-3,2.744604E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,7,9,11,-1,13,-1,-1,15,-1,-1,-1,-1,-1],"loss_changes":[1.02719806E-1,6.535084E-2,7.23716E-2,0E0,3.0312624E-2,7.367487E-2,4.341662E-2,0E0,1.9853411E-2,0E0,0E0,2.9648038E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,11,11],"right_children":[2,4,6,-1,8,10,12,-1,14,-1,-1,16,-1,-1,-1,-1,-1],"split_conditions":[7.455661E-1,1.25313E5,7.281736E2,-2.9969476E-3,2.384E3,1.0370839E6,8.418953E0,4.4646156E-3,8.251192E0,-1.8847788E-3,-8.620684E-3,3.7818575E0,-3.131903E-3,2.107575E-3,-2.9376106E-4,-1.041198E-3,2.744604E-3],"split_indices":[36,1,61,0,10,37,65,0,66,0,0,65,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,3.9E1,3.7E1,6E0,3.3E1,1.4E1,2.3E1,6E0,2.7E1,9E0,5E0,1.5E1,8E0,1.5E1,1.2E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.0703319E-2,2.8948072E-2,-1.0552662E-2,1.1784658E-2,8.193135E-3,6.4743534E-2,-8.047975E-3,-4.0627542E-4,8.993359E-2,-5.3415652E-2,7.6397196E-3,9.340746E-4,5.514208E-3,-4.0237787E-3,-2.1207951E-4,3.701444E-3,-4.9989265E-3,-1.6817138E-2,1.7321924E-3,-4.218151E-3,-2.9004351E-3,4.553891E-4,-1.5375401E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,7,9,-1,11,13,15,-1,-1,-1,-1,-1,17,19,-1,21,-1,-1,-1],"loss_changes":[3.8680413E-1,2.079328E-1,0E0,7.719396E-2,0E0,3.7974656E-2,3.8840648E-2,0E0,3.1212032E-2,2.4086595E-2,3.904601E-2,0E0,0E0,0E0,0E0,0E0,1.9139707E-2,1.7117426E-2,0E0,1.063823E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,9,9,10,10,16,16,17,17,19,19],"right_children":[2,4,-1,6,-1,8,10,-1,12,14,16,-1,-1,-1,-1,-1,18,20,-1,22,-1,-1,-1],"split_conditions":[1.5135763E8,4.9891987E3,-1.0552662E-2,1.7329262E7,8.193135E-3,1.0269974E5,5.255E2,-4.0627542E-4,4.4939005E2,3.29E2,2.5036643E8,9.340746E-4,5.514208E-3,-4.0237787E-3,-2.1207951E-4,3.701444E-3,4.02451E0,4.501324E7,1.7321924E-3,3.5307167E0,-2.9004351E-3,4.553891E-4,-1.5375401E-3],"split_indices":[54,4,0,54,0,42,61,0,4,10,7,0,0,0,0,0,62,59,0,62,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,7.9E1,5E0,7.2E1,7E0,1.9E1,5.3E1,5E0,1.4E1,1.3E1,4E1,5E0,9E0,7E0,6E0,5E0,3.5E1,2.8E1,7E0,2.3E1,5E0,1.6E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[9.265228E-3,9.519965E-3,-7.02995E-3,4.950572E-3,-5.589178E-3,-4.566655E-2,1.9856561E-2,-3.4166542E-3,7.8014634E-4,2.9823175E-2,-2.9200977E-3,6.722234E-2,1.4450632E-2,4.3787467E-3,1.4659626E-3,-1.2599977E-2,4.905139E-2,3.1363926E-4,-2.8222196E-3,2.9428247E-3,4.949982E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,-1,3,5,-1,7,9,-1,-1,11,-1,13,15,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[2.65006E-1,0E0,1.0582391E-1,5.296716E-2,0E0,3.1759374E-2,4.752109E-2,0E0,0E0,2.7199693E-2,0E0,1.1402987E-2,3.4313023E-2,0E0,0E0,2.1009888E-2,9.207509E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,9,9,11,11,12,12,15,15,16,16],"right_children":[2,-1,4,6,-1,8,10,-1,-1,12,-1,14,16,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[1E0,9.519965E-3,1.5277152E8,1.528E3,-5.589178E-3,3.7566137E-1,7.149144E7,-3.4166542E-3,7.8014634E-4,4.654812E5,-2.9200977E-3,9.5E2,8.43E2,4.3787467E-3,1.4659626E-3,1.7554E4,1.49E2,3.1363926E-4,-2.8222196E-3,2.9428247E-3,4.949982E-4],"split_indices":[13,0,54,11,0,67,57,0,0,41,0,0,0,0,0,9,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,5E0,7.4E1,6.8E1,6E0,1.5E1,5.3E1,1E1,5E0,4.8E1,5E0,1.3E1,3.5E1,6E0,7E0,2E1,1.5E1,1.5E1,5E0,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-7.1000643E-3,8.979216E-3,-8.445587E-3,-6.3811E-2,2.703647E-2,-1.2639231E-2,-6.6190893E-3,1.091906E-2,8.942638E-2,-2.7207914E-3,1.6773444E-3,-2.066394E-3,4.9902345E-3,5.8560963E-3,1.6224858E-3,-1.6431635E-2,2.7204184E-3,2.3905933E-3,-3.6084738E-2,-4.8660006E-2,6.2961347E-4,-3.0057313E-2,-4.1518984E-3,-2.3059498E-3,6.08691E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,-1,5,7,9,-1,11,13,-1,-1,15,-1,-1,-1,17,-1,-1,19,21,-1,23,-1,-1,-1],"loss_changes":[2.4831466E-1,1.0556214E-1,0E0,6.6245414E-2,6.3379645E-2,2.847174E-2,0E0,6.7685515E-2,2.3510784E-2,0E0,0E0,4.317638E-2,0E0,0E0,0E0,5.520056E-2,0E0,0E0,2.0197995E-2,1.7478433E-2,0E0,1.8203577E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,11,11,15,15,18,18,19,19,21,21],"right_children":[2,4,-1,6,8,10,-1,12,14,-1,-1,16,-1,-1,-1,18,-1,-1,20,22,-1,24,-1,-1,-1],"split_conditions":[3.698723E10,3.2716873E0,-8.445587E-3,1.5E1,6.7469894E5,1.3694E4,-6.6190893E-3,1.5092398E8,1.2E1,-2.7207914E-3,1.6773444E-3,5.2518907E0,4.9902345E-3,5.8560963E-3,1.6224858E-3,2.8216465E0,2.7204184E-3,2.3905933E-3,1.859E3,5.963347E8,6.2961347E-4,3.1457312E8,-4.1518984E-3,-2.3059498E-3,6.08691E-4],"split_indices":[5,65,0,3,37,9,0,54,3,0,0,63,0,0,0,62,0,0,0,7,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.4E1,7.8E1,6E0,1.5E1,6.3E1,1E1,5E0,5.1E1,1.2E1,5E0,5E0,4.6E1,5E0,6E0,6E0,3.8E1,8E0,8E0,3E1,2.4E1,6E0,1.8E1,6E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.0847528E-2,3.0143633E-2,-5.12144E-3,1.21859135E-2,9.082861E-3,-2.551472E-2,2.2354474E-2,-3.3526076E-3,1.2188533E-3,2.9984813E-3,8.599924E-3,-1.0367319E-3,2.4699744E-2,1.9057577E-3,-8.1218703E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,-1,5,-1,7,9,-1,-1,-1,11,-1,13,-1,-1],"loss_changes":[1.7891338E-1,1.947153E-1,0E0,2.3188673E-2,0E0,3.5466768E-2,2.8022442E-2,0E0,0E0,0E0,1.9406103E-2,0E0,1.9590255E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,10,10,12,12],"right_children":[2,4,-1,6,-1,8,10,-1,-1,-1,12,-1,14,-1,-1],"split_conditions":[1.5092398E8,3.6591887E3,-5.12144E-3,3.8045087E2,9.082861E-3,1.5039519E5,9.693049E2,-3.3526076E-3,1.2188533E-3,2.9984813E-3,8.1637933E2,-1.0367319E-3,1.5710696E2,1.9057577E-3,-8.1218703E-4],"split_indices":[54,4,0,61,0,42,4,0,0,0,61,0,67,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.3E1,9E0,5.8E1,5E0,1.2E1,4.6E1,6E0,6E0,1E1,3.6E1,1.2E1,2.4E1,1.7E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-9.251901E-3,8.6616E-3,-8.431109E-3,-4.5336753E-2,1.8857438E-2,-5.096535E-4,-3.281785E-3,6.5649934E-2,6.991409E-4,2.0817848E-2,7.174126E-3,1.7214308E-2,-2.1447984E-3,-8.098654E-4,3.4021614E-3,-1.6615642E-3,3.5280302E-2,1.9557688E-4,6.386539E-2,4.809081E-3,1.4284024E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,7,-1,-1,9,11,13,-1,15,-1,-1,-1,-1,17,-1,19,-1,-1],"loss_changes":[2.7899668E-1,4.4899613E-2,0E0,1.1355538E-2,5.77318E-2,0E0,0E0,7.646945E-2,4.0727887E-2,3.1092439E-2,0E0,3.8450666E-2,0E0,0E0,0E0,0E0,2.5368486E-2,0E0,1.7312117E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,9,9,11,11,16,16,18,18],"right_children":[2,4,-1,6,8,-1,-1,10,12,14,-1,16,-1,-1,-1,-1,18,-1,20,-1,-1],"split_conditions":[1.5414109E8,1.259E3,-8.431109E-3,3.5665236E0,1.1E1,-5.096535E-4,-3.281785E-3,7.285458E-1,2.781511E9,8.3744875E5,7.174126E-3,3.8045087E2,-2.1447984E-3,-8.098654E-4,3.4021614E-3,-1.6615642E-3,7.5E1,1.9557688E-4,3.9380467E9,4.809081E-3,1.4284024E-3],"split_indices":[54,11,0,63,8,0,0,48,12,37,0,61,0,0,0,0,8,0,5,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.6E1,7.9E1,7E0,1.2E1,6.7E1,6E0,6E0,1.8E1,4.9E1,1.3E1,5E0,3.7E1,1.2E1,8E0,5E0,9E0,2.8E1,1.4E1,1.4E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-5.3794305E-3,-5.4681428E-2,2.179381E-2,-1.8411703E-2,-6.441828E-3,-1.8811072E-3,3.1197818E-2,-4.389271E-2,2.368895E-3,1.5197858E-2,5.973973E-3,-2.823393E-3,-9.449306E-5,-1.5459459E-2,3.6280215E-2,-1.9203072E-3,1.3269766E-3,5.287988E-2,-1.0056294E-3,3.873558E-3,1.1878865E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,-1,9,11,-1,13,-1,-1,-1,15,17,-1,-1,19,-1,-1,-1],"loss_changes":[1.0568793E-1,8.7113366E-2,3.1790618E-2,4.0310837E-2,0E0,0E0,7.103966E-2,1.2444599E-2,0E0,2.641401E-2,0E0,0E0,0E0,2.2019781E-2,2.4531607E-2,0E0,0E0,1.4417335E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,9,9,13,13,14,14,17,17],"right_children":[2,4,6,8,-1,-1,10,12,-1,14,-1,-1,-1,16,18,-1,-1,20,-1,-1,-1],"split_conditions":[6.6504064E0,1.0789844E6,1.1269477E5,1.264233E6,-6.441828E-3,-1.8811072E-3,1.1997641E0,6.7469894E5,2.368895E-3,3.7956493E0,5.973973E-3,-2.823393E-3,-9.449306E-5,8.8E1,1.49E2,-1.9203072E-3,1.3269766E-3,4.434805E8,-1.0056294E-3,3.873558E-3,1.1878865E-3],"split_indices":[67,37,42,1,0,0,48,37,0,66,0,0,0,8,8,0,0,7,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,2.7E1,5E1,2E1,7E0,6E0,4.4E1,1.5E1,5E0,3.9E1,5E0,1E1,5E0,1.6E1,2.3E1,1E1,6E0,1.8E1,5E0,7E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.1510021E-2,-3.9059866E-2,3.7259713E-2,-1.8955523E-2,-8.232795E-3,5.224085E-2,-1.7499916E-3,-4.6754284E-3,-5.628118E-3,6.52905E-2,1.6952594E-4,3.3726245E-2,-1.7496726E-2,8.2749815E-4,7.800783E-2,8.599597E-6,2.7757112E-3,-3.065812E-2,1.4590122E-3,4.2815064E-3,1.3897605E-3,-1.8677401E-3,4.17417E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,-1,9,-1,-1,11,13,-1,15,17,-1,19,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[1.2237924E-1,1.6417375E-1,3.9663173E-2,5.823417E-2,0E0,1.7466925E-2,0E0,0E0,2.2075849E-2,1.1692204E-2,0E0,1.0285453E-2,2.4635108E-2,0E0,1.0748282E-2,0E0,0E0,1.2911797E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,8,8,9,9,11,11,12,12,14,14,17,17],"right_children":[2,4,6,8,-1,10,-1,-1,12,14,-1,16,18,-1,20,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[1.2052782E3,9.8788344E5,2.3297022E7,2.0672107E8,-8.232795E-3,3.9581906E2,-1.7499916E-3,-4.6754284E-3,1.2094463E7,2.7014925E0,1.6952594E-4,1.2566517E5,7.6006556E0,8.2749815E-4,1.7581408E1,8.599597E-6,2.7757112E-3,1.6E1,1.4590122E-3,4.2815064E-3,1.3897605E-3,-1.8677401E-3,4.17417E-4],"split_indices":[61,37,41,7,0,67,0,0,54,62,0,42,63,0,65,0,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,5.7E1,3.2E1,5.1E1,6E0,2.7E1,5E0,6E0,4.5E1,2.1E1,6E0,1E1,3.5E1,5E0,1.6E1,5E0,5E0,2.8E1,7E0,1.1E1,5E0,2.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.4646899E-2,3.1333793E-2,-8.592666E-3,7.4058045E-3,1.3387407E-2,-2.5665015E-2,3.2655004E-2,1.2470824E-3,-6.012549E-3,7.821551E-2,9.207064E-4,-1.6404991E-3,1.792349E-2,5.228612E-3,1.975325E-3,-2.419211E-2,2.0472473E-3,2.6488819E-3,-1.050288E-4,1.7294877E-4,-2.7370963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,-1,7,9,11,-1,13,15,-1,17,-1,-1,19,-1,-1,-1,-1,-1],"loss_changes":[2.734078E-1,4.6236488E-1,0E0,5.7618886E-2,0E0,8.724563E-2,5.6315385E-2,1.6361222E-2,0E0,1.5734226E-2,2.7946081E-2,0E0,1.5049748E-2,0E0,0E0,1.6645346E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,9,9,10,10,12,12,15,15],"right_children":[2,4,-1,6,-1,8,10,12,-1,14,16,-1,18,-1,-1,20,-1,-1,-1,-1,-1],"split_conditions":[1.3866357E8,6.7780703E3,-8.592666E-3,7.281736E2,1.3387407E-2,1.052686E6,1.9188015E9,1.648E3,-6.012549E-3,1.2E1,1.4377E4,-1.6404991E-3,1.1768685E9,5.228612E-3,1.975325E-3,4.2347652E7,2.0472473E-3,2.6488819E-3,-1.050288E-4,1.7294877E-4,-2.7370963E-3],"split_indices":[57,4,0,61,0,37,12,11,0,3,2,0,5,0,0,59,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.2E1,5E0,6.7E1,5E0,2.9E1,3.8E1,2.4E1,5E0,1.5E1,2.3E1,7E0,1.7E1,6E0,9E0,1.5E1,8E0,5E0,1.2E1,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.9210104E-2,3.7863888E-3,9.40262E-3,1.5375453E-2,-4.9979337E-3,-3.791404E-2,2.9629396E-2,-3.856181E-3,-2.4723276E-4,1.9145807E-2,3.99995E-3,3.9033508E-3,9.276891E-3,-1.6443547E-2,3.4575652E-2,-1.424035E-3,1.3197325E-3,-7.6539356E-5,2.9245957E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,-1,-1,13,15,17,-1,-1,-1,-1],"loss_changes":[2.3314984E-1,1.01467475E-1,0E0,5.353477E-2,0E0,2.2738991E-2,3.2432664E-2,0E0,0E0,3.118833E-2,0E0,0E0,2.8548442E-2,1.6205937E-2,2.4348307E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9,12,12,13,13,14,14],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,-1,-1,14,16,18,-1,-1,-1,-1],"split_conditions":[8.791079E3,3.698723E10,9.40262E-3,2.074E3,-4.9979337E-3,2.8703241E0,1.1686677E8,-3.856181E-3,-2.4723276E-4,4.80373E5,3.99995E-3,3.9033508E-3,1E0,1.144E4,1.3611112E0,-1.424035E-3,1.3197325E-3,-7.6539356E-5,2.9245957E-3],"split_indices":[4,5,0,2,0,62,54,0,0,12,0,0,18,2,66,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.9E1,7.4E1,5E0,6.8E1,6E0,1.4E1,5.4E1,5E0,9E0,4.7E1,7E0,5E0,4.2E1,2.1E1,2.1E1,1.6E1,5E0,1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.1598635E-3,-1.55872265E-2,9.40462E-3,9.23594E-3,-1.3633552E-1,-1.2890636E-2,5.3027295E-2,5.146452E-4,-1.0965602E-2,6.0546696E-3,-2.5938065E-3,3.870623E-3,8.228042E-4,-3.0430772E-3,2.8201334E-2,3.3988005E-3,7.0757847E-3,-1.2155935E-3,1.5141411E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,-1,5,7,9,11,-1,-1,13,-1,-1,-1,-1,15,-1,17,-1,-1],"loss_changes":[2.7836555E-1,2.1107405E-1,0E0,5.785176E-2,1.9282913E-1,3.4230743E-2,2.1317802E-2,0E0,0E0,4.9196545E-2,0E0,0E0,0E0,0E0,2.2436835E-2,0E0,1.6284807E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,9,9,14,14,16,16],"right_children":[2,4,-1,6,8,10,12,-1,-1,14,-1,-1,-1,-1,16,-1,18,-1,-1],"split_conditions":[4.0235266E3,1.0232127E12,9.40462E-3,1.2052782E3,1.3E1,5.7229916E7,1.4743697E5,5.146452E-4,-1.0965602E-2,1.296E3,-2.5938065E-3,3.870623E-3,8.228042E-4,-3.0430772E-3,4.2317244E5,3.3988005E-3,2.748086E8,-1.2155935E-3,1.5141411E-3],"split_indices":[61,40,0,61,3,54,42,0,0,2,0,0,0,0,56,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,5.8E1,1.1E1,3.9E1,1.9E1,5E0,6E0,2.8E1,1.1E1,9E0,1E1,6E0,2.2E1,6E0,1.6E1,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.51216835E-2,-4.96507E-3,8.05811E-3,3.7868455E-2,-2.0598507E-2,-8.8867225E-4,2.7712598E-3,-3.861911E-2,4.5165315E-2,-2.4602627E-2,-5.0258413E-3,3.3058964E-3,4.2278093E-4,-1.7076535E-3,3.0145625E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,-1,5,7,-1,-1,9,11,13,-1,-1,-1,-1,-1],"loss_changes":[2.4111123E-1,4.497595E-2,0E0,2.5760254E-2,5.9814334E-2,0E0,0E0,3.715369E-2,1.0427922E-2,1.4427049E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,7,7,8,8,9,9],"right_children":[2,4,-1,6,8,-1,-1,10,12,14,-1,-1,-1,-1,-1],"split_conditions":[6.7780703E3,1.7329262E7,8.05811E-3,1.0269974E5,2.009E3,-8.8867225E-4,2.7712598E-3,1.2427474E8,1.910669E-2,1.6E1,-5.0258413E-3,3.3058964E-3,4.2278093E-4,-1.7076535E-3,3.0145625E-4],"split_indices":[4,54,0,42,0,0,0,54,47,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.2E1,6.5E1,7E0,1.7E1,4.8E1,5E0,1.2E1,3.8E1,1E1,3.3E1,5E0,5E0,5E0,2.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-5.908673E-3,8.689436E-3,-6.426051E-3,-1.4351593E-2,6.2401384E-2,-5.525721E-2,1.0670163E-2,7.7106943E-3,1.4709002E-2,-5.700936E-4,-7.550074E-2,2.9563555E-2,-1.6525917E-3,2.8086933E-3,-6.916819E-4,-4.796834E-3,-1.2033896E-3,1.8393178E-4,2.111426E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,-1,5,7,9,11,-1,13,-1,15,17,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4502053E-1,8.2575746E-2,0E0,4.8695527E-2,1.0128929E-1,1.4565837E-2,2.7934032E-2,0E0,2.2456333E-2,0E0,1.5311778E-2,9.519834E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,10,10,11,11],"right_children":[2,4,-1,6,8,10,12,-1,14,-1,16,18,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.698723E10,9.334E3,-6.426051E-3,3.731E3,1.61E4,1.762456E3,4.8178736E7,7.7106943E-3,6.1351955E8,-5.700936E-4,1.9598669E6,2.912E3,-1.6525917E-3,2.8086933E-3,-6.916819E-4,-4.796834E-3,-1.2033896E-3,1.8393178E-4,2.111426E-3],"split_indices":[5,2,0,11,10,56,54,0,7,0,41,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.1E1,6.5E1,6E0,4.6E1,1.9E1,1.7E1,2.9E1,5E0,1.4E1,6E0,1.1E1,2.1E1,8E0,5E0,9E0,6E0,5E0,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[-1.11022005E-2,1.4173898E-3,-7.3697222E-3,1.8712431E-2,-5.6545176E-3,6.32952E-3,6.371236E-3,-3.4627523E-2,2.427962E-2,-6.0634136E-2,3.922544E-4,3.330185E-3,1.2566496E-2,-8.3454413E-4,-4.487155E-3,-6.5263333E-3,5.1296014E-2,-1.0545427E-3,1.4543303E-3,3.3224262E-3,7.436224E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,-1,5,-1,7,-1,9,11,13,-1,-1,15,-1,-1,17,19,-1,-1,-1,-1],"loss_changes":[1.5111902E-1,1.648729E-1,0E0,9.944439E-2,0E0,4.568691E-2,0E0,2.1942768E-2,2.4089742E-2,1.7863024E-2,0E0,0E0,2.6955191E-2,0E0,0E0,1.7151184E-2,8.166738E-3,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,9,9,12,12,15,15,16,16],"right_children":[2,4,-1,6,-1,8,-1,10,12,14,-1,-1,16,-1,-1,18,20,-1,-1,-1,-1],"split_conditions":[1.89293E7,1.3237324E6,-7.3697222E-3,2.1118015E3,-5.6545176E-3,1E1,6.371236E-3,7.97E2,1.7329262E7,1E0,3.922544E-4,3.330185E-3,1.868E3,-8.3454413E-4,-4.487155E-3,4.02451E0,1.5308358E-1,-1.0545427E-3,1.4543303E-3,3.3224262E-3,7.436224E-4],"split_indices":[1,37,0,61,0,8,0,10,54,22,0,0,0,0,0,62,50,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.8E1,7.3E1,5E0,6.5E1,8E0,6E1,5E0,1.8E1,4.2E1,1.1E1,7E0,7E0,3.5E1,6E0,5E0,2.4E1,1.1E1,1.7E1,7E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-7.1154875E-3,-1.8787872E-2,1.5954406E-3,-6.784576E-3,-5.2809496E-3,8.8193E-3,-4.28167E-2,-3.0444777E-2,2.7470257E-2,-2.9870118E-3,6.8766624E-4,7.2894496E-4,-3.0369863E-3,3.1752954E-3,1.056931E-2,1.6682021E-3,-3.2450515E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,-1,7,9,11,13,-1,-1,-1,-1,-1,15,-1,-1],"loss_changes":[4.138755E-2,7.6045044E-2,0E0,3.354982E-2,0E0,3.1593453E-2,2.5600985E-2,2.5245998E-2,2.0351674E-2,0E0,0E0,0E0,0E0,0E0,1.0621007E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,14,14],"right_children":[2,4,-1,6,-1,8,10,12,14,-1,-1,-1,-1,-1,16,-1,-1],"split_conditions":[2.009E3,1.0104842E12,1.5954406E-3,8.0159146E-1,-5.2809496E-3,3.67827E2,3.9497445E0,1.2817016E7,3.1841638E5,-2.9870118E-3,6.8766624E-4,7.2894496E-4,-3.0369863E-3,3.1752954E-3,4.58708E5,1.6682021E-3,-3.2450515E-4],"split_indices":[0,40,0,36,0,61,63,12,41,0,0,0,0,0,38,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.1E1,6.4E1,1.7E1,5.8E1,6E0,4.1E1,1.7E1,1.3E1,2.8E1,1.2E1,5E0,6E0,7E0,7E0,2.1E1,8E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-9.316279E-3,9.810546E-3,-8.011058E-3,3.3185102E-2,-9.134361E-3,7.30063E-3,6.4904657E-3,-2.4135403E-2,2.3878222E-2,3.0339496E-2,-2.8305782E-3,-4.4603672E-2,1.3116896E-4,-3.8984002E-4,2.0396647E-3,4.605358E-2,1.9229512E-4,-3.6070342E-4,-3.2500995E-3,3.1561085E-3,2.8513823E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,-1,5,7,9,-1,11,13,15,-1,17,-1,-1,-1,19,-1,-1,-1,-1,-1],"loss_changes":[2.5325754E-1,3.1743966E-2,0E0,9.084736E-2,2.0385237E-2,4.5549806E-2,0E0,1.5631607E-2,9.450592E-3,8.406056E-3,0E0,1.5564922E-2,0E0,0E0,0E0,1.1967074E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,8,8,9,9,11,11,15,15],"right_children":[2,4,-1,6,8,10,-1,12,14,16,-1,18,-1,-1,-1,20,-1,-1,-1,-1,-1],"split_conditions":[1.5092398E8,7.33E2,-8.011058E-3,2.8490616E11,1E0,7.83225E-1,6.4904657E-3,1.6361E4,3E1,4.4605106E5,-2.8305782E-3,8.418953E0,1.3116896E-4,-3.8984002E-4,2.0396647E-3,3.194E3,1.9229512E-4,-3.6070342E-4,-3.2500995E-3,3.1561085E-3,2.8513823E-4],"split_indices":[54,0,0,40,22,36,0,9,8,37,0,65,0,0,0,10,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7E1,7E0,3.1E1,3.9E1,2.6E1,5E0,2.7E1,1.2E1,2E1,6E0,1.5E1,1.2E1,5E0,7E0,1.2E1,8E0,7E0,8E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[1.4999912E-3,1.60147E-2,-7.89593E-3,-2.692657E-3,1.0148212E-2,-3.18687E-2,2.4126844E-2,-4.822507E-3,-4.2145406E-3,4.0381566E-2,-1.3114798E-3,-2.4869079E-3,1.1348515E-2,2.731249E-3,7.104787E-4,1.9521205E-3,-7.144994E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,-1,7,9,11,-1,13,-1,-1,15,-1,-1,-1,-1],"loss_changes":[2.0302343E-1,2.8649968E-1,0E0,5.3978004E-2,0E0,5.463882E-2,3.2336958E-2,2.0188952E-2,0E0,1.3008282E-2,0E0,0E0,1.7236095E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,9,9,12,12],"right_children":[2,4,-1,6,-1,8,10,12,-1,14,-1,-1,16,-1,-1,-1,-1],"split_conditions":[1.06102504E8,3.2311362E3,-7.89593E-3,8.6646006E5,1.0148212E-2,1.0112447E6,2.9123648E7,1.43E2,-4.2145406E-3,4.800481E5,-1.3114798E-3,-2.4869079E-3,2.2858976E8,2.731249E-3,7.104787E-4,1.9521205E-3,-7.144994E-4],"split_indices":[60,4,0,56,0,41,59,10,0,37,0,0,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.7E1,7.2E1,5E0,6.7E1,5E0,3.2E1,3.5E1,2.3E1,9E0,2.7E1,8E0,5E0,1.8E1,1.4E1,1.3E1,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[8.132251E-3,-9.883916E-3,9.770099E-3,-7.6576034E-5,-5.155235E-3,-2.9228002E-2,3.2838795E-2,-6.941099E-2,-5.245097E-3,7.110639E-2,1.2507351E-3,-6.233637E-4,-5.181364E-3,-2.4205094E-2,1.8710196E-3,4.5033977E-3,1.6629694E-3,7.1563304E-4,-1.3815738E-3,7.759894E-4,-1.9321206E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,-1,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8432256E-1,7.17998E-2,0E0,6.3329265E-2,0E0,3.33656E-2,3.7383832E-2,3.1385407E-2,2.1412082E-2,1.0135107E-2,8.902966E-3,0E0,0E0,1.3913582E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13],"right_children":[2,4,-1,6,-1,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0235266E3,3.698723E10,9.770099E-3,2.1933217E0,-5.155235E-3,6E0,8.903939E0,1.5138E4,1E0,1.5094787E7,3.7582534E5,-6.233637E-4,-5.181364E-3,2.7252597E-1,1.8710196E-3,4.5033977E-3,1.6629694E-3,7.1563304E-4,-1.3815738E-3,7.759894E-4,-1.9321206E-3],"split_indices":[61,5,0,66,0,8,65,9,22,60,37,0,0,36,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6.9E1,5E0,6.4E1,5E0,3.4E1,3E1,1.2E1,2.2E1,1.3E1,1.7E1,6E0,6E0,1.6E1,6E0,6E0,7E0,1.2E1,5E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[2.4538586E-2,7.4001215E-3,1.000069E-2,1.72357E-2,-3.0998893E-3,-1.0067171E-3,7.202587E-2,-4.188415E-2,2.0081861E-2,5.361287E-3,1.7932939E-3,-3.2497097E-3,5.423649E-5,3.338734E-3,5.3220894E-3,-2.7565297E-4,1.7513509E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,-1,3,5,-1,7,9,11,13,-1,-1,-1,-1,-1,15,-1,-1],"loss_changes":[1.5078269E-1,0E0,4.054477E-2,6.404084E-2,0E0,4.308141E-2,1.949142E-2,2.2268994E-2,2.5937373E-2,0E0,0E0,0E0,0E0,0E0,1.0610489E-2,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,8,8,14,14],"right_children":[2,-1,4,6,-1,8,10,12,14,-1,-1,-1,-1,-1,16,-1,-1],"split_conditions":[1E0,7.4001215E-3,5.3111816E2,1.2777201E3,-3.0998893E-3,2.8586518E8,2.1777522E6,1.4324325E1,1.1E1,5.361287E-3,1.7932939E-3,-3.2497097E-3,5.423649E-5,3.338734E-3,1.0379711E0,-2.7565297E-4,1.7513509E-3],"split_indices":[13,0,67,61,0,7,56,65,3,0,0,0,0,0,51,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.4E1,6E0,6.8E1,6.3E1,5E0,4.8E1,1.5E1,1.6E1,3.2E1,5E0,1E1,9E0,7E0,6E0,2.6E1,2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[1.674403E-4,1.4682315E-2,-8.272832E-3,2.6995093E-3,5.2325055E-3,-7.2283964E-3,2.5083777E-3,2.0273643E-2,-3.4489486E-2,-9.101757E-4,4.152726E-2,7.881363E-4,-5.7605088E-2,-2.2193635E-4,3.0134988E-3,-9.866887E-2,-5.090477E-4,-6.448685E-3,-1.6970431E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,-1,5,-1,7,-1,9,11,-1,13,-1,15,-1,-1,17,-1,-1,-1],"loss_changes":[2.1888952E-1,9.213664E-2,0E0,3.6711924E-2,0E0,4.4933625E-2,0E0,2.665358E-2,3.70628E-2,0E0,2.3923613E-2,0E0,3.8811453E-2,0E0,0E0,2.4479263E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,7,7,8,8,10,10,12,12,15,15],"right_children":[2,4,-1,6,-1,8,-1,10,12,-1,14,-1,16,-1,-1,18,-1,-1,-1],"split_conditions":[1.3034653E7,4.9891987E3,-8.272832E-3,5.0175633E0,5.2325055E-3,1.5368E4,2.5083777E-3,2.727E3,4.5243898E-1,-9.101757E-4,1.1317E4,7.881363E-4,8.06853E5,-2.2193635E-4,3.0134988E-3,3.554517E0,-5.090477E-4,-6.448685E-3,-1.6970431E-3],"split_indices":[1,4,0,63,0,9,0,2,36,0,9,0,1,0,0,63,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.5E1,5E0,6.8E1,7E0,5.8E1,1E1,2.9E1,2.9E1,1E1,1.9E1,9E0,2E1,7E0,1.2E1,1E1,1E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[6.508777E-3,1.9697191E-2,-7.2782724E-3,-4.5455967E-5,1.1259271E-2,-3.718651E-2,1.4240439E-2,-5.8961675E-2,1.0559307E-3,2.6323025E-3,9.5203466E-4,-1.0506668E-3,-3.6079527E-3,9.193161E-3,-1.8187701E-3,6.206465E-5,1.8127452E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,-1,5,-1,7,9,11,-1,-1,13,-1,-1,15,-1,-1,-1],"loss_changes":[1.8279907E-1,3.4562075E-1,0E0,3.820167E-2,0E0,2.7789634E-2,3.0497018E-2,9.439599E-3,0E0,0E0,1.4340072E-2,0E0,0E0,8.4408E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,7,7,10,10,13,13],"right_children":[2,4,-1,6,-1,8,10,12,-1,-1,14,-1,-1,16,-1,-1,-1],"split_conditions":[2.1282262E7,6.7780703E3,-7.2782724E-3,4.279412E0,1.1259271E-2,1.7058623E3,1.3645554E7,1.4654219E4,1.0559307E-3,2.6323025E-3,2.5922572E7,-1.0506668E-3,-3.6079527E-3,2.2980049E10,-1.8187701E-3,6.206465E-5,1.8127452E-3],"split_indices":[56,4,0,67,0,4,54,42,0,0,41,0,0,5,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.5E1,5E0,7E1,5E0,1.9E1,5.1E1,1.4E1,5E0,1.1E1,4E1,6E0,8E0,3.4E1,6E0,2.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-1.749194E-2,-5.619104E-3,-7.874057E-3,-3.0912327E-2,1.5302415E-2,-4.450049E-2,2.103378E-3,2.269631E-3,6.351122E-3,-2.483001E-2,-5.655891E-3,-1.3883754E-4,1.3027668E-3,-3.1988092E-3,2.2195752E-3,2.2789408E-3,-2.028458E-2,-1.8138723E-3,4.3940236E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,-1,5,7,9,-1,-1,11,13,-1,-1,-1,-1,15,-1,17,-1,-1],"loss_changes":[1.5970021E-1,4.333602E-2,0E0,4.1104123E-2,1.3818256E-2,4.7930412E-2,0E0,0E0,7.9710195E-3,3.3903886E-2,0E0,0E0,0E0,0E0,2.062777E-2,0E0,8.202622E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,8,8,9,9,14,14,16,16],"right_children":[2,4,-1,6,8,10,-1,-1,12,14,-1,-1,-1,-1,16,-1,18,-1,-1],"split_conditions":[1.278211E8,8.6646006E5,-7.874057E-3,1.9315491E3,2.156825E7,1.052686E6,2.103378E-3,2.269631E-3,2.103989E10,5.212173E-1,-5.655891E-3,-1.3883754E-4,1.3027668E-3,-3.1988092E-3,1.12296E5,2.2789408E-3,1.2E1,-1.8138723E-3,4.3940236E-4],"split_indices":[57,56,0,4,54,37,0,0,5,36,0,0,0,0,1,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.5E1,8E1,5E0,3.6E1,4.4E1,3.1E1,5E0,8E0,3.6E1,2.6E1,5E0,2.6E1,1E1,9E0,1.7E1,5E0,1.2E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.2161051E-2,-2.2689688E-3,8.25387E-3,1.1289327E-2,-5.0428588E-2,-4.4690156E-3,4.3041226E-2,-8.871038E-2,1.2680389E-3,3.6792822E-2,-2.4170307E-2,3.6171568E-3,8.8501704E-4,-1.117986E-3,-6.2056063E-3,-7.004001E-4,3.217608E-3,-1.7281468E-3,9.863074E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9056419E-1,4.75826E-2,0E0,2.871272E-2,5.0633896E-2,3.259284E-2,1.49895325E-2,3.1080551E-2,0E0,2.4698358E-2,1.851356E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.661358E3,1.9E1,8.25387E-3,1.0781599E3,8.8E1,1.7329262E7,1.2440008E-1,1.2566517E5,1.2680389E-3,3.8734267E0,5.0175633E0,3.6171568E-3,8.8501704E-4,-1.117986E-3,-6.2056063E-3,-7.004001E-4,3.217608E-3,-1.7281468E-3,9.863074E-4],"split_indices":[61,3,0,61,8,54,51,42,0,65,63,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7.1E1,5E0,5.6E1,1.5E1,3.8E1,1.8E1,1E1,5E0,1.2E1,2.6E1,6E0,1.2E1,5E0,5E0,5E0,7E0,2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[4.259922E-3,1.9035148E-2,-6.4889584E-3,2.2403796E-3,6.7057204E-3,2.6029902E-2,-2.1619154E-2,-6.760358E-4,4.0336333E-2,-4.465303E-3,8.911023E-5,3.887205E-3,1.0668258E-3,-5.681495E-4,1.3623963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,13,-1,-1,-1,-1],"loss_changes":[1.693399E-1,1.52357E-1,0E0,3.632198E-2,0E0,1.9395143E-2,5.3874683E-2,0E0,1.6326122E-2,0E0,1.0347206E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,10,10],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,14,-1,-1,-1,-1],"split_conditions":[3.698723E10,3.3077192E3,-6.4889584E-3,1.5368E4,6.7057204E-3,6.8177136E2,6E0,-6.760358E-4,3.7894852E-2,-4.465303E-3,4.894405E0,3.887205E-3,1.0668258E-3,-5.681495E-4,1.3623963E-3],"split_indices":[5,61,0,9,0,4,8,0,51,0,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.5E1,6.9E1,6E0,6.2E1,7E0,3.1E1,3.1E1,8E0,2.3E1,6E0,2.5E1,5E0,1.8E1,1.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[6.1396584E-3,-2.5128212E-3,4.2842925E-3,1.2666057E-2,-5.430831E-2,6.8507185E-3,2.3976124E-3,3.4204486E-3,-6.9239372E-3,1.3082856E-2,-1.5191317E-3,-1.5151207E-3,1.3866312E-3,2.1309072E-2,-1.5428103E-2,3.8851615E-2,-2.121099E-4,-1.6908612E-3,5.831961E-4,3.0127198E-3,7.907421E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,-1,5,7,9,-1,11,-1,13,-1,-1,-1,15,17,19,-1,-1,-1,-1,-1],"loss_changes":[6.9547005E-2,6.594226E-2,0E0,1.4973585E-2,1.0960974E-1,1.5111812E-2,0E0,1.4226093E-2,0E0,1.2290162E-2,0E0,0E0,0E0,1.8440872E-2,8.019619E-3,1.3159715E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,7,7,9,9,13,13,14,14,15,15],"right_children":[2,4,-1,6,8,10,-1,12,-1,14,-1,-1,-1,16,18,20,-1,-1,-1,-1,-1],"split_conditions":[8.325696E8,1.9E1,4.2842925E-3,1.052686E6,1.7495675E5,1.8E2,2.3976124E-3,5.416164E2,-6.9239372E-3,5.557455E8,-1.5191317E-3,-1.5151207E-3,1.3866312E-3,4.784127E5,2.916545E3,8.7863175E5,-2.121099E-4,-1.6908612E-3,5.831961E-4,3.0127198E-3,7.907421E-4],"split_indices":[7,3,0,37,42,8,0,61,0,7,0,0,0,37,4,41,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E1,8.2E1,7E0,6.4E1,1.8E1,5.7E1,7E0,1.2E1,6E0,5E1,7E0,5E0,7E0,3.9E1,1.1E1,2.3E1,1.6E1,6E0,5E0,9E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[7.064008E-3,-5.8494373E-3,7.6307594E-3,7.7245045E-3,-7.4525517E-3,-3.7497733E-2,1.664021E-2,-2.9279138E-3,3.880468E-5,6.957361E-2,5.1876716E-4,4.9394867E-3,1.1606034E-3,1.2371844E-3,-8.480811E-3,-1.8240632E-2,1.2589961E-3,-3.1074073E-2,4.1654773E-4,-1.2534263E-2,-2.8691078E-3,-1.4120431E-3,8.010707E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,-1,5,-1,7,9,-1,-1,11,13,-1,-1,-1,15,17,-1,19,-1,21,-1,-1,-1],"loss_changes":[1.7085253E-1,1.6597652E-1,0E0,2.9250327E-2,0E0,1.2725117E-2,5.1165678E-2,0E0,0E0,2.1773212E-2,1.1634647E-2,0E0,0E0,0E0,1.3313116E-2,1.0518745E-2,0E0,1.1304347E-2,0E0,8.548845E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,9,9,10,10,14,14,15,15,17,17,19,19],"right_children":[2,4,-1,6,-1,8,10,-1,-1,12,14,-1,-1,-1,16,18,-1,20,-1,22,-1,-1,-1],"split_conditions":[5.661358E3,3.698723E10,7.6307594E-3,1.528E3,-7.4525517E-3,4.9382422E2,1.1349875E-2,-2.9279138E-3,3.880468E-5,6.3E2,1.7329262E7,4.9394867E-3,1.1606034E-3,1.2371844E-3,9.734147E7,1.05E2,1.2589961E-3,4.6069685E6,4.1654773E-4,1.7807747E3,-2.8691078E-3,-1.4120431E-3,8.010707E-4],"split_indices":[61,5,0,11,0,61,47,0,0,0,54,0,0,0,54,8,0,56,0,4,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.5E1,5E0,7E1,5E0,1.1E1,5.9E1,6E0,5E0,1.3E1,4.6E1,6E0,7E0,1.1E1,3.5E1,2.8E1,7E0,1.9E1,9E0,1.3E1,6E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.8093565E-2,-4.2433357E-3,-8.1631895E-3,1.24259E-2,-5.1395725E-2,4.6075745E-3,3.3414809E-3,-4.49934E-3,1.02011814E-4,3.190535E-2,-8.373953E-3,1.2541207E-4,2.4426894E-3,-1.637465E-3,4.8034284E-3,1.5661576E-3,-3.2065916E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,-1,5,7,9,-1,-1,-1,11,13,-1,-1,-1,15,-1,-1],"loss_changes":[1.732646E-1,5.7230227E-2,0E0,2.5740411E-2,4.9697023E-2,1.7631149E-2,0E0,0E0,0E0,1.0257542E-2,1.2714781E-2,0E0,0E0,0E0,8.826424E-3,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,9,9,10,10,14,14],"right_children":[2,4,-1,6,8,10,-1,-1,-1,12,14,-1,-1,-1,16,-1,-1],"split_conditions":[1.278211E8,8.533027E-1,-8.1631895E-3,1.0202749E8,1.9418182E8,2.156825E7,3.3414809E-3,-4.49934E-3,1.02011814E-4,3.8734267E0,6.797331E2,1.2541207E-4,2.4426894E-3,-1.637465E-3,1.8471765E8,1.5661576E-3,-3.2065916E-4],"split_indices":[57,36,0,54,12,54,0,0,0,65,61,0,0,0,12,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[7.6E1,7.1E1,5E0,5.3E1,1.8E1,4.8E1,5E0,9E0,9E0,1.5E1,3.3E1,7E0,8E0,1E1,2.3E1,6E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[9.5368605E-3,-4.1731982E-3,6.9828113E-3,5.9466427E-3,-5.4731374E-3,-1.5863471E-3,4.6495073E-2,-1.8167165E-3,6.044296E-3,3.6639364E-3,1.7452045E-4,2.1760953E-3,-5.4466627E-3,-1.7189546E-2,9.532077E-4,6.352475E-4,-1.1693205E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,-1,5,-1,7,9,-1,11,-1,-1,-1,13,15,-1,-1,-1],"loss_changes":[1.6226155E-1,8.980466E-2,0E0,2.1455323E-2,0E0,1.7988598E-2,1.6032357E-2,0E0,2.50412E-2,0E0,0E0,0E0,1.3177702E-2,8.332926E-3,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,6,6,8,8,12,12,13,13],"right_children":[2,4,-1,6,-1,8,10,-1,12,-1,-1,-1,14,16,-1,-1,-1],"split_conditions":[6.5317163E3,1.5092398E8,6.9828113E-3,1.3310406E3,-5.4731374E-3,1.259E3,5.303548E9,-1.8167165E-3,8.1735584E8,3.6639364E-3,1.7452045E-4,2.1760953E-3,4.6911945E0,1.2015E4,9.532077E-4,6.352475E-4,-1.1693205E-3],"split_indices":[4,54,0,61,0,11,12,0,5,0,0,0,63,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8E1,7.4E1,6E0,6.9E1,5E0,5.9E1,1E1,9E0,5E1,5E0,5E0,1E1,4E1,2.8E1,1.2E1,6E0,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"-3.1569883E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics index 607a7c5a3..c14bdca06 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/databricks-azure_photon.metrics @@ -1,116 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,input_bytesRead_mean,0.07950352,48660306.00109597,49428056.486782335,466150.2392857143,9457240.909980843,31206626.71954965,76555115.52221651,240185411.32144082 -1,executorCPUTime_mean,0.079072654,1085.2604647622206,2189.793265614089,9.16245487364621,172.2175053763441,492.0533589788088,1134.9706920150506,19830.688929551692 -2,sw_bytesWrittenRatio,0.078284755,0.5919399675594721,1.476025676540793,3.811915686539279e-07,0.004988403730711104,0.15715677052090143,0.6968701139378873,19.10476372587304 -3,sr_localBytesReadRatio,0.07264526,0.11475462581453616,0.25076456148616505,4.950703452775733e-08,0.0013396561493017212,0.03231300375934251,0.11114098070492967,2.3968573169615706 -4,duration_mean,0.062883504,1838.369521293885,2822.267657577495,48.0,445.2089864158829,956.2881858350818,2117.1486501006557,23602.61970074813 -5,cache_hit_ratio,0.046830792,0.5797031024761284,0.2958995453171238,2.3751764226164482e-05,0.34724318043450464,0.6077699084792183,0.8571339338875497,1.0 -6,scan_bw,0.033360828,682180.7475050243,709768.054847744,18541.21305321391,250700.44646224368,415709.30636541,843321.6773468265,5003420.826327434 -7,sr_totalBytesReadRatio,0.033145826,0.657924062246373,1.5241815199065611,3.811915686539279e-07,0.006417516514179832,0.19052090935635582,0.799098277696303,19.18647529336742 -8,sr_remoteBytesReadRatio,0.027683333,0.5431694364318369,1.3308894015763957,3.3168453412617064e-07,0.004778540082677,0.14424033071079345,0.597866155985421,16.78961797640585 -9,sw_writeTime_mean,0.02694475,103.69509381282296,238.56070095484813,0.0,0.657938732288993,13.45055733972235,96.60870167372909,1805.0101408450705 -10,data_size,0.02552685,668568812798.4546,1985841754832.3271,57436094.0,27908715879.25,115710188148.5,545671877091.0,22044104226233.0 -11,sr_remoteBlocksFetched_sum,0.021250235,73710.92727272728,139978.87892030226,1.0,1302.75,17643.0,70517.25,940427.0 -12,shuffle_read_bw,0.019611133,21097884.55916633,142877158.37645754,0.0,173985.0551887179,593988.9257987014,2812606.8795373663,1459791338.8333333 -13,duration_max,0.01865861,7443.127272727273,13363.593275228666,156.0,1387.75,3476.0,7819.75,121568.0 -14,duration_sum,0.014015283,2471074.709090909,4840603.443290839,240.0,134260.75,568925.0,2163656.75,31586217.0 -15,peakExecutionMemory_max,0.013457124,480456154.1818182,474651559.48118466,159385856.0,220580735.75,346536456.5,531331736.0,4222092096.0 -16,input_recordsRead_sum,0.01112748,8228875634.568182,19243603112.479984,600000.0,413535743.5,1810167936.0,7537489709.0,230400506548.0 -17,jvmGCTime_mean,0.010980322,12.723160383736214,14.422087022171816,0.0,3.3102295226222775,8.111694964818351,16.933459050062602,84.78 -18,executorDeserializeTime_mean,0.010133571,6.522922206172232,15.239475132670595,1.4,2.7173863626133223,3.2269249889527174,3.976825559667314,136.86302681992336 -19,resultSize_max,0.009981952,464635.34545454546,2610280.2584496085,9745.0,13582.0,16288.5,21372.5,33908356.0 -20,sw_bytesWritten_mean,0.008407471,16381353.803431287,34748537.32729478,49.30337459725284,90522.77177520556,2317902.8085535243,12524134.635000767,243100435.13633803 -21,sw_recordsWritten_sum,0.008123444,1993995676.168182,5072609516.6370945,200.0,1520171.75,126090791.5,1522106638.75,40948495624.0 -22,sr_localBlocksFetched_sum,0.0073375343,10735.163636363637,20529.949658652342,1.0,210.0,2536.0,10083.0,135165.0 -23,sr_fetchWaitTime_mean,0.0072551467,42.158621909106756,151.8981045105995,0.0,0.33875324808973734,2.0408804539229113,9.230470613341367,1115.2329896907218 -24,resultSerializationTime_sum,0.0070036524,75.90454545454546,130.42653936613254,0.0,3.0,25.5,99.0,1069.0 -25,shuffle_write_bw,0.006702726,163094.76249962486,120588.59781638657,0.0,79856.67746661245,156255.07795132295,206153.42535075612,795840.9797297297 -26,executorDeserializeCPUTime_mean,0.0058940696,3.6773019175887676,2.542707287649343,1.4,2.7563532651455547,3.0896527012127892,3.6717515122080444,24.409252669039144 -27,sr_localBytesRead_mean,0.005697522,2916849.994645489,5960186.011236432,7.631067961165049,44445.41409518829,469171.9910395135,2269699.288765448,34055682.08306709 -28,sr_remoteBytesRead_mean,0.004859755,14571247.101848954,31581916.287754808,51.05526590198123,270542.5920445972,1917402.5879213854,11389068.207380045,212694915.64563382 -29,sqlOp_AQEShuffleRead,0.0048106434,0.9090909090909091,0.28813538628606516,0.0,1.0,1.0,1.0,1.0 -30,duration_min,0.0029910987,17.595454545454544,22.043439415246258,5.0,11.0,12.0,16.0,268.0 -31,sqlOp_Sort,0.0027633216,0.2772727272727273,0.4486731510945088,0.0,0.0,0.0,1.0,1.0 -32,numTasks_sum,0.0022300421,957.8454545454546,1285.2573516712155,5.0,276.75,561.5,1139.0,11794.0 -33,executorRunTime_mean,0.0018716134,1820.9901176973162,2822.0779471701867,41.0,433.6194879832811,944.1120382732532,2108.383181217503,23591.60349127182 -34,scan_time,0.0015481719,1326606.9181818182,4390988.808916784,156.0,51219.75,227012.5,950299.75,58655184.0 -35,sr_totalBytesRead_mean,0.0012700245,17488097.096494444,36315663.17124838,58.75728155339806,345292.2404184723,2879392.1611024104,15671497.58876525,243101856.34140846 -36,sqlOp_Filter,0.0012403111,0.4727272727272727,0.5003941991110044,0.0,0.0,0.0,1.0,1.0 -37,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -38,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -39,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -40,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -41,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -42,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -43,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -44,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -45,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -46,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_HashAggregate,0.0,0.990909090909091,0.09512832433347063,0.0,1.0,1.0,1.0,1.0 -49,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,executorCPUTime_mean,0.07755299,1085.2604647622206,2189.793265614089,9.16245487364621,172.2175053763441,492.0533589788088,1134.9706920150506,19830.688929551692 +1,input_bytesRead_mean,0.0775263,48660306.00109597,49428056.486782335,466150.2392857143,9457240.909980843,31206626.71954965,76555115.52221651,240185411.32144082 +2,sw_bytesWrittenRatio,0.070281014,0.5919399675594721,1.476025676540793,3.811915686539279e-07,0.004988403730711104,0.15715677052090143,0.6968701139378873,19.10476372587304 +3,sr_localBytesReadRatio,0.06783049,0.11475462581453616,0.25076456148616505,4.950703452775733e-08,0.0013396561493017212,0.03231300375934251,0.11114098070492967,2.3968573169615706 +4,duration_mean,0.06341674,1838.369521293885,2822.267657577495,48.0,445.2089864158829,956.2881858350818,2117.1486501006557,23602.61970074813 +5,cache_hit_ratio,0.04834608,0.5797031024761284,0.2958995453171238,2.3751764226164482e-05,0.34724318043450464,0.6077699084792183,0.8571339338875497,1.0 +6,sr_remoteBytesReadRatio,0.03946106,0.5431694364318369,1.3308894015763957,3.3168453412617064e-07,0.004778540082677,0.14424033071079345,0.597866155985421,16.78961797640585 +7,scan_bw,0.035654936,682180.7475050243,709768.054847744,18541.21305321391,250700.44646224368,415709.30636541,843321.6773468265,5003420.826327434 +8,sr_totalBytesReadRatio,0.02907213,0.657924062246373,1.5241815199065611,3.811915686539279e-07,0.006417516514179832,0.19052090935635582,0.799098277696303,19.18647529336742 +9,data_size,0.028059693,668568812798.4546,1985841754832.3271,57436094.0,27908715879.25,115710188148.5,545671877091.0,22044104226233.0 +10,sr_remoteBlocksFetched_sum,0.027271805,73710.92727272728,139978.87892030226,1.0,1302.75,17643.0,70517.25,940427.0 +11,duration_max,0.025520394,7443.127272727273,13363.593275228666,156.0,1387.75,3476.0,7819.75,121568.0 +12,sw_writeTime_mean,0.023166072,103.69509381282296,238.56070095484813,0.0,0.657938732288993,13.45055733972235,96.60870167372909,1805.0101408450705 +13,duration_sum,0.017088735,2471074.709090909,4840603.443290839,240.0,134260.75,568925.0,2163656.75,31586217.0 +14,shuffle_read_bw,0.016677875,21097884.55916633,142877158.37645754,0.0,173985.0551887179,593988.9257987014,2812606.8795373663,1459791338.8333333 +15,input_recordsRead_sum,0.01474298,8228875634.568182,19243603112.479984,600000.0,413535743.5,1810167936.0,7537489709.0,230400506548.0 +16,sr_fetchWaitTime_mean,0.013151251,42.158621909106756,151.8981045105995,0.0,0.33875324808973734,2.0408804539229113,9.230470613341367,1115.2329896907218 +17,jvmGCTime_mean,0.01228824,12.723160383736214,14.422087022171816,0.0,3.3102295226222775,8.111694964818351,16.933459050062602,84.78 +18,peakExecutionMemory_max,0.011334074,480456154.1818182,474651559.48118466,159385856.0,220580735.75,346536456.5,531331736.0,4222092096.0 +19,resultSize_max,0.011116942,464635.34545454546,2610280.2584496085,9745.0,13582.0,16288.5,21372.5,33908356.0 +20,sr_localBytesRead_mean,0.008846231,2916849.994645489,5960186.011236432,7.631067961165049,44445.41409518829,469171.9910395135,2269699.288765448,34055682.08306709 +21,shuffle_write_bw,0.007977458,163094.76249962486,120588.59781638657,0.0,79856.67746661245,156255.07795132295,206153.42535075612,795840.9797297297 +22,sr_localBlocksFetched_sum,0.007961682,10735.163636363637,20529.949658652342,1.0,210.0,2536.0,10083.0,135165.0 +23,resultSerializationTime_sum,0.007957702,75.90454545454546,130.42653936613254,0.0,3.0,25.5,99.0,1069.0 +24,sw_recordsWritten_sum,0.00792488,1993995676.168182,5072609516.6370945,200.0,1520171.75,126090791.5,1522106638.75,40948495624.0 +25,executorDeserializeCPUTime_mean,0.006787035,3.6773019175887676,2.542707287649343,1.4,2.7563532651455547,3.0896527012127892,3.6717515122080444,24.409252669039144 +26,executorDeserializeTime_mean,0.006378709,6.522922206172232,15.239475132670595,1.4,2.7173863626133223,3.2269249889527174,3.976825559667314,136.86302681992336 +27,numTasks_sum,0.0056990455,957.8454545454546,1285.2573516712155,5.0,276.75,561.5,1139.0,11794.0 +28,sr_remoteBytesRead_mean,0.0052060043,14571247.101848954,31581916.287754808,51.05526590198123,270542.5920445972,1917402.5879213854,11389068.207380045,212694915.64563382 +29,sqlOp_AQEShuffleRead,0.0040279194,0.9090909090909091,0.28813538628606516,0.0,1.0,1.0,1.0,1.0 +30,duration_min,0.0037214016,17.595454545454544,22.043439415246258,5.0,11.0,12.0,16.0,268.0 +31,scan_time,0.0035636288,1326606.9181818182,4390988.808916784,156.0,51219.75,227012.5,950299.75,58655184.0 +32,sqlOp_Sort,0.0031821313,0.2772727272727273,0.4486731510945088,0.0,0.0,0.0,1.0,1.0 +33,sw_bytesWritten_mean,0.0025342433,16381353.803431287,34748537.32729478,49.30337459725284,90522.77177520556,2317902.8085535243,12524134.635000767,243100435.13633803 +34,sqlOp_SortMergeJoin,0.0013597038,0.5590909090909091,0.49762824776768727,0.0,0.0,1.0,1.0,1.0 +35,sqlOp_Filter,0.0013540883,0.4727272727272727,0.5003941991110044,0.0,0.0,0.0,1.0,1.0 +36,sr_totalBytesRead_mean,0.0011417965,17488097.096494444,36315663.17124838,58.75728155339806,345292.2404184723,2879392.1611024104,15671497.58876525,243101856.34140846 +37,sqlOp_Subquery,0.00096312945,0.2818181818181818,0.45091116645433477,0.0,0.0,0.0,1.0,1.0 +38,sqlOp_TakeOrderedAndProject,0.0008282325,0.6181818181818182,0.4869402780332598,0.0,0.0,1.0,1.0,1.0 +39,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +40,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 +41,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +42,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 +43,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +44,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +45,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +46,sqlOp_BroadcastHashJoin,0.0,0.9045454545454545,0.2945118594681246,0.0,1.0,1.0,1.0,1.0 +47,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +48,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +49,sqlOp_HashAggregate,0.0,0.990909090909091,0.09512832433347063,0.0,1.0,1.0,1.0,1.0 +50,sqlOp_Project,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +51,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 55,diskBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_Window,0.0,0.11818181818181818,0.32355948919311184,0.0,0.0,0.0,0.0,1.0 -60,sqlOp_TakeOrderedAndProject,0.0,0.6181818181818182,0.4869402780332598,0.0,0.0,1.0,1.0,1.0 -61,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Subquery,0.0,0.2818181818181818,0.45091116645433477,0.0,0.0,0.0,1.0,1.0 -64,sqlOp_SortMergeJoin,0.0,0.5590909090909091,0.49762824776768727,0.0,0.0,1.0,1.0,1.0 -65,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,executorOffHeap,0.0,35262.0,0.0,35262.0,35262.0,35262.0,35262.0,35262.0 -78,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 -79,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 -80,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 -81,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,executorMemory,0.0,11754.0,0.0,11754.0,11754.0,11754.0,11754.0,11754.0 -83,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -87,diskBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -89,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -92,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -94,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -95,sqlOp_Scan parquet,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -96,memoryBytesSpilledRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_BroadcastHashJoin,0.0,0.9045454545454545,0.2945118594681246,0.0,1.0,1.0,1.0,1.0 -98,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_ColumnarToRow,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -103,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_BroadcastNestedLoopJoin,0.0,0.03636363636363636,0.18762017042954973,0.0,0.0,0.0,0.0,1.0 -105,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -107,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -108,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -109,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -111,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -112,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -113,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +59,executorCores,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +60,maxMem,0.0,43359456460.0,0.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0,43359456460.0 +61,maxOnHeapMem,0.0,6384569548.0,0.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0,6384569548.0 +62,maxOffHeapMem,0.0,36974886912.0,0.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0,36974886912.0 +63,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +64,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,diskBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,memoryBytesSpilled_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,executorRunTime_mean,0.0,1820.9901176973162,2822.0779471701867,41.0,433.6194879832811,944.1120382732532,2108.383181217503,23591.60349127182 +68,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sqlOp_BroadcastExchange,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +75,sqlOp_BroadcastNestedLoopJoin,0.0,0.03636363636363636,0.18762017042954973,0.0,0.0,0.0,0.0,1.0 +76,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_SubqueryBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_Window,0.0,0.11818181818181818,0.32355948919311184,0.0,0.0,0.0,0.0,1.0 +113,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg index dfa5fa97f..c87e812a4 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0457615778","gamma":"0.00797297619","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0457615778","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"5","max_leaves":"0","min_child_weight":"3","min_split_loss":"0.00797297619","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.819992304"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"78"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0477306731","gamma":"0.0901460424","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0477306731","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"8","max_leaves":"0","min_child_weight":"1","min_split_loss":"0.0901460424","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.711684704"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json index 0f272b640..479358f0d 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortAggregate","sqlOp_SortMergeJoin","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_dataproc","platform_databricks-aws","platform_databricks-azure","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-2.9023793E-3,-2.8369835E-1,5.416892E-1,-6.1223924E-1,-1.8412052E-1,6.33682E-1,-5.82664E-1,-6.5558666E-1,-3.0437765E-1,-2.173319E-1,3.440377E-1,3.2511374E-1,8.968922E-1,-4.689302E-2,-5.7365548E-2,-3.0585064E-2,-1.3588668E-2,-1.7759481E-2,-2.785241E-3,-3.3582267E-1,-1.2250839E-1,2.1855064E-2,9.24113E-3,4.668868E-2,2.8010365E-1,7.8835046E-1,1.3677841E0,-9.834596E-3,5.2406043E-3,-1.7206868E-2,-3.431305E-3,-8.125904E-3,6.5788818E-3,-3.0713265E-3,1.4952179E-2,3.757484E-2,2.328943E-3,2.491815E-2,7.060135E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,-1,-1,29,31,-1,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.4166534E1,1.0356199E1,1.7435574E1,7.8798485E-1,4.407364E0,1.2134808E1,2.9599957E0,9.464836E-2,2.006079E-1,2.5767794E0,1.8820691E-1,1.9669976E0,3.1128464E0,0E0,2.1416774E-1,0E0,0E0,0E0,0E0,1.0515356E0,1.9461876E0,0E0,0E0,0E0,1.1392713E0,1.6246643E0,1.1771832E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,19,19,20,20,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,-1,-1,30,32,-1,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0434409E3,3.5002026E8,1.4777102E9,3.9837E4,1.6857977E4,2.5050928E6,3.49399E5,7.3822034E2,6.1614815E6,8.504733E7,1.152089E1,-4.689302E-2,1.6130411E5,-3.0585064E-2,-1.3588668E-2,-1.7759481E-2,-2.785241E-3,7.164074E8,2.103E3,2.1855064E-2,9.24113E-3,4.668868E-2,3.0811954E-2,5.22E2,6.069097E4,-9.834596E-3,5.2406043E-3,-1.7206868E-2,-3.431305E-3,-8.125904E-3,6.5788818E-3,-3.0713265E-3,1.4952179E-2,3.757484E-2,2.328943E-3,2.491815E-2,7.060135E-2],"split_indices":[2,1,67,7,7,2,42,59,1,61,54,7,62,0,37,0,0,0,0,7,0,0,0,0,47,3,37,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,3.19E2,1.64E2,7.3E1,2.46E2,1.52E2,1.2E1,6.3E1,1E1,2.32E2,1.4E1,7.1E1,8.1E1,6E0,6E0,6E1,3E0,7E0,3E0,1.02E2,1.3E2,6E0,8E0,3E0,6.8E1,6.8E1,1.3E1,3E0,3E0,8.8E1,1.4E1,1.08E2,2.2E1,8E0,6E1,6.5E1,3E0,3E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.4164956E-2,-2.9536834E-1,5.3674215E-1,-5.921586E-1,-1.9601977E-1,3.9247462E-1,1.0593588E0,-6.143498E-1,-1.5368152E-3,-2.2959144E-1,3.0719486E-1,4.3075106E-1,-4.871125E-2,1.9363787E-3,1.1743835E0,-2.92901E-2,-1.4746343E-2,-2.74807E-1,-5.3567626E-2,3.8989517E-1,-8.1020093E-4,2.8436318E-1,6.504831E-1,7.007735E-1,1.3357966E0,-1.356109E-2,1.8585974E-3,-3.6748396E-3,1.1314127E-2,2.4594612E-2,9.5055625E-3,5.630435E-3,1.6241182E-2,-2.5435318E-3,3.2984715E-2,3.6877707E-2,1.4432959E-2,3.6415048E-2,6.7015715E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,23,-1,-1,25,27,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.958027E1,8.995007E0,1.3381805E1,9.33815E-1,4.006776E0,8.544607E0,4.5127296E0,3.4947205E-1,0E0,1.728303E0,4.0994263E-1,4.382612E0,0E0,0E0,1.7395477E0,0E0,0E0,1.1939669E0,3.898111E-1,2.1093476E-1,0E0,9.4645596E-1,2.8201733E0,1.913395E-1,5.716591E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,14,14,17,17,18,18,19,19,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,24,-1,-1,26,28,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.0500806E3,1.2852292E7,1.4096699E9,4.673502E7,5.3140095E1,2.9778808E7,-1.5368152E-3,9.570385E4,1.5E2,4.1331047E-1,-4.871125E-2,1.9363787E-3,6.069097E4,-2.92901E-2,-1.4746343E-2,2.9063344E5,1.4998456E5,6.088569E6,-8.1020093E-4,1.2587E4,2.7011695E4,5.3412586E4,9.430977E3,-1.356109E-2,1.8585974E-3,-3.6748396E-3,1.1314127E-2,2.4594612E-2,9.5055625E-3,5.630435E-3,1.6241182E-2,-2.5435318E-3,3.2984715E-2,3.6877707E-2,1.4432959E-2,3.6415048E-2,6.7015715E-2],"split_indices":[2,1,61,9,7,57,65,12,0,42,3,44,0,0,37,0,0,37,37,54,0,2,42,37,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.9E2,3.08E2,1.82E2,7.6E1,2.32E2,1.44E2,3.8E1,7.3E1,3E0,2.18E2,1.4E1,1.41E2,3E0,4E0,3.4E1,6.6E1,7E0,1.73E2,4.5E1,1.1E1,3E0,8.6E1,5.5E1,1E1,2.4E1,1.62E2,1.1E1,4.2E1,3E0,5E0,6E0,2.7E1,5.9E1,5E0,5E1,7E0,3E0,6E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.0740048E-3,-2.7225897E-1,4.8837975E-1,-5.59318E-1,-1.6748437E-1,3.4900758E-1,9.853364E-1,-5.7867086E-1,-1.4840708E-3,-2.0295937E-1,2.889353E-1,4.152262E-1,-6.536197E-1,1.8654893E-3,1.0952635E0,-2.7665831E-2,-1.5419739E-2,-2.564603E-1,-1.7569283E-2,-1.8276778E-3,3.6226118E-1,2.680009E-1,6.6421986E-1,-2.590668E-3,-5.124853E-2,6.884409E-2,8.6210287E-1,-4.7414824E-3,-1.4298119E-2,-2.3293286E-3,1.3084776E-2,2.1624502E-2,8.441571E-3,5.6841597E-3,1.6624851E-2,2.2817368E-2,3.6455166E-2,4.26724E-2,1.2677028E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,-1,31,33,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.510349E1,9.290188E0,1.18462105E1,8.2756805E-1,3.7826843E0,9.499968E0,3.9010353E0,2.9699516E-1,0E0,2.118062E0,4.2442358E-1,4.6242085E0,2.4419715E0,0E0,1.9465256E0,0E0,0E0,1.3881884E0,5.102579E-1,0E0,1.873424E-1,1.1058311E0,6.553459E-1,0E0,0E0,0E0,7.3010254E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,20,20,21,21,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,-1,32,34,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.508E3,7.41718E5,6.0500806E3,1.2852292E7,1.4777102E9,8.625407E2,5.3140095E1,2.9778808E7,-1.4840708E-3,9.570385E4,2.0532622E3,4.1331047E-1,1.805424E-1,1.8654893E-3,1.417988E6,-2.7665831E-2,-1.5419739E-2,7.556039E6,1.4998456E5,-1.8276778E-3,6.1614815E6,1.3664E4,1.30817086E5,-2.590668E-3,-5.124853E-2,6.884409E-2,1.5096262E7,-4.7414824E-3,-1.4298119E-2,-2.3293286E-3,1.3084776E-2,2.1624502E-2,8.441571E-3,5.6841597E-3,1.6624851E-2,2.2817368E-2,3.6455166E-2,4.26724E-2,1.2677028E-2],"split_indices":[2,1,61,9,7,67,65,12,0,42,4,44,47,0,38,0,0,54,37,0,54,2,37,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,3.11E2,1.75E2,8.2E1,2.29E2,1.38E2,3.7E1,7.9E1,3E0,2.13E2,1.6E1,1.3E2,8E0,4E0,3.3E1,7E1,9E0,1.65E2,4.8E1,3E0,1.3E1,8.3E1,4.7E1,4E0,4E0,1E1,2.3E1,4.5E1,1.2E2,4.4E1,4E0,7E0,6E0,3.4E1,4.9E1,2.3E1,2.4E1,2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.8712027E-3,-2.660465E-1,5.154369E-1,-5.40148E-1,-1.702005E-1,3.6596426E-1,9.995473E-1,-5.58607E-1,-1.4331349E-3,-2.0318787E-1,2.077682E-1,4.435388E-1,-2.73858E-1,7.373477E-1,6.1629973E-2,-2.664765E-2,-1.3922579E-2,-2.546438E-1,-2.3648268E-2,-3.8227865E-3,3.1043908E-1,2.9789838E-1,6.6030926E-1,-3.8902644E-2,1.2973577E-2,4.0914126E-2,2.0661907E-1,-5.9421887E-3,-1.3482051E-2,-2.90974E-3,1.0821002E-2,1.9681184E-2,5.1374417E-3,1.5984075E-2,5.558315E-3,2.1808362E-2,3.3853136E-2,1.158259E-2,-4.974969E-3,2.2245297E-2,-5.6986716E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,-1,31,33,35,-1,37,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.87632E1,8.472586E0,1.2080338E1,7.7147484E-1,3.0719876E0,6.7227764E0,2.648552E0,3.113041E-1,0E0,2.0633755E0,6.2410456E-1,3.5601883E0,2.4903636E0,1.8947144E0,0E0,0E0,0E0,8.296776E-1,5.44824E-1,0E0,3.0736482E-1,6.194935E-1,3.3368874E-1,0E0,3.4895334E-1,0E0,7.0841384E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,20,20,21,21,22,22,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,-1,32,34,36,-1,38,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,6.0500806E3,1.2730462E7,1.1576422E9,3.5301748E7,2.1678066E0,2.9778808E7,-1.4331349E-3,9.570385E4,2.1530056E3,4.1331047E-1,2.2782404E0,4.42384E5,6.1629973E-2,-2.664765E-2,-1.3922579E-2,7.556039E6,1.4998456E5,-3.8227865E-3,1.21E2,1.7458724E7,5.2060513E9,-3.8902644E-2,9.8E1,4.0914126E-2,1.4609149E4,-5.9421887E-3,-1.3482051E-2,-2.90974E-3,1.0821002E-2,1.9681184E-2,5.1374417E-3,1.5984075E-2,5.558315E-3,2.1808362E-2,3.3853136E-2,1.158259E-2,-4.974969E-3,2.2245297E-2,-5.6986716E-3],"split_indices":[2,1,61,9,7,54,51,12,0,42,4,44,63,2,0,0,0,54,37,0,3,54,12,0,3,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.96E2,3.25E2,1.71E2,8.3E1,2.42E2,1.32E2,3.9E1,8E1,3E0,2.23E2,1.9E1,1.18E2,1.4E1,2.4E1,1.5E1,7.2E1,8E0,1.73E2,5E1,5E0,1.4E1,7.2E1,4.6E1,4E0,1E1,1.8E1,6E0,4.3E1,1.3E2,4.4E1,6E0,8E0,6E0,5.5E1,1.7E1,1.6E1,3E1,3E0,7E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-2.3861953E-4,-2.4855125E-1,4.3679297E-1,-5.1567703E-1,-1.6337739E-1,3.1119695E-1,8.8141024E-1,-5.410147E-1,-2.3901467E-1,-1.8728054E-1,2.964297E-1,3.867461E-1,-3.8557732E-1,7.0723897E-4,9.828328E-1,-2.519192E-2,-1.162049E-2,-2.6608412E-3,-1.5371629E-2,-2.3148832E-1,-2.0362688E-2,4.0522805E-1,-1.0284487E-3,2.6468137E-1,5.9786826E-1,-1.20987885E-1,-4.6530556E-2,6.0209725E-2,7.893051E-1,-1.1407239E-2,2.72727E-3,-2.516094E-3,1.0577376E-2,2.2192454E-2,8.435039E-3,1.942211E-3,1.4783302E-2,1.5235469E-2,3.234325E-2,1.23613E-3,-1.3911094E-2,3.9697077E-2,8.309422E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,-1,-1,29,31,33,-1,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.241543E1,6.9234066E0,9.498085E0,3.8942719E-1,2.6383624E0,7.413537E0,3.3222523E0,4.2722702E-2,1.2066671E-1,1.6481972E0,4.2546165E-1,3.0630798E0,2.2152016E0,0E0,1.1519089E0,0E0,0E0,0E0,0E0,9.312887E-1,4.3064728E-1,6.912291E-2,0E0,1.0370159E0,1.0631237E0,3.0614066E-1,0E0,0E0,9.8263836E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,19,19,20,20,21,21,23,23,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,-1,-1,30,32,34,-1,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,5.932891E3,5.609174E8,1.4096699E9,3.3503532E7,5.3140095E1,2.5050928E6,1.93E2,9.570385E4,1.5E2,4.1331047E-1,1.6244629E-1,7.0723897E-4,1.417988E6,-2.519192E-2,-1.162049E-2,-2.6608412E-3,-1.5371629E-2,2.9063344E5,1.4998456E5,7.200105E8,-1.0284487E-3,4.3290512E1,1E0,9.0301914E4,-4.6530556E-2,6.0209725E-2,4.42384E5,-1.1407239E-2,2.72727E-3,-2.516094E-3,1.0577376E-2,2.2192454E-2,8.435039E-3,1.942211E-3,1.4783302E-2,1.5235469E-2,3.234325E-2,1.23613E-3,-1.3911094E-2,3.9697077E-2,8.309422E-3],"split_indices":[2,1,61,7,7,54,65,59,10,42,3,44,47,0,38,0,0,0,0,37,37,5,0,65,109,37,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.81E2,3.07E2,1.74E2,7.3E1,2.34E2,1.37E2,3.7E1,6.6E1,7E0,2.23E2,1.1E1,1.24E2,1.3E1,4E0,3.3E1,6.3E1,3E0,3E0,4E0,1.76E2,4.7E1,8E0,3E0,8E1,4.4E1,1E1,3E0,1E1,2.3E1,1.66E2,1E1,4.2E1,5E0,5E0,3E0,1.7E1,6.3E1,1.4E1,3E1,6E0,4E0,2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.0015702E-2,-2.0717254E-1,5.2670485E-1,-4.690694E-1,-1.301758E-1,5.990116E-1,-4.085785E-1,-4.9186516E-1,-1.7333592E-3,-1.73698E-1,2.4907511E-1,4.7531593E-1,1.0347505E0,-1.204135E-1,-4.0396668E-2,-2.368167E-2,-1.3636736E-2,-2.891477E-1,-9.6596144E-2,1.14809126E-1,3.5778505E-1,3.2097438E-1,6.1540294E-1,1.1434951E0,1.2999823E-2,-1.1686763E-2,3.5878299E-3,-7.501257E-3,-1.5849506E-2,-6.639347E-3,1.0761006E-2,8.554554E-3,-5.1365593E-3,1.8764602E-2,9.657683E-3,1.7274572E-2,9.924703E-3,1.9247489E-2,3.1643283E-2,6.4290665E-2,4.006708E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.498412E1,6.895934E0,1.0050701E1,7.480583E-1,4.4668274E0,6.7845345E0,1.396783E0,2.1357155E-1,0E0,2.117303E0,3.6761224E-1,2.1159458E0,2.0424767E0,2.3469688E-1,0E0,0E0,0E0,6.308985E-1,2.382728E0,2.5026268E-1,2.850604E-2,2.2993517E-1,5.242615E-1,5.2093124E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3664E4,7.32424E5,1.0434409E3,1.2852292E7,1.4777102E9,7.710634E3,7.339209E6,2.4434292E7,-1.7333592E-3,7.3822034E2,1.0284004E2,4.1331047E-1,1.92E2,1.6130411E5,-4.0396668E-2,-2.368167E-2,-1.3636736E-2,7.37365E6,2.3171705E5,5.78E2,5.88824E7,1.9470884E5,1E0,3.3616E4,1.2999823E-2,-1.1686763E-2,3.5878299E-3,-7.501257E-3,-1.5849506E-2,-6.639347E-3,1.0761006E-2,8.554554E-3,-5.1365593E-3,1.8764602E-2,9.657683E-3,1.7274572E-2,9.924703E-3,1.9247489E-2,3.1643283E-2,6.4290665E-2,4.006708E-2],"split_indices":[2,1,67,9,7,61,56,12,0,61,63,44,8,37,0,0,0,54,37,8,9,42,109,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,3.44E2,1.44E2,7.7E1,2.67E2,1.34E2,1E1,7.3E1,4E0,2.4E2,2.7E1,1.06E2,2.8E1,7E0,3E0,6.3E1,1E1,9.5E1,1.45E2,1.3E1,1.4E1,5.2E1,5.4E1,2.4E1,4E0,4E0,3E0,3.1E1,6.4E1,1.27E2,1.8E1,1E1,3E0,9E0,5E0,3.2E1,2E1,1.7E1,3.7E1,1E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[7.730294E-3,-2.3478466E-1,4.0877026E-1,-4.757759E-1,-1.4981993E-1,2.9193845E-1,8.9112264E-1,-2.2545144E-2,-1.0712231E-2,-1.7569044E-1,2.835931E-1,3.6758724E-1,-2.7483538E-1,9.665825E-1,2.9591017E-3,-2.1047246E-1,2.0142893E-4,7.411203E-4,3.6219293E-1,2.4485473E-1,5.890527E-1,-7.604021E-2,-4.3547105E-2,5.365999E-1,1.0875517E0,-1.0092564E-2,1.2830353E-2,-8.791006E-3,2.062296E-3,2.070844E-2,8.298909E-3,7.517647E-3,1.636306E-2,3.6232192E-2,2.1298708E-2,9.6565543E-4,-1.5945915E-2,1.1958839E-2,3.2241344E-2,5.2912723E-2,2.081667E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,25,27,-1,29,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.697442E1,6.091921E0,9.996294E0,2.0306778E-1,2.5614514E0,6.446636E0,2.1202526E0,0E0,0E0,1.2970319E0,2.673607E-1,3.416481E0,2.3493457E0,1.0808907E0,0E0,9.370804E-1,3.1925622E-1,0E0,8.129883E-2,7.223387E-1,7.842388E-1,4.0429926E-1,0E0,2.3196745E-1,5.174198E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,26,28,-1,30,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.651788E3,3.5002026E8,1.4096699E9,3.5301748E7,1.4656219E8,-2.2545144E-2,-1.0712231E-2,4.1619555E8,1E0,4.1331047E-1,1.6244629E-1,6.274818E4,2.9591017E-3,3.7069206E5,1.667732E1,7.411203E-4,7.200105E8,2.0663544E9,1.5990765E6,2.5503985E9,-4.3547105E-2,2.1152368E-1,2.7674397E5,-1.0092564E-2,1.2830353E-2,-8.791006E-3,2.062296E-3,2.070844E-2,8.298909E-3,7.517647E-3,1.636306E-2,3.6232192E-2,2.1298708E-2,9.6565543E-4,-1.5945915E-2,1.1958839E-2,3.2241344E-2,5.2912723E-2,2.081667E-2],"split_indices":[2,1,61,7,7,54,1,0,0,12,13,44,47,37,0,37,63,0,5,12,56,7,0,47,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.81E2,3E2,1.81E2,7.7E1,2.23E2,1.47E2,3.4E1,7.1E1,6E0,2.11E2,1.2E1,1.3E2,1.7E1,3.1E1,3E0,1.76E2,3.5E1,3E0,9E0,8.5E1,4.5E1,1.4E1,3E0,8E0,2.3E1,1.73E2,3E0,6E0,2.9E1,5E0,4E0,5.1E1,3.4E1,1.5E1,3E1,1.1E1,3E0,4E0,4E0,2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[6.199969E-3,-2.1495649E-1,4.3669865E-1,-4.456822E-1,-1.3016799E-1,3.0858037E-1,8.2915133E-1,-4.607872E-1,-6.0771324E-4,-1.8291743E-1,5.6865532E-2,3.908224E-1,-2.9668045E-1,-3.817022E-3,9.0339077E-1,-2.2161864E-2,-1.1836359E-2,-2.673573E-1,-7.418479E-2,-2.7957319E-3,2.2752598E-1,2.5294927E-1,5.751282E-1,-8.239187E-2,-4.205252E-2,5.4154027E-1,1.0132823E0,-1.2937103E-2,1.6283687E-3,-9.969236E-3,1.5046574E-3,-1.9762784E-3,9.132046E-3,-1.7538348E-3,1.3889329E-2,3.8052371E-3,1.35560455E-2,1.4653872E-2,2.9002858E-2,9.411755E-3,-8.666199E-3,2.8682448E-2,1.0784881E-2,4.8947424E-2,1.8664401E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7412033E1,6.3781767E0,8.24152E0,5.686474E-1,2.3979526E0,6.5372944E0,2.8530254E0,2.7854538E-1,0E0,1.7154446E0,5.5045426E-1,2.7513561E0,2.057804E0,0E0,9.623108E-1,0E0,0E0,5.0295544E-1,1.2992272E0,3.4371802E-1,2.9445767E-1,4.677658E-1,5.147047E-1,4.3958727E-1,0E0,1.3907385E-1,5.187397E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,6.0500806E3,1.2852292E7,7.164074E8,3.5301748E7,5.0314346E1,2.9778808E7,-6.0771324E-4,2.735322E6,1.484798E9,4.1331047E-1,1.6244629E-1,-3.817022E-3,6.274818E4,-2.2161864E-2,-1.1836359E-2,5.368828E8,7.335186E2,1E0,2.0532622E3,4.3290512E1,5.4730445E9,1.869E4,-4.205252E-2,2.36316E5,8.75E3,-1.2937103E-2,1.6283687E-3,-9.969236E-3,1.5046574E-3,-1.9762784E-3,9.132046E-3,-1.7538348E-3,1.3889329E-2,3.8052371E-3,1.35560455E-2,1.4653872E-2,2.9002858E-2,9.411755E-3,-8.666199E-3,2.8682448E-2,1.0784881E-2,4.8947424E-2,1.8664401E-2],"split_indices":[2,1,61,9,7,54,65,12,0,1,7,44,47,0,37,0,0,41,61,108,4,65,5,9,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.96E2,3.28E2,1.68E2,8.7E1,2.41E2,1.28E2,4E1,8.4E1,3E0,1.88E2,5.3E1,1.13E2,1.5E1,3E0,3.7E1,7.4E1,1E1,1.05E2,8.3E1,4E1,1.3E1,6.6E1,4.7E1,1.2E1,3E0,1E1,2.7E1,1E2,5E0,3.5E1,4.8E1,3.4E1,6E0,3E0,1E1,1.4E1,5.2E1,1E1,3.7E1,3E0,9E0,7E0,3E0,2.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[8.667423E-3,-1.7574525E-1,4.3931806E-1,-2.5154382E-1,7.320554E-2,5.05041E-1,-3.5310543E-1,-3.631672E-1,-1.1791023E-1,-1.2458875E-2,2.3019427E-1,3.9639425E-1,8.6726046E-1,-3.3332396E-2,1.9013008E-2,-4.4247004E-1,-2.8738716E-1,1.9672122E-2,-2.1241419E-1,-5.0359637E-2,1.3689142E-2,9.570006E-2,3.1950194E-1,4.743034E-1,2.5751093E-1,6.672631E-1,5.16336E-2,9.170901E-3,-7.6482864E-3,-2.0824073E-2,-5.6830035E-3,-2.6222907E-3,-1.384605E-2,-9.850916E-4,1.2891531E-2,-1.19486E-2,5.233007E-4,-1.1673393E-3,-1.4806172E-2,-4.036887E-3,8.568495E-3,1.6968098E-2,8.843776E-3,1.6790805E-2,2.7734455E-2,1.3933957E-2,-1.1585491E-2,3.32709E-2,1.5673168E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,-1,39,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9468254E1,6.5852146E0,7.9383106E0,3.9399204E0,1.1056324E0,5.0870705E0,1.6896647E0,7.4422836E-1,1.6034341E0,6.5279675E-1,3.177532E-1,1.0464516E0,8.6564827E-1,0E0,2.6985428E-1,2.198906E-1,2.5163603E-1,5.5664295E-1,7.986517E-1,3.2440352E-1,0E0,2.3538598E-1,3.8420916E-2,7.397547E-1,1.0607395E0,1.3931656E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,-1,40,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.164074E8,1.0434409E3,1.732734E6,1.5522031E9,4.026531E9,3.5226266E0,7.41718E5,1.4074289E7,3.5921E4,1.0284004E2,2.228899E6,8.010436E0,-3.3332396E-2,1.8431219E4,1.2254369E7,1.0885876E4,3.0224E4,1.4854412E3,1E0,1.3689142E-2,2.6978243E1,6.3349745E6,3.9417793E3,1.92144E5,4.547513E7,5.16336E-2,9.170901E-3,-7.6482864E-3,-2.0824073E-2,-5.6830035E-3,-2.6222907E-3,-1.384605E-2,-9.850916E-4,1.2891531E-2,-1.19486E-2,5.233007E-4,-1.1673393E-3,-1.4806172E-2,-4.036887E-3,8.568495E-3,1.6968098E-2,8.843776E-3,1.6790805E-2,2.7734455E-2,1.3933957E-2,-1.1585491E-2,3.32709E-2,1.5673168E-2],"split_indices":[2,7,67,1,7,7,63,1,54,10,63,9,62,0,42,9,37,10,61,18,0,67,54,61,2,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.95E2,3.47E2,1.48E2,2.66E2,8.1E1,1.37E2,1.1E1,1.44E2,1.22E2,5.3E1,2.8E1,1.07E2,3E1,5E0,6E0,6.8E1,7.6E1,5E1,7.2E1,4.8E1,5E0,1.2E1,1.6E1,6.7E1,4E1,1.9E1,1.1E1,3E0,3E0,6.5E1,3E0,5E0,7.1E1,4.4E1,6E0,5.9E1,1.3E1,4.5E1,3E0,4E0,8E0,1E1,6E0,3.9E1,2.8E1,3.7E1,3E0,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-4.881382E-3,-2.1856993E-1,3.0547476E-1,-2.6319763E-1,1.4807124E-1,9.048142E-2,5.8120775E-1,-4.1696098E-1,-2.0143463E-1,-1.2453343E-2,2.0092487E-1,-7.203054E-2,1.7710301E-1,5.090547E-1,3.8822487E-2,-4.3874806E-1,3.1370989E-3,-1.8439807E-1,-4.2581044E-2,2.387004E-1,-4.3403604E-3,-2.4853633E-1,-4.9107214E-3,8.873447E-2,2.7845615E-1,4.6070018E-1,8.3225906E-1,-2.113263E-2,-1.2498952E-2,-3.5042083E-3,-1.0318171E-2,-1.6993012E-3,1.2643617E-2,-1.4770551E-2,-1.7357897E-3,2.9479966E-3,-6.86496E-3,-4.876503E-5,8.821443E-3,1.8450662E-2,5.1678466E-3,2.209901E-2,3.0796474E-4,1.7308196E-2,4.7090225E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,27,-1,29,-1,31,-1,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2364388E1,4.7676945E0,1.1744282E1,2.3662052E0,7.65381E-1,1.6034627E0,1.245163E0,8.022671E-1,2.1386704E0,0E0,3.466556E-1,4.7266102E-1,6.38067E-1,7.261925E-1,0E0,1.4109421E-1,0E0,7.932811E-1,0E0,2.7986443E-1,0E0,1.5973514E-1,3.1147477E-1,3.8324252E-1,6.564634E-1,6.422186E-1,3.255329E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,15,15,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,28,-1,30,-1,32,-1,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,1.4777102E9,2.5096E4,7.41718E5,3.0972284E1,1.1021018E3,8.625562E0,8.332E3,5.825042E6,-1.2453343E-2,9.705148E-1,9.274153E-2,1.153215E9,4.2875492E1,3.8822487E-2,2.7299898E8,3.1370989E-3,7.2123305E6,-4.2581044E-2,1.9265735E3,-4.3403604E-3,9.407814E6,2.0421524E6,7.959718E4,1.1532659E7,1.2042546E4,1.3590209E7,-2.113263E-2,-1.2498952E-2,-3.5042083E-3,-1.0318171E-2,-1.6993012E-3,1.2643617E-2,-1.4770551E-2,-1.7357897E-3,2.9479966E-3,-6.86496E-3,-4.876503E-5,8.821443E-3,1.8450662E-2,5.1678466E-3,2.209901E-2,3.0796474E-4,1.7308196E-2,4.7090225E-2],"split_indices":[42,7,2,1,65,61,44,2,56,0,43,47,7,63,0,7,0,54,0,64,0,1,56,37,54,61,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,2.88E2,1.98E2,2.57E2,3.1E1,1.12E2,8.6E1,7.2E1,1.85E2,3E0,2.8E1,3.9E1,7.3E1,7E1,1.6E1,6.9E1,3E0,1.82E2,3E0,2.5E1,3E0,1E1,2.9E1,4E1,3.3E1,6.3E1,7E0,5.9E1,1E1,5.1E1,1.31E2,3E0,2.2E1,7E0,3E0,2E1,9E0,2.2E1,1.8E1,1.8E1,1.5E1,6E1,3E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[3.4506149E-3,-2.0101179E-1,3.323031E-1,-3.9186963E-1,-1.3679254E-1,2.3629071E-1,7.337922E-1,-4.0725446E-1,-6.6127335E-3,-1.8223394E-1,4.7946773E-2,3.088444E-1,-2.9547727E-1,-5.1799896E-3,8.142212E-1,-1.8988257E-2,-1.019718E-2,-2.1545196E-1,2.791601E-2,-2.5366267E-2,1.8699412E-1,2.0485462E-1,4.6296114E-1,-8.198047E-2,-4.0127054E-2,8.6628276E-1,1.00768935E-2,-1.2250847E-2,-6.7228545E-3,-4.4408785E-3,4.5660753E-3,-4.0684724E-3,3.178166E-3,-3.3128096E-3,1.1551198E-2,6.237513E-3,1.329668E-2,2.6013503E-2,1.7053887E-2,-8.436664E-3,1.1241659E-2,5.0267298E-2,2.9766614E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3147877E1,3.6724463E0,7.112171E0,2.3038197E-1,1.9311495E0,6.0328426E0,2.544571E0,8.156776E-3,0E0,1.2904758E0,4.736666E-1,2.084772E0,2.286507E0,0E0,8.297806E-1,0E0,0E0,5.3018713E-1,2.4316482E-1,1.9293779E-1,2.8980893E-1,4.4805288E-1,3.025341E-1,5.4844683E-1,0E0,7.886791E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.651788E3,3.5002026E8,7.164074E8,3.5301748E7,5.3140095E1,2.4434292E7,-6.6127335E-3,4.1619555E8,1.2538845E9,2.7371603E-1,1.6244629E-1,-5.1799896E-3,1.93E2,-1.8988257E-2,-1.019718E-2,7.898574E2,7.424E3,9.470162E-2,2.0532622E3,7.228959E6,1.1532659E7,3.2996133E3,-4.0127054E-2,3.3616E4,1.00768935E-2,-1.2250847E-2,-6.7228545E-3,-4.4408785E-3,4.5660753E-3,-4.0684724E-3,3.178166E-3,-3.3128096E-3,1.1551198E-2,6.237513E-3,1.329668E-2,2.6013503E-2,1.7053887E-2,-8.436664E-3,1.1241659E-2,5.0267298E-2,2.9766614E-2],"split_indices":[2,1,61,7,7,54,65,12,0,12,7,44,47,0,8,0,0,61,10,47,4,1,54,61,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.91E2,3.03E2,1.88E2,7.5E1,2.28E2,1.53E2,3.5E1,7E1,5E0,1.83E2,4.5E1,1.35E2,1.8E1,3E0,3.2E1,6.6E1,4E0,1.58E2,2.5E1,3E1,1.5E1,8.2E1,5.3E1,1.4E1,4E0,2.9E1,3E0,8.8E1,7E1,9E0,1.6E1,1.8E1,1.2E1,3E0,1.2E1,4.7E1,3.5E1,2.2E1,3.1E1,1.1E1,3E0,1.2E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[6.0572852E-3,-1.8199144E-1,3.4493575E-1,-3.790284E-1,-1.0879841E-1,3.8998207E-1,-6.55416E-1,-4.0758693E-1,-2.286107E-1,-1.3072756E-1,1.4622772E-1,3.06783E-1,7.5313985E-1,-3.8658034E-2,-1.1663204E-2,-4.1552907E-1,-7.8103016E-3,-1.1799377E-2,-3.498371E-3,-2.084759E-1,-6.1905928E-2,2.6840997E-1,-1.9353667E-2,2.1731754E-1,4.8487943E-1,5.1438544E-2,5.503083E-1,-1.9348415E-2,-8.847798E-3,-1.2831727E-2,-5.6384495E-3,2.6766814E-3,-6.9022686E-3,1.5864013E-2,4.812682E-3,-4.3719513E-3,2.7777737E-3,7.307238E-3,1.5415922E-2,2.3545159E-2,2.061297E-3,3.1516116E-2,-1.1195195E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,29,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.160698E1,4.5592947E0,8.231607E0,2.762518E-1,1.3277631E0,4.9053974E0,3.9145732E-1,6.267357E-2,5.146146E-2,1.1428599E0,3.895858E-1,2.1253138E0,1.7136574E0,0E0,0E0,2.8243065E-2,0E0,0E0,0E0,5.8485794E-1,1.2507641E0,1.0406351E-1,6.0689148E-2,6.008439E-1,5.7648563E-1,0E0,2.58885E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,19,19,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,30,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,2.3033237E3,2.1915232E8,1.1576422E9,7.3073276E3,3.1E1,2.4434292E7,1.2511433E3,2.025313E6,1.24E2,6.0891E4,1.417988E6,-3.8658034E-2,-1.1663204E-2,3.74301E5,-7.8103016E-3,-1.1799377E-2,-3.498371E-3,3.006846E6,1.4074289E7,9.565983E6,2.8720352E3,1.5271514E5,7.217412E2,5.1438544E-2,1.5096262E7,-1.9348415E-2,-8.847798E-3,-1.2831727E-2,-5.6384495E-3,2.6766814E-3,-6.9022686E-3,1.5864013E-2,4.812682E-3,-4.3719513E-3,2.7777737E-3,7.307238E-3,1.5415922E-2,2.3545159E-2,2.061297E-3,3.1516116E-2,-1.1195195E-2],"split_indices":[2,1,67,7,7,61,3,12,4,1,3,2,38,0,0,38,0,0,0,9,54,54,4,37,67,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.94E2,3.18E2,1.76E2,8.5E1,2.33E2,1.69E2,7E0,7E1,1.5E1,2.15E2,1.8E1,1.39E2,3E1,4E0,3E0,6.7E1,3E0,1.2E1,3E0,1E2,1.15E2,1E1,8E0,9.4E1,4.5E1,9E0,2.1E1,6.4E1,3E0,5.3E1,4.7E1,4.9E1,6.6E1,6E0,4E0,4E0,4E0,6.5E1,2.9E1,4.2E1,3E0,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-5.54752E-3,-1.6443579E-1,3.2501355E-1,-3.552566E-1,-9.967609E-2,3.84754E-1,-3.3401087E-1,-3.6708194E-1,-1.0993207E-3,-1.4213769E-1,1.1100005E-1,2.3196991E-1,5.786293E-1,-6.0693152E-2,-3.7242793E-2,-3.8275248E-1,-1.0577338E-2,-2.3540384E-1,-6.0714696E-2,2.8739604E-1,4.8837747E-2,1.7508768E-1,3.4767663E-1,4.8590788E-1,4.1740857E-2,-2.2820817E-1,1.1332003E-2,-1.9530574E-2,-1.530939E-2,-6.5949857E-3,-1.2606876E-2,-4.6014627E-3,8.675451E-3,1.52983125E-2,5.570542E-3,-3.1721408E-3,5.908071E-3,-4.081322E-3,9.458382E-3,8.796002E-3,1.7524011E-2,2.4416724E-2,-3.583239E-3,-4.4600004E-3,-1.3815539E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,-1,29,31,33,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6209536E1,4.1290674E0,6.506521E0,3.2412338E-1,2.2790263E0,4.279314E0,1.7866625E0,8.6214066E-2,0E0,1.5880437E0,4.5746708E-1,4.985671E-1,1.5668049E0,5.7300144E-1,0E0,1.0064125E-2,0E0,3.189187E-1,1.1557894E0,4.47883E-2,3.2192686E-1,5.1115656E-1,7.441783E-2,1.4668427E0,0E0,3.8023084E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,-1,30,32,34,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.0434409E3,1.2852292E7,3.53E2,6.8963E4,7.339209E6,2.9778808E7,-1.0993207E-3,7.898574E2,3.08031E5,1.5271514E5,1.1275989E1,3.236383E1,-3.7242793E-2,2.0011E5,-1.0577338E-2,7.37365E6,2.332E3,3.9032572E6,1.570662E5,2.8529238E1,1.5147E4,5.37E2,4.1740857E-2,2.2201815E9,1.1332003E-2,-1.9530574E-2,-1.530939E-2,-6.5949857E-3,-1.2606876E-2,-4.6014627E-3,8.675451E-3,1.52983125E-2,5.570542E-3,-3.1721408E-3,5.908071E-3,-4.081322E-3,9.458382E-3,8.796002E-3,1.7524011E-2,2.4416724E-2,-3.583239E-3,-4.4600004E-3,-1.3815539E-2],"split_indices":[2,1,67,9,8,2,56,12,0,61,38,37,44,65,0,9,0,54,0,54,56,65,2,3,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.97E2,3.36E2,1.61E2,8.4E1,2.52E2,1.48E2,1.3E1,8.1E1,3E0,2.1E2,4.2E1,8.4E1,6.4E1,9E0,4E0,7.1E1,1E1,9.7E1,1.13E2,1E1,3.2E1,5.8E1,2.6E1,5.2E1,1.2E1,6E0,3E0,3.3E1,3.8E1,3.1E1,6.6E1,9.8E1,1.5E1,7E0,3E0,1.3E1,1.9E1,6E0,5.2E1,6E0,2E1,4.8E1,4E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[9.351734E-3,-1.3564564E-1,3.6082488E-1,-3.4785393E-1,-7.461422E-2,4.2306665E-1,-4.0190372E-1,-3.661468E-1,-8.477121E-4,-1.08813375E-1,1.8483797E-1,3.390708E-1,7.4277824E-1,-3.095027E-2,2.8685327E-3,-2.0979326E-2,-3.3095998E-1,-1.20707765E-1,2.0945638E-2,8.098962E-2,3.0515456E-1,2.4200372E-1,4.5164144E-1,4.495106E-3,8.225791E-1,-1.5631884E-2,-7.152537E-3,-7.450078E-3,-2.1503161E-4,6.3256975E-3,-2.5870404E-3,1.5792107E-2,7.5159776E-3,8.078379E-3,1.4451264E-2,1.691324E-2,2.5777254E-2,1.8159365E-2,4.204565E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,25,27,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.425622E1,4.3386583E0,6.76886E0,4.4504642E-1,2.353724E0,3.21237E0,1.4448807E0,7.886505E-2,0E0,1.6253145E0,3.626207E-1,1.0270987E0,1.2565241E0,0E0,0E0,0E0,4.8763275E-2,1.117512E0,0E0,1.4951198E-1,2.2114992E-2,2.244637E-1,2.2236538E-1,0E0,5.7760525E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,16,16,17,17,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,26,28,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0434409E3,1.2852292E7,1.4777102E9,7.710634E3,3.5226266E0,1.44E2,-8.477121E-4,3.059144E3,1.1593098E2,5.9161E4,6.841449E1,-3.095027E-2,2.8685327E-3,-2.0979326E-2,7.338093E4,2.6919386E-1,2.0945638E-2,5.78E2,6.3349745E6,1.5271514E5,1.646359E5,4.495106E-3,6.274818E4,-1.5631884E-2,-7.152537E-3,-7.450078E-3,-2.1503161E-4,6.3256975E-3,-2.5870404E-3,1.5792107E-2,7.5159776E-3,8.078379E-3,1.4451264E-2,1.691324E-2,2.5777254E-2,1.8159365E-2,4.204565E-2],"split_indices":[2,1,67,9,7,61,63,0,0,61,63,2,65,0,0,0,42,48,0,8,54,37,37,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.74E2,3.36E2,1.38E2,7.4E1,2.62E2,1.28E2,1E1,7E1,4E0,2.32E2,3E1,1.03E2,2.5E1,6E0,4E0,1.6E1,5.4E1,2.28E2,4E0,1.7E1,1.3E1,5.7E1,4.6E1,3E0,2.2E1,5E1,4E0,1.67E2,6.1E1,1.2E1,5E0,9E0,4E0,3.2E1,2.5E1,2.9E1,1.7E1,5E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[9.837996E-4,-1.5627488E-1,2.9572588E-1,-3.3217576E-1,-9.435746E-2,2.1544689E-1,7.277941E-1,-3.8198858E-1,-2.537642E-1,-1.3791156E-1,6.1442345E-2,2.644163E-1,-3.6565533E-1,8.213659E-1,-7.1319443E-4,-1.8712286E-2,-2.7391666E-1,-6.886825E-4,-2.750095E-1,-1.9168416E-1,-3.3165213E-2,8.293081E-3,2.1270503E-1,1.3458729E-1,3.6552E-1,-6.498087E-1,1.2263858E-3,4.509304E-2,4.3850297E-1,-6.1008167E-3,-1.3854988E-2,-5.7843095E-3,-1.3072114E-2,-9.726327E-3,2.09191E-3,4.5728395E-3,-8.54161E-3,-3.0642562E-3,2.946512E-3,1.2832553E-2,3.4631784E-3,-4.726832E-3,8.096637E-3,1.784909E-2,2.95372E-3,-3.607515E-2,-1.5963323E-2,2.3533747E-2,9.849201E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,29,-1,31,33,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3314291E1,3.5410128E0,5.913249E0,2.3506641E-1,1.6641893E0,4.3366647E0,1.8909006E0,3.3456326E-2,1.7843795E-1,1.0694473E0,4.3236905E-1,1.7622194E0,1.3556226E0,1.0750809E0,0E0,0E0,1.2364268E-2,0E0,1.6108751E-2,6.336627E-1,1.3669212E0,1.7738959E-1,1.0868412E-1,6.427827E-1,5.4319954E-1,1.683569E-2,0E0,0E0,4.156375E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,30,-1,32,34,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,7.710634E3,1.425966E6,7.164074E8,9.064268E2,1.92E2,5.87114E5,2E1,1.1501869E3,1.4777102E9,2.5096E4,3.5226266E0,5.192243E6,-7.1319443E-4,-1.8712286E-2,1.1802526E3,-6.886825E-4,2.586E4,2.889654E5,6.935338E1,1.9252769E5,3.74301E5,4.3290512E1,1.773296E6,1.2799757E9,1.2263858E-3,4.509304E-2,1E0,-6.1008167E-3,-1.3854988E-2,-5.7843095E-3,-1.3072114E-2,-9.726327E-3,2.09191E-3,4.5728395E-3,-8.54161E-3,-3.0642562E-3,2.946512E-3,1.2832553E-2,3.4631784E-3,-4.726832E-3,8.096637E-3,1.784909E-2,2.95372E-3,-3.607515E-2,-1.5963323E-2,2.3533747E-2,9.849201E-3],"split_indices":[2,1,61,9,7,67,8,1,10,61,7,2,63,38,0,0,64,0,38,37,63,56,38,65,11,7,0,0,13,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[5.01E2,3.27E2,1.74E2,8.4E1,2.43E2,1.48E2,2.6E1,4.9E1,3.5E1,1.9E2,5.3E1,1.37E2,1.1E1,2.3E1,3E0,3.7E1,1.2E1,3E0,3.2E1,1.25E2,6.5E1,4E1,1.3E1,6.1E1,7.6E1,6E0,5E0,1.5E1,8E0,3E0,9E0,3E0,2.9E1,1.15E2,1E1,3.5E1,3E1,1.7E1,2.3E1,8E0,5E0,9E0,5.2E1,7E1,6E0,3E0,3E0,5E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[7.4503543E-3,-1.4274189E-1,2.7536464E-1,-2.7888697E-1,-7.398962E-2,3.2289678E-1,-2.977144E-1,-3.167151E-1,-1.038172E-2,-9.6129835E-2,1.8302166E-1,2.539031E-1,6.610168E-1,-4.957422E-2,-3.3609677E-2,-3.2945514E-1,-5.260269E-3,-6.262431E-2,4.4013397E-3,-1.8678229E-1,-3.7426345E-2,-1.7217173E-3,2.329902E-1,1.5889613E-1,3.844585E-1,7.5692475E-1,-3.6465114E-3,-2.0931187E-1,1.1090787E-2,-1.6466646E-2,-1.0283511E-2,5.1922643E-6,-4.780639E-3,-9.3151005E-3,-8.58053E-4,-3.2641205E-3,6.62454E-3,1.4761844E-2,5.741886E-3,9.375822E-3,8.120591E-4,1.2277114E-2,2.2303073E-2,4.3872487E-2,2.6172591E-2,-4.4474704E-3,-1.2314802E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,-1,33,35,-1,37,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9716002E1,2.9184933E0,4.890315E0,1.0632019E0,1.2125537E0,3.634655E0,1.4807998E0,1.9861603E-1,8.3961904E-2,1.0254028E0,1.9619638E-1,1.6382151E0,1.9783745E0,5.170582E-1,0E0,1.9278336E-1,0E0,2.6263908E-2,0E0,2.1040654E-1,7.4534625E-1,0E0,9.43799E-2,5.221548E-1,5.507536E-1,3.4759808E-1,0E0,2.077794E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,19,19,20,20,22,22,23,23,24,24,25,25,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,-1,34,36,-1,38,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,9.54947E5,9.064268E2,1.2852292E7,1.4777102E9,7.710634E3,7.339209E6,1.6120845E5,4.17E2,7.317205E2,2.0532622E3,4.1331047E-1,1.4656219E8,3.236383E1,-3.3609677E-2,3.006846E6,-5.260269E-3,2.92E2,4.4013397E-3,7.164074E8,2.194E3,-1.7217173E-3,6.088569E6,1.99218E7,1.30817086E5,1.681178E6,-3.6465114E-3,2.2201815E9,1.1090787E-2,-1.6466646E-2,-1.0283511E-2,5.1922643E-6,-4.780639E-3,-9.3151005E-3,-8.58053E-4,-3.2641205E-3,6.62454E-3,1.4761844E-2,5.741886E-3,9.375822E-3,8.120591E-4,1.2277114E-2,2.2303073E-2,4.3872487E-2,2.6172591E-2,-4.4474704E-3,-1.2314802E-2],"split_indices":[2,1,67,9,7,61,56,37,0,61,4,44,1,65,0,9,0,0,0,7,0,0,54,54,37,38,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,3.13E2,1.75E2,1.04E2,2.09E2,1.62E2,1.3E1,9.1E1,1.3E1,1.93E2,1.6E1,1.36E2,2.6E1,9E0,4E0,8.5E1,6E0,9E0,4E0,7.5E1,1.18E2,3E0,1.3E1,8E1,5.6E1,2.3E1,3E0,6E0,3E0,6.4E1,2.1E1,4E0,5E0,6.8E1,7E0,1E2,1.8E1,6E0,7E0,6E1,2E1,2.8E1,2.8E1,9E0,1.4E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-7.99595E-3,-1.385066E-1,2.4992868E-1,-2.9344872E-1,-8.899087E-2,2.9560578E-1,-4.9219465E-1,-3.0558148E-1,4.775705E-4,-1.2904409E-1,4.5823995E-2,2.4229899E-1,3.1749E-2,-1.0556629E-2,-3.2379255E-2,-2.2660562E-1,-3.421168E-1,-1.8441938E-1,-4.8521247E-2,2.2674633E-2,1.3118576E-2,1.4190578E-1,3.699901E-1,-1.1103288E-2,-4.921961E-3,-1.756114E-2,-1.3048026E-2,-4.3618907E-3,-1.0046647E-2,2.4136025E-3,-7.153949E-3,-1.4814977E-3,4.7691492E-3,8.978257E-3,-1.4083548E-3,1.8628925E-2,-1.1916631E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,27,29,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6629938E1,2.4903245E0,5.7725716E0,2.9445934E-1,1.3558558E0,3.1643772E0,3.999803E-1,1.4176512E-1,0E0,8.5163355E-1,3.1687793E-1,1.7422009E0,0E0,0E0,0E0,2.5421739E-2,9.214878E-3,3.3584762E-1,8.816263E-1,2.4577108E-1,0E0,7.560377E-1,1.532526E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,28,30,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0434409E3,1.2852292E7,7.10711E8,8.625562E0,1.805424E-1,4.3156998E1,4.775705E-4,2.735322E6,3.4540886E3,1.5333999E7,3.1749E-2,-1.0556629E-2,-3.2379255E-2,3.23251E5,6E1,7.477264E6,1.7377544E7,2.9124984E7,1.3118576E-2,1.8322262E7,5.37E2,-1.1103288E-2,-4.921961E-3,-1.756114E-2,-1.3048026E-2,-4.3618907E-3,-1.0046647E-2,2.4136025E-3,-7.153949E-3,-1.4814977E-3,4.7691492E-3,8.978257E-3,-1.4083548E-3,1.8628925E-2,-1.1916631E-2],"split_indices":[2,1,67,9,7,44,47,62,0,1,4,1,0,0,0,38,10,54,54,9,0,54,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,3.27E2,1.65E2,7.8E1,2.49E2,1.56E2,9E0,7.5E1,3E0,1.92E2,5.7E1,1.39E2,1.7E1,5E0,4E0,2.6E1,4.9E1,1.13E2,7.9E1,5.3E1,4E0,7.9E1,6E1,2.2E1,4E0,2.5E1,2.4E1,3.3E1,8E1,4.1E1,3.8E1,3.2E1,2.1E1,6E1,1.9E1,5.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-5.121914E-4,-1.4090703E-1,2.5518182E-1,-2.7925816E-1,-9.250209E-2,1.7495018E-1,5.482795E-1,-2.9009873E-1,4.6118032E-4,-1.5863063E-2,-1.6123353E-1,2.1920486E-1,-3.2274255E-1,-2.3998101E-3,6.2049496E-1,-3.581403E-1,-1.2056841E-2,-5.7497513E-2,1.6464311E-1,-2.0259802E-1,-5.793346E-2,1.0110612E-1,3.2385725E-1,-5.832638E-1,1.6011694E-3,8.5437363E-1,4.714933E-1,-7.735406E-3,-1.7461156E-2,-5.332491E-3,2.152004E-3,-1.0623375E-3,1.2522589E-2,-1.8799845E-3,-1.0160492E-2,-7.855312E-3,5.2932537E-5,7.7133235E-3,-1.0764947E-3,1.577317E-2,-5.321836E-4,-3.257219E-2,-1.4137187E-2,2.002481E-2,4.3230064E-2,2.3530787E-2,6.4096595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,33,35,37,39,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8128609E1,2.157351E0,4.113391E0,2.663951E-1,1.2774472E0,3.198329E0,1.6810322E0,5.2396297E-2,0E0,8.8123846E-1,5.352013E-1,1.5863266E0,1.1387691E0,0E0,7.8201103E-1,2.4106503E-2,0E0,5.916362E-1,4.5685804E-1,2.7650404E-1,2.5555718E-1,5.3354996E-1,4.8606062E-1,2.6902676E-2,0E0,3.8276672E-2,2.535534E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,34,36,38,40,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,6.651788E3,1.2852292E7,1.2885352E7,7.217412E2,5.3140095E1,3.088E3,4.6118032E-4,2.972948E6,7.194E3,2.5096E4,3.5226266E0,-2.3998101E-3,1.417988E6,6.795808E7,-1.2056841E-2,7.164074E8,7.28843E2,5.1698097E1,9.203515E7,1.3508742E7,9.677921E7,1.2799757E9,1.6011694E-3,9.256843E3,1.93E2,-7.735406E-3,-1.7461156E-2,-5.332491E-3,2.152004E-3,-1.0623375E-3,1.2522589E-2,-1.8799845E-3,-1.0160492E-2,-7.855312E-3,5.2932537E-5,7.7133235E-3,-1.0764947E-3,1.577317E-2,-5.321836E-4,-3.257219E-2,-1.4137187E-2,2.002481E-2,4.3230064E-2,2.3530787E-2,6.4096595E-3],"split_indices":[2,1,61,9,54,67,65,2,0,1,2,2,63,0,38,7,0,7,61,57,7,54,1,7,0,4,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[5.03E2,3.25E2,1.78E2,8.3E1,2.42E2,1.41E2,3.7E1,8E1,3E0,1.15E2,1.27E2,1.3E2,1.1E1,4E0,3.3E1,1.9E1,6.1E1,9.4E1,2.1E1,9E1,3.7E1,6.2E1,6.8E1,6E0,5E0,1.1E1,2.2E1,3E0,1.6E1,6E1,3.4E1,8E0,1.3E1,1E1,8E1,1.2E1,2.5E1,4E1,2.2E1,6.4E1,4E0,3E0,3E0,3E0,8E0,1.9E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[3.2637094E-3,-1.2865132E-1,2.575953E-1,-1.44011E-1,5.831074E-1,1.9396739E-1,6.112431E-1,-1.8427028E-1,5.526809E-3,3.237513E-2,1.4321717E-2,2.1841075E-1,-1.9423488E-1,2.3519753E-2,4.10743E-2,-2.1033446E-1,-4.691209E-2,-8.909564E-2,7.8171596E-2,1.4280614E-1,3.4013584E-1,1.252525E-3,-1.4167759E-2,-1.2278425E-2,-6.880381E-3,-6.597268E-3,1.2805504E-3,-5.316319E-3,3.1819185E-3,-9.604111E-3,5.3581004E-3,-6.208454E-3,7.4091083E-3,2.2383817E-2,1.415858E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6405619E1,3.6042032E0,3.6392527E0,1.9034281E0,1.3764143E-2,1.3993449E0,2.3503399E-1,8.815851E-1,4.7438496E-1,0E0,0E0,1.1938286E0,2.3856279E-1,0E0,0E0,6.864872E-1,3.0267233E-1,1.3694105E-1,4.6174234E-1,4.7408545E-1,8.223772E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1576422E9,4.23218E3,7.765071E0,9.570385E4,1.3843572E5,1.4609149E4,1.354316E6,1.6120845E5,9.53E2,3.237513E-2,1.4321717E-2,4.1331047E-1,1.4459738E-1,2.3519753E-2,4.10743E-2,1.592757E6,4.863447E2,1.2175E4,1.5802073E3,5.767668E0,2.0159504E1,1.252525E-3,-1.4167759E-2,-1.2278425E-2,-6.880381E-3,-6.597268E-3,1.2805504E-3,-5.316319E-3,3.1819185E-3,-9.604111E-3,5.3581004E-3,-6.208454E-3,7.4091083E-3,2.2383817E-2,1.415858E-2],"split_indices":[7,61,44,42,42,4,2,37,0,0,0,44,47,0,0,9,61,2,4,63,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.87E2,3.21E2,1.66E2,3.15E2,6E0,1.42E2,2.4E1,2.48E2,6.7E1,3E0,3E0,1.34E2,8E0,2E1,4E0,2.08E2,4E1,2.9E1,3.8E1,8.4E1,5E1,3E0,5E0,1.04E2,1.04E2,1.7E1,2.3E1,2.5E1,4E0,4E0,3.4E1,5E0,7.9E1,6E0,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0813302E-2,-9.956078E-2,3.136165E-1,-1.3359281E-1,1.2557599E-1,3.4809306E-1,-1.8474811E-1,-2.0387933E-1,-5.212846E-2,3.3092376E-2,2.1282727E-1,3.0478802E-1,3.2767497E-2,-2.430644E-2,3.522746E-3,-2.2536732E-1,-3.682301E-2,2.4932796E-2,-1.386224E-1,-1.1522052E-1,1.0094006E-1,1.4346236E-2,1.1852562E-1,3.2235202E-1,-1.1039803E-2,-1.3808767E-2,-8.9100115E-3,6.2447255E-3,-4.0578255E-3,-2.857843E-4,1.0728757E-2,-7.684622E-3,1.7034257E-2,-7.1818815E-3,-1.5679937E-3,6.797058E-4,8.045405E-3,6.8299784E-3,1.5270178E-3,1.27066625E-2,2.5586266E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6140448E1,2.7290752E0,2.280776E0,1.7529325E0,3.7221473E-1,1.7098093E0,8.5686713E-1,5.840025E-1,9.63264E-1,2.5313488E-1,1.906879E-1,1.1327257E0,0E0,0E0,0E0,2.8320932E-1,1.9179794E-1,5.044489E-1,1.0805231E0,2.1642908E-2,1.0696104E-1,0E0,3.1650662E-2,9.767113E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7267E4,3.44E2,1.0434409E3,1.732734E6,1.4096699E9,1.6828056E1,2.8E1,7.164074E8,1.8071064E7,1.8374788E7,6.088569E6,5.37E2,3.2767497E-2,-2.430644E-2,3.522746E-3,4.48149E5,4.45E3,2.09216E5,2.6084324E3,1.662639E7,3.6828573E8,1.4346236E-2,1.1797827E9,8.1033794E3,-1.1039803E-2,-1.3808767E-2,-8.9100115E-3,6.2447255E-3,-4.0578255E-3,-2.857843E-4,1.0728757E-2,-7.684622E-3,1.7034257E-2,-7.1818815E-3,-1.5679937E-3,6.797058E-4,8.045405E-3,6.8299784E-3,1.5270178E-3,1.27066625E-2,2.5586266E-2],"split_indices":[2,8,67,1,7,44,3,7,54,12,54,3,0,0,0,1,2,11,61,54,5,0,5,61,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.81E2,3.53E2,1.28E2,3.07E2,4.6E1,1.2E2,8E0,1.64E2,1.43E2,2.3E1,2.3E1,1.09E2,1.1E1,3E0,5E0,1.45E2,1.9E1,7.6E1,6.7E1,7E0,1.6E1,1E1,1.3E1,1.06E2,3E0,3.9E1,1.06E2,4E0,1.5E1,6.7E1,9E0,6.4E1,3E0,4E0,3E0,8E0,8E0,9E0,4E0,9.1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.5635574E-3,-1.15034774E-1,2.6125193E-1,-2.4201158E-1,-5.9667528E-2,1.9689451E-1,5.2713513E-1,-2.703325E-1,-4.4684935E-2,-9.36484E-2,7.8971446E-2,2.1975856E-1,-1.39405355E-2,3.6334133E-1,7.425474E-1,-2.7929217E-1,-4.568914E-3,-3.7495426E-3,1.7734964E-3,-1.482942E-1,1.126635E-2,2.0973864E-1,1.7138738E-2,3.3460635E-1,1.538453E-1,1.8796742E-2,3.7916312E-3,1.8559486E-2,3.937053E-2,-5.956142E-3,-1.3229467E-2,-7.489307E-3,4.4332673E-3,-2.119928E-3,1.3885756E-2,1.1363509E-2,1.9018631E-3,-4.9462295E-3,4.756449E-3,7.075281E-4,1.6754884E-2,9.486797E-3,1.2782254E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,-1,-1,-1,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.51649065E1,2.3547897E0,2.5827227E0,5.579443E-1,1.1192086E0,1.534718E0,7.886982E-1,1.0806656E-1,4.669126E-2,1.0909688E0,3.764328E-1,8.796315E-1,0E0,2.186377E-1,1.273961E-1,8.518314E-2,0E0,0E0,0E0,4.8674035E-1,1.1246156E0,8.700764E-2,3.6993536E-1,4.370675E-1,5.327537E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,-1,-1,-1,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,9.50639E5,4.026531E9,1.2852292E7,3.15E2,9.533205E6,2.2876814E1,2.5050928E6,4.17E2,1.1501869E3,4.16606E5,5.2245E4,-1.39405355E-2,1.3936486E8,2.502264E7,7.547656E1,-4.568914E-3,-3.7495426E-3,1.7734964E-3,2.889654E5,5.6024197E9,3.457E3,1.570662E5,5.274031E6,2.019058E7,1.8796742E-2,3.7916312E-3,1.8559486E-2,3.937053E-2,-5.956142E-3,-1.3229467E-2,-7.489307E-3,4.4332673E-3,-2.119928E-3,1.3885756E-2,1.1363509E-2,1.9018631E-3,-4.9462295E-3,4.756449E-3,7.075281E-4,1.6754884E-2,9.486797E-3,1.2782254E-3],"split_indices":[2,1,7,9,8,56,63,59,0,61,38,9,0,1,60,63,0,0,0,37,5,10,56,1,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,3.36E2,1.56E2,1.01E2,2.35E2,1.27E2,2.9E1,8.8E1,1.3E1,1.89E2,4.6E1,1.22E2,5E0,1.8E1,1.1E1,8.3E1,5E0,9E0,4E0,1.24E2,6.5E1,1.4E1,3.2E1,4.3E1,7.9E1,1.5E1,3E0,4E0,7E0,6E0,7.7E1,1.17E2,7E0,5.5E1,1E1,1.1E1,3E0,1.3E1,1.9E1,4E0,3.9E1,5.5E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.759133E-5,-1.0045388E-1,2.64826E-1,-1.9448927E-1,-3.0218768E-3,1.839926E-1,5.1442856E-1,-2.4570806E-1,-1.11788094E-1,-1.8859867E-2,2.9793106E-2,2.0321615E-1,-9.978468E-3,3.414517E-1,7.276695E-1,-2.6021802E-1,-9.673854E-2,-4.832523E-2,-1.7024378E-1,6.0781274E-2,-1.2781258E-1,1.2712051E-1,2.962149E-1,1.7536273E-2,4.0754112E-3,1.4971137E-2,4.2883016E-2,-1.3409234E-2,-7.783203E-3,-7.404221E-3,6.2671147E-4,1.9375648E-3,-3.358868E-3,-9.219045E-3,-7.599327E-4,4.3969933E-4,1.0638248E-2,-2.3420542E-3,-1.0446638E-2,6.9209295E-3,-9.709174E-3,1.5343913E-2,2.6603852E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,27,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2478066E1,3.1145115E0,2.5080814E0,7.053356E-1,1.7526234E0,8.0945826E-1,8.9499474E-1,2.039504E-1,2.4403703E-1,1.4394931E0,0E0,6.346412E-1,0E0,1.6612148E-1,7.8384495E-1,2.3403406E-1,8.125058E-2,8.040193E-2,1.6450405E-1,8.412082E-1,5.272771E-1,4.7397548E-1,4.7478032E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,28,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,1.732734E6,1.7552137E0,3.006846E6,5.2023345E3,1.505649E6,8.205238E0,2.1592189E5,1E0,1.9051096E7,2.9793106E-2,4.704E4,-9.978468E-3,2.228899E6,1.32737E5,4.9208847E5,5.987804E2,1.03E2,7.164074E8,2.8191E4,1E0,2.67E2,6.029086E10,1.7536273E-2,4.0754112E-3,1.4971137E-2,4.2883016E-2,-1.3409234E-2,-7.783203E-3,-7.404221E-3,6.2671147E-4,1.9375648E-3,-3.358868E-3,-9.219045E-3,-7.599327E-4,4.3969933E-4,1.0638248E-2,-2.3420542E-3,-1.0446638E-2,6.9209295E-3,-9.709174E-3,1.5343913E-2,2.6603852E-4],"split_indices":[7,1,51,9,61,2,62,37,23,54,0,10,0,9,2,60,61,3,7,10,22,3,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.67E2,3.39E2,1.28E2,1.72E2,1.67E2,9.8E1,3E1,1.05E2,6.7E1,1.64E2,3E0,9.4E1,4E0,1.8E1,1.2E1,9.5E1,1E1,3.3E1,3.4E1,9.5E1,6.9E1,5.3E1,4.1E1,1.5E1,3E0,5E0,7E0,6.8E1,2.7E1,6E0,4E0,7E0,2.6E1,2.8E1,6E0,7.4E1,2.1E1,4E1,2.9E1,5E1,3E0,3.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[8.952157E-3,-1.1170179E-1,2.517495E-1,-1.4057007E-1,3.9924324E-2,1.7490384E-1,5.440607E-1,-2.0661587E-1,-6.878223E-2,-9.97552E-4,1.4814626E-1,1.9914955E-1,-2.0447334E-2,3.890815E-2,3.662429E-1,-2.1936809E-1,-7.890938E-2,-1.4594717E-1,-1.744641E-2,-2.4946311E-2,7.415825E-3,-2.0910115E-3,2.0516564E-1,1.1998548E-1,2.859305E-1,-4.168347E-3,4.432472E-1,-1.53345205E-2,-9.439565E-3,-5.384639E-4,-6.606648E-3,-4.605915E-3,-1.1828047E-2,-2.6197014E-3,6.4229956E-3,5.1650484E-3,-2.154635E-3,1.1604048E-2,2.6108394E-3,7.048788E-3,-3.7835005E-3,-1.1930676E-3,1.3899931E-2,1.5126617E-2,2.4333041E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,35,-1,-1,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.37672205E1,1.380049E0,3.4090462E0,1.2374883E0,2.2600868E-1,1.9732614E0,1.4307976E0,2.0579958E-1,5.045035E-1,1.5242773E-1,1.6411155E-1,7.9695463E-1,0E0,0E0,8.151374E-1,1.1873627E-1,5.9835054E-2,2.339313E-1,4.979266E-1,1.09044746E-1,0E0,0E0,6.4405054E-2,4.6124715E-1,3.2476473E-1,0E0,1.3703585E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,36,-1,-1,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.164074E8,7.710634E3,1.533115E6,1.4096699E9,9.533205E6,3.0418E4,1.73E2,7.28843E2,1.2E3,2.4237612E3,4.1331047E-1,-2.0447334E-2,3.890815E-2,6.7211235E1,9.306416E0,1.21E2,6.9E1,1.4998456E5,1.4193897E-3,7.415825E-3,-2.0910115E-3,1.5E2,2.5916522E7,2.2782404E0,-4.168347E-3,1.2472753E5,-1.53345205E-2,-9.439565E-3,-5.384639E-4,-6.606648E-3,-4.605915E-3,-1.1828047E-2,-2.6197014E-3,6.4229956E-3,5.1650484E-3,-2.154635E-3,1.1604048E-2,2.6108394E-3,7.048788E-3,-3.7835005E-3,-1.1930676E-3,1.3899931E-2,1.5126617E-2,2.4333041E-2],"split_indices":[2,7,61,1,7,56,10,8,61,0,4,44,0,0,65,65,3,8,37,48,0,0,3,54,63,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.68E2,3.13E2,1.55E2,2.63E2,5E1,1.24E2,3.1E1,1.36E2,1.27E2,3.7E1,1.3E1,1.2E2,4E0,1E1,2.1E1,1.23E2,1.3E1,5E1,7.7E1,3.3E1,4E0,3E0,1E1,6.4E1,5.6E1,3E0,1.8E1,1E1,1.13E2,7E0,6E0,3.7E1,1.3E1,6.2E1,1.5E1,4E0,2.9E1,7E0,3E0,5.5E1,9E0,3E0,5.3E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[-2.1667744E-3,-1.2127288E-1,1.8325698E-1,-1.4353812E-1,7.1075656E-2,8.068883E-2,3.903378E-1,-1.5101707E-1,1.8004254E-2,1.1933453E-1,-9.765024E-3,3.6026645E-2,2.2359821E-1,7.095136E-1,3.1809205E-1,-1.4314538E-1,-2.9047685E-2,7.386158E-2,1.7101413E-2,-6.5224795E-3,1.6563688E-1,1.1086878E-2,1.69146E-3,1.6691634E-2,3.9099272E-2,3.6655578E-1,-6.3939616E-3,-1.0443941E-2,-4.870731E-3,5.609958E-3,-4.6164338E-3,-2.4707336E-3,5.6118546E-3,9.125333E-3,-2.8155474E-3,5.033656E-3,1.774925E-2,-1.0972904E-2,8.310167E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,-1,29,-1,31,33,-1,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.09543495E1,1.3035731E0,4.0890436E0,1.1477523E0,4.6475905E-1,8.28071E-1,1.2500916E0,9.296484E-1,0E0,2.96275E-1,0E0,5.580754E-1,9.910226E-2,2.3786259E-1,8.5153675E-1,8.033376E-1,0E0,2.2524361E-1,0E0,4.7869667E-1,2.040714E-1,0E0,0E0,0E0,0E0,2.1427155E-1,3.9454627E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,-1,30,-1,32,34,-1,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,1.577058E9,5.9161E4,3.7241616E3,1.505649E6,1.5271514E5,1.417988E6,5.825042E6,1.8004254E-2,4.79054E5,-9.765024E-3,1.29186E9,9.234E3,1.08E2,1.5096262E7,7.41718E5,-2.9047685E-2,1.6345643E3,1.7101413E-2,2.144067E3,6.969457E9,1.1086878E-2,1.69146E-3,1.6691634E-2,3.9099272E-2,1.3590209E7,3.677E3,-1.0443941E-2,-4.870731E-3,5.609958E-3,-4.6164338E-3,-2.4707336E-3,5.6118546E-3,9.125333E-3,-2.8155474E-3,5.033656E-3,1.774925E-2,-1.0972904E-2,8.310167E-3],"split_indices":[42,7,2,61,2,37,38,56,0,2,0,7,0,3,9,1,0,61,0,61,5,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.94E2,3.01E2,1.93E2,2.7E2,3.1E1,1.3E2,6.3E1,2.67E2,3E0,2.7E1,4E0,1E2,3E1,1E1,5.3E1,2.64E2,3E0,2.4E1,3E0,7.6E1,2.4E1,2.7E1,3E0,4E0,6E0,4.6E1,7E0,7.8E1,1.86E2,1.9E1,5E0,5.6E1,2E1,2.1E1,3E0,4E0,4.2E1,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[9.987624E-3,-8.638789E-2,2.402963E-1,-1.2276193E-1,4.3696534E-2,1.7624399E-1,4.820904E-1,-4.522577E-2,-1.9190733E-1,6.138085E-2,-1.347163E-2,2.0553248E-1,-1.9805472E-1,2.7224964E-1,7.02462E-1,-1.4652E-1,3.840736E-2,-2.2607367E-1,-1.4690404E-1,1.47290705E-2,1.4694071E-1,1.4118236E-1,2.9391524E-1,-2.0136656E-2,3.8227234E-3,1.6813366E-2,-5.897846E-3,1.5371852E-2,3.5323035E-2,-3.7256468E-3,-1.0894622E-2,4.115184E-3,-5.0354768E-3,-1.0731238E-2,-2.4129942E-3,-5.922563E-3,-1.3602116E-2,-5.8799047E-5,8.933001E-3,1.5376214E-3,9.379647E-3,7.894357E-3,6.743421E-4,1.4449855E-2,-4.1070057E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.098564E1,1.6567261E0,2.1762056E0,1.4549475E0,4.784359E-1,1.3233359E0,1.196044E0,1.1069294E0,1.8820286E-1,2.932081E-1,0E0,5.784354E-1,6.500102E-1,6.962898E-1,9.7010136E-2,3.3229625E-1,5.605335E-1,1.04531765E-1,1.3235927E-1,1.4186944E-1,1.6410929E-1,2.549448E-1,3.0063272E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.164074E8,7.710634E3,2.329099E1,1E0,9.064268E2,2.2876814E1,7.317205E2,4.822E3,1.5522031E9,-1.347163E-2,6.050605E9,3.5226266E0,1.4656219E8,1.8E1,1.7377544E7,2.8184534E7,1.7212875E5,9.6E3,2.478066E5,2.7777952E3,2.237815E7,9.677921E7,-2.0136656E-2,3.8227234E-3,1.6813366E-2,-5.897846E-3,1.5371852E-2,3.5323035E-2,-3.7256468E-3,-1.0894622E-2,4.115184E-3,-5.0354768E-3,-1.0731238E-2,-2.4129942E-3,-5.922563E-3,-1.3602116E-2,-5.8799047E-5,8.933001E-3,1.5376214E-3,9.379647E-3,7.894357E-3,6.743421E-4,1.4449855E-2,-4.1070057E-4],"split_indices":[2,7,61,62,18,67,63,61,2,7,0,12,63,1,8,54,54,37,10,37,4,54,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.93E2,3.48E2,1.45E2,2.72E2,7.6E1,1.16E2,2.9E1,1.29E2,1.43E2,7.3E1,3E0,1.08E2,8E0,1.6E1,1.3E1,5.8E1,7.1E1,7.9E1,6.4E1,4.8E1,2.5E1,6.4E1,4.4E1,4E0,4E0,1.3E1,3E0,3E0,1E1,3.5E1,2.3E1,5.3E1,1.8E1,7.5E1,4E0,5.9E1,5E0,4.5E1,3E0,9E0,1.6E1,5.1E1,1.3E1,4.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.0551211E-3,-8.327889E-2,2.1325734E-1,-2.0591386E-1,-4.7805328E-2,2.5022858E-1,-2.4642365E-1,-2.161251E-1,1.6943301E-3,-8.310427E-2,4.4322267E-2,1.8848313E-1,4.4332737E-1,-4.4251984E-1,9.489792E-3,-2.3061031E-1,-1.1088288E-1,-1.101802E-1,4.5366615E-2,6.172092E-2,-1.3009269E-2,9.197824E-2,2.4523246E-1,3.248129E-2,3.197827E-1,-2.7359618E-2,-1.0512955E-2,-1.0997185E-2,-6.5679345E-3,-7.6037426E-3,-1.6989125E-3,-6.4740097E-3,-7.6238276E-4,1.1685205E-2,-1.6996297E-3,1.3675571E-3,7.543167E-3,1.0335041E-2,2.7820729E-3,1.2137661E-2,-1.8243744E-3,1.9176172E-2,-5.554152E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,29,31,33,35,-1,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.830875E0,1.5084105E0,2.4579544E0,2.0149732E-1,8.882293E-1,1.4783487E0,1.0706367E0,8.876324E-2,0E0,6.902567E-1,4.52061E-1,5.305791E-1,7.935562E-1,1.271056E-1,0E0,2.0441532E-2,3.8679406E-2,4.746586E-1,6.158659E-1,2.3472083E-1,0E0,1.4907199E-1,3.6987996E-1,0E0,1.0580323E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,30,32,34,36,-1,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.41718E5,1.0434409E3,1.2852292E7,7.164074E8,6.9428994E3,3.236383E1,2.9778808E7,1.6943301E-3,2.0964778E5,1E0,3.0561674E9,1.37893E6,1.2799757E9,9.489792E-3,9.703059E6,2.7011695E4,1.4675482E3,1.36E2,2.9920125E3,-1.3009269E-2,1.5963512E-2,1.773296E6,3.248129E-2,1.5096262E7,-2.7359618E-2,-1.0512955E-2,-1.0997185E-2,-6.5679345E-3,-7.6037426E-3,-1.6989125E-3,-6.4740097E-3,-7.6238276E-4,1.1685205E-2,-1.6996297E-3,1.3675571E-3,7.543167E-3,1.0335041E-2,2.7820729E-3,1.2137661E-2,-1.8243744E-3,1.9176172E-2,-5.554152E-3],"split_indices":[2,1,67,9,7,61,65,12,0,37,18,12,38,7,0,9,42,61,3,64,0,66,11,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,3.48E2,1.4E2,7.7E1,2.71E2,1.3E2,1E1,7.4E1,3E0,1.96E2,7.5E1,1E2,3E1,7E0,3E0,6.4E1,1E1,1.62E2,3.4E1,7.2E1,3E0,3.8E1,6.2E1,8E0,2.2E1,3E0,4E0,5.6E1,8E0,5E0,5E0,1.21E2,4.1E1,9E0,2.5E1,5.6E1,1.6E1,6E0,3.2E1,5.8E1,4E0,1.8E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.3726824E-3,-8.5312314E-2,1.8261543E-1,-1.7494725E-1,-4.030824E-2,2.1477589E-1,-2.0498818E-1,-2.0211312E-1,-4.5311723E-2,1.7176989E-3,-1.1374238E-1,1.5998612E-1,4.2170453E-1,-3.4658757E-1,9.164092E-3,-2.4204604E-1,-1.6296114E-1,5.0600213E-3,-7.932522E-2,-2.5993159E-2,1.0326692E-1,-1.34878E-1,3.9367676E-3,9.789015E-2,2.5910148E-1,-4.7470024E-3,4.817062E-1,-1.6520885E-1,-2.6420606E-2,-1.2060605E-2,-4.432485E-3,-2.1427614E-3,-8.312467E-3,-5.9600407E-3,-2.4546615E-3,-2.7732125E-3,3.7817103E-3,7.834111E-3,5.8131922E-5,-2.6216016E-3,-7.968659E-3,2.4372325E-3,-5.5075856E-3,2.456323E-3,8.06052E-3,1.6264547E-2,8.442525E-3,9.073233E-3,2.6778135E-2,-2.8400288E-3,-1.0390352E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.78463E0,1.2999437E0,2.0759058E0,3.7118173E-1,6.6988736E-1,1.6445069E0,8.153794E-1,9.970307E-2,1.14815354E-1,3.9383247E-1,1.9702148E-1,7.193508E-1,1.0273108E0,3.2317626E-1,0E0,1.0561776E-1,9.566152E-2,0E0,1.5246019E-2,4.1856134E-1,2.0763269E-1,1.9409835E-1,8.610439E-2,2.5348872E-1,2.6500678E-1,0E0,7.0505476E-1,3.0562937E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,9.50639E5,1.0434409E3,1.1482625E7,1.8071064E7,6.651788E3,3.236383E1,1.86005E5,2.409091E0,1.2833E4,6.7204064E8,5.9161E4,5.3140095E1,8.91995E5,9.164092E-3,7.6E1,1.43479E5,5.0600213E-3,5.90495E5,2.4259783E3,1.2035191E1,2.6E1,3.149453E1,1.5271514E5,1.1291153E7,-4.7470024E-3,6.274818E4,2.2201815E9,-2.6420606E-2,-1.2060605E-2,-4.432485E-3,-2.1427614E-3,-8.312467E-3,-5.9600407E-3,-2.4546615E-3,-2.7732125E-3,3.7817103E-3,7.834111E-3,5.8131922E-5,-2.6216016E-3,-7.968659E-3,2.4372325E-3,-5.5075856E-3,2.456323E-3,8.06052E-3,1.6264547E-2,8.442525E-3,9.073233E-3,2.6778135E-2,-2.8400288E-3,-1.0390352E-2],"split_indices":[2,1,67,9,54,61,65,9,67,10,7,2,65,9,0,8,12,0,1,64,62,8,67,37,54,0,37,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,3.23E2,1.62E2,1.07E2,2.16E2,1.5E2,1.2E1,8.8E1,1.9E1,1.38E2,7.8E1,1.2E2,3E1,9E0,3E0,4.1E1,4.7E1,3E0,1.6E1,1.09E2,2.9E1,6.6E1,1.2E1,7.5E1,4.5E1,3E0,2.7E1,6E0,3E0,3.5E1,6E0,7E0,4E1,4E0,1.2E1,8.3E1,2.6E1,1.7E1,1.2E1,2.3E1,4.3E1,9E0,3E0,4.9E1,2.6E1,1.8E1,2.7E1,8E0,1.9E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[3.811055E-3,-7.425608E-2,2.0863526E-1,-8.729454E-2,3.7647733E-1,1.5081261E-1,4.1079882E-1,-1.2567924E-1,2.237106E-2,8.533514E-3,2.4216177E-2,2.2985299E-1,9.0031795E-2,2.8425124E-1,5.9203565E-1,-1.1823702E-1,-2.551382E-2,-2.2044735E-2,1.987914E-1,-2.542699E-3,2.5156286E-1,4.077246E-2,2.1512817E-1,1.4733343E-2,2.8536082E-3,1.3503745E-2,3.0500295E-2,-6.08231E-3,1.0733349E-4,-6.513869E-3,2.0077602E-4,2.9934722E-3,1.011815E-2,2.5245124E-3,1.293969E-2,3.1882767E-3,-9.35156E-3,4.6461895E-3,1.1086717E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.706701E0,2.0838923E0,1.4927678E0,1.4340353E0,1.9145668E-1,4.8374033E-1,4.9683046E-1,7.440038E-1,7.01773E-1,0E0,0E0,2.928145E-1,3.6712068E-1,1.3889718E-1,4.656744E-2,4.4095945E-1,0E0,2.2956732E-1,3.9108515E-2,0E0,2.4176097E-1,3.330137E-1,2.7819991E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,4.0297854E3,4.026531E9,9.570385E4,7.0331914E3,4.08326E5,8.010436E0,5.825042E6,1.5271514E5,8.533514E-3,2.4216177E-2,1.5963967E0,3.2318832E7,5.9108553E10,2.502264E7,2.0964778E5,-2.551382E-2,1.6861708E3,1.114E3,-2.542699E-3,2.4390244E-3,6.6901914E3,2.678316E3,1.4733343E-2,2.8536082E-3,1.3503745E-2,3.0500295E-2,-6.08231E-3,1.0733349E-4,-6.513869E-3,2.0077602E-4,2.9934722E-3,1.011815E-2,2.5245124E-3,1.293969E-2,3.1882767E-3,-9.35156E-3,4.6461895E-3,1.1086717E-2],"split_indices":[7,61,7,42,4,9,62,56,37,0,0,62,9,5,60,37,0,64,0,0,66,4,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.8E2,3.48E2,1.32E2,3.39E2,9E0,1.04E2,2.8E1,2.51E2,8.8E1,5E0,4E0,4.4E1,6E1,1.8E1,1E1,2.48E2,3E0,7.1E1,1.7E1,3E0,4.1E1,4.4E1,1.6E1,1.5E1,3E0,3E0,7E0,2.21E2,2.7E1,1.2E1,5.9E1,3E0,1.4E1,6E0,3.5E1,4E1,4E0,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[7.0903655E-3,-7.571805E-2,1.826203E-1,-1.5553194E-1,-3.216871E-2,1.5030396E-1,3.2075886E-2,-1.7587933E-1,2.2341434E-2,1.9751757E-2,-8.927145E-2,-2.4831094E-1,1.7450981E-1,-2.5317472E-1,-1.5495977E-1,-1.0615393E-3,4.568957E-3,-1.6568152E-2,1.2803039E-1,-1.1371014E-1,7.595659E-2,2.9303753E-3,-1.8998235E-2,1.20274745E-1,2.4060974E-1,-1.2638962E-2,-5.6873625E-3,-1.2018092E-3,-7.517885E-3,-3.0773175E-3,3.3239902E-3,1.03559755E-2,2.9469247E-3,-2.877763E-3,-8.096997E-3,1.5075971E-2,-1.058307E-3,-2.0273305E-4,6.514007E-3,1.2790132E-2,-2.8218073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,-1,-1,29,31,33,35,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.165223E0,1.1594042E0,2.5411515E0,4.3100142E-1,6.480899E-1,1.4906542E0,0E0,1.3216877E-1,4.8196934E-2,4.5437828E-1,4.2858464E-1,4.9560875E-1,4.801216E-1,2.7974725E-2,9.888363E-2,0E0,0E0,3.9819413E-1,1.667746E-1,2.8130496E-1,3.5925037E-1,0E0,0E0,2.2082853E-1,7.5773406E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,-1,-1,30,32,34,36,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,1.039769E6,2.3767982E4,1.8126572E7,1.4074289E7,1.6857977E4,3.2075886E-2,1.44E2,4.17E2,2.4374688E3,1.7081438E3,9.0301914E4,6.8963E4,5.2E1,2E1,-1.0615393E-3,4.568957E-3,2.306533E6,6.088569E6,5.050505E-3,7.6E1,2.9303753E-3,-1.8998235E-2,4.3871765E0,7.217412E2,-1.2638962E-2,-5.6873625E-3,-1.2018092E-3,-7.517885E-3,-3.0773175E-3,3.3239902E-3,1.03559755E-2,2.9469247E-3,-2.877763E-3,-8.096997E-3,1.5075971E-2,-1.058307E-3,-2.0273305E-4,6.514007E-3,1.2790132E-2,-2.8218073E-3],"split_indices":[2,1,61,9,54,42,0,0,0,64,61,37,2,10,10,0,0,1,54,66,3,0,0,62,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.91E2,3.34E2,1.57E2,1.17E2,2.17E2,1.49E2,8E0,1.05E2,1.2E1,1.14E2,1.03E2,8E0,1.41E2,2E1,8.5E1,8E0,4E0,8.6E1,2.8E1,9E1,1.3E1,3E0,5E0,7.9E1,6.2E1,1.6E1,4E0,6E0,7.9E1,5.5E1,3.1E1,1E1,1.8E1,5.1E1,3.9E1,3E0,1E1,1.2E1,6.7E1,5.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-4.186655E-4,-7.713907E-2,1.7703274E-1,-1.0109513E-1,7.816912E-2,2.1298611E-1,-2.0994718E-1,-1.3058214E-1,-2.1579038E-2,3.7946194E-2,2.027015E-1,1.6340691E-1,3.9661357E-1,-2.2364609E-2,-2.3137348E-2,-1.4183837E-1,-3.509136E-2,-1.0775128E-1,7.398469E-2,9.5276386E-2,-2.4677124E-2,5.91718E-3,1.2123017E-2,1.1914915E-1,2.5231168E-1,3.1725693E-1,2.9045414E-2,-7.3839086E-3,8.773123E-3,-7.1266564E-3,-1.9578135E-3,5.9190067E-3,-3.406755E-3,-1.4518726E-3,-7.3776706E-3,-2.0646208E-3,5.6005726E-3,2.9115404E-3,9.063828E-3,7.989086E-4,-6.302295E-3,1.6392851E-2,4.5931037E-3,1.4575703E-2,5.2980892E-3,1.5716385E-2,4.897596E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.6301303E0,1.272897E0,2.0899615E0,6.888664E-1,2.2272483E-1,1.1642447E0,7.0967895E-1,2.2611904E-1,6.7500216E-1,1.3159859E-1,1.5975684E-2,3.948877E-1,3.2768393E-1,2.9872975E-1,0E0,2.57622E-1,1.6493964E-1,1.6967607E-1,2.3034358E-1,4.921536E-2,8.8139996E-2,0E0,0E0,2.9627037E-1,2.794881E-1,8.98509E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,3.44E2,1.0434409E3,1.0980201E3,3.4540886E3,4.026531E9,7.339209E6,7.164074E8,9.0301914E4,1.16E2,1E0,1.486066E0,5.22E2,3.236383E1,-2.3137348E-2,2.1592189E5,4.45E3,7.094791E1,1.947899E6,2.9624592E3,3.6377136E-2,5.91718E-3,1.2123017E-2,7.201152E-4,1.6282871E7,4.547513E7,2.9045414E-2,-7.3839086E-3,8.773123E-3,-7.1266564E-3,-1.9578135E-3,5.9190067E-3,-3.406755E-3,-1.4518726E-3,-7.3776706E-3,-2.0646208E-3,5.6005726E-3,2.9115404E-3,9.063828E-3,7.989086E-4,-6.302295E-3,1.6392851E-2,4.5931037E-3,1.4575703E-2,5.2980892E-3,1.5716385E-2,4.897596E-3],"split_indices":[2,8,67,61,4,7,56,7,37,3,23,48,3,65,0,37,2,65,1,4,66,0,0,66,52,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,3.39E2,1.46E2,2.94E2,4.5E1,1.34E2,1.2E1,2.14E2,8E1,3.5E1,1E1,1.07E2,2.7E1,8E0,4E0,1.91E2,2.3E1,4.2E1,3.8E1,1.8E1,1.7E1,6E0,4E0,7.3E1,3.4E1,2.2E1,5E0,5E0,3E0,1.67E2,2.4E1,4E0,1.9E1,1.8E1,2.4E1,1.1E1,2.7E1,1.5E1,3E0,1.3E1,4E0,4E0,6.9E1,2.2E1,1.2E1,1.9E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.1915521E-2,-8.040431E-2,1.5033385E-1,-1.4621255E-1,-4.7256604E-2,7.237246E-2,2.7856612E-1,-1.9948977E-1,-7.466696E-2,-3.7908748E-2,-1.5348245E-2,4.0055275E-2,1.7462909E-1,5.272563E-1,2.2099991E-1,-2.0988579E-1,-2.4506438E-3,3.3626903E-2,-1.281063E-1,-7.793448E-2,2.7309678E-2,-1.2921758E-1,6.640274E-2,1.9577096E-1,1.5556162E-3,1.2005338E-2,3.0204128E-2,1.0053219E-1,2.731036E-1,-1.054499E-2,-5.6700315E-3,4.395253E-3,-8.5382996E-4,-1.8410172E-3,-7.579894E-3,-4.5135356E-3,3.7918207E-3,3.156403E-3,-4.751555E-3,-6.7207767E-3,-2.4159597E-3,1.8244123E-3,7.254784E-3,3.1048192E-3,9.615928E-3,-2.7410374E-3,6.0573164E-3,-6.6352956E-4,1.3527064E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,-1,31,33,35,37,39,41,43,-1,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.197006E0,6.278758E-1,1.9192896E0,3.5484457E-1,5.12515E-1,3.9617205E-1,9.285555E-1,7.220411E-2,2.5315535E-1,4.981072E-1,0E0,4.2781198E-1,7.957041E-2,2.8410387E-1,3.5542965E-1,6.1550856E-2,0E0,5.0746545E-2,8.898395E-2,4.0087473E-1,4.0604046E-1,9.779111E-3,1.9414577E-1,3.166777E-2,0E0,0E0,0E0,1.1007352E-1,2.7601218E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,-1,32,34,36,38,40,42,44,-1,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,9.50639E5,5.6503E4,1.49529E6,4.166257E7,1.5271514E5,1.417988E6,1.7626338E6,1.16E2,2.1721223E3,-1.5348245E-2,3.1547388E1,3.8158787E12,1.27E2,6.219584E5,7.6E1,-2.4506438E-3,9.1E1,3.6319303E-3,2.889654E5,3.5301748E7,4.016E4,1.3210104E9,3.059068E-1,1.5556162E-3,1.2005338E-2,3.0204128E-2,5.9209933E9,2.5111986E-3,-1.054499E-2,-5.6700315E-3,4.395253E-3,-8.5382996E-4,-1.8410172E-3,-7.579894E-3,-4.5135356E-3,3.7918207E-3,3.156403E-3,-4.751555E-3,-6.7207767E-3,-2.4159597E-3,1.8244123E-3,7.254784E-3,3.1048192E-3,9.615928E-3,-2.7410374E-3,6.0573164E-3,-6.6352956E-4,1.3527064E-2],"split_indices":[42,1,2,9,60,37,38,59,3,64,0,65,40,3,41,8,0,3,51,37,54,10,7,50,0,0,0,12,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,2.9E2,1.93E2,9.6E1,1.94E2,1.21E2,7.2E1,5.4E1,4.2E1,1.89E2,5E0,9.3E1,2.8E1,1.2E1,6E1,5E1,4E0,1.4E1,2.8E1,1.17E2,7.2E1,1.2E1,8.1E1,2.4E1,4E0,5E0,7E0,1.9E1,4.1E1,3.9E1,1.1E1,6E0,8E0,9E0,1.9E1,1.04E2,1.3E1,5.5E1,1.7E1,9E0,3E0,6.4E1,1.7E1,3E0,2.1E1,3E0,1.6E1,3E0,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.6331411E-3,-4.6679188E-2,2.5098866E-1,-7.643797E-2,1.08683586E-1,4.4498965E-1,1.8849647E-1,-7.1285486E-2,-2.2665242E-2,1.637912E-1,-4.7369555E-2,2.1711913E-1,2.87548E-2,2.4021049E-1,5.609502E-2,-8.587654E-2,1.01043254E-1,1.9725366E-1,4.4177487E-2,-9.054679E-3,9.762862E-3,4.6153683E-3,1.1556524E-2,2.7829695E-1,2.927351E-2,1.296862E-1,-1.154535E-2,-6.787311E-3,-2.5230122E-3,-3.982445E-3,7.835241E-3,1.1015057E-3,9.92661E-3,5.146495E-3,-1.9609812E-3,-2.4372693E-3,2.9144476E-3,5.4038926E-3,1.4824232E-2,-4.4156215E-3,5.6758607E-3,-4.969474E-4,7.4570864E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,27,29,31,33,-1,35,-1,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.988246E0,1.8946579E0,8.981285E-1,7.04931E-1,5.7534003E-1,6.5748525E-1,4.1788697E-1,8.635926E-1,0E0,1.8741167E-1,1.5670021E-1,1.5708327E-2,0E0,3.5337615E-1,4.639179E-1,5.945926E-1,3.7124255E-1,1.24203205E-1,7.613388E-2,0E0,5.0971065E-2,0E0,0E0,2.3964834E-1,1.07305855E-1,7.658258E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,28,30,32,34,-1,36,-1,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.5522031E9,1.417988E6,3.372042E7,1.3508742E7,9.256843E3,1.5096262E7,2.48374E5,-2.2665242E-2,1.59E2,2.7011695E4,8.963466E6,2.87548E-2,2.2576077E10,1.92144E5,4.407E3,2.8346753E0,1.504E4,6.12E2,-9.054679E-3,4.2464733E-2,4.6153683E-3,1.1556524E-2,2.1360708E7,8.4E1,6.873727E0,-1.154535E-2,-6.787311E-3,-2.5230122E-3,-3.982445E-3,7.835241E-3,1.1015057E-3,9.92661E-3,5.146495E-3,-1.9609812E-3,-2.4372693E-3,2.9144476E-3,5.4038926E-3,1.4824232E-2,-4.4156215E-3,5.6758607E-3,-4.969474E-4,7.4570864E-3],"split_indices":[61,7,38,57,54,4,9,11,0,3,42,1,0,12,2,2,62,9,0,0,47,0,0,1,3,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.87E2,4.07E2,8E1,3.42E2,6.5E1,1.8E1,6.2E1,3.39E2,3E0,4.8E1,1.7E1,9E0,9E0,4.4E1,1.8E1,3.13E2,2.6E1,3.7E1,1.1E1,4E0,1.3E1,3E0,6E0,3.7E1,7E0,1.5E1,3E0,1.02E2,2.11E2,7E0,1.9E1,4E0,3.3E1,6E0,5E0,6E0,7E0,9E0,2.8E1,3E0,4E0,3E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[7.9457E-3,-6.2385272E-2,1.587954E-1,-1.311308E-1,-2.7889103E-2,1.8439783E-1,-2.0010954E-1,-1.677639E-1,-5.4413773E-2,1.1632316E-2,-9.404997E-2,1.465143E-1,3.285226E-1,-3.223903E-1,5.08202E-4,-1.7740048E-1,-1.1793001E-3,-9.398161E-2,5.865846E-3,-1.7729132E-2,1.1403387E-1,4.6250675E-4,-1.5112284E-1,9.041029E-2,2.0801683E-1,2.6238462E-2,2.1342357E-1,-1.813062E-2,-7.6872855E-3,-8.463978E-3,-2.0911233E-4,6.941544E-4,-5.0846897E-3,1.8194494E-3,-2.6198428E-3,-2.0833449E-3,5.5245976E-3,2.3936636E-3,9.162659E-3,-3.2341685E-3,5.278153E-3,-7.95616E-3,-9.4451243E-4,-2.9197796E-3,4.917703E-3,1.3120946E-2,5.960282E-3,1.2363313E-2,-5.0094384E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,33,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.272172E0,8.0026686E-1,1.4887362E0,3.0844438E-1,5.946471E-1,7.5994015E-1,2.876831E-1,9.94215E-2,9.118635E-2,4.3197381E-1,4.5856822E-1,3.9180708E-1,7.2309065E-1,1.3217747E-2,0E0,9.18951E-2,0E0,4.4692263E-2,3.650357E-2,4.3654162E-1,1.5977788E-1,2.778289E-1,1.5366054E-1,1.7516798E-1,3.054366E-1,0E0,4.326656E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,34,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2934E4,1.035048E6,1.0434409E3,3.006846E6,1.8071064E7,7.710634E3,1.31805E5,1.7367111E6,3.1633773E8,7.3229E4,1.4E4,4.1331047E-1,3.0418E4,4.4997E4,5.08202E-4,1.6120845E5,-1.1793001E-3,7.093639E2,2.678316E3,1.4777102E9,4.95E9,2.392867E6,2.8884522E8,4.3871765E0,6.148067E-1,2.6238462E-2,5.01E2,-1.813062E-2,-7.6872855E-3,-8.463978E-3,-2.0911233E-4,6.941544E-4,-5.0846897E-3,1.8194494E-3,-2.6198428E-3,-2.0833449E-3,5.5245976E-3,2.3936636E-3,9.162659E-3,-3.2341685E-3,5.278153E-3,-7.95616E-3,-9.4451243E-4,-2.9197796E-3,4.917703E-3,1.3120946E-2,5.960282E-3,1.2363313E-2,-5.0094384E-3],"split_indices":[2,1,67,9,54,61,37,60,7,11,11,44,10,10,0,37,0,56,4,7,5,1,12,62,43,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.95E2,3.38E2,1.57E2,1.12E2,2.26E2,1.47E2,1E1,7.5E1,3.7E1,1.42E2,8.4E1,1.18E2,2.9E1,6E0,4E0,7E1,5E0,2.2E1,1.5E1,1.11E2,3.1E1,3.2E1,5.2E1,6.3E1,5.5E1,8E0,2.1E1,3E0,3E0,6.7E1,3E0,3E0,1.9E1,1E1,5E0,9.3E1,1.8E1,1.9E1,1.2E1,2E1,1.2E1,4.4E1,8E0,6E0,5.7E1,2.6E1,2.9E1,1.8E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[8.981127E-3,-5.680763E-2,1.53917E-1,-1.3481705E-1,-1.909911E-2,1.7986703E-1,-1.9506668E-1,-1.6923153E-1,-6.327448E-2,-5.210065E-2,4.461025E-2,1.1324777E-1,2.8318754E-1,-3.8021183E-1,4.720301E-3,-1.8663195E-1,-9.55781E-2,5.965641E-2,-9.3946904E-2,-7.809769E-2,6.9222815E-2,-2.8577497E-2,1.2240203E-1,8.303852E-2,2.2611493E-1,2.1805555E-1,2.7084261E-2,-2.1265078E-2,-9.183333E-3,-3.36892E-3,-9.072514E-3,-2.6646662E-3,-6.7782574E-3,5.4364405E-3,1.8801911E-5,-5.335444E-3,-7.824428E-4,-1.3770444E-3,-5.305012E-3,8.062002E-3,1.0516549E-3,-2.7704714E-3,1.7110998E-3,2.796911E-3,7.308661E-3,7.626032E-3,1.2788461E-3,7.373017E-3,1.3968081E-2,1.1198259E-2,-5.9326994E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.642827E0,9.828676E-1,1.4116187E0,2.5766098E-1,4.7959334E-1,9.4533443E-1,6.465657E-1,7.571888E-2,1.4511633E-1,4.785138E-1,4.475835E-1,2.8127134E-1,9.7667265E-1,1.2916923E-2,0E0,6.15983E-2,2.2773698E-2,2.7982784E-2,5.021432E-2,2.2170085E-1,1.294825E-1,8.8717684E-2,7.712346E-2,3.2238427E-1,4.338813E-2,4.6728802E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.227E4,9.50639E5,1.4246185E3,3.006846E6,1.0980201E3,9.3058E4,3.5226266E0,1.6395413E7,5.9073865E2,3.94E2,8.459549E4,4.23218E3,1.261448E1,1.2799757E9,4.720301E-3,6.738547E7,1.7509084E-2,1.19E2,1.425263E10,1.0270569E7,1.3212844E-3,1.91E2,2.0584187E3,7.8920895E6,1.0963892E5,2.0531621E4,2.7084261E-2,-2.1265078E-2,-9.183333E-3,-3.36892E-3,-9.072514E-3,-2.6646662E-3,-6.7782574E-3,5.4364405E-3,1.8801911E-5,-5.335444E-3,-7.824428E-4,-1.3770444E-3,-5.305012E-3,8.062002E-3,1.0516549E-3,-2.7704714E-3,1.7110998E-3,2.796911E-3,7.308661E-3,7.626032E-3,1.2788461E-3,7.373017E-3,1.3968081E-2,1.1198259E-2,-5.9326994E-3],"split_indices":[2,1,67,9,61,2,63,12,56,8,37,61,62,7,0,7,66,3,40,54,48,8,64,54,37,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,3.34E2,1.51E2,1.08E2,2.26E2,1.41E2,1E1,7.2E1,3.6E1,1.49E2,7.7E1,8.7E1,5.4E1,6E0,4E0,5.7E1,1.5E1,7E0,2.9E1,1.23E2,2.6E1,4E1,3.7E1,7E1,1.7E1,4.6E1,8E0,3E0,3E0,6E0,5.1E1,1E1,5E0,3E0,4E0,2.2E1,7E0,5.5E1,6.8E1,7E0,1.9E1,2.7E1,1.3E1,1.5E1,2.2E1,2.7E1,4.3E1,1.1E1,6E0,4.3E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[1.2364988E-2,-2.8951822E-2,2.1983851E-1,-5.6487415E-2,1.20740294E-1,1.361953E-1,3.4495717E-1,-9.905442E-2,-3.0439734E-3,1.681064E-1,3.3114873E-2,1.7530361E-1,-8.614769E-2,2.694046E-2,2.1847898E-1,-1.4681533E-1,-5.6933086E-2,9.104628E-3,-2.0535236E-2,1.278845E-1,2.5130045E-1,-3.3204485E-2,1.08481854E-1,-3.7228284E-4,1.9482705E-1,-6.722885E-3,5.1909825E-4,1.1623707E-2,-2.239239E-4,-7.4176374E-3,-2.1682174E-3,-3.230565E-3,3.4071826E-3,-5.8270217E-4,6.791614E-3,8.332986E-3,2.9812614E-3,5.682349E-3,1.28077185E-2,2.8327599E-3,-4.59374E-3,1.1090884E-3,9.368738E-3,1.3574786E-2,7.001541E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.078133E0,1.6431993E0,7.7977943E-1,7.6407385E-1,2.517627E-1,4.4117945E-1,8.2710433E-1,3.6783922E-1,8.1628054E-1,1.0171139E-1,1.1856275E-1,1.5200198E-1,4.905771E-2,0E0,1.7582846E-1,1.24485135E-1,1.8656024E-1,4.4909698E-1,0E0,8.676696E-2,8.844495E-3,8.9274526E-2,8.4230855E-2,0E0,1.16438866E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.613448E9,7.3073276E3,2.025313E6,1.1532659E7,9.234E3,3.0418E4,1.38093E6,3.372042E7,3.0467994E9,8.4E1,3.6308475E1,5.767668E0,2.694046E-2,1.4656219E8,1.4910261E5,4.8384636E7,2.481428E5,-2.0535236E-2,7.200105E8,8.130405E6,1.3185959E7,1.2511627E7,-3.7228284E-4,2.1207E4,-6.722885E-3,5.1909825E-4,1.1623707E-2,-2.239239E-4,-7.4176374E-3,-2.1682174E-3,-3.230565E-3,3.4071826E-3,-5.8270217E-4,6.791614E-3,8.332986E-3,2.9812614E-3,5.682349E-3,1.28077185E-2,2.8327599E-3,-4.59374E-3,1.1090884E-3,9.368738E-3,1.3574786E-2,7.001541E-3],"split_indices":[61,7,61,1,54,0,10,9,57,7,3,65,63,0,1,56,9,37,0,5,1,54,1,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.74E2,3.96E2,7.8E1,3.35E2,6.1E1,4.8E1,3E1,1.86E2,1.49E2,3.9E1,2.2E1,4.1E1,7E0,9E0,2.1E1,8.6E1,1E2,1.46E2,3E0,2.8E1,1.1E1,1.2E1,1E1,4E0,3.7E1,4E0,3E0,1.8E1,3E0,7.4E1,1.2E1,9.1E1,9E0,1.27E2,1.9E1,1.4E1,1.4E1,3E0,8E0,5E0,7E0,6E0,4E0,9E0,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.9450533E-3,-5.493704E-2,1.3884094E-1,-7.210964E-2,6.267588E-2,8.088057E-2,2.6584446E-1,-7.694234E-2,1.42220855E-2,3.3750013E-2,8.797171E-3,1.0633435E-1,-1.9112858E-1,2.068239E-1,4.7894627E-1,-1.1165362E-1,-3.4321804E-2,9.218999E-2,-2.3455465E-2,1.2517339E-1,-6.2547E-2,-2.8714624E-3,-1.1205209E-2,6.212401E-4,2.2347593E-1,2.7441924E-2,1.2009283E-2,-6.059403E-3,-1.6336644E-3,7.6565094E-4,-4.247392E-3,6.912194E-3,2.443928E-3,-5.3412314E-3,1.6355899E-3,4.681538E-3,1.0626707E-2,1.9636801E-3,-1.0592064E-2,1.1875931E-2,6.2131984E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,29,31,33,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.8465817E0,7.094017E-1,1.0458679E0,5.882914E-1,1.6215989E-1,7.1961343E-1,4.669249E-1,4.4295073E-1,0E0,1.2919965E-1,0E0,3.037144E-1,4.6718657E-2,1.1603129E-1,7.787657E-2,2.5700855E-1,4.1067523E-1,3.530577E-2,1.145898E-1,1.818918E-1,1.880668E-1,0E0,0E0,0E0,7.223666E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,30,32,34,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,3.44E2,1.6452132E0,3.059144E3,1.3581547E5,2.8659653E2,2.3767982E4,1.732734E6,1.42220855E-2,1.3404826E-3,8.797171E-3,4.17E2,2.9991518E7,7.7444346E8,2.3879834E9,4.838871E8,1.8071064E7,7.202981E-4,1.570662E5,6.0500806E3,1.4864173E0,-2.8714624E-3,-1.1205209E-2,6.212401E-4,1.22342285E2,2.7441924E-2,1.2009283E-2,-6.059403E-3,-1.6336644E-3,7.6565094E-4,-4.247392E-3,6.912194E-3,2.443928E-3,-5.3412314E-3,1.6355899E-3,4.681538E-3,1.0626707E-2,1.9636801E-3,-1.0592064E-2,1.1875931E-2,6.2131984E-3],"split_indices":[2,8,48,61,42,67,61,1,0,66,0,3,54,7,5,7,54,48,56,61,51,0,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,3.48E2,1.44E2,3.04E2,4.4E1,1E2,4.4E1,3.01E2,3E0,3.7E1,7E0,9.2E1,8E0,3.6E1,8E0,1.65E2,1.36E2,1.8E1,1.9E1,8.3E1,9E0,3E0,5E0,3E0,3.3E1,4E0,4E0,1.29E2,3.6E1,7.3E1,6.3E1,6E0,1.2E1,7E0,1.2E1,7E1,1.3E1,6E0,3E0,2.2E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.0725288E-2,-7.705238E-2,9.0203024E-2,-7.2087616E-2,-1.905939E-2,5.771804E-2,2.1616417E-1,-1.2546113E-1,-4.274621E-2,2.280405E-2,1.715249E-1,1.6125014E-1,1.3043192E-2,-1.3945292E-1,3.4615517E-2,-6.437953E-2,9.452218E-2,-1.0550412E-2,9.417936E-2,1.6783759E-2,1.2820336E-1,3.8665987E-3,8.0998875E-3,-6.96752E-3,-7.6299E-4,-2.7900524E-3,4.236126E-3,-1.2459311E-3,-5.07764E-3,8.317614E-3,-1.025135E-3,-2.0560804E-3,2.4633948E-3,4.0829927E-4,6.8364986E-3,-2.0015077E-3,7.835239E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2402885E0,4.620669E-1,7.7315795E-1,4.4722664E-1,0E0,6.08E-1,1.01515174E-1,2.332828E-1,5.643647E-1,2.8429517E-1,2.5747156E-1,1.4061272E-2,0E0,1.4523685E-1,5.549959E-2,2.7905858E-1,2.6927572E-1,1.8391962E-1,1.7754921E-1,0E0,2.3676819E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,7.339209E6,1.7670108E5,4.407E3,-1.905939E-2,4.0297854E3,5.6503E4,2.5653894E5,4.4537253E11,1.484798E9,1.37893E6,1.901216E3,1.3043192E-2,1.73E2,2.455358E6,1.5757263E7,8.213E3,1.1762006E9,8.4E1,1.6783759E-2,2.696912E6,3.8665987E-3,8.0998875E-3,-6.96752E-3,-7.6299E-4,-2.7900524E-3,4.236126E-3,-1.2459311E-3,-5.07764E-3,8.317614E-3,-1.025135E-3,-2.0560804E-3,2.4633948E-3,4.0829927E-4,6.8364986E-3,-2.0015077E-3,7.835239E-3],"split_indices":[42,56,37,2,0,61,2,37,40,7,38,4,0,8,1,54,0,12,3,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,2.91E2,1.91E2,2.88E2,3E0,1.53E2,3.8E1,1.01E2,1.87E2,1.18E2,3.5E1,2.3E1,1.5E1,9.3E1,8E0,1.62E2,2.5E1,8.1E1,3.7E1,5E0,3E1,5E0,1.8E1,8.4E1,9E0,3E0,5E0,9.1E1,7.1E1,1.4E1,1.1E1,5.3E1,2.8E1,1.5E1,2.2E1,6E0,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.861175E-3,-5.929335E-2,1.0204459E-1,-7.5781204E-2,4.550353E-2,5.1975448E-2,2.0294736E-1,-9.8787665E-2,8.559617E-3,-1.0079723E-1,1.09517545E-1,2.202143E-2,1.420854E-1,4.772966E-1,1.6013056E-1,-1.0877593E-1,3.140164E-3,1.0705083E-1,-3.3513136E-2,-1.438201E-1,1.4625058E-3,1.6723658E-1,-6.3888445E-3,-5.2665467E-3,3.71794E-2,1.5974815E-1,6.7530095E-4,1.1362329E-2,2.585709E-2,1.8996252E-1,-4.2606503E-2,-6.411986E-3,-3.1940106E-3,-3.427091E-3,1.3243823E-3,2.0602585E-3,6.679779E-3,-4.5255586E-3,2.011459E-3,-2.250889E-3,-1.0461803E-2,-1.9175513E-3,9.663E-3,2.655218E-3,-1.9524703E-3,7.777897E-3,2.5604318E-3,-1.2422156E-3,9.367937E-3,6.2809996E-3,-1.1750732E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,-1,-1,43,45,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0258045E0,5.1662755E-1,9.5440555E-1,4.9952412E-1,3.9501628E-1,3.450621E-1,6.6361594E-1,2.0599437E-1,2.3564021E-1,7.884672E-2,4.4035348E-1,2.079503E-1,6.96097E-2,2.0450354E-2,3.467841E-1,2.1500516E-1,4.041897E-2,3.446749E-2,2.0694566E-1,6.89992E-2,0E0,2.2931367E-1,0E0,0E0,1.5059456E-1,2.0185411E-2,0E0,0E0,0E0,1.624645E-1,3.4341916E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,24,24,25,25,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,-1,-1,44,46,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,4.723148E11,5.9161E4,1.2852292E7,3.521081E2,1.5271514E5,9.73307E5,2.1592189E5,2.1037032E-4,1.5592E4,1.505649E6,2.8529238E1,3.8158787E12,1.69595E5,1.5096262E7,1.49529E6,4.4643167E2,2.7059703E-4,1.6167939E7,1.6991478E2,1.4625058E-3,8.766172E4,-6.3888445E-3,-5.2665467E-3,7.309061E1,1.134618E2,6.7530095E-4,1.1362329E-2,2.585709E-2,3.0811954E-2,5.358657E7,-6.411986E-3,-3.1940106E-3,-3.427091E-3,1.3243823E-3,2.0602585E-3,6.679779E-3,-4.5255586E-3,2.011459E-3,-2.250889E-3,-1.0461803E-2,-1.9175513E-3,9.663E-3,2.655218E-3,-1.9524703E-3,7.777897E-3,2.5604318E-3,-1.2422156E-3,9.367937E-3,6.2809996E-3,-1.1750732E-2],"split_indices":[42,40,2,9,61,37,38,37,47,0,2,65,40,2,9,9,61,48,41,42,0,37,0,0,65,67,0,0,0,47,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,2.96E2,1.9E2,2.56E2,4E1,1.28E2,6.2E1,2.01E2,5.5E1,1.2E1,2.8E1,9.7E1,3.1E1,7E0,5.5E1,1.83E2,1.8E1,1.6E1,3.9E1,9E0,3E0,2.3E1,5E0,9E0,8.8E1,2.7E1,4E0,3E0,4E0,4.8E1,7E0,1E2,8.3E1,4E0,1.4E1,7E0,9E0,2.1E1,1.8E1,5E0,4E0,4E0,1.9E1,7E1,1.8E1,2.4E1,3E0,3E0,4.5E1,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[4.4887615E-3,-5.84884E-2,9.9580616E-2,-5.3440295E-2,-1.8755564E-2,4.539443E-2,1.9491908E-1,-7.729479E-2,1.9859355E-2,2.3383748E-2,1.2441899E-1,4.1184026E-1,1.3822056E-1,-1.1124335E-1,-2.0327946E-2,-6.671155E-2,4.3811552E-2,-5.039432E-3,4.1323576E-2,1.4006451E-1,-2.5575366E-5,1.191248E-2,2.6106354E-2,1.6307187E-1,-6.258471E-2,-9.723419E-3,-4.56814E-3,-4.0396457E-3,2.2013162E-3,2.4946211E-4,-5.64899E-3,1.4479423E-3,9.022399E-3,4.4274456E-3,8.1424194E-4,1.687702E-3,7.0037595E-3,-4.0409965E-3,8.265577E-3,2.433222E-3,-7.445179E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9162912E0,4.9780035E-1,9.93361E-1,5.0860757E-1,0E0,2.1451366E-1,8.040128E-1,4.2127204E-1,1.522193E-1,2.416656E-1,5.28726E-2,1.8803501E-1,2.9465127E-1,1.3715804E-1,3.90181E-1,6.6176526E-2,1.0014336E-1,0E0,1.130494E-1,2.6510298E-2,0E0,0E0,0E0,2.4095523E-1,8.9769624E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,7.339209E6,5.6503E4,6.4065235E8,-1.8755564E-2,1.5271514E5,1.799389E6,2.09299E6,1.4698442E6,2.8529238E1,1.5953E4,1.92144E5,5.30113E5,1.2304464E1,1E0,1.2E3,1.0304171E7,-5.039432E-3,1.4040858E-2,1.436034E9,-2.5575366E-5,1.191248E-2,2.6106354E-2,4.2464733E-2,1.14633336E8,-9.723419E-3,-4.56814E-3,-4.0396457E-3,2.2013162E-3,2.4946211E-4,-5.64899E-3,1.4479423E-3,9.022399E-3,4.4274456E-3,8.1424194E-4,1.687702E-3,7.0037595E-3,-4.0409965E-3,8.265577E-3,2.433222E-3,-7.445179E-3],"split_indices":[42,56,2,7,0,37,38,1,54,65,0,2,2,65,13,11,38,0,66,5,0,0,0,47,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,2.92E2,1.93E2,2.89E2,3E0,1.24E2,6.9E1,2.18E2,7.1E1,9.8E1,2.6E1,1.3E1,5.6E1,1.36E2,8.2E1,1.5E1,5.6E1,1.1E1,8.7E1,2.3E1,3E0,8E0,5E0,5E1,6E0,1.2E1,1.24E2,4.1E1,4.1E1,7E0,8E0,5.3E1,3E0,2.5E1,6.2E1,3E0,2E1,3E0,4.7E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[3.2525742E-3,-6.0488563E-2,9.51955E-2,-7.516163E-2,5.8373902E-2,4.432295E-2,1.8322591E-1,-6.959743E-2,-1.8111851E-2,8.682172E-2,-3.4151378E-3,7.4985005E-2,-1.2433578E-2,3.8721338E-1,1.3434552E-1,-8.80858E-2,3.899151E-3,5.059156E-2,1.2513032E-2,1.155554E-2,1.0037662E-1,-5.1321693E-2,8.424744E-2,1.0414124E-2,2.4506742E-2,1.5991145E-1,-1.1475748E-2,2.4303824E-3,-4.320362E-3,-3.2976593E-3,4.402436E-3,4.2351643E-3,-6.0438603E-4,-2.9104427E-3,2.4303235E-3,7.3713595E-3,2.7681517E-3,-8.3198067E-4,-6.7943064E-3,1.5149207E-3,6.333247E-3,4.526684E-3,1.100428E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,23,25,27,29,31,-1,33,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7484941E0,4.8749673E-1,8.5171866E-1,4.1310835E-1,1.2377225E-1,2.1498576E-1,6.459489E-1,3.3189845E-1,0E0,1.6195747E-1,0E0,1.276862E-1,1.697524E-1,1.8593359E-1,6.11164E-1,1.7914617E-1,3.5750964E-1,6.278877E-2,0E0,7.832807E-2,1.2826914E-1,1.0032937E-1,3.1419843E-2,0E0,0E0,2.4645352E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,24,26,28,30,32,-1,34,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,1.577058E9,5.9161E4,5.825042E6,2.5503985E9,1.5E2,1.417988E6,5.4779566E9,-1.8111851E-2,1.150124E0,-3.4151378E-3,2.0192318E3,2.6201914E3,1.92144E5,5.22E2,6.453125E0,6.6088306E2,5.198287E5,1.2513032E-2,1.5029658E-1,1.196045E6,7.280323E1,2.5096E4,1.0414124E-2,2.4506742E-2,1.2384782E5,-1.1475748E-2,2.4303824E-3,-4.320362E-3,-3.2976593E-3,4.402436E-3,4.2351643E-3,-6.0438603E-4,-2.9104427E-3,2.4303235E-3,7.3713595E-3,2.7681517E-3,-8.3198067E-4,-6.7943064E-3,1.5149207E-3,6.333247E-3,4.526684E-3,1.100428E-2],"split_indices":[42,7,2,56,7,3,38,5,0,44,0,64,61,2,3,56,61,41,0,47,56,65,2,0,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.67E2,2.76E2,1.91E2,2.46E2,3E1,1.22E2,6.9E1,2.43E2,3E0,2.5E1,5E0,7.9E1,4.3E1,1.2E1,5.7E1,1.94E2,4.9E1,2.2E1,3E0,2.3E1,5.6E1,3.1E1,1.2E1,7E0,5E0,5.4E1,3E0,8E0,1.86E2,2.7E1,2.2E1,1.3E1,9E0,8E0,1.5E1,2.1E1,3.5E1,2.4E1,7E0,7E0,5E0,3.2E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-4.9459864E-3,-3.225036E-2,1.4205973E-1,-5.397787E-2,7.946287E-2,3.6057895E-1,9.9532194E-2,-1.3724314E-1,-4.094371E-2,9.3070515E-2,-7.2360546E-3,4.7178427E-3,2.1802342E-2,1.2579788E-1,-1.3606893E-1,-1.8908258E-1,-9.376048E-2,-1.323997E-2,-9.649728E-2,1.4207872E-2,7.728867E-2,1.4520796E-1,-3.1968618E-3,-1.2606769E-2,1.7099926E-3,-3.1347214E-3,-9.442086E-3,-1.7651236E-3,-5.9165913E-3,-1.2937819E-3,6.421428E-3,-1.2768996E-3,-5.9660044E-3,5.925823E-3,1.9349525E-3,3.0259634E-4,7.4314345E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9473085E0,9.9703467E-1,6.7038524E-1,3.6663634E-1,2.313286E-1,3.0884767E-1,4.1935194E-1,8.5960865E-2,4.5735675E-1,1.9559151E-1,0E0,0E0,0E0,2.3420924E-1,1.7955671E-1,2.7465582E-2,4.795742E-2,4.656488E-1,2.2639525E-1,0E0,1.05998695E-1,1.2755406E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.6053742E9,1E0,4.66085E5,2.4070468E7,7.0331914E3,3.768034E4,5.6E1,1.8071064E7,1.5988636E1,-7.2360546E-3,4.7178427E-3,2.1802342E-2,3.0173841E10,9.820362E4,1.13695905E-5,1.2E3,3.0224E4,6.465659E4,1.4207872E-2,6.693226E6,3.6308475E1,-3.1968618E-3,-1.2606769E-2,1.7099926E-3,-3.1347214E-3,-9.442086E-3,-1.7651236E-3,-5.9165913E-3,-1.2937819E-3,6.421428E-3,-1.2768996E-3,-5.9660044E-3,5.925823E-3,1.9349525E-3,3.0259634E-4,7.4314345E-3],"split_indices":[61,7,25,1,57,4,4,10,54,65,0,0,0,5,37,47,11,10,37,0,54,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,4.08E2,7.5E1,3.42E2,6.6E1,1.1E1,6.4E1,4.5E1,2.97E2,6.3E1,3E0,4E0,7E0,5.8E1,6E0,1.9E1,2.6E1,1.99E2,9.8E1,3E0,6E1,5.3E1,5E0,3E0,3E0,3E0,1.6E1,1.1E1,1.5E1,1.82E2,1.7E1,3.3E1,6.5E1,2.3E1,3.7E1,6E0,4.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.5933925E-3,-3.253888E-2,1.4267464E-1,-5.3235214E-2,8.148833E-2,1.6956091E-1,-6.248538E-2,-6.398041E-2,4.5084752E-2,4.5524828E-2,1.6977692E-1,2.1719005E-2,1.4912316E-1,-4.9339277E-3,2.0185405E-4,-1.0922319E-1,-3.8669392E-2,-5.7707676E-3,8.015812E-2,-6.598761E-2,7.6084815E-2,8.0703065E-2,1.306476E-2,1.6578957E-1,-7.275486E-3,-6.2423893E-3,-1.786785E-3,-1.4662199E-3,-1.5660593E-2,8.1248665E-3,7.408334E-4,1.3389378E-3,-5.0789607E-3,7.6366467E-3,2.1703024E-3,2.8472216E-4,4.9356506E-3,1.0594225E-2,4.5331274E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0361493E0,9.199968E-1,4.518739E-1,3.5046226E-1,1.8278101E-1,3.7187815E-1,3.080155E-2,3.3588314E-1,2.082471E-1,1.5496469E-1,1.5218818E-1,0E0,3.755443E-1,0E0,0E0,1.969204E-1,3.7538537E-1,0E0,1.715638E-1,4.610819E-2,8.315648E-2,2.1574982E-2,0E0,2.6212168E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,18,18,19,19,20,20,21,21,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.613448E9,2.3040023E10,2.1592189E5,1.646359E5,2.909019E3,5.767668E0,1.6858337E3,6.52487E2,3.5875823E0,6.7304E4,2.1719005E-2,1.4656219E8,-4.9339277E-3,2.0185405E-4,5.175E3,3.1822095E0,-5.7707676E-3,9.54047E0,1E0,4.1124658E6,1.9381E4,1.306476E-2,1.7001536E7,-7.275486E-3,-6.2423893E-3,-1.786785E-3,-1.4662199E-3,-1.5660593E-2,8.1248665E-3,7.408334E-4,1.3389378E-3,-5.0789607E-3,7.6366467E-3,2.1703024E-3,2.8472216E-4,4.9356506E-3,1.0594225E-2,4.5331274E-3],"split_indices":[61,7,5,37,37,41,63,4,4,62,2,0,1,0,0,2,66,0,62,13,54,2,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.66E2,3.87E2,7.9E1,3.28E2,5.9E1,7E1,9E0,2.96E2,3.2E1,4.3E1,1.6E1,3E0,6.7E1,5E0,4E0,1.05E2,1.91E2,5E0,2.7E1,9E0,3.4E1,1E1,6E0,6.4E1,3E0,7.5E1,3E1,1.88E2,3E0,1E1,1.7E1,3E0,6E0,7E0,2.7E1,3E0,7E0,3.1E1,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.442493E-5,-5.776262E-2,7.96099E-2,-6.283667E-2,1.438737E-2,4.2491842E-2,1.8943311E-1,-7.325451E-2,3.8267214E-2,1.7548846E-2,1.3147704E-1,3.6418062E-1,1.3366717E-1,-8.4854506E-2,4.982341E-3,6.902914E-2,-5.074551E-3,-5.6672534E-3,1.0878265E-1,1.5008974E-1,-2.9720174E-4,9.133591E-3,2.2675104E-2,1.8690036E-1,-2.8505234E-2,-6.626413E-3,-3.3195114E-3,-9.472531E-3,1.4723265E-3,7.233892E-3,1.6250521E-3,-1.9725943E-3,2.317019E-3,-1.8654434E-3,6.3704215E-3,3.1486354E-4,7.5848466E-3,1.0771942E-2,5.088826E-3,3.555584E-3,-1.069839E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,-1,-1,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.262819E0,5.5793935E-1,8.3577037E-1,2.9982948E-1,0E0,3.439437E-1,4.5805883E-1,2.3275483E-1,1.315414E-1,2.6114196E-1,8.828759E-2,1.60478E-1,3.5927886E-1,1.5413237E-1,2.026377E-1,6.3035086E-2,0E0,2.1085314E-1,1.1942893E-1,6.612766E-2,0E0,0E0,0E0,8.761656E-2,2.5798035E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,-1,-1,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.4023336E4,6.0500806E3,5.8145845E3,3.94E2,1.438737E-2,1.6768953E5,1.417988E6,2.0964778E5,6.843157E-2,3.53E2,6.878971E1,1.92144E5,1.5096262E7,3.586E3,2.1115233E2,4.55E3,-5.074551E-3,2.0241737E3,7.7E1,1.1100566E3,-2.9720174E-4,9.133591E-3,2.2675104E-2,1.3282626E10,5.358657E7,-6.626413E-3,-3.3195114E-3,-9.472531E-3,1.4723265E-3,7.233892E-3,1.6250521E-3,-1.9725943E-3,2.317019E-3,-1.8654434E-3,6.3704215E-3,3.1486354E-4,7.5848466E-3,1.0771942E-2,5.088826E-3,3.555584E-3,-1.069839E-2],"split_indices":[42,61,4,8,0,37,38,37,66,8,65,2,9,2,61,12,0,61,3,4,0,0,0,12,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.9E2,2.84E2,2.06E2,2.81E2,3E0,1.55E2,5.1E1,2.55E2,2.6E1,1.22E2,3.3E1,1.1E1,4E1,2.22E2,3.3E1,2.2E1,4E0,9.8E1,2.4E1,2.9E1,4E0,6E0,5E0,3E1,1E1,3.6E1,1.86E2,3E0,3E1,5E0,1.7E1,5.9E1,3.9E1,4E0,2E1,3E0,2.6E1,1.7E1,1.3E1,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.5549058E-4,-2.6828062E-2,1.4911735E-1,-5.937784E-2,3.680419E-2,3.4908044E-1,1.11836225E-1,-1.968175E-2,-9.116266E-2,5.0474245E-2,-1.2358874E-2,5.179293E-3,2.1403247E-2,1.3912542E-2,9.002669E-2,-7.240518E-2,3.0807996E-2,-1.0681791E-1,-5.0450586E-2,2.1827534E-2,1.1134035E-1,-6.558749E-3,1.0996611E-1,9.577308E-4,-4.663447E-3,3.4417894E-3,-2.7212463E-3,-5.5035064E-3,-1.5740249E-3,-1.3556101E-3,-7.016637E-3,1.8569955E-3,-4.805264E-3,6.0336613E-3,-4.5959634E-4,-5.1963767E-3,5.735795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,-1,21,23,25,27,29,31,33,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9132727E0,8.3688545E-1,5.0516427E-1,3.3508837E-1,5.883466E-1,2.5490248E-1,2.3669368E-1,3.2166982E-1,8.894014E-2,2.2772673E-1,0E0,0E0,0E0,0E0,2.8562808E-1,1.6459304E-1,2.5299674E-1,9.9090934E-2,8.7594055E-2,2.2069791E-1,1.0588443E-1,0E0,1.9980663E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,-1,22,24,26,28,30,32,34,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,7.164074E8,1.064816E6,2.2414202E1,2.4070468E7,9.256843E3,1.3812E4,7.1972437E2,8.2132086E4,1.9702967E9,-1.2358874E-2,5.179293E-3,2.1403247E-2,1.3912542E-2,1E0,5.1698097E1,2.515304E7,8.067265E5,8.42E2,2.4722598E7,7.3416536E7,-6.558749E-3,1.9813E4,9.577308E-4,-4.663447E-3,3.4417894E-3,-2.7212463E-3,-5.5035064E-3,-1.5740249E-3,-1.3556101E-3,-7.016637E-3,1.8569955E-3,-4.805264E-3,6.0336613E-3,-4.5959634E-4,-5.1963767E-3,5.735795E-3],"split_indices":[61,7,38,62,57,4,9,61,37,7,0,0,0,0,8,57,54,56,0,54,53,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.74E2,4.02E2,7.2E1,2.66E2,1.36E2,1E1,6.2E1,1.19E2,1.47E2,1.31E2,5E0,4E0,6E0,5E0,5.7E1,5.8E1,6.1E1,1.05E2,4.2E1,9E1,4.1E1,4E0,5.3E1,1.4E1,4.4E1,4.1E1,2E1,8.8E1,1.7E1,3.6E1,6E0,7.9E1,1.1E1,3.5E1,6E0,3E0,5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.9223184E-3,-3.2292016E-2,8.856483E-2,-7.672655E-2,-1.2695453E-3,1.8294528E-1,6.1408445E-2,-9.78527E-2,2.0479605E-2,7.176615E-2,-3.619046E-2,6.7582145E-2,2.8687432E-1,-9.532901E-3,7.609658E-2,6.372853E-2,-1.0724557E-1,-6.4770523E-3,4.7766354E-2,1.8448928E-1,4.33001E-2,1.4390895E-2,-6.2473044E-2,6.8103885E-3,-5.8131218E-2,1.20600605E-4,1.6120212E-2,8.896045E-2,-4.3087736E-2,4.954008E-4,4.608156E-3,-4.299025E-3,-9.118882E-3,6.331729E-4,5.644418E-3,2.9932568E-3,1.0531832E-2,2.809289E-3,-9.068442E-3,2.822291E-3,-3.1044788E-3,-4.1127214E-3,2.65562E-4,-5.2329344E-3,5.7762576E-4,3.106028E-3,8.07983E-3,2.8280574E-3,-4.125202E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,33,35,37,39,41,-1,43,-1,-1,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4595127E0,4.7271743E-1,3.5015297E-1,2.9156047E-1,5.203434E-1,3.5228956E-1,4.556075E-1,1.828891E-1,1.2170585E-1,2.0379123E-1,1.8435745E-1,1.7206083E-1,2.9645777E-1,0E0,1.6634917E-1,1.260085E-2,1.1592686E-1,0E0,5.7273462E-2,5.562389E-2,2.4803975E-1,1.9081424E-1,1.7075992E-1,0E0,2.9288288E-2,0E0,0E0,1.6534966E-1,5.9864547E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,24,24,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,34,36,38,40,42,-1,44,-1,-1,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2587E4,1.6858337E3,2.1207E4,2.1592189E5,1.24E2,1.2511627E7,1.6857977E4,1E0,5.7E3,3.34031E5,2.5212479E1,1.567619E6,1.5963967E0,-9.532901E-3,1.5953E4,1.43552E6,1.7377544E7,-6.4770523E-3,6.6826764E2,9.271896E1,1E0,3.271774E7,7.164074E8,6.8103885E-3,3.116E3,1.20600605E-4,1.6120212E-2,3.9596844E9,7.507126E7,4.954008E-4,4.608156E-3,-4.299025E-3,-9.118882E-3,6.331729E-4,5.644418E-3,2.9932568E-3,1.0531832E-2,2.809289E-3,-9.068442E-3,2.822291E-3,-3.1044788E-3,-4.1127214E-3,2.65562E-4,-5.2329344E-3,5.7762576E-4,3.106028E-3,8.07983E-3,2.8280574E-3,-4.125202E-3],"split_indices":[2,4,9,37,3,1,42,20,12,12,62,38,62,0,0,1,54,0,61,42,18,54,7,0,0,0,0,7,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,3.42E2,1.4E2,1.4E2,2.02E2,3E1,1.1E2,1.15E2,2.5E1,6.5E1,1.37E2,1.5E1,1.5E1,5E0,1.05E2,6E0,1.09E2,3E0,2.2E1,1.2E1,5.3E1,4.7E1,9E1,9E0,6E0,3E0,1.2E1,9.5E1,1E1,3E0,3E0,9.7E1,1.2E1,1.6E1,6E0,4E0,8E0,5E1,3E0,3E1,1.7E1,6.4E1,2.6E1,3E0,3E0,7.8E1,1.7E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-2.3556384E-3,-4.8239477E-2,7.271714E-2,-5.2562766E-2,1.308349E-2,2.6510501E-2,1.5894036E-1,-4.755338E-2,-1.5403449E-2,-4.3553032E-2,4.807321E-2,3.2792062E-1,1.13638744E-1,-1.08449295E-1,-3.1145226E-2,-1.5179667E-2,-9.934456E-3,6.986092E-2,-1.7477335E-2,8.737756E-3,2.0863017E-2,1.3995397E-1,-7.5524025E-2,3.2710698E-3,-5.4369243E-3,-2.5109984E-3,1.1388485E-3,-5.346932E-3,3.9426712E-4,4.651987E-3,7.073631E-4,-5.6506577E-3,1.1310908E-3,4.027987E-3,8.8140005E-3,5.807066E-4,-6.628879E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,27,-1,29,31,-1,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6707072E0,4.4985443E-1,7.288581E-1,4.080068E-1,0E0,1.8489565E-1,4.5266044E-1,2.9038393E-1,0E0,1.39497E-1,1.3404329E-1,1.4085865E-1,2.6941872E-1,1.2407112E-1,3.111194E-1,6.390381E-2,0E0,1.1951569E-1,1.1039766E-1,0E0,0E0,1.0882974E-1,4.465103E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,21,21,22,22],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,28,-1,30,32,-1,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,6.0500806E3,5.6503E4,7.339209E6,1.308349E-2,9.9850745E2,2.35312E5,3.836E3,-1.5403449E-2,1E0,7.309061E1,1.99814E5,5.30113E5,5.5347366E0,1.2028214E5,2.2366652E0,-9.934456E-3,1.1868755E7,3.685938E6,8.737756E-3,2.0863017E-2,1.2384782E5,1.1367206E8,3.2710698E-3,-5.4369243E-3,-2.5109984E-3,1.1388485E-3,-5.346932E-3,3.9426712E-4,4.651987E-3,7.073631E-4,-5.6506577E-3,1.1310908E-3,4.027987E-3,8.8140005E-3,5.807066E-4,-6.628879E-3],"split_indices":[42,61,2,56,0,61,11,2,0,18,65,2,2,56,37,62,0,57,1,0,0,37,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.83E2,3E2,1.83E2,2.97E2,3E0,1.2E2,6.3E1,2.93E2,4E0,2.8E1,9.2E1,1.2E1,5.1E1,6.1E1,2.32E2,2.5E1,3E0,6.9E1,2.3E1,7E0,5E0,4.5E1,6E0,3E0,5.8E1,1.63E2,6.9E1,4E0,2.1E1,4.3E1,2.6E1,6E0,1.7E1,2.4E1,2.1E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.1679979E-3,-3.6933012E-2,1.0784E-1,-5.6848045E-2,1.987763E-2,1.4664302E-2,8.696205E-2,-8.3686166E-2,-2.4140561E-2,3.0672938E-2,-1.0803517E-2,1.6169554E-1,3.0112008E-2,-9.328701E-2,-8.6307125E-3,-7.7814594E-2,3.3106452E-3,1.8634116E-2,1.1872287E-1,1.3901064E-1,1.6709125E-2,8.561923E-2,-9.0734124E-2,1.2800489E-3,-4.4975053E-3,-2.603318E-3,2.1026656E-3,-1.8540547E-3,-7.6735737E-3,-9.648844E-4,5.27677E-3,-5.1087193E-4,2.4809584E-3,7.2739E-3,3.9280206E-4,-2.9829243E-4,7.2454643E-3,1.1176554E-2,2.887033E-3,4.3237917E-3,-5.9022293E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.847358E0,4.1634858E-1,4.8704123E-1,2.3674339E-1,2.7250576E-1,0E0,4.4973153E-1,1.0656357E-1,1.8296131E-1,9.636933E-2,0E0,1.6153193E-1,4.2365426E-1,8.2687974E-2,5.002366E-2,1.3399506E-1,2.2921233E-1,8.8604465E-2,4.737459E-2,1.2193239E-1,0E0,1.3826087E-1,1.5385598E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1722E4,7.164074E8,1E0,1.732734E6,1E0,1.4664302E-2,1.423795E7,8.067265E5,7.127066E2,1E0,-1.0803517E-2,1.4113879E4,6.27427E5,2.6E2,4.827E3,7.8E1,7.1999524E9,5.841568E1,4.194301E7,1.1E1,1.6709125E-2,2.0159504E1,5.3930223E4,1.2800489E-3,-4.4975053E-3,-2.603318E-3,2.1026656E-3,-1.8540547E-3,-7.6735737E-3,-9.648844E-4,5.27677E-3,-5.1087193E-4,2.4809584E-3,7.2739E-3,3.9280206E-4,-2.9829243E-4,7.2454643E-3,1.1176554E-2,2.887033E-3,4.3237917E-3,-5.9022293E-3],"split_indices":[2,7,25,1,18,0,54,56,61,108,0,4,9,12,2,8,5,65,41,8,0,65,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.81E2,3.66E2,1.15E2,2.71E2,9.5E1,9E0,1.06E2,1.48E2,1.23E2,9.2E1,3E0,4.5E1,6.1E1,1.31E2,1.7E1,4.1E1,8.2E1,8.2E1,1E1,4.2E1,3E0,4.2E1,1.9E1,5E0,1.26E2,9E0,8E0,3E1,1.1E1,6.8E1,1.4E1,4.5E1,3.7E1,7E0,3E0,5E0,3.7E1,4E0,3.8E1,3E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[7.200377E-4,-2.128416E-2,1.7365268E-1,-3.8172908E-2,6.744281E-2,1.2986934E-1,3.118525E-1,-3.4912206E-2,-1.5261721E-2,8.395114E-2,-5.744339E-2,1.5323174E-1,6.7193165E-2,4.0409244E-3,1.836851E-2,-4.958724E-2,1.7136237E-2,5.1555905E-2,1.7048171E-1,4.084912E-4,-5.1401514E-3,2.1318649E-1,1.16289295E-1,1.7820153E-2,4.6674707E-3,-1.8975019E-3,-6.340646E-3,3.4244594E-3,-1.0268914E-3,5.706089E-4,6.6624368E-3,1.4752975E-3,9.051751E-3,3.4513052E-3,1.16885025E-2,5.7480624E-3,1.9654012E-3,2.3019398E-3,-8.7485265E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,25,27,29,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9215391E0,6.738968E-1,3.043983E-1,3.5140365E-1,1.5326214E-1,5.5595815E-2,2.247802E-1,2.8651556E-1,0E0,1.706141E-1,3.378482E-2,4.6084702E-2,2.2238508E-2,0E0,0E0,2.0526785E-1,1.9120215E-1,1.7460828E-1,5.7830334E-2,0E0,0E0,4.4741273E-2,9.642392E-3,9.360554E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,26,28,30,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,1.5522031E9,2.0531621E4,3.764913E7,9.82576E5,4.307733E1,1.5485187E2,1.472533E3,-1.5261721E-2,3.28496E5,1.2083144E7,2.8E1,4.7757564E0,4.0409244E-3,1.836851E-2,1E0,1.36E2,1.1593098E2,2.6838362E-1,4.084912E-4,-5.1401514E-3,1.31E3,4.52E2,6.6E1,4.6674707E-3,-1.8975019E-3,-6.340646E-3,3.4244594E-3,-1.0268914E-3,5.706089E-4,6.6624368E-3,1.4752975E-3,9.051751E-3,3.4513052E-3,1.16885025E-2,5.7480624E-3,1.9654012E-3,2.3019398E-3,-8.7485265E-4],"split_indices":[61,7,4,57,11,66,67,61,0,11,54,8,44,0,0,18,3,63,50,0,0,0,3,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[5.03E2,4.47E2,5.6E1,3.76E2,7.1E1,4.4E1,1.2E1,3.73E2,3E0,6.3E1,8E0,3.1E1,1.3E1,4E0,8E0,2.91E2,8.2E1,4.7E1,1.6E1,4E0,4E0,1E1,2.1E1,6E0,7E0,2.68E2,2.3E1,3.3E1,4.9E1,3.4E1,1.3E1,3E0,1.3E1,3E0,7E0,1.8E1,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.5846592E-3,-2.4209784E-2,1.1210524E-1,-3.9146762E-2,6.194787E-2,2.8897566E-1,7.814157E-2,-3.5830066E-2,-1.473792E-2,2.3126078E-1,3.9677545E-2,4.1959183E-3,1.6770242E-2,9.8737024E-2,-7.3873843E-3,-4.638343E-2,4.1331008E-2,1.315711E-2,5.362892E-3,5.168795E-2,-5.920618E-3,6.843165E-2,1.7010128E-1,-4.708287E-3,-1.511198E-3,-6.2840334E-3,2.8910043E-3,-7.812968E-4,3.4075056E-3,-6.599071E-4,4.1466556E-3,9.611218E-3,4.0704184E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,21,-1,23,25,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.286266E0,5.39077E-1,4.7601402E-1,3.2250768E-1,2.2460464E-1,1.6514778E-1,3.6626238E-1,2.893681E-1,0E0,1.1223018E-2,1.2039232E-1,0E0,0E0,1.3109928E-1,0E0,2.3137623E-1,1.7648461E-1,0E0,0E0,8.426829E-2,0E0,8.952519E-2,4.2881727E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,15,15,16,16,19,19,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,22,-1,24,26,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.613448E9,1E0,3.372042E7,1.5E1,7.0331914E3,3.768034E4,2.1592189E5,-1.473792E-2,1.4E1,2.4070468E7,4.1959183E-3,1.6770242E-2,6.477944E0,-7.3873843E-3,1.2808544E3,5.8955963E1,1.315711E-2,5.362892E-3,5.2E1,-5.920618E-3,4.9762596E1,4.1923336E7,-4.708287E-3,-1.511198E-3,-6.2840334E-3,2.8910043E-3,-7.812968E-4,3.4075056E-3,-6.599071E-4,4.1466556E-3,9.611218E-3,4.0704184E-3],"split_indices":[61,7,25,57,3,4,4,37,0,3,57,0,0,44,0,4,59,0,0,3,0,65,52,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.98E2,4.16E2,8.2E1,3.55E2,6.1E1,1.2E1,7E1,3.52E2,3E0,6E0,5.5E1,4E0,8E0,6.5E1,5E0,3.1E2,4.2E1,3E0,3E0,5.2E1,3E0,4.7E1,1.8E1,5.8E1,2.52E2,4E0,3.8E1,1.3E1,3.9E1,1E1,3.7E1,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.7090638E-3,-2.6250998E-2,9.1787994E-2,-3.1226177E-2,1.3114593E-2,7.103143E-2,2.6748863E-1,-2.8108716E-2,-1.4232097E-2,8.1823595E-2,-1.0279185E-2,7.4058357E-3,1.46362E-2,-3.7464056E-2,5.6849025E-2,7.069509E-2,1.5539739E-2,-6.1024153E-3,-1.4294639E-3,-3.3162495E-3,4.3483377E-3,3.7521943E-3,-4.170631E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,-1,9,11,13,-1,15,-1,-1,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2440853E0,5.957179E-1,4.0769207E-1,3.1571668E-1,0E0,3.5660142E-1,1.460737E-2,2.954141E-1,0E0,2.7144378E-1,0E0,0E0,0E0,1.9466284E-1,1.8953137E-1,1.9033396E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,13,13,14,14,15,15],"right_children":[2,4,6,8,-1,10,12,14,-1,16,-1,-1,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5522031E9,8.1033794E3,1.6941456E1,3.764913E7,1.3114593E-2,3.782676E4,2.3031281E5,2.1603052E9,-1.4232097E-2,1.5325492E4,-1.0279185E-2,7.4058357E-3,1.46362E-2,1.44E2,2E1,9.064268E2,1.5539739E-2,-6.1024153E-3,-1.4294639E-3,-3.3162495E-3,4.3483377E-3,3.7521943E-3,-4.170631E-3],"split_indices":[7,61,44,57,0,61,42,12,0,61,0,0,0,0,8,67,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,3.76E2,1.16E2,3.71E2,5E0,1.05E2,1.1E1,3.68E2,3E0,1.02E2,3E0,5E0,6E0,3.32E2,3.6E1,9.9E1,3E0,1.9E1,3.13E2,8E0,2.8E1,9.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.254128E-3,-2.5025573E-2,9.147824E-2,-1.7533202E-2,-1.5226847E-1,2.3716763E-1,7.549653E-2,-3.4316894E-2,5.6169406E-2,-7.094721E-2,-1.3415389E-2,3.9537637E-3,1.5943704E-2,8.737082E-2,-1.0492134E-1,-7.004757E-2,-1.4994907E-2,1.4191963E-1,2.702302E-2,1.7139456E-3,-1.0391602E-1,5.6168307E-2,1.3680206E-1,-7.2011435E-3,-1.2012476E-3,-3.5579377E-3,5.9665795E-3,4.5329302E-6,-5.262423E-3,7.77612E-3,7.0547464E-4,-5.6482514E-4,3.6654342E-3,-6.116359E-3,-1.8988918E-3,7.706551E-4,4.941401E-3,1.332288E-2,3.932822E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1923811E0,3.424433E-1,2.5218523E-1,4.2621917E-1,2.08348E-1,1.5438485E-1,2.3625433E-1,1.9225821E-1,1.5612072E-1,5.3925984E-2,0E0,0E0,0E0,1.46554E-1,2.4748333E-2,1.6141999E-1,2.762354E-1,5.4068446E-2,1.0336724E-1,0E0,1.4874883E-2,1.2655589E-1,2.732618E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6053742E9,3.9608764E7,1.3812E4,1.7990698E3,5.974836E4,2.822E3,1.0434409E3,4.39E3,1.5022098E5,2.335226E6,-1.3415389E-2,3.9537637E-3,1.5943704E-2,1.556838E0,1.08935805E5,7.10711E8,1E0,9.1747506E5,2.7798245E6,1.7139456E-3,4.375082E4,1E0,3.0418E4,-7.2011435E-3,-1.2012476E-3,-3.5579377E-3,5.9665795E-3,4.5329302E-6,-5.262423E-3,7.77612E-3,7.0547464E-4,-5.6482514E-4,3.6654342E-3,-6.116359E-3,-1.8988918E-3,7.706551E-4,4.941401E-3,1.332288E-2,3.932822E-3],"split_indices":[7,54,9,61,37,0,67,2,42,1,0,0,0,50,37,7,18,56,56,0,37,22,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,3.61E2,1.15E2,3.42E2,1.9E1,1E1,1.05E2,2.79E2,6.3E1,1.3E1,6E0,5E0,5E0,9.9E1,6E0,9.7E1,1.82E2,1.5E1,4.8E1,3E0,1E1,6.2E1,3.7E1,3E0,3E0,9.4E1,3E0,1.59E2,2.3E1,1.2E1,3E0,2.8E1,2E1,6E0,4E0,3.6E1,2.6E1,8E0,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[9.904063E-4,-2.6949393E-2,8.152787E-2,-3.047139E-2,1.0778971E-2,6.3470215E-2,2.3638636E-1,-4.35233E-2,2.71748E-2,7.299321E-2,-1.0132619E-2,1.2354479E-2,4.2703934E-3,-4.003005E-2,-1.3283202E-2,-3.9250724E-2,4.5123097E-2,5.972318E-2,1.591044E-2,-2.0954632E-3,3.2288558E-3,1.4426643E-3,-4.1156975E-3,3.3667807E-3,1.638191E-4,3.30566E-3,-2.6691258E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1182966E0,3.483213E-1,3.4293342E-1,2.7580616E-1,0E0,3.3087346E-1,3.727728E-2,2.4524081E-1,8.284241E-2,3.9113355E-1,0E0,0E0,0E0,1.9116974E-1,0E0,5.6647122E-2,6.355755E-2,1.6522887E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,13,13,15,15,16,16,17,17],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5522031E9,8.1033794E3,1.6941456E1,1.926086E8,1.0778971E-2,3.782676E4,9.2E1,9.163994E6,8.6E2,3.305542E4,-1.0132619E-2,1.2354479E-2,4.2703934E-3,2.889654E5,-1.3283202E-2,1.99742E5,1.7239808E7,1.5953E4,1.591044E-2,-2.0954632E-3,3.2288558E-3,1.4426643E-3,-4.1156975E-3,3.3667807E-3,1.638191E-4,3.30566E-3,-2.6691258E-3],"split_indices":[7,61,44,41,0,61,8,56,0,4,0,0,0,37,0,38,54,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.95E2,3.68E2,1.27E2,3.64E2,4E0,1.15E2,1.2E1,2.97E2,6.7E1,1.12E2,3E0,9E0,3E0,2.94E2,3E0,1.4E1,5.3E1,1.08E2,4E0,2.8E2,1.4E1,6E0,8E0,3.1E1,2.2E1,9.8E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.1828738E-3,-3.144145E-2,5.5347275E-2,-5.1350333E-2,2.4020815E-2,1.6473418E-2,4.4962216E-2,-6.3473776E-2,2.0779166E-2,5.136031E-2,-5.7846818E-2,3.0380495E-2,2.125506E-1,2.8133495E-2,-7.195216E-2,-6.4758467E-3,5.254491E-2,1.0960384E-1,1.9012373E-2,2.5943786E-2,-1.1785138E-1,-1.2808529E-1,3.9614417E-2,1.1336402E-2,3.6730482E-3,2.6340173E-3,-1.2049162E-3,-6.209928E-3,-2.6956475E-3,1.0718814E-3,7.556518E-3,6.592896E-3,4.686559E-4,-1.2529937E-3,4.1907285E-3,-3.6989094E-4,3.6153807E-3,-1.6651964E-3,-7.514833E-3,-8.628665E-3,-9.3693915E-4,6.3602645E-3,1.3133952E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.9234207E-1,3.578874E-1,4.773463E-1,2.0943987E-1,1.9545794E-1,0E0,3.649892E-1,1.604672E-1,1.878721E-1,1.20379955E-1,1.1366951E-1,2.1139382E-1,3.595698E-2,3.0363377E-2,1.475277E-1,0E0,9.449065E-2,7.729268E-2,1.4726837E-1,1.8693527E-2,4.3126464E-2,4.819873E-2,1.413355E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5140823E3,7.164074E8,8.534E3,2.0964778E5,1.3185959E7,1.6473418E-2,1.5820756E1,5.8981678E1,2.4143283E2,1.674785E1,3.4129692E-3,3.5226266E0,9.2E1,4.276419E-6,1E0,-6.4758467E-3,4.131E3,1.24E2,2.1603485E6,2.3065826E7,1.1459E4,2.632124E9,2.1207E4,1.1336402E-2,3.6730482E-3,2.6340173E-3,-1.2049162E-3,-6.209928E-3,-2.6956475E-3,1.0718814E-3,7.556518E-3,6.592896E-3,4.686559E-4,-1.2529937E-3,4.1907285E-3,-3.6989094E-4,3.6153807E-3,-1.6651964E-3,-7.514833E-3,-8.628665E-3,-9.3693915E-4,6.3602645E-3,1.3133952E-3],"split_indices":[61,7,9,37,54,0,44,59,61,67,66,63,8,50,13,0,0,3,59,54,2,7,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.77E2,3.22E2,1.55E2,2.37E2,8.5E1,4E0,1.51E2,2.03E2,3.4E1,6.4E1,2.1E1,1.4E2,1.1E1,1.7E1,1.86E2,5E0,2.9E1,2.2E1,4.2E1,9E0,1.2E1,7E0,1.33E2,8E0,3E0,1.1E1,6E0,3E1,1.56E2,2.4E1,5E0,1.6E1,6E0,2.6E1,1.6E1,6E0,3E0,5E0,7E0,4E0,3E0,1.2E1,1.21E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.8081583E-3,-3.0949097E-2,5.1300097E-2,-3.47638E-2,1.172481E-2,3.7131656E-2,1.7434916E-1,-3.0487895E-2,-1.25825275E-2,-6.763592E-2,4.6052482E-2,9.51757E-3,3.5406828E-3,-1.4950483E-1,-2.3885198E-2,1.2642352E-2,-8.70204E-3,8.374814E-2,2.1201462E-2,-2.7417305E-3,-7.6380605E-3,4.6229505E-4,-1.9567737E-3,3.6837338E-3,-1.2785135E-3,-1.9785787E-4,5.1797614E-3,1.3842356E-3,-9.829934E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,-1,-1,19,21,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.2170206E-1,3.331591E-1,3.526693E-1,2.9254246E-1,0E0,1.7695284E-1,5.2401006E-2,2.235347E-1,0E0,1.4994487E-1,1.5934986E-1,0E0,0E0,1.3390064E-2,1.7684796E-1,2.9785626E-2,0E0,1.7681435E-1,2.3070079E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,17,17,18,18],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,-1,-1,20,22,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.571489E4,5.932891E3,7.765071E0,7.339209E6,1.172481E-2,3.6340196E-2,1.1548442E10,1.44E2,-1.25825275E-2,4.565957E6,7.238065E6,9.51757E-3,3.5406828E-3,1.0366701E-4,1.2755817E4,2.1E1,-8.70204E-3,8.80484E2,3.782676E4,-2.7417305E-3,-7.6380605E-3,4.6229505E-4,-1.9567737E-3,3.6837338E-3,-1.2785135E-3,-1.9785787E-4,5.1797614E-3,1.3842356E-3,-9.829934E-3],"split_indices":[42,61,44,56,0,47,12,0,0,59,59,0,0,47,59,3,0,61,61,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[5E2,2.95E2,2.05E2,2.92E2,3E0,1.85E2,2E1,2.88E2,4E0,1.4E1,1.71E2,1.4E1,6E0,1.4E1,2.74E2,9E0,5E0,6.7E1,1.04E2,3E0,1.1E1,9.8E1,1.76E2,3E0,6E0,1.7E1,5E1,1.01E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.4004848E-3,-1.5702723E-2,1.1634486E-1,-3.0620119E-2,5.1626883E-2,2.9859874E-1,7.1254544E-2,-2.7845018E-2,-1.2099314E-2,6.353362E-2,-7.821768E-2,7.815738E-3,1.62608E-2,1.3796725E-2,3.990995E-2,-7.380644E-2,-1.4293916E-2,3.510772E-2,1.6083002E-1,-3.3686575E-4,-5.9270216E-3,7.665633E-2,-1.120643E-1,-5.0892327E-3,-1.6668867E-3,-1.2104749E-3,2.8821854E-3,2.6572167E-3,-3.5971077E-3,9.221325E-3,-5.299609E-4,4.6143556E-4,4.1932007E-3,-1.0063397E-2,2.322767E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,21,23,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.0847347E-1,4.314565E-1,4.0605342E-1,2.2035259E-1,1.255594E-1,1.1826456E-2,2.9828492E-1,2.1568915E-1,0E0,1.9348672E-1,2.4492044E-2,0E0,0E0,0E0,2.2638607E-1,1.0641104E-1,2.5533575E-1,1.5337539E-1,1.14546865E-1,0E0,0E0,3.0978054E-2,1.5163875E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,14,14,15,15,16,16,17,17,18,18,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,22,24,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,1.6053742E9,1.37893E6,3.764913E7,2.3817926E7,1E0,1.3812E4,1.192104E3,-1.2099314E-2,9.9179E4,1.0736829E5,7.815738E-3,1.62608E-2,1.3796725E-2,2.0531621E4,1.65602E5,1.6130411E5,1.662639E7,5.9108553E10,-3.3686575E-4,-5.9270216E-3,3.638653E6,1.2408761E0,-5.0892327E-3,-1.6668867E-3,-1.2104749E-3,2.8821854E-3,2.6572167E-3,-3.5971077E-3,9.221325E-3,-5.299609E-4,4.6143556E-4,4.1932007E-3,-1.0063397E-2,2.322767E-3],"split_indices":[61,7,38,57,52,22,9,64,0,2,37,0,0,0,4,9,37,54,5,0,0,56,43,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,4.27E2,5.1E1,3.5E2,7.7E1,9E0,4.2E1,3.47E2,3E0,7.1E1,6E0,4E0,5E0,4E0,3.8E1,7.8E1,2.69E2,5.6E1,1.5E1,3E0,3E0,3.1E1,7E0,3.8E1,4E1,2.33E2,3.6E1,4.7E1,9E0,1.2E1,3E0,6E0,2.5E1,4E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.6320916E-3,-2.8931037E-2,5.0288092E-2,-2.5168238E-2,-1.2874166E-2,3.3878364E-2,1.4870997E-1,-2.8579729E-2,9.791057E-3,-5.0121997E-2,4.0842548E-2,1.7192873E-1,1.3152718E-4,-3.414734E-2,6.94597E-2,-5.8934174E-4,-4.4678645E-3,6.0957503E-2,9.608939E-4,1.3534904E-1,1.1287333E-2,-2.763675E-3,-1.0407546E-4,-2.1593482E-3,4.6930145E-3,2.2568386E-3,1.0228914E-2,-2.3469965E-3,1.910542E-3,7.0560984E-3,3.1494226E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,-1,21,23,-1,-1,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.415839E-1,2.5789446E-1,3.260526E-1,2.3126929E-1,0E0,1.0559107E-1,9.770352E-2,1.5138769E-1,0E0,2.3515541E-2,1.3150513E-1,3.4952283E-2,0E0,2.1652299E-1,6.2743574E-2,0E0,0E0,1.9524023E-1,1.2148623E-1,1.3206512E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,-1,22,24,-1,-1,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.4023336E4,7.339209E6,4.026531E9,6.0500806E3,-1.2874166E-2,1.5802073E3,4.547513E7,2.889654E5,9.791057E-3,1.384081E3,2.0946211E5,1.0871896E10,1.3152718E-4,5.495E3,3E1,-5.8934174E-4,-4.4678645E-3,9.747583E9,3.2357117E3,5.145648E7,1.1287333E-2,-2.763675E-3,-1.0407546E-4,-2.1593482E-3,4.6930145E-3,2.2568386E-3,1.0228914E-2,-2.3469965E-3,1.910542E-3,7.0560984E-3,3.1494226E-3],"split_indices":[42,56,7,61,0,4,38,37,0,4,42,7,0,2,8,0,0,12,61,1,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.82E2,2.78E2,2.04E2,2.75E2,3E0,1.76E2,2.8E1,2.72E2,3E0,1.3E1,1.63E2,2.4E1,4E0,2.58E2,1.4E1,8E0,5E0,1.08E2,5.5E1,1.8E1,6E0,1.41E2,1.17E2,3E0,1.1E1,1.02E2,6E0,2.4E1,3.1E1,1.3E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-1.959382E-3,-4.342658E-2,3.0083722E-2,-5.910345E-2,2.2181164E-2,2.115884E-2,2.0969985E-1,-6.402033E-2,7.5341742E-3,3.7290618E-2,-5.980182E-3,3.2094143E-2,-1.2211372E-1,1.2513255E-2,4.1074194E-3,-1.2910521E-1,-5.641992E-2,-4.1169174E-2,6.2200863E-2,7.2337106E-2,1.0532483E-2,-4.2073388E-2,-1.1592105E-2,-2.937111E-3,-6.9011934E-3,-6.901938E-4,-2.925787E-3,-4.429465E-3,1.5474252E-3,3.7946268E-3,-8.11251E-4,-1.6623416E-3,3.8415825E-3,1.2047223E-3,-3.526817E-3,1.7235348E-3,-4.421336E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,23,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.523879E-1,2.2141516E-1,4.3960857E-1,2.0086068E-1,1.01879254E-1,4.1910225E-1,7.485092E-2,7.681012E-2,0E0,7.8784406E-2,0E0,2.1356127E-1,1.8886906E-1,0E0,0E0,1.4273196E-2,4.687938E-2,4.498319E-2,5.115562E-2,1.1155167E-1,2.2616363E-1,6.0177367E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,24,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.025313E6,7.164074E8,1.3554651E2,1.5704E4,6.915E3,4.2663252E7,3.3616E4,1.2304464E1,7.5341742E-3,8.8E1,-5.980182E-3,1.0595947E7,5.9438918E4,1.2513255E-2,4.1074194E-3,7.694E3,1.6225006E4,9.0510735E1,2.1058936E2,5.5343207E4,1.93E2,3.58777E3,-1.1592105E-2,-2.937111E-3,-6.9011934E-3,-6.901938E-4,-2.925787E-3,-4.429465E-3,1.5474252E-3,3.7946268E-3,-8.11251E-4,-1.6623416E-3,3.8415825E-3,1.2047223E-3,-3.526817E-3,1.7235348E-3,-4.421336E-3],"split_indices":[1,7,65,2,10,54,10,65,0,3,0,54,37,0,0,38,37,63,63,37,8,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.89E2,2.13E2,2.76E2,1.72E2,4.1E1,2.64E2,1.2E1,1.69E2,3E0,3.8E1,3E0,2.46E2,1.8E1,7E0,5E0,1.6E1,1.53E2,9E0,2.9E1,8.5E1,1.61E2,1.2E1,6E0,5E0,1.1E1,2.4E1,1.29E2,5E0,4E0,2.3E1,6E0,8E0,7.7E1,1.37E2,2.4E1,5E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.686365E-3,-1.9415075E-2,8.473448E-2,-3.1493172E-2,3.8456064E-2,2.297379E-1,5.6661014E-2,-4.3926932E-2,4.221128E-2,-1.2462291E-1,7.15335E-2,3.6520008E-3,3.2543212E-1,1.6024025E-1,3.2594126E-2,-4.954852E-3,-5.8280926E-2,1.6531473E-2,1.2708595E-1,-5.543612E-2,-9.628107E-3,1.0589091E-1,2.1946091E-2,7.3950132E-3,1.866649E-2,1.0769053E-3,2.0319457E-1,-4.887938E-2,5.7036065E-2,-2.2300866E-3,2.1167924E-3,-5.3149727E-3,-1.8067451E-3,5.779794E-3,-5.6422944E-4,4.018945E-4,7.795577E-3,-4.525287E-3,-4.3872124E-4,6.885658E-3,2.8728668E-3,1.8479303E-3,-7.0084655E-4,2.3986045E-3,1.2348687E-2,-4.782194E-3,2.792275E-3,3.4786994E-3,-1.5164935E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,-1,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.919158E-1,2.7870345E-1,2.9635972E-1,3.0335754E-1,3.8111693E-1,1.4620155E-1,1.5681104E-1,1.5746355E-1,1.0251822E-1,5.9549764E-2,9.649041E-2,0E0,2.8668165E-2,6.568429E-2,1.10321894E-1,1.7488402E-1,2.1950054E-1,1.2194595E-1,5.4807737E-2,1.4989881E-2,0E0,5.589521E-2,1.7791804E-2,0E0,0E0,0E0,7.6305926E-2,8.584312E-2,7.4410126E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,26,26,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,-1,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2789043E3,1.646359E5,1.064816E6,3.53E2,1.06790736E-4,9.256843E3,2.1207E4,1.2755817E4,1.1191666E0,2.124059E-6,1.2035191E1,3.6520008E-3,1E0,2.397E3,5.0314346E1,2.616931E6,1.3177096E4,7.202981E-4,1.9745543E3,2.09E2,-9.628107E-3,7.221312E-2,1.5963512E-2,7.3950132E-3,1.866649E-2,1.0769053E-3,4.861786E1,1.6130411E5,2.1897722E10,-2.2300866E-3,2.1167924E-3,-5.3149727E-3,-1.8067451E-3,5.779794E-3,-5.6422944E-4,4.018945E-4,7.795577E-3,-4.525287E-3,-4.3872124E-4,6.885658E-3,2.8728668E-3,1.8479303E-3,-7.0084655E-4,2.3986045E-3,1.2348687E-2,-4.782194E-3,2.792275E-3,3.4786994E-3,-1.5164935E-3],"split_indices":[61,37,38,8,50,4,9,59,50,47,62,0,22,0,65,9,42,48,4,3,0,47,66,0,0,0,65,37,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.71E2,3.96E2,7.5E1,3.28E2,6.8E1,1.1E1,6.4E1,2.81E2,4.7E1,1.1E1,5.7E1,5E0,6E0,1.1E1,5.3E1,7.6E1,2.05E2,3.7E1,1E1,7E0,4E0,3.3E1,2.4E1,3E0,3E0,3E0,8E0,1.2E1,4.1E1,4.1E1,3.5E1,4.9E1,1.56E2,7E0,3E1,3E0,7E0,3E0,4E0,1.5E1,1.8E1,1.6E1,8E0,3E0,5E0,8E0,4E0,3.4E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[2.5071178E-3,-1.1044495E-2,9.732662E-2,-2.3706377E-2,4.847872E-2,1.5386236E-2,7.885426E-2,-3.5774987E-2,5.2216314E-2,-3.96048E-2,7.749851E-2,-3.0824438E-2,1.1528045E-1,-3.1937152E-2,-9.9991765E-3,-1.4704829E-3,9.7700156E-2,-1.2026724E-1,1.7470555E-2,1.1350179E-1,1.2088195E-2,2.9310226E-2,-5.761908E-3,1.3267635E-1,-6.455567E-4,4.0099183E-3,-1.6875349E-3,4.0094797E-3,-1.7420038E-3,8.803E-4,5.7474393E-3,-2.5678028E-3,-7.7974843E-3,-2.7908178E-3,2.3063372E-3,7.395756E-3,2.8347971E-3,-2.3273346E-3,1.5987924E-3,-1.7673956E-5,2.7037733E-3,4.290052E-3,1.4857413E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.165694E-1,3.1769314E-1,2.3826939E-1,3.1981766E-1,1.9222555E-1,0E0,2.3127857E-1,2.0340875E-1,1.1735399E-1,8.9574054E-2,1.3050714E-1,8.9460894E-2,9.8656476E-2,1.7745438E-1,0E0,7.832167E-2,5.449149E-2,1.616551E-2,3.4075066E-2,7.9285026E-2,3.19805E-2,8.864546E-3,0E0,2.5357544E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0297854E3,1.646359E5,8.381319E3,3.53E2,1.0291666E3,1.5386236E-2,6.219584E5,7.217412E2,6.1707415E-2,2.72E2,1.3524064E1,1.4609149E4,1.93E2,1E0,-9.9991765E-3,1.0557411E1,2.0532622E3,5.791844E6,1.964127E6,7.221312E-2,1.0931615E9,1.0078588E5,-5.761908E-3,4.42384E5,-6.455567E-4,4.0099183E-3,-1.6875349E-3,4.0094797E-3,-1.7420038E-3,8.803E-4,5.7474393E-3,-2.5678028E-3,-7.7974843E-3,-2.7908178E-3,2.3063372E-3,7.395756E-3,2.8347971E-3,-2.3273346E-3,1.5987924E-3,-1.7673956E-5,2.7037733E-3,4.290052E-3,1.4857413E-2],"split_indices":[61,37,42,8,64,0,41,67,47,3,62,4,8,20,0,67,4,54,1,47,5,37,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,4.19E2,5.9E1,3.46E2,7.3E1,3E0,5.6E1,2.99E2,4.7E1,1.8E1,5.5E1,1.4E1,4.2E1,2.94E2,5E0,2.2E1,2.5E1,7E0,1.1E1,3.5E1,2E1,9E0,5E0,3.7E1,5E0,1.1E1,2.83E2,6E0,1.6E1,7E0,1.8E1,4E0,3E0,3E0,8E0,1.7E1,1.8E1,5E0,1.5E1,5E0,4E0,3.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[3.2862525E-3,-1.0231335E-2,9.614068E-2,-3.4697965E-2,1.2964586E-2,1.9056636E-1,5.994117E-2,-5.908261E-2,-3.1230145E-3,3.4271576E-2,-6.339638E-2,5.968319E-2,2.7621627E-1,8.2063615E-2,-9.078687E-3,7.098369E-3,-7.3336676E-2,-2.85927E-2,8.037428E-2,-1.0159397E-2,5.7541292E-2,-1.3762535E-1,-2.3236113E-2,3.2515614E-4,4.10979E-3,1.8119028E-2,7.703545E-3,7.4643E-3,6.749031E-2,2.170251E-3,-1.1197072E-3,-8.139305E-3,-2.719953E-3,2.3319224E-4,-4.044932E-3,2.5554579E-3,6.9837878E-3,-4.403056E-3,2.1260364E-4,4.5234296E-3,1.1095514E-3,-8.426163E-3,-9.918736E-4,-2.007385E-3,4.624913E-3,3.461087E-3,-1.0372663E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,-1,-1,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.2229174E-1,2.4618864E-1,2.0392162E-1,1.6227365E-1,3.6499864E-1,1.7408681E-1,2.848829E-1,1.1277902E-1,2.0007655E-1,1.8151642E-1,1.4287387E-1,1.2033343E-2,6.250328E-2,4.096228E-2,0E0,2.9166028E-2,1.3180625E-1,1.4549662E-1,3.0463994E-2,7.816789E-2,1.5576234E-1,8.768064E-2,8.905681E-2,0E0,0E0,0E0,0E0,0E0,3.0345172E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,-1,-1,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0297854E3,6.24E3,1.417988E6,6.510362E2,2.4722598E7,9.256843E3,2.3767982E4,7.117012E1,1.4555195E5,7.0532586E4,1.13E2,7.0331914E3,4.542969E1,1.3812E4,-9.078687E-3,3.329E3,1E0,9.9105975E2,3.677E3,2.0532622E3,6.8E1,1.2028214E5,1.613448E9,3.2515614E-4,4.10979E-3,1.8119028E-2,7.703545E-3,7.4643E-3,2.9928794E5,2.170251E-3,-1.1197072E-3,-8.139305E-3,-2.719953E-3,2.3319224E-4,-4.044932E-3,2.5554579E-3,6.9837878E-3,-4.403056E-3,2.1260364E-4,4.5234296E-3,1.1095514E-3,-8.426163E-3,-9.918736E-4,-2.007385E-3,4.624913E-3,3.461087E-3,-1.0372663E-3],"split_indices":[61,2,38,61,54,4,61,59,37,37,3,4,63,9,0,2,13,61,0,4,8,37,7,0,0,0,0,0,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.94E2,4.32E2,6.2E1,2.1E2,2.22E2,1.6E1,4.6E1,1.18E2,9.2E1,1.74E2,4.8E1,7E0,9E0,4.3E1,3E0,2.1E1,9.7E1,7.1E1,2.1E1,6E1,1.14E2,1.6E1,3.2E1,3E0,4E0,3E0,6E0,5E0,3.8E1,9E0,1.2E1,1E1,8.7E1,4.6E1,2.5E1,1.7E1,4E0,8E0,5.2E1,5E1,6.4E1,1.1E1,5E0,2.8E1,4E0,3.5E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-7.199284E-5,-2.826937E-2,4.265603E-2,7.4243965E-3,-4.7828067E-2,3.3135545E-2,1.8587556E-1,1.1203471E-1,-1.8206311E-2,-6.779071E-2,7.22117E-3,1.7217385E-2,7.820992E-2,5.0017643E-3,1.0294418E-2,7.873758E-2,1.0701944E-2,-4.171485E-2,3.4709197E-2,-1.4286175E-1,-5.564322E-2,-3.6334656E-2,5.9550244E-2,-5.9556134E-2,2.8495945E-2,1.2830971E-1,-1.2418028E-2,-8.3454495E-4,4.5463326E-3,-1.3668999E-3,-7.5979703E-3,5.0044227E-3,-7.154863E-4,-4.229532E-3,-9.890012E-3,-6.443719E-3,-2.219166E-3,-3.6536043E-3,4.3899054E-4,6.466438E-3,1.1187581E-3,2.2116909E-3,-3.8602434E-3,3.684979E-4,2.5808432E-3,3.9294455E-3,1.7999208E-2,-3.327531E-3,3.296751E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,35,37,39,41,43,45,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.9397644E-1,2.0769778E-1,2.610908E-1,2.862508E-1,2.1148372E-1,1.3139115E-1,1.13245845E-2,6.6773415E-2,1.08429E-1,1.2063205E-1,1.20684005E-1,1.2119061E-1,2.1953452E-1,0E0,0E0,3.7788518E-2,0E0,8.339269E-2,1.0318047E-1,5.022809E-2,6.806296E-2,5.8710746E-2,6.542271E-2,5.223655E-2,6.856477E-2,3.149075E-1,9.6905954E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,36,38,40,42,44,46,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.921638E4,1.2755817E4,1.6941456E1,1.24E2,1.2128688E5,9.3058E4,2.3031281E5,1.4690361E5,3.57E2,2.9648642E1,2.662171E6,4.3871765E0,5.358657E7,5.0017643E-3,1.0294418E-2,6.9E1,1.0701944E-2,2.7174674E11,1.743E3,1.0736829E5,4.7880264E2,7.6E1,4.1752555E-3,1.133E3,8.766172E4,3.63796E5,9.146786E1,-8.3454495E-4,4.5463326E-3,-1.3668999E-3,-7.5979703E-3,5.0044227E-3,-7.154863E-4,-4.229532E-3,-9.890012E-3,-6.443719E-3,-2.219166E-3,-3.6536043E-3,4.3899054E-4,6.466438E-3,1.1187581E-3,2.2116909E-3,-3.8602434E-3,3.684979E-4,2.5808432E-3,3.9294455E-3,1.7999208E-2,-3.327531E-3,3.296751E-3],"split_indices":[42,59,44,3,37,2,42,37,10,65,1,62,1,0,0,8,0,40,10,37,42,8,51,0,37,2,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.91E2,2.96E2,1.95E2,1.05E2,1.91E2,1.84E2,1.1E1,2E1,8.5E1,1.4E2,5.1E1,1.37E2,4.7E1,5E0,6E0,1.7E1,3E0,5.9E1,2.6E1,1.8E1,1.22E2,2.8E1,2.3E1,1.7E1,1.2E2,3E1,1.7E1,3E0,1.4E1,5.5E1,4E0,1E1,1.6E1,1.2E1,6E0,8E0,1.14E2,1.4E1,1.4E1,6E0,1.7E1,3E0,1.4E1,7E1,5E1,2.7E1,3E0,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.1615766E-3,-2.6238125E-2,4.1243102E-2,-2.3215396E-2,-9.066393E-3,1.1852545E-2,7.951653E-2,-3.3022344E-2,2.7528862E-2,5.588842E-2,-1.11771645E-2,6.363804E-2,2.3034668E-1,-4.2032428E-2,3.8628742E-2,-2.343604E-2,9.417117E-2,-1.8123867E-3,1.0318056E-1,-1.1148011E-1,1.0938749E-2,8.19005E-2,-2.2176826E-2,5.2643865E-3,1.4501572E-2,2.8174154E-3,-2.1400673E-3,3.552583E-3,-1.3136868E-3,-2.4899843E-3,1.7280536E-3,6.9399347E-4,7.533083E-3,1.5624858E-3,-1.7200675E-3,5.938455E-3,1.2660319E-3,-1.7348889E-3,-8.220281E-3,2.3773976E-3,-1.2253694E-3,2.5164303E-3,5.9943344E-3,-7.0319185E-3,1.0754146E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4909486E-1,1.4970966E-1,2.274077E-1,1.4671943E-1,0E0,1.18264146E-1,1.960972E-1,1.604369E-1,1.6523507E-1,1.09338775E-1,1.7202789E-1,1.2875876E-1,4.3380976E-2,1.10266805E-1,7.576643E-2,5.5133708E-2,1.1437984E-1,2.5724022E-2,4.0582597E-2,6.338449E-2,1.0038949E-1,8.130625E-2,9.370084E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,6.3903E6,1.3590209E7,2.916E3,-9.066393E-3,1.4040858E-2,4.2875492E1,4.13E2,3.295663E0,8.951946E2,1.2193457E3,1.73E2,8.926081E-2,5.8601086E1,1.5155188E7,2.72E2,5.0764175E2,3.6556778E1,1.5631589E9,8.459549E4,1.1106066E7,1.3047821E5,5.743529E6,5.2643865E-3,1.4501572E-2,2.8174154E-3,-2.1400673E-3,3.552583E-3,-1.3136868E-3,-2.4899843E-3,1.7280536E-3,6.9399347E-4,7.533083E-3,1.5624858E-3,-1.7200675E-3,5.938455E-3,1.2660319E-3,-1.7348889E-3,-8.220281E-3,2.3773976E-3,-1.2253694E-3,2.5164303E-3,5.9943344E-3,-7.0319185E-3,1.0754146E-3],"split_indices":[42,56,1,0,0,66,63,8,62,61,61,8,66,59,54,3,61,67,41,37,57,37,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.98E2,2.96E2,2.02E2,2.92E2,4E0,1.15E2,8.7E1,2.45E2,4.7E1,3.9E1,7.6E1,8E1,7E0,2.18E2,2.7E1,2.7E1,2E1,1.8E1,2.1E1,1.3E1,6.3E1,6.6E1,1.4E1,4E0,3E0,9E0,2.09E2,1.7E1,1E1,1.8E1,9E0,1E1,1E1,9E0,9E0,1.5E1,6E0,7E0,6E0,3E1,3.3E1,4.4E1,2.2E1,3E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.4250445E-3,-6.68722E-3,1.2831737E-1,-5.241307E-2,7.099534E-3,2.3468377E-1,5.96319E-2,-6.276074E-2,6.953448E-2,2.2843724E-2,-4.145111E-2,1.5996203E-2,6.6258316E-3,-2.276529E-3,8.670221E-2,-5.0669312E-2,-1.8073213E-1,-1.7328085E-4,4.8885318E-3,-1.1368905E-3,4.9029715E-2,-6.738764E-3,-1.030719E-1,7.1860556E-3,5.5391233E-2,-3.6534392E-3,-7.4926944E-4,-9.849118E-3,-3.835144E-3,9.4072457E-4,-2.2569818E-3,-9.090743E-4,2.8538303E-3,-1.7485777E-3,2.9892141E-3,-2.7357154E-3,-1.1400978E-2,3.6754664E-3,8.690477E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,-1,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.778817E-1,2.9666534E-1,2.3440349E-1,1.419082E-1,2.776563E-1,8.3628476E-2,6.945385E-2,1.3491318E-1,2.5012873E-2,1.7211571E-1,1.8967663E-1,0E0,0E0,0E0,3.0939758E-2,9.172198E-2,1.2053072E-2,0E0,0E0,1.5153608E-1,1.21575534E-1,1.3403237E-1,1.8744928E-1,0E0,1.1177409E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,16,16,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,-1,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.651788E3,4.407E3,1.681178E6,9.28E3,2.3790412E7,4.542969E1,6.7211235E1,4.752E3,1.7121376E0,1.138E3,1E0,1.5996203E-2,6.6258316E-3,-2.276529E-3,4.198476E6,2.0011E5,4.069E3,-1.7328085E-4,4.8885318E-3,3.902439E-2,6.510362E2,2.4480755E11,4.352718E7,7.1860556E-3,1.08935805E5,-3.6534392E-3,-7.4926944E-4,-9.849118E-3,-3.835144E-3,9.4072457E-4,-2.2569818E-3,-9.090743E-4,2.8538303E-3,-1.7485777E-3,2.9892141E-3,-2.7357154E-3,-1.1400978E-2,3.6754664E-3,8.690477E-4],"split_indices":[61,2,38,10,54,63,65,10,62,0,22,0,0,0,56,9,2,0,0,66,61,40,54,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[5.02E2,4.69E2,3.3E1,1.08E2,3.61E2,1.2E1,2.1E1,1E2,8E0,2.73E2,8.8E1,4E0,8E0,4E0,1.7E1,9.2E1,8E0,3E0,5E0,1.43E2,1.3E2,5.7E1,3.1E1,4E0,1.3E1,4.9E1,4.3E1,5E0,3E0,9.9E1,4.4E1,2.1E1,1.09E2,4E1,1.7E1,2.5E1,6E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[8.6027186E-4,-1.1624913E-2,5.542068E-2,-5.57263E-2,4.0346924E-3,1.6229019E-1,3.8268697E-2,-9.451318E-2,-1.5086219E-2,-8.351912E-3,7.975431E-3,1.17098E-2,3.8213579E-3,-4.6591192E-2,5.289631E-2,-1.3544057E-1,-5.45954E-2,1.4822948E-1,-4.0155634E-2,-7.2031077E-3,6.828338E-2,-6.336768E-3,-5.777909E-4,7.067559E-2,5.7074096E-4,-1.5321735E-3,-8.3035175E-3,5.206325E-4,-4.2045205E-3,2.9789654E-3,8.89166E-3,-7.687512E-3,-1.2366972E-3,-8.0281764E-4,3.205235E-3,7.532739E-3,1.5144175E-3,1.8960132E-3,-1.3036696E-3,1.0585847E-3,4.136399E-3,-3.334983E-3,1.2246456E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,27,29,31,33,35,-1,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2557514E-1,2.688723E-1,1.5692851E-1,1.5935692E-1,2.1317042E-1,6.711969E-2,9.9285714E-2,7.755956E-2,2.1475792E-1,0E0,2.5954622E-1,0E0,0E0,5.065383E-2,6.229049E-2,1.11205935E-1,6.9916345E-2,1.4163181E-2,7.099618E-2,1.8254375E-1,1.8776262E-1,0E0,1.1733079E-2,4.477364E-2,3.657613E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,28,30,32,34,36,-1,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.646359E5,1.7066003E3,2.8E1,1.477382E6,1.5153618E0,4.7431055E-2,4.4643167E2,2.7276272E-2,8.547769E7,-8.351912E-3,3.4565508E11,1.17098E-2,3.8213579E-3,2.040228E5,2.5519034E7,1.8484789E2,2.961349E5,2.8156E5,2.9E1,4.2524424E7,3.3219E4,-6.336768E-3,3.251E3,2.455358E6,7.8367496E-1,-1.5321735E-3,-8.3035175E-3,5.206325E-4,-4.2045205E-3,2.9789654E-3,8.89166E-3,-7.687512E-3,-1.2366972E-3,-8.0281764E-4,3.205235E-3,7.532739E-3,1.5144175E-3,1.8960132E-3,-1.3036696E-3,1.0585847E-3,4.136399E-3,-3.334983E-3,1.2246456E-3],"split_indices":[37,4,3,9,62,47,61,51,7,0,40,0,0,37,9,42,57,7,3,9,9,0,2,1,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.76E2,3.88E2,8.8E1,1.01E2,2.87E2,1.1E1,7.7E1,5.1E1,5E1,5E0,2.82E2,4E0,7E0,1.1E1,6.6E1,2.4E1,2.7E1,6E0,4.4E1,2.26E2,5.6E1,3E0,8E0,4.9E1,1.7E1,8E0,1.6E1,1E1,1.7E1,3E0,3E0,3E0,4.1E1,2E2,2.6E1,1.4E1,4.2E1,3E0,5E0,1.5E1,3.4E1,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[9.137725E-4,-1.055163E-2,9.304471E-2,-2.9758938E-2,1.6717345E-2,1.3861521E-2,7.476049E-2,-6.839778E-3,-5.631051E-2,2.103188E-3,6.184057E-2,1.7413996E-1,4.6274275E-2,-3.2698307E-2,3.93252E-2,-1.529468E-1,-4.789199E-2,-5.4928653E-2,1.5323944E-2,3.6751907E-2,1.4346685E-1,3.0696634E-3,1.3234095E-2,7.0604324E-2,-8.198658E-2,2.8834746E-3,-2.4230895E-3,6.0959305E-3,5.730981E-4,-2.288749E-3,-1.0309608E-2,-1.0323564E-3,-3.4647768E-3,1.7605437E-4,-4.831634E-3,1.4110573E-3,-1.1989337E-3,2.4169404E-3,-1.603913E-3,9.606932E-3,3.88928E-3,-4.4063875E-4,3.8575872E-3,-8.624144E-3,2.0584327E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.154088E-1,2.2781089E-1,1.8456203E-1,1.5479244E-1,1.1856874E-1,0E0,1.3632E-1,1.6602455E-1,8.867216E-2,1.0414028E-1,8.483468E-2,1.1609906E-1,1.3373291E-1,1.758452E-1,1.2414593E-1,5.5749834E-2,7.5819224E-2,7.781442E-2,7.294401E-2,4.2220756E-2,2.1025762E-2,0E0,0E0,3.926447E-2,1.0310677E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,6.974661E4,8.381319E3,1.4361977E-3,1.646359E5,1.3861521E-2,1.417988E6,5.068113E2,2E0,2.8529238E1,6.7304E4,5.9E1,1.5291998E4,5.125401E1,9.1417163E2,8.86836E5,3.3027522E-2,1.5151515E-3,1.9470884E5,6.878971E1,2.1207E4,3.0696634E-3,1.3234095E-2,2.7538282E6,1.505649E6,2.8834746E-3,-2.4230895E-3,6.0959305E-3,5.730981E-4,-2.288749E-3,-1.0309608E-2,-1.0323564E-3,-3.4647768E-3,1.7605437E-4,-4.831634E-3,1.4110573E-3,-1.1989337E-3,2.4169404E-3,-1.603913E-3,9.606932E-3,3.88928E-3,-4.4063875E-4,3.8575872E-3,-8.624144E-3,2.0584327E-3],"split_indices":[61,42,42,47,37,0,38,42,8,65,2,8,4,57,56,9,66,66,42,65,9,0,0,56,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,4.33E2,5.3E1,2.54E2,1.79E2,3E0,5E1,1.37E2,1.17E2,1.36E2,4.3E1,1E1,4E1,8.8E1,4.9E1,8E0,1.09E2,2.5E1,1.11E2,3.4E1,9E0,6E0,4E0,3.4E1,6E0,1.5E1,7.3E1,1E1,3.9E1,4E0,4E0,5.8E1,5.1E1,1.2E1,1.3E1,8.1E1,3E1,2.8E1,6E0,3E0,6E0,5E0,2.9E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.0353564E-4,-1.1371207E-2,8.023377E-2,-4.8519074E-4,-5.7494637E-2,1.3385779E-2,6.392948E-2,-3.2390077E-2,2.9079141E-2,1.7420918E-2,-8.111458E-2,-2.210224E-2,9.245382E-2,-4.467212E-2,3.158443E-2,4.2301133E-2,-2.5772974E-2,-4.5584165E-3,4.3933388E-2,-4.342679E-2,-1.5620321E-1,1.4148195E-3,-4.886616E-3,6.264442E-2,1.21651525E-2,-3.2827489E-3,3.9169017E-6,5.632729E-4,3.8851157E-3,9.270637E-4,4.800508E-3,-3.944422E-3,-2.1406004E-4,2.9869506E-3,-9.305039E-4,-5.159635E-3,-1.0890877E-3,-9.133209E-3,-9.939957E-4,3.3383176E-3,-2.0493205E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4042087E-1,2.103522E-1,1.889618E-1,3.2164574E-1,1.4297318E-1,0E0,1.4192134E-1,1.3027191E-1,1.2946023E-1,6.643312E-2,1.65842E-1,7.064835E-2,2.055909E-1,1.671674E-1,2.6702084E-2,1.9520012E-1,4.4208463E-2,0E0,2.4640556E-2,5.4464214E-2,1.1233941E-1,0E0,0E0,2.6613325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0297854E3,2.3790412E7,8.381319E3,6.24E3,1.89285E3,1.3385779E-2,6.219584E5,2.1592189E5,1.1106066E7,1.2304464E1,1E0,1.4609149E4,4.42384E5,3.135717E6,5.313189E11,6.1355795E6,2.0241737E3,-4.5584165E-3,8.3776385E6,2.7883528E7,1.484798E9,1.4148195E-3,-4.886616E-3,2.6066291E0,1.21651525E-2,-3.2827489E-3,3.9169017E-6,5.632729E-4,3.8851157E-3,9.270637E-4,4.800508E-3,-3.944422E-3,-2.1406004E-4,2.9869506E-3,-9.305039E-4,-5.159635E-3,-1.0890877E-3,-9.133209E-3,-9.939957E-4,3.3383176E-3,-2.0493205E-4],"split_indices":[61,54,42,2,4,0,41,37,57,65,22,4,2,9,40,60,61,0,41,54,7,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.77E2,4.18E2,5.9E1,3.39E2,7.9E1,3E0,5.6E1,1.63E2,1.76E2,1.9E1,6E1,1.4E1,4.2E1,1.37E2,2.6E1,1.42E2,3.4E1,3E0,1.6E1,4.1E1,1.9E1,9E0,5E0,3.7E1,5E0,8.5E1,5.2E1,2E1,6E0,1.06E2,3.6E1,8E0,2.6E1,1.2E1,4E0,8E0,3.3E1,1.4E1,5E0,3.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-3.3986228E-4,-2.1304062E-2,3.1940147E-2,-1.827728E-2,-8.73042E-3,2.5315467E-2,7.396029E-3,-4.77357E-2,-2.268844E-3,-7.177758E-2,3.1413175E-2,-6.5465886E-3,-3.911528E-2,1.2307179E-1,-9.297706E-3,7.161185E-3,-9.688216E-3,3.737078E-2,-5.86402E-2,-2.5884334E-3,1.8454324E-4,4.0170585E-4,1.0781871E-2,4.040662E-4,-2.389573E-3,-1.128335E-3,2.065833E-3,1.4520538E-3,1.1477247E-2,1.6258728E-5,-7.4120546E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,-1,19,21,23,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.2483155E-1,1.4578691E-1,1.5827723E-1,1.3523607E-1,0E0,1.09414905E-1,0E0,7.73983E-2,1.6589166E-1,1.2302375E-1,9.39333E-2,0E0,7.09964E-2,1.2655534E-1,1.3895157E-1,1.0781238E-2,0E0,1.8584643E-1,6.7114204E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,18,18],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,-1,20,22,24,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.921638E4,7.339209E6,7.986019E9,4.39E3,-8.73042E-3,3.0811954E-2,7.396029E-3,9.331E3,1.3812E4,4.565957E6,5.37E2,-6.5465886E-3,8E1,6.783E3,3.902439E-2,3.3759567E1,-9.688216E-3,2.0531621E4,1.281E3,-2.5884334E-3,1.8454324E-4,4.0170585E-4,1.0781871E-2,4.040662E-4,-2.389573E-3,-1.128335E-3,2.065833E-3,1.4520538E-3,1.1477247E-2,1.6258728E-5,-7.4120546E-3],"split_indices":[42,56,7,2,0,47,0,9,9,59,3,0,8,2,66,65,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.78E2,2.9E2,1.88E2,2.86E2,4E0,1.8E2,8E0,1E2,1.86E2,1E1,1.7E2,7E0,9.3E1,9E0,1.77E2,7E0,3E0,1.6E2,1E1,6.6E1,2.7E1,5E0,4E0,1.25E2,5.2E1,4E0,3E0,1.57E2,3E0,7E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[-5.0090003E-4,-1.9553663E-2,4.00505E-2,-2.7276374E-2,6.2075917E-2,4.893112E-2,-1.3035746E-1,8.387519E-2,-3.1342328E-2,-2.01382E-3,7.60088E-2,1.3801043E-1,3.4050528E-2,-4.724235E-4,-9.166642E-3,1.24161765E-1,-8.0845936E-4,-3.3461507E-3,-4.2339183E-2,4.598365E-4,9.123512E-2,3.4006415E-3,2.1468456E-1,3.0830258E-3,5.852365E-2,2.0608606E-3,7.4422527E-3,-6.8070315E-4,3.8190223E-3,2.1475786E-3,-2.1301482E-3,4.7771945E-3,4.2170577E-4,1.4887789E-2,4.8112636E-3,6.334673E-4,-6.664455E-3,-5.5986983E-3,3.1191374E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,-1,25,-1,27,29,-1,31,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.770472E-1,2.1098652E-1,2.4245793E-1,1.400178E-1,4.6207987E-2,1.9280809E-1,6.510839E-2,4.719209E-2,9.0372086E-2,0E0,2.5195867E-2,8.6610526E-2,9.758499E-2,0E0,0E0,1.7027617E-2,0E0,8.530139E-2,8.147648E-2,0E0,2.170156E-2,0E0,7.488677E-2,9.482534E-2,1.3383691E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,15,15,17,17,18,18,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,-1,26,-1,28,30,-1,32,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.560409E6,4.2524424E7,5.240162E7,1E0,6.2581446E8,1.417988E6,5.458592E6,1.1351523E1,1.04E2,-2.01382E-3,2.2414202E1,9.256843E3,4.51698E0,-4.724235E-4,-9.166642E-3,6.54925E6,-8.0845936E-4,8.9162E4,5.8601086E1,4.598365E-4,1.9181998E6,3.4006415E-3,4.542969E1,1.3460449E1,1.379E3,2.0608606E-3,7.4422527E-3,-6.8070315E-4,3.8190223E-3,2.1475786E-3,-2.1301482E-3,4.7771945E-3,4.2170577E-4,1.4887789E-2,4.8112636E-3,6.334673E-4,-6.664455E-3,-5.5986983E-3,3.1191374E-3],"split_indices":[1,9,54,20,7,38,38,62,3,0,62,4,62,0,0,9,0,11,59,0,56,0,63,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,3.31E2,1.55E2,3.03E2,2.8E1,1.48E2,7E0,1E1,2.93E2,3E0,2.5E1,2E1,1.28E2,3E0,4E0,7E0,3E0,8.3E1,2.1E2,5E0,2E1,1.2E1,8E0,5.7E1,7.1E1,3E0,4E0,7.4E1,9E0,9E0,2.01E2,1.7E1,3E0,3E0,5E0,5.4E1,3E0,3E0,6.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[4.2722267E-4,-8.255364E-3,6.72445E-2,-1.8340085E-2,3.6911238E-2,1.2371503E-2,5.064725E-2,-4.593645E-2,4.514342E-3,-4.5802698E-2,6.032962E-2,-2.8309224E-2,8.255926E-2,-5.659126E-2,1.3863184E-2,1.2624186E-1,-2.7873812E-3,1.2729289E-3,-7.625753E-2,2.1773584E-2,1.0950323E-1,5.546704E-4,-4.471495E-3,4.072936E-2,1.5986834E-1,-2.2613963E-3,-6.9712475E-3,1.5150683E-3,-3.191281E-3,-1.033966E-3,8.460395E-3,-6.646883E-4,2.7662504E-3,-5.713514E-3,5.7078494E-5,-2.7684662E-3,1.4225465E-3,5.93572E-3,-1.7897942E-3,-2.4054323E-3,2.9948778E-3,4.082187E-3,1.1673216E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.825065E-1,1.9699423E-1,1.7508045E-1,2.2284417E-1,1.5578835E-1,0E0,1.3587917E-1,1.0272914E-1,1.727831E-1,4.2478323E-2,1.1516397E-1,4.6080306E-2,1.16374254E-1,8.589342E-2,4.253039E-2,1.0017952E-1,1.374518E-1,0E0,4.911969E-2,2.9461931E-2,8.615339E-2,0E0,0E0,6.339191E-2,6.471479E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,1.646359E5,8.381319E3,1.656968E6,9.870879E2,1.2371503E-2,6.219584E5,4.552E4,8.593763E7,3.251E3,5.4730445E9,1.4609149E4,1.9568459E0,2.5096E4,3.0113987E10,7.93E2,4.2524424E7,1.2729289E-3,1.0777661E1,2.29033E5,5.52488E1,5.546704E-4,-4.471495E-3,8.713209E10,3.99083E5,-2.2613963E-3,-6.9712475E-3,1.5150683E-3,-3.191281E-3,-1.033966E-3,8.460395E-3,-6.646883E-4,2.7662504E-3,-5.713514E-3,5.7078494E-5,-2.7684662E-3,1.4225465E-3,5.93572E-3,-1.7897942E-3,-2.4054323E-3,2.9948778E-3,4.082187E-3,1.1673216E-2],"split_indices":[61,37,42,9,64,0,41,10,7,2,5,4,48,2,12,0,9,0,62,38,65,0,0,40,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,4.3E2,5.5E1,3.52E2,7.8E1,3E0,5.2E1,1.59E2,1.93E2,1.7E1,6.1E1,1.5E1,3.7E1,1.35E2,2.4E1,1E1,1.83E2,5E0,1.2E1,3.5E1,2.6E1,1E1,5E0,2.5E1,1.2E1,1.27E2,8E0,2E1,4E0,3E0,7E0,1.55E2,2.8E1,7E0,5E0,3E0,3.2E1,2.3E1,3E0,5E0,2E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.0555875E-3,-8.357932E-3,9.884576E-2,-1.6295578E-2,4.549277E-2,1.9938634E-1,4.864608E-2,-2.4777811E-2,3.208391E-2,-4.5362017E-3,5.7592865E-2,2.5218013E-1,3.9170883E-3,-2.201891E-2,1.101333E-1,-3.2763377E-2,1.7161068E-2,4.396672E-3,7.763688E-2,1.1970909E-2,4.1109968E-2,1.4232142E-2,5.9631392E-3,3.269141E-2,-6.388884E-3,3.1838685E-3,8.419918E-3,-1.2690005E-3,-5.652285E-3,-4.615796E-3,1.7816656E-3,-5.9735533E-3,8.816934E-4,4.0850302E-4,4.4035036E-3,2.5523398E-3,-2.8933352E-3,-4.158953E-4,2.5212795E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,31,33,-1,35,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6412415E-1,1.9950455E-1,1.6190216E-1,1.6740362E-1,1.1070367E-1,4.449457E-2,1.0670486E-1,1.16439626E-1,7.659405E-2,0E0,1.7754667E-1,9.657711E-3,0E0,8.176707E-2,2.6142716E-2,1.2948287E-1,1.4783357E-1,8.076914E-2,2.851972E-2,0E0,8.461997E-2,0E0,0E0,8.9252405E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,32,34,-1,36,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.205023E3,6.0784E4,4.0601485E6,3.53E2,1.5153618E0,4.542969E1,1.0158273E2,1.7212875E5,9.715051E-1,-4.5362017E-3,1.803947E0,1.6441814E9,3.9170883E-3,5.01E2,8.625407E2,2.1247113E-1,2.4143283E2,5.1E1,1.9745543E3,1.1970909E-2,9.677921E7,1.4232142E-2,5.9631392E-3,5.7E1,-6.388884E-3,3.1838685E-3,8.419918E-3,-1.2690005E-3,-5.652285E-3,-4.615796E-3,1.7816656E-3,-5.9735533E-3,8.816934E-4,4.0850302E-4,4.4035036E-3,2.5523398E-3,-2.8933352E-3,-4.158953E-4,2.5212795E-3],"split_indices":[61,10,56,8,62,63,65,37,50,0,62,7,0,3,67,66,61,3,4,0,1,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.97E2,4.64E2,3.3E1,4.05E2,5.9E1,1E1,2.3E1,3.45E2,6E1,4E0,5.5E1,6E0,4E0,1.1E1,1.2E1,2.9E2,5.5E1,3.8E1,2.2E1,3E0,5.2E1,3E0,3E0,8E0,3E0,9E0,3E0,2.76E2,1.4E1,8E0,4.7E1,3E0,3.5E1,5E0,1.7E1,4.6E1,6E0,3E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[6.441421E-4,-1.3969045E-2,4.7400486E-2,-2.0904705E-2,3.772046E-2,1.2168313E-1,2.7447063E-2,-6.1741923E-3,-1.82194E-2,1.4143475E-2,8.3864175E-2,2.431821E-1,6.4496316E-2,-6.2572444E-3,3.68931E-2,-2.771639E-2,3.726802E-2,4.8155896E-3,-1.41941E-2,5.066723E-3,-5.013381E-4,4.2718537E-3,1.520284E-2,2.4316192E-2,4.46222E-3,4.7840852E-2,-4.8904363E-2,2.8590295E-3,-1.4441735E-3,1.921192E-6,4.0364778E-3,2.8013808E-3,-1.3072522E-3,-2.08509E-3,3.059181E-3,-4.6900003E-3,2.5410482E-3,-4.372328E-3,7.709121E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,25,27,29,-1,31,-1,-1,-1,-1,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1564465E-1,1.2698081E-1,1.5970144E-1,9.2760116E-2,4.450691E-2,1.4412287E-1,1.4152177E-1,0E0,1.6175678E-1,7.634941E-2,3.6863446E-2,6.2370688E-2,2.017945E-2,0E0,8.1239946E-2,9.241991E-2,8.5327335E-2,0E0,2.6677158E-2,0E0,0E0,0E0,0E0,2.9796764E-2,0E0,9.2360616E-2,3.2276653E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,23,23,25,25,26,26],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,26,28,30,-1,32,-1,-1,-1,-1,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1959E4,4.19E2,1.417988E6,5.349E3,1.3987483E-1,1.01267644E11,1.9813E4,-6.1741923E-3,5.5292703E9,7.202981E-4,1.9181998E6,1E0,9.03027E5,-6.2572444E-3,1.773296E6,5.8601086E1,1.9417112E1,4.8155896E-3,9.481747E8,5.066723E-3,-5.013381E-4,4.2718537E-3,1.520284E-2,2.0964778E5,4.46222E-3,1.5153618E0,4.0297854E3,2.8590295E-3,-1.4441735E-3,1.921192E-6,4.0364778E-3,2.8013808E-3,-1.3072522E-3,-2.08509E-3,3.059181E-3,-4.6900003E-3,2.5410482E-3,-4.372328E-3,7.709121E-4],"split_indices":[2,8,38,9,47,40,10,0,5,48,56,22,38,0,11,59,65,0,7,0,0,0,0,37,0,62,61,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.6E2,3.51E2,1.09E2,3.1E2,4.1E1,2.2E1,8.7E1,6E0,3.04E2,2.8E1,1.3E1,6E0,1.6E1,4E0,8.3E1,2.6E2,4.4E1,6E0,2.2E1,1E1,3E0,3E0,3E0,8E0,8E0,7.4E1,9E0,1E1,2.5E2,2.6E1,1.8E1,3E0,1.9E1,3E0,5E0,3E0,7.1E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.5499245E-3,-7.1741897E-3,8.141263E-2,-4.938241E-2,1.1098593E-3,4.3379493E-2,6.315867E-3,-3.7775334E-2,-9.3503E-3,-9.052656E-3,4.5895174E-2,3.886232E-3,1.668651E-3,3.9601367E-2,-4.9646165E-2,9.581049E-3,-3.306078E-2,1.1369451E-1,1.8935906E-2,-3.3045216E-3,1.7572559E-3,2.7431296E-3,-2.6992423E-4,-4.645376E-3,-1.7261124E-3,-1.4707266E-3,1.266321E-3,-2.1711604E-3,1.3808006E-3,3.2099965E-3,1.05393715E-2,5.049691E-3,1.036087E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2781993E-1,1.5984713E-1,6.0750753E-2,1.2716506E-1,1.7473081E-1,3.451498E-2,0E0,6.718139E-2,0E0,1.4034054E-1,1.2762234E-1,0E0,3.2624125E-2,9.609424E-3,3.452316E-2,1.3448179E-1,1.2604451E-1,8.5413426E-2,7.8998275E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9596844E9,1.1231155E3,2.2876814E1,1.7239808E7,1.5271514E5,8.686297E4,6.315867E-3,1.1760288E-5,-9.3503E-3,2.5871767E-2,3.4193E4,3.886232E-3,6.001104E9,3.329E3,2.398E3,7.1306056E7,6.0784E4,1.082556E7,2.5159248E1,-3.3045216E-3,1.7572559E-3,2.7431296E-3,-2.6992423E-4,-4.645376E-3,-1.7261124E-3,-1.4707266E-3,1.266321E-3,-2.1711604E-3,1.3808006E-3,3.2099965E-3,1.05393715E-2,5.049691E-3,1.036087E-4],"split_indices":[7,64,63,54,37,52,0,50,0,66,9,0,7,2,2,7,10,1,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.86E2,4.56E2,3E1,7.4E1,3.82E2,1.9E1,1.1E1,7E1,4E0,3.12E2,7E1,9E0,1E1,9E0,6.1E1,1.76E2,1.36E2,1.9E1,5.1E1,3E0,7E0,6E0,3E0,1E1,5.1E1,5.3E1,1.23E2,1.11E2,2.5E1,1.5E1,4E0,7E0,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.8149374E-3,-6.408598E-3,6.631016E-2,2.9626612E-3,-4.381997E-2,1.110593E-2,5.344626E-2,-3.500839E-2,1.7776208E-2,1.9872135E-2,-6.701982E-2,1.366022E-1,2.9811399E-2,-6.629501E-3,-2.6268793E-2,-8.065232E-3,3.410533E-2,-3.044823E-2,7.272985E-2,-3.7142664E-2,-1.2642817E-1,3.1259847E-3,1.062617E-2,4.9448173E-2,-9.226801E-2,-3.0919309E-3,-1.4266014E-4,-1.8781755E-3,1.002012E-3,7.2560525E-3,1.3604644E-3,-3.8998383E-3,1.3121715E-3,-7.214554E-5,5.767381E-3,-2.244681E-3,3.0000266E-3,-8.166794E-3,-8.9518673E-4,2.6104162E-3,-1.69188E-3,-8.639766E-3,1.2506883E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.909076E-1,1.5228659E-1,1.2997279E-1,1.9636743E-1,1.2988715E-1,0E0,1.1385043E-1,9.0289176E-2,1.06189065E-1,6.605016E-2,1.0890108E-1,6.901558E-2,1.1963028E-1,0E0,8.7749384E-2,9.772811E-2,7.915659E-2,4.4541284E-2,4.772616E-2,5.7030268E-2,1.1328712E-1,0E0,0E0,2.9678486E-2,8.5975215E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,2.3471036E7,2.909019E3,4.39E3,1.9703646E3,1.110593E-2,1.417988E6,4.6E1,7.4378266E4,2.231E3,1E0,1.5096262E7,2.0531621E4,-6.629501E-3,4.48149E5,1.31E2,1.803947E0,6.631343E2,1.5716302E3,1.6942582E7,4.4997E4,3.1259847E-3,1.062617E-2,2.9928794E5,1.0787629E0,-3.0919309E-3,-1.4266014E-4,-1.8781755E-3,1.002012E-3,7.2560525E-3,1.3604644E-3,-3.8998383E-3,1.3121715E-3,-7.214554E-5,5.767381E-3,-2.244681E-3,3.0000266E-3,-8.166794E-3,-8.9518673E-4,2.6104162E-3,-1.69188E-3,-8.639766E-3,1.2506883E-3],"split_indices":[61,54,41,2,4,0,38,3,37,0,22,9,4,0,1,8,62,61,4,9,10,0,0,42,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.95E2,4.33E2,6.2E1,3.47E2,8.6E1,3E0,5.9E1,9.7E1,2.5E2,2.3E1,6.3E1,1.2E1,4.7E1,6E0,9.1E1,9.7E1,1.53E2,1.2E1,1.1E1,4.3E1,2E1,8E0,4E0,4.1E1,6E0,3.2E1,5.9E1,4.6E1,5.1E1,4E0,1.49E2,6E0,6E0,5E0,6E0,3.9E1,4E0,1.3E1,7E0,3.8E1,3E0,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.5642615E-3,-2.112636E-2,2.2177706E-2,-3.3319276E-2,2.6797688E-2,2.778066E-2,-8.2967415E-2,-4.300846E-2,3.954287E-2,-6.8218587E-3,6.874674E-2,6.0759312E-3,2.3115246E-2,-1.3586139E-2,-1.0785031E-2,-3.561669E-2,-1.1860601E-1,8.004326E-2,-1.7061925E-2,2.2906002E-2,-8.028878E-2,-7.4773567E-4,9.6357524E-2,5.55545E-3,5.0563868E-2,8.681788E-4,-3.0736208E-3,-2.3769669E-3,-7.050032E-4,-2.3680162E-3,-7.426283E-3,5.778743E-3,1.6181642E-3,1.1277238E-3,-1.7598146E-3,2.390835E-3,-8.584138E-4,-1.4367678E-3,-5.5523226E-3,7.1498347E-3,1.483653E-3,5.8220914E-3,-3.4224468E-5,8.8149E-3,1.6594001E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,35,37,-1,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2497982E-1,1.3470536E-1,1.5076393E-1,1.3089405E-1,6.675346E-2,1.1449146E-1,1.3453704E-1,8.630124E-2,5.1801044E-2,6.0807485E-2,5.0909907E-2,0E0,1.1105928E-1,1.8718738E-2,0E0,4.836294E-2,3.2484442E-2,2.239196E-2,9.870242E-3,2.5428262E-2,1.2243867E-2,0E0,5.6193963E-2,1.0899985E-1,1.7587546E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,36,38,-1,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.97E3,2.103E3,4.5859156E7,7.164074E8,1E0,6.798766E-4,2.046E3,1.316826E6,6.173444E1,2.2083527E2,4.58149E2,6.0759312E-3,1.3590209E7,6.941001E7,-1.0785031E-2,2.7E2,9.30419E5,3.1536636E1,1.7608592E-4,1.0468137E7,9.5E1,-7.4773567E-4,1.68804E6,7.062E3,1.6459E4,8.681788E-4,-3.0736208E-3,-2.3769669E-3,-7.050032E-4,-2.3680162E-3,-7.426283E-3,5.778743E-3,1.6181642E-3,1.1277238E-3,-1.7598146E-3,2.390835E-3,-8.584138E-4,-1.4367678E-3,-5.5523226E-3,7.1498347E-3,1.483653E-3,5.8220914E-3,-3.4224468E-5,8.8149E-3,1.6594001E-3],"split_indices":[2,0,54,7,13,48,0,38,65,42,61,0,1,54,0,3,9,67,47,1,8,0,12,2,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.79E2,2.28E2,2.51E2,1.82E2,4.6E1,2.39E2,1.2E1,1.61E2,2.1E1,2.6E1,2E1,9E0,2.3E2,9E0,3E0,1.48E2,1.3E1,1.2E1,9E0,1.9E1,7E0,5E0,1.5E1,1.41E2,8.9E1,6E0,3E0,8.1E1,6.7E1,6E0,7E0,5E0,7E0,3E0,6E0,1.1E1,8E0,4E0,3E0,7E0,8E0,6E0,1.35E2,7E0,8.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.9743359E-3,-7.6809046E-3,4.9260333E-2,-5.508333E-3,-9.932241E-3,1.0524506E-1,2.7549753E-2,-1.778755E-2,1.7991576E-2,1.2896854E-1,-1.5445759E-3,4.213753E-3,8.495553E-2,-1.18826935E-2,-8.687072E-2,-7.331473E-3,4.934553E-2,1.139413E-2,9.090288E-2,4.5208886E-2,-2.7111098E-2,1.9661554E-3,5.3772763E-3,-3.8170427E-4,-5.8142846E-3,-5.494635E-3,1.6814669E-3,-1.7403716E-3,1.2347882E-3,6.27791E-3,8.982413E-4,4.75521E-3,1.1850039E-3,-1.2358917E-3,2.7657086E-3,-3.6492224E-3,9.1246446E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1541704E-1,1.7710748E-1,9.4337404E-2,1.12570055E-1,0E0,7.689822E-2,7.8679666E-2,1.0339686E-1,1.0675478E-1,6.417069E-2,0E0,5.6444824E-2,1.8187262E-2,9.561042E-2,8.649261E-2,8.003497E-2,1.5392993E-1,0E0,1.0841012E-2,2.2527304E-2,6.350063E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2729566E4,1.69595E5,2.412E4,1.235225E3,-9.932241E-3,2.5508283E5,9.70625E1,1E0,8.459549E4,2.9E1,-1.5445759E-3,1.423795E7,2.08617E5,3.9034148E7,1.5271514E5,9.6E1,1.3482595E8,1.139413E-2,3.7137984E7,1.1E1,5.0314346E1,1.9661554E-3,5.3772763E-3,-3.8170427E-4,-5.8142846E-3,-5.494635E-3,1.6814669E-3,-1.7403716E-3,1.2347882E-3,6.27791E-3,8.982413E-4,4.75521E-3,1.1850039E-3,-1.2358917E-3,2.7657086E-3,-3.6492224E-3,9.1246446E-4],"split_indices":[52,2,9,61,0,42,65,18,37,3,0,54,9,54,37,8,7,0,1,8,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.7E2,3.91E2,7.9E1,3.88E2,3E0,2.1E1,5.8E1,2.55E2,1.33E2,1.8E1,3E0,4.2E1,1.6E1,2.36E2,1.9E1,7.4E1,5.9E1,3E0,1.5E1,1.8E1,2.4E1,8E0,8E0,2.3E2,6E0,1.5E1,4E0,3.9E1,3.5E1,1.4E1,4.5E1,1.2E1,3E0,3E0,1.5E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.6477791E-3,-6.4590303E-3,7.472544E-2,4.942714E-3,-3.0290475E-2,4.7270074E-2,7.05922E-3,-1.8581193E-2,2.395083E-2,-8.867219E-3,-2.4444351E-2,3.2301012E-3,-5.269423E-3,-4.5311302E-2,1.9990882E-2,-4.2576656E-2,3.243524E-2,-4.8484866E-2,1.245237E-2,-1.9561998E-3,1.1791404E-3,-2.9321543E-3,-9.795469E-4,-1.613317E-4,3.519715E-3,-4.8895404E-3,9.025367E-4,3.2577005E-3,8.120078E-4,-1.3952795E-3,-4.9487595E-3,1.5853331E-3,-3.0672771E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,-1,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.797986E-1,1.2514094E-1,5.6034625E-2,1.3995431E-1,1.3766587E-1,2.8546426E-2,0E0,1.4525083E-1,9.903021E-2,0E0,1.292211E-1,0E0,1.0407013E-2,3.5858303E-2,7.7911675E-2,8.2188636E-2,8.6412385E-2,9.1172025E-2,1.0461842E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,-1,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.9596844E9,1.7458724E7,1.0871896E10,7.4378266E4,1.01319025E3,5.848343E7,7.05922E-3,1.31E2,3.807E3,-8.867219E-3,5.5292703E9,3.2301012E-3,4.4269304E7,1.0368186E-1,3.0442734E3,2.040228E5,1.0545219E5,1.9359E4,2.6650991E0,-1.9561998E-3,1.1791404E-3,-2.9321543E-3,-9.795469E-4,-1.613317E-4,3.519715E-3,-4.8895404E-3,9.025367E-4,3.2577005E-3,8.120078E-4,-1.3952795E-3,-4.9487595E-3,1.5853331E-3,-3.0672771E-3],"split_indices":[7,54,7,37,4,1,0,8,2,0,5,0,52,47,4,37,37,10,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.87E2,4.59E2,2.8E1,3.11E2,1.48E2,2.2E1,6E0,1.39E2,1.72E2,4E0,1.44E2,1.5E1,7E0,8.2E1,5.7E1,1.9E1,1.53E2,8.7E1,5.7E1,3E0,4E0,4.5E1,3.7E1,4.1E1,1.6E1,9E0,1E1,4.1E1,1.12E2,6.8E1,1.9E1,4.5E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[2.677362E-3,-1.3512606E-2,3.0416882E-2,-2.0034203E-2,5.0820027E-2,1.07363164E-1,1.6904276E-2,-2.964198E-2,1.632886E-2,-3.1284042E-5,6.428549E-2,1.0901113E-2,6.679604E-2,-1.00741684E-1,2.565223E-2,-1.6180316E-2,-7.525275E-2,-8.521953E-2,4.0864613E-2,9.501162E-2,2.2128647E-2,-7.8058094E-3,4.673659E-3,-7.651265E-3,-1.7792513E-3,1.6501807E-2,8.247481E-2,-2.248393E-3,6.587419E-5,-1.3901902E-3,-5.080834E-3,-5.601669E-3,9.0441474E-4,-3.341867E-3,2.4075725E-3,5.5041467E-3,3.7024956E-4,-2.0278673E-4,2.5117134E-3,-3.1504533E-3,1.9488329E-3,1.0424538E-3,-5.212581E-3,2.1755197E-3,5.9002237E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,25,27,29,31,33,35,37,39,-1,-1,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2181825E-1,1.3191824E-1,1.8741868E-1,9.970458E-2,2.015604E-2,1.274122E-1,1.6313344E-1,1.369317E-1,1.5297644E-1,0E0,2.7689807E-2,0E0,5.878979E-2,3.872092E-2,7.4551806E-2,1.0167861E-1,7.850182E-2,4.9271584E-2,6.9396704E-2,2.7577974E-2,9.813898E-3,2.7539179E-2,0E0,0E0,0E0,1.0657589E-1,2.6331022E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,26,28,30,32,34,36,38,40,-1,-1,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.472533E3,4.2524424E7,2.1207E4,5.5292703E9,2.3929121E1,3.7137297E3,5.767668E0,3.1174E4,7.532E3,-3.1284042E-5,1.21E2,1.0901113E-2,8.586521E-2,3.2E1,3.221225E9,1.65602E5,1.0632783E7,1.7E2,6.105804E2,4.1124658E6,2.7797632E1,9.5430945E4,4.673659E-3,-7.651265E-3,-1.7792513E-3,3.63796E5,1E0,-2.248393E-3,6.587419E-5,-1.3901902E-3,-5.080834E-3,-5.601669E-3,9.0441474E-4,-3.341867E-3,2.4075725E-3,5.5041467E-3,3.7024956E-4,-2.0278673E-4,2.5117134E-3,-3.1504533E-3,1.9488329E-3,1.0424538E-3,-5.212581E-3,2.1755197E-3,5.9002237E-3],"split_indices":[61,9,9,5,62,41,63,11,9,0,3,0,47,3,7,9,54,3,64,54,67,37,0,0,0,2,13,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,3.11E2,1.81E2,2.83E2,2.8E1,2.6E1,1.55E2,2.24E2,5.9E1,6E0,2.2E1,5E0,2.1E1,1E1,1.45E2,1.74E2,5E1,1.1E1,4.8E1,1.2E1,1E1,7E0,1.4E1,4E0,6E0,1.26E2,1.9E1,6E1,1.14E2,2.3E1,2.7E1,8E0,3E0,4E0,4.4E1,9E0,3E0,6E0,4E0,3E0,4E0,1.21E2,5E0,1.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.0198154E-3,-7.0830295E-3,6.247209E-2,-5.641481E-2,-2.0088144E-3,9.835577E-3,5.024832E-2,-3.3909917E-2,-7.3621226E-3,2.2032443E-2,-1.4477161E-2,1.1881178E-1,2.6197447E-2,2.6940487E-3,-4.7780782E-2,5.1851247E-2,-1.3815853E-2,-2.445756E-2,2.5717597E-2,2.9405707E-3,1.0942981E-2,-4.7570486E-3,3.887447E-2,-6.279187E-3,-1.5237677E-3,1.5676852E-3,6.091957E-3,-1.3448596E-3,1.7378202E-3,-4.0824202E-4,-2.0600923E-3,1.8029659E-3,-6.9434424E-3,-1.8479022E-3,2.6300796E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,21,-1,23,25,27,29,31,-1,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.479178E-1,1.1002751E-1,9.72901E-2,9.0934664E-2,1.20220855E-1,0E0,8.758305E-2,4.785642E-2,0E0,1.4716297E-1,1.06535174E-1,7.652661E-2,7.333773E-2,0E0,3.558415E-2,1.0268015E-1,5.192704E-2,6.73189E-2,1.3464962E-1,0E0,0E0,0E0,5.9809923E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,22,22],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,22,-1,24,26,28,30,32,-1,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,9.357879E2,2.909019E3,1.0361716E7,7.477264E6,9.835577E-3,1.417988E6,9.161074E-6,-7.3621226E-3,1.33E2,4.723148E11,4.916536E7,8.713209E10,2.6940487E-3,5.5345844E2,5.841568E1,5.336051E6,1.4040858E-2,1.2622761E8,2.9405707E-3,1.0942981E-2,-4.7570486E-3,4.3290512E1,-6.279187E-3,-1.5237677E-3,1.5676852E-3,6.091957E-3,-1.3448596E-3,1.7378202E-3,-4.0824202E-4,-2.0600923E-3,1.8029659E-3,-6.9434424E-3,-1.8479022E-3,2.6300796E-3],"split_indices":[61,64,41,54,54,0,38,50,0,3,40,1,40,0,64,65,54,66,1,0,0,0,65,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.96E2,4.39E2,5.7E1,4E1,3.99E2,3E0,5.4E1,3.4E1,6E0,1.36E2,2.63E2,1.3E1,4.1E1,4E0,3E1,7.4E1,6.2E1,2.11E2,5.2E1,1E1,3E0,3E0,3.8E1,3E0,2.7E1,6.2E1,1.2E1,4.8E1,1.4E1,1.21E2,9E1,4.9E1,3E0,7E0,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.030512E-3,-9.0735145E-3,3.423375E-2,1.1722574E-2,-2.609044E-2,1.0405411E-1,1.7832728E-2,2.8405685E-2,-5.5897553E-2,-4.985205E-2,-1.0669883E-3,1.2633564E-1,-1.5269311E-3,-5.852866E-2,3.0944217E-2,-1.1061545E-2,5.5202536E-2,6.9846205E-2,-8.7601736E-2,-9.0833075E-2,-2.8311647E-2,-8.563631E-2,1.5651235E-2,9.900361E-3,7.230619E-2,-3.8468698E-3,-7.933755E-4,1.9246725E-2,7.729012E-2,-1.463742E-3,3.282967E-3,1.8160396E-3,5.8824145E-3,-6.0590963E-5,5.6540677E-3,-4.933877E-3,8.633304E-4,-6.8873557E-4,-4.730834E-3,-1.5864422E-3,1.9258622E-3,-7.065318E-3,-6.86567E-4,2.2471726E-3,-1.4661776E-3,-2.7802656E-4,4.3123583E-3,-3.2534278E-3,1.1806118E-3,1.592569E-3,5.5252383E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,-1,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7016377E-1,1.2558131E-1,1.3784972E-1,1.8208608E-1,1.15893304E-1,7.464042E-2,1.0193228E-1,1.3704069E-1,1.3368995E-1,8.570315E-2,1.3706982E-1,8.162162E-2,0E0,1.4319744E-2,4.513567E-2,9.412324E-2,8.253288E-2,2.6920833E-2,5.79931E-2,3.0772686E-2,3.1436983E-2,7.53889E-2,1.3080776E-1,0E0,2.4636365E-2,0E0,0E0,4.3271556E-2,2.718202E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,-1,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.144067E3,1.33E2,2.1207E4,1.3185959E7,1.1639729E9,2.4121311E5,3.6183386E0,1.49529E6,2.1862803E8,1.21506645E2,9.687575E2,1.4459738E-1,-1.5269311E-3,1.9078612E-1,1E0,3.3264E4,5.782887E1,3.183103E6,5.368828E8,2.1694159E1,3.522659E2,2.72E2,5.050505E-3,9.900361E-3,1.8140173E-1,-3.8468698E-3,-7.933755E-4,2.2849871E-1,3.5365555E6,-1.463742E-3,3.282967E-3,1.8160396E-3,5.8824145E-3,-6.0590963E-5,5.6540677E-3,-4.933877E-3,8.633304E-4,-6.8873557E-4,-4.730834E-3,-1.5864422E-3,1.9258622E-3,-7.065318E-3,-6.86567E-4,2.2471726E-3,-1.4661776E-3,-2.7802656E-4,4.3123583E-3,-3.2534278E-3,1.1806118E-3,1.592569E-3,5.5252383E-3],"split_indices":[61,3,9,54,5,42,62,9,7,63,4,47,0,47,18,11,65,1,41,67,63,3,66,0,47,0,0,51,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.74E2,3.53E2,1.21E2,1.59E2,1.94E2,2.2E1,9.9E1,1.28E2,3.1E1,9.9E1,9.5E1,1.9E1,3E0,1.4E1,8.5E1,5.2E1,7.6E1,6E0,2.5E1,3.3E1,6.6E1,1.5E1,8E1,6E0,1.3E1,8E0,6E0,6.9E1,1.6E1,4.2E1,1E1,6.4E1,1.2E1,3E0,3E0,2.1E1,4E0,5E0,2.8E1,6.1E1,5E0,7E0,8E0,4.7E1,3.3E1,3E0,1E1,4E0,6.5E1,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[2.809719E-4,-1.1090273E-2,3.6530487E-2,-3.1010091E-2,2.0790976E-4,9.8615676E-2,2.3487283E-2,-3.872169E-2,3.326414E-2,6.8468004E-2,-7.3775076E-3,1.3530424E-2,4.6448525E-2,3.2409444E-2,-6.816043E-3,-3.02189E-2,-1.4117654E-1,5.9476253E-3,-2.8085692E-2,1.12989545E-1,-5.1611094E-3,-1.4814646E-2,3.9226573E-2,1.8457223E-2,3.4610208E-3,-3.9313473E-3,4.0021844E-2,-4.2869984E-3,-1.0276443E-3,-1.0011449E-2,4.8113614E-4,-3.742512E-3,3.0251313E-4,1.3730119E-3,8.321825E-3,-1.9380285E-3,1.2213895E-3,-8.6472044E-4,3.609325E-3,-1.6699018E-3,2.2419149E-3,2.59287E-3,-7.5300556E-4,1.0029143E-2,1.4288792E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,-1,31,33,35,37,39,41,-1,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.01972E-1,8.3952986E-2,9.242919E-2,6.8201765E-2,1.2425486E-1,1.9186397E-1,1.5559886E-1,1.0114321E-1,9.264348E-2,7.925711E-2,7.544694E-2,0E0,1.2490876E-2,8.810307E-2,0E0,5.3562365E-2,1.08237535E-1,0E0,1.9171577E-2,8.026281E-2,1.2975707E-2,7.299727E-2,2.3968488E-2,1.4269918E-2,0E0,0E0,1.3341719E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,-1,32,34,36,38,40,42,-1,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.1959E4,1.6858337E3,1.417988E6,8.9824E4,1E0,1.01267644E11,3.768034E4,1.7377544E7,1.6776918E3,2.8184534E7,4.2524424E7,1.3530424E-2,7.3E1,2.4950776E0,-6.816043E-3,5E1,2.515304E7,5.9476253E-3,2.2776287E1,1.133E3,1.7044863E1,6.969457E9,6.2581446E8,5.2E1,3.4610208E-3,-3.9313473E-3,1.803947E0,-4.2869984E-3,-1.0276443E-3,-1.0011449E-2,4.8113614E-4,-3.742512E-3,3.0251313E-4,1.3730119E-3,8.321825E-3,-1.9380285E-3,1.2213895E-3,-8.6472044E-4,3.609325E-3,-1.6699018E-3,2.2419149E-3,2.59287E-3,-7.5300556E-4,1.0029143E-2,1.4288792E-3],"split_indices":[2,4,38,11,109,40,4,54,42,54,9,0,8,63,0,3,54,0,67,0,65,5,7,3,0,0,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,3.72E2,1.16E2,1.34E2,2.38E2,1.9E1,9.7E1,1.2E2,1.4E1,2.3E1,2.15E2,3E0,1.6E1,9.3E1,4E0,1.12E2,8E0,5E0,9E0,1.4E1,9E0,1.86E2,2.9E1,9E0,7E0,5E0,8.8E1,1.1E1,1.01E2,5E0,3E0,3E0,6E0,7E0,7E0,4E0,5E0,1.79E2,7E0,3E0,2.6E1,4E0,5E0,3E0,8.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.0404365E-3,-6.9130235E-4,8.443728E-2,1.1064081E-3,-6.237451E-3,1.6985357E-1,3.9914828E-2,1.7756363E-2,-1.2484382E-2,1.1499928E-2,4.0001143E-3,-1.9490385E-3,5.952615E-2,-2.3922034E-2,2.8709542E-2,-3.193114E-2,4.863001E-3,-1.06362866E-4,7.126837E-2,-2.850984E-3,4.144646E-4,2.180125E-3,-3.2032697E-4,3.7189508E-5,-2.0681608E-3,-1.5641481E-3,1.317701E-3,4.312691E-3,1.2935343E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,17,19,21,23,25,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2304855E-1,1.1180707E-1,1.187174E-1,1.0272926E-1,0E0,5.1894367E-2,3.9750468E-2,9.3815E-2,8.449286E-2,0E0,0E0,0E0,1.3964616E-2,5.4989234E-2,1.0985036E-1,5.1282406E-2,1.2525716E-1,0E0,1.3142914E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,18,18],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,18,20,22,24,26,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.651788E3,5.183582E7,3.0418E4,1.26E2,-6.237451E-3,4.542969E1,6.7211235E1,1.3350947E3,1.1639729E9,1.1499928E-2,4.0001143E-3,-1.9490385E-3,1.14003E5,5.24228E5,1.9181998E6,1.1183373E4,7.127066E2,-1.06362866E-4,1.0028699E10,-2.850984E-3,4.144646E-4,2.180125E-3,-3.2032697E-4,3.7189508E-5,-2.0681608E-3,-1.5641481E-3,1.317701E-3,4.312691E-3,1.2935343E-3],"split_indices":[61,59,10,3,0,63,65,4,5,0,0,0,2,9,56,57,61,0,12,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.89E2,4.57E2,3.2E1,4.52E2,5E0,1E1,2.2E1,2.03E2,2.49E2,4E0,6E0,4E0,1.8E1,4.2E1,1.61E2,1.17E2,1.32E2,3E0,1.5E1,1.9E1,2.3E1,1.05E2,5.6E1,3.4E1,8.3E1,5E1,8.2E1,9E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-1.7959897E-4,-1.2943243E-2,2.1048598E-2,2.3910406E-3,-3.8741313E-2,8.455461E-2,9.333359E-3,2.3352195E-2,-2.255601E-2,-1.6757347E-2,-7.707599E-2,1.0461776E-1,-1.4951234E-3,-9.341148E-2,1.6855909E-2,-5.4171365E-3,4.8683487E-2,-3.6409903E-2,1.7059276E-2,2.8739471E-3,-1.10604584E-1,-9.654161E-2,2.5538637E-3,1.6006508E-1,6.9109306E-2,-6.0225255E-3,2.8974403E-4,-5.3856596E-3,2.137916E-2,-7.973666E-4,4.9667004E-3,2.6993705E-3,-9.977535E-4,-2.5688524E-3,-4.672436E-4,1.3182979E-3,-2.2371074E-3,6.2192924E-4,-7.257556E-3,-7.030387E-3,1.1262851E-3,-1.95554E-3,-6.2008565E-3,2.5764466E-3,1.0608248E-2,3.4634655E-4,3.7412841E-3,2.5731986E-3,3.878985E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,-1,43,45,-1,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3493416E-1,1.2312266E-1,1.3833661E-1,1.0301886E-1,9.646723E-2,7.162419E-2,1.2446614E-1,7.824151E-2,5.0015472E-2,1.3849357E-1,1.1402796E-1,3.815922E-2,0E0,4.2739943E-2,9.319495E-2,7.1671456E-2,4.2685866E-2,3.4052633E-2,2.0002844E-2,1.1117143E-1,7.941328E-2,7.177925E-2,0E0,5.3954244E-2,1.2612134E-2,0E0,0E0,0E0,6.472628E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,-1,44,46,-1,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.472533E3,1.3185959E7,2.1207E4,1.33E2,5.050505E-3,2.4121311E5,5.767668E0,3.006846E6,1.688021E6,1.4555195E5,1.5271514E5,2.9E1,-1.4951234E-3,1.457806E0,2.4903E4,1.1356496E12,1.8722918E6,8E1,8.487486E-2,4.2663252E7,1E0,2.530058E6,2.5538637E-3,9.904738E6,1.1E1,-6.0225255E-3,2.8974403E-4,-5.3856596E-3,1E0,-7.973666E-4,4.9667004E-3,2.6993705E-3,-9.977535E-4,-2.5688524E-3,-4.672436E-4,1.3182979E-3,-2.2371074E-3,6.2192924E-4,-7.257556E-3,-7.030387E-3,1.1262851E-3,-1.95554E-3,-6.2008565E-3,2.5764466E-3,1.0608248E-2,3.4634655E-4,3.7412841E-3,2.5731986E-3,3.878985E-4],"split_indices":[61,54,9,3,66,42,63,9,1,37,37,3,0,50,9,40,56,8,66,54,23,1,0,1,8,0,0,0,25,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.96E2,3.1E2,1.86E2,1.95E2,1.15E2,2.8E1,1.58E2,1.06E2,8.9E1,7.4E1,4.1E1,2.4E1,4E0,1E1,1.48E2,5E1,5.6E1,6.6E1,2.3E1,6.2E1,1.2E1,3.6E1,5E0,8E0,1.6E1,7E0,3E0,4E0,1.44E2,4.6E1,4E0,4.9E1,7E0,3.7E1,2.9E1,2E1,3E0,5.9E1,3E0,9E0,3E0,1.6E1,2E1,4E0,4E0,3E0,1.3E1,3.8E1,1.06E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[3.095825E-3,-1.12879705E-2,2.3395125E-2,5.8148876E-2,-1.5250526E-2,-6.428815E-2,2.80083E-2,-1.0805523E-3,3.6864146E-3,-1.22561894E-1,-1.0914414E-2,5.9056777E-4,-6.592535E-3,2.4054054E-2,5.147685E-3,-1.5145984E-3,-7.1468377E-3,1.4877737E-2,-2.3516983E-2,2.7448937E-2,-4.5257593E-3,-1.1446992E-3,2.405615E-3,-4.183331E-3,-6.547122E-4,1.0870026E-3,8.17128E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,-1,19,-1,-1,-1,21,23,25,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.366192E-1,7.62772E-2,8.027328E-2,2.9383969E-2,1.19643986E-1,6.3439906E-2,5.909498E-2,0E0,0E0,2.4903148E-2,8.193676E-2,0E0,0E0,7.701365E-2,0E0,0E0,0E0,1.2611073E-1,1.0437541E-1,9.29223E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,13,13,17,17,18,18,19,19],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,-1,20,-1,-1,-1,22,24,26,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.921638E4,5.8601086E1,3.0811954E-2,2.518512E6,3.9870991E-7,1.0669252E-1,1.0871896E10,-1.0805523E-3,3.6864146E-3,1.6E1,1.2755817E4,5.9056777E-4,-6.592535E-3,3.768034E4,5.147685E-3,-1.5145984E-3,-7.1468377E-3,4.407E3,2.600076E-4,1.4609149E4,-4.5257593E-3,-1.1446992E-3,2.405615E-3,-4.183331E-3,-6.547122E-4,1.0870026E-3,8.17128E-3],"split_indices":[42,59,47,9,47,48,7,0,0,8,59,0,0,4,0,0,0,2,47,4,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.66E2,2.73E2,1.93E2,1.4E1,2.59E2,9E0,1.84E2,3E0,1.1E1,9E0,2.5E2,5E0,4E0,1.77E2,7E0,3E0,6E0,8.2E1,1.68E2,1.73E2,4E0,4E1,4.2E1,1.9E1,1.49E2,1.7E2,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.297696E-3,-1.6204981E-3,4.840598E-2,1.5414668E-4,-5.854816E-3,1.0987395E-1,2.635085E-2,-3.2207318E-2,5.8280635E-3,1.7652728E-1,3.8808308E-4,-3.483139E-3,4.1439153E-2,-2.4476083E-2,-6.5007457E-3,1.953853E-2,-1.0544457E-2,3.905837E-3,1.2317119E-2,-4.840482E-2,5.9241235E-2,-1.7207969E-3,8.3690265E-4,1.2983924E-3,-1.3386984E-3,-1.0888221E-3,1.5352663E-3,-4.1900175E-3,3.1362587E-4,7.8766345E-4,3.3993938E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,-1,19,21,-1,23,25,-1,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2798871E-1,9.7087644E-2,7.573441E-2,7.8775465E-2,0E0,9.987606E-2,7.117966E-2,5.087646E-2,8.213936E-2,5.303806E-2,0E0,0E0,6.524454E-2,3.4932345E-2,0E0,8.6609624E-2,9.835857E-2,0E0,0E0,1.7321032E-2,1.958625E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,12,12,13,13,15,15,16,16,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,-1,20,22,-1,24,26,-1,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,9.163994E6,2.0357608E5,1.0780007E3,-5.854816E-3,2.7125356E0,2.2444414E5,1.7239808E7,1.8887723E-2,1.69595E5,3.8808308E-4,-3.483139E-3,5.0314346E1,3.116E3,-6.5007457E-3,3.0840238E7,9.407814E6,3.905837E-3,1.2317119E-2,1E0,4.648618E0,-1.7207969E-3,8.3690265E-4,1.2983924E-3,-1.3386984E-3,-1.0888221E-3,1.5352663E-3,-4.1900175E-3,3.1362587E-4,7.8766345E-4,3.3993938E-3],"split_indices":[61,56,42,64,0,44,42,54,66,2,0,0,65,0,0,54,1,0,0,22,62,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.89E2,4.32E2,5.7E1,4.27E2,5E0,1.4E1,4.3E1,6.3E1,3.64E2,8E0,6E0,5E0,3.8E1,6E1,3E0,1.98E2,1.66E2,5E0,3E0,6E0,3.2E1,4.6E1,1.4E1,1.68E2,3E1,1.28E2,3.8E1,3E0,3E0,9E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.5541693E-3,-1.3286344E-2,2.4675498E-2,-2.20859E-2,2.1052944E-2,1.2195035E-1,1.8499432E-2,-1.5158095E-2,-9.63575E-2,-6.265242E-2,5.592634E-2,1.6282317E-1,-1.893335E-3,7.329423E-2,1.3545236E-2,-2.5387967E-2,4.5573987E-2,-1.3493663E-1,3.563951E-4,1.6019363E-3,-1.0521645E-1,1.32990545E-2,8.296912E-2,4.7854437E-3,1.0758918E-2,9.3519576E-2,-4.6360106E-4,-2.0595364E-2,2.191925E-2,-1.8502772E-3,-3.2833114E-4,3.6354903E-3,-9.0166024E-4,-2.164962E-3,-7.4080243E-3,-6.167028E-4,-6.525985E-3,-1.8488677E-3,1.7764538E-3,4.87771E-3,-4.2148007E-4,5.511297E-3,1.1029495E-3,-2.0433825E-3,2.438782E-3,3.4935386E-3,6.219449E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,-1,-1,35,37,39,-1,-1,41,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7623878E-1,6.7848995E-2,1.6055426E-1,9.012736E-2,1.3778207E-1,1.13545E-1,6.850204E-2,1.0304718E-1,6.1318606E-2,6.1877616E-2,3.6937684E-2,3.0151486E-2,0E0,3.6380738E-2,6.79224E-2,3.8346313E-2,5.4628316E-2,1.8314958E-2,0E0,0E0,3.238152E-2,2.075532E-2,4.452826E-2,0E0,0E0,2.8860092E-2,0E0,8.592548E-2,8.507968E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,25,25,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,-1,-1,36,38,40,-1,-1,42,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.783E3,1.903E3,8.547769E7,1.121092E6,2.4143283E2,6.6E1,7.41E3,7.164074E8,3.5581547E1,1.30817086E5,3.295663E0,1.154375E6,-1.893335E-3,3.2914574E1,2.405576E3,2.23E2,6.173444E1,1.6E1,3.563951E-4,1.6019363E-3,6.453125E0,1.0763E4,3.0374873E3,4.7854437E-3,1.0758918E-2,1.37893E6,-4.6360106E-4,1E0,2.1207E4,-1.8502772E-3,-3.2833114E-4,3.6354903E-3,-9.0166024E-4,-2.164962E-3,-7.4080243E-3,-6.167028E-4,-6.525985E-3,-1.8488677E-3,1.7764538E-3,4.87771E-3,-4.2148007E-4,5.511297E-3,1.1029495E-3,-2.0433825E-3,2.438782E-3,3.4935386E-3,6.219449E-4],"split_indices":[2,0,7,38,61,8,2,7,67,37,62,9,0,62,4,3,65,8,0,0,56,12,42,0,0,38,0,108,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,2.22E2,2.7E2,1.77E2,4.5E1,1.5E1,2.55E2,1.63E2,1.4E1,1.3E1,3.2E1,1.2E1,3E0,2E1,2.35E2,1.4E2,2.3E1,1E1,4E0,4E0,9E0,1.3E1,1.9E1,8E0,4E0,1.6E1,4E0,4.6E1,1.89E2,7.6E1,6.4E1,1.5E1,8E0,3E0,7E0,3E0,6E0,4E0,9E0,1.5E1,4E0,1.1E1,5E0,3.5E1,1.1E1,2.4E1,1.65E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.0757364E-3,-4.8935898E-3,4.4235934E-2,-1.3850963E-2,1.5047932E-2,7.8699235E-3,3.4495603E-2,-1.10497745E-2,-1.2952214E-1,-2.2237826E-2,3.3937898E-2,9.1914706E-2,1.714129E-2,-5.4095904E-3,-5.5494573E-2,-2.4078465E-3,-7.964645E-3,3.1124935E-3,-3.245279E-2,7.033589E-2,5.9867525E-3,1.549898E-3,9.795282E-3,-7.963112E-3,7.32984E-2,-5.581233E-4,2.0021254E-3,-1.0015943E-2,-1.458012E-3,-2.6946473E-3,6.4885383E-4,2.267474E-3,7.2431318E-3,7.22151E-4,-3.2825016E-3,6.022885E-4,-5.705005E-3,2.2356878E-3,5.5917352E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,-1,-1,-1,29,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2723166E-1,7.7731095E-2,6.872177E-2,9.577383E-2,9.58388E-2,0E0,5.5223197E-2,7.3398784E-2,1.48111805E-2,4.461624E-2,9.114183E-2,8.491322E-2,6.402426E-2,8.7947726E-2,1.2044581E-1,0E0,0E0,0E0,5.2602343E-2,6.344983E-2,4.071233E-2,0E0,0E0,8.053167E-2,1.0762207E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,18,18,19,19,20,20,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,-1,-1,-1,30,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0297854E3,1.2472753E5,8.381319E3,2.8659653E2,7.127066E2,7.8699235E-3,1.3058374E7,1.0735683E5,9E0,1.30817086E5,7.221312E-2,3.701371E7,1.16987236E2,9.5430945E4,1.0785559E5,-2.4078465E-3,-7.964645E-3,3.1124935E-3,1E0,1.4991647E3,2.0946211E5,1.549898E-3,9.795282E-3,2.0531621E4,8.625407E2,-5.581233E-4,2.0021254E-3,-1.0015943E-2,-1.458012E-3,-2.6946473E-3,6.4885383E-4,2.267474E-3,7.2431318E-3,7.22151E-4,-3.2825016E-3,6.022885E-4,-5.705005E-3,2.2356878E-3,5.5917352E-3],"split_indices":[61,37,42,67,61,0,54,37,26,37,47,1,65,37,37,0,0,0,23,61,42,0,0,4,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.92E2,4.33E2,5.9E1,2.99E2,1.34E2,3E0,5.6E1,2.93E2,6E0,4.5E1,8.9E1,1.2E1,4.4E1,2.61E2,3.2E1,3E0,3E0,4E0,4.1E1,3.8E1,5.1E1,9E0,3E0,3.1E1,1.3E1,2.3E2,3.1E1,3E0,2.9E1,2.6E1,1.5E1,3.2E1,6E0,4.6E1,5E0,2.7E1,4E0,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.3092774E-3,-4.817329E-3,3.2994717E-2,6.169194E-3,-2.4458732E-2,8.743758E-2,1.6037457E-2,-3.106781E-2,1.9886443E-2,-3.1440876E-2,4.9146462E-2,-6.519504E-3,1.265093E-1,-2.9370364E-2,3.1927742E-2,-2.209198E-2,-6.1484976E-3,2.3118474E-2,-5.7142703E-3,-1.5764773E-2,-6.129801E-2,4.294334E-3,1.7460475E-4,-3.1871041E-3,2.0723348E-3,8.3635E-3,3.6248178E-3,-5.6888495E-2,1.0276979E-3,3.8735203E-3,2.0745598E-2,-3.074185E-3,-2.713441E-4,3.1581202E-3,7.148558E-4,-2.5285461E-3,2.5908386E-5,-4.653395E-3,-7.352014E-4,-3.5520021E-3,6.564303E-4,2.0247442E-3,1.6493117E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,-1,-1,-1,-1,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1450759E-1,8.064083E-2,9.276431E-2,1.2320726E-1,7.041076E-2,8.893032E-2,5.7939637E-2,5.729504E-2,8.5229956E-2,5.6602992E-2,2.395474E-2,2.9316017E-2,3.0754834E-2,3.1227998E-2,3.3024456E-2,4.4052728E-2,0E0,5.844689E-2,0E0,5.2913796E-2,7.50968E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.1787934E-2,0E0,0E0,1.9981667E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,27,27,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,-1,-1,-1,-1,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4689211E7,2.6548672E-2,2.1207E4,1.3063165E3,2.416182E9,2.1397605E0,3.7431133E0,1.7239808E7,1.1274355E7,8.884E3,4.70436E5,8.494E3,6E0,1.31805E5,2.822E3,2.722E3,-6.1484976E-3,1E0,-5.7142703E-3,6.510362E2,2.0241737E3,4.294334E-3,1.7460475E-4,-3.1871041E-3,2.0723348E-3,8.3635E-3,3.6248178E-3,1.3299019E0,1.0276979E-3,3.8735203E-3,3.2917362E7,-3.074185E-3,-2.713441E-4,3.1581202E-3,7.148558E-4,-2.5285461E-3,2.5908386E-5,-4.653395E-3,-7.352014E-4,-3.5520021E-3,6.564303E-4,2.0247442E-3,1.6493117E-4],"split_indices":[1,66,9,4,7,63,62,54,1,2,38,0,8,37,0,2,0,109,0,61,61,0,0,0,0,0,0,48,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.73E2,3.72E2,1.01E2,2.39E2,1.33E2,2.3E1,7.8E1,6.4E1,1.75E2,1.22E2,1.1E1,7E0,1.6E1,2E1,5.8E1,6E1,4E0,1.72E2,3E0,8.1E1,4.1E1,5E0,6E0,3E0,4E0,6E0,1E1,1.3E1,7E0,9E0,4.9E1,1.5E1,4.5E1,2.3E1,1.49E2,2.3E1,5.8E1,2.1E1,2E1,1E1,3E0,2E1,2.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-1.7405031E-3,-9.553759E-3,2.868698E-2,-9.884991E-4,-2.9082878E-2,7.581835E-3,2.286209E-2,-2.1269167E-2,1.6576566E-2,-2.0247838E-2,-1.0764717E-1,-6.658246E-3,4.216128E-2,-4.1993238E-2,1.1083256E-3,-3.2979004E-2,2.6833568E-2,-5.271302E-2,2.1942998E-3,6.741188E-4,-7.8104814E-3,6.8939306E-2,-4.5610372E-2,6.781657E-2,8.293086E-3,-5.7792687E-4,-4.0184394E-3,-3.0634538E-4,4.2539313E-3,-4.8021516E-3,-2.8116957E-4,2.3262922E-3,-6.2096666E-4,-1.6654199E-3,-9.65069E-3,1.6517424E-3,-1.3208111E-3,9.058227E-4,3.9637685E-3,-1.1303333E-3,-7.306618E-3,8.152858E-3,2.1583694E-3,-2.1847973E-3,1.5742002E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1866746E-1,6.637822E-2,7.760576E-2,9.902863E-2,8.180875E-2,0E0,5.6682855E-2,5.9870653E-2,7.6620385E-2,8.026035E-2,9.50771E-2,1.2093176E-1,5.1571503E-2,8.903577E-2,4.577317E-2,4.9561672E-2,1.2093711E-1,1.08487844E-1,7.052959E-2,0E0,0E0,1.0448791E-2,6.0448788E-2,6.9072664E-2,4.1141555E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6219076E9,1.7458724E7,1.6779835E9,7.4378266E4,2.11E2,7.581835E-3,4.2729566E4,1.0650363E-1,3.9610416E2,4.1286648E1,2.322371E-3,1.1797827E9,9.83854E-1,2.2198479E1,1E0,2.407E3,5.1359596E1,4.460804E7,2.1929544E1,6.741188E-4,-7.8104814E-3,1.56904E5,8.848604E1,3.7137297E3,6.407166E1,-5.7792687E-4,-4.0184394E-3,-3.0634538E-4,4.2539313E-3,-4.8021516E-3,-2.8116957E-4,2.3262922E-3,-6.2096666E-4,-1.6654199E-3,-9.65069E-3,1.6517424E-3,-1.3208111E-3,9.058227E-4,3.9637685E-3,-1.1303333E-3,-7.306618E-3,8.152858E-3,2.1583694E-3,-2.1847973E-3,1.5742002E-3],"split_indices":[7,54,7,37,8,0,52,47,61,65,47,5,43,67,108,11,67,54,62,0,0,38,65,41,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.97E2,3.96E2,1.01E2,2.76E2,1.2E2,3E0,9.8E1,1.28E2,1.48E2,1.09E2,1.1E1,3.9E1,5.9E1,6.6E1,6.2E1,2.5E1,1.23E2,4.4E1,6.5E1,4E0,7E0,1.3E1,2.6E1,3.3E1,2.6E1,4.1E1,2.5E1,5.8E1,4E0,6E0,1.9E1,7.7E1,4.6E1,4.1E1,3E0,3.1E1,3.4E1,4E0,9E0,2.3E1,3E0,4E0,2.9E1,8E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.1693717E-3,-6.488647E-3,3.0451797E-2,-5.619795E-2,-2.3851735E-3,3.923175E-2,-5.5124965E-2,-1.0571052E-1,-2.6156135E-2,1.21826995E-2,-2.0919515E-2,-3.996645E-3,4.4364065E-2,-8.311005E-4,-4.04789E-3,-2.2197156E-3,-6.0162437E-3,5.058239E-3,-3.3685088E-3,-1.11585595E-2,3.986396E-2,-1.7182207E-2,-6.1360365E-3,7.5068395E-3,3.8420003E-2,-1.353756E-3,1.3914607E-3,-7.5241685E-4,6.205627E-3,2.1621946E-3,-3.49283E-3,-1.590773E-3,7.34598E-4,5.5851373E-3,1.5094331E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,-1,-1,25,-1,27,29,31,-1,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1142328E-1,8.0234386E-2,7.955276E-2,4.185845E-2,9.880702E-2,6.49357E-2,1.0713825E-2,9.831637E-3,2.9997177E-2,1.328985E-1,6.578976E-2,0E0,5.8432683E-2,0E0,0E0,0E0,0E0,1.2314801E-2,0E0,8.907979E-2,8.426659E-2,9.228837E-2,0E0,0E0,3.597416E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,17,17,19,19,20,20,21,21,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,-1,-1,26,-1,28,30,32,-1,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6219076E9,9.687575E2,1.5953E4,2.2794174E2,1.8887723E-2,1.5153618E0,9.117005E5,7E1,4.875E0,5.2789326E10,9.163994E6,-3.996645E-3,1E0,-8.311005E-4,-4.04789E-3,-2.2197156E-3,-6.0162437E-3,2.09E2,-3.3685088E-3,8.588E3,4.7374536E7,1.9184847E3,-6.1360365E-3,7.5068395E-3,1.6779835E9,-1.353756E-3,1.3914607E-3,-7.5241685E-4,6.205627E-3,2.1621946E-3,-3.49283E-3,-1.590773E-3,7.34598E-4,5.5851373E-3,1.5094331E-3],"split_indices":[7,4,0,61,66,62,56,10,67,40,56,0,8,0,0,0,0,3,0,2,54,61,0,0,7,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.95E2,3.93E2,1.02E2,2.9E1,3.64E2,9.3E1,9E0,1E1,1.9E1,2.04E2,1.6E2,3E0,9E1,5E0,4E0,4E0,6E0,1.2E1,7E0,1.11E2,9.3E1,1.56E2,4E0,3E0,8.7E1,5E0,7E0,1.08E2,3E0,8.8E1,5E0,1.02E2,5.4E1,4E0,8.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-4.398496E-3,4.001885E-3,-2.1535086E-2,-1.2958193E-2,1.7957984E-2,-5.2552693E-2,1.091487E-2,-3.481335E-2,1.7775325E-2,5.5304743E-2,9.004642E-3,-4.0511575E-2,-1.3640171E-1,-2.660796E-2,3.0532312E-2,-4.2321984E-2,2.1594562E-2,5.804965E-2,-8.497326E-3,1.2104114E-2,8.352878E-2,-2.4840722E-3,2.6874786E-2,-2.6936684E-2,-1.246972E-1,-7.4276053E-3,-2.6065842E-3,-4.4955365E-2,7.834606E-4,3.80325E-2,-3.242349E-3,-9.475932E-4,-3.9957673E-3,-7.4838963E-4,2.560092E-3,3.3808635E-3,-2.0103011E-4,-2.787155E-3,7.802898E-4,-1.1504939E-3,1.8180008E-3,6.401013E-3,8.9631305E-4,7.167071E-4,-1.4627639E-3,3.107124E-3,3.073734E-4,-5.493178E-3,-4.4984743E-4,-2.053196E-3,-7.7995714E-3,-1.2135092E-3,-5.1287278E-3,3.718245E-3,1.0872415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.091149E-2,7.8579254E-2,1.6362984E-1,1.0137168E-1,6.035885E-2,7.827395E-2,5.970055E-2,3.8186006E-2,6.71544E-2,4.1664325E-2,3.0451896E-2,8.120907E-2,1.1337474E-2,2.3234159E-2,4.2996414E-2,7.405162E-2,1.5253641E-2,2.5036715E-2,5.3258143E-2,1.6431449E-2,7.292448E-2,4.9181454E-2,4.7506772E-2,1.0234228E-1,2.886875E-2,0E0,0E0,2.1778055E-2,0E0,2.8914973E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7733024E7,7.959718E4,5.9485336E1,9.703059E6,1.0545219E5,1.62E2,1.1004841E9,2.0241737E3,1.6259234E7,7.616E3,1.17264E5,4.352718E7,6.6E2,6.4065235E8,6.941001E7,6.904204E6,1.04E2,1.7E2,2.0532622E3,6.471207E6,3.311258E-2,6.624E3,4.8837247E-1,1.9281628E7,5.974836E4,-7.4276053E-3,-2.6065842E-3,1.215705E8,7.834606E-4,1.7E1,-3.242349E-3,-9.475932E-4,-3.9957673E-3,-7.4838963E-4,2.560092E-3,3.3808635E-3,-2.0103011E-4,-2.787155E-3,7.802898E-4,-1.1504939E-3,1.8180008E-3,6.401013E-3,8.9631305E-4,7.167071E-4,-1.4627639E-3,3.107124E-3,3.073734E-4,-5.493178E-3,-4.4984743E-4,-2.053196E-3,-7.7995714E-3,-1.2135092E-3,-5.1287278E-3,3.718245E-3,1.0872415E-3],"split_indices":[54,37,65,9,37,8,5,61,9,11,11,54,3,7,54,54,8,3,4,54,66,10,48,54,37,0,0,7,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.91E2,3.3E2,1.61E2,1.49E2,1.81E2,8.2E1,7.9E1,8.7E1,6.2E1,3.4E1,1.47E2,7.3E1,9E0,2.7E1,5.2E1,7.7E1,1E1,2.4E1,3.8E1,1.4E1,2E1,9E1,5.7E1,6.4E1,9E0,6E0,3E0,1.9E1,8E0,4.9E1,3E0,5.3E1,2.4E1,5E0,5E0,1.9E1,5E0,1.2E1,2.6E1,6E0,8E0,1E1,1E1,5.6E1,3.4E1,1.8E1,3.9E1,9E0,5.5E1,4E0,5E0,1.6E1,3E0,1.1E1,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[5.704205E-4,-1.2803448E-2,1.2683336E-2,-1.5352627E-3,-4.7174133E-2,-8.158916E-3,2.6631495E-2,-5.072042E-3,5.4790173E-3,-6.4119406E-2,2.1713912E-2,-1.6224574E-2,6.077429E-2,2.253588E-2,6.5601305E-3,-1.992053E-3,-6.093775E-3,-4.7966596E-2,-7.4962205E-3,-6.103866E-4,2.6994436E-3,-3.699919E-2,4.0224413E-3,4.301602E-3,-9.5510413E-4,3.8989976E-2,-3.9522685E-3,-8.0784404E-4,8.525785E-4,-3.912945E-3,4.6044224E-4,-5.0712604E-4,-2.669109E-3,-1.196023E-3,1.0519983E-3,2.112755E-3,-3.2765593E-3,3.7189587E-3,-9.730527E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,-1,29,-1,-1,-1,31,33,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.873071E-2,8.9392774E-2,7.43486E-2,7.566253E-2,6.7929894E-2,5.825319E-2,6.991145E-2,6.719773E-2,0E0,6.621587E-2,1.6465891E-2,3.926685E-2,3.1802244E-2,6.509404E-2,0E0,5.458977E-2,0E0,9.01756E-2,0E0,0E0,0E0,2.4779841E-2,2.8056428E-2,0E0,0E0,7.634506E-2,8.729388E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,17,17,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,-1,30,-1,-1,-1,32,34,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.523215E2,4.592E3,7.959718E4,9.508E3,2.4851996E7,9.813996E8,1.5325492E4,4.086885E7,5.4790173E-3,2.075841E7,3.0379269E1,9.4E1,4.199425E7,1.0530315E7,6.5601305E-3,1.592757E6,-6.093775E-3,1.3421E4,-7.4962205E-3,-6.103866E-4,2.6994436E-3,6.950835E1,3.0066213E3,4.301602E-3,-9.5510413E-4,3.271774E7,9.222873E4,-8.0784404E-4,8.525785E-4,-3.912945E-3,4.6044224E-4,-5.0712604E-4,-2.669109E-3,-1.196023E-3,1.0519983E-3,2.112755E-3,-3.2765593E-3,3.7189587E-3,-9.730527E-4],"split_indices":[61,10,37,2,54,41,61,54,0,54,65,8,1,57,0,9,0,10,0,0,0,65,64,0,0,54,37,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.84E2,2.3E2,2.54E2,1.74E2,5.6E1,1.02E2,1.52E2,1.7E2,4E0,4.5E1,1.1E1,9.2E1,1E1,1.48E2,4E0,1.67E2,3E0,4E1,5E0,6E0,5E0,4.5E1,4.7E1,7E0,3E0,9.1E1,5.7E1,9.5E1,7.2E1,2.4E1,1.6E1,2.1E1,2.4E1,1.8E1,2.9E1,8.6E1,5E0,9E0,4.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-4.6168617E-4,-1.2844783E-2,1.0498204E-2,-1.7419197E-2,6.2389508E-2,6.4786253E-3,7.972635E-3,1.5597872E-3,-3.3892646E-2,4.0804846E-3,-9.223238E-4,9.635846E-3,-4.720101E-3,-4.026696E-3,6.189045E-3,-1.9418633E-2,-6.695231E-2,-6.633311E-2,1.28688365E-2,-8.410421E-4,1.6163178E-3,2.1751053E-3,-1.6291057E-3,6.1083084E-4,-3.8730889E-3,1.2990453E-3,-5.078751E-3,1.6753734E-3,9.936339E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.446489E-2,7.783282E-2,1.6471185E-1,6.6021875E-2,3.0532982E-2,8.648247E-2,0E0,7.4481025E-2,5.2771077E-2,0E0,0E0,6.0044885E-2,0E0,5.48616E-2,0E0,8.83586E-2,4.941073E-2,4.544291E-2,5.8844127E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,13,13,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.361085E4,9.813996E8,3.305542E4,7.477264E6,4.199425E7,3.9608764E7,7.972635E-3,1.5704E4,6.465659E4,4.0804846E-3,-9.223238E-4,9.284744E2,-4.720101E-3,7.164074E8,6.189045E-3,6.798766E-4,1.1254387E7,3.251E3,6.079182E-3,-8.410421E-4,1.6163178E-3,2.1751053E-3,-1.6291057E-3,6.1083084E-4,-3.8730889E-3,1.2990453E-3,-5.078751E-3,1.6753734E-3,9.936339E-5],"split_indices":[37,41,4,54,1,54,0,2,37,0,0,4,0,7,0,48,54,2,48,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.73E2,2.22E2,2.51E2,2.1E2,1.2E1,2.46E2,5E0,9.8E1,1.12E2,9E0,3E0,2.4E2,6E0,9.5E1,3E0,7.9E1,3.3E1,9E0,2.31E2,7E1,2.5E1,1.5E1,6.4E1,6E0,2.7E1,3E0,6E0,7.1E1,1.6E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[2.862115E-3,-1.6819194E-2,1.0110916E-2,-2.2581583E-2,4.086809E-3,4.9875923E-3,8.685745E-3,-2.8509228E-2,4.9030006E-2,1.0717471E-2,-5.5641476E-2,-1.1774542E-2,-4.9935006E-2,6.9699757E-4,4.38948E-3,4.78473E-2,6.490821E-3,5.6917977E-4,-8.140525E-3,-1.2944279E-3,1.4382062E-3,-3.1820976E-3,-1.2405166E-4,-2.6210886E-3,2.7224761E-3,1.8589571E-4,3.3009686E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,-1,-1,9,11,13,15,17,19,21,-1,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.993367E-2,8.273967E-2,4.9502965E-2,5.4883286E-2,0E0,0E0,4.6862468E-2,4.136283E-2,1.438757E-2,5.3675E-2,9.3761235E-2,4.868384E-2,4.702024E-2,0E0,0E0,4.6254784E-2,4.9126618E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16],"right_children":[2,4,6,8,-1,-1,10,12,14,16,18,20,22,-1,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.759E3,7.164074E8,1.3E1,2.9063344E5,4.086809E-3,4.9875923E-3,4.7374536E7,8.09994E5,4.204E3,9.345519E0,2.003E3,5.98405E5,2.72E2,6.9699757E-4,4.38948E-3,2.58E2,1.6941456E1,5.6917977E-4,-8.140525E-3,-1.2944279E-3,1.4382062E-3,-3.1820976E-3,-1.2405166E-4,-2.6210886E-3,2.7224761E-3,1.8589571E-4,3.3009686E-3],"split_indices":[2,7,3,37,0,0,54,1,2,67,0,1,3,0,0,12,44,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.88E2,1.31E2,3.57E2,1.25E2,6E0,4E0,3.53E2,1.16E2,9E0,3.43E2,1E1,6.6E1,5E1,6E0,3E0,3.4E1,3.09E2,7E0,3E0,4.8E1,1.8E1,3.5E1,1.5E1,3E0,3.1E1,2.99E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[-3.6888302E-3,-9.067473E-3,3.671459E-2,3.1927289E-3,-2.4746437E-2,-4.4986524E-2,4.881056E-2,1.95282E-2,-1.703288E-2,-4.7914874E-2,-1.1548317E-2,-3.825777E-3,-2.332253E-4,9.433194E-2,2.7222985E-2,-1.9004468E-2,3.2558743E-2,-5.9887063E-2,-1.0672144E-2,-2.8283678E-2,-1.2301718E-1,-4.078392E-2,7.706253E-3,5.6184832E-2,8.911017E-3,-3.5712277E-3,3.9157182E-2,-1.9921E-3,9.562569E-4,-2.0884324E-4,2.07385E-3,-1.1933182E-3,-6.310217E-3,-9.7289373E-4,1.2076196E-3,8.087182E-4,-2.7845562E-3,-8.308559E-3,-2.5815659E-3,3.8485063E-4,-2.314979E-3,2.156751E-3,-1.7708006E-3,4.0967525E-3,1.1054837E-3,3.294376E-3,1.8400725E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0725218E-1,8.369786E-2,5.9514537E-2,8.127937E-2,5.785062E-2,1.1897009E-2,4.7549516E-2,6.8904564E-2,2.9345132E-2,9.825066E-2,6.9553316E-2,0E0,0E0,5.033584E-2,4.828001E-2,3.5163417E-2,4.8521847E-2,3.3327896E-2,3.8624242E-2,8.488414E-2,4.434769E-2,2.4057753E-2,1.3902634E-1,1.1718225E-2,0E0,0E0,3.7779987E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,1.3058374E7,5.0314346E1,1.33E2,3.8499493E1,1.9078612E-1,4.0601485E6,2.6E1,1.8E1,9E1,1.1639729E9,-3.825777E-3,-2.332253E-4,1.92144E5,2.9457372E9,1.6485538E3,4.6E1,1.6E1,1.9844815E9,1E0,2.1694159E1,1.4533922E2,9.304E3,3.902439E-2,8.911017E-3,-3.5712277E-3,9.11557E4,-1.9921E-3,9.562569E-4,-2.0884324E-4,2.07385E-3,-1.1933182E-3,-6.310217E-3,-9.7289373E-4,1.2076196E-3,8.087182E-4,-2.7845562E-3,-8.308559E-3,-2.5815659E-3,3.8485063E-4,-2.314979E-3,2.156751E-3,-1.7708006E-3,4.0967525E-3,1.1054837E-3,3.294376E-3,1.8400725E-4],"split_indices":[61,54,65,3,65,47,56,8,8,8,5,0,0,2,12,4,3,8,5,22,67,42,2,66,0,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.91E2,4.34E2,5.7E1,2.44E2,1.9E2,7E0,5E1,1.35E2,1.09E2,6.8E1,1.22E2,3E0,4E0,1.5E1,3.5E1,3.4E1,1.01E2,1.3E1,9.6E1,5.5E1,1.3E1,4.8E1,7.4E1,1.2E1,3E0,3E0,3.2E1,2.1E1,1.3E1,2.6E1,7.5E1,1E1,3E0,7.5E1,2.1E1,2.3E1,3.2E1,6E0,7E0,8E0,4E1,4E1,3.4E1,5E0,7E0,1.6E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.500262E-3,-1.4297055E-2,1.0854232E-2,-3.39321E-2,-3.6640547E-3,5.0885346E-2,9.39058E-4,-2.4652699E-2,-1.2248462E-1,3.1990744E-2,-2.0334193E-2,8.511075E-3,3.5430126E-2,1.7563587E-2,-1.8466007E-2,-6.976678E-2,-7.4126553E-3,-7.3764124E-4,-8.37803E-3,5.2187108E-2,-5.8052696E-2,-3.4612298E-2,2.3226336E-2,1.13701336E-1,1.6459074E-2,1.2581246E-2,6.8157963E-3,-5.3183425E-2,4.2073475E-3,-3.907025E-3,9.988577E-4,-1.8498721E-3,1.0849232E-3,4.2290688E-3,8.935504E-4,-4.3918886E-3,-4.289946E-5,-4.7575854E-4,-2.6807406E-3,4.210671E-4,4.808447E-3,8.538509E-3,2.1123895E-3,-1.6351427E-3,1.6331414E-3,3.0248608E-3,2.9779738E-4,2.1090551E-5,-3.866631E-3,8.509023E-4,-2.4569794E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,-1,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.983653E-2,5.165298E-2,1.0195949E-1,6.8469584E-2,9.693852E-2,9.990005E-2,6.710254E-2,6.170434E-2,4.8610896E-2,9.75942E-2,6.984167E-2,0E0,6.7918286E-2,7.143788E-2,7.5791165E-2,3.332098E-2,6.157169E-2,0E0,0E0,5.4692656E-2,2.1568462E-2,4.8145615E-2,3.1175584E-2,3.5691574E-2,4.041469E-2,3.5001118E-2,0E0,6.386736E-2,5.014394E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,-1,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.979E3,1.65602E5,7.801943E3,6.08E3,3.6319303E-3,2.9E1,1.423795E7,3.9610416E2,1E0,3.271774E7,8.6419773E-1,8.511075E-3,1.3474197E8,2.5212732E4,5.2610065E1,2.889654E5,4.59E2,-7.3764124E-4,-8.37803E-3,2.5212479E1,5.441669E9,1E0,2.4339888E3,2.7E1,2.9648642E1,7.37E3,6.8157963E-3,1.0401872E9,1.93E2,-3.907025E-3,9.988577E-4,-1.8498721E-3,1.0849232E-3,4.2290688E-3,8.935504E-4,-4.3918886E-3,-4.289946E-5,-4.7575854E-4,-2.6807406E-3,4.210671E-4,4.808447E-3,8.538509E-3,2.1123895E-3,-1.6351427E-3,1.6331414E-3,3.0248608E-3,2.9779738E-4,2.1090551E-5,-3.866631E-3,8.509023E-4,-2.4569794E-3],"split_indices":[2,9,41,2,51,3,54,61,13,54,50,0,7,4,65,37,0,0,0,62,5,23,4,8,65,2,0,7,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[5.03E2,2.47E2,2.56E2,8.6E1,1.61E2,5E1,2.06E2,7.9E1,7E0,5.1E1,1.1E2,4E0,4.6E1,1.11E2,9.5E1,2.1E1,5.8E1,3E0,4E0,4.2E1,9E0,8.3E1,2.7E1,8E0,3.8E1,1.08E2,3E0,3.7E1,5.8E1,1.8E1,3E0,2.8E1,3E1,1.8E1,2.4E1,5E0,4E0,4.2E1,4.1E1,2.4E1,3E0,3E0,5E0,1E1,2.8E1,1E1,9.8E1,1.4E1,2.3E1,4.7E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[1.9829502E-4,1.4515276E-2,-1.1119165E-2,1.5100687E-3,3.8281247E-2,-3.0474305E-2,-8.9289556E-4,-1.04678925E-2,3.7743036E-2,4.623353E-2,-1.8827181E-2,-1.6222434E-2,-7.136633E-2,2.4483887E-2,-1.2346177E-2,1.1142615E-2,-5.6559045E-2,1.5208059E-2,9.255967E-2,8.586647E-2,2.6273571E-2,2.2327402E-3,-3.296553E-3,-3.4020744E-2,4.1469266E-3,-2.0390416E-3,-5.464544E-3,1.5800357E-3,5.980893E-2,3.4213671E-3,-5.0984103E-2,-8.598174E-4,2.106343E-3,-8.311551E-3,-1.766247E-3,-1.4684031E-3,1.5706203E-3,7.636582E-3,1.6872053E-3,7.4836877E-3,2.128791E-3,1.5138888E-3,-2.4476235E-3,-2.0749986E-3,1.0342747E-3,-2.971093E-3,8.371977E-4,7.34402E-4,-4.0471572E-3,1.565774E-3,4.2546764E-3,-1.3884922E-3,8.162502E-4,-3.1192505E-3,8.0827525E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,45,-1,-1,47,49,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.890888E-2,6.639469E-2,5.380999E-2,6.1154682E-2,3.538529E-2,5.3624332E-2,5.2328177E-2,1.06211856E-1,4.1827552E-2,5.0745785E-2,3.949451E-2,2.588144E-2,2.5592707E-2,4.5213956E-2,7.5718366E-2,7.709123E-2,6.937131E-2,2.4605524E-2,3.5235107E-2,5.962673E-2,2.674964E-2,0E0,0E0,2.538773E-2,3.441348E-2,0E0,0E0,4.7004987E-2,1.5492745E-2,4.3859225E-2,4.3658152E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,27,27,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,46,-1,-1,48,50,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.26E2,4.98884E1,1.204977E6,1.646359E5,5.41E2,3.48927E5,9.17E2,9.409E3,2.724043E6,6.1453987E1,5.3930223E4,1E0,5.36E2,2.9E2,2.748728E6,1.5802073E3,2.8884522E8,1.3350947E3,4.7431055E-2,2.7E1,5.30113E5,2.2327402E-3,-3.296553E-3,6.6521835E-1,8.6E1,-2.0390416E-3,-5.464544E-3,9.9921934E-2,7.177645E10,4.5808883E0,7.567335E7,-8.598174E-4,2.106343E-3,-8.311551E-3,-1.766247E-3,-1.4684031E-3,1.5706203E-3,7.636582E-3,1.6872053E-3,7.4836877E-3,2.128791E-3,1.5138888E-3,-2.4476235E-3,-2.0749986E-3,1.0342747E-3,-2.971093E-3,8.371977E-4,7.34402E-4,-4.0471572E-3,1.565774E-3,4.2546764E-3,-1.3884922E-3,8.162502E-4,-3.1192505E-3,8.0827525E-4],"split_indices":[3,65,1,37,8,38,0,2,38,65,37,22,0,3,9,4,12,4,47,8,2,0,0,48,11,0,0,66,40,67,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.85E2,2.14E2,2.71E2,1.39E2,7.5E1,9.3E1,1.78E2,1.05E2,3.4E1,6.6E1,9E0,7E1,2.3E1,5.5E1,1.23E2,7.2E1,3.3E1,2.5E1,9E0,2.1E1,4.5E1,4E0,5E0,3.7E1,3.3E1,1.6E1,7E0,3.4E1,2.1E1,8.8E1,3.5E1,3.9E1,3.3E1,3E0,3E1,7E0,1.8E1,3E0,6E0,6E0,1.5E1,4.2E1,3E0,3.1E1,6E0,5E0,2.8E1,3E1,4E0,1.3E1,8E0,2.6E1,6.2E1,2.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[2.820667E-4,-4.5020074E-2,3.0826638E-3,-2.6090592E-2,-5.181797E-3,-3.6197403E-3,2.7539143E-2,-1.1690951E-2,-4.354949E-3,1.1891802E-2,-1.7223177E-2,4.2004008E-2,-2.2005592E-3,7.8282127E-4,-5.19883E-2,-1.323946E-2,2.5697572E-2,-8.7326914E-2,-8.808478E-3,6.582781E-2,1.766077E-2,1.5413876E-3,-2.601815E-2,-3.984756E-3,-2.975644E-4,-1.820763E-3,9.344099E-4,3.4210929E-3,6.507037E-4,-7.8030745E-3,-2.641482E-3,-2.3710371E-3,6.436363E-5,1.4234167E-3,4.7110096E-3,-2.4051294E-3,1.5559439E-3,5.6070153E-4,-2.5508567E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.319469E-2,3.4492377E-2,7.697686E-2,2.2759305E-2,0E0,7.80676E-2,4.3540694E-2,2.5259035E-2,0E0,6.0363688E-2,1.1544144E-1,3.827393E-2,2.9934818E-2,0E0,1.3797905E-2,5.6214795E-2,6.2008046E-2,4.187496E-2,7.782695E-2,4.0619567E-2,4.1945845E-2,0E0,2.4571933E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.687575E2,1.0361716E7,1.5271514E5,2.7577666E5,-5.181797E-3,8.227176E-3,9.3E1,3.834424E1,-4.354949E-3,1.86005E5,2.9648642E1,8.387783E-2,1E0,7.8282127E-4,1.18E2,9.5430945E4,2.7E1,6.979E3,2.3808822E-2,2.629627E6,2.0466278E6,1.5413876E-3,6.7728635E6,-3.984756E-3,-2.975644E-4,-1.820763E-3,9.344099E-4,3.4210929E-3,6.507037E-4,-7.8030745E-3,-2.641482E-3,-2.3710371E-3,6.436363E-5,1.4234167E-3,4.7110096E-3,-2.4051294E-3,1.5559439E-3,5.6070153E-4,-2.5508567E-3],"split_indices":[4,54,37,37,0,66,8,62,0,9,65,47,13,0,3,37,8,2,47,1,59,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.96E2,2.8E1,4.68E2,2.3E1,5E0,3.68E2,1E2,2E1,3E0,1.72E2,1.96E2,6.7E1,3.3E1,1.2E1,8E0,6.1E1,1.11E2,2E1,1.76E2,3.3E1,3.4E1,1.3E1,2E1,4E0,4E0,3.4E1,2.7E1,2E1,9.1E1,4E0,1.6E1,3.3E1,1.43E2,1.8E1,1.5E1,6E0,2.8E1,9E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_Project","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_dataproc","platform_databricks-aws","platform_databricks-azure","platform_emr","platform_onprem","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan parquet","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"78"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-1.1543759E-2,-2.9164094E-1,5.225375E-1,-6.1542785E-1,-1.7896846E-1,3.337165E-1,1.095846E0,-6.6168624E-1,-2.809605E-1,-2.2536336E-1,2.6998156E-1,4.3044218E-1,-6.378941E-1,-9.0299025E-3,1.1820724E0,-3.2120116E-2,-1.0844816E-2,-1.7497713E-2,-2.9050882E-3,-3.680241E-1,-1.3123357E-1,3.896498E-1,9.171716E-3,3.1121385E-1,6.9928056E-1,-5.22249E-2,3.7201483E-2,7.2044194E-1,1.3313869E0,-2.1037541E-1,-4.2572212E-1,-3.827662E-3,-2.3734139E-1,8.229552E-3,2.404999E-2,-4.406951E-3,8.366387E-3,3.7350854E-1,1.1396172E-1,7.377814E-1,5.712394E-3,-6.837401E-3,1.8113934E-2,3.944785E-2,-5.579441E-3,3.6469277E-2,6.9043785E-2,-1.1405463E-2,-1.3925537E-3,-4.3448472E-1,2.961201E-4,1.0293514E-1,-1.2822478E-1,-2.640787E-1,2.8239745E-3,4.2787313E-1,1.7750145E-1,-1.6634671E-1,1.2790778E-2,8.275606E-3,3.593263E-2,-5.811612E-3,-2.1239117E-2,1.7118621E-1,-4.2523644E-3,8.528703E-4,-1.676108E-1,-2.791858E-1,-2.598382E-4,6.5293033E-3,-6.29342E-3,-5.4401555E-3,4.4468918E-1,-6.696587E-4,1.2609747E-2,-1.6164575E-2,-1.4170322E-3,-7.644391E-4,1.1010883E-2,-9.549201E-3,3.4443554E-3,-1.4046588E-2,-5.5781608E-3,1.2920933E-2,2.2951575E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,-1,-1,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,-1,-1,-1,-1,55,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1,61,-1,63,65,67,69,71,73,75,-1,-1,-1,-1,-1,77,-1,-1,79,81,-1,-1,-1,-1,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.043904E1,9.557554E0,1.4681416E1,8.661823E-1,4.1849837E0,1.0236423E1,3.9909554E0,1.3570976E-1,1.6616076E-1,2.3688545E0,5.894288E-1,2.891529E0,3.1209216E0,0E0,1.2465096E0,0E0,0E0,0E0,0E0,5.562563E-1,1.4861554E0,2.3568249E-1,1.342075E-1,8.0528975E-1,5.5884075E-1,0E0,3.632071E-1,9.843998E-1,3.9718246E-1,1.0177803E-1,1.9571972E-1,6.9052905E-1,3.8602877E-1,0E0,0E0,0E0,0E0,4.781561E-1,8.2167524E-1,9.865475E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.0990238E-1,0E0,3.8221842E-1,1.4109164E-1,2.094698E-1,1.4433593E-1,4.1515398E-1,2.1954155E-1,1.5478455E-1,0E0,0E0,0E0,0E0,0E0,2.3760957E-1,0E0,0E0,1.6228485E-1,9.063101E-2,0E0,0E0,0E0,0E0,1.1523485E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,37,37,38,38,39,39,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,63,63,66,66,67,67,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,-1,-1,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,-1,-1,-1,-1,56,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1,62,-1,64,66,68,70,72,74,76,-1,-1,-1,-1,-1,78,-1,-1,80,82,-1,-1,-1,-1,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,6.0500806E3,3.5002026E8,1.1576422E9,9.064268E2,5.0314346E1,2.9778808E7,3.49399E5,6.631343E2,1.24E2,2.7371603E-1,3.5226266E0,-9.0299025E-3,6.069097E4,-3.2120116E-2,-1.0844816E-2,-1.7497713E-2,-2.9050882E-3,7.2123305E6,1.69E2,2.4237612E3,1.4777102E9,1.7458724E7,1.5631589E9,-5.22249E-2,2.5212732E4,2.5212732E4,7.710634E3,6.1614815E6,7.519047E8,1.819258E1,2.029E3,8.229552E-3,2.404999E-2,-4.406951E-3,8.366387E-3,2.0014305E5,3.672717E3,7.532E3,5.712394E-3,-6.837401E-3,1.8113934E-2,3.944785E-2,-5.579441E-3,3.6469277E-2,6.9043785E-2,-1.1405463E-2,-1.3925537E-3,5.6800385E0,2.961201E-4,1.8071064E7,4.2393894E1,3.4015296E8,6.874457E6,2.3600652E4,2.0241737E3,2.9673992E9,1.2790778E-2,8.275606E-3,3.593263E-2,-5.811612E-3,-2.1239117E-2,9.0417197E8,-4.2523644E-3,8.528703E-4,5.1526318E1,6.4065235E8,-2.598382E-4,6.5293033E-3,-6.29342E-3,-5.4401555E-3,1.227E4,-6.696587E-4,1.2609747E-2,-1.6164575E-2,-1.4170322E-3,-7.644391E-4,1.1010883E-2,-9.549201E-3,3.4443554E-3,-1.4046588E-2,-5.5781608E-3,1.2920933E-2,2.2951575E-2],"split_indices":[2,1,63,7,7,69,67,12,1,63,3,46,65,0,39,0,0,0,0,56,3,4,7,56,43,0,4,4,63,56,7,64,0,0,0,0,0,44,4,9,0,0,0,0,0,0,0,0,0,67,0,56,67,43,1,39,63,5,0,0,0,0,0,5,0,0,64,7,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.02E2,2.64E2,1.38E2,6.7E1,1.97E2,1.05E2,3.3E1,5.8E1,9E0,1.79E2,1.8E1,9.6E1,9E0,2E0,3.1E1,5.6E1,2E0,6E0,3E0,7E1,1.09E2,1.2E1,6E0,6.8E1,2.8E1,5E0,4E0,9E0,2.2E1,2E1,5E1,5E1,5.9E1,5E0,7E0,4E0,2E0,5.1E1,1.7E1,2.6E1,2E0,3E0,1E0,8E0,1E0,5E0,1.7E1,1.7E1,3E0,4.9E1,1E0,2.7E1,2.3E1,5.3E1,6E0,3.9E1,1.2E1,6E0,1.1E1,1E0,2.5E1,2E0,4.7E1,2E1,7E0,5E0,1.8E1,5E1,3E0,3E0,3E0,1E0,3.8E1,4E0,8E0,2E0,4E0,5E0,1.5E1,1.6E1,2E0,4.5E1,5E0,8E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.7457087E-2,-2.9294568E-1,5.564628E-1,-6.0236806E-1,-1.888511E-1,4.074421E-1,1.081494E0,-6.289257E-1,-1.5986547E-3,-2.2155657E-1,2.2710636E-1,4.708386E-1,-3.3700344E-1,1.1549801E-1,1.1742436E0,-3.106927E-2,-1.717879E-2,-2.608465E-1,5.2881986E-2,3.6670464E-1,-2.071917E-3,3.073939E-1,7.037266E-1,-4.1547954E-2,1.5975973E-2,1.2821938E-2,-8.207311E-3,1.3566816E0,7.357889E-1,-2.9467E-1,-8.845308E-2,-8.986041E-3,1.10524766E-1,1.9414436E-2,1.5034243E-4,1.5839276E-1,2.0234214E-2,4.9834764E-1,3.8177602E-2,-7.830999E-3,1.4830934E-2,6.869601E-2,4.0908936E-1,3.7712164E-2,4.597531E-3,-3.047954E-1,1.2049495E-2,-1.2618057E-1,3.3846379E-3,1.0225651E-2,2.9562908E-4,-7.591756E-3,3.3594903E-1,2.5851874E-2,5.5306233E-3,-5.4462864E-3,3.4735452E-2,-3.653569E-1,-2.201839E-1,2.765258E-3,-7.479138E-3,-1.6413365E-2,8.9031324E-2,1.7138885E-2,2.9084837E-4,-1.1952586E-2,-1.931612E-2,-3.3202434E-3,-1.5855338E-2,-2.4508378E-3,8.244264E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,-1,35,37,-1,39,-1,-1,41,43,45,47,-1,49,-1,-1,51,-1,53,-1,-1,-1,-1,55,-1,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,-1,-1,65,67,-1,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.95962E1,8.389469E0,1.1451916E1,9.7606087E-1,2.7575297E0,5.8221054E0,2.8210373E0,1.3863182E-1,0E0,2.0085878E0,5.8236927E-1,4.008816E0,1.8969216E0,2.2226259E-1,1.4014969E0,0E0,0E0,9.185486E-1,3.5441795E-1,1.4430428E-1,0E0,1.09619E0,4.8981667E-1,0E0,4.2244533E-1,0E0,0E0,9.7590256E-1,3.0595064E-1,8.248186E-1,1.7729518E-1,0E0,2.1508013E-1,0E0,0E0,9.1616976E-1,0E0,2.1860027E-1,0E0,0E0,0E0,0E0,5.831843E-1,0E0,0E0,6.0107327E-1,0E0,1.3829413E-1,0E0,0E0,0E0,5.829912E-1,1.1223483E-1,0E0,0E0,0E0,0E0,2.4224854E-1,9.553282E-1,0E0,0E0,0E0,1.7335142E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,21,21,22,22,24,24,27,27,28,28,29,29,30,30,32,32,35,35,37,37,42,42,45,45,47,47,51,51,52,52,57,57,58,58,62,62],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,-1,36,38,-1,40,-1,-1,42,44,46,48,-1,50,-1,-1,52,-1,54,-1,-1,-1,-1,56,-1,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,-1,-1,66,68,-1,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,6.0500806E3,1.2852292E7,1.2538845E9,8.625407E2,5.3140095E1,2.9778808E7,-1.5986547E-3,5.9170656E8,1.5E2,4.1331047E-1,3.5226266E0,1.0014992E8,5.192243E6,-3.106927E-2,-1.717879E-2,7.10711E8,4E3,1.436E3,-2.071917E-3,1.3428E4,1E0,-4.1547954E-2,3.0972284E1,1.2821938E-2,-8.207311E-3,1.92E2,2.617838E8,3.7069206E5,1.4555919E-1,-8.986041E-3,7.753E3,1.9414436E-2,1.5034243E-4,2.58E2,2.0234214E-2,1.5631589E9,3.8177602E-2,-7.830999E-3,1.4830934E-2,6.869601E-2,1.379E3,3.7712164E-2,4.597531E-3,7.898574E2,1.2049495E-2,4.45E3,3.3846379E-3,1.0225651E-2,2.9562908E-4,2.3808822E-2,4.2663252E7,2.5851874E-2,5.5306233E-3,-5.4462864E-3,3.4735452E-2,7.37365E6,2.5212479E1,2.765258E-3,-7.479138E-3,-1.6413365E-2,8.42E2,1.7138885E-2,2.9084837E-4,-1.1952586E-2,-1.931612E-2,-3.3202434E-3,-1.5855338E-2,-2.4508378E-3,8.244264E-3],"split_indices":[2,1,63,9,7,69,67,12,0,12,3,46,65,1,40,0,0,7,2,0,0,2,27,0,67,0,0,8,1,39,49,0,2,0,0,8,0,43,0,0,0,0,0,0,0,63,0,2,0,0,0,49,56,0,0,0,0,56,64,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.14E2,2.63E2,1.51E2,6.5E1,1.98E2,1.19E2,3.2E1,6.2E1,3E0,1.84E2,1.4E1,1.1E2,9E0,3E0,2.9E1,5.6E1,6E0,1.61E2,2.3E1,9E0,5E0,6.6E1,4.4E1,3E0,6E0,2E0,1E0,1.9E1,1E1,1.34E2,2.7E1,4E0,1.9E1,8E0,1E0,3E1,3.6E1,1.6E1,2.8E1,4E0,2E0,1.7E1,2E0,9E0,1E0,1.32E2,2E0,2.2E1,5E0,9E0,1E1,1.6E1,1.4E1,1.4E1,2E0,1E0,1E0,7.5E1,5.7E1,3E0,1.9E1,3E0,1.3E1,1.3E1,1E0,2.1E1,5.4E1,2.5E1,3.2E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[2.0540901E-3,-2.5098792E-1,5.656321E-1,-5.5768657E-1,-1.4715676E-1,6.540885E-1,-5.7894444E-1,-5.7943135E-1,-1.5414259E-3,-2.2402859E-1,7.794863E-2,5.219359E-1,1.1512493E0,-2.231027E-1,-5.0448053E-2,-2.8853305E-2,-1.4626324E-2,-3.476052E-1,-1.1459681E-1,-9.225947E-3,2.2783008E-1,3.5153764E-1,7.356745E-1,-8.011441E-3,1.2132053E0,-1.6957667E-2,3.4154763E-3,-1.0645952E-2,-3.873985E-1,-2.2852969E-1,4.4327054E-2,-4.8099283E-2,1.5441059E-2,9.136655E-2,1.7603232E-2,-7.9462E-3,3.7340155E-1,5.8648175E-1,4.1241404E-2,6.914829E-1,6.621947E-2,-9.236275E-3,-1.9813238E-2,-2.9126772E-1,-1.0423378E-1,1.3346104E-1,-1.18007004E-1,-1.312726E-1,9.640041E-3,9.402319E-3,-2.247398E-3,-5.7193497E-3,3.8475436E-1,8.118933E-3,2.9499335E-2,3.923888E-2,-5.3163082E-3,-9.577458E-3,-1.7663969E-2,-3.9072445E-4,-9.062057E-3,2.1006842E-1,-2.6532612E-3,8.8144944E-4,-9.349383E-3,6.7428744E-4,-9.1091655E-3,3.915895E-3,-3.0374732E-3,4.0519124E-1,7.337517E-3,2.9592887E-3,1.2517307E-2,1.9905213E-2,4.039144E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,-1,39,-1,-1,-1,41,43,45,47,-1,49,-1,-1,51,53,-1,55,-1,-1,-1,57,59,61,63,65,67,-1,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,73,-1,-1,-1,-1,-1],"loss_changes":[6.0750755E1,9.280325E0,1.3743038E1,8.2670593E-1,3.8417044E0,7.4803543E0,1.4650731E0,3.0070686E-1,0E0,2.191495E0,7.468555E-1,3.3470325E0,2.2467804E0,2.980504E-1,0E0,0E0,0E0,2.952652E-1,1.6216222E0,4.9413013E-1,3.7029552E-1,7.0773935E-1,3.16782E-1,0E0,1.2212753E0,0E0,0E0,0E0,2.2587776E-1,3.7514043E-1,5.6558967E-1,1.645461E-1,0E0,1.8475476E-1,0E0,0E0,3.5407734E-1,1.9707346E-1,0E0,9.3043995E-1,0E0,0E0,0E0,1.6853333E-1,1.5470237E-1,3.7374142E-1,1.5240039E-1,1.2396142E-1,1.1663472E-1,0E0,0E0,0E0,1.9267035E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.228134E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5070057E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,24,24,28,28,29,29,30,30,31,31,33,33,36,36,37,37,39,39,43,43,44,44,45,45,46,46,47,47,48,48,52,52,61,61,69,69],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,-1,40,-1,-1,-1,42,44,46,48,-1,50,-1,-1,52,54,-1,56,-1,-1,-1,58,60,62,64,66,68,-1,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,74,-1,-1,-1,-1,-1],"split_conditions":[1.2175E4,7.41718E5,1.0434409E3,1.2852292E7,7.164074E8,7.710634E3,4.352718E7,2.9778808E7,-1.5414259E-3,7.3822034E2,1.4777102E9,5.9161E4,5.0314346E1,1.173433E5,-5.0448053E-2,-2.8853305E-2,-1.4626324E-2,7.2123305E6,1.07E3,2.3171705E5,1.0284004E2,2.37E2,1.3047821E5,-8.011441E-3,6.274818E4,-1.6957667E-2,3.4154763E-3,-1.0645952E-2,5.1698097E1,1.4854412E3,1.8071064E7,3.6340196E-2,1.5441059E-2,5.78E2,1.7603232E-2,-7.9462E-3,1.6748715E8,1E0,4.1241404E-2,4.42384E5,6.621947E-2,-9.236275E-3,-1.9813238E-2,1.662639E7,7.094791E1,6.702495E-1,2.58E2,7.85935E8,5.757576E-2,9.402319E-3,-2.247398E-3,-5.7193497E-3,2.5916522E7,8.118933E-3,2.9499335E-2,3.923888E-2,-5.3163082E-3,-9.577458E-3,-1.7663969E-2,-3.9072445E-4,-9.062057E-3,3.724162E6,-2.6532612E-3,8.8144944E-4,-9.349383E-3,6.7428744E-4,-9.1091655E-3,3.915895E-3,-3.0374732E-3,5.4276E4,7.337517E-3,2.9592887E-3,1.2517307E-2,1.9905213E-2,4.039144E-3],"split_indices":[2,1,69,9,7,63,56,12,0,63,7,2,67,39,0,0,0,56,0,39,65,10,39,0,39,0,0,0,59,63,56,49,0,8,0,0,5,8,0,2,0,0,0,56,67,50,3,7,68,0,0,0,56,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,2,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,2.93E2,1.31E2,7.3E1,2.2E2,1.22E2,9E0,7E1,3E0,1.64E2,5.6E1,9.8E1,2.4E1,6E0,3E0,6.3E1,7E0,7.6E1,8.8E1,3.6E1,2E1,5.6E1,4.2E1,1E0,2.3E1,4E0,2E0,2E1,5.6E1,5.1E1,3.7E1,3.3E1,3E0,1.1E1,9E0,2E0,5.4E1,2.2E1,2E1,7E0,1.6E1,8E0,4.8E1,3.3E1,1.8E1,2.4E1,1.3E1,1.3E1,2E1,6E0,5E0,1E0,5.3E1,2E0,2E1,6E0,1E0,1.7E1,1.6E1,9E0,9E0,1.7E1,7E0,5E0,8E0,4E0,9E0,1E1,1E1,4.8E1,5E0,5E0,1.2E1,4.6E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-1.4026879E-3,-2.6144567E-1,4.759223E-1,-5.270975E-1,-1.7665638E-1,3.3803946E-1,9.2718655E-1,-2.7237935E-2,-2.7988604E-1,-2.1183571E-1,1.7341602E-1,4.1505155E-1,-3.1590834E-1,6.644843E-2,6.8551356E-1,-1.6691143E-2,-2.5223773E-3,-2.566753E-1,-8.0229045E-4,3.8058782E-1,3.5858501E-3,2.8335536E-1,6.169271E-1,-6.640257E-1,-1.5331753E-2,7.9537886E-1,-4.3016896E-3,-2.839188E-1,-7.287897E-2,-6.524276E-2,1.2709722E-2,4.304461E-3,2.4378927E-2,-4.8961104E-3,8.244548E-3,1.1713889E-1,3.3863172E-1,2.0050192E-2,3.4096044E-2,-1.5297963E-2,-4.2991504E-2,-1.2845188E-1,1.8531652E-2,1.1827856E-2,3.9986186E-2,-3.1760564E-1,-7.950662E-2,3.462E-3,-1.0922207E-1,-1.2631613E-1,4.3986528E-4,-2.385375E-2,1.414553E-2,-8.0343155E-4,3.6218008E-1,-1.0007438E-2,2.3731752E-3,-3.4404528E-1,-1.9229035E-1,-7.1973107E-3,5.910663E-3,-7.13451E-3,1.2089545E-3,-7.9491865E-3,1.7182423E-3,6.1464976E-3,-8.23385E-3,7.6793963E-3,3.828035E-1,-1.02513805E-2,-1.7338749E-2,3.9758817E-3,-1.2156674E-2,1.4906495E-2,2.3110323E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,25,-1,-1,27,29,31,33,35,37,39,41,43,-1,45,47,49,-1,-1,-1,-1,-1,51,53,-1,-1,-1,-1,55,-1,-1,-1,57,59,-1,61,63,-1,65,-1,-1,67,-1,-1,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1],"loss_changes":[5.349818E1,6.1912003E0,9.144253E0,5.617981E-1,2.6598978E0,6.07382E0,2.978611E0,0E0,1.7171115E-1,1.8355494E0,7.0231557E-1,2.6433315E0,1.3500865E0,0E0,2.202076E0,0E0,0E0,7.8046894E-1,6.197546E-1,2.9416347E-1,2.3320276E-1,5.667305E-1,4.8618317E-1,1.9915175E-1,4.1510195E-1,3.0278587E-1,0E0,9.319029E-1,1.2418516E-1,1.3303927E-1,0E0,0E0,0E0,0E0,0E0,4.6191236E-1,4.1625643E-1,0E0,0E0,0E0,0E0,1.11713536E-1,0E0,0E0,0E0,3.28125E-1,3.230635E-1,0E0,1.0127348E-1,1.1046475E-1,0E0,2.7178922E-1,0E0,0E0,1.2937021E-1,0E0,0E0,1.6294384E-1,4.1675776E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4181805E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,35,35,36,36,41,41,45,45,46,46,48,48,49,49,51,51,54,54,57,57,58,58,68,68],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,26,-1,-1,28,30,32,34,36,38,40,42,44,-1,46,48,50,-1,-1,-1,-1,-1,52,54,-1,-1,-1,-1,56,-1,-1,-1,58,60,-1,62,64,-1,66,-1,-1,68,-1,-1,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,6.0500806E3,2.7299898E8,1.1576422E9,3.653602E7,1.417988E6,-2.7237935E-2,2.033552E6,3.5212005E6,1.21E2,4.1331047E-1,3.5226266E0,6.644843E-2,2.0531621E4,-1.6691143E-2,-2.5223773E-3,7.164074E8,2.4401142E9,2.3726836E3,1.577058E9,1.3428E4,1E0,8.494E3,3.087E3,5.3140095E1,-4.3016896E-3,2.1898598E5,4.45E3,7.788E3,1.2709722E-2,4.304461E-3,2.4378927E-2,-4.8961104E-3,8.244548E-3,3.44E2,1E0,2.0050192E-2,3.4096044E-2,-1.5297963E-2,-4.2991504E-2,9.704588E6,1.8531652E-2,1.1827856E-2,3.9986186E-2,1.1888741E3,7.987553E2,3.462E-3,6.136946E5,1.7997152E7,4.3986528E-4,2.6402641E-2,1.414553E-2,-8.0343155E-4,3.3555597E8,-1.0007438E-2,2.3731752E-3,5.381665E1,4.0757E1,-7.1973107E-3,5.910663E-3,-7.13451E-3,1.2089545E-3,-7.9491865E-3,1.7182423E-3,6.1464976E-3,-8.23385E-3,7.6793963E-3,1.9437997E-1,-1.02513805E-2,-1.7338749E-2,3.9758817E-3,-1.2156674E-2,1.4906495E-2,2.3110323E-2],"split_indices":[2,1,63,7,7,56,40,0,9,59,3,46,65,0,4,0,0,7,5,4,7,2,27,0,0,67,0,39,2,10,0,0,0,0,0,8,8,0,0,0,0,9,0,0,0,63,63,0,58,9,0,68,0,0,7,0,0,59,67,0,0,0,0,0,0,0,0,0,49,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,2.78E2,1.51E2,6.6E1,2.12E2,1.17E2,3.4E1,5.5E1,1.1E1,1.93E2,1.9E1,1.05E2,1.2E1,1E1,2.4E1,8E0,3E0,1.59E2,3.4E1,8E0,1.1E1,6.5E1,4E1,5E0,7E0,2.1E1,3E0,1.38E2,2.1E1,2.8E1,6E0,3E0,5E0,7E0,4E0,1.7E1,4.8E1,1.5E1,2.5E1,3E0,2E0,6E0,1E0,2E0,1.9E1,1.18E2,2E1,4E0,1.7E1,1.5E1,1.3E1,1E1,7E0,3E0,4.5E1,4E0,2E0,9.6E1,2.2E1,1.5E1,5E0,1.3E1,4E0,1.2E1,3E0,5E0,5E0,5E0,4E1,1.4E1,8.2E1,4E0,1.8E1,2.6E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[1.272253E-2,-2.4174872E-1,4.7859406E-1,-5.2000976E-1,-1.5844427E-1,5.2707314E-1,-4.3567404E-1,-2.6403382E-2,-1.3948654E-2,-1.8793239E-1,2.3743005E-1,3.0914634E-1,7.771238E-1,-3.7508316E-2,-6.841244E-2,-2.4140967E-1,-3.908974E-2,1.6587706E-2,2.9572991E-3,2.2960533E-1,4.654074E-1,4.6404085E-1,9.059598E-1,-1.0882127E-2,5.4398403E-3,-2.5724214E-1,2.4659906E-2,-1.0997413E-1,9.354413E-2,-9.260327E-2,2.7270862E-1,3.4404922E-2,1.8253839E-2,5.3964794E-1,-8.089597E-3,7.184747E-1,1.0860153E0,-2.6793626E-1,9.1259566E-4,-3.7592968E-3,1.23027945E-2,-1.683972E-1,-4.9215846E-4,1.3700226E-1,-4.9484507E-3,-1.3281104E-2,1.7806392E-3,-9.739583E-5,2.9831412E-1,8.446695E-3,2.7297487E-2,7.467065E-1,1.2531195E-2,6.1462E-1,5.8880504E-2,-2.9366842E-1,-1.715157E-1,-1.0102763E-2,-8.4711396E-4,6.114938E-4,1.0034691E-2,3.2011038E-1,3.2762666E-3,3.7646946E-2,1.669989E-2,3.5921186E-2,-5.0867717E-3,-4.703874E-3,-1.4867681E-2,3.5000017E-3,-1.1895514E-2,9.324159E-3,1.7010983E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,-1,23,25,27,-1,-1,29,31,33,35,-1,-1,37,39,41,43,45,47,-1,-1,49,-1,51,53,55,-1,-1,-1,57,-1,59,-1,-1,-1,-1,61,-1,-1,63,-1,65,-1,67,69,-1,-1,-1,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9669983E1,6.1994905E0,6.7894135E0,3.0669022E-1,2.4960747E0,7.4432335E0,9.7503746E-1,0E0,0E0,1.550664E0,2.682523E-1,8.501859E-1,2.2014618E0,0E0,1.7150445E-1,6.129494E-1,5.08576E-1,0E0,0E0,7.613311E-1,2.5413322E-1,1.0973406E0,7.3882294E-1,0E0,0E0,4.0706348E-1,2.3726173E-1,2.0205525E-1,1.7704791E-1,1.7920184E-1,3.3125854E-1,0E0,0E0,1.21085644E-1,0E0,1.2575531E-1,7.0277786E-1,2.7465725E-1,0E0,0E0,0E0,1.3963497E-1,0E0,1.4282542E-1,0E0,0E0,0E0,0E0,1.9328594E-1,0E0,0E0,1.0765457E-1,0E0,7.766783E-1,0E0,3.240099E-1,5.8904904E-1,0E0,0E0,0E0,0E0,1.06928825E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,36,36,37,37,41,41,43,43,48,48,51,51,53,53,55,55,56,56,61,61],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,-1,24,26,28,-1,-1,30,32,34,36,-1,-1,38,40,42,44,46,48,-1,-1,50,-1,52,54,56,-1,-1,-1,58,-1,60,-1,-1,-1,-1,62,-1,-1,64,-1,66,-1,68,70,-1,-1,-1,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,1.0434409E3,2.6452028E7,1.4777102E9,5.9161E4,1.2E1,-2.6403382E-2,-1.3948654E-2,1.0911816E6,4.16606E5,1.5271514E5,6.219584E5,-3.7508316E-2,1.08935805E5,2.9063344E5,7.788E3,1.6587706E-2,2.9572991E-3,4.3290512E1,3.4E1,1.4609149E4,6.205023E3,-1.0882127E-2,5.4398403E-3,7.8E2,3.2322E4,6.7842746E8,1.6130304E7,1.6707555E8,2.8280062E-3,3.4404922E-2,1.8253839E-2,2.4126951E-1,-8.089597E-3,5.848343E7,6.274818E4,1.1501869E3,9.1259566E-4,-3.7592968E-3,1.23027945E-2,5.368828E8,-4.9215846E-4,7.556039E6,-4.9484507E-3,-1.3281104E-2,1.7806392E-3,-9.739583E-5,3.27E2,8.446695E-3,2.7297487E-2,2.1845728E7,1.2531195E-2,4.42384E5,5.8880504E-2,3.7E1,4.98884E1,-1.0102763E-2,-8.4711396E-4,6.114938E-4,1.0034691E-2,5.52488E1,3.2762666E-3,3.7646946E-2,1.669989E-2,3.5921186E-2,-5.0867717E-3,-4.703874E-3,-1.4867681E-2,3.5000017E-3,-1.1895514E-2,9.324159E-3,1.7010983E-2],"split_indices":[2,1,69,12,7,2,8,0,0,59,40,39,43,0,39,39,10,0,0,67,8,4,63,0,0,8,9,7,56,43,50,0,0,52,0,1,39,63,0,0,0,43,0,56,0,0,0,0,3,0,0,54,0,2,0,10,67,0,0,0,0,67,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.17E2,2.7E2,1.47E2,6.1E1,2.09E2,1.4E2,7E0,5.2E1,9E0,1.95E2,1.4E1,7.6E1,6.4E1,3E0,4E0,1.43E2,5.2E1,8E0,6E0,5.2E1,2.4E1,2E1,4.4E1,2E0,2E0,1.35E2,8E0,3.4E1,1.8E1,6E0,4.6E1,4E0,2E1,1.8E1,2E0,2.4E1,2E1,1.3E2,5E0,6E0,2E0,2.1E1,1.3E1,1.5E1,3E0,2E0,4E0,4E0,4.2E1,2E0,1.6E1,2.2E1,2E0,6E0,1.4E1,1.01E2,2.9E1,1.6E1,5E0,6E0,9E0,3.8E1,4E0,1.9E1,3E0,5E0,1E0,9E0,9.2E1,7E0,2.2E1,1E1,2.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[6.777529E-3,-2.1648827E-1,5.24217E-1,-4.80895E-1,-1.289547E-1,5.873731E-1,-2.860774E-1,-5.0196904E-1,6.3923624E-4,-1.6426978E-1,2.4300423E-1,4.7884062E-1,5.0072834E-2,9.320945E-2,-5.5447024E-1,-2.5131451E-2,-1.3092266E-2,-2.1926543E-1,1.2801895E-2,2.8999764E-3,1.5119663E-2,5.7619494E-1,2.924651E-1,-8.160865E-3,1.5575781E-2,-5.3021004E-3,-2.9637445E-2,-2.3000577E-1,1.2015677E-2,-6.271978E-3,7.835859E-2,5.0590706E-1,4.3203842E-2,3.4027502E-1,-8.356824E-3,-3.2397443E-1,-1.9221234E-1,1.4729798E-1,-3.1526142E-3,3.8303456E-1,6.0286057E-1,2.6207903E-1,2.6341487E-2,-3.595466E-1,-4.279912E-3,-2.0865804E-1,-9.902719E-4,-4.048976E-3,1.8449858E-1,2.982365E-1,2.6115855E-2,5.6795827E-3,6.158919E-1,8.630343E-3,1.7013995E-2,-1.7746957E-2,1.9593064E-3,-1.1121277E-2,-2.9465405E-3,-9.088053E-5,1.2371149E-2,2.137256E-3,1.5460487E-2,3.186814E-2,1.743072E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,25,-1,-1,27,29,-1,-1,31,33,-1,-1,-1,-1,35,-1,-1,37,39,-1,41,-1,43,45,47,-1,49,51,53,-1,55,-1,57,-1,-1,59,61,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9443317E1,6.858183E0,6.7696114E0,7.7755356E-1,3.0086005E0,5.5107613E0,1.0696607E0,2.5253868E-1,0E0,2.0186095E0,2.4962735E-1,1.6277847E0,0E0,3.6372662E-1,1.07831E-1,0E0,0E0,8.5718393E-1,4.8297808E-1,0E0,0E0,1.0218945E0,8.6429954E-1,0E0,0E0,0E0,0E0,4.8819447E-1,0E0,0E0,3.5816962E-1,4.2429352E-1,0E0,4.0139794E-1,0E0,3.2615662E-1,3.143158E-1,2.2289783E-1,0E0,2.1257067E-1,1.0957146E-1,1.3496244E-1,0E0,1.9709873E-1,0E0,3.5400057E-1,0E0,0E0,2.9285622E-1,1.1564481E-1,0E0,0E0,1.1961174E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,14,14,17,17,18,18,21,21,22,22,27,27,30,30,31,31,33,33,35,35,36,36,37,37,39,39,40,40,41,41,43,43,45,45,48,48,49,49,52,52],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,26,-1,-1,28,30,-1,-1,32,34,-1,-1,-1,-1,36,-1,-1,38,40,-1,42,-1,44,46,48,-1,50,52,54,-1,56,-1,58,-1,-1,60,62,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3428E4,7.41718E5,1.0434409E3,1.2852292E7,1.5522031E9,7.765071E0,2.909019E3,2.9778808E7,6.3923624E-4,2.8884522E8,1.0284004E2,1.319525E7,5.0072834E-2,2.5212732E4,3.087E3,-2.5131451E-2,-1.3092266E-2,3.7069206E5,6.719E3,2.8999764E-3,1.5119663E-2,7.710634E3,1.92144E5,-8.160865E-3,1.5575781E-2,-5.3021004E-3,-2.9637445E-2,4.407E3,1.2015677E-2,-6.271978E-3,6.744351E1,5.5835126E9,4.3203842E-2,4.23218E3,-8.356824E-3,2.889654E5,9.243714E-1,2.292258E-2,-3.1526142E-3,1.4773067E5,8.951946E2,1.4512005E5,2.6341487E-2,4.94E2,-4.279912E-3,3.7126023E11,-9.902719E-4,-4.048976E-3,6.212394E4,2.8529238E1,2.6115855E-2,5.6795827E-3,6.520156E7,8.630343E-3,1.7013995E-2,-1.7746957E-2,1.9593064E-3,-1.1121277E-2,-2.9465405E-3,-9.088053E-5,1.2371149E-2,2.137256E-3,1.5460487E-2,3.186814E-2,1.743072E-2],"split_indices":[2,1,69,9,7,46,43,12,0,12,65,9,0,4,0,0,0,39,2,0,0,63,2,0,0,0,0,2,0,0,65,12,0,63,0,39,52,49,0,39,63,39,0,3,0,42,0,0,39,67,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,2.98E2,1.28E2,7.3E1,2.25E2,1.19E2,9E0,7E1,3E0,2.06E2,1.9E1,9.8E1,2.1E1,4E0,5E0,6.2E1,8E0,1.57E2,4.9E1,6E0,1.3E1,6.3E1,3.5E1,2E0,2E0,1E0,4E0,1.54E2,3E0,1.5E1,3.4E1,5.4E1,9E0,3.2E1,3E0,4.2E1,1.12E2,2.3E1,1.1E1,2.6E1,2.8E1,2.5E1,7E0,3.6E1,6E0,1.02E2,1E1,3E0,2E1,1.9E1,7E0,1E0,2.7E1,1.5E1,1E1,3.5E1,1E0,8.7E1,1.5E1,6E0,1.4E1,2E0,1.7E1,2.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-9.109507E-3,-2.3055162E-1,4.5352593E-1,-4.5688304E-1,-1.5222739E-1,3.3307615E-1,9.4567794E-1,-4.7595376E-1,-6.535365E-4,-2.1266392E-1,5.743548E-2,3.8163462E-1,-5.7221013E-1,-7.3250844E-3,9.9578243E-1,-2.379718E-2,-9.755932E-3,-2.9643625E-1,-1.387385E-1,-3.219237E-2,2.387032E-1,2.348643E-1,5.7977384E-1,-3.948689E-2,-1.1352793E-2,1.0455942E0,-4.765936E-3,-3.4860563E-3,-1.4693076E-2,-4.4831213E-2,-2.5265414E-1,-6.8861027E-3,4.7871567E-2,2.9501206E-1,-4.946767E-3,-7.537241E-3,2.5751245E-1,5.0038397E-3,5.998939E-1,6.51807E-2,7.625551E-1,-1.0821693E-1,7.871184E-3,-2.935341E-3,-1.3278916E-2,-3.4649204E-3,5.647554E-3,2.063319E-2,8.148511E-3,5.3868815E-4,2.851742E-1,6.6888416E-1,2.1286903E-2,3.8424585E-2,4.0074144E-3,-7.4550747E-3,-1.1393471E-2,3.3307284E-1,1.6414925E-1,1.1577796E-2,6.932871E-1,1.698055E-3,-9.330036E-3,-5.0457604E-3,1.6619716E-2,9.690158E-3,-4.0527126E-3,7.355732E-3,3.3856876E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,31,33,35,37,-1,-1,39,-1,-1,-1,41,43,-1,45,47,-1,-1,49,-1,51,-1,53,55,-1,-1,-1,-1,-1,-1,-1,-1,57,59,-1,-1,-1,-1,61,63,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.27225E1,4.929529E0,7.687765E0,6.02026E-1,2.6858582E0,5.0539017E0,1.5846939E0,3.2190514E-1,0E0,9.7450924E-1,7.8752303E-1,2.9173317E0,3.149494E-1,0E0,1.46879E0,0E0,0E0,1.7101574E-1,9.407513E-1,3.0302778E-1,3.390053E-1,5.9219503E-1,3.5756588E-1,0E0,0E0,1.1323948E0,0E0,0E0,0E0,6.825702E-1,1.7825961E-1,0E0,1.7832342E-1,1.6506875E-1,0E0,0E0,3.9863348E-1,0E0,1.4767265E-1,0E0,3.6475277E-1,1.7937398E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.5256348E-1,1.1272526E-1,0E0,0E0,0E0,0E0,1.2799881E-1,2.8236437E-1,1.8180451E-1,0E0,1.0695839E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,25,25,29,29,30,30,32,32,33,33,36,36,38,38,40,40,41,41,50,50,51,51,56,56,57,57,58,58,60,60],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,32,34,36,38,-1,-1,40,-1,-1,-1,42,44,-1,46,48,-1,-1,50,-1,52,-1,54,56,-1,-1,-1,-1,-1,-1,-1,-1,58,60,-1,-1,-1,-1,62,64,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1153E4,7.41718E5,7.710634E3,1.2852292E7,7.164074E8,1.190724E4,5.0314346E1,2.9778808E7,-6.535365E-4,7.898574E2,1.4777102E9,4.1331047E-1,3.1E1,-7.3250844E-3,1.92E2,-2.379718E-2,-9.755932E-3,8.840007E7,1.8071064E7,5.3642064E-2,7.159E3,2.605513E3,3.963397E4,-3.948689E-2,-1.1352793E-2,3.3616E4,-4.765936E-3,-3.4860563E-3,-1.4693076E-2,2.325945E9,7.8471E-1,-6.8861027E-3,1.22E2,5.390361E6,-4.946767E-3,-7.537241E-3,5.767668E0,5.0038397E-3,9.83854E-1,6.51807E-2,2.617838E8,1.8931546E8,7.871184E-3,-2.935341E-3,-1.3278916E-2,-3.4649204E-3,5.647554E-3,2.063319E-2,8.148511E-3,5.3868815E-4,1.9470884E5,4.2464733E-2,2.1286903E-2,3.8424585E-2,4.0074144E-3,-7.4550747E-3,1.01267644E11,1.6748715E8,1.48075E0,1.1577796E-2,7.94203E5,1.698055E-3,-9.330036E-3,-5.0457604E-3,1.6619716E-2,9.690158E-3,-4.0527126E-3,7.355732E-3,3.3856876E-2],"split_indices":[2,1,63,9,7,4,67,12,0,63,7,46,3,0,8,0,0,5,56,49,10,44,44,0,0,10,0,0,0,5,69,0,8,56,0,0,65,0,45,0,1,43,0,0,0,0,0,0,0,0,44,49,0,0,0,0,42,5,50,0,40,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.15E2,2.81E2,1.34E2,7.1E1,2.1E2,1.09E2,2.5E1,6.8E1,3E0,1.63E2,4.7E1,1.04E2,5E0,1E0,2.4E1,6.2E1,6E0,7.5E1,8.8E1,3.2E1,1.5E1,6.1E1,4.3E1,2E0,3E0,2.3E1,1E0,4E0,7.1E1,4.9E1,3.9E1,1.3E1,1.9E1,1.3E1,2E0,3E0,5.8E1,2E0,4.1E1,9E0,1.4E1,3.8E1,1.1E1,5E0,3.4E1,7E0,1.2E1,5E0,8E0,6E0,5.2E1,2.6E1,1.5E1,1.3E1,1E0,2.5E1,1.3E1,3.6E1,1.6E1,2E0,2.4E1,1.1E1,2E0,1E0,3.5E1,1.4E1,2E0,1E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[4.6712793E-3,-2.1342064E-1,3.897534E-1,-4.3510687E-1,-1.3421042E-1,4.570639E-1,-3.9617658E-1,-4.5321527E-1,-4.502506E-4,-1.9101907E-1,-3.1680245E-2,2.3523408E-1,6.697195E-1,-2.8824339E-2,-4.6688966E-2,-2.2806166E-2,-1.2308284E-2,-6.6072084E-2,-2.4598381E-1,-1.331123E-1,7.05855E-2,2.6532993E-1,-8.643102E-2,1.0575188E0,5.616077E-1,-2.3581587E-1,1.3787913E-2,-8.766189E-2,1.1637505E-2,-2.854287E-1,-1.5915908E-1,-1.821568E-1,2.1555007E-3,1.9316219E-2,1.4074616E-2,1.8539624E-1,3.768837E-1,-9.498784E-3,6.205349E-3,2.7648255E-2,6.262261E-2,6.0267204E-1,-7.82598E-3,1.7788885E-3,-1.4218336E-2,-1.1728892E-1,-4.1802897E-4,-3.0257082E-1,-1.4077878E-3,-1.9833353E-1,4.019628E-3,-5.138903E-4,-9.933252E-3,-1.64087E-3,8.815623E-3,2.507256E-1,-7.474435E-2,2.517858E-2,1.4213598E-2,3.198596E-1,6.4903563E-1,-7.34793E-3,-8.937289E-4,-1.719334E-2,-2.1776873E-1,-1.0267188E-2,3.6055915E-4,1.5631466E-1,1.576964E-2,-1.0500817E-2,1.649147E-3,1.8342061E-2,1.9782165E-3,9.6841715E-3,6.640225E-1,-7.985312E-3,-1.8914718E-2,-3.8480368E-3,9.225411E-3,1.0214246E-2,3.243846E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,41,43,-1,45,-1,47,49,51,-1,53,-1,55,57,-1,-1,-1,-1,59,-1,-1,-1,61,-1,63,-1,65,-1,-1,-1,-1,-1,67,69,-1,-1,71,73,-1,-1,-1,75,-1,-1,77,-1,-1,-1,-1,-1,-1,79,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6699898E1,4.8407936E0,8.53949E0,5.584278E-1,1.1996412E0,6.729767E0,2.751186E0,2.130928E-1,0E0,8.971486E-1,7.873725E-1,7.296295E-1,2.5085907E0,6.599584E-1,0E0,0E0,0E0,3.0237132E-1,2.6965237E-1,3.4047008E-1,4.3127355E-1,5.2755976E-1,1.9643454E-1,1.0656471E0,1.8865147E0,1.1280587E-1,0E0,9.248018E-2,0E0,2.6308584E-1,3.1225765E-1,1.3120562E-1,0E0,2.9050425E-1,0E0,7.155279E-1,1.6459799E-1,0E0,0E0,0E0,0E0,5.397968E-1,0E0,0E0,0E0,1.0189605E-1,0E0,2.0508385E-1,0E0,9.4866395E-2,0E0,0E0,0E0,0E0,0E0,1.965487E-1,1.5048473E-1,0E0,0E0,1.634357E-1,1.8621635E-1,0E0,0E0,0E0,1.6794586E-1,0E0,0E0,1.644767E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.7733765E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,31,31,33,33,35,35,36,36,41,41,45,45,47,47,49,49,55,55,56,56,59,59,60,60,64,64,67,67,74,74],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,42,44,-1,46,-1,48,50,52,-1,54,-1,56,58,-1,-1,-1,-1,60,-1,-1,-1,62,-1,64,-1,66,-1,-1,-1,-1,-1,68,70,-1,-1,72,74,-1,-1,-1,76,-1,-1,78,-1,-1,-1,-1,-1,-1,80,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0434409E3,1.2852292E7,6.979E3,3.9837E4,7.339209E6,2.9778808E7,-4.502506E-4,7.37365E6,2.45009E6,2.8184534E7,1.417988E6,3.236383E1,-4.6688966E-2,-2.2806166E-2,-1.2308284E-2,3.7069206E5,8.636199E2,1.7990698E3,8.307654E8,6.340124E6,1.2538845E9,1.4E2,2.0531621E4,4.6976192E8,1.3787913E-2,5.1654645E6,1.1637505E-2,2.3176976E7,2.194E3,7.104E3,2.1555007E-3,1.4777102E9,1.4074616E-2,1.020494E6,2.1207E4,-9.498784E-3,6.205349E-3,2.7648255E-2,6.262261E-2,6.0891E4,-7.82598E-3,1.7788885E-3,-1.4218336E-2,7.10711E8,-4.1802897E-4,1.903E3,-1.4077878E-3,1.7212875E5,4.019628E-3,-5.138903E-4,-9.933252E-3,-1.64087E-3,8.815623E-3,3.44E2,1.7715422E5,2.517858E-2,1.4213598E-2,1.9196308E7,3.0811954E-2,-7.34793E-3,-8.937289E-4,-1.719334E-2,6.9E1,-1.0267188E-2,3.6055915E-4,2.403401E6,1.576964E-2,-1.0500817E-2,1.649147E-3,1.8342061E-2,1.9782165E-3,9.6841715E-3,3.963397E4,-7.985312E-3,-1.8914718E-2,-3.8480368E-3,9.225411E-3,1.0214246E-2,3.243846E-2],"split_indices":[2,1,69,9,2,2,58,12,0,56,9,56,40,67,0,0,0,39,63,63,12,1,7,3,4,7,0,56,0,9,0,2,0,7,0,40,9,0,0,0,0,2,0,0,0,7,0,0,0,39,0,0,0,0,0,8,44,0,0,56,49,0,0,0,8,0,0,56,0,0,0,0,0,0,44,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,2.78E2,1.57E2,7.2E1,2.06E2,1.45E2,1.2E1,6.9E1,3E0,1.32E2,7.4E1,7.2E1,7.3E1,8E0,4E0,6E1,9E0,4.1E1,9.1E1,3.7E1,3.7E1,6.6E1,6E0,1.4E1,5.9E1,5E0,3E0,3.9E1,2E0,6.1E1,3E1,2.9E1,8E0,3.1E1,6E0,4E1,2.6E1,4E0,2E0,6E0,8E0,5.6E1,3E0,1E0,4E0,2.8E1,1.1E1,5.7E1,4E0,2.6E1,4E0,4E0,2.5E1,2.4E1,7E0,3.2E1,8E0,7E0,1.9E1,9E0,4.7E1,2E1,8E0,3.2E1,2.5E1,2.4E1,2E0,1.6E1,1.6E1,3E0,5E0,7E0,2E0,2E0,4.5E1,2.1E1,4E0,2E0,1.4E1,2E0,4.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-4.9364767E-3,-1.8641898E-1,4.2144257E-1,-2.742534E-1,6.258304E-2,3.0817205E-1,8.667286E-1,-4.3038863E-1,-2.0555745E-1,-1.423721E-2,3.8617724E-1,3.4796804E-2,4.215951E-1,4.0167356E-1,1.1032175E0,-2.184332E-2,-1.20540755E-2,-1.8375349E-1,-4.6645656E-2,-8.091029E-2,9.10887E-3,1.1390976E-2,2.316996E-2,2.4369803E-1,-7.628464E-2,2.6193234E-1,5.2318317E-1,2.2116328E-2,-3.6627266E-3,3.4127112E-2,6.036272E-2,-1.9318356E-1,2.9372444E-2,-1.1565712E-1,2.3692795E-3,2.9364151E-3,1.6181216E-2,-2.9665756E-1,5.509165E-2,2.8558755E-1,-2.2032585E-3,5.425871E-1,-6.963499E-3,-2.0883907E-1,7.529047E-2,-1.3104312E-2,-8.421606E-2,-1.7262422E-2,3.779814E-3,-3.0018876E-3,7.2283843E-3,8.436246E-3,1.6496532E-2,5.5407095E-1,1.7123646E-3,-2.2865649E-1,-4.3124943E-3,-2.047115E-3,1.1156418E-2,-6.2726242E-3,-1.0930978E-3,5.712642E-1,6.9624E-3,-1.2347263E-2,-5.9410273E-3,1.1607627E-2,2.8249322E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,-1,33,-1,-1,-1,35,37,39,41,-1,-1,-1,-1,43,-1,45,-1,-1,-1,47,49,51,-1,53,-1,55,57,-1,59,-1,-1,-1,-1,-1,-1,61,-1,63,-1,-1,-1,-1,-1,65,-1,-1,-1,-1,-1],"loss_changes":[3.265569E1,6.490966E0,6.0880604E0,2.2604923E0,1.9444678E0,3.1480503E0,2.3063755E0,2.0426178E-1,2.4116821E0,8.9051646E-1,1.0378981E-1,7.379469E-1,1.0307884E0,3.306557E-1,9.854889E-2,0E0,0E0,1.2567978E0,0E0,2.2801274E-1,0E0,0E0,0E0,1.7014623E-1,6.243307E-1,2.3182249E-1,6.37414E-1,0E0,0E0,0E0,0E0,6.4618826E-1,0E0,1.7170972E-1,0E0,0E0,0E0,2.2409773E-1,1.6867195E-1,1.2185526E-1,0E0,2.2451973E-1,0E0,3.127761E-1,1.7855054E-1,0E0,9.5390245E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.04422E-1,0E0,3.4637308E-1,0E0,0E0,0E0,0E0,0E0,1.1944103E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,23,23,24,24,25,25,26,26,31,31,33,33,37,37,38,38,39,39,41,41,43,43,44,44,46,46,53,53,55,55,61,61],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,-1,34,-1,-1,-1,36,38,40,42,-1,-1,-1,-1,44,-1,46,-1,-1,-1,48,50,52,-1,54,-1,56,58,-1,60,-1,-1,-1,-1,-1,-1,62,-1,64,-1,-1,-1,-1,-1,66,-1,-1,-1,-1,-1],"split_conditions":[1.29186E9,9.570385E4,2.1455586E0,7.41718E5,1.5271514E5,8.143145E4,6.3248245E1,2.1915232E8,5.825042E6,2.2974563E3,6.874457E6,8.381319E3,2.7702793E-1,4.452787E7,2.2876814E1,-2.184332E-2,-1.20540755E-2,3.7241616E3,-4.6645656E-2,3.15E2,9.10887E-3,1.1390976E-2,2.316996E-2,3.149493E3,2.86225E4,2.4755282E7,7.79557E5,2.2116328E-2,-3.6627266E-3,3.4127112E-2,6.036272E-2,2.5653894E5,2.9372444E-2,1.5571355E3,2.3692795E-3,2.9364151E-3,1.6181216E-2,1.2128688E5,1.3412494E6,6.929085E-1,-2.2032585E-3,5.771707E9,-6.963499E-3,7.164074E8,3.2322E4,-1.3104312E-2,1.6707555E8,-1.7262422E-2,3.779814E-3,-3.0018876E-3,7.2283843E-3,8.436246E-3,1.6496532E-2,3.0037847E5,1.7123646E-3,1.0869124E3,-4.3124943E-3,-2.047115E-3,1.1156418E-2,-6.2726242E-3,-1.0930978E-3,4.2464733E-2,6.9624E-3,-1.2347263E-2,-5.9410273E-3,1.1607627E-2,2.8249322E-2],"split_indices":[7,44,53,1,39,44,67,7,58,63,1,44,46,1,65,0,0,63,0,8,0,0,0,4,44,1,2,0,0,0,0,39,0,66,0,0,0,39,59,50,0,43,0,7,9,0,43,0,0,0,0,0,0,44,0,63,0,0,0,0,0,49,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,2.95E2,1.25E2,2.18E2,7.7E1,1.01E2,2.4E1,6.5E1,1.53E2,6.3E1,1.4E1,3E1,7.1E1,9E0,1.5E1,5.5E1,1E1,1.5E2,3E0,4.8E1,1.5E1,7E0,7E0,1E1,2E1,2.9E1,4.2E1,8E0,1E0,6E0,9E0,1.49E2,1E0,3.8E1,1E1,4E0,6E0,7E0,1.3E1,2.7E1,2E0,4.1E1,1E0,1.41E2,8E0,5E0,3.3E1,6E0,1E0,6E0,7E0,1.1E1,1.6E1,4E1,1E0,1.2E2,2.1E1,5E0,3E0,1.8E1,1.5E1,3.8E1,2E0,9.2E1,2.8E1,3E0,3.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[7.4965004E-3,-1.799733E-1,3.6686966E-1,-4.0387592E-1,-1.1248614E-1,4.1065368E-1,-4.342633E-1,-2.0170003E-2,-4.708999E-3,-1.3872558E-1,2.4662136E-1,2.2871187E-1,5.4767674E-1,2.515264E-1,-7.0478094E-1,-1.7738798E-1,-4.5938143E-3,3.3986366E-1,2.3919723E-3,1.0907744E-1,3.2691452E-1,9.9082416E-1,4.7551513E-1,1.8839959E-2,-8.316743E-4,-4.1103363E-2,-1.24748545E-2,-7.018869E-2,-2.188753E-1,-1.0425208E-2,3.2940157E-2,7.2669648E-3,2.098073E-2,-1.6081484E-2,2.2696452E-1,8.437698E-3,1.7939586E-2,5.2094415E-2,1.8723909E-2,5.1211053E-1,-4.5501026E-3,-6.0496554E-3,-5.2997875E-3,-2.559129E-1,-1.2637986E-1,-5.507776E-4,1.0635057E-2,-6.4970334E-3,2.74685E-3,1.58836E-2,5.701371E-3,1.8676277E-4,5.2818173E-1,-3.0552973E-3,7.729147E-3,-2.6877445E-1,-4.067901E-3,-1.2476156E-2,-1.8805861E-2,2.9790613E-3,-2.7139029E-3,4.659109E-1,3.4024816E-2,-1.5152926E-2,-1.0157923E-2,1.4680037E-3,-8.288078E-3,2.385627E-2,1.0761555E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,-1,-1,-1,-1,41,43,-1,45,-1,-1,47,49,-1,-1,-1,-1,51,-1,-1,53,55,57,59,-1,-1,-1,-1,-1,-1,61,-1,-1,63,-1,-1,65,-1,-1,67,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.849685E1,4.155303E0,5.264471E0,3.2366467E-1,2.0601094E0,3.3151112E0,1.6614164E0,0E0,0E0,1.041542E0,2.5781012E-1,6.809945E-1,2.0231514E0,1.2240876E-1,1.9095969E-1,6.784339E-1,3.7637448E-1,1.201303E-1,0E0,4.3153393E-1,1.7601156E-1,1.7398739E-1,1.4902458E0,0E0,0E0,0E0,0E0,1.6447468E-1,3.5134792E-1,0E0,2.5448465E-1,0E0,0E0,1.4040926E-1,1.3089561E-1,0E0,0E0,0E0,0E0,5.288086E-1,0E0,0E0,2.2637124E-1,1.5052271E-1,4.8887336E-1,1.2763824E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.923416E-1,0E0,0E0,1.313405E-1,0E0,0E0,1.5882042E-1,0E0,0E0,2.9485703E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,27,27,28,28,30,30,33,33,34,34,39,39,42,42,43,43,44,44,45,45,52,52,55,55,58,58,61,61],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,-1,-1,-1,-1,42,44,-1,46,-1,-1,48,50,-1,-1,-1,-1,52,-1,-1,54,56,58,60,-1,-1,-1,-1,-1,-1,62,-1,-1,64,-1,-1,66,-1,-1,68,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,2.3033237E3,6.932264E8,1.4777102E9,1.2511627E7,2.909019E3,-2.0170003E-2,-4.708999E-3,9.570385E4,8.749E3,1.5522031E9,1.417988E6,8.494E3,3.1E1,7.2123305E6,1.5571355E3,1E0,2.3919723E-3,1.7267E4,2.2863716E-1,2.231E3,2.0531621E4,1.8839959E-2,-8.316743E-4,-4.1103363E-2,-1.24748545E-2,6.314963E2,1.0628232E3,-1.0425208E-2,1.2920962E5,7.2669648E-3,2.098073E-2,9.704588E6,2.0711772E5,8.437698E-3,1.7939586E-2,5.2094415E-2,1.8723909E-2,1.5153618E0,-4.5501026E-3,-6.0496554E-3,1.16E3,2.3961474E7,9.309699E8,4.502202E1,1.0635057E-2,-6.4970334E-3,2.74685E-3,1.58836E-2,5.701371E-3,1.8676277E-4,7.0484486E0,-3.0552973E-3,7.729147E-3,2.72E2,-4.067901E-3,-1.2476156E-2,5.374925E7,2.9790613E-3,-2.7139029E-3,1.773296E6,3.4024816E-2,-1.5152926E-2,-1.0157923E-2,1.4680037E-3,-8.288078E-3,2.385627E-2,1.0761555E-2],"split_indices":[2,1,69,7,7,1,43,0,0,44,2,7,40,0,3,56,66,24,0,2,52,0,4,0,0,0,0,63,63,0,39,0,0,9,44,0,0,0,0,64,0,0,0,9,5,69,0,0,0,0,0,0,46,0,0,3,0,0,56,0,0,11,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.21E2,2.77E2,1.44E2,6.3E1,2.14E2,1.37E2,7E0,5.9E1,4E0,2E2,1.4E1,6E1,7.7E1,2E0,5E0,1.55E2,4.5E1,9E0,5E0,2.8E1,3.2E1,9E0,6.8E1,1E0,1E0,3E0,2E0,4.4E1,1.11E2,6E0,3.9E1,4E0,5E0,1.4E1,1.4E1,9E0,2.3E1,7E0,2E0,6.4E1,4E0,2.3E1,2.1E1,7.8E1,3.3E1,3.4E1,5E0,5E0,9E0,6E0,8E0,2E0,6.2E1,1.6E1,5E0,7.2E1,6E0,1.4E1,1.9E1,1.6E1,1.8E1,4.9E1,1.3E1,3.6E1,3.6E1,1.5E1,4E0,4.2E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[8.708108E-3,-1.8562332E-1,3.6127833E-1,-4.0083644E-1,-1.1540745E-1,2.701779E-1,9.4447726E-1,-2.021524E-2,-4.752017E-3,-1.6304582E-1,4.2501643E-2,3.01458E-1,-3.8478076E-2,5.522873E-2,3.1756677E-2,-1.9518332E-1,1.0092992E-2,-2.2164173E-2,2.0574935E-1,1.2048989E-1,4.2210543E-1,-2.1952829E-1,-7.344086E-2,-9.876754E-3,5.895689E-2,-1.10724896E-1,1.5610801E-3,1.5173453E-2,3.9091986E-3,1.7396425E-1,-7.67112E-2,1.3525863E-1,4.4885787E-1,-5.5892696E-3,-2.3416047E-1,-7.067681E-3,3.2170822E-3,-1.1339653E-3,9.864829E-3,5.2639837E-3,-6.5013543E-3,2.7049118E-1,1.16040386E-1,-1.094238E-2,5.4431558E-3,1.9712025E-2,-1.2480319E-3,4.7838145E-1,7.648955E-3,-3.1608537E-1,-2.073643E-1,1.3740654E-3,1.45799E-2,-5.572377E-3,6.607705E-3,4.8505956E-1,1.5354331E-3,-1.6090238E-2,2.9908684E-3,-3.5510643E-3,-1.1530873E-2,2.011584E-2,2.7426265E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,-1,-1,-1,21,23,25,27,29,31,33,35,-1,37,39,-1,-1,-1,41,43,45,47,-1,49,-1,-1,-1,-1,-1,-1,51,53,-1,-1,-1,-1,55,-1,57,59,-1,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9117624E1,4.0942564E0,7.7674046E0,4.1586113E-1,1.5716162E0,4.687149E0,4.1715813E-1,0E0,0E0,8.922496E-1,5.21831E-1,2.7656412E0,0E0,0E0,0E0,3.8399315E-1,2.8791496E-1,1.7855772E-1,1.684143E-1,5.722407E-1,5.301752E-1,1.4043903E-1,2.622558E-1,0E0,2.7371898E-1,9.387365E-2,0E0,0E0,0E0,2.0344305E-1,3.753112E-1,3.6941132E-1,5.197849E-1,0E0,1.4739656E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.1799419E-1,1.6214561E-1,0E0,0E0,0E0,0E0,1.7212772E-1,0E0,1.9626617E-1,3.2781196E-1,0E0,0E0,0E0,0E0,1.1101246E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,29,29,30,30,31,31,32,32,34,34,41,41,42,42,47,47,49,49,50,50,55,55],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,-1,-1,-1,22,24,26,28,30,32,34,36,-1,38,40,-1,-1,-1,42,44,46,48,-1,50,-1,-1,-1,-1,-1,-1,52,54,-1,-1,-1,-1,56,-1,58,60,-1,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,8.1033794E3,3.5002026E8,6.794917E8,6.131644E7,2.231E3,-2.021524E-2,-4.752017E-3,3.5212005E6,1.4777102E9,2.5096E4,-3.8478076E-2,5.522873E-2,3.1756677E-2,2.0964778E5,4.656E3,3.6340196E-2,3.74301E5,2.4722598E7,3.963397E4,5.1698097E1,7.987553E2,-9.876754E-3,6.212394E4,4.138E3,1.5610801E-3,1.5173453E-2,3.9091986E-3,3.84142E5,2.6919386E-1,2.53383E5,1.773296E6,-5.5892696E-3,1.4448131E3,-7.067681E-3,3.2170822E-3,-1.1339653E-3,9.864829E-3,5.2639837E-3,-6.5013543E-3,7.6723E4,2.5575414E8,-1.094238E-2,5.4431558E-3,1.9712025E-2,-1.2480319E-3,5.771707E9,7.648955E-3,5.54E2,1E0,1.3740654E-3,1.45799E-2,-5.572377E-3,6.607705E-3,1.646359E5,1.5354331E-3,-1.6090238E-2,2.9908684E-3,-3.5510643E-3,-1.1530873E-2,2.011584E-2,2.7426265E-2],"split_indices":[2,1,63,7,7,61,0,0,0,59,7,2,0,0,0,39,2,49,40,56,44,59,63,0,39,2,0,0,0,40,50,9,11,0,4,0,0,0,0,0,0,40,12,0,0,0,0,43,0,3,27,0,0,0,0,39,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.23E2,2.73E2,1.5E2,6.6E1,2.07E2,1.31E2,1.9E1,6.1E1,5E0,1.59E2,4.8E1,1.28E2,3E0,9E0,1E1,1.34E2,2.5E1,3.5E1,1.3E1,5.2E1,7.6E1,1.11E2,2.3E1,4E0,2.1E1,1.3E1,2.2E1,6E0,7E0,4.1E1,1.1E1,7E0,6.9E1,1.5E1,9.6E1,1.5E1,8E0,1.4E1,7E0,1E0,1.2E1,1.4E1,2.7E1,6E0,5E0,2E0,5E0,6.2E1,7E0,2.1E1,7.5E1,2E0,1.2E1,2E0,2.5E1,6.1E1,1E0,2E1,1E0,1.6E1,5.9E1,3.9E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.1142737E-3,-1.8143001E-1,3.3159566E-1,-3.6761975E-1,-1.2027605E-1,3.832608E-1,-4.4554147E-1,-1.8848946E-2,-1.13638654E-1,-1.4323778E-1,2.1141939E-1,2.2457774E-1,5.673926E-1,-3.1598236E-2,-4.0722143E-2,-9.494211E-3,1.0196642E-3,-1.6352712E-1,6.761321E-2,1.7176427E-2,5.521781E-4,1.927677E-1,3.6887422E-2,4.6978567E-2,4.753793E-1,-8.977807E-3,1.7919904E-2,-6.638126E-2,-2.0247401E-1,-3.29471E-2,9.423561E-3,1.278422E-1,2.7804518E-1,5.159598E-1,-8.646762E-3,-6.122449E-3,3.4034643E-2,-2.2978015E-1,-1.2022232E-1,2.201765E-3,-8.728146E-3,1.7792869E-1,-2.4722999E-2,1.4000819E-2,1.5521615E-3,5.3265923E-1,8.5068755E-3,9.941811E-2,-1.6439818E-3,-2.4215393E-1,-2.8050644E-4,-2.705882E-2,-1.1284748E-2,3.9956458E-2,2.2600043E-1,-1.0201498E-2,4.6058684E-3,1.4477912E-2,5.6911993E-1,1.2808231E-3,1.048791E-2,-3.8750828E-3,-1.2364006E-2,-4.608671E-3,3.2748156E-3,-5.1944284E-3,5.6927428E-3,6.5268558E-3,1.3668325E-2,6.8064653E-3,2.8012216E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,-1,-1,-1,27,29,-1,-1,31,-1,-1,33,-1,-1,35,37,39,-1,41,43,45,-1,-1,47,49,51,-1,-1,53,55,-1,-1,57,-1,59,-1,61,-1,63,-1,65,67,-1,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5940655E1,3.1097717E0,6.313843E0,4.2631054E-1,1.6297064E0,4.0800514E0,1.660372E0,0E0,9.634574E-2,8.538275E-1,4.111644E-1,1.2287648E0,2.063591E0,4.5281148E-1,0E0,0E0,0E0,6.675949E-1,2.4148789E-1,0E0,0E0,3.8524175E-1,0E0,0E0,1.5854568E0,0E0,0E0,3.337441E-1,2.5986052E-1,1.3883702E-1,0E0,3.4826434E-1,1.1054826E-1,2.039833E-1,0E0,0E0,9.74475E-2,2.617774E-1,3.6168307E-1,0E0,0E0,2.1647704E-1,2.9556224E-1,0E0,0E0,2.4545574E-1,0E0,9.0164006E-2,0E0,2.2355461E-1,0E0,1.3959892E-1,0E0,1.3098288E-1,9.5537424E-2,0E0,0E0,0E0,2.3447418E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,17,17,18,18,21,21,24,24,27,27,28,28,29,29,31,31,32,32,33,33,36,36,37,37,38,38,41,41,42,42,45,45,47,47,49,49,51,51,53,53,54,54,58,58],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,-1,-1,-1,28,30,-1,-1,32,-1,-1,34,-1,-1,36,38,40,-1,42,44,46,-1,-1,48,50,52,-1,-1,54,56,-1,-1,58,-1,60,-1,62,-1,64,-1,66,68,-1,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0434409E3,5.609174E8,1.1576422E9,6.0891E4,1.758748E-1,-1.8848946E-2,2.033552E6,8.464347E8,1.24E2,6.651788E3,1.417988E6,3.305542E4,-4.0722143E-2,-9.494211E-3,1.0196642E-3,7.37365E6,1.685028E6,1.7176427E-2,5.521781E-4,2.1959E4,3.6887422E-2,4.6978567E-2,5.01E2,-8.977807E-3,1.7919904E-2,1.849533E6,1.1888741E3,5.4977E5,9.423561E-3,1.3508742E7,2.5916522E7,5.30113E5,-8.646762E-3,-6.122449E-3,1.36E2,4.91E2,2.700904E1,2.201765E-3,-8.728146E-3,7.10711E8,3.0811954E-2,1.4000819E-2,1.5521615E-3,5.5835126E9,8.5068755E-3,2.920516E6,-1.6439818E-3,5.1547173E1,-2.8050644E-4,8.26042E4,-1.1284748E-2,2.0241737E3,3.076578E3,-1.0201498E-2,4.6058684E-3,1.4477912E-2,8.825362E-2,1.2808231E-3,1.048791E-2,-3.8750828E-3,-1.2364006E-2,-4.608671E-3,3.2748156E-3,-5.1944284E-3,5.6927428E-3,6.5268558E-3,1.3668325E-2,6.8064653E-3,2.8012216E-2],"split_indices":[2,1,69,7,7,2,49,0,9,12,3,63,40,4,0,0,0,56,9,0,0,2,0,0,3,0,0,1,63,9,0,56,56,2,0,0,3,8,64,0,0,7,49,0,0,12,0,1,0,59,0,39,0,63,4,0,0,0,50,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.76E2,1.52E2,6.7E1,2.09E2,1.43E2,9E0,6E1,7E0,1.96E2,1.3E1,7.8E1,6.5E1,5E0,4E0,4E0,3E0,1.79E2,1.7E1,7E0,6E0,7.5E1,3E0,1E1,5.5E1,4E0,1E0,5.2E1,1.27E2,1E1,7E0,4.4E1,3.1E1,5.2E1,3E0,3.2E1,2E1,9.4E1,3.3E1,7E0,3E0,3.3E1,1.1E1,2.9E1,2E0,4.9E1,3E0,1E1,1E1,8.9E1,5E0,1.9E1,1.4E1,9E0,2.4E1,4E0,7E0,8E0,4.1E1,7E0,3E0,9E0,8E1,1.1E1,8E0,3E0,6E0,1.1E1,1.3E1,2E0,3.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-1.1406869E-2,-1.7717521E-1,2.8535944E-1,-3.4699643E-1,-1.15906246E-1,3.187144E-1,-3.757158E-1,-3.7248382E-1,-1.0592099E-2,-1.4103176E-1,1.5714264E-1,1.3628255E-1,4.3224445E-1,1.1788029E-2,-2.9804906E-2,-1.8025765E-2,-1.6069983E-3,-1.7221054E-1,-1.5098829E-2,9.194985E-2,1.7294252E-2,2.71972E-2,1.1107463E-1,3.668748E-1,6.492315E-1,-1.794041E-1,1.6493794E-4,-4.2141996E-2,8.4589785E-3,1.1648359E-3,9.030677E-3,-3.0168474E-2,1.534912E-1,3.925793E-1,-8.337224E-3,3.338087E-2,1.3072575E-2,-1.9828585E-1,-9.207497E-2,-8.7734265E-3,-5.9202494E-4,-4.4248854E-3,4.8178784E-3,1.3018766E-2,1.1039149E-1,2.3466197E-1,4.6006927E-1,-2.1559228E-1,-9.494157E-2,-8.162676E-3,-1.0587259E-3,-5.9460066E-3,1.3714862E-1,1.2152325E-2,-4.8689796E-3,8.576682E-3,5.035564E-1,-4.21269E-3,-1.0905191E-2,-7.904994E-3,9.489555E-5,1.1925447E-2,4.1566542E-3,2.202476E-2,4.2157613E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,-1,-1,-1,25,27,29,-1,-1,31,33,35,37,-1,39,-1,-1,-1,41,43,45,-1,-1,-1,47,49,-1,-1,-1,-1,-1,51,53,55,57,59,-1,-1,-1,61,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1548538E1,2.883522E0,3.5816164E0,1.5325451E-1,1.4491451E0,3.0478163E0,1.3932292E0,9.24263E-2,0E0,7.47349E-1,2.0746505E-1,5.8147347E-1,1.0667744E0,0E0,0E0,0E0,0E0,1.9396019E-1,2.1030734E-1,9.391555E-2,0E0,0E0,3.461203E-1,1.0847464E0,1.8476486E-1,2.241354E-1,0E0,1.4517626E-1,0E0,0E0,0E0,1.2414401E-1,1.9605994E-1,6.380873E-1,0E0,0E0,0E0,1.9415379E-1,1.5069488E-1,0E0,0E0,0E0,0E0,0E0,2.3084328E-1,1.8037641E-1,4.9477673E-1,1.4713478E-1,1.304891E-1,0E0,0E0,0E0,1.531471E-1,0E0,0E0,0E0,2.4809265E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,17,17,18,18,19,19,22,22,23,23,24,24,25,25,27,27,31,31,32,32,33,33,37,37,38,38,44,44,45,45,46,46,47,47,48,48,52,52,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,-1,-1,-1,26,28,30,-1,-1,32,34,36,38,-1,40,-1,-1,-1,42,44,46,-1,-1,-1,48,50,-1,-1,-1,-1,-1,52,54,56,58,60,-1,-1,-1,62,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,2.3033237E3,1.6395413E7,1.1576422E9,2.5096E4,2.909019E3,2.7299898E8,-1.0592099E-2,9.570385E4,4.433448E6,8.504733E7,4.026531E9,1.1788029E-2,-2.9804906E-2,-1.8025765E-2,-1.6069983E-3,3.94E2,1.4998456E5,6.498131E1,1.7294252E-2,2.71972E-2,7.10711E8,3.782676E4,4.547513E7,2.0964778E5,1.6493794E-4,1.6826648E3,8.4589785E-3,1.1648359E-3,9.030677E-3,2.6084324E3,5.4629724E10,1E0,-8.337224E-3,3.338087E-2,1.3072575E-2,1.4854412E3,2.077852E6,-8.7734265E-3,-5.9202494E-4,-4.4248854E-3,4.8178784E-3,1.3018766E-2,3.0811954E-2,3.653602E7,1.1335203E-1,2.42E2,7.203055E10,-8.162676E-3,-1.0587259E-3,-5.9460066E-3,3.9E1,1.2152325E-2,-4.8689796E-3,8.576682E-3,4.52E2,-4.21269E-3,-1.0905191E-2,-7.904994E-3,9.489555E-5,1.1925447E-2,4.1566542E-3,2.202476E-2,4.2157613E-2],"split_indices":[2,1,69,12,7,2,43,7,0,44,59,7,7,0,0,0,0,8,39,67,0,0,7,63,40,39,0,4,0,0,0,63,42,27,0,0,0,63,1,0,0,0,0,0,49,56,49,11,42,0,0,0,3,0,0,0,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.36E2,2.8E2,1.56E2,7.3E1,2.07E2,1.49E2,7E0,5.9E1,1.4E1,1.9E2,1.7E1,5.8E1,9.1E1,2E0,5E0,5.8E1,1E0,1.52E2,3.8E1,1.4E1,3E0,2E0,5.6E1,7.2E1,1.9E1,1.46E2,6E0,3.4E1,4E0,9E0,5E0,1.3E1,4.3E1,6.9E1,3E0,1.6E1,3E0,1.19E2,2.7E1,5E0,2.9E1,9E0,4E0,1E1,3.3E1,2.2E1,4.7E1,1.01E2,1.8E1,1.2E1,1.5E1,3E0,3E1,2.1E1,1E0,7E0,4E1,1E1,9.1E1,1E1,8E0,8E0,2.2E1,3.8E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-9.996277E-3,-1.6350117E-1,2.697455E-1,-3.2819474E-1,-1.13939E-1,3.0028087E-1,-3.1092137E-2,-1.6541637E-2,9.6316024E-4,-1.6151592E-1,-1.7133856E-2,2.5384277E-1,8.3023953E-1,-2.3358628E-1,-1.07915975E-1,-4.9032144E-2,2.0096941E-1,1.24775656E-1,3.6637688E-1,4.695521E-2,2.1444071E-2,-1.1823037E-2,-3.3390713E-3,-1.6910708E-1,7.637283E-3,-5.4306863E-3,5.729843E-3,1.4830407E-2,2.4359229E-3,1.520583E-1,-2.6641139E-3,-2.9196361E-2,4.0391806E-1,-6.7885113E-3,-1.3590169E-2,1.1024886E-2,-1.2895422E-3,-4.7981456E-2,3.3007618E-3,2.0663084E-1,4.6224795E-2,-9.987794E-3,4.0416894E-3,4.128697E-1,-3.819281E-3,2.5500248E-3,-5.3261463E-3,1.4961457E-2,1.401577E-1,-6.313845E-3,5.467326E-3,1.4112467E-2,2.2570333E-2,2.9378862E-3,1.0856845E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,-1,-1,-1,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,33,35,-1,37,-1,-1,39,-1,41,43,-1,-1,-1,-1,45,-1,47,49,-1,-1,51,-1,-1,-1,-1,53,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7693157E1,2.1338491E0,4.3022575E0,3.9731216E-1,9.460051E-1,3.283906E0,0E0,0E0,0E0,5.0788164E-1,4.898953E-1,1.8715134E0,2.0134735E-1,1.1664796E-1,5.7402885E-1,2.169332E-1,1.322285E-1,3.188885E-1,1.0513134E0,0E0,0E0,0E0,0E0,1.2423086E-1,2.3069562E-1,0E0,1.1914209E-1,0E0,0E0,3.0810666E-1,0E0,1.612446E-1,3.1028557E-1,0E0,0E0,0E0,0E0,1.2851314E-1,0E0,2.1170259E-1,2.590639E-1,0E0,0E0,3.0319786E-1,0E0,0E0,0E0,0E0,1.5070099E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,23,23,24,24,26,26,29,29,31,31,32,32,37,37,39,39,40,40,43,43,48,48],"right_children":[2,4,6,8,10,12,-1,-1,-1,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,34,36,-1,38,-1,-1,40,-1,42,44,-1,-1,-1,-1,46,-1,48,50,-1,-1,52,-1,-1,-1,-1,54,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,7.41718E5,2.3033237E3,1.2852292E7,8.298118E6,1.3554651E2,-3.1092137E-2,-1.6541637E-2,9.6316024E-4,1.964127E6,1.484798E9,2.5096E4,2.231E3,5.3413135E-1,1.1566843E3,5.3642064E-2,6.1614815E6,2.9991518E7,3.963397E4,4.695521E-2,2.1444071E-2,-1.1823037E-2,-3.3390713E-3,2.748728E6,2.3199144E1,-5.4306863E-3,1.782402E6,1.4830407E-2,2.4359229E-3,5.1359596E1,-2.6641139E-3,9.835548E-2,1.4656219E8,-6.7885113E-3,-1.3590169E-2,1.1024886E-2,-1.2895422E-3,1.376072E8,3.3007618E-3,4.8E1,1.0336E4,-9.987794E-3,4.0416894E-3,6.0891E4,-3.819281E-3,2.5500248E-3,-5.3261463E-3,1.4961457E-2,3.0995308E3,-6.313845E-3,5.467326E-3,1.4112467E-2,2.2570333E-2,2.9378862E-3,1.0856845E-2],"split_indices":[2,1,69,9,9,67,0,0,0,1,7,2,0,52,63,49,56,56,44,0,0,0,0,9,67,0,1,0,0,69,0,49,1,0,0,0,0,12,0,3,10,0,0,2,0,0,0,0,66,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.1E2,2.65E2,1.45E2,6E1,2.05E2,1.41E2,4E0,5.7E1,3E0,1.37E2,6.8E1,1.31E2,1E1,5.7E1,8E1,6E1,8E0,6.2E1,6.9E1,6E0,4E0,5.2E1,5E0,5.2E1,2.8E1,2.7E1,3.3E1,4E0,4E0,5.4E1,8E0,6E0,6.3E1,4.4E1,8E0,3E0,2.5E1,1.8E1,1.5E1,3.5E1,1.9E1,2E0,4E0,6.2E1,1E0,7E0,1.1E1,1.2E1,2.3E1,5E0,1.4E1,2.3E1,3.9E1,1.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-4.277555E-3,-1.655542E-1,2.7695015E-1,-3.178445E-1,-1.1017878E-1,3.2785642E-1,-3.0823863E-1,-1.8127676E-2,-2.3744E-1,-3.5278983E-2,-1.8252978E-1,1.8476349E-1,5.141283E-1,-1.6417056E-2,-3.6841862E-2,-1.3794511E-4,-1.241213E-2,-8.3202004E-2,1.0431738E-1,-2.502524E-1,-1.3480142E-1,2.2430806E-1,5.151417E-3,4.360097E-1,4.418498E-2,-9.520281E-3,1.2517328E-2,-1.146083E-1,1.9415177E-1,-2.5374254E-3,1.6513382E-1,-1.7281495E-1,-1.72044E-2,-1.8532932E-1,-5.2606963E-2,1.7866915E-1,3.179442E-1,-5.930738E-3,5.7088886E-3,4.4973108E-1,-6.95013E-3,-1.7977747E-1,-3.2126177E-3,2.2645472E-3,1.5703306E-2,1.3653481E-2,3.3774308E-3,-9.545423E-3,4.8757414E-3,-2.1332216E-1,-2.5627033E-3,9.3036256E-4,-6.9754166E-3,2.1629661E-1,2.8985038E-3,3.3290648E-1,4.4625628E-4,4.0105194E-1,3.06983E-2,-2.130688E-1,-2.3610038E-3,-1.1583387E-2,-2.412377E-3,1.4743622E-1,2.7014875E-1,3.6111411E-3,1.7053531E-2,7.6575847E-3,4.2061394E-1,-1.1239457E-2,3.5032388E-4,-3.6753188E-3,8.606005E-3,-1.7308654E-3,1.38030555E-2,2.1292144E-2,1.1128369E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,-1,-1,-1,27,29,31,33,35,37,39,-1,-1,-1,41,43,-1,45,47,-1,49,51,53,55,-1,-1,57,-1,59,-1,-1,-1,-1,-1,-1,-1,61,-1,-1,-1,63,-1,65,-1,67,-1,69,-1,-1,-1,71,73,-1,-1,-1,75,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9412632E1,2.2587934E0,4.7482615E0,2.6843834E-1,1.0827944E0,3.7432108E0,1.7461962E0,0E0,1.7951775E-1,6.776674E-1,2.95959E-1,5.88367E-1,1.589838E0,5.1137453E-1,0E0,0E0,0E0,6.755505E-1,2.577883E-1,2.8757143E-1,2.4774337E-1,2.3378754E-1,2.5183928E-1,4.9643898E-1,0E0,0E0,0E0,1.9751471E-1,1.4265478E-1,0E0,1.965782E-1,2.1467113E-1,0E0,1.2857413E-1,1.7144138E-1,1.9988406E-1,9.3850374E-2,0E0,0E0,2.410984E-1,0E0,1.1093211E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3589132E-1,0E0,0E0,0E0,9.366834E-2,0E0,9.8404884E-2,0E0,1.4433384E-1,0E0,1.1038345E-1,0E0,0E0,0E0,1.4666313E-1,1.2131977E-1,0E0,0E0,0E0,9.1799736E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,27,27,28,28,30,30,31,31,33,33,34,34,35,35,36,36,39,39,41,41,49,49,53,53,55,55,57,57,59,59,63,63,64,64,68,68],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,-1,-1,-1,28,30,32,34,36,38,40,-1,-1,-1,42,44,-1,46,48,-1,50,52,54,56,-1,-1,58,-1,60,-1,-1,-1,-1,-1,-1,-1,62,-1,-1,-1,64,-1,66,-1,68,-1,70,-1,-1,-1,72,74,-1,-1,-1,76,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,1.0434409E3,1.425966E6,1.2273383E7,6.4945E4,7.339209E6,-1.8127676E-2,2E1,2.612106E6,7.127066E2,1.99218E7,1.152089E1,3.236383E1,-3.6841862E-2,-1.3794511E-4,-1.241213E-2,3.116216E3,3.9E1,7.5E1,1.247106E6,2.5096E4,2.2989038E6,5.01E2,4.418498E-2,-9.520281E-3,1.2517328E-2,1.6845247E3,1E0,-2.5374254E-3,1.03E2,4.61E2,-1.72044E-2,6.794917E8,3.52E2,2.0014305E5,6.2185767E10,-5.930738E-3,5.7088886E-3,9.317209E0,-6.95013E-3,5.855E3,-3.2126177E-3,2.2645472E-3,1.5703306E-2,1.3653481E-2,3.3774308E-3,-9.545423E-3,4.8757414E-3,1.4555195E5,-2.5627033E-3,9.3036256E-4,-6.9754166E-3,5.841568E1,2.8985038E-3,2E0,4.4625628E-4,3E0,3.06983E-2,3.53E2,-2.3610038E-3,-1.1583387E-2,-2.412377E-3,9.06E2,1.0262E4,3.6111411E-3,1.7053531E-2,7.6575847E-3,2.8878675E5,-1.1239457E-2,3.5032388E-4,-3.6753188E-3,8.606005E-3,-1.7308654E-3,1.38030555E-2,2.1292144E-2,1.1128369E-2],"split_indices":[2,1,69,9,56,2,58,0,10,1,63,56,64,67,0,0,0,4,8,8,40,2,58,3,0,0,0,4,24,0,3,3,0,7,3,44,5,0,0,46,0,2,0,0,0,0,0,0,0,39,0,0,0,67,0,8,0,8,0,3,0,0,0,0,2,0,0,0,44,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,2.71E2,1.55E2,7.1E1,2E2,1.43E2,1.2E1,3.8E1,3.3E1,9.9E1,1.01E2,8.2E1,6.1E1,8E0,4E0,3E0,3E1,7.4E1,2.5E1,4E1,6.1E1,6.7E1,1.5E1,5.3E1,8E0,5E0,3E0,6.7E1,7E0,7E0,1.8E1,2.5E1,1.5E1,3.7E1,2.4E1,4.7E1,2E1,7E0,8E0,5.2E1,1E0,2.7E1,4E1,4E0,3E0,7E0,1.1E1,2.3E1,2E0,3E1,7E0,1.4E1,1E1,3.5E1,1.2E1,1.9E1,1E0,4.4E1,8E0,2.1E1,6E0,2.5E1,5E0,1.7E1,1.8E1,2E0,1.7E1,4E0,4E1,1.9E1,2E0,2E0,1.5E1,1E0,1.7E1,3.4E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[3.3665998E-3,-1.2703465E-1,3.2988074E-1,-1.9881015E-1,5.6385014E-2,2.64411E-1,6.4728236E-1,-1.8692267E-1,-3.76146E-2,-3.5902664E-2,2.1324243E-1,2.8246754E-1,-1.1865427E-2,3.2200493E-2,3.047563E-3,-2.8010952E-1,-1.226267E-1,-6.206494E-2,7.2723497E-3,5.010673E-3,1.503257E-2,8.559418E-2,3.2516047E-1,-3.2825258E-1,-1.4758463E-1,-1.3237855E-1,2.686305E-2,-9.913684E-3,-4.0794976E-2,9.691018E-3,-3.2197011E-3,3.0346218E-1,3.7619527E-2,-3.3978924E-1,-3.4910103E-3,-2.418189E-3,-1.0506398E-2,-1.8208434E-1,-7.301057E-2,-8.265201E-4,-8.906878E-3,7.237455E-2,3.249238E-1,-1.7008133E-2,-7.8089912E-3,-2.8241113E-1,-1.5375113E-1,-1.1010739E-1,4.035739E-3,5.9933066E-3,-7.1586296E-3,3.714766E-1,2.3357746E-1,-4.8568584E-3,-1.6443113E-2,-8.974749E-3,-3.6801936E-3,-2.8398666E-3,-8.616175E-3,1.8379781E-2,6.321883E-3,4.26161E-3,1.49494205E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,23,25,27,-1,-1,-1,29,31,33,35,37,-1,-1,39,-1,-1,41,-1,43,-1,-1,-1,45,47,-1,-1,49,51,-1,-1,53,55,57,-1,-1,-1,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8052338E1,4.000216E0,2.343257E0,1.379715E0,1.2525926E0,1.0150495E0,2.7602196E-1,1.2538323E0,0E0,2.8035533E-1,3.1928456E-1,8.042736E-1,0E0,0E0,0E0,5.0662804E-1,9.367678E-1,1.4331235E-1,0E0,0E0,0E0,3.552087E-1,5.745783E-1,1.6056585E-1,1.645999E-1,3.6442542E-1,0E0,0E0,1.4740127E-1,0E0,0E0,3.6828566E-1,0E0,1.1661291E-1,0E0,0E0,0E0,1.5271664E-1,3.600148E-1,0E0,0E0,1.1344936E-1,2.1940565E-1,0E0,0E0,1.2197387E-1,1.3263226E-1,1.6386533E-1,0E0,0E0,0E0,9.849024E-2,2.8225207E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,15,15,16,16,17,17,21,21,22,22,23,23,24,24,25,25,28,28,31,31,33,33,37,37,38,38,41,41,42,42,45,45,46,46,47,47,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,24,26,28,-1,-1,-1,30,32,34,36,38,-1,-1,40,-1,-1,42,-1,44,-1,-1,-1,46,48,-1,-1,50,52,-1,-1,54,56,58,-1,-1,-1,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,9.570385E4,7.17892E0,5.825042E6,2.1700479E3,3.768034E4,8.927E3,9.50639E5,-3.76146E-2,1.5271514E5,2.1583E4,1.2754084E6,-1.1865427E-2,3.2200493E-2,3.047563E-3,3.006846E6,3.7241616E3,2.18432E1,7.2723497E-3,5.010673E-3,1.503257E-2,3.68E3,3.305542E4,9.65325E8,1.3436554E3,7.898574E2,2.686305E-2,-9.913684E-3,1E0,9.691018E-3,-3.2197011E-3,3E0,3.7619527E-2,1.4910261E5,-3.4910103E-3,-2.418189E-3,-1.0506398E-2,2.407E3,2.094E3,-8.265201E-4,-8.906878E-3,7.602E3,1.1559897E7,-1.7008133E-2,-7.8089912E-3,1.6225006E4,5.446E3,3.164557E-2,4.035739E-3,5.9933066E-3,-7.1586296E-3,5.30113E5,7.3036104E-1,-4.8568584E-3,-1.6443113E-2,-8.974749E-3,-3.6801936E-3,-2.8398666E-3,-8.616175E-3,1.8379781E-2,6.321883E-3,4.26161E-3,1.49494205E-2],"split_indices":[7,44,46,58,63,4,0,1,0,39,10,61,0,0,0,9,63,67,0,0,0,11,4,5,58,63,0,0,19,0,0,8,0,58,0,0,0,11,0,0,0,0,9,0,0,39,2,68,0,0,0,2,50,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.22E2,3.02E2,1.2E2,2.17E2,8.5E1,1.01E2,1.9E1,2.14E2,3E0,5.4E1,3.1E1,9.8E1,3E0,1.8E1,1E0,8.6E1,1.28E2,4.8E1,6E0,1.6E1,1.5E1,1.8E1,8E1,6.2E1,2.4E1,1.27E2,1E0,5E0,4.3E1,1E1,8E0,7.8E1,2E0,5.9E1,3E0,1.1E1,1.3E1,6.8E1,5.9E1,3.8E1,5E0,7E0,7.1E1,5.3E1,6E0,1.3E1,5.5E1,4.8E1,1.1E1,6E0,1E0,4.5E1,2.6E1,4E0,9E0,3.7E1,1.8E1,2.9E1,1.9E1,4.2E1,3E0,1E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.741665E-4,-1.1761303E-1,3.0786183E-1,-1.5549707E-1,1.3955157E-1,2.0156133E-1,5.4033506E-1,-3.2571083E-1,-1.276535E-1,1.9799939E-1,-3.2756116E-2,2.3364961E-1,-1.2659322E-2,3.604883E-1,7.2013766E-1,-1.6990997E-2,-6.5152226E-3,-8.799392E-2,-1.9821166E-1,3.0876298E-2,1.4411978E-1,1.7606582E-1,4.2541143E-1,1.8394576E-2,-3.2790923E-3,4.4517435E-2,2.3869846E-2,-1.162159E-1,9.175179E-2,-1.2188342E-1,-2.2535443E-1,4.0001906E-2,1.2018566E-2,7.68544E-2,2.4192484E-1,4.7707564E-1,5.981502E-3,-1.5775153E-1,-5.5152718E-2,8.405625E-3,3.2369758E-4,-7.1912315E-3,3.191914E-3,-2.0159449E-1,-2.1987174E-2,-5.640702E-3,4.1493666E-3,9.744633E-3,-2.3427237E-2,4.8488537E-3,2.8197467E-1,3.8203849E-3,2.385188E-2,-1.9502376E-1,-2.5861906E-2,-9.979173E-2,1.5174156E-2,-2.2156663E-1,2.788332E-3,-4.8554735E-3,3.2260122E-3,2.2866302E-3,1.4659996E-2,-9.891085E-3,1.34022515E-2,5.059585E-3,-3.843659E-3,-6.592398E-3,-7.0765585E-4,5.0864452E-3,-2.443299E-3,-1.4852175E-2,-8.0935825E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,25,-1,-1,27,29,-1,31,33,35,-1,-1,-1,-1,37,39,41,43,45,-1,47,49,51,-1,53,55,-1,-1,-1,-1,57,-1,-1,-1,-1,59,-1,61,-1,-1,63,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6125774E1,3.1685061E0,2.9306679E0,1.2949219E0,2.163114E0,1.350107E0,9.6920204E-1,1.7474651E-1,6.6534567E-1,8.742039E-1,0E0,8.195319E-1,0E0,2.5084782E-1,3.7540436E-1,0E0,0E0,8.121623E-1,1.52776E-1,0E0,4.047492E-1,3.9958096E-1,2.1928787E-1,0E0,0E0,0E0,0E0,3.387723E-1,1.5649119E-1,1.4588243E-1,2.5934458E-1,1.5874413E-1,0E0,3.4947255E-1,1.8493295E-1,9.48503E-2,0E0,3.9313817E-1,1.8045564E-1,0E0,0E0,0E0,0E0,3.193314E-1,0E0,0E0,0E0,0E0,1.20898046E-1,0E0,1.5610671E-1,0E0,0E0,4.239943E-1,1.4549842E-1,1.1192277E-1,1.449511E-1,2.0348573E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,17,17,18,18,20,20,21,21,22,22,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,37,37,38,38,43,43,48,48,50,50,53,53,54,54,55,55,56,56,57,57],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,26,-1,-1,28,30,-1,32,34,36,-1,-1,-1,-1,38,40,42,44,46,-1,48,50,52,-1,54,56,-1,-1,-1,-1,58,-1,-1,-1,-1,60,-1,62,-1,-1,64,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,2.144067E3,1.486066E0,4.26944E5,1.0576174E4,1.4609149E4,6.625959E1,7.154E3,1.7458724E7,7.532E3,-3.2756116E-2,9.9179E4,-1.2659322E-2,9.677921E7,1.681178E6,-1.6990997E-2,-6.5152226E-3,1.2833E4,6.465659E4,3.0876298E-2,2.0663544E9,8.330997E4,7.507126E7,1.8394576E-2,-3.2790923E-3,4.4517435E-2,2.3869846E-2,1.533115E6,1.2035191E1,2.4422344E11,4.086885E7,5.2E2,1.2018566E-2,7.12E2,4.51698E0,1.615586E9,5.981502E-3,7.10711E8,1.656968E6,8.405625E-3,3.2369758E-4,-7.1912315E-3,3.191914E-3,1.7081438E3,-2.1987174E-2,-5.640702E-3,4.1493666E-3,9.744633E-3,2.058478E7,4.8488537E-3,7.763925E2,3.8203849E-3,2.385188E-2,8.287E3,4.45E3,2.0856753E5,6.726204E1,6.2702277E2,2.788332E-3,-4.8554735E-3,3.2260122E-3,2.2866302E-3,1.4659996E-2,-9.891085E-3,1.34022515E-2,5.059585E-3,-3.843659E-3,-6.592398E-3,-7.0765585E-4,5.0864452E-3,-2.443299E-3,-1.4852175E-2,-8.0935825E-3],"split_indices":[7,63,50,1,4,4,67,2,56,9,0,2,0,1,40,0,0,10,39,0,12,44,1,0,0,0,0,1,64,42,56,10,0,0,64,7,0,7,9,0,0,0,0,63,0,0,0,0,43,0,63,0,0,2,2,39,65,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.44E2,3.22E2,1.22E2,2.81E2,4.1E1,8.5E1,3.7E1,3.8E1,2.43E2,3.9E1,2E0,8E1,5E0,2E1,1.7E1,3.2E1,6E0,1.57E2,8.6E1,3E0,3.6E1,6.3E1,1.7E1,1.9E1,1E0,7E0,1E1,1.36E2,2.1E1,2.4E1,6.2E1,1.9E1,1.7E1,2.6E1,3.7E1,1.4E1,3E0,8E1,5.6E1,1E1,1.1E1,2.1E1,3E0,5.8E1,4E0,4E0,1.5E1,1.1E1,1.5E1,9E0,2.8E1,1E0,1.3E1,6.2E1,1.8E1,3.4E1,2.2E1,5.4E1,4E0,8E0,7E0,3E0,2.5E1,6.1E1,1E0,5E0,1.3E1,2.3E1,1.1E1,9E0,1.3E1,1.8E1,3.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[2.3604361E-3,-1.389904E-1,2.7555963E-1,-2.9650143E-1,-8.800974E-2,1.8266994E-1,6.094791E-1,-1.8461298E-2,-1.2533572E-2,-1.083133E-1,1.3141264E-1,2.4359095E-1,-2.5189385E-1,6.6422725E-1,2.8726927E-3,-1.628635E-1,-5.983153E-2,1.1462994E-2,-7.6856354E-4,1.6647057E-1,3.7315607E-1,-2.6481302E-2,-3.3288453E-2,3.1591353E-1,7.606946E-1,-1.9235185E-1,-2.4436547E-3,-8.670134E-2,4.7773016E-3,2.2268589E-3,1.9445117E-1,4.747509E-1,1.28064E-2,-1.0060768E-1,1.7058523E-2,1.9011395E-2,-6.2248236E-3,4.2982392E-2,2.572705E-2,-2.4419782E-1,-1.4217782E-1,-5.0397318E-2,-7.6970425E-3,1.5293507E-1,1.2047611E-2,3.3336617E-3,4.9446058E-1,-8.864849E-3,3.5083052E-3,-2.67786E-1,-4.083704E-3,-9.608223E-3,-2.8033662E-3,-8.687829E-2,1.3574073E-3,2.0416641E-4,8.238958E-3,2.4635926E-2,3.5205649E-3,-1.4661216E-2,-5.7800272E-3,-6.520447E-3,-1.4766881E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,25,27,-1,-1,29,31,33,-1,35,37,39,-1,41,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,49,51,53,-1,55,-1,-1,57,-1,-1,59,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6875282E1,2.2865496E0,4.5103006E0,1.1274338E-1,9.8989093E-1,3.1852994E0,9.223528E-1,0E0,0E0,5.218384E-1,3.086618E-1,9.7705936E-1,1.4879478E0,7.3411274E-1,0E0,2.9968166E-1,4.7275662E-1,0E0,0E0,2.1554673E-1,2.774086E-1,3.4896213E-1,0E0,3.4613723E-1,1.9656658E-1,1.5777826E-1,0E0,2.4360305E-1,0E0,0E0,9.133005E-2,1.09116554E-1,0E0,1.6185142E-1,0E0,0E0,0E0,0E0,0E0,1.080606E-1,1.8551725E-1,1.8653616E-1,0E0,9.872353E-2,0E0,0E0,1.1701155E-1,0E0,0E0,1.3609982E-1,0E0,0E0,0E0,1.1817992E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,20,20,21,21,23,23,24,24,25,25,27,27,30,30,31,31,33,33,39,39,40,40,41,41,43,43,46,46,49,49,53,53],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,26,28,-1,-1,30,32,34,-1,36,38,40,-1,42,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,50,52,54,-1,56,-1,-1,58,-1,-1,60,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0863E4,7.41718E5,6.651788E3,1.44E2,1.2678202E9,3.653602E7,1.579228E8,-1.8461298E-2,-1.2533572E-2,7.3822034E2,2.3878E4,6.8963E4,1.6244629E-1,6.069097E4,2.8726927E-3,3.135717E6,2.103E3,1.1462994E-2,-7.6856354E-4,1.3428E4,2.5534925E6,3.305542E4,-3.3288453E-2,5.358657E7,3.3616E4,1.903E3,-2.4436547E-3,2.3790412E7,4.7773016E-3,2.2268589E-3,1.7278508E5,7.532E3,1.28064E-2,8.5E1,1.7058523E-2,1.9011395E-2,-6.2248236E-3,4.2982392E-2,2.572705E-2,1.3085492E-1,2.58E2,1.1662405E9,-7.6970425E-3,2.8529238E1,1.2047611E-2,3.3336617E-3,5.848343E7,-8.864849E-3,3.5083052E-3,2.0964778E5,-4.083704E-3,-9.608223E-3,-2.8033662E-3,1.0865863E7,1.3574073E-3,2.0416641E-4,8.238958E-3,2.4635926E-2,3.5205649E-3,-1.4661216E-2,-5.7800272E-3,-6.520447E-3,-1.4766881E-3],"split_indices":[2,1,63,0,7,56,1,0,0,63,11,2,49,39,0,9,0,0,0,2,58,4,0,1,10,0,0,56,0,0,39,9,0,8,0,0,0,0,0,49,3,5,0,67,0,0,1,0,0,39,0,0,0,9,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.35E2,2.87E2,1.48E2,6.9E1,2.18E2,1.17E2,3.1E1,1.6E1,5.3E1,2E2,1.8E1,1.03E2,1.4E1,2.8E1,3E0,9.3E1,1.07E2,1E1,8E0,6.6E1,3.7E1,1E1,4E0,7E0,2.1E1,7.3E1,2E1,9.2E1,1.5E1,1.3E1,5.3E1,1.7E1,2E1,9E0,1E0,6E0,1E0,1.1E1,1E1,3.4E1,3.9E1,6.3E1,2.9E1,3.3E1,2E1,1E0,1.6E1,6E0,3E0,2.9E1,5E0,2.2E1,1.7E1,4.3E1,2E1,4E0,2.9E1,1.5E1,1E0,2.2E1,7E0,2.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[1.6083964E-3,-1.2607625E-1,2.3360005E-1,-2.5778073E-1,-8.136146E-2,1.5612139E-1,5.3347987E-1,-2.7138016E-1,1.6445398E-3,-9.636284E-2,1.7294139E-1,1.9182517E-1,-5.7207304E-1,4.0619567E-2,3.1628418E-1,-1.7678207E-2,-1.1250196E-2,-1.3097844E-1,-2.0107541E-2,1.5310563E-2,2.8128107E-3,1.2344559E-1,2.889736E-1,-1.8729233E-3,-3.2017346E-2,3.981092E-1,-5.276306E-3,-1.5101844E-1,5.4442924E-2,-5.7224337E-2,5.6851022E-2,1.5415251E-1,1.1678142E-3,3.039639E-1,-9.9177295E-3,5.0118715E-3,2.0533249E-2,-1.7060587E-1,-2.2793964E-3,-8.060371E-4,8.888147E-3,-8.573822E-2,3.734503E-3,1.0144497E-2,3.6834143E-4,8.842481E-3,1.995166E-3,1.9861367E-1,1.704185E-2,-3.548124E-3,-1.8745618E-1,-6.882288E-3,-1.2338508E-3,1.2254796E-1,1.4349685E-2,-1.8386472E-2,-8.270991E-3,7.870949E-3,-2.8393082E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,-1,-1,27,29,-1,-1,31,33,-1,-1,35,-1,37,39,41,43,45,-1,47,-1,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1,-1,53,-1,-1,55,-1,-1,57,-1,-1,-1,-1,-1],"loss_changes":[1.2766813E1,1.6162066E0,3.4715576E0,2.8754663E-1,8.1397784E-1,3.3076508E0,1.8688288E0,1.4934349E-1,0E0,5.2001166E-1,1.8334481E-1,7.4819756E-1,2.892828E-1,0E0,7.4251175E-1,0E0,0E0,5.135765E-1,1.8320996E-1,0E0,0E0,2.1202123E-1,4.2059422E-1,0E0,0E0,1.1468005E-1,0E0,2.4063969E-1,1.3473466E-1,1.7280205E-1,1.5899782E-1,1.8097246E-1,0E0,1.9190025E-1,0E0,0E0,0E0,1.5314889E-1,0E0,0E0,0E0,1.2229282E-1,0E0,0E0,0E0,0E0,0E0,1.0284817E-1,0E0,0E0,1.77068E-1,0E0,0E0,1.0233094E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,21,21,22,22,25,25,27,27,28,28,29,29,30,30,31,31,33,33,37,37,41,41,47,47,50,50,53,53],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,-1,-1,28,30,-1,-1,32,34,-1,-1,36,-1,38,40,42,44,46,-1,48,-1,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1,-1,54,-1,-1,56,-1,-1,58,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,6.205023E3,1.2852292E7,1.4096699E9,9.163994E6,1.417988E6,1.44E2,1.6445398E-3,1.8523391E6,6.088569E6,4.1331047E-1,8.42E2,4.0619567E-2,2.0531621E4,-1.7678207E-2,-1.1250196E-2,2.889654E5,3.107511E6,1.5310563E-2,2.8128107E-3,1.99218E7,2.8756912E8,-1.8729233E-3,-3.2017346E-2,5.3140095E1,-5.276306E-3,7.164074E8,4.037E3,4.79E2,1E0,2.0014305E5,1.1678142E-3,1E0,-9.9177295E-3,5.0118715E-3,2.0533249E-2,5.1698097E1,-2.2793964E-3,-8.060371E-4,8.888147E-3,7.8367496E-1,3.734503E-3,1.0144497E-2,3.6834143E-4,8.842481E-3,1.995166E-3,4.3628708E7,1.704185E-2,-3.548124E-3,9.284744E2,-6.882288E-3,-1.2338508E-3,2.7246006E7,1.4349685E-2,-1.8386472E-2,-8.270991E-3,7.870949E-3,-2.8393082E-3],"split_indices":[2,1,63,9,7,58,40,0,0,62,56,46,0,0,4,0,0,39,1,0,0,56,1,0,0,67,0,7,10,8,24,44,0,27,0,0,0,59,0,0,0,50,0,0,0,0,0,1,0,0,4,0,0,56,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,2.77E2,1.52E2,6.9E1,2.08E2,1.22E2,3E1,6.6E1,3E0,1.97E2,1.1E1,1.17E2,5E0,1.1E1,1.9E1,1.5E1,5.1E1,1.35E2,6.2E1,4E0,7E0,7E1,4.7E1,1E0,4E0,1.6E1,3E0,1.22E2,1.3E1,4.2E1,2E1,5.3E1,1.7E1,4.6E1,1E0,2E0,1.4E1,1.02E2,2E1,9E0,4E0,3.5E1,7E0,4E0,1.6E1,4.1E1,1.2E1,1.7E1,2.9E1,1.6E1,8.6E1,1.7E1,1.8E1,1.1E1,6E0,4E0,8.2E1,9E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[1.6460314E-2,-1.12950146E-1,2.4792863E-1,-2.3830414E-1,-6.663119E-2,2.7976358E-1,-3.6007088E-1,-2.4628015E-1,1.4269581E-3,-8.325595E-2,9.582471E-3,1.3383484E-1,3.9665776E-1,-2.1697354E-2,-2.4357599E-3,-1.252446E-2,-6.42011E-3,-1.4199358E-1,-3.6915008E-2,3.738994E-2,1.845521E-1,7.7193224E-1,3.25644E-1,-1.6218081E-1,2.3314863E-4,-9.5816515E-2,3.2149043E-2,6.26368E-3,-1.7731767E-2,1.1241338E-1,2.3199545E-1,4.0750615E-2,1.1543986E-2,4.2542E-1,1.817545E-1,-2.81308E-3,-1.8152498E-1,-1.0833606E-1,4.786495E-3,8.294076E-2,-4.818557E-2,-4.8543094E-3,2.8103208E-3,1.7760578E-4,8.377075E-3,1.2319657E-2,1.4500057E-3,5.621594E-1,3.4674025E-1,2.2644003E-1,-7.890427E-3,-2.2247893E-1,-1.0418895E-1,-1.1419309E-2,-4.317731E-3,1.01291016E-1,-5.827535E-3,-4.9545537E-3,2.301326E-3,2.9407762E-2,8.444608E-3,1.7447751E-2,3.254179E-3,1.2439662E-2,3.024512E-3,-1.10984305E-2,3.7983435E-3,-1.3265721E-2,-2.5289787E-3,9.71424E-3,2.8405474E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,-1,-1,-1,23,25,27,29,31,33,35,-1,37,39,-1,41,43,45,-1,-1,47,49,-1,51,53,-1,55,57,-1,-1,-1,-1,-1,-1,59,61,63,-1,65,67,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2698724E1,1.5634263E0,3.0427914E0,1.6352177E-1,9.057813E-1,2.4203558E0,2.1045578E-1,9.261942E-2,0E0,5.092083E-1,0E0,3.1593335E-1,1.8805981E0,0E0,0E0,0E0,0E0,2.4689245E-1,4.3835396E-1,1.2646945E-1,1.21088386E-1,3.140211E-1,9.1329145E-1,1.3509548E-1,0E0,1.5325725E-1,2.0801316E-1,0E0,1.08916216E-1,1.2964542E-1,1.2377572E-1,0E0,0E0,2.3542166E-1,5.026977E-1,0E0,1.7002034E-1,1.07062936E-1,0E0,1.2900138E-1,1.12234294E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.612277E-1,1.2175679E-1,1.3398838E-1,0E0,1.3681352E-1,1.8988836E-1,0E0,0E0,1.11750185E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,33,33,34,34,36,36,37,37,39,39,40,40,47,47,48,48,49,49,51,51,52,52,55,55],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,-1,-1,-1,24,26,28,30,32,34,36,-1,38,40,-1,42,44,46,-1,-1,48,50,-1,52,54,-1,56,58,-1,-1,-1,-1,-1,-1,60,62,64,-1,66,68,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,7.41718E5,1.4246185E3,1.2852292E7,1.4777102E9,1.3590209E7,1.6857977E4,1.7367111E6,1.4269581E-3,7.3822034E2,9.582471E-3,7.497444E4,1.417988E6,-2.1697354E-2,-2.4357599E-3,-1.252446E-2,-6.42011E-3,7.164074E8,9.69E2,5.4629724E10,2.1722E4,2.231E3,5.24228E5,5.1698097E1,2.3314863E-4,7.02E2,2.58E2,6.26368E-3,1.3664E4,1.5029658E-1,1.2083983E7,4.0750615E-2,1.1543986E-2,2.1207E4,1.74E2,-2.81308E-3,5.175E3,9.5751E4,4.786495E-3,1.9878894E5,1.5660614E7,-4.8543094E-3,2.8103208E-3,1.7760578E-4,8.377075E-3,1.2319657E-2,1.4500057E-3,2.8184534E7,5.30113E5,6.029086E10,-7.890427E-3,5.54E2,1E0,-1.1419309E-2,-4.317731E-3,2.5159248E1,-5.827535E-3,-4.9545537E-3,2.301326E-3,2.9407762E-2,8.444608E-3,1.7447751E-2,3.254179E-3,1.2439662E-2,3.024512E-3,-1.10984305E-2,3.7983435E-3,-1.3265721E-2,-2.5289787E-3,9.71424E-3,2.8405474E-3],"split_indices":[2,1,69,9,7,1,44,62,0,63,0,39,40,0,0,0,0,7,0,42,2,0,9,59,0,8,3,0,2,49,1,0,0,9,3,0,2,9,0,44,43,0,0,0,0,0,0,56,2,5,0,3,13,0,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.22E2,2.71E2,1.51E2,7.2E1,1.99E2,1.44E2,7E0,7E1,2E0,1.88E2,1.1E1,6.5E1,7.9E1,5E0,2E0,6E1,1E1,8.2E1,1.06E2,2.3E1,4.2E1,1.1E1,6.8E1,7.2E1,1E1,5.7E1,4.9E1,8E0,1.5E1,1.8E1,2.4E1,9E0,2E0,3.9E1,2.9E1,1.2E1,6E1,5.4E1,3E0,3E1,1.9E1,7E0,8E0,7E0,1.1E1,2.1E1,3E0,1.2E1,2.7E1,2.6E1,3E0,3.8E1,2.2E1,5E0,4.9E1,2.8E1,2E0,1.2E1,7E0,1E1,2E0,2.5E1,2E0,2.1E1,5E0,3.7E1,1E0,4E0,1.8E1,7E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[1.4323846E-2,-1.1700679E-1,2.4492225E-1,-2.3040165E-1,-6.278172E-2,1.7253006E-1,4.652621E-1,-2.512557E-1,-3.8596839E-3,-7.9145044E-2,7.447398E-3,2.116318E-1,-1.0486386E-1,3.645495E-2,3.2641846E-1,-4.059103E-3,-2.619132E-1,-1.4559506E-1,-3.765074E-2,1.3530244E-1,3.132162E-1,1.4793543E-2,-1.9252089E-1,3.9799008E-1,-6.8094775E-2,-1.2819582E-2,-1.1196795E-3,1.2054421E-3,-1.602535E-1,-7.0585445E-2,1.07497744E-1,6.639628E-2,1.7344822E-1,2.2420815E-1,1.889114E-2,-1.326212E-2,3.2905561E-3,8.949205E-3,2.2015957E-2,-7.6079657E-3,7.0904074E-3,-2.0072457E-1,-9.118335E-2,-8.309224E-2,3.021626E-3,9.363197E-3,-3.7636677E-5,7.563701E-4,7.5985347E-3,-4.153158E-3,8.727047E-3,4.646315E-3,1.4262413E-2,-1.2368179E-2,-1.1677638E-1,-7.928418E-3,-1.1536531E-4,-1.0425753E-1,-8.908076E-4,-1.5882594E-3,-9.859032E-3,-2.7415594E-3,-6.845991E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,39,-1,-1,-1,41,43,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,53,55,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,59,-1,-1,61,-1,-1,-1,-1,-1],"loss_changes":[1.2687636E1,1.6272762E0,2.344263E0,2.4787092E-1,6.639964E-1,1.2803969E0,1.2841597E0,1.1058378E-1,0E0,4.6325874E-1,0E0,7.485156E-1,6.050738E-1,0E0,7.8947186E-1,0E0,1.0500765E-1,1.6683292E-1,5.143125E-1,1.4619231E-1,2.2728539E-1,0E0,3.0919904E-1,2.1980405E-1,1.2257524E-1,0E0,0E0,0E0,1.4941788E-1,1.5094605E-1,1.9218853E-1,1.0537019E-1,1.0885835E-1,1.8859124E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.5113497E-1,1.5921612E-1,1.0714698E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2056965E-1,0E0,0E0,1.0180247E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,12,12,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,28,28,29,29,30,30,31,31,32,32,33,33,41,41,42,42,43,43,54,54,57,57],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,40,-1,-1,-1,42,44,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,54,56,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,60,-1,-1,62,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,9.54947E5,6.0500806E3,1.1482625E7,1.4096699E9,3.3503532E7,1.417988E6,7.547656E1,-3.8596839E-3,7.28843E2,7.447398E-3,4.1331047E-1,1.869E4,3.645495E-2,1.5096262E7,-4.059103E-3,1.5990765E6,1.2628019E8,5.5292703E9,7.3E1,1.646359E5,1.4793543E-2,9.704588E6,4.0424395E0,3.677E3,-1.2819582E-2,-1.1196795E-3,1.2054421E-3,2.72E2,1.1762006E9,1.6777562E7,7.228959E6,1.7864888E3,8.4E1,1.889114E-2,-1.326212E-2,3.2905561E-3,8.949205E-3,2.2015957E-2,-7.6079657E-3,7.0904074E-3,4.759E3,2.361581E11,6.7842746E8,3.021626E-3,9.363197E-3,-3.7636677E-5,7.563701E-4,7.5985347E-3,-4.153158E-3,8.727047E-3,4.646315E-3,1.4262413E-2,-1.2368179E-2,5.745967E2,-7.928418E-3,-1.1536531E-4,2.5212479E1,-8.908076E-4,-1.5882594E-3,-9.859032E-3,-2.7415594E-3,-6.845991E-3],"split_indices":[2,1,63,9,7,56,40,65,0,63,0,46,9,0,9,0,58,5,5,8,39,0,9,64,0,0,0,0,3,12,56,1,4,3,0,0,0,0,0,0,0,2,42,7,0,0,0,0,0,0,0,0,0,0,63,0,0,64,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.17E2,2.66E2,1.51E2,8.5E1,1.81E2,1.15E2,3.6E1,7.4E1,1.1E1,1.69E2,1.2E1,1.01E2,1.4E1,1E1,2.6E1,5E0,6.9E1,6.4E1,1.05E2,5.9E1,4.2E1,2E0,1.2E1,2.2E1,4E0,6.7E1,2E0,5E0,5.9E1,8.6E1,1.9E1,2.2E1,3.7E1,2.2E1,2E1,9E0,3E0,6E0,1.6E1,3E0,1E0,3.6E1,2.3E1,7.9E1,7E0,1E1,9E0,1.5E1,7E0,1E0,3.6E1,9E0,1.3E1,2E1,1.6E1,1.2E1,1.1E1,5.9E1,2E1,9E0,7E0,2.8E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-1.3014622E-4,-1.17458045E-1,2.1552275E-1,-1.4048508E-1,4.589497E-2,2.5185066E-1,-5.2106583E-1,-1.1083187E-2,-1.0504382E-1,-6.751095E-3,2.1125782E-1,1.867221E-1,5.8084774E-1,-3.956019E-3,-2.769729E-2,-1.590482E-1,-6.141225E-2,2.1800862E-3,-6.224966E-3,1.31415455E-2,6.061825E-4,6.6899255E-2,2.6279536E-1,6.140414E-1,-2.7735995E-3,-1.9237565E-1,-1.0661416E-1,1.3717448E-2,-1.1979027E-1,1.0701388E-1,-1.5481525E-3,9.338307E-3,3.1419376E-1,3.5614192E-2,1.694403E-2,-2.3424743E-1,-1.1888041E-1,-8.137556E-3,-8.0540945E-4,-3.420114E-2,6.508746E-3,-9.16502E-3,-7.3781714E-2,7.396061E-3,-1.4095662E-4,3.6322528E-1,9.746366E-3,-1.2834384E-2,-4.622741E-3,-2.2475694E-3,-1.0737932E-2,5.019768E-4,-5.558842E-3,1.1059243E-3,-5.231815E-3,1.795659E-2,1.2332688E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,-1,25,27,-1,-1,-1,-1,29,31,33,-1,35,37,39,41,43,-1,-1,45,-1,-1,47,49,-1,-1,51,-1,-1,53,-1,-1,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0196759E1,9.882088E-1,3.9309707E0,7.1999216E-1,2.887144E-1,2.8056307E0,1.3354039E-1,0E0,3.83734E-1,1.7452447E-1,9.827489E-2,1.0306802E0,5.0228596E-1,0E0,0E0,1.0880458E-1,4.1392002E-1,0E0,0E0,0E0,0E0,1.8677013E-1,1.8012619E-1,4.537921E-1,0E0,1.0755706E-1,1.7481938E-1,2.5149992E-1,1.6195017E-1,1.7442822E-1,0E0,0E0,1.307807E-1,0E0,0E0,1.02478385E-1,1.2650013E-1,0E0,0E0,1.1526054E-1,0E0,0E0,1.2064654E-1,0E0,0E0,9.977174E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,15,15,16,16,21,21,22,22,23,23,25,25,26,26,27,27,28,28,29,29,32,32,35,35,36,36,39,39,42,42,45,45],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,-1,26,28,-1,-1,-1,-1,30,32,34,-1,36,38,40,42,44,-1,-1,46,-1,-1,48,50,-1,-1,52,-1,-1,54,-1,-1,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.134E3,2.91E2,1.0434409E3,7.41718E5,1.2740336E5,7.710634E3,3.087E3,-1.1083187E-2,6.631343E2,1.36E2,2.0175627E8,2.0663544E9,1.579228E8,-3.956019E-3,-2.769729E-2,2.72E2,1.423795E7,2.1800862E-3,-6.224966E-3,1.31415455E-2,6.061825E-4,1.4074289E7,6.4945E4,5.192243E6,-2.7735995E-3,4.822E3,2.361581E11,2.327793E9,1.643477E9,2.0097266E-1,-1.5481525E-3,9.338307E-3,2.1845728E7,3.5614192E-2,1.694403E-2,1.4599414E3,5.1665356E2,-8.137556E-3,-8.0540945E-4,1.7359158E-1,6.508746E-3,-9.16502E-3,2.41E2,7.396061E-3,-1.4095662E-4,3.0037847E5,9.746366E-3,-1.2834384E-2,-4.622741E-3,-2.2475694E-3,-1.0737932E-2,5.019768E-4,-5.558842E-3,1.1059243E-3,-5.231815E-3,1.795659E-2,1.2332688E-3],"split_indices":[2,8,69,1,44,63,0,0,63,3,43,12,1,0,0,3,56,0,0,0,0,56,2,40,0,2,42,5,5,49,0,0,54,0,0,66,63,0,0,49,0,0,3,0,0,44,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.01E2,2.6E2,1.41E2,2.28E2,3.2E1,1.35E2,6E0,6.2E1,1.66E2,2.5E1,7E0,1.14E2,2.1E1,1E0,5E0,7.3E1,9.3E1,1.8E1,7E0,5E0,2E0,4.5E1,6.9E1,2E1,1E0,4.3E1,3E1,4.1E1,5.2E1,3.2E1,1.3E1,3.2E1,3.7E1,1.2E1,8E0,2.6E1,1.7E1,1.7E1,1.3E1,3E1,1.1E1,1.9E1,3.3E1,2.2E1,1E1,2.4E1,1.3E1,2E1,6E0,1.1E1,6E0,2E1,1E1,9E0,2.4E1,2.3E1,1E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[1.3459752E-2,-6.2396187E-2,3.4071818E-1,-1.1505193E-1,1.3249138E-1,2.2531469E-1,5.217E-1,-1.088648E-1,-2.9944139E-2,1.6166602E-1,-1.4157725E-2,2.4841699E-1,-1.0473407E-2,3.6042884E-2,3.856999E-1,-1.6413844E-1,-4.7558747E-2,1.184301E-1,2.869506E-1,3.8091567E-1,2.0352483E-1,4.4645205E-1,-6.0918825E-3,-1.9787934E-1,-6.836342E-2,8.206502E-3,-1.15003124E-1,9.259658E-3,7.402378E-2,3.3080918E-3,1.5686106E-2,5.1815836E-3,2.0494469E-2,2.5643087E-3,1.0720015E-2,7.2168205E-3,2.3500774E-2,-2.0482543E-1,1.2186175E-2,-6.6930917E-3,-4.64212E-4,-2.7019318E-2,4.456333E-3,-1.5672982E-1,1.5062074E-2,-6.975984E-3,5.114261E-3,-2.1750079E-1,-3.8276596E-3,-7.8044683E-3,-8.628738E-3,-1.7469566E-1,-6.601274E-4,6.6575273E-3,-2.9161319E-3,-1.1425459E-2,-5.4547135E-3,-2.1699243E-3,2.7148456E-3,-5.3355326E-3,-1.1910475E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,19,-1,-1,21,23,25,27,29,31,33,35,-1,37,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,47,-1,-1,-1,49,-1,51,53,-1,-1,55,-1,-1,57,59,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0422724E1,3.5145683E0,1.532443E0,7.843511E-1,9.6184576E-1,5.68238E-1,6.6114235E-1,8.9695215E-1,0E0,3.3977497E-1,0E0,2.0798516E-1,0E0,0E0,7.118852E-1,4.3888712E-1,4.8235846E-1,1.6349125E-1,1.3146317E-1,9.8574996E-2,1.0786712E-1,1.8310189E-1,0E0,3.7652826E-1,1.5876345E-1,2.1550469E-1,3.1713772E-1,0E0,2.7092397E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4369392E-1,0E0,0E0,0E0,1.2660691E-1,0E0,1.10545635E-1,1.5065356E-1,0E0,0E0,1.8035221E-1,0E0,0E0,1.13576084E-1,1.5572786E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,28,28,37,37,41,41,43,43,44,44,47,47,50,50,51,51],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,20,-1,-1,22,24,26,28,30,32,34,36,-1,38,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,48,-1,-1,-1,50,-1,52,54,-1,-1,56,-1,-1,58,60,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.1576422E9,7.710634E3,7.78623E6,3.38298E6,1.505649E6,1.417988E6,1.732734E6,-2.9944139E-2,2.6421343E9,-1.4157725E-2,2.1207E4,-1.0473407E-2,3.6042884E-2,1.5096262E7,1.0865863E7,1.8071064E7,5.473125E3,8.825362E-2,9.36E2,6.873727E0,2.789355E0,-6.0918825E-3,1.0043E4,1.22E2,8.9824E4,1.5311475E3,9.259658E-3,4.4892816E4,3.3080918E-3,1.5686106E-2,5.1815836E-3,2.0494469E-2,2.5643087E-3,1.0720015E-2,7.2168205E-3,2.3500774E-2,2.0964778E5,1.2186175E-2,-6.6930917E-3,-4.64212E-4,2.9E1,4.456333E-3,6.794917E8,2.6E1,-6.975984E-3,5.114261E-3,1.7626338E6,-3.8276596E-3,-7.8044683E-3,1.133457E6,6E1,-6.601274E-4,6.6575273E-3,-2.9161319E-3,-1.1425459E-2,-5.4547135E-3,-2.1699243E-3,2.7148456E-3,-5.3355326E-3,-1.1910475E-2],"split_indices":[63,7,63,58,58,2,40,1,0,7,0,9,0,0,9,9,56,43,50,0,65,64,0,2,8,11,63,0,44,0,0,0,0,0,0,0,0,39,0,0,0,3,0,7,8,0,0,61,0,0,9,8,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.18E2,3.4E2,7.8E1,2.68E2,7.2E1,4.9E1,2.9E1,2.66E2,2E0,6.8E1,4E0,4.7E1,2E0,9E0,2E1,1.39E2,1.27E2,5.2E1,1.6E1,1E1,3.7E1,1.8E1,2E0,1.02E2,3.7E1,7E1,5.7E1,1.8E1,3.4E1,3E0,1.3E1,2E0,8E0,5E0,3.2E1,3E0,1.5E1,1.01E2,1E0,1.6E1,2.1E1,5E1,2E1,4.3E1,1.4E1,4E0,3E1,9.1E1,1E1,5E0,4.5E1,3.8E1,5E0,5E0,9E0,7.4E1,1.7E1,2.9E1,1.6E1,2.2E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"61","size_leaf_vector":"1"}},{"base_weights":[8.253994E-3,-9.640086E-2,1.9848195E-1,-1.8201551E-1,-5.2849106E-2,1.4352953E-1,5.5948657E-1,-9.76912E-3,-4.8407304E-4,-6.624215E-2,1.1738229E-1,-3.2142338E-1,1.7146777E-1,6.4529204E-1,2.6224044E-3,-1.1977962E-1,-3.076737E-2,-3.6310172E-3,8.451125E-3,-1.5485341E-2,-2.9759603E-2,1.0407008E-1,2.8162915E-1,3.605732E-2,1.5152665E-2,-1.4007208E-1,-1.7845834E-3,-2.6259748E-3,4.3358747E-3,6.6243587E-3,-7.856235E-3,5.152797E-2,1.429276E-1,2.9606342E-1,-9.469361E-3,-1.0681662E-1,-2.0249629E-1,-9.185074E-3,3.7963947E-3,7.5700474E-3,-2.9246581E-3,2.1632814E-1,1.8351583E-2,-1.3707633E-2,-7.714988E-2,-1.4524151E-3,-1.1908074E-2,-2.1793123E-3,2.5097892E-1,-4.5354543E-3,3.8492384E-3,1.3879965E-2,3.9643515E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,25,27,-1,-1,29,-1,31,33,-1,-1,35,-1,-1,-1,-1,-1,37,39,41,-1,43,45,-1,-1,-1,-1,47,-1,-1,49,-1,-1,-1,51,-1,-1,-1,-1],"loss_changes":[8.639418E0,1.0344076E0,2.9747605E0,3.6396194E-1,4.3412024E-1,1.8087587E0,8.3040094E-1,0E0,0E0,3.2678068E-1,1.7509508E-1,7.296546E-1,9.210427E-1,3.7400675E-1,0E0,1.10125184E-1,3.1757945E-1,0E0,0E0,1.3786048E-1,0E0,1.5800929E-1,3.913169E-1,0E0,0E0,9.254706E-2,0E0,0E0,0E0,0E0,0E0,2.6130772E-1,1.5692598E-1,2.4817944E-1,0E0,1.811817E-1,1.3794112E-1,0E0,0E0,0E0,0E0,2.5656652E-1,0E0,0E0,1.0354634E-1,0E0,0E0,0E0,1.3632178E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,16,16,19,19,21,21,22,22,25,25,31,31,32,32,33,33,35,35,36,36,41,41,44,44,48,48],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,26,28,-1,-1,30,-1,32,34,-1,-1,36,-1,-1,-1,-1,-1,38,40,42,-1,44,46,-1,-1,-1,-1,48,-1,-1,50,-1,-1,-1,52,-1,-1,-1,-1],"split_conditions":[1.0043E4,9.50639E5,8.1033794E3,1.2677199E7,1.484798E9,2.4950776E0,1.579228E8,-9.76912E-3,-4.8407304E-4,7.317205E2,2.0532622E3,8.91995E5,4.1331047E-1,5.192243E6,2.6224044E-3,8.702814E6,2.103E3,-3.6310172E-3,8.451125E-3,2.203418E7,-2.9759603E-2,5.52488E1,2.8756912E8,3.605732E-2,1.5152665E-2,8.7E1,-1.7845834E-3,-2.6259748E-3,4.3358747E-3,6.6243587E-3,-7.856235E-3,3.0811954E-2,4.2663252E7,1.2384782E5,-9.469361E-3,2.2794174E2,1.0887379E3,-9.185074E-3,3.7963947E-3,7.5700474E-3,-2.9246581E-3,3.564335E6,1.8351583E-2,-1.3707633E-2,4.61E2,-1.4524151E-3,-1.1908074E-2,-2.1793123E-3,9.82576E5,-4.5354543E-3,3.8492384E-3,1.3879965E-2,3.9643515E-3],"split_indices":[2,1,63,9,7,65,1,0,0,63,4,9,46,40,0,43,0,0,0,1,0,67,1,0,0,8,0,0,0,0,0,49,56,39,0,63,4,0,0,0,0,40,0,0,3,0,0,0,11,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,2.79E2,1.53E2,9.3E1,1.86E2,1.34E2,1.9E1,8.2E1,1.1E1,1.73E2,1.3E1,7E0,1.27E2,1.6E1,3E0,6.8E1,1.05E2,3E0,1E1,4E0,3E0,8E1,4.7E1,1.1E1,5E0,5.4E1,1.4E1,8.8E1,1.7E1,2E0,2E0,3.5E1,4.5E1,4.6E1,1E0,3.7E1,1.7E1,3E0,3.2E1,4.2E1,3E0,2.6E1,2E1,4E0,3.3E1,4E0,1.3E1,3E0,2.3E1,3E1,3E0,1.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.2309924E-2,-9.9078044E-2,2.0768686E-1,-9.544731E-3,-6.749866E-2,1.4146869E-1,5.569574E-1,-1.2833904E-2,-1.18108E-1,1.6635785E-1,-1.5422326E-1,3.6129013E-2,1.7811978E-2,-3.3112265E-2,1.1989215E-1,-1.4206152E-1,2.5556836E-4,5.4961126E-2,2.3277687E-1,-1.812443E-2,9.6980266E-2,-8.1329875E-2,1.1969915E-3,2.0930308E-3,1.0213031E-2,-1.5996441E-1,-1.8281874E-3,2.2024736E-2,9.318855E-3,8.281891E-3,2.8473973E-1,-9.7102544E-4,1.6314365E-2,4.5597443E-4,-4.977805E-3,-1.3000226E-1,-1.0476052E-2,-1.1352107E-2,2.0028036E-3,2.1741809E-1,1.8254498E-2,-1.04577765E-2,-1.00042515E-1,4.7601187E-3,1.2196545E-2,-7.142662E-3,-1.9546624E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,-1,7,9,11,13,15,17,19,-1,-1,21,23,25,-1,27,29,-1,31,33,-1,-1,-1,35,-1,37,-1,-1,39,-1,-1,-1,-1,41,-1,-1,-1,43,-1,-1,45,-1,-1,-1,-1],"loss_changes":[9.53085E0,8.7543106E-1,3.5985694E0,0E0,5.905253E-1,1.0191913E0,6.363292E-1,2.8388244E-1,3.290274E-1,9.120636E-1,6.59935E-1,0E0,0E0,2.573671E-1,9.0772584E-2,1.666292E-1,0E0,2.1747063E-1,1.9079733E-1,0E0,1.7929336E-1,1.05327904E-1,0E0,0E0,0E0,1.1234689E-1,0E0,2.1720137E-1,0E0,0E0,1.7996359E-1,0E0,0E0,0E0,0E0,1.2489432E-1,0E0,0E0,0E0,9.113133E-2,0E0,0E0,1.1990678E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,20,20,21,21,25,25,27,27,30,30,35,35,39,39,42,42],"right_children":[2,4,6,-1,8,10,12,14,16,18,20,-1,-1,22,24,26,-1,28,30,-1,32,34,-1,-1,-1,36,-1,38,-1,-1,40,-1,-1,-1,-1,42,-1,-1,-1,44,-1,-1,46,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.41718E5,8.1033794E3,-9.544731E-3,1.2273383E7,9.064268E2,3.0418E4,1.1576422E9,1.472533E3,2.0663544E9,3.5226266E0,3.6129013E-2,1.7811978E-2,1.849533E6,1E0,6.7204064E8,2.5556836E-4,2.1597655E9,5.9161E4,-1.812443E-2,2.5212732E4,1.99742E5,1.1969915E-3,2.0930308E-3,1.0213031E-2,8.2E1,-1.8281874E-3,9.523215E2,9.318855E-3,8.281891E-3,1.619807E5,-9.7102544E-4,1.6314365E-2,4.5597443E-4,-4.977805E-3,1.6415557E9,-1.0476052E-2,-1.1352107E-2,2.0028036E-3,7.769761E0,1.8254498E-2,-1.04577765E-2,5.4031058E-5,4.7601187E-3,1.2196545E-2,-7.142662E-3,-1.9546624E-3],"split_indices":[2,1,63,0,56,69,10,7,63,12,65,0,0,1,26,7,0,7,2,0,4,40,0,0,0,8,0,63,0,0,39,0,0,0,0,5,0,0,0,65,0,0,49,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.36E2,2.78E2,1.58E2,6.5E1,2.13E2,1.34E2,2.4E1,1.03E2,1.1E2,1.24E2,1E1,1E1,1.4E1,9E1,1.3E1,9.2E1,1.8E1,4.7E1,7.7E1,5E0,5E0,4.9E1,4.1E1,8E0,5E0,7.8E1,1.4E1,3.9E1,8E0,3.8E1,3.9E1,4E0,1E0,1E1,3.9E1,5.4E1,2.4E1,2E0,3.7E1,2.5E1,1.4E1,1.2E1,4.2E1,7E0,1.8E1,2.2E1,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[7.5719873E-3,-8.418221E-2,1.8056096E-1,-1.6911018E-1,-4.585957E-2,1.4180532E-1,4.7314572E-1,-1.9272245E-1,5.512605E-4,-6.398503E-2,1.073697E-1,1.6020614E-1,-2.3467874E-2,1.4986601E-2,2.9256823E-2,-1.3506853E-2,-1.6817333E-1,-1.18783094E-1,-3.650814E-2,-4.9698073E-3,1.6325724E-1,2.4428334E-2,1.47871E-1,-1.5816898E-3,-8.677734E-3,-6.3763335E-2,-1.6149405E-1,-3.38482E-3,2.5152989E-2,3.1068008E-3,1.1612136E-2,-1.4263006E-2,1.6491336E-1,-9.081304E-3,-1.2584443E-3,-1.9123854E-1,1.7838815E-3,4.6713944E-3,-4.591683E-4,-7.7393968E-3,4.698275E-3,1.8197286E-1,3.3215235E-3,-1.0766496E-2,-2.6742178E-3,1.438402E-1,1.3380754E-2,1.4932844E-1,-1.1023582E-2,1.214467E-2,5.2818186E-3,1.6048908E-2,1.4789416E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,-1,-1,23,25,27,-1,29,-1,31,-1,-1,33,35,-1,37,-1,-1,39,41,-1,-1,43,-1,-1,-1,-1,-1,45,47,-1,-1,49,-1,51,-1,-1,-1,-1,-1],"loss_changes":[6.824487E0,9.068053E-1,1.6224627E0,3.7334895E-1,5.4845345E-1,1.6290865E0,1.8583846E-1,1.2302351E-1,0E0,2.5915754E-1,2.6521325E-1,4.8811746E-1,0E0,0E0,0E0,0E0,1.0995364E-1,1.2599951E-1,2.521284E-1,0E0,1.1771724E-1,0E0,3.5327148E-1,0E0,0E0,1.2271392E-1,1.964345E-1,0E0,1.09670535E-1,0E0,0E0,2.3261824E-1,3.1641364E-1,0E0,0E0,1.1732924E-1,0E0,0E0,0E0,0E0,0E0,3.4977508E-1,4.4495785E-1,0E0,0E0,2.5442672E-1,0E0,1.6653834E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,16,16,17,17,18,18,20,20,22,22,25,25,26,26,28,28,31,31,32,32,35,35,41,41,42,42,45,45,47,47],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,-1,-1,24,26,28,-1,30,-1,32,-1,-1,34,36,-1,38,-1,-1,40,42,-1,-1,44,-1,-1,-1,-1,-1,46,48,-1,-1,50,-1,52,-1,-1,-1,-1,-1],"split_conditions":[1.0262E4,9.54947E5,6.001104E9,1.2852292E7,1.1576422E9,6.131644E7,1.03468E6,1.44E2,5.512605E-4,6.631343E2,2.1530056E3,8.504733E7,-2.3467874E-2,1.4986601E-2,2.9256823E-2,-1.3506853E-2,1.3819203E2,5.9E1,1.16E3,-4.9698073E-3,1E0,2.4428334E-2,1.20142E3,-1.5816898E-3,-8.677734E-3,1.533115E6,2.58E2,-3.38482E-3,1.1291153E7,3.1068008E-3,1.1612136E-2,1.9128645E9,4.42384E5,-9.081304E-3,-1.2584443E-3,2.0964778E5,1.7838815E-3,4.6713944E-3,-4.591683E-4,-7.7393968E-3,4.698275E-3,4.23218E3,3.9E1,-1.0766496E-2,-2.6742178E-3,3.5366E4,1.3380754E-2,1.3812E4,-1.1023582E-2,1.214467E-2,5.2818186E-3,1.6048908E-2,1.4789416E-3],"split_indices":[2,1,7,9,7,61,9,0,0,63,4,7,0,0,0,0,44,8,0,0,24,0,63,0,0,1,8,0,56,0,0,7,2,0,0,39,0,0,0,0,0,63,8,0,0,9,0,9,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.8E2,1.48E2,8.6E1,1.94E2,1.32E2,1.6E1,7.6E1,1E1,1.74E2,2E1,1.29E2,3E0,9E0,7E0,1.4E1,6.2E1,5.7E1,1.17E2,4E0,1.6E1,3E0,1.26E2,6E0,5.6E1,2.6E1,3.1E1,7.5E1,4.2E1,8E0,8E0,1.2E1,1.14E2,5E0,2.1E1,2.7E1,4E0,1.3E1,2.9E1,5E0,7E0,1.03E2,1.1E1,2.1E1,6E0,7.6E1,2.7E1,7E0,4E0,1.6E1,6E1,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[5.7571237E-3,-8.279284E-2,1.74609E-1,-2.0526637E-1,-5.4937378E-2,1.22249395E-1,4.105476E-1,-1.2107674E-2,-6.725803E-3,-7.3823154E-2,5.5630174E-2,-4.1556025E-1,1.4812113E-1,4.9056965E-1,-1.8931585E-3,-8.1359275E-2,4.6215057E-3,1.6233522E-1,1.2450149E-2,-2.7548406E-2,-2.9387313E-3,8.539046E-2,2.2321063E-1,1.4613035E-1,5.5539215E-1,-1.0158946E-1,-1.2117313E-2,5.9714494E-4,1.0727873E-2,-7.272103E-3,1.9364789E-3,1.4124447E-1,-2.5875017E-3,2.6475823E-1,7.1809955E-2,1.25909755E-2,-7.7447006E-3,3.355046E-2,1.8338308E-2,-8.6118E-2,-1.6060475E-1,-7.381777E-2,4.0348815E-3,7.5651594E-3,-7.002893E-4,-8.815974E-3,3.7115213E-2,2.2366226E-1,1.864322E-2,7.7520977E-3,-2.9097986E-3,-5.883984E-2,-6.406933E-3,-5.531375E-3,-1.2843943E-2,-8.400866E-4,-1.4730734E-1,6.3973814E-3,-6.521105E-4,8.100163E-3,1.472208E-2,-6.0064737E-3,-1.468769E-3,-1.1260303E-2,-2.4015887E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,19,21,23,-1,25,-1,27,29,-1,-1,31,33,35,37,39,41,-1,-1,-1,-1,43,45,47,49,-1,-1,-1,-1,51,53,55,-1,-1,-1,-1,57,59,-1,-1,-1,61,-1,-1,-1,-1,63,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.5633574E0,9.712517E-1,1.8152122E0,1.2933898E-1,4.9691123E-1,1.800812E0,9.921813E-1,0E0,0E0,2.6765406E-1,1.5924232E-1,3.077016E-1,5.461898E-1,4.3237066E-1,0E0,2.7051997E-1,0E0,9.071344E-2,1.2667853E-1,0E0,0E0,3.2959828E-1,3.2066536E-1,2.2423027E-1,2.0975828E-1,1.231271E-1,2.7459547E-1,0E0,0E0,0E0,0E0,1.1261195E-1,2.0207767E-1,1.2983227E-1,1.6628684E-1,0E0,0E0,0E0,0E0,1.5090626E-1,1.1435413E-1,1.1308588E-1,0E0,0E0,0E0,0E0,1.1501722E-1,1.03184104E-1,0E0,0E0,0E0,1.446931E-1,0E0,0E0,0E0,0E0,9.1258705E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,34,34,39,39,40,40,41,41,46,46,47,47,51,51,56,56],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,20,22,24,-1,26,-1,28,30,-1,-1,32,34,36,38,40,42,-1,-1,-1,-1,44,46,48,50,-1,-1,-1,-1,52,54,56,-1,-1,-1,-1,58,60,-1,-1,-1,62,-1,-1,-1,-1,64,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,5.63321E5,7.3073276E3,1.425966E6,1.1383163E5,2.4950776E0,1.4656219E8,-1.2107674E-2,-6.725803E-3,1.577058E9,9.457892E6,1.2799757E9,3.116E3,6.069097E4,-1.8931585E-3,2.231E3,4.6215057E-3,5.748E3,4.3159333E-1,-2.7548406E-2,-2.9387313E-3,1.535373E6,9.82576E5,2.36316E5,2.3222E4,3.271774E7,6.6088306E2,5.9714494E-4,1.0727873E-2,-7.272103E-3,1.9364789E-3,3.14E2,2.3808822E-2,2.1030048E10,9.677921E7,1.25909755E-2,-7.7447006E-3,3.355046E-2,1.8338308E-2,2.4853801E-2,1E0,5.861E3,4.0348815E-3,7.5651594E-3,-7.002893E-4,-8.815974E-3,4.200502E4,3.5832284E7,1.864322E-2,7.7520977E-3,-2.9097986E-3,4.0208E4,-6.406933E-3,-5.531375E-3,-1.2843943E-2,-8.400866E-4,1.2833E4,6.3973814E-3,-6.521105E-4,8.100163E-3,1.472208E-2,-6.0064737E-3,-1.468769E-3,-1.1260303E-2,-2.4015887E-3],"split_indices":[2,1,63,9,44,65,1,0,0,7,43,7,0,39,0,0,0,2,50,0,0,40,11,2,10,56,63,0,0,0,0,3,49,5,1,0,0,0,0,68,105,10,0,0,0,0,39,43,0,0,0,9,0,0,0,0,10,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,2.87E2,1.5E2,5.2E1,2.35E2,1.24E2,2.6E1,2.8E1,2.4E1,2.01E2,3.4E1,5E0,1.19E2,2.2E1,4E0,1.93E2,8E0,9E0,2.5E1,3E0,2E0,6.6E1,5.3E1,4E0,1.8E1,1.49E2,4.4E1,3E0,6E0,3E0,2.2E1,4E1,2.6E1,4.1E1,1.2E1,3E0,1E0,8E0,1E1,1.2E2,2.9E1,2.7E1,1.7E1,3.6E1,4E0,4E0,2.2E1,3.3E1,8E0,7E0,5E0,7.8E1,4.2E1,2.2E1,7E0,1.6E1,1.1E1,7E0,1.5E1,2.2E1,1.1E1,2.2E1,5.6E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[6.334446E-3,-6.709365E-2,2.0595177E-1,-1.098287E-1,3.6959175E-2,1.4911743E-1,4.243347E-1,-1.31260175E-2,-9.7268395E-2,-3.625762E-3,8.649609E-3,1.7295188E-1,-7.875959E-3,2.4724762E-1,3.209084E-2,-9.18327E-2,-2.0737693E-2,-4.278822E-2,4.7891065E-3,1.21082395E-1,2.3528484E-1,1.3736628E-2,-5.189092E-4,-1.0272508E-1,7.978333E-3,-7.996363E-4,-1.303455E-1,6.828244E-2,9.270643E-3,2.776419E-1,6.322973E-3,-1.23869084E-1,-3.845706E-2,8.339725E-3,-8.9590217E-4,-1.259333E-2,-2.698268E-3,4.847069E-3,-9.845005E-4,3.948839E-3,3.093196E-1,-9.788824E-4,-1.3445452E-1,-5.4292757E-2,6.9376472E-3,1.5484887E-2,-1.8697426E-4,-6.6705802E-3,2.9025406E-3,-2.2707504E-4,-4.945463E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,-1,15,17,-1,19,-1,21,-1,23,-1,25,-1,27,29,-1,-1,31,33,-1,35,37,-1,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,45,-1,47,49,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.2139487E0,1.3825374E0,1.3516526E0,4.2966866E-1,5.3343433E-1,7.173438E-1,8.438592E-1,0E0,3.3768737E-1,2.9743397E-1,0E0,2.4956465E-1,0E0,1.6011786E-1,0E0,2.226044E-1,0E0,1.18633255E-1,0E0,1.7598033E-1,1.2870455E-1,0E0,0E0,2.4527073E-1,9.69433E-2,0E0,1.03331804E-1,9.0828806E-2,0E0,1.3494134E-1,0E0,1.4823103E-1,1.4469486E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.05345964E-1,0E0,1.3786316E-1,1.0700071E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,11,11,13,13,15,15,17,17,19,19,20,20,23,23,24,24,26,26,27,27,29,29,31,31,32,32,40,40,42,42,43,43],"right_children":[2,4,6,8,10,12,14,-1,16,18,-1,20,-1,22,-1,24,-1,26,-1,28,30,-1,-1,32,34,-1,36,38,-1,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,46,-1,48,50,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4096699E9,9.570385E4,7.710634E3,1.44E2,1.5271514E5,4.1805066E2,2.2876814E1,-1.31260175E-2,5.825042E6,2.4350447E3,8.649609E-3,5.9161E4,-7.875959E-3,1.354316E6,3.209084E-2,2.1592189E5,-2.0737693E-2,1.716E3,4.7891065E-3,7.8367496E-1,2.3566682E7,1.3736628E-2,-5.189092E-4,6.97E3,1.33E2,-7.996363E-4,4.665693E6,2.82015E-1,9.270643E-3,4.2464733E-2,6.322973E-3,2E1,1.3664E4,8.339725E-3,-8.9590217E-4,-1.259333E-2,-2.698268E-3,4.847069E-3,-9.845005E-4,3.948839E-3,2.7674397E5,-9.788824E-4,7.8E2,1.858736E-3,6.9376472E-3,1.5484887E-2,-1.8697426E-4,-6.6705802E-3,2.9025406E-3,-2.2707504E-4,-4.945463E-3],"split_indices":[7,44,63,0,39,69,65,0,58,63,0,2,0,2,0,39,0,0,0,50,54,0,0,2,3,0,1,50,0,49,0,10,2,0,0,0,0,0,0,0,44,0,8,68,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.22E2,3.09E2,1.13E2,2.19E2,9E1,9.1E1,2.2E1,1.4E1,2.05E2,7.1E1,1.9E1,8.5E1,6E0,1.4E1,8E0,2.03E2,2E0,5.2E1,1.9E1,4.8E1,3.7E1,1.2E1,2E0,1.83E2,2E1,4.1E1,1.1E1,2.9E1,1.9E1,2.5E1,1.2E1,1.37E2,4.6E1,2E0,1.8E1,3E0,8E0,2.1E1,8E0,4E0,2.1E1,1.3E1,1.24E2,4.3E1,3E0,2E1,1E0,1.21E2,3E0,2.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[3.958878E-4,-9.049381E-2,1.429129E-1,-1.0628568E-1,5.875773E-2,6.821472E-2,2.9650158E-1,-1.01069614E-1,-2.3133373E-2,-8.643391E-3,4.118682E-3,3.116245E-2,8.697889E-3,2.602956E-1,2.8948769E-2,-1.8633522E-1,-7.96709E-2,-5.485605E-2,7.013549E-2,1.468219E-2,7.468545E-3,-1.183665E-2,-6.2757013E-3,-8.373672E-2,1.6631767E-2,-6.474827E-3,-4.5076752E-4,3.9910328E-2,7.644798E-3,-8.8417254E-2,5.157914E-3,-1.6016418E-3,4.341619E-3,-1.0034705E-1,-2.2639764E-2,-7.426107E-3,-3.8022688E-3,-1.0103377E-2,2.9611742E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,-1,19,-1,21,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,31,-1,33,-1,-1,-1,35,37,-1,-1,-1,-1],"loss_changes":[5.621724E0,6.308391E-1,1.9186764E0,4.2469072E-1,1.8731627E-1,4.8013532E-1,4.5944405E-1,4.194033E-1,0E0,0E0,0E0,2.9885882E-1,0E0,2.0236969E-1,0E0,1.2634361E-1,3.6338532E-1,1.0145577E-1,1.5959886E-1,0E0,0E0,0E0,0E0,1.8102026E-1,0E0,0E0,0E0,1.793248E-1,0E0,1.4446938E-1,0E0,0E0,0E0,1.6814148E-1,1.6488716E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,13,13,15,15,16,16,17,17,18,18,23,23,27,27,29,29,33,33,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,-1,20,-1,22,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,32,-1,34,-1,-1,-1,36,38,-1,-1,-1,-1],"split_conditions":[9.570385E4,1.577058E9,6.0891E4,5.2530425E6,2.9648642E1,1.5271514E5,6.15E2,5.42218E5,-2.3133373E-2,-8.643391E-3,4.118682E-3,2.3441157E3,8.697889E-3,9.118089E-1,2.8948769E-2,3.564E3,3.9417793E3,3.1739944E-1,3.53E2,1.468219E-2,7.468545E-3,-1.183665E-2,-6.2757013E-3,1.3664E4,1.6631767E-2,-6.474827E-3,-4.5076752E-4,2.4350447E3,7.644798E-3,2.0964778E5,5.157914E-3,-1.6016418E-3,4.341619E-3,2.53383E5,2.1115233E2,-7.426107E-3,-3.8022688E-3,-1.0103377E-2,2.9611742E-4],"split_indices":[44,7,2,58,67,39,3,1,0,0,0,4,0,68,0,2,63,53,8,0,0,0,0,2,0,0,0,63,0,39,0,0,0,9,63,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.32E2,2.64E2,1.68E2,2.39E2,2.5E1,1.14E2,5.4E1,2.37E2,2E0,2E0,2.3E1,8.7E1,2.7E1,5E1,4E0,4.6E1,1.91E2,2.7E1,6E1,3.3E1,1.7E1,2E1,2.6E1,1.9E2,1E0,9E0,1.8E1,4.6E1,1.4E1,1.86E2,4E0,1.9E1,2.7E1,1.57E2,2.9E1,4.1E1,1.16E2,3E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-2.886228E-3,-9.534308E-2,1.2928593E-1,-8.710181E-2,-2.3230044E-2,5.839493E-2,3.032834E-1,-1.16576985E-1,1.8984744E-2,1.5259789E-2,8.178304E-3,2.453788E-1,3.0175377E-2,-1.376634E-1,2.577944E-2,1.5186747E-2,7.5520235E-3,-2.6724278E-3,4.6813272E-2,2.9158977E-1,-1.035907E-1,-1.8065098E-1,-1.0182423E-1,-1.3024838E-3,2.0721959E-1,2.1289608E-2,-6.2686703E-3,2.1451943E-2,1.2992091E-1,4.0300035E-1,9.150974E-3,-1.030059E-2,3.8452118E-3,-2.2243767E-3,-1.935747E-1,-1.140823E-1,-3.5952748E-4,1.4184376E-2,8.689572E-4,2.427064E-6,5.6340476E-3,-1.5139738E-3,3.9421236E-3,8.858277E-3,7.3923555E-4,1.2933336E-2,2.5496488E-2,-1.0953653E-2,-6.6458657E-3,-1.320284E-1,-1.9543322E-3,-1.52745405E-2,-5.642543E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,21,23,-1,25,-1,27,29,31,33,35,-1,37,39,-1,41,43,45,-1,-1,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,51,-1,-1,-1],"loss_changes":[5.2548013E0,7.7358365E-1,2.1655593E0,7.808161E-1,0E0,6.161318E-1,8.1620836E-1,5.8889747E-1,1.8572836E-1,2.1170092E-1,0E0,7.5588584E-1,0E0,2.4351382E-1,2.5599822E-1,0E0,1.0582596E-1,0E0,1.3421474E-1,3.677938E-1,1.4137363E-1,1.22641325E-1,1.0894495E-1,0E0,9.660724E-2,1.0273509E-1,0E0,1.6841185E-1,9.267953E-2,1.6700983E-1,0E0,0E0,0E0,0E0,1.02699995E-1,1.048466E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.3613427E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,14,14,16,16,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,29,29,34,34,35,35,49,49],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,22,24,-1,26,-1,28,30,32,34,36,-1,38,40,-1,42,44,46,-1,-1,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,52,-1,-1,-1],"split_conditions":[9.570385E4,7.339209E6,9.3058E4,7.164074E8,-2.3230044E-2,1.5271514E5,4.2875492E1,2.1592189E5,1.3812E4,2.4237612E3,8.178304E-3,1.852084E7,3.0175377E-2,4.822E3,4.466882E6,1.5186747E-2,2.3566682E7,-2.6724278E-3,2.58E2,6.54121E6,3.677E3,5.8955963E1,7.2008003E9,-1.3024838E-3,5.599E3,2.2201815E9,-6.2686703E-3,2.6201914E3,2.0175627E8,2.20229E5,9.150974E-3,-1.030059E-2,3.8452118E-3,-2.2243767E-3,1.237883E6,1.2472753E5,-3.5952748E-4,1.4184376E-2,8.689572E-4,2.427064E-6,5.6340476E-3,-1.5139738E-3,3.9421236E-3,8.858277E-3,7.3923555E-4,1.2933336E-2,2.5496488E-2,-1.0953653E-2,-6.6458657E-3,8.412805E0,-1.9543322E-3,-1.52745405E-2,-5.642543E-3],"split_indices":[44,58,2,7,0,39,65,39,9,4,0,9,0,2,1,0,54,0,8,40,0,61,5,0,0,7,0,63,43,2,0,0,0,0,9,39,0,0,0,0,0,0,0,0,0,0,0,0,0,64,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.28E2,2.52E2,1.76E2,2.48E2,4E0,1.26E2,5E1,1.94E2,5.4E1,9.2E1,3.4E1,4.4E1,6E0,1.69E2,2.5E1,1E0,5.3E1,2.8E1,6.4E1,3.9E1,5E0,7.5E1,9.4E1,2E1,5E0,4.9E1,4E0,5E1,1.4E1,1.7E1,2.2E1,3E0,2E0,7E0,6.8E1,8.3E1,1.1E1,3E0,2E0,4.1E1,8E0,2.7E1,2.3E1,9E0,5E0,1E1,7E0,3.9E1,2.9E1,6.6E1,1.7E1,3E0,6.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.4012745E-2,-6.711745E-2,1.5483089E-1,-1.4794615E-1,-4.364167E-2,1.0707954E-1,3.1740677E-1,-1.9720963E-1,-4.4748043E-3,-5.328833E-2,1.0367321E-1,1.1593221E-1,-1.3071883E-2,-2.87297E-3,3.6975956E-1,-1.0115409E-2,2.810029E-4,-9.175711E-2,-2.932157E-2,7.18496E-3,-4.9018185E-3,8.3147585E-2,1.8738796E-1,2.3839757E-2,1.2197185E-2,-1.1675892E-1,-6.3186977E-4,-4.6943266E-2,2.9600246E-3,5.9955724E-2,8.356054E-3,5.6532226E-3,1.1591133E-2,-1.6836257E-1,-5.074077E-2,-2.7690725E-2,-4.5082434E-3,7.8873664E-2,-4.2682327E-3,-9.44182E-3,-1.8674718E-3,-3.6845088E-4,-9.205284E-3,2.876739E-3,-4.742454E-2,1.461697E-1,1.8562167E-3,-4.1725454E-3,-2.816455E-4,1.2624363E-3,9.040754E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,-1,23,-1,-1,25,27,-1,-1,29,31,-1,-1,33,-1,35,-1,37,-1,-1,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,47,49,-1,-1,-1,-1,-1],"loss_changes":[4.7516026E0,4.9448764E-1,1.1460211E0,1.3903177E-1,2.982704E-1,4.3306518E-1,6.943791E-1,9.0917945E-2,0E0,1.7617214E-1,1.411709E-1,2.5850916E-1,0E0,0E0,3.168025E-1,0E0,0E0,1.4372474E-1,1.9766162E-1,0E0,0E0,1.6430944E-1,1.0888755E-1,0E0,0E0,1.8223667E-1,0E0,9.067969E-2,0E0,1.9639128E-1,0E0,0E0,0E0,1.102556E-1,1.5747833E-1,1.313079E-1,0E0,1.5576252E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0078707E-1,1.0888356E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,14,14,17,17,18,18,21,21,22,22,25,25,27,27,29,29,33,33,34,34,35,35,37,37,44,44,45,45],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,-1,24,-1,-1,26,28,-1,-1,30,32,-1,-1,34,-1,36,-1,38,-1,-1,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,48,50,-1,-1,-1,-1,-1],"split_conditions":[9.508E3,7.41718E5,6.651788E3,5.72911E5,1.4777102E9,4.673502E7,5.3140095E1,2.4434292E7,-4.4748043E-3,6.510362E2,1.87E2,1.646359E5,-1.3071883E-2,-2.87297E-3,1.681178E6,-1.0115409E-2,2.810029E-4,3.312442E6,2.103E3,7.18496E-3,-4.9018185E-3,4.0297854E3,1.8618166E-1,2.3839757E-2,1.2197185E-2,2.72E2,-6.3186977E-4,2.3790412E7,2.9600246E-3,3.5301748E7,8.356054E-3,5.6532226E-3,1.1591133E-2,1.0368186E-1,3.907182E-4,1.0878975E4,-4.5082434E-3,1.5022098E5,-4.2682327E-3,-9.44182E-3,-1.8674718E-3,-3.6845088E-4,-9.205284E-3,2.876739E-3,4.3159333E-1,6.3649645E4,1.8562167E-3,-4.1725454E-3,-2.816455E-4,1.2624363E-3,9.040754E-3],"split_indices":[2,1,63,9,7,59,67,12,0,63,3,39,0,0,40,0,0,9,0,0,0,63,46,0,0,3,0,56,0,56,0,0,0,49,49,59,0,44,0,0,0,0,0,0,50,39,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.14E2,2.63E2,1.51E2,5.8E1,2.05E2,1.18E2,3.3E1,2.9E1,2.9E1,1.93E2,1.2E1,1.16E2,2E0,4E0,2.9E1,2.7E1,2E0,7.3E1,1.2E2,1E1,2E0,8.1E1,3.5E1,1.2E1,1.7E1,5.5E1,1.8E1,1.01E2,1.9E1,6.6E1,1.5E1,1.7E1,1.8E1,3E1,2.5E1,7.3E1,2.8E1,5.9E1,7E0,2.4E1,6E0,2E1,5E0,1.3E1,6E1,2.1E1,3.8E1,3E1,3E1,6E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[7.3649055E-5,-6.7417115E-2,1.2740573E-1,-8.9655906E-2,2.640289E-2,1.5139209E-1,-1.8339485E-2,-1.2938653E-1,-3.0163968E-2,6.493344E-3,7.7322675E-3,5.986428E-2,2.3479947E-1,-6.549421E-3,-1.2995626E-3,-1.0512968E-1,2.1244267E-2,8.078442E-3,-7.5645454E-4,1.1874688E-1,4.1379915E-3,2.2965666E-2,1.9391589E-1,-1.3117789E-3,-1.7363761E-1,-1.4366511E-3,3.5159662E-3,8.205421E-3,2.3002825E-3,-5.4199435E-3,1.6889127E-3,2.3765701E-1,1.4473383E-2,-5.1659048E-3,-1.3347734E-2,1.7143521E-2,1.9727995E-1,4.5693247E-3,-1.0147758E-2,3.8967473E-3,1.0803555E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,-1,-1,23,25,-1,-1,27,29,-1,31,-1,33,-1,-1,-1,-1,-1,-1,35,37,-1,-1,-1,39,-1,-1,-1,-1],"loss_changes":[3.721085E0,5.9283066E-1,1.8990042E0,5.373732E-1,1.4742799E-1,1.0852532E0,0E0,1.0584712E-1,3.6064196E-1,1.8086E-1,0E0,2.2920024E-1,6.620774E-1,0E0,0E0,1.9736576E-1,1.5293196E-1,0E0,0E0,1.1924663E-1,1.4008135E-1,0E0,5.145955E-1,0E0,1.08409345E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.8918252E-1,2.7867958E-1,0E0,0E0,0E0,1.22681856E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,15,15,16,16,19,19,20,20,22,22,24,24,31,31,32,32,36,36],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,-1,-1,24,26,-1,-1,28,30,-1,32,-1,34,-1,-1,-1,-1,-1,-1,36,38,-1,-1,-1,40,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.164074E8,2.3033237E3,1.947899E6,1E0,1.3590209E7,-1.8339485E-2,1.5990765E6,7.317205E2,2.409091E0,7.7322675E-3,7.8920895E6,1E0,-6.549421E-3,-1.2995626E-3,1.7377544E7,9.0931484E4,8.078442E-3,-7.5645454E-4,3.311258E-2,3.0811954E-2,2.2965666E-2,1.2852292E7,-1.3117789E-3,6.4E1,-1.4366511E-3,3.5159662E-3,8.205421E-3,2.3002825E-3,-5.4199435E-3,1.6889127E-3,2.1207E4,1.92144E5,-5.1659048E-3,-1.3347734E-2,1.7143521E-2,3.558942E0,4.5693247E-3,-1.0147758E-2,3.8967473E-3,1.0803555E-2],"split_indices":[2,7,69,1,26,1,0,58,63,69,0,56,25,0,0,56,39,0,0,68,49,0,9,0,8,0,0,0,0,0,0,9,2,0,0,0,64,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.82E2,1.49E2,2.28E2,5.4E1,1.43E2,6E0,1.36E2,9.2E1,4.8E1,6E0,6.9E1,7.4E1,1.26E2,1E1,3.7E1,5.5E1,5E0,4.3E1,3.3E1,3.6E1,9E0,6.5E1,1.8E1,1.9E1,2.8E1,2.7E1,1.8E1,1.5E1,7E0,2.9E1,5.2E1,1.3E1,1.3E1,6E0,1.1E1,4.1E1,1E1,3E0,9E0,3.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[4.409343E-3,-5.119107E-2,1.4947702E-1,-8.592644E-2,3.111503E-2,1.24693014E-1,1.8638229E-2,-8.089702E-2,-2.0658543E-2,-2.143517E-2,1.23314954E-1,1.356314E-1,-6.620362E-3,-9.909139E-3,-7.14612E-2,-4.258351E-2,5.147111E-3,2.9579352E-3,8.812114E-3,9.072349E-2,2.1082945E-1,-7.5074725E-2,6.367453E-3,-9.4848033E-4,-6.1840005E-3,-8.814795E-4,1.0713643E-1,2.3417406E-1,-7.405186E-3,-4.893755E-2,-1.07854664E-1,-5.306315E-3,5.556852E-3,1.9840766E-1,2.7937364E-2,-6.0250074E-2,4.46316E-3,-5.991995E-3,-8.171363E-4,1.5713807E-2,7.95619E-3,-3.6291524E-3,1.7300431E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,-1,-1,21,23,-1,-1,-1,25,27,29,-1,-1,-1,-1,31,33,-1,35,37,-1,-1,39,-1,41,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5808113E0,9.2039114E-1,6.86013E-1,3.592775E-1,4.6777523E-1,3.443792E-1,0E0,2.5297773E-1,0E0,1.7409275E-1,1.2018567E-1,3.4896445E-1,0E0,0E0,1.6547501E-1,1.04097985E-1,0E0,0E0,0E0,1.2742907E-1,3.7807238E-1,1.720742E-1,0E0,0E0,0E0,0E0,1.365835E-1,3.6110973E-1,0E0,1.9416872E-1,1.4385676E-1,0E0,0E0,9.449065E-2,0E0,1.6877192E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,14,14,15,15,19,19,20,20,21,21,26,26,27,27,29,29,30,30,33,33,35,35],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,-1,-1,22,24,-1,-1,-1,26,28,30,-1,-1,-1,-1,32,34,-1,36,38,-1,-1,40,-1,42,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4096699E9,9.570385E4,1.6828056E1,5.825042E6,2.1700479E3,1.505649E6,1.8638229E-2,2.571E3,-2.0658543E-2,1.5271514E5,1.0545219E5,9.9179E4,-6.620362E-3,-9.909139E-3,1.3664E4,1.0170089E5,5.147111E-3,2.9579352E-3,8.812114E-3,4.3871765E0,5.22E2,5.4849293E-3,6.367453E-3,-9.4848033E-4,-6.1840005E-3,-8.814795E-4,1.4698442E6,2.0531621E4,-7.405186E-3,1.2833E4,7.164074E8,-5.306315E-3,5.556852E-3,2.0159504E1,2.7937364E-2,1.38E2,4.46316E-3,-5.991995E-3,-8.171363E-4,1.5713807E-2,7.95619E-3,-3.6291524E-3,1.7300431E-3],"split_indices":[7,44,46,58,63,2,0,2,0,39,39,2,0,0,2,39,0,0,0,64,3,50,0,0,0,0,56,4,0,10,7,0,0,67,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,3.2E2,1.22E2,2.25E2,9.5E1,1.12E2,1E1,2.23E2,2E0,6.1E1,3.4E1,1.08E2,4E0,1.4E1,2.09E2,5.3E1,8E0,1.8E1,1.6E1,6.9E1,3.9E1,2.06E2,3E0,4.3E1,1E1,9E0,6E1,3.7E1,2E0,1.16E2,9E1,2E0,5.8E1,3.5E1,2E0,1.08E2,8E0,7.5E1,1.5E1,5E0,3E1,9.3E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[6.29794E-3,-4.252283E-2,2.2656603E-1,-8.5984044E-2,5.1770955E-2,6.926793E-3,3.6296326E-1,-7.881879E-2,-2.2401195E-2,-1.3284435E-2,1.2218921E-1,2.6191533E-2,2.3959161E-1,-1.2663084E-1,-4.2537335E-2,1.3670921E-2,-7.4254745E-3,8.255199E-3,1.3168041E-3,1.4209643E-2,-7.1695508E-3,-1.325349E-1,1.0504521E-2,-5.1321298E-2,3.8786195E-3,-1.6520731E-3,3.2293466E-3,-1.6639894E-1,-2.8450747E-3,-5.8138273E-3,-3.5460316E-2,-8.509523E-3,-9.467182E-4,-6.4358825E-4,-5.0311303E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,-1,19,21,23,25,-1,-1,-1,-1,-1,27,-1,29,-1,-1,-1,31,-1,-1,33,-1,-1,-1,-1],"loss_changes":[4.3863726E0,1.3732244E0,7.693682E-1,5.920106E-1,4.8721844E-1,0E0,4.873426E-1,3.8665366E-1,0E0,2.1690728E-1,2.4014968E-1,0E0,4.5246208E-1,2.2772646E-1,1.4553703E-1,1.2746148E-1,0E0,0E0,0E0,0E0,0E0,2.2716689E-1,0E0,1.3206926E-1,0E0,0E0,0E0,1.0967052E-1,0E0,0E0,1.5383394E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,21,21,23,23,27,27,30,30],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,-1,20,22,24,26,-1,-1,-1,-1,-1,28,-1,30,-1,-1,-1,32,-1,-1,34,-1,-1,-1,-1],"split_conditions":[3.2996133E3,8.4023336E4,8.1033794E3,5.38168E6,1.227E4,6.926793E-3,1.417988E6,1.129877E6,-2.2401195E-2,1E0,1.5127435E7,2.6191533E-2,1.5096262E7,1.0043E4,2.889654E5,9.53E2,-7.4254745E-3,8.255199E-3,1.3168041E-3,1.4209643E-2,-7.1695508E-3,3.006846E6,1.0504521E-2,4.407E3,3.8786195E-3,-1.6520731E-3,3.2293466E-3,1.7212875E5,-2.8450747E-3,-5.8138273E-3,3.3503532E7,-8.509523E-3,-9.467182E-4,-6.4358825E-4,-5.0311303E-3],"split_indices":[63,44,63,58,2,0,40,1,0,19,61,0,9,2,39,0,0,0,0,0,0,9,0,2,0,0,0,39,0,0,56,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.06E2,3.33E2,7.3E1,2.28E2,1.05E2,4.7E1,2.6E1,2.25E2,3E0,5.5E1,5E1,9E0,1.7E1,9.6E1,1.29E2,4.7E1,8E0,3.2E1,1.8E1,1.5E1,2E0,9.5E1,1E0,1.21E2,8E0,2.5E1,2.2E1,6.4E1,3.1E1,2.1E1,1E2,5.9E1,5E0,7.7E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[7.068075E-3,-2.3478542E-2,2.4811181E-1,-7.2697476E-2,6.202679E-2,4.3011928E-1,2.0095734E-1,-6.600838E-2,-2.1599278E-2,3.0422744E-3,1.1847616E-1,5.0985236E-3,2.4210483E-2,2.9827778E-3,1.0670404E-2,-8.848783E-2,1.0395728E-2,-7.8837795E-4,4.64545E-3,1.8297145E-1,2.590107E-3,-1.3521789E-1,-4.9594853E-2,6.214461E-3,-6.0934934E-4,1.8078275E-3,9.827343E-3,-1.5958788E-1,-1.8304322E-3,-9.059653E-3,-3.5642605E-2,-1.290136E-3,-8.3460435E-3,-3.0129817E-3,1.7070398E-2,2.2062105E-3,-4.668556E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,-1,-1,-1,21,23,-1,-1,25,-1,27,29,-1,-1,-1,-1,31,-1,-1,33,-1,-1,-1,35,-1,-1],"loss_changes":[3.1424935E0,1.5995237E0,3.2552266E-1,5.824369E-1,4.6245933E-1,1.7019105E-1,1.11186504E-1,4.0969002E-1,0E0,1.2885375E-1,2.8410435E-1,0E0,0E0,0E0,0E0,3.2770872E-1,1.5428102E-1,0E0,0E0,1.0859835E-1,0E0,1.8981707E-1,1.9277588E-1,0E0,0E0,0E0,0E0,1.2886107E-1,0E0,0E0,1.3725168E-1,0E0,0E0,0E0,1.15036935E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,15,15,16,16,19,19,21,21,22,22,27,27,30,30,34,34],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,-1,-1,-1,22,24,-1,-1,26,-1,28,30,-1,-1,-1,-1,32,-1,-1,34,-1,-1,-1,36,-1,-1],"split_conditions":[4.23218E3,9.570385E4,1E0,5.38168E6,1.01685606E9,7.0331914E3,4.3290512E1,7.164074E8,-2.1599278E-2,1.5271514E5,9.565983E6,5.0985236E-3,2.4210483E-2,2.9827778E-3,1.0670404E-2,7.4378266E4,3.5938916E3,-7.8837795E-4,4.64545E-3,9.9850745E2,2.590107E-3,7.177E3,2.2794174E2,6.214461E-3,-6.0934934E-4,1.8078275E-3,9.827343E-3,2.86968E5,-1.8304322E-3,-9.059653E-3,2.629627E6,-1.290136E-3,-8.3460435E-3,-3.0129817E-3,6.9E1,2.2062105E-3,-4.668556E-3],"split_indices":[63,44,25,58,7,4,67,7,0,39,56,0,0,0,0,39,59,0,0,63,0,2,63,0,0,0,0,7,0,0,1,0,0,0,8,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,3.78E2,4.7E1,2.4E2,1.38E2,8E0,3.9E1,2.37E2,3E0,6.8E1,7E1,2E0,6E0,6E0,3.3E1,1.83E2,5.4E1,5.7E1,1.1E1,3.4E1,3.6E1,8.2E1,1.01E2,8E0,4.6E1,5E0,2.9E1,6.5E1,1.7E1,8E0,9.3E1,7E0,5.8E1,6.1E1,3.2E1,2.6E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[8.208039E-3,-5.9716485E-2,1.3260527E-1,-7.19329E-2,4.3103574E-3,8.3392635E-2,3.4229583E-1,-8.062786E-2,1.593958E-3,-7.45354E-2,1.0129874E-1,9.531601E-2,4.053214E-1,-9.5683604E-2,-3.5425404E-4,1.3975869E-2,-1.2954782E-1,2.5515717E-3,1.6197973E-1,8.1957085E-3,-8.663884E-3,2.6775276E-2,1.2143465E-2,-1.0554717E-1,-1.3770898E-2,6.924329E-4,-1.1193636E-2,1.0284858E-2,4.749411E-3,-1.1522293E-1,-1.2298474E-3,3.0397198E-3,-3.9914493E-3,-8.499229E-4,-5.8513973E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,-1,-1,25,-1,27,-1,-1,-1,-1,29,31,-1,-1,-1,-1,33,-1,-1,-1,-1,-1],"loss_changes":[3.56521E0,5.0683236E-1,1.5090489E0,2.3439515E-1,0E0,3.5255134E-1,3.972156E-1,2.5719762E-1,0E0,3.0064154E-1,3.1067562E-1,1.792007E-1,3.740065E-1,1.5528905E-1,0E0,0E0,1.8485741E-1,0E0,1.389873E-1,0E0,0E0,0E0,0E0,1.3057697E-1,1.2435779E-1,0E0,0E0,0E0,0E0,1.0836816E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,18,18,23,23,24,24,29,29],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,-1,-1,26,-1,28,-1,-1,-1,-1,30,32,-1,-1,-1,-1,34,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.669497E8,8.1033794E3,2.1592189E5,4.3103574E-3,2.7011695E4,6.069097E4,7.164074E8,1.593958E-3,1.3812E4,4.1331047E-1,5.37E2,3.3616E4,1.4675482E3,-3.5425404E-4,1.3975869E-2,2.646573E6,2.5515717E-3,5.867931E-1,8.1957085E-3,-8.663884E-3,2.6775276E-2,1.2143465E-2,4.723148E11,3.2814958E3,6.924329E-4,-1.1193636E-2,1.0284858E-2,4.749411E-3,7.3938284E0,-1.2298474E-3,3.0397198E-3,-3.9914493E-3,-8.499229E-4,-5.8513973E-3],"split_indices":[2,12,63,39,0,44,39,7,0,9,46,3,10,63,0,0,40,0,45,0,0,0,0,42,4,0,0,0,0,58,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,2.72E2,1.48E2,2.52E2,2E1,1.21E2,2.7E1,2.33E2,1.9E1,1.2E1,1.09E2,6E0,2.1E1,1.93E2,4E1,1E0,1.1E1,6.2E1,4.7E1,5E0,1E0,9E0,1.2E1,1.72E2,2.1E1,5E0,6E0,2.4E1,2.3E1,1.53E2,1.9E1,1E1,1.1E1,1.1E1,1.42E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[4.275796E-3,-3.849913E-2,1.1423253E-1,-4.5276355E-2,1.8068215E-2,8.835179E-2,1.2149547E-2,-5.733672E-2,5.8274753E-2,9.927354E-2,-9.739414E-3,-5.3500045E-2,-1.8244877E-2,-1.6792306E-4,7.105096E-3,1.07165754E-1,-9.021069E-3,-6.3088566E-2,2.9946216E-3,9.8066196E-2,1.7163783E-2,-1.1075148E-1,-4.069991E-2,9.365661E-3,8.551279E-2,-1.473568E-1,-2.766912E-3,-2.5684848E-2,-9.438461E-2,-8.041021E-4,1.0397196E-1,-7.7094813E-3,5.200975E-5,2.8318593E-3,-2.1288325E-3,-2.4463013E-3,-9.8190075E-3,3.2223929E-3,7.091658E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,17,-1,-1,-1,19,-1,21,-1,23,-1,25,27,-1,29,31,-1,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9986358E0,8.820185E-1,4.0964246E-1,3.804983E-1,0E0,3.4599692E-1,0E0,3.1737113E-1,1.7964067E-1,2.4549055E-1,0E0,3.0377734E-1,0E0,0E0,0E0,1.8754542E-1,0E0,2.612025E-1,0E0,1.0027522E-1,0E0,1.4409435E-1,1.3541532E-1,0E0,1.6424602E-1,9.689337E-2,0E0,2.1974608E-1,1.6713178E-1,0E0,1.0868788E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,11,11,15,15,17,17,19,19,21,21,22,22,24,24,25,25,27,27,28,28,30,30],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,18,-1,-1,-1,20,-1,22,-1,24,-1,26,28,-1,30,32,-1,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3596371E9,6.9428994E3,7.17892E0,2.1898598E5,1.8068215E-2,3.782676E4,1.2149547E-2,3.764913E7,4.466882E6,2.8756912E8,-9.739414E-3,1.3989E4,-1.8244877E-2,-1.6792306E-4,7.105096E-3,1.5291998E4,-9.021069E-3,3.88672E5,2.9946216E-3,3.907527E8,1.7163783E-2,1.732734E6,2.3790412E7,9.365661E-3,5.975293E4,1.6395413E7,-2.766912E-3,4.1421578E-4,6.54925E6,-8.041021E-4,8.51E4,-7.7094813E-3,5.200975E-5,2.8318593E-3,-2.1288325E-3,-2.4463013E-3,-9.8190075E-3,3.2223929E-3,7.091658E-3],"split_indices":[7,63,46,39,0,63,0,59,1,1,0,2,0,0,0,4,0,9,0,5,0,1,56,0,44,12,0,49,9,0,2,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.23E2,3.05E2,1.18E2,3.01E2,4E0,1.01E2,1.7E1,2.7E2,3.1E1,9.8E1,3E0,2.68E2,2E0,1.9E1,1.2E1,9.6E1,2E0,2.48E2,2E1,9.4E1,2E0,7.8E1,1.7E2,9E0,8.5E1,4.5E1,3.3E1,1.34E2,3.6E1,1.3E1,7.2E1,4.1E1,4E0,2.4E1,1.1E2,2.7E1,9E0,4.1E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[4.961721E-3,-5.1537532E-2,1.1296393E-1,-6.8673916E-2,2.4123648E-2,7.8488976E-2,2.9691526E-1,-9.3110956E-2,-1.3415595E-2,7.2718645E-3,7.592526E-3,9.199701E-2,-1.6429017E-1,2.3746483E-2,1.3242516E-1,-1.0223368E-1,9.025799E-5,2.8397823E-3,-8.0217935E-2,-3.038945E-3,2.5044992E-3,5.3229265E-2,1.490397E-1,6.222497E-4,-1.434519E-2,9.133672E-3,-7.037895E-3,-6.5706284E-3,-3.5533812E-3,5.0198906E-3,-1.247424E-1,9.753666E-3,2.0922022E-3,9.5449544E-2,1.0185541E-2,-2.8318337E-3,6.334555E-3,-7.6345406E-3,6.399736E-4,1.0900164E-3,7.0871357E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,27,-1,-1,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,35,37,-1,-1,39,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.629701E0,3.6858284E-1,9.1526914E-1,3.1068158E-1,1.1119048E-1,4.2832762E-1,6.930213E-1,1.3886166E-1,3.5546038E-1,0E0,1.5685077E-1,2.5743663E-1,1.7304891E-1,0E0,2.5913185E-1,1.3347578E-1,0E0,0E0,1.448426E-1,0E0,0E0,9.469375E-2,1.4517725E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2291279E-1,1.2374464E-1,0E0,0E0,1.0443202E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,18,18,21,21,22,22,29,29,30,30,33,33],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,28,-1,-1,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,36,38,-1,-1,40,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.0043E4,7.164074E8,7.710634E3,2.735322E6,2.409091E0,9.533205E6,3.3616E4,2.1592189E5,1.7001536E7,7.2718645E-3,1.7933134E5,3.5257432E6,1.618E3,2.3746483E-2,1.4656219E8,1.5031561E3,9.025799E-5,2.8397823E-3,2.6E1,-3.038945E-3,2.5044992E-3,7.0120715E2,1.30817086E5,6.222497E-4,-1.434519E-2,9.133672E-3,-7.037895E-3,-6.5706284E-3,-3.5533812E-3,1.8093442E3,2.058478E7,9.753666E-3,2.0922022E-3,3.6308475E1,1.0185541E-2,-2.8318337E-3,6.334555E-3,-7.6345406E-3,6.399736E-4,1.0900164E-3,7.0871357E-3],"split_indices":[2,7,63,1,69,58,10,39,56,0,58,55,0,0,1,4,0,0,8,0,0,63,39,0,0,0,0,0,0,4,43,0,0,67,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.29E2,2.82E2,1.47E2,2.3E2,5.2E1,1.25E2,2.2E1,1.59E2,7.1E1,5E0,4.7E1,1.19E2,6E0,9E0,1.3E1,1.45E2,1.4E1,3.4E1,3.7E1,1.8E1,2.9E1,7.2E1,4.7E1,3E0,3E0,1.1E1,2E0,6.2E1,8.3E1,1.3E1,2.4E1,3E0,6.9E1,2.7E1,2E1,9E0,4E0,1.9E1,5E0,1.2E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[2.8638588E-3,-6.095684E-2,9.4335936E-2,-6.6591755E-2,1.4291575E-2,2.688548E-2,1.7241102E-1,-6.1182346E-2,-1.800464E-2,-2.0284832E-3,9.176259E-2,1.4123769E-1,4.026408E-1,-1.1746664E-1,-3.6175124E-2,-4.0614977E-3,6.836839E-4,6.7568067E-3,4.495235E-4,-8.791402E-3,1.4841534E-1,4.3781524E-4,2.1511152E-2,-6.429127E-3,-1.2534236E-3,6.1559663E-3,-4.3045443E-2,1.2444384E-1,1.1731844E-2,-4.6690084E-2,1.3063575E-2,1.1195496E-2,5.016991E-3,-3.5420004E-2,-1.1001722E-1,-4.5601674E-3,-9.5813925E-4,-7.293646E-3,-2.5458852E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,-1,-1,-1,-1,27,-1,-1,-1,-1,-1,29,31,-1,33,-1,-1,-1,35,37,-1,-1,-1,-1],"loss_changes":[2.457631E0,5.2355283E-1,9.0641725E-1,3.8858962E-1,0E0,1.7651644E-1,5.1729393E-1,3.3636892E-1,0E0,9.08651E-2,1.17585436E-1,1.9551313E-1,1.6597533E-1,1.1274791E-1,1.9730242E-1,0E0,0E0,0E0,0E0,0E0,1.351465E-1,0E0,0E0,0E0,0E0,0E0,2.0094612E-1,1.0094714E-1,0E0,1.1172289E-1,0E0,0E0,0E0,1.2632394E-1,1.0669348E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,20,20,26,26,27,27,29,29,33,33,34,34],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,-1,-1,-1,-1,28,-1,-1,-1,-1,-1,30,32,-1,34,-1,-1,-1,36,38,-1,-1,-1,-1],"split_conditions":[9.570385E4,5.932891E3,2.5096E4,7.339209E6,1.4291575E-2,1.1613266E9,4.2875492E1,9.26787E5,-1.800464E-2,4.656E3,1.1532659E7,1.379E3,4.9E1,9.703059E6,1.12608824E8,-4.0614977E-3,6.836839E-4,6.7568067E-3,4.495235E-4,-8.791402E-3,9.317209E0,4.3781524E-4,2.1511152E-2,-6.429127E-3,-1.2534236E-3,6.1559663E-3,3.305542E4,2.0159504E1,1.1731844E-2,3.5848716E7,1.3063575E-2,1.1195496E-2,5.016991E-3,4.58149E2,5.9485336E1,-4.5601674E-3,-9.5813925E-4,-7.293646E-3,-2.5458852E-4],"split_indices":[44,63,2,58,0,7,65,1,0,2,56,0,3,9,5,0,0,0,0,0,46,0,0,0,0,0,4,67,0,56,0,0,0,63,67,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,2.47E2,1.72E2,2.44E2,3E0,9.3E1,7.9E1,2.41E2,3E0,6.5E1,2.8E1,7.1E1,8E0,7.3E1,1.68E2,1E1,5.5E1,1.7E1,1.1E1,1E0,7E1,1E0,7E0,6.1E1,1.2E1,6E0,1.62E2,5.8E1,1.2E1,1.61E2,1E0,7E0,5.1E1,1.38E2,2.3E1,2.7E1,1.11E2,1.6E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[9.209475E-3,-4.5619916E-2,8.623897E-2,-3.9198622E-2,-1.37677705E-2,3.56705E-2,1.7835334E-1,-6.5585845E-3,-2.2960437E-2,-3.9394993E-2,5.794995E-2,3.324406E-1,1.369778E-1,-2.8491855E-2,9.57839E-3,-9.021545E-5,-9.38087E-3,8.2327776E-2,-4.0363093E-4,8.304584E-3,2.0083409E-2,1.6336986E-1,-4.471306E-2,1.5127027E-2,-4.8488162E-2,5.089966E-2,6.2659704E-3,2.2643723E-1,5.6136064E-3,5.786165E-3,-9.520988E-3,4.21023E-3,-7.553431E-3,-3.6089942E-3,-1.679282E-2,1.5140962E-3,8.534308E-3,8.183686E-3,2.032691E-2,-3.301648E-2,4.5369966E-3,-6.851918E-2,1.0885131E-3,-2.6025923E-3,3.6360275E-3,6.598879E-4,-5.1669446E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,-1,9,11,-1,13,15,17,19,21,23,-1,-1,-1,25,-1,-1,-1,27,29,31,33,35,-1,37,-1,-1,-1,-1,39,-1,41,-1,-1,-1,-1,43,-1,45,-1,-1,-1,-1,-1],"loss_changes":[1.7271142E0,3.613574E-1,7.85512E-1,3.7001532E-1,0E0,1.8792129E-1,3.369894E-1,0E0,2.5559673E-1,1.5286255E-1,1.3991669E-1,9.473455E-2,2.422694E-1,1.7343235E-1,0E0,0E0,0E0,9.023735E-2,0E0,0E0,0E0,9.746456E-2,2.0394564E-1,1.05089486E-1,1.1564779E-1,9.244405E-2,0E0,1.13387525E-1,0E0,0E0,0E0,0E0,1.3116068E-1,0E0,1.3265172E-1,0E0,0E0,0E0,0E0,9.810529E-2,0E0,9.310654E-2,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,17,17,21,21,22,22,23,23,24,24,25,25,27,27,32,32,34,34,39,39,41,41],"right_children":[2,4,6,8,-1,10,12,-1,14,16,18,20,22,24,-1,-1,-1,26,-1,-1,-1,28,30,32,34,36,-1,38,-1,-1,-1,-1,40,-1,42,-1,-1,-1,-1,44,-1,46,-1,-1,-1,-1,-1],"split_conditions":[9.570385E4,1.190724E4,5.6503E4,4.26944E5,-1.37677705E-2,9.9850745E2,1.417988E6,-6.5585845E-3,9.256843E3,4.1286E4,7.201163E1,1.27E2,1.5096262E7,7.2123305E6,9.57839E-3,-9.021545E-5,-9.38087E-3,2.9387E4,-4.0363093E-4,8.304584E-3,2.0083409E-2,5.993377E6,5.358657E7,6.4E1,2.09299E6,2.8673116E7,6.2659704E-3,3.63796E5,5.6136064E-3,5.786165E-3,-9.520988E-3,4.21023E-3,2.84E2,-3.6089942E-3,1E0,1.5140962E-3,8.534308E-3,8.183686E-3,2.032691E-2,2.889654E5,4.5369966E-3,2.7654392E9,1.0885131E-3,-2.6025923E-3,3.6360275E-3,6.598879E-4,-5.1669446E-3],"split_indices":[44,4,2,1,0,63,40,0,4,11,67,3,9,56,0,0,0,10,0,0,0,40,1,3,1,9,0,2,0,0,0,0,8,0,13,0,0,0,0,39,0,5,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.07E2,2.38E2,1.69E2,2.33E2,5E0,1.1E2,5.9E1,3.2E1,2.01E2,2.5E1,8.5E1,1.1E1,4.8E1,1.97E2,4E0,2.1E1,4E0,6.2E1,2.3E1,5E0,6E0,4.2E1,6E0,6.2E1,1.35E2,3.9E1,2.3E1,1.6E1,2.6E1,3E0,3E0,1.4E1,4.8E1,7.2E1,6.3E1,3.5E1,4E0,1.4E1,2E0,3.9E1,9E0,2.7E1,3.6E1,3.3E1,6E0,9E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-4.30297E-3,-5.8466375E-2,6.872327E-2,-5.4066367E-2,-1.9081678E-2,2.1305924E-2,1.4389022E-1,-7.558393E-2,2.2262689E-3,-1.3306972E-3,5.294576E-3,1.1856949E-1,1.901764E-2,-8.323092E-2,2.9215312E-3,1.1068036E-1,-1.307467E-3,-9.0872037E-4,5.142634E-3,1.3642724E-1,-9.180155E-3,-4.728532E-3,-5.888586E-4,1.03725E-2,7.372467E-4,4.2896895E-3,1.9141056E-1,1.0304663E-2,-4.510671E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,-1,21,-1,23,-1,-1,-1,25,-1,-1,-1,-1,-1,-1,27,-1,-1],"loss_changes":[1.7087737E0,3.5082537E-1,6.5275025E-1,2.9814327E-1,0E0,2.3141909E-1,4.1037357E-1,1.9129169E-1,2.2467962E-1,1.7974475E-1,0E0,3.9267194E-1,0E0,1.8851006E-1,0E0,1.4896992E-1,0E0,0E0,0E0,1.4405954E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3972557E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,13,13,15,15,19,19,26,26],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,-1,22,-1,24,-1,-1,-1,26,-1,-1,-1,-1,-1,-1,28,-1,-1],"split_conditions":[8.330997E4,7.339209E6,1.3590209E7,6.979E3,-1.9081678E-2,3.53E2,4.2875492E1,2.889654E5,1.572445E3,1.7670108E5,5.294576E-3,2.0531621E4,1.901764E-2,9.703059E6,2.9215312E-3,1.32E2,-1.307467E-3,-9.0872037E-4,5.142634E-3,1.2307312E5,-9.180155E-3,-4.728532E-3,-5.888586E-4,1.03725E-2,7.372467E-4,4.2896895E-3,3.667752E6,1.0304663E-2,-4.510671E-4],"split_indices":[44,58,1,2,0,8,65,39,58,39,0,4,0,9,0,3,0,0,0,39,0,0,0,0,0,0,11,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,2.47E2,1.83E2,2.45E2,2E0,1.13E2,7E1,1.77E2,6.8E1,9.1E1,2.2E1,6.5E1,5E0,1.68E2,9E0,1.4E1,5.4E1,7.9E1,1.2E1,6.2E1,3E0,1.37E2,3.1E1,6E0,8E0,3.5E1,2.7E1,2.4E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[4.894354E-3,-3.2918412E-2,9.4974E-2,-5.2377332E-2,3.2738242E-2,1.1563242E-1,-1.00587875E-1,-6.4308226E-2,3.2157365E-2,5.197508E-3,1.0141117E-2,5.6364212E-2,1.613457E-1,1.3111584E-2,-1.5967757E-1,-5.479393E-3,-4.828914E-2,-2.8122652E-3,4.9710274E-3,-4.882061E-4,6.8039545E-3,1.8252311E-2,1.0678421E-2,1.3123587E-1,3.2073188E-1,-1.118857E-2,1.4179663E-3,-1.8052557E-4,-3.082794E-3,1.7397763E-3,-1.071076E-2,1.2492018E-3,7.127732E-3,2.2365114E-2,5.1906332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,-1,25,-1,27,-1,-1,-1,-1,29,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3826215E0,3.6711046E-1,4.9643123E-1,2.2496301E-1,1.12472825E-1,2.8622448E-1,3.0997056E-1,1.5136427E-1,1.8972187E-1,0E0,1.4170599E-1,3.0845514E-1,2.5065303E-1,0E0,1.6265017E-1,0E0,1.0783744E-1,0E0,0E0,0E0,0E0,1.892221E-1,0E0,9.6859634E-2,2.310952E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,14,14,16,16,21,21,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,-1,26,-1,28,-1,-1,-1,-1,30,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3989E4,7.10711E8,9.064268E2,2.1592189E5,1.4193897E-3,1E0,1.3812E4,1.3063165E3,1.1441211E3,5.197508E-3,1E0,4.3431563E0,8.1033794E3,1.3111584E-2,1.31805E5,-5.479393E-3,1.24E2,-2.8122652E-3,4.9710274E-3,-4.882061E-4,6.8039545E-3,2.5212732E4,1.0678421E-2,1E1,2.3222E4,-1.118857E-2,1.4179663E-3,-1.8052557E-4,-3.082794E-3,1.7397763E-3,-1.071076E-2,1.2492018E-3,7.127732E-3,2.2365114E-2,5.1906332E-3],"split_indices":[2,7,69,39,50,105,9,4,66,0,26,46,63,0,39,0,3,0,0,0,0,4,0,8,10,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.04E2,2.85E2,1.19E2,2.2E2,6.5E1,1.08E2,1.1E1,1.93E2,2.7E1,1.4E1,5.1E1,4.8E1,6E1,1E0,1E1,4.5E1,1.48E2,1.2E1,1.5E1,4.5E1,6E0,4E1,8E0,5.2E1,8E0,7E0,3E0,4E1,1.08E2,3.8E1,2E0,8E0,4.4E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[9.2253136E-4,-2.441907E-2,1.8691143E-1,-5.9110396E-2,3.4253303E-2,-1.0460983E-3,2.2721004E-1,-3.968941E-2,-9.5271386E-2,1.9493852E-2,9.619827E-3,7.980451E-2,2.8351212E-1,-7.3981555E-3,-1.4547143E-3,-6.7554876E-2,-1.4717583E-1,-5.538959E-4,4.165003E-3,5.439828E-3,-7.879667E-3,6.1557097E-3,3.54336E-1,-3.6595806E-3,9.919016E-3,-8.186496E-3,-2.7604942E-4,-2.2238018E-2,3.6443013E-3,1.8970544E-2,3.938669E-3,-5.7741827E-3,-5.360533E-3,-4.847082E-3,5.1733176E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,-1,-1,-1,29,-1,-1,-1,-1,31,-1,-1,-1,33,-1,-1,-1],"loss_changes":[1.965279E0,7.4902475E-1,4.2575383E-1,1.5844584E-1,3.3434024E-1,0E0,3.2594228E-1,1.5673125E-1,1.0514641E-1,1.7258659E-1,0E0,1.2758073E-1,2.8266573E-1,0E0,0E0,1.5149532E-1,9.192264E-2,1.6679256E-1,0E0,0E0,0E0,0E0,2.0158386E-1,0E0,0E0,0E0,0E0,1.15017414E-1,0E0,0E0,0E0,1.0770503E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,15,15,16,16,17,17,22,22,27,27,31,31],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,-1,-1,-1,30,-1,-1,-1,-1,32,-1,-1,-1,34,-1,-1,-1],"split_conditions":[4.23218E3,6.974661E4,5.0314346E1,1.3800008E-2,9.3058E4,-1.0460983E-3,6.274818E4,1.44E2,1.2049959E7,1.646359E5,9.619827E-3,2.5212732E4,9.430977E3,-7.3981555E-3,-1.4547143E-3,1.0863E4,1.4690361E5,3.53E2,4.165003E-3,5.439828E-3,-7.879667E-3,6.1557097E-3,2.00036E5,-3.6595806E-3,9.919016E-3,-8.186496E-3,-2.7604942E-4,2.4722598E7,3.6443013E-3,1.8970544E-2,3.938669E-3,2.4522E1,-5.360533E-3,-4.847082E-3,5.1733176E-4],"split_indices":[63,44,67,50,2,0,39,0,56,39,0,4,4,0,0,2,39,8,0,0,0,0,10,0,0,0,0,56,0,0,0,67,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.15E2,3.66E2,4.9E1,2.3E2,1.36E2,8E0,4.1E1,1.51E2,7.9E1,1.26E2,1E1,1.2E1,2.9E1,1E1,1.41E2,5.3E1,2.6E1,9.8E1,2.8E1,1.1E1,1E0,1E1,1.9E1,5.2E1,1E0,2.2E1,4E0,7.7E1,2.1E1,1.6E1,3E0,6.6E1,1.1E1,9E0,5.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.1182636E-3,-2.443198E-2,1.6443734E-1,-4.308512E-2,8.137618E-2,4.184048E-3,2.328974E-1,-6.886009E-2,-9.679303E-3,1.4447445E-2,6.7043334E-2,2.760209E-1,-1.3984956E-3,-9.3809575E-2,-1.4875259E-3,-2.6964075E-3,-1.49731245E-2,8.380753E-3,3.7019692E-2,1.999382E-1,2.3083823E-2,-7.512684E-3,-3.701649E-3,-1.930983E-2,4.6358914E-3,-3.812468E-4,4.3875193E-3,1.5258389E-2,6.003927E-3,-5.4026447E-2,7.806169E-4,2.4731823E-3,-7.199744E-2,-1.2919293E-3,-5.897839E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,-1,21,-1,23,-1,-1,25,27,-1,-1,-1,29,-1,-1,-1,-1,-1,31,-1,-1,33,-1,-1],"loss_changes":[1.7274014E0,7.1007454E-1,2.7393222E-1,2.6194876E-1,1.509955E-1,0E0,3.359865E-1,1.5883785E-1,2.8362897E-1,0E0,1.6295712E-1,2.9818487E-1,0E0,9.426767E-2,0E0,2.2078331E-1,0E0,0E0,1.0451688E-1,1.3736236E-1,0E0,0E0,0E0,1.4203256E-1,0E0,0E0,0E0,0E0,0E0,1.1405137E-1,0E0,0E0,1.1250287E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,13,13,15,15,18,18,19,19,23,23,29,29,32,32],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,-1,22,-1,24,-1,-1,26,28,-1,-1,-1,30,-1,-1,-1,-1,-1,32,-1,-1,34,-1,-1],"split_conditions":[3.3311924E3,1.5522031E9,8.908335E3,2.025313E6,1.4E1,4.184048E-3,1.93E2,8E1,5.9161E4,1.4447445E-2,6.72E2,3.305542E4,-1.3984956E-3,3.133E3,-1.4875259E-3,2.481428E5,-1.49731245E-2,8.380753E-3,4.8928266E9,3.69814E6,2.3083823E-2,-7.512684E-3,-3.701649E-3,1.1021018E3,4.6358914E-3,-3.812468E-4,4.3875193E-3,1.5258389E-2,6.003927E-3,6.3349745E6,7.806169E-4,2.4731823E-3,2.02977E-3,-1.2919293E-3,-5.897839E-3],"split_indices":[63,7,4,1,3,0,8,8,2,0,0,4,0,2,0,39,0,0,5,58,0,0,0,63,0,0,0,0,0,56,0,0,68,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.12E2,3.57E2,5.5E1,3.04E2,5.3E1,2.7E1,2.8E1,1.71E2,1.33E2,2E0,5.1E1,2.4E1,4E0,1.02E2,6.9E1,1.31E2,2E0,1E1,4.1E1,1.9E1,5E0,1.9E1,8.3E1,1.13E2,1.8E1,2.3E1,1.8E1,6E0,1.3E1,5.7E1,5.6E1,8E0,4.9E1,2.7E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-1.4491656E-3,-4.594358E-2,6.358971E-2,-6.388164E-2,3.5997983E-2,1.4756101E-1,4.3657307E-2,-7.372848E-2,9.17697E-3,4.7317076E-2,-1.1026792E-2,3.2113332E-3,1.3161996E-2,5.03959E-2,-5.3931405E-3,1.2929342E-3,-3.8418802E-3,-5.297006E-3,2.1366307E-3,-1.1413756E-3,4.242012E-3,3.3070832E-2,5.914659E-3,5.8522876E-2,-9.625316E-4,-3.6685017E-3,3.5232075E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,-1,-1,-1,-1,-1,-1,-1,23,-1,25,-1,-1,-1],"loss_changes":[1.2038761E0,3.6535817E-1,2.764216E-1,1.4671403E-1,1.4693987E-1,3.107766E-1,1.5136573E-1,1.2421435E-1,1.1186679E-1,1.3236918E-1,0E0,0E0,0E0,1.6531259E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4887336E-1,0E0,1.5886629E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,21,21,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,-1,-1,-1,-1,-1,-1,-1,24,-1,26,-1,-1,-1],"split_conditions":[7.164074E8,1.4675482E3,2.1207E4,4.723148E11,3.372042E7,1.0988122E0,1.4656219E8,1E0,7.469E3,5.0293133E4,-1.1026792E-2,3.2113332E-3,1.3161996E-2,4.23218E3,-5.3931405E-3,1.2929342E-3,-3.8418802E-3,-5.297006E-3,2.1366307E-3,-1.1413756E-3,4.242012E-3,1.5155188E7,5.914659E-3,1.4338295E6,-9.625316E-4,-3.6685017E-3,3.5232075E-3],"split_indices":[7,63,9,42,59,46,1,21,9,39,0,0,0,63,0,0,0,0,0,0,0,56,0,56,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.14E2,2.46E2,1.68E2,2.02E2,4.4E1,3.1E1,1.37E2,1.78E2,2.4E1,4.3E1,1E0,2E1,1.1E1,1.32E2,5E0,1.1E1,1.67E2,5E0,1.9E1,1.6E1,2.7E1,1.08E2,2.4E1,7.3E1,3.5E1,7E0,6.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[1.5152805E-4,-1.4085535E-2,2.0631969E-1,-4.3789603E-2,4.2566545E-2,5.672794E-2,2.6632E-1,-3.817484E-2,-1.5232703E-2,2.4453783E-2,5.2938005E-3,4.9959696E-3,-7.7993968E-3,1.8549077E-2,6.7799385E-3,-8.0611035E-2,-1.3004981E-2,-8.735783E-6,3.0890526E-3,-1.20916866E-1,-1.8837159E-3,-5.9996434E-2,1.3582486E-2,-7.319045E-3,-2.6098492E-3,-3.6918423E-3,2.7738858E-3,4.9753333E-3,-3.0546184E-3,-1.604182E-2,5.380516E-3,-2.595484E-3,7.151659E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,19,21,-1,-1,23,-1,25,27,-1,-1,-1,-1,-1,29,31,-1,-1,-1],"loss_changes":[1.2268968E0,6.598377E-1,2.272321E-1,3.8514397E-1,1.6440235E-1,1.1208587E-1,2.3357356E-1,2.6918432E-1,0E0,1.0714691E-1,0E0,0E0,0E0,0E0,0E0,1.5111798E-1,2.0071629E-1,0E0,0E0,9.2556596E-2,0E0,1.2335743E-1,1.5566708E-1,0E0,0E0,0E0,0E0,0E0,1.3413166E-1,9.596495E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,15,15,16,16,19,19,21,21,22,22,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,20,22,-1,-1,24,-1,26,28,-1,-1,-1,-1,-1,30,32,-1,-1,-1],"split_conditions":[6.9428994E3,9.570385E4,6.274818E4,7.339209E6,1.7670108E5,5.37E2,3.0418E4,9.50639E5,-1.5232703E-2,1.3210104E9,5.2938005E-3,4.9959696E-3,-7.7993968E-3,1.8549077E-2,6.7799385E-3,1.237883E6,1E0,-8.735783E-6,3.0890526E-3,9.0301914E4,-1.8837159E-3,1.5311475E3,9.345519E0,-7.319045E-3,-2.6098492E-3,-3.6918423E-3,2.7738858E-3,4.9753333E-3,7.757358E9,1.61E2,5.380516E-3,-2.595484E-3,7.151659E-4],"split_indices":[63,44,39,58,39,3,10,1,0,7,0,0,0,0,0,9,13,0,0,39,0,63,69,0,0,0,0,0,5,3,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.16E2,3.9E2,2.6E1,2.56E2,1.34E2,8E0,1.8E1,2.52E2,4E0,1.07E2,2.7E1,7E0,1E0,8E0,1E1,9.3E1,1.59E2,6.7E1,4E1,4.6E1,4.7E1,5.7E1,1.02E2,3E1,1.6E1,5E1,7E0,1.5E1,8.7E1,7.9E1,8E0,3.5E1,4.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.7872383E-3,-3.155554E-2,9.252205E-2,-4.758547E-2,6.8763886E-3,5.672943E-2,2.0170878E-1,-9.891191E-3,-6.890214E-2,9.848456E-4,-4.6562026E-3,6.619185E-2,-1.1142082E-2,5.7418807E-3,1.7193096E-2,3.9194273E-3,-3.3802304E-2,-3.5303452E-3,3.9888206E-3,3.5123222E-3,-1.0379028E-1,-8.378725E-3,-1.0324231E-3,5.9395744E-3,-1.0565023E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,-1,-1,21,-1,-1,-1,23,-1,-1,-1,-1],"loss_changes":[1.2947445E0,1.9315428E-1,4.379863E-1,1.7689568E-1,1.3561808E-1,2.5707078E-1,3.1797493E-1,1.8097422E-1,1.1500031E-1,0E0,0E0,1.1573255E-1,0E0,0E0,0E0,0E0,1.0834554E-1,0E0,0E0,0E0,1.348636E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,16,16,20,20],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,-1,-1,22,-1,-1,-1,24,-1,-1,-1,-1],"split_conditions":[1.5704E4,1.8523391E6,2.1809912E0,1.3436554E3,1E0,9.533205E6,1.261448E1,1.32E2,7.2008003E9,9.848456E-4,-4.6562026E-3,2.3767982E4,-1.1142082E-2,5.7418807E-3,1.7193096E-2,3.9194273E-3,2.1115233E2,-3.5303452E-3,3.9888206E-3,3.5123222E-3,1.92144E5,-8.378725E-3,-1.0324231E-3,5.9395744E-3,-1.0565023E-2],"split_indices":[2,62,53,58,19,58,64,3,5,0,0,63,0,0,0,0,63,0,0,0,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,3.12E2,1.14E2,2.2E2,9.2E1,8.7E1,2.7E1,8E1,1.4E2,8.2E1,1E1,8.5E1,2E0,1.9E1,8E0,1.6E1,6.4E1,1.36E2,4E0,8.2E1,3E0,4E0,6E1,1E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[6.9629555E-3,-1.2771096E-2,1.65096E-1,-3.0940339E-2,7.609627E-2,1.9100824E-1,-8.431296E-4,-2.7272748E-2,-1.5287186E-2,8.739146E-2,-3.8872994E-3,1.3733166E-1,1.950642E-2,-4.177737E-2,2.1439627E-2,6.2206347E-2,7.820037E-3,8.78464E-3,2.8477006E-3,-5.0432883E-2,1.0021778E-3,3.9291335E-3,-1.0476776E-4,5.770562E-3,1.0025714E-3,-5.2542716E-2,9.823459E-3,-6.4346604E-2,-6.750995E-3,-9.384793E-4,-3.7428238E-3,5.6750665E-4,-7.7315676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,17,-1,19,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,27,-1,29,31,-1,-1,-1,-1],"loss_changes":[1.3782955E0,6.3691294E-1,2.3542464E-1,3.3827096E-1,1.2633914E-1,4.46275E-1,0E0,2.2981934E-1,0E0,1.1110115E-1,0E0,1.19638085E-1,0E0,1.3689399E-1,1.08094916E-1,1.15952626E-1,0E0,0E0,0E0,1.297524E-1,0E0,0E0,0E0,0E0,0E0,1.1793953E-1,0E0,1.07845485E-1,1.350337E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,11,11,13,13,14,14,15,15,19,19,25,25,27,27,28,28],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,18,-1,20,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,28,-1,30,32,-1,-1,-1,-1],"split_conditions":[4.23218E3,1.5522031E9,1.93E2,3.764913E7,2.4070468E7,3.305542E4,-8.431296E-4,1.4675482E3,-1.5287186E-2,9.3058E4,-3.8872994E-3,1.1448067E-1,1.950642E-2,2.0964778E5,1.03E2,6.3504875E6,7.820037E-3,8.78464E-3,2.8477006E-3,1.7267E4,1.0021778E-3,3.9291335E-3,-1.0476776E-4,5.770562E-3,1.0025714E-3,6.4065235E8,9.823459E-3,1E0,9.778E3,-9.384793E-4,-3.7428238E-3,5.6750665E-4,-7.7315676E-3],"split_indices":[63,7,8,59,59,4,0,63,0,2,0,68,0,39,3,56,0,0,0,2,0,0,0,0,0,7,0,27,2,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.4E2,3.92E2,4.8E1,3.26E2,6.6E1,4.2E1,6E0,3.23E2,3E0,6.2E1,4E0,3.5E1,7E0,2.49E2,7.4E1,4.8E1,1.4E1,2.1E1,1.4E1,2.19E2,3E1,2E1,5.4E1,1.9E1,2.9E1,2.18E2,1E0,1.73E2,4.5E1,4.2E1,1.31E2,4.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-2.3677533E-3,-2.2655416E-2,1.0310429E-1,-4.4905156E-2,2.5650904E-2,7.455997E-2,1.287624E-2,-3.521489E-3,-2.1447193E-2,3.8389713E-2,-1.0329649E-2,9.65133E-2,-5.33238E-3,1.3827725E-2,-5.1798474E-2,1.2354344E-2,3.2028582E-2,2.1182306E-1,7.288203E-2,-1.3662254E-3,4.999896E-2,-1.1954023E-2,-4.2620203E-3,5.0436263E-3,9.636213E-4,2.4033554E-3,1.579547E-2,-5.952118E-3,8.4183566E-2,7.2784103E-3,1.1654854E-3,-7.1599156E-2,1.8967986E-3,4.8658634E-3,-1.4000259E-3,9.972928E-4,-6.3175373E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,-1,-1,13,15,-1,17,-1,19,21,-1,23,25,27,-1,29,31,-1,-1,-1,-1,-1,-1,33,-1,-1,35,-1,-1,-1,-1,-1],"loss_changes":[9.416999E-1,3.9775413E-1,3.1765574E-1,1.6977602E-1,3.6909342E-1,2.643266E-1,0E0,0E0,1.5167634E-1,1.4876497E-1,0E0,1.371572E-1,0E0,1.0277535E-1,1.12473026E-1,0E0,9.3774125E-2,1.5642607E-1,1.1768207E-1,0E0,9.066142E-2,1.2000972E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.573084E-2,0E0,0E0,1.03921704E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,11,11,13,13,14,14,16,16,17,17,18,18,20,20,21,21,28,28,31,31],"right_children":[2,4,6,8,10,12,-1,-1,14,16,-1,18,-1,20,22,-1,24,26,28,-1,30,32,-1,-1,-1,-1,-1,-1,34,-1,-1,36,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,7.164074E8,1.6828056E1,4.759E3,2.4045518E7,1.4656219E8,1.287624E-2,-3.521489E-3,1.423795E7,1.2662238E1,-1.0329649E-2,1.064816E6,-5.33238E-3,6.700628E4,2.1528782E3,1.2354344E-2,2.40071E5,9.396283E6,1.379E3,-1.3662254E-3,9.747805E4,2.6E2,-4.2620203E-3,5.0436263E-3,9.636213E-4,2.4033554E-3,1.579547E-2,-5.952118E-3,2.1897722E10,7.2784103E-3,1.1654854E-3,6.281509E1,1.8967986E-3,4.8658634E-3,-1.4000259E-3,9.972928E-4,-6.3175373E-3],"split_indices":[63,7,46,2,59,1,0,0,56,67,0,40,0,39,58,0,40,1,0,0,39,10,0,0,0,0,0,0,5,0,0,59,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.38E2,3.68E2,7E1,2.52E2,1.16E2,6.1E1,9E0,1.12E2,1.4E2,1.11E2,5E0,5.5E1,6E0,6.5E1,7.5E1,2E0,1.09E2,8E0,4.7E1,3E1,3.5E1,3.7E1,3.8E1,1.4E1,9.5E1,4E0,4E0,2E0,4.5E1,6E0,2.9E1,1.7E1,2E1,3.9E1,6E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-1.9993301E-4,-3.7721336E-2,5.5118475E-2,-3.2869183E-2,-1.320547E-2,2.5734814E-2,1.1549632E-1,-5.5060778E-2,6.805806E-3,-3.8341754E-3,3.308105E-2,8.655447E-2,1.4789776E-2,-8.2440965E-2,-1.1418907E-2,2.0750621E-3,-3.9866693E-2,2.2951213E-3,-1.291959E-3,3.9310303E-2,8.161223E-3,-4.8125237E-3,-1.1097784E-3,2.259043E-3,-2.1731486E-3,8.213106E-4,-5.1003983E-3,6.642347E-2,-8.622274E-3,-7.4665376E-4,5.3621694E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,23,-1,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,29,-1,-1,-1],"loss_changes":[8.9874876E-1,2.9003063E-1,3.074991E-1,2.2400934E-1,0E0,9.537894E-2,2.9381764E-1,1.9383106E-1,1.5918332E-1,0E0,1.0237688E-1,1.9670957E-1,0E0,1.0648084E-1,1.3040562E-1,0E0,1.5860003E-1,0E0,0E0,2.1476363E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1857751E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,16,16,19,19,27,27],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,24,-1,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,30,-1,-1,-1],"split_conditions":[9.570385E4,6.6053805E6,6.0891E4,2.09299E6,-1.320547E-2,4.39E3,4.2875492E1,3.006846E6,6.9E1,-3.8341754E-3,7.309061E1,1.646359E5,1.4789776E-2,4.9208847E5,1.0878975E4,2.0750621E-3,1.410781E6,2.2951213E-3,-1.291959E-3,4.52E2,8.161223E-3,-4.8125237E-3,-1.1097784E-3,2.259043E-3,-2.1731486E-3,8.213106E-4,-5.1003983E-3,4.7746178E1,-8.622274E-3,-7.4665376E-4,5.3621694E-3],"split_indices":[44,58,2,1,0,2,65,9,8,0,67,39,0,62,59,0,40,0,0,3,0,0,0,0,0,0,0,67,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.31E2,2.57E2,1.74E2,2.53E2,4E0,1.18E2,5.6E1,1.62E2,9.1E1,7E0,1.11E2,5E1,6E0,9.9E1,6.3E1,5.1E1,4E1,8.9E1,2.2E1,3.3E1,1.7E1,7.5E1,2.4E1,2.3E1,4E1,2.2E1,1.8E1,3E1,3E0,1.1E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.9651911E-3,-2.8158572E-2,8.504197E-2,-3.4332786E-2,6.0126707E-3,6.948719E-2,3.1732526E-1,-4.4227596E-2,3.3751242E-2,-1.5965948E-3,9.495666E-2,1.8176978E-2,-1.5194173E-3,-8.124757E-2,-2.5245238E-2,-6.034707E-3,2.952275E-3,-2.9435577E-3,3.8998337E-3,7.2752815E-3,6.657992E-2,-9.0582855E-2,4.2942204E-3,3.8576254E-3,-6.2057506E-2,1.8592422E-1,2.0436263E-3,9.116073E-4,-4.817764E-3,-2.6467714E-2,1.7508455E-3,-5.3209934E-2,-1.2391444E-2,1.1813752E-2,-9.425594E-4,-4.8214216E-3,8.0802274E-5,-1.738934E-3,-6.96813E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,-1,21,23,-1,-1,-1,-1,-1,25,27,-1,29,31,33,-1,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0778552E0,2.9552314E-1,3.9707106E-1,2.0022786E-1,0E0,1.9930291E-1,1.6732025E-1,1.7925233E-1,1.7887379E-1,1.551654E-1,1.21940255E-1,0E0,0E0,1.4720547E-1,1.8451029E-1,0E0,0E0,0E0,0E0,0E0,1.5086187E-1,9.6182525E-2,0E0,9.752501E-2,1.19024605E-1,1.1888313E-1,0E0,0E0,0E0,1.072323E-1,0E0,1.0987072E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,20,20,21,21,23,23,24,24,25,25,29,29,31,31],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,-1,22,24,-1,-1,-1,-1,-1,26,28,-1,30,32,34,-1,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4096699E9,4.0297854E3,3.305542E4,2.1592189E5,6.0126707E-3,9.921638E4,1.92E2,4.822E3,2.4143283E2,6.3761925E1,9.296044E6,1.8176978E-2,-1.5194173E-3,7.164074E8,1.7458724E7,-6.034707E-3,2.952275E-3,-2.9435577E-3,3.8998337E-3,7.2752815E-3,1.6459E4,5.8955963E1,4.2942204E-3,7.959718E4,9.256843E3,2.3471036E7,2.0436263E-3,9.116073E-4,-4.817764E-3,1.7589E4,1.7508455E-3,2.11E2,-1.2391444E-2,1.1813752E-2,-9.425594E-4,-4.8214216E-3,8.0802274E-5,-1.738934E-3,-6.96813E-3],"split_indices":[7,63,4,39,0,44,8,2,63,67,56,0,0,7,56,0,0,0,0,0,9,61,0,39,4,56,0,0,0,44,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,3.05E2,1.15E2,2.94E2,1.1E1,1.09E2,6E0,2.57E2,3.7E1,2.9E1,8E1,5E0,1E0,8.6E1,1.71E2,5E0,3.2E1,1.7E1,1.2E1,2.5E1,5.5E1,8.2E1,4E0,9.6E1,7.5E1,8E0,4.7E1,7E0,7.5E1,5E1,4.6E1,7.3E1,2E0,6E0,2E0,1.3E1,3.7E1,6.3E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.4613067E-3,-2.1614023E-2,8.1232786E-2,-2.4621626E-3,-1.0704489E-3,1.9556849E-1,5.43243E-2,4.196564E-2,-1.0471238E-3,1.5738262E-2,5.4660337E-3,-6.623505E-3,9.8671205E-2,9.631174E-3,5.715375E-3,1.4392528E-3,-7.0787906E-3,1.1536176E-1,-3.7208067E-3,1.9893795E-3,-2.9362342E-3,9.93361E-2,1.5144589E-2,-3.939123E-3,5.236731E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,-1,7,9,11,13,-1,-1,-1,15,17,19,-1,-1,-1,21,-1,-1,-1,23,-1,-1,-1],"loss_changes":[8.559194E-1,2.043574E-1,3.185721E-1,0E0,1.7873758E-1,1.7971724E-1,2.3850408E-1,1.623128E-1,0E0,0E0,0E0,2.0247898E-1,1.5933967E-1,1.0973123E-1,0E0,0E0,0E0,1.2057686E-1,0E0,0E0,0E0,9.3987375E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,11,11,12,12,13,13,17,17,21,21],"right_children":[2,4,6,-1,8,10,12,14,-1,-1,-1,16,18,20,-1,-1,-1,22,-1,-1,-1,24,-1,-1,-1],"split_conditions":[2.1959E4,1.7864888E3,1.417988E6,-2.4621626E-3,1.24E2,1.01267644E11,6.6685635E9,2.9049585E3,-1.0471238E-3,1.5738262E-2,5.4660337E-3,3.1837756E7,2.8756912E8,1.3508742E7,5.715375E-3,1.4392528E-3,-7.0787906E-3,3.305542E4,-3.7208067E-3,1.9893795E-3,-2.9362342E-3,3E0,1.5144589E-2,-3.939123E-3,5.236731E-3],"split_indices":[2,4,40,0,3,42,12,4,0,0,0,62,1,56,0,0,0,4,0,0,0,8,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,3.31E2,1.06E2,1.34E2,1.97E2,1.9E1,8.7E1,6.4E1,1.33E2,6E0,1.3E1,3.7E1,5E1,4.6E1,1.8E1,3E1,7E0,4.6E1,4E0,3.2E1,1.4E1,4.4E1,2E0,2E0,4.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[7.329131E-4,-2.3149459E-2,5.9258293E-2,-2.998627E-2,2.5288395E-3,4.274523E-2,7.922612E-3,-1.7302681E-3,2.688479E-3,1.3387996E-1,1.7582359E-2,2.5019143E-3,9.745931E-3,-8.887606E-3,2.994465E-2,2.0159087E-3,-8.7484896E-2,-7.434895E-3,3.7345549E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,-1,9,-1,-1,-1,11,13,-1,-1,-1,15,-1,17,-1,-1],"loss_changes":[5.884162E-1,1.57216E-1,2.0791739E-1,1.5073258E-1,0E0,2.4301973E-1,0E0,0E0,0E0,1.21025234E-1,2.1971266E-1,0E0,0E0,0E0,1.2059895E-1,0E0,1.0271804E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,9,9,10,10,14,14,16,16],"right_children":[2,4,6,8,-1,10,-1,-1,-1,12,14,-1,-1,-1,16,-1,18,-1,-1],"split_conditions":[1.3428E4,4.7619216E7,6.001104E9,3.4490742E3,2.5288395E-3,2.1207E4,7.922612E-3,-1.7302681E-3,2.688479E-3,1.2511627E7,1.6857977E4,2.5019143E-3,9.745931E-3,-8.887606E-3,3.63796E5,2.0159087E-3,1.08935805E5,-7.434895E-3,3.7345549E-3],"split_indices":[2,9,7,66,0,9,0,0,0,1,44,0,0,0,2,0,39,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,2.98E2,1.21E2,2.74E2,2.4E1,1.06E2,1.5E1,2.56E2,1.8E1,2.2E1,8.4E1,1.1E1,1.1E1,4E0,8E1,7.3E1,7E0,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[2.4396456E-3,-9.255186E-3,1.5110704E-1,-1.91145E-2,6.853453E-2,6.0588848E-2,1.1352189E-2,-3.294224E-2,3.0837309E-2,-7.0274062E-3,3.8990316E-3,-6.8704374E-2,5.7365773E-3,-3.874075E-2,2.3851343E-3,1.897927E-3,-6.9908737E-3,2.48734E-3,-9.875066E-3,-3.6240395E-2,-1.173498E-2,-5.962537E-3,-3.077463E-2,-1.0863715E-3,-7.935432E-2,-6.418836E-3,-2.6190726E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,19,-1,-1,-1,-1,-1,21,-1,-1,23,-1,25,-1,-1],"loss_changes":[7.629229E-1,3.1345996E-1,2.3681426E-1,2.5105673E-1,1.4257933E-1,1.3925183E-1,0E0,1.3847494E-1,1.3084865E-1,0E0,0E0,1.10953026E-1,0E0,1.2884265E-1,0E0,0E0,0E0,0E0,0E0,1.2411752E-1,0E0,0E0,9.55255E-2,0E0,1.4178205E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,13,13,19,19,22,22,24,24],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,20,-1,-1,-1,-1,-1,22,-1,-1,24,-1,26,-1,-1],"split_conditions":[6.0500806E3,2.4082131E5,1.000944E5,1.29186E9,3.516E3,3.908618E6,1.1352189E-2,9.355285E8,1.505649E6,-7.0274062E-3,3.8990316E-3,1.207E3,5.7365773E-3,3.764913E7,2.3851343E-3,1.897927E-3,-6.9908737E-3,2.48734E-3,-9.875066E-3,1.44E2,-1.173498E-2,-5.962537E-3,1E0,-1.0863715E-3,1.3478658E3,-6.418836E-3,-2.6190726E-4],"split_indices":[63,39,39,7,12,40,0,43,2,0,0,0,0,59,0,0,0,0,0,0,0,0,19,0,63,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.37E2,4.06E2,3.1E1,3.61E2,4.5E1,1.6E1,1.5E1,2.83E2,7.8E1,2E0,4.3E1,5E0,1.1E1,2.65E2,1.8E1,7.5E1,3E0,3E0,2E0,2.63E2,2E0,1.4E1,2.49E2,2.15E2,3.4E1,1.9E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[4.361692E-4,-1.3160201E-2,1.00400925E-1,-2.5771302E-3,3.9930814E-3,-3.0827909E-3,1.3259842E-1,1.6017076E-2,-5.514682E-2,1.1037403E-2,9.5698625E-2,2.8753134E-3,2.6102471E-3,-1.961955E-3,-1.1361568E-2,5.520737E-3,-3.7215094E-3,-2.882659E-4,3.2578518E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,-1,7,-1,9,11,13,-1,15,17,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.7896334E-1,2.6287287E-1,2.7948666E-1,0E0,1.8932422E-1,0E0,1.3438946E-1,1.12152934E-1,1.0578181E-1,0E0,1.2344232E-1,9.716398E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,10,10,11,11],"right_children":[2,4,6,-1,8,-1,10,12,14,-1,16,18,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.23218E3,4.759E3,5.0314346E1,-2.5771302E-3,2.7394568E7,-3.0827909E-3,1.417988E6,4.4912686E9,1.805424E-1,1.1037403E-2,1.3790944E1,5.4210108E7,2.6102471E-3,-1.961955E-3,-1.1361568E-2,5.520737E-3,-3.7215094E-3,-2.882659E-4,3.2578518E-3],"split_indices":[63,2,67,0,56,0,40,5,49,0,64,9,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,3.74E2,5E1,1.1E2,2.64E2,8E0,4.2E1,2.2E2,4.4E1,1E1,3.2E1,1.65E2,5.5E1,4.2E1,2E0,2.9E1,3E0,1.46E2,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[3.9781216E-3,-1.7214088E-2,6.46019E-2,-1.0536904E-2,-1.4008342E-1,5.0540313E-2,8.843693E-3,-3.1568184E-2,4.8917174E-2,-1.2585486E-2,-2.8606814E-3,5.6946136E-2,-5.528858E-3,-6.2597594E-3,-2.5699666E-2,-3.8298778E-3,3.0941025E-3,1.16709486E-1,1.8941101E-3,-1.3720511E-3,6.567522E-3,1.4771726E-2,3.8246256E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,-1,13,15,-1,-1,17,-1,-1,19,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[5.407078E-1,2.5449532E-1,1.7560548E-1,3.729116E-1,1.426926E-1,1.1210728E-1,0E0,1.2519382E-1,1.6545211E-1,0E0,0E0,9.441039E-2,0E0,0E0,1.0791199E-1,0E0,0E0,1.2328529E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,11,11,14,14,17,17],"right_children":[2,4,6,8,10,12,-1,14,16,-1,-1,18,-1,-1,20,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[1.4777102E9,3.931053E7,1.6828056E1,3.685938E6,2.9580229E1,2.8756912E8,8.843693E-3,2.571E3,2.4143283E2,-1.2585486E-2,-2.8606814E-3,3.798313E3,-5.528858E-3,-6.2597594E-3,1.5704E4,-3.8298778E-3,3.0941025E-3,2.9E1,1.8941101E-3,-1.3720511E-3,6.567522E-3,1.4771726E-2,3.8246256E-3],"split_indices":[7,56,46,1,67,1,0,2,63,0,0,43,0,0,2,0,0,3,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.19E2,3.11E2,1.08E2,2.96E2,1.5E1,9.8E1,1E1,2.19E2,7.7E1,5E0,1E1,9.5E1,3E0,1.1E1,2.08E2,8E0,6.9E1,2E1,7.5E1,2.05E2,3E0,2E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.369263E-3,-1.2314113E-2,8.744994E-2,-9.175836E-3,-1.1464944E-2,6.3214265E-2,2.4919784E-1,-2.0920353E-2,2.944195E-2,-8.4792176E-4,4.41787E-3,2.86167E-4,1.571609E-2,-2.9797494E-2,3.1986684E-2,-6.5551134E-4,2.7007891E-3,2.1026421E-2,-1.8510631E-3,8.705673E-3,7.308919E-4,-2.8728652E-3,2.7541453E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,-1,17,19,-1,-1,21,-1,-1,-1,-1,-1],"loss_changes":[5.432607E-1,2.6312453E-1,2.3513159E-1,1.6706778E-1,0E0,1.3709052E-1,1.5380818E-1,1.3339816E-1,1.0155778E-1,0E0,0E0,0E0,0E0,1.1131212E-1,1.0002804E-1,0E0,0E0,1.1368152E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,13,13,14,14,17,17],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,-1,18,20,-1,-1,22,-1,-1,-1,-1,-1],"split_conditions":[1.849676E0,1.4609149E4,2.3767982E4,5.5292703E9,-1.1464944E-2,1.114E3,1.535691E2,1.227E4,1.733731E1,-8.4792176E-4,4.41787E-3,2.86167E-4,1.571609E-2,4.875E0,1.4285094E3,-6.5551134E-4,2.7007891E-3,8.986427E2,-1.8510631E-3,8.705673E-3,7.308919E-4,-2.8728652E-3,2.7541453E-3],"split_indices":[52,4,63,5,0,0,69,2,67,0,0,0,0,69,63,0,0,66,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.33E2,3.7E2,6.3E1,3.66E2,4E0,5.6E1,7E0,2.81E2,8.5E1,1.5E1,4.1E1,2E0,5E0,2.41E2,4E1,3.3E1,5.2E1,3.6E1,2.05E2,3E0,3.7E1,1.1E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.3986026E-3,-7.834801E-3,1.521526E-1,-2.6675941E-2,2.374857E-2,2.4366234E-1,6.938403E-2,-2.3502242E-2,-1.0157529E-2,7.4100574E-3,3.4424434E-3,1.4562918E-2,2.857125E-3,-3.0509198E-5,1.1675013E-2,-4.3725404E-3,-1.4206489E-2,-5.991495E-2,1.0913699E-3,-2.5358533E-2,2.2942782E-3,-2.4423643E-4,-7.079511E-3,-8.844566E-3,-2.1663364E-2,-8.380472E-3,-2.8721846E-3,-2.2574186E-3,5.2289094E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,-1,17,-1,-1,-1,-1,-1,-1,19,21,-1,23,-1,-1,-1,-1,25,27,-1,-1,-1],"loss_changes":[5.788607E-1,2.3389965E-1,1.6487283E-1,1.4031844E-1,1.1564759E-1,1.05965555E-1,1.7192686E-1,1.5254185E-1,0E0,1.1686718E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.5053181E-1,1.00976184E-1,0E0,1.04141735E-1,0E0,0E0,0E0,0E0,9.157353E-2,1.0209622E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,16,16,17,17,19,19,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,-1,18,-1,-1,-1,-1,-1,-1,20,22,-1,24,-1,-1,-1,-1,26,28,-1,-1,-1],"split_conditions":[8.1033794E3,9.570385E4,4.0601485E6,6.6053805E6,1.5271514E5,2.4121311E5,6.15E2,9.252759E2,-1.0157529E-2,2.0231652E3,3.4424434E-3,1.4562918E-2,2.857125E-3,-3.0509198E-5,1.1675013E-2,-4.3725404E-3,2.231E3,1.2E3,1.0913699E-3,8.412805E0,2.2942782E-3,-2.4423643E-4,-7.079511E-3,-8.844566E-3,4.6263345E-2,1.65602E5,-2.8721846E-3,-2.2574186E-3,5.2289094E-4],"split_indices":[63,44,58,58,39,44,3,66,0,66,0,0,0,0,0,0,0,0,0,64,0,0,0,0,68,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.14E2,3.91E2,2.3E1,2.45E2,1.46E2,1E1,1.3E1,2.42E2,3E0,1.1E2,3.6E1,7E0,3E0,1E1,3E0,2.8E1,2.14E2,2E1,9E1,1.82E2,3.2E1,1.3E1,7E0,3E0,1.79E2,1.34E2,4.5E1,4.4E1,9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-3.2735737E-3,-1.4840803E-2,6.030757E-2,-2.6145829E-2,3.9370947E-2,8.37693E-2,-3.680458E-3,-3.4093946E-2,3.1193625E-3,-6.422133E-4,1.039972E-1,1.2890044E-2,6.8129234E-2,-7.511666E-2,-2.3161283E-2,2.9229512E-3,9.70075E-3,3.8012993E-3,-4.4291737E-3,-9.503462E-4,-5.92521E-3,1.8963162E-4,-1.9737065E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1052268E-1,2.1982229E-1,2.1602115E-1,2.174932E-1,2.1306348E-1,1.4476082E-1,0E0,1.2069726E-1,0E0,0E0,1.06371194E-1,0E0,1.0557057E-1,1.5178734E-1,1.07267134E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,10,10,12,12,13,13,14,14],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.2996133E3,1.646359E5,1.62E2,4.7619216E7,8.9824E4,8.381319E3,-3.680458E-3,2.9648642E1,3.1193625E-3,-6.422133E-4,4.338E3,1.2890044E-2,1.8661473E4,1E0,1.32E2,2.9229512E-3,9.70075E-3,3.8012993E-3,-4.4291737E-3,-9.503462E-4,-5.92521E-3,1.8963162E-4,-1.9737065E-3],"split_indices":[63,39,8,9,11,44,0,67,0,0,0,0,68,105,3,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,3.56E2,6.4E1,2.95E2,6.1E1,5.5E1,9E0,2.72E2,2.3E1,3.4E1,2.7E1,3E0,5.2E1,5.6E1,2.16E2,2E1,7E0,4.9E1,3E0,2.7E1,2.9E1,8.7E1,1.29E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[3.968763E-3,-8.652853E-3,8.3695784E-2,-2.1629274E-2,2.849273E-2,-2.2267683E-3,1.0505737E-1,-9.270634E-4,-1.0342264E-2,1.1831012E-2,2.3625566E-2,1.579179E-3,1.3412675E-1,2.2478662E-3,-8.889129E-4,9.9300705E-2,1.26114115E-2,8.9721205E-3,2.4985466E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,13,-1,15,-1,-1,17,-1,-1,-1],"loss_changes":[4.3443257E-1,1.8040852E-1,1.6918391E-1,1.1500494E-1,9.771529E-2,0E0,1.0226393E-1,0E0,0E0,0E0,9.606619E-2,0E0,1.3686591E-1,0E0,0E0,1.15066856E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,10,10,12,12,15,15],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,14,-1,16,-1,-1,18,-1,-1,-1],"split_conditions":[4.23218E3,1.0564597E9,5.0314346E1,4.4533052E0,1.3E1,-2.2267683E-3,6.274818E4,-9.270634E-4,-1.0342264E-2,1.1831012E-2,1.3026638E7,1.579179E-3,1.5325492E4,2.2478662E-3,-8.889129E-4,1.3812E4,1.26114115E-2,8.9721205E-3,2.4985466E-3],"split_indices":[63,7,67,68,3,0,39,0,0,0,56,0,63,0,0,9,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,3.72E2,5.8E1,2.76E2,9.6E1,8E0,5E1,2.74E2,2E0,1E0,9.5E1,1.5E1,3.5E1,6.1E1,3.4E1,2.9E1,6E0,9E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[1.2084324E-3,-8.80641E-3,1.20439485E-1,-3.2276828E-2,2.3821054E-2,1.133752E-2,5.10968E-2,-6.258512E-2,-1.6812673E-2,5.377057E-4,3.3475633E-3,3.9235726E-3,-5.223103E-3,-5.1797954E-3,-2.5666581E-2,2.0539455E-2,-1.9632427E-3,7.8610296E-4,-4.173802E-3,2.5355397E-3,-8.2921214E-2,-1.1070669E-3,-9.438109E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,-1,17,19,-1,-1,-1,-1,21,-1,-1],"loss_changes":[5.085448E-1,3.0172047E-1,2.5580466E-1,1.0627827E-1,9.515642E-2,0E0,1.1884631E-1,1.277987E-1,1.3982488E-1,0E0,0E0,0E0,0E0,0E0,1.15976505E-1,2.1003543E-1,0E0,0E0,0E0,0E0,9.827781E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,14,14,15,15,20,20],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,-1,18,20,-1,-1,-1,-1,22,-1,-1],"split_conditions":[6.651788E3,6.974661E4,3.0418E4,1.65602E5,1.7670108E5,1.133752E-2,2.0531621E4,2.55E2,3.6319303E-3,5.377057E-4,3.3475633E-3,3.9235726E-3,-5.223103E-3,-5.1797954E-3,7.653459E1,3.271774E7,-1.9632427E-3,7.8610296E-4,-4.173802E-3,2.5355397E-3,6.588828E6,-1.1070669E-3,-9.438109E-3],"split_indices":[63,44,10,9,39,0,4,3,53,0,0,0,0,0,67,56,0,0,0,0,9,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.24E2,3.92E2,3.2E1,2.28E2,1.64E2,1.1E1,2.1E1,7.6E1,1.52E2,1.3E2,3.4E1,1.8E1,3E0,3.3E1,4.3E1,6E1,9.2E1,2.6E1,1.7E1,4.6E1,1.4E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.9248444E-3,-1.6842458E-2,4.1484997E-2,-1.3347288E-2,-9.952061E-3,6.437367E-3,6.685962E-2,-2.9073315E-2,1.6459001E-2,7.826599E-4,-9.30316E-3,2.4943608E-3,1.9903235E-1,-1.8391646E-3,2.7004763E-2,1.0036974E-3,-1.0075912E-2,1.7376635E-2,1.298608E-3,-6.524125E-3,2.537429E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,-1,17,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[3.26171E-1,1.867964E-1,1.2852104E-1,1.3125543E-1,0E0,1.2753238E-1,1.5170142E-1,9.810968E-2,1.0529903E-1,0E0,0E0,0E0,2.1693704E-1,0E0,1.2286976E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,12,12,14,14],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,-1,18,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2451478E5,6.3903E6,8.459549E4,6.979E3,-9.952061E-3,3.782676E4,3.63796E5,2.0964778E5,1.1981917E5,7.826599E-4,-9.30316E-3,2.4943608E-3,5.358657E7,-1.8391646E-3,2.1115233E2,1.0036974E-3,-1.0075912E-2,1.7376635E-2,1.298608E-3,-6.524125E-3,2.537429E-3],"split_indices":[44,58,39,2,0,63,2,39,44,0,0,0,1,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.26E2,2.82E2,1.44E2,2.78E2,4E0,6.1E1,8.3E1,1.82E2,9.6E1,5.9E1,2E0,7.6E1,7E0,1.56E2,2.6E1,9.5E1,1E0,3E0,4E0,3E0,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[7.2745686E-5,-2.2298904E-2,5.6274105E-2,-1.4126358E-3,1.1834478E-3,-8.7882485E-3,6.381834E-2,5.1686224E-2,1.9210304E-1,5.586396E-3,3.031873E-2,1.3329679E-2,-4.981788E-3,5.922367E-3,1.758294E-2,-2.2733714E-3,1.7746332E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,-1,-1,-1,7,9,11,-1,13,-1,-1,-1,15,-1,-1],"loss_changes":[5.4315853E-1,1.0691929E-1,2.348235E-1,0E0,0E0,0E0,1.7683476E-1,1.5145695E-1,2.875731E-1,0E0,9.931761E-2,0E0,0E0,0E0,9.88969E-2,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,6,6,7,7,8,8,10,10,14,14],"right_children":[2,4,6,-1,-1,-1,8,10,12,-1,14,-1,-1,-1,16,-1,-1],"split_conditions":[1.2934E4,3.73E2,1.5153618E0,-1.4126358E-3,1.1834478E-3,-8.7882485E-3,2.5212732E4,2.412E4,1.92E2,5.586396E-3,1.4285094E3,1.3329679E-2,-4.981788E-3,5.922367E-3,4.9E1,-2.2733714E-3,1.7746332E-3],"split_indices":[2,8,64,0,0,0,4,9,8,0,63,0,0,0,3,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.3E2,3.08E2,1.22E2,2.67E2,4.1E1,3E0,1.19E2,1.1E2,9E0,2.6E1,8.4E1,7E0,2E0,9E0,7.5E1,1.7E1,5.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[2.7184922E-3,-4.340164E-3,1.4375569E-1,-3.4097425E-4,-1.1457918E-1,1.4287204E-2,5.5700455E-2,-3.344065E-2,1.43029E-2,-2.1798802E-3,-1.09532755E-2,4.6137176E-3,-6.0479757E-3,4.842092E-3,-2.0417674E-3,1.9123007E-2,-3.9180727E-3,2.236398E-3,1.1611829E-3,-5.144394E-4,3.575697E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,-1,-1,17,-1,-1,19,-1,-1],"loss_changes":[4.1775227E-1,1.7630762E-1,2.5730917E-1,1.8806009E-1,1.0036771E-1,0E0,1.1685234E-1,1.5419967E-1,1.2654985E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.279595E-1,0E0,0E0,1.3905314E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,15,15,18,18],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,-1,-1,18,-1,-1,20,-1,-1],"split_conditions":[8.1033794E3,4.5859156E7,3.0418E4,1.204977E6,5.974836E4,1.4287204E-2,5.22E2,2.1394231E0,4.0950096E10,-2.1798802E-3,-1.09532755E-2,4.6137176E-3,-6.0479757E-3,4.842092E-3,-2.0417674E-3,1.4450867E-2,-3.9180727E-3,2.236398E-3,2.478066E5,-5.144394E-4,3.575697E-3],"split_indices":[63,56,10,1,39,0,3,69,5,0,0,0,0,0,0,68,0,0,39,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.18E2,3.99E2,1.9E1,3.86E2,1.3E1,6E0,1.3E1,1.18E2,2.68E2,9E0,4E0,1.1E1,2E0,7E0,1.11E2,2.56E2,1.2E1,1E2,1.56E2,1.35E2,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.1520474E-3,-2.369972E-2,3.1871043E-2,-1.4557461E-3,1.6709162E-3,-6.0517243E-3,3.852999E-2,7.8492024E-4,8.119677E-2,1.0988765E-2,5.8250032E-2,2.9680235E-2,9.591967E-3,2.3770502E-3,-4.741872E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,-1,-1,-1,7,-1,9,-1,11,13,-1,-1,-1],"loss_changes":[3.082726E-1,9.9264726E-2,1.823786E-1,0E0,0E0,0E0,1.5131815E-1,0E0,1.7467281E-1,0E0,1.9381829E-1,1.1398611E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,6,6,8,8,10,10,11,11],"right_children":[2,4,6,-1,-1,-1,8,-1,10,-1,12,14,-1,-1,-1],"split_conditions":[7.384604E8,2.0964778E5,2.4950776E0,-1.4557461E-3,1.6709162E-3,-6.0517243E-3,8.51E4,7.8492024E-4,1.6459E4,1.0988765E-2,2.4928955E1,1.4656219E8,9.591967E-3,2.3770502E-3,-4.741872E-3],"split_indices":[7,39,65,0,0,0,2,0,9,0,65,1,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.12E2,2.45E2,1.67E2,2.2E2,2.5E1,6E0,1.61E2,1.07E2,5.4E1,6E0,4.8E1,4.1E1,7E0,3.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-2.756164E-4,-7.594087E-3,1.1331807E-1,-2.9105851E-2,1.8483186E-2,-1.3145858E-4,1.5861028E-1,-1.0940571E-2,-6.452758E-2,7.3325966E-4,9.227548E-3,1.0412675E-2,2.716367E-3,-4.967245E-3,-1.1103497E-4,6.911542E-4,-9.113169E-2,-8.418489E-3,-2.5690286E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,-1,11,13,15,-1,-1,-1,-1,-1,-1,-1,17,-1,-1],"loss_changes":[3.4336936E-1,2.1819362E-1,1.3186604E-1,1.3634792E-1,9.306228E-2,0E0,9.658697E-2,1.1367433E-1,1.5265885E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.6224167E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,16,16],"right_children":[2,4,6,8,10,-1,12,14,16,-1,-1,-1,-1,-1,-1,-1,18,-1,-1],"split_conditions":[8.1033794E3,6.974661E4,6.274818E4,1.8071064E7,1.4646042E3,-1.3145858E-4,5.3632E4,1.44E2,6.465659E4,7.3325966E-4,9.227548E-3,1.0412675E-2,2.716367E-3,-4.967245E-3,-1.1103497E-4,6.911542E-4,2.9648642E1,-8.418489E-3,-2.5690286E-3],"split_indices":[63,44,39,56,68,0,10,0,39,0,0,0,0,0,0,0,67,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.11E2,3.87E2,2.4E1,2.12E2,1.75E2,7E0,1.7E1,1.41E2,7.1E1,1.73E2,2E0,1E1,7E0,1.1E1,1.3E2,1.8E1,5.3E1,1.5E1,3.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"19","size_leaf_vector":"1"}},{"base_weights":[6.644184E-3,-4.524112E-4,4.4769347E-2,1.0423746E-2,3.8710013E-2,1.2034621E-3,5.502799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":66,"left_children":[1,-1,3,-1,5,-1,-1],"loss_changes":[2.585886E-1,0E0,1.23046964E-1,0E0,1.2318446E-1,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4],"right_children":[2,-1,4,-1,6,-1,-1],"split_conditions":[1.227E4,-4.524112E-4,1.655567E6,1.0423746E-2,7.0484486E0,1.2034621E-3,5.502799E-3],"split_indices":[2,0,62,0,46,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.19E2,2.95E2,1.24E2,3E0,1.21E2,1.04E2,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-2.891274E-4,-1.1125566E-2,4.0015932E-2,-8.699275E-3,-9.894575E-3,3.0171322E-2,7.208685E-3,1.6750101E-2,-2.4050608E-2,4.0700976E-2,-6.788805E-3,4.9937266E-4,2.9377076E-3,-1.8024367E-3,8.184626E-3,1.386479E-3,1.5571079E-2,5.929842E-4,-5.2908687E-3,-2.6332447E-3,1.7880388E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,-1,9,-1,11,13,15,-1,17,-1,-1,19,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8344903E-1,1.5574494E-1,9.270908E-2,1.285465E-1,0E0,1.5646237E-1,0E0,9.0240255E-2,9.09213E-2,2.533764E-1,0E0,1.2352832E-1,0E0,0E0,1.1706175E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,11,11,14,14],"right_children":[2,4,6,8,-1,10,-1,12,14,16,-1,18,-1,-1,20,-1,-1,-1,-1,-1,-1],"split_conditions":[1.615586E9,3.764913E7,1.0871896E10,1.24E2,-9.894575E-3,3.768034E4,7.208685E-3,1.5631128E3,1.2472753E5,3.305542E4,-6.788805E-3,9.409E3,2.9377076E-3,-1.8024367E-3,4.9439902E2,1.386479E-3,1.5571079E-2,5.929842E-4,-5.2908687E-3,-2.6332447E-3,1.7880388E-3],"split_indices":[7,59,7,3,0,4,0,63,39,4,0,2,0,0,63,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.18E2,3.3E2,8.8E1,3.27E2,3E0,8.2E1,6E0,1.23E2,2.04E2,7.8E1,4E0,9.1E1,3.2E1,1.43E2,6.1E1,7.6E1,2E0,8.3E1,8E0,1.9E1,4.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[5.38802E-4,-1.1940126E-2,4.7496714E-2,-6.8574883E-3,-1.001241E-1,6.367727E-2,-1.1197572E-3,-8.525738E-4,1.7629383E-3,-9.94286E-3,-1.6308309E-3,2.1589936E-3,9.353305E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.595722E-1,1.561859E-1,1.0826883E-1,1.6065855E-1,1.2846091E-1,1.7656723E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5],"right_children":[2,4,6,8,10,12,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.615586E9,4.086885E7,9.677921E7,1.9184847E3,3.8499493E1,3.63796E5,-1.1197572E-3,-8.525738E-4,1.7629383E-3,-9.94286E-3,-1.6308309E-3,2.1589936E-3,9.353305E-3],"split_indices":[7,56,1,63,67,2,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.41E2,3.49E2,9.2E1,3.31E2,1.8E1,7.5E1,1.7E1,2.65E2,6.6E1,6E0,1.2E1,6.7E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-2.1004477E-3,2.222432E-2,-2.0795984E-2,7.6571386E-3,1.693223E-2,-1.2109787E-3,3.457559E-3,-7.2758007E-3,2.0522092E-2,-1.4655994E-3,3.1102251E-2,3.7734114E-2,-2.7654886E-3,2.1385115E-2,4.009081E-3,5.1340383E-2,-2.6196616E-2,2.8704002E-3,-4.7753723E-3,-3.11912E-4,-6.8292175E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,-1,7,-1,-1,-1,9,-1,11,13,-1,15,-1,17,19,-1,-1,-1,-1],"loss_changes":[2.019229E-1,1.3843703E-1,1.08904086E-1,0E0,1.16824865E-1,0E0,0E0,0E0,1.0068252E-1,0E0,9.2534676E-2,1.0635464E-1,0E0,1.5385455E-1,0E0,9.3913734E-2,9.5585205E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,8,8,10,10,11,11,13,13,15,15,16,16],"right_children":[2,4,6,-1,8,-1,-1,-1,10,-1,12,14,-1,16,-1,18,20,-1,-1,-1,-1],"split_conditions":[1.33E2,9.957E3,2.785032E9,7.6571386E-3,1.5153618E0,-1.2109787E-3,3.457559E-3,-7.2758007E-3,1.3350947E3,-1.4655994E-3,1.5953E4,3.978477E11,-2.7654886E-3,1.121092E6,4.009081E-3,1E0,2.11E2,2.8704002E-3,-4.7753723E-3,-3.11912E-4,-6.8292175E-3],"split_indices":[3,9,7,0,64,0,0,0,4,0,0,42,0,40,0,19,8,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.42E2,1.92E2,2.5E2,6E0,1.86E2,2.39E2,1.1E1,3E0,1.83E2,3.1E1,1.52E2,1.42E2,1E1,1.06E2,3.6E1,6.5E1,4.1E1,6.2E1,3E0,3.6E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[3.129112E-3,-2.179929E-3,5.0579268E-3,-5.0947596E-3,8.685837E-4,-8.087483E-3,3.380734E-2,-1.6405271E-2,3.3496853E-2,-3.8327782E-3,5.2885655E-2,1.5341715E-4,-1.8755387E-3,-8.403256E-4,3.5453164E-3,7.0900475E-3,1.7343982E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,-1,-1,5,7,9,11,13,-1,15,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2244331E-1,1.23768464E-1,0E0,0E0,1.11784056E-1,1.0362466E-1,1.8058193E-1,1.11944534E-1,1.0550725E-1,0E0,1.0541093E-1,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,5,5,6,6,7,7,8,8,10,10],"right_children":[2,4,-1,-1,6,8,10,12,14,-1,16,-1,-1,-1,-1,-1,-1],"split_conditions":[7.0484486E0,2.398E3,5.0579268E-3,-5.0947596E-3,1.5271514E5,3.53E2,4.2030402E2,1.8887723E-2,2.6327672E5,-3.8327782E-3,2.3E1,1.5341715E-4,-1.8755387E-3,-8.403256E-4,3.5453164E-3,7.0900475E-3,1.7343982E-3],"split_indices":[46,2,0,0,39,8,63,68,58,0,3,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.06E2,3.87E2,1.9E1,1E1,3.77E2,2.97E2,8E1,2.48E2,4.9E1,1.1E1,6.9E1,1.34E2,1.14E2,2.2E1,2.7E1,9E0,6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[-5.382031E-3,-8.89562E-3,6.063061E-3,-6.6257925E-3,-7.7209743E-3,-1.1382024E-2,2.4621044E-3,-9.215188E-4,2.7075065E-2,5.51149E-3,-8.367257E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,-1,5,-1,7,-1,-1,9,-1,-1],"loss_changes":[1.9792189E-1,1.4223036E-1,0E0,1.1354427E-1,0E0,1.1572109E-1,0E0,0E0,1.6584022E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,5,5,8,8],"right_children":[2,4,-1,6,-1,8,-1,-1,10,-1,-1],"split_conditions":[1.6828056E1,4.2433594E3,6.063061E-3,4.2524424E7,-7.7209743E-3,5.313189E11,2.4621044E-3,-9.215188E-4,5.993377E6,5.51149E-3,-8.367257E-5],"split_indices":[46,69,0,9,0,42,0,0,40,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.22E2,4.12E2,1E1,4.07E2,5E0,3.77E2,3E1,3.13E2,6.4E1,1.5E1,4.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[3.304659E-3,-7.165029E-3,7.522475E-2,-5.3024534E-3,-8.746633E-3,-2.4175404E-3,9.7259015E-2,-1.9919476E-3,4.0316144E-3,3.3050533E-3,1.0069477E-2,-5.7670153E-3,6.0774077E-2,1.585014E-2,-1.336925E-3,4.0287706E-3,-1.4500305E-3,-1.0620558E-3,3.3679802E-2,-2.8139537E-3,4.7154002E-2,6.6361437E-3,1.5293971E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,-1,-1,9,-1,11,-1,-1,13,15,17,-1,-1,-1,-1,19,-1,21,-1,-1],"loss_changes":[3.318506E-1,1.252972E-1,1.6024482E-1,1.3006577E-1,0E0,0E0,1.382952E-1,0E0,1.6993E-1,0E0,0E0,1.2593926E-1,9.9501505E-2,9.1100395E-2,0E0,0E0,0E0,0E0,1.163641E-1,0E0,1.0662156E-1,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,6,6,8,8,11,11,12,12,13,13,18,18,20,20],"right_children":[2,4,6,8,-1,-1,10,-1,12,-1,-1,14,16,18,-1,-1,-1,-1,20,-1,22,-1,-1],"split_conditions":[4.23218E3,3.764913E7,5.0314346E1,1.192104E3,-8.746633E-3,-2.4175404E-3,3.305542E4,-1.9919476E-3,1.646359E5,3.3050533E-3,1.0069477E-2,1.4669927E-2,2.5519034E7,6.944307E7,-1.336925E-3,4.0287706E-3,-1.4500305E-3,-1.0620558E-3,4.6E1,-2.8139537E-3,8.593763E7,6.6361437E-3,1.5293971E-3],"split_indices":[63,59,67,66,0,0,4,0,39,0,0,68,9,7,0,0,0,0,3,0,7,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.39E2,3.84E2,5.5E1,3.81E2,3E0,8E0,4.7E1,7.7E1,3.04E2,3.9E1,8E0,2.6E2,4.4E1,1.32E2,1.28E2,3.5E1,9E0,4.2E1,9E1,1.1E1,7.9E1,1E1,6.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.347509E-3,-8.72992E-3,3.8079135E-2,-1.1187164E-2,7.2198655E-3,2.293159E-3,-3.299106E-3,-9.076827E-3,-8.433522E-3,1.1692854E-2,-1.0918871E-3,9.910814E-4,-4.650089E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,-1,-1,9,-1,11,-1,-1,-1],"loss_changes":[1.6894943E-1,1.2985606E-1,1.1120756E-1,1.1085999E-1,0E0,0E0,0E0,9.169434E-2,0E0,1.2871912E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,9,9],"right_children":[2,4,6,8,-1,-1,-1,10,-1,12,-1,-1,-1],"split_conditions":[1.6219076E9,7.3073276E3,1.5953E4,4.2433594E3,7.2198655E-3,2.293159E-3,-3.299106E-3,1.33E2,-8.433522E-3,1E0,-1.0918871E-3,9.910814E-4,-4.650089E-3],"split_indices":[7,63,0,69,0,0,0,3,0,19,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.25E2,3.25E2,1E2,3.21E2,4E0,9.2E1,8E0,3.18E2,3E0,1.27E2,1.91E2,1.18E2,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[2.699376E-3,-1.5975699E-2,4.189627E-2,-2.088972E-2,3.0672392E-3,2.4294325E-3,-4.856797E-3,-9.210986E-3,-5.212388E-2,-7.678168E-4,2.92583E-3,-1.1360189E-1,-1.2883852E-3,-8.385102E-3,-8.694075E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,-1,-1,9,11,-1,-1,13,-1,-1,-1],"loss_changes":[2.9493526E-1,1.08982705E-1,1.7329691E-1,9.3631946E-2,0E0,0E0,0E0,9.261076E-2,1.0508411E-1,0E0,0E0,1.1522016E-1,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,7,7,8,8,11,11],"right_children":[2,4,6,8,-1,-1,-1,10,12,-1,-1,14,-1,-1,-1],"split_conditions":[7.115465E6,6.56E2,4.5859156E7,1.8071064E7,3.0672392E-3,2.4294325E-3,-4.856797E-3,1.347608E6,3.8499493E1,-7.678168E-4,2.92583E-3,2.7276272E-2,-1.2883852E-3,-8.385102E-3,-8.694075E-4],"split_indices":[1,8,56,56,0,0,0,40,67,0,0,53,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.01E2,2.72E2,1.29E2,2.57E2,1.5E1,1.22E2,7E0,1.88E2,6.9E1,1.72E2,1.6E1,1.9E1,5E1,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"15","size_leaf_vector":"1"}},{"base_weights":[-2.0862117E-3,-4.863251E-3,6.008786E-3,-2.204657E-2,4.6149455E-4,-7.956845E-4,-5.591764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,5,-1,-1,-1],"loss_changes":[1.5058821E-1,1.0359205E-1,0E0,9.519689E-2,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3],"right_children":[2,4,-1,6,-1,-1,-1],"split_conditions":[1.6828056E1,9.523215E2,6.008786E-3,1E0,4.6149455E-4,-7.956845E-4,-5.591764E-3],"split_indices":[46,63,0,19,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.21E2,4.13E2,8E0,1.89E2,2.24E2,1.8E2,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}},{"base_weights":[-1.1219947E-3,-1.22791715E-2,3.529104E-2,-1.02603985E-2,-8.1481645E-3,1.6230829E-2,4.2089853E-3,-1.0046504E-3,8.79183E-4,1.197237E-3,-9.624214E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,-1,-1,-1,-1],"loss_changes":[1.7147161E-1,1.01556286E-1,9.837186E-2,9.9395126E-2,0E0,1.4777642E-1,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5],"right_children":[2,4,6,8,-1,10,-1,-1,-1,-1,-1],"split_conditions":[1.619002E9,6.1681726E2,1.8770729E0,1.472533E3,-8.1481645E-3,3.768034E4,4.2089853E-3,-1.0046504E-3,8.79183E-4,1.197237E-3,-9.624214E-3],"split_indices":[7,68,52,63,0,4,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[4.2E2,3.22E2,9.8E1,3.19E2,3E0,7.3E1,2.5E1,2.32E2,8.7E1,7.1E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"11","size_leaf_vector":"1"}},{"base_weights":[-1.7868793E-3,-8.94641E-3,4.4112943E-2,-3.1522888E-4,-7.305416E-3,9.866656E-3,1.5135763E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,-1,-1,-1,-1],"loss_changes":[1.4367348E-1,1.2656964E-1,1.1242561E-1,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2],"right_children":[2,4,6,-1,-1,-1,-1],"split_conditions":[3.5928743E3,2.4045518E7,8.381319E3,-3.1522888E-4,-7.305416E-3,9.866656E-3,1.5135763E-3],"split_indices":[63,59,44,0,0,0,0],"split_type":[0,0,0,0,0,0,0],"sum_hessian":[4.35E2,3.77E2,5.8E1,3.72E2,5E0,3E0,5.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"7","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.6499995E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics index 80a382a27..7fd4636dd 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/dataproc.metrics @@ -1,116 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,duration_max,0.2218884,82414.25463743677,311681.84687962633,1504.0,4702.0,7190.0,20850.0,3183955.0 -1,peakExecutionMemory_max,0.074162014,1233353538.7048905,2601007219.6365647,0.0,85937632.0,471303176.0,1493186488.0,25501360736.0 -2,executorCPUTime_mean,0.07329501,2722.669419032867,7012.891604818392,23.83747911639131,585.7264150943396,1064.374717832957,2164.2514029180697,68323.50815850816 -3,duration_sum,0.06656143,16930121.96627319,45073353.38047576,3197.0,1082442.0,2678990.0,10368109.0,515693115.0 -4,shuffle_write_bw,0.052876204,88045.62818445469,93534.36606485402,0.0,876.6320096006982,52173.31607617228,163767.7222699914,324254.78942752053 -5,input_bytesRead_mean,0.04069925,15367291.833589979,13710603.535320966,21863.21726572529,5113159.273237375,11711469.0760095,21549125.812765956,92298335.07638542 -6,scan_bw,0.03297953,110505.86766836004,89612.1377495093,0.0,45187.54901663616,92722.52201161183,143609.24321367222,465063.5526726399 -7,memoryBytesSpilledRatio,0.021535791,1.1100054272857585,3.9835905644793064,0.0,0.0,0.0,0.0,43.971493342592005 -8,resultSize_max,0.01712785,10957325.65767285,22245624.20798865,2645.0,40208.0,1477382.0,12756524.0,263794922.0 -9,sw_writeTime_mean,0.016270768,131.60579301460444,570.3995830034571,0.0,16.563237774030355,33.421933085501855,70.6938202247191,6476.0359186655305 -10,jvmGCTime_mean,0.014246284,63.70179049174891,111.16738988131927,0.0,31.140514631685168,49.11107416029265,70.23493975903614,1390.436668309512 -11,scan_time,0.013151489,5138075.225969646,16908006.48547024,0.0,322677.0,911898.0,2940448.0,217849366.0 -12,duration_mean,0.009667594,4389.021479495678,8051.0668064364545,245.4699965313909,1580.207299270073,2302.596153846154,3826.246829791034,72233.70872350912 -13,resultSerializationTime_sum,0.009420892,142.54300168634063,201.73330481784959,0.0,27.0,68.0,152.0,1135.0 -14,duration_min,0.009281027,217.46880269814503,213.6000053492964,5.0,77.0,148.0,295.0,1779.0 -15,sr_fetchWaitTime_mean,0.008003406,282.57444030933357,2063.740776556858,0.0,0.0,0.014450867052023121,0.0744336569579288,26216.59561667383 -16,sw_recordsWritten_sum,0.007871636,2556345618.8684654,5824955615.583718,0.0,319167.0,53631567.0,2066354478.0,44204763473.0 -17,sr_localBytesRead_mean,0.0068754195,1550809.257874584,3239093.401866476,0.0,2585.3312883435583,284627.3147440794,1918199.7854019508,29629639.667323805 -18,numTasks_sum,0.006489214,3448.059021922428,7298.431350220087,4.0,527.0,1156.0,3101.0,86122.0 -19,executorDeserializeCPUTime_mean,0.0060304287,27.44671316347629,29.05547500604061,1.4147597693786034,6.826179604261796,19.729810568295115,37.08288043478261,246.07692307692307 -20,input_recordsRead_sum,0.0052329036,7537500441.369308,19524836240.42767,30000.0,410551846.0,1658123725.0,5999989709.0,230400923204.0 -21,executorRunTime_mean,0.005220965,4297.896908257712,8062.7718056424565,226.90115532734274,1490.5288200108755,2236.8281325012003,3741.984555984556,72194.51503203549 -22,sr_remoteBytesRead_mean,0.0048081777,8621230.876874011,20763598.714249212,0.0,14815.464675201518,1045794.9007633588,7696222.868940513,207400140.1710202 -23,sr_localBlocksFetched_sum,0.004553382,34560.099494097805,81564.66670257768,0.0,329.0,5582.0,30418.0,870491.0 -24,executorDeserializeTime_mean,0.004410795,77.619961219705,78.66729885547291,1.2399579021224347,16.74,57.15866922584773,111.59245283018868,438.3862068965517 -25,sr_localBytesReadRatio,0.004280239,0.13775142996534326,0.21834409881656594,0.0,0.0003435677540369934,0.052422868005860185,0.18397368378307108,1.6954497523177325 -26,sr_totalBytesRead_mean,0.0035336576,10172040.134748595,23817088.532298744,0.0,19575.48201438849,1695974.662708024,9072792.773504274,237029779.838344 -27,sr_remoteBlocksFetched_sum,0.0030647377,224234.64755480608,509217.62116016017,0.0,2164.0,37041.0,193536.0,3942472.0 -28,data_size,0.0029502832,650411028780.6053,2082773012482.3335,0.0,21721183788.0,89736852644.0,333792838797.0,21374406089219.0 -29,sr_remoteBytesReadRatio,0.0025289594,0.6856243468838018,1.1473671005608985,0.0,0.0015316802906526895,0.13301082808430215,1.0519672879672015,11.86796722526039 -30,shuffle_read_bw,0.0025215303,276816955.4836904,847565126.6737152,0.0,0.0,4930489.4375,137218859.4920635,10530883927.666666 -31,sqlOp_Expand,0.0023906063,0.11129848229342328,0.3147669133570336,0.0,0.0,0.0,0.0,1.0 -32,sqlOp_SortMergeJoin,0.0023107673,0.41652613827993257,0.49339906934430416,0.0,0.0,0.0,1.0,1.0 -33,sw_bytesWrittenRatio,0.0021444862,0.8014652434613514,1.2780756184305504,0.0,0.0023701803469738034,0.22863716255153424,1.206131247096677,13.563416977578123 -34,sqlOp_AQEShuffleRead,0.0013885087,0.7065767284991569,0.45571511734167724,0.0,0.0,1.0,1.0,1.0 -35,sqlOp_Scan parquet,0.0012765201,0.5311973018549747,0.49944707588925713,0.0,0.0,1.0,1.0,1.0 -36,sr_totalBytesReadRatio,0.0012660599,0.823375776849145,1.319042835980761,0.0,0.002400770847119191,0.22866221128988315,1.2670990125430908,13.563416977578123 -37,sw_bytesWritten_mean,0.0012242775,9961926.690653043,23574771.301399454,0.0,19575.48201438849,1687125.7618110236,9963399.731213873,237029779.838344 -38,diskBytesSpilled_mean,0.0009520437,4546045.185000375,15069027.640347425,0.0,0.0,0.0,0.0,111631484.38187319 -39,sqlOp_SubqueryBroadcast,0.0009050891,0.7976391231028668,0.40209900029018336,0.0,1.0,1.0,1.0,1.0 -40,diskBytesSpilledRatio,0.0007024729,0.24474497918684088,0.8105072165034726,0.0,0.0,0.0,0.0,6.729163387759685 -41,sqlOp_Subquery,0.00058933673,0.09106239460370995,0.28794070420987344,0.0,0.0,0.0,0.0,1.0 -42,sqlOp_HashAggregate,0.00046937025,0.9797639123102867,0.14092579360637603,0.0,1.0,1.0,1.0,1.0 -43,sqlOp_Sort,0.00044839192,0.6053962900505903,0.4891780403788745,0.0,0.0,1.0,1.0,1.0 -44,memoryBytesSpilled_mean,0.00013901091,18194132.33421661,71075266.1814507,0.0,0.0,0.0,0.0,724082083.4348422 -45,numExecutors,3.914403e-05,8.062394603709949,0.3504133651237026,8.0,8.0,8.0,8.0,14.0 -46,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_Window,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -48,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -51,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -82,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -83,sqlOp_Filter,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -84,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -85,sqlOp_Project,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -86,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -89,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,executorMemory,0.0,40960.0,0.0,40960.0,40960.0,40960.0,40960.0,40960.0 -92,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,maxOnHeapMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 -94,maxMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 -95,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -96,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_BroadcastNestedLoopJoin,0.0,0.04047217537942664,0.1972302630779745,0.0,0.0,0.0,0.0,1.0 -103,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,sqlOp_Exchange,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 -108,platform_dataproc,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -109,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,sqlOp_BroadcastExchange,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 -111,sqlOp_BroadcastHashJoin,0.0,0.9494097807757167,0.21934443720756758,0.0,1.0,1.0,1.0,1.0 -112,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -113,sqlOp_ColumnarToRow,0.0,0.9898819561551433,0.10016280444089157,0.0,1.0,1.0,1.0,1.0 +0,duration_max,0.2244238,82414.25463743677,311681.84687962633,1504.0,4702.0,7190.0,20850.0,3183955.0 +1,peakExecutionMemory_max,0.08521888,1233353538.7048905,2601007219.6365647,0.0,85937632.0,471303176.0,1493186488.0,25501360736.0 +2,executorCPUTime_mean,0.06784733,2722.669419032867,7012.891604818392,23.83747911639131,585.7264150943396,1064.374717832957,2164.2514029180697,68323.50815850816 +3,duration_sum,0.0671249,16930121.96627319,45073353.38047576,3197.0,1082442.0,2678990.0,10368109.0,515693115.0 +4,shuffle_write_bw,0.046630975,88045.62818445469,93534.36606485402,0.0,876.6320096006982,52173.31607617228,163767.7222699914,324254.78942752053 +5,scan_bw,0.03046397,110505.86766836004,89612.1377495093,0.0,45187.54901663616,92722.52201161183,143609.24321367222,465063.5526726399 +6,input_bytesRead_mean,0.02720214,15367291.833589979,13710603.535320966,21863.21726572529,5113159.273237375,11711469.0760095,21549125.812765956,92298335.07638542 +7,memoryBytesSpilledRatio,0.017770054,1.1100054272857585,3.9835905644793064,0.0,0.0,0.0,0.0,43.971493342592005 +8,sw_writeTime_mean,0.015521495,131.60579301460444,570.3995830034571,0.0,16.563237774030355,33.421933085501855,70.6938202247191,6476.0359186655305 +9,resultSize_max,0.015324278,10957325.65767285,22245624.20798865,2645.0,40208.0,1477382.0,12756524.0,263794922.0 +10,scan_time,0.014721475,5138075.225969646,16908006.485470243,0.0,322677.0,911898.0,2940448.0,217849366.0 +11,sw_recordsWritten_sum,0.0113062365,2556345618.8684654,5824955615.583718,0.0,319167.0,53631567.0,2066354478.0,44204763473.0 +12,jvmGCTime_mean,0.011148932,63.70179049174891,111.16738988131927,0.0,31.140514631685168,49.11107416029265,70.23493975903614,1390.436668309512 +13,sr_localBytesRead_mean,0.009041049,1550809.2578745843,3239093.401866476,0.0,2585.3312883435583,284627.3147440794,1918199.7854019508,29629639.667323805 +14,duration_min,0.008132119,217.46880269814503,213.6000053492964,5.0,77.0,148.0,295.0,1779.0 +15,duration_mean,0.007919153,4389.021479495678,8051.0668064364545,245.4699965313909,1580.207299270073,2302.596153846154,3826.246829791034,72233.70872350912 +16,numTasks_sum,0.0077487556,3448.059021922428,7298.431350220087,4.0,527.0,1156.0,3101.0,86122.0 +17,resultSerializationTime_sum,0.0071508945,142.54300168634063,201.7333048178496,0.0,27.0,68.0,152.0,1135.0 +18,sr_remoteBytesRead_mean,0.0066635134,8621230.876874011,20763598.71424921,0.0,14815.464675201518,1045794.9007633588,7696222.868940513,207400140.1710202 +19,executorDeserializeTime_mean,0.006357161,77.61996121970503,78.66729885547291,1.2399579021224347,16.74,57.15866922584773,111.59245283018868,438.3862068965517 +20,input_recordsRead_sum,0.0053491383,7537500441.369308,19524836240.427666,30000.0,410551846.0,1658123725.0,5999989709.0,230400923204.0 +21,sr_localBytesReadRatio,0.005151674,0.13775142996534326,0.21834409881656594,0.0,0.0003435677540369934,0.052422868005860185,0.18397368378307108,1.6954497523177325 +22,sr_fetchWaitTime_mean,0.0043934365,282.57444030933357,2063.740776556858,0.0,0.0,0.014450867052023121,0.0744336569579288,26216.59561667383 +23,sr_localBlocksFetched_sum,0.00412811,34560.099494097805,81564.66670257768,0.0,329.0,5582.0,30418.0,870491.0 +24,executorDeserializeCPUTime_mean,0.0032041718,27.44671316347629,29.05547500604061,1.4147597693786034,6.826179604261796,19.729810568295115,37.08288043478261,246.07692307692307 +25,sr_remoteBytesReadRatio,0.0031569276,0.6856243468838018,1.1473671005608985,0.0,0.0015316802906526895,0.13301082808430215,1.0519672879672015,11.86796722526039 +26,data_size,0.0028129178,650411028780.6053,2082773012482.3337,0.0,21721183788.0,89736852644.0,333792838797.0,21374406089219.0 +27,executorRunTime_mean,0.0027795013,4297.896908257712,8062.7718056424565,226.90115532734274,1490.5288200108755,2236.8281325012003,3741.984555984556,72194.51503203549 +28,shuffle_read_bw,0.002563467,276816955.4836903,847565126.6737152,0.0,0.0,4930489.4375,137218859.4920635,10530883927.666666 +29,sw_bytesWrittenRatio,0.0024008835,0.8014652434613513,1.2780756184305502,0.0,0.0023701803469738034,0.22863716255153424,1.206131247096677,13.563416977578123 +30,sr_totalBytesReadRatio,0.002361337,0.8233757768491451,1.319042835980761,0.0,0.002400770847119191,0.22866221128988315,1.2670990125430908,13.563416977578123 +31,sr_totalBytesRead_mean,0.0023611812,10172040.134748595,23817088.532298747,0.0,19575.48201438849,1695974.662708024,9072792.773504274,237029779.838344 +32,sw_bytesWritten_mean,0.0019182155,9961926.690653043,23574771.301399454,0.0,19575.48201438849,1687125.7618110236,9963399.731213873,237029779.838344 +33,sqlOp_SubqueryBroadcast,0.0017531143,0.7976391231028668,0.4020990002901833,0.0,1.0,1.0,1.0,1.0 +34,sqlOp_Expand,0.0017457285,0.11129848229342328,0.3147669133570336,0.0,0.0,0.0,0.0,1.0 +35,sr_remoteBlocksFetched_sum,0.0017040576,224234.64755480608,509217.62116016017,0.0,2164.0,37041.0,193536.0,3942472.0 +36,sqlOp_AQEShuffleRead,0.0011831847,0.7065767284991569,0.4557151173416773,0.0,0.0,1.0,1.0,1.0 +37,sqlOp_SortMergeJoin,0.001145791,0.41652613827993257,0.49339906934430416,0.0,0.0,0.0,1.0,1.0 +38,diskBytesSpilled_mean,0.000899613,4546045.185000375,15069027.640347425,0.0,0.0,0.0,0.0,111631484.38187319 +39,sqlOp_Scan parquet,0.0005778013,0.5311973018549747,0.49944707588925713,0.0,0.0,1.0,1.0,1.0 +40,memoryBytesSpilled_mean,0.00056815345,18194132.33421661,71075266.1814507,0.0,0.0,0.0,0.0,724082083.4348422 +41,sqlOp_Subquery,0.0005635291,0.09106239460370995,0.28794070420987344,0.0,0.0,0.0,0.0,1.0 +42,diskBytesSpilledRatio,0.00055829406,0.24474497918684082,0.8105072165034726,0.0,0.0,0.0,0.0,6.729163387759685 +43,sqlOp_Sort,0.00052243046,0.6053962900505903,0.4891780403788745,0.0,0.0,1.0,1.0,1.0 +44,sqlOp_HashAggregate,0.00018995629,0.9797639123102867,0.14092579360637603,0.0,1.0,1.0,1.0,1.0 +45,sqlOp_Exchange,0.0,0.9898819561551433,0.10016280444089158,0.0,1.0,1.0,1.0,1.0 +46,sqlOp_ColumnarToRow,0.0,0.9898819561551433,0.10016280444089158,0.0,1.0,1.0,1.0,1.0 +47,sqlOp_BroadcastExchange,0.0,0.9494097807757167,0.2193444372075676,0.0,1.0,1.0,1.0,1.0 +48,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_BroadcastNestedLoopJoin,0.0,0.04047217537942664,0.1972302630779745,0.0,0.0,0.0,0.0,1.0 +50,sqlOp_BroadcastHashJoin,0.0,0.9494097807757167,0.2193444372075676,0.0,1.0,1.0,1.0,1.0 +51,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +52,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +53,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +54,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +58,sqlOp_Filter,0.0,0.9898819561551433,0.10016280444089158,0.0,1.0,1.0,1.0,1.0 +59,sqlOp_Project,0.0,0.9898819561551433,0.10016280444089158,0.0,1.0,1.0,1.0,1.0 +60,numExecutors,0.0,8.062394603709949,0.35041336512370264,8.0,8.0,8.0,8.0,14.0 +61,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +62,maxMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 +63,maxOnHeapMem,0.0,25581060096.0,0.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0,25581060096.0 +64,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,executorMemory,0.0,40960.0,0.0,40960.0,40960.0,40960.0,40960.0,40960.0 +66,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +67,sqlOp_LocalTableScan,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +71,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,platform_dataproc,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +75,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_Window,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_WindowGroupLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg index 3e3fd1e94..acb8bb5f0 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0460410565","gamma":"0.00246754545","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0460410565","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"5","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00246754545","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.871091902"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0460410565","gamma":"0.00246754545","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0460410565","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"5","max_leaves":"0","min_child_weight":"4","min_split_loss":"0.00246754545","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.871091902"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json index 69e31eb4b..377b390cd 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Scan parquet","sqlOp_Sort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_emr","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_onprem","sqlOp_BatchEvalPython","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Filter","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_LocalTableScan","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_Project","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.1085903E-2,-2.2772919E-1,4.3721128E-1,-3.6880127E-1,-1.9833224E-3,2.2673543E-1,8.4530383E-1,-4.9474227E-1,-1.5492713E-1,-2.1003313E-1,9.9773996E-2,9.0155266E-2,3.040459E-1,5.9119016E-1,1.0942323E0,-4.0589273E-1,-7.4147564E-1,2.5695986E-3,-2.9754654E-1,1.3038836E-3,-2.6457384E-1,1.7128408E-1,-1.7006904E-2,2.3424982E-1,-1.2763274E-1,3.7088796E-1,1.0321534E-2,2.9997287E-2,1.344041E-2,5.8344405E-2,2.7380887E-2,-2.0452904E-2,1.2894061E-3,-3.7609708E-2,-1.8979337E-2,-5.047128E-3,6.992408E-3,-1.9480048E-2,-6.67327E-3,-1.9714128E-2,-8.620859E-3,-1.1689346E-3,9.905409E-3,-3.9632274E-3,4.7806343E-3,1.5076465E-2,4.6091946E-3,-1.07842935E-2,-1.5311907E-3,1.887391E-2,2.5142783E-3,5.6988043E-3,-4.8434185E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.795136E1,5.381317E0,8.483597E0,2.7246437E0,1.4182148E0,6.9060564E-1,1.503809E0,1.1239796E0,8.992482E-1,2.9349154E-1,3.7870854E-1,8.458316E-1,8.403976E-1,1.5292072E-1,6.2901115E-1,8.442049E-1,6.0456276E-2,3.516622E-1,3.4104168E-1,0E0,1.4756632E-1,2.4698633E-1,1.5918228E-1,1.6726196E-1,1.0287437E-1,4.098711E-1,1.3097732E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,1.1306552E-1,2.0224316E-1,1.9462195E2,2.176974E8,4.638253E2,1.4968063E3,5.039375E-3,1.5248811E1,4.0072045E0,1.47615E7,1.1936242E7,6.191915E0,1.3589149E6,2.65826E6,7.725518E-4,4.214E3,6.135367E9,1.281E3,1.3038836E-3,8E0,6.3972777E4,3.643059E5,9E0,1.1E1,1.1107071E7,3.7808718E11,2.9997287E-2,1.344041E-2,5.8344405E-2,2.7380887E-2,-2.0452904E-2,1.2894061E-3,-3.7609708E-2,-1.8979337E-2,-5.047128E-3,6.992408E-3,-1.9480048E-2,-6.67327E-3,-1.9714128E-2,-8.620859E-3,-1.1689346E-3,9.905409E-3,-3.9632274E-3,4.7806343E-3,1.5076465E-2,4.6091946E-3,-1.07842935E-2,-1.5311907E-3,1.887391E-2,2.5142783E-3,5.6988043E-3,-4.8434185E-3],"split_indices":[53,40,39,53,7,53,4,39,57,55,33,46,58,48,48,39,10,5,0,0,3,29,29,3,3,9,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.68E2,1E2,1.03E2,6.5E1,6.7E1,3.3E1,6.4E1,3.9E1,2.1E1,4.4E1,2.5E1,4.2E1,1.8E1,1.5E1,4.9E1,1.5E1,1.9E1,2E1,4E0,1.7E1,2.7E1,1.7E1,1.5E1,1E1,3.4E1,8E0,1.4E1,4E0,1E1,5E0,4.5E1,4E0,1.1E1,4E0,1.1E1,8E0,1E1,1E1,4E0,1.3E1,5E0,2.2E1,1.1E1,6E0,8E0,7E0,4E0,6E0,3E1,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[8.05193E-3,-1.8821597E-1,4.5129177E-1,-3.3232948E-1,2.6672075E-2,2.0962085E-1,8.257544E-1,-4.4819686E-1,-1.48785E-1,-5.948454E-2,1.8264762E-1,-2.3600906E-3,2.6519588E-1,2.0474736E-2,8.9396125E-1,-1.0349917E-1,-5.107083E-1,4.05704E-4,-2.9799333E-1,-1.6677873E-1,8.952956E-2,2.4102871E-1,-1.2398086E-3,-9.103933E-3,7.617095E-3,3.3745804E-1,3.086987E-2,2.5632085E-2,9.841587E-1,-1.0999542E-2,1.2592459E-3,-2.6992962E-2,-1.3329538E-2,-4.1856277E-3,8.950496E-3,-1.8250458E-2,-6.005194E-3,-1.0323307E-2,2.4856112E-3,1.14772655E-2,1.849914E-3,3.6762117E-3,1.2542095E-2,2.5272542E-3,1.785252E-2,6.5926206E-3,-3.986938E-3,5.670145E-2,3.481712E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,-1,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4095217E1,5.954771E0,7.519127E0,2.3789997E0,1.059054E0,6.250105E-1,4.6012306E-1,1.4554701E0,1.0241038E0,8.3033735E-1,3.4907818E-1,4.2612413E-1,7.0076513E-1,0E0,2.733841E-1,2.1915147E-1,8.165121E-1,4.4283566E-1,3.275318E-1,3.9253372E-1,1.6342817E-1,8.95313E-2,0E0,0E0,0E0,4.3348217E-1,1.5753034E-1,0E0,1.9645119E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,-1,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.797104E2,2.1619815E-1,2.0224316E-1,1.9462195E2,2.2599022E8,4.638253E2,2.3616267E8,1.1266387E10,1.5248811E1,1.0289595E6,1.4E1,2.4223125E5,2.1030048E10,2.0474736E-2,2.2324745E5,4.5474484E-3,1.021924E6,6.135367E9,3.03E9,9.74979E6,1.0632315E6,3.0933383E-1,-1.2398086E-3,-9.103933E-3,7.617095E-3,7.5E2,3.7578902E0,2.5632085E-2,2.9898608E-1,-1.0999542E-2,1.2592459E-3,-2.6992962E-2,-1.3329538E-2,-4.1856277E-3,8.950496E-3,-1.8250458E-2,-6.005194E-3,-1.0323307E-2,2.4856112E-3,1.14772655E-2,1.849914E-3,3.6762117E-3,1.2542095E-2,2.5272542E-3,1.785252E-2,6.5926206E-3,-3.986938E-3,5.670145E-2,3.481712E-2],"split_indices":[53,42,39,53,12,53,7,32,57,34,3,29,5,0,29,40,30,5,5,9,49,43,0,0,0,0,58,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.91E2,8.4E1,1.14E2,7.7E1,5.2E1,3.2E1,6.9E1,4.5E1,5E1,2.7E1,1.1E1,4.1E1,6E0,2.6E1,1.1E1,5.8E1,2.3E1,2.2E1,2.9E1,2.1E1,2.1E1,6E0,5E0,6E0,3.1E1,1E1,7E0,1.9E1,5E0,6E0,4.2E1,1.6E1,1.6E1,7E0,1.3E1,9E0,2.3E1,6E0,4E0,1.7E1,4E0,1.7E1,5E0,2.6E1,5E0,5E0,7E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[3.3098315E-3,-2.2334237E-1,3.5490754E-1,-3.068199E-1,6.591925E-2,1.780472E-1,7.461082E-1,-3.6551002E-1,-3.4820978E-2,-7.2370075E-2,1.0520874E-1,5.880241E-2,2.6542E-1,1.3290277E-2,8.581645E-1,-4.3094838E-1,-1.8536793E-1,1.3821995E-1,-1.478602E-1,-1.4917747E-3,-4.5058155E-3,1.4362985E-1,-5.3233947E-3,1.5904938E-1,-1.5615053E-1,3.257397E-1,1.3522787E-3,2.1919698E-2,4.4838067E-2,-1.6532997E-2,-2.9766759E-2,-4.3412945E-3,-1.78364E-2,1.1596651E-2,-1.1883961E-3,6.404258E-4,-1.0595716E-2,3.7644408E-4,8.464975E-3,1.3952501E-2,2.8567943E-3,-1.2937324E-2,2.295715E-3,1.8079627E-2,5.48387E-3,4.8313425E-3,-4.7192737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1755054E1,4.0286016E0,7.27026E0,2.0465841E0,2.1408E-1,7.6579905E-1,1.4438534E0,1.1696653E0,4.8988712E-1,6.000217E-3,2.7114466E-1,7.3593223E-1,6.844847E-1,0E0,6.8577385E-1,9.6060944E-1,5.1548564E-1,1.9293371E-1,2.028482E-1,0E0,0E0,1.4016885E-1,0E0,3.024519E-1,2.9692715E-1,4.3738198E-1,1.07573576E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1309677E2,4.9615598E-1,1.5972575E0,2.727992E6,2.0358758E8,4.638253E2,2.744E3,1.1670399E-1,1.6E1,2.0074467E5,1.1771224E8,1.409674E7,6.191915E0,1.3290277E-2,2.2324745E5,3.98231E-2,3.4047124E-1,1.9096774E0,3.1791415E11,-1.4917747E-3,-4.5058155E-3,1.95E2,-5.3233947E-3,9E0,2.136E3,1.8751362E7,3.7808718E11,2.1919698E-2,4.4838067E-2,-1.6532997E-2,-2.9766759E-2,-4.3412945E-3,-1.78364E-2,1.1596651E-2,-1.1883961E-3,6.404258E-4,-1.0595716E-2,3.7644408E-4,8.464975E-3,1.3952501E-2,2.8567943E-3,-1.2937324E-2,2.295715E-3,1.8079627E-2,5.48387E-3,4.8313425E-3,-4.7192737E-3],"split_indices":[53,40,42,1,7,53,2,40,3,29,33,46,58,0,29,43,43,54,32,0,0,0,0,3,0,45,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.65E2,1.06E2,1.28E2,3.7E1,7.4E1,3.2E1,1.05E2,2.3E1,8E0,2.9E1,3.2E1,4.2E1,7E0,2.5E1,7.6E1,2.9E1,9E0,1.4E1,4E0,4E0,2.5E1,4E0,2.2E1,1E1,3.4E1,8E0,7E0,1.8E1,5.9E1,1.7E1,2.1E1,8E0,5E0,4E0,5E0,9E0,6E0,1.9E1,8E0,1.4E1,6E0,4E0,2.5E1,9E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.1728342E-3,-1.34235E-1,5.659846E-1,-3.1162438E-1,5.227209E-2,2.7925435E-1,9.2783916E-1,-3.8934636E-1,-1.0033659E-2,-6.283367E-2,1.5673715E-1,2.0447267E-2,4.205759E-1,2.1932395E-2,4.594042E-2,-3.496029E-1,-3.150391E-2,-7.901294E-2,1.0201296E-2,-1.6959472E-1,3.504982E-2,2.0324224E-1,6.772725E-3,3.7339362E-3,-2.4734316E-3,1.1890882E-2,2.2472931E-2,-2.0664664E-2,-1.0800635E-2,3.0884955E-3,-8.379436E-3,-4.7979713E-3,-1.5293842E-2,7.199374E-3,-2.7525688E-3,1.2581257E-2,2.0711978E-3,-7.47909E-3,3.8247998E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1084963E1,7.559527E0,5.239893E0,2.7386007E0,1.3565956E0,1.1252258E0,2.5128937E-1,8.508129E-1,4.169024E-1,5.70252E-1,4.1009593E-1,5.8340065E-2,9.315419E-2,0E0,0E0,8.5338783E-1,0E0,3.131229E-1,0E0,2.4177301E-1,3.4300086E-1,4.8635304E-1,2.0716569E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,1.7613442E-1,7.2610445E8,2.727992E6,3.614121E8,2.6801108E2,2.2324745E5,2.771097E0,1.22E2,6.75553E5,2.3601532E0,1.2495134E6,1.9215278E5,2.1932395E-2,4.594042E-2,1.9268292E0,-3.150391E-2,4.7E3,1.0201296E-2,2.32E3,4.045147E0,2.3103538E7,2.697134E8,3.7339362E-3,-2.4734316E-3,1.1890882E-2,2.2472931E-2,-2.0664664E-2,-1.0800635E-2,3.0884955E-3,-8.379436E-3,-4.7979713E-3,-1.5293842E-2,7.199374E-3,-2.7525688E-3,1.2581257E-2,2.0711978E-3,-7.47909E-3,3.8247998E-3],"split_indices":[43,42,12,1,12,4,29,54,8,9,54,46,29,0,0,54,0,0,0,2,55,46,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,2.27E2,5.2E1,1.16E2,1.11E2,3E1,2.2E1,9.2E1,2.4E1,5.3E1,5.8E1,1.1E1,1.9E1,4E0,1.8E1,8.3E1,9E0,1.9E1,5E0,2.5E1,2.8E1,4.4E1,1.4E1,6E0,5E0,7E0,1.2E1,4.3E1,4E1,8E0,1.1E1,1.9E1,6E0,1.2E1,1.6E1,3E1,1.4E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-9.496169E-3,-1.7978296E-1,4.131077E-1,-3.0226317E-1,5.437915E-3,1.7675185E-1,7.5473183E-1,-3.7400904E-1,-2.0408144E-2,-8.356252E-2,1.2569034E-1,8.318677E-2,3.424772E-1,1.5751595E-2,8.565782E-1,-4.3764114E-1,-2.0838907E-1,-8.013996E-2,6.6592176E-3,-2.0129164E-1,4.29184E-2,-1.5173384E-2,1.9371966E-1,2.4396244E-1,-2.5476754E-2,2.6302315E-2,1.2518758E-1,4.846919E-2,2.8879559E-2,-1.0297241E-2,-2.3558494E-2,3.8563093E-4,-1.5160369E-2,4.851225E-3,-6.290731E-3,-1.1396859E-2,1.1025744E-3,9.115021E-3,-1.7169984E-3,2.3040317E-3,-5.363087E-3,1.1058021E-2,1.0156563E-3,5.296566E-3,1.5563398E-2,-1.3192813E-2,2.656533E-3,8.729507E-4,8.8788215E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9792366E1,4.448078E0,6.2454424E0,2.3785553E0,8.560019E-1,7.158036E-1,1.0516815E0,8.994427E-1,2.5805157E-1,6.936016E-1,3.2875675E-1,5.6526977E-1,7.736615E-1,0E0,4.642582E-1,9.4713306E-1,7.364912E-1,2.1351448E-1,0E0,2.559178E-1,2.9344204E-1,8.511527E-2,1.7812508E-1,1.18774116E-1,4.5082474E-1,0E0,6.821506E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.797104E2,2.1619815E-1,2.0224316E-1,2.219883E6,2.7538432E8,7.230905E2,2.3616267E8,1.9462195E2,1E0,4.343271E5,1.571112E5,1.2269897E3,9.521E3,1.5751595E-2,2.9898608E-1,2.4747229E2,1.2288136E1,2.207E3,6.6592176E-3,1.1709996E0,8.237578E4,1.2315645E5,2.8808496E7,4.50805E-1,1.3548306E3,2.6302315E-2,2.2894168E0,4.846919E-2,2.8879559E-2,-1.0297241E-2,-2.3558494E-2,3.8563093E-4,-1.5160369E-2,4.851225E-3,-6.290731E-3,-1.1396859E-2,1.1025744E-3,9.115021E-3,-1.7169984E-3,2.3040317E-3,-5.363087E-3,1.1058021E-2,1.0156563E-3,5.296566E-3,1.5563398E-2,-1.3192813E-2,2.656533E-3,8.729507E-4,8.8788215E-3],"split_indices":[53,42,39,1,7,53,7,53,108,48,29,4,2,0,39,56,57,0,0,40,29,29,9,58,4,0,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.95E2,7.8E1,1.17E2,7.8E1,4.7E1,3.1E1,9.3E1,2.4E1,4.5E1,3.3E1,3.1E1,1.6E1,7E0,2.4E1,6.6E1,2.7E1,1.8E1,6E0,2.3E1,2.2E1,1.1E1,2.2E1,1.2E1,1.9E1,7E0,9E0,1.1E1,1.3E1,1.8E1,4.8E1,1E1,1.7E1,4E0,1.4E1,1.9E1,4E0,7E0,1.5E1,7E0,4E0,1.7E1,5E0,6E0,6E0,4E0,1.5E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-8.5547956E-4,-1.5124227E-1,4.1323495E-1,-2.9142067E-1,2.8436407E-2,1.8857329E-1,7.1153224E-1,-3.9878193E-1,-1.07372575E-1,-8.622849E-2,9.1209516E-2,2.3138136E-1,-5.247452E-3,1.2566412E-2,7.8537935E-1,-1.9025414E-1,-4.6601295E-1,1.00603355E-2,-3.0616936E-1,-1.491725E-1,7.3611313E-3,1.2469521E-1,-6.3201636E-2,3.185047E-2,3.0150485E-1,5.1404282E-2,6.302224E-1,-1.228212E-2,3.5602557E-3,-7.937771E-3,-2.2698479E-2,-2.6173724E-3,8.491763E-3,-1.7663874E-2,-8.320865E-3,-2.4596206E-4,-1.0359407E-2,3.0030967E-3,9.118384E-3,2.6758283E-3,-6.483959E-3,-9.433184E-3,9.042378E-3,2.0103846E-2,6.223371E-3,1.650125E-2,3.111349E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7063227E1,5.0708885E0,4.757988E0,2.1900196E0,6.4847547E-1,5.8328295E-1,7.896948E-1,9.067278E-1,1.0069275E0,5.195675E-1,3.0777368E-1,5.220854E-1,0E0,0E0,6.32082E-1,4.0961158E-1,3.3503342E-1,3.3717495E-1,8.6182475E-2,2.8237534E-1,0E0,1.964702E-1,1.11781254E-1,4.6873865E-1,5.7040143E-1,0E0,5.452156E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,2.4210773E-2,2.0224316E-1,1.7785811E2,2.4118989E8,9.456436E0,1.7246376E0,2.5041176E2,1.9587677E1,7.004062E5,2.835443E0,3.386E3,-5.247452E-3,1.2566412E-2,3.0485725E0,1.0665042E8,1.1537334E0,6.87E3,1.281E3,2.8507206E-1,7.3611313E-3,2.2599022E8,3.7741747E8,1.4E1,3.5630266E8,5.1404282E-2,6.4495115E0,-1.228212E-2,3.5602557E-3,-7.937771E-3,-2.2698479E-2,-2.6173724E-3,8.491763E-3,-1.7663874E-2,-8.320865E-3,-2.4596206E-4,-1.0359407E-2,3.0030967E-3,9.118384E-3,2.6758283E-3,-6.483959E-3,-9.433184E-3,9.042378E-3,2.0103846E-2,6.223371E-3,1.650125E-2,3.111349E-2],"split_indices":[53,39,39,53,7,58,54,56,57,48,54,2,0,0,55,7,55,10,0,43,0,12,7,3,7,0,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2E2,7.2E1,1.12E2,8.8E1,4.2E1,3E1,7E1,4.2E1,3.1E1,5.7E1,3.7E1,5E0,5E0,2.5E1,1.8E1,5.2E1,2.7E1,1.5E1,2.5E1,6E0,4.7E1,1E1,1E1,2.7E1,6E0,1.9E1,1.4E1,4E0,5E0,4.7E1,2E1,7E0,8E0,7E0,9E0,1.6E1,2.7E1,2E1,4E0,6E0,4E0,6E0,1.4E1,1.3E1,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.5965978E-2,-1.6973527E-1,3.3239034E-1,-2.8115204E-1,-2.5269142E-4,1.3585745E-1,6.358597E-1,-3.8342574E-1,-9.037479E-2,-7.975257E-2,6.9061376E-2,-1.0452388E-2,1.96517E-1,3.9780077E-1,7.723055E-1,-1.1247442E-1,-4.3754542E-1,8.296243E-2,-2.4290015E-1,-1.351644E-1,6.675223E-2,1.4069478E-1,-2.2327319E-2,1.33327E-1,-1.1178186E-2,2.961138E-1,5.2235503E-2,2.0015052E-2,1.0232433E-2,1.873837E-2,8.917609E-1,-1.10272635E-2,1.4101589E-3,-2.318691E-2,-9.879268E-3,-4.7807943E-4,8.648084E-3,-1.6017655E-2,-5.3324327E-3,-1.2605592E-3,-7.723888E-3,-5.3495535E-4,6.788633E-3,-3.0176472E-3,9.448006E-3,-3.1434821E-3,3.7269583E-3,1.0321122E-2,-2.6178758E-3,1.9719439E-2,7.449925E-3,8.453698E-3,-1.5023103E-3,4.7617476E-2,2.9206924E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6685612E1,3.3800216E0,6.175993E0,2.0725899E0,4.0226096E-1,5.78935E-1,1.0245438E0,9.772539E-1,1.0512462E0,2.8503782E-1,2.584542E-1,7.004698E-1,6.455511E-1,1.3573647E-2,7.280245E-1,2.4366336E-1,7.463503E-1,1.8794858E-1,2.3991823E-1,8.3242476E-2,6.495506E-2,3.061618E-1,9.094318E-2,2.3735417E-1,0E0,4.1003275E-1,2.2897764E-1,0E0,0E0,0E0,4.0273666E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,2.4210773E-2,1.8829846E-1,1.9462195E2,2.4118989E8,3.165913E6,3.856E3,1.1266387E10,1.5248811E1,7.598717E5,3.2744168E6,1.1936242E7,3.3996282E0,1.3589149E6,2.174459E5,6.846054E7,6.608892E2,6.135367E9,1.281E3,3.3529866E-2,1.0289595E6,6E0,4.083633E6,2.3797293E0,-1.1178186E-2,7.5547155E6,3.7808718E11,2.0015052E-2,1.0232433E-2,1.873837E-2,2.9898608E-1,-1.10272635E-2,1.4101589E-3,-2.318691E-2,-9.879268E-3,-4.7807943E-4,8.648084E-3,-1.6017655E-2,-5.3324327E-3,-1.2605592E-3,-7.723888E-3,-5.3495535E-4,6.788633E-3,-3.0176472E-3,9.448006E-3,-3.1434821E-3,3.7269583E-3,1.0321122E-2,-2.6178758E-3,1.9719439E-2,7.449925E-3,8.453698E-3,-1.5023103E-3,4.7617476E-2,2.9206924E-2],"split_indices":[53,39,39,53,7,1,2,32,57,48,51,46,58,48,29,7,4,5,0,39,34,3,52,54,0,33,32,0,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.78E2,1.04E2,1.07E2,7.1E1,6.4E1,4E1,6.9E1,3.8E1,3.3E1,3.8E1,1.9E1,4.5E1,1.6E1,2.4E1,1.2E1,5.7E1,1.8E1,2E1,2.4E1,9E0,2.1E1,1.7E1,1.2E1,7E0,2.6E1,1.9E1,1.2E1,4E0,7E0,1.7E1,6E0,6E0,4.3E1,1.4E1,1E1,8E0,1E1,1E1,6E0,1.8E1,5E0,4E0,5E0,1.6E1,1.2E1,5E0,8E0,4E0,1.2E1,1.4E1,7E0,1.2E1,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[5.995876E-4,-1.7347676E-1,3.2858977E-1,-2.772065E-1,4.059655E-3,1.5712646E-1,6.180375E-1,-3.4299368E-1,-1.8954381E-2,-1.0166349E-1,9.792302E-2,3.0239883E-1,1.03181146E-1,8.146918E-1,4.800139E-1,-3.062804E-1,-2.7904196E-2,8.940504E-2,-1.0274004E-1,-2.1722323E-1,8.394098E-2,-3.0153783E-3,1.3962671E-1,1.4694941E-1,2.215816E-2,1.7923576E-1,-8.34605E-2,4.802176E-2,2.7501302E-2,1.03110215E-2,5.2213454E-1,-1.9017814E-2,-9.947347E-3,-2.028609E-3,9.575163E-3,-8.734323E-3,2.0642108E-3,2.2078224E-4,-1.2570478E-2,-9.5630065E-4,9.648653E-3,8.224101E-3,1.5246839E-3,2.3588135E-3,9.310498E-3,5.70954E-3,1.8742599E-2,-1.0078216E-2,3.8327414E-3,1.3050525E-2,2.6100017E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,-1,41,43,-1,45,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5644029E1,3.2975554E0,4.617708E0,1.913332E0,6.7483175E-1,4.4684064E-1,5.58939E-1,6.836338E-1,2.2708063E-1,7.0458287E-1,2.544869E-1,3.741634E-1,6.6792715E-1,2.219677E-2,1.3113213E-1,6.911535E-1,0E0,1.8323165E-1,1.88183E-1,2.4910134E-1,1.7536034E-1,0E0,1.1454445E-1,4.2544305E-2,0E0,3.647622E-1,3.3431494E-1,0E0,0E0,0E0,4.225731E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,-1,42,44,-1,46,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.5106274E-2,2.0224316E-1,2.727992E6,9.50379E5,2.3842E4,7.013774E5,2.771097E0,1.6E1,6.4796996E-1,1.8681856E8,7.12911E4,2.4914644E1,2.7479675E-1,2.0787016E5,1.8462508E0,-2.7904196E-2,1.04783256E6,3.9E1,3.3529866E-2,1.5130641E0,-3.0153783E-3,1.1225629E6,1.5552E4,2.215816E-2,7.230905E2,1.4813267E3,4.802176E-2,2.7501302E-2,1.03110215E-2,2.744E3,-1.9017814E-2,-9.947347E-3,-2.028609E-3,9.575163E-3,-8.734323E-3,2.0642108E-3,2.2078224E-4,-1.2570478E-2,-9.5630065E-4,9.648653E-3,8.224101E-3,1.5246839E-3,2.3588135E-3,9.310498E-3,5.70954E-3,1.8742599E-2,-1.0078216E-2,3.8327414E-3,1.3050525E-2,2.6100017E-2],"split_indices":[53,39,39,1,9,9,34,54,3,40,7,29,57,39,29,54,0,33,3,39,57,0,34,9,0,53,56,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.78E2,9.4E1,1.12E2,6.6E1,6E1,3.4E1,8.9E1,2.3E1,3.1E1,3.5E1,1.5E1,4.5E1,1.2E1,2.2E1,8E1,9E0,1E1,1.3E1,1.9E1,1.2E1,7E0,2.8E1,9E0,6E0,3.2E1,1.3E1,4E0,8E0,4E0,1.8E1,3.5E1,4.5E1,5E0,5E0,8E0,5E0,4E0,1.5E1,7E0,5E0,2E1,8E0,4E0,5E0,2.7E1,5E0,7E0,6E0,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-2.2932042E-4,-1.6076626E-1,2.791689E-1,-2.4847326E-1,-1.1363867E-2,1.2656887E-1,5.408803E-1,-3.4279287E-1,-8.47643E-2,-9.967158E-2,9.725088E-2,-1.0100473E-1,1.6826609E-1,3.3819383E-1,6.583631E-1,-1.17292136E-1,-3.9049146E-1,5.0314553E-2,-2.1599023E-1,-1.7354113E-1,7.484087E-2,1.546897E-2,1.9545256E-1,5.5201887E-3,-1.4045857E-2,2.4351569E-1,7.563631E-2,1.7509794E-2,7.803059E-3,1.6527753E-2,3.4204446E-2,-1.0716689E-2,3.47444E-3,-1.9195572E-2,-9.966454E-4,-1.5373593E-3,7.094945E-3,-1.3927387E-2,-4.6953233E-3,-1.2894858E-3,-1.0270729E-2,-9.854082E-4,9.84646E-3,4.65181E-3,-2.217704E-3,1.0614773E-2,3.9672107E-3,7.59205E-3,2.0669263E-2,1.4315463E-2,3.6960765E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2783342E1,2.3689346E0,4.0850067E0,1.7312126E0,6.619165E-1,6.527709E-1,6.8220806E-1,7.286358E-1,7.7307886E-1,5.0285196E-1,2.4594131E-1,5.3244543E-1,3.7889147E-1,4.9229503E-2,3.1550026E-1,3.2917523E-1,5.661726E-1,1.962696E-1,1.8613267E-1,1.8840152E-1,1.6513681E-1,1.0288088E-1,3.3833563E-2,0E0,0E0,4.2660427E-1,4.2701286E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,1.7613442E-1,1.8829846E-1,1.9462195E2,9.50379E5,7.5E2,3.301E3,2.431768E2,1.5248811E1,1.2185057E0,2.052E3,3.9378E5,3.7724147E8,5.4374854E2,2.2324745E5,1.081E3,9.4983295E2,6.135367E9,9.877899E2,1E0,1.7518893E2,4.045147E0,3.174073E6,5.5201887E-3,-1.4045857E-2,7.230905E2,1.897231E0,1.7509794E-2,7.803059E-3,1.6527753E-2,3.4204446E-2,-1.0716689E-2,3.47444E-3,-1.9195572E-2,-9.966454E-4,-1.5373593E-3,7.094945E-3,-1.3927387E-2,-4.6953233E-3,-1.2894858E-3,-1.0270729E-2,-9.854082E-4,9.84646E-3,4.65181E-3,-2.217704E-3,1.0614773E-2,3.9672107E-3,7.59205E-3,2.0669263E-2,1.4315463E-2,3.6960765E-4],"split_indices":[53,42,39,53,9,0,2,56,57,42,2,1,7,53,29,2,4,5,4,107,53,55,1,0,0,53,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.8E2,1.03E2,1.13E2,6.7E1,6.6E1,3.7E1,7.1E1,4.2E1,3.7E1,3E1,1E1,5.6E1,1.5E1,2.2E1,1.3E1,5.8E1,2.1E1,2.1E1,2.6E1,1.1E1,1.7E1,1.3E1,5E0,5E0,3E1,2.6E1,1.1E1,4E0,6E0,1.6E1,8E0,5E0,5.4E1,4E0,1.2E1,9E0,1.1E1,1E1,7E0,1.9E1,7E0,4E0,7E0,1E1,9E0,4E0,2.3E1,7E0,5E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-8.568767E-3,-1.8477075E-1,2.4255592E-1,-2.5562903E-1,4.3443054E-2,1.047099E-1,5.0142497E-1,-3.0027148E-1,-2.4495613E-2,-8.791906E-2,9.818956E-2,-1.6025668E-1,1.5275505E-1,2.9869998E-1,6.2946415E-1,-1.4635389E-1,-3.379046E-1,-1.0504028E-1,7.460755E-2,-5.6306818E-3,-7.058365E-4,1.6652746E-1,6.528328E-3,3.9131995E-3,-1.351333E-2,1.9199795E-1,1.1761697E-2,1.5748665E-2,8.650935E-3,1.6605457E-2,3.335774E-2,-1.3596179E-4,-1.2228896E-2,-2.0160748E-2,-9.96515E-3,-9.779533E-4,-1.0042071E-2,8.086266E-3,-2.833499E-3,1.4913988E-3,1.0338657E-2,-3.4182414E-3,5.2088927E-3,3.4956157E-3,1.3262765E-2,5.795708E-3,-2.6651435E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2124703E1,2.6146097E0,3.9797955E0,1.2618036E0,2.891058E-1,9.792569E-1,8.242922E-1,5.47163E-1,1.7546366E-1,2.807039E-2,1.7430308E-1,4.1709977E-1,3.5185313E-1,1.7398953E-2,3.2636833E-1,3.7544218E-1,9.010458E-1,1.09070435E-1,1.4835364E-1,0E0,0E0,1.172578E-1,1.2034082E-1,0E0,0E0,5.3566706E-1,1.2650971E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1309677E2,7.087292E-2,1.2711891E0,2.727992E6,2.0358758E8,7.5E2,3.301E3,2.431768E2,4.8E1,1.1709996E0,2.7352352E6,4.6557466E8,6.191915E0,1.5E1,2.174459E5,6.7897725E-1,5.283708E0,1.4618938E0,4.654E3,-5.6306818E-3,-7.058365E-4,7E0,2.3581433E5,3.9131995E-3,-1.351333E-2,4.6020203E2,3.7808718E11,1.5748665E-2,8.650935E-3,1.6605457E-2,3.335774E-2,-1.3596179E-4,-1.2228896E-2,-2.0160748E-2,-9.96515E-3,-9.779533E-4,-1.0042071E-2,8.086266E-3,-2.833499E-3,1.4913988E-3,1.0338657E-2,-3.4182414E-3,5.2088927E-3,3.4956157E-3,1.3262765E-2,5.795708E-3,-2.6651435E-3],"split_indices":[53,39,40,1,7,0,2,56,8,40,49,5,58,3,29,57,57,54,2,0,0,3,29,0,0,53,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.6E2,1.12E2,1.22E2,3.8E1,7.4E1,3.8E1,1.02E2,2E1,1.1E1,2.7E1,1.1E1,6.3E1,1.6E1,2.2E1,2.1E1,8.1E1,1.1E1,9E0,7E0,4E0,1.5E1,1.2E1,4E0,7E0,4.9E1,1.4E1,1E1,6E0,7E0,1.5E1,1E1,1.1E1,4.3E1,3.8E1,7E0,4E0,5E0,4E0,5E0,1E1,7E0,5E0,2.3E1,2.6E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-3.596769E-4,-1.8368325E-1,2.3429736E-1,-2.9083514E-1,-5.836663E-2,1.02255315E-1,4.439845E-1,-3.3659607E-1,-5.179366E-2,4.2336747E-2,-1.5753405E-1,-2.6449973E-2,1.5329044E-1,1.4937297E-1,6.157723E-1,-2.637285E-1,-5.2597773E-1,2.6845357E-3,-1.31673E-1,-3.3140503E-2,1.6027789E-1,-2.607038E-1,-6.1601207E-2,8.009401E-3,-7.77979E-2,1.7347395E-1,-4.0182974E-3,-1.3884074E-4,2.5812146E-1,4.0030707E-2,5.007615E-1,1.7797025E-3,-1.42537E-2,-1.1270505E-2,-2.691749E-2,-7.8072743E-3,-3.1049799E-3,6.30815E-3,-3.9194035E-3,1.1555192E-2,4.810408E-3,-4.7462294E-3,-1.5235984E-2,-4.9033207E-3,4.759315E-3,-5.4215686E-3,-3.746539E-4,5.159625E-3,1.0051347E-2,2.0515637E-3,-2.0630697E-3,7.5246883E-3,1.5610643E-2,2.460694E-2,1.254935E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,-1,45,47,-1,49,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2260195E1,2.1314907E0,3.4155402E0,9.232993E-1,7.5565654E-1,5.163967E-1,2.3373318E0,8.5486984E-1,1.3959955E-1,3.4358284E-1,3.6004037E-1,2.5022107E-1,2.870475E-1,3.0896032E-1,4.399023E-1,7.6077175E-1,1.7029333E-1,0E0,1.0473698E-2,2.2396392E-1,4.968801E-2,1.6510177E-1,1.6655323E-1,0E0,5.1927097E-2,1.1871493E-1,0E0,1.9966932E-2,2.888751E-2,0E0,3.1169415E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,-1,46,48,-1,50,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5300907E-1,1.9462195E2,1.5972575E0,1.848519E-2,1.560615E1,3.9324117E0,2.744E3,5.1653096E-3,1E0,3.165913E6,3.919E3,1.0348846E6,9.723711E0,6.765595E7,2.7479675E-1,1.5166431E5,3.12986E10,2.6845357E-3,8.138103E1,2.8558656E4,3.995065E6,3.2829E4,4.2709158E2,8.009401E-3,3.2826756E5,4.876025E2,-4.0182974E-3,2.0356708E2,1.2984486E6,4.0030707E-2,2.505319E0,1.7797025E-3,-1.42537E-2,-1.1270505E-2,-2.691749E-2,-7.8072743E-3,-3.1049799E-3,6.30815E-3,-3.9194035E-3,1.1555192E-2,4.810408E-3,-4.7462294E-3,-1.5235984E-2,-4.9033207E-3,4.759315E-3,-5.4215686E-3,-3.746539E-4,5.159625E-3,1.0051347E-2,2.0515637E-3,-2.0630697E-3,7.5246883E-3,1.5610643E-2,2.460694E-2,1.254935E-2],"split_indices":[43,53,42,39,57,59,2,39,107,1,10,46,58,5,39,29,32,0,53,34,30,9,53,0,29,53,0,4,34,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.59E2,1.24E2,8.5E1,7.4E1,7.7E1,4.7E1,7.1E1,1.4E1,3.7E1,3.7E1,2.2E1,5.5E1,1.8E1,2.9E1,5.3E1,1.8E1,6E0,8E0,2.3E1,1.4E1,1.7E1,2E1,4E0,1.8E1,5.1E1,4E0,8E0,1E1,7E0,2.2E1,7E0,4.6E1,4E0,1.4E1,4E0,4E0,5E0,1.8E1,4E0,1E1,6E0,1.1E1,1.6E1,4E0,1.1E1,7E0,2.3E1,2.8E1,4E0,4E0,6E0,4E0,1.8E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.3580224E-2,-1.3836537E-1,3.4709883E-1,-2.3893131E-1,7.0762103E-3,1.7716682E-1,5.539774E-1,-3.0390304E-1,-8.560234E-2,-1.438354E-1,4.63419E-2,2.626264E-1,5.3378113E-2,1.0326719E-2,6.3731706E-1,-2.6912305E-1,-2.3783604E-2,-1.7900838E-1,-1.690239E-2,-1.31039675E-2,-4.9727768E-2,4.2479597E-3,1.5689185E-1,3.9813077E-1,6.62912E-2,9.279113E-2,-3.1148603E-3,1.52902E-2,7.375629E-1,-1.8078666E-2,-8.070591E-3,-1.4476354E-3,-1.2173431E-2,1.4574539E-3,-7.3554767E-3,2.4368017E-4,-4.8633316E-3,-4.047023E-3,3.300304E-3,3.2097457E-3,9.850741E-3,2.4707617E-2,1.204215E-2,-2.8300765E-3,6.817667E-3,6.8351824E-4,6.878275E-3,4.1859776E-2,2.512445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2605088E1,3.0439966E0,2.433033E0,1.1993175E0,5.1627135E-1,4.1971767E-1,7.362766E-1,4.970932E-1,2.408199E-1,2.2431833E-1,3.204239E-1,6.1211026E-1,9.2142E-2,0E0,5.199766E-1,8.1216764E-1,0E0,1.9331014E-1,1.6460419E-1,0E0,3.7468463E-2,3.2321015E-1,7.960236E-2,1.2448859E-1,1.24041766E-1,5.954951E-2,0E0,0E0,9.6570015E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,2.0998143E-1,2.0224316E-1,4.95E9,1.8681856E8,6.464779E6,3.301E3,2.771097E0,1.9462195E2,2.29462E5,2.704375E6,4.3147E5,5.615313E3,1.0326719E-2,2.2324745E5,1.8662952E0,-2.3783604E-2,1.335552E0,2.142943E1,-1.31039675E-2,1.5871658E5,8.749103E5,4.152263E0,8.312613E5,1.4E1,4.005725E1,-3.1148603E-3,1.52902E-2,2.9375613E-1,-1.8078666E-2,-8.070591E-3,-1.4476354E-3,-1.2173431E-2,1.4574539E-3,-7.3554767E-3,2.4368017E-4,-4.8633316E-3,-4.047023E-3,3.300304E-3,3.2097457E-3,9.850741E-3,2.4707617E-2,1.204215E-2,-2.8300765E-3,6.817667E-3,6.8351824E-4,6.878275E-3,4.1859776E-2,2.512445E-2],"split_indices":[53,43,39,5,7,30,2,54,53,9,9,9,4,0,29,54,0,55,57,0,29,34,59,48,3,57,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,2.07E2,7.1E1,1.22E2,8.5E1,4E1,3.1E1,8.5E1,3.7E1,1.7E1,6.8E1,2.3E1,1.7E1,7E0,2.4E1,7.5E1,1E1,1.5E1,2.2E1,6E0,1.1E1,5E1,1.8E1,1.3E1,1E1,1.3E1,4E0,7E0,1.7E1,3.1E1,4.4E1,6E0,9E0,1.7E1,5E0,6E0,5E0,2.1E1,2.9E1,8E0,1E1,5E0,8E0,4E0,6E0,6E0,7E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[5.877288E-4,-1.2417702E-1,2.9708055E-1,-2.1908262E-1,2.6360616E-2,1.4931536E-1,5.25398E-1,-2.871291E-1,-4.4770997E-2,-2.4798097E-2,1.3785537E-1,2.2076918E-1,6.0456306E-2,7.016789E-1,3.912528E-1,-1.707563E-1,-3.494655E-1,-1.1833131E-1,3.536328E-2,-8.723113E-2,1.6329106E-2,1.6928416E-1,-9.998593E-4,3.5230267E-1,5.4675963E-2,1.4380012E-1,-8.029524E-3,1.6454859E-2,8.149832E-1,4.5519656E-1,7.1606375E-3,-1.2303925E-2,-1.7109883E-3,-2.1576341E-2,-1.0048361E-2,4.879607E-4,-7.0635783E-3,4.498231E-3,-6.7336615E-3,-7.416376E-3,-1.0917642E-3,1.8079918E-3,-5.735647E-3,8.8046165E-3,2.799195E-3,1.9212412E-2,8.031611E-3,-9.944658E-4,4.578838E-3,3.8882762E-3,1.3493867E-2,2.1922933E-2,4.56179E-2,2.3183249E-2,1.1424944E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.061656E1,2.8917227E0,2.789226E0,1.4579449E0,4.5402884E-1,3.231671E-1,5.0582504E-1,5.859556E-1,2.1639389E-1,1.4264768E-1,1.2905523E-1,6.1429524E-1,5.1442254E-1,2.158432E-1,2.4574137E-1,4.1541892E-1,7.8902054E-1,8.757931E-2,2.1807545E-1,9.9379376E-2,1.1479305E-1,3.8379908E-2,0E0,1.1213887E-1,4.9961645E-2,1.4358124E-1,0E0,0E0,6.4380646E-2,3.5182476E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7186426E2,2.1619815E-1,2.0224316E-1,4.95E9,2.427E3,5.4582E4,7.013774E5,9.0649E4,2.0306636E2,1.503E3,4.99E3,6.068019E6,3.6182167E1,2.4214827E5,2.094037E7,1.081E3,6.608892E2,6.035827E9,2.142943E1,8.503368E-1,2.302E3,2.5659466E1,-9.998593E-4,6.5E1,1.74E4,8.834E3,-8.029524E-3,1.6454859E-2,6.886E3,4.9691647E-1,7.1606375E-3,-1.2303925E-2,-1.7109883E-3,-2.1576341E-2,-1.0048361E-2,4.879607E-4,-7.0635783E-3,4.498231E-3,-6.7336615E-3,-7.416376E-3,-1.0917642E-3,1.8079918E-3,-5.735647E-3,8.8046165E-3,2.799195E-3,1.9212412E-2,8.031611E-3,-9.944658E-4,4.578838E-3,3.8882762E-3,1.3493867E-2,2.1922933E-2,4.56179E-2,2.3183249E-2,1.1424944E-2],"split_indices":[53,42,39,5,2,9,34,30,53,2,2,30,57,29,9,2,4,5,57,43,2,57,0,8,9,2,0,0,2,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,2.01E2,8.4E1,1.23E2,7.8E1,5.2E1,3.2E1,8.8E1,3.5E1,5.4E1,2.4E1,2.8E1,2.4E1,1.2E1,2E1,3.2E1,5.6E1,1.8E1,1.7E1,2.1E1,3.3E1,2E1,4E0,1.5E1,1.3E1,1.8E1,6E0,4E0,8E0,1.5E1,5E0,1.8E1,1.4E1,2.8E1,2.8E1,4E0,1.4E1,1.3E1,4E0,9E0,1.2E1,2.9E1,4E0,1.6E1,4E0,1E1,5E0,5E0,8E0,1.4E1,4E0,4E0,4E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.4342601E-2,-1.07165396E-1,3.081849E-1,-1.948525E-1,1.4563464E-2,1.487435E-1,5.134866E-1,-2.6648492E-1,-7.037843E-2,-1.2724888E-1,5.865227E-2,1.9358988E-1,1.8090768E-3,3.6575672E-1,6.8607605E-1,-8.759942E-2,-3.120232E-1,-1.3680433E-1,1.1262216E-1,-1.3406685E-2,-5.318928E-2,1.5617897E-2,1.400989E-1,-2.2907217E-3,2.2556266E-1,-3.509075E-3,3.1505649E-3,6.7552733E-3,1.8815292E-2,1.431593E-2,3.879194E-2,-8.164921E-3,4.2139017E-3,-1.6489454E-2,-5.028345E-3,-9.910956E-3,5.478642E-4,7.2356444E-3,3.2555868E-3,1.3077437E-3,-3.9338255E-3,-4.5785587E-3,3.7205967E-3,8.15121E-3,-9.630316E-4,1.22322105E-2,2.0410132E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.8873825E0,2.0949519E0,2.5750446E0,9.9771214E-1,5.264075E-1,3.0935514E-1,6.414442E-1,5.5918884E-1,5.3811425E-1,2.2733971E-1,2.2213225E-1,2.9131615E-1,6.759233E-2,1.37398E-1,6.185136E-1,2.7342647E-1,4.9407816E-1,3.7591076E-1,1.1285409E-2,0E0,4.19005E-2,3.305899E-1,1.3456911E-1,0E0,2.2088325E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7186426E2,2.1619815E-1,1.9530877E-1,1.9462195E2,2.0358758E8,6.191915E0,1.6507233E3,1.1266387E10,4.1081388E5,8E0,2.427E3,7.5E2,5.4527334E9,1.9643908E5,2.174459E5,1.175E3,6.608892E2,5.5173693E9,1.9391481E0,-1.3406685E-2,7.364257E4,6.4796996E-1,4.99E3,-2.2907217E-3,2.1030048E10,-3.509075E-3,3.1505649E-3,6.7552733E-3,1.8815292E-2,1.431593E-2,3.879194E-2,-8.164921E-3,4.2139017E-3,-1.6489454E-2,-5.028345E-3,-9.910956E-3,5.478642E-4,7.2356444E-3,3.2555868E-3,1.3077437E-3,-3.9338255E-3,-4.5785587E-3,3.7205967E-3,8.15121E-3,-9.630316E-4,1.22322105E-2,2.0410132E-3],"split_indices":[53,42,39,53,7,58,4,32,29,3,2,0,12,29,29,2,4,5,54,0,29,40,2,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.95E2,8E1,1.13E2,8.2E1,4.6E1,3.4E1,7.1E1,4.2E1,1.9E1,6.3E1,3.5E1,1.1E1,2E1,1.4E1,1.5E1,5.6E1,3.1E1,1.1E1,5E0,1.4E1,4.2E1,2.1E1,4E0,3.1E1,5E0,6E0,4E0,1.6E1,5E0,9E0,1E1,5E0,4.5E1,1.1E1,2E1,1.1E1,4E0,7E0,4E0,1E1,1.5E1,2.7E1,1.7E1,4E0,2.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.562115E-3,-1.3048783E-1,2.2347076E-1,-2.0250554E-1,-1.1698365E-2,9.9489175E-2,4.502752E-1,-2.4757701E-1,-6.868763E-3,-8.476708E-2,6.6012576E-2,1.3355128E-1,-5.9242588E-2,6.518211E-1,3.1071287E-1,-2.1511847E-1,-2.1366872E-2,1.00871116E-1,-8.921722E-2,-1.520396E-1,6.244792E-2,-2.2451725E-2,1.5579757E-1,1.7636213E-1,1.6952205E-3,-8.606896E-3,1.2879231E-3,1.5280052E-2,7.571422E-1,3.5717335E-1,5.860538E-3,-1.8638441E-2,-8.411605E-3,8.233474E-3,-5.9174263E-4,-8.577653E-3,9.491502E-4,-1.3062648E-2,-3.91669E-3,-9.709322E-4,8.453895E-3,-6.597653E-3,1.613792E-3,3.699599E-3,7.957266E-3,1.0919501E-2,4.4508916E-3,-5.195139E-3,7.974824E-3,2.0238478E-2,4.2508848E-2,7.6565524E-3,1.85597E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8970885E0,1.5375216E0,2.6828012E0,9.8587465E-1,3.9739415E-1,3.539433E-1,7.538295E-1,5.396671E-1,2.0418814E-1,3.660214E-1,2.73552E-1,2.9887903E-1,1.388759E-1,1.8677759E-1,1.4201856E-1,4.2421436E-1,0E0,9.095485E-2,1.4246404E-1,1.9632614E-1,1.2533586E-1,1.3010634E-1,7.956773E-3,1.7089939E-1,2.9456225E-1,0E0,0E0,0E0,6.8992615E-2,8.202815E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,2.5106274E-2,1.8829846E-1,3.165913E6,9.50379E5,2.8812434E7,7.013774E5,2.771097E0,1.6E1,1.2185057E0,5.3164E4,2.3797293E0,1.04E2,2.4214827E5,9.8163464E7,3.4936884E7,-2.1366872E-2,1.6868966E8,3.9E1,8E0,1.7518893E2,9.033382E-1,2.052E3,1.1936242E7,4.3818486E9,-8.606896E-3,1.2879231E-3,1.5280052E-2,6.886E3,2.8641683E8,5.860538E-3,-1.8638441E-2,-8.411605E-3,8.233474E-3,-5.9174263E-4,-8.577653E-3,9.491502E-4,-1.3062648E-2,-3.91669E-3,-9.709322E-4,8.453895E-3,-6.597653E-3,1.613792E-3,3.699599E-3,7.957266E-3,1.0919501E-2,4.4508916E-3,-5.195139E-3,7.974824E-3,2.0238478E-2,4.2508848E-2,7.6565524E-3,1.85597E-2],"split_indices":[53,39,39,1,9,46,34,54,3,42,30,54,8,29,33,7,0,7,3,3,53,42,2,46,12,0,0,0,2,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.79E2,9.6E1,1.11E2,6.8E1,6.3E1,3.3E1,9E1,2.1E1,3.5E1,3.3E1,5.2E1,1.1E1,1.2E1,2.1E1,8E1,1E1,9E0,1.2E1,2.4E1,1.1E1,1.7E1,1.6E1,3.9E1,1.3E1,4E0,7E0,4E0,8E0,1.6E1,5E0,1E1,7E1,5E0,4E0,6E0,6E0,7E0,1.7E1,7E0,4E0,5E0,1.2E1,4E0,1.2E1,2.1E1,1.8E1,8E0,5E0,4E0,4E0,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.2804228E-3,-1.2672013E-1,2.263347E-1,-1.872631E-1,-2.7809761E-2,9.588739E-2,4.447069E-1,-2.42015E-1,-2.023026E-2,-1.456923E-2,-4.826E-3,-7.280936E-2,1.3370061E-1,3.0428183E-1,6.012206E-1,-2.888798E-1,-1.2360803E-1,9.657179E-2,-9.048539E-2,-4.9619943E-2,1.3265707E-1,1.4213595E-3,-5.9166797E-3,1.6225491E-1,-4.949379E-3,4.1639605E-3,1.6825927E-2,3.5240296E-2,1.788566E-2,1.3498644E-3,-1.4410529E-2,-9.877811E-3,2.849631E-3,-8.9196855E-4,9.043458E-3,-7.703012E-3,1.2180454E-3,-3.2389776E-3,5.095308E-3,1.9713729E-3,7.8085917E-3,2.7058087E-3,9.688545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.794818E0,1.048738E0,2.7205067E0,9.9194884E-1,4.4957182E-1,4.0523046E-1,6.082835E-1,4.193201E-1,2.3850471E-1,0E0,4.0072063E-1,7.326687E-2,3.6869437E-1,2.4165642E-1,2.6155758E-1,4.5419836E-1,4.349951E-1,1.3115343E-1,1.6613193E-1,1.7060067E-1,4.5841843E-2,0E0,0E0,2.1697533E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.083633E6,1.1670399E-1,1.5972575E0,2.396E3,3.3789E4,3.359399E-1,5.76E3,2.2E1,8.305048E4,-1.456923E-2,2.427E3,3.6002667E2,1.25440235E1,2.0787016E5,2.9375613E-1,1.6E1,1.9462195E2,6.2229652E0,4.231592E6,1.3431462E7,1.7300606E6,1.4213595E-3,-5.9166797E-3,1.0705044E3,-4.949379E-3,4.1639605E-3,1.6825927E-2,3.5240296E-2,1.788566E-2,1.3498644E-3,-1.4410529E-2,-9.877811E-3,2.849631E-3,-8.9196855E-4,9.043458E-3,-7.703012E-3,1.2180454E-3,-3.2389776E-3,5.095308E-3,1.9713729E-3,7.8085917E-3,2.7058087E-3,9.688545E-3],"split_indices":[52,40,42,2,9,43,2,3,34,0,2,53,58,29,39,10,53,57,1,9,52,0,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.75E2,9.6E1,1.08E2,6.7E1,6.1E1,3.5E1,8.1E1,2.7E1,4E0,6.3E1,1.1E1,5E1,2E1,1.5E1,5.7E1,2.4E1,1E1,1.7E1,4.8E1,1.5E1,4E0,7E0,4.5E1,5E0,5E0,1.5E1,7E0,8E0,4E0,5.3E1,1.6E1,8E0,5E0,5E0,1E1,7E0,4.3E1,5E0,5E0,1E1,1.5E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.6173291E-3,-8.5729115E-2,2.832087E-1,-1.5077375E-1,7.1107745E-2,1.3178344E-1,4.2221588E-1,-2.1220489E-1,-5.4644745E-2,1.1440147E-2,1.7007707E-1,9.530249E-2,1.4931689E-2,2.6542822E-1,5.9181195E-1,-1.6032451E-1,-3.7234953E-1,-1.1518315E-1,2.0817775E-2,1.2416999E-1,-6.567375E-2,1.9964406E-2,7.790098E-2,1.3290417E-1,-2.1746214E-3,4.2201085E-3,1.395696E-2,3.2665662E-2,1.9404117E-2,-1.4587144E-2,-4.329589E-3,-2.3598464E-2,-1.2557757E-2,-6.5033445E-3,2.9586204E-3,-6.226961E-3,3.2011797E-3,1.0076781E-2,3.060576E-3,2.6669793E-4,-6.0814843E-3,7.1549444E-3,-3.4032168E-4,2.3125403E-3,7.877769E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.039637E0,2.214223E0,1.3757038E0,8.9191747E-1,3.75991E-1,2.0789605E-1,7.7358675E-1,7.1298075E-1,2.8106907E-1,3.6456925E-1,5.49241E-1,1.6706368E-1,0E0,1.0328221E-1,2.1744251E-2,7.1714914E-1,1.6740847E-1,1.681189E-1,2.2226313E-1,7.8329474E-2,1.1942528E-1,0E0,1.2674636E-1,6.7201525E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.165913E6,2.174459E5,1.1670399E-1,6.639E3,4.328683E0,4.063E3,5.1653096E-3,9.50379E5,9.953969E6,6.747254E5,1.1598054E8,1.4931689E-2,4.7436612E2,2.9898608E-1,3.3617156E-4,1.659E3,1.0289595E6,2.5188733E2,4.986835E6,5.102E3,1.9964406E-2,8.223634E6,1E0,-2.1746214E-3,4.2201085E-3,1.395696E-2,3.2665662E-2,1.9404117E-2,-1.4587144E-2,-4.329589E-3,-2.3598464E-2,-1.2557757E-2,-6.5033445E-3,2.9586204E-3,-6.226961E-3,3.2011797E-3,1.0076781E-2,3.060576E-3,2.6669793E-4,-6.0814843E-3,7.1549444E-3,-3.4032168E-4,2.3125403E-3,7.877769E-3],"split_indices":[43,1,29,40,2,36,2,39,9,46,48,33,0,4,39,40,10,34,4,1,0,0,30,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.15E2,6.7E1,1.52E2,6.3E1,3.3E1,3.4E1,9.2E1,6E1,4E1,2.3E1,2.9E1,4E0,1.9E1,1.5E1,7.1E1,2.1E1,3.3E1,2.7E1,1.6E1,2.4E1,5E0,1.8E1,2.3E1,6E0,4E0,1.5E1,7E0,8E0,2E1,5.1E1,7E0,1.4E1,2.9E1,4E0,6E0,2.1E1,5E0,1.1E1,1.2E1,1.2E1,9E0,9E0,8E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.448896E-3,-1.1091449E-1,2.0679466E-1,-1.7352903E-1,-7.671845E-3,8.73633E-2,4.1192907E-1,-2.4279347E-1,-5.2307483E-2,-1.1494169E-1,3.8457513E-2,-9.4130084E-2,1.2085565E-1,2.8959206E-1,6.0096455E-1,3.0339844E-4,-2.7017272E-1,6.9259778E-3,-2.1986778E-1,-1.2515523E-2,-4.8923224E-2,-4.374927E-2,9.666044E-2,5.486807E-3,-1.0729481E-2,2.0315604E-1,4.5276098E-2,5.8826846E-3,3.2365894E-1,3.246025E-2,1.5870722E-2,-1.4313898E-2,-3.982747E-3,1.993537E-3,-5.946832E-3,-1.4400913E-2,-5.621118E-3,-5.264809E-3,-4.0461618E-4,-5.7544215E-3,3.611996E-3,6.191279E-3,-1.0630519E-4,6.827361E-3,1.5595774E-2,1.1583481E-2,-1.8767477E-4,1.6414108E-2,7.081248E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,-1,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.653422E0,1.162277E0,2.5119586E0,9.276905E-1,3.4585232E-1,4.1864502E-1,7.047553E-1,4.865632E-1,4.1838017E-1,2.0421454E-1,2.3867792E-1,3.5370338E-1,3.453549E-1,9.646487E-2,1.3030958E-1,0E0,4.5177794E-1,1.6399291E-1,6.1749935E-2,0E0,4.100979E-2,2.182728E-1,1.0883677E-1,0E0,0E0,1.5062082E-1,3.1667027E-1,0E0,5.6767464E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,-1,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.5106274E-2,1.8829846E-1,1.9462195E2,2.0283392E8,7.5E2,2.0551236E9,1E1,4.908866E6,8E0,9.50379E5,3.13E2,3.5630266E8,1.9643908E5,9.780233E9,3.0339844E-4,6.608892E2,2.142943E1,2.26E3,-1.2515523E-2,1.8508842E2,1.1685816E0,1.1225629E6,5.486807E-3,-1.0729481E-2,2.610156E3,1.897231E0,5.8826846E-3,4.1489546E8,3.246025E-2,1.5870722E-2,-1.4313898E-2,-3.982747E-3,1.993537E-3,-5.946832E-3,-1.4400913E-2,-5.621118E-3,-5.264809E-3,-4.0461618E-4,-5.7544215E-3,3.611996E-3,6.191279E-3,-1.0630519E-4,6.827361E-3,1.5595774E-2,1.1583481E-2,-1.8767477E-4,1.6414108E-2,7.081248E-3],"split_indices":[53,39,39,53,7,0,5,10,9,3,9,0,7,29,5,0,4,57,2,0,4,42,34,0,0,4,55,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.79E2,1.03E2,1.11E2,6.8E1,6.6E1,3.7E1,7E1,4.1E1,2E1,4.8E1,1E1,5.6E1,2.4E1,1.3E1,7E0,6.3E1,3.1E1,1E1,5E0,1.5E1,2E1,2.8E1,4E0,6E0,2.6E1,3E1,5E0,1.9E1,8E0,5E0,5.1E1,1.2E1,2.5E1,6E0,4E0,6E0,5E0,1E1,1.2E1,8E0,2E1,8E0,2E1,6E0,5E0,2.5E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.3387234E-2,-9.8158866E-2,2.0095628E-1,-1.5815048E-1,3.3144096E-3,9.399044E-2,3.8426563E-1,-2.0907788E-1,-1.5388826E-3,-2.7632961E-2,9.4141126E-2,1.6830553E-1,3.962011E-2,2.1663459E-1,4.533017E-1,-6.14332E-2,-2.3814444E-1,6.1698775E-3,-3.6832076E-2,-8.5555054E-2,1.3833603E-2,1.4041278E-1,-6.4882846E-4,2.1202146E-1,-1.2579839E-3,-1.4520428E-1,1.1473296E-1,1.1762786E-2,5.926803E-3,2.4302903E-1,5.332546E-1,-6.6719046E-3,6.192841E-3,-1.5215636E-2,-8.203149E-3,4.5509716E-3,-5.3307908E-3,2.5909973E-4,-6.056625E-3,-3.3308335E-3,3.1093373E-3,9.219971E-3,3.1796014E-3,1.5120998E-2,7.4329404E-3,-2.6809094E-3,-8.35247E-3,1.2690519E-2,5.0229883E-5,1.4495721E-2,5.645043E-3,1.71185E-2,2.878624E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,-1,43,-1,45,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.836207E0,1.0658447E0,2.003098E0,8.770187E-1,1.8811736E-1,2.6404756E-1,3.4155798E-1,3.4156656E-1,1.3888466E-1,1.2127505E-1,8.711721E-2,2.4525148E-1,5.719672E-1,1.1508822E-2,3.0751324E-1,2.6484454E-1,3.2322145E-1,0E0,2.5781983E-1,8.880988E-2,1.437203E-1,3.74095E-2,0E0,8.239436E-2,0E0,2.7228385E-2,5.299703E-1,0E0,0E0,3.922844E-2,3.4814835E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,25,25,26,26,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,-1,44,-1,46,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.5106274E-2,1.8829846E-1,1.781973E6,9.749826E6,1.1936242E7,3.301E3,2.431768E2,2.6017979E11,1.804E3,2.8808496E7,2.332406E0,1.4813267E3,9.3735106E5,2.2324745E5,3.084419E6,1.8662952E0,6.1698775E-3,1.2321745E7,1.71E2,3.7230852E0,1.8609113E0,-6.4882846E-4,4.0012E4,-1.2579839E-3,1.177948E3,6.464779E6,1.1762786E-2,5.926803E-3,1.61628E5,1.6507233E3,-6.6719046E-3,6.192841E-3,-1.5215636E-2,-8.203149E-3,4.5509716E-3,-5.3307908E-3,2.5909973E-4,-6.056625E-3,-3.3308335E-3,3.1093373E-3,9.219971E-3,3.1796014E-3,1.5120998E-2,7.4329404E-3,-2.6809094E-3,-8.35247E-3,1.2690519E-2,5.0229883E-5,1.4495721E-2,5.645043E-3,1.71185E-2,2.878624E-2],"split_indices":[53,39,39,1,9,46,2,56,32,2,9,54,56,48,29,9,54,0,46,0,55,54,0,9,0,4,30,0,0,29,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,1.74E2,1.03E2,1.09E2,6.5E1,6.6E1,3.7E1,8.2E1,2.7E1,4.9E1,1.6E1,2.7E1,3.9E1,1.2E1,2.5E1,1.4E1,6.8E1,5E0,2.2E1,2E1,2.9E1,1.1E1,5E0,2.2E1,5E0,1.1E1,2.8E1,7E0,5E0,8E0,1.7E1,1E1,4E0,2.5E1,4.3E1,8E0,1.4E1,7E0,1.3E1,1.1E1,1.8E1,5E0,6E0,5E0,1.7E1,4E0,7E0,1.1E1,1.7E1,4E0,4E0,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[5.261088E-3,-9.0237655E-2,2.4455974E-1,-1.7128584E-1,-1.3149979E-2,1.16032735E-1,3.954769E-1,-4.172483E-2,-2.5845715E-1,-5.402384E-2,1.05261154E-1,-4.9818265E-3,1.6863844E-1,2.7543652E-1,5.208101E-1,-7.745435E-2,7.720929E-3,-2.9164845E-1,-7.661237E-2,-8.8633694E-2,1.5540023E-1,5.056547E-3,1.4951561E-1,-9.90787E-2,6.3073384E-3,1.9304242E-2,8.850711E-2,4.951181E-3,1.4304491E-2,3.1204592E-2,1.28521295E-2,-6.621332E-3,2.8491096E-4,-2.0483453E-2,-1.04234805E-2,-7.895453E-3,7.006872E-4,-7.0248987E-3,-4.1183087E-4,2.94758E-3,1.1613927E-2,3.765061E-3,-3.3460052E-3,7.93193E-3,2.5724815E-3,-9.286988E-3,1.0759494E-3,5.153018E-3,-1.0522591E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.16945E0,1.2035762E0,1.4476757E0,1.0508521E0,4.8967856E-1,2.7458048E-1,3.8312626E-1,3.0480742E-1,3.1565356E-1,5.573841E-1,1.14540994E-1,2.0060562E-1,5.654253E-1,8.496845E-2,4.176607E-1,1.896056E-1,0E0,3.692701E-1,8.973447E-2,3.2985753E-1,8.120319E-2,5.9614412E-2,2.8743893E-2,1.1781798E-1,0E0,0E0,6.5139234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,1.6425813E2,2.0224316E-1,1.1998492E10,4.354824E5,3.935E3,1.6507233E3,9.74979E6,6.608892E2,1.2730925E0,1.575011E9,2.5312589E8,8.312613E5,1.9643908E5,2.65826E6,9.9409575E5,7.720929E-3,2.851239E5,8.34E2,2.8800612E9,1.2311872E6,6.8618E4,1E0,7.5E2,6.3073384E-3,1.9304242E-2,1.4524763E3,4.951181E-3,1.4304491E-2,3.1204592E-2,1.28521295E-2,-6.621332E-3,2.8491096E-4,-2.0483453E-2,-1.04234805E-2,-7.895453E-3,7.006872E-4,-7.0248987E-3,-4.1183087E-4,2.94758E-3,1.1613927E-2,3.765061E-3,-3.3460052E-3,7.93193E-3,2.5724815E-3,-9.286988E-3,1.0759494E-3,5.153018E-3,-1.0522591E-3],"split_indices":[53,53,39,32,29,2,4,9,4,43,5,7,48,29,48,34,0,29,0,5,34,30,85,0,0,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.92E2,7.6E1,9.3E1,9.9E1,4.2E1,3.4E1,3.8E1,5.5E1,7.4E1,2.5E1,1.3E1,2.9E1,1.9E1,1.5E1,3.3E1,5E0,4.6E1,9E0,6.4E1,1E1,8E0,1.7E1,8E0,5E0,6E0,2.3E1,4E0,1.5E1,8E0,7E0,1.8E1,1.5E1,1.2E1,3.4E1,4E0,5E0,3.5E1,2.9E1,6E0,4E0,4E0,4E0,1.3E1,4E0,4E0,4E0,1.9E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.15079265E-2,-6.397281E-2,2.4555098E-1,-1.2206563E-1,7.136642E-2,1.11058585E-1,3.6918497E-1,-1.5477005E-1,1.1730725E-2,1.7081004E-2,4.520793E-2,7.9825655E-2,1.2718587E-2,2.671033E-1,5.976788E-1,-1.1897293E-1,-2.6176205E-1,5.246055E-2,-7.363438E-3,1.2089093E-1,-3.4504768E-2,1.0733184E-1,-2.214011E-3,4.5019686E-3,1.3939472E-2,3.2542974E-2,1.598396E-2,-1.3264752E-2,-3.7661728E-3,-5.580006E-3,-1.6291892E-2,9.3092555E-3,2.2639714E-4,3.7685502E-3,1.4158694E-2,-4.4517163E-3,2.0014355E-3,3.594364E-5,6.166306E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.926785E0,1.6676435E0,1.0896916E0,6.494236E-1,4.8484936E-1,1.5336278E-1,6.56734E-1,4.2462826E-1,2.1814393E-1,0E0,3.6614507E-1,1.10713676E-1,0E0,1.2737691E-1,2.8036833E-2,5.4856396E-1,3.3252525E-1,1.7697695E-1,0E0,2.0069253E-1,1.4967111E-1,7.074866E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.165913E6,2.174459E5,4.9615598E-1,2.207E3,4.328683E0,1.623319E3,4.352025E0,2.5417683E0,1.7081004E-2,1.955886E7,1.1598054E8,1.2718587E-2,4.796397E2,2.7479675E-1,1E0,7.6630195E6,9.0871654E-2,-7.363438E-3,8.834E3,2.2296374E0,1.64E2,-2.214011E-3,4.5019686E-3,1.3939472E-2,3.2542974E-2,1.598396E-2,-1.3264752E-2,-3.7661728E-3,-5.580006E-3,-1.6291892E-2,9.3092555E-3,2.2639714E-4,3.7685502E-3,1.4158694E-2,-4.4517163E-3,2.0014355E-3,3.594364E-5,6.166306E-3],"split_indices":[43,1,29,40,0,36,4,55,54,0,46,33,0,4,39,109,46,39,0,2,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.1E2,6.7E1,1.47E2,6.3E1,3.3E1,3.4E1,1.18E2,2.9E1,4E0,5.9E1,2.9E1,4E0,2.5E1,9E0,9E1,2.8E1,2.4E1,5E0,3E1,2.9E1,2.4E1,5E0,5E0,2E1,5E0,4E0,1.5E1,7.5E1,1.2E1,1.6E1,5E0,1.9E1,2.6E1,4E0,1.6E1,1.3E1,5E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.907077E-3,-1.10231504E-1,1.560937E-1,-1.7390074E-1,-1.4979227E-2,6.4361654E-2,3.3062205E-1,-2.3123567E-1,-3.241137E-2,-1.1863115E-1,3.0680554E-2,1.5049924E-2,4.4690855E-2,2.5125198E-2,2.73239E-1,-1.971846E-1,-2.1857517E-2,6.502721E-3,-6.455878E-2,-1.0979424E-2,-5.479746E-2,1.5467468E-1,-1.899492E-3,-7.1558125E-2,8.332842E-2,4.954007E-3,3.0732498E-1,3.3359108E-3,-9.961819E-3,-6.188587E-3,1.9760157E-3,-5.4547014E-3,-5.1157834E-4,1.0084352E-2,2.141548E-3,-2.753475E-3,2.3288236E-3,3.6607396E-3,-6.2513426E-3,-1.9615863E-3,4.7614938E-3,8.62813E-3,1.6565144E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7295275E0,1.0101895E0,1.7558129E0,8.037338E-1,3.2546487E-1,3.6752412E-1,3.1990767E-1,4.9509E-1,1.765889E-1,1.475777E-1,1.9420104E-1,0E0,3.2037327E-1,0E0,1.4760303E-1,3.4672666E-1,0E0,0E0,2.0097452E-1,0E0,4.041077E-2,5.9418827E-2,1.2154306E-1,1.8542328E-1,1.3849643E-1,0E0,9.61647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,18,18,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1309677E2,1.1670399E-1,1.3305582E0,5.5015685E9,2.0370931E8,2.4383854E4,1.8681E4,1.4031231E-2,2.7864842E11,1.8990291E0,1.9124458E0,1.5049924E-2,3.0327642E-1,2.5125198E-2,1.7246376E0,1.1623532E6,-2.1857517E-2,6.502721E-3,4.5E1,-1.0979424E-2,1.8508842E2,1.2183861E7,9.50379E5,1.409674E7,3.614121E8,4.954007E-3,7.2610445E8,3.3359108E-3,-9.961819E-3,-6.188587E-3,1.9760157E-3,-5.4547014E-3,-5.1157834E-4,1.0084352E-2,2.141548E-3,-2.753475E-3,2.3288236E-3,3.6607396E-3,-6.2513426E-3,-1.9615863E-3,4.7614938E-3,8.62813E-3,1.6565144E-2],"split_indices":[53,40,40,5,7,34,9,39,32,54,59,0,43,0,54,46,0,0,8,0,4,33,9,46,12,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.66E2,1.1E2,9.9E1,6.7E1,7.3E1,3.7E1,7E1,2.9E1,2E1,4.7E1,4E0,6.9E1,6E0,3.1E1,6.3E1,7E0,4E0,2.5E1,6E0,1.4E1,9E0,3.8E1,1.7E1,5.2E1,6E0,2.5E1,4E0,5.9E1,1.5E1,1E1,5E0,9E0,5E0,4E0,1.8E1,2E1,5E0,1.2E1,7E0,4.5E1,9E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.78766E-3,-5.9377663E-2,2.805717E-1,-1.324866E-1,3.744333E-2,9.797968E-2,4.031186E-1,-4.683293E-2,-2.2750878E-1,-3.3787608E-3,1.2183019E-1,4.3040212E-2,9.286942E-3,3.0033776E-1,5.981496E-1,-8.650509E-3,-1.2641405E-2,-2.2197314E-2,-1.8958826E-1,-6.8728946E-2,5.4599367E-2,1.5231122E-2,5.8440927E-2,9.559811E-2,-1.4126698E-3,1.5144936E-2,6.584264E-3,1.6348228E-2,3.3222765E-2,-1.55101E-3,6.284026E-3,-1.12308115E-2,-2.2411167E-3,-4.661134E-3,3.8490896E-3,3.9403453E-3,-2.7932785E-3,-4.4625686E-3,5.1844753E-3,1.1539118E-3,6.768676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,29,-1,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.627358E0,1.5551479E0,1.0569315E0,1.0067198E0,3.2767218E-1,1.10995054E-1,4.0167093E-1,3.2075566E-1,4.7824E-1,2.5002983E-1,3.9133808E-1,6.105414E-2,0E0,4.7471285E-2,1.3820648E-2,0E0,1.7722012E-1,0E0,3.9397097E-1,1.6198213E-1,1.3019392E-1,0E0,2.2129868E-1,2.8955169E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,30,-1,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,1.249246E6,2.174459E5,1.19331E5,4.876025E2,2.2486712E1,1.7598395E3,1.3692686E4,9.975593E4,4.8E1,6.1650256E5,1.020435E4,9.286942E-3,4.7201157E-1,2.4394053E9,-8.650509E-3,2.427E3,-2.2197314E-2,1.1306552E-1,4.0159574E-1,2.2946768E0,1.5231122E-2,2.51838E6,1E0,-1.4126698E-3,1.5144936E-2,6.584264E-3,1.6348228E-2,3.3222765E-2,-1.55101E-3,6.284026E-3,-1.12308115E-2,-2.2411167E-3,-4.661134E-3,3.8490896E-3,3.9403453E-3,-2.7932785E-3,-4.4625686E-3,5.1844753E-3,1.1539118E-3,6.768676E-3],"split_indices":[43,1,29,30,53,57,4,49,29,8,48,33,0,39,5,0,2,0,40,58,54,0,33,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.18E2,4.8E1,1.24E2,9.4E1,2E1,2.8E1,6.6E1,5.8E1,6.4E1,3E1,1.4E1,6E0,2E1,8E0,1.2E1,5.4E1,6E0,5.2E1,3E1,3.4E1,6E0,2.4E1,8E0,6E0,1.6E1,4E0,4E0,4E0,4.8E1,6E0,3.7E1,1.5E1,2.5E1,5E0,2.7E1,7E0,6E0,1.8E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.4872545E-3,-6.5911904E-2,2.459096E-1,-1.15997404E-1,5.143252E-2,1.05173156E-1,3.6781096E-1,-1.4529198E-1,-6.30318E-4,1.5969163E-2,2.7549699E-2,5.7816297E-2,1.1666496E-2,2.5568223E-1,5.7611805E-1,-1.7830033E-2,-1.299662E-1,3.5455097E-2,-8.7172285E-2,8.3272494E-2,-2.4066662E-2,5.2803885E-3,1.3995697E-2,2.765587E-1,6.341746E-3,1.5746074E-2,3.1999074E-2,-8.576292E-3,-3.4202011E-3,-1.5801423E-3,4.0399237E-3,3.5253723E-4,-6.982954E-3,1.3579358E-3,6.743551E-3,-4.7590914E-3,1.6856256E-3,2.1072815E-3,-2.1177272E-3,1.4204722E-2,6.6544395E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,-1,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1708198E0,1.3283479E0,8.7058234E-1,5.3389025E-1,4.7020566E-1,1.6785002E-1,5.0669885E-1,4.1692805E-1,1.06146485E-1,0E0,1.8608138E-1,5.073069E-2,0E0,1.115036E-2,1.2820959E-2,0E0,3.6239648E-1,9.0578474E-2,6.2322155E-2,1.010357E-1,1.6738361E-1,0E0,2.6885673E-2,2.2930145E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,-1,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4272063E0,3.165913E6,2.174459E5,4.9615598E-1,2.207E3,1.7604632E0,1.6507233E3,1E0,2.3317811E0,1.5969163E-2,1.955886E7,1.7046362E0,1.1666496E-2,4.7201157E-1,2.4394053E9,-1.7830033E-2,4.9005847E0,3.3550136E0,7.826598E5,2.160255E9,2.0382197E0,5.2803885E-3,1.6E1,2.094037E7,6.341746E-3,1.5746074E-2,3.1999074E-2,-8.576292E-3,-3.4202011E-3,-1.5801423E-3,4.0399237E-3,3.5253723E-4,-6.982954E-3,1.3579358E-3,6.743551E-3,-4.7590914E-3,1.6856256E-3,2.1072815E-3,-2.1177272E-3,1.4204722E-2,6.6544395E-3],"split_indices":[43,1,29,40,0,36,4,67,54,0,46,40,0,39,5,0,57,57,34,12,55,0,3,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.24E2,5.2E1,1.57E2,6.7E1,2.5E1,2.7E1,1.25E2,3.2E1,4E0,6.3E1,2E1,5E0,1.9E1,8E0,6E0,1.19E2,2.3E1,9E0,3E1,3.3E1,8E0,1.2E1,1.5E1,4E0,4E0,4E0,5.8E1,6.1E1,1E1,1.3E1,4E0,5E0,1.7E1,1.3E1,1.4E1,1.9E1,8E0,4E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-4.660276E-3,-7.5700805E-2,1.734371E-1,-1.3291112E-1,2.1073115E-3,7.142443E-2,3.054702E-1,-1.854218E-1,-3.4727223E-2,-8.941836E-2,2.8995635E-2,1.2489289E-2,4.6526223E-2,2.192649E-1,3.838485E-1,-1.5541852E-1,-3.6790165E-1,4.9233317E-2,-1.8913843E-1,-1.11503545E-2,-2.2351334E-2,-9.1770105E-3,9.9361196E-2,-1.13287054E-1,8.6291745E-2,2.856018E-3,1.1835231E-2,2.2234E-2,1.2256544E-2,3.7611583E-3,-7.908463E-3,-2.2461448E-2,-6.923422E-3,1.0301124E-4,8.743266E-3,-1.0047784E-2,-5.024026E-3,8.989629E-4,-3.3609788E-3,6.0103675E-3,-1.3768893E-3,5.9764185E-3,-6.265475E-4,-7.498035E-3,-1.8905048E-3,6.2626805E-3,9.1052956E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.593683E0,9.0391946E-1,1.0691748E0,5.967779E-1,2.1669729E-1,2.2123474E-1,1.5231729E-1,3.589208E-1,5.538187E-1,1.9949609E-1,1.8356255E-1,0E0,2.8304324E-1,9.6958876E-2,6.95672E-2,2.7379453E-1,1.8757331E-1,1.8467844E-1,1.1110127E-2,0E0,3.3239976E-2,1.3423409E-1,8.4311366E-2,2.553396E-2,1.4649001E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8045242E2,1.8509178E-1,1.8829846E-1,1.9462195E2,1.8681856E8,1.4153711E-3,5.76E3,1.2869189E5,2.142943E1,8E0,2.704375E6,1.2489289E-2,1.479938E6,2.0787016E5,2.9898608E-1,3.8757736E-7,1.7218688E8,7.032E3,1.3469651E3,-1.11503545E-2,1.7011462E5,2.3755E4,2.8808496E7,4.3085215E11,2.3797293E0,2.856018E-3,1.1835231E-2,2.2234E-2,1.2256544E-2,3.7611583E-3,-7.908463E-3,-2.2461448E-2,-6.923422E-3,1.0301124E-4,8.743266E-3,-1.0047784E-2,-5.024026E-3,8.989629E-4,-3.3609788E-3,6.0103675E-3,-1.3768893E-3,5.9764185E-3,-6.265475E-4,-7.498035E-3,-1.8905048E-3,6.2626805E-3,9.1052956E-5],"split_indices":[53,40,39,53,7,39,2,48,57,3,9,0,33,29,39,39,7,10,4,0,29,9,9,32,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.02E2,8E1,1.16E2,8.6E1,4.6E1,3.4E1,7.5E1,4.1E1,1.9E1,6.7E1,4E0,4.2E1,1.8E1,1.6E1,6.6E1,9E0,2.7E1,1.4E1,5E0,1.4E1,4.4E1,2.3E1,8E0,3.4E1,4E0,1.4E1,7E0,9E0,4E0,6.2E1,5E0,4E0,2.1E1,6E0,9E0,5E0,8E0,6E0,5E0,3.9E1,1.8E1,5E0,4E0,4E0,2.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.4954677E-3,-6.2547676E-2,1.9570342E-1,-1.2018361E-1,1.2433611E-2,2.976111E-1,9.40211E-2,-7.982474E-2,-2.2363111E-1,8.368774E-3,-4.0375446E-3,2.8246444E-2,2.080085E-1,1.2471063E-1,-2.345195E-3,-1.2463708E-1,2.6658533E-2,-3.1819725E-1,-9.5242985E-2,-6.591116E-2,6.412437E-2,6.102004E-2,2.8627875E-1,2.4245268E-2,1.8159871E-1,-1.2777416E-2,-4.4097817E-3,8.058981E-3,-2.863397E-3,-1.8598793E-2,-5.7881386E-3,-4.4350702E-4,-6.9635296E-3,-5.1220683E-3,1.3425859E-3,4.655637E-3,-2.6230586E-3,7.960848E-3,-2.8102314E-3,5.7578427E-3,1.4890083E-2,-3.7201787E-3,3.9995285E-3,6.123717E-3,1.1321517E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4429915E0,8.8742566E-1,6.870105E-1,4.6645153E-1,2.5175536E-1,8.347542E-1,1.6650549E-1,4.0892267E-1,3.5821128E-1,0E0,3.542727E-1,0E0,3.046664E-1,1.6705552E-1,0E0,2.3813432E-1,3.536628E-1,2.4635649E-1,7.045956E-2,1.929037E-1,1.8495904E-1,1.6077776E-1,6.730461E-2,8.5959695E-2,2.6661694E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,2.1619815E-1,9.520617E5,4.1295107E-2,2.3755E4,1E0,2.008498E1,4.780434E-3,5.267875E5,8.368774E-3,9.50379E5,2.8246444E-2,1.1047089E-1,1.19881555E-1,-2.345195E-3,1.9E3,1.6316831E4,2.0201733E9,6.065435E5,1.1102881E0,5.0128145E0,3.738E3,4.063E3,2.7713644E1,6.7E1,-1.2777416E-2,-4.4097817E-3,8.058981E-3,-2.863397E-3,-1.8598793E-2,-5.7881386E-3,-4.4350702E-4,-6.9635296E-3,-5.1220683E-3,1.3425859E-3,4.655637E-3,-2.6230586E-3,7.960848E-3,-2.8102314E-3,5.7578427E-3,1.4890083E-2,-3.7201787E-3,3.9995285E-3,6.123717E-3,1.1321517E-2],"split_indices":[53,42,34,43,9,67,59,40,34,0,9,0,39,39,0,12,48,5,34,42,55,0,2,57,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2.04E2,6.8E1,1.15E2,8.9E1,3.3E1,3.5E1,8.4E1,3.1E1,7E0,8.2E1,6E0,2.7E1,2.9E1,6E0,5.9E1,2.5E1,1.7E1,1.4E1,4.3E1,3.9E1,1E1,1.7E1,1.1E1,1.8E1,8E0,5.1E1,9E0,1.6E1,1.1E1,6E0,6E0,8E0,2.9E1,1.4E1,3E1,9E0,5E0,5E0,4E0,1.3E1,4E0,7E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.0292259E-2,-3.4356654E-2,2.1874754E-1,-7.8566894E-2,6.119629E-2,7.5043626E-2,3.2384533E-1,-1.257327E-1,-9.874781E-3,8.480252E-2,-7.956635E-2,-4.1844492E-4,1.3796355E-1,2.3364724E-1,4.2463887E-1,-7.779738E-2,-2.9680678E-1,-1.1077852E-2,9.19967E-3,1.6997752E-1,1.6199687E-2,-5.853418E-3,1.3548402E-4,2.0969927E-3,-2.1323129E-3,2.7834452E-3,9.456622E-3,1.1812086E-2,5.863621E-3,1.0908755E-2,2.4165334E-2,-9.9446485E-3,-2.1899713E-3,-1.811231E-2,-6.2175826E-3,-1.2788196E-3,2.6146667E-3,2.2568621E-3,1.2502025E-2,-9.481612E-4,5.9332373E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5856378E0,9.7220004E-1,7.1574235E-1,5.074257E-1,2.4931735E-1,1.04515225E-1,1.553433E-1,7.39555E-1,2.8828573E-1,3.6537084E-1,4.354716E-2,2.5314309E-2,5.0300643E-2,8.719981E-3,1.0139418E-1,2.9889545E-1,2.5936043E-1,0E0,1.0891681E-1,3.3066946E-1,1.5189107E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5648934E0,3.135E3,2.787E3,1.5415665E-2,1.5830926E1,6.765595E7,2.9391026E1,5.1653096E-3,2.718E4,4.8883E4,1E0,1.4679102E2,1.2984486E6,3.160848E6,2.6680366E5,9.421523E-5,1.1152775E7,-1.1077852E-2,5.101512E0,7.051287E4,1.19881555E-1,-5.853418E-3,1.3548402E-4,2.0969927E-3,-2.1323129E-3,2.7834452E-3,9.456622E-3,1.1812086E-2,5.863621E-3,1.0908755E-2,2.4165334E-2,-9.9446485E-3,-2.1899713E-3,-1.811231E-2,-6.2175826E-3,-1.2788196E-3,2.6146667E-3,2.2568621E-3,1.2502025E-2,-9.481612E-4,5.9332373E-3],"split_indices":[42,2,2,39,58,5,57,39,9,9,109,53,34,9,29,43,46,0,57,29,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.28E2,4.8E1,1.56E2,7.2E1,2.1E1,2.7E1,9.2E1,6.4E1,6.2E1,1E1,1E1,1.1E1,1.6E1,1.1E1,7.3E1,1.9E1,4E0,6E1,2.7E1,3.5E1,6E0,4E0,5E0,5E0,6E0,5E0,1.2E1,4E0,5E0,6E0,1.2E1,6.1E1,1.1E1,8E0,3.4E1,2.6E1,1.3E1,1.4E1,2.7E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-5.9847976E-3,-8.624195E-2,1.14453435E-1,-1.3104282E-1,4.211257E-2,2.0858116E-2,9.802297E-2,-1.6680013E-2,-1.18596114E-1,1.4497058E-2,7.822042E-3,3.0587737E-2,1.4084576E-1,-2.5856104E-1,-9.953093E-2,7.431583E-2,-5.2982405E-2,5.0977536E-2,-4.9284142E-3,1.4990207E-2,1.0656606E-1,-1.834578E-2,-4.780183E-3,-2.9989132E-3,-8.4703015E-3,-5.472433E-4,4.447988E-3,2.3905463E-3,-4.0005483E-3,1.5483845E-3,7.4683735E-3,6.4264913E-3,-3.3000865E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,15,17,19,21,23,25,27,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.755023E0,9.888363E-1,5.8977664E-1,3.2256746E-1,4.1842562E-1,0E0,3.133459E-1,0E0,2.9981697E-1,0E0,1.6970527E-1,1.289332E-1,3.8959837E-1,2.617069E-1,3.0459678E-1,3.958281E-2,8.27689E-2,6.979805E-2,0E0,0E0,2.1879053E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,16,18,20,22,24,26,28,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.4269168E-1,2.796589E6,1.4396E4,1E0,2.207E3,2.0858116E-2,3.4071463E2,-1.6680013E-2,1E0,1.4497058E-2,1.955886E7,1.9E1,8E0,1.9E1,4.352025E0,2.80215E6,1E1,7.432E3,-4.9284142E-3,1.4990207E-2,6.812E3,-1.834578E-2,-4.780183E-3,-2.9989132E-3,-8.4703015E-3,-5.472433E-4,4.447988E-3,2.3905463E-3,-4.0005483E-3,1.5483845E-3,7.4683735E-3,6.4264913E-3,-3.3000865E-4],"split_indices":[43,1,9,67,0,0,53,0,109,0,46,3,3,3,55,12,3,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.7E2,1.13E2,1.26E2,4.4E1,4E0,1.09E2,5E0,1.21E2,4E0,4E1,4.3E1,6.6E1,1.3E1,1.08E2,1.9E1,2.1E1,3.8E1,5E0,9E0,5.7E1,6E0,7E0,7.8E1,3E1,4E0,1.5E1,5E0,1.6E1,3.4E1,4E0,4.4E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-8.450708E-3,-7.176439E-2,1.3501188E-1,-5.7520848E-2,-2.689406E-1,5.7696655E-2,2.4293911E-1,-1.4114341E-1,-2.7192185E-2,-1.5506827E-2,-4.281715E-3,7.692729E-2,-4.721945E-3,-6.0996685E-5,2.735135E-1,-2.1443115E-1,-3.3215422E-2,-4.106821E-4,-1.0831231E-1,1.1778847E-1,4.551584E-3,1.5411691E-1,3.470945E-1,-1.40729165E-2,-6.079914E-3,-4.7486667E-3,3.2908816E-3,-2.6033137E-3,2.0756775E-3,-7.715837E-3,1.513634E-5,3.5040742E-3,1.02624325E-2,-2.4551002E-3,5.9480625E-3,8.493144E-3,2.8816864E-3,2.2447936E-2,1.2759445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.580219E0,5.3780234E-1,7.119478E-1,4.6535176E-1,1.2389976E-1,1.6814464E-1,2.6921844E-1,3.8045776E-1,2.9759058E-1,0E0,0E0,1.3845962E-1,0E0,0E0,2.2763705E-1,1.7757607E-1,1.6110398E-1,2.6813293E-1,2.1900013E-1,1.1446679E-1,1.367331E-1,2.734676E-2,5.160451E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0471938E6,3.7085085E0,1.2711891E0,6.846054E7,2.5234535E1,1.25440235E1,1E0,5.3091907E2,1.7303246E7,-1.5506827E-2,-4.281715E-3,2.247882E0,-4.721945E-3,-6.0996685E-5,7.2610445E8,3.5483692E7,5.101512E0,6.75553E5,3.135E3,2.7307575E5,1.19881555E-1,3E1,3.0256798E0,-1.40729165E-2,-6.079914E-3,-4.7486667E-3,3.2908816E-3,-2.6033137E-3,2.0756775E-3,-7.715837E-3,1.513634E-5,3.5040742E-3,1.02624325E-2,-2.4551002E-3,5.9480625E-3,8.493144E-3,2.8816864E-3,2.2447936E-2,1.2759445E-2],"split_indices":[49,54,40,7,54,58,17,4,46,0,0,54,0,0,12,7,57,9,2,29,39,8,55,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.96E2,8.6E1,1.84E2,1.2E1,5.1E1,3.5E1,4.8E1,1.36E2,8E0,4E0,4.6E1,5E0,4E0,3.1E1,2.8E1,2E1,1.03E2,3.3E1,2.9E1,1.7E1,1.3E1,1.8E1,1.2E1,1.6E1,1.2E1,8E0,4.6E1,5.7E1,2.1E1,1.2E1,2.2E1,7E0,1.2E1,5E0,9E0,4E0,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.3999596E-3,-5.3627178E-2,1.7385219E-1,-9.98431E-2,3.265941E-2,8.181625E-2,2.6619783E-1,-3.333813E-2,-1.4404352E-1,9.161319E-2,-9.618459E-3,-2.5562001E-3,1.0132802E-1,2.0116904E-1,2.0454504E-2,-1.1670704E-1,2.830405E-2,-2.3186274E-2,-1.1914832E-1,4.2665742E-2,1.7672983E-1,-1.1445278E-1,3.8036346E-2,-7.920985E-3,1.3967207E-1,2.1414132E-1,4.781695E-3,-7.895158E-3,-2.1935427E-3,3.466034E-3,-3.3377171E-3,-1.1594231E-2,-3.9683166E-3,-1.0427313E-3,4.3066004E-3,1.2174628E-2,2.7428912E-3,-7.4464804E-3,1.382859E-4,4.7936807E-3,-1.4006656E-3,1.0964437E-3,-1.7528867E-3,3.313915E-3,7.335737E-3,5.9034945E-3,1.0675498E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,-1,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6284397E0,8.407019E-1,5.4766154E-1,3.9791596E-1,1.8532175E-1,9.941433E-2,2.9275393E-1,2.910896E-1,6.739975E-1,1.21613204E-1,2.2417118E-1,0E0,1.3096684E-1,1.0581255E-2,0E0,8.261943E-2,1.6172038E-1,0E0,3.1936193E-1,7.189447E-2,9.726593E-2,7.823719E-2,1.4340574E-1,9.518452E-3,1.913932E-2,3.7784576E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,-1,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,1.781973E6,2.174459E5,9.0649E4,1.3844932E7,3.9579918E4,1.7598395E3,1.09E3,2.4540474E8,1.0705044E3,1.4813267E3,-2.5562001E-3,1E0,4.7201157E-1,2.0454504E-2,1.3692686E4,5.2158275E0,-2.3186274E-2,9.723579E1,4.354824E5,1.405115E3,2.3684846E7,8.652176E6,1.8454477E2,2.300289E8,3.5055136E2,4.781695E-3,-7.895158E-3,-2.1935427E-3,3.466034E-3,-3.3377171E-3,-1.1594231E-2,-3.9683166E-3,-1.0427313E-3,4.3066004E-3,1.2174628E-2,2.7428912E-3,-7.4464804E-3,1.382859E-4,4.7936807E-3,-1.4006656E-3,1.0964437E-3,-1.7528867E-3,3.313915E-3,7.335737E-3,5.9034945E-3,1.0675498E-2],"split_indices":[43,1,29,30,46,29,4,2,5,56,56,0,107,39,0,49,55,0,53,29,4,46,30,56,7,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.09E2,6.6E1,1.36E2,7.3E1,3.4E1,3.2E1,5.5E1,8.1E1,3E1,4.3E1,4E0,3E1,2.5E1,7E0,2.3E1,3.2E1,4E0,7.7E1,2E1,1E1,1.3E1,3E1,8E0,2.2E1,2.1E1,4E0,1.2E1,1.1E1,2.2E1,1E1,1.4E1,6.3E1,9E0,1.1E1,5E0,5E0,9E0,4E0,1.5E1,1.5E1,4E0,4E0,6E0,1.6E1,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.8314502E-3,-6.785432E-2,1.0660577E-1,-1.0692218E-1,4.023653E-2,8.684095E-2,1.785696E-2,-7.905963E-2,-1.95647E-1,1.3693513E-2,7.0827315E-3,1.888039E-2,1.2837237E-1,-1.00642435E-1,7.501121E-2,-2.3125635E-1,6.6332467E-4,6.9846846E-2,-3.7670296E-2,5.4251872E-2,-6.1477557E-2,1.6090888E-1,9.190066E-3,7.925395E-3,-5.379347E-3,2.5073162E-4,5.023686E-3,-4.3994854E-3,-1.5595466E-2,4.4568987E-3,-6.541401E-4,-3.7478583E-3,1.4592746E-3,7.066802E-3,1.2514908E-3,9.802931E-4,-5.334769E-3,1.1568111E-2,4.988328E-3,1.946574E-3,-3.0131019E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.009789E0,6.9263124E-1,5.83336E-1,2.8118432E-1,3.7306362E-1,2.9393852E-1,0E0,3.1680053E-1,2.127688E-1,0E0,1.1498651E-1,1.1987284E-1,2.5057673E-1,3.8237488E-1,2.7873583E-2,3.1765282E-1,0E0,3.9893344E-2,7.53841E-2,7.373153E-2,6.0993165E-2,2.1497655E-1,3.9810337E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.259334E-1,3.343327E6,4.328683E0,4.352025E0,2.207E3,3.4071463E2,1.785696E-2,2.2615366E0,2.2291185E2,1.3693513E-2,1.1572553E7,1.5E1,6.464779E6,6.7473096E9,3.1676166E2,7.6630195E6,6.6332467E-4,1.9879582E7,2.0382197E0,1.1623532E6,1.101829E6,4.0012E4,9.723711E0,7.925395E-3,-5.379347E-3,2.5073162E-4,5.023686E-3,-4.3994854E-3,-1.5595466E-2,4.4568987E-3,-6.541401E-4,-3.7478583E-3,1.4592746E-3,7.066802E-3,1.2514908E-3,9.802931E-4,-5.334769E-3,1.1568111E-2,4.988328E-3,1.946574E-3,-3.0131019E-3],"split_indices":[43,1,36,55,0,53,0,54,53,0,46,3,30,32,4,46,0,33,55,46,46,9,58,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.62E2,1.1E2,1.19E2,4.3E1,1.04E2,6E0,9.2E1,2.7E1,4E0,3.9E1,4E1,6.4E1,8.1E1,1.1E1,2.3E1,4E0,1.6E1,2.3E1,2.8E1,1.2E1,5E1,1.4E1,4E0,7.7E1,4E0,7E0,1.1E1,1.2E1,1.2E1,4E0,1.4E1,9E0,5E0,2.3E1,5E0,7E0,1.7E1,3.3E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[8.826118E-3,-6.6067144E-2,1.15737684E-1,-1.11695945E-1,-1.0645387E-3,3.6324617E-2,2.0036052E-1,-8.591993E-2,-2.80672E-1,2.1383706E-1,-3.3828095E-2,-7.3578716E-3,7.096409E-2,1.2487603E-1,3.7183142E-1,-7.113199E-2,-1.21476175E-2,-1.9641371E-2,-5.3172866E-3,1.393858E-2,3.7829317E-3,-1.2053422E-1,7.177866E-3,2.015588E-2,-6.098117E-3,-1.5559113E-2,1.0925936E-1,-1.1436571E-3,1.5840265E-1,2.49162E-2,2.4566016E-1,-4.057148E-3,4.0760175E-3,-2.7465879E-3,-1.11647835E-2,4.008514E-3,-8.292376E-4,-3.6880438E-4,4.663739E-3,1.5948742E-4,-1.7992699E-3,1.469273E-3,8.172358E-3,7.945264E-3,2.181881E-3,1.4243299E-2,6.4925323E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,37,-1,39,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1775312E0,4.7443968E-1,7.453798E-1,3.8530433E-1,4.789767E-1,8.97973E-2,6.525104E-1,1.9961512E-1,2.399984E-1,8.048308E-2,2.1063778E-1,9.559665E-2,1.1104317E-1,1.9969398E-1,2.0891309E-1,2.1936658E-1,0E0,0E0,0E0,0E0,0E0,1.2660754E-1,8.406933E-2,5.337534E-2,0E0,5.057159E-3,1.1573723E-1,0E0,4.21471E-2,0E0,2.9699922E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,21,21,22,22,23,23,25,25,26,26,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,38,-1,40,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.4269168E-1,1.6408995E11,2.216135E5,4.33435E5,1.904E3,7.025488E6,1.3662124E3,3.5300907E-1,6.132018E2,6.878795E7,1.9462195E2,8.6788906E5,1.8491526E0,1.652E3,3.0256798E0,8.749103E5,-1.21476175E-2,-1.9641371E-2,-5.3172866E-3,1.393858E-2,3.7829317E-3,1.591046E0,5.658968E4,5.636998E5,-6.098117E-3,7.980742E6,5.792986E4,-1.1436571E-3,1.3589149E6,2.49162E-2,1.91016E5,-4.057148E-3,4.0760175E-3,-2.7465879E-3,-1.11647835E-2,4.008514E-3,-8.292376E-4,-3.6880438E-4,4.663739E-3,1.5948742E-4,-1.7992699E-3,1.469273E-3,8.172358E-3,7.945264E-3,2.181881E-3,1.4243299E-2,6.4925323E-3],"split_indices":[43,32,29,30,0,52,4,43,4,7,53,48,54,2,55,34,0,0,0,0,0,55,34,48,0,30,29,0,48,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.59E2,1.11E2,9.3E1,6.6E1,5.8E1,5.3E1,8.2E1,1.1E1,8E0,5.8E1,2.6E1,3.2E1,3.8E1,1.5E1,7.7E1,5E0,5E0,6E0,4E0,4E0,1.8E1,4E1,2.2E1,4E0,1E1,2.2E1,7E0,3.1E1,5E0,1E1,7E1,7E0,1.3E1,5E0,9E0,3.1E1,1.7E1,5E0,6E0,4E0,1.1E1,1.1E1,2.7E1,4E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[5.0148596E-3,-4.0661376E-2,1.4523451E-1,-9.008749E-2,1.707953E-2,1.1323153E-1,1.7454334E-2,-6.537531E-2,-1.2222227E-2,-6.785297E-2,3.529285E-2,5.6190148E-2,1.7817006E-1,-9.044338E-2,4.3179322E-2,-3.4122176E-2,-5.9681884E-3,2.0582078E-2,1.2971044E-1,8.834468E-2,-5.0209495E-3,1.4130776E-3,2.0053892E-1,5.12178E-3,-4.944072E-3,3.5777597E-3,-3.4967444E-3,-4.571528E-3,2.7048524E-4,1.16039504E-4,5.1651546E-3,8.40273E-3,3.3826302E-3,5.4376996E-3,-1.9528405E-3,-1.7457104E-3,2.541237E-3,1.0513924E-2,2.0066393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,27,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8248419E0,6.1512905E-1,4.809891E-1,4.8538065E-1,1.57085E-1,2.2335213E-1,0E0,2.8419453E-1,0E0,3.3083506E-2,1.1261358E-1,6.933108E-2,8.951378E-2,2.989486E-1,8.8168316E-2,3.4469355E-2,0E0,1.204974E-1,1.9252628E-2,9.451428E-2,2.7843595E-2,0E0,9.9212885E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,28,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2185057E0,1.1670399E-1,1.3821641E0,4.4326835E0,1.503E3,2.0074467E5,1.7454334E-2,4.5E1,-1.2222227E-2,3.68646E5,1.08540284E12,1.948181E0,1.8618844E0,1E1,4.7907645E8,9.57E2,-5.9681884E-3,4.7321553E5,2.4434419E12,4.2656995E6,3.1889508E0,1.4130776E-3,4.9691647E-1,5.12178E-3,-4.944072E-3,3.5777597E-3,-3.4967444E-3,-4.571528E-3,2.7048524E-4,1.16039504E-4,5.1651546E-3,8.40273E-3,3.3826302E-3,5.4376996E-3,-1.9528405E-3,-1.7457104E-3,2.541237E-3,1.0513924E-2,2.0066393E-3],"split_indices":[43,40,35,55,2,29,0,8,0,29,32,43,54,10,12,2,0,29,32,48,43,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.14E2,6.9E1,1.15E2,9.9E1,6.2E1,7E0,1.02E2,1.3E1,1.7E1,8.2E1,3.4E1,2.8E1,8.3E1,1.9E1,1.2E1,5E0,7.2E1,1E1,2.2E1,1.2E1,4E0,2.4E1,6E0,7.7E1,1.5E1,4E0,4E0,8E0,6.1E1,1.1E1,4E0,6E0,1.8E1,4E0,8E0,4E0,2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.2870723E-2,-4.9312957E-2,1.5442432E-1,-1.2296996E-1,-5.5058524E-3,1.9981392E-2,1.19145766E-1,-3.973438E-2,-1.8724707E-1,-1.9829726E-2,1.23802274E-1,-4.528465E-3,1.6050774E-1,-8.023268E-2,7.763429E-2,-2.2347286E-2,-1.472971E-1,-3.8787395E-2,7.155142E-2,2.0040686E-3,1.6356274E-1,2.520448E-3,-1.8880231E-3,-3.815458E-4,1.8407553E-1,-8.574226E-3,-1.3131945E-3,-4.6848468E-4,6.347679E-3,-9.633589E-3,-3.471098E-3,-3.8900268E-3,2.7353826E-4,4.9813315E-3,-3.4378606E-3,3.717361E-3,9.837722E-3,9.444492E-3,1.9327286E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6354089E0,7.09197E-1,4.217013E-1,4.3127143E-1,2.5990984E-1,0E0,2.2580492E-1,1.8209839E-1,4.7637892E-1,2.2105438E-1,3.756495E-2,2.819101E-2,1.328035E-1,1.4816095E-1,5.429561E-2,0E0,1.7129254E-1,2.151461E-1,1.2595034E-1,0E0,2.010025E-2,0E0,0E0,0E0,7.815844E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,5.283708E0,1.6771E4,1.56221E5,6.767991E5,1.9981392E-2,1.2358527E8,1E0,9.975593E4,9.821357E-1,6.5284515E6,1.9066478E0,1E0,4.780434E-3,1.073E3,-2.2347286E-2,4.24048E5,2.8352614E9,5.2380065E6,2.0040686E-3,1.2718601E-3,2.520448E-3,-1.8880231E-3,-3.815458E-4,5.557553E-1,-8.574226E-3,-1.3131945E-3,-4.6848468E-4,6.347679E-3,-9.633589E-3,-3.471098E-3,-3.8900268E-3,2.7353826E-4,4.9813315E-3,-3.4378606E-3,3.717361E-3,9.837722E-3,9.444492E-3,1.9327286E-3],"split_indices":[43,57,9,1,29,0,5,8,29,42,46,54,17,42,2,0,1,5,48,0,58,0,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.19E2,4.7E1,8.1E1,1.38E2,4E0,4.3E1,3.6E1,4.5E1,1.25E2,1.3E1,1.1E1,3.2E1,2.7E1,9E0,4E0,4.1E1,1.04E2,2.1E1,5E0,8E0,4E0,7E0,4E0,2.8E1,8E0,1.9E1,4E0,5E0,2.1E1,2E1,5.1E1,5.3E1,1.7E1,4E0,4E0,4E0,2.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.1806652E-3,-5.6144875E-2,9.210386E-2,-8.710738E-2,1.0394218E-3,6.804833E-2,1.834925E-2,-1.2235583E-1,-5.7547363E-3,2.2315871E-2,-1.2200666E-1,1.4027314E-1,2.7860343E-2,-2.0972583E-1,-6.398443E-2,7.5510456E-3,-3.38565E-2,7.849221E-2,-1.5247225E-2,-1.3981955E-3,-8.7129725E-3,7.693887E-2,1.7759348E-1,-9.5497906E-2,6.6637225E-2,-1.3643722E-2,-2.5374359E-3,-8.6167234E-4,-6.5732924E-3,-3.6350319E-3,1.8584944E-3,-7.659624E-5,5.890809E-3,-2.870576E-3,2.291214E-3,-1.3748664E-4,5.0280285E-3,9.0616075E-3,3.7092215E-3,3.810046E-5,-8.282146E-3,-2.3530498E-4,5.318013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,-1,29,31,33,-1,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3690357E0,2.975462E-1,7.0265937E-1,3.118593E-1,1.5980172E-1,2.6603094E-1,0E0,3.7417376E-1,1.6775301E-1,1.1092497E-1,4.9706176E-2,6.433684E-2,2.9815406E-1,3.8541102E-1,1.6495657E-1,0E0,1.0359995E-1,8.345856E-2,1.0109051E-1,0E0,0E0,3.6433116E-2,2.1446764E-2,1.22079834E-1,1.6538303E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,-1,30,32,34,-1,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.0283392E8,1.3821641E0,2.951829E9,7.807318E2,7.3917985E6,1.834925E-2,7.5646E4,2.6017979E11,4.53E2,7E0,2.744E3,1.299938E6,1.8196162E0,4.4326835E0,7.5510456E-3,4.5E1,2.25E2,2.636E3,-1.3981955E-3,-8.7129725E-3,2.1120381E0,2.5497077E0,1.1936242E7,6.2143734E4,-1.3643722E-2,-2.5374359E-3,-8.6167234E-4,-6.5732924E-3,-3.6350319E-3,1.8584944E-3,-7.659624E-5,5.890809E-3,-2.870576E-3,2.291214E-3,-1.3748664E-4,5.0280285E-3,9.0616075E-3,3.7092215E-3,3.810046E-5,-8.282146E-3,-2.3530498E-4,5.318013E-3],"split_indices":[53,7,35,5,4,46,0,9,32,0,3,2,30,54,55,0,8,0,10,0,0,54,54,46,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.67E2,9.8E1,1.08E2,5.9E1,9.2E1,6E0,7.5E1,3.3E1,5.1E1,8E0,3.2E1,6E1,2.9E1,4.6E1,4E0,2.9E1,2E1,3.1E1,4E0,4E0,1.3E1,1.9E1,1.4E1,4.6E1,1.8E1,1.1E1,3E1,1.6E1,1.8E1,1.1E1,8E0,1.2E1,1.8E1,1.3E1,4E0,9E0,1.5E1,4E0,7E0,7E0,1.9E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.9337335E-3,-3.9891645E-2,1.4060307E-1,-8.3292015E-2,1.1802642E-2,7.7317625E-2,2.5395557E-1,-1.2125488E-1,5.492631E-3,-7.631845E-2,3.3685382E-2,5.234726E-2,1.2734922E-2,1.9376314E-1,2.0993467E-2,-7.747123E-2,-2.0292796E-1,5.597367E-3,-1.7151792E-2,-9.814065E-3,-1.6494332E-2,6.527197E-3,9.31723E-2,-1.8495886E-2,9.937916E-2,1.2404649E-2,1.0048562E-1,-9.509178E-3,-1.5011583E-3,-1.8705236E-3,-1.1744777E-2,1.6511722E-3,-3.8715273E-3,8.897604E-4,-2.3136665E-3,-2.1418722E-3,2.067697E-3,5.5621294E-3,-6.664678E-4,-3.954101E-3,6.4587255E-4,1.5890079E-4,6.904385E-3,6.5283575E-3,1.0385205E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,-1,31,-1,33,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7388164E0,4.8334655E-1,4.896878E-1,3.950168E-1,1.9347557E-1,2.1939912E-1,2.1564698E-1,2.7750647E-1,9.671413E-2,1.602114E-1,1.2863638E-1,1.4510925E-1,0E0,1.21136725E-1,0E0,3.1175548E-1,2.2669852E-1,0E0,1.13023594E-1,0E0,1.9109117E-2,1.1611238E-1,7.612191E-2,4.0654875E-2,1.2564111E-1,0E0,3.2212287E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,-1,32,-1,34,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,2.7558826E-2,2.8041378E5,4.5537444E9,1.8681856E8,1.3333261E2,2.9409406E-1,5.1653096E-3,2.6017979E11,8E0,2.704375E6,2E0,1.2734922E-2,4.0012E4,2.0993467E-2,3.3617156E-4,1.609E3,5.597367E-3,1.710607E7,-9.814065E-3,1.7011462E5,8.749103E5,2.8808496E7,1.479938E6,1.19881555E-1,1.2404649E-2,2.6674094E7,-9.509178E-3,-1.5011583E-3,-1.8705236E-3,-1.1744777E-2,1.6511722E-3,-3.8715273E-3,8.897604E-4,-2.3136665E-3,-2.1418722E-3,2.067697E-3,5.5621294E-3,-6.664678E-4,-3.954101E-3,6.4587255E-4,1.5890079E-4,6.904385E-3,6.5283575E-3,1.0385205E-3],"split_indices":[53,39,29,5,7,57,35,39,32,3,9,54,0,9,0,40,2,0,46,0,29,34,9,33,39,0,33,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.14E2,7E1,1.16E2,9.8E1,4.6E1,2.4E1,8.1E1,3.5E1,1.9E1,7.9E1,4.2E1,4E0,2E1,4E0,5.4E1,2.7E1,5E0,3E1,5E0,1.4E1,5.5E1,2.4E1,1.7E1,2.5E1,1E1,1E1,1.3E1,4.1E1,7E0,2E1,1.7E1,1.3E1,7E0,7E0,2.3E1,3.2E1,1.9E1,5E0,5E0,1.2E1,9E0,1.6E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[8.771363E-3,-2.8280202E-2,1.6915616E-1,-8.118126E-2,1.4528615E-2,1.08422175E-1,3.0768958E-1,-1.1792901E-1,-4.11214E-3,2.0634761E-1,-6.0964277E-4,1.5790923E-2,1.421452E-1,2.0596057E-2,1.931297E-1,-8.907537E-2,-2.9397744E-1,-2.9001337E-2,5.189877E-3,1.1955125E-2,5.145708E-3,-8.2372695E-2,2.4201728E-2,2.3687992E-3,-1.716851E-3,1.7574988E-1,9.243631E-2,3.3352436E-3,1.2040457E-2,-2.9449489E-3,-1.0197102E-2,-1.6838739E-2,-7.5243185E-3,2.3902429E-3,-3.537763E-3,9.6614705E-4,-6.2255375E-3,1.7316738E-3,-3.93636E-3,9.433623E-3,3.5211614E-3,1.2946448E-3,5.428114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6152034E0,5.016631E-1,3.9573765E-1,2.7989465E-1,3.572939E-1,1.13336265E-1,1.535821E-1,3.1414956E-1,9.923074E-2,1.636371E-2,2.3528871E-1,2.2739183E-2,2.8856456E-2,0E0,6.358999E-2,1.8200159E-1,2.453971E-2,1.1237423E-1,0E0,0E0,0E0,1.5030728E-1,1.341376E-1,0E0,0E0,2.7730197E-2,1.7973207E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,2.0139322E2,1.623319E3,1.848519E-2,2.8558656E4,1.7518893E2,2.7479675E-1,4.583037E-2,1.3431462E7,6.878795E7,8.12E2,6.6875E-1,3.13423E6,2.0596057E-2,3.91101E5,4.4326835E0,1.3628491E11,2.5187686E-1,5.189877E-3,1.1955125E-2,5.145708E-3,1.0706395E1,1.5830926E1,2.3687992E-3,-1.716851E-3,4.7201157E-1,2.300289E8,3.3352436E-3,1.2040457E-2,-2.9449489E-3,-1.0197102E-2,-1.6838739E-2,-7.5243185E-3,2.3902429E-3,-3.537763E-3,9.6614705E-4,-6.2255375E-3,1.7316738E-3,-3.93636E-3,9.433623E-3,3.5211614E-3,1.2946448E-3,5.428114E-3],"split_indices":[43,53,4,39,34,53,39,43,9,7,0,57,46,0,9,55,32,43,0,0,0,57,58,0,0,39,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,2.2E2,5E1,9.8E1,1.22E2,3.6E1,1.4E1,6.6E1,3.2E1,8E0,1.14E2,1E1,2.6E1,5E0,9E0,5.8E1,8E0,2.7E1,5E0,4E0,4E0,2.6E1,8.8E1,6E0,4E0,1.4E1,1.2E1,4E0,5E0,5E1,8E0,4E0,4E0,1E1,1.7E1,9E0,1.7E1,7.9E1,9E0,1E1,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[7.593201E-3,-2.7012795E-2,1.4464566E-1,-6.884295E-2,3.174992E-2,3.0718607E-1,9.333496E-2,-1.02020964E-1,-6.039154E-3,4.2197295E-2,-5.83662E-3,1.8307963E-2,7.957926E-3,3.1013912E-2,1.5424629E-1,-6.2414363E-2,-2.826908E-1,1.1083914E-2,-7.766553E-2,1.047227E-2,2.8296769E-2,5.7203073E-2,-3.4396206E-3,1.8428743E-1,2.9257082E-3,-6.063977E-3,-8.95553E-4,-1.620407E-2,-3.3971935E-3,5.3111142E-3,-2.1069408E-4,-5.1049036E-3,-1.3315418E-3,-3.917683E-3,1.99068E-3,-6.94043E-4,3.5673548E-3,4.5531774E-3,9.415893E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,-1,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3176732E0,5.4787356E-1,4.3837214E-1,2.7026558E-1,1.5936816E-1,8.924997E-2,1.622428E-1,5.90592E-1,5.727815E-2,2.2017954E-1,0E0,0E0,0E0,6.795497E-2,4.8233807E-2,2.0881054E-1,1.9105434E-1,6.257869E-2,1.1363413E-2,0E0,1.4428504E-1,2.901655E-2,0E0,7.4062347E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,-1,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,2.426E3,7.013774E5,1.848519E-2,3.8016708E7,3.0256798E0,2.0787016E5,5.740651E-3,2.3424656E0,4.92E2,-5.83662E-3,1.8307963E-2,7.957926E-3,1.10018056E8,9.042351E8,4.949154E-4,2.2394615E2,1.3523328E9,1.8E1,1.047227E-2,2E0,1.647E3,-3.4396206E-3,3.6893954E2,2.9257082E-3,-6.063977E-3,-8.95553E-4,-1.620407E-2,-3.3971935E-3,5.3111142E-3,-2.1069408E-4,-5.1049036E-3,-1.3315418E-3,-3.917683E-3,1.99068E-3,-6.94043E-4,3.5673548E-3,4.5531774E-3,9.415893E-3],"split_indices":[43,2,34,39,46,55,29,39,54,10,0,0,0,33,5,40,53,32,3,0,8,2,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.21E2,5.5E1,1.29E2,9.2E1,1.2E1,4.3E1,8.4E1,4.5E1,8.7E1,5E0,6E0,6E0,2.2E1,2.1E1,7E1,1.4E1,3.7E1,8E0,5E0,8.2E1,1.8E1,4E0,1.5E1,6E0,2.6E1,4.4E1,1E1,4E0,4E0,3.3E1,4E0,4E0,9E0,7.3E1,4E0,1.4E1,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.2400521E-3,-3.636617E-2,1.23159766E-1,-5.4971147E-2,8.100857E-2,1.9564688E-1,5.1278602E-2,-9.992361E-2,-1.8375896E-2,2.800381E-2,1.22324705E-1,1.650536E-2,1.4794816E-1,7.042806E-2,-2.5878157E-3,-3.3005163E-2,-1.4763047E-1,-4.3446276E-2,8.5331455E-2,-1.3967525E-2,4.896269E-3,6.936388E-2,8.310783E-3,2.583225E-4,1.7864713E-1,8.586292E-2,-1.2433751E-3,-5.0911373E-3,3.2949608E-4,-1.4714035E-2,-5.0437413E-3,-4.2766487E-4,-4.8597143E-3,4.836669E-3,2.3766013E-4,1.0423907E-3,-2.5370282E-3,4.3165586E-3,1.2073019E-3,9.2616435E-3,2.6412285E-3,8.2868186E-4,4.8064836E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,35,-1,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3201089E0,4.6085787E-1,3.494891E-1,2.959172E-1,6.0083047E-2,2.110573E-1,7.805679E-2,2.5371742E-1,2.663056E-1,4.7518987E-2,3.6782295E-2,0E0,1.2134683E-1,4.8939005E-2,0E0,1.0983272E-1,2.72761E-1,1.7297171E-1,3.1076223E-2,1.6306661E-2,0E0,8.064065E-3,0E0,0E0,5.1259995E-2,3.2079056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,21,21,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,36,-1,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,1.2730925E0,9.520617E5,5.283708E0,1.651875E2,2.3842E4,2.008498E1,1.56221E5,5.8536225E5,2.069532E6,1.2311872E6,1.650536E-2,3.302E3,1.7021878E7,-2.5878157E-3,1.36877295E4,2.07684E5,2.142943E1,4.986835E6,2.2998267E2,4.896269E-3,3.1225768E-1,8.310783E-3,2.583225E-4,9.299407E9,1.5552E4,-1.2433751E-3,-5.0911373E-3,3.2949608E-4,-1.4714035E-2,-5.0437413E-3,-4.2766487E-4,-4.8597143E-3,4.836669E-3,2.3766013E-4,1.0423907E-3,-2.5370282E-3,4.3165586E-3,1.2073019E-3,9.2616435E-3,2.6412285E-3,8.2868186E-4,4.8064836E-3],"split_indices":[53,43,34,57,53,9,59,1,29,9,34,0,2,9,0,49,1,57,1,4,0,39,0,0,12,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.08E2,6.8E1,1.8E2,2.8E1,3.3E1,3.5E1,8E1,1E2,1.3E1,1.5E1,6E0,2.7E1,3E1,5E0,3.4E1,4.6E1,8.1E1,1.9E1,9E0,4E0,9E0,6E0,5E0,2.2E1,2.6E1,4E0,1.1E1,2.3E1,7E0,3.9E1,5.3E1,2.8E1,1.5E1,4E0,5E0,4E0,5E0,4E0,1.8E1,4E0,6E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.3484013E-4,-3.591501E-2,1.0267224E-1,-7.48736E-2,1.1144335E-2,3.864537E-2,1.7134868E-1,-1.8937495E-2,-1.23732276E-1,-7.1473983E-3,1.12259395E-1,6.948704E-3,2.0462375E-2,1.1614703E-1,2.8669775E-1,-7.037037E-2,4.3999624E-2,-1.917723E-2,-9.433054E-2,1.989276E-2,-7.688111E-2,6.9479677E-3,2.3965482E-3,5.5984515E-3,4.7229216E-3,8.152174E-4,1.3494985E-1,1.587437E-2,7.350493E-3,1.9360607E-3,-4.1455263E-3,8.52789E-3,-1.3412618E-4,-1.037372E-2,-2.5591059E-3,2.9748275E-3,-6.7167636E-4,-5.3989706E-3,3.8616697E-3,1.2316433E-3,-2.978971E-3,6.8428447E-3,2.9558314E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.840427E-1,3.656741E-1,2.9678053E-1,2.9554993E-1,1.6911322E-1,7.244958E-2,1.7682707E-1,1.7142218E-1,4.6000797E-1,1.4860699E-1,2.4721757E-2,0E0,3.9705504E-2,4.2033643E-2,1.8755376E-2,6.791268E-2,1.595447E-1,0E0,2.6154205E-1,8.883655E-2,1.5265547E-1,0E0,0E0,4.4855084E-2,0E0,0E0,9.808391E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,2.8704E4,2.174459E5,9.0649E4,5.8536225E5,1.1832633E0,1.6507233E3,1.175E3,2.4540474E8,2.1935484E0,5.8816573E9,6.948704E-3,1.1054057E0,1.8618844E0,2.9375613E-1,2E0,5.039375E-3,-1.917723E-2,9.526745E1,1.8400878E7,1.6507233E3,6.9479677E-3,2.3965482E-3,1.10018056E8,4.7229216E-3,8.152174E-4,4.7201157E-1,1.587437E-2,7.350493E-3,1.9360607E-3,-4.1455263E-3,8.52789E-3,-1.3412618E-4,-1.037372E-2,-2.5591059E-3,2.9748275E-3,-6.7167636E-4,-5.3989706E-3,3.8616697E-3,1.2316433E-3,-2.978971E-3,6.8428447E-3,2.9558314E-3],"split_indices":[43,11,29,30,29,40,4,2,5,54,5,0,35,54,39,10,39,0,53,46,4,0,0,33,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,1.98E2,6.8E1,1.08E2,9E1,3.6E1,3.2E1,5.1E1,5.7E1,7.7E1,1.3E1,4E0,3.2E1,2.3E1,9E0,2.8E1,2.3E1,4E0,5.3E1,5.6E1,2.1E1,7E0,6E0,2.8E1,4E0,4E0,1.9E1,5E0,4E0,4E0,2.4E1,5E0,1.8E1,1.1E1,4.2E1,2.4E1,3.2E1,1.7E1,4E0,2.2E1,6E0,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.4413285E-3,-1.559685E-2,1.6814202E-1,-4.9367003E-2,4.32944E-2,2.9527488E-1,4.6075262E-2,-8.067344E-2,5.025364E-4,5.6911945E-2,-8.256091E-2,3.4786907E-1,5.548531E-3,9.937335E-2,-3.1400055E-3,-8.081147E-3,-1.2765297E-1,-6.9893585E-3,1.4092635E-2,1.614593E-2,9.161419E-2,-9.620789E-5,-6.7459373E-3,2.0643875E-2,1.0422784E-2,2.253422E-3,7.3751337E-3,5.108412E-3,-1.5592917E-3,-1.8470878E-2,-4.5783105E-3,1.5986183E-3,-3.880003E-3,-5.903157E-4,4.0706466E-3,8.472347E-3,2.8426806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.268126E-1,4.794413E-1,4.6747458E-1,2.3890993E-1,1.5551664E-1,9.594965E-2,1.1031847E-1,3.195415E-1,1.2633352E-1,1.11694574E-1,4.601569E-2,3.2815814E-2,0E0,2.8961003E-2,0E0,1.2039707E-1,3.9998478E-1,0E0,1.16559744E-1,8.068293E-2,1.0732925E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,4.1677386E-1,8.982092E5,1.6408995E11,1E0,2.660842E6,2.008498E1,9.0649E4,5.7060448E1,3.4071463E2,3.8773947E0,3.0256798E0,5.548531E-3,2.2663E4,-3.1400055E-3,9.042859E4,2.4540474E8,-6.9893585E-3,6.604274E5,9.750871E6,8E0,-9.620789E-5,-6.7459373E-3,2.0643875E-2,1.0422784E-2,2.253422E-3,7.3751337E-3,5.108412E-3,-1.5592917E-3,-1.8470878E-2,-4.5783105E-3,1.5986183E-3,-3.880003E-3,-5.903157E-4,4.0706466E-3,8.472347E-3,2.8426806E-3],"split_indices":[53,43,34,32,90,30,59,30,53,53,55,55,0,9,0,29,5,0,34,9,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,2.39E2,3E1,1.52E2,8.7E1,1.4E1,1.6E1,9.3E1,5.9E1,7.9E1,8E0,1E1,4E0,1.1E1,5E0,3.7E1,5.6E1,4E0,5.5E1,3.7E1,4.2E1,4E0,4E0,4E0,6E0,7E0,4E0,6E0,3.1E1,4E0,5.2E1,4.6E1,9E0,2.7E1,1E1,9E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.0476116E-3,-4.7822103E-2,7.8535974E-2,-8.6223185E-2,-1.777505E-2,1.2548237E-1,7.918414E-3,-1.21029675E-1,2.6398191E-2,-2.1910308E-4,-1.465004E-1,8.9054085E-2,2.2009426E-1,-1.13999195E-1,6.859442E-2,-1.8277246E-1,-5.525257E-2,6.8731145E-3,-1.6323056E-2,4.1211545E-2,-4.549867E-2,-2.511922E-3,-8.547598E-3,1.6610068E-1,5.9241936E-2,1.3198484E-2,1.3646294E-1,-1.058897E-3,-1.480994E-1,-1.0688999E-2,1.9209506E-1,-4.4334037E-3,-1.5934963E-2,-8.0197083E-4,-6.5192487E-3,1.9558193E-3,-3.3649395E-3,1.0611853E-2,8.399912E-4,1.2107608E-3,-4.3027075E-3,9.233111E-3,3.5809043E-3,4.423011E-3,7.719349E-4,7.875456E-3,3.4337612E-3,-3.8125243E-3,-9.062275E-3,-4.08997E-3,1.6476169E-3,2.997242E-3,1.1757235E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,-1,-1,41,43,-1,45,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0181408E0,2.015431E-1,3.3053035E-1,3.0503148E-1,2.2595742E-1,1.8719256E-1,3.111161E-1,2.2951579E-1,1.02182224E-1,1.6883568E-1,3.306693E-2,9.3521476E-2,4.996246E-2,4.0037632E-2,2.762168E-1,3.9041877E-1,9.553551E-2,0E0,5.317177E-2,2.0010917E-1,1.5050587E-1,0E0,0E0,2.0905524E-2,5.1570624E-2,0E0,6.5074265E-3,0E0,1.5518591E-2,6.855016E-2,7.175991E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24,26,26,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,-1,-1,42,44,-1,46,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,1.83437E5,1.1936242E7,4.5510144E9,3.3457212E0,5.96E3,1.4813267E3,1.8662952E0,2.207E3,1.3E1,2.7181172E5,4.0012E4,2.9375613E-1,2.629593E0,7.12911E4,4.03E2,1.3710982E7,6.8731145E-3,2.5E1,4.5595855E-1,2.043855E6,-2.511922E-3,-8.547598E-3,3.7840486E8,1.3015172E7,1.3198484E-2,1.6E1,-1.058897E-3,7.1376E4,2.51838E6,1.6626391E0,-4.4334037E-3,-1.5934963E-2,-8.0197083E-4,-6.5192487E-3,1.9558193E-3,-3.3649395E-3,1.0611853E-2,8.399912E-4,1.2107608E-3,-4.3027075E-3,9.233111E-3,3.5809043E-3,4.423011E-3,7.719349E-4,7.875456E-3,3.4337612E-3,-3.8125243E-3,-9.062275E-3,-4.08997E-3,1.6476169E-3,2.997242E-3,1.1757235E-2],"split_indices":[53,9,46,5,54,2,56,54,0,3,34,9,39,55,29,10,46,0,3,57,9,0,0,7,33,0,3,0,9,33,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.75E2,9.9E1,7.6E1,9.9E1,5.9E1,4E1,5.8E1,1.8E1,8.8E1,1.1E1,4.4E1,1.5E1,1.3E1,2.7E1,2.9E1,2.9E1,4E0,1.4E1,4.6E1,4.2E1,4E0,7E0,1.1E1,3.3E1,7E0,8E0,4E0,9E0,1.7E1,1E1,2E1,9E0,2.1E1,8E0,7E0,7E0,4E0,4.2E1,1.7E1,2.5E1,7E0,4E0,1.7E1,1.6E1,4E0,4E0,5E0,4E0,6E0,1.1E1,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.5396263E-3,-3.8254604E-2,1.0125758E-1,-6.724777E-2,4.454634E-3,7.500814E-2,2.970856E-1,-4.14754E-2,-1.6146196E-1,-8.545177E-2,1.8321054E-2,1.1031076E-1,1.277544E-3,1.8135602E-2,6.485042E-3,-8.671476E-2,3.685543E-3,-4.2059823E-4,-2.2019261E-1,-1.4307218E-3,-6.6524264E-3,-4.0761188E-2,3.354759E-2,7.984688E-2,1.9504853E-1,-4.5407615E-3,4.8214972E-2,-1.1214835E-2,-2.6619704E-3,-3.5607882E-3,1.1218673E-3,-1.1850048E-3,-1.3490167E-2,-2.5607707E-4,-5.0666947E-3,3.920722E-3,-6.1037485E-5,8.668828E-3,2.8083802E-3,1.2345215E-2,3.55773E-3,4.1910266E-3,1.0493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,31,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0990965E0,2.4547821E-1,3.838058E-1,2.798218E-1,1.0254664E-1,1.8466383E-1,8.064836E-2,1.9236648E-1,2.2164458E-1,3.082288E-2,6.524031E-2,1.0833514E-1,1.1729429E-1,0E0,0E0,1.989724E-1,8.231751E-2,0E0,2.4730831E-1,0E0,0E0,3.597024E-2,1.0270113E-1,6.414342E-2,8.257389E-2,0E0,1.642989E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,18,18,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,32,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8045242E2,2.0283392E8,1.9297871E3,1.2869189E5,1.1670399E-1,3.3754787E1,5.1145615E3,6.846054E7,1E0,1.10980764E-1,1.549E3,7.121294E2,6.408734E-2,1.8135602E-2,6.485042E-3,1.393E3,1.9328261E2,-4.2059823E-4,1.9719312E8,-1.4307218E-3,-6.6524264E-3,3.68646E5,2.7379935E6,8E0,8.9156006E5,-4.5407615E-3,1.055304E1,-1.1214835E-2,-2.6619704E-3,-3.5607882E-3,1.1218673E-3,-1.1850048E-3,-1.3490167E-2,-2.5607707E-4,-5.0666947E-3,3.920722E-3,-6.1037485E-5,8.668828E-3,2.8083802E-3,1.2345215E-2,3.55773E-3,4.1910266E-3,1.0493E-3],"split_indices":[53,7,53,48,40,57,4,7,109,43,2,53,39,0,0,12,56,0,5,0,0,29,33,3,34,0,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.97E2,7.8E1,1.17E2,8E1,7E1,8E0,9.3E1,2.4E1,1E1,7E1,4.7E1,2.3E1,4E0,4E0,4.6E1,4.7E1,7E0,1.7E1,6E0,4E0,1.4E1,5.6E1,3.6E1,1.1E1,7E0,1.6E1,6E0,4E1,9E0,3.8E1,5E0,1.2E1,1E1,4E0,2.2E1,3.4E1,4E0,3.2E1,6E0,5E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.824203E-3,-3.8133048E-2,8.169153E-2,7.809944E-2,-4.9809888E-2,1.24406025E-1,1.2787035E-2,-3.487356E-3,8.448301E-3,-1.463914E-1,-3.7726346E-2,9.720717E-2,1.4748747E-2,-1.6493153E-2,9.907618E-3,1.9075597E-3,-1.8572357E-3,-1.0458407E-2,-1.249523E-3,-2.4958918E-2,-1.5085043E-1,1.6505769E-1,5.6812238E-2,-9.9708796E-2,2.0781456E-2,-3.9829337E-3,-2.3164117E-4,-9.888598E-3,-1.3728351E-3,9.924459E-3,4.281956E-3,5.149592E-3,5.605506E-4,-9.942665E-4,-6.5037645E-3,4.8944573E-3,-1.5640355E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0020853E-1,2.3597609E-1,2.89788E-1,1.3822201E-1,1.7653647E-1,2.9393917E-1,2.3467973E-1,1.8224575E-2,0E0,1.5756387E-1,1.984375E-1,1.4136761E-1,0E0,1.1063616E-1,0E0,0E0,0E0,0E0,0E0,1.5533528E-1,1.0191813E-1,5.2726507E-2,8.707714E-2,3.2652386E-2,1.2067756E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,1.1623532E6,1.1936242E7,1E0,2.7414852E-1,1.3821E4,1.3339948E2,2.5742075E0,8.448301E-3,4.74406E5,3.7085085E0,4.0012E4,1.4748747E-2,1.4813267E3,9.907618E-3,1.9075597E-3,-1.8572357E-3,-1.0458407E-2,-1.249523E-3,4.0835E4,5.3725034E-2,4.048974E8,8.1309015E6,2.629593E0,6.464779E6,-3.9829337E-3,-2.3164117E-4,-9.888598E-3,-1.3728351E-3,9.924459E-3,4.281956E-3,5.149592E-3,5.605506E-4,-9.942665E-4,-6.5037645E-3,4.8944573E-3,-1.5640355E-3],"split_indices":[53,46,46,8,59,2,57,54,0,1,54,9,0,56,0,0,0,0,0,9,43,7,33,55,30,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.7E2,9.8E1,1.5E1,1.55E2,6E1,3.8E1,9E0,6E0,1.6E1,1.39E2,5.4E1,6E0,3.4E1,4E0,4E0,5E0,9E0,7E0,1.26E2,1.3E1,1.9E1,3.5E1,1E1,2.4E1,3E1,9.6E1,8E0,5E0,1E1,9E0,1.5E1,2E1,4E0,6E0,9E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.1614976E-3,-2.7545594E-2,1.0395306E-1,-4.813036E-2,3.6321875E-2,1.7037725E-1,3.0353071E-2,-6.752264E-2,1.3766762E-2,1.6716229E-2,1.0787781E-1,1.5717622E-2,1.1543363E-1,-3.0862438E-3,4.6097018E-2,1.8207029E-2,-8.552788E-2,7.7099584E-2,-2.4783604E-2,4.4019846E-3,9.114382E-4,1.1527641E-3,6.981582E-3,8.797296E-3,1.576483E-1,1.1064257E-3,9.422219E-2,8.200701E-3,-1.8095442E-3,-1.2579008E-2,-3.0592056E-3,6.52015E-3,8.4501604E-4,6.1219186E-4,-4.3259826E-3,-1.4273782E-3,2.4528848E-3,2.8350858E-3,-2.1060195E-3,9.316284E-3,3.7078313E-3,-2.643826E-3,1.5161828E-3,2.573676E-3,6.9992975E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,-1,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.793192E-1,2.791493E-1,3.2237947E-1,1.9298169E-1,7.114751E-2,2.89441E-1,5.3686656E-2,1.8987453E-1,9.712945E-2,5.2282628E-2,3.6079556E-2,0E0,1.24658406E-1,0E0,6.2685974E-2,2.093215E-1,3.4358174E-1,5.4252118E-2,6.710499E-2,0E0,6.1813056E-2,0E0,0E0,2.8724056E-2,5.2643746E-2,3.169401E-2,2.2513032E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,-1,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,6.005889E-1,9.477338E5,3.165913E6,1.3289025E6,1E0,1.9241573E-1,9.338697E9,1E1,1.2415916E6,1.31903E5,1.5717622E-2,4.2403465E6,-3.0862438E-3,6.2143734E4,1.7690372E0,2.6712021E8,4.587251E6,4.99E3,4.4019846E-3,2.424E3,1.1527641E-3,6.981582E-3,5.5053414E9,2.6004646E7,2.3361564E0,8.0318604E2,8.200701E-3,-1.8095442E-3,-1.2579008E-2,-3.0592056E-3,6.52015E-3,8.4501604E-4,6.1219186E-4,-4.3259826E-3,-1.4273782E-3,2.4528848E-3,2.8350858E-3,-2.1060195E-3,9.316284E-3,3.7078313E-3,-2.643826E-3,1.5161828E-3,2.573676E-3,6.9992975E-3],"split_indices":[53,40,34,1,34,67,58,32,3,52,1,0,33,0,29,54,5,9,2,0,2,0,0,5,33,55,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.1E2,6.6E1,1.59E2,5.1E1,3.4E1,3.2E1,1.21E2,3.8E1,4.1E1,1E1,7E0,2.7E1,4E0,2.8E1,2.1E1,1E2,1.4E1,2.4E1,6E0,3.5E1,4E0,6E0,8E0,1.9E1,1.5E1,1.3E1,5E0,1.6E1,8E0,9.2E1,6E0,8E0,1.6E1,8E0,2.2E1,1.3E1,4E0,4E0,1.1E1,8E0,5E0,1E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[7.139892E-3,-4.15615E-2,6.6522375E-2,-8.5915215E-2,-1.2644255E-2,3.4027457E-2,1.4464441E-1,-1.3156319E-1,1.4241298E-2,7.036237E-2,-4.169385E-2,8.925563E-3,2.0464376E-2,1.4921679E-2,1.13071755E-1,-2.2239901E-1,-4.685537E-2,6.477722E-3,-3.728939E-2,8.316822E-3,2.6030758E-2,-2.2642918E-2,-1.3347402E-1,4.8252523E-2,-1.8698728E-2,8.356707E-2,1.8246569E-1,-4.4051693E-3,-1.8440314E-2,-2.984403E-3,1.5965993E-4,1.8926734E-3,-3.2014558E-3,3.0929297E-3,-8.158109E-4,-1.9494162E-4,-5.823697E-3,-8.095554E-3,-1.7957985E-3,6.0458074E-4,4.6168924E-3,-4.3933606E-3,6.6670135E-4,4.478592E-3,4.2749435E-4,3.5730815E-3,1.1548562E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,-1,31,-1,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.154195E-1,1.9771469E-1,3.174764E-1,2.8076205E-1,2.32008E-1,1.9330572E-1,1.7298621E-1,3.127486E-1,1.3557078E-1,1.1751662E-1,1.2112112E-1,0E0,9.340522E-2,0E0,5.231762E-2,4.1310525E-1,2.1018166E-2,0E0,4.0777985E-2,0E0,3.5393655E-2,1.141658E-1,4.116185E-2,9.004256E-2,9.302509E-2,2.455102E-2,4.5053244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,-1,32,-1,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.751724E2,7.5646E4,1.5972575E0,2.951829E9,1.8278716E0,2.8558656E4,1.8426E4,1.8196162E0,2.207E3,1.19532734E5,1.2271346E1,8.925563E-3,2.637815E1,1.4921679E-2,8.574E3,3.3656168E2,2.8270894E-1,6.477722E-3,2.9655823E-1,8.316822E-3,3.3958147E5,1E0,2.26E3,1.2119408E3,1.479938E6,9.0746456E7,3.91101E5,-4.4051693E-3,-1.8440314E-2,-2.984403E-3,1.5965993E-4,1.8926734E-3,-3.2014558E-3,3.0929297E-3,-8.158109E-4,-1.9494162E-4,-5.823697E-3,-8.095554E-3,-1.7957985E-3,6.0458074E-4,4.6168924E-3,-4.3933606E-3,6.6670135E-4,4.478592E-3,4.2749435E-4,3.5730815E-3,1.1548562E-2],"split_indices":[53,9,42,5,54,34,9,54,0,29,57,0,57,0,2,4,43,0,59,0,29,82,2,4,33,33,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.54E2,1.26E2,6E1,9.4E1,9E1,3.6E1,4.1E1,1.9E1,2.4E1,7E1,6E0,8.4E1,4E0,3.2E1,1.9E1,2.2E1,5E0,1.4E1,6E0,1.8E1,5.9E1,1.1E1,4.9E1,3.5E1,2.4E1,8E0,1.2E1,7E0,1.6E1,6E0,4E0,1E1,9E0,9E0,5.1E1,8E0,7E0,4E0,3E1,1.9E1,1E1,2.5E1,2E1,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[3.4827837E-3,-2.4586197E-2,9.832739E-2,-5.705439E-2,2.0420939E-2,3.8023643E-2,1.6771717E-1,-1.40051E-2,-1.0128554E-1,6.065021E-3,1.0902019E-2,5.6533813E-2,-1.4509801E-3,1.2002019E-1,1.3834245E-2,-8.133476E-3,-2.173574E-4,-1.4422009E-2,-7.246766E-2,-1.2929211E-3,8.41794E-2,6.0403673E-3,3.862378E-2,2.6304151E-3,5.99698E-3,-2.8406132E-3,1.9559693E-3,-5.8904476E-3,-1.316753E-3,-1.3492907E-3,1.2318445E-3,6.6149584E-3,1.6792138E-3,-1.0691644E-3,2.5838786E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,-1,25,-1,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.345201E-1,3.124221E-1,2.57244E-1,2.346923E-1,9.3923256E-2,4.6832632E-2,1.4757085E-1,1.4348817E-1,3.498969E-1,0E0,7.517236E-2,3.237403E-2,0E0,7.3592365E-3,0E0,0E0,1.6013482E-1,0E0,1.3017932E-1,5.8142055E-2,2.8820463E-2,0E0,2.7274612E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,-1,26,-1,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.249246E6,2.3893447E5,1.19331E5,1.903E3,1.1598054E8,1.9936699E3,2.670956E0,9.975593E4,6.065021E-3,1.5461028E12,1.7437479E-1,-1.4509801E-3,2.744E3,1.3834245E-2,-8.133476E-3,1.09E3,-1.4422009E-2,6.1100146E2,6.477987E-1,6.97696E5,6.0403673E-3,1.5098532E0,2.6304151E-3,5.99698E-3,-2.8406132E-3,1.9559693E-3,-5.8904476E-3,-1.316753E-3,-1.3492907E-3,1.2318445E-3,6.6149584E-3,1.6792138E-3,-1.0691644E-3,2.5838786E-3],"split_indices":[43,1,29,30,0,33,4,55,29,0,32,39,0,2,0,0,2,0,4,58,10,0,42,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.12E2,6.2E1,1.23E2,8.9E1,3.4E1,2.8E1,6.3E1,6E1,6E0,8.3E1,2.7E1,7E0,2.2E1,6E0,4E0,5.9E1,6E0,5.4E1,7.2E1,1.1E1,4E0,2.3E1,4E0,1.8E1,2.4E1,3.5E1,2.3E1,3.1E1,3.6E1,3.6E1,4E0,7E0,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.1237782E-3,-4.2186882E-2,5.8209665E-2,4.6128226E-3,-6.4099014E-2,2.4231482E-2,1.2165155E-1,8.7656066E-2,-2.9178029E-2,-1.2279467E-1,-1.2394148E-2,8.582667E-3,8.974844E-3,2.0506383E-1,7.867077E-2,8.634591E-3,3.428505E-2,-1.08173475E-1,8.403639E-3,-2.135321E-1,-6.958048E-2,2.8970655E-2,-8.6654544E-2,-8.693239E-2,2.8315768E-2,1.5134305E-2,5.6507452E-3,1.1418171E-1,2.3450175E-2,-2.1998622E-3,4.051862E-3,-2.0469893E-3,-7.5726877E-3,-5.735213E-4,3.6839452E-3,-1.6450046E-2,-2.891198E-3,-5.5834753E-3,-9.101022E-4,3.722662E-3,-1.2509127E-3,-3.9166314E-4,-7.2249738E-3,-7.26955E-4,-7.7027217E-3,2.9138033E-3,-8.705248E-4,2.333067E-3,6.905664E-3,3.333718E-3,-2.1056707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.983124E-1,1.6129625E-1,2.6435155E-1,1.4566882E-1,3.2320714E-1,2.0122367E-1,1.3802785E-1,7.353462E-2,1.1075394E-1,2.2656828E-1,1.8067291E-1,0E0,1.4343658E-1,1.0217875E-1,5.615048E-2,0E0,5.2699402E-2,3.5734683E-2,3.98362E-2,3.676166E-1,8.257474E-2,1.1288361E-1,1.1398539E-1,7.168833E-2,1.0689014E-1,0E0,0E0,3.333275E-2,4.7343872E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.751724E2,1.47887E5,1.2711891E0,1.7690372E0,5.283708E0,2.8558656E4,7.013774E5,2.6E1,1.9268292E0,7.359468E8,1.609091E1,8.582667E-3,7.88E2,2.7479675E-1,4.7291425E10,8.634591E-3,1.549E3,3.7705657E0,1.1192772E8,1.721649E8,8.5902435E1,2.0303578E0,3.58918E5,2.2912737E1,2.637815E1,1.5134305E-2,5.6507452E-3,2.744E3,1.4E1,-2.1998622E-3,4.051862E-3,-2.0469893E-3,-7.5726877E-3,-5.735213E-4,3.6839452E-3,-1.6450046E-2,-2.891198E-3,-5.5834753E-3,-9.101022E-4,3.722662E-3,-1.2509127E-3,-3.9166314E-4,-7.2249738E-3,-7.26955E-4,-7.7027217E-3,2.9138033E-3,-8.705248E-4,2.333067E-3,6.905664E-3,3.333718E-3,-2.1056707E-3],"split_indices":[53,1,40,54,57,34,34,10,54,5,57,0,0,39,32,0,2,55,5,7,53,54,30,57,57,0,0,2,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.56E2,1.23E2,5E1,1.06E2,8.1E1,4.2E1,1.4E1,3.6E1,4.9E1,5.7E1,6E0,7.5E1,1.3E1,2.9E1,4E0,1E1,1.1E1,2.5E1,1.7E1,3.2E1,3.7E1,2E1,1.2E1,6.3E1,4E0,9E0,1.7E1,1.2E1,4E0,6E0,6E0,5E0,2E1,5E0,8E0,9E0,1.5E1,1.7E1,1.9E1,1.8E1,1E1,1E1,7E0,5E0,3.6E1,2.7E1,7E0,1E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[4.4667735E-3,-1.9271143E-2,1.0929903E-1,-3.524879E-2,4.063013E-2,7.347263E-2,2.4525735E-1,-1.861916E-2,-1.0624239E-1,1.04474336E-1,-1.0531298E-2,-1.1403218E-3,9.4427E-2,3.665123E-3,1.5765578E-2,-5.9897605E-2,1.0251988E-2,-1.027324E-2,-7.200273E-2,8.400187E-3,6.112077E-2,-5.0974224E-2,2.5221322E-2,3.8990096E-4,1.089554E-1,-4.7489153E-3,1.4019388E-4,1.398148E-3,-3.0529455E-3,-1.0101859E-3,-5.719571E-3,-6.3339074E-4,4.1252663E-3,2.3877557E-4,-3.5226298E-3,-2.7353835E-4,2.1481127E-3,5.350595E-3,2.3422695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,-1,29,-1,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.740011E-1,2.1294717E-1,2.2552514E-1,2.0462638E-1,1.546192E-1,8.6739644E-2,1.3370001E-1,1.710352E-1,1.1600864E-1,5.9839442E-2,4.0326033E-2,0E0,4.153785E-2,0E0,0E0,1.60919E-1,1.3291885E-1,0E0,6.7865014E-2,0E0,3.3219278E-2,1.9040532E-2,1.0296783E-2,0E0,6.3132644E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,18,18,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,-1,30,-1,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5242589E0,3.131673E6,2.2973125E3,1.7592884E7,1.8000048E10,1E0,2.174459E5,1E0,1.868632E7,1.824614E0,1.04E2,-1.1403218E-3,6.765595E7,3.665123E-3,1.5765578E-2,1.9268292E0,2.5417683E0,-1.027324E-2,1.5E1,8.400187E-3,1.7612903E0,9.515241E6,5.4527334E9,3.8990096E-4,5.557553E-1,-4.7489153E-3,1.4019388E-4,1.398148E-3,-3.0529455E-3,-1.0101859E-3,-5.719571E-3,-6.3339074E-4,4.1252663E-3,2.3877557E-4,-3.5226298E-3,-2.7353835E-4,2.1481127E-3,5.350595E-3,2.3422695E-3],"split_indices":[43,30,4,46,5,107,29,8,46,55,8,0,5,0,0,54,54,0,3,0,58,1,12,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,2.2E2,4.9E1,1.74E2,4.6E1,4E1,9E0,1.42E2,3.2E1,2E1,2.6E1,7E0,3.3E1,4E0,5E0,5.8E1,8.4E1,6E0,2.6E1,6E0,1.4E1,1.2E1,1.4E1,5E0,2.8E1,3.4E1,2.4E1,6.7E1,1.7E1,1.4E1,1.2E1,4E0,1E1,4E0,8E0,6E0,8E0,2.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.1395643E-3,-4.015425E-2,4.6032365E-2,6.2255193E-2,-5.294301E-2,2.192633E-2,1.1681605E-1,-1.5810654E-3,7.495262E-3,-6.710322E-2,4.0586167E-3,-3.2719853E-3,6.9989964E-2,2.3386975E-1,3.6739185E-2,1.767672E-3,-1.6399372E-3,-7.9762705E-2,2.8114744E-2,-4.914371E-2,5.0356988E-2,-2.159232E-2,8.029972E-2,9.229252E-2,5.2178865E-3,5.3531444E-3,1.5289885E-2,-1.7948225E-3,6.656736E-2,-1.9578694E-3,-5.564064E-3,5.2439384E-3,-1.4823972E-3,-1.2425052E-4,-3.4368103E-3,3.6833284E-3,3.750581E-5,7.9014304E-4,-2.810015E-3,5.1996973E-3,8.5400796E-4,2.2829887E-3,5.586135E-3,-1.3178602E-3,1.7502868E-3,5.539833E-3,1.5208478E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.041914E-1,2.1022704E-1,2.0049E-1,1.15783125E-1,1.14081204E-1,1.0923006E-1,2.7260667E-1,1.7695315E-2,0E0,1.3845384E-1,7.390698E-2,9.359755E-2,4.430048E-2,1.0000074E-1,4.5509495E-2,0E0,0E0,1.4786315E-1,7.609901E-2,1.638133E-2,2.3432944E-2,7.747132E-2,2.0073742E-2,2.2505328E-2,1.1077509E-2,0E0,0E0,0E0,2.1263462E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.751724E2,1.1623532E6,7.3717834E2,1E0,4.7033575E-1,1.4429921E-1,2.45226E6,1E0,7.495262E-3,3.402546E6,2.6801108E2,4.43001E5,1.177948E3,6.0908795E6,1.04E2,1.767672E-3,-1.6399372E-3,1.609E3,1.8000048E10,1.56E2,1E1,2.142943E1,4.128E3,2.8938934E8,8.664146E-1,5.3531444E-3,1.5289885E-2,-1.7948225E-3,2.2452548E7,-1.9578694E-3,-5.564064E-3,5.2439384E-3,-1.4823972E-3,-1.2425052E-4,-3.4368103E-3,3.6833284E-3,3.750581E-5,7.9014304E-4,-2.810015E-3,5.1996973E-3,8.5400796E-4,2.2829887E-3,5.586135E-3,-1.3178602E-3,1.7502868E-3,5.539833E-3,1.5208478E-3],"split_indices":[53,46,53,8,40,39,30,107,0,1,4,29,4,46,8,0,0,2,5,10,3,57,0,12,58,0,0,0,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.57E2,1.18E2,1.7E1,1.4E2,8.9E1,2.9E1,1.1E1,6E0,1.12E2,2.8E1,5.9E1,3E1,1.1E1,1.8E1,5E0,6E0,9.9E1,1.3E1,1.3E1,1.5E1,4.9E1,1E1,2.2E1,8E0,6E0,5E0,5E0,1.3E1,5.3E1,4.6E1,5E0,8E0,5E0,8E0,9E0,6E0,2.5E1,2.4E1,6E0,4E0,1E1,1.2E1,4E0,4E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[3.8150148E-3,-4.938551E-2,3.9701525E-2,-3.167981E-2,-1.660038E-1,2.9594693E-2,1.0498834E-2,-2.0798726E-2,-7.619764E-3,-1.1222272E-2,-5.0890497E-3,-1.4824146E-2,4.1464336E-2,-4.7968432E-2,2.6761029E-2,2.470132E-2,-5.3477604E-2,8.1058875E-2,2.3741255E-2,-5.0988817E-3,-3.8083564E-4,6.1015277E-3,-8.4815756E-4,-1.6965251E-3,3.2905876E-3,-3.2445334E-3,6.442447E-5,4.347204E-3,-1.0744801E-3,2.9712205E-3,1.1834663E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.117121E-1,2.1774033E-1,2.9779938E-1,1.3488746E-1,3.343138E-2,8.128175E-2,0E0,1.16288096E-1,0E0,0E0,0E0,5.1738232E-2,8.298579E-2,1.4113271E-1,1.6013677E-1,5.1570255E-2,1.5951343E-2,5.381368E-2,7.324166E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1670399E-1,4.4326835E0,4.328683E0,1.10980764E-1,4.160985E2,3.2744165E2,1.0498834E-2,9.378309E7,-7.619764E-3,-1.1222272E-2,-5.0890497E-3,6.7897725E-1,2.0745E5,2.7307575E5,1.6581868E9,2.3780487E-1,2.8591623E2,2.8123772E0,9E0,-5.0988817E-3,-3.8083564E-4,6.1015277E-3,-8.4815756E-4,-1.6965251E-3,3.2905876E-3,-3.2445334E-3,6.442447E-5,4.347204E-3,-1.0744801E-3,2.9712205E-3,1.1834663E-4],"split_indices":[40,55,36,43,56,4,0,7,0,0,0,57,30,29,5,57,56,54,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,1.07E2,1.59E2,9.4E1,1.3E1,1.52E2,7E0,8.8E1,6E0,4E0,9E0,3.2E1,1.2E2,5.6E1,3.2E1,1.6E1,1.6E1,3.6E1,8.4E1,2.1E1,3.5E1,9E0,2.3E1,7E0,9E0,1.2E1,4E0,3.2E1,4E0,2.8E1,5.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[8.326176E-3,-1.3728847E-2,7.663917E-2,-5.2891996E-2,1.2079227E-2,1.229571E-2,6.0565077E-2,1.808629E-2,-8.502168E-2,-3.4261846E-3,7.94121E-2,2.3854816E-2,9.43276E-2,1.1140378E-1,-2.7068641E-2,-1.5338336E-2,-1.1857669E-1,-1.4790639E-2,5.627162E-2,1.6924437E-2,1.15776405E-1,5.0231875E-3,7.7800853E-3,1.20855585E-1,4.96928E-2,7.515314E-3,1.7171516E-3,1.4030898E-3,-2.5769798E-3,-3.8420546E-3,9.1217994E-4,-1.27291605E-2,-4.015926E-3,7.5304473E-4,-2.9149824E-3,6.9252397E-3,7.2442193E-4,-1.8380774E-3,2.8304295E-3,6.4681876E-3,2.5458564E-3,1.0336379E-3,-1.815763E-3,3.051114E-3,7.343203E-3,-3.9514544E-4,3.8224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1989934E-1,2.1390195E-1,1.9196239E-1,1.9310056E-1,1.3390712E-1,0E0,7.707341E-2,1.1678665E-1,1.3379833E-1,7.206776E-2,5.2576244E-2,4.3001775E-2,3.3068717E-2,2.84791E-2,3.3305746E-2,4.953561E-2,1.6894895E-1,1.3554084E-1,6.251046E-2,2.778064E-2,1.4649898E-2,0E0,2.028101E-2,3.1611323E-2,2.7903907E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.6437392E2,1.6771E4,1.433458E10,4.7321553E5,1.229571E-2,2.174459E5,1.7690372E0,1.591046E0,8.834E3,2.0497458E2,1.7278711E-1,3.9280094E5,1.5497989E5,1.25E2,6.5284515E6,1E0,2.142943E1,2.637815E1,1.647E3,2.8004E4,5.0231875E-3,2.011653E7,1.693196E1,5.2768206E5,7.515314E-3,1.7171516E-3,1.4030898E-3,-2.5769798E-3,-3.8420546E-3,9.1217994E-4,-1.27291605E-2,-4.015926E-3,7.5304473E-4,-2.9149824E-3,6.9252397E-3,7.2442193E-4,-1.8380774E-3,2.8304295E-3,6.4681876E-3,2.5458564E-3,1.0336379E-3,-1.815763E-3,3.051114E-3,7.343203E-3,-3.9514544E-4,3.8224E-3],"split_indices":[43,53,9,32,29,0,29,54,55,2,53,39,29,29,0,46,109,57,57,2,10,0,33,57,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.1E2,6.7E1,8.3E1,1.27E2,4E0,6.3E1,2.6E1,5.7E1,1.04E2,2.3E1,3.1E1,3.2E1,8E0,1.8E1,1.9E1,3.8E1,8.8E1,1.6E1,9E0,1.4E1,4E0,2.7E1,1.9E1,1.3E1,4E0,4E0,6E0,1.2E1,6E0,1.3E1,5E0,3.3E1,5.4E1,3.4E1,4E0,1.2E1,4E0,5E0,9E0,5E0,2.1E1,6E0,9E0,1E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.9457481E-3,-1.5870022E-2,1.0341738E-1,-3.2973208E-2,3.5419323E-2,2.229818E-1,2.0529639E-2,-1.7153427E-2,-8.87046E-2,-2.3598976E-2,4.887533E-2,1.714879E-2,1.15158506E-1,4.763684E-2,-3.2375765E-3,5.2892286E-3,-2.377449E-2,-9.544814E-3,-6.1676692E-2,1.1435341E-4,-1.7012536E-3,2.1261418E-2,7.354734E-2,7.3686056E-3,2.1750287E-3,-2.0995973E-4,7.177737E-2,-4.327336E-3,-4.3853765E-4,-5.965284E-4,-6.902357E-3,5.596778E-3,-1.6721123E-4,4.654438E-3,2.3026448E-3,1.4744838E-3,5.206076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.021609E-1,2.1345302E-1,3.1255588E-1,1.5894024E-1,4.9596585E-2,1.666416E-1,5.260302E-2,1.277427E-1,1.15440935E-1,4.270794E-3,3.2500826E-2,0E0,1.9876055E-2,2.0488378E-2,0E0,0E0,1.3617614E-1,0E0,1.4422742E-1,0E0,0E0,6.286068E-2,1.1799261E-2,0E0,0E0,0E0,1.4436856E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,16,16,18,18,21,21,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1.19881555E-1,2.660842E6,4.352025E0,1.8681856E8,3.0256798E0,3.6661228E7,1E1,1.9268292E0,9.141936E7,2.0699957E8,1.714879E-2,1.353E3,1.2E2,-3.2375765E-3,5.2892286E-3,9.526745E1,-9.544814E-3,7.188264E-2,1.1435341E-4,-1.7012536E-3,1.0632315E6,2.1140351E0,7.3686056E-3,2.1750287E-3,-2.0995973E-4,1.2244738E7,-4.327336E-3,-4.3853765E-4,-5.965284E-4,-6.902357E-3,5.596778E-3,-1.6721123E-4,4.654438E-3,2.3026448E-3,1.4744838E-3,5.206076E-3],"split_indices":[53,39,30,55,7,55,49,10,54,7,12,0,0,8,0,0,53,0,58,0,0,49,54,0,0,0,33,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2.41E2,3.1E1,1.81E2,6E1,1.2E1,1.9E1,1.42E2,3.9E1,1.1E1,4.9E1,4E0,8E0,1.5E1,4E0,6E0,1.36E2,6E0,3.3E1,4E0,7E0,2.4E1,2.5E1,4E0,4E0,5E0,1E1,2.2E1,1.14E2,2.2E1,1.1E1,4E0,2E1,1E1,1.5E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.218216E-3,-1.3792882E-2,1.1012109E-1,-4.2544812E-2,1.9182375E-2,2.0248948E-1,3.0720327E-2,-7.6438725E-2,-6.4982655E-3,-7.5200975E-2,2.9491797E-2,1.4391599E-2,1.0517586E-1,5.6515057E-2,-2.6688562E-3,-3.7148193E-2,-1.4567575E-1,2.7952781E-2,-5.5481967E-2,-6.8561605E-4,-6.124779E-3,3.6892947E-2,-3.912678E-2,6.4820186E-3,1.9063924E-3,-2.4190178E-4,8.506956E-2,-3.4179145E-3,-2.6542273E-6,-1.6340613E-2,-4.09398E-3,1.9403584E-3,-3.6747758E-3,2.308031E-3,-5.8365255E-3,1.9560067E-3,-2.5744622E-3,-5.271041E-3,9.341977E-4,1.4694491E-3,6.559495E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,-1,-1,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.723274E-1,2.4163407E-1,2.5047368E-1,1.6566245E-1,1.1800932E-1,1.3853973E-1,4.902925E-2,1.8501362E-1,1.1461604E-1,3.9869756E-2,5.6291975E-2,0E0,1.688005E-2,2.8667532E-2,0E0,6.327403E-2,2.653268E-1,6.453848E-2,2.1715648E-1,0E0,0E0,5.3227797E-2,5.157683E-2,0E0,0E0,0E0,2.9014602E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,-1,-1,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,2.0718746E8,2.660842E6,5.43944E5,2.2013508E-2,3.0256798E0,6.236639E3,9.0649E4,2.059925E0,1E1,1E0,1.4391599E-2,1.91016E5,1.18E2,-2.6688562E-3,1.9268292E0,2.07684E5,1.9944454E3,1.3879616E7,-6.8561605E-4,-6.124779E-3,2.179529E3,2.5952345E5,6.4820186E-3,1.9063924E-3,-2.4190178E-4,3.0635E4,-3.4179145E-3,-2.6542273E-6,-1.6340613E-2,-4.09398E-3,1.9403584E-3,-3.6747758E-3,2.308031E-3,-5.8365255E-3,1.9560067E-3,-2.5744622E-3,-5.271041E-3,9.341977E-4,1.4694491E-3,6.559495E-3],"split_indices":[53,7,30,1,39,55,4,30,54,3,90,0,10,8,0,54,1,4,46,0,0,4,29,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.87E2,2.53E2,3.4E1,1.35E2,1.18E2,1.5E1,1.9E1,6.9E1,6.6E1,1.1E1,1.07E2,6E0,9E0,1.5E1,4E0,4.5E1,2.4E1,3.9E1,2.7E1,6E0,5E0,9.7E1,1E1,5E0,4E0,5E0,1E1,2.2E1,2.3E1,4E0,2E1,3.5E1,4E0,1.1E1,1.6E1,9.2E1,5E0,4E0,6E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.8651754E-3,-1.1474913E-2,1.10247016E-1,-2.7378276E-2,3.6669753E-2,2.0101932E-1,3.35215E-2,-3.798584E-2,2.0683385E-2,-2.19896E-2,4.950134E-2,5.0855074E-3,1.3967071E-2,5.7987913E-2,-1.154525E-3,-2.807056E-2,-1.17406614E-1,-2.6747555E-2,8.5707806E-2,1.7313604E-4,-1.6268467E-3,-1.3231227E-2,6.2939115E-2,1.4546964E-2,5.735981E-3,1.5696298E-3,-2.1549403E-3,-1.1543729E-2,-1.8859623E-3,-2.8204147E-3,1.2004658E-3,5.694538E-3,-6.8974856E-4,-1.4600082E-3,5.3365034E-4,5.1810374E-3,2.227867E-3,1.9686858E-3,-7.631219E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0898725E-1,1.939508E-1,2.0916265E-1,9.765093E-2,4.8508152E-2,9.624988E-2,2.7260168E-2,1.1955109E-1,1.102384E-1,4.2565214E-3,4.4668034E-2,0E0,0E0,3.579667E-2,0E0,1.646347E-1,1.6298556E-1,3.9878674E-2,5.920954E-2,0E0,0E0,4.9546E-3,2.5582701E-2,8.610905E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1.19881555E-1,2.5304102E6,5.1428344E5,1.8681856E8,9.8999884E2,1.1598054E8,4.3399894E5,2.0139322E2,9.141936E7,2.29E2,5.0855074E-3,1.3967071E-2,3.0635E4,-1.154525E-3,1.7816157E0,3.559166E7,5.577E3,4.986835E6,1.7313604E-4,-1.6268467E-3,1.3026711E6,5.1E2,4.9237654E-1,5.735981E-3,1.5696298E-3,-2.1549403E-3,-1.1543729E-2,-1.8859623E-3,-2.8204147E-3,1.2004658E-3,5.694538E-3,-6.8974856E-4,-1.4600082E-3,5.3365034E-4,5.1810374E-3,2.227867E-3,1.9686858E-3,-7.631219E-4],"split_indices":[53,39,48,29,7,53,33,29,53,7,0,0,0,9,0,54,7,10,1,0,0,34,10,35,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,2.51E2,3E1,1.89E2,6.2E1,1.3E1,1.7E1,1.55E2,3.4E1,1.1E1,5.1E1,8E0,5E0,1.2E1,5E0,1.39E2,1.6E1,2E1,1.4E1,4E0,7E0,9E0,4.2E1,8E0,4E0,3.2E1,1.07E2,5E0,1.1E1,1.2E1,8E0,1E1,4E0,5E0,4E0,8E0,3.4E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.1548715E-3,-2.0165397E-2,7.966207E-2,-3.4287713E-2,2.5246624E-2,2.2602713E-2,1.24136634E-1,-4.81905E-2,2.4468936E-2,7.3706284E-2,-1.4362658E-2,4.431256E-2,-1.6782688E-3,8.752128E-2,1.0437987E-2,-3.7425354E-2,-1.4009675E-1,-1.579166E-2,8.762306E-2,1.20676056E-1,7.226004E-3,-5.867639E-2,1.843963E-2,-1.1138735E-4,5.783733E-2,4.816184E-3,1.9526527E-3,-6.939586E-3,-1.2545121E-3,-2.7803206E-3,-8.855074E-3,-1.6763937E-3,2.6460667E-3,1.9127405E-3,4.764628E-3,7.920868E-3,2.7923342E-3,-2.946998E-3,2.6278016E-3,-4.900761E-4,-4.5270384E-3,1.5230638E-3,-1.2082505E-3,1.1097655E-3,4.05748E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.194997E-1,1.4259803E-1,1.2915409E-1,1.3923207E-1,1.02789596E-1,3.238469E-2,8.9075476E-2,1.3030246E-1,8.529026E-2,7.406953E-2,4.4689726E-2,1.1516605E-2,0E0,1.3289794E-2,0E0,1.3888209E-1,4.6673894E-2,3.3815928E-2,5.203329E-3,3.27307E-2,4.271382E-2,2.3711339E-2,1.2643613E-2,0E0,1.2180869E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,3.131673E6,2.3893447E5,5.1428344E5,1.775966E10,2.011653E7,6.0908795E6,1E0,5.403929E2,2.035952E0,1.08E2,2.300289E8,-1.6782688E-3,1.3233988E6,1.0437987E-2,9.501452E-5,5.743459E-3,4.1966243E0,2.9635E4,8.833194E5,4.876025E2,4.1380672E0,1.52831E1,-1.1138735E-4,1E0,4.816184E-3,1.9526527E-3,-6.939586E-3,-1.2545121E-3,-2.7803206E-3,-8.855074E-3,-1.6763937E-3,2.6460667E-3,1.9127405E-3,4.764628E-3,7.920868E-3,2.7923342E-3,-2.946998E-3,2.6278016E-3,-4.900761E-4,-4.5270384E-3,1.5230638E-3,-1.2082505E-3,1.1097655E-3,4.05748E-3],"split_indices":[43,30,29,29,5,33,46,90,56,54,8,7,0,48,0,42,58,59,9,34,53,58,59,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.2E2,5.1E1,1.68E2,5.2E1,2.3E1,2.8E1,1.36E2,3.2E1,2.3E1,2.9E1,1.7E1,6E0,2.2E1,6E0,1.23E2,1.3E1,2E1,1.2E1,1.3E1,1E1,1.2E1,1.7E1,4E0,1.3E1,1.5E1,7E0,9E0,1.14E2,6E0,7E0,1.6E1,4E0,4E0,8E0,6E0,7E0,4E0,6E0,6E0,6E0,1.3E1,4E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.6610433E-4,-3.808482E-2,3.598843E-2,-3.0279525E-2,-1.0889985E-2,8.53482E-3,7.749504E-2,-4.035627E-2,7.385095E-2,-9.54287E-2,2.4419786E-2,5.895432E-2,1.2073612E-2,-5.6782678E-2,1.6230818E-2,6.2793777E-3,-5.2558875E-4,-7.5898734E-3,-1.8467861E-3,3.248072E-2,-3.578536E-3,1.0029088E-1,6.992278E-3,4.10007E-3,-2.9583087E-3,4.703174E-3,-3.171778E-4,2.8124666E-3,1.3800942E-4,2.9666338E-3,6.8465746E-3,1.765536E-3,-3.6424124E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.863362E-1,2.0348918E-1,1.657065E-1,1.4148745E-1,0E0,1.4931002E-1,1.8320316E-1,1.1339386E-1,6.554325E-2,3.9469764E-2,6.674832E-2,1.15287066E-1,0E0,1.0813868E-1,5.63251E-2,0E0,0E0,0E0,0E0,6.1383374E-2,0E0,4.2497754E-2,7.0528306E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,19,19,21,21,22,22],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,1.2816449E7,-1.0889985E-2,1.0584392E8,3.1746092E-1,9.199832E5,2.459286E7,2.265E3,3.8016708E7,6.6E1,1.2073612E-2,3.8757736E-7,2.3749721E-1,6.2793777E-3,-5.2558875E-4,-7.5898734E-3,-1.8467861E-3,3.1471178E8,-3.578536E-3,1.55992E5,2.1E1,4.10007E-3,-2.9583087E-3,4.703174E-3,-3.171778E-4,2.8124666E-3,1.3800942E-4,2.9666338E-3,6.8465746E-3,1.765536E-3,-3.6424124E-3],"split_indices":[53,53,29,9,0,7,35,34,9,2,46,8,0,39,40,0,0,0,0,7,0,30,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.35E2,1.45E2,1.31E2,4E0,8.8E1,5.7E1,1.2E2,1.1E1,1.1E1,7.7E1,5.3E1,4E0,9.3E1,2.7E1,6E0,5E0,4E0,7E0,7.2E1,5E0,2.9E1,2.4E1,4E0,8.9E1,5E0,2.2E1,3.6E1,3.6E1,1.8E1,1.1E1,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.6536114E-4,-2.7709948E-2,4.9623653E-2,-4.2046502E-2,2.3574479E-2,3.6762558E-2,8.73471E-3,-3.5253476E-2,-9.680402E-3,-2.7711282E-2,4.044631E-2,6.0875498E-2,-2.2648696E-2,-4.6331894E-2,2.7994413E-2,-2.205099E-3,9.440538E-5,4.844452E-3,1.0760032E-2,4.3209128E-2,1.282172E-1,-5.2907118E-3,9.184552E-3,-3.8804938E-3,-1.1328061E-3,6.738825E-3,-4.9476716E-4,-3.1498203E-4,2.5738634E-3,3.5285202E-3,2.5535826E-4,8.675505E-3,1.2230461E-3,-6.533607E-4,3.9974656E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.725685E-1,1.267176E-1,1.6959575E-1,1.4570291E-1,3.4001634E-2,1.3174352E-1,0E0,9.238629E-2,0E0,6.1049894E-3,5.463185E-2,7.1789294E-2,8.035631E-2,8.945632E-2,9.328832E-2,0E0,0E0,0E0,1.7068826E-2,6.630439E-2,7.4564695E-2,0E0,3.9343953E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,6.862291E-1,2.0704236E3,4.1150245E-1,2.0358758E8,1.8018788E7,8.73471E-3,3.131673E6,-9.680402E-3,1.7427321E-1,2.2514071E0,7.3717834E2,1.3548306E3,1.8662952E0,4.150475E6,-2.205099E-3,9.440538E-5,4.844452E-3,2.0813715E8,5.7E1,7.643508E5,-5.2907118E-3,7.12911E4,-3.8804938E-3,-1.1328061E-3,6.738825E-3,-4.9476716E-4,-3.1498203E-4,2.5738634E-3,3.5285202E-3,2.5535826E-4,8.675505E-3,1.2230461E-3,-6.533607E-4,3.9974656E-3],"split_indices":[53,42,53,43,7,46,0,30,0,39,59,53,4,54,30,0,0,0,12,8,34,0,29,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,1.7E2,9.7E1,1.33E2,3.7E1,9E1,7E0,1.29E2,4E0,9E0,2.8E1,6.4E1,2.6E1,1.1E2,1.9E1,5E0,4E0,8E0,2E1,5.2E1,1.2E1,6E0,2E1,3.9E1,7.1E1,4E0,1.5E1,1.5E1,5E0,2.7E1,2.5E1,7E0,5E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0404793E-3,-1.21584395E-2,8.368001E-2,-1.2256621E-1,-6.2683444E-3,1.6521272E-1,2.3323348E-2,-7.986367E-3,-7.651994E-4,8.045934E-3,-1.0299069E-2,1.4868271E-2,8.572735E-2,4.5196574E-2,-2.967787E-3,-3.154716E-3,-1.0253196E-1,5.6318087E-3,4.618525E-4,-2.9979597E-4,6.956272E-2,-6.936959E-3,2.1929083E-5,-6.49639E-3,4.851598E-4,1.4466698E-3,5.010379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,-1,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0097434E-1,1.5400605E-1,1.8314627E-1,6.1823517E-2,1.6743213E-1,1.7290154E-1,4.707555E-2,0E0,0E0,0E0,1.4662114E-1,0E0,3.2013297E-2,2.439155E-2,0E0,1.1148192E-1,7.1260616E-2,0E0,0E0,0E0,1.6047932E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,15,15,16,16,20,20],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,-1,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[6.662007E2,1.0742589E-1,2.660842E6,1.6E1,1.55E2,1.431E4,3.6661228E7,-7.986367E-3,-7.651994E-4,8.045934E-3,3.7085085E0,1.4868271E-2,2.4840117E0,1.2E2,-2.967787E-3,4.9759598E1,1.00058334E2,5.6318087E-3,4.618525E-4,-2.9979597E-4,5.4582E4,-6.936959E-3,2.1929083E-5,-6.49639E-3,4.851598E-4,1.4466698E-3,5.010379E-3],"split_indices":[53,59,30,3,11,9,49,0,0,0,54,0,54,8,0,49,55,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.37E2,3.7E1,1.1E1,2.26E2,1.5E1,2.2E1,7E0,4E0,4E0,2.22E2,4E0,1.1E1,1.8E1,4E0,2.07E2,1.5E1,7E0,4E0,6E0,1.2E1,4E0,2.03E2,1.1E1,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[8.2585146E-4,-1.786127E-2,4.580317E-2,5.207858E-2,-2.3978915E-2,3.4938917E-2,9.390168E-3,7.815496E-3,-1.2913048E-3,-3.4782536E-2,2.1772722E-2,7.5003E-3,8.0095924E-2,2.20724E-3,-1.9938396E-3,-1.3508888E-1,-2.798877E-2,3.6182955E-2,-3.3115149E-3,-6.0110893E-2,2.421559E-2,1.04646765E-1,-2.7015724E-3,-1.8307549E-3,-9.364587E-3,-1.7037713E-3,1.7664E-3,3.906549E-3,5.7170243E-4,-4.3278025E-3,-3.4177504E-4,1.6358009E-3,-2.858475E-3,3.011131E-3,8.281355E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3195468E-1,8.4682904E-2,1.3204584E-1,1.00701466E-1,8.97283E-2,9.480579E-2,0E0,0E0,2.689753E-2,9.571184E-2,4.985981E-2,5.683252E-2,1.049435E-1,0E0,0E0,5.0515234E-2,8.398506E-2,3.4800693E-2,0E0,1.715694E-2,4.1260768E-2,6.204021E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8045242E2,1.1623532E6,1.5266737E2,1.8462508E0,3.165913E6,2.8041378E5,9.390168E-3,7.815496E-3,2.5742075E0,1E0,1.980818E3,1.9241573E-1,2.8123772E0,2.20724E-3,-1.9938396E-3,1.418E3,3E1,1.0613555E8,-3.3115149E-3,3.07619E5,2.5497008E7,1.3000824E3,-2.7015724E-3,-1.8307549E-3,-9.364587E-3,-1.7037713E-3,1.7664E-3,3.906549E-3,5.7170243E-4,-4.3278025E-3,-3.4177504E-4,1.6358009E-3,-2.858475E-3,3.011131E-3,8.281355E-3],"split_indices":[53,46,57,54,1,29,0,0,54,67,4,58,54,0,0,2,3,7,0,30,44,4,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.94E2,8E1,1.5E1,1.79E2,7.6E1,4E0,4E0,1.1E1,1.45E2,3.4E1,4.8E1,2.8E1,5E0,6E0,8E0,1.37E2,3E1,4E0,9E0,3.9E1,2.4E1,4E0,4E0,4E0,1.21E2,1.6E1,9E0,2.1E1,5E0,4E0,3.5E1,4E0,1.7E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.7523934E-3,-8.844985E-3,8.7980606E-2,-2.5938668E-3,-1.1828045E-1,1.7884427E-1,1.45616E-2,-7.862449E-2,7.531358E-3,-2.5686484E-3,-8.374275E-3,1.4005396E-2,1.1171169E-1,2.5816143E-2,-1.2254365E-3,-1.2963317E-1,-4.060858E-3,6.916115E-3,3.1695364E-3,6.149751E-3,2.5043758E-3,-1.6158835E-4,4.2131096E-2,-9.2741445E-4,-7.571509E-3,1.3923616E-3,-2.6764972E-3,-6.1821137E-3,3.0827048E-4,8.20593E-4,2.7356339E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,25,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9362926E-1,1.6455984E-1,2.3755604E-1,1.7688255E-1,4.152465E-2,1.0065842E-1,1.0419304E-2,1.0216522E-1,1.258642E-1,0E0,0E0,0E0,7.76951E-3,8.281499E-3,0E0,5.768305E-2,2.4015749E-2,0E0,9.685546E-2,0E0,0E0,0E0,3.563121E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,16,16,18,18,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,26,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.662007E2,3.7085085E0,2.660842E6,3.3617156E-4,5.743459E-3,1.431E4,6.236639E3,3.3E1,1E1,-2.5686484E-3,-8.374275E-3,1.4005396E-2,8.982092E5,1.18E2,-1.2254365E-3,1.3049372E0,1.7E1,6.916115E-3,3.4967444E7,6.149751E-3,2.5043758E-3,-1.6158835E-4,6.708E3,-9.2741445E-4,-7.571509E-3,1.3923616E-3,-2.6764972E-3,-6.1821137E-3,3.0827048E-4,8.20593E-4,2.7356339E-3],"split_indices":[53,54,30,40,58,9,4,3,10,0,0,0,34,8,0,54,8,0,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.4E2,3.5E1,2.28E2,1.2E1,1.5E1,2E1,2.6E1,2.02E2,7E0,5E0,4E0,1.1E1,1.6E1,4E0,1.5E1,1.1E1,5E0,1.97E2,7E0,4E0,6E0,1E1,4E0,1.1E1,7E0,4E0,4E0,1.93E2,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.7240166E-5,-1.9982655E-2,4.4589706E-2,-5.160299E-2,4.818867E-3,1.1611166E-1,2.3814652E-2,-9.762541E-3,-3.9383415E-2,-7.945473E-2,1.7538918E-2,1.3856167E-1,9.6913136E-4,5.6648673E-3,5.1485218E-2,-2.6163757E-2,-8.249335E-3,6.260417E-4,-7.4851285E-3,5.9114406E-3,7.708621E-3,3.5620057E-3,1.0481725E-2,2.4320425E-2,-3.2023907E-2,6.537517E-2,-1.1732917E-3,3.524182E-3,-1.7278314E-3,1.8305408E-3,-3.8862522E-4,-1.7241209E-3,2.0602222E-3,4.3713042E-4,-2.209261E-3,-1.1824545E-4,3.8393724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,25,27,-1,-1,-1,-1,29,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5880212E-1,1.5709513E-1,1.3096337E-1,1.6417058E-1,1.22525536E-1,3.97664E-2,3.5365593E-2,0E0,1.4869423E-1,1.1822859E-1,1.06895305E-1,6.962633E-2,0E0,3.171546E-2,3.1600602E-2,9.2747375E-2,0E0,0E0,0E0,0E0,4.7954287E-2,0E0,0E0,3.95278E-2,1.0395478E-2,2.9590227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,14,14,15,15,20,20,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,26,28,-1,-1,-1,-1,30,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.821357E-1,8.461558E10,7.013774E5,2.3361564E0,9.277164E1,2.65826E6,2.216135E5,-9.762541E-3,2.781E3,1.3049372E0,2.9655823E-1,2.1061536E3,9.6913136E-4,1.243356E5,9.927293E8,1E1,-8.249335E-3,6.260417E-4,-7.4851285E-3,5.9114406E-3,2.3E1,3.5620057E-3,1.0481725E-2,3.9579918E4,6.788778E1,1.7427321E-1,-1.1732917E-3,3.524182E-3,-1.7278314E-3,1.8305408E-3,-3.8862522E-4,-1.7241209E-3,2.0602222E-3,4.3713042E-4,-2.209261E-3,-1.1824545E-4,3.8393724E-3],"split_indices":[42,32,34,55,53,48,29,0,2,54,59,4,0,29,5,10,0,0,0,0,8,0,0,29,53,39,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.88E2,1.99E2,8.9E1,8.7E1,1.12E2,1.9E1,7E1,5E0,8.2E1,1.4E1,9.8E1,1.5E1,4E0,4.3E1,2.7E1,7.6E1,6E0,7E0,7E0,7E0,9.1E1,1E1,5E0,2.9E1,1.4E1,2.3E1,4E0,7E0,6.9E1,3E1,6.1E1,7E0,2.2E1,4E0,1E1,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.6302058E-3,-6.9470066E-3,8.675293E-2,-2.7212687E-2,1.7960887E-2,1.6635017E-1,1.168328E-2,3.5056085E-3,-3.3179972E-2,-6.0463727E-2,2.6005743E-2,1.3030447E-2,7.0088275E-2,-2.0607125E-3,4.1407395E-2,-5.7126988E-2,-1.7705924E-3,-4.5004957E-3,1.763036E-4,5.1429313E-2,4.517171E-3,4.8098513E-3,9.986376E-4,-4.9396086E-4,3.348664E-3,-7.030902E-4,-7.643865E-3,1.5486891E-3,-3.337862E-3,2.9642184E-3,-3.3536658E-3,1.2084269E-3,-1.2596821E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,-1,23,25,27,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2860478E-1,1.292065E-1,1.7134126E-1,8.9486755E-2,7.41276E-2,1.3739312E-1,2.8696064E-2,0E0,1.0068843E-1,2.6743744E-2,5.7130896E-2,0E0,1.2709606E-2,0E0,1.8744962E-2,3.4386873E-1,1.5019107E-1,0E0,0E0,8.195232E-2,4.089332E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,14,14,15,15,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,-1,24,26,28,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,2.0718746E8,2.660842E6,3.299865E1,1.1670399E-1,3.0256798E0,1.08E2,3.5056085E-3,6.608892E2,5.896198E6,2.0817778E0,1.3030447E-2,1.353E3,-2.0607125E-3,3.8029985E6,1.647E3,2.2733334E1,-4.5004957E-3,1.763036E-4,1.9647126E7,1.6091988E7,4.8098513E-3,9.986376E-4,-4.9396086E-4,3.348664E-3,-7.030902E-4,-7.643865E-3,1.5486891E-3,-3.337862E-3,2.9642184E-3,-3.3536658E-3,1.2084269E-3,-1.2596821E-3],"split_indices":[53,7,30,53,40,55,8,0,4,33,54,0,0,0,33,2,57,0,0,46,33,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.54E2,2.8E1,1.4E2,1.14E2,1.3E1,1.5E1,7E0,1.33E2,1E1,1.04E2,5E0,8E0,5E0,1E1,7.5E1,5.8E1,6E0,4E0,4.7E1,5.7E1,4E0,4E0,4E0,6E0,5.5E1,2E1,3.9E1,1.9E1,4.3E1,4E0,3.4E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.2992085E-3,-6.186166E-3,7.541433E-3,-2.1961667E-2,2.9620828E-2,-9.434576E-3,-6.528359E-2,8.000968E-2,1.8237442E-2,-6.9916554E-2,1.2006471E-3,-7.2051836E-3,-4.2570077E-2,1.1788558E-3,4.487425E-3,2.5036087E-2,-2.6391195E-3,-5.669194E-3,2.880813E-4,6.6165924E-3,-2.1408034E-4,-8.3313236E-5,-5.5499566E-3,7.872919E-4,4.0155845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2450328E-1,1.5316631E-1,0E0,1.0141124E-1,4.614927E-2,9.464013E-2,8.035049E-2,1.1749946E-2,3.6880054E-2,8.995235E-2,1.0571988E-1,0E0,1.1102041E-1,0E0,0E0,3.0092005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,1.19881555E-1,7.541433E-3,4.352025E0,6.743105E5,2.7414852E-1,1.9268292E0,4.2403465E6,2.008498E1,6.846054E7,6.7473096E9,-7.2051836E-3,1E0,1.1788558E-3,4.487425E-3,1.5513661E6,-2.6391195E-3,-5.669194E-3,2.880813E-4,6.6165924E-3,-2.1408034E-4,-8.3313236E-5,-5.5499566E-3,7.872919E-4,4.0155845E-3],"split_indices":[53,39,0,55,34,59,54,33,59,7,32,0,90,0,0,34,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.69E2,7E0,1.87E2,8.2E1,1.46E2,4.1E1,1.4E1,6.8E1,2.1E1,1.25E2,7E0,3.4E1,4E0,1E1,6.3E1,5E0,1.2E1,9E0,4E0,1.21E2,2.3E1,1.1E1,5.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.6445782E-4,-1.0475103E-2,8.043979E-2,-2.9651217E-2,1.1412919E-2,-7.1429117E-3,1.377257E-1,-9.233023E-3,-8.237862E-2,-3.569281E-3,1.7777296E-2,1.5757794E-2,-2.371839E-3,1.2649076E-2,8.630252E-2,-4.0853035E-2,1.8916083E-2,-1.6305467E-1,-2.6182115E-2,5.7349976E-2,5.4635657E-3,1.4126676E-3,-2.441901E-4,7.598621E-4,1.1652876E-1,-6.970347E-3,-9.432436E-4,4.867763E-3,1.4101231E-4,-2.2441898E-3,-1.50277065E-2,2.5870441E-3,-2.1400328E-3,-5.210417E-4,4.074509E-3,2.5014165E-3,-3.477042E-4,6.5692505E-3,3.266779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,25,27,29,31,33,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.408431E-1,1.04850486E-1,1.6655317E-1,1.4242697E-1,6.760335E-2,1.5038142E-2,1.1719847E-1,8.7110065E-2,1.6347823E-1,0E0,5.328826E-2,3.3061649E-3,0E0,0E0,3.1832755E-2,1.0045742E-1,7.1240425E-2,2.641704E-1,4.1069098E-2,5.661004E-2,5.44727E-2,0E0,0E0,0E0,2.9876977E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,26,28,30,32,34,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,2.0718746E8,7.6411234E4,1.2869189E5,1.01861015E-1,6.5414023E-1,1.431E4,6.846054E7,6.4138586E2,-3.569281E-3,4.75E2,4.9332496E4,-2.371839E-3,1.2649076E-2,2.177322E-1,1.0742589E-1,3.9449276E1,1.779E3,6.7000134E8,1.95E2,3.3001E4,1.4126676E-3,-2.441901E-4,7.598621E-4,2.7479675E-1,-6.970347E-3,-9.432436E-4,4.867763E-3,1.4101231E-4,-2.2441898E-3,-1.50277065E-2,2.5870441E-3,-2.1400328E-3,-5.210417E-4,4.074509E-3,2.5014165E-3,-3.477042E-4,6.5692505E-3,3.266779E-3],"split_indices":[53,7,29,48,40,35,9,7,4,0,0,29,0,0,58,59,53,2,5,0,9,0,0,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.48E2,3.2E1,1.32E2,1.16E2,1.3E1,1.9E1,9.6E1,3.6E1,7E0,1.09E2,9E0,4E0,4E0,1.5E1,4.5E1,5.1E1,1.4E1,2.2E1,2.5E1,8.4E1,5E0,4E0,5E0,1E1,6E0,3.9E1,7E0,4.4E1,9E0,5E0,4E0,1.8E1,8E0,1.7E1,1.7E1,6.7E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4997874E-3,-3.3806123E-2,2.9913362E-2,-2.6885899E-2,-9.847766E-3,9.612255E-3,7.117494E-2,-1.8126047E-3,-5.1563814E-2,6.27013E-3,2.703877E-3,5.3347144E-2,9.680545E-3,7.5899705E-2,-1.3439241E-2,-1.16839865E-2,-2.7902044E-2,-8.9354426E-2,1.6142849E-2,8.112999E-2,-9.998025E-3,5.5239773E-3,7.661275E-4,-9.591232E-5,-6.3508647E-3,-5.6021623E-3,-7.220093E-4,-8.744022E-4,-6.303036E-3,1.04238E-3,-3.2196818E-3,2.0278431E-3,5.3075883E-3,-2.4809325E-3,2.6473196E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,23,-1,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8618237E-1,1.6747497E-1,1.191006E-1,8.3438665E-2,0E0,8.44502E-2,1.0532269E-1,6.2460262E-2,3.182742E-1,0E0,1.1649915E-1,7.648722E-2,0E0,2.1512829E-2,8.4542274E-2,0E0,6.9141015E-2,3.754548E-2,4.7441937E-2,3.2916382E-2,4.4569988E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,24,-1,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,3.7786137E10,-9.847766E-3,9.037E3,3.1746092E-1,1.7145833E0,7.359468E8,6.27013E-3,1.0584392E8,1.07E2,9.680545E-3,4.4E1,1.4088401E7,-1.16839865E-2,7E0,1.945711E1,3.605082E7,6.0908795E6,4.43001E5,5.5239773E-3,7.661275E-4,-9.591232E-5,-6.3508647E-3,-5.6021623E-3,-7.220093E-4,-8.744022E-4,-6.303036E-3,1.04238E-3,-3.2196818E-3,2.0278431E-3,5.3075883E-3,-2.4809325E-3,2.6473196E-3],"split_indices":[53,53,29,32,0,9,35,54,5,0,7,8,0,10,46,0,3,57,46,46,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.38E2,1.42E2,1.34E2,4E0,9.6E1,4.6E1,6.7E1,6.7E1,4E0,9.2E1,4.2E1,4E0,8E0,5.9E1,6E0,6.1E1,1.1E1,8.1E1,2.9E1,1.3E1,4E0,4E0,5.5E1,4E0,6E0,5.5E1,5E0,6E0,7.6E1,5E0,1.5E1,1.4E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.8596204E-4,-1.6340856E-2,4.859123E-2,-7.195226E-2,-1.0008341E-2,8.836621E-2,3.7278784E-3,1.6763784E-3,-9.959118E-2,-2.233174E-3,-7.1362846E-2,9.953862E-3,5.5593874E-2,-2.9382505E-3,1.5245181E-2,-1.4627877E-1,-1.3251902E-3,6.941909E-2,-7.2391727E-3,-1.1384955E-1,-3.645952E-2,-3.02262E-4,7.1407154E-2,3.0594535E-2,-1.7203838E-3,-3.700943E-3,-9.028527E-3,-2.9900225E-4,6.158571E-3,-7.8040996E-4,1.6670662E-3,-6.8524242E-3,-2.5827326E-3,-6.0510065E-4,-2.8302413E-3,4.1196956E-3,6.784765E-4,1.9124127E-4,2.0451157E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,-1,23,25,-1,27,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1580678E-1,7.217702E-2,1.21012434E-1,6.652178E-2,8.90474E-2,1.3879418E-1,2.6645167E-2,0E0,5.198954E-2,6.040396E-2,2.6990578E-2,0E0,2.995687E-2,0E0,2.456222E-2,1.7065033E-2,0E0,5.459852E-2,6.6855766E-2,9.834647E-3,6.773902E-3,0E0,2.3260564E-2,8.22293E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,14,14,15,15,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,-1,24,26,-1,28,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,2.598304E-1,9.520617E5,1.1078758E0,2.9665058E0,1E0,8.715196E5,1.6763784E-3,6.846054E7,1.1623532E6,5.3725034E-2,9.953862E-3,1.5457413E-1,-2.9382505E-3,2.008498E1,2.3101475E5,-1.3251902E-3,1E0,5.0090684E5,5.8835944E2,7.321865E7,-3.02262E-4,3.21879E6,5.1227652E4,-1.7203838E-3,-3.700943E-3,-9.028527E-3,-2.9900225E-4,6.158571E-3,-7.8040996E-4,1.6670662E-3,-6.8524242E-3,-2.5827326E-3,-6.0510065E-4,-2.8302413E-3,4.1196956E-3,6.784765E-4,1.9124127E-4,2.0451157E-3],"split_indices":[53,59,34,55,54,67,48,0,7,46,43,0,58,0,59,29,0,8,29,4,12,0,30,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.06E2,6.7E1,2E1,1.86E2,3.5E1,3.2E1,4E0,1.6E1,1.66E2,2E1,6E0,2.9E1,4E0,2.8E1,9E0,7E0,1E1,1.56E2,8E0,1.2E1,6E0,2.3E1,2.2E1,6E0,5E0,4E0,5E0,5E0,1.28E2,2.8E1,4E0,4E0,7E0,5E0,1.7E1,6E0,8E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.3371255E-5,-3.1028034E-2,2.0835297E-2,-1.6813101E-2,-1.4926928E-1,1.5616176E-2,1.1188454E-1,-8.193008E-2,-2.8258236E-3,-1.1570735E-2,-3.077064E-3,8.104795E-3,5.4305855E-2,1.3984346E-3,7.873873E-3,-7.777474E-3,-1.1213035E-2,1.2475116E-2,-7.506189E-2,3.586205E-3,3.7141119E-3,1.3836331E-2,9.371105E-2,-2.1831472E-3,1.9206341E-3,1.2273088E-3,-3.0987037E-3,-5.129966E-3,-8.0989045E-4,4.1300248E-4,-2.222923E-3,-3.5811187E-4,2.4999338E-3,6.544114E-3,2.5965169E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8215322E-1,1.8741974E-1,7.856108E-2,9.267102E-2,8.414957E-2,4.6348542E-2,3.8186125E-2,1.0884194E-1,9.488517E-2,0E0,0E0,4.1382607E-2,4.0165834E-2,0E0,0E0,0E0,2.3056807E-2,8.212288E-2,2.9384568E-2,0E0,3.5688568E-2,1.2666093E-2,1.5475281E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,16,16,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1670399E-1,1.4031231E-2,4.328683E0,9.421523E-5,6.132018E2,4.7321553E5,2.174459E5,8.877883E1,2.142943E1,-1.1570735E-2,-3.077064E-3,1.549133E0,6.46838E5,1.3984346E-3,7.873873E-3,-7.777474E-3,7.87256E5,4.4326835E0,7.88E2,3.586205E-3,3.7930825E5,7.8744044E5,6.6E1,-2.1831472E-3,1.9206341E-3,1.2273088E-3,-3.0987037E-3,-5.129966E-3,-8.0989045E-4,4.1300248E-4,-2.222923E-3,-3.5811187E-4,2.4999338E-3,6.544114E-3,2.5965169E-3],"split_indices":[40,39,36,43,4,29,29,53,57,0,0,54,1,0,0,0,7,55,0,0,29,48,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.12E2,1.68E2,1.01E2,1.1E1,1.6E2,8E0,1.7E1,8.4E1,4E0,7E0,1.35E2,2.5E1,4E0,4E0,7E0,1E1,7E1,1.4E1,7E0,1.28E2,1.3E1,1.2E1,6E0,4E0,6E1,1E1,8E0,6E0,1.17E2,1.1E1,9E0,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.589759E-5,-3.067964E-3,7.5481976E-3,-1.653295E-2,2.774567E-2,-3.1164067E-2,1.1871366E-2,8.249559E-2,1.1563938E-2,-1.3065111E-2,-7.483544E-2,5.8817483E-2,-1.1343878E-2,9.8734334E-2,7.956013E-4,3.272501E-2,-1.8323408E-2,-3.858915E-3,2.545208E-4,-1.4572021E-2,-1.6552575E-3,4.0481766E-3,-7.420578E-4,1.0773154E-3,-2.4016986E-3,3.0017581E-3,7.033631E-3,-9.956771E-4,2.4039142E-3,-2.7603523E-3,-7.327606E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3697606E-1,1.1455763E-1,0E0,8.0162644E-2,7.346569E-2,9.940385E-2,7.259835E-2,1.8415138E-2,4.2341225E-2,1.2017048E-1,3.3630395E-1,4.9403347E-2,6.504045E-2,1.7222047E-2,0E0,4.2436253E-2,1.9408267E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5266737E2,1.19881555E-1,7.5481976E-3,7.629E3,7.013774E5,1.2869189E5,1E1,3.21879E6,3.133129E8,1E0,9.975593E4,4.397396E7,5.0341E4,6.886E3,7.956013E-4,1.8681856E8,3.5300705E2,-3.858915E-3,2.545208E-4,-1.4572021E-2,-1.6552575E-3,4.0481766E-3,-7.420578E-4,1.0773154E-3,-2.4016986E-3,3.0017581E-3,7.033631E-3,-9.956771E-4,2.4039142E-3,-2.7603523E-3,-7.327606E-5],"split_indices":[57,39,0,10,34,48,3,30,7,109,29,33,9,2,0,7,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,2.74E2,4E0,1.91E2,8.3E1,1.26E2,6.5E1,1.8E1,6.5E1,9E1,3.6E1,2.1E1,4.4E1,1.4E1,4E0,3.8E1,2.7E1,1.8E1,7.2E1,4E0,3.2E1,1.5E1,6E0,2.4E1,2E1,1E1,4E0,1E1,2.8E1,7E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.9479691E-3,-6.1025335E-3,6.570009E-2,-1.2826966E-4,-7.2709166E-2,-4.1228845E-3,1.1827347E-1,-2.1588098E-2,2.0767951E-2,-1.8375515E-4,-1.0940418E-1,1.4558834E-2,-2.21616E-3,1.1170241E-2,7.202472E-2,-6.1529605E-3,-8.375057E-2,4.6207937E-3,1.6196072E-2,-1.8788242E-3,-7.4757603E-3,-2.3791735E-4,1.353022E-3,4.0713765E-4,9.993981E-2,-2.8882546E-3,4.5449435E-4,-8.595389E-3,-1.0712073E-3,-3.0795769E-3,9.5920236E-4,5.8259866E-3,2.6097929E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5771984E-1,9.668659E-2,1.2896664E-1,1.00896515E-1,4.9995556E-2,1.3856163E-2,9.753838E-2,1.0598767E-1,3.9858744E-2,0E0,4.0608004E-2,3.6619927E-3,0E0,0E0,2.7335852E-2,8.221786E-2,1.3263261E-1,0E0,4.339181E-2,0E0,0E0,0E0,0E0,0E0,5.4834858E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,15,15,16,16,18,18,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,1E0,7.6411234E4,1.7546342E8,1.03847E5,2.5586937E1,1.431E4,9.8845E5,1E0,-1.8375515E-4,5.743459E-3,1.5552E4,-2.21616E-3,1.1170241E-2,2.177322E-1,1E0,6E0,4.6207937E-3,1.32461E7,-1.8788242E-3,-7.4757603E-3,-2.3791735E-4,1.353022E-3,4.0713765E-4,2.7479675E-1,-2.8882546E-3,4.5449435E-4,-8.595389E-3,-1.0712073E-3,-3.0795769E-3,9.5920236E-4,5.8259866E-3,2.6097929E-3],"split_indices":[53,90,29,7,30,59,9,52,17,0,58,9,0,0,58,109,8,0,12,0,0,0,0,0,39,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.42E2,3.4E1,2.23E2,1.9E1,1.5E1,1.9E1,1.1E2,1.13E2,7E0,1.2E1,1.1E1,4E0,4E0,1.5E1,8.9E1,2.1E1,5E0,1.08E2,6E0,6E0,5E0,6E0,5E0,1E1,1.9E1,7E1,7E0,1.4E1,5E0,1.03E2,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.2324462E-3,-7.5858575E-3,6.690328E-2,-5.875678E-3,-3.8178023E-3,1.22768864E-1,-9.702716E-3,-2.0907314E-2,1.4889422E-2,1.9395949E-1,4.6420418E-2,-2.6487885E-3,1.5181703E-2,7.289406E-2,-2.9321136E-2,-5.571751E-2,2.1871038E-2,4.091844E-3,1.1982336E-2,9.781831E-4,3.1006704E-3,-9.838765E-4,1.9848663E-3,1.8952134E-4,6.484813E-3,-2.2772874E-3,1.0567203E-3,-6.292595E-4,-4.375475E-3,1.9192853E-3,-5.0805474E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,7,9,11,13,15,17,19,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5165402E-1,1.0399145E-1,1.3385998E-1,0E0,7.22432E-2,8.883163E-2,1.7535927E-2,9.526622E-2,5.4457247E-2,3.956905E-2,3.8370378E-3,0E0,1.112969E-2,4.6157714E-2,1.15893215E-1,1.5233859E-2,6.501205E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,6,-1,8,10,12,14,16,18,20,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1.393E3,5.577948E11,-5.875678E-3,2.0718746E8,7.643508E5,4.904973E6,6.7897725E-1,1.0478362E-1,1.0851409E3,3.13373E3,-2.6487885E-3,6.157198E1,1E0,1.051196E6,9E0,2.2048602E0,4.091844E-3,1.1982336E-2,9.781831E-4,3.1006704E-3,-9.838765E-4,1.9848663E-3,1.8952134E-4,6.484813E-3,-2.2772874E-3,1.0567203E-3,-6.292595E-4,-4.375475E-3,1.9192853E-3,-5.0805474E-4],"split_indices":[53,12,32,0,7,34,33,57,40,53,4,0,57,8,30,3,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,2.3E2,3E1,6E0,2.24E2,1.7E1,1.3E1,1.17E2,1.07E2,8E0,9E0,4E0,9E0,9E0,1.08E2,9E0,9.8E1,4E0,4E0,5E0,4E0,4E0,5E0,5E0,4E0,7.8E1,3E1,5E0,4E0,6.1E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.10813E-3,6.2053115E-3,1.7062627E-4,-9.529811E-3,4.523369E-2,-3.731356E-2,5.673263E-3,6.2087933E-3,7.288687E-2,-8.9933574E-2,-4.995276E-3,7.296518E-2,-5.0530196E-3,1.8669074E-2,-1.72184E-3,7.034394E-3,5.320442E-2,-2.1996384E-3,-8.425229E-3,1.9213033E-3,-1.5001298E-3,-1.8195294E-4,5.28027E-3,4.9531157E-3,-4.878689E-4,1.6219923E-4,1.8325793E-3,3.4376464E-3,7.2254107E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,-1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0634153E-1,0E0,1.1845655E-1,9.440653E-2,5.1347256E-2,1.3389726E-1,1.05028555E-1,1.2108576E-2,3.5903826E-2,1.0895437E-1,6.581086E-2,6.4634345E-2,7.98104E-2,5.3014746E-3,0E0,0E0,1.8148273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16],"right_children":[2,-1,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E1,6.2053115E-3,1.5057492E0,5.283708E0,2.174459E5,7.5608E4,7.014272E7,1.10018056E8,1.8426E4,3.5342257E0,1.8278716E0,1E0,6.186634E0,3.5617366E8,-1.72184E-3,7.034394E-3,1.4E1,-2.1996384E-3,-8.425229E-3,1.9213033E-3,-1.5001298E-3,-1.8195294E-4,5.28027E-3,4.9531157E-3,-4.878689E-4,1.6219923E-4,1.8325793E-3,3.4376464E-3,7.2254107E-4],"split_indices":[10,0,43,57,29,9,7,33,9,57,54,8,57,7,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,5E0,2.69E2,2.22E2,4.7E1,7.8E1,1.44E2,2E1,2.7E1,2.9E1,4.9E1,1.9E1,1.25E2,1.6E1,4E0,4E0,2.3E1,2.1E1,8E0,1.8E1,3.1E1,7E0,1.2E1,5E0,1.2E2,1E1,6E0,1.4E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.773961E-4,-8.464028E-3,5.476503E-2,6.265524E-4,-4.2521432E-2,1.0550452E-1,-2.1806126E-3,-8.615522E-2,6.3765785E-3,-2.1358268E-2,-1.3890971E-1,4.7468036E-2,8.744282E-3,1.7477887E-2,-2.4335731E-3,-8.4018465E-3,-6.9886184E-4,1.4331637E-2,-5.4915853E-2,1.3853628E-2,-7.103333E-2,-7.949128E-3,-3.5628616E-3,3.2532907E-3,3.988344E-5,-3.2758046E-4,1.822024E-3,-9.4913185E-4,1.3116873E-3,1.2824076E-3,-3.7994068E-3,-3.7416408E-4,4.073529E-3,-4.1980296E-3,-7.156176E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,-1,-1,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1918372E-1,7.4950226E-2,9.846942E-2,9.635689E-2,1.0106705E-1,7.9173654E-2,1.7859451E-2,7.927621E-2,8.903991E-2,7.619765E-2,5.3239465E-3,1.2908589E-2,0E0,7.3458166E-3,0E0,0E0,0E0,8.024508E-2,5.0282627E-2,4.3364953E-2,1.8705934E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,-1,-1,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,1.7303246E7,6.464779E6,3.4955364E7,2.4320412E5,5.505334E9,3.6661228E7,1.332E3,1E0,2.142943E1,1.3E1,1.91016E5,8.744282E-3,4.0873584E7,-2.4335731E-3,-8.4018465E-3,-6.9886184E-4,3.81E2,9E0,5.4494965E-1,3.084419E6,-7.949128E-3,-3.5628616E-3,3.2532907E-3,3.988344E-5,-3.2758046E-4,1.822024E-3,-9.4913185E-4,1.3116873E-3,1.2824076E-3,-3.7994068E-3,-3.7416408E-4,4.073529E-3,-4.1980296E-3,-7.156176E-4],"split_indices":[53,46,30,7,29,5,49,0,90,57,8,10,0,45,0,0,0,0,3,58,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.41E2,3.3E1,1.91E2,5E1,1.7E1,1.6E1,1.1E1,1.8E2,4.2E1,8E0,1.1E1,6E0,1.2E1,4E0,4E0,7E0,1.6E2,2E1,2.5E1,1.7E1,4E0,4E0,7E0,4E0,6E0,6E0,4.6E1,1.14E2,5E0,1.5E1,2E1,5E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.0185017E-3,-1.5962647E-2,2.4150021E-2,-1.3301289E-1,-5.439939E-3,7.054804E-3,1.8878033E-2,-6.3760113E-4,-8.78758E-3,4.3586013E-3,-1.08299665E-2,4.8070557E-2,1.1265427E-3,-4.3094933E-2,6.378897E-3,2.9208936E-2,1.0885883E-1,1.3159337E-2,-4.7827095E-2,-1.14338574E-4,-4.3156664E-3,7.9006737E-4,-2.0754023E-3,2.4170827E-3,-9.0318144E-4,6.729134E-3,1.6055589E-3,1.3149536E-3,-1.6478837E-3,-7.631443E-4,-5.2148853E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1108924E-1,1.791025E-1,8.625292E-2,8.008231E-2,7.3868945E-2,0E0,6.5758616E-2,0E0,0E0,0E0,7.190129E-2,5.1855497E-2,4.7753297E-2,9.23132E-2,4.7927845E-2,4.424071E-2,2.5256842E-2,5.0025683E-2,3.0843902E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.006438E6,2.3361564E0,4.92E2,1.3295195E0,1E1,7.054804E-3,9.397667E6,-6.3760113E-4,-8.78758E-3,4.3586013E-3,1.53091E5,7.120612E2,3.160848E6,6.5284515E6,1E0,1.7090584E1,2.6056337E0,9.906531E6,2.1881187E0,-1.14338574E-4,-4.3156664E-3,7.9006737E-4,-2.0754023E-3,2.4170827E-3,-9.0318144E-4,6.729134E-3,1.6055589E-3,1.3149536E-3,-1.6478837E-3,-7.631443E-4,-5.2148853E-3],"split_indices":[1,55,10,54,10,0,46,0,0,0,9,53,9,46,90,57,54,30,54,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.45E2,1.3E2,1.1E1,1.34E2,4E0,1.26E2,4E0,7E0,6E0,1.28E2,4.7E1,7.9E1,4.4E1,8.4E1,3.7E1,1E1,6.4E1,1.5E1,2.5E1,1.9E1,7E1,1.4E1,2.5E1,1.2E1,6E0,4E0,4.9E1,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.1318335E-4,-2.3540951E-2,2.2438744E-2,-1.6739797E-2,-8.820643E-3,6.1444105E-3,5.4549992E-2,-4.765783E-2,1.4069596E-3,4.5384695E-3,-1.2777979E-3,9.416091E-2,5.25755E-3,-7.030939E-2,2.4268355E-2,1.2203398E-2,-8.6320475E-2,-4.5638257E-3,4.56166E-3,1.3145782E-1,3.9693024E-2,4.1257378E-2,-2.8959513E-3,-5.282765E-3,-6.4005767E-4,3.6747127E-3,-6.206844E-4,-4.5480225E-3,9.239956E-4,-9.839027E-4,-6.169812E-3,4.9174903E-4,-2.9592053E-3,4.3434035E-3,8.6018825E-3,3.0024275E-3,-4.178631E-4,3.027109E-3,1.7843115E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4307447E-1,1.4650947E-1,7.291722E-2,7.132192E-2,0E0,6.461322E-2,9.127332E-2,7.8280605E-2,7.777206E-2,0E0,5.073321E-2,4.760064E-2,5.6575045E-2,8.8629425E-2,2.6237717E-2,6.570856E-2,2.501142E-2,0E0,3.6067333E-2,1.3206601E-2,1.552622E-2,1.3477802E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,7.5646E4,-8.820643E-3,2.3152277E2,6.6E1,4.5510144E9,3.7085085E0,4.5384695E-3,1.744E3,8.4550006E5,1.5E1,1.8130841E0,2.9655823E-1,2.3111668E-1,5.743459E-3,-4.5638257E-3,3.605082E7,5.2E1,3.1E1,8.3395955E6,-2.8959513E-3,-5.282765E-3,-6.4005767E-4,3.6747127E-3,-6.206844E-4,-4.5480225E-3,9.239956E-4,-9.839027E-4,-6.169812E-3,4.9174903E-4,-2.9592053E-3,4.3434035E-3,8.6018825E-3,3.0024275E-3,-4.178631E-4,3.027109E-3,1.7843115E-4],"split_indices":[53,53,29,9,0,53,8,5,54,0,2,34,3,54,59,59,58,0,46,8,8,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,1.3E2,1.39E2,1.26E2,4E0,9.3E1,4.6E1,4.6E1,8E1,6E0,8.7E1,2.5E1,2.1E1,3.5E1,1.1E1,7.2E1,8E0,4E0,8.3E1,1.4E1,1.1E1,1.4E1,7E0,1.9E1,1.6E1,4E0,7E0,4E0,6.8E1,4E0,4E0,7.7E1,6E0,1E1,4E0,7E0,4E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.45128E-3,-9.6646714E-4,5.5275573E-3,-5.26827E-3,2.4113716E-3,5.3020795E-3,-1.6022695E-4,-1.4579112E-2,1.700076E-2,-5.3464344E-5,-2.7475825E-3,4.4376818E-3,4.4940962E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,-1,5,-1,7,9,11,-1,-1,-1,-1],"loss_changes":[1.1386705E-1,1.060519E-1,0E0,0E0,7.8013316E-2,0E0,6.557262E-2,8.7067306E-2,6.8600334E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,8,8],"right_children":[2,4,-1,-1,6,-1,8,10,12,-1,-1,-1,-1],"split_conditions":[2.0704236E3,1.393E3,5.5275573E-3,-5.26827E-3,1.55E2,5.3020795E-3,2.8955648E6,1.609091E1,3.2239552E6,-5.3464344E-5,-2.7475825E-3,4.4376818E-3,4.4940962E-4],"split_indices":[53,12,0,0,11,0,51,57,51,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.75E2,7E0,7E0,2.68E2,5E0,2.63E2,1.43E2,1.2E2,1.11E2,3.2E1,9E0,1.11E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-2.0428915E-3,-2.298763E-2,1.7783502E-2,-1.6606E-2,-8.838524E-3,1.3478886E-2,6.362863E-3,-7.863222E-2,-5.8612945E-3,4.8478255E-3,7.000703E-2,-1.13282606E-1,9.7142556E-4,2.9943136E-3,-5.8129244E-2,4.047851E-3,-7.774816E-4,8.818214E-2,6.815482E-5,-3.3162963E-3,-7.5119673E-3,-1.7218709E-3,8.3968084E-4,-4.5681596E-3,-8.9523644E-4,-4.4629625E-3,2.3876995E-4,2.7285644E-3,6.177155E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,-1,21,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1958804E-1,1.477862E-1,7.48524E-2,9.0176776E-2,0E0,6.991791E-2,0E0,7.150474E-2,5.43292E-2,5.8947764E-2,2.3533605E-2,1.9110903E-2,0E0,6.287007E-2,2.509335E-2,0E0,6.8741135E-2,1.1995606E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,-1,22,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1423952E2,1.5266737E2,7E0,-8.838524E-3,4.7321553E5,6.362863E-3,3.3936841E0,2.7449496E0,2.3152277E2,8.543362E5,2.0189648E0,9.7142556E-4,3.3617156E-4,6.433292E-2,4.047851E-3,1.744E3,2.1E1,6.815482E-5,-3.3162963E-3,-7.5119673E-3,-1.7218709E-3,8.3968084E-4,-4.5681596E-3,-8.9523644E-4,-4.4629625E-3,2.3876995E-4,2.7285644E-3,6.177155E-3],"split_indices":[53,53,57,3,0,29,0,55,54,53,29,59,0,40,40,0,2,3,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.39E2,1.47E2,1.35E2,4E0,1.43E2,4E0,1.9E1,1.16E2,1.25E2,1.8E1,1.4E1,5E0,1E2,1.6E1,7E0,1.18E2,1.4E1,4E0,9E0,5E0,2.7E1,7.3E1,7E0,9E0,6E0,1.12E2,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[5.7333894E-3,-6.719583E-3,3.6949817E-2,-1.7911758E-2,2.0521104E-2,7.7641965E-3,2.5922228E-2,-1.2368122E-2,-7.7552693E-3,-1.7887685E-2,2.8724687E-2,-5.597706E-3,7.084077E-2,-5.918074E-3,-6.1608693E-3,3.9922236E-4,-1.579337E-3,6.104172E-2,1.6593711E-2,1.9138519E-2,-3.747491E-2,3.8252782E-2,1.1302008E-1,-3.1683547E-3,1.0989693E-4,7.0366677E-4,4.039191E-3,-5.175623E-4,2.1454971E-3,4.318641E-3,5.985876E-5,-2.5381346E-3,5.9405447E-4,2.6148122E-3,-5.241003E-4,6.393806E-3,2.020415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,21,-1,23,-1,-1,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0909474E-1,6.164896E-2,1.1180337E-1,1.172484E-1,1.910411E-2,0E0,1.06583744E-1,9.944131E-2,0E0,5.0930437E-3,1.819691E-2,3.620069E-2,3.828737E-2,0E0,7.139406E-2,0E0,0E0,1.4535017E-2,3.1426847E-2,3.4505825E-2,1.8497344E-2,1.8131569E-2,1.7141044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,22,-1,24,-1,-1,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.165913E6,1.19881555E-1,2.207E3,9.922613E5,1.8681856E8,7.7641965E-3,2.3101475E5,1E0,-7.7552693E-3,9.141936E7,2.4920886E0,1.3907746E10,3.5659203E8,-5.918074E-3,1E0,3.9922236E-4,-1.579337E-3,4.0973E4,6.098826E6,1.2323925E6,4.904973E6,1.9879582E7,6.97696E5,-3.1683547E-3,1.0989693E-4,7.0366677E-4,4.039191E-3,-5.175623E-4,2.1454971E-3,4.318641E-3,5.985876E-5,-2.5381346E-3,5.9405447E-4,2.6148122E-3,-5.241003E-4,6.393806E-3,2.020415E-3],"split_indices":[1,39,0,34,7,0,29,67,0,7,59,5,7,0,109,0,0,9,49,33,33,33,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,2E2,7.9E1,1.42E2,5.8E1,5E0,7.4E1,1.38E2,4E0,1E1,4.8E1,4.4E1,3E1,6E0,1.32E2,4E0,6E0,1.2E1,3.6E1,2.5E1,1.9E1,1.8E1,1.2E1,1.5E1,1.17E2,5E0,7E0,1.9E1,1.7E1,4E0,2.1E1,1.4E1,5E0,1.3E1,5E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.3789573E-3,1.0063862E-3,5.9794886E-3,-1.5833447E-2,1.3962005E-2,4.0154916E-4,-9.459326E-2,9.232384E-2,8.38452E-3,1.6629538E-2,-4.2887956E-2,-9.929272E-3,-2.7302591E-2,6.7551676E-3,3.9517356E-4,1.3423104E-2,-5.9970953E-2,5.9556314E-3,3.7276512E-4,7.9784606E-4,-5.5066603E-3,-2.3681887E-3,1.216665E-3,-1.188269E-3,9.26776E-4,-1.1442644E-3,-3.6483232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,21,-1,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.150084E-2,5.869075E-2,0E0,1.4964181E-1,6.565934E-2,6.955836E-2,1.5704681E-1,4.4292867E-2,5.005631E-2,6.82098E-2,1.268008E-1,0E0,1.9512625E-2,0E0,0E0,3.5774782E-2,4.7977194E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,22,-1,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5266737E2,1.7546342E8,5.9794886E-3,1.3805645E8,1.8913731E5,3.774648E0,6E0,6.064E3,3.5410094E0,1.8E1,6.8581295E6,-9.929272E-3,8.7E1,6.7551676E-3,3.9517356E-4,2.9266727E2,1.4532832E8,5.9556314E-3,3.7276512E-4,7.9784606E-4,-5.5066603E-3,-2.3681887E-3,1.216665E-3,-1.188269E-3,9.26776E-4,-1.1442644E-3,-3.6483232E-3],"split_indices":[57,7,0,7,34,55,8,0,54,10,46,0,8,0,0,4,12,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.67E2,4E0,1.16E2,1.51E2,9.7E1,1.9E1,9E0,1.42E2,7.1E1,2.6E1,6E0,1.3E1,5E0,4E0,1.33E2,9E0,4E0,6.7E1,1.5E1,1.1E1,9E0,4E0,1.9E1,1.14E2,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.8497586E-4,-4.8974995E-3,3.4094239E-3,3.8608295E-4,4.765679E-3,5.3401053E-2,-3.6771046E-3,1.1053218E-2,7.816129E-3,-3.4633618E-2,8.691376E-3,1.313361E-3,-9.1668055E-4,-4.7582462E-3,-7.340466E-4,4.8860833E-3,1.8795228E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,-1,3,5,-1,7,9,11,-1,13,15,-1,-1,-1,-1,-1,-1],"loss_changes":[9.367142E-2,0E0,8.257971E-2,5.749536E-2,0E0,9.1750495E-2,9.525651E-2,8.683111E-3,0E0,8.953697E-2,7.950284E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,9,9,10,10],"right_children":[2,-1,4,6,-1,8,10,12,-1,14,16,-1,-1,-1,-1,-1,-1],"split_conditions":[1.393E3,-4.8974995E-3,2.0704236E3,6.7897725E-1,4.765679E-3,3.084419E6,5.283708E0,1.714493E6,7.816129E-3,1.571112E5,6.186634E0,1.313361E-3,-9.1668055E-4,-4.7582462E-3,-7.340466E-4,4.8860833E-3,1.8795228E-4],"split_indices":[12,0,53,57,0,9,57,9,0,29,57,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,7E0,2.72E2,2.65E2,7E0,1.8E1,2.47E2,1.4E1,4E0,7E1,1.77E2,9E0,5E0,1.4E1,5.6E1,7E0,1.7E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[3.10983E-3,-7.624493E-3,2.9443417E-2,-1.7544327E-2,3.103107E-2,7.3052705E-3,1.8854015E-2,-2.7903078E-2,2.4438936E-2,4.288795E-3,2.1797735E-2,4.2605728E-2,-1.1739445E-2,4.6737855E-3,-3.289646E-2,4.4845756E-2,-3.269469E-2,3.2397922E-2,-1.3011157E-3,5.5790726E-2,-3.1657107E-3,-4.199061E-2,1.7483294E-2,-3.1305389E-3,-6.786296E-4,-9.278801E-4,2.7096653E-3,-4.106672E-3,1.3971356E-3,-8.55749E-4,1.9759678E-3,2.1691662E-3,4.851416E-3,-2.6312477E-3,-5.603203E-4,3.8929696E-3,-3.5541147E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,-1,23,25,27,29,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.909032E-2,7.6852314E-2,1.07850276E-1,6.983009E-2,2.1486253E-2,0E0,5.5725336E-2,8.6056374E-2,3.8775522E-2,0E0,2.0548508E-2,6.697456E-2,3.079088E-2,0E0,7.801902E-2,2.3037449E-2,3.4763258E-2,1.7424144E-2,0E0,1.1813842E-2,0E0,6.9895517E-3,3.1079432E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,-1,24,26,28,30,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.165913E6,1.2730925E0,2.207E3,2E1,1.7278711E-1,7.3052705E-3,1.9647126E7,1E1,2.5742075E0,4.288795E-3,9.8163464E7,4.613948E7,2.2296374E0,4.6737855E-3,1.31654E5,1.3634276E0,7.05635E0,2.0270285E8,-1.3011157E-3,7.332955E5,-3.1657107E-3,7.386567E0,1.9054878E0,-3.1305389E-3,-6.786296E-4,-9.278801E-4,2.7096653E-3,-4.106672E-3,1.3971356E-3,-8.55749E-4,1.9759678E-3,2.1691662E-3,4.851416E-3,-2.6312477E-3,-5.603203E-4,3.8929696E-3,-3.5541147E-4],"split_indices":[1,43,0,3,39,0,46,10,54,0,33,33,55,0,9,54,54,7,0,29,0,59,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,1.98E2,8E1,1.58E2,4E1,5E0,7.5E1,1.27E2,3.1E1,4E0,3.6E1,4.2E1,3.3E1,4E0,1.23E2,2.3E1,8E0,3E1,6E0,3.8E1,4E0,1.6E1,1.7E1,4.1E1,8.2E1,4E0,1.9E1,4E0,4E0,5E0,2.5E1,3.4E1,4E0,1E1,6E0,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.7198535E-4,-2.3544746E-2,1.4797467E-2,-3.4545183E-2,6.45874E-2,5.692478E-2,3.5581242E-3,-2.0373061E-2,-8.94155E-2,7.3223184E-3,-1.159409E-4,9.994532E-2,-1.2835043E-2,4.893333E-3,-1.834599E-3,4.524906E-2,-3.2174483E-2,-1.7877202E-1,-8.176937E-3,5.7646364E-2,9.677123E-3,-2.2535345E-3,7.6785084E-4,4.1204602E-3,-7.6089375E-2,1.8263121E-4,3.9535593E-3,-5.371586E-4,-4.1156937E-3,-1.2559498E-2,-2.2560365E-3,-2.3323393E-3,1.0743535E-3,3.5475001E-3,7.433986E-4,-4.602491E-4,1.4851124E-3,-4.524676E-3,-1.5768595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,33,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.668275E-2,1.04027845E-1,8.313052E-2,7.127543E-2,7.6459184E-2,1.123237E-1,7.774955E-2,6.0312867E-2,1.3646291E-1,0E0,0E0,9.513281E-2,1.6802294E-2,0E0,5.9567202E-2,1.9782662E-2,7.522719E-2,9.643978E-2,1.5907137E-2,1.3250995E-2,0E0,0E0,0E0,4.9971424E-2,5.9166625E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,34,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.283708E0,9.74979E6,2.3891E4,3.5342257E0,2.2E2,8.652176E6,6.0949473E0,1.7144943E-6,1.8196162E0,7.3223184E-3,-1.159409E-4,6.874319E6,6.048052E5,4.893333E-3,2.7317348E7,2.3063551E2,3.5300705E2,1.332E3,4.706415E2,3.3145252E1,9.677123E-3,-2.2535345E-3,7.6785084E-4,9.821357E-1,1.328903E8,1.8263121E-4,3.9535593E-3,-5.371586E-4,-4.1156937E-3,-1.2559498E-2,-2.2560365E-3,-2.3323393E-3,1.0743535E-3,3.5475001E-3,7.433986E-4,-4.602491E-4,1.4851124E-3,-4.524676E-3,-1.5768595E-3],"split_indices":[57,9,9,57,10,30,57,39,54,0,0,1,33,0,46,34,4,0,4,57,0,0,0,42,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.04E2,1.75E2,9.3E1,1.1E1,3.6E1,1.39E2,7.5E1,1.8E1,4E0,7E0,2.2E1,1.4E1,6E0,1.33E2,1.1E1,6.4E1,8E0,1E1,1.7E1,5E0,6E0,8E0,1.24E2,9E0,6E0,5E0,4.8E1,1.6E1,4E0,4E0,4E0,6E0,1.1E1,6E0,8.3E1,4.1E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.9344317E-3,-3.9738747E-3,5.162234E-2,5.633973E-3,-2.8893141E-2,1.0010953E-1,1.0626008E-2,-1.1397568E-3,4.4696245E-2,1.1824694E-3,-9.196501E-2,1.3570584E-1,1.1094714E-3,1.900634E-3,-6.698067E-3,-2.796468E-2,1.2843429E-2,8.304936E-2,1.3590881E-2,-2.8411347E-2,2.0164834E-2,-9.492934E-3,-4.3385044E-2,9.435343E-3,2.5506152E-3,1.8425869E-3,-3.190861E-2,1.0081771E-3,-2.3975053E-3,8.933171E-4,-2.8422517E-3,1.3516669E-3,5.6584133E-3,-9.967816E-4,1.3063625E-3,1.710608E-4,-2.0619181E-3,1.3646035E-3,-8.9839863E-4,1.267218E-3,-3.5966546E-3,-1.5156793E-4,-2.492823E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,35,37,-1,39,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9338146E-2,5.5947762E-2,6.453355E-2,4.476788E-2,1.2340889E-1,3.7316084E-2,1.0366918E-2,5.4748356E-2,2.869597E-2,2.585214E-2,1.07578635E-1,4.4241503E-2,0E0,0E0,1.658841E-2,6.124526E-2,4.8157215E-2,1.9927569E-2,8.428732E-3,9.634585E-3,1.1103762E-2,0E0,4.1556478E-2,0E0,0E0,0E0,5.5483356E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,36,38,-1,40,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,4.352025E0,2.5304102E6,3.8576598E0,7.3917985E6,7.643508E5,1.874657E1,8.461558E10,4.5194294E5,1.664E3,4.752366E0,6.6E1,1.1094714E-3,1.900634E-3,1.6E1,1.47887E5,2.1061536E3,2.199787E8,2.0541462E8,9.141936E7,1.10018056E8,-9.492934E-3,2.7181172E5,9.435343E-3,2.5506152E-3,1.8425869E-3,9.5350234E4,1.0081771E-3,-2.3975053E-3,8.933171E-4,-2.8422517E-3,1.3516669E-3,5.6584133E-3,-9.967816E-4,1.3063625E-3,1.710608E-4,-2.0619181E-3,1.3646035E-3,-8.9839863E-4,1.267218E-3,-3.5966546E-3,-1.5156793E-4,-2.492823E-3],"split_indices":[53,55,48,55,46,34,59,32,48,2,57,8,0,0,3,1,4,7,7,7,33,0,34,0,0,0,29,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,2.32E2,3.2E1,1.68E2,6.4E1,1.4E1,1.8E1,1.44E2,2.4E1,4.4E1,2E1,9E0,5E0,6E0,1.2E1,4.9E1,9.5E1,1E1,1.4E1,1.7E1,2.7E1,5E0,1.5E1,4E0,5E0,4E0,8E0,1.6E1,3.3E1,8.8E1,7E0,5E0,5E0,4E0,1E1,6E0,1.1E1,2.2E1,5E0,5E0,1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.0727954E-3,-7.935507E-3,2.9022856E-2,-1.4421671E-2,2.8189072E-2,7.7344336E-2,1.5957605E-2,-2.3930179E-2,1.576601E-2,-1.7996109E-3,4.521769E-2,5.8754752E-3,1.6223729E-3,3.3855986E-2,-3.1947007E-3,1.4694356E-2,-4.034782E-2,4.7490828E-2,-2.3092072E-2,5.9222557E-2,-1.1716188E-3,-4.626999E-4,4.6287075E-2,-2.4246399E-3,1.0441747E-2,-2.2834345E-5,4.7600544E-3,-8.4693665E-3,-1.2354819E-3,-1.342718E-3,3.1859463E-3,-2.4168193E-3,2.640477E-4,7.390635E-4,4.3214182E-3,3.6260777E-3,1.0178307E-3,-1.0025614E-3,1.5847764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.921874E-2,4.9143E-2,4.1065034E-2,5.1183328E-2,3.842727E-2,2.5136836E-2,1.8619103E-2,8.619959E-2,5.4105584E-2,0E0,2.723803E-2,0E0,0E0,1.5747454E-2,1.8763168E-2,5.5289406E-2,1.7942044E-1,4.195271E-2,1.7251756E-2,3.1390227E-2,0E0,0E0,1.5531905E-2,0E0,1.7745424E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,5.8536225E5,7.013774E5,3.131673E6,1.6814E4,2.7479675E-1,3.133129E8,1.56221E5,9.906531E6,-1.7996109E-3,4.986835E6,5.8754752E-3,1.6223729E-3,1.718354E8,3.4519572E0,9.74979E6,2.07684E5,1.0705044E3,1.08E2,5.855073E6,-1.1716188E-3,-4.626999E-4,3.13423E6,-2.4246399E-3,3.5055136E2,-2.2834345E-5,4.7600544E-3,-8.4693665E-3,-1.2354819E-3,-1.342718E-3,3.1859463E-3,-2.4168193E-3,2.640477E-4,7.390635E-4,4.3214182E-3,3.6260777E-3,1.0178307E-3,-1.0025614E-3,1.5847764E-3],"split_indices":[43,29,34,30,9,39,7,1,30,0,1,0,0,7,55,9,1,56,8,46,0,0,46,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.07E2,6.6E1,1.76E2,3.1E1,1.3E1,5.3E1,1.34E2,4.2E1,6E0,2.5E1,5E0,8E0,2.7E1,2.6E1,4E1,9.4E1,2.3E1,1.9E1,2.1E1,4E0,6E0,2.1E1,5E0,2.1E1,3.5E1,5E0,7E0,8.7E1,5E0,1.8E1,9E0,1E1,1E1,1.1E1,8E0,1.3E1,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.6682731E-3,-4.392072E-3,4.1339127E-3,-1.0219255E-3,-5.4091185E-2,7.665787E-3,-2.3719488E-2,-5.265569E-3,-4.989092E-3,-2.2933478E-2,1.810739E-2,-9.479532E-3,-1.0821128E-1,-1.574431E-3,1.4044524E-3,-2.7756707E-4,-7.538209E-3,4.6751383E-3,5.5921223E-4,7.8708807E-4,-1.4984503E-3,-2.7210808E-3,-6.699026E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.889884E-2,4.4832926E-2,0E0,4.9873684E-2,4.4475958E-2,5.8884747E-2,8.319537E-2,1.139161E-2,0E0,1.1087772E-1,6.690985E-2,3.794863E-2,9.713791E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,3.7085085E0,4.1339127E-3,1.6585214E7,5.743459E-3,3.81E2,2.597438E5,5.3725034E-2,-4.989092E-3,8.5585E4,4.56E2,2.236127E1,1.671E3,-1.574431E-3,1.4044524E-3,-2.7756707E-4,-7.538209E-3,4.6751383E-3,5.5921223E-4,7.8708807E-4,-1.4984503E-3,-2.7210808E-3,-6.699026E-3],"split_indices":[53,54,0,46,58,0,29,43,0,30,0,57,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.67E2,7E0,2.51E2,1.6E1,1.82E2,6.9E1,9E0,7E0,4.6E1,1.36E2,6E1,9E0,5E0,4E0,4.2E1,4E0,8E0,1.28E2,2.8E1,3.2E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.5023522E-3,-1.5602255E-2,1.7441092E-2,-1.0162039E-2,-7.2873607E-3,7.895297E-4,4.7273334E-2,4.0240665E-3,-1.5418164E-2,4.009565E-3,-5.931247E-3,3.6895003E-2,6.4239395E-3,-5.0314046E-2,2.2090576E-3,3.4455676E-2,-2.030968E-2,5.093208E-2,-1.0630324E-2,-3.1732633E-3,-9.529626E-5,1.5703777E-3,-7.976968E-4,4.5256414E-3,-4.788931E-4,-5.6292643E-3,-4.6572112E-4,3.8760952E-3,1.3937361E-3,-1.999745E-3,1.3541731E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,15,17,-1,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.79705E-2,1.0550657E-1,7.3482186E-2,7.006374E-2,0E0,5.6195017E-2,4.559607E-2,0E0,7.884618E-2,0E0,5.2962836E-2,3.3230133E-2,0E0,3.8460568E-2,5.4181363E-2,6.975081E-2,6.8401605E-2,2.3560517E-2,1.704001E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,16,18,-1,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,1E1,-7.2873607E-3,2.3152277E2,1.9936699E3,4.0240665E-3,7.5608E4,4.009565E-3,2.3891E4,9.12179E6,6.4239395E-3,4.5510144E9,1.9179105E0,1.0326834E7,9.577834E7,3.0855308E0,2.820341E0,-3.1732633E-3,-9.529626E-5,1.5703777E-3,-7.976968E-4,4.5256414E-3,-4.788931E-4,-5.6292643E-3,-4.6572112E-4,3.8760952E-3,1.3937361E-3,-1.999745E-3,1.3541731E-3],"split_indices":[53,53,29,10,0,53,4,0,9,0,9,9,0,5,54,1,7,55,55,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,1.37E2,1.47E2,1.33E2,4E0,9.5E1,5.2E1,6E0,1.27E2,6E0,8.9E1,4.8E1,4E0,4.2E1,8.5E1,2.3E1,6.6E1,3.7E1,1.1E1,3E1,1.2E1,3.2E1,5.3E1,9E0,1.4E1,5E0,6.1E1,1.3E1,2.4E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.789992E-4,-1.875018E-3,3.9721313E-3,-5.6791026E-2,1.5086506E-3,1.4423865E-3,-9.168243E-2,-2.0941436E-2,9.950489E-3,-5.61779E-3,-2.1209961E-3,-4.1190768E-3,-8.806837E-3,9.836872E-4,4.7898598E-2,6.4780624E-3,-7.6117937E-4,-3.2906476E-3,2.6410862E-4,3.621947E-3,3.852045E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,-1,9,11,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0291413E-2,4.802661E-2,0E0,4.899134E-2,4.6451256E-2,0E0,9.599768E-3,1.9120288E-1,6.0521506E-2,0E0,0E0,1.1378614E-1,0E0,5.034206E-2,4.069835E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14],"right_children":[2,4,-1,6,8,-1,10,12,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,8.89E2,3.9721313E-3,1.4E1,1.8196162E0,1.4423865E-3,1.6E1,1.7885118E0,4.3857228E5,-5.61779E-3,-2.1209961E-3,3.9449276E1,-8.806837E-3,1.3944E5,3.0855308E0,6.4780624E-3,-7.6117937E-4,-3.2906476E-3,2.6410862E-4,3.621947E-3,3.852045E-4],"split_indices":[53,2,0,3,54,0,3,54,29,0,0,53,0,12,55,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,2.57E2,7E0,1.4E1,2.43E2,4E0,1E1,6.6E1,1.77E2,5E0,5E0,6.1E1,5E0,1.44E2,3.3E1,4E0,5.7E1,8E0,1.36E2,1.8E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.148217E-3,-2.1452382E-2,1.1460967E-2,6.602388E-3,-4.762114E-2,4.3823905E-3,7.3816353E-3,-5.7850122E-2,1.934617E-2,-1.2612973E-1,-2.7812105E-2,2.1267667E-2,-1.0790346E-2,-4.4773915E-3,-3.1687386E-4,5.3495683E-2,-5.401492E-3,-9.006185E-3,-1.6706236E-4,-6.262814E-2,-9.923683E-4,5.697159E-2,5.4963017E-3,1.393674E-2,-3.659143E-2,3.4991547E-4,3.6664694E-3,-1.8212503E-3,1.3416331E-3,1.5623749E-3,-4.0925746E-3,1.3614483E-3,-1.9180661E-3,5.968352E-4,4.232015E-3,2.4256683E-3,-4.2030716E-4,-1.6058675E-3,2.0196324E-3,-3.2132403E-3,-8.836243E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,-1,25,27,-1,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.218719E-2,7.960352E-2,5.872459E-2,4.4651724E-2,8.358212E-2,0E0,4.2369716E-2,1.7402785E-2,3.831938E-2,9.291853E-2,4.2890675E-2,5.3181328E-2,4.7082916E-2,0E0,0E0,2.21885E-2,3.3006832E-2,0E0,0E0,5.373402E-2,3.4790438E-2,4.340905E-2,4.6720725E-2,5.7048574E-2,1.9455835E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,-1,26,28,-1,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.283708E0,1.51331E5,6.186634E0,3.9490446E-1,2.07684E5,4.3823905E-3,1.3879616E7,2.83E2,4.352025E0,1.8550434E0,9.526745E1,4.0012E4,2.5659466E1,-4.4773915E-3,-3.1687386E-4,1.2E1,5.268293E0,-9.006185E-3,-1.6706236E-4,1.1078758E0,1.7E1,1.68834E5,1.13874E5,9.707008E2,1.3548306E3,3.4991547E-4,3.6664694E-3,-1.8212503E-3,1.3416331E-3,1.5623749E-3,-4.0925746E-3,1.3614483E-3,-1.9180661E-3,5.968352E-4,4.232015E-3,2.4256683E-3,-4.2030716E-4,-1.6058675E-3,2.0196324E-3,-3.2132403E-3,-8.836243E-4],"split_indices":[57,1,57,59,1,0,46,0,55,54,53,9,57,0,0,3,55,0,0,55,8,30,30,4,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.07E2,1.73E2,5.2E1,5.5E1,7E0,1.66E2,8E0,4.4E1,1E1,4.5E1,9.4E1,7.2E1,4E0,4E0,1.8E1,2.6E1,6E0,4E0,1.9E1,2.6E1,2.8E1,6.6E1,3.7E1,3.5E1,7E0,1.1E1,1.3E1,1.3E1,4E0,1.5E1,1.5E1,1.1E1,1.3E1,1.5E1,1.5E1,5.1E1,1.4E1,2.3E1,1.1E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.9003861E-3,-3.8611423E-3,4.8409794E-3,-1.2841899E-2,1.8640252E-2,-5.1921415E-3,-4.7663875E-2,2.45833E-2,-3.363066E-3,3.7808663E-3,-9.101748E-3,-6.942747E-3,-2.9492326E-2,5.644717E-2,1.4089741E-2,-2.708027E-3,-1.5446395E-4,-3.8435345E-4,-3.6948293E-3,6.5807E-4,3.5144568E-3,-1.0141828E-3,1.0240912E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8335505E-2,5.5205017E-2,0E0,5.1696558E-2,4.4297084E-2,5.570542E-2,6.1144345E-2,2.3947716E-2,0E0,0E0,4.408755E-2,0E0,3.26008E-2,1.3997875E-2,1.7275069E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,10,10,12,12,13,13,14,14],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5266737E2,7.91732E-1,4.8409794E-3,4.352025E0,7.2331117E8,1E1,4.5200844E0,7.013774E5,-3.363066E-3,3.7808663E-3,3.4585872E8,-6.942747E-3,7.188264E-2,5.2247315E6,3.4519572E0,-2.708027E-3,-1.5446395E-4,-3.8435345E-4,-3.6948293E-3,6.5807E-4,3.5144568E-3,-1.0141828E-3,1.0240912E-3],"split_indices":[57,40,0,55,7,10,55,34,0,0,5,0,58,33,55,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.71E2,4E0,1.94E2,7.7E1,1.6E2,3.4E1,7.3E1,4E0,6E0,1.54E2,4E0,3E1,1.7E1,5.6E1,1.5E1,1.39E2,2.2E1,8E0,6E0,1.1E1,1E1,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.186097E-3,-1.1866848E-2,1.5157879E-2,-6.546152E-3,-6.806321E-3,4.703188E-2,4.199202E-3,-1.3600078E-2,4.4563077E-2,7.7345185E-2,-8.293204E-3,-4.8439032E-3,5.3423297E-2,-3.1408814E-3,-4.7572985E-2,6.698947E-3,-1.2073182E-3,1.3222776E-1,2.0734707E-2,-1.8886598E-3,9.843785E-4,-1.0600052E-2,2.30472E-3,6.8500645E-2,1.628716E-4,-3.7994857E-3,7.2084506E-5,-8.4348134E-4,-3.8035568E-3,-1.4896881E-3,1.1815849E-3,7.8587036E-4,8.64069E-3,1.7055051E-3,-5.878248E-4,-1.8866541E-3,6.123393E-5,3.8793094E-3,1.2172322E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.085698E-2,9.597147E-2,5.045055E-2,4.7473762E-2,0E0,6.2693626E-2,4.877889E-2,4.071345E-2,7.4094005E-2,7.1824595E-2,1.4473284E-2,2.9920764E-2,1.2544144E-2,3.33813E-2,2.6161447E-2,0E0,1.0874222E-2,7.341756E-2,7.575712E-3,0E0,0E0,3.0858992E-2,0E0,6.3885376E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,21,21,23,23],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.3891E4,9.750871E6,-6.806321E-3,8.652176E6,4.7321553E5,1.714493E6,1.17951E5,8.833194E5,5.4527334E9,4.4E1,1.53E2,5.6923714E1,2.1894946E0,6.698947E-3,3.5631923E8,1.401746E6,6.427584E6,-1.8886598E-3,9.843785E-4,1.83437E5,2.30472E-3,9.0871654E-2,1.628716E-4,-3.7994857E-3,7.2084506E-5,-8.4348134E-4,-3.8035568E-3,-1.4896881E-3,1.1815849E-3,7.8587036E-4,8.64069E-3,1.7055051E-3,-5.878248E-4,-1.8866541E-3,6.123393E-5,3.8793094E-3,1.2172322E-3],"split_indices":[53,53,9,9,0,30,29,9,1,34,12,3,8,51,54,0,7,1,1,0,0,9,0,39,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,1.33E2,1.44E2,1.29E2,4E0,3.6E1,1.08E2,1.14E2,1.5E1,2.3E1,1.3E1,9.2E1,1.6E1,8.8E1,2.6E1,4E0,1.1E1,1.1E1,1.2E1,6E0,7E0,8.4E1,8E0,1.2E1,4E0,4E0,8.4E1,1.5E1,1.1E1,5E0,6E0,4E0,7E0,8E0,4E0,2.3E1,6.1E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.6985955E-3,-1.5817555E-2,1.2537015E-2,-2.4081336E-2,5.9994448E-2,8.117583E-2,6.564742E-3,-1.2527099E-2,-6.395175E-2,6.3077626E-3,-1.6493232E-4,7.228045E-4,1.0579659E-1,4.46471E-3,3.0899923E-3,4.208224E-2,-2.2120904E-2,-1.4637566E-1,-6.3969824E-3,2.6258028E-3,6.5910104E-3,3.9268356E-2,-6.0501467E-3,2.0237388E-4,3.6389187E-3,-4.977804E-4,-5.4104105E-3,-9.13431E-3,-2.996411E-3,1.9949274E-3,-1.409797E-3,1.1378855E-3,5.198764E-3,2.5137726E-5,-3.4834195E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.411496E-2,6.9732584E-2,7.161473E-2,4.4805422E-2,5.4346256E-2,2.0908296E-2,5.09058E-2,4.1795984E-2,1.034292E-1,0E0,0E0,0E0,1.0053404E-2,0E0,5.248427E-2,1.6364764E-2,7.100971E-2,2.5148034E-2,1.8190356E-2,0E0,0E0,3.1508498E-2,5.835324E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.283708E0,9.750914E6,2.222512E3,3.7570093E0,4.56E2,8.562227E4,6.186634E0,1.7144943E-6,4.0054E4,6.3077626E-3,-1.6493232E-4,7.228045E-4,7.59093E5,4.46471E-3,2.3891E4,2.3063551E2,1.3599619E7,9.48E2,4.0098386E0,2.6258028E-3,6.5910104E-3,1.7258567E0,2.7317348E7,2.0237388E-4,3.6389187E-3,-4.977804E-4,-5.4104105E-3,-9.13431E-3,-2.996411E-3,1.9949274E-3,-1.409797E-3,1.1378855E-3,5.198764E-3,2.5137726E-5,-3.4834195E-3],"split_indices":[57,9,48,57,0,29,57,39,9,0,0,0,1,0,9,34,46,0,57,0,0,40,46,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.08E2,1.75E2,9.8E1,1E1,1.3E1,1.62E2,7.7E1,2.1E1,4E0,6E0,4E0,9E0,5E0,1.57E2,1.1E1,6.6E1,8E0,1.3E1,5E0,4E0,3.1E1,1.26E2,6E0,5E0,6E1,6E0,4E0,4E0,4E0,9E0,2.7E1,4E0,1.16E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.1254176E-3,-3.7178583E-3,4.7106627E-2,-3.2828588E-4,-4.796238E-2,8.963617E-2,1.1411061E-2,-3.9945E-3,1.6465288E-3,-4.5591295E-3,-2.3175336E-3,8.226878E-3,1.6642828E-3,-5.4929177E-3,2.5994773E-3,4.17021E-3,-2.6566945E-3,1.2388675E-3,-1.2102257E-3,-8.9356425E-4,1.0790093E-3,-2.9429408E-5,1.9185381E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,-1,-1,19,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.087845E-2,3.5827085E-2,4.603876E-2,3.8205758E-2,3.9391737E-2,6.0224503E-2,1.401724E-2,0E0,3.304215E-2,0E0,7.7121193E-3,0E0,0E0,6.0904752E-3,0E0,3.808103E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,13,13,15,15],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,-1,-1,20,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1E0,2.45226E6,4.9759598E1,2.5952345E5,3.0256798E0,3.2773333E0,-3.9945E-3,2.179529E3,-4.5591295E-3,1.162461E6,8.226878E-3,1.6642828E-3,3.91101E5,2.5994773E-3,3.646166E6,-2.6566945E-3,1.2388675E-3,-1.2102257E-3,-8.9356425E-4,1.0790093E-3,-2.9429408E-5,1.9185381E-3],"split_indices":[53,90,30,49,29,55,55,0,4,0,9,0,0,9,0,30,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,2.38E2,3E1,2.22E2,1.6E1,1.3E1,1.7E1,4E0,2.18E2,7E0,9E0,4E0,9E0,1.3E1,4E0,2.1E2,8E0,4E0,5E0,9E0,4E0,1.87E2,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.3789119E-3,-3.79684E-3,3.5624779E-3,5.2657295E-3,-2.4627104E-2,-6.0390597E-5,5.0961412E-2,-5.7459297E-3,-1.6172167E-2,4.8569096E-3,-4.5848973E-2,7.337332E-2,-4.524266E-4,-4.8507955E-2,-4.11966E-3,-3.8473706E-3,3.6245395E-4,-4.1481538E-4,-5.7594357E-3,1.5623573E-3,4.5447606E-3,-5.057729E-3,-1.2111566E-3,3.8432356E-4,-2.2785233E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1136993E-2,4.9211364E-2,0E0,4.429837E-2,6.489128E-2,3.71483E-2,2.6606128E-2,0E0,2.863999E-2,4.0386137E-2,4.5581926E-2,1.0640584E-2,0E0,2.4349615E-2,3.1522315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,14,14],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,1.3879616E7,3.5624779E-3,1.1152775E7,2.6376405E0,1E0,2.3E1,-5.7459297E-3,1.4E1,4.9759598E1,4E1,2.39869E5,-4.524266E-4,4.68646E5,3.6E1,-3.8473706E-3,3.6245395E-4,-4.1481538E-4,-5.7594357E-3,1.5623573E-3,4.5447606E-3,-5.057729E-3,-1.2111566E-3,3.8432356E-4,-2.2785233E-3],"split_indices":[53,46,0,46,57,82,8,0,3,49,8,12,0,30,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.59E2,7E0,1.81E2,7.8E1,1.63E2,1.8E1,5E0,7.3E1,1.48E2,1.5E1,1.3E1,5E0,1.9E1,5.4E1,4E0,1.44E2,1.1E1,4E0,6E0,7E0,4E0,1.5E1,4.3E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.1883086E-3,5.3948365E-2,-1.2574306E-3,-1.523033E-3,8.32709E-2,-4.680003E-3,7.3944486E-4,6.0291803E-3,1.0908832E-3,-4.9699387E-3,3.1123426E-2,-2.577991E-3,-4.786399E-3,7.080273E-2,1.158934E-2,-3.4626914E-4,1.6499638E-3,4.4658636E-3,4.3701072E-4,-1.4053816E-3,1.6888662E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,-1,7,-1,9,-1,-1,11,13,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[4.857949E-2,4.6136703E-2,5.1395074E-2,0E0,3.3826627E-2,0E0,4.387008E-2,0E0,0E0,5.016561E-2,3.0183397E-2,3.9954428E-2,0E0,1.995761E-2,3.0708361E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,9,9,10,10,11,11,13,13,14,14],"right_children":[2,4,6,-1,8,-1,10,-1,-1,12,14,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[7.4745026E-6,3.1982166E2,9.421523E-5,-1.523033E-3,1.904E3,-4.680003E-3,5.1428344E5,6.0291803E-3,1.0908832E-3,4.8143003E5,5.658968E4,5.707753E9,-4.786399E-3,3.0855308E0,2.238E3,-3.4626914E-4,1.6499638E-3,4.4658636E-3,4.3701072E-4,-1.4053816E-3,1.6888662E-3],"split_indices":[39,4,43,0,0,0,29,0,0,29,34,12,0,55,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.6E1,2.55E2,4E0,1.2E1,4E0,2.51E2,6E0,6E0,2.12E2,3.9E1,2.08E2,4E0,1.2E1,2.7E1,1.85E2,2.3E1,8E0,4E0,1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.6577973E-4,-4.9572866E-3,3.5426162E-2,2.8655713E-3,-7.140143E-3,5.246036E-3,1.3757978E-2,-6.971195E-2,-4.1682194E-3,5.1900104E-2,-5.0334493E-3,-4.4403644E-3,-8.4463606E-4,2.7000222E-2,-8.535775E-3,3.556E-3,7.45164E-4,1.0096858E-2,-2.2752422E-3,2.9915277E-4,2.7605947E-3,-1.0845154E-3,1.4230073E-4,-9.628285E-4,1.071563E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,-1,7,-1,9,11,13,15,17,-1,-1,19,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7563683E-2,3.7261296E-2,5.4575536E-2,0E0,4.508333E-2,0E0,1.961335E-2,1.3334982E-2,3.2084867E-2,6.8936944E-3,1.32583575E-2,0E0,0E0,1.920723E-2,3.604572E-2,0E0,0E0,6.6159507E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17],"right_children":[2,4,6,-1,8,-1,10,12,14,16,18,-1,-1,20,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,3.299865E1,1.4881903E5,2.8655713E-3,1.0742589E-1,5.246036E-3,1.91016E5,1.49724E5,2.8558656E4,8.982092E5,2.011653E7,-4.4403644E-3,-8.4463606E-4,6.2229652E0,2.0718746E8,3.556E-3,7.45164E-4,1.08E2,-2.2752422E-3,2.9915277E-4,2.7605947E-3,-1.0845154E-3,1.4230073E-4,-9.628285E-4,1.071563E-3],"split_indices":[53,53,34,0,59,0,10,12,34,34,33,0,0,57,7,0,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.5E2,3.2E1,7E0,2.43E2,6E0,2.6E1,1E1,2.33E2,8E0,1.8E1,6E0,4E0,2.8E1,2.05E2,4E0,4E0,1.4E1,4E0,1.8E1,1E1,8.9E1,1.16E2,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.9935807E-3,-3.381235E-3,3.906448E-3,4.2769965E-3,1.9144501E-3,-2.1533134E-4,3.7737158E-3,-7.6527786E-3,1.828326E-2,-2.5807107E-3,-1.2574975E-5,1.1034271E-3,-2.3946678E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,-1,3,-1,5,7,-1,9,11,-1,-1,-1,-1],"loss_changes":[4.083338E-2,0E0,4.8570175E-2,0E0,4.6049505E-2,3.646164E-2,0E0,6.749025E-2,3.1606868E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,7,7,8,8],"right_children":[2,-1,4,-1,6,8,-1,10,12,-1,-1,-1,-1],"split_conditions":[5.491887E-2,-3.381235E-3,1.8E1,4.2769965E-3,2.0704236E3,9.821357E-1,3.7737158E-3,2.8490282E8,7.06699E8,-2.5807107E-3,-1.2574975E-5,1.1034271E-3,-2.3946678E-3],"split_indices":[59,0,10,0,53,42,0,5,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,6E0,2.74E2,5E0,2.69E2,2.63E2,6E0,1.88E2,7.5E1,2.4E1,1.64E2,7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[2.6580675E-3,-1.7229673E-3,3.8347915E-2,-3.4149294E-3,9.542251E-5,7.9968974E-2,1.8159931E-3,4.1128984E-3,-2.1833414E-3,7.482641E-3,1.4132778E-3,1.703889E-2,-1.7553951E-3,-5.3416777E-2,9.328564E-5,5.0892515E-5,1.4060145E-3,-1.0593937E-3,-3.6474573E-3,1.336763E-3,-2.982345E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,-1,7,9,11,-1,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2908292E-2,3.228325E-2,4.5469698E-2,0E0,4.899833E-2,5.1957414E-2,1.0986385E-2,0E0,2.741525E-2,0E0,0E0,2.7624494E-3,0E0,6.023675E-3,4.3165687E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,11,11,13,13,14,14],"right_children":[2,4,6,-1,8,10,12,-1,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,5.447539E-2,2.45226E6,-3.4149294E-3,1.8E1,3.0256798E0,6.236639E3,4.1128984E-3,8.89E2,7.482641E-3,1.4132778E-3,1.4196E4,-1.7553951E-3,6.078693E1,2.83E2,5.0892515E-5,1.4060145E-3,-1.0593937E-3,-3.6474573E-3,1.336763E-3,-2.982345E-4],"split_indices":[53,59,30,0,10,55,4,0,2,0,0,2,0,53,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.44E2,2.9E1,5E0,2.39E2,1.3E1,1.6E1,5E0,2.34E2,4E0,9E0,1.2E1,4E0,9E0,2.25E2,6E0,6E0,5E0,4E0,4.1E1,1.84E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Filter","sqlOp_GenerateBloomFilter","sqlOp_HashAggregate","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalTableScan","sqlOp_OutputAdapter","sqlOp_Project","sqlOp_ReusedSort","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_emr","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_onprem","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Generate","sqlOp_GlobalLimit","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_PartialWindow","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_TakeOrderedAndProject"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"97"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[2.1085903E-2,-2.2772919E-1,4.3721128E-1,-3.6880127E-1,-1.9833224E-3,2.2673543E-1,8.4530383E-1,-4.9474227E-1,-1.5492713E-1,-2.1003313E-1,9.9773996E-2,9.0155266E-2,3.040459E-1,5.9119016E-1,1.0942323E0,-4.0589273E-1,-7.4147564E-1,2.5695986E-3,-2.9754654E-1,1.3038836E-3,-2.6457384E-1,1.7128408E-1,-1.7006904E-2,2.3424982E-1,-1.2763274E-1,3.7088796E-1,1.0321534E-2,2.9997287E-2,1.344041E-2,5.8344405E-2,2.7380887E-2,-2.0452904E-2,1.2894061E-3,-3.7609708E-2,-1.8979337E-2,-5.047128E-3,6.992408E-3,-1.9480048E-2,-6.67327E-3,-1.9714128E-2,-8.620859E-3,-1.1689346E-3,9.905409E-3,-3.9632274E-3,4.7806343E-3,1.5076465E-2,4.6091946E-3,-1.07842935E-2,-1.5311907E-3,1.887391E-2,2.5142783E-3,5.6988043E-3,-4.8434185E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.795136E1,5.381317E0,8.483597E0,2.7246437E0,1.4182148E0,6.9060564E-1,1.503809E0,1.1239796E0,8.992482E-1,2.9349154E-1,3.7870854E-1,8.458316E-1,8.403976E-1,1.5292072E-1,6.2901115E-1,8.442049E-1,6.0456276E-2,3.516622E-1,3.4104168E-1,0E0,1.4756632E-1,2.4698633E-1,1.5918228E-1,1.6726196E-1,1.0287437E-1,4.098711E-1,1.3097732E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,1.1306552E-1,2.0224316E-1,1.9462195E2,2.176974E8,4.638253E2,1.4968063E3,5.039375E-3,1.5248811E1,4.0072045E0,1.47615E7,1.1936242E7,6.191915E0,1.3589149E6,2.65826E6,7.725518E-4,4.214E3,6.135367E9,1.281E3,1.3038836E-3,8E0,6.3972777E4,3.643059E5,9E0,1.1E1,1.1107071E7,3.7808718E11,2.9997287E-2,1.344041E-2,5.8344405E-2,2.7380887E-2,-2.0452904E-2,1.2894061E-3,-3.7609708E-2,-1.8979337E-2,-5.047128E-3,6.992408E-3,-1.9480048E-2,-6.67327E-3,-1.9714128E-2,-8.620859E-3,-1.1689346E-3,9.905409E-3,-3.9632274E-3,4.7806343E-3,1.5076465E-2,4.6091946E-3,-1.07842935E-2,-1.5311907E-3,1.887391E-2,2.5142783E-3,5.6988043E-3,-4.8434185E-3],"split_indices":[71,58,57,71,7,71,4,57,75,73,51,64,76,66,66,57,10,5,0,0,3,47,47,3,3,9,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.68E2,1E2,1.03E2,6.5E1,6.7E1,3.3E1,6.4E1,3.9E1,2.1E1,4.4E1,2.5E1,4.2E1,1.8E1,1.5E1,4.9E1,1.5E1,1.9E1,2E1,4E0,1.7E1,2.7E1,1.7E1,1.5E1,1E1,3.4E1,8E0,1.4E1,4E0,1E1,5E0,4.5E1,4E0,1.1E1,4E0,1.1E1,8E0,1E1,1E1,4E0,1.3E1,5E0,2.2E1,1.1E1,6E0,8E0,7E0,4E0,6E0,3E1,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[8.05193E-3,-1.8821597E-1,4.5129177E-1,-3.3232948E-1,2.6672075E-2,2.0962085E-1,8.257544E-1,-4.4819686E-1,-1.48785E-1,-5.948454E-2,1.8264762E-1,-2.3600906E-3,2.6519588E-1,2.0474736E-2,8.9396125E-1,-1.0349917E-1,-5.107083E-1,4.05704E-4,-2.9799333E-1,-1.6677873E-1,8.952956E-2,2.4102871E-1,-1.2398086E-3,-9.103933E-3,7.617095E-3,3.3745804E-1,3.086987E-2,2.5632085E-2,9.841587E-1,-1.0999542E-2,1.2592459E-3,-2.6992962E-2,-1.3329538E-2,-4.1856277E-3,8.950496E-3,-1.8250458E-2,-6.005194E-3,-1.0323307E-2,2.4856112E-3,1.14772655E-2,1.849914E-3,3.6762117E-3,1.2542095E-2,2.5272542E-3,1.785252E-2,6.5926206E-3,-3.986938E-3,5.670145E-2,3.481712E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,37,39,41,-1,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4095217E1,5.954771E0,7.519127E0,2.3789997E0,1.059054E0,6.250105E-1,4.6012306E-1,1.4554701E0,1.0241038E0,8.3033735E-1,3.4907818E-1,4.2612413E-1,7.0076513E-1,0E0,2.733841E-1,2.1915147E-1,8.165121E-1,4.4283566E-1,3.275318E-1,3.9253372E-1,1.6342817E-1,8.95313E-2,0E0,0E0,0E0,4.3348217E-1,1.5753034E-1,0E0,1.9645119E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,38,40,42,-1,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.797104E2,2.1619815E-1,2.0224316E-1,1.9462195E2,2.2599022E8,4.638253E2,2.3616267E8,1.1266387E10,1.5248811E1,1.0289595E6,1.4E1,2.4223125E5,2.1030048E10,2.0474736E-2,2.2324745E5,4.5474484E-3,1.021924E6,6.135367E9,3.03E9,9.74979E6,1.0632315E6,3.0933383E-1,-1.2398086E-3,-9.103933E-3,7.617095E-3,7.5E2,3.7578902E0,2.5632085E-2,2.9898608E-1,-1.0999542E-2,1.2592459E-3,-2.6992962E-2,-1.3329538E-2,-4.1856277E-3,8.950496E-3,-1.8250458E-2,-6.005194E-3,-1.0323307E-2,2.4856112E-3,1.14772655E-2,1.849914E-3,3.6762117E-3,1.2542095E-2,2.5272542E-3,1.785252E-2,6.5926206E-3,-3.986938E-3,5.670145E-2,3.481712E-2],"split_indices":[71,60,57,71,12,71,7,50,75,52,3,47,5,0,47,58,48,5,5,9,67,61,0,0,0,0,76,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.91E2,8.4E1,1.14E2,7.7E1,5.2E1,3.2E1,6.9E1,4.5E1,5E1,2.7E1,1.1E1,4.1E1,6E0,2.6E1,1.1E1,5.8E1,2.3E1,2.2E1,2.9E1,2.1E1,2.1E1,6E0,5E0,6E0,3.1E1,1E1,7E0,1.9E1,5E0,6E0,4.2E1,1.6E1,1.6E1,7E0,1.3E1,9E0,2.3E1,6E0,4E0,1.7E1,4E0,1.7E1,5E0,2.6E1,5E0,5E0,7E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[3.3098315E-3,-2.2334237E-1,3.5490754E-1,-3.068199E-1,6.591925E-2,1.780472E-1,7.461082E-1,-3.6551002E-1,-3.4820978E-2,-7.2370075E-2,1.0520874E-1,5.880241E-2,2.6542E-1,1.3290277E-2,8.581645E-1,-4.3094838E-1,-1.8536793E-1,1.3821995E-1,-1.478602E-1,-1.4917747E-3,-4.5058155E-3,1.4362985E-1,-5.3233947E-3,1.5904938E-1,-1.5615053E-1,3.257397E-1,1.3522787E-3,2.1919698E-2,4.4838067E-2,-1.6532997E-2,-2.9766759E-2,-4.3412945E-3,-1.78364E-2,1.1596651E-2,-1.1883961E-3,6.404258E-4,-1.0595716E-2,3.7644408E-4,8.464975E-3,1.3952501E-2,2.8567943E-3,-1.2937324E-2,2.295715E-3,1.8079627E-2,5.48387E-3,4.8313425E-3,-4.7192737E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,35,-1,-1,37,-1,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1755054E1,4.0286016E0,7.27026E0,2.0465841E0,2.1408E-1,7.6579905E-1,1.4438534E0,1.1696653E0,4.8988712E-1,6.000217E-3,2.7114466E-1,7.3593223E-1,6.844847E-1,0E0,6.8577385E-1,9.6060944E-1,5.1548564E-1,1.9293371E-1,2.028482E-1,0E0,0E0,1.4016885E-1,0E0,3.024519E-1,2.9692715E-1,4.3738198E-1,1.07573576E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,21,21,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,36,-1,-1,38,-1,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1309677E2,4.9615598E-1,1.5972575E0,2.727992E6,2.0358758E8,4.638253E2,2.744E3,1.1670399E-1,1.6E1,2.0074467E5,1.1771224E8,1.409674E7,6.191915E0,1.3290277E-2,2.2324745E5,3.98231E-2,3.4047124E-1,1.9096774E0,3.1791415E11,-1.4917747E-3,-4.5058155E-3,1.95E2,-5.3233947E-3,9E0,2.136E3,1.8751362E7,3.7808718E11,2.1919698E-2,4.4838067E-2,-1.6532997E-2,-2.9766759E-2,-4.3412945E-3,-1.78364E-2,1.1596651E-2,-1.1883961E-3,6.404258E-4,-1.0595716E-2,3.7644408E-4,8.464975E-3,1.3952501E-2,2.8567943E-3,-1.2937324E-2,2.295715E-3,1.8079627E-2,5.48387E-3,4.8313425E-3,-4.7192737E-3],"split_indices":[71,58,60,1,7,71,2,58,3,47,51,64,76,0,47,61,61,72,50,0,0,0,0,3,0,63,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.65E2,1.06E2,1.28E2,3.7E1,7.4E1,3.2E1,1.05E2,2.3E1,8E0,2.9E1,3.2E1,4.2E1,7E0,2.5E1,7.6E1,2.9E1,9E0,1.4E1,4E0,4E0,2.5E1,4E0,2.2E1,1E1,3.4E1,8E0,7E0,1.8E1,5.9E1,1.7E1,2.1E1,8E0,5E0,4E0,5E0,9E0,6E0,1.9E1,8E0,1.4E1,6E0,4E0,2.5E1,9E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-2.1728342E-3,-1.34235E-1,5.659846E-1,-3.1162438E-1,5.227209E-2,2.7925435E-1,9.2783916E-1,-3.8934636E-1,-1.0033659E-2,-6.283367E-2,1.5673715E-1,2.0447267E-2,4.205759E-1,2.1932395E-2,4.594042E-2,-3.496029E-1,-3.150391E-2,-7.901294E-2,1.0201296E-2,-1.6959472E-1,3.504982E-2,2.0324224E-1,6.772725E-3,3.7339362E-3,-2.4734316E-3,1.1890882E-2,2.2472931E-2,-2.0664664E-2,-1.0800635E-2,3.0884955E-3,-8.379436E-3,-4.7979713E-3,-1.5293842E-2,7.199374E-3,-2.7525688E-3,1.2581257E-2,2.0711978E-3,-7.47909E-3,3.8247998E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1084963E1,7.559527E0,5.239893E0,2.7386007E0,1.3565956E0,1.1252258E0,2.5128937E-1,8.508129E-1,4.169024E-1,5.70252E-1,4.1009593E-1,5.8340065E-2,9.315419E-2,0E0,0E0,8.5338783E-1,0E0,3.131229E-1,0E0,2.4177301E-1,3.4300086E-1,4.8635304E-1,2.0716569E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,1.7613442E-1,7.2610445E8,2.727992E6,3.614121E8,2.6801108E2,2.2324745E5,2.771097E0,1.22E2,6.75553E5,2.3601532E0,1.2495134E6,1.9215278E5,2.1932395E-2,4.594042E-2,1.9268292E0,-3.150391E-2,4.7E3,1.0201296E-2,2.32E3,4.045147E0,2.3103538E7,2.697134E8,3.7339362E-3,-2.4734316E-3,1.1890882E-2,2.2472931E-2,-2.0664664E-2,-1.0800635E-2,3.0884955E-3,-8.379436E-3,-4.7979713E-3,-1.5293842E-2,7.199374E-3,-2.7525688E-3,1.2581257E-2,2.0711978E-3,-7.47909E-3,3.8247998E-3],"split_indices":[61,60,12,1,12,4,47,72,8,9,72,64,47,0,0,72,0,0,0,2,73,64,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,2.27E2,5.2E1,1.16E2,1.11E2,3E1,2.2E1,9.2E1,2.4E1,5.3E1,5.8E1,1.1E1,1.9E1,4E0,1.8E1,8.3E1,9E0,1.9E1,5E0,2.5E1,2.8E1,4.4E1,1.4E1,6E0,5E0,7E0,1.2E1,4.3E1,4E1,8E0,1.1E1,1.9E1,6E0,1.2E1,1.6E1,3E1,1.4E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-9.496169E-3,-1.7978296E-1,4.131077E-1,-3.0226317E-1,5.437915E-3,1.7675185E-1,7.5473183E-1,-3.7400904E-1,-2.0408144E-2,-8.356252E-2,1.2569034E-1,8.318677E-2,3.424772E-1,1.5751595E-2,8.565782E-1,-4.3764114E-1,-2.0838907E-1,-8.013996E-2,6.6592176E-3,-2.0129164E-1,4.29184E-2,-1.5173384E-2,1.9371966E-1,2.4396244E-1,-2.5476754E-2,2.6302315E-2,1.2518758E-1,4.846919E-2,2.8879559E-2,-1.0297241E-2,-2.3558494E-2,3.8563093E-4,-1.5160369E-2,4.851225E-3,-6.290731E-3,-1.1396859E-2,1.1025744E-3,9.115021E-3,-1.7169984E-3,2.3040317E-3,-5.363087E-3,1.1058021E-2,1.0156563E-3,5.296566E-3,1.5563398E-2,-1.3192813E-2,2.656533E-3,8.729507E-4,8.8788215E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,37,39,41,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9792366E1,4.448078E0,6.2454424E0,2.3785553E0,8.560019E-1,7.158036E-1,1.0516815E0,8.994427E-1,2.5805157E-1,6.936016E-1,3.2875675E-1,5.6526977E-1,7.736615E-1,0E0,4.642582E-1,9.4713306E-1,7.364912E-1,2.1351448E-1,0E0,2.559178E-1,2.9344204E-1,8.511527E-2,1.7812508E-1,1.18774116E-1,4.5082474E-1,0E0,6.821506E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,38,40,42,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.797104E2,2.1619815E-1,2.0224316E-1,2.219883E6,2.7538432E8,7.230905E2,2.3616267E8,1.9462195E2,1E0,4.343271E5,1.571112E5,1.2269897E3,9.521E3,1.5751595E-2,2.9898608E-1,2.4747229E2,1.2288136E1,2.207E3,6.6592176E-3,1.1709996E0,8.237578E4,1.2315645E5,2.8808496E7,4.50805E-1,1.3548306E3,2.6302315E-2,2.2894168E0,4.846919E-2,2.8879559E-2,-1.0297241E-2,-2.3558494E-2,3.8563093E-4,-1.5160369E-2,4.851225E-3,-6.290731E-3,-1.1396859E-2,1.1025744E-3,9.115021E-3,-1.7169984E-3,2.3040317E-3,-5.363087E-3,1.1058021E-2,1.0156563E-3,5.296566E-3,1.5563398E-2,-1.3192813E-2,2.656533E-3,8.729507E-4,8.8788215E-3],"split_indices":[71,60,57,1,7,71,7,71,30,66,47,4,2,0,57,74,75,0,0,58,47,47,9,76,4,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,1.95E2,7.8E1,1.17E2,7.8E1,4.7E1,3.1E1,9.3E1,2.4E1,4.5E1,3.3E1,3.1E1,1.6E1,7E0,2.4E1,6.6E1,2.7E1,1.8E1,6E0,2.3E1,2.2E1,1.1E1,2.2E1,1.2E1,1.9E1,7E0,9E0,1.1E1,1.3E1,1.8E1,4.8E1,1E1,1.7E1,4E0,1.4E1,1.9E1,4E0,7E0,1.5E1,7E0,4E0,1.7E1,5E0,6E0,6E0,4E0,1.5E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[-8.5547956E-4,-1.5124227E-1,4.1323495E-1,-2.9142067E-1,2.8436407E-2,1.8857329E-1,7.1153224E-1,-3.9878193E-1,-1.07372575E-1,-8.622849E-2,9.1209516E-2,2.3138136E-1,-5.247452E-3,1.2566412E-2,7.8537935E-1,-1.9025414E-1,-4.6601295E-1,1.00603355E-2,-3.0616936E-1,-1.491725E-1,7.3611313E-3,1.2469521E-1,-6.3201636E-2,3.185047E-2,3.0150485E-1,5.1404282E-2,6.302224E-1,-1.228212E-2,3.5602557E-3,-7.937771E-3,-2.2698479E-2,-2.6173724E-3,8.491763E-3,-1.7663874E-2,-8.320865E-3,-2.4596206E-4,-1.0359407E-2,3.0030967E-3,9.118384E-3,2.6758283E-3,-6.483959E-3,-9.433184E-3,9.042378E-3,2.0103846E-2,6.223371E-3,1.650125E-2,3.111349E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7063227E1,5.0708885E0,4.757988E0,2.1900196E0,6.4847547E-1,5.8328295E-1,7.896948E-1,9.067278E-1,1.0069275E0,5.195675E-1,3.0777368E-1,5.220854E-1,0E0,0E0,6.32082E-1,4.0961158E-1,3.3503342E-1,3.3717495E-1,8.6182475E-2,2.8237534E-1,0E0,1.964702E-1,1.11781254E-1,4.6873865E-1,5.7040143E-1,0E0,5.452156E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,2.4210773E-2,2.0224316E-1,1.7785811E2,2.4118989E8,9.456436E0,1.7246376E0,2.5041176E2,1.9587677E1,7.004062E5,2.835443E0,3.386E3,-5.247452E-3,1.2566412E-2,3.0485725E0,1.0665042E8,1.1537334E0,6.87E3,1.281E3,2.8507206E-1,7.3611313E-3,2.2599022E8,3.7741747E8,1.4E1,3.5630266E8,5.1404282E-2,6.4495115E0,-1.228212E-2,3.5602557E-3,-7.937771E-3,-2.2698479E-2,-2.6173724E-3,8.491763E-3,-1.7663874E-2,-8.320865E-3,-2.4596206E-4,-1.0359407E-2,3.0030967E-3,9.118384E-3,2.6758283E-3,-6.483959E-3,-9.433184E-3,9.042378E-3,2.0103846E-2,6.223371E-3,1.650125E-2,3.111349E-2],"split_indices":[71,57,57,71,7,76,72,74,75,66,72,2,0,0,73,7,73,10,0,61,0,12,7,3,7,0,77,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2E2,7.2E1,1.12E2,8.8E1,4.2E1,3E1,7E1,4.2E1,3.1E1,5.7E1,3.7E1,5E0,5E0,2.5E1,1.8E1,5.2E1,2.7E1,1.5E1,2.5E1,6E0,4.7E1,1E1,1E1,2.7E1,6E0,1.9E1,1.4E1,4E0,5E0,4.7E1,2E1,7E0,8E0,7E0,9E0,1.6E1,2.7E1,2E1,4E0,6E0,4E0,6E0,1.4E1,1.3E1,4E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[1.5965978E-2,-1.6973527E-1,3.3239034E-1,-2.8115204E-1,-2.5269142E-4,1.3585745E-1,6.358597E-1,-3.8342574E-1,-9.037479E-2,-7.975257E-2,6.9061376E-2,-1.0452388E-2,1.96517E-1,3.9780077E-1,7.723055E-1,-1.1247442E-1,-4.3754542E-1,8.296243E-2,-2.4290015E-1,-1.351644E-1,6.675223E-2,1.4069478E-1,-2.2327319E-2,1.33327E-1,-1.1178186E-2,2.961138E-1,5.2235503E-2,2.0015052E-2,1.0232433E-2,1.873837E-2,8.917609E-1,-1.10272635E-2,1.4101589E-3,-2.318691E-2,-9.879268E-3,-4.7807943E-4,8.648084E-3,-1.6017655E-2,-5.3324327E-3,-1.2605592E-3,-7.723888E-3,-5.3495535E-4,6.788633E-3,-3.0176472E-3,9.448006E-3,-3.1434821E-3,3.7269583E-3,1.0321122E-2,-2.6178758E-3,1.9719439E-2,7.449925E-3,8.453698E-3,-1.5023103E-3,4.7617476E-2,2.9206924E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,-1,-1,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6685612E1,3.3800216E0,6.175993E0,2.0725899E0,4.0226096E-1,5.78935E-1,1.0245438E0,9.772539E-1,1.0512462E0,2.8503782E-1,2.584542E-1,7.004698E-1,6.455511E-1,1.3573647E-2,7.280245E-1,2.4366336E-1,7.463503E-1,1.8794858E-1,2.3991823E-1,8.3242476E-2,6.495506E-2,3.061618E-1,9.094318E-2,2.3735417E-1,0E0,4.1003275E-1,2.2897764E-1,0E0,0E0,0E0,4.0273666E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,-1,-1,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,2.4210773E-2,1.8829846E-1,1.9462195E2,2.4118989E8,3.165913E6,3.856E3,1.1266387E10,1.5248811E1,7.598717E5,3.2744168E6,1.1936242E7,3.3996282E0,1.3589149E6,2.174459E5,6.846054E7,6.608892E2,6.135367E9,1.281E3,3.3529866E-2,1.0289595E6,6E0,4.083633E6,2.3797293E0,-1.1178186E-2,7.5547155E6,3.7808718E11,2.0015052E-2,1.0232433E-2,1.873837E-2,2.9898608E-1,-1.10272635E-2,1.4101589E-3,-2.318691E-2,-9.879268E-3,-4.7807943E-4,8.648084E-3,-1.6017655E-2,-5.3324327E-3,-1.2605592E-3,-7.723888E-3,-5.3495535E-4,6.788633E-3,-3.0176472E-3,9.448006E-3,-3.1434821E-3,3.7269583E-3,1.0321122E-2,-2.6178758E-3,1.9719439E-2,7.449925E-3,8.453698E-3,-1.5023103E-3,4.7617476E-2,2.9206924E-2],"split_indices":[71,57,57,71,7,1,2,50,75,66,69,64,76,66,47,7,4,5,0,57,52,3,70,72,0,51,50,0,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.78E2,1.04E2,1.07E2,7.1E1,6.4E1,4E1,6.9E1,3.8E1,3.3E1,3.8E1,1.9E1,4.5E1,1.6E1,2.4E1,1.2E1,5.7E1,1.8E1,2E1,2.4E1,9E0,2.1E1,1.7E1,1.2E1,7E0,2.6E1,1.9E1,1.2E1,4E0,7E0,1.7E1,6E0,6E0,4.3E1,1.4E1,1E1,8E0,1E1,1E1,6E0,1.8E1,5E0,4E0,5E0,1.6E1,1.2E1,5E0,8E0,4E0,1.2E1,1.4E1,7E0,1.2E1,9E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[5.995876E-4,-1.7347676E-1,3.2858977E-1,-2.772065E-1,4.059655E-3,1.5712646E-1,6.180375E-1,-3.4299368E-1,-1.8954381E-2,-1.0166349E-1,9.792302E-2,3.0239883E-1,1.03181146E-1,8.146918E-1,4.800139E-1,-3.062804E-1,-2.7904196E-2,8.940504E-2,-1.0274004E-1,-2.1722323E-1,8.394098E-2,-3.0153783E-3,1.3962671E-1,1.4694941E-1,2.215816E-2,1.7923576E-1,-8.34605E-2,4.802176E-2,2.7501302E-2,1.03110215E-2,5.2213454E-1,-1.9017814E-2,-9.947347E-3,-2.028609E-3,9.575163E-3,-8.734323E-3,2.0642108E-3,2.2078224E-4,-1.2570478E-2,-9.5630065E-4,9.648653E-3,8.224101E-3,1.5246839E-3,2.3588135E-3,9.310498E-3,5.70954E-3,1.8742599E-2,-1.0078216E-2,3.8327414E-3,1.3050525E-2,2.6100017E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,-1,41,43,-1,45,47,-1,-1,-1,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5644029E1,3.2975554E0,4.617708E0,1.913332E0,6.7483175E-1,4.4684064E-1,5.58939E-1,6.836338E-1,2.2708063E-1,7.0458287E-1,2.544869E-1,3.741634E-1,6.6792715E-1,2.219677E-2,1.3113213E-1,6.911535E-1,0E0,1.8323165E-1,1.88183E-1,2.4910134E-1,1.7536034E-1,0E0,1.1454445E-1,4.2544305E-2,0E0,3.647622E-1,3.3431494E-1,0E0,0E0,0E0,4.225731E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,-1,42,44,-1,46,48,-1,-1,-1,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.5106274E-2,2.0224316E-1,2.727992E6,9.50379E5,2.3842E4,7.013774E5,2.771097E0,1.6E1,6.4796996E-1,1.8681856E8,7.12911E4,2.4914644E1,2.7479675E-1,2.0787016E5,1.8462508E0,-2.7904196E-2,1.04783256E6,3.9E1,3.3529866E-2,1.5130641E0,-3.0153783E-3,1.1225629E6,1.5552E4,2.215816E-2,7.230905E2,1.4813267E3,4.802176E-2,2.7501302E-2,1.03110215E-2,2.744E3,-1.9017814E-2,-9.947347E-3,-2.028609E-3,9.575163E-3,-8.734323E-3,2.0642108E-3,2.2078224E-4,-1.2570478E-2,-9.5630065E-4,9.648653E-3,8.224101E-3,1.5246839E-3,2.3588135E-3,9.310498E-3,5.70954E-3,1.8742599E-2,-1.0078216E-2,3.8327414E-3,1.3050525E-2,2.6100017E-2],"split_indices":[71,57,57,1,9,9,52,72,3,58,7,47,75,57,47,72,0,51,3,57,75,0,52,9,0,71,74,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.78E2,9.4E1,1.12E2,6.6E1,6E1,3.4E1,8.9E1,2.3E1,3.1E1,3.5E1,1.5E1,4.5E1,1.2E1,2.2E1,8E1,9E0,1E1,1.3E1,1.9E1,1.2E1,7E0,2.8E1,9E0,6E0,3.2E1,1.3E1,4E0,8E0,4E0,1.8E1,3.5E1,4.5E1,5E0,5E0,8E0,5E0,4E0,1.5E1,7E0,5E0,2E1,8E0,4E0,5E0,2.7E1,5E0,7E0,6E0,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-2.2932042E-4,-1.6076626E-1,2.791689E-1,-2.4847326E-1,-1.1363867E-2,1.2656887E-1,5.408803E-1,-3.4279287E-1,-8.47643E-2,-9.967158E-2,9.725088E-2,-1.0100473E-1,1.6826609E-1,3.3819383E-1,6.583631E-1,-1.17292136E-1,-3.9049146E-1,5.0314553E-2,-2.1599023E-1,-1.7354113E-1,7.484087E-2,1.546897E-2,1.9545256E-1,5.5201887E-3,-1.4045857E-2,2.4351569E-1,7.563631E-2,1.7509794E-2,7.803059E-3,1.6527753E-2,3.4204446E-2,-1.0716689E-2,3.47444E-3,-1.9195572E-2,-9.966454E-4,-1.5373593E-3,7.094945E-3,-1.3927387E-2,-4.6953233E-3,-1.2894858E-3,-1.0270729E-2,-9.854082E-4,9.84646E-3,4.65181E-3,-2.217704E-3,1.0614773E-2,3.9672107E-3,7.59205E-3,2.0669263E-2,1.4315463E-2,3.6960765E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2783342E1,2.3689346E0,4.0850067E0,1.7312126E0,6.619165E-1,6.527709E-1,6.8220806E-1,7.286358E-1,7.7307886E-1,5.0285196E-1,2.4594131E-1,5.3244543E-1,3.7889147E-1,4.9229503E-2,3.1550026E-1,3.2917523E-1,5.661726E-1,1.962696E-1,1.8613267E-1,1.8840152E-1,1.6513681E-1,1.0288088E-1,3.3833563E-2,0E0,0E0,4.2660427E-1,4.2701286E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,1.7613442E-1,1.8829846E-1,1.9462195E2,9.50379E5,7.5E2,3.301E3,2.431768E2,1.5248811E1,1.2185057E0,2.052E3,3.9378E5,3.7724147E8,5.4374854E2,2.2324745E5,1.081E3,9.4983295E2,6.135367E9,9.877899E2,1E0,1.7518893E2,4.045147E0,3.174073E6,5.5201887E-3,-1.4045857E-2,7.230905E2,1.897231E0,1.7509794E-2,7.803059E-3,1.6527753E-2,3.4204446E-2,-1.0716689E-2,3.47444E-3,-1.9195572E-2,-9.966454E-4,-1.5373593E-3,7.094945E-3,-1.3927387E-2,-4.6953233E-3,-1.2894858E-3,-1.0270729E-2,-9.854082E-4,9.84646E-3,4.65181E-3,-2.217704E-3,1.0614773E-2,3.9672107E-3,7.59205E-3,2.0669263E-2,1.4315463E-2,3.6960765E-4],"split_indices":[71,60,57,71,9,0,2,74,75,60,2,1,7,71,47,2,4,5,4,29,71,73,1,0,0,71,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.8E2,1.03E2,1.13E2,6.7E1,6.6E1,3.7E1,7.1E1,4.2E1,3.7E1,3E1,1E1,5.6E1,1.5E1,2.2E1,1.3E1,5.8E1,2.1E1,2.1E1,2.6E1,1.1E1,1.7E1,1.3E1,5E0,5E0,3E1,2.6E1,1.1E1,4E0,6E0,1.6E1,8E0,5E0,5.4E1,4E0,1.2E1,9E0,1.1E1,1E1,7E0,1.9E1,7E0,4E0,7E0,1E1,9E0,4E0,2.3E1,7E0,5E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[-8.568767E-3,-1.8477075E-1,2.4255592E-1,-2.5562903E-1,4.3443054E-2,1.047099E-1,5.0142497E-1,-3.0027148E-1,-2.4495613E-2,-8.791906E-2,9.818956E-2,-1.6025668E-1,1.5275505E-1,2.9869998E-1,6.2946415E-1,-1.4635389E-1,-3.379046E-1,-1.0504028E-1,7.460755E-2,-5.6306818E-3,-7.058365E-4,1.6652746E-1,6.528328E-3,3.9131995E-3,-1.351333E-2,1.9199795E-1,1.1761697E-2,1.5748665E-2,8.650935E-3,1.6605457E-2,3.335774E-2,-1.3596179E-4,-1.2228896E-2,-2.0160748E-2,-9.96515E-3,-9.779533E-4,-1.0042071E-2,8.086266E-3,-2.833499E-3,1.4913988E-3,1.0338657E-2,-3.4182414E-3,5.2088927E-3,3.4956157E-3,1.3262765E-2,5.795708E-3,-2.6651435E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,-1,-1,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2124703E1,2.6146097E0,3.9797955E0,1.2618036E0,2.891058E-1,9.792569E-1,8.242922E-1,5.47163E-1,1.7546366E-1,2.807039E-2,1.7430308E-1,4.1709977E-1,3.5185313E-1,1.7398953E-2,3.2636833E-1,3.7544218E-1,9.010458E-1,1.09070435E-1,1.4835364E-1,0E0,0E0,1.172578E-1,1.2034082E-1,0E0,0E0,5.3566706E-1,1.2650971E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,-1,-1,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1309677E2,7.087292E-2,1.2711891E0,2.727992E6,2.0358758E8,7.5E2,3.301E3,2.431768E2,4.8E1,1.1709996E0,2.7352352E6,4.6557466E8,6.191915E0,1.5E1,2.174459E5,6.7897725E-1,5.283708E0,1.4618938E0,4.654E3,-5.6306818E-3,-7.058365E-4,7E0,2.3581433E5,3.9131995E-3,-1.351333E-2,4.6020203E2,3.7808718E11,1.5748665E-2,8.650935E-3,1.6605457E-2,3.335774E-2,-1.3596179E-4,-1.2228896E-2,-2.0160748E-2,-9.96515E-3,-9.779533E-4,-1.0042071E-2,8.086266E-3,-2.833499E-3,1.4913988E-3,1.0338657E-2,-3.4182414E-3,5.2088927E-3,3.4956157E-3,1.3262765E-2,5.795708E-3,-2.6651435E-3],"split_indices":[71,57,58,1,7,0,2,74,8,58,67,5,76,3,47,75,75,72,2,0,0,3,47,0,0,71,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.6E2,1.12E2,1.22E2,3.8E1,7.4E1,3.8E1,1.02E2,2E1,1.1E1,2.7E1,1.1E1,6.3E1,1.6E1,2.2E1,2.1E1,8.1E1,1.1E1,9E0,7E0,4E0,1.5E1,1.2E1,4E0,7E0,4.9E1,1.4E1,1E1,6E0,7E0,1.5E1,1E1,1.1E1,4.3E1,3.8E1,7E0,4E0,5E0,4E0,5E0,1E1,7E0,5E0,2.3E1,2.6E1,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-3.596769E-4,-1.8368325E-1,2.3429736E-1,-2.9083514E-1,-5.836663E-2,1.02255315E-1,4.439845E-1,-3.3659607E-1,-5.179366E-2,4.2336747E-2,-1.5753405E-1,-2.6449973E-2,1.5329044E-1,1.4937297E-1,6.157723E-1,-2.637285E-1,-5.2597773E-1,2.6845357E-3,-1.31673E-1,-3.3140503E-2,1.6027789E-1,-2.607038E-1,-6.1601207E-2,8.009401E-3,-7.77979E-2,1.7347395E-1,-4.0182974E-3,-1.3884074E-4,2.5812146E-1,4.0030707E-2,5.007615E-1,1.7797025E-3,-1.42537E-2,-1.1270505E-2,-2.691749E-2,-7.8072743E-3,-3.1049799E-3,6.30815E-3,-3.9194035E-3,1.1555192E-2,4.810408E-3,-4.7462294E-3,-1.5235984E-2,-4.9033207E-3,4.759315E-3,-5.4215686E-3,-3.746539E-4,5.159625E-3,1.0051347E-2,2.0515637E-3,-2.0630697E-3,7.5246883E-3,1.5610643E-2,2.460694E-2,1.254935E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,-1,45,47,-1,49,51,-1,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2260195E1,2.1314907E0,3.4155402E0,9.232993E-1,7.5565654E-1,5.163967E-1,2.3373318E0,8.5486984E-1,1.3959955E-1,3.4358284E-1,3.6004037E-1,2.5022107E-1,2.870475E-1,3.0896032E-1,4.399023E-1,7.6077175E-1,1.7029333E-1,0E0,1.0473698E-2,2.2396392E-1,4.968801E-2,1.6510177E-1,1.6655323E-1,0E0,5.1927097E-2,1.1871493E-1,0E0,1.9966932E-2,2.888751E-2,0E0,3.1169415E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,24,24,25,25,27,27,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,-1,46,48,-1,50,52,-1,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5300907E-1,1.9462195E2,1.5972575E0,1.848519E-2,1.560615E1,3.9324117E0,2.744E3,5.1653096E-3,1E0,3.165913E6,3.919E3,1.0348846E6,9.723711E0,6.765595E7,2.7479675E-1,1.5166431E5,3.12986E10,2.6845357E-3,8.138103E1,2.8558656E4,3.995065E6,3.2829E4,4.2709158E2,8.009401E-3,3.2826756E5,4.876025E2,-4.0182974E-3,2.0356708E2,1.2984486E6,4.0030707E-2,2.505319E0,1.7797025E-3,-1.42537E-2,-1.1270505E-2,-2.691749E-2,-7.8072743E-3,-3.1049799E-3,6.30815E-3,-3.9194035E-3,1.1555192E-2,4.810408E-3,-4.7462294E-3,-1.5235984E-2,-4.9033207E-3,4.759315E-3,-5.4215686E-3,-3.746539E-4,5.159625E-3,1.0051347E-2,2.0515637E-3,-2.0630697E-3,7.5246883E-3,1.5610643E-2,2.460694E-2,1.254935E-2],"split_indices":[61,71,60,57,75,77,2,57,29,1,10,64,76,5,57,47,50,0,71,52,48,9,71,0,47,71,0,4,52,0,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.59E2,1.24E2,8.5E1,7.4E1,7.7E1,4.7E1,7.1E1,1.4E1,3.7E1,3.7E1,2.2E1,5.5E1,1.8E1,2.9E1,5.3E1,1.8E1,6E0,8E0,2.3E1,1.4E1,1.7E1,2E1,4E0,1.8E1,5.1E1,4E0,8E0,1E1,7E0,2.2E1,7E0,4.6E1,4E0,1.4E1,4E0,4E0,5E0,1.8E1,4E0,1E1,6E0,1.1E1,1.6E1,4E0,1.1E1,7E0,2.3E1,2.8E1,4E0,4E0,6E0,4E0,1.8E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[-1.3580224E-2,-1.3836537E-1,3.4709883E-1,-2.3893131E-1,7.0762103E-3,1.7716682E-1,5.539774E-1,-3.0390304E-1,-8.560234E-2,-1.438354E-1,4.63419E-2,2.626264E-1,5.3378113E-2,1.0326719E-2,6.3731706E-1,-2.6912305E-1,-2.3783604E-2,-1.7900838E-1,-1.690239E-2,-1.31039675E-2,-4.9727768E-2,4.2479597E-3,1.5689185E-1,3.9813077E-1,6.62912E-2,9.279113E-2,-3.1148603E-3,1.52902E-2,7.375629E-1,-1.8078666E-2,-8.070591E-3,-1.4476354E-3,-1.2173431E-2,1.4574539E-3,-7.3554767E-3,2.4368017E-4,-4.8633316E-3,-4.047023E-3,3.300304E-3,3.2097457E-3,9.850741E-3,2.4707617E-2,1.204215E-2,-2.8300765E-3,6.817667E-3,6.8351824E-4,6.878275E-3,4.1859776E-2,2.512445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,-1,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2605088E1,3.0439966E0,2.433033E0,1.1993175E0,5.1627135E-1,4.1971767E-1,7.362766E-1,4.970932E-1,2.408199E-1,2.2431833E-1,3.204239E-1,6.1211026E-1,9.2142E-2,0E0,5.199766E-1,8.1216764E-1,0E0,1.9331014E-1,1.6460419E-1,0E0,3.7468463E-2,3.2321015E-1,7.960236E-2,1.2448859E-1,1.24041766E-1,5.954951E-2,0E0,0E0,9.6570015E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,-1,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,2.0998143E-1,2.0224316E-1,4.95E9,1.8681856E8,6.464779E6,3.301E3,2.771097E0,1.9462195E2,2.29462E5,2.704375E6,4.3147E5,5.615313E3,1.0326719E-2,2.2324745E5,1.8662952E0,-2.3783604E-2,1.335552E0,2.142943E1,-1.31039675E-2,1.5871658E5,8.749103E5,4.152263E0,8.312613E5,1.4E1,4.005725E1,-3.1148603E-3,1.52902E-2,2.9375613E-1,-1.8078666E-2,-8.070591E-3,-1.4476354E-3,-1.2173431E-2,1.4574539E-3,-7.3554767E-3,2.4368017E-4,-4.8633316E-3,-4.047023E-3,3.300304E-3,3.2097457E-3,9.850741E-3,2.4707617E-2,1.204215E-2,-2.8300765E-3,6.817667E-3,6.8351824E-4,6.878275E-3,4.1859776E-2,2.512445E-2],"split_indices":[71,61,57,5,7,48,2,72,71,9,9,9,4,0,47,72,0,73,75,0,47,52,77,66,3,75,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,2.07E2,7.1E1,1.22E2,8.5E1,4E1,3.1E1,8.5E1,3.7E1,1.7E1,6.8E1,2.3E1,1.7E1,7E0,2.4E1,7.5E1,1E1,1.5E1,2.2E1,6E0,1.1E1,5E1,1.8E1,1.3E1,1E1,1.3E1,4E0,7E0,1.7E1,3.1E1,4.4E1,6E0,9E0,1.7E1,5E0,6E0,5E0,2.1E1,2.9E1,8E0,1E1,5E0,8E0,4E0,6E0,6E0,7E0,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[5.877288E-4,-1.2417702E-1,2.9708055E-1,-2.1908262E-1,2.6360616E-2,1.4931536E-1,5.25398E-1,-2.871291E-1,-4.4770997E-2,-2.4798097E-2,1.3785537E-1,2.2076918E-1,6.0456306E-2,7.016789E-1,3.912528E-1,-1.707563E-1,-3.494655E-1,-1.1833131E-1,3.536328E-2,-8.723113E-2,1.6329106E-2,1.6928416E-1,-9.998593E-4,3.5230267E-1,5.4675963E-2,1.4380012E-1,-8.029524E-3,1.6454859E-2,8.149832E-1,4.5519656E-1,7.1606375E-3,-1.2303925E-2,-1.7109883E-3,-2.1576341E-2,-1.0048361E-2,4.879607E-4,-7.0635783E-3,4.498231E-3,-6.7336615E-3,-7.416376E-3,-1.0917642E-3,1.8079918E-3,-5.735647E-3,8.8046165E-3,2.799195E-3,1.9212412E-2,8.031611E-3,-9.944658E-4,4.578838E-3,3.8882762E-3,1.3493867E-2,2.1922933E-2,4.56179E-2,2.3183249E-2,1.1424944E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,-1,45,47,49,-1,-1,51,53,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.061656E1,2.8917227E0,2.789226E0,1.4579449E0,4.5402884E-1,3.231671E-1,5.0582504E-1,5.859556E-1,2.1639389E-1,1.4264768E-1,1.2905523E-1,6.1429524E-1,5.1442254E-1,2.158432E-1,2.4574137E-1,4.1541892E-1,7.8902054E-1,8.757931E-2,2.1807545E-1,9.9379376E-2,1.1479305E-1,3.8379908E-2,0E0,1.1213887E-1,4.9961645E-2,1.4358124E-1,0E0,0E0,6.4380646E-2,3.5182476E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,-1,46,48,50,-1,-1,52,54,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7186426E2,2.1619815E-1,2.0224316E-1,4.95E9,2.427E3,5.4582E4,7.013774E5,9.0649E4,2.0306636E2,1.503E3,4.99E3,6.068019E6,3.6182167E1,2.4214827E5,2.094037E7,1.081E3,6.608892E2,6.035827E9,2.142943E1,8.503368E-1,2.302E3,2.5659466E1,-9.998593E-4,6.5E1,1.74E4,8.834E3,-8.029524E-3,1.6454859E-2,6.886E3,4.9691647E-1,7.1606375E-3,-1.2303925E-2,-1.7109883E-3,-2.1576341E-2,-1.0048361E-2,4.879607E-4,-7.0635783E-3,4.498231E-3,-6.7336615E-3,-7.416376E-3,-1.0917642E-3,1.8079918E-3,-5.735647E-3,8.8046165E-3,2.799195E-3,1.9212412E-2,8.031611E-3,-9.944658E-4,4.578838E-3,3.8882762E-3,1.3493867E-2,2.1922933E-2,4.56179E-2,2.3183249E-2,1.1424944E-2],"split_indices":[71,60,57,5,2,9,52,48,71,2,2,48,75,47,9,2,4,5,75,61,2,75,0,8,9,2,0,0,2,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.85E2,2.01E2,8.4E1,1.23E2,7.8E1,5.2E1,3.2E1,8.8E1,3.5E1,5.4E1,2.4E1,2.8E1,2.4E1,1.2E1,2E1,3.2E1,5.6E1,1.8E1,1.7E1,2.1E1,3.3E1,2E1,4E0,1.5E1,1.3E1,1.8E1,6E0,4E0,8E0,1.5E1,5E0,1.8E1,1.4E1,2.8E1,2.8E1,4E0,1.4E1,1.3E1,4E0,9E0,1.2E1,2.9E1,4E0,1.6E1,4E0,1E1,5E0,5E0,8E0,1.4E1,4E0,4E0,4E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"55","size_leaf_vector":"1"}},{"base_weights":[1.4342601E-2,-1.07165396E-1,3.081849E-1,-1.948525E-1,1.4563464E-2,1.487435E-1,5.134866E-1,-2.6648492E-1,-7.037843E-2,-1.2724888E-1,5.865227E-2,1.9358988E-1,1.8090768E-3,3.6575672E-1,6.8607605E-1,-8.759942E-2,-3.120232E-1,-1.3680433E-1,1.1262216E-1,-1.3406685E-2,-5.318928E-2,1.5617897E-2,1.400989E-1,-2.2907217E-3,2.2556266E-1,-3.509075E-3,3.1505649E-3,6.7552733E-3,1.8815292E-2,1.431593E-2,3.879194E-2,-8.164921E-3,4.2139017E-3,-1.6489454E-2,-5.028345E-3,-9.910956E-3,5.478642E-4,7.2356444E-3,3.2555868E-3,1.3077437E-3,-3.9338255E-3,-4.5785587E-3,3.7205967E-3,8.15121E-3,-9.630316E-4,1.22322105E-2,2.0410132E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.8873825E0,2.0949519E0,2.5750446E0,9.9771214E-1,5.264075E-1,3.0935514E-1,6.414442E-1,5.5918884E-1,5.3811425E-1,2.2733971E-1,2.2213225E-1,2.9131615E-1,6.759233E-2,1.37398E-1,6.185136E-1,2.7342647E-1,4.9407816E-1,3.7591076E-1,1.1285409E-2,0E0,4.19005E-2,3.305899E-1,1.3456911E-1,0E0,2.2088325E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7186426E2,2.1619815E-1,1.9530877E-1,1.9462195E2,2.0358758E8,6.191915E0,1.6507233E3,1.1266387E10,4.1081388E5,8E0,2.427E3,7.5E2,5.4527334E9,1.9643908E5,2.174459E5,1.175E3,6.608892E2,5.5173693E9,1.9391481E0,-1.3406685E-2,7.364257E4,6.4796996E-1,4.99E3,-2.2907217E-3,2.1030048E10,-3.509075E-3,3.1505649E-3,6.7552733E-3,1.8815292E-2,1.431593E-2,3.879194E-2,-8.164921E-3,4.2139017E-3,-1.6489454E-2,-5.028345E-3,-9.910956E-3,5.478642E-4,7.2356444E-3,3.2555868E-3,1.3077437E-3,-3.9338255E-3,-4.5785587E-3,3.7205967E-3,8.15121E-3,-9.630316E-4,1.22322105E-2,2.0410132E-3],"split_indices":[71,60,57,71,7,76,4,50,47,3,2,0,12,47,47,2,4,5,72,0,47,58,2,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.95E2,8E1,1.13E2,8.2E1,4.6E1,3.4E1,7.1E1,4.2E1,1.9E1,6.3E1,3.5E1,1.1E1,2E1,1.4E1,1.5E1,5.6E1,3.1E1,1.1E1,5E0,1.4E1,4.2E1,2.1E1,4E0,3.1E1,5E0,6E0,4E0,1.6E1,5E0,9E0,1E1,5E0,4.5E1,1.1E1,2E1,1.1E1,4E0,7E0,4E0,1E1,1.5E1,2.7E1,1.7E1,4E0,2.5E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-6.562115E-3,-1.3048783E-1,2.2347076E-1,-2.0250554E-1,-1.1698365E-2,9.9489175E-2,4.502752E-1,-2.4757701E-1,-6.868763E-3,-8.476708E-2,6.6012576E-2,1.3355128E-1,-5.9242588E-2,6.518211E-1,3.1071287E-1,-2.1511847E-1,-2.1366872E-2,1.00871116E-1,-8.921722E-2,-1.520396E-1,6.244792E-2,-2.2451725E-2,1.5579757E-1,1.7636213E-1,1.6952205E-3,-8.606896E-3,1.2879231E-3,1.5280052E-2,7.571422E-1,3.5717335E-1,5.860538E-3,-1.8638441E-2,-8.411605E-3,8.233474E-3,-5.9174263E-4,-8.577653E-3,9.491502E-4,-1.3062648E-2,-3.91669E-3,-9.709322E-4,8.453895E-3,-6.597653E-3,1.613792E-3,3.699599E-3,7.957266E-3,1.0919501E-2,4.4508916E-3,-5.195139E-3,7.974824E-3,2.0238478E-2,4.2508848E-2,7.6565524E-3,1.85597E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,-1,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8970885E0,1.5375216E0,2.6828012E0,9.8587465E-1,3.9739415E-1,3.539433E-1,7.538295E-1,5.396671E-1,2.0418814E-1,3.660214E-1,2.73552E-1,2.9887903E-1,1.388759E-1,1.8677759E-1,1.4201856E-1,4.2421436E-1,0E0,9.095485E-2,1.4246404E-1,1.9632614E-1,1.2533586E-1,1.3010634E-1,7.956773E-3,1.7089939E-1,2.9456225E-1,0E0,0E0,0E0,6.8992615E-2,8.202815E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28,29,29],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,-1,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4071463E2,2.5106274E-2,1.8829846E-1,3.165913E6,9.50379E5,2.8812434E7,7.013774E5,2.771097E0,1.6E1,1.2185057E0,5.3164E4,2.3797293E0,1.04E2,2.4214827E5,9.8163464E7,3.4936884E7,-2.1366872E-2,1.6868966E8,3.9E1,8E0,1.7518893E2,9.033382E-1,2.052E3,1.1936242E7,4.3818486E9,-8.606896E-3,1.2879231E-3,1.5280052E-2,6.886E3,2.8641683E8,5.860538E-3,-1.8638441E-2,-8.411605E-3,8.233474E-3,-5.9174263E-4,-8.577653E-3,9.491502E-4,-1.3062648E-2,-3.91669E-3,-9.709322E-4,8.453895E-3,-6.597653E-3,1.613792E-3,3.699599E-3,7.957266E-3,1.0919501E-2,4.4508916E-3,-5.195139E-3,7.974824E-3,2.0238478E-2,4.2508848E-2,7.6565524E-3,1.85597E-2],"split_indices":[71,57,57,1,9,64,52,72,3,60,48,72,8,47,51,7,0,7,3,3,71,60,2,64,12,0,0,0,2,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.79E2,9.6E1,1.11E2,6.8E1,6.3E1,3.3E1,9E1,2.1E1,3.5E1,3.3E1,5.2E1,1.1E1,1.2E1,2.1E1,8E1,1E1,9E0,1.2E1,2.4E1,1.1E1,1.7E1,1.6E1,3.9E1,1.3E1,4E0,7E0,4E0,8E0,1.6E1,5E0,1E1,7E1,5E0,4E0,6E0,6E0,7E0,1.7E1,7E0,4E0,5E0,1.2E1,4E0,1.2E1,2.1E1,1.8E1,8E0,5E0,4E0,4E0,4E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.2804228E-3,-1.2672013E-1,2.263347E-1,-1.872631E-1,-2.7809761E-2,9.588739E-2,4.447069E-1,-2.42015E-1,-2.023026E-2,-1.456923E-2,-4.826E-3,-7.280936E-2,1.3370061E-1,3.0428183E-1,6.012206E-1,-2.888798E-1,-1.2360803E-1,9.657179E-2,-9.048539E-2,-4.9619943E-2,1.3265707E-1,1.4213595E-3,-5.9166797E-3,1.6225491E-1,-4.949379E-3,4.1639605E-3,1.6825927E-2,3.5240296E-2,1.788566E-2,1.3498644E-3,-1.4410529E-2,-9.877811E-3,2.849631E-3,-8.9196855E-4,9.043458E-3,-7.703012E-3,1.2180454E-3,-3.2389776E-3,5.095308E-3,1.9713729E-3,7.8085917E-3,2.7058087E-3,9.688545E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.794818E0,1.048738E0,2.7205067E0,9.9194884E-1,4.4957182E-1,4.0523046E-1,6.082835E-1,4.193201E-1,2.3850471E-1,0E0,4.0072063E-1,7.326687E-2,3.6869437E-1,2.4165642E-1,2.6155758E-1,4.5419836E-1,4.349951E-1,1.3115343E-1,1.6613193E-1,1.7060067E-1,4.5841843E-2,0E0,0E0,2.1697533E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.083633E6,1.1670399E-1,1.5972575E0,2.396E3,3.3789E4,3.359399E-1,5.76E3,2.2E1,8.305048E4,-1.456923E-2,2.427E3,3.6002667E2,1.25440235E1,2.0787016E5,2.9375613E-1,1.6E1,1.9462195E2,6.2229652E0,4.231592E6,1.3431462E7,1.7300606E6,1.4213595E-3,-5.9166797E-3,1.0705044E3,-4.949379E-3,4.1639605E-3,1.6825927E-2,3.5240296E-2,1.788566E-2,1.3498644E-3,-1.4410529E-2,-9.877811E-3,2.849631E-3,-8.9196855E-4,9.043458E-3,-7.703012E-3,1.2180454E-3,-3.2389776E-3,5.095308E-3,1.9713729E-3,7.8085917E-3,2.7058087E-3,9.688545E-3],"split_indices":[70,58,60,2,9,61,2,3,52,0,2,71,76,47,57,10,71,75,1,9,70,0,0,74,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.75E2,9.6E1,1.08E2,6.7E1,6.1E1,3.5E1,8.1E1,2.7E1,4E0,6.3E1,1.1E1,5E1,2E1,1.5E1,5.7E1,2.4E1,1E1,1.7E1,4.8E1,1.5E1,4E0,7E0,4.5E1,5E0,5E0,1.5E1,7E0,8E0,4E0,5.3E1,1.6E1,8E0,5E0,5E0,1E1,7E0,4.3E1,5E0,5E0,1E1,1.5E1,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.6173291E-3,-8.5729115E-2,2.832087E-1,-1.5077375E-1,7.1107745E-2,1.3178344E-1,4.2221588E-1,-2.1220489E-1,-5.4644745E-2,1.1440147E-2,1.7007707E-1,9.530249E-2,1.4931689E-2,2.6542822E-1,5.9181195E-1,-1.6032451E-1,-3.7234953E-1,-1.1518315E-1,2.0817775E-2,1.2416999E-1,-6.567375E-2,1.9964406E-2,7.790098E-2,1.3290417E-1,-2.1746214E-3,4.2201085E-3,1.395696E-2,3.2665662E-2,1.9404117E-2,-1.4587144E-2,-4.329589E-3,-2.3598464E-2,-1.2557757E-2,-6.5033445E-3,2.9586204E-3,-6.226961E-3,3.2011797E-3,1.0076781E-2,3.060576E-3,2.6669793E-4,-6.0814843E-3,7.1549444E-3,-3.4032168E-4,2.3125403E-3,7.877769E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,-1,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.039637E0,2.214223E0,1.3757038E0,8.9191747E-1,3.75991E-1,2.0789605E-1,7.7358675E-1,7.1298075E-1,2.8106907E-1,3.6456925E-1,5.49241E-1,1.6706368E-1,0E0,1.0328221E-1,2.1744251E-2,7.1714914E-1,1.6740847E-1,1.681189E-1,2.2226313E-1,7.8329474E-2,1.1942528E-1,0E0,1.2674636E-1,6.7201525E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,-1,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.165913E6,2.174459E5,1.1670399E-1,6.639E3,4.328683E0,4.063E3,5.1653096E-3,9.50379E5,9.953969E6,6.747254E5,1.1598054E8,1.4931689E-2,4.7436612E2,2.9898608E-1,3.3617156E-4,1.659E3,1.0289595E6,2.5188733E2,4.986835E6,5.102E3,1.9964406E-2,8.223634E6,1E0,-2.1746214E-3,4.2201085E-3,1.395696E-2,3.2665662E-2,1.9404117E-2,-1.4587144E-2,-4.329589E-3,-2.3598464E-2,-1.2557757E-2,-6.5033445E-3,2.9586204E-3,-6.226961E-3,3.2011797E-3,1.0076781E-2,3.060576E-3,2.6669793E-4,-6.0814843E-3,7.1549444E-3,-3.4032168E-4,2.3125403E-3,7.877769E-3],"split_indices":[61,1,47,58,2,54,2,57,9,64,66,51,0,4,57,58,10,52,4,1,0,0,48,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.15E2,6.7E1,1.52E2,6.3E1,3.3E1,3.4E1,9.2E1,6E1,4E1,2.3E1,2.9E1,4E0,1.9E1,1.5E1,7.1E1,2.1E1,3.3E1,2.7E1,1.6E1,2.4E1,5E0,1.8E1,2.3E1,6E0,4E0,1.5E1,7E0,8E0,2E1,5.1E1,7E0,1.4E1,2.9E1,4E0,6E0,2.1E1,5E0,1.1E1,1.2E1,1.2E1,9E0,9E0,8E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[5.448896E-3,-1.1091449E-1,2.0679466E-1,-1.7352903E-1,-7.671845E-3,8.73633E-2,4.1192907E-1,-2.4279347E-1,-5.2307483E-2,-1.1494169E-1,3.8457513E-2,-9.4130084E-2,1.2085565E-1,2.8959206E-1,6.0096455E-1,3.0339844E-4,-2.7017272E-1,6.9259778E-3,-2.1986778E-1,-1.2515523E-2,-4.8923224E-2,-4.374927E-2,9.666044E-2,5.486807E-3,-1.0729481E-2,2.0315604E-1,4.5276098E-2,5.8826846E-3,3.2365894E-1,3.246025E-2,1.5870722E-2,-1.4313898E-2,-3.982747E-3,1.993537E-3,-5.946832E-3,-1.4400913E-2,-5.621118E-3,-5.264809E-3,-4.0461618E-4,-5.7544215E-3,3.611996E-3,6.191279E-3,-1.0630519E-4,6.827361E-3,1.5595774E-2,1.1583481E-2,-1.8767477E-4,1.6414108E-2,7.081248E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,-1,37,39,41,-1,-1,43,45,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.653422E0,1.162277E0,2.5119586E0,9.276905E-1,3.4585232E-1,4.1864502E-1,7.047553E-1,4.865632E-1,4.1838017E-1,2.0421454E-1,2.3867792E-1,3.5370338E-1,3.453549E-1,9.646487E-2,1.3030958E-1,0E0,4.5177794E-1,1.6399291E-1,6.1749935E-2,0E0,4.100979E-2,2.182728E-1,1.0883677E-1,0E0,0E0,1.5062082E-1,3.1667027E-1,0E0,5.6767464E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,-1,38,40,42,-1,-1,44,46,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.5106274E-2,1.8829846E-1,1.9462195E2,2.0283392E8,7.5E2,2.0551236E9,1E1,4.908866E6,8E0,9.50379E5,3.13E2,3.5630266E8,1.9643908E5,9.780233E9,3.0339844E-4,6.608892E2,2.142943E1,2.26E3,-1.2515523E-2,1.8508842E2,1.1685816E0,1.1225629E6,5.486807E-3,-1.0729481E-2,2.610156E3,1.897231E0,5.8826846E-3,4.1489546E8,3.246025E-2,1.5870722E-2,-1.4313898E-2,-3.982747E-3,1.993537E-3,-5.946832E-3,-1.4400913E-2,-5.621118E-3,-5.264809E-3,-4.0461618E-4,-5.7544215E-3,3.611996E-3,6.191279E-3,-1.0630519E-4,6.827361E-3,1.5595774E-2,1.1583481E-2,-1.8767477E-4,1.6414108E-2,7.081248E-3],"split_indices":[71,57,57,71,7,0,5,10,9,3,9,0,7,47,5,0,4,75,2,0,4,60,52,0,0,4,73,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.79E2,1.03E2,1.11E2,6.8E1,6.6E1,3.7E1,7E1,4.1E1,2E1,4.8E1,1E1,5.6E1,2.4E1,1.3E1,7E0,6.3E1,3.1E1,1E1,5E0,1.5E1,2E1,2.8E1,4E0,6E0,2.6E1,3E1,5E0,1.9E1,8E0,5E0,5.1E1,1.2E1,2.5E1,6E0,4E0,6E0,5E0,1E1,1.2E1,8E0,2E1,8E0,2E1,6E0,5E0,2.5E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.3387234E-2,-9.8158866E-2,2.0095628E-1,-1.5815048E-1,3.3144096E-3,9.399044E-2,3.8426563E-1,-2.0907788E-1,-1.5388826E-3,-2.7632961E-2,9.4141126E-2,1.6830553E-1,3.962011E-2,2.1663459E-1,4.533017E-1,-6.14332E-2,-2.3814444E-1,6.1698775E-3,-3.6832076E-2,-8.5555054E-2,1.3833603E-2,1.4041278E-1,-6.4882846E-4,2.1202146E-1,-1.2579839E-3,-1.4520428E-1,1.1473296E-1,1.1762786E-2,5.926803E-3,2.4302903E-1,5.332546E-1,-6.6719046E-3,6.192841E-3,-1.5215636E-2,-8.203149E-3,4.5509716E-3,-5.3307908E-3,2.5909973E-4,-6.056625E-3,-3.3308335E-3,3.1093373E-3,9.219971E-3,3.1796014E-3,1.5120998E-2,7.4329404E-3,-2.6809094E-3,-8.35247E-3,1.2690519E-2,5.0229883E-5,1.4495721E-2,5.645043E-3,1.71185E-2,2.878624E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,-1,43,-1,45,47,-1,-1,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.836207E0,1.0658447E0,2.003098E0,8.770187E-1,1.8811736E-1,2.6404756E-1,3.4155798E-1,3.4156656E-1,1.3888466E-1,1.2127505E-1,8.711721E-2,2.4525148E-1,5.719672E-1,1.1508822E-2,3.0751324E-1,2.6484454E-1,3.2322145E-1,0E0,2.5781983E-1,8.880988E-2,1.437203E-1,3.74095E-2,0E0,8.239436E-2,0E0,2.7228385E-2,5.299703E-1,0E0,0E0,3.922844E-2,3.4814835E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,25,25,26,26,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,-1,44,-1,46,48,-1,-1,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.5106274E-2,1.8829846E-1,1.781973E6,9.749826E6,1.1936242E7,3.301E3,2.431768E2,2.6017979E11,1.804E3,2.8808496E7,2.332406E0,1.4813267E3,9.3735106E5,2.2324745E5,3.084419E6,1.8662952E0,6.1698775E-3,1.2321745E7,1.71E2,3.7230852E0,1.8609113E0,-6.4882846E-4,4.0012E4,-1.2579839E-3,1.177948E3,6.464779E6,1.1762786E-2,5.926803E-3,1.61628E5,1.6507233E3,-6.6719046E-3,6.192841E-3,-1.5215636E-2,-8.203149E-3,4.5509716E-3,-5.3307908E-3,2.5909973E-4,-6.056625E-3,-3.3308335E-3,3.1093373E-3,9.219971E-3,3.1796014E-3,1.5120998E-2,7.4329404E-3,-2.6809094E-3,-8.35247E-3,1.2690519E-2,5.0229883E-5,1.4495721E-2,5.645043E-3,1.71185E-2,2.878624E-2],"split_indices":[71,57,57,1,9,64,2,74,50,2,9,72,74,66,47,9,72,0,64,0,73,72,0,9,0,4,48,0,0,47,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,1.74E2,1.03E2,1.09E2,6.5E1,6.6E1,3.7E1,8.2E1,2.7E1,4.9E1,1.6E1,2.7E1,3.9E1,1.2E1,2.5E1,1.4E1,6.8E1,5E0,2.2E1,2E1,2.9E1,1.1E1,5E0,2.2E1,5E0,1.1E1,2.8E1,7E0,5E0,8E0,1.7E1,1E1,4E0,2.5E1,4.3E1,8E0,1.4E1,7E0,1.3E1,1.1E1,1.8E1,5E0,6E0,5E0,1.7E1,4E0,7E0,1.1E1,1.7E1,4E0,4E0,8E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[5.261088E-3,-9.0237655E-2,2.4455974E-1,-1.7128584E-1,-1.3149979E-2,1.16032735E-1,3.954769E-1,-4.172483E-2,-2.5845715E-1,-5.402384E-2,1.05261154E-1,-4.9818265E-3,1.6863844E-1,2.7543652E-1,5.208101E-1,-7.745435E-2,7.720929E-3,-2.9164845E-1,-7.661237E-2,-8.8633694E-2,1.5540023E-1,5.056547E-3,1.4951561E-1,-9.90787E-2,6.3073384E-3,1.9304242E-2,8.850711E-2,4.951181E-3,1.4304491E-2,3.1204592E-2,1.28521295E-2,-6.621332E-3,2.8491096E-4,-2.0483453E-2,-1.04234805E-2,-7.895453E-3,7.006872E-4,-7.0248987E-3,-4.1183087E-4,2.94758E-3,1.1613927E-2,3.765061E-3,-3.3460052E-3,7.93193E-3,2.5724815E-3,-9.286988E-3,1.0759494E-3,5.153018E-3,-1.0522591E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,-1,-1,47,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.16945E0,1.2035762E0,1.4476757E0,1.0508521E0,4.8967856E-1,2.7458048E-1,3.8312626E-1,3.0480742E-1,3.1565356E-1,5.573841E-1,1.14540994E-1,2.0060562E-1,5.654253E-1,8.496845E-2,4.176607E-1,1.896056E-1,0E0,3.692701E-1,8.973447E-2,3.2985753E-1,8.120319E-2,5.9614412E-2,2.8743893E-2,1.1781798E-1,0E0,0E0,6.5139234E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,-1,-1,48,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,1.6425813E2,2.0224316E-1,1.1998492E10,4.354824E5,3.935E3,1.6507233E3,9.74979E6,6.608892E2,1.2730925E0,1.575011E9,2.5312589E8,8.312613E5,1.9643908E5,2.65826E6,9.9409575E5,7.720929E-3,2.851239E5,8.34E2,2.8800612E9,1.2311872E6,6.8618E4,1E0,7.5E2,6.3073384E-3,1.9304242E-2,1.4524763E3,4.951181E-3,1.4304491E-2,3.1204592E-2,1.28521295E-2,-6.621332E-3,2.8491096E-4,-2.0483453E-2,-1.04234805E-2,-7.895453E-3,7.006872E-4,-7.0248987E-3,-4.1183087E-4,2.94758E-3,1.1613927E-2,3.765061E-3,-3.3460052E-3,7.93193E-3,2.5724815E-3,-9.286988E-3,1.0759494E-3,5.153018E-3,-1.0522591E-3],"split_indices":[71,71,57,50,47,2,4,9,4,61,5,7,66,47,66,52,0,47,0,5,52,48,22,0,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.92E2,7.6E1,9.3E1,9.9E1,4.2E1,3.4E1,3.8E1,5.5E1,7.4E1,2.5E1,1.3E1,2.9E1,1.9E1,1.5E1,3.3E1,5E0,4.6E1,9E0,6.4E1,1E1,8E0,1.7E1,8E0,5E0,6E0,2.3E1,4E0,1.5E1,8E0,7E0,1.8E1,1.5E1,1.2E1,3.4E1,4E0,5E0,3.5E1,2.9E1,6E0,4E0,4E0,4E0,1.3E1,4E0,4E0,4E0,1.9E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"49","size_leaf_vector":"1"}},{"base_weights":[1.15079265E-2,-6.397281E-2,2.4555098E-1,-1.2206563E-1,7.136642E-2,1.11058585E-1,3.6918497E-1,-1.5477005E-1,1.1730725E-2,1.7081004E-2,4.520793E-2,7.9825655E-2,1.2718587E-2,2.671033E-1,5.976788E-1,-1.1897293E-1,-2.6176205E-1,5.246055E-2,-7.363438E-3,1.2089093E-1,-3.4504768E-2,1.0733184E-1,-2.214011E-3,4.5019686E-3,1.3939472E-2,3.2542974E-2,1.598396E-2,-1.3264752E-2,-3.7661728E-3,-5.580006E-3,-1.6291892E-2,9.3092555E-3,2.2639714E-4,3.7685502E-3,1.4158694E-2,-4.4517163E-3,2.0014355E-3,3.594364E-5,6.166306E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.926785E0,1.6676435E0,1.0896916E0,6.494236E-1,4.8484936E-1,1.5336278E-1,6.56734E-1,4.2462826E-1,2.1814393E-1,0E0,3.6614507E-1,1.10713676E-1,0E0,1.2737691E-1,2.8036833E-2,5.4856396E-1,3.3252525E-1,1.7697695E-1,0E0,2.0069253E-1,1.4967111E-1,7.074866E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,3.165913E6,2.174459E5,4.9615598E-1,2.207E3,4.328683E0,1.623319E3,4.352025E0,2.5417683E0,1.7081004E-2,1.955886E7,1.1598054E8,1.2718587E-2,4.796397E2,2.7479675E-1,1E0,7.6630195E6,9.0871654E-2,-7.363438E-3,8.834E3,2.2296374E0,1.64E2,-2.214011E-3,4.5019686E-3,1.3939472E-2,3.2542974E-2,1.598396E-2,-1.3264752E-2,-3.7661728E-3,-5.580006E-3,-1.6291892E-2,9.3092555E-3,2.2639714E-4,3.7685502E-3,1.4158694E-2,-4.4517163E-3,2.0014355E-3,3.594364E-5,6.166306E-3],"split_indices":[61,1,47,58,0,54,4,73,72,0,64,51,0,4,57,31,64,57,0,2,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.1E2,6.7E1,1.47E2,6.3E1,3.3E1,3.4E1,1.18E2,2.9E1,4E0,5.9E1,2.9E1,4E0,2.5E1,9E0,9E1,2.8E1,2.4E1,5E0,3E1,2.9E1,2.4E1,5E0,5E0,2E1,5E0,4E0,1.5E1,7.5E1,1.2E1,1.6E1,5E0,1.9E1,2.6E1,4E0,1.6E1,1.3E1,5E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.907077E-3,-1.10231504E-1,1.560937E-1,-1.7390074E-1,-1.4979227E-2,6.4361654E-2,3.3062205E-1,-2.3123567E-1,-3.241137E-2,-1.1863115E-1,3.0680554E-2,1.5049924E-2,4.4690855E-2,2.5125198E-2,2.73239E-1,-1.971846E-1,-2.1857517E-2,6.502721E-3,-6.455878E-2,-1.0979424E-2,-5.479746E-2,1.5467468E-1,-1.899492E-3,-7.1558125E-2,8.332842E-2,4.954007E-3,3.0732498E-1,3.3359108E-3,-9.961819E-3,-6.188587E-3,1.9760157E-3,-5.4547014E-3,-5.1157834E-4,1.0084352E-2,2.141548E-3,-2.753475E-3,2.3288236E-3,3.6607396E-3,-6.2513426E-3,-1.9615863E-3,4.7614938E-3,8.62813E-3,1.6565144E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,-1,-1,29,-1,31,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7295275E0,1.0101895E0,1.7558129E0,8.037338E-1,3.2546487E-1,3.6752412E-1,3.1990767E-1,4.9509E-1,1.765889E-1,1.475777E-1,1.9420104E-1,0E0,3.2037327E-1,0E0,1.4760303E-1,3.4672666E-1,0E0,0E0,2.0097452E-1,0E0,4.041077E-2,5.9418827E-2,1.2154306E-1,1.8542328E-1,1.3849643E-1,0E0,9.61647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,18,18,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,-1,-1,30,-1,32,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.1309677E2,1.1670399E-1,1.3305582E0,5.5015685E9,2.0370931E8,2.4383854E4,1.8681E4,1.4031231E-2,2.7864842E11,1.8990291E0,1.9124458E0,1.5049924E-2,3.0327642E-1,2.5125198E-2,1.7246376E0,1.1623532E6,-2.1857517E-2,6.502721E-3,4.5E1,-1.0979424E-2,1.8508842E2,1.2183861E7,9.50379E5,1.409674E7,3.614121E8,4.954007E-3,7.2610445E8,3.3359108E-3,-9.961819E-3,-6.188587E-3,1.9760157E-3,-5.4547014E-3,-5.1157834E-4,1.0084352E-2,2.141548E-3,-2.753475E-3,2.3288236E-3,3.6607396E-3,-6.2513426E-3,-1.9615863E-3,4.7614938E-3,8.62813E-3,1.6565144E-2],"split_indices":[71,58,58,5,7,52,9,57,50,72,77,0,61,0,72,64,0,0,8,0,4,51,9,64,12,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,1.66E2,1.1E2,9.9E1,6.7E1,7.3E1,3.7E1,7E1,2.9E1,2E1,4.7E1,4E0,6.9E1,6E0,3.1E1,6.3E1,7E0,4E0,2.5E1,6E0,1.4E1,9E0,3.8E1,1.7E1,5.2E1,6E0,2.5E1,4E0,5.9E1,1.5E1,1E1,5E0,9E0,5E0,4E0,1.8E1,2E1,5E0,1.2E1,7E0,4.5E1,9E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[2.78766E-3,-5.9377663E-2,2.805717E-1,-1.324866E-1,3.744333E-2,9.797968E-2,4.031186E-1,-4.683293E-2,-2.2750878E-1,-3.3787608E-3,1.2183019E-1,4.3040212E-2,9.286942E-3,3.0033776E-1,5.981496E-1,-8.650509E-3,-1.2641405E-2,-2.2197314E-2,-1.8958826E-1,-6.8728946E-2,5.4599367E-2,1.5231122E-2,5.8440927E-2,9.559811E-2,-1.4126698E-3,1.5144936E-2,6.584264E-3,1.6348228E-2,3.3222765E-2,-1.55101E-3,6.284026E-3,-1.12308115E-2,-2.2411167E-3,-4.661134E-3,3.8490896E-3,3.9403453E-3,-2.7932785E-3,-4.4625686E-3,5.1844753E-3,1.1539118E-3,6.768676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,-1,29,-1,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.627358E0,1.5551479E0,1.0569315E0,1.0067198E0,3.2767218E-1,1.10995054E-1,4.0167093E-1,3.2075566E-1,4.7824E-1,2.5002983E-1,3.9133808E-1,6.105414E-2,0E0,4.7471285E-2,1.3820648E-2,0E0,1.7722012E-1,0E0,3.9397097E-1,1.6198213E-1,1.3019392E-1,0E0,2.2129868E-1,2.8955169E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,-1,30,-1,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,1.249246E6,2.174459E5,1.19331E5,4.876025E2,2.2486712E1,1.7598395E3,1.3692686E4,9.975593E4,4.8E1,6.1650256E5,1.020435E4,9.286942E-3,4.7201157E-1,2.4394053E9,-8.650509E-3,2.427E3,-2.2197314E-2,1.1306552E-1,4.0159574E-1,2.2946768E0,1.5231122E-2,2.51838E6,1E0,-1.4126698E-3,1.5144936E-2,6.584264E-3,1.6348228E-2,3.3222765E-2,-1.55101E-3,6.284026E-3,-1.12308115E-2,-2.2411167E-3,-4.661134E-3,3.8490896E-3,3.9403453E-3,-2.7932785E-3,-4.4625686E-3,5.1844753E-3,1.1539118E-3,6.768676E-3],"split_indices":[61,1,47,48,71,75,4,67,47,8,66,51,0,57,5,0,2,0,58,76,72,0,51,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.18E2,4.8E1,1.24E2,9.4E1,2E1,2.8E1,6.6E1,5.8E1,6.4E1,3E1,1.4E1,6E0,2E1,8E0,1.2E1,5.4E1,6E0,5.2E1,3E1,3.4E1,6E0,2.4E1,8E0,6E0,1.6E1,4E0,4E0,4E0,4.8E1,6E0,3.7E1,1.5E1,2.5E1,5E0,2.7E1,7E0,6E0,1.8E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-6.4872545E-3,-6.5911904E-2,2.459096E-1,-1.15997404E-1,5.143252E-2,1.05173156E-1,3.6781096E-1,-1.4529198E-1,-6.30318E-4,1.5969163E-2,2.7549699E-2,5.7816297E-2,1.1666496E-2,2.5568223E-1,5.7611805E-1,-1.7830033E-2,-1.299662E-1,3.5455097E-2,-8.7172285E-2,8.3272494E-2,-2.4066662E-2,5.2803885E-3,1.3995697E-2,2.765587E-1,6.341746E-3,1.5746074E-2,3.1999074E-2,-8.576292E-3,-3.4202011E-3,-1.5801423E-3,4.0399237E-3,3.5253723E-4,-6.982954E-3,1.3579358E-3,6.743551E-3,-4.7590914E-3,1.6856256E-3,2.1072815E-3,-2.1177272E-3,1.4204722E-2,6.6544395E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,25,-1,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1708198E0,1.3283479E0,8.7058234E-1,5.3389025E-1,4.7020566E-1,1.6785002E-1,5.0669885E-1,4.1692805E-1,1.06146485E-1,0E0,1.8608138E-1,5.073069E-2,0E0,1.115036E-2,1.2820959E-2,0E0,3.6239648E-1,9.0578474E-2,6.2322155E-2,1.010357E-1,1.6738361E-1,0E0,2.6885673E-2,2.2930145E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,26,-1,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4272063E0,3.165913E6,2.174459E5,4.9615598E-1,2.207E3,1.7604632E0,1.6507233E3,1E0,2.3317811E0,1.5969163E-2,1.955886E7,1.7046362E0,1.1666496E-2,4.7201157E-1,2.4394053E9,-1.7830033E-2,4.9005847E0,3.3550136E0,7.826598E5,2.160255E9,2.0382197E0,5.2803885E-3,1.6E1,2.094037E7,6.341746E-3,1.5746074E-2,3.1999074E-2,-8.576292E-3,-3.4202011E-3,-1.5801423E-3,4.0399237E-3,3.5253723E-4,-6.982954E-3,1.3579358E-3,6.743551E-3,-4.7590914E-3,1.6856256E-3,2.1072815E-3,-2.1177272E-3,1.4204722E-2,6.6544395E-3],"split_indices":[61,1,47,58,0,54,4,15,72,0,64,58,0,57,5,0,75,75,52,12,73,0,3,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.24E2,5.2E1,1.57E2,6.7E1,2.5E1,2.7E1,1.25E2,3.2E1,4E0,6.3E1,2E1,5E0,1.9E1,8E0,6E0,1.19E2,2.3E1,9E0,3E1,3.3E1,8E0,1.2E1,1.5E1,4E0,4E0,4E0,5.8E1,6.1E1,1E1,1.3E1,4E0,5E0,1.7E1,1.3E1,1.4E1,1.9E1,8E0,4E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-4.660276E-3,-7.5700805E-2,1.734371E-1,-1.3291112E-1,2.1073115E-3,7.142443E-2,3.054702E-1,-1.854218E-1,-3.4727223E-2,-8.941836E-2,2.8995635E-2,1.2489289E-2,4.6526223E-2,2.192649E-1,3.838485E-1,-1.5541852E-1,-3.6790165E-1,4.9233317E-2,-1.8913843E-1,-1.11503545E-2,-2.2351334E-2,-9.1770105E-3,9.9361196E-2,-1.13287054E-1,8.6291745E-2,2.856018E-3,1.1835231E-2,2.2234E-2,1.2256544E-2,3.7611583E-3,-7.908463E-3,-2.2461448E-2,-6.923422E-3,1.0301124E-4,8.743266E-3,-1.0047784E-2,-5.024026E-3,8.989629E-4,-3.3609788E-3,6.0103675E-3,-1.3768893E-3,5.9764185E-3,-6.265475E-4,-7.498035E-3,-1.8905048E-3,6.2626805E-3,9.1052956E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.593683E0,9.0391946E-1,1.0691748E0,5.967779E-1,2.1669729E-1,2.2123474E-1,1.5231729E-1,3.589208E-1,5.538187E-1,1.9949609E-1,1.8356255E-1,0E0,2.8304324E-1,9.6958876E-2,6.95672E-2,2.7379453E-1,1.8757331E-1,1.8467844E-1,1.1110127E-2,0E0,3.3239976E-2,1.3423409E-1,8.4311366E-2,2.553396E-2,1.4649001E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8045242E2,1.8509178E-1,1.8829846E-1,1.9462195E2,1.8681856E8,1.4153711E-3,5.76E3,1.2869189E5,2.142943E1,8E0,2.704375E6,1.2489289E-2,1.479938E6,2.0787016E5,2.9898608E-1,3.8757736E-7,1.7218688E8,7.032E3,1.3469651E3,-1.11503545E-2,1.7011462E5,2.3755E4,2.8808496E7,4.3085215E11,2.3797293E0,2.856018E-3,1.1835231E-2,2.2234E-2,1.2256544E-2,3.7611583E-3,-7.908463E-3,-2.2461448E-2,-6.923422E-3,1.0301124E-4,8.743266E-3,-1.0047784E-2,-5.024026E-3,8.989629E-4,-3.3609788E-3,6.0103675E-3,-1.3768893E-3,5.9764185E-3,-6.265475E-4,-7.498035E-3,-1.8905048E-3,6.2626805E-3,9.1052956E-5],"split_indices":[71,58,57,71,7,57,2,66,75,3,9,0,51,47,57,57,7,10,4,0,47,9,9,50,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.02E2,8E1,1.16E2,8.6E1,4.6E1,3.4E1,7.5E1,4.1E1,1.9E1,6.7E1,4E0,4.2E1,1.8E1,1.6E1,6.6E1,9E0,2.7E1,1.4E1,5E0,1.4E1,4.4E1,2.3E1,8E0,3.4E1,4E0,1.4E1,7E0,9E0,4E0,6.2E1,5E0,4E0,2.1E1,6E0,9E0,5E0,8E0,6E0,5E0,3.9E1,1.8E1,5E0,4E0,4E0,2.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.4954677E-3,-6.2547676E-2,1.9570342E-1,-1.2018361E-1,1.2433611E-2,2.976111E-1,9.40211E-2,-7.982474E-2,-2.2363111E-1,8.368774E-3,-4.0375446E-3,2.8246444E-2,2.080085E-1,1.2471063E-1,-2.345195E-3,-1.2463708E-1,2.6658533E-2,-3.1819725E-1,-9.5242985E-2,-6.591116E-2,6.412437E-2,6.102004E-2,2.8627875E-1,2.4245268E-2,1.8159871E-1,-1.2777416E-2,-4.4097817E-3,8.058981E-3,-2.863397E-3,-1.8598793E-2,-5.7881386E-3,-4.4350702E-4,-6.9635296E-3,-5.1220683E-3,1.3425859E-3,4.655637E-3,-2.6230586E-3,7.960848E-3,-2.8102314E-3,5.7578427E-3,1.4890083E-2,-3.7201787E-3,3.9995285E-3,6.123717E-3,1.1321517E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,-1,25,27,29,31,33,35,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4429915E0,8.8742566E-1,6.870105E-1,4.6645153E-1,2.5175536E-1,8.347542E-1,1.6650549E-1,4.0892267E-1,3.5821128E-1,0E0,3.542727E-1,0E0,3.046664E-1,1.6705552E-1,0E0,2.3813432E-1,3.536628E-1,2.4635649E-1,7.045956E-2,1.929037E-1,1.8495904E-1,1.6077776E-1,6.730461E-2,8.5959695E-2,2.6661694E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,-1,26,28,30,32,34,36,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.2709158E2,2.1619815E-1,9.520617E5,4.1295107E-2,2.3755E4,1E0,2.008498E1,4.780434E-3,5.267875E5,8.368774E-3,9.50379E5,2.8246444E-2,1.1047089E-1,1.19881555E-1,-2.345195E-3,1.9E3,1.6316831E4,2.0201733E9,6.065435E5,1.1102881E0,5.0128145E0,3.738E3,4.063E3,2.7713644E1,6.7E1,-1.2777416E-2,-4.4097817E-3,8.058981E-3,-2.863397E-3,-1.8598793E-2,-5.7881386E-3,-4.4350702E-4,-6.9635296E-3,-5.1220683E-3,1.3425859E-3,4.655637E-3,-2.6230586E-3,7.960848E-3,-2.8102314E-3,5.7578427E-3,1.4890083E-2,-3.7201787E-3,3.9995285E-3,6.123717E-3,1.1321517E-2],"split_indices":[71,60,52,61,9,15,77,58,52,0,9,0,57,57,0,12,66,5,52,60,73,0,2,75,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2.04E2,6.8E1,1.15E2,8.9E1,3.3E1,3.5E1,8.4E1,3.1E1,7E0,8.2E1,6E0,2.7E1,2.9E1,6E0,5.9E1,2.5E1,1.7E1,1.4E1,4.3E1,3.9E1,1E1,1.7E1,1.1E1,1.8E1,8E0,5.1E1,9E0,1.6E1,1.1E1,6E0,6E0,8E0,2.9E1,1.4E1,3E1,9E0,5E0,5E0,4E0,1.3E1,4E0,7E0,1.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[1.0292259E-2,-3.4356654E-2,2.1874754E-1,-7.8566894E-2,6.119629E-2,7.5043626E-2,3.2384533E-1,-1.257327E-1,-9.874781E-3,8.480252E-2,-7.956635E-2,-4.1844492E-4,1.3796355E-1,2.3364724E-1,4.2463887E-1,-7.779738E-2,-2.9680678E-1,-1.1077852E-2,9.19967E-3,1.6997752E-1,1.6199687E-2,-5.853418E-3,1.3548402E-4,2.0969927E-3,-2.1323129E-3,2.7834452E-3,9.456622E-3,1.1812086E-2,5.863621E-3,1.0908755E-2,2.4165334E-2,-9.9446485E-3,-2.1899713E-3,-1.811231E-2,-6.2175826E-3,-1.2788196E-3,2.6146667E-3,2.2568621E-3,1.2502025E-2,-9.481612E-4,5.9332373E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5856378E0,9.7220004E-1,7.1574235E-1,5.074257E-1,2.4931735E-1,1.04515225E-1,1.553433E-1,7.39555E-1,2.8828573E-1,3.6537084E-1,4.354716E-2,2.5314309E-2,5.0300643E-2,8.719981E-3,1.0139418E-1,2.9889545E-1,2.5936043E-1,0E0,1.0891681E-1,3.3066946E-1,1.5189107E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5648934E0,3.135E3,2.787E3,1.5415665E-2,1.5830926E1,6.765595E7,2.9391026E1,5.1653096E-3,2.718E4,4.8883E4,1E0,1.4679102E2,1.2984486E6,3.160848E6,2.6680366E5,9.421523E-5,1.1152775E7,-1.1077852E-2,5.101512E0,7.051287E4,1.19881555E-1,-5.853418E-3,1.3548402E-4,2.0969927E-3,-2.1323129E-3,2.7834452E-3,9.456622E-3,1.1812086E-2,5.863621E-3,1.0908755E-2,2.4165334E-2,-9.9446485E-3,-2.1899713E-3,-1.811231E-2,-6.2175826E-3,-1.2788196E-3,2.6146667E-3,2.2568621E-3,1.2502025E-2,-9.481612E-4,5.9332373E-3],"split_indices":[60,2,2,57,76,5,75,57,9,9,31,71,52,9,47,61,64,0,75,47,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.28E2,4.8E1,1.56E2,7.2E1,2.1E1,2.7E1,9.2E1,6.4E1,6.2E1,1E1,1E1,1.1E1,1.6E1,1.1E1,7.3E1,1.9E1,4E0,6E1,2.7E1,3.5E1,6E0,4E0,5E0,5E0,6E0,5E0,1.2E1,4E0,5E0,6E0,1.2E1,6.1E1,1.1E1,8E0,3.4E1,2.6E1,1.3E1,1.4E1,2.7E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-5.9847976E-3,-8.624195E-2,1.14453435E-1,-1.3104282E-1,4.211257E-2,2.0858116E-2,9.802297E-2,-1.6680013E-2,-1.18596114E-1,1.4497058E-2,7.822042E-3,3.0587737E-2,1.4084576E-1,-2.5856104E-1,-9.953093E-2,7.431583E-2,-5.2982405E-2,5.0977536E-2,-4.9284142E-3,1.4990207E-2,1.0656606E-1,-1.834578E-2,-4.780183E-3,-2.9989132E-3,-8.4703015E-3,-5.472433E-4,4.447988E-3,2.3905463E-3,-4.0005483E-3,1.5483845E-3,7.4683735E-3,6.4264913E-3,-3.3000865E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,-1,11,-1,13,-1,15,17,19,21,23,25,27,29,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.755023E0,9.888363E-1,5.8977664E-1,3.2256746E-1,4.1842562E-1,0E0,3.133459E-1,0E0,2.9981697E-1,0E0,1.6970527E-1,1.289332E-1,3.8959837E-1,2.617069E-1,3.0459678E-1,3.958281E-2,8.27689E-2,6.979805E-2,0E0,0E0,2.1879053E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20],"right_children":[2,4,6,8,10,-1,12,-1,14,-1,16,18,20,22,24,26,28,30,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.4269168E-1,2.796589E6,1.4396E4,1E0,2.207E3,2.0858116E-2,3.4071463E2,-1.6680013E-2,1E0,1.4497058E-2,1.955886E7,1.9E1,8E0,1.9E1,4.352025E0,2.80215E6,1E1,7.432E3,-4.9284142E-3,1.4990207E-2,6.812E3,-1.834578E-2,-4.780183E-3,-2.9989132E-3,-8.4703015E-3,-5.472433E-4,4.447988E-3,2.3905463E-3,-4.0005483E-3,1.5483845E-3,7.4683735E-3,6.4264913E-3,-3.3000865E-4],"split_indices":[61,1,9,15,0,0,71,0,31,0,64,3,3,3,73,12,3,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.7E2,1.13E2,1.26E2,4.4E1,4E0,1.09E2,5E0,1.21E2,4E0,4E1,4.3E1,6.6E1,1.3E1,1.08E2,1.9E1,2.1E1,3.8E1,5E0,9E0,5.7E1,6E0,7E0,7.8E1,3E1,4E0,1.5E1,5E0,1.6E1,3.4E1,4E0,4.4E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-8.450708E-3,-7.176439E-2,1.3501188E-1,-5.7520848E-2,-2.689406E-1,5.7696655E-2,2.4293911E-1,-1.4114341E-1,-2.7192185E-2,-1.5506827E-2,-4.281715E-3,7.692729E-2,-4.721945E-3,-6.0996685E-5,2.735135E-1,-2.1443115E-1,-3.3215422E-2,-4.106821E-4,-1.0831231E-1,1.1778847E-1,4.551584E-3,1.5411691E-1,3.470945E-1,-1.40729165E-2,-6.079914E-3,-4.7486667E-3,3.2908816E-3,-2.6033137E-3,2.0756775E-3,-7.715837E-3,1.513634E-5,3.5040742E-3,1.02624325E-2,-2.4551002E-3,5.9480625E-3,8.493144E-3,2.8816864E-3,2.2447936E-2,1.2759445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,-1,21,23,25,27,29,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.580219E0,5.3780234E-1,7.119478E-1,4.6535176E-1,1.2389976E-1,1.6814464E-1,2.6921844E-1,3.8045776E-1,2.9759058E-1,0E0,0E0,1.3845962E-1,0E0,0E0,2.2763705E-1,1.7757607E-1,1.6110398E-1,2.6813293E-1,2.1900013E-1,1.1446679E-1,1.367331E-1,2.734676E-2,5.160451E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,-1,22,24,26,28,30,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0471938E6,3.7085085E0,1.2711891E0,6.846054E7,2.5234535E1,1.25440235E1,1E0,5.3091907E2,1.7303246E7,-1.5506827E-2,-4.281715E-3,2.247882E0,-4.721945E-3,-6.0996685E-5,7.2610445E8,3.5483692E7,5.101512E0,6.75553E5,3.135E3,2.7307575E5,1.19881555E-1,3E1,3.0256798E0,-1.40729165E-2,-6.079914E-3,-4.7486667E-3,3.2908816E-3,-2.6033137E-3,2.0756775E-3,-7.715837E-3,1.513634E-5,3.5040742E-3,1.02624325E-2,-2.4551002E-3,5.9480625E-3,8.493144E-3,2.8816864E-3,2.2447936E-2,1.2759445E-2],"split_indices":[67,72,58,7,72,76,28,4,64,0,0,72,0,0,12,7,75,9,2,47,57,8,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,1.96E2,8.6E1,1.84E2,1.2E1,5.1E1,3.5E1,4.8E1,1.36E2,8E0,4E0,4.6E1,5E0,4E0,3.1E1,2.8E1,2E1,1.03E2,3.3E1,2.9E1,1.7E1,1.3E1,1.8E1,1.2E1,1.6E1,1.2E1,8E0,4.6E1,5.7E1,2.1E1,1.2E1,2.2E1,7E0,1.2E1,5E0,9E0,4E0,4E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.3999596E-3,-5.3627178E-2,1.7385219E-1,-9.98431E-2,3.265941E-2,8.181625E-2,2.6619783E-1,-3.333813E-2,-1.4404352E-1,9.161319E-2,-9.618459E-3,-2.5562001E-3,1.0132802E-1,2.0116904E-1,2.0454504E-2,-1.1670704E-1,2.830405E-2,-2.3186274E-2,-1.1914832E-1,4.2665742E-2,1.7672983E-1,-1.1445278E-1,3.8036346E-2,-7.920985E-3,1.3967207E-1,2.1414132E-1,4.781695E-3,-7.895158E-3,-2.1935427E-3,3.466034E-3,-3.3377171E-3,-1.1594231E-2,-3.9683166E-3,-1.0427313E-3,4.3066004E-3,1.2174628E-2,2.7428912E-3,-7.4464804E-3,1.382859E-4,4.7936807E-3,-1.4006656E-3,1.0964437E-3,-1.7528867E-3,3.313915E-3,7.335737E-3,5.9034945E-3,1.0675498E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,-1,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6284397E0,8.407019E-1,5.4766154E-1,3.9791596E-1,1.8532175E-1,9.941433E-2,2.9275393E-1,2.910896E-1,6.739975E-1,1.21613204E-1,2.2417118E-1,0E0,1.3096684E-1,1.0581255E-2,0E0,8.261943E-2,1.6172038E-1,0E0,3.1936193E-1,7.189447E-2,9.726593E-2,7.823719E-2,1.4340574E-1,9.518452E-3,1.913932E-2,3.7784576E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,-1,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3394321E0,1.781973E6,2.174459E5,9.0649E4,1.3844932E7,3.9579918E4,1.7598395E3,1.09E3,2.4540474E8,1.0705044E3,1.4813267E3,-2.5562001E-3,1E0,4.7201157E-1,2.0454504E-2,1.3692686E4,5.2158275E0,-2.3186274E-2,9.723579E1,4.354824E5,1.405115E3,2.3684846E7,8.652176E6,1.8454477E2,2.300289E8,3.5055136E2,4.781695E-3,-7.895158E-3,-2.1935427E-3,3.466034E-3,-3.3377171E-3,-1.1594231E-2,-3.9683166E-3,-1.0427313E-3,4.3066004E-3,1.2174628E-2,2.7428912E-3,-7.4464804E-3,1.382859E-4,4.7936807E-3,-1.4006656E-3,1.0964437E-3,-1.7528867E-3,3.313915E-3,7.335737E-3,5.9034945E-3,1.0675498E-2],"split_indices":[61,1,47,48,64,47,4,2,5,74,74,0,29,57,0,67,73,0,71,47,4,64,48,74,7,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.09E2,6.6E1,1.36E2,7.3E1,3.4E1,3.2E1,5.5E1,8.1E1,3E1,4.3E1,4E0,3E1,2.5E1,7E0,2.3E1,3.2E1,4E0,7.7E1,2E1,1E1,1.3E1,3E1,8E0,2.2E1,2.1E1,4E0,1.2E1,1.1E1,2.2E1,1E1,1.4E1,6.3E1,9E0,1.1E1,5E0,5E0,9E0,4E0,1.5E1,1.5E1,4E0,4E0,6E0,1.6E1,5E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[2.8314502E-3,-6.785432E-2,1.0660577E-1,-1.0692218E-1,4.023653E-2,8.684095E-2,1.785696E-2,-7.905963E-2,-1.95647E-1,1.3693513E-2,7.0827315E-3,1.888039E-2,1.2837237E-1,-1.00642435E-1,7.501121E-2,-2.3125635E-1,6.6332467E-4,6.9846846E-2,-3.7670296E-2,5.4251872E-2,-6.1477557E-2,1.6090888E-1,9.190066E-3,7.925395E-3,-5.379347E-3,2.5073162E-4,5.023686E-3,-4.3994854E-3,-1.5595466E-2,4.4568987E-3,-6.541401E-4,-3.7478583E-3,1.4592746E-3,7.066802E-3,1.2514908E-3,9.802931E-4,-5.334769E-3,1.1568111E-2,4.988328E-3,1.946574E-3,-3.0131019E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,-1,13,15,-1,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.009789E0,6.9263124E-1,5.83336E-1,2.8118432E-1,3.7306362E-1,2.9393852E-1,0E0,3.1680053E-1,2.127688E-1,0E0,1.1498651E-1,1.1987284E-1,2.5057673E-1,3.8237488E-1,2.7873583E-2,3.1765282E-1,0E0,3.9893344E-2,7.53841E-2,7.373153E-2,6.0993165E-2,2.1497655E-1,3.9810337E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,-1,14,16,-1,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.259334E-1,3.343327E6,4.328683E0,4.352025E0,2.207E3,3.4071463E2,1.785696E-2,2.2615366E0,2.2291185E2,1.3693513E-2,1.1572553E7,1.5E1,6.464779E6,6.7473096E9,3.1676166E2,7.6630195E6,6.6332467E-4,1.9879582E7,2.0382197E0,1.1623532E6,1.101829E6,4.0012E4,9.723711E0,7.925395E-3,-5.379347E-3,2.5073162E-4,5.023686E-3,-4.3994854E-3,-1.5595466E-2,4.4568987E-3,-6.541401E-4,-3.7478583E-3,1.4592746E-3,7.066802E-3,1.2514908E-3,9.802931E-4,-5.334769E-3,1.1568111E-2,4.988328E-3,1.946574E-3,-3.0131019E-3],"split_indices":[61,1,54,73,0,71,0,72,71,0,64,3,48,50,4,64,0,51,73,64,64,9,76,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,1.62E2,1.1E2,1.19E2,4.3E1,1.04E2,6E0,9.2E1,2.7E1,4E0,3.9E1,4E1,6.4E1,8.1E1,1.1E1,2.3E1,4E0,1.6E1,2.3E1,2.8E1,1.2E1,5E1,1.4E1,4E0,7.7E1,4E0,7E0,1.1E1,1.2E1,1.2E1,4E0,1.4E1,9E0,5E0,2.3E1,5E0,7E0,1.7E1,3.3E1,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[8.826118E-3,-6.6067144E-2,1.15737684E-1,-1.11695945E-1,-1.0645387E-3,3.6324617E-2,2.0036052E-1,-8.591993E-2,-2.80672E-1,2.1383706E-1,-3.3828095E-2,-7.3578716E-3,7.096409E-2,1.2487603E-1,3.7183142E-1,-7.113199E-2,-1.21476175E-2,-1.9641371E-2,-5.3172866E-3,1.393858E-2,3.7829317E-3,-1.2053422E-1,7.177866E-3,2.015588E-2,-6.098117E-3,-1.5559113E-2,1.0925936E-1,-1.1436571E-3,1.5840265E-1,2.49162E-2,2.4566016E-1,-4.057148E-3,4.0760175E-3,-2.7465879E-3,-1.11647835E-2,4.008514E-3,-8.292376E-4,-3.6880438E-4,4.663739E-3,1.5948742E-4,-1.7992699E-3,1.469273E-3,8.172358E-3,7.945264E-3,2.181881E-3,1.4243299E-2,6.4925323E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,37,-1,39,41,-1,43,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1775312E0,4.7443968E-1,7.453798E-1,3.8530433E-1,4.789767E-1,8.97973E-2,6.525104E-1,1.9961512E-1,2.399984E-1,8.048308E-2,2.1063778E-1,9.559665E-2,1.1104317E-1,1.9969398E-1,2.0891309E-1,2.1936658E-1,0E0,0E0,0E0,0E0,0E0,1.2660754E-1,8.406933E-2,5.337534E-2,0E0,5.057159E-3,1.1573723E-1,0E0,4.21471E-2,0E0,2.9699922E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,21,21,22,22,23,23,25,25,26,26,28,28,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,38,-1,40,42,-1,44,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.4269168E-1,1.6408995E11,2.216135E5,4.33435E5,1.904E3,7.025488E6,1.3662124E3,3.5300907E-1,6.132018E2,6.878795E7,1.9462195E2,8.6788906E5,1.8491526E0,1.652E3,3.0256798E0,8.749103E5,-1.21476175E-2,-1.9641371E-2,-5.3172866E-3,1.393858E-2,3.7829317E-3,1.591046E0,5.658968E4,5.636998E5,-6.098117E-3,7.980742E6,5.792986E4,-1.1436571E-3,1.3589149E6,2.49162E-2,1.91016E5,-4.057148E-3,4.0760175E-3,-2.7465879E-3,-1.11647835E-2,4.008514E-3,-8.292376E-4,-3.6880438E-4,4.663739E-3,1.5948742E-4,-1.7992699E-3,1.469273E-3,8.172358E-3,7.945264E-3,2.181881E-3,1.4243299E-2,6.4925323E-3],"split_indices":[61,50,47,48,0,70,4,61,4,7,71,66,72,2,73,52,0,0,0,0,0,73,52,66,0,48,47,0,66,0,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,1.59E2,1.11E2,9.3E1,6.6E1,5.8E1,5.3E1,8.2E1,1.1E1,8E0,5.8E1,2.6E1,3.2E1,3.8E1,1.5E1,7.7E1,5E0,5E0,6E0,4E0,4E0,1.8E1,4E1,2.2E1,4E0,1E1,2.2E1,7E0,3.1E1,5E0,1E1,7E1,7E0,1.3E1,5E0,9E0,3.1E1,1.7E1,5E0,6E0,4E0,1.1E1,1.1E1,2.7E1,4E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[5.0148596E-3,-4.0661376E-2,1.4523451E-1,-9.008749E-2,1.707953E-2,1.1323153E-1,1.7454334E-2,-6.537531E-2,-1.2222227E-2,-6.785297E-2,3.529285E-2,5.6190148E-2,1.7817006E-1,-9.044338E-2,4.3179322E-2,-3.4122176E-2,-5.9681884E-3,2.0582078E-2,1.2971044E-1,8.834468E-2,-5.0209495E-3,1.4130776E-3,2.0053892E-1,5.12178E-3,-4.944072E-3,3.5777597E-3,-3.4967444E-3,-4.571528E-3,2.7048524E-4,1.16039504E-4,5.1651546E-3,8.40273E-3,3.3826302E-3,5.4376996E-3,-1.9528405E-3,-1.7457104E-3,2.541237E-3,1.0513924E-2,2.0066393E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,27,-1,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8248419E0,6.1512905E-1,4.809891E-1,4.8538065E-1,1.57085E-1,2.2335213E-1,0E0,2.8419453E-1,0E0,3.3083506E-2,1.1261358E-1,6.933108E-2,8.951378E-2,2.989486E-1,8.8168316E-2,3.4469355E-2,0E0,1.204974E-1,1.9252628E-2,9.451428E-2,2.7843595E-2,0E0,9.9212885E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,28,-1,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2185057E0,1.1670399E-1,1.3821641E0,4.4326835E0,1.503E3,2.0074467E5,1.7454334E-2,4.5E1,-1.2222227E-2,3.68646E5,1.08540284E12,1.948181E0,1.8618844E0,1E1,4.7907645E8,9.57E2,-5.9681884E-3,4.7321553E5,2.4434419E12,4.2656995E6,3.1889508E0,1.4130776E-3,4.9691647E-1,5.12178E-3,-4.944072E-3,3.5777597E-3,-3.4967444E-3,-4.571528E-3,2.7048524E-4,1.16039504E-4,5.1651546E-3,8.40273E-3,3.3826302E-3,5.4376996E-3,-1.9528405E-3,-1.7457104E-3,2.541237E-3,1.0513924E-2,2.0066393E-3],"split_indices":[61,58,53,73,2,47,0,8,0,47,50,61,72,10,12,2,0,47,50,66,61,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,2.14E2,6.9E1,1.15E2,9.9E1,6.2E1,7E0,1.02E2,1.3E1,1.7E1,8.2E1,3.4E1,2.8E1,8.3E1,1.9E1,1.2E1,5E0,7.2E1,1E1,2.2E1,1.2E1,4E0,2.4E1,6E0,7.7E1,1.5E1,4E0,4E0,8E0,6.1E1,1.1E1,4E0,6E0,1.8E1,4E0,8E0,4E0,2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.2870723E-2,-4.9312957E-2,1.5442432E-1,-1.2296996E-1,-5.5058524E-3,1.9981392E-2,1.19145766E-1,-3.973438E-2,-1.8724707E-1,-1.9829726E-2,1.23802274E-1,-4.528465E-3,1.6050774E-1,-8.023268E-2,7.763429E-2,-2.2347286E-2,-1.472971E-1,-3.8787395E-2,7.155142E-2,2.0040686E-3,1.6356274E-1,2.520448E-3,-1.8880231E-3,-3.815458E-4,1.8407553E-1,-8.574226E-3,-1.3131945E-3,-4.6848468E-4,6.347679E-3,-9.633589E-3,-3.471098E-3,-3.8900268E-3,2.7353826E-4,4.9813315E-3,-3.4378606E-3,9.837722E-3,3.717361E-3,9.444492E-3,1.9327286E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6354089E0,7.09197E-1,4.217013E-1,4.3127143E-1,2.5990984E-1,0E0,2.2580492E-1,1.8209839E-1,4.7637892E-1,2.2105438E-1,3.756495E-2,2.819101E-2,1.328035E-1,1.4816095E-1,5.429561E-2,0E0,1.7129254E-1,2.151461E-1,1.2595034E-1,0E0,2.010025E-2,0E0,0E0,0E0,7.815844E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,5.283708E0,1.6771E4,1.56221E5,6.767991E5,1.9981392E-2,1.2358527E8,1E0,9.975593E4,9.821357E-1,6.5284515E6,1.9066478E0,1E0,4.780434E-3,1.073E3,-2.2347286E-2,4.24048E5,2.8352614E9,5.2380065E6,2.0040686E-3,1E0,2.520448E-3,-1.8880231E-3,-3.815458E-4,5.557553E-1,-8.574226E-3,-1.3131945E-3,-4.6848468E-4,6.347679E-3,-9.633589E-3,-3.471098E-3,-3.8900268E-3,2.7353826E-4,4.9813315E-3,-3.4378606E-3,9.837722E-3,3.717361E-3,9.444492E-3,1.9327286E-3],"split_indices":[61,75,9,1,47,0,5,8,47,60,64,72,28,60,2,0,1,5,66,0,22,0,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.19E2,4.7E1,8.1E1,1.38E2,4E0,4.3E1,3.6E1,4.5E1,1.25E2,1.3E1,1.1E1,3.2E1,2.7E1,9E0,4E0,4.1E1,1.04E2,2.1E1,5E0,8E0,4E0,7E0,4E0,2.8E1,8E0,1.9E1,4E0,5E0,2.1E1,2E1,5.1E1,5.3E1,1.7E1,4E0,4E0,4E0,2.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.1806652E-3,-5.6144875E-2,9.210386E-2,-8.710738E-2,1.0394218E-3,6.804833E-2,1.834925E-2,-1.2235583E-1,-5.7547363E-3,2.2315871E-2,-1.2200666E-1,1.4027314E-1,2.7860343E-2,-2.0972583E-1,-6.398443E-2,7.5510456E-3,-3.38565E-2,7.849221E-2,-1.5247225E-2,-1.3981955E-3,-8.7129725E-3,7.693887E-2,1.7759348E-1,-9.5497906E-2,6.6637225E-2,-1.3643722E-2,-2.5374359E-3,-8.6167234E-4,-6.5732924E-3,-3.6350319E-3,1.8584944E-3,-7.659624E-5,5.890809E-3,-2.870576E-3,2.291214E-3,-1.3748664E-4,5.0280285E-3,9.0616075E-3,3.7092215E-3,3.810046E-5,-8.282146E-3,-2.3530498E-4,5.318013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,-1,29,31,33,-1,-1,35,37,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3690357E0,2.975462E-1,7.0265937E-1,3.118593E-1,1.5980172E-1,2.6603094E-1,0E0,3.7417376E-1,1.6775301E-1,1.1092497E-1,4.9706176E-2,6.433684E-2,2.9815406E-1,3.8541102E-1,1.6495657E-1,0E0,1.0359995E-1,8.345856E-2,1.0109051E-1,0E0,0E0,3.6433116E-2,2.1446764E-2,1.22079834E-1,1.6538303E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,-1,30,32,34,-1,-1,36,38,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,2.0283392E8,1.3821641E0,2.951829E9,7.807318E2,7.3917985E6,1.834925E-2,7.5646E4,2.6017979E11,4.53E2,7E0,2.744E3,1.299938E6,1.8196162E0,4.4326835E0,7.5510456E-3,4.5E1,2.25E2,2.636E3,-1.3981955E-3,-8.7129725E-3,2.1120381E0,2.5497077E0,1.1936242E7,6.2143734E4,-1.3643722E-2,-2.5374359E-3,-8.6167234E-4,-6.5732924E-3,-3.6350319E-3,1.8584944E-3,-7.659624E-5,5.890809E-3,-2.870576E-3,2.291214E-3,-1.3748664E-4,5.0280285E-3,9.0616075E-3,3.7092215E-3,3.810046E-5,-8.282146E-3,-2.3530498E-4,5.318013E-3],"split_indices":[71,7,53,5,4,64,0,9,50,0,3,2,48,72,73,0,8,0,10,0,0,72,72,64,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.65E2,1.67E2,9.8E1,1.08E2,5.9E1,9.2E1,6E0,7.5E1,3.3E1,5.1E1,8E0,3.2E1,6E1,2.9E1,4.6E1,4E0,2.9E1,2E1,3.1E1,4E0,4E0,1.3E1,1.9E1,1.4E1,4.6E1,1.8E1,1.1E1,3E1,1.6E1,1.8E1,1.1E1,8E0,1.2E1,1.8E1,1.3E1,4E0,9E0,1.5E1,4E0,7E0,7E0,1.9E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[4.9337335E-3,-3.9891645E-2,1.4060307E-1,-8.3292015E-2,1.1802642E-2,7.7317625E-2,2.5395557E-1,-1.2125488E-1,5.492631E-3,-7.631845E-2,3.3685382E-2,5.234726E-2,1.2734922E-2,1.9376314E-1,2.0993467E-2,-7.747123E-2,-2.0292796E-1,5.597367E-3,-1.7151792E-2,-9.814065E-3,-1.6494332E-2,6.527197E-3,9.31723E-2,-1.8495886E-2,9.937916E-2,1.2404649E-2,1.0048562E-1,-9.509178E-3,-1.5011583E-3,-1.8705236E-3,-1.1744777E-2,1.6511722E-3,-3.8715273E-3,8.897604E-4,-2.3136665E-3,-2.1418722E-3,2.067697E-3,5.5621294E-3,-6.664678E-4,-3.954101E-3,6.4587255E-4,1.5890079E-4,6.904385E-3,6.5283575E-3,1.0385205E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,-1,31,-1,33,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.7388164E0,4.8334655E-1,4.896878E-1,3.950168E-1,1.9347557E-1,2.1939912E-1,2.1564698E-1,2.7750647E-1,9.671413E-2,1.602114E-1,1.2863638E-1,1.4510925E-1,0E0,1.21136725E-1,0E0,3.1175548E-1,2.2669852E-1,0E0,1.13023594E-1,0E0,1.9109117E-2,1.1611238E-1,7.612191E-2,4.0654875E-2,1.2564111E-1,0E0,3.2212287E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,-1,32,-1,34,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,2.7558826E-2,2.8041378E5,4.5537444E9,1.8681856E8,1.3333261E2,2.9409406E-1,5.1653096E-3,2.6017979E11,8E0,2.704375E6,2E0,1.2734922E-2,4.0012E4,2.0993467E-2,3.3617156E-4,1.609E3,5.597367E-3,1.710607E7,-9.814065E-3,1.7011462E5,8.749103E5,2.8808496E7,1.479938E6,1.19881555E-1,1.2404649E-2,2.6674094E7,-9.509178E-3,-1.5011583E-3,-1.8705236E-3,-1.1744777E-2,1.6511722E-3,-3.8715273E-3,8.897604E-4,-2.3136665E-3,-2.1418722E-3,2.067697E-3,5.5621294E-3,-6.664678E-4,-3.954101E-3,6.4587255E-4,1.5890079E-4,6.904385E-3,6.5283575E-3,1.0385205E-3],"split_indices":[71,57,47,5,7,75,53,57,50,3,9,72,0,9,0,58,2,0,64,0,47,52,9,51,57,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,2.14E2,7E1,1.16E2,9.8E1,4.6E1,2.4E1,8.1E1,3.5E1,1.9E1,7.9E1,4.2E1,4E0,2E1,4E0,5.4E1,2.7E1,5E0,3E1,5E0,1.4E1,5.5E1,2.4E1,1.7E1,2.5E1,1E1,1E1,1.3E1,4.1E1,7E0,2E1,1.7E1,1.3E1,7E0,7E0,2.3E1,3.2E1,1.9E1,5E0,5E0,1.2E1,9E0,1.6E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[8.771363E-3,-2.8280202E-2,1.6915616E-1,-8.118126E-2,1.4528615E-2,1.08422175E-1,3.0768958E-1,-1.1792901E-1,-4.11214E-3,2.0634761E-1,-6.0964277E-4,1.5790923E-2,1.421452E-1,2.0596057E-2,1.931297E-1,-8.907537E-2,-2.9397744E-1,-2.9001337E-2,5.189877E-3,1.1955125E-2,5.145708E-3,-8.2372695E-2,2.4201728E-2,2.3687992E-3,-1.716851E-3,1.7574988E-1,9.243631E-2,3.3352436E-3,1.2040457E-2,-2.9449489E-3,-1.0197102E-2,-1.6838739E-2,-7.5243185E-3,2.3902429E-3,-3.537763E-3,9.6614705E-4,-6.2255375E-3,1.7316738E-3,-3.93636E-3,9.433623E-3,3.5211614E-3,1.2946448E-3,5.428114E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,-1,-1,35,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6152034E0,5.016631E-1,3.9573765E-1,2.7989465E-1,3.572939E-1,1.13336265E-1,1.535821E-1,3.1414956E-1,9.923074E-2,1.636371E-2,2.3528871E-1,2.2739183E-2,2.8856456E-2,0E0,6.358999E-2,1.8200159E-1,2.453971E-2,1.1237423E-1,0E0,0E0,0E0,1.5030728E-1,1.341376E-1,0E0,0E0,2.7730197E-2,1.7973207E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,21,21,22,22,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,-1,-1,36,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,2.0139322E2,1.623319E3,1.848519E-2,2.8558656E4,1.7518893E2,2.7479675E-1,4.583037E-2,1.3431462E7,6.878795E7,8.12E2,6.6875E-1,3.13423E6,2.0596057E-2,3.91101E5,4.4326835E0,1.3628491E11,2.5187686E-1,5.189877E-3,1.1955125E-2,5.145708E-3,1.0706395E1,1.5830926E1,2.3687992E-3,-1.716851E-3,4.7201157E-1,2.300289E8,3.3352436E-3,1.2040457E-2,-2.9449489E-3,-1.0197102E-2,-1.6838739E-2,-7.5243185E-3,2.3902429E-3,-3.537763E-3,9.6614705E-4,-6.2255375E-3,1.7316738E-3,-3.93636E-3,9.433623E-3,3.5211614E-3,1.2946448E-3,5.428114E-3],"split_indices":[61,71,4,57,52,71,57,61,9,7,0,75,64,0,9,73,50,61,0,0,0,75,76,0,0,57,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.7E2,2.2E2,5E1,9.8E1,1.22E2,3.6E1,1.4E1,6.6E1,3.2E1,8E0,1.14E2,1E1,2.6E1,5E0,9E0,5.8E1,8E0,2.7E1,5E0,4E0,4E0,2.6E1,8.8E1,6E0,4E0,1.4E1,1.2E1,4E0,5E0,5E1,8E0,4E0,4E0,1E1,1.7E1,9E0,1.7E1,7.9E1,9E0,1E1,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[7.593201E-3,-2.7012795E-2,1.4464566E-1,-6.884295E-2,3.174992E-2,3.0718607E-1,9.333496E-2,-1.02020964E-1,-6.039154E-3,4.2197295E-2,-5.83662E-3,1.8307963E-2,7.957926E-3,3.1013912E-2,1.5424629E-1,-6.2414363E-2,-2.826908E-1,1.1083914E-2,-7.766553E-2,1.047227E-2,2.8296769E-2,5.7203073E-2,-3.4396206E-3,1.8428743E-1,2.9257082E-3,-6.063977E-3,-8.95553E-4,-1.620407E-2,-3.3971935E-3,5.3111142E-3,-2.1069408E-4,-5.1049036E-3,-1.3315418E-3,-3.917683E-3,1.99068E-3,-6.94043E-4,3.5673548E-3,4.5531774E-3,9.415893E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,-1,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3176732E0,5.4787356E-1,4.3837214E-1,2.7026558E-1,1.5936816E-1,8.924997E-2,1.622428E-1,5.90592E-1,5.727815E-2,2.2017954E-1,0E0,0E0,0E0,6.795497E-2,4.8233807E-2,2.0881054E-1,1.9105434E-1,6.257869E-2,1.1363413E-2,0E0,1.4428504E-1,2.901655E-2,0E0,7.4062347E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,-1,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,2.426E3,7.013774E5,1.848519E-2,3.8016708E7,3.0256798E0,2.0787016E5,5.740651E-3,2.3424656E0,4.92E2,-5.83662E-3,1.8307963E-2,7.957926E-3,1.10018056E8,9.042351E8,4.949154E-4,2.2394615E2,1.3523328E9,1.8E1,1.047227E-2,2E0,1.647E3,-3.4396206E-3,3.6893954E2,2.9257082E-3,-6.063977E-3,-8.95553E-4,-1.620407E-2,-3.3971935E-3,5.3111142E-3,-2.1069408E-4,-5.1049036E-3,-1.3315418E-3,-3.917683E-3,1.99068E-3,-6.94043E-4,3.5673548E-3,4.5531774E-3,9.415893E-3],"split_indices":[61,2,52,57,64,73,47,57,72,10,0,0,0,51,5,58,71,50,3,0,8,2,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.21E2,5.5E1,1.29E2,9.2E1,1.2E1,4.3E1,8.4E1,4.5E1,8.7E1,5E0,6E0,6E0,2.2E1,2.1E1,7E1,1.4E1,3.7E1,8E0,5E0,8.2E1,1.8E1,4E0,1.5E1,6E0,2.6E1,4.4E1,1E1,4E0,4E0,3.3E1,4E0,4E0,9E0,7.3E1,4E0,1.4E1,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[3.2400521E-3,-3.636617E-2,1.23159766E-1,-5.4971147E-2,8.100857E-2,1.9564688E-1,5.1278602E-2,-9.992361E-2,-1.8375896E-2,2.800381E-2,1.22324705E-1,1.650536E-2,1.4794816E-1,7.042806E-2,-2.5878157E-3,-3.3005163E-2,-1.4763047E-1,-4.3446276E-2,8.5331455E-2,-1.3967525E-2,4.896269E-3,6.936388E-2,8.310783E-3,2.583225E-4,1.7864713E-1,8.586292E-2,-1.2433751E-3,-5.0911373E-3,3.2949608E-4,-1.4714035E-2,-5.0437413E-3,-4.2766487E-4,-4.8597143E-3,4.836669E-3,2.3766013E-4,1.0423907E-3,-2.5370282E-3,4.3165586E-3,1.2073019E-3,9.2616435E-3,2.6412285E-3,8.2868186E-4,4.8064836E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,35,-1,37,-1,-1,39,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3201089E0,4.6085787E-1,3.494891E-1,2.959172E-1,6.0083047E-2,2.110573E-1,7.805679E-2,2.5371742E-1,2.663056E-1,4.7518987E-2,3.6782295E-2,0E0,1.2134683E-1,4.8939005E-2,0E0,1.0983272E-1,2.72761E-1,1.7297171E-1,3.1076223E-2,1.6306661E-2,0E0,8.064065E-3,0E0,0E0,5.1259995E-2,3.2079056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,21,21,24,24,25,25],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,36,-1,38,-1,-1,40,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,1.2730925E0,9.520617E5,5.283708E0,1.651875E2,2.3842E4,2.008498E1,1.56221E5,5.8536225E5,2.069532E6,1.2311872E6,1.650536E-2,3.302E3,1.7021878E7,-2.5878157E-3,1.36877295E4,2.07684E5,2.142943E1,4.986835E6,2.2998267E2,4.896269E-3,3.1225768E-1,8.310783E-3,2.583225E-4,9.299407E9,1.5552E4,-1.2433751E-3,-5.0911373E-3,3.2949608E-4,-1.4714035E-2,-5.0437413E-3,-4.2766487E-4,-4.8597143E-3,4.836669E-3,2.3766013E-4,1.0423907E-3,-2.5370282E-3,4.3165586E-3,1.2073019E-3,9.2616435E-3,2.6412285E-3,8.2868186E-4,4.8064836E-3],"split_indices":[71,61,52,75,71,9,77,1,47,9,52,0,2,9,0,67,1,75,1,4,0,57,0,0,12,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.08E2,6.8E1,1.8E2,2.8E1,3.3E1,3.5E1,8E1,1E2,1.3E1,1.5E1,6E0,2.7E1,3E1,5E0,3.4E1,4.6E1,8.1E1,1.9E1,9E0,4E0,9E0,6E0,5E0,2.2E1,2.6E1,4E0,1.1E1,2.3E1,7E0,3.9E1,5.3E1,2.8E1,1.5E1,4E0,5E0,4E0,5E0,4E0,1.8E1,4E0,6E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[-2.3484013E-4,-3.591501E-2,1.0267224E-1,-7.48736E-2,1.1144335E-2,3.864537E-2,1.7134868E-1,-1.8937495E-2,-1.23732276E-1,-7.1473983E-3,1.12259395E-1,6.948704E-3,2.0462375E-2,1.1614703E-1,2.8669775E-1,-7.037037E-2,4.3999624E-2,-1.917723E-2,-9.433054E-2,1.989276E-2,-7.688111E-2,6.9479677E-3,2.3965482E-3,5.5984515E-3,4.7229216E-3,8.152174E-4,1.3494985E-1,1.587437E-2,7.350493E-3,1.9360607E-3,-4.1455263E-3,8.52789E-3,-1.3412618E-4,-1.037372E-2,-2.5591059E-3,2.9748275E-3,-6.7167636E-4,-5.3989706E-3,3.8616697E-3,1.2316433E-3,-2.978971E-3,6.8428447E-3,2.9558314E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,33,35,37,-1,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.840427E-1,3.656741E-1,2.9678053E-1,2.9554993E-1,1.6911322E-1,7.244958E-2,1.7682707E-1,1.7142218E-1,4.6000797E-1,1.4860699E-1,2.4721757E-2,0E0,3.9705504E-2,4.2033643E-2,1.8755376E-2,6.791268E-2,1.595447E-1,0E0,2.6154205E-1,8.883655E-2,1.5265547E-1,0E0,0E0,4.4855084E-2,0E0,0E0,9.808391E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,34,36,38,-1,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,2.8704E4,2.174459E5,9.0649E4,5.8536225E5,1.1832633E0,1.6507233E3,1.175E3,2.4540474E8,2.1935484E0,5.8816573E9,6.948704E-3,1.1054057E0,1.8618844E0,2.9375613E-1,2E0,5.039375E-3,-1.917723E-2,9.526745E1,1.8400878E7,1.6507233E3,6.9479677E-3,2.3965482E-3,1.10018056E8,4.7229216E-3,8.152174E-4,4.7201157E-1,1.587437E-2,7.350493E-3,1.9360607E-3,-4.1455263E-3,8.52789E-3,-1.3412618E-4,-1.037372E-2,-2.5591059E-3,2.9748275E-3,-6.7167636E-4,-5.3989706E-3,3.8616697E-3,1.2316433E-3,-2.978971E-3,6.8428447E-3,2.9558314E-3],"split_indices":[61,11,47,48,47,58,4,2,5,72,5,0,53,72,57,10,57,0,71,64,4,0,0,51,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,1.98E2,6.8E1,1.08E2,9E1,3.6E1,3.2E1,5.1E1,5.7E1,7.7E1,1.3E1,4E0,3.2E1,2.3E1,9E0,2.8E1,2.3E1,4E0,5.3E1,5.6E1,2.1E1,7E0,6E0,2.8E1,4E0,4E0,1.9E1,5E0,4E0,4E0,2.4E1,5E0,1.8E1,1.1E1,4.2E1,2.4E1,3.2E1,1.7E1,4E0,2.2E1,6E0,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.4413285E-3,-1.559685E-2,1.6814202E-1,-4.9367003E-2,4.32944E-2,2.9527488E-1,4.6075262E-2,-8.067344E-2,5.025364E-4,5.6911945E-2,-8.256091E-2,3.4786907E-1,5.548531E-3,9.937335E-2,-3.1400055E-3,-8.081147E-3,-1.2765297E-1,-6.9893585E-3,1.4092635E-2,1.614593E-2,9.161419E-2,-9.620789E-5,-6.7459373E-3,2.0643875E-2,1.0422784E-2,2.253422E-3,7.3751337E-3,5.108412E-3,-1.5592917E-3,-1.8470878E-2,-4.5783105E-3,1.5986183E-3,-3.880003E-3,-5.903157E-4,4.0706466E-3,8.472347E-3,2.8426806E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.268126E-1,4.794413E-1,4.6747458E-1,2.3890993E-1,1.5551664E-1,9.594965E-2,1.1031847E-1,3.195415E-1,1.2633352E-1,1.11694574E-1,4.601569E-2,3.2815814E-2,0E0,2.8961003E-2,0E0,1.2039707E-1,3.9998478E-1,0E0,1.16559744E-1,8.068293E-2,1.0732925E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,4.1677386E-1,8.982092E5,1.6408995E11,1E0,2.660842E6,2.008498E1,9.0649E4,5.7060448E1,3.4071463E2,3.8773947E0,3.0256798E0,5.548531E-3,2.2663E4,-3.1400055E-3,9.042859E4,2.4540474E8,-6.9893585E-3,6.604274E5,9.750871E6,8E0,-9.620789E-5,-6.7459373E-3,2.0643875E-2,1.0422784E-2,2.253422E-3,7.3751337E-3,5.108412E-3,-1.5592917E-3,-1.8470878E-2,-4.5783105E-3,1.5986183E-3,-3.880003E-3,-5.903157E-4,4.0706466E-3,8.472347E-3,2.8426806E-3],"split_indices":[71,61,52,50,24,48,77,48,71,71,73,73,0,9,0,47,5,0,52,9,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,2.39E2,3E1,1.52E2,8.7E1,1.4E1,1.6E1,9.3E1,5.9E1,7.9E1,8E0,1E1,4E0,1.1E1,5E0,3.7E1,5.6E1,4E0,5.5E1,3.7E1,4.2E1,4E0,4E0,4E0,6E0,7E0,4E0,6E0,3.1E1,4E0,5.2E1,4.6E1,9E0,2.7E1,1E1,9E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[-2.0476116E-3,-4.7822103E-2,7.8535974E-2,-8.6223185E-2,-1.777505E-2,1.2548237E-1,7.918414E-3,-1.21029675E-1,2.6398191E-2,-2.1910308E-4,-1.465004E-1,8.9054085E-2,2.2009426E-1,-1.13999195E-1,6.859442E-2,-1.8277246E-1,-5.525257E-2,6.8731145E-3,-1.6323056E-2,4.1211545E-2,-4.549867E-2,-2.511922E-3,-8.547598E-3,1.6610068E-1,5.9241936E-2,1.3198484E-2,1.3646294E-1,-1.058897E-3,-1.480994E-1,-1.0688999E-2,1.9209506E-1,-4.4334037E-3,-1.5934963E-2,-8.0197083E-4,-6.5192487E-3,1.9558193E-3,-3.3649395E-3,1.0611853E-2,8.399912E-4,1.2107608E-3,-4.3027075E-3,9.233111E-3,3.5809043E-3,4.423011E-3,7.719349E-4,7.875456E-3,3.4337612E-3,-3.8125243E-3,-9.062275E-3,-4.08997E-3,1.6476169E-3,2.997242E-3,1.1757235E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,-1,-1,41,43,-1,45,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0181408E0,2.015431E-1,3.3053035E-1,3.0503148E-1,2.2595742E-1,1.8719256E-1,3.111161E-1,2.2951579E-1,1.02182224E-1,1.6883568E-1,3.306693E-2,9.3521476E-2,4.996246E-2,4.0037632E-2,2.762168E-1,3.9041877E-1,9.553551E-2,0E0,5.317177E-2,2.0010917E-1,1.5050587E-1,0E0,0E0,2.0905524E-2,5.1570624E-2,0E0,6.5074265E-3,0E0,1.5518591E-2,6.855016E-2,7.175991E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24,26,26,28,28,29,29,30,30],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,-1,-1,42,44,-1,46,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,1.83437E5,1.1936242E7,4.5510144E9,3.3457212E0,5.96E3,1.4813267E3,1.8662952E0,2.207E3,1.3E1,2.7181172E5,4.0012E4,2.9375613E-1,2.629593E0,7.12911E4,4.03E2,1.3710982E7,6.8731145E-3,2.5E1,4.5595855E-1,2.043855E6,-2.511922E-3,-8.547598E-3,3.7840486E8,1.3015172E7,1.3198484E-2,1.6E1,-1.058897E-3,7.1376E4,2.51838E6,1.6626391E0,-4.4334037E-3,-1.5934963E-2,-8.0197083E-4,-6.5192487E-3,1.9558193E-3,-3.3649395E-3,1.0611853E-2,8.399912E-4,1.2107608E-3,-4.3027075E-3,9.233111E-3,3.5809043E-3,4.423011E-3,7.719349E-4,7.875456E-3,3.4337612E-3,-3.8125243E-3,-9.062275E-3,-4.08997E-3,1.6476169E-3,2.997242E-3,1.1757235E-2],"split_indices":[71,9,64,5,72,2,74,72,0,3,52,9,57,73,47,10,64,0,3,75,9,0,0,7,51,0,3,0,9,51,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.75E2,9.9E1,7.6E1,9.9E1,5.9E1,4E1,5.8E1,1.8E1,8.8E1,1.1E1,4.4E1,1.5E1,1.3E1,2.7E1,2.9E1,2.9E1,4E0,1.4E1,4.6E1,4.2E1,4E0,7E0,1.1E1,3.3E1,7E0,8E0,4E0,9E0,1.7E1,1E1,2E1,9E0,2.1E1,8E0,7E0,7E0,4E0,4.2E1,1.7E1,2.5E1,7E0,4E0,1.7E1,1.6E1,4E0,4E0,5E0,4E0,6E0,1.1E1,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[1.5396263E-3,-3.8254604E-2,1.0125758E-1,-6.724777E-2,4.454634E-3,7.500814E-2,2.970856E-1,-4.14754E-2,-1.6146196E-1,-8.545177E-2,1.8321054E-2,1.1031076E-1,1.277544E-3,1.8135602E-2,6.485042E-3,-8.671476E-2,3.685543E-3,-4.2059823E-4,-2.2019261E-1,-1.4307218E-3,-6.6524264E-3,-4.0761188E-2,3.354759E-2,7.984688E-2,1.9504853E-1,-4.5407615E-3,4.8214972E-2,-1.1214835E-2,-2.6619704E-3,-3.5607882E-3,1.1218673E-3,-1.1850048E-3,-1.3490167E-2,-2.5607707E-4,-5.0666947E-3,3.920722E-3,-6.1037485E-5,8.668828E-3,2.8083802E-3,1.2345215E-2,3.55773E-3,4.1910266E-3,1.0493E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,31,-1,-1,33,35,37,39,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0990965E0,2.4547821E-1,3.838058E-1,2.798218E-1,1.0254664E-1,1.8466383E-1,8.064836E-2,1.9236648E-1,2.2164458E-1,3.082288E-2,6.524031E-2,1.0833514E-1,1.1729429E-1,0E0,0E0,1.989724E-1,8.231751E-2,0E0,2.4730831E-1,0E0,0E0,3.597024E-2,1.0270113E-1,6.414342E-2,8.257389E-2,0E0,1.642989E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,18,18,21,21,22,22,23,23,24,24,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,32,-1,-1,34,36,38,40,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8045242E2,2.0283392E8,1.9297871E3,1.2869189E5,1.1670399E-1,3.3754787E1,5.1145615E3,6.846054E7,1E0,1.10980764E-1,1.549E3,7.121294E2,6.408734E-2,1.8135602E-2,6.485042E-3,1.393E3,1.9328261E2,-4.2059823E-4,1.9719312E8,-1.4307218E-3,-6.6524264E-3,3.68646E5,2.7379935E6,8E0,8.9156006E5,-4.5407615E-3,1.055304E1,-1.1214835E-2,-2.6619704E-3,-3.5607882E-3,1.1218673E-3,-1.1850048E-3,-1.3490167E-2,-2.5607707E-4,-5.0666947E-3,3.920722E-3,-6.1037485E-5,8.668828E-3,2.8083802E-3,1.2345215E-2,3.55773E-3,4.1910266E-3,1.0493E-3],"split_indices":[71,7,71,66,58,75,4,7,31,61,2,71,57,0,0,12,74,0,5,0,0,47,51,3,52,0,77,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.97E2,7.8E1,1.17E2,8E1,7E1,8E0,9.3E1,2.4E1,1E1,7E1,4.7E1,2.3E1,4E0,4E0,4.6E1,4.7E1,7E0,1.7E1,6E0,4E0,1.4E1,5.6E1,3.6E1,1.1E1,7E0,1.6E1,6E0,4E1,9E0,3.8E1,5E0,1.2E1,1E1,4E0,2.2E1,3.4E1,4E0,3.2E1,6E0,5E0,5E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[5.824203E-3,-3.8133048E-2,8.169153E-2,7.809944E-2,-4.9809888E-2,1.24406025E-1,1.2787035E-2,-3.487356E-3,8.448301E-3,-1.463914E-1,-3.7726346E-2,9.720717E-2,1.4748747E-2,-1.6493153E-2,9.907618E-3,1.9075597E-3,-1.8572357E-3,-1.0458407E-2,-1.249523E-3,-2.4958918E-2,-1.5085043E-1,1.6505769E-1,5.6812238E-2,-9.9708796E-2,2.0781456E-2,-3.9829337E-3,-2.3164117E-4,-9.888598E-3,-1.3728351E-3,9.924459E-3,4.281956E-3,5.149592E-3,5.605506E-4,-9.942665E-4,-6.5037645E-3,4.8944573E-3,-1.5640355E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,-1,23,-1,-1,-1,-1,-1,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.0020853E-1,2.3597609E-1,2.89788E-1,1.3822201E-1,1.7653647E-1,2.9393917E-1,2.3467973E-1,1.8224575E-2,0E0,1.5756387E-1,1.984375E-1,1.4136761E-1,0E0,1.1063616E-1,0E0,0E0,0E0,0E0,0E0,1.5533528E-1,1.0191813E-1,5.2726507E-2,8.707714E-2,3.2652386E-2,1.2067756E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,13,13,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,-1,24,-1,-1,-1,-1,-1,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,1.1623532E6,1.1936242E7,1E0,2.7414852E-1,1.3821E4,1.3339948E2,2.5742075E0,8.448301E-3,4.74406E5,3.7085085E0,4.0012E4,1.4748747E-2,1.4813267E3,9.907618E-3,1.9075597E-3,-1.8572357E-3,-1.0458407E-2,-1.249523E-3,4.0835E4,5.3725034E-2,4.048974E8,8.1309015E6,2.629593E0,6.464779E6,-3.9829337E-3,-2.3164117E-4,-9.888598E-3,-1.3728351E-3,9.924459E-3,4.281956E-3,5.149592E-3,5.605506E-4,-9.942665E-4,-6.5037645E-3,4.8944573E-3,-1.5640355E-3],"split_indices":[71,64,64,8,77,2,75,72,0,1,72,9,0,74,0,0,0,0,0,9,61,7,51,73,48,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,1.7E2,9.8E1,1.5E1,1.55E2,6E1,3.8E1,9E0,6E0,1.6E1,1.39E2,5.4E1,6E0,3.4E1,4E0,4E0,5E0,9E0,7E0,1.26E2,1.3E1,1.9E1,3.5E1,1E1,2.4E1,3E1,9.6E1,8E0,5E0,1E1,9E0,1.5E1,2E1,4E0,6E0,9E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.1614976E-3,-2.7545594E-2,1.0395306E-1,-4.813036E-2,3.6321875E-2,1.7037725E-1,3.0353071E-2,-6.752264E-2,1.3766762E-2,1.6716229E-2,1.0787781E-1,1.5717622E-2,1.1543363E-1,-3.0862438E-3,4.6097018E-2,1.8207029E-2,-8.552788E-2,7.7099584E-2,-2.4783604E-2,4.4019846E-3,9.114382E-4,1.1527641E-3,6.981582E-3,8.797296E-3,1.576483E-1,1.1064257E-3,9.422219E-2,8.200701E-3,-1.8095442E-3,-1.2579008E-2,-3.0592056E-3,6.52015E-3,8.4501604E-4,6.1219186E-4,-4.3259826E-3,-1.4273782E-3,2.4528848E-3,2.8350858E-3,-2.1060195E-3,9.316284E-3,3.7078313E-3,-2.643826E-3,1.5161828E-3,2.573676E-3,6.9992975E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,-1,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.793192E-1,2.791493E-1,3.2237947E-1,1.9298169E-1,7.114751E-2,2.89441E-1,5.3686656E-2,1.8987453E-1,9.712945E-2,5.2282628E-2,3.6079556E-2,0E0,1.24658406E-1,0E0,6.2685974E-2,2.093215E-1,3.4358174E-1,5.4252118E-2,6.710499E-2,0E0,6.1813056E-2,0E0,0E0,2.8724056E-2,5.2643746E-2,3.169401E-2,2.2513032E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,-1,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,6.005889E-1,9.477338E5,3.165913E6,1.3289025E6,1E0,1.9241573E-1,9.338697E9,1E1,1.2415916E6,1.31903E5,1.5717622E-2,4.2403465E6,-3.0862438E-3,6.2143734E4,1.7690372E0,2.6712021E8,4.587251E6,4.99E3,4.4019846E-3,2.424E3,1.1527641E-3,6.981582E-3,5.5053414E9,2.6004646E7,2.3361564E0,8.0318604E2,8.200701E-3,-1.8095442E-3,-1.2579008E-2,-3.0592056E-3,6.52015E-3,8.4501604E-4,6.1219186E-4,-4.3259826E-3,-1.4273782E-3,2.4528848E-3,2.8350858E-3,-2.1060195E-3,9.316284E-3,3.7078313E-3,-2.643826E-3,1.5161828E-3,2.573676E-3,6.9992975E-3],"split_indices":[71,58,52,1,52,15,76,50,3,70,1,0,51,0,47,72,5,9,2,0,2,0,0,5,51,73,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.1E2,6.6E1,1.59E2,5.1E1,3.4E1,3.2E1,1.21E2,3.8E1,4.1E1,1E1,7E0,2.7E1,4E0,2.8E1,2.1E1,1E2,1.4E1,2.4E1,6E0,3.5E1,4E0,6E0,8E0,1.9E1,1.5E1,1.3E1,5E0,1.6E1,8E0,9.2E1,6E0,8E0,1.6E1,8E0,2.2E1,1.3E1,4E0,4E0,1.1E1,8E0,5E0,1E1,9E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[7.139892E-3,-4.15615E-2,6.6522375E-2,-8.5915215E-2,-1.2644255E-2,3.4027457E-2,1.4464441E-1,-1.3156319E-1,1.4241298E-2,7.036237E-2,-4.169385E-2,8.925563E-3,2.0464376E-2,1.4921679E-2,1.13071755E-1,-2.2239901E-1,-4.685537E-2,6.477722E-3,-3.728939E-2,8.316822E-3,2.6030758E-2,-2.2642918E-2,-1.3347402E-1,4.8252523E-2,-1.8698728E-2,8.356707E-2,1.8246569E-1,-4.4051693E-3,-1.8440314E-2,-2.984403E-3,1.5965993E-4,1.8926734E-3,-3.2014558E-3,3.0929297E-3,-8.158109E-4,-1.9494162E-4,-5.823697E-3,-8.095554E-3,-1.7957985E-3,6.0458074E-4,4.6168924E-3,-4.3933606E-3,6.6670135E-4,4.478592E-3,4.2749435E-4,3.5730815E-3,1.1548562E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,-1,31,-1,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.154195E-1,1.9771469E-1,3.174764E-1,2.8076205E-1,2.32008E-1,1.9330572E-1,1.7298621E-1,3.127486E-1,1.3557078E-1,1.1751662E-1,1.2112112E-1,0E0,9.340522E-2,0E0,5.231762E-2,4.1310525E-1,2.1018166E-2,0E0,4.0777985E-2,0E0,3.5393655E-2,1.141658E-1,4.116185E-2,9.004256E-2,9.302509E-2,2.455102E-2,4.5053244E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,-1,32,-1,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.751724E2,7.5646E4,1.5972575E0,2.951829E9,1.8278716E0,2.8558656E4,1.8426E4,1.8196162E0,2.207E3,1.19532734E5,1.2271346E1,8.925563E-3,2.637815E1,1.4921679E-2,8.574E3,3.3656168E2,2.8270894E-1,6.477722E-3,2.9655823E-1,8.316822E-3,3.3958147E5,1E0,2.26E3,1.2119408E3,1.479938E6,9.0746456E7,3.91101E5,-4.4051693E-3,-1.8440314E-2,-2.984403E-3,1.5965993E-4,1.8926734E-3,-3.2014558E-3,3.0929297E-3,-8.158109E-4,-1.9494162E-4,-5.823697E-3,-8.095554E-3,-1.7957985E-3,6.0458074E-4,4.6168924E-3,-4.3933606E-3,6.6670135E-4,4.478592E-3,4.2749435E-4,3.5730815E-3,1.1548562E-2],"split_indices":[71,9,60,5,72,52,9,72,0,47,75,0,75,0,2,4,61,0,77,0,47,20,2,4,51,51,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.54E2,1.26E2,6E1,9.4E1,9E1,3.6E1,4.1E1,1.9E1,2.4E1,7E1,6E0,8.4E1,4E0,3.2E1,1.9E1,2.2E1,5E0,1.4E1,6E0,1.8E1,5.9E1,1.1E1,4.9E1,3.5E1,2.4E1,8E0,1.2E1,7E0,1.6E1,6E0,4E0,1E1,9E0,9E0,5.1E1,8E0,7E0,4E0,3E1,1.9E1,1E1,2.5E1,2E1,4E0,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[3.4827837E-3,-2.4586197E-2,9.832739E-2,-5.705439E-2,2.0420939E-2,3.8023643E-2,1.6771717E-1,-1.40051E-2,-1.0128554E-1,6.065021E-3,1.0902019E-2,5.6533813E-2,-1.4509801E-3,1.2002019E-1,1.3834245E-2,-8.133476E-3,-2.173574E-4,-1.4422009E-2,-7.246766E-2,-1.2929211E-3,8.41794E-2,6.0403673E-3,3.862378E-2,2.6304151E-3,5.99698E-3,-2.8406132E-3,1.9559693E-3,-5.8904476E-3,-1.316753E-3,-1.3492907E-3,1.2318445E-3,6.6149584E-3,1.6792138E-3,-1.0691644E-3,2.5838786E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,23,-1,-1,25,-1,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.345201E-1,3.124221E-1,2.57244E-1,2.346923E-1,9.3923256E-2,4.6832632E-2,1.4757085E-1,1.4348817E-1,3.498969E-1,0E0,7.517236E-2,3.237403E-2,0E0,7.3592365E-3,0E0,0E0,1.6013482E-1,0E0,1.3017932E-1,5.8142055E-2,2.8820463E-2,0E0,2.7274612E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,13,13,16,16,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,24,-1,-1,26,-1,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.249246E6,2.3893447E5,1.19331E5,1.903E3,1.1598054E8,1.9936699E3,2.670956E0,9.975593E4,6.065021E-3,1.5461028E12,1.7437479E-1,-1.4509801E-3,2.744E3,1.3834245E-2,-8.133476E-3,1.09E3,-1.4422009E-2,6.1100146E2,6.477987E-1,6.97696E5,6.0403673E-3,1.5098532E0,2.6304151E-3,5.99698E-3,-2.8406132E-3,1.9559693E-3,-5.8904476E-3,-1.316753E-3,-1.3492907E-3,1.2318445E-3,6.6149584E-3,1.6792138E-3,-1.0691644E-3,2.5838786E-3],"split_indices":[61,1,47,48,0,51,4,73,47,0,50,57,0,2,0,0,2,0,4,76,10,0,60,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.12E2,6.2E1,1.23E2,8.9E1,3.4E1,2.8E1,6.3E1,6E1,6E0,8.3E1,2.7E1,7E0,2.2E1,6E0,4E0,5.9E1,6E0,5.4E1,7.2E1,1.1E1,4E0,2.3E1,4E0,1.8E1,2.4E1,3.5E1,2.3E1,3.1E1,3.6E1,3.6E1,4E0,7E0,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.1237782E-3,-4.2186882E-2,5.8209665E-2,4.6128226E-3,-6.4099014E-2,2.4231482E-2,1.2165155E-1,8.7656066E-2,-2.9178029E-2,-1.2279467E-1,-1.2394148E-2,8.582667E-3,8.974844E-3,2.0506383E-1,7.867077E-2,8.634591E-3,3.428505E-2,-1.08173475E-1,8.403639E-3,-2.135321E-1,-6.958048E-2,2.8970655E-2,-8.6654544E-2,-8.693239E-2,2.8315768E-2,1.5134305E-2,5.6507452E-3,1.1418171E-1,2.3450175E-2,-2.1998622E-3,4.051862E-3,-2.0469893E-3,-7.5726877E-3,-5.735213E-4,3.6839452E-3,-1.6450046E-2,-2.891198E-3,-5.5834753E-3,-9.101022E-4,3.722662E-3,-1.2509127E-3,-3.9166314E-4,-7.2249738E-3,-7.26955E-4,-7.7027217E-3,2.9138033E-3,-8.705248E-4,2.333067E-3,6.905664E-3,3.333718E-3,-2.1056707E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,-1,-1,47,49,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.983124E-1,1.6129625E-1,2.6435155E-1,1.4566882E-1,3.2320714E-1,2.0122367E-1,1.3802785E-1,7.353462E-2,1.1075394E-1,2.2656828E-1,1.8067291E-1,0E0,1.4343658E-1,1.0217875E-1,5.615048E-2,0E0,5.2699402E-2,3.5734683E-2,3.98362E-2,3.676166E-1,8.257474E-2,1.1288361E-1,1.1398539E-1,7.168833E-2,1.0689014E-1,0E0,0E0,3.333275E-2,4.7343872E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,-1,-1,48,50,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.751724E2,1.47887E5,1.2711891E0,1.7690372E0,5.283708E0,2.8558656E4,7.013774E5,2.6E1,1.9268292E0,7.359468E8,1.609091E1,8.582667E-3,7.88E2,2.7479675E-1,4.7291425E10,8.634591E-3,1.549E3,3.7705657E0,1.1192772E8,1.721649E8,8.5902435E1,2.0303578E0,3.58918E5,2.2912737E1,2.637815E1,1.5134305E-2,5.6507452E-3,2.744E3,1.4E1,-2.1998622E-3,4.051862E-3,-2.0469893E-3,-7.5726877E-3,-5.735213E-4,3.6839452E-3,-1.6450046E-2,-2.891198E-3,-5.5834753E-3,-9.101022E-4,3.722662E-3,-1.2509127E-3,-3.9166314E-4,-7.2249738E-3,-7.26955E-4,-7.7027217E-3,2.9138033E-3,-8.705248E-4,2.333067E-3,6.905664E-3,3.333718E-3,-2.1056707E-3],"split_indices":[71,1,58,72,75,52,52,10,72,5,75,0,0,57,50,0,2,73,5,7,71,72,48,75,75,0,0,2,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.56E2,1.23E2,5E1,1.06E2,8.1E1,4.2E1,1.4E1,3.6E1,4.9E1,5.7E1,6E0,7.5E1,1.3E1,2.9E1,4E0,1E1,1.1E1,2.5E1,1.7E1,3.2E1,3.7E1,2E1,1.2E1,6.3E1,4E0,9E0,1.7E1,1.2E1,4E0,6E0,6E0,5E0,2E1,5E0,8E0,9E0,1.5E1,1.7E1,1.9E1,1.8E1,1E1,1E1,7E0,5E0,3.6E1,2.7E1,7E0,1E1,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"51","size_leaf_vector":"1"}},{"base_weights":[4.4667735E-3,-1.9271143E-2,1.0929903E-1,-3.524879E-2,4.063013E-2,7.347263E-2,2.4525735E-1,-1.861916E-2,-1.0624239E-1,1.04474336E-1,-1.0531298E-2,-1.1403218E-3,9.4427E-2,3.665123E-3,1.5765578E-2,-5.9897605E-2,1.0251988E-2,-1.027324E-2,-7.200273E-2,8.400187E-3,6.112077E-2,-5.0974224E-2,2.5221322E-2,3.8990096E-4,1.089554E-1,-4.7489153E-3,1.4019388E-4,1.398148E-3,-3.0529455E-3,-1.0101859E-3,-5.719571E-3,-6.3339074E-4,4.1252663E-3,2.3877557E-4,-3.5226298E-3,-2.7353835E-4,2.1481127E-3,5.350595E-3,2.3422695E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,-1,29,-1,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.740011E-1,2.1294717E-1,2.2552514E-1,2.0462638E-1,1.546192E-1,8.6739644E-2,1.3370001E-1,1.710352E-1,1.1600864E-1,5.9839442E-2,4.0326033E-2,0E0,4.153785E-2,0E0,0E0,1.60919E-1,1.3291885E-1,0E0,6.7865014E-2,0E0,3.3219278E-2,1.9040532E-2,1.0296783E-2,0E0,6.3132644E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,18,18,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,-1,30,-1,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5242589E0,3.131673E6,2.2973125E3,1.7592884E7,1.8000048E10,1E0,2.174459E5,1E0,1.868632E7,1.824614E0,1.04E2,-1.1403218E-3,6.765595E7,3.665123E-3,1.5765578E-2,1.9268292E0,2.5417683E0,-1.027324E-2,1.5E1,8.400187E-3,1.7612903E0,9.515241E6,5.4527334E9,3.8990096E-4,5.557553E-1,-4.7489153E-3,1.4019388E-4,1.398148E-3,-3.0529455E-3,-1.0101859E-3,-5.719571E-3,-6.3339074E-4,4.1252663E-3,2.3877557E-4,-3.5226298E-3,-2.7353835E-4,2.1481127E-3,5.350595E-3,2.3422695E-3],"split_indices":[61,48,4,64,5,29,47,8,64,73,8,0,5,0,0,72,72,0,3,0,76,1,12,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,2.2E2,4.9E1,1.74E2,4.6E1,4E1,9E0,1.42E2,3.2E1,2E1,2.6E1,7E0,3.3E1,4E0,5E0,5.8E1,8.4E1,6E0,2.6E1,6E0,1.4E1,1.2E1,1.4E1,5E0,2.8E1,3.4E1,2.4E1,6.7E1,1.7E1,1.4E1,1.2E1,4E0,1E1,4E0,8E0,6E0,8E0,2.4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-3.1395643E-3,-4.015425E-2,4.6032365E-2,6.2255193E-2,-5.294301E-2,2.192633E-2,1.1681605E-1,-1.5810654E-3,7.495262E-3,-6.710322E-2,4.0586167E-3,-3.2719853E-3,6.9989964E-2,2.3386975E-1,3.6739185E-2,1.767672E-3,-1.6399372E-3,-7.9762705E-2,2.8114744E-2,-4.914371E-2,5.0356988E-2,-2.159232E-2,8.029972E-2,9.229252E-2,5.2178865E-3,5.3531444E-3,1.5289885E-2,-1.7948225E-3,6.656736E-2,-1.9578694E-3,-5.564064E-3,5.2439384E-3,-1.4823972E-3,-1.2425052E-4,-3.4368103E-3,3.6833284E-3,3.750581E-5,7.9014304E-4,-2.810015E-3,5.1996973E-3,8.5400796E-4,2.2829887E-3,5.586135E-3,-1.3178602E-3,1.7502868E-3,5.539833E-3,1.5208478E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,-1,-1,29,31,33,35,37,39,41,43,-1,-1,-1,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.041914E-1,2.1022704E-1,2.0049E-1,1.15783125E-1,1.14081204E-1,1.0923006E-1,2.7260667E-1,1.7695315E-2,0E0,1.3845384E-1,7.390698E-2,9.359755E-2,4.430048E-2,1.0000074E-1,4.5509495E-2,0E0,0E0,1.4786315E-1,7.609901E-2,1.638133E-2,2.3432944E-2,7.747132E-2,2.0073742E-2,2.2505328E-2,1.1077509E-2,0E0,0E0,0E0,2.1263462E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,-1,-1,30,32,34,36,38,40,42,44,-1,-1,-1,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.751724E2,1.1623532E6,7.3717834E2,1E0,4.7033575E-1,1.4429921E-1,2.45226E6,1E0,7.495262E-3,3.402546E6,2.6801108E2,4.43001E5,1.177948E3,6.0908795E6,1.04E2,1.767672E-3,-1.6399372E-3,1.609E3,1.8000048E10,1.56E2,1E1,2.142943E1,4.128E3,2.8938934E8,8.664146E-1,5.3531444E-3,1.5289885E-2,-1.7948225E-3,2.2452548E7,-1.9578694E-3,-5.564064E-3,5.2439384E-3,-1.4823972E-3,-1.2425052E-4,-3.4368103E-3,3.6833284E-3,3.750581E-5,7.9014304E-4,-2.810015E-3,5.1996973E-3,8.5400796E-4,2.2829887E-3,5.586135E-3,-1.3178602E-3,1.7502868E-3,5.539833E-3,1.5208478E-3],"split_indices":[71,64,71,8,58,57,48,29,0,1,4,47,4,64,8,0,0,2,5,10,3,75,0,12,76,0,0,0,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.57E2,1.18E2,1.7E1,1.4E2,8.9E1,2.9E1,1.1E1,6E0,1.12E2,2.8E1,5.9E1,3E1,1.1E1,1.8E1,5E0,6E0,9.9E1,1.3E1,1.3E1,1.5E1,4.9E1,1E1,2.2E1,8E0,6E0,5E0,5E0,1.3E1,5.3E1,4.6E1,5E0,8E0,5E0,8E0,9E0,6E0,2.5E1,2.4E1,6E0,4E0,1E1,1.2E1,4E0,4E0,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[3.8150148E-3,-4.938551E-2,3.9701525E-2,-3.167981E-2,-1.660038E-1,2.9594693E-2,1.0498834E-2,-2.0798726E-2,-7.619764E-3,-1.1222272E-2,-5.0890497E-3,-1.4824146E-2,4.1464336E-2,-4.7968432E-2,2.6761029E-2,2.470132E-2,-5.3477604E-2,8.1058875E-2,2.3741255E-2,-5.0988817E-3,-3.8083564E-4,6.1015277E-3,-8.4815756E-4,-1.6965251E-3,3.2905876E-3,-3.2445334E-3,6.442447E-5,4.347204E-3,-1.0744801E-3,2.9712205E-3,1.1834663E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,-1,13,-1,-1,-1,15,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.117121E-1,2.1774033E-1,2.9779938E-1,1.3488746E-1,3.343138E-2,8.128175E-2,0E0,1.16288096E-1,0E0,0E0,0E0,5.1738232E-2,8.298579E-2,1.4113271E-1,1.6013677E-1,5.1570255E-2,1.5951343E-2,5.381368E-2,7.324166E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,12,-1,14,-1,-1,-1,16,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1670399E-1,4.4326835E0,4.328683E0,1.10980764E-1,4.160985E2,3.2744165E2,1.0498834E-2,9.378309E7,-7.619764E-3,-1.1222272E-2,-5.0890497E-3,6.7897725E-1,2.0745E5,2.7307575E5,1.6581868E9,2.3780487E-1,2.8591623E2,2.8123772E0,9E0,-5.0988817E-3,-3.8083564E-4,6.1015277E-3,-8.4815756E-4,-1.6965251E-3,3.2905876E-3,-3.2445334E-3,6.442447E-5,4.347204E-3,-1.0744801E-3,2.9712205E-3,1.1834663E-4],"split_indices":[58,73,54,61,74,4,0,7,0,0,0,75,48,47,5,75,74,72,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,1.07E2,1.59E2,9.4E1,1.3E1,1.52E2,7E0,8.8E1,6E0,4E0,9E0,3.2E1,1.2E2,5.6E1,3.2E1,1.6E1,1.6E1,3.6E1,8.4E1,2.1E1,3.5E1,9E0,2.3E1,7E0,9E0,1.2E1,4E0,3.2E1,4E0,2.8E1,5.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[8.326176E-3,-1.3728847E-2,7.663917E-2,-5.2891996E-2,1.2079227E-2,1.229571E-2,6.0565077E-2,1.808629E-2,-8.502168E-2,-3.4261846E-3,7.94121E-2,2.3854816E-2,9.43276E-2,1.1140378E-1,-2.7068641E-2,-1.5338336E-2,-1.1857669E-1,-1.4790639E-2,5.627162E-2,1.6924437E-2,1.15776405E-1,5.0231875E-3,7.7800853E-3,1.20855585E-1,4.96928E-2,7.515314E-3,1.7171516E-3,1.4030898E-3,-2.5769798E-3,-3.8420546E-3,9.1217994E-4,-1.27291605E-2,-4.015926E-3,7.5304473E-4,-2.9149824E-3,6.9252397E-3,7.2442193E-4,-1.8380774E-3,2.8304295E-3,6.4681876E-3,2.5458564E-3,1.0336379E-3,-1.815763E-3,3.051114E-3,7.343203E-3,-3.9514544E-4,3.8224E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1989934E-1,2.1390195E-1,1.9196239E-1,1.9310056E-1,1.3390712E-1,0E0,7.707341E-2,1.1678665E-1,1.3379833E-1,7.206776E-2,5.2576244E-2,4.3001775E-2,3.3068717E-2,2.84791E-2,3.3305746E-2,4.953561E-2,1.6894895E-1,1.3554084E-1,6.251046E-2,2.778064E-2,1.4649898E-2,0E0,2.028101E-2,3.1611323E-2,2.7903907E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,1.6437392E2,1.6771E4,1.433458E10,4.7321553E5,1.229571E-2,2.174459E5,1.7690372E0,1.591046E0,8.834E3,2.0497458E2,1.7278711E-1,3.9280094E5,1.5497989E5,1.25E2,6.5284515E6,1E0,2.142943E1,2.637815E1,1.647E3,2.8004E4,5.0231875E-3,2.011653E7,1.693196E1,5.2768206E5,7.515314E-3,1.7171516E-3,1.4030898E-3,-2.5769798E-3,-3.8420546E-3,9.1217994E-4,-1.27291605E-2,-4.015926E-3,7.5304473E-4,-2.9149824E-3,6.9252397E-3,7.2442193E-4,-1.8380774E-3,2.8304295E-3,6.4681876E-3,2.5458564E-3,1.0336379E-3,-1.815763E-3,3.051114E-3,7.343203E-3,-3.9514544E-4,3.8224E-3],"split_indices":[61,71,9,50,47,0,47,72,73,2,71,57,47,47,0,64,31,75,75,2,10,0,51,75,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,2.1E2,6.7E1,8.3E1,1.27E2,4E0,6.3E1,2.6E1,5.7E1,1.04E2,2.3E1,3.1E1,3.2E1,8E0,1.8E1,1.9E1,3.8E1,8.8E1,1.6E1,9E0,1.4E1,4E0,2.7E1,1.9E1,1.3E1,4E0,4E0,6E0,1.2E1,6E0,1.3E1,5E0,3.3E1,5.4E1,3.4E1,4E0,1.2E1,4E0,5E0,9E0,5E0,2.1E1,6E0,9E0,1E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"47","size_leaf_vector":"1"}},{"base_weights":[-1.9457481E-3,-1.5870022E-2,1.0341738E-1,-3.2973208E-2,3.5419323E-2,2.229818E-1,2.0529639E-2,-1.7153427E-2,-8.87046E-2,-2.3598976E-2,4.887533E-2,1.714879E-2,1.15158506E-1,4.763684E-2,-3.2375765E-3,5.2892286E-3,-2.377449E-2,-9.544814E-3,-6.1676692E-2,1.1435341E-4,-1.7012536E-3,2.1261418E-2,7.354734E-2,7.3686056E-3,2.1750287E-3,-2.0995973E-4,7.177737E-2,-4.327336E-3,-4.3853765E-4,-5.965284E-4,-6.902357E-3,5.596778E-3,-1.6721123E-4,4.654438E-3,2.3026448E-3,1.4744838E-3,5.206076E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,-1,27,-1,29,-1,-1,31,33,-1,-1,-1,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.021609E-1,2.1345302E-1,3.1255588E-1,1.5894024E-1,4.9596585E-2,1.666416E-1,5.260302E-2,1.277427E-1,1.15440935E-1,4.270794E-3,3.2500826E-2,0E0,1.9876055E-2,2.0488378E-2,0E0,0E0,1.3617614E-1,0E0,1.4422742E-1,0E0,0E0,6.286068E-2,1.1799261E-2,0E0,0E0,0E0,1.4436856E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,16,16,18,18,21,21,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,-1,28,-1,30,-1,-1,32,34,-1,-1,-1,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1.19881555E-1,2.660842E6,4.352025E0,1.8681856E8,3.0256798E0,3.6661228E7,1E1,1.9268292E0,9.141936E7,2.0699957E8,1.714879E-2,1.353E3,1.2E2,-3.2375765E-3,5.2892286E-3,9.526745E1,-9.544814E-3,7.188264E-2,1.1435341E-4,-1.7012536E-3,1.0632315E6,2.1140351E0,7.3686056E-3,2.1750287E-3,-2.0995973E-4,1.2244738E7,-4.327336E-3,-4.3853765E-4,-5.965284E-4,-6.902357E-3,5.596778E-3,-1.6721123E-4,4.654438E-3,2.3026448E-3,1.4744838E-3,5.206076E-3],"split_indices":[71,57,48,73,7,73,67,10,72,7,12,0,0,8,0,0,71,0,76,0,0,67,72,0,0,0,51,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.72E2,2.41E2,3.1E1,1.81E2,6E1,1.2E1,1.9E1,1.42E2,3.9E1,1.1E1,4.9E1,4E0,8E0,1.5E1,4E0,6E0,1.36E2,6E0,3.3E1,4E0,7E0,2.4E1,2.5E1,4E0,4E0,5E0,1E1,2.2E1,1.14E2,2.2E1,1.1E1,4E0,2E1,1E1,1.5E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[1.218216E-3,-1.3792882E-2,1.1012109E-1,-4.2544812E-2,1.9182375E-2,2.0248948E-1,3.0720327E-2,-7.6438725E-2,-6.4982655E-3,-7.5200975E-2,2.9491797E-2,1.4391599E-2,1.0517586E-1,5.6515057E-2,-2.6688562E-3,-3.7148193E-2,-1.4567575E-1,2.7952781E-2,-5.5481967E-2,-6.8561605E-4,-6.124779E-3,3.6892947E-2,-3.912678E-2,6.4820186E-3,1.9063924E-3,-2.4190178E-4,8.506956E-2,-3.4179145E-3,-2.6542273E-6,-1.6340613E-2,-4.09398E-3,1.9403584E-3,-3.6747758E-3,2.308031E-3,-5.8365255E-3,1.9560067E-3,-2.5744622E-3,-5.271041E-3,9.341977E-4,1.4694491E-3,6.559495E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,-1,-1,35,37,-1,-1,-1,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.723274E-1,2.4163407E-1,2.5047368E-1,1.6566245E-1,1.1800932E-1,1.3853973E-1,4.902925E-2,1.8501362E-1,1.1461604E-1,3.9869756E-2,5.6291975E-2,0E0,1.688005E-2,2.8667532E-2,0E0,6.327403E-2,2.653268E-1,6.453848E-2,2.1715648E-1,0E0,0E0,5.3227797E-2,5.157683E-2,0E0,0E0,0E0,2.9014602E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,-1,-1,36,38,-1,-1,-1,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,2.0718746E8,2.660842E6,5.43944E5,2.2013508E-2,3.0256798E0,6.236639E3,9.0649E4,2.059925E0,1E1,1E0,1.4391599E-2,1.91016E5,1.18E2,-2.6688562E-3,1.9268292E0,2.07684E5,1.9944454E3,1.3879616E7,-6.8561605E-4,-6.124779E-3,2.179529E3,2.5952345E5,6.4820186E-3,1.9063924E-3,-2.4190178E-4,3.0635E4,-3.4179145E-3,-2.6542273E-6,-1.6340613E-2,-4.09398E-3,1.9403584E-3,-3.6747758E-3,2.308031E-3,-5.8365255E-3,1.9560067E-3,-2.5744622E-3,-5.271041E-3,9.341977E-4,1.4694491E-3,6.559495E-3],"split_indices":[71,7,48,1,57,73,4,48,72,3,24,0,10,8,0,72,1,4,64,0,0,4,47,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.87E2,2.53E2,3.4E1,1.35E2,1.18E2,1.5E1,1.9E1,6.9E1,6.6E1,1.1E1,1.07E2,6E0,9E0,1.5E1,4E0,4.5E1,2.4E1,3.9E1,2.7E1,6E0,5E0,9.7E1,1E1,5E0,4E0,5E0,1E1,2.2E1,2.3E1,4E0,2E1,3.5E1,4E0,1.1E1,1.6E1,9.2E1,5E0,4E0,6E0,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[1.8651754E-3,-1.1474913E-2,1.10247016E-1,-2.7378276E-2,3.6669753E-2,2.0101932E-1,3.35215E-2,-3.798584E-2,2.0683385E-2,-2.19896E-2,4.950134E-2,5.0855074E-3,1.3967071E-2,5.7987913E-2,-1.154525E-3,-2.807056E-2,-1.17406614E-1,-2.6747555E-2,8.5707806E-2,1.7313604E-4,-1.6268467E-3,-1.3231227E-2,6.2939115E-2,1.4546964E-2,5.735981E-3,1.5696298E-3,-2.1549403E-3,-1.1543729E-2,-1.8859623E-3,-2.8204147E-3,1.2004658E-3,5.694538E-3,-6.8974856E-4,-1.4600082E-3,5.3365034E-4,5.1810374E-3,2.227867E-3,1.9686858E-3,-7.631219E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,31,-1,-1,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0898725E-1,1.939508E-1,2.0916265E-1,9.765093E-2,4.8508152E-2,9.624988E-2,2.7260168E-2,1.1955109E-1,1.102384E-1,4.2565214E-3,4.4668034E-2,0E0,0E0,3.579667E-2,0E0,1.646347E-1,1.6298556E-1,3.9878674E-2,5.920954E-2,0E0,0E0,4.9546E-3,2.5582701E-2,8.610905E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,32,-1,-1,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1.19881555E-1,2.5304102E6,5.1428344E5,1.8681856E8,9.8999884E2,1.1598054E8,4.3399894E5,2.0139322E2,9.141936E7,2.29E2,5.0855074E-3,1.3967071E-2,3.0635E4,-1.154525E-3,1.7816157E0,3.559166E7,5.577E3,4.986835E6,1.7313604E-4,-1.6268467E-3,1.3026711E6,5.1E2,4.9237654E-1,5.735981E-3,1.5696298E-3,-2.1549403E-3,-1.1543729E-2,-1.8859623E-3,-2.8204147E-3,1.2004658E-3,5.694538E-3,-6.8974856E-4,-1.4600082E-3,5.3365034E-4,5.1810374E-3,2.227867E-3,1.9686858E-3,-7.631219E-4],"split_indices":[71,57,66,47,7,71,51,47,71,7,0,0,0,9,0,72,7,10,1,0,0,52,10,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.81E2,2.51E2,3E1,1.89E2,6.2E1,1.3E1,1.7E1,1.55E2,3.4E1,1.1E1,5.1E1,8E0,5E0,1.2E1,5E0,1.39E2,1.6E1,2E1,1.4E1,4E0,7E0,9E0,4.2E1,8E0,4E0,3.2E1,1.07E2,5E0,1.1E1,1.2E1,8E0,1E1,4E0,5E0,4E0,8E0,3.4E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.1548715E-3,-2.0165397E-2,7.966207E-2,-3.4287713E-2,2.5246624E-2,2.2602713E-2,1.24136634E-1,-4.81905E-2,2.4468936E-2,7.3706284E-2,-1.4362658E-2,4.431256E-2,-1.6782688E-3,8.752128E-2,1.0437987E-2,-3.7425354E-2,-1.4009675E-1,-1.579166E-2,8.762306E-2,1.20676056E-1,7.226004E-3,-5.867639E-2,1.843963E-2,-1.1138735E-4,5.783733E-2,4.816184E-3,1.9526527E-3,-6.939586E-3,-1.2545121E-3,-2.7803206E-3,-8.855074E-3,-1.6763937E-3,2.6460667E-3,1.9127405E-3,4.764628E-3,7.920868E-3,2.7923342E-3,-2.946998E-3,2.6278016E-3,-4.900761E-4,-4.5270384E-3,1.5230638E-3,-1.2082505E-3,1.1097655E-3,4.05748E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,37,39,41,-1,43,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.194997E-1,1.4259803E-1,1.2915409E-1,1.3923207E-1,1.02789596E-1,3.238469E-2,8.9075476E-2,1.3030246E-1,8.529026E-2,7.406953E-2,4.4689726E-2,1.1516605E-2,0E0,1.3289794E-2,0E0,1.3888209E-1,4.6673894E-2,3.3815928E-2,5.203329E-3,3.27307E-2,4.271382E-2,2.3711339E-2,1.2643613E-2,0E0,1.2180869E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,38,40,42,-1,44,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5057492E0,3.131673E6,2.3893447E5,5.1428344E5,1.775966E10,2.011653E7,6.0908795E6,1E0,5.403929E2,2.035952E0,1.08E2,2.300289E8,-1.6782688E-3,1.3233988E6,1.0437987E-2,9.501452E-5,5.743459E-3,4.1966243E0,2.9635E4,8.833194E5,4.876025E2,4.1380672E0,1.52831E1,-1.1138735E-4,1E0,4.816184E-3,1.9526527E-3,-6.939586E-3,-1.2545121E-3,-2.7803206E-3,-8.855074E-3,-1.6763937E-3,2.6460667E-3,1.9127405E-3,4.764628E-3,7.920868E-3,2.7923342E-3,-2.946998E-3,2.6278016E-3,-4.900761E-4,-4.5270384E-3,1.5230638E-3,-1.2082505E-3,1.1097655E-3,4.05748E-3],"split_indices":[61,48,47,47,5,51,64,24,74,72,8,7,0,66,0,60,76,77,9,52,71,76,77,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.2E2,5.1E1,1.68E2,5.2E1,2.3E1,2.8E1,1.36E2,3.2E1,2.3E1,2.9E1,1.7E1,6E0,2.2E1,6E0,1.23E2,1.3E1,2E1,1.2E1,1.3E1,1E1,1.2E1,1.7E1,4E0,1.3E1,1.5E1,7E0,9E0,1.14E2,6E0,7E0,1.6E1,4E0,4E0,8E0,6E0,7E0,4E0,6E0,6E0,6E0,1.3E1,4E0,7E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"45","size_leaf_vector":"1"}},{"base_weights":[2.6610433E-4,-3.808482E-2,3.598843E-2,-3.0279525E-2,-1.0889985E-2,8.53482E-3,7.749504E-2,-4.035627E-2,7.385095E-2,-9.54287E-2,2.4419786E-2,5.895432E-2,1.2073612E-2,-5.6782678E-2,1.6230818E-2,6.2793777E-3,-5.2558875E-4,-7.5898734E-3,-1.8467861E-3,3.248072E-2,-3.578536E-3,1.0029088E-1,6.992278E-3,4.10007E-3,-2.9583087E-3,4.703174E-3,-3.171778E-4,2.8124666E-3,1.3800942E-4,2.9666338E-3,6.8465746E-3,1.765536E-3,-3.6424124E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,-1,-1,-1,-1,27,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.863362E-1,2.0348918E-1,1.657065E-1,1.4148745E-1,0E0,1.4931002E-1,1.8320316E-1,1.1339386E-1,6.554325E-2,3.9469764E-2,6.674832E-2,1.15287066E-1,0E0,1.0813868E-1,5.63251E-2,0E0,0E0,0E0,0E0,6.1383374E-2,0E0,4.2497754E-2,7.0528306E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,19,19,21,21,22,22],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,-1,-1,-1,-1,28,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,1.2816449E7,-1.0889985E-2,1.0584392E8,3.1746092E-1,9.199832E5,2.459286E7,2.265E3,3.8016708E7,6.6E1,1.2073612E-2,3.8757736E-7,2.3749721E-1,6.2793777E-3,-5.2558875E-4,-7.5898734E-3,-1.8467861E-3,3.1471178E8,-3.578536E-3,1.55992E5,2.1E1,4.10007E-3,-2.9583087E-3,4.703174E-3,-3.171778E-4,2.8124666E-3,1.3800942E-4,2.9666338E-3,6.8465746E-3,1.765536E-3,-3.6424124E-3],"split_indices":[71,71,47,9,0,7,53,52,9,2,64,8,0,57,58,0,0,0,0,7,0,48,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.35E2,1.45E2,1.31E2,4E0,8.8E1,5.7E1,1.2E2,1.1E1,1.1E1,7.7E1,5.3E1,4E0,9.3E1,2.7E1,6E0,5E0,4E0,7E0,7.2E1,5E0,2.9E1,2.4E1,4E0,8.9E1,5E0,2.2E1,3.6E1,3.6E1,1.8E1,1.1E1,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[4.6536114E-4,-2.7709948E-2,4.9623653E-2,-4.2046502E-2,2.3574479E-2,3.6762558E-2,8.73471E-3,-3.5253476E-2,-9.680402E-3,-2.7711282E-2,4.044631E-2,6.0875498E-2,-2.2648696E-2,-4.6331894E-2,2.7994413E-2,-2.205099E-3,9.440538E-5,4.844452E-3,1.0760032E-2,4.3209128E-2,1.282172E-1,-5.2907118E-3,9.184552E-3,-3.8804938E-3,-1.1328061E-3,6.738825E-3,-4.9476716E-4,-3.1498203E-4,2.5738634E-3,3.5285202E-3,2.5535826E-4,8.675505E-3,1.2230461E-3,-6.533607E-4,3.9974656E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.725685E-1,1.267176E-1,1.6959575E-1,1.4570291E-1,3.4001634E-2,1.3174352E-1,0E0,9.238629E-2,0E0,6.1049894E-3,5.463185E-2,7.1789294E-2,8.035631E-2,8.945632E-2,9.328832E-2,0E0,0E0,0E0,1.7068826E-2,6.630439E-2,7.4564695E-2,0E0,3.9343953E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,18,18,19,19,20,20,22,22],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.3151303E2,6.862291E-1,2.0704236E3,4.1150245E-1,2.0358758E8,1.8018788E7,8.73471E-3,3.131673E6,-9.680402E-3,1.7427321E-1,2.2514071E0,7.3717834E2,1.3548306E3,1.8662952E0,4.150475E6,-2.205099E-3,9.440538E-5,4.844452E-3,2.0813715E8,5.7E1,7.643508E5,-5.2907118E-3,7.12911E4,-3.8804938E-3,-1.1328061E-3,6.738825E-3,-4.9476716E-4,-3.1498203E-4,2.5738634E-3,3.5285202E-3,2.5535826E-4,8.675505E-3,1.2230461E-3,-6.533607E-4,3.9974656E-3],"split_indices":[71,60,71,61,7,64,0,48,0,57,77,71,4,72,48,0,0,0,12,8,52,0,47,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.67E2,1.7E2,9.7E1,1.33E2,3.7E1,9E1,7E0,1.29E2,4E0,9E0,2.8E1,6.4E1,2.6E1,1.1E2,1.9E1,5E0,4E0,8E0,2E1,5.2E1,1.2E1,6E0,2E1,3.9E1,7.1E1,4E0,1.5E1,1.5E1,5E0,2.7E1,2.5E1,7E0,5E0,1.6E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[1.0404793E-3,-1.21584395E-2,8.368001E-2,-1.2256621E-1,-6.2683444E-3,1.6521272E-1,2.3323348E-2,-7.986367E-3,-7.651994E-4,8.045934E-3,-1.0299069E-2,1.4868271E-2,8.572735E-2,4.5196574E-2,-2.967787E-3,-3.154716E-3,-1.0253196E-1,5.6318087E-3,4.618525E-4,-2.9979597E-4,6.956272E-2,-6.936959E-3,2.1929083E-5,-6.49639E-3,4.851598E-4,1.4466698E-3,5.010379E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,15,-1,17,19,-1,21,23,-1,-1,-1,25,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0097434E-1,1.5400605E-1,1.8314627E-1,6.1823517E-2,1.6743213E-1,1.7290154E-1,4.707555E-2,0E0,0E0,0E0,1.4662114E-1,0E0,3.2013297E-2,2.439155E-2,0E0,1.1148192E-1,7.1260616E-2,0E0,0E0,0E0,1.6047932E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,10,10,12,12,13,13,15,15,16,16,20,20],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,16,-1,18,20,-1,22,24,-1,-1,-1,26,-1,-1,-1,-1,-1,-1],"split_conditions":[6.662007E2,1.0742589E-1,2.660842E6,1.6E1,1.55E2,1.431E4,3.6661228E7,-7.986367E-3,-7.651994E-4,8.045934E-3,3.7085085E0,1.4868271E-2,2.4840117E0,1.2E2,-2.967787E-3,4.9759598E1,1.00058334E2,5.6318087E-3,4.618525E-4,-2.9979597E-4,5.4582E4,-6.936959E-3,2.1929083E-5,-6.49639E-3,4.851598E-4,1.4466698E-3,5.010379E-3],"split_indices":[71,77,48,3,11,9,67,0,0,0,72,0,72,8,0,67,73,0,0,0,9,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.37E2,3.7E1,1.1E1,2.26E2,1.5E1,2.2E1,7E0,4E0,4E0,2.22E2,4E0,1.1E1,1.8E1,4E0,2.07E2,1.5E1,7E0,4E0,6E0,1.2E1,4E0,2.03E2,1.1E1,4E0,7E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[8.2585146E-4,-1.786127E-2,4.580317E-2,5.207858E-2,-2.3978915E-2,3.4938917E-2,9.390168E-3,7.815496E-3,-1.2913048E-3,-3.4782536E-2,2.1772722E-2,7.5003E-3,8.0095924E-2,2.20724E-3,-1.9938396E-3,-1.3508888E-1,-2.798877E-2,3.6182955E-2,-3.3115149E-3,-6.0110893E-2,2.421559E-2,1.04646765E-1,-2.7015724E-3,-1.8307549E-3,-9.364587E-3,-1.7037713E-3,1.7664E-3,3.906549E-3,5.7170243E-4,-4.3278025E-3,-3.4177504E-4,1.6358009E-3,-2.858475E-3,3.011131E-3,8.281355E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,-1,-1,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3195468E-1,8.4682904E-2,1.3204584E-1,1.00701466E-1,8.97283E-2,9.480579E-2,0E0,0E0,2.689753E-2,9.571184E-2,4.985981E-2,5.683252E-2,1.049435E-1,0E0,0E0,5.0515234E-2,8.398506E-2,3.4800693E-2,0E0,1.715694E-2,4.1260768E-2,6.204021E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21],"right_children":[2,4,6,8,10,12,-1,-1,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8045242E2,1.1623532E6,1.5266737E2,1.8462508E0,3.165913E6,2.8041378E5,9.390168E-3,7.815496E-3,2.5742075E0,1E0,1.980818E3,1.9241573E-1,2.8123772E0,2.20724E-3,-1.9938396E-3,1.418E3,3E1,1.0613555E8,-3.3115149E-3,3.07619E5,2.5497008E7,1.3000824E3,-2.7015724E-3,-1.8307549E-3,-9.364587E-3,-1.7037713E-3,1.7664E-3,3.906549E-3,5.7170243E-4,-4.3278025E-3,-3.4177504E-4,1.6358009E-3,-2.858475E-3,3.011131E-3,8.281355E-3],"split_indices":[71,64,75,72,1,47,0,0,72,15,4,76,72,0,0,2,3,7,0,48,62,4,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,1.94E2,8E1,1.5E1,1.79E2,7.6E1,4E0,4E0,1.1E1,1.45E2,3.4E1,4.8E1,2.8E1,5E0,6E0,8E0,1.37E2,3E1,4E0,9E0,3.9E1,2.4E1,4E0,4E0,4E0,1.21E2,1.6E1,9E0,2.1E1,5E0,4E0,3.5E1,4E0,1.7E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.7523934E-3,-8.844985E-3,8.7980606E-2,-2.5938668E-3,-1.1828045E-1,1.7884427E-1,1.45616E-2,-7.862449E-2,7.531358E-3,-2.5686484E-3,-8.374275E-3,1.4005396E-2,1.1171169E-1,2.5816143E-2,-1.2254365E-3,-1.2963317E-1,-4.060858E-3,6.916115E-3,3.1695364E-3,6.149751E-3,2.5043758E-3,-1.6158835E-4,4.2131096E-2,-9.2741445E-4,-7.571509E-3,1.3923616E-3,-2.6764972E-3,-6.1821137E-3,3.0827048E-4,8.20593E-4,2.7356339E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,-1,23,25,-1,27,-1,-1,-1,29,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9362926E-1,1.6455984E-1,2.3755604E-1,1.7688255E-1,4.152465E-2,1.0065842E-1,1.0419304E-2,1.0216522E-1,1.258642E-1,0E0,0E0,0E0,7.76951E-3,8.281499E-3,0E0,5.768305E-2,2.4015749E-2,0E0,9.685546E-2,0E0,0E0,0E0,3.563121E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,15,15,16,16,18,18,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,-1,24,26,-1,28,-1,-1,-1,30,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.662007E2,3.7085085E0,2.660842E6,3.3617156E-4,5.743459E-3,1.431E4,6.236639E3,3.3E1,1E1,-2.5686484E-3,-8.374275E-3,1.4005396E-2,8.982092E5,1.18E2,-1.2254365E-3,1.3049372E0,1.7E1,6.916115E-3,3.4967444E7,6.149751E-3,2.5043758E-3,-1.6158835E-4,6.708E3,-9.2741445E-4,-7.571509E-3,1.3923616E-3,-2.6764972E-3,-6.1821137E-3,3.0827048E-4,8.20593E-4,2.7356339E-3],"split_indices":[71,72,48,58,76,9,4,3,10,0,0,0,52,8,0,72,8,0,7,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.4E2,3.5E1,2.28E2,1.2E1,1.5E1,2E1,2.6E1,2.02E2,7E0,5E0,4E0,1.1E1,1.6E1,4E0,1.5E1,1.1E1,5E0,1.97E2,7E0,4E0,6E0,1E1,4E0,1.1E1,7E0,4E0,4E0,1.93E2,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[5.7240166E-5,-1.9982655E-2,4.4589706E-2,-5.160299E-2,4.818867E-3,1.1611166E-1,2.3814652E-2,-9.762541E-3,-3.9383415E-2,-7.945473E-2,1.7538918E-2,1.3856167E-1,9.6913136E-4,5.6648673E-3,5.1485218E-2,-2.6163757E-2,-8.249335E-3,6.260417E-4,-7.4851285E-3,5.9114406E-3,7.708621E-3,3.5620057E-3,1.0481725E-2,2.4320425E-2,-3.2023907E-2,6.537517E-2,-1.1732917E-3,3.524182E-3,-1.7278314E-3,1.8305408E-3,-3.8862522E-4,-1.7241209E-3,2.0602222E-3,4.3713042E-4,-2.209261E-3,-1.1824545E-4,3.8393724E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,-1,23,25,27,-1,-1,-1,-1,29,-1,-1,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5880212E-1,1.5709513E-1,1.3096337E-1,1.6417058E-1,1.22525536E-1,3.97664E-2,3.5365593E-2,0E0,1.4869423E-1,1.1822859E-1,1.06895305E-1,6.962633E-2,0E0,3.171546E-2,3.1600602E-2,9.2747375E-2,0E0,0E0,0E0,0E0,4.7954287E-2,0E0,0E0,3.95278E-2,1.0395478E-2,2.9590227E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,14,14,15,15,20,20,23,23,24,24,25,25],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,-1,24,26,28,-1,-1,-1,-1,30,-1,-1,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.821357E-1,8.461558E10,7.013774E5,2.3361564E0,9.277164E1,2.65826E6,2.216135E5,-9.762541E-3,2.781E3,1.3049372E0,2.9655823E-1,2.1061536E3,9.6913136E-4,1.243356E5,9.927293E8,1E1,-8.249335E-3,6.260417E-4,-7.4851285E-3,5.9114406E-3,2.3E1,3.5620057E-3,1.0481725E-2,3.9579918E4,6.788778E1,1.7427321E-1,-1.1732917E-3,3.524182E-3,-1.7278314E-3,1.8305408E-3,-3.8862522E-4,-1.7241209E-3,2.0602222E-3,4.3713042E-4,-2.209261E-3,-1.1824545E-4,3.8393724E-3],"split_indices":[60,50,52,73,71,66,47,0,2,72,77,4,0,47,5,10,0,0,0,0,8,0,0,47,71,57,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.88E2,1.99E2,8.9E1,8.7E1,1.12E2,1.9E1,7E1,5E0,8.2E1,1.4E1,9.8E1,1.5E1,4E0,4.3E1,2.7E1,7.6E1,6E0,7E0,7E0,7E0,9.1E1,1E1,5E0,2.9E1,1.4E1,2.3E1,4E0,7E0,6.9E1,3E1,6.1E1,7E0,2.2E1,4E0,1E1,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[2.6302058E-3,-6.9470066E-3,8.675293E-2,-2.7212687E-2,1.7960887E-2,1.6635017E-1,1.168328E-2,3.5056085E-3,-3.3179972E-2,-6.0463727E-2,2.6005743E-2,1.3030447E-2,7.0088275E-2,-2.0607125E-3,4.1407395E-2,-5.7126988E-2,-1.7705924E-3,-4.5004957E-3,1.763036E-4,5.1429313E-2,4.517171E-3,4.8098513E-3,9.986376E-4,-4.9396086E-4,3.348664E-3,-7.030902E-4,-7.643865E-3,1.5486891E-3,-3.337862E-3,2.9642184E-3,-3.3536658E-3,1.2084269E-3,-1.2596821E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,-1,23,25,27,-1,-1,29,31,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2860478E-1,1.292065E-1,1.7134126E-1,8.9486755E-2,7.41276E-2,1.3739312E-1,2.8696064E-2,0E0,1.0068843E-1,2.6743744E-2,5.7130896E-2,0E0,1.2709606E-2,0E0,1.8744962E-2,3.4386873E-1,1.5019107E-1,0E0,0E0,8.195232E-2,4.089332E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,14,14,15,15,16,16,19,19,20,20],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,-1,24,26,28,-1,-1,30,32,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,2.0718746E8,2.660842E6,3.299865E1,1.1670399E-1,3.0256798E0,1.08E2,3.5056085E-3,6.608892E2,5.896198E6,2.0817778E0,1.3030447E-2,1.353E3,-2.0607125E-3,3.8029985E6,1.647E3,2.2733334E1,-4.5004957E-3,1.763036E-4,1.9647126E7,1.6091988E7,4.8098513E-3,9.986376E-4,-4.9396086E-4,3.348664E-3,-7.030902E-4,-7.643865E-3,1.5486891E-3,-3.337862E-3,2.9642184E-3,-3.3536658E-3,1.2084269E-3,-1.2596821E-3],"split_indices":[71,7,48,71,58,73,8,0,4,51,72,0,0,0,51,2,75,0,0,64,51,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.54E2,2.8E1,1.4E2,1.14E2,1.3E1,1.5E1,7E0,1.33E2,1E1,1.04E2,5E0,8E0,5E0,1E1,7.5E1,5.8E1,6E0,4E0,4.7E1,5.7E1,4E0,4E0,4E0,6E0,5.5E1,2E1,3.9E1,1.9E1,4.3E1,4E0,3.4E1,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[-1.2992085E-3,-6.186166E-3,7.541433E-3,-2.1961667E-2,2.9620828E-2,-9.434576E-3,-6.528359E-2,8.000968E-2,1.8237442E-2,-6.9916554E-2,1.2006471E-3,-7.2051836E-3,-4.2570077E-2,1.1788558E-3,4.487425E-3,2.5036087E-2,-2.6391195E-3,-5.669194E-3,2.880813E-4,6.6165924E-3,-2.1408034E-4,-8.3313236E-5,-5.5499566E-3,7.872919E-4,4.0155845E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2450328E-1,1.5316631E-1,0E0,1.0141124E-1,4.614927E-2,9.464013E-2,8.035049E-2,1.1749946E-2,3.6880054E-2,8.995235E-2,1.0571988E-1,0E0,1.1102041E-1,0E0,0E0,3.0092005E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,1.19881555E-1,7.541433E-3,4.352025E0,6.743105E5,2.7414852E-1,1.9268292E0,4.2403465E6,2.008498E1,6.846054E7,6.7473096E9,-7.2051836E-3,1E0,1.1788558E-3,4.487425E-3,1.5513661E6,-2.6391195E-3,-5.669194E-3,2.880813E-4,6.6165924E-3,-2.1408034E-4,-8.3313236E-5,-5.5499566E-3,7.872919E-4,4.0155845E-3],"split_indices":[71,57,0,73,52,77,72,51,77,7,50,0,24,0,0,52,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.69E2,7E0,1.87E2,8.2E1,1.46E2,4.1E1,1.4E1,6.8E1,2.1E1,1.25E2,7E0,3.4E1,4E0,1E1,6.3E1,5E0,1.2E1,9E0,4E0,1.21E2,2.3E1,1.1E1,5.7E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.6445782E-4,-1.0475103E-2,8.043979E-2,-2.9651217E-2,1.1412919E-2,-7.1429117E-3,1.377257E-1,-9.233023E-3,-8.237862E-2,-3.569281E-3,1.7777296E-2,1.5757794E-2,-2.371839E-3,1.2649076E-2,8.630252E-2,-4.0853035E-2,1.8916083E-2,-1.6305467E-1,-2.6182115E-2,5.7349976E-2,5.4635657E-3,1.4126676E-3,-2.441901E-4,7.598621E-4,1.1652876E-1,-6.970347E-3,-9.432436E-4,4.867763E-3,1.4101231E-4,-2.2441898E-3,-1.50277065E-2,2.5870441E-3,-2.1400328E-3,-5.210417E-4,4.074509E-3,2.5014165E-3,-3.477042E-4,6.5692505E-3,3.266779E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,25,27,29,31,33,35,-1,-1,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.408431E-1,1.04850486E-1,1.6655317E-1,1.4242697E-1,6.760335E-2,1.5038142E-2,1.1719847E-1,8.7110065E-2,1.6347823E-1,0E0,5.328826E-2,3.3061649E-3,0E0,0E0,3.1832755E-2,1.0045742E-1,7.1240425E-2,2.641704E-1,4.1069098E-2,5.661004E-2,5.44727E-2,0E0,0E0,0E0,2.9876977E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,26,28,30,32,34,36,-1,-1,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,2.0718746E8,7.6411234E4,1.2869189E5,1.01861015E-1,6.5414023E-1,1.431E4,6.846054E7,6.4138586E2,-3.569281E-3,4.75E2,4.9332496E4,-2.371839E-3,1.2649076E-2,2.177322E-1,1.0742589E-1,3.9449276E1,1.779E3,6.7000134E8,1.95E2,3.3001E4,1.4126676E-3,-2.441901E-4,7.598621E-4,2.7479675E-1,-6.970347E-3,-9.432436E-4,4.867763E-3,1.4101231E-4,-2.2441898E-3,-1.50277065E-2,2.5870441E-3,-2.1400328E-3,-5.210417E-4,4.074509E-3,2.5014165E-3,-3.477042E-4,6.5692505E-3,3.266779E-3],"split_indices":[71,7,47,66,58,53,9,7,4,0,0,47,0,0,76,77,71,2,5,0,9,0,0,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,2.48E2,3.2E1,1.32E2,1.16E2,1.3E1,1.9E1,9.6E1,3.6E1,7E0,1.09E2,9E0,4E0,4E0,1.5E1,4.5E1,5.1E1,1.4E1,2.2E1,2.5E1,8.4E1,5E0,4E0,5E0,1E1,6E0,3.9E1,7E0,4.4E1,9E0,5E0,4E0,1.8E1,8E0,1.7E1,1.7E1,6.7E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.4997874E-3,-3.3806123E-2,2.9913362E-2,-2.6885899E-2,-9.847766E-3,9.612255E-3,7.117494E-2,-1.8126047E-3,-5.1563814E-2,6.27013E-3,2.703877E-3,5.3347144E-2,9.680545E-3,7.5899705E-2,-1.3439241E-2,-1.16839865E-2,-2.7902044E-2,-8.9354426E-2,1.6142849E-2,8.112999E-2,-9.998025E-3,5.5239773E-3,7.661275E-4,-9.591232E-5,-6.3508647E-3,-5.6021623E-3,-7.220093E-4,-8.744022E-4,-6.303036E-3,1.04238E-3,-3.2196818E-3,2.0278431E-3,5.3075883E-3,-2.4809325E-3,2.6473196E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,-1,21,23,-1,25,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8618237E-1,1.6747497E-1,1.191006E-1,8.3438665E-2,0E0,8.44502E-2,1.0532269E-1,6.2460262E-2,3.182742E-1,0E0,1.1649915E-1,7.648722E-2,0E0,2.1512829E-2,8.4542274E-2,0E0,6.9141015E-2,3.754548E-2,4.7441937E-2,3.2916382E-2,4.4569988E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,-1,22,24,-1,26,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,3.7786137E10,-9.847766E-3,9.037E3,3.1746092E-1,1.7145833E0,7.359468E8,6.27013E-3,1.0584392E8,1.07E2,9.680545E-3,4.4E1,1.4088401E7,-1.16839865E-2,7E0,1.945711E1,3.605082E7,6.0908795E6,4.43001E5,5.5239773E-3,7.661275E-4,-9.591232E-5,-6.3508647E-3,-5.6021623E-3,-7.220093E-4,-8.744022E-4,-6.303036E-3,1.04238E-3,-3.2196818E-3,2.0278431E-3,5.3075883E-3,-2.4809325E-3,2.6473196E-3],"split_indices":[71,71,47,50,0,9,53,72,5,0,7,8,0,10,64,0,3,75,64,64,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.38E2,1.42E2,1.34E2,4E0,9.6E1,4.6E1,6.7E1,6.7E1,4E0,9.2E1,4.2E1,4E0,8E0,5.9E1,6E0,6.1E1,1.1E1,8.1E1,2.9E1,1.3E1,4E0,4E0,5.5E1,4E0,6E0,5.5E1,5E0,6E0,7.6E1,5E0,1.5E1,1.4E1,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-2.8596204E-4,-1.6340856E-2,4.859123E-2,-7.195226E-2,-1.0008341E-2,8.836621E-2,3.7278784E-3,1.6763784E-3,-9.959118E-2,-2.233174E-3,-7.1362846E-2,9.953862E-3,5.5593874E-2,-2.9382505E-3,1.5245181E-2,-1.4627877E-1,-1.3251902E-3,6.941909E-2,-7.2391727E-3,-1.1384955E-1,-3.645952E-2,-3.02262E-4,7.1407154E-2,3.0594535E-2,-1.7203838E-3,-3.700943E-3,-9.028527E-3,-2.9900225E-4,6.158571E-3,-7.8040996E-4,1.6670662E-3,-6.8524242E-3,-2.5827326E-3,-6.0510065E-4,-2.8302413E-3,4.1196956E-3,6.784765E-4,1.9124127E-4,2.0451157E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,21,-1,23,25,-1,27,29,31,33,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1580678E-1,7.217702E-2,1.21012434E-1,6.652178E-2,8.90474E-2,1.3879418E-1,2.6645167E-2,0E0,5.198954E-2,6.040396E-2,2.6990578E-2,0E0,2.995687E-2,0E0,2.456222E-2,1.7065033E-2,0E0,5.459852E-2,6.6855766E-2,9.834647E-3,6.773902E-3,0E0,2.3260564E-2,8.22293E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,12,12,14,14,15,15,17,17,18,18,19,19,20,20,22,22,23,23],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,22,-1,24,26,-1,28,30,32,34,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.638253E2,2.598304E-1,9.520617E5,1.1078758E0,2.9665058E0,1E0,8.715196E5,1.6763784E-3,6.846054E7,1.1623532E6,5.3725034E-2,9.953862E-3,1.5457413E-1,-2.9382505E-3,2.008498E1,2.3101475E5,-1.3251902E-3,1E0,5.0090684E5,5.8835944E2,7.321865E7,-3.02262E-4,3.21879E6,5.1227652E4,-1.7203838E-3,-3.700943E-3,-9.028527E-3,-2.9900225E-4,6.158571E-3,-7.8040996E-4,1.6670662E-3,-6.8524242E-3,-2.5827326E-3,-6.0510065E-4,-2.8302413E-3,4.1196956E-3,6.784765E-4,1.9124127E-4,2.0451157E-3],"split_indices":[71,77,52,73,72,15,66,0,7,64,61,0,76,0,77,47,0,8,47,4,12,0,48,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.06E2,6.7E1,2E1,1.86E2,3.5E1,3.2E1,4E0,1.6E1,1.66E2,2E1,6E0,2.9E1,4E0,2.8E1,9E0,7E0,1E1,1.56E2,8E0,1.2E1,6E0,2.3E1,2.2E1,6E0,5E0,4E0,5E0,5E0,1.28E2,2.8E1,4E0,4E0,7E0,5E0,1.7E1,6E0,8E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[5.3371255E-5,-3.1028034E-2,2.0835297E-2,-1.6813101E-2,-1.4926928E-1,1.5616176E-2,1.1188454E-1,-8.193008E-2,-2.8258236E-3,-1.1570735E-2,-3.077064E-3,8.104795E-3,5.4305855E-2,1.3984346E-3,7.873873E-3,-7.777474E-3,-1.1213035E-2,1.2475116E-2,-7.506189E-2,3.586205E-3,3.7141119E-3,1.3836331E-2,9.371105E-2,-2.1831472E-3,1.9206341E-3,1.2273088E-3,-3.0987037E-3,-5.129966E-3,-8.0989045E-4,4.1300248E-4,-2.222923E-3,-3.5811187E-4,2.4999338E-3,6.544114E-3,2.5965169E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,-1,-1,23,25,27,-1,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8215322E-1,1.8741974E-1,7.856108E-2,9.267102E-2,8.414957E-2,4.6348542E-2,3.8186125E-2,1.0884194E-1,9.488517E-2,0E0,0E0,4.1382607E-2,4.0165834E-2,0E0,0E0,0E0,2.3056807E-2,8.212288E-2,2.9384568E-2,0E0,3.5688568E-2,1.2666093E-2,1.5475281E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,16,16,17,17,18,18,20,20,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,-1,-1,24,26,28,-1,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.1670399E-1,1.4031231E-2,4.328683E0,9.421523E-5,6.132018E2,4.7321553E5,2.174459E5,8.877883E1,2.142943E1,-1.1570735E-2,-3.077064E-3,1.549133E0,6.46838E5,1.3984346E-3,7.873873E-3,-7.777474E-3,7.87256E5,4.4326835E0,7.88E2,3.586205E-3,3.7930825E5,7.8744044E5,6.6E1,-2.1831472E-3,1.9206341E-3,1.2273088E-3,-3.0987037E-3,-5.129966E-3,-8.0989045E-4,4.1300248E-4,-2.222923E-3,-3.5811187E-4,2.4999338E-3,6.544114E-3,2.5965169E-3],"split_indices":[58,57,54,61,4,47,47,71,75,0,0,72,1,0,0,0,7,73,0,0,47,66,8,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.12E2,1.68E2,1.01E2,1.1E1,1.6E2,8E0,1.7E1,8.4E1,4E0,7E0,1.35E2,2.5E1,4E0,4E0,7E0,1E1,7E1,1.4E1,7E0,1.28E2,1.3E1,1.2E1,6E0,4E0,6E1,1E1,8E0,6E0,1.17E2,1.1E1,9E0,4E0,4E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[-8.589759E-5,-3.067964E-3,7.5481976E-3,-1.653295E-2,2.774567E-2,-3.1164067E-2,1.1871366E-2,8.249559E-2,1.1563938E-2,-1.3065111E-2,-7.483544E-2,5.8817483E-2,-1.1343878E-2,9.8734334E-2,7.956013E-4,3.272501E-2,-1.8323408E-2,-3.858915E-3,2.545208E-4,-1.4572021E-2,-1.6552575E-3,4.0481766E-3,-7.420578E-4,1.0773154E-3,-2.4016986E-3,3.0017581E-3,7.033631E-3,-9.956771E-4,2.4039142E-3,-2.7603523E-3,-7.327606E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3697606E-1,1.1455763E-1,0E0,8.0162644E-2,7.346569E-2,9.940385E-2,7.259835E-2,1.8415138E-2,4.2341225E-2,1.2017048E-1,3.3630395E-1,4.9403347E-2,6.504045E-2,1.7222047E-2,0E0,4.2436253E-2,1.9408267E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5266737E2,1.19881555E-1,7.5481976E-3,7.629E3,7.013774E5,1.2869189E5,1E1,3.21879E6,3.133129E8,1E0,9.975593E4,4.397396E7,5.0341E4,6.886E3,7.956013E-4,1.8681856E8,3.5300705E2,-3.858915E-3,2.545208E-4,-1.4572021E-2,-1.6552575E-3,4.0481766E-3,-7.420578E-4,1.0773154E-3,-2.4016986E-3,3.0017581E-3,7.033631E-3,-9.956771E-4,2.4039142E-3,-2.7603523E-3,-7.327606E-5],"split_indices":[75,57,0,10,52,66,3,48,7,31,47,51,9,2,0,7,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,2.74E2,4E0,1.91E2,8.3E1,1.26E2,6.5E1,1.8E1,6.5E1,9E1,3.6E1,2.1E1,4.4E1,1.4E1,4E0,3.8E1,2.7E1,1.8E1,7.2E1,4E0,3.2E1,1.5E1,6E0,2.4E1,2E1,1E1,4E0,1E1,2.8E1,7E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.9479691E-3,-6.1025335E-3,6.570009E-2,-1.2826966E-4,-7.2709166E-2,-4.1228845E-3,1.1827347E-1,-2.1588098E-2,2.0767951E-2,-1.8375515E-4,-1.0940418E-1,1.4558834E-2,-2.21616E-3,1.1170241E-2,7.202472E-2,-6.1529605E-3,-8.375057E-2,4.6207937E-3,1.6196072E-2,-1.8788242E-3,-7.4757603E-3,-2.3791735E-4,1.353022E-3,4.0713765E-4,9.993981E-2,-2.8882546E-3,4.5449435E-4,-8.595389E-3,-1.0712073E-3,-3.0795769E-3,9.5920236E-4,5.8259866E-3,2.6097929E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,-1,-1,-1,-1,31,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5771984E-1,9.668659E-2,1.2896664E-1,1.00896515E-1,4.9995556E-2,1.3856163E-2,9.753838E-2,1.0598767E-1,3.9858744E-2,0E0,4.0608004E-2,3.6619927E-3,0E0,0E0,2.7335852E-2,8.221786E-2,1.3263261E-1,0E0,4.339181E-2,0E0,0E0,0E0,0E0,0E0,5.4834858E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,15,15,16,16,18,18,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,-1,-1,-1,-1,32,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,1E0,7.6411234E4,1.7546342E8,1.03847E5,2.5586937E1,1.431E4,9.8845E5,1E0,-1.8375515E-4,5.743459E-3,1.5552E4,-2.21616E-3,1.1170241E-2,2.177322E-1,1E0,6E0,4.6207937E-3,1.32461E7,-1.8788242E-3,-7.4757603E-3,-2.3791735E-4,1.353022E-3,4.0713765E-4,2.7479675E-1,-2.8882546E-3,4.5449435E-4,-8.595389E-3,-1.0712073E-3,-3.0795769E-3,9.5920236E-4,5.8259866E-3,2.6097929E-3],"split_indices":[71,24,47,7,48,77,9,70,28,0,76,9,0,0,76,31,8,0,12,0,0,0,0,0,57,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.76E2,2.42E2,3.4E1,2.23E2,1.9E1,1.5E1,1.9E1,1.1E2,1.13E2,7E0,1.2E1,1.1E1,4E0,4E0,1.5E1,8.9E1,2.1E1,5E0,1.08E2,6E0,6E0,5E0,6E0,5E0,1E1,1.9E1,7E1,7E0,1.4E1,5E0,1.03E2,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"33","size_leaf_vector":"1"}},{"base_weights":[1.2324462E-3,-7.5858575E-3,6.690328E-2,-5.875678E-3,-3.8178023E-3,1.22768864E-1,-9.702716E-3,-2.0907314E-2,1.4889422E-2,1.9395949E-1,4.6420418E-2,-2.6487885E-3,1.5181703E-2,7.289406E-2,-2.9321136E-2,-5.571751E-2,2.1871038E-2,4.091844E-3,1.1982336E-2,9.781831E-4,3.1006704E-3,-9.838765E-4,1.9848663E-3,1.8952134E-4,6.484813E-3,-2.2772874E-3,1.0567203E-3,-6.292595E-4,-4.375475E-3,1.9192853E-3,-5.0805474E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,-1,7,9,11,13,15,17,19,-1,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5165402E-1,1.0399145E-1,1.3385998E-1,0E0,7.22432E-2,8.883163E-2,1.7535927E-2,9.526622E-2,5.4457247E-2,3.956905E-2,3.8370378E-3,0E0,1.112969E-2,4.6157714E-2,1.15893215E-1,1.5233859E-2,6.501205E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16],"right_children":[2,4,6,-1,8,10,12,14,16,18,20,-1,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1.393E3,5.577948E11,-5.875678E-3,2.0718746E8,7.643508E5,4.904973E6,6.7897725E-1,1.0478362E-1,1.0851409E3,3.13373E3,-2.6487885E-3,6.157198E1,1E0,1.051196E6,9E0,2.2048602E0,4.091844E-3,1.1982336E-2,9.781831E-4,3.1006704E-3,-9.838765E-4,1.9848663E-3,1.8952134E-4,6.484813E-3,-2.2772874E-3,1.0567203E-3,-6.292595E-4,-4.375475E-3,1.9192853E-3,-5.0805474E-4],"split_indices":[71,12,50,0,7,52,51,75,58,71,4,0,75,8,48,3,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.6E2,2.3E2,3E1,6E0,2.24E2,1.7E1,1.3E1,1.17E2,1.07E2,8E0,9E0,4E0,9E0,9E0,1.08E2,9E0,9.8E1,4E0,4E0,5E0,4E0,4E0,5E0,5E0,4E0,7.8E1,3E1,5E0,4E0,6.1E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[3.10813E-3,6.2053115E-3,1.7062627E-4,-9.529811E-3,4.523369E-2,-3.731356E-2,5.673263E-3,6.2087933E-3,7.288687E-2,-8.9933574E-2,-4.995276E-3,7.296518E-2,-5.0530196E-3,1.8669074E-2,-1.72184E-3,7.034394E-3,5.320442E-2,-2.1996384E-3,-8.425229E-3,1.9213033E-3,-1.5001298E-3,-1.8195294E-4,5.28027E-3,4.9531157E-3,-4.878689E-4,1.6219923E-4,1.8325793E-3,3.4376464E-3,7.2254107E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,-1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0634153E-1,0E0,1.1845655E-1,9.440653E-2,5.1347256E-2,1.3389726E-1,1.05028555E-1,1.2108576E-2,3.5903826E-2,1.0895437E-1,6.581086E-2,6.4634345E-2,7.98104E-2,5.3014746E-3,0E0,0E0,1.8148273E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,16,16],"right_children":[2,-1,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E1,6.2053115E-3,1.5057492E0,5.283708E0,2.174459E5,7.5608E4,7.014272E7,1.10018056E8,1.8426E4,3.5342257E0,1.8278716E0,1E0,6.186634E0,3.5617366E8,-1.72184E-3,7.034394E-3,1.4E1,-2.1996384E-3,-8.425229E-3,1.9213033E-3,-1.5001298E-3,-1.8195294E-4,5.28027E-3,4.9531157E-3,-4.878689E-4,1.6219923E-4,1.8325793E-3,3.4376464E-3,7.2254107E-4],"split_indices":[10,0,61,75,47,9,7,51,9,75,72,8,75,7,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,5E0,2.69E2,2.22E2,4.7E1,7.8E1,1.44E2,2E1,2.7E1,2.9E1,4.9E1,1.9E1,1.25E2,1.6E1,4E0,4E0,2.3E1,2.1E1,8E0,1.8E1,3.1E1,7E0,1.2E1,5E0,1.2E2,1E1,6E0,1.4E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[-6.773961E-4,-8.464028E-3,5.476503E-2,6.265524E-4,-4.2521432E-2,1.0550452E-1,-2.1806126E-3,-8.615522E-2,6.3765785E-3,-2.1358268E-2,-1.3890971E-1,4.7468036E-2,8.744282E-3,1.7477887E-2,-2.4335731E-3,-8.4018465E-3,-6.9886184E-4,1.4331637E-2,-5.4915853E-2,1.3853628E-2,-7.103333E-2,-7.949128E-3,-3.5628616E-3,3.2532907E-3,3.988344E-5,-3.2758046E-4,1.822024E-3,-9.4913185E-4,1.3116873E-3,1.2824076E-3,-3.7994068E-3,-3.7416408E-4,4.073529E-3,-4.1980296E-3,-7.156176E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,-1,-1,27,29,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1918372E-1,7.4950226E-2,9.846942E-2,9.635689E-2,1.0106705E-1,7.9173654E-2,1.7859451E-2,7.927621E-2,8.903991E-2,7.619765E-2,5.3239465E-3,1.2908589E-2,0E0,7.3458166E-3,0E0,0E0,0E0,8.024508E-2,5.0282627E-2,4.3364953E-2,1.8705934E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,17,17,18,18,19,19,20,20],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,-1,-1,28,30,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,1.7303246E7,6.464779E6,3.4955364E7,2.4320412E5,5.505334E9,3.6661228E7,1.332E3,1E0,2.142943E1,1.3E1,1.91016E5,8.744282E-3,4.0873584E7,-2.4335731E-3,-8.4018465E-3,-6.9886184E-4,3.81E2,9E0,5.4494965E-1,3.084419E6,-7.949128E-3,-3.5628616E-3,3.2532907E-3,3.988344E-5,-3.2758046E-4,1.822024E-3,-9.4913185E-4,1.3116873E-3,1.2824076E-3,-3.7994068E-3,-3.7416408E-4,4.073529E-3,-4.1980296E-3,-7.156176E-4],"split_indices":[71,64,48,7,47,5,67,0,24,75,8,10,0,63,0,0,0,0,3,76,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.41E2,3.3E1,1.91E2,5E1,1.7E1,1.6E1,1.1E1,1.8E2,4.2E1,8E0,1.1E1,6E0,1.2E1,4E0,4E0,7E0,1.6E2,2E1,2.5E1,1.7E1,4E0,4E0,7E0,4E0,6E0,6E0,4.6E1,1.14E2,5E0,1.5E1,2E1,5E0,1.2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[3.0185017E-3,-1.5962647E-2,2.4150021E-2,-1.3301289E-1,-5.439939E-3,7.054804E-3,1.8878033E-2,-6.3760113E-4,-8.78758E-3,4.3586013E-3,-1.08299665E-2,4.8070557E-2,1.1265427E-3,-4.3094933E-2,6.378897E-3,2.9208936E-2,1.0885883E-1,1.3159337E-2,-4.7827095E-2,-1.14338574E-4,-4.3156664E-3,7.9006737E-4,-2.0754023E-3,2.4170827E-3,-9.0318144E-4,6.729134E-3,1.6055589E-3,1.3149536E-3,-1.6478837E-3,-7.631443E-4,-5.2148853E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,-1,11,-1,-1,-1,13,15,17,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1108924E-1,1.791025E-1,8.625292E-2,8.008231E-2,7.3868945E-2,0E0,6.5758616E-2,0E0,0E0,0E0,7.190129E-2,5.1855497E-2,4.7753297E-2,9.23132E-2,4.7927845E-2,4.424071E-2,2.5256842E-2,5.0025683E-2,3.0843902E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,10,-1,12,-1,-1,-1,14,16,18,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.006438E6,2.3361564E0,4.92E2,1.3295195E0,1E1,7.054804E-3,9.397667E6,-6.3760113E-4,-8.78758E-3,4.3586013E-3,1.53091E5,7.120612E2,3.160848E6,6.5284515E6,1E0,1.7090584E1,2.6056337E0,9.906531E6,2.1881187E0,-1.14338574E-4,-4.3156664E-3,7.9006737E-4,-2.0754023E-3,2.4170827E-3,-9.0318144E-4,6.729134E-3,1.6055589E-3,1.3149536E-3,-1.6478837E-3,-7.631443E-4,-5.2148853E-3],"split_indices":[1,73,10,72,10,0,64,0,0,0,9,71,9,64,24,75,72,48,72,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,1.45E2,1.3E2,1.1E1,1.34E2,4E0,1.26E2,4E0,7E0,6E0,1.28E2,4.7E1,7.9E1,4.4E1,8.4E1,3.7E1,1E1,6.4E1,1.5E1,2.5E1,1.9E1,7E1,1.4E1,2.5E1,1.2E1,6E0,4E0,4.9E1,1.5E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[2.1318335E-4,-2.3540951E-2,2.2438744E-2,-1.6739797E-2,-8.820643E-3,6.1444105E-3,5.4549992E-2,-4.765783E-2,1.4069596E-3,4.5384695E-3,-1.2777979E-3,9.416091E-2,5.25755E-3,-7.030939E-2,2.4268355E-2,1.2203398E-2,-8.6320475E-2,-4.5638257E-3,4.56166E-3,1.3145782E-1,3.9693024E-2,4.1257378E-2,-2.8959513E-3,-5.282765E-3,-6.4005767E-4,3.6747127E-3,-6.206844E-4,-4.5480225E-3,9.239956E-4,-9.839027E-4,-6.169812E-3,4.9174903E-4,-2.9592053E-3,4.3434035E-3,8.6018825E-3,3.0024275E-3,-4.178631E-4,3.027109E-3,1.7843115E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4307447E-1,1.4650947E-1,7.291722E-2,7.132192E-2,0E0,6.461322E-2,9.127332E-2,7.8280605E-2,7.777206E-2,0E0,5.073321E-2,4.760064E-2,5.6575045E-2,8.8629425E-2,2.6237717E-2,6.570856E-2,2.501142E-2,0E0,3.6067333E-2,1.3206601E-2,1.552622E-2,1.3477802E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,7.5646E4,-8.820643E-3,2.3152277E2,6.6E1,4.5510144E9,3.7085085E0,4.5384695E-3,1.744E3,8.4550006E5,1.5E1,1.8130841E0,2.9655823E-1,2.3111668E-1,5.743459E-3,-4.5638257E-3,3.605082E7,5.2E1,3.1E1,8.3395955E6,-2.8959513E-3,-5.282765E-3,-6.4005767E-4,3.6747127E-3,-6.206844E-4,-4.5480225E-3,9.239956E-4,-9.839027E-4,-6.169812E-3,4.9174903E-4,-2.9592053E-3,4.3434035E-3,8.6018825E-3,3.0024275E-3,-4.178631E-4,3.027109E-3,1.7843115E-4],"split_indices":[71,71,47,9,0,71,8,5,72,0,2,52,3,72,77,77,76,0,64,8,8,64,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.69E2,1.3E2,1.39E2,1.26E2,4E0,9.3E1,4.6E1,4.6E1,8E1,6E0,8.7E1,2.5E1,2.1E1,3.5E1,1.1E1,7.2E1,8E0,4E0,8.3E1,1.4E1,1.1E1,1.4E1,7E0,1.9E1,1.6E1,4E0,7E0,4E0,6.8E1,4E0,4E0,7.7E1,6E0,1E1,4E0,7E0,4E0,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.45128E-3,-9.6646714E-4,5.5275573E-3,-5.26827E-3,2.4113716E-3,5.3020795E-3,-1.6022695E-4,-1.4579112E-2,1.700076E-2,-5.3464344E-5,-2.7475825E-3,4.4376818E-3,4.4940962E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,-1,-1,5,-1,7,9,11,-1,-1,-1,-1],"loss_changes":[1.1386705E-1,1.060519E-1,0E0,0E0,7.8013316E-2,0E0,6.557262E-2,8.7067306E-2,6.8600334E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,4,4,6,6,7,7,8,8],"right_children":[2,4,-1,-1,6,-1,8,10,12,-1,-1,-1,-1],"split_conditions":[2.0704236E3,1.393E3,5.5275573E-3,-5.26827E-3,1.55E2,5.3020795E-3,2.8955648E6,1.609091E1,3.2239552E6,-5.3464344E-5,-2.7475825E-3,4.4376818E-3,4.4940962E-4],"split_indices":[71,12,0,0,11,0,69,75,69,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.75E2,7E0,7E0,2.68E2,5E0,2.63E2,1.43E2,1.2E2,1.11E2,3.2E1,9E0,1.11E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[-2.0428915E-3,-2.298763E-2,1.7783502E-2,-1.6606E-2,-8.838524E-3,1.3478886E-2,6.362863E-3,-7.863222E-2,-5.8612945E-3,4.8478255E-3,7.000703E-2,-1.13282606E-1,9.7142556E-4,2.9943136E-3,-5.8129244E-2,4.047851E-3,-7.774816E-4,8.818214E-2,6.815482E-5,-3.3162963E-3,-7.5119673E-3,-1.7218709E-3,8.3968084E-4,-4.5681596E-3,-8.9523644E-4,-4.4629625E-3,2.3876995E-4,2.7285644E-3,6.177155E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,-1,9,-1,11,13,15,17,19,-1,21,23,-1,25,27,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1958804E-1,1.477862E-1,7.48524E-2,9.0176776E-2,0E0,6.991791E-2,0E0,7.150474E-2,5.43292E-2,5.8947764E-2,2.3533605E-2,1.9110903E-2,0E0,6.287007E-2,2.509335E-2,0E0,6.8741135E-2,1.1995606E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17],"right_children":[2,4,6,8,-1,10,-1,12,14,16,18,20,-1,22,24,-1,26,28,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1423952E2,1.5266737E2,7E0,-8.838524E-3,4.7321553E5,6.362863E-3,3.3936841E0,2.7449496E0,2.3152277E2,8.543362E5,2.0189648E0,9.7142556E-4,3.3617156E-4,6.433292E-2,4.047851E-3,1.744E3,2.1E1,6.815482E-5,-3.3162963E-3,-7.5119673E-3,-1.7218709E-3,8.3968084E-4,-4.5681596E-3,-8.9523644E-4,-4.4629625E-3,2.3876995E-4,2.7285644E-3,6.177155E-3],"split_indices":[71,71,75,3,0,47,0,73,72,71,47,77,0,58,58,0,2,3,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.86E2,1.39E2,1.47E2,1.35E2,4E0,1.43E2,4E0,1.9E1,1.16E2,1.25E2,1.8E1,1.4E1,5E0,1E2,1.6E1,7E0,1.18E2,1.4E1,4E0,9E0,5E0,2.7E1,7.3E1,7E0,9E0,6E0,1.12E2,1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"29","size_leaf_vector":"1"}},{"base_weights":[5.7333894E-3,-6.719583E-3,3.6949817E-2,-1.7911758E-2,2.0521104E-2,7.7641965E-3,2.5922228E-2,-1.2368122E-2,-7.7552693E-3,-1.7887685E-2,2.8724687E-2,-5.597706E-3,7.084077E-2,-5.918074E-3,-6.1608693E-3,3.9922236E-4,-1.579337E-3,6.104172E-2,1.6593711E-2,1.9138519E-2,-3.747491E-2,3.8252782E-2,1.1302008E-1,-3.1683547E-3,1.0989693E-4,7.0366677E-4,4.039191E-3,-5.175623E-4,2.1454971E-3,4.318641E-3,5.985876E-5,-2.5381346E-3,5.9405447E-4,2.6148122E-3,-5.241003E-4,6.393806E-3,2.020415E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,-1,11,13,-1,15,17,19,21,-1,23,-1,-1,25,27,29,31,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0909474E-1,6.164896E-2,1.1180337E-1,1.172484E-1,1.910411E-2,0E0,1.06583744E-1,9.944131E-2,0E0,5.0930437E-3,1.819691E-2,3.620069E-2,3.828737E-2,0E0,7.139406E-2,0E0,0E0,1.4535017E-2,3.1426847E-2,3.4505825E-2,1.8497344E-2,1.8131569E-2,1.7141044E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,-1,16,18,20,22,-1,24,-1,-1,26,28,30,32,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.165913E6,1.19881555E-1,2.207E3,9.922613E5,1.8681856E8,7.7641965E-3,2.3101475E5,1E0,-7.7552693E-3,9.141936E7,2.4920886E0,1.3907746E10,3.5659203E8,-5.918074E-3,1E0,3.9922236E-4,-1.579337E-3,4.0973E4,6.098826E6,1.2323925E6,4.904973E6,1.9879582E7,6.97696E5,-3.1683547E-3,1.0989693E-4,7.0366677E-4,4.039191E-3,-5.175623E-4,2.1454971E-3,4.318641E-3,5.985876E-5,-2.5381346E-3,5.9405447E-4,2.6148122E-3,-5.241003E-4,6.393806E-3,2.020415E-3],"split_indices":[1,57,0,52,7,0,47,15,0,7,77,5,7,0,31,0,0,9,67,51,51,51,10,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,2E2,7.9E1,1.42E2,5.8E1,5E0,7.4E1,1.38E2,4E0,1E1,4.8E1,4.4E1,3E1,6E0,1.32E2,4E0,6E0,1.2E1,3.6E1,2.5E1,1.9E1,1.8E1,1.2E1,1.5E1,1.17E2,5E0,7E0,1.9E1,1.7E1,4E0,2.1E1,1.4E1,5E0,1.3E1,5E0,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[3.3789573E-3,1.0063862E-3,5.9794886E-3,-1.5833447E-2,1.3962005E-2,4.0154916E-4,-9.459326E-2,9.232384E-2,8.38452E-3,1.6629538E-2,-4.2887956E-2,-9.929272E-3,-2.7302591E-2,6.7551676E-3,3.9517356E-4,1.3423104E-2,-5.9970953E-2,5.9556314E-3,3.7276512E-4,7.9784606E-4,-5.5066603E-3,-2.3681887E-3,1.216665E-3,-1.188269E-3,9.26776E-4,-1.1442644E-3,-3.6483232E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,21,-1,-1,23,25,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.150084E-2,5.869075E-2,0E0,1.4964181E-1,6.565934E-2,6.955836E-2,1.5704681E-1,4.4292867E-2,5.005631E-2,6.82098E-2,1.268008E-1,0E0,1.9512625E-2,0E0,0E0,3.5774782E-2,4.7977194E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,22,-1,-1,24,26,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5266737E2,1.7546342E8,5.9794886E-3,1.3805645E8,1.8913731E5,3.774648E0,6E0,6.064E3,3.5410094E0,1.8E1,6.8581295E6,-9.929272E-3,8.7E1,6.7551676E-3,3.9517356E-4,2.9266727E2,1.4532832E8,5.9556314E-3,3.7276512E-4,7.9784606E-4,-5.5066603E-3,-2.3681887E-3,1.216665E-3,-1.188269E-3,9.26776E-4,-1.1442644E-3,-3.6483232E-3],"split_indices":[75,7,0,7,52,73,8,0,72,10,64,0,8,0,0,4,12,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,2.67E2,4E0,1.16E2,1.51E2,9.7E1,1.9E1,9E0,1.42E2,7.1E1,2.6E1,6E0,1.3E1,5E0,4E0,1.33E2,9E0,4E0,6.7E1,1.5E1,1.1E1,9E0,4E0,1.9E1,1.14E2,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"27","size_leaf_vector":"1"}},{"base_weights":[2.8497586E-4,-4.8974995E-3,3.4094239E-3,3.8608295E-4,4.765679E-3,5.3401053E-2,-3.6771046E-3,1.1053218E-2,7.816129E-3,-3.4633618E-2,8.691376E-3,1.313361E-3,-9.1668055E-4,-4.7582462E-3,-7.340466E-4,4.8860833E-3,1.8795228E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,-1,3,5,-1,7,9,11,-1,13,15,-1,-1,-1,-1,-1,-1],"loss_changes":[9.367142E-2,0E0,8.257971E-2,5.749536E-2,0E0,9.1750495E-2,9.525651E-2,8.683111E-3,0E0,8.953697E-2,7.950284E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,3,3,5,5,6,6,7,7,9,9,10,10],"right_children":[2,-1,4,6,-1,8,10,12,-1,14,16,-1,-1,-1,-1,-1,-1],"split_conditions":[1.393E3,-4.8974995E-3,2.0704236E3,6.7897725E-1,4.765679E-3,3.084419E6,5.283708E0,1.714493E6,7.816129E-3,1.571112E5,6.186634E0,1.313361E-3,-9.1668055E-4,-4.7582462E-3,-7.340466E-4,4.8860833E-3,1.8795228E-4],"split_indices":[12,0,71,75,0,9,75,9,0,47,75,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,7E0,2.72E2,2.65E2,7E0,1.8E1,2.47E2,1.4E1,4E0,7E1,1.77E2,9E0,5E0,1.4E1,5.6E1,7E0,1.7E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"17","size_leaf_vector":"1"}},{"base_weights":[3.10983E-3,-7.624493E-3,2.9443417E-2,-1.7544327E-2,3.103107E-2,7.3052705E-3,1.8854015E-2,-2.7903078E-2,2.4438936E-2,4.288795E-3,2.1797735E-2,4.2605728E-2,-1.1739445E-2,4.6737855E-3,-3.289646E-2,4.4845756E-2,-3.269469E-2,3.2397922E-2,-1.3011157E-3,5.5790726E-2,-3.1657107E-3,-4.199061E-2,1.7483294E-2,-3.1305389E-3,-6.786296E-4,-9.278801E-4,2.7096653E-3,-4.106672E-3,1.3971356E-3,-8.55749E-4,1.9759678E-3,2.1691662E-3,4.851416E-3,-2.6312477E-3,-5.603203E-4,3.8929696E-3,-3.5541147E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,-1,23,25,27,29,-1,31,-1,33,35,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.909032E-2,7.6852314E-2,1.07850276E-1,6.983009E-2,2.1486253E-2,0E0,5.5725336E-2,8.6056374E-2,3.8775522E-2,0E0,2.0548508E-2,6.697456E-2,3.079088E-2,0E0,7.801902E-2,2.3037449E-2,3.4763258E-2,1.7424144E-2,0E0,1.1813842E-2,0E0,6.9895517E-3,3.1079432E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,21,21,22,22],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,-1,24,26,28,30,-1,32,-1,34,36,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.165913E6,1.2730925E0,2.207E3,2E1,1.7278711E-1,7.3052705E-3,1.9647126E7,1E1,2.5742075E0,4.288795E-3,9.8163464E7,4.613948E7,2.2296374E0,4.6737855E-3,1.31654E5,1.3634276E0,7.05635E0,2.0270285E8,-1.3011157E-3,7.332955E5,-3.1657107E-3,7.386567E0,1.9054878E0,-3.1305389E-3,-6.786296E-4,-9.278801E-4,2.7096653E-3,-4.106672E-3,1.3971356E-3,-8.55749E-4,1.9759678E-3,2.1691662E-3,4.851416E-3,-2.6312477E-3,-5.603203E-4,3.8929696E-3,-3.5541147E-4],"split_indices":[1,61,0,3,57,0,64,10,72,0,51,51,73,0,9,72,72,7,0,47,0,77,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.78E2,1.98E2,8E1,1.58E2,4E1,5E0,7.5E1,1.27E2,3.1E1,4E0,3.6E1,4.2E1,3.3E1,4E0,1.23E2,2.3E1,8E0,3E1,6E0,3.8E1,4E0,1.6E1,1.7E1,4.1E1,8.2E1,4E0,1.9E1,4E0,4E0,5E0,2.5E1,3.4E1,4E0,1E1,6E0,4E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"37","size_leaf_vector":"1"}},{"base_weights":[4.7198535E-4,-2.3544746E-2,1.4797467E-2,-3.4545183E-2,6.45874E-2,5.692478E-2,3.5581242E-3,-2.0373061E-2,-8.94155E-2,7.3223184E-3,-1.159409E-4,9.994532E-2,-1.2835043E-2,4.893333E-3,-1.834599E-3,4.524906E-2,-3.2174483E-2,-1.7877202E-1,-8.176937E-3,5.7646364E-2,9.677123E-3,-2.2535345E-3,7.6785084E-4,4.1204602E-3,-7.6089375E-2,1.8263121E-4,3.9535593E-3,-5.371586E-4,-4.1156937E-3,-1.2559498E-2,-2.2560365E-3,-2.3323393E-3,1.0743535E-3,3.5475001E-3,7.433986E-4,-4.602491E-4,1.4851124E-3,-4.524676E-3,-1.5768595E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,31,33,-1,-1,-1,35,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.668275E-2,1.04027845E-1,8.313052E-2,7.127543E-2,7.6459184E-2,1.123237E-1,7.774955E-2,6.0312867E-2,1.3646291E-1,0E0,0E0,9.513281E-2,1.6802294E-2,0E0,5.9567202E-2,1.9782662E-2,7.522719E-2,9.643978E-2,1.5907137E-2,1.3250995E-2,0E0,0E0,0E0,4.9971424E-2,5.9166625E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,23,23,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,32,34,-1,-1,-1,36,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.283708E0,9.74979E6,2.3891E4,3.5342257E0,2.2E2,8.652176E6,6.0949473E0,1.7144943E-6,1.8196162E0,7.3223184E-3,-1.159409E-4,6.874319E6,6.048052E5,4.893333E-3,2.7317348E7,2.3063551E2,3.5300705E2,1.332E3,4.706415E2,3.3145252E1,9.677123E-3,-2.2535345E-3,7.6785084E-4,9.821357E-1,1.328903E8,1.8263121E-4,3.9535593E-3,-5.371586E-4,-4.1156937E-3,-1.2559498E-2,-2.2560365E-3,-2.3323393E-3,1.0743535E-3,3.5475001E-3,7.433986E-4,-4.602491E-4,1.4851124E-3,-4.524676E-3,-1.5768595E-3],"split_indices":[75,9,9,75,10,48,75,57,72,0,0,1,51,0,64,52,4,0,4,75,0,0,0,60,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.79E2,1.04E2,1.75E2,9.3E1,1.1E1,3.6E1,1.39E2,7.5E1,1.8E1,4E0,7E0,2.2E1,1.4E1,6E0,1.33E2,1.1E1,6.4E1,8E0,1E1,1.7E1,5E0,6E0,8E0,1.24E2,9E0,6E0,5E0,4.8E1,1.6E1,4E0,4E0,4E0,6E0,1.1E1,6E0,8.3E1,4.1E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[2.9344317E-3,-3.9738747E-3,5.162234E-2,5.633973E-3,-2.8893141E-2,1.0010953E-1,1.0626008E-2,-1.1397568E-3,4.4696245E-2,1.1824694E-3,-9.196501E-2,1.3570584E-1,1.1094714E-3,1.900634E-3,-6.698067E-3,-2.796468E-2,1.2843429E-2,8.304936E-2,1.3590881E-2,-2.8411347E-2,2.0164834E-2,-9.492934E-3,-4.3385044E-2,9.435343E-3,2.5506152E-3,1.8425869E-3,-3.190861E-2,1.0081771E-3,-2.3975053E-3,8.933171E-4,-2.8422517E-3,1.3516669E-3,5.6584133E-3,-9.967816E-4,1.3063625E-3,1.710608E-4,-2.0619181E-3,1.3646035E-3,-8.9839863E-4,1.267218E-3,-3.5966546E-3,-1.5156793E-4,-2.492823E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,35,37,-1,39,-1,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.9338146E-2,5.5947762E-2,6.453355E-2,4.476788E-2,1.2340889E-1,3.7316084E-2,1.0366918E-2,5.4748356E-2,2.869597E-2,2.585214E-2,1.07578635E-1,4.4241503E-2,0E0,0E0,1.658841E-2,6.124526E-2,4.8157215E-2,1.9927569E-2,8.428732E-3,9.634585E-3,1.1103762E-2,0E0,4.1556478E-2,0E0,0E0,0E0,5.5483356E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,26,26],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,36,38,-1,40,-1,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,4.352025E0,2.5304102E6,3.8576598E0,7.3917985E6,7.643508E5,1.874657E1,8.461558E10,4.5194294E5,1.664E3,4.752366E0,6.6E1,1.1094714E-3,1.900634E-3,1.6E1,1.47887E5,2.1061536E3,2.199787E8,2.0541462E8,9.141936E7,1.10018056E8,-9.492934E-3,2.7181172E5,9.435343E-3,2.5506152E-3,1.8425869E-3,1E0,1.0081771E-3,-2.3975053E-3,8.933171E-4,-2.8422517E-3,1.3516669E-3,5.6584133E-3,-9.967816E-4,1.3063625E-3,1.710608E-4,-2.0619181E-3,1.3646035E-3,-8.9839863E-4,1.267218E-3,-3.5966546E-3,-1.5156793E-4,-2.492823E-3],"split_indices":[71,73,66,73,64,52,77,50,66,2,75,8,0,0,3,1,4,7,7,7,51,0,52,0,0,0,31,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,2.32E2,3.2E1,1.68E2,6.4E1,1.4E1,1.8E1,1.44E2,2.4E1,4.4E1,2E1,9E0,5E0,6E0,1.2E1,4.9E1,9.5E1,1E1,1.4E1,1.7E1,2.7E1,5E0,1.5E1,4E0,5E0,4E0,8E0,1.6E1,3.3E1,8.8E1,7E0,5E0,5E0,4E0,1E1,6E0,1.1E1,2.2E1,5E0,5E0,1E1,4E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"43","size_leaf_vector":"1"}},{"base_weights":[1.0727954E-3,-7.935507E-3,2.9022856E-2,-1.4421671E-2,2.8189072E-2,7.7344336E-2,1.5957605E-2,-2.3930179E-2,1.576601E-2,-1.7996109E-3,4.521769E-2,5.8754752E-3,1.6223729E-3,3.3855986E-2,-3.1947007E-3,1.4694356E-2,-4.034782E-2,4.7490828E-2,-2.3092072E-2,5.9222557E-2,-1.1716188E-3,-4.626999E-4,4.6287075E-2,-2.4246399E-3,1.0441747E-2,-2.2834345E-5,4.7600544E-3,-8.4693665E-3,-1.2354819E-3,-1.342718E-3,3.1859463E-3,-2.4168193E-3,2.640477E-4,7.390635E-4,4.3214182E-3,3.6260777E-3,1.0178307E-3,-1.0025614E-3,1.5847764E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,-1,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.921874E-2,4.9143E-2,4.1065034E-2,5.1183328E-2,3.842727E-2,2.5136836E-2,1.8619103E-2,8.619959E-2,5.4105584E-2,0E0,2.723803E-2,0E0,0E0,1.5747454E-2,1.8763168E-2,5.5289406E-2,1.7942044E-1,4.195271E-2,1.7251756E-2,3.1390227E-2,0E0,0E0,1.5531905E-2,0E0,1.7745424E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,22,22,24,24],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,-1,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.2730925E0,5.8536225E5,7.013774E5,3.131673E6,1.6814E4,2.7479675E-1,3.133129E8,1.56221E5,9.906531E6,-1.7996109E-3,4.986835E6,5.8754752E-3,1.6223729E-3,1.718354E8,3.4519572E0,9.74979E6,2.07684E5,1.0705044E3,1.08E2,5.855073E6,-1.1716188E-3,-4.626999E-4,3.13423E6,-2.4246399E-3,3.5055136E2,-2.2834345E-5,4.7600544E-3,-8.4693665E-3,-1.2354819E-3,-1.342718E-3,3.1859463E-3,-2.4168193E-3,2.640477E-4,7.390635E-4,4.3214182E-3,3.6260777E-3,1.0178307E-3,-1.0025614E-3,1.5847764E-3],"split_indices":[61,47,52,48,9,57,7,1,48,0,1,0,0,7,73,9,1,74,8,64,0,0,64,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.07E2,6.6E1,1.76E2,3.1E1,1.3E1,5.3E1,1.34E2,4.2E1,6E0,2.5E1,5E0,8E0,2.7E1,2.6E1,4E1,9.4E1,2.3E1,1.9E1,2.1E1,4E0,6E0,2.1E1,5E0,2.1E1,3.5E1,5E0,7E0,8.7E1,5E0,1.8E1,9E0,1E1,1E1,1.1E1,8E0,1.3E1,9E0,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[-1.6682731E-3,-4.392072E-3,4.1339127E-3,-1.0219255E-3,-5.4091185E-2,7.665787E-3,-2.3719488E-2,-5.265569E-3,-4.989092E-3,-2.2933478E-2,1.810739E-2,-9.479532E-3,-1.0821128E-1,-1.574431E-3,1.4044524E-3,-2.7756707E-4,-7.538209E-3,4.6751383E-3,5.5921223E-4,7.8708807E-4,-1.4984503E-3,-2.7210808E-3,-6.699026E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.889884E-2,4.4832926E-2,0E0,4.9873684E-2,4.4475958E-2,5.8884747E-2,8.319537E-2,1.139161E-2,0E0,1.1087772E-1,6.690985E-2,3.794863E-2,9.713791E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,3.7085085E0,4.1339127E-3,1.6585214E7,5.743459E-3,3.81E2,2.597438E5,5.3725034E-2,-4.989092E-3,8.5585E4,4.56E2,2.236127E1,1.671E3,-1.574431E-3,1.4044524E-3,-2.7756707E-4,-7.538209E-3,4.6751383E-3,5.5921223E-4,7.8708807E-4,-1.4984503E-3,-2.7210808E-3,-6.699026E-3],"split_indices":[71,72,0,64,76,0,47,61,0,48,0,75,2,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.74E2,2.67E2,7E0,2.51E2,1.6E1,1.82E2,6.9E1,9E0,7E0,4.6E1,1.36E2,6E1,9E0,5E0,4E0,4.2E1,4E0,8E0,1.28E2,2.8E1,3.2E1,5E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[1.5023522E-3,-1.5602255E-2,1.7441092E-2,-1.0162039E-2,-7.2873607E-3,7.895297E-4,4.7273334E-2,4.0240665E-3,-1.5418164E-2,4.009565E-3,-5.931247E-3,3.6895003E-2,6.4239395E-3,-5.0314046E-2,2.2090576E-3,3.4455676E-2,-2.030968E-2,5.093208E-2,-1.0630324E-2,-3.1732633E-3,-9.529626E-5,1.5703777E-3,-7.976968E-4,4.5256414E-3,-4.788931E-4,-5.6292643E-3,-4.6572112E-4,3.8760952E-3,1.3937361E-3,-1.999745E-3,1.3541731E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,-1,9,11,-1,13,-1,15,17,-1,19,21,23,25,27,29,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.79705E-2,1.0550657E-1,7.3482186E-2,7.006374E-2,0E0,5.6195017E-2,4.559607E-2,0E0,7.884618E-2,0E0,5.2962836E-2,3.3230133E-2,0E0,3.8460568E-2,5.4181363E-2,6.975081E-2,6.8401605E-2,2.3560517E-2,1.704001E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,8,8,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18],"right_children":[2,4,6,8,-1,10,12,-1,14,-1,16,18,-1,20,22,24,26,28,30,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.8041378E5,1E1,-7.2873607E-3,2.3152277E2,1.9936699E3,4.0240665E-3,7.5608E4,4.009565E-3,2.3891E4,9.12179E6,6.4239395E-3,4.5510144E9,1.9179105E0,1.0326834E7,9.577834E7,3.0855308E0,2.820341E0,-3.1732633E-3,-9.529626E-5,1.5703777E-3,-7.976968E-4,4.5256414E-3,-4.788931E-4,-5.6292643E-3,-4.6572112E-4,3.8760952E-3,1.3937361E-3,-1.999745E-3,1.3541731E-3],"split_indices":[71,71,47,10,0,71,4,0,9,0,9,9,0,5,72,1,7,73,73,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.84E2,1.37E2,1.47E2,1.33E2,4E0,9.5E1,5.2E1,6E0,1.27E2,6E0,8.9E1,4.8E1,4E0,4.2E1,8.5E1,2.3E1,6.6E1,3.7E1,1.1E1,3E1,1.2E1,3.2E1,5.3E1,9E0,1.4E1,5E0,6.1E1,1.3E1,2.4E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"31","size_leaf_vector":"1"}},{"base_weights":[7.789992E-4,-1.875018E-3,3.9721313E-3,-5.6791026E-2,1.5086506E-3,1.4423865E-3,-9.168243E-2,-2.0941436E-2,9.950489E-3,-5.61779E-3,-2.1209961E-3,-4.1190768E-3,-8.806837E-3,9.836872E-4,4.7898598E-2,6.4780624E-3,-7.6117937E-4,-3.2906476E-3,2.6410862E-4,3.621947E-3,3.852045E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,-1,9,11,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[6.0291413E-2,4.802661E-2,0E0,4.899134E-2,4.6451256E-2,0E0,9.599768E-3,1.9120288E-1,6.0521506E-2,0E0,0E0,1.1378614E-1,0E0,5.034206E-2,4.069835E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,6,6,7,7,8,8,11,11,13,13,14,14],"right_children":[2,4,-1,6,8,-1,10,12,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,8.89E2,3.9721313E-3,1.4E1,1.8196162E0,1.4423865E-3,1.6E1,1.7885118E0,4.3857228E5,-5.61779E-3,-2.1209961E-3,3.9449276E1,-8.806837E-3,1.3944E5,3.0855308E0,6.4780624E-3,-7.6117937E-4,-3.2906476E-3,2.6410862E-4,3.621947E-3,3.852045E-4],"split_indices":[71,2,0,3,72,0,3,72,47,0,0,71,0,12,73,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.64E2,2.57E2,7E0,1.4E1,2.43E2,4E0,1E1,6.6E1,1.77E2,5E0,5E0,6.1E1,5E0,1.44E2,3.3E1,4E0,5.7E1,8E0,1.36E2,1.8E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-1.148217E-3,-2.1452382E-2,1.1460967E-2,6.602388E-3,-4.762114E-2,4.3823905E-3,7.3816353E-3,-5.7850122E-2,1.934617E-2,-1.2612973E-1,-2.7812105E-2,2.1267667E-2,-1.0790346E-2,-4.4773915E-3,-3.1687386E-4,5.3495683E-2,-5.401492E-3,-9.006185E-3,-1.6706236E-4,-6.262814E-2,-9.923683E-4,5.697159E-2,5.4963017E-3,1.393674E-2,-3.659143E-2,3.4991547E-4,3.6664694E-3,-1.8212503E-3,1.3416331E-3,1.5623749E-3,-4.0925746E-3,1.3614483E-3,-1.9180661E-3,5.968352E-4,4.232015E-3,2.4256683E-3,-4.2030716E-4,-1.6058675E-3,2.0196324E-3,-3.2132403E-3,-8.836243E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,-1,25,27,-1,-1,29,31,33,35,37,39,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.218719E-2,7.960352E-2,5.872459E-2,4.4651724E-2,8.358212E-2,0E0,4.2369716E-2,1.7402785E-2,3.831938E-2,9.291853E-2,4.2890675E-2,5.3181328E-2,4.7082916E-2,0E0,0E0,2.21885E-2,3.3006832E-2,0E0,0E0,5.373402E-2,3.4790438E-2,4.340905E-2,4.6720725E-2,5.7048574E-2,1.9455835E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,-1,26,28,-1,-1,30,32,34,36,38,40,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.283708E0,1.51331E5,6.186634E0,3.9490446E-1,2.07684E5,4.3823905E-3,1.3879616E7,2.83E2,4.352025E0,1.8550434E0,9.526745E1,4.0012E4,2.5659466E1,-4.4773915E-3,-3.1687386E-4,1.2E1,5.268293E0,-9.006185E-3,-1.6706236E-4,1.1078758E0,1.7E1,1.68834E5,1.13874E5,9.707008E2,1.3548306E3,3.4991547E-4,3.6664694E-3,-1.8212503E-3,1.3416331E-3,1.5623749E-3,-4.0925746E-3,1.3614483E-3,-1.9180661E-3,5.968352E-4,4.232015E-3,2.4256683E-3,-4.2030716E-4,-1.6058675E-3,2.0196324E-3,-3.2132403E-3,-8.836243E-4],"split_indices":[75,1,75,77,1,0,64,0,73,72,71,9,75,0,0,3,73,0,0,73,8,48,48,4,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,1.07E2,1.73E2,5.2E1,5.5E1,7E0,1.66E2,8E0,4.4E1,1E1,4.5E1,9.4E1,7.2E1,4E0,4E0,1.8E1,2.6E1,6E0,4E0,1.9E1,2.6E1,2.8E1,6.6E1,3.7E1,3.5E1,7E0,1.1E1,1.3E1,1.3E1,4E0,1.5E1,1.5E1,1.1E1,1.3E1,1.5E1,1.5E1,5.1E1,1.4E1,2.3E1,1.1E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"41","size_leaf_vector":"1"}},{"base_weights":[-1.9003861E-3,-3.8611423E-3,4.8409794E-3,-1.2841899E-2,1.8640252E-2,-5.1921415E-3,-4.7663875E-2,2.45833E-2,-3.363066E-3,3.7808663E-3,-9.101748E-3,-6.942747E-3,-2.9492326E-2,5.644717E-2,1.4089741E-2,-2.708027E-3,-1.5446395E-4,-3.8435345E-4,-3.6948293E-3,6.5807E-4,3.5144568E-3,-1.0141828E-3,1.0240912E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,5,7,9,11,13,-1,-1,15,-1,17,19,21,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.8335505E-2,5.5205017E-2,0E0,5.1696558E-2,4.4297084E-2,5.570542E-2,6.1144345E-2,2.3947716E-2,0E0,0E0,4.408755E-2,0E0,3.26008E-2,1.3997875E-2,1.7275069E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,10,10,12,12,13,13,14,14],"right_children":[2,4,-1,6,8,10,12,14,-1,-1,16,-1,18,20,22,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5266737E2,7.91732E-1,4.8409794E-3,4.352025E0,7.2331117E8,1E1,4.5200844E0,7.013774E5,-3.363066E-3,3.7808663E-3,3.4585872E8,-6.942747E-3,7.188264E-2,5.2247315E6,3.4519572E0,-2.708027E-3,-1.5446395E-4,-3.8435345E-4,-3.6948293E-3,6.5807E-4,3.5144568E-3,-1.0141828E-3,1.0240912E-3],"split_indices":[75,58,0,73,7,10,73,52,0,0,5,0,76,51,73,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.75E2,2.71E2,4E0,1.94E2,7.7E1,1.6E2,3.4E1,7.3E1,4E0,6E0,1.54E2,4E0,3E1,1.7E1,5.6E1,1.5E1,1.39E2,2.2E1,8E0,6E0,1.1E1,1E1,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[2.186097E-3,-1.1866848E-2,1.5157879E-2,-6.546152E-3,-6.806321E-3,4.703188E-2,4.199202E-3,-1.3600078E-2,4.4563077E-2,7.7345185E-2,-8.293204E-3,-4.8439032E-3,5.3423297E-2,-3.1408814E-3,-4.7572985E-2,6.698947E-3,-1.2073182E-3,1.3222776E-1,2.0734707E-2,-1.8886598E-3,9.843785E-4,-1.0600052E-2,2.30472E-3,6.8500645E-2,1.628716E-4,-3.7994857E-3,7.2084506E-5,-8.4348134E-4,-3.8035568E-3,-1.4896881E-3,1.1815849E-3,7.8587036E-4,8.64069E-3,1.7055051E-3,-5.878248E-4,-1.8866541E-3,6.123393E-5,3.8793094E-3,1.2172322E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,-1,35,-1,37,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.085698E-2,9.597147E-2,5.045055E-2,4.7473762E-2,0E0,6.2693626E-2,4.877889E-2,4.071345E-2,7.4094005E-2,7.1824595E-2,1.4473284E-2,2.9920764E-2,1.2544144E-2,3.33813E-2,2.6161447E-2,0E0,1.0874222E-2,7.341756E-2,7.575712E-3,0E0,0E0,3.0858992E-2,0E0,6.3885376E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,21,21,23,23],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,-1,36,-1,38,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.2394615E2,2.1450777E2,2.3891E4,9.750871E6,-6.806321E-3,8.652176E6,4.7321553E5,1.714493E6,1.17951E5,8.833194E5,5.4527334E9,4.4E1,1.53E2,5.6923714E1,2.1894946E0,6.698947E-3,3.5631923E8,1.401746E6,6.427584E6,-1.8886598E-3,9.843785E-4,1.83437E5,2.30472E-3,9.0871654E-2,1.628716E-4,-3.7994857E-3,7.2084506E-5,-8.4348134E-4,-3.8035568E-3,-1.4896881E-3,1.1815849E-3,7.8587036E-4,8.64069E-3,1.7055051E-3,-5.878248E-4,-1.8866541E-3,6.123393E-5,3.8793094E-3,1.2172322E-3],"split_indices":[71,71,9,9,0,48,47,9,1,52,12,3,8,69,72,0,7,1,1,0,0,9,0,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.77E2,1.33E2,1.44E2,1.29E2,4E0,3.6E1,1.08E2,1.14E2,1.5E1,2.3E1,1.3E1,9.2E1,1.6E1,8.8E1,2.6E1,4E0,1.1E1,1.1E1,1.2E1,6E0,7E0,8.4E1,8E0,1.2E1,4E0,4E0,8.4E1,1.5E1,1.1E1,5E0,6E0,4E0,7E0,8E0,4E0,2.3E1,6.1E1,8E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"39","size_leaf_vector":"1"}},{"base_weights":[1.6985955E-3,-1.5817555E-2,1.2537015E-2,-2.4081336E-2,5.9994448E-2,8.117583E-2,6.564742E-3,-1.2527099E-2,-6.395175E-2,6.3077626E-3,-1.6493232E-4,7.228045E-4,1.0579659E-1,4.46471E-3,3.0899923E-3,4.208224E-2,-2.2120904E-2,-1.4637566E-1,-6.3969824E-3,2.6258028E-3,6.5910104E-3,3.9268356E-2,-6.0501467E-3,2.0237388E-4,3.6389187E-3,-4.977804E-4,-5.4104105E-3,-9.13431E-3,-2.996411E-3,1.9949274E-3,-1.409797E-3,1.1378855E-3,5.198764E-3,2.5137726E-5,-3.4834195E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,-1,21,23,25,27,29,-1,-1,31,33,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.411496E-2,6.9732584E-2,7.161473E-2,4.4805422E-2,5.4346256E-2,2.0908296E-2,5.09058E-2,4.1795984E-2,1.034292E-1,0E0,0E0,0E0,1.0053404E-2,0E0,5.248427E-2,1.6364764E-2,7.100971E-2,2.5148034E-2,1.8190356E-2,0E0,0E0,3.1508498E-2,5.835324E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,14,14,15,15,16,16,17,17,18,18,21,21,22,22],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,-1,22,24,26,28,30,-1,-1,32,34,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.283708E0,9.750914E6,2.222512E3,3.7570093E0,4.56E2,8.562227E4,6.186634E0,1.7144943E-6,4.0054E4,6.3077626E-3,-1.6493232E-4,7.228045E-4,7.59093E5,4.46471E-3,2.3891E4,2.3063551E2,1.3599619E7,9.48E2,4.0098386E0,2.6258028E-3,6.5910104E-3,1.7258567E0,2.7317348E7,2.0237388E-4,3.6389187E-3,-4.977804E-4,-5.4104105E-3,-9.13431E-3,-2.996411E-3,1.9949274E-3,-1.409797E-3,1.1378855E-3,5.198764E-3,2.5137726E-5,-3.4834195E-3],"split_indices":[75,9,66,75,0,47,75,57,9,0,0,0,1,0,9,52,64,0,75,0,0,58,64,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.83E2,1.08E2,1.75E2,9.8E1,1E1,1.3E1,1.62E2,7.7E1,2.1E1,4E0,6E0,4E0,9E0,5E0,1.57E2,1.1E1,6.6E1,8E0,1.3E1,5E0,4E0,3.1E1,1.26E2,6E0,5E0,6E1,6E0,4E0,4E0,4E0,9E0,2.7E1,4E0,1.16E2,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"35","size_leaf_vector":"1"}},{"base_weights":[2.1254176E-3,-3.7178583E-3,4.7106627E-2,-3.2828588E-4,-4.796238E-2,8.963617E-2,1.1411061E-2,-3.9945E-3,1.6465288E-3,-4.5591295E-3,-2.3175336E-3,8.226878E-3,1.6642828E-3,-5.4929177E-3,2.5994773E-3,4.17021E-3,-2.6566945E-3,1.2388675E-3,-1.2102257E-3,-8.9356425E-4,1.0790093E-3,-2.9429408E-5,1.9185381E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,-1,15,-1,17,-1,-1,19,-1,21,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.087845E-2,3.5827085E-2,4.603876E-2,3.8205758E-2,3.9391737E-2,6.0224503E-2,1.401724E-2,0E0,3.304215E-2,0E0,7.7121193E-3,0E0,0E0,6.0904752E-3,0E0,3.808103E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,10,10,13,13,15,15],"right_children":[2,4,6,8,10,12,14,-1,16,-1,18,-1,-1,20,-1,22,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.230905E2,1E0,2.45226E6,4.9759598E1,2.5952345E5,3.0256798E0,3.2773333E0,-3.9945E-3,2.179529E3,-4.5591295E-3,1.162461E6,8.226878E-3,1.6642828E-3,3.91101E5,2.5994773E-3,3.646166E6,-2.6566945E-3,1.2388675E-3,-1.2102257E-3,-8.9356425E-4,1.0790093E-3,-2.9429408E-5,1.9185381E-3],"split_indices":[71,24,48,67,47,73,73,0,4,0,9,0,0,9,0,48,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.68E2,2.38E2,3E1,2.22E2,1.6E1,1.3E1,1.7E1,4E0,2.18E2,7E0,9E0,4E0,9E0,1.3E1,4E0,2.1E2,8E0,4E0,5E0,9E0,4E0,1.87E2,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"23","size_leaf_vector":"1"}},{"base_weights":[-1.3789119E-3,-3.79684E-3,3.5624779E-3,5.2657295E-3,-2.4627104E-2,-6.0390597E-5,5.0961412E-2,-5.7459297E-3,-1.6172167E-2,4.8569096E-3,-4.5848973E-2,7.337332E-2,-4.524266E-4,-4.8507955E-2,-4.11966E-3,-3.8473706E-3,3.6245395E-4,-4.1481538E-4,-5.7594357E-3,1.5623573E-3,4.5447606E-3,-5.057729E-3,-1.2111566E-3,3.8432356E-4,-2.2785233E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,-1,5,7,9,11,-1,13,15,17,19,-1,21,23,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.1136993E-2,4.9211364E-2,0E0,4.429837E-2,6.489128E-2,3.71483E-2,2.6606128E-2,0E0,2.863999E-2,4.0386137E-2,4.5581926E-2,1.0640584E-2,0E0,2.4349615E-2,3.1522315E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,13,13,14,14],"right_children":[2,4,-1,6,8,10,12,-1,14,16,18,20,-1,22,24,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.0704236E3,1.3879616E7,3.5624779E-3,1.1152775E7,2.6376405E0,1E0,2.3E1,-5.7459297E-3,1.4E1,4.9759598E1,4E1,2.39869E5,-4.524266E-4,4.68646E5,3.6E1,-3.8473706E-3,3.6245395E-4,-4.1481538E-4,-5.7594357E-3,1.5623573E-3,4.5447606E-3,-5.057729E-3,-1.2111566E-3,3.8432356E-4,-2.2785233E-3],"split_indices":[71,64,0,64,75,20,8,0,3,67,8,12,0,48,3,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.66E2,2.59E2,7E0,1.81E2,7.8E1,1.63E2,1.8E1,5E0,7.3E1,1.48E2,1.5E1,1.3E1,5E0,1.9E1,5.4E1,4E0,1.44E2,1.1E1,4E0,6E0,7E0,4E0,1.5E1,4.3E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[2.1883086E-3,5.3948365E-2,-1.2574306E-3,-1.523033E-3,8.32709E-2,-4.680003E-3,7.3944486E-4,6.0291803E-3,1.0908832E-3,-4.9699387E-3,3.1123426E-2,-2.577991E-3,-4.786399E-3,7.080273E-2,1.158934E-2,-3.4626914E-4,1.6499638E-3,4.4658636E-3,4.3701072E-4,-1.4053816E-3,1.6888662E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,-1,7,-1,9,-1,-1,11,13,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[4.857949E-2,4.6136703E-2,5.1395074E-2,0E0,3.3826627E-2,0E0,4.387008E-2,0E0,0E0,5.016561E-2,3.0183397E-2,3.9954428E-2,0E0,1.995761E-2,3.0708361E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,9,9,10,10,11,11,13,13,14,14],"right_children":[2,4,6,-1,8,-1,10,-1,-1,12,14,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[7.4745026E-6,3.1982166E2,9.421523E-5,-1.523033E-3,1.904E3,-4.680003E-3,5.1428344E5,6.0291803E-3,1.0908832E-3,4.8143003E5,5.658968E4,5.707753E9,-4.786399E-3,3.0855308E0,2.238E3,-3.4626914E-4,1.6499638E-3,4.4658636E-3,4.3701072E-4,-1.4053816E-3,1.6888662E-3],"split_indices":[57,4,61,0,0,0,47,0,0,47,52,12,0,73,2,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.71E2,1.6E1,2.55E2,4E0,1.2E1,4E0,2.51E2,6E0,6E0,2.12E2,3.9E1,2.08E2,4E0,1.2E1,2.7E1,1.85E2,2.3E1,8E0,4E0,1E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}},{"base_weights":[-2.6577973E-4,-4.9572866E-3,3.5426162E-2,2.8655713E-3,-7.140143E-3,5.246036E-3,1.3757978E-2,-6.971195E-2,-4.1682194E-3,5.1900104E-2,-5.0334493E-3,-4.4403644E-3,-8.4463606E-4,2.7000222E-2,-8.535775E-3,3.556E-3,7.45164E-4,1.0096858E-2,-2.2752422E-3,2.9915277E-4,2.7605947E-3,-1.0845154E-3,1.4230073E-4,-9.628285E-4,1.071563E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,-1,7,-1,9,11,13,15,17,-1,-1,19,21,-1,-1,23,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7563683E-2,3.7261296E-2,5.4575536E-2,0E0,4.508333E-2,0E0,1.961335E-2,1.3334982E-2,3.2084867E-2,6.8936944E-3,1.32583575E-2,0E0,0E0,1.920723E-2,3.604572E-2,0E0,0E0,6.6159507E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17],"right_children":[2,4,6,-1,8,-1,10,12,14,16,18,-1,-1,20,22,-1,-1,24,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,3.299865E1,1.4881903E5,2.8655713E-3,1.0742589E-1,5.246036E-3,1.91016E5,1.49724E5,2.8558656E4,8.982092E5,2.011653E7,-4.4403644E-3,-8.4463606E-4,6.2229652E0,2.0718746E8,3.556E-3,7.45164E-4,1.08E2,-2.2752422E-3,2.9915277E-4,2.7605947E-3,-1.0845154E-3,1.4230073E-4,-9.628285E-4,1.071563E-3],"split_indices":[71,71,52,0,77,0,10,12,52,52,51,0,0,75,7,0,0,8,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.82E2,2.5E2,3.2E1,7E0,2.43E2,6E0,2.6E1,1E1,2.33E2,8E0,1.8E1,6E0,4E0,2.8E1,2.05E2,4E0,4E0,1.4E1,4E0,1.8E1,1E1,8.9E1,1.16E2,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"25","size_leaf_vector":"1"}},{"base_weights":[1.9935807E-3,-3.381235E-3,3.906448E-3,4.2769965E-3,1.9144501E-3,-2.1533134E-4,3.7737158E-3,-7.6527786E-3,1.828326E-2,-2.5807107E-3,-1.2574975E-5,1.1034271E-3,-2.3946678E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,-1,3,-1,5,7,-1,9,11,-1,-1,-1,-1],"loss_changes":[4.083338E-2,0E0,4.8570175E-2,0E0,4.6049505E-2,3.646164E-2,0E0,6.749025E-2,3.1606868E-2,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,2,2,4,4,5,5,7,7,8,8],"right_children":[2,-1,4,-1,6,8,-1,10,12,-1,-1,-1,-1],"split_conditions":[5.491887E-2,-3.381235E-3,1.8E1,4.2769965E-3,2.0704236E3,9.821357E-1,3.7737158E-3,2.8490282E8,7.06699E8,-2.5807107E-3,-1.2574975E-5,1.1034271E-3,-2.3946678E-3],"split_indices":[77,0,10,0,71,60,0,5,7,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.8E2,6E0,2.74E2,5E0,2.69E2,2.63E2,6E0,1.88E2,7.5E1,2.4E1,1.64E2,7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"13","size_leaf_vector":"1"}},{"base_weights":[2.6580675E-3,-1.7229673E-3,3.8347915E-2,-3.4149294E-3,9.542251E-5,7.9968974E-2,1.8159931E-3,4.1128984E-3,-2.1833414E-3,7.482641E-3,1.4132778E-3,1.703889E-2,-1.7553951E-3,-5.3416777E-2,9.328564E-5,5.0892515E-5,1.4060145E-3,-1.0593937E-3,-3.6474573E-3,1.336763E-3,-2.982345E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,-1,7,9,11,-1,13,-1,-1,15,-1,17,19,-1,-1,-1,-1,-1,-1],"loss_changes":[4.2908292E-2,3.228325E-2,4.5469698E-2,0E0,4.899833E-2,5.1957414E-2,1.0986385E-2,0E0,2.741525E-2,0E0,0E0,2.7624494E-3,0E0,6.023675E-3,4.3165687E-2,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,4,4,5,5,6,6,8,8,11,11,13,13,14,14],"right_children":[2,4,6,-1,8,10,12,-1,14,-1,-1,16,-1,18,20,-1,-1,-1,-1,-1,-1],"split_conditions":[7.3717834E2,5.447539E-2,2.45226E6,-3.4149294E-3,1.8E1,3.0256798E0,6.236639E3,4.1128984E-3,8.89E2,7.482641E-3,1.4132778E-3,1.4196E4,-1.7553951E-3,6.078693E1,2.83E2,5.0892515E-5,1.4060145E-3,-1.0593937E-3,-3.6474573E-3,1.336763E-3,-2.982345E-4],"split_indices":[71,77,48,0,10,73,4,0,2,0,0,2,0,71,10,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[2.73E2,2.44E2,2.9E1,5E0,2.39E2,1.3E1,1.6E1,5E0,2.34E2,4E0,9E0,1.2E1,4E0,9E0,2.25E2,6E0,6E0,5E0,4E0,4.1E1,1.84E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"21","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"2.5273216E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics index aa269ac37..dcbcdc3b2 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/emr.metrics @@ -1,16 +1,16 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,executorCPUTime_mean,0.17345361,385.24405903321025,467.8813683958216,19.72340425531915,124.86993483142004,227.4531807372176,450.4204615898145,2701.2746979693256 -1,sw_bytesWrittenRatio,0.060461305,0.7574515843694644,1.0990437874397825,3.111405844716284e-06,0.010259891494622527,0.2834431853138735,1.2217158523672178,9.081261069488061 +0,executorCPUTime_mean,0.17345336,385.24405903321025,467.8813683958216,19.72340425531915,124.86993483142004,227.4531807372176,450.4204615898145,2701.2746979693256 +1,sw_bytesWrittenRatio,0.060457923,0.7574515843694644,1.0990437874397825,3.111405844716284e-06,0.010259891494622527,0.2834431853138735,1.2217158523672178,9.081261069488061 2,sr_localBytesReadRatio,0.050699808,0.10907855488413434,0.1572488413305017,0.0,0.0012885253155292813,0.041012881630626864,0.16840515372594633,1.1303846138795548 3,duration_sum,0.040852662,5830139.981012658,13826598.408761906,833.0,221623.25,849893.0,4253101.0,108868560.0 4,sr_remoteBytesReadRatio,0.03617715,0.6841161818135831,0.988720163289033,0.0,0.00840144787396097,0.2724875681349891,1.0153017802153128,7.92803688197291 5,sr_totalBytesReadRatio,0.03286415,0.7931947366977176,1.1357829439148655,0.0,0.009510088645474939,0.31144224935284237,1.2217168915759054,9.058421495852466 -6,scan_bw,0.030713854,304418.38097439846,279823.7113834629,0.0,106685.49402420361,226914.32332729362,395659.04820326006,2351867.4072114737 +6,scan_bw,0.030746786,304418.38097439846,279823.7113834629,0.0,106685.49402420361,226914.32332729362,395659.04820326006,2351867.4072114737 7,peakExecutionMemory_max,0.022286894,267416911.3164557,282858013.903724,0.0,91147194.0,209626522.0,356306803.0,2484692224.0 8,duration_max,0.020744069,4532.588607594937,7861.628366103739,279.0,1495.0,2260.0,4454.25,76443.0 -9,input_bytesRead_mean,0.018767685,11518480.280642062,10003487.271005625,246703.55,3864346.768096071,8773197.283440515,16950376.052759163,48542081.34584912 +9,input_bytesRead_mean,0.018743865,11518480.280642062,10003487.271005625,246703.55,3864346.768096071,8773197.283440515,16950376.052759163,48542081.34584912 10,input_recordsRead_sum,0.015658852,5752550017.886076,10469873796.719795,600000.0,342468680.5,1546503054.5,5940458836.25,62250956051.0 -11,jvmGCTime_mean,0.014921852,20.763371524713037,31.45530846264389,0.0,3.1836860637509266,10.763938423623948,24.154533487525864,264.73770491803276 +11,jvmGCTime_mean,0.014941257,20.763371524713037,31.45530846264389,0.0,3.1836860637509266,10.763938423623948,24.154533487525864,264.73770491803276 12,executorDeserializeCPUTime_mean,0.01475114,2.977540440295376,5.362990521015718,1.117794486215539,1.7836434782608697,2.0341148870122012,2.313938980177974,67.83163664839468 13,resultSize_max,0.01412687,4259722.604430379,7980080.385944204,4478.0,29505.25,209965.0,3181904.0,36244371.0 14,duration_mean,0.013623247,1142.8704390258872,1289.0697905015472,51.63636363636363,370.90520661942685,705.6111359893133,1235.5146013340639,7346.105752669652 @@ -22,15 +22,15 @@ 20,numTasks_sum,0.007906574,3060.3417721518986,4694.388746013052,16.0,450.5,1190.0,3503.0,28649.0 21,executorRunTime_mean,0.007216744,1127.6041510906016,1290.907970733605,43.31818181818182,357.7099260827749,691.5260484807051,1221.9890104225306,7335.390285911127 22,sw_bytesWritten_mean,0.0059271795,5047440.067791227,8402003.71819362,49.84697781178271,75227.21439230934,1929614.095980947,6084266.838750081,53230647.67712235 -23,sr_localBytesRead_mean,0.0055841953,706832.6039216269,1136165.0469603462,0.0,9099.940787169458,281254.8628198895,868796.7317053758,6654921.041510612 +23,sr_localBytesRead_mean,0.005583745,706832.6039216269,1136165.0469603462,0.0,9099.940787169458,281254.8628198895,868796.7317053758,6654921.041510612 24,sr_localBlocksFetched_sum,0.00523463,157263.2753164557,416390.35593890015,0.0,395.0,3009.5,54033.5,3003536.0 -25,duration_min,0.0050956905,21.89873417721519,30.24821965080589,2.0,8.0,14.0,22.0,307.0 +25,duration_min,0.005096033,21.89873417721519,30.24821965080589,2.0,8.0,14.0,22.0,307.0 26,resultSerializationTime_sum,0.0046446347,62.12341772151899,102.07936822944863,0.0,0.0,16.5,74.25,655.0 -27,sr_fetchWaitTime_mean,0.0043132,2.0310323687364957,7.772144195192904,0.0,0.0,0.009293922766632705,0.5706619382246294,73.13925518040864 +27,sr_fetchWaitTime_mean,0.004317692,2.0310323687364957,7.772144195192904,0.0,0.0,0.009293922766632705,0.5706619382246294,73.13925518040864 28,sr_remoteBytesRead_mean,0.0037650703,4582114.2944864975,7539697.016437123,0.0,63015.63798596112,1893140.6053443695,5593406.281292146,46577051.96917915 29,shuffle_read_bw,0.003720415,57400724.30115602,355780622.4636368,0.0,0.0,787237.6107834032,15536830.513087355,5344125829.0 -30,sw_writeTime_mean,0.0034172973,13.955817771143279,76.94026532595454,0.0,0.9470078594476574,2.976893695794274,6.720336490000438,1237.209407934196 -31,sqlOp_SubqueryBroadcast,0.0026658853,0.7879746835443038,0.4093909677025221,0.0,1.0,1.0,1.0,1.0 +30,sw_writeTime_mean,0.0034189348,13.955817771143279,76.94026532595454,0.0,0.9470078594476574,2.976893695794274,6.720336490000438,1237.209407934196 +31,sqlOp_SubqueryBroadcast,0.0026436863,0.7879746835443038,0.4093909677025221,0.0,1.0,1.0,1.0,1.0 32,sqlOp_OutputAdapter,0.0023861902,0.0759493670886076,0.2653372528417936,0.0,0.0,0.0,0.0,1.0 33,memoryBytesSpilledRatio,0.0017477395,0.34731956316194224,2.239254586395395,0.0,0.0,0.0,0.0,35.42291753534751 34,sr_remoteBlocksFetched_sum,0.0017427115,1100644.082278481,2914598.137768614,0.0,2770.5,21088.5,378179.75,21016512.0 @@ -42,75 +42,75 @@ 40,sqlOp_GenerateBloomFilter,0.00047086703,0.0759493670886076,0.2653372528417936,0.0,0.0,0.0,0.0,1.0 41,sqlOp_Sort,0.0003860592,0.7468354430379747,0.4355140565698905,0.0,0.0,1.0,1.0,1.0 42,sqlOp_Subquery,0.00029955225,0.08544303797468354,0.2799832079697408,0.0,0.0,0.0,0.0,1.0 -43,sqlOp_HashAggregatePrefixGroupingSets,9.980674e-05,0.10443037974683544,0.30630308399462325,0.0,0.0,0.0,0.0,1.0 +43,sqlOp_HashAggregatePrefixGroupingSets,0.00019456695,0.10443037974683544,0.30630308399462325,0.0,0.0,0.0,0.0,1.0 44,diskBytesSpilled_mean,8.3468614e-05,1407298.6859824378,5836681.87707674,0.0,0.0,0.0,0.0,56913253.10538942 -45,sqlOp_ReusedSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 -46,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -47,sqlOp_Filter,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -48,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -49,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -50,sqlOp_Project,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -51,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -52,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -53,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -54,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -55,sqlOp_HashAggregate,0.0,0.990506329113924,0.09712565886833821,0.0,1.0,1.0,1.0,1.0 -56,sqlOp_LocalTableScan,0.0,0.0031645569620253164,0.056254395046301184,0.0,0.0,0.0,0.0,1.0 -57,maxOnHeapMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 -58,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_WindowGroupLimit,0.0,0.0189873417721519,0.1366965815037675,0.0,0.0,0.0,0.0,1.0 -61,sqlOp_Window,0.0,0.12341772151898735,0.3294377550897916,0.0,0.0,0.0,0.0,1.0 -62,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_SubqueryOutputBroadcast,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 -64,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -66,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -73,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_BroadcastNestedLoopJoin,0.0,0.028481012658227847,0.16660637848720578,0.0,0.0,0.0,0.0,1.0 -78,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,sqlOp_AQEShuffleRead,0.0,0.930379746835443,0.25490959760979287,0.0,1.0,1.0,1.0,1.0 -80,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -82,sqlOp_Scan parquet,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -83,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -84,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -85,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 -86,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 -88,maxMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 -89,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -92,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,executorMemory,0.0,16384.0,0.0,16384.0,16384.0,16384.0,16384.0,16384.0 -95,sqlOp_ColumnarToRow,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 -96,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -97,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -98,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -101,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -102,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -105,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -106,sqlOp_BroadcastExchange,0.0,0.9240506329113924,0.2653372528417936,0.0,1.0,1.0,1.0,1.0 -107,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -108,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -109,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -111,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -112,platform_emr,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -113,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -114,sqlOp_WindowSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +45,sqlOp_Exchange,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +46,sqlOp_ColumnarToRow,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +47,sqlOp_AQEShuffleRead,0.0,0.930379746835443,0.25490959760979287,0.0,1.0,1.0,1.0,1.0 +48,output_recordsWritten_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +49,sqlOp_BroadcastNestedLoopJoin,0.0,0.028481012658227847,0.16660637848720578,0.0,0.0,0.0,0.0,1.0 +50,sqlOp_BroadcastExchange,0.0,0.9240506329113924,0.2653372528417936,0.0,1.0,1.0,1.0,1.0 +51,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +52,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +53,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +54,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +55,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +56,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +57,executorMemory,0.0,16384.0,0.0,16384.0,16384.0,16384.0,16384.0,16384.0 +58,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +59,maxMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 +60,executorCores,0.0,16.0,0.0,16.0,16.0,16.0,16.0,16.0 +61,numExecutors,0.0,8.0,0.0,8.0,8.0,8.0,8.0,8.0 +62,maxOnHeapMem,0.0,8974132838.0,0.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0,8974132838.0 +63,sqlOp_WindowSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +64,sqlOp_SubqueryOutputBroadcast,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +65,sqlOp_Window,0.0,0.12341772151898735,0.3294377550897916,0.0,0.0,0.0,0.0,1.0 +66,sqlOp_WindowGroupLimit,0.0,0.0189873417721519,0.1366965815037675,0.0,0.0,0.0,0.0,1.0 +67,sqlOp_Filter,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +68,sqlOp_LocalTableScan,0.0,0.0031645569620253164,0.056254395046301184,0.0,0.0,0.0,0.0,1.0 +69,sqlOp_Project,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +70,sqlOp_Scan parquet,0.0,0.9968354430379747,0.056254395046301184,0.0,1.0,1.0,1.0,1.0 +71,sqlOp_HashAggregate,0.0,0.990506329113924,0.09712565886833821,0.0,1.0,1.0,1.0,1.0 +72,sqlOp_ReusedSort,0.0,0.00949367088607595,0.0971256588683382,0.0,0.0,0.0,0.0,1.0 +73,output_bytesWrittenRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +75,output_bytesWritten_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,platform_emr,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +79,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,platform_onprem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +83,sqlOp_BatchEvalPython,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_DeserializeToObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +93,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_Expand,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_MapElements,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_ObjectHashAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_SerializeFromObject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +114,sqlOp_TakeOrderedAndProject,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg index b442b4f78..359e92de8 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.cfg @@ -1 +1 @@ -{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0465075746","gamma":"0.0313147344","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0465075746","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"7","max_leaves":"0","min_child_weight":"2","min_split_loss":"0.0313147344","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.60073185"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"generic_param":{"device":"cpu","fail_on_invalid_gpu_id":"0","n_jobs":"0","nthread":"0","random_state":"0","seed":"0","seed_per_iteration":"0","validate_parameters":"1"},"gradient_booster":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"94"},"gbtree_train_param":{"process_type":"default","tree_method":"auto","updater":"grow_quantile_histmaker","updater_seq":"grow_quantile_histmaker"},"name":"gbtree","specified_updater":false,"tree_train_param":{"alpha":"0","cache_opt":"1","colsample_bylevel":"1","colsample_bynode":"1","colsample_bytree":"1","eta":"0.0463994034","gamma":"0.00824537966","grow_policy":"depthwise","interaction_constraints":"","lambda":"1","learning_rate":"0.0463994034","max_bin":"256","max_cat_threshold":"64","max_cat_to_onehot":"4","max_delta_step":"0","max_depth":"8","max_leaves":"0","min_child_weight":"5","min_split_loss":"0.00824537966","monotone_constraints":"()","refresh_leaf":"1","reg_alpha":"0","reg_lambda":"1","sampling_method":"uniform","sketch_ratio":"2","sparse_threshold":"0.20000000000000001","subsample":"0.628560543"},"updater":[{"hist_train_param":{"debug_synchronize":"0","max_cached_hist_node":"65536"},"name":"grow_quantile_histmaker"}]},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"learner_train_param":{"booster":"gbtree","disable_default_eval_metric":"0","multi_strategy":"one_output_per_tree","objective":"reg:squarederror"},"metrics":[{"name":"mae"},{"name":"mape"}],"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json index e6b3bd59b..be9cdced3 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.json @@ -1 +1 @@ -{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_BroadcastNestedLoopJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Expand","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_ObjectHashAggregate","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","sqlOp_SubqueryBroadcast","sqlOp_Window","sqlOp_WindowGroupLimit","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_onprem","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","sqlOp_BatchEvalPython","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_WindowSort"],"feature_types":["float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"99"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94,95,96,97,98,99],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[-2.141504E-2,-2.4401788E-1,6.385739E-1,-7.5088686E-1,-4.05462E-2,7.461596E-1,-9.173697E-1,-9.7974837E-1,-5.911197E-1,-2.2912894E-1,1.6641095E-1,1.7129138E0,6.6343397E-1,-5.0671108E-2,-1.6517477E-2,-4.6513196E-2,-6.6902013E-3,-7.730028E-1,-2.800583E-1,-5.89445E-1,-1.4499177E-1,1.9808824E-1,-3.7272045E-1,8.512532E-2,2.7620299E-2,4.7938082E-1,1.0106499E0,-6.9978404E-1,-1.156981E0,5.911774E-1,-3.819613E-1,-6.1238337E-1,-1.2756364E-2,-2.560834E-1,1.04140015E-2,1.5112062E-1,4.366603E-1,-2.215562E-1,-3.733392E-2,2.373614E-1,6.651033E-1,1.2739384E0,8.715073E-1,-3.3179495E-2,-8.800631E-3,-2.6174447E-2,-6.78686E-2,1.1399313E-2,3.4424398E-2,-3.5787642E-2,-2.7643758E-1,-6.2685335E-1,-9.789831E-3,-2.3064637E-1,-5.4077424E-2,-1.9452278E-2,3.502552E-1,3.1107217E-2,2.4083155E-1,2.078488E-1,5.793037E-1,-2.5412568E-3,-2.773189E-1,-9.829759E-2,3.180113E-1,7.617316E-1,1.6053186E-1,3.6074966E-2,6.6111915E-2,9.4691926E-1,4.97791E-1,-2.651559E-2,-9.470323E-3,-1.2764118E-2,-3.0111423E-2,-1.5103311E-2,-3.1225216E-3,-1.1433862E-2,9.395954E-4,5.657617E-3,1.9410757E-2,6.676445E-3,-9.470655E-3,1.7119164E-2,3.6582917E-3,3.8474163E-3,1.3118471E-2,3.782949E-3,2.9180704E-2,-1.517215E-2,-4.048843E-3,-8.798239E-3,3.1053848E-3,-1.2410105E-2,1.692653E-2,-4.2396667E-3,3.684276E-2,-6.6043045E-3,1.7085394E-2,2.0858342E-2,4.7733456E-2,2.6215725E-2,7.261577E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,51,-1,53,55,57,59,61,-1,63,65,67,69,-1,-1,-1,-1,-1,-1,-1,71,73,-1,75,-1,77,79,81,83,85,87,-1,89,91,93,95,97,-1,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3105746E2,6.875958E1,3.8348892E1,6.404686E0,1.8654745E1,1.5738411E1,1.0648584E0,1.1673889E0,6.2898445E0,7.4935284E0,3.9676685E0,1.0156708E0,1.2041115E1,0E0,0E0,0E0,0E0,1.2146034E0,4.1321936E0,1.7180634E-1,3.5245805E0,2.3543262E0,6.672106E-1,0E0,0E0,5.6854286E0,1.344284E0,3.0364227E-1,9.750414E-1,7.6417804E-2,1.4230676E0,1.1810303E-1,0E0,2.476345E0,8.7969536E-1,1.9520135E0,1.0219383E0,8.7230325E-2,0E0,1.6013701E0,3.4368973E0,4.4166565E-1,8.532181E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,5.8955836E-1,1.19924545E-1,0E0,1.7784114E0,0E0,7.231955E-1,5.662352E-2,2.1145725E0,2.117434E0,1.1596912E-1,4.523344E-1,0E0,4.604578E-2,1.9265166E-1,1.35994E0,1.656765E0,8.6565006E-1,0E0,0E0,1.044281E0,1.3712382E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,50,50,51,51,53,53,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,65,65,66,66,69,69,70,70],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,52,-1,54,56,58,60,62,-1,64,66,68,70,-1,-1,-1,-1,-1,-1,-1,72,74,-1,76,-1,78,80,82,84,86,88,-1,90,92,94,96,98,-1,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,3.9661028E7,9.9E1,9.31E2,3.6E1,6.165E3,1.1408248E0,6.76E2,3.89E2,9.932432E0,1.059448E7,2.8530578E6,-5.0671108E-2,-1.6517477E-2,-4.6513196E-2,-6.6902013E-3,5.035714E0,6.511E3,1.795688E2,1.5859042E6,2.8971518E6,3.7E1,8.512532E-2,2.7620299E-2,5.709E3,3.9914E4,8.407724E6,3.89E2,1.9E1,1E0,9.750042E6,-1.2756364E-2,6.1063123E0,7.4711205E6,1.273801E6,5.4007E7,1.7E1,-3.733392E-2,2.8204132E7,2.5345264E7,1.41565E3,2.577232E0,-3.3179495E-2,-8.800631E-3,-2.6174447E-2,-6.78686E-2,1.1399313E-2,3.4424398E-2,-3.5787642E-2,8.01E2,1.9428572E1,-9.789831E-3,1.559733E6,-5.4077424E-2,5.8899284E1,3.0914426E2,3.14403E5,2.165756E0,1E0,7.2E1,-2.5412568E-3,1.1111689E8,3.164034E0,3.167E3,1.1770261E8,2.9925186E-2,3.6074966E-2,6.6111915E-2,8.788122E4,2.880929E6,-2.651559E-2,-9.470323E-3,-1.2764118E-2,-3.0111423E-2,-1.5103311E-2,-3.1225216E-3,-1.1433862E-2,9.395954E-4,5.657617E-3,1.9410757E-2,6.676445E-3,-9.470655E-3,1.7119164E-2,3.6582917E-3,3.8474163E-3,1.3118471E-2,3.782949E-3,2.9180704E-2,-1.517215E-2,-4.048843E-3,-8.798239E-3,3.1053848E-3,-1.2410105E-2,1.692653E-2,-4.2396667E-3,3.684276E-2,-6.6043045E-3,1.7085394E-2,2.0858342E-2,4.7733456E-2,2.6215725E-2,7.261577E-3],"split_indices":[2,43,60,44,2,0,2,53,2,2,69,12,43,0,0,0,0,68,1,67,43,43,3,0,0,2,9,9,2,0,29,9,0,68,43,9,7,8,0,5,60,4,57,0,0,0,0,0,0,0,2,67,0,9,0,62,48,9,68,8,0,0,7,53,2,7,72,0,0,48,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.9E2,6.66E2,2.24E2,1.9E2,4.76E2,2.1E2,1.4E1,7.6E1,1.14E2,2.49E2,2.27E2,1.5E1,1.95E2,1E1,4E0,7.4E1,2E0,7.1E1,4.3E1,4.6E1,2.03E2,2.15E2,1.2E1,1.3E1,2E0,1.29E2,6.6E1,6.2E1,9E0,4E0,3.9E1,4.2E1,4E0,1.18E2,8.5E1,1.81E2,3.4E1,1E1,2E0,5.7E1,7.2E1,2E1,4.6E1,6E1,2E0,4E0,5E0,2E0,2E0,7E0,3.2E1,4E1,2E0,1.16E2,2E0,7.9E1,6E0,7.8E1,1.03E2,1.4E1,2E1,3E0,7E0,1.1E1,4.6E1,6E1,1.2E1,6E0,1.4E1,3.7E1,9E0,5E0,2.7E1,3E0,3.7E1,7.3E1,4.3E1,1.1E1,6.8E1,2E0,4E0,5.3E1,2.5E1,5.7E1,4.6E1,6E0,8E0,2E0,1.8E1,5E0,2E0,7E0,4E0,3E0,4.3E1,2E0,5.8E1,5E0,7E0,6E0,3.1E1,7E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-1.611827E-3,-2.3953192E-1,7.22545E-1,-6.626536E-1,-4.169042E-2,8.3661044E-1,-1.6181886E-1,-8.877335E-1,-4.9952382E-1,-2.3234195E-1,1.7748898E-1,4.419712E-1,1.1003993E0,-8.379092E-1,2.2277683E-1,-9.0764904E-1,-1.0356345E-2,4.503012E-1,-5.487178E-1,-2.873598E-1,1.4648603E-1,3.100961E-1,3.965176E-2,1.2906365E-1,5.824367E-1,1.6929948E0,9.7341573E-1,-4.6391554E-2,-1.62376E-2,3.5954645E-1,-9.521657E-3,-9.306508E-1,-2.020056E-2,3.356899E-2,3.0802367E-3,-5.7869023E-1,2.8635848E-1,-2.655344E-1,-5.5150542E-2,2.7840322E-1,-5.2430365E-2,3.527184E-1,-1.1719418E-1,-9.1599986E-2,1.546592E-1,2.4836392E-3,4.2938074E-1,6.9303674E-1,5.6924E-2,5.354018E-2,9.107334E-2,8.629708E-1,6.701195E-2,2.6544118E-1,2.8654182E-2,-4.422921E-2,-1.947546E-2,-6.337018E-1,-2.691969E-1,4.5458893E-3,1.765051E-2,-5.318593E-1,-1.9553351E-1,1.8294384E-1,2.1888843E-2,-2.9107246E-1,7.213371E-2,2.0433354E-1,5.470451E-1,2.8805528E-2,-2.400389E-2,-2.1067105E-2,-3.8573137E-1,1.17924055E-2,3.2354605E-1,-8.931692E-3,9.9572115E-2,2.3252754E-2,6.7463787E-3,7.9331386E-1,3.697495E-1,-1.4877425E-1,1.5511055E-2,6.527548E-1,9.54783E-1,4.013082E-3,1.3610107E-2,-3.0954596E-2,-1.439221E-2,-4.322182E-3,-1.6601264E-2,-2.5582435E-2,-9.6586505E-3,-5.941181E-3,-1.7805645E-2,1.172912E-2,3.1517183E-3,-2.0193545E-2,-2.3682446E-3,8.490535E-3,-3.4791147E-3,1.1198202E-2,-2.643743E-2,1.9629762E-2,3.274628E-2,-1.1975415E-2,9.993122E-3,-5.622463E-3,4.6404283E-3,-9.125238E-3,-3.546413E-2,2.566248E-3,-1.126456E-2,1.6515346E-2,-2.2018042E-3,7.657989E-4,8.386517E-3,2.2692768E-2,3.963523E-2,1.9123292E-3,1.980118E-2,-1.6772296E-2,3.7659646E-4,3.2927904E-2,1.14371255E-2,4.570696E-2,5.727252E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,-1,-1,53,-1,55,-1,-1,-1,57,59,61,-1,63,65,67,69,71,73,75,77,79,81,-1,-1,83,-1,85,-1,-1,-1,87,89,-1,-1,91,93,95,-1,97,99,101,103,105,-1,107,109,111,113,-1,115,-1,-1,117,119,121,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5954268E2,5.8318966E1,2.3235825E1,7.7077866E0,1.9931797E1,2.064624E1,7.207246E0,1.0164108E0,6.3681564E0,5.356881E0,4.0454636E0,3.5729046E0,7.457245E0,4.3179035E-1,1.1260786E0,5.6468964E-1,0E0,6.821071E-1,3.2943726E0,4.058077E0,8.805329E-1,2.0993042E0,1.6746087E0,1.0253154E0,3.273489E0,6.198082E-1,4.022751E0,0E0,0E0,1.744839E-1,0E0,4.7721863E-1,0E0,0E0,0E0,1.8895226E0,5.0760448E-2,3.978815E0,0E0,2.8093445E-1,4.371634E-1,2.8582401E0,6.555548E-1,1.0706621E0,1.4197556E0,3.9685845E-1,8.805096E-2,1.2303333E0,6.754598E-1,0E0,0E0,1.0172958E0,0E0,3.3226907E-2,0E0,0E0,0E0,8.483887E-1,2.7620578E-1,0E0,0E0,1.5142536E-1,2.173232E0,1.02555394E-1,0E0,1.4974964E-1,1.7592232E-1,1.8269048E0,5.6643295E-1,5.347614E-1,0E0,5.3079385E-1,5.6452525E-1,3.7703472E-1,3.3290243E-1,0E0,9.098637E-2,0E0,0E0,2.1389008E-1,2.2179186E-1,2.3556595E-1,0E0,4.7781754E-1,1.2063751E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,31,31,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,51,51,53,53,57,57,58,58,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,76,76,79,79,80,80,81,81,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,-1,-1,54,-1,56,-1,-1,-1,58,60,62,-1,64,66,68,70,72,74,76,78,80,82,-1,-1,84,-1,86,-1,-1,-1,88,90,-1,-1,92,94,96,-1,98,100,102,104,106,-1,108,110,112,114,-1,116,-1,-1,118,120,122,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.856934E7,1.16E2,9.31E2,1.0879832E3,1.3655363E2,1.3745962E0,6E0,3.3486558E6,3.6666667E0,5.486433E5,1.059448E7,1.665909E3,3.0070068E7,1.75E2,-1.0356345E-2,7.45E2,8.788122E4,1.1290322E1,3.5364E4,2.48334E5,7.026624E7,4.032683E6,1E0,4.626E3,6.663214E6,-4.6391554E-2,-1.62376E-2,4.435876E2,-9.521657E-3,9.374418E0,-2.020056E-2,3.356899E-2,3.0802367E-3,7.997723E6,2.126E3,3.89E2,-5.5150542E-2,3.3817584E7,2.76E2,4.2708197E2,1.20086E5,1.0948052E1,8.995735E5,1.5204346E8,2.697849E8,2E0,1.038946E6,5.354018E-2,9.107334E-2,1.3711089E6,6.701195E-2,2.5393645E5,2.8654182E-2,-4.422921E-2,-1.947546E-2,3.1363916E2,2.0131578E0,4.5458893E-3,1.765051E-2,1.917381E2,1.45E2,1E0,2.1888843E-2,2.1557376E0,5E0,2.483E3,1.0029973E8,2.0467092E6,-2.400389E-2,1.6E1,4.9E1,1E0,2E1,-8.931692E-3,2.3625128E6,2.3252754E-2,6.7463787E-3,4.7185095E2,2.68334E5,4.433E3,1.5511055E-2,4.909091E0,8.604875E3,4.013082E-3,1.3610107E-2,-3.0954596E-2,-1.439221E-2,-4.322182E-3,-1.6601264E-2,-2.5582435E-2,-9.6586505E-3,-5.941181E-3,-1.7805645E-2,1.172912E-2,3.1517183E-3,-2.0193545E-2,-2.3682446E-3,8.490535E-3,-3.4791147E-3,1.1198202E-2,-2.643743E-2,1.9629762E-2,3.274628E-2,-1.1975415E-2,9.993122E-3,-5.622463E-3,4.6404283E-3,-9.125238E-3,-3.546413E-2,2.566248E-3,-1.126456E-2,1.6515346E-2,-2.2018042E-3,7.657989E-4,8.386517E-3,2.2692768E-2,3.963523E-2,1.9123292E-3,1.980118E-2,-1.6772296E-2,3.7659646E-4,3.2927904E-2,1.14371255E-2,4.570696E-2,5.727252E-3],"split_indices":[2,43,60,44,2,67,71,53,3,43,69,43,12,4,58,6,0,2,48,69,9,1,7,60,6,2,43,0,0,71,0,68,0,0,0,9,2,2,0,7,10,67,9,69,43,7,5,32,9,0,0,43,0,43,0,0,0,67,68,0,0,67,0,28,0,69,8,2,5,43,0,3,3,19,3,0,60,0,0,4,1,10,0,68,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.24E2,6.96E2,2.28E2,2.21E2,4.75E2,2.02E2,2.6E1,9.1E1,1.3E2,2.54E2,2.21E2,8.2E1,1.2E2,9E0,1.7E1,8.8E1,3E0,6E0,1.24E2,2.22E2,3.2E1,1.12E2,1.09E2,2.6E1,5.6E1,1.9E1,1.01E2,6E0,3E0,1.3E1,4E0,8.3E1,5E0,3E0,3E0,1.2E2,4E0,2.18E2,4E0,1.9E1,1.3E1,1.02E2,1E1,5.1E1,5.8E1,1.9E1,7E0,4.6E1,1E1,8E0,1.1E1,8.4E1,1.7E1,1.1E1,2E0,7.9E1,4E0,1.01E2,1.9E1,2E0,2E0,4.4E1,1.74E2,1.4E1,5E0,4E0,9E0,5.9E1,4.3E1,8E0,2E0,4.2E1,9E0,3.2E1,2.6E1,6E0,1.3E1,5E0,2E0,3.4E1,1.2E1,6E0,4E0,2.8E1,5.6E1,2E0,9E0,9.1E1,1E1,7E0,1.2E1,4.1E1,3E0,1.29E2,4.5E1,8E0,6E0,2E0,2E0,5E0,4E0,5.7E1,2E0,2.6E1,1.7E1,3E0,5E0,2.3E1,1.9E1,7E0,2E0,2.8E1,4E0,2.4E1,2E0,7E0,6E0,7E0,2.7E1,2E0,1E1,2E0,4E0,2.4E1,4E0,5.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[3.3552062E-3,-2.075869E-1,6.398197E-1,-6.4527136E-1,-1.1015471E-2,7.4770886E-1,-2.0303583E-1,-9.324796E-1,-4.9820742E-1,-1.7566967E-1,1.9427913E-1,4.8582E-1,1.064096E0,-6.38538E-1,1.3628054E-1,-8.3671E-1,-1.3416021E0,-7.250521E-1,-3.8548106E-1,-2.388268E-1,9.774057E-2,3.4113184E-1,2.5147056E-2,1.4997473E-1,6.0397935E-1,8.5167795E-2,9.48206E-1,-3.9271306E-2,-1.3577183E-2,3.2579657E-1,-1.0877144E-2,-8.6132497E-1,-3.4281686E-3,-2.527628E-2,-6.729071E-2,-3.508829E-2,-1.5212619E-2,7.447635E-1,-4.9302745E-1,-4.393172E-1,-1.4888151E-1,2.565574E-1,2.4440292E-2,3.747631E-1,-2.2031017E-2,5.9731513E-2,-3.02681E-2,-3.5075098E-3,5.916746E-1,6.9448197E-1,1.4136843E-1,5.7110095E-1,1.1229466E0,6.4661023E-3,1.9191654E-2,-4.439266E-2,-7.240485E-1,1.24138575E-2,4.367981E-2,-7.5000924E-1,-3.6072937E-1,-3.734432E-1,-6.4341925E-2,-3.4261572E-1,-5.0554775E-2,1.5984774E-2,9.541326E-2,-1.6282335E-2,5.6769196E-2,2.362881E-1,5.273349E-1,3.2130632E-1,2.2538153E-2,1.1724154E-2,-1.2358298E-1,3.5435654E-2,1.07401395E-2,7.622056E-1,3.0363384E-1,-1.1647476E-1,3.8154057E-1,6.1652493E-1,6.7168996E-3,1.1705518E0,1.0387115E-2,-2.4904063E-2,-3.9613858E-2,-1.3771617E-3,-3.8552765E-2,-5.436678E-3,-2.0939821E-2,-1.037255E-2,-2.4039181E-2,-2.1953177E-2,-8.796393E-3,9.050108E-4,-8.610317E-3,-3.5136105E-3,8.320581E-3,7.8069675E-3,-1.3668897E-3,4.678921E-3,1.6010346E-2,2.9426494E-3,2.6758462E-2,1.6546417E-2,4.6179853E-3,2.799692E-3,-1.1148348E-2,-1.3105156E-2,-2.4755755E-3,2.2836022E-2,3.697897E-2,7.3874807E-3,1.962382E-2,1.2887891E-3,-1.2444904E-2,2.2595782E-2,2.127173E-3,3.027197E-2,6.857244E-3,5.563471E-2,1.6373217E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,-1,53,-1,55,-1,-1,-1,-1,-1,57,59,61,63,65,67,69,-1,71,-1,73,75,77,79,81,83,-1,-1,-1,85,-1,-1,87,89,91,-1,93,95,-1,97,-1,99,101,103,105,107,-1,109,-1,-1,111,113,115,117,119,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.24589195E2,5.996016E1,2.1157791E1,8.658836E0,1.6326992E1,1.6468994E1,4.076892E0,1.6985359E0,3.42836E0,4.6511135E0,5.331484E0,4.4333076E0,6.4430695E0,6.09745E-1,1.1986158E0,1.082161E0,2.218914E-1,3.0332565E-1,1.2401004E1,3.8658247E0,5.874369E-1,3.3289995E0,2.4000509E0,2.103661E0,3.4186916E0,0E0,4.7621155E0,0E0,0E0,1.2107158E-1,0E0,6.373978E-2,0E0,0E0,0E0,0E0,0E0,5.854883E-1,2.823286E0,3.6504326E0,2.8781657E0,1.9189322E-1,4.5578098E-1,2.2205372E0,0E0,9.364563E-1,0E0,7.677417E-1,3.1539845E-1,1.6214447E0,9.733411E-1,4.536333E-1,2.093979E0,0E0,0E0,0E0,2.182455E-1,0E0,0E0,1.6816597E0,1.3006248E0,1.265584E0,0E0,9.2372084E-1,9.6472144E-1,0E0,1.1343774E-1,0E0,3.3038005E-1,8.442986E-1,1.1224289E0,5.6512594E-2,8.6380094E-1,0E0,1.7421222E-1,0E0,0E0,1.3617325E-1,1.3854456E-1,1.8172362E-1,2.580012E-1,3.0719376E-1,0E0,6.7264557E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,31,31,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,47,47,48,48,49,49,50,50,51,51,52,52,56,56,59,59,60,60,61,61,63,63,64,64,66,66,68,68,69,69,70,70,71,71,72,72,74,74,77,77,78,78,79,79,80,80,81,81,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,-1,54,-1,56,-1,-1,-1,-1,-1,58,60,62,64,66,68,70,-1,72,-1,74,76,78,80,82,84,-1,-1,-1,86,-1,-1,88,90,92,-1,94,96,-1,98,-1,100,102,104,106,108,-1,110,-1,-1,112,114,116,118,120,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.7909248E5,3.2856784E7,3.434405E7,9.99E2,7.459E3,1.3655363E2,1E0,2.95E2,3.3486558E6,3.875E0,5.486433E5,1.059448E7,1.4916515E3,4.4910244E3,2.9073685E2,1.4E1,1.5933333E2,7.336111E4,1E0,5.894636E6,1.4676277E7,1.2784314E1,9.6215985E2,2E0,8.5167795E-2,1.5484655E6,-3.9271306E-2,-1.3577183E-2,1.4595416E3,-1.0877144E-2,4.2E1,-3.4281686E-3,-2.527628E-2,-6.729071E-2,-3.508829E-2,-1.5212619E-2,2.123E3,1.9902479E4,6.3344407E0,5.51E2,2.1111E4,5.8899284E1,5.8036E4,-2.2031017E-2,2.9276064E3,-3.02681E-2,7.6937294E-1,7.149866E8,5.5E1,5.071E3,4.314233E0,1.9513452E2,6.4661023E-3,1.9191654E-2,-4.439266E-2,1E0,1.24138575E-2,4.367981E-2,1E0,2.347826E0,2.0131578E0,-6.4341925E-2,1.3871127E2,3.7509E4,1.5984774E-2,3.57E2,-1.6282335E-2,3.5364E4,2.118835E6,6.374563E5,5.869281E0,2.1129E4,1.1724154E-2,8.295515E7,3.5435654E-2,1.07401395E-2,4.4149057E2,5.8165E4,1.6E1,3.5510652E6,1.2426182E9,6.7168996E-3,1E0,1.0387115E-2,-2.4904063E-2,-3.9613858E-2,-1.3771617E-3,-3.8552765E-2,-5.436678E-3,-2.0939821E-2,-1.037255E-2,-2.4039181E-2,-2.1953177E-2,-8.796393E-3,9.050108E-4,-8.610317E-3,-3.5136105E-3,8.320581E-3,7.8069675E-3,-1.3668897E-3,4.678921E-3,1.6010346E-2,2.9426494E-3,2.6758462E-2,1.6546417E-2,4.6179853E-3,2.799692E-3,-1.1148348E-2,-1.3105156E-2,-2.4755755E-3,2.2836022E-2,3.697897E-2,7.3874807E-3,1.962382E-2,1.2887891E-3,-1.2444904E-2,2.2595782E-2,2.127173E-3,3.027197E-2,6.857244E-3,5.563471E-2,1.6373217E-2],"split_indices":[2,43,60,7,2,2,71,6,2,43,69,43,12,4,4,67,3,4,60,29,60,62,68,67,6,0,43,0,0,4,0,2,0,0,0,0,0,1,43,68,2,9,62,1,0,62,0,53,46,8,2,68,73,0,0,0,8,0,0,13,68,68,0,67,1,0,0,0,9,9,43,68,44,0,7,0,0,4,10,3,43,47,0,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,6.96E2,2.3E2,2.15E2,4.81E2,2.04E2,2.6E1,7.1E1,1.44E2,2.67E2,2.14E2,1.13E2,9.1E1,1.1E1,1.5E1,6E1,1.1E1,4.6E1,9.8E1,2.17E2,5E1,1.14E2,1E2,3E1,8.3E1,1E1,8.1E1,6E0,5E0,1E1,5E0,5.8E1,2E0,2E0,9E0,4.2E1,4E0,8E0,9E1,6.6E1,1.51E2,1.5E1,3.5E1,1.1E2,4E0,9.6E1,4E0,2.3E1,7E0,6.9E1,1.4E1,2.7E1,5.4E1,4E0,6E0,3.1E1,2.7E1,3E0,5E0,2.9E1,6.1E1,6.3E1,3E0,5E1,1.01E2,9E0,6E0,2E0,3.3E1,5.9E1,5.1E1,1.1E1,8.5E1,7E0,1.6E1,4E0,3E0,5.8E1,1.1E1,7E0,7E0,2.4E1,3E0,5.1E1,3E0,1.3E1,1.4E1,3E0,2.6E1,1.7E1,4.4E1,3.2E1,3.1E1,2.6E1,2.4E1,6.7E1,3.4E1,2E0,4E0,1.4E1,1.9E1,2.7E1,3.2E1,5E0,4.6E1,9E0,2E0,7.5E1,1E1,4E0,1.2E1,8E0,5E1,6E0,5E0,4E0,3E0,5E0,2E0,2.2E1,2E0,4.9E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[-2.036363E-2,-2.0174159E-1,6.1455715E-1,-6.0745305E-1,-2.4844015E-2,7.190911E-1,-6.022125E-1,-8.97652E-1,-4.4604352E-1,-2.189464E-1,1.16046675E-1,1.5522872E0,6.4483815E-1,-7.4827164E-1,2.7694146E-3,-9.1273916E-1,-1.0056132E-2,-4.929242E-1,7.1436644E-1,-1.6568632E-1,-6.3368493E-1,2.2059855E-1,-6.1416216E-3,7.800047E-2,2.2963515E-2,4.224061E-1,8.77877E-1,-3.9974548E-2,-1.7491737E-2,-3.5358787E-2,-1.0120617E0,-6.363004E-1,-2.8773114E-1,4.283307E-2,9.336138E-3,-2.979832E-1,-7.172933E-2,-6.986436E-1,-6.2427404E-3,9.864545E-2,3.6658847E-1,4.024809E-2,-2.6430613E-1,1.8176298E-1,5.5173045E-1,9.285158E-1,1.7373635E-1,-4.8010197E-2,-1.8509628E-2,-5.177937E-2,-6.740744E-1,5.8741766E-1,-3.7964153E-1,-3.928663E-1,-1.4634657E-1,2.7594395E-2,-1.5992521E-1,-2.6149426E-2,-4.3990973E-2,1.5319821E-1,-1.7705788E-1,4.4492343E-1,-1.635806E-1,-8.768022E-3,1.8928899E-1,-1.3129723E-1,-3.0865923E-2,-1.0472789E-1,2.7394202E-1,6.9186604E-1,4.0749103E-1,9.574594E-1,-6.209806E-3,1.27315195E-2,-2.3657358E-3,6.0413363E-3,-8.1432015E-3,-3.3031445E-2,-1.2924637E-2,1.1081939E-2,3.2667603E-2,-2.3104833E-2,-8.680713E-3,-5.3405883E-3,-1.9014625E-2,-1.328231E-2,-1.0313626E-3,-9.639116E-4,1.015572E-2,7.586918E-3,-9.247638E-3,5.681781E-3,2.590325E-2,-3.832579E-2,-6.5697037E-4,2.3006627E-2,-1.1088832E-3,-2.5416858E-2,2.1650887E-5,-4.8719165E-3,2.8623166E-3,3.7752076E-3,1.470148E-2,3.8035943E-3,-9.448103E-3,-2.1933475E-4,-1.0629938E-2,2.382333E-2,8.224326E-3,3.185872E-3,3.539961E-2,1.4290087E-2,2.927593E-2,3.5696562E-2,5.2887157E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,-1,31,33,35,37,39,41,-1,-1,43,45,-1,-1,-1,47,49,51,-1,-1,53,55,57,-1,59,61,63,65,67,69,71,73,-1,-1,75,77,79,81,83,85,87,89,-1,-1,91,93,95,97,99,101,103,-1,105,107,109,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0779918E2,5.2233013E1,2.6889359E1,9.951447E0,1.3917794E1,1.0877548E1,1.6877112E0,6.045418E-1,8.185846E0,4.622053E0,3.7699847E0,1.1545525E0,8.855858E0,2.5640774E-1,0E0,3.813858E-1,0E0,3.906269E0,4.518783E-1,2.3527975E0,6.8487453E-1,2.794662E0,1.651374E0,0E0,0E0,2.793972E0,2.9051743E0,0E0,0E0,0E0,1.23786926E-1,1.7535515E0,4.9686894E0,0E0,0E0,1.0791378E0,9.942866E-1,1.7627716E-1,0E0,1.350678E0,3.062995E0,8.5639215E-1,1.0283738E0,9.265766E-1,9.308548E-1,2.586975E0,1.7117189E-1,0E0,0E0,1.5716723E-1,8.9147186E-1,7.3524E-2,1.1282039E0,1.6636324E-1,5.469615E-1,5.0230813E-1,7.867718E-1,0E0,0E0,8.5907936E-1,1.5696567E0,1.4813862E0,6.284354E-1,6.0712373E-1,3.7196422E-1,2.769002E-1,0E0,1.10387325E-1,5.1106036E-1,1.2004862E0,5.185852E-1,1.7893524E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,30,30,31,31,32,32,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,-1,32,34,36,38,40,42,-1,-1,44,46,-1,-1,-1,48,50,52,-1,-1,54,56,58,-1,60,62,64,66,68,70,72,74,-1,-1,76,78,80,82,84,86,88,90,-1,-1,92,94,96,98,100,102,104,-1,106,108,110,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.8988005E5,4.217427E7,8.8E1,7.9E2,3.6E1,3.82E3,1.7971E4,6.6890576E2,6.2233735E-6,4.066351E0,1.059448E7,2.36433E6,1.7026364E3,2.7694146E-3,6.48334E5,-1.0056132E-2,6.76E2,2.3E1,6.978874E5,1.3E1,4.0446457E2,9.639872E0,7.800047E-2,2.2963515E-2,8.507324E2,1.0424884E-7,-3.9974548E-2,-1.7491737E-2,-3.5358787E-2,8.9044176E-2,1.0204082E0,6.511E3,4.283307E-2,9.336138E-3,1.339646E6,1E0,5.5E1,-6.2427404E-3,8.315271E0,1.0767881E7,1.4603988E6,4E1,6.516E3,1.1E2,5.5766084E3,2.82E2,-4.8010197E-2,-1.8509628E-2,9.1E1,7.997723E6,1.9E1,5.13657E6,1.5043378E0,5.75E2,4.82425E6,9.59799E-1,-2.6149426E-2,-4.3990973E-2,7.168071E6,1.0130841E1,2E0,1E0,4.220534E5,1.2585366E1,3.588785E0,-3.0865923E-2,1.6E1,3.3412E5,1.2593857E0,1.000501E6,7.459E3,-6.209806E-3,1.27315195E-2,-2.3657358E-3,6.0413363E-3,-8.1432015E-3,-3.3031445E-2,-1.2924637E-2,1.1081939E-2,3.2667603E-2,-2.3104833E-2,-8.680713E-3,-5.3405883E-3,-1.9014625E-2,-1.328231E-2,-1.0313626E-3,-9.639116E-4,1.015572E-2,7.586918E-3,-9.247638E-3,5.681781E-3,2.590325E-2,-3.832579E-2,-6.5697037E-4,2.3006627E-2,-1.1088832E-3,-2.5416858E-2,2.1650887E-5,-4.8719165E-3,2.8623166E-3,3.7752076E-3,1.470148E-2,3.8035943E-3,-9.448103E-3,-2.1933475E-4,-1.0629938E-2,2.382333E-2,8.224326E-3,3.185872E-3,3.539961E-2,1.4290087E-2,2.927593E-2,3.5696562E-2,5.2887157E-2],"split_indices":[2,43,60,44,2,0,0,6,67,52,69,12,43,4,0,46,0,2,0,43,8,67,69,0,0,67,52,0,0,0,57,68,1,0,0,9,8,0,0,71,62,43,3,44,8,4,0,0,0,0,9,0,9,69,2,43,68,0,0,43,71,6,6,66,71,71,0,3,1,68,43,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.34E2,7.27E2,2.07E2,2.2E2,5.07E2,1.91E2,1.6E1,7.7E1,1.43E2,2.13E2,2.94E2,1.4E1,1.77E2,1.3E1,3E0,7.5E1,2E0,1.38E2,5E0,1.9E2,2.3E1,1.58E2,1.36E2,1.2E1,2E0,9.2E1,8.5E1,9E0,4E0,3.3E1,4.2E1,8E1,5.8E1,3E0,2E0,7.8E1,1.12E2,2E1,3E0,8.7E1,7.1E1,1.16E2,2E1,3.3E1,5.9E1,7.9E1,6E0,4E1,2E0,5E0,7.5E1,5E0,5.3E1,4.7E1,3.1E1,5.3E1,5.9E1,1.5E1,5E0,7.3E1,1.4E1,6.2E1,9E0,8.8E1,2.8E1,1.6E1,4E0,8E0,2.5E1,2.8E1,3.1E1,7.7E1,2E0,4E0,2E0,2E0,3E0,6.8E1,7E0,2E0,3E0,3.2E1,2.1E1,3E0,4.4E1,1.4E1,1.7E1,4.3E1,1E1,6E0,5.3E1,6.9E1,4E0,2E0,1.2E1,5.6E1,6E0,2E0,7E0,3.7E1,5.1E1,1.6E1,1.2E1,4E0,1.2E1,5E0,3E0,6E0,1.9E1,3E0,2.5E1,2.3E1,8E0,4E1,3.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[4.563436E-5,-1.8859692E-1,5.5521715E-1,-5.639306E-1,-2.5736935E-2,6.156289E-1,-2.8371543E-1,-7.796237E-1,-4.3681353E-1,-2.0706545E-1,1.2464715E-1,2.1988381E-1,7.997033E-1,-5.689261E-1,1.6425923E-1,-8.137509E-1,-1.45917E-1,-4.7796875E-1,4.933893E-1,-4.4991922E-1,-1.2981017E-1,2.7556049E-2,3.315177E-1,1.165713E-1,5.840375E-1,1.3598945E0,7.246051E-1,-9.321788E-3,-3.247237E-2,2.0857887E-2,-7.489116E-3,-8.593116E-1,-4.5445603E-1,-1.0630586E-2,-6.798237E-4,-2.0494527E-1,-5.6970865E-1,3.437195E-2,8.640546E-3,-4.7249004E-1,-3.096926E-3,-5.164867E-1,-9.566047E-2,2.0714366E-1,-7.193146E-2,2.6677793E-1,6.269124E-1,-5.573953E-2,2.3313427E-1,1.8149441E-2,3.4869645E-2,8.0595505E-1,7.154802E-2,4.9018127E-1,8.8471967E-1,-1.92504E-2,-4.0704012E-2,-2.3850417E-2,-6.8510454E-3,-2.1491135E-2,-7.3047705E-2,-6.175102E-1,-1.9558875E-1,-4.9388474E-1,-5.286837E-3,-4.2175777E-2,-2.422262E-1,-1.3822427E-1,1.7441185E-1,7.044831E-2,2.8398228E-1,-2.8380638E-1,-3.6962464E-2,1.03450984E-1,3.693107E-1,6.616195E-3,3.236149E-2,-1.3644096E-1,6.2816426E-2,3.316774E-1,-1.802656E-1,4.6786577E-2,1.2583926E-2,5.493078E-1,2.7778864E-1,9.1387624E-1,-6.0172714E-3,1.7343229E-2,-7.375508E-3,-2.5377696E-2,-3.9188877E-2,-2.456256E-2,-1.211784E-3,-1.6384158E-2,-2.5414834E-2,-1.5163579E-2,-2.6016755E-3,-4.8471396E-3,-1.448085E-2,4.2469194E-3,1.5233874E-2,-1.1523315E-3,1.2277844E-2,9.771256E-3,2.2538414E-2,-2.4550186E-2,-5.8077476E-3,-2.4339627E-3,1.5868995E-2,7.364895E-3,-8.408475E-3,-5.5343644E-3,1.844961E-2,-2.294137E-3,-1.1365051E-2,7.4644396E-3,8.5263496E-4,5.591845E-3,1.8587703E-2,-4.864147E-5,-1.4622862E-2,2.7584225E-2,4.954482E-3,3.356798E-4,1.4988569E-2,6.930898E-3,4.3393996E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,-1,-1,55,57,-1,-1,59,61,-1,-1,63,-1,65,67,69,71,73,75,77,79,-1,-1,81,-1,83,85,-1,-1,-1,-1,-1,87,89,91,93,-1,-1,95,97,99,101,103,105,107,109,111,-1,-1,113,115,117,119,-1,-1,121,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.415134E1,4.1001595E1,1.1730019E1,5.225357E0,1.2814934E1,1.5340469E1,2.1377258E0,1.481636E0,5.212578E0,3.9378452E0,5.1527843E0,2.5278103E0,5.11467E0,3.364756E-1,7.194054E-1,7.5535965E-1,5.0925203E-2,3.0141315E0,3.1610572E-1,4.0979195E-1,2.10299E0,3.1594434E0,1.4009037E0,1.1176732E0,1.8914747E-1,3.4235E-1,4.4458313E0,0E0,0E0,0E0,0E0,6.974411E-2,1.0374403E-1,0E0,0E0,1.0855399E0,1.5423889E0,0E0,0E0,3.1233883E-1,0E0,1.1721609E0,1.7598993E0,6.4171314E-1,8.341418E-1,1.1172938E0,3.6864328E-1,2.2862636E-1,1.4041349E0,0E0,0E0,3.7039256E-1,0E0,5.3810215E-1,2.365387E0,0E0,0E0,0E0,0E0,0E0,9.3652433E-1,8.6642075E-1,6.6278255E-1,1.4252377E-1,0E0,0E0,1.2228924E-1,7.387576E-1,2.373262E-1,4.4887927E-1,4.9111652E-1,5.5471337E-1,5.933864E-1,4.6564275E-1,6.0895824E-1,0E0,0E0,1.19573444E-1,4.0510025E-2,3.4011388E-1,1.6797161E-1,0E0,0E0,7.514353E-1,1.6621554E-1,9.4628906E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,35,35,36,36,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,51,51,53,53,54,54,60,60,61,61,62,62,63,63,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,77,77,78,78,79,79,80,80,83,83,84,84,85,85],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,-1,-1,56,58,-1,-1,60,62,-1,-1,64,-1,66,68,70,72,74,76,78,80,-1,-1,82,-1,84,86,-1,-1,-1,-1,-1,88,90,92,94,-1,-1,96,98,100,102,104,106,108,110,112,-1,-1,114,116,118,120,-1,-1,122,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,3.9661028E7,8.8E1,8.1E2,8.716064E2,1.037575E6,1.1408248E0,6.6890576E2,3.95E2,2.0766992E6,3.1993368E6,1.059448E7,9.7172376E7,5.3142116E7,1.0795462E-6,6.5E1,1.88E0,2.3E1,1.1643481E6,2.2953334E2,2.1770449E0,6.0052995E6,2.4361508E6,5.941442E6,1.266059E3,1.8062708E6,-9.321788E-3,-3.247237E-2,2.0857887E-2,-7.489116E-3,1.798E3,5.5E1,-1.0630586E-2,-6.798237E-4,3.06E2,3.518913E2,3.437195E-2,8.640546E-3,1.9685597E2,-3.096926E-3,5.862757E1,1.0254199E10,1.453785E6,5.1066592E2,3.9157894E0,3.3852024E7,5.357143E2,6.1095314E-8,1.8149441E-2,3.4869645E-2,1.6E1,7.154802E-2,2.1609572E7,6.5026445E3,-1.92504E-2,-4.0704012E-2,-2.3850417E-2,-6.8510454E-3,-2.1491135E-2,2E0,7.382199E0,2.1525E4,1.4452E4,-5.286837E-3,-4.2175777E-2,6.3157897E0,3.6507E4,2.0886075E0,1.1942981E0,1.9539816E7,2.06E2,1.0784811E3,4.5620965E6,9.99E2,6.616195E-3,3.236149E-2,1.7E1,2.9487667E0,4.1032645E6,1.1E1,4.6786577E-2,1.2583926E-2,4.909091E0,6.64E2,3.596E3,-6.0172714E-3,1.7343229E-2,-7.375508E-3,-2.5377696E-2,-3.9188877E-2,-2.456256E-2,-1.211784E-3,-1.6384158E-2,-2.5414834E-2,-1.5163579E-2,-2.6016755E-3,-4.8471396E-3,-1.448085E-2,4.2469194E-3,1.5233874E-2,-1.1523315E-3,1.2277844E-2,9.771256E-3,2.2538414E-2,-2.4550186E-2,-5.8077476E-3,-2.4339627E-3,1.5868995E-2,7.364895E-3,-8.408475E-3,-5.5343644E-3,1.844961E-2,-2.294137E-3,-1.1365051E-2,7.4644396E-3,8.5263496E-4,5.591845E-3,1.8587703E-2,-4.864147E-5,-1.4622862E-2,2.7584225E-2,4.954482E-3,3.356798E-4,1.4988569E-2,6.930898E-3,4.3393996E-2],"split_indices":[2,43,60,44,2,67,44,53,67,2,43,43,12,60,1,52,0,68,0,43,48,68,43,60,60,67,43,0,0,0,0,9,0,0,0,2,67,0,0,67,0,62,46,9,48,71,7,67,52,0,0,0,0,60,4,0,0,0,0,0,10,69,9,9,0,0,71,1,68,57,12,0,4,62,2,0,0,3,68,60,8,0,0,68,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.97E2,6.7E2,2.27E2,2.02E2,4.68E2,2.12E2,1.5E1,7.3E1,1.29E2,2.12E2,2.56E2,6.8E1,1.44E2,9E0,6E0,6.9E1,4E0,1.24E2,5E0,5E1,1.62E2,1.75E2,8.1E1,5.4E1,1.4E1,1.5E1,1.29E2,3E0,6E0,3E0,3E0,6E1,9E0,2E0,2E0,3.2E1,9.2E1,2E0,3E0,4.7E1,3E0,1.2E1,1.5E2,6.2E1,1.13E2,6.8E1,1.3E1,2.2E1,3.2E1,8E0,6E0,5E0,1E1,5.4E1,7.5E1,3E0,5.7E1,7E0,2E0,1E1,2.2E1,8.1E1,1.1E1,4.4E1,3E0,4E0,8E0,1.3E2,2E1,2.3E1,3.9E1,1.5E1,9.8E1,2.7E1,4.1E1,2E0,1.1E1,1.3E1,9E0,2.6E1,6E0,3E0,2E0,4.1E1,1.3E1,7.3E1,2E0,3E0,1.9E1,6.4E1,1.7E1,3E0,8E0,1.4E1,3E1,5E0,3E0,1.1E2,2E1,1.4E1,6E0,1.6E1,7E0,3E1,9E0,5E0,1E1,9.5E1,3E0,2.3E1,4E0,2E0,3.9E1,8E0,5E0,2E0,7E0,7E0,1.9E1,3E0,3E0,3.7E1,4E0,2E0,1.1E1,2E0,7.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[3.156146E-2,-1.9168489E-1,5.6971633E-1,-5.4174197E-1,-1.661807E-2,-6.031586E-1,6.124712E-1,-8.100824E-1,-4.4337457E-1,-1.7292286E-1,1.47103E-1,-3.4800634E-2,-9.227503E-3,3.0365032E-1,7.759951E-1,-1.1598461E0,-6.8038297E-1,8.502482E-2,-4.7522077E-1,-4.533534E-1,-1.07258E-1,3.0458748E-1,2.7138442E-2,1.0924832E-1,6.0301876E-1,9.39834E-1,4.82811E-1,-2.3245167E-2,-1.3738773E0,-7.2112024E-1,5.4289917E-3,-2.4424763E-2,4.2156586E-1,-6.350902E-1,-3.6405495E-1,-2.9064655E-1,-5.4230195E-1,-4.799275E-1,-7.5751305E-2,2.3873141E-1,5.620663E-1,5.5647183E-2,-3.0280879E-2,-2.488828E-1,1.6436432E-1,-9.1639906E-4,6.6022706E-1,8.1594306E-1,1.4772208E0,5.555274E-1,-8.655504E-2,-7.5371645E-2,-3.7345424E-2,-7.5006604E-1,-1.2027738E-2,3.0923191E-2,3.3873182E-3,-5.202332E-1,-7.604569E-1,-4.5708355E-1,-1.753425E-1,-1.6301267E-2,-3.5754119E-3,-8.990756E-3,-5.7941645E-1,-1.3790017E-1,-4.2460512E-2,-1.4488189E-1,6.63315E-2,1.468442E-1,3.7941995E-1,1.2974885E-2,6.494836E-1,8.12333E-2,-2.6325497E-1,-4.9477904E-3,-1.6965132E-2,2.0068136E-1,-1.4112806E-2,1.3002205E-2,6.8897355E-1,8.6616606E-1,4.1731286E-1,7.170301E-2,2.8795324E-2,4.421821E-1,8.6397886E-1,-1.2380979E-2,1.4027386E-2,-3.601507E-2,-1.5980927E-2,-2.8584016E-2,-1.2019733E-2,-1.196238E-2,-3.779415E-2,-1.6750959E-3,-2.2241866E-2,-9.5762815E-3,8.778771E-3,-2.8383927E-2,-7.908131E-3,-1.1856781E-2,3.126565E-4,-1.1546033E-2,-2.371445E-3,5.358924E-4,1.0693713E-2,9.108089E-3,-1.4562778E-3,2.1096138E-2,3.7885704E-3,8.725644E-3,3.3318173E-2,1.7731842E-2,2.8024337E-3,-1.5898004E-2,3.652892E-4,4.3790983E-3,1.640351E-2,3.3061918E-2,1.1846975E-2,4.53835E-2,3.179863E-2,2.202814E-2,4.2056986E-3,2.8018847E-2,1.3496256E-2,5.5854067E-2,2.832715E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,-1,-1,55,57,59,61,63,65,67,69,71,73,-1,75,77,-1,79,81,83,85,87,-1,-1,89,-1,-1,-1,91,93,95,97,-1,-1,-1,99,101,-1,103,105,107,109,-1,111,113,115,-1,-1,117,-1,-1,119,121,123,-1,-1,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1063902E2,3.9886475E1,1.3908882E1,5.3687553E0,1.1157256E1,4.3890905E-1,1.2944763E1,1.7966194E0,2.7496338E0,4.0577927E0,4.0158486E0,0E0,0E0,5.2883215E0,7.7121735E0,1.2910557E0,1.5129623E0,2.2749808E0,2.4701385E0,4.502287E-1,2.1006136E0,1.4258194E0,2.395175E0,1.1660168E0,1.2955055E0,5.9850235E0,2.6564255E0,0E0,1.0731125E-1,3.6829185E-1,0E0,0E0,5.4559755E-1,4.827385E-1,1.5349741E0,1.9059312E-1,2.2474957E-1,1.9477239E0,1.6695732E0,9.1384506E-1,2.6423693E-1,9.842021E-1,0E0,9.3313545E-2,9.1038156E-1,0E0,1.6848469E-1,1.4403801E0,9.744263E-2,1.5758076E0,6.38201E-1,0E0,0E0,1.194973E-1,0E0,0E0,0E0,7.0588493E-1,5.000887E-1,5.2622604E-1,3.949945E-1,0E0,0E0,0E0,1.870842E-1,1.5405755E-1,0E0,1.0938754E0,4.9949723E-1,4.164009E-1,6.0407925E-1,0E0,2.6739216E-1,6.69304E-1,1.944235E-1,0E0,0E0,7.2791076E-1,0E0,0E0,1.04465485E-1,9.1332245E-1,1.7814112E-1,0E0,0E0,8.726435E-1,4.7104645E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,48,48,49,49,50,50,53,53,57,57,58,58,59,59,60,60,64,64,65,65,67,67,68,68,69,69,70,70,72,72,73,73,74,74,77,77,80,80,81,81,82,82,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,-1,-1,56,58,60,62,64,66,68,70,72,74,-1,76,78,-1,80,82,84,86,88,-1,-1,90,-1,-1,-1,92,94,96,98,-1,-1,-1,100,102,-1,104,106,108,110,-1,112,114,116,-1,-1,118,-1,-1,120,122,124,-1,-1,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,1.972052E5,1E0,3.3817584E7,9.31E2,2.3017536E5,8.507324E2,2E0,6E0,3.79E2,3.1842105E0,-3.4800634E-2,-9.227503E-3,2.2318378E6,5.489183E1,8.317E3,8.57E2,4.54E2,4.7721977E4,1.09333336E2,5.862757E1,4.5866325E6,4.1E1,1.0144968E8,1.2772413E0,1.7379E4,3.0070068E7,-2.3245167E-2,3.6255838E2,7.7759676E0,5.4289917E-3,-2.4424763E-2,2.00087E5,1.5E1,7.9E2,3E0,2.2116136E5,5E0,2.8045622E6,4.3764517E2,5.5177975E6,2.1129E4,-3.0280879E-2,3.2710236E2,8.3227225E6,-9.1639906E-4,7.156528E7,2E0,2.8E1,3.8537518E2,1.2090626E0,-7.5371645E-2,-3.7345424E-2,1.2707317E1,-1.2027738E-2,3.0923191E-2,3.3873182E-3,1.36E2,6.48334E5,1.6E1,6.6890576E2,-1.6301267E-2,-3.5754119E-3,-8.990756E-3,2.0549193E2,6.99E2,-4.2460512E-2,2.6676828E2,2.7889166E2,1.0292E4,1E0,1.2974885E-2,4.2212E4,1.8461539E0,1.839E3,-4.9477904E-3,-1.6965132E-2,4.032683E6,-1.4112806E-2,1.3002205E-2,2.6506329E1,2.1089442E0,2.2E1,7.170301E-2,2.8795324E-2,7.886154E1,4.407599E-1,-1.2380979E-2,1.4027386E-2,-3.601507E-2,-1.5980927E-2,-2.8584016E-2,-1.2019733E-2,-1.196238E-2,-3.779415E-2,-1.6750959E-3,-2.2241866E-2,-9.5762815E-3,8.778771E-3,-2.8383927E-2,-7.908131E-3,-1.1856781E-2,3.126565E-4,-1.1546033E-2,-2.371445E-3,5.358924E-4,1.0693713E-2,9.108089E-3,-1.4562778E-3,2.1096138E-2,3.7885704E-3,8.725644E-3,3.3318173E-2,1.7731842E-2,2.8024337E-3,-1.5898004E-2,3.652892E-4,4.3790983E-3,1.640351E-2,3.3061918E-2,1.1846975E-2,4.53835E-2,3.179863E-2,2.202814E-2,4.2056986E-3,2.8018847E-2,1.3496256E-2,5.5854067E-2,2.832715E-2],"split_indices":[2,43,17,7,2,48,67,32,3,2,69,0,0,43,71,9,2,1,43,4,62,43,3,7,68,2,58,0,4,68,0,0,5,3,2,8,43,8,43,67,60,44,0,4,62,0,7,6,8,71,49,0,0,71,0,0,0,44,46,10,67,0,0,0,4,2,0,4,67,44,6,0,1,68,2,0,0,60,0,0,71,53,3,0,0,73,57,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.19E2,6.5E2,2.69E2,2.16E2,4.34E2,9E0,2.6E2,5.6E1,1.6E2,2.22E2,2.12E2,6E0,3E0,9.1E1,1.69E2,1.3E1,4.3E1,9E0,1.51E2,4.1E1,1.81E2,9.1E1,1.21E2,5.6E1,3.5E1,1.07E2,6.2E1,4E0,9E0,4.1E1,2E0,3E0,6E0,6E1,9.1E1,1.6E1,2.5E1,1.3E1,1.68E2,7.4E1,1.7E1,1.17E2,4E0,7E0,4.9E1,3E0,3.2E1,8.9E1,1.8E1,5.5E1,7E0,5E0,4E0,3.8E1,3E0,3E0,3E0,3.4E1,2.6E1,6E1,3.1E1,1.2E1,4E0,3E0,2.2E1,8E0,5E0,1.13E2,5.5E1,4.6E1,2.8E1,5E0,1.2E1,1.09E2,8E0,4E0,3E0,4.6E1,3E0,3E0,2.9E1,7.8E1,1.1E1,1.6E1,2E0,4.2E1,1.3E1,5E0,2E0,3.5E1,3E0,2.4E1,1E1,3E0,2.3E1,3E0,5.7E1,2.9E1,2E0,2E1,2E0,4E0,4E0,5.3E1,6E1,4.2E1,1.3E1,3.6E1,1E1,2.2E1,6E0,2E0,1E1,6E0,1.03E2,6E0,2E0,2.8E1,1.8E1,2.7E1,2E0,4.6E1,3.2E1,9E0,2E0,1.9E1,2.3E1,4E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[2.1784418E-4,-3.103883E-1,3.0316278E-1,-5.4600877E-1,-1.4662534E-1,1.5138231E-1,6.790598E-1,-7.567568E-1,-4.2105594E-1,-3.499699E-1,-5.8186114E-2,-8.649724E-1,1.9392702E-1,7.8137344E-1,8.7497346E-2,-1.0611479E0,-7.0172423E-1,-6.517001E-1,-3.148859E-1,-2.5255433E-1,-5.350874E-1,-1.7222742E-3,-2.2385776E-1,-4.4154357E-2,-2.0354351E-2,4.211277E-2,1.6191204E-1,6.860494E-2,7.041836E-1,3.0775659E-2,-9.7420305E-2,-1.8685097E-2,-5.645934E-2,-3.3182018E-2,-1.1555198E-2,-7.976688E-1,-5.164033E-1,2.6052505E-2,-3.6831266E-1,-2.8467235E-1,5.242012E-3,-4.5961916E-1,-3.8845815E-2,2.9028662E-2,-2.7176818E-1,-3.9511386E-2,-4.0648627E-1,1.1856441E-1,6.944273E-1,4.3175206E-1,8.1477207E-1,1.129128E-2,-2.310536E-1,-1.3607552E-2,-3.9664768E-2,-8.7528E-3,-2.6074378E-2,-2.9797557E-1,-5.7840014E-1,-3.290153E-1,-1.01577036E-1,-7.1531865E-3,-2.2389676E-2,-4.9445927E-2,1.2076145E-1,-2.1827739E-2,-6.781122E-3,-1.2447198E-2,5.8228145E-3,6.799142E-4,-4.4728774E-1,1.445927E-1,-5.830416E-1,1.2950677E-2,7.221005E-1,2.5378823E-1,6.0466087E-1,8.595318E-1,3.6044255E-1,6.806553E-4,-1.3099087E-2,-1.1391096E-2,-2.4807436E-2,-1.0559824E-2,-3.0814333E-2,-1.7542034E-2,-8.004316E-3,-3.7377613E-4,-7.156606E-3,-4.965832E-3,4.697468E-3,4.3381173E-3,2.1468004E-2,-1.8802426E-3,5.9617977E-3,-2.6317088E-2,-1.2104255E-2,-7.0510856E-3,8.513266E-3,-8.795225E-3,-3.0596603E-2,1.8313605E-2,3.6255926E-2,1.6771238E-2,2.3098078E-3,3.2699015E-2,1.5804924E-2,5.2228272E-2,3.5407845E-2,5.188393E-3,1.9756882E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,-1,49,-1,51,-1,-1,-1,-1,53,55,-1,57,59,-1,61,-1,63,65,67,69,71,73,75,77,-1,79,-1,-1,-1,-1,81,83,85,87,-1,-1,89,91,-1,-1,-1,93,-1,95,97,99,-1,101,103,105,107,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.289907E1,1.6704815E1,2.5343212E1,4.36874E0,4.6194143E0,1.396147E1,7.6788216E0,3.4420776E-1,2.5863628E0,1.2714615E0,1.6927712E0,5.350113E-2,6.847189E0,4.798332E0,2.1881974E0,3.1847382E-1,1.3166618E-1,2.7929878E-1,3.9464636E0,6.538644E-1,2.0508575E-1,1.1372554E0,1.5326166E0,0E0,0E0,0E0,6.72723E0,0E0,2.6609993E0,0E0,7.8349745E-1,0E0,0E0,0E0,0E0,1.6865349E-1,1.9942951E-1,0E0,9.506979E-1,3.644097E-1,0E0,7.9294205E-2,0E0,8.9141035E-1,2.7997243E-1,2.4976501E-1,4.0171862E-1,5.087982E0,5.2164078E-2,8.004613E-1,1.1136894E0,0E0,1.5330839E-1,0E0,0E0,0E0,0E0,6.061797E-1,3.8184166E-1,2.2986269E-1,5.258041E-2,0E0,0E0,5.886684E-1,4.9089605E-1,0E0,0E0,0E0,1.2403549E-1,0E0,3.1844378E-1,3.0465164E0,1.7041111E-1,0E0,7.039642E-2,3.527819E-1,1.5206337E-1,7.74971E-1,8.076751E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,26,26,28,28,30,30,35,35,36,36,38,38,39,39,41,41,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,57,57,58,58,59,59,60,60,63,63,64,64,68,68,70,70,71,71,72,72,74,74,75,75,76,76,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,-1,50,-1,52,-1,-1,-1,-1,54,56,-1,58,60,-1,62,-1,64,66,68,70,72,74,76,78,-1,80,-1,-1,-1,-1,82,84,86,88,-1,-1,90,92,-1,-1,-1,94,-1,96,98,100,-1,102,104,106,108,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.071E3,2.0976269E5,5.467E3,3.574764E7,5.02E2,1E0,2.856934E7,2E0,2.08E2,4.084788E0,4.164E3,7.295292E-2,1.2E1,1.059448E7,1.2176E4,3.466E3,1.317484E1,3.89E2,2E0,4.0705118E6,7.668863E6,1E0,1.9047619E0,-4.4154357E-2,-2.0354351E-2,4.211277E-2,5.405423E6,6.860494E-2,1.1251919E6,3.0775659E-2,1.04328945E5,-1.8685097E-2,-5.645934E-2,-3.3182018E-2,-1.1555198E-2,2.416E3,6.101E3,2.6052505E-2,1.9E1,2.241E2,5.242012E-3,1.0153885E-5,-3.8845815E-2,2.0480016E7,2.6138635E2,6.176636E5,8E0,2.7188679E1,5.294E3,1.5146261E3,1E0,1.129128E-2,3.5056704E7,-1.3607552E-2,-3.9664768E-2,-8.7528E-3,-2.6074378E-2,6.340348E8,7.026578E7,1.4208E4,2.2034105E6,-7.1531865E-3,-2.2389676E-2,3.6648486E2,5.6530495E6,-2.1827739E-2,-6.781122E-3,-1.2447198E-2,1.407E3,6.799142E-4,8.75E2,1.8439855E5,1.404E3,1.2950677E-2,1E0,1.4831979E0,1E0,3.3382E4,1E1,6.806553E-4,-1.3099087E-2,-1.1391096E-2,-2.4807436E-2,-1.0559824E-2,-3.0814333E-2,-1.7542034E-2,-8.004316E-3,-3.7377613E-4,-7.156606E-3,-4.965832E-3,4.697468E-3,4.3381173E-3,2.1468004E-2,-1.8802426E-3,5.9617977E-3,-2.6317088E-2,-1.2104255E-2,-7.0510856E-3,8.513266E-3,-8.795225E-3,-3.0596603E-2,1.8313605E-2,3.6255926E-2,1.6771238E-2,2.3098078E-3,3.2699015E-2,1.5804924E-2,5.2228272E-2,3.5407845E-2,5.188393E-3,1.9756882E-2],"split_indices":[2,43,2,7,2,17,60,32,2,69,44,53,33,12,9,1,69,1,10,43,5,6,69,0,0,0,43,0,43,0,48,0,0,0,0,9,9,0,3,4,0,53,0,5,4,43,3,68,12,67,6,0,60,0,0,0,0,46,46,1,60,0,0,4,43,0,0,0,0,0,2,43,2,0,29,53,23,9,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.79E2,4.34E2,4.45E2,1.77E2,2.57E2,3.18E2,1.27E2,6.4E1,1.13E2,7.7E1,1.8E2,1.2E1,3.06E2,1.08E2,1.9E1,7E0,5.7E1,3.4E1,7.9E1,5.2E1,2.5E1,1.35E2,4.5E1,9E0,3E0,1.2E1,2.94E2,9E0,9.9E1,4E0,1.5E1,2E0,5E0,5.5E1,2E0,1.4E1,2E1,4E0,7.5E1,4.8E1,4E0,2.2E1,3E0,1.22E2,1.3E1,2.3E1,2.2E1,2.73E2,2.1E1,3E1,6.9E1,4E0,1.1E1,2E0,1.2E1,3E0,1.7E1,5.8E1,1.7E1,3.8E1,1E1,2E0,2E1,6.6E1,5.6E1,4E0,9E0,3E0,2E1,2E0,2E1,2.64E2,9E0,2E0,1.9E1,1.6E1,1.4E1,6.2E1,7E0,2E0,9E0,4.9E1,9E0,4E0,1.3E1,2.8E1,1E1,4E0,6E0,4.8E1,1.8E1,5.3E1,3E0,1.5E1,5E0,1.1E1,9E0,3E1,2.34E2,2E0,7E0,4E0,1.5E1,1E1,6E0,9E0,5E0,1.4E1,4.8E1,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-2.2006906E-3,-1.814537E-1,5.347919E-1,-5.029138E-1,-1.5051012E-2,5.935502E-1,-4.9619755E-1,-7.078212E-1,-3.8921097E-1,-1.5577058E-1,1.2573428E-1,3.2398006E-1,7.721623E-1,-3.0494783E-2,-5.109449E-3,-3.3396248E-2,-9.0615535E-3,-4.7657272E-1,-1.283157E-1,-1.3586906E-1,-4.744635E-2,2.3834583E-1,5.96736E-3,1.8364133E-1,6.3943785E-1,6.1796248E-2,7.151833E-1,-4.197852E-1,-9.2901677E-1,2.4887735E-2,-1.9657189E-1,-4.018306E-1,-7.794254E-2,1.9664828E-1,2.7452808E-2,2.4421161E-2,-3.1283837E-2,4.18594E-2,3.5937378E-1,2.1317972E-2,3.4195583E-2,4.3993652E-1,8.9014095E-1,1.1312218E-1,-4.4223273E-1,-2.1389108E-2,-1.106084E0,-9.602387E-2,-1.8512106E-2,-2.6950347E-1,-4.7881943E-1,-1.2458374E-1,1.3423528E-1,2.1819752E-1,-2.0246346E-1,-2.5034046E-1,5.7478923E-2,-7.710832E-2,1.40889E-1,4.886558E-1,1.7261331E-1,3.7417522E-1,3.2119706E-2,9.3338126E-1,3.4910628E-1,-1.24662705E-2,2.1234667E-2,-2.4872871E-2,-1.4281174E-2,-6.69965E-2,-2.8708857E-2,-8.858045E-4,-1.1129699E-2,-1.5840735E-2,-3.8313954E-3,-2.3525927E-2,-1.01012625E-2,-3.4783937E-3,-1.6007777E-2,1.1165723E-2,2.238735E-3,5.2402182E-3,1.7321609E-2,-2.3133073E-2,3.2256788E-3,-1.4426582E-2,-2.4683743E-3,1.7201653E-3,1.4953135E-2,-8.959337E-3,1.0710633E-3,7.6656314E-3,1.8309505E-4,2.8256316E-2,1.1579427E-2,1.4231156E-2,6.77664E-4,1.871789E-2,-5.5054966E-3,4.7031295E-2,2.6338072E-2,7.1901125E-3,2.122304E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,-1,27,29,31,-1,33,35,37,39,-1,41,43,45,-1,47,49,51,53,-1,55,-1,57,59,-1,-1,61,63,65,67,-1,69,71,-1,73,75,77,79,81,83,85,87,89,91,93,95,97,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.884649E1,3.700877E1,1.4288315E1,5.2229614E0,9.073326E0,1.029435E1,7.290156E-1,2.9696274E-1,3.4488544E0,3.8009171E0,3.0871327E0,3.8226871E0,3.2041168E0,0E0,0E0,0E0,0E0,2.5647297E0,1.9165711E0,3.4350815E0,0E0,1.6155787E0,1.419034E0,1.5546451E0,1.3145638E-1,0E0,5.435337E0,1.2941208E0,4.8807526E-1,0E0,7.1131384E-1,2.784648E-1,1.8574219E0,9.6443653E-1,0E0,1.0135198E0,0E0,4.3499142E-1,5.917051E-1,0E0,0E0,5.494051E-1,1.3819504E0,7.769762E-1,1.087101E0,0E0,4.1862583E-1,2.8221542E-1,0E0,1.9812024E-1,5.9864044E-2,1.6473022E0,2.9829836E-1,1.6182756E0,5.15527E-1,1.2523168E-1,5.2377486E-1,2.011161E-1,6.491402E-2,3.0471992E-1,2.6958364E-1,6.190429E-1,0E0,1.3426208E0,7.544625E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,17,17,18,18,19,19,21,21,22,22,23,23,24,24,26,26,27,27,28,28,30,30,31,31,32,32,33,33,35,35,37,37,38,38,41,41,42,42,43,43,44,44,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,-1,28,30,32,-1,34,36,38,40,-1,42,44,46,-1,48,50,52,54,-1,56,-1,58,60,-1,-1,62,64,66,68,-1,70,72,-1,74,76,78,80,82,84,86,88,90,92,94,96,98,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,2.2116136E5,3.9661028E7,9.9E1,9.31E2,1.1048219E3,1.584E3,1.3745962E0,8.57E2,1.05039425E1,3.7105262E0,5.53648E8,1.059448E7,-3.0494783E-2,-5.109449E-3,-3.3396248E-2,-9.0615535E-3,9.932432E0,7.402E3,3.89E2,-4.744635E-2,6.0052995E6,3.8652172E1,7.207765E2,6.714162E8,6.1796248E-2,1.7587205E6,6E0,2.49E2,2.4887735E-2,1.6942337E1,1.09333336E2,3.3486558E6,1.4676277E7,2.7452808E-2,2.691875E2,-3.1283837E-2,5.989087E7,1.3092E4,2.1317972E-2,3.4195583E-2,1.7620745E0,1E0,3.72E2,1.339646E6,-2.1389108E-2,6.8E1,7.3E1,-1.8512106E-2,1E0,1.6423E2,4.988E3,2.1525E4,4.3764517E2,5.59E2,1.688E3,4.86392E5,2.0312773E8,1.3026532E6,7.7191065E6,4.771E3,1.4603988E6,3.2119706E-2,2.395631E7,4.7420635E1,-1.24662705E-2,2.1234667E-2,-2.4872871E-2,-1.4281174E-2,-6.69965E-2,-2.8708857E-2,-8.858045E-4,-1.1129699E-2,-1.5840735E-2,-3.8313954E-3,-2.3525927E-2,-1.01012625E-2,-3.4783937E-3,-1.6007777E-2,1.1165723E-2,2.238735E-3,5.2402182E-3,1.7321609E-2,-2.3133073E-2,3.2256788E-3,-1.4426582E-2,-2.4683743E-3,1.7201653E-3,1.4953135E-2,-8.959337E-3,1.0710633E-3,7.6656314E-3,1.8309505E-4,2.8256316E-2,1.1579427E-2,1.4231156E-2,6.77664E-4,1.871789E-2,-5.5054966E-3,4.7031295E-2,2.6338072E-2,7.1901125E-3,2.122304E-2],"split_indices":[2,43,60,44,2,67,0,53,2,69,69,7,12,0,0,0,0,69,1,2,0,43,68,67,7,0,43,3,2,0,71,4,43,62,0,70,0,5,10,0,0,50,6,2,9,0,0,8,0,8,67,44,9,67,0,2,1,7,43,62,2,43,0,62,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.21E2,6.91E2,2.3E2,2.35E2,4.56E2,2.18E2,1.2E1,8.2E1,1.53E2,2.28E2,2.28E2,8.8E1,1.3E2,8E0,4E0,8E1,2E0,1.14E2,3.9E1,2.24E2,4E0,1.17E2,1.11E2,6.2E1,2.6E1,1E1,1.2E2,1.03E2,1.1E1,3E0,3.6E1,3.9E1,1.85E2,1.06E2,1.1E1,1.09E2,2E0,3.5E1,2.7E1,1.1E1,1.5E1,4.8E1,7.2E1,4E0,9.9E1,4E0,7E0,2.5E1,1.1E1,1.6E1,2.3E1,1.52E2,3.3E1,1.01E2,5E0,1.1E1,9.8E1,1.6E1,1.9E1,1.5E1,1.2E1,4E1,8E0,6.6E1,6E0,2E0,2E0,5.7E1,4.2E1,3E0,4E0,1.7E1,8E0,1.1E1,5E0,2E1,3E0,1.25E2,2.7E1,1.4E1,1.9E1,6.1E1,4E1,2E0,3E0,8E0,3E0,9.2E1,6E0,7E0,9E0,1.6E1,3E0,9E0,6E0,6E0,6E0,3.8E1,2E0,5.3E1,1.3E1,3E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.3598823E-2,-1.5995248E-1,4.6894303E-1,-5.017012E-1,3.2175893E-3,-6.7289954E-1,5.498934E-1,5.15646E-1,-5.237446E-1,-1.3452548E-1,1.4094622E-1,-3.9590538E-2,-3.6582315E-1,1.3538597E0,4.8067084E-1,8.630963E-3,3.133811E-2,-6.164482E-1,-2.8480145E-1,-1.0086976E-1,-5.636419E-1,7.130355E-2,2.953472E-1,-1.3842707E-3,-2.1992596E-2,4.4882033E-2,7.995785E-2,3.2484263E-1,7.7740854E-1,-5.450903E-1,-1.0494523E0,-3.2134047E-1,8.436271E-3,-3.7004566E-1,-6.86626E-2,-7.056903E-1,-5.1157055E-3,-3.8142838E-2,1.7983578E-1,-9.248308E-2,3.2709152E-1,1.09391786E-1,4.6474272E-1,8.065231E-1,-8.39351E-3,-7.2868E-1,-4.851618E-1,-2.3241017E-2,-1.1993797E0,-4.6342012E-1,-2.4184431E-1,-2.9891564E-3,-3.9740935E-1,-4.0428004E-1,-4.126428E-2,-2.0161718E-2,-4.211789E-2,3.551524E-2,-3.2578757E-1,2.271165E-1,-9.91345E-2,1.970473E-3,-1.1229625E-2,3.4599617E-1,-1.998784E-3,3.410452E-1,-5.258175E-2,5.6733936E-1,2.157375E-1,5.702249E-1,9.356304E-1,-1.37994895E-2,-3.4831278E-2,-2.0130487E-2,-3.6246475E-2,-6.787365E-2,-2.6327845E-2,-2.311058E-2,-9.932273E-3,-6.9652456E-3,-1.6127829E-2,-1.9391993E-2,-6.8650176E-3,1.8514168E-3,-2.2398619E-2,-4.16236E-3,4.8033064E-3,3.2946612E-3,-1.1962406E-2,-2.1271443E-2,8.525827E-3,1.1633749E-2,-5.0458936E-3,2.799472E-3,-1.0303271E-2,1.6799353E-2,-3.1946008E-3,5.014632E-3,2.3250535E-2,-1.7682394E-2,1.9953004E-3,2.0388497E-2,4.0708117E-2,1.4687639E-2,-8.704247E-3,2.8795766E-2,-2.045405E-3,4.5367796E-2,1.9609157E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,-1,-1,29,31,33,35,37,39,-1,-1,-1,-1,41,43,45,47,49,-1,51,53,55,-1,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,-1,81,83,85,-1,-1,87,89,91,93,-1,-1,95,-1,97,99,101,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.1593155E1,3.6582245E1,2.3478157E1,5.0243416E0,8.461129E0,6.197357E-1,1.2371941E1,1.3600087E-1,4.43182E0,3.1637602E0,2.3768368E0,0E0,2.7374947E-1,8.980293E-1,9.781567E0,0E0,0E0,4.0317993E0,1.0803518E0,1.7778254E0,9.5340824E-1,1.8480208E0,8.797369E-1,0E0,0E0,0E0,0E0,4.2895517E0,2.2092972E0,1.0767632E0,1.0490208E0,5.204325E-1,0E0,1.5853333E-1,1.7028486E0,2.603736E-1,0E0,1.6641845E0,1.0513284E0,1.3076764E-1,4.6261597E-1,2.1941924E0,2.1002026E0,1.663044E0,0E0,4.029751E-2,1.2172871E0,0E0,1.6439247E0,5.2914143E-2,3.1522727E-1,0E0,3.614688E-2,4.999659E-1,1.2223699E0,0E0,0E0,6.6970456E-1,1.1557311E0,5.450139E-1,2.473667E-1,0E0,0E0,4.0985012E-1,0E0,8.3542347E-1,1.1112128E0,2.0386562E0,1.1477311E0,8.688555E-1,5.1070786E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,48,48,49,49,50,50,52,52,53,53,54,54,57,57,58,58,59,59,60,60,63,63,65,65,66,66,67,67,68,68,69,69,70,70],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,-1,-1,30,32,34,36,38,40,-1,-1,-1,-1,42,44,46,48,50,-1,52,54,56,-1,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,-1,82,84,86,-1,-1,88,90,92,94,-1,-1,96,-1,98,100,102,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,2.2116136E5,1E0,6E0,9.31E2,1.5900173E3,6.743651E7,6.9E2,1.559733E6,2E0,2.0467092E6,-3.9590538E-2,1.91E2,2.35184E5,2.8530578E6,8.630963E-3,3.133811E-2,4.7238097E0,5.189808E2,3.56E2,6.23E2,2.026496E6,2.28E4,-1.3842707E-3,-2.1992596E-2,4.4882033E-2,7.995785E-2,5.202E3,1.0534078E4,5.4E1,4.327E3,1.901875E2,8.436271E-3,1.9428572E1,2.2953334E2,5.4E1,-5.1157055E-3,1.5187539E7,7.1E1,1.1422222E1,2.5345264E7,3.53969E5,4.05E2,1.2400667E3,-8.39351E-3,4E0,4.5593776E7,-2.3241017E-2,1E0,2.511352E6,1.6E1,-2.9891564E-3,7.9016544E5,1E0,2.9652428E6,-2.0161718E-2,-4.211789E-2,3.0084E4,4.888E3,1E0,1.3022917E8,1.970473E-3,-1.1229625E-2,1.0358873E3,-1.998784E-3,5.723202E5,7.70947E5,3.4915986E3,4.1325716E7,3.2E1,4.9036694E0,-1.37994895E-2,-3.4831278E-2,-2.0130487E-2,-3.6246475E-2,-6.787365E-2,-2.6327845E-2,-2.311058E-2,-9.932273E-3,-6.9652456E-3,-1.6127829E-2,-1.9391993E-2,-6.8650176E-3,1.8514168E-3,-2.2398619E-2,-4.16236E-3,4.8033064E-3,3.2946612E-3,-1.1962406E-2,-2.1271443E-2,8.525827E-3,1.1633749E-2,-5.0458936E-3,2.799472E-3,-1.0303271E-2,1.6799353E-2,-3.1946008E-3,5.014632E-3,2.3250535E-2,-1.7682394E-2,1.9953004E-3,2.0388497E-2,4.0708117E-2,1.4687639E-2,-8.704247E-3,2.8795766E-2,-2.045405E-3,4.5367796E-2,1.9609157E-2],"split_indices":[2,43,17,3,2,70,7,2,9,6,43,0,0,1,43,0,0,68,67,2,2,9,1,0,0,0,0,2,4,2,1,4,0,67,48,0,0,60,8,73,60,9,8,4,0,0,5,0,29,5,3,0,43,68,43,0,0,44,10,79,7,0,0,4,0,43,9,4,66,3,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.04E2,6.55E2,2.49E2,2.11E2,4.44E2,1.6E1,2.33E2,4E0,2.07E2,2.22E2,2.22E2,9E0,7E0,1.7E1,2.16E2,2E0,2E0,1.48E2,5.9E1,2.07E2,1.5E1,1.54E2,6.8E1,2E0,5E0,1E1,7E0,1.43E2,7.3E1,1.29E2,1.9E1,5.5E1,4E0,2.1E1,1.86E2,1.1E1,4E0,7.7E1,7.7E1,5E0,6.3E1,5.7E1,8.6E1,7.1E1,2E0,2.9E1,1E2,5E0,1.4E1,1.8E1,3.7E1,2E0,1.9E1,1.3E1,1.73E2,6E0,5E0,6.2E1,1.5E1,6.6E1,1.1E1,3E0,2E0,6E1,3E0,2.3E1,3.4E1,6E1,2.6E1,2.7E1,4.4E1,2E0,2.7E1,8.7E1,1.3E1,9E0,5E0,1.5E1,3E0,2.1E1,1.6E1,1.7E1,2E0,2E0,1.1E1,1.3E2,4.3E1,5.6E1,6E0,1.2E1,3E0,6.2E1,4E0,5E0,6E0,5.8E1,2E0,1E1,1.3E1,7E0,2.7E1,4.4E1,1.6E1,2.1E1,5E0,2.5E1,2E0,4E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[1.679151E-2,-1.553751E-1,4.9455565E-1,-4.7842288E-1,-1.8504846E-3,-5.757761E-1,5.49442E-1,2.8503117E-1,-5.138492E-1,-1.6296212E-1,1.1555587E-1,-7.614703E-3,-2.9849129E-2,1.3131168E0,4.9254832E-1,3.0193882E-2,-6.596894E-2,-7.2361284E-1,-4.135706E-1,-3.9205736E-1,-9.023818E-2,7.816683E-2,4.60895E-1,2.3218168E-2,6.5107204E-2,3.5601246E-1,7.353865E-1,-7.151245E-3,3.398883E-3,-6.0238713E-1,-1.0794579E0,-6.5533787E-1,-3.3514172E-1,-2.804337E-1,-2.2232633E-2,-6.946017E-2,-3.721087E-2,1.4336732E-1,-1.8740913E-2,2.4307117E-1,5.6822896E-1,9.90859E-2,4.6470937E-1,5.948212E-1,8.565995E-1,-6.5231824E-1,-2.2543382E-2,-2.4915861E-2,-5.7133198E-2,-7.0301604E-1,-2.4773689E-1,4.057683E-1,-3.8317367E-1,-2.992248E-1,-2.8686852E-3,-1.4705789E-1,4.0880013E-2,8.9354984E-2,2.6313105E-1,2.538519E-1,-5.7459638E-2,2.986136E-3,1.9025939E-2,2.855124E-2,7.704618E-3,2.6934206E-1,-5.3271014E-2,-1.866797E-2,4.8740733E-1,2.527837E-1,6.389849E-1,9.199785E-3,9.074152E-1,-3.1811524E-2,-1.7582145E-2,-7.883835E-3,6.136438E-3,-2.5617931E-2,-3.9435904E-2,-1.6213117E-2,-2.9896183E-3,-7.784049E-4,2.6886862E-2,-2.2666099E-2,-1.1770915E-2,-1.5707435E-2,-7.118809E-3,-9.493223E-3,1.4481291E-3,-1.16563495E-2,3.0991696E-3,6.918044E-3,-1.7578723E-3,1.5514628E-2,-1.8661879E-3,1.5432645E-2,-4.218001E-3,-8.176365E-3,3.784916E-4,2.144103E-2,6.076975E-3,-4.3704454E-3,7.606569E-3,1.45264845E-2,2.8692303E-2,1.900144E-2,-1.822541E-3,3.0776083E-2,8.874517E-3,4.32997E-2,1.6056204E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,27,29,31,33,35,37,39,-1,-1,41,43,-1,-1,45,47,49,51,53,-1,55,-1,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,81,83,85,-1,87,89,91,93,95,97,-1,-1,-1,-1,99,101,-1,103,105,107,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.328382E1,3.2484352E1,1.418055E1,5.8532486E0,8.436241E0,2.2143483E-1,9.129089E0,1.3211495E0,3.9497604E0,3.0968342E0,3.2952428E0,0E0,0E0,3.609047E-1,6.7341347E0,0E0,8.44866E-2,2.1105537E0,2.4160233E0,2.9785347E-1,2.068365E0,1.4811999E0,4.4637728E-1,0E0,0E0,3.7799702E0,7.683525E-1,0E0,0E0,1.4329071E0,5.571823E-1,4.6714783E-1,3.9633274E0,7.197571E-2,0E0,1.2132742E0,0E0,8.8210845E-1,1.0172585E0,2.7068645E-1,1.9250107E-1,1.1038157E0,2.0828915E0,4.124899E-1,1.0926151E0,1.4121437E-1,1.3589562E-1,0E0,0E0,1.7254162E-1,7.012072E-2,5.194067E-1,1.2454739E0,6.0886502E-2,0E0,8.50348E-1,4.5972E-1,7.457519E-1,9.3222237E-1,3.5250378E-1,6.535523E-1,0E0,0E0,0E0,0E0,4.7138786E-1,2.1834975E-1,0E0,1.9388256E0,2.8891608E-1,2.103138E-1,0E0,1.4311981E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,29,29,30,30,31,31,32,32,33,33,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,65,65,66,66,68,68,69,69,70,70,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,28,30,32,34,36,38,40,-1,-1,42,44,-1,-1,46,48,50,52,54,-1,56,-1,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,82,84,86,-1,88,90,92,94,96,98,-1,-1,-1,-1,100,102,-1,104,106,108,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,6E0,8.1E2,2.01E2,2.4481E4,1.89991E5,3.3817584E7,3.79E2,5.1167645E6,-7.614703E-3,-2.9849129E-2,1.5E1,2.8045622E6,3.0193882E-2,1.1770261E8,2.2E1,1.2877E4,3.1E1,5.6E1,2.5555556E0,3.7788504E7,2.3218168E-2,6.5107204E-2,7.079871E2,7.459E3,-7.151245E-3,3.398883E-3,1.7E1,4.869229E7,2.2470966E-1,7.336111E4,3E0,-2.2232633E-2,1.6802971E6,-3.721087E-2,5.372237E2,9E0,1.2E1,1.4793489E9,1.124641E0,6.27907E0,3.532346E-1,1E0,4.631579E0,5.41E2,-2.4915861E-2,-5.7133198E-2,1.00033E5,6.14E2,4.13E2,1.339646E6,1E0,-2.8686852E-3,4.39776E6,8.612209E-6,5.0149255E0,2.7994856E5,1.0942544E6,1.221E3,2.986136E-3,1.9025939E-2,2.855124E-2,7.704618E-3,2.9E1,1.0378262E10,-1.866797E-2,1.2615725E5,3.744E3,2.6506329E1,9.199785E-3,1E0,-3.1811524E-2,-1.7582145E-2,-7.883835E-3,6.136438E-3,-2.5617931E-2,-3.9435904E-2,-1.6213117E-2,-2.9896183E-3,-7.784049E-4,2.6886862E-2,-2.2666099E-2,-1.1770915E-2,-1.5707435E-2,-7.118809E-3,-9.493223E-3,1.4481291E-3,-1.16563495E-2,3.0991696E-3,6.918044E-3,-1.7578723E-3,1.5514628E-2,-1.8661879E-3,1.5432645E-2,-4.218001E-3,-8.176365E-3,3.784916E-4,2.144103E-2,6.076975E-3,-4.3704454E-3,7.606569E-3,1.45264845E-2,2.8692303E-2,1.900144E-2,-1.822541E-3,3.0776083E-2,8.874517E-3,4.32997E-2,1.6056204E-2],"split_indices":[2,43,17,3,2,0,12,9,7,2,43,0,0,3,43,0,7,3,9,10,3,68,7,0,0,67,2,0,0,3,46,53,60,8,0,43,0,4,3,3,5,53,71,53,27,69,2,0,0,5,1,2,9,29,0,9,53,71,48,66,2,0,0,0,0,8,46,0,48,10,71,0,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.89E2,6.54E2,2.35E2,2.1E2,4.44E2,1.1E1,2.24E2,9E0,2.01E2,1.87E2,2.57E2,2E0,9E0,1.4E1,2.1E2,4E0,5E0,6.3E1,1.38E2,4.4E1,1.43E2,2.33E2,2.4E1,2E0,1.2E1,1.36E2,7.4E1,3E0,2E0,4.9E1,1.4E1,3.2E1,1.06E2,2.1E1,2.3E1,1.4E2,3E0,1.39E2,9.4E1,9E0,1.5E1,4.1E1,9.5E1,3.7E1,3.7E1,4.5E1,4E0,4E0,1E1,2.8E1,4E0,6E0,1E2,1.9E1,2E0,8.2E1,5.8E1,9.7E1,4.2E1,1.1E1,8.3E1,5E0,4E0,1.3E1,2E0,1.9E1,2.2E1,2E0,9.3E1,5E0,3.2E1,3E0,3.4E1,3.9E1,6E0,2E0,2E0,1.6E1,1.2E1,2E0,2E0,2E0,4E0,5.4E1,4.6E1,1.4E1,5E0,6.2E1,2E1,4E0,5.4E1,6.6E1,3.1E1,3.4E1,8E0,9E0,2E0,2.9E1,5.4E1,7E0,1.2E1,1.9E1,3E0,4.1E1,5.2E1,3E0,2E0,3E1,2E0,3.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[6.231791E-3,-1.6114408E-1,4.4783705E-1,-4.4452974E-1,-2.1334935E-2,-6.1294323E-1,4.947858E-1,-6.7981845E-1,-3.278629E-1,-1.8061326E-1,1.1038821E-1,-3.3548765E-2,-1.1295354E-2,1.9476333E-1,6.12954E-1,-6.392207E-1,-5.05939E-2,-4.603306E-1,-7.700205E-2,-3.532778E-1,-9.293314E-2,7.541855E-2,4.2488688E-1,5.1109616E-2,4.619845E-1,1.1588022E0,5.491322E-1,-6.839223E-1,-3.773451E-1,-3.9667708E-1,-8.7284887E-1,5.485052E-1,-2.2067809E-1,-4.1041976E-1,-1.2782437E-1,-7.14829E-2,-3.591293E-2,9.583482E-2,-2.847484E-1,-1.5406689E-3,4.6803033E-1,3.318004E-1,-3.6076725E-2,5.0233114E-1,5.634107E-4,1.8203638E-2,5.743809E-2,4.231001E-1,7.526153E-1,-2.1347238E-2,-3.3356413E-2,-5.090868E-3,-4.1997513E-1,-4.8590833E-1,-2.5532076E-1,-1.5792953E-2,-1.2052454E0,7.056556E-3,2.9240848E-2,-1.4876917E-1,-2.9733295E-2,-3.171846E-1,-4.884542E-1,4.3124445E-2,-1.1467923E-2,6.0204986E-2,-1.1826221E-1,-6.952801E-2,1.3202168E-1,-6.1241772E-2,-4.9804807E-1,5.098621E-1,7.08411E-3,-3.526809E-5,1.8872151E-2,-1.4337618E-3,-1.4550452E-2,3.313262E-1,2.8705623E-2,4.6795943E-1,-7.250409E-2,7.938894E-1,1.5633158E-1,-2.1829879E-2,-6.893738E-3,-3.0344592E-2,-2.0216035E-2,-8.3805E-4,-1.759598E-2,-6.425214E-2,-2.3703532E-2,-9.747724E-3,3.2793074E-3,-1.590683E-2,-4.131558E-3,-6.9663753E-3,-2.36982E-2,-5.7047345E-3,6.2307E-3,5.956042E-3,-4.627406E-3,-8.864662E-3,1.9556057E-4,-1.0491464E-3,-1.2757619E-2,-2.6909576E-3,7.814501E-3,-5.250489E-3,2.105E-3,-2.7685894E-2,-9.41149E-3,1.50719555E-2,2.862627E-2,-2.1383325E-3,6.6745225E-3,1.807694E-2,6.221359E-3,1.9416897E-2,3.6852494E-2,-1.2446315E-2,1.8925436E-2,2.8955354E-3,3.8174894E-2,-1.1634016E-3,1.3281073E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,65,-1,67,69,-1,71,73,75,77,-1,-1,-1,79,81,-1,-1,-1,83,85,87,-1,89,-1,-1,91,-1,93,95,97,-1,99,101,103,105,107,109,111,-1,-1,-1,113,-1,115,-1,117,119,121,123,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.696439E1,2.6020792E1,1.2704044E1,5.7338448E0,9.272401E0,2.6614904E-1,8.35392E0,4.8078156E-1,4.8495064E0,2.9950185E0,2.6294503E0,0E0,0E0,2.625113E0,5.320793E0,5.262451E-1,0E0,2.174551E0,4.794304E0,8.10689E-1,1.8972263E0,1.6376543E0,4.8975182E-1,1.1377009E0,4.2953253E-1,5.1099014E-1,3.6566048E0,9.025383E-2,9.1066E-2,9.4807625E-1,1.7178125E0,2.2292781E-1,1.1993692E0,2.3067379E-1,3.151554E-1,8.141152E-1,0E0,1.2538431E0,5.4158545E-1,0E0,2.1288347E-1,2.7095985E-1,3.4474325E-1,2.2773981E-1,0E0,0E0,0E0,2.2127113E0,1.3033295E0,0E0,0E0,0E0,6.468928E-2,1.5151691E-1,1.0149779E0,0E0,1.5425968E-1,0E0,0E0,5.216139E-1,0E0,1.22776985E-1,1.1662483E-1,1.2186234E-1,0E0,3.9205867E-1,8.6020565E-1,3.5931915E-1,1.1787918E0,4.3618716E-2,5.2064776E-2,1.7348719E-1,0E0,0E0,0E0,2.1966776E-1,0E0,7.265842E-2,0E0,1.1425705E0,9.5080996E-1,1.0133324E0,1.24326654E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,43,43,47,47,48,48,52,52,53,53,54,54,56,56,59,59,61,61,62,62,63,63,65,65,66,66,67,67,68,68,69,69,70,70,71,71,75,75,77,77,79,79,80,80,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,66,-1,68,70,-1,72,74,76,78,-1,-1,-1,80,82,-1,-1,-1,84,86,88,-1,90,-1,-1,92,-1,94,96,98,-1,100,102,104,106,108,110,112,-1,-1,-1,114,-1,116,-1,118,120,122,124,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,2.16577E5,1E0,7.7E1,8.1E2,1.665909E3,7.449545E2,1.05039425E1,6.76E2,4.67E2,5.1167645E6,-3.3548765E-2,-1.1295354E-2,1.8953742E6,1.059448E7,4.1741933E-7,-5.05939E-2,8.719101E0,2.74614E5,7.9016544E5,4.6E1,9.932432E0,4.745397E6,3.1463416E0,2.6506329E1,3.116E3,2.9251662E6,1.863E3,7.980392E0,3.39E2,3.89E2,2.123E3,2.5E1,3.8275862E0,7.0407925E0,1.724E0,-3.591293E-2,3.9595376E7,1.6955555E1,-1.5406689E-3,1.4793489E9,6.009E4,4.2E1,5.941442E6,5.634107E-4,1.8203638E-2,5.743809E-2,3.0070068E7,3.2E1,-2.1347238E-2,-3.3356413E-2,-5.090868E-3,5.6666665E0,5.4E1,8E0,-1.5792953E-2,6.8E1,7.056556E-3,2.9240848E-2,2.0263722E8,-2.9733295E-2,3.774648E0,2.2783158E5,3.2E1,-1.1467923E-2,1.3E1,2.796E2,1.3E1,3.808001E5,2.22E3,7.9784E7,2.47E2,7.08411E-3,-3.526809E-5,1.8872151E-2,2.1373269E8,-1.4550452E-2,1.8996E4,2.8705623E-2,4.907764E3,3.1096E4,1E0,2.8366232E0,-2.1829879E-2,-6.893738E-3,-3.0344592E-2,-2.0216035E-2,-8.3805E-4,-1.759598E-2,-6.425214E-2,-2.3703532E-2,-9.747724E-3,3.2793074E-3,-1.590683E-2,-4.131558E-3,-6.9663753E-3,-2.36982E-2,-5.7047345E-3,6.2307E-3,5.956042E-3,-4.627406E-3,-8.864662E-3,1.9556057E-4,-1.0491464E-3,-1.2757619E-2,-2.6909576E-3,7.814501E-3,-5.250489E-3,2.105E-3,-2.7685894E-2,-9.41149E-3,1.50719555E-2,2.862627E-2,-2.1383325E-3,6.6745225E-3,1.807694E-2,6.221359E-3,1.9416897E-2,3.6852494E-2,-1.2446315E-2,1.8925436E-2,2.8955354E-3,3.8174894E-2,-1.1634016E-3,1.3281073E-2],"split_indices":[2,43,17,44,2,4,67,69,2,2,43,0,0,43,12,52,0,69,5,43,3,69,60,69,71,2,43,9,61,2,2,1,3,69,73,68,0,7,73,0,5,1,3,60,0,0,0,58,3,0,0,0,69,2,32,0,0,0,0,7,0,69,43,10,0,3,70,8,43,10,5,0,0,0,0,5,0,44,0,4,9,27,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.04E2,6.56E2,2.48E2,2.16E2,4.4E2,1E1,2.38E2,7E1,1.46E2,1.99E2,2.41E2,7E0,3E0,6.8E1,1.7E2,6.6E1,4E0,9.5E1,5.1E1,6.6E1,1.33E2,2.18E2,2.3E1,4.5E1,2.3E1,1.6E1,1.54E2,5.5E1,1.1E1,8.4E1,1.1E1,9E0,4.2E1,5.2E1,1.4E1,1.3E2,3E0,2.07E2,1.1E1,2E0,2.1E1,1E1,3.5E1,2.1E1,2E0,2E0,1.4E1,9.7E1,5.7E1,9E0,4.6E1,2E0,9E0,5E1,3.4E1,5E0,6E0,2E0,7E0,3.7E1,5E0,2.6E1,2.6E1,6E0,8E0,3.4E1,9.6E1,3.7E1,1.7E2,6E0,5E0,1.8E1,3E0,2E0,8E0,3.2E1,3E0,1E1,1.1E1,8.9E1,8E0,5.3E1,4E0,7E0,2E0,9E0,4.1E1,1.2E1,2.2E1,4E0,2E0,2.9E1,8E0,2.3E1,3E0,2E0,2.4E1,2E0,4E0,2.4E1,1E1,6E1,3.6E1,3.1E1,6E0,2.7E1,1.43E2,4E0,2E0,3E0,2E0,8E0,1E1,2.5E1,7E0,7E0,3E0,7.9E1,1E1,6E0,2E0,2E0,5.1E1,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[2.243344E-2,-1.3370192E-1,4.8297206E-1,-4.3000925E-1,-2.6721568E-3,-5.673116E-1,5.3566957E-1,-3.7854853E-1,-7.800258E-1,-1.1925039E-1,1.3165785E-1,-3.345104E-2,-1.3742357E-2,3.1762204E-1,6.543336E-1,-5.511832E-1,-2.332866E-1,-3.5768318E-1,-4.1097004E-2,-2.6725644E-1,-5.8320668E-2,1.6010298E-1,-2.3416072E-1,4.5434162E-1,1.265064E-1,7.030132E-1,-1.5858531E-2,-6.42634E-1,-4.221506E-1,4.733915E-2,-3.8445315E-1,-2.0173483E-2,-5.1924763E-3,-9.353594E-2,-3.3073133E-1,-3.3580464E-1,-2.349532E-2,1.1202011E-1,3.1158313E-1,-4.6215296E-1,-6.778743E-2,5.577285E-1,2.3300503E-1,-5.287825E-2,3.0882874E-1,1.1037438E0,6.4141124E-1,-1.4773971E-1,1.7908726E-2,-1.1723445E-2,-3.0410845E-2,-1.1949194E-2,-4.6315247E-1,2.4093153E-2,-7.566115E-2,-3.1964338E-1,-6.6566265E-1,2.8452205E-2,-1.6527574E-1,-2.0192692E-1,-4.0813634E-1,3.0942686E-2,-5.213885E-1,-1.1534988E-1,3.2185692E-2,1.0532085E-2,1.8201151E-1,4.1641813E-2,4.0283564E-1,-2.8284067E-2,-9.329755E-3,-8.226016E-3,9.2159444E-4,3.945667E-1,2.9574037E-2,-7.8250334E-4,2.877419E-1,-1.00241736E-1,6.9666137E-3,2.2098886E-2,8.969361E-2,1.54993655E-2,5.50769E-2,4.098137E-1,7.535802E-1,-1.037228E-2,3.5889414E-3,-2.3217764E-2,-1.0959803E-2,-1.2178661E-3,-1.5240188E-2,-1.6176654E-2,2.2641204E-3,-3.4070067E-2,-1.0266521E-2,-4.953712E-3,3.824332E-3,-1.0645094E-2,-1.9676974E-3,-1.8926518E-2,-7.014497E-3,-2.1738485E-2,-1.2855542E-2,7.724567E-3,-6.7775697E-3,-3.0940337E-2,-1.0169362E-2,-2.936338E-3,-1.5642606E-2,9.184418E-4,1.6017402E-2,-3.6976133E-3,7.840115E-3,4.097432E-4,1.0685294E-2,3.3396808E-3,-4.3209777E-3,8.993247E-3,2.2885166E-2,2.0752952E-2,3.4239911E-3,3.0862435E-3,1.8759226E-2,-1.2421461E-4,-8.177587E-3,9.686591E-3,-2.1780129E-3,2.6503507E-2,1.5198366E-2,3.0984629E-2,4.588876E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,-1,87,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,115,117,-1,-1,-1,-1,119,-1,-1,121,123,-1,-1,125,-1,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.844752E1,2.7603481E1,1.3642601E1,3.631359E0,7.751718E0,2.831092E-1,5.7432785E0,4.686247E0,8.6572456E-1,2.3741744E0,2.4308145E0,0E0,0E0,2.1112509E0,4.839596E0,7.553215E-1,4.5148497E0,8.260703E-2,0E0,8.1813526E-1,1.8185073E0,1.5237861E0,6.135127E-1,9.684448E-1,1.1937401E0,2.6986847E0,6.385161E-1,5.058098E-2,1.4520168E-1,2.2392943E0,1.0447588E0,0E0,0E0,1.972334E-1,4.759121E-1,1.4454432E0,8.680724E-1,1.1630225E0,1.2394996E0,1.4531207E-1,1.0862562E-1,1.5722847E-1,2.3731518E-1,1.9740288E-1,6.1349213E-1,6.5989685E-1,2.8604965E0,1.6959822E-1,0E0,0E0,0E0,0E0,1.2854815E-1,0E0,3.7796384E-1,6.190462E-1,1.9550323E-1,7.408277E-2,9.982893E-2,1.7960101E-1,1.362853E-1,2.1522371E-1,4.6435022E-1,7.172512E-1,4.048187E-1,9.810439E-1,8.000717E-1,6.349882E-2,6.1515045E-1,0E0,0E0,0E0,0E0,1.8269014E-1,0E0,0E0,3.3156896E-1,1.1753918E-1,0E0,0E0,1.5546387E-1,0E0,0E0,3.718791E-1,9.92054E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,52,52,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,73,73,76,76,77,77,80,80,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,-1,88,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,116,118,-1,-1,-1,-1,120,-1,-1,122,124,-1,-1,126,-1,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,1E0,1.007E3,9.3326636E-2,9.939072E2,2.71E2,4.427818E7,4.97E2,9.639872E0,-3.345104E-2,-1.3742357E-2,1.1590965E0,2.871605E2,3E1,3.8396227E0,1.9E1,-4.1097004E-2,1.7754011E0,2.508E2,2.6750835E6,5.62E2,1.4787234E1,7.079871E2,1.059448E7,5.7254577E0,4E0,1E0,1.07933836E5,2.5E1,-2.0173483E-2,-5.1924763E-3,5.2846E4,3.2E1,1.2354949E0,6.374563E5,4.5432812E2,6.1723955E9,4.4761734E5,2.3819518E4,4.7185095E2,2.5393645E5,1.0378262E10,1.4047974E8,3.167E3,1.3711089E6,1.2090626E0,1.7908726E-2,-1.1723445E-2,-3.0410845E-2,-1.1949194E-2,3.328629E6,2.4093153E-2,1E0,2.8366232E0,7.582512E7,5.8182236E1,3.927456E7,1.02E2,1.9750606E2,7E0,2.86152E5,7.03E3,6.663214E6,9.750085E6,1E0,5.74E2,5.4007E7,-2.8284067E-2,-9.329755E-3,-8.226016E-3,9.2159444E-4,4.0953247E2,2.9574037E-2,-7.8250334E-4,5.657353E1,1.1E1,6.9666137E-3,2.2098886E-2,4.771E3,1.54993655E-2,5.50769E-2,2.392405E0,6.663214E6,-1.037228E-2,3.5889414E-3,-2.3217764E-2,-1.0959803E-2,-1.2178661E-3,-1.5240188E-2,-1.6176654E-2,2.2641204E-3,-3.4070067E-2,-1.0266521E-2,-4.953712E-3,3.824332E-3,-1.0645094E-2,-1.9676974E-3,-1.8926518E-2,-7.014497E-3,-2.1738485E-2,-1.2855542E-2,7.724567E-3,-6.7775697E-3,-3.0940337E-2,-1.0169362E-2,-2.936338E-3,-1.5642606E-2,9.184418E-4,1.6017402E-2,-3.6976133E-3,7.840115E-3,4.097432E-4,1.0685294E-2,3.3396808E-3,-4.3209777E-3,8.993247E-3,2.2885166E-2,2.0752952E-2,3.4239911E-3,3.0862435E-3,1.8759226E-2,-1.2421461E-4,-8.177587E-3,9.686591E-3,-2.1780129E-3,2.6503507E-2,1.5198366E-2,3.0984629E-2,4.588876E-2],"split_indices":[2,43,17,6,2,53,67,2,46,2,69,0,0,53,73,70,69,3,0,68,48,43,10,71,67,12,69,0,20,60,3,0,0,9,10,68,43,4,46,43,48,4,43,46,5,2,43,49,0,0,0,0,9,0,30,53,7,62,7,2,70,3,7,44,43,9,29,10,7,0,0,0,0,67,0,0,71,3,0,0,2,0,0,68,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E2,7.1E2,2.4E2,2.17E2,4.93E2,1.1E1,2.29E2,1.91E2,2.6E1,2.64E2,2.29E2,6E0,5E0,8.2E1,1.47E2,8.6E1,1.05E2,6E0,2E1,7.6E1,1.88E2,2.13E2,1.6E1,4.7E1,3.5E1,1.37E2,1E1,4.8E1,3.8E1,3.7E1,6.8E1,4E0,2E0,2.1E1,5.5E1,2E1,1.68E2,1.63E2,5E1,6E0,1E1,3.1E1,1.6E1,1.8E1,1.7E1,1.6E1,1.21E2,8E0,2E0,2E0,4.6E1,9E0,2.9E1,7E0,3E1,5.7E1,1.1E1,8E0,1.3E1,2.2E1,3.3E1,7E0,1.3E1,6.3E1,1.05E2,6.7E1,9.6E1,1.3E1,3.7E1,3E0,3E0,4E0,6E0,1.2E1,1.9E1,3E0,1.3E1,1.5E1,3E0,9E0,8E0,2E0,1.4E1,4.1E1,8E1,6E0,2E0,2.4E1,5E0,2.6E1,4E0,5.3E1,4E0,9E0,2E0,2E0,6E0,8E0,5E0,3E0,1.9E1,2.1E1,1.2E1,4E0,3E0,8E0,5E0,5.2E1,1.1E1,1.02E2,3E0,4.3E1,2.4E1,2.1E1,7.5E1,1.1E1,2E0,1.2E1,2.5E1,1E1,2E0,5E0,8E0,7E0,8E0,4E0,4E0,1.2E1,2.9E1,6.1E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-3.5632546E-3,-1.3352665E-1,4.1394553E-1,-3.0615392E-1,5.3827606E-2,4.8469457E-1,-1.7426452E-1,-4.7297946E-1,-2.003055E-1,-4.6867885E-2,1.5654017E-1,3.4658998E-1,7.57792E-1,6.941337E-2,-4.9071032E-1,-4.0099248E-1,-7.2709256E-1,-3.3902827E-1,-1.2830935E-1,-1.7092645E-2,-2.935729E-1,1.25776E-1,4.722044E-1,1.5844442E-1,5.020843E-1,1.070994E0,5.330543E-1,2.7779388E-1,-9.693139E-3,-7.218356E-3,-2.5487129E-2,-4.7886544E-1,-4.6496045E-2,-5.851786E-1,-5.2556235E-2,-2.4078552E-1,-4.1567832E-1,-9.090095E-2,-4.0797877E-1,-4.646305E-2,1.4219125E-1,-1.7660258E-2,-9.163038E-2,-9.180023E-3,1.6394246E-1,5.124171E-1,6.536535E-3,2.5905952E-2,2.6373503E-1,5.571715E-1,1.35933E-1,5.4712377E-2,2.352997E-2,3.0479273E-2,3.5694274E-1,4.4363984E-3,2.2414323E-2,-2.737019E-2,-3.7273034E-1,2.9150562E-2,-2.8139198E-1,-6.5958574E-3,-6.1756504E-1,-7.7364704E-5,-2.544021E-1,-4.430643E-1,-1.7625976E-3,-5.5095114E-2,-4.04103E-1,-2.4168048E-2,-9.460665E-3,-2.273872E-1,-7.2787385E-3,1.8185747E-1,-6.3845795E-4,-9.2996005E-3,-3.8634916E-4,5.1462956E-2,-1.9682707E-1,1.8041866E-1,-2.6427475E-1,9.115005E-3,2.5678325E-2,1.3493185E-1,-8.484701E-2,3.9245543E-1,1.0435724E-1,5.958599E-1,2.886685E-1,-3.6541854E-3,1.9233094E-1,4.292004E-1,-9.917134E-4,-1.2767144E-2,-3.111569E-2,2.3110795E-3,-1.9695256E-2,-2.9814744E-2,-1.1494593E-2,-1.5487117E-2,-8.657149E-3,-2.2114167E-2,-1.1808289E-2,-8.277365E-3,-7.4970385E-4,-1.2923331E-2,-3.6011316E-2,-1.6741497E-2,1.5620542E-3,1.4306732E-3,-6.5525235E-3,1.5282484E-2,4.4412576E-3,-7.5815804E-3,5.4584886E-3,2.2060599E-4,-1.3877995E-2,4.179739E-3,1.3291588E-2,-2.2806818E-2,2.3221895E-3,1.3361365E-4,9.332861E-3,-5.784743E-3,2.5566225E-3,-1.9777475E-3,2.2168864E-2,-4.686857E-3,8.149394E-3,2.3115356E-2,3.6627114E-2,1.7600564E-2,2.762192E-3,1.17662875E-2,-6.201318E-4,3.0526728E-3,2.2158507E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,-1,57,59,61,-1,63,65,67,69,71,73,-1,75,77,79,81,-1,83,85,87,89,-1,-1,-1,91,-1,-1,-1,93,-1,95,-1,97,-1,99,101,-1,103,105,-1,-1,107,109,111,-1,-1,-1,113,115,117,119,-1,-1,121,123,125,127,129,131,-1,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.165735E1,2.34971E1,9.489525E0,6.5800667E0,3.6169357E0,7.3559914E0,1.961833E0,2.3791924E0,2.2835875E0,1.295171E0,1.6206365E0,3.8992062E0,4.1451187E0,9.263259E-1,1.2644887E-1,3.1799946E0,1.1114922E0,4.9480534E-1,1.5845218E0,7.504456E-1,2.9577708E-1,8.204813E-1,1.280334E-1,8.7233603E-1,1.4168587E0,1.0132141E0,6.4920235E-1,2.8917456E-1,0E0,0E0,0E0,8.8617516E-1,3.6557305E0,2.7138138E-1,0E0,1.20049715E-1,4.280796E-1,1.5253317E0,3.0549645E-1,9.554124E-1,1.571148E-1,0E0,6.1522767E-2,4.1979045E-1,9.225483E-1,5.5157423E-2,0E0,3.6162183E-1,6.770754E-1,4.9282455E-1,1.4818573E-1,0E0,0E0,0E0,5.2847266E-1,0E0,0E0,0E0,1.3005362E0,0E0,8.2080495E-1,0E0,4.182911E-2,0E0,1.2578869E-1,1.2466812E-1,0E0,5.958065E-1,4.3906164E-1,0E0,0E0,8.4252954E-1,5.7360214E-1,2.2208351E-1,0E0,0E0,0E0,4.1492236E-1,1.8568769E-1,1.1192791E0,3.7971884E-1,0E0,0E0,1.2965107E-1,8.975556E-2,7.1629095E-1,2.6480144E-1,6.5060425E-1,1.8335968E-1,0E0,1.1566576E-1,2.4357414E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,31,31,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,54,54,58,58,60,60,62,62,64,64,65,65,67,67,68,68,71,71,72,72,73,73,77,77,78,78,79,79,80,80,83,83,84,84,85,85,86,86,87,87,88,88,90,90,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,-1,58,60,62,-1,64,66,68,70,72,74,-1,76,78,80,82,-1,84,86,88,90,-1,-1,-1,92,-1,-1,-1,94,-1,96,-1,98,-1,100,102,-1,104,106,-1,-1,108,110,112,-1,-1,-1,114,116,118,120,-1,-1,122,124,126,128,130,132,-1,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,4.9963706E5,3.5056704E7,6.593462E4,9.99E2,1.6471942E3,9.7172376E7,1E0,5.41E2,6.346204E-7,5.405423E6,1.5484655E6,1.4595818E7,4.43533E-1,2.01E2,6.76E2,3.3706608E0,9.439024E0,2.0734E4,4.5866325E6,3.33E2,5.036081E7,2.041E3,7.262122E2,2E0,3.7936268E-3,7.153514E1,4.435876E2,-9.693139E-3,-7.218356E-3,-2.5487129E-2,3.89E2,2.00087E5,8.910034E0,-5.2556235E-2,1.2755102E-1,7.997723E6,2.2E1,1.3092E4,1.5024445E2,3.723499E-3,-1.7660258E-2,1.3994015E7,3.4111922E0,1.4676277E7,5.294E3,6.536535E-3,1.3E1,2.9308079E1,2.4571428E1,5.071E3,5.4712377E-2,2.352997E-2,3.0479273E-2,4.1831533E8,4.4363984E-3,2.2414323E-2,-2.737019E-2,8E0,2.9150562E-2,3.390625E0,-6.5958574E-3,2.7125288E5,-7.7364704E-5,1.066109E3,8.1407714E-1,-1.7625976E-3,7.13E2,5.3E1,-2.4168048E-2,-9.460665E-3,2.331083E6,5.1942E4,5.894636E6,-6.3845795E-4,-9.2996005E-3,-3.8634916E-4,1.9548485E2,3E0,4.3764517E2,5.59E2,9.115005E-3,2.5678325E-2,2.12E2,3.164034E0,1.53505E0,2.829581E10,7.098E3,1.006331E5,-3.6541854E-3,1.4627697E3,2.2371138E3,-9.917134E-4,-1.2767144E-2,-3.111569E-2,2.3110795E-3,-1.9695256E-2,-2.9814744E-2,-1.1494593E-2,-1.5487117E-2,-8.657149E-3,-2.2114167E-2,-1.1808289E-2,-8.277365E-3,-7.4970385E-4,-1.2923331E-2,-3.6011316E-2,-1.6741497E-2,1.5620542E-3,1.4306732E-3,-6.5525235E-3,1.5282484E-2,4.4412576E-3,-7.5815804E-3,5.4584886E-3,2.2060599E-4,-1.3877995E-2,4.179739E-3,1.3291588E-2,-2.2806818E-2,2.3221895E-3,1.3361365E-4,9.332861E-3,-5.784743E-3,2.5566225E-3,-1.9777475E-3,2.2168864E-2,-4.686857E-3,8.149394E-3,2.3115356E-2,3.6627114E-2,1.7600564E-2,2.762192E-3,1.17662875E-2,-6.201318E-4,3.0526728E-3,2.2158507E-2],"split_indices":[2,43,60,43,2,67,60,6,2,52,43,43,62,49,0,2,68,73,44,43,10,7,2,67,6,72,71,71,0,0,0,1,5,71,0,73,9,3,10,70,57,0,5,68,62,12,0,3,71,71,2,0,0,0,47,0,0,0,8,0,69,0,48,0,48,53,0,2,3,0,0,43,1,60,0,0,0,67,8,67,0,0,0,0,53,68,46,2,48,0,67,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E2,7.25E2,2.25E2,3.77E2,3.48E2,2.01E2,2.4E1,1.45E2,2.32E2,1.76E2,1.72E2,1.35E2,6.6E1,1.4E1,1E1,1.15E2,3E1,7.8E1,1.54E2,1.58E2,1.8E1,1.58E2,1.4E1,6.2E1,7.3E1,2.6E1,4E1,8E0,6E0,2E0,8E0,9.4E1,2.1E1,2.4E1,6E0,3.6E1,4.2E1,1.37E2,1.7E1,1.34E2,2.4E1,1.2E1,6E0,3.5E1,1.23E2,1.2E1,2E0,2.8E1,3.4E1,6.3E1,1E1,2.1E1,5E0,2.2E1,1.8E1,5E0,3E0,4.4E1,5E1,5E0,1.6E1,2E0,2.2E1,2E0,3.4E1,3.9E1,3E0,1.24E2,1.3E1,1E1,7E0,2.3E1,1.11E2,1.9E1,5E0,2E0,4E0,2.7E1,8E0,1.19E2,4E0,2E0,1E1,1.4E1,1.4E1,1.8E1,1.6E1,5.4E1,9E0,2E0,8E0,1.5E1,3E0,3.9E1,1.1E1,5E0,1.1E1,2E1,2E0,1.4E1,2E1,3.2E1,7E0,2.9E1,9.5E1,1.1E1,2E0,1.5E1,8E0,8.7E1,2.4E1,6E0,1.3E1,6E0,2.1E1,3E0,5E0,6.5E1,5.4E1,2E0,2E0,5E0,9E0,1.1E1,3E0,3E0,1.5E1,4E0,1.2E1,3.8E1,1.6E1,6E0,3E0,6E0,2E0,2E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[8.98041E-3,-1.2767678E-1,4.088996E-1,-3.749437E-1,-1.385104E-2,4.7631106E-1,-6.0881145E-2,-3.309193E-1,-7.096819E-1,-1.4165075E-1,8.795046E-2,6.643838E-2,4.4005734E-1,-4.6203616E-1,1.3304415E-1,-5.151079E-1,-2.2954684E-1,-5.625143E-2,-4.7587717E-1,-2.0847856E-1,-4.9736924E-3,6.0766492E-2,4.2955938E-1,3.5618934E-1,6.861038E-1,-7.851325E-3,-2.3915978E-2,2.6510876E-1,-1.6895773E-1,-5.6219923E-1,-3.4353352E-1,-2.8610417E-1,1.4658816E-1,-4.6254564E-3,-5.167138E-1,-1.7825352E-1,-4.2969275E-1,3.312753E-2,-4.1055778E-1,7.31736E-2,-4.39442E-1,2.3301486E-2,2.3547076E-1,1.7022522E-1,4.4839117E-1,7.526119E-1,8.402718E-2,1.3385653E-3,3.8039413E-1,-1.0004841E-2,-1.6601643E-3,-4.6134055E-1,-3.3554133E-2,-7.8758765E-3,-4.0199944E-1,6.87843E-2,-3.3912757E-1,2.3342956E-2,-4.3292098E-2,-5.781348E-1,-1.2369191E-2,-2.1454911E-1,-4.6149004E-2,-2.4596894E-1,-2.9335612E-2,-1.45563325E-2,4.9340762E-2,-3.5838408E-3,-2.5953189E-2,1.7781556E-2,1.6508642E-1,-2.0313193E-3,-2.9132582E-2,3.4800174E-3,1.5684536E-2,2.4036545E-1,-1.4135715E-2,7.0124054E-1,3.8741946E-1,4.47467E-2,6.2276185E-1,1.2247974E-2,-8.514832E-3,2.2891605E-2,7.9124905E-3,-2.2479493E-2,-8.158477E-3,-6.0699005E-3,-2.0614251E-2,-4.7260365E-3,1.9994253E-2,-1.9118179E-2,-1.0009933E-2,-4.684289E-3,2.662677E-3,-1.1047429E-2,-2.8950946E-2,-8.221735E-3,-1.5271794E-2,2.0370993E-3,-7.893873E-3,-1.3485615E-2,-2.169759E-3,1.0100721E-3,8.873876E-3,2.1196976E-3,-8.129516E-3,9.712867E-3,-4.966869E-3,3.1317386E-4,1.576377E-2,-1.1168011E-2,2.900898E-3,3.4388606E-2,1.2272081E-2,6.5089758E-3,2.0927839E-2,3.4491844E-2,2.3361128E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,33,35,37,39,41,43,45,-1,-1,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,-1,73,75,77,79,81,-1,83,-1,-1,85,-1,-1,87,89,91,-1,93,95,-1,97,99,101,-1,-1,103,-1,-1,105,107,-1,-1,-1,-1,109,111,113,115,-1,117,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9785378E1,1.9105495E1,7.37611E0,2.9187946E0,6.0752125E0,6.3793564E0,2.3952947E0,3.4480438E0,2.2291193E0,1.8896656E0,2.3883314E0,0E0,3.820797E0,6.625342E-2,8.8235104E-1,3.4820366E-1,2.701088E0,0E0,2.2530222E-1,8.527889E-1,1.079875E0,1.5340416E0,1.4556813E-1,2.4784603E0,1.8988647E0,0E0,0E0,3.977275E-1,3.5384923E-2,4.8644447E-1,1.0505152E-1,2.0744238E0,1.1687189E0,0E0,9.2662334E-2,5.846555E-1,4.3593454E-1,3.745892E-1,2.5211263E-1,1.203688E0,4.1660106E-1,0E0,8.9212894E-2,6.628847E-1,1.2545757E0,5.979233E-1,3.3561888E-1,0E0,1.5136838E-1,0E0,0E0,1.2917995E-1,0E0,0E0,7.652104E-2,9.667504E-1,7.583666E-1,0E0,7.5056545E-2,3.3456326E-2,0E0,3.58922E-1,3.1869492E-1,7.416463E-2,0E0,0E0,2.3812531E-1,0E0,0E0,8.0556035E-1,1.0811536E0,0E0,0E0,0E0,0E0,8.499241E-1,2.7434576E-1,1.054554E-1,1.2066612E0,0E0,5.7045937E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,44,44,45,45,46,46,48,48,51,51,54,54,55,55,56,56,58,58,59,59,61,61,62,62,63,63,66,66,69,69,70,70,75,75,76,76,77,77,78,78,80,80],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,34,36,38,40,42,44,46,-1,-1,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,-1,74,76,78,80,82,-1,84,-1,-1,86,-1,-1,88,90,92,-1,94,96,-1,98,100,102,-1,-1,104,-1,-1,106,108,-1,-1,-1,-1,110,112,114,116,-1,118,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,2.856934E7,5.035714E0,8.1E2,3.3817584E7,1.465E4,9.2E1,2E0,1.5859042E6,5.1167645E6,6.643838E-2,4.1973075E6,4E0,7.5491136E-1,3.5988172E-8,3.5467294E2,-5.625143E-2,6.48334E5,2E0,7.89E2,1.2784314E1,2.041E3,8.40413E2,6.292039E0,-7.851325E-3,-2.3915978E-2,1.5896863E3,1.01E3,9.55188E5,1E0,1.25E0,2.7E1,-4.6254564E-3,1.3337367E7,6.946813E7,1.7913036E6,9.3E1,5.5E1,4.2708197E2,3.4E1,2.3301486E-2,2.483E3,1.835821E1,1.16E2,3.9914E4,4.493856E6,1.3385653E-3,5.2224144E5,-1.0004841E-2,-1.6601643E-3,5.4051723E0,-3.3554133E-2,-7.8758765E-3,2.3316922E0,1.8029197E0,1.318849E6,2.3342956E-2,7.080933E2,1.9E1,-1.2369191E-2,3.4698796E0,5.0405306E-1,2.5912744E2,-2.9335612E-2,-1.45563325E-2,5.5426865E6,-3.5838408E-3,-2.5953189E-2,6.6E1,1.5608E4,-2.0313193E-3,-2.9132582E-2,3.4800174E-3,1.5684536E-2,2.4358286E7,1.4293194E0,1E0,2.6463525E8,4.47467E-2,1.4598765E0,1.2247974E-2,-8.514832E-3,2.2891605E-2,7.9124905E-3,-2.2479493E-2,-8.158477E-3,-6.0699005E-3,-2.0614251E-2,-4.7260365E-3,1.9994253E-2,-1.9118179E-2,-1.0009933E-2,-4.684289E-3,2.662677E-3,-1.1047429E-2,-2.8950946E-2,-8.221735E-3,-1.5271794E-2,2.0370993E-3,-7.893873E-3,-1.3485615E-2,-2.169759E-3,1.0100721E-3,8.873876E-3,2.1196976E-3,-8.129516E-3,9.712867E-3,-4.966869E-3,3.1317386E-4,1.576377E-2,-1.1168011E-2,2.900898E-3,3.4388606E-2,1.2272081E-2,6.5089758E-3,2.0927839E-2,3.4491844E-2,2.3361128E-2],"split_indices":[2,43,60,68,2,7,44,44,32,43,43,0,43,8,53,52,67,0,46,6,2,68,2,67,53,0,0,4,8,46,15,68,0,0,5,7,60,12,0,67,3,0,2,71,0,9,1,0,43,0,0,69,0,0,69,69,9,0,4,3,0,71,57,4,0,0,43,0,0,8,44,0,0,0,0,5,68,30,7,0,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.09E2,6.78E2,2.31E2,2.13E2,4.65E2,2.02E2,2.9E1,1.9E2,2.3E1,2.06E2,2.59E2,6E0,1.96E2,9E0,2E1,6.6E1,1.24E2,6E0,1.7E1,1.38E2,6.8E1,2.41E2,1.8E1,1.48E2,4.8E1,2E0,7E0,1.4E1,6E0,5E1,1.6E1,1.08E2,1.6E1,2E0,1.5E1,1.23E2,1.5E1,6.3E1,5E0,2.36E2,5E0,1.2E1,6E0,5E1,9.8E1,4.3E1,5E0,5E0,9E0,4E0,2E0,3.3E1,1.7E1,5E0,1.1E1,1.4E1,9.4E1,5E0,1.1E1,1.1E1,4E0,9.6E1,2.7E1,9E0,6E0,2E0,6.1E1,2E0,3E0,1.48E2,8.8E1,2E0,3E0,3E0,3E0,3.6E1,1.4E1,1.7E1,8.1E1,1.4E1,2.9E1,3E0,2E0,5E0,4E0,3E1,3E0,2E0,9E0,1E1,4E0,5.8E1,3.6E1,7E0,4E0,2E0,9E0,7.4E1,2.2E1,1.6E1,1.1E1,7E0,2E0,5.2E1,9E0,1.3E2,1.8E1,7.6E1,1.2E1,1.1E1,2.5E1,3E0,1.1E1,1.5E1,2E0,1.7E1,6.4E1,1.2E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-1.5113368E-3,-1.3308439E-1,3.7634593E-1,-2.5537106E-1,5.106766E-2,4.4291002E-1,-5.768089E-2,-3.9591822E-1,-1.5261196E-1,-6.411712E-2,1.500127E-1,1.6263317E-1,5.593714E-1,-4.5097423E-1,9.220748E-2,-3.6645338E-1,-9.467745E-1,-9.433592E-2,-5.47687E-1,-2.1474425E-2,-1.7761743E-1,1.7188396E-1,-1.0671739E-1,1.2454297E-2,4.2829323E-1,5.963201E-1,-1.3867737E-1,-5.801793E-3,-2.4479726E-2,-4.3325566E-2,3.2290983E-1,-5.315007E-1,-3.009611E-1,-5.637316E-3,-5.589092E-2,1.8467896E-1,-1.3745369E-1,-3.4628382E-1,-3.5026133E-2,-7.8700416E-2,1.19206674E-1,5.3766016E-3,-2.7228674E-1,9.10276E-2,2.8998545E-1,-2.1659845E-1,1.8260459E-3,-2.0623818E-1,7.357236E-2,4.7273186E-1,8.494866E-3,6.2227253E-2,5.5779785E-1,-1.009226E-2,2.9857256E-3,1.0249122E-1,-2.454459E-1,1.915111E-2,3.6733775E-4,-1.7307825E-2,-5.7816887E-1,1.1327334E-1,-3.1635424E-1,2.5285214E-2,8.2997136E-2,-2.3010293E-1,2.3017803E-4,-3.2561882E-3,-4.0048346E-1,-1.7309505E-2,-1.7248222E-1,-4.821764E-3,1.51387E-1,-6.8488425E-1,-1.4143571E-1,1.1498074E-1,-2.651309E-2,7.272259E-2,3.2079646E-1,-5.511295E-4,-1.3772178E-2,-1.3716006E-2,-3.5489427E-3,-4.72137E-3,9.518582E-2,1.3765782E-2,2.4948379E-2,6.75223E-1,3.4751034E-1,-3.9024656E-3,1.5958041E-1,-6.142794E-4,-1.9362135E-2,-3.0422615E-2,-1.9920884E-2,1.50372E-2,-6.2570865E-3,-1.1081865E-2,-1.7533984E-2,-6.771246E-3,8.9343665E-3,-1.5172566E-2,-5.0996374E-3,1.3194911E-2,-9.448274E-4,-1.9976791E-2,-7.0119E-3,9.925321E-4,-7.306612E-3,-1.0507974E-2,-2.9929422E-3,1.2220474E-2,5.034346E-3,-3.7421323E-2,-1.3809514E-2,3.9506378E-3,-9.28751E-3,6.2465067E-3,-9.207488E-3,-1.8483786E-3,6.8901447E-3,1.6509285E-2,3.4763115E-3,6.226933E-3,-2.6517906E-4,3.63194E-2,2.520985E-2,1.2086576E-2,3.2931298E-2,1.7326669E-3,1.0488582E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,57,59,61,-1,-1,63,65,67,-1,69,71,-1,73,75,77,79,-1,81,83,85,-1,-1,87,-1,-1,89,91,-1,-1,-1,93,95,97,-1,99,101,103,-1,105,107,109,-1,111,113,115,117,-1,119,121,-1,-1,-1,-1,-1,123,-1,-1,125,127,-1,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.504291E1,1.5146039E1,6.7859497E0,5.7743473E0,3.075401E0,6.53524E0,1.9279864E0,2.412302E0,5.3588333E0,6.0118055E-1,8.325193E-1,2.4283543E0,3.7709465E0,1.55671E-1,7.644173E-1,1.6019592E0,1.5383787E0,2.5036676E0,9.804945E-1,7.5003827E-1,9.752966E-1,1.2532036E0,2.0099069E-1,5.4981583E-1,1.5380096E-1,3.1393585E0,1.4105365E-1,0E0,0E0,4.9671665E-1,2.4871701E-1,1.1948013E-1,7.946224E-1,0E0,0E0,9.769833E-1,2.2834318E0,2.2651386E-1,0E0,3.7699953E-1,2.0935196E-1,0E0,1.3068417E0,1.3482349E0,3.3815193E-1,1.10474974E-1,0E0,8.119604E-2,1.3076116E-1,4.3888092E-2,0E0,0E0,3.000145E0,0E0,0E0,1.1980555E-1,2.7228865E-1,0E0,0E0,0E0,9.5458984E-2,3.0377102E-1,4.5479107E-1,0E0,6.016191E-1,1.2032504E0,4.3094587E-1,0E0,3.6815405E-2,2.2488572E-1,1.3575083E-1,0E0,8.6868525E-2,3.9812803E-2,2.939477E-1,5.045222E-1,0E0,7.375325E-2,3.6878204E-1,0E0,0E0,0E0,0E0,0E0,1.14004016E-1,0E0,0E0,7.4507904E-1,1.3641748E0,0E0,5.6130454E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,35,35,36,36,37,37,39,39,40,40,42,42,43,43,44,44,45,45,47,47,48,48,49,49,52,52,55,55,56,56,60,60,61,61,62,62,64,64,65,65,66,66,68,68,69,69,70,70,72,72,73,73,74,74,75,75,77,77,78,78,84,84,87,87,88,88,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,58,60,62,-1,-1,64,66,68,-1,70,72,-1,74,76,78,80,-1,82,84,86,-1,-1,88,-1,-1,90,92,-1,-1,-1,94,96,98,-1,100,102,104,-1,106,108,110,-1,112,114,116,118,-1,120,122,-1,-1,-1,-1,-1,124,-1,-1,126,128,-1,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,7.9016544E5,1E0,5.02E2,9.48E2,7.079871E2,1E0,4.5E1,2.3E1,2.835E3,8.695652E0,1.6208625E6,2.871605E2,3.714144E8,5.854E3,3.89E2,5.5E1,3.5E1,2.347364E7,3.0479442E9,1.1712844E2,4.2259482E2,2.0394794E8,1E0,1.584E3,3.3817584E7,1.2090626E0,-5.801793E-3,-2.4479726E-2,9E0,1.5057336E0,2.416E3,1.0952381E0,-5.637316E-3,-5.589092E-2,2.04115E5,2.026453E6,4.888889E0,-3.5026133E-2,1E0,1E0,5.3766016E-3,1.5933333E2,2.483E3,4.727453E7,3E0,1.8260459E-3,1.453785E6,6.12E2,5.45901E6,8.494866E-3,6.2227253E-2,8E0,-1.009226E-2,2.9857256E-3,2.1883681E5,1.7234043E1,1.915111E-2,3.6733775E-4,-1.7307825E-2,3.9351077E2,1.3E1,9.114943E0,2.5285214E-2,7.24E2,1.5454154E-1,5E0,-3.2561882E-3,1.6576804E7,3.9649122E0,3.4289816E2,-4.821764E-3,2.6654E4,4.37E2,1.1057693E0,1E0,-2.651309E-2,9.526E4,3.4052496E8,-5.511295E-4,-1.3772178E-2,-1.3716006E-2,-3.5489427E-3,-4.72137E-3,3.6545064E0,1.3765782E-2,2.4948379E-2,2.1298597E0,4.435876E2,-3.9024656E-3,1.6E1,-6.142794E-4,-1.9362135E-2,-3.0422615E-2,-1.9920884E-2,1.50372E-2,-6.2570865E-3,-1.1081865E-2,-1.7533984E-2,-6.771246E-3,8.9343665E-3,-1.5172566E-2,-5.0996374E-3,1.3194911E-2,-9.448274E-4,-1.9976791E-2,-7.0119E-3,9.925321E-4,-7.306612E-3,-1.0507974E-2,-2.9929422E-3,1.2220474E-2,5.034346E-3,-3.7421323E-2,-1.3809514E-2,3.9506378E-3,-9.28751E-3,6.2465067E-3,-9.207488E-3,-1.8483786E-3,6.8901447E-3,1.6509285E-2,3.4763115E-3,6.226933E-3,-2.6517906E-4,3.63194E-2,2.520985E-2,1.2086576E-2,3.2931298E-2,1.7326669E-3,1.0488582E-2],"split_indices":[2,43,6,2,2,67,17,6,3,44,61,43,73,7,2,1,0,0,5,46,70,67,7,26,0,7,49,0,0,8,57,9,68,0,0,5,9,71,0,26,29,0,4,2,7,8,0,9,44,60,0,0,32,0,0,48,71,0,0,0,48,0,73,0,2,53,3,0,5,69,4,0,9,0,69,79,0,1,7,0,0,0,0,0,68,0,0,53,71,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.04E2,6.71E2,2.33E2,4.03E2,2.68E2,2.02E2,3.1E1,1.69E2,2.34E2,1.24E2,1.44E2,6E1,1.42E2,8E0,2.3E1,1.62E2,7E0,2.05E2,2.9E1,9.1E1,3.3E1,1.33E2,1.1E1,3.9E1,2.1E1,1.35E2,7E0,2E0,6E0,1.5E1,8E0,4.4E1,1.18E2,2E0,5E0,2.7E1,1.78E2,1.6E1,1.3E1,6.5E1,2.6E1,8E0,2.5E1,8E1,5.3E1,6E0,5E0,8E0,3.1E1,1.7E1,4E0,5E0,1.3E2,5E0,2E0,9E0,6E0,6E0,2E0,1.2E1,3.2E1,4E0,1.14E2,5E0,2.2E1,1.06E2,7.2E1,3E0,1.3E1,4E1,2.5E1,3E0,2.3E1,5E0,2E1,7.8E1,2E0,7E0,4.6E1,2E0,4E0,4E0,4E0,3E0,2.8E1,6E0,1.1E1,8.2E1,4.8E1,2E0,7E0,3E0,3E0,1.9E1,1.3E1,2E0,2E0,5.2E1,6.2E1,7E0,1.5E1,5.8E1,4.8E1,4E0,6.8E1,1.1E1,2E0,3.2E1,8E0,1.6E1,9E0,5E0,1.8E1,3E0,2E0,4E0,1.6E1,7.4E1,4E0,3E0,4E0,4E1,6E0,2E1,8E0,4.3E1,3.9E1,4E1,8E0,3E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-4.1387575E-3,-1.9973798E-1,2.0709999E-1,-3.8629413E-1,-8.88869E-2,2.8051317E-1,-2.426398E-1,-4.446905E-1,-1.8316731E-1,-2.9555514E-1,-4.703028E-2,1.5876986E-1,5.671489E-1,-5.5076737E-2,-5.373198E-1,4.0142974E-3,-4.5731705E-1,-2.3719975E-1,-1.4242456E-2,-2.420508E-1,-4.0426192E-1,-1.1154202E-2,-1.5985402E-1,2.8714758E-1,-2.2273995E-3,6.8552864E-1,2.9819366E-1,-2.530944E-1,4.1889385E-3,-3.316094E-2,-3.6088774E-1,-4.237166E-1,-6.9339275E-1,-2.8459004E-1,-4.173391E-3,-6.7298845E-3,3.7661744E-3,-1.911774E-2,-2.080779E-1,-2.012483E-2,-6.7986106E-3,-9.034799E-2,4.6022896E-2,1.6086547E-1,-1.984685E-1,2.1174127E-1,4.351779E-1,-1.4979853E-1,5.377346E-2,4.7688368E-1,8.871147E-1,3.9207795E-1,-1.1002486E-2,-4.5968736E-5,-1.5114195E-2,-5.202902E-2,2.0436895E-1,-4.3570185E-3,-1.9777102E-2,-4.9506336E-1,-2.9973015E-1,-5.196376E-3,-8.947484E-1,-1.4594281E-2,-6.9535924E-3,-2.9512843E-1,-1.4109114E-1,5.0413072E-2,-1.758129E-1,-1.3796096E-1,7.399955E-2,-4.7132818E-4,1.0756846E-2,-2.231524E-1,1.6833274E-3,2.461122E-1,-6.082165E-2,4.7149023E-1,-4.241876E-3,-2.0584863E-1,8.779454E-2,-6.4458167E-3,1.7443427E-1,2.9016748E-1,6.319999E-1,9.5166886E-1,1.2590693E-2,3.028061E-1,2.5545133E-2,-1.2513408E-1,5.5325874E-3,1.1976807E-2,2.2162977E-3,-2.9467333E-2,-2.0886665E-2,1.8273234E-2,-1.5819248E-2,-5.502399E-2,-2.1265732E-2,-6.5319347E-3,-1.619717E-2,5.085191E-4,-7.3756496E-3,-9.074098E-5,8.307703E-3,-4.7082524E-3,-1.3143787E-2,8.299324E-4,-1.2860364E-2,1.1243781E-3,7.1687093E-3,-1.1018637E-2,3.1134381E-3,4.2910194E-3,1.3457824E-2,3.6241282E-3,-9.885484E-3,1.7060831E-2,2.8550986E-2,-1.6461916E-2,-5.806728E-3,-7.150063E-5,7.2169453E-3,-4.514166E-3,2.5181393E-3,7.421129E-4,1.0002963E-2,2.068948E-2,5.72772E-3,1.0200773E-2,3.0876111E-2,4.7827423E-2,2.9035617E-2,5.393077E-3,1.7778432E-2,-2.5230604E-5,-1.3059594E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,-1,-1,-1,-1,65,-1,-1,67,69,71,73,75,77,79,81,83,85,87,-1,-1,-1,89,91,-1,-1,93,95,-1,97,-1,-1,99,101,103,105,107,109,-1,-1,111,-1,113,115,117,-1,119,121,123,125,127,129,131,-1,133,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.764087E1,9.747278E0,1.4529432E1,2.0062351E0,2.5639296E0,1.3065737E1,3.3972125E0,9.5132065E-1,3.7084615E-1,1.9296217E-1,1.0060878E0,5.4984503E0,3.3615303E0,4.587506E-1,4.866681E-1,0E0,7.611866E-1,1.9110131E-1,1.4837065E-1,1.21177435E-1,4.6916246E-2,8.644916E-1,7.74992E-1,1.5598936E0,9.914842E-1,2.8155632E0,1.9001815E0,1.6279173E-1,3.594982E-1,0E0,2.0091128E-1,9.0429497E-1,1.6568389E0,4.3842316E-2,0E0,0E0,0E0,0E0,1.4495933E-1,0E0,0E0,9.712737E-1,5.81554E-1,8.664492E-2,3.2086682E-1,9.517932E-1,9.796057E-1,4.575373E-1,6.3327575E-1,9.7512245E-1,1.2544041E0,2.686906E-1,0E0,0E0,0E0,3.2739198E-1,4.6038717E-2,0E0,0E0,1.8663406E-1,1.4212708E0,0E0,8.4678364E-1,0E0,0E0,5.794382E-2,5.4940313E-2,2.084836E-1,3.6964E-1,3.2876405E-1,3.8358444E-1,0E0,0E0,2.1158242E-1,0E0,5.655832E-1,2.6919687E-1,4.603443E-1,0E0,2.7970243E-1,4.2374942E-2,3.2919765E-1,1.8338066E-1,4.6192706E-1,1.3077927E-1,2.1173668E-1,0E0,2.8231335E-1,0E0,3.4896448E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,38,38,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,55,55,56,56,59,59,60,60,62,62,65,65,66,66,67,67,68,68,69,69,70,70,73,73,75,75,76,76,77,77,79,79,80,80,81,81,82,82,83,83,84,84,85,85,87,87,89,89],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,-1,-1,-1,-1,66,-1,-1,68,70,72,74,76,78,80,82,84,86,88,-1,-1,-1,90,92,-1,-1,94,96,-1,98,-1,-1,100,102,104,106,108,110,-1,-1,112,-1,114,116,118,-1,120,122,124,126,128,130,132,-1,134,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,3.0923106E-10,1.339646E6,4.03E2,1.1603809E3,4.217427E7,1.0476191E0,8.0509944E5,6.2233735E-6,2.652174E0,8E0,1.6007428E7,4.05E2,1.5570383E5,4.0142974E-3,9.932432E0,6.54E2,1.86619E5,1.02E2,8.314423E6,2.6676828E2,3.0620155E0,5.747068E2,2.5711747E5,1.6978182E3,4.24038E8,8.620714E2,4E1,-3.316094E-2,1E0,2.55E2,1.3E1,7.997723E6,-4.173391E-3,-6.7298845E-3,3.7661744E-3,-1.911774E-2,1.4E1,-2.012483E-2,-6.7986106E-3,1.6964285E0,1.4268688E-5,1.1742609E7,1.7587205E6,1.4329944E0,3.1773497E5,1.7022566E7,8.7390656E2,1.5484655E6,4.262058E0,2.7561485E6,-1.1002486E-2,-4.5968736E-5,-1.5114195E-2,1.038946E6,3.0359353E5,-4.3570185E-3,-1.9777102E-2,1.2593857E0,2.00087E5,-5.196376E-3,6.9E1,-1.4594281E-2,-6.9535924E-3,8.411777E7,5.1066592E2,4.49239E6,6.15E2,3.1275E4,6.9E1,-4.7132818E-4,1.0756846E-2,5.8222644E2,1.6833274E-3,5.80418E7,1.9333333E0,1.0378262E10,-4.241876E-3,1.609E3,2.686063E6,6.946813E7,5.095006E5,1.20267E5,1.308179E8,2.5503825E6,1.2590693E-2,2.338255E3,2.5545133E-2,1.2E1,5.5325874E-3,1.1976807E-2,2.2162977E-3,-2.9467333E-2,-2.0886665E-2,1.8273234E-2,-1.5819248E-2,-5.502399E-2,-2.1265732E-2,-6.5319347E-3,-1.619717E-2,5.085191E-4,-7.3756496E-3,-9.074098E-5,8.307703E-3,-4.7082524E-3,-1.3143787E-2,8.299324E-4,-1.2860364E-2,1.1243781E-3,7.1687093E-3,-1.1018637E-2,3.1134381E-3,4.2910194E-3,1.3457824E-2,3.6241282E-3,-9.885484E-3,1.7060831E-2,2.8550986E-2,-1.6461916E-2,-5.806728E-3,-7.150063E-5,7.2169453E-3,-4.514166E-3,2.5181393E-3,7.421129E-4,1.0002963E-2,2.068948E-2,5.72772E-3,1.0200773E-2,3.0876111E-2,4.7827423E-2,2.9035617E-2,5.393077E-3,1.7778432E-2,-2.5230604E-5,-1.3059594E-2],"split_indices":[2,43,52,9,2,67,60,68,60,52,68,32,66,10,48,0,69,2,12,2,5,4,73,67,43,67,47,4,3,0,15,2,0,9,0,0,0,0,3,0,0,68,53,5,43,53,48,9,4,43,53,43,0,0,0,9,48,0,0,68,5,0,10,0,0,46,48,43,2,1,0,0,0,67,0,7,68,46,0,2,12,7,43,9,12,47,0,67,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.09E2,4.72E2,4.37E2,1.75E2,2.97E2,3.76E2,6.1E1,1.35E2,4E1,4.9E1,2.48E2,2.65E2,1.11E2,3.8E1,2.3E1,3E0,1.32E2,3E1,1E1,3.5E1,1.4E1,1.89E2,5.9E1,1.47E2,1.18E2,7.6E1,3.5E1,8E0,3E1,1E1,1.3E1,1.18E2,1.4E1,2.2E1,8E0,4E0,6E0,4E0,3.1E1,1.2E1,2E0,7.9E1,1.1E2,6E0,5.3E1,9.9E1,4.8E1,3.2E1,8.6E1,3.9E1,3.7E1,3E1,5E0,2E0,6E0,2.4E1,6E0,3E0,1E1,7.3E1,4.5E1,4E0,1E1,1.7E1,5E0,1.2E1,1.9E1,3E1,4.9E1,1.4E1,9.6E1,2E0,4E0,4.8E1,5E0,8.8E1,1.1E1,4.5E1,3E0,2.6E1,6E0,5.8E1,2.8E1,1.9E1,2E1,3.3E1,4E0,2.1E1,9E0,1.7E1,7E0,4E0,2E0,1.5E1,5.8E1,2E0,4.3E1,5E0,5E0,4E0,8E0,2E0,1.7E1,2.2E1,8E0,3E1,1.9E1,7E0,7E0,6E1,3.6E1,4.6E1,2E0,2E1,6.8E1,6E0,5E0,2.8E1,1.7E1,8E0,1.8E1,3E0,3E0,2.3E1,3.5E1,6E0,2.2E1,9E0,1E1,2E0,1.8E1,2.5E1,8E0,7E0,1.4E1,1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[5.1417276E-3,-1.1659712E-1,3.6427525E-1,-3.6130655E-1,-1.399637E-2,-4.5374048E-1,4.129066E-1,-5.0729156E-1,-2.7119958E-1,-1.105341E-1,8.453608E-2,-2.3256678E-2,-5.451013E-3,5.9599716E-2,3.836584E-1,-4.2136505E-1,-7.5646126E-1,-3.8824478E-1,-1.7128299E-1,-1.6660531E-1,9.069275E-2,1.6977485E-1,-2.0604759E-5,3.045682E-1,6.4963734E-1,-4.4124338E-1,5.1172534E-3,-1.5929397E-2,-8.4401363E-1,-5.362956E-1,-3.3152726E-1,3.8437095E-1,-2.3603979E-1,-2.928878E-1,-9.580947E-2,-1.4559181E-2,1.2612337E-1,8.742477E-2,2.7844334E-1,1.0341518E-2,-2.1108434E-2,9.571727E-2,3.8171324E-1,6.791809E-1,-1.8419437E-3,-9.967559E-3,-2.1107465E-2,-9.237613E-1,-1.0096905E-2,-2.833691E-2,-8.521042E-3,-2.791076E-1,-2.1995462E-2,2.4382077E-2,5.2747237E-3,-1.6216797E-1,-2.4404153E-2,-3.6510583E-2,-2.5963017E-1,-2.9795936E-1,-4.676607E-2,4.4499654E-3,-2.9460466E-3,-6.7455317E-3,1.4330766E-1,1.1753472E-1,-2.0895828E-1,3.194487E-1,-9.600697E-2,-1.1731696E-1,5.1004022E-2,-3.1916446E-3,2.2303787E-1,4.767489E-1,2.0005465E-1,5.7027495E-1,8.442285E-1,-2.9471064E-2,-5.3789362E-2,-1.4247522E-2,-2.2763072E-3,-9.907472E-3,5.3859553E-3,-1.3449336E-2,1.824936E-3,-6.01569E-3,-1.5517001E-2,1.8699529E-3,-4.593495E-3,5.1355134E-3,1.2140465E-2,-5.0357957E-3,7.0016375E-3,5.2898577E-3,-3.14921E-2,1.1126641E-2,2.1621658E-2,-1.510184E-2,4.628802E-3,-7.6971394E-3,-1.9948652E-3,4.5808083E-5,8.702139E-3,3.924487E-3,-4.501679E-3,1.5201451E-2,6.8728817E-3,2.3361294E-2,-8.0825563E-4,4.0097022E-3,2.0740863E-2,1.3956279E-2,2.7818784E-2,4.164073E-2,1.4286803E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,47,49,51,53,55,57,59,61,63,65,67,-1,69,71,73,75,-1,-1,-1,77,-1,-1,-1,79,-1,-1,-1,81,-1,-1,83,85,87,-1,-1,-1,89,91,93,95,97,99,101,103,105,107,109,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.162067E1,1.7846285E1,9.774622E0,2.6248188E0,4.784418E0,1.596322E-1,5.3006706E0,1.3726673E0,1.4772081E0,2.8809996E0,1.7946727E0,0E0,0E0,0E0,4.472206E0,6.929083E-1,3.9951706E-1,3.2602882E-1,2.71744E0,1.7510543E0,2.1066818E-1,1.0853155E0,5.955791E-1,2.7508087E0,1.0450344E0,7.1513176E-2,0E0,0E0,5.493698E-1,2.7499104E-1,2.0899582E-1,2.4376953E-1,1.3224499E0,9.583082E-1,1.2649866E0,8.673546E-2,2.1649623E-1,6.670936E-1,8.444567E-1,0E0,8.107723E-1,6.0518277E-1,2.079893E0,3.8134003E-1,0E0,0E0,0E0,1.4503384E-1,0E0,0E0,0E0,2.1071935E-1,0E0,0E0,0E0,7.8813374E-1,0E0,0E0,6.120138E-1,1.07266426E-1,4.788713E-1,0E0,0E0,0E0,1.2604111E-1,5.083264E-1,1.1345931E0,4.4876242E-1,3.0064425E-1,1.7220968E-1,4.559523E-1,2.3769276E-1,1.15778685E-1,1.0444908E0,1.2212958E0,6.156254E-2,1.9170666E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,47,47,51,51,55,55,58,58,59,59,60,60,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,48,50,52,54,56,58,60,62,64,66,68,-1,70,72,74,76,-1,-1,-1,78,-1,-1,-1,80,-1,-1,-1,82,-1,-1,84,86,88,-1,-1,-1,90,92,94,96,98,100,102,104,106,108,110,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.7471867E5,1E0,3.741267E7,9.24E2,1.7026364E3,3.3817584E7,1E0,3.95E2,2.331083E6,3.7105262E0,-2.3256678E-2,-5.451013E-3,5.9599716E-2,4.391553E6,4.2259482E2,1.4E1,3.89E2,3E1,1E0,3.0479442E9,4.2708197E2,4.5795207E0,7.606269E2,6.292039E0,2.3E1,5.1172534E-3,-1.5929397E-2,1.753E3,1.4473684E-1,1.737736E1,2.04115E5,2.1609572E7,1.21E2,1.561132E2,3.863E3,2.2953334E2,4.5158855E6,2.48334E5,1.0341518E-2,5.714731E5,6.7148806E5,6.9E1,1.6471942E3,-1.8419437E-3,-9.967559E-3,-2.1107465E-2,2E0,-1.0096905E-2,-2.833691E-2,-8.521042E-3,3.0416667E0,-2.1995462E-2,2.4382077E-2,5.2747237E-3,8.788122E4,-2.4404153E-2,-3.6510583E-2,2.0734694E1,2.8E1,2.0348837E0,4.4499654E-3,-2.9460466E-3,-6.7455317E-3,3.083671E2,4.0449125E5,4.4E2,1.399844E8,2E0,7.5982756E5,2.9308079E1,8.998703E-1,7.714286E0,3.2E1,7.802E-2,1.57922E5,7.3274844E9,-2.9471064E-2,-5.3789362E-2,-1.4247522E-2,-2.2763072E-3,-9.907472E-3,5.3859553E-3,-1.3449336E-2,1.824936E-3,-6.01569E-3,-1.5517001E-2,1.8699529E-3,-4.593495E-3,5.1355134E-3,1.2140465E-2,-5.0357957E-3,7.0016375E-3,5.2898577E-3,-3.14921E-2,1.1126641E-2,2.1621658E-2,-1.510184E-2,4.628802E-3,-7.6971394E-3,-1.9948652E-3,4.5808083E-5,8.702139E-3,3.924487E-3,-4.501679E-3,1.5201451E-2,6.8728817E-3,2.3361294E-2,-8.0825563E-4,4.0097022E-3,2.0740863E-2,1.3956279E-2,2.7818784E-2,4.164073E-2,1.4286803E-2],"split_indices":[2,43,17,7,2,4,7,6,2,43,69,0,0,0,43,67,3,1,0,29,46,67,73,67,53,44,0,0,44,71,73,5,60,48,4,12,48,62,1,0,66,43,8,67,0,0,0,6,0,0,0,71,0,0,0,48,0,0,73,10,68,0,0,0,4,43,10,5,6,43,71,57,71,3,72,1,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.5E2,7.1E2,2.4E2,2.09E2,5.01E2,1.3E1,2.27E2,7.8E1,1.31E2,2.53E2,2.48E2,1.1E1,2E0,6E0,2.21E2,6E1,1.8E1,5.9E1,7.2E1,1.98E2,5.5E1,1.23E2,1.25E2,1.72E2,4.9E1,5.8E1,2E0,4E0,1.4E1,1.4E1,4.5E1,7E0,6.5E1,7E1,1.28E2,1.4E1,4.1E1,7.1E1,5.2E1,1E1,1.15E2,4.7E1,1.25E2,4.7E1,2E0,4E0,5.4E1,1.2E1,2E0,1.1E1,3E0,3.5E1,1E1,4E0,3E0,5.3E1,1.2E1,3E0,6.7E1,2.4E1,1.04E2,4E0,1E1,2E0,3.9E1,6.5E1,6E0,4.7E1,5E0,4.9E1,6.6E1,2.7E1,2E1,8.1E1,4.4E1,3.1E1,1.6E1,7E0,5E0,3.1E1,4E0,4.5E1,8E0,6.1E1,6E0,5E0,1.9E1,3.9E1,6.5E1,3.2E1,7E0,8E0,5.7E1,4E0,2E0,3.2E1,1.5E1,2E0,3E0,2.9E1,2E1,4.9E1,1.7E1,1.4E1,1.3E1,7E0,1.3E1,7.7E1,4E0,3.1E1,1.3E1,4E0,2.7E1,1.4E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[3.366158E-3,-2.0402244E-1,1.5598854E-1,-3.4113404E-1,-9.796814E-2,4.0414985E-2,3.7560987E-1,-3.6472085E-1,3.6828938E-1,-1.670867E-1,9.258877E-2,7.050098E-2,-2.5164095E-1,4.3494835E-1,2.2852607E-1,7.505379E-2,-3.8455403E-1,2.965387E-2,-5.282004E-3,-1.5051824E-1,-3.6843117E-2,5.4568052E-2,3.705691E-2,-1.3137171E-2,1.4481376E-1,-1.8722774E-1,-3.875773E-2,4.814612E-1,7.3452905E-2,1.6154867E-1,5.137695E-1,-1.2491759E-2,1.5578317E-2,-3.3541098E-1,-5.1780933E-1,-1.9998616E-1,-4.8795026E-2,-7.990302E-2,8.0897495E-2,-4.943764E-2,1.810127E-1,1.7702273E-1,-9.0612456E-2,-2.5494075E-1,-8.679474E-2,4.9979967E-1,6.50123E-2,1.6975565E-1,-1.7236428E-1,2.0268023E-1,-9.656717E-2,9.020064E-3,2.6485192E-2,-3.7926176E-1,1.2960242E-1,-3.7034124E-2,-4.261653E-1,-2.1776307E-1,8.761831E-2,5.923746E-2,-1.371956E-1,1.8782906E-3,-6.3820174E-3,1.079062E-1,-7.101042E-2,-3.8298205E-2,-2.3291675E-2,2.882457E-1,9.745653E-2,2.2557692E-1,8.211246E-2,1.0856261E-2,-1.5758786E-1,-1.2928276E-2,-4.874005E-3,-2.5346428E-2,-1.4122322E-2,4.407047E-1,6.830944E-1,1.617982E-2,-7.599519E-3,-1.4264943E-3,1.2733944E-2,-8.4044E-4,-1.2519968E-2,1.4941815E-1,2.9092282E-1,-1.0252544E-2,2.3931107E-3,-1.8818531E-2,-6.5321173E-3,2.5926532E-2,-9.047175E-3,-1.1582115E-2,-2.6041297E-2,-8.435216E-3,-1.5573964E-2,-3.368695E-3,7.726098E-3,-5.8408355E-4,7.82201E-3,-7.6672877E-3,3.899434E-3,1.1781956E-2,3.3291283E-3,4.0300936E-3,-6.4184098E-3,-2.616771E-3,8.173648E-3,5.2141417E-3,1.6171359E-2,2.7308422E-3,1.0828683E-2,1.1499641E-2,-6.890867E-3,-2.3428411E-3,5.932822E-3,7.780322E-4,-1.2558797E-2,-3.9828173E-3,2.2890186E-3,9.118865E-3,2.2043185E-2,4.0900465E-2,2.1672927E-2,3.3173691E-3,8.890235E-3,1.5038568E-2,2.9862635E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,39,41,43,-1,45,47,49,51,-1,-1,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,-1,-1,89,91,-1,93,95,97,99,101,-1,-1,103,105,107,-1,109,111,113,115,-1,117,-1,-1,119,-1,121,123,-1,-1,-1,-1,-1,-1,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9341707E1,5.681301E0,1.3539033E1,2.9956684E0,2.9526083E0,3.1019049E0,1.506504E0,1.4767361E0,8.510794E-1,1.5793257E0,1.5629817E0,1.9846098E0,1.0805025E0,2.1504784E0,9.6776843E-1,8.045123E-1,8.675442E-1,0E0,0E0,8.0039954E-1,0E0,2.1181577E-1,0E0,1.0731562E0,1.2975397E0,1.8869174E-1,0E0,8.45768E-1,4.080257E-1,5.0794065E-1,6.772828E-2,0E0,0E0,2.475707E0,7.07057E-1,5.712533E-1,5.2383435E-1,7.449496E-2,2.0870146E-1,6.2441456E-1,1.8694812E-1,6.6811275E-1,4.9254602E-1,3.313756E-2,1.7886433E-1,8.9045525E-1,4.4453958E-1,2.5864777E-1,6.984286E-2,1.445303E-1,1.3970539E-1,0E0,0E0,6.036024E-1,1.6339906E0,0E0,6.424794E-1,3.7393808E-1,9.998949E-2,1.9766828E-1,1.9727856E-1,0E0,0E0,2.0372996E-1,9.646418E-2,4.9446452E-1,0E0,6.57599E-2,6.4994365E-2,8.309603E-1,3.1957805E-1,0E0,3.57352E-1,0E0,0E0,5.8162395E-2,0E0,6.069088E-1,6.5575314E-1,0E0,0E0,0E0,0E0,0E0,0E0,7.436597E-2,8.218372E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,21,21,23,23,24,24,25,25,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,53,53,54,54,56,56,57,57,58,58,59,59,60,60,63,63,64,64,65,65,67,67,68,68,69,69,70,70,72,72,75,75,77,77,78,78,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,40,42,44,-1,46,48,50,52,-1,-1,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,-1,-1,90,92,-1,94,96,98,100,102,-1,-1,104,106,108,-1,110,112,114,116,-1,118,-1,-1,120,-1,122,124,-1,-1,-1,-1,-1,-1,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.095006E5,8.147158E4,3.068E3,1.665909E3,1.639E3,1.3767015E1,8.7E1,6E0,1.5E1,8.773336E6,5.428706E3,1.007E3,1.83154E6,2E0,3.8537518E2,4.54E2,1E0,2.965387E-2,-5.282004E-3,3.328629E6,-3.6843117E-2,4.6712E4,3.705691E-2,4.82425E6,2.2968E4,4.88E2,-3.875773E-2,1.8258025E2,1.8265169E0,1.8258025E2,6.176636E5,-1.2491759E-2,1.5578317E-2,4.0953247E2,1E0,9.7899484E4,2.6666667E0,1.8439855E5,5.444797E0,4.1E1,5.894636E6,3.8610423E0,8.138348E1,1.5115256E6,1.72797E5,1.9856038E3,4.88E2,1.5031847E0,1.0322473E8,1.8062708E6,3.313E3,9.020064E-3,2.6485192E-2,1.0433326E8,2.3E1,-3.7034124E-2,4.3919165E8,2.347364E7,7.24E2,3.056087E2,5.2405543E0,1.8782906E-3,-6.3820174E-3,1.7985714E1,1.441215E5,9.750085E6,-2.3291675E-2,1.27493895E-5,3.774872E7,1.1048219E3,3.0839285E2,1.0856261E-2,5.49E2,-1.2928276E-2,-4.874005E-3,5.893737E2,-1.4122322E-2,8.484036E5,1.4676277E7,1.617982E-2,-7.599519E-3,-1.4264943E-3,1.2733944E-2,-8.4044E-4,-1.2519968E-2,8.023379E4,1.803948E7,-1.0252544E-2,2.3931107E-3,-1.8818531E-2,-6.5321173E-3,2.5926532E-2,-9.047175E-3,-1.1582115E-2,-2.6041297E-2,-8.435216E-3,-1.5573964E-2,-3.368695E-3,7.726098E-3,-5.8408355E-4,7.82201E-3,-7.6672877E-3,3.899434E-3,1.1781956E-2,3.3291283E-3,4.0300936E-3,-6.4184098E-3,-2.616771E-3,8.173648E-3,5.2141417E-3,1.6171359E-2,2.7308422E-3,1.0828683E-2,1.1499641E-2,-6.890867E-3,-2.3428411E-3,5.932822E-3,7.780322E-4,-1.2558797E-2,-3.9828173E-3,2.2890186E-3,9.118865E-3,2.2043185E-2,4.0900465E-2,2.1672927E-2,3.3173691E-3,8.890235E-3,1.5038568E-2,2.9862635E-3],"split_indices":[43,43,2,4,2,61,8,3,0,60,67,2,9,6,71,1,6,0,0,9,0,1,0,43,44,4,0,73,68,73,43,0,0,67,26,48,68,43,68,3,60,69,62,43,1,67,0,68,5,43,0,0,0,7,0,0,46,5,2,4,68,0,0,73,43,9,0,53,7,67,67,0,0,0,0,4,0,43,62,0,0,0,0,0,0,48,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.25E2,3.92E2,5.33E2,1.7E2,2.22E2,3.5E2,1.83E2,1.65E2,5E0,1.63E2,5.9E1,3.18E2,3.2E1,1.29E2,5.4E1,7E0,1.58E2,3E0,2E0,1.6E2,3E0,5.7E1,2E0,1.5E2,1.68E2,3E1,2E0,1.14E2,1.5E1,4.5E1,9E0,3E0,4E0,1.18E2,4E1,1.07E2,5.3E1,9E0,4.8E1,1.27E2,2.3E1,1.48E2,2E1,1.7E1,1.3E1,1.09E2,5E0,1.1E1,4E0,3.9E1,6E0,2E0,7E0,1.08E2,1E1,8E0,3.2E1,1.01E2,6E0,2.4E1,2.9E1,3E0,6E0,4.1E1,7E0,1.25E2,2E0,9E0,1.4E1,9.7E1,5.1E1,3E0,1.7E1,1.4E1,3E0,1.1E1,2E0,8.5E1,2.4E1,2E0,3E0,4E0,7E0,2E0,2E0,2.6E1,1.3E1,3E0,3E0,9.7E1,1.1E1,4E0,6E0,1.5E1,1.7E1,7.9E1,2.2E1,2E0,4E0,1.5E1,9E0,2.6E1,3E0,7E0,3.4E1,2E0,5E0,1.16E2,9E0,3E0,6E0,1.2E1,2E0,9.2E1,5E0,1.3E1,3.8E1,7E0,1E1,6E0,5E0,1.1E1,7.4E1,1.1E1,1.3E1,1E1,1.6E1,1.1E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[5.0563575E-3,-1.3041122E-1,2.4400784E-1,-3.3149803E-1,-2.928897E-2,-4.4558033E-1,2.88639E-1,-3.6554506E-1,-2.3880769E-2,-2.0763688E-1,1.0615267E-2,-4.874457E-3,-2.2134563E-2,1.6199921E-1,4.4939992E-1,8.666788E-2,-3.8434365E-1,5.4305557E-2,-9.989612E-2,-1.1970194E-1,-3.821831E-1,-3.700844E-1,2.7555332E-2,3.6660707E-1,6.903889E-2,5.4233547E-2,4.0860543E-1,-2.65149E-1,2.3475721E-2,-3.6746395E-1,-7.3213947E-1,-1.4580585E-3,6.0883616E-3,-6.345336E-3,2.0009456E-3,-1.4427778E-1,3.933016E-3,-1.2301684E-2,-4.4734234E-1,-2.9847244E-2,-8.6097464E-2,1.02301665E-1,-3.6360987E-2,7.2666146E-2,4.0042666E-1,8.945606E-2,-1.6315367E-2,5.3748405E-1,1.8832286E-1,-2.2334678E-3,-1.8318927E-2,-4.1138834E-1,-2.312311E-1,-7.2018523E-3,-4.3348935E-2,-1.8825358E-1,-5.280578E-2,-1.5733093E-2,-2.9550819E-2,-1.8905684E-4,-1.02997115E-2,1.3803433E-1,-5.8975697E-2,-1.6213588E-1,-7.1440265E-3,-1.1592936E-3,9.817717E-3,2.756323E-1,4.6201E-1,1.1001936E-1,-3.195832E-1,5.756011E-1,1.5588646E-1,2.306693E-1,-1.1505898E-2,-1.7265791E-2,-2.9977243E-2,-1.5900355E-2,-5.587584E-3,-9.654222E-3,-2.3293516E-3,-4.4440203E-3,4.6195914E-3,5.178441E-3,1.6141864E-2,-9.608877E-3,6.5388647E-4,-4.3774084E-3,-1.3181318E-2,9.563248E-4,-8.52096E-3,1.4536894E-2,-1.1887955E-3,7.201205E-3,2.2716485E-2,2.4303377E-3,1.0714262E-2,-2.35386E-2,1.6587201E-3,-1.6793231E-3,2.75614E-2,1.4469082E-2,-5.391345E-3,5.728185E-3,2.0056749E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,-1,51,53,-1,-1,-1,-1,55,-1,-1,57,-1,59,61,63,65,67,69,-1,71,73,-1,-1,75,77,-1,-1,79,81,-1,-1,-1,-1,83,85,87,89,-1,-1,91,93,95,97,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0330309E1,1.2150234E1,1.0562056E1,2.0859604E0,2.8418221E0,1.6733742E-1,6.4216347E0,1.5634861E0,1.3023551E-1,1.0747077E0,2.119046E0,0E0,0E0,3.398364E0,3.6168842E0,1.3106085E0,7.465954E-1,7.628369E-2,6.331618E-2,2.6320547E-1,6.569481E-2,1.024924E0,1.5038581E0,5.27596E-1,1.1029003E0,0E0,3.703001E0,1.2084988E-1,0E0,9.04192E-1,6.6363716E-1,0E0,0E0,0E0,0E0,1.7107415E-1,0E0,0E0,7.2158575E-2,0E0,8.793466E-2,8.437568E-1,6.235509E-1,9.9832945E-2,2.5691795E-1,1.0445042E0,0E0,1.1320839E0,9.9808097E-1,0E0,0E0,9.307232E-1,4.8750353E-1,0E0,0E0,6.97279E-2,1.1354977E-1,0E0,0E0,0E0,0E0,6.2526107E-1,2.9403126E-1,2.3729622E-1,6.8186617E-1,0E0,0E0,2.1937501E-1,1.8423033E-1,7.874582E-1,4.156615E-1,8.126888E-1,4.157936E-1,9.603803E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,30,30,35,35,38,38,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,51,51,52,52,55,55,56,56,61,61,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,73,73],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,-1,52,54,-1,-1,-1,-1,56,-1,-1,58,-1,60,62,64,66,68,70,-1,72,74,-1,-1,76,78,-1,-1,80,82,-1,-1,-1,-1,84,86,88,90,-1,-1,92,94,96,98,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.8988005E5,3.964269E3,7.997723E6,4.97E2,5.6E1,1.0879832E3,6E0,2.2941177E0,4.325E0,1.45064E5,-4.874457E-3,-2.2134563E-2,8E0,3.3817584E7,2.123E3,1E2,1.3871127E2,7.2140925E5,4.49239E6,4.9236734E5,1.9548485E2,3.2857144E0,5.338167E5,1.2784314E1,5.4233547E-2,8E0,1.9E1,2.3475721E-2,1.7813443E-1,4.68418E5,-1.4580585E-3,6.0883616E-3,-6.345336E-3,2.0009456E-3,4.26E2,3.933016E-3,-1.2301684E-2,6.18826E5,-2.9847244E-2,1.0203835E-5,8.485E3,2.61E2,1.32476E5,4.4762436E2,1.4676277E7,-1.6315367E-2,1E0,3.0070068E7,-2.2334678E-3,-1.8318927E-2,1.9032816E7,1.2469576E2,-7.2018523E-3,-4.3348935E-2,2.3043478E2,7.026624E7,-1.5733093E-2,-2.9550819E-2,-1.8905684E-4,-1.02997115E-2,8.0274E4,4.4380944E8,1.0926341E1,1E0,-1.1592936E-3,9.817717E-3,3.478E4,1.2003246E6,6.9706165E2,3.596E3,3.4285715E0,1.1E1,7.802E-2,-1.1505898E-2,-1.7265791E-2,-2.9977243E-2,-1.5900355E-2,-5.587584E-3,-9.654222E-3,-2.3293516E-3,-4.4440203E-3,4.6195914E-3,5.178441E-3,1.6141864E-2,-9.608877E-3,6.5388647E-4,-4.3774084E-3,-1.3181318E-2,9.563248E-4,-8.52096E-3,1.4536894E-2,-1.1887955E-3,7.201205E-3,2.2716485E-2,2.4303377E-3,1.0714262E-2,-2.35386E-2,1.6587201E-3,-1.6793231E-3,2.75614E-2,1.4469082E-2,-5.391345E-3,5.728185E-3,2.0056749E-2],"split_indices":[2,43,43,9,2,0,67,3,68,69,7,0,0,3,7,1,6,67,60,43,43,67,69,43,68,0,32,0,0,53,9,0,0,0,0,2,0,0,9,0,53,44,70,1,67,62,0,6,58,0,0,5,67,0,0,4,7,0,0,0,0,1,5,73,19,0,0,44,66,67,2,71,8,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.35E2,5.97E2,3.38E2,1.99E2,3.98E2,2E1,3.18E2,1.79E2,2E1,7.2E1,3.26E2,2E0,1.8E1,1.79E2,1.39E2,7E0,1.72E2,1E1,1E1,4.9E1,2.3E1,1.3E1,3.13E2,5.5E1,1.24E2,6E0,1.33E2,4E0,3E0,1.66E2,6E0,5E0,5E0,8E0,2E0,4.4E1,5E0,1E1,1.3E1,6E0,7E0,1.44E2,1.69E2,6E0,4.9E1,1.19E2,5E0,8.3E1,5E1,2E0,2E0,1.24E2,4.2E1,2E0,4E0,2.9E1,1.5E1,1E1,3E0,5E0,2E0,1.18E2,2.6E1,3.1E1,1.38E2,4E0,2E0,1.8E1,3.1E1,1.14E2,5E0,7.5E1,8E0,4.6E1,4E0,1.08E2,1.6E1,2E1,2.2E1,2.5E1,4E0,1.2E1,3E0,1.06E2,1.2E1,8E0,1.8E1,2.1E1,1E1,1.2E2,1.8E1,1.6E1,2E0,3E0,2.8E1,7.8E1,3.6E1,3E0,2E0,2E0,7.3E1,5E0,3E0,3.1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-1.09481765E-2,-1.1362508E-1,2.791597E-1,-3.2023048E-1,-2.2775633E-2,-4.0244478E-1,3.199672E-1,-4.4396633E-1,-2.4804686E-1,-8.6307555E-2,9.270064E-2,-2.3205118E-2,-1.289383E-1,4.1868472E-1,1.5324798E-1,-4.6043143E-1,4.8680925E-3,8.742022E-3,-2.962728E-1,-1.1783483E-1,6.241799E-2,1.2772514E-1,-1.5855305E-1,2.8544497E-3,-1.2848796E-2,5.4594916E-1,2.8737277E-1,7.5546436E-2,3.3981788E-1,-3.9908752E-1,-6.2569964E-1,-2.692489E-1,1.0634223E-1,-3.272004E-1,1.744408E-2,-1.7061499E-1,-2.2579666E-2,1.1482215E-1,-1.1168913E-1,1.6289617E-1,-5.1832184E-2,-2.3437332E-1,7.005983E-2,3.674634E-1,7.0003164E-1,9.319343E-2,3.7661624E-1,-2.046099E-1,1.1492287E-1,4.1397706E-1,-7.116221E-3,-4.2186925E-1,1.1824115E-4,-3.3653315E-2,-1.9201184E-2,-1.6063069E-3,-1.7578442E-2,-1.1791108E-2,2.0156726E-1,-3.9305532E-1,-2.1074106E-1,1.9235082E-1,-1.6037E-1,-6.1715585E-1,-1.554041E-1,-1.5259336E-1,2.1085473E-2,1.8568198E-1,3.5120316E-2,7.0739E-3,-2.4582991E-1,-1.7585635E-2,1.9572227E-1,-1.8435946E-1,4.8809968E-2,-1.518392E-1,-2.5179578E-2,-4.003855E-3,7.89036E-3,4.0003482E-1,6.988396E-4,3.7818383E-2,5.354361E-1,-3.903047E-2,1.6859928E-2,4.0461743E-1,-8.725174E-3,-2.1856523E-3,-1.1595054E-2,2.7082515E-1,6.979219E-2,4.8785937E-1,1.356477E-1,-1.7621927E-2,-2.7431047E-2,1.6846215E-2,3.3515966E-3,-1.3193665E-3,-1.9092552E-2,-2.1002125E-3,-1.402952E-2,1.1799E-2,2.1595384E-3,-1.00524975E-2,-1.5135092E-3,-3.604757E-2,-1.1789799E-2,-8.910486E-3,-8.273313E-4,-1.3439166E-2,-5.0541763E-3,1.9782924E-3,-1.2185048E-2,1.0514398E-2,1.8731387E-3,-4.1625155E-3,2.4625326E-3,-1.8448103E-2,1.9480115E-3,-3.2572087E-3,3.248301E-3,1.1319936E-2,5.2163466E-3,-4.145923E-3,-1.3058754E-2,-2.6013483E-3,1.0010664E-2,-1.0200846E-2,-1.1759679E-3,7.2491486E-3,2.0439032E-2,1.1456712E-2,2.6827076E-2,-1.0041035E-2,3.5844212E-3,-7.087839E-3,2.0126643E-2,1.681477E-2,-4.6966445E-5,9.666216E-3,1.7699555E-3,2.714758E-2,1.5709713E-2,-3.6536957E-4,9.736995E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,93,-1,-1,-1,-1,-1,-1,95,97,99,101,103,105,107,109,111,113,115,-1,117,119,121,123,125,127,-1,-1,-1,129,-1,-1,131,133,-1,135,-1,-1,-1,137,139,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.7346746E1,1.2718756E1,6.804266E0,1.7387371E0,3.4707954E0,3.0520654E-1,3.6718292E0,7.257004E-1,1.6496487E0,1.437473E0,1.4986573E0,0E0,1.5715623E-1,2.2161198E0,1.22105E0,4.9038124E-1,0E0,6.2553596E-1,1.0923891E0,1.2630982E0,5.0551695E-1,9.430952E-1,3.804235E-1,0E0,0E0,1.6953716E0,1.2126074E0,7.07471E-1,9.770403E-1,5.1346207E-1,7.635021E-2,1.4005235E-1,6.336754E-1,6.9944763E-1,3.72957E-1,1.0044413E0,5.198305E-1,2.306804E-1,4.973984E-1,7.380817E-1,3.4244436E-1,3.2324654E-1,1.0791909E-1,3.9576674E-1,2.5359535E-1,8.688231E-1,8.5005665E-1,4.4653058E-2,3.6807734E-1,3.8622236E-1,0E0,1.5895939E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.651726E-1,3.9611053E-1,5.735128E-1,4.1931406E-2,3.574501E-2,9.067917E-2,7.834332E-1,1.0635334E-1,4.3015712E-1,1.2012267E-1,5.1398054E-2,0E0,4.0720338E-1,9.660671E-2,3.910184E-1,7.5964004E-2,2.7354005E-1,1.1404386E-1,0E0,0E0,0E0,2.3967457E-1,0E0,0E0,7.331753E-2,3.6573613E-1,0E0,8.027382E-1,0E0,0E0,0E0,2.9630822E-1,1.8485314E-1,4.7394753E-2,6.5115914E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,72,72,73,73,74,74,75,75,79,79,82,82,83,83,85,85,89,89,90,90,91,91,92,92],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,94,-1,-1,-1,-1,-1,-1,96,98,100,102,104,106,108,110,112,114,116,-1,118,120,122,124,126,128,-1,-1,-1,130,-1,-1,132,134,-1,136,-1,-1,-1,138,140,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,1E0,7.7E1,1.221E3,1.5900173E3,8E0,1.5045853E0,2.2007043E0,2.331083E6,7.627907E0,-2.3205118E-2,1.91E2,2.0926375E5,2.3464934E3,1.0195397E8,4.8680925E-3,3.8575E4,3.3284525E2,1.852364E6,8E0,1.2706E4,2.9541414E1,2.8544497E-3,-1.2848796E-2,1.0089981E3,9.9811395E2,8.147158E4,6.6756144E7,4.5E1,5.1120926E4,1.0677966E0,4.54E2,1.295313E6,4.046E3,1.21E2,1.5341322E2,9.166947E6,1.4285715E0,1.2501388E6,1.862E3,4.9E1,1.99E2,1E0,8.025477E0,1E0,1.0534078E4,3.94E2,3.3382E4,2E1,-7.116221E-3,7.1E1,1.1824115E-4,-3.3653315E-2,-1.9201184E-2,-1.6063069E-3,-1.7578442E-2,-1.1791108E-2,2.00087E5,8E1,2.347826E0,1.89991E5,1E0,1.75E2,3.5467294E2,2.35E2,4.1E1,1.9785424E3,1.4189452E9,7.0739E-3,8.1E2,1.0617E4,3.7105262E0,9.6E1,2.5176924E1,2.6E1,-2.5179578E-2,-4.003855E-3,7.89036E-3,3.076124E5,6.988396E-4,3.7818383E-2,1.41565E3,4.454E3,1.6859928E-2,1.2593857E0,-8.725174E-3,-2.1856523E-3,-1.1595054E-2,1.584E3,6.354E3,2.1298597E0,7.6599895E9,-1.7621927E-2,-2.7431047E-2,1.6846215E-2,3.3515966E-3,-1.3193665E-3,-1.9092552E-2,-2.1002125E-3,-1.402952E-2,1.1799E-2,2.1595384E-3,-1.00524975E-2,-1.5135092E-3,-3.604757E-2,-1.1789799E-2,-8.910486E-3,-8.273313E-4,-1.3439166E-2,-5.0541763E-3,1.9782924E-3,-1.2185048E-2,1.0514398E-2,1.8731387E-3,-4.1625155E-3,2.4625326E-3,-1.8448103E-2,1.9480115E-3,-3.2572087E-3,3.248301E-3,1.1319936E-2,5.2163466E-3,-4.145923E-3,-1.3058754E-2,-2.6013483E-3,1.0010664E-2,-1.0200846E-2,-1.1759679E-3,7.2491486E-3,2.0439032E-2,1.1456712E-2,2.6827076E-2,-1.0041035E-2,3.5844212E-3,-7.087839E-3,2.0126643E-2,1.681477E-2,-4.6966445E-5,9.666216E-3,1.7699555E-3,2.714758E-2,1.5709713E-2,-3.6536957E-4,9.736995E-3],"split_indices":[2,43,17,44,2,70,32,53,69,43,69,0,0,48,4,46,0,9,67,9,8,44,71,0,0,4,4,43,59,6,48,68,1,9,44,48,67,60,71,60,2,3,0,6,71,28,4,0,9,3,0,44,0,0,0,0,0,0,5,44,68,9,29,0,67,2,8,48,46,0,2,10,69,8,71,3,0,0,0,43,0,0,4,2,0,68,0,0,0,0,10,53,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.16E2,6.77E2,2.39E2,2.06E2,4.71E2,1.3E1,2.26E2,7.4E1,1.32E2,3.04E2,1.67E2,9E0,4E0,1.41E2,8.5E1,7.2E1,2E0,2.1E1,1.11E2,2.51E2,5.3E1,1.47E2,2E1,2E0,2E0,7E1,7.1E1,6.1E1,2.4E1,5.5E1,1.7E1,5E0,1.6E1,1.01E2,1E1,1.61E2,9E1,4.1E1,1.2E1,1.23E2,2.4E1,1.5E1,5E0,3.4E1,3.6E1,2.3E1,4.8E1,7E0,5.4E1,2.1E1,3E0,5.2E1,3E0,1E1,7E0,2E0,3E0,3E0,1.3E1,6.3E1,3.8E1,5E0,5E0,4E0,1.57E2,2.2E1,6.8E1,2.1E1,2E1,4E0,8E0,1.9E1,1.04E2,1E1,1.4E1,1.3E1,2E0,2E0,3E0,3.1E1,3E0,1.9E1,1.7E1,1.6E1,7E0,4.6E1,2E0,2E0,5E0,1.1E1,4.3E1,1.6E1,5E0,4.4E1,8E0,5E0,8E0,3E0,6E1,1.4E1,2.4E1,3E0,2E0,3E0,2E0,2E0,2E0,1.24E2,3.3E1,4E0,1.8E1,6.4E1,4E0,1.6E1,5E0,2E0,1.8E1,5E0,3E0,1.2E1,7E0,6.5E1,3.9E1,6E0,4E0,9E0,5E0,8E0,5E0,5E0,2.6E1,3E0,1.4E1,6E0,1E1,2E0,4.4E1,8E0,3E0,7E0,3.6E1,8E0,8E0,2E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[1.09839225E-2,-8.29779E-2,2.9322398E-1,-2.819482E-1,-7.6662563E-3,-3.5167763E-1,3.2358423E-1,-3.6520168E-1,-2.0719089E-1,-4.290404E-1,2.2322003E-2,-1.9288572E-2,-2.1050891E-3,5.248825E-2,2.962509E-1,2.5294132E-3,-3.802954E-1,-9.0517394E-2,-2.702715E-1,-5.8891267E-1,-1.8727472E-1,7.5349905E-2,-8.354189E-2,2.5072843E-1,6.0899407E-1,-3.3982712E-1,-5.5188805E-1,-1.174917E-1,3.3916829E-3,-3.0673346E-1,-9.521938E-2,-9.392957E-3,-6.2068635E-1,-1.8524181E-2,-1.3846087E-2,1.0145441E-1,-1.539635E-1,-1.1160311E-1,2.4235061E-1,3.3146504E-2,3.114304E-1,6.8392986E-1,1.3821466E-2,-3.704926E-1,2.2321087E-3,-4.82279E-3,-6.1286336E-1,-1.4203317E-1,-2.5810148E-3,-2.4146655E-1,-3.809754E-1,2.9407025E-3,-6.0355994E-3,-3.0630669E-2,-9.836133E-3,-8.6408015E-3,3.9783604E-3,2.4168862E-2,1.8708639E-1,-2.7672458E-1,-4.894354E-2,-1.4300837E-1,6.529536E-2,1.0995775E-1,1.5818248E-2,-2.1194322E-2,2.7353692E-1,2.990204E-2,2.725758E-1,3.3732772E-2,1.5194091E-2,-1.9866228E-2,-1.2008397E-2,6.73801E-3,-3.897472E-3,-3.488292E-2,-1.8050829E-2,-7.5256587E-3,-3.5401277E-4,-1.233652E-2,3.5782696E-3,-1.4922968E-2,-2.7963871E-2,2.1250583E-3,-5.9949723E-3,2.448E-2,7.91181E-3,-7.444074E-3,-1.9767683E-2,-5.0189462E-3,3.1425208E-3,-7.878089E-3,2.466496E-4,6.8745622E-3,-3.697075E-3,7.597451E-3,-7.3005905E-4,-5.6772074E-3,1.5966094E-3,2.54269E-3,1.5690707E-2,1.373013E-2,-5.89208E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,-1,53,55,-1,57,59,61,63,65,67,69,-1,71,73,-1,75,77,-1,79,81,-1,-1,-1,-1,-1,-1,83,85,87,89,91,93,95,-1,97,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4846853E1,1.0532139E1,4.6953716E0,1.1255283E0,6.4667144E0,1.9378412E-1,4.594467E0,5.906677E-1,7.4095917E-1,1.203867E0,2.6896627E0,0E0,0E0,0E0,2.9509392E0,0E0,4.1658592E-1,1.761033E-1,3.9843655E-1,1.2053728E-1,2.740425E-1,1.9251907E0,1.4902377E0,2.525547E0,4.276123E-1,7.6655483E-1,3.4637356E-1,4.2325437E-2,0E0,1.7451191E-1,7.938878E-2,0E0,1.400795E-1,1.377429E-1,0E0,1.8948772E0,4.119041E-1,8.3155775E-1,1.3087589E-1,5.6750697E-1,1.7569036E0,7.352066E-2,0E0,3.1604576E-1,9.8050304E-2,0E0,1.0968208E-1,5.9929818E-2,0E0,2.628225E-1,1.2484765E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.066847E-1,6.933775E-1,1.9151485E-1,1.3784441E-1,4.93968E-1,2.86557E-1,4.9535826E-2,0E0,2.0583017E-1,9.3337595E-2,0E0,1.2671795E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,47,47,49,49,50,50,57,57,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,-1,54,56,-1,58,60,62,64,66,68,70,-1,72,74,-1,76,78,-1,80,82,-1,-1,-1,-1,-1,-1,84,86,88,90,92,94,96,-1,98,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,4.76E2,1E0,8.147158E4,3.964269E3,1.75E2,3.3817584E7,1.0204082E0,8.510249E-3,5.4225586E-2,8E0,-1.9288572E-2,-2.1050891E-3,5.248825E-2,6.663214E6,2.5294132E-3,5E1,4.49239E6,3.422351E6,3.13276E5,2.1609572E7,5.2409735E-7,7.262122E2,7.079871E2,1.9047619E0,2.95E2,3.0914426E2,3.6E2,3.3916829E-3,4.309091E0,1E1,-9.392957E-3,1.266059E3,6.8E1,-1.3846087E-2,1.017E3,7.582512E7,1.7022566E7,2.008E3,6.103178E8,1.84E2,1.1450447E9,1.3821466E-2,3.89E2,1.3E1,-4.82279E-3,2.443913E1,1.2E1,-2.5810148E-3,1.605076E6,2.0303884E6,2.9407025E-3,-6.0355994E-3,-3.0630669E-2,-9.836133E-3,-8.6408015E-3,3.9783604E-3,1.0292E4,2.04115E5,5.3E1,1.2728624E0,1.862E3,3.425E1,2.118835E6,1.5818248E-2,6.516E3,2.12E2,2.990204E-2,7.134432E7,3.3732772E-2,1.5194091E-2,-1.9866228E-2,-1.2008397E-2,6.73801E-3,-3.897472E-3,-3.488292E-2,-1.8050829E-2,-7.5256587E-3,-3.5401277E-4,-1.233652E-2,3.5782696E-3,-1.4922968E-2,-2.7963871E-2,2.1250583E-3,-5.9949723E-3,2.448E-2,7.91181E-3,-7.444074E-3,-1.9767683E-2,-5.0189462E-3,3.1425208E-3,-7.878089E-3,2.466496E-4,6.8745622E-3,-3.697075E-3,7.597451E-3,-7.3005905E-4,-5.6772074E-3,1.5966094E-3,2.54269E-3,1.5690707E-2,1.373013E-2,-5.89208E-3],"split_indices":[2,2,17,43,43,6,7,68,53,53,32,0,0,0,43,0,0,43,9,46,60,52,67,67,69,2,48,2,0,69,3,0,67,0,0,2,7,9,2,7,10,7,0,1,3,0,73,8,0,12,60,0,0,0,0,0,0,44,5,0,53,2,73,9,0,44,0,0,59,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.35E2,7.02E2,2.33E2,1.92E2,5.1E2,1E1,2.23E2,8.9E1,1.03E2,3.3E1,4.77E2,8E0,2E0,6E0,2.17E2,3E0,8.6E1,3.7E1,6.6E1,1.9E1,1.4E1,3.18E2,1.59E2,1.91E2,2.6E1,7.2E1,1.4E1,3.2E1,5E0,5.4E1,1.2E1,2E0,1.7E1,6E0,8E0,2.86E2,3.2E1,1.47E2,1.2E1,4.2E1,1.49E2,2E1,6E0,6.6E1,6E0,2E0,1.2E1,2.2E1,1E1,3.1E1,2.3E1,2E0,1E1,1.5E1,2E0,2E0,4E0,1.51E2,1.35E2,1.4E1,1.8E1,1.25E2,2.2E1,6E0,6E0,3.5E1,7E0,1.4E1,1.35E2,1.7E1,3E0,4.2E1,2.4E1,2E0,4E0,6E0,6E0,1.9E1,3E0,2.9E1,2E0,2E1,3E0,1.33E2,1.8E1,5E0,1.3E2,9E0,5E0,1.2E1,6E0,1.06E2,1.9E1,1.4E1,8E0,4E0,2E0,1.2E1,2.3E1,2E0,5E0,1.28E2,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.6440721E-2,-1.1168855E-1,2.4742243E-1,-2.909755E-1,-2.633379E-2,-4.1899312E-1,2.861949E-1,-4.4249678E-1,-2.3045638E-1,-5.060813E-2,2.7925906E-1,-2.8285747E-2,-2.7370834E-1,1.9458558E-1,5.4795784E-1,-3.8223767E-1,-7.752715E-1,-3.1356826E-1,-1.3881566E-1,-1.0356963E-1,5.025224E-2,4.8731882E-2,3.6475837E-1,-1.4663467E-2,-2.0398202E-3,1.450552E-1,4.0812203E-1,7.451971E-1,3.2072717E-1,-4.0022153E-1,-1.5472564E-1,-4.4585563E-2,-9.821384E-3,-3.6009932E-1,-1.4731862E-1,1.5894722E-2,-1.6712537E-1,-9.371126E-2,-3.0663421E-2,6.834658E-2,-2.6711532E-1,5.405288E-3,-3.0085284E-3,1.847746E-2,1.7242805E-3,2.7164329E-2,1.219122E-1,2.8013578E-2,3.2860428E-1,5.39649E-1,8.683083E-1,3.697067E-1,-8.7446E-3,-6.039981E-3,-4.104923E-1,-6.8000983E-4,-1.131318E-2,-3.186892E-1,-2.9684171E-2,-4.077283E-3,-1.269393E-2,-1.1436572E-1,-4.0613773E-1,-7.336677E-2,-3.5148045E-1,-8.562429E-2,1.0976654E-1,-2.6123556E-3,-1.5824627E-2,4.029766E-3,1.8543115E-1,3.4927028E-1,5.7713245E-4,6.1532396E-1,-2.038604E-3,4.3547675E-2,1.3672187E-2,5.582121E-1,2.7197605E-1,-2.1380935E-2,-1.4735153E-2,-1.9415874E-2,-1.2223461E-2,1.4353123E-3,-8.022769E-3,-2.1457851E-2,-9.027619E-3,-1.6287509E-3,-7.930531E-3,-1.9537464E-2,-1.0915175E-3,-1.1697098E-2,-5.242025E-4,6.1394717E-3,-5.4623764E-3,-5.5047515E-4,1.5128702E-2,1.00892885E-2,1.6690818E-4,8.542011E-3,1.7442374E-2,3.299858E-2,1.58152E-2,2.9382274E-2,1.2628451E-2,1.4173851E-2,-3.2583103E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,-1,-1,57,59,-1,61,63,-1,65,67,-1,-1,-1,-1,-1,69,-1,71,73,75,77,-1,-1,79,-1,-1,81,-1,-1,-1,83,85,87,89,91,93,-1,-1,95,97,99,-1,101,-1,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6545218E1,8.822166E0,8.388258E0,1.603631E0,2.922014E0,3.2832217E-1,7.1592903E0,7.4640274E-1,9.846902E-1,1.9495895E0,5.5381227E-1,0E0,1.00866616E-1,2.3310976E0,3.2568626E0,1.2628603E-1,5.21739E-1,4.8845434E-1,9.550568E-1,1.247648E0,7.4181855E-1,7.641336E-2,2.091999E-1,0E0,0E0,1.8084764E0,4.507203E-1,5.689354E-1,1.0164156E0,6.33111E-2,5.845961E-2,0E0,0E0,4.1861105E-1,1.03459686E-1,0E0,7.479949E-1,1.2118402E0,0E0,7.7469474E-1,8.889261E-2,0E0,0E0,0E0,0E0,0E0,1.3245881E0,0E0,2.0476937E-1,8.2176256E-1,6.627064E-1,4.929757E-1,0E0,0E0,3.880453E-2,0E0,0E0,1.6068268E-1,0E0,0E0,0E0,4.5282227E-1,3.9291143E-2,8.1362605E-1,3.7273145E-1,3.1785062E-1,4.9564862E-1,0E0,0E0,3.2497424E-1,6.581006E-1,5.915928E-2,0E0,1.73635E-1,0E0,0E0,0E0,6.042409E-2,3.0172145E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,37,37,39,39,40,40,46,46,48,48,49,49,50,50,51,51,54,54,57,57,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,71,71,73,73,77,77,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,-1,-1,58,60,-1,62,64,-1,66,68,-1,-1,-1,-1,-1,70,-1,72,74,76,78,-1,-1,80,-1,-1,82,-1,-1,-1,84,86,88,90,92,94,-1,-1,96,98,100,-1,102,-1,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.008E3,1.8988005E5,1E0,3.3817584E7,5.5426865E6,6.0546756E-2,1.6471942E3,5.035714E0,1.3428947E2,7.969174E7,3.3852024E7,-2.8285747E-2,1.75E2,3.9813398E6,1.7103828E7,3.047619E0,3.6255838E2,3.328629E6,1.6E1,5.9E1,1E0,9.380531E-1,1.1394967E7,-1.4663467E-2,-2.0398202E-3,3.284E4,1.3E0,1.9856038E3,4.1325716E7,4E0,3.83E2,-4.4585563E-2,-9.821384E-3,1.580025E2,3.477E4,1.5894722E-2,4.3264876E7,2E0,-3.0663421E-2,3.257507E5,1.5611285E0,5.405288E-3,-3.0085284E-3,1.847746E-2,1.7242805E-3,2.7164329E-2,4.1032645E6,2.8013578E-2,2E0,4.163636E0,1.0656093E2,2.047619E0,-8.7446E-3,-6.039981E-3,3.421543E2,-6.8000983E-4,-1.131318E-2,5.4E1,-2.9684171E-2,-4.077283E-3,-1.269393E-2,3.7326388E0,1.24141544E-1,2.7402837E0,2.8266037E2,1.212945E6,2.111111E1,-2.6123556E-3,-1.5824627E-2,9.7885155E1,6.3055553E0,7.575958E7,5.7713245E-4,3.361317E1,-2.038604E-3,4.3547675E-2,1.3672187E-2,2.171312E0,1.803948E7,-2.1380935E-2,-1.4735153E-2,-1.9415874E-2,-1.2223461E-2,1.4353123E-3,-8.022769E-3,-2.1457851E-2,-9.027619E-3,-1.6287509E-3,-7.930531E-3,-1.9537464E-2,-1.0915175E-3,-1.1697098E-2,-5.242025E-4,6.1394717E-3,-5.4623764E-3,-5.5047515E-4,1.5128702E-2,1.00892885E-2,1.6690818E-4,8.542011E-3,1.7442374E-2,3.299858E-2,1.58152E-2,2.9382274E-2,1.2628451E-2,1.4173851E-2,-3.2583103E-3],"split_indices":[2,43,17,7,43,53,67,68,67,7,7,0,6,43,62,68,4,9,10,3,19,68,60,0,0,1,69,67,66,0,12,0,0,70,1,0,5,6,0,43,68,0,0,0,0,0,60,0,6,68,73,68,0,0,48,0,0,2,0,0,0,69,53,68,4,9,61,0,0,73,69,7,0,71,0,0,0,57,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.95E2,5.76E2,3.19E2,1.85E2,3.91E2,1.7E1,3.02E2,5.1E1,1.34E2,3.63E2,2.8E1,6E0,1.1E1,2.25E2,7.7E1,4.5E1,6E0,6.9E1,6.5E1,2.38E2,1.25E2,8E0,2E1,9E0,2E0,1.84E2,4.1E1,4E1,3.7E1,4.1E1,4E0,4E0,2E0,5.3E1,1.6E1,3E0,6.2E1,2.35E2,3E0,1.19E2,6E0,5E0,3E0,1.8E1,2E0,8E0,1.76E2,1E1,3.1E1,1.7E1,2.3E1,3.4E1,3E0,2E0,3.9E1,2E0,2E0,4.8E1,5E0,1.2E1,4E0,5.2E1,1E1,2.19E2,1.6E1,2.5E1,9.4E1,2E0,4E0,6.2E1,1.14E2,2.9E1,2E0,1.5E1,2E0,2E1,3E0,1E1,2.4E1,2.3E1,1.6E1,1.5E1,3.3E1,1.5E1,3.7E1,7E0,3E0,1.58E2,6.1E1,1.3E1,3E0,7E0,1.8E1,8.6E1,8E0,6E1,2E0,9.7E1,1.7E1,5E0,2.4E1,1E1,5E0,7E0,3E0,2.2E1,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[-1.0276073E-2,-9.928892E-2,2.5801727E-1,-2.03517E-1,4.248803E-2,-3.6164916E-1,2.8602976E-1,-1.6213158E-1,-4.00463E-1,6.4039454E-2,-1.2272092E-1,-1.993767E-3,-2.0276617E-2,1.3604094E-1,4.16232E-1,-3.3982164E-1,-1.0802102E-1,-2.663728E-1,-5.623583E-1,-9.838436E-3,1.1303037E-1,-1.6620308E-1,4.6809562E-3,2.4956962E-1,5.2918132E-2,4.8548114E-1,2.2577377E-1,-2.0918576E-2,-2.7804172E-1,1.8509626E-2,-1.3114864E-1,-1.9668463E-1,-4.009735E-1,-3.946202E-1,-8.9126295E-1,2.1731962E-2,-1.671117E-1,-8.040706E-2,1.3817099E-1,-1.859254E-1,-6.4544125E-5,3.011352E-1,1.2492882E-1,1.6825361E-2,1.7620537E-2,5.2347505E-1,1.9265929E-1,1.5244944E-1,3.2598633E-2,-2.121552E-1,-1.8298842E-2,-2.6905105E-1,-8.775813E-2,-2.548335E-1,-2.6726898E-2,-2.285915E-2,-1.0627602E-2,-4.1214403E-2,-2.9623497E-1,-4.59734E-2,-1.7080164E-2,-1.5563868E-1,3.9645206E-2,-2.6325843E-1,-1.2110252E-3,-1.5829796E-1,6.460691E-2,1.7364852E-1,3.7299425E-3,-1.235647E-3,-2.0644818E-1,3.2463133E-1,5.7403133E-3,9.177699E-3,2.1372526E-3,1.5296972E-1,-8.0212325E-2,4.084927E-1,5.915214E-1,1.3266924E-2,6.794498E-2,1.8668629E-1,-5.0322358E-3,-1.2260214E-2,-3.747895E-3,-1.4301308E-2,-1.5844734E-3,-6.349796E-3,8.3858846E-4,-7.100243E-3,-1.3863984E-2,2.184117E-3,-7.6829093E-3,-1.980384E-2,-8.611292E-3,4.2137215E-3,-1.4952381E-2,3.318242E-3,-3.6420247E-3,-6.415951E-3,-1.6787881E-2,1.9093169E-3,-9.40726E-3,-2.913237E-3,8.171481E-3,1.0937852E-2,4.1053053E-3,8.938926E-3,-3.0218528E-3,-1.2339017E-2,-6.2918095E-3,9.329682E-3,1.7263427E-2,-4.7939745E-4,1.2030124E-2,-1.0805355E-2,-2.3476008E-5,2.0470597E-2,-1.0788499E-3,2.9183662E-2,1.2326772E-2,-6.247769E-4,7.1529485E-3,1.1846601E-3,1.2581994E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,-1,49,-1,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,75,77,79,81,-1,83,-1,85,87,89,91,-1,-1,-1,93,-1,-1,95,97,99,-1,101,103,105,107,-1,109,111,-1,-1,-1,113,115,117,119,-1,121,123,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1184675E1,9.850222E0,3.9396496E0,3.0649405E0,1.015477E0,2.182777E-1,4.083496E0,3.030447E0,1.2829695E0,9.095676E-1,3.3724475E-1,0E0,0E0,9.3047535E-1,1.3807545E0,3.7767696E-1,2.9311023E0,2.7757764E-1,1.2482147E0,5.0511473E-1,7.454438E-1,9.075707E-2,0E0,2.3232079E-1,6.368414E-1,8.114357E-1,1.0080438E0,0E0,2.9674935E-1,0E0,1.3885312E0,2.600261E-1,7.505894E-2,7.531016E-1,9.560633E-2,2.7623948E-1,2.2372702E-1,2.1354198E-1,6.384835E-1,7.6275945E-2,0E0,8.101964E-2,7.09891E-2,0E0,7.273164E-1,3.2170486E-1,1.0765782E-1,2.789932E-1,0E0,2.0896411E-1,0E0,4.9553728E-1,9.3638766E-1,4.2898893E-2,8.938839E-2,0E0,0E0,0E0,1.9463706E-1,0E0,0E0,3.5587317E-1,2.972738E-1,7.2639346E-2,0E0,1.1350393E-1,1.0996172E-1,5.361352E-1,3.8801444E-1,0E0,5.626279E-2,6.991625E-2,0E0,0E0,0E0,3.996125E-1,3.8789412E-1,4.2027473E-1,3.388071E-1,0E0,4.3987736E-2,3.4457177E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,28,28,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,44,44,45,45,46,46,47,47,49,49,51,51,52,52,53,53,54,54,58,58,61,61,62,62,63,63,65,65,66,66,67,67,68,68,70,70,71,71,75,75,76,76,77,77,78,78,80,80,81,81],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,-1,50,-1,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,76,78,80,82,-1,84,-1,86,88,90,92,-1,-1,-1,94,-1,-1,96,98,100,-1,102,104,106,108,-1,110,112,-1,-1,-1,114,116,118,120,-1,122,124,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,6.978874E5,1E0,2.1E1,6.1114804E-8,2.01E2,1.0879832E3,2.55E2,5.444797E0,4.7563504E7,2.0926375E5,-1.993767E-3,-2.0276617E-2,4.6816683E-1,8.7E1,3.6E1,2.74614E5,1.4381613E7,8.15792E5,5.8780793E-2,6.34E2,2.331083E6,4.6809562E-3,7.3E1,3.3382E4,4.9036694E0,1.0534078E4,-2.0918576E-2,7.714286E0,1.8509626E-2,8.147158E4,1.24272164E5,2.2003597E1,3.964269E3,9E1,1.45064E5,2.6750835E6,6.789622E7,6.354E3,1.4473684E-1,-6.4544125E-5,1.7172844E7,3.8488917E-2,1.6825361E-2,1.2E1,6.355E3,1.3E1,3.3571344E7,3.2598633E-2,1.43E2,-1.8298842E-2,7.997723E6,4.23025E2,1E0,3E1,-2.285915E-2,-1.0627602E-2,-4.1214403E-2,1.6254545E1,-4.59734E-2,-1.7080164E-2,1.1626786E2,2.7142856E0,1.6E1,-1.2110252E-3,7.3835544E5,1.724E0,3.0416667E0,3.3382E4,-1.235647E-3,7.549744E7,6.941074E6,5.7403133E-3,9.177699E-3,2.1372526E-3,7.233077E2,1.7083334E0,1.0657745E7,2.395631E7,1.3266924E-2,4E0,1.3388E4,-5.0322358E-3,-1.2260214E-2,-3.747895E-3,-1.4301308E-2,-1.5844734E-3,-6.349796E-3,8.3858846E-4,-7.100243E-3,-1.3863984E-2,2.184117E-3,-7.6829093E-3,-1.980384E-2,-8.611292E-3,4.2137215E-3,-1.4952381E-2,3.318242E-3,-3.6420247E-3,-6.415951E-3,-1.6787881E-2,1.9093169E-3,-9.40726E-3,-2.913237E-3,8.171481E-3,1.0937852E-2,4.1053053E-3,8.938926E-3,-3.0218528E-3,-1.2339017E-2,-6.2918095E-3,9.329682E-3,1.7263427E-2,-4.7939745E-4,1.2030124E-2,-1.0805355E-2,-2.3476008E-5,2.0470597E-2,-1.0788499E-3,2.9183662E-2,1.2326772E-2,-6.247769E-4,7.1529485E-3,1.1846601E-3,1.2581994E-2],"split_indices":[2,43,17,3,52,0,67,2,68,7,48,0,0,53,8,2,5,5,9,53,2,43,0,8,9,53,4,0,73,0,43,43,73,43,0,7,43,7,10,71,0,60,53,0,3,2,3,66,0,2,0,9,4,20,3,0,0,0,69,0,0,4,68,8,0,43,68,69,9,0,7,60,0,0,0,70,68,60,62,0,8,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.85E2,6.65E2,2.2E2,3.83E2,2.82E2,9E0,2.11E2,3.18E2,6.5E1,2.5E2,3.2E1,2E0,7E0,9.9E1,1.12E2,7.3E1,2.45E2,3.7E1,2.8E1,1E2,1.5E2,2.7E1,5E0,4.1E1,5.8E1,8.1E1,3.1E1,2.4E1,4.9E1,1E1,2.35E2,2.6E1,1.1E1,2E1,8E0,8.4E1,1.6E1,1.7E1,1.33E2,2.4E1,3E0,2.8E1,1.3E1,5E0,5.3E1,7.1E1,1E1,2.8E1,3E0,3.3E1,1.6E1,5.5E1,1.8E2,1.9E1,7E0,6E0,5E0,2E0,1.8E1,6E0,2E0,7E0,7.7E1,9E0,7E0,1.1E1,6E0,1.05E2,2.8E1,3E0,2.1E1,2.4E1,4E0,6E0,7E0,2.2E1,3.1E1,2.9E1,4.2E1,5E0,5E0,2.5E1,3E0,2.3E1,1E1,4.7E1,8E0,1.23E2,5.7E1,7E0,1.2E1,5E0,2E0,7E0,1.1E1,3E0,4E0,6.1E1,1.6E1,5E0,4E0,2E0,9E0,3E0,3E0,6E1,4.5E1,7E0,2.1E1,1E1,1.1E1,8E0,1.6E1,9E0,1.3E1,1E1,2.1E1,2.7E1,2E0,3.7E1,5E0,3E0,2E0,9E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[5.687187E-3,-8.715214E-2,2.7806887E-1,-1.927621E-1,1.294E-2,3.2831812E-1,-1.3608508E-1,-2.479754E-1,-7.623217E-2,-1.9093946E-2,1.4221498E-1,8.2757634E-1,2.8714758E-1,-3.5164857E-1,8.9198776E-2,1.3508148E-1,-2.6382756E-1,-5.9575778E-2,-1.9723639E-2,-1.523852E-1,2.8685007E-2,7.7787474E-2,2.8803387E-1,2.6045408E-2,4.767605E-2,1.4891359E-1,3.8428947E-1,2.6685093E-3,-4.6841744E-1,3.0315542E-1,-1.09163135E-1,2.0160137E-2,-1.3609813E-1,-3.0340105E-1,-1.6538605E-1,-1.519516E-1,-1.3831109E-2,-2.079608E-1,-3.3235494E-2,-4.7086004E-2,9.5717154E-2,4.4830292E-2,1.965524E-1,3.292367E-3,3.186093E-1,2.708073E-2,1.0255594E-1,5.266659E-1,3.073357E-1,-2.650354E-2,-1.0821953E-2,1.6597413E-2,5.2353656E-3,-8.025838E-3,2.5132415E-3,-9.572688E-3,1.0439606E-4,-2.6403624E-1,-4.3644577E-1,-2.1826163E-1,3.2466725E-2,-1.0593725E-1,-1.2939517E-2,8.7491255E-3,-2.7119863E-1,-1.6813563E-1,-4.4564977E-1,3.27516E-3,-1.0922903E-2,-1.1467812E-1,4.9895518E-2,2.3352776E-1,6.457403E-2,1.3278363E-2,1.5904532E-1,4.545974E-3,1.3746221E-2,3.4469336E-1,6.260882E-3,2.1491393E-1,5.1662344E-2,8.634468E-3,5.5939865E-1,3.466468E-1,3.2808375E-2,2.6936156E-3,-1.2666786E-2,-1.6183006E-2,-3.4446295E-2,-1.1748653E-2,2.0276096E-3,6.7716045E-3,-5.4707443E-3,-1.19290445E-2,-3.2628141E-3,-8.7100896E-4,7.224152E-3,-1.589313E-2,-4.0347413E-3,-8.718773E-3,4.026433E-3,-7.4886665E-4,-2.8567206E-2,4.775553E-3,-3.7713638E-3,-5.1364035E-4,-7.0356796E-3,1.5359147E-2,1.3098755E-3,7.016994E-3,1.7546017E-2,5.2777273E-3,-5.9039784E-3,2.6263292E-3,-6.2392973E-3,9.527356E-3,2.351793E-3,5.043384E-3,1.724025E-2,2.816304E-3,1.3408518E-2,7.525275E-4,1.3610587E-2,6.8252515E-3,2.6944416E-2,9.463934E-3,2.0437641E-2,1.1753042E-2,-3.778494E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,-1,-1,45,47,-1,49,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,-1,77,-1,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,85,87,89,91,93,-1,95,97,99,101,103,-1,105,107,109,111,113,115,-1,-1,117,-1,119,121,-1,123,125,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4174017E1,7.539423E0,5.101736E0,2.2136946E0,1.5225313E0,4.22274E0,1.3425667E0,1.4626265E0,6.294725E-1,1.8830835E0,6.6341114E-1,2.715721E-1,2.6552925E0,6.9553864E-1,6.2726533E-1,8.6819375E-1,8.302431E-1,4.5849982E-1,0E0,5.0959015E-1,1.1117494E0,1.9472468E-1,1.23574734E-1,0E0,0E0,1.6479455E0,1.1309929E0,0E0,1.3047338E-1,3.149426E-2,9.211162E-2,0E0,5.834406E-2,7.227278E-1,7.0801187E-1,1.7969775E-1,4.3241775E-1,4.2676902E-1,1.9216946E-1,6.809062E-1,4.827751E-1,1.4924374E-1,7.872671E-2,0E0,4.4781446E-2,0E0,4.3492472E-1,3.426304E-1,8.411913E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.5342407E-1,6.8437576E-1,4.878304E-1,2.7184844E-1,1.328443E-1,0E0,2.808236E-1,4.8410058E-2,2.4768996E-1,4.9705756E-1,2.0122142E-1,0E0,2.29716E-1,2.5266832E-1,1.9256544E-1,9.136089E-1,2.2409149E-1,3.4365654E-2,0E0,0E0,5.809748E-2,0E0,2.5443745E-1,4.6555525E-1,0E0,2.1143627E-1,8.2723045E-1,2.964196E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,25,25,26,26,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,46,46,47,47,48,48,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,69,69,70,70,71,71,72,72,73,73,74,74,77,77,79,79,80,80,82,82,83,83,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,-1,-1,46,48,-1,50,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,-1,78,-1,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,86,88,90,92,94,-1,96,98,100,102,104,-1,106,108,110,112,114,116,-1,-1,118,-1,120,122,-1,124,126,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,4.4761734E5,1E0,1.339646E6,3.3486558E6,2.4481E4,1.4692029E-7,6E0,1.2784314E1,1E0,5.4007E7,5.1191E4,1.1251919E6,1.7234043E1,1.81E2,1.3050649E4,1.972052E5,1.3209375E2,-1.9723639E-2,1.737736E1,6.789622E7,3.1316666E2,5.88E2,2.6045408E-2,4.767605E-2,3.6E1,1.6954315E0,2.6685093E-3,7E0,1.1E1,5.071E3,2.0160137E-2,4.54E2,4.88451E5,3.478493E2,3.7890625E0,2.0205908E7,2.1931148E6,2.3043E5,5.751617E6,1.0164831E-2,1E0,4.5866325E6,3.292367E-3,1.4793489E9,2.708073E-2,9.5139277E-1,2.707947E8,2.2040408E7,-2.650354E-2,-1.0821953E-2,1.6597413E-2,5.2353656E-3,-8.025838E-3,2.5132415E-3,-9.572688E-3,1.0439606E-4,1.0204082E0,3.9045289E0,1.1840488E8,2.8474576E0,1.9902479E4,-1.2939517E-2,9.422379E4,1.2856612E0,9.67E2,1E0,1.2694215E1,-1.0922903E-2,3.429012E7,2.1230527E2,4.4149057E2,5.941442E6,6.5590655E6,8.785115E6,4.545974E-3,1.3746221E-2,7.778E3,6.260882E-3,8.877906E6,1E0,8.634468E-3,4.3024124E2,1.0089981E3,4.4889745E6,2.6936156E-3,-1.2666786E-2,-1.6183006E-2,-3.4446295E-2,-1.1748653E-2,2.0276096E-3,6.7716045E-3,-5.4707443E-3,-1.19290445E-2,-3.2628141E-3,-8.7100896E-4,7.224152E-3,-1.589313E-2,-4.0347413E-3,-8.718773E-3,4.026433E-3,-7.4886665E-4,-2.8567206E-2,4.775553E-3,-3.7713638E-3,-5.1364035E-4,-7.0356796E-3,1.5359147E-2,1.3098755E-3,7.016994E-3,1.7546017E-2,5.2777273E-3,-5.9039784E-3,2.6263292E-3,-6.2392973E-3,9.527356E-3,2.351793E-3,5.043384E-3,1.724025E-2,2.816304E-3,1.3408518E-2,7.525275E-4,1.3610587E-2,6.8252515E-3,2.6944416E-2,9.463934E-3,2.0437641E-2,1.1753042E-2,-3.778494E-3],"split_indices":[2,43,6,9,43,12,52,3,68,29,7,1,43,71,0,43,43,67,0,73,7,4,2,0,0,0,69,0,6,8,2,0,1,9,67,68,12,43,1,60,53,16,43,0,5,0,53,7,62,0,0,0,0,0,0,0,0,68,68,7,68,43,0,48,53,0,8,61,0,7,67,4,60,43,43,0,0,12,0,60,19,0,4,4,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.54E2,7.12E2,2.42E2,3.46E2,3.66E2,2.16E2,2.6E1,2.34E2,1.12E2,2.94E2,7.2E1,1.5E1,2.01E2,1.3E1,1.3E1,9E0,2.25E2,1.08E2,4E0,7.7E1,2.17E2,5.1E1,2.1E1,8E0,7E0,8.4E1,1.17E2,3E0,1E1,6E0,7E0,4E0,5E0,1.59E2,6.6E1,3.5E1,7.3E1,5.2E1,2.5E1,1.02E2,1.15E2,4.1E1,1E1,3E0,1.8E1,7E0,7.7E1,3.9E1,7.8E1,6E0,4E0,4E0,2E0,5E0,2E0,3E0,2E0,1.25E2,3.4E1,5.2E1,1.4E1,2.7E1,8E0,6.8E1,5E0,4.6E1,6E0,2.2E1,3E0,6E1,4.2E1,2E1,9.5E1,3.3E1,8E0,6E0,4E0,1.5E1,3E0,2.3E1,5.4E1,4E0,3.5E1,6.8E1,1E1,3E0,1.22E2,2.8E1,6E0,4.6E1,6E0,8E0,6E0,4E0,2.3E1,5.8E1,1E1,3E0,2E0,4.3E1,3E0,2E0,4E0,1E1,1.2E1,1.6E1,4.4E1,2E0,4E1,1.4E1,6E0,7.6E1,1.9E1,2.6E1,7E0,5E0,3E0,2E0,1.3E1,8E0,1.5E1,4.8E1,6E0,2E0,3.3E1,2.8E1,4E1,3E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[-4.028764E-3,-9.431744E-2,2.4294275E-1,-2.1104772E-1,1.3920052E-2,6.843001E-1,2.1025914E-1,-3.5570645E-1,-1.661476E-1,-2.8147548E-2,1.423298E-1,4.8956951E-1,4.4633843E-2,2.534416E-1,-6.4367026E-2,-2.8125122E-1,-4.3914428E-1,-1.2348406E-1,-2.976515E-1,-2.073294E-2,-3.190791E-2,-3.0701289E-2,1.9292867E-1,2.4896534E-2,8.086323E-3,1.783838E-1,3.896125E-1,8.904591E-2,-2.1399367E-1,-3.0193657E-1,-9.9921346E-2,-4.6970525E-1,2.018796E-3,-2.378284E-1,-4.6854045E-2,-4.6705055E-1,-2.1773712E-1,-4.6090566E-2,1.0056155E-1,-7.983227E-2,9.211452E-3,1.4398752E-1,3.4604704E-1,-2.838135E-1,2.0226659E-1,9.975643E-2,4.242858E-1,-3.9760126E-3,2.1977143E-1,5.0897216E-3,-3.219296E-1,-3.1374744E-1,-2.586915E-3,-9.228936E-3,-4.8946396E-5,-2.2730745E-2,-5.40901E-3,-2.0526361E-1,-3.848628E-1,3.4595308E-1,-8.6428344E-2,-4.9745306E-1,-6.468561E-3,-1.1771334E-1,-3.124103E-1,-8.248235E-2,7.288071E-2,1.2343955E-1,-8.196348E-3,1.1346854E-3,-1.11355014E-1,1.8804738E-1,3.7030913E-2,1.8425953E-2,4.4908905E-3,4.6386863E-3,-2.7737772E-2,2.4518427E-2,2.564291E-1,6.708923E-3,-1.735865E-3,4.4492847E-1,1.5875076E-4,2.6641098E-1,1.0297787E-3,-1.7273089E-2,-1.5447221E-3,-7.5623323E-3,-1.5708197E-2,-1.1003286E-2,-3.8590548E-3,-2.0694816E-2,-6.434828E-3,2.234857E-2,7.1487343E-3,-7.247666E-3,1.0286544E-3,-7.54677E-3,-2.4710692E-2,-1.0132469E-2,-2.314847E-3,-1.938861E-2,-1.096926E-2,-3.2543787E-3,-2.2613129E-2,1.6751544E-3,1.1584647E-2,6.040647E-5,8.676964E-3,-6.98756E-3,-1.6026468E-3,3.897575E-3,1.1411259E-2,-4.8421845E-3,4.1145096E-3,-1.3966714E-3,8.608648E-3,1.4692792E-2,4.483912E-3,1.1417727E-2,2.1683397E-2,1.4757792E-2,3.5247558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,33,35,37,-1,39,41,-1,-1,43,45,47,49,51,53,55,-1,57,59,61,63,65,67,69,-1,71,73,75,77,79,81,-1,83,-1,85,87,-1,-1,-1,-1,-1,89,91,93,95,97,-1,99,101,103,105,107,-1,-1,109,111,113,-1,-1,-1,-1,115,117,-1,-1,119,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1072918E1,8.746077E0,3.505684E0,2.1037989E0,1.9486867E0,4.418807E-1,2.8269043E0,3.6105728E-1,1.3995128E0,1.3126758E0,7.8513396E-1,8.025098E-2,0E0,2.0257397E0,7.765591E-1,1.3486814E-1,5.3654337E-1,1.6974871E0,7.376261E-1,8.350848E-1,0E0,2.5184003E-1,4.6724534E-1,0E0,0E0,1.5365739E0,6.812954E-1,4.066695E-1,6.2870085E-1,9.600592E-2,5.823341E-2,1.6530275E-1,0E0,2.94477E-1,1.8642178E0,1.2025881E-1,3.7988615E-1,9.758365E-1,3.193865E-1,6.1857983E-2,0E0,2.4751103E-1,1.6191065E-1,8.9878094E-1,1.2260056E0,6.028258E-2,5.5450916E-1,0E0,8.6274445E-2,0E0,1.7335534E-1,7.934046E-2,0E0,0E0,0E0,0E0,0E0,2.3321128E-1,1.3107085E-1,2.1073377E-1,8.171412E-1,1.06794834E-1,0E0,1.4565772E-1,7.763028E-2,8.345156E-1,3.3528638E-1,3.3903795E-1,0E0,0E0,3.6689535E-2,2.0647776E-1,1.326383E-1,0E0,0E0,0E0,0E0,2.7711117E-1,9.024949E-1,0E0,0E0,1.4471531E-1,0E0,5.3586066E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,48,48,50,50,51,51,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,70,70,71,71,72,72,77,77,78,78,81,81,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,34,36,38,-1,40,42,-1,-1,44,46,48,50,52,54,56,-1,58,60,62,64,66,68,70,-1,72,74,76,78,80,82,-1,84,-1,86,88,-1,-1,-1,-1,-1,90,92,94,96,98,-1,100,102,104,106,108,-1,-1,110,112,114,-1,-1,-1,-1,116,118,-1,-1,120,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.688E3,4.3827028E5,4.70127E5,8.8E1,4.2708197E2,5.4567E5,1E0,9.55188E5,2E1,1.0948052E1,1E0,1.94261E5,4.4633843E-2,2.8530578E6,2.2092846E7,4.019139E0,1E0,4.13E2,6.593462E4,9.750085E6,-3.190791E-2,6.6414773E-1,1.7068776E10,2.4896534E-2,8.086323E-3,1.2825651E0,1.0950326E6,1.0089981E3,1.5209424E1,1.1408248E0,2.2E1,8.9044176E-2,2.018796E-3,6.642857E0,4.427818E7,1.775373E1,8.6136844E5,3.3486558E6,1.11E2,1.512E4,9.211452E-3,2.5357144E0,6.808571E2,1.3E1,2.0134516E6,8.392765E4,6.292039E0,-3.9760126E-3,1.214874E2,5.0897216E-3,1.75E2,1.957E3,-2.586915E-3,-9.228936E-3,-4.8946396E-5,-2.2730745E-2,-5.40901E-3,1.7142857E0,2.511352E6,2.04115E5,1.345672E6,2.7E1,-6.468561E-3,3.2216358E-1,2.4123944E2,4.5158855E6,3.1517188E2,1.088E3,-8.196348E-3,1.1346854E-3,1.5382514E1,5.281628E2,9.1470585E0,1.8425953E-2,4.4908905E-3,4.6386863E-3,-2.7737772E-2,7.262122E2,4.05E2,6.708923E-3,-1.735865E-3,1.57922E5,1.5875076E-4,9.408E3,1.0297787E-3,-1.7273089E-2,-1.5447221E-3,-7.5623323E-3,-1.5708197E-2,-1.1003286E-2,-3.8590548E-3,-2.0694816E-2,-6.434828E-3,2.234857E-2,7.1487343E-3,-7.247666E-3,1.0286544E-3,-7.54677E-3,-2.4710692E-2,-1.0132469E-2,-2.314847E-3,-1.938861E-2,-1.096926E-2,-3.2543787E-3,-2.2613129E-2,1.6751544E-3,1.1584647E-2,6.040647E-5,8.676964E-3,-6.98756E-3,-1.6026468E-3,3.897575E-3,1.1411259E-2,-4.8421845E-3,4.1145096E-3,-1.3966714E-3,8.608648E-3,1.4692792E-2,4.483912E-3,1.1417727E-2,2.1683397E-2,1.4757792E-2,3.5247558E-3],"split_indices":[2,43,12,44,67,1,6,46,8,69,29,1,0,43,60,68,16,2,43,9,0,53,46,0,0,69,62,4,71,53,5,57,0,69,46,71,60,43,8,9,0,68,4,3,60,48,53,0,73,0,6,9,0,0,0,0,0,71,5,5,9,8,0,53,67,62,70,44,0,0,71,4,71,0,0,0,0,67,8,0,0,1,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.43E2,6.91E2,2.52E2,3.32E2,3.59E2,1.6E1,2.36E2,7.7E1,2.55E2,2.71E2,8.8E1,1.1E1,5E0,2.04E2,3.2E1,4.3E1,3.4E1,1.94E2,6.1E1,2.69E2,2E0,2E1,6.8E1,9E0,2E0,1.33E2,7.1E1,1.6E1,1.6E1,3.8E1,5E0,3.2E1,2E0,7.7E1,1.17E2,1.8E1,4.3E1,2.23E2,4.6E1,1.7E1,3E0,5.3E1,1.5E1,6E0,1.27E2,8E0,6.3E1,7E0,9E0,4E0,1.2E1,3.6E1,2E0,2E0,3E0,3E1,2E0,6.5E1,1.2E1,1E1,1.07E2,1.6E1,2E0,2.2E1,2.1E1,1.71E2,5.2E1,4.3E1,3E0,4E0,1.3E1,3.7E1,1.6E1,1.2E1,3E0,3E0,3E0,3E1,9.7E1,6E0,2E0,6E1,3E0,7E0,2E0,1E1,2E0,6E0,3E1,5.1E1,1.4E1,9E0,3E0,5E0,5E0,6.5E1,4.2E1,2E0,1.4E1,8E0,1.4E1,7E0,1.4E1,1.67E2,4E0,4.4E1,8E0,1.5E1,2.8E1,8E0,5E0,1.4E1,2.3E1,4E0,1.2E1,2.3E1,7E0,7E1,2.7E1,7E0,5.3E1,5E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[1.586624E-2,-7.8804255E-2,2.1534237E-1,-1.7695104E-1,2.4234308E-2,2.707883E-1,-9.8284915E-2,-3.528975E-1,-1.3348615E-1,4.4538427E-2,-2.9295388E-1,9.7095974E-2,3.5598335E-1,4.9195476E-2,-3.1404522E-1,-2.7430195E-1,-4.249695E-1,-1.733559E-1,-2.3640264E-2,3.0333571E-2,3.5925242E-1,-3.305845E-2,-1.22791186E-1,1.7802903E-1,-2.2398593E-2,3.229693E-1,4.133003E-2,1.0895497E-2,-2.7295554E-2,-1.4203854E-1,-3.8821203E-1,-3.0465713E-1,6.6732155E-4,-4.609611E-1,-3.5245237E-3,-5.5092994E-2,-2.1829061E-1,2.9515833E-2,-1.2892462E-1,-1.344146E-1,5.3939946E-2,5.556683E-3,1.8382547E-2,3.277236E-3,-8.459912E-3,6.750492E-2,2.3463543E-1,1.4966653E-1,-1.00328505E-1,4.0846717E-1,1.7146486E-1,-1.2074193E-1,1.0033246E-1,-8.387539E-4,-8.7449625E-3,-2.02711E-2,-8.554427E-3,-1.5400177E-2,-1.1134812E-1,-4.9708837E-1,-1.1704448E-2,-2.8417894E-1,6.429672E-2,-5.4792476E-1,-1.95629E-1,1.0026984E-2,-9.411878E-3,-1.6289489E-1,3.4023968E-3,-1.728887E-1,-8.332259E-3,-4.4911984E-2,8.281461E-2,1.1316359E-2,1.6119795E-2,2.9517463E-1,1.09138854E-1,-3.2342682E-3,9.585523E-3,-4.245955E-2,-1.2410954E-2,3.3780584E-1,4.765072E-1,1.2707144E-1,2.4882909E-2,-1.6877206E-1,3.7022717E-3,-3.6772472E-4,8.766919E-3,-8.824187E-3,1.933018E-4,-2.4293656E-2,-7.5770416E-3,-1.8882181E-2,-7.894461E-3,1.2587418E-2,5.706219E-4,-8.191909E-3,-2.9880913E-2,-6.8399864E-3,-1.3613125E-2,1.4841963E-3,-9.435671E-3,-8.638875E-3,1.3276051E-3,-8.6749215E-3,-2.8612727E-4,-4.353199E-3,3.6556758E-3,4.305013E-3,-5.6826463E-3,2.5217044E-3,9.030681E-3,2.8103285E-3,-2.0067955E-3,1.9256845E-2,9.587874E-3,-4.4631303E-4,6.224834E-3,-4.9352064E-4,-9.709758E-3,7.597789E-3,1.8794134E-2,2.333027E-2,1.0089289E-2,1.9843362E-2,3.7088308E-3,-1.0175383E-2,9.7042456E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,43,45,47,49,-1,-1,51,53,55,57,-1,59,-1,61,63,65,67,69,71,-1,-1,-1,-1,73,75,77,79,81,83,85,87,-1,-1,-1,-1,-1,89,91,-1,93,95,97,99,-1,101,103,-1,105,107,109,111,-1,113,115,117,-1,-1,119,-1,121,123,125,-1,127,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6899715E1,6.1428204E0,5.034752E0,2.3312626E0,1.925795E0,3.5908775E0,1.415548E0,2.2896719E-1,1.0951633E0,1.2408334E0,1.1926079E0,7.970145E-1,2.5971298E0,3.8862872E-1,1.744883E-1,2.8475666E-1,3.420987E-1,9.657693E-1,3.8436767E-1,1.0534201E0,5.6376457E-2,0E0,1.727543E-1,2.907231E-1,4.709225E-1,1.9506779E0,0E0,0E0,2.5064185E-1,3.653334E-2,3.664446E-2,1.11549854E-1,0E0,7.9051495E-2,0E0,1.4404981E0,8.8632584E-1,3.352435E-1,1.698074E-1,1.6349018E-1,6.773382E-1,0E0,0E0,0E0,0E0,1.5869743E-1,2.1090388E-1,1.504276E-1,2.1995565E-1,3.1500626E-1,8.6812496E-1,1.289079E-1,8.738421E-2,0E0,0E0,0E0,0E0,0E0,4.605969E-2,1.00679874E-1,0E0,1.8201435E-1,3.7244478E-1,1.6810918E-1,5.508895E-1,0E0,3.182889E-1,9.281814E-2,0E0,5.7976723E-2,7.4074626E-2,5.849889E-1,5.721799E-1,0E0,4.1998815E-2,1.3687575E-1,3.6487237E-2,0E0,0E0,9.842559E-2,0E0,5.3540754E-1,2.0436668E-1,7.17428E-1,0E0,9.9417925E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,28,28,29,29,30,30,31,31,33,33,35,35,36,36,37,37,38,38,39,39,40,40,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,58,58,59,59,61,61,62,62,63,63,64,64,66,66,67,67,69,69,70,70,71,71,72,72,74,74,75,75,76,76,79,79,81,81,82,82,83,83,85,85],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,44,46,48,50,-1,-1,52,54,56,58,-1,60,-1,62,64,66,68,70,72,-1,-1,-1,-1,74,76,78,80,82,84,86,88,-1,-1,-1,-1,-1,90,92,-1,94,96,98,100,-1,102,104,-1,106,108,110,112,-1,114,116,118,-1,-1,120,-1,122,124,126,-1,128,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.239E3,5.095006E5,1E0,7.7E1,4.2173915E0,6.327361E2,3.0161016E7,1.139605E6,8.57E2,7.82261E6,1E0,1.3E1,7.626E3,2E0,5.17259E8,2.0376764E-7,7.6588124E-2,1.88E0,8.303E3,5.02E2,5.02E2,-3.305845E-2,3.7777777E0,3.221296E6,4.4971E4,8E0,4.133003E-2,1.0895497E-2,5.071E3,1.81E2,4.6E1,3.1269147E0,6.6732155E-4,2.911362E4,-3.5245237E-3,2.25E2,1.4268688E-5,3.261058E0,1.3205625E7,2.1701321E2,1E0,5.556683E-3,1.8382547E-2,3.277236E-3,-8.459912E-3,2.2307692E0,8E0,2.5393645E5,4.55E2,1.3E1,2.9152632E0,3.91157E5,3.714144E8,-8.387539E-4,-8.7449625E-3,-2.02711E-2,-8.554427E-3,-1.5400177E-2,3.4E1,1.731E3,-1.1704448E-2,1.1176471E1,1.2E1,1.3E2,1.3689265E1,1.0026984E-2,3.5017543E0,3.1158695E6,3.4023968E-3,4.1973075E6,2.4E1,1.3E1,5.163891E2,1.1316359E-2,7.383931E4,2.821E3,3.493E3,-3.2342682E-3,9.585523E-3,4.4522205E6,-1.2410954E-2,1.9791039E6,2.395631E7,1.2083E4,2.4882909E-2,4.5E1,3.7022717E-3,-3.6772472E-4,8.766919E-3,-8.824187E-3,1.933018E-4,-2.4293656E-2,-7.5770416E-3,-1.8882181E-2,-7.894461E-3,1.2587418E-2,5.706219E-4,-8.191909E-3,-2.9880913E-2,-6.8399864E-3,-1.3613125E-2,1.4841963E-3,-9.435671E-3,-8.638875E-3,1.3276051E-3,-8.6749215E-3,-2.8612727E-4,-4.353199E-3,3.6556758E-3,4.305013E-3,-5.6826463E-3,2.5217044E-3,9.030681E-3,2.8103285E-3,-2.0067955E-3,1.9256845E-2,9.587874E-3,-4.4631303E-4,6.224834E-3,-4.9352064E-4,-9.709758E-3,7.597789E-3,1.8794134E-2,2.333027E-2,1.0089289E-2,1.9843362E-2,3.7088308E-3,-1.0175383E-2,9.7042456E-4],"split_indices":[2,43,6,44,68,67,60,46,2,43,29,3,67,8,46,52,57,68,44,2,2,0,73,60,9,32,0,0,2,0,6,68,0,43,0,2,53,73,9,67,29,0,0,0,0,71,3,43,8,3,50,1,7,0,0,0,0,0,2,2,0,67,10,2,73,0,68,60,0,43,0,3,70,0,48,10,44,0,0,62,0,43,62,9,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.93E2,6.06E2,2.87E2,3.1E2,2.96E2,2.44E2,4.3E1,6E1,2.5E2,2.79E2,1.7E1,8.1E1,1.63E2,2.6E1,1.7E1,3.1E1,2.9E1,1.83E2,6.7E1,2.68E2,1.1E1,4E0,1.3E1,4.8E1,3.3E1,1.55E2,8E0,7E0,1.9E1,6E0,1.1E1,2.8E1,3E0,2.6E1,3E0,5.1E1,1.32E2,4.5E1,2.2E1,3.3E1,2.35E2,2E0,9E0,3E0,1E1,1.7E1,3.1E1,1E1,2.3E1,9.8E1,5.7E1,1.1E1,8E0,2E0,4E0,8E0,3E0,2.4E1,4E0,2.1E1,5E0,1.7E1,3.4E1,7E0,1.25E2,7E0,3.8E1,1.9E1,3E0,2.5E1,8E0,5.3E1,1.82E2,3E0,1.4E1,2E1,1.1E1,2E0,8E0,1.8E1,5E0,5.1E1,4.7E1,5.2E1,5E0,9E0,2E0,4E0,4E0,2E0,2E0,1.9E1,2E0,7E0,1E1,6E0,2.8E1,2E0,5E0,8.5E1,4E1,3.2E1,6E0,1.7E1,2E0,2.3E1,2E0,4E0,4E0,1.9E1,3.4E1,1.46E2,3.6E1,8E0,6E0,7E0,1.3E1,2E0,9E0,1.6E1,2E0,1.5E1,3.6E1,4.2E1,5E0,6E0,4.6E1,7E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"129","size_leaf_vector":"1"}},{"base_weights":[4.281981E-3,-8.3699845E-2,2.0976919E-1,-2.5735563E-1,-3.2579884E-2,2.648122E-1,-1.4339106E-1,-3.5761464E-1,-2.1384616E-1,-1.4423805E-1,9.419434E-3,1.410374E-1,3.6501402E-1,-3.3122835E-1,-4.395738E-2,-2.5444735E-2,-3.12099E-1,-1.16500996E-1,-2.588213E-1,1.3139167E-1,-1.8438053E-1,-9.842707E-3,1.7543425E-1,4.086724E-2,2.7230725E-1,4.6693644E-1,2.3099245E-1,-2.3969136E-2,-1.9211799E-1,5.3022366E-2,-2.044272E-1,-3.2798094E-1,-6.7370194E-3,-4.515657E-2,-1.8938589E-1,-1.2742852E-1,-2.8607437E-1,2.0799795E-1,-8.777362E-3,-2.2706307E-1,5.4461557E-2,3.2281227E-2,-8.50826E-2,2.8723305E-1,7.125269E-2,1.0212809E-1,-7.7942885E-2,2.990086E-1,-1.01843225E-2,3.4953138E-1,6.473329E-1,-6.1640954E-3,2.5283414E-1,-1.6126044E-3,-1.0796666E-2,-6.4254636E-3,1.1526166E-2,-6.1548557E-2,-1.5579858E-2,-4.518386E-3,-1.5800439E-2,1.2472091E-3,-3.7409838E-3,-1.0534365E-2,-2.5574376E-3,-8.248978E-3,5.3394237E-3,-3.554958E-3,-2.921964E-1,-4.7217458E-4,2.4234053E-1,-3.0981368E-1,-1.06316574E-1,-2.5770005E-2,6.0109627E-3,4.1123167E-2,-2.7647246E-2,1.4398718E-1,-1.123734E-1,2.0015305E-1,1.9906832E-2,1.1058152E-1,-7.671584E-2,1.3586245E-2,6.855679E-2,-1.9922802E-1,2.1348976E-2,3.1138974E-1,4.383537E-3,4.543772E-1,1.17994055E-1,3.1172816E-2,1.0823825E-2,2.9484186E-1,3.0114556E-2,2.3328227E-3,-5.502377E-3,4.617039E-3,-7.7564903E-3,-1.4011019E-2,-6.4860815E-3,1.2615794E-2,2.5816616E-3,-1.5306326E-2,7.7486266E-3,4.745239E-3,-6.960459E-3,-5.682982E-3,4.4225776E-3,1.0547562E-3,9.965845E-3,9.787451E-3,-2.0652456E-4,-8.98798E-3,-2.8101748E-3,1.0529015E-2,1.7310203E-3,5.0227554E-4,7.0346063E-3,1.6197643E-3,-7.5662103E-3,-5.4861093E-3,4.115165E-3,-1.3755562E-3,-1.1066182E-2,-1.3867234E-3,5.0776135E-3,1.0394008E-2,1.711488E-2,-4.4391085E-3,2.2984691E-2,8.410285E-3,-7.852107E-4,1.7539011E-2,9.314399E-3,1.0780461E-2,-6.649459E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,-1,61,63,65,67,69,-1,71,73,75,77,79,81,83,85,87,-1,89,91,-1,93,-1,-1,95,-1,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,99,-1,101,103,105,107,-1,109,-1,111,113,115,-1,117,119,-1,121,123,125,127,-1,129,131,-1,-1,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6704603E1,5.735214E0,5.422576E0,5.647707E-1,2.3508317E0,2.9251099E0,6.951794E-1,2.0012188E-1,4.2954874E-1,1.5400577E0,1.168924E0,1.4221475E0,1.6952343E0,2.3401463E-1,4.1545957E-1,0E0,5.0765038E-2,1.7344561E-1,2.1778727E-1,4.8067486E-1,1.235692E0,1.0423099E0,4.2213404E-1,4.6649992E-1,6.820276E-1,1.3410149E0,5.0198007E-1,0E0,4.86736E-2,1.9847181E-1,1.6593659E-1,5.6557655E-2,0E0,5.0404865E-2,7.235986E-2,1.8972501E-1,5.210209E-2,1.1483836E-1,0E0,9.862304E-1,1.1733355E-1,1.1937397E0,7.53857E-1,1.3502502E-1,1.3069135E-1,2.4787378E-1,2.691911E-1,8.472967E-2,0E0,1.0873451E0,1.1032486E-1,0E0,5.150535E-1,0E0,0E0,9.457027E-2,0E0,1.18098274E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.553343E-2,0E0,5.5189013E-2,6.065607E-1,4.036088E-1,1.2816249E-1,0E0,6.626387E-1,0E0,1.2917578E-1,4.337268E-1,4.7154367E-2,0E0,6.736413E-2,5.361422E-2,0E0,1.4795734E-1,5.864376E-2,6.1675843E-2,1.2675333E-1,0E0,7.4807787E-1,1.3867061E-1,0E0,0E0,2.9016733E-1,3.822401E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,52,52,55,55,57,57,68,68,70,70,71,71,72,72,73,73,75,75,77,77,78,78,79,79,81,81,82,82,84,84,85,85,86,86,87,87,89,89,90,90,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,-1,62,64,66,68,70,-1,72,74,76,78,80,82,84,86,88,-1,90,92,-1,94,-1,-1,96,-1,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,100,-1,102,104,106,108,-1,110,-1,112,114,116,-1,118,120,-1,122,124,126,128,-1,130,132,-1,-1,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.56E3,3.95E2,2E0,3.89E2,1E0,1.1048219E3,1E0,1.1649485E0,3.9E2,1E0,5.8222644E2,1.8062708E6,1.7103828E7,2.5330253E-2,1E1,-2.5444735E-2,5.513889E0,1.45E2,2.90099E0,7.0858893E0,2.6224005E6,2E0,2.8474576E0,3.6363637E0,1.1708007E7,1.798658E3,2.0750147E8,-2.3969136E-2,1.91E2,5.854E3,1.7951853E1,2.1126761E0,-6.7370194E-3,3.2226672E7,8.267672E1,1.766298E2,6.123E3,8.249158E-1,-8.777362E-3,6.001692E-7,2.0734E4,2.483E3,3.261058E0,1.57E2,1.9E1,5E0,5.0607144E2,7.7191065E6,-1.01843225E-2,8E0,2.53E2,-6.1640954E-3,4.1325716E7,-1.6126044E-3,-1.0796666E-2,2.6750835E6,1.1526166E-2,2E1,-1.5579858E-2,-4.518386E-3,-1.5800439E-2,1.2472091E-3,-3.7409838E-3,-1.0534365E-2,-2.5574376E-3,-8.248978E-3,5.3394237E-3,-3.554958E-3,7.997723E6,-4.7217458E-4,6E0,2.284E3,1.9010988E0,3.3852024E7,6.0109627E-3,7.6606E4,-2.7647246E-2,3.125E0,7.9E2,1.15E2,1.9906832E-2,1.609E3,1.651E3,1.3586245E-2,5.4468E4,1.70595E5,6.68523E4,5.696981E2,4.383537E-3,3.4285715E0,8.598131E1,3.1172816E-2,1.0823825E-2,2.3584905E0,4.88E2,2.3328227E-3,-5.502377E-3,4.617039E-3,-7.7564903E-3,-1.4011019E-2,-6.4860815E-3,1.2615794E-2,2.5816616E-3,-1.5306326E-2,7.7486266E-3,4.745239E-3,-6.960459E-3,-5.682982E-3,4.4225776E-3,1.0547562E-3,9.965845E-3,9.787451E-3,-2.0652456E-4,-8.98798E-3,-2.8101748E-3,1.0529015E-2,1.7310203E-3,5.0227554E-4,7.0346063E-3,1.6197643E-3,-7.5662103E-3,-5.4861093E-3,4.115165E-3,-1.3755562E-3,-1.1066182E-2,-1.3867234E-3,5.0776135E-3,1.0394008E-2,1.711488E-2,-4.4391085E-3,2.2984691E-2,8.410285E-3,-7.852107E-4,1.7539011E-2,9.314399E-3,1.0780461E-2,-6.649459E-3],"split_indices":[2,2,6,1,29,67,17,68,12,21,67,43,62,53,8,0,69,44,69,71,43,32,68,68,62,67,7,0,0,2,71,69,0,7,62,4,43,68,0,52,44,2,73,0,3,3,67,62,0,32,8,0,66,0,0,43,0,6,0,0,0,0,0,0,0,0,0,0,9,0,8,0,68,7,0,1,0,71,2,0,0,2,2,0,1,1,48,67,0,71,73,0,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.22E2,6.46E2,2.76E2,1.46E2,5E2,2.39E2,3.7E1,4.2E1,1.04E2,1.36E2,3.64E2,1.08E2,1.31E2,1.2E1,2.5E1,6E0,3.6E1,3.4E1,7E1,1.7E1,1.19E2,3.27E2,3.7E1,6.2E1,4.6E1,7.3E1,5.8E1,4E0,8E0,1.6E1,9E0,3.2E1,4E0,1.8E1,1.6E1,1.3E1,5.7E1,1.4E1,3E0,1.01E2,1.8E1,2.1E2,1.17E2,1.7E1,2E1,4.1E1,2.1E1,4.4E1,2E0,4.6E1,2.7E1,3E0,5.5E1,2E0,6E0,1.3E1,3E0,5E0,4E0,2E0,3E1,6E0,1.2E1,1.2E1,4E0,1.1E1,2E0,2E0,5.5E1,2E0,1.2E1,5.9E1,4.2E1,9E0,9E0,2.08E2,2E0,1.2E1,1.05E2,1.2E1,5E0,1.6E1,4E0,5E0,3.6E1,9E0,1.2E1,4.1E1,3E0,3.1E1,1.5E1,2.5E1,2E0,4.6E1,9E0,9E0,4E0,2E0,3E0,5.1E1,4E0,1E1,2E0,5.7E1,2E0,7E0,3.5E1,5E0,4E0,1.89E2,1.9E1,8E0,4E0,4E1,6.5E1,1E1,2E0,5E0,1.1E1,2E0,2E0,3E0,3.3E1,2E0,7E0,8E0,4E0,1.8E1,2.3E1,2E0,2.9E1,1E1,5E0,2.3E1,2.3E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-1.9318291E-3,-6.704857E-2,2.0721307E-1,-2.0402244E-1,-9.069468E-3,2.5231922E-1,-1.278987E-1,-1.8017007E-1,-4.2399663E-1,-1.5363221E-1,2.076618E-2,1.8841562E-1,4.753154E-1,-2.614E-1,-3.6709725E-3,-2.57905E-1,-1.1789815E-1,-2.2969361E-2,-8.306774E-3,-1.3281718E-1,-3.1868964E-2,2.7551645E-2,-2.0864699E-2,2.6121318E-2,1.668214E-1,6.071812E-1,2.9331067E-1,-1.1397065E-3,-1.4056629E-2,1.6874835E-1,-1.7563142E-1,-2.890318E-1,-1.422913E-1,1.9916524E-1,-1.5193653E-1,-1.867759E-1,-6.8793185E-2,6.475393E-2,-5.3646795E-2,2.2557718E-1,3.7009586E-2,3.8370636E-1,6.603558E-1,-2.9748138E-3,3.365614E-1,1.5139903E-2,2.89415E-3,-1.2913173E-2,-2.7385675E-3,-2.981565E-1,-5.491595E-3,2.9681916E-3,-1.687384E-1,1.1439017E-2,2.594369E-3,-1.0598428E-1,-3.1756946E-1,-9.356094E-2,-2.51296E-1,-8.07785E-5,-1.0483362E-1,2.7452411E-2,1.4847398E-1,-1.9170849E-1,-2.3748038E-2,3.2244095E-1,1.2810092E-1,1.7745144E-3,2.0855875E-1,6.972722E-3,2.079972E-2,1.1968583E-2,3.230638E-2,1.1231457E-2,2.490148E-2,-6.268634E-3,-1.4282448E-2,8.882203E-5,-1.0121511E-2,-7.1336553E-3,4.2033487E-4,-1.9364327E-2,-4.882064E-3,-9.346188E-3,-2.518432E-3,-6.179679E-3,-1.3791624E-2,-5.9110075E-3,6.555369E-4,2.3272121E-3,-8.13625E-3,8.019575E-3,-1.2975091E-2,6.018482E-3,-1.1456978E-2,3.620615E-3,-2.8848168E-3,4.75321E-3,1.74378E-2,-9.654081E-3,7.459768E-3,1.3284829E-3,-8.904849E-3,2.6695211E-3,1.1597988E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,37,-1,-1,39,41,43,-1,-1,45,47,49,51,53,55,57,59,61,63,65,67,69,71,-1,73,-1,-1,-1,-1,75,-1,-1,77,-1,-1,79,81,83,85,-1,87,89,91,93,95,97,99,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2733884E1,5.6605268E0,3.3879337E0,1.0359516E0,2.1683779E0,2.703494E0,4.4682363E-1,9.038491E-1,2.5477505E-1,8.606198E-1,1.3405355E0,1.1475215E0,8.539152E-1,1.1837703E-1,4.7437707E-1,2.6476192E-1,1.2066239E0,0E0,0E0,2.7734184E-1,0E0,1.2480298E0,0E0,0E0,1.1128492E0,3.1814575E-2,3.305757E-1,0E0,0E0,1.1334877E-1,7.894023E-2,6.774616E-2,1.1979097E-1,6.0084403E-2,7.19852E-1,2.4712944E-1,9.64874E-2,8.797004E-1,5.320709E-1,9.123292E-1,2.8372362E-1,3.690648E-2,6.9993496E-2,0E0,2.1098757E-1,0E0,0E0,0E0,0E0,4.9188614E-2,0E0,0E0,1.5059888E-1,0E0,0E0,4.32123E-1,3.9738536E-1,7.377002E-2,9.9918365E-2,0E0,7.063553E-2,9.093483E-1,9.39234E-1,4.3541896E-1,4.2486984E-1,5.398531E-1,5.970734E-1,2.1255395E-1,3.504789E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,21,21,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,49,49,52,52,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,38,-1,-1,40,42,44,-1,-1,46,48,50,52,54,56,58,60,62,64,66,68,70,72,-1,74,-1,-1,-1,-1,76,-1,-1,78,-1,-1,80,82,84,86,-1,88,90,92,94,96,98,100,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2.7555334E7,4.217427E7,4.97E2,1.4176E4,1.465E4,2.71E2,1.5570383E5,6.160415E6,4.4E1,1.0918E4,5.489183E1,3.6451373E8,4.1831533E8,1.9E1,1.1E1,-2.2969361E-2,-8.306774E-3,3.56E2,-3.1868964E-2,2E0,-2.0864699E-2,2.6121318E-2,8E0,1.7379E4,2.4545455E0,-1.1397065E-3,-1.4056629E-2,1.6673345E7,1.1997242E0,3.328629E6,4.4E0,7.122041E7,2.0734E4,3.1E1,4.8421054E0,5.2767E4,7.37E2,2.1027016E5,1.9616238E2,2.82E2,6.203939E2,-2.9748138E-3,2.6108465E0,1.5139903E-2,2.89415E-3,-1.2913173E-2,-2.7385675E-3,7E0,-5.491595E-3,2.9681916E-3,6.5E1,1.1439017E-2,2.594369E-3,8.314423E6,3.2525E2,3.89E2,3.5675676E0,-8.07785E-5,7.156528E7,3.493E3,1.4676277E7,7.8E1,9.114943E0,4.797342E2,1.2772413E0,1.4993486E2,1E0,6.972722E-3,2.079972E-2,1.1968583E-2,3.230638E-2,1.1231457E-2,2.490148E-2,-6.268634E-3,-1.4282448E-2,8.882203E-5,-1.0121511E-2,-7.1336553E-3,4.2033487E-4,-1.9364327E-2,-4.882064E-3,-9.346188E-3,-2.518432E-3,-6.179679E-3,-1.3791624E-2,-5.9110075E-3,6.555369E-4,2.3272121E-3,-8.13625E-3,8.019575E-3,-1.2975091E-2,6.018482E-3,-1.1456978E-2,3.620615E-3,-2.8848168E-3,4.75321E-3,1.74378E-2,-9.654081E-3,7.459768E-3,1.3284829E-3,-8.904849E-3,2.6695211E-3,1.1597988E-2],"split_indices":[2,43,60,60,2,2,44,2,48,12,3,9,71,7,47,3,10,0,0,2,0,32,0,0,32,2,69,0,0,47,53,9,69,7,44,10,73,1,2,48,71,0,4,0,50,0,0,0,0,3,0,0,44,0,0,5,67,1,69,0,7,44,62,10,73,4,68,73,27,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.33E2,7.12E2,2.21E2,2.11E2,5.01E2,1.95E2,2.6E1,1.92E2,1.9E1,8.5E1,4.16E2,1.53E2,4.2E1,1.2E1,1.4E1,8.4E1,1.08E2,1.4E1,5E0,8.3E1,2E0,4.11E2,5E0,7E0,1.46E2,2.3E1,1.9E1,2E0,1E1,7E0,7E0,6.5E1,1.9E1,1E1,9.8E1,4.4E1,3.9E1,2.82E2,1.29E2,1E2,4.6E1,6E0,1.7E1,2E0,1.7E1,2E0,5E0,3E0,4E0,6.1E1,4E0,2E0,1.7E1,7E0,3E0,7.8E1,2E1,1.9E1,2.5E1,1.4E1,2.5E1,1.96E2,8.6E1,2.2E1,1.07E2,4.9E1,5.1E1,3.9E1,7E0,2E0,4E0,2E0,1.5E1,1.3E1,4E0,4E0,5.7E1,4E0,1.3E1,5.5E1,2.3E1,1.3E1,7E0,4E0,1.5E1,8E0,1.7E1,2.1E1,4E0,1.77E2,1.9E1,8.2E1,4E0,3E0,1.9E1,2.9E1,7.8E1,1E1,3.9E1,4E0,4.7E1,3.5E1,4E0,2E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-7.4705365E-3,-7.551637E-2,1.87987E-1,-2.1064855E-1,-1.2208651E-2,1.6139205E-1,5.9388167E-1,-1.874214E-1,-4.105344E-1,-1.0302599E-1,3.6360014E-2,8.35767E-2,3.410534E-1,3.149769E-2,9.081904E-3,-2.7211854E-1,-1.261806E-1,-7.2798724E-3,-5.355675E-1,-1.3988125E-1,7.485649E-2,6.0380295E-2,-1.1568493E-1,-2.4221517E-1,1.1754421E-1,3.628804E-1,-1.9639159E-2,-3.785912E-1,-2.4039932E-1,2.8658533E-1,-1.5766054E-1,-3.867323E-2,-3.0442318E-1,-1.1591077E-1,-2.8895807E-1,2.1985252E-3,6.4317556E-3,4.945033E-2,3.201912E-1,-2.3368818E-1,2.468473E-2,5.0099824E-2,-3.7964016E-1,3.930384E-1,9.093292E-2,5.1353747E-1,3.0954593E-1,3.4502842E-3,-4.972567E-3,-3.674232E-3,-1.911936E-2,-2.5009108E-1,-4.509099E-3,1.551322E-2,3.7876533E-3,-1.2090286E-1,-3.602836E-1,-5.5708946E-3,-1.8954227E-2,-1.0351412E-1,-2.1483328E-2,-3.3478716E-1,-1.553453E-3,5.1743057E-2,-6.8359016E-3,6.2221978E-2,-1.5479375E-1,6.085381E-4,1.7997652E-2,1.0175424E-2,-3.1554133E-1,2.065922E-1,-4.5073368E-2,5.771956E-3,-3.0358986E-3,-2.0910077E-2,-2.0089597E-3,2.0231916E-2,5.026442E-3,-4.5991838E-2,1.4672062E-1,8.776642E-3,2.5414584E-2,2.2086531E-1,1.731304E-2,-7.1327575E-3,-1.2430172E-2,-3.9597554E-3,-1.7212112E-2,-1.836835E-2,-6.923425E-3,3.6361371E-3,-5.4446217E-3,-1.147176E-2,-2.229222E-2,6.3770916E-4,5.7844864E-3,-1.8039112E-3,3.8962113E-3,4.016002E-3,-9.088094E-3,-6.7032436E-3,4.6844743E-3,-1.1540425E-2,-2.6501445E-2,2.0923503E-3,1.6426405E-2,1.6010436E-4,-5.020156E-3,1.7889848E-4,-1.0400153E-2,9.347753E-3,-6.2289514E-4,7.3477086E-3,1.7819893E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,-1,57,59,61,63,-1,65,67,69,71,73,75,77,79,81,83,-1,-1,-1,-1,85,-1,-1,-1,87,89,-1,-1,91,-1,93,-1,95,-1,97,99,-1,-1,101,103,105,107,-1,-1,-1,-1,-1,-1,109,111,-1,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2875365E1,6.139855E0,2.6028395E0,9.8625374E-1,2.1650758E0,3.270873E0,3.6626673E-1,1.0393553E0,6.466205E-1,1.1315311E0,1.175677E0,1.8654832E0,5.660982E-1,0E0,0E0,1.9942904E-1,1.6304325E0,0E0,7.730675E-1,4.7393584E-1,1.3797492E-1,7.7076995E-1,7.345877E-1,6.6176206E-1,1.071279E0,3.7939548E-1,4.887825E-2,1.4283943E-1,7.167935E-2,5.956924E-2,8.055074E-1,0E0,1.4159602E-1,4.8408198E-1,2.1134341E-1,1.1847363E-1,0E0,7.0611E-1,2.205714E-1,4.822749E-1,2.757917E-1,5.9334554E-2,2.3951328E-1,1.0851908E-1,1.0685297E0,6.798458E-2,2.0699215E-1,0E0,0E0,0E0,0E0,6.536436E-2,0E0,0E0,0E0,8.4178734E-1,6.581259E-2,0E0,0E0,3.0717564E-1,0E0,8.556044E-2,0E0,3.1442873E-2,0E0,5.518896E-1,1.7358887E-1,0E0,0E0,1.1232518E-1,1.6710854E-1,1.2626162E-1,4.9174257E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.6382732E-1,8.62998E-1,0E0,0E0,1.8437207E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,51,51,55,55,56,56,59,59,61,61,63,63,65,65,66,66,69,69,70,70,71,71,72,72,79,79,80,80,83,83],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,-1,58,60,62,64,-1,66,68,70,72,74,76,78,80,82,84,-1,-1,-1,-1,86,-1,-1,-1,88,90,-1,-1,92,-1,94,-1,96,-1,98,100,-1,-1,102,104,106,108,-1,-1,-1,-1,-1,-1,110,112,-1,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.7909248E5,5.428706E3,5.035714E0,6.99E2,2.9251662E6,1E0,2.85E2,1.3E1,2.9652428E6,2E1,1.4598765E0,2E0,3.149769E-2,9.081904E-3,1.2772413E0,2.74614E5,-7.2798724E-3,1.753E3,2E0,4.49239E6,7.4711205E6,8.484036E5,1.3E1,5.1E1,2.5959E4,3.5510652E6,1.6666666E0,3.328629E6,2.04115E5,4.217427E7,-3.867323E-2,6.185E3,7.76E2,9E0,1E0,6.4317556E-3,5.94E2,1E0,1.4195632E6,1.5756368E3,5.844366E8,6.3314816E1,1E0,3.917053E5,4E0,6.426937E6,3.4502842E-3,-4.972567E-3,-3.674232E-3,-1.911936E-2,2.416E3,-4.509099E-3,1.551322E-2,3.7876533E-3,2.0734E4,9.643839E6,-5.5708946E-3,-1.8954227E-2,1.552356E0,-2.1483328E-2,3E0,-1.553453E-3,1.1255652E3,-6.8359016E-3,3.927456E7,1.162E3,6.085381E-4,1.7997652E-2,2.956111E2,5.3E1,3.3817584E7,5.7791035E4,5.771956E-3,-3.0358986E-3,-2.0910077E-2,-2.0089597E-3,2.0231916E-2,5.026442E-3,1.317484E1,1.6553869E7,8.776642E-3,2.5414584E-2,5.4264073E0,1.731304E-2,-7.1327575E-3,-1.2430172E-2,-3.9597554E-3,-1.7212112E-2,-1.836835E-2,-6.923425E-3,3.6361371E-3,-5.4446217E-3,-1.147176E-2,-2.229222E-2,6.3770916E-4,5.7844864E-3,-1.8039112E-3,3.8962113E-3,4.016002E-3,-9.088094E-3,-6.7032436E-3,4.6844743E-3,-1.1540425E-2,-2.6501445E-2,2.0923503E-3,1.6426405E-2,1.6010436E-4,-5.020156E-3,1.7889848E-4,-1.0400153E-2,9.347753E-3,-6.2289514E-4,7.3477086E-3,1.7819893E-2],"split_indices":[2,43,67,68,2,43,30,2,0,43,3,69,6,0,0,68,5,0,44,6,43,43,43,3,0,9,43,69,9,5,60,0,44,10,8,16,0,0,29,60,48,7,73,30,43,3,60,0,0,0,0,9,0,0,0,44,62,0,0,69,0,8,0,48,0,7,10,0,0,4,3,7,48,0,0,0,0,0,0,69,66,0,0,50,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.66E2,7.17E2,2.49E2,2.28E2,4.89E2,2.35E2,1.4E1,2.06E2,2.2E1,1.7E2,3.19E2,1.65E2,7E1,1.1E1,3E0,8.5E1,1.21E2,8E0,1.4E1,1.41E2,2.9E1,2.76E2,4.3E1,1.5E1,1.5E2,6.6E1,4E0,1.7E1,6.8E1,8E0,1.13E2,5E0,9E0,1.23E2,1.8E1,1.4E1,1.5E1,2.66E2,1E1,2.3E1,2E1,5E0,1E1,1.2E1,1.38E2,1.5E1,5.1E1,2E0,2E0,2E0,1.5E1,6.3E1,5E0,6E0,2E0,9.7E1,1.6E1,4E0,5E0,1.2E2,3E0,1.5E1,3E0,1.1E1,3E0,2.51E2,1.5E1,2E0,8E0,6E0,1.7E1,5E0,1.5E1,3E0,2E0,8E0,2E0,1E1,2E0,4E1,9.8E1,2E0,1.3E1,2.3E1,2.8E1,1.1E1,5.2E1,8.6E1,1.1E1,1.3E1,3E0,8E0,1.12E2,1.1E1,4E0,8E0,3E0,4.4E1,2.07E2,2E0,1.3E1,2E0,4E0,1.5E1,2E0,3E0,2E0,9E0,6E0,3.2E1,8E0,7.3E1,2.5E1,1.8E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.1755071E-2,-7.039245E-2,1.7081332E-1,-2.191927E-1,-3.1505693E-2,-4.1356772E-1,2.0323873E-1,1.8600799E-1,-2.3522177E-1,-9.907946E-2,1.976104E-2,-8.022015E-3,-2.2080036E-2,1.01196446E-1,2.6731834E-1,-5.887716E-3,1.7391957E-2,-2.5073728E-1,1.479091E-1,2.503937E-2,-1.3324666E-1,2.754392E-2,-3.8305354E-1,1.5324606E-1,-6.5169386E-3,4.0245882E-1,1.7379753E-1,-2.2874184E-1,-4.3617663E-1,1.7383405E-4,1.0187965E-2,-1.8598038E-1,4.649894E-2,-1.5558556E-1,4.968207E-2,6.941876E-2,-3.600295E-2,-1.6119676E-3,-2.551336E-2,1.8042067E-1,-6.2750615E-2,6.949759E-2,-6.787144E-2,3.024678E-1,3.0334946E-2,2.086003E-1,-1.1620136E-1,-2.0979227E-1,-5.569765E-1,-2.2942811E-2,-8.570905E-3,-1.9208353E-3,-1.2494992E-2,1.0215247E-1,9.701739E-3,-1.4368448E-1,-2.3721807E-2,-1.7105432E-1,1.2935309E-1,7.711611E-2,-1.9804085E-2,-7.2648E-2,9.0923056E-2,4.726939E-2,2.2122452E-1,-5.5356966E-3,2.4166107E-3,3.8779585E-4,9.535044E-3,-3.447912E-4,-8.388031E-3,-3.6333941E-3,3.2515183E-1,7.0685917E-3,2.5443965E-1,-8.7044295E-3,4.0976047E-3,-1.6166024E-2,-7.8185415E-3,-6.2557138E-3,-3.4163654E-2,1.906897E-3,7.698949E-3,-8.094434E-3,2.4841093E-3,-6.019916E-3,-1.8683067E-2,-6.3208316E-4,-1.145892E-2,9.597951E-4,9.841779E-3,1.7486067E-3,8.47702E-3,-1.7255188E-3,-1.0446529E-2,6.267429E-3,-4.2916257E-3,-4.4442527E-4,6.5157656E-3,1.4070403E-2,6.5571493E-3,2.8692128E-3,-2.0746363E-3,1.7698998E-2,6.3942927E-3,6.7005875E-3,-5.287714E-3,1.678408E-2,7.441128E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,-1,27,29,31,33,35,37,39,41,43,45,47,49,-1,-1,51,53,55,57,59,61,-1,-1,63,65,67,69,71,-1,73,75,77,79,-1,-1,-1,-1,81,83,85,-1,87,89,91,-1,93,95,97,99,-1,-1,-1,-1,101,-1,-1,103,105,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.775288E0,3.9914687E0,4.2906837E0,9.725361E-1,1.903851E0,9.514165E-2,1.3526402E0,3.9986986E-1,8.5742426E-1,1.0084045E0,9.9182355E-1,0E0,0E0,4.663332E-1,1.5604143E0,0E0,0E0,4.4271183E-1,6.0729012E-2,2.4412435E-1,7.678051E-1,8.220919E-1,3.2700628E-1,3.4002054E-1,1.3530423E-1,1.101553E0,8.114748E-1,6.357584E-1,9.618235E-2,0E0,0E0,4.8717797E-2,9.730785E-2,6.2701035E-1,3.9143854E-1,7.418817E-1,5.7870424E-1,0E0,0E0,2.6119697E-1,5.137471E-2,1.0604135E-1,1.21471666E-1,3.6210418E-1,0E0,6.448672E-1,1.4696938E-1,6.078305E-1,3.5138786E-1,0E0,0E0,0E0,0E0,6.686212E-2,2.50251E-1,5.434699E-1,0E0,6.782535E-2,1.387263E-1,7.577628E-1,0E0,5.1025933E-1,2.3731026E-1,6.99164E-2,2.0526528E-1,0E0,0E0,0E0,0E0,3.2958575E-2,0E0,0E0,3.3241558E-1,2.4801792E-1,5.1529384E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,53,53,54,54,55,55,57,57,58,58,59,59,61,61,62,62,63,63,64,64,69,69,72,72,73,73,74,74],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,-1,28,30,32,34,36,38,40,42,44,46,48,50,-1,-1,52,54,56,58,60,62,-1,-1,64,66,68,70,72,-1,74,76,78,80,-1,-1,-1,-1,82,84,86,-1,88,90,92,-1,94,96,98,100,-1,-1,-1,-1,102,-1,-1,104,106,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,6.829817E4,1E0,7E0,2.778995E2,1.04719896E8,9.6215985E2,4.9E2,1.5045853E0,1.7083334E0,4.9E1,-8.022015E-3,-2.2080036E-2,5.117647E0,5.974481E1,-5.887716E-3,1.7391957E-2,3.94E2,1.017E3,3.51E2,1E0,2E0,7.31E2,2E0,7.5935423E-1,2.0597144E3,2.871605E2,3.8652172E1,4.868063E1,1.7383405E-4,1.0187965E-2,2.707673E6,9.59799E-1,5.035714E0,2.5733176E7,9.8897E2,1.862E3,-1.6119676E-3,-2.551336E-2,6.978874E5,5.12009E5,2E1,1.6E1,3.8181818E0,3.0334946E-2,3.3580637E8,2.75E0,3.9E1,4.9E1,-2.2942811E-2,-8.570905E-3,-1.9208353E-3,-1.2494992E-2,4E0,1.41E2,2.8302418E10,-2.3721807E-2,5.88E2,2.01E2,5.281628E2,-1.9804085E-2,2.0734E4,1.5116882E8,5.357143E2,9.48E0,-5.5356966E-3,2.4166107E-3,3.8779585E-4,9.535044E-3,2.3238909E1,-8.388031E-3,-3.6333941E-3,1.647E4,3.53969E5,2.047619E0,-8.7044295E-3,4.0976047E-3,-1.6166024E-2,-7.8185415E-3,-6.2557138E-3,-3.4163654E-2,1.906897E-3,7.698949E-3,-8.094434E-3,2.4841093E-3,-6.019916E-3,-1.8683067E-2,-6.3208316E-4,-1.145892E-2,9.597951E-4,9.841779E-3,1.7486067E-3,8.47702E-3,-1.7255188E-3,-1.0446529E-2,6.267429E-3,-4.2916257E-3,-4.4442527E-4,6.5157656E-3,1.4070403E-2,6.5571493E-3,2.8692128E-3,-2.0746363E-3,1.7698998E-2,6.3942927E-3,6.7005875E-3,-5.287714E-3,1.678408E-2,7.441128E-3],"split_indices":[2,43,17,3,70,60,67,1,53,68,3,0,0,69,73,0,0,0,2,2,23,32,2,6,53,67,73,68,73,0,0,5,68,68,5,4,2,0,0,43,9,3,3,71,0,7,68,2,3,0,0,0,0,8,67,46,0,2,0,4,0,44,12,67,71,0,0,0,0,73,0,0,44,9,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.11E2,6.9E2,2.21E2,1.42E2,5.48E2,1.1E1,2.1E2,5E0,1.37E2,2.36E2,3.12E2,3E0,8E0,8.2E1,1.28E2,2E0,3E0,1.32E2,5E0,5.1E1,1.85E2,3.07E2,5E0,5.5E1,2.7E1,5.1E1,7.7E1,1.2E2,1.2E1,2E0,3E0,4E0,4.7E1,1.65E2,2E1,1.85E2,1.22E2,2E0,3E0,4.9E1,6E0,1.2E1,1.5E1,3.8E1,1.3E1,6.9E1,8E0,1.15E2,5E0,9E0,3E0,2E0,2E0,1.8E1,2.9E1,1.61E2,4E0,5E0,1.5E1,1.83E2,2E0,9.5E1,2.7E1,1.2E1,3.7E1,4E0,2E0,9E0,3E0,1E1,5E0,2E0,3.6E1,1.3E1,5.6E1,6E0,2E0,2.5E1,9E1,2E0,3E0,1E1,8E0,5E0,2.4E1,1.54E2,7E0,2E0,3E0,7E0,8E0,1.34E2,4.9E1,7.8E1,1.7E1,2.2E1,5E0,8E0,4E0,1.7E1,2E1,4E0,6E0,2.7E1,9E0,6E0,7E0,2.5E1,3.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[9.354484E-3,-9.0168335E-2,1.0774568E-1,-1.9710848E-1,-3.6393642E-2,1.4703196E-1,-1.5376952E-1,-1.8090884E-1,-5.503481E-1,2.698402E-3,-1.1304103E-1,5.1919658E-2,2.1655023E-1,-6.516745E-2,-4.339071E-1,-2.7681014E-1,-1.3964647E-1,-4.34112E-3,-3.5137195E-2,-9.177661E-3,2.0434615E-1,-2.0510739E-1,-4.2835943E-2,1.7853397E-1,7.594375E-3,3.532232E-2,2.0328653E-1,-1.06658526E-1,1.1725792E-1,-2.7707815E-2,-1.2919182E-2,-3.646337E-2,-3.0115023E-1,1.7697269E-1,-1.6127864E-1,-1.1427906E-1,1.750921E-2,2.6344527E-3,1.30525455E-2,-1.3755019E-1,-3.105133E-1,-7.6240287E-3,-9.817318E-3,2.1998602E-1,-3.0729708E-3,1.0634878E-1,-5.0004322E-2,2.7374643E-1,1.0517851E-1,-3.2738145E-2,-1.869735E-1,1.1267733E-2,6.682503E-4,2.630645E-3,-5.457016E-3,-3.1006935E-1,-4.359496E-3,1.286041E-2,8.1848025E-2,-9.4547875E-2,-1.9974129E-1,-2.524491E-2,-2.651058E-1,2.4889655E-2,-9.3713505E-3,-8.314647E-2,-1.1308967E-2,-3.4854975E-1,-5.0817193E-3,2.8179985E-2,-1.5007769E-1,1.3869707E-3,2.3588803E-1,5.5865046E-2,1.8177542E-1,-2.7888734E-2,-1.462334E-2,3.633952E-1,1.5928417E-1,7.826017E-2,2.2646284E-2,3.733277E-2,-1.7821322E-1,1.3439427E-3,-2.1576342E-1,-6.0761003E-3,-1.4995038E-2,-6.238571E-3,1.25828255E-2,-6.8154265E-3,-3.960865E-4,-1.283458E-3,-1.0496497E-2,-5.7065818E-3,4.2116796E-3,-6.1377934E-3,-3.298616E-2,3.5486266E-4,6.196792E-3,-7.2621433E-3,2.5447636E-4,-6.4766076E-3,-1.8482622E-2,2.579522E-3,-8.087897E-3,-1.703222E-3,-8.997801E-3,1.1924421E-2,3.0534202E-3,4.6761916E-3,-1.2273958E-3,-3.3100299E-3,1.0131306E-2,-2.254237E-3,5.866541E-3,1.371474E-2,2.188718E-2,1.2508315E-3,1.23714665E-2,4.738455E-3,-9.908474E-3,6.891879E-3,-7.6504983E-4,-1.1201166E-2,-6.7067536E-4,-6.477845E-3,-1.6564E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,45,-1,47,49,51,-1,-1,53,55,57,59,61,63,-1,-1,65,67,69,-1,71,-1,73,75,77,79,81,83,-1,-1,-1,-1,85,-1,-1,87,89,91,93,95,97,-1,99,-1,101,-1,103,105,-1,107,109,111,113,-1,115,117,119,-1,121,123,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.724745E0,2.5403748E0,4.6253433E0,7.47345E-1,8.872108E-1,2.5674534E0,1.4361488E0,5.2721214E-1,4.9205482E-1,4.7364736E-1,6.3756657E-1,9.302718E-1,1.4659367E0,3.6068165E-1,1.8828464E-1,2.3472142E-1,7.2715604E-1,0E0,0E0,5.265037E-1,1.0808021E-1,2.6348877E-1,3.420159E-1,4.505353E-1,7.106454E-1,0E0,1.49891E0,2.1948156E-1,1.1228726E-1,0E0,0E0,4.4253767E-2,4.1232824E-2,4.365489E-2,2.2751236E-1,5.035993E-1,2.4747485E-1,0E0,0E0,1.406638E-1,8.439076E-2,2.541494E-1,0E0,1.0485256E-1,0E0,1.6500485E-1,4.5321608E-1,1.2555771E0,9.2786634E-1,2.207182E-1,1.18103266E-1,0E0,0E0,0E0,0E0,3.76606E-2,0E0,0E0,2.4008487E-1,1.6426522E-1,2.6183248E-1,2.932602E-1,7.34238E-1,2.720197E-1,0E0,1.3024265E-1,0E0,7.759726E-2,0E0,2.301976E-1,4.2145014E-2,0E0,1.01873636E-1,1.0923915E-1,1.7971742E-1,2.4070808E-1,0E0,3.8270473E-1,8.123442E-1,6.342373E-1,0E0,9.186948E-2,6.8338305E-2,0E0,1.2218392E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,41,41,43,43,45,45,46,46,47,47,48,48,49,49,50,50,55,55,58,58,59,59,60,60,61,61,62,62,63,63,65,65,67,67,69,69,70,70,72,72,73,73,74,74,75,75,77,77,78,78,79,79,81,81,82,82,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,46,-1,48,50,52,-1,-1,54,56,58,60,62,64,-1,-1,66,68,70,-1,72,-1,74,76,78,80,82,84,-1,-1,-1,-1,86,-1,-1,88,90,92,94,96,98,-1,100,-1,102,-1,104,106,-1,108,110,112,114,-1,116,118,120,-1,122,124,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.221E3,1.7909248E5,3.0923106E-10,2.5013582E-5,4.066351E0,5.867288E2,3.583621E1,3.89E2,5.5E1,7.168071E6,1.8479175E2,9.912121E0,3.3817584E7,1.4433751E0,8.317E3,5E0,1.25E0,-4.34112E-3,-3.5137195E-2,1.5933333E2,3.3852024E7,4.9236734E5,4.356E3,1.1208894E0,2.978142E0,3.532232E-2,8E0,6E0,4.9948E4,-2.7707815E-2,-1.2919182E-2,7.67525E2,1.16E2,1.3E1,9.114943E0,5.185771E0,2.344898E1,2.6344527E-3,1.30525455E-2,4.0695653E0,6.4921465E0,1E0,-9.817318E-3,1.4473684E-1,-3.0729708E-3,2.8801656E7,3.7E1,2.1298597E0,4.91027E0,4.217427E7,1.1649485E0,1.1267733E-2,6.682503E-4,2.630645E-3,-5.457016E-3,1.798E3,-4.359496E-3,1.286041E-2,4.97E2,2.824856E-3,2.0348837E0,5.44E2,2.2318378E6,1.065548E0,-9.3713505E-3,4.1719616E5,-1.1308967E-2,3.1E1,-5.0817193E-3,1.124641E0,3.14403E5,1.3869707E-3,2.511E3,1.1492429E5,1.0950326E6,1.6305691E8,-1.462334E-2,1.3359968E10,2.9251662E6,7.134432E7,2.2646284E-2,1.22E3,2.297E3,1.3439427E-3,1.1358E4,-6.0761003E-3,-1.4995038E-2,-6.238571E-3,1.25828255E-2,-6.8154265E-3,-3.960865E-4,-1.283458E-3,-1.0496497E-2,-5.7065818E-3,4.2116796E-3,-6.1377934E-3,-3.298616E-2,3.5486266E-4,6.196792E-3,-7.2621433E-3,2.5447636E-4,-6.4766076E-3,-1.8482622E-2,2.579522E-3,-8.087897E-3,-1.703222E-3,-8.997801E-3,1.1924421E-2,3.0534202E-3,4.6761916E-3,-1.2273958E-3,-3.3100299E-3,1.0131306E-2,-2.254237E-3,5.866541E-3,1.371474E-2,2.188718E-2,1.2508315E-3,1.23714665E-2,4.738455E-3,-9.908474E-3,6.891879E-3,-7.6504983E-4,-1.1201166E-2,-6.7067536E-4,-6.477845E-3,-1.6564E-2],"split_indices":[2,43,52,52,69,67,71,1,0,43,67,73,7,53,9,0,68,0,0,4,7,43,44,53,69,0,32,8,9,0,0,48,44,0,73,73,61,0,0,71,69,19,0,71,0,12,3,53,50,60,68,0,0,0,0,9,0,0,2,53,68,12,43,53,0,60,0,0,0,53,9,0,2,48,62,12,0,46,43,59,0,44,2,0,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.89E2,4.42E2,4.47E2,1.47E2,2.95E2,3.89E2,5.8E1,1.42E2,5E0,1.96E2,9.9E1,1.65E2,2.24E2,4.5E1,1.3E1,4.1E1,1.01E2,2E0,3E0,1.86E2,1E1,4.2E1,5.7E1,4.2E1,1.23E2,4E0,2.2E2,3.7E1,8E0,5E0,8E0,4E0,3.7E1,6E0,9.5E1,3.7E1,1.49E2,4E0,6E0,2.7E1,1.5E1,4.8E1,9E0,3.6E1,6E0,4.5E1,7.8E1,1.27E2,9.3E1,2E1,1.7E1,3E0,5E0,2E0,2E0,3.5E1,2E0,2E0,4E0,3.6E1,5.9E1,2.4E1,1.3E1,1.45E2,4E0,1.9E1,8E0,1.2E1,3E0,3.9E1,9E0,3E0,3.3E1,2.8E1,1.7E1,7.3E1,5E0,7E1,5.7E1,8.8E1,5E0,1.4E1,6E0,2E0,1.5E1,3E0,3.2E1,2E0,2E0,2.2E1,1.4E1,8E0,5.1E1,1.3E1,1.1E1,1.1E1,2E0,1.26E2,1.9E1,1E1,9E0,3E0,9E0,3.5E1,4E0,3E0,6E0,2.9E1,4E0,1.8E1,1E1,2E0,1.5E1,6.5E1,8E0,4.5E1,2.5E1,2.6E1,3.1E1,8.2E1,6E0,4E0,1E1,4E0,2E0,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[-2.377732E-3,-6.387758E-2,2.1318182E-1,-1.3617496E-1,4.554339E-2,3.230638E-2,1.8507311E-1,-9.762082E-2,-3.1993324E-1,6.58859E-2,-9.42365E-2,8.162091E-2,2.7580371E-1,-2.0722322E-1,-3.797348E-2,-2.1181181E-1,-4.511238E-1,1.1219828E-2,1.624795E-1,1.9172585E-2,-1.7243236E-1,-9.341022E-3,1.06948145E-1,1.4931273E-1,3.4636953E-1,-1.7583524E-1,-2.913031E-1,2.5348035E-1,-5.070164E-2,-2.4193761E-1,-2.231658E-2,-6.672459E-1,-2.5675768E-1,-2.0705087E-2,1.3201672E-1,1.7751569E-1,-1.5708096E-2,-3.4800237E-3,7.0095663E-3,-2.1550532E-1,3.5007324E-3,1.2482105E-1,-2.0755714E-1,2.2915705E-1,2.0214228E-2,4.4152954E-1,1.829876E-1,-2.5290483E-1,-1.2001857E-1,-1.7284647E-1,-1.7692149E-2,6.0824296E-3,1.6278597E-2,-1.2258944E-1,1.4198916E-3,-1.7410389E-1,-1.7149633E-2,-3.933014E-3,2.1167065E-3,-3.3575234E-3,-3.503133E-2,-2.6269706E-2,-1.8728544E-1,-1.910096E-1,-1.6053977E-3,1.5181376E-2,1.02672786E-1,2.2809494E-1,4.6185367E-2,-1.03477195E-1,-3.0598292E-1,8.455027E-2,2.5390047E-1,-1.337541E-2,-2.7128884E-3,1.3899368E-1,1.5025572E-2,1.21304505E-1,-7.188982E-2,5.9837718E-3,4.7187567E-1,2.2539471E-1,-1.0104477E-2,2.535504E-4,-1.2418332E-2,-6.1696977E-3,1.766584E-3,-1.1513736E-2,-4.802257E-3,5.538288E-3,-6.6583334E-3,1.1400162E-3,-7.266187E-3,1.5643275E-3,-8.967453E-3,-1.7796604E-4,-1.0525813E-2,-2.0029275E-3,-1.9491553E-2,7.8522984E-4,-8.43229E-3,6.1961156E-3,-2.1786788E-3,1.17440885E-2,-3.1943206E-4,-5.1099726E-4,9.4381645E-3,-6.852019E-3,-4.014993E-5,-1.9130103E-2,-5.504921E-3,4.9449163E-3,-5.020368E-3,1.5895283E-2,2.5312754E-3,3.7436874E-3,9.46774E-3,-8.583478E-4,9.713204E-3,-7.553103E-3,1.5349447E-3,6.016948E-3,2.26841E-2,4.4704145E-3,1.1829421E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,-1,69,-1,71,73,75,77,79,81,83,85,87,-1,-1,-1,89,91,93,-1,-1,-1,-1,-1,-1,95,97,99,-1,101,103,105,107,109,111,113,-1,-1,115,-1,117,119,-1,121,123,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2302745E1,5.717836E0,2.659317E0,3.0505877E0,8.2512933E-1,0E0,1.8171992E0,2.3521652E0,9.682846E-1,1.3288361E0,3.3142644E-1,6.868205E-1,8.765807E-1,2.8394413E-1,8.846247E-1,2.4007487E-1,1.2149048E0,6.2747514E-1,7.4442124E-1,2.0912255E-1,2.5094777E-1,0E0,5.093242E-1,3.9737117E-1,9.4533587E-1,3.7831473E-1,2.8258896E-1,7.267451E-2,8.4697396E-1,2.5407696E-1,3.3406004E-2,7.131796E-1,3.0088294E-1,4.192998E-1,1.6044766E-1,5.806558E-1,0E0,0E0,0E0,1.6092247E-1,0E0,4.0477931E-1,4.2943135E-2,1.5833163E-1,1.5769374E-1,3.2855463E-1,4.902832E-1,1.4365268E-1,1.188907E-1,5.7632297E-2,0E0,0E0,0E0,4.8947966E-1,4.844019E-1,1.0755974E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2088966E-1,4.2063656E-1,3.9180717E-1,0E0,1.4786723E-1,3.6902857E-1,2.3524158E-1,4.4873156E-2,1.4896679E-1,2.7973846E-1,3.1445444E-1,0E0,0E0,3.6502093E-2,0E0,1.0174018E-1,9.08114E-2,0E0,1.3807201E-1,6.540036E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,53,53,54,54,55,55,62,62,63,63,64,64,66,66,67,67,68,68,69,69,70,70,71,71,72,72,75,75,77,77,78,78,80,80,81,81],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,-1,70,-1,72,74,76,78,80,82,84,86,88,-1,-1,-1,90,92,94,-1,-1,-1,-1,-1,-1,96,98,100,-1,102,104,106,108,110,112,114,-1,-1,116,-1,118,120,-1,122,124,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.659E3,7.785302E5,4.70127E5,2.1E1,1E0,3.230638E-2,1.7587205E6,4.33E2,5.444797E0,6.946813E7,1.934E3,1E0,1.055912E3,1.1644444E1,1.4E1,5.7E1,2E0,4.5866325E6,2.1190704E8,1.9024921E1,3.2645503E5,-9.341022E-3,3.0070068E7,4.22718E6,1.7103828E7,3.07E2,1.0140845E0,7.77E2,1.605992E-1,7.814751E6,4E0,8E0,3.964269E3,1.4110284E2,4E0,1.3558896E7,-1.5708096E-2,-3.4800237E-3,7.0095663E-3,2E0,3.5007324E-3,3.4915986E3,5.5766084E3,6.9706165E2,1.5209424E1,3.8737908E-1,1.0534078E4,2.3E1,1.9E1,2.49E2,-1.7692149E-2,6.0824296E-3,1.6278597E-2,5.6591835E1,2.0734E4,4.4E0,-1.7149633E-2,-3.933014E-3,2.1167065E-3,-3.3575234E-3,-3.503133E-2,-2.6269706E-2,1.308677E6,1.1290909E2,6.763314E7,1.5181376E-2,2.96059E5,1E0,5.281628E2,8.023379E4,5E0,4.7238097E0,2.629E3,-1.337541E-2,-2.7128884E-3,6.000379E8,1.5025572E-2,1.5031847E0,8.629636E2,5.9837718E-3,3.812E3,5.467E3,-1.0104477E-2,2.535504E-4,-1.2418332E-2,-6.1696977E-3,1.766584E-3,-1.1513736E-2,-4.802257E-3,5.538288E-3,-6.6583334E-3,1.1400162E-3,-7.266187E-3,1.5643275E-3,-8.967453E-3,-1.7796604E-4,-1.0525813E-2,-2.0029275E-3,-1.9491553E-2,7.8522984E-4,-8.43229E-3,6.1961156E-3,-2.1786788E-3,1.17440885E-2,-3.1943206E-4,-5.1099726E-4,9.4381645E-3,-6.852019E-3,-4.014993E-5,-1.9130103E-2,-5.504921E-3,4.9449163E-3,-5.020368E-3,1.5895283E-2,2.5312754E-3,3.7436874E-3,9.46774E-3,-8.583478E-4,9.713204E-3,-7.553103E-3,1.5349447E-3,6.016948E-3,2.26841E-2,4.4704145E-3,1.1829421E-2],"split_indices":[2,43,12,3,6,0,43,2,68,7,44,17,67,73,10,6,32,43,12,73,48,0,58,62,62,1,71,2,53,60,8,0,43,4,3,9,0,0,0,6,0,4,4,67,71,53,4,44,3,2,0,0,0,62,44,69,0,0,0,0,0,0,46,67,7,0,12,19,4,48,6,68,0,0,0,5,0,68,67,0,2,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,7.21E2,2.05E2,4.34E2,2.87E2,1E1,1.95E2,3.6E2,7.4E1,2.51E2,3.6E1,9.2E1,1.03E2,1.26E2,2.34E2,4.2E1,3.2E1,1.61E2,9E1,1.5E1,2.1E1,7E0,8.5E1,3.8E1,6.5E1,9.4E1,3.2E1,9E0,2.25E2,3.6E1,6E0,1.4E1,1.8E1,1.28E2,3.3E1,8.8E1,2E0,9E0,6E0,1.8E1,3E0,8.1E1,4E0,2.3E1,1.5E1,4E1,2.5E1,3.8E1,5.6E1,1.5E1,1.7E1,5E0,4E0,9.4E1,1.31E2,2.5E1,1.1E1,3E0,3E0,2E0,1.2E1,2E0,1.6E1,1.2E1,1.16E2,3E0,3E1,6.3E1,2.5E1,9E0,9E0,6.3E1,1.8E1,2E0,2E0,1.3E1,1E1,7E0,8E0,4E0,3.6E1,2.3E1,2E0,2E0,3.6E1,5.2E1,4E0,6E0,9E0,7E0,8.7E1,1.15E2,1.6E1,2E0,2.3E1,3E0,1.3E1,8E0,4E0,1.06E2,1E1,2.5E1,5E0,5.7E1,6E0,1.9E1,6E0,6E0,3E0,5E0,4E0,5.7E1,6E0,1.2E1,6E0,8E0,5E0,3E0,4E0,4E0,4E0,2E0,3.4E1,5E0,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[-1.0137049E-3,-1.0774979E-1,1.13899134E-1,-2.1406914E-1,-6.0042184E-2,6.67705E-2,3.1937066E-1,-2.7282414E-1,-1.8169956E-1,-3.8882505E-2,-4.610336E-1,-2.698003E-1,9.172472E-2,4.4060075E-1,1.5783341E-1,-2.4662846E-1,-2.4440428E-2,-1.1953529E-1,-2.4030109E-1,2.5415359E-2,-7.714455E-2,-6.972688E-1,-2.394494E-2,-3.4294733E-1,-1.3887733E-1,2.1171523E-2,8.073136E-2,3.6972865E-1,3.4822322E-2,8.146522E-2,2.9842877E-1,-1.6846025E-2,-2.1523914E-1,-1.460541E-1,-2.1177962E-2,-2.553102E-1,-2.3791064E-3,1.0991449E-2,1.0521566E-2,-2.3461193E-2,-6.607595E-2,-3.5510764E-2,-1.2371401E-2,-5.6263814E-3,6.7788535E-3,-1.8323287E-2,-7.062988E-3,-2.42053E-4,-7.813463E-3,4.008543E-2,1.9620912E-1,8.700452E-3,4.083391E-1,1.1806934E-2,-9.263611E-3,1.1861623E-1,1.8416284E-2,-3.5503653E-3,-1.0426811E-2,1.7602055E-3,-1.5672913E-1,9.898366E-4,-3.947584E-3,-3.1570848E-3,-2.711666E-1,-1.4006957E-1,3.991033E-2,-8.935983E-2,1.4794695E-1,-1.6628484E-1,5.2923534E-2,1.6946562E-1,2.48099E-2,4.292024E-1,6.965895E-3,2.90526E-2,-9.791187E-3,9.34281E-3,-1.4239732E-3,-5.981884E-3,-1.2223602E-2,-8.673278E-3,-1.4213337E-2,-1.1264659E-2,3.2384195E-3,9.3520895E-5,5.1193982E-3,-3.3095411E-3,-1.0446827E-2,9.198621E-3,-9.320698E-3,-1.423274E-2,-1.8192653E-3,3.1971775E-3,-5.719087E-3,9.126927E-3,1.6267414E-3,6.256898E-3,2.0716578E-2,-6.603416E-3,2.9592074E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,-1,49,51,-1,53,55,-1,57,59,61,63,-1,-1,65,-1,67,-1,-1,-1,-1,-1,-1,-1,-1,69,71,-1,73,-1,75,77,-1,-1,-1,-1,79,-1,-1,-1,81,83,85,87,89,91,93,95,-1,97,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1639821E1,2.482689E0,4.404077E0,2.3951292E-1,2.8754077E0,3.1575236E0,1.5950279E0,2.14602E-1,3.3921075E-1,8.015151E-1,1.7386515E0,2.0136487E-1,1.3538983E0,7.714329E-1,3.8369495E-1,9.9826336E-2,0E0,1.2993115E-1,1.3172412E-1,3.8076657E-1,9.315423E-1,1.1130905E-1,1.3290079E-1,9.620273E-2,4.1953623E-2,0E0,1.5832992E0,2.1275377E-1,0E0,4.0904874E-1,1.8107271E-1,0E0,3.389311E-2,8.445811E-2,3.4265235E-2,1.21272564E-1,0E0,0E0,5.145443E-1,0E0,1.0098771E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.764498E-1,7.0329165E-1,0E0,1.13729E-1,0E0,1.4492743E-1,7.9817355E-2,0E0,0E0,0E0,0E0,7.951915E-2,0E0,0E0,0E0,6.842661E-2,4.2383206E-1,2.5772738E-1,4.295733E-1,3.8688818E-1,2.5459504E-1,6.707995E-1,2.920711E-1,0E0,8.3658695E-2,0E0,1.0365527E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,30,30,32,32,33,33,34,34,35,35,38,38,40,40,49,49,50,50,52,52,54,54,55,55,60,60,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,73,73,75,75],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,-1,50,52,-1,54,56,-1,58,60,62,64,-1,-1,66,-1,68,-1,-1,-1,-1,-1,-1,-1,-1,70,72,-1,74,-1,76,78,-1,-1,-1,-1,80,-1,-1,-1,82,84,86,88,90,92,94,96,-1,98,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.14E3,3.56E2,8.439E3,3.89E2,9.932432E0,1E0,8.788792E1,6.737403E7,3.1E1,1.9010988E0,1.03E2,1.3146555E3,2.04115E5,2E1,4.435876E2,6.25E0,-2.4440428E-2,2E0,1.901875E2,4.9069305E1,1.45064E5,1E0,9.277255E6,7.295292E-2,2.7496547E8,2.1171523E-2,2.9652428E6,4.25681E-1,3.4822322E-2,4.1681065E1,1.11545664E5,-1.6846025E-2,1.0677966E0,7E0,6.710987E7,3.994386E0,-2.3791064E-3,1.0991449E-2,5.8182236E1,-2.3461193E-2,1.6100003E8,-3.5510764E-2,-1.2371401E-2,-5.6263814E-3,6.7788535E-3,-1.8323287E-2,-7.062988E-3,-2.42053E-4,-7.813463E-3,3.574764E7,2.1481724E3,8.700452E-3,1.4676277E7,1.1806934E-2,1.0534078E4,1.1987969E7,1.8416284E-2,-3.5503653E-3,-1.0426811E-2,1.7602055E-3,3.14403E5,9.898366E-4,-3.947584E-3,-3.1570848E-3,3.5419354E0,3.9351077E2,2.5677419E0,4.911E3,7.366243E7,2.234E3,1.6007428E7,1.49660635E1,2.48099E-2,1.57922E5,6.965895E-3,4.5478998E8,-9.791187E-3,9.34281E-3,-1.4239732E-3,-5.981884E-3,-1.2223602E-2,-8.673278E-3,-1.4213337E-2,-1.1264659E-2,3.2384195E-3,9.3520895E-5,5.1193982E-3,-3.3095411E-3,-1.0446827E-2,9.198621E-3,-9.320698E-3,-1.423274E-2,-1.8192653E-3,3.1971775E-3,-5.719087E-3,9.126927E-3,1.6267414E-3,6.256898E-3,2.0716578E-2,-6.603416E-3,2.9592074E-3],"split_indices":[2,2,2,1,69,17,73,7,10,68,0,67,5,3,71,67,0,8,4,62,7,29,9,53,7,0,43,53,0,71,48,0,68,3,7,73,0,0,62,0,5,0,0,0,0,0,0,0,0,7,4,0,62,0,4,1,0,0,0,0,9,0,0,0,69,48,69,44,7,12,66,71,0,1,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.47E2,4.91E2,4.56E2,1.51E2,3.4E2,3.72E2,8.4E1,5.1E1,1E2,3.24E2,1.6E1,2.5E1,3.47E2,4.7E1,3.7E1,4.8E1,3E0,5E1,5E1,1.21E2,2.03E2,1E1,6E0,1.5E1,1E1,9E0,3.38E2,4E1,7E0,2.5E1,1.2E1,8E0,4E1,3.9E1,1.1E1,4.6E1,4E0,7E0,1.14E2,4E0,1.99E2,8E0,2E0,4E0,2E0,1.1E1,4E0,2E0,8E0,2.51E2,8.7E1,8E0,3.2E1,8E0,1.7E1,5E0,7E0,3E0,3.7E1,2E0,3.7E1,7E0,4E0,4E0,4.2E1,1.8E1,9.6E1,1.8E2,1.9E1,1.4E1,2.37E2,8.2E1,5E0,2.9E1,3E0,1.5E1,2E0,3E0,2E0,3.1E1,6E0,1.4E1,2.8E1,1.2E1,6E0,6.3E1,3.3E1,1.6E2,2E1,1.7E1,2E0,6E0,8E0,2.18E2,1.9E1,6.8E1,1.4E1,2E0,2.7E1,2E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-6.081674E-3,-1.0869398E-1,7.389865E-2,-2.3181948E-1,-7.176332E-2,3.7519824E-2,2.7779657E-1,-2.0117164E-1,-2.1248613E-2,-1.2447308E-1,3.3326242E-2,-6.69673E-3,1.0148259E-1,3.6301067E-1,3.7330512E-2,-2.1777683E-1,4.268947E-2,-1.14593565E-1,-2.7200457E-2,1.5161091E-2,1.4984745E-3,4.045121E-3,-2.9956323E-1,2.2016162E-1,5.9940316E-2,2.952513E-1,5.8685744E-1,1.1551815E-1,-1.0974559E-2,-2.2826093E-1,1.3023084E-3,6.4646956E-3,-4.648826E-3,-1.5423042E-1,-3.934523E-2,1.9462908E-2,-2.6222962E-1,1.1393138E-2,-2.0619513E-2,-7.745512E-3,-2.3723017E-2,1.3988367E-1,3.4415564E-1,-1.0122101E-2,1.15745E-1,3.6969754E-1,1.9256526E-1,9.036485E-3,3.0041033E-2,5.8993142E-2,1.3230296E-2,-2.3816766E-1,-4.5895863E-2,-8.754832E-3,-1.9051512E-1,2.9237024E-2,-1.2656926E-1,2.590544E-1,-1.227377E-3,-1.2585518E-3,-1.734958E-2,2.1279808E-2,-1.5184453E-1,1.3190123E-2,1.0554259E-1,8.206187E-3,1.7774494E-2,-3.9867625E-2,9.819803E-2,2.2242974E-1,7.678582E-2,6.416634E-3,4.072662E-1,1.1437308E-2,1.2170103E-1,7.0452294E-3,-9.223316E-3,-1.1355862E-2,-1.1475537E-3,5.5590593E-3,-9.116569E-3,9.118676E-3,-2.4602984E-3,-6.025104E-3,-1.0920787E-2,-1.0178337E-3,6.9922414E-3,-6.97354E-3,1.161254E-3,1.6046407E-2,1.3679691E-3,-1.2371645E-3,4.5277704E-3,-1.3229524E-4,4.319167E-3,-3.705552E-3,-1.4526454E-2,7.154021E-3,1.8797466E-3,-9.704271E-4,-7.7311574E-3,1.1451358E-3,9.812831E-3,1.8625086E-3,1.32409725E-2,5.3621596E-3,-2.6277786E-3,1.9817734E-2,7.074418E-3,1.1007847E-3,8.108042E-3,2.6997195E-3,-7.586208E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,41,43,45,47,49,-1,51,-1,-1,-1,53,55,57,59,61,-1,-1,-1,63,65,67,69,71,73,-1,-1,75,-1,77,79,81,83,85,87,89,91,-1,-1,93,95,-1,97,-1,-1,99,101,103,105,-1,107,-1,109,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.3450594E0,1.7658296E0,3.7206135E0,5.288868E-1,1.6825396E0,1.2111549E0,1.5437722E0,3.3729506E-1,0E0,8.517573E-1,9.496307E-1,8.0027753E-1,8.495877E-1,6.761222E-1,4.7600833E-1,2.0222616E-1,9.632815E-2,5.8747077E-1,0E0,0E0,4.457122E-1,8.137829E-1,1.6708624E-1,3.9978743E-1,5.133568E-1,2.728157E-1,1.5280485E-1,1.4557387E-1,0E0,1.20954275E-1,0E0,0E0,0E0,6.845422E-1,4.2244193E-1,4.3655294E-1,1.4627141E-1,3.9540333E-1,0E0,0E0,0E0,1.13205016E-1,4.0986538E-2,1.940152E-1,2.8633416E-1,1.6145205E-1,4.947591E-2,0E0,0E0,8.9730345E-2,0E0,8.2429886E-2,1.4760564E-1,2.5451097E-1,2.5046754E-1,2.5094604E-1,1.1355206E-1,1.2805223E-1,2.0770794E-1,0E0,0E0,3.974915E-1,1.3727686E-1,0E0,7.239166E-2,0E0,0E0,1.093865E-1,1.0269222E-1,2.0440114E-1,2.8882986E-1,0E0,3.619623E-2,0E0,5.263494E-2,0E0,1.0264434E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,33,33,34,34,35,35,36,36,37,37,41,41,42,42,43,43,44,44,45,45,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,64,64,67,67,68,68,69,69,70,70,72,72,74,74,76,76],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,42,44,46,48,50,-1,52,-1,-1,-1,54,56,58,60,62,-1,-1,-1,64,66,68,70,72,74,-1,-1,76,-1,78,80,82,84,86,88,90,92,-1,-1,94,96,-1,98,-1,-1,100,102,104,106,-1,108,-1,110,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.7567694E5,7.7E1,7.098E3,1.1558494E8,2.9627435E2,3.9792767E2,1.3358269E2,1.0652307E7,-2.1248613E-2,4.8E1,4.427818E7,2E0,6.4651165E0,4.0277927E3,4.1831533E8,1.665909E3,1.84E2,2.118835E6,-2.7200457E-2,1.5161091E-2,3.8652172E1,4.093403E6,7.1E1,6.009825E2,7.5491136E-1,1.6386554E0,1.8917E4,4.435876E2,-1.0974559E-2,1.75E2,1.3023084E-3,6.4646956E-3,-4.648826E-3,2.308943E0,2.41E2,1.3358E4,1.0183612E2,1.2856612E0,-2.0619513E-2,-7.745512E-3,-2.3723017E-2,2.06E2,9.558022E7,5.603014E6,3.9914E4,3.97371E5,6.921944E7,9.036485E-3,3.0041033E-2,2.4344263E0,1.3230296E-2,1E0,9.61E2,1.3050649E4,1E0,1.8543E4,6.845E3,5.0935E4,8.076923E0,-1.2585518E-3,-1.734958E-2,1E0,2.3E1,1.3190123E-2,2.7857144E0,8.206187E-3,1.7774494E-2,4.5154482E-1,1.3508157E7,1.3E1,2.9487667E0,6.416634E-3,2.59672E3,1.1437308E-2,1.9125667E3,7.0452294E-3,1.5822886E0,-1.1355862E-2,-1.1475537E-3,5.5590593E-3,-9.116569E-3,9.118676E-3,-2.4602984E-3,-6.025104E-3,-1.0920787E-2,-1.0178337E-3,6.9922414E-3,-6.97354E-3,1.161254E-3,1.6046407E-2,1.3679691E-3,-1.2371645E-3,4.5277704E-3,-1.3229524E-4,4.319167E-3,-3.705552E-3,-1.4526454E-2,7.154021E-3,1.8797466E-3,-9.704271E-4,-7.7311574E-3,1.1451358E-3,9.812831E-3,1.8625086E-3,1.32409725E-2,5.3621596E-3,-2.6277786E-3,1.9817734E-2,7.074418E-3,1.1007847E-3,8.108042E-3,2.6997195E-3,-7.586208E-3],"split_indices":[43,44,2,60,67,67,73,66,0,3,46,6,71,4,47,4,0,9,0,0,68,66,0,70,53,69,2,71,0,6,0,0,0,69,10,9,48,53,0,0,0,10,7,9,9,1,47,0,0,69,0,16,2,43,8,1,44,10,69,0,0,27,8,0,69,0,0,53,60,3,68,0,67,0,4,0,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.93E2,3.91E2,5.02E2,8.9E1,3.02E2,4.27E2,7.5E1,8E1,9E0,2.01E2,1.01E2,2.53E2,1.74E2,5.5E1,2E1,7.5E1,5E0,1.98E2,3E0,9E0,9.2E1,2.45E2,8E0,4.4E1,1.3E2,4.4E1,1.1E1,1.6E1,4E0,7.2E1,3E0,3E0,2E0,1.29E2,6.9E1,8.7E1,5E0,2.42E2,3E0,6E0,2E0,2.8E1,1.6E1,5.8E1,7.2E1,2.4E1,2E1,2E0,9E0,1.3E1,3E0,6.8E1,4E0,2.6E1,1.03E2,3.9E1,3E1,6E0,8.1E1,2E0,3E0,2.29E2,1.3E1,4E0,2.4E1,4E0,1.2E1,4.6E1,1.2E1,1.8E1,5.4E1,4E0,2E1,1E1,1E1,5E0,8E0,6.6E1,2E0,2E0,2E0,4E0,2.2E1,4.5E1,5.8E1,2.8E1,1.1E1,2.6E1,4E0,4E0,2E0,6.5E1,1.6E1,1.72E2,5.7E1,1E1,3E0,1.3E1,1.1E1,4.1E1,5E0,8E0,4E0,5E0,1.3E1,4.2E1,1.2E1,1.8E1,2E0,4E0,6E0,6E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-8.858557E-3,-9.528278E-2,7.444525E-2,-1.7468317E-1,-4.9737826E-2,9.9790774E-2,-2.74006E-1,-2.5037542E-1,-1.4305419E-1,-1.8108556E-2,-1.6015732E-1,6.606898E-2,2.6703277E-1,-4.7951254E-1,-1.2672648E-1,-1.4002587E-1,-3.0281308E-1,-3.999318E-2,-1.9342616E-1,3.6314327E-2,-6.776587E-2,-4.0180278E-1,-1.1330983E-1,3.2309778E-2,1.8593225E-1,3.5820067E-1,7.766238E-2,-3.164725E-2,-1.6332356E-2,-1.6407542E-1,6.4928895E-3,-1.6725776E-1,1.990939E-3,-1.4631442E-2,-4.088114E-3,6.610627E-2,-8.782531E-2,-8.945042E-2,-2.2486097E-1,6.363492E-2,-1.8755329E-1,1.2332485E-1,-9.0562224E-2,2.2249694E-3,-2.4192955E-2,-1.564426E-1,5.4286137E-2,4.372391E-2,-2.5701845E-1,1.5305E-1,1.4799579E-2,3.8686612E-1,6.0897623E-3,1.5725179E-1,-2.0057939E-1,2.2848166E-4,-8.627433E-3,-7.421837E-2,-9.597099E-3,5.4390403E-3,-1.2702502E-2,-1.9550471E-1,-3.497831E-2,-7.4522994E-3,-2.638592E-3,-2.623885E-1,-1.2271964E-1,-2.3881989E-2,1.1124534E-1,4.9409807E-2,-1.45201795E-2,4.266413E-2,9.781395E-3,-1.3618143E-1,-3.6745075E-2,1.5967654E-2,-1.9519953E-1,1.2303535E-1,-8.974719E-3,1.713524E-1,2.2626286E-2,-3.9978097E-3,-3.4248585E-1,-5.194947E-3,1.7191288E-1,4.0475976E-1,6.215585E-4,-3.2604337E-3,2.1097323E-1,-2.6409735E-3,-1.2011962E-2,-7.187291E-3,2.1289375E-4,2.3471653E-3,-4.3110787E-3,-1.2770184E-2,-1.3027363E-3,-2.7853572E-3,3.703986E-3,-2.105787E-3,5.2063833E-3,-1.2825876E-2,-2.9240642E-3,-2.5762257E-3,-1.0259765E-2,7.0162583E-4,-1.2124933E-2,1.1688351E-2,4.0136576E-3,-1.8558308E-3,5.6857145E-3,5.7580248E-3,-1.4316916E-3,-3.5299137E-3,-8.413436E-3,2.1812606E-3,-5.1035727E-3,-3.7729696E-3,3.629238E-3,-1.6898973E-2,-7.271723E-3,1.4192759E-3,1.1821457E-2,3.7981772E-3,1.262435E-2,1.1782803E-2,4.8040648E-4,-1.8681385E-2,-6.0301255E-3,-4.658249E-3,3.3400327E-3,9.093802E-3,2.9347737E-3,7.100775E-3,2.0255432E-2,5.285795E-3,-7.350996E-3,1.1620492E-2,1.9329751E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,-1,57,-1,-1,-1,59,61,63,65,67,69,71,73,-1,-1,75,77,79,81,83,-1,85,-1,87,89,-1,-1,91,-1,-1,93,95,97,-1,99,101,103,105,107,109,-1,111,-1,113,115,117,119,121,-1,123,125,-1,127,129,131,133,-1,135,137,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.616204E0,1.6222572E0,4.16057E0,3.5680962E-1,1.0029364E0,2.4469066E0,9.0777445E-1,2.2956252E-1,6.0484815E-1,6.1052203E-1,6.7898273E-1,1.4709609E0,1.2324896E0,9.631348E-2,2.2185358E-1,9.180206E-2,5.089593E-2,2.088043E-1,2.3809409E-1,6.7248404E-1,5.251968E-1,5.5722463E-1,4.060809E-1,9.557331E-1,2.992866E-1,2.6855373E-1,5.8516896E-1,0E0,0E0,6.609863E-2,0E0,3.924924E-2,0E0,0E0,0E0,5.3575218E-2,1.5249649E-1,1.224819E-1,1.9778633E-1,4.068283E-1,3.6989778E-1,8.2247406E-2,2.552362E-1,0E0,0E0,2.9712892E-1,2.277292E-1,7.3929656E-1,1.3137853E-1,1.9790006E-1,0E0,2.8197193E-1,0E0,1.7314681E-1,3.5114944E-2,0E0,0E0,3.8681664E-2,0E0,0E0,3.4997933E-2,1.1151427E-1,6.1878774E-2,0E0,5.3927813E-2,9.755206E-2,1.0640296E-1,3.269386E-1,1.9843793E-1,3.7408076E-2,0E0,5.149047E-2,0E0,1.3994074E-1,3.1041926E-1,6.0568128E-2,1.7833817E-1,1.13579005E-1,0E0,3.2793236E-1,6.726263E-1,0E0,3.6111772E-2,6.570145E-2,1.3930237E-1,2.515192E-1,0E0,1.2655468E-1,8.843851E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,31,31,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,57,57,60,60,61,61,62,62,64,64,65,65,66,66,67,67,68,68,69,69,71,71,73,73,74,74,75,75,76,76,77,77,79,79,80,80,82,82,83,83,84,84,85,85,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,-1,58,-1,-1,-1,60,62,64,66,68,70,72,74,-1,-1,76,78,80,82,84,-1,86,-1,88,90,-1,-1,92,-1,-1,94,96,98,-1,100,102,104,106,108,110,-1,112,-1,114,116,118,120,122,-1,124,126,-1,128,130,132,134,-1,136,138,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.071E3,4.26E2,4.217427E7,3.89E2,6.516E3,1.5146261E3,4.4522205E6,5.3E1,1.288E3,1E0,2.2953334E2,2.9251662E6,1.3358269E2,2.2953334E2,3.3E1,3.175E1,1.870878E6,1.5E5,1.8540772E0,1.2E1,1.5953E4,1E0,5.48744E8,1.2784314E1,7.4711205E6,4.0278234E0,4.24038E8,-3.164725E-2,-1.6332356E-2,2.1650919E1,6.4928895E-3,2.4E1,1.990939E-3,-1.4631442E-2,-4.088114E-3,1.7097713E2,7.692308E-2,1E0,1.9470909E2,5.95E2,6.54E2,2.712E3,2.4438E4,2.2249694E-3,-2.4192955E-2,1.0865825E0,8.82E2,6E0,2.2227E4,4.8581E4,1.4799579E-2,1.0424884E-7,6.0897623E-3,6.95E2,1.3407223E5,2.2848166E-4,-8.627433E-3,5.322E3,-9.597099E-3,5.4390403E-3,6.101E3,3.0416667E0,6.5062125E5,-7.4522994E-3,1.6262903E2,2.3E1,8.24291E5,5.887958E0,3.28E2,2.4E1,-1.45201795E-2,4.3E1,9.781395E-3,2.0886075E0,2E0,1.3020051E-5,5.851852E0,1.5E1,-8.974719E-3,5.1042255E2,4.1E1,-3.9978097E-3,2.26E2,2.5912744E2,1.2333243E3,1.6471942E3,6.215585E-4,1.9513452E2,1.226E3,-2.6409735E-3,-1.2011962E-2,-7.187291E-3,2.1289375E-4,2.3471653E-3,-4.3110787E-3,-1.2770184E-2,-1.3027363E-3,-2.7853572E-3,3.703986E-3,-2.105787E-3,5.2063833E-3,-1.2825876E-2,-2.9240642E-3,-2.5762257E-3,-1.0259765E-2,7.0162583E-4,-1.2124933E-2,1.1688351E-2,4.0136576E-3,-1.8558308E-3,5.6857145E-3,5.7580248E-3,-1.4316916E-3,-3.5299137E-3,-8.413436E-3,2.1812606E-3,-5.1035727E-3,-3.7729696E-3,3.629238E-3,-1.6898973E-2,-7.271723E-3,1.4192759E-3,1.1821457E-2,3.7981772E-3,1.262435E-2,1.1782803E-2,4.8040648E-4,-1.8681385E-2,-6.0301255E-3,-4.658249E-3,3.3400327E-3,9.093802E-3,2.9347737E-3,7.100775E-3,2.0255432E-2,5.285795E-3,-7.350996E-3,1.1620492E-2,1.9329751E-3],"split_indices":[2,2,60,1,44,67,62,44,12,26,48,43,73,48,3,70,46,5,68,71,9,68,5,68,43,53,47,0,0,73,0,2,0,0,0,48,73,8,4,2,2,44,1,0,0,71,2,3,1,1,0,52,0,0,48,0,0,60,0,0,9,71,43,0,4,3,12,69,44,0,0,0,0,68,32,53,73,3,0,4,0,0,0,4,67,67,0,73,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.17E2,4.5E2,4.67E2,1.63E2,2.87E2,4.36E2,3.1E1,4.6E1,1.17E2,2.24E2,6.3E1,3.64E2,7.2E1,1.2E1,1.9E1,1.6E1,3E1,3.9E1,7.8E1,1.07E2,1.17E2,9E0,5.4E1,2.85E2,7.9E1,4.8E1,2.4E1,3E0,9E0,1.7E1,2E0,1.4E1,2E0,2.8E1,2E0,1.2E1,2.7E1,1.9E1,5.9E1,9.6E1,1.1E1,1.2E1,1.05E2,2E0,7E0,4.3E1,1.1E1,2.75E2,1E1,6.5E1,1.4E1,4.2E1,6E0,1.9E1,5E0,2E0,1.5E1,5E0,9E0,7E0,5E0,8E0,1.9E1,1E1,9E0,4.2E1,1.7E1,3.4E1,6.2E1,4E0,7E0,7E0,5E0,5.6E1,4.9E1,8E0,3.5E1,9E0,2E0,3.8E1,2.37E2,4E0,6E0,7E0,5.8E1,4E1,2E0,5E0,1.4E1,2E0,3E0,2E0,3E0,3E0,2E0,5E0,3E0,1.6E1,3E0,7E0,2E0,3.9E1,3E0,1.1E1,6E0,3E1,4E0,8E0,5.4E1,2E0,2E0,3E0,4E0,2.5E1,3.1E1,2.3E1,2.6E1,3E0,5E0,5E0,3E1,6E0,3E0,2.1E1,1.7E1,1.1E1,2.26E2,4E0,2E0,3E0,4E0,4.7E1,1.1E1,5E0,3.5E1,3E0,2E0,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"139","size_leaf_vector":"1"}},{"base_weights":[-7.2189667E-3,-1.6219811E-1,4.930272E-2,-1.15804926E-1,-3.2145506E-1,-9.541069E-3,1.67868E-1,-2.1646897E-1,-5.5274837E-2,-4.9996287E-1,-2.0352744E-1,-1.0822161E-1,2.0482019E-2,1.4988986E-1,4.3154186E-1,-3.2879418E-1,-1.8394129E-1,3.0687475E-1,-9.652984E-2,-1.5059096E-2,-6.792969E-1,-1.6946188E-1,-2.2681545E-2,9.67699E-2,-1.4841963E-1,-6.3385226E-2,5.379998E-2,8.113294E-2,2.0318875E-1,2.7478496E-2,5.849836E-2,-1.6327888E-1,-2.1904321E-2,-1.3343841E-1,-1.1795727E-2,7.1408194E-3,1.694499E-2,-7.031127E-2,-2.6005554E-1,-3.4971923E-2,-1.3176202E-2,-5.506272E-2,-2.1375868E-1,-4.937307E-3,6.088066E-3,-1.22096635E-1,-3.8322613E-1,-1.2848051E-1,1.6808722E-2,4.23546E-2,1.4335036E-2,-6.0197506E-2,1.03940256E-1,3.6281982E-1,1.7280419E-1,-3.2837985E-3,7.81816E-3,-1.1581877E-2,-7.1006274E-4,-2.7375078E-2,-1.650204E-1,-9.355595E-2,6.4217016E-2,-1.3585197E-2,-4.9716397E-3,1.036446E-3,-6.365402E-3,-2.2987771E-1,-1.6298833E-3,-3.4071314E-1,-9.882322E-2,-2.3655266E-2,-2.1724509E-4,-1.4616981E-1,5.596421E-3,-2.6160412E-2,1.0768481E-1,-5.1588483E-2,7.7967085E-2,-7.0402917E-4,-7.3387567E-3,2.0064577E-1,2.6850985E-2,3.9010617E-1,4.9041403E-3,3.4335952E-2,2.0652576E-1,-4.958385E-3,4.572994E-3,-9.288104E-3,-2.8835216E-3,-9.840432E-3,-2.8878695E-3,4.6058977E-3,-2.2131961E-3,-1.7878676E-3,-1.1580972E-2,-2.0938352E-2,-2.0761446E-3,7.5818633E-4,-6.5669566E-3,-4.4646547E-3,-1.231247E-2,-3.9997557E-3,3.291303E-3,-1.4908767E-3,8.784388E-3,7.676972E-3,-3.7958724E-3,4.3310085E-3,-3.2094626E-3,-9.438609E-3,1.079964E-2,8.321497E-6,9.103588E-3,2.0006152E-2,8.017387E-3,3.033584E-3,-7.5556245E-3,1.1442755E-2,2.9455917E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,43,45,47,49,51,53,-1,55,57,-1,59,-1,-1,-1,61,63,-1,-1,65,67,-1,-1,69,71,73,75,77,-1,79,81,83,85,-1,-1,-1,-1,87,89,91,93,-1,-1,-1,-1,95,-1,97,99,-1,-1,101,-1,103,105,107,109,-1,-1,111,113,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.637681E0,1.6842794E0,4.459796E0,1.0969977E0,9.928956E-1,1.27025E0,9.4027424E-1,1.8797112E-1,1.7477779E0,4.1188908E-1,2.3680568E-1,8.3838785E-1,9.227302E-1,7.164483E-1,7.379596E-1,2.5258255E-1,1.5972781E-1,4.981029E-2,4.276058E-1,0E0,4.592657E-2,1.453228E-1,0E0,1.3165677E-1,4.723978E-1,4.9286285E-1,6.7418885E-1,2.931366E-1,4.8053885E-1,0E0,8.262406E-2,9.773731E-2,0E0,1.0937202E-1,0E0,0E0,0E0,2.9057088E-1,3.7498474E-2,0E0,0E0,6.632519E-2,5.5318832E-2,0E0,0E0,3.5810184E-1,3.7741005E-1,2.4645919E-1,1.7072634E-1,7.6346123E-1,0E0,5.4782562E-2,5.686441E-1,7.825017E-2,4.4116783E-1,0E0,0E0,0E0,0E0,1.00128986E-1,8.053821E-2,2.7650976E-1,5.921304E-2,0E0,0E0,0E0,0E0,6.368923E-2,0E0,2.0684266E-1,3.4870178E-1,0E0,0E0,2.6608086E-1,0E0,1.8009965E-1,1.6490515E-1,4.233638E-1,3.740183E-1,0E0,0E0,4.8029125E-1,1.9817308E-1,5.6681633E-2,0E0,1.3218454E-1,4.2009377E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,33,33,37,37,38,38,41,41,42,42,45,45,46,46,47,47,48,48,49,49,51,51,52,52,53,53,54,54,59,59,60,60,61,61,62,62,67,67,69,69,70,70,73,73,75,75,76,76,77,77,78,78,81,81,82,82,83,83,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,44,46,48,50,52,54,-1,56,58,-1,60,-1,-1,-1,62,64,-1,-1,66,68,-1,-1,70,72,74,76,78,-1,80,82,84,86,-1,-1,-1,-1,88,90,92,94,-1,-1,-1,-1,96,-1,98,100,-1,-1,102,-1,104,106,108,110,-1,-1,112,114,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,2.1E1,2.483E3,2.71E2,2E0,1E0,5.428706E3,1.2772413E0,2.74614E5,4.238532E0,3.8652172E1,1E0,4.7567694E5,1.5484655E6,1.1806228E6,1.00019E5,2.772925E3,3.806E3,1.8776652E7,-1.5059096E-2,1.740204E2,3.5318289E3,-2.2681545E-2,3.4846212E5,3.6E1,1.944578E6,7.82261E6,2.4358286E7,3.5046455E6,2.7478496E-2,1.5E1,1.5857142E1,-2.1904321E-2,3.5419354E0,-1.1795727E-2,7.1408194E-3,1.694499E-2,3.019898E8,7.978062E6,-3.4971923E-2,-1.3176202E-2,1E0,1.7171982E6,-4.937307E-3,6.088066E-3,1.4076087E0,2.6676828E2,1.3092E4,5.1120926E4,2.61E2,1.4335036E-2,1.3E1,6.1E1,1E0,5.45901E6,-3.2837985E-3,7.81816E-3,-1.1581877E-2,-7.1006274E-4,1.7352941E0,1.4E2,6.782E4,1.8996E4,-1.3585197E-2,-4.9716397E-3,1.036446E-3,-6.365402E-3,6.123E3,-1.6298833E-3,9.61E2,2.308943E0,-2.3655266E-2,-2.1724509E-4,1.3E1,5.596421E-3,1.537007E7,1.185E3,2.920341E-4,7.1E1,-7.0402917E-4,-7.3387567E-3,1.1818181E0,1.2023678E-1,1.9001543E1,4.9041403E-3,2.9366477E0,2E0,-4.958385E-3,4.572994E-3,-9.288104E-3,-2.8835216E-3,-9.840432E-3,-2.8878695E-3,4.6058977E-3,-2.2131961E-3,-1.7878676E-3,-1.1580972E-2,-2.0938352E-2,-2.0761446E-3,7.5818633E-4,-6.5669566E-3,-4.4646547E-3,-1.231247E-2,-3.9997557E-3,3.291303E-3,-1.4908767E-3,8.784388E-3,7.676972E-3,-3.7958724E-3,4.3310085E-3,-3.2094626E-3,-9.438609E-3,1.079964E-2,8.321497E-6,9.103588E-3,2.0006152E-2,8.017387E-3,3.033584E-3,-7.5556245E-3,1.1442755E-2,2.9455917E-3],"split_indices":[43,3,2,2,32,29,67,68,5,68,68,68,43,43,43,5,62,1,60,0,61,62,0,43,3,9,43,5,60,0,3,4,0,69,0,0,0,7,62,0,0,20,60,0,0,69,4,10,48,70,0,3,8,28,60,0,0,0,0,68,12,9,44,0,0,0,0,43,0,2,69,0,0,8,0,9,44,53,8,0,0,68,72,71,0,57,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.7E2,2.32E2,6.38E2,1.81E2,5.1E1,4.27E2,2.11E2,6.7E1,1.14E2,1.9E1,3.2E1,9.9E1,3.28E2,1.99E2,1.2E1,1.3E1,5.4E1,1.1E1,1.03E2,1.1E1,8E0,3E1,2E0,1.6E1,8.3E1,9.3E1,2.35E2,8.8E1,1.11E2,8E0,4E0,7E0,6E0,3.3E1,2.1E1,4E0,7E0,9E1,1.3E1,6E0,2E0,9E0,2.1E1,2E0,1.4E1,7.6E1,7E0,5.1E1,4.2E1,2.26E2,9E0,1.2E1,7.6E1,1.6E1,9.5E1,2E0,2E0,4E0,3E0,8E0,2.5E1,7.7E1,1.3E1,1E1,3E0,5E0,4E0,1.9E1,2E0,6E0,7E1,5E0,2E0,4.8E1,3E0,2.9E1,1.3E1,6.2E1,1.64E2,9E0,3E0,3.3E1,4.3E1,1.4E1,2E0,1.9E1,7.6E1,5E0,3E0,1.8E1,7E0,1.5E1,6.2E1,1E1,3E0,2E0,1.7E1,4E0,2E0,1.9E1,5.1E1,3.5E1,1.3E1,1.8E1,1.1E1,5E0,8E0,7E0,5.5E1,1.49E2,1.5E1,2E0,3.1E1,3.8E1,5E0,1.1E1,3E0,1.7E1,2E0,5.9E1,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[-1.2916989E-3,-4.9708087E-2,1.5766631E-1,-1.4550227E-1,-8.740114E-4,2.4308054E-2,1.389092E-1,-1.1533898E-1,-3.9426038E-1,-1.2642704E-1,2.2247963E-2,-1.2802346E-1,1.6243298E-1,-1.2111002E-1,1.51222395E-2,-6.0678196E-1,-1.5141104E-1,-1.0575431E-1,-2.6353441E-2,-9.35543E-3,8.4328525E-2,-2.4853525E-4,-9.043571E-3,5.3023517E-2,2.1244965E-1,-1.0348475E-1,-2.66221E-1,-4.984944E-3,-3.799275E-2,-1.1346005E-2,-3.8680958E-3,-1.6083893E-1,-5.513465E-2,-5.1438667E-2,4.415334E-2,1.04701824E-1,-3.6096117E-1,1.23131916E-1,-1.2169208E-2,2.9055792E-1,1.1317146E-1,-1.6066384E-1,-1.3226903E-2,-1.3727869E-2,7.2297873E-4,-6.7266576E-2,-2.1829763E-1,-1.0657254E-2,-1.2262285E-1,-2.7881104E-2,-3.9663594E-2,6.469814E-2,-1.0868402E-1,5.8928035E-2,2.5073949E-1,-4.879169E-3,-2.7069297E-2,1.7216846E-1,-4.967994E-3,-5.1335096E-2,3.6801097E-3,1.6444868E-1,3.5552058E-1,1.6707034E-1,1.7379174E-2,9.905683E-4,-7.947007E-3,1.2353811E-2,-1.9411453E-3,-6.9936425E-3,-2.0397996E-4,-1.1482243E-2,-5.136038E-3,2.7019696E-3,-2.594342E-3,-7.05192E-3,-2.300823E-4,-3.240503E-3,4.614968E-3,6.6507435E-3,2.0951612E-3,-1.13537E-3,-1.1882223E-2,4.7579296E-3,-1.0265341E-3,1.6314351E-3,1.2715289E-2,4.3472163E-3,1.038981E-2,-3.746196E-3,5.099433E-3,-5.5432394E-3,9.6725335E-4,1.0867254E-2,-2.642035E-3,7.928043E-3,1.8137632E-2,1.2490721E-2,5.292411E-3,5.0248546E-3,-5.8406857E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,-1,27,29,31,-1,33,35,-1,-1,37,39,41,43,-1,-1,-1,-1,45,47,49,51,53,55,57,59,61,63,65,67,-1,-1,69,71,73,75,-1,77,79,81,83,85,-1,-1,87,89,91,-1,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.988265E0,3.255775E0,1.3788304E0,1.7178168E0,1.3440144E0,0E0,1.295486E0,5.757663E-1,1.198395E0,5.9531593E-1,7.679138E-1,1.3750458E-1,1.0120549E0,5.069928E-1,0E0,1.2772017E0,6.1811537E-2,1.863178E-1,0E0,5.8774495E-1,1.2353001E0,0E0,0E0,2.7573186E-1,9.5968676E-1,9.6811676E-1,1.8407011E-1,0E0,0E0,0E0,0E0,1.6274178E-1,1.1275775E-1,9.1842055E-1,3.6813316E-1,8.341812E-1,2.7858162E-1,1.8302107E-1,1.1848724E-1,5.231948E-1,2.9655606E-1,2.1732712E-1,5.9943074E-1,0E0,0E0,7.250442E-2,3.4415603E-2,7.7703744E-2,5.0450474E-2,0E0,6.072022E-1,1.5049103E-1,1.6696464E-1,3.4658155E-1,1.3703525E-1,0E0,0E0,6.3783824E-2,8.679809E-2,1.1505398E-1,0E0,4.1146213E-1,2.2696352E-1,1.6951323E-1,2.9872954E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,23,23,24,24,25,25,26,26,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,57,57,58,58,59,59,61,61,62,62,63,63,64,64],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,-1,28,30,32,-1,34,36,-1,-1,38,40,42,44,-1,-1,-1,-1,46,48,50,52,54,56,58,60,62,64,66,68,-1,-1,70,72,74,76,-1,78,80,82,84,86,-1,-1,88,90,92,-1,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.0976269E5,4.116997E6,5.119403E0,4.97E2,2.4308054E-2,1.7302156E5,4.797448E8,1.753E3,6.062295E6,1.9791039E6,5.092769E7,9.6215985E2,5.884866E7,1.51222395E-2,1.1742308E2,4.166152E4,3.51E2,-2.6353441E-2,1.852364E6,9.427676E2,-2.4853525E-4,-9.043571E-3,1.0717949E1,1.7103828E7,1.3428947E2,1.4912975E-1,-4.984944E-3,-3.799275E-2,-1.1346005E-2,-3.8680958E-3,1.6667E4,1.1644444E1,5E0,1.11E2,5.235714E2,2.62E3,5.27E2,4.439357E8,1.5484655E6,6.9E1,1.0204082E0,2.00087E5,-1.3727869E-2,7.2297873E-4,8.3E1,5.118881E0,9.144571E5,9E0,-2.7881104E-2,2.682E3,4.92E2,1.6869704E6,1.0164831E-2,5E1,-4.879169E-3,-2.7069297E-2,9.335507E7,7.25E2,8.81059E5,3.6801097E-3,2.1356704E0,4.454E3,5.6516E4,3.85374E8,9.905683E-4,-7.947007E-3,1.2353811E-2,-1.9411453E-3,-6.9936425E-3,-2.0397996E-4,-1.1482243E-2,-5.136038E-3,2.7019696E-3,-2.594342E-3,-7.05192E-3,-2.300823E-4,-3.240503E-3,4.614968E-3,6.6507435E-3,2.0951612E-3,-1.13537E-3,-1.1882223E-2,4.7579296E-3,-1.0265341E-3,1.6314351E-3,1.2715289E-2,4.3472163E-3,1.038981E-2,-3.746196E-3,5.099433E-3,-5.5432394E-3,9.6725335E-4,1.0867254E-2,-2.642035E-3,7.928043E-3,1.8137632E-2,1.2490721E-2,5.292411E-3,5.0248546E-3,-5.8406857E-3],"split_indices":[2,43,12,68,2,0,43,7,44,12,43,5,67,5,0,67,43,2,0,9,4,0,0,71,62,67,53,0,0,0,0,9,73,3,8,4,2,0,12,43,8,68,5,0,0,12,71,60,8,0,10,44,60,53,0,0,0,5,0,9,0,53,2,9,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.06E2,6.95E2,2.11E2,2.34E2,4.61E2,9E0,2.02E2,2.1E2,2.4E1,7.1E1,3.9E2,1.6E1,1.86E2,2.08E2,2E0,1.2E1,1.2E1,6.9E1,2E0,2.59E2,1.31E2,6E0,1E1,5.9E1,1.27E2,1.87E2,2.1E1,4E0,8E0,4E0,8E0,3.2E1,3.7E1,1.45E2,1.14E2,1.26E2,5E0,2.8E1,3.1E1,7E1,5.7E1,1.14E2,7.3E1,1.9E1,2E0,1.3E1,1.9E1,2.3E1,1.4E1,2E0,1.43E2,1.01E2,1.3E1,9.7E1,2.9E1,3E0,2E0,2E1,8E0,2.2E1,9E0,2.5E1,4.5E1,3.6E1,2.1E1,6E0,1.08E2,6E0,6.7E1,5E0,8E0,1.4E1,5E0,9E0,1.4E1,1.1E1,3E0,1.18E2,2.5E1,1.9E1,8.2E1,9E0,4E0,6.3E1,3.4E1,3E0,2.6E1,9E0,1.1E1,5E0,3E0,1.1E1,1.1E1,1.9E1,6E0,8E0,3.7E1,1.1E1,2.5E1,1.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-3.9536245E-3,-7.152332E-2,1.218133E-1,-1.1743735E-1,7.2424496E-3,1.5531582E-1,-7.894232E-2,-1.6179906E-1,-4.2473108E-2,-1.91994E-1,2.253351E-2,2.7746461E-2,1.4171582E-1,-2.2552762E-2,-4.0240515E-2,-1.3433757E-1,-2.8045022E-1,5.520872E-2,-8.936561E-2,-8.429819E-2,-2.2975061E-2,4.5218676E-2,-6.0642056E-2,1.9372614E-1,5.8928788E-2,2.7081229E-2,-1.8643577E-1,-1.212327E-1,-3.65965E-1,-4.5354947E-1,-1.9322959E-1,1.4610039E-1,2.337792E-2,-7.442977E-2,-3.4929907E-1,4.367712E-3,-1.4715324E-1,3.5077617E-2,1.4250865E-2,-1.2116804E-1,4.6175327E-2,9.4368644E-2,2.3949231E-1,3.4705266E-2,2.4578759E-1,-1.6750852E-2,9.883172E-3,-1.2402718E-2,-6.8092495E-2,-1.7306206E-1,-7.910823E-2,-2.4375293E-2,-1.0299265E-1,-2.8029481E-2,-1.2387807E-2,-6.0705982E-2,-1.0666647E-2,3.549688E-4,9.970306E-3,-1.7427064E-2,8.8744834E-2,-1.4932698E-1,-2.665857E-2,-2.2550792E-2,-4.524295E-3,-2.0593134E-4,-2.4156548E-1,3.9133616E-3,1.0925811E-1,-1.4126898E-1,4.6567945E-3,1.2025332E-1,-3.57315E-3,-5.251948E-2,1.4709248E-1,2.9656523E-1,1.14246145E-1,-7.9970965E-3,5.3097963E-2,8.792057E-2,1.609114E-2,7.42201E-2,-1.2906863E-1,5.324192E-3,-7.628058E-3,-3.557616E-3,-9.151587E-3,9.290503E-3,-4.844571E-3,-8.394324E-3,4.1109344E-4,4.1216523E-3,-5.82521E-3,-3.178026E-3,2.5851084E-3,7.744928E-3,1.9057657E-3,-2.114916E-3,-8.721154E-3,3.8337728E-3,-2.2499512E-3,-1.696436E-2,-4.128668E-3,-3.7157658E-4,8.617829E-3,7.923861E-3,6.148107E-5,-7.4801864E-3,3.2041867E-3,6.033255E-4,8.357567E-3,2.4055494E-3,-5.10716E-3,2.8480645E-3,8.657914E-3,1.8079247E-2,1.1252649E-2,-2.6230015E-2,7.865216E-3,3.692188E-3,-2.165311E-3,-2.237624E-3,7.8116762E-3,5.528225E-3,-1.8067048E-3,-7.268286E-3,2.1694384E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,-1,69,71,73,75,77,79,81,-1,-1,83,85,87,-1,89,-1,-1,91,-1,-1,-1,93,95,97,99,-1,-1,-1,101,103,105,107,-1,109,-1,111,113,115,117,-1,119,121,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.8778906E0,2.181572E0,2.1914268E0,1.2601595E0,6.8370396E-1,1.5838614E0,7.189657E-1,7.410674E-1,6.599517E-1,4.787644E-1,3.9520723E-1,0E0,1.1554499E0,0E0,4.3810216E-1,5.50581E-1,5.7666516E-1,1.3256273E-1,3.5058707E-1,1.5943971E-1,0E0,4.236647E-1,2.9653022E-1,7.3223686E-1,4.7126594E-1,2.555148E-1,1.14792824E-1,3.9458346E-1,3.619095E-1,2.3030162E-1,1.3448E-1,1.1181864E-1,9.756921E-2,3.2876664E-1,1.2367976E-1,0E0,1.3368054E-1,3.671454E-1,0E0,1.4314577E-1,1.6414206E-1,4.2592847E-1,7.738614E-1,3.7022483E-1,1.5940368E-1,2.752385E-1,0E0,0E0,1.4136975E-1,1.7501783E-1,7.545776E-1,0E0,4.4930883E-2,0E0,0E0,8.8210285E-2,0E0,0E0,0E0,8.928397E-2,4.51945E-2,1.3426852E-1,1.4141497E-1,0E0,0E0,0E0,8.05617E-2,2.454509E-1,3.098352E-1,1.2212473E-1,0E0,6.782499E-2,0E0,9.526775E-2,1.2276816E-1,2.935419E-1,1.4723566E0,0E0,2.3426163E-1,7.3414326E-2,0E0,8.033982E-2,4.4400632E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,48,48,49,49,50,50,52,52,55,55,59,59,60,60,61,61,62,62,66,66,67,67,68,68,69,69,71,71,73,73,74,74,75,75,76,76,78,78,79,79,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,-1,70,72,74,76,78,80,82,-1,-1,84,86,88,-1,90,-1,-1,92,-1,-1,-1,94,96,98,100,-1,-1,-1,102,104,106,108,-1,110,-1,112,114,116,118,-1,120,122,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,7.785302E5,1E0,1.131645E6,5.862757E1,3.3817584E7,4.75E0,4.68418E5,2.027338E0,2.0131578E0,4.066351E0,2.7746461E-2,8E0,-2.2552762E-2,1.1E1,4E1,4.48E2,9E0,1.8703672E6,1.2526851E6,-2.2975061E-2,3.113253E0,6.2136955E0,6.5969476E2,1.4669823E5,1.6322028E3,1.6E1,2.71E2,4.9969565E2,6.593462E4,1.8181819E0,1.5645E4,3.388854E2,1.5341322E2,9E1,4.367712E-3,2.2404422E2,1.158E3,1.4250865E-2,1.3997573E7,1.3E1,6.374563E5,1.7103828E7,8.147158E4,1.5E1,1.8645384E1,9.883172E-3,-1.2402718E-2,1.8E1,1E0,1.1E1,-2.4375293E-2,4.5E1,-2.8029481E-2,-1.2387807E-2,8.771579E2,-1.0666647E-2,3.549688E-4,9.970306E-3,9.4E1,2.701265E8,4.0753E4,9.624326E7,-2.2550792E-2,-4.524295E-3,-2.0593134E-4,1.5E1,1.944898E1,1.5321098E8,2.63E2,4.6567945E-3,7.54E2,-3.57315E-3,2.3847478E5,1.1813864E6,7.5052085E0,3.167E3,-7.9970965E-3,3.1790426E8,3.754E3,1.609114E-2,3.6828358E0,5.735198E-1,5.324192E-3,-7.628058E-3,-3.557616E-3,-9.151587E-3,9.290503E-3,-4.844571E-3,-8.394324E-3,4.1109344E-4,4.1216523E-3,-5.82521E-3,-3.178026E-3,2.5851084E-3,7.744928E-3,1.9057657E-3,-2.114916E-3,-8.721154E-3,3.8337728E-3,-2.2499512E-3,-1.696436E-2,-4.128668E-3,-3.7157658E-4,8.617829E-3,7.923861E-3,6.148107E-5,-7.4801864E-3,3.2041867E-3,6.033255E-4,8.357567E-3,2.4055494E-3,-5.10716E-3,2.8480645E-3,8.657914E-3,1.8079247E-2,1.1252649E-2,-2.6230015E-2,7.865216E-3,3.692188E-3,-2.165311E-3,-2.237624E-3,7.8116762E-3,5.528225E-3,-1.8067048E-3,-7.268286E-3,2.1694384E-4],"split_indices":[2,43,6,9,62,7,73,9,68,68,69,0,32,0,8,3,2,3,62,43,0,68,69,4,48,4,8,2,4,43,71,1,4,67,0,0,67,2,0,60,3,43,62,43,3,71,0,0,8,20,10,0,3,0,0,48,0,0,0,10,46,12,46,0,0,0,3,73,7,8,0,2,0,43,62,71,2,0,47,2,0,69,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.25E2,6.02E2,3.23E2,3.8E2,2.22E2,2.77E2,4.6E1,2.38E2,1.42E2,1.5E1,2.07E2,7E0,2.7E2,3E0,4.3E1,1.95E2,4.3E1,4.6E1,9.6E1,1.2E1,3E0,1.63E2,4.4E1,1.65E2,1.05E2,3E1,1.3E1,1.86E2,9E0,1.3E1,3E1,1.1E1,3.5E1,9.2E1,4E0,3E0,9E0,1.58E2,5E0,2.8E1,1.6E1,5.3E1,1.12E2,9.4E1,1.1E1,2.5E1,5E0,7E0,6E0,8.2E1,1.04E2,5E0,4E0,6E0,7E0,7E0,2.3E1,4E0,7E0,2.2E1,1.3E1,3.5E1,5.7E1,2E0,2E0,4E0,5E0,1.12E2,4.6E1,2.6E1,2E0,1E1,6E0,1.4E1,3.9E1,7.6E1,3.6E1,7E0,8.7E1,5E0,6E0,1.4E1,1.1E1,2E0,4E0,1.7E1,6.5E1,8E0,9.6E1,2E0,2E0,2E0,5E0,1.3E1,9E0,4E0,9E0,1E1,2.5E1,9E0,4.8E1,2E0,3E0,1.06E2,6E0,2.9E1,1.7E1,2.4E1,2E0,4E0,6E0,5E0,9E0,1.3E1,2.6E1,2.6E1,5E1,2E0,3.4E1,6.9E1,1.8E1,2E0,3E0,1E1,4E0,9E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[-1.1978592E-2,-9.093692E-2,6.270527E-2,-1.5185958E-1,-3.5736028E-2,1.3965842E-3,1.5107366E-1,-1.2217507E-1,-2.444778E-1,6.302164E-3,-1.986563E-1,1.15870545E-2,-2.4144885E-1,1.8114293E-1,-1.3329233E-2,-2.2026357E-1,-8.238421E-2,-1.6723898E-1,-5.2898526E-1,3.109302E-1,-1.4987595E-2,-2.533901E-1,2.1758956E-1,-1.9821467E-1,2.33366E-2,-1.5078638E-2,-5.50823E-3,2.1249366E-1,6.8125315E-2,-9.36558E-2,4.5424912E-2,-1.9703372E-1,-2.2314504E-2,1.2975313E-2,-9.13284E-2,2.757499E-3,-2.0895879E-1,-3.2356475E-2,-1.4039309E-2,5.134286E-3,1.5812447E-2,5.389492E-3,-1.617652E-1,-3.402439E-1,-1.4083871E-1,1.7555736E-2,2.0125418E-3,2.8045764E-3,-3.4853634E-1,8.958748E-3,1.412227E-1,1.6723004E-1,3.3224395E-1,-1.2938076E-1,1.3875255E-1,1.8427516E-3,-1.1894361E-2,1.05622865E-1,-6.1489684E-3,-1.2089671E-2,-1.2516077E-1,-7.146984E-2,-2.149824E-1,-1.6411637E-1,-1.817845E-2,3.843209E-2,-9.133476E-2,-1.9430871E-1,4.570087E-3,-2.6214712E-3,-3.645957E-1,2.2567776E-3,-1.6457495E-1,-1.9459285E-2,-3.2237442E-3,2.346056E-2,-1.018227E-1,1.1439371E-2,7.518521E-2,1.99842E-1,5.4380946E-2,2.0503806E-2,2.4917798E-1,-1.5725331E-2,8.384125E-3,-5.0130505E-2,1.951063E-1,-5.0918444E-4,7.1317605E-3,-1.842278E-3,-7.5548063E-3,-4.7551785E-3,-1.1296398E-3,-1.2954751E-2,-1.4944761E-3,-8.5458887E-4,-8.38187E-3,1.0045391E-3,7.792209E-3,-8.40887E-3,1.2574132E-4,-1.0279051E-2,-1.8087234E-3,-9.871189E-3,-1.9794015E-2,-8.601954E-3,-2.1855906E-3,1.98643E-3,-3.989004E-3,-1.0059445E-2,1.3621957E-3,6.172677E-3,1.6442942E-3,3.1247202E-3,1.0890899E-2,-3.1293384E-3,3.5872697E-3,8.084647E-3,1.900126E-2,3.4506882E-3,-4.8413216E-3,-5.0842394E-3,1.004205E-3,1.6116644E-3,1.0734942E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,-1,51,53,55,57,59,-1,-1,61,-1,63,-1,-1,-1,-1,65,67,69,71,-1,-1,-1,73,75,77,79,81,83,85,-1,-1,87,-1,-1,89,91,93,95,-1,97,99,101,-1,-1,103,-1,105,-1,-1,107,109,-1,111,113,115,-1,117,-1,119,121,123,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.2657485E0,1.4541085E0,2.4865365E0,5.3649235E-1,1.5696442E0,6.759186E-1,9.317875E-1,5.9853196E-1,9.883089E-1,1.1914988E0,1.1259226E0,6.5047526E-1,7.359886E-2,5.488906E-1,1.4583987E-1,1.7677093E-1,3.939544E-1,3.9042163E-1,1.6866922E-1,3.241229E-2,5.1530665E-1,3.6265206E-1,1.5089527E-1,5.650708E-1,4.2077714E-1,0E0,0E0,6.165595E-1,5.20127E-1,2.909829E-1,2.064482E-1,1.6191125E-1,0E0,0E0,2.5966138E-1,0E0,2.1320236E-1,0E0,0E0,0E0,0E0,4.8930538E-1,1.9680232E-1,1.3844633E-1,9.7881645E-2,0E0,0E0,0E0,1.4659071E-1,3.606454E-1,1.682688E-1,3.2771397E-1,1.8790865E-1,2.9040992E-1,2.9717606E-1,0E0,0E0,7.9563305E-2,0E0,0E0,6.373185E-2,1.3981235E-1,1.654014E-1,6.477517E-2,0E0,2.4414225E-1,3.286262E-1,7.027781E-2,0E0,0E0,1.04861975E-1,0E0,3.4448028E-2,0E0,0E0,4.2003027E-1,3.9656922E-1,0E0,3.6559224E-2,3.0762863E-1,6.60902E-2,0E0,1.8115616E-1,0E0,5.9542235E-2,3.207747E-2,1.1234313E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,28,28,29,29,30,30,31,31,34,34,36,36,41,41,42,42,43,43,44,44,48,48,49,49,50,50,51,51,52,52,53,53,54,54,57,57,60,60,61,61,62,62,63,63,65,65,66,66,67,67,70,70,72,72,75,75,76,76,78,78,79,79,80,80,82,82,84,84,85,85,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,-1,52,54,56,58,60,-1,-1,62,-1,64,-1,-1,-1,-1,66,68,70,72,-1,-1,-1,74,76,78,80,82,84,86,-1,-1,88,-1,-1,90,92,94,96,-1,98,100,102,-1,-1,104,-1,106,-1,-1,108,110,-1,112,114,116,-1,118,-1,120,122,124,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.072304E5,6.99E2,5.747068E2,1.481E3,2.2E1,2.727619E1,1E0,3.89E2,1.9E1,1.1E1,1.94E3,1.45064E5,7.785302E5,1.6553869E7,8.7390656E2,1.43E2,1.0952381E0,9.3E1,6.6E1,5E0,3.0161016E7,7.163082E7,1.6327E4,1E0,7.500379E8,-1.5078638E-2,-5.50823E-3,1.6978182E3,3.3580637E8,6E0,3.8807898E6,3.6E1,-2.2314504E-2,1.2975313E-2,2.0303884E6,2.757499E-3,1.2218E4,-3.2356475E-2,-1.4039309E-2,5.134286E-3,1.5812447E-2,3.84E2,1E0,7.6E2,1.754328E8,1.7555736E-2,2.0125418E-3,2.8045764E-3,1.8202913E2,1E0,7E0,7.1E1,1.1152E4,4.771E3,8.484036E5,1.8427516E-3,-1.1894361E-2,4E0,-6.1489684E-3,-1.2089671E-2,2.7444445E1,1.4495105E2,1.1320151E7,2.8367348E0,-1.817845E-2,7.87041E2,1.609E3,3.82E3,4.570087E-3,-2.6214712E-3,3.11626E8,2.2567776E-3,5.7E1,-1.9459285E-2,-3.2237442E-3,1E0,8.877906E6,1.1439371E-2,1.9913E4,4.20749E9,2.4131148E0,2.0503806E-2,4.0277927E3,-1.5725331E-2,1.159E3,4.8098917E0,1.2615725E5,-5.0918444E-4,7.1317605E-3,-1.842278E-3,-7.5548063E-3,-4.7551785E-3,-1.1296398E-3,-1.2954751E-2,-1.4944761E-3,-8.5458887E-4,-8.38187E-3,1.0045391E-3,7.792209E-3,-8.40887E-3,1.2574132E-4,-1.0279051E-2,-1.8087234E-3,-9.871189E-3,-1.9794015E-2,-8.601954E-3,-2.1855906E-3,1.98643E-3,-3.989004E-3,-1.0059445E-2,1.3621957E-3,6.172677E-3,1.6442942E-3,3.1247202E-3,1.0890899E-2,-3.1293384E-3,3.5872697E-3,8.084647E-3,1.900126E-2,3.4506882E-3,-4.8413216E-3,-5.0842394E-3,1.004205E-3,1.6116644E-3,1.0734942E-2],"split_indices":[43,2,67,44,3,61,6,1,3,10,0,7,43,66,4,2,68,0,10,3,60,7,2,68,5,0,0,67,7,8,43,2,0,0,60,0,44,0,0,0,0,0,16,2,46,0,0,0,67,19,3,8,10,2,43,0,0,8,0,0,4,4,5,69,0,67,2,0,0,0,46,0,6,0,0,30,60,0,9,46,68,0,4,0,0,69,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.91E2,4.33E2,4.58E2,2.05E2,2.28E2,2.71E2,1.87E2,1.57E2,4.8E1,1.82E2,4.6E1,2.61E2,1E1,1.58E2,2.9E1,4.4E1,1.13E2,3.9E1,9E0,1.1E1,1.71E2,4.1E1,5E0,1.3E1,2.48E2,5E0,5E0,1.23E2,3.5E1,1.2E1,1.7E1,4.2E1,2E0,2E0,1.11E2,6E0,3.3E1,4E0,5E0,2E0,9E0,1.51E2,2E1,2.2E1,1.9E1,2E0,3E0,5E0,8E0,2.22E2,2.6E1,9.1E1,3.2E1,9E0,2.6E1,7E0,5E0,1.3E1,4E0,2.1E1,2.1E1,9.7E1,1.4E1,2.8E1,5E0,1.13E2,3.8E1,1.8E1,2E0,2E0,2E1,2E0,1.7E1,6E0,2E0,1.97E2,2.5E1,9E0,1.7E1,7E1,2.1E1,1.2E1,2E1,3E0,6E0,6E0,2E1,4E0,9E0,7E0,1.4E1,5.8E1,3.9E1,1E1,4E0,3E0,2.5E1,1.01E2,1.2E1,1.9E1,1.9E1,1.5E1,3E0,7E0,1.3E1,1.4E1,3E0,1.68E2,2.9E1,1.3E1,1.2E1,6E0,1.1E1,1.5E1,5.5E1,3E0,1.8E1,1.5E1,5E0,4E0,2E0,3E0,3E0,4E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[4.0228732E-4,-2.8780382E-2,1.9977058E-1,-9.646922E-2,2.7882926E-2,2.4542823E-2,1.6975835E-1,-7.253952E-2,-2.8130737E-1,4.2946454E-2,-7.528858E-2,2.1385819E-1,-1.6824717E-2,-1.4210753E-1,-1.164984E-2,-5.4811805E-1,-1.8616498E-1,1.8674629E-2,1.078082E-1,-2.5726658E-1,-4.2711716E-2,1.16813175E-1,2.898347E-1,-7.138326E-2,1.08313775E-2,7.680921E-2,-1.4881511E-1,3.1742212E-1,-3.448618E-2,-1.1121349E-2,-3.321828E-2,-7.580209E-2,-2.725931E-1,2.6641926E-2,-3.3499894E-1,7.108295E-2,2.3478486E-1,-5.0908555E-3,-1.7566118E-2,3.972823E-2,-8.654293E-2,-8.344115E-3,1.4648251E-1,5.8185766E-3,3.25556E-1,-1.0809718E-2,-8.768058E-3,7.4551883E-3,-1.5015047E-3,-1.4102182E-1,-1.7785795E-2,4.670684E-3,1.64862E-2,-1.4328728E-2,-2.5821707E-1,-1.5714055E-1,1.5011639E-2,-1.5980268E-2,-6.9646616E-3,-1.042627E-2,8.287306E-2,-2.4929186E-2,-4.6730917E-3,5.0831E-3,1.5390897E-1,2.9196534E-1,5.569295E-3,-2.4250052E-3,5.081033E-3,-1.2035233E-1,1.195391E-3,2.093967E-1,8.568132E-2,4.012381E-1,1.9641572E-1,4.085017E-3,-1.4969987E-3,-9.010198E-3,-4.942934E-3,1.1767981E-3,-3.5650074E-3,-2.9344354E-3,-1.3392213E-2,-2.6520947E-3,-1.1301334E-2,-2.0171315E-3,3.9177244E-3,2.374631E-3,-1.7686092E-3,8.373063E-3,1.7586653E-3,-8.567723E-3,1.5511116E-3,1.3365959E-2,2.3827741E-3,3.9464612E-3,1.4855084E-2,-6.9458857E-3,9.165275E-4,1.6018633E-3,1.0716631E-2,6.783417E-3,1.0784038E-3,1.7104377E-3,2.0193465E-2,3.8639414E-3,1.0569783E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,49,51,53,-1,-1,55,57,59,61,63,65,-1,-1,67,69,-1,71,-1,73,-1,75,-1,-1,77,-1,-1,-1,79,81,83,85,-1,-1,87,89,-1,-1,91,93,95,-1,-1,-1,97,-1,99,101,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.148949E0,2.9636757E0,1.0226073E0,1.5365324E0,6.575209E-1,0E0,8.6756754E-1,1.324204E0,9.133599E-1,5.773339E-1,3.0913833E-1,5.928607E-1,3.0273592E-1,2.2581697E-1,1.2734329E0,3.4254456E-1,2.7502918E-1,7.6691926E-1,4.4963562E-1,1.0106468E-1,1.7326999E-1,3.690468E-1,2.3582554E-1,1.7947268E-1,0E0,5.0717484E-2,2.045958E-1,5.2864194E-2,7.088555E-1,0E0,0E0,1.1315866E-1,9.683788E-2,5.511422E-1,2.290054E-1,4.3107158E-1,1.0974729E-1,0E0,0E0,1.1427817E-1,1.2073757E-1,0E0,1.19767904E-1,0E0,2.9258776E-1,0E0,3.450195E-2,0E0,0E0,2.3395562E-1,0E0,0E0,0E0,3.6257336E-1,5.7270348E-2,5.490975E-2,3.5987347E-2,0E0,0E0,2.7348176E-1,4.528858E-1,0E0,0E0,2.4695772E-1,4.6496707E-1,5.2478552E-2,0E0,0E0,0E0,1.0041526E-1,0E0,5.461383E-2,7.182881E-2,2.6031518E-1,3.7067473E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,42,42,44,44,46,46,49,49,53,53,54,54,55,55,56,56,59,59,60,60,63,63,64,64,65,65,69,69,71,71,72,72,73,73,74,74],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,50,52,54,-1,-1,56,58,60,62,64,66,-1,-1,68,70,-1,72,-1,74,-1,76,-1,-1,78,-1,-1,-1,80,82,84,86,-1,-1,88,90,-1,-1,92,94,96,-1,-1,-1,98,-1,100,102,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,4.539777E5,1.059448E7,2.6E1,1E0,2.4542823E-2,1.8776652E7,5.47E2,3.662857E0,1.6451254E10,4.48E2,6.7947706E8,6.5026445E3,1.0204082E0,2.04115E5,5.2405543E0,2.9E1,2.3544883E8,2.593592E6,5E1,1.676E3,1.36E2,4.598404E-1,1.3655363E2,1.08313775E-2,1.5E1,9.687E3,6.511E3,7.042942E7,-1.1121349E-2,-3.321828E-2,3E0,1.318181E7,3.6746317E2,3.14088E5,2.3235295E0,9.51E3,-5.0908555E-3,-1.7566118E-2,3.0536E4,2.9307975E5,-8.344115E-3,5.6516E4,5.8185766E-3,2.1253996E7,-1.0809718E-2,3.49E2,7.4551883E-3,-1.5015047E-3,4.9E2,-1.7785795E-2,4.670684E-3,1.64862E-2,2E1,1.3344E4,1.7217412E4,2.0976269E5,-1.5980268E-2,-6.9646616E-3,1.8411793E0,2.283645E9,-2.4929186E-2,-4.6730917E-3,3.49E2,2.88864E2,1.54E2,5.569295E-3,-2.4250052E-3,5.081033E-3,2.4971938E7,1.195391E-3,1.4992306E8,1.4127266E6,1.4E1,3.359873E6,4.085017E-3,-1.4969987E-3,-9.010198E-3,-4.942934E-3,1.1767981E-3,-3.5650074E-3,-2.9344354E-3,-1.3392213E-2,-2.6520947E-3,-1.1301334E-2,-2.0171315E-3,3.9177244E-3,2.374631E-3,-1.7686092E-3,8.373063E-3,1.7586653E-3,-8.567723E-3,1.5511116E-3,1.3365959E-2,2.3827741E-3,3.9464612E-3,1.4855084E-2,-6.9458857E-3,9.165275E-4,1.6018633E-3,1.0716631E-2,6.783417E-3,1.0784038E-3,1.7104377E-3,2.0193465E-2,3.8639414E-3,1.0569783E-2],"split_indices":[2,43,12,3,6,0,60,2,73,46,2,7,4,68,5,68,0,12,9,0,44,0,53,71,0,0,44,1,60,0,0,8,12,67,1,68,44,0,0,1,48,0,9,0,62,0,8,0,0,1,0,0,0,8,46,43,43,0,0,68,46,0,0,10,67,0,0,0,0,60,0,5,43,73,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.83E2,7.71E2,1.12E2,3.51E2,4.2E2,8E0,1.04E2,3.12E2,3.9E1,3.67E2,5.3E1,8.4E1,2E1,1.45E2,1.67E2,9E0,3E1,2.68E2,9.9E1,7E0,4.6E1,3.8E1,4.6E1,1.7E1,3E0,4E0,1.41E2,1E1,1.57E2,4E0,5E0,1.4E1,1.6E1,2.63E2,5E0,7.8E1,2.1E1,4E0,3E0,1.6E1,3E1,3E0,3.5E1,9E0,3.7E1,4E0,1.3E1,2E0,2E0,1.38E2,3E0,2E0,8E0,1.45E2,1.2E1,7E0,7E0,9E0,7E0,1.59E2,1.04E2,2E0,3E0,4.4E1,3.4E1,1.3E1,8E0,7E0,9E0,2.3E1,7E0,1.6E1,1.9E1,2.2E1,1.5E1,2E0,1.1E1,5.3E1,8.5E1,8.9E1,5.6E1,2E0,1E1,4E0,3E0,4E0,3E0,4.9E1,1.1E2,3.2E1,7.2E1,5E0,3.9E1,1.4E1,2E1,2E0,1.1E1,1.9E1,4E0,2E0,1.4E1,9E0,1E1,2E0,2E1,4E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[1.3574356E-3,-5.6923922E-2,1.03860065E-1,-1.359288E-1,-1.979928E-2,1.2856267E-1,-7.942384E-2,-1.24590024E-1,-2.9468295E-1,-1.0919962E-1,1.0357695E-2,9.838611E-2,2.849399E-1,-2.145163E-2,-4.8016068E-2,-2.0647831E-1,-1.00475684E-1,-7.1034255E-3,-1.6117845E-2,7.941791E-2,-1.6140954E-1,-2.1840842E-3,1.8067154E-1,2.0037705E-1,6.493911E-2,3.8159743E-1,1.1732592E-1,9.15604E-2,-1.1344118E-1,-1.3960195E-1,-1.1619407E-2,-5.1999185E-2,-1.2908086E-1,-5.590348E-3,1.0415337E-1,-1.2933029E-1,-4.379664E-1,5.280003E-3,-1.0812954E-2,2.550021E-1,4.181789E-3,-3.122826E-3,2.1998465E-1,6.7316624E-3,1.3813281E-1,9.220759E-3,4.392523E-1,7.088671E-2,1.0844559E-2,1.5460695E-2,-1.0053978E-1,-3.4915596E-2,-2.0868853E-1,-7.891967E-3,-2.3783825E-3,-8.757541E-2,7.981558E-2,-1.5537179E-1,-5.2497912E-2,1.24404036E-1,-2.7355114E-3,1.0803801E-2,-1.6817425E-1,-1.2667985E-3,-3.1576566E-2,1.9193549E-2,-6.718777E-2,1.322028E-2,3.824102E-3,2.5103387E-1,9.319626E-2,4.9364258E-2,-6.1437756E-2,1.6182223E-1,-5.5720024E-2,4.7299263E-1,3.1578322E-3,5.6931577E-3,-8.186503E-3,4.207209E-3,-1.7928448E-1,3.8651895E-2,-6.279642E-3,-5.186765E-3,-1.3359968E-2,-2.1137123E-3,-8.431576E-3,8.263438E-3,-7.194685E-4,-6.163435E-3,-1.402606E-2,-5.487786E-3,-5.7544623E-4,9.137787E-3,3.5357198E-3,2.2369814E-3,-8.337642E-3,-1.3667614E-3,-9.8440545E-3,1.064239E-3,-1.2674806E-2,-4.731631E-3,2.9725407E-3,1.4259001E-2,7.6027955E-3,-9.060611E-3,7.5934683E-3,9.1198925E-3,1.2159384E-3,-5.4906905E-3,6.1942585E-4,1.0117437E-2,4.3655E-3,-5.950771E-3,7.2307126E-3,2.3901364E-2,1.1506595E-2,-2.2740024E-3,-1.364417E-2,-3.263761E-4,5.065173E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,35,37,39,41,43,45,47,49,51,53,-1,55,57,-1,59,61,63,65,-1,67,-1,-1,69,71,73,-1,75,77,-1,-1,79,81,83,-1,-1,85,87,89,91,93,-1,95,97,-1,-1,99,101,-1,-1,103,105,107,109,111,113,115,-1,-1,-1,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6035433E0,1.7512592E0,1.5495913E0,3.0710006E-1,1.1011169E0,1.3876438E0,4.6953756E-1,3.3469677E-1,3.54383E-2,1.0271326E0,6.5402246E-1,8.5004425E-1,7.3288393E-1,0E0,3.6652714E-1,8.111429E-2,1.8880963E-1,0E0,0E0,1.2608698E-1,6.6196823E-1,4.92887E-1,1.1873484E-1,3.3999228E-1,8.1707054E-1,2.3001432E-1,8.5692555E-2,6.3495684E-1,1.9465607E-1,3.9235145E-2,0E0,2.6022142E-1,1.6902566E-1,0E0,7.6623395E-2,4.0462577E-1,7.733537E-1,2.8159243E-1,0E0,3.2234013E-2,0E0,0E0,2.0493197E-1,3.169067E-1,3.99359E-1,0E0,2.4353552E-1,2.123867E-1,0E0,0E0,1.3654219E-1,1.2305461E-1,5.9579074E-2,0E0,0E0,1.6152778E-1,1.1464872E-1,1.7716324E-1,6.1615862E-2,5.131519E-2,0E0,1.2913735E-1,3.438679E-1,0E0,0E0,2.5810134E-1,2.1034183E-1,0E0,0E0,1.7366624E-1,2.941942E-1,2.2314084E-1,1.7935404E-1,2.67362E-1,1.7244439E-1,5.751419E-2,0E0,0E0,0E0,0E0,7.491249E-2,3.285122E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,34,34,35,35,36,36,37,37,39,39,42,42,43,43,44,44,46,46,47,47,50,50,51,51,52,52,55,55,56,56,57,57,58,58,59,59,61,61,62,62,65,65,66,66,69,69,70,70,71,71,72,72,73,73,74,74,75,75,80,80,81,81],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,36,38,40,42,44,46,48,50,52,54,-1,56,58,-1,60,62,64,66,-1,68,-1,-1,70,72,74,-1,76,78,-1,-1,80,82,84,-1,-1,86,88,90,92,94,-1,96,98,-1,-1,100,102,-1,-1,104,106,108,110,112,114,116,-1,-1,-1,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,4.58E2,2.856934E7,3.1E1,1E0,1.9856038E3,8.612209E-6,5.4E1,1.2E1,1.2E1,5.1167645E6,2E0,2.171312E0,-2.145163E-2,1E0,6.48334E5,3.8575E4,-7.1034255E-3,-1.6117845E-2,1.26E2,3.9824562E0,3.0977E4,8.773336E6,5.441268E7,1.8062708E6,1.1E1,9.033567E6,2.047619E0,1.4669823E5,3.6E1,-1.1619407E-2,1.8224286E2,3.328629E6,-5.590348E-3,8.785115E6,4.4463258E8,6.112797E6,4.3E1,-1.0812954E-2,6.576113E4,4.181789E-3,-3.122826E-3,2.737878E5,3.7315829E0,3.0359353E5,9.220759E-3,1E0,7.127857E6,1.0844559E-2,1.5460695E-2,8.26048E2,3.0070068E7,6.354E3,-7.891967E-3,-2.3783825E-3,4.5795207E0,1.3E1,1.9355932E1,4.795239E7,7E0,-2.7355114E-3,9.07021E5,1.6386554E0,-1.2667985E-3,-3.1576566E-2,3.8652172E1,1.7022566E7,1.322028E-2,3.824102E-3,5.194E3,1.48E2,9.3326636E-2,1.6876252E0,1.3587301E1,1.5896863E3,1E0,3.1578322E-3,5.6931577E-3,-8.186503E-3,4.207209E-3,1.401246E-8,1.132E3,-6.279642E-3,-5.186765E-3,-1.3359968E-2,-2.1137123E-3,-8.431576E-3,8.263438E-3,-7.194685E-4,-6.163435E-3,-1.402606E-2,-5.487786E-3,-5.7544623E-4,9.137787E-3,3.5357198E-3,2.2369814E-3,-8.337642E-3,-1.3667614E-3,-9.8440545E-3,1.064239E-3,-1.2674806E-2,-4.731631E-3,2.9725407E-3,1.4259001E-2,7.6027955E-3,-9.060611E-3,7.5934683E-3,9.1198925E-3,1.2159384E-3,-5.4906905E-3,6.1942585E-4,1.0117437E-2,4.3655E-3,-5.950771E-3,7.2307126E-3,2.3901364E-2,1.1506595E-2,-2.2740024E-3,-1.364417E-2,-3.263761E-4,5.065173E-3],"split_indices":[2,2,60,8,29,67,53,2,3,3,43,8,57,0,15,46,9,0,0,44,68,44,60,7,43,3,60,68,48,2,0,4,9,0,43,46,5,8,0,48,0,0,48,68,48,0,6,60,0,0,70,58,10,0,0,73,3,73,7,3,0,9,69,0,0,68,9,0,0,44,10,53,53,71,4,19,0,0,0,0,52,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.36E2,5.97E2,3.39E2,1.9E2,4.07E2,2.99E2,4E1,1.79E2,1.1E1,1.02E2,3.05E2,2.52E2,4.7E1,2E0,3.8E1,3.9E1,1.4E2,4E0,7E0,2.2E1,8E1,2.85E2,2E1,6.1E1,1.91E2,2.9E1,1.8E1,1.2E1,2.6E1,1.7E1,2.2E1,5.3E1,8.7E1,2E0,2E1,7.3E1,7E0,2.77E2,8E0,1E1,1E1,4E0,5.7E1,1.07E2,8.4E1,8E0,2.1E1,1.4E1,4E0,5E0,7E0,1.5E1,1.1E1,1.2E1,5E0,4.2E1,1.1E1,6.4E1,2.3E1,1.8E1,2E0,1.6E1,5.7E1,3E0,4E0,2.33E2,4.4E1,8E0,2E0,4.5E1,1.2E1,6.6E1,4.1E1,7.5E1,9E0,1.9E1,2E0,1.2E1,2E0,2E0,5E0,9E0,6E0,6E0,5E0,3E1,1.2E1,5E0,6E0,5.7E1,7E0,8E0,1.5E1,6E0,1.2E1,1.4E1,2E0,1.4E1,4.3E1,2.31E2,2E0,3.5E1,9E0,2.6E1,1.9E1,2E0,1E1,8E0,5.8E1,2.3E1,1.8E1,4E1,3.5E1,7E0,2E0,1.5E1,4E0,3E0,2E0,6E0,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[7.9264154E-4,-4.4239745E-2,1.335869E-1,-2.7481271E-2,-2.5029168E-1,3.1359744E-1,1.1445691E-1,-8.6570375E-2,1.2963068E-2,-1.9924644E-1,-2.8296638E-2,1.5592604E-2,3.0034613E-3,2.876087E-2,1.764237E-1,-9.542038E-2,1.4510682E-1,2.0306775E-2,-2.3656619E-1,-1.3014789E-1,-3.8993818E-1,5.0186287E-3,1.6941447E-2,-6.2927715E-2,1.9797143E-1,-8.874849E-2,-1.8940775E-2,3.6581953E-3,1.0469574E-2,-5.5642184E-3,9.910506E-2,-1.2519172E-2,-2.7835886E-3,-3.205554E-2,-1.7785843E-1,-4.9983254E-3,-4.7215164E-1,-8.304242E-3,3.1522717E-2,6.5047923E-3,-1.5645821E-1,2.7285272E-1,1.470053E-1,-1.8791322E-1,-6.7031436E-2,5.7094943E-2,-2.8654179E-2,1.5513492E-1,2.8709155E-2,-4.710666E-3,3.0507662E-3,-2.1063251E-2,-1.3226837E-1,-2.7273266E-2,-8.497011E-3,-6.550043E-2,5.6454647E-2,-9.792172E-3,1.8036549E-4,1.6960402E-3,3.014891E-1,1.05886884E-1,2.689694E-1,-1.2989687E-2,-7.5842054E-3,2.5668622E-3,-4.3028723E-3,1.0759035E-3,8.073658E-3,-7.267976E-4,-1.8451937E-2,9.1492245E-3,-4.325434E-3,6.2648426E-3,-1.0906789E-3,-8.378833E-3,-2.0032134E-3,-8.110847E-3,3.0321712E-3,7.0412452E-3,1.0326037E-3,1.0719725E-2,1.9387752E-2,8.388435E-3,1.6491712E-3,2.0795438E-2,9.248084E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,-1,-1,21,23,25,27,29,31,33,35,37,-1,39,41,43,-1,-1,-1,45,47,-1,-1,49,51,-1,53,-1,55,-1,57,59,61,63,65,67,69,71,73,-1,-1,-1,75,-1,-1,77,79,-1,-1,-1,81,83,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.5494094E0,2.3861306E0,7.733531E-1,1.5353864E0,8.293998E-1,9.7079754E-2,1.1313317E0,5.489863E-1,7.0480734E-1,5.685468E-1,0E0,0E0,0E0,7.2306025E-1,6.5198565E-1,4.881606E-1,3.5457596E-2,7.589069E-1,4.7295988E-2,1.6277999E-1,2.2792542E-1,4.240615E-1,0E0,2.3052365E-1,3.9495993E-1,5.214883E-1,0E0,0E0,0E0,4.0818614E-1,3.5841924E-1,0E0,0E0,9.4534E-2,2.4103492E-1,0E0,1.9055009E-1,0E0,1.8753983E-1,0E0,7.019967E-2,2.9361916E-1,3.2256746E-1,5.4332018E-2,6.4374006E-1,2.966845E-1,9.8158574E-1,5.446303E-1,2.353717E-1,0E0,0E0,0E0,8.621895E-2,0E0,0E0,2.3909396E-1,1.9501303E-1,0E0,0E0,0E0,2.3158002E-1,2.7685505E-1,1.234678E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,29,29,30,30,33,33,34,34,36,36,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,52,52,55,55,56,56,60,60,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,-1,-1,22,24,26,28,30,32,34,36,38,-1,40,42,44,-1,-1,-1,46,48,-1,-1,50,52,-1,54,-1,56,-1,58,60,62,64,66,68,70,72,74,-1,-1,-1,76,-1,-1,78,80,-1,-1,-1,82,84,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,9.932432E0,5.6E1,1.9685597E2,2E1,1E0,1.1362371E6,6.9307615E6,4.217427E7,4.9E1,-2.8296638E-2,1.5592604E-2,3.0034613E-3,5.428706E3,1E0,3.4E1,3.0714285E0,4.5742856E2,1.5224156E5,2.9333334E0,1E1,8.147158E4,1.6941447E-2,9.522029E6,5.6516E4,4.54E2,-1.8940775E-2,3.6581953E-3,1.0469574E-2,2.4344263E0,3.8275862E0,-1.2519172E-2,-2.7835886E-3,2.1369863E0,3.909839E-5,-4.9983254E-3,2.06E2,-8.304242E-3,1.96E2,6.5047923E-3,3.6196588E5,1E0,1.3388E4,1.1649485E0,1.5031847E0,1.4892023E5,2.9080848E6,2.7994856E5,1E0,-4.710666E-3,3.0507662E-3,-2.1063251E-2,1.8285715E1,-2.7273266E-2,-8.497011E-3,2.4076016E5,2.9E1,-9.792172E-3,1.8036549E-4,1.6960402E-3,1.798658E3,1.1621469E1,1.61E9,-1.2989687E-2,-7.5842054E-3,2.5668622E-3,-4.3028723E-3,1.0759035E-3,8.073658E-3,-7.267976E-4,-1.8451937E-2,9.1492245E-3,-4.325434E-3,6.2648426E-3,-1.0906789E-3,-8.378833E-3,-2.0032134E-3,-8.110847E-3,3.0321712E-3,7.0412452E-3,1.0326037E-3,1.0719725E-2,1.9387752E-2,8.388435E-3,1.6491712E-3,2.0795438E-2,9.248084E-3],"split_indices":[2,69,0,67,6,30,43,43,60,3,0,0,0,67,27,3,68,67,48,71,8,43,0,9,9,1,0,0,0,69,69,0,0,71,53,0,0,0,0,0,48,14,2,68,68,48,66,48,26,0,0,0,69,0,0,48,8,0,0,0,67,71,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,6.92E2,2.34E2,6.41E2,5.1E1,2.1E1,2.13E2,2.6E2,3.81E2,4.6E1,5E0,1.9E1,2E0,9E1,1.23E2,2.51E2,9E0,3.71E2,1E1,3.5E1,1.1E1,8.5E1,5E0,1E1,1.13E2,2.47E2,4E0,6E0,3E0,2.8E2,9.1E1,8E0,2E0,1.2E1,2.3E1,3E0,8E0,1E1,7.5E1,3E0,7E0,4.4E1,6.9E1,4.3E1,2.04E2,7.5E1,2.05E2,5E1,4.1E1,7E0,5E0,2E0,2.1E1,5E0,3E0,1.5E1,6E1,5E0,2E0,5E0,3.9E1,5.3E1,1.6E1,7E0,3.6E1,3.5E1,1.69E2,5.9E1,1.6E1,1.99E2,6E0,4.3E1,7E0,1.3E1,2.8E1,1.3E1,8E0,8E0,7E0,1.5E1,4.5E1,2.6E1,1.3E1,2.5E1,2.8E1,3E0,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-1.4364308E-3,-4.5348108E-2,1.3729109E-1,-3.130369E-2,-2.2986631E-1,9.967807E-2,2.4566352E-1,-1.004391E-1,-1.6922916E-3,-1.3230735E-1,-5.196475E-1,1.4132798E-1,-7.896753E-3,4.2193884E-1,1.3601227E-1,-9.384396E-2,-1.5392465E-2,1.5331711E-2,-1.11622855E-1,-6.926292E-2,-4.016094E-1,-2.7768543E-2,-1.0023415E-2,1.1323231E-1,2.4243838E-1,-4.7855858E-2,1.0029289E-1,4.8923E-3,2.09165E-2,2.0564346E-1,1.1903001E-2,-1.23957634E-1,-4.8041318E-2,1.9434078E-2,-2.4454677E-2,-2.0364422E-1,-6.836638E-2,-8.057613E-3,-1.5809703E-1,-2.4589624E-2,-8.096664E-3,1.2724423E-1,-3.479765E-2,2.8428018E-1,-2.2463684E-3,-7.346599E-2,7.859276E-3,-3.5676398E-4,6.935639E-3,4.5055253E-3,2.8185767E-1,4.4383765E-3,-3.965271E-3,-1.0858209E-2,-1.1234716E-1,8.317973E-3,-5.9656616E-2,-9.669772E-3,7.367606E-2,-2.3538868E-1,-1.1978145E-3,-1.5492505E-1,-3.377043E-2,-2.0162878E-3,3.7915858E-3,-1.0401596E-2,-4.146955E-3,1.2211185E-2,1.1212176E-1,2.8691874E-3,-5.782225E-3,7.024997E-3,1.5105689E-2,-1.3074471E-1,-1.2740276E-3,1.6940456E-2,5.814513E-3,-2.286893E-3,-6.3697426E-3,-4.2358594E-4,-4.196258E-3,-1.1038039E-3,5.638191E-3,6.0103633E-3,-1.0484676E-3,-1.1859677E-2,-3.0407484E-3,-5.691724E-5,-8.629151E-3,6.1438524E-4,-3.5346323E-3,8.366521E-4,6.6598533E-3,-7.936224E-3,5.424473E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,39,-1,-1,41,43,45,47,-1,-1,49,51,53,55,57,-1,59,61,63,65,-1,-1,67,69,71,-1,73,-1,-1,-1,-1,75,-1,-1,-1,77,-1,79,81,83,85,-1,87,89,-1,-1,-1,-1,-1,91,-1,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.574093E0,1.7938629E0,8.7227964E-1,1.3243661E0,1.3165135E0,7.403953E-1,1.0250244E0,2.6389146E-1,8.5135335E-1,6.173471E-1,1.5389156E-1,3.1059933E-1,2.0798819E-1,1.3763976E-1,3.086598E-1,2.5610912E-1,0E0,8.849021E-1,2.2888929E-1,1.7271394E-1,1.1039531E-1,0E0,0E0,2.0147538E-1,3.1785762E-1,2.0678496E-1,6.968734E-2,0E0,0E0,1.633535E-1,1.2176253E-1,1.1872256E-1,2.1359809E-1,6.195434E-1,0E0,1.0123038E-1,1.2353501E-1,6.677412E-2,3.880194E-2,0E0,0E0,1.4859414E-1,8.542028E-2,6.970596E-2,0E0,8.086465E-2,0E0,0E0,0E0,0E0,1.2244916E-1,0E0,0E0,0E0,1.565398E-1,0E0,1.1353147E-1,4.7417542E-1,7.3528767E-1,3.668654E-2,0E0,5.624497E-2,6.449325E-2,0E0,0E0,0E0,0E0,0E0,2.3145568E-1,0E0,0E0,0E0,0E0,8.1537575E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,45,45,50,50,54,54,56,56,57,57,58,58,59,59,61,61,62,62,68,68,73,73],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,40,-1,-1,42,44,46,48,-1,-1,50,52,54,56,58,-1,60,62,64,66,-1,-1,68,70,72,-1,74,-1,-1,-1,-1,76,-1,-1,-1,78,-1,80,82,84,86,-1,88,90,-1,-1,-1,-1,-1,92,-1,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,9.932432E0,1.9186024E3,4.97E2,1E0,2.7578741E1,1.7103828E7,4.2E1,1E0,4.6E1,1.753E3,8.83E3,1.3388E4,8E0,7.3274844E9,3.06E2,-1.5392465E-2,3.0011453E5,8.823713E-8,2.1E1,7.3E1,-2.7768543E-2,-1.0023415E-2,2.2E1,1E0,1.9616238E2,1.9501E4,4.8923E-3,2.09165E-2,4.493856E6,3.3571344E7,6.909091E0,1.4E1,3.5947333E2,-2.4454677E-2,2.7125288E5,8.314423E6,1.2E1,7.5052085E0,-2.4589624E-2,-8.096664E-3,6.617746E3,3.82E2,9.0505896E2,-2.2463684E-3,2.2715E4,7.859276E-3,-3.5676398E-4,6.935639E-3,4.5055253E-3,2.230298E2,4.4383765E-3,-3.965271E-3,-1.0858209E-2,3.5675676E0,8.317973E-3,1.8461539E0,8.8474586E8,8E0,1.3293153E3,-1.1978145E-3,5.47E2,7.09E2,-2.0162878E-3,3.7915858E-3,-1.0401596E-2,-4.146955E-3,1.2211185E-2,3.2231922E6,2.8691874E-3,-5.782225E-3,7.024997E-3,1.5105689E-2,5E0,-1.2740276E-3,1.6940456E-2,5.814513E-3,-2.286893E-3,-6.3697426E-3,-4.2358594E-4,-4.196258E-3,-1.1038039E-3,5.638191E-3,6.0103633E-3,-1.0484676E-3,-1.1859677E-2,-3.0407484E-3,-5.691724E-5,-8.629151E-3,6.1438524E-4,-3.5346323E-3,8.366521E-4,6.6598533E-3,-7.936224E-3,5.424473E-4],"split_indices":[2,69,67,2,6,71,62,8,6,3,44,2,2,3,5,2,0,48,52,8,0,0,0,3,6,71,9,0,0,1,66,67,12,67,0,48,5,8,71,0,0,48,0,4,0,44,0,0,0,0,73,0,0,0,69,0,68,5,32,4,0,2,10,0,0,0,0,0,62,0,0,0,0,6,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.13E2,6.94E2,2.19E2,6.46E2,4.8E1,1.64E2,5.5E1,1.93E2,4.53E2,3.7E1,1.1E1,1.18E2,4.6E1,2E1,3.5E1,1.89E2,4E0,3.93E2,6E1,3.1E1,6E0,8E0,3E0,9.4E1,2.4E1,3.4E1,1.2E1,2E0,1.8E1,2.2E1,1.3E1,1.13E2,7.6E1,3.91E2,2E0,1.8E1,4.2E1,1.9E1,1.2E1,3E0,3E0,8.6E1,8E0,2.1E1,3E0,3.1E1,3E0,4E0,8E0,1E1,1.2E1,7E0,6E0,9E0,1.04E2,3E0,7.3E1,2.55E2,1.36E2,1.5E1,3E0,1.1E1,3.1E1,1.4E1,5E0,5E0,7E0,7E0,7.9E1,4E0,4E0,6E0,1.5E1,1.3E1,1.8E1,7E0,5E0,3E1,7.4E1,2.8E1,4.5E1,2.31E2,2.4E1,8.6E1,5E1,1.3E1,2E0,2E0,9E0,1.5E1,1.6E1,2E1,5.9E1,1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[3.0786786E-3,-6.445692E-2,7.045975E-2,-1.291724E-1,-2.8358003E-2,4.270695E-2,2.09027E-1,-1.010938E-1,-2.956902E-1,-4.1852087E-2,6.365089E-2,-1.8280612E-2,1.0128621E-1,3.0202377E-1,3.8817417E-2,-1.576562E-1,-3.4429625E-2,-1.2265999E-1,-4.19355E-1,-4.7486424E-2,1.3945238E-1,1.268435E-1,-1.3609974E-2,2.411659E-1,-4.054163E-2,-3.471791E-2,1.2622465E-1,2.334105E-1,4.7407505E-1,1.0211908E-1,-1.65676E-1,-1.7169172E-2,-1.4390676E-1,1.0589575E-2,-6.443709E-2,-9.195795E-3,3.2395247E-4,-2.8485399E-2,-1.395998E-2,-7.561992E-2,-9.180719E-3,9.076511E-3,-8.5810915E-4,2.1517894E-3,8.530413E-3,6.0226596E-3,-5.0864894E-2,1.3646963E-2,1.1192673E-3,-1.8078113E-1,-6.437484E-3,6.0147494E-2,-1.3991751E-1,1.5337892E-1,-3.197447E-2,1.397545E-1,2.9811248E-1,1.2074243E-2,2.5139315E-2,6.9464752E-3,-2.0453742E-2,-1.0999824E-2,3.5426562E-4,-1.5999691E-1,-5.7337053E-2,-1.3558044E-2,-1.5828112E-1,-6.115742E-2,-2.633017E-1,1.5554178E-2,-1.8911133E-2,-1.230109E-2,-7.154729E-3,-1.1525686E-2,-1.2747736E-1,1.1461428E-1,-2.8576663E-2,1.3674979E-1,-4.312382E-2,-2.3016381E-1,-1.8130113E-4,1.6465673E-1,-5.4696687E-2,-7.285479E-3,-7.6688046E-4,8.06735E-3,1.1236767E-3,3.6214766E-1,3.612088E-3,-6.1616073E-3,2.365209E-3,-5.5186613E-3,-8.711466E-3,3.4550258E-3,-4.8485994E-3,-2.6036098E-3,4.190218E-3,-9.330295E-3,-8.836633E-4,-3.6835829E-3,9.025707E-5,8.785337E-4,-1.7995624E-2,8.457816E-4,-5.2209347E-3,-2.0926257E-3,2.9921264E-3,4.3903557E-3,-7.2459467E-3,7.5469385E-3,-4.973584E-4,-6.5097245E-3,-4.107642E-4,-9.798944E-4,8.317336E-3,-5.0704475E-3,8.474233E-4,-1.1416926E-3,-1.3273453E-2,8.570657E-3,3.8968615E-4,-1.04474025E-2,7.971633E-3,-4.9153054E-3,1.7964282E-3,1.8549241E-2,7.0854044E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,-1,63,-1,65,-1,-1,-1,-1,67,69,-1,-1,-1,-1,-1,71,-1,-1,73,75,77,79,81,83,85,87,-1,-1,-1,89,-1,-1,91,93,95,97,99,101,-1,103,105,-1,-1,107,109,111,113,115,117,-1,119,121,-1,123,-1,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.154708E0,1.0616251E0,1.7485256E0,7.3222804E-1,3.6768723E-1,1.3665872E0,1.1912756E0,5.262387E-1,4.4071007E-1,2.6915538E-1,1.8725431E-1,1.0955795E0,6.657696E-1,4.6779108E-1,3.7937233E-1,1.6789627E-1,5.3380936E-1,1.084137E-1,1.1540604E-1,2.6872402E-1,7.397479E-2,8.724627E-2,1.0255381E-1,1.6315293E-1,8.310345E-1,3.1778955E-1,7.137084E-1,1.8254757E-1,4.497099E-2,1.3045105E-1,8.773537E-2,0E0,9.250927E-2,0E0,2.8433567E-1,0E0,0E0,0E0,0E0,3.7476343E-1,3.6409098E-1,0E0,0E0,0E0,0E0,0E0,5.767432E-2,0E0,0E0,8.991134E-2,3.8115337E-1,1.403811E-1,1.832312E-1,3.398819E-1,9.715156E-2,6.20493E-2,2.6831496E-1,0E0,0E0,0E0,6.26611E-2,0E0,0E0,4.66491E-2,8.802667E-2,1.809451E-1,1.2002152E-1,1.5387839E-1,3.5620534E-1,0E0,3.6565238E-1,3.29621E-2,0E0,0E0,1.466822E-1,1.3312113E-1,2.6269495E-1,7.0191905E-2,3.432779E-2,9.5218E-2,0E0,4.0949988E-1,3.4589773E-1,0E0,9.087992E-2,0E0,0E0,6.241584E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,34,34,39,39,40,40,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,60,60,63,63,64,64,65,65,66,66,67,67,68,68,70,70,71,71,74,74,75,75,76,76,77,77,78,78,79,79,81,81,82,82,84,84,87,87],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,-1,64,-1,66,-1,-1,-1,-1,68,70,-1,-1,-1,-1,-1,72,-1,-1,74,76,78,80,82,84,86,88,-1,-1,-1,90,-1,-1,92,94,96,98,100,102,-1,104,106,-1,-1,108,110,112,114,116,118,-1,120,122,-1,124,-1,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,1.6471942E3,3.8170104E0,9.750085E6,1.1643481E6,2.2040408E7,2.71E2,3.89E2,6.663214E6,1E0,1.9184E4,5.80418E7,5.5766084E3,6.915691E7,1.0952381E0,1.1E1,1.317484E1,8.3E1,2.6801266E2,1E0,2.2584E4,1.3558896E7,9.522029E6,8.147158E4,2.9375E0,1E0,1E0,3.62349E5,3.1879792E8,7.661857E-2,-1.7169172E-2,5.2083335E0,1.0589575E-2,1.84E2,-9.195795E-3,3.2395247E-4,-2.8485399E-2,-1.395998E-2,2.5818555E2,2.1453061E2,9.076511E-3,-8.5810915E-4,2.1517894E-3,8.530413E-3,6.0226596E-3,4.4149057E2,1.3646963E-2,1.1192673E-3,9.558022E7,1.3826087E1,2.2293772E3,1.2915888E1,1.9269184E7,1.4598765E0,1.8365753E5,3.245783E0,1.2074243E-2,2.5139315E-2,6.9464752E-3,8E0,-1.0999824E-2,3.5426562E-4,6E0,1E0,7.86127E6,6.354E3,6.7723384E7,4.3E1,1.5554178E-2,2.306E3,3.364486E0,-7.154729E-3,-1.1525686E-2,1.24E2,6.479833E5,1E0,2.5178E4,3.574764E7,1.5594314E6,-1.8130113E-4,2.3821254E7,8.598131E1,-7.285479E-3,7.080933E2,8.06735E-3,1.1236767E-3,1.4595818E7,3.612088E-3,-6.1616073E-3,2.365209E-3,-5.5186613E-3,-8.711466E-3,3.4550258E-3,-4.8485994E-3,-2.6036098E-3,4.190218E-3,-9.330295E-3,-8.836633E-4,-3.6835829E-3,9.025707E-5,8.785337E-4,-1.7995624E-2,8.457816E-4,-5.2209347E-3,-2.0926257E-3,2.9921264E-3,4.3903557E-3,-7.2459467E-3,7.5469385E-3,-4.973584E-4,-6.5097245E-3,-4.107642E-4,-9.798944E-4,8.317336E-3,-5.0704475E-3,8.474233E-4,-1.1416926E-3,-1.3273453E-2,8.570657E-3,3.8968615E-4,-1.04474025E-2,7.971633E-3,-4.9153054E-3,1.7964282E-3,1.8549241E-2,7.0854044E-3],"split_indices":[2,43,67,68,9,43,62,2,2,43,26,1,7,4,59,68,10,69,12,70,16,1,9,9,43,69,6,29,1,47,72,0,69,0,0,0,0,0,0,4,67,0,0,0,0,0,4,0,0,7,73,48,73,62,69,48,68,0,0,0,8,0,0,10,20,5,10,7,0,0,44,71,0,0,0,43,26,1,7,60,0,9,73,0,4,0,0,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.11E2,4.55E2,4.56E2,1.62E2,2.93E2,3.81E2,7.5E1,1.4E2,2.2E1,2.56E2,3.7E1,1.87E2,1.94E2,4.8E1,2.7E1,7.5E1,6.5E1,1E1,1.2E1,2.49E2,7E0,2E1,1.7E1,1.4E1,1.73E2,3E1,1.64E2,3.6E1,1.2E1,2.1E1,6E0,3E0,7.2E1,6E0,5.9E1,6E0,4E0,3E0,9E0,1.43E2,1.06E2,5E0,2E0,9E0,1.1E1,3E0,1.4E1,1.1E1,3E0,3.3E1,1.4E2,1.6E1,1.4E1,1.4E2,2.4E1,1.6E1,2E1,4E0,8E0,1.5E1,6E0,4E0,2E0,6E1,1.2E1,3.9E1,2E1,1.34E2,9E0,2E0,1.04E2,1.1E1,3E0,1.3E1,2E1,2.1E1,1.19E2,9E0,7E0,8E0,6E0,1.33E2,7E0,4E0,2E1,1.2E1,4E0,1.5E1,5E0,2E0,4E0,2.6E1,3.4E1,3E0,9E0,2.8E1,1.1E1,1.5E1,5E0,1.04E2,3E1,3E0,6E0,7.5E1,2.9E1,8E0,3E0,2E0,1.8E1,1.5E1,6E0,1.7E1,1.02E2,2E0,7E0,3E0,4E0,2E0,6E0,1.18E2,1.5E1,4E0,3E0,5E0,1.5E1,1.2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[-8.921624E-3,-7.6879606E-2,4.7888946E-2,-1.6144326E-1,-4.5926206E-2,6.568831E-2,-1.2650245E-1,-1.40848E-1,-3.0428115E-1,-3.9121404E-2,-2.5337547E-2,4.5517128E-2,2.2901046E-1,-2.0555884E-1,-1.3451236E-2,-1.8124968E-1,-9.315151E-2,-1.2087336E-1,-2.0659603E-2,-5.275944E-2,4.8317872E-2,6.041845E-2,-5.3184327E-2,2.9596275E-1,9.7261354E-2,-2.2910775E-1,2.7533316E-3,-3.6877221E-3,9.522293E-2,-1.9834901E-1,-4.166966E-3,5.9366883E-3,-1.1110728E-1,-7.7925795E-3,-1.2926335E-4,-6.490744E-2,2.9898068E-2,-7.102481E-3,1.2697616E-1,1.7435363E-2,9.444917E-2,-1.4920545E-1,4.8629246E-3,1.852494E-1,3.82754E-1,-3.834322E-3,1.2487748E-1,-1.2265657E-2,-5.699915E-3,8.159282E-3,5.583144E-4,-1.545137E-1,-2.69874E-1,-1.2228635E-2,-1.5684304E-1,-5.798459E-2,-2.7419913E-1,1.0728019E-1,-3.6839407E-2,7.410692E-3,-2.8504057E-2,2.5239899E-3,8.019908E-3,3.2793332E-2,-3.1196073E-1,1.14639886E-1,8.544147E-3,-1.8788657E-1,2.4412156E-3,-5.8091115E-2,1.251196E-1,1.8180457E-3,9.456705E-3,2.0285493E-2,4.6548583E-3,5.530118E-2,1.0772137E-2,-8.020307E-3,-2.6513818E-3,-1.368288E-2,-3.4634443E-3,-7.2046253E-3,1.0063858E-3,-8.015378E-3,-2.0863675E-3,-2.9495447E-3,8.815101E-3,-3.4104132E-3,-2.407143E-2,7.4491883E-3,7.625106E-4,-3.6625247E-3,3.1204054E-3,-4.4599525E-3,2.4990293E-3,4.1416027E-3,-7.2253053E-4,-1.3505336E-3,-3.160236E-2,7.1243215E-3,1.9711433E-3,-1.4193182E-2,2.0535057E-3,-5.4972912E-5,-1.0594903E-2,3.1450028E-3,-4.9502114E-3,7.829481E-3,-1.5930517E-4,3.9504534E-3,-2.420974E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,-1,-1,49,51,-1,-1,53,-1,-1,55,57,59,61,63,65,67,69,71,73,-1,75,-1,-1,-1,-1,77,79,81,83,85,87,89,91,-1,93,-1,-1,95,97,99,101,103,-1,105,107,-1,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4937835E0,1.0710089E0,1.5386095E0,2.8073907E-1,1.0073016E0,1.4654678E0,4.0835637E-1,1.7196405E-1,2.7996242E-1,3.6029917E-1,0E0,5.915262E-1,4.034388E-1,1.8190384E-1,1.5065634E-1,6.372225E-2,2.0052186E-1,3.8124084E-2,0E0,2.6301265E-1,1.7963195E-1,5.073886E-1,2.9613495E-1,2.3658156E-1,9.962587E-2,5.9913278E-2,0E0,0E0,5.129789E-2,9.311962E-2,0E0,0E0,1.9706279E-1,0E0,0E0,3.1299973E-1,1.795379E-1,9.359724E-2,4.6838343E-2,7.943527E-1,3.3551013E-1,1.6589898E-1,2.646702E-1,3.4349322E-2,2.1305394E-1,0E0,1.0602033E-1,0E0,0E0,0E0,0E0,4.0890217E-2,4.9413085E-2,7.583828E-2,4.4348896E-2,3.093956E-1,3.042603E-1,7.428619E-2,8.80488E-2,0E0,1.3262534E-1,0E0,0E0,4.0650564E-1,7.0818144E-1,4.30789E-1,4.3800113E-1,1.2644762E-1,0E0,1.46992E-1,6.7258984E-2,0E0,0E0,0E0,0E0,3.9425425E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,28,28,29,29,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,63,63,64,64,65,65,66,66,67,67,69,69,70,70,75,75],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,-1,-1,50,52,-1,-1,54,-1,-1,56,58,60,62,64,66,68,70,72,74,-1,76,-1,-1,-1,-1,78,80,82,84,86,88,90,92,-1,94,-1,-1,96,98,100,102,104,-1,106,108,-1,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.48E2,6.829817E4,2.856934E7,1.3E1,6.3344407E0,1.4176E4,1.2218E4,3.175E1,1.6E1,3.2911258E6,-2.5337547E-2,2E1,8.788792E1,1.0652307E7,2E1,1.2E2,2.1818182E0,3.72E2,-2.0659603E-2,3.3967392E2,1.9130435E0,4.3764517E2,2.9847334E9,1.6978182E3,5E0,1.3146555E3,2.7533316E-3,-3.6877221E-3,2.284E3,2.6412E5,-4.166966E-3,5.9366883E-3,1.0005E5,-7.7925795E-3,-1.2926335E-4,2.829581E10,1E0,4.31903E6,5.1167645E6,4.75356E6,2.330807E0,9.277255E6,4E1,2.82E2,5.00401E5,-3.834322E-3,1.3407223E5,-1.2265657E-2,-5.699915E-3,8.159282E-3,5.583144E-4,1.7097713E2,2.8776436E0,2.4526315E0,7.668863E6,2.0200005E8,1.4596775E0,4.3024124E2,8.69E2,7.410692E-3,9.356961E6,2.5239899E-3,8.019908E-3,4.744186E0,1.1860938E8,8E0,1.2825651E0,1.0183612E2,2.4412156E-3,1.09E2,3.5861215E6,1.8180457E-3,9.456705E-3,2.0285493E-2,4.6548583E-3,7.6599895E9,1.0772137E-2,-8.020307E-3,-2.6513818E-3,-1.368288E-2,-3.4634443E-3,-7.2046253E-3,1.0063858E-3,-8.015378E-3,-2.0863675E-3,-2.9495447E-3,8.815101E-3,-3.4104132E-3,-2.407143E-2,7.4491883E-3,7.625106E-4,-3.6625247E-3,3.1204054E-3,-4.4599525E-3,2.4990293E-3,4.1416027E-3,-7.2253053E-4,-1.3505336E-3,-3.160236E-2,7.1243215E-3,1.9711433E-3,-1.4193182E-2,2.0535057E-3,-5.4972912E-5,-1.0594903E-2,3.1450028E-3,-4.9502114E-3,7.829481E-3,-1.5930517E-4,3.9504534E-3,-2.420974E-3],"split_indices":[2,43,60,8,68,2,44,70,3,43,0,3,73,66,3,44,69,2,0,67,68,67,46,67,8,67,0,0,0,7,0,0,5,0,0,46,26,60,43,62,53,9,3,0,10,0,48,0,0,0,0,48,68,68,5,5,68,4,2,0,60,0,0,71,5,32,69,48,0,0,43,0,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.03E2,4.11E2,4.92E2,1.09E2,3.02E2,4.47E2,4.5E1,9.7E1,1.2E1,2.99E2,3E0,3.99E2,4.8E1,2.6E1,1.9E1,5.1E1,4.6E1,6E0,6E0,2.59E2,4E1,3.47E2,5.2E1,3.1E1,1.7E1,2.4E1,2E0,1.2E1,7E0,4.2E1,9E0,3E0,4.3E1,4E0,2E0,2.26E2,3.3E1,2.4E1,1.6E1,1.54E2,1.93E2,1.9E1,3.3E1,1.5E1,1.6E1,2E0,1.5E1,1.7E1,7E0,3E0,4E0,2.8E1,1.4E1,1.4E1,2.9E1,2.2E2,6E0,1.5E1,1.8E1,2E0,2.2E1,7E0,9E0,1.48E2,6E0,1.56E2,3.7E1,1.6E1,3E0,2.2E1,1.1E1,2E0,1.3E1,1.3E1,3E0,1E1,5E0,2.3E1,5E0,1.2E1,2E0,2E0,1.2E1,2.5E1,4E0,2.16E2,4E0,4E0,2E0,9E0,6E0,1.3E1,5E0,1.2E1,1E1,6.8E1,8E1,4E0,2E0,1.01E2,5.5E1,3E0,3.4E1,3E0,1.3E1,6E0,1.6E1,8E0,3E0,8E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[4.5362087E-3,-4.4130668E-2,8.875919E-2,-1.0472095E-1,-7.970189E-4,1.1170269E-1,-5.030213E-2,1.206958E-1,-1.1213847E-1,7.1911463E-3,-2.3383375E-1,8.590957E-2,2.1612696E-1,-8.681679E-2,1.7075546E-1,9.664929E-3,-8.5462374E-4,-1.4056404E-1,-3.8256068E-2,-3.16209E-4,1.7183931E-1,-2.8072265E-3,-1.999497E-2,3.1457624E-1,7.168363E-2,3.1740004E-1,4.9790096E-2,-1.4027719E-1,3.9119262E-2,7.9749303E-4,1.0639496E-2,-1.0746438E-1,-2.0848285E-1,-8.500739E-2,3.0690517E-2,6.603008E-3,-1.4609142E-1,1.8821575E-3,1.1176962E-2,1.7774781E-2,3.1111273E-3,1.1187062E-1,5.1380293E-3,3.4582245E-1,7.3652836E-3,1.07018E-1,-1.6618839E-1,-4.5140557E-2,-1.8083166E-1,4.2853355E-3,-2.983315E-3,-1.5280303E-1,-7.163272E-2,-2.716933E-1,-8.106975E-2,-3.6157295E-2,-1.5296958E-1,8.288E-2,-6.058182E-3,3.1522833E-2,-2.7540136E-2,-1.2643363E-2,-6.859998E-2,1.3008414E-1,-5.3537052E-2,4.829315E-2,-6.3801005E-2,4.9060825E-3,3.6025226E-1,3.1306934E-2,8.336507E-3,-1.4310799E-4,-1.273859E-2,-7.2298963E-3,8.698886E-5,-5.0602045E-2,-2.2086805E-1,-7.6782373E-3,-1.7293291E-3,2.314785E-3,-5.8923475E-3,-1.4252904E-2,-2.444292E-3,6.128295E-3,-6.8185506E-3,-7.423609E-3,-1.6200355E-4,-8.853772E-3,-1.5165291E-3,4.931851E-3,-1.956954E-3,2.2942217E-3,-2.3480838E-3,-3.519819E-3,1.9300667E-3,-4.9776733E-3,7.34882E-4,1.750908E-3,7.850048E-3,-2.0027505E-2,2.102018E-3,-1.6292381E-4,6.3117025E-3,-4.1720085E-3,6.5092207E-3,7.2965333E-3,1.7670862E-2,7.6231766E-3,-1.0386792E-3,2.1401753E-3,-5.135199E-3,-1.1951339E-2,-2.6661246E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,37,-1,-1,39,41,43,45,47,49,-1,-1,51,53,55,57,59,61,-1,-1,-1,-1,63,65,67,-1,69,71,73,75,-1,-1,77,79,81,83,85,87,89,-1,91,93,-1,95,97,99,101,103,-1,105,107,-1,-1,-1,-1,-1,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6601086E0,1.4860051E0,1.0489297E0,4.0813446E-1,6.178498E-1,7.382858E-1,3.94201E-1,1.0074438E-1,4.7555685E-1,3.9683354E-1,3.4823966E-1,7.158803E-1,9.10696E-1,2.8152388E-1,5.8456615E-2,0E0,0E0,3.4853005E-1,2.1212631E-1,3.1162894E-1,1.1623216E-1,0E0,0E0,1.921432E-1,5.743927E-1,9.891653E-2,2.8970507E-1,1.03725374E-1,7.709256E-2,0E0,0E0,1.726296E-1,4.048276E-1,1.2394744E-1,2.3759729E-1,2.518663E-1,1.1779022E-1,0E0,0E0,0E0,0E0,4.106604E-1,2.4703711E-1,5.9099436E-2,0E0,9.2826635E-2,8.700496E-2,5.2151285E-2,9.310204E-2,0E0,0E0,6.3507676E-2,4.4083202E-1,2.505777E-1,2.843671E-1,9.62614E-2,6.582248E-2,6.52474E-2,0E0,2.5221714E-1,4.186123E-1,0E0,3.4126062E-2,4.265709E-1,5.407103E-1,2.3140591E-1,1.8146533E-1,0E0,4.45745E-2,7.479106E-2,0E0,0E0,0E0,0E0,0E0,3.9756633E-2,7.384604E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,41,41,42,42,43,43,45,45,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,62,62,63,63,64,64,65,65,66,66,68,68,69,69,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,38,-1,-1,40,42,44,46,48,50,-1,-1,52,54,56,58,60,62,-1,-1,-1,-1,64,66,68,-1,70,72,74,76,-1,-1,78,80,82,84,86,88,90,-1,92,94,-1,96,98,100,102,104,-1,106,108,-1,-1,-1,-1,-1,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,2.9726138E5,1E0,7E0,3.7E1,1.4176E4,1.419902E3,1.3050649E4,1.585814E6,7.82261E6,3.93617E0,1.8992E4,1.1797023E2,2.2318378E6,1.2E1,9.664929E-3,-8.5462374E-4,6E0,1.5341322E2,1.4979966E-5,6.83E2,-2.8072265E-3,-1.999497E-2,1.584E3,8E0,8.175577E11,4.1831533E8,1.9434364E7,4.888E3,7.9749303E-4,1.0639496E-2,3.8714287E1,1.162E3,1.44E2,1.244E3,1E0,6.6E1,1.8821575E-3,1.1176962E-2,1.7774781E-2,3.1111273E-3,3.3000212E5,5.1985867E1,1.1074469E6,7.3652836E-3,4.435876E2,1.3596486E5,7.653112E2,1.72797E5,4.2853355E-3,-2.983315E-3,1.6E1,2.1E1,4.88E2,4.911E3,1.0146154E2,4.9982754E4,3.01E2,-6.058182E-3,8.931E3,1.158E3,-1.2643363E-2,1E2,5.404E2,1.3934426E0,6.9706165E2,1E0,4.9060825E-3,9.427676E2,5.984843E6,8.336507E-3,-1.4310799E-4,-1.273859E-2,-7.2298963E-3,8.698886E-5,1.2083E4,9.18E2,-7.6782373E-3,-1.7293291E-3,2.314785E-3,-5.8923475E-3,-1.4252904E-2,-2.444292E-3,6.128295E-3,-6.8185506E-3,-7.423609E-3,-1.6200355E-4,-8.853772E-3,-1.5165291E-3,4.931851E-3,-1.956954E-3,2.2942217E-3,-2.3480838E-3,-3.519819E-3,1.9300667E-3,-4.9776733E-3,7.34882E-4,1.750908E-3,7.850048E-3,-2.0027505E-2,2.102018E-3,-1.6292381E-4,6.3117025E-3,-4.1720085E-3,6.5092207E-3,7.2965333E-3,1.7670862E-2,7.6231766E-3,-1.0386792E-3,2.1401753E-3,-5.135199E-3,-1.1951339E-2,-2.6661246E-3],"split_indices":[2,43,6,3,3,2,67,43,9,43,68,9,73,43,3,0,0,8,67,52,2,0,0,0,32,46,47,60,10,0,0,70,10,10,2,26,10,0,0,0,0,48,73,60,0,71,48,4,1,0,0,0,0,4,44,4,48,0,0,44,2,0,6,4,68,67,30,0,4,1,0,0,0,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.91E2,5.65E2,3.26E2,2.35E2,3.3E2,2.8E2,4.6E1,7E0,2.28E2,3.2E2,1E1,2.26E2,5.4E1,4E1,6E0,4E0,3E0,1.64E2,6.4E1,3.07E2,1.3E1,6E0,4E0,1.2E1,2.14E2,3.3E1,2.1E1,2.8E1,1.2E1,2E0,4E0,1.12E2,5.2E1,3.8E1,2.6E1,2.94E2,1.3E1,5E0,8E0,9E0,3E0,1.33E2,8.1E1,2.7E1,6E0,1.7E1,4E0,9E0,1.9E1,8E0,4E0,4.8E1,6.4E1,3.4E1,1.8E1,2.3E1,1.5E1,2E1,6E0,1.7E2,1.24E2,4E0,9E0,1.2E2,1.3E1,5E1,3.1E1,2E0,2.5E1,9E0,8E0,2E0,2E0,2E0,7E0,5E0,1.4E1,4.3E1,5E0,2E1,4.4E1,2.9E1,5E0,4E0,1.4E1,4E0,1.9E1,1.1E1,4E0,1.7E1,3E0,1.4E2,3E1,7.3E1,5.1E1,6E0,3E0,3.6E1,8.4E1,2E0,1.1E1,3.2E1,1.8E1,2.8E1,3E0,3E0,2.2E1,2E0,7E0,2E0,3E0,1.1E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-1.1836475E-3,-1.01313815E-1,3.3169307E-2,-1.9924475E-1,-6.269761E-2,-8.200639E-3,1.1345865E-1,-3.4886545E-1,-1.1650564E-1,-4.045731E-2,-1.7899042E-1,-6.6759076E-4,-1.4282607E-1,1.3695662E-1,-3.8417887E-2,-9.675466E-3,-4.606181E-1,-1.0904305E-2,-7.947481E-2,-1.36504E-1,-3.4827876E-3,-1.3024592E-2,-1.12900555E-1,8.041605E-3,-8.5783295E-2,-1.0202143E-1,-1.4944922E-2,1.6944523E-1,2.9192206E-2,5.1824383E-2,-1.4414741E-1,-2.6905451E-2,-1.7985527E-1,1.6565228E-2,-1.0854463E-1,-2.2275227E-1,-9.158501E-2,1.2668316E-2,-3.5389587E-2,-7.4095925E-4,-6.4863116E-3,9.169259E-2,-3.377635E-3,-5.0403427E-2,-1.2922996E-2,1.6962E-3,-1.2712978E-1,1.5095031E-1,3.978378E-1,-1.3162822E-2,4.8919607E-2,9.600296E-2,-3.8248065E-3,-1.8376242E-1,3.5144233E-3,-1.4654246E-5,-1.39264E-2,-3.7820928E-2,6.415465E-3,-8.988697E-3,-5.5442605E-2,-1.2164785E-2,-2.8768516E-4,-1.8775911E-3,-1.4757176E-1,2.1783884E-3,-5.0180275E-2,-1.0006912E-2,1.1011231E-1,-3.3812605E-2,1.4287291E-2,-5.0348244E-3,2.5065197E-2,-2.1037252E-3,-7.215585E-3,1.9169757E-1,6.969658E-2,8.214738E-3,2.3207642E-2,-2.510211E-2,1.12101935E-1,1.7204784E-2,9.217788E-3,-5.652107E-5,-2.38405E-1,2.4871135E-3,-3.9548115E-3,-5.3158833E-4,-5.719137E-3,-8.387972E-3,1.0153452E-3,-2.7299242E-4,-4.060479E-3,3.109204E-3,9.83414E-3,-1.1747014E-3,-1.7625067E-2,9.075645E-4,-6.8275584E-3,3.100945E-3,-5.4151844E-3,9.810136E-3,2.4481467E-3,1.9203978E-4,6.283117E-3,-6.238376E-3,3.0601321E-4,2.0450887E-3,9.761832E-3,-1.1257576E-3,5.027226E-3,-6.48065E-4,-1.3616525E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,-1,33,35,37,-1,39,41,43,45,-1,47,49,51,53,-1,55,57,59,61,63,-1,65,-1,-1,67,69,71,-1,-1,73,75,77,-1,79,81,-1,83,-1,-1,-1,85,-1,-1,87,-1,-1,-1,89,-1,91,-1,93,95,97,-1,99,-1,-1,101,103,-1,-1,105,107,109,-1,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1645067E0,8.762121E-1,2.2762027E0,7.764013E-1,4.324444E-1,4.5930997E-1,8.367901E-1,2.6613617E-1,1.7356074E-1,5.109089E-1,1.487208E-1,3.1944424E-1,1.4204046E-1,7.022793E-1,3.1279108E-1,0E0,2.9317236E-1,0E0,9.951344E-2,1.3456666E-1,9.338813E-1,0E0,4.414943E-2,3.7380493E-1,2.578326E-1,7.7660486E-2,0E0,5.9468174E-1,3.0948907E-1,1.14505656E-1,1.444459E-1,0E0,1.072603E-1,6.462937E-2,1.1076528E-1,1.0765582E-1,7.4325666E-2,0E0,1.1789168E-1,0E0,0E0,2.893494E-1,1.8596596E-1,1.5553111E-1,0E0,0E0,3.4285188E-2,4.6671462E-1,1.5803206E-1,0E0,2.1794976E-1,1.0933089E-1,0E0,1.2938333E-1,0E0,0E0,0E0,3.472209E-2,0E0,0E0,5.209297E-2,0E0,0E0,0E0,7.613897E-2,0E0,1.3274398E-1,0E0,1.831361E-1,3.6540878E-1,1.8727124E-1,0E0,1.0197187E-1,0E0,0E0,2.4572492E-1,2.1385466E-1,0E0,0E0,7.688012E-2,1.5733749E-1,3.649088E-2,0E0,0E0,1.1797768E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,18,18,19,19,20,20,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,38,38,41,41,42,42,43,43,46,46,47,47,48,48,50,50,51,51,53,53,57,57,60,60,64,64,66,66,68,68,69,69,70,70,72,72,75,75,76,76,79,79,80,80,81,81,84,84],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,-1,34,36,38,-1,40,42,44,46,-1,48,50,52,54,-1,56,58,60,62,64,-1,66,-1,-1,68,70,72,-1,-1,74,76,78,-1,80,82,-1,84,-1,-1,-1,86,-1,-1,88,-1,-1,-1,90,-1,92,-1,94,96,98,-1,100,-1,-1,102,104,-1,-1,106,108,110,-1,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.8988005E5,3.434405E7,5.867288E2,2E0,3.94E2,2E0,1E0,2.429044E0,2.8636363E0,3.06E2,2.443913E1,1E0,8.7E1,1.7493458E7,1E1,-9.675466E-3,6.0714287E1,-1.0904305E-2,3.7105262E0,3.89E2,2.74614E5,-1.3024592E-2,5.49E2,6E0,5.603014E6,3.202146E5,-1.4944922E-2,3.8776313E3,1.3926982E3,1E0,3.2645503E5,-2.6905451E-2,2.01E4,1.9375E0,5.04E2,3.9125E1,8.6E1,1.2668316E-2,9.25E0,-7.4095925E-4,-6.4863116E-3,9.3E1,2.524828E2,3.88E2,-1.2922996E-2,1.6962E-3,2.57E1,4.066351E0,3.53562E5,-1.3162822E-2,1.4669823E5,7E0,-3.8248065E-3,1.2593857E0,3.5144233E-3,-1.4654246E-5,-1.39264E-2,8.683527E3,6.415465E-3,-8.988697E-3,2.772925E3,-1.2164785E-2,-2.8768516E-4,-1.8775911E-3,1.4979E4,2.1783884E-3,4.135135E0,-1.0006912E-2,3.0714285E0,3.6E1,5.747068E2,-5.0348244E-3,2.77843E5,-2.1037252E-3,-7.215585E-3,8.992806E-4,5.0935E4,8.214738E-3,2.3207642E-2,7.958E3,6.46E2,4.5E1,9.217788E-3,-5.652107E-5,2.483E3,2.4871135E-3,-3.9548115E-3,-5.3158833E-4,-5.719137E-3,-8.387972E-3,1.0153452E-3,-2.7299242E-4,-4.060479E-3,3.109204E-3,9.83414E-3,-1.1747014E-3,-1.7625067E-2,9.075645E-4,-6.8275584E-3,3.100945E-3,-5.4151844E-3,9.810136E-3,2.4481467E-3,1.9203978E-4,6.283117E-3,-6.238376E-3,3.0601321E-4,2.0450887E-3,9.761832E-3,-1.1257576E-3,5.027226E-3,-6.48065E-4,-1.3616525E-2],"split_indices":[43,7,67,32,0,6,6,68,69,2,73,19,0,66,8,0,68,0,69,1,5,0,0,3,9,43,0,67,4,113,48,0,1,68,7,4,10,0,73,0,0,12,4,10,0,0,61,69,1,0,48,6,0,68,0,0,0,43,0,0,62,0,0,0,1,0,69,0,68,3,67,0,1,0,0,72,10,0,0,2,0,3,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.18E2,2.34E2,6.84E2,6.5E1,1.69E2,4.52E2,2.32E2,2.2E1,4.3E1,1.43E2,2.6E1,4.29E2,2.3E1,2.01E2,3.1E1,1.1E1,1.1E1,9E0,3.4E1,3.9E1,1.04E2,9E0,1.7E1,3.9E2,3.9E1,2E1,3E0,1.54E2,4.7E1,1.7E1,1.4E1,7E0,4E0,8E0,2.6E1,1.2E1,2.7E1,1E1,9.4E1,4E0,1.3E1,4.6E1,3.44E2,3.4E1,5E0,3E0,1.7E1,1.44E2,1E1,2E0,4.5E1,1.3E1,4E0,1.2E1,2E0,2E0,2E0,6E0,2E0,9E0,1.7E1,1E1,2E0,1.5E1,1.2E1,1.4E1,8E1,2E0,4.4E1,1.26E2,2.18E2,1.9E1,1.5E1,5E0,1.2E1,9.5E1,4.9E1,4E0,6E0,2.1E1,2.4E1,8E0,5E0,3E0,9E0,2E0,4E0,1.1E1,6E0,1E1,2E0,3.7E1,4.3E1,3.2E1,1.2E1,1.24E2,2E0,2.12E2,6E0,1.2E1,3E0,8.3E1,1.2E1,2.5E1,2.4E1,4E0,1.7E1,1.5E1,9E0,6E0,2E0,2E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[9.420579E-4,-4.8745293E-2,5.5886988E-2,-5.3756088E-2,1.6987044E-2,6.3816356E-3,1.1253619E-1,-3.6974512E-2,-1.581517E-1,1.3828398E-1,-5.693381E-3,1.3153665E-1,-4.5186378E-2,-5.0025888E-2,9.554261E-2,-1.0322398E-1,-4.283997E-1,3.962532E-3,1.0449944E-2,-1.27457E-1,3.8846144E-3,1.07151255E-1,2.3668495E-1,-1.5652135E-1,-1.3509471E-2,-3.851669E-2,-1.3782364E-1,2.0127816E-1,-3.3123467E-2,-1.4985776E-1,-9.458596E-3,-2.4490207E-2,4.1657485E-4,-1.6467659E-1,4.1261604E-3,6.2446542E-2,-2.4575775E-2,2.7247724E-1,9.103419E-2,2.910399E-1,1.1278085E-1,-5.0429266E-4,-1.1628089E-2,1.731741E-2,-4.8036845E-3,-8.576285E-2,-9.943798E-3,-2.6286596E-1,-8.3287264E-4,1.2554523E-2,7.972738E-2,-6.988158E-2,3.5542916E-3,-1.6635777E-1,4.562154E-3,4.0382747E-2,-5.013054E-3,-8.9025893E-4,-1.13200415E-2,-1.0777121E-1,7.496467E-2,9.904372E-3,-3.1431906E-2,1.6392343E-2,7.141847E-3,3.216468E-2,1.2874782E-1,1.4398138E-2,2.985661E-3,9.354935E-3,9.7319373E-4,-2.7824104E-2,5.2630417E-3,-5.9742955E-3,-2.505756E-3,2.3336036E-3,-1.9067905E-3,-1.3223644E-2,-1.1614695E-3,1.1061141E-2,-3.119882E-3,1.0534597E-3,8.665708E-3,2.4007028E-3,-4.391869E-3,-8.620866E-3,-3.5653955E-3,-2.503634E-3,8.137693E-3,-9.130239E-3,7.766095E-4,-6.007928E-4,5.134663E-3,-8.851401E-4,-7.882128E-3,-2.6482908E-4,6.797009E-3,8.871758E-3,2.6763293E-3,1.3795558E-3,-4.778248E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,-1,-1,33,35,37,39,41,43,45,47,49,51,53,55,-1,-1,57,-1,59,61,63,65,67,69,-1,-1,71,-1,73,75,77,79,-1,81,83,-1,85,-1,87,-1,-1,-1,89,91,-1,93,-1,-1,95,97,-1,-1,-1,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5526078E0,1.0381954E0,1.2444527E0,8.45587E-1,0E0,3.798554E-1,6.2628126E-1,7.311823E-1,9.503341E-1,7.259157E-2,2.5590414E-1,4.4899416E-1,7.900067E-2,3.81706E-1,5.236411E-1,2.4782276E-1,4.7708058E-1,0E0,0E0,1.4334661E-1,3.4154052E-1,3.8020992E-1,1.9615948E-1,6.5396585E-2,5.4373063E-2,4.581334E-1,7.534853E-1,1.537565E-1,7.782222E-2,1.7178518E-1,1.02372296E-1,0E0,0E0,1.5574253E-1,0E0,1.5084317E-1,2.3143971E-1,6.9527805E-2,3.0668437E-1,7.6892495E-2,9.319553E-2,0E0,0E0,6.804948E-2,0E0,1.673286E-1,3.9974803E-1,1.10357285E-1,3.638158E-1,0E0,5.0538946E-2,4.6637684E-2,0E0,4.7163486E-2,0E0,1.8694988E-1,0E0,0E0,0E0,5.838474E-2,1.9764516E-1,0E0,2.2786283E-1,0E0,0E0,2.4248964E-1,3.6432588E-1,0E0,0E0,0E0,0E0,5.0422132E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,36,36,37,37,38,38,39,39,40,40,43,43,45,45,46,46,47,47,48,48,50,50,51,51,53,53,55,55,59,59,60,60,62,62,65,65,66,66,71,71],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,-1,-1,34,36,38,40,42,44,46,48,50,52,54,56,-1,-1,58,-1,60,62,64,66,68,70,-1,-1,72,-1,74,76,78,80,-1,82,84,-1,86,-1,88,-1,-1,-1,90,92,-1,94,-1,-1,96,98,-1,-1,-1,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.502167E5,7.626E3,5.281628E2,1E0,1.6987044E-2,1.1333333E0,1E0,7.262122E2,9.639872E0,2.4177586E2,1.279012E7,5.6530495E6,5E0,2.0734E4,2.08E4,4.5E1,1.753E3,3.962532E-3,1.0449944E-2,1.407E3,3.5108426E-3,1.9197379E9,1.8029197E0,4E0,1.4916515E3,1.545E2,1.862E3,4.4889745E6,8.93E2,1E0,1.33E2,-2.4490207E-2,4.1657485E-4,1.1015E4,4.1261604E-3,2.0966116E2,1.1160929E0,3.1158695E6,1.7587205E6,1.0593E5,6.968415E6,-5.0429266E-4,-1.1628089E-2,1.7234043E1,-4.8036845E-3,1.3E1,2.14E0,5.869281E0,2.992648E-2,1.2554523E-2,3.425E1,5.526796E7,3.5542916E-3,2.8096385E0,4.562154E-3,5.8E1,-5.013054E-3,-8.9025893E-4,-1.13200415E-2,1.4283589E6,1.3287E4,9.904372E-3,4.5158855E6,1.6392343E-2,7.141847E-3,1.9616238E2,2.1089442E0,1.4398138E-2,2.985661E-3,9.354935E-3,9.7319373E-4,4.4133335E1,5.2630417E-3,-5.9742955E-3,-2.505756E-3,2.3336036E-3,-1.9067905E-3,-1.3223644E-2,-1.1614695E-3,1.1061141E-2,-3.119882E-3,1.0534597E-3,8.665708E-3,2.4007028E-3,-4.391869E-3,-8.620866E-3,-3.5653955E-3,-2.503634E-3,8.137693E-3,-9.130239E-3,7.766095E-4,-6.007928E-4,5.134663E-3,-8.851401E-4,-7.882128E-3,-2.6482908E-4,6.797009E-3,8.871758E-3,2.6763293E-3,1.3795558E-3,-4.778248E-3],"split_indices":[43,67,4,6,0,68,6,67,69,4,7,43,6,44,10,6,44,0,0,0,53,46,69,6,4,70,2,60,8,16,0,0,0,9,0,48,69,60,43,44,9,0,0,71,0,3,68,68,53,0,73,5,0,68,0,0,0,0,0,43,1,0,62,0,0,71,53,0,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.33E2,4.9E2,4.43E2,4.85E2,5E0,2.37E2,2.06E2,4.19E2,6.6E1,1.9E1,2.18E2,1.84E2,2.2E1,3.82E2,3.7E1,5.6E1,1E1,1.3E1,6E0,1.5E1,2.03E2,1.51E2,3.3E1,4E0,1.8E1,3.39E2,4.3E1,2E1,1.7E1,3.7E1,1.9E1,8E0,2E0,1.3E1,2E0,6.6E1,1.37E2,1.2E1,1.39E2,2.2E1,1.1E1,2E0,2E0,1.4E1,4E0,1.27E2,2.12E2,2.2E1,2.1E1,1.2E1,8E0,1.3E1,4E0,3.5E1,2E0,1.3E1,6E0,5E0,8E0,4E0,6.2E1,3E0,1.34E2,6E0,6E0,5.5E1,8.4E1,2E1,2E0,5E0,6E0,1E1,4E0,5.3E1,7.4E1,7.2E1,1.4E2,2E1,2E0,4E0,1.7E1,6E0,2E0,2E0,1.1E1,2.8E1,7E0,8E0,5E0,2E0,2E0,1.8E1,4.4E1,1.24E2,1E1,4.2E1,1.3E1,4.4E1,4E1,6E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-4.9171183E-4,-9.5586814E-2,2.9766519E-2,-7.540871E-2,-2.1877922E-1,-1.939886E-2,7.898701E-2,-5.7919532E-2,-1.6549957E-1,-1.326669E-1,-2.0917656E-2,-1.5201537E-2,-1.7757293E-2,1.37607E-2,6.8454996E-2,-1.1497455E-1,5.3761725E-4,-1.205278E-2,-1.1540514E-1,-1.5167065E-2,-8.950116E-2,-2.2344038E-2,1.7167287E-1,4.168014E-2,1.24078736E-1,-1.2549408E-1,1.9040975E-3,1.0638382E-2,-3.433489E-2,-7.352443E-2,-8.34838E-3,-7.355236E-3,-4.3879528E-2,-1.7848331E-2,-1.4798174E-2,9.536609E-3,2.1067658E-3,-5.082974E-2,6.8471335E-2,1.4943193E-1,-4.4763368E-2,-1.341693E-1,-1.8335516E-2,-6.6421986E-2,3.260688E-2,-5.3066695E-3,2.9584198E-4,1.686394E-3,-4.298732E-3,-3.3939246E-2,4.9265444E-2,-1.867559E-1,-2.1871438E-2,8.145401E-2,-1.3603412E-1,8.430715E-2,1.9731727E-1,1.9602766E-3,-6.981432E-3,-6.858547E-3,-2.5750494E-3,2.992206E-3,-4.4845017E-3,1.7376912E-3,-4.701544E-3,4.600478E-3,-2.5464986E-3,8.171224E-4,-2.3889015E-3,3.9031405E-3,-6.9646807E-3,-2.2021183E-3,-1.1560267E-2,-7.220223E-3,3.5150096E-4,4.331562E-3,-7.6198936E-3,-8.346442E-3,1.8417311E-3,-1.7474749E-3,6.246528E-3,1.2089948E-2,4.3898225E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,-1,23,25,27,-1,29,-1,31,33,35,37,39,41,-1,-1,43,45,-1,-1,47,49,-1,-1,-1,51,53,55,57,59,61,63,65,-1,-1,-1,-1,67,69,71,73,75,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.670204E0,5.4031897E-1,1.7051728E0,2.960837E-1,5.569531E-1,5.3117186E-1,7.83829E-1,5.470468E-1,1.1553568E-1,1.7122528E-1,0E0,4.7099864E-1,0E0,0E0,4.9672854E-1,1.4127159E-1,6.604238E-1,0E0,4.5862734E-2,0E0,5.8831006E-2,4.4387096E-1,4.5553118E-2,5.710456E-1,4.732839E-1,7.005048E-2,0E0,0E0,1.5681943E-1,4.935088E-2,0E0,0E0,5.7824288E-2,3.6236602E-1,0E0,0E0,0E0,2.0059727E-1,4.836753E-1,2.7962446E-1,1.4367217E-1,6.730759E-2,5.1395465E-2,1.8009137E-1,1.4445084E-1,0E0,0E0,0E0,0E0,2.4326149E-1,4.609304E-1,6.578261E-2,1.7388204E-1,4.967004E-1,9.101294E-2,2.610222E-1,3.236184E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,14,14,15,15,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,28,28,29,29,32,32,33,33,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,-1,24,26,28,-1,30,-1,32,34,36,38,40,42,-1,-1,44,46,-1,-1,48,50,-1,-1,-1,52,54,56,58,60,62,64,66,-1,-1,-1,-1,68,70,72,74,76,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7909248E5,4.7238097E0,3.8223077E2,4.15E2,1E0,4.093403E6,1.1057693E0,3.72E2,4.5683857E-2,1E0,-2.0917656E-2,7.4711205E6,-1.7757293E-2,1.37607E-2,5.97E3,7.0454545E0,3.6E1,-1.205278E-2,2.742234E6,-1.5167065E-2,1.317484E1,1E0,2.1525E4,1E0,6.747343E8,3.480315E0,1.9040975E-3,1.0638382E-2,1.1939979E8,2.5391157E0,-8.34838E-3,-7.355236E-3,7.67525E2,9.750085E6,-1.4798174E-2,9.536609E-3,2.1067658E-3,5.615327E5,4.556604E0,1.419902E3,9.942005E-3,5.66946E5,1.99E2,9.439024E0,7.9893243E2,-5.3066695E-3,2.9584198E-4,1.686394E-3,-4.298732E-3,4.8421054E0,2.5617392E1,3.3924444E5,1.4634147E0,1.766486E7,5.59E2,1.1251919E6,2.1298597E0,1.9602766E-3,-6.981432E-3,-6.858547E-3,-2.5750494E-3,2.992206E-3,-4.4845017E-3,1.7376912E-3,-4.701544E-3,4.600478E-3,-2.5464986E-3,8.171224E-4,-2.3889015E-3,3.9031405E-3,-6.9646807E-3,-2.2021183E-3,-1.1560267E-2,-7.220223E-3,3.5150096E-4,4.331562E-3,-7.6198936E-3,-8.346442E-3,1.8417311E-3,-1.7474749E-3,6.246528E-3,1.2089948E-2,4.3898225E-3],"split_indices":[43,68,67,0,6,66,69,2,53,13,0,43,0,0,2,69,10,0,9,0,69,113,9,29,47,68,0,0,7,68,0,0,48,9,0,0,0,43,68,67,72,9,2,73,4,0,0,0,0,73,73,43,68,66,0,43,53,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,2.23E2,7.03E2,1.93E2,3E1,3.52E2,3.51E2,1.63E2,3E1,2.3E1,7E0,3.49E2,3E0,1.5E1,3.36E2,8.2E1,8.1E1,9E0,2.1E1,3E0,2E1,3.37E2,1.2E1,2.28E2,1.08E2,7.7E1,5E0,1E1,7.1E1,1.4E1,7E0,7E0,1.3E1,3.33E2,4E0,9E0,3E0,5.1E1,1.77E2,9.4E1,1.4E1,7.1E1,6E0,4.8E1,2.3E1,9E0,5E0,5E0,8E0,2.69E2,6.4E1,8E0,4.3E1,1.67E2,1E1,4.1E1,5.3E1,8E0,6E0,6E1,1.1E1,3E0,3E0,1.2E1,3.6E1,1.3E1,1E1,6.8E1,2.01E2,5.5E1,9E0,3E0,5E0,7E0,3.6E1,1.6E2,7E0,8E0,2E0,1.2E1,2.9E1,3.2E1,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[-2.2689588E-3,-3.959761E-2,6.707921E-2,-6.342948E-2,2.8638165E-2,9.0953365E-2,-6.8277344E-2,-4.9587682E-2,-1.6939186E-1,1.8110054E-2,1.6577147E-1,3.2445827E-1,7.430979E-2,-1.4076981E-1,1.2648255E-3,-3.961547E-2,-1.6540284E-1,-1.3133188E-1,-3.8986427E-1,-9.5924206E-2,3.5925604E-2,2.8662023E-3,1.0067844E-2,1.8826677E-2,9.628902E-2,1.1936966E-1,1.4505225E-2,-1.6979238E-1,5.5337367E-3,-3.4509752E-2,4.0029692E-3,-6.819045E-2,-2.1238171E-5,3.942438E-2,-2.0630902E-1,1.9414368E-4,-1.6074337E-1,-5.234072E-3,-2.2243856E-2,4.54169E-3,-2.3311481E-1,2.7200881E-2,9.642914E-3,-1.1722643E-3,1.05193565E-2,8.929782E-2,2.1304817E-1,-3.470099E-4,1.28803905E-2,-2.812869E-1,-9.9009156E-2,3.6607143E-3,-8.056847E-2,7.983804E-2,-7.747135E-2,6.1268438E-2,-3.8730282E-2,-2.203082E-3,4.40261E-3,-2.716486E-1,-6.524241E-3,-1.9666642E-1,-6.9514826E-2,-3.7910774E-1,2.19036E-3,3.7016783E-2,-1.17302805E-1,2.4103361E-1,6.888334E-2,1.0033054E-3,2.3032483E-1,1.04293E-2,-1.557842E-1,-4.1729403E-3,-1.75928E-2,-4.407003E-4,-6.642593E-3,-1.1966654E-1,2.1901967E-3,2.3199049E-5,9.612232E-3,-6.2354123E-3,-2.7762114E-3,-8.641127E-3,3.605812E-3,-4.5600533E-3,1.179507E-4,-1.3924034E-2,-4.1214246E-3,-1.835205E-3,-1.0020331E-2,4.579828E-3,-5.2535078E-3,-8.094242E-3,-3.0828534E-2,1.9076385E-4,3.5725117E-3,-1.5616206E-2,5.3414464E-4,4.183004E-3,1.2899654E-2,5.04162E-3,-1.119459E-4,1.15028825E-2,2.298895E-3,-1.6810157E-3,1.9329644E-3,-1.4696233E-3,-1.1209382E-2,-7.0042675E-3,6.0633343E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,-1,43,45,47,49,-1,51,-1,53,55,57,59,-1,61,-1,-1,-1,63,65,-1,-1,-1,67,69,71,-1,73,75,-1,77,79,81,83,85,-1,-1,87,-1,89,91,93,-1,95,97,99,101,-1,103,105,107,-1,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.389413E0,9.775283E-1,1.049992E0,6.4428616E-1,2.2222362E-1,1.0446374E0,2.4720089E-1,4.4964623E-1,3.7674809E-1,3.000573E-1,4.474604E-2,2.910248E-1,6.935233E-1,2.0113346E-1,8.1852145E-2,4.1296118E-1,2.6787335E-1,1.8001604E-1,1.1781895E-1,5.5390936E-1,1.8429492E-1,0E0,0E0,0E0,1.0039206E-1,3.95087E-1,4.366642E-1,1.4577276E-1,0E0,1.0542403E-1,0E0,2.9743862E-1,3.6773252E-1,3.3251543E-2,7.405841E-2,0E0,1.0758531E-1,0E0,0E0,0E0,5.087618E-1,1.773911E-1,0E0,0E0,0E0,3.3342654E-1,1.108129E-1,1.8085393E-1,0E0,1.1469644E-1,5.749552E-2,0E0,7.551411E-2,1.3072398E-1,1.9324529E-1,2.5549436E-1,2.3279928E-1,0E0,0E0,3.4404695E-2,0E0,6.829727E-2,9.870438E-2,3.5015595E-1,0E0,1.5033278E-1,2.2895187E-1,4.641795E-2,2.848247E-1,0E0,8.623099E-2,1.4795226E-1,6.6480905E-2,0E0,0E0,0E0,0E0,4.7125474E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,36,36,40,40,41,41,45,45,46,46,47,47,49,49,50,50,52,52,53,53,54,54,55,55,56,56,59,59,61,61,62,62,63,63,65,65,66,66,67,67,68,68,70,70,71,71,72,72,77,77],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,-1,44,46,48,50,-1,52,-1,54,56,58,60,-1,62,-1,-1,-1,64,66,-1,-1,-1,68,70,72,-1,74,76,-1,78,80,82,84,86,-1,-1,88,-1,90,92,94,-1,96,98,100,102,-1,104,106,108,-1,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.4603988E6,1E0,1E0,7.82261E6,1.2083E4,1.3711089E6,2.0734E4,9.639872E0,1.6262903E2,5.51E2,1.562E5,8E0,5E1,7.12274E5,6.99E2,2E0,1.3789044E6,4.9E1,5.88E2,4.5E1,2.8662023E-3,1.0067844E-2,1.8826677E-2,7.555942E6,6.46E2,4.91027E0,5.1987316E7,5.5337367E-3,1.925508E6,4.0029692E-3,1.0952381E0,1.0252778E1,8.7961E4,2.6095572E0,1.9414368E-4,2.5757575E0,-5.234072E-3,-2.2243856E-2,4.54169E-3,8.1E2,1.388E1,9.642914E-3,-1.1722643E-3,1.05193565E-2,2.2940238E5,7.575958E7,2.5652106E7,1.28803905E-2,1.6135485E1,2.558E3,3.6607143E-3,6.1657715E8,1.2E1,1E0,3.8443E4,6.64E2,-2.203082E-3,4.40261E-3,4.2492856E2,-6.524241E-3,1.4473684E-1,1.4858E4,7.271717E6,2.19036E-3,5.4007E7,9.61E2,8.771579E2,5.0741018E8,1.0033054E-3,4.7E1,1.14E2,8.25E2,-4.1729403E-3,-1.75928E-2,-4.407003E-4,-6.642593E-3,2.3601625E1,2.1901967E-3,2.3199049E-5,9.612232E-3,-6.2354123E-3,-2.7762114E-3,-8.641127E-3,3.605812E-3,-4.5600533E-3,1.179507E-4,-1.3924034E-2,-4.1214246E-3,-1.835205E-3,-1.0020331E-2,4.579828E-3,-5.2535078E-3,-8.094242E-3,-3.0828534E-2,1.9076385E-4,3.5725117E-3,-1.5616206E-2,5.3414464E-4,4.183004E-3,1.2899654E-2,5.04162E-3,-1.119459E-4,1.15028825E-2,2.298895E-3,-1.6810157E-3,1.9329644E-3,-1.4696233E-3,-1.1209382E-2,-7.0042675E-3,6.0633343E-4],"split_indices":[2,43,6,6,43,9,43,44,69,4,2,10,32,3,9,2,32,60,0,2,8,0,0,0,1,0,50,60,0,43,0,68,73,1,68,0,68,0,0,0,2,71,0,0,0,62,7,58,0,71,10,0,7,3,13,9,10,0,0,4,0,71,9,60,0,7,2,48,7,0,8,8,8,0,0,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.21E2,5.99E2,3.22E2,4.44E2,1.55E2,2.74E2,4.8E1,3.94E2,5E1,1.45E2,1E1,1.7E1,2.57E2,2.3E1,2.5E1,3.64E2,3E1,4.4E1,6E0,1.9E1,1.26E2,4E0,6E0,1.2E1,5E0,1.46E2,1.11E2,2.1E1,2E0,1.8E1,7E0,2.11E2,1.53E2,5E0,2.5E1,8E0,3.6E1,2E0,4E0,8E0,1.1E1,1.21E2,5E0,3E0,2E0,1.12E2,3.4E1,1.06E2,5E0,7E0,1.4E1,5E0,1.3E1,1.2E1,1.99E2,5.9E1,9.4E1,2E0,3E0,1.1E1,1.4E1,2.5E1,1.1E1,7E0,4E0,1.14E2,7E0,1.2E1,1E2,3E0,3.1E1,1E2,6E0,3E0,4E0,5E0,9E0,1E1,3E0,8E0,4E0,4.6E1,1.53E2,3E0,5.6E1,3.8E1,5.6E1,9E0,2E0,3E0,2.2E1,2E0,9E0,5E0,2E0,6.3E1,5.1E1,2E0,5E0,3E0,9E0,6.4E1,3.6E1,2.8E1,3E0,4E1,6E1,3E0,3E0,8E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-6.895316E-3,-6.71838E-2,3.312592E-2,-1.37605E-1,-5.0188333E-2,4.6439033E-2,-1.571363E-1,3.624711E-3,-1.6279134E-1,-3.7029132E-2,-1.8440512E-1,2.5232486E-2,1.2939133E-1,-1.5840836E-2,-1.09088466E-1,-1.1391808E-1,-3.257107E-1,-1.7666481E-2,-1.05647616E-1,-3.7167233E-1,-8.052679E-2,8.337551E-2,8.095155E-3,2.0772322E-1,8.1143826E-2,-1.2692572E-1,4.360338E-3,-1.4328553E-1,5.8171716E-2,-2.7809013E-2,-1.0720432E-2,-4.291568E-2,5.472971E-2,-6.974225E-2,-3.258266E-1,-7.7807554E-3,-2.0741198E-2,-4.9891863E-3,-1.0732574E-2,3.3010682E-3,1.2736145E-1,1.3641219E-1,-1.4692703E-3,1.2735209E-1,2.8562164E-1,-4.6988167E-2,1.1048775E-1,-1.4181153E-1,2.0646194E-3,-3.2226247E-1,-1.1125534E-1,6.863607E-3,-3.1686581E-3,-3.5983257E-2,-9.830977E-3,1.3777082E-1,2.5746252E-2,3.3791102E-2,-1.126285E-1,-3.0458784E-2,1.2158011E-4,3.344393E-2,-4.642604E-3,-2.360137E-2,9.811689E-3,1.1023901E-1,1.3657438E-2,-4.2175115E-3,2.3409563E-1,-5.63139E-2,1.7733028E-2,3.2974184E-3,1.7222606E-1,1.7169893E-2,8.068891E-3,2.2844134E-3,-6.342838E-3,8.694355E-2,1.5060649E-2,-2.907196E-3,-1.6247644E-1,-5.175036E-3,-1.8600192E-2,-7.568082E-3,-3.089219E-3,-2.334402E-3,1.092012E-3,4.033449E-3,9.873784E-3,-2.9361926E-3,2.1080046E-3,5.990987E-3,-9.819318E-4,-9.391084E-3,-2.597238E-3,2.9914333E-3,-2.7925018E-3,2.9430839E-3,-5.069766E-3,5.8908276E-3,-6.276182E-3,2.6975782E-3,-7.847362E-3,1.2617377E-2,3.8400232E-3,9.077245E-4,-4.1379654E-3,3.1773092E-3,1.4971102E-4,9.291414E-3,1.4950813E-3,5.7083457E-3,-6.4514996E-4,-8.634231E-3,-2.595999E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,-1,-1,53,55,57,59,-1,-1,61,-1,63,65,67,69,71,73,75,77,79,-1,81,83,-1,-1,85,-1,87,89,91,93,-1,-1,95,-1,97,-1,99,-1,101,103,105,107,-1,109,-1,-1,-1,-1,111,-1,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1954784E0,4.2754698E-1,1.3928388E0,3.9269948E-1,5.124302E-1,8.974103E-1,2.801981E-1,0E0,4.6452677E-1,3.5553667E-1,4.758485E-1,4.0685937E-1,3.7619364E-1,0E0,1.2044537E-1,2.6102716E-1,2.2500789E-1,3.880453E-1,4.4370693E-1,6.2055588E-2,2.031315E-1,3.2713938E-1,3.8924775E-1,2.0779622E-1,2.533447E-1,7.770163E-2,0E0,2.106511E-1,1.0039622E-1,0E0,0E0,1.7575347E-1,1.2882794E-1,2.34694E-1,8.6641616E-1,0E0,0E0,5.0721236E-2,0E0,1.9493012E-1,1.3847387E-1,3.0320552E-1,3.127289E-1,4.098621E-2,1.1396921E-1,1.1838837E-1,2.4308819E-1,3.2451153E-2,0E0,5.383116E-2,7.687917E-2,0E0,0E0,1.29622E-1,0E0,3.4850836E-2,7.595211E-2,8.619773E-2,1.7633474E-1,0E0,0E0,3.2728847E-2,0E0,2.3693472E-1,0E0,2.426154E-1,0E0,1.1214937E-1,5.0881743E-2,1.9244029E-1,1.613018E-1,0E0,3.6038935E-2,0E0,0E0,0E0,0E0,1.8214506E-1,0E0,0E0,3.907448E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,31,31,32,32,33,33,34,34,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,53,53,55,55,56,56,57,57,58,58,61,61,63,63,65,65,67,67,68,68,69,69,70,70,72,72,77,77,80,80],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,-1,-1,54,56,58,60,-1,-1,62,-1,64,66,68,70,72,74,76,78,80,-1,82,84,-1,-1,86,-1,88,90,92,94,-1,-1,96,-1,98,-1,100,-1,102,104,106,108,-1,110,-1,-1,-1,-1,112,-1,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.57E2,2.2953334E2,1E0,7E0,2E0,6.355E3,1.0479E4,3.624711E-3,1.1320151E7,3.3512E4,3.9595376E7,9.25E0,1.3092E4,-1.5840836E-2,1E0,2.49E2,3.0536E4,1.0885427E9,1.2600376E7,4.612245E0,6.72E2,3.7788504E7,1.16E0,1.4039243E3,3.115515E10,3.2E1,4.360338E-3,1.1649485E0,5.539245E6,-2.7809013E-2,-1.0720432E-2,2.4E1,1.1937528E3,2E0,1.187E3,-7.7807554E-3,-2.0741198E-2,1.0315458E8,-1.0732574E-2,2.5345264E7,9.6203804E-1,7.9016544E5,3.2525E2,4.433E3,1.137863E6,4.88E2,5.428706E3,9.29023E5,2.0646194E-3,5.04E2,1.0891155E4,6.863607E-3,-3.1686581E-3,3.422351E6,-9.830977E-3,2.8636363E0,1.2463E4,5.8703518E1,6.23E2,-3.0458784E-2,1.2158011E-4,8.596237E4,-4.642604E-3,2E0,9.811689E-3,2.057307E6,1.3657438E-2,2.4595E5,1.1716E4,2.0131578E0,3.9914E4,3.2974184E-3,1.41565E3,1.7169893E-2,8.068891E-3,2.2844134E-3,-6.342838E-3,1.8258025E2,1.5060649E-2,-2.907196E-3,2.4076016E5,-5.175036E-3,-1.8600192E-2,-7.568082E-3,-3.089219E-3,-2.334402E-3,1.092012E-3,4.033449E-3,9.873784E-3,-2.9361926E-3,2.1080046E-3,5.990987E-3,-9.819318E-4,-9.391084E-3,-2.597238E-3,2.9914333E-3,-2.7925018E-3,2.9430839E-3,-5.069766E-3,5.8908276E-3,-6.276182E-3,2.6975782E-3,-7.847362E-3,1.2617377E-2,3.8400232E-3,9.077245E-4,-4.1379654E-3,3.1773092E-3,1.4971102E-4,9.291414E-3,1.4950813E-3,5.7083457E-3,-6.4514996E-4,-8.634231E-3,-2.595999E-3],"split_indices":[2,48,83,3,6,2,9,0,5,1,7,73,10,0,16,2,1,46,60,69,44,7,69,67,46,8,0,68,5,0,0,8,48,32,10,0,0,7,0,60,53,43,67,10,9,0,67,62,0,7,43,0,0,9,0,69,1,62,2,0,0,48,0,8,0,12,0,9,44,68,9,0,4,0,0,0,0,73,0,0,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.08E2,3.62E2,5.46E2,6.9E1,2.93E2,5.11E2,3.5E1,7E0,6.2E1,2.68E2,2.5E1,4.08E2,1.03E2,6E0,2.9E1,4.9E1,1.3E1,2.1E2,5.8E1,8E0,1.7E1,9.2E1,3.16E2,3.8E1,6.5E1,2.7E1,2E0,4.2E1,7E0,2E0,1.1E1,1.56E2,5.4E1,5.1E1,7E0,3E0,5E0,1.2E1,5E0,3.3E1,5.9E1,2.1E1,2.95E2,2E1,1.8E1,1.2E1,5.3E1,2.5E1,2E0,5E0,3.7E1,4E0,3E0,1.51E2,5E0,1.3E1,4.1E1,1.5E1,3.6E1,3E0,4E0,9E0,3E0,3E1,3E0,5.5E1,4E0,9E0,1.2E1,7.6E1,2.19E2,1E1,1E1,9E0,9E0,6E0,6E0,4.9E1,4E0,6E0,1.9E1,2E0,3E0,1.6E1,2.1E1,1.22E2,2.9E1,9E0,4E0,7E0,3.4E1,5E0,1E1,1.3E1,2.3E1,7E0,2E0,1.5E1,1.5E1,5.2E1,3E0,7E0,2E0,9E0,3E0,2.3E1,5.3E1,4.8E1,1.71E2,8E0,2E0,3.6E1,1.3E1,1.5E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[3.8341692E-4,-7.717314E-2,2.8646912E-2,-5.6673933E-2,-2.67846E-1,-6.020736E-4,8.626115E-2,-4.108622E-2,-1.4394872E-1,-1.7276108E-1,-2.2048604E-2,3.5326995E-3,-2.3259096E-1,1.4308332E-1,5.9102286E-2,-9.370647E-2,1.1416344E-2,-2.3010258E-1,-5.4584026E-2,-3.4391245E-1,-9.929553E-2,-5.1716566E-3,9.266403E-2,-4.952185E-3,-1.5444253E-2,1.1057179E-1,3.0184525E-1,3.9504517E-2,1.724151E-1,-2.3398355E-2,-1.10532835E-1,1.1109085E-2,-2.0087263E-2,-1.2273006E-2,-4.4753295E-3,-2.6631001E-2,-8.625859E-3,-2.336219E-2,-3.2953676E-3,-8.250568E-3,-1.6831679E-3,-1.1573205E-1,2.6155894E-3,1.2738779E-1,-8.9231946E-2,1.3090637E-1,-1.5149073E-1,1.585084E-2,3.3162315E-3,4.542383E-2,-1.10715795E-2,1.0956986E-2,2.2875577E-1,-6.388313E-2,1.0917529E-3,-9.0898145E-3,-9.213237E-2,-7.349214E-2,2.9110389E-2,3.1379797E-3,-2.9588826E-3,-2.5755632E-1,-3.051123E-2,9.688453E-3,-1.3376777E-1,1.7214836E-1,5.9214514E-2,-8.740777E-3,-5.6547957E-4,-9.104565E-2,1.6253541E-1,-1.3044273E-2,1.3018284E-3,9.5579475E-2,2.2036422E-3,4.0392107E-3,-5.542991E-3,1.18160015E-2,1.2069619E-3,-4.8482623E-3,2.512588E-4,2.5665737E-4,-4.9156933E-3,-4.2663445E-3,5.7084775E-3,-1.3403753E-3,3.670621E-3,-1.4590863E-2,-1.0187671E-3,-5.2225776E-3,9.1799867E-4,4.0652254E-3,-1.609627E-4,-2.6416725E-2,-2.2913474E-3,2.4765958E-3,1.0199134E-2,-2.3554773E-4,4.773076E-3,1.3902362E-3,-6.340868E-3,9.250242E-3,2.7710302E-3,1.1439553E-3,5.666874E-3,-1.5788598E-3,3.2025676E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,53,55,-1,57,-1,-1,59,-1,-1,-1,-1,-1,61,63,65,67,69,71,-1,-1,73,-1,75,77,79,-1,-1,81,83,85,-1,-1,87,89,91,93,95,97,-1,-1,99,101,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0517488E0,9.6198106E-1,1.1560705E0,3.0289972E-1,3.8874447E-1,4.3822664E-1,3.4802842E-1,5.3703326E-1,2.4920994E-1,1.9217819E-1,0E0,3.488273E-1,6.5012306E-2,3.485825E-1,3.44056E-1,1.121676E-1,7.0701605E-1,5.150783E-2,6.091763E-2,1.8069196E-1,6.258409E-2,3.5329673E-1,2.6401156E-1,0E0,0E0,3.5555393E-1,8.352816E-2,2.3426023E-1,2.0671666E-1,3.9452568E-2,1.0386628E-1,0E0,2.307227E-1,0E0,0E0,5.9987668E-2,0E0,0E0,0E0,0E0,0E0,3.184739E-1,3.7171045E-1,9.3556404E-2,5.097062E-2,4.289856E-1,1.2360428E-1,0E0,0E0,2.90266E-1,0E0,7.9489715E-2,8.066827E-2,3.1326003E-2,0E0,0E0,8.800447E-2,1.6161019E-1,1.3502248E-1,0E0,0E0,1.2550342E-1,7.619005E-2,3.743528E-1,6.691855E-1,1.0044682E-1,4.238058E-2,0E0,0E0,4.7898695E-2,1.8281603E-1,0E0,0E0,1.1260986E-1,1.7825384E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,32,32,35,35,41,41,42,42,43,43,44,44,45,45,46,46,49,49,51,51,52,52,53,53,56,56,57,57,58,58,61,61,62,62,63,63,64,64,65,65,66,66,69,69,70,70,73,73,74,74],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,54,56,-1,58,-1,-1,60,-1,-1,-1,-1,-1,62,64,66,68,70,72,-1,-1,74,-1,76,78,80,-1,-1,82,84,86,-1,-1,88,90,92,94,96,98,-1,-1,100,102,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,5.869281E0,5.867288E2,3.94E2,3.9401392E6,1E0,1.0717949E1,3.95E2,2.6363637E0,3.964269E3,-2.2048604E-2,4.5866325E6,1.892914E-5,1.3800834E3,3.6966505E0,1E0,2.04115E5,1.9001543E1,1.2820834E7,6.34E2,1.39848995E1,3.010204E-1,1.9E1,-4.952185E-3,-1.5444253E-2,1.853776E7,1E0,2.3040113E10,5.68E2,1.0891155E4,8.071428E0,1.1109085E-2,1.1805571E-1,-1.2273006E-2,-4.4753295E-3,5.49E2,-8.625859E-3,-2.336219E-2,-3.2953676E-3,-8.250568E-3,-1.6831679E-3,4.066655E2,4.5158855E6,1.162E3,1.45064E5,5.1604336E7,1.1E1,1.585084E-2,3.3162315E-3,5.6516E4,-1.10715795E-2,9.099893E8,4.262058E0,6.44E2,1.0917529E-3,-9.0898145E-3,1E1,1E0,2.836842E2,3.1379797E-3,-2.9588826E-3,8.1E2,3.56E2,1.6368421E0,3.1157124E-1,8.233192E9,5.11E2,-8.740777E-3,-5.6547957E-4,4.2E1,4.5478998E8,-1.3044273E-2,1.3018284E-3,4.25681E-1,7.958E3,4.0392107E-3,-5.542991E-3,1.18160015E-2,1.2069619E-3,-4.8482623E-3,2.512588E-4,2.5665737E-4,-4.9156933E-3,-4.2663445E-3,5.7084775E-3,-1.3403753E-3,3.670621E-3,-1.4590863E-2,-1.0187671E-3,-5.2225776E-3,9.1799867E-4,4.0652254E-3,-1.609627E-4,-2.6416725E-2,-2.2913474E-3,2.4765958E-3,1.0199134E-2,-2.3554773E-4,4.773076E-3,1.3902362E-3,-6.340868E-3,9.250242E-3,2.7710302E-3,1.1439553E-3,5.666874E-3,-1.5788598E-3,3.2025676E-3],"split_indices":[43,68,67,0,60,113,71,2,68,43,0,43,52,67,50,24,5,71,9,2,69,73,8,0,0,62,19,5,0,43,67,0,57,0,0,0,0,0,0,0,0,48,62,10,7,7,8,0,0,9,0,7,53,1,0,0,3,16,4,0,0,2,2,68,53,46,0,0,0,0,7,0,0,53,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.34E2,2.49E2,6.85E2,2.26E2,2.3E1,4.55E2,2.3E2,1.93E2,3.3E1,1.7E1,6E0,4.48E2,7E0,7.3E1,1.57E2,9.6E1,9.7E1,1.6E1,1.7E1,4E0,1.3E1,4.09E2,3.9E1,4E0,3E0,6.2E1,1.1E1,1.35E2,2.2E1,1.9E1,7.7E1,1.1E1,8.6E1,1.2E1,4E0,1.5E1,2E0,2E0,2E0,5E0,8E0,2.6E1,3.83E2,3.3E1,6E0,5.8E1,4E0,9E0,2E0,1.33E2,2E0,6E0,1.6E1,1E1,9E0,1.2E1,6.5E1,4.1E1,4.5E1,4E0,1.1E1,9E0,1.7E1,3.65E2,1.8E1,1.9E1,1.4E1,2E0,4E0,7E0,5.1E1,2E0,2E0,6.1E1,7.2E1,4E0,2E0,1.4E1,2E0,6E0,4E0,8E0,5.7E1,3.8E1,3E0,2.1E1,2.4E1,7E0,2E0,6E0,1.1E1,5.2E1,3.13E2,2E0,1.6E1,6E0,1.3E1,6E0,8E0,2E0,5E0,3.7E1,1.4E1,1.7E1,4.4E1,4.7E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[-5.1760958E-3,-5.923329E-2,3.5193834E-2,-5.16318E-2,-2.5752774E-1,4.345023E-2,-1.5255007E-1,-9.325459E-2,-2.4752155E-2,-3.0627474E-4,-4.5040554E-1,3.350956E-2,1.914793E-1,-1.2595688E-2,-7.9296954E-2,-7.937999E-2,-1.7669559E-1,9.681324E-3,-3.416814E-2,-4.0906286E-3,2.4344353E-3,-2.5836617E-2,-4.1861515E-3,1.7753823E-2,1.2238839E-1,8.70357E-2,2.3025005E-1,1.9850757E-3,-4.713374E-3,-5.1596943E-2,-1.12594314E-1,-1.806064E-2,-1.3160549E-1,-2.2894673E-2,-1.4575806E-1,2.351638E-2,-1.0488649E-1,1.8175219E-1,1.034754E-3,5.4022735E-3,-9.133315E-4,1.1351249E-2,2.8530618E-3,-4.4640735E-2,-9.040644E-3,-1.2934111E-1,-8.906775E-3,-7.549922E-3,-2.0037927E-3,-3.4350753E-2,5.913015E-2,2.0493811E-3,-1.9596805E-1,8.437055E-2,9.260661E-3,2.7245935E-4,-1.4793131E-1,1.1145995E-1,3.2427832E-1,-6.369111E-2,5.3943195E-3,-3.0081845E-3,5.349629E-4,-6.377217E-3,1.6591516E-3,-4.6327687E-3,1.4528875E-3,-1.1976602E-3,-1.1138575E-2,5.758816E-3,-4.363586E-3,-1.0233843E-2,-8.500621E-4,2.0051084E-3,8.812394E-3,-5.226547E-3,9.4709056E-4,8.400766E-4,-9.279936E-3,8.965626E-3,1.5340628E-3,1.7634599E-3,1.699466E-2,-8.669494E-3,1.3292558E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,33,-1,-1,-1,-1,35,37,39,41,-1,-1,43,45,-1,47,49,51,53,55,57,59,-1,-1,-1,-1,61,-1,63,65,-1,-1,67,69,-1,71,73,75,-1,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9705058E0,5.685339E-1,8.080237E-1,4.1500628E-1,6.944345E-1,7.234515E-1,1.6914165E-1,1.549995E-1,5.07153E-1,3.6908165E-2,2.531036E-1,6.5039086E-1,1.0549009E-1,0E0,4.4669516E-2,1.11385524E-1,1.3975179E-1,0E0,2.7342406E-1,0E0,0E0,0E0,0E0,2.830351E-1,5.0409305E-1,3.3348173E-2,3.6391973E-2,0E0,0E0,6.1846867E-2,9.782481E-2,0E0,4.1972876E-2,1.9090742E-1,1.992563E-1,3.2887626E-1,8.6672544E-2,4.3472457E-1,1.859581E-1,0E0,0E0,0E0,0E0,7.83803E-2,0E0,6.780726E-2,3.5885878E-2,0E0,0E0,3.0606788E-1,2.5900492E-1,0E0,6.4434886E-2,3.0406743E-1,4.196395E-1,0E0,1.149635E-1,2.0422721E-1,1.6285181E-1,1.8650587E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,18,18,23,23,24,24,25,25,26,26,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,43,43,45,45,46,46,49,49,50,50,52,52,53,53,54,54,56,56,57,57,58,58,59,59],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,34,-1,-1,-1,-1,36,38,40,42,-1,-1,44,46,-1,48,50,52,54,56,58,60,-1,-1,-1,-1,62,-1,64,66,-1,-1,68,70,-1,72,74,76,-1,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.31E2,6.550562E0,7.042942E7,3.95E2,1.3E1,6.663214E6,1.4464286E1,7.122041E7,1.2E1,9.374418E0,9E1,1.5146261E3,1.599E3,-1.2595688E-2,3.9961785E-2,4.5795207E0,1.3E1,9.681324E-3,1.9734E4,-4.0906286E-3,2.4344353E-3,-2.5836617E-2,-4.1861515E-3,2.1894855E1,2.2040408E7,2.041E3,3.7E1,1.9850757E-3,-4.713374E-3,1.998171E-6,4.3398426E4,-1.806064E-2,1.852364E6,6.787E4,7.908163E-1,1.0181818E1,8.387763E2,2.59672E3,2.6108465E0,5.4022735E-3,-9.133315E-4,1.1351249E-2,2.8530618E-3,4.9E2,-9.040644E-3,1.2585366E1,4.582E3,-7.549922E-3,-2.0037927E-3,4.5865917E8,8.7E1,2.0493811E-3,1.71E4,5.0857143E0,1.68E2,2.7245935E-4,1.0183612E2,2.392405E0,1E1,1.3655363E2,5.3943195E-3,-3.0081845E-3,5.349629E-4,-6.377217E-3,1.6591516E-3,-4.6327687E-3,1.4528875E-3,-1.1976602E-3,-1.1138575E-2,5.758816E-3,-4.363586E-3,-1.0233843E-2,-8.500621E-4,2.0051084E-3,8.812394E-3,-5.226547E-3,9.4709056E-4,8.400766E-4,-9.279936E-3,8.965626E-3,1.5340628E-3,1.7634599E-3,1.699466E-2,-8.669494E-3,1.3292558E-3],"split_indices":[2,68,60,2,0,43,73,7,10,68,0,67,10,0,53,73,0,0,44,0,0,0,0,69,66,2,8,0,0,52,48,0,9,1,68,73,4,67,50,0,0,0,0,1,0,71,1,0,0,5,8,0,10,69,10,0,48,68,3,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.01E2,3.85E2,5.16E2,3.72E2,1.3E1,4.95E2,2.1E1,1.45E2,2.27E2,6E0,7E0,4.65E2,3E1,7E0,1.4E1,1.26E2,1.9E1,8E0,2.19E2,2E0,4E0,5E0,2E0,3.96E2,6.9E1,9E0,2.1E1,2E0,1.2E1,7E1,5.6E1,2E0,1.7E1,2E2,1.9E1,3.79E2,1.7E1,4.6E1,2.3E1,7E0,2E0,1.9E1,2E0,6.8E1,2E0,4.8E1,8E0,1.2E1,5E0,1.76E2,2.4E1,4E0,1.5E1,7.1E1,3.08E2,5E0,1.2E1,3.2E1,1.4E1,1.5E1,8E0,5E1,1.8E1,4.6E1,2E0,2E0,6E0,1.7E2,6E0,1.7E1,7E0,1.3E1,2E0,5.2E1,1.9E1,2.5E1,2.83E2,3E0,9E0,1.5E1,1.7E1,2E0,1.2E1,6E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[5.8335178E-3,-1.0751224E-2,1.1508187E-1,-2.5235737E-2,7.042643E-2,1.7649395E-2,9.813382E-2,-4.232527E-2,2.2755155E-2,5.3087704E-2,1.7439574E-1,8.255364E-2,1.2752013E-2,-7.743605E-2,-1.1117881E-2,3.2213025E-2,-2.1559119E-1,2.2467297E-2,1.5139914E-1,2.1669663E-1,1.3735673E-3,-7.919617E-2,9.628559E-2,-6.0691305E-2,-1.9544351E-1,-1.09520026E-1,8.174618E-3,6.382777E-2,-2.4502276E-2,-1.2853813E-2,-1.9724313E-3,-1.9439494E-2,6.917067E-2,2.5072685E-3,1.7307562E-1,1.5585858E-4,1.1867898E-2,-3.700255E-4,-1.0186273E-2,1.2406065E-1,-2.0944425E-3,-5.4565724E-2,-2.588458E-1,-2.2152048E-1,-2.7235632E-3,5.3842533E-2,-2.04793E-1,9.620696E-2,-1.196415E-2,8.341496E-2,-1.0635005E-1,1.5720949E-2,-1.1176825E-1,-3.376952E-2,3.578553E-3,9.479856E-2,-2.1774943E-4,8.7153595E-3,1.3700278E-3,-3.649641E-3,1.3042307E-1,2.570127E-2,-6.5366114E-3,-2.139127E-3,-7.794394E-3,-2.1247052E-2,-2.122148E-3,-7.7658086E-3,-1.4404799E-2,-6.505025E-3,5.7765064E-3,-5.596851E-3,-2.64719E-2,-1.0370764E-2,5.475878E-3,1.7612379E-3,-2.0282397E-3,1.1989024E-3,6.099522E-3,1.4434997E-3,-9.716284E-3,-3.2956444E-3,2.9276074E-3,-6.882426E-3,1.2314821E-3,6.5910537E-4,-3.3234044E-3,6.3761296E-3,1.845615E-3,5.093209E-3,1.0386525E-2,-5.533009E-3,2.2502833E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,-1,-1,53,55,-1,57,-1,-1,-1,-1,59,61,63,65,67,-1,69,71,73,75,77,79,81,83,85,-1,87,-1,-1,-1,-1,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6772039E0,9.4713545E-1,5.0947595E-1,5.6112015E-1,2.1067059E-1,0E0,2.9555488E-1,5.5137986E-1,4.127096E-1,3.1569406E-1,9.777212E-2,2.4750847E-1,0E0,4.5645154E-1,5.096727E-1,3.1355283E-1,6.1971664E-2,1.6183409E-1,4.349935E-2,1.0588372E-1,0E0,8.7909296E-2,2.73511E-1,2.3956245E-1,9.054184E-2,6.958442E-1,4.0004522E-1,3.8220137E-1,2.2289458E-1,0E0,0E0,6.337125E-2,7.42587E-2,0E0,3.561437E-2,0E0,0E0,0E0,0E0,1.107465E-1,9.212674E-2,1.8990785E-1,2.3246118E-1,6.374657E-2,0E0,2.4908684E-1,7.9316986E-1,3.1495556E-1,2.918359E-1,2.7450424E-1,1.7558421E-1,1.8438435E-1,1.0405612E-1,7.148226E-2,0E0,6.216842E-2,0E0,0E0,0E0,0E0,1.23182654E-1,7.1391E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,34,34,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,60,60,61,61],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,-1,-1,54,56,-1,58,-1,-1,-1,-1,60,62,64,66,68,-1,70,72,74,76,78,80,82,84,86,-1,88,-1,-1,-1,-1,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,2.9652428E6,1.273782E4,3.422351E6,1.4433751E0,1.7649395E-2,9.059046E6,2.540633E2,1E0,2.4210527E0,1.4320667E3,1.1160929E0,1.2752013E-2,2.4E1,1.6129032E0,1.079E3,1.1E1,4.5866325E6,5.2692295E6,8.3E1,1.3735673E-3,4.88451E5,1.0907809E9,2.2681375E6,5.3871745E-1,6E0,8.138889E0,1E0,4.7420635E1,-1.2853813E-2,-1.9724313E-3,2.4702472E5,2.0588236E0,2.5072685E-3,1.584E3,1.5585858E-4,1.1867898E-2,-3.700255E-4,-1.0186273E-2,1.0201538E-1,3.0070068E7,1.4979966E-5,1.4495105E2,7.143135E3,-2.7235632E-3,8.69E2,1E0,1E0,1.4E1,1.264E3,1.6966831E-3,5.4814816E0,2.8E1,3.588785E0,3.578553E-3,1.103E3,-2.1774943E-4,8.7153595E-3,1.3700278E-3,-3.649641E-3,1.3439851E7,7.383931E4,-6.5366114E-3,-2.139127E-3,-7.794394E-3,-2.1247052E-2,-2.122148E-3,-7.7658086E-3,-1.4404799E-2,-6.505025E-3,5.7765064E-3,-5.596851E-3,-2.64719E-2,-1.0370764E-2,5.475878E-3,1.7612379E-3,-2.0282397E-3,1.1989024E-3,6.099522E-3,1.4434997E-3,-9.716284E-3,-3.2956444E-3,2.9276074E-3,-6.882426E-3,1.2314821E-3,6.5910537E-4,-3.3234044E-3,6.3761296E-3,1.845615E-3,5.093209E-3,1.0386525E-2,-5.533009E-3,2.2502833E-3],"split_indices":[2,43,48,9,57,0,43,67,79,68,4,69,0,8,68,10,3,43,60,0,0,9,7,43,57,71,73,19,73,0,0,48,69,0,0,0,0,0,0,53,58,52,4,48,0,2,29,17,3,44,53,71,71,71,0,10,0,0,0,0,60,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.24E2,8.03E2,1.21E2,6.82E2,1.21E2,6E0,1.15E2,5.03E2,1.79E2,1.05E2,1.6E1,1.07E2,8E0,2.36E2,2.67E2,1.73E2,6E0,8.1E1,2.4E1,1.2E1,4E0,8E0,9.9E1,2.08E2,2.8E1,4.3E1,2.24E2,1.11E2,6.2E1,4E0,2E0,4.3E1,3.8E1,5E0,1.9E1,2E0,1E1,6E0,2E0,7.7E1,2.2E1,2.03E2,5E0,2.3E1,5E0,1.6E1,2.7E1,4.1E1,1.83E2,1E2,1.1E1,4.3E1,1.9E1,3.8E1,5E0,2.8E1,1E1,1.7E1,2E0,2E0,7.5E1,1.9E1,3E0,1.9E2,1.3E1,2E0,3E0,1.6E1,7E0,4E0,1.2E1,2.3E1,4E0,2E0,3.9E1,7.1E1,1.12E2,4.6E1,5.4E1,5E0,6E0,1.5E1,2.8E1,1.5E1,4E0,1.7E1,2.1E1,1.5E1,1.3E1,6.3E1,1.2E1,2E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-5.8880793E-3,-1.8897459E-2,1.2473415E-1,-4.925233E-2,2.124212E-2,1.5396106E-2,9.594672E-2,-3.792944E-2,-1.6804835E-1,7.0232097E-3,9.4115466E-2,1.15304016E-1,-1.1133283E-1,7.8866266E-2,-4.3611277E-2,-1.970364E-1,3.871936E-2,1.3377129E-2,-1.6632636E-1,-5.7005123E-3,1.07670434E-1,9.096479E-2,2.7016267E-1,-8.02829E-3,1.2989049E-3,1.6819939E-1,-1.763651E-2,-3.32346E-2,-1.0057081E-1,-6.666621E-2,-2.4285255E-1,-1.3745549E-3,5.434227E-3,-1.639963E-2,5.919063E-2,-2.2082983E-2,-4.50126E-2,1.722986E-3,1.3201086E-1,4.3434236E-2,1.6441174E-1,2.1037145E-3,1.5252908E-2,1.2879209E-2,5.2678194E-3,-8.2748495E-3,3.7752524E-2,-6.515484E-2,3.1612927E-4,-1.5292856E-1,-4.8334967E-2,-4.9491283E-3,3.4789457E-3,-3.9112088E-1,-1.3262877E-1,-4.561555E-3,-2.1955836E-1,1.9820341E-1,2.3448551E-2,3.5963657E-3,-4.232848E-3,1.1151846E-1,1.0993406E-2,1.1742452E-1,-7.70238E-3,3.2902084E-4,1.8655159E-1,4.9831397E-3,-1.8227387E-3,3.343813E-3,-3.3751358E-3,5.092497E-4,-5.360358E-3,-8.611714E-3,-5.271773E-4,4.8427755E-4,-4.430446E-3,-4.348499E-3,-2.2837115E-2,8.175532E-4,-7.7235955E-3,1.2580412E-3,-2.1499665E-3,-3.0420015E-2,-1.3564031E-3,1.0418927E-2,6.083926E-4,-1.3026803E-3,3.778612E-3,8.179015E-3,2.5169298E-3,2.681363E-3,9.64718E-3,-3.8855888E-3,4.1637262E-3,9.81075E-3,1.4009502E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,-1,-1,43,45,47,49,51,53,-1,-1,55,57,-1,59,-1,61,63,65,-1,-1,-1,-1,-1,67,69,71,73,75,-1,-1,77,79,81,83,85,87,-1,-1,89,-1,91,93,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5946146E0,1.0403684E0,4.7306597E-1,6.473515E-1,3.8065308E-1,0E0,3.1777716E-1,2.9874778E-1,2.5936854E-1,3.4257278E-1,1.8942863E-1,2.3926449E-1,6.456907E-2,1.8400523E-1,2.4785417E-1,2.0481312E-1,3.5457987E-2,4.0885982E-1,3.9030397E-1,0E0,9.3091965E-2,2.0929927E-1,1.02179825E-1,0E0,0E0,3.4331262E-2,1.0437745E-1,3.8663256E-1,1.7038006E-1,6.981704E-2,3.893789E-1,0E0,0E0,4.3671E-1,5.8164793E-1,0E0,5.768899E-2,0E0,6.346691E-2,1.4846534E-1,8.2282364E-2,0E0,0E0,0E0,0E0,0E0,5.2255124E-2,1.9327217E-1,2.18791E-1,1.4397722E-1,9.472704E-2,0E0,0E0,2.8120708E-1,8.8317096E-2,2.2911935E-1,8.082367E-1,1.1179918E-1,2.848286E-1,0E0,0E0,1.2713915E-1,0E0,7.411727E-2,1.8446636E-1,0E0,7.3792696E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,33,33,34,34,36,36,38,38,39,39,40,40,46,46,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,61,61,63,63,64,64,66,66],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,-1,-1,44,46,48,50,52,54,-1,-1,56,58,-1,60,-1,62,64,66,-1,-1,-1,-1,-1,68,70,72,74,76,-1,-1,78,80,82,84,86,88,-1,-1,90,-1,92,94,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5146261E3,7.785302E5,1.1411E4,3.1E1,4.5866325E6,1.5396106E-2,4.0280737E2,6E0,7.3911E4,1.7493458E7,1.45064E5,6.5026445E3,2.506329E0,1.3422592E8,3.84E2,3.79E2,2.5393645E5,5.281628E2,6.27907E0,-5.7005123E-3,1.7068776E10,1.7831801E6,1.3E1,-8.02829E-3,1.2989049E-3,1.1634076E4,4.2E1,6.07E2,3.2216358E-1,4E1,4.75E0,-1.3745549E-3,5.434227E-3,2.511E3,5E0,-2.2082983E-2,2E0,1.722986E-3,8.598131E1,5.974481E1,1.4559597E0,2.1037145E-3,1.5252908E-2,1.2879209E-2,5.2678194E-3,-8.2748495E-3,1.639E3,1.0677966E0,3.2856784E7,3.6828618E8,7.44488E5,-4.9491283E-3,3.4789457E-3,5.04E2,2.9E1,6.0546756E-2,3.068E3,4.362075E8,1.038946E6,3.5963657E-3,-4.232848E-3,3.5364E4,1.0993406E-2,2.224E3,2.0805957E-2,3.2902084E-4,2.6E1,4.9831397E-3,-1.8227387E-3,3.343813E-3,-3.3751358E-3,5.092497E-4,-5.360358E-3,-8.611714E-3,-5.271773E-4,4.8427755E-4,-4.430446E-3,-4.348499E-3,-2.2837115E-2,8.175532E-4,-7.7235955E-3,1.2580412E-3,-2.1499665E-3,-3.0420015E-2,-1.3564031E-3,1.0418927E-2,6.083926E-4,-1.3026803E-3,3.778612E-3,8.179015E-3,2.5169298E-3,2.681363E-3,9.64718E-3,-3.8855888E-3,4.1637262E-3,9.81075E-3,1.4009502E-3],"split_indices":[67,43,9,3,43,0,73,3,44,66,7,4,68,7,0,2,43,4,71,0,46,43,3,0,0,43,0,2,53,3,73,0,0,2,3,0,8,0,73,73,57,0,0,0,0,0,2,68,60,12,9,0,0,7,0,53,2,7,9,0,0,9,0,44,72,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.36E2,8.52E2,8.4E1,4.85E2,3.67E2,9E0,7.5E1,4.44E2,4.1E1,3.08E2,5.9E1,6.9E1,6E0,2E1,4.24E2,3.6E1,5E0,2.98E2,1E1,3E0,5.6E1,6.1E1,8E0,4E0,2E0,1E1,1E1,3.6E2,6.4E1,1E1,2.6E1,3E0,2E0,1.81E2,1.17E2,2E0,8E0,1.5E1,4.1E1,3.8E1,2.3E1,2E0,6E0,2E0,8E0,2E0,8E0,1.84E2,1.76E2,3.1E1,3.3E1,8E0,2E0,1E1,1.6E1,1.72E2,9E0,2.3E1,9.4E1,2E0,6E0,3.6E1,5E0,1.5E1,2.3E1,3E0,2E1,4E0,4E0,9E0,1.75E2,1.62E2,1.4E1,2.5E1,6E0,1.5E1,1.8E1,3E0,7E0,3E0,1.3E1,9.8E1,7.4E1,2E0,7E0,2E1,3E0,5E1,4.4E1,1.6E1,2E1,1E1,5E0,1.3E1,1E1,1.7E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[3.979981E-4,-3.7468873E-2,5.1697336E-2,-5.952634E-2,3.1045843E-2,7.2241604E-2,-4.634605E-2,-5.4383133E-2,-4.5393455E-1,-2.0223354E-1,4.40695E-2,2.3027173E-1,6.212708E-2,-1.2101895E-2,-1.6653527E-1,-7.741962E-2,-1.0435453E-2,-5.137635E-3,-3.0048026E-2,-2.1307643E-3,-1.432867E-2,3.45587E-2,1.4806904E-2,-3.4491045E-3,2.7044997E-1,9.0665385E-2,1.3767932E-2,-4.044393E-2,1.1664745E-1,-7.930547E-2,-1.5121184E-2,-6.524208E-2,-2.2780728E-1,3.3289175E-2,-5.0371427E-2,4.118565E-2,-1.1051179E-2,3.5637561E-3,1.3542284E-2,5.5544313E-2,1.5195853E-1,4.8045088E-3,9.683327E-3,4.302261E-3,-5.937542E-2,9.540178E-3,-5.9823296E-4,-6.2777647E-3,5.309508E-3,-7.059653E-2,9.9530965E-2,-5.122493E-4,-3.6035335E-1,5.4238494E-2,-8.615237E-3,-7.600092E-2,4.408185E-2,-1.7059015E-2,5.7026252E-2,1.7981362E-1,2.6695415E-2,1.6511454E-1,-8.953266E-3,-6.9039896E-2,2.1981485E-2,-7.681891E-2,2.4311452E-3,-2.7289714E-3,-6.8250275E-3,-1.8577588E-3,8.892517E-3,2.6163918E-3,-3.3181368E-3,-2.3613842E-2,-5.968838E-3,-5.130292E-4,5.8897417E-3,-7.2124684E-3,-2.78696E-3,6.4790156E-3,5.2970607E-4,2.5873226E-3,-2.4396454E-3,3.8871244E-3,1.7651517E-4,-2.309647E-3,9.489093E-3,-2.3644343E-3,2.3961856E-3,6.0961484E-3,1.085615E-2,-5.847924E-3,2.746645E-4,6.5390184E-3,3.286089E-4,-1.315447E-3,-6.494584E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,35,-1,-1,37,39,41,43,45,47,-1,49,51,53,55,57,-1,-1,-1,59,61,63,-1,-1,65,-1,-1,-1,-1,67,69,71,73,75,-1,77,79,81,83,85,87,89,-1,91,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.779366E0,7.982827E-1,7.8688335E-1,7.8469443E-1,4.0083376E-1,5.0037825E-1,2.7785754E-1,3.9909017E-1,2.5861943E-1,1.0179326E-1,3.0988908E-1,2.5245798E-1,4.182986E-1,2.0176467E-1,1.817329E-1,4.5945704E-1,2.4090426E-1,0E0,0E0,0E0,0E0,2.262333E-1,0E0,0E0,4.6007156E-2,4.0272152E-1,1.9768608E-1,1.1875552E-1,1.1723511E-1,1.3390297E-1,0E0,2.1965349E-1,5.722298E-1,3.1220296E-1,1.7764711E-1,1.09843194E-1,0E0,0E0,0E0,4.3615887E-1,3.444847E-1,1.410442E-1,0E0,0E0,8.1917986E-2,0E0,0E0,0E0,0E0,2.0377874E-1,1.0992945E-1,3.6183435E-2,3.4518695E-1,2.896615E-1,0E0,6.317115E-2,4.8225254E-2,6.981946E-2,1.3113853E-1,1.3796228E-1,1.9704148E-1,1.251843E-1,0E0,8.998267E-2,1.5824395E-1,1.027742E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,21,21,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,39,39,40,40,41,41,44,44,49,49,50,50,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,36,-1,-1,38,40,42,44,46,48,-1,50,52,54,56,58,-1,-1,-1,60,62,64,-1,-1,66,-1,-1,-1,-1,68,70,72,74,76,-1,78,80,82,84,86,88,90,-1,92,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7667856E2,1.7831801E6,1E0,3.0355552E6,2.5084135E6,1.2083E4,3.9416668E1,1.380863E6,7.9E1,1.8671024E0,2.2503355E1,5.357143E2,8E0,5.53648E8,1.4743204E3,9.932432E0,2.56E2,-5.137635E-3,-3.0048026E-2,-2.1307643E-3,-1.432867E-2,1.7951853E1,1.4806904E-2,-3.4491045E-3,9.2E1,2.43E2,4.91027E0,1.2E1,1.81E2,1E0,-1.5121184E-2,2.4571428E1,1.3027534E2,1E0,3.004207E2,3.0479442E9,-1.1051179E-2,3.5637561E-3,1.3542284E-2,3.63E2,8.604875E3,4E0,9.683327E-3,4.302261E-3,3.2645503E5,9.540178E-3,-5.9823296E-4,-6.2777647E-3,5.309508E-3,1.8E1,3.0805944E7,1.308677E6,6.9E1,7.06E2,-8.615237E-3,2.511352E6,7.6E1,1.6834415E0,1.394E3,4.23025E2,6.374563E5,8.318076E6,-8.953266E-3,1.5E1,5.668E3,2.511E3,2.4311452E-3,-2.7289714E-3,-6.8250275E-3,-1.8577588E-3,8.892517E-3,2.6163918E-3,-3.3181368E-3,-2.3613842E-2,-5.968838E-3,-5.130292E-4,5.8897417E-3,-7.2124684E-3,-2.78696E-3,6.4790156E-3,5.2970607E-4,2.5873226E-3,-2.4396454E-3,3.8871244E-3,1.7651517E-4,-2.309647E-3,9.489093E-3,-2.3644343E-3,2.3961856E-3,6.0961484E-3,1.085615E-2,-5.847924E-3,2.746645E-4,6.5390184E-3,3.286089E-4,-1.315447E-3,-6.494584E-3],"split_indices":[67,43,6,62,60,9,71,9,0,68,71,67,32,7,70,69,10,0,0,0,0,71,0,0,10,0,50,3,0,17,0,71,67,19,67,46,0,0,0,44,4,8,0,0,48,0,0,0,0,8,5,46,10,2,0,5,0,68,10,4,43,60,0,3,44,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.14E2,5.26E2,3.88E2,3.98E2,1.28E2,3.21E2,6.7E1,3.94E2,4E0,6E0,1.22E2,1.8E1,3.03E2,5.3E1,1.4E1,2.58E2,1.36E2,2E0,2E0,3E0,3E0,1.19E2,3E0,2E0,1.6E1,1.9E2,1.13E2,4.4E1,9E0,1E1,4E0,2.4E2,1.8E1,6.5E1,7.1E1,1.17E2,2E0,2E0,1.4E1,1.22E2,6.8E1,1.09E2,4E0,5E0,3.9E1,5E0,4E0,8E0,2E0,2.33E2,7E0,7E0,1.1E1,6E1,5E0,5.6E1,1.5E1,2.5E1,9.2E1,2.2E1,1E2,6.6E1,2E0,2E1,8.9E1,3.4E1,5E0,2.03E2,3E1,3E0,4E0,4E0,3E0,6E0,5E0,3.2E1,2.8E1,8E0,4.8E1,3E0,1.2E1,8E0,1.7E1,6.1E1,3.1E1,2E0,2E1,2.4E1,7.6E1,4.6E1,2E1,1.1E1,9E0,9E0,8E1,2E1,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-3.8300373E-3,-3.215501E-2,5.56003E-2,1.2361349E-1,-4.0770356E-2,7.590109E-2,-9.932792E-2,-7.937023E-6,1.5014476E-1,-5.71178E-2,1.4126223E-2,4.0961403E-2,1.1727297E-1,-6.1112102E-2,-1.507048E-2,1.7323114E-1,-1.4734211E-3,-1.22048855E-1,-4.509503E-2,2.8007982E-2,-6.391847E-3,8.88387E-2,-2.5805034E-2,1.5978731E-1,5.920276E-2,-7.7787978E-3,-3.0183699E-2,9.594745E-4,8.631982E-3,-6.945243E-2,-2.0908527E-1,1.6006902E-2,-5.7696056E-2,-2.8325714E-2,5.1584136E-2,1.23527646E-1,2.7020092E-3,-8.483767E-3,-1.2983649E-2,1.8624122E-1,1.4202158E-2,2.69185E-3,1.15613975E-1,-7.003035E-2,5.2863337E-2,-1.4263515E-1,6.6241436E-3,-3.7350613E-1,-7.27018E-2,3.9591186E-2,-7.849184E-3,-1.5624409E-2,-5.412676E-2,-2.8901682E-3,-8.531484E-2,6.954287E-2,-4.524022E-2,7.865209E-2,2.073965E-1,-6.965614E-3,2.9102052E-2,-6.6815935E-2,3.808836E-2,1.286277E-1,2.3598845E-1,3.8445464E-3,-3.1642884E-3,4.9451444E-2,-6.727387E-3,1.0771811E-2,7.872239E-2,1.9228953E-3,-8.653648E-2,-1.3820705E-3,7.604833E-3,-8.471416E-3,-3.6915382E-3,-1.0330677E-2,2.9523566E-3,-6.090772E-3,-1.9017804E-2,-4.905722E-3,4.8603867E-3,1.042778E-3,1.3159021E-2,-3.040478E-4,-3.3150192E-3,-1.2599333E-3,2.8368735E-3,-5.9010735E-4,-6.2961136E-3,4.1097933E-3,-1.0616791E-3,2.596714E-3,-6.5417383E-3,-2.6584906E-4,5.8476347E-3,1.1709733E-2,3.844239E-3,3.8441513E-3,-5.2227126E-4,3.5527232E-3,-4.7826096E-3,-2.2837964E-3,3.5562196E-3,5.589754E-4,7.044441E-3,1.1935613E-2,3.594244E-3,3.022232E-3,-1.706335E-3,-4.133231E-4,4.883886E-3,-6.122803E-3,-1.1610513E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,-1,27,-1,29,31,33,-1,35,37,39,41,-1,43,-1,-1,45,47,49,51,53,55,57,59,61,-1,63,65,67,69,71,73,75,77,79,81,83,-1,-1,85,87,89,91,93,95,97,-1,99,101,103,105,107,-1,-1,109,-1,-1,111,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6009825E0,8.6928695E-1,9.7310495E-1,1.11686766E-1,5.4912984E-1,3.883778E-1,2.8935397E-1,0E0,1.2302393E-1,3.6218274E-1,3.0050766E-1,4.785853E-1,2.9667354E-1,1.000058E-1,0E0,4.337275E-2,0E0,3.1965828E-1,3.0813235E-1,1.7411828E-1,0E0,2.5962144E-1,2.7411652E-1,2.707969E-1,1.7183264E-1,0E0,8.9739814E-2,0E0,0E0,2.6261687E-1,5.783012E-1,3.0737576E-1,3.1055987E-1,5.628587E-2,1.6311967E-1,2.1703494E-1,1.09018244E-1,1.655441E-1,0E0,1.4161229E-1,7.3189154E-2,2.0006514E-1,9.731141E-2,3.666897E-2,8.71006E-2,4.5536906E-2,3.2625043E-1,4.951501E-2,1.0396756E-1,2.527069E-1,0E0,0E0,2.6726955E-1,4.4944543E-2,4.1913353E-2,1.37653E-1,1.5252058E-1,1.6755599E-1,9.545398E-2,0E0,5.0607033E-2,1.4712593E-1,1.1752762E-1,7.808313E-2,8.175957E-2,0E0,0E0,3.1819273E-2,0E0,0E0,5.1605523E-2,0E0,4.1158505E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,67,67,70,70,72,72],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,-1,28,-1,30,32,34,-1,36,38,40,42,-1,44,-1,-1,46,48,50,52,54,56,58,60,62,-1,64,66,68,70,72,74,76,78,80,82,84,-1,-1,86,88,90,92,94,96,98,-1,100,102,104,106,108,-1,-1,110,-1,-1,112,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.194E3,6E0,2.856934E7,1.1190476E0,3.422351E6,1.1603809E3,4.3E1,-7.937023E-6,1.9756098E1,2.2953334E2,4.1104166E8,1.1623707E0,1.7103828E7,1.2376862E10,-1.507048E-2,5.14E2,-1.4734211E-3,2.0007E5,1.6964285E0,6.54E2,-6.391847E-3,4.612245E0,1.6063418E7,8.913649E-3,1.2E1,-7.7787978E-3,2.2E1,9.594745E-4,8.631982E-3,1.1176471E1,5.8182236E1,3.1E1,1.342376E0,6.3844144E-1,1E0,7.286575E-1,2.0480016E7,5.486433E5,-1.2983649E-2,3.97371E5,8.598131E1,1.2816234E7,1.4076087E0,3.1146461E1,4.435876E2,1.5827801E4,3.964269E3,1.43E2,2.284E3,1.6942337E1,-7.849184E-3,-1.5624409E-2,5.004138E-3,1.157602E6,8.8790035E-1,1.124641E0,1.2316317E-3,6.5718125E6,1.2E1,-6.965614E-3,3.7890625E0,5.249058E5,4E0,1E0,1.1865458E9,3.8445464E-3,-3.1642884E-3,6E0,-6.727387E-3,1.0771811E-2,1.6471942E3,1.9228953E-3,4.9236734E5,-1.3820705E-3,7.604833E-3,-8.471416E-3,-3.6915382E-3,-1.0330677E-2,2.9523566E-3,-6.090772E-3,-1.9017804E-2,-4.905722E-3,4.8603867E-3,1.042778E-3,1.3159021E-2,-3.040478E-4,-3.3150192E-3,-1.2599333E-3,2.8368735E-3,-5.9010735E-4,-6.2961136E-3,4.1097933E-3,-1.0616791E-3,2.596714E-3,-6.5417383E-3,-2.6584906E-4,5.8476347E-3,1.1709733E-2,3.844239E-3,3.8441513E-3,-5.2227126E-4,3.5527232E-3,-4.7826096E-3,-2.2837964E-3,3.5562196E-3,5.589754E-4,7.044441E-3,1.1935613E-2,3.594244E-3,3.022232E-3,-1.706335E-3,-4.133231E-4,4.883886E-3,-6.122803E-3,-1.1610513E-3],"split_indices":[2,3,60,68,9,67,3,0,73,48,7,57,62,46,0,2,0,5,68,2,0,69,62,72,8,0,3,0,0,67,62,3,69,53,19,57,5,43,0,1,73,9,69,71,71,43,43,2,0,71,0,0,53,12,71,53,53,60,8,0,68,60,8,15,47,0,0,8,0,0,67,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.49E2,6.43E2,3.06E2,3.3E1,6.1E2,2.71E2,3.5E1,6E0,2.7E1,4.7E2,1.4E2,1.48E2,1.23E2,3.1E1,4E0,2.4E1,3E0,7.2E1,3.98E2,1.29E2,1.1E1,8.6E1,6.2E1,7E1,5.3E1,6E0,2.5E1,2E0,2.2E1,4.6E1,2.6E1,6.8E1,3.3E2,3.8E1,9.1E1,6.1E1,2.5E1,5.9E1,3E0,5.9E1,1.1E1,2.7E1,2.6E1,1.7E1,8E0,2.3E1,2.3E1,1.1E1,1.5E1,6.1E1,7E0,3E0,3.27E2,2.7E1,1.1E1,7.7E1,1.4E1,4.1E1,2E1,3E0,2.2E1,2.6E1,3.3E1,2.9E1,3E1,6E0,5E0,2.1E1,6E0,5E0,2.1E1,2E0,1.5E1,5E0,3E0,1.3E1,1E1,4E0,1.9E1,2E0,9E0,1.3E1,2E0,5.8E1,3E0,8.7E1,2.4E2,2E1,7E0,5E0,6E0,6.4E1,1.3E1,7E0,7E0,1.5E1,2.6E1,1.4E1,6E0,9E0,1.3E1,5E0,2.1E1,1E1,2.3E1,5E0,2.4E1,2.6E1,4E0,1.8E1,3E0,5E0,1.6E1,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[2.9021583E-3,-1.2035605E-2,1.0889493E-1,-3.8932253E-2,2.2987645E-2,1.2850873E-1,-5.1865783E-2,-2.8019058E-2,-1.1152026E-1,2.8119748E-2,-1.9928321E-1,-5.4466106E-2,1.4271395E-1,-9.600136E-2,3.3224823E-3,-6.338586E-2,-3.0054364E-3,-2.3623817E-1,-6.363375E-2,-8.216396E-3,6.4931706E-2,-2.118475E-2,-1.7651966E-3,1.8632758E-3,-4.309086E-3,1.6906755E-1,3.6752284E-2,3.6230034E-4,-7.7432343E-3,-5.5965252E-2,-1.3401993E-2,6.859482E-2,-3.1713292E-2,-6.59019E-3,-2.940221E-1,-7.821824E-2,3.058751E-3,-5.1624157E-2,2.7137997E-2,7.4467085E-2,-7.9667605E-2,2.0445955E-1,1.023574E-1,-6.5057375E-2,1.259654E-1,-2.8313525E-2,-9.1113E-2,1.096798E-1,1.0977271E-2,-7.845873E-2,1.1142932E-2,-5.9303255E-3,-1.6557833E-2,-4.215899E-2,-6.0307374E-3,-3.142495E-2,-2.8877714E-1,3.690869E-2,-1.33968005E-2,3.4215093E-2,1.1741125E-1,1.0899132E-3,-4.8918175E-3,1.3153204E-1,2.6110852E-1,9.18303E-3,6.1256595E-2,-5.155262E-3,1.4575572E-3,1.8607074E-1,2.0866008E-3,-3.389065E-3,8.768414E-4,-5.794276E-3,-2.3163222E-3,2.7411992E-3,7.7016857E-3,7.843497E-3,-9.764891E-4,5.106806E-3,-4.42786E-3,5.465177E-3,-4.12109E-4,-6.854104E-3,-3.9465947E-4,2.171055E-3,-4.512631E-3,-6.178282E-4,-1.9682115E-2,9.218176E-4,7.629225E-3,3.2672584E-3,-1.37955E-3,1.6125501E-3,7.1725603E-3,7.1251392E-3,1.0637576E-3,1.28719015E-2,2.26866E-3,2.6216608E-4,4.7670957E-3,3.5619806E-3,1.25580905E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,-1,-1,41,43,-1,-1,45,-1,47,49,-1,51,53,-1,55,57,59,61,63,65,67,69,71,73,75,77,79,81,-1,-1,83,-1,85,87,89,-1,91,93,-1,-1,95,97,-1,99,-1,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4484421E0,7.5638545E-1,3.6296892E-1,3.565191E-1,4.0371403E-1,2.702918E-1,7.760629E-2,3.501315E-1,3.3733326E-1,4.5799434E-1,3.1340623E-1,3.259086E-2,2.562784E-1,7.446243E-2,0E0,2.574277E-1,4.8110452E-1,4.574001E-2,8.818424E-2,2.668963E-1,2.4033022E-1,0E0,0E0,0E0,0E0,1.5614748E-1,1.898501E-1,0E0,0E0,1.5292358E-1,0E0,1.5739948E-1,3.3543077E-1,0E0,4.7517717E-2,6.874874E-2,0E0,3.645687E-1,3.0627972E-1,2.7255678E-1,3.1403042E-2,1.6395688E-1,9.734216E-2,4.761485E-2,4.5271486E-2,1.9262546E-1,9.058863E-2,1.0089645E-1,1.4974059E-1,2.627073E-1,1.8828793E-1,0E0,0E0,8.7323934E-2,0E0,3.7859583E-1,2.16577E-1,2.0095888E-1,0E0,1.9545001E-1,2.2917819E-1,0E0,0E0,5.0657153E-2,7.295871E-2,0E0,4.759665E-2,0E0,0E0,3.4466133E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,25,25,26,26,29,29,31,31,32,32,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,53,53,55,55,56,56,57,57,59,59,60,60,63,63,64,64,66,66,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,-1,-1,42,44,-1,-1,46,-1,48,50,-1,52,54,-1,56,58,60,62,64,66,68,70,72,74,76,78,80,82,-1,-1,84,-1,86,88,90,-1,92,94,-1,-1,96,98,-1,100,-1,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.4978E3,7.785302E5,2.4157143E2,1E0,1.9269184E7,1E0,5.7254577E0,1.5199841E2,3.3817584E7,1E0,1E0,1.91E2,1.2820834E7,4.5698816E7,3.3224823E-3,2.732381E2,2.701265E8,4.1E1,1.8314E4,2.4888637E2,2E1,-2.118475E-2,-1.7651966E-3,1.8632758E-3,-4.309086E-3,1.0014754E2,2.4081968E3,3.6230034E-4,-7.7432343E-3,1.7039736E4,-1.3401993E-2,1E0,1.7161779E-1,-6.59019E-3,1.584022E0,5.90352E5,3.058751E-3,5.3386245E0,1.055912E3,5.831829E6,1.0937695E6,4E0,2.047619E0,1.7899E4,9.2732E4,3.9125E1,2.118835E6,1.4580257E-1,2.4595E5,1.0153885E-5,3.3382E4,-5.9303255E-3,-1.6557833E-2,1.4141E4,-6.0307374E-3,1.319909E3,1.7685275E6,1.5707873E7,-1.33968005E-2,4.1104166E8,1E0,1.0899132E-3,-4.8918175E-3,1.83154E6,7.71183E6,9.18303E-3,3.0427273E3,-5.155262E-3,1.4575572E-3,3.359873E6,2.0866008E-3,-3.389065E-3,8.768414E-4,-5.794276E-3,-2.3163222E-3,2.7411992E-3,7.7016857E-3,7.843497E-3,-9.764891E-4,5.106806E-3,-4.42786E-3,5.465177E-3,-4.12109E-4,-6.854104E-3,-3.9465947E-4,2.171055E-3,-4.512631E-3,-6.178282E-4,-1.9682115E-2,9.218176E-4,7.629225E-3,3.2672584E-3,-1.37955E-3,1.6125501E-3,7.1725603E-3,7.1251392E-3,1.0637576E-3,1.28719015E-2,2.26866E-3,2.6216608E-4,4.7670957E-3,3.5619806E-3,1.25580905E-2],"split_indices":[70,43,73,6,62,17,69,67,7,27,8,0,9,62,0,4,46,0,6,67,6,0,0,0,0,73,67,0,0,62,0,26,53,0,68,9,0,69,67,60,43,8,68,44,44,4,9,53,9,53,9,0,0,9,0,48,60,12,0,7,29,0,0,9,58,0,4,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.13E2,8.01E2,1.12E2,4.53E2,3.48E2,1E2,1.2E1,3.95E2,5.8E1,3.41E2,7E0,7E0,9.3E1,9E0,3E0,1.63E2,2.32E2,1.5E1,4.3E1,1.72E2,1.69E2,2E0,5E0,2E0,5E0,7.4E1,1.9E1,4E0,5E0,1.59E2,4E0,6.6E1,1.66E2,7E0,8E0,3.9E1,4E0,7.7E1,9.5E1,1.59E2,1E1,4.7E1,2.7E1,9E0,1E1,9E1,6.9E1,3.8E1,2.8E1,7.9E1,8.7E1,3E0,5E0,2.4E1,1.5E1,7.2E1,5E0,9.3E1,2E0,8.3E1,7.6E1,2E0,8E0,2.2E1,2.5E1,7E0,2E1,6E0,3E0,5E0,5E0,4.6E1,4.4E1,3.7E1,3.2E1,2.1E1,1.7E1,4E0,2.4E1,6E0,7.3E1,1.3E1,7.4E1,5E0,1.9E1,3.3E1,3.9E1,2E0,3E0,8.3E1,1E1,5.3E1,3E1,2.4E1,5.2E1,1.8E1,4E0,2.3E1,2E0,9E0,1.1E1,3E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[6.079362E-4,-5.827161E-2,2.8437484E-2,-4.9182422E-2,-9.463307E-3,3.839283E-2,-1.13573454E-1,6.8455584E-2,-5.710695E-2,2.4252163E-2,1.196663E-1,-7.959312E-2,-1.5920157E-2,1.1305185E-1,1.1298014E-2,-1.23853646E-1,-4.374067E-2,1.45542035E-2,1.003637E-1,1.3149253E-1,-7.238651E-3,-9.121599E-2,3.8386423E-3,6.6736042E-3,-2.70387E-4,-3.4201422E-3,3.0682597E-3,-1.2797057E-2,-4.9124374E-3,-1.1683528E-2,-8.3939515E-2,1.7451064E-1,9.684464E-3,-1.3634256E-2,1.3514528E-1,1.0245282E-1,2.5630903E-1,-5.901051E-3,-2.109505E-2,3.1501972E-3,-1.630042E-1,-1.0109721E-1,3.4702316E-2,3.2257186E-3,1.076932E-2,-3.863494E-3,5.749951E-2,2.4179192E-3,-3.2253477E-3,9.628412E-2,9.3604475E-3,1.2933283E-1,1.9107632E-2,3.0100596E-1,6.4690164E-3,-2.775961E-3,5.92718E-3,-6.8397014E-3,7.4393064E-4,-9.409844E-3,4.3278717E-4,-3.3698403E-3,-8.600581E-3,7.439875E-3,-1.21696154E-4,-1.3234649E-3,9.857243E-4,3.9390596E-3,-3.8174563E-3,7.428787E-3,2.778621E-3,9.797994E-3,4.7610193E-3,5.300433E-3,5.8996525E-6,1.564991E-2,4.9304324E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,-1,-1,-1,-1,-1,-1,-1,39,41,43,45,47,49,51,53,-1,55,57,59,61,63,-1,-1,65,67,-1,-1,69,-1,71,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.500968E0,3.7801826E-1,8.83724E-1,2.6306802E-1,0E0,6.659677E-1,2.9143292E-1,4.4604987E-2,2.2743505E-1,3.6538824E-1,3.0047262E-1,7.725264E-2,0E0,3.70207E-2,4.659829E-2,8.952451E-2,2.812359E-1,3.4260422E-1,2.2388422E-1,2.7458513E-1,0E0,8.4860116E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.7660534E-1,2.0096344E-1,6.1924815E-2,2.7877456E-1,5.4794576E-2,9.111273E-2,1.5211082E-1,3.6717176E-2,0E0,8.2131736E-2,2.2040159E-1,7.641995E-2,1.9067496E-1,6.119247E-2,0E0,0E0,2.0767108E-1,3.676008E-1,0E0,0E0,5.7688117E-2,0E0,9.2476904E-2,3.2395396E-2,3.35415E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,41,41,42,42,45,45,46,46,49,49,51,51,52,52,53,53],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,-1,-1,-1,-1,-1,-1,-1,40,42,44,46,48,50,52,54,-1,56,58,60,62,64,-1,-1,66,68,-1,-1,70,-1,72,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.06E2,3.1E1,1E0,1.0677966E0,-9.463307E-3,1.5146261E3,4.238532E0,4.31903E6,3.89E2,4.82425E6,7.706545E7,1E0,-1.5920157E-2,9.07021E5,5.8687E4,1.1649485E0,4.201202E-2,1.4E1,1.7068776E10,6.5026445E3,-7.238651E-3,7.295292E-2,3.8386423E-3,6.6736042E-3,-2.70387E-4,-3.4201422E-3,3.0682597E-3,-1.2797057E-2,-4.9124374E-3,1E0,2.3572223E2,3.3527696E-1,9.751103E6,2.0317461E0,5.933284E2,1.3358269E2,1E0,-5.901051E-3,5.9395876E-7,3.964269E3,1.6105798E3,1.7500778E6,1.7407408E0,3.2257186E-3,1.076932E-2,1E0,4.4744192E8,2.4179192E-3,-3.2253477E-3,4E0,9.3604475E-3,2.5959E4,2.01E2,1.94E3,6.4690164E-3,-2.775961E-3,5.92718E-3,-6.8397014E-3,7.4393064E-4,-9.409844E-3,4.3278717E-4,-3.3698403E-3,-8.600581E-3,7.439875E-3,-1.21696154E-4,-1.3234649E-3,9.857243E-4,3.9390596E-3,-3.8174563E-3,7.428787E-3,2.778621E-3,9.797994E-3,4.7610193E-3,5.300433E-3,5.8996525E-6,1.564991E-2,4.9304324E-3],"split_indices":[2,8,83,68,0,67,68,60,1,43,59,16,0,9,1,68,57,10,46,4,0,53,0,0,0,0,0,0,0,16,67,71,9,69,67,73,19,0,52,43,48,60,68,0,0,27,7,0,0,3,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.14E2,2.93E2,6.21E2,2.77E2,1.6E1,5.81E2,4E1,1.7E1,2.6E2,4.96E2,8.5E1,3.6E1,4E0,9E0,8E0,4.2E1,2.18E2,4.41E2,5.5E1,8.2E1,3E0,3.4E1,2E0,7E0,2E0,3E0,5E0,3E0,3.9E1,1.22E2,9.6E1,1.2E1,4.29E2,1.3E1,4.2E1,6.8E1,1.4E1,2.2E1,1.2E1,1.12E2,1E1,8.4E1,1.2E1,5E0,7E0,3.35E2,9.4E1,6E0,7E0,2.8E1,1.4E1,5.1E1,1.7E1,9E0,5E0,1E1,2E0,8E0,1.04E2,8E0,2E0,6.4E1,2E1,2E0,1E1,1.69E2,1.66E2,7.9E1,1.5E1,9E0,1.9E1,1.1E1,4E1,2E0,1.5E1,7E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-4.0874146E-3,-3.6411386E-2,3.8605656E-2,-5.150025E-2,1.2501272E-2,5.9429366E-2,-6.487448E-2,-2.3710556E-2,-1.0222523E-1,2.264525E-2,-1.1423525E-2,8.857316E-2,1.6937068E-2,-1.6296224E-1,-3.7442494E-2,-4.182704E-2,5.9591558E-2,-8.9799866E-2,-4.3963066E-1,8.392562E-2,-3.2570682E-2,1.0628593E-1,1.2850821E-2,2.8622472E-3,1.6917577E-1,-6.428793E-2,-9.574542E-3,-5.7165537E-2,5.9000064E-2,-5.990595E-2,3.5665706E-2,1.7325427E-2,1.8846834E-1,-2.6832944E-1,-7.301675E-2,-2.9371228E-2,-4.705699E-3,2.0685745E-2,1.3861407E-1,7.941832E-2,-5.8448613E-2,3.6227144E-2,1.3617137E-1,1.1928529E-1,-5.5781275E-2,-1.6937438E-2,7.3086575E-2,3.4531187E-3,1.0528107E-2,-6.869526E-3,6.6733063E-4,-2.7962929E-2,-1.4000957E-1,-2.0019668E-3,7.89026E-3,-4.0118154E-2,-1.6292025E-1,-3.212737E-2,9.0673864E-2,4.8798166E-2,-7.378994E-3,8.662251E-2,1.4590729E-2,-1.7072318E-1,-1.957323E-2,-1.6385086E-2,-6.386144E-2,3.990268E-3,-1.8099682E-2,1.5852271E-1,7.0122816E-2,-3.0148748E-4,7.1609435E-3,-8.171813E-3,-2.8486038E-2,1.5024278E-1,8.218014E-3,1.8000482E-1,1.02583E-1,-4.7569657E-3,1.5824398E-1,-1.2614132E-2,-2.5254322E-2,-1.18714E-1,-3.4781331E-3,1.17040366E-1,-3.8031088E-3,5.3924103E-3,-3.983842E-2,-1.2710398E-2,-1.8232594E-3,-8.612439E-3,-1.4371298E-3,-2.717182E-4,-1.0850828E-2,9.896044E-4,-5.343522E-3,1.0876708E-2,1.8920144E-3,-1.687851E-3,3.2114051E-3,5.868639E-3,-9.943369E-4,-9.753723E-3,-1.0610789E-3,-3.4576268E-3,3.9694617E-3,-3.6563708E-3,1.1047744E-3,1.0410739E-3,7.90077E-3,6.2329706E-4,6.402849E-3,7.552892E-4,-5.5353804E-3,9.270739E-3,-1.4305476E-3,1.6716332E-3,-3.4036245E-3,5.3873206E-3,1.1255727E-2,6.3680974E-3,2.2617246E-3,9.456822E-3,1.693079E-3,-4.3841542E-4,-5.935725E-3,3.9572187E-5,-6.6371914E-3,-1.0015236E-3,4.5884633E-3,1.0326391E-2,3.5131506E-3,-3.3831784E-6,-3.3126667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,65,-1,-1,67,69,71,73,75,77,79,81,83,85,-1,-1,-1,-1,87,89,-1,-1,91,93,95,97,99,-1,101,-1,103,-1,-1,105,-1,107,109,111,-1,-1,-1,113,115,117,119,121,-1,123,-1,125,127,129,131,-1,-1,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.328975E0,4.0508777E-1,8.9811414E-1,5.882119E-1,3.4596074E-1,4.269817E-1,1.822502E-1,4.12979E-1,5.8099437E-1,4.2907375E-1,0E0,2.7359092E-1,3.037838E-1,5.0274312E-2,1.0989265E-1,3.1595704E-1,2.6364374E-1,4.1187227E-1,2.6085204E-1,2.0464212E-1,1.9881082E-1,3.426273E-1,2.9884672E-1,1.8338287E-1,4.834816E-2,4.1478597E-2,0E0,1.0979025E-1,1.2022243E-1,3.644575E-1,1.6326752E-1,2.1821728E-1,1.2591016E-1,1.068058E-1,3.1758404E-1,0E0,0E0,7.478997E-2,3.251809E-2,8.425498E-2,1.8823348E-1,1.6163269E-1,1.5232086E-1,1.5429834E-1,1.5757322E-1,1.406808E-1,2.0696539E-1,0E0,0E0,0E0,0E0,6.614314E-2,1.505204E-1,0E0,0E0,2.0018987E-1,3.1929445E-1,9.085086E-2,1.6229315E-1,5.9376083E-2,0E0,3.6882203E-2,0E0,4.7131002E-2,0E0,0E0,2.0971012E-1,0E0,4.9994428E-2,3.33094E-2,3.2638513E-2,0E0,0E0,0E0,1.8415071E-1,9.499547E-2,9.74202E-2,1.669507E-1,1.1933595E-1,0E0,6.951928E-2,0E0,3.606482E-2,3.455335E-2,1.7179722E-1,8.3449066E-2,0E0,0E0,4.24353E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,51,51,52,52,55,55,56,56,57,57,58,58,59,59,61,61,63,63,66,66,68,68,69,69,70,70,74,74,75,75,76,76,77,77,78,78,80,80,82,82,83,83,84,84,85,85,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,66,-1,-1,68,70,72,74,76,78,80,82,84,86,-1,-1,-1,-1,88,90,-1,-1,92,94,96,98,100,-1,102,-1,104,-1,-1,106,-1,108,110,112,-1,-1,-1,114,116,118,120,122,-1,124,-1,126,128,130,132,-1,-1,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7667856E2,4.39776E6,1E0,3.53969E5,1E0,8E0,2.992648E-2,7.271717E6,4.1388846E2,2.56E2,-1.1423525E-2,2.342612E0,1.4669823E5,1.8256016E7,5.854E3,7.026624E7,2.4210527E0,3.3817584E7,2.93E2,1.6494E4,1.1962106E-1,5.304868E2,3.72424E5,1.9616238E2,1.6E1,1.1269E4,-9.574542E-3,5.0467205E1,1.5896863E3,6.33E3,6.99E2,2.2272727E0,9E0,3.7666667E0,4.157143E1,-2.9371228E-2,-4.705699E-3,5.13657E6,1.6566709E6,5.320943E9,1.2E1,2.4361508E6,8.025477E0,4E0,3.0416667E0,3.9960475E0,3.430961E7,3.4531187E-3,1.0528107E-2,-6.869526E-3,6.6733063E-4,1E0,4.888E3,-2.0019668E-3,7.89026E-3,6.25E0,1.8877551E-1,3.074398E0,6.613774E5,5.483871E-1,-7.378994E-3,1.3E1,1.4590729E-2,8.39777E5,-1.957323E-2,-1.6385086E-2,1.3131897E5,3.990268E-3,5E0,1.193128E0,2.8055556E0,-3.0148748E-4,7.1609435E-3,-8.171813E-3,1.8E1,2.1340163E8,1E0,1.32476E5,4.5478998E8,-4.7569657E-3,1.6E1,-1.2614132E-2,2.880929E6,1.1E1,1.68991E6,3.6828618E8,-3.8031088E-3,5.3924103E-3,1.307E3,-1.2710398E-2,-1.8232594E-3,-8.612439E-3,-1.4371298E-3,-2.717182E-4,-1.0850828E-2,9.896044E-4,-5.343522E-3,1.0876708E-2,1.8920144E-3,-1.687851E-3,3.2114051E-3,5.868639E-3,-9.943369E-4,-9.753723E-3,-1.0610789E-3,-3.4576268E-3,3.9694617E-3,-3.6563708E-3,1.1047744E-3,1.0410739E-3,7.90077E-3,6.2329706E-4,6.402849E-3,7.552892E-4,-5.5353804E-3,9.270739E-3,-1.4305476E-3,1.6716332E-3,-3.4036245E-3,5.3873206E-3,1.1255727E-2,6.3680974E-3,2.2617246E-3,9.456822E-3,1.693079E-3,-4.3841542E-4,-5.935725E-3,3.9572187E-5,-6.6371914E-3,-1.0015236E-3,4.5884633E-3,1.0326391E-2,3.5131506E-3,-3.3831784E-6,-3.3126667E-3],"split_indices":[67,9,6,9,19,32,53,60,4,10,0,57,48,5,2,7,68,7,0,1,53,70,9,71,3,1,0,73,4,44,2,68,8,68,67,0,0,9,43,46,3,60,71,3,71,69,62,0,0,0,0,26,10,0,0,67,73,71,60,71,0,3,0,9,0,0,48,0,8,68,71,0,0,0,3,7,30,1,7,0,3,0,1,3,43,12,0,0,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.61E2,5.47E2,4.14E2,4.18E2,1.29E2,3.45E2,6.9E1,2.71E2,1.47E2,1.25E2,4E0,2.04E2,1.41E2,1.4E1,5.5E1,2.23E2,4.8E1,1.43E2,4E0,5.9E1,6.6E1,1.65E2,3.9E1,1.3E2,1.1E1,5E0,9E0,4.6E1,9E0,1.81E2,4.2E1,3.7E1,1.1E1,1.1E1,1.32E2,2E0,2E0,2.8E1,3.1E1,1.2E1,5.4E1,5E1,1.15E2,1.5E1,2.4E1,1.02E2,2.8E1,5E0,6E0,2E0,3E0,3.5E1,1.1E1,5E0,4E0,1.53E2,2.8E1,1.9E1,2.3E1,3.2E1,5E0,7E0,4E0,8E0,3E0,3E0,1.29E2,1E1,1.8E1,2.3E1,8E0,6E0,6E0,1E1,4.4E1,9E0,4.1E1,4.8E1,6.7E1,2E0,1.3E1,2E0,2.2E1,1.1E1,9.1E1,2.2E1,6E0,2E0,3.3E1,4E0,7E0,8E0,1.45E2,9E0,1.9E1,1.2E1,7E0,5E0,1.8E1,6E0,2.6E1,5E0,2E0,6E0,2E0,1.21E2,8E0,7E0,1.1E1,2E0,2.1E1,5E0,3E0,3E1,1.4E1,7E0,2E0,3.1E1,1E1,2.5E1,2.3E1,4E1,2.7E1,9E0,4E0,2E1,2E0,2E0,9E0,7.8E1,1.3E1,5E0,1.7E1,1.5E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[1.0551693E-3,-2.7874727E-2,4.2458728E-2,-4.419874E-2,2.4946105E-2,5.3307876E-2,-1.16623074E-1,-4.0162466E-2,-2.6649648E-2,2.4076323E-3,1.1153993E-1,5.6987554E-2,-1.10201865E-1,-9.040352E-3,-3.5872277E-2,-3.200905E-2,-1.4449082E-1,8.7363675E-2,-1.8342715E-2,1.9829741E-1,4.4316167E-4,3.3380378E-2,9.601059E-2,-7.7768154E-3,-5.294233E-4,1.5311736E-3,-3.2604656E-3,-2.5385221E-2,-1.4864396E-1,-3.1939417E-1,-3.5626203E-2,5.6990953E-3,8.966519E-4,-2.5615928E-1,1.0828795E-2,6.5265377E-3,1.4330183E-2,3.8409583E-2,-1.8452224E-1,1.350871E-1,4.1689266E-2,-8.206027E-3,-2.0072788E-2,-2.7948283E-2,-2.1148407E-1,-1.0151349E-3,-3.8309488E-1,-7.0171766E-2,1.563552E-3,-2.6542716E-2,-3.9417106E-3,-7.11057E-2,5.1579468E-2,1.7384738E-2,7.8465745E-2,-1.6500033E-2,2.1972293E-3,1.1236149E-1,1.2524252E-2,1.16518565E-1,1.5753563E-2,5.491535E-4,-1.599633E-3,3.2254125E-3,-3.345782E-3,-1.3901711E-2,-5.556585E-3,-2.0197514E-2,-6.3229073E-3,-5.201206E-3,2.3965798E-3,-1.4398505E-4,-1.0010264E-2,4.020327E-3,3.2054994E-4,5.013302E-3,1.415853E-4,4.636599E-3,-1.1884497E-3,6.2117223E-3,-3.8319547E-3,1.0730035E-3,7.7142133E-3,-5.920467E-4,5.6557013E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,29,31,33,35,-1,37,39,-1,-1,-1,-1,41,43,45,47,-1,-1,49,51,-1,-1,53,55,57,59,-1,61,63,65,-1,67,69,-1,-1,-1,71,73,75,77,-1,-1,79,-1,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0804019E0,4.5895624E-1,6.4652675E-1,8.435756E-1,2.4564473E-1,2.1565896E-1,1.4373016E-1,3.3903056E-1,0E0,1.796521E-1,2.2821632E-1,3.109324E-1,4.3168873E-2,0E0,3.2924805E-2,2.8671288E-1,5.408041E-1,4.553801E-2,5.716463E-1,4.58861E-2,0E0,2.4160501E-1,2.660724E-1,0E0,0E0,0E0,0E0,2.8294888E-1,1.4577991E-1,2.0014548E-1,4.7809497E-2,0E0,0E0,4.3688226E-1,2.5074515E-1,0E0,0E0,1.7549255E-1,2.1406384E-1,1.9543815E-1,1.04905695E-1,0E0,1.5839125E-1,3.9233103E-2,6.885964E-2,0E0,5.4816246E-2,7.168094E-2,0E0,0E0,0E0,2.4439614E-1,7.730508E-2,1.7988865E-1,1.6154656E-1,0E0,0E0,2.7940577E-1,0E0,6.073758E-2,1.281835E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,27,27,28,28,29,29,30,30,33,33,34,34,37,37,38,38,39,39,40,40,42,42,43,43,44,44,46,46,47,47,51,51,52,52,53,53,54,54,57,57,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,30,32,34,36,-1,38,40,-1,-1,-1,-1,42,44,46,48,-1,-1,50,52,-1,-1,54,56,58,60,-1,62,64,66,-1,68,70,-1,-1,-1,72,74,76,78,-1,-1,80,-1,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.518317E2,1.7831801E6,1E0,4.5158855E6,2.4210527E0,4.1325716E7,2.5330253E-2,9.369565E0,-2.6649648E-2,1.1057693E0,4.70127E5,1.2456025E3,3.82E3,-9.040352E-3,3.6451373E8,3.0084E4,2E0,1.3E1,1.4076087E0,1.3E1,4.4316167E-4,1.5449402E7,1.7103828E7,-7.7768154E-3,-5.294233E-4,1.5311736E-3,-3.2604656E-3,1.7857143E1,5.7777777E0,3.95E2,1.6254545E1,5.6990953E-3,8.966519E-4,5.1942E4,2.524828E2,6.5265377E-3,1.4330183E-2,1.8062708E6,3.596E3,2.59672E3,1.4598765E0,-8.206027E-3,1.9010988E0,3.000176E5,7.3911E4,-1.0151349E-3,5.75E2,1.4601138E4,1.563552E-3,-2.6542716E-2,-3.9417106E-3,6E0,1E0,6.5434785E0,2.3821254E7,-1.6500033E-2,2.1972293E-3,4.7238097E0,1.2524252E-2,7.886154E1,4.907764E3,5.491535E-4,-1.599633E-3,3.2254125E-3,-3.345782E-3,-1.3901711E-2,-5.556585E-3,-2.0197514E-2,-6.3229073E-3,-5.201206E-3,2.3965798E-3,-1.4398505E-4,-1.0010264E-2,4.020327E-3,3.2054994E-4,5.013302E-3,1.415853E-4,4.636599E-3,-1.1884497E-3,6.2117223E-3,-3.8319547E-3,1.0730035E-3,7.7142133E-3,-5.920467E-4,5.6557013E-3],"split_indices":[70,43,83,62,68,66,53,69,0,69,12,70,0,0,7,44,32,3,69,3,0,62,62,0,0,0,0,4,71,2,69,0,0,1,4,0,0,43,2,67,69,0,68,43,44,0,2,48,0,0,0,8,26,73,9,0,0,68,0,73,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9E2,5.3E2,3.7E2,4.05E2,1.25E2,3.47E2,2.3E1,4.03E2,2E0,1E2,2.5E1,3.4E2,7E0,1.1E1,1.2E1,3.75E2,2.8E1,1.9E1,8.1E1,1.3E1,1.2E1,2.13E2,1.27E2,4E0,3E0,4E0,8E0,3.56E2,1.9E1,1E1,1.8E1,1.2E1,7E0,8E0,7.3E1,1E1,3E0,2.09E2,4E0,7.3E1,5.4E1,1.1E1,3.45E2,7E0,1.2E1,2E0,8E0,1.2E1,6E0,2E0,6E0,2.4E1,4.9E1,1.38E2,7.1E1,2E0,2E0,6.4E1,9E0,1.3E1,4.1E1,1.07E2,2.38E2,2E0,5E0,5E0,7E0,6E0,2E0,9E0,3E0,1.7E1,7E0,2.7E1,2.2E1,1.8E1,1.2E2,5.9E1,1.2E1,5.8E1,6E0,5E0,8E0,3.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[5.0494825E-3,-5.815605E-3,1.10062435E-1,-3.4690963E-3,-3.357041E-1,1.5350702E-1,2.5228618E-2,5.8135185E-2,-1.3930225E-2,-3.1156961E-3,-2.1082403E-2,1.2881878E-1,1.8111845E-2,-8.135994E-2,6.584029E-2,3.2094993E-2,1.3118401E-1,-4.7211625E-2,4.0472806E-3,1.4833118E-1,-1.0874943E-3,-8.0004735E-3,-3.4213133E-4,2.0169942E-2,5.293904E-3,7.8604296E-2,-1.6123101E-2,-9.801005E-3,1.610036E-1,-3.80759E-2,-1.4182124E-1,8.645714E-3,-1.8903683E-1,1.7531283E-1,2.8269578E-2,2.5967602E-3,-3.4432218E-3,2.9509721E-2,1.3798283E-1,-5.9288964E-2,9.801248E-2,1.8771233E-1,3.6195137E-3,-2.9804137E-2,-1.2967347E-1,-1.4762836E-2,-2.9281503E-1,-8.544653E-3,7.496381E-2,-2.1555722E-2,-2.8181434E-2,9.509635E-2,2.1643022E-1,-2.8477116E-3,6.046753E-3,-3.3090643E-3,2.8675687E-3,2.9401951E-3,8.780019E-3,-6.671182E-3,-5.9551897E-4,6.472938E-3,2.003495E-4,8.100831E-4,9.481537E-3,-1.7089846E-3,4.2181234E-3,-7.603427E-3,-1.3561976E-3,-4.826796E-3,1.6898211E-3,-1.6855396E-2,-4.4599606E-4,-6.8549816E-3,-1.4088357E-4,4.5694443E-3,9.194059E-5,-5.2138125E-3,2.07401E-3,5.879941E-3,3.3223117E-4,1.165761E-2,5.2255164E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,29,31,33,-1,-1,-1,35,-1,37,39,-1,41,43,45,47,49,51,53,-1,-1,55,57,59,61,63,-1,65,67,69,71,73,75,-1,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0708745E0,6.575829E-1,3.2045746E-1,5.460148E-1,1.5924495E-1,2.8766763E-1,1.3954768E-1,2.2976094E-1,4.3337226E-1,0E0,0E0,1.6383904E-1,0E0,5.912038E-2,4.811213E-2,2.0758565E-1,3.602031E-1,2.141391E-1,4.1982794E-1,1.5053809E-1,0E0,0E0,0E0,4.7812115E-2,0E0,1.3294086E-1,2.3244205E-1,0E0,5.2037835E-2,1.726731E-1,4.1774783E-1,5.261955E-1,4.725555E-1,1.0805154E-1,9.9025294E-2,0E0,0E0,9.176125E-2,6.7828596E-2,1.3116542E-1,4.9549416E-2,5.6986094E-2,0E0,1.8319131E-1,5.9809446E-2,6.29052E-2,1.9367081E-1,2.8060722E-1,1.6127777E-1,0E0,5.3861756E-2,4.0434733E-2,6.0896397E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,28,28,29,29,30,30,31,31,32,32,33,33,34,34,37,37,38,38,39,39,40,40,41,41,43,43,44,44,45,45,46,46,47,47,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,30,32,34,-1,-1,-1,36,-1,38,40,-1,42,44,46,48,50,52,54,-1,-1,56,58,60,62,64,-1,66,68,70,72,74,76,-1,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6471942E3,3.8652172E1,2.2040408E7,8E0,4.5E1,4.7263342E2,1.2615725E5,1E0,1E0,-3.1156961E-3,-2.1082403E-2,1.8776652E7,1.8111845E-2,7.739144E8,1.5E1,4.556962E0,1E0,4.7238097E0,3.2645503E5,1.3948464E7,-1.0874943E-3,-8.0004735E-3,-3.4213133E-4,5.984843E6,5.293904E-3,5E0,6.929E3,-9.801005E-3,4.643602E8,4.217427E7,5.9089812E7,5.8222644E2,1E0,4.48375E5,2.101E3,2.5967602E-3,-3.4432218E-3,3.09627E5,1.7559428E2,4E0,1.3587301E1,7.09E2,3.6195137E-3,4.1E1,1.24141544E-1,6.2E1,3.3580637E8,2.661509E4,5.467E3,-2.1555722E-2,9.5E2,2.5175922E0,1.0656093E2,-2.8477116E-3,6.046753E-3,-3.3090643E-3,2.8675687E-3,2.9401951E-3,8.780019E-3,-6.671182E-3,-5.9551897E-4,6.472938E-3,2.003495E-4,8.100831E-4,9.481537E-3,-1.7089846E-3,4.2181234E-3,-7.603427E-3,-1.3561976E-3,-4.826796E-3,1.6898211E-3,-1.6855396E-2,-4.4599606E-4,-6.8549816E-3,-1.4088357E-4,4.5694443E-3,9.194059E-5,-5.2138125E-3,2.07401E-3,5.879941E-3,3.3223117E-4,1.165761E-2,5.2255164E-3],"split_indices":[67,68,66,3,3,71,48,23,29,0,0,60,0,7,3,71,26,68,48,9,0,0,0,1,0,3,2,0,7,60,46,67,8,1,0,0,0,9,4,8,71,2,0,3,53,4,7,43,2,0,10,68,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.37E2,8.5E2,8.7E1,8.45E2,5E0,5.7E1,3E1,1.22E2,7.23E2,2E0,3E0,5.3E1,4E0,8E0,2.2E1,9.1E1,3.1E1,2.53E2,4.7E2,4.7E1,6E0,3E0,5E0,1.2E1,1E1,4.6E1,4.5E1,2E0,2.9E1,2.32E2,2.1E1,4.6E2,1E1,3.8E1,9E0,9E0,3E0,2.6E1,2E1,3.3E1,1.2E1,2.1E1,8E0,2.14E2,1.8E1,1.2E1,9E0,3.66E2,9.4E1,3E0,7E0,1.4E1,2.4E1,5E0,4E0,6E0,2E1,9E0,1.1E1,1.1E1,2.2E1,8E0,4E0,2E0,1.9E1,2.03E2,1.1E1,1.3E1,5E0,4E0,8E0,7E0,2E0,1.3E1,3.53E2,7.1E1,2.3E1,3E0,4E0,1E1,4E0,1.7E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"83","size_leaf_vector":"1"}},{"base_weights":[4.9011153E-3,-3.75645E-3,1.2333007E-1,7.9096965E-2,-1.4373613E-2,1.4325188E-1,-5.081706E-2,9.7650945E-2,-4.544747E-2,-2.4197854E-2,6.0731918E-2,1.9062972E-1,7.877329E-2,-4.8804E-3,2.6194504E-3,1.1744487E-1,-2.9250214E-2,-2.3184945E-1,4.652383E-3,-1.956482E-2,-1.5085633E-1,2.770744E-2,1.3187031E-1,1.5306692E-1,1.1822454E-2,-9.168573E-5,1.04375385E-1,7.528581E-4,1.2810044E-1,-3.9370297E-3,1.0425572E-3,-3.5914462E-3,-1.3480548E-2,-4.7973953E-2,5.7741534E-3,-1.5063533E-2,-1.08705945E-1,-4.813277E-2,4.970862E-2,1.6462728E-1,6.8648864E-4,9.424393E-3,3.409586E-3,2.5416093E-3,7.7164196E-3,1.3439107E-1,-6.648526E-5,-3.984835E-2,-1.9916216E-1,1.1484822E-2,-1.8154214E-1,-6.2839077E-3,-1.14095244E-4,2.7778444E-3,-3.859174E-3,-3.8137662E-4,7.4524544E-2,8.80053E-3,2.2351434E-3,5.3756204E-3,1.0260468E-2,-1.5536573E-3,-1.0868131E-2,-1.3758747E-2,-2.1585003E-3,1.7311506E-3,-1.2249132E-3,-2.0803845E-2,-1.7363834E-4,6.2397043E-3,1.0937352E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,-1,33,35,37,39,41,-1,-1,43,-1,45,-1,-1,-1,-1,47,49,-1,51,53,55,57,-1,-1,-1,-1,-1,59,-1,61,63,65,67,-1,-1,-1,-1,-1,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.1811246E-1,7.3565984E-1,2.1890861E-1,2.2396356E-1,5.4822004E-1,1.5300047E-1,4.64999E-2,2.1192074E-1,3.7572977E-1,3.8199776E-1,1.9838667E-1,3.5119653E-2,5.188659E-2,0E0,0E0,7.561064E-2,3.6248036E-2,3.1433523E-2,0E0,4.566671E-1,1.325438E-1,1.0250706E-1,1.011461E-1,7.2128356E-2,0E0,0E0,4.9077332E-2,0E0,5.31559E-2,0E0,0E0,0E0,0E0,3.5938382E-1,3.6150464E-1,0E0,5.579087E-2,5.7576984E-2,6.81531E-2,5.1125944E-2,0E0,0E0,0E0,0E0,0E0,7.503474E-2,0E0,3.480612E-1,2.077806E-1,3.1931096E-1,4.709087E-1,0E0,0E0,0E0,0E0,0E0,9.723441E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,26,26,28,28,33,33,34,34,36,36,37,37,38,38,39,39,45,45,47,47,48,48,49,49,50,50,56,56],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,-1,34,36,38,40,42,-1,-1,44,-1,46,-1,-1,-1,-1,48,50,-1,52,54,56,58,-1,-1,-1,-1,-1,60,-1,62,64,66,68,-1,-1,-1,-1,-1,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.798658E3,7E0,2.4971938E7,1E0,3.2911258E6,1.159499E6,1.3E1,7.234179E6,2.0778275E-1,4.217427E7,3.7748723E8,3.4915986E3,9.522029E6,-4.8804E-3,2.6194504E-3,1.4473684E-1,2.58466E5,9.82E2,4.652383E-3,2.700023E2,2.2953334E2,1.2E1,9.033567E6,1.9376624E0,1.1822454E-2,-9.168573E-5,4.2470217E0,7.528581E-4,2.5503825E6,-3.9370297E-3,1.0425572E-3,-3.5914462E-3,-1.3480548E-2,1.2674157E1,3.3000212E5,-1.5063533E-2,2.3017536E5,3.9152692E6,1.0976E4,9.67E2,6.8648864E-4,9.424393E-3,3.409586E-3,2.5416093E-3,7.7164196E-3,3E1,-6.648526E-5,2.2681375E6,6.34E2,8E0,1E0,-6.2839077E-3,-1.14095244E-4,2.7778444E-3,-3.859174E-3,-3.8137662E-4,9.912121E0,8.80053E-3,2.2351434E-3,5.3756204E-3,1.0260468E-2,-1.5536573E-3,-1.0868131E-2,-1.3758747E-2,-2.1585003E-3,1.7311506E-3,-1.2249132E-3,-2.0803845E-2,-1.7363834E-4,6.2397043E-3,1.0937352E-3],"split_indices":[67,3,60,19,43,9,3,62,53,60,7,4,9,0,0,71,1,2,0,67,48,3,60,69,0,0,69,0,47,0,0,0,0,69,48,0,48,43,12,0,0,0,0,0,0,8,0,43,2,32,8,0,0,0,0,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.94E2,8.34E2,6E1,9.4E1,7.4E2,5.4E1,6E0,8.2E1,1.2E1,6.55E2,8.5E1,3E1,2.4E1,4E0,2E0,7.1E1,1.1E1,5E0,7E0,6.33E2,2.2E1,5.9E1,2.6E1,2.1E1,9E0,6E0,1.8E1,7E0,6.4E1,5E0,6E0,2E0,3E0,2.98E2,3.35E2,3E0,1.9E1,1.3E1,4.6E1,2E1,6E0,1.2E1,9E0,1.1E1,7E0,6.1E1,3E0,2.84E2,1.4E1,3.26E2,9E0,1.5E1,4E0,3E0,1E1,1.4E1,3.2E1,1.6E1,4E0,5.2E1,9E0,2.76E2,8E0,8E0,6E0,1.94E2,1.32E2,3E0,6E0,1.4E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[9.327503E-3,-1.3176566E-2,5.155512E-2,2.2015204E-4,-7.548602E-2,5.7297584E-2,-1.3161676E-1,-3.8495693E-2,2.4961669E-2,-1.290061E-1,-4.611846E-2,1.213808E-2,5.1730428E-2,-1.1716144E-2,-4.4070195E-2,5.061944E-2,-5.4562513E-2,-3.3130616E-2,4.0106885E-2,3.3753037E-3,-1.4259863E-1,-1.2842814E-2,-1.0936167E-1,5.667491E-2,-5.3989002E-3,-4.909055E-3,3.399363E-3,2.1646883E-2,8.890054E-3,-3.6289595E-2,-2.0076008E-1,-4.7676045E-2,6.749718E-3,5.289591E-2,-2.3256829E-2,-1.8411504E-1,-7.4969076E-2,-2.9411871E-2,6.1492147E-3,1.1834646E-2,-1.3419266E-1,4.7681313E-2,1.8480916E-1,-4.1512962E-2,3.2895566E-3,-4.6570092E-2,6.470159E-2,-3.4992778E-1,-7.402357E-2,5.543463E-3,-6.5334365E-2,1.2389907E-1,3.8123436E-2,-1.1884131E-1,1.5752193E-2,-9.499119E-3,2.2188183E-4,2.0971103E-3,-1.22726835E-1,-4.400279E-3,-1.1201693E-3,-3.7673607E-3,4.684695E-3,-9.009671E-3,-5.5288017E-2,1.3897703E-2,7.83402E-2,2.4448654E-1,4.247807E-3,5.0937006E-4,-4.4555995E-3,-1.4264786E-3,-5.495266E-3,-4.4121905E-3,4.6136803E-3,-2.8904447E-2,-7.246833E-3,-5.351275E-3,1.235245E-3,-1.4332774E-3,-6.7614685E-3,3.0934103E-3,7.267076E-3,-1.4105935E-3,2.5675818E-3,-7.180532E-3,-4.2486715E-4,6.677968E-3,-4.3892127E-4,-7.6720472E-3,-3.52156E-4,-2.7964704E-3,2.085311E-3,2.106574E-3,-4.877059E-3,8.690359E-3,1.9788841E-4,4.2914576E-3,1.9737547E-4,5.6055463E-3,1.3863929E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,-1,35,37,39,41,-1,-1,-1,43,-1,45,47,49,-1,51,53,55,57,59,-1,61,63,65,67,69,-1,71,73,75,77,-1,79,81,83,85,87,-1,-1,-1,89,-1,91,-1,-1,-1,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.720982E-1,5.0002724E-1,3.4308392E-1,4.7415793E-1,1.6065633E-1,3.380375E-1,9.42184E-2,2.7909872E-1,2.6703447E-1,1.11729205E-1,1.4590758E-1,0E0,2.5799865E-1,0E0,5.8140457E-2,1.1801048E-1,4.3083328E-1,1.7027202E-1,1.9571668E-1,0E0,8.446783E-2,1.16855584E-1,7.381433E-2,3.311425E-1,0E0,0E0,0E0,8.35411E-2,0E0,1.5563442E-1,3.1438702E-1,1.8164043E-1,0E0,2.05055E-1,1.5474695E-1,8.083737E-2,9.1541536E-2,7.921553E-2,0E0,4.942472E-2,8.3236665E-2,2.8581887E-1,8.365625E-2,3.522991E-2,0E0,1.4888418E-1,8.6487316E-2,3.2487053E-1,4.8462305E-2,0E0,1.4472257E-1,5.2737594E-2,1.9697954E-1,4.539478E-2,9.79606E-2,0E0,0E0,0E0,5.2252084E-2,0E0,8.676685E-2,0E0,0E0,0E0,5.461701E-2,2.2102492E-1,1.4931673E-1,3.7173867E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,27,27,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,50,50,51,51,52,52,53,53,54,54,58,58,60,60,64,64,65,65,66,66,67,67],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,-1,36,38,40,42,-1,-1,-1,44,-1,46,48,50,-1,52,54,56,58,60,-1,62,64,66,68,70,-1,72,74,76,78,-1,80,82,84,86,88,-1,-1,-1,90,-1,92,-1,-1,-1,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.008E3,1.9E1,1.2784314E1,1E0,7.45E2,1.9E2,5E1,1.0952381E0,6.613774E5,1.2463E4,1.2951E4,1.213808E-2,2.871605E2,-1.1716144E-2,1.1941613E3,1.8029197E0,9.540121E9,1.321E3,1E0,3.3753037E-3,2.871287E0,7.0267264E9,5.8899284E1,3.4915986E3,-5.3989002E-3,-4.909055E-3,3.399363E-3,1.1251919E6,8.890054E-3,1.2E2,5.48744E8,1.6E1,6.749718E-3,4.304348E0,6.264151E0,5.208E3,2.9266055E0,3.2E1,6.1492147E-3,9.14E2,5.25E0,1.68991E6,2.2040408E7,2.5493E4,3.2895566E-3,7.70947E5,3.9595376E7,1.8370886E2,1.503E3,5.543463E-3,2.4743947E5,1.32E2,1.0549E4,7.2E1,6.299666E-4,-9.499119E-3,2.2188183E-4,2.0971103E-3,2.4E1,-4.400279E-3,7.883366E3,-3.7673607E-3,4.684695E-3,-9.009671E-3,1.5365228E5,1.9184E4,1.853776E7,3.6E1,4.247807E-3,5.0937006E-4,-4.4555995E-3,-1.4264786E-3,-5.495266E-3,-4.4121905E-3,4.6136803E-3,-2.8904447E-2,-7.246833E-3,-5.351275E-3,1.235245E-3,-1.4332774E-3,-6.7614685E-3,3.0934103E-3,7.267076E-3,-1.4105935E-3,2.5675818E-3,-7.180532E-3,-4.2486715E-4,6.677968E-3,-4.3892127E-4,-7.6720472E-3,-3.52156E-4,-2.7964704E-3,2.085311E-3,2.106574E-3,-4.877059E-3,8.690359E-3,1.9788841E-4,4.2914576E-3,1.9737547E-4,5.6055463E-3,1.3863929E-2],"split_indices":[2,8,68,29,2,48,3,68,60,1,44,0,73,0,4,69,46,10,19,0,68,46,62,4,0,0,0,43,0,0,5,10,0,73,73,10,68,8,0,2,69,43,62,1,0,9,7,4,2,0,43,12,1,0,53,0,0,0,3,0,48,0,0,0,43,1,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.16E2,5.98E2,3.18E2,4.93E2,1.05E2,3.09E2,9E0,1.92E2,3.01E2,3.6E1,6.9E1,7E0,3.02E2,3E0,6E0,2.9E1,1.63E2,6.2E1,2.39E2,2E0,3.4E1,4.6E1,2.3E1,2.94E2,8E0,4E0,2E0,2.5E1,4E0,1.46E2,1.7E1,5.8E1,4E0,1.99E2,4E1,2E1,1.4E1,4.2E1,4E0,4E0,1.9E1,2.76E2,1.8E1,1.1E1,1.4E1,1.33E2,1.3E1,7E0,1E1,5E0,5.3E1,3.3E1,1.66E2,1.1E1,2.9E1,1.8E1,2E0,4E0,1E1,1.2E1,3E1,2E0,2E0,1E1,9E0,1.32E2,1.44E2,1E1,8E0,6E0,5E0,1.1E2,2.3E1,2E0,1.1E1,2E0,5E0,7E0,3E0,3.8E1,1.5E1,1.3E1,2E1,3.3E1,1.33E2,8E0,3E0,4E0,2.5E1,7E0,3E0,1.3E1,1.7E1,3E0,6E0,6E0,1.26E2,1.21E2,2.3E1,4E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[7.906604E-4,-2.5221713E-2,2.7213205E-2,-1.6629841E-2,-8.5221276E-2,-1.2369898E-1,3.541265E-2,-5.2786063E-2,-2.1864547E-4,-2.1074486E-1,-5.991016E-2,-1.2765495E-2,-6.656579E-2,3.0314792E-2,2.2220442E-1,-2.2216778E-2,-1.0914248E-1,3.6448777E-2,-4.3631125E-2,2.994456E-4,-3.8577938E-1,3.7947255E-3,-8.2067706E-2,4.392604E-3,-9.259574E-2,1.3040401E-1,2.2069843E-2,1.5517105E-2,3.4289071E-3,-6.2121306E-2,2.235018E-2,-8.121635E-2,-2.678176E-1,4.969945E-2,-5.0840512E-2,-5.468933E-2,1.5059137E-1,-2.3854062E-2,-6.048711E-3,2.6337502E-3,-9.418887E-2,-5.3183734E-3,5.9100326E-5,1.8216476E-1,3.3269104E-2,2.5289789E-2,-8.504149E-3,-4.379007E-2,-1.0692247E-2,2.983285E-4,4.384831E-3,-1.0347868E-1,7.0701813E-4,-2.0883342E-3,-1.7117068E-2,1.2955513E-3,6.989783E-2,-9.608136E-2,7.073504E-2,4.9284004E-2,-6.681049E-2,1.1433841E-3,1.4436186E-2,-1.0435385E-1,9.693681E-4,-8.049733E-4,2.0529002E-1,-2.1965092E-3,4.535177E-3,-2.6864126E-2,3.672218E-2,-1.3398473E-3,-7.97211E-3,-5.493331E-3,3.3725905E-3,7.121014E-4,-7.999296E-3,3.952312E-3,-2.2105821E-3,-5.400502E-3,1.0596201E-3,-7.0854806E-4,7.524161E-3,7.6869717E-3,-9.597583E-4,-2.5662764E-3,-8.097467E-3,-3.772036E-3,-8.793675E-3,1.1435233E-2,3.8662856E-3,-1.2081341E-2,3.962311E-4,2.7747098E-3,3.9079186E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,-1,37,-1,39,-1,41,43,45,-1,-1,47,49,51,53,55,57,59,61,-1,-1,-1,63,-1,-1,65,67,69,-1,71,-1,-1,-1,73,-1,-1,-1,75,77,79,81,83,85,-1,-1,87,-1,-1,89,-1,-1,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1239505E-1,2.300461E-1,5.513096E-1,2.3377499E-1,1.6529423E-1,1.7986739E-1,3.932039E-1,2.0947662E-1,4.3456638E-1,3.446154E-1,1.5719433E-1,0E0,8.418743E-2,3.354665E-1,1.5741324E-1,1.4544073E-1,1.6878963E-1,1.7370975E-1,2.767048E-1,0E0,9.583795E-2,0E0,7.594374E-2,0E0,3.2824755E-2,1.498006E-1,2.5536638E-1,0E0,0E0,1.221856E-1,4.409994E-2,8.3140165E-2,1.1753175E-1,1.260302E-1,1.16799936E-1,1.5326574E-1,1.333316E-1,0E0,0E0,0E0,4.7777772E-2,0E0,0E0,9.581065E-2,6.6665456E-2,2.2457801E-1,0E0,7.1538985E-2,0E0,0E0,0E0,8.842966E-2,0E0,0E0,0E0,9.736076E-2,1.652312E-1,3.837575E-2,4.9429514E-2,1.0885145E-1,1.2392661E-1,0E0,0E0,5.5574596E-2,0E0,0E0,6.88079E-2,0E0,0E0,5.6261176E-1,1.995838E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,22,22,24,24,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,40,40,43,43,44,44,45,45,47,47,51,51,55,55,56,56,57,57,58,58,59,59,60,60,63,63,66,66,69,69,70,70],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,-1,38,-1,40,-1,42,44,46,-1,-1,48,50,52,54,56,58,60,62,-1,-1,-1,64,-1,-1,66,68,70,-1,72,-1,-1,-1,74,-1,-1,-1,76,78,80,82,84,86,-1,-1,88,-1,-1,90,-1,-1,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,3.5691056E0,1E0,1.463E2,1.21E2,1.0479E4,4.7263342E2,4.8421054E0,1E0,8E0,3.0620155E0,-1.2765495E-2,7.080933E2,5.26E2,4.8647133E-1,2.2953334E2,5.369854E5,1.0839028E1,3.5E1,2.994456E-4,1.5470404E2,3.7947255E-3,2.491E3,4.392604E-3,1.24141544E-1,1.8703672E6,7.134432E7,1.5517105E-2,3.4289071E-3,3.09627E5,1.5272727E0,1.295313E6,1.1626786E2,3.3852024E7,1.4553613E1,1.512E4,1.6100003E8,-2.3854062E-2,-6.048711E-3,2.6337502E-3,5.1042255E2,-5.3183734E-3,5.9100326E-5,4E0,5.6440186E2,5.9754E4,-8.504149E-3,1.0645059E2,-1.0692247E-2,2.983285E-4,4.384831E-3,3.3706608E0,7.0701813E-4,-2.0883342E-3,-1.7117068E-2,2.7142856E0,1E0,8.57E2,1.7145834E1,1.1948844E4,3.94E2,1.1433841E-3,1.4436186E-2,2.88864E2,9.693681E-4,-8.049733E-4,7.884708E0,-2.1965092E-3,4.535177E-3,4E0,2E0,-1.3398473E-3,-7.97211E-3,-5.493331E-3,3.3725905E-3,7.121014E-4,-7.999296E-3,3.952312E-3,-2.2105821E-3,-5.400502E-3,1.0596201E-3,-7.0854806E-4,7.524161E-3,7.6869717E-3,-9.597583E-4,-2.5662764E-3,-8.097467E-3,-3.772036E-3,-8.793675E-3,1.1435233E-2,3.8662856E-3,-1.2081341E-2,3.962311E-4,2.7747098E-3,3.9079186E-4],"split_indices":[2,68,17,70,48,9,71,73,26,0,73,0,4,44,57,48,43,71,3,0,67,0,62,0,53,62,59,0,0,9,71,9,4,7,71,9,5,0,0,0,4,0,0,3,4,1,0,67,0,0,0,68,0,0,0,68,19,2,73,48,0,0,0,67,0,0,71,0,0,3,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.89E2,4.48E2,4.41E2,3.93E2,5.5E1,2.2E1,4.19E2,1.22E2,2.71E2,8E0,4.7E1,5E0,1.7E1,4.09E2,1E1,8E1,4.2E1,1.47E2,1.24E2,4E0,4E0,6E0,4.1E1,2E0,1.5E1,3E1,3.79E2,5E0,5E0,4.2E1,3.8E1,3.7E1,5E0,1.28E2,1.9E1,1.18E2,6E0,2E0,2E0,3E0,3.8E1,1.2E1,3E0,1.9E1,1.1E1,3.74E2,5E0,3.9E1,3E0,3.2E1,6E0,3E1,7E0,2E0,3E0,3.8E1,9E1,1.4E1,5E0,1.2E1,1.06E2,4E0,2E0,3.5E1,3E0,2E0,1.7E1,5E0,6E0,6.7E1,3.07E2,3.6E1,3E0,2.8E1,2E0,3.6E1,2E0,8E1,1E1,1.2E1,2E0,3E0,2E0,4E0,8E0,9.7E1,9E0,2.9E1,6E0,1.2E1,5E0,8E0,5.9E1,1.69E2,1.38E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-4.512751E-3,-1.6866151E-2,8.0324255E-2,-4.3726347E-2,6.0464484E-3,9.5224336E-2,-3.7163798E-2,-3.2732993E-2,-1.251827E-1,1.4953628E-1,1.9997868E-3,7.5883016E-2,1.9806775E-1,-3.4312953E-3,5.6593237E-3,-7.6742984E-2,-1.24256285E-2,3.1236278E-2,-1.6081016E-1,2.9397022E-3,1.04794875E-2,-1.4392828E-2,4.504818E-2,-1.4674308E-3,8.5324E-2,1.1884965E-2,5.0527165E-3,-4.4832937E-2,-1.3331434E-1,4.7657993E-2,-2.6323846E-2,-3.3194819E-3,4.392073E-3,-3.8190112E-1,-1.1283093E-1,-6.4088823E-3,-1.1373994E-1,6.811465E-2,-2.4883954E-2,7.266907E-3,6.738026E-2,-7.667287E-2,2.5226597E-2,-1.4232732E-2,-1.09073795E-1,4.7922204E-3,3.6600358E-3,-3.0047184E-2,9.7468E-3,-6.5480233E-3,-2.1730924E-2,-6.379645E-2,-1.2642403E-2,-2.699252E-2,3.9175037E-2,-1.5683536E-1,7.168957E-3,8.3427474E-2,-2.7823951E-2,1.1420727E-2,-8.863029E-2,9.5731795E-2,2.8277252E-2,-6.295825E-4,-4.7924886E-3,3.7897422E-3,-2.3455636E-3,2.3137636E-3,-5.86222E-3,1.3032363E-3,-4.6224957E-3,-9.169566E-4,-6.0350834E-3,-4.0609245E-3,2.2207922E-3,-9.056937E-4,-2.3798108E-2,7.870484E-4,4.9849045E-3,-2.3771701E-3,-1.2528356E-2,5.53496E-3,-2.8542008E-3,2.907247E-3,7.706945E-3,3.7392871E-3,-2.5491072E-3,3.6900886E-3,-1.8823157E-3,-7.351965E-3,-2.0499762E-4,3.1867903E-3,1.2420047E-2,-8.9633244E-4,4.2759366E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,37,-1,39,-1,-1,41,43,45,47,-1,-1,49,51,53,55,57,59,-1,61,63,65,-1,67,-1,69,71,-1,-1,-1,73,-1,75,77,79,81,83,85,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[9.728744E-1,4.9876833E-1,2.1010756E-1,3.2991582E-1,2.5402775E-1,1.9382846E-1,9.0407014E-2,2.939763E-1,2.5022668E-1,6.427243E-2,3.0197188E-1,9.397882E-2,5.4480553E-2,0E0,0E0,1.8176335E-1,1.9081207E-1,6.5107256E-2,3.3878952E-1,0E0,0E0,2.4515744E-1,1.9203931E-1,0E0,9.510714E-2,0E0,0E0,1.5362118E-1,1.2152922E-1,1.0440403E-1,1.6878809E-1,0E0,0E0,5.7689965E-2,2.2435555E-1,2.712829E-1,1.2096861E-1,1.3307035E-1,7.044093E-2,0E0,7.267985E-2,7.6868415E-2,9.775628E-2,0E0,9.7935796E-2,0E0,6.554918E-2,1.8596461E-1,0E0,0E0,0E0,6.976242E-2,0E0,7.1524256E-1,1.3438804E-1,1.8850955E-1,6.096361E-2,1.2216711E-1,4.237525E-2,7.370739E-2,6.364564E-2,1.6350552E-1,9.2587955E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,21,21,22,22,24,24,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,44,44,46,46,47,47,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,38,-1,40,-1,-1,42,44,46,48,-1,-1,50,52,54,56,58,60,-1,62,64,66,-1,68,-1,70,72,-1,-1,-1,74,-1,76,78,80,82,84,86,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,8.01E2,6.747343E8,6.364486E0,2.04115E5,4.620305E3,3.6966505E0,2.71E2,8E0,1.5E1,8.328548E6,1.5203704E1,1E0,-3.4312953E-3,5.6593237E-3,1.4452E4,2.907685E6,3.23E2,5.764631E1,2.9397022E-3,1.04794875E-2,1.1048219E3,2.3821254E7,-1.4674308E-3,8.237624E0,1.1884965E-2,5.0527165E-3,4.54E2,1.0952381E0,2.992648E-2,1.9719212E1,-3.3194819E-3,4.392073E-3,1.654E3,8.198588E-1,1.7831801E6,1.3457517E10,1.597E4,2.4E0,7.266907E-3,1.8048353E0,5.3E1,7.392347E1,-1.4232732E-2,2.3857143E1,4.7922204E-3,1.3E1,9.1470585E0,9.7468E-3,-6.5480233E-3,-2.1730924E-2,4.676173E6,-1.2642403E-2,1.7587205E6,2.5695364E0,7.472532E9,3.01E2,6.965855E9,3.01E2,1.3543621E6,5.11E2,2.6055083E5,1.4176E4,-6.295825E-4,-4.7924886E-3,3.7897422E-3,-2.3455636E-3,2.3137636E-3,-5.86222E-3,1.3032363E-3,-4.6224957E-3,-9.169566E-4,-6.0350834E-3,-4.0609245E-3,2.2207922E-3,-9.056937E-4,-2.3798108E-2,7.870484E-4,4.9849045E-3,-2.3771701E-3,-1.2528356E-2,5.53496E-3,-2.8542008E-3,2.907247E-3,7.706945E-3,3.7392871E-3,-2.5491072E-3,3.6900886E-3,-1.8823157E-3,-7.351965E-3,-2.0499762E-4,3.1867903E-3,1.2420047E-2,-8.9633244E-4,4.2759366E-3],"split_indices":[2,2,47,69,5,67,50,2,0,0,9,73,19,0,0,9,5,1,62,0,0,67,9,0,71,0,0,1,68,53,71,0,0,1,57,43,46,44,68,0,53,44,4,0,4,0,73,71,0,0,0,60,0,43,68,46,0,46,0,43,0,48,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,8.09E2,1.17E2,3.72E2,4.37E2,1.04E2,1.3E1,3.29E2,4.3E1,1.1E1,4.26E2,8.9E1,1.5E1,1.1E1,2E0,1.03E2,2.26E2,8E0,3.5E1,6E0,5E0,3.09E2,1.17E2,7E0,8.2E1,8E0,7E0,6.7E1,3.6E1,4.2E1,1.84E2,3E0,5E0,5E0,3E1,2.87E2,2.2E1,8.8E1,2.9E1,1.5E1,6.7E1,4.6E1,2.1E1,3E0,3.3E1,1.8E1,2.4E1,1.82E2,2E0,2E0,3E0,2.4E1,6E0,1.98E2,8.9E1,1.6E1,6E0,7.6E1,1.2E1,1.9E1,1E1,3.8E1,2.9E1,1.4E1,3.2E1,1.2E1,9E0,3E0,3E1,2E1,4E0,1.66E2,1.6E1,2E1,4E0,1.96E2,2E0,6.8E1,2.1E1,9E0,7E0,2E0,4E0,6.2E1,1.4E1,2E0,1E1,8E0,1.1E1,5E0,5E0,3.4E1,4E0,1.7E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[2.258123E-3,-6.330367E-3,9.794487E-2,-2.7574603E-2,2.0482576E-2,1.6696505E-1,4.1092988E-2,-7.848528E-2,-1.4060651E-2,-2.3753294E-3,4.5524616E-2,1.0255414E-1,1.1241591E-2,5.7801224E-2,-4.5321565E-3,-4.2541187E-2,-1.6377996E-1,6.342825E-2,-2.5357623E-2,6.163251E-3,-1.4731526E-1,5.343416E-2,-9.349011E-2,-1.2002374E-3,1.470618E-1,-3.9315457E-3,7.201084E-2,-1.779504E-2,-1.5157823E-1,-2.3370977E-1,-9.235512E-2,-5.815693E-2,1.4509292E-1,-2.0139245E-2,-1.3290516E-1,-3.7434563E-2,3.2567367E-2,5.1393115E-4,-2.6663136E-2,6.659765E-2,-3.3593275E-2,-1.4883762E-2,1.4641297E-4,2.8810897E-3,9.158298E-3,4.9543507E-2,7.3487614E-3,-4.0919334E-2,4.690896E-2,-2.9092499E-3,-9.273559E-3,-3.8944892E-4,-1.2583458E-2,1.4128344E-3,-7.0182187E-3,-6.899732E-3,-8.902743E-3,8.745227E-3,5.7723477E-2,-7.6613046E-2,-9.153249E-3,-4.0520965E-3,-1.3346396E-2,3.966572E-4,-1.6750416E-1,1.6521002E-1,1.7243978E-2,9.186525E-2,6.3977356E-4,2.7572407E-4,-5.1645506E-3,3.7125822E-3,1.7676706E-4,3.871272E-4,-3.2255629E-3,3.130569E-3,-2.9740043E-3,-7.4940273E-3,1.3904544E-3,5.008053E-3,-5.1807025E-3,1.0901823E-3,-4.724621E-3,3.826533E-4,-2.808193E-3,-5.71297E-3,6.923703E-4,-1.7589357E-4,-1.3692917E-2,9.255073E-3,2.2266381E-3,1.2171483E-3,-7.827943E-3,5.503753E-3,1.0761621E-3,-2.5394633E-3,3.2735157E-3,-1.4925378E-3,3.9241975E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,-1,27,29,31,33,35,37,39,41,-1,43,-1,45,47,49,51,53,55,57,59,61,63,65,-1,-1,67,69,-1,-1,-1,-1,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,77,-1,79,81,83,-1,-1,85,87,89,91,93,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.5832397E-1,4.8303336E-1,2.913574E-1,3.2432255E-1,2.1476136E-1,1.389119E-1,1.0516941E-1,2.965507E-1,3.298796E-1,2.4471146E-1,2.0181373E-1,1.18056655E-1,0E0,8.496697E-2,0E0,1.8887821E-1,1.2326276E-1,4.8504037E-1,1.8140474E-1,2.1648636E-1,7.4842346E-1,1.9698665E-1,2.1993053E-1,0E0,5.1455438E-2,0E0,6.1731443E-2,9.019731E-2,4.2870104E-2,1.1401081E-1,1.19560674E-1,8.753401E-2,9.795815E-2,1.9411096E-1,8.0789E-2,3.4943557E-1,2.3495772E-1,0E0,0E0,2.4659765E-1,6.036646E-2,0E0,0E0,0E0,0E0,4.1254677E-2,0E0,6.2190406E-2,4.049506E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.751069E-2,0E0,1.0493414E-1,1.2982339E-1,2.3556168E-1,0E0,0E0,1.0183169E-1,3.313582E-1,3.8050592E-2,1.7999263E-1,1.9195521E-1,1.656706E-1,4.898556E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,45,45,47,47,48,48,56,56,58,58,59,59,60,60,63,63,64,64,65,65,66,66,67,67,68,68,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,-1,28,30,32,34,36,38,40,42,-1,44,-1,46,48,50,52,54,56,58,60,62,64,66,-1,-1,68,70,-1,-1,-1,-1,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,78,-1,80,82,84,-1,-1,86,88,90,92,94,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6978182E3,7.785302E5,1.4595818E7,3.3820656E7,4.5432812E2,1.7E1,4.0280737E2,6E0,3.261058E0,2.511E3,1.9269184E7,1E0,1.1241591E-2,3.3580637E8,-4.5321565E-3,1E0,2.2953334E2,2.08E2,6.550562E0,4.385E4,1.8321838E1,5.2409735E-7,5.36763E5,-1.2002374E-3,4.900919E8,-3.9315457E-3,6.7473444E2,7.1428573E-1,1.4E1,2.2227E4,3.435414E-3,3.07E2,2.6003375E5,1.5024445E2,2.5711747E5,1.22896E5,9.2310345E2,5.1393115E-4,-2.6663136E-2,5.1291872E8,1.72797E5,-1.4883762E-2,1.4641297E-4,2.8810897E-3,9.158298E-3,8.318076E6,7.3487614E-3,3.959E3,1.1567898E3,-2.9092499E-3,-9.273559E-3,-3.8944892E-4,-1.2583458E-2,1.4128344E-3,-7.0182187E-3,-6.899732E-3,1.3E1,8.745227E-3,1E0,3.8488917E-2,3.84E2,-4.0520965E-3,-1.3346396E-2,2.0936034E6,8.4E1,4.76E2,2E0,2E0,3.1075312E10,5.513889E0,-5.1645506E-3,3.7125822E-3,1.7676706E-4,3.871272E-4,-3.2255629E-3,3.130569E-3,-2.9740043E-3,-7.4940273E-3,1.3904544E-3,5.008053E-3,-5.1807025E-3,1.0901823E-3,-4.724621E-3,3.826533E-4,-2.808193E-3,-5.71297E-3,6.923703E-4,-1.7589357E-4,-1.3692917E-2,9.255073E-3,2.2266381E-3,1.2171483E-3,-7.827943E-3,5.503753E-3,1.0761621E-3,-2.5394633E-3,3.2735157E-3,-1.4925378E-3,3.9241975E-3],"split_indices":[67,43,62,7,4,3,73,8,73,2,62,29,0,7,0,6,48,2,68,12,73,52,1,0,7,0,71,71,3,1,53,1,43,70,43,9,48,0,0,7,1,0,0,0,0,60,0,9,48,0,0,0,0,0,0,0,0,0,26,53,0,0,0,60,10,0,6,32,46,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.21E2,8.46E2,7.5E1,4.72E2,3.74E2,3.3E1,4.2E1,9.8E1,3.74E2,1.96E2,1.78E2,1.9E1,1.4E1,3.8E1,4E0,7E1,2.8E1,4.7E1,3.27E2,1.86E2,1E1,1.69E2,9E0,5E0,1.4E1,3E0,3.5E1,5.8E1,1.2E1,1.3E1,1.5E1,1.9E1,2.8E1,3.13E2,1.4E1,7E1,1.16E2,8E0,2E0,1.47E2,2.2E1,2E0,7E0,6E0,8E0,2.9E1,6E0,4.3E1,1.5E1,5E0,7E0,2E0,1.1E1,5E0,1E1,6E0,1.3E1,1.8E1,1E1,5E1,2.63E2,1.2E1,2E0,5.5E1,1.5E1,1.1E1,1.05E2,1.06E2,4.1E1,1.6E1,6E0,1.7E1,1.2E1,1.6E1,2.7E1,1.3E1,2E0,2E0,1.1E1,8E0,2E0,1E1,4E1,1.97E2,6.6E1,5E0,5E1,7E0,8E0,8E0,3E0,1.01E2,4E0,7.6E1,3E1,2.3E1,1.8E1,1.2E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[-7.1923276E-3,-2.6239976E-2,2.043576E-2,-2.3674464E-2,-2.9858032E-1,3.460417E-2,-7.947937E-2,-2.6717106E-2,9.276762E-2,-2.068503E-2,-2.458715E-3,7.521019E-3,2.6752474E-2,-1.2731743E-1,-3.228918E-2,-1.2553272E-2,-5.0260097E-2,2.4363145E-2,9.814169E-3,1.3317167E-2,8.5360244E-2,-2.1266951E-3,-1.6778329E-1,-1.0746898E-2,-8.6822035E-3,-1.6923755E-2,1.0503907E-1,-3.8170174E-2,-1.5034324E-1,-8.200533E-4,5.9637115E-3,2.2854121E-2,-1.2154454E-1,1.0077361E-2,6.0015097E-2,-5.319734E-3,-1.1706167E-2,-2.2675998E-3,3.8647885E-3,-8.803873E-3,-7.354283E-2,9.415244E-3,1.4431411E-3,-1.8204911E-1,-2.943077E-2,-1.0444923E-2,-1.6264915E-3,-3.263611E-2,4.12239E-2,-1.6954977E-2,-4.3428723E-2,7.910894E-2,1.4496379E-2,-1.415952E-3,9.139161E-4,-5.8198995E-3,-7.297031E-4,-2.5011228E-2,2.1968987E-3,-7.07736E-3,-9.265256E-4,4.077979E-3,-3.3834137E-3,-1.9627155E-3,7.589309E-3,2.3907023E-3,-4.910704E-3,-4.855933E-3,7.1435346E-4,1.7556739E-3,6.4389957E-3,-8.5339806E-4,4.205568E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,29,-1,31,33,-1,35,37,-1,39,41,43,45,-1,-1,47,49,-1,51,-1,-1,-1,-1,53,55,-1,-1,57,59,-1,61,63,65,-1,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8468632E-1,3.7371856E-1,5.3503036E-1,1.9415021E-1,1.5608647E-1,3.2502803E-1,1.0199091E-1,1.7543545E-1,1.081076E-1,0E0,0E0,0E0,2.4453624E-1,6.826419E-2,8.114395E-2,1.7189027E-1,2.3384082E-1,4.5881208E-2,0E0,3.3075112E-1,1.800414E-1,0E0,3.8428664E-2,8.609733E-2,0E0,1.4646941E-1,8.0225214E-2,2.1846229E-1,2.314969E-1,0E0,0E0,2.449684E-1,3.0688944E-1,0E0,4.2309538E-2,0E0,0E0,0E0,0E0,1.7364122E-1,1.1743541E-1,0E0,0E0,8.410658E-1,1.9421774E-1,0E0,5.719552E-2,1.1927929E-1,2.744663E-1,0E0,5.179523E-2,8.0673486E-2,4.1563943E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,22,22,23,23,25,25,26,26,27,27,28,28,31,31,32,32,34,34,39,39,40,40,43,43,44,44,46,46,47,47,48,48,50,50,51,51,52,52],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,30,-1,32,34,-1,36,38,-1,40,42,44,46,-1,-1,48,50,-1,52,-1,-1,-1,-1,54,56,-1,-1,58,60,-1,62,64,66,-1,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,1.0910448E1,3.2856784E7,2.2503355E1,1.6E1,3.63E2,1.7500046E8,3.4285715E0,1.3748E4,-2.068503E-2,-2.458715E-3,7.521019E-3,1.76812E3,1.7E1,4.533E3,2.89196E6,3.0977E4,4.3024124E2,9.814169E-3,1.9269184E7,1.0918E4,-2.1266951E-3,7E0,3.4E1,-8.6822035E-3,1E0,1.925508E6,4E0,2.0467092E6,-8.200533E-4,5.9637115E-3,1E0,2.1253996E7,1.0077361E-2,1.0905187E9,-5.319734E-3,-1.1706167E-2,-2.2675998E-3,3.8647885E-3,5.47E2,1.295313E6,9.415244E-3,1.4431411E-3,1.7831801E6,5.862757E1,-1.0444923E-2,1.357E3,6.1E1,1.6254545E1,-1.6954977E-2,1.4320667E3,6.1236826E8,4.95252E5,-1.415952E-3,9.139161E-4,-5.8198995E-3,-7.297031E-4,-2.5011228E-2,2.1968987E-3,-7.07736E-3,-9.265256E-4,4.077979E-3,-3.3834137E-3,-1.9627155E-3,7.589309E-3,2.3907023E-3,-4.910704E-3,-4.855933E-3,7.1435346E-4,1.7556739E-3,6.4389957E-3,-8.5339806E-4,4.205568E-3],"split_indices":[4,68,60,71,0,44,12,71,44,0,0,0,67,3,8,62,44,4,0,62,9,0,8,3,0,30,43,3,43,0,0,29,62,0,7,0,0,0,0,2,9,0,0,43,62,0,2,3,69,0,4,7,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.19E2,5.44E2,3.75E2,5.4E2,4E0,3.29E2,4.6E1,5.27E2,1.3E1,2E0,2E0,1.8E1,3.11E2,2.2E1,2.4E1,3.3E2,1.97E2,9E0,4E0,2.54E2,5.7E1,8E0,1.4E1,2.2E1,2E0,3.19E2,1.1E1,1.77E2,2E1,7E0,2E0,2.38E2,1.6E1,8E0,4.9E1,1E1,4E0,1.6E1,6E0,2.8E2,3.9E1,4E0,7E0,9E0,1.68E2,1.3E1,7E0,5.9E1,1.79E2,3E0,1.3E1,3.4E1,1.5E1,1.59E2,1.21E2,2E1,1.9E1,3E0,6E0,1.1E1,1.57E2,3E0,4E0,5.7E1,2E0,1.68E2,1.1E1,6E0,7E0,2.1E1,1.3E1,1.1E1,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-3.0187105E-3,3.6558185E-3,-1.10867694E-1,-1.7483458E-2,3.2039523E-2,-2.4927141E-1,-6.538875E-2,-1.4614189E-2,-2.5176242E-1,4.554541E-2,-6.741552E-2,-2.9273736E-1,-3.1651978E-4,4.153245E-3,-1.1028342E-1,1.5157058E-2,-3.5292417E-2,1.9260602E-4,-2.3674525E-2,3.254275E-2,8.840239E-2,-5.7719885E-3,-3.7808705E-2,-5.529577E-4,-1.6455632E-2,-1.0608669E-2,-5.6595415E-2,-5.0564613E-2,4.036637E-2,-8.336635E-2,-1.8961672E-2,4.3554768E-2,-7.994991E-2,1.4276963E-1,4.335847E-2,4.348565E-3,-6.0725234E-2,2.012839E-3,-8.5156254E-2,-6.897473E-2,4.2224135E-3,4.4945084E-2,-9.111756E-3,-1.0696685E-1,-1.4296642E-2,-2.3556845E-2,1.093239E-2,1.332027E-1,3.44691E-2,-1.7829353E-2,-3.279783E-2,1.6029033E-1,-4.2651077E-3,2.7447995E-2,1.1457086E-2,-4.274493E-2,-8.851291E-3,-1.0612741E-3,-6.4635063E-3,-2.2476902E-3,-1.7821137E-2,2.35937E-3,-5.9697344E-3,-1.7278233E-3,-8.0440445E-3,-3.9641866E-3,2.394893E-3,4.3629613E-3,-1.4935147E-3,7.1814377E-3,2.069377E-4,3.453443E-3,2.0974144E-4,1.4089001E-3,-4.8939097E-3,3.948772E-3,9.103124E-3,8.642574E-5,3.565586E-3,-3.1974863E-3,2.3751075E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,25,27,29,-1,-1,31,33,-1,35,-1,-1,-1,37,39,41,43,45,47,49,51,53,-1,55,-1,57,59,-1,61,-1,63,65,67,-1,69,71,-1,73,75,-1,77,-1,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.669783E-1,5.243697E-1,3.2360107E-1,3.3309752E-1,5.040947E-1,1.3501465E-1,3.0153084E-1,3.0588073E-1,3.9714837E-1,1.8045998E-1,7.0840135E-2,1.8451864E-1,0E0,0E0,1.9583747E-1,3.400279E-1,2.284486E-1,0E0,0E0,3.181301E-1,1.7992717E-1,0E0,9.896292E-2,0E0,0E0,0E0,7.2146036E-2,1.545963E-1,1.6892493E-1,1.1909664E-1,2.6234454E-1,1.8437523E-1,3.1647894E-1,1.5437901E-1,1.3211417E-1,0E0,5.47786E-2,0E0,6.0574546E-2,3.1231707E-1,0E0,1.5290755E-1,0E0,2.4698555E-1,9.773506E-2,2.2093864E-1,0E0,5.666536E-2,2.5236297E-1,0E0,9.915286E-2,6.888229E-2,0E0,5.1496744E-2,0E0,6.4506784E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,14,14,15,15,16,16,19,19,20,20,22,22,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,38,38,39,39,41,41,43,43,44,44,45,45,47,47,48,48,50,50,51,51,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,26,28,30,-1,-1,32,34,-1,36,-1,-1,-1,38,40,42,44,46,48,50,52,54,-1,56,-1,58,60,-1,62,-1,64,66,68,-1,70,72,-1,74,76,-1,78,-1,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.05039425E1,3.8903564E2,2.0966116E2,4.75356E6,3.2856784E7,6.0078123E-6,8.589474E0,2.0131578E0,2.4203823E0,6.64E2,1.59E2,8E0,-3.1651978E-4,4.153245E-3,7.714286E0,1.1742308E2,2.375E0,1.9260602E-4,-2.3674525E-2,2.0295714E7,5.5E1,-5.7719885E-3,2.33E2,-5.529577E-4,-1.6455632E-2,-1.0608669E-2,1.2183674E1,1.4702564E11,1.889865E1,1.392E1,4E1,3.45E2,2.1253996E7,2.1E1,1.0534078E4,4.348565E-3,3.8E1,2.012839E-3,7.44488E5,4.4380944E8,4.2224135E-3,2E0,-9.111756E-3,1.2916E4,2.3043478E2,6E0,1.093239E-2,2.1609572E7,2.6947792E0,-1.7829353E-2,4.88E2,5.3892307E1,-4.2651077E-3,2.0094643E0,1.1457086E-2,2.3043478E0,-8.851291E-3,-1.0612741E-3,-6.4635063E-3,-2.2476902E-3,-1.7821137E-2,2.35937E-3,-5.9697344E-3,-1.7278233E-3,-8.0440445E-3,-3.9641866E-3,2.394893E-3,4.3629613E-3,-1.4935147E-3,7.1814377E-3,2.069377E-4,3.453443E-3,2.0974144E-4,1.4089001E-3,-4.8939097E-3,3.948772E-3,9.103124E-3,8.642574E-5,3.565586E-3,-3.1974863E-3,2.3751075E-3],"split_indices":[69,67,48,62,60,52,73,68,69,0,0,0,0,0,71,67,68,0,0,62,8,0,0,0,0,0,71,46,71,73,3,44,62,3,4,0,3,0,9,5,0,6,0,1,4,3,0,60,69,0,0,73,0,57,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.25E2,8.72E2,5.3E1,5E2,3.72E2,1.2E1,4.1E1,4.95E2,5E0,3.28E2,4.4E1,1E1,2E0,9E0,3.2E1,2.03E2,2.92E2,3E0,2E0,2.53E2,7.5E1,1.4E1,3E1,2E0,8E0,9E0,2.3E1,5.6E1,1.47E2,7.3E1,2.19E2,2.31E2,2.2E1,3.3E1,4.2E1,4E0,2.6E1,5E0,1.8E1,5E1,6E0,1.45E2,2E0,5.4E1,1.9E1,2.16E2,3E0,2E1,2.11E2,2E0,2E1,3.1E1,2E0,4E1,2E0,2.4E1,2E0,9E0,9E0,4.8E1,2E0,1.41E2,4E0,2.7E1,2.7E1,9E0,1E1,1.4E1,2.02E2,1.7E1,3E0,9E1,1.21E2,1.1E1,9E0,1.1E1,2E1,2.7E1,1.3E1,1.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-5.8955806E-3,-1.6694933E-2,4.814049E-2,-6.0973674E-2,-5.5236253E-3,2.698165E-2,1.0190651E-1,-5.0282385E-2,-2.0524163E-2,2.0394023E-3,-7.490731E-2,8.6491285E-3,8.517204E-2,7.022029E-3,2.0848075E-2,-5.7587415E-2,4.8874132E-3,1.962381E-2,-1.5879435E-2,-1.4501616E-2,-3.701992E-2,-4.5267716E-3,5.532135E-3,1.1857857E-1,-2.2393247E-4,-5.3436784E-3,2.1676081E-3,-3.235912E-2,-1.0062948E-1,5.8063146E-2,-8.964962E-3,-3.935182E-2,3.812896E-2,-5.063615E-2,9.406493E-3,2.4704969E-2,-4.7756877E-2,6.5204306E-3,1.1358477E-4,-1.7098788E-2,-4.78206E-3,-1.9000043E-1,-2.8757436E-2,-1.1772084E-2,7.716807E-2,7.303611E-2,-4.026133E-2,-2.6696177E-2,-1.9473656E-1,8.119652E-2,-3.170149E-2,4.5328483E-2,-8.213363E-2,-1.6144059E-3,5.755151E-2,-1.5249467E-2,-1.7130582E-2,-1.989024E-3,2.6468704E-3,-1.2535569E-2,-6.0278955E-3,-5.3258E-4,-6.4358744E-3,1.3872898E-3,-2.0588164E-3,4.0710927E-3,-2.847884E-3,1.7960409E-4,6.605697E-3,9.125305E-3,-2.2485156E-3,-2.6149435E-3,1.5739279E-3,-1.9106813E-2,-4.5788083E-3,7.964844E-3,1.976115E-3,5.483936E-3,-3.2910407E-3,-1.981705E-3,4.9119703E-3,-6.2652295E-3,-1.7565843E-3,3.7371148E-3,-7.070262E-4,5.966806E-4,-5.2611437E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,-1,25,27,-1,29,31,-1,33,35,-1,37,-1,-1,-1,39,41,43,45,47,49,51,-1,53,55,-1,-1,57,-1,59,61,63,65,67,69,71,73,75,77,79,81,-1,83,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.254193E-1,3.7065175E-1,1.678944E-1,5.9181595E-1,3.1500322E-1,1.1554199E-1,1.6394371E-1,1.7403018E-1,0E0,1.7109157E-1,5.180229E-1,1.2261818E-1,7.8730926E-2,0E0,6.480057E-2,1.5093842E-1,0E0,3.0162826E-1,3.4251335E-1,0E0,1.7966028E-1,9.7220376E-2,0E0,4.7370493E-2,0E0,0E0,0E0,9.60947E-2,3.2854617E-1,1.5700942E-1,4.0842158E-1,3.6450216E-1,2.4918303E-1,1.5485518E-1,0E0,9.177443E-2,2.6034766E-1,0E0,0E0,1.468186E-1,0E0,7.553941E-2,5.5330783E-2,3.646972E-2,1.3737738E-1,2.0946081E-1,2.2937886E-1,3.1475115E-1,2.50843E-1,1.7050877E-1,1.953624E-1,7.342199E-2,8.3553344E-2,0E0,5.099284E-2,0E0,8.4855594E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,20,20,21,21,23,23,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,36,36,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,56,56],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,-1,26,28,-1,30,32,-1,34,36,-1,38,-1,-1,-1,40,42,44,46,48,50,52,-1,54,56,-1,-1,58,-1,60,62,64,66,68,70,72,74,76,78,80,82,-1,84,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9652428E6,1.7667647E2,1.6326721E0,2.4240502E8,4.909091E0,2.5555556E0,1.508924E6,3.907812E7,-2.0524163E-2,2.335307E9,1.9874651E-5,7.098E3,5.8475E5,7.022029E-3,1E0,3.53969E5,4.8874132E-3,1E0,3.361317E1,-1.4501616E-2,2.1749277E0,2.8302418E10,5.532135E-3,1.7E1,-2.2393247E-4,-5.3436784E-3,2.1676081E-3,6.9E1,1.295313E6,5.14E2,1.9010988E0,1.9677316E0,2.707256E7,1.0647887E1,9.406493E-3,1.10566504E8,4.988E3,6.5204306E-3,1.1358477E-4,1.43E2,-4.78206E-3,5.9668895E4,6.0147805E0,1.397998E6,1E0,4.8709216E5,1.2083E4,2.3894514E8,1.5714285E0,2.7402837E0,1.2083E4,1.5953E4,2.2E1,-1.6144059E-3,1.394E3,-1.5249467E-2,1.3540794E8,-1.989024E-3,2.6468704E-3,-1.2535569E-2,-6.0278955E-3,-5.3258E-4,-6.4358744E-3,1.3872898E-3,-2.0588164E-3,4.0710927E-3,-2.847884E-3,1.7960409E-4,6.605697E-3,9.125305E-3,-2.2485156E-3,-2.6149435E-3,1.5739279E-3,-1.9106813E-2,-4.5788083E-3,7.964844E-3,1.976115E-3,5.483936E-3,-3.2910407E-3,-1.981705E-3,4.9119703E-3,-6.2652295E-3,-1.7565843E-3,3.7371148E-3,-7.070262E-4,5.966806E-4,-5.2611437E-3],"split_indices":[43,70,53,5,68,68,1,5,0,46,57,2,1,0,27,9,0,26,71,0,50,46,0,3,0,0,0,10,9,2,68,57,60,73,0,5,44,0,0,2,0,43,69,60,23,43,9,7,68,68,9,9,71,0,10,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.98E2,7.49E2,1.49E2,1.5E2,5.99E2,1.08E2,4.1E1,1.47E2,3E0,5.41E2,5.8E1,8.3E1,2.5E1,2.5E1,1.6E1,1.41E2,6E0,2.73E2,2.68E2,7E0,5.1E1,7.5E1,8E0,1.8E1,7E0,2E0,1.4E1,9E1,5.1E1,1.16E2,1.57E2,1.87E2,8.1E1,4.9E1,2E0,4.5E1,3E1,1.5E1,3E0,7.5E1,1.5E1,2.2E1,2.9E1,2.5E1,9.1E1,4.3E1,1.14E2,1.74E2,1.3E1,5E1,3.1E1,1.2E1,3.7E1,1.6E1,2.9E1,2E0,2.8E1,5.6E1,1.9E1,8E0,1.4E1,2.6E1,3E0,1.1E1,1.4E1,8.5E1,6E0,2.2E1,2.1E1,3E0,1.11E2,1.17E2,5.7E1,3E0,1E1,1.4E1,3.6E1,6E0,2.5E1,5E0,7E0,1.6E1,2.1E1,2.2E1,7E0,2.2E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[-5.3547574E-3,-3.6141284E-2,1.0302226E-2,-2.9701613E-2,-8.9282775E-3,-3.6629792E-2,1.9024264E-2,-1.6097797E-3,-5.6666553E-2,-8.270869E-3,-2.8468512E-2,4.1261833E-2,2.677351E-3,-1.7809901E-2,7.3845014E-2,-8.321149E-2,3.7126653E-2,4.454336E-2,-5.5286843E-2,2.2693817E-2,1.0088986E-1,-1.075132E-2,5.8936562E-2,-3.5371136E-2,1.6064117E-2,1.2046278E-1,-1.2834016E-3,6.8393806E-3,-1.10005036E-1,-1.8348028E-3,1.8766867E-1,3.0500647E-3,-3.8278329E-3,-3.5824627E-2,-1.6263902E-1,6.1719738E-2,-7.854972E-3,1.3034622E-1,2.830742E-2,-1.24302156E-1,-4.2010457E-3,1.0157024E-1,1.7180571E-2,-9.763821E-3,-9.131668E-2,-2.8996677E-3,5.3364445E-2,-4.7302558E-5,6.311605E-3,5.4048274E-2,-7.10333E-2,-1.2829238E-1,3.4656767E-2,-2.8070878E-2,7.5621023E-3,1.1430217E-2,1.315006E-3,-4.8347324E-2,2.2549503E-3,-2.0853528E-3,-9.614008E-3,-5.6176293E-3,7.126713E-2,-1.2122345E-2,6.276647E-3,1.8666399E-1,6.3658245E-2,-1.6631272E-3,4.5040366E-3,-8.078293E-5,-2.4754763E-2,4.947723E-3,-7.511836E-2,1.14010416E-1,-7.317426E-5,4.681016E-2,-4.6759624E-2,-4.8846127E-3,2.7852596E-4,-5.7214247E-3,-1.4630135E-3,3.795901E-3,-3.1879912E-3,-1.4711417E-3,3.7572489E-3,-5.1611816E-3,1.2374998E-3,-4.898883E-3,-1.305724E-2,3.45846E-3,-4.4654384E-3,9.2278386E-4,-6.1559775E-3,-1.2744447E-3,-5.3080474E-3,3.915545E-3,-2.6546535E-3,-1.3473546E-3,2.1150014E-3,4.982476E-3,1.0423724E-2,-4.34093E-3,4.144589E-3,-1.4256933E-3,1.9332793E-3,-1.2588605E-2,-1.3563296E-4,-5.1947805E-4,5.8311797E-3,7.509675E-4,4.921865E-3,8.712791E-4,-4.350527E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,-1,57,59,61,63,65,67,69,71,73,75,77,79,-1,81,-1,-1,83,85,87,89,91,-1,-1,-1,93,-1,-1,-1,-1,95,-1,97,99,101,-1,-1,-1,-1,103,105,107,-1,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.390143E-1,3.0237737E-1,2.4793997E-1,2.241272E-1,0E0,1.0441989E-1,1.8528251E-1,1.7978218E-1,3.7903935E-1,0E0,1.806458E-1,2.3686004E-1,2.2350213E-1,7.254743E-2,1.2627646E-1,2.854606E-1,1.9976577E-1,7.211632E-2,1.341574E-1,1.9838911E-1,1.05349004E-1,1.7724341E-1,9.9727035E-2,1.13717616E-1,1.2616913E-1,3.348109E-2,0E0,1.0677516E-1,2.451849E-1,1.2535931E-1,5.7878852E-2,0E0,0E0,6.3580096E-2,4.2657375E-2,1.3580993E-1,3.374461E-1,1.2332308E-1,7.3720254E-2,6.4911586E-1,1.4765066E-1,3.6106855E-2,5.9024476E-2,8.4666446E-2,4.357542E-2,0E0,1.05487436E-1,0E0,0E0,4.3794684E-2,4.5852765E-2,2.539364E-1,6.4214E-2,1.2755111E-1,0E0,0E0,0E0,6.714221E-2,0E0,0E0,0E0,0E0,1.20922565E-1,0E0,1.2566946E-1,3.260261E-2,8.026112E-2,0E0,0E0,0E0,0E0,2.646469E-1,3.6632648E-1,3.6985666E-2,0E0,3.6294892E-2,3.2393634E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,46,46,49,49,50,50,51,51,52,52,53,53,57,57,62,62,64,64,65,65,66,66,71,71,72,72,73,73,75,75,76,76],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,-1,58,60,62,64,66,68,70,72,74,76,78,80,-1,82,-1,-1,84,86,88,90,92,-1,-1,-1,94,-1,-1,-1,-1,96,-1,98,100,102,-1,-1,-1,-1,104,106,108,-1,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.036081E7,2.856934E7,6.34E2,1E0,-8.9282775E-3,3.2E1,1.8839285E1,2.8515152E2,1.49660635E1,-8.270869E-3,1.8591549E0,1.2031894E5,8.83E3,1.1117E4,1E0,4.1E1,1.925508E6,1E0,5.17259E8,1.67806E4,1.2E1,1.9756098E1,7.153514E1,2.57E2,2.1605E4,3.0502679E-6,-1.2834016E-3,4.8421054E0,1.409E3,6.23694E5,1.409E3,3.0500647E-3,-3.8278329E-3,3.5358025E1,2.0217392E0,1E0,2.87834E5,2.652E3,1.361E3,1.9559006E1,3.0359353E5,3.3E1,3.1790426E8,9E0,3.2E1,-2.8996677E-3,1.944898E1,-4.7302558E-5,6.311605E-3,3.9E1,8.038E3,2.8172503E10,1.237036E6,1.6100003E8,7.5621023E-3,1.1430217E-2,1.315006E-3,1.3091352E4,2.2549503E-3,-2.0853528E-3,-9.614008E-3,-5.6176293E-3,3.624E3,-1.2122345E-2,2.593592E6,9.7199225E-1,3.8121E4,-1.6631272E-3,4.5040366E-3,-8.078293E-5,-2.4754763E-2,8.079573E5,1.4598765E0,3.3E1,-7.317426E-5,1.2406364E8,6.374563E5,-4.8846127E-3,2.7852596E-4,-5.7214247E-3,-1.4630135E-3,3.795901E-3,-3.1879912E-3,-1.4711417E-3,3.7572489E-3,-5.1611816E-3,1.2374998E-3,-4.898883E-3,-1.305724E-2,3.45846E-3,-4.4654384E-3,9.2278386E-4,-6.1559775E-3,-1.2744447E-3,-5.3080474E-3,3.915545E-3,-2.6546535E-3,-1.3473546E-3,2.1150014E-3,4.982476E-3,1.0423724E-2,-4.34093E-3,4.144589E-3,-1.4256933E-3,1.9332793E-3,-1.2588605E-2,-1.3563296E-4,-5.1947805E-4,5.8311797E-3,7.509675E-4,4.921865E-3,8.712791E-4,-4.350527E-3],"split_indices":[7,60,2,8,0,2,73,67,71,0,68,48,2,1,26,10,43,19,46,48,8,73,71,44,9,53,0,73,10,1,10,0,0,61,71,68,12,2,44,73,48,8,47,70,10,0,73,0,0,12,1,46,1,5,0,0,0,48,0,0,0,0,10,0,9,53,1,0,0,0,0,43,69,0,0,47,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.09E2,3.06E2,6.03E2,2.95E2,1.1E1,9.4E1,5.09E2,1.45E2,1.5E2,4E0,9E1,2.15E2,2.94E2,1.2E2,2.5E1,1.17E2,3.3E1,2.4E1,6.6E1,1.65E2,5E1,2.38E2,5.6E1,7.9E1,4.1E1,1.7E1,8E0,2.7E1,9E1,2.7E1,6E0,2.1E1,3E0,5.7E1,9E0,7.2E1,9.3E1,3.5E1,1.5E1,1.2E1,2.26E2,2.7E1,2.9E1,5.5E1,2.4E1,1.3E1,2.8E1,2E0,1.5E1,1.7E1,1E1,8E1,1E1,2.4E1,3E0,4E0,2E0,5E1,7E0,3E0,6E0,3E0,6.9E1,4E0,8.9E1,1.8E1,1.7E1,8E0,7E0,1E1,2E0,2.01E2,2.5E1,2.4E1,3E0,2E1,9E0,7E0,4.8E1,1.5E1,9E0,2.3E1,5E0,4E0,1.3E1,7E0,3E0,7.1E1,9E0,8E0,2E0,1.7E1,7E0,3.9E1,1.1E1,6.3E1,6E0,4.7E1,4.2E1,7E0,1.1E1,2E0,1.5E1,1.02E2,9.9E1,6E0,1.9E1,2E0,2.2E1,1.4E1,6E0,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-6.1315876E-3,-3.3869278E-2,9.025228E-3,-3.976103E-2,1.5628606E-1,1.3295871E-1,4.4990093E-3,-8.945562E-2,-2.3444615E-2,1.039788E-2,-2.507543E-5,2.1548626E-4,7.575171E-3,-8.426955E-3,3.7950113E-2,-6.3523404E-2,-2.5944236E-1,4.0577627E-3,-5.287869E-2,2.277497E-2,-2.9153228E-2,6.801792E-2,-2.1097201E-3,3.4713205E-3,-1.0936056E-1,-1.4979839E-2,-2.7387426E-4,9.251976E-3,-9.778571E-3,-6.2195476E-2,8.067236E-2,4.3366972E-2,-4.3688808E-2,-2.3554528E-2,-1.7707884E-1,7.662697E-2,-3.1694756E-3,8.5820764E-2,-4.4199124E-2,-2.1802792E-2,1.2914762E-1,-2.7732074E-2,-1.9368386E-1,2.0274671E-3,5.984724E-3,-6.866889E-2,5.8087395E-3,8.210912E-3,-2.7598883E-3,5.431704E-2,-1.07452475E-1,2.1237272E-3,-1.1254035E-1,-7.517623E-2,-9.45097E-3,-2.4207672E-2,-2.1379332E-4,5.604091E-2,6.4486205E-3,-5.240614E-3,1.10091746E-1,-1.4028701E-1,-8.272965E-3,-3.4405114E-3,1.4905209E-3,6.288211E-4,9.38175E-3,-3.8674846E-3,4.7864597E-3,-1.482218E-3,-1.1417469E-2,2.6404944E-3,-6.622101E-4,-1.6323687E-3,-4.783535E-3,-5.591936E-3,2.8542941E-3,9.5462747E-4,-9.949864E-3,2.7684076E-3,-2.1965469E-3,-7.1276035E-3,-8.9000247E-4,5.606721E-3,-4.1680126E-3,-6.852388E-4,6.151773E-3,5.307987E-3,1.5055538E-3,6.2179663E-3,1.3391064E-3,-9.4965653E-4,-8.896559E-3,2.9740909E-3,-1.6145227E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,-1,19,21,23,25,27,29,31,33,35,37,39,41,-1,-1,43,-1,45,47,49,51,53,55,57,-1,59,61,63,65,67,69,71,-1,73,-1,-1,-1,75,77,79,81,83,85,-1,-1,87,-1,-1,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.879284E-1,3.7144732E-1,3.3428907E-1,2.549382E-1,1.0564487E-1,7.9879165E-2,2.4980754E-1,3.2735372E-1,1.9449627E-1,0E0,0E0,0E0,0E0,2.7033767E-1,1.9406982E-1,2.1226814E-1,1.5696031E-1,1.4109528E-1,1.4935234E-1,2.3045456E-1,2.0370497E-1,1.1307129E-1,2.6285914E-1,9.4930135E-2,2.7683926E-1,0E0,0E0,1.0586707E-1,0E0,1.4059073E-1,1.1787152E-1,2.1722633E-1,1.2640955E-1,1.7620559E-1,5.307286E-1,1.0332525E-1,0E0,1.2321672E-1,1.6453871E-1,7.261336E-2,3.9232068E-2,1.6786957E-1,1.5985698E-1,1.05002195E-1,0E0,1.1803794E-1,0E0,0E0,0E0,1.5517011E-1,1.2704551E-1,7.363657E-2,5.7916254E-2,1.5781403E-1,1.4570491E-1,0E0,0E0,8.8425726E-2,0E0,0E0,3.6452472E-2,7.5574994E-2,7.0969254E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,49,49,50,50,51,51,52,52,53,53,54,54,57,57,60,60,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,-1,20,22,24,26,28,30,32,34,36,38,40,42,-1,-1,44,-1,46,48,50,52,54,56,58,-1,60,62,64,66,68,70,72,-1,74,-1,-1,-1,76,78,80,82,84,86,-1,-1,88,-1,-1,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.8776313E3,1.512E4,3.0352948E2,6.747343E8,4.03E2,5.867288E2,1.6028708E0,1E0,1.039788E-2,-2.507543E-5,2.1548626E-4,7.575171E-3,1E0,2.1298597E0,6.0921145E4,6.0078123E-6,3.0070068E7,1.465E4,1.586036E1,4.5620965E6,5.444797E0,3.9914E4,2.135961E4,1.1015E4,-1.4979839E-2,-2.7387426E-4,1.4433751E0,-9.778571E-3,6.1E1,5.59E2,1E0,6.4356956E0,1.6964E4,2.1027016E5,1.7E1,-3.1694756E-3,4E0,9.643839E6,1.1176471E1,8.071428E0,5.095006E5,1.3050649E4,1.5611285E0,5.984724E-3,8.2E1,5.8087395E-3,8.210912E-3,-2.7598883E-3,1.0204082E0,2.4915715E-3,1.4E1,8.2631064E7,2.6818182E0,3E1,-2.4207672E-2,-2.1379332E-4,5E0,6.4486205E-3,-5.240614E-3,1.9617874E3,3.202146E5,1.0717949E1,-3.4405114E-3,1.4905209E-3,6.288211E-4,9.38175E-3,-3.8674846E-3,4.7864597E-3,-1.482218E-3,-1.1417469E-2,2.6404944E-3,-6.622101E-4,-1.6323687E-3,-4.783535E-3,-5.591936E-3,2.8542941E-3,9.5462747E-4,-9.949864E-3,2.7684076E-3,-2.1965469E-3,-7.1276035E-3,-8.9000247E-4,5.606721E-3,-4.1680126E-3,-6.852388E-4,6.151773E-3,5.307987E-3,1.5055538E-3,6.2179663E-3,1.3391064E-3,-9.4965653E-4,-8.896559E-3,2.9740909E-3,-1.6145227E-3],"split_indices":[29,67,9,48,47,2,67,73,6,0,0,0,0,26,53,60,52,58,44,73,62,68,9,43,9,0,0,57,0,3,0,23,69,1,48,3,0,3,62,67,67,43,43,68,0,0,0,0,0,68,53,3,5,69,3,0,0,3,0,0,4,43,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.21E2,3.25E2,5.96E2,3.16E2,9E0,2E1,5.76E2,7.7E1,2.39E2,6E0,3E0,4E0,1.6E1,4.16E2,1.6E2,6.8E1,9E0,1.24E2,1.15E2,1.66E2,2.5E2,9.1E1,6.9E1,2.8E1,4E1,7E0,2E0,1.22E2,2E0,1.08E2,7E0,1.27E2,3.9E1,2.42E2,8E0,8.6E1,5E0,2.2E1,4.7E1,2.4E1,4E0,2.1E1,1.9E1,1.16E2,6E0,1.05E2,3E0,4E0,3E0,1.19E2,8E0,2.4E1,1.5E1,5.1E1,1.91E2,2E0,6E0,6.6E1,2E1,2E0,2E1,1.2E1,3.5E1,1.2E1,1.2E1,2E0,2E0,1.5E1,6E0,5E0,1.4E1,2.6E1,9E1,5.4E1,5.1E1,4E0,1.15E2,4E0,4E0,1.1E1,1.3E1,1E1,5E0,3E0,4.8E1,1.85E2,6E0,1.8E1,4.8E1,1.5E1,5E0,4E0,8E0,9E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-1.8830965E-3,-9.931902E-3,5.6846883E-2,-5.2415114E-3,-7.8833975E-2,9.633676E-2,-1.126331E-2,3.6054015E-2,-1.156768E-2,4.7611897E-3,-1.0028965E-1,-2.2674657E-3,1.0572662E-1,-4.638478E-2,1.05763875E-1,-1.693795E-2,6.521536E-2,-1.3266001E-2,6.974723E-3,-2.1625508E-1,-3.917288E-2,4.7964603E-2,1.2759149E-1,2.3425603E-3,-6.9751255E-2,7.676126E-3,-1.1361665E-3,-4.294594E-2,3.8466067E-3,8.410432E-2,-8.224265E-2,-3.9523676E-2,-1.9839979E-3,-1.0642463E-1,-1.4546754E-2,3.0191878E-3,-6.569075E-2,1.070486E-3,6.604021E-3,1.4002755E-1,2.2280742E-3,-4.0760428E-2,-1.052273E-2,1.0462985E-3,-2.9691188E-3,1.3120756E-1,3.0852433E-2,-1.3752537E-2,1.7763935E-3,-1.6884677E-2,-1.1189823E-1,5.6110803E-2,-1.11289695E-2,-4.2713393E-4,-8.4820585E-3,-3.8919854E-3,2.5206523E-3,2.3600515E-3,1.5101823E-1,-5.520256E-3,-6.5129E-4,6.7058834E-4,7.962227E-3,3.2041562E-3,-1.3386272E-3,8.787338E-5,-3.818155E-3,-9.901576E-3,-2.9112052E-3,7.6747805E-5,4.3873293E-3,-6.234111E-3,-3.3990544E-4,5.8412757E-3,1.04310885E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,-1,21,23,25,27,29,31,-1,33,35,37,39,-1,41,-1,-1,43,-1,45,47,49,51,53,-1,-1,55,-1,-1,57,-1,59,-1,-1,-1,61,63,-1,-1,65,67,69,71,-1,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.3923596E-1,2.6362294E-1,3.0232733E-1,2.0052879E-1,2.1244055E-1,1.01884305E-1,1.7753285E-1,1.5872693E-1,1.8442708E-1,0E0,3.2463798E-1,0E0,7.849836E-2,7.811989E-2,8.1220284E-2,9.9442095E-2,1.9074866E-1,1.951432E-1,0E0,1.3633859E-1,9.259328E-2,4.364926E-2,4.0857255E-2,0E0,1.1328393E-1,0E0,0E0,4.246074E-2,0E0,1.4394373E-1,2.1696758E-1,3.225311E-1,2.4607363E-1,6.4798534E-2,0E0,0E0,6.062591E-2,0E0,0E0,3.2568574E-2,0E0,4.8384428E-2,0E0,0E0,0E0,1.4233214E-1,6.7775495E-2,0E0,0E0,1.8625072E-1,2.2071874E-1,1.3099492E-1,1.8685408E-1,0E0,0E0,0E0,0E0,0E0,3.887999E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,24,24,27,27,29,29,30,30,31,31,32,32,33,33,36,36,39,39,41,41,45,45,46,46,49,49,50,50,51,51,52,52,58,58],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,-1,22,24,26,28,30,32,-1,34,36,38,40,-1,42,-1,-1,44,-1,46,48,50,52,54,-1,-1,56,-1,-1,58,-1,60,-1,-1,-1,62,64,-1,-1,66,68,70,72,-1,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.355E3,9.932432E0,8.166121E1,1.6954315E0,8.75477E5,1E0,4.435876E2,6.072304E5,6.1E1,4.7611897E-3,2.224E3,-2.2674657E-3,1.3293153E3,2.4994128E8,9.4326395E-1,3.472499E-1,3.1083834E5,3.7788504E7,6.974723E-3,3E0,3.18E3,8.2471845E6,3.8520416E5,2.3425603E-3,1.4669823E5,7.676126E-3,-1.1361665E-3,2.5493E4,3.8466067E-3,2.936348E10,1.102794E3,1.26E2,4.75E0,4.427818E7,-1.4546754E-2,3.0191878E-3,6.001692E-7,1.070486E-3,6.604021E-3,4E0,2.2280742E-3,4.6982746E8,-1.052273E-2,1.0462985E-3,-2.9691188E-3,2.2051822E2,1.1533101E0,-1.3752537E-2,1.7763935E-3,1.89991E5,8.138889E0,5.88E2,1.4693878E0,-4.2713393E-4,-8.4820585E-3,-3.8919854E-3,2.5206523E-3,2.3600515E-3,1.7E1,-5.520256E-3,-6.5129E-4,6.7058834E-4,7.962227E-3,3.2041562E-3,-1.3386272E-3,8.787338E-5,-3.818155E-3,-9.901576E-3,-2.9112052E-3,7.6747805E-5,4.3873293E-3,-6.234111E-3,-3.3990544E-4,5.8412757E-3,1.04310885E-2],"split_indices":[2,69,71,69,46,17,71,43,3,0,44,0,4,12,49,57,48,7,0,8,44,43,48,0,48,0,0,1,0,46,70,10,73,46,0,0,52,0,0,3,0,7,0,0,0,4,69,0,0,9,73,2,68,0,0,0,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.27E2,8.16E2,1.11E2,7.65E2,5.1E1,7E1,4.1E1,1.01E2,6.64E2,5E0,4.6E1,4E0,6.6E1,3.2E1,9E0,3.6E1,6.5E1,6.58E2,6E0,1.5E1,3.1E1,1.9E1,4.7E1,6E0,2.6E1,6E0,3E0,2.9E1,7E0,5.8E1,7E0,1.97E2,4.61E2,8E0,7E0,6E0,2.5E1,1.6E1,3E0,4E1,7E0,2.3E1,3E0,7E0,2.2E1,3E1,2.8E1,2E0,5E0,1.51E2,4.6E1,6.2E1,3.99E2,4E0,4E0,2.2E1,3E0,5E0,3.5E1,5E0,1.8E1,8E0,2.2E1,1.7E1,1.1E1,1.18E2,3.3E1,1.4E1,3.2E1,2.6E1,3.6E1,1.1E1,3.88E2,2.8E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-7.5851902E-3,-3.3919707E-2,1.11112865E-2,-2.7033148E-2,-1.6272405E-1,-3.893571E-3,3.885576E-2,1.243815E-2,-3.780366E-2,-1.0305161E-3,-2.5458822E-1,2.5214821E-2,-2.8844966E-2,1.3339089E-2,6.904404E-2,3.174541E-2,-1.2974867E-1,-2.5795935E-2,-1.2729613E-1,-1.6052773E-2,-2.7322364E-3,3.9598405E-2,-5.189877E-2,-1.1340869E-1,-7.2538652E-3,-2.3070702E-2,3.2638144E-2,8.879241E-2,1.9675894E-2,-2.217468E-2,6.204535E-2,-1.32189905E-2,-1.0667145E-3,-2.9718304E-2,6.8747043E-3,-1.7124924E-1,-1.3385781E-3,1.11663476E-1,2.338408E-2,-1.2818108E-1,8.556638E-3,-2.2072949E-2,-7.129482E-2,-1.2231012E-2,8.716637E-3,1.8166224E-2,-8.090687E-2,-1.551147E-3,6.988976E-2,1.475968E-1,5.4475904E-2,-6.322262E-2,3.7494423E-3,1.24128405E-2,-1.452998E-1,9.434141E-2,-1.31423455E-2,-4.3891966E-2,1.7007548E-2,-1.1014406E-2,-2.3658037E-1,2.601836E-3,6.9200736E-3,-6.621615E-2,4.8182163E-2,-8.148297E-4,-7.5863753E-3,-1.6083036E-2,5.612979E-3,-1.01052605E-1,3.8268053E-4,7.3719546E-3,-1.6643751E-2,-1.7879572E-4,5.7519637E-3,-7.0700846E-3,-1.5385988E-1,-4.551875E-2,4.481218E-2,9.032416E-2,-3.633432E-4,1.8676388E-1,5.0813034E-2,6.6406853E-3,1.4391824E-2,-1.5065247E-1,-3.1445295E-5,-3.8596787E-3,2.1093222E-3,-1.041957E-2,-2.3216354E-3,8.396884E-3,2.9852078E-3,-3.1724097E-3,2.7690586E-3,-1.5943643E-3,-5.233848E-3,1.3911147E-3,-4.9421084E-3,2.5461533E-3,-6.458315E-3,-2.488733E-2,-7.492351E-3,3.5351638E-3,-5.2407254E-3,6.8689524E-3,1.3337141E-3,-3.4879958E-3,2.490315E-3,-8.698009E-3,-2.2173403E-3,-1.0805964E-3,3.6868085E-3,-4.3726233E-3,3.7807608E-3,-1.3141613E-3,-8.883786E-3,-4.564963E-3,3.009359E-4,3.7596736E-3,-1.3253925E-3,4.909609E-5,5.3846086E-3,9.890917E-3,1.6737558E-4,-1.8803572E-3,5.285391E-3,-2.9649641E-3,2.6347234E-3,2.4549337E-4,-1.1922961E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,-1,-1,37,39,41,43,45,47,49,51,53,55,-1,-1,57,-1,59,-1,61,63,65,67,-1,69,71,-1,73,75,77,79,81,83,85,-1,87,89,91,93,95,97,99,101,-1,-1,103,105,-1,-1,107,-1,109,-1,-1,111,-1,-1,113,115,117,119,121,-1,123,125,-1,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.7206038E-1,3.486346E-1,2.3369665E-1,1.6159302E-1,2.5261843E-1,2.65825E-1,1.5082896E-1,2.2922847E-1,3.169475E-1,0E0,1.9258416E-1,1.895048E-1,3.588646E-1,7.664184E-2,8.599761E-2,1.2064143E-1,1.5848906E-1,1.8417218E-1,1.4662129E-1,0E0,0E0,1.6448246E-1,1.2561212E-1,5.746258E-1,1.5538499E-1,9.209923E-2,9.053446E-2,1.2110907E-1,1.4150582E-1,1.1678559E-1,1.1536838E-1,0E0,0E0,1.7227611E-1,0E0,2.4863404E-1,0E0,4.5467436E-2,2.6537633E-1,4.354015E-2,4.614775E-2,0E0,8.8389516E-2,1.194213E-1,0E0,5.3890318E-2,8.509806E-2,7.9502195E-2,5.4690868E-2,7.769686E-2,1.4661981E-1,6.551929E-2,0E0,7.2903976E-2,3.387776E-2,7.4193925E-2,6.410104E-2,1.2742111E-1,1.00211635E-1,7.4864104E-2,2.9687524E-1,0E0,0E0,1.8040648E-1,1.7627613E-1,0E0,0E0,6.144739E-2,0E0,1.14979565E-1,0E0,0E0,9.793605E-2,0E0,0E0,7.679166E-2,3.193982E-2,5.5365972E-2,5.248706E-2,6.123072E-2,0E0,7.516569E-2,5.045987E-2,0E0,1.0268198E-1,8.377337E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,33,33,35,35,37,37,38,38,39,39,40,40,42,42,43,43,45,45,46,46,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,63,63,64,64,67,67,69,69,72,72,75,75,76,76,77,77,78,78,79,79,81,81,82,82,84,84,85,85],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,-1,-1,38,40,42,44,46,48,50,52,54,56,-1,-1,58,-1,60,-1,62,64,66,68,-1,70,72,-1,74,76,78,80,82,84,86,-1,88,90,92,94,96,98,100,102,-1,-1,104,106,-1,-1,108,-1,110,-1,-1,112,-1,-1,114,116,118,120,122,-1,124,126,-1,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.49E2,1.05039425E1,1.8062708E6,1.6834415E0,3.89E2,2.335307E9,6.441311E-1,3.5548245E2,4.064E3,-1.0305161E-3,2.6235E4,2.2125713E1,3.653E3,1.5958088E10,2.855146E5,3.95E2,3.7789655E2,3.653E3,4.8E0,-1.6052773E-2,-2.7322364E-3,5.457143E0,6.3111855E4,4E0,4.43837E0,1.03E2,1E0,4.2904656E8,3.7748723E8,7.122041E7,6.69E2,-1.32189905E-2,-1.0667145E-3,2.8788235E2,6.8747043E-3,5.7777777E0,-1.3385781E-3,8.771579E2,1.48E2,2.5176924E1,3.3E1,-2.2072949E-2,1.380863E6,3.7974394E-6,8.716637E-3,1.1860938E8,2.769E3,2.5695364E0,8.681474E4,1.4E1,3.9914E4,1.1293673E3,3.7494423E-3,1.18E2,1.0952381E0,5.862757E1,1.1057693E0,1.6523365E1,4.3024124E2,1.5E1,1.5933333E2,2.601836E-3,6.9200736E-3,3.5E1,2.72E2,-8.148297E-4,-7.5863753E-3,1.0652307E7,5.612979E-3,8.215373E6,3.8268053E-4,7.3719546E-3,3.002566E0,-1.7879572E-4,5.7519637E-3,1.3826087E1,4.8581E4,2.77843E5,8.620714E2,3.6977E4,-3.633432E-4,1.8484E4,3.6451373E8,6.6406853E-3,1.1342433E3,8.3068E4,-3.1445295E-5,-3.8596787E-3,2.1093222E-3,-1.041957E-2,-2.3216354E-3,8.396884E-3,2.9852078E-3,-3.1724097E-3,2.7690586E-3,-1.5943643E-3,-5.233848E-3,1.3911147E-3,-4.9421084E-3,2.5461533E-3,-6.458315E-3,-2.488733E-2,-7.492351E-3,3.5351638E-3,-5.2407254E-3,6.8689524E-3,1.3337141E-3,-3.4879958E-3,2.490315E-3,-8.698009E-3,-2.2173403E-3,-1.0805964E-3,3.6868085E-3,-4.3726233E-3,3.7807608E-3,-1.3141613E-3,-8.883786E-3,-4.564963E-3,3.009359E-4,3.7596736E-3,-1.3253925E-3,4.909609E-5,5.3846086E-3,9.890917E-3,1.6737558E-4,-1.8803572E-3,5.285391E-3,-2.9649641E-3,2.6347234E-3,2.4549337E-4,-1.1922961E-2],"split_indices":[2,69,43,68,2,46,57,4,44,0,1,71,44,46,48,2,4,44,69,0,0,73,48,3,50,0,29,7,7,7,2,0,0,70,0,73,0,48,10,71,8,0,9,53,0,5,44,68,48,3,9,4,0,1,68,62,69,73,4,3,4,0,0,0,10,0,0,66,0,12,0,0,53,0,0,73,1,1,4,12,0,44,7,0,4,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.57E2,3.97E2,5.6E2,3.78E2,1.9E1,3.64E2,1.96E2,8.1E1,2.97E2,8E0,1.1E1,1.68E2,1.96E2,1.07E2,8.9E1,7.2E1,9E0,2.63E2,3.4E1,7E0,4E0,1.42E2,2.6E1,3.9E1,1.57E2,3.7E1,7E1,6.3E1,2.6E1,2.6E1,4.6E1,3E0,6E0,2.58E2,5E0,2.3E1,1.1E1,2.5E1,1.17E2,1.1E1,1.5E1,3E0,3.6E1,1.54E2,3E0,2.2E1,1.5E1,3.7E1,3.3E1,2.2E1,4.1E1,1.1E1,1.5E1,2.1E1,5E0,3.2E1,1.4E1,1.98E2,6E1,7E0,1.6E1,1.1E1,1.4E1,2.5E1,9.2E1,3E0,8E0,1.3E1,2E0,2.6E1,1E1,3E0,1.51E2,1.9E1,3E0,8E0,7E0,1.9E1,1.8E1,2.6E1,7E0,1.5E1,7E0,1.2E1,2.9E1,4E0,7E0,5E0,1.6E1,2E0,3E0,7E0,2.5E1,8E0,6E0,1.75E2,2.3E1,5.5E1,5E0,5E0,2E0,2E0,1.4E1,6E0,1.9E1,1.4E1,7.8E1,7E0,6E0,9E0,1.7E1,1.42E2,9E0,4E0,4E0,2E0,5E0,9E0,1E1,1.2E1,6E0,6E0,2E1,1.3E1,2E0,3E0,4E0,1E1,1.9E1,2E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-3.764493E-3,-1.1070946E-2,5.182764E-2,-1.3463433E-2,1.4835386E-1,7.792042E-2,5.0036144E-3,-1.2080119E-2,-1.3207857E-2,1.3640304E-3,8.593202E-3,9.243445E-2,-1.9760642E-3,-4.068018E-2,5.043723E-2,-4.8961286E-2,-2.0721115E-3,-1.8507877E-4,1.0302469E-1,2.0430137E-4,-1.08838856E-1,5.586778E-3,8.9290657E-4,-3.293109E-2,-1.3586077E-1,3.7324846E-2,-9.042435E-3,6.609657E-3,5.72177E-2,-7.581477E-3,-6.468162E-4,-2.0527765E-2,-8.843357E-2,-2.2065419E-1,-1.7390196E-4,-6.5335065E-2,5.764426E-2,-6.9811635E-2,-2.1583778E-3,3.9376295E-4,1.3924783E-1,-3.5390106E-4,-3.7415565E-3,-8.021237E-3,-1.5718369E-3,-1.4493021E-2,-7.9503603E-4,-2.9518711E-3,4.1148267E-3,1.5719879E-3,-4.706429E-3,-9.819139E-3,3.133092E-3,-1.9919532E-3,-1.5145583E-2,1.936987E-4,-2.6215762E-3,2.0039142E-3,9.123537E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,-1,19,21,23,25,-1,27,-1,29,-1,-1,31,33,35,37,-1,39,-1,-1,41,43,45,47,49,51,53,55,-1,57,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7060496E-1,3.094232E-1,1.2911764E-1,2.940807E-1,4.6593755E-2,1.2276608E-1,8.299941E-2,2.9223126E-1,0E0,0E0,0E0,6.269395E-2,0E0,6.1920494E-2,4.1232646E-2,2.3094556E-1,1.7169386E-1,0E0,9.036428E-2,0E0,3.887754E-2,0E0,0E0,9.731646E-2,2.9910085E-1,1.9984919E-1,2.2198716E-1,0E0,1.0679555E-1,0E0,0E0,9.1505155E-2,1.1355029E-1,2.909671E-1,6.733986E-2,6.0303792E-2,2.2067523E-1,3.5881266E-1,1.6408682E-1,0E0,4.6286628E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,13,13,14,14,15,15,16,16,18,18,20,20,23,23,24,24,25,25,26,26,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,-1,20,22,24,26,-1,28,-1,30,-1,-1,32,34,36,38,-1,40,-1,-1,42,44,46,48,50,52,54,56,-1,58,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.6878237E8,2.243416E6,8.7E1,2.5817454E8,1.4E1,2.2E1,3.558194E6,3.3852024E7,-1.3207857E-2,1.3640304E-3,8.593202E-3,1.7068776E10,-1.9760642E-3,1.0046621E7,2.4508352E9,3.9824562E0,4.75E0,-1.8507877E-4,1.2010758E9,2.0430137E-4,1.3388E4,5.586778E-3,8.9290657E-4,8E0,2E0,3.764706E1,1.5933333E2,6.609657E-3,1.8E1,-7.581477E-3,-6.468162E-4,2.6575898E4,6.479833E5,6.0714287E1,6.550562E0,6.711294E7,1E0,2.2318378E6,2E0,3.9376295E-4,7.0527176E1,-3.5390106E-4,-3.7415565E-3,-8.021237E-3,-1.5718369E-3,-1.4493021E-2,-7.9503603E-4,-2.9518711E-3,4.1148267E-3,1.5719879E-3,-4.706429E-3,-9.819139E-3,3.133092E-3,-1.9919532E-3,-1.5145583E-2,1.936987E-4,-2.6215762E-3,2.0039142E-3,9.123537E-3],"split_indices":[12,1,8,12,3,3,1,7,0,0,0,46,0,62,12,68,73,0,5,0,2,0,0,8,32,4,4,0,8,0,0,66,43,68,68,7,68,43,6,0,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.1E2,8.05E2,1.05E2,7.94E2,1.1E1,6.7E1,3.8E1,7.91E2,3E0,3E0,8E0,6E1,7E0,1.9E1,1.9E1,1.68E2,6.23E2,6E0,5.4E1,1.2E1,7E0,5E0,1.4E1,1.43E2,2.5E1,9.3E1,5.3E2,2.8E1,2.6E1,4E0,3E0,1.18E2,2.5E1,1.5E1,1E1,1.5E1,7.8E1,5.3E1,4.77E2,1.7E1,9E0,9.8E1,2E1,9E0,1.6E1,1E1,5E0,6E0,4E0,4E0,1.1E1,2E0,7.6E1,4.9E1,4E0,4.28E2,4.9E1,4E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"59","size_leaf_vector":"1"}},{"base_weights":[-1.5955168E-3,-2.2241378E-2,2.6986903E-2,-1.5100869E-2,-9.822992E-2,-3.0108398E-2,3.7999194E-2,-2.5502797E-2,2.7541753E-2,-2.63705E-1,-6.0808063E-2,-8.172456E-2,3.1445324E-2,7.4912235E-2,2.6544187E-2,-1.9990062E-2,-9.305736E-2,9.5607154E-2,5.083092E-4,-1.5203698E-2,-5.5239378E-3,-9.563888E-3,-9.764183E-2,-4.0281244E-2,-3.1403956E-1,7.706067E-2,-3.9598137E-2,1.1235563E-2,6.703645E-2,3.1223005E-2,-5.9356485E-2,-4.7198083E-2,-6.1893915E-3,9.951958E-4,-1.1222625E-1,6.3169554E-2,9.236892E-3,-5.319194E-2,3.0330485E-2,-4.1643355E-2,4.8881886E-3,-3.0595519E-2,-8.140585E-3,-1.3812861E-2,-1.2738071E-2,-6.959229E-4,-2.3646107E-2,5.9469906E-3,-1.1332628E-4,-5.9872824E-3,2.0553E-4,1.0323165E-1,3.2834686E-2,-9.9854745E-2,3.4535475E-2,2.8617939E-3,-4.7334014E-3,-8.663574E-2,-3.1424467E-2,1.4158846E-1,-1.4588175E-2,-8.3190305E-3,-3.2713525E-3,2.878184E-4,8.934808E-2,-4.6362127E-3,2.551063E-4,1.4135255E-1,-5.4287477E-3,-3.187018E-3,3.709991E-3,-3.515675E-3,1.6045659E-3,-4.9888883E-3,5.193936E-4,7.316639E-2,9.90799E-3,-1.3857928E-2,8.807929E-2,-2.05063E-4,-9.014587E-3,7.759001E-2,2.1052731E-2,-4.7220397E-3,9.3355513E-4,-3.6289112E-4,-3.1956134E-3,-1.7803119E-4,8.631629E-3,-4.4219353E-4,-4.8944196E-3,7.3529957E-3,2.3241325E-3,1.0377029E-2,1.6752388E-3,2.733508E-3,-2.6286426E-3,6.499873E-3,1.9134333E-3,3.6411034E-4,-5.2742665E-3,8.433304E-3,5.6981307E-4,4.4344E-3,-3.7375563E-3,1.48173E-3,-2.2449354E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,-1,39,41,43,45,47,49,-1,51,53,55,57,59,-1,61,63,-1,65,67,69,-1,71,-1,73,-1,-1,-1,-1,-1,-1,-1,75,77,79,81,-1,-1,83,85,87,89,-1,-1,-1,91,-1,-1,93,95,-1,-1,-1,-1,-1,-1,97,-1,99,101,-1,-1,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.3464854E-1,2.839226E-1,2.4006921E-1,2.1450096E-1,2.6262194E-1,1.9955422E-1,1.3289723E-1,1.4263505E-1,1.747554E-1,3.4451544E-2,7.079178E-2,3.1469887E-1,9.7030945E-2,8.225629E-2,1.0032211E-1,1.3488388E-1,6.602728E-2,7.7665746E-2,1.1211396E-1,0E0,0E0,6.691183E-2,1.08805194E-1,1.8171605E-1,2.8308785E-1,7.301942E-2,4.7653288E-2,0E0,8.6980015E-2,1.0461512E-1,7.292978E-2,7.254854E-2,2.9994255E-1,0E0,5.7208955E-2,3.2263935E-2,0E0,6.87522E-2,1.7942023E-1,5.116339E-2,0E0,4.0688004E-2,0E0,6.656805E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.99625E-2,1.01821676E-1,5.296275E-2,1.30719E-1,0E0,0E0,5.6085795E-2,7.6972894E-2,8.390331E-2,1.0399887E-1,0E0,0E0,0E0,3.2707416E-2,0E0,0E0,8.67091E-2,1.1813341E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.6094646E-2,0E0,4.83839E-2,1.250578E-1,0E0,0E0,1.5912926E-1,1.3264626E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,31,31,32,32,34,34,35,35,37,37,38,38,39,39,41,41,43,43,51,51,52,52,53,53,54,54,57,57,58,58,59,59,60,60,64,64,67,67,68,68,75,75,77,77,78,78,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,-1,40,42,44,46,48,50,-1,52,54,56,58,60,-1,62,64,-1,66,68,70,-1,72,-1,74,-1,-1,-1,-1,-1,-1,-1,76,78,80,82,-1,-1,84,86,88,90,-1,-1,-1,92,-1,-1,94,96,-1,-1,-1,-1,-1,-1,98,-1,100,102,-1,-1,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.860918E5,5.615114E0,2.4568965E2,5.8222644E2,3.964269E3,5.1942E4,9E0,1E0,1.273782E4,6.2E1,1.0647887E1,2.829581E10,1.3E1,5.26E2,1E0,3.51E2,1.6896813E0,5.36763E5,1.08367164E5,-1.5203698E-2,-5.5239378E-3,5E1,1.453785E6,1.3E1,7.059674E2,2.821E3,3.2911258E6,1.1235563E-2,2.2842104E0,9.6682465E-1,2E1,1.5827801E4,1.1E1,9.951958E-4,2.5625E0,1.609E3,9.236892E-3,1.5570383E5,2.5427E4,9.277255E6,4.8881886E-3,3.0739096E4,-8.140585E-3,1.1742609E7,-1.2738071E-2,-6.959229E-4,-2.3646107E-2,5.9469906E-3,-1.1332628E-4,-5.9872824E-3,2.0553E-4,4.117083E1,2.7959185E0,6.165E3,1.7047619E0,2.8617939E-3,-4.7334014E-3,7.8625E1,3.1E1,5E0,4.026E3,-8.3190305E-3,-3.2713525E-3,2.878184E-4,1.1941613E3,-4.6362127E-3,2.551063E-4,4.111111E0,3.8908466E1,-3.187018E-3,3.709991E-3,-3.515675E-3,1.6045659E-3,-4.9888883E-3,5.193936E-4,1.1057693E0,9.90799E-3,2.4624279E0,2.7496547E8,-2.05063E-4,-9.014587E-3,3.3000212E5,1.3602516E7,-4.7220397E-3,9.3355513E-4,-3.6289112E-4,-3.1956134E-3,-1.7803119E-4,8.631629E-3,-4.4219353E-4,-4.8944196E-3,7.3529957E-3,2.3241325E-3,1.0377029E-2,1.6752388E-3,2.733508E-3,-2.6286426E-3,6.499873E-3,1.9134333E-3,3.6411034E-4,-5.2742665E-3,8.433304E-3,5.6981307E-4,4.4344E-3,-3.7375563E-3,1.48173E-3,-2.2449354E-3],"split_indices":[43,68,70,67,43,1,3,19,48,0,73,46,3,44,113,2,68,1,43,0,0,3,9,8,48,10,43,0,69,69,6,43,10,0,68,2,0,48,9,9,0,48,0,5,0,0,0,0,0,0,0,73,68,2,69,0,0,4,10,3,2,0,0,0,4,0,0,69,73,0,0,0,0,0,0,69,0,68,7,0,0,48,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.04E2,5.25E2,3.79E2,4.81E2,4.4E1,6.1E1,3.18E2,3.87E2,9.4E1,7E0,3.7E1,3.3E1,2.8E1,7.4E1,2.44E2,3.59E2,2.8E1,2.6E1,6.8E1,4E0,3E0,1.6E1,2.1E1,2.9E1,4E0,1.7E1,1.1E1,2E0,7.2E1,2.32E2,1.2E1,1.2E2,2.39E2,4E0,2.4E1,2.1E1,5E0,2.4E1,4.4E1,1.3E1,3E0,1.2E1,9E0,2.7E1,2E0,2E0,2E0,1E1,7E0,3E0,8E0,3.4E1,3.8E1,5E0,2.27E2,3E0,9E0,3.3E1,8.7E1,1.2E1,2.27E2,8E0,1.6E1,7E0,1.4E1,1.3E1,1.1E1,1E1,3.4E1,1.1E1,2E0,7E0,5E0,5E0,2.2E1,2.8E1,6E0,2.1E1,1.7E1,3E0,2E0,5.3E1,1.74E2,2.9E1,4E0,5.4E1,3.3E1,3E0,9E0,2.16E2,1.1E1,4E0,1E1,5E0,5E0,1.5E1,1.9E1,8E0,2E1,1.8E1,3E0,7E0,1E1,4.8E1,5E0,1.51E2,2.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[8.410529E-4,-6.81879E-3,5.0499704E-2,-8.237207E-2,-2.0190857E-3,7.555103E-2,1.5529887E-3,-1.3076364E-1,-1.3927515E-2,2.4167055E-4,-1.3142249E-1,3.2459658E-2,9.978722E-2,-3.2696255E-2,9.2754886E-2,-1.0449203E-1,-1.2197551E-2,2.2618918E-2,-5.072275E-3,2.2170633E-2,-1.2197535E-2,-1.2741052E-2,-3.7634447E-2,-9.372603E-4,3.3288812E-3,1.1397676E-1,3.4869698E-4,1.6534032E-3,-2.475487E-3,-2.2132779E-3,7.854006E-3,9.169363E-4,-5.401851E-3,5.8160145E-3,-1.020738E-3,5.5580908E-3,5.321584E-2,-8.524899E-3,-1.534957E-1,-6.2676175E-3,9.369509E-3,4.7093675E-2,6.365529E-3,2.4475267E-3,8.611108E-3,6.6893555E-2,-6.836633E-2,-3.127711E-2,7.449369E-3,-1.2536597E-2,3.486047E-4,4.8982045E-3,-2.3288683E-3,-1.3096E-3,5.3771376E-3,-4.916064E-3,3.2149372E-4,-4.169453E-6,4.3367883E-3,-9.493126E-3,-4.1451765E-4,-7.5610046E-4,-4.9236044E-3,4.497503E-3,-5.184143E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,-1,35,37,-1,39,-1,-1,41,-1,-1,-1,-1,-1,-1,-1,-1,-1,43,45,47,49,-1,51,53,-1,55,-1,57,59,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.5295424E-1,2.9139096E-1,1.5192902E-1,1.5716147E-1,2.2149178E-1,8.239934E-2,1.3734736E-1,6.9331706E-2,7.548133E-2,2.0349067E-1,1.6347358E-1,6.524269E-2,6.724191E-2,4.6731155E-2,1.3623464E-1,3.8490057E-2,0E0,7.578864E-2,0E0,1.3895395E-1,2.457053E-1,0E0,4.235259E-2,0E0,0E0,6.245935E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.842744E-2,1.608921E-1,1.6923721E-1,2.2624537E-1,0E0,4.52003E-2,7.0292376E-2,0E0,8.524167E-2,0E0,1.5006539E-1,7.88909E-2,2.1337669E-1,2.0955014E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,22,22,25,25,35,35,36,36,37,37,38,38,40,40,41,41,43,43,45,45,46,46,47,47,48,48],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,-1,36,38,-1,40,-1,-1,42,-1,-1,-1,-1,-1,-1,-1,-1,-1,44,46,48,50,-1,52,54,-1,56,-1,58,60,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,3.964269E3,8.166121E1,5E0,1.9269184E7,1.3733966E3,4.435876E2,4.23E2,2.121474E-6,2.018868E0,2.395631E7,8.83E3,1.0905187E9,4.9538185E6,1E1,1.00037E5,-1.2197551E-2,1.2E1,-5.072275E-3,3.1157124E-1,4.2E1,-1.2741052E-2,2.908E3,-9.372603E-4,3.3288812E-3,3E0,3.4869698E-4,1.6534032E-3,-2.475487E-3,-2.2132779E-3,7.854006E-3,9.169363E-4,-5.401851E-3,5.8160145E-3,-1.020738E-3,2.9308079E1,5.1291872E8,2.796E2,1.75E2,-6.2676175E-3,3.443E3,3.5510652E6,6.365529E-3,1.5857142E1,8.611108E-3,8.295515E7,3E0,2.096295E6,3.63E2,-1.2536597E-2,3.486047E-4,4.8982045E-3,-2.3288683E-3,-1.3096E-3,5.3771376E-3,-4.916064E-3,3.2149372E-4,-4.169453E-6,4.3367883E-3,-9.493126E-3,-4.1451765E-4,-7.5610046E-4,-4.9236044E-3,4.497503E-3,-5.184143E-5],"split_indices":[2,43,71,6,62,4,71,44,52,68,62,2,7,60,3,5,0,3,0,53,6,0,2,0,0,8,0,0,0,0,0,0,0,0,0,71,7,70,6,0,2,43,0,4,0,7,8,12,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,8.03E2,1.23E2,4.7E1,7.56E2,8.1E1,4.2E1,2.7E1,2E1,7.44E2,1.2E1,3E1,5.1E1,3.1E1,1.1E1,2.4E1,3E0,1.5E1,5E0,2.69E2,4.75E2,4E0,8E0,1.3E1,1.7E1,4.4E1,7E0,7E0,2.4E1,4E0,7E0,2E0,2.2E1,4E0,1.1E1,1.76E2,9.3E1,4.64E2,1.1E1,2E0,6E0,1.2E1,3.2E1,1.74E2,2E0,8.4E1,9E0,1.91E2,2.73E2,6E0,5E0,2E0,4E0,6E0,6E0,6E0,1.68E2,2.4E1,6E1,2E0,7E0,1.6E2,3.1E1,2.3E1,2.5E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-4.8644756E-4,-3.23938E-2,1.2457375E-2,-2.5717795E-2,-1.5406287E-1,1.879217E-2,-5.503729E-2,-2.1103632E-2,-2.1753344E-1,-8.810356E-3,-8.493359E-4,-1.2507753E-1,2.2101264E-2,-7.207124E-3,-3.352033E-2,-5.908723E-2,1.0665064E-3,-1.7372953E-2,-2.1457146E-3,-1.426461E-2,-4.7656547E-2,5.1998604E-2,8.641062E-3,-2.214111E-2,-7.25524E-3,-1.2270775E-2,-4.9437597E-2,1.3564049E-1,-1.0603682E-2,-8.2800835E-2,3.4258335E-3,4.4211928E-2,1.7256619E-1,2.0478064E-2,-2.377647E-2,-5.4613296E-2,3.6465164E-2,-6.1935842E-2,1.3018913E-2,8.31638E-3,1.4321237E-3,-1.0763046E-3,-1.1119095E-2,3.0528298E-3,-6.259469E-3,3.0634789E-2,1.1850585E-1,1.0828335E-2,3.8853283E-3,6.9190795E-3,6.6949956E-2,-7.65152E-2,1.1893824E-3,-6.803444E-2,4.1324366E-3,1.1527429E-1,-7.9068035E-4,-1.0695596E-3,-4.0382734E-3,6.3809506E-3,-1.2181657E-3,1.7099654E-3,-1.2430589E-3,3.8062125E-3,7.001689E-4,7.62504E-3,2.2339744E-3,4.5827075E-4,-7.7769244E-3,6.0257665E-3,5.79458E-4,-1.0652026E-2,-7.995203E-4,5.440363E-3,-7.3741697E-4,4.063742E-3,-3.8764037E-3,7.909278E-3,-6.7283254E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,-1,-1,-1,29,31,33,35,-1,-1,37,39,41,43,-1,45,47,49,51,53,55,57,59,-1,-1,61,-1,-1,-1,63,65,-1,-1,67,69,71,73,75,-1,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9069146E-1,2.1778566E-1,2.887854E-1,2.2552712E-1,6.379807E-2,2.9602867E-1,1.195157E-1,2.1479896E-1,1.4321074E-1,0E0,0E0,1.8225963E-1,2.4221244E-1,0E0,6.483908E-2,1.7268458E-1,2.558599E-1,0E0,0E0,0E0,5.2999396E-2,1.6793007E-1,1.605008E-1,8.953288E-2,0E0,0E0,7.1840644E-2,5.3344756E-2,3.2626167E-1,7.8024514E-2,0E0,1.7490932E-1,3.956291E-2,1.9235095E-1,1.4755774E-1,6.3810505E-2,7.35915E-2,7.167399E-2,8.150498E-2,0E0,0E0,1.4174297E-1,0E0,0E0,0E0,1.192327E-1,7.859722E-2,0E0,0E0,1.2327142E-1,2.3380956E-1,3.2180125E-1,1.5382533E-1,7.392913E-2,0E0,5.2220084E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,20,20,21,21,22,22,23,23,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,41,41,45,45,46,46,49,49,50,50,51,51,52,52,53,53,55,55],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,-1,-1,-1,30,32,34,36,-1,-1,38,40,42,44,-1,46,48,50,52,54,56,58,60,-1,-1,62,-1,-1,-1,64,66,-1,-1,68,70,72,74,76,-1,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.99E2,2.9E1,3.2856784E7,6.83E2,2.871287E0,1.6716E5,2.022131E6,2.71E2,1.1E1,-8.810356E-3,-8.493359E-4,7.6E2,2.01184E5,-7.207124E-3,2.3040113E10,1.0952381E0,1.68E2,-1.7372953E-2,-2.1457146E-3,-1.426461E-2,4.23025E2,1.53053E5,1.3400493E0,5.97E3,-7.25524E-3,-1.2270775E-2,2.1E1,1.2E1,5.75E0,1.3E1,3.4258335E-3,1.5632523E3,1.82407E5,7.5491136E-1,1.034073E6,1E0,1E0,6.0921145E4,4.4615383E0,8.31638E-3,1.4321237E-3,1.7211578E2,-1.1119095E-2,3.0528298E-3,-6.259469E-3,1.798E3,1.0656093E2,1.0828335E-2,3.8853283E-3,3.8652172E1,8E0,6.27907E0,1.7171717E0,8.82E2,4.1324366E-3,8.837831E-2,-7.9068035E-4,-1.0695596E-3,-4.0382734E-3,6.3809506E-3,-1.2181657E-3,1.7099654E-3,-1.2430589E-3,3.8062125E-3,7.001689E-4,7.62504E-3,2.2339744E-3,4.5827075E-4,-7.7769244E-3,6.0257665E-3,5.79458E-4,-1.0652026E-2,-7.995203E-4,5.440363E-3,-7.3741697E-4,4.063742E-3,-3.8764037E-3,7.909278E-3,-6.7283254E-4],"split_indices":[2,8,60,2,68,7,12,2,3,0,0,2,9,0,5,68,44,0,0,0,4,9,57,2,0,0,3,10,68,3,0,67,9,53,9,16,14,60,69,0,0,67,0,0,0,44,73,0,0,68,8,71,69,2,0,72,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.44E2,2.72E2,6.72E2,2.59E2,1.3E1,6.15E2,5.7E1,2.54E2,5E0,1E1,3E0,1.3E1,6.02E2,9E0,4.8E1,9.3E1,1.61E2,2E0,3E0,3E0,1E1,1.86E2,4.16E2,4.5E1,3E0,3E0,9E1,1.2E1,1.49E2,8E0,2E0,1.76E2,1E1,3.05E2,1.11E2,2.9E1,1.6E1,7.5E1,1.5E1,8E0,4E0,1.44E2,5E0,2E0,6E0,1.5E2,2.6E1,5E0,5E0,2.37E2,6.8E1,3.5E1,7.6E1,2.7E1,2E0,6E0,1E1,3E1,4.5E1,3E0,1.2E1,5.8E1,8.6E1,3.4E1,1.16E2,1.5E1,1.1E1,2.34E2,3E0,3.1E1,3.7E1,9E0,2.6E1,9E0,6.7E1,2E0,2.5E1,4E0,2E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-6.122535E-4,-1.7804293E-2,1.9562284E-2,-1.4045389E-2,-1.5580331E-1,-1.7949415E-3,5.0796792E-2,7.967744E-3,-2.5318457E-2,-1.5808992E-2,-1.6837202E-3,-3.6024544E-2,1.4429578E-2,7.4761555E-2,1.33977905E-2,-3.9523605E-2,3.9548162E-2,-6.568805E-2,-1.5127159E-2,-4.976319E-2,5.409836E-3,2.0221129E-2,-6.892807E-3,8.345547E-2,-2.6280446E-2,4.622428E-3,-2.197485E-2,5.0265778E-2,-4.9340308E-2,7.326243E-2,-1.6500067E-2,2.7549123E-2,-8.773411E-2,-4.7515333E-2,4.093109E-4,3.8491169E-3,-6.6822514E-2,-4.8637938E-2,2.7383538E-2,9.15767E-2,-3.2882504E-2,-3.730315E-3,1.5302832E-3,-1.7414069E-1,1.196338E-2,-2.1216092E-3,4.545801E-3,-5.53626E-2,3.8851257E-3,6.995949E-4,1.121936E-1,-9.9103205E-2,3.5814397E-2,4.2906883E-3,-1.1359965E-3,3.8949484E-3,-4.747218E-3,-6.9080177E-3,-9.542116E-2,2.9900737E-2,-2.5007868E-2,-4.6026085E-2,-1.5830398E-1,9.332241E-5,-6.203178E-3,-4.9205823E-3,3.1727366E-2,4.21176E-2,1.2085334E-1,2.6980403E-3,-3.7598237E-3,-1.3378036E-2,-3.229601E-3,-8.040162E-3,2.5875237E-2,-4.751271E-3,-1.7050448E-3,-5.3496327E-4,5.741188E-3,-1.6268429E-3,-7.78488E-3,3.3281483E-3,-1.4749998E-3,-2.7998225E-3,3.8620674E-3,8.8435126E-4,-3.4497385E-3,1.66105E-3,-5.63325E-3,-1.6587214E-3,2.7815397E-3,-2.0625112E-3,1.1453169E-3,-3.3202164E-3,-3.03656E-4,-1.26648415E-2,-4.1876268E-3,1.1199496E-3,4.773892E-3,3.066202E-3,-1.5810982E-3,6.536712E-3,2.9217622E-3,3.3632622E-3,-1.0166428E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,33,35,-1,37,-1,39,41,-1,43,45,47,49,51,53,55,57,59,-1,61,63,65,67,69,-1,-1,71,73,-1,-1,75,-1,-1,77,79,81,-1,-1,83,-1,85,87,89,91,93,95,-1,-1,-1,97,99,101,-1,-1,-1,-1,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1736135E-1,2.5406182E-1,2.809329E-1,1.19812705E-1,2.7396333E-1,1.3991489E-1,1.5225855E-1,2.4752975E-1,1.3024886E-1,0E0,0E0,1.7532395E-1,1.6406618E-1,9.355217E-2,2.0902666E-1,6.0654886E-2,1.8828171E-1,1.3396981E-1,1.288381E-1,1.750913E-1,0E0,8.320256E-2,0E0,9.371275E-2,3.1364623E-2,0E0,2.5513104E-1,3.632532E-2,5.2638605E-2,1.388394E-1,1.6776022E-1,4.5975707E-2,6.872338E-2,1.6080296E-1,1.3117622E-1,0E0,1.2018052E-1,6.893517E-2,9.071544E-2,1.252296E-1,3.520585E-2,0E0,0E0,8.698541E-2,1.0990455E-1,0E0,0E0,4.701495E-2,0E0,0E0,5.2920848E-2,5.9825554E-2,6.020522E-2,0E0,0E0,4.558338E-2,0E0,8.162523E-2,1.3241562E-1,1.6096848E-1,9.132527E-2,5.5636406E-2,6.887397E-2,0E0,0E0,0E0,7.724348E-2,6.5675065E-2,5.5182934E-2,0E0,0E0,0E0,0E0,0E0,8.803871E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,43,43,44,44,47,47,50,50,51,51,52,52,55,55,57,57,58,58,59,59,60,60,61,61,62,62,66,66,67,67,68,68,74,74],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,34,36,-1,38,-1,40,42,-1,44,46,48,50,52,54,56,58,60,-1,62,64,66,68,70,-1,-1,72,74,-1,-1,76,-1,-1,78,80,82,-1,-1,84,-1,86,88,90,92,94,96,-1,-1,-1,98,100,102,-1,-1,-1,-1,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.9016544E5,1.0910448E1,7.5935423E-1,8.138889E0,3.662857E0,3.3880932E9,2.1298597E0,1.2253E4,1.0722478E5,-1.5808992E-2,-1.6837202E-3,1.388E1,3.49E2,2E1,2.1412018E6,2.2E1,1.783854E8,1.4338E4,3.64E2,1.6E1,5.409836E-3,1.2083E4,-6.892807E-3,1.7103828E7,1.72797E5,4.622428E-3,1.2317392E7,3.6865557E3,9.374418E0,3.89E2,9.4E1,2.059E3,2.3653846E0,4.442623E0,1.34214E5,3.8491169E-3,1.884E3,1.1015E4,2.5084135E6,2.1216373E1,2E0,-3.730315E-3,1.5302832E-3,1.5382514E1,1.172766E3,-2.1216092E-3,4.545801E-3,1.00011E5,3.8851257E-3,6.995949E-4,5E0,3.2608695E0,2.9036145E0,4.2906883E-3,-1.1359965E-3,8.8E1,-4.747218E-3,1.2727593E0,2.7E1,2.8655008E2,9.0754684E1,1.2915888E1,3.4151352E2,9.332241E-5,-6.203178E-3,-4.9205823E-3,6.8804877E2,1.992126E0,1.2820834E7,2.6980403E-3,-3.7598237E-3,-1.3378036E-2,-3.229601E-3,-8.040162E-3,1.8461539E0,-4.751271E-3,-1.7050448E-3,-5.3496327E-4,5.741188E-3,-1.6268429E-3,-7.78488E-3,3.3281483E-3,-1.4749998E-3,-2.7998225E-3,3.8620674E-3,8.8435126E-4,-3.4497385E-3,1.66105E-3,-5.63325E-3,-1.6587214E-3,2.7815397E-3,-2.0625112E-3,1.1453169E-3,-3.3202164E-3,-3.03656E-4,-1.26648415E-2,-4.1876268E-3,1.1199496E-3,4.773892E-3,3.066202E-3,-1.5810982E-3,6.536712E-3,2.9217622E-3,3.3632622E-3,-1.0166428E-3],"split_indices":[43,68,53,73,73,46,53,9,62,0,0,71,8,3,60,5,46,12,10,10,0,9,0,62,1,0,62,60,68,2,0,44,69,69,1,0,44,9,60,73,8,0,0,71,4,0,0,5,0,0,3,69,68,0,0,10,0,57,0,4,71,73,4,0,0,0,67,68,9,0,0,0,0,0,68,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.13E2,4.93E2,4.2E2,4.81E2,1.2E1,2.5E2,1.7E2,1.63E2,3.18E2,4E0,8E0,8E1,1.7E2,1.03E2,6.7E1,6.5E1,9.8E1,6.3E1,2.55E2,7.4E1,6E0,1.65E2,5E0,9.5E1,8E0,1.9E1,4.8E1,6E0,5.9E1,6.1E1,3.7E1,1.2E1,5.1E1,8.2E1,1.73E2,8E0,6.6E1,1.5E1,1.5E2,8.9E1,6E0,4E0,4E0,8E0,4E1,2E0,4E0,5.7E1,2E0,2.5E1,3.6E1,1.4E1,2.3E1,5E0,7E0,7E0,4.4E1,4.5E1,3.7E1,8E1,9.3E1,5.5E1,1.1E1,1E1,5E0,4E0,1.46E2,3.4E1,5.5E1,2E0,4E0,3E0,5E0,2E0,3.8E1,1.5E1,4.2E1,3E0,3.3E1,8E0,6E0,1.5E1,8E0,4E0,3E0,3.3E1,1.2E1,6E0,3.1E1,2.5E1,5.5E1,6.7E1,2.6E1,3.3E1,2.2E1,3E0,8E0,1.33E2,1.3E1,2.6E1,8E0,4E1,1.5E1,1.9E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-4.6683066E-3,-3.06327E-2,1.1457478E-2,-2.4778038E-2,-1.20339446E-1,7.1236175E-3,4.8248717E-3,-1.9393314E-2,-1.2823156E-1,-3.4977472E-1,-2.6780004E-2,8.084914E-2,2.7384607E-3,-6.747885E-2,-1.0177838E-2,3.1313755E-2,-2.2709012E-1,7.299852E-4,-2.494825E-2,-3.8792724E-3,1.8732353E-3,1.5023117E-1,1.8955167E-2,2.5169734E-2,-1.1883667E-2,4.177215E-3,-7.5904086E-2,2.619346E-2,-2.1765765E-2,4.612003E-3,-2.063431E-3,-1.462106E-2,-8.1667124E-4,9.196558E-3,1.170169E-3,2.8841917E-3,-3.723968E-2,1.063148E-1,1.1341963E-2,-9.776521E-2,-2.229474E-3,-6.0384914E-2,-7.3563685E-3,4.8598014E-2,-2.5526457E-2,-9.934543E-3,-1.8749086E-2,-4.0104333E-3,3.4023952E-3,1.2258936E-1,-3.5157143E-3,-2.3099822E-3,7.603817E-2,-2.4099582E-1,5.380225E-3,1.2345999E-2,-3.4202874E-2,-3.714481E-3,4.6269293E-4,4.3552124E-4,3.7024783E-3,-8.275491E-3,6.01668E-5,3.5054935E-3,-1.3291963E-3,7.6842913E-4,6.7875194E-3,-2.044206E-3,9.239104E-4,5.453667E-3,9.5558434E-4,1.0631267E-3,-1.9010559E-2,5.6195315E-3,-1.5562304E-3,8.108777E-4,-6.8590688E-3,-2.3089447E-3,2.3194037E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,29,31,-1,-1,-1,-1,33,35,37,39,-1,41,43,45,-1,-1,-1,-1,-1,-1,-1,47,49,51,53,55,57,-1,59,61,-1,63,-1,-1,65,-1,67,69,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7927106E-1,1.7926356E-1,2.2283635E-1,1.7967469E-1,4.4141552E-1,1.7274302E-1,0E0,1.3757011E-1,2.5946978E-1,4.1773129E-1,6.412138E-2,1.2598221E-1,1.6626355E-1,7.3080674E-2,1.11416176E-1,4.034623E-2,1.7737848E-1,0E0,0E0,0E0,0E0,7.821751E-2,4.3445535E-2,2.2320527E-1,2.5432092E-1,0E0,5.1734418E-2,7.540128E-2,1.1174222E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.957745E-2,9.5121324E-2,1.5209103E-1,4.592777E-1,1.2951186E-1,5.834882E-2,0E0,5.4006755E-2,8.198453E-2,0E0,1.8537202E-1,0E0,0E0,6.4472795E-2,0E0,1.3293293E-1,6.622097E-2,5.8480346E-1,8.924546E-2,1.5793686E-1,1.1543895E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,21,21,22,22,23,23,24,24,26,26,27,27,28,28,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,46,46,49,49,51,51,52,52,53,53,54,54,55,55,56,56],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,30,32,-1,-1,-1,-1,34,36,38,40,-1,42,44,46,-1,-1,-1,-1,-1,-1,-1,48,50,52,54,56,58,-1,60,62,-1,64,-1,-1,66,-1,68,70,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,2.4240502E8,7.82261E6,6.3344407E0,5.1942E4,1.1428572E0,4.8248717E-3,1.63757E4,2.95E2,1.2445087E2,6.228E1,2.5818555E2,3.72381E5,7E0,1.9913E4,1.1634076E4,2.6235E4,7.299852E-4,-2.494825E-2,-3.8792724E-3,1.8732353E-3,2.9849844E7,1E0,1.6869704E6,1E0,4.177215E-3,1.1146532E8,8E0,7.7E1,4.612003E-3,-2.063431E-3,-1.462106E-2,-8.1667124E-4,9.196558E-3,1.170169E-3,2.8841917E-3,4.407599E-1,6.56E2,1.53053E5,1.048796E6,4.29E2,7.31E2,-7.3563685E-3,3.9125E1,8.4E1,-9.934543E-3,3.421543E2,-4.0104333E-3,3.4023952E-3,1.5E1,-3.5157143E-3,4.356E3,9.07826E0,1E0,2.3316922E0,1E0,2.0569372E-3,-3.714481E-3,4.6269293E-4,4.3552124E-4,3.7024783E-3,-8.275491E-3,6.01668E-5,3.5054935E-3,-1.3291963E-3,7.6842913E-4,6.7875194E-3,-2.044206E-3,9.239104E-4,5.453667E-3,9.5558434E-4,1.0631267E-3,-1.9010559E-2,5.6195315E-3,-1.5562304E-3,8.108777E-4,-6.8590688E-3,-2.3089447E-3,2.3194037E-3],"split_indices":[2,5,43,68,1,71,0,43,2,4,62,4,9,3,9,43,1,0,0,0,0,5,26,60,8,0,46,32,44,0,0,0,0,0,0,0,53,8,9,9,0,1,0,4,2,0,48,0,0,0,0,44,71,29,69,79,49,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.04E2,3.46E2,5.58E2,3.26E2,2E1,5.34E2,2.4E1,3.11E2,1.5E1,5E0,1.5E1,2.9E1,5.05E2,4.9E1,2.62E2,6E0,9E0,2E0,3E0,8E0,7E0,1.3E1,1.6E1,1.99E2,3.06E2,2E0,4.7E1,6.3E1,1.99E2,3E0,3E0,6E0,3E0,9E0,4E0,9E0,7E0,2.8E1,1.71E2,3E1,2.76E2,4.1E1,6E0,4.4E1,1.9E1,2E0,1.97E2,5E0,2E0,2.6E1,2E0,1.42E2,2.9E1,1.2E1,1.8E1,1.9E2,8.6E1,3.2E1,9E0,2E1,2.4E1,2E0,1.7E1,1.8E1,1.79E2,5E0,2.1E1,4.9E1,9.3E1,1.6E1,1.3E1,5E0,7E0,4E0,1.4E1,1.85E2,5E0,7.3E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[7.5126864E-4,-1.522578E-2,2.2593955E-2,-5.595435E-2,-8.290375E-3,2.8156836E-2,-5.011895E-2,-1.7210929E-2,-2.3737255E-1,2.067088E-2,-2.9112332E-2,2.3870347E-2,1.11846045E-1,-4.7779367E-3,-5.700653E-4,-6.2796385E-3,3.398411E-3,1.7792141E-3,-3.1602195E-1,2.9460486E-2,-6.8861865E-2,-1.3374001E-1,-2.4659438E-2,3.7296694E-2,5.6370273E-3,7.2059785E-3,-1.6961496E-4,1.8469775E-2,-6.0598613E-3,-1.9653358E-2,-7.291902E-3,-7.4680904E-3,5.0217602E-2,-5.3572077E-3,1.6031438E-4,1.37585E-2,-2.4726007E-2,-6.2022705E-2,-1.0399449E-2,4.1564975E-2,-8.998229E-2,-1.2197926E-2,5.306269E-2,-4.0283613E-3,3.0094383E-3,2.9259454E-2,-3.298594E-3,-2.2618022E-2,7.963333E-2,5.787069E-2,-7.721913E-2,4.2355293E-3,-4.9135764E-3,-3.8767144E-2,-1.3634884E-1,-2.8800282E-3,-1.1429296E-1,2.4577687E-2,8.174515E-2,2.660065E-3,-9.983567E-3,2.4589838E-3,-7.226978E-2,1.6303544E-1,2.6409982E-2,2.5918052E-4,4.3067113E-3,-2.0848121E-4,-4.197391E-3,-2.823204E-3,5.688143E-3,1.5383522E-3,4.3860725E-3,-5.7053156E-3,1.1292193E-3,-3.394318E-3,-1.494424E-4,9.227679E-4,-7.963742E-3,-4.67558E-3,2.4532393E-4,-1.2604322E-3,-9.836315E-3,1.8316436E-3,-4.8091654E-3,4.765401E-3,-1.3965715E-3,-2.760385E-4,6.8844478E-3,1.6272729E-3,-3.9412924E-3,-5.089386E-4,1.0364314E-2,4.7886157E-3,-3.8742094E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,-1,27,-1,29,31,33,35,37,39,41,-1,43,45,-1,-1,-1,47,49,-1,-1,51,-1,53,55,57,59,61,63,-1,-1,65,-1,67,69,71,73,-1,-1,75,77,79,81,83,85,-1,-1,87,89,91,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3292353E-1,1.5517661E-1,1.6442195E-1,5.582156E-1,2.8523195E-1,1.3185498E-1,5.6511797E-2,1.631382E-1,3.155827E-1,1.5784629E-1,1.2482341E-1,8.7430626E-2,8.691998E-2,0E0,0E0,0E0,1.1927618E-1,0E0,1.2039268E-1,1.3911708E-1,6.069594E-2,6.529261E-1,1.4054693E-1,1.15652174E-1,1.3005991E-1,0E0,3.9256297E-2,5.319647E-2,0E0,0E0,0E0,8.889347E-2,1.1760172E-1,0E0,0E0,9.270941E-2,0E0,1.2089437E-1,1.504461E-1,1.3450137E-1,1.4073294E-1,9.879011E-2,1.1879923E-1,0E0,0E0,7.1502045E-2,0E0,6.950921E-2,6.730984E-2,9.6256495E-2,3.52748E-2,0E0,0E0,6.9108725E-2,9.602934E-2,1.4467171E-1,1.0445002E-1,2.7315375E-1,1.3971788E-1,0E0,0E0,1.1207683E-1,3.2403387E-2,8.569333E-2,9.39407E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,31,31,32,32,35,35,37,37,38,38,39,39,40,40,41,41,42,42,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,64,64],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,-1,28,-1,30,32,34,36,38,40,42,-1,44,46,-1,-1,-1,48,50,-1,-1,52,-1,54,56,58,60,62,64,-1,-1,66,-1,68,70,72,74,-1,-1,76,78,80,82,84,86,-1,-1,88,90,92,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,5.8182236E1,7.042942E7,2.4332604E0,9.970589E0,5E1,4.4522205E6,6.25E0,4.64E2,1E0,4E0,8E0,5.3700186E8,-4.7779367E-3,-5.700653E-4,-6.2796385E-3,8E0,1.7792141E-3,6.845E3,1E0,8.4E1,7.6825094E0,1.2915888E1,5.489183E1,1.6326721E0,7.2059785E-3,2.101E3,4.1361522E6,-6.0598613E-3,-1.9653358E-2,-7.291902E-3,5.6516E4,1.617695E8,-5.3572077E-3,1.6031438E-4,5.603014E6,-2.4726007E-2,1.1E1,3.3343E4,1.3758875E3,1.934E3,1.1521033E0,3.825498E1,-4.0283613E-3,3.0094383E-3,2.135961E4,-3.298594E-3,4.1665E3,2E0,1.7143776E7,4.306E4,4.2355293E-3,-4.9135764E-3,5.7E1,1.6896813E0,1.6262903E2,2.1650919E1,3.1773497E5,2.395631E7,2.660065E-3,-9.983567E-3,4.435876E2,2.2734E4,7.2140925E5,4.0663E4,2.5918052E-4,4.3067113E-3,-2.0848121E-4,-4.197391E-3,-2.823204E-3,5.688143E-3,1.5383522E-3,4.3860725E-3,-5.7053156E-3,1.1292193E-3,-3.394318E-3,-1.494424E-4,9.227679E-4,-7.963742E-3,-4.67558E-3,2.4532393E-4,-1.2604322E-3,-9.836315E-3,1.8316436E-3,-4.8091654E-3,4.765401E-3,-1.3965715E-3,-2.760385E-4,6.8844478E-3,1.6272729E-3,-3.9412924E-3,-5.089386E-4,1.0364314E-2,4.7886157E-3,-3.8742094E-4],"split_indices":[4,62,60,71,73,3,62,67,44,19,3,32,7,0,0,0,32,0,44,29,10,71,73,71,53,0,0,60,0,0,0,9,7,0,0,9,0,8,44,70,44,57,73,0,0,43,0,62,8,5,1,0,0,0,68,4,73,48,62,0,0,71,9,60,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.52E2,5.5E2,4.02E2,7.9E1,4.71E2,3.74E2,2.8E1,6.6E1,1.3E1,1.97E2,2.74E2,3.57E2,1.7E1,1.1E1,1.7E1,9E0,5.7E1,3E0,1E1,1.8E2,1.7E1,1E1,2.64E2,2.05E2,1.52E2,1.2E1,5E0,5.2E1,5E0,5E0,5E0,6.5E1,1.15E2,1E1,7E0,8E0,2E0,7.2E1,1.92E2,1.99E2,6E0,1.11E2,4.1E1,2E0,3E0,4.7E1,5E0,5.6E1,9E0,1.09E2,6E0,5E0,3E0,5.6E1,1.6E1,1.8E2,1.2E1,1.41E2,5.8E1,3E0,3E0,9E1,2.1E1,7E0,3.4E1,3.5E1,1.2E1,4.5E1,1.1E1,2E0,7E0,6.6E1,4.3E1,4E0,2E0,2.8E1,2.8E1,3E0,1.3E1,1.3E1,1.67E2,7E0,5E0,1.27E2,1.4E1,4.9E1,9E0,8.6E1,4E0,2E0,1.9E1,2E0,5E0,1E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[2.6494614E-3,-2.6590537E-2,1.7728778E-2,6.3995956E-3,-5.940912E-2,4.233521E-2,3.84149E-3,-1.159681E-2,4.799978E-2,5.896809E-3,-6.6461116E-2,5.1682338E-2,-5.7299882E-2,1.6842602E-2,-3.325009E-2,-3.0230868E-4,-6.1842225E-2,9.0490684E-2,2.5254337E-2,-9.4100595E-2,-1.9714251E-2,-7.1252063E-3,5.6696545E-2,-2.2241335E-1,1.6749726E-3,1.6508668E-3,6.529783E-2,-5.41545E-2,1.4710026E-3,-3.263073E-2,1.8986614E-2,-5.5008125E-3,-2.2859244E-4,7.4984727E-4,5.71245E-3,3.8006846E-4,3.957542E-3,-1.5002139E-1,-5.516917E-2,1.959969E-2,-8.315005E-2,4.5945004E-2,1.2149514E-1,-1.5844787E-2,2.2722008E-3,-9.610774E-2,1.1467557E-2,1.0676899E-1,2.974773E-2,-6.7421064E-2,4.011162E-3,-5.5523943E-2,1.2959511E-3,5.492644E-2,-2.921342E-2,-1.1103853E-2,-2.4013701E-1,-1.04661705E-2,-4.490029E-2,4.989743E-4,5.349306E-3,-1.0155041E-1,3.1269698E-3,7.5308025E-2,2.3676116E-2,-1.0026041E-3,1.4027502E-1,2.6958599E-3,-1.26707E-1,-5.366362E-3,8.5317865E-2,7.8066057E-3,4.6587583E-2,6.253029E-2,-3.1763022E-3,-5.5535622E-2,-1.6645637E-1,-9.0684195E-4,-5.004762E-3,3.3967986E-3,-1.8020021E-3,2.3894666E-3,-2.668021E-3,-3.7851334E-3,8.8858785E-4,-1.4260935E-2,-3.5866774E-3,-2.7939775E-3,1.3671768E-3,7.5840484E-4,-3.9515086E-3,-1.5344636E-3,-7.7578165E-3,4.1285255E-3,-8.173987E-3,-6.211082E-5,2.90882E-3,7.1869465E-3,1.4116338E-3,-8.115828E-3,-2.377491E-3,6.4926106E-5,-8.489883E-3,4.481606E-3,-3.3476418E-3,5.6831475E-4,7.219269E-3,-8.821165E-4,4.6578497E-3,-5.1246965E-3,-1.3957111E-3,-1.4803192E-3,-9.949883E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,-1,45,47,49,-1,51,53,-1,-1,-1,-1,-1,-1,55,57,59,61,63,65,-1,-1,67,69,71,73,75,-1,77,-1,79,81,83,85,-1,87,89,-1,91,-1,93,95,-1,97,-1,99,101,103,-1,105,107,-1,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.0038508E-1,3.3493122E-1,2.0452562E-1,1.1659798E-1,2.1195632E-1,2.041226E-1,1.8573703E-1,6.1838478E-2,4.313901E-2,0E0,1.9195372E-1,2.1210122E-1,3.007537E-1,2.096861E-1,1.3733941E-1,5.674178E-2,6.3672826E-2,3.6497638E-2,3.2014996E-2,1.9636858E-1,1.4378265E-1,0E0,1.2957472E-1,2.4124435E-1,0E0,2.1030931E-1,9.7076505E-2,1.4737648E-1,0E0,4.8635505E-2,1.0034573E-1,0E0,0E0,0E0,0E0,0E0,0E0,4.7304183E-1,8.932671E-2,6.555613E-2,6.7705154E-2,1.0806912E-1,7.556E-2,0E0,0E0,1.0163559E-1,2.4785267E-1,1.06146574E-1,1.2640786E-1,7.315123E-2,0E0,4.4813477E-2,0E0,5.8815055E-2,5.9671775E-2,3.5527475E-2,2.2568536E-1,0E0,6.3778184E-2,4.3267302E-2,0E0,8.397257E-2,0E0,2.589678E-1,9.4384566E-2,0E0,3.292349E-2,0E0,5.2498817E-2,1.9555733E-1,7.121903E-2,0E0,6.157639E-2,9.08169E-2,0E0,8.5978255E-2,3.7939355E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,25,25,26,26,27,27,29,29,30,30,37,37,38,38,39,39,40,40,41,41,42,42,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,55,55,56,56,58,58,59,59,61,61,63,63,64,64,66,66,68,68,69,69,70,70,72,72,73,73,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,-1,46,48,50,-1,52,54,-1,-1,-1,-1,-1,-1,56,58,60,62,64,66,-1,-1,68,70,72,74,76,-1,78,-1,80,82,84,86,-1,88,90,-1,92,-1,94,96,-1,98,-1,100,102,104,-1,106,108,-1,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.1604336E7,1E0,1.9130435E0,5.3E1,9E0,3.1083834E5,4.7420635E1,3.6E1,1.38E3,5.896809E-3,7.714286E0,3.2E1,1.4598765E0,5.8222644E2,1.9616238E2,3.07E2,2.7015875E2,1.9452055E0,1.1358E4,1E0,1.2238068E7,-7.1252063E-3,2.0094643E0,4.44E2,1.6749726E-3,2.0608695E0,4.2222223E0,1E0,1.4710026E-3,3.421543E2,1.783854E8,-5.5008125E-3,-2.2859244E-4,7.4984727E-4,5.71245E-3,3.8006846E-4,3.957542E-3,1.1392951E2,2.1557376E0,6.2369446E2,7.153514E1,1.5321098E8,4E0,-1.5844787E-2,2.2722008E-3,3.011015E8,9.751103E6,4.9469455E6,8.954333E6,1.7101741E2,4.011162E-3,3.959E3,1.2959511E-3,1E0,1.6834415E0,4.204547E5,8.39E2,-1.04661705E-2,2.2318378E6,4.126333E7,5.349306E-3,8E0,3.1269698E-3,2.62E3,5.467E3,-1.0026041E-3,2.395631E7,2.6958599E-3,4.05E2,9.750042E6,3.222E3,7.8066057E-3,2.7959185E0,7.449545E2,-3.1763022E-3,4.37E2,2E0,-9.0684195E-4,-5.004762E-3,3.3967986E-3,-1.8020021E-3,2.3894666E-3,-2.668021E-3,-3.7851334E-3,8.8858785E-4,-1.4260935E-2,-3.5866774E-3,-2.7939775E-3,1.3671768E-3,7.5840484E-4,-3.9515086E-3,-1.5344636E-3,-7.7578165E-3,4.1285255E-3,-8.173987E-3,-6.211082E-5,2.90882E-3,7.1869465E-3,1.4116338E-3,-8.115828E-3,-2.377491E-3,6.4926106E-5,-8.489883E-3,4.481606E-3,-3.3476418E-3,5.6831475E-4,7.219269E-3,-8.821165E-4,4.6578497E-3,-5.1246965E-3,-1.3957111E-3,-1.4803192E-3,-9.949883E-3],"split_indices":[7,8,68,0,3,48,73,0,44,0,71,2,69,67,71,1,4,68,44,29,60,0,57,0,0,68,69,30,0,48,46,0,0,0,0,0,0,67,69,67,71,7,3,0,0,46,9,62,60,73,0,9,0,26,68,43,2,0,43,7,0,8,0,2,2,0,62,0,10,9,2,0,68,67,0,0,32,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.06E2,3.08E2,5.98E2,1.54E2,1.54E2,2.15E2,3.83E2,1.08E2,4.6E1,5E0,1.49E2,1.97E2,1.8E1,2.84E2,9.9E1,8.9E1,1.9E1,1.5E1,3.1E1,9.3E1,5.6E1,4E0,1.93E2,6E0,1.2E1,2.17E2,6.7E1,7.5E1,2.4E1,3.3E1,5.6E1,9E0,1E1,5E0,1E1,2.5E1,6E0,3.7E1,5.6E1,3.5E1,2.1E1,1.67E2,2.6E1,4E0,2E0,1.9E1,1.98E2,3E1,3.7E1,6.9E1,6E0,2.4E1,9E0,3.2E1,2.4E1,1.5E1,2.2E1,2E0,5.4E1,3E1,5E0,1.9E1,2E0,7.1E1,9.6E1,3E0,2.3E1,3E0,1.6E1,1.62E2,3.6E1,1.4E1,1.6E1,2.8E1,9E0,6.3E1,6E0,1.5E1,9E0,2.7E1,5E0,6E0,1.8E1,4E0,1.1E1,1.5E1,7E0,4.5E1,9E0,2.6E1,4E0,1E1,9E0,6.8E1,3E0,5.9E1,3.7E1,2E1,3E0,9E0,7E0,1.57E2,5E0,3.4E1,2E0,1.3E1,3E0,9E0,1.9E1,1.9E1,4.4E1,2E0,4E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-1.4260991E-3,-2.0534394E-2,1.8585112E-2,-1.5726674E-2,-1.17066614E-1,1.0502288E-1,1.4280468E-2,-1.8996239E-2,6.307165E-2,3.1240625E-2,-2.2214398E-1,8.275679E-3,2.9761193E-3,-2.9320609E-2,2.4137378E-2,-1.4006573E-2,-7.4765444E-2,1.1647237E-1,-2.9365744E-3,-1.5215322E-3,4.73169E-3,-1.3294901E-2,-2.9307613E-3,-5.581608E-2,3.3723854E-2,1.230273E-1,1.8370572E-2,-1.9650655E-2,3.8405715E-3,-1.0053298E-1,5.043487E-4,7.705661E-3,2.1361357E-4,-2.7728297E-2,-1.3809885E-1,6.7704287E-3,-1.1648102E-2,2.3264308E-3,7.979496E-3,2.7529377E-2,-3.7362862E-2,-2.767819E-2,2.0416673E-2,-1.7330804E-1,-4.752953E-2,2.8808478E-3,-4.394384E-2,-1.1839889E-3,-7.743731E-3,-2.2041698E-3,1.8118522E-3,-1.6929258E-2,3.6643546E-2,3.2417286E-2,-7.442414E-2,-1.1022656E-3,-1.2424561E-2,1.8570793E-3,-2.0784333E-3,-9.400989E-3,-1.3508683E-3,1.6060112E-4,-5.6206896E-3,-1.5053404E-3,-6.109031E-3,1.1282307E-3,-3.3951565E-3,3.9392263E-3,1.1126047E-3,-1.2890939E-3,3.594279E-3,1.8048367E-3,-4.3082987E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,-1,-1,-1,-1,33,35,37,39,41,-1,43,-1,-1,-1,45,47,-1,49,-1,-1,51,53,55,57,59,61,-1,63,-1,-1,-1,-1,65,67,69,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.4911233E-1,2.1489358E-1,1.644496E-1,1.1610387E-1,3.4978208E-1,5.1444024E-2,1.8379056E-1,1.1833052E-1,1.2867233E-1,4.8417516E-2,1.1380285E-1,0E0,0E0,1.3384336E-1,1.96197E-1,2.1695286E-1,7.829869E-2,7.081702E-2,0E0,0E0,0E0,0E0,0E0,1.2559463E-1,1.23495504E-1,5.8756143E-2,1.6947763E-1,1.20859146E-1,0E0,9.591067E-2,0E0,0E0,0E0,6.524743E-2,4.05564E-2,0E0,3.440761E-2,0E0,0E0,1.1569813E-1,1.239613E-1,2.9084346E-1,8.182579E-2,3.987971E-2,6.396934E-2,0E0,3.31885E-2,0E0,0E0,0E0,0E0,1.14939034E-1,1.4224279E-1,4.8009057E-2,6.752354E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,23,23,24,24,25,25,26,26,27,27,29,29,33,33,34,34,36,36,39,39,40,40,41,41,42,42,43,43,44,44,46,46,51,51,52,52,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,-1,-1,-1,-1,34,36,38,40,42,-1,44,-1,-1,-1,46,48,-1,50,-1,-1,52,54,56,58,60,62,-1,64,-1,-1,-1,-1,66,68,70,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.188E3,9.932432E0,2.9E1,1.5570383E5,1.3027534E2,1.1941613E3,3.257507E5,9.914298E5,4.4889745E6,1.317484E1,2.6235E4,8.275679E-3,2.9761193E-3,1.33E2,1.2354949E0,6.356704E5,7.694314E-7,3E0,-2.9365744E-3,-1.5215322E-3,4.73169E-3,-1.3294901E-2,-2.9307613E-3,4.826902E6,3.047619E0,1.839E3,2E1,2.331083E6,3.8405715E-3,3E0,5.043487E-4,7.705661E-3,2.1361357E-4,3.9682608E2,6E0,6.7704287E-3,2.147382E4,2.3264308E-3,7.979496E-3,3.1657144E2,3.0957446E0,2.2681375E6,8E0,4.1228E2,8.39E2,2.8808478E-3,5.119403E0,-1.1839889E-3,-7.743731E-3,-2.2041698E-3,1.8118522E-3,1.2090909E1,1.4327235E7,1.5994397E0,6E0,-1.1022656E-3,-1.2424561E-2,1.8570793E-3,-2.0784333E-3,-9.400989E-3,-1.3508683E-3,1.6060112E-4,-5.6206896E-3,-1.5053404E-3,-6.109031E-3,1.1282307E-3,-3.3951565E-3,3.9392263E-3,1.1126047E-3,-1.2890939E-3,3.594279E-3,1.8048367E-3,-4.3082987E-3],"split_indices":[2,69,0,48,67,4,43,66,60,69,1,0,0,8,68,66,52,8,0,0,0,0,0,9,68,2,3,43,0,8,0,0,0,4,3,0,48,0,0,67,69,43,8,4,2,0,68,0,0,0,0,73,12,69,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.11E2,4.66E2,4.45E2,4.45E2,2.1E1,2E1,4.25E2,4.28E2,1.7E1,9E0,1.2E1,6E0,1.4E1,7.8E1,3.47E2,3.94E2,3.4E1,1.2E1,5E0,5E0,4E0,8E0,4E0,5.5E1,2.3E1,1.8E1,3.29E2,3.73E2,2.1E1,2.6E1,8E0,8E0,4E0,4.2E1,1.3E1,6E0,1.7E1,8E0,1E1,2.83E2,4.6E1,3.11E2,6.2E1,1E1,1.6E1,6E0,3.6E1,3E0,1E1,1E1,7E0,4.8E1,2.35E2,1.6E1,3E1,3.07E2,4E0,4.8E1,1.4E1,8E0,2E0,1E1,6E0,3.3E1,3E0,2.8E1,2E1,4.8E1,1.87E2,7E0,9E0,4E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[-1.2522186E-3,-6.476154E-2,2.3332422E-3,-1.4642723E-2,-1.2788355E-1,7.9702556E-2,-1.2277678E-3,6.8369426E-4,-3.789612E-3,-4.2765224E-3,-1.2398063E-2,9.747281E-2,-2.819742E-3,-5.7994775E-3,5.047022E-2,8.637361E-3,3.3862523E-3,-3.1687055E-2,4.3628514E-3,-1.3974101E-3,6.625877E-2,-2.3454374E-2,-1.17545724E-1,-8.607753E-3,3.554447E-2,7.692322E-2,-4.508542E-3,9.546083E-3,-3.9181203E-2,-2.1253069E-1,-5.735799E-2,-4.57255E-3,-1.8206179E-1,6.2197987E-2,2.5315158E-4,1.00301474E-1,7.894432E-3,-3.620931E-4,3.4637596E-3,-1.0637822E-2,-1.4771082E-3,-2.4068134E-3,-1.2393913E-2,1.95264E-3,-4.2489115E-3,-2.62404E-5,-5.204118E-3,-2.3161435E-2,-9.6013764E-4,3.319256E-3,-1.5049257E-3,3.7369732E-4,-6.234356E-3,6.2638605E-3,3.140981E-3,5.3259367E-3,-1.92022E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,9,11,13,-1,-1,-1,-1,15,-1,17,19,-1,-1,21,23,-1,25,27,29,31,33,35,-1,37,39,41,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1306016E-1,1.5630743E-1,2.4420227E-1,5.8076877E-2,8.512315E-2,1.0316566E-1,2.0071599E-1,0E0,0E0,0E0,0E0,6.264964E-2,0E0,2.0533913E-1,8.970708E-2,0E0,0E0,1.527494E-1,2.271993E-1,0E0,1.0840303E-1,1.0521051E-1,9.6431315E-2,2.770367E-1,1.551895E-1,8.796567E-2,0E0,7.485446E-2,1.8543817E-1,4.6940356E-2,4.7747575E-2,1.6728356E-1,4.487196E-1,8.3835304E-2,7.63283E-2,3.7092358E-2,8.3388574E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,11,11,13,13,14,14,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36],"right_children":[2,4,6,8,10,12,14,-1,-1,-1,-1,16,-1,18,20,-1,-1,22,24,-1,26,28,30,32,34,36,-1,38,40,42,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.54E2,2.6412E5,1.6E1,7.1E1,3.09627E5,3.0923106E-10,6.714162E8,6.8369426E-4,-3.789612E-3,-4.2765224E-3,-1.2398063E-2,3.5419354E0,-2.819742E-3,1E0,3.532346E-1,8.637361E-3,3.3862523E-3,1.0493796E7,6.965855E9,-1.3974101E-3,5.6E1,1.7754011E0,4.888E3,1.2319699E8,3.94E2,1.2820834E7,-4.508542E-3,4.1221875E-1,1.8181819E0,1.6464853E8,3.1075312E10,2.0734E4,2.1027016E5,2.5345264E7,1.8258025E2,1.7094556E5,1.8029197E0,-3.620931E-4,3.4637596E-3,-1.0637822E-2,-1.4771082E-3,-2.4068134E-3,-1.2393913E-2,1.95264E-3,-4.2489115E-3,-2.62404E-5,-5.204118E-3,-2.3161435E-2,-9.6013764E-4,3.319256E-3,-1.5049257E-3,3.7369732E-4,-6.234356E-3,6.2638605E-3,3.140981E-3,5.3259367E-3,-1.92022E-3],"split_indices":[1,7,0,44,9,52,7,0,0,0,0,69,0,29,53,0,0,66,46,0,3,68,10,5,0,9,0,53,68,5,46,44,48,60,73,48,69,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.34E2,4.9E1,8.85E2,2.8E1,2.1E1,3.8E1,8.47E2,2E1,8E0,1.8E1,3E0,3.4E1,4E0,7.79E2,6.8E1,6E0,2.8E1,2.19E2,5.6E2,1.1E1,5.7E1,2.01E2,1.8E1,3.96E2,1.64E2,5.4E1,3E0,6.5E1,1.36E2,6E0,1.2E1,3.88E2,8E0,9.3E1,7.1E1,4E1,1.4E1,5.2E1,1.3E1,4E0,1.32E2,2E0,4E0,3E0,9E0,3.75E2,1.3E1,2E0,6E0,8.5E1,8E0,6.8E1,3E0,1.8E1,2.2E1,4E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"57","size_leaf_vector":"1"}},{"base_weights":[-4.0936847E-3,-2.213924E-3,-1.7493054E-1,-1.218478E-2,2.3484375E-2,-1.5157524E-3,-2.0644134E-1,-1.4632812E-2,1.1021254E-1,1.213139E-2,6.757139E-2,-2.4597816E-1,-2.7232019E-3,-1.2699752E-2,-1.6696519E-1,6.587053E-3,1.909504E-4,3.690994E-2,-1.5046532E-2,9.071187E-2,2.0668314E-4,-3.3804618E-3,-1.4624424E-2,8.994345E-3,-2.2269376E-2,-2.1404354E-2,3.4864916E-4,4.928753E-2,-1.9668876E-3,-2.4852227E-2,1.7230985E-1,-1.5491749E-2,1.07272774E-1,-1.7965335E-2,4.6693966E-2,-2.5539923E-2,4.147793E-2,7.096778E-3,3.356844E-2,-7.6907566E-3,-9.688451E-2,1.1363447E-2,1.9927423E-3,2.7401838E-3,-5.094546E-3,8.0937944E-2,7.429543E-3,-2.056568E-3,1.3357322E-3,3.0375428E-3,-3.6936372E-3,-1.1016389E-3,-8.153305E-3,9.099498E-3,-1.6992565E-4,-5.165962E-3,2.2579334E-3,2.8307782E-4,-4.679053E-3,-6.9212248E-3,3.1583207E-3,4.5896647E-3,-1.5103748E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,-1,-1,27,29,31,-1,-1,-1,33,35,-1,-1,37,-1,39,41,43,45,47,49,51,53,-1,55,57,59,-1,-1,-1,-1,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9505455E-1,2.3371536E-1,3.8123935E-2,1.9847128E-1,1.2681268E-1,0E0,3.237298E-2,1.8781115E-1,4.2760834E-2,1.3793498E-1,7.555826E-2,4.8339963E-2,0E0,1.3265187E-1,4.1276342E-1,0E0,0E0,1.06979385E-1,1.8432394E-1,6.84973E-2,0E0,0E0,0E0,2.0004731E-1,9.341982E-2,0E0,0E0,1.4589891E-1,0E0,1.1545579E-1,3.615372E-2,4.8444405E-2,3.267473E-2,1.4222905E-1,1.9773182E-1,1.12939745E-1,1.5379798E-1,0E0,1.830867E-1,9.918541E-2,1.6416033E-1,0E0,0E0,0E0,0E0,5.1514983E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,17,17,18,18,19,19,23,23,24,24,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,38,38,39,39,40,40,45,45],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,-1,-1,28,30,32,-1,-1,-1,34,36,-1,-1,38,-1,40,42,44,46,48,50,52,54,-1,56,58,60,-1,-1,-1,-1,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8652172E1,1.8062708E6,8.612209E-6,6.5026445E3,2.6095572E0,-1.5157524E-3,5.4E1,1.7587205E6,5.619497E7,2.0588236E0,5.8475E5,4.5E1,-2.7232019E-3,1.9333333E0,8E0,6.587053E-3,1.909504E-4,1E0,3.9960475E0,1.9791039E6,2.0668314E-4,-3.3804618E-3,-1.4624424E-2,2.0778275E-1,1.9616238E2,-2.1404354E-2,3.4864916E-4,2.2681375E6,-1.9668876E-3,1.388E1,7.13E2,1.8953742E6,9E0,8.39777E5,1.7412816E0,1.4665158E9,7.958E3,7.096778E-3,2.8530578E6,1.6E1,3.2645503E5,1.1363447E-2,1.9927423E-3,2.7401838E-3,-5.094546E-3,2.0130434E1,7.429543E-3,-2.056568E-3,1.3357322E-3,3.0375428E-3,-3.6936372E-3,-1.1016389E-3,-8.153305E-3,9.099498E-3,-1.6992565E-4,-5.165962E-3,2.2579334E-3,2.8307782E-4,-4.679053E-3,-6.9212248E-3,3.1583207E-3,4.5896647E-3,-1.5103748E-3],"split_indices":[68,43,53,4,68,0,0,43,62,69,1,3,0,68,3,0,0,6,69,43,0,0,0,53,71,0,0,43,0,71,2,43,8,9,53,12,2,0,43,8,48,0,0,0,0,71,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.19E2,9.1E2,9E0,6.56E2,2.54E2,2E0,7E0,6.44E2,1.2E1,2.03E2,5.1E1,5E0,2E0,6.37E2,7E0,9E0,3E0,1.06E2,9.7E1,3.7E1,1.4E1,2E0,3E0,1.95E2,4.42E2,2E0,5E0,9.2E1,1.4E1,9.3E1,4E0,5E0,3.2E1,1.14E2,8.1E1,4.21E2,2.1E1,1.1E1,8.1E1,7.6E1,1.7E1,2E0,2E0,3E0,2E0,2.3E1,9E0,7.3E1,4.1E1,7.1E1,1E1,4.17E2,4E0,4E0,1.7E1,7E0,7.4E1,6.7E1,9E0,1.3E1,4E0,2E1,3E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"63","size_leaf_vector":"1"}},{"base_weights":[-2.620948E-3,-1.577485E-2,1.402735E-2,-1.3364302E-2,-2.5222835E-1,2.954692E-2,-1.552824E-2,-8.439204E-3,-9.255419E-2,1.127045E-3,-2.0677926E-2,3.8012404E-2,-5.5280842E-2,-2.2627195E-2,6.6615967E-3,-1.0800211E-2,1.2896384E-1,-1.5797004E-1,-2.096806E-2,3.3827953E-2,8.998087E-3,-3.598429E-3,2.053023E-3,-1.7765282E-2,-8.081487E-3,7.664245E-3,-2.9297845E-2,8.180957E-4,8.941997E-3,-2.4958227E-3,-2.4252896E-1,4.8092487E-3,-2.2350268E-3,3.7409622E-2,-1.073165E-1,-3.3762988E-2,9.934242E-3,-2.3409447E-2,2.4487944E-2,-7.073577E-2,-1.26460325E-2,-1.543872E-2,-5.6961183E-3,5.7745542E-2,5.228694E-3,2.1256862E-4,-7.645972E-3,5.9308605E-3,-3.9389234E-2,9.7824454E-2,-8.7616476E-4,-2.350139E-3,5.8838265E-4,-2.726804E-3,1.4030855E-3,-3.856925E-3,1.3043935E-3,-4.2733512E-4,-9.182556E-3,1.5102379E-3,4.079536E-3,-1.7800365E-3,1.9311015E-3,-3.4142241E-3,-3.9096057E-4,7.0183324E-3,9.1110193E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,-1,25,27,29,31,33,-1,-1,-1,35,-1,37,39,-1,-1,-1,41,-1,-1,43,45,47,49,51,53,55,57,-1,-1,59,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9643638E-1,2.821904E-1,1.8210009E-1,1.9305785E-1,2.7671137E-1,1.888094E-1,1.5764908E-1,1.5354092E-1,1.3249084E-1,0E0,0E0,1.4763036E-1,5.613161E-2,9.391043E-2,0E0,1.5802006E-1,5.3022474E-2,1.19283706E-1,5.5508178E-2,1.2129825E-1,0E0,0E0,0E0,5.749903E-2,0E0,1.2185693E-1,1.5849929E-1,0E0,0E0,0E0,4.5234412E-2,0E0,0E0,1.4825341E-1,3.9074704E-2,7.943349E-2,1.2323026E-1,8.084141E-2,7.30715E-2,8.183545E-2,1.0424943E-1,0E0,0E0,1.0244033E-1,1.42109E-1,0E0,0E0,0E0,8.371319E-2,4.68783E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,30,30,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,48,48,49,49],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,-1,26,28,30,32,34,-1,-1,-1,36,-1,38,40,-1,-1,-1,42,-1,-1,44,46,48,50,52,54,56,58,-1,-1,60,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8223077E2,4.5620965E6,8E0,9.639872E0,9.7E1,3.713753E7,3.6966505E0,3.1146461E1,8E0,1.127045E-3,-2.0677926E-2,4.1681065E1,1.24141544E-1,3.0392378E7,6.6615967E-3,4.96583E-2,5.4444447E0,4.23E2,3.0620155E0,3.8652172E1,8.998087E-3,-3.598429E-3,2.053023E-3,5.8165E4,-8.081487E-3,3.3820656E7,1.7813443E-1,8.180957E-4,8.941997E-3,-2.4958227E-3,1.2916E4,4.8092487E-3,-2.2350268E-3,2.0926375E5,1.6E1,1.0918E4,2.0263722E8,5.9565744E5,3.89E2,1.28636E7,3.3343E4,-1.543872E-2,-5.6961183E-3,7.233077E2,6.941074E6,2.1256862E-4,-7.645972E-3,5.9308605E-3,5.0710382E0,4.0876604E7,-8.7616476E-4,-2.350139E-3,5.8838265E-4,-2.726804E-3,1.4030855E-3,-3.856925E-3,1.3043935E-3,-4.2733512E-4,-9.182556E-3,1.5102379E-3,4.079536E-3,-1.7800365E-3,1.9311015E-3,-3.4142241E-3,-3.9096057E-4,7.0183324E-3,9.1110193E-4],"split_indices":[67,62,32,69,0,60,50,71,32,0,0,71,53,58,0,53,69,44,73,68,0,0,0,10,0,7,53,0,0,0,1,0,0,48,0,9,7,43,1,60,44,0,0,70,60,0,0,0,69,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.95E2,5E2,3.95E2,4.96E2,4E0,2.59E2,1.36E2,4.68E2,2.8E1,2E0,2E0,2.36E2,2.3E1,1.31E2,5E0,4.61E2,7E0,1.4E1,1.4E1,2.31E2,5E0,1.9E1,4E0,1.28E2,3E0,2.31E2,2.3E2,3E0,4E0,7E0,7E0,2E0,1.2E1,2.26E2,5E0,8.1E1,4.7E1,8.1E1,1.5E2,6.5E1,1.65E2,3E0,4E0,1.38E2,8.8E1,2E0,3E0,2E0,7.9E1,1.1E1,3.6E1,4.6E1,3.5E1,9E0,1.41E2,5.8E1,7E0,1.63E2,2E0,7.6E1,6.2E1,4E1,4.8E1,3.7E1,4.2E1,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[-2.807301E-3,-1.31887095E-2,2.2502974E-2,-9.394612E-3,-7.316695E-2,3.170816E-2,-5.2195456E-2,-1.1990658E-2,9.6111946E-2,-1.5955044E-1,-1.841018E-2,4.799963E-2,-2.0521825E-2,-1.1225554E-1,-2.7527727E-4,-9.469893E-3,-9.8770685E-2,-1.2113384E-3,1.8044461E-1,1.0476052E-3,-2.0694655E-1,-9.044569E-3,5.3552603E-3,5.3963978E-2,-6.117639E-2,4.3944277E-2,-7.069143E-2,-1.7199653E-3,-7.1937176E-3,3.8251348E-2,-1.5553582E-2,-2.128679E-2,-2.0398939E-2,1.2776694E-2,4.070265E-3,-1.1530577E-2,-2.9299478E-3,3.4723433E-3,-1.7015138E-3,4.6069454E-2,1.2953648E-1,-1.1101471E-2,6.065949E-4,7.5797397E-3,1.4893553E-2,-1.2653489E-2,-2.5013387E-2,-4.6255935E-2,5.7530317E-2,-2.2284858E-2,1.0580634E-2,-5.707883E-3,2.001672E-3,8.866725E-3,6.289918E-2,7.595855E-3,1.0481175E-3,-4.8037656E-3,2.1809675E-3,5.293607E-3,-4.6226587E-2,8.74423E-4,-5.826942E-3,3.211944E-3,-4.965832E-3,-6.720536E-4,-3.8234391E-3,2.5945993E-3,-5.9521635E-4,3.740163E-3,-1.215769E-3,3.220055E-3,-2.720681E-3,2.3393529E-3,-3.3891639E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,33,-1,35,-1,37,39,41,43,45,-1,-1,47,49,-1,51,-1,-1,-1,-1,-1,-1,53,55,-1,-1,-1,57,-1,59,61,63,65,67,-1,-1,69,71,-1,-1,-1,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4387185E-1,1.4904927E-1,1.8731053E-1,1.7097037E-1,1.8153581E-1,2.0605277E-1,8.2716644E-2,1.3162836E-1,1.5922847E-1,1.3410693E-1,1.0564802E-1,1.2311542E-1,1.9069232E-1,3.5792485E-2,0E0,1.7188445E-1,4.6887967E-1,0E0,5.48051E-2,0E0,5.5172324E-2,0E0,6.95877E-2,9.843892E-2,1.34872E-1,8.748154E-2,2.967532E-1,0E0,0E0,1.11818425E-1,9.241609E-2,0E0,1.0265858E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.962016E-2,5.4165334E-2,0E0,0E0,0E0,8.8048436E-2,0E0,8.772589E-2,6.9198854E-2,1.1159071E-1,1.9458887E-1,1.1486094E-1,0E0,0E0,1.2985514E-1,8.868784E-2,0E0,0E0,0E0,0E0,0E0,6.7968965E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,18,18,20,20,22,22,23,23,24,24,25,25,26,26,29,29,30,30,32,32,39,39,40,40,44,44,46,46,47,47,48,48,49,49,50,50,53,53,54,54,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,34,-1,36,-1,38,40,42,44,46,-1,-1,48,50,-1,52,-1,-1,-1,-1,-1,-1,54,56,-1,-1,-1,58,-1,60,62,64,66,68,-1,-1,70,72,-1,-1,-1,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.867288E2,9.932432E0,4.217427E7,3.5E1,2E0,2.342612E0,4.4522205E6,4.75356E6,7E0,8.317E3,2.94552E4,2.5842668E7,2.4595E5,1.3934426E0,-2.7527727E-4,7E0,4.9469455E6,-1.2113384E-3,2.4624279E0,1.0476052E-3,1.764442E7,-9.044569E-3,3.18E3,1.9616238E2,1.1736916E0,3.222E3,6.27907E0,-1.7199653E-3,-7.1937176E-3,1.4473684E-1,1.0674361E8,-2.128679E-2,2.1623019E5,1.2776694E-2,4.070265E-3,-1.1530577E-2,-2.9299478E-3,3.4723433E-3,-1.7015138E-3,1E0,2.4508352E9,-1.1101471E-2,6.065949E-4,7.5797397E-3,5E0,-1.2653489E-2,1.4076087E0,6.54E2,5.2E1,2.8E1,1.390602E8,-5.707883E-3,2.001672E-3,1.753E3,1.260841E2,7.595855E-3,1.0481175E-3,-4.8037656E-3,2.1809675E-3,5.293607E-3,1.524E3,8.74423E-4,-5.826942E-3,3.211944E-3,-4.965832E-3,-6.720536E-4,-3.8234391E-3,2.5945993E-3,-5.9521635E-4,3.740163E-3,-1.215769E-3,3.220055E-3,-2.720681E-3,2.3393529E-3,-3.3891639E-3],"split_indices":[67,69,60,3,32,57,62,62,8,9,60,62,9,68,0,3,62,0,68,0,60,0,44,71,53,2,71,0,0,71,7,0,48,0,0,0,0,0,0,29,12,0,0,0,3,0,69,2,8,8,7,0,0,44,71,0,0,0,0,0,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.26E2,6.57E2,2.69E2,6.19E2,3.8E1,2.4E2,2.9E1,6.05E2,1.4E1,1.4E1,2.4E1,1.83E2,5.7E1,1.2E1,1.7E1,5.89E2,1.6E1,6E0,8E0,3E0,1.1E1,2E0,2.2E1,1.74E2,9E0,2.5E1,3.2E1,5E0,7E0,6.6E1,5.23E2,2E0,1.4E1,3E0,5E0,8E0,3E0,8E0,1.4E1,1.59E2,1.5E1,2E0,7E0,4E0,2.1E1,5E0,2.7E1,1.2E1,5.4E1,4.16E2,1.07E2,5E0,9E0,5E1,1.09E2,1.1E1,4E0,4E0,1.7E1,3E0,2.4E1,7E0,5E0,5.1E1,3E0,3.69E2,4.7E1,3.6E1,7.1E1,1.6E1,3.4E1,1.04E2,5E0,5E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}},{"base_weights":[-4.04417E-3,-1.715601E-2,1.6456628E-2,-4.828326E-3,-4.008854E-2,2.3486165E-2,-4.53541E-2,-8.748269E-3,1.0228614E-1,-3.2253258E-2,-3.329458E-1,1.0200145E-1,1.768874E-2,-1.3762939E-1,-1.51530085E-2,-2.625827E-3,-5.9592105E-2,7.969083E-3,8.6549413E-4,-3.7060745E-2,4.6552303E-3,-2.3242652E-2,-2.564849E-3,1.285262E-1,-2.3017281E-3,1.4000239E-2,5.7666106E-3,-2.5522548E-3,-1.0573438E-2,6.913874E-2,-4.511578E-2,-1.3536558E-2,2.880781E-2,-5.1864353E-3,-1.5248851E-1,2.4435718E-2,-4.7206644E-2,3.6783176E-3,9.393323E-3,2.5141506E-2,-2.0546447E-2,-9.3457324E-4,7.599167E-3,-1.081304E-3,-6.404046E-3,6.0205676E-2,-1.8814111E-2,-2.8159232E-3,3.6658864E-2,2.4200452E-2,-3.9163996E-3,-1.3337643E-2,-8.9791074E-2,4.3028433E-2,-4.3358775E-3,-8.2365125E-2,-1.599626E-2,6.0606826E-2,6.9993404E-3,-8.869412E-2,9.253459E-3,4.51571E-3,-1.8776401E-3,-1.3855222E-2,-6.9319195E-4,2.1256143E-3,-2.7456374E-3,5.5207447E-3,-4.3649812E-5,2.3401745E-3,-6.912039E-3,-4.504357E-4,3.2550918E-3,-4.624173E-3,2.502714E-3,1.7698095E-3,-1.3889282E-3,2.0256494E-3,8.409134E-3,-2.2139348E-4,4.5530144E-3,3.2215638E-3,-5.45448E-3,-7.6953915E-4,1.7090285E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,-1,-1,-1,37,-1,39,-1,-1,-1,41,43,45,47,49,51,53,55,-1,-1,57,59,-1,-1,-1,-1,61,63,-1,65,67,-1,-1,69,71,-1,73,75,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4731246E-1,1.5840396E-1,1.5705097E-1,1.5457183E-1,4.39007E-1,1.4522097E-1,1.01026624E-1,1.10039614E-1,7.193866E-2,1.2587313E-1,2.0414168E-1,9.47638E-2,1.166792E-1,5.4342896E-2,7.6362066E-2,1.09109536E-1,1.912646E-1,0E0,0E0,1.1720577E-1,0E0,0E0,0E0,5.3013384E-2,0E0,1.1329047E-1,0E0,0E0,0E0,7.0571184E-2,4.133525E-2,9.2978865E-2,5.974544E-2,6.0094282E-2,9.212938E-2,6.3079834E-2,1.7425203E-1,0E0,0E0,1.4241451E-1,1.4703792E-1,0E0,0E0,0E0,0E0,6.3286364E-2,2.3668164E-1,0E0,6.848702E-2,4.5251317E-2,0E0,0E0,9.814836E-2,3.4788433E-2,0E0,1.7962384E-1,6.560104E-2,1.4585134E-1,1.5867008E-1,1.0747112E-1,3.678444E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,23,23,25,25,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,39,39,40,40,45,45,46,46,48,48,49,49,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,-1,-1,-1,38,-1,40,-1,-1,-1,42,44,46,48,50,52,54,56,-1,-1,58,60,-1,-1,-1,-1,62,64,-1,66,68,-1,-1,70,72,-1,74,76,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.88E2,3.962617E0,3.0161016E7,2.848973E6,1.9359898E5,1.0972E4,1.33E2,6.2394366E0,2.0312219E5,3.1797794E1,4.1957852E2,1.4791E4,8.785115E6,2.62E3,1E0,1.157602E6,9.183673E-1,7.969083E-3,8.6549413E-4,4.9E1,4.6552303E-3,-2.3242652E-2,-2.564849E-3,4.911E3,-2.3017281E-3,5.1291872E8,5.7666106E-3,-2.5522548E-3,-1.0573438E-2,4.435876E2,2.3040113E10,1.0476191E0,3.2E2,4.1855645E-1,6.74502E0,1E0,9.66E2,3.6783176E-3,9.393323E-3,1.3E1,7.03E3,-9.3457324E-4,7.599167E-3,-1.081304E-3,-6.404046E-3,4.5157553E2,1.342376E0,-2.8159232E-3,1.5608E4,3.4111922E0,-3.9163996E-3,-1.3337643E-2,7.9227E4,3.625E0,-4.3358775E-3,9.750085E6,1.6666666E0,1.6527965E7,1.537007E7,1E0,1.4176E4,4.51571E-3,-1.8776401E-3,-1.3855222E-2,-6.9319195E-4,2.1256143E-3,-2.7456374E-3,5.5207447E-3,-4.3649812E-5,2.3401745E-3,-6.912039E-3,-4.504357E-4,3.2550918E-3,-4.624173E-3,2.502714E-3,1.7698095E-3,-1.3889282E-3,2.0256494E-3,8.409134E-3,-2.2139348E-4,4.5530144E-3,3.2215638E-3,-5.45448E-3,-7.6953915E-4,1.7090285E-3],"split_indices":[4,71,60,66,48,9,0,69,48,71,4,2,43,2,14,12,71,0,0,0,0,0,0,44,0,7,0,0,0,71,5,68,44,53,69,30,10,0,0,3,44,0,0,0,0,48,69,0,44,68,0,0,9,69,0,9,69,60,9,29,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.18E2,5.6E2,3.58E2,3.65E2,1.95E2,3.22E2,3.6E1,3.53E2,1.2E1,1.91E2,4E0,2.1E1,3.01E2,8E0,2.8E1,3.16E2,3.7E1,6E0,6E0,1.85E2,6E0,2E0,2E0,1.8E1,3E0,2.92E2,9E0,5E0,3E0,7E0,2.1E1,2.35E2,8.1E1,2.4E1,1.3E1,2.6E1,1.59E2,1.2E1,6E0,2.21E2,7.1E1,4E0,3E0,1.8E1,3E0,1.5E1,2.2E2,6E0,7.5E1,1.8E1,6E0,3E0,1E1,2.3E1,3E0,7.4E1,8.5E1,7.4E1,1.47E2,2.1E1,5E1,1.1E1,4E0,2E0,2.18E2,6.9E1,6E0,3E0,1.5E1,3E0,7E0,8E0,1.5E1,6.6E1,8E0,1.7E1,6.8E1,6.6E1,8E0,1.31E2,1.6E1,3E0,1.8E1,2.6E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[3.3417085E-4,-1.7929185E-2,1.4133285E-2,-1.5579539E-2,-1.4929771E-1,7.157093E-2,9.105042E-3,-2.7899243E-2,6.9138613E-3,-1.3191959E-2,-1.8056887E-3,8.586876E-2,-6.868689E-3,-4.2236128E-4,3.1817086E-2,-1.7972512E-2,-7.577078E-2,4.275218E-2,-2.4771418E-2,1.5399561E-3,1.21858515E-1,-6.245172E-2,7.3575983E-3,2.0952672E-2,8.61715E-2,-2.3813915E-2,4.728139E-2,-1.2444911E-1,-1.5718298E-3,6.435044E-2,8.371416E-3,6.2247343E-2,-4.901388E-2,7.1174265E-3,2.459714E-3,-2.4126086E-2,-3.1824082E-1,4.279304E-2,-3.8871942E-3,3.2198474E-2,-8.57542E-2,-3.0516887E-3,5.1614195E-3,-1.17174275E-2,-5.6686997E-2,5.8073834E-5,5.554869E-3,-1.0474283E-3,-1.5692347E-1,4.6788592E-2,7.3945383E-3,1.3124304E-3,-2.4355734E-3,1.4171508E-2,8.129294E-3,-1.6925983E-1,-2.9938756E-2,-3.1420896E-3,-1.2502506E-1,-3.599886E-3,-2.1067794E-2,5.338126E-2,-3.1480906E-3,2.2426257E-3,-9.475804E-2,6.6863745E-2,-7.652746E-3,-8.752689E-3,6.65356E-4,-1.6638937E-3,7.992376E-4,6.3123065E-5,-4.4098925E-3,-8.187944E-3,-1.009562E-3,2.8677196E-3,-2.6860326E-3,4.0196693E-3,-1.2811417E-3,-1.1962747E-2,-3.0247844E-3,5.2527683E-3,-2.0420393E-3,-1.1531715E-3,2.7673768E-3,1.7921436E-3,-1.0066026E-2,-6.329628E-4,3.0844351E-3,-7.5910694E-4,1.6571261E-3,-6.976625E-3,-1.9186803E-3,1.02674E-3,6.191286E-3,-1.1991924E-3,2.6252344E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,-1,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,-1,49,51,53,55,-1,-1,57,59,61,63,65,67,-1,-1,69,71,-1,-1,-1,73,75,-1,-1,-1,77,-1,79,81,83,85,-1,-1,87,-1,89,91,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3715046E-1,1.22685164E-1,1.5415078E-1,1.1076057E-1,9.288363E-2,1.4748481E-1,1.06908515E-1,1.2124184E-1,1.6230674E-1,0E0,0E0,7.381195E-2,0E0,1.6888198E-1,8.441168E-2,8.307843E-2,8.569473E-2,4.921312E-2,1.6337669E-1,0E0,4.3516904E-2,3.7465E-1,1.2408872E-1,1.5036169E-1,9.7658396E-2,7.8070246E-2,5.9640724E-2,6.2666416E-2,0E0,6.0709283E-2,3.402909E-2,8.950692E-2,1.3255495E-1,0E0,0E0,7.371139E-2,1.2720716E-1,9.134965E-2,1.3222669E-1,1.5523447E-1,1.2570097E-1,0E0,0E0,1.02139965E-1,1.1744195E-1,0E0,0E0,0E0,3.4983873E-2,5.953572E-2,0E0,0E0,0E0,4.156979E-2,0E0,5.661051E-2,1.1074301E-1,4.2170033E-2,9.8049484E-2,0E0,0E0,6.0715735E-2,0E0,1.3880944E-1,3.8153738E-2,1.751172E-1,6.3056E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,43,43,44,44,48,48,49,49,53,53,55,55,56,56,57,57,58,58,61,61,63,63,64,64,65,65,66,66],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,-1,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,-1,50,52,54,56,-1,-1,58,60,62,64,66,68,-1,-1,70,72,-1,-1,-1,74,76,-1,-1,-1,78,-1,80,82,84,86,-1,-1,88,-1,90,92,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.9236734E5,6.6E1,1.1160929E0,5.0857143E0,4.6066E4,2.7994856E5,4.39776E6,2.89E2,3.614271E0,-1.3191959E-2,-1.8056887E-3,6.485856E7,-6.868689E-3,1.584022E0,1.1046409E7,1.8303784E7,4.1032645E6,4.407599E-1,3.2931E4,1.5399561E-3,1.1293673E3,2.4E0,1.8671024E0,1E0,1.0029973E8,2.3572223E2,5.1766735E-1,3.9157894E0,-1.5718298E-3,1.9756216E-1,3.7760816E5,2.4818E4,3.3817584E7,7.1174265E-3,2.459714E-3,5.4544777E1,5.5E1,1E0,2.707256E7,1.617695E8,1.935E3,-3.0516887E-3,5.1614195E-3,2.71E2,2.0134516E6,5.8073834E-5,5.554869E-3,-1.0474283E-3,2.871287E0,1.3177083E1,7.3945383E-3,1.3124304E-3,-2.4355734E-3,6.48334E5,8.129294E-3,1.2916E4,3E1,2.1126761E0,1.2376238E0,-3.599886E-3,-2.1067794E-2,4.97E2,-3.1480906E-3,2.364E3,2.284E3,1.0315458E8,6.149315E6,-8.752689E-3,6.65356E-4,-1.6638937E-3,7.992376E-4,6.3123065E-5,-4.4098925E-3,-8.187944E-3,-1.009562E-3,2.8677196E-3,-2.6860326E-3,4.0196693E-3,-1.2811417E-3,-1.1962747E-2,-3.0247844E-3,5.2527683E-3,-2.0420393E-3,-1.1531715E-3,2.7673768E-3,1.7921436E-3,-1.0066026E-2,-6.329628E-4,3.0844351E-3,-7.5910694E-4,1.6571261E-3,-6.976625E-3,-1.9186803E-3,1.02674E-3,6.191286E-3,-1.1991924E-3,2.6252344E-3],"split_indices":[43,3,69,69,1,48,9,0,68,0,0,7,0,68,60,12,60,53,9,0,4,69,68,19,5,67,53,71,0,53,43,1,7,0,0,73,0,19,60,7,2,0,0,2,60,0,0,0,68,71,0,0,0,46,0,1,0,69,69,0,0,2,0,2,0,7,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.39E2,4.04E2,5.35E2,3.98E2,6E0,4.2E1,4.93E2,2.57E2,1.41E2,2E0,4E0,4E1,2E0,3.48E2,1.45E2,2.14E2,4.3E1,6.6E1,7.5E1,1.7E1,2.3E1,3.8E1,3.1E2,1.22E2,2.3E1,1.97E2,1.7E1,1.9E1,2.4E1,4E1,2.6E1,1.6E1,5.9E1,1.5E1,8E0,3.4E1,4E0,7.4E1,2.36E2,1.11E2,1.1E1,3E0,2E1,1.45E2,5.2E1,1.1E1,6E0,5E0,1.4E1,3.5E1,5E0,2E1,6E0,1.2E1,4E0,7E0,5.2E1,2.9E1,5E0,2E0,2E0,6.8E1,6E0,2.22E2,1.4E1,5.9E1,5.2E1,5E0,6E0,7.9E1,6.6E1,2.1E1,3.1E1,1.2E1,2E0,3.1E1,4E0,4E0,8E0,3E0,4E0,4E0,4.8E1,2.2E1,7E0,2E0,3E0,1.1E1,5.7E1,1.43E2,7.9E1,6E0,8E0,3.6E1,2.3E1,4.1E1,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-4.6762507E-3,-2.0151034E-2,8.865793E-3,-2.7267136E-2,4.601742E-2,6.3816006E-3,1.3044992E-1,-1.971588E-2,-1.0020595E-1,-3.3305027E-2,7.779724E-2,1.2307158E-2,-4.8809648E-2,1.1103593E-2,2.3221031E-3,-1.0670985E-2,-7.991643E-2,-7.6407425E-2,-2.1043043E-1,8.6099125E-4,-3.7375868E-3,1.09358594E-1,1.871188E-2,-3.6696114E-2,1.7897788E-2,-5.315118E-3,-3.4843493E-2,-7.3593885E-2,-4.755451E-3,-2.992452E-2,-1.7490548E-1,-1.3141766E-1,-2.0138526E-2,-1.665005E-2,-2.192376E-3,4.7064162E-4,5.8727935E-3,2.3824451E-3,-2.749769E-3,-4.9665608E-3,-1.4259088E-2,4.0691E-2,4.8286235E-3,-1.1717966E-3,-6.9132904E-3,-5.0699968E-2,-1.115462E-2,-1.2985439E-2,4.2629723E-2,-1.0993885E-1,2.3581024E-2,-1.9480914E-1,-1.1031553E-3,-2.811594E-3,-9.060791E-3,-4.366051E-3,8.5618393E-4,-3.0818295E-2,4.398298E-3,-2.9346389E-3,7.2925806E-2,1.3574505E-1,-2.166221E-3,-4.5825462E-4,-5.6842207E-3,4.595972E-3,-8.1986666E-4,-5.0101522E-3,3.2519752E-3,-9.198021E-4,-9.851741E-3,6.5851295E-3,-2.5921137E-4,-1.0115305E-2,-1.8192497E-3,-1.9511085E-3,4.222721E-3,2.0221026E-4,-6.9777505E-3,-1.618256E-4,4.2956336E-3,1.0645736E-2,3.2047294E-3,2.4630988E-4,-4.2235777E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,29,31,33,-1,-1,35,37,39,41,-1,43,45,47,49,51,53,55,-1,-1,-1,-1,-1,-1,-1,57,59,61,-1,-1,63,-1,65,67,69,71,73,-1,-1,-1,-1,-1,75,-1,77,79,81,83,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9716524E-1,2.0796292E-1,1.5079078E-1,2.1668386E-1,1.1098773E-1,1.6183087E-1,7.5281456E-2,1.9571447E-1,8.09792E-2,3.318904E-2,5.5762917E-2,1.228426E-1,3.9910994E-2,0E0,0E0,1.1690012E-1,2.1795419E-1,9.640513E-2,1.2771064E-1,0E0,0E0,3.229958E-2,3.3399418E-2,7.082178E-2,1.19256705E-1,0E0,4.1270826E-2,9.0605766E-2,1.1325761E-1,1.3959119E-1,4.3525428E-2,6.021419E-2,5.0051205E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.779195E-2,2.0552853E-1,2.3472351E-1,0E0,0E0,7.183095E-2,0E0,1.2997164E-1,1.839974E-1,1.15237996E-1,6.960068E-2,4.0946722E-2,0E0,0E0,0E0,0E0,0E0,4.7139756E-2,0E0,6.8141825E-2,1.2507784E-1,6.517066E-2,1.6189344E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,31,31,32,32,40,40,41,41,42,42,45,45,47,47,48,48,49,49,50,50,51,51,57,57,59,59,60,60,61,61,62,62],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,30,32,34,-1,-1,36,38,40,42,-1,44,46,48,50,52,54,56,-1,-1,-1,-1,-1,-1,-1,58,60,62,-1,-1,64,-1,66,68,70,72,74,-1,-1,-1,-1,-1,76,-1,78,80,82,84,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.49E2,9.750085E6,4.435876E2,2.49E2,1.5645E4,8.913649E-3,7.969064E8,9.29023E5,1.2238068E7,3.4846212E5,1.6928803E8,6.593462E4,1.7899E4,1.1103593E-2,2.3221031E-3,4.8465605E3,1.307E3,2.0170736E7,2.3285185E2,8.6099125E-4,-3.7375868E-3,8.1E2,2.23099E7,2.147382E4,1.68574E5,-5.315118E-3,4.533E3,3.28E2,6.848509E7,1.6957658E6,8.327149E7,5.201557E7,1.9095E4,-1.665005E-2,-2.192376E-3,4.7064162E-4,5.8727935E-3,2.3824451E-3,-2.749769E-3,-4.9665608E-3,5.417598E-1,1E0,1.3333334E0,-1.1717966E-3,-6.9132904E-3,2.707256E7,-1.115462E-2,1.0952381E0,7.392347E1,1.138871E0,5E0,1E0,-1.1031553E-3,-2.811594E-3,-9.060791E-3,-4.366051E-3,8.5618393E-4,1E0,4.398298E-3,6.849626E1,1E1,2E0,1.557718E7,-4.5825462E-4,-5.6842207E-3,4.595972E-3,-8.1986666E-4,-5.0101522E-3,3.2519752E-3,-9.198021E-4,-9.851741E-3,6.5851295E-3,-2.5921137E-4,-1.0115305E-2,-1.8192497E-3,-1.9511085E-3,4.222721E-3,2.0221026E-4,-6.9777505E-3,-1.618256E-4,4.2956336E-3,1.0645736E-2,3.2047294E-3,2.4630988E-4,-4.2235777E-3],"split_indices":[10,9,71,10,1,72,12,62,60,43,7,43,44,0,0,43,44,5,67,0,0,2,9,48,9,0,8,44,7,66,12,7,12,0,0,0,0,0,0,0,53,29,68,0,0,60,0,68,4,53,3,26,0,0,0,0,0,16,0,73,3,8,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.39E2,4.38E2,5.01E2,3.96E2,4.2E1,4.92E2,9E0,3.6E2,3.6E1,1.2E1,3E1,4.45E2,4.7E1,3E0,6E0,3.14E2,4.6E1,3.1E1,5E0,6E0,6E0,1.9E1,1.1E1,4.5E1,4E2,7E0,4E1,2.6E1,2.88E2,3.1E1,1.5E1,1.5E1,1.6E1,2E0,3E0,3E0,1.6E1,8E0,3E0,1E1,3.5E1,1.45E2,2.55E2,3.8E1,2E0,2.4E1,2E0,2.46E2,4.2E1,1.2E1,1.9E1,1.3E1,2E0,8E0,7E0,5E0,1.1E1,3.1E1,4E0,6.2E1,8.3E1,1.2E1,2.43E2,1.6E1,8E0,9E0,2.37E2,6E0,3.6E1,7E0,5E0,3E0,1.6E1,1.1E1,2E0,2.9E1,2E0,6E1,2E0,1.7E1,6.6E1,4E0,8E0,2.25E2,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"85","size_leaf_vector":"1"}},{"base_weights":[1.7334038E-3,-9.584436E-3,2.1149717E-2,-1.8853497E-2,1.9713508E-2,5.0200555E-2,3.951617E-3,-3.8058641E-3,-4.390939E-2,2.610896E-2,-1.24272026E-1,2.2006718E-2,8.465827E-2,5.555054E-2,-1.6714297E-2,-4.379971E-3,1.1127788E-3,-3.243921E-2,-1.4654733E-1,-5.868304E-3,3.1853754E-2,-9.542601E-3,-1.5124351E-3,4.7252517E-2,-3.407533E-2,-3.1014018E-2,1.05007045E-1,-3.773888E-2,8.4763385E-2,-2.4187338E-2,4.2008893E-3,1.6843708E-2,-2.894757E-2,-7.2681434E-2,-8.976237E-3,-1.0298824E-2,-2.9620423E-3,3.928355E-2,-3.2903883E-3,8.515431E-2,5.5374706E-4,-5.313127E-3,1.9894455E-2,-6.594069E-3,9.71532E-4,6.098218E-3,5.5646542E-2,-3.0963207E-3,1.5464746E-3,3.0282613E-2,1.4817941E-1,3.0100893E-2,-3.6146875E-2,5.6616315E-3,1.2927372E-2,-9.8172024E-2,1.1795396E-3,-5.1026974E-2,-6.707313E-3,-2.239625E-2,2.4707592E-3,2.5718745E-2,1.0200721E-1,3.962391E-2,6.528801E-3,-2.926344E-3,2.241707E-3,7.253641E-2,-2.4640465E-3,-6.3459477E-3,4.9862385E-2,1.8087533E-1,-1.6741352E-3,5.754819E-2,-2.8138246E-2,-2.1360781E-2,-8.911383E-2,7.86953E-4,-2.705579E-3,-1.3250972E-2,-3.3408853E-3,-7.434187E-4,3.8961102E-3,-6.1019068E-3,-1.0583054E-3,-3.356104E-4,-7.1770116E-3,5.717906E-4,5.330313E-3,5.337497E-3,-5.953939E-4,-1.0540519E-3,3.5692975E-3,4.300145E-4,6.263219E-3,5.3236465E-4,6.5818457E-3,1.0359328E-2,3.872196E-3,4.1754297E-3,-2.7563935E-4,-5.8195847E-3,8.1156875E-4,-7.547968E-3,-6.876903E-4,-5.328402E-3,4.9272116E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,31,33,35,-1,37,-1,-1,39,41,43,45,47,49,51,-1,53,55,57,59,-1,-1,61,-1,63,-1,-1,65,-1,-1,-1,67,-1,-1,69,71,73,75,-1,77,79,81,83,-1,85,-1,87,89,91,-1,-1,-1,93,-1,-1,95,97,-1,99,101,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9753885E-1,1.5462025E-1,1.6517559E-1,1.6271064E-1,1.2940142E-1,1.1763087E-1,2.2379333E-1,1.2056598E-1,1.8596482E-1,1.1950158E-1,3.7870206E-2,9.9302836E-2,1.3271505E-1,1.6671382E-1,1.2217335E-1,0E0,1.22489914E-1,1.3814531E-1,8.5188806E-2,0E0,1.00356445E-1,0E0,0E0,6.3290454E-2,9.745771E-2,5.4706663E-2,5.329126E-2,3.2922186E-2,1.5444067E-1,9.2634045E-2,0E0,6.826097E-2,1.857282E-1,7.629365E-2,7.9530574E-2,0E0,0E0,9.9476606E-2,0E0,5.227135E-2,0E0,0E0,3.5852194E-2,0E0,0E0,0E0,3.686797E-2,0E0,0E0,9.1682844E-2,1.3296917E-1,4.3180488E-2,8.89447E-2,0E0,4.7836788E-2,1.1232805E-1,9.065304E-2,9.448141E-2,0E0,1.5508841E-1,0E0,1.1792237E-1,3.2231092E-2,3.4111552E-2,0E0,0E0,0E0,6.1675027E-2,0E0,0E0,8.288774E-2,5.5022895E-2,0E0,3.7358034E-2,4.1979775E-2,8.2840584E-2,6.4670086E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,20,20,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,33,33,34,34,37,37,39,39,42,42,46,46,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,59,59,61,61,62,62,63,63,67,67,70,70,71,71,73,73,74,74,75,75,76,76],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,32,34,36,-1,38,-1,-1,40,42,44,46,48,50,52,-1,54,56,58,60,-1,-1,62,-1,64,-1,-1,66,-1,-1,-1,68,-1,-1,70,72,74,76,-1,78,80,82,84,-1,86,-1,88,90,92,-1,-1,-1,94,-1,-1,96,98,-1,100,102,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,4.39776E6,1.1544592E1,3.53969E5,1E0,6.9706165E2,2.0263722E8,1.2571428E1,8.719101E0,3.0502679E-6,3.9433E4,4.179E3,1.4414759E0,2.5711747E5,4.3E1,-4.379971E-3,3.9433E4,1.084437E6,9E1,-5.868304E-3,4.7420635E1,-9.542601E-3,-1.5124351E-3,1.8671024E0,6.516E3,3.0984934E8,2.5603516E5,1.1770261E8,3.1797794E1,2E0,4.2008893E-3,1.1818181E0,2.9E2,9.07021E5,2.73E3,-1.0298824E-2,-2.9620423E-3,4.046E3,-3.2903883E-3,1.9559006E1,5.5374706E-4,-5.313127E-3,8.681474E4,-6.594069E-3,9.71532E-4,6.098218E-3,8.58E2,-3.0963207E-3,1.5464746E-3,1.935E3,1.05039425E1,3.1146461E1,2.4076016E5,5.6616315E-3,1E0,4.6066E4,1E0,3.06E2,-6.707313E-3,2.9E1,2.4707592E-3,1E0,1.651E3,1.29223805E5,6.528801E-3,-2.926344E-3,2.241707E-3,1.5787431E0,-2.4640465E-3,-6.3459477E-3,1.6208625E6,4.117083E1,-1.6741352E-3,4.7126865E1,1.3293153E3,1.33E2,3.2645503E5,7.86953E-4,-2.705579E-3,-1.3250972E-2,-3.3408853E-3,-7.434187E-4,3.8961102E-3,-6.1019068E-3,-1.0583054E-3,-3.356104E-4,-7.1770116E-3,5.717906E-4,5.330313E-3,5.337497E-3,-5.953939E-4,-1.0540519E-3,3.5692975E-3,4.300145E-4,6.263219E-3,5.3236465E-4,6.5818457E-3,1.0359328E-2,3.872196E-3,4.1754297E-3,-2.7563935E-4,-5.8195847E-3,8.1156875E-4,-7.547968E-3,-6.876903E-4,-5.328402E-3,4.9272116E-4],"split_indices":[2,9,71,9,79,67,7,70,69,53,1,2,68,43,3,0,1,9,0,0,73,0,0,68,44,7,48,7,71,8,0,68,10,9,10,0,0,44,0,73,0,0,48,0,0,0,0,0,0,2,69,71,48,0,19,1,16,2,0,8,0,28,2,48,0,0,0,53,0,0,43,73,0,73,4,0,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.97E2,5.67E2,3.3E2,4.31E2,1.36E2,1.22E2,2.08E2,2.7E2,1.61E2,1.31E2,5E0,6.8E1,5.4E1,5.9E1,1.49E2,1.3E1,2.57E2,1.46E2,1.5E1,4E0,1.27E2,2E0,3E0,4.7E1,2.1E1,8E0,4.6E1,1.4E1,4.5E1,1.4E2,9E0,1.69E2,8.8E1,5.3E1,9.3E1,7E0,8E0,1.19E2,8E0,2.2E1,2.5E1,8E0,1.3E1,2E0,6E0,2.9E1,1.7E1,1E1,4E0,2.5E1,2E1,2.5E1,1.15E2,5E0,1.64E2,2.6E1,6.2E1,4.2E1,1.1E1,7.7E1,1.6E1,9.9E1,2E1,1.3E1,9E0,3E0,1E1,1.5E1,2E0,2E0,2.3E1,1.7E1,3E0,1.7E1,8E0,9.1E1,2.4E1,1.56E2,8E0,2E0,2.4E1,5.2E1,1E1,1E1,3.2E1,7E1,7E0,8.7E1,1.2E1,1.8E1,2E0,5E0,8E0,8E0,7E0,1.7E1,6E0,1.1E1,6E0,1.1E1,6E0,2E0,6E0,3E0,8.8E1,1.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[-7.448023E-4,3.4866787E-2,-5.5008302E-3,5.1997174E-2,-3.5239603E-2,-2.7405284E-2,2.987653E-3,6.1357144E-2,-4.9272195E-2,-9.028189E-3,1.0852426E-2,1.0859167E-2,-5.432855E-2,5.817466E-2,-4.9799186E-4,7.9003975E-2,2.699028E-2,1.7115871E-3,-6.7225443E-3,-3.522127E-3,6.045583E-2,-1.968872E-2,6.147067E-2,-2.0126134E-1,-4.586872E-2,1.5884206E-3,4.7143823E-3,-5.662606E-3,1.4980334E-3,8.696458E-5,9.916124E-2,-9.656709E-4,3.3924927E-3,6.086063E-3,-3.966472E-4,-9.080976E-3,-1.0148365E-2,1.0878935E-1,2.1857147E-3,-1.5613282E-2,-7.670276E-4,-5.6672152E-2,1.8275274E-3,3.38639E-2,-4.118954E-3,1.1186973E-1,-2.3038425E-3,-2.370522E-2,6.348592E-2,1.3507952E-1,1.159544E-3,4.88501E-3,-3.754055E-2,-7.1436586E-3,-4.6901E-2,8.832379E-2,-1.6677122E-3,-1.1108709E-2,3.9280895E-2,9.389969E-3,4.310853E-3,1.3008476E-3,-2.307138E-3,-2.2686478E-3,4.5414628E-3,-2.1234406E-4,7.2977184E-3,-6.518947E-3,-2.854655E-4,-1.6914424E-3,-6.213127E-3,-1.6368848E-3,4.986844E-3,1.1586742E-3,-2.54547E-3,-2.270606E-4,-4.243373E-3,3.0376543E-3,-1.7705422E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,29,31,-1,-1,-1,33,35,37,39,41,-1,-1,-1,43,-1,45,-1,-1,-1,-1,47,-1,49,51,-1,-1,53,-1,55,57,59,-1,61,63,65,-1,-1,67,-1,69,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.566894E-1,1.3273586E-1,1.5180533E-1,8.6436495E-2,1.6321836E-1,2.3547646E-1,1.1333037E-1,4.6991467E-2,7.092575E-2,0E0,8.188663E-2,1.4808735E-1,1.5733391E-1,3.2851517E-2,1.34512E-1,8.22196E-2,6.5155685E-2,0E0,0E0,0E0,5.9399083E-2,1.2436949E-1,1.007525E-1,1.6836289E-1,1.1999506E-1,0E0,0E0,0E0,9.958333E-2,0E0,8.49103E-2,0E0,0E0,0E0,0E0,6.3056625E-2,0E0,4.0724456E-2,7.3403314E-2,0E0,0E0,1.00767225E-1,0E0,1.5688616E-1,1.4210282E-1,4.862544E-2,0E0,6.697592E-2,4.311803E-2,4.6453953E-2,0E0,0E0,4.1036148E-2,0E0,9.1009796E-2,7.848944E-2,7.189018E-2,2.0062688E-1,1.3338229E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,20,20,21,21,22,22,23,23,24,24,28,28,30,30,35,35,37,37,38,38,41,41,43,43,44,44,45,45,47,47,48,48,49,49,52,52,54,54,55,55,56,56,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,30,32,-1,-1,-1,34,36,38,40,42,-1,-1,-1,44,-1,46,-1,-1,-1,-1,48,-1,50,52,-1,-1,54,-1,56,58,60,-1,62,64,66,-1,-1,68,-1,70,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7E0,1.4553613E1,1.795688E2,1E0,3E0,4.304348E0,3.617263E6,3.307E3,1.212945E6,-9.028189E-3,8.995735E5,2.35E2,7.5456814E-4,1.0024194E1,5.141301E6,4.2384106E-1,1.9630163E8,1.7115871E-3,-6.7225443E-3,-3.522127E-3,1.2141494E5,6.737509E7,6.76E2,7.9E2,2.5436695E6,1.5884206E-3,4.7143823E-3,-5.662606E-3,1.5123151E6,8.696458E-5,9.465E3,-9.656709E-4,3.3924927E-3,6.086063E-3,-3.966472E-4,7.1428573E-1,-1.0148365E-2,1.9741463E2,2.2783158E5,-1.5613282E-2,-7.670276E-4,3.3817584E7,1.8275274E-3,2.8096385E0,1.5146261E3,5.26E2,-2.3038425E-3,3.959E3,5.684E3,1.16385914E-1,1.159544E-3,4.88501E-3,1.45064E5,-7.1436586E-3,1.662258E2,1.4026549E0,2.02065E5,2.067251E0,1.7101741E2,9.389969E-3,4.310853E-3,1.3008476E-3,-2.307138E-3,-2.2686478E-3,4.5414628E-3,-2.1234406E-4,7.2977184E-3,-6.518947E-3,-2.854655E-4,-1.6914424E-3,-6.213127E-3,-1.6368848E-3,4.986844E-3,1.1586742E-3,-2.54547E-3,-2.270606E-4,-4.243373E-3,3.0376543E-3,-1.7705422E-3],"split_indices":[3,71,67,19,8,73,5,2,9,0,43,2,53,73,5,71,12,0,0,0,48,7,2,2,60,0,0,0,60,0,10,0,0,0,0,71,0,4,43,0,0,7,0,68,67,44,0,9,9,69,0,0,7,0,67,68,1,53,73,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.23E2,1.08E2,8.15E2,8.7E1,2.1E1,2.27E2,5.88E2,8E1,7E0,4E0,1.7E1,9.4E1,1.33E2,3.4E1,5.54E2,5.2E1,2.8E1,4E0,3E0,6E0,1.1E1,5.9E1,3.5E1,6E0,1.27E2,2.3E1,1.1E1,8E0,5.46E2,1.1E1,4.1E1,1.4E1,1.4E1,5E0,6E0,5.7E1,2E0,1.9E1,1.6E1,3E0,3E0,1.13E2,1.4E1,8E1,4.66E2,3.8E1,3E0,4.8E1,9E0,1.4E1,5E0,4E0,1.2E1,9E0,1.04E2,3.1E1,4.9E1,4.02E2,6.4E1,5E0,3.3E1,1.6E1,3.2E1,2E0,7E0,2E0,1.2E1,2E0,1E1,9.4E1,1E1,4E0,2.7E1,3.3E1,1.6E1,3.74E2,2.8E1,4.8E1,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[2.4144925E-4,-3.90729E-2,4.6359557E-3,-1.9151451E-2,-2.0051366E-1,-2.4944097E-2,1.0788055E-2,-1.152006E-2,-7.766365E-3,-3.3565788E-4,-1.1530883E-2,3.977496E-3,-5.1574446E-2,5.1437426E-2,6.495179E-3,-4.0156655E-2,3.7546154E-2,-2.039929E-2,4.265247E-2,-3.7639674E-2,-8.351506E-3,6.994982E-2,-2.1300784E-3,-1.1145521E-3,3.088496E-2,-1.658214E-2,-9.962007E-3,9.362071E-2,-8.078543E-4,6.8100885E-4,-5.4514304E-2,-2.764194E-2,6.863761E-2,-2.3961693E-2,-5.328642E-3,5.602254E-3,2.2322438E-3,-1.226E-2,5.6334113E-4,4.555602E-2,1.0036639E-2,3.7748378E-2,-3.2722518E-2,-9.728155E-4,1.23998344E-1,-5.548123E-3,-4.1743668E-4,2.8500354E-3,-4.336924E-3,4.2601256E-3,-8.637266E-4,-4.3785363E-2,1.3994729E-2,5.3426083E-3,-6.288764E-2,4.1334527E-3,2.9275738E-2,-2.410475E-2,4.49556E-2,-1.28953E-3,5.6676297E-3,-3.2353413E-3,3.7139264E-4,8.7774955E-3,1.2933136E-3,2.265234E-3,-2.7355617E-3,3.5716807E-3,-4.8512532E-4,-1.6433993E-4,1.8922039E-3,-1.1465391E-2,-9.901592E-4,-1.3118584E-3,2.1363334E-3,-2.445144E-3,1.629042E-3,4.839787E-3,3.368306E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,9,11,13,15,-1,-1,-1,17,19,21,23,25,27,29,31,33,-1,35,-1,37,39,41,-1,43,-1,-1,45,47,49,51,-1,-1,-1,-1,53,55,57,59,61,-1,63,-1,-1,-1,-1,-1,-1,65,67,69,71,-1,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5809034E-1,2.920442E-1,1.5003814E-1,9.171887E-2,8.987549E-2,1.0957241E-1,1.18676744E-1,1.1391487E-1,0E0,0E0,0E0,6.5921314E-2,1.2522963E-1,1.2021093E-1,1.1473504E-1,2.057026E-1,9.400884E-2,5.2203335E-2,5.121575E-2,6.881305E-2,0E0,5.5202097E-2,0E0,2.0803824E-1,4.4505194E-2,4.157331E-2,0E0,5.478464E-2,0E0,0E0,6.382982E-2,5.2388545E-2,4.175306E-2,4.5006465E-2,0E0,0E0,0E0,0E0,1.4286311E-1,5.8051914E-2,7.4998915E-2,6.396227E-2,5.454927E-2,0E0,6.947279E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.7096675E-2,3.301546E-2,1.3755105E-1,2.4683605E-1,0E0,6.2584445E-2,5.5714197E-2,6.8350896E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,23,23,24,24,25,25,27,27,30,30,31,31,32,32,33,33,38,38,39,39,40,40,41,41,42,42,44,44,51,51,52,52,53,53,54,54,56,56,57,57,58,58],"right_children":[2,4,6,8,10,12,14,16,-1,-1,-1,18,20,22,24,26,28,30,32,34,-1,36,-1,38,40,42,-1,44,-1,-1,46,48,50,52,-1,-1,-1,-1,54,56,58,60,62,-1,64,-1,-1,-1,-1,-1,-1,66,68,70,72,-1,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8182236E1,8E0,5.02E2,1.9495455E6,1.3020051E-5,4.5981E4,1.9200362E8,2.35E2,-7.766365E-3,-3.3565788E-4,-1.1530883E-2,4.336364E0,4.909091E0,1.2909952E1,2.9652428E6,1.3E2,2.2051822E2,3.994386E0,4.9E2,3.477E4,-8.351506E-3,2.01184E5,-2.1300784E-3,1.3085094E2,5.929839E8,2.4E1,-9.962007E-3,5E0,-8.078543E-4,6.8100885E-4,1.0976E4,8.060703E2,1.02E2,1.9741463E2,-5.328642E-3,5.602254E-3,2.2322438E-3,-1.226E-2,2.4985248E6,1E0,1.0836781E11,1.5857142E1,3.23E2,-9.728155E-4,3.000176E5,-5.548123E-3,-4.1743668E-4,2.8500354E-3,-4.336924E-3,4.2601256E-3,-8.637266E-4,3.466E3,9.183432E4,1.5859042E6,3.6522612E6,4.1334527E-3,3.7509E4,4.7429975E6,5.1167645E6,-1.28953E-3,5.6676297E-3,-3.2353413E-3,3.7139264E-4,8.7774955E-3,1.2933136E-3,2.265234E-3,-2.7355617E-3,3.5716807E-3,-4.8512532E-4,-1.6433993E-4,1.8922039E-3,-1.1465391E-2,-9.901592E-4,-1.3118584E-3,2.1363334E-3,-2.445144E-3,1.629042E-3,4.839787E-3,3.368306E-4],"split_indices":[62,8,2,43,53,12,46,2,0,0,0,69,68,71,43,2,4,73,1,1,0,9,0,4,5,2,0,3,0,0,12,48,0,4,0,0,0,0,43,26,46,4,1,0,43,0,0,0,0,0,0,1,43,43,60,0,1,43,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.13E2,9.1E1,8.22E2,8.2E1,9E0,1.41E2,6.81E2,7.9E1,3E0,2E0,7E0,6.8E1,7.3E1,6.4E1,6.17E2,5E1,2.9E1,4.2E1,2.6E1,6.7E1,6E0,5.4E1,1E1,4.71E2,1.46E2,4.5E1,5E0,1.4E1,1.5E1,2.1E1,2.1E1,7E0,1.9E1,5.8E1,9E0,1.5E1,3.9E1,2E0,4.69E2,8.5E1,6.1E1,1E1,3.5E1,3E0,1.1E1,8E0,1.3E1,3E0,4E0,1.5E1,4E0,3.8E1,2E1,4.37E2,3.2E1,2.2E1,6.3E1,3.1E1,3E1,6E0,4E0,1.8E1,1.7E1,6E0,5E0,5E0,3.3E1,5E0,1.5E1,3.5E2,8.7E1,5E0,2.7E1,1.4E1,4.9E1,2.1E1,1E1,1.1E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-1.8104968E-4,-9.073028E-3,1.9979037E-2,-8.198647E-2,-5.914119E-3,-1.318095E-2,4.4309657E-2,-4.1478332E-2,-1.7613478E-2,-2.5536727E-2,2.0399462E-3,-2.5216324E-2,1.3538261E-1,5.5981632E-2,-6.066681E-3,-1.601605E-2,-7.6637845E-3,-2.9448442E-2,4.1344925E-3,3.8326968E-2,-7.543256E-3,-8.368896E-3,-1.9196333E-2,1.1441847E-3,1.0189186E-2,3.0941376E-2,9.954333E-2,2.992137E-3,-4.8005287E-2,-5.7728466E-2,3.5640456E-2,-2.5723334E-2,-5.876695E-3,6.095529E-2,-8.961524E-3,-2.1320803E-2,1.5550013E-2,-2.7001588E-2,5.9214782E-2,5.1096965E-2,-8.3069675E-2,-5.3910734E-3,1.11695364E-1,-3.2886737E-3,1.5933178E-3,5.211574E-4,-5.9776385E-3,4.6064244E-3,-1.0761018E-3,-8.745066E-2,-2.0713849E-2,6.6895016E-2,-3.5428382E-3,1.8199068E-2,-5.6105372E-3,3.398669E-3,-4.1843772E-2,8.033906E-2,-3.394311E-3,-2.9283313E-3,-6.493493E-2,-2.2971865E-3,4.3038647E-3,7.151818E-2,-3.8613677E-2,-1.6640241E-3,-8.39589E-3,9.531306E-3,8.9125365E-2,-1.2863018E-3,-6.6335504E-3,2.591207E-4,-1.7334035E-3,3.4784111E-3,-7.129732E-4,-1.1256716E-3,3.0766055E-3,8.314572E-4,-2.3506384E-3,-2.9542877E-3,-3.2563982E-4,5.041627E-3,-3.201166E-4,6.163241E-3,-6.2690215E-4,2.120955E-3,-1.0564716E-3,-4.96715E-3,-5.8047596E-4,4.01065E-3,1.0508565E-3,-9.839364E-3,3.6470796E-4,2.918953E-3,7.2860536E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,-1,33,35,-1,37,-1,-1,39,41,-1,43,45,47,49,-1,51,53,55,57,59,61,63,65,-1,67,-1,-1,-1,-1,-1,-1,69,71,73,-1,75,-1,77,79,81,83,85,87,-1,-1,89,91,-1,-1,-1,93,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6223508E-1,1.4416046E-1,2.2406848E-1,2.9681736E-1,9.416803E-2,2.1439996E-1,9.441826E-2,7.2713286E-2,0E0,8.0507495E-2,1.4981984E-1,9.9008135E-2,7.806493E-2,1.3872352E-1,9.461951E-2,4.7306538E-2,0E0,5.7759568E-2,0E0,9.685157E-2,1.0882704E-1,0E0,6.7084886E-2,0E0,0E0,1.9727322E-1,1.360074E-1,0E0,3.9782804E-2,6.0008522E-2,3.9561138E-2,4.8900984E-2,0E0,5.478233E-2,9.3190685E-2,1.0870805E-1,1.5737793E-1,8.930139E-2,4.0766273E-2,1.3466564E-1,5.3454913E-2,0E0,7.643944E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.564965E-2,6.679678E-2,3.94589E-2,0E0,5.2435834E-2,0E0,7.744504E-2,8.79295E-2,7.173593E-2,1.3886067E-1,5.9617765E-2,8.212E-2,0E0,0E0,4.0535033E-2,1.1414269E-1,0E0,0E0,0E0,5.7825357E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,19,19,20,20,22,22,25,25,26,26,28,28,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,49,49,50,50,51,51,53,53,55,55,56,56,57,57,58,58,59,59,60,60,63,63,64,64,68,68],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,-1,34,36,-1,38,-1,-1,40,42,-1,44,46,48,50,-1,52,54,56,58,60,62,64,66,-1,68,-1,-1,-1,-1,-1,-1,70,72,74,-1,76,-1,78,80,82,84,86,88,-1,-1,90,92,-1,-1,-1,94,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.5222016E8,4E0,1E0,1.3508157E7,1.41E2,9.932432E0,6.6E1,1E0,-1.7613478E-2,7.3012676E0,2.3781754E8,5.1942E4,1.8E1,1.2E1,4.0663E4,3.1657144E2,-7.6637845E-3,5.035714E0,4.1344925E-3,1.1197369E1,5.3871745E-1,-8.368896E-3,1.4433751E0,1.1441847E-3,1.0189186E-2,8E0,1.8727492E6,2.992137E-3,1.3888889E-2,3.0948E2,1.9355932E1,1.1649485E0,-5.876695E-3,8.407724E6,1.8E1,1E0,3.6245735E0,5.905559E6,8E0,1.7103828E7,2.1153846E0,-5.3910734E-3,4.409194E3,-3.2886737E-3,1.5933178E-3,5.211574E-4,-5.9776385E-3,4.6064244E-3,-1.0761018E-3,1.0204082E0,3.83E2,1E0,-3.5428382E-3,6E0,-5.6105372E-3,5.3386245E0,1.0590052E8,4.093403E6,1.0918E4,1.1E1,1.8E1,-2.2971865E-3,4.3038647E-3,1E0,2.78E2,-1.6640241E-3,-8.39589E-3,9.531306E-3,2.4076016E5,-1.2863018E-3,-6.6335504E-3,2.591207E-4,-1.7334035E-3,3.4784111E-3,-7.129732E-4,-1.1256716E-3,3.0766055E-3,8.314572E-4,-2.3506384E-3,-2.9542877E-3,-3.2563982E-4,5.041627E-3,-3.201166E-4,6.163241E-3,-6.2690215E-4,2.120955E-3,-1.0564716E-3,-4.96715E-3,-5.8047596E-4,4.01065E-3,1.0508565E-3,-9.839364E-3,3.6470796E-4,2.918953E-3,7.2860536E-3],"split_indices":[5,3,29,60,67,69,8,19,0,68,46,1,3,8,9,67,0,68,0,73,57,0,57,0,0,8,60,0,72,4,73,68,0,9,3,26,71,66,3,62,68,0,48,0,0,0,0,0,0,68,12,30,0,8,0,69,7,66,9,3,3,0,0,30,0,0,0,0,48,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.03E2,6.27E2,2.76E2,2.5E1,6.02E2,1.17E2,1.59E2,2.3E1,2E0,1.73E2,4.29E2,1.09E2,8E0,1.29E2,3E1,2E1,3E0,1.68E2,5E0,8.9E1,3.4E2,3E0,1.06E2,4E0,4E0,8.3E1,4.6E1,1.1E1,1.9E1,1.1E1,9E0,1.63E2,5E0,6E1,2.9E1,2.13E2,1.27E2,9.7E1,9E0,7.1E1,1.2E1,2E0,4.4E1,1.5E1,4E0,6E0,5E0,4E0,5E0,1.1E1,1.52E2,5.8E1,2E0,2.4E1,5E0,9.7E1,1.16E2,2.8E1,9.9E1,6E1,3.7E1,2E0,7E0,5.8E1,1.3E1,9E0,3E0,7E0,3.7E1,6E0,5E0,5.9E1,9.3E1,5.3E1,5E0,1.3E1,1.1E1,7.7E1,2E1,7.1E1,4.5E1,2.1E1,7E0,6E0,9.3E1,1.7E1,4.3E1,2E1,1.7E1,4.4E1,1.4E1,2E0,1.1E1,2.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"95","size_leaf_vector":"1"}},{"base_weights":[-6.30358E-3,-2.0923525E-2,1.0892998E-2,-3.0379803E-3,-4.0354952E-2,1.574153E-2,-2.8745996E-2,4.940295E-3,-8.269415E-2,-1.0733867E-1,-2.1810174E-2,9.231812E-2,1.147907E-2,-5.1009413E-3,-1.1840586E-2,1.1439444E-2,-5.1409964E-2,-3.4675855E-2,-1.02767665E-2,-8.376924E-2,-1.7171968E-2,-1.7809395E-2,-1.2336396E-2,4.104991E-2,8.448856E-3,4.3733944E-3,4.8769724E-2,-1.1040544E-3,-6.691965E-3,5.5791494E-3,8.9595445E-2,-1.0195628E-2,-1.8561302E-2,-6.80601E-2,2.2788728E-3,-1.2029158E-1,1.6128864E-2,-6.112681E-3,-8.172887E-2,4.2968723E-3,-1.990668E-3,5.4315183E-2,-4.2210864E-3,6.547708E-2,-2.9688517E-3,4.5928457E-3,-5.806447E-4,1.2592469E-2,-3.1072707E-3,8.48789E-3,3.921854E-2,-3.2998607E-3,2.3906885E-3,1.7898768E-3,-4.679441E-3,-1.4955437E-1,-4.9303148E-2,4.843168E-2,-3.5688095E-3,6.368912E-3,-8.46958E-2,-1.0049024E-2,-5.159092E-2,1.2449022E-1,2.1916255E-2,-2.1577299E-2,1.4973942E-2,1.4989677E-1,2.6138404E-2,-2.0186468E-4,2.2708673E-3,-4.5088495E-4,4.643949E-3,-4.771574E-3,-9.301061E-3,-5.74285E-3,9.916374E-4,6.3316487E-3,-6.871921E-5,7.835151E-4,-2.4793395E-3,-1.748726E-3,-1.3358031E-2,-6.4599635E-3,-1.3183046E-3,3.3496968E-3,9.511873E-3,2.6690252E-3,-4.1214883E-4,-5.2977254E-4,-7.0143426E-3,2.5752326E-3,-5.5183955E-6,2.726112E-3,9.2458585E-3,-2.6401E-3,2.8001505E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":94,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,33,-1,35,-1,37,-1,39,-1,41,43,45,-1,47,49,-1,51,53,-1,55,57,59,61,-1,-1,63,65,67,-1,-1,-1,69,-1,-1,71,-1,-1,-1,-1,73,75,77,-1,79,81,-1,83,85,87,89,91,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3404202E-1,1.7469087E-1,8.271025E-2,1.6754998E-1,2.9702866E-1,1.2362685E-1,6.300762E-2,8.848298E-2,1.516919E-1,2.900461E-1,1.8033053E-1,8.415699E-2,9.597274E-2,0E0,5.655115E-2,9.843184E-2,1.3374346E-1,5.641062E-2,0E0,1.8071732E-1,0E0,1.4005819E-1,0E0,6.422585E-2,0E0,1.3157173E-1,1.1329758E-1,4.4419296E-2,0E0,1.03397384E-1,6.305023E-2,0E0,8.429114E-2,5.2435145E-2,0E0,6.740999E-2,4.5713656E-2,1.5747422E-1,1.0626933E-1,0E0,0E0,9.958346E-2,8.760919E-2,1.6545084E-1,0E0,0E0,0E0,1.13722816E-1,0E0,0E0,3.359262E-2,0E0,0E0,0E0,0E0,3.868878E-2,6.5499686E-2,4.835447E-2,0E0,8.7818734E-2,1.990359E-1,0E0,4.6800748E-2,4.4054747E-2,3.5445843E-2,1.7995994E-1,7.622114E-2,5.977401E-2,1.0510336E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,19,19,21,21,23,23,25,25,26,26,27,27,29,29,30,30,32,32,33,33,35,35,36,36,37,37,38,38,41,41,42,42,43,43,47,47,50,50,55,55,56,56,57,57,59,59,60,60,62,62,63,63,64,64,65,65,66,66,67,67,68,68],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,34,-1,36,-1,38,-1,40,-1,42,44,46,-1,48,50,-1,52,54,-1,56,58,60,62,-1,-1,64,66,68,-1,-1,-1,70,-1,-1,72,-1,-1,-1,-1,74,76,78,-1,80,82,-1,84,86,88,90,92,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.512E4,4.28649E5,2.7555334E7,5.194E3,1.118178E6,1.1015E4,7.937392E-3,1E0,8E0,2.41E3,3.07E2,3.8153846E0,1.28636E7,-5.1009413E-3,4.533E3,3.289E3,1.5933333E2,1.1358E4,-1.02767665E-2,2.9077E4,-1.7171968E-2,2.1651245E6,-1.2336396E-2,5.1034E4,8.448856E-3,2.165E3,3.4338624E0,1.2157188E6,-6.691965E-3,1.15E2,2.3625128E6,-1.0195628E-2,9.5E1,1.0183612E2,2.2788728E-3,1.392E1,2.6095572E0,1.0130841E1,2.2823529E2,4.2968723E-3,-1.990668E-3,1.8454931E8,1.3992583E10,9.275615E4,-2.9688517E-3,4.5928457E-3,-5.806447E-4,1.1269E4,-3.1072707E-3,8.48789E-3,3.3512E4,-3.2998607E-3,2.3906885E-3,1.7898768E-3,-4.679441E-3,8.15792E5,1.2999985E6,4.5593776E7,-3.5688095E-3,1.45752E8,1.2954694E6,-1.0049024E-2,9.024187E6,9.558022E7,7.6E0,2.1373269E8,1.5215946E0,1.72959E5,7.457162E2,-2.0186468E-4,2.2708673E-3,-4.5088495E-4,4.643949E-3,-4.771574E-3,-9.301061E-3,-5.74285E-3,9.916374E-4,6.3316487E-3,-6.871921E-5,7.835151E-4,-2.4793395E-3,-1.748726E-3,-1.3358031E-2,-6.4599635E-3,-1.3183046E-3,3.3496968E-3,9.511873E-3,2.6690252E-3,-4.1214883E-4,-5.2977254E-4,-7.0143426E-3,2.5752326E-3,-5.5183955E-6,2.726112E-3,9.2458585E-3,-2.6401E-3,2.8001505E-3],"split_indices":[1,9,60,44,9,9,57,30,8,2,0,69,60,0,8,44,4,44,0,1,0,62,0,44,0,44,68,62,0,0,60,0,0,48,0,73,68,71,67,0,0,7,46,48,0,0,0,1,0,0,1,0,0,0,0,9,60,5,0,7,43,0,5,7,71,5,69,12,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.29E2,5.02E2,4.27E2,2.62E2,2.4E2,3.81E2,4.6E1,2.39E2,2.3E1,5.1E1,1.89E2,1.9E1,3.62E2,7E0,3.9E1,2.15E2,2.4E1,1.8E1,5E0,4.8E1,3E0,1.87E2,2E0,1.3E1,6E0,3.05E2,5.7E1,3.7E1,2E0,2.01E2,1.4E1,3E0,2.1E1,1.3E1,5E0,3.5E1,1.3E1,1.59E2,2.8E1,8E0,5E0,4.4E1,2.61E2,5E1,7E0,3E0,3.4E1,1.84E2,1.7E1,4E0,1E1,1.2E1,9E0,3E0,1E1,2.4E1,1.1E1,1E1,3E0,1.38E2,2.1E1,4E0,2.4E1,1.3E1,3.1E1,1.37E2,1.24E2,1.5E1,3.5E1,1.26E2,5.8E1,6E0,4E0,1.4E1,1E1,5E0,6E0,3E0,7E0,1.18E2,2E1,1.8E1,3E0,4E0,2E1,9E0,4E0,1.4E1,1.7E1,1.28E2,9E0,3.3E1,9.1E1,6E0,9E0,1E1,2.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-3.2928148E-3,-8.9045725E-4,-5.2291147E-2,1.0904275E-2,-1.1658719E-2,-6.8773404E-2,3.880231E-3,4.7997953E-3,6.8852015E-2,-5.197804E-2,3.428625E-4,-2.4518932E-3,-7.6356074E-3,1.1504429E-2,-4.0696163E-2,-5.937498E-3,1.21137656E-1,8.112969E-3,-5.8188815E-2,-1.34184E-1,3.1034055E-3,1.4439434E-3,5.250979E-2,-5.1798854E-2,3.0298182E-3,5.071628E-3,-3.836682E-2,8.552882E-3,4.8277523E-2,-4.2955276E-2,-1.4978614E-1,-2.2827797E-3,-1.0756702E-2,8.526106E-2,-2.2810623E-3,4.177286E-3,-8.180727E-3,4.0343483E-3,1.081716E-2,-4.234579E-2,-7.760974E-3,-3.1536142E-3,2.427222E-3,-9.830436E-4,3.9823083E-3,-3.135483E-2,-8.03231E-3,-7.8600414E-2,-1.788683E-2,9.39643E-4,7.89507E-3,2.496126E-2,-1.8239463E-2,4.7400434E-4,-2.8407138E-3,-3.2850597E-3,2.2829995E-3,-1.5317268E-3,-7.148574E-3,-4.7701397E-3,-7.65123E-4,-5.237665E-3,2.8368249E-3,1.3488184E-3,-6.5553775E-3,-3.2025648E-3,-3.3873282E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":95,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,-1,21,23,25,27,-1,29,31,33,35,37,39,-1,-1,41,-1,43,45,47,-1,-1,49,51,53,-1,-1,55,57,-1,-1,-1,-1,-1,59,-1,61,-1,-1,-1,63,65,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0577609E-1,1.0910065E-1,9.969308E-2,1.4470325E-1,2.1728694E-1,4.954405E-2,0E0,1.13940746E-1,1.5322557E-1,1.5499511E-1,1.2928084E-1,0E0,0E0,1.339977E-1,5.978132E-2,6.757604E-2,9.630269E-2,0E0,1.3449642E-1,4.6492964E-2,1.5104444E-1,1.277378E-1,9.0415195E-2,4.0797904E-2,0E0,0E0,4.086502E-2,0E0,3.2921277E-2,1.2704313E-1,2.037862E-1,0E0,0E0,1.1201067E-1,1.4000806E-1,1.02281794E-1,0E0,0E0,9.698647E-2,3.895382E-2,0E0,0E0,0E0,0E0,0E0,8.533857E-2,0E0,6.385739E-2,0E0,0E0,0E0,8.38699E-2,1.1208527E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,26,26,28,28,29,29,30,30,33,33,34,34,35,35,38,38,39,39,45,45,47,47,51,51,52,52],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,-1,22,24,26,28,-1,30,32,34,36,38,40,-1,-1,42,-1,44,46,48,-1,-1,50,52,54,-1,-1,56,58,-1,-1,-1,-1,-1,60,-1,62,-1,-1,-1,64,66,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.74E1,3.72381E5,1.5E1,7.2132964E0,1.084437E6,1.1424088E3,3.880231E-3,1.9434364E7,5.0149255E0,3.7289145E4,1.424196E-1,-2.4518932E-3,-7.6356074E-3,1.1774061E5,6.1454544E0,3.4111922E0,1.4715686E1,8.112969E-3,9.44363E5,6.2E1,1.4076087E0,3.6E1,2.3017536E5,2.1199985E-7,3.0298182E-3,5.071628E-3,1.317484E1,8.552882E-3,1.273782E4,2.6591675E0,2.41E3,-2.2827797E-3,-1.0756702E-2,2.423301E1,1.2283186E1,1E0,-8.180727E-3,4.0343483E-3,2.5603516E5,8.277983E-1,-7.760974E-3,-3.1536142E-3,2.427222E-3,-9.830436E-4,3.9823083E-3,9.1E1,-8.03231E-3,1.839E3,-1.788683E-2,9.39643E-4,7.89507E-3,1E0,2.8655008E2,4.7400434E-4,-2.8407138E-3,-3.2850597E-3,2.2829995E-3,-1.5317268E-3,-7.148574E-3,-4.7701397E-3,-7.65123E-4,-5.237665E-3,2.8368249E-3,1.3488184E-3,-6.5553775E-3,-3.2025648E-3,-3.3873282E-4],"split_indices":[61,9,8,69,9,70,0,60,71,60,73,0,0,48,69,68,73,0,9,0,69,3,48,52,0,0,69,0,48,53,2,0,0,73,73,19,0,0,48,49,0,0,0,0,0,10,0,2,0,0,0,79,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[8.98E2,8.57E2,4.1E1,4.09E2,4.48E2,3.7E1,4E0,3.71E2,3.8E1,1.02E2,3.46E2,3.3E1,4E0,3.24E2,4.7E1,1.6E1,2.2E1,2E0,1E2,6E0,3.4E2,2.61E2,6.3E1,4.3E1,4E0,3E0,1.3E1,1.1E1,1.1E1,8.7E1,1.3E1,4E0,2E0,2E1,3.2E2,2.58E2,3E0,3.4E1,2.9E1,4.1E1,2E0,1E1,3E0,4E0,7E0,8.1E1,6E0,1.1E1,2E0,1.2E1,8E0,1.18E2,2.02E2,2.37E2,2.1E1,9E0,2E1,3.9E1,2E0,1.3E1,6.8E1,9E0,2E0,1.16E2,2E0,3.5E1,1.67E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"67","size_leaf_vector":"1"}},{"base_weights":[1.7231015E-3,-1.0223471E-2,1.9426677E-2,-8.221187E-3,-1.06495894E-1,8.4045924E-2,1.4861885E-2,-2.825233E-3,-4.120675E-2,-3.0927334E-4,-6.655554E-3,-1.3940921E-4,1.3647185E-1,-3.3243973E-2,2.0592583E-2,-5.157301E-3,8.466842E-2,-8.4178045E-2,6.174722E-4,3.9251902E-4,7.8028785E-3,4.422485E-3,-8.29576E-2,9.696218E-2,1.5518048E-2,-9.36279E-3,3.5818815E-2,5.4835835E-3,-2.5277208E-3,2.9326328E-3,-9.439312E-2,-1.6903793E-2,7.3553817E-3,3.1040371E-3,-8.2936964E-4,-5.1627825E-3,1.3464161E-3,1.1979832E-1,-3.0119834E-3,-2.3331216E-2,2.3513671E-2,7.244889E-3,-2.1376114E-2,5.4102484E-2,-5.1298244E-3,-4.8628215E-2,-1.303726E-1,-5.566065E-2,2.0702904E-2,1.4062145E-1,-1.1278157E-3,-3.2226846E-2,2.938108E-3,8.643046E-2,1.7739573E-2,-8.09364E-4,1.5474698E-3,-4.866865E-3,-6.342961E-4,4.5358418E-3,1.0017911E-3,-4.403345E-3,6.983228E-4,-2.2810705E-3,-7.633497E-3,-3.969604E-3,8.3488366E-4,1.7510798E-3,-3.824788E-3,2.7738747E-3,8.145619E-3,-4.545371E-3,-7.377924E-4,4.6918904E-5,7.265839E-3,1.214123E-3,-1.2885621E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":96,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,-1,19,21,23,25,27,29,31,-1,-1,33,35,37,39,41,43,-1,-1,-1,45,47,-1,-1,-1,-1,-1,49,-1,51,53,55,57,59,-1,61,63,65,67,69,-1,71,-1,73,75,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9266091E-1,1.03988536E-1,1.0700618E-1,9.492947E-2,3.9258383E-2,1.0992913E-1,9.551853E-2,9.429559E-2,1.3484506E-1,0E0,0E0,0E0,5.870396E-2,6.9650725E-2,1.17857724E-1,7.7709116E-2,6.1858483E-2,6.1600775E-2,1.1032307E-1,0E0,0E0,3.1703446E-2,5.3441517E-2,7.7929035E-2,9.062957E-2,8.155635E-2,1.18174896E-1,0E0,0E0,0E0,5.1291168E-2,5.362296E-2,0E0,0E0,0E0,0E0,0E0,5.4402277E-2,0E0,4.0512215E-2,8.60191E-2,1.1092499E-1,1.5127456E-1,5.1633544E-2,0E0,5.1247034E-2,4.411784E-2,4.087642E-2,3.6246512E-2,3.148967E-2,0E0,4.775643E-2,0E0,1.1908874E-1,8.513161E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,12,12,13,13,14,14,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,30,30,31,31,37,37,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,-1,20,22,24,26,28,30,32,-1,-1,34,36,38,40,42,44,-1,-1,-1,46,48,-1,-1,-1,-1,-1,50,-1,52,54,56,58,60,-1,62,64,66,68,70,-1,72,-1,74,76,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,3.713753E7,1.0164831E-2,2E1,8.474894E-9,5.941442E6,2.778995E2,1.7E1,9.66E2,-3.0927334E-4,-6.655554E-3,-1.3940921E-4,2.583E3,3.716E3,3.92E2,1.3131897E5,1.88E2,1.2463E4,4.1E1,3.9251902E-4,7.8028785E-3,1.71E2,5.603014E6,2.9487667E0,1.361E3,3.5E1,6.354E3,5.4835835E-3,-2.5277208E-3,2.9326328E-3,1.47460895E-2,5.735591E2,7.3553817E-3,3.1040371E-3,-8.2936964E-4,-5.1627825E-3,1.3464161E-3,5.9908E6,-3.0119834E-3,2.6E1,1.2877E4,2.35E2,5.8182236E1,1.4715686E1,-5.1298244E-3,5.4051723E0,7.23E2,1E0,5.444797E0,4.23E2,-1.1278157E-3,5.281628E2,2.938108E-3,3.8737908E-1,1.7493458E7,-8.09364E-4,1.5474698E-3,-4.866865E-3,-6.342961E-4,4.5358418E-3,1.0017911E-3,-4.403345E-3,6.983228E-4,-2.2810705E-3,-7.633497E-3,-3.969604E-3,8.3488366E-4,1.7510798E-3,-3.824788E-3,2.7738747E-3,8.145619E-3,-4.545371E-3,-7.377924E-4,4.6918904E-5,7.265839E-3,1.214123E-3,-1.2885621E-3],"split_indices":[27,60,53,8,52,60,70,8,10,0,0,0,44,44,10,48,10,1,3,0,0,10,9,68,44,0,10,0,0,0,53,48,0,0,0,0,0,9,0,8,9,2,62,73,0,69,44,16,68,44,0,4,0,53,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.09E2,5.43E2,3.66E2,5.33E2,1E1,2.3E1,3.43E2,4.59E2,7.4E1,3E0,7E0,9E0,1.4E1,3.6E1,3.07E2,4.48E2,1.1E1,3.6E1,3.8E1,3E0,1.1E1,2.1E1,1.5E1,1.8E1,2.89E2,4.07E2,4.1E1,9E0,2E0,2E0,3.4E1,3.5E1,3E0,5E0,1.6E1,1.2E1,3E0,1.6E1,2E0,4.9E1,2.4E2,1.71E2,2.36E2,3.7E1,4E0,1.6E1,1.8E1,1.7E1,1.8E1,1.4E1,2E0,4.5E1,4E0,1.9E1,2.21E2,8.8E1,8.3E1,1.9E1,2.17E2,1.5E1,2.2E1,9E0,7E0,6E0,1.2E1,1.2E1,5E0,1.6E1,2E0,5E0,9E0,8E0,3.7E1,9E0,1E1,1.87E2,3.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"77","size_leaf_vector":"1"}},{"base_weights":[-1.0260734E-3,5.968505E-4,-1.1280405E-1,-8.899347E-3,1.5506075E-2,-1.5074742E-1,2.972902E-3,-2.6181466E-3,-1.2122089E-1,1.21867366E-1,1.1684058E-2,-1.9878251E-3,-2.0635882E-1,-7.999251E-2,-3.1021561E-6,-2.70153E-1,-5.162042E-2,1.8353438E-3,8.143E-3,3.2908835E-2,-3.1255814E-3,-1.1941946E-2,-2.490335E-3,-5.743899E-3,2.2852612E-3,3.3697058E-2,-8.956019E-3,-1.2725256E-1,-2.388333E-2,-8.205966E-2,7.365322E-3,2.3765614E-2,1.196996E-1,-3.2380883E-2,1.3227187E-2,2.3090336E-2,6.195392E-3,-8.745515E-2,-4.8009166E-3,-1.1033801E-2,6.7693763E-4,-1.0381468E-1,1.1847955E-3,4.2769402E-2,-3.2270915E-4,-3.4357698E-3,7.5161364E-3,-1.9840274E-2,-9.136737E-3,4.5118935E-2,-1.4297555E-2,1.6819604E-3,-2.109677E-3,-5.9014712E-3,-5.784274E-4,-1.157268E-3,4.912185E-4,-1.9459221E-4,-5.882487E-3,2.284104E-3,-2.3617574E-3,-1.3003688E-3,2.0889908E-3,-3.1633582E-3,8.2450226E-4,2.9389071E-3,-3.967804E-4,-6.6501773E-3,1.05639636E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":97,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,-1,21,23,25,27,29,-1,-1,31,33,-1,-1,-1,-1,35,37,39,-1,41,-1,43,45,47,49,51,-1,53,55,-1,-1,57,-1,59,61,-1,-1,63,-1,65,67,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6478501E-1,1.269945E-1,9.680922E-2,3.8630435E-1,1.4044969E-1,5.724953E-2,0E0,1.052152E-1,2.8666115E-1,4.5719266E-2,1.0634683E-1,0E0,4.0179074E-2,1.0158678E-1,1.5236601E-1,2.4766743E-1,1.4722541E-1,0E0,0E0,1.0745826E-1,9.610933E-2,0E0,0E0,0E0,0E0,1.0881054E-1,1.2972273E-1,1.12640284E-1,0E0,4.7093794E-2,0E0,5.81505E-2,1.1740823E-1,1.4425044E-1,1.13882855E-1,8.84567E-2,0E0,5.7592183E-2,1.175036E-1,0E0,0E0,3.5608485E-2,0E0,4.4626057E-2,7.252445E-2,0E0,0E0,1.2428418E-1,0E0,5.8716826E-2,1.4958769E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,19,19,20,20,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,37,37,38,38,41,41,43,43,44,44,47,47,49,49,50,50],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,-1,22,24,26,28,30,-1,-1,32,34,-1,-1,-1,-1,36,38,40,-1,42,-1,44,46,48,50,52,-1,54,56,-1,-1,58,-1,60,62,-1,-1,64,-1,66,68,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8652172E1,1.131645E6,1E2,9.07021E5,1.1160929E0,4.9E1,2.972902E-3,8.071428E0,2E0,2.06E2,2.992306E4,-1.9878251E-3,6.2E1,4.2909092E1,1.5310282E8,4.6066E4,2.6172416E10,1.8353438E-3,8.143E-3,2.823793E10,4.306E4,-1.1941946E-2,-2.490335E-3,-5.743899E-3,2.2852612E-3,1.1146532E8,7.635E3,1.631273E6,-2.388333E-2,1.6135485E1,7.365322E-3,1E0,5.293933E2,1E0,4.843E3,1.11E2,6.195392E-3,8.599521E5,1E0,-1.1033801E-2,6.7693763E-4,3.8709676E0,1.1847955E-3,2.2092846E7,2.0164769E5,-3.4357698E-3,7.5161364E-3,3.328629E6,-9.136737E-3,3.1095755E0,6.648E3,1.6819604E-3,-2.109677E-3,-5.9014712E-3,-5.784274E-4,-1.157268E-3,4.912185E-4,-1.9459221E-4,-5.882487E-3,2.284104E-3,-2.3617574E-3,-1.3003688E-3,2.0889908E-3,-3.1633582E-3,8.2450226E-4,2.9389071E-3,-3.967804E-4,-6.6501773E-3,1.05639636E-4],"split_indices":[68,9,3,9,69,3,0,67,8,10,48,0,0,48,46,1,46,0,0,46,1,0,0,0,0,46,1,12,0,71,0,26,48,19,44,10,0,43,29,0,0,71,0,60,62,0,0,9,0,53,44,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.07E2,8.95E2,1.2E1,5.47E2,3.48E2,1E1,2E0,5.19E2,2.8E1,1.1E1,3.37E2,4E0,6E0,1.6E1,5.03E2,8E0,2E1,5E0,6E0,1.38E2,1.99E2,4E0,2E0,1.2E1,4E0,1.05E2,3.98E2,6E0,2E0,1.8E1,2E0,1.26E2,1.2E1,7.1E1,1.28E2,9.6E1,9E0,1.9E1,3.79E2,3E0,3E0,1.5E1,3E0,7E1,5.6E1,2E0,1E1,6.7E1,4E0,5.9E1,6.9E1,8.1E1,1.5E1,1.2E1,7E0,1.64E2,2.15E2,3E0,1.2E1,6.6E1,4E0,3.5E1,2.1E1,2.9E1,3.8E1,4.4E1,1.5E1,7E0,6.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"69","size_leaf_vector":"1"}},{"base_weights":[3.1239066E-5,-3.0996474E-3,7.7586874E-2,-9.38208E-3,2.011117E-2,1.5516113E-1,1.848208E-2,-4.8533846E-3,-4.6586435E-2,6.629291E-2,7.320891E-3,1.8424547E-1,1.3542006E-3,1.7847982E-3,-3.5189192E-3,-1.345487E-2,2.23299E-2,-1.2904747E-1,-1.7776683E-2,8.499314E-2,-3.7953588E-3,4.5412374E-4,5.199449E-3,3.928018E-3,1.1322516E-2,-8.823623E-3,-4.703925E-2,1.5075535E-2,5.5905436E-3,-6.4948335E-2,-2.0893998E-2,3.798953E-3,-8.943448E-2,2.2012918E-4,1.11571155E-1,2.9733265E-2,-1.930595E-2,-1.2146519E-2,6.5208375E-3,-1.934643E-1,-2.8890438E-2,3.5554625E-3,6.700404E-2,-1.009679E-2,-3.0188126E-2,-7.202289E-3,4.2344728E-3,2.2727605E-3,-7.270639E-3,3.3718324E-3,1.816564E-1,4.557869E-2,-2.70113E-3,-3.171575E-3,-4.7183447E-3,-3.8388843E-4,-2.8736747E-3,-2.2716892E-3,-1.4966192E-2,1.86144E-3,-2.7307742E-3,1.3527569E-3,-1.1329971E-3,4.6234243E-3,-7.703652E-5,1.4265303E-3,-5.887492E-3,8.839875E-4,-3.2528397E-3,1.0010892E-2,1.986449E-3,2.5316828E-3,-1.1856927E-3,1.1198496E-3,-2.0292073E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":98,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,29,31,33,-1,35,-1,-1,-1,37,39,41,-1,43,-1,45,47,-1,49,51,53,55,-1,57,59,61,63,-1,65,67,-1,-1,-1,-1,69,71,-1,-1,73,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1901941E-1,1.2661497E-1,1.5760767E-1,1.1485067E-1,1.08789995E-1,4.962343E-2,4.2236056E-2,1.4292859E-1,1.7350644E-1,1.1756681E-1,1.04692444E-1,5.0337404E-2,0E0,0E0,0E0,7.175438E-2,1.02563344E-1,3.608031E-1,8.6919256E-2,7.6265305E-2,0E0,8.0415025E-2,0E0,0E0,0E0,2.0424962E-1,1.4322741E-1,8.2316145E-2,0E0,8.335611E-2,0E0,4.2889602E-2,1.2791812E-1,0E0,6.0762852E-2,8.0567166E-2,5.8871455E-2,7.697113E-2,0E0,9.564094E-2,1.0717948E-1,8.196315E-2,5.57938E-2,0E0,9.18919E-2,6.710508E-2,0E0,0E0,0E0,0E0,3.2818377E-2,3.1604454E-2,0E0,0E0,7.388946E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,21,21,25,25,26,26,27,27,29,29,31,31,32,32,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,45,45,50,50,51,51,54,54],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,30,32,34,-1,36,-1,-1,-1,38,40,42,-1,44,-1,46,48,-1,50,52,54,56,-1,58,60,62,64,-1,66,68,-1,-1,-1,-1,70,72,-1,-1,74,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.4915986E3,5.044E3,2.2040408E7,4.75356E6,3.84E2,1.9407515E9,5.619497E7,1E0,1.4693878E0,1.853776E7,1E0,2.880929E6,1.3542006E-3,1.7847982E-3,-3.5189192E-3,2.87E2,1.7743998E5,2.379747E0,1.1E1,5.323779E6,-3.7953588E-3,2.2993813E0,5.199449E-3,3.928018E-3,1.1322516E-2,2.57E2,5.1942E4,1.2432787E1,5.5905436E-3,1.26E2,-2.0893998E-2,1.7500046E8,1.9052632E0,2.2012918E-4,1.6E1,1.4569893E0,2.6095572E0,3.1E1,6.5208375E-3,4.3992E4,2.6210527E0,3.8709676E0,1.9196894E6,-1.009679E-2,1.4026549E0,9.946467E7,4.2344728E-3,2.2727605E-3,-7.270639E-3,3.3718324E-3,4.01969E5,2.7599E4,-2.70113E-3,-3.171575E-3,1.212945E6,-3.8388843E-4,-2.8736747E-3,-2.2716892E-3,-1.4966192E-2,1.86144E-3,-2.7307742E-3,1.3527569E-3,-1.1329971E-3,4.6234243E-3,-7.703652E-5,1.4265303E-3,-5.887492E-3,8.839875E-4,-3.2528397E-3,1.0010892E-2,1.986449E-3,2.5316828E-3,-1.1856927E-3,1.1198496E-3,-2.0292073E-3],"split_indices":[4,10,62,62,0,12,62,27,68,62,30,1,0,0,0,0,48,69,8,12,0,68,0,0,0,0,1,71,0,10,0,12,68,0,3,61,68,3,0,1,71,71,62,0,68,12,0,0,0,0,1,2,0,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9E2,8.66E2,3.4E1,6.82E2,1.84E2,1.4E1,2E1,6.09E2,7.3E1,3.9E1,1.45E2,1.1E1,3E0,1.7E1,3E0,4.63E2,1.46E2,1.8E1,5.5E1,3.5E1,4E0,1.37E2,8E0,5E0,6E0,4.08E2,5.5E1,1.37E2,9E0,1.6E1,2E0,4.3E1,1.2E1,9E0,2.6E1,5.5E1,8.2E1,4E2,8E0,5E0,5E1,1.13E2,2.4E1,2E0,1.4E1,3.9E1,4E0,4E0,8E0,1.8E1,8E0,4.7E1,8E0,1.8E1,6.4E1,3.72E2,2.8E1,3E0,2E0,1.5E1,3.5E1,5.9E1,5.4E1,1.6E1,8E0,9E0,5E0,2.8E1,1.1E1,6E0,2E0,4.2E1,5E0,3.7E1,2.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"75","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,0,3]} \ No newline at end of file +{"learner":{"attributes":{},"feature_names":["numTasks_sum","duration_sum","duration_max","duration_min","duration_mean","input_recordsRead_sum","output_recordsWritten_sum","peakExecutionMemory_max","resultSerializationTime_sum","resultSize_max","sr_localBlocksFetched_sum","sr_remoteBlocksFetched_sum","sw_recordsWritten_sum","sqlOp_AQEShuffleRead","sqlOp_BroadcastExchange","sqlOp_BroadcastHashJoin","sqlOp_ColumnarToRow","sqlOp_Exchange","sqlOp_Filter","sqlOp_HashAggregate","sqlOp_LocalTableScan","sqlOp_Project","sqlOp_Scan parquet","sqlOp_Sort","sqlOp_SortMergeJoin","sqlOp_Subquery","numExecutors","executorCores","maxMem","maxOnHeapMem","maxOffHeapMem","executorMemory","numGpusPerExecutor","executorOffHeap","taskCpu","taskGpu","cache_hit_ratio","scan_bw","scan_time","decode_time","data_size","shuffle_read_bw","shuffle_write_bw","diskBytesSpilledRatio","memoryBytesSpilledRatio","input_bytesReadRatio","output_bytesWrittenRatio","sr_localBytesReadRatio","sr_remoteBytesReadRatio","sr_remoteBytesReadToDiskRatio","sr_totalBytesReadRatio","sw_bytesWrittenRatio","diskBytesSpilled_mean","memoryBytesSpilled_mean","input_bytesRead_mean","output_bytesWritten_mean","sr_localBytesRead_mean","sr_remoteBytesRead_mean","sr_remoteBytesReadToDisk_mean","sr_totalBytesRead_mean","sw_bytesWritten_mean","executorCPUTime_mean","executorDeserializeCPUTime_mean","executorDeserializeTime_mean","executorRunTime_mean","jvmGCTime_mean","sr_fetchWaitTime_mean","sw_writeTime_mean","platform_onprem","platform_databricks-aws","platform_databricks-azure","platform_dataproc","platform_emr","sqlOp_BatchEvalPython","sqlOp_BroadcastNestedLoopJoin","sqlOp_CartesianProduct","sqlOp_CommandResult","sqlOp_CustomShuffleReader","sqlOp_DeserializeToObject","sqlOp_Execute InsertIntoHadoopFsRelationCommand csv","sqlOp_Execute InsertIntoHadoopFsRelationCommand json","sqlOp_Execute InsertIntoHadoopFsRelationCommand orc","sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet","sqlOp_Execute InsertIntoHadoopFsRelationCommand text","sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown","sqlOp_Expand","sqlOp_Generate","sqlOp_GenerateBloomFilter","sqlOp_GlobalLimit","sqlOp_HashAggregatePrefixGroupingSets","sqlOp_LocalLimit","sqlOp_MapElements","sqlOp_ObjectHashAggregate","sqlOp_OutputAdapter","sqlOp_PartialWindow","sqlOp_ReusedSort","sqlOp_RunningWindowFunction","sqlOp_Scan ExistingRDD","sqlOp_Scan ExistingRDD Delta Table Checkpoint","sqlOp_Scan ExistingRDD Delta Table State","sqlOp_Scan OneRowRelation","sqlOp_Scan csv","sqlOp_Scan jdbc","sqlOp_Scan json","sqlOp_Scan orc","sqlOp_Scan text","sqlOp_Scan unknown","sqlOp_SerializeFromObject","sqlOp_SortAggregate","sqlOp_SubqueryBroadcast","sqlOp_SubqueryOutputBroadcast","sqlOp_TakeOrderedAndProject","sqlOp_Window","sqlOp_WindowGroupLimit","sqlOp_WindowSort"],"feature_types":["int","int","int","int","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","float","float","int","int","int","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","float","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int","int"],"gradient_booster":{"model":{"gbtree_model_param":{"num_parallel_tree":"1","num_trees":"94"},"iteration_indptr":[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,34,35,36,37,38,39,40,41,42,43,44,45,46,47,48,49,50,51,52,53,54,55,56,57,58,59,60,61,62,63,64,65,66,67,68,69,70,71,72,73,74,75,76,77,78,79,80,81,82,83,84,85,86,87,88,89,90,91,92,93,94],"tree_info":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"trees":[{"base_weights":[7.919761E-3,-2.9362088E-1,5.7550114E-1,-7.2065884E-1,-8.0954336E-2,6.445587E-1,-7.898262E-1,-9.3067086E-1,-5.632717E-1,-2.1908195E-1,1.5221426E-1,3.9592466E-1,1.0153017E0,-4.7825832E-2,-2.2639198E-2,-9.606087E-1,-1.7464165E-2,-7.3615515E-1,-2.864444E-1,-3.1109965E-1,1.15108E-1,3.0246064E-1,1.9423149E-3,2.1051852E-1,6.8003935E-1,1.2382919E0,6.0126776E-1,-9.8947155E-1,-2.6779842E-2,-6.407794E-1,-5.8621336E-2,2.6945805E-2,-4.0350723E-1,-5.220097E-1,-2.300714E-1,-6.846113E-3,1.495776E-1,3.3621147E-1,-6.836232E-3,3.7935756E-2,-1.2723793E-2,4.043556E-1,7.824283E-2,8.64719E-1,4.6620974E-1,1.347073E0,4.4336614E-1,4.7008625E-1,8.880156E-1,-1.0298125E0,-3.1016912E-2,-3.5690373E-1,-3.2506287E-2,-4.8727587E-1,-1.23134136E-1,-5.536625E-1,-1.0480105E-2,-7.509546E-1,-1.865693E-1,2.9940847E-1,6.331902E-2,2.6484653E-1,5.3162795E-1,-1.4801665E-1,8.79603E-2,3.0492514E-1,3.2155007E-2,1.2173922E-1,-1.2464481E-2,4.4441346E-2,5.832269E-1,-7.781936E-3,5.7386607E-1,1.4283348E0,2.5469026E-2,9.7812135E-3,2.7933981E-2,5.7786405E-1,2.5982314E-1,5.285293E-2,2.3667667E-2,-4.1863903E-2,-5.5656366E-2,-2.1042373E-2,-7.823764E-3,-1.2001296E-2,-5.654917E-1,-5.064279E-4,-9.968048E-3,-1.7026514E-2,-2.7749281E-2,-4.4332057E-2,-1.954833E-2,-2.3689865E-1,-3.878699E-2,3.8886194E-3,1.7440848E-2,-2.910079E-3,1.061596E-1,1.8755749E-1,3.9734963E-1,1.2142001E-2,3.5843324E-2,-1.1577309E-2,2.1258057E-3,1.3349657E-2,5.283588E-2,3.685677E-1,-3.46676E-3,3.5460785E-2,2.4801287E-1,1.6773267E-2,3.2013852E-2,1.1181386E-2,6.609166E-1,1.169259E0,1.7011678E0,9.1130175E-3,3.3284824E-2,7.4468837E-3,1.6191373E-2,-3.0336034E-2,-1.2538912E-2,-1.2329962E-2,-8.8330044E-4,-1.5721095E-4,-9.985902E-3,3.0995882E-3,1.0191964E-2,2.2585941E-3,1.2700595E-2,1.09897405E-2,2.0311462E-2,-1.8001337E-3,6.4351107E-3,3.3353248E-3,2.2581259E-2,-2.1705728E-3,7.027927E-3,2.624085E-4,1.4305838E-2,3.4121983E-2,1.7398806E-2,5.8625236E-2,2.8388632E-2,8.9003E-2,5.1907178E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":0,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,-1,27,-1,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,-1,53,55,57,-1,59,61,-1,63,-1,65,67,69,71,73,75,77,79,81,-1,83,-1,85,87,89,-1,91,93,95,97,99,101,103,105,107,-1,109,-1,-1,111,-1,113,115,-1,-1,-1,117,119,-1,-1,-1,-1,-1,-1,-1,121,-1,-1,-1,-1,-1,-1,123,125,-1,-1,-1,127,129,131,-1,-1,-1,-1,-1,133,135,-1,137,139,-1,-1,-1,141,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5933742E2,5.5158207E1,3.096354E1,6.152519E0,1.3141605E1,2.798925E1,6.608248E-1,1.0489349E0,5.4364815E0,7.908165E0,3.43124E0,9.654686E0,1.0611496E1,0E0,0E0,3.143158E-1,0E0,2.8111038E0,5.005516E0,3.3149242E0,5.2907294E-1,1.2032886E0,7.757942E-1,2.8913136E0,2.4796257E0,6.2340927E0,1.2752762E0,2.568817E-1,0E0,8.1936455E-1,0E0,0E0,9.2643166E-1,3.9050102E-1,3.1908426E0,0E0,6.272516E-1,8.4935856E-1,0E0,6.5487033E-1,0E0,1.1089091E0,1.0737693E0,4.3704605E-1,2.5531082E0,3.6913605E0,2.7898693E-1,5.9769344E-1,6.525011E-1,2.8147125E-1,0E0,1.5998411E-1,0E0,4.2595816E-1,1.1084774E-1,1.8664932E-1,0E0,3.3897495E-1,1.0097885E0,2.652899E-1,1.848416E-1,4.7848988E-1,9.6822023E-1,3.0653366E-1,3.7068713E-1,8.973446E-1,0E0,6.666761E-1,0E0,0E0,3.457737E-2,0E0,7.3904324E-1,2.3933182E0,0E0,0E0,0E0,9.76521E-1,5.9088647E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.2301798E-1,0E0,0E0,0E0,0E0,0E0,0E0,6.357479E-1,2.241044E-1,0E0,0E0,0E0,9.700024E-2,4.1747785E-1,1.9526005E-2,0E0,0E0,0E0,0E0,0E0,3.829953E-1,1.0512428E0,0E0,3.6961976E-1,3.636502E-1,0E0,0E0,0E0,2.35363E-1,1.065422E0,1.3935776E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,32,32,33,33,34,34,36,36,37,37,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,51,51,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,70,70,72,72,73,73,77,77,78,78,86,86,93,93,94,94,98,98,99,99,100,100,106,106,107,107,109,109,110,110,114,114,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,-1,28,-1,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,-1,54,56,58,-1,60,62,-1,64,-1,66,68,70,72,74,76,78,80,82,-1,84,-1,86,88,90,-1,92,94,96,98,100,102,104,106,108,-1,110,-1,-1,112,-1,114,116,-1,-1,-1,118,120,-1,-1,-1,-1,-1,-1,-1,122,-1,-1,-1,-1,-1,-1,124,126,-1,-1,-1,128,130,132,-1,-1,-1,-1,-1,134,136,-1,138,140,-1,-1,-1,142,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,2.0162934E5,4.217427E7,1.16E2,9.31E2,1.1464103E3,1.3800834E3,1.1408248E0,6.83E2,2.331083E6,3.7326388E0,1.9791039E6,8E0,-4.7825832E-2,-2.2639198E-2,2.4360857E4,-1.7464165E-2,2E1,7.402E3,4.39E2,1.45064E5,1E0,1.3254E4,3.875E0,8E0,1E0,3.8537518E2,5.159652E-7,-2.6779842E-2,2E0,-5.8621336E-2,2.6945805E-2,9.751103E6,1.9685597E2,2.2953334E2,-6.846113E-3,2.1525E4,1.5958088E10,-6.836232E-3,3.202146E5,-1.2723793E-2,8.9064636E2,1.6254545E1,1.2909952E1,1.5031847E0,6.292039E0,5.854E3,1.3358269E2,2.230298E2,9.55188E5,-3.1016912E-2,3.6E2,-3.2506287E-2,1E1,4.4E1,2.638E3,-1.0480105E-2,5.862757E1,4.39776E6,2.2584E4,1.102E3,4.3764517E2,5.4007E7,9.751103E6,5.8891744E8,3.3412E5,3.2155007E-2,8.015909E2,-1.2464481E-2,4.4441346E-2,1E0,-7.781936E-3,6.787E4,1.9617874E3,2.5469026E-2,9.7812135E-3,2.7933981E-2,9.422379E4,2.9925186E-2,5.285293E-2,2.3667667E-2,-4.1863903E-2,-5.5656366E-2,-2.1042373E-2,-7.823764E-3,-1.2001296E-2,8.943293E-1,-5.064279E-4,-9.968048E-3,-1.7026514E-2,-2.7749281E-2,-4.4332057E-2,-1.954833E-2,1.5209424E1,1.1997242E0,3.8886194E-3,1.7440848E-2,-2.910079E-3,2.9211267E2,2.378914E6,3.655004E7,1.2142001E-2,3.5843324E-2,-1.1577309E-2,2.1258057E-3,1.3349657E-2,7.026624E7,1.6525911E9,-3.46676E-3,4.07E2,2.6886544E7,1.6773267E-2,3.2013852E-2,1.1181386E-2,2.9E1,3.1773497E5,1.7103828E7,9.1130175E-3,3.3284824E-2,7.4468837E-3,1.6191373E-2,-3.0336034E-2,-1.2538912E-2,-1.2329962E-2,-8.8330044E-4,-1.5721095E-4,-9.985902E-3,3.0995882E-3,1.0191964E-2,2.2585941E-3,1.2700595E-2,1.09897405E-2,2.0311462E-2,-1.8001337E-3,6.4351107E-3,3.3353248E-3,2.2581259E-2,-2.1705728E-3,7.027927E-3,2.624085E-4,1.4305838E-2,3.4121983E-2,1.7398806E-2,5.8625236E-2,2.8388632E-2,8.9003E-2,5.1907178E-2],"split_indices":[2,37,54,38,2,61,61,47,2,37,63,37,26,0,0,37,0,3,1,2,7,6,38,63,3,6,65,46,0,26,0,0,9,61,42,0,9,40,0,37,0,61,63,65,62,47,2,67,67,40,0,2,0,8,8,1,0,56,9,1,38,61,7,9,40,1,0,64,0,0,25,0,1,4,0,0,0,42,66,0,0,0,0,0,0,0,47,0,0,0,0,0,0,65,47,0,0,0,4,9,7,0,0,0,0,0,7,40,0,0,12,0,0,0,8,42,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.29E2,6.07E2,3.22E2,2.01E2,4.06E2,3.07E2,1.5E1,8.4E1,1.17E2,2.55E2,1.51E2,1.85E2,1.22E2,7E0,8E0,7.9E1,5E0,7.1E1,4.6E1,2E2,5.5E1,7.5E1,7.6E1,1.13E2,7.2E1,7.8E1,4.4E1,7.2E1,7E0,6.2E1,9E0,5E0,4.1E1,5.4E1,1.46E2,6E0,4.9E1,7E1,5E0,6.8E1,8E0,4.5E1,6.8E1,3.7E1,3.5E1,6.8E1,1E1,3.2E1,1.2E1,6.2E1,1E1,1.2E1,5E1,3.1E1,1E1,4.8E1,6E0,1E1,1.36E2,1.7E1,3.2E1,5.3E1,1.7E1,1.4E1,5.4E1,3.5E1,1E1,6.1E1,7E0,2.6E1,1.1E1,5E0,3E1,6.1E1,7E0,5E0,5E0,2E1,1.2E1,6E0,6E0,3.9E1,2.3E1,7E0,5E0,9E0,2.2E1,5E0,5E0,1.1E1,3.7E1,5E0,5E0,1.01E2,3.5E1,5E0,1.2E1,8E0,2.4E1,3.5E1,1.8E1,9E0,8E0,9E0,5E0,7E0,4.7E1,3E1,5E0,3.7E1,2.4E1,5E0,6E0,7E0,2.3E1,3.4E1,2.7E1,6E0,1.4E1,7E0,5E0,1.6E1,6E0,8.9E1,1.2E1,3E1,5E0,1.9E1,5E0,1.4E1,2.1E1,5E0,1.3E1,2.3E1,2.4E1,9E0,2.1E1,2.2E1,1.5E1,5E0,1.9E1,1.7E1,6E0,2.8E1,6E0,1.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-9.493533E-3,-2.3824015E-1,6.902419E-1,-6.598596E-1,-2.5635218E-2,8.3034366E-1,-1.747906E-1,-9.5753473E-1,-5.2718234E-1,-2.2149308E-1,1.3510825E-1,4.934519E-1,1.069665E0,-7.1025646E-1,1.9050996E-1,-3.8768653E-2,-4.9948063E-2,-6.718258E-1,-2.8096634E-1,-5.7191074E-1,-1.4943326E-1,2.6356182E-1,-1.0729145E-2,1.4552884E-1,6.740162E-1,1.733321E0,9.420623E-1,-4.4422537E-2,-1.8802362E-2,3.8654044E-1,-1.1914022E-2,-6.049994E-1,-6.6034585E-2,2.8768264E-2,-3.8310015E-1,-1.9470405E-2,-3.1568825E-2,-2.3574407E-1,-6.735161E-3,2.0996249E-1,5.4637796E-1,2.1357952E-2,-2.5949372E-2,-2.9805691E-3,2.361187E-2,7.788742E-1,1.3099317E-3,5.2099533E-2,9.024619E-2,5.950214E-1,1.0922092E0,8.409926E-3,2.1856913E-2,-7.392905E-1,-4.7196227E-1,-4.5848826E-1,4.0212395E-3,-1.9973688E-1,-3.1916868E-2,-1.788994E-2,2.8337274E-2,5.5545274E-2,3.072877E-1,1.3420684E-2,3.257912E-2,-2.0352222E-1,5.5421997E-2,-7.899872E-3,6.6073716E-2,3.956836E-2,4.4604716E-1,1.4312265E-2,6.3517714E-1,1.6144015E-2,1.1441082E0,-3.566296E-2,-1.8836547E-2,4.176056E-3,-5.3237534E-1,-3.67272E-2,-3.7811625E-1,-2.9100403E-1,-6.2445175E-2,1.3032742E-1,-7.451158E-2,2.0643318E-1,-1.482836E-1,1.5685521E-1,3.9544538E-1,5.14625E-4,-1.5469532E-2,-1.101181E-1,1.058304E-1,7.3324577E-3,-1.541538E-3,1.2469316E-2,2.547394E-2,1.956531E-2,3.2456186E-2,1.8854475E-2,5.5624317E-2,-2.8764566E-2,-1.5601468E-2,-1.9918272E-2,-4.668756E-3,-1.7176427E-2,-7.831489E-3,-5.879632E-3,2.6889313E-3,2.0402858E-3,1.5058792E-2,-7.968147E-3,2.486942E-3,1.4625599E-3,1.7589724E-2,-2.0360248E-2,2.9812723E-3,-4.148529E-4,9.302854E-3,2.2081623E-2,1.3272144E-2,3.888199E-4,-1.1098236E-2,1.2043734E-2,2.374253E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":1,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,-1,-1,31,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,53,-1,-1,55,-1,-1,57,59,61,63,65,-1,67,-1,69,-1,-1,-1,71,73,-1,-1,75,77,79,-1,81,-1,-1,83,85,87,-1,-1,89,91,-1,93,-1,95,-1,97,-1,99,-1,-1,-1,101,-1,103,105,107,109,111,113,115,117,119,-1,-1,121,123,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5462437E2,6.523982E1,2.898896E1,9.13192E0,1.5299119E1,1.6000397E1,6.7859144E0,1.743393E-1,5.9011536E0,5.444515E0,5.004627E0,5.367464E0,8.444519E0,6.6741276E-1,1.9405471E0,0E0,0E0,4.511463E0,6.2188597E0,3.256235E-1,2.2523112E0,2.0378237E0,2.2296445E0,1.6743238E0,3.8378067E0,1.4195633E-1,4.665039E0,0E0,0E0,1.7488575E-1,0E0,1.4376526E0,0E0,0E0,2.1292048E0,0E0,0E0,1.7185125E0,9.409773E-1,1.8014026E0,6.78082E-1,9.30519E-1,0E0,2.769278E-1,0E0,8.2487106E-1,0E0,0E0,0E0,1.8381119E-1,2.305809E0,0E0,0E0,1.5704536E-1,1.8472643E0,1.0658941E0,0E0,1.3225617E0,0E0,0E0,6.9153297E-1,1.5068656E0,9.193435E-1,0E0,0E0,4.4967186E-1,8.878381E-1,0E0,1.6047543E-1,0E0,5.3285122E-2,0E0,8.9549065E-2,0E0,2.1362991E0,0E0,0E0,0E0,6.1586E-1,0E0,5.672903E-1,5.568881E-1,3.47826E-1,5.3920084E-1,4.202065E-1,8.336623E-1,1.3248613E0,2.11617E-1,2.6878357E-1,0E0,0E0,3.843709E-1,6.67385E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,31,31,34,34,37,37,38,38,39,39,40,40,41,41,43,43,45,45,49,49,50,50,53,53,54,54,55,55,57,57,60,60,61,61,62,62,65,65,66,66,68,68,70,70,72,72,74,74,78,78,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,88,88,91,91,92,92],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,-1,-1,32,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,54,-1,-1,56,-1,-1,58,60,62,64,66,-1,68,-1,70,-1,-1,-1,72,74,-1,-1,76,78,80,-1,82,-1,-1,84,86,88,-1,-1,90,92,-1,94,-1,96,-1,98,-1,100,-1,-1,-1,102,-1,104,106,108,110,112,114,116,118,120,-1,-1,122,124,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,2.856934E7,8E1,8.1E2,1.1048219E3,1.3655363E2,8.75477E5,6.83E2,3.56E2,3.7326388E0,5.486433E5,1.059448E7,1.419902E3,3.0070068E7,-3.8768653E-2,-4.9948063E-2,5.035714E0,7.402E3,3.1E1,1.6802971E6,5.1167645E6,1.39848995E1,7.262122E2,1E0,4.626E3,1.7831801E6,-4.4422537E-2,-1.8802362E-2,1.5896863E3,-1.1914022E-2,2.71E2,-6.6034585E-2,2.8768264E-2,8.788122E4,-1.9470405E-2,-3.1568825E-2,6.8049283E0,2.1539131E2,1.295313E6,3.7788504E7,2.2108696E2,-2.5949372E-2,1.5204346E8,2.361187E-2,2E0,1.3099317E-3,5.2099533E-2,9.024619E-2,9.422379E4,1E0,8.409926E-3,2.1856913E-2,3.328629E6,1.2E1,3.741267E7,4.0212395E-3,1.339646E6,-3.1916868E-2,-1.788994E-2,5.8606E4,2.11429E5,1.14E3,1.3420684E-2,3.257912E-2,5E0,4.2586466E5,-7.899872E-3,5.1042255E2,3.956836E-2,9.293198E2,1.4312265E-2,3.3580637E8,1.6144015E-2,1E0,-3.566296E-2,-1.8836547E-2,4.176056E-3,1.3858744E4,-3.67272E-2,2.4361508E6,9.860918E5,3.056087E2,5.5426865E6,2.7976523E2,2.0535284E6,1.5031847E0,9.183673E-1,8E0,5.14625E-4,-1.5469532E-2,1.5E1,4.186E4,7.3324577E-3,-1.541538E-3,1.2469316E-2,2.547394E-2,1.956531E-2,3.2456186E-2,1.8854475E-2,5.5624317E-2,-2.8764566E-2,-1.5601468E-2,-1.9918272E-2,-4.668756E-3,-1.7176427E-2,-7.831489E-3,-5.879632E-3,2.6889313E-3,2.0402858E-3,1.5058792E-2,-7.968147E-3,2.486942E-3,1.4625599E-3,1.7589724E-2,-2.0360248E-2,2.9812723E-3,-4.148529E-4,9.302854E-3,2.2081623E-2,1.3272144E-2,3.888199E-4,-1.1098236E-2,1.2043734E-2,2.374253E-3],"split_indices":[2,37,54,38,2,61,65,40,2,2,63,37,12,61,52,0,0,62,1,10,37,37,63,61,6,2,37,0,0,4,0,2,0,0,42,0,0,63,42,9,7,61,0,7,0,26,0,0,0,42,24,0,0,9,10,7,0,9,0,0,9,9,2,0,0,8,37,0,4,0,4,0,7,0,15,0,0,0,42,0,54,37,4,37,4,60,62,65,8,0,0,3,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.64E2,7.27E2,2.37E2,2.43E2,4.84E2,2.04E2,3.3E1,7.3E1,1.7E2,2.18E2,2.66E2,8.6E1,1.18E2,1.3E1,2E1,4E1,3.3E1,1.06E2,6.4E1,3.6E1,1.82E2,1.41E2,1.25E2,3E1,5.6E1,1.7E1,1.01E2,6E0,7E0,1.4E1,6E0,9.9E1,7E0,6E0,5.8E1,1.7E1,1.9E1,1.13E2,6.9E1,1.2E2,2.1E1,1.19E2,6E0,2.2E1,8E0,4.8E1,8E0,6E0,1.1E1,3.2E1,6.9E1,5E0,9E0,4.7E1,5.2E1,5E1,8E0,1.06E2,7E0,5E0,6.4E1,4.7E1,7.3E1,9E0,1.2E1,1.5E1,1.04E2,6E0,1.6E1,3.8E1,1E1,5E0,2.7E1,5E0,6.4E1,4.2E1,5E0,5E0,4.7E1,8E0,4.2E1,6.3E1,4.3E1,3.2E1,3.2E1,2.7E1,2E1,2.8E1,4.5E1,6E0,9E0,2.4E1,8E1,8E0,8E0,5E0,5E0,8E0,1.9E1,5E0,5.9E1,3.1E1,1.6E1,3.5E1,7E0,3.7E1,2.6E1,2.8E1,1.5E1,2.3E1,9E0,1.8E1,1.4E1,1.4E1,1.3E1,8E0,1.2E1,6E0,2.2E1,2.4E1,2.1E1,1.3E1,1.1E1,2E1,6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[2.4026535E-3,-2.3275371E-1,6.3036793E-1,-6.6263205E-1,-3.1183273E-2,-8.625139E-1,7.174194E-1,-7.684905E-1,-3.3714586E-1,-2.0803536E-1,1.8312928E-1,-4.8268735E-2,-2.3697618E-2,4.3924937E-1,9.3639505E-1,-9.2860265E-3,-7.9548883E-1,-4.4412056E-1,-1.3261202E-1,-4.0768057E-1,-1.04171924E-1,1.15633346E-1,4.5513287E-1,2.7873072E-1,8.1507486E-1,7.8461155E-2,8.1016535E-1,-8.661026E-1,-5.602297E-1,-5.1119083E-1,-9.832577E-3,-3.4364317E-2,-1.2054134E-2,-4.9635234E-1,-1.5727986E-1,-7.6916166E-2,-3.98597E-2,-2.8060877E-1,1.5160865E-1,4.5475573E-3,5.467831E-1,9.55439E-2,4.3413466E-1,2.4081348E-2,8.8905835E-1,4.6700338E-1,1.03722E0,-8.2711285E-1,-1.154582E0,-6.954915E-1,-2.8842732E-1,-2.7976077E-2,-1.6654875E-2,-4.87616E-3,3.046832E-3,-4.4140524E-1,-4.511208E-2,-2.4426088E-1,2.6952208E-3,-1.0247384E-2,-2.618795E-1,-5.4989303E-3,-2.0830117E-2,1.7368084E-1,-1.6794002E-2,3.9757413E-1,3.2006063E-2,3.1443003E-1,-1.1717514E-1,5.054312E-1,-4.165069E-3,4.4995114E-2,2.7672796E-2,5.800043E-1,1.00769766E-1,1.1180316E0,6.556004E-1,-4.3923475E-2,-7.3127496E-1,-2.4235979E-2,-7.106387E-2,-2.3805333E-2,-3.782577E-2,-2.0766254E-3,-2.0689607E-2,-4.7699645E-1,-6.7737983E-3,-7.5720544E-3,-1.4253724E-2,-5.0448414E-2,1.2595773E-1,-1.7635553E-1,-2.522169E-2,3.492163E-1,1.0160247E-1,2.841526E-1,2.765228E-2,2.248912E-2,2.1900411E-1,-1.3698399E-2,-1.2559296E-2,3.5845906E-2,3.8244012E-1,4.857937E-1,3.5592303E-2,1.8843483E-2,-1.0160119E-2,1.1642395E0,2.429116E-2,3.5914328E-2,1.6191462E-2,-1.29275555E-2,-3.620351E-2,-2.3866985E-2,-1.3181785E-2,-6.3710185E-3,2.3536494E-3,2.2663404E-3,1.149328E-2,-1.206078E-2,-2.5815065E-3,8.88058E-3,2.6694337E-2,-4.2697745E-3,5.9371493E-3,6.6973073E-3,1.721275E-2,3.582596E-3,1.4648134E-2,-5.2950517E-3,2.9384429E-3,4.9545956E-3,2.0310098E-2,1.2254022E-2,2.4788952E-2,3.970658E-2,5.7240866E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":2,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,-1,27,29,31,33,35,37,39,41,43,-1,45,47,49,51,-1,53,-1,55,57,59,-1,61,63,-1,65,67,69,-1,71,73,75,77,79,81,83,-1,-1,-1,-1,85,-1,87,-1,89,91,-1,-1,93,-1,95,-1,97,99,101,-1,-1,-1,103,105,107,109,-1,111,-1,-1,-1,-1,-1,-1,113,-1,-1,-1,115,117,119,-1,121,123,125,-1,-1,127,129,-1,-1,131,133,-1,-1,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4309131E2,6.1068123E1,3.4927567E1,7.4933853E0,1.826878E1,4.0676022E-1,1.4816803E1,2.4340515E0,1.1882615E0,5.431793E0,3.9496632E0,0E0,0E0,6.529436E0,1.1870163E1,0E0,2.189392E0,4.6792173E-1,2.534678E-1,1.9202452E0,3.5345402E0,2.5514102E0,1.3538208E0,2.2359576E0,2.6142883E-1,0E0,8.943512E0,4.1725922E-1,1.2699804E0,1.7469263E-1,0E0,9.887279E-2,0E0,1.2967072E0,4.8250997E-1,2.0936344E0,0E0,3.5606003E-1,1.8991721E0,0E0,4.5126915E-1,1.8061708E0,1.6515179E0,0E0,1.1903763E-2,2.0107908E0,1.4874496E0,5.792694E-1,2.0601978E0,1.4967728E-1,5.5562747E-1,0E0,0E0,0E0,0E0,5.7173157E-1,0E0,4.1704655E-2,0E0,6.962782E-1,9.882755E-1,0E0,0E0,1.9382792E0,0E0,2.5818372E-1,0E0,2.0254111E-1,3.1397152E-1,9.529724E-1,0E0,0E0,0E0,2.9710484E-1,1.3581331E0,9.838104E-1,2.7460957E-1,0E0,1.246006E0,0E0,0E0,0E0,0E0,0E0,0E0,2.4329472E-1,0E0,0E0,0E0,8.681865E-1,2.6015976E-1,3.4876E-1,0E0,1.4956489E0,5.83308E-1,1.1638641E-1,0E0,0E0,1.6756743E-1,1.0778425E-1,0E0,0E0,3.799591E-1,1.5970469E-1,0E0,0E0,0E0,7.9208374E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,31,31,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,44,44,45,45,46,46,47,47,48,48,49,49,50,50,55,55,57,57,59,59,60,60,63,63,65,65,67,67,68,68,69,69,73,73,74,74,75,75,76,76,78,78,85,85,89,89,90,90,91,91,93,93,94,94,95,95,98,98,99,99,102,102,103,103,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,-1,28,30,32,34,36,38,40,42,44,-1,46,48,50,52,-1,54,-1,56,58,60,-1,62,64,-1,66,68,70,-1,72,74,76,78,80,82,84,-1,-1,-1,-1,86,-1,88,-1,90,92,-1,-1,94,-1,96,-1,98,100,102,-1,-1,-1,104,106,108,110,-1,112,-1,-1,-1,-1,-1,-1,114,-1,-1,-1,116,118,120,-1,122,124,126,-1,-1,128,130,-1,-1,132,134,-1,-1,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.8988005E5,1E0,1.559733E6,9.31E2,9.3326636E-2,1.1603809E3,7E0,8.0509944E5,4.97E2,2.8530578E6,-4.8268735E-2,-2.3697618E-2,3.022612E6,1.059448E7,-9.2860265E-3,2.2470966E-1,1.2860047E5,2.9036145E0,9.860918E5,4.6E1,3.574764E7,4.831007E6,7.207765E2,1.0950326E6,7.8461155E-2,1.5484655E6,1.073125E1,7.82E2,1.26942376E8,-9.832577E-3,6.300312E6,-1.2054134E-2,4.8667232E8,4.1973075E6,4.066351E0,-3.98597E-2,2.6E1,1.606722E8,4.5475573E-3,5.405423E6,1.124641E0,1.2118524E7,2.4081348E-2,1.8761517E5,3.7481603E8,2.3955788E7,3.89E2,1E0,1.6E1,2.87E2,-2.7976077E-2,-1.6654875E-2,-4.87616E-3,3.046832E-3,4.67E2,-4.511208E-2,3.440772E7,2.6952208E-3,3.1892856E2,5.261E3,-5.4989303E-3,-2.0830117E-2,1.8591549E0,-1.6794002E-2,1E0,3.2006063E-2,1.0013907E1,1.6E1,1.0717949E1,-4.165069E-3,4.4995114E-2,2.7672796E-2,8.3720225E-1,1.7101741E2,1E0,9.522029E6,-4.3923475E-2,2.6333334E1,-2.4235979E-2,-7.106387E-2,-2.3805333E-2,-3.782577E-2,-2.0766254E-3,-2.0689607E-2,1.8202913E2,-6.7737983E-3,-7.5720544E-3,-1.4253724E-2,1.9791039E6,8.6E1,1.345672E6,-2.522169E-2,7.6937294E-1,1E0,1.9204545E0,2.765228E-2,2.248912E-2,9.195571E2,6.125E0,-1.2559296E-2,3.5845906E-2,2.5393645E5,1.96E2,3.5592303E-2,1.8843483E-2,-1.0160119E-2,5.467E3,2.429116E-2,3.5914328E-2,1.6191462E-2,-1.29275555E-2,-3.620351E-2,-2.3866985E-2,-1.3181785E-2,-6.3710185E-3,2.3536494E-3,2.2663404E-3,1.149328E-2,-1.206078E-2,-2.5815065E-3,8.88058E-3,2.6694337E-2,-4.2697745E-3,5.9371493E-3,6.6973073E-3,1.721275E-2,3.582596E-3,1.4648134E-2,-5.2950517E-3,2.9384429E-3,4.9545956E-3,2.0310098E-2,1.2254022E-2,2.4788952E-2,3.970658E-2,5.7240866E-2],"split_indices":[2,37,16,9,2,47,61,3,54,2,37,0,0,37,12,0,47,37,62,37,3,7,54,61,56,0,37,63,2,40,0,5,0,40,37,63,0,3,12,0,37,47,56,0,42,41,60,1,14,3,0,0,0,0,0,2,0,7,0,61,38,0,0,62,0,25,0,65,3,65,0,0,0,44,67,6,9,0,56,0,0,0,0,0,0,61,0,0,0,37,0,9,0,47,109,62,0,0,4,65,0,0,37,0,0,0,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.67E2,7.04E2,2.63E2,2.24E2,4.8E2,1.4E1,2.49E2,1.68E2,5.6E1,2.63E2,2.17E2,8E0,6E0,1.11E2,1.38E2,8E0,1.6E2,3.6E1,2E1,8.9E1,1.74E2,1.75E2,4.2E1,7.9E1,3.2E1,1.8E1,1.2E2,1.21E2,3.9E1,2.7E1,9E0,1.2E1,8E0,6.5E1,2.4E1,1.69E2,5E0,1.4E1,1.61E2,9E0,3.3E1,3.7E1,4.2E1,8E0,2.4E1,4.9E1,7.1E1,1.1E2,1.1E1,2.5E1,1.4E1,1.5E1,1.2E1,7E0,5E0,6E1,5E0,1.7E1,7E0,1.25E2,4.4E1,8E0,6E0,1.55E2,6E0,1.8E1,1.5E1,1.8E1,1.9E1,3.7E1,5E0,1.7E1,7E0,3.7E1,1.2E1,5.7E1,1.4E1,4.5E1,6.5E1,5E0,6E0,1.2E1,1.3E1,6E0,8E0,5.3E1,7E0,9E0,8E0,9.7E1,2.8E1,3.5E1,9E0,4.4E1,1.11E2,1.3E1,5E0,5E0,1.3E1,1.2E1,7E0,1E1,2.7E1,2.7E1,1E1,6E0,6E0,5.2E1,5E0,9E0,5E0,7E0,5.8E1,4.3E1,1E1,5.2E1,4.5E1,1.8E1,1E1,2E1,1.5E1,2.7E1,1.7E1,1.3E1,9.8E1,6E0,7E0,6E0,7E0,5E0,7E0,5E0,2.2E1,6E0,2.1E1,1.2E1,4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-1.0086974E-2,-2.3388885E-1,5.147246E-1,-6.1934215E-1,-5.9155397E-2,-9.5988125E-1,5.9864813E-1,1.4317508E-2,-6.525322E-1,-2.1674806E-1,1.3963515E-1,-5.537803E-2,-3.2496043E-2,3.4682631E-1,8.572277E-1,-7.502957E-1,-3.504533E-1,-2.936966E-1,3.6210112E-2,-3.211269E-2,2.6585114E-1,6.637129E-1,1.9624151E-1,1.5250853E0,7.414454E-1,-6.791351E-1,-9.90638E-1,-4.0644488E-1,3.8539362E-4,-4.6761775E-1,-1.9085394E-1,-2.5052184E-2,2.2983545E-1,2.1718325E-2,-2.9912692E-1,2.2087301E-1,3.0263932E-2,5.0174296E-1,4.077549E-2,1.5563558E-1,3.6477298E-2,4.879021E-2,8.230782E-2,8.047312E-1,-4.9832305E-3,-7.2928065E-1,-2.0683408E-3,-6.136493E-2,-7.888325E-1,-2.6206113E-3,-4.694259E-1,-4.010094E-1,-4.1950695E-2,-4.0433127E-1,-1.3511351E-1,1.0159674E-1,-8.569857E-2,1.8281404E-2,5.0275E-3,6.7646496E-2,-1.4968658E-2,-2.133677E-2,-4.6869395E-3,6.783093E-2,2.7043772E-1,3.3072072E-1,2.9134842E-2,-1.1379038E-2,1.9153415E-1,8.781526E-1,3.847588E-1,-7.620247E-1,-1.46536045E-2,-4.139344E-2,-2.2442022E-2,-5.1057917E-1,-1.1659179E-2,-4.3752727E-1,1.7209748E-3,-1.2811189E-2,-4.6721494E-1,-8.204041E-2,-2.6771244E-1,8.242429E-4,8.500783E-3,-2.1741185E-2,-1.4854708E-2,-7.435911E-2,1.0656041E-1,1.3211682E-1,-5.076021E-3,2.9477757E-1,3.9192464E-4,6.2251193E-3,2.6467893E-2,2.4405795E-1,-6.9847494E-2,9.692648E-1,5.7961535E-1,2.159729E-2,6.763609E-3,-3.376083E-2,-4.8459925E-2,-1.6253712E-2,-2.5570618E-2,-2.1771705E-2,-3.1909575E-3,-2.4376135E-2,-1.3119736E-2,-8.528602E-3,-1.4172919E-3,-1.4372887E-2,-4.2361203E-3,-7.3632784E-3,2.5775433E-3,-1.0098905E-2,3.629746E-3,6.633073E-3,-5.3839227E-3,1.2416211E-2,3.8090587E-4,1.4986636E-2,8.4018736E-4,2.0308131E-2,7.1447054E-3,4.566924E-3,-1.5713746E-2,3.5481174E-2,5.0230607E-2,3.4675382E-2,1.0071695E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":3,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,-1,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,53,55,57,59,61,63,-1,65,-1,67,-1,-1,-1,69,-1,71,-1,-1,73,-1,75,77,-1,79,81,83,85,-1,-1,87,-1,-1,-1,89,91,93,-1,-1,95,97,99,101,-1,-1,-1,103,-1,105,-1,-1,107,109,111,-1,-1,113,-1,115,117,119,-1,121,-1,-1,-1,123,125,127,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1393314E2,4.576244E1,3.6463326E1,6.7304153E0,1.4723065E1,1.3423729E-1,1.7609581E1,0E0,5.791176E0,5.110403E0,4.517421E0,0E0,0E0,6.598034E0,9.30825E0,1.9610748E0,1.047926E0,3.5059433E0,7.411977E-1,1.2857512E0,1.9414673E0,1.1267166E0,2.2011974E0,3.7342072E-1,6.3716507E0,3.8495674E0,1.1670456E0,9.653406E-1,0E0,1.7889729E0,1.468482E0,3.7738508E-1,2.5645787E-1,1.2078819E0,4.311719E-1,8.084736E-1,0E0,3.9652205E-1,0E0,1.370934E0,0E0,0E0,0E0,3.0231323E0,0E0,1.289196E0,0E0,0E0,2.933998E-1,0E0,2.128458E-1,1.0721273E0,0E0,8.06427E-2,7.0289266E-1,1.0621369E-1,4.878685E-1,0E0,0E0,3.7816533E-1,0E0,0E0,0E0,3.3346707E-1,5.204096E-1,5.8311605E-1,0E0,0E0,1.1839914E0,1.9406357E0,2.7933145E-1,2.5065994E-1,0E0,0E0,0E0,8.275032E-3,0E0,6.807165E-1,0E0,0E0,2.3041964E-2,3.8742584E-1,1.8685985E-1,0E0,0E0,2.9462045E-1,0E0,3.4499598E-1,4.4596565E-1,3.503233E-1,0E0,5.796075E-1,0E0,0E0,0E0,1.1708899E0,7.118518E-1,7.715454E-1,1.2970486E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,43,43,45,45,48,48,50,50,51,51,53,53,54,54,55,55,56,56,59,59,63,63,64,64,65,65,68,68,69,69,70,70,71,71,75,75,77,77,80,80,81,81,82,82,85,85,87,87,88,88,89,89,91,91,95,95,96,96,97,97,98,98],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,-1,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,54,56,58,60,62,64,-1,66,-1,68,-1,-1,-1,70,-1,72,-1,-1,74,-1,76,78,-1,80,82,84,86,-1,-1,88,-1,-1,-1,90,92,94,-1,-1,96,98,100,102,-1,-1,-1,104,-1,106,-1,-1,108,110,112,-1,-1,114,-1,116,118,120,-1,122,-1,-1,-1,124,126,128,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1.8988005E5,1E0,6E0,9.31E2,2.5330253E-2,1.1464103E3,1.4317508E-2,1.559733E6,2.331083E6,9.860918E5,-5.537803E-2,-3.2496043E-2,8E0,1.059448E7,1E0,3.004207E2,1E0,2.9711267E2,1.8E1,6.0052995E6,1.6305691E8,3.3778846E0,4.626E3,7.06191E7,3.2921512E2,1E0,1.8461539E0,3.8539362E-4,3.9045289E0,4.03E2,1.1712844E2,6.2868685E6,2.0734E4,1.72797E5,4.727453E7,3.0263932E-2,5.357143E2,4.077549E-2,9.183432E4,3.6477298E-2,4.879021E-2,8.230782E-2,2.1E1,-4.9832305E-3,1.0590052E8,-2.0683408E-3,-6.136493E-2,2.5555556E0,-2.6206113E-3,1.363176E7,1.059448E7,-4.1950695E-2,3.2E1,2.7142856E0,4.9321495E6,5.504831E8,1.8281404E-2,5.0275E-3,5.753789E5,-1.4968658E-2,-2.133677E-2,-4.6869395E-3,9.822E3,7.45E0,1E0,2.9134842E-2,-1.1379038E-2,2.9366477E0,6.9E1,1E0,3.1E1,-1.46536045E-2,-4.139344E-2,-2.2442022E-2,7.3351803E0,-1.1659179E-2,1.2673605E7,1.7209748E-3,-1.2811189E-2,6.763314E7,5.51E2,6.789622E7,8.242429E-4,8.500783E-3,5.526796E7,-1.4854708E-2,2.7664474E2,1E0,4.181123E-3,-5.076021E-3,1E0,3.9192464E-4,6.2251193E-3,2.6467893E-2,9.527559E0,2.4860917E5,2.4994128E8,1.1797023E2,2.159729E-2,6.763609E-3,-3.376083E-2,-4.8459925E-2,-1.6253712E-2,-2.5570618E-2,-2.1771705E-2,-3.1909575E-3,-2.4376135E-2,-1.3119736E-2,-8.528602E-3,-1.4172919E-3,-1.4372887E-2,-4.2361203E-3,-7.3632784E-3,2.5775433E-3,-1.0098905E-2,3.629746E-3,6.633073E-3,-5.3839227E-3,1.2416211E-2,3.8090587E-4,1.4986636E-2,8.4018736E-4,2.0308131E-2,7.1447054E-3,4.566924E-3,-1.5713746E-2,3.5481174E-2,5.0230607E-2,3.4675382E-2,1.0071695E-2],"split_indices":[2,37,16,3,2,47,61,0,9,37,37,0,0,3,12,6,61,109,4,3,37,12,51,2,53,61,23,62,0,62,2,64,54,38,1,7,0,61,0,37,0,0,0,3,0,7,0,0,62,0,9,12,0,10,62,37,5,0,0,54,0,0,0,38,63,92,0,0,47,8,6,3,0,0,0,67,0,54,0,0,7,2,7,0,0,5,0,4,85,47,0,6,0,0,0,65,42,12,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.68E2,6.79E2,2.89E2,2.11E2,4.68E2,1.5E1,2.74E2,7E0,2.04E2,2.61E2,2.07E2,6E0,9E0,1.4E2,1.34E2,1.53E2,5.1E1,2E2,6.1E1,8.8E1,1.19E2,4.4E1,9.6E1,1.8E1,1.16E2,1.21E2,3.2E1,4.4E1,7E0,7.3E1,1.27E2,4.7E1,1.4E1,7.4E1,1.4E1,1.08E2,1.1E1,2.7E1,1.7E1,9.1E1,5E0,8E0,1E1,1.08E2,8E0,1.12E2,9E0,1E1,2.2E1,7E0,3.7E1,6.5E1,8E0,2.5E1,1.02E2,1.5E1,3.2E1,5E0,9E0,6.6E1,8E0,7E0,7E0,2.7E1,8.1E1,1.3E1,1.4E1,7E0,8.4E1,9.1E1,1.7E1,1.03E2,9E0,1.5E1,7E0,3E1,7E0,6E1,5E0,1E1,1.5E1,7.4E1,2.8E1,8E0,7E0,2.6E1,6E0,1.4E1,5.2E1,2E1,7E0,7.4E1,7E0,8E0,5E0,7E1,1.4E1,6.8E1,2.3E1,1.2E1,5E0,9.5E1,8E0,8E0,2.2E1,5.5E1,5E0,1E1,5E0,2.4E1,5E1,2.2E1,6E0,9E0,1.7E1,7E0,7E0,4.5E1,7E0,9E0,1.1E1,6.7E1,7E0,2.1E1,4.9E1,9E0,5E0,2.7E1,4.1E1,1.5E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[6.5762233E-3,-1.996318E-1,6.2783843E-1,-5.97717E-1,-2.7452633E-2,7.015954E-1,-7.289253E-2,8.6092375E-2,-6.3575697E-1,-1.8822856E-1,1.4289604E-1,2.913893E-1,8.347558E-1,1.6387339E-1,-3.5775866E-2,-2.3124276E-2,2.6668752E-2,-7.190836E-1,-3.6429968E-1,-4.9968475E-1,-1.16254434E-1,2.8876522E-1,-5.3997193E-3,4.9861822E-2,6.29457E-1,9.9010515E-1,5.498412E-1,2.675543E-1,-4.0868986E-3,-7.512842E-1,-8.663116E-3,-4.8281634E-1,-1.8328623E-1,-8.797666E-3,-5.320843E-1,-2.1125291E-1,1.6114207E-2,1.7442504E-1,4.120811E-1,3.441331E-2,-2.05458E-2,1.5499317E-2,-4.3070085E-2,1.8803675E-2,3.49619E-2,1.027359E0,1.0626695E-2,4.617607E-1,5.7361055E-2,4.2179623E-3,1.8837279E-2,-7.968991E-1,-5.257941E-1,-5.448951E-1,-7.146985E-3,-1.0232932E-1,-1.7103916E-2,-4.029876E-1,-2.7323643E-2,-3.8899976E-1,-1.4219724E-1,-1.7511642E-2,4.4712216E-2,2.2452487E-1,-1.2323949E-2,4.585793E-1,-2.7646446E-3,-2.0821754E-2,2.0498118E-1,-1.0597774E-2,5.7581004E-2,8.671219E-1,1.2940524E0,5.2575445E-1,1.9930255E-1,-7.659583E-1,-5.0254375E-2,-9.616468E-3,-6.069363E-1,-2.7395424E-2,-1.4379064E-2,1.906328E-4,-1.5108366E-1,-2.2049464E-2,-1.1239929E-2,-4.512343E-1,-1.565446E-3,4.3515425E-2,-2.0858042E-1,1.8865684E-2,1.1704718E-2,1.7397206E-1,2.9401168E-2,3.3774263E-1,3.0191053E-2,1.6067748E-1,-5.9416994E-2,2.595613E-1,5.5483315E-4,9.475437E-2,-4.203849E-4,9.0816766E-1,1.8978167E-2,7.056152E-2,4.3800116E-2,3.6433223E-1,3.4414846E-2,1.5714986E-2,1.6088717E-4,-3.9010163E-2,-2.9075472E-2,-1.8017739E-2,-3.1017797E-2,-2.8360889E-3,-1.0015929E-2,-9.597894E-3,-2.7709493E-2,-1.7407747E-3,7.456978E-3,4.8626605E-3,-1.12391375E-2,3.0433591E-3,-1.1818055E-2,1.1518686E-2,4.0501296E-3,1.1228589E-2,1.9967046E-2,-1.4624645E-3,1.2060487E-2,-9.503823E-3,9.6981874E-4,8.2233865E-3,1.8508926E-2,1.6614876E-3,6.398847E-3,4.933262E-2,3.691638E-2,8.06769E-3,2.192376E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":4,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,-1,-1,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,53,55,-1,57,59,61,63,65,67,-1,-1,69,-1,-1,71,-1,73,-1,-1,-1,75,77,79,-1,81,-1,83,-1,85,87,-1,89,91,-1,93,-1,95,97,-1,99,101,103,105,107,109,-1,-1,111,-1,-1,-1,113,-1,-1,115,-1,117,119,121,-1,123,-1,125,-1,127,129,131,-1,133,-1,135,-1,-1,-1,137,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1926694E2,4.7894424E1,1.2041786E1,5.543289E0,1.3419612E1,1.1227631E1,3.9282048E0,3.713803E0,4.261963E0,5.5909395E0,5.149923E0,4.2962036E0,6.4488907E0,4.937777E-1,0E0,0E0,0E0,2.4654312E0,9.6270895E-1,3.712921E-1,2.5938168E0,1.6201172E0,2.0875466E0,8.594861E-1,3.064642E-1,2.6283646E0,2.9357643E0,2.8098208E-1,0E0,1.051857E0,0E0,5.079746E-1,2.773702E-1,0E0,7.063484E-2,1.4176164E0,9.9597585E-1,1.4705452E0,1.2774229E0,1.0406141E0,0E0,0E0,4.7950125E-1,0E0,0E0,2.9803772E0,0E0,7.853861E-1,0E0,0E0,0E0,2.0641327E-1,5.7207966E-1,7.382059E-2,0E0,8.3649606E-2,0E0,7.063389E-2,0E0,7.1439075E-1,1.0939577E0,0E0,4.3512118E-1,1.0893548E0,0E0,9.872894E-1,0E0,5.979675E-1,2.8134775E-1,0E0,4.297216E-2,8.131485E-1,1.2626114E0,1.1938848E0,3.2638773E-1,6.0892487E-1,0E0,0E0,2.3027897E-2,0E0,0E0,0E0,5.090925E-2,0E0,0E0,8.75226E-1,0E0,2.3395285E-1,7.1073484E-1,9.6260583E-1,0E0,3.1958437E-1,0E0,1.9725966E-1,0E0,2.9432568E-1,8.2181615E-1,1.7021894E-1,0E0,2.3042783E-2,0E0,1.4874268E-1,0E0,0E0,0E0,4.5348358E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,31,31,32,32,34,34,35,35,36,36,37,37,38,38,39,39,42,42,45,45,47,47,51,51,52,52,53,53,55,55,57,57,59,59,60,60,62,62,63,63,65,65,67,67,68,68,70,70,71,71,72,72,73,73,74,74,75,75,78,78,82,82,85,85,87,87,88,88,89,89,91,91,93,93,95,95,96,96,97,97,99,99,101,101,105,105],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,-1,-1,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,54,56,-1,58,60,62,64,66,68,-1,-1,70,-1,-1,72,-1,74,-1,-1,-1,76,78,80,-1,82,-1,84,-1,86,88,-1,90,92,-1,94,-1,96,98,-1,100,102,104,106,108,110,-1,-1,112,-1,-1,-1,114,-1,-1,116,-1,118,120,122,-1,124,-1,126,-1,128,130,132,-1,134,-1,136,-1,-1,-1,138,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,2E0,6E0,9.31E2,7.079871E2,7.25E2,3.72E2,1.559733E6,3.79E2,3.6666667E0,1.6566709E6,8E0,1.7938144E0,-3.5775866E-2,-2.3124276E-2,2.6668752E-2,1.4178663E0,4.22201E6,1.9428572E1,1.5859042E6,4.2708197E2,1.2784314E1,4.0663E4,5.941442E6,7.153514E1,6.3344407E0,1.342376E0,-4.0868986E-3,8.147158E4,-8.663116E-3,6.077143E5,2.1E1,-8.797666E-3,3.1E1,1E0,5.862757E1,4.5158855E6,2.48334E5,1.5115256E6,-2.05458E-2,1.5499317E-2,1.5204346E8,1.8803675E-2,3.49619E-2,1.6978182E3,1.0626695E-2,7.61E2,5.7361055E-2,4.2179623E-3,1.8837279E-2,3.842105E1,9.593451E4,7.997723E6,-7.146985E-3,2.4114338E8,-1.7103916E-2,1E0,-2.7323643E-2,3.478493E2,2.0131578E0,-1.7511642E-2,6.9307615E6,2.828125E0,-1.2323949E-2,6.965855E9,-2.7646446E-3,1.26E2,1.1949272E7,-1.0597774E-2,9.6603775E0,1.9269184E7,1.4676277E7,1.9186024E3,1.8682626E8,9.2E1,-5.0254375E-2,-9.616468E-3,1E0,-2.7395424E-2,-1.4379064E-2,1.906328E-4,3.5014236E2,-2.2049464E-2,-1.1239929E-2,2.5055911E2,-1.565446E-3,6.9E1,2.6363637E0,2.8302418E10,1.1704718E-2,3.235955E0,2.9401168E-2,1.515E3,3.0191053E-2,2.4827585E0,1.503E3,5.7941175E0,5.5483315E-4,1E0,-4.203849E-4,9.1470585E0,1.8978167E-2,7.056152E-2,4.3800116E-2,2.391191E10,3.4414846E-2,1.5714986E-2,1.6088717E-4,-3.9010163E-2,-2.9075472E-2,-1.8017739E-2,-3.1017797E-2,-2.8360889E-3,-1.0015929E-2,-9.597894E-3,-2.7709493E-2,-1.7407747E-3,7.456978E-3,4.8626605E-3,-1.12391375E-2,3.0433591E-3,-1.1818055E-2,1.1518686E-2,4.0501296E-3,1.1228589E-2,1.9967046E-2,-1.4624645E-3,1.2060487E-2,-9.503823E-3,9.6981874E-4,8.2233865E-3,1.8508926E-2,1.6614876E-3,6.398847E-3,4.933262E-2,3.691638E-2,8.06769E-3,2.192376E-2],"split_indices":[2,37,6,3,2,61,0,2,9,2,63,37,26,62,0,0,0,47,5,61,37,61,62,9,54,65,62,63,0,37,0,56,3,0,10,109,56,56,1,37,0,0,7,0,0,61,0,8,0,0,0,55,37,9,0,40,0,109,0,61,62,0,37,62,0,40,0,10,54,0,65,56,56,61,41,38,0,0,8,0,0,0,4,0,0,4,0,0,67,40,0,65,0,2,0,62,2,63,0,24,0,65,0,0,0,40,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.29E2,6.98E2,2.31E2,2.1E2,4.88E2,2.09E2,2.2E1,1.1E1,1.99E2,2.51E2,2.37E2,5.2E1,1.57E2,1.7E1,5E0,5E0,6E0,1.51E2,4.8E1,4.6E1,2.05E2,1.19E2,1.18E2,3.1E1,2.1E1,1E2,5.7E1,1.2E1,5E0,1.42E2,9E0,2.8E1,2E1,5E0,4.1E1,1.19E2,8.6E1,6.3E1,5.6E1,1.09E2,9E0,7E0,2.4E1,9E0,1.2E1,9.5E1,5E0,5.2E1,5E0,6E0,6E0,1.16E2,2.6E1,2.3E1,5E0,1.5E1,5E0,1.5E1,2.6E1,3.2E1,8.7E1,5E0,8.1E1,5.7E1,6E0,5.1E1,5E0,8.3E1,2.6E1,8E0,1.6E1,6.2E1,3.3E1,4.1E1,1.1E1,1.08E2,8E0,6E0,2E1,1.8E1,5E0,5E0,1E1,9E0,6E0,2.7E1,5E0,2.3E1,6.4E1,7.3E1,8E0,5.2E1,5E0,3.3E1,1.8E1,1.4E1,6.9E1,2E1,6E0,1E1,6E0,5.6E1,6E0,1.8E1,1.5E1,2.5E1,1.6E1,6E0,5E0,6.7E1,4.1E1,6E0,1.4E1,5E0,5E0,1.1E1,1.6E1,1.4E1,9E0,6E0,5.8E1,6.3E1,1E1,2.7E1,2.5E1,1.8E1,1.5E1,5E0,9E0,2.4E1,4.5E1,1.4E1,6E0,5E0,5E0,2E1,3.6E1,1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"139","size_leaf_vector":"1"}},{"base_weights":[2.9777225E-2,-1.7852071E-1,6.088104E-1,-5.3681046E-1,-2.460878E-2,6.821481E-1,-3.5750523E-2,-8.158992E-1,-4.2960557E-1,-1.6882138E-1,1.4353724E-1,4.0556836E-1,8.823198E-1,-2.4515351E-2,1.9947E-1,-5.0697803E-2,-7.1167123E-1,-6.366233E-1,-3.3724704E-1,-2.9336506E-1,-3.8281187E-2,5.926423E-2,3.0386585E-1,1.055777E-1,5.12432E-1,6.920964E-2,8.032246E-1,3.6861718E-1,-1.0600024E-2,-3.6472265E-2,-1.2500544E-2,-3.0780861E-2,-1.5922349E-2,1.7958662E-2,-3.978488E-1,-2.5973284E-1,-4.5674622E-2,-6.1640665E-2,1.7987108E-2,8.398499E-2,-1.83838E-2,3.3501866E-1,-2.9891862E-3,-2.1896422E-2,1.9073207E-2,5.7987463E-1,8.571601E-2,5.445867E-1,9.3722725E-1,6.6025434E-3,2.4154102E-2,-5.6387246E-1,-2.672914E-1,-3.3096042E-1,-6.0599267E-2,6.588648E-3,-1.3680272E-1,3.2582194E-1,5.357441E-2,2.2290516E-1,5.546293E-1,-6.127773E-3,4.7584448E-2,6.787259E-1,3.005312E-1,-1.3286087E-3,8.62009E-3,5.8773863E-1,8.376335E-3,1.0184476E0,5.639215E-1,-6.398611E-1,-3.0791435E-1,-3.3288583E-1,8.931136E-3,-3.7071815E-1,-8.890368E-2,-1.166874E-2,-1.4039364E-2,3.0962553E-2,-1.1431567E-2,1.1385798E-1,-1.8921202E-1,3.8356686E-1,7.686184E-3,7.9552524E-2,-2.3469363E-1,3.9591643E-1,1.2984349E-1,6.0133016E-1,1.648611E-2,-2.1082032E-3,6.2085697E-3,3.4667715E-2,4.5280227E-1,1.9635792E-1,1.8847581E-2,1.4375532E-2,2.971881E-2,1.8849496E-2,1.0650226E0,4.218673E-1,3.3888604E-2,-3.336E-2,-1.7910268E-2,-6.372682E-3,-1.9029772E-2,-2.6983419E-3,-1.8799916E-2,-1.11660035E-2,-1.9911487E-2,3.4037705E-3,-9.522507E-3,2.4604409E-3,-6.2501235E-3,-5.4109367E-3,3.7371684E-3,1.2656827E-3,8.419711E-3,-1.8842423E-2,-5.457876E-3,2.1654326E-2,1.0974006E-2,9.6584973E-4,9.54677E-3,-1.8068397E-2,-1.8959494E-3,4.1779797E-3,2.3935582E-2,-3.2049953E-3,8.9878235E-3,1.6934272E-2,3.0816838E-2,9.853151E-3,2.675099E-2,4.223431E-3,1.2479867E-2,3.0418385E-2,5.1055036E-2,2.541694E-2,1.0469444E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":5,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,29,31,33,35,37,39,41,43,45,-1,47,49,-1,-1,-1,-1,-1,-1,51,53,-1,55,-1,57,-1,59,-1,61,-1,63,65,67,69,-1,-1,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,-1,-1,97,-1,99,101,103,105,107,-1,109,111,-1,113,115,-1,117,119,121,-1,123,125,127,129,131,-1,-1,-1,-1,133,135,-1,-1,-1,-1,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.16257484E2,3.908464E1,1.20784E1,6.0223694E0,1.2075872E1,1.2320381E1,3.2338989E0,8.953552E-1,2.7679977E0,4.3459015E0,3.0896082E0,3.0875893E0,5.1632004E0,0E0,1.4594646E0,0E0,1.2684994E0,1.4153099E-1,4.982259E0,2.9282513E0,1.3372588E0,1.7447369E0,9.279671E-1,1.0604302E0,2.022375E0,0E0,3.613922E0,4.0737927E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.060194E0,1.8522062E0,0E0,6.4706135E-1,0E0,1.0489327E0,0E0,1.673727E0,0E0,1.5937753E-1,0E0,1.5080547E0,1.3118559E-1,5.694227E-1,1.7603226E0,0E0,0E0,6.5877247E-1,1.8434381E0,9.005337E-1,3.1642315E-1,4.20728E-1,8.0976045E-1,8.444452E-2,9.921964E-1,7.5362897E-1,9.745598E-3,0E0,1.1034578E-1,3.5741043E-1,1.1838627E-1,0E0,0E0,3.587618E-1,0E0,1.3351593E0,7.019091E-2,3.852358E-1,1.5288997E-1,1.0091729E0,0E0,5.273504E-1,2.9818264E-1,0E0,2.4991795E-1,4.6305314E-1,0E0,5.9434444E-2,7.2848725E-1,2.4088502E-2,0E0,8.852192E-1,3.1396747E-1,5.77235E-1,4.4529194E-1,2.4931908E-2,0E0,0E0,0E0,0E0,2.6434207E-1,5.964747E-2,0E0,0E0,0E0,0E0,1.0285187E-1,1.4819789E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,34,34,35,35,37,37,39,39,41,41,43,43,45,45,46,46,47,47,48,48,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,62,62,63,63,64,64,67,67,69,69,70,70,71,71,72,72,73,73,75,75,76,76,78,78,79,79,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,94,94,95,95,100,100,101,101],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,30,32,34,36,38,40,42,44,46,-1,48,50,-1,-1,-1,-1,-1,-1,52,54,-1,56,-1,58,-1,60,-1,62,-1,64,66,68,70,-1,-1,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,-1,-1,98,-1,100,102,104,106,108,-1,110,112,-1,114,116,-1,118,120,122,-1,124,126,128,130,132,-1,-1,-1,-1,134,136,-1,-1,-1,-1,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,2.0162934E5,2.856934E7,3.3817584E7,9.31E2,1.0879832E3,1.465E4,2E0,2.71E2,9.4996644E5,1.9791039E6,4.7567694E5,1.059448E7,-2.4515351E-2,7.5491136E-1,-5.0697803E-2,1.1913043E0,3.328629E6,8.093982E-6,1.1290322E1,7.4711205E6,1.2784314E1,2.737878E5,4.032683E6,2E0,6.920964E-2,1.5484655E6,1.3245402E3,-1.0600024E-2,-3.6472265E-2,-1.2500544E-2,-3.0780861E-2,-1.5922349E-2,1.7958662E-2,1.4872362E-1,4.629112E6,-4.5674622E-2,2.49E2,1.7987108E-2,7.9555137E3,-1.83838E-2,5.281628E2,-2.9891862E-3,2.6463525E8,1.9073207E-2,2E0,4.179E3,4.909091E0,2.395631E7,6.6025434E-3,2.4154102E-2,1.559733E6,8.469667E4,2.9768292E2,1.766298E2,3.6507E4,9.380531E-1,1.4552647E7,2.453E4,1.0164831E-2,3.8396227E0,-6.127773E-3,3.443E3,2.3282397E5,1.6566709E6,-1.3286087E-3,8.62009E-3,1.4504672E3,8.376335E-3,1E0,1.5840335E9,1.08367164E5,2.4818E4,2.14E0,8.931136E-3,1.9619734E0,1E0,-1.166874E-2,1.4E1,5.02E2,-1.1431567E-2,8.33E2,3.6977E4,1.3E1,7.686184E-3,1.9539816E7,5.4051723E0,1.7697E4,1.3214286E1,4.087912E0,1.648611E-2,-2.1082032E-3,6.2085697E-3,3.4667715E-2,3.8343322E6,4.5179688E1,1.8847581E-2,1.4375532E-2,2.971881E-2,1.8849496E-2,8.2631064E7,4.990618E6,3.3888604E-2,-3.336E-2,-1.7910268E-2,-6.372682E-3,-1.9029772E-2,-2.6983419E-3,-1.8799916E-2,-1.11660035E-2,-1.9911487E-2,3.4037705E-3,-9.522507E-3,2.4604409E-3,-6.2501235E-3,-5.4109367E-3,3.7371684E-3,1.2656827E-3,8.419711E-3,-1.8842423E-2,-5.457876E-3,2.1654326E-2,1.0974006E-2,9.6584973E-4,9.54677E-3,-1.8068397E-2,-1.8959494E-3,4.1779797E-3,2.3935582E-2,-3.2049953E-3,8.9878235E-3,1.6934272E-2,3.0816838E-2,9.853151E-3,2.675099E-2,4.223431E-3,1.2479867E-2,3.0418385E-2,5.1055036E-2,2.541694E-2,1.0469444E-2],"split_indices":[2,37,54,7,2,61,38,26,2,37,37,37,12,0,47,0,65,9,47,63,37,62,42,54,6,0,37,61,0,0,0,0,0,0,51,9,0,10,0,56,0,4,0,7,0,26,2,62,56,0,0,9,42,61,4,1,62,54,38,47,63,0,2,42,37,0,0,4,0,15,12,37,1,62,0,62,8,0,3,2,0,0,12,8,0,12,63,9,67,65,0,0,0,0,54,65,0,0,0,0,5,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.62E2,7.08E2,2.54E2,2.12E2,4.96E2,2.28E2,2.6E1,5.7E1,1.55E2,2.67E2,2.29E2,9.7E1,1.31E2,8E0,1.8E1,1.3E1,4.4E1,4.6E1,1.09E2,1.36E2,1.31E2,1.51E2,7.8E1,2.6E1,7.1E1,1.3E1,1.18E2,1.3E1,5E0,3.7E1,7E0,4.1E1,5E0,8E0,1.01E2,1.31E2,5E0,1.25E2,6E0,1.44E2,7E0,7.2E1,6E0,1.9E1,7E0,6.1E1,1E1,4.2E1,7.6E1,6E0,7E0,4.3E1,5.8E1,9.6E1,3.5E1,6.6E1,5.9E1,1.5E1,1.29E2,4.9E1,2.3E1,7E0,1.2E1,4.4E1,1.7E1,5E0,5E0,3.7E1,5E0,6.1E1,1.5E1,3.2E1,1.1E1,5.1E1,7E0,8.2E1,1.4E1,6E0,2.9E1,6.1E1,5E0,1E1,4.9E1,1E1,5E0,1.19E2,1E1,1.6E1,3.3E1,1.7E1,6E0,6E0,6E0,3.2E1,1.2E1,1E1,7E0,7E0,3E1,5E0,5.6E1,1E1,5E0,2.3E1,9E0,5E0,6E0,1.1E1,4E1,2.7E1,5.5E1,6E0,8E0,1.9E1,1E1,1.5E1,4.6E1,5E0,5E0,1.1E1,3.8E1,5E0,5E0,8.2E1,3.7E1,5E0,5E0,5E0,1.1E1,8E0,2.5E1,5E0,1.2E1,5E0,7E0,5E0,5E0,6E0,5E1,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[1.3802322E-2,-1.8315278E-1,5.650586E-1,-5.5247515E-1,-1.4603534E-2,4.011608E-1,8.89925E-1,-7.644686E-1,-4.277455E-1,-1.682721E-1,1.3978107E-1,4.4029105E-1,-3.260257E-2,1.0563883E0,3.9548305E-1,-7.237931E-1,-4.981606E-2,-4.7390342E-1,-2.073418E-2,-4.3548048E-1,-1.1151793E-1,1.5851219E-1,-2.7911179E-2,2.8886726E-1,6.54829E-1,1.1237994E0,1.2174635E-2,5.464442E-1,-4.5542843E-3,-3.7035134E-2,-5.702505E-1,-4.183508E-1,-7.559625E-1,7.139361E-3,-8.943211E-3,-2.5611767E-1,-5.2910835E-1,-4.1339263E-1,-7.866884E-2,1.2089553E-1,5.086653E-1,4.0210254E-2,2.3306248E-1,7.357643E-1,6.31535E-3,1.0122942E0,7.6668344E-2,1.2589247E-2,3.1173171E-2,-1.4812999E-2,-2.843285E-2,-4.644144E-1,-8.5764974E-2,-2.1426938E-2,-3.9727647E-2,-1.3901974E-2,-6.2029352E-3,-1.457229E-2,-2.7459461E-2,-3.750109E-2,-1.6899194E-1,-1.294126E-1,6.551938E-2,3.330407E-2,2.1976642E-1,2.7367992E-2,1.358988E-2,1.0665401E-1,3.8914865E-1,5.944811E-1,4.0158458E-2,5.1166836E-2,3.5377942E-2,-4.087868E-1,-3.5774574E-2,7.85114E-3,-1.1424338E-2,1.3365232E-3,-1.374422E-2,-8.771721E-2,-2.6825306E-1,1.7197095E-1,-4.366216E-2,5.4645147E-2,-1.6371062E-2,-1.8417085E-2,2.772842E-1,2.5918114E-1,-3.287582E-2,5.337544E-1,3.0511072E-1,3.068575E-2,1.6935611E-2,-2.4168821E-2,-1.31618865E-2,-6.334968E-3,-8.889691E-4,-2.7889223E-3,-1.583196E-2,4.3331254E-3,1.3574929E-2,-9.313118E-3,2.967308E-3,8.170854E-3,-8.7977365E-5,2.6281525E-3,-9.418854E-3,1.56118E-2,1.0885913E-4,1.816106E-2,3.1241425E-3,4.074814E-3,-5.6302543E-3,1.4520365E-2,2.9354293E-2,1.9524831E-3,1.7550971E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":6,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,-1,31,33,35,37,39,-1,41,43,45,-1,47,-1,-1,49,51,53,-1,-1,55,57,59,61,63,65,-1,67,69,-1,71,-1,-1,-1,-1,-1,73,75,-1,-1,-1,-1,-1,-1,-1,77,79,81,83,85,-1,-1,87,89,91,-1,-1,-1,93,-1,-1,-1,-1,-1,95,97,99,101,103,-1,105,107,109,111,113,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.00641846E2,4.2508003E1,1.2634743E1,5.331566E0,1.1173829E1,7.3929234E0,6.2490387E0,2.038002E-1,2.5639648E0,3.5304203E0,3.3585153E0,4.976139E0,0E0,2.916603E0,1.6930754E0,3.6720276E-1,0E0,1.6110458E0,4.8015538E-1,5.530133E-1,1.9110754E0,2.9679708E0,0E0,2.860911E0,2.6287384E0,1.5688553E0,0E0,4.0421534E-1,0E0,0E0,9.2624664E-2,1.5687809E0,1.6491508E-1,0E0,0E0,4.5153856E-2,1.6469336E-1,1.6968594E0,1.3108432E0,1.8026721E0,2.1256256E-1,0E0,1.6947894E0,5.118923E-1,0E0,2.4388885E-1,0E0,0E0,0E0,0E0,0E0,1.2539139E0,6.1441565E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.356695E-1,7.4061847E-1,5.5376834E-1,9.4221634E-1,1.3476858E0,0E0,0E0,1.072625E0,3.110881E-1,2.1890163E-1,0E0,0E0,0E0,9.7433186E-1,0E0,0E0,0E0,0E0,0E0,3.4208393E-1,4.3506503E-1,2.0012212E-1,4.18464E-1,7.3511463E-1,0E0,2.8858307E-1,1.2846355E0,5.824665E-1,2.9895386E-1,8.588529E-2,5.035665E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,27,27,30,30,31,31,32,32,35,35,36,36,37,37,38,38,39,39,40,40,42,42,43,43,45,45,51,51,52,52,60,60,61,61,62,62,63,63,64,64,67,67,68,68,69,69,73,73,79,79,80,80,81,81,82,82,83,83,85,85,86,86,87,87,88,88,89,89,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,-1,32,34,36,38,40,-1,42,44,46,-1,48,-1,-1,50,52,54,-1,-1,56,58,60,62,64,66,-1,68,70,-1,72,-1,-1,-1,-1,-1,74,76,-1,-1,-1,-1,-1,-1,-1,78,80,82,84,86,-1,-1,88,90,92,-1,-1,-1,94,-1,-1,-1,-1,-1,96,98,100,102,104,-1,106,108,110,112,114,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.972052E5,1.5632523E3,1.6E1,9.31E2,3.9661028E7,1.3358269E2,3.09627E5,2.1340163E8,3.56E2,1.2784314E1,2.2318378E6,-3.260257E-2,1E0,2.1680816E7,3.23E2,-4.981606E-2,1E0,8.910034E0,1.09333336E2,2.508E2,5.1167645E6,-2.7911179E-2,5.1E1,1E0,7.626E3,1.2174635E-2,2.338255E3,-4.5542843E-3,-3.7035134E-2,2.8107122E-5,1.255E3,7.807738E6,7.139361E-3,-8.943211E-3,2E0,3.5675676E0,5.862757E1,2.331083E6,4.0446457E2,1.162E3,4.0210254E-2,8.507324E2,1.9630163E8,6.31535E-3,8.166121E1,7.6668344E-2,1.2589247E-2,3.1173171E-2,-1.4812999E-2,-2.843285E-2,1.711E4,1.312E1,-2.1426938E-2,-3.9727647E-2,-1.3901974E-2,-6.2029352E-3,-1.457229E-2,-2.7459461E-2,-3.750109E-2,1.0476191E0,1.23E2,3.8575E4,3.4474048E6,1E0,2.7367992E-2,1.358988E-2,1.3E1,3.8908466E1,5.0741018E8,4.0158458E-2,5.1166836E-2,3.5377942E-2,1.2469576E2,-3.5774574E-2,7.85114E-3,-1.1424338E-2,1.3365232E-3,-1.374422E-2,2.87834E5,1.6E0,5.405423E6,2.2907576E2,1.8861789E0,-1.6371062E-2,1.4949E4,3.5919855E0,1.124641E0,3.39498E5,2.35184E5,6.853003E5,3.068575E-2,1.6935611E-2,-2.4168821E-2,-1.31618865E-2,-6.334968E-3,-8.889691E-4,-2.7889223E-3,-1.583196E-2,4.3331254E-3,1.3574929E-2,-9.313118E-3,2.967308E-3,8.170854E-3,-8.7977365E-5,2.6281525E-3,-9.418854E-3,1.56118E-2,1.0885913E-4,1.816106E-2,3.1241425E-3,4.074814E-3,-5.6302543E-3,1.4520365E-2,2.9354293E-2,1.9524831E-3,1.7550971E-2],"split_indices":[2,37,61,0,2,54,67,9,7,2,62,37,0,6,52,1,0,6,65,4,42,37,0,0,6,61,0,61,0,0,47,2,40,0,0,8,63,56,37,61,10,0,61,12,0,65,0,0,0,0,0,38,67,0,0,0,0,0,0,0,62,0,9,60,109,0,0,3,67,7,0,0,0,61,0,0,0,0,0,12,62,37,4,62,0,38,62,47,9,1,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.25E2,6.82E2,2.43E2,2.13E2,4.69E2,1.63E2,8E1,7.7E1,1.36E2,2.35E2,2.34E2,1.58E2,5E0,5.9E1,2.1E1,7.1E1,6E0,1.22E2,1.4E1,4E1,1.95E2,2.29E2,5E0,9.4E1,6.4E1,5.4E1,5E0,1.6E1,5E0,4.5E1,2.6E1,1.04E2,1.8E1,7E0,7E0,1.5E1,2.5E1,1.8E1,1.77E2,2.08E2,2.1E1,7E0,8.7E1,5.5E1,9E0,4.7E1,7E0,6E0,1E1,5E0,2.1E1,9.1E1,1.3E1,6E0,1.2E1,1E1,5E0,7E0,1.8E1,6E0,1.2E1,1.31E2,4.6E1,1.11E2,9.7E1,1.4E1,7E0,4.9E1,3.8E1,2.9E1,2.6E1,3.2E1,1.5E1,7.9E1,1.2E1,5E0,8E0,5E0,7E0,1.02E2,2.9E1,2.3E1,2.3E1,1.06E2,5E0,1.9E1,7.8E1,2.3E1,2.6E1,1.2E1,2.6E1,2.1E1,8E0,4E1,3.9E1,5.9E1,4.3E1,8E0,2.1E1,1.5E1,8E0,9E0,1.4E1,3.3E1,7.3E1,1.4E1,5E0,6.4E1,1.4E1,1.3E1,1E1,1.1E1,1.5E1,5E0,7E0,6E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[8.1706485E-3,-3.1231356E-1,2.86195E-1,-7.1519536E-1,-2.0736824E-1,6.273287E-2,6.275923E-1,-7.471432E-1,-1.3101181E-2,-3.5867676E-1,1.1810365E-1,1.086568E-1,-3.090752E-1,5.118878E-1,9.604117E-1,-6.522862E-1,-4.36598E-2,-4.6364924E-1,-2.3703824E-1,4.6318494E-2,3.3956207E-2,2.9878144E-3,2.0706773E-1,-5.178009E-1,-8.317864E-2,9.311334E-1,4.3178338E-1,5.036389E-2,3.3600338E-2,-3.2376453E-2,-1.447277E-2,-1.6417773E-1,-5.1932853E-1,-5.5573706E-2,-3.4716138E-1,-3.7856095E-2,3.264277E-1,-1.0260653E-1,9.974113E-2,3.250299E-1,9.414163E-2,-3.9209387E-1,-3.265896E-2,-1.1370972E-2,1.8187901E-2,2.9673737E-2,5.4261524E-2,4.7170663E-1,3.0046862E-2,-3.052746E-1,3.4125953E-3,-6.4376575E-1,-4.1260675E-1,-8.0354735E-3,-1.4499371E-2,-4.256132E-1,-1.4951573E-1,1.2777379E-2,-3.5116062E-1,3.22471E-2,1.678153E-1,-2.6924422E-1,-9.744118E-4,6.5773875E-2,1.40943155E-2,3.5166466E-1,1.4511137E-3,-2.0437606E-2,1.8738286E-1,-2.143096E-2,-1.0843384E-2,-2.4760705E-3,3.5690453E-3,4.0286958E-1,7.05084E-1,8.129368E-3,-7.818485E-3,-1.9816414E-2,-8.8228155E-3,-5.599469E-1,-8.8241744E-1,-2.857911E-1,-4.905689E-1,-1.02689415E-1,1.0809909E-1,-3.2107162E-1,-6.342309E-1,-2.0095004E-1,9.806096E-4,-3.631002E-2,1.4958324E-1,-8.922027E-3,-2.0284507E-2,1.775543E-2,3.536859E-2,-1.5554763E-1,-2.2800947E-2,9.389317E-3,-6.256094E-2,9.905857E-2,-4.734106E-2,3.1323963E-1,2.5795715E-2,-1.0187093E-2,3.718221E-2,2.6560926E-1,5.671995E-2,4.2346117E-1,1.2458569E-3,3.532955E-2,1.8113649E-2,-2.7654916E-2,-1.7329386E-2,-5.8586948E-2,-1.6476398E-2,-7.793013E-3,-1.4511143E-2,-2.4745919E-2,-8.388087E-3,4.3713252E-4,-1.0006327E-2,1.2125534E-2,1.3484611E-3,-6.8290457E-3,-1.695892E-2,-3.6820475E-2,-1.9587021E-2,-1.465355E-2,-2.9583767E-3,1.3394385E-3,-6.4194133E-3,8.926649E-3,8.186198E-4,5.623749E-3,-2.6150998E-3,-1.3112627E-2,-2.2736997E-3,-5.1064626E-3,6.011772E-3,-6.6782936E-4,5.963149E-3,-1.0094258E-2,2.7616597E-3,1.0188209E-2,2.0085268E-2,-3.1570237E-4,9.261277E-3,1.8964235E-2,5.2779885E-3,7.99439E-3,-1.0200555E-3,2.6350676E-3,2.083221E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":7,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,-1,31,33,-1,35,37,39,41,43,45,47,-1,-1,-1,-1,49,51,53,55,57,59,61,63,65,67,69,-1,-1,71,-1,-1,73,75,77,-1,79,81,83,-1,85,87,89,91,-1,93,95,97,99,-1,101,-1,103,105,-1,-1,-1,-1,107,109,-1,-1,-1,-1,111,113,115,117,119,121,123,125,127,-1,129,131,-1,-1,-1,133,135,-1,-1,137,139,141,143,-1,-1,145,147,149,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.669694E1,1.8962505E1,3.9707394E1,1.0750656E0,1.777116E1,5.429248E0,7.4411697E0,9.672928E-1,0E0,3.0311966E0,8.483301E0,2.9318943E0,1.6071954E0,4.803383E0,7.500229E-1,7.2686386E-1,0E0,2.0990791E0,2.298787E0,0E0,2.2386808E0,1.4099087E0,1.9142685E0,1.450324E-1,2.9983872E-1,9.199333E-1,2.1029263E0,0E0,0E0,0E0,0E0,7.60385E-1,1.1951962E0,5.4709065E-1,1.0527716E0,1.3687648E0,1.1109979E0,1.1174967E0,4.789794E-1,5.445132E-1,8.157764E-1,3.5761356E-2,0E0,0E0,5.4533757E-2,0E0,0E0,1.6437435E0,4.0419692E-1,1.1511612E-1,0E0,4.860649E-1,4.727831E-1,4.3981183E-1,0E0,9.0863705E-1,1.9734648E-1,5.0908965E-1,7.991564E-2,0E0,4.41764E-1,5.550388E-1,5.386078E-1,2.4188429E-1,0E0,3.360815E-1,0E0,4.0971667E-1,4.1422868E-1,0E0,0E0,0E0,0E0,7.241974E-1,1.6372204E-1,0E0,0E0,0E0,0E0,3.9715767E-2,1.7572603E0,3.0050278E-2,3.9365292E-1,2.8065687E-1,2.104011E-1,2.2432446E-1,2.0566893E-1,2.4775392E-1,0E0,3.7643862E-1,1.0955703E-1,0E0,0E0,0E0,9.343989E-2,2.2982001E-1,0E0,0E0,3.1537688E-1,1.6561735E-1,2.8577965E-1,5.436754E-1,0E0,0E0,2.0139301E-1,5.185424E-1,1.58424E-1,8.164959E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,44,44,47,47,48,48,49,49,51,51,52,52,53,53,55,55,56,56,57,57,58,58,60,60,61,61,62,62,63,63,65,65,67,67,68,68,73,73,74,74,79,79,80,80,81,81,82,82,83,83,84,84,85,85,86,86,87,87,89,89,90,90,94,94,95,95,98,98,99,99,100,100,101,101,104,104,105,105,106,106,107,107],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,-1,32,34,-1,36,38,40,42,44,46,48,-1,-1,-1,-1,50,52,54,56,58,60,62,64,66,68,70,-1,-1,72,-1,-1,74,76,78,-1,80,82,84,-1,86,88,90,92,-1,94,96,98,100,-1,102,-1,104,106,-1,-1,-1,-1,108,110,-1,-1,-1,-1,112,114,116,118,120,122,124,126,128,-1,130,132,-1,-1,-1,134,136,-1,-1,138,140,142,144,-1,-1,146,148,150,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.486433E5,7.7E1,2.62E3,1.5632523E3,1.266E3,1.2425086E-7,4.391553E6,4.217427E7,-1.3101181E-2,1.5199841E2,8E0,1.007E3,3.4289816E2,1.481E3,1.684E0,7.668863E6,-4.36598E-2,2.2007043E0,2.4131148E0,4.6318494E-2,5.202E3,2.0766992E6,3.0416667E0,3.94E2,7.582512E7,5.526E3,2.7555334E7,5.036389E-2,3.3600338E-2,-3.2376453E-2,-1.447277E-2,8.93004E5,8.147158E4,8.303E3,3.355102E2,2.2E1,2.026087E1,4.0663E4,7.168071E6,1.9E1,6.7723384E7,4.26E2,-3.265896E-2,-1.1370972E-2,1.9375E0,2.9673737E-2,5.4261524E-2,1.8702E3,1.3407223E5,3.5443038E-1,3.4125953E-3,1.8224286E2,9.25E0,6.83E2,-1.4499371E-2,5.668E3,7.919006E7,4.032683E6,2.0619047E1,3.22471E-2,3.78E2,6E0,1.6964285E0,1E0,1.40943155E-2,5.5426865E6,1.4511137E-3,7.5982756E5,3.19E2,-2.143096E-2,-1.0843384E-2,-2.4760705E-3,3.5690453E-3,1E0,2.6551678E0,8.129368E-3,-7.818485E-3,-1.9816414E-2,-8.8228155E-3,3.6545064E0,1.131645E6,1.8666667E0,1.321E3,1.8461539E0,3.1E1,1.4473684E-1,1.162E3,5.0857143E0,9.806096E-4,2.5338028E1,4.238532E0,-8.922027E-3,-2.0284507E-2,1.775543E-2,1.1997242E0,4.39E2,-2.2800947E-2,9.389317E-3,4.8E2,4.76E2,2.6742857E0,4.5007843E2,2.5795715E-2,-1.0187093E-2,1.57922E5,1.3558896E7,1.2906634E7,1.2593857E0,1.2458569E-3,3.532955E-2,1.8113649E-2,-2.7654916E-2,-1.7329386E-2,-5.8586948E-2,-1.6476398E-2,-7.793013E-3,-1.4511143E-2,-2.4745919E-2,-8.388087E-3,4.3713252E-4,-1.0006327E-2,1.2125534E-2,1.3484611E-3,-6.8290457E-3,-1.695892E-2,-3.6820475E-2,-1.9587021E-2,-1.465355E-2,-2.9583767E-3,1.3394385E-3,-6.4194133E-3,8.926649E-3,8.186198E-4,5.623749E-3,-2.6150998E-3,-1.3112627E-2,-2.2736997E-3,-5.1064626E-3,6.011772E-3,-6.6782936E-4,5.963149E-3,-1.0094258E-2,2.7616597E-3,1.0188209E-2,2.0085268E-2,-3.1570237E-4,9.261277E-3,1.8964235E-2,5.2779885E-3,7.99439E-3,-1.0200555E-3,2.6350676E-3,2.083221E-2],"split_indices":[37,38,2,61,2,46,37,54,0,61,10,2,4,38,63,5,0,63,62,0,2,37,63,38,7,2,54,0,0,0,0,9,37,38,61,3,67,9,37,3,7,2,0,0,62,0,0,61,42,65,0,4,67,2,0,38,7,54,65,0,0,8,62,112,0,37,0,37,0,0,0,0,0,111,51,0,0,0,0,62,9,67,10,62,0,65,10,63,0,67,62,0,0,0,47,2,0,0,10,2,65,61,0,0,1,9,12,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.71E2,4.51E2,5.2E2,9.2E1,3.59E2,3.15E2,2.05E2,8.5E1,7E0,2.45E2,1.14E2,2.81E2,3.4E1,1.54E2,5.1E1,6E1,2.5E1,1.3E2,1.15E2,9E0,1.05E2,1.36E2,1.45E2,1.7E1,1.7E1,2.3E1,1.31E2,3.1E1,2E1,5.2E1,8E0,2.1E1,1.09E2,4.4E1,7.1E1,8.5E1,2E1,6.5E1,7.1E1,7E1,7.5E1,1.2E1,5E0,6E0,1.1E1,1.2E1,1.1E1,1.19E2,1.2E1,1.3E1,8E0,4.8E1,6.1E1,3.8E1,6E0,5E1,2.1E1,7.4E1,1.1E1,5E0,1.5E1,2.4E1,4.1E1,6.2E1,9E0,6.4E1,6E0,3.4E1,4.1E1,7E0,5E0,5E0,6E0,9.4E1,2.5E1,7E0,5E0,5E0,8E0,3.8E1,1E1,2.5E1,3.6E1,2.1E1,1.7E1,3.5E1,1.5E1,1.6E1,5E0,5.5E1,1.9E1,5E0,6E0,5E0,1E1,1.7E1,7E0,9E0,3.2E1,4.8E1,1.4E1,5.6E1,8E0,7E0,2.7E1,2.5E1,1.6E1,8.9E1,5E0,2E1,5E0,3E1,8E0,5E0,5E0,6E0,1.9E1,3.1E1,5E0,1.1E1,1E1,5E0,1.2E1,8E0,2.7E1,7E0,8E0,8E0,8E0,3.4E1,2.1E1,1.4E1,5E0,5E0,5E0,7E0,1E1,2.6E1,6E0,1E1,3.8E1,5E0,9E0,3.3E1,2.3E1,2.2E1,5E0,1.2E1,1.3E1,6E0,1E1,6E0,8.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[2.533548E-4,-2.7424982E-1,2.4890102E-1,-5.2003425E-1,-1.2977341E-1,3.099552E-1,-7.159112E-1,-7.147642E-1,-4.232321E-1,-3.295891E-1,-6.833058E-2,1.6344771E-1,6.339619E-1,-4.670311E-2,-5.4342294E-1,-5.957759E-1,-5.4692596E-2,-5.556233E-1,-3.1488147E-1,-4.155169E-1,-1.06882885E-1,-4.9443554E-2,-3.4044497E-2,2.8290996E-1,-3.5042413E-2,6.675292E-1,-1.2088466E-2,-3.6422387E-1,-3.1237E-2,-3.0169435E-2,-1.7093986E-2,-5.8971524E-1,-1.405559E-2,1.998061E-2,-3.841838E-1,-3.4985492E-1,-6.045749E-1,1.4582631E-3,-9.3592005E-3,-8.1127755E-2,1.15821704E-1,2.3557171E-1,6.105355E-1,-2.9307747E-1,3.377344E-2,6.0843205E-1,5.0630983E-2,-9.180357E-3,-2.1802554E-2,-3.154231E-2,-4.9766237E-1,-3.287515E-1,-2.6842847E-2,-1.135542E-2,-4.077728E-1,-1.7437574E-2,-3.399094E-2,-4.2460963E-2,-2.470559E-1,2.2087353E-1,3.526936E-2,2.7616128E-1,-6.664308E-2,2.0217283E-2,3.426057E-2,-3.8286906E-1,-5.156087E-3,-5.814321E-2,1.4790168E-1,9.0134746E-1,5.232606E-1,-2.4825394E-2,-1.3462175E-2,-3.7628844E-1,-1.7672896E-3,-2.0649932E-2,-1.0578466E-2,5.0164584E-2,-1.4638893E-1,-1.02373414E-1,-3.8571644E-1,1.0502759E-3,1.6263785E-2,-2.9526865E-2,5.080637E-3,1.28338E-1,4.3804124E-1,1.0363734E-1,-2.1717988E-1,-2.7063316E-1,-2.5219437E-2,1.5279867E-2,-1.9806482E-1,9.045586E-2,1.5505376E-2,5.5830646E-1,5.697056E-2,3.658341E-1,6.673402E-1,-2.2352137E-2,-1.2446032E-2,3.2832678E-3,-9.846814E-3,-1.0031371E-2,-2.139296E-3,-7.0399083E-3,1.0029387E-3,-2.2495864E-2,-5.7164757E-3,2.169597E-3,-5.271254E-3,7.4819457E-3,-6.09422E-3,2.1380395E-2,4.622705E-3,8.06788E-3,7.480896E-4,-1.6260363E-2,-1.1836097E-3,-6.6857086E-3,-1.579605E-2,7.709478E-3,-2.169338E-3,-1.5296236E-2,-7.059015E-4,2.24242E-4,7.5308583E-3,3.1491123E-2,1.2277514E-2,1.915275E-2,4.9231923E-3,3.339414E-2,1.3728937E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":8,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,-1,31,33,35,37,39,-1,41,43,45,-1,47,-1,-1,-1,49,-1,-1,51,53,55,-1,-1,57,59,61,63,65,67,69,-1,-1,-1,-1,71,73,-1,-1,75,-1,-1,77,79,81,83,85,87,-1,-1,89,-1,91,93,95,97,-1,-1,99,-1,-1,-1,101,103,105,107,-1,-1,109,-1,111,113,115,117,119,-1,121,123,125,-1,127,-1,129,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.491011E1,1.5983059E1,2.9615906E1,2.8454857E0,3.4887705E0,2.2207E1,9.569502E-1,2.2789211E0,1.4603386E0,1.2377443E0,2.7285404E0,7.71634E0,4.557667E0,0E0,2.4324656E-1,3.1741524E-1,0E0,2.5210857E-1,3.5236726E0,3.843546E-1,2.6865143E-1,1.1352003E0,0E0,3.004549E0,2.1939974E0,2.866394E0,0E0,1.0040271E-1,0E0,0E0,0E0,1.8267632E-2,0E0,0E0,4.5615292E-1,1.3156748E-1,4.6778202E-2,0E0,0E0,1.1507894E0,2.8838596E-1,2.2141562E0,2.3700619E-1,3.8223934E-1,1.0366474E0,2.671814E0,0E0,0E0,0E0,0E0,3.867054E-2,6.6271114E-1,0E0,0E0,5.2667618E-2,0E0,0E0,1.4320486E0,6.4261246E-1,3.7756997E-1,1.04237445E-1,3.7246628E0,5.8644426E-1,0E0,0E0,1.5944433E-1,0E0,5.6806755E-1,4.4059187E-1,2.3384056E0,2.0240612E0,0E0,0E0,3.4930754E-1,0E0,0E0,0E0,4.419446E-1,4.800644E-1,1.1342032E-1,3.825915E-1,0E0,0E0,8.228094E-2,0E0,7.3911417E-1,5.3148556E-1,6.481535E-2,2.9757297E-1,5.6943774E-2,0E0,3.540618E-1,4.5218468E-1,2.1453366E-1,0E0,3.5079527E-1,0E0,5.6586504E-1,7.7915573E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,27,27,31,31,34,34,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,50,50,51,51,54,54,57,57,58,58,59,59,60,60,61,61,62,62,65,65,67,67,68,68,69,69,70,70,73,73,77,77,78,78,79,79,80,80,83,83,85,85,86,86,87,87,88,88,89,89,91,91,92,92,93,93,95,95,97,97,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,-1,32,34,36,38,40,-1,42,44,46,-1,48,-1,-1,-1,50,-1,-1,52,54,56,-1,-1,58,60,62,64,66,68,70,-1,-1,-1,-1,72,74,-1,-1,76,-1,-1,78,80,82,84,86,88,-1,-1,90,-1,92,94,96,98,-1,-1,100,-1,-1,-1,102,104,106,108,-1,-1,110,-1,112,114,116,118,120,-1,122,124,126,-1,128,-1,130,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.037E3,1.972052E5,1E0,3.3832976E7,4.58E2,1.0879832E3,1.5889913E-2,2.36562E5,2.85E2,7.9016544E5,4.2E1,8E0,7.706545E7,-4.670311E-2,3.9416668E1,2.9266055E0,-5.4692596E-2,3.328629E6,2.00087E5,1.7913036E6,8.032258E0,3.130786E6,-3.4044497E-2,6.0052995E6,1.8439855E5,6.663214E6,-1.2088466E-2,2.4860917E5,-3.1237E-2,-3.0169435E-2,-1.7093986E-2,3.89E2,-1.405559E-2,1.998061E-2,9.3262803E8,3.1E1,1.0065E5,1.4582631E-3,-9.3592005E-3,6.33E3,2.1111E4,1.4586467E0,5.90352E5,2.147382E4,2.126E3,3.277778E1,5.0630983E-2,-9.180357E-3,-2.1802554E-2,-3.154231E-2,1.8E1,4.0911578E2,-2.6842847E-2,-1.135542E-2,3.4698796E0,-1.7437574E-2,-3.399094E-2,1E0,1.6386554E0,2.8784848E2,1.3826087E1,4.716184E2,3E0,2.0217283E-2,3.426057E-2,6.4553E4,-5.156087E-3,7.1E1,1.6566709E6,1.5632523E3,1.7831801E6,-2.4825394E-2,-1.3462175E-2,7.450166E4,-1.7672896E-3,-2.0649932E-2,-1.0578466E-2,1.2E1,2.118835E6,9.3411E4,1.2585366E1,1.0502759E-3,1.6263785E-2,1.6496063E2,5.080637E-3,1.2728624E0,3.0011453E5,1.753E3,2.2318378E6,3.2216358E-1,-2.5219437E-2,8.762215E0,1.9212E4,3.1797794E1,1.5505376E-2,6.149315E6,5.697056E-2,4.909091E0,4.052642E0,-2.2352137E-2,-1.2446032E-2,3.2832678E-3,-9.846814E-3,-1.0031371E-2,-2.139296E-3,-7.0399083E-3,1.0029387E-3,-2.2495864E-2,-5.7164757E-3,2.169597E-3,-5.271254E-3,7.4819457E-3,-6.09422E-3,2.1380395E-2,4.622705E-3,8.06788E-3,7.480896E-4,-1.6260363E-2,-1.1836097E-3,-6.6857086E-3,-1.579605E-2,7.709478E-3,-2.169338E-3,-1.5296236E-2,-7.059015E-4,2.24242E-4,7.5308583E-3,3.1491123E-2,1.2277514E-2,1.915275E-2,4.9231923E-3,3.339414E-2,1.3728937E-2],"split_indices":[2,37,78,7,2,61,47,9,2,37,3,26,53,0,65,62,0,9,5,54,67,37,0,37,37,37,0,42,0,0,0,1,0,0,40,10,12,0,0,38,9,55,9,42,2,67,0,0,0,0,3,4,0,0,65,0,0,23,63,42,67,64,8,0,0,38,0,8,37,61,37,0,0,37,0,0,0,65,9,1,65,0,0,4,0,47,42,38,37,47,0,67,10,65,0,54,0,62,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.49E2,4.51E2,4.98E2,1.66E2,2.85E2,4.69E2,2.9E1,5.3E1,1.13E2,6.6E1,2.19E2,3.24E2,1.45E2,9E0,2E1,4.4E1,9E0,4.9E1,6.4E1,4.7E1,1.9E1,2.14E2,5E0,2.02E2,1.22E2,1.4E2,5E0,1E1,1E1,3.4E1,1E1,4.2E1,7E0,5E0,5.9E1,3.7E1,1E1,8E0,1.1E1,1.8E2,3.4E1,1.78E2,2.4E1,2.5E1,9.7E1,1.25E2,1.5E1,5E0,5E0,1.8E1,2.4E1,4.8E1,1.1E1,1.5E1,2.2E1,5E0,5E0,1.47E2,3.3E1,1.4E1,2E1,1.57E2,2.1E1,1.2E1,1.2E1,1.6E1,9E0,5.4E1,4.3E1,2.6E1,9.9E1,1.9E1,5E0,4.1E1,7E0,1.7E1,5E0,7.8E1,6.9E1,1.7E1,1.6E1,6E0,8E0,1.1E1,9E0,8.3E1,7.4E1,1E1,1.1E1,1.1E1,5E0,3.6E1,1.8E1,3.4E1,9E0,1.4E1,1.2E1,4.9E1,5E1,1.9E1,2.2E1,7.3E1,5E0,4E1,2.9E1,1.2E1,5E0,1.1E1,5E0,6E0,5E0,7.4E1,9E0,6.9E1,5E0,5E0,5E0,6E0,5E0,5E0,6E0,1E1,2.6E1,1E1,8E0,1.6E1,1.8E1,9E0,5E0,4.1E1,8E0,4.3E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"133","size_leaf_vector":"1"}},{"base_weights":[7.910148E-4,-1.5733899E-1,4.5996022E-1,-4.5424694E-1,-1.2044883E-2,5.6436545E-1,-2.8620863E-1,-6.3163894E-1,-3.2509404E-1,-1.7275664E-1,1.1881369E-1,2.6014945E-1,6.706347E-1,1.03602104E-1,-5.5183816E-1,-6.637081E-1,-3.9868838E-1,1.9239506E-2,-3.6300898E-1,-2.4310169E-1,2.0329216E-2,4.6874378E-2,3.1135252E-1,2.8555855E-2,4.907232E-1,7.312863E-2,6.299465E-1,-2.9075537E-3,9.992164E-3,-6.742954E-1,-9.026586E-3,-6.786886E-1,-1.6742775E-2,-1.0302979E-2,-2.1907419E-2,-4.74579E-1,-1.3922578E-1,-3.0578536E-1,-2.914445E-2,-1.6270531E-2,7.0851035E-2,6.2650174E-2,-2.063517E-2,1.3174039E-1,4.0820947E-1,-6.40873E-2,6.844283E-3,2.8367865E-1,6.080062E-1,7.5124264E-1,4.0677598E-1,-3.641016E-2,-1.9241415E-2,-7.152347E-1,-5.076299E-1,-4.1681832E-1,-4.539826E-2,1.34781655E-2,-2.4278522E-1,-3.6210215E-1,-1.1852282E-1,-9.168837E-2,8.856314E-2,-3.9997905E-2,1.2745191E-1,-5.5093788E-2,1.159297E-1,1.90543E-1,-4.4555226E-3,3.18818E-1,2.7068252E-2,1.0512189E-2,-7.988109E-3,1.7209882E-2,6.2468443E-3,1.6834892E-2,3.2885898E-2,7.78389E-1,1.6487299E-2,5.769518E-1,2.6643234E-1,-3.4500558E-2,-2.1760503E-2,-3.112623E-2,-4.4830926E-3,-4.9366766E-1,-3.0829862E-1,-1.779779E-1,-2.4923848E-2,-2.8473684E-1,-4.6731812E-1,5.5120876E-3,-1.729142E-1,1.3487942E-3,-1.2499665E-1,7.766275E-3,8.528684E-4,-9.173589E-3,5.533351E-2,8.656632E-2,1.4097234E-2,2.3027746E-2,-1.6672544E-1,5.3075444E-2,2.563938E-1,2.34878E-1,4.1359887E-3,4.0377206E-3,3.585204E-1,-2.3732467E-3,3.2674721E-3,5.8875424E-1,8.695945E-1,3.788216E-1,3.7882883E-2,1.5206708E-1,2.7399765E-2,-2.0125965E-2,-2.8641844E-2,-4.962937E-3,-1.7347736E-2,-1.13736E-2,-3.0693628E-3,-1.4849058E-2,-6.1067087E-3,-1.6447892E-2,-2.7526736E-2,-1.1529378E-2,-3.5560725E-3,-7.7676787E-3,-2.4135427E-4,4.9360604E-3,-6.2385335E-4,7.6980023E-3,1.975106E-3,3.0254386E-3,-7.1915914E-3,-1.09190205E-2,-1.2176377E-3,5.213775E-3,-6.15269E-3,1.5413385E-2,3.677141E-3,6.1383233E-3,1.2860539E-2,1.8761238E-2,9.2404755E-3,1.580198E-2,3.259749E-2,4.499962E-2,3.240942E-2,2.278905E-2,1.2692216E-2,9.0062E-3,-6.214106E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":9,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,43,45,47,-1,49,-1,-1,51,-1,53,-1,-1,-1,55,57,59,61,-1,63,65,-1,67,69,71,-1,73,75,77,79,-1,-1,81,83,85,-1,-1,87,89,91,93,95,97,99,101,103,105,-1,107,-1,109,-1,-1,-1,-1,-1,111,-1,113,115,-1,-1,-1,-1,117,119,121,-1,123,125,-1,127,-1,129,-1,-1,-1,131,133,-1,135,137,139,141,143,-1,-1,145,-1,-1,147,149,151,-1,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.774343E1,2.993298E1,1.8780563E1,5.0182953E0,9.841775E0,6.614586E0,3.1635282E0,4.4508743E-1,3.908866E0,2.8572946E0,3.5589924E0,2.976354E0,4.7174454E0,2.5503507E-1,6.563878E-1,1.2714386E-1,7.682276E-2,0E0,3.1297646E0,2.0564795E0,1.0932058E0,1.4936405E0,1.16399E0,3.2903636E-1,5.074401E-1,0E0,3.7548676E0,0E0,0E0,4.722643E-2,0E0,1.1829376E-1,0E0,0E0,0E0,2.0658798E0,2.028535E0,1.21206E0,2.7314842E-1,0E0,3.2507932E-1,1.160682E0,0E0,3.665207E-1,5.186329E-1,1.3886634E-1,0E0,1.0607785E-1,1.6278076E-1,7.551727E-1,1.1598511E0,0E0,0E0,7.671356E-2,1.0755095E0,4.982977E-1,0E0,0E0,5.9118986E-1,6.076832E-1,3.9487725E-1,1.0016698E-1,6.8832226E-2,2.8156707E-1,2.1887517E-1,5.096478E-1,1.1076452E0,7.347411E-2,0E0,2.6329398E-1,0E0,4.4235606E-2,0E0,0E0,0E0,0E0,0E0,1.0273438E0,0E0,8.295636E-1,1.0962603E0,0E0,0E0,0E0,0E0,5.8135033E-2,4.2213774E-1,2.1655542E-1,0E0,2.5508928E-1,3.1546974E-1,0E0,1.55559E-1,0E0,9.56642E-2,0E0,0E0,0E0,4.3562967E-2,9.417425E-2,0E0,2.728968E-1,2.2511995E-1,9.947194E-1,4.9731588E-1,2.4072051E-2,0E0,0E0,1.16645336E-1,0E0,0E0,6.6444397E-1,3.173561E-1,4.3036222E-2,0E0,1.7647523E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,29,29,31,31,35,35,36,36,37,37,38,38,40,40,41,41,43,43,44,44,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,69,69,71,71,77,77,79,79,80,80,85,85,86,86,87,87,89,89,90,90,92,92,94,94,98,98,99,99,101,101,102,102,103,103,104,104,105,105,108,108,111,111,112,112,113,113,115,115],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,44,46,48,-1,50,-1,-1,52,-1,54,-1,-1,-1,56,58,60,62,-1,64,66,-1,68,70,72,-1,74,76,78,80,-1,-1,82,84,86,-1,-1,88,90,92,94,96,98,100,102,104,106,-1,108,-1,110,-1,-1,-1,-1,-1,112,-1,114,116,-1,-1,-1,-1,118,120,122,-1,124,126,-1,128,-1,130,-1,-1,-1,132,134,-1,136,138,140,142,144,-1,-1,146,-1,-1,148,150,152,-1,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,1.16E2,8.49E2,7.079871E2,1.8057558E7,8.9044176E-2,6E0,2.0766992E6,2.0766992E6,9.4996644E5,3.3817584E7,8.620714E2,2.3017536E5,4.238532E0,1.4E1,1.9239506E-2,8.01E2,4.39776E6,2.1539131E2,1.2784314E1,5.1604336E7,5.989087E7,5.941442E6,7.312863E-2,8E0,-2.9075537E-3,9.992164E-3,9.3326636E-2,-9.026586E-3,7.7E1,-1.6742775E-2,-1.0302979E-2,-2.1907419E-2,5.2405543E0,1.088E3,2.5871112E2,1.7913036E6,-1.6270531E-2,2.9547384E9,4.7567694E5,-2.063517E-2,2.2292225E-1,4.9321495E6,1.3E1,6.844283E-3,7.714286E0,2.4180895E6,6.292039E0,8.598131E1,-3.641016E-2,-1.9241415E-2,5.159652E-7,5.4E1,1.1513467E2,-4.539826E-2,1.34781655E-2,2.5E1,2.28E0,2.63E2,1.6129032E0,7.3131656E5,2.4888637E2,3.056087E2,1.9E1,7.6937294E-1,2.138772E4,-4.4555226E-3,1.194E3,2.7068252E-2,2.2003597E1,-7.988109E-3,1.7209882E-2,6.2468443E-3,1.6834892E-2,3.2885898E-2,1.2400667E3,1.6487299E-2,2.6463525E8,4.435876E2,-3.4500558E-2,-2.1760503E-2,-3.112623E-2,-4.4830926E-3,1.737736E1,2E0,3.422351E6,-2.4923848E-2,8.1632656E-1,4.8667232E8,5.5120876E-3,2.9891E4,1.3487942E-3,3.056087E2,7.766275E-3,8.528684E-4,-9.173589E-3,3.416724E7,1.0140845E0,1.4097234E-2,1E0,1.33E2,1.2906634E7,3.52E2,6.264151E0,4.1359887E-3,4.0377206E-3,1.5E1,-2.3732467E-3,3.2674721E-3,4E0,1.7103828E7,1.2E1,3.7882883E-2,4.1885E4,2.7399765E-2,-2.0125965E-2,-2.8641844E-2,-4.962937E-3,-1.7347736E-2,-1.13736E-2,-3.0693628E-3,-1.4849058E-2,-6.1067087E-3,-1.6447892E-2,-2.7526736E-2,-1.1529378E-2,-3.5560725E-3,-7.7676787E-3,-2.4135427E-4,4.9360604E-3,-6.2385335E-4,7.6980023E-3,1.975106E-3,3.0254386E-3,-7.1915914E-3,-1.09190205E-2,-1.2176377E-3,5.213775E-3,-6.15269E-3,1.5413385E-2,3.677141E-3,6.1383233E-3,1.2860539E-2,1.8761238E-2,9.2404755E-3,1.580198E-2,3.259749E-2,4.499962E-2,3.240942E-2,2.278905E-2,1.2692216E-2,9.0062E-3,-6.214106E-4],"split_indices":[2,37,6,38,2,61,54,51,3,37,37,37,7,4,42,62,3,0,2,9,42,62,7,5,54,0,26,0,0,47,0,38,0,0,0,62,38,61,54,0,40,37,0,51,37,3,0,65,37,47,67,0,0,46,2,61,0,0,3,65,38,62,37,61,4,8,47,56,0,2,0,67,0,0,0,0,0,4,0,7,65,0,0,0,0,67,26,9,0,65,40,0,12,0,4,0,0,0,7,65,0,85,8,12,0,67,0,0,3,0,0,8,56,3,0,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.31E2,6.93E2,2.38E2,2.27E2,4.66E2,2.09E2,2.9E1,9.4E1,1.33E2,2.09E2,2.57E2,5.5E1,1.54E2,1.2E1,1.7E1,8.1E1,1.3E1,6E0,1.27E2,1.53E2,5.6E1,1.88E2,6.9E1,2.8E1,2.7E1,5E0,1.49E2,5E0,7E0,1.2E1,5E0,7.6E1,5E0,5E0,8E0,8.4E1,4.3E1,1.18E2,3.5E1,6E0,5E1,1.83E2,5E0,2.5E1,4.4E1,1.6E1,1.2E1,1.1E1,1.6E1,9.5E1,5.4E1,7E0,5E0,6E1,1.6E1,7.7E1,7E0,8E0,3.5E1,9E1,2.8E1,2.3E1,1.2E1,1.7E1,3.3E1,5.7E1,1.26E2,2E1,5E0,3.1E1,1.3E1,1E1,6E0,6E0,5E0,6E0,1E1,8.8E1,7E0,2.3E1,3.1E1,5.2E1,8E0,1.1E1,5E0,4.3E1,3.4E1,3E1,5E0,5.4E1,3.6E1,5E0,2.3E1,5E0,1.8E1,5E0,7E0,6E0,1.1E1,2.8E1,5E0,3.4E1,2.3E1,8.8E1,3.8E1,1.3E1,7E0,5E0,2.6E1,5E0,5E0,3.1E1,5.7E1,1.4E1,9E0,2.4E1,7E0,3.2E1,1.1E1,9E0,2.5E1,1.8E1,1.2E1,4.3E1,1.1E1,2.1E1,1.5E1,1.2E1,1.1E1,1.3E1,5E0,6E0,5E0,9E0,1.9E1,2.8E1,6E0,1.5E1,8E0,6.7E1,2.1E1,2.6E1,1.2E1,5E0,8E0,1.9E1,7E0,1.1E1,2E1,3.3E1,2.4E1,5E0,9E0,1.9E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"155","size_leaf_vector":"1"}},{"base_weights":[-2.2268744E-2,-1.6335955E-1,4.3987218E-1,-4.8660174E-1,-2.0297447E-2,5.3016996E-1,-1.5570572E-1,-6.5271616E-1,-3.6495864E-1,-1.4276879E-1,1.1769675E-1,5.424879E-2,4.6580175E-1,-2.9115647E-2,7.637299E-2,-5.1668537E-1,-7.643216E-1,-3.987707E-1,1.3500596E-2,-1.8381317E-1,1.3572127E-1,1.5275857E-1,-2.5188306E-1,1.5583901E-1,6.23425E-1,-1.08821414E-1,2.546244E-1,-5.487637E-1,-1.0055481E-2,-3.7499208E-2,-5.393294E-1,-1.8195013E-2,-4.3533328E-1,-2.9641682E-1,-9.455567E-2,2.2993444E-1,4.428205E-2,1.2102937E-1,4.6082625E-1,-1.0217728E-1,-2.4902461E-2,-7.5175926E-2,2.3280422E-1,7.554973E-1,4.940787E-1,1.1991212E-3,-1.0456078E-2,4.104108E-3,1.7555661E-2,-2.6574757E-2,-1.4729317E-2,-3.3110052E-2,-1.2768313E-2,-4.9126646E-3,4.0429668E-3,-3.9584503E-1,-3.4242593E-2,-2.2539029E-1,-4.4696346E-1,-2.6591964E-2,-6.721888E-2,5.767804E-3,1.5461489E-2,-1.9309752E-3,5.8348305E-3,-7.0546895E-2,1.5679967E-1,1.2055827E-2,2.451789E-2,-9.767767E-3,2.3993477E-3,-1.0022312E-2,8.471868E-3,2.8029647E-1,-6.0563725E-3,1.2231221E-2,3.8065493E-2,5.287671E-1,3.8136824E-3,-4.7234625E-1,-2.6863965E-1,-2.9364583E-1,-1.4046805E-1,-1.5273926E-2,-5.315231E-1,-3.0032653E-2,-2.5171986E-1,-2.1479748E-1,1.5241393E-2,1.6976012E-1,-1.0274029E-2,2.7257716E-3,-2.0051072E-3,2.1163261E-1,2.7077463E-2,4.1336668E-1,6.0600597E-1,-2.3092508E-2,-6.896534E-3,-1.4240776E-2,-4.5675472E-3,-1.4803224E-2,-6.1240387E-3,-8.477459E-3,2.252755E-3,-1.4643219E-2,-2.7616726E-2,1.9910128E-3,-5.851358E-3,-1.6895691E-2,-2.8476336E-3,-1.2599084E-2,-6.2861717E-3,6.4775376E-3,-4.1603628E-3,1.1218601E-2,5.0415206E-3,3.6463372E-3,1.3301738E-2,2.1092242E-2,9.290345E-3,1.6654892E-2,2.9624322E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":10,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,-1,-1,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,79,-1,81,83,-1,85,-1,-1,-1,-1,87,89,-1,-1,-1,-1,-1,91,93,-1,-1,-1,95,-1,97,99,101,103,-1,105,107,109,111,113,115,-1,-1,-1,117,-1,119,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.1885098E1,3.36093E1,1.20215645E1,4.267872E0,8.551472E0,7.393738E0,3.3327293E0,1.0169945E0,3.0213623E0,3.0883083E0,3.1225152E0,0E0,8.550594E0,0E0,7.2094303E-1,3.1408787E-1,4.7676086E-2,1.7314339E0,0E0,2.3238401E0,2.9133528E-1,2.075121E0,8.405684E-1,1.1020881E0,1.6254463E0,1.7843829E-1,1.9270897E-1,4.002285E-2,0E0,0E0,3.099606E-1,1.20052025E-1,1.0703526E0,1.000618E0,1.6883194E0,1.440534E-1,1.3819847E-1,1.3747575E0,1.3438892E-1,2.609639E-1,0E0,1.9030508E-1,8.303354E-1,1.5711975E0,8.423767E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,8.754406E-1,0E0,3.7609363E-1,1.4373255E-1,0E0,8.56913E-1,0E0,0E0,0E0,0E0,3.9927414E-1,8.611207E-1,0E0,0E0,0E0,0E0,0E0,3.1121673E-2,7.30288E-1,0E0,0E0,0E0,2.5756073E-1,0E0,4.4287872E-1,2.3689532E-1,1.17990255E-1,2.8032082E-1,0E0,4.5894146E-2,7.4826527E-1,4.2701316E-1,1.7216563E-2,2.8649005E-1,6.9109154E-1,0E0,0E0,0E0,3.2693505E-1,0E0,1.3737202E-1,2.7431488E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,55,55,57,57,58,58,60,60,65,65,66,66,72,72,73,73,77,77,79,79,80,80,81,81,82,82,84,84,85,85,86,86,87,87,88,88,89,89,93,93,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,-1,-1,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,80,-1,82,84,-1,86,-1,-1,-1,-1,88,90,-1,-1,-1,-1,-1,92,94,-1,-1,-1,96,-1,98,100,102,104,-1,106,108,110,112,114,116,-1,-1,-1,118,-1,120,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,9.9E1,9.31E2,3.6E1,6.102135E-8,8.75477E5,4.9213043E2,3.3486558E6,9.639872E0,5.424879E-2,8.995735E5,-2.9115647E-2,1.0089981E3,9.932432E0,7.7E1,1.9186046E0,1.3500596E-2,5.47E2,2.1525E4,4.9321495E6,8.387763E2,2.8204132E7,2.2842104E0,1.6E1,1.9495455E6,1.8388145E-7,-1.0055481E-2,-3.7499208E-2,9E0,1.4328432E2,5.444797E0,4.2619047E0,1.45064E5,3.3817584E7,1.4165217E1,4.6303625E5,3.574764E7,1.1290322E1,-2.4902461E-2,1.5204346E8,2.6912618E7,3.0984934E8,2.4157143E2,1.1991212E-3,-1.0456078E-2,4.104108E-3,1.7555661E-2,-2.6574757E-2,-1.4729317E-2,-3.3110052E-2,-1.2768313E-2,-4.9126646E-3,4.0429668E-3,1.318849E6,-3.4242593E-2,1.5341322E2,7.106795E6,-2.6591964E-2,1.8E1,5.767804E-3,1.5461489E-2,-1.9309752E-3,5.8348305E-3,1.212945E6,1.4650772E7,1.2055827E-2,2.451789E-2,-9.767767E-3,2.3993477E-3,-1.0022312E-2,2.4702472E5,2.2371138E3,-6.0563725E-3,1.2231221E-2,3.8065493E-2,1.2615725E5,3.8136824E-3,2.9768292E2,1.1995476E6,5E0,9.33949E5,-1.5273926E-2,3.235955E0,1E0,3.9595376E7,1.7741072E1,2.192E3,3.464E0,-1.0274029E-2,2.7257716E-3,-2.0051072E-3,3.807421E6,2.7077463E-2,1.10198E7,2.512195E0,-2.3092508E-2,-6.896534E-3,-1.4240776E-2,-4.5675472E-3,-1.4803224E-2,-6.1240387E-3,-8.477459E-3,2.252755E-3,-1.4643219E-2,-2.7616726E-2,1.9910128E-3,-5.851358E-3,-1.6895691E-2,-2.8476336E-3,-1.2599084E-2,-6.2861717E-3,6.4775376E-3,-4.1603628E-3,1.1218601E-2,5.0415206E-3,3.6463372E-3,1.3301738E-2,2.1092242E-2,9.290345E-3,1.6654892E-2,2.9624322E-2],"split_indices":[2,37,6,38,2,0,46,40,61,37,63,0,37,0,4,63,38,63,0,2,9,37,4,5,63,3,37,46,0,0,3,61,62,63,7,7,67,37,7,63,0,7,56,7,67,0,0,0,0,0,0,0,0,0,0,9,0,61,5,0,8,0,0,0,0,9,60,0,0,0,0,0,42,61,0,0,0,42,0,61,54,8,12,0,65,23,7,67,38,63,0,0,0,56,0,56,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.47E2,7.26E2,2.21E2,2.22E2,5.04E2,1.92E2,2.9E1,9.2E1,1.3E2,2.67E2,2.37E2,1.6E1,1.76E2,9E0,2E1,4.4E1,4.8E1,1.24E2,6E0,2.33E2,3.4E1,2.17E2,2E1,6E1,1.16E2,1E1,1E1,3.9E1,5E0,3.8E1,1E1,1.1E1,1.13E2,1.02E2,1.31E2,1.6E1,1.8E1,1.98E2,1.9E1,1.4E1,6E0,1.5E1,4.5E1,5.5E1,6.1E1,5E0,5E0,5E0,5E0,3.4E1,5E0,5E0,5E0,6E0,5E0,1.02E2,1.1E1,7.1E1,3.1E1,6E0,1.25E2,9E0,7E0,9E0,9E0,3.1E1,1.67E2,6E0,1.3E1,8E0,6E0,5E0,1E1,4E1,5E0,7E0,4.8E1,5.6E1,5E0,6.2E1,4E1,3.8E1,3.3E1,1.5E1,1.6E1,1.05E2,2E1,1.1E1,2E1,1.62E2,5E0,5E0,5E0,3.4E1,6E0,2.5E1,3.1E1,5.7E1,5E0,3.2E1,8E0,3.2E1,6E0,2.7E1,6E0,5E0,1.1E1,6E1,4.5E1,1.2E1,8E0,5E0,6E0,9E0,1.1E1,7.3E1,8.9E1,1.3E1,2.1E1,2E1,5E0,5E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[7.978986E-3,-1.3934496E-1,4.589381E-1,-4.2467365E-1,-1.54729E-2,-2.6123673E-2,5.0029665E-1,-5.764647E-1,-3.5044515E-1,-1.349661E-1,1.1014876E-1,2.4679703E-1,5.972624E-1,-6.6627645E-1,-4.2552522E-1,-1.407205E-1,-4.1270873E-1,-1.9191803E-1,6.2376846E-2,2.2577047E-1,-9.071494E-3,1.4175831E-1,5.473957E-1,9.1876376E-1,5.0091416E-1,-5.02786E-1,-3.7440285E-2,-4.864165E-1,-9.153312E-3,-4.539488E-1,6.967709E-2,-4.5051298E-1,-1.5938245E-1,-3.889553E-1,-1.4866775E-1,3.2855805E-2,1.3243175E-2,2.4953556E-1,-7.787636E-3,1.9375047E-2,-1.7826188E-2,6.238622E-3,2.7182573E-1,1.499518E-2,2.9710356E-2,3.3923894E-2,5.5131864E-2,3.8248563E-1,7.165836E-1,-2.6936214E-2,-1.3198358E-2,-2.464238E-2,-1.2589027E-2,-2.6700374E-2,-1.43379625E-2,1.9418422E-2,-7.8056924E-2,-4.8077983E-1,-2.0399024E-1,5.5443654E-3,-2.855379E-1,-2.1588716E-1,-4.8491472E-1,-2.015384E-1,6.198483E-3,7.437582E-2,-6.43287E-2,1.7618746E-1,3.6848375E-1,4.8111837E-2,-1.558758E-1,1.3899283E-1,-8.5419804E-2,3.5383618E-1,-2.8078614E-3,4.3148497E-1,1.1525646E-1,3.7754325E-3,7.920643E-1,-9.060441E-3,-1.2945329E-2,-4.9961954E-1,-2.983438E-1,-4.8963847E-3,-1.1708378E-2,-1.8929867E-2,-6.4866072E-3,-1.2763278E-2,-6.0186624E-3,-5.597286E-1,-1.0855151E-2,-1.1827578E-1,-2.896081E-1,-4.381179E-2,8.364788E-2,3.9038178E-2,1.0687753E-2,3.682566E-2,-1.1874348E-2,1.3578817E-1,3.6773863E-1,3.9917412E-1,4.5170845E-3,-1.3386905E-1,7.358041E-2,-8.805207E-4,-1.4268505E-2,8.913541E-3,2.9099628E-3,-1.0876795E-2,3.616669E-3,2.5005214E-2,2.4387817E-1,2.3042381E-1,5.233859E-1,-8.19928E-3,1.2941315E-2,1.7879542E-2,8.378499E-1,-2.3110544E-3,1.4949191E-3,-2.106887E-2,-3.17902E-2,-1.7035097E-2,-8.343691E-3,-1.9428452E-2,-3.2542184E-2,-3.269786E-3,-1.2492517E-2,-1.5332348E-2,-2.0519397E-3,-4.7821547E-3,5.438213E-3,8.7559195E-3,-1.966376E-3,5.1206397E-3,-5.957234E-4,-7.366556E-4,4.276809E-3,4.3726484E-3,1.0515895E-2,1.016652E-2,2.0102246E-2,7.4360017E-3,2.0983487E-2,-3.6297055E-4,-9.821369E-3,-9.5991464E-4,4.822608E-3,3.089039E-3,-2.7813849E-3,1.4043145E-2,7.937225E-3,-7.419747E-3,1.5590333E-2,1.8420424E-2,2.8598126E-2,2.6281897E-2,4.210028E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":11,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,-1,51,-1,53,55,57,59,61,63,65,-1,67,-1,69,-1,71,73,-1,-1,-1,-1,75,77,-1,-1,-1,-1,-1,-1,-1,79,81,83,-1,85,87,89,91,93,95,97,99,101,103,105,107,109,111,-1,113,115,-1,117,-1,119,121,123,-1,-1,-1,-1,-1,-1,125,-1,127,129,131,133,135,-1,137,-1,139,141,143,-1,145,147,-1,-1,-1,-1,-1,149,-1,151,153,155,-1,-1,-1,157,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.6102394E1,2.6501621E1,1.0636993E1,2.3556366E0,7.880595E0,0E0,5.6536674E0,7.060814E-1,1.9659462E0,3.0352788E0,3.5308673E0,2.0378938E0,4.8057137E0,6.059761E-1,3.2100248E-1,2.4700215E0,1.0678349E0,1.7232556E0,3.922693E-1,1.2579236E0,1.3626385E0,8.972658E-1,1.4725828E-1,1.052414E0,3.1229248E0,2.2527218E-1,0E0,7.6928616E-2,0E0,8.6322784E-2,1.2179465E0,6.7045593E-1,6.3220596E-1,5.097914E-1,1.4177523E0,2.2671513E-1,0E0,1.0082064E0,0E0,6.068009E-1,0E0,3.282418E-1,7.3005295E-1,0E0,0E0,0E0,0E0,1.1025448E0,2.1410923E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2721568E-1,1.6371346E-1,3.525901E-2,0E0,1.5709335E-1,4.0823758E-2,3.0089092E-1,9.1169214E-1,1.779617E-1,2.0595612E-1,3.388824E-1,5.536709E-1,3.4391117E-1,4.8723382E-1,3.4707078E-1,3.2515556E-2,2.1310636E-1,2.9593396E-1,0E0,1.2467489E0,7.6601434E-1,0E0,4.4064713E-1,0E0,2.1582961E-2,3.732891E-1,2.3671627E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.5810223E-2,0E0,4.7177166E-1,6.066394E-1,2.7921996E-1,2.4632418E-1,1.2351619E-1,0E0,3.646692E-2,0E0,2.2777379E-1,3.164041E-2,4.472909E-1,0E0,1.258292E-1,2.62561E-1,0E0,0E0,0E0,0E0,0E0,4.800947E-2,0E0,1.2427211E-2,1.0840086E0,3.427744E-1,0E0,0E0,0E0,1.6510963E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,39,39,41,41,42,42,47,47,48,48,56,56,57,57,58,58,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,78,78,80,80,81,81,82,82,89,89,91,91,92,92,93,93,94,94,95,95,97,97,99,99,100,100,101,101,103,103,104,104,110,110,112,112,113,113,114,114,118,118],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,-1,52,-1,54,56,58,60,62,64,66,-1,68,-1,70,-1,72,74,-1,-1,-1,-1,76,78,-1,-1,-1,-1,-1,-1,-1,80,82,84,-1,86,88,90,92,94,96,98,100,102,104,106,108,110,112,-1,114,116,-1,118,-1,120,122,124,-1,-1,-1,-1,-1,-1,126,-1,128,130,132,134,136,-1,138,-1,140,142,144,-1,146,148,-1,-1,-1,-1,-1,150,-1,152,154,156,-1,-1,-1,158,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.7909248E5,1E0,7.7E1,9.48E2,-2.6123673E-2,8.40413E2,1.666618E-2,1.9333333E0,2.331083E6,3.7326388E0,6.103178E8,2.7271997E1,8.75477E5,5.9395876E-7,1.16E2,3.518913E2,1.5933333E2,6.9307615E6,4.11566E5,1.2784314E1,5.486433E5,6.714162E8,1.3388E4,2.9251662E6,3.89E2,-3.7440285E-2,2.6666667E0,-9.153312E-3,1.1176471E1,2E0,7.997723E6,2.059E3,4.8421054E0,3.422351E6,6.99E2,1.3243175E-2,4.797342E2,-7.787636E-3,2.1129E4,-1.7826188E-2,1.138871E0,8.954333E6,1.499518E-2,2.9710356E-2,3.3923894E-2,5.5131864E-2,2.7555334E7,1E0,-2.6936214E-2,-1.3198358E-2,-2.464238E-2,-1.2589027E-2,-2.6700374E-2,-1.43379625E-2,1.9418422E-2,6.848509E7,6.077143E5,3.1478872E0,5.5443654E-3,1E0,2.08E2,6.769866E7,1E0,7.919006E7,6.15E2,5.1167645E6,5.197839E6,2E0,2.7015875E2,1.9539816E7,3.6689E4,8.295515E7,3.402E3,-2.8078614E-3,3.6451373E8,3.433001E-2,3.7754325E-3,1.2400667E3,-9.060441E-3,2.1088235E0,8.009E3,7.450166E4,-4.8963847E-3,-1.1708378E-2,-1.8929867E-2,-6.4866072E-3,-1.2763278E-2,-6.0186624E-3,1.4473684E-1,-1.0855151E-2,4.556962E0,1.4272727E1,8.84439E5,1.3858744E4,8.83284E-4,1.0687753E-2,5.036081E7,-1.1874348E-2,6.933837E-1,1.99E2,5.1604336E7,4.5170845E-3,1.9225262E2,3.808001E5,-8.805207E-4,-1.4268505E-2,8.913541E-3,2.9099628E-3,-1.0876795E-2,7.0858893E0,2.5005214E-2,7.6E0,1.8181819E0,1.76812E3,-8.19928E-3,1.2941315E-2,1.7879542E-2,4.0705118E6,-2.3110544E-3,1.4949191E-3,-2.106887E-2,-3.17902E-2,-1.7035097E-2,-8.343691E-3,-1.9428452E-2,-3.2542184E-2,-3.269786E-3,-1.2492517E-2,-1.5332348E-2,-2.0519397E-3,-4.7821547E-3,5.438213E-3,8.7559195E-3,-1.966376E-3,5.1206397E-3,-5.957234E-4,-7.366556E-4,4.276809E-3,4.3726484E-3,1.0515895E-2,1.016652E-2,2.0102246E-2,7.4360017E-3,2.0983487E-2,-3.6297055E-4,-9.821369E-3,-9.5991464E-4,4.822608E-3,3.089039E-3,-2.7813849E-3,1.4043145E-2,7.937225E-3,-7.419747E-3,1.5590333E-2,1.8420424E-2,2.8598126E-2,2.6281897E-2,4.210028E-2],"split_indices":[2,37,16,38,2,0,61,47,62,37,63,7,67,40,46,2,61,4,37,1,62,37,7,2,37,1,0,62,0,61,10,9,38,67,9,2,0,4,0,38,0,47,54,0,0,0,0,54,24,0,0,0,0,0,0,0,7,56,62,0,109,2,7,8,7,2,37,37,6,4,12,10,7,10,0,7,66,0,4,0,65,38,37,0,0,0,0,0,0,65,0,65,65,12,42,47,0,7,0,51,0,7,0,4,37,0,0,0,0,0,65,0,65,62,61,0,0,0,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.93E2,7.49E2,2.44E2,2.26E2,5.23E2,9E0,2.35E2,7.2E1,1.54E2,2.68E2,2.55E2,6.6E1,1.69E2,4.3E1,2.9E1,3.6E1,1.18E2,2.08E2,6E1,1.29E2,1.26E2,5E1,1.6E1,3.7E1,1.32E2,2.2E1,2.1E1,2.2E1,7E0,1.4E1,2.2E1,1.02E2,1.6E1,3.6E1,1.72E2,5.4E1,6E0,1.22E2,7E0,1.18E2,8E0,2.5E1,2.5E1,6E0,1E1,2.4E1,1.3E1,8.7E1,4.5E1,1.5E1,7E0,1.7E1,5E0,6E0,8E0,6E0,1.6E1,9E1,1.2E1,5E0,1.1E1,1.4E1,2.2E1,1.28E2,4.4E1,3.8E1,1.6E1,7.7E1,4.5E1,1.02E2,1.6E1,1E1,1.5E1,2E1,5E0,7.3E1,1.4E1,5E0,4E1,5E0,1.1E1,8E1,1E1,5E0,7E0,5E0,6E0,7E0,7E0,1.6E1,6E0,6.7E1,6.1E1,2.7E1,1.7E1,3.2E1,6E0,1.1E1,5E0,6.5E1,1.2E1,4E1,5E0,1.2E1,9E1,9E0,7E0,5E0,5E0,5E0,1E1,6E0,1.4E1,2.4E1,4.9E1,5E0,9E0,5E0,3.5E1,6E0,5E0,6.7E1,1.3E1,5E0,5E0,1E1,6E0,5.2E1,1.5E1,5.2E1,9E0,2E1,7E0,9E0,8E0,1.3E1,1.9E1,6E0,5E0,4.6E1,1.9E1,5E0,7E0,8E0,3.2E1,5E0,7E0,2.2E1,6.8E1,5E0,5E0,6E0,8E0,5E0,1.9E1,2.3E1,2.6E1,9E0,2.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"159","size_leaf_vector":"1"}},{"base_weights":[-4.3546436E-3,-1.3082828E-1,4.298533E-1,-4.08138E-1,-1.1001722E-2,-2.6152316E-2,4.740627E-1,-5.799107E-1,-3.0856052E-1,-2.8204724E-1,3.4196343E-2,2.4286312E-1,5.840014E-1,-5.0913095E-1,-8.111514E-1,1.7238303E-1,-3.6368394E-1,-3.518019E-1,-7.928239E-2,2.3808913E-1,4.912449E-3,4.9008548E-1,1.14192285E-1,4.180792E-1,6.9620436E-1,-6.1934906E-1,-3.3349326E-1,-4.4172928E-2,-2.6192851E-2,-1.8011814E-2,3.0230004E-2,-3.1305107E-1,-5.944315E-1,-2.8231904E-1,-2.2580903E-2,-1.497328E-1,1.451766E-3,1.8612254E-1,2.0564586E-2,-8.705455E-2,5.020166E-2,4.0557005E-2,3.135255E-1,2.9272902E-1,6.842166E-3,5.7805765E-1,2.9092592E-1,7.34505E-1,3.2245487E-3,-1.857247E-2,-3.0698279E-2,-1.7161656E-2,-7.958919E-3,-3.826822E-1,-1.8313669E-1,-3.7535843E-2,-3.9872852E-1,-3.289884E-1,-1.4600815E-1,-1.7193307E-3,-1.0000209E-2,1.1909264E-1,1.9901952E-2,-3.1610742E-2,-2.1655722E-1,6.0284253E-2,-1.5780834E-2,8.099517E-3,2.0095328E-2,-2.1412084E-3,4.660128E-1,5.9833877E-2,-1.0390169E-2,3.0230397E-2,1.7023625E-2,1.902934E-1,2.2101574E-2,5.566322E-1,8.901246E-1,-2.0158392E-1,-4.3931526E-1,5.0405012E-3,-2.571606E-1,-1.0914988E-2,-2.4527261E-2,-3.6739904E-1,-7.228272E-3,-9.815951E-3,-2.0974388E-3,1.9244972E-1,-1.1108651E-2,5.9949044E-2,-9.928966E-2,-2.8979018E-1,2.8560704E-3,1.0557425E-2,1.8678114E-1,2.5909133E-2,1.3018112E-2,1.1793643E-2,1.4581453E-3,1.4079744E-2,6.2502176E-2,1.2481746E-2,6.201322E-1,5.2186202E-2,6.576297E-1,-4.4831517E-3,-1.4266604E-2,-2.1803891E-2,-7.7353306E-3,-1.8948937E-2,-7.5623645E-3,-1.8923106E-2,-8.577885E-3,2.7172351E-3,1.0748324E-2,4.2050756E-3,-6.5110293E-3,5.611103E-3,-3.0760919E-3,-9.109598E-3,-1.3291886E-3,-1.0426743E-2,-2.3785695E-2,3.1549267E-3,-2.9119735E-3,-1.0921754E-4,1.2683201E-2,-7.445114E-4,1.1872862E-2,-2.4640698E-3,3.7170784E-3,-1.6897542E-3,7.075587E-3,1.6657915E-2,3.092904E-2,1.9048406E-2,3.4339137E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":12,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,-1,53,55,57,-1,59,-1,61,-1,63,65,-1,67,69,71,73,75,77,-1,-1,-1,-1,-1,79,81,-1,83,85,87,-1,-1,89,-1,91,93,95,-1,-1,-1,-1,97,99,-1,-1,-1,101,-1,103,105,107,109,-1,111,-1,-1,113,-1,-1,-1,115,117,119,121,123,125,127,129,-1,-1,-1,131,-1,133,-1,135,-1,137,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.4532764E1,2.5548664E1,1.0105621E1,3.7893524E0,6.60054E0,0E0,5.3229904E0,9.6185684E-1,4.012869E0,1.0611796E0,2.7573285E0,2.2025676E0,2.4083939E0,1.0904675E0,8.795452E-2,4.550352E0,1.3794899E0,3.96729E-1,1.6826989E-1,5.357499E-1,1.6912409E0,1.3531256E0,9.179622E-1,1.0726757E0,1.9892387E0,1.0550213E-1,1.0151434E-1,0E0,0E0,0E0,0E0,9.4307804E-1,6.4312553E-1,2.0642686E-1,0E0,8.838618E-2,0E0,7.3012507E-1,0E0,9.552907E-1,1.0909156E0,0E0,2.2348452E-1,1.0784973E0,3.9249614E-1,1.6431999E-1,5.810449E-1,1.7189636E0,0E0,0E0,0E0,0E0,0E0,6.498842E-1,9.134512E-1,0E0,1.7293096E-1,1.3569903E-1,6.9723755E-2,0E0,0E0,3.7463415E-1,0E0,5.946357E-1,6.43559E-1,1.6712451E0,0E0,0E0,0E0,0E0,4.8913956E-2,2.9459673E-1,0E0,0E0,0E0,3.4253377E-1,0E0,5.637493E-1,1.4965553E0,1.8146765E-1,3.7214184E-1,0E0,4.034264E-1,0E0,0E0,9.434295E-2,0E0,0E0,0E0,1.1763829E-1,2.0990866E-1,3.236599E-1,3.7141305E-1,3.198278E-1,5.128222E-2,6.520933E-1,1.0552983E0,0E0,0E0,0E0,9.437726E-2,0E0,1.2127785E-1,0E0,1.4882278E-1,0E0,1.6474628E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,35,35,37,37,39,39,40,40,42,42,43,43,44,44,45,45,46,46,47,47,53,53,54,54,56,56,57,57,58,58,61,61,63,63,64,64,65,65,70,70,71,71,75,75,77,77,78,78,79,79,80,80,82,82,85,85,89,89,90,90,91,91,92,92,93,93,94,94,95,95,96,96,100,100,102,102,104,104,106,106],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,-1,54,56,58,-1,60,-1,62,-1,64,66,-1,68,70,72,74,76,78,-1,-1,-1,-1,-1,80,82,-1,84,86,88,-1,-1,90,-1,92,94,96,-1,-1,-1,-1,98,100,-1,-1,-1,102,-1,104,106,108,110,-1,112,-1,-1,114,-1,-1,-1,116,118,120,122,124,126,128,130,-1,-1,-1,132,-1,134,-1,136,-1,138,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.7909248E5,1E0,1.324E4,4.48E2,-2.6152316E-2,7.175182E5,5.61E2,6.666982E4,9.860918E5,7E0,7.6937294E-1,5.97E3,3.23E2,7.87E3,1.3E2,1E0,4.060171E6,2.1931148E6,2.8096385E0,5.9565744E5,1E0,1.8E1,1.1621469E1,6.292039E0,3.8E1,1.7559428E2,-4.4172928E-2,-2.6192851E-2,-1.8011814E-2,3.0230004E-2,8.57E2,1E0,4.3827028E5,-2.2580903E-2,7.0407925E0,1.451766E-3,5.235714E2,2.0564586E-2,5.812E3,1.4676277E7,4.0557005E-2,6.624E3,8.1636E2,4.7621E4,5.1291872E8,3.1993368E6,1.4822142E3,3.2245487E-3,-1.857247E-2,-3.0698279E-2,-1.7161656E-2,-7.958919E-3,2E0,1.392E1,-3.7535843E-2,1.24E2,2.707673E6,1.05E2,-1.7193307E-3,-1.0000209E-2,1.8591549E0,1.9901952E-2,1E0,1.862E3,4.2708197E2,-1.5780834E-2,8.099517E-3,2.0095328E-2,-2.1412084E-3,2.1298597E0,5.2846E4,-1.0390169E-2,3.0230397E-2,1.7023625E-2,5.044E3,2.2101574E-2,1.5484655E6,5.4544777E1,1E0,7.997723E6,5.0405012E-3,5.6321215E4,-1.0914988E-2,-2.4527261E-2,6.613774E5,-7.228272E-3,-9.815951E-3,-2.0974388E-3,1.852364E6,3.7439062E2,9.705882E0,8.83425E7,2.1E1,6.5969476E2,2.7943015E6,1E0,2.5909133E-2,1.3018112E-2,1.1793643E-2,3.0990322E5,1.4079744E-2,1.3E1,1.2481746E-2,1E0,5.2186202E-2,1.5822886E0,-4.4831517E-3,-1.4266604E-2,-2.1803891E-2,-7.7353306E-3,-1.8948937E-2,-7.5623645E-3,-1.8923106E-2,-8.577885E-3,2.7172351E-3,1.0748324E-2,4.2050756E-3,-6.5110293E-3,5.611103E-3,-3.0760919E-3,-9.109598E-3,-1.3291886E-3,-1.0426743E-2,-2.3785695E-2,3.1549267E-3,-2.9119735E-3,-1.0921754E-4,1.2683201E-2,-7.445114E-4,1.1872862E-2,-2.4640698E-3,3.7170784E-3,-1.6897542E-3,7.075587E-3,1.6657915E-2,3.092904E-2,1.9048406E-2,3.4339137E-2],"split_indices":[2,37,16,9,2,0,37,2,54,37,3,47,2,1,9,2,6,5,37,62,37,24,8,65,47,38,4,0,0,0,0,2,24,37,0,67,0,4,0,38,56,0,2,4,2,7,37,61,0,0,0,0,0,26,67,0,0,5,38,0,0,62,0,23,2,61,0,0,0,0,47,9,0,0,0,10,0,37,67,23,9,0,37,0,0,54,0,0,0,9,4,67,7,3,4,60,109,0,0,0,37,0,3,0,109,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.91E2,7.68E2,2.23E2,2.31E2,5.37E2,9E0,2.14E2,8.3E1,1.48E2,7.6E1,4.61E2,7E1,1.44E2,6.6E1,1.7E1,1.5E1,1.33E2,5.6E1,2E1,5.7E1,4.04E2,2.3E1,4.7E1,6E1,8.4E1,3.9E1,2.7E1,9E0,8E0,7E0,8E0,1.11E2,2.2E1,3.9E1,1.7E1,1.2E1,8E0,4.7E1,1E1,1.33E2,2.71E2,6E0,1.7E1,1.7E1,3E1,2.5E1,3.5E1,7.9E1,5E0,8E0,3.1E1,2.1E1,6E0,7.1E1,4E1,9E0,1.3E1,2.8E1,1.1E1,5E0,7E0,3.8E1,9E0,9.4E1,3.9E1,2.65E2,6E0,9E0,8E0,6E0,1.1E1,2.5E1,5E0,1.7E1,8E0,2.4E1,1.1E1,3.9E1,4E1,1.8E1,5.3E1,8E0,3.2E1,7E0,6E0,2.2E1,6E0,6E0,5E0,2.4E1,1.4E1,4E1,5.4E1,2.9E1,1E1,1.91E2,7.4E1,6E0,5E0,5E0,2E1,1.2E1,1.2E1,8E0,3.1E1,1.8E1,2.2E1,1E1,8E0,4.7E1,6E0,1.1E1,2.1E1,1.7E1,5E0,6E0,1.8E1,8E0,6E0,2.7E1,1.3E1,2.2E1,3.2E1,2.4E1,5E0,5E0,5E0,1.83E2,8E0,1.9E1,5.5E1,1.2E1,8E0,6E0,6E0,6E0,2.5E1,7E0,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"139","size_leaf_vector":"1"}},{"base_weights":[2.3404374E-3,-1.2207888E-1,4.1612262E-1,-3.9067906E-1,-2.0507541E-3,-2.4381695E-2,4.585747E-1,-5.9434223E-1,-3.0168048E-1,-2.7341485E-1,4.7693595E-2,5.466406E-2,4.2458215E-1,-4.7306648E-1,-7.3218924E-1,-3.375239E-1,2.45904E-2,-1.571988E-1,-3.772932E-1,-6.4679496E-2,1.1040167E-1,2.4264725E-1,5.8405536E-1,-5.218955E-1,-1.1707248E-2,-3.7197545E-2,-2.3804586E-2,-2.8529975E-1,-6.085172E-1,-2.033277E-1,8.888331E-4,-2.4119087E-1,-4.7332147E-1,-3.657156E-2,-2.6753498E-2,6.351169E-2,2.9026628E-1,1.9457097E-1,3.4946837E-2,5.1386523E-1,7.965321E-1,-2.5799554E-2,-1.3316081E-2,-4.4541076E-1,-2.0473714E-1,-6.7693084E-1,-1.3476927E-2,-2.423234E-1,-2.826865E-3,-1.4134658E-2,-2.4576548E-3,-2.6042728E-2,-1.5142428E-2,-7.857279E-2,1.7321739E-1,1.5222648E-1,3.6686251E-3,2.0864104E-1,4.5552984E-1,2.2369108E-1,-1.1618049E-2,3.305503E-1,5.73581E-1,2.6169501E-2,4.4904757E-2,-2.887792E-2,-3.8717917E-1,2.190617E-2,-2.6827535E-1,-1.8452678E-2,-4.1224778E-2,-2.7670473E-1,-6.448249E-3,-2.4551487E-1,-1.7837329E-2,6.967625E-2,2.3759972E-1,7.592048E-2,3.4453303E-1,2.909727E-2,-1.9830987E-1,1.3476509E-1,1.9338917E-2,1.3684143E-2,2.3722053E-2,-5.0619803E-2,2.831951E-1,3.9612207E-1,3.6536416E-3,1.4598019E-2,6.173315E-1,-1.0093129E-2,-1.9245481E-2,-1.7548291E-2,-8.074656E-3,-7.6823193E-3,-1.4347331E-2,-8.506528E-3,-2.295746E-2,2.3896303E-3,-5.0698356E-3,7.264173E-4,5.2006333E-3,1.6135275E-2,5.824484E-3,5.010631E-3,-1.1944106E-2,1.0194292E-2,1.9167729E-2,7.7737407E-3,-1.057233E-4,-1.4026632E-2,3.7406993E-4,8.1345225E-3,-1.3599213E-3,4.400342E-3,-5.8165193E-3,1.10651925E-2,2.5973778E-2,9.063714E-3,2.2282574E-2,1.341022E-2,2.9979086E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":13,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,27,-1,29,31,33,35,37,39,41,-1,-1,-1,43,45,47,-1,49,51,53,-1,55,57,59,-1,61,63,-1,-1,65,67,69,-1,71,-1,-1,-1,-1,-1,73,75,77,79,81,83,85,-1,87,89,-1,-1,-1,91,-1,93,-1,-1,95,-1,97,99,101,103,105,107,109,111,113,-1,-1,-1,115,117,119,-1,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.9216583E1,2.3695286E1,9.074959E0,3.935238E0,6.8839455E0,0E0,4.6851387E0,7.904167E-1,4.948366E0,8.9440393E-1,3.0442681E0,0E0,5.7770844E0,2.8870296E-1,4.5591354E-2,2.0058594E0,0E0,3.2116228E-1,4.216957E-1,2.2091756E0,2.317741E0,2.2342448E0,1.2013779E0,8.901119E-2,0E0,0E0,0E0,1.5985718E0,3.2561684E-1,1.5738285E-1,0E0,2.1073902E-1,1.1413622E-1,1.3176962E0,0E0,1.1727226E0,6.7386913E-1,1.2221742E0,0E0,7.299328E-1,4.4969845E-1,0E0,0E0,2.0214558E-1,3.954229E0,7.69022E-1,0E0,4.6825647E-2,0E0,0E0,0E0,0E0,0E0,1.2528944E0,1.5010047E-1,1.2798419E0,6.8886346E-1,5.589261E-1,1.0326862E-2,1.4017568E0,0E0,3.4857106E-1,5.082283E-1,0E0,0E0,0E0,7.365513E-2,0E0,7.8374577E-1,0E0,0E0,1.0222912E-2,0E0,4.20784E-1,5.8885926E-1,2.3445047E-2,1.415121E-1,7.2263265E-1,1.23220444E-1,5.1728725E-1,3.2435274E-1,2.1138841E-1,0E0,0E0,0E0,1.8582545E-1,7.323098E-1,2.1017599E-1,0E0,0E0,3.0456734E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,27,27,28,28,29,29,31,31,32,32,33,33,35,35,36,36,37,37,39,39,40,40,43,43,44,44,45,45,47,47,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,62,62,66,66,68,68,71,71,73,73,74,74,75,75,76,76,77,77,78,78,79,79,80,80,81,81,85,85,86,86,87,87,90,90],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,28,-1,30,32,34,36,38,40,42,-1,-1,-1,44,46,48,-1,50,52,54,-1,56,58,60,-1,62,64,-1,-1,66,68,70,-1,72,-1,-1,-1,-1,-1,74,76,78,80,82,84,86,-1,88,90,-1,-1,-1,92,-1,94,-1,-1,96,-1,98,100,102,104,106,108,110,112,114,-1,-1,-1,116,118,120,-1,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,1.972052E5,1E0,7.7E1,4.97E2,-2.4381695E-2,2.2968E4,1.139605E6,5.867288E2,3.875E0,5.036081E7,5.466406E-2,1.8062708E6,3.0088105E0,7.919006E7,4.909091E0,2.45904E-2,2.9652428E6,3.2E1,4.2E1,2.4180895E6,5.428706E3,6.663214E6,1.8388145E-7,-1.1707248E-2,-3.7197545E-2,-2.3804586E-2,2.71E2,6.340348E8,2.1478824E2,8.888331E-4,3E0,4.795239E7,4.5866325E6,-2.6753498E-2,3.3358974E0,4.9321495E6,3.0070068E7,3.4946837E-2,1.0089981E3,8E0,-2.5799554E-2,-1.3316081E-2,3.6E1,2.2E1,1.9225262E2,-1.3476927E-2,3.1706784E0,-2.826865E-3,-1.4134658E-2,-2.4576548E-3,-2.6042728E-2,-1.5142428E-2,1E0,1.2E1,4.6695483E2,1E0,2.483E3,3.662857E0,6.136927E2,-1.1618049E-2,2.0619047E1,1E0,2.6169501E-2,4.4904757E-2,-2.887792E-2,7.641E3,2.190617E-2,1.212945E6,-1.8452678E-2,-4.1224778E-2,1E0,-6.448249E-3,4.159558E0,1E0,1.7594937E0,2.1525E4,2.126E3,6.083812E2,4.5683857E-2,3.2525E2,3.9152692E6,1.9338917E-2,1.3684143E-2,2.3722053E-2,9.5139277E-1,1.7620745E0,4.797448E8,3.6536416E-3,1.4598019E-2,1E0,-1.0093129E-2,-1.9245481E-2,-1.7548291E-2,-8.074656E-3,-7.6823193E-3,-1.4347331E-2,-8.506528E-3,-2.295746E-2,2.3896303E-3,-5.0698356E-3,7.264173E-4,5.2006333E-3,1.6135275E-2,5.824484E-3,5.010631E-3,-1.1944106E-2,1.0194292E-2,1.9167729E-2,7.7737407E-3,-1.057233E-4,-1.4026632E-2,3.7406993E-4,8.1345225E-3,-1.3599213E-3,4.400342E-3,-5.8165193E-3,1.10651925E-2,2.5973778E-2,9.063714E-3,2.2282574E-2,1.341022E-2,2.9979086E-2],"split_indices":[2,37,16,38,2,0,12,40,61,63,7,0,37,62,7,62,0,37,10,3,37,61,37,46,0,0,0,2,40,4,0,8,7,37,0,63,37,52,0,4,3,0,0,2,0,4,0,63,0,0,0,0,0,109,3,64,85,2,67,61,0,65,24,0,0,0,9,0,9,0,0,109,0,63,23,65,9,2,4,47,61,37,0,0,0,47,44,7,0,0,14,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.54E2,7.34E2,2.2E2,2.26E2,5.08E2,9E0,2.11E2,6.7E1,1.59E2,7.8E1,4.3E2,8E0,2.03E2,3.8E1,2.9E1,1.53E2,6E0,3.8E1,4E1,1.54E2,2.76E2,9.6E1,1.07E2,3E1,8E0,2E1,9E0,1.3E2,2.3E1,3E1,8E0,1.8E1,2.2E1,1.47E2,7E0,2.2E2,5.6E1,8.9E1,7E0,8.3E1,2.4E1,2.5E1,5E0,4.2E1,8.8E1,1.8E1,5E0,2.3E1,7E0,1.3E1,5E0,1.2E1,1E1,1.23E2,2.4E1,8.8E1,1.32E2,3.9E1,1.7E1,8.4E1,5E0,2.2E1,6.1E1,1.2E1,1.2E1,8E0,3.4E1,7E0,8.1E1,9E0,9E0,1.6E1,7E0,3.2E1,9.1E1,1E1,1.4E1,6.4E1,2.4E1,1.18E2,1.4E1,3E1,9E0,6E0,1.1E1,1.5E1,6.9E1,1.7E1,5E0,1E1,5.1E1,6E0,2.8E1,3.6E1,4.5E1,5E0,1.1E1,2.7E1,5E0,5.2E1,3.9E1,5E0,5E0,6E0,8E0,5.9E1,5E0,1E1,1.4E1,2.1E1,9.7E1,9E0,5E0,2.4E1,6E0,5E0,1E1,6.1E1,8E0,6E0,1.1E1,5E0,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[5.459244E-4,-1.2286759E-1,3.9117414E-1,-3.7196285E-1,-1.774587E-2,4.573966E-1,-2.2471474E-1,-3.2703564E-1,-6.503887E-1,-1.242748E-1,8.6392954E-2,2.5178522E-1,5.86552E-1,-4.1560096E-1,3.0158418E-3,-3.758509E-1,7.90595E-2,-5.083718E-1,-4.7517776E-2,-1.9183305E-1,1.7222729E-2,4.3340176E-2,2.577706E-1,4.0536734E-1,1.5538463E-1,8.0545735E-1,4.6490455E-1,-2.4442198E-2,-1.0633331E-2,-5.2404016E-1,-3.190415E-1,2.7857639E-2,-1.2137987E-1,-9.588357E-3,-2.694429E-2,-1.5636314E-1,-4.1481796E-1,-1.3484618E-2,5.393428E-2,-1.1400776E-1,8.3595395E-2,1.1797353E-1,3.6457607E-1,4.7878805E-1,6.237669E-3,2.9276854E-1,8.029049E-2,5.9413767E-1,5.000414E-2,6.464015E-1,3.1531513E-1,-1.7179113E-2,-2.6389493E-2,-7.73862E-3,-3.585915E-1,-1.4554063E-2,-5.460246E-3,-1.8003134E-1,8.641591E-2,-9.3011735E-3,-5.383017E-1,9.449222E-4,1.9014764E-1,-4.928465E-2,-1.6380902E-2,9.899427E-2,-1.0896045E-2,9.333874E-3,4.4183165E-2,3.9262814E-3,4.1937461E-1,1.10845985E-2,2.5309963E-2,2.0682644E-2,1.5489225E-1,1.6893171E-1,-2.3502078E-2,2.9888008E-2,1.6668275E-2,3.7202872E-2,5.288211E-1,3.7029433E-1,1.6670402E-3,7.6855035E-3,-5.1499032E-3,-3.0669507E-1,-5.415607E-1,-4.5722444E-3,4.1077654E-3,-2.3105857E-1,-4.5884542E-2,-9.3390007E-4,7.834044E-3,-1.78613E-2,-3.1487435E-2,8.538743E-2,-4.9747583E-2,2.8469715E-1,7.831196E-2,-2.2201759E-1,2.8689228E-2,5.5008575E-2,2.5451437E-1,-1.6634518E-3,4.2979694E-3,1.1978618E-2,4.988906E-1,1.7034292E-3,1.1472568E-2,1.5224916E-2,8.863841E-2,3.8815245E-2,-6.6917087E-3,2.6820216E-2,1.5160526E-2,2.5149912E-1,4.7651458E-1,-1.6809719E-2,-1.003795E-2,-2.8381221E-2,-1.54808285E-2,-1.3085705E-2,-4.8459186E-3,-9.167188E-3,2.8800935E-4,-4.1343024E-4,5.355064E-3,-5.436796E-3,1.3241235E-3,7.979251E-3,1.6238676E-2,3.4374674E-4,6.3179047E-3,-1.5669944E-2,-3.2161102E-3,-2.6760073E-3,8.752355E-3,4.4010826E-3,-7.4658906E-3,1.0435912E-3,1.54957725E-2,1.4231991E-2,2.6183145E-2,1.2672548E-3,6.0884985E-3,-3.018551E-3,5.6747654E-3,5.6370175E-3,1.3706565E-2,2.643515E-2,1.0301104E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":14,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,37,39,41,43,45,47,49,-1,-1,51,53,-1,55,-1,-1,57,59,-1,61,63,65,67,69,71,-1,73,75,77,-1,79,81,-1,-1,83,85,-1,87,89,91,-1,93,95,97,99,-1,101,-1,-1,103,-1,105,-1,-1,-1,107,109,111,-1,-1,-1,113,115,-1,-1,-1,117,119,-1,-1,121,123,-1,-1,-1,-1,125,127,129,131,133,135,137,139,-1,-1,-1,141,-1,-1,-1,143,145,-1,-1,-1,147,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6183884E1,1.9056314E1,9.48349E0,2.4892178E0,5.701854E0,5.375923E0,1.2989602E0,3.7565575E0,1.0185146E0,2.431373E0,1.9071817E0,1.1510558E0,3.0074577E0,1.6945553E-1,0E0,1.2471237E0,2.2672644E0,3.8195372E-1,0E0,1.2955289E0,9.536909E-1,1.3333521E0,7.334063E-1,5.522895E-1,5.129632E-1,1.8909359E0,2.0767822E0,0E0,0E0,9.8828316E-2,1.5239182E0,0E0,3.5492897E-1,0E0,0E0,8.774793E-1,5.008993E-1,0E0,5.411597E-1,6.478531E-1,8.421179E-1,1.3843909E-1,4.0940857E-1,2.5366306E-1,0E0,3.106277E-1,3.2598072E-1,8.577442E-2,0E0,2.520342E-1,7.239995E-1,0E0,0E0,2.8690678E-1,8.784046E-1,0E0,1.0495999E-1,9.271817E-1,1.2634163E-1,0E0,4.002142E-2,2.3968095E-1,1.9975895E-1,4.7777247E-1,0E0,1.080505E0,0E0,0E0,5.7604104E-2,0E0,1.7885828E-1,0E0,0E0,0E0,1.1099467E-1,2.1377993E-1,1.3348553E-1,0E0,0E0,0E0,4.9792767E-2,3.8489532E-1,0E0,0E0,0E0,3.636284E-1,1.3120031E-1,0E0,0E0,6.0940313E-1,3.093896E-1,0E0,0E0,0E0,0E0,6.048523E-2,1.8809034E-1,2.076155E-2,4.41119E-2,1.7093962E-1,3.5620293E-1,1.09921E0,6.376908E-1,0E0,0E0,0E0,1.5450001E-2,0E0,0E0,0E0,3.2974914E-2,1.1201947E-1,0E0,0E0,0E0,8.398104E-2,3.479929E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,32,32,35,35,36,36,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,47,47,49,49,50,50,53,53,54,54,56,56,57,57,58,58,60,60,61,61,62,62,63,63,65,65,68,68,70,70,74,74,75,75,76,76,80,80,81,81,85,85,86,86,89,89,90,90,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,106,106,110,110,111,111,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,38,40,42,44,46,48,50,-1,-1,52,54,-1,56,-1,-1,58,60,-1,62,64,66,68,70,72,-1,74,76,78,-1,80,82,-1,-1,84,86,-1,88,90,92,-1,94,96,98,100,-1,102,-1,-1,104,-1,106,-1,-1,-1,108,110,112,-1,-1,-1,114,116,-1,-1,-1,118,120,-1,-1,122,124,-1,-1,-1,-1,126,128,130,132,134,136,138,140,-1,-1,-1,142,-1,-1,-1,144,146,-1,-1,-1,148,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.7909248E5,3.5056704E7,1E0,9.24E2,9.939072E2,1.3655363E2,3.2525E2,4.4561405E0,1.5859042E6,2.8530578E6,8E0,4.4770187E1,1.6E1,3.0158418E-3,3.89E2,8.51124E6,1.764442E7,-4.7517776E-2,2E0,1.45064E5,4.7563504E7,5.4007E7,1.1590965E0,4.4971E4,1.9617874E3,2.0317461E0,-2.4442198E-2,-1.0633331E-2,5.3E1,1.3520408E0,2.7857639E-2,2.11429E5,-9.588357E-3,-2.694429E-2,3.5947333E2,1.3789044E6,-1.3484618E-2,1.65281E8,4.6842103E0,1.0767881E7,3.531793E3,5.320943E9,4.6134964E2,6.237669E-3,1.4047974E8,1.138871E0,2.1298597E0,5.000414E-2,1.6063418E7,1.9513452E2,-1.7179113E-2,-2.6389493E-2,8.0509944E5,8.172E3,-1.4554063E-2,3.1158154E4,2.118835E6,1.5E1,-9.3011735E-3,5.4E1,1.8333334E0,3.52E2,4.0119403E2,-1.6380902E-2,5.747068E2,-1.0896045E-2,9.333874E-3,1.2915888E1,3.9262814E-3,1.20086E5,1.10845985E-2,2.5309963E-2,2.0682644E-2,4.11566E5,1.1621469E1,1.6E1,2.9888008E-2,1.6668275E-2,3.7202872E-2,1.8E1,1.7831801E6,1.6670402E-3,7.6855035E-3,-5.1499032E-3,1.8224286E2,1.0433326E8,-4.5722444E-3,4.1077654E-3,8.747289E5,4.26E2,-9.3390007E-4,7.834044E-3,-1.78613E-2,-3.1487435E-2,4.67E2,2.8266037E2,2.087E2,4.49239E6,2.3584905E0,4.7659574E0,1.597E4,6.285983E5,-1.6634518E-3,4.2979694E-3,1.1978618E-2,4.2782607E0,1.7034292E-3,1.1472568E-2,1.5224916E-2,9.195571E2,2.8846003E5,-6.6917087E-3,2.6820216E-2,1.5160526E-2,1.75531E3,3.7035952E6,-1.6809719E-2,-1.003795E-2,-2.8381221E-2,-1.54808285E-2,-1.3085705E-2,-4.8459186E-3,-9.167188E-3,2.8800935E-4,-4.1343024E-4,5.355064E-3,-5.436796E-3,1.3241235E-3,7.979251E-3,1.6238676E-2,3.4374674E-4,6.3179047E-3,-1.5669944E-2,-3.2161102E-3,-2.6760073E-3,8.752355E-3,4.4010826E-3,-7.4658906E-3,1.0435912E-3,1.54957725E-2,1.4231991E-2,2.6183145E-2,1.2672548E-3,6.0884985E-3,-3.018551E-3,5.6747654E-3,5.6370175E-3,1.3706565E-2,2.643515E-2,1.0301104E-2],"split_indices":[2,37,54,6,2,61,65,61,62,37,37,3,67,8,0,1,5,54,0,6,7,7,7,47,9,4,63,0,0,38,62,0,9,0,0,61,54,0,5,62,56,42,40,4,0,5,47,47,0,56,67,0,0,54,38,0,42,9,3,0,0,62,0,4,0,61,0,0,67,0,9,0,0,0,1,65,3,0,0,0,3,37,0,0,0,4,7,0,0,37,2,0,0,0,0,2,4,4,37,62,63,38,37,0,0,0,65,0,0,0,4,37,0,0,0,64,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.56E2,7.27E2,2.29E2,2.15E2,5.12E2,2.07E2,2.2E1,1.87E2,2.8E1,2.53E2,2.59E2,8.1E1,1.26E2,1.3E1,9E0,1.67E2,2E1,2.2E1,6E0,1.71E2,8.2E1,2.08E2,5.1E1,3E1,5.1E1,4.3E1,8.3E1,7E0,6E0,4.4E1,1.23E2,5E0,1.5E1,5E0,1.7E1,1.49E2,2.2E1,8E0,7.4E1,4.2E1,1.66E2,2.3E1,2.8E1,2.3E1,7E0,1.7E1,3.4E1,2.6E1,1.7E1,3.6E1,4.7E1,1.2E1,3.2E1,1.4E1,1.09E2,5E0,1E1,1.36E2,1.3E1,9E0,1.3E1,5.4E1,2E1,3.4E1,8E0,1.59E2,7E0,1E1,1.3E1,5E0,2.3E1,6E0,1.7E1,7E0,1E1,1.8E1,1.6E1,2E1,6E0,1.3E1,2.3E1,3.9E1,8E0,5E0,9E0,8.7E1,2.2E1,5E0,5E0,9.8E1,3.8E1,6E0,7E0,8E0,5E0,2E1,3.4E1,1E1,1E1,1E1,2.4E1,1.25E2,3.4E1,5E0,8E0,9E0,1.4E1,5E0,5E0,5E0,1.3E1,1.1E1,5E0,1.7E1,6E0,2E1,1.9E1,5.2E1,3.5E1,1.5E1,7E0,6.9E1,2.9E1,9E0,2.9E1,5E0,1.5E1,1.8E1,1.6E1,5E0,5E0,5E0,5E0,5E0,5E0,1.6E1,8E0,1.06E2,1.9E1,9E0,2.5E1,5E0,9E0,6E0,7E0,5E0,6E0,6E0,1.4E1,1.3E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-9.781885E-3,-2.1222027E-1,2.081143E-1,-4.4514996E-1,-1.19762875E-1,-5.65662E-1,2.5210577E-1,-4.1827375E-1,-4.1575924E-2,-1.7423978E-1,2.0895626E-2,-1.0614854E-2,-6.384461E-1,1.3775875E-1,5.0518984E-1,-5.317261E-1,-3.3747494E-1,-2.7281922E-1,-1.0824402E-1,-4.9117062E-2,1.04152195E-1,-3.6776293E-2,-2.179031E-2,3.9923954E-1,8.902403E-2,4.0044475E-2,4.3913665E-1,-2.5586145E-2,-1.487728E-2,1.341074E-3,-3.6195907E-1,-1.8197493E-1,-3.5666996E-1,-4.4023342E-2,-1.8764916E-1,5.1993582E-2,-1.4156087E-1,1.8545325E-1,7.0304517E-3,2.3635508E-1,5.0132704E-1,-4.1513454E-2,1.4262229E-1,3.6016282E-1,7.2686297E-1,-3.243458E-1,-2.6808271E-2,-2.6908255E-1,-6.423165E-2,-5.9170937E-3,-3.9543435E-1,2.558188E-2,-1.1584222E-1,2.6033698E-3,-2.0756936E-1,1.3159777E-1,-4.477232E-3,-2.1271108E-2,-5.644398E-2,1.1766549E-1,1.591292E-2,3.3674468E-3,-4.39294E-2,1.4401938E-1,1.6557915E-2,2.6150525E-2,1.5445572E-2,-2.6921606E-1,1.8294309E-3,1.6086005E-1,-1.5483295E-2,3.902632E-1,-6.5579447E-3,3.654029E-2,1.9193118E-2,-3.456616E-1,-2.3211273E-3,-1.772635E-2,-2.2300546E-1,9.93035E-4,-1.1448087E-1,-9.849817E-3,-4.364651E-1,-3.365774E-2,1.1631119E-1,-2.050039E-1,-3.056088E-3,-1.4017503E-1,-2.7478996E-1,5.6925684E-2,1.1173319E-2,-1.1725251E-1,2.6879925E-3,6.398098E-3,2.3598657E-3,7.571541E-5,-4.5172665E-3,1.0161464E-2,9.299434E-4,-5.727327E-3,-1.8301047E-2,-5.4955058E-2,1.4999841E-1,2.7018562E-1,9.2184134E-2,-5.798371E-4,4.1988584E-1,-7.454479E-3,-1.7355504E-2,-7.278167E-3,-1.2554196E-2,2.7551148E-3,-2.6706418E-3,-7.5023677E-3,-1.505838E-3,-2.1459054E-2,-1.2356807E-2,2.9822355E-3,-4.251787E-3,9.670839E-4,9.719002E-3,-1.4689564E-2,-3.1104232E-3,1.5360442E-3,-4.3127793E-3,1.191767E-3,-8.917815E-3,-1.5003298E-2,-8.942385E-3,4.8764255E-3,3.481801E-4,-6.825751E-3,-1.9939793E-3,2.5485174E-4,-7.514403E-3,1.5769234E-2,2.3585886E-3,1.07326945E-2,2.1744596E-2,-2.2219985E-3,7.3813032E-3,2.598124E-2,1.6508667E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":15,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,-1,21,23,25,27,29,31,33,35,37,-1,-1,39,41,-1,43,-1,-1,-1,45,47,49,51,53,55,57,59,61,63,65,67,69,71,73,75,-1,77,79,-1,81,83,85,-1,87,89,-1,-1,91,93,-1,-1,95,97,-1,-1,-1,99,101,103,-1,105,-1,-1,-1,107,-1,-1,109,111,113,-1,115,117,119,121,123,125,127,129,-1,131,-1,-1,-1,-1,-1,-1,-1,-1,-1,133,135,137,139,-1,141,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.203741E1,1.0585506E1,1.5766764E1,1.3217735E0,2.723878E0,4.66949E-1,1.2519047E1,1.0489101E0,0E0,1.6359601E0,5.880173E-1,0E0,1.4827251E-1,3.80016E0,2.812008E0,4.7823906E-2,7.3706627E-1,7.120657E-1,7.821542E-1,5.214482E-1,3.619168E-1,0E0,0E0,6.6304636E-1,1.7900496E0,0E0,2.3740616E0,0E0,0E0,0E0,4.2952633E-1,5.0578976E-1,4.114704E-1,4.3787378E-1,3.466885E-1,3.3184314E-1,7.562928E-1,2.2656375E-1,7.6671526E-2,1.7113698E-1,1.0607672E-1,7.406891E-1,1.6260202E0,1.4590263E0,1.3018322E-1,3.7448215E-1,0E0,6.573868E-2,7.5495936E-2,0E0,2.3799849E-1,2.4577942E-1,4.317901E-1,0E0,2.5141978E-1,1.3308024E-1,0E0,0E0,1.7371291E-1,1.8660247E-2,0E0,0E0,3.18033E-2,1.16457134E-1,0E0,0E0,0E0,1.7034924E-1,5.4671997E-1,1.2889986E0,0E0,1.0551567E0,0E0,0E0,0E0,2.6547813E-1,0E0,0E0,2.9919267E-2,4.1021116E-2,4.5095146E-2,0E0,3.1793118E-2,1.6486572E-1,1.5571117E-1,3.4826314E-1,6.808928E-2,2.982996E-1,6.122017E-2,2.7779695E-2,0E0,2.9160663E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.0486745E-1,3.216227E-1,4.0849113E-1,1.0290073E0,0E0,5.368633E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,26,26,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,50,50,51,51,52,52,54,54,55,55,58,58,59,59,62,62,63,63,67,67,68,68,69,69,71,71,75,75,78,78,79,79,80,80,82,82,83,83,84,84,85,85,86,86,87,87,88,88,89,89,91,91,101,101,102,102,103,103,104,104,106,106],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,-1,22,24,26,28,30,32,34,36,38,-1,-1,40,42,-1,44,-1,-1,-1,46,48,50,52,54,56,58,60,62,64,66,68,70,72,74,76,-1,78,80,-1,82,84,86,-1,88,90,-1,-1,92,94,-1,-1,96,98,-1,-1,-1,100,102,104,-1,106,-1,-1,-1,108,-1,-1,110,112,114,-1,116,118,120,122,124,126,128,130,-1,132,-1,-1,-1,-1,-1,-1,-1,-1,-1,134,136,138,140,-1,142,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,8.147158E4,1E0,6.958929E1,1.6802971E6,1.4E1,1.1464103E3,3.89E2,-4.1575924E-2,4.97E2,2.732381E2,-1.0614854E-2,5E0,5E0,1.307E3,1.4473684E-1,1.1649485E0,3.875E0,2.6818182E0,5.457143E0,9.244374E6,-3.6776293E-2,-2.179031E-2,4.9969565E2,4.204547E5,4.0044475E-2,5.1167645E6,-2.5586145E-2,-1.487728E-2,1.341074E-3,2.9E1,1.7131818E2,4.3922543E2,1E0,2.6363637E0,8E0,7.478261E0,1.10566504E8,1.4E1,1E0,1.0236775E1,5.57767E3,3.1083834E5,2.871605E2,2.165756E0,4.1718604E2,-2.6808271E-2,1.0231935E3,1.3E1,-5.9170937E-3,7.026578E7,1.72959E5,2.118835E6,2.6033698E-3,3.3057997E0,4.5866325E6,-4.477232E-3,-2.1271108E-2,4.1973075E6,2.1506848E0,1.591292E-2,3.3674468E-3,3.7253174E2,4.6066E4,1.6557915E-2,2.6150525E-2,1.5445572E-2,3.480315E0,5.933284E2,3.1463416E0,-1.5483295E-2,1.162E3,-6.5579447E-3,3.654029E-2,1.9193118E-2,6.101E3,-2.3211273E-3,-1.772635E-2,3.2E1,1.2501388E6,4.330197E5,-9.849817E-3,3.328629E6,1.4786703E2,3.6648486E2,3.4289816E2,1.2999985E6,5.9653606E5,1.1948844E4,5.748115E-5,1.1173319E-2,6.4E1,2.6879925E-3,6.398098E-3,2.3598657E-3,7.571541E-5,-4.5172665E-3,1.0161464E-2,9.299434E-4,-5.727327E-3,-1.8301047E-2,2.9654377E1,1.5137369E6,1.0089981E3,6.848509E7,-5.798371E-4,5.4544777E1,-7.454479E-3,-1.7355504E-2,-7.278167E-3,-1.2554196E-2,2.7551148E-3,-2.6706418E-3,-7.5023677E-3,-1.505838E-3,-2.1459054E-2,-1.2356807E-2,2.9822355E-3,-4.251787E-3,9.670839E-4,9.719002E-3,-1.4689564E-2,-3.1104232E-3,1.5360442E-3,-4.3127793E-3,1.191767E-3,-8.917815E-3,-1.5003298E-2,-8.942385E-3,4.8764255E-3,3.481801E-4,-6.825751E-3,-1.9939793E-3,2.5485174E-4,-7.514403E-3,1.5769234E-2,2.3585886E-3,1.07326945E-2,2.1744596E-2,-2.2219985E-3,7.3813032E-3,2.598124E-2,1.6508667E-2],"split_indices":[2,37,16,55,37,3,61,1,0,2,4,0,6,3,38,65,62,63,62,67,54,0,0,4,37,0,37,0,0,0,3,4,42,23,67,8,67,5,3,92,65,42,42,67,62,61,0,42,3,0,40,12,9,0,62,37,0,0,37,62,0,0,4,1,0,0,0,62,61,63,0,10,0,0,0,9,0,0,10,54,37,0,9,61,4,4,54,54,42,47,0,0,0,0,0,0,0,0,0,0,0,67,56,4,7,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.51E2,4.93E2,4.58E2,1.39E2,3.54E2,2.4E1,4.34E2,1.33E2,6E0,2.55E2,9.9E1,5E0,1.9E1,3E2,1.34E2,5.3E1,8E1,1.01E2,1.54E2,5.4E1,4.5E1,8E0,1.1E1,4.6E1,2.54E2,1.9E1,1.15E2,4.7E1,6E0,5E0,7.5E1,5E1,5.1E1,8.6E1,6.8E1,2.6E1,2.8E1,2.4E1,2.1E1,1.9E1,2.7E1,7.4E1,1.8E2,9.2E1,2.3E1,6.6E1,9E0,2.8E1,2.2E1,8E0,4.3E1,4.4E1,4.2E1,5E0,6.3E1,1.7E1,9E0,5E0,2.3E1,1.8E1,6E0,9E0,1.2E1,1.2E1,7E0,1.8E1,9E0,1.1E1,6.3E1,1.74E2,6E0,8.7E1,5E0,1.8E1,5E0,6.1E1,5E0,6E0,2.2E1,1E1,1.2E1,9E0,3.4E1,2.7E1,1.7E1,2.3E1,1.9E1,3.3E1,3E1,1.1E1,6E0,1.5E1,8E0,1.3E1,5E0,7E0,5E0,7E0,5E0,6E0,5E0,4.6E1,1.7E1,6.6E1,1.08E2,6E0,8.1E1,9E0,5.2E1,1.1E1,1.1E1,5E0,5E0,7E0,5E0,2.8E1,6E0,1E1,1.7E1,9E0,8E0,1.2E1,1.1E1,1.4E1,5E0,8E0,2.5E1,1.7E1,1.3E1,5E0,6E0,1E1,5E0,3E1,1.6E1,5E0,1.2E1,5.7E1,9E0,3.5E1,7.3E1,2.3E1,5.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"143","size_leaf_vector":"1"}},{"base_weights":[7.935553E-3,-1.1239282E-1,3.5303268E-1,-3.4619188E-1,-1.2812076E-2,-4.3568704E-1,4.000727E-1,-5.5806595E-1,-2.61927E-1,-1.1134844E-1,8.539584E-2,-2.498843E-2,-8.891666E-3,1.8157272E-1,4.9783716E-1,-7.9946285E-1,-4.240802E-1,-4.044548E-1,-1.8550289E-1,-5.6047715E-2,-2.6361498E-1,1.620253E-1,-2.2950044E-2,6.238656E-2,3.6621815E-1,5.9625734E-2,4.6580562E-1,-1.822205E-2,-9.359779E-1,-4.8384356E-1,-7.0150075E-3,-4.2979205E-1,-7.856098E-3,2.6124738E-2,-2.2813414E-1,-2.4581797E-1,-2.7474476E-2,-4.317602E-1,-1.6434287E-1,1.7899148E-1,-1.0087745E-2,5.386223E-4,-1.6949257E-2,-7.0907E-3,1.15967765E-1,4.0147147E-1,7.344695E-3,5.3364116E-1,2.1167353E-1,-5.7051998E-2,-3.0912023E-2,-2.436236E-2,-1.15936715E-2,-9.144405E-3,-4.517575E-1,-1.8002003E-1,-2.2827908E-2,-1.5887633E-2,-2.0454939E-1,-6.0968634E-2,6.543108E-2,-2.8737015E-1,-3.5356164E-2,-2.4583596E-3,-2.3003162E-1,1.2230969E-1,2.8985894E-1,2.7634403E-2,-1.7368701E-1,1.5692785E-1,-6.285604E-3,2.964344E-1,2.2068767E-2,4.8101977E-1,3.759303E-2,2.8679523E-1,1.4870151E-2,-2.6925838E-2,-4.0212208E-1,-4.096826E-3,-2.137026E-1,-6.836231E-3,-1.3444592E-2,-3.8865723E-2,-2.6402366E-1,1.5824297E-1,-2.2372782E-2,-5.3985547E-3,-1.578221E-2,4.4510947E-3,-4.0943245E-3,-1.4209974E-1,-3.0908227E-1,1.5854962E-1,-1.1736854E-2,3.3811715E-1,1.2577349E-5,2.0957749E-1,2.2717987E-3,1.3396783E-3,-1.34293E-2,2.9276553E-1,5.889664E-2,7.2887614E-3,1.6884295E-2,3.213141E-2,4.2333996E-1,3.931903E-3,3.7188956E-1,6.670329E-3,-5.4053906E-3,-1.2444593E-2,-2.0330459E-2,-2.6028606E-3,2.246441E-3,-1.16904E-2,-2.8625585E-3,1.214974E-3,-5.0083925E-3,-2.0037686E-2,-3.8257658E-3,9.175969E-3,9.234783E-4,5.0457725E-3,-2.9046652E-3,-8.612932E-3,-1.4552756E-3,-1.7296202E-2,-6.5335864E-3,1.165884E-2,5.567034E-3,-9.023175E-3,4.944839E-3,3.1616357E-3,1.768841E-2,5.10647E-3,1.272136E-2,5.8530597E-3,-1.3110178E-3,7.6247165E-3,1.604775E-2,-1.01537524E-4,4.86696E-3,1.5735019E-2,2.5501208E-2,1.2866216E-2,2.2314796E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":16,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,-1,49,51,-1,53,-1,-1,55,57,59,61,63,65,-1,67,-1,-1,69,71,-1,73,75,-1,-1,-1,-1,-1,77,79,-1,-1,81,83,85,87,-1,89,91,93,95,97,99,101,-1,103,-1,105,-1,107,109,-1,111,113,115,-1,-1,117,119,121,123,-1,-1,-1,-1,125,127,129,131,133,-1,135,137,-1,-1,139,141,-1,-1,-1,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.144008E1,1.7224161E1,9.746468E0,3.7994614E0,5.0415754E0,2.7336E-1,5.1218185E0,1.666275E0,1.6677818E0,2.174542E0,2.1706672E0,0E0,0E0,1.6717057E0,3.6376572E0,7.9847336E-1,6.1182594E-1,2.5490952E-1,3.5110507E0,1.0343378E0,1.0808005E0,1.0238662E0,8.7667865E-1,5.712082E-1,1.5635157E-1,0E0,2.697029E0,0E0,3.688898E-1,1.9667292E-1,0E0,1.6143703E-1,0E0,0E0,1.1936326E0,2.9615879E-2,5.2677053E-1,9.7309446E-1,4.7796416E-1,8.8831997E-1,0E0,4.9104002E-1,0E0,0E0,4.2291194E-1,5.3732872E-2,0E0,1.4730797E0,5.279665E-1,0E0,0E0,0E0,0E0,0E0,4.7389984E-2,5.14647E-1,0E0,0E0,5.481267E-2,5.462638E-1,3.702563E-1,1.3188899E-1,0E0,1.2662499E-1,1.7851388E-1,4.8278666E-1,6.8468237E-1,4.1483343E-1,3.3658E-1,4.3551242E-1,0E0,6.503463E-2,0E0,1.0486107E0,0E0,4.155228E-1,2.0299675E-1,0E0,5.1404E-2,4.3675426E-2,4.0889072E-1,0E0,0E0,5.105803E-1,3.3006567E-1,1.16334915E-1,1.3340145E-1,0E0,0E0,0E0,0E0,7.6116264E-2,1.18971944E-1,2.4879396E-1,4.963316E-1,4.6277714E-1,0E0,4.053977E-2,3.0608138E-1,0E0,0E0,3.8637996E-2,5.9232615E-2,0E0,0E0,0E0,7.600117E-1,0E0,5.230713E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,26,26,28,28,29,29,31,31,34,34,35,35,36,36,37,37,38,38,39,39,41,41,44,44,45,45,47,47,48,48,54,54,55,55,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71,73,73,75,75,76,76,78,78,79,79,80,80,83,83,84,84,85,85,86,86,91,91,92,92,93,93,94,94,95,95,97,97,98,98,101,101,102,102,106,106,108,108],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,-1,50,52,-1,54,-1,-1,56,58,60,62,64,66,-1,68,-1,-1,70,72,-1,74,76,-1,-1,-1,-1,-1,78,80,-1,-1,82,84,86,88,-1,90,92,94,96,98,100,102,-1,104,-1,106,-1,108,110,-1,112,114,116,-1,-1,118,120,122,124,-1,-1,-1,-1,126,128,130,132,134,-1,136,138,-1,-1,140,142,-1,-1,-1,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.8988005E5,1E0,3.3817584E7,9.31E2,1.0778182E-8,8.40413E2,2E0,3.06E2,4.325E0,4.111111E0,-2.498843E-2,-8.891666E-3,2.2318378E6,3.3817584E7,4.869229E7,1.7E1,1.8E6,8.093982E-6,3.89E2,1E0,4.11566E5,1.2784314E1,1E0,2.0619047E1,5.9625734E-2,8.7E1,-1.822205E-2,4.1618698E2,4.631579E0,-7.0150075E-3,5.4009717E3,-7.856098E-3,2.6124738E-2,8.9535345E2,7.2998254E2,2.9652428E6,5.4E1,7.478261E0,5.372237E2,-1.0087745E-2,1E0,-1.6949257E-2,-7.0907E-3,2.3447667E5,5.5177975E6,7.344695E-3,6.663214E6,4.24038E8,-5.7051998E-2,-3.0912023E-2,-2.436236E-2,-1.15936715E-2,-9.144405E-3,3.9E1,1.8411793E0,-2.2827908E-2,-1.5887633E-2,5.1E1,3.29E2,3.5364E4,9.40386E-6,-3.5356164E-2,1.11E2,2.7777777E0,3.3358974E0,8.695652E0,2.347826E0,5.195011E7,1.0617E4,-6.285604E-3,1.3826087E1,2.2068767E-2,1.307E3,3.759303E-2,2.15E2,4.900919E8,-2.6925838E-2,1.1066655E-2,1.3809524E0,3.422351E6,-6.836231E-3,-1.3444592E-2,1.9452055E0,7.31E2,3.7346938E0,1.1E1,-5.3985547E-3,-1.578221E-2,4.4510947E-3,-4.0943245E-3,2.3819518E4,5.7777777E0,1.0164831E-2,2.378914E6,1.2259939E3,1.2577349E-5,2.8543878E6,8.315789E0,1.3396783E-3,-1.34293E-2,4.7185095E2,5.357143E2,7.2887614E-3,1.6884295E-2,3.213141E-2,3.2585382E10,3.931903E-3,1.8682626E8,6.670329E-3,-5.4053906E-3,-1.2444593E-2,-2.0330459E-2,-2.6028606E-3,2.246441E-3,-1.16904E-2,-2.8625585E-3,1.214974E-3,-5.0083925E-3,-2.0037686E-2,-3.8257658E-3,9.175969E-3,9.234783E-4,5.0457725E-3,-2.9046652E-3,-8.612932E-3,-1.4552756E-3,-1.7296202E-2,-6.5335864E-3,1.165884E-2,5.567034E-3,-9.023175E-3,4.944839E-3,3.1616357E-3,1.768841E-2,5.10647E-3,1.272136E-2,5.8530597E-3,-1.3110178E-3,7.6247165E-3,1.604775E-2,-1.01537524E-4,4.86696E-3,1.5735019E-2,2.5501208E-2,1.2866216E-2,2.2314796E-2],"split_indices":[2,37,16,7,2,46,61,26,2,63,63,0,0,37,7,40,3,5,47,2,109,1,62,24,65,0,8,0,4,63,0,37,0,0,4,56,37,0,67,4,0,85,0,0,42,54,0,37,41,0,0,0,0,0,2,62,0,0,10,0,9,46,0,10,65,63,55,62,7,10,0,67,0,38,0,8,7,0,47,63,9,0,0,62,2,65,3,0,0,0,0,42,65,47,9,42,0,54,67,0,0,4,61,0,0,0,40,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.96E2,7.39E2,2.57E2,2.2E2,5.19E2,1.4E1,2.43E2,6.1E1,1.59E2,2.59E2,2.6E2,9E0,5E0,7.6E1,1.67E2,2E1,4.1E1,5.4E1,1.05E2,1.91E2,6.8E1,1.52E2,1.08E2,4.7E1,2.9E1,5E0,1.62E2,6E0,1.4E1,3.3E1,8E0,4.8E1,6E0,5E0,1E2,2.4E1,1.67E2,2.4E1,4.4E1,1.46E2,6E0,1.02E2,6E0,9E0,3.8E1,2.4E1,5E0,1.27E2,3.5E1,5E0,9E0,2.7E1,6E0,5E0,4.3E1,8.6E1,1.4E1,5E0,1.9E1,1.23E2,4.4E1,1.8E1,6E0,1.3E1,3.1E1,9.8E1,4.8E1,8.9E1,1.3E1,3.3E1,5E0,1.2E1,1.2E1,1.09E2,1.8E1,2.5E1,1E1,9E0,3.4E1,1.4E1,7.2E1,1.3E1,6E0,1.12E2,1.1E1,2.1E1,2.3E1,5E0,1.3E1,6E0,7E0,1.6E1,1.5E1,7.7E1,2.1E1,4.1E1,7E0,1E1,7.9E1,5E0,8E0,1.3E1,2E1,5E0,7E0,2.1E1,8.8E1,8E0,1.7E1,5E0,5E0,9E0,2.5E1,7E0,7E0,5.7E1,1.5E1,5.8E1,5.4E1,5E0,6E0,1.6E1,5E0,5E0,1.8E1,1.1E1,5E0,1E1,5E0,2.1E1,5.6E1,8E0,1.3E1,6E0,3.5E1,5E0,5E0,1.5E1,6.4E1,5E0,8E0,9E0,1.1E1,5.5E1,3.3E1,1.1E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[4.6168067E-3,-1.1760387E-1,3.6185855E-1,-3.3220103E-1,-9.39157E-3,-4.949357E-1,4.0771586E-1,-3.9621598E-1,-1.3996054E-1,-1.0837633E-1,9.045719E-2,-3.1770296E-2,-1.087847E-2,2.0776677E-1,5.135202E-1,-3.7256166E-1,-3.995781E-2,-2.1658462E-1,-4.5299176E-2,-9.1080494E-2,-3.5889894E-2,1.7515914E-1,4.2955894E-3,8.839574E-2,3.95252E-1,6.7559063E-1,3.6869454E-1,-4.825745E-1,-2.938744E-1,-1.4096716E-3,-2.554003E-1,8.896217E-3,-1.1810081E-2,-2.5474358E-1,-5.349155E-2,1.9479632E-1,-5.8725714E-3,2.5996191E-2,-1.6227165E-2,-3.24476E-2,1.9313459E-1,4.363408E-1,8.63826E-3,5.458155E-1,4.6493944E-2,5.9074724E-1,2.2495645E-1,-4.2404985E-1,-3.6988202E-2,-1.1376509E-1,-3.4648514E-1,-1.6435365E-1,-1.500327E-2,-6.1713126E-2,9.795391E-2,-2.803434E-1,-2.1291394E-3,-2.5697193E-1,-3.5004526E-2,1.6938622E-1,1.689857E-2,5.570805E-2,-1.3556194E-1,4.2548E-2,-8.52812E-2,2.891924E-1,-4.0315354E-4,1.407794E-2,2.2683257E-2,7.19628E-1,4.5120397E-1,3.1735875E-2,4.0188304E-1,3.1980765E-1,-8.446736E-3,-2.2881322E-2,-2.4173725E-1,-4.1470453E-1,1.5230906E-1,-6.693326E-3,-3.7008822E-1,-1.037829E-2,-3.390919E-3,2.1370568E-3,-5.8789635E-3,7.5298934E-3,8.0261193E-4,-2.4172014E-1,-1.6105346E-2,-2.1742402E-2,-1.1818155E-1,-4.8236422E-2,1.1778803E-2,1.07161626E-1,2.3859829E-1,1.9327313E-1,3.017571E-2,-1.4692858E-2,8.469912E-3,5.777984E-3,-2.1586139E-3,-1.2379118E-3,-7.829306E-3,1.56019805E-2,6.086838E-3,3.720126E-2,2.0203361E-2,2.9096383E-1,5.0780696E-1,1.0631145E-2,2.2328231E-2,1.3325076E-1,4.153072E-1,7.076371E-3,-1.1004927E-2,-6.6924645E-3,-1.2789659E-2,-2.2976775E-2,-1.1677848E-2,2.0130945E-2,-5.2472414E-3,-1.8725473E-2,-1.1557081E-2,-1.2452611E-2,-5.7259044E-3,-9.620843E-3,-4.323374E-4,1.6049712E-3,-3.9168647E-3,6.7011863E-3,-1.1290824E-3,1.2352506E-2,2.5387146E-4,5.945339E-3,1.1572616E-2,-6.025738E-3,2.4999222E-3,-3.2620982E-3,3.982596E-3,2.1472188E-2,1.9502107E-3,1.3664931E-2,2.5847796E-2,1.259009E-2,-4.4587045E-3,2.1642206E-2,1.2911649E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":17,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,-1,35,37,39,41,43,45,47,49,-1,51,53,-1,55,57,59,-1,61,-1,63,65,67,-1,69,-1,71,73,75,-1,77,79,81,-1,83,85,87,-1,89,91,93,-1,95,97,99,101,103,-1,-1,-1,105,107,-1,109,111,113,-1,115,117,119,-1,121,-1,-1,-1,-1,-1,-1,123,-1,-1,125,127,-1,129,131,133,135,-1,137,-1,-1,-1,-1,-1,-1,-1,-1,139,141,-1,-1,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.230769E1,1.6763199E1,9.90649E0,2.9226646E0,4.7637696E0,4.8210716E-1,4.8648453E0,1.6698341E0,4.398961E-1,2.7134814E0,1.7507809E0,0E0,0E0,1.8226266E0,3.3421211E0,1.3644314E0,0E0,2.379793E-1,3.311068E-1,1.4443787E0,0E0,7.4158263E-1,9.3099433E-1,6.644304E-1,1.9367313E-1,2.4609642E0,2.5162544E0,9.6918297E-1,9.460535E-1,0E0,1.200819E-1,1.5696436E-1,0E0,2.2239041E-1,7.207837E-1,4.290247E-1,0E0,5.547738E-1,0E0,1.0268573E-1,5.4534066E-1,3.3226013E-2,0E0,5.468683E-1,0E0,2.885399E-1,1.1539073E0,6.6172314E-1,0E0,2.0649686E0,3.2847404E-1,5.945283E-2,0E0,1.0389264E-1,5.4267764E-2,1.6658068E-2,0E0,4.1455483E-1,6.964158E-1,4.0940738E-1,0E0,3.3486667E-1,4.7219333E-1,8.6114384E-2,6.885818E-2,9.717548E-2,0E0,0E0,0E0,4.7721863E-2,1.8783951E-1,0E0,6.791949E-2,6.038556E-1,6.252258E-1,0E0,2.305317E-2,3.283167E-2,1.0951942E0,0E0,1.8715763E-1,0E0,0E0,0E0,0E0,0E0,0E0,5.5196166E-2,0E0,0E0,1.0484548E-1,5.183455E-1,0E0,2.7300543E-1,3.0120683E-1,2.2895277E-2,3.2185122E-1,0E0,7.307114E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,4.937638E-1,1.0915136E-1,0E0,0E0,4.6946013E-1,3.834915E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,33,33,34,34,35,35,37,37,39,39,40,40,41,41,43,43,45,45,46,46,47,47,49,49,50,50,51,51,53,53,54,54,55,55,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,69,69,70,70,72,72,73,73,74,74,76,76,77,77,78,78,80,80,87,87,90,90,91,91,93,93,94,94,95,95,96,96,98,98,107,107,108,108,111,111,112,112],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,-1,36,38,40,42,44,46,48,50,-1,52,54,-1,56,58,60,-1,62,-1,64,66,68,-1,70,-1,72,74,76,-1,78,80,82,-1,84,86,88,-1,90,92,94,-1,96,98,100,102,104,-1,-1,-1,106,108,-1,110,112,114,-1,116,118,120,-1,122,-1,-1,-1,-1,-1,-1,124,-1,-1,126,128,-1,130,132,134,136,-1,138,-1,-1,-1,-1,-1,-1,-1,-1,140,142,-1,-1,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,2.16577E5,1E0,1.380863E6,9.31E2,5.97E3,9.6215985E2,9.886539E1,1.5199841E2,9.639872E0,3.8610423E0,-3.1770296E-2,-1.087847E-2,2.36433E6,5.974481E1,1.0479E4,-3.995781E-2,1.8111588E0,1.7500242E7,3.89E2,-3.5889894E-2,2.6776E4,1.2784314E1,3.9401392E6,8.954333E6,2.0597144E3,2.047619E0,5.356E3,7.807738E6,-1.4096716E-3,1.257E4,1.2860047E5,-1.1810081E-2,7.9016544E5,2.508E2,2.4526315E0,-5.8725714E-3,1E0,-1.6227165E-2,5.8746633E4,4.4133335E1,4.4149057E2,8.63826E-3,2.2034105E6,4.6493944E-2,2.395631E7,2.948485E8,3.23E2,-3.6988202E-2,2.25E2,1.4953704E0,2.49E2,-1.500327E-2,2.2941177E0,1E0,2.1959582E8,-2.1291394E-3,4.649123E-1,5.6530495E6,4.2060452E2,1.689857E-2,1.26E2,5.6440186E2,4.6603775E0,1.2E1,5.1034E4,-4.0315354E-4,1.407794E-2,2.2683257E-2,2.6818182E0,1.5484655E6,3.1735875E-2,5.854E3,7.958E3,1.8258025E2,-2.2881322E-2,1.00013E5,1.76421E6,2.3E1,-6.693326E-3,6.2394366E0,-1.037829E-2,-3.390919E-3,2.1370568E-3,-5.8789635E-3,7.5298934E-3,8.0261193E-4,1.4495105E2,-1.6105346E-2,-2.1742402E-2,1.1643481E6,1.7754011E0,1.1778803E-2,3.8580435E2,2E1,2.1618656E7,1.244E3,-1.4692858E-2,8.4797815E2,5.777984E-3,-2.1586139E-3,-1.2379118E-3,-7.829306E-3,1.56019805E-2,6.086838E-3,3.720126E-2,2.0203361E-2,6.374563E5,2.4860917E5,1.0631145E-2,2.2328231E-2,1.7754222E3,4.0278234E0,7.076371E-3,-1.1004927E-2,-6.6924645E-3,-1.2789659E-2,-2.2976775E-2,-1.1677848E-2,2.0130945E-2,-5.2472414E-3,-1.8725473E-2,-1.1557081E-2,-1.2452611E-2,-5.7259044E-3,-9.620843E-3,-4.323374E-4,1.6049712E-3,-3.9168647E-3,6.7011863E-3,-1.1290824E-3,1.2352506E-2,2.5387146E-4,5.945339E-3,1.1572616E-2,-6.025738E-3,2.4999222E-3,-3.2620982E-3,3.982596E-3,2.1472188E-2,1.9502107E-3,1.3664931E-2,2.5847796E-2,1.259009E-2,-4.4587045E-3,2.1642206E-2,1.2911649E-2],"split_indices":[2,37,16,9,2,2,61,55,61,63,63,0,0,37,67,9,0,62,12,2,0,38,62,54,54,61,62,1,40,0,12,37,0,37,42,62,0,85,0,42,67,4,0,54,0,56,41,1,0,2,63,2,0,62,23,40,0,67,37,61,0,10,4,63,3,38,0,0,0,62,37,0,2,2,67,0,5,40,0,0,63,0,0,0,0,0,0,4,0,0,37,62,0,4,6,5,2,0,4,0,0,0,0,0,0,0,0,37,42,0,0,4,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.67E2,7.21E2,2.46E2,2.41E2,4.8E2,1.2E1,2.34E2,1.8E2,6.1E1,2.41E2,2.39E2,6E0,6E0,8.2E1,1.52E2,1.73E2,7E0,3.3E1,2.8E1,2.36E2,5E0,1.2E2,1.19E2,5.1E1,3.1E1,7E1,8.2E1,7E1,1.03E2,6E0,2.7E1,2.3E1,5E0,4.3E1,1.93E2,1.13E2,7E0,1.13E2,6E0,2.4E1,2.7E1,2.5E1,6E0,5.2E1,1.8E1,3.1E1,5.1E1,6.1E1,9E0,2.4E1,7.9E1,1.3E1,1.4E1,1.3E1,1E1,3.8E1,5E0,1.5E1,1.78E2,1E2,1.3E1,9.6E1,1.7E1,1E1,1.4E1,1.8E1,9E0,9E0,1.6E1,1.6E1,3.6E1,1.9E1,1.2E1,3.6E1,1.5E1,4.3E1,1.8E1,1.1E1,1.3E1,9E0,7E1,7E0,6E0,5E0,8E0,5E0,5E0,2.7E1,1.1E1,5E0,1E1,1.71E2,7E0,5.4E1,4.6E1,1.4E1,8.2E1,7E0,1E1,5E0,5E0,9E0,5E0,1.3E1,5E0,1.1E1,5E0,1.1E1,2.5E1,5E0,7E0,1.3E1,2.3E1,9E0,6E0,6E0,1.2E1,6E0,5E0,6E0,7E0,5.3E1,1.7E1,2.1E1,6E0,5E0,5E0,5.2E1,1.19E2,4.2E1,1.2E1,4.1E1,5E0,8E0,6E0,1E1,7.2E1,5E0,5E0,6E0,5E0,6E0,1.9E1,8E0,5E0,1.5E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[1.1362983E-3,-1.14706405E-1,3.2657325E-1,-3.1962496E-1,-1.6412651E-2,-4.6288857E-1,3.6375603E-1,-4.474065E-1,-2.4508457E-1,-1.23935886E-1,6.3618556E-2,-1.1119877E-2,-2.9982315E-2,5.747517E-2,3.3294705E-1,-3.642667E-1,-5.165989E-1,-1.9395955E-1,-5.0540924E-1,-1.9125783E-1,-1.5189391E-2,5.6308457E-3,1.8268427E-1,1.683854E-1,4.329925E-1,-3.913862E-1,-8.01581E-3,-5.6112516E-1,-1.3042589E-2,-2.8227174E-1,-5.2313346E-2,-4.039966E-2,-1.6864134E-2,-3.142841E-2,-1.6619506E-1,-1.2115812E-2,1.5338894E-2,2.09842E-2,-1.8501643E-2,-5.8198413E-3,2.0953397E-1,2.8008202E-2,1.0182547E-1,3.0949512E-1,5.661825E-1,-1.9331373E-2,-1.0545984E-2,-3.908659E-1,-2.8786406E-2,-1.1258042E-1,-3.2424724E-1,2.106114E-2,-1.4527778E-1,5.2950732E-2,-2.1138936E-1,-1.19035775E-2,1.6996951E-1,-4.607542E-2,8.709711E-2,2.31439E-1,-5.449856E-3,1.2870075E-1,-9.310991E-3,3.6686778E-1,9.4683945E-2,6.3984716E-1,3.6382946E-1,-2.2007441E-2,-1.0596539E-2,-1.9086818E-1,2.9119572E-3,-2.7605662E-1,-2.4247423E-2,-1.9899333E-1,2.9075904E-2,-1.6736452E-2,7.2136964E-3,-1.7823815E-1,-3.9399397E-1,5.6415773E-3,-4.2480245E-2,2.7670083E-3,1.1691545E-2,2.581635E-2,-1.536393E-1,1.1874737E-1,-9.998273E-2,1.400649E-1,2.9261518E-1,-1.1703798E-2,2.0792215E-1,3.9351442E-1,4.699565E-3,-1.984144E-3,9.693631E-3,3.8201872E-2,5.7948995E-1,2.3099497E-1,2.5557308E-2,-1.1052342E-2,-5.708569E-3,-1.6798927E-2,-7.0475214E-3,-1.1597686E-2,-1.6391671E-3,-1.4242546E-3,3.8976131E-3,-3.847036E-3,2.8434899E-3,-1.2825303E-2,-4.7823326E-3,-1.1634923E-2,-2.4214568E-2,3.7724231E-3,-3.428373E-3,9.374694E-3,-3.60248E-4,-4.3180017E-3,-2.0931417E-2,6.176033E-3,-3.736319E-3,-8.385539E-3,1.753652E-3,7.504156E-3,5.5717054E-4,1.7724179E-2,8.864446E-3,-7.003665E-3,1.5372853E-3,1.1071862E-2,-2.771404E-4,8.902273E-3,2.0099342E-2,2.8438311E-2,1.821592E-2,1.5643615E-2,1.5433108E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":18,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,-1,23,25,27,29,31,33,35,37,39,41,43,45,-1,47,-1,49,51,-1,-1,-1,53,-1,55,57,-1,-1,59,-1,61,63,65,-1,-1,67,-1,69,71,-1,73,75,77,79,81,83,85,87,-1,89,-1,91,93,95,97,-1,-1,99,-1,101,-1,103,105,107,-1,109,111,-1,113,-1,-1,115,117,119,121,123,125,127,129,131,-1,-1,-1,-1,133,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.6907566E1,1.4537434E1,7.7124577E0,2.1191692E0,4.2158346E0,3.3613682E-1,6.2188263E0,3.007946E-1,1.8983364E0,1.5271893E0,1.9390931E0,0E0,0E0,0E0,3.8619003E0,1.505189E-1,3.036623E-1,1.5738873E0,9.2802095E-1,1.4589953E0,6.1214024E-1,1.1879944E0,7.87375E-1,2.6037784E0,2.2591648E0,5.4433823E-2,0E0,1.20848656E-1,0E0,5.1908064E-1,2.41898E0,0E0,0E0,0E0,1.2393067E0,0E0,3.0953723E-1,8.1977814E-1,0E0,0E0,6.36014E-1,0E0,6.843292E-1,9.3888235E-1,8.1064415E-1,0E0,0E0,5.4374456E-2,0E0,2.4533577E-1,4.483223E-1,0E0,4.0850264E-1,1.5977554E-1,5.532923E-1,2.623125E-1,8.450374E-2,7.1539855E-1,5.625373E-1,4.0620804E-1,0E0,8.366618E-1,0E0,3.9891052E-1,2.9155034E-1,7.660675E-2,3.943553E-1,0E0,0E0,9.224296E-3,0E0,5.0025034E-1,0E0,2.6640368E-1,3.8691178E-2,7.736194E-2,0E0,6.282935E-1,1.4389348E-1,0E0,2.0817298E-1,0E0,0E0,3.3296424E-1,6.247709E-1,2.396189E-1,1.6257325E-1,9.238446E-2,3.5368013E-1,1.7980054E-1,3.1660724E-1,3.648262E-1,0E0,0E0,0E0,0E0,1.5419006E-2,2.8265494E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,29,29,30,30,34,34,36,36,37,37,40,40,42,42,43,43,44,44,47,47,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,61,61,63,63,64,64,65,65,66,66,69,69,71,71,73,73,74,74,75,75,77,77,78,78,80,80,83,83,84,84,85,85,86,86,87,87,88,88,89,89,90,90,91,91,96,96,97,97],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,-1,24,26,28,30,32,34,36,38,40,42,44,46,-1,48,-1,50,52,-1,-1,-1,54,-1,56,58,-1,-1,60,-1,62,64,66,-1,-1,68,-1,70,72,-1,74,76,78,80,82,84,86,88,-1,90,-1,92,94,96,98,-1,-1,100,-1,102,-1,104,106,108,-1,110,112,-1,114,-1,-1,116,118,120,122,124,126,128,130,132,-1,-1,-1,-1,134,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,1.972052E5,1E0,8.8E1,8.1E2,5.111912E9,3.3817584E7,6.48334E5,2.2E1,2.61E2,2.0766992E6,-1.1119877E-2,-2.9982315E-2,5.747517E-2,1.1251919E6,2.0376764E-7,4.2E1,6.15E2,2E0,2.2953334E2,5.764631E1,1.2784314E1,2.9077E4,5E1,7.459E3,5.1485147E0,-8.01581E-3,1E0,-1.3042589E-2,2E0,2.7E1,-4.039966E-2,-1.6864134E-2,-3.142841E-2,1.4026549E0,-1.2115812E-2,1.6100003E8,1.273801E6,-1.8501643E-2,-5.8198413E-3,8.629636E2,2.8008202E-2,3.0070068E7,1E0,8.166121E1,-1.9331373E-2,-1.0545984E-2,5.4E1,-2.8786406E-2,1.7559428E2,1.91E2,2.106114E-2,2.1340163E8,4.49E2,2E0,1.7407408E0,7.67525E2,3.14403E5,3.84E2,5.8036E4,-5.449856E-3,4.1032645E6,-9.310991E-3,2.7578741E1,1.3453986E7,2.5959E4,9.730783E8,-2.2007441E-2,-1.0596539E-2,1.273801E6,2.9119572E-3,2.71E2,-2.4247423E-2,3.3284525E2,6.0875E4,1E0,7.2136964E-3,1.6310204E2,5E1,5.6415773E-3,2.63E2,2.7670083E-3,1.1691545E-2,5.2571965E8,2.0312219E5,1.9E1,1.2300693E0,1.3558896E7,1.162E3,1.5204346E8,5.444797E0,4.1718604E2,4.699565E-3,-1.984144E-3,9.693631E-3,3.8201872E-2,2.171312E0,6.2868685E6,2.5557308E-2,-1.1052342E-2,-5.708569E-3,-1.6798927E-2,-7.0475214E-3,-1.1597686E-2,-1.6391671E-3,-1.4242546E-3,3.8976131E-3,-3.847036E-3,2.8434899E-3,-1.2825303E-2,-4.7823326E-3,-1.1634923E-2,-2.4214568E-2,3.7724231E-3,-3.428373E-3,9.374694E-3,-3.60248E-4,-4.3180017E-3,-2.0931417E-2,6.176033E-3,-3.736319E-3,-8.385539E-3,1.753652E-3,7.504156E-3,5.5717054E-4,1.7724179E-2,8.864446E-3,-7.003665E-3,1.5372853E-3,1.1071862E-2,-2.771404E-4,8.902273E-3,2.0099342E-2,2.8438311E-2,1.821592E-2,1.5643615E-2,1.5433108E-3],"split_indices":[2,37,16,38,2,40,7,40,3,64,37,0,0,0,37,46,6,2,26,42,56,62,1,0,2,63,0,21,0,26,0,0,0,0,62,0,5,9,0,0,61,0,52,6,65,0,0,2,0,4,0,0,7,10,6,62,42,9,0,1,0,54,0,65,56,9,7,0,0,9,0,2,0,61,1,8,0,64,0,0,38,0,0,40,42,3,47,9,10,7,62,61,0,0,0,0,51,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.77E2,7.21E2,2.56E2,2.33E2,4.88E2,1.1E1,2.45E2,8.4E1,1.49E2,2.08E2,2.8E2,6E0,5E0,7E0,2.38E2,4.1E1,4.3E1,1.26E2,2.3E1,1.28E2,8E1,1.89E2,9.1E1,9.1E1,1.47E2,3.5E1,6E0,3.5E1,8E0,7.7E1,4.9E1,5E0,1.8E1,5E0,1.23E2,8E0,7.2E1,1.83E2,6E0,7E0,8.4E1,1.1E1,8E1,7.8E1,6.9E1,2.9E1,6E0,1.1E1,2.4E1,1.6E1,6.1E1,7E0,4.2E1,2.1E1,1.02E2,6.2E1,1E1,9.1E1,9.2E1,7.9E1,5E0,7.4E1,6E0,6.1E1,1.7E1,4.9E1,2E1,6E0,5E0,1.1E1,5E0,5.1E1,1E1,3.2E1,1E1,1.3E1,8E0,8.8E1,1.4E1,1.1E1,5.1E1,5E0,5E0,5.5E1,3.6E1,7.9E1,1.3E1,3.3E1,4.6E1,2.7E1,4.7E1,5.5E1,6E0,8E0,9E0,9E0,4E1,1.3E1,7E0,5E0,6E0,2.9E1,2.2E1,2.4E1,8E0,5E0,5E0,7E0,6E0,3.7E1,5.1E1,8E0,6E0,1E1,4.1E1,8E0,4.7E1,3.1E1,5E0,7.4E1,5E0,8E0,5E0,2.8E1,5E0,2.3E1,2.3E1,6E0,2.1E1,4.1E1,6E0,1E1,4.5E1,3.2E1,8E0,8E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[-9.941617E-3,-1.1259863E-1,2.852374E-1,-3.1001148E-1,-1.7344737E-2,-4.577365E-1,3.2495528E-1,-4.4213814E-1,-2.3889941E-1,-1.0530404E-1,7.710663E-2,-2.49235E-2,-1.2785851E-2,7.520282E-1,2.8409624E-1,-3.8332465E-1,-4.2851888E-2,-3.933442E-1,-1.7494968E-1,-5.8247983E-2,-2.329609E-1,2.1871451E-2,1.8658845E-1,2.4791304E-2,4.6948284E-2,1.7062986E-1,3.929169E-1,-4.402026E-1,-2.9206416E-1,-2.2584662E-2,-3.2258874E-1,1.9518267E-2,-2.195119E-1,-8.304042E-2,8.899104E-2,-4.1004694E-1,-1.4888072E-1,5.992733E-2,-1.16444565E-1,7.181498E-3,2.4424185E-1,1.4326946E-1,2.528462E-2,2.8130573E-1,5.034524E-1,-2.1669915E-2,-9.137355E-3,-1.1202624E-1,-4.1586784E-1,-6.3827625E-3,-3.568312E-1,-1.8269674E-1,-2.2432664E-2,-1.3300762E-1,1.2952804E-2,1.3537185E-1,-8.7286165E-4,-2.606654E-1,-3.4338184E-2,-4.6969894E-2,-2.1635701E-1,1.2916698E-1,-4.374589E-2,-1.5304203E-1,3.7050212E-3,-5.128397E-3,6.5522894E-2,1.3928214E-1,3.1732157E-1,1.6708463E-1,-1.2324641E-2,1.4355527E-1,3.4640318E-1,5.4773E-1,2.688531E-3,-7.7808327E-3,-4.5730287E-4,-2.2311652E-2,-9.5461495E-3,-1.8492265E-2,-9.490978E-3,-1.4906628E-1,-1.602614E-2,-1.7828402E-1,-2.0637855E-2,1.21735334E-1,-5.2686792E-2,3.618016E-3,8.838235E-3,-3.351395E-1,-3.7437496E-3,-8.981966E-3,4.6864375E-2,-1.5967706E-1,-1.292798E-2,-4.181924E-3,1.6091761E-1,-1.4703374E-1,1.654031E-1,-2.2122712E-1,-2.1354876E-2,5.219866E-4,4.8636524E-3,1.9458422E-1,-1.7920043E-3,3.5868984E-1,2.9922072E-3,-7.3881205E-3,1.9423378E-1,2.8635052E-1,-1.293386E-3,2.5429273E-1,2.0166144E-2,5.7669073E-1,1.3529204E-2,-2.1871654E-3,-8.735517E-3,-4.1114422E-3,-1.0440401E-2,-3.3625504E-3,3.1304914E-3,3.3053805E-3,1.0955216E-2,-1.1687774E-2,1.2044418E-3,-1.8922213E-2,-9.58662E-3,5.972856E-3,-9.461137E-4,-9.596283E-3,-3.7486036E-3,4.53688E-3,1.1972368E-2,-2.0656607E-3,-1.9805586E-2,1.3867386E-2,3.0360536E-3,-6.6186953E-3,-1.2884437E-2,3.7884566E-3,-5.605021E-3,1.2958087E-2,3.8032874E-3,1.7696768E-2,9.654067E-3,1.618771E-2,6.7309705E-3,6.8266205E-3,1.7531952E-2,3.2091988E-3,1.4282779E-2,2.778233E-2,1.5811661E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":19,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,35,37,39,-1,-1,41,43,45,47,-1,49,-1,51,53,55,57,59,61,63,65,67,69,-1,71,73,-1,-1,75,77,-1,79,81,-1,83,85,87,-1,89,-1,91,93,95,97,99,-1,-1,101,103,105,107,-1,109,111,113,-1,-1,-1,-1,-1,-1,-1,115,-1,117,119,121,123,-1,-1,125,-1,-1,127,129,-1,-1,131,133,135,137,139,-1,-1,141,-1,143,-1,-1,145,147,-1,149,-1,151,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9122223E1,1.3402118E1,7.467064E0,2.0741978E0,4.0125165E0,4.0059566E-2,3.9045029E0,1.9110355E0,1.4423199E0,1.4881871E0,1.404928E0,0E0,0E0,5.906868E-1,2.612362E0,2.5035477E-1,0E0,1.337595E-1,3.0119143E0,6.790662E-1,9.365437E-1,8.2812124E-1,8.050325E-1,0E0,0E0,1.0279677E0,1.2154446E0,1.8866539E-1,6.44382E-1,0E0,1.2831593E-1,0E0,9.125867E-1,7.5957155E-1,1.375463E-1,8.4233713E-1,3.130362E-1,8.885811E-1,2.5647333E-1,1.445879E-1,4.0835834E-1,1.0376353E0,0E0,4.614029E-1,1.0334721E0,0E0,0E0,7.79715E-2,1.4689255E-1,0E0,6.458068E-2,4.5190287E-1,0E0,5.254415E-1,3.9913812E-1,4.5246065E-2,0E0,1.8742085E-1,0E0,2.842118E-1,5.432999E-2,5.297489E-1,1.1044474E0,2.556696E-1,0E0,0E0,2.86807E-2,2.633143E-1,3.3248425E-1,8.902633E-1,0E0,4.975708E-1,1.9825792E-1,1.9534111E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.027947E-1,0E0,2.901404E-1,1.4645731E-1,1.0446593E-1,5.5487347E-1,0E0,0E0,1.8486619E-2,0E0,0E0,7.4198574E-2,4.6512425E-2,0E0,0E0,3.7331522E-1,9.562958E-1,2.0726973E-1,4.4683695E-2,1.2253784E-1,0E0,0E0,1.6785365E-1,0E0,1.9848585E-2,0E0,0E0,6.337054E-1,8.453345E-2,0E0,1.8681645E-1,0E0,1.9887924E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,43,43,44,44,47,47,48,48,50,50,51,51,53,53,54,54,55,55,57,57,59,59,60,60,61,61,62,62,63,63,66,66,67,67,68,68,69,69,71,71,72,72,73,73,81,81,83,83,84,84,85,85,86,86,89,89,92,92,93,93,96,96,97,97,98,98,99,99,100,100,103,103,105,105,108,108,109,109,111,111,113,113],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,36,38,40,-1,-1,42,44,46,48,-1,50,-1,52,54,56,58,60,62,64,66,68,70,-1,72,74,-1,-1,76,78,-1,80,82,-1,84,86,88,-1,90,-1,92,94,96,98,100,-1,-1,102,104,106,108,-1,110,112,114,-1,-1,-1,-1,-1,-1,-1,116,-1,118,120,122,124,-1,-1,126,-1,-1,128,130,-1,-1,132,134,136,138,140,-1,-1,142,-1,144,-1,-1,146,148,-1,150,-1,152,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1.972052E5,1E0,3.741267E7,9.48E2,5.97E3,3.6E1,2E0,2.55E2,4.325E0,1.836095E6,-2.49235E-2,-1.2785851E-2,1.84146E5,1.7587205E6,9.2E1,-4.2851888E-2,4.54E2,2.3E1,3.355102E2,1E0,3.9E2,3.8795E4,2.4791304E-2,4.6948284E-2,5.428706E3,6.929E3,2.9036145E0,1E0,-2.2584662E-2,6.695E3,1.9518267E-2,7.042942E7,2.331083E6,3.2608695E0,7.1E1,8.315789E0,4.4889745E6,3.4779343E1,2.524828E2,4.0953247E2,3.0070068E7,2.528462E-2,2.9251662E6,6.292039E0,-2.1669915E-2,-9.137355E-3,3.72E2,6.27907E0,-6.3827625E-3,9.07021E5,2.0734E4,-2.2432664E-2,1.852364E6,1.4285715E0,8.6E1,-8.7286165E-4,9E0,-3.4338184E-2,5.95E2,3.9961785E-2,3.693316E7,1.399844E8,8.484036E5,3.7050212E-3,-5.128397E-3,3.6401E4,3.474468E2,7.653112E2,3.64E2,-1.2324641E-2,3.39498E5,1.4433751E0,2.53E2,2.688531E-3,-7.7808327E-3,-4.5730287E-4,-2.2311652E-2,-9.5461495E-3,-1.8492265E-2,-9.490978E-3,3.390625E0,-1.602614E-2,4.8421054E0,1.7733E4,5.036081E7,7.271717E6,3.618016E-3,8.838235E-3,1.901875E2,-3.7437496E-3,-8.981966E-3,6.176636E5,1.3430429E4,-1.292798E-2,-4.181924E-3,4.0446457E2,1.594E3,3.4338624E0,6.1723955E9,1.4E1,5.219866E-4,4.8636524E-3,1.162E3,-1.7920043E-3,3.8396227E0,2.9922072E-3,-7.3881205E-3,2.6818182E0,2.68334E5,-1.293386E-3,1.5714285E0,2.0166144E-2,1.9995576E7,1.3529204E-2,-2.1871654E-3,-8.735517E-3,-4.1114422E-3,-1.0440401E-2,-3.3625504E-3,3.1304914E-3,3.3053805E-3,1.0955216E-2,-1.1687774E-2,1.2044418E-3,-1.8922213E-2,-9.58662E-3,5.972856E-3,-9.461137E-4,-9.596283E-3,-3.7486036E-3,4.53688E-3,1.1972368E-2,-2.0656607E-3,-1.9805586E-2,1.3867386E-2,3.0360536E-3,-6.6186953E-3,-1.2884437E-2,3.7884566E-3,-5.605021E-3,1.2958087E-2,3.8032874E-3,1.7696768E-2,9.654067E-3,1.618771E-2,6.7309705E-3,6.8266205E-3,1.7531952E-2,3.2091988E-3,1.4282779E-2,2.778233E-2,1.5811661E-2],"split_indices":[2,37,16,7,2,2,0,6,2,63,37,0,0,1,37,38,0,1,0,61,109,0,1,0,0,61,2,62,8,0,9,0,54,37,63,0,67,54,65,4,61,52,0,37,47,0,0,2,65,0,9,38,0,9,65,0,0,8,0,2,47,7,5,37,0,0,9,4,4,10,0,9,47,8,0,0,0,0,0,0,0,63,0,67,1,7,54,0,0,4,0,0,37,56,0,0,61,2,62,40,3,0,0,10,0,63,0,0,63,1,0,63,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.59E2,7.12E2,2.47E2,2.31E2,4.81E2,1.2E1,2.35E2,7.9E1,1.52E2,2.49E2,2.32E2,7E0,5E0,1.9E1,2.16E2,7.2E1,7E0,4.3E1,1.09E2,1.83E2,6.6E1,1.55E2,7.7E1,1.2E1,7E0,1.07E2,1.09E2,4.2E1,3E1,1.6E1,2.7E1,7E0,1.02E2,1.57E2,2.6E1,2E1,4.6E1,1.22E2,3.3E1,1.9E1,5.8E1,1.01E2,6E0,5.6E1,5.3E1,3.7E1,5E0,1.3E1,1.7E1,5E0,2.2E1,9.1E1,1.1E1,1.03E2,5.4E1,1.8E1,8E0,1.5E1,5E0,1.9E1,2.7E1,7.3E1,4.9E1,2.8E1,5E0,6E0,1.3E1,2.5E1,3.3E1,9.6E1,5E0,1.9E1,3.7E1,4.8E1,5E0,8E0,5E0,1.2E1,5E0,1.6E1,6E0,7.7E1,1.4E1,7.3E1,3E1,2E1,3.4E1,1E1,8E0,1E1,5E0,7E0,1.2E1,1.6E1,1.1E1,9E0,6.4E1,3.3E1,1.6E1,1.8E1,1E1,6E0,7E0,1.9E1,6E0,2.8E1,5E0,7E0,8.9E1,1E1,9E0,2E1,1.7E1,4.2E1,6E0,2.2E1,5.5E1,2.6E1,4.7E1,1.9E1,1.1E1,1.5E1,5E0,9E0,2.5E1,5E0,5E0,5E0,7E0,9E0,7E0,4E1,2.4E1,2.5E1,8E0,6E0,1E1,9E0,9E0,5E0,5E0,1E1,9E0,2.3E1,5E0,2E1,6.9E1,5E0,5E0,5E0,1.5E1,3.7E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"153","size_leaf_vector":"1"}},{"base_weights":[1.0668243E-2,-8.592487E-2,3.3548513E-1,-2.5083745E-1,4.69919E-3,3.9230636E-1,-9.152475E-2,2.0960905E-1,-2.696424E-1,-1.9948885E-1,3.6774695E-2,1.4449899E-1,4.7337437E-1,-1.5122566E-2,4.3780405E-2,2.0912865E-2,-3.0823522E-3,-4.0706977E-1,-2.1573392E-1,-3.738746E-2,-2.5949797E-1,-3.016978E-1,4.9189255E-2,3.5514675E-2,1.6423889E-2,8.262936E-1,4.2433646E-1,1.7504469E-1,-8.699814E-3,-3.634117E-1,-3.8022626E-2,-3.5923415E-1,-1.7138499E-1,4.1796933E-3,-1.3828078E-1,-1.6407548E-1,-3.1635836E-1,-3.2565596E-3,-2.5809988E-2,6.2174555E-3,1.446328E-1,6.923404E-2,-4.8457826E-3,2.4416424E-2,4.5644064E-2,2.0316662E-1,4.76669E-1,1.2540402E-2,3.1744589E-3,-3.259497E-1,-3.1723578E-2,-4.1739154E-1,-1.5210435E-1,-1.988967E-1,1.49023775E-2,-8.579423E-3,-3.183511E-3,-1.1412593E-2,-4.1288957E-3,-1.96851E-2,-2.3724243E-1,2.0852337E-2,-1.5136658E-1,5.9907254E-2,2.2840744E-1,9.28772E-3,2.4425937E-2,3.1002563E-1,1.9551147E-2,3.0430121E-2,4.2469007E-1,-1.839867E-2,-2.2860624E-1,-4.75046E-1,-9.783933E-3,9.856493E-4,-1.39244925E-2,-2.4553369E-1,-8.756222E-2,-1.3395404E-2,-5.001426E-3,-1.6637983E-2,8.867876E-2,-1.28553E-2,-5.7481572E-2,1.1030065E-1,-7.500369E-2,2.5240108E-1,-1.9301699E-3,-3.4978164E-3,9.932462E-2,8.972145E-3,1.6521825E-2,3.4678485E-3,-1.8047191E-3,3.774848E-1,5.9130615E-1,-6.427039E-3,-1.6483156E-2,-2.9383186E-2,-1.8737359E-2,-1.9677817E-3,-1.23449825E-2,4.115955E-4,-9.005288E-3,-2.489454E-3,3.7997968E-3,8.067869E-3,1.3085095E-3,-5.261485E-3,2.1013645E-3,2.689985E-3,1.1260853E-2,-1.0172202E-2,-1.2377427E-4,8.629965E-3,1.9260285E-2,8.012592E-3,-6.98149E-4,6.7636105E-3,1.8431658E-2,3.1963944E-2,1.8347103E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":20,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,-1,-1,29,31,33,35,37,39,41,-1,43,45,47,-1,49,-1,51,53,-1,55,57,59,-1,-1,61,63,65,-1,-1,-1,67,69,-1,-1,71,-1,73,75,77,-1,-1,-1,-1,-1,-1,79,81,83,85,87,-1,89,91,93,-1,95,-1,97,99,-1,-1,-1,101,103,-1,-1,105,107,-1,109,111,113,115,-1,-1,117,-1,-1,-1,-1,119,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.05254E1,1.1209964E1,5.4466076E0,2.3597298E0,3.1838114E0,3.8891792E0,8.705766E-1,7.6204216E-1,1.8087597E0,6.323173E-1,1.7796751E0,1.1288688E0,2.2107239E0,0E0,5.7927626E-1,0E0,0E0,9.80505E-1,1.1221514E0,2.5680953E-1,2.0852947E-1,8.4496284E-1,1.6642612E0,1.6767853E-1,0E0,2.854128E-1,1.430975E0,1.033552E-1,0E0,5.6907177E-1,0E0,4.5449638E-1,2.015253E0,0E0,2.3043841E-2,1.0109234E-1,1.4689922E-1,0E0,0E0,6.5120524E-1,8.806462E-1,1.5938511E-1,0E0,0E0,0E0,5.237049E-1,7.0861053E-1,0E0,0E0,3.284421E-1,0E0,2.927704E-1,2.8857705E-1,6.898465E-1,0E0,0E0,0E0,0E0,0E0,0E0,9.543979E-2,6.5752673E-1,2.67286E-1,4.4338465E-1,4.1864133E-1,0E0,1.7195307E-1,2.247715E-2,3.8387932E-2,0E0,4.4532967E-1,0E0,2.7024436E-1,2.597475E-2,0E0,0E0,0E0,3.8895988E-1,4.331461E-1,0E0,0E0,6.1346066E-1,4.6914512E-1,0E0,9.132983E-2,3.115282E-1,1.8720673E-1,5.49448E-1,0E0,0E0,1.1167592E-1,0E0,0E0,0E0,0E0,2.5793457E-1,5.2558422E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,27,27,29,29,31,31,32,32,34,34,35,35,36,36,39,39,40,40,41,41,45,45,46,46,49,49,51,51,52,52,53,53,60,60,61,61,62,62,63,63,64,64,66,66,67,67,68,68,70,70,72,72,73,73,77,77,78,78,81,81,82,82,84,84,85,85,86,86,87,87,90,90,95,95,96,96],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,-1,-1,30,32,34,36,38,40,42,-1,44,46,48,-1,50,-1,52,54,-1,56,58,60,-1,-1,62,64,66,-1,-1,-1,68,70,-1,-1,72,-1,74,76,78,-1,-1,-1,-1,-1,-1,80,82,84,86,88,-1,90,92,94,-1,96,-1,98,100,-1,-1,-1,102,104,-1,-1,106,108,-1,110,112,114,116,-1,-1,118,-1,-1,-1,-1,120,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,2.2783158E5,2.856934E7,6E0,4.67E2,8.507324E2,1.465E4,1.89991E5,3.434405E7,1.7407408E0,1.45064E5,5.53648E8,1.059448E7,-1.5122566E-2,6.947958E-1,2.0912865E-2,-3.0823522E-3,1.605076E6,7.7E1,3.5364E4,6.97E3,1.6736842E0,2.0766992E6,7.1913286E8,1.6423889E-2,6.5026445E3,3.3580637E8,5.369854E5,-8.699814E-3,2.347364E7,-3.8022626E-2,3.0088105E0,5.933284E2,4.1796933E-3,4.685271E8,8.3E1,4.795239E7,-3.2565596E-3,-2.5809988E-2,1.1021739E1,5.80418E7,3.9914E4,-4.8457826E-3,2.4416424E-2,4.5644064E-2,1.4553613E1,9.1470585E0,1.2540402E-2,3.1744589E-3,6.2E1,-3.1723578E-2,8.579633E-2,1.8E1,7.82E2,1.49023775E-2,-8.579423E-3,-3.183511E-3,-1.1412593E-2,-4.1288957E-3,-1.96851E-2,5.417598E-1,3.7667856E2,9.010751E6,6.5139695E-3,2.737878E5,9.28772E-3,4.6589856E2,1.7608696E0,9.643839E6,3.0430121E-2,4.91027E0,-1.839867E-2,6.925789E2,1.3934426E0,-9.783933E-3,9.856493E-4,-1.39244925E-2,1.0952381E0,5.4444447E0,-1.3395404E-2,-5.001426E-3,9.751103E6,2.1506848E0,-1.28553E-2,1.038946E6,6.3157897E0,2.9652428E6,5.281628E2,-1.9301699E-3,-3.4978164E-3,1.3922E4,8.972145E-3,1.6521825E-2,3.4678485E-3,-1.8047191E-3,1E0,1.034073E6,-6.427039E-3,-1.6483156E-2,-2.9383186E-2,-1.8737359E-2,-1.9677817E-3,-1.23449825E-2,4.115955E-4,-9.005288E-3,-2.489454E-3,3.7997968E-3,8.067869E-3,1.3085095E-3,-5.261485E-3,2.1013645E-3,2.689985E-3,1.1260853E-2,-1.0172202E-2,-1.2377427E-4,8.629965E-3,1.9260285E-2,8.012592E-3,-6.98149E-4,6.7636105E-3,1.8431658E-2,3.1963944E-2,1.8347103E-2],"split_indices":[2,37,54,3,2,61,38,9,7,62,7,7,12,0,43,0,0,12,38,9,12,62,37,5,0,4,7,37,0,5,0,62,61,0,40,12,7,0,0,55,7,9,0,0,0,65,65,0,0,4,0,47,3,2,0,0,0,0,0,0,47,61,12,47,42,0,61,62,56,0,44,0,42,62,0,0,0,62,63,0,0,9,62,0,9,65,37,4,0,0,38,0,0,0,0,14,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.71E2,7.49E2,2.22E2,2.65E2,4.84E2,1.96E2,2.6E1,1E1,2.55E2,6.5E1,4.19E2,4.9E1,1.47E2,9E0,1.7E1,5E0,5E0,7E1,1.85E2,1.8E1,4.7E1,1.4E1,4.05E2,3.3E1,1.6E1,1.6E1,1.31E2,1.1E1,6E0,6.5E1,5E0,4.2E1,1.43E2,8E0,1E1,1.9E1,2.8E1,8E0,6E0,2.8E2,1.25E2,2.7E1,6E0,7E0,9E0,2.6E1,1.05E2,5E0,6E0,6E1,5E0,3.2E1,1E1,1.36E2,7E0,5E0,5E0,8E0,1.1E1,1E1,1.8E1,2.57E2,2.3E1,6.3E1,6.2E1,6E0,2.1E1,1.6E1,1E1,2.1E1,8.4E1,3.3E1,2.7E1,2.4E1,8E0,5E0,5E0,9.5E1,4.1E1,1.2E1,6E0,1.66E2,9.1E1,9E0,1.4E1,4.6E1,1.7E1,5.7E1,5E0,9E0,1.2E1,6E0,1E1,5E0,5E0,6.8E1,1.6E1,1.7E1,1E1,5E0,1.9E1,9E0,8.6E1,2.2E1,1.9E1,1.21E2,4.5E1,3.7E1,5.4E1,9E0,5E0,3.4E1,1.2E1,5E0,1.2E1,4.2E1,1.5E1,7E0,5E0,6E0,6.2E1,9E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[8.4056575E-3,-9.333274E-2,2.7849245E-1,-1.9071312E-1,5.5229068E-2,-4.0131226E-1,3.103919E-1,-3.8690308E-1,-1.4182937E-1,-4.2329464E-2,1.06986545E-1,-8.818634E-3,-2.4362285E-2,2.1560355E-1,4.303917E-1,-3.4357485E-1,-2.58691E-2,-2.4555074E-1,-4.563828E-2,-1.4806508E-2,-1.5766088E-2,7.20939E-2,2.2556382E-1,2.8510666E-1,6.73775E-2,7.222989E-1,3.3376545E-1,-3.8497594E-1,3.3501608E-3,-2.1531592E-1,-3.143751E-2,2.6379913E-2,-7.0957385E-2,-6.102273E-2,1.2661591E-1,2.8527575E-2,1.7203213E-1,7.288677E-2,3.0497122E-1,3.1268084E-1,-4.9280245E-3,-4.1957315E-2,1.3291712E-1,5.0853354E-1,4.033048E-2,5.450422E-1,2.3330061E-1,-2.376744E-2,-1.627005E-2,-1.15420856E-1,-2.5798023E-1,1.0732244E-2,-1.471366E-1,-1.7127737E-1,-2.3074655E-2,2.4427394E-3,1.5106429E-1,1.0175925E-1,-2.1604536E-2,2.2286426E-1,-7.892752E-3,1.3150949E-3,6.2629324E-3,6.1445646E-3,3.8852793E-1,2.3797876E-1,4.322365E-1,-7.027677E-3,1.1810105E-2,1.7355402E-1,2.1116615E-4,2.7538078E-2,1.4406478E-2,6.3290596E-1,1.002875E-2,2.653469E-1,-4.1137007E-3,-2.4202836E-1,-4.226768E-2,-2.1869782E-1,-3.689269E-1,-2.3466028E-2,1.2536158E-1,-1.844405E-2,-1.1315157E-1,-1.011761E-3,-2.508252E-1,-1.566443E-1,1.7573105E-2,9.048754E-3,3.0944494E-3,4.654497E-3,1.8939428E-1,-1.8105106E-1,2.2957364E-2,1.360997E-1,3.1693435E-1,1.0606744E-2,2.0235535E-2,1.3658726E-1,3.0254984E-1,1.066334E-2,4.985684E-1,3.3534248E-3,-3.2839389E-3,1.2372034E-1,1.326205E-2,3.629384E-2,2.2844499E-2,3.8366783E-1,1.1748079E-1,-7.427005E-3,-1.5601779E-2,-6.58425E-3,5.5379514E-4,-1.2318977E-2,-6.117601E-3,-8.734601E-3,-1.9196274E-2,9.346475E-4,-6.311792E-3,4.1803016E-4,8.276241E-3,-8.9814365E-3,-2.4592504E-3,-1.55055495E-2,-5.8310395E-3,-4.005955E-3,-9.026099E-3,2.4502291E-3,-5.088008E-3,-2.6757848E-3,4.8880647E-3,1.0493014E-2,2.7758644E-3,-1.511268E-2,1.9488084E-3,-4.291902E-3,2.6290729E-3,9.760207E-3,2.9176474E-3,6.7067742E-3,1.7479496E-2,9.647698E-3,-2.512827E-4,1.7643547E-2,8.295038E-3,2.5575392E-2,1.3548799E-2,7.239461E-3,1.7859702E-3,2.0514864E-2,1.2249101E-2,-4.8914715E-4,1.0153267E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":21,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,-1,29,31,33,-1,35,37,39,41,43,45,47,-1,49,-1,-1,51,53,55,57,59,61,63,65,-1,67,69,71,-1,73,75,-1,-1,77,79,81,83,85,87,-1,89,91,93,95,-1,-1,-1,-1,97,99,101,-1,103,105,-1,-1,-1,107,-1,109,-1,111,113,115,117,119,121,-1,123,-1,125,127,129,-1,-1,131,133,135,137,139,141,-1,-1,143,145,-1,147,-1,-1,149,-1,-1,-1,151,153,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.5910456E1,9.920431E0,5.7193966E0,3.9155922E0,1.3779608E0,2.1390331E-1,2.7167282E0,4.146738E-1,3.3111587E0,7.7273524E-1,7.202089E-1,0E0,0E0,1.4230938E0,2.8051586E0,1.1983242E0,0E0,1.9438982E0,2.7461185E0,5.828953E-1,0E0,6.002901E-1,4.6271896E-1,1.0582786E0,3.3520192E-1,3.744583E-1,1.6345901E0,9.120083E-2,0E0,6.137786E-1,0E0,0E0,1.0470518E0,2.7637255E-1,3.1832933E-2,3.6198178E-1,7.683548E-1,3.766153E-2,3.2339478E-1,6.9200134E-1,0E0,1.0776744E-1,1.504857E-1,3.4480095E-2,0E0,6.1456823E-1,6.2194586E-1,0E0,0E0,4.2507178E-1,3.7693357E-1,3.2439202E-1,6.9528866E-1,1.9666341E-1,2.7964738E-1,0E0,4.633081E-2,3.3950958E-1,4.2336562E-1,2.588575E-1,0E0,0E0,0E0,0E0,2.9690027E-2,3.3185196E-1,3.2850933E-1,0E0,7.0028886E-2,8.772361E-2,0E0,0E0,0E0,3.4597397E-2,0E0,8.821964E-1,0E0,7.746029E-2,1.6911171E-1,2.9017138E-1,1.3253164E-1,3.1691238E-1,1.1557728E-1,0E0,3.6702877E-1,0E0,7.2752595E-2,1.5169352E-2,1.8510456E-1,0E0,0E0,1.3158517E-1,8.64687E-2,4.331393E-1,1.8813656E-1,1.0094237E-1,1.2075162E-1,0E0,0E0,2.4425751E-1,2.5902605E-1,0E0,9.997511E-2,0E0,0E0,4.0800273E-2,0E0,0E0,0E0,8.619881E-2,3.2666063E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,29,29,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,56,56,57,57,58,58,59,59,64,64,65,65,66,66,68,68,69,69,73,73,75,75,77,77,78,78,79,79,80,80,81,81,82,82,84,84,86,86,87,87,88,88,91,91,92,92,93,93,94,94,95,95,96,96,99,99,100,100,102,102,105,105,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,-1,30,32,34,-1,36,38,40,42,44,46,48,-1,50,-1,-1,52,54,56,58,60,62,64,66,-1,68,70,72,-1,74,76,-1,-1,78,80,82,84,86,88,-1,90,92,94,96,-1,-1,-1,-1,98,100,102,-1,104,106,-1,-1,-1,108,-1,110,-1,112,114,116,118,120,122,-1,124,-1,126,128,130,-1,-1,132,134,136,138,140,142,-1,-1,144,146,-1,148,-1,-1,150,-1,-1,-1,152,154,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,7.785302E5,1E0,9.2E1,8.01E2,1.04719896E8,1.3631968E3,4.217427E7,6.99E2,7.54E2,3.1993368E6,-8.818634E-3,-2.4362285E-2,8E0,4.4133335E1,3.284E4,-2.58691E-2,4.909091E0,2.04115E5,4.0705118E6,-1.5766088E-2,6.6858566E-1,5.1942E4,3.3000212E5,1.9840922E8,3E0,1.9619734E0,1.2772413E0,3.3501608E-3,1.9452055E0,-3.143751E-2,2.6379913E-2,4.638E3,1E0,9.183673E-1,1.5753187E4,5.4544777E1,7.4391775E6,1E0,1.7500046E8,-4.9280245E-3,8.147158E4,1.0656093E2,3.8275862E0,4.033048E-2,2.7994856E5,1.9307388E7,-2.376744E-2,-1.627005E-2,5.748115E-5,1.6523365E1,9.751103E6,9.66E2,1.4875445E0,5.02E2,2.4427394E-3,8.773336E6,5.036081E7,3.49E2,3.0812828E6,-7.892752E-3,1.3150949E-3,6.2629324E-3,6.1445646E-3,1.99E2,6.9706165E2,1E0,-7.027677E-3,6.2136955E0,1.8664102E2,2.1116615E-4,2.7538078E-2,1.4406478E-2,1.4595818E7,1.002875E-2,3.09627E5,-4.1137007E-3,1.3871127E2,1.6262903E2,4.33E2,2E0,3.5333333E0,2.5677419E0,-1.844405E-2,1.188E3,-1.011761E-3,8.012931E0,7.635E3,4.066351E0,9.048754E-3,3.0944494E-3,2.153533E6,9.522029E6,1.2031894E5,1E0,1.1925396E6,1.01E2,1.0606744E-2,2.0235535E-2,4.5158855E6,7.978062E6,1.066334E-2,1.2213524E3,3.3534248E-3,-3.2839389E-3,7.120907E6,1.326205E-2,3.629384E-2,2.2844499E-2,4.5709968E0,1.9856038E3,-7.427005E-3,-1.5601779E-2,-6.58425E-3,5.5379514E-4,-1.2318977E-2,-6.117601E-3,-8.734601E-3,-1.9196274E-2,9.346475E-4,-6.311792E-3,4.1803016E-4,8.276241E-3,-8.9814365E-3,-2.4592504E-3,-1.55055495E-2,-5.8310395E-3,-4.005955E-3,-9.026099E-3,2.4502291E-3,-5.088008E-3,-2.6757848E-3,4.8880647E-3,1.0493014E-2,2.7758644E-3,-1.511268E-2,1.9488084E-3,-4.291902E-3,2.6290729E-3,9.760207E-3,2.9176474E-3,6.7067742E-3,1.7479496E-2,9.647698E-3,-2.512827E-4,1.7643547E-2,8.295038E-3,2.5575392E-2,1.3548799E-2,7.239461E-3,1.7859702E-3,2.0514864E-2,1.2249101E-2,-4.8914715E-4,1.0153267E-2],"split_indices":[2,37,16,38,2,54,61,54,2,2,37,0,0,26,67,1,0,62,5,37,0,51,1,42,5,8,62,62,0,62,0,0,38,109,65,42,67,54,109,12,0,37,67,63,0,42,52,0,0,47,67,9,12,63,2,0,54,7,10,60,0,0,0,0,0,61,109,0,63,65,0,0,0,56,0,9,0,61,4,2,8,62,63,0,2,0,55,1,63,0,0,37,9,42,109,37,0,0,0,56,56,0,61,0,0,54,0,0,0,63,61,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.41E2,6.84E2,2.57E2,4.13E2,2.71E2,1.1E1,2.46E2,8.1E1,3.32E2,9.4E1,1.77E2,5E0,6E0,1.39E2,1.07E2,6.7E1,1.4E1,1.59E2,1.73E2,8.7E1,7E0,1.38E2,3.9E1,9.4E1,4.5E1,2.5E1,8.2E1,6.1E1,6E0,1.5E2,9E0,6E0,1.67E2,6.6E1,2.1E1,9.7E1,4.1E1,1.4E1,2.5E1,8.8E1,6E0,1.7E1,2.8E1,1.2E1,1.3E1,2.5E1,5.7E1,1E1,5.1E1,4.6E1,1.04E2,8.1E1,8.6E1,1.6E1,5E1,6E0,1.5E1,3.9E1,5.8E1,3.6E1,5E0,9E0,5E0,9E0,1.6E1,5.6E1,3.2E1,5E0,1.2E1,2.1E1,7E0,7E0,5E0,1.9E1,6E0,5.2E1,5E0,1.6E1,3E1,7.9E1,2.5E1,6.3E1,1.8E1,9E0,7.7E1,6E0,1E1,1.1E1,3.9E1,9E0,6E0,1.9E1,2E1,1.2E1,4.6E1,2E1,1.6E1,5E0,1.1E1,2.3E1,3.3E1,9E0,2.3E1,7E0,5E0,1.6E1,5E0,7E0,1.2E1,2.8E1,2.4E1,1E1,6E0,1E1,2E1,5E1,2.9E1,6E0,1.9E1,4.6E1,1.7E1,6E0,1.2E1,3.2E1,4.5E1,5E0,5E0,5E0,6E0,3.1E1,8E0,1.2E1,7E0,1.5E1,5E0,7E0,5E0,1E1,3.6E1,9E0,1.1E1,5E0,1.1E1,1.5E1,8E0,1.9E1,1.4E1,1.7E1,6E0,1.1E1,5E0,1.7E1,1.1E1,1.1E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"155","size_leaf_vector":"1"}},{"base_weights":[7.835916E-4,-7.960553E-2,2.629665E-1,-2.5229427E-1,-9.1440845E-3,3.1978473E-1,-1.0375372E-1,-2.1321099E-1,-4.437389E-1,-2.3041488E-1,1.4015268E-2,1.2459343E-1,3.8170224E-1,2.3938317E-2,-2.1246247E-2,-3.206519E-1,-1.12872444E-1,-3.4784976E-1,-3.5444696E-2,-2.5969967E-1,-1.0110292E-3,6.9630675E-2,-3.5707194E-2,9.769181E-3,2.2524801E-1,4.6255913E-1,3.0222455E-1,1.7683637E-1,-1.309546E-1,-1.8905828E-2,-2.6905113E-1,2.178782E-2,-1.6699955E-1,-1.8132944E-1,-4.6140692E-1,-1.38398E-1,-3.2902387E-1,-2.0774575E-2,1.3633513E-1,-5.163789E-2,1.3516198E-1,-3.09242E-3,6.365497E-2,3.7768413E-3,2.7666736E-1,3.6586612E-1,5.910787E-1,5.712563E-1,2.1958448E-1,1.4500534E-2,-5.070847E-4,3.8402755E-4,-1.0745536E-2,-5.431004E-3,-2.8191036E-1,-1.2543362E-1,-3.6586958E-1,-3.909907E-3,-1.0739634E-2,-2.4221355E-2,-1.2684867E-2,-7.595319E-3,-3.003853E-3,-3.5591528E-1,-8.354982E-3,-2.8026426E-1,1.386592E-2,1.5369546E-1,-1.0838631E-2,-8.98576E-2,-1.403885E-4,1.214465E-2,6.708073E-2,5.5438997E-3,-4.2878056E-4,1.57696E-2,4.344131E-3,3.985717E-1,4.7886055E-3,2.999561E-2,1.5430881E-2,1.6307551E-2,3.0586211E-2,1.7012827E-1,3.4186506E-1,-2.957295E-1,-7.1814405E-3,-1.4428451E-2,-1.7497823E-1,-2.2926481E-2,-7.750567E-3,-1.9728348E-2,-1.2459659E-2,-2.4197496E-2,3.5665618E-4,-2.676263E-2,9.4627805E-2,9.938503E-2,2.4200432E-1,-2.4865888E-2,-1.4506984E-1,-1.6215876E-1,2.4935555E-2,8.281958E-3,-3.0041602E-4,4.2584765E-1,1.024682E-2,2.0154305E-1,-2.8963701E-3,2.0481614E-2,9.993828E-3,-8.207501E-3,-1.4719058E-2,-4.7420957E-3,1.8187346E-3,-9.230363E-3,-1.2687685E-3,-2.3855073E-3,8.686958E-3,7.772113E-3,3.3176602E-3,9.988882E-3,2.5125064E-3,8.520671E-3,1.723972E-2,3.0478963E-3,-5.182666E-3,-1.1012187E-2,-3.8865253E-3,-3.310129E-4,-1.5860802E-2,2.2076215E-3,-6.0160393E-3,2.1619454E-2,1.0292147E-2,6.187049E-3,1.9212121E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":22,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,37,39,41,43,45,47,49,51,-1,53,-1,55,57,59,61,63,65,67,69,71,-1,73,-1,75,77,79,81,83,-1,-1,-1,-1,-1,85,87,89,-1,-1,-1,-1,-1,-1,91,-1,93,95,97,-1,99,101,-1,103,-1,-1,-1,-1,105,-1,-1,-1,-1,-1,107,109,111,-1,113,115,-1,-1,-1,-1,-1,-1,117,119,121,123,125,127,129,131,-1,-1,133,-1,135,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9580069E1,8.64907E0,4.575511E0,1.4467411E0,2.5962205E0,2.2366486E0,1.3676078E0,1.8288269E0,7.780881E-1,2.8759933E-1,1.2717749E0,5.410061E-1,7.791691E-1,5.678625E-1,0E0,2.6215363E-1,2.9472826E0,4.4580126E-1,0E0,3.0764747E-1,0E0,1.3114363E0,6.684413E-1,9.895167E-2,1.6239583E-1,6.412029E-1,1.5420208E0,3.0912504E-1,1.7005207E-1,0E0,7.450724E-2,0E0,6.484697E-1,4.3752193E-2,3.9630413E-2,2.1078736E-2,4.005909E-2,8.3984923E-1,8.3866453E-1,4.389003E-1,1.634138E-1,0E0,5.8078263E-2,0E0,1.7640626E-1,3.1966782E-1,1.85915E-1,9.690189E-2,2.9864335E-1,0E0,0E0,0E0,0E0,0E0,4.2073727E-2,3.8795447E-1,2.7443945E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.2940884E-2,0E0,7.6812744E-1,2.7487525E-1,5.514796E-1,0E0,4.5570004E-1,3.9406884E-1,0E0,1.2407983E-1,0E0,0E0,0E0,0E0,8.532524E-2,0E0,0E0,0E0,0E0,0E0,3.3402443E-1,1.06387734E-1,5.6288242E-2,0E0,1.12264976E-1,1.6784894E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.0500686E-1,3.521198E-2,3.867789E-1,2.7202296E-1,4.7911134E-1,3.704201E-1,3.596608E-1,3.0020854E-1,0E0,0E0,1.5024519E-1,0E0,5.100378E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,48,48,54,54,55,55,56,56,63,63,65,65,66,66,67,67,69,69,70,70,72,72,77,77,83,83,84,84,85,85,87,87,88,88,95,95,96,96,97,97,98,98,99,99,100,100,101,101,102,102,105,105,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,38,40,42,44,46,48,50,52,-1,54,-1,56,58,60,62,64,66,68,70,72,-1,74,-1,76,78,80,82,84,-1,-1,-1,-1,-1,86,88,90,-1,-1,-1,-1,-1,-1,92,-1,94,96,98,-1,100,102,-1,104,-1,-1,-1,-1,106,-1,-1,-1,-1,-1,108,110,112,-1,114,116,-1,-1,-1,-1,-1,-1,118,120,122,124,126,128,130,132,-1,-1,134,-1,136,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,1.7909248E5,2.7555334E7,1E0,4.03E2,7.449545E2,9.7172376E7,3.6E2,4.4561405E0,8.599521E5,3.2857144E0,6.7148806E5,3.66025E5,1E0,-2.1246247E-2,3.9E1,2.00087E5,4.217427E7,-3.5444696E-2,3.1E1,-1.0110292E-3,8.1E2,3.1146461E1,9.0927E4,1.2157188E6,1.5632523E3,7.556899E-1,5.2224144E5,3.592428E7,-1.8905828E-2,2.416E3,2.178782E-2,3.478E4,4.313463E-2,2.77915E7,1E0,9E0,5.862757E1,1.4676277E7,9.860918E5,2.9215686E0,-3.09242E-3,1.49E2,3.7768413E-3,4.6E1,1.2673605E7,1.214874E2,1.2979348E3,1.803948E7,1.4500534E-2,-5.070847E-4,3.8402755E-4,-1.0745536E-2,-5.431004E-3,4.267606E0,3.8396227E0,7.163082E7,-3.909907E-3,-1.0739634E-2,-2.4221355E-2,-1.2684867E-2,-7.595319E-3,-3.003853E-3,1.1392951E2,-8.354982E-3,3.8248322E2,2.732381E2,4.5432812E2,-1.0838631E-2,1.1054945E1,2.0671213E2,1.214465E-2,6.257143E2,5.5438997E-3,-4.2878056E-4,1.57696E-2,4.344131E-3,1.8484E4,4.7886055E-3,2.999561E-2,1.5430881E-2,1.6307551E-2,3.0586211E-2,4.49239E6,1.4875445E0,3.7289145E4,-7.1814405E-3,1.31306E5,4.6695483E2,-2.2926481E-2,-7.750567E-3,-1.9728348E-2,-1.2459659E-2,-2.4197496E-2,3.5665618E-4,1E0,2.6089566E2,1.8716538E3,1.11791E5,1E0,4.05E2,1.9310344E0,1E0,8.281958E-3,-3.0041602E-4,2.737878E5,1.024682E-2,3.430961E7,-2.8963701E-3,2.0481614E-2,9.993828E-3,-8.207501E-3,-1.4719058E-2,-4.7420957E-3,1.8187346E-3,-9.230363E-3,-1.2687685E-3,-2.3855073E-3,8.686958E-3,7.772113E-3,3.3176602E-3,9.988882E-3,2.5125064E-3,8.520671E-3,1.723972E-2,3.0478963E-3,-5.182666E-3,-1.1012187E-2,-3.8865253E-3,-3.310129E-4,-1.5860802E-2,2.2076215E-3,-6.0160393E-3,2.1619454E-2,1.0292147E-2,6.187049E-3,1.9212121E-2],"split_indices":[2,37,54,6,2,61,54,2,62,37,63,37,9,14,0,2,5,54,0,10,0,2,65,1,56,61,51,37,54,0,9,0,38,47,12,8,8,56,56,37,62,0,8,0,8,54,67,4,9,0,0,0,0,0,65,63,7,0,0,0,0,0,0,61,0,42,4,4,0,67,4,0,4,0,0,0,0,38,0,0,0,0,0,37,63,54,0,9,64,0,0,0,0,0,0,25,61,42,1,23,10,65,85,0,0,42,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.27E2,7.1E2,2.17E2,2.05E2,5.05E2,1.88E2,2.9E1,1.72E2,3.3E1,4.7E1,4.58E2,4.6E1,1.42E2,2.2E1,7E0,8.2E1,9E1,2.7E1,6E0,4.1E1,6E0,2.16E2,2.42E2,2.2E1,2.4E1,6.8E1,7.4E1,1.1E1,1.1E1,2.8E1,5.4E1,7E0,8.3E1,1.2E1,1.5E1,1.6E1,2.5E1,9.2E1,1.24E2,2.22E2,2E1,9E0,1.3E1,7E0,1.7E1,4.1E1,2.7E1,1.6E1,5.8E1,6E0,5E0,5E0,6E0,5E0,4.9E1,7E1,1.3E1,5E0,7E0,1E1,5E0,1.1E1,5E0,2E1,5E0,1E1,8.2E1,1.19E2,5E0,1.27E2,9.5E1,6E0,1.4E1,7E0,6E0,1.2E1,5E0,3.6E1,5E0,2.1E1,6E0,6E0,1E1,4.3E1,1.5E1,4.3E1,6E0,2.2E1,4.8E1,7E0,6E0,9E0,1.1E1,5E0,5E0,5.5E1,2.7E1,7.5E1,4.4E1,5.9E1,6.8E1,1.2E1,8.3E1,5E0,9E0,3E1,6E0,3.8E1,5E0,7E0,8E0,8E0,3.5E1,8E0,1.4E1,4.1E1,7E0,5E1,5E0,5E0,2.2E1,2E1,5.5E1,3.2E1,1.2E1,2.9E1,3E1,2.6E1,4.2E1,7E0,5E0,7.3E1,1E1,2.4E1,6E0,3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"137","size_leaf_vector":"1"}},{"base_weights":[2.216326E-3,-8.5598685E-2,2.6890466E-1,-1.9236791E-1,1.8266905E-2,3.121548E-1,-1.0407366E-1,-3.7630317E-1,-1.4252283E-1,-5.4921422E-2,9.618047E-2,8.4462637E-1,2.824945E-1,3.520985E-3,-1.8402897E-2,-2.5344324E-1,-4.5359045E-1,-1.1843666E-1,-5.4458547E-1,-1.0077703E-1,8.283043E-2,1.08368024E-1,-1.0392984E-2,2.420904E-2,4.7639586E-2,1.6257134E-1,3.878572E-1,-1.6010647E-1,8.498472E-3,-2.9685608E-1,-2.5978703E-3,-3.80188E-1,-4.081178E-2,-2.0659009E-1,-3.5657164E-2,-1.084805E-2,-3.527896E-2,-2.1987616E-1,-4.3792676E-2,1.620185E-1,1.7653E-2,1.5483013E-1,2.8585268E-2,1.3393722E-1,2.7100835E-2,5.3481084E-1,3.234221E-1,-1.3303648E-3,-1.2289183E-2,-1.5718589E-2,-4.67204E-3,-4.5882237E-1,-1.5191974E-1,-1.722551E-2,-1.8548296E-1,2.5469378E-2,-8.410147E-2,-1.570191E-2,-2.8182656E-1,-1.5511669E-1,-1.22367125E-2,8.163611E-2,1.496864E-2,-3.4209214E-2,5.8687735E-2,1.0346889E-1,2.8403756E-1,-1.0410258E-2,2.2450928E-1,-1.1402914E-1,1.6481452E-1,1.288396E-2,2.6547153E-2,3.452719E-1,1.7607781E-6,-2.6211008E-2,-1.6551636E-2,-3.4937381E-3,-1.0021942E-2,-1.1791556E-1,-2.436712E-1,-1.7299983E-1,4.7252914E-3,-3.3118988E-3,1.5898114E-3,-1.5640807E-1,-5.1962954E-1,-1.1670075E-2,-1.0324831E-1,4.9212575E-2,-7.652348E-2,-1.8593137E-3,6.5237954E-3,2.9202693E-3,-4.7695437E-3,4.409908E-3,1.5554334E-3,1.2942784E-1,-6.9406526E-3,2.284422E-1,2.0509264E-2,-8.926491E-2,6.608424E-2,4.4911373E-3,1.46068735E-2,-1.179317E-2,2.0932162E-3,2.719191E-1,8.654501E-2,2.3787469E-1,4.2143464E-1,2.3713515E-3,-6.8000252E-3,-8.125123E-3,-1.4755557E-2,-1.7253415E-3,-1.0929563E-2,1.8394064E-3,-8.196034E-3,-9.5975E-3,-1.0845036E-3,-3.4494333E-2,-1.1765717E-2,-2.1901745E-3,-6.131934E-3,3.7258025E-3,-6.751347E-3,-4.638656E-3,1.2452344E-3,4.353773E-4,7.076416E-3,1.3687554E-2,6.696257E-3,-5.7066293E-3,1.3677662E-3,8.669708E-3,5.1878305E-4,1.6615298E-2,6.830312E-3,5.432929E-3,-7.2833216E-3,1.2443155E-2,5.088376E-3,2.3790946E-2,1.3173607E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":23,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,39,41,-1,-1,-1,43,45,47,-1,49,-1,51,-1,53,55,-1,-1,57,59,61,63,65,67,69,-1,71,73,-1,-1,-1,-1,75,77,-1,79,-1,81,83,85,87,89,91,-1,93,95,97,99,101,103,105,107,-1,-1,109,-1,-1,-1,-1,-1,111,113,115,117,-1,-1,119,121,-1,123,125,127,-1,-1,-1,-1,-1,-1,129,-1,131,-1,133,135,-1,-1,-1,-1,137,139,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2973873E1,8.1876745E0,3.9524097E0,3.283619E0,2.1436684E0,3.2043285E0,8.1978124E-1,6.161938E-1,2.7245364E0,1.2337862E0,7.344538E-1,1.11055374E-1,2.565115E0,6.1719936E-1,0E0,2.5768447E-1,1.10394E0,1.9847956E0,8.950615E-1,9.812418E-1,2.5004232E-1,6.4798427E-1,0E0,0E0,0E0,1.1166453E0,8.68618E-1,1.4385143E-1,0E0,1.7964602E-1,0E0,6.662345E-1,0E0,3.8988972E-1,4.0674973E0,0E0,0E0,5.900626E-1,3.4939557E-1,2.576456E-1,6.159592E-2,7.072351E-1,5.0658774E-1,7.385075E-1,0E0,1.483221E-1,5.581894E-1,0E0,0E0,0E0,0E0,1.23408794E-1,4.2649835E-2,0E0,4.3913078E-1,0E0,1.0431623E0,3.582832E-2,9.924216E-1,8.403045E-2,3.1584567E-1,1.20457865E-1,0E0,9.3084924E-2,1.1451758E-2,5.514503E-1,1.712935E-1,3.4368807E-1,9.638673E-2,2.5678542E-1,6.8064046E-1,0E0,0E0,5.050478E-1,0E0,0E0,0E0,0E0,0E0,2.8340292E-1,2.6540327E-1,5.519643E-1,4.3130508E-1,0E0,0E0,1.629132E-1,5.259793E-1,0E0,1.9683823E-2,2.5985587E-1,9.720689E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.0362294E-1,0E0,9.7393274E-2,0E0,1.1575243E-1,1.9019336E-1,0E0,0E0,0E0,0E0,3.2342887E-1,3.902939E-1,9.878874E-2,3.817916E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,18,18,19,19,20,20,21,21,25,25,26,26,27,27,29,29,31,31,33,33,34,34,37,37,38,38,39,39,40,40,41,41,42,42,43,43,45,45,46,46,51,51,52,52,54,54,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,73,73,79,79,80,80,81,81,82,82,85,85,86,86,88,88,89,89,90,90,97,97,99,99,101,101,102,102,107,107,108,108,109,109,110,110],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,40,42,-1,-1,-1,44,46,48,-1,50,-1,52,-1,54,56,-1,-1,58,60,62,64,66,68,70,-1,72,74,-1,-1,-1,-1,76,78,-1,80,-1,82,84,86,88,90,92,-1,94,96,98,100,102,104,106,108,-1,-1,110,-1,-1,-1,-1,-1,112,114,116,118,-1,-1,120,122,-1,124,126,128,-1,-1,-1,-1,-1,-1,130,-1,132,-1,134,136,-1,-1,-1,-1,138,140,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,4.9236734E5,1E0,7.7E1,9.99E2,2.4481E4,7.25E2,6.48334E5,3.1E1,3.3486558E6,1.0046621E7,5.1191E4,1.7831801E6,5.071E3,-1.8402897E-2,2.0376764E-7,9.932432E0,1.5341322E2,6.3344407E0,1E0,2.1525E4,3.875E0,-1.0392984E-2,2.420904E-2,4.7639586E-2,5.428706E3,1.5215946E0,1.7234043E1,8.498472E-3,1.9E1,-2.5978703E-3,8.1643506E-8,-4.081178E-2,5.4E1,1.9722162E7,-1.084805E-2,-3.527896E-2,1.6386554E0,1.9875325E2,2.2266E4,9.356961E6,5.304868E2,4.86392E5,6.125E0,2.7100835E-2,5.97E3,4.704547E8,-1.3303648E-3,-1.2289183E-2,-1.5718589E-2,-4.67204E-3,2.512195E0,1.2909952E1,-1.722551E-2,8.032258E0,2.5469378E-2,1.339646E6,2.43E2,8.3E1,3.39E2,2.0886075E0,9E0,1.496864E-2,1.1057693E0,1.8333334E0,4.5620965E6,8.9473E4,6.946813E7,3.4187552E6,1.5204346E8,2.8E1,1.288396E-2,2.6547153E-2,1.1253407E3,1.7607781E-6,-2.6211008E-2,-1.6551636E-2,-3.4937381E-3,-1.0021942E-2,1.05E2,4.442623E0,1E0,4.07E2,-3.3118988E-3,1.5898114E-3,1.1925396E6,2.3285185E2,-1.1670075E-2,1.1E1,7.2744107E0,6.2136955E0,-1.8593137E-3,6.5237954E-3,2.9202693E-3,-4.7695437E-3,4.409908E-3,1.5554334E-3,1.071E3,-6.9406526E-3,1.737736E1,2.0509264E-2,1.9791039E6,9.912121E0,4.4911373E-3,1.46068735E-2,-1.179317E-2,2.0932162E-3,4.029152E8,6.947958E-1,2.1129E4,2.171312E0,2.3713515E-3,-6.8000252E-3,-8.125123E-3,-1.4755557E-2,-1.7253415E-3,-1.0929563E-2,1.8394064E-3,-8.196034E-3,-9.5975E-3,-1.0845036E-3,-3.4494333E-2,-1.1765717E-2,-2.1901745E-3,-6.131934E-3,3.7258025E-3,-6.751347E-3,-4.638656E-3,1.2452344E-3,4.353773E-4,7.076416E-3,1.3687554E-2,6.696257E-3,-5.7066293E-3,1.3677662E-3,8.669708E-3,5.1878305E-4,1.6615298E-2,6.830312E-3,5.432929E-3,-7.2833216E-3,1.2443155E-2,5.088376E-3,2.3790946E-2,1.3173607E-2],"split_indices":[2,37,6,38,2,12,0,40,3,37,56,1,37,2,0,46,63,61,62,109,9,63,0,0,0,61,63,65,0,3,0,46,0,2,40,0,0,63,64,12,54,64,1,65,0,2,41,0,0,0,0,63,65,0,67,0,9,0,0,2,62,3,0,63,62,56,1,7,54,7,8,0,0,64,0,0,0,0,0,38,63,8,0,0,0,37,61,0,3,65,63,0,0,0,0,0,0,2,0,67,0,37,67,0,0,0,0,7,43,38,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.79E2,7.37E2,2.42E2,3.63E2,3.74E2,2.17E2,2.5E1,7.6E1,2.87E2,1.93E2,1.81E2,1E1,2.07E2,1.9E1,6E0,3.1E1,4.5E1,2.72E2,1.5E1,1.45E2,4.8E1,1.75E2,6E0,5E0,5E0,9.8E1,1.09E2,1E1,9E0,2.5E1,6E0,4E1,5E0,1.31E2,1.41E2,7E0,8E0,4.6E1,9.9E1,2.1E1,2.7E1,1.1E2,6.5E1,9.3E1,5E0,3.1E1,7.8E1,5E0,5E0,2E1,5E0,2.9E1,1.1E1,1.3E1,1.18E2,1E1,1.31E2,1.1E1,3.5E1,2.1E1,7.8E1,1.5E1,6E0,1.2E1,1.5E1,8E1,3E1,5.5E1,1E1,1E1,8.3E1,5E0,2.6E1,7.3E1,5E0,1.2E1,1.7E1,6E0,5E0,5.6E1,6.2E1,6.5E1,6.6E1,5E0,6E0,2.4E1,1.1E1,6E0,1.5E1,4E1,3.8E1,5E0,1E1,5E0,7E0,5E0,1E1,7.3E1,7E0,2.4E1,6E0,2.7E1,2.8E1,5E0,5E0,5E0,5E0,3.4E1,4.9E1,3.2E1,4.1E1,8E0,4.8E1,3.4E1,2.8E1,2.1E1,4.4E1,5.6E1,1E1,1.7E1,7E0,5E0,6E0,6E0,9E0,3.5E1,5E0,3.1E1,7E0,1.2E1,6.1E1,1.2E1,1.2E1,2.1E1,6E0,8E0,2E1,1.9E1,1.5E1,4.4E1,5E0,2.5E1,7E0,2.3E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[-3.959029E-3,-9.358908E-2,2.4001133E-1,-1.9086102E-1,4.2965945E-2,2.6811442E-1,-2.022288E-2,-3.4927875E-1,-1.4760217E-1,-3.0282652E-2,9.278045E-2,7.462038E-1,2.3737288E-1,-3.06624E-1,-5.028631E-1,-2.3316815E-1,-6.852697E-2,-1.8747844E-1,-6.713992E-3,6.208219E-2,2.3281983E-1,2.5368968E-2,4.4276915E-2,9.0968E-2,2.929052E-1,-3.4756675E-1,-4.425868E-2,-3.322357E-2,-3.6579454E-1,-2.0560504E-1,-5.5597943E-1,2.3339376E-2,-9.179462E-2,9.74926E-4,-1.633129E-2,-6.6840306E-2,5.4650106E-2,1.5370685E-2,1.3307652E-1,2.371815E-3,2.7278513E-1,1.7941846E-1,-5.881422E-2,3.5040268E-1,1.2886097E-1,-3.1101322E-1,-4.438038E-1,-7.1014557E-3,3.336566E-3,-9.400782E-3,-2.0529967E-2,-1.8074018E-1,-2.006604E-2,-5.922764E-3,-3.914697E-2,-6.642539E-2,-2.4843287E-2,6.015405E-2,-1.2048297E-1,9.915338E-2,-9.333354E-2,4.2768538E-2,-1.1089749E-2,-3.3805005E-3,1.5776435E-1,3.4090194E-1,6.018375E-3,2.7371114E-1,9.336546E-2,-1.2694064E-2,-9.763937E-3,3.9393902E-1,1.0774264E-1,1.9521612E-1,-4.149163E-2,-1.5118593E-2,-9.016206E-3,-2.3334647E-2,-1.2589499E-2,-2.5371462E-1,-1.10359155E-1,-2.985861E-2,-2.9610375E-1,5.0321603E-3,7.191621E-4,2.7154107E-3,-1.4852856E-1,6.408563E-2,1.0236381E-2,-1.2107866E-3,-7.768629E-3,-3.6813878E-2,8.179603E-2,2.005421E-1,1.0714109E-2,1.0364274E-2,1.859858E-2,7.052095E-3,1.5877651E-2,3.8410943E-2,1.1684631E-2,-4.5065167E-3,4.673421E-2,4.3498868E-1,1.7659907E-1,-1.7369498E-2,1.0349851E-2,4.8711464E-2,2.7291358E-1,3.8004091E-3,-1.01927165E-2,-1.9256694E-2,-1.0702812E-2,7.078034E-3,-6.355519E-3,-5.2113854E-3,1.4352851E-4,-1.5704218E-2,-7.2844713E-3,-8.370531E-3,-1.8535367E-3,7.1651335E-3,1.8194376E-3,3.085058E-3,-4.140977E-3,1.0644611E-3,6.688971E-3,1.11785615E-2,3.8457988E-3,7.6427353E-3,-4.924218E-3,4.1010557E-3,-1.9187527E-3,4.5530518E-4,3.5717965E-3,1.7279133E-2,3.0278731E-2,1.23882145E-2,-7.655781E-4,6.3539185E-3,-7.8314645E-3,5.014781E-3,-1.7893119E-3,1.5093403E-2,7.475441E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":24,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,43,45,47,-1,49,51,53,-1,55,-1,-1,57,59,61,63,-1,65,67,69,71,73,75,77,-1,-1,-1,-1,79,-1,-1,-1,81,-1,83,85,87,89,91,-1,-1,93,95,-1,97,99,101,-1,103,105,107,109,-1,-1,-1,-1,111,113,115,117,-1,-1,-1,119,121,-1,-1,-1,123,125,127,129,-1,-1,-1,-1,131,-1,-1,133,135,137,139,-1,141,143,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.1649E1,9.6250725E0,5.169158E0,2.847252E0,1.1047602E0,3.5853577E0,0E0,4.3540955E-1,2.243915E0,4.5444325E-1,7.593597E-1,1.0070515E-1,1.9408636E0,7.6878023E-1,2.6402187E-1,1.3093176E0,2.3733816E0,5.561198E-1,4.0212932E-1,4.9207604E-1,2.1843314E-1,0E0,0E0,9.114372E-1,1.6054697E0,8.359289E-2,1.5002577E-1,0E0,7.300115E-2,7.6231575E-1,1.3711603E0,0E0,1.8664452E0,0E0,0E0,3.819249E-1,3.6488515E-1,6.444358E-1,3.114525E-1,0E0,2.0917737E-1,3.23354E-1,1.7913778E-1,1.3203793E0,5.3937703E-1,2.1442413E-2,1.645708E-2,0E0,0E0,0E0,0E0,6.704979E-1,0E0,0E0,0E0,1.3433328E0,0E0,3.4984287E-2,2.0448452E-1,1.6475523E-1,6.0398422E-2,2.6080108E-1,0E0,0E0,3.3124685E-1,3.0215979E-2,0E0,1.147145E-1,1.9932358E-1,1.0990763E-1,0E0,8.9565086E-1,3.06854E-1,3.7372935E-1,3.2581472E-1,0E0,0E0,0E0,0E0,1.4963722E-1,5.176972E-1,3.8074476E-1,7.648265E-2,0E0,0E0,0E0,1.10461295E-1,7.034168E-2,0E0,0E0,0E0,1.5759404E-1,2.0223847E-1,1.7481422E-1,2.5139946E-1,0E0,0E0,0E0,0E0,7.939287E-2,0E0,0E0,1.3761664E-2,9.9621964E-1,3.360414E-1,2.8012383E-1,0E0,7.152408E-2,7.64277E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,32,32,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,45,45,46,46,51,51,55,55,57,57,58,58,59,59,60,60,61,61,64,64,65,65,67,67,68,68,69,69,71,71,72,72,73,73,74,74,79,79,80,80,81,81,82,82,86,86,87,87,91,91,92,92,93,93,94,94,99,99,102,102,103,103,104,104,105,105,107,107,108,108],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,44,46,48,-1,50,52,54,-1,56,-1,-1,58,60,62,64,-1,66,68,70,72,74,76,78,-1,-1,-1,-1,80,-1,-1,-1,82,-1,84,86,88,90,92,-1,-1,94,96,-1,98,100,102,-1,104,106,108,110,-1,-1,-1,-1,112,114,116,118,-1,-1,-1,120,122,-1,-1,-1,124,126,128,130,-1,-1,-1,-1,132,-1,-1,134,136,138,140,-1,142,144,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,7.502167E5,9.7172376E7,7.7E1,8.1E2,2.4481E4,-2.022288E-2,1.4504054E8,6.34E2,5.8899284E1,3.7951445E6,6.23694E5,7.079871E2,2.0376764E-7,1.0252778E1,9.639872E0,2.04115E5,2.28E0,2.732381E2,2.437372E6,5.1942E4,2.5368968E-2,4.4276915E-2,1.1590965E0,9.3E1,6.711294E7,1.8257E4,-3.322357E-2,3.583621E1,2.4E1,2.49E2,2.3339376E-2,3E1,9.74926E-4,-1.633129E-2,5.457143E0,2.4624279E0,2.511E3,1.6964E4,2.371815E-3,4.76E2,8E0,1.8E1,2E0,4.24038E8,2.8096385E0,1.4473684E-1,-7.1014557E-3,3.336566E-3,-9.400782E-3,-2.0529967E-2,3.06E2,-2.006604E-2,-5.922764E-3,-3.914697E-2,2.0734E4,-2.4843287E-2,3.54E2,1.684E0,1.623E3,9E0,1E0,-1.1089749E-2,-3.3805005E-3,1E0,2.47E2,6.018375E-3,5.45901E6,5.306E3,9.8E1,-9.763937E-3,2.395631E7,4.797448E8,1.185E4,6.619098E8,-1.5118593E-2,-9.016206E-3,-2.3334647E-2,-1.2589499E-2,3.62E2,1.424196E-1,2.7005264E2,4.6695483E2,5.0321603E-3,7.191621E-4,2.7154107E-3,2.2646774E2,9.25E0,1.0236381E-2,-1.2107866E-3,-7.768629E-3,8.433083E6,5.4007E7,2.23099E7,8.6E1,1.0364274E-2,1.859858E-2,7.052095E-3,1.5877651E-2,2.88721E5,1.1684631E-2,-4.5065167E-3,4.3616666E2,6.7947706E8,4.9321495E6,2.0045958E6,1.0349851E-2,6.849626E1,2.3876712E7,3.8004091E-3,-1.01927165E-2,-1.9256694E-2,-1.0702812E-2,7.078034E-3,-6.355519E-3,-5.2113854E-3,1.4352851E-4,-1.5704218E-2,-7.2844713E-3,-8.370531E-3,-1.8535367E-3,7.1651335E-3,1.8194376E-3,3.085058E-3,-4.140977E-3,1.0644611E-3,6.688971E-3,1.11785615E-2,3.8457988E-3,7.6427353E-3,-4.924218E-3,4.1010557E-3,-1.9187527E-3,4.5530518E-4,3.5717965E-3,1.7279133E-2,3.0278731E-2,1.23882145E-2,-7.655781E-4,6.3539185E-3,-7.8314645E-3,5.014781E-3,-1.7893119E-3,1.5093403E-2,7.475441E-3],"split_indices":[2,37,54,38,2,12,0,40,2,56,37,1,61,46,67,63,5,65,4,9,1,0,0,47,8,7,1,0,65,8,2,0,3,0,0,67,62,2,1,0,0,3,3,6,41,62,65,0,0,0,0,2,0,0,0,38,0,42,63,38,8,109,0,0,85,0,0,54,2,0,0,56,7,2,7,0,0,0,0,1,67,64,64,0,0,0,61,67,0,0,0,54,7,9,10,0,0,0,0,9,0,0,61,7,37,37,0,67,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.88E2,7.23E2,2.65E2,4.22E2,3.01E2,2.55E2,1E1,8.9E1,3.33E2,1.22E2,1.79E2,1.4E1,2.41E2,7.2E1,1.7E1,1.59E2,1.74E2,1.5E1,1.07E2,1.48E2,3.1E1,9E0,5E0,6.7E1,1.74E2,6.2E1,1E1,5E0,1.2E1,1.48E2,1.1E1,6E0,1.68E2,7E0,8E0,5.4E1,5.3E1,9E1,5.8E1,6E0,2.5E1,4.2E1,2.5E1,1.28E2,4.6E1,4.8E1,1.4E1,5E0,5E0,5E0,7E0,1.35E2,1.3E1,5E0,6E0,1.6E2,8E0,1.6E1,3.8E1,4.1E1,1.2E1,8.2E1,8E0,6E0,5.2E1,1.6E1,9E0,1.9E1,2.3E1,2E1,5E0,1.08E2,2E1,3.3E1,1.3E1,4.1E1,7E0,9E0,5E0,6.5E1,7E1,1.39E2,2.1E1,7E0,9E0,5E0,3.3E1,3.3E1,8E0,7E0,5E0,2.7E1,5.5E1,4E1,1.2E1,7E0,9E0,8E0,1.1E1,1.8E1,5E0,8E0,1.2E1,9E1,1.8E1,1E1,1E1,1.2E1,2.1E1,8E0,5E0,6E0,5.9E1,6E0,6.4E1,3.9E1,1E2,1.5E1,6E0,2.5E1,8E0,6E0,2.7E1,9E0,1.8E1,2.9E1,2.6E1,2.9E1,1.1E1,5E0,7E0,1.1E1,7E0,6E0,6E0,7.2E1,1.8E1,1.2E1,6E0,5E0,5E0,7E0,5E0,1.3E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"145","size_leaf_vector":"1"}},{"base_weights":[3.9238E-3,-8.301332E-2,2.500506E-1,-2.1204793E-1,-1.9849526E-2,-3.530646E-1,2.829097E-1,-1.8391009E-1,-4.523989E-1,-9.914624E-2,3.4721125E-2,-2.5386294E-2,-7.5810077E-3,2.5132906E-1,4.1373145E-2,-2.7096677E-1,-1.15531184E-1,-3.1260994E-1,-3.6950905E-2,-2.2341049E-1,-4.6475016E-2,6.790334E-2,-8.791043E-2,1.8170032E-1,3.8719022E-1,-3.0642447E-1,-9.003943E-2,1.6044812E-1,-1.5192127E-1,-7.3544984E-3,-2.0838166E-2,-1.7391631E-1,-2.8158681E-2,-1.020095E-1,1.2287927E-2,4.360008E-2,2.1794033E-1,-3.2973967E-2,-2.468591E-1,6.623976E-2,2.5537223E-1,2.6785874E-1,4.9928227E-1,-3.2891348E-1,-6.013518E-3,-8.950673E-3,-4.1897057E-4,-4.131732E-3,1.4335128E-2,-1.6737051E-1,2.4340989E-3,2.4132642E-3,-2.2356759E-1,-3.3916224E-2,-1.5351929E-1,-8.2116164E-2,7.975015E-2,5.947731E-2,-1.4348248E-1,1.4230487E-1,3.1953478E-1,6.141495E-3,-9.959662E-3,-1.6114987E-2,-1.7769763E-3,1.12130195E-1,-7.584754E-2,3.145845E-1,6.920678E-2,3.11195E-1,-7.021928E-4,1.2130493E-2,5.282758E-1,-2.9389912E-1,-2.020569E-2,-2.0566736E-1,-5.6353305E-2,-1.6460712E-1,-2.2960959E-2,-1.2343222E-1,1.4588163E-2,-8.7927796E-2,-2.1143483E-1,-1.182834E-2,-1.2374171E-2,3.2717526E-2,1.5449917E-2,4.129028E-2,1.7708391E-1,1.6750484E-3,-1.6419709E-2,1.1422847E-2,1.1225884E-3,7.872033E-3,1.8668925E-2,-9.696443E-2,4.642805E-2,1.9346668E-1,5.0740447E-2,-1.008145E-2,7.699497E-4,3.8742265E-1,1.7380597E-1,1.3101017E-1,-2.9255212E-3,3.704509E-1,3.6986738E-3,5.611517E-1,1.3562E-2,-1.597681E-2,-9.421559E-3,-1.0911865E-2,-5.506244E-3,1.3515813E-3,-5.613515E-3,-3.5356495E-3,-9.541364E-3,-7.5949575E-3,-2.9048754E-3,-1.2411191E-3,4.690187E-3,6.9613213E-4,-5.718183E-3,-1.1983527E-2,-4.062804E-3,-3.2822401E-3,2.7329645E-3,4.8602475E-3,-1.7227607E-3,3.7467468E-3,-2.3846433E-3,1.204102E-2,2.0407236E-3,-6.573695E-3,-1.6746387E-3,3.604465E-3,-2.7113825E-3,1.6823784E-3,1.14616975E-2,5.2398555E-3,4.3034084E-5,2.3248427E-2,1.5272574E-2,1.1964264E-2,1.0184108E-3,-1.6161522E-3,1.2243323E-2,1.1750721E-2,2.172278E-2,1.5657334E-2,2.7684445E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":25,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,-1,-1,51,-1,53,55,57,59,61,63,65,67,69,71,73,-1,-1,-1,-1,-1,75,-1,-1,77,79,81,83,85,87,89,91,93,95,-1,-1,-1,97,99,101,103,105,-1,-1,107,109,-1,111,113,115,-1,117,119,121,123,-1,125,127,-1,129,131,-1,-1,-1,-1,-1,-1,133,135,137,139,-1,-1,141,143,145,-1,147,-1,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9920485E1,5.603208E0,4.915613E0,1.4452276E0,2.0069702E0,3.891269E-1,4.2169075E0,1.1830745E0,8.5695267E-1,1.2266502E0,1.1249957E0,0E0,0E0,2.0202723E0,0E0,5.4336834E-1,1.1893533E0,3.0743694E-1,0E0,9.572954E-1,4.38545E-1,7.8176403E-1,5.0705266E-1,1.2420015E0,8.560953E-1,2.5163507E-1,1.315879E-1,5.462207E-1,3.3400536E-1,0E0,0E0,5.8372366E-1,0E0,2.3680866E-1,4.2688364E-1,5.6696033E-1,1.8505287E-1,3.2110482E-1,3.009528E-1,3.9321128E-1,9.7032356E-1,4.7073507E-1,1.1704159E-1,1.1453724E-1,0E0,0E0,0E0,0E0,0E0,3.9656496E-1,0E0,0E0,5.903423E-1,1.3640012E-1,1.2956089E-1,3.3430123E-1,4.5359218E-1,3.6485934E-1,5.7952636E-1,2.2716233E-1,8.0687284E-2,1.623456E-1,0E0,0E0,0E0,2.1487856E-1,1.9971254E-1,6.299386E-1,1.9626157E-1,4.2311883E-1,0E0,0E0,8.424759E-2,1.6832685E-1,0E0,1.5433836E-1,1.4716555E-1,1.1490941E-1,0E0,1.6686007E-2,7.778751E-2,7.4539706E-2,9.341556E-2,0E0,9.1525406E-2,1.7494583E-1,0E0,5.6005657E-1,2.4091572E-1,0E0,0E0,0E0,0E0,0E0,0E0,2.4825655E-2,9.631108E-2,1.5101206E-1,8.353628E-2,0E0,0E0,1.2639523E-1,3.1342226E-1,3.6172467E-1,0E0,1.7920113E-1,0E0,1.3102531E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,49,49,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,65,65,66,66,67,67,68,68,69,69,72,72,73,73,75,75,76,76,77,77,79,79,80,80,81,81,82,82,84,84,85,85,87,87,88,88,95,95,96,96,97,97,98,98,101,101,102,102,103,103,105,105,107,107],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,-1,-1,52,-1,54,56,58,60,62,64,66,68,70,72,74,-1,-1,-1,-1,-1,76,-1,-1,78,80,82,84,86,88,90,92,94,96,-1,-1,-1,98,100,102,104,106,-1,-1,108,110,-1,112,114,116,-1,118,120,122,124,-1,126,128,-1,130,132,-1,-1,-1,-1,-1,-1,134,136,138,140,-1,-1,142,144,146,-1,148,-1,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,2.2116136E5,1E0,5.119403E0,7.9E2,2.5330253E-2,7.626E3,8.7601E4,3.9401392E6,1E0,1.0292E4,-2.5386294E-2,-7.5810077E-3,3.422592E6,4.1373145E-2,2.2292225E-1,1.89991E5,1.5E1,-3.6950905E-2,3.7666667E0,9.4996644E5,1.5958088E10,5.27E2,8.629636E2,7.459E3,3.3057997E0,8.085012E4,6.9E2,2.041E3,-7.3544984E-3,-2.0838166E-2,1E0,-2.8158681E-2,2.0804598E0,2.6818182E0,8.387763E2,2.5E0,9.639872E0,1.4283589E6,2.0926375E5,1.27107956E2,2E1,4.5478998E8,3.8795E4,-6.013518E-3,-8.950673E-3,-4.1897057E-4,-4.131732E-3,1.4335128E-2,7.997723E6,2.4340989E-3,2.4132642E-3,1.4390917E8,1.766298E2,1E0,1.2541E4,1.6100003E8,5.8222644E2,1.7E1,1.103E3,2.53E2,2.87E2,-9.959662E-3,-1.6114987E-2,-1.7769763E-3,2E0,1.8591549E0,6.45365E5,4.24038E8,1.4702564E11,-7.021928E-4,1.2130493E-2,2.395631E7,3.23E2,-2.020569E-2,9.558022E7,2.2941177E0,1E0,-2.2960959E-2,1.2380953E0,7.513552E5,4.8421054E0,4.330197E5,-1.182834E-2,2.8080197E2,2.824856E-3,1.5449917E-2,1.9151516E1,2.4E0,1.6750484E-3,-1.6419709E-2,1.1422847E-2,1.1225884E-3,7.872033E-3,1.8668925E-2,2.2503355E1,4.138602E7,3.257507E5,3.754E3,-1.008145E-2,7.699497E-4,4.4889745E6,1.2800076E3,2.338255E3,-2.9255212E-3,1.1E1,3.6986738E-3,1.5466666E1,1.3562E-2,-1.597681E-2,-9.421559E-3,-1.0911865E-2,-5.506244E-3,1.3515813E-3,-5.613515E-3,-3.5356495E-3,-9.541364E-3,-7.5949575E-3,-2.9048754E-3,-1.2411191E-3,4.690187E-3,6.9613213E-4,-5.718183E-3,-1.1983527E-2,-4.062804E-3,-3.2822401E-3,2.7329645E-3,4.8602475E-3,-1.7227607E-3,3.7467468E-3,-2.3846433E-3,1.204102E-2,2.0407236E-3,-6.573695E-3,-1.6746387E-3,3.604465E-3,-2.7113825E-3,1.6823784E-3,1.14616975E-2,5.2398555E-3,4.3034084E-5,2.3248427E-2,1.5272574E-2,1.1964264E-2,1.0184108E-3,-1.6161522E-3,1.2243323E-2,1.1750721E-2,2.172278E-2,1.5657334E-2,2.7684445E-2],"split_indices":[2,37,16,62,2,47,61,9,54,109,38,0,0,37,0,51,9,8,0,62,37,40,0,61,2,62,56,2,2,0,0,62,0,62,63,4,63,63,37,42,67,3,7,1,0,0,0,0,0,9,0,0,5,4,23,56,5,61,3,10,0,0,0,0,0,26,62,9,41,40,0,0,56,1,0,7,62,6,0,65,56,67,37,0,4,47,0,67,62,0,0,0,0,0,0,65,12,37,2,0,0,54,61,61,0,3,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.29E2,6.87E2,2.42E2,2.25E2,4.62E2,1.2E1,2.3E2,2.03E2,2.2E1,1.88E2,2.74E2,5E0,7E0,2.2E2,1E1,8.8E1,1.15E2,1.7E1,5E0,5.5E1,1.33E2,2.16E2,5.8E1,1.47E2,7.3E1,7.3E1,1.5E1,1.3E1,1.02E2,9E0,8E0,5E1,5E0,6.8E1,6.5E1,1.87E2,2.9E1,4.4E1,1.4E1,5.8E1,8.9E1,3.7E1,3.6E1,6.4E1,9E0,6E0,9E0,5E0,8E0,9.5E1,7E0,9E0,4.1E1,3E1,3.8E1,2.7E1,3.8E1,1.73E2,1.4E1,1.8E1,1.1E1,3.7E1,7E0,9E0,5E0,4.4E1,1.4E1,6.7E1,2.2E1,3.2E1,5E0,5E0,3.1E1,5.1E1,1.3E1,7E1,2.5E1,3.5E1,6E0,1E1,2E1,1.9E1,1.9E1,7E0,2E1,3.3E1,5E0,1.51E2,2.2E1,8E0,6E0,9E0,9E0,5E0,6E0,1E1,2.7E1,1.8E1,2.6E1,5E0,9E0,4.3E1,2.4E1,1.5E1,7E0,2.5E1,7E0,2.6E1,5E0,3.1E1,2E1,5.1E1,1.9E1,1.1E1,1.4E1,1.2E1,2.3E1,5E0,5E0,1.4E1,6E0,5E0,1.4E1,1.3E1,6E0,1.1E1,9E0,1.6E1,1.7E1,1.06E2,4.5E1,1.3E1,9E0,5E0,5E0,2.1E1,6E0,5E0,1.3E1,1.1E1,1.5E1,1.2E1,3.1E1,1.5E1,9E0,7E0,8E0,1.3E1,1.2E1,5E0,2.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[-2.7694313E-3,-1.2922835E-1,1.21609285E-1,-2.4146065E-1,-6.910083E-2,-3.5286647E-1,1.5035102E-1,-2.0762663E-1,-5.081167E-1,-6.3339868E-3,-1.5692568E-1,-3.0102128E-2,-2.4267879E-1,7.865366E-2,3.077357E-1,-3.108641E-1,-1.628146E-1,-1.4104688E-2,-4.0168777E-2,2.3746893E-1,-3.111432E-2,-1.2003124E-1,-5.4514444E-1,-5.180237E-3,-2.9638517E-1,1.0762945E-1,-1.07811905E-1,3.7538633E-1,8.8259906E-2,-7.765377E-3,-3.2497087E-1,-1.3969794E-1,-1.4729101E-2,1.787984E-2,2.932723E-3,-1.1795563E-2,-2.796653E-1,-8.876889E-2,-3.0422255E-1,-3.5273463E-2,-1.1099563E-2,-1.5619844E-2,-8.347165E-3,8.693591E-2,3.102597E-1,-4.4128336E-2,-1.7674066E-2,4.2971814E-1,1.5354922E-1,1.9381705E-1,-1.596663E-1,-9.494256E-3,-1.6003301E-2,-1.631265E-1,-1.528829E-2,3.316316E-2,-6.470801E-2,-2.3586636E-2,1.5651367E-3,-1.0968748E-1,1.2582705E-2,-1.7838094E-2,-4.318423E-3,1.5983695E-2,7.588867E-2,6.4522987E-3,3.691118E-1,-8.194496E-3,3.035473E-5,3.575711E-1,5.943399E-1,9.863131E-4,2.100985E-1,6.724878E-2,3.2532862E-1,-3.2608744E-3,-1.0321232E-2,-1.8800935E-1,-9.436784E-2,2.7651386E-3,-3.718999E-3,-5.926716E-3,1.2209506E-1,1.2616161E-2,-1.1715026E-1,-1.2626559E-1,5.857194E-3,-1.6587254E-2,1.05958275E-1,8.949713E-3,2.151641E-2,5.3895418E-2,-5.552763E-3,2.5133017E-1,4.8991713E-1,3.169822E-2,1.8225394E-2,1.107316E-2,5.6950613E-3,7.1618985E-3,-1.9444207E-3,8.0400165E-3,1.8985791E-2,-6.9413637E-3,-1.2199234E-2,3.3323487E-4,-6.061218E-3,-1.5123136E-3,3.8984981E-3,6.846864E-3,2.1503964E-3,-1.4333556E-3,8.899227E-3,-9.732977E-3,-2.5286886E-3,-9.554807E-3,-4.359254E-3,1.3299388E-3,-4.2050434E-3,6.255941E-3,3.0231607E-4,-2.0591703E-3,4.6020355E-3,2.3010954E-2,8.246348E-3,2.4777725E-2,1.4775718E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":26,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,-1,-1,33,35,37,39,-1,41,43,45,47,49,-1,51,53,-1,-1,-1,55,57,59,61,-1,-1,-1,-1,63,65,67,-1,69,71,73,75,-1,-1,77,79,81,83,-1,-1,85,-1,-1,-1,-1,87,-1,89,-1,91,93,95,-1,97,99,101,-1,-1,103,105,-1,-1,107,109,111,113,115,-1,117,119,-1,-1,121,-1,123,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5131091E1,3.202231E0,6.667122E0,1.3921604E0,1.7189023E0,7.5547004E-1,5.143897E0,6.3870287E-1,1.0507984E0,1.1129863E0,1.7965698E0,0E0,1.2173271E-1,1.7189476E0,2.090085E0,3.468609E-2,3.345468E-1,0E0,0E0,4.1372615E-1,7.985844E-1,6.639848E-1,5.41893E-1,0E0,9.774923E-3,1.121947E0,7.310451E-1,1.2477732E0,9.469093E-1,0E0,2.298212E-2,2.7149153E-1,0E0,0E0,0E0,3.7325054E-1,8.7713623E-1,8.128311E-1,2.5219023E-1,0E0,0E0,0E0,0E0,6.8514717E-1,2.0055795E-1,2.1060964E-1,0E0,8.207798E-1,1.6759759E-1,3.9425188E-1,4.6093494E-2,0E0,0E0,1.1741972E-1,8.249101E-2,2.9621232E-1,2.9344192E-1,0E0,0E0,4.031415E-1,0E0,0E0,0E0,0E0,6.7190945E-1,0E0,2.1066713E-1,0E0,1.8698928E-1,7.626848E-1,1.7919254E-1,0E0,1.0613978E-2,1.395787E-1,8.209634E-2,0E0,0E0,1.24666214E-1,8.5636854E-2,0E0,0E0,1.4665397E-1,4.3321222E-2,2.3979694E-1,2.3807573E-1,2.2005355E-1,0E0,2.0361623E-1,5.216234E-1,0E0,0E0,9.341423E-2,0E0,5.7800436E-1,3.4081936E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,30,30,31,31,35,35,36,36,37,37,38,38,43,43,44,44,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,59,59,64,64,66,66,68,68,69,69,70,70,72,72,73,73,74,74,77,77,78,78,81,81,82,82,83,83,84,84,85,85,87,87,88,88,91,91,93,93,94,94],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,-1,-1,34,36,38,40,-1,42,44,46,48,50,-1,52,54,-1,-1,-1,56,58,60,62,-1,-1,-1,-1,64,66,68,-1,70,72,74,76,-1,-1,78,80,82,84,-1,-1,86,-1,-1,-1,-1,88,-1,90,-1,92,94,96,-1,98,100,102,-1,-1,104,106,-1,-1,108,110,112,114,116,-1,118,120,-1,-1,122,-1,124,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,4.48E2,1E0,3.4736134E5,3.962617E0,1.0479E4,1.0879832E3,4.54E2,4.2E1,6E0,9.932432E0,-3.0102128E-2,1E0,7.382199E0,2.2040408E7,2E0,2.05E1,-1.4104688E-2,-4.0168777E-2,1.6916667E2,2.829581E10,1.7899E4,3.284E4,-5.180237E-3,9.3326636E-2,6.0052995E6,4E1,8.7E1,4.1831533E8,-7.765377E-3,1.957E3,4.03E2,-1.4729101E-2,1.787984E-2,2.932723E-3,1E0,2.2715E4,2.4067578E10,6.176636E5,-3.5273463E-2,-1.1099563E-2,-1.5619844E-2,-8.347165E-3,3.5E1,1E0,5.6842103E0,-1.7674066E-2,1.9186024E3,1.9125667E3,2.1129E4,3.433001E-2,-9.494256E-3,-1.6003301E-2,1.8E1,1.319909E3,3.3316666E2,2.978142E0,-2.3586636E-2,1.5651367E-3,1.3205625E7,1.2582705E-2,-1.7838094E-2,-4.318423E-3,1.5983695E-2,4.7567694E5,6.4522987E-3,1.99E2,-8.194496E-3,2.51E2,2.6878237E8,1.4497429E7,9.863131E-4,1.4127854E7,3.19E2,1.4669823E5,-3.2608744E-3,-1.0321232E-2,4.64E2,7.67525E2,2.7651386E-3,-3.718999E-3,4.436827E7,9.39E2,3.373828E-1,9.44363E5,2.8266037E2,5.857194E-3,3.82E2,1.23E4,8.949713E-3,2.151641E-2,4.162742E5,-5.552763E-3,1.281E3,1.6063418E7,3.169822E-2,1.8225394E-2,1.107316E-2,5.6950613E-3,7.1618985E-3,-1.9444207E-3,8.0400165E-3,1.8985791E-2,-6.9413637E-3,-1.2199234E-2,3.3323487E-4,-6.061218E-3,-1.5123136E-3,3.8984981E-3,6.846864E-3,2.1503964E-3,-1.4333556E-3,8.899227E-3,-9.732977E-3,-2.5286886E-3,-9.554807E-3,-4.359254E-3,1.3299388E-3,-4.2050434E-3,6.255941E-3,3.0231607E-4,-2.0591703E-3,4.6020355E-3,2.3010954E-2,8.246348E-3,2.4777725E-2,1.4775718E-2],"split_indices":[2,2,16,56,65,9,61,1,6,3,63,0,15,63,60,26,67,0,0,61,40,38,1,0,47,37,3,8,41,0,9,2,0,0,0,23,38,40,37,0,0,0,0,0,109,65,0,61,4,38,66,0,0,3,42,64,63,0,0,9,0,0,0,0,37,0,0,0,8,12,56,0,60,0,42,0,0,38,42,0,0,5,2,47,9,4,0,0,10,0,0,56,0,38,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.6E2,4.76E2,4.84E2,1.65E2,3.11E2,2.7E1,4.57E2,1.48E2,1.7E1,1.82E2,1.29E2,6E0,2.1E1,3.15E2,1.42E2,4.3E1,1.05E2,1.2E1,5E0,1.6E1,1.66E2,1.19E2,1E1,7E0,1.4E1,2.73E2,4.2E1,1.08E2,3.4E1,5E0,3.8E1,9.3E1,1.2E1,8E0,8E0,1.55E2,1.1E1,1.03E2,1.6E1,5E0,5E0,9E0,5E0,2.49E2,2.4E1,3.5E1,7E0,8.6E1,2.2E1,2.4E1,1E1,7E0,3.1E1,7.8E1,1.5E1,8.4E1,7.1E1,6E0,5E0,9.8E1,5E0,1.1E1,5E0,9E0,2.4E2,7E0,1.7E1,8E0,2.7E1,6.2E1,2.4E1,7E0,1.5E1,1.3E1,1.1E1,5E0,5E0,5.6E1,2.2E1,7E0,8E0,5.9E1,2.5E1,2.9E1,4.2E1,9.2E1,6E0,5.9E1,1.81E2,7E0,1E1,1.9E1,8E0,3.6E1,2.6E1,1.5E1,9E0,1E1,5E0,7E0,6E0,5E0,6E0,3.9E1,1.7E1,6E0,1.6E1,4.6E1,1.3E1,1.8E1,7E0,2.4E1,5E0,1.6E1,2.6E1,2.5E1,6.7E1,3.7E1,2.2E1,1.4E2,4.1E1,6E0,1.3E1,7E0,2.9E1,1.9E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[3.984298E-3,-6.4204425E-2,2.2498202E-1,-2.0061694E-1,-1.3872215E-3,2.563943E-1,-2.3816508E-1,-3.1335938E-1,-1.4404505E-1,-4.3814123E-2,1.008453E-1,2.9762141E-2,2.2830002E-1,-1.7407397E-2,-3.3126983E-3,-2.3498829E-1,-4.220717E-1,-1.660555E-1,1.096673E-2,-7.20371E-2,9.5243E-2,1.7605451E-1,2.628837E-2,1.4403759E-1,3.3930168E-1,-2.8341812E-1,-1.3285221E-1,-5.295664E-1,-2.077412E-1,3.2708634E-2,-2.0447989E-1,-1.2244752E-1,2.507833E-2,5.2081723E-2,2.506014E-1,2.0381877E-1,-3.6413353E-3,5.855329E-2,-1.0882585E-1,1.9004837E-2,1.20574355E-1,2.8964636E-1,4.79204E-1,-1.4444468E-2,-8.98851E-3,-1.781446E-1,-9.338058E-4,-1.446363E-2,-5.814005E-1,-4.00843E-3,-1.30883185E-2,-1.0086646E-2,1.6318838E-1,-1.7686778E-1,-1.9894369E-2,-1.073159E-1,-2.7201718E-2,4.545182E-2,-1.2102014E-2,2.0314967E-2,8.862754E-3,4.845134E-3,1.5261253E-2,1.4471993E-1,3.3546507E-1,-1.4716034E-2,1.118788E-1,-7.315577E-3,-4.3100782E-4,5.101046E-3,1.5712564E-1,3.9759595E-3,3.1630087E-1,1.1358764E-2,2.4923611E-2,-3.0461291E-3,-1.1558981E-2,-3.226373E-2,-1.8317465E-2,2.1476662E-2,-2.4898848E-4,-5.886185E-2,-2.0972332E-1,-1.2023225E-1,3.2718956E-2,6.073751E-2,-5.139053E-3,6.2746055E-2,-4.3278694E-2,2.2190563E-1,4.341383E-2,1.8498944E-2,5.6372364E-3,-6.724953E-3,4.7741733E-2,1.7688787E-1,9.073583E-3,-8.3332084E-2,6.202242E-2,1.2890819E-1,2.5029272E-1,3.536304E-1,4.721424E-3,2.1687383E-3,-2.5533005E-3,-5.725942E-3,3.5618814E-3,-1.0867551E-2,-4.7075087E-3,-4.633159E-3,-1.4744584E-2,-1.4284708E-3,5.931651E-3,-1.729421E-3,4.33724E-3,6.161029E-4,4.9827043E-3,-4.8794947E-3,1.8321441E-3,1.4712285E-2,7.662591E-3,4.8046135E-3,-2.549814E-3,5.519164E-3,-1.7206166E-4,9.72024E-3,3.5772806E-3,-5.3056665E-3,6.095058E-3,-7.1114544E-3,2.2762435E-5,5.0260825E-3,-1.8983884E-3,1.2448505E-2,4.282105E-3,1.4645046E-2,5.9932866E-3,1.3369483E-2,1.9262074E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":27,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,53,55,57,59,61,63,-1,65,67,-1,69,71,73,-1,-1,75,-1,-1,77,-1,-1,-1,79,81,-1,83,-1,85,-1,87,-1,-1,-1,89,91,93,95,-1,-1,97,99,-1,101,-1,-1,-1,-1,-1,-1,-1,103,105,107,109,111,113,-1,115,117,119,121,-1,-1,-1,123,125,127,129,131,133,135,137,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4571631E1,6.3322515E0,3.3781223E0,1.4409475E0,2.2034848E0,2.1684828E0,3.1592488E-1,5.5685043E-1,1.3537736E0,1.4157946E0,8.308697E-1,0E0,1.8311348E0,0E0,0E0,1.9482923E-1,6.0420895E-1,1.144659E0,0E0,1.466864E0,3.9598185E-1,5.389466E-1,3.3769026E-1,6.733835E-1,4.5863438E-1,1.7242908E-2,8.3211124E-2,4.460287E-2,8.388394E-2,8.5128677E-1,6.982827E-1,1.3196528E0,6.118092E-1,2.1237153E-1,1.1446744E-1,4.727211E-1,0E0,2.4388808E-1,7.14553E-2,0E0,4.550339E-1,3.3169603E-1,1.5375423E-1,0E0,0E0,7.945618E-2,0E0,0E0,4.4469357E-2,0E0,0E0,0E0,8.327475E-1,4.257729E-1,0E0,3.5121155E-1,0E0,2.3833957E-1,0E0,1.1334786E-1,0E0,0E0,0E0,3.6569667E-1,2.2212505E-1,2.2651754E-1,2.3899278E-1,0E0,0E0,1.4107932E-1,1.8330812E-1,0E0,4.285288E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,3.3461045E-2,2.3707464E-1,2.0960546E-1,6.767764E-1,1.06626414E-1,2.890601E-1,0E0,5.378162E-2,9.1223925E-2,9.349525E-2,1.35828E-1,0E0,0E0,0E0,7.004979E-2,5.5424392E-2,2.4141333E-1,6.455776E-2,8.5452706E-2,3.0701327E-1,1.0168314E-1,7.776022E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,37,37,38,38,40,40,41,41,42,42,45,45,48,48,52,52,53,53,55,55,57,57,59,59,63,63,64,64,65,65,66,66,69,69,70,70,72,72,80,80,81,81,82,82,83,83,84,84,85,85,87,87,88,88,89,89,90,90,94,94,95,95,96,96,97,97,98,98,99,99,100,100,101,101],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,54,56,58,60,62,64,-1,66,68,-1,70,72,74,-1,-1,76,-1,-1,78,-1,-1,-1,80,82,-1,84,-1,86,-1,88,-1,-1,-1,90,92,94,96,-1,-1,98,100,-1,102,-1,-1,-1,-1,-1,-1,-1,104,106,108,110,112,114,-1,116,118,120,122,-1,-1,-1,124,126,128,130,132,134,136,138,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.7909248E5,4.217427E7,7.7E1,3.8903564E2,3.6E1,1.5900173E3,1.139605E6,4.9213043E2,2.9652428E6,3.875E0,2.9762141E-2,2.6224005E6,-1.7407397E-2,-3.3126983E-3,7.2E1,9.558022E7,1.3520408E0,1.096673E-2,4.39776E6,6.789622E7,2.737878E5,3.9824562E0,1.0918E4,6.9307615E6,1.00013E5,2.8096385E0,5.797E3,5.6E1,4.03E2,3.0977E4,1.0948052E1,1E0,3.1517188E2,2.8636363E0,1.09652E5,-3.6413353E-3,1.535E3,2.3601625E1,1.9004837E-2,3.917053E5,3.532346E-1,3.53562E5,-1.4444468E-2,-8.98851E-3,3.0791914E0,-9.338058E-4,-1.446363E-2,1.0252778E1,-4.00843E-3,-1.30883185E-2,-1.0086646E-2,2.00087E5,2.347826E0,-1.9894369E-2,1.6527965E7,-2.7201718E-2,2.2681375E6,-1.2102014E-2,2.0487332E-3,8.862754E-3,4.845134E-3,1.5261253E-2,5.4204545E0,3.43E2,9.4939E4,1.3736341E8,-7.315577E-3,-4.3100782E-4,2.0528813E8,6.7947706E8,3.9759595E-3,3.1146461E1,1.1358764E-2,2.4923611E-2,-3.0461291E-3,-1.1558981E-2,-3.226373E-2,-1.8317465E-2,2.1476662E-2,1.44E2,1.4977306E2,1.547E3,1.2238068E7,3.8153846E0,8.0509944E5,-5.139053E-3,4.49239E6,9.356961E6,2.3066575E6,1.7642576E8,1.8498944E-2,5.6372364E-3,-6.724953E-3,1E0,1.0017886E0,1.1997242E0,5.989087E7,1.4342732E2,6.627033E7,9.730783E8,6.355E3,4.721424E-3,2.1687383E-3,-2.5533005E-3,-5.725942E-3,3.5618814E-3,-1.0867551E-2,-4.7075087E-3,-4.633159E-3,-1.4744584E-2,-1.4284708E-3,5.931651E-3,-1.729421E-3,4.33724E-3,6.161029E-4,4.9827043E-3,-4.8794947E-3,1.8321441E-3,1.4712285E-2,7.662591E-3,4.8046135E-3,-2.549814E-3,5.519164E-3,-1.7206166E-4,9.72024E-3,3.5772806E-3,-5.3056665E-3,6.095058E-3,-7.1114544E-3,2.2762435E-5,5.0260825E-3,-1.8983884E-3,1.2448505E-2,4.282105E-3,1.4645046E-2,5.9932866E-3,1.3369483E-2,1.9262074E-2],"split_indices":[2,37,54,38,61,0,64,40,61,37,63,0,37,0,0,2,7,62,0,9,7,42,62,9,37,5,62,9,0,2,38,63,85,64,63,1,0,2,65,0,37,47,1,0,0,63,0,0,67,0,0,0,5,62,0,54,0,37,0,47,0,0,0,65,0,12,7,0,0,7,7,0,65,0,0,0,0,0,0,0,10,61,10,54,63,54,0,37,54,54,7,0,0,0,23,47,47,5,65,12,7,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,7.38E2,2.27E2,2.32E2,5.06E2,2.13E2,1.4E1,7.6E1,1.56E2,3.58E2,1.48E2,1.3E1,2E2,7E0,7E0,4.6E1,3E1,1.48E2,8E0,2.98E2,6E1,7.3E1,7.5E1,1.15E2,8.5E1,3E1,1.6E1,1.9E1,1.1E1,2.4E1,1.24E2,1.96E2,1.02E2,4.8E1,1.2E1,6.6E1,7E0,6.1E1,1.4E1,8E0,1.07E2,6.5E1,2E1,2.1E1,9E0,1.1E1,5E0,5E0,1.4E1,5E0,6E0,8E0,1.6E1,1.12E2,1.2E1,1.91E2,5E0,9.6E1,6E0,4E1,8E0,5E0,7E0,4.7E1,1.9E1,2.6E1,3.5E1,9E0,5E0,2.6E1,8.1E1,8E0,5.7E1,5E0,1.5E1,5E0,6E0,7E0,7E0,5E0,1.1E1,2.5E1,8.7E1,1.75E2,1.6E1,8.8E1,8E0,2.4E1,1.6E1,2.6E1,2.1E1,1.4E1,5E0,8E0,1.8E1,2.1E1,1.4E1,1E1,1.6E1,6.4E1,1.7E1,4.8E1,9E0,6E0,5E0,1.7E1,8E0,7E1,1.7E1,1.6E2,1.5E1,1E1,6E0,2.2E1,6.6E1,1.2E1,1.2E1,9E0,7E0,8E0,1.8E1,1.3E1,8E0,7E0,1.1E1,1.5E1,6E0,7E0,7E0,5E0,5E0,1.1E1,5E0,1.2E1,5.2E1,1E1,7E0,2.6E1,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"139","size_leaf_vector":"1"}},{"base_weights":[2.9669784E-3,-1.3068718E-1,1.1260166E-1,-1.1017107E-1,-4.766953E-1,5.2907478E-2,2.9615626E-1,-2.2705525E-1,-5.4954283E-2,-2.9348978E-1,-4.203174E-2,8.0081776E-2,-3.7638882E-1,2.642132E-1,3.909211E-2,-2.929933E-1,-1.8454555E-1,1.4038938E-2,-6.609069E-2,-1.277104E-1,-2.5957622E-2,1.3976067E-1,1.9996377E-2,-2.502448E-2,-2.5102776E-1,1.3925926E-1,3.8510233E-1,-3.1286654E-1,-8.270182E-3,-9.398869E-2,-2.3915665E-1,-7.618627E-2,1.0121885E-2,-1.14613045E-2,4.5645804E-4,2.181517E-2,1.2511352E-1,-1.3066107E-1,5.0006676E-2,-1.5174498E-2,-7.2182924E-3,3.345724E-1,8.872473E-2,4.128941E-1,6.2859263E-3,-1.5283402E-2,-9.252603E-3,-1.7561184E-1,-3.5097603E-2,-2.73963E-1,-1.10986084E-1,-2.103045E-1,-5.9520688E-2,1.64481E-1,3.4916546E-2,-7.4196555E-2,-1.266653E-2,6.754968E-2,-1.066942E-1,8.925209E-3,1.853253E-2,1.1863838E-1,-7.6731606E-3,2.3079818E-2,3.3378893E-1,-4.3223402E-3,-1.1356751E-2,1.4623667E-3,-5.788445E-2,-2.2503667E-1,-1.5950548E-2,-1.0978219E-3,-7.5448765E-3,-1.1492657E-1,-1.6917238E-2,-4.923353E-2,-2.1466072E-1,1.3996536E-1,1.7544866E-2,-1.2766943E-2,1.15691826E-1,-1.3186274E-1,1.1184294E-2,3.8973566E-3,1.3376448E-1,-1.9237919E-1,3.8260096E-3,-4.4066682E-3,1.4568552E-1,2.824451E-1,2.4023846E-2,-1.5450036E-3,-4.3296376E-3,-1.1798404E-2,-5.744279E-3,-7.533004E-3,-1.6977568E-4,-6.869092E-4,-4.1442215E-3,-1.401324E-2,-1.5449224E-3,7.2385063E-3,-8.851862E-3,6.978446E-3,-2.5270309E-3,8.559483E-3,1.3984453E-3,-2.760807E-3,-7.521045E-3,3.7116539E-3,-2.7602552E-3,5.1125204E-3,-1.7598422E-3,1.0307238E-2,2.9345131E-3,-1.5169001E-3,-1.4001993E-2,8.179399E-3,-3.35819E-3,1.7222812E-2,1.0912641E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":28,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,-1,31,33,-1,35,37,-1,39,41,43,45,-1,47,49,51,-1,-1,-1,-1,53,55,57,-1,-1,59,61,63,-1,-1,-1,65,67,69,71,73,75,77,79,81,-1,83,85,-1,-1,87,-1,-1,89,-1,-1,-1,91,93,-1,-1,-1,95,-1,97,99,101,-1,103,105,107,109,111,113,115,-1,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4697152E1,3.1561108E0,6.0217166E0,2.7498212E0,1.6998076E0,4.8947916E0,2.1334572E0,3.2994127E-1,1.1825962E0,7.662419E-1,0E0,1.4064364E0,3.7526846E-1,1.89499E0,0E0,6.282282E-2,4.0282726E-1,0E0,8.3157015E-1,2.1575971E-1,0E0,8.9454174E-1,8.9722514E-1,0E0,6.522083E-2,6.1186874E-1,3.9500904E-1,1.491642E-2,0E0,1.572702E-1,2.0603466E-1,6.0215116E-1,0E0,0E0,0E0,0E0,6.689143E-1,2.449491E-1,4.6079296E-1,0E0,0E0,4.304874E-2,4.2247036E-1,2.2176456E-1,0E0,0E0,0E0,5.702442E-2,3.370098E-2,6.266379E-2,5.4754794E-2,3.9959013E-1,3.8203537E-1,6.38448E-1,2.2872423E-1,1.245645E-1,0E0,6.274727E-1,2.9139107E-1,0E0,0E0,2.911592E-1,0E0,0E0,1.6547441E-1,0E0,0E0,0E0,1.0829683E-2,4.8450828E-2,0E0,0E0,0E0,1.0494271E-1,0E0,3.188421E-1,2.2758424E-1,6.663296E-1,0E0,2.6671755E-1,1.23898834E-1,2.3168623E-2,5.8251515E-2,3.4648517E-1,4.409119E-1,1.9975573E-1,0E0,0E0,2.9965436E-1,3.0119896E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,18,18,19,19,21,21,22,22,24,24,25,25,26,26,27,27,29,29,30,30,31,31,36,36,37,37,38,38,41,41,42,42,43,43,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,57,57,58,58,61,61,64,64,68,68,69,69,73,73,75,75,76,76,77,77,79,79,80,80,81,81,82,82,83,83,84,84,85,85,88,88,89,89],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,-1,32,34,-1,36,38,-1,40,42,44,46,-1,48,50,52,-1,-1,-1,-1,54,56,58,-1,-1,60,62,64,-1,-1,-1,66,68,70,72,74,76,78,80,82,-1,84,86,-1,-1,88,-1,-1,90,-1,-1,-1,92,94,-1,-1,-1,96,-1,98,100,102,-1,104,106,108,110,112,114,116,-1,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.61E2,5.119403E0,5.97E3,3.56E2,2.439973E5,3.713753E7,1.0534078E4,3.89E2,1.2E1,3.1316666E2,-4.203174E-2,9E0,2.5807297E-2,1.7587205E6,3.909211E-2,4.631579E0,4.8421054E0,1.4038938E-2,7.4711205E6,1.2788898E2,-2.5957622E-2,7.336111E4,1.7302156E5,-2.502448E-2,5E0,3.019731E1,3.0546486E8,7.08E2,-8.270182E-3,1.8056113E0,1.5933333E2,8.147158E4,1.0121885E-2,-1.14613045E-2,4.5645804E-4,2.181517E-2,4.111111E0,5.5952E4,5.1071826E-8,-1.5174498E-2,-7.2182924E-3,1.172766E3,2.5652106E7,1.684E0,6.2859263E-3,-1.5283402E-2,-9.252603E-3,2.8282208E0,1.2E1,1.0140845E0,2.0976269E5,5.449711E8,3.8819444E0,2.7038124E0,9.7899484E4,2.1340163E8,-1.266653E-2,5.281628E2,2.0467092E6,8.925209E-3,1.853253E-2,1.8029197E0,-7.6731606E-3,2.3079818E-2,1.8E1,-4.3223402E-3,-1.1356751E-2,1.4623667E-3,3.3831256E7,2.55E2,-1.5950548E-2,-1.0978219E-3,-7.5448765E-3,2.9266055E0,-1.6917238E-2,1E0,2.700023E2,1.71E4,1.7544866E-2,4.5795207E0,1.0017886E0,2.8524646E-1,3.425E1,1.0181818E1,1.8201E4,5.189808E2,3.8260096E-3,-4.4066682E-3,5.444797E0,5.620512E8,2.4023846E-2,-1.5450036E-3,-4.3296376E-3,-1.1798404E-2,-5.744279E-3,-7.533004E-3,-1.6977568E-4,-6.869092E-4,-4.1442215E-3,-1.401324E-2,-1.5449224E-3,7.2385063E-3,-8.851862E-3,6.978446E-3,-2.5270309E-3,8.559483E-3,1.3984453E-3,-2.760807E-3,-7.521045E-3,3.7116539E-3,-2.7602552E-3,5.1125204E-3,-1.7598422E-3,1.0307238E-2,2.9345131E-3,-1.5169001E-3,-1.4001993E-2,8.179399E-3,-3.35819E-3,1.7222812E-2,1.0912641E-2],"split_indices":[2,62,2,2,56,54,4,1,10,4,0,8,47,37,0,63,67,0,37,4,0,54,37,0,6,67,41,42,0,62,4,37,0,0,0,0,63,38,46,0,0,4,52,63,0,0,0,63,27,65,37,40,62,62,42,7,0,4,37,0,0,63,0,0,3,0,0,0,7,2,0,0,0,62,0,23,61,10,0,67,47,47,67,67,10,61,0,0,62,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.001E3,4.51E2,5.5E2,4.27E2,2.4E1,4.16E2,1.34E2,1.36E2,2.91E2,1.8E1,6E0,3.92E2,2.4E1,1.28E2,6E0,5.1E1,8.5E1,8E0,2.83E2,1.2E1,6E0,1.96E2,1.96E2,9E0,1.5E1,6.4E1,6.4E1,4.2E1,9E0,3.3E1,5.2E1,2.74E2,9E0,6E0,6E0,7E0,1.89E2,3.2E1,1.64E2,7E0,8E0,1.2E1,5.2E1,5.7E1,7E0,3.5E1,7E0,1.3E1,2E1,4E1,1.2E1,2.9E1,2.45E2,1.31E2,5.8E1,2.4E1,8E0,1.48E2,1.6E1,5E0,7E0,4.7E1,5E0,2.5E1,3.2E1,7E0,6E0,5E0,1.5E1,2.6E1,1.4E1,5E0,7E0,1.9E1,1E1,2.31E2,1.4E1,1.19E2,1.2E1,3.7E1,2.1E1,1.4E1,1E1,7.6E1,7.2E1,1.1E1,5E0,5E0,4.2E1,2.7E1,5E0,1E1,5E0,1.9E1,7E0,1.3E1,6E0,1.25E2,1.06E2,9E0,5E0,1.14E2,5E0,7E0,3E1,1.1E1,1E1,5E0,9E0,5E0,5E0,2.1E1,5.5E1,3.1E1,4.1E1,5E0,6E0,3.7E1,5E0,7E0,2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[-8.308868E-3,-8.638832E-2,1.5010011E-1,-2.0373358E-1,-3.074724E-2,2.0003964E-1,-1.3998018E-1,5.885723E-3,-2.177311E-1,-1.3424097E-1,1.260548E-3,5.5409E-2,2.696665E-1,-2.9609302E-2,-6.548166E-2,-2.5039932E-1,-1.04340464E-1,4.3098666E-2,-1.8293938E-1,-5.3562082E-2,5.259006E-2,1.7359939E-1,-1.7338905E-2,2.8866145E-1,-3.9862944E-3,-2.109174E-1,5.2383367E-2,3.3485396E-3,-2.6178446E-1,-4.749373E-2,-1.82623E-1,-1.981268E-3,9.170479E-2,-1.407299E-1,-3.1326872E-1,-1.0739687E-1,2.9610377E-3,3.945744E-2,1.4779053E-2,3.632849E-2,2.4682315E-1,2.3336349E-2,-1.1202545E-1,2.7488833E-2,2.6639014E-1,-3.0932722E-3,-1.195591E-2,-5.6152106E-3,1.3090312E-1,-2.934834E-1,-1.6230121E-1,-1.0844087E-2,-8.355067E-3,-2.591445E-3,-1.2515942E-2,6.203233E-3,1.1423745E-3,-1.9480976E-1,-5.0736118E-2,-1.3776617E-1,-2.4933366E-2,-6.874314E-2,-2.1824393E-1,2.3829266E-2,-6.7510894E-3,-3.1330176E-2,7.202136E-2,5.4302826E-3,-1.3335792E-3,1.345215E-2,7.707588E-3,-6.282576E-3,6.1111234E-2,-9.158918E-3,-1.5093201E-3,3.7461683E-1,2.1795975E-1,-2.1600183E-4,2.0526727E-1,-2.5447148E-1,-4.6898994E-1,-3.469064E-5,-1.9875693E-1,-3.9273165E-3,2.0180985E-2,-1.8562585E-2,-1.4536832E-1,6.618307E-3,-7.1192877E-3,-3.6687127E-3,-8.504372E-3,-1.3052055E-1,4.437064E-2,-1.3110346E-1,-1.5936304E-2,-4.8030097E-2,5.49708E-2,3.1299535E-2,-1.02727436E-1,8.210356E-2,-4.3532583E-3,1.1789713E-1,7.305566E-4,7.3805256E-3,4.1279608E-1,1.5833506E-1,3.217132E-1,1.21776415E-2,5.283536E-3,-1.5243884E-2,-8.4103355E-3,-3.455799E-2,-1.47220185E-2,-1.0566392E-2,-4.0368615E-3,-1.1795602E-3,4.127674E-3,-1.2651039E-2,-4.6683387E-3,-1.5390763E-3,3.948228E-3,7.9468597E-4,-7.029645E-3,2.2464554E-4,7.21802E-3,-1.0985563E-2,-2.1389264E-3,-3.146885E-3,7.4475794E-4,3.8656096E-3,-3.8694432E-3,-3.3794753E-3,2.8596714E-3,-6.543199E-3,-1.730891E-4,2.0411005E-3,5.8201426E-3,8.325943E-3,2.0069531E-3,3.1304315E-3,-4.836927E-3,1.2106601E-2,2.0757098E-2,9.762586E-3,3.7951604E-3,1.7907938E-2,7.1299328E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":29,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,-1,25,27,29,31,33,35,37,39,41,43,-1,45,47,-1,49,51,53,-1,55,57,59,61,63,65,-1,67,69,71,73,-1,75,-1,-1,-1,77,79,81,83,-1,-1,-1,-1,-1,85,87,89,-1,91,93,95,-1,97,99,-1,-1,-1,-1,-1,101,-1,-1,103,105,-1,107,109,111,-1,113,-1,115,-1,117,119,-1,-1,-1,121,123,125,-1,127,129,131,133,135,-1,137,139,-1,141,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.1837238E1,4.178963E0,4.605727E0,9.808521E-1,1.4444652E0,2.7040052E0,1.7024845E0,0E0,7.0731926E-1,8.9740884E-1,9.426992E-1,7.682413E-1,1.2540655E0,0E0,7.3100364E-1,5.80781E-1,1.9603139E-1,9.983398E-2,4.0150547E-1,4.9326947E-1,5.9160566E-1,3.3278382E-1,2.1829264E-1,1.0126152E0,0E0,1.1097056E-1,3.426141E-1,0E0,4.1908646E-1,1.3409412E-1,1.9477952E-1,0E0,3.8958773E-2,2.9672766E-1,6.7322564E-1,3.4062332E-1,2.5131577E-1,3.8490668E-1,0E0,7.1632616E-2,2.83643E-2,2.4812764E-1,1.0894829E-1,0E0,7.72707E-1,0E0,0E0,0E0,1.7232719E-1,6.406555E-1,2.2365439E-1,5.4368287E-2,0E0,0E0,0E0,0E0,0E0,3.376124E-1,1.4827123E-1,1.7571196E-2,0E0,4.460275E-1,1.8410909E-1,1.6152482E-1,0E0,2.4020958E-1,1.9831806E-1,0E0,0E0,0E0,0E0,0E0,1.1306331E-1,0E0,0E0,3.3758974E-1,6.604824E-1,0E0,2.760619E-2,4.4473505E-1,5.5864E-1,0E0,8.055055E-2,0E0,5.558208E-2,0E0,1.6056502E-1,5.5902604E-2,0E0,0E0,0E0,1.2961262E-1,1.0033925E-1,1.14825934E-1,0E0,2.8990455E-2,2.0301668E-1,9.6088395E-2,9.424281E-2,1.7284971E-1,0E0,7.033338E-2,1.2613058E-1,0E0,9.84807E-2,2.8169465E-1,3.8329744E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,41,41,42,42,44,44,48,48,49,49,50,50,51,51,57,57,58,58,59,59,61,61,62,62,63,63,65,65,66,66,72,72,75,75,76,76,78,78,79,79,80,80,82,82,84,84,86,86,87,87,91,91,92,92,93,93,95,95,96,96,97,97,98,98,99,99,101,101,102,102,104,104,105,105,106,106],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,-1,26,28,30,32,34,36,38,40,42,44,-1,46,48,-1,50,52,54,-1,56,58,60,62,64,66,-1,68,70,72,74,-1,76,-1,-1,-1,78,80,82,84,-1,-1,-1,-1,-1,86,88,90,-1,92,94,96,-1,98,100,-1,-1,-1,-1,-1,102,-1,-1,104,106,-1,108,110,112,-1,114,-1,116,-1,118,120,-1,-1,-1,122,124,126,-1,128,130,132,134,136,-1,138,140,-1,142,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,1.7909248E5,1E0,7E0,1E0,5.933284E2,4.75E0,5.885723E-3,1.380863E6,1.1649485E0,7.5982756E5,9E0,7.134432E7,-2.9609302E-2,1.3711089E6,1.0677966E0,3.2692308E0,5.44E2,2.2E1,1.585814E6,7.82261E6,3.526271E6,6.4356956E0,7.778E3,-3.9862944E-3,1.8729467E8,1.4634147E0,3.3485396E-3,6.593462E4,1.8E1,1.151282E1,-1.981268E-3,8.798077E-1,7.13E2,7E0,1.3E1,5.2E1,1.7733E4,1.4779053E-2,2.779E3,6.45365E5,5.8687E4,3.1158154E4,2.7488833E-2,1.0717949E1,-3.0932722E-3,-1.195591E-2,-5.6152106E-3,3.7748723E8,8.91E2,9.183432E4,1.631108E6,-8.355067E-3,-2.591445E-3,-1.2515942E-2,6.203233E-3,1.1423745E-3,1.8E0,2.2741935E0,6.2233735E-6,-2.4933366E-2,3.6255838E2,2.23607E5,8.0509944E5,-6.7510894E-3,2.0131578E0,1E0,5.4302826E-3,-1.3335792E-3,1.345215E-2,7.707588E-3,-6.282576E-3,3.549E3,-9.158918E-3,-1.5093201E-3,1E0,1.3388E4,-2.1600183E-4,1E0,4.9E2,1E0,-3.469064E-5,2.5871112E2,-3.9273165E-3,8.0509944E5,-1.8562585E-2,3.3820656E7,2.1605E4,-7.1192877E-3,-3.6687127E-3,-8.504372E-3,1.512E4,1.8578552E7,5.0710382E0,-1.5936304E-2,2.558E3,1E0,4.39E2,3.4151352E2,6.946813E7,-4.3532583E-3,5.617715E6,2.8025E4,7.3805256E-3,3.019898E8,2.1356704E0,7.3274844E9,1.21776415E-2,5.283536E-3,-1.5243884E-2,-8.4103355E-3,-3.455799E-2,-1.47220185E-2,-1.0566392E-2,-4.0368615E-3,-1.1795602E-3,4.127674E-3,-1.2651039E-2,-4.6683387E-3,-1.5390763E-3,3.948228E-3,7.9468597E-4,-7.029645E-3,2.2464554E-4,7.21802E-3,-1.0985563E-2,-2.1389264E-3,-3.146885E-3,7.4475794E-4,3.8656096E-3,-3.8694432E-3,-3.3794753E-3,2.8596714E-3,-6.543199E-3,-1.730891E-4,2.0411005E-3,5.8201426E-3,8.325943E-3,2.0069531E-3,3.1304315E-3,-4.836927E-3,1.2106601E-2,2.0757098E-2,9.762586E-3,3.7951604E-3,1.7907938E-2,7.1299328E-3],"split_indices":[2,37,6,3,109,61,67,0,9,62,37,3,53,0,37,62,62,12,3,9,37,54,63,12,0,7,62,0,37,3,67,0,62,2,8,8,8,1,0,2,9,1,42,0,65,0,0,0,7,38,37,5,0,0,0,0,0,62,62,46,0,4,9,54,0,62,73,0,0,0,0,0,10,0,0,24,2,0,111,1,109,0,61,0,54,0,7,9,0,0,0,9,5,63,0,10,85,2,4,7,0,12,38,0,7,47,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.55E2,6.4E2,3.15E2,2.05E2,4.35E2,2.69E2,4.6E1,8E0,1.97E2,1.02E2,3.33E2,8.8E1,1.81E2,5E0,4.1E1,1.52E2,4.5E1,2.2E1,8E1,1.61E2,1.72E2,3.3E1,5.5E1,1.72E2,9E0,1.8E1,2.3E1,5E0,1.47E2,2.7E1,1.8E1,8E0,1.4E1,6.2E1,1.8E1,8.2E1,7.9E1,1.65E2,7E0,1.2E1,2.1E1,3.9E1,1.6E1,1E1,1.62E2,5E0,1.3E1,7E0,1.6E1,1.1E2,3.7E1,2.2E1,5E0,8E0,1E1,8E0,6E0,3.8E1,2.4E1,1.1E1,7E0,6.2E1,2E1,7E1,9E0,5.2E1,1.13E2,5E0,7E0,1.2E1,9E0,7E0,3.2E1,7E0,9E0,4.8E1,1.14E2,6E0,1E1,9.2E1,1.8E1,7E0,3E1,6E0,1.6E1,6E0,3.2E1,1.6E1,8E0,6E0,5E0,4E1,2.2E1,1.3E1,7E0,2.1E1,4.9E1,2.8E1,2.4E1,1.07E2,6E0,1.6E1,1.6E1,8E0,4E1,7.4E1,4E1,5E0,5E0,4.4E1,4.8E1,5E0,1.3E1,2.3E1,7E0,1E1,6E0,7E0,2.5E1,1.1E1,5E0,5E0,3.5E1,1.7E1,5E0,5E0,8E0,1.6E1,5E0,4.1E1,8E0,6E0,2.2E1,1.7E1,7E0,5.8E1,4.9E1,8E0,8E0,1E1,6E0,9E0,3.1E1,4.3E1,3.1E1,2.8E1,1.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[5.0191255E-3,-5.5762142E-2,2.2916281E-1,-1.5603112E-1,3.2524087E-2,3.5015587E-2,2.0922601E-1,-2.9311827E-1,-1.13504335E-1,4.6689164E-2,-2.1104829E-1,-4.74718E-3,2.247669E-1,-2.1977074E-1,-5.6063694E-1,-1.7814074E-1,-5.6608114E-2,5.429852E-2,-1.438978E-2,1.4441994E-3,-2.8410676E-1,2.4173513E-1,-6.870067E-2,-2.6063824E-1,-1.4232253E-1,-3.3765513E-1,-4.237013E-2,-1.5289903E-1,-3.4054378E-1,2.1390123E-2,-8.642021E-2,2.3131453E-2,1.2842977E-1,-1.270313E-1,-2.2752084E-2,1.0356522E-1,2.826278E-1,-8.718191E-3,2.874129E-3,-2.887441E-3,-2.9197764E-1,-1.9246012E-1,2.2514479E-3,-6.9195363E-3,-2.32612E-2,-1.50335785E-2,-1.22471936E-1,-1.8139934E-2,-8.879529E-3,-5.1010754E-2,-2.691964E-1,2.8536462E-3,9.731175E-2,1.8746449E-1,4.030339E-2,-8.126948E-3,-2.6790411E-3,2.8207395E-2,2.2425874E-1,2.332E-2,2.630823E-1,-1.8369639E-1,-1.5417295E-2,-1.2503049E-1,-1.2897909E-2,-1.4868873E-1,-3.3937253E-2,-1.9631747E-2,-1.7703752E-1,-3.198645E-1,-3.7477126E-3,1.1639863E-2,-1.354678E-2,1.22442394E-1,-3.1912308E-3,1.9697237E-3,2.100121E-1,6.2413914E-3,8.159702E-3,8.414661E-2,-6.080843E-2,5.6060185E-3,1.2724541E-2,2.1302657E-1,3.2081693E-1,-1.0646868E-2,-4.6254606E-3,-3.3269394E-3,-7.4469773E-3,-3.399645E-3,-8.762587E-3,1.690931E-3,-2.5757197E-3,3.3216542E-3,-2.0997224E-3,-1.2210092E-2,-2.7455299E-3,-8.480802E-3,-1.7540023E-2,-3.400817E-4,4.724293E-3,4.157143E-3,1.1993174E-2,1.0682512E-2,1.4671705E-4,2.8162624E-3,-4.062515E-3,7.805522E-3,1.6284053E-3,-5.035176E-3,-1.3752829E-4,1.2373328E-2,4.4944133E-3,6.290663E-3,1.577355E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":30,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,-1,21,23,25,27,29,31,-1,-1,33,35,37,39,41,43,-1,45,47,-1,49,51,53,55,-1,57,59,-1,-1,-1,61,63,-1,-1,-1,-1,65,-1,-1,67,69,71,73,75,77,-1,-1,79,81,-1,83,85,-1,87,-1,89,91,93,95,97,-1,99,-1,101,-1,-1,103,105,-1,107,109,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.2900834E1,6.6000905E0,1.9583902E0,2.0013561E0,1.3795915E0,0E0,9.7194386E-1,1.4857788E0,9.755039E-1,1.0377371E0,3.980853E-1,0E0,9.477978E-1,1.7180276E-1,1.027977E0,4.6034765E-1,2.2619007E0,8.495866E-1,0E0,0E0,4.884454E-1,9.7080326E-1,1.8293014E-1,2.4298596E-1,2.4855238E-1,2.9550922E-1,0E0,5.3300285E-1,4.5494795E-2,0E0,8.729328E-1,3.9232206E-1,5.5951893E-1,2.6565447E-2,0E0,3.7567273E-1,4.5248318E-1,0E0,0E0,0E0,9.6040964E-2,8.057034E-2,0E0,0E0,0E0,0E0,2.1545804E-1,0E0,0E0,4.552574E-1,1.8420208E-1,5.369999E-1,2.4216121E-1,2.0361495E-1,2.06679E-1,0E0,0E0,1.3956267E-1,4.958433E-2,0E0,2.9868126E-1,2.326116E-2,0E0,1.3079971E-2,0E0,2.0728314E-1,3.6946885E-2,2.2293565E-1,2.1787763E-1,6.644118E-2,0E0,3.4458008E-1,0E0,1.9579405E-1,0E0,0E0,2.3346257E-1,1.9429815E-1,0E0,6.420627E-2,3.0035477E-2,0E0,0E0,4.1659737E-1,1.5594816E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,30,30,31,31,32,32,33,33,35,35,36,36,40,40,41,41,46,46,49,49,50,50,51,51,52,52,53,53,54,54,57,57,58,58,60,60,61,61,63,63,65,65,66,66,67,67,68,68,69,69,71,71,73,73,76,76,77,77,79,79,80,80,83,83,84,84],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,-1,22,24,26,28,30,32,-1,-1,34,36,38,40,42,44,-1,46,48,-1,50,52,54,56,-1,58,60,-1,-1,-1,62,64,-1,-1,-1,-1,66,-1,-1,68,70,72,74,76,78,-1,-1,80,82,-1,84,86,-1,88,-1,90,92,94,96,98,-1,100,-1,102,-1,-1,104,106,-1,108,110,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.921E3,4.539777E5,2.2968E4,3.3820656E7,5.174898E-6,3.5015587E-2,1.7909248E5,4.6842103E0,7.06E2,1.4676277E7,3.7105262E0,-4.74718E-3,2.871605E2,9.2E1,3.4402E4,3.1E1,2.04115E5,4.2259482E2,-1.438978E-2,1.4441994E-3,6.18826E5,7.606269E2,3.6086957E0,1.863E3,6.47E2,1.0183612E2,-4.237013E-2,5.4E1,4.9506173E0,2.1390123E-2,1.3439851E7,8.328548E6,3.875E0,2.3043478E2,-2.2752084E-2,6.103178E8,1.2083E4,-8.718191E-3,2.874129E-3,-2.887441E-3,1E0,8E0,2.2514479E-3,-6.9195363E-3,-2.32612E-2,-1.50335785E-2,4.39776E6,-1.8139934E-2,-8.879529E-3,2.0734E4,1E2,4.093403E6,3.57E2,5.1604336E7,9.8897E2,-8.126948E-3,-2.6790411E-3,7.6937294E-1,4.433E3,2.332E-2,2.6224005E6,3.23E2,-1.5417295E-2,1.5476E4,-1.2897909E-2,3.9961785E-2,1.8461539E0,3.14403E5,2.147382E4,1.609E3,-3.7477126E-3,4.5866325E6,-1.354678E-2,1E0,-3.1912308E-3,1.9697237E-3,4.8060772E1,1.1891E4,8.159702E-3,4.0906172E5,6.257143E2,5.6060185E-3,1.2724541E-2,1.8809018E0,5.735198E-1,-1.0646868E-2,-4.6254606E-3,-3.3269394E-3,-7.4469773E-3,-3.399645E-3,-8.762587E-3,1.690931E-3,-2.5757197E-3,3.3216542E-3,-2.0997224E-3,-1.2210092E-2,-2.7455299E-3,-8.480802E-3,-1.7540023E-2,-3.400817E-4,4.724293E-3,4.157143E-3,1.1993174E-2,1.0682512E-2,1.4671705E-4,2.8162624E-3,-4.062515E-3,7.805522E-3,1.6284053E-3,-5.035176E-3,-1.3752829E-4,1.2373328E-2,4.4944133E-3,6.290663E-3,1.577355E-2],"split_indices":[2,37,12,7,46,0,37,62,2,56,63,0,67,38,9,8,5,61,0,0,9,61,63,9,2,42,0,2,63,0,54,9,63,4,0,7,9,0,0,0,21,26,0,0,0,0,9,0,0,38,6,60,0,7,4,0,0,47,10,0,37,1,0,9,0,47,62,9,42,2,0,37,0,25,0,0,67,38,0,37,4,0,0,51,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.45E2,7.44E2,2.01E2,3.48E2,3.96E2,6E0,1.95E2,8.1E1,2.67E2,3.75E2,2.1E1,9E0,1.86E2,6.5E1,1.6E1,1.24E2,1.43E2,3.68E2,7E0,5E0,1.6E1,1.76E2,1E1,4.1E1,2.4E1,1.1E1,5E0,1.09E2,1.5E1,7E0,1.36E2,2.6E2,1.08E2,1E1,6E0,4.1E1,1.35E2,5E0,5E0,6E0,3.5E1,1.9E1,5E0,6E0,5E0,1.5E1,9.4E1,1E1,5E0,1.15E2,2.1E1,2.05E2,5.5E1,6.4E1,4.4E1,5E0,5E0,2.6E1,1.5E1,9E0,1.26E2,1.1E1,2.4E1,1.2E1,7E0,7.2E1,2.2E1,9.3E1,2.2E1,1.6E1,5E0,2E2,5E0,4.8E1,7E0,9E0,5.5E1,3.6E1,8E0,1.6E1,1E1,6E0,9E0,7E1,5.6E1,6E0,5E0,6E0,6E0,2.6E1,4.6E1,5E0,1.7E1,2E1,7.3E1,1.2E1,1E1,6E0,1E1,1.66E2,3.4E1,4E1,8E0,5E1,5E0,2.3E1,1.3E1,5E0,1.1E1,5E0,5E0,4.7E1,2.3E1,6E0,5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[-1.1971922E-2,-6.959134E-2,1.6755739E-1,-1.8752861E-1,-1.6206613E-2,2.1259813E-1,-1.8729712E-1,8.289788E-3,-2.0165671E-1,-4.956995E-2,6.696836E-2,6.4927095E-1,1.873921E-1,2.1951582E-2,-3.8316748E-1,-2.4432673E-1,-1.1759344E-1,-1.1265872E-1,1.2104555E-3,4.058316E-2,2.1264093E-1,1.8882835E-2,3.634777E-2,9.559139E-2,2.5930986E-1,-3.8449306E-3,5.1467596E-3,-8.859497E-3,-2.1749489E-2,-2.5674194E-1,-9.851514E-2,-6.201129E-2,-2.1066242E-1,-5.2246153E-2,-1.5855579E-1,4.9808227E-2,-6.707225E-2,-6.9439225E-3,1.2417391E-1,1.4659403E-1,1.3537201E-2,6.657545E-2,1.9765288E-2,3.565088E-1,2.0170893E-1,-2.2965857E-1,-3.9828002E-1,-8.006582E-4,-8.836384E-3,-1.1733013E-2,-1.2318313E-1,-3.12462E-1,-6.916581E-2,-1.558353E-1,-8.567367E-3,-9.9580124E-2,-2.2498979E-1,7.749579E-2,-1.6140622E-1,-1.3208841E-1,6.078901E-2,1.6777048E-2,-8.104511E-3,5.726596E-2,2.2135438E-1,1.0709633E-2,3.44094E-3,8.5989565E-2,-7.2610667E-3,7.250407E-3,3.8725078E-1,2.6034448E-1,1.2452313E-1,-1.8123156E-1,-2.901999E-1,-2.4586348E-2,-1.3300642E-2,3.2741755E-2,-5.8933115E-3,-1.6101263E-1,-8.870693E-4,-2.0002918E-2,-6.4928182E-3,7.340181E-4,-5.765549E-3,-2.0062135E-1,-4.8409804E-4,-3.9565094E-2,5.5755444E-2,-1.4185964E-1,6.3216306E-2,-1.8021165E-1,-2.1511847E-2,1.0690109E-1,1.5630875E-2,-3.2354384E-3,-1.2621531E-2,-1.7927088E-1,8.046368E-3,1.295668E-1,-6.3079325E-3,-8.733329E-3,3.515901E-2,1.3111235E-1,-1.547507E-2,1.5467823E-1,1.5262017E-2,-1.0789546E-2,1.5144482E-1,1.9549454E-2,1.0385466E-2,1.6881306E-1,4.0164265E-1,1.1856037E-2,5.645778E-2,-1.089435E-2,7.6731824E-4,-1.0151016E-2,-1.6494317E-2,6.4409496E-3,-4.214089E-4,-1.0184052E-2,-5.3118113E-3,-3.6909361E-3,-1.1526705E-2,1.0340777E-3,-2.4652218E-3,-3.5851853E-4,5.2451105E-3,-5.0524063E-3,-8.561121E-3,6.2879473E-3,-9.1041625E-4,-9.604751E-3,-3.3855336E-4,5.055659E-5,6.3874717E-3,-3.2406459E-3,3.523519E-3,-2.164794E-2,-6.3113044E-3,9.745602E-4,7.4360436E-3,-7.5304735E-5,5.2920757E-3,2.6296536E-3,1.0250435E-2,-3.0840803E-3,3.3450492E-3,9.094933E-3,3.42354E-3,-2.4888883E-3,5.775262E-3,1.1289749E-2,5.005551E-3,9.625398E-3,4.0021683E-3,2.1284577E-2,1.11156525E-2,-4.2185653E-3,5.2812304E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":31,"left_children":[1,3,5,7,9,11,13,-1,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,-1,41,43,-1,-1,-1,-1,45,47,49,51,53,55,57,59,61,63,65,-1,67,-1,69,71,73,75,-1,-1,77,79,81,83,85,87,89,91,93,95,97,99,101,-1,103,105,-1,-1,107,-1,-1,109,111,113,115,117,-1,-1,119,-1,121,-1,-1,-1,-1,-1,123,-1,125,127,129,131,133,-1,135,137,-1,-1,139,-1,141,-1,-1,143,145,147,149,-1,151,153,-1,-1,155,157,-1,159,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0004327E1,4.605669E0,3.795802E0,1.2156029E0,1.4046919E0,2.17875E0,1.1150229E0,0E0,7.6040363E-1,1.1566622E0,5.4869235E-1,3.8617134E-2,1.289031E0,1.3975239E-1,1.4080286E-1,2.3719788E-1,3.800826E-1,4.3813062E-1,6.703366E-1,4.9349242E-1,7.2184145E-2,0E0,0E0,8.2484555E-1,5.4955053E-1,0E0,0E0,0E0,0E0,4.1836548E-1,9.382245E-2,1.492598E-1,3.8170862E-1,3.1991497E-1,3.3413243E-1,7.0257384E-1,7.063255E-1,3.2121748E-1,2.7992052E-1,6.828955E-2,0E0,3.7289858E-1,0E0,1.8572855E-1,2.9344082E-1,2.8163576E-1,1.4191246E-1,0E0,0E0,1.4771558E-1,8.352974E-2,2.6717794E-1,6.283334E-2,1.3465846E-1,1.03892885E-1,3.531156E-1,3.6524367E-1,1.8934047E-1,1.22999996E-1,8.382961E-1,4.1001853E-1,2.7416423E-1,0E0,1.5243551E-1,7.82277E-2,0E0,0E0,4.8940605E-1,0E0,0E0,9.505129E-2,4.495268E-1,2.7981627E-1,7.030332E-1,1.5071774E-1,0E0,0E0,9.4425514E-2,0E0,1.8408239E-2,0E0,0E0,0E0,0E0,0E0,7.284057E-2,0E0,3.0550446E-2,6.269767E-2,3.6155522E-2,6.8541184E-2,1.6988802E-1,0E0,2.3095095E-1,1.8571909E-1,0E0,0E0,5.4536307E-1,0E0,6.994501E-2,0E0,0E0,1.9171342E-1,8.106844E-2,7.1771905E-2,2.9006958E-2,0E0,1.9214654E-1,1.5786219E-1,0E0,0E0,6.503433E-2,2.888298E-2,0E0,2.0479028E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,24,24,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,67,67,70,70,71,71,72,72,73,73,74,74,77,77,79,79,85,85,87,87,88,88,89,89,90,90,91,91,93,93,94,94,97,97,99,99,102,102,103,103,104,104,105,105,107,107,108,108,111,111,112,112,114,114],"right_children":[2,4,6,8,10,12,14,-1,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,-1,42,44,-1,-1,-1,-1,46,48,50,52,54,56,58,60,62,64,66,-1,68,-1,70,72,74,76,-1,-1,78,80,82,84,86,88,90,92,94,96,98,100,102,-1,104,106,-1,-1,108,-1,-1,110,112,114,116,118,-1,-1,120,-1,122,-1,-1,-1,-1,-1,124,-1,126,128,130,132,134,-1,136,138,-1,-1,140,-1,142,-1,-1,144,146,148,150,-1,152,154,-1,-1,156,158,-1,160,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.116E3,2.0976269E5,1E0,6E0,2.0766992E6,7.778E3,2.2092846E7,8.289788E-3,8.147158E4,2.9711267E2,1E0,3.284E4,1.1362371E6,1.342376E0,5.111912E9,1.3631483E8,1.5885367E5,2.1463416E0,2.6789763E9,4.4210525E0,5.941442E6,1.8882835E-2,3.634777E-2,7.626E3,1.684E0,-3.8449306E-3,5.1467596E-3,-8.859497E-3,-2.1749489E-2,1.4E1,1.0885427E9,3.287234E0,1.09285E5,1.4110284E2,2.6742857E0,1E0,1.852364E6,1.639E3,5.281628E2,8.8790035E-1,1.3537201E-2,2.5652106E7,1.9765288E-2,1.5116882E8,2.1089442E0,2.0007E5,2.2003597E1,-8.006582E-4,-8.836384E-3,1.1024147E9,4.7238097E0,9.07021E5,2.3572223E2,9E0,2.992306E4,1.6223962E7,2E1,1.6955555E1,4E0,3.97371E5,7.1E1,2.8784848E2,-8.104511E-3,1.4601138E4,2.06E2,1.0709633E-2,3.44094E-3,3.9401392E6,-7.2610667E-3,7.250407E-3,2.395631E7,6.888377E6,2.192E3,4.36875E1,2.11429E5,-2.4586348E-2,-1.3300642E-2,3.8575E4,-5.8933115E-3,1E0,-8.870693E-4,-2.0002918E-2,-6.4928182E-3,7.340181E-4,-5.765549E-3,4.2E1,-4.8409804E-4,1.92E2,2.416335E6,1E0,5E0,2.7976523E2,-2.1511847E-2,7.06E2,1.047E3,-3.2354384E-3,-1.2621531E-2,2.6086957E0,8.046368E-3,9.008784E7,-6.3079325E-3,-8.733329E-3,4.7176E4,4.4E2,2.1E1,4.56E2,1.5262017E-2,5.486433E5,1.5E1,1.9549454E-2,1.0385466E-2,8.958E3,1.1949272E7,1.1856037E-2,4.988E3,-1.089435E-2,7.6731824E-4,-1.0151016E-2,-1.6494317E-2,6.4409496E-3,-4.214089E-4,-1.0184052E-2,-5.3118113E-3,-3.6909361E-3,-1.1526705E-2,1.0340777E-3,-2.4652218E-3,-3.5851853E-4,5.2451105E-3,-5.0524063E-3,-8.561121E-3,6.2879473E-3,-9.1041625E-4,-9.604751E-3,-3.3855336E-4,5.055659E-5,6.3874717E-3,-3.2406459E-3,3.523519E-3,-2.164794E-2,-6.3113044E-3,9.745602E-4,7.4360436E-3,-7.5304735E-5,5.2920757E-3,2.6296536E-3,1.0250435E-2,-3.0840803E-3,3.3450492E-3,9.094933E-3,3.42354E-3,-2.4888883E-3,5.775262E-3,1.1289749E-2,5.005551E-3,9.625398E-3,4.0021683E-3,2.1284577E-2,1.11156525E-2,-4.2185653E-3,5.2812304E-3],"split_indices":[2,37,6,3,37,12,54,0,37,4,92,1,37,63,40,7,37,62,40,65,54,0,0,61,63,0,0,0,0,8,40,62,12,4,65,85,9,2,4,65,0,52,0,12,47,5,67,0,0,40,62,9,61,8,42,5,3,67,8,1,8,42,0,42,0,0,0,54,0,0,56,56,38,64,9,0,0,9,0,23,0,0,0,0,0,0,0,12,12,23,8,4,0,2,2,0,0,63,0,5,0,0,1,10,67,10,0,37,8,0,0,10,54,0,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,7.31E2,2.34E2,2.27E2,5.04E2,2.08E2,2.6E1,8E0,2.19E2,3.6E2,1.44E2,1E1,1.98E2,1.3E1,1.3E1,1.44E2,7.5E1,1.6E2,2E2,1.23E2,2.1E1,5E0,5E0,8.8E1,1.1E2,6E0,7E0,5E0,8E0,1.32E2,1.2E1,4.8E1,2.7E1,7E1,9E1,1.17E2,8.3E1,7.9E1,4.4E1,1.3E1,8E0,8.2E1,6E0,3.9E1,7.1E1,1.13E2,1.9E1,7E0,5E0,2.7E1,2.1E1,1.5E1,1.2E1,2E1,5E1,4.9E1,4.1E1,1.04E2,1.3E1,5.5E1,2.8E1,7E1,9E0,2.7E1,1.7E1,5E0,8E0,7.6E1,6E0,6E0,3.3E1,3.9E1,3.2E1,6.5E1,4.8E1,7E0,1.2E1,2E1,7E0,1.5E1,6E0,8E0,7E0,5E0,7E0,1.5E1,5E0,3.4E1,1.6E1,3.9E1,1E1,3.6E1,5E0,7E1,3.4E1,8E0,5E0,4.8E1,7E0,2.1E1,7E0,5E0,6.5E1,1.3E1,1.4E1,1.2E1,5E0,3.1E1,4.5E1,2.6E1,7E0,2.5E1,1.4E1,1E1,2.2E1,5.1E1,1.4E1,2.5E1,2.3E1,5E0,1.5E1,5E0,1E1,5E0,1E1,6E0,2.8E1,8E0,8E0,2.4E1,1.5E1,5E0,5E0,3.1E1,5E0,1.6E1,5.4E1,1.4E1,2E1,5E0,4.3E1,5E0,1.6E1,4.5E1,2E1,8E0,5E0,9E0,5E0,7E0,5E0,2.4E1,7E0,1.3E1,3.2E1,1.6E1,9E0,9E0,5E0,6E0,1.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"161","size_leaf_vector":"1"}},{"base_weights":[-5.3104907E-3,-1.0479773E-1,1.0017351E-1,-1.9760223E-1,-5.152646E-2,-2.471578E-1,1.23773284E-1,-1.7084399E-1,-3.1648606E-1,-1.11189656E-1,-2.7235188E-2,-3.5485935E-3,-2.9375237E-1,6.5557905E-2,2.7723366E-1,-2.3610817E-1,-1.07523255E-1,-5.538872E-1,-1.5445271E-1,-2.3036694E-2,-1.680076E-1,-3.906503E-2,2.1506388E-1,-1.5621436E-2,-6.561803E-3,1.2515853E-1,1.0468032E-3,3.7670186E-1,8.7820955E-2,-1.9200398E-2,-2.1173024E-1,-3.8269997E-2,-1.9110309E-1,-3.6679484E-2,-1.4252948E-2,-8.42567E-4,-2.3692891E-1,3.885974E-2,-8.6978376E-2,-1.0768529E-1,-3.1585318E-1,-4.932921E-2,1.632382E-1,4.1824304E-3,1.4112103E-2,1.4257643E-1,-1.7520241E-2,-1.6130668E-1,1.6607776E-2,2.939119E-1,5.610817E-1,1.5285112E-1,-1.2116512E-1,-1.565568E-1,-1.2030338E-2,6.057281E-3,-6.758912E-2,-1.2987271E-2,-6.2610984E-2,-1.293812E-2,-6.56812E-3,-1.7985188E-3,7.489475E-2,-9.222747E-3,1.2393526E-2,-5.4514993E-2,-1.6344687E-1,-4.346499E-3,-2.0406088E-2,-6.6944435E-2,5.6248695E-2,1.1000891E-2,2.8850602E-3,8.847351E-2,2.2119905E-1,-3.1848277E-3,-9.51939E-3,2.7040832E-2,-7.7367546E-3,3.218655E-1,-9.266302E-5,3.103907E-2,1.7296871E-2,5.5098615E-4,2.0312758E-1,-1.5856295E-3,-8.721351E-3,-1.8593393E-1,-7.740821E-4,-7.938114E-3,-1.7354175E-1,-4.7660065E-3,1.6832384E-4,4.629051E-3,1.6597998E-3,2.619436E-3,-1.5651736E-3,-7.1427533E-3,-1.8344501E-2,-1.9372791E-1,-2.807308E-3,-8.900329E-2,2.5601534E-2,-9.826041E-3,9.8497264E-2,2.2050662E-2,1.5445535E-1,2.6135787E-1,-7.0056696E-3,-9.2777774E-2,4.338128E-2,2.0635933E-2,2.6165637E-1,2.765515E-3,1.1861158E-2,-6.4918E-3,-1.2414029E-2,-2.3000087E-3,2.076976E-3,-4.718599E-3,-1.0711784E-2,-1.7667434E-3,1.2744222E-3,-1.0410729E-2,-5.1209345E-3,-1.7603349E-3,-5.4720966E-3,2.3859122E-3,-5.199838E-3,1.8110261E-3,-2.0991436E-3,7.2975233E-3,-9.2025315E-5,1.1657297E-2,-1.2264106E-3,4.600774E-3,1.0223152E-2,1.3143079E-2,4.259558E-3,4.7456073E-3,-5.341549E-3,2.2027656E-3,-9.08043E-3,2.8397029E-3,-3.6940905E-3,1.6097542E-2,7.7746646E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":32,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,-1,53,55,57,-1,-1,-1,59,61,63,65,67,69,71,-1,-1,73,-1,75,77,79,81,83,85,87,-1,-1,89,-1,91,-1,-1,-1,93,-1,95,97,99,-1,-1,101,103,-1,-1,105,107,-1,-1,109,-1,111,-1,-1,-1,-1,113,-1,-1,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,121,123,-1,125,127,129,131,133,135,137,139,141,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.04208555E1,2.5161066E0,3.981516E0,5.376191E-1,4.6944183E-1,2.240678E-1,4.0198812E0,6.110296E-1,1.1838589E0,4.669447E-1,6.7626333E-1,0E0,1.09304905E-1,1.2648419E0,2.3031445E0,2.3727894E-1,4.5574057E-1,5.1608634E-1,2.317583E-1,1.5377492E-1,4.743353E-1,4.711068E-1,9.499544E-2,0E0,0E0,1.5512712E0,4.0437308E-1,1.0715408E0,6.1649776E-1,0E0,1.3779736E-1,2.3155338E-1,3.932321E-1,0E0,0E0,0E0,1.2494445E-2,5.8844827E-2,2.135309E-1,1.1561382E-1,3.9938486E-1,3.998983E-1,6.7356884E-2,0E0,0E0,6.8654656E-1,0E0,4.2813003E-2,2.8379288E-1,4.8040295E-1,2.3812103E-1,2.3857272E-1,5.749564E-2,1.3660711E-1,0E0,0E0,2.4510947E-1,0E0,4.287772E-2,0E0,0E0,0E0,1.0053255E-2,0E0,2.4260128E-2,7.9890326E-2,5.0623775E-2,0E0,0E0,3.7586546E-1,8.7506294E-2,0E0,0E0,4.3486428E-1,6.158478E-1,0E0,0E0,2.7617154E-1,0E0,2.7575016E-1,0E0,0E0,0E0,0E0,1.7315531E-1,0E0,0E0,6.7736804E-2,0E0,5.9263807E-2,3.417656E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.7317038E-2,1.355499E-2,0E0,2.1417594E-1,1.3372645E-1,2.425926E-2,1.125287E-1,5.7332844E-1,1.6029286E-1,1.8567014E-1,1.4174122E-1,2.5468954E-1,2.7446324E-1,0E0,2.4679422E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,25,25,26,26,27,27,28,28,30,30,31,31,32,32,36,36,37,37,38,38,39,39,40,40,41,41,42,42,45,45,47,47,48,48,49,49,50,50,51,51,52,52,53,53,56,56,58,58,62,62,64,64,65,65,66,66,69,69,70,70,73,73,74,74,77,77,79,79,84,84,87,87,89,89,90,90,98,98,99,99,101,101,102,102,103,103,104,104,105,105,106,106,107,107,108,108,109,109,110,110,112,112],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,-1,54,56,58,-1,-1,-1,60,62,64,66,68,70,72,-1,-1,74,-1,76,78,80,82,84,86,88,-1,-1,90,-1,92,-1,-1,-1,94,-1,96,98,100,-1,-1,102,104,-1,-1,106,108,-1,-1,110,-1,112,-1,-1,-1,-1,114,-1,-1,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,122,124,-1,126,128,130,132,134,136,138,140,142,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,3.964269E3,2.2E1,1E0,1.3E1,1.1464103E3,2.55E2,2E0,1.7754011E0,1.9505818E8,-3.5485935E-3,2.3017536E5,3.9649122E0,1.7493458E7,1.1649485E0,2.49E2,1.1046409E7,1E0,2.0217392E0,2.5178E4,1.5074378E5,2.8636363E0,-1.5621436E-2,-6.561803E-3,1.5449402E7,8.638015E4,2.0597144E3,3.85374E8,-1.9200398E-2,9.766231E-3,1.1E1,1.9578822E5,-3.6679484E-2,-1.4252948E-2,-8.42567E-4,6.593462E4,5.44E2,1.3333334E0,6.4153387E-6,3E0,9.750085E6,1.1251919E6,4.1824304E-3,1.4112103E-2,6.009825E2,-1.7520241E-2,1.3748E4,2.5243637E-6,1.0905187E9,1.159499E6,1.2615725E5,4.5698816E7,4.294737E1,-1.2030338E-2,6.057281E-3,2.0689879E2,-1.2987271E-2,8.01E2,-1.293812E-2,-6.56812E-3,-1.7985188E-3,4.8064E4,-9.222747E-3,3.640566E2,4.9E2,1.52754345E1,-4.346499E-3,-2.0406088E-2,8.611817E6,1.088E3,1.1000891E-2,2.8850602E-3,4.745397E6,1E0,-3.1848277E-3,-9.51939E-3,2.197238E2,-7.7367546E-3,2.1412018E6,-9.266302E-5,3.103907E-2,1.7296871E-2,5.5098615E-4,1.5896863E3,-1.5856295E-3,-8.721351E-3,1.00033E5,-7.740821E-4,9.5E1,1.1320151E7,-4.7660065E-3,1.6832384E-4,4.629051E-3,1.6597998E-3,2.619436E-3,-1.5651736E-3,-7.1427533E-3,4.066351E0,4.556962E0,-2.807308E-3,5.1E1,3.1275E4,1.2E1,3.44E2,4.92E2,1.28738E5,1.7172844E7,7.1913286E8,7.714286E0,9.639872E0,2.0635933E-2,2.2421052E0,2.765515E-3,1.1861158E-2,-6.4918E-3,-1.2414029E-2,-2.3000087E-3,2.076976E-3,-4.718599E-3,-1.0711784E-2,-1.7667434E-3,1.2744222E-3,-1.0410729E-2,-5.1209345E-3,-1.7603349E-3,-5.4720966E-3,2.3859122E-3,-5.199838E-3,1.8110261E-3,-2.0991436E-3,7.2975233E-3,-9.2025315E-5,1.1657297E-2,-1.2264106E-3,4.600774E-3,1.0223152E-2,1.3143079E-2,4.259558E-3,4.7456073E-3,-5.341549E-3,2.2027656E-3,-9.08043E-3,2.8397029E-3,-3.6940905E-3,1.6097542E-2,7.7746646E-3],"split_indices":[2,37,37,3,109,3,61,2,26,62,5,0,42,63,60,62,10,54,18,65,1,42,63,0,0,56,37,61,41,0,47,10,56,0,0,0,37,12,62,46,8,9,37,0,0,64,0,38,46,7,9,42,56,56,0,0,61,0,2,0,0,0,38,0,61,1,67,0,0,54,38,0,0,54,6,0,0,61,0,54,0,0,0,0,4,0,0,5,0,10,5,0,0,0,0,0,0,0,63,65,0,10,1,3,10,38,1,54,5,67,63,0,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.91E2,5.1E2,4.81E2,1.85E2,3.25E2,3E1,4.51E2,1.53E2,3.2E1,9.3E1,2.32E2,7E0,2.3E1,3.28E2,1.23E2,7.4E1,7.9E1,1.2E1,2E1,3.7E1,5.6E1,2.22E2,1E1,1.7E1,6E0,1.7E2,1.58E2,8E1,4.3E1,7E0,6.7E1,4.4E1,3.5E1,5E0,7E0,8E0,1.2E1,1.9E1,1.8E1,4.1E1,1.5E1,2.12E2,1E1,5E0,5E0,1.65E2,5E0,1.3E1,1.45E2,5.7E1,2.3E1,3.3E1,1E1,3.3E1,3.4E1,6E0,3.8E1,2E1,1.5E1,7E0,5E0,6E0,1.3E1,8E0,1E1,2.2E1,1.9E1,6E0,9E0,1.82E2,3E1,5E0,5E0,9.9E1,6.6E1,5E0,8E0,1.38E2,7E0,5.2E1,5E0,1.3E1,1E1,9E0,2.4E1,5E0,5E0,2.7E1,6E0,2.5E1,1.3E1,9E0,6E0,7E0,6E0,5E0,5E0,5E0,1.7E1,1.4E1,5E0,1.47E2,3.5E1,1.2E1,1.8E1,5E1,4.9E1,5.6E1,1E1,1.6E1,1.22E2,1.5E1,3.7E1,7E0,1.7E1,1.9E1,8E0,1.4E1,1.1E1,7E0,6E0,1.2E1,5E0,9E0,5E0,5.4E1,9.3E1,3E1,5E0,5E0,7E0,1.1E1,7E0,8E0,4.2E1,2.8E1,2.1E1,4.9E1,7E0,5E0,5E0,7E0,9E0,1.07E2,1.5E1,1.8E1,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[5.324722E-3,-4.77352E-2,1.7297041E-1,-1.391072E-1,1.0215273E-2,2.0755729E-1,-7.778573E-2,-1.831728E-1,-3.7934195E-2,-4.582369E-2,5.5748142E-2,1.5102778E-1,3.3505803E-1,-1.9363861E-1,2.1206336E-2,-1.5741077E-1,-3.4159932E-1,-1.1350703E-1,3.2286483E-3,-1.2425876E-1,-9.4622215E-3,6.568691E-2,-1.23615496E-1,2.0508641E-1,7.854156E-2,4.492007E-1,2.3456067E-1,-1.4271384E-2,-2.414516E-3,1.3340025E-1,-8.723872E-3,4.0971925E-3,-1.6553959E-1,-2.9927562E-3,-4.6117127E-1,-4.9795754E-2,-1.094794E-2,8.639356E-2,-4.1812617E-2,-4.895339E-2,-2.3375256E-1,1.6268326E-2,-8.067312E-2,1.7307828E-1,5.04758E-2,-1.9826356E-3,-9.7838035E-3,9.4011314E-2,2.5780228E-1,9.8854564E-2,-4.281778E-3,5.2906996E-1,1.2087784E-2,3.8161178E-3,2.6426244E-1,-3.3999656E-4,1.1687766E-2,-1.9153623E-1,-9.146823E-2,-3.264083E-2,-6.528557E-3,-7.8345634E-2,8.5821474E-4,3.992614E-2,7.7480716E-3,-1.574628E-2,-8.067201E-3,-9.993916E-2,8.527888E-2,2.2881983E-3,-3.6986184E-1,1.6931138E-3,6.833515E-3,-1.093851E-1,3.7638096E-3,-5.867392E-4,2.4800004E-1,7.411906E-2,-6.445402E-2,-6.1032805E-3,1.2287326E-2,1.5605885E-1,3.430711E-1,1.3656507E-1,-1.0811103E-2,1.8498583E-2,3.1412937E-2,2.9245207E-1,7.2954632E-3,-1.7389433E-1,-2.9423758E-1,-1.4111742E-1,2.5002712E-2,-5.159924E-3,-4.881044E-4,-1.662746E-4,4.5720264E-3,-4.0070284E-2,2.8876446E-3,-7.865754E-3,-6.36296E-2,1.7640525E-3,5.490245E-3,-2.4458295E-2,-5.895043E-3,-5.968956E-2,1.4593504E-2,2.737444E-3,-1.4403549E-1,7.3178113E-3,1.426936E-2,8.185818E-2,-6.889396E-3,-1.5903905E-1,-1.4475918E-2,-3.7539245E-3,3.6046503E-3,-1.7179827E-3,2.291441E-1,1.7525235E-2,7.105306E-3,7.249219E-2,1.7467202E-1,2.856898E-3,-3.7974517E-3,9.518489E-3,1.5814727E-2,-8.785532E-3,-1.8399407E-3,-1.5676267E-2,-8.810931E-3,-2.4453416E-3,-8.114876E-3,4.458286E-3,-3.9805034E-3,1.5470613E-4,-3.7301625E-3,-1.3881242E-3,-4.710002E-3,-5.000888E-3,8.687849E-5,4.122634E-3,-1.6738127E-4,-9.282486E-3,-3.792133E-3,1.2815976E-2,3.2500518E-3,-3.2654314E-3,-9.851317E-3,2.218823E-3,-6.7300205E-3,1.2857753E-2,3.6670486E-3,8.575444E-4,5.495912E-3,9.567719E-3,2.9757535E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":33,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,-1,55,-1,-1,57,-1,59,61,-1,63,65,67,69,71,73,75,77,-1,-1,79,81,83,-1,85,-1,-1,87,-1,-1,89,91,-1,-1,93,-1,95,-1,97,-1,99,101,-1,103,105,-1,107,-1,-1,109,111,113,115,-1,117,119,121,123,-1,-1,125,-1,127,129,131,133,-1,-1,-1,-1,135,-1,-1,137,-1,-1,-1,-1,139,141,-1,143,-1,-1,145,-1,147,149,-1,-1,-1,151,-1,-1,153,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.325332E0,3.7662017E0,1.9675412E0,1.2235289E0,1.1150585E0,1.3764935E0,3.2522482E-1,7.29856E-1,2.647839E-1,5.57819E-1,4.3773967E-1,5.2856445E-1,5.828972E-1,1.9373345E-1,4.0065867E-1,3.4817696E-1,8.317208E-1,2.2228071E-1,2.1338011E-1,4.9977624E-1,2.4867228E-1,3.6534494E-1,8.2729414E-2,4.3834138E-1,2.2078732E-1,2.6906252E-1,1.3208318E-1,0E0,0E0,1.85276E-1,0E0,0E0,2.9442835E-1,0E0,1.2987299E0,4.23934E-2,0E0,6.823175E-2,1.2462015E-1,2.6859227E-1,9.81444E-1,1.9069698E-1,1.7610505E-1,3.92749E-1,5.53675E-1,0E0,0E0,4.6331862E-1,4.0626478E-1,2.2893155E-1,0E0,7.730627E-2,0E0,0E0,3.849733E-2,0E0,0E0,1.6444397E-1,2.5796658E-1,0E0,0E0,3.2262646E-2,0E0,3.6054667E-2,0E0,6.319153E-2,0E0,5.99567E-2,1.2681149E-2,0E0,5.821805E-1,7.293038E-2,0E0,1.8936774E-1,0E0,0E0,5.8998942E-2,3.015719E-1,1.6314907E-1,1.1910305E-1,0E0,3.7538725E-1,1.2634659E-1,8.7444186E-2,8.216136E-2,0E0,0E0,1.879704E-2,0E0,2.0820856E-1,8.6631775E-3,8.3354294E-2,1.184963E-1,0E0,0E0,0E0,0E0,4.4022165E-2,0E0,0E0,2.2504173E-2,0E0,0E0,0E0,0E0,4.756962E-2,1.03038624E-1,0E0,7.440287E-2,0E0,0E0,3.548826E-1,0E0,4.174149E-2,2.0215103E-1,0E0,0E0,0E0,1.1490494E-1,0E0,0E0,4.0006325E-2,7.391149E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,32,32,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,47,47,48,48,49,49,51,51,54,54,57,57,58,58,61,61,63,63,65,65,67,67,68,68,70,70,71,71,73,73,76,76,77,77,78,78,79,79,81,81,82,82,83,83,84,84,87,87,89,89,90,90,91,91,92,92,97,97,100,100,105,105,106,106,108,108,111,111,113,113,114,114,118,118,121,121,122,122],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,-1,56,-1,-1,58,-1,60,62,-1,64,66,68,70,72,74,76,78,-1,-1,80,82,84,-1,86,-1,-1,88,-1,-1,90,92,-1,-1,94,-1,96,-1,98,-1,100,102,-1,104,106,-1,108,-1,-1,110,112,114,116,-1,118,120,122,124,-1,-1,126,-1,128,130,132,134,-1,-1,-1,-1,136,-1,-1,138,-1,-1,-1,-1,140,142,-1,144,-1,-1,146,-1,148,150,-1,-1,-1,152,-1,-1,154,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.068E3,3.000176E5,2.856934E7,1.585814E6,8.69E2,1.6471942E3,1.465E4,9.369565E0,4.76E2,1.5470404E2,1.2694215E1,1.4553613E1,1.4595818E7,2.0312219E5,5.735198E-1,6E0,1.654E3,3.7684184E-1,2.29E2,1.0865825E0,4.066351E0,1.9913E4,8.620714E2,7.257846E2,2.1199985E-7,1.159499E6,7.958E3,-1.4271384E-2,-2.414516E-3,3.112395E6,-8.723872E-3,4.0971925E-3,1.7813443E-1,-2.9927562E-3,1.753E3,3.9157894E0,-1.094794E-2,4.888889E0,2.1E1,1.4407171E8,1.0836275E2,3.8153846E0,2.0045958E6,2.33E2,3.84E2,-1.9826356E-3,-9.7838035E-3,5.941442E6,1.7159091E0,2.5503825E6,-4.281778E-3,2.59672E3,1.2087784E-2,3.8161178E-3,5.6666665E0,-3.3999656E-4,1.1687766E-2,2.3904348E1,4.0911578E2,-3.264083E-2,-6.528557E-3,4.923077E0,8.5821474E-4,3.7029E4,7.7480716E-3,7.09617E6,-8.067201E-3,1.1937528E3,1.288E3,2.2881983E-3,1.5933333E2,1E0,6.833515E-3,1.6507974E8,3.7638096E-3,-5.867392E-4,2.4E0,1.0564016E3,1.594E3,6.185E3,1.2287326E-2,2.5247778E6,3.1083834E5,8.629636E2,6.2394366E0,1.8498583E-2,3.1412937E-2,2.3653846E0,7.2954632E-3,1E0,1.5E1,4.325E0,8.485E3,-5.159924E-3,-4.881044E-4,-1.662746E-4,4.5720264E-3,5.7254577E0,2.8876446E-3,-7.865754E-3,5.1E1,1.7640525E-3,5.490245E-3,-2.4458295E-2,-5.895043E-3,4E0,2.6654E4,2.737444E-3,2.2646774E2,7.3178113E-3,1.426936E-2,1.1160929E0,-6.889396E-3,2.9487667E0,2.2968E4,-3.7539245E-3,3.6046503E-3,-1.7179827E-3,5.929839E8,1.7525235E-2,7.105306E-3,2.9654377E1,3.422351E6,2.856898E-3,-3.7974517E-3,9.518489E-3,1.5814727E-2,-8.785532E-3,-1.8399407E-3,-1.5676267E-2,-8.810931E-3,-2.4453416E-3,-8.114876E-3,4.458286E-3,-3.9805034E-3,1.5470613E-4,-3.7301625E-3,-1.3881242E-3,-4.710002E-3,-5.000888E-3,8.687849E-5,4.122634E-3,-1.6738127E-4,-9.282486E-3,-3.792133E-3,1.2815976E-2,3.2500518E-3,-3.2654314E-3,-9.851317E-3,2.218823E-3,-6.7300205E-3,1.2857753E-2,3.6670486E-3,8.575444E-4,5.495912E-3,9.567719E-3,2.9757535E-3],"split_indices":[2,37,54,9,2,61,38,63,2,61,55,65,56,42,47,3,1,47,10,65,63,9,4,4,46,9,2,0,0,56,0,0,47,0,38,65,0,63,3,5,61,63,37,0,0,0,0,54,62,41,0,61,0,0,63,0,0,67,4,0,0,63,0,1,0,12,0,42,12,0,4,109,0,40,0,0,62,4,2,38,0,37,42,61,63,0,0,63,0,24,3,63,38,0,0,0,0,63,0,0,10,0,0,0,0,8,9,0,61,0,0,63,0,62,38,0,0,0,5,0,0,67,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.34E2,7.1E2,2.24E2,2.75E2,4.35E2,1.97E2,2.7E1,1.91E2,8.4E1,1.95E2,2.4E2,1.38E2,5.9E1,1.2E1,1.5E1,1.66E2,2.5E1,2.9E1,5.5E1,6.1E1,1.34E2,2.28E2,1.2E1,7.8E1,6E1,2.6E1,3.3E1,6E0,6E0,1E1,5E0,5E0,1.61E2,8E0,1.7E1,2E1,9E0,1.9E1,3.6E1,3.7E1,2.4E1,9.9E1,3.5E1,2.7E1,2.01E2,7E0,5E0,2.6E1,5.2E1,5.4E1,6E0,1.7E1,9E0,6E0,2.7E1,5E0,5E0,1.18E2,4.3E1,9E0,8E0,1.4E1,6E0,1.3E1,6E0,3.1E1,5E0,2.7E1,1E1,8E0,1.6E1,9E1,9E0,3E1,5E0,8E0,1.9E1,1.67E2,3.4E1,1.7E1,9E0,2.5E1,2.7E1,4E1,1.4E1,1.1E1,6E0,2E1,7E0,1.03E2,1.5E1,3E1,1.3E1,9E0,5E0,8E0,5E0,2.4E1,7E0,8E0,1.9E1,5E0,5E0,9E0,7E0,1.5E1,7.5E1,5E0,2.5E1,9E0,1E1,1.62E2,5E0,1.1E1,2.3E1,9E0,8E0,7E0,1.8E1,2.2E1,5E0,1.6E1,2.4E1,7E0,7E0,9E0,1.1E1,9.2E1,1.1E1,9E0,6E0,9E0,2.1E1,8E0,5E0,1.2E1,1.2E1,1.1E1,8E0,8E0,7E0,1.4E1,6.1E1,1.2E1,1.3E1,8E0,1.54E2,5E0,6E0,1.6E1,7E0,1.3E1,5E0,8E0,8E0,1.8E1,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-7.3745293E-3,-5.25303E-2,1.6963685E-1,-3.5650786E-2,-3.3253378E-1,3.1174807E-2,1.5314414E-1,-1.7981106E-1,5.114136E-4,-1.8175546E-2,-4.498501E-1,1.0689819E-1,3.0286643E-1,-1.3856898E-2,-1.630174E-1,9.321699E-3,-2.7850938E-1,-4.0046372E-3,3.5122917E-3,-3.2108513E-1,-3.0924488E-2,-1.2776548E-1,1.2838526E-1,3.4702E-1,-1.9473191E-3,-1.2432709E-1,-1.981197E-1,2.7138445E-1,-7.0128136E-4,-1.5028379E-2,-7.466785E-3,-2.403633E-2,-1.4488943E-1,-1.1006103E-2,1.830279E-3,7.713355E-2,2.132973E-1,3.9843765E-1,1.9384238E-1,-1.4822847E-1,-2.6670065E-2,-5.557801E-2,-2.2215195E-1,1.705246E-2,1.6866058E-1,-4.522691E-2,4.770613E-2,-4.004891E-3,-9.226176E-3,2.8443725E-3,1.4029787E-1,1.2762304E-1,3.043632E-1,3.2553345E-1,2.3403965E-2,3.862533E-3,1.3482053E-2,-1.7693429E-1,-5.7674482E-2,1.0078917E-3,-4.399279E-3,-5.3186733E-3,5.908982E-4,-2.3710212E-1,-2.577884E-3,4.6173083E-3,1.0264641E-2,-1.7171396E-2,-1.2018116E-1,2.2372091E-2,1.4814217E-1,-5.664319E-3,2.7747462E-2,2.8557804E-1,7.0009984E-2,1.0451168E-2,2.8002342E-2,1.5977876E-2,9.926639E-3,9.048276E-3,1.65201E-2,-3.6076773E-3,-1.0371873E-2,1.4669483E-3,-5.140728E-3,-9.159358E-3,-1.3971949E-2,-1.2999394E-3,6.928619E-3,1.0297591E-3,-7.5753247E-3,-7.2460426E-3,1.6156359E-3,5.0266697E-3,1.3469068E-2,2.9109698E-3,-1.6839241E-3,7.9077035E-3,1.5131354E-2,5.7681063E-3,-1.597411E-4,5.993807E-3,-4.9221264E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":34,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,-1,25,27,29,-1,-1,31,-1,33,35,37,-1,39,41,43,45,-1,-1,-1,47,-1,-1,49,51,53,55,57,59,61,63,-1,65,67,69,-1,-1,71,73,75,77,79,-1,-1,-1,81,83,-1,-1,-1,-1,85,-1,-1,-1,87,89,91,93,-1,95,97,99,-1,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.786183E0,3.6557705E0,1.5372071E0,3.821333E0,1.6145153E0,0E0,1.3039656E0,2.4467468E-1,1.4455907E0,8.967808E-2,6.786399E-1,7.676965E-1,6.9972086E-1,0E0,1.5328193E-1,1.4973807E0,4.3931007E-2,0E0,0E0,6.883652E-1,0E0,2.4724789E-1,5.7975626E-1,2.3780727E-1,0E0,1.4776587E-1,2.171681E-1,1.4538682E-1,1.1876115E0,0E0,0E0,0E0,2.3921877E-2,0E0,0E0,4.0784198E-1,3.5905838E-1,5.9832573E-2,1.04178965E-1,1.2496555E-1,4.822589E-2,4.583257E-2,1.2879133E-1,0E0,2.160129E-2,6.0146284E-1,6.684207E-1,0E0,0E0,1.3093579E-1,4.5994413E-1,2.659934E-1,1.3118744E-2,1.020956E-2,0E0,0E0,0E0,1.628735E-1,6.990421E-2,0E0,0E0,0E0,0E0,7.272172E-2,0E0,0E0,0E0,3.8336855E-1,4.8206103E-1,4.7660103E-1,2.6844442E-1,0E0,8.0702186E-2,1.44331455E-2,1.3205823E-1,0E0,2.1719515E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,21,21,22,22,23,23,25,25,26,26,27,27,28,28,32,32,35,35,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,57,57,58,58,63,63,67,67,68,68,69,69,70,70,72,72,73,73,74,74,76,76],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,-1,26,28,30,-1,-1,32,-1,34,36,38,-1,40,42,44,46,-1,-1,-1,48,-1,-1,50,52,54,56,58,60,62,64,-1,66,68,70,-1,-1,72,74,76,78,80,-1,-1,-1,82,84,-1,-1,-1,-1,86,-1,-1,-1,88,90,92,94,-1,96,98,100,-1,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.812E3,1.05039425E1,3.3817584E7,3.95E2,2.1E1,3.1174807E-2,3.9813398E6,8.071428E0,3.713753E7,1.83154E6,4.9E1,1.2142857E0,4.9036694E0,-1.3856898E-2,4.96583E-2,1.4E1,1.839E3,-4.0046372E-3,3.5122917E-3,3.3E1,-3.0924488E-2,2.769E3,1.5146261E3,2.1347518E0,-1.9473191E-3,1.7E1,2.90099E0,2.00087E5,1.037E3,-1.5028379E-2,-7.466785E-3,-2.403633E-2,1.6E1,-1.1006103E-2,1.830279E-3,8.484036E5,1.4669823E5,1.7265024E3,1.5382514E1,2.55E2,2.6003375E5,1.33162E6,2.022131E6,1.705246E-2,3.5548245E2,4.064E3,2.8530578E6,-4.004891E-3,-9.226176E-3,1.7302156E5,9.318287E0,1.3667021E0,2.1298597E0,4.9321495E6,2.3403965E-2,3.862533E-3,1.3482053E-2,6.0921145E4,5.1066592E2,1.0078917E-3,-4.399279E-3,-5.3186733E-3,5.908982E-4,2.23329E5,-2.577884E-3,4.6173083E-3,1.0264641E-2,8.216578E9,1.0476191E0,1.8202913E2,2.625E0,-5.664319E-3,8.877906E6,6.6993866E0,2.36433E6,1.0451168E-2,1.9493858E7,1.5977876E-2,9.926639E-3,9.048276E-3,1.65201E-2,-3.6076773E-3,-1.0371873E-2,1.4669483E-3,-5.140728E-3,-9.159358E-3,-1.3971949E-2,-1.2999394E-3,6.928619E-3,1.0297591E-3,-7.5753247E-3,-7.2460426E-3,1.6156359E-3,5.0266697E-3,1.3469068E-2,2.9109698E-3,-1.6839241E-3,7.9077035E-3,1.5131354E-2,5.7681063E-3,-1.597411E-4,5.993807E-3,-4.9221264E-3],"split_indices":[2,63,7,2,3,0,37,61,54,9,3,63,47,0,47,10,2,0,0,3,0,38,61,63,0,3,63,5,2,0,0,0,0,0,0,37,42,4,65,2,37,5,12,0,4,38,37,0,0,37,65,51,47,37,0,0,0,54,42,0,0,0,0,12,0,0,0,40,62,61,62,0,54,65,37,0,60,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.72E2,7.75E2,1.97E2,7.32E2,4.3E1,5E0,1.92E2,1.46E2,5.86E2,1.2E1,3.1E1,1.48E2,4.4E1,1.6E1,1.3E2,5.69E2,1.7E1,7E0,5E0,2.1E1,1E1,1.2E1,1.36E2,3.9E1,5E0,6.4E1,6.6E1,2E1,5.49E2,1.1E1,6E0,9E0,1.2E1,7E0,5E0,8.6E1,5E1,2.8E1,1.1E1,5.1E1,1.3E1,1E1,5.6E1,9E0,1.1E1,2.86E2,2.63E2,7E0,5E0,4E1,4.6E1,2.7E1,2.3E1,1.9E1,9E0,6E0,5E0,3.8E1,1.3E1,8E0,5E0,5E0,5E0,5.1E1,5E0,6E0,5E0,2.09E2,7.7E1,2.11E2,5.2E1,6E0,3.4E1,1.4E1,3.2E1,1.3E1,1.4E1,1.4E1,9E0,5E0,1.4E1,1.3E1,2.5E1,5E0,8E0,3.4E1,1.7E1,1.97E2,1.2E1,1.8E1,5.9E1,1.3E1,1.98E2,4.2E1,1E1,2.2E1,1.2E1,5E0,9E0,1.8E1,1.4E1,8E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[-3.5438866E-3,-5.1759023E-2,1.540646E-1,-1.06116824E-1,2.4333086E-2,1.8449609E-1,-7.5415425E-2,-1.6347049E-1,-5.38245E-2,3.3418555E-2,-1.9434553E-2,4.3803895E-1,1.6196705E-1,-1.4855489E-1,6.6967383E-3,-1.4472872E-1,-4.1616392E-1,1.9567996E-2,-6.934608E-2,-1.6445935E-2,9.9371254E-2,1.4654778E-2,2.6387237E-2,8.307463E-2,2.4262503E-1,-4.9080665E-4,-2.0693235E-1,-1.7269728E-1,-7.771672E-2,-3.0734264E-3,-3.2787554E-2,-1.9779405E-1,-2.5007239E-2,-5.777116E-2,6.800626E-2,4.738236E-2,2.0098306E-1,5.2730583E-2,1.5014308E-2,1.8687798E-1,3.129081E-1,-3.9082975E-3,-1.338293E-2,-8.6158403E-4,-1.8273412E-1,-2.8510608E-2,-1.5836926E-1,-2.6463312E-1,-1.1563532E-1,-5.59278E-3,-1.4476967E-1,-2.0058747E-1,-1.6541854E-2,-1.6991876E-3,1.1110961E-1,9.227991E-2,-2.480113E-2,3.0406623E-3,2.309264E-1,-3.5633948E-2,8.990959E-2,2.65563E-1,9.442305E-2,3.746255E-1,7.894903E-3,-2.7602684E-1,-1.6038238E-1,-1.144069E-1,1.3860038E-2,-1.2420912E-2,-7.04996E-2,-3.2732788E-1,1.4036465E-3,4.7831265E-3,-1.911958E-1,1.1453979E-1,-1.8041825E-2,-9.855936E-3,7.508658E-6,-9.42046E-2,-2.1375539E-2,2.2129526E-2,-1.0473014E-1,7.144034E-2,-4.4826767E-3,6.953625E-2,1.10212965E-2,-3.3435626E-3,1.2835257E-1,-1.2444159E-2,2.8240783E-2,2.5600109E-1,3.3056943E-3,-1.0269912E-3,-5.7756375E-3,1.3566536E-1,-5.0301306E-2,7.326027E-3,1.5298796E-2,1.323595E-1,-1.0224741E-3,9.319926E-3,2.0628275E-2,-7.008345E-3,-1.4100853E-2,-6.826671E-3,-1.4636799E-2,-7.026607E-3,-2.4191155E-3,6.417181E-3,-1.1212823E-3,7.4386096E-4,-5.873497E-3,-1.9989558E-2,-1.0330181E-2,-4.6102605E-3,5.01714E-3,-4.5394176E-3,-1.1736287E-2,7.624093E-3,1.3494608E-3,2.3269784E-3,-1.8171936E-3,-1.6994447E-3,-8.7081315E-3,6.985822E-5,9.946581E-3,-9.106812E-3,-1.3701549E-3,1.318076E-4,5.2876766E-3,7.3307115E-3,1.7047102E-3,7.784748E-3,2.38205E-3,3.6329501E-3,-1.2717356E-3,8.417127E-3,1.5490728E-2,-2.6614792E-3,3.8790458E-3,-2.0031868E-3,7.4099987E-3,2.98306E-3,-8.836686E-3,3.0701945E-3,9.980539E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":35,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,-1,27,29,-1,31,33,35,-1,-1,37,39,-1,41,43,45,-1,-1,47,49,51,53,55,57,59,-1,61,63,-1,-1,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,-1,91,93,95,97,99,101,-1,103,105,107,109,-1,111,113,-1,115,117,119,121,-1,-1,123,-1,125,127,129,-1,131,-1,-1,133,-1,135,137,-1,139,-1,141,143,-1,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.0067587E0,2.922739E0,1.5223331E0,1.2268543E0,1.2018323E0,1.0200577E0,4.3931073E-1,8.6800957E-1,1.6309979E0,9.553566E-1,0E0,5.9878588E-2,1.1061492E0,1.5890604E-1,0E0,3.3143663E-1,1.2745659E0,0E0,1.1967481E0,5.8326983E-1,6.508125E-1,0E0,0E0,6.470696E-1,2.8239632E-1,0E0,1.1570084E-1,1.9616532E-1,2.1772629E-1,0E0,0E0,2.6593518E-1,3.6655974E-1,6.5616226E-1,1.6543838E-1,2.7452102E-1,1.552124E-1,2.7259886E-1,0E0,3.4594047E-1,2.622497E-1,0E0,0E0,0E0,2.0804548E-1,1.3260701E-1,1.828633E-1,5.460458E-1,2.3748523E-1,2.0695488E-1,2.1572155E-1,6.4835095E-1,3.0242953E-1,1.5962104E-1,1.621789E-1,3.1751522E-1,4.336093E-1,0E0,1.17896676E-1,7.6736465E-2,3.789207E-1,1.3445377E-1,1.133682E-1,2.1458936E-1,0E0,3.0742764E-2,1.5911722E-1,1.9775301E-2,1.2164373E-1,0E0,6.512092E-2,1.6410708E-1,0E0,1.29135E-1,6.7487955E-2,5.0517365E-2,1.8201318E-1,0E0,0E0,9.538418E-2,0E0,2.4549048E-1,1.800682E-1,3.759511E-2,0E0,7.2522625E-2,0E0,0E0,1.2194538E-1,0E0,8.0143064E-2,1.0785735E-1,0E0,9.537555E-2,0E0,1.9602275E-1,2.5314033E-1,0E0,0E0,8.982408E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,15,15,16,16,18,18,19,19,20,20,23,23,24,24,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,36,36,37,37,39,39,40,40,44,44,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,61,61,62,62,63,63,65,65,66,66,67,67,68,68,70,70,71,71,73,73,74,74,75,75,76,76,79,79,81,81,82,82,83,83,85,85,88,88,90,90,91,91,93,93,95,95,96,96,99,99],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,-1,28,30,-1,32,34,36,-1,-1,38,40,-1,42,44,46,-1,-1,48,50,52,54,56,58,60,-1,62,64,-1,-1,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,-1,92,94,96,98,100,102,-1,104,106,108,110,-1,112,114,-1,116,118,120,122,-1,-1,124,-1,126,128,130,-1,132,-1,-1,134,-1,136,138,-1,140,-1,142,144,-1,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.167E3,7.785302E5,1E0,6.69E2,1.4676277E7,5.617715E6,1.465E4,5.444797E0,2.00087E5,6.946813E7,-1.9434553E-2,5.4444447E0,1.5484655E6,2.459E3,6.6967383E-3,8.39777E5,3.89E2,1.9567996E-2,1E0,3.2911258E6,7.472532E9,1.4654778E-2,2.6387237E-2,5.5766084E3,9.154E3,-4.9080665E-4,6E0,1.7288135E0,1.44E2,-3.0734264E-3,-3.2787554E-2,2.5454E4,2.0734E4,1.9875325E2,7.478261E0,4.9390244E0,3.1720988E6,2.4361508E6,1.5014308E-2,9.6603775E0,1.4676277E7,-3.9082975E-3,-1.338293E-2,-8.6158403E-4,2.754386E0,4.126333E7,2.1478824E2,5.1303135E1,1.0013907E1,7.9555137E3,2.3922667E1,7.9784E7,2.6818182E0,7.77E2,7.0093E4,1.6494E4,3.1066666E2,3.0406623E-3,6.968415E6,5.896973E1,1.3358269E2,8.1636E2,2.7023384E1,4.7749802E-1,7.894903E-3,2.416E3,2.4E1,2.5E1,2.5E2,-1.2420912E-2,2.701265E8,3.909839E-5,1.4036465E-3,2.6235E4,7.70947E5,6.9508715E0,3.14403E5,-9.855936E-3,7.508658E-6,1.9741463E2,-2.1375539E-2,3.4E1,4.2492856E2,1.27493895E-5,-4.4826767E-3,9.705882E0,1.10212965E-2,-3.3435626E-3,1.9333333E0,-1.2444159E-2,1.4359008E8,4.4971E4,3.3056943E-3,2.4358286E7,-5.7756375E-3,6E0,3.406022E8,7.326027E-3,1.5298796E-2,1.2E1,-1.0224741E-3,9.319926E-3,2.0628275E-2,-7.008345E-3,-1.4100853E-2,-6.826671E-3,-1.4636799E-2,-7.026607E-3,-2.4191155E-3,6.417181E-3,-1.1212823E-3,7.4386096E-4,-5.873497E-3,-1.9989558E-2,-1.0330181E-2,-4.6102605E-3,5.01714E-3,-4.5394176E-3,-1.1736287E-2,7.624093E-3,1.3494608E-3,2.3269784E-3,-1.8171936E-3,-1.6994447E-3,-8.7081315E-3,6.985822E-5,9.946581E-3,-9.106812E-3,-1.3701549E-3,1.318076E-4,5.2876766E-3,7.3307115E-3,1.7047102E-3,7.784748E-3,2.38205E-3,3.6329501E-3,-1.2717356E-3,8.417127E-3,1.5490728E-2,-2.6614792E-3,3.8790458E-3,-2.0031868E-3,7.4099987E-3,2.98306E-3,-8.836686E-3,3.0701945E-3,9.980539E-3],"split_indices":[2,37,6,2,56,12,38,62,5,7,0,63,37,10,0,9,2,0,109,37,40,0,0,4,2,0,8,63,10,0,0,9,38,64,67,65,54,54,0,65,56,0,0,0,63,7,4,65,65,56,65,5,62,2,1,1,61,0,9,67,67,4,65,51,0,9,8,0,38,0,40,47,0,1,9,65,9,0,0,4,0,3,4,47,0,67,0,0,62,0,7,9,0,5,0,65,41,0,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.2E2,7.05E2,2.15E2,4.11E2,2.94E2,1.9E2,2.5E1,1.95E2,2.16E2,2.89E2,5E0,1.4E1,1.76E2,1.9E1,6E0,1.83E2,1.2E1,6E0,2.1E2,1.65E2,1.24E2,9E0,5E0,9E1,8.6E1,6E0,1.3E1,1.28E2,5.5E1,6E0,6E0,5.3E1,1.57E2,1.11E2,5.4E1,8.3E1,4.1E1,8.1E1,9E0,5E1,3.6E1,6E0,7E0,8E0,1.2E2,3.5E1,2E1,2.8E1,2.5E1,1.36E2,2.1E1,2.4E1,8.7E1,2.1E1,3.3E1,5.1E1,3.2E1,8E0,3.3E1,2.4E1,5.7E1,2.6E1,2.4E1,2.4E1,1.2E1,2.1E1,9.9E1,1.1E1,2.4E1,8E0,1.2E1,2.3E1,5E0,1E1,1.5E1,1.2E1,1.24E2,1.4E1,7E0,1.8E1,6E0,6.1E1,2.6E1,1.2E1,9E0,2.6E1,7E0,9E0,4.2E1,5E0,2.7E1,2.8E1,5E0,1.8E1,6E0,4.3E1,1.4E1,1.1E1,1.5E1,1.8E1,6E0,8E0,1.6E1,5E0,1.6E1,9.3E1,6E0,6E0,5E0,5E0,1.9E1,5E0,7E0,1E1,1.3E1,5E0,5E0,7E0,8E0,7E0,5E0,2.9E1,9.5E1,1.2E1,6E0,5.6E1,5E0,1.1E1,1.5E1,5E0,7E0,6E0,2E1,2.7E1,1.5E1,1.4E1,1.3E1,1.6E1,1.2E1,1.1E1,7E0,5E0,3.8E1,8E0,6E0,1.1E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-1.7821062E-3,-6.615195E-2,1.101387E-1,-1.13738805E-1,1.562799E-2,1.4789982E-1,-1.0098559E-1,-1.4494565E-1,4.103081E-3,4.5671263E-3,1.3075827E-2,2.056099E-1,6.2487565E-2,-2.8530288E-2,-3.522016E-1,-1.282241E-1,-3.1298587E-1,5.1665477E-2,-6.124258E-2,-2.4841283E-2,5.8741502E-2,9.387064E-2,2.4974243E-1,2.0134766E-1,2.278225E-2,6.3206375E-2,-1.3718778E-1,-2.1764819E-2,-9.359204E-3,-1.8254972E-1,-9.7501546E-2,-1.8156354E-1,-2.7951859E-2,-3.947427E-3,1.2972091E-1,-1.22242674E-1,3.8989183E-2,1.0150614E-2,-9.808852E-2,7.437163E-2,-6.4314324E-3,-4.975965E-2,1.5344399E-1,3.2893297E-1,1.5289685E-1,1.2704824E-1,1.5582313E-2,5.1958743E-2,-1.1585903E-1,-1.5031781E-2,8.08477E-3,-1.2235053E-2,-9.714237E-3,-1.6056554E-1,-2.6782095E-1,2.778583E-2,-1.3210714E-1,-1.8281315E-2,-8.448514E-2,5.422116E-2,-4.2507187E-2,8.502117E-3,8.680902E-2,-6.284649E-2,-1.665023E-1,-1.3447917E-3,5.1550544E-3,-3.0659726E-2,7.4320875E-2,-1.7883155E-1,2.6063783E-2,9.496328E-2,-1.7719755E-2,-4.8904E-3,8.840643E-4,1.1873886E-3,1.8783146E-1,2.6993883E-1,4.3518478E-1,1.845848E-1,-5.0514387E-3,1.1448833E-2,3.6662865E-2,2.2736827E-2,2.0081502E-1,8.1634185E-5,-7.6565198E-3,2.9503396E-3,-6.660896E-3,-5.730734E-3,4.904387E-3,-2.2154328E-1,-1.05505094E-1,-1.65994E-2,-9.224009E-3,9.6878335E-3,-1.0297419E-2,-1.09338135E-1,-2.6553112E-1,-7.2943037E-3,9.782613E-4,-4.8497407E-4,5.5974624E-3,-4.167604E-3,6.860546E-4,9.1273576E-4,5.8607636E-3,-4.890375E-3,-4.5569765E-4,-1.0251571E-2,-4.456837E-3,-8.3624534E-2,1.2302316E-2,9.715333E-2,-1.8872184E-3,-2.078999E-2,-4.8734535E-2,-2.3794686E-3,4.035705E-3,3.482434E-2,1.2728153E-1,1.0041127E-3,-3.4246033E-3,2.8758734E-3,2.1146782E-1,1.615499E-2,2.1400076E-1,1.0511548E-2,2.229744E-2,2.6698474E-2,2.6502067E-1,3.9680544E-3,-1.2271268E-3,-2.5844067E-2,6.4138226E-2,1.2938679E-2,4.365616E-3,-3.5905035E-3,-1.1184154E-2,-3.5130805E-3,-1.189738E-2,-2.0763157E-3,4.1932454E-3,-4.2894306E-3,-1.4524672E-2,-4.427828E-3,-1.4656183E-2,1.016514E-3,-5.644779E-3,4.937674E-3,-5.412798E-4,2.624125E-3,6.4678104E-3,7.198485E-4,-5.984805E-3,-6.63801E-4,7.805475E-3,8.298993E-3,3.0757915E-3,7.4116215E-3,1.2431321E-2,8.142229E-3,1.4638821E-2,5.084937E-3,-7.942008E-3,4.790253E-3,1.4343547E-2,1.5653255E-4,-7.0998594E-3,6.0639856E-3,1.2511712E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":36,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,25,27,29,31,33,35,37,39,41,43,45,47,49,51,-1,-1,53,55,57,-1,59,61,63,65,67,69,71,-1,73,75,77,79,81,-1,83,85,87,-1,-1,89,91,93,95,97,-1,99,101,103,-1,105,107,109,-1,-1,111,113,115,117,119,121,-1,-1,-1,123,125,127,129,-1,-1,131,133,135,-1,-1,-1,-1,-1,-1,137,139,-1,-1,-1,141,143,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,147,149,151,-1,-1,153,-1,-1,155,157,-1,-1,-1,159,-1,161,-1,-1,163,165,-1,-1,167,169,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.9450502E0,2.3834674E0,2.8211818E0,1.4237771E0,6.641033E-1,1.4609675E0,9.728559E-1,8.196945E-1,2.580198E-1,3.4877294E-1,0E0,8.543339E-1,6.680579E-1,4.3728977E-1,1.164459E-1,4.495268E-1,9.1764593E-1,2.0887478E-1,2.187613E-1,3.6495012E-1,2.4782538E-1,4.5257533E-1,9.2370224E-1,2.2711635E-1,3.943463E-1,2.1111438E-1,3.199519E-1,0E0,0E0,1.4445925E-1,7.884221E-1,3.791706E-1,0E0,6.735082E-2,3.0023426E-2,4.7370672E-2,7.184308E-2,2.5630325E-1,4.6578497E-1,1.3821733E-1,0E0,6.326636E-2,1.5786582E-1,3.1503534E-1,5.094894E-1,1.9651195E-1,0E0,3.4342337E-1,9.857495E-2,1.6069147E-1,0E0,0E0,1.575226E-1,2.5193787E-1,4.6821E-2,2.7788246E-1,4.0235376E-1,0E0,1.18471436E-1,5.2804533E-2,5.0120782E-2,0E0,3.1992413E-2,2.3783877E-2,2.4799973E-2,0E0,0E0,1.3755734E-1,1.0372433E-1,9.589079E-1,9.397775E-2,1.1023903E-1,3.250388E-2,0E0,0E0,0E0,7.315791E-2,1.3117361E-1,1.0884905E-1,6.6600966E-1,0E0,0E0,3.92618E-2,1.3842714E-1,8.2039356E-2,0E0,0E0,0E0,0E0,0E0,0E0,8.8157654E-2,1.8289274E-1,0E0,0E0,0E0,1.2196373E-1,3.9429498E-1,1.411128E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1103198E-1,7.793634E-2,4.6993524E-2,0E0,0E0,1.05120264E-1,0E0,0E0,1.4659427E-1,1.0768658E-1,0E0,0E0,0E0,2.7293444E-2,0E0,6.192243E-2,0E0,0E0,3.3038506E-1,2.1785927E-1,0E0,0E0,1.194177E-1,8.8012606E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,52,52,53,53,54,54,55,55,56,56,58,58,59,59,60,60,62,62,63,63,64,64,67,67,68,68,69,69,70,70,71,71,72,72,76,76,77,77,78,78,79,79,82,82,83,83,84,84,91,91,92,92,96,96,97,97,98,98,111,111,112,112,113,113,116,116,119,119,120,120,124,124,126,126,129,129,130,130,133,133,134,134],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,26,28,30,32,34,36,38,40,42,44,46,48,50,52,-1,-1,54,56,58,-1,60,62,64,66,68,70,72,-1,74,76,78,80,82,-1,84,86,88,-1,-1,90,92,94,96,98,-1,100,102,104,-1,106,108,110,-1,-1,112,114,116,118,120,122,-1,-1,-1,124,126,128,130,-1,-1,132,134,136,-1,-1,-1,-1,-1,-1,138,140,-1,-1,-1,142,144,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,148,150,152,-1,-1,154,-1,-1,156,158,-1,-1,-1,160,-1,162,-1,-1,164,166,-1,-1,168,170,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,7.9016544E5,1E0,4.39776E6,8.785115E6,8E0,3.9416668E1,5.119403E0,5.92E2,1.158E3,1.3075827E-2,6.2024915E2,2.5959E4,1E1,6.0546756E-2,2.71E2,1.4601138E4,1.8543E4,1.221E3,3.7029E4,1E0,6.374563E5,2.1298597E0,1.3631968E3,5.76E2,1.4127854E7,1.0358873E3,-2.1764819E-2,-9.359204E-3,3.14403E5,3.5E1,1E0,-2.7951859E-2,5.13657E6,8.315789E0,2.165E3,3.373828E-1,1.185E3,1.3254E4,2.423301E1,-6.4314324E-3,5.9754E4,7.575958E7,1.5E1,2.9626667E3,8.561815E7,1.5582313E-2,1.9616238E2,1.6182771E-2,1.5512196E1,8.08477E-3,-1.2235053E-2,8.829276E-8,5.4E1,1.8002975E5,2.00087E5,1.8996E4,-1.8281315E-2,1.6254545E1,4.92E2,8.138889E0,8.502117E-3,5.9506125E8,3.1275E4,4.164E3,-1.3447917E-3,5.1550544E-3,2.8266037E2,6.346204E-7,7.478261E0,1.994E3,5.80418E7,5.9986052E7,-4.8904E-3,8.840643E-4,1.1873886E-3,3.221296E6,7.927835E0,1.025173E3,2.8045622E6,-5.0514387E-3,1.1448833E-2,2.5345264E7,1.14E2,2.41E2,8.1634185E-5,-7.6565198E-3,2.9503396E-3,-6.660896E-3,-5.730734E-3,4.904387E-3,2.5E1,6E0,-1.65994E-2,-9.224009E-3,9.6878335E-3,8.69E2,7.042942E7,6.27907E0,-7.2943037E-3,9.782613E-4,-4.8497407E-4,5.5974624E-3,-4.167604E-3,6.860546E-4,9.1273576E-4,5.8607636E-3,-4.890375E-3,-4.5569765E-4,-1.0251571E-2,-4.456837E-3,7.3351803E0,1.8E1,2.72E0,-1.8872184E-3,-2.078999E-2,3.0416667E0,-2.3794686E-3,4.035705E-3,5.747068E2,1.4260629E8,1.0041127E-3,-3.4246033E-3,2.8758734E-3,4.4149057E2,1.615499E-2,1.3388E4,1.0511548E-2,2.229744E-2,4.7126865E1,5E0,3.9680544E-3,-1.2271268E-3,7.4390995E6,1.7479E4,1.2938679E-2,4.365616E-3,-3.5905035E-3,-1.1184154E-2,-3.5130805E-3,-1.189738E-2,-2.0763157E-3,4.1932454E-3,-4.2894306E-3,-1.4524672E-2,-4.427828E-3,-1.4656183E-2,1.016514E-3,-5.644779E-3,4.937674E-3,-5.412798E-4,2.624125E-3,6.4678104E-3,7.198485E-4,-5.984805E-3,-6.63801E-4,7.805475E-3,8.298993E-3,3.0757915E-3,7.4116215E-3,1.2431321E-2,8.142229E-3,1.4638821E-2,5.084937E-3,-7.942008E-3,4.790253E-3,1.4343547E-2,1.5653255E-4,-7.0998594E-3,6.0639856E-3,1.2511712E-3],"split_indices":[2,37,6,9,37,26,65,62,10,2,0,64,9,8,47,2,42,1,2,1,73,37,47,61,8,56,4,0,0,9,0,13,0,9,67,38,47,38,38,67,0,1,7,3,61,12,0,65,66,65,0,0,46,2,54,5,38,0,63,2,67,0,40,1,38,0,0,4,46,67,12,7,5,0,0,0,54,65,4,37,0,0,54,8,8,0,0,0,0,0,0,38,8,0,0,0,2,54,65,0,0,0,0,0,0,0,0,0,0,0,0,67,10,63,0,0,63,0,0,61,7,0,0,0,4,0,2,0,0,67,3,0,0,56,38,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.62E2,6.11E2,3.51E2,3.86E2,2.25E2,2.98E2,5.3E1,3.05E2,8.1E1,2.17E2,8E0,1.77E2,1.21E2,4.2E1,1.1E1,2.79E2,2.6E1,4.7E1,3.4E1,1.41E2,7.6E1,5.1E1,1.26E2,2.6E1,9.5E1,2.3E1,1.9E1,5E0,6E0,9.9E1,1.8E2,1.9E1,7E0,2.8E1,1.9E1,2.1E1,1.3E1,9.6E1,4.5E1,7.1E1,5E0,1.5E1,3.6E1,6.8E1,5.8E1,1.8E1,8E0,7.9E1,1.6E1,1.4E1,9E0,9E0,1E1,8.1E1,1.8E1,3.9E1,1.41E2,5E0,1.4E1,1.1E1,1.7E1,7E0,1.2E1,1E1,1.1E1,7E0,6E0,5.9E1,3.7E1,2.7E1,1.8E1,5.8E1,1.3E1,8E0,7E0,8E0,2.8E1,4.6E1,2.2E1,5.2E1,6E0,7E0,1.1E1,6.7E1,1.2E1,5E0,1.1E1,9E0,5E0,5E0,5E0,3.7E1,4.4E1,6E0,1.2E1,6E0,3.3E1,1.22E2,1.9E1,8E0,6E0,6E0,5E0,9E0,8E0,5E0,7E0,5E0,5E0,5E0,6E0,2.6E1,3.3E1,3.1E1,6E0,8E0,1.9E1,8E0,1E1,2.1E1,3.7E1,8E0,5E0,5E0,2.3E1,1.7E1,2.9E1,5E0,1.7E1,1.8E1,3.4E1,6E0,5E0,3.1E1,3.6E1,6E0,6E0,5E0,3.2E1,3.8E1,6E0,2.5E1,8E0,1.14E2,8E0,5E0,1.4E1,7E0,1.9E1,6E0,2.7E1,1.7E1,1.4E1,1.1E1,8E0,1.6E1,5E0,1.9E1,1.8E1,1.4E1,9E0,2.3E1,6E0,1.3E1,5E0,8E0,2.6E1,2.6E1,5E0,1.2E1,2.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"171","size_leaf_vector":"1"}},{"base_weights":[-3.1278117E-3,-5.3186312E-2,1.4789504E-1,-1.4179991E-1,-1.1112823E-2,1.21386044E-1,4.680955E-1,-1.0455449E-1,-2.8317177E-1,-5.3802878E-3,-2.2018906E-2,1.157692E-2,1.9023281E-1,2.629584E-2,1.0771179E-2,-1.9901685E-1,-4.450367E-2,-4.362233E-1,-1.5826987E-1,-3.7600413E-2,3.9125446E-2,4.8775177E-2,-1.4446151E-1,2.2049604E-1,-7.060776E-2,-2.4522318E-1,-1.3697837E-1,1.6678851E-2,-7.401503E-2,-1.1906926E-2,-2.821674E-2,-1.3949462E-2,-9.649741E-2,-6.109319E-2,4.889942E-2,1.9585899E-1,1.6268795E-2,1.1836507E-2,2.5805147E-2,-1.0117748E-2,-8.449469E-2,1.3100246E-1,2.669613E-1,-1.0489236E-2,1.379305E-3,-2.0452565E-1,-1.7745338E-2,-5.743391E-2,-1.011065E-2,-1.3280424E-1,-8.149641E-3,-3.5478463E-4,-1.4037679E-1,-1.3194026E-1,-3.1112181E-2,8.772914E-2,1.0968449E-2,1.4499289E-1,1.2191233E-2,-2.7675249E-2,8.905421E-2,-1.5101983E-2,1.0644645E-1,-1.7129259E-4,-7.016296E-3,7.390397E-2,1.6000398E-2,2.8743774E-1,1.269751E-1,-1.5001794E-2,-1.6010757E-1,-1.0088765E-1,4.4052626E-4,-7.0763215E-2,-2.0938623E-1,-2.4880562E-2,5.765938E-3,-8.15233E-3,-2.9694359E-3,-2.3328612E-2,-1.7535074E-1,-1.4661758E-2,-9.904874E-3,-7.870888E-3,1.359227E-1,-3.596512E-2,4.497379E-2,2.465707E-3,8.29789E-3,4.271543E-3,-1.2480914E-1,1.2225836E-1,-4.7751673E-2,-6.661066E-2,8.892216E-2,7.52227E-3,3.6034748E-2,1.5638083E-1,-8.121681E-3,1.9425184E-2,2.6077813E-1,3.2060836E-3,8.042708E-3,-8.096395E-3,-3.8544978E-3,-6.792467E-3,-1.7895992E-3,-1.235414E-3,-7.266975E-3,-1.139448E-2,-3.7699684E-3,4.1651954E-3,-2.1895098E-3,9.556621E-4,-4.259035E-3,-6.2699537E-3,-1.7904554E-2,3.9918134E-3,-1.3684983E-3,2.468515E-3,-3.1380567E-3,2.3289637E-3,7.845135E-3,-4.919515E-3,6.838278E-4,3.5628883E-4,2.6793322E-3,4.1098148E-3,-2.4384342E-3,-1.9738812E-3,-1.9458447E-2,8.667316E-3,1.9992245E-3,-9.011034E-3,2.5607906E-3,-1.7524657E-3,-8.151533E-3,5.627477E-3,9.3570974E-4,-2.3396758E-3,5.404992E-3,1.2790009E-2,3.4231278E-3,-8.475555E-3,3.1209534E-3,3.8956893E-3,1.2939392E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":37,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,-1,25,27,29,31,33,35,37,39,41,43,45,47,-1,49,-1,-1,-1,51,53,55,57,59,-1,61,-1,63,65,67,-1,-1,69,-1,71,-1,73,75,-1,77,79,81,83,85,87,-1,89,91,93,95,-1,-1,97,-1,99,101,-1,103,105,-1,107,109,111,-1,-1,-1,113,115,117,-1,119,121,123,125,-1,-1,127,129,131,133,135,137,-1,139,141,143,-1,145,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.288124E0,2.6977012E0,1.9803452E0,1.1972446E0,1.3045328E0,1.6836786E0,2.8735065E-1,1.0462358E0,8.485472E-1,6.998195E-1,0E0,5.120253E-1,1.0970297E0,0E0,0E0,1.7280269E-1,1.3975888E0,4.923153E-1,2.1723461E-1,5.780778E-1,7.312065E-1,3.2990724E-1,5.6601882E-2,4.7650337E-1,2.4068466E-1,1.4516759E-1,1.9991392E-1,0E0,4.1711247E-1,0E0,0E0,0E0,8.079997E-2,4.6956187E-1,8.888152E-2,5.036229E-2,5.781919E-1,0E0,2.1602984E-1,0E0,5.8745176E-2,5.117476E-1,1.928153E-1,0E0,0E0,1.2236071E-1,0E0,5.3306848E-2,0E0,2.5600302E-1,1.1829581E-1,0E0,2.6526272E-2,3.070917E-1,4.7084814E-1,1.39289E-1,5.2689232E-2,4.3342173E-2,0E0,3.5118526E-1,3.147297E-1,2.4168156E-1,8.039954E-2,0E0,0E0,2.4481794E-1,0E0,1.3219452E-1,2.0222247E-2,0E0,1.4318347E-2,2.5546513E-2,0E0,1.2088063E-1,9.597027E-2,1.24441296E-1,0E0,0E0,0E0,6.2034596E-2,3.4583306E-1,2.2058775E-1,0E0,4.3744937E-2,4.836127E-2,5.129429E-2,8.60633E-3,0E0,0E0,4.1663805E-1,6.606786E-1,2.729833E-1,2.2178584E-1,8.773799E-2,3.0929454E-2,0E0,8.238965E-2,1.5700561E-1,2.622861E-1,0E0,1.6852236E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,32,32,33,33,34,34,35,35,36,36,38,38,40,40,41,41,42,42,45,45,47,47,49,49,50,50,52,52,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,65,65,67,67,68,68,70,70,71,71,73,73,74,74,75,75,79,79,80,80,81,81,83,83,84,84,85,85,86,86,89,89,90,90,91,91,92,92,93,93,94,94,96,96,97,97,98,98,100,100],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,-1,26,28,30,32,34,36,38,40,42,44,46,48,-1,50,-1,-1,-1,52,54,56,58,60,-1,62,-1,64,66,68,-1,-1,70,-1,72,-1,74,76,-1,78,80,82,84,86,88,-1,90,92,94,96,-1,-1,98,-1,100,102,-1,104,106,-1,108,110,112,-1,-1,-1,114,116,118,-1,120,122,124,126,-1,-1,128,130,132,134,136,138,-1,140,142,144,-1,146,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.222E3,1.972052E5,5.428706E3,2.2E1,5.6E1,1.000501E6,1.3342042E0,1.318E2,2E0,6.953366E7,-2.2018906E-2,3.592428E7,6.1095314E-8,2.629584E-2,1.0771179E-2,3.89E2,2.04115E5,4.238532E0,5.684E3,3.422592E6,1.0164831E-2,1.6005026E1,1.6E1,5.831829E6,5.071E3,6.737099E7,4.090909E0,1.6678851E-2,1.4912975E-1,-1.1906926E-2,-2.821674E-2,-1.3949462E-2,1E0,1.9875325E2,1.125584E-3,1.194E3,4.8865112E2,1.1836507E-2,7.607E3,-1.0117748E-2,3.8610423E0,1.4791E4,3.0359977E9,-1.0489236E-2,1.379305E-3,2.2E1,-1.7745338E-2,1.8056113E0,-1.011065E-2,4.7176E4,1.3407223E5,-3.5478463E-4,1.3337367E7,3.7760816E5,7.2E1,2.234E3,1.3286492E1,2.8282208E0,1.2191233E-2,4.1618698E2,5.0467205E1,5.615327E5,1.7493458E7,-1.7129259E-4,-7.016296E-3,2.08E3,1.6000398E-2,1.8992E4,6.93729E5,-1.5001794E-2,1.688129E6,1.63757E4,4.4052626E-4,1.629932E1,1.212945E6,3.1096E4,5.765938E-3,-8.15233E-3,-2.9694359E-3,1E0,4.1032645E6,1.9200362E8,-9.904874E-3,1.0526316E0,4.7429975E6,2.1506848E0,5.25E2,2.465707E-3,8.29789E-3,1.9333333E0,1.0029973E8,3.8396227E0,5.974481E1,1.3733966E3,1.45E2,7.52227E-3,1E1,7.6E0,5.11E2,1.9425184E-2,3.0238653E8,3.2060836E-3,8.042708E-3,-8.096395E-3,-3.8544978E-3,-6.792467E-3,-1.7895992E-3,-1.235414E-3,-7.266975E-3,-1.139448E-2,-3.7699684E-3,4.1651954E-3,-2.1895098E-3,9.556621E-4,-4.259035E-3,-6.2699537E-3,-1.7904554E-2,3.9918134E-3,-1.3684983E-3,2.468515E-3,-3.1380567E-3,2.3289637E-3,7.845135E-3,-4.919515E-3,6.838278E-4,3.5628883E-4,2.6793322E-3,4.1098148E-3,-2.4384342E-3,-1.9738812E-3,-1.9458447E-2,8.667316E-3,1.9992245E-3,-9.011034E-3,2.5607906E-3,-1.7524657E-3,-8.151533E-3,5.627477E-3,9.3570974E-4,-2.3396758E-3,5.404992E-3,1.2790009E-2,3.4231278E-3,-8.475555E-3,3.1209534E-3,3.8956893E-3,1.2939392E-2],"split_indices":[2,37,61,3,3,37,47,64,26,7,0,54,46,0,0,1,5,62,9,37,47,67,8,54,2,7,67,0,47,0,0,0,8,64,47,2,64,0,2,0,63,2,5,0,0,2,0,62,0,1,42,0,5,37,8,12,67,63,0,4,67,37,60,0,0,38,0,9,9,0,40,37,0,67,9,9,0,0,0,23,54,40,0,65,37,62,2,0,0,62,5,63,67,4,8,0,3,65,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.62E2,7.23E2,2.39E2,2.32E2,4.91E2,2.22E2,1.7E1,1.85E2,4.7E1,4.86E2,5E0,8.6E1,1.36E2,1.1E1,6E0,7.1E1,1.14E2,2E1,2.7E1,2.82E2,2.04E2,7E1,1.6E1,1.22E2,1.4E1,3.9E1,3.2E1,7E0,1.07E2,1.1E1,9E0,7E0,2E1,2.22E2,6E1,2.5E1,1.79E2,6E0,6.4E1,6E0,1E1,4.3E1,7.9E1,5E0,9E0,3.2E1,7E0,1.7E1,1.5E1,5.6E1,5.1E1,7E0,1.3E1,6.5E1,1.57E2,2.9E1,3.1E1,1.6E1,9E0,1.12E2,6.7E1,4.3E1,2.1E1,5E0,5E0,3.5E1,8E0,6.8E1,1.1E1,7E0,2.5E1,1E1,7E0,3.2E1,2.4E1,4.6E1,5E0,8E0,5E0,1.9E1,4.6E1,1.45E2,1.2E1,1E1,1.9E1,1.3E1,1.8E1,5E0,1.1E1,8.5E1,2.7E1,5.4E1,1.3E1,2.9E1,1.4E1,1.1E1,1E1,1.7E1,1.8E1,9E0,5.9E1,6E0,5E0,2E1,5E0,5E0,5E0,2.2E1,1E1,1.8E1,6E0,7E0,3.9E1,1.2E1,7E0,4E1,6E0,1.8E1,1.27E2,5E0,5E0,6E0,1.3E1,5E0,8E0,5E0,1.3E1,3.4E1,5.1E1,2.2E1,5E0,2.9E1,2.5E1,5E0,8E0,2.4E1,5E0,9E0,5E0,5E0,5E0,6E0,1.1E1,5E0,1.3E1,6E0,5.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"147","size_leaf_vector":"1"}},{"base_weights":[-2.3187522E-3,-5.938009E-2,1.3863993E-1,-1.4307812E-1,-1.6248964E-2,-2.1192896E-1,1.5304929E-1,-1.2071853E-1,-4.2760524E-1,-1.0558964E-1,1.7796542E-2,-1.5728688E-2,-2.2991705E-3,5.141225E-2,1.9442289E-1,-2.2114539E-1,-8.878026E-2,-3.015838E-3,-3.6288437E-2,-1.5205033E-1,-4.7072265E-2,5.7995025E-2,-3.9286125E-2,1.5330866E-1,-4.640057E-3,2.1235698E-1,-8.161577E-3,-1.4701098E-1,-2.6566795E-1,1.12059295E-1,-1.19706616E-1,-8.22367E-2,-1.866748E-1,-2.9622363E-3,-8.373188E-2,3.5951648E-2,1.8188593E-1,1.2042379E-1,-5.4800447E-2,7.3513485E-2,1.3066495E-2,4.9908035E-2,-6.5813325E-2,1.8381229E-1,4.668422E-1,-9.377602E-3,-3.0714436E-3,-3.2167625E-1,-9.765134E-3,-2.5326498E-3,1.7796189E-1,-2.5377458E-1,-9.655731E-2,-7.2965506E-3,-4.849411E-2,-2.135451E-1,-1.06172904E-1,3.5215344E-3,-3.6009844E-2,1.9077973E-3,-1.103463E-1,4.8884004E-2,-7.570782E-2,2.2425513E-1,2.2239708E-3,2.2172118E-3,7.678262E-3,-4.2947743E-2,-8.44797E-3,-8.361497E-4,1.1010515E-1,9.42786E-2,1.8261904E-2,-6.5878844E-3,-1.0646585E-2,2.863476E-1,1.4575894E-1,2.8084911E-2,1.4356109E-2,-1.8086262E-2,-9.632684E-3,1.413058E-2,3.3203156E-3,-2.9206122E-3,-3.51687E-1,-6.928459E-2,-1.2583716E-2,-4.224135E-3,-4.793224E-3,-1.5844324E-1,-2.65431E-1,-7.74809E-3,-1.4008629E-3,-2.8912611E-3,-2.8332588E-4,-2.5902125E-3,-6.1581265E-3,5.8893573E-2,-4.2285444E-3,-7.4229115E-3,7.876268E-4,1.3008083E-1,1.3828998E-2,-1.2796755E-1,-2.1921553E-2,7.0533087E-3,1.6646833E-3,1.2823885E-3,6.737474E-3,3.0789485E-3,-6.4980314E-4,3.1882422E-3,-2.6539383E-3,1.7750986E-2,2.0561412E-1,1.1380542E-1,2.7279285E-1,-2.001982E-2,-8.662722E-3,-4.6935715E-4,-5.5666156E-3,1.4394757E-3,-1.7988042E-3,-9.299535E-3,-3.7544009E-3,-1.4675794E-2,-7.5222068E-3,1.1013321E-2,2.3294208E-3,4.527624E-4,1.0612638E-2,-1.5823275E-3,-7.1255076E-3,-4.592517E-5,-4.061164E-3,1.0854316E-2,3.5704987E-3,7.6447255E-3,3.3000954E-3,1.4630654E-2,7.96233E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":38,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,-1,-1,31,33,35,37,39,41,43,-1,45,47,49,51,53,55,57,59,61,63,65,67,69,-1,71,73,75,77,-1,-1,79,-1,-1,81,83,85,-1,87,89,91,-1,93,-1,95,97,99,101,-1,-1,-1,103,-1,-1,105,107,109,-1,111,113,115,-1,-1,-1,-1,-1,-1,-1,117,119,-1,121,-1,123,125,-1,-1,-1,-1,-1,-1,127,-1,-1,-1,129,-1,131,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,135,137,139,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[7.4804497E0,2.3874884E0,1.3862319E0,1.3798094E0,1.3341212E0,2.1014333E-1,1.0749173E0,6.539333E-1,2.0057812E0,3.218124E-1,7.319728E-1,0E0,0E0,4.3478757E-1,1.2527256E0,1.259768E-1,1.0134232E0,0E0,0E0,1.466049E-1,8.86901E-2,5.0415295E-1,3.3366767E-1,2.557174E-1,1.7012887E-1,1.1854239E0,0E0,8.012444E-2,2.5066614E-2,2.513223E-1,4.099977E-1,5.312574E-2,7.250416E-2,7.049109E-2,1.04220405E-1,2.347441E-1,1.4816701E-1,3.1367794E-2,1.7470846E-1,6.227316E-2,0E0,3.6190517E-2,9.933595E-2,5.786505E-1,1.8326306E-1,0E0,0E0,2.009368E-2,0E0,0E0,1.9370252E-1,3.5155487E-1,5.5754447E-1,0E0,4.3239072E-2,4.9432516E-2,5.502799E-2,0E0,1.4563827E-2,0E0,2.3087412E-2,2.0718545E-1,1.3549325E-1,1.0715687E-1,0E0,0E0,0E0,1.9741078E-1,0E0,0E0,3.498563E-2,3.3318967E-2,2.731118E-2,0E0,5.9344355E-2,2.4760747E-1,4.4793797E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.056151E-2,3.166917E-1,0E0,1.4596167E-2,0E0,4.2366803E-2,2.754283E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.9744807E-1,0E0,0E0,0E0,1.2832695E-1,0E0,4.7630727E-2,1.2487656E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.3085785E-2,1.970619E-1,4.4864655E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,47,47,50,50,51,51,52,52,54,54,55,55,56,56,58,58,60,60,61,61,62,62,63,63,67,67,70,70,71,71,72,72,74,74,75,75,76,76,84,84,85,85,87,87,89,89,90,90,97,97,101,101,103,103,104,104,114,114,115,115,116,116],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,-1,-1,32,34,36,38,40,42,44,-1,46,48,50,52,54,56,58,60,62,64,66,68,70,-1,72,74,76,78,-1,-1,80,-1,-1,82,84,86,-1,88,90,92,-1,94,-1,96,98,100,102,-1,-1,-1,104,-1,-1,106,108,110,-1,112,114,116,-1,-1,-1,-1,-1,-1,-1,118,120,-1,122,-1,124,126,-1,-1,-1,-1,-1,-1,128,-1,-1,-1,130,-1,132,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,136,138,140,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.41E3,1E0,1E0,9.932432E0,5.69E2,7.6588124E-2,6.9706165E2,3.89E2,4.327E3,1.3871127E2,2E0,-1.5728688E-2,-2.2991705E-3,9E0,4.5698816E7,6.1E1,1E0,-3.015838E-3,-3.6288437E-2,8.032258E0,1.9204545E0,1.9505818E8,3.261058E0,7.177158E6,9.5139277E-1,3.5680007E3,-8.161577E-3,3.421543E2,3.887931E0,2.95E2,2.508E2,3.625E0,6.946813E7,1.2561695E-3,4.8679228E5,1.4E1,1.8421166E7,3.202146E5,4.8064E4,1.6005026E1,1.3066495E-2,1.6955555E1,1.4559597E0,9.527559E0,4.8647133E-1,-9.377602E-3,-3.0714436E-3,1.688129E6,-9.765134E-3,-2.5326498E-3,2.00087E5,1.5215946E0,4.217427E7,-7.2965506E-3,2.9266055E0,2.1088235E0,2.0811986E5,3.5215344E-3,1E0,1.9077973E-3,2.4E1,1.1621469E1,2.9073685E2,2.2852762E0,2.2239708E-3,2.2172118E-3,7.678262E-3,7.69E2,-8.44797E-3,-8.361497E-4,2.1935484E0,2.4595E5,3.6024924E7,-6.5878844E-3,5.117647E0,1.6622167E5,4.49239E6,2.8084911E-2,1.4356109E-2,-1.8086262E-2,-9.632684E-3,1.413058E-2,3.3203156E-3,-2.9206122E-3,1.6716E5,3.6828358E0,-1.2583716E-2,1.4473684E-1,-4.793224E-3,1.2358929E2,7.0858893E0,-7.74809E-3,-1.4008629E-3,-2.8912611E-3,-2.8332588E-4,-2.5902125E-3,-6.1581265E-3,1.5476E4,-4.2285444E-3,-7.4229115E-3,7.876268E-4,8.2103E4,1.3828998E-2,1.16E2,4.019139E0,7.0533087E-3,1.6646833E-3,1.2823885E-3,6.737474E-3,3.0789485E-3,-6.4980314E-4,3.1882422E-3,-2.6539383E-3,1.7750986E-2,4.5665024E1,2.6818182E0,1.200096E9,-2.001982E-2,-8.662722E-3,-4.6935715E-4,-5.5666156E-3,1.4394757E-3,-1.7988042E-3,-9.299535E-3,-3.7544009E-3,-1.4675794E-2,-7.5222068E-3,1.1013321E-2,2.3294208E-3,4.527624E-4,1.0612638E-2,-1.5823275E-3,-7.1255076E-3,-4.592517E-5,-4.061164E-3,1.0854316E-2,3.5704987E-3,7.6447255E-3,3.3000954E-3,1.4630654E-2,7.96233E-3],"split_indices":[2,109,16,63,2,51,61,1,1,61,26,0,0,3,56,38,19,0,0,67,62,5,67,54,47,61,0,42,63,2,42,63,7,47,54,8,54,37,38,67,0,67,51,65,51,0,0,40,0,0,5,63,54,0,62,65,54,0,8,0,0,65,61,62,0,0,0,2,0,0,62,9,12,0,63,42,37,0,0,0,0,0,0,0,7,63,0,65,0,64,65,0,0,0,0,0,0,9,0,0,0,1,0,0,62,0,0,0,0,0,0,0,0,0,67,63,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.28E2,6.61E2,2.67E2,2.24E2,4.37E2,1E1,2.57E2,2.09E2,1.5E1,1.2E2,3.17E2,5E0,5E0,7.5E1,1.82E2,4.9E1,1.6E2,8E0,7E0,6.6E1,5.4E1,1.86E2,1.31E2,2.6E1,4.9E1,1.74E2,8E0,2E1,2.9E1,2.1E1,1.39E2,2.3E1,4.3E1,2.5E1,2.9E1,1.59E2,2.7E1,1.1E1,1.2E2,1.7E1,9E0,2.6E1,2.3E1,1.58E2,1.6E1,1.1E1,9E0,1.2E1,1.7E1,6E0,1.5E1,1.9E1,1.2E2,6E0,1.7E1,3.1E1,1.2E1,7E0,1.8E1,5E0,2.4E1,1.43E2,1.6E1,2E1,7E0,5E0,6E0,1.11E2,9E0,5E0,1.2E1,1E1,1.6E1,9E0,1.4E1,4.1E1,1.17E2,7E0,9E0,6E0,6E0,6E0,9E0,7E0,1.2E1,1.05E2,1.5E1,1E1,7E0,1.7E1,1.4E1,6E0,6E0,9E0,9E0,8E0,1.6E1,1.34E2,9E0,8E0,8E0,1E1,1E1,2.1E1,9E1,7E0,5E0,5E0,5E0,6E0,1E1,5E0,9E0,1.7E1,2.4E1,9.5E1,2.2E1,7E0,5E0,4.9E1,5.6E1,5E0,5E0,1E1,7E0,8E0,6E0,5E0,1.29E2,5E0,5E0,5E0,1.6E1,6.9E1,2.1E1,1.9E1,5E0,4.2E1,5.3E1,1.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"141","size_leaf_vector":"1"}},{"base_weights":[4.697021E-3,-3.791382E-2,1.2981285E-1,-1.22879885E-1,1.0934602E-3,1.5777391E-1,-9.350564E-2,-2.15773E-1,-9.1921404E-2,1.6014187E-2,-2.3927586E-1,4.6217E-1,1.3872129E-1,-1.9883636E-1,3.5222534E-2,-1.6495867E-1,-2.6047775E-1,-3.645548E-2,-1.430937E-1,1.6585443E-2,9.826131E-3,-1.640721E-1,-1.5802907E-2,1.36052715E-2,2.7016012E-2,-2.1886533E-2,1.5971488E-1,-2.4825796E-3,-1.2349284E-2,-3.8897812E-3,6.924918E-3,-2.0013584E-1,-2.4127124E-3,-5.439311E-3,-1.4140391E-2,4.0006638E-4,-8.0294505E-2,-1.24141045E-1,-1.7404683E-2,3.8250256E-2,-4.3417886E-2,-2.0720712E-3,-9.584525E-3,3.4454282E-2,-9.160334E-2,2.2562835E-1,1.2221694E-1,-1.0434976E-2,-5.497888E-3,-1.6648289E-2,3.933916E-2,-4.299375E-2,-1.6455139E-1,-1.0085512E-1,-1.2963325E-2,4.3609753E-2,-1.033319E-2,-1.5935448E-1,-1.5469348E-2,-8.7787694E-4,3.0720527E-3,-1.6157109E-3,-6.17658E-3,8.568101E-2,2.6757687E-1,1.5128317E-1,-8.1643514E-2,1.9723784E-4,-4.1610263E-3,4.1125766E-3,-4.0420608E-4,-7.609696E-2,-1.4905265E-3,-1.0853415E-2,-3.1442437E-3,-1.2564687E-1,2.1326484E-2,2.6127825E-2,1.362661E-1,-6.8325244E-2,-2.162448E-1,1.0234809E-2,-2.7046459E-2,8.458074E-3,2.5196873E-2,3.0724028E-1,1.3715689E-1,1.12083256E-1,2.6004896E-1,-9.646885E-3,2.5439907E-3,2.9218858E-3,-1.1563994E-3,-4.974101E-3,-5.3696503E-4,-2.3324771E-3,2.2040377E-3,-2.1539237E-3,-7.354092E-3,4.1742655E-3,-1.522703E-3,1.6225257E-3,-7.512311E-3,7.9007605E-3,-1.9636191E-3,5.817508E-4,-5.58797E-3,-1.1620269E-2,-4.9236403E-3,7.747711E-5,-7.254336E-3,-3.4843064E-3,5.6276927E-3,1.813084E-2,1.0624481E-2,8.739182E-3,3.78711E-3,8.4814E-3,2.3427694E-3,1.5129279E-2,6.5136864E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":39,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,37,-1,39,41,-1,-1,-1,43,45,-1,-1,-1,-1,47,-1,-1,-1,49,51,53,-1,55,57,-1,-1,59,61,63,65,-1,-1,67,69,71,73,75,-1,77,-1,79,81,-1,-1,-1,-1,83,85,87,89,91,-1,-1,-1,93,95,-1,-1,97,99,101,103,105,107,-1,109,-1,111,113,115,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.973685E0,2.3068838E0,1.4946344E0,6.099727E-1,1.7181716E0,1.1595917E0,3.730973E-1,7.970047E-2,4.659444E-1,9.493697E-1,1.5744519E-1,7.2719574E-2,6.7784286E-1,1.3250583E-1,1.8898739E-1,1.0085428E-1,1.355226E-1,1.309047E-1,3.3159733E-1,0E0,6.736169E-1,8.211392E-2,0E0,0E0,0E0,9.742577E-2,4.0990067E-1,0E0,0E0,0E0,0E0,1.5802383E-2,0E0,0E0,0E0,3.0528067E-2,1.09210625E-1,2.6463866E-1,0E0,4.1529647E-1,4.997749E-1,0E0,0E0,2.498098E-2,2.1294318E-2,3.469205E-1,6.937785E-1,0E0,0E0,3.938509E-2,3.3933364E-2,3.6981598E-2,5.7994813E-2,2.2077465E-1,0E0,4.5808595E-1,0E0,1.4058554E-1,3.5029572E-1,0E0,0E0,0E0,0E0,8.889837E-2,2.0615888E-1,4.0147758E-1,2.698755E-1,4.4146057E-2,0E0,0E0,0E0,2.8864682E-2,3.345501E-2,0E0,0E0,1.4878881E-1,5.126404E-2,4.062937E-1,2.8053194E-1,5.6285605E-2,4.1211665E-2,0E0,4.4953185E-1,0E0,1.1511548E-1,1.45823E-1,2.158542E-2,3.2175308E-1,1.5962648E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,25,25,26,26,31,31,35,35,36,36,37,37,39,39,40,40,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,55,55,57,57,58,58,63,63,64,64,65,65,66,66,67,67,71,71,72,72,75,75,76,76,77,77,78,78,79,79,80,80,82,82,84,84,85,85,86,86,87,87,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,38,-1,40,42,-1,-1,-1,44,46,-1,-1,-1,-1,48,-1,-1,-1,50,52,54,-1,56,58,-1,-1,60,62,64,66,-1,-1,68,70,72,74,76,-1,78,-1,80,82,-1,-1,-1,-1,84,86,88,90,92,-1,-1,-1,94,96,-1,-1,98,100,102,104,106,108,-1,110,-1,112,114,116,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,5.02E2,2.856934E7,3.89E2,1E0,7.778E3,1.3655363E2,6.7E1,9.114943E0,2.04115E5,1.9E1,2.35184E5,3.808001E5,2.01E2,1.01E3,4.066655E2,2.416E3,3.883E4,4.909091E0,1.6585443E-2,8E0,7.8661694E5,-1.5802907E-2,1.36052715E-2,2.7016012E-2,6.2136955E0,5.6516E4,-2.4825796E-3,-1.2349284E-2,-3.8897812E-3,6.924918E-3,4.309091E0,-2.4127124E-3,-5.439311E-3,-1.4140391E-2,3.6E2,1.6E1,2.1E1,-1.7404683E-2,4.5E1,7.06E2,-2.0720712E-3,-9.584525E-3,2.12E2,1.500502E6,1E0,2.9366477E0,-1.0434976E-2,-5.497888E-3,4.1665E3,3.1720988E6,2.35E2,6.18826E5,8E0,-1.2963325E-2,5.281628E2,-1.033319E-2,1.9E2,1.0918E4,-8.7787694E-4,3.0720527E-3,-1.6157109E-3,-6.17658E-3,1.0657745E7,4.2222223E0,1.3388E4,1.5E1,1.00019E5,-4.1610263E-3,4.1125766E-3,-4.0420608E-4,4E0,3.741267E7,-1.0853415E-2,-3.1442437E-3,3.88613E-2,2.2581031E0,4.5620965E6,8.695652E0,1.4253987E9,5.7254577E0,1.0234809E-2,2.2968E4,8.458074E-3,1.4342732E2,2.1298597E0,3.812E3,1.1771428E1,8.992806E-4,-9.646885E-3,2.5439907E-3,2.9218858E-3,-1.1563994E-3,-4.974101E-3,-5.3696503E-4,-2.3324771E-3,2.2040377E-3,-2.1539237E-3,-7.354092E-3,4.1742655E-3,-1.522703E-3,1.6225257E-3,-7.512311E-3,7.9007605E-3,-1.9636191E-3,5.817508E-4,-5.58797E-3,-1.1620269E-2,-4.9236403E-3,7.747711E-5,-7.254336E-3,-3.4843064E-3,5.6276927E-3,1.813084E-2,1.0624481E-2,8.739182E-3,3.78711E-3,8.4814E-3,2.3427694E-3,1.5129279E-2,6.5136864E-3],"split_indices":[2,2,54,1,78,12,65,38,67,5,3,1,37,0,8,42,9,9,62,0,26,56,0,0,0,63,9,0,0,0,0,63,0,0,0,2,3,8,0,3,2,0,0,0,9,109,51,0,0,56,54,2,9,8,0,4,0,10,9,0,0,0,0,54,63,2,3,5,0,0,0,8,7,0,0,51,62,56,55,40,63,0,38,0,65,47,2,65,66,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.31E2,6.95E2,2.36E2,2.18E2,4.77E2,2.1E2,2.6E1,5.3E1,1.65E2,4.5E2,2.7E1,1.1E1,1.99E2,1.4E1,1.2E1,2.7E1,2.6E1,8E1,8.5E1,7E0,4.43E2,1.7E1,1E1,6E0,5E0,2.3E1,1.76E2,5E0,9E0,6E0,6E0,2E1,7E0,7E0,1.9E1,4.4E1,3.6E1,8E1,5E0,2.89E2,1.54E2,5E0,1.2E1,1.3E1,1E1,6.2E1,1.14E2,1.4E1,6E0,3.1E1,1.3E1,2.6E1,1E1,7.1E1,9E0,2.84E2,5E0,2.9E1,1.25E2,5E0,8E0,5E0,5E0,1.5E1,4.7E1,1E2,1.4E1,2.6E1,5E0,6E0,7E0,1.4E1,1.2E1,5E0,5E0,5.9E1,1.2E1,2.4E2,4.4E1,1.2E1,1.7E1,5E0,1.2E2,5E0,1E1,3.5E1,1.2E1,7.5E1,2.5E1,7E0,7E0,7E0,1.9E1,9E0,5E0,6E0,6E0,1.8E1,4.1E1,5E0,7E0,2.3E2,1E1,3.7E1,7E0,5E0,7E0,1.2E1,5E0,9.9E1,2.1E1,5E0,5E0,1.5E1,2E1,5E0,7E0,3.4E1,4.1E1,1.5E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[5.1748427E-3,-7.2963454E-2,7.9263896E-2,-1.3539195E-1,-3.398929E-2,5.3549584E-2,2.1455894E-1,-1.1848276E-1,-2.838105E-1,-1.8889643E-2,-1.4561841E-1,7.9206206E-2,-5.6072973E-2,2.9706982E-1,2.1152245E-2,-1.4655322E-1,-5.0428245E-2,-4.2044125E-3,-1.9919576E-2,1.1154693E-2,-4.833566E-2,-8.4879816E-2,-1.4460578E-2,1.7075788E-2,7.123446E-2,-1.9773652E-1,1.0700688E-2,2.4472103E-1,2.2688221E-2,1.0369539E-1,-7.758624E-3,-1.6458486E-1,-7.711226E-2,8.920223E-3,-8.788984E-2,-1.259547E-2,6.490745E-2,-1.0488898E-2,-3.807865E-2,3.2908344E-3,-1.3277254E-1,2.8063955E-2,1.3773991E-1,-1.0374781E-1,-2.2909282E-2,-6.9533706E-2,9.341843E-2,1.7844763E-2,1.6463618E-1,-3.049887E-4,9.961685E-3,-1.0912359E-3,-1.7415325E-1,-3.2581765E-2,-6.732687E-3,-4.682849E-2,-1.12458095E-2,1.1622083E-2,-1.7299621E-1,9.760557E-2,-1.9521967E-2,-6.826687E-2,1.2399426E-2,-8.573768E-3,-1.1673942E-3,-3.0461771E-2,6.1356824E-2,-3.373961E-2,1.6168311E-1,2.7266468E-3,-1.7267045E-1,-1.504578E-1,5.7646915E-2,-3.1171404E-3,1.5015052E-1,2.0225395E-1,3.0351463E-3,-1.9442773E-1,-1.0214045E-1,2.8674086E-4,-2.7994092E-3,-7.7458792E-3,5.7886736E-3,-5.0429612E-2,4.9861692E-2,-1.7386896E-3,-1.402537E-2,5.9468253E-3,6.476484E-2,1.4153444E-3,-2.7659654E-3,-9.403926E-2,3.9089385E-2,3.536064E-2,-5.8991844E-3,8.8110566E-2,-7.518099E-2,9.3898416E-2,-1.50227705E-2,-6.450635E-3,2.1714265E-2,1.0729345E-1,2.0126174E-1,-1.0850349E-2,-1.9455998E-3,-3.1596102E-4,-2.051833E-1,-7.886456E-4,5.2613234E-3,2.0303212E-1,1.8021018E-3,2.4846128E-1,1.7457418E-3,-8.066516E-3,-1.7225055E-2,-1.6697417E-3,-7.3779286E-3,2.8039254E-3,-2.4824298E-3,-3.7827126E-3,1.0682661E-3,-1.025271E-3,3.7737708E-3,-1.0250336E-4,4.8798327E-3,-1.7456214E-3,-6.7695607E-3,-2.5852849E-3,5.5224295E-3,6.10497E-3,-1.06554E-3,-1.3473782E-3,6.9023473E-3,-5.945785E-3,-3.016042E-4,5.290665E-3,-3.829679E-3,-8.478264E-3,6.56791E-4,-6.126687E-4,2.252337E-3,6.3358583E-3,-2.0606627E-3,7.4637067E-3,1.1616125E-2,-1.3157635E-2,-5.6023956E-3,4.1008405E-3,1.1920325E-2,1.4536618E-2,7.969079E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":40,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,35,37,39,-1,-1,41,43,45,47,-1,49,-1,51,53,-1,55,57,59,-1,61,-1,63,65,67,69,-1,71,73,-1,75,-1,-1,-1,77,79,-1,81,-1,83,85,87,89,91,93,-1,-1,95,97,99,101,-1,103,105,107,-1,109,111,-1,113,115,-1,-1,-1,117,119,121,-1,-1,-1,123,-1,-1,125,127,129,-1,131,133,135,137,-1,139,141,143,-1,-1,-1,145,-1,-1,147,-1,149,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.673403E0,1.1559865E0,1.7384572E0,4.2563677E-1,4.944896E-1,1.1969602E0,1.2703688E0,3.097229E-1,4.670689E-1,2.3144221E-1,3.265146E-1,7.657857E-1,7.6832664E-1,4.429183E-1,4.060476E-1,1.3380814E-1,4.7144994E-1,0E0,0E0,1.667803E-1,2.320131E-1,2.1088734E-1,0E0,0E0,9.608153E-1,6.721232E-1,3.7815696E-1,4.519472E-1,0E0,2.217254E-1,0E0,1.2008381E-1,7.399134E-2,0E0,2.677489E-1,3.555013E-1,1.12332776E-1,0E0,1.928507E-1,0E0,1.1288524E-1,4.0130937E-1,5.474167E-1,2.4536358E-1,0E0,3.0714077E-1,2.6519853E-1,0E0,1.02478504E-1,0E0,0E0,0E0,1.0570288E-1,1.8659363E-2,0E0,2.2848222E-1,0E0,1.923411E-1,1.9891724E-1,2.0804524E-2,2.5884708E-2,2.2225529E-1,1.5828663E-1,0E0,0E0,4.0430897E-1,3.2701293E-1,1.0227217E-1,2.2811556E-1,0E0,1.1016616E-1,1.4010474E-1,5.1859647E-2,0E0,1.15727484E-1,1.5800798E-1,0E0,1.7559266E-1,7.328196E-2,0E0,0E0,0E0,8.7465204E-2,7.268246E-2,1.15025856E-1,0E0,0E0,0E0,4.3530017E-2,0E0,0E0,1.8175739E-1,1.2787892E-1,2.3871201E-1,0E0,1.5352403E-1,1.9923022E-1,3.3609825E-1,1.9831297E-1,0E0,1.1882532E-2,2.3242414E-1,9.0632915E-2,0E0,0E0,0E0,5.1811397E-2,0E0,0E0,6.376922E-2,0E0,3.0739307E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,24,24,25,25,26,26,27,27,29,29,31,31,32,32,34,34,35,35,36,36,38,38,40,40,41,41,42,42,43,43,45,45,46,46,48,48,52,52,53,53,55,55,57,57,58,58,59,59,60,60,61,61,62,62,65,65,66,66,67,67,68,68,70,70,71,71,72,72,74,74,75,75,77,77,78,78,82,82,83,83,84,84,88,88,91,91,92,92,93,93,95,95,96,96,97,97,98,98,100,100,101,101,102,102,106,106,109,109,111,111],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,36,38,40,-1,-1,42,44,46,48,-1,50,-1,52,54,-1,56,58,60,-1,62,-1,64,66,68,70,-1,72,74,-1,76,-1,-1,-1,78,80,-1,82,-1,84,86,88,90,92,94,-1,-1,96,98,100,102,-1,104,106,108,-1,110,112,-1,114,116,-1,-1,-1,118,120,122,-1,-1,-1,124,-1,-1,126,128,130,-1,132,134,136,138,-1,140,142,144,-1,-1,-1,146,-1,-1,148,-1,150,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.972052E5,1.6471942E3,5.035714E0,3.3706608E0,1.5187539E7,2.2040408E7,1.5341322E2,3.89E2,2.0131578E0,2.0303884E6,2.04115E5,7.472532E9,3.8776313E3,4.1831533E8,5.1578946E0,6.511E3,-4.2044125E-3,-1.9919576E-2,5E0,1.1379E4,3.0620155E0,-1.4460578E-2,1.7075788E-2,1.836095E6,2.617E3,1.038946E6,1.6386554E0,2.2688221E-2,2.4199E4,-7.758624E-3,1.0677966E0,6.642857E0,8.920223E-3,1.2951E4,9.51E3,1E0,-1.0488898E-2,2.0480016E7,3.2908344E-3,4.4561405E0,1E0,5.1604336E7,7.457162E2,-2.2909282E-2,1.9212E4,3.1066666E2,1.7844763E-2,9.44363E5,-3.049887E-4,9.961685E-3,-1.0912359E-3,1.295313E6,5.707547E0,-6.732687E-3,9.4E1,-1.12458095E-2,4.97E2,1E0,1.95E2,1.2E1,3.2921512E2,2.8302418E10,-8.573768E-3,-1.1673942E-3,2.28E4,2.0734E4,4.353644E6,2.43E2,2.7266468E-3,9.643839E6,1.6438356E1,2E1,-3.1171404E-3,3.8908466E1,1.0657745E7,3.0351463E-3,4.68418E5,6.5344668E4,2.8674086E-4,-2.7994092E-3,-7.7458792E-3,4.5172415E0,9.874E3,1E0,-1.7386896E-3,-1.402537E-2,5.9468253E-3,1.925508E6,1.4153444E-3,-2.7659654E-3,1.1074469E6,8.926338E3,2.2823529E2,-5.8991844E-3,2.8515152E2,8.315271E0,2.2E1,2.4199E4,-6.450635E-3,1.574E3,1.84146E5,4.9321495E6,-1.0850349E-2,-1.9455998E-3,-3.1596102E-4,2.4571428E1,-7.886456E-4,5.2613234E-3,4.1718604E2,1.8021018E-3,1.3899029E7,1.7457418E-3,-8.066516E-3,-1.7225055E-2,-1.6697417E-3,-7.3779286E-3,2.8039254E-3,-2.4824298E-3,-3.7827126E-3,1.0682661E-3,-1.025271E-3,3.7737708E-3,-1.0250336E-4,4.8798327E-3,-1.7456214E-3,-6.7695607E-3,-2.5852849E-3,5.5224295E-3,6.10497E-3,-1.06554E-3,-1.3473782E-3,6.9023473E-3,-5.945785E-3,-3.016042E-4,5.290665E-3,-3.829679E-3,-8.478264E-3,6.56791E-4,-6.126687E-4,2.252337E-3,6.3358583E-3,-2.0606627E-3,7.4637067E-3,1.1616125E-2,-1.3157635E-2,-5.6023956E-3,4.1008405E-3,1.1920325E-2,1.4536618E-2,7.969079E-3],"split_indices":[2,37,61,62,62,54,60,61,2,62,54,5,40,61,41,63,1,0,0,8,9,67,0,0,37,38,9,63,0,38,0,62,63,0,38,38,23,0,5,0,62,24,7,4,0,10,61,0,9,0,0,0,9,63,0,10,0,2,8,10,3,61,40,0,0,1,38,54,0,0,56,65,3,0,67,54,0,9,56,0,0,0,63,1,109,0,0,0,37,0,0,54,42,61,0,61,65,3,38,0,2,1,37,0,0,0,65,0,0,61,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.78E2,4.76E2,5.02E2,1.82E2,2.94E2,4.23E2,7.9E1,1.65E2,1.7E1,2.6E2,3.4E1,3.43E2,8E1,5.5E1,2.4E1,1.16E2,4.9E1,8E0,9E0,1.29E2,1.31E2,2.6E1,8E0,8E0,3.35E2,2.5E1,5.5E1,4.5E1,1E1,1.7E1,7E0,9.1E1,2.5E1,6E0,4.3E1,9E1,3.9E1,6E0,1.25E2,6E0,2E1,2.04E2,1.31E2,2E1,5E0,2.8E1,2.7E1,1.5E1,3E1,9E0,8E0,6E0,8.5E1,1.6E1,9E0,3.5E1,8E0,7.9E1,1.1E1,2.8E1,1.1E1,7.8E1,4.7E1,1.3E1,7E0,7.4E1,1.3E2,1.6E1,1.15E2,6E0,1.4E1,1.7E1,1.1E1,7E0,2E1,2.1E1,9E0,6.5E1,2E1,7E0,9E0,1E1,2.5E1,3E1,4.9E1,6E0,5E0,1.3E1,1.5E1,5E0,6E0,6.3E1,1.5E1,4.1E1,6E0,2E1,5.4E1,9.1E1,3.9E1,5E0,1.1E1,5E1,6.5E1,9E0,5E0,5E0,1.2E1,5E0,6E0,1.3E1,7E0,1.6E1,5E0,6E1,5E0,1E1,1E1,1.3E1,1.2E1,2.1E1,9E0,1.5E1,3.4E1,6E0,9E0,3.1E1,3.2E1,7E0,8E0,1.5E1,2.6E1,7E0,1.3E1,3E1,2.4E1,8.2E1,9E0,5E0,3.4E1,5E0,6E0,4.2E1,8E0,3.8E1,2.7E1,5E0,7E0,5E0,8E0,7E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"151","size_leaf_vector":"1"}},{"base_weights":[7.2910385E-3,-3.4655508E-2,1.3922034E-1,-8.356093E-2,3.2721385E-2,2.255241E-2,1.2148786E-1,-6.333539E-2,-2.6176608E-1,1.4067924E-1,1.4265442E-2,-6.566347E-2,1.3824098E-1,-1.4500721E-1,-2.818448E-2,-4.7308606E-1,-1.1901772E-1,6.8008766E-2,1.9609527E-1,-1.4249173E-1,2.5585458E-2,2.3879418E-3,-1.3827711E-1,6.590594E-2,1.6702084E-1,-1.1963786E-2,-1.2809823E-1,3.2897514E-1,-4.760655E-2,-1.1492474E-2,-2.9970359E-2,-1.8784885E-1,-3.0441977E-2,1.2286828E-1,-2.7497616E-3,1.2475379E-2,1.4603281E-1,-2.1765549E-2,-1.7646458E-2,-2.3516517E-2,5.1797103E-2,-7.873677E-3,-3.645995E-3,1.0662328E-1,-9.270513E-3,2.4256954E-1,1.15910225E-1,-1.0174225E-1,-1.7564416E-1,7.184138E-3,1.8954964E-2,-9.0828754E-2,9.673162E-3,-3.0400713E-3,-1.3302568E-2,3.1436333E-3,-4.6530208E-3,1.760384E-3,7.495292E-3,7.8278966E-2,1.0748018E-2,-5.366582E-3,2.3838351E-3,4.6725776E-2,-8.7139554E-2,6.000364E-2,-7.960907E-3,1.3723107E-1,-1.3518827E-3,-5.224386E-3,7.092373E-2,-2.8883826E-3,2.7250066E-1,1.6612446E-1,2.1274742E-2,-1.4462319E-1,-5.717957E-2,-1.9424352E-1,-3.4102493E-3,-6.49898E-2,-1.9283454E-1,-5.2541874E-2,3.12944E-2,1.2249198E-3,5.433925E-3,-4.993674E-2,7.8493334E-2,-2.8267382E-2,-1.4478272E-1,1.07382685E-1,2.710723E-2,-5.643045E-4,1.6534688E-1,-1.1613346E-3,6.2185815E-3,2.9622447E-1,6.53609E-3,1.2370663E-1,1.3483733E-2,-2.2667166E-2,9.997564E-3,-4.7893086E-3,-1.02919275E-2,3.0393005E-4,-6.1835637E-3,-9.859357E-3,-4.124358E-3,-4.847672E-3,9.693129E-4,-1.1074759E-2,-1.9787655E-3,7.438393E-4,-5.9000617E-3,2.0855297E-3,-4.747602E-3,-3.5962036E-3,-6.516939E-4,4.71652E-3,-8.691601E-4,-2.9642012E-3,3.125353E-3,-3.1080912E-3,-1.1099685E-2,5.9587723E-3,-1.6457454E-3,-2.8124606E-4,3.4353628E-3,3.1312017E-3,9.41911E-3,1.1498319E-2,1.6350528E-2,7.798376E-3,1.8907778E-3,-4.7211517E-3,4.6860776E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":41,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,29,31,33,35,37,39,-1,41,43,45,-1,47,49,51,-1,-1,53,55,57,-1,-1,59,61,-1,63,65,-1,-1,67,69,71,73,75,77,-1,-1,79,81,-1,-1,-1,-1,-1,-1,83,-1,-1,-1,85,87,89,-1,91,-1,-1,93,-1,95,97,99,101,103,105,-1,107,109,111,113,-1,-1,115,117,119,121,123,125,-1,127,-1,-1,129,-1,131,-1,133,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.345114E0,2.418039E0,1.3739333E0,1.5152469E0,6.136645E-1,0E0,7.082641E-1,1.095444E0,1.2408144E0,1.6788375E-1,4.738865E-1,1.6871369E-1,4.157598E-1,1.8638468E-1,1.8816552E0,5.02285E-1,1.5889427E-1,1.539048E-1,5.1817477E-2,5.085324E-1,3.204049E-1,0E0,9.171933E-3,1.8464753E-1,5.3563976E-1,0E0,1.099478E-1,1.306721E-1,6.34672E-1,0E0,0E0,1.6259593E-1,9.5946066E-2,4.313436E-2,0E0,0E0,8.047253E-2,9.522191E-2,0E0,3.9287713E-1,3.0725712E-1,0E0,0E0,1.6605443E-1,1.9028011E-1,5.4613256E-1,4.1939807E-1,1.2167108E-1,5.2218556E-2,0E0,0E0,3.7228334E-1,1.5095772E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.90693E-2,0E0,0E0,0E0,1.3289046E-1,1.5201193E-1,2.417894E-1,0E0,1.3642061E-1,0E0,0E0,8.2063496E-2,0E0,1.2378454E-1,2.7029324E-1,2.779465E-1,8.576578E-2,1.7059207E-1,3.6857367E-2,0E0,4.0234193E-1,1.8951297E-1,1.5042189E-1,1.5601042E-1,0E0,0E0,9.795675E-3,7.395092E-2,8.604433E-2,1.5295622E-1,1.969207E-1,1.4677495E-1,0E0,8.5793674E-2,0E0,0E0,3.2308817E-2,0E0,1.5876389E-1,0E0,7.09923E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,26,26,27,27,28,28,31,31,32,32,33,33,36,36,37,37,39,39,40,40,43,43,44,44,45,45,46,46,47,47,48,48,51,51,52,52,59,59,63,63,64,64,65,65,67,67,70,70,72,72,73,73,74,74,75,75,76,76,77,77,79,79,80,80,81,81,82,82,85,85,86,86,87,87,88,88,89,89,90,90,92,92,95,95,97,97,99,99],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,30,32,34,36,38,40,-1,42,44,46,-1,48,50,52,-1,-1,54,56,58,-1,-1,60,62,-1,64,66,-1,-1,68,70,72,74,76,78,-1,-1,80,82,-1,-1,-1,-1,-1,-1,84,-1,-1,-1,86,88,90,-1,92,-1,-1,94,-1,96,98,100,102,104,106,-1,108,110,112,114,-1,-1,116,118,120,122,124,126,-1,128,-1,-1,130,-1,132,-1,134,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.414E3,7.502167E5,4.70127E5,5.119403E0,7E0,2.255241E-2,1.7302156E5,3.51E2,2E0,1.515E3,2.2953334E2,4.36853E5,1.0089981E3,8.071428E0,2.04115E5,2.6E1,7.86127E6,1.390602E8,4.8E2,2.1E1,2.732381E2,2.3879418E-3,6.1092503E1,4.9469455E6,1.9619734E0,-1.1963786E-2,3.1E1,1.5E1,1.380863E6,-1.1492474E-2,-2.9970359E-2,2.49E2,3.1892856E2,3.3512E4,-2.7497616E-3,1.2475379E-2,1E0,9.48E0,-1.7646458E-2,7.3351803E0,1.4650772E7,-7.873677E-3,-3.645995E-3,1.49E2,7.12274E5,2.6844707E8,1.214874E2,3.89E2,6.946813E7,7.184138E-3,1.8954964E-2,1.3439851E7,1.5199841E2,-3.0400713E-3,-1.3302568E-2,3.1436333E-3,-4.6530208E-3,1.760384E-3,7.495292E-3,1E1,1.0748018E-2,-5.366582E-3,2.3838351E-3,4.48E2,1.22896E5,2.8E0,-7.960907E-3,1.2003246E6,-1.3518827E-3,-5.224386E-3,8.620714E2,-2.8883826E-3,2E0,3.4915986E3,4.7263342E2,5.853E3,4.4615383E0,3.7666667E0,-3.4102493E-3,3.355102E2,1.862E3,2.6846153E-1,2.3783188E6,1.2249198E-3,5.433925E-3,4.5555815E6,1.238E3,4.0705118E6,2.1153846E0,6.1114804E-8,7.969174E7,-5.643045E-4,4.577732E2,-1.1613346E-3,6.2185815E-3,1.7265024E3,6.53609E-3,1.302273E6,1.3483733E-2,6.95E2,9.997564E-3,-4.7893086E-3,-1.02919275E-2,3.0393005E-4,-6.1835637E-3,-9.859357E-3,-4.124358E-3,-4.847672E-3,9.693129E-4,-1.1074759E-2,-1.9787655E-3,7.438393E-4,-5.9000617E-3,2.0855297E-3,-4.747602E-3,-3.5962036E-3,-6.516939E-4,4.71652E-3,-8.691601E-4,-2.9642012E-3,3.125353E-3,-3.1080912E-3,-1.1099685E-2,5.9587723E-3,-1.6457454E-3,-2.8124606E-4,3.4353628E-3,3.1312017E-3,9.41911E-3,1.1498319E-2,1.6350528E-2,7.798376E-3,1.8907778E-3,-4.7211517E-3,4.6860776E-4],"split_indices":[2,37,12,62,3,0,37,2,26,2,42,1,4,61,5,0,5,7,10,3,4,0,67,56,62,0,10,0,9,0,0,2,61,1,0,0,92,65,0,67,60,0,0,8,9,7,67,1,7,0,0,54,61,0,0,0,0,0,0,8,0,0,0,2,9,63,0,60,0,0,4,0,6,4,65,9,63,62,0,61,2,47,60,0,0,54,2,37,62,46,7,0,4,0,0,4,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.64E2,7.32E2,2.32E2,4.24E2,3.08E2,1E1,2.22E2,3.82E2,4.2E1,4.4E1,2.64E2,1.8E1,2.04E2,1.14E2,2.68E2,1.6E1,2.6E1,2E1,2.4E1,1.7E1,2.47E2,7E0,1.1E1,5.9E1,1.45E2,1.3E1,1.01E2,1.3E1,2.55E2,8E0,8E0,1.4E1,1.2E1,1.4E1,6E0,8E0,1.6E1,1.2E1,5E0,8.6E1,1.61E2,6E0,5E0,3.8E1,2.1E1,5.7E1,8.8E1,6.7E1,3.4E1,5E0,8E0,1.45E2,1.1E2,7E0,7E0,5E0,7E0,5E0,9E0,1E1,6E0,5E0,7E0,4.1E1,4.5E1,1.56E2,5E0,3.1E1,7E0,9E0,1.2E1,5E0,5.2E1,5.7E1,3.1E1,3.3E1,3.4E1,2.8E1,6E0,1.17E2,2.8E1,2.8E1,8.2E1,5E0,5E0,1E1,3.1E1,2.3E1,2.2E1,6.3E1,9.3E1,5E0,2.6E1,5E0,7E0,4.3E1,9E0,4.4E1,1.3E1,2.6E1,5E0,2.3E1,1E1,1.9E1,1.5E1,2.3E1,5E0,8E1,3.7E1,2.1E1,7E0,1.5E1,1.3E1,7.5E1,7E0,5E0,5E0,2.5E1,6E0,1.7E1,6E0,1.3E1,9E0,5.5E1,8E0,5.5E1,3.8E1,8E0,1.8E1,2.6E1,1.7E1,2.8E1,1.6E1,7E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"135","size_leaf_vector":"1"}},{"base_weights":[-2.0761607E-4,-1.1506809E-1,4.1810844E-2,-8.547752E-2,-2.2080752E-1,1.4569728E-3,1.479651E-1,-1.0623483E-1,1.2222449E-1,-1.5791568E-1,-4.6373636E-1,-1.5104347E-2,8.180046E-2,2.5405103E-2,1.3017525E-1,4.0411863E-2,-1.14796646E-1,1.116748E-2,-3.9230735E-4,-2.2847338E-2,-1.948E-1,-2.8463949E-2,-1.0984051E-2,-8.667876E-3,-2.1588968E-2,2.4604717E-2,1.4982024E-1,1.4649992E-1,-8.318124E-2,-1.4561465E-3,4.893805E-3,-1.399883E-2,-1.0756721E-1,-4.6355044E-3,2.6919825E-3,-2.1466492E-1,-2.7699505E-3,-8.312405E-2,1.8829444E-2,4.134817E-2,-4.745797E-3,1.7225565E-1,-2.7946636E-4,1.07161954E-1,2.1826E-1,1.5478322E-3,-8.108582E-3,3.5274543E-3,-1.1586697E-1,-1.0811287E-2,-4.6127667E-3,3.907871E-3,-9.8270155E-2,-5.0926413E-3,6.887753E-2,1.0947844E-2,8.898705E-2,1.20088086E-1,2.5130168E-1,3.8981225E-2,1.3529977E-1,3.0889362E-1,1.6295502E-1,-1.3240755E-1,-6.1280973E-2,-8.509573E-2,-1.398782E-2,1.0281369E-2,-1.3693789E-1,1.20371506E-1,5.81131E-3,5.1619112E-2,-3.3410504E-2,3.2569412E-2,7.6526958E-3,6.4111136E-2,1.1083735E-2,7.3182727E-3,1.4336471E-2,-4.086766E-2,7.728145E-2,1.5069994E-1,-3.8195492E-3,7.4810823E-3,1.8500285E-2,1.8596315E-1,7.338987E-5,-4.298882E-3,-7.676416E-3,-5.0387025E-4,-5.9790076E-3,-2.6336138E-3,-7.4056187E-3,-5.51255E-4,3.270329E-3,-1.6419046E-2,-1.553192E-3,-3.0423386E-4,6.59409E-3,5.6344764E-3,-1.3771039E-3,4.0869E-3,-1.2223724E-4,-5.1579913E-3,1.8003442E-3,2.6814248E-3,8.911104E-5,3.9652823E-3,4.978042E-4,2.775213E-3,-5.6294426E-3,6.4846785E-3,-1.1210662E-3,5.590145E-3,9.629356E-3,1.0160744E-2,4.361972E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":42,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,-1,27,29,31,-1,-1,33,35,-1,-1,37,-1,39,41,43,45,-1,-1,-1,47,-1,-1,49,-1,51,53,55,-1,57,-1,59,61,-1,-1,-1,63,-1,-1,-1,65,67,69,71,73,75,77,79,81,83,85,87,89,91,-1,93,95,97,99,101,103,105,-1,107,-1,-1,-1,109,111,113,-1,-1,-1,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.6669784E0,7.915082E-1,3.0326688E0,8.924973E-1,7.823503E-1,6.8516093E-1,1.3148265E0,2.3817372E-1,2.9615316E-1,2.247169E-1,2.2863865E-1,1.2332327E0,3.3867252E-1,0E0,6.669848E-1,5.469016E-2,2.0536232E-1,0E0,0E0,7.4339576E-2,8.380139E-2,0E0,0E0,8.6554486E-1,0E0,1.0833026E-1,1.4089388E-1,4.706173E-1,1.5523985E-1,0E0,0E0,0E0,2.693714E-1,0E0,0E0,4.2360187E-2,0E0,2.972263E-1,3.703024E-1,6.2748246E-2,0E0,1.1414909E-1,0E0,2.1534836E-1,2.5542617E-1,0E0,0E0,0E0,1.3995886E-1,0E0,0E0,0E0,2.5542963E-1,4.2707583E-1,3.240535E-1,5.2239712E-2,6.746605E-2,1.3762206E-1,2.1432936E-2,1.10196374E-1,2.8386605E-1,2.2751331E-1,1.4822185E-1,1.4884567E-1,1.3489743E-1,2.0207345E-1,0E0,2.529956E-1,4.8191953E-1,1.527304E-1,1.9236663E-1,2.990476E-2,8.2920134E-2,8.39185E-3,0E0,1.8457048E-2,0E0,0E0,0E0,1.04462E-1,1.5548313E-1,1.0552871E-1,0E0,0E0,0E0,8.568847E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,14,14,15,15,16,16,19,19,20,20,23,23,25,25,26,26,27,27,28,28,32,32,35,35,37,37,38,38,39,39,41,41,43,43,44,44,48,48,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,65,65,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,79,79,80,80,81,81,85,85],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,-1,28,30,32,-1,-1,34,36,-1,-1,38,-1,40,42,44,46,-1,-1,-1,48,-1,-1,50,-1,52,54,56,-1,58,-1,60,62,-1,-1,-1,64,-1,-1,-1,66,68,70,72,74,76,78,80,82,84,86,88,90,92,-1,94,96,98,100,102,104,106,-1,108,-1,-1,-1,110,112,114,-1,-1,-1,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7909248E5,1E0,3.812E3,5.134561E2,7.45E0,3.2911258E6,3.3817584E7,1.0204082E0,1E0,2.1609572E7,1.2E1,1.4676277E7,5.4007E7,2.5405103E-2,2.871605E2,1.1E1,6.25E0,1.116748E-2,-3.9230735E-4,1E0,1.665909E3,-2.8463949E-2,-1.0984051E-2,2.4568965E2,-2.1588968E-2,1E0,2E1,1.6978182E3,3.85374E8,-1.4561465E-3,4.893805E-3,-1.399883E-2,6E0,-4.6355044E-3,2.6919825E-3,9.3326636E-2,-2.7699505E-3,9E0,7.5491136E-1,2.909091E0,-4.745797E-3,5.144203E2,-2.7946636E-4,1.1251919E6,1.4595818E7,1.5478322E-3,-8.108582E-3,3.5274543E-3,2.026453E6,-1.0811287E-2,-4.6127667E-3,3.907871E-3,1.0254199E10,1.11545664E5,1.1E1,6.34E2,2.8802464E8,3.113253E0,6.149315E6,5.8222644E2,2.2E1,6.7947706E8,5.6E1,4.0832064E7,4.598404E-1,1.1644444E1,-1.398782E-2,1.7831801E6,5E0,1.1074469E6,1.68574E5,3.5364E4,2.1605E4,8.8E1,7.6526958E-3,4.4631204E2,1.1083735E-2,7.3182727E-3,1.4336471E-2,1.3E1,3.6828618E8,8.439E3,-3.8195492E-3,7.4810823E-3,1.8500285E-2,3.7307932E0,7.338987E-5,-4.298882E-3,-7.676416E-3,-5.0387025E-4,-5.9790076E-3,-2.6336138E-3,-7.4056187E-3,-5.51255E-4,3.270329E-3,-1.6419046E-2,-1.553192E-3,-3.0423386E-4,6.59409E-3,5.6344764E-3,-1.3771039E-3,4.0869E-3,-1.2223724E-4,-5.1579913E-3,1.8003442E-3,2.6814248E-3,8.911104E-5,3.9652823E-3,4.978042E-4,2.775213E-3,-5.6294426E-3,6.4846785E-3,-1.1210662E-3,5.590145E-3,9.629356E-3,1.0160744E-2,4.361972E-3],"split_indices":[37,6,2,61,63,37,7,62,8,54,8,56,7,0,67,3,61,0,0,8,4,0,0,64,0,112,3,61,41,0,0,0,3,0,0,47,0,3,47,65,0,4,0,37,56,0,0,0,9,0,0,0,40,42,8,2,5,62,54,61,3,7,3,7,47,67,0,37,3,54,9,9,9,0,0,4,0,0,0,3,12,2,0,0,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,2.58E2,7.07E2,2.03E2,5.5E1,5.13E2,1.94E2,1.85E2,1.8E1,4.5E1,1E1,4.26E2,8.7E1,7E0,1.87E2,1E1,1.75E2,9E0,9E0,1E1,3.5E1,5E0,5E0,4.21E2,5E0,4.8E1,3.9E1,1.74E2,1.3E1,5E0,5E0,5E0,1.7E2,5E0,5E0,3E1,5E0,1.13E2,3.08E2,4.3E1,5E0,3.4E1,5E0,1.14E2,6E1,6E0,7E0,7E0,1.63E2,2.5E1,5E0,9E0,1.04E2,2.09E2,9.9E1,2.7E1,1.6E1,2.2E1,1.2E1,3.4E1,8E1,2.1E1,3.9E1,1.24E2,3.9E1,9.9E1,5E0,1.88E2,2.1E1,5.4E1,4.5E1,1.4E1,1.3E1,1E1,6E0,1.6E1,6E0,6E0,6E0,1.1E1,2.3E1,7.5E1,5E0,9E0,1.2E1,3.4E1,5E0,5.8E1,6.6E1,2.3E1,1.6E1,7.3E1,2.6E1,1.38E2,5E1,6E0,1.5E1,8E0,4.6E1,1E1,3.5E1,8E0,6E0,6E0,7E0,5E0,5E0,1.1E1,5E0,5E0,6E0,1.4E1,9E0,5.1E1,2.4E1,2.4E1,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[2.9023741E-3,-3.9162695E-2,1.10496156E-1,-1.100524E-1,-6.3606887E-3,7.1897276E-2,1.895295E-1,-9.525955E-2,-2.1449016E-1,-3.4701962E-2,3.9178137E-2,1.3119711E-1,2.03352E-2,2.6173452E-1,1.13016985E-1,-1.4545251E-1,-4.4074293E-2,-2.1705896E-2,-1.3190053E-1,-3.915897E-3,-7.9282485E-2,5.158155E-2,-1.2747406E-2,2.1330515E-1,7.9147026E-2,-3.0096233E-2,1.08025335E-1,2.0721106E-1,1.9784389E-2,1.400539E-1,-3.7842921E-3,-1.562813E-1,1.9963968E-3,1.0380843E-2,-7.434956E-2,-1.7290883E-1,-2.6062338E-5,-3.241148E-2,5.4143813E-2,-5.2408732E-2,-1.469146E-1,1.0794549E-1,1.6137425E-2,3.908525E-3,2.6014778E-1,1.1302453E-1,-2.3299202E-2,-9.930128E-3,-3.284814E-3,1.7053218E-1,-4.95386E-2,6.640602E-2,2.689798E-1,-1.1134481E-3,1.7060852E-1,-1.6952994E-1,-7.785293E-2,-2.1402167E-2,-1.3056934E-1,-1.0191766E-2,-2.7094071E-3,-1.1198045E-2,-1.225417E-1,8.0932215E-2,-8.678836E-3,7.331667E-2,-6.991624E-2,-8.125106E-2,-9.818296E-3,1.4629455E-1,1.6976383E-2,3.2465402E-2,-1.0567974E-1,1.400092E-2,6.7014303E-3,1.8508023E-1,7.553352E-2,-7.774083E-3,3.5010593E-3,4.798836E-2,-6.878308E-2,9.291974E-2,2.1323247E-1,4.0222023E-4,-4.61625E-3,-9.629899E-4,6.107477E-3,2.9849443E-1,6.0637905E-3,2.7640435E-4,1.9979781E-1,-1.4182405E-1,-2.2480445E-1,-4.6368712E-4,-7.144537E-3,1.5411721E-2,-7.21387E-3,-1.3687712E-3,-1.5348276E-1,5.9890285E-2,-3.0953469E-2,-1.6055824E-1,2.5962488E-4,1.1850688E-1,1.982166E-2,4.8559537E-4,5.7511292E-3,-1.5997003E-1,-4.125718E-2,-7.295729E-3,-1.8601703E-2,4.4764284E-2,2.0776974E-1,-3.3079334E-2,4.0549748E-3,1.2361279E-1,8.369126E-3,2.7639917E-4,-1.0992732E-2,3.4145769E-3,1.1393937E-2,-5.0704996E-3,1.3090429E-1,1.414679E-1,-1.5172757E-2,-3.024596E-2,-8.61991E-3,1.2776911E-3,6.6265794E-3,1.2225351E-2,6.7221057E-3,1.5568412E-2,7.5319232E-3,2.300456E-1,2.3454109E-3,-9.035548E-3,-4.7444752E-3,-1.2834238E-2,-6.423608E-3,-2.8063874E-3,2.1232876E-3,-1.6158086E-3,-8.015051E-3,3.3833814E-4,4.966784E-3,-9.782142E-3,-7.003406E-4,-3.3917252E-3,-1.0680132E-2,2.453737E-3,7.7305944E-3,2.82319E-3,-4.309511E-3,-1.2572999E-2,-2.0448265E-3,7.427448E-4,-3.5142521E-3,-2.9116564E-3,1.3292921E-3,4.1161253E-4,5.6168516E-3,1.072927E-2,3.6781447E-3,7.920701E-5,-3.4311425E-3,7.314248E-3,1.7588178E-3,-2.9758727E-3,2.0629524E-3,7.612696E-3,7.0158986E-4,2.539101E-3,8.762217E-3,2.891651E-3,-6.4589498E-3,-3.3807044E-3,5.217415E-4,1.2183647E-2,6.9718664E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":43,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,-1,43,45,47,49,51,-1,53,-1,55,-1,-1,57,59,-1,61,63,65,67,69,71,-1,73,75,77,-1,79,81,83,85,87,-1,89,91,93,95,97,-1,-1,99,101,103,-1,105,107,109,-1,111,113,115,117,-1,-1,119,121,-1,-1,123,125,127,129,-1,-1,-1,-1,131,-1,-1,133,135,137,-1,-1,139,-1,-1,141,143,145,147,-1,149,151,-1,-1,153,155,-1,157,159,161,163,-1,165,167,-1,-1,-1,-1,-1,169,171,173,175,-1,-1,-1,-1,-1,-1,-1,177,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.299444E0,1.5874966E0,8.0088305E-1,3.132732E-1,6.053736E-1,5.507617E-1,4.5105886E-1,4.8429477E-1,4.8224783E-1,3.9633012E-1,7.150116E-1,3.4211087E-1,4.366404E-1,3.0666804E-1,2.4289495E-1,2.0266032E-1,7.949813E-1,0E0,1.1300847E-1,2.8631943E-1,2.0676875E-1,3.4587005E-1,0E0,1.7155528E-1,1.8657637E-1,3.099563E-1,3.6300558E-1,2.8654528E-1,0E0,1.9958222E-1,0E0,8.1356764E-2,0E0,0E0,2.5617763E-1,7.282075E-2,0E0,2.2041647E-1,3.8342056E-1,1.9442393E-1,1.22982025E-1,2.3125124E-1,2.1824592E-1,0E0,6.713843E-2,9.4077945E-2,2.1207301E-1,0E0,1.9136582E-1,6.635249E-2,3.2844942E-2,7.0949495E-2,5.895078E-2,0E0,1.5740967E-1,8.60374E-2,7.59495E-2,2.2943744E-1,9.273797E-2,0E0,0E0,1.3526991E-1,1.0806766E-1,1.1705387E-1,0E0,3.3707634E-2,1.8954521E-1,8.27642E-2,0E0,2.8405416E-1,7.691054E-2,2.0981151E-1,1.918706E-1,0E0,0E0,6.958896E-2,3.0083573E-1,0E0,0E0,1.9108315E-1,1.07095584E-1,3.1953357E-2,1.678598E-2,0E0,0E0,0E0,0E0,4.1352034E-2,0E0,0E0,1.1477673E-1,9.314835E-2,6.9070935E-2,0E0,0E0,8.79923E-2,0E0,0E0,7.167745E-2,5.130402E-2,2.1077105E-1,8.66864E-2,0E0,9.220362E-2,1.02742314E-1,0E0,0E0,2.1963301E-1,1.17714085E-1,0E0,2.4745202E-2,5.0953753E-2,6.912017E-2,1.8607989E-2,0E0,5.234617E-2,2.0434733E-1,0E0,0E0,0E0,0E0,0E0,8.1997246E-2,4.309067E-2,2.0090675E-1,3.6181666E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,2.02533E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,31,31,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56,57,57,58,58,61,61,62,62,63,63,65,65,66,66,67,67,69,69,70,70,71,71,72,72,75,75,76,76,79,79,80,80,81,81,82,82,87,87,90,90,91,91,92,92,95,95,98,98,99,99,100,100,101,101,103,103,104,104,107,107,108,108,110,110,111,111,112,112,113,113,115,115,116,116,122,122,123,123,124,124,125,125,133,133],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,-1,44,46,48,50,52,-1,54,-1,56,-1,-1,58,60,-1,62,64,66,68,70,72,-1,74,76,78,-1,80,82,84,86,88,-1,90,92,94,96,98,-1,-1,100,102,104,-1,106,108,110,-1,112,114,116,118,-1,-1,120,122,-1,-1,124,126,128,130,-1,-1,-1,-1,132,-1,-1,134,136,138,-1,-1,140,-1,-1,142,144,146,148,-1,150,152,-1,-1,154,156,-1,158,160,162,164,-1,166,168,-1,-1,-1,-1,-1,170,172,174,176,-1,-1,-1,-1,-1,-1,-1,178,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.62E3,1.972052E5,8.83E3,2.5E1,1.071E3,3.72381E5,8.788792E1,1.545E2,9.264256E-4,1E0,2.511E3,1.1057851E1,5.603014E6,4.907764E3,6.915691E7,6.74502E0,3.6E1,-2.1705896E-2,4.6E1,2.956111E2,4.888889E0,2.8636363E0,-1.2747406E-2,5.893737E2,5.0741018E8,1.2376238E0,4.5478998E8,1E0,1.9784389E-2,1.2118524E7,-3.7842921E-3,3.4338624E0,1.9963968E-3,1.0380843E-2,1.0130841E1,1.56E2,-2.6062338E-5,1.8E1,8.931E3,3.2931E4,3.2E2,1.8729467E8,1E0,3.908525E-3,7.6825094E0,2.2007043E0,9.45635E6,-9.930128E-3,1.1208894E0,1.208686E6,1.5822886E0,4.5154482E-1,3.6666667E0,-1.1134481E-3,4.3948618E8,6.737099E7,1.0392858E1,9.822E3,5.592863E6,-1.0191766E-2,-2.7094071E-3,1.2354949E0,2.2646774E2,5.831829E6,-8.678836E-3,2.49E2,1.4977306E2,1.0647887E1,-9.818296E-3,5.4007E7,2.33E2,9.912121E0,6.763312E7,1.400092E-2,6.7014303E-3,1.625E0,3.8610423E0,-7.774083E-3,3.5010593E-3,1.3922E4,1.9E1,5.6115704E0,2.5603516E5,4.0222023E-4,-4.61625E-3,-9.629899E-4,6.107477E-3,3.4141E11,6.0637905E-3,2.7640435E-4,1.514526E6,3.4E1,1.76421E6,-4.6368712E-4,-7.144537E-3,1.0140845E0,-7.21387E-3,-1.3687712E-3,1E0,4.3992E4,1.4598765E0,6.5062125E5,2.5962488E-4,2.28E0,8.23E2,4.8559537E-4,5.7511292E-3,5.0200376E7,1.9821428E0,-7.295729E-3,3E0,1.8291844E7,1.124641E0,2.0211798E6,4.0549748E-3,2.0778275E-1,4.2492856E2,2.7639917E-4,-1.0992732E-2,3.4145769E-3,1.1393937E-2,-5.0704996E-3,7.973882E7,7.755551E7,1.5E1,3.754E3,-8.61991E-3,1.2776911E-3,6.6265794E-3,1.2225351E-2,6.7221057E-3,1.5568412E-2,7.5319232E-3,2.342612E0,2.3454109E-3,-9.035548E-3,-4.7444752E-3,-1.2834238E-2,-6.423608E-3,-2.8063874E-3,2.1232876E-3,-1.6158086E-3,-8.015051E-3,3.3833814E-4,4.966784E-3,-9.782142E-3,-7.003406E-4,-3.3917252E-3,-1.0680132E-2,2.453737E-3,7.7305944E-3,2.82319E-3,-4.309511E-3,-1.2572999E-2,-2.0448265E-3,7.427448E-4,-3.5142521E-3,-2.9116564E-3,1.3292921E-3,4.1161253E-4,5.6168516E-3,1.072927E-2,3.6781447E-3,7.920701E-5,-3.4311425E-3,7.314248E-3,1.7588178E-3,-2.9758727E-3,2.0629524E-3,7.612696E-3,7.0158986E-4,2.539101E-3,8.762217E-3,2.891651E-3,-6.4589498E-3,-3.3807044E-3,5.217415E-4,1.2183647E-2,6.9718664E-3],"split_indices":[2,37,2,3,2,9,67,64,47,23,2,65,9,4,53,63,10,0,3,4,65,63,0,4,7,63,7,109,0,56,0,62,0,0,65,38,0,3,38,9,10,7,85,0,65,63,56,0,47,37,47,47,63,0,7,7,67,38,5,0,0,62,61,54,0,10,61,67,0,7,0,67,7,0,0,62,63,0,0,38,3,63,42,0,0,0,0,40,0,0,10,2,40,0,0,65,0,0,13,1,63,37,0,65,2,0,0,7,62,0,8,54,47,37,0,47,4,0,0,0,0,0,41,5,3,2,0,0,0,0,0,0,0,51,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.48E2,6.82E2,2.66E2,2.15E2,4.67E2,1.8E2,8.6E1,1.9E2,2.5E1,2.88E2,1.79E2,8.3E1,9.7E1,4.3E1,4.3E1,9.5E1,9.5E1,5E0,2E1,1.71E2,1.17E2,1.73E2,6E0,3.1E1,5.2E1,6.2E1,3.5E1,3.4E1,9E0,3.8E1,5E0,9E1,5E0,9E0,8.6E1,1.5E1,5E0,1.15E2,5.6E1,8.5E1,3.2E1,6.6E1,1.07E2,9E0,2.2E1,3.9E1,1.3E1,7E0,5.5E1,2.5E1,1E1,1.1E1,2.3E1,6E0,3.2E1,7.6E1,1.4E1,4.5E1,4.1E1,1E1,5E0,9.4E1,2.1E1,5.1E1,5E0,1E1,7.5E1,1.7E1,1.5E1,4.6E1,2E1,9.5E1,1.2E1,1.5E1,7E0,1.2E1,2.7E1,5E0,8E0,3.1E1,2.4E1,1E1,1.5E1,5E0,5E0,5E0,6E0,1.8E1,5E0,5E0,2.7E1,5.3E1,2.3E1,8E0,6E0,3.6E1,9E0,8E0,3.3E1,2E1,7.4E1,1.6E1,5E0,3.1E1,2E1,5E0,5E0,1.7E1,5.8E1,7E0,1E1,1.8E1,2.8E1,1.2E1,8E0,1.9E1,7.6E1,7E0,5E0,5E0,7E0,6E0,2.1E1,1.2E1,1.9E1,1.9E1,5E0,5E0,5E0,7E0,8E0,1.3E1,5E0,2.2E1,5E0,2.1E1,3.2E1,1.3E1,1E1,1E1,2.6E1,5E0,2.8E1,1E1,1E1,5E0,6.9E1,8E0,8E0,1.4E1,1.7E1,1.5E1,5E0,8E0,9E0,2.2E1,3.6E1,5E0,5E0,1.3E1,5E0,2.3E1,5E0,7E0,5E0,1.3E1,6E0,2.5E1,5.1E1,1.6E1,5E0,5E0,7E0,1.2E1,7E0,9E0,1E1,1.4E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"179","size_leaf_vector":"1"}},{"base_weights":[-2.085973E-3,-3.5134893E-2,1.1134524E-1,-8.398461E-2,1.3015738E-2,9.391186E-2,1.9624535E-2,-7.778933E-2,-1.4802458E-2,1.8110795E-2,-1.383879E-2,1.0571251E-1,-7.990142E-3,-1.2241888E-1,-4.3769076E-2,8.210883E-4,1.0931474E-1,7.215254E-2,1.7654032E-1,-1.3046305E-1,9.643456E-4,3.03233E-1,-6.7157865E-2,8.093293E-3,-1.9560307E-1,5.2722287E-3,1.392543E-1,1.6968961E-1,5.041342E-2,1.9107644E-1,1.625428E-3,-1.0750089E-1,-1.8608654E-1,7.927702E-3,1.6917698E-2,-4.9544886E-2,-1.8765046E-1,-5.177946E-2,3.1169318E-2,-1.734939E-2,7.103025E-4,-4.407666E-3,4.2848075E-3,1.7172384E-1,2.3282664E-2,9.291997E-3,4.9583963E-3,-3.510377E-2,8.399013E-2,8.38296E-2,2.2155038E-1,-1.5687515E-1,-1.9922018E-2,-2.0388947E-1,-2.2389805E-3,-2.0715645E-2,-1.0532261E-1,-1.0284506E-2,-1.2731703E-1,1.7218294E-2,-1.263825E-1,1.1492688E-2,2.3768278E-2,9.240791E-3,2.5167232E-3,2.514591E-3,-5.34038E-4,6.710713E-2,-9.18384E-2,1.0103571E-1,-5.020627E-3,4.990684E-4,8.327232E-3,2.4758717E-1,3.5584152E-3,-1.0585857E-3,-7.8733945E-3,3.7264696E-3,-2.058193E-3,-7.1600615E-3,-1.2563655E-2,-1.4271962E-3,4.530525E-3,-7.1430327E-3,-6.791158E-4,-8.141667E-3,-2.6886293E-3,2.2395144E-3,-7.434733E-4,-2.5424748E-3,-1.2567201E-2,7.0106395E-4,9.0279505E-3,-2.0379913E-3,7.084674E-3,-5.955569E-3,9.671539E-4,3.9034472E-3,9.920469E-3,5.182356E-3,1.2337747E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":44,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,17,-1,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,47,49,-1,51,53,-1,-1,55,57,59,61,-1,-1,-1,-1,63,65,-1,-1,67,69,71,73,75,77,79,-1,81,83,-1,85,87,89,-1,91,-1,-1,-1,-1,93,95,97,-1,-1,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.539011E0,1.7205111E0,1.1173577E0,5.0375795E-1,5.9061164E-1,6.55692E-1,0E0,5.3364134E-1,0E0,5.7389796E-1,0E0,4.5077324E-1,0E0,1.7895913E-1,1.6679039E0,4.4011307E-1,1.799351E-1,2.7557492E-1,1.20839834E-1,1.6519451E-1,0E0,4.332614E-2,3.960495E-1,4.1195658E-1,4.194103E-1,1.3100046E-1,1.6545129E-1,2.4696112E-2,3.2167217E-1,1.6444874E-1,0E0,4.509096E-1,9.076774E-2,0E0,0E0,2.6796708E-1,2.1013856E-2,4.2941386E-1,3.3864623E-1,0E0,0E0,0E0,0E0,1.0200286E-1,1.2454161E-2,0E0,0E0,1.9172773E-1,2.7301806E-1,9.5911264E-2,1.4319444E-1,1.11209154E-1,9.908742E-2,7.758331E-2,0E0,1.3613763E-1,2.4909031E-1,0E0,2.6576757E-2,4.6184767E-2,3.988343E-1,0E0,3.0616418E-1,0E0,0E0,0E0,0E0,1.2073217E-1,8.908452E-2,1.2311542E-1,0E0,0E0,0E0,5.990839E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,31,31,32,32,35,35,36,36,37,37,38,38,43,43,44,44,47,47,48,48,49,49,50,50,51,51,52,52,53,53,55,55,56,56,58,58,59,59,60,60,62,62,67,67,68,68,69,69,73,73],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,18,-1,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,48,50,-1,52,54,-1,-1,56,58,60,62,-1,-1,-1,-1,64,66,-1,-1,68,70,72,74,76,78,80,-1,82,84,-1,86,88,90,-1,92,-1,-1,-1,-1,94,96,98,-1,-1,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.443E3,4.539777E5,1.0534078E4,3.8652172E1,1.0564016E3,9.7172376E7,1.9624535E-2,4.76E2,-1.4802458E-2,6.009825E2,-1.383879E-2,3.130786E6,-7.990142E-3,7.997723E6,2.04115E5,4.5620965E6,5.1604336E7,2.059E3,2E1,1.0829268E1,9.643456E-4,1.5E1,2.1609572E7,1E0,2.1027016E5,4.353644E6,2.0319266E-7,2.067251E0,1.0378262E10,1.5203704E1,1.625428E-3,4.157143E1,1E0,7.927702E-3,1.6917698E-2,8.303E3,1.731E3,2.308943E0,1.1160929E0,-1.734939E-2,7.103025E-4,-4.407666E-3,4.2848075E-3,3.5919855E0,7.0093E4,9.291997E-3,4.9583963E-3,1.03984375E1,3.0070068E7,7.82261E6,2.395631E7,1.863E3,8E0,2.53692E5,-2.2389805E-3,5.2405543E0,1.2194473E8,-1.0284506E-2,9.234473E-9,1.3E1,8.3E1,1.1492688E-2,7.82261E6,9.240791E-3,2.5167232E-3,2.514591E-3,-5.34038E-4,3.019898E8,7.1E1,4.435876E2,-5.020627E-3,4.990684E-4,8.327232E-3,1E0,3.5584152E-3,-1.0585857E-3,-7.8733945E-3,3.7264696E-3,-2.058193E-3,-7.1600615E-3,-1.2563655E-2,-1.4271962E-3,4.530525E-3,-7.1430327E-3,-6.791158E-4,-8.141667E-3,-2.6886293E-3,2.2395144E-3,-7.434733E-4,-2.5424748E-3,-1.2567201E-2,7.0106395E-4,9.0279505E-3,-2.0379913E-3,7.084674E-3,-5.955569E-3,9.671539E-4,3.9034472E-3,9.920469E-3,5.182356E-3,1.2337747E-2],"split_indices":[2,37,4,62,4,54,0,2,0,64,0,37,0,9,5,56,7,38,3,67,0,0,54,109,42,54,46,47,40,67,0,61,74,0,0,38,2,63,63,0,0,0,0,62,1,0,0,65,52,37,56,9,10,12,0,62,7,0,46,3,0,0,37,0,0,0,0,7,8,65,0,0,0,109,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.42E2,7.3E2,2.12E2,3.62E2,3.68E2,2.02E2,1E1,3.54E2,8E0,3.63E2,5E0,1.94E2,8E0,1.52E2,2.02E2,3.06E2,5.7E1,1.33E2,6.1E1,1.44E2,8E0,1.2E1,1.9E2,2.96E2,1E1,1.3E1,4.4E1,2.3E1,1.1E2,5.5E1,6E0,1.04E2,4E1,5E0,7E0,1.67E2,2.3E1,8.2E1,2.14E2,5E0,5E0,6E0,7E0,3.4E1,1E1,1.4E1,9E0,3.1E1,7.9E1,1.3E1,4.2E1,6.6E1,3.8E1,3.5E1,5E0,1.11E2,5.6E1,1.3E1,1E1,4.3E1,3.9E1,6E0,2.08E2,2.7E1,7E0,5E0,5E0,1.1E1,2E1,7.3E1,6E0,8E0,5E0,3.5E1,7E0,6E0,6E1,7E0,3.1E1,2.2E1,1.3E1,1.03E2,8E0,3.6E1,2E1,5E0,5E0,2.2E1,2.1E1,2.7E1,1.2E1,1.99E2,9E0,5E0,6E0,1.5E1,5E0,6.5E1,8E0,5E0,3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[2.300875E-3,-7.8811884E-2,5.3823985E-2,-9.275543E-2,3.9762694E-2,2.8994765E-2,1.5806885E-1,-8.074156E-2,-3.0637562E-1,-2.822882E-3,8.036014E-2,4.273347E-2,-1.7855938E-1,1.8104054E-1,-5.5605182E-3,-1.03328176E-1,-1.772362E-2,-1.6027461E-1,-2.5404371E-2,3.6814786E-2,-2.1409628E-3,6.127594E-3,2.2628636E-3,1.1975473E-2,3.5504628E-2,-2.272335E-1,3.1915333E-4,1.409198E-1,3.3848956E-1,-8.0939725E-2,-1.6466685E-1,-9.750918E-3,-2.6942273E-3,-3.6573778E-3,-9.613641E-3,-4.150527E-4,3.5467236E-3,-2.5685748E-2,6.304444E-2,-2.711555E-1,-4.0879664E-3,7.1551E-2,2.1023539E-1,2.3050036E-2,1.1720938E-2,-1.02104165E-1,3.9391782E-2,-1.0830541E-1,-3.0085722E-1,5.28757E-2,-2.4095003E-2,-5.6703556E-3,-1.3121139E-1,2.5053734E-1,5.4813266E-2,-8.301287E-3,-1.4747442E-2,7.912149E-3,3.2200675E-2,2.87762E-3,2.4428742E-1,-1.6175628E-1,-7.3355675E-2,6.762884E-3,7.1268706E-3,-1.6244352E-1,-6.0662705E-2,-1.9040607E-2,-7.327519E-3,8.890283E-2,-2.6918047E-3,4.979956E-2,-5.1671624E-2,-1.5125261E-2,6.417686E-3,-1.8647163E-1,1.7640047E-3,5.331141E-3,1.7029904E-2,6.2520154E-2,-7.673958E-2,-5.566961E-3,6.48258E-2,1.2707734E-2,4.4198954E-3,-5.955439E-3,-9.394518E-3,-5.024436E-4,-5.053522E-3,-1.673067E-3,1.972917E-3,-7.6879456E-4,-1.0383149E-2,-4.3262276E-4,-7.0014717E-3,9.1840076E-4,5.7906224E-3,-6.278784E-4,5.3277034E-3,-4.4506197E-3,-9.6291665E-4,-1.1306162E-3,6.6335164E-3,-1.40226735E-2,-4.0276037E-3,1.7160815E-3,4.6454747E-3,-1.603259E-2,5.525739E-3,4.851499E-3,-6.114271E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":45,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,33,-1,35,-1,-1,-1,-1,37,39,-1,41,43,45,47,-1,49,-1,-1,-1,-1,51,53,55,-1,57,59,-1,-1,61,63,65,67,69,71,73,75,77,79,-1,-1,-1,81,-1,83,85,87,-1,89,91,93,-1,-1,95,-1,97,99,101,-1,103,-1,-1,-1,105,107,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.915956E0,6.055076E-1,1.4785233E0,8.119106E-1,6.76526E-2,1.3307381E0,7.239411E-1,4.3941092E-1,5.111693E-1,3.6040112E-2,2.6225552E-2,6.696091E-1,2.6331758E-1,5.966313E-1,0E0,2.9977894E-1,2.3947634E-1,2.952692E-2,0E0,2.0629004E-2,0E0,0E0,0E0,0E0,7.144467E-1,1.1665988E-1,0E0,3.8404953E-1,1.4637947E-1,4.344226E-1,4.2796886E-1,0E0,9.4034724E-2,0E0,0E0,0E0,0E0,2.7805805E-1,4.369068E-1,1.6340971E-2,0E0,1.619754E-1,1.9178009E-1,0E0,0E0,2.3508108E-1,8.818693E-2,1.036343E-1,2.0128143E-1,9.304823E-2,1.1872569E-1,1.5456416E-1,2.0347384E-1,1.4744079E-1,2.8969324E-1,0E0,0E0,0E0,1.6665408E-1,0E0,1.1943281E-1,3.657484E-2,2.1892977E-1,0E0,3.3630837E-2,1.7523378E-1,1.1440449E-1,0E0,0E0,3.9703384E-2,0E0,6.744206E-2,5.5469632E-2,1.5835391E-1,0E0,1.5834534E-1,0E0,0E0,0E0,2.516836E-1,8.833606E-1,0E0,8.506058E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,17,17,19,19,24,24,25,25,27,27,28,28,29,29,30,30,32,32,37,37,38,38,39,39,41,41,42,42,45,45,46,46,47,47,48,48,49,49,50,50,51,51,52,52,53,53,54,54,58,58,60,60,61,61,62,62,64,64,65,65,66,66,69,69,71,71,72,72,73,73,75,75,79,79,80,80,82,82],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,34,-1,36,-1,-1,-1,-1,38,40,-1,42,44,46,48,-1,50,-1,-1,-1,-1,52,54,56,-1,58,60,-1,-1,62,64,66,68,70,72,74,76,78,80,-1,-1,-1,82,-1,84,86,88,-1,90,92,94,-1,-1,96,-1,98,100,102,-1,104,-1,-1,-1,106,108,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,2.9652428E6,7.098E3,1.0404834E7,1.1451E4,3.2856784E7,4.1325716E7,2.2646774E2,2E0,5.766206E6,5.894636E6,2.020055E6,5.615327E5,4.907764E3,-5.5605182E-3,2.6742857E0,2.0588236E0,8.025477E0,-2.5404371E-2,4.5866325E6,-2.1409628E-3,6.127594E-3,2.2628636E-3,1.1975473E-2,4.8709216E5,1.8495639E5,3.1915333E-4,2.36433E6,1.11545664E5,5.02E2,5.17259E8,-9.750918E-3,1.875E0,-3.6573778E-3,-9.613641E-3,-4.150527E-4,3.5467236E-3,2.2E1,1.0918E4,1.0273995E-8,-4.0879664E-3,3.6828618E8,1E0,2.3050036E-2,1.1720938E-2,3E1,7.52E2,5.118881E0,6.764934E7,3.4289816E2,3.617263E6,9.6215985E2,2.611575E-6,7.156528E7,1.9269184E7,-8.301287E-3,-1.4747442E-2,7.912149E-3,2.453E4,2.87762E-3,2.395631E7,7.1E1,7.52E2,6.762884E-3,5.2117184E4,7.9227E4,6.03E2,-1.9040607E-2,-7.327519E-3,2.64E0,-2.6918047E-3,3.083671E2,3.1316666E2,1.70244E7,6.417686E-3,1.5382514E1,1.7640047E-3,5.331141E-3,1.7029904E-2,6.880842E-1,5.59E2,-5.566961E-3,2.8501965E8,1.2707734E-2,4.4198954E-3,-5.955439E-3,-9.394518E-3,-5.024436E-4,-5.053522E-3,-1.673067E-3,1.972917E-3,-7.6879456E-4,-1.0383149E-2,-4.3262276E-4,-7.0014717E-3,9.1840076E-4,5.7906224E-3,-6.278784E-4,5.3277034E-3,-4.4506197E-3,-9.6291665E-4,-1.1306162E-3,6.6335164E-3,-1.40226735E-2,-4.0276037E-3,1.7160815E-3,4.6454747E-3,-1.603259E-2,5.525739E-3,4.851499E-3,-6.114271E-4],"split_indices":[2,37,2,54,12,54,60,61,8,54,54,5,37,4,0,65,63,65,0,37,0,0,0,0,37,42,0,37,42,2,40,0,62,0,0,0,0,3,9,46,0,12,109,0,0,64,12,65,7,4,5,61,46,7,56,0,0,0,38,0,56,38,12,0,56,9,10,0,0,63,0,4,4,9,0,65,0,0,0,51,0,0,41,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.35E2,3.63E2,5.72E2,3.25E2,3.8E1,4.63E2,1.09E2,3.09E2,1.6E1,1.9E1,1.9E1,4.35E2,2.8E1,1.01E2,8E0,2.27E2,8.2E1,1.1E1,5E0,1E1,9E0,6E0,1.3E1,1.3E1,4.22E2,2.2E1,6E0,8.2E1,1.9E1,1.68E2,5.9E1,5E0,7.7E1,5E0,6E0,5E0,5E0,1.31E2,2.91E2,1.6E1,6E0,4.2E1,4E1,5E0,1.4E1,1.43E2,2.5E1,4.3E1,1.6E1,2.1E1,5.6E1,1.11E2,2E1,1.1E1,2.8E2,7E0,9E0,1.1E1,3.1E1,8E0,3.2E1,4.5E1,9.8E1,5E0,2E1,1.9E1,2.4E1,8E0,8E0,1.6E1,5E0,1.5E1,4.1E1,1.05E2,6E0,1.5E1,5E0,6E0,5E0,2.65E2,1.5E1,5E0,2.6E1,2.6E1,6E0,2.7E1,1.8E1,3.6E1,6.2E1,9E0,1.1E1,6E0,1.3E1,1.6E1,8E0,6E0,1E1,8E0,7E0,1.6E1,2.5E1,1E2,5E0,6E0,9E0,1.59E2,1.06E2,6E0,9E0,1.7E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[-8.3208067E-4,-3.1210704E-2,1.08679585E-1,-1.9232303E-2,-1.8679558E-1,-5.8192726E-2,1.236797E-1,-1.13893755E-1,2.3576751E-5,-3.5305414E-1,-7.861247E-2,1.3008129E-3,-6.030926E-3,1.3564777E-1,-5.052935E-3,-1.0378445E-2,-1.01769894E-1,1.3142056E-2,-5.3850655E-3,-2.4392752E-3,-4.995405E-1,2.3598887E-2,-1.6054769E-1,1.1434149E-1,2.9558024E-1,-4.4241272E-2,-1.1664102E-1,-1.16859086E-1,7.0829084E-3,-3.1364657E-2,-1.311809E-2,-2.4434633E-3,3.462383E-3,-4.3346123E-3,-1.2169581E-2,1.4723346E-1,1.5292596E-2,2.1893227E-2,9.790646E-3,7.9890713E-4,-8.226474E-2,-1.6806678E-1,-9.402352E-2,-4.0856253E-2,-1.7733464E-1,-1.2904137E-2,5.49794E-2,2.2571696E-1,1.066906E-1,3.2263037E-2,-4.2644404E-3,-5.1760017E-3,-1.5245004E-3,-1.9136272E-1,-3.7502525E-3,-6.340299E-2,-1.3680579E-1,-7.976921E-2,2.1266195E-3,-2.285609E-1,-8.4883876E-2,-7.672093E-3,-1.3032883E-2,3.681732E-2,1.8914959E-1,1.466346E-1,2.5719684E-1,6.4213836E-4,1.3200602E-1,-1.2834338E-3,5.1814853E-3,-5.5196835E-3,-1.199852E-2,-5.596742E-3,-5.261599E-4,-2.1253994E-3,-7.5692036E-3,-5.01052E-4,-6.5313107E-3,-3.7936533E-3,-1.2622657E-2,-1.3709795E-3,-6.7055766E-3,-6.3303305E-4,5.6575285E-3,2.2346936E-3,-8.311591E-3,1.0806438E-2,2.9153372E-3,8.573168E-3,3.8096083E-3,1.2955213E-2,7.7607944E-3,3.056802E-3,-4.731187E-3,3.0656758E-4,6.7987083E-3,1.4252561E-3,-5.493917E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":46,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,-1,-1,25,-1,27,-1,29,31,33,35,37,39,41,43,45,-1,-1,-1,-1,-1,-1,47,49,-1,-1,-1,51,53,55,57,59,61,63,65,67,69,-1,-1,-1,71,-1,73,75,77,-1,79,81,83,-1,85,87,89,91,93,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.217094E0,1.4071963E0,5.328491E-1,1.2832466E0,9.435096E-1,1.1506279E-1,5.519955E-1,1.3562405E-1,8.99162E-1,8.980882E-1,2.8852892E-1,0E0,0E0,5.931952E-1,0E0,0E0,8.809078E-2,0E0,8.013843E-1,0E0,3.444388E-1,6.618549E-2,1.05249435E-1,5.320461E-1,2.1348286E-1,5.7501946E-2,8.422637E-2,2.59341E-1,4.97553E-1,0E0,0E0,0E0,0E0,0E0,0E0,3.672974E-1,7.937279E-2,0E0,0E0,0E0,1.8041372E-2,3.5342216E-2,7.254708E-2,9.473601E-2,1.3214779E-1,5.124107E-1,3.658679E-1,6.435561E-2,2.2274214E-1,9.87065E-2,0E0,0E0,0E0,6.1721742E-2,0E0,1.11832336E-1,5.2417964E-2,7.840843E-2,0E0,1.05422735E-1,3.8629584E-2,2.810288E-1,0E0,3.4181306E-1,8.4843874E-2,1.928091E-2,8.734822E-3,1.2051544E-1,1.2102294E-1,1.0483178E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,16,16,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,35,35,36,36,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,53,53,55,55,56,56,57,57,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,-1,-1,26,-1,28,-1,30,32,34,36,38,40,42,44,46,-1,-1,-1,-1,-1,-1,48,50,-1,-1,-1,52,54,56,58,60,62,64,66,68,70,-1,-1,-1,72,-1,74,76,78,-1,80,82,84,-1,86,88,90,92,94,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.812E3,9.932432E0,1.7909248E5,3.51E2,1E0,3.53562E5,2.871605E2,8.071428E0,1.2E1,4.327E3,1.6670103E1,1.3008129E-3,-6.030926E-3,4.907764E3,-5.052935E-3,-1.0378445E-2,5.7018555E4,1.3142056E-2,8.147158E4,-2.4392752E-3,4.0568292E2,7.547528E6,4E1,9E1,1.26999164E5,1.2E1,2.0550128E5,4.685271E8,4.4935583E2,-3.1364657E-2,-1.311809E-2,-2.4434633E-3,3.462383E-3,-4.3346123E-3,-1.2169581E-2,5.6516E4,8.9370746E8,2.1893227E-2,9.790646E-3,7.9890713E-4,4.631579E0,7.8625E1,5.1077608E-2,2.0376764E-7,6.0546756E-2,3.4474048E6,3.2911258E6,1.3146555E3,8.484036E5,2.234478E3,-4.2644404E-3,-5.1760017E-3,-1.5245004E-3,4.456432E6,-3.7502525E-3,3.3820656E7,1.9052632E0,1.662258E2,2.1266195E-3,5.6E1,4.0805513E-1,7.2132964E0,-1.3032883E-2,3.0359353E5,3.43E2,5.376E3,1.4780379E7,1.3E1,3.72424E5,1.3358269E2,5.1814853E-3,-5.5196835E-3,-1.199852E-2,-5.596742E-3,-5.261599E-4,-2.1253994E-3,-7.5692036E-3,-5.01052E-4,-6.5313107E-3,-3.7936533E-3,-1.2622657E-2,-1.3709795E-3,-6.7055766E-3,-6.3303305E-4,5.6575285E-3,2.2346936E-3,-8.311591E-3,1.0806438E-2,2.9153372E-3,8.573168E-3,3.8096083E-3,1.2955213E-2,7.7607944E-3,3.056802E-3,-4.731187E-3,3.0656758E-4,6.7987083E-3,1.4252561E-3,-5.493917E-3],"split_indices":[2,63,37,2,109,1,67,61,27,1,67,0,0,4,0,0,54,0,37,0,4,5,3,8,42,3,54,40,64,0,0,0,0,0,0,9,41,0,0,0,63,4,51,46,47,56,37,61,37,4,0,0,0,7,0,7,62,61,0,0,51,63,0,42,0,2,60,3,9,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,7.56E2,2.09E2,7.03E2,5.3E1,1.7E1,1.92E2,1.18E2,5.85E2,2E1,3.3E1,8E0,9E0,1.83E2,9E0,1E1,1.08E2,1E1,5.75E2,7E0,1.3E1,1.5E1,1.8E1,1.63E2,2E1,2.3E1,8.5E1,5.7E1,5.18E2,6E0,7E0,6E0,9E0,1.2E1,6E0,1.22E2,4.1E1,5E0,1.5E1,9E0,1.4E1,2.4E1,6.1E1,2.6E1,3.1E1,3.66E2,1.52E2,4E1,8.2E1,3.6E1,5E0,8E0,6E0,1.8E1,6E0,3.7E1,2.4E1,1.8E1,8E0,1.9E1,1.2E1,3.6E2,6E0,1.35E2,1.7E1,1.3E1,2.7E1,1.6E1,6.6E1,2.6E1,1E1,1E1,8E0,1.7E1,2E1,6E0,1.8E1,9E0,9E0,5E0,1.4E1,7E0,5E0,3.45E2,1.5E1,1.29E2,6E0,1.2E1,5E0,7E0,6E0,2E1,7E0,1E1,6E0,7E0,5.9E1,2.1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[-2.1772473E-3,-5.4511372E-2,6.01807E-2,-3.6380745E-2,-1.3289309E-1,8.460128E-2,-1.07106425E-1,-1.1154409E-1,-1.649454E-2,-2.0380661E-1,-6.7500085E-2,5.8245778E-2,1.6876416E-1,-1.5730347E-1,4.3932173E-2,-9.6659884E-2,-1.7434232E-1,1.2880348E-2,-2.1681651E-2,-1.0425059E-1,-3.7336132E-1,-3.8419403E-2,-1.3404578E-2,8.5225604E-2,8.099566E-3,2.348495E-1,1.0329323E-1,-2.5332782E-1,-1.1196088E-1,6.8158484E-3,-1.9038375E-3,-1.3042842E-1,-3.179623E-2,-9.788414E-3,-3.6262518E-3,-4.25895E-2,2.974248E-2,2.7640502E-4,-1.3664271E-1,-5.045976E-1,-5.2800467E-3,-4.3123422E-4,-1.4066207E-1,1.0671395E-1,-2.622245E-2,-9.075986E-2,2.689182E-2,1.7851555E-1,1.7394744E-2,1.4926735E-1,3.1466108E-2,-7.314145E-3,-1.4515194E-2,-3.4255084E-2,-1.8266912E-1,-7.418747E-2,-1.7732668E-1,-4.9977466E-2,8.1808603E-4,-5.7721112E-2,4.7687702E-2,1.3034232E-1,-1.0935582E-3,-1.6089864E-1,-1.7194895E-3,-2.985355E-2,-1.3070334E-2,-4.721157E-2,9.7063735E-2,-9.8213665E-3,-1.0458904E-3,1.1641225E-1,-5.784358E-3,-6.109108E-2,3.9912686E-3,-6.6215703E-3,1.3113985E-3,-5.5328747E-3,1.10120356E-1,1.3726724E-2,1.010741E-1,9.04593E-2,1.9812423E-1,-4.4127677E-2,6.9772056E-3,-4.4863E-3,2.7760502E-3,-4.1178083E-3,-2.1729307E-1,-9.3315216E-4,-1.05292425E-1,-4.0587285E-3,-2.1975037E-1,-7.5622957E-4,-3.8059969E-3,-4.77628E-2,-1.1512979E-2,-3.3668388E-2,9.729213E-2,1.1646615E-3,1.6600987E-1,-2.3591192E-2,8.95904E-3,-4.748102E-3,-8.655135E-3,2.0500896E-3,-1.01382785E-1,8.5514E-3,-9.692341E-4,4.0817212E-2,1.4300859E-1,-1.9960115E-2,-7.6097334E-3,9.696892E-3,-4.1883015E-3,1.7749564E-1,8.234024E-3,7.6693073E-3,5.257356E-2,1.5768822E-3,6.966703E-3,4.8509627E-3,1.0878692E-2,-4.101762E-3,1.032771E-3,-1.304455E-2,-5.43961E-3,-6.6220183E-3,-2.5383048E-3,-1.19678555E-2,-5.95578E-3,-3.2538832E-3,8.183572E-4,1.1047705E-3,-3.166592E-3,6.0082953E-3,4.5454464E-4,1.02582155E-2,1.3080536E-3,-5.883633E-3,-3.7009668E-4,-6.6192923E-3,-4.0890014E-4,5.4641357E-3,1.05022795E-4,8.574256E-3,4.899183E-3,3.984665E-3,-2.9646142E-3,-2.5237605E-3,1.5994208E-3,3.9800634E-3,9.9655E-3,-2.1228062E-3,2.823238E-3,-8.642997E-4,5.0251805E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":47,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,35,37,39,41,-1,43,45,47,49,51,53,-1,-1,55,57,-1,-1,59,61,-1,63,65,-1,67,69,71,73,75,77,79,-1,81,83,-1,-1,85,87,89,91,93,-1,95,97,99,101,103,-1,-1,-1,105,107,-1,-1,109,-1,111,-1,-1,-1,113,115,-1,117,119,121,123,-1,-1,-1,-1,125,-1,127,-1,129,-1,-1,131,-1,133,135,-1,137,139,-1,-1,-1,-1,141,-1,-1,143,145,147,-1,149,-1,151,153,-1,155,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1753528E0,7.469324E-1,1.8229119E0,6.4238155E-1,4.4533062E-1,8.5081434E-1,4.3906707E-1,6.727135E-2,5.2773523E-1,7.5444615E-1,3.301322E-1,4.0113163E-1,3.7379527E-1,1.588589E-1,1.3724905E-1,1.6012925E-1,3.9868444E-2,0E0,3.635983E-1,1.1147964E-1,5.216744E-1,1.8637192E-1,0E0,4.6500146E-1,1.975088E-1,2.8808665E-1,1.5380818E-1,2.4710357E-2,1.6407028E-1,0E0,0E0,1.1608708E-1,2.5144868E-2,0E0,0E0,3.3060214E-1,3.0414838E-1,0E0,5.2009612E-2,1.5910625E-1,0E0,1.6877304E-1,1.0426679E-1,3.780371E-1,1.3049455E-1,1.0914509E-1,2.4078998E-1,2.7901053E-1,0E0,6.5400004E-2,1.8640766E-1,0E0,0E0,9.976949E-2,3.274715E-2,3.8251325E-2,7.988685E-2,2.0251058E-2,0E0,3.7875122E-1,1.4452246E-1,8.216834E-2,3.3656594E-1,9.439826E-3,0E0,0E0,0E0,1.2825406E-1,1.2732652E-1,0E0,0E0,3.0780864E-1,0E0,1.0255016E-1,0E0,0E0,0E0,8.475585E-2,1.70154E-1,0E0,5.9828147E-2,4.5459114E-2,2.6486993E-2,4.0176474E-2,0E0,0E0,0E0,0E0,3.7309706E-2,0E0,2.059099E-2,0E0,2.6212037E-2,0E0,0E0,2.8938183E-1,0E0,2.9449783E-2,6.0710877E-2,0E0,1.22803956E-1,1.10192806E-1,0E0,0E0,0E0,0E0,5.9877917E-2,0E0,0E0,1.2432862E-1,1.6030335E-1,9.3833044E-2,0E0,9.078778E-2,0E0,3.286627E-2,3.4026705E-2,0E0,5.756894E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,35,35,36,36,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,53,53,54,54,55,55,56,56,57,57,59,59,60,60,61,61,62,62,63,63,67,67,68,68,71,71,73,73,77,77,78,78,80,80,81,81,82,82,83,83,88,88,90,90,92,92,95,95,97,97,98,98,100,100,101,101,106,106,109,109,110,110,111,111,113,113,115,115,116,116,118,118],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,36,38,40,42,-1,44,46,48,50,52,54,-1,-1,56,58,-1,-1,60,62,-1,64,66,-1,68,70,72,74,76,78,80,-1,82,84,-1,-1,86,88,90,92,94,-1,96,98,100,102,104,-1,-1,-1,106,108,-1,-1,110,-1,112,-1,-1,-1,114,116,-1,118,120,122,124,-1,-1,-1,-1,126,-1,128,-1,130,-1,-1,132,-1,134,136,-1,138,140,-1,-1,-1,-1,142,-1,-1,144,146,148,-1,150,-1,152,154,-1,156,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.5467294E2,1.8E1,2.856934E7,2.55E2,1.5470404E2,1.3631968E3,2.9925186E-2,8.138889E0,2.74614E5,1.449E3,2.2386675E6,8E0,1.762852E7,1.6748972E1,5.735198E-1,3.89E2,9.540551E1,1.2880348E-2,1E0,8E0,7.0858893E0,1.11E2,-1.3404578E-2,2E0,1E0,2.59672E3,1.3358269E2,1.651E3,3.53562E5,6.8158484E-3,-1.9038375E-3,6.1E1,4.066351E0,-9.788414E-3,-3.6262518E-3,9.750085E6,2.5454E4,2.7640502E-4,1.19827585E1,2.76E2,-5.2800467E-3,1.03E2,1.928E1,3.1773497E5,8.693602E-6,7.5982756E5,8.100412E2,1.9376624E0,1.7394744E-2,8.148046E1,9.132011E6,-7.314145E-3,-1.4515194E-2,1.75E2,1.4447577E5,2.4E1,2.416E3,1.36E2,8.1808603E-4,3.7103873E2,7.06E2,2.41E2,1.3131897E5,2.1E1,-1.7194895E-3,-2.985355E-2,-1.3070334E-2,4.4E1,2.89E2,-9.8213665E-3,-1.0458904E-3,4.6134964E2,-5.784358E-3,2.3E1,3.9912686E-3,-6.6215703E-3,1.3113985E-3,5.035714E0,1.1048219E3,1.3726724E-2,5.386213E6,1.6E1,1.3809524E0,6.95E2,6.9772056E-3,-4.4863E-3,2.7760502E-3,-4.1178083E-3,1.20086E5,-9.3315216E-4,1.1E1,-4.0587285E-3,3.9351077E2,-7.5622957E-4,-3.8059969E-3,2.5247778E6,-1.1512979E-2,4.0753E4,4.164E3,1.1646615E-3,1.4612363E7,6.34E2,8.95904E-3,-4.748102E-3,-8.655135E-3,2.0500896E-3,1.6223962E7,8.5514E-3,-9.692341E-4,1E0,1.302273E6,1.4283589E6,-7.6097334E-3,6.9003E4,-4.1883015E-3,6.763312E7,8.023379E4,7.6693073E-3,1.0494994E7,1.5768822E-3,6.966703E-3,4.8509627E-3,1.0878692E-2,-4.101762E-3,1.032771E-3,-1.304455E-2,-5.43961E-3,-6.6220183E-3,-2.5383048E-3,-1.19678555E-2,-5.95578E-3,-3.2538832E-3,8.183572E-4,1.1047705E-3,-3.166592E-3,6.0082953E-3,4.5454464E-4,1.02582155E-2,1.3080536E-3,-5.883633E-3,-3.7009668E-4,-6.6192923E-3,-4.0890014E-4,5.4641357E-3,1.05022795E-4,8.574256E-3,4.899183E-3,3.984665E-3,-2.9646142E-3,-2.5237605E-3,1.5994208E-3,3.9800634E-3,9.9655E-3,-2.1228062E-3,2.823238E-3,-8.642997E-4,5.0251805E-3],"split_indices":[61,3,54,2,61,61,66,67,5,38,56,26,56,67,47,1,61,0,24,0,65,8,0,6,109,61,67,2,1,0,0,38,63,0,0,9,9,0,67,10,0,0,65,42,46,37,61,63,0,67,1,0,0,0,42,2,9,38,0,64,2,10,42,3,0,0,0,10,0,0,0,4,0,3,0,0,0,62,61,0,60,3,63,0,0,0,0,0,9,0,12,0,42,0,0,37,0,12,38,0,12,2,0,0,0,0,5,0,0,23,9,37,0,12,0,7,42,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.71E2,5.28E2,4.43E2,4.3E2,9.8E1,3.87E2,5.6E1,8.9E1,3.41E2,4.6E1,5.2E1,2.96E2,9.1E1,4.2E1,1.4E1,7.4E1,1.5E1,5E0,3.36E2,3E1,1.6E1,4.7E1,5E0,1.92E2,1.04E2,4.4E1,4.7E1,1.2E1,3E1,6E0,8E0,4.8E1,2.6E1,1E1,5E0,2.39E2,9.7E1,7E0,2.3E1,1E1,6E0,3.5E1,1.2E1,1.61E2,3.1E1,1.6E1,8.8E1,3.3E1,1.1E1,2.8E1,1.9E1,6E0,6E0,1.5E1,1.5E1,2.3E1,2.5E1,1.9E1,7E0,2.05E2,3.4E1,2.2E1,7.5E1,1.8E1,5E0,5E0,5E0,2.4E1,1.1E1,7E0,5E0,1.55E2,6E0,2.4E1,7E0,1.1E1,5E0,6.4E1,2.4E1,1.2E1,2.1E1,1.4E1,1.4E1,1.2E1,7E0,9E0,6E0,5E0,1E1,9E0,1.4E1,9E0,1.6E1,1E1,9E0,1.96E2,9E0,1.3E1,2.1E1,6E0,1.6E1,6.8E1,7E0,7E0,1.1E1,9E0,1.5E1,6E0,5E0,4.1E1,1.14E2,1.8E1,6E0,5.5E1,9E0,1.4E1,1E1,8E0,1.3E1,8E0,6E0,5E0,9E0,7E0,5E0,5E0,5E0,7E0,7E0,1E1,6E0,1.46E2,5E1,5E0,8E0,1.5E1,6E0,1.1E1,5E0,8E0,6E1,1E1,5E0,1.3E1,2.8E1,5.2E1,6.2E1,5E0,1.3E1,1.5E1,4E1,5E0,9E0,5E0,5E0,6E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"157","size_leaf_vector":"1"}},{"base_weights":[-7.432238E-4,-2.1016153E-2,1.1887236E-1,-1.3425791E-1,-7.502902E-3,2.9779026E-1,9.977232E-2,-7.521388E-2,-1.865326E-1,-3.2774154E-2,3.973836E-2,1.9289589E-2,7.985905E-3,-2.389956E-2,1.3787493E-1,-1.0313323E-1,1.5381799E-2,-1.457969E-1,-2.0128272E-2,-1.7397394E-2,-1.014224E-1,-1.0441334E-1,4.7194596E-2,-5.571573E-2,3.458444E-3,-1.5265156E-3,1.4937E-1,-1.22726746E-1,-5.950701E-4,2.3594948E-3,-1.0510334E-3,-2.1312577E-1,-7.1525306E-2,-4.2702507E-2,3.2209598E-2,-1.8064613E-2,-6.776517E-2,-1.7360331E-2,5.147584E-3,7.803174E-2,1.3920423E-3,1.1713655E-3,-7.8349955E-2,2.3011199E-1,1.293439E-1,-8.952484E-3,-9.799811E-2,-7.3130014E-3,-1.5109843E-2,-1.7756367E-2,-1.18791945E-1,-1.7303957E-1,-2.761012E-2,1.2429544E-1,5.592598E-3,6.1745368E-2,-9.617264E-2,3.8871568E-2,1.2821023E-1,-1.10328704E-1,3.4317493E-2,-1.1074608E-1,-3.244653E-4,5.6407657E-3,1.2305451E-2,1.5394479E-1,6.1962362E-2,-1.357576E-1,-6.9363945E-4,-3.7244065E-3,2.213951E-3,-7.2377753E-3,-2.8673296E-3,-1.06057435E-1,-1.6205737E-2,4.4967286E-2,-4.5418162E-2,-2.8416396E-3,1.8342641E-1,-6.5085976E-3,6.2822565E-3,-1.8430986E-3,5.4032817E-3,-1.07859716E-1,1.7944844E-3,1.9830834E-2,1.6342999E-1,2.1431221E-1,8.985918E-2,-6.0956575E-2,-1.1138501E-2,7.620284E-2,-2.212833E-2,-2.8416135E-3,-6.701411E-3,1.2337432E-1,1.2296346E-2,6.8426635E-3,2.5652903E-3,-3.0840274E-3,-7.598191E-3,-6.5697E-3,-1.4206547E-3,-2.5389917E-4,3.1914308E-3,-2.926497E-3,1.6037773E-3,1.0919094E-2,2.63058E-3,-1.9172026E-3,1.192451E-3,-8.28895E-3,-4.198113E-3,-3.171241E-3,1.9276483E-3,3.6164296E-3,1.0285837E-2,1.1685599E-2,4.222804E-3,2.8232946E-3,8.733108E-3,-2.7087252E-4,-4.8472425E-3,-3.482769E-4,5.1245424E-3,4.9458714E-3,-2.2270496E-3,3.15653E-3,6.782397E-3,5.1035793E-3,-3.2172315E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":48,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,-1,23,25,27,29,31,-1,33,35,37,39,41,-1,-1,43,45,-1,-1,-1,47,49,51,53,-1,55,-1,-1,57,59,-1,61,63,65,-1,67,-1,-1,69,71,73,75,77,79,81,83,85,87,89,91,93,-1,-1,-1,95,97,99,-1,-1,-1,-1,-1,101,-1,103,105,-1,107,109,-1,-1,-1,111,-1,113,115,117,119,121,-1,123,125,-1,-1,127,-1,-1,129,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.3425827E0,1.2629911E0,4.4871616E-1,2.575829E-1,8.835544E-1,1.21138334E-1,6.064595E-1,1.1034939E-1,4.0010142E-1,5.0702864E-1,2.822336E-1,0E0,0E0,1.0124035E-1,1.972946E-1,5.6818098E-2,1.5991503E-2,1.8978155E-1,0E0,4.972231E-1,8.26348E-1,7.9666364E-1,3.4734672E-1,4.5957707E-2,0E0,0E0,1.1899543E-1,2.797401E-2,0E0,0E0,0E0,7.9856396E-2,5.126171E-2,5.106019E-1,3.2774544E-1,0E0,3.0026358E-1,0E0,0E0,2.83871E-1,3.716318E-1,0E0,4.3147735E-2,3.2516897E-2,1.1644232E-1,0E0,6.695989E-2,0E0,0E0,4.885031E-2,1.3680756E-2,2.8166652E-1,3.0706322E-1,3.4036562E-1,1.6581173E-1,8.788948E-2,1.0818559E-1,1.959805E-1,1.9207656E-1,1.3268101E-1,1.8609792E-1,1.2831271E-2,0E0,0E0,0E0,1.5404832E-1,1.1136161E-1,1.8216252E-2,0E0,0E0,0E0,0E0,0E0,5.1956803E-2,0E0,5.6832016E-2,2.7045867E-1,0E0,1.3794398E-1,1.0981847E-1,0E0,0E0,0E0,5.8888793E-2,0E0,1.4452508E-1,3.749916E-2,6.501621E-2,1.1980763E-1,4.2559206E-2,0E0,1.298126E-1,1.1833309E-1,0E0,0E0,4.6839416E-2,0E0,0E0,1.1576732E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,26,26,27,27,31,31,32,32,33,33,34,34,36,36,39,39,40,40,42,42,43,43,44,44,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,57,57,58,58,59,59,60,60,61,61,65,65,66,66,67,67,73,73,75,75,76,76,78,78,79,79,83,83,85,85,86,86,87,87,88,88,89,89,91,91,92,92,95,95,98,98],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,-1,24,26,28,30,32,-1,34,36,38,40,42,-1,-1,44,46,-1,-1,-1,48,50,52,54,-1,56,-1,-1,58,60,-1,62,64,66,-1,68,-1,-1,70,72,74,76,78,80,82,84,86,88,90,92,94,-1,-1,-1,96,98,100,-1,-1,-1,-1,-1,102,-1,104,106,-1,108,110,-1,-1,-1,112,-1,114,116,118,120,122,-1,124,126,-1,-1,128,-1,-1,130,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.709E3,7.7E1,2.0263722E8,1.139605E6,3.8223077E2,2.443913E1,8.079573E5,1.4806053E4,8.719101E0,1.9559006E1,1.2825651E0,1.9289589E-2,7.985905E-3,2.2E1,1E0,2.8474576E0,5.766E3,7.102397E-3,-2.0128272E-2,6.946813E7,2.1E1,2E0,1E1,4.831007E6,3.458444E-3,-1.5265156E-3,3.6444E4,1.2571428E1,-5.950701E-4,2.3594948E-3,-1.0510334E-3,6.887749E7,5.1917776E-2,5.764631E1,1.5455682E3,-1.8064613E-2,1.7754011E0,-1.7360331E-2,5.147584E-3,1.798E3,1E0,1.1713655E-3,2.4693803E2,7.0296685E6,2.171312E0,-8.952484E-3,7.701384E3,-7.3130014E-3,-1.5109843E-2,1.666618E-2,7.568006E-2,2E0,1.7083334E0,2.8546712E-1,3.2E1,2.8080197E2,2.0110958E0,1.91E2,3.7384888E6,7.079871E2,7.1E1,1.7026364E3,-3.244653E-4,5.6407657E-3,1.2305451E-2,5.965614E-1,4.88E2,1E0,-6.9363945E-4,-3.7244065E-3,2.213951E-3,-7.2377753E-3,-2.8673296E-3,3.8575E4,-1.6205737E-2,1.2105896E2,2.7655884E2,-2.8416396E-3,4.629112E6,4.39776E6,6.2822565E-3,-1.8430986E-3,5.4032817E-3,2.511352E6,1.7944844E-3,1E0,8.81059E5,3.13E2,9.812221E9,7.09E2,-1.1138501E-2,5.92E2,3.3382E4,-2.8416135E-3,-6.701411E-3,8.9064636E2,1.2296346E-2,6.8426635E-3,2.7578741E1,-3.0840274E-3,-7.598191E-3,-6.5697E-3,-1.4206547E-3,-2.5389917E-4,3.1914308E-3,-2.926497E-3,1.6037773E-3,1.0919094E-2,2.63058E-3,-1.9172026E-3,1.192451E-3,-8.28895E-3,-4.198113E-3,-3.171241E-3,1.9276483E-3,3.6164296E-3,1.0285837E-2,1.1685599E-2,4.222804E-3,2.8232946E-3,8.733108E-3,-2.7087252E-4,-4.8472425E-3,-3.482769E-4,5.1245424E-3,4.9458714E-3,-2.2270496E-3,3.15653E-3,6.782397E-3,5.1035793E-3,-3.2172315E-3],"split_indices":[2,38,7,40,61,67,37,37,63,67,63,0,0,3,14,62,9,51,0,7,67,8,8,54,0,0,9,64,0,0,0,7,47,56,42,0,62,0,0,2,109,0,65,56,51,0,37,0,0,47,47,8,62,65,8,4,47,0,60,61,8,4,0,0,0,43,0,17,0,0,0,0,0,9,0,61,61,0,9,9,0,0,0,5,0,109,9,0,40,10,0,10,9,0,0,61,0,0,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.64E2,8.25E2,1.39E2,8.7E1,7.38E2,1.2E1,1.27E2,4.2E1,4.5E1,4.81E2,2.57E2,5E0,7E0,3E1,9.7E1,3.2E1,1E1,4E1,5E0,3.94E2,8.7E1,1.2E1,2.45E2,2.3E1,7E0,6E0,9.1E1,2.6E1,6E0,5E0,5E0,2E1,2E1,2.61E2,1.33E2,8E0,7.9E1,5E0,7E0,1.46E2,9.9E1,5E0,1.8E1,1.6E1,7.5E1,5E0,2.1E1,1.5E1,5E0,1E1,1E1,2.6E1,2.35E2,2.9E1,1.04E2,1.4E1,6.5E1,8.3E1,6.3E1,2.2E1,7.7E1,1.2E1,6E0,5E0,1.1E1,5.4E1,2.1E1,1.4E1,7E0,5E0,5E0,5E0,5E0,2E1,6E0,4.6E1,1.89E2,7E0,2.2E1,9.6E1,8E0,5E0,9E0,6E1,5E0,7.3E1,1E1,1.8E1,4.5E1,1.7E1,5E0,4.4E1,3.3E1,6E0,6E0,4.4E1,1E1,8E0,1.3E1,5E0,9E0,1.3E1,7E0,1.5E1,3.1E1,1.55E2,3.4E1,1.5E1,7E0,4.6E1,5E1,1E1,5E1,1.4E1,5.9E1,5E0,5E0,1.3E1,5E0,3.6E1,9E0,8E0,9E0,1.3E1,3.1E1,5E0,2.8E1,1.4E1,3E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"131","size_leaf_vector":"1"}},{"base_weights":[-3.51611E-3,-4.6941616E-2,4.5539297E-2,-9.434822E-2,-8.032484E-3,2.7778184E-2,1.7461523E-1,-8.15176E-2,-2.5925907E-1,1.138713E-2,-2.336254E-2,1.1012936E-2,1.06828295E-1,2.010256E-1,-1.10089306E-4,2.9356303E-3,-8.821537E-2,-3.328062E-3,-2.2741549E-2,-7.6357275E-3,-1.6702966E-1,1.758594E-2,-1.4910315E-1,6.708676E-2,1.5068297E-1,2.7367864E-3,2.1653105E-1,-8.2278736E-2,-1.0364612E-2,-2.9309578E-2,4.8442062E-2,-1.9519255E-1,-1.8848042E-3,-9.9429265E-3,7.2738364E-2,-1.511778E-2,2.269514E-3,3.7381083E-2,8.969022E-3,1.1263811E-2,9.779641E-2,2.6277393E-1,1.5777515E-1,-9.9725425E-2,-3.1711522E-2,1.27253E-3,-8.54812E-2,2.8671794E-2,8.0115395E-3,-9.970161E-3,-5.1117665E-3,8.8060414E-4,-1.193639E-1,1.0604389E-1,-1.0185916E-2,5.7747483E-2,-1.435438E-3,6.71639E-3,6.444251E-4,8.836719E-3,1.6058011E-2,4.257221E-3,8.262238E-3,-1.0033656E-2,-9.4059855E-2,3.5270778E-3,-5.3025898E-2,-2.5690181E-2,8.025729E-2,-1.4362967E-1,1.0148943E-2,5.4381385E-2,-5.577285E-3,-1.04143724E-1,1.4308829E-2,-1.645378E-2,-1.0017322E-2,-2.5698992E-3,1.2614582E-1,-6.09997E-2,5.587366E-2,8.207668E-2,2.4551123E-2,-3.1152475E-3,-5.622554E-3,1.5620374E-3,-3.539333E-3,-7.926487E-3,-3.8938166E-4,6.2826583E-3,-4.2458565E-4,-5.042286E-3,-1.4474857E-2,-1.4512736E-3,2.8745045E-3,1.1828508E-3,5.6613986E-3,-1.1158173E-3,-1.4556608E-2,-1.0731478E-3,1.6040753E-3,4.399801E-3,-5.9579397E-3,6.78392E-3,-1.8597636E-3,-1.5017316E-4,-4.9594734E-3,4.4740736E-3,-6.619336E-4,1.0770675E-3,5.722107E-3,2.8169956E-3,-4.6172616E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":49,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,-1,-1,27,-1,-1,29,31,33,35,37,39,-1,41,43,-1,45,47,49,-1,51,53,-1,-1,55,-1,-1,57,59,61,63,65,67,69,71,-1,-1,-1,73,75,77,79,81,-1,-1,-1,-1,-1,-1,-1,-1,83,-1,85,87,89,91,93,95,-1,97,99,-1,101,-1,103,105,107,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0322716E0,9.325843E-1,1.022204E0,4.6129298E-1,1.089752E0,5.223149E-1,2.528901E-1,2.1213722E-1,6.575662E-1,0E0,5.9521306E-1,3.467712E-1,1.1014801E-1,9.074235E-2,0E0,0E0,1.4531279E-1,0E0,0E0,2.9190558E-1,8.463168E-2,4.7898263E-1,4.708378E-1,1.3524616E-1,1.320836E-1,0E0,7.265663E-2,1.7063689E-1,0E0,2.9729623E-1,1.5959568E-1,1.1473656E-2,0E0,2.4999687E-1,2.9107302E-1,0E0,0E0,4.6310812E-2,0E0,0E0,8.466758E-2,6.6645026E-2,1.1797607E-2,7.6042056E-2,1.2342987E-1,2.427115E-1,3.4066328E-1,2.3235777E-1,0E0,0E0,0E0,2.736762E-1,4.8519555E-1,2.503665E-1,1.0746712E-1,1.8175952E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,9.479451E-2,0E0,9.126731E-2,2.1197504E-1,1.4422475E-1,1.897142E-1,5.344186E-2,9.6534535E-2,0E0,3.617578E-1,1.3127233E-1,0E0,1.8594453E-1,0E0,2.2928691E-1,4.736434E-2,4.1195437E-2,3.1127952E-2,1.5575685E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,16,16,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,29,29,30,30,31,31,33,33,34,34,37,37,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,51,51,52,52,53,53,54,54,55,55,64,64,66,66,67,67,68,68,69,69,70,70,71,71,73,73,74,74,76,76,78,78,79,79,80,80,81,81,82,82],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,-1,-1,28,-1,-1,30,32,34,36,38,40,-1,42,44,-1,46,48,50,-1,52,54,-1,-1,56,-1,-1,58,60,62,64,66,68,70,72,-1,-1,-1,74,76,78,80,82,-1,-1,-1,-1,-1,-1,-1,-1,84,-1,86,88,90,92,94,96,-1,98,100,-1,102,-1,104,106,108,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.785302E5,6.76E2,8.83E3,8.719101E0,1.2E1,4.82425E6,2.948485E8,1.0204082E0,3.89E2,1.138713E-2,4.217427E7,2.0295714E7,3.1316666E2,1.2176E4,-1.10089306E-4,2.9356303E-3,7.234E3,-3.328062E-3,-2.2741549E-2,5.867288E2,1.665909E3,6.880842E-1,5.59E2,3.0714285E0,1.6166983E0,2.7367864E-3,5.4544777E1,1.5199841E2,-1.0364612E-2,1.879253E9,2.5E1,2.3017536E5,-1.8848042E-3,1.9873723E5,3.925926E0,-1.511778E-2,2.269514E-3,8.785115E6,8.969022E-3,1.1263811E-2,6.808571E2,2.1511E4,4.4889745E6,1.1190476E0,1.176582E6,1.8471494E7,1.731E3,4.7238097E0,8.0115395E-3,-9.970161E-3,-5.1117665E-3,1.279012E7,2.1623019E5,8.051603E7,4.24587E5,1.4E1,-1.435438E-3,6.71639E-3,6.444251E-4,8.836719E-3,1.6058011E-2,4.257221E-3,8.262238E-3,-1.0033656E-2,4.8421054E0,3.5270778E-3,1.7211578E2,8.234307E7,1.2208346E9,5.444797E0,3.4187552E6,1.7743998E5,-5.577285E-3,1.6736842E0,4.676173E6,-1.645378E-2,1.5209424E1,-2.5698992E-3,2E1,8.43E2,3.776793E8,3.716E3,5.6530495E6,-3.1152475E-3,-5.622554E-3,1.5620374E-3,-3.539333E-3,-7.926487E-3,-3.8938166E-4,6.2826583E-3,-4.2458565E-4,-5.042286E-3,-1.4474857E-2,-1.4512736E-3,2.8745045E-3,1.1828508E-3,5.6613986E-3,-1.1158173E-3,-1.4556608E-2,-1.0731478E-3,1.6040753E-3,4.399801E-3,-5.9579397E-3,6.78392E-3,-1.8597636E-3,-1.5017316E-4,-4.9594734E-3,4.4740736E-3,-6.619336E-4,1.0770675E-3,5.722107E-3,2.8169956E-3,-4.6172616E-4],"split_indices":[37,2,2,63,27,37,41,62,2,0,54,56,4,9,0,0,38,0,0,61,4,51,0,62,63,0,67,61,0,40,3,42,0,42,63,0,0,37,0,0,4,2,54,62,5,5,2,62,0,0,0,7,42,7,1,3,0,0,0,0,0,0,0,0,67,0,61,40,40,62,54,42,0,62,54,0,65,0,6,10,12,38,37,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.52E2,5.05E2,4.47E2,2.27E2,2.78E2,3.94E2,5.3E1,2.12E2,1.5E1,1.5E1,2.63E2,3.26E2,6.8E1,4.6E1,7E0,9E0,2.03E2,9E0,6E0,2.38E2,2.5E1,3.14E2,1.2E1,3.7E1,3.1E1,5E0,4.1E1,1.96E2,7E0,1.72E2,6.6E1,2E1,5E0,2.1E2,1.04E2,6E0,6E0,3.1E1,6E0,1E1,2.1E1,2.1E1,2E1,1.45E2,5.1E1,1.12E2,6E1,5.8E1,8E0,1.5E1,5E0,1.92E2,1.8E1,7.4E1,3E1,2.4E1,7E0,1.3E1,8E0,1.3E1,8E0,6E0,1.4E1,5E0,1.4E2,8E0,4.3E1,8.4E1,2.8E1,3.7E1,2.3E1,5E1,8E0,2.1E1,1.71E2,5E0,1.3E1,8E0,6.6E1,1.7E1,1.3E1,1.3E1,1.1E1,7.2E1,6.8E1,9E0,3.4E1,8E0,7.6E1,1.7E1,1.1E1,3.2E1,5E0,1.3E1,1E1,3.6E1,1.4E1,1.6E1,5E0,6E1,1.11E2,7E0,6E0,5.9E1,7E0,8E0,9E0,8E0,5E0,6E0,7E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-9.379438E-4,-5.183068E-2,4.500875E-2,-4.5814145E-2,-2.3807912E-1,5.875535E-2,-1.5958971E-1,-1.04974836E-1,-2.4687374E-2,-2.3628823E-2,-1.4312518E-3,4.2585026E-2,1.5960005E-1,-1.3824328E-2,-1.0893341E-1,-1.0983553E-2,-9.563689E-2,-5.3108595E-3,-1.04229756E-1,7.112062E-2,1.5993021E-2,2.0347846E-1,-1.4776845E-2,-5.795635E-2,-1.5153098E-1,-7.4548565E-2,-1.4258392E-1,1.1026396E-2,-7.5454846E-2,-4.184184E-2,-1.9988123E-1,9.726123E-3,6.1802976E-2,-1.2252066E-1,2.3419905E-2,2.291026E-1,1.5733024E-3,-3.3829403E-3,1.7602082E-3,-4.9707266E-3,8.011653E-4,-8.561428E-3,-3.8184815E-3,-1.3814035E-1,-4.709929E-2,-1.6098659E-1,-1.5284363E-3,7.125708E-2,-9.617788E-3,-1.2477112E-3,-1.7787194E-1,-9.250133E-2,5.5394745E-3,-3.2489562E-1,-3.7374333E-2,6.776319E-2,-3.6786413E-3,-9.656078E-3,-7.662127E-4,3.3181194E-2,-6.7554936E-2,1.965717E-1,1.7521214E-2,-4.1949167E-3,-7.533737E-3,-7.158552E-2,4.3520886E-3,-4.0161684E-3,-8.228996E-3,1.04815485E-2,5.127498E-2,-2.4762383E-2,6.646141E-2,7.495103E-2,-7.260605E-2,-8.7450355E-2,-1.24624465E-2,-5.3065997E-2,-9.244833E-3,-2.1371504E-2,-5.725349E-3,9.42285E-4,-4.567626E-3,6.022765E-2,9.438062E-3,-5.275006E-3,6.414868E-2,-1.3585989E-1,3.6518965E-3,1.4342138E-1,2.4497092E-1,-2.446571E-3,-7.580188E-3,2.547947E-3,-8.179236E-4,9.3929295E-4,4.1689514E-3,-6.0919794E-4,-5.7705306E-3,5.2362382E-3,-5.2311574E-4,1.074315E-3,6.903683E-3,-7.3747193E-3,-8.776114E-4,-5.5397274E-3,-1.6522731E-3,-3.8132616E-3,-1.8946093E-4,7.8582335E-3,2.3972832E-3,-6.9461614E-3,6.6414557E-4,3.9151483E-3,2.2398737E-4,-2.4411292E-3,-1.1047215E-2,8.845036E-3,2.7513916E-3,7.4953334E-3,1.3500507E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":50,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,-1,25,27,29,31,33,35,37,39,41,43,45,47,49,51,53,-1,55,57,59,61,-1,-1,-1,-1,-1,-1,-1,63,65,67,-1,69,71,73,75,77,-1,79,81,83,-1,-1,-1,85,87,89,-1,-1,-1,91,93,-1,-1,-1,95,97,99,101,103,105,-1,107,-1,-1,-1,-1,-1,109,-1,111,113,115,-1,117,119,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.2424748E0,4.9895215E-1,1.4268935E0,5.4985297E-1,7.710241E-1,7.645203E-1,1.9181335E-1,1.20064855E-1,5.034365E-1,0E0,0E0,3.0921948E-1,5.0034654E-1,0E0,4.5505285E-2,0E0,9.827435E-2,3.0336553E-1,3.7355632E-1,2.4126643E-1,2.2198784E-1,2.1817684E-1,4.566634E-2,4.9935803E-2,1.450336E-2,1.2964648E-1,6.128186E-2,2.6801917E-1,3.797821E-1,3.3775377E-1,4.9574977E-1,0E0,1.6177917E-1,9.636487E-2,1.8251199E-1,1.4318037E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.1917293E-2,7.086317E-2,1.8753946E-2,0E0,1.583702E-1,1.8714811E-1,1.6856438E-1,1.4877313E-1,1.19920105E-1,0E0,3.2598937E-1,4.4269204E-2,1.7071658E-1,0E0,0E0,0E0,2.1946245E-1,2.1049944E-1,6.519079E-2,0E0,0E0,0E0,5.7154313E-2,2.2083385E-2,0E0,0E0,0E0,5.8031663E-2,1.547902E-1,9.863849E-2,5.3923592E-2,7.116048E-2,1.5619084E-2,0E0,3.3894442E-2,0E0,0E0,0E0,0E0,0E0,1.5102041E-1,0E0,2.3696359E-1,1.22003496E-1,1.0662064E-1,0E0,7.209134E-2,2.5904536E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,43,43,44,44,45,45,47,47,48,48,49,49,50,50,51,51,53,53,54,54,55,55,59,59,60,60,61,61,65,65,66,66,70,70,71,71,72,72,73,73,74,74,75,75,77,77,83,83,85,85,86,86,87,87,89,89,90,90],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,-1,26,28,30,32,34,36,38,40,42,44,46,48,50,52,54,-1,56,58,60,62,-1,-1,-1,-1,-1,-1,-1,64,66,68,-1,70,72,74,76,78,-1,80,82,84,-1,-1,-1,86,88,90,-1,-1,-1,92,94,-1,-1,-1,96,98,100,102,104,106,-1,108,-1,-1,-1,-1,-1,110,-1,112,114,116,-1,118,120,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.047E3,2.4942012E-5,1E0,2.71E2,1E0,1.3388E4,1.0479E4,1.1649485E0,1.8E1,-2.3628823E-2,-1.4312518E-3,1E1,7.3274844E9,-1.3824328E-2,1.91E2,-1.0983553E-2,4.5795207E0,8.931E3,7.31E2,1.3358E4,1.8202913E2,1.9513452E2,1.11545664E5,1.36E2,9.18E2,3.6E1,6.764934E7,1.7407408E0,1.2895454E3,1.6223962E7,4.4E2,9.726123E-3,2.0295714E7,7.163082E7,1E0,1.0534078E4,1.5733024E-3,-3.3829403E-3,1.7602082E-3,-4.9707266E-3,8.011653E-4,-8.561428E-3,-3.8184815E-3,3.984E3,8.14087E2,4.1665E3,-1.5284363E-3,1.5088889E2,3.1540035E9,1.3E1,3.78E2,2.56E2,5.5394745E-3,4.0119403E2,4.5E1,7.886154E1,-3.6786413E-3,-9.656078E-3,-7.662127E-4,3.526271E6,5.9456447E-6,1.9705E4,1.7521214E-2,-4.1949167E-3,-7.533737E-3,3.53E2,5.7018555E4,-4.0161684E-3,-8.228996E-3,1.04815485E-2,2.992306E4,2.335307E9,1.1304348E1,1.1E1,1.6100003E8,1.686E3,-1.24624465E-2,6.74502E0,-9.244833E-3,-2.1371504E-2,-5.725349E-3,9.42285E-4,-4.567626E-3,1.1190476E0,9.438062E-3,7.582512E7,3.6923077E0,1.7951853E1,3.6518965E-3,5.236559E0,5.967713E8,-2.446571E-3,-7.580188E-3,2.547947E-3,-8.179236E-4,9.3929295E-4,4.1689514E-3,-6.0919794E-4,-5.7705306E-3,5.2362382E-3,-5.2311574E-4,1.074315E-3,6.903683E-3,-7.3747193E-3,-8.776114E-4,-5.5397274E-3,-1.6522731E-3,-3.8132616E-3,-1.8946093E-4,7.8582335E-3,2.3972832E-3,-6.9461614E-3,6.6414557E-4,3.9151483E-3,2.2398737E-4,-2.4411292E-3,-1.1047215E-2,8.845036E-3,2.7513916E-3,7.4953334E-3,1.3500507E-2],"split_indices":[2,46,78,2,23,2,9,62,3,0,0,8,5,0,0,0,67,38,2,9,61,67,42,0,0,2,7,62,42,5,10,0,56,7,6,4,0,0,0,0,0,0,0,9,60,56,0,42,40,3,0,10,0,4,8,67,0,0,0,54,46,2,0,0,0,38,54,0,0,0,42,40,67,3,5,10,0,63,0,0,0,0,0,62,0,7,62,65,0,63,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.57E2,4.54E2,5.03E2,4.41E2,1.3E1,4.72E2,3.1E1,1.15E2,3.26E2,5E0,8E0,4.08E2,6.4E1,7E0,2.4E1,6E0,1.09E2,2.63E2,6.3E1,1.96E2,2.12E2,5.1E1,1.3E1,1.2E1,1.2E1,7.7E1,3.2E1,2.14E2,4.9E1,3.9E1,2.4E1,1.1E1,1.85E2,1E1,2.02E2,4.4E1,7E0,6E0,7E0,7E0,5E0,7E0,5E0,2.2E1,5.5E1,2.7E1,5E0,5.4E1,1.6E2,2.9E1,2E1,3E1,9E0,1.3E1,1.1E1,1.78E2,7E0,5E0,5E0,1.83E2,1.9E1,3.8E1,6E0,9E0,1.3E1,3.7E1,1.8E1,6E0,2.1E1,5E0,4.9E1,1.34E2,2.6E1,1.4E1,1.5E1,1.1E1,9E0,2.3E1,7E0,7E0,6E0,6E0,5E0,1.7E2,8E0,8.2E1,1.01E2,1.3E1,6E0,2E1,1.8E1,3.2E1,5E0,5E0,1.3E1,2.8E1,2.1E1,1.21E2,1.3E1,1.6E1,1E1,9E0,5E0,5E0,1E1,6E0,5E0,1.4E1,9E0,1.1E1,1.59E2,9E0,7.3E1,7.5E1,2.6E1,8E0,5E0,1.2E1,8E0,8E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"121","size_leaf_vector":"1"}},{"base_weights":[1.4242395E-3,-5.7370678E-2,4.200858E-2,-4.7763243E-2,-2.5807452E-1,5.4443557E-2,-9.642855E-2,-8.1344195E-2,-1.3400677E-2,7.964704E-4,-4.3478778E-1,2.9805457E-2,1.3021334E-1,-1.4612034E-1,8.400023E-2,2.908113E-3,-8.872927E-2,-1.6682106E-1,-2.120532E-3,-3.0013673E-2,-6.9718002E-3,1.4054719E-2,1.03037E-1,1.6783793E-1,6.665724E-2,-1.1318275E-2,-1.1190632E-1,-1.3498871E-3,8.495415E-3,-7.65207E-2,-1.7515056E-1,-1.4314383E-2,-1.8423592E-3,2.9325621E-2,-4.8823558E-2,1.8575866E-2,-1.0889486E-2,7.507385E-2,1.87445E-1,1.3455619E-1,2.5652117E-1,-1.4771484E-3,8.34313E-2,2.8240407E-4,-1.4165808E-1,-8.909749E-2,3.9074756E-3,-2.5443214E-1,-2.1361286E-4,1.355759E-1,1.0528553E-2,2.0487212E-2,-6.742266E-2,-2.131089E-3,6.493134E-2,4.1326383E-4,9.534211E-2,1.0298681E-2,4.045089E-3,1.7268308E-1,5.371055E-2,1.4795349E-2,5.352016E-3,1.0960234E-1,1.4605651E-2,-8.582601E-3,-1.4805257E-3,-1.7390703E-1,-7.9125375E-2,4.3863663E-3,-2.6518771E-2,-1.5358162E-2,-3.916811E-3,1.8705685E-3,1.0476893E-2,2.262061E-2,-8.056937E-2,2.9761079E-3,-2.0787958E-3,-8.0297E-2,2.5701597E-3,8.205787E-2,-2.456964E-2,-4.189641E-2,8.705001E-2,1.1266813E-1,-1.2709734E-3,2.0432262E-1,2.5133733E-3,5.5599282E-3,-1.728925E-2,7.985417E-3,7.164542E-2,5.3501604E-3,-5.665211E-3,-4.4580917E-3,-9.58E-3,-3.2958812E-3,-8.269957E-3,1.5492848E-3,-2.7494323E-3,-4.9511513E-5,2.6830717E-3,-7.0369313E-3,1.8325094E-4,-3.1381347E-3,-7.7076294E-3,-7.96213E-4,5.540112E-3,-1.1463083E-2,-7.7956857E-4,1.7827273E-3,-3.955877E-3,6.2917536E-3,2.1318113E-3,6.847669E-3,2.67359E-3,6.93297E-3,1.2260565E-2,2.2210658E-3,-3.6917857E-3,-1.7561134E-3,5.4735458E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":51,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,23,25,27,-1,29,31,33,-1,-1,35,37,39,41,-1,43,-1,-1,45,47,-1,-1,49,51,53,-1,55,57,59,61,-1,63,-1,65,67,69,71,-1,73,75,77,79,81,83,-1,85,-1,-1,87,89,-1,-1,91,93,-1,-1,95,97,-1,99,-1,-1,-1,-1,101,103,-1,-1,105,-1,107,109,111,113,115,-1,117,-1,-1,119,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.331295E0,7.570449E-1,1.0002555E0,4.3971038E-1,8.829576E-1,9.874021E-1,4.4263396E-1,2.1083546E-1,3.2846102E-1,0E0,5.6654286E-1,4.6223903E-1,2.9967785E-1,9.9870086E-2,1.286006E-1,0E0,1.8073213E-1,2.2187924E-1,2.62848E-1,0E0,0E0,3.777309E-1,1.5351003E-1,2.0455766E-1,8.526224E-2,0E0,1.0617694E-1,0E0,0E0,1.6522723E-1,2.962954E-1,0E0,0E0,2.1227232E-1,9.419781E-2,3.140194E-1,0E0,7.2910964E-2,3.9476156E-2,1.7875099E-1,1.4806092E-1,0E0,7.585245E-2,0E0,1.0818699E-1,1.05487585E-1,6.592841E-2,1.67324E-1,0E0,1.2841201E-1,1.0316617E-1,4.847583E-2,9.4573885E-2,4.3083692E-1,2.4229875E-1,0E0,9.239897E-2,0E0,0E0,1.4267218E-1,1.00651935E-1,0E0,0E0,6.1210483E-2,1.9303732E-1,0E0,0E0,1.5639931E-2,9.000081E-2,0E0,3.728087E-2,0E0,0E0,0E0,0E0,6.844345E-2,6.669237E-2,0E0,0E0,4.527822E-2,0E0,1.7989165E-1,3.069486E-1,6.596303E-2,1.6140473E-1,6.260443E-2,0E0,6.5786004E-2,0E0,0E0,4.8444092E-2,0E0,1.1097199E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,21,21,22,22,23,23,24,24,26,26,29,29,30,30,33,33,34,34,35,35,37,37,38,38,39,39,40,40,42,42,44,44,45,45,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,56,56,59,59,60,60,63,63,64,64,67,67,68,68,70,70,75,75,76,76,79,79,81,81,82,82,83,83,84,84,85,85,87,87,90,90,92,92],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,24,26,28,-1,30,32,34,-1,-1,36,38,40,42,-1,44,-1,-1,46,48,-1,-1,50,52,54,-1,56,58,60,62,-1,64,-1,66,68,70,72,-1,74,76,78,80,82,84,-1,86,-1,-1,88,90,-1,-1,92,94,-1,-1,96,98,-1,100,-1,-1,-1,-1,102,104,-1,-1,106,-1,108,110,112,114,116,-1,118,-1,-1,120,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.9E2,9.932432E0,3.5056704E7,1.5341322E2,1.1742308E2,1.1464103E3,1.3655363E2,3.5511714E-1,2.86152E5,7.964704E-4,1.8E1,2.9652428E6,1.7103828E7,1E0,1.1E1,2.908113E-3,2.78E2,7.2744107E0,2.4624279E0,-3.0013673E-2,-6.9718002E-3,3.3000212E5,7.82261E6,2.2371138E3,1.3631968E3,-1.1318275E-2,7.080933E2,-1.3498871E-3,8.495415E-3,3.11626E8,1.409E3,-1.4314383E-2,-1.8423592E-3,8.771579E2,4.084788E0,5.867288E2,-1.0889486E-2,5.1942E4,1.2706E4,2.7578741E1,1.159499E6,-1.4771484E-3,4.6553187E0,2.8240407E-4,4.2E1,6.909091E0,2.8784848E2,6.769866E7,-2.1361286E-4,3.434405E7,7.2744107E0,6E0,4.7238097E0,8.589474E0,2.5807297E-2,4.1326383E-4,7.7191065E6,1.0298681E-2,4.045089E-3,3.0359353E5,3.09627E5,1.4795349E-2,5.352016E-3,6.9756895E6,3.82E2,-8.582601E-3,-1.4805257E-3,1.3888889E1,5.592863E6,4.3863663E-3,7.501215E6,-1.5358162E-2,-3.916811E-3,1.8705685E-3,1.0476893E-2,1.142E3,1.2183674E1,2.9761079E-3,-2.0787958E-3,8.39E2,2.5701597E-3,3.7788504E7,1.6494E4,8.620714E2,1.1E1,5.1291872E8,-1.2709734E-3,3.5849323E1,2.5133733E-3,5.5599282E-3,1.1521033E0,7.985417E-3,2.338255E3,5.3501604E-3,-5.665211E-3,-4.4580917E-3,-9.58E-3,-3.2958812E-3,-8.269957E-3,1.5492848E-3,-2.7494323E-3,-4.9511513E-5,2.6830717E-3,-7.0369313E-3,1.8325094E-4,-3.1381347E-3,-7.7076294E-3,-7.96213E-4,5.540112E-3,-1.1463083E-2,-7.7956857E-4,1.7827273E-3,-3.955877E-3,6.2917536E-3,2.1318113E-3,6.847669E-3,2.67359E-3,6.93297E-3,1.2260565E-2,2.2210658E-3,-3.6917857E-3,-1.7561134E-3,5.4735458E-3],"split_indices":[2,63,54,61,61,61,65,63,7,0,8,37,56,24,3,0,0,65,62,0,0,42,37,61,61,0,4,0,0,40,10,0,0,42,63,61,0,1,38,65,9,0,47,0,6,61,42,7,0,7,65,8,62,67,47,0,56,0,0,42,9,0,0,54,0,0,0,4,5,0,5,0,0,0,0,38,65,0,0,2,0,7,1,4,8,7,0,67,0,0,51,0,61,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.75E2,3.98E2,5.77E2,3.81E2,1.7E1,5.3E2,4.7E1,1.92E2,1.89E2,7E0,1E1,4.01E2,1.29E2,3.7E1,1E1,9E0,1.83E2,1.2E1,1.77E2,5E0,5E0,3.31E2,7E1,8E1,4.9E1,8E0,2.9E1,5E0,5E0,1.62E2,2.1E1,5E0,7E0,1.06E2,7.1E1,3.26E2,5E0,5.4E1,1.6E1,6E1,2E1,7E0,4.2E1,6E0,2.3E1,1.4E2,2.2E1,1.4E1,7E0,1.5E1,9.1E1,1.5E1,5.6E1,2.26E2,1E2,1.3E1,4.1E1,1.1E1,5E0,4E1,2E1,1.3E1,7E0,3E1,1.2E1,1.6E1,7E0,1.3E1,1.27E2,5E0,1.7E1,9E0,5E0,8E0,7E0,8.1E1,1E1,9E0,6E0,5.1E1,5E0,4.7E1,1.79E2,1.7E1,8.3E1,3.6E1,5E0,3.1E1,9E0,1E1,1E1,1E1,2E1,7E0,5E0,5E0,8E0,1.19E2,8E0,6E0,1.1E1,4.9E1,3.2E1,5E0,5E0,4.6E1,5E0,1.3E1,3.4E1,5E0,1.74E2,6E0,1.1E1,3.7E1,4.6E1,2.1E1,1.5E1,1.8E1,1.3E1,5E0,5E0,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[-1.7275658E-3,-5.5718154E-2,3.0596267E-2,-4.7772467E-2,-2.0948473E-1,1.4473365E-2,1.2583564E-1,-8.751124E-2,-2.1630164E-2,-2.2699772E-3,-1.608996E-2,2.5731033E-2,-1.1675887E-1,2.0519847E-1,5.4088935E-2,-9.4166994E-2,1.7498103E-4,1.2766741E-2,-7.510495E-2,1.77143E-2,1.3739388E-1,-1.5731171E-1,4.45425E-3,3.2488754E-1,1.20389506E-1,-7.60971E-2,9.717875E-2,-1.0533565E-1,-3.759707E-2,8.958531E-3,2.0531327E-3,-4.53184E-2,-2.5782803E-1,2.384849E-2,-1.17847405E-1,7.054716E-4,1.7239898E-1,-1.2125416E-2,-1.2605558E-1,-2.2180586E-3,2.596962E-3,9.725079E-3,1.9208463E-2,9.095317E-3,6.309397E-3,-8.568461E-3,1.2914924E-3,1.3412751E-1,-2.0549968E-3,-8.703701E-2,-1.6875169E-1,7.85518E-3,-8.3763435E-2,3.7485033E-2,-2.2428785E-2,2.2318939E-2,-1.0624681E-1,-2.3500396E-2,1.5681074E-3,1.0642061E-2,2.0290308E-2,-1.6078472E-2,1.4350941E-2,9.740495E-3,3.2990775E-3,5.2179297E-5,-7.3267836E-3,-2.2236325E-3,2.760345E-3,8.09865E-2,1.7478397E-1,-5.2293774E-2,-1.5217419E-1,-1.1916578E-2,-7.395417E-2,1.7692354E-3,-8.91672E-4,-6.2479177E-3,-8.7746663E-4,4.6316724E-2,-1.5556894E-3,-4.328309E-3,-7.883985E-3,-4.5969475E-2,8.557361E-2,-1.2650868E-2,-7.1725704E-2,2.4104737E-2,-7.088241E-3,-2.0448668E-3,4.1692187E-3,5.324377E-3,1.9171893E-3,4.999861E-3,9.627673E-3,-3.1697445E-3,1.2356248E-3,-8.51145E-3,-3.052574E-3,-4.9324003E-3,-1.6546788E-3,-4.7283882E-4,2.575497E-3,1.9558894E-3,-1.4894274E-3,1.2845739E-3,-4.6536843E-3,2.1196783E-3,5.2393056E-3,5.2483263E-4,-4.8584905E-3,3.981787E-3,6.639977E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":52,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,-1,29,31,33,35,37,39,41,43,45,47,49,51,-1,53,55,57,59,61,-1,63,-1,65,-1,-1,-1,-1,-1,67,-1,-1,69,-1,71,73,75,77,79,81,83,85,-1,-1,-1,87,-1,89,-1,-1,-1,-1,-1,-1,91,93,95,97,-1,99,-1,-1,-1,-1,101,-1,-1,103,105,107,-1,109,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6404314E0,4.2005932E-1,9.010734E-1,3.471651E-1,3.5776538E-1,7.475983E-1,4.7289288E-1,8.115971E-2,3.761897E-1,0E0,0E0,4.1382086E-1,1.9732225E-1,3.6691523E-1,2.6544088E-1,7.438409E-2,0E0,2.4107161E-1,4.2373085E-1,3.6453974E-1,1.2997371E-1,6.4591706E-2,3.228853E-2,7.758057E-2,2.1446812E-1,1.4054649E-1,1.888879E-1,1.04836345E-1,4.6822008E-2,0E0,1.0406621E-1,2.9084378E-1,8.147602E-1,3.0027306E-1,5.7935375E-1,0E0,7.630992E-2,0E0,1.01448804E-1,0E0,0E0,0E0,0E0,0E0,3.4577258E-2,0E0,0E0,4.633957E-2,0E0,1.7761451E-1,1.6468984E-1,1.0568328E-2,3.375838E-2,3.2284103E-2,7.243509E-2,1.5079886E-1,1.9298941E-1,0E0,0E0,0E0,2.7498758E-1,0E0,6.130419E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.4009386E-2,1.0529339E-2,7.136156E-2,6.355399E-2,0E0,1.1073723E-2,0E0,0E0,0E0,0E0,2.3415923E-2,0E0,0E0,7.4055806E-2,7.080077E-2,1.4475167E-2,0E0,8.883016E-2,2.4234952E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,30,30,31,31,32,32,33,33,34,34,36,36,38,38,44,44,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,60,60,62,62,69,69,70,70,71,71,72,72,74,74,79,79,82,82,83,83,84,84,86,86,87,87],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,-1,30,32,34,36,38,40,42,44,46,48,50,52,-1,54,56,58,60,62,-1,64,-1,66,-1,-1,-1,-1,-1,68,-1,-1,70,-1,72,74,76,78,80,82,84,86,-1,-1,-1,88,-1,90,-1,-1,-1,-1,-1,-1,92,94,96,98,-1,100,-1,-1,-1,-1,102,-1,-1,104,106,108,-1,110,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,1.05039425E1,1.6471942E3,3.72E2,4.327E3,2.856934E7,7.541586E1,2.3E1,1.01E2,-2.2699772E-3,-1.608996E-2,5.405423E6,5.97E3,1.9359898E5,7.25E2,9.07021E5,1.7498103E-4,1.1E1,1.1949272E7,1.6007428E7,2.47E2,2.41E2,1.5E1,2.480065E6,2.1668362E1,2.6095572E0,2.4157143E2,3.53969E5,1.8332372E5,8.958531E-3,1.9619734E0,2.6742857E0,5.1942E4,2.04115E5,1.172766E3,7.054716E-4,1.2706E4,-1.2125416E-2,8.620714E2,-2.2180586E-3,2.596962E-3,9.725079E-3,1.9208463E-2,9.095317E-3,2.3464934E3,-8.568461E-3,1.2914924E-3,2.101E3,-2.0549968E-3,4.090909E0,1.1851851E1,2.707673E6,1.7909248E5,1.2751362E8,7.402E3,5.526796E7,1.5589E4,-2.3500396E-2,1.5681074E-3,1.0642061E-2,9.185E1,-1.6078472E-2,2.5842668E7,9.740495E-3,3.2990775E-3,5.2179297E-5,-7.3267836E-3,-2.2236325E-3,2.760345E-3,1.016E4,1.1492429E5,3.54E1,6.782E4,-1.1916578E-2,2.08E2,1.7692354E-3,-8.91672E-4,-6.2479177E-3,-8.7746663E-4,1E0,-1.5556894E-3,-4.328309E-3,1.1652361E6,1.9470909E2,3.3512E4,-1.2650868E-2,6.300312E6,6E0,-7.088241E-3,-2.0448668E-3,4.1692187E-3,5.324377E-3,1.9171893E-3,4.999861E-3,9.627673E-3,-3.1697445E-3,1.2356248E-3,-8.51145E-3,-3.052574E-3,-4.9324003E-3,-1.6546788E-3,-4.7283882E-4,2.575497E-3,1.9558894E-3,-1.4894274E-3,1.2845739E-3,-4.6536843E-3,2.1196783E-3,5.2393056E-3,5.2483263E-4,-4.8584905E-3,3.981787E-3,6.639977E-4],"split_indices":[2,63,61,2,1,54,67,3,0,0,0,37,2,42,0,9,0,10,54,60,0,10,3,1,65,62,67,9,56,0,62,65,1,5,4,0,38,0,4,0,0,0,0,0,4,0,0,0,0,67,67,5,37,7,1,5,9,0,0,0,63,0,56,0,0,0,0,0,0,2,42,64,9,0,2,0,0,0,0,109,0,0,54,4,1,0,5,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.38E2,3.51E2,5.87E2,3.35E2,1.6E1,5.03E2,8.4E1,1.32E2,2.03E2,8E0,8E0,4.64E2,3.9E1,3.9E1,4.5E1,1.23E2,9E0,1.24E2,7.9E1,4.34E2,3E1,2.9E1,1E1,1.5E1,2.4E1,1.1E1,3.4E1,1.02E2,2.1E1,6E0,1.18E2,6.9E1,1E1,4.16E2,1.8E1,7E0,2.3E1,5E0,2.4E1,5E0,5E0,8E0,7E0,1.4E1,1E1,5E0,6E0,2.7E1,7E0,8.1E1,2.1E1,1.1E1,1E1,4.8E1,7E1,3.3E1,3.6E1,5E0,5E0,6E0,4.1E2,6E0,1.2E1,1.6E1,7E0,5E0,1.9E1,5E0,5E0,1.3E1,1.4E1,5.4E1,2.7E1,1E1,1.1E1,5E0,6E0,5E0,5E0,4.3E1,5E0,1.1E1,5.9E1,1.6E1,1.7E1,5E0,3.1E1,4.02E2,8E0,7E0,5E0,6E0,7E0,6E0,8E0,4.5E1,9E0,1.9E1,8E0,5E0,6E0,6E0,3.7E1,1.9E1,4E1,7E0,9E0,8E0,9E0,9E0,2.2E1,5.4E1,3.48E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[1.5338714E-4,-2.689368E-2,7.840754E-2,-7.142203E-2,-2.5060507E-3,1.701167E-2,6.871554E-2,-5.3635705E-2,-2.605913E-1,-4.2879436E-2,2.6865523E-2,7.541552E-2,-7.92569E-3,5.443391E-2,-7.008301E-2,-1.9875814E-3,-4.080496E-1,-3.2425944E-2,-1.794454E-1,1.8432481E-2,8.480183E-3,-7.160231E-3,8.196558E-2,3.01694E-2,7.171115E-3,-7.644157E-2,5.684411E-3,-2.4671836E-2,-1.04899295E-2,-9.727119E-2,-1.632197E-2,-1.0278341E-2,-4.0101935E-3,1.1480445E-2,1.5070762E-1,4.09065E-2,1.1338315E-1,-9.0456766E-4,5.2106895E-2,-8.251631E-2,4.8117824E-3,-1.3588168E-1,-3.158369E-2,1.5008657E-2,-4.149055E-2,5.047791E-2,-1.6187696E-2,1.136651E-2,1.6200143E-3,1.1663945E-1,9.651162E-3,5.6966983E-2,1.4494279E-1,3.9068786E-3,3.0817201E-2,-1.2608096E-2,-7.296852E-2,-1.5839922E-1,-2.1196215E-3,7.547278E-4,-3.029451E-3,9.526182E-2,-5.6923046E-3,-8.8991165E-2,-2.2030193E-3,6.056163E-2,-5.76821E-3,1.1345938E-1,-3.0687835E-2,-2.7957596E-3,1.5442836E-1,-6.82536E-3,6.7945262E-3,1.458531E-1,-4.1893283E-3,1.323597E-1,1.3291715E-2,-9.735322E-4,3.6290602E-3,-1.22634005E-2,-3.0744215E-3,-8.742579E-3,-3.9117695E-3,3.391443E-4,6.6496013E-3,1.3509594E-3,-2.2562528E-3,-5.2200216E-3,6.141832E-4,1.3459361E-3,-2.736959E-3,1.5696478E-3,5.6105657E-3,8.697431E-3,1.1734086E-3,-1.407817E-2,-8.19604E-4,9.358771E-3,2.2261634E-3,-6.842194E-3,3.1991547E-4,4.2042164E-3,9.43326E-3,4.425319E-3,-1.7927836E-3,6.959935E-3,1.8439812E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":53,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,-1,23,25,-1,27,29,31,33,-1,-1,35,37,-1,39,-1,-1,-1,41,43,-1,-1,45,47,49,51,-1,53,55,-1,57,59,61,63,65,67,-1,-1,69,71,73,75,-1,77,-1,79,81,-1,-1,-1,83,85,87,89,91,-1,93,95,-1,97,99,-1,101,103,105,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0847895E0,7.947298E-1,6.8156326E-1,8.574611E-1,5.6323534E-1,0E0,4.0767407E-1,4.2684895E-1,6.8893576E-1,2.7969956E-1,3.581754E-1,3.7379205E-1,0E0,7.307528E-2,2.6286066E-1,0E0,1.7236209E-1,1.936901E-1,3.564465E-2,2.3920023E-1,0E0,0E0,2.9721653E-1,3.0432815E-2,0E0,2.2851849E-1,0E0,0E0,0E0,8.954874E-2,1.1969338E-1,0E0,0E0,2.7056396E-1,1.333426E-1,2.4268137E-1,2.2676551E-1,0E0,1.038738E-2,3.3245337E-1,0E0,3.947869E-2,2.5255032E-2,1.1351186E-1,1.5656322E-1,1.9083688E-1,2.7982724E-1,0E0,0E0,2.0984346E-1,1.6777566E-1,2.6697245E-1,1.1166799E-1,0E0,3.3556957E-2,0E0,2.2251493E-1,2.4152875E-2,0E0,0E0,0E0,5.81187E-2,8.361169E-2,9.2820585E-2,8.50074E-2,1.5564656E-1,0E0,9.311207E-2,4.6704248E-1,0E0,1.1612517E-1,1.3029815E-1,0E0,4.505655E-2,1.0658053E-1,1.2311387E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,19,19,22,22,23,23,25,25,29,29,30,30,33,33,34,34,35,35,36,36,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,54,54,56,56,57,57,61,61,62,62,63,63,64,64,65,65,67,67,68,68,70,70,71,71,73,73,74,74,75,75],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,-1,24,26,-1,28,30,32,34,-1,-1,36,38,-1,40,-1,-1,-1,42,44,-1,-1,46,48,50,52,-1,54,56,-1,58,60,62,64,66,68,-1,-1,70,72,74,76,-1,78,-1,80,82,-1,-1,-1,84,86,88,90,92,-1,94,96,-1,98,100,-1,102,104,106,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.908E3,1E0,3.3817584E7,9.369565E0,4.539777E5,1.701167E-2,5.619497E7,1.0204082E0,4.327E3,3.0977E4,7.168071E6,1.25E0,-7.92569E-3,5.175119E-1,1.6158306E-4,-1.9875814E-3,4.0568292E2,1.545E2,3.004207E2,2.9308079E1,8.480183E-3,-7.160231E-3,1.1251919E6,5.04E2,7.171115E-3,1.4559597E0,5.684411E-3,-2.4671836E-2,-1.04899295E-2,3.6616925E5,1E0,-1.0278341E-2,-4.0101935E-3,1E0,4.857143E0,1E1,5.831829E6,-9.0456766E-4,1.2253E4,1.4076087E0,4.8117824E-3,2.560356E5,1.257E4,6.12E2,8.83425E7,1.3062992E1,1.4414759E0,1.136651E-2,1.6200143E-3,1.53505E0,3.8537518E2,3.5046455E6,4.907764E3,3.9068786E-3,1.3E1,-1.2608096E-2,1.1190476E0,1.1146532E8,-2.1196215E-3,7.547278E-4,-3.029451E-3,6.742E3,1.397998E6,5.8536587E0,3.425E1,6.7723384E7,-5.76821E-3,3.074398E0,1.53505E0,-2.7957596E-3,2.1298597E0,1.33E2,6.7945262E-3,2.5338028E1,3.0238653E8,4.7E1,1.3291715E-2,-9.735322E-4,3.6290602E-3,-1.22634005E-2,-3.0744215E-3,-8.742579E-3,-3.9117695E-3,3.391443E-4,6.6496013E-3,1.3509594E-3,-2.2562528E-3,-5.2200216E-3,6.141832E-4,1.3459361E-3,-2.736959E-3,1.5696478E-3,5.6105657E-3,8.697431E-3,1.1734086E-3,-1.407817E-2,-8.19604E-4,9.358771E-3,2.2261634E-3,-6.842194E-3,3.1991547E-4,4.2042164E-3,9.43326E-3,4.425319E-3,-1.7927836E-3,6.959935E-3,1.8439812E-3],"split_indices":[2,109,7,63,37,0,56,62,1,38,37,62,0,51,46,0,4,64,61,65,0,0,37,7,0,51,0,0,0,54,23,0,0,23,63,8,54,0,9,63,0,56,12,38,7,65,62,0,0,62,65,54,4,0,3,0,62,40,0,0,0,1,54,65,67,7,0,65,62,0,47,0,0,67,7,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.83E2,7.31E2,2.52E2,2.58E2,4.73E2,7E0,2.45E2,2.37E2,2.1E1,1.99E2,2.74E2,2.39E2,6E0,3.1E1,2.06E2,9E0,1.2E1,1.86E2,1.3E1,2.61E2,1.3E1,6E0,2.33E2,2.6E1,5E0,2E2,6E0,6E0,6E0,3.6E1,1.5E2,8E0,5E0,2.49E2,1.2E1,1.02E2,1.31E2,8E0,1.8E1,1.94E2,6E0,2.2E1,1.4E1,6.7E1,8.3E1,1.03E2,1.46E2,6E0,6E0,2.9E1,7.3E1,4.8E1,8.3E1,6E0,1.2E1,8E0,1.86E2,1.7E1,5E0,6E0,8E0,1.3E1,5.4E1,3.7E1,4.6E1,9.8E1,5E0,1.4E1,1.32E2,5E0,2.4E1,6.6E1,7E0,1.9E1,2.9E1,7.8E1,5E0,6E0,6E0,5E0,1.81E2,1.1E1,6E0,5E0,8E0,3E1,2.4E1,3E1,7E0,3E1,1.6E1,6.9E1,2.9E1,7E0,7E0,5E0,1.27E2,1.6E1,8E0,5E0,6.1E1,1.1E1,8E0,7E0,2.2E1,6.5E1,1.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"107","size_leaf_vector":"1"}},{"base_weights":[3.0481133E-3,-3.8193606E-2,4.3784793E-2,-4.806604E-2,4.0762283E-2,1.3823062E-2,9.630749E-2,-4.2037725E-2,-2.0368186E-1,7.1886644E-2,-3.3038437E-2,4.5122756E-3,2.72422E-1,7.6656386E-2,1.4451015E-1,-5.9577554E-2,-4.994141E-4,-4.773444E-2,-2.1141354E-2,-1.569509E-2,1.0685429E-1,4.978367E-2,-7.3528057E-3,1.2772991E-1,-9.790176E-3,7.750426E-3,1.5423308E-2,-1.4917393E-2,8.783699E-2,7.788991E-2,1.7273968E-1,-4.2838514E-2,-1.1300877E-1,1.0565463E-2,-1.064962E-1,1.7377206E-3,-5.798278E-3,8.676437E-4,-2.4687366E-3,7.743945E-3,2.699763E-3,3.6939787E-4,3.8654783E-3,1.05083E-3,1.5110864E-1,-1.0858339E-1,8.40597E-3,-7.2331065E-3,3.3016268E-3,9.6271284E-2,-2.656367E-3,5.547538E-3,5.940789E-5,9.77455E-3,3.828037E-3,-7.99544E-2,-4.686608E-3,4.1283728E-3,-1.3333495E-1,-4.3702926E-2,3.293814E-2,-2.4253214E-3,-6.3920566E-3,1.7895813E-1,2.3177303E-3,-1.2249458E-2,-7.62455E-2,-1.3402244E-1,2.1616029E-2,7.383929E-2,1.7791232E-1,-1.2907474E-2,-6.8415135E-2,7.358328E-3,-1.3713953E-2,-1.1113278E-1,-1.5285532E-2,1.8710053E-3,-6.314882E-2,5.5773046E-2,-5.135113E-2,6.344082E-3,1.1036266E-2,8.388599E-4,-9.885721E-2,8.982809E-3,-1.8600913E-2,-7.767447E-2,3.2764353E-2,6.0400367E-2,8.0095995E-3,4.2474973E-3,1.9681838E-1,-5.389891E-3,-1.6721066E-3,-2.8784259E-3,1.8439836E-4,-1.0761559E-3,-5.6850757E-3,-6.509351E-3,-1.6580194E-3,3.3160488E-3,1.5043175E-4,3.202467E-4,-5.8013285E-3,-1.6419897E-3,-5.2547306E-3,-3.628755E-3,3.9045555E-3,-1.2242476E-3,-9.7461445E-3,7.90321E-3,1.0181373E-3,4.579927E-3,7.6431077E-4,5.6905127E-3,1.0378918E-2],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":54,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,39,41,-1,43,45,-1,-1,47,49,51,53,55,57,59,61,-1,-1,-1,-1,-1,-1,-1,-1,-1,63,65,67,-1,-1,69,-1,-1,-1,-1,-1,71,73,-1,75,77,79,-1,-1,81,-1,-1,83,85,87,89,91,-1,93,-1,95,97,-1,-1,99,101,103,-1,-1,-1,105,107,-1,109,111,113,-1,-1,115,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6565224E0,3.8390774E-1,7.7920955E-1,4.004494E-1,1.2871003E-1,7.620305E-1,1.5942478E-1,3.0670035E-1,6.069221E-1,1.2111564E-1,1.8449044E-1,5.422809E-1,1.4008462E-2,1.3441074E-1,8.245897E-2,2.5989747E-1,1.489032E-1,7.7048026E-2,0E0,1.6477065E-2,7.2111845E-2,1.4759872E-2,0E0,7.5191975E-2,4.9706572E-1,0E0,0E0,1.9310103E-1,1.4746147E-1,5.411718E-2,1.0238814E-1,3.2106677E-1,2.9925215E-1,1.4106436E-1,1.3143763E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,6.430709E-2,1.95983E-1,4.4316512E-1,0E0,0E0,1.8831384E-1,0E0,0E0,0E0,0E0,0E0,2.4401915E-1,1.687283E-1,0E0,2.4203348E-1,5.8101617E-2,1.6069248E-1,0E0,0E0,1.7222524E-2,0E0,0E0,8.070235E-2,7.6686144E-1,2.405333E-1,1.0639751E-1,1.9540012E-2,0E0,1.6587085E-1,0E0,9.255658E-2,5.6264818E-2,0E0,0E0,5.4202102E-2,5.335711E-2,7.811959E-2,0E0,0E0,0E0,2.3397148E-2,9.8335706E-2,0E0,1.4382505E-1,2.8506255E-1,1.313084E-1,0E0,0E0,8.440256E-3,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,20,20,21,21,23,23,24,24,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,44,44,45,45,46,46,49,49,55,55,56,56,58,58,59,59,60,60,63,63,66,66,67,67,68,68,69,69,70,70,72,72,74,74,75,75,78,78,79,79,80,80,84,84,85,85,87,87,88,88,89,89,92,92],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,40,42,-1,44,46,-1,-1,48,50,52,54,56,58,60,62,-1,-1,-1,-1,-1,-1,-1,-1,-1,64,66,68,-1,-1,70,-1,-1,-1,-1,-1,72,74,-1,76,78,80,-1,-1,82,-1,-1,84,86,88,90,92,-1,94,-1,96,98,-1,-1,100,102,104,-1,-1,-1,106,108,-1,110,112,114,-1,-1,116,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,2.9251662E6,1.8062708E6,1.6631816E8,8E0,7.626E3,5.197839E6,1.559733E6,1.925508E6,5.460753E9,5.6530495E6,2.3514E4,1.6E1,3.9157894E0,1.28738E5,5.90352E5,3.0778393E1,5.68E2,-2.1141354E-2,4.9E1,2.6654E4,1.753E3,-7.3528057E-3,5E0,8.638015E4,7.750426E-3,1.5423308E-2,1.084437E6,1.8258025E2,1.639E3,1.465E4,1.3871127E2,3.8248322E2,5.201557E7,2.4210527E0,1.7377206E-3,-5.798278E-3,8.676437E-4,-2.4687366E-3,7.743945E-3,2.699763E-3,3.6939787E-4,3.8654783E-3,1.05083E-3,8.315789E0,7.937523E-3,1.4848485E0,-7.2331065E-3,3.3016268E-3,1.7133154E3,-2.656367E-3,5.547538E-3,5.940789E-5,9.77455E-3,3.828037E-3,3.964269E3,1.2E1,4.1283728E-3,9.639872E0,2.118835E6,1.3422592E8,-2.4253214E-3,-6.3920566E-3,6.743651E7,2.3177303E-3,-1.2249458E-2,1E0,4.75356E6,2.6089566E2,1.9851096E7,1E0,-1.2907474E-2,5.4E1,7.358328E-3,1.0302E4,1.5611285E0,-1.5285532E-2,1.8710053E-3,1E0,5.44E2,2.0131578E0,6.344082E-3,1.1036266E-2,8.388599E-4,1.5154321E0,7.09617E6,-1.8600913E-2,3.9401392E6,1.2083E4,4.8149962E8,8.0095995E-3,4.2474973E-3,6.000379E8,-5.389891E-3,-1.6721066E-3,-2.8784259E-3,1.8439836E-4,-1.0761559E-3,-5.6850757E-3,-6.509351E-3,-1.6580194E-3,3.3160488E-3,1.5043175E-4,3.202467E-4,-5.8013285E-3,-1.6419897E-3,-5.2547306E-3,-3.628755E-3,3.9045555E-3,-1.2242476E-3,-9.7461445E-3,7.90321E-3,1.0181373E-3,4.579927E-3,7.6431077E-4,5.6905127E-3,1.0378918E-2],"split_indices":[2,37,37,5,8,61,37,9,37,40,37,1,3,65,1,9,67,0,0,0,9,38,0,3,37,0,0,9,67,2,38,61,42,7,62,0,0,0,0,0,0,0,0,0,67,47,62,0,0,64,0,0,0,0,0,37,10,0,63,9,7,0,0,7,0,0,15,56,61,54,109,0,2,0,1,62,0,0,23,10,62,0,0,0,62,12,0,54,9,5,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.84E2,4.89E2,4.95E2,4.35E2,5.4E1,3.16E2,1.79E2,4.2E2,1.5E1,3.8E1,1.6E1,3.06E2,1E1,1.29E2,5E1,2.95E2,1.25E2,1E1,5E0,1.1E1,2.7E1,1E1,6E0,3.1E1,2.75E2,5E0,5E0,1.4E1,1.15E2,1.6E1,3.4E1,2.26E2,6.9E1,1.14E2,1.1E1,5E0,5E0,6E0,5E0,1.1E1,1.6E1,5E0,5E0,6E0,2.5E1,4.2E1,2.33E2,5E0,9E0,1.09E2,6E0,1E1,6E0,2.3E1,1.1E1,1.14E2,1.12E2,6E0,6.3E1,3.3E1,8.1E1,5E0,6E0,1.9E1,6E0,6E0,3.6E1,1.9E1,2.14E2,8.7E1,2.2E1,5E0,1.09E2,5E0,1.07E2,5.8E1,5E0,6E0,2.7E1,6.4E1,1.7E1,1.3E1,6E0,7E0,2.9E1,1.3E1,6E0,2.1E1,1.93E2,7.8E1,9E0,5E0,1.7E1,4.3E1,6.6E1,2.8E1,7.9E1,7E0,5.1E1,6E0,2.1E1,4.9E1,1.5E1,1E1,7E0,6E0,2.3E1,6E0,7E0,1.6E1,5E0,1.3E1,1.8E2,4.1E1,3.7E1,6E0,1.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"117","size_leaf_vector":"1"}},{"base_weights":[5.2314424E-3,-4.361332E-2,3.844015E-2,-8.064987E-2,-1.654426E-2,-1.4925355E-2,5.8488093E-2,-7.407775E-2,-1.1199392E-2,1.9164208E-1,-2.8087012E-2,1.7218519E-2,-6.579471E-2,5.1370736E-2,2.0449966E-1,-1.5815479E-1,-6.550819E-2,4.566691E-3,1.1329256E-2,-3.6756057E-2,9.900111E-2,8.005955E-2,-1.6434819E-2,-1.1462498E-1,-7.960444E-3,1.8439616E-1,4.6040654E-2,1.0706258E-2,5.066161E-3,-9.96555E-3,-4.7714133E-3,-5.6087375E-2,-1.5757592E-1,-5.1278784E-3,-7.232539E-2,8.8261295E-4,9.394463E-3,2.929282E-2,1.10091686E-1,-3.2693088E-2,3.0100492E-3,-3.6643352E-2,-1.1508872E-2,-1.0385257E-1,6.583709E-2,3.215567E-3,1.0904468E-2,5.2352108E-2,-4.2876765E-2,-4.6348035E-2,-1.431729E-1,-3.606594E-3,-9.176127E-3,-2.4842922E-2,6.490989E-2,-4.2240232E-2,-1.204583E-1,-5.2836735E-4,2.8408677E-3,1.2938578E-1,1.8695309E-3,2.2877837E-3,-4.802573E-2,1.5310075E-3,-4.05501E-3,-4.3952654E-4,-8.509483E-3,5.0833216E-3,-1.5113712E-3,4.154234E-2,1.1134532E-1,4.7600605E-3,-9.164588E-2,-8.206519E-2,-2.4408597E-2,-1.5297962E-3,-9.64775E-3,-4.5764573E-2,4.544068E-2,1.1292997E-1,-2.0994109E-3,3.8388541E-3,-5.5105887E-2,-2.1636999E-1,-4.958858E-2,2.4534957E-3,7.393982E-3,-9.081922E-2,-1.7272092E-2,4.840421E-2,-1.4625709E-1,1.3540067E-1,-4.9959817E-3,-7.2039864E-3,-2.5483124E-2,-2.5398547E-3,-6.0263867E-3,2.5105919E-3,-2.1292784E-3,1.9379817E-3,-3.5720675E-3,3.2622274E-4,4.740939E-3,7.936965E-3,1.2135454E-3,-4.4776765E-3,-3.9910397E-4,-5.0906762E-3,-1.2772983E-2,-5.6116004E-3,9.2568574E-4,-1.6621893E-3,-6.8645673E-3,3.219395E-5,-3.30684E-3,8.539613E-3,1.9488811E-3,-1.5764602E-2,3.3231578E-3,7.254814E-3,1.6678136E-3,-3.0771457E-3,9.0940914E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":55,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,-1,-1,33,35,37,39,41,43,45,47,-1,-1,-1,-1,49,51,53,55,-1,-1,57,59,61,-1,63,-1,65,67,-1,-1,69,71,73,75,-1,-1,77,79,81,83,-1,-1,85,-1,-1,87,-1,-1,-1,-1,-1,-1,89,91,-1,93,95,97,-1,-1,99,101,103,-1,-1,105,107,109,-1,-1,111,113,115,117,119,-1,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5588651E0,3.8865554E-1,6.128378E-1,1.553477E-1,5.484704E-1,2.5784957E-1,4.2180872E-1,1.0419965E-1,0E0,3.472686E-2,2.405114E-1,2.064524E-1,1.7135537E-1,2.737069E-1,2.2328794E-2,2.1767855E-2,1.1779547E-1,0E0,0E0,2.2687775E-1,1.1200282E-1,4.8612624E-2,8.6723626E-2,3.3423838E-1,2.1205886E-1,7.109892E-2,2.1774483E-1,0E0,0E0,0E0,0E0,1.0701701E-1,2.6343286E-2,1.507387E-1,1.3248003E-1,0E0,0E0,1.88841E-2,2.3067892E-2,7.026043E-2,0E0,8.063665E-2,0E0,9.585813E-2,7.670868E-2,0E0,0E0,2.243098E-1,1.9385178E-1,9.359786E-2,8.5915476E-2,0E0,0E0,1.2706774E-1,1.3206294E-1,1.01032466E-1,2.33971E-1,0E0,0E0,2.8259933E-2,0E0,0E0,5.89699E-2,0E0,0E0,0E0,0E0,0E0,0E0,4.0011925E-1,3.0435044E-1,0E0,8.022164E-2,5.2994877E-2,1.3235657E-1,0E0,0E0,1.8358046E-1,4.286617E-2,8.1274986E-2,0E0,0E0,1.0505277E-1,6.4049125E-2,1.1157215E-1,0E0,0E0,5.439204E-2,2.7212398E-2,2.4843782E-1,4.8809046E-1,9.858519E-2,0E0,0E0,2.1550557E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,31,31,32,32,33,33,34,34,37,37,38,38,39,39,41,41,43,43,44,44,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,59,59,62,62,69,69,70,70,72,72,73,73,74,74,77,77,78,78,79,79,82,82,83,83,84,84,87,87,88,88,89,89,90,90,91,91,94,94],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,-1,-1,34,36,38,40,42,44,46,48,-1,-1,-1,-1,50,52,54,56,-1,-1,58,60,62,-1,64,-1,66,68,-1,-1,70,72,74,76,-1,-1,78,80,82,84,-1,-1,86,-1,-1,88,-1,-1,-1,-1,-1,-1,90,92,-1,94,96,98,-1,-1,100,102,104,-1,-1,106,108,110,-1,-1,112,114,116,118,120,-1,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.539777E5,5.47E2,2.540633E2,3.842105E1,2.04115E5,2.6742857E0,8.2471845E6,1.7857143E1,-1.1199392E-2,1.296E3,1.5900173E3,9.264256E-4,2.0766992E6,3.28E2,1.2E1,1.1634076E4,2.7E1,4.566691E-3,1.1329256E-2,9.237895E0,2.1E1,1.3020051E-5,1.0590052E8,4.2609915E6,7.271717E6,1.6910205E8,2.7555334E7,1.0706258E-2,5.066161E-3,-9.96555E-3,-4.7714133E-3,7.122041E7,3.8996825E5,9.7899484E4,2.283645E9,8.8261295E-4,9.394463E-3,1.1E1,1.2673605E7,4.9236734E5,3.0100492E-3,1.2E1,-1.1508872E-2,1.4E1,2.3786941E4,3.215567E-3,1.0904468E-2,1.6471942E3,1.6368421E0,4.9E2,3.14403E5,-3.606594E-3,-9.176127E-3,9.750085E6,1.5787431E0,3.3317648E2,1.775373E1,-5.2836735E-4,2.8408677E-3,2.6732E4,1.8695309E-3,2.2877837E-3,5.75E2,1.5310075E-3,-4.05501E-3,-4.3952654E-4,-8.509483E-3,5.0833216E-3,-1.5113712E-3,3.3000212E5,2.871605E2,4.7600605E-3,1.3655363E2,5.853E3,4.066655E2,-1.5297962E-3,-9.64775E-3,6.83E2,3.7439062E2,1.862E3,-2.0994109E-3,3.8388541E-3,1.0041E5,1.4601138E4,1.7302156E5,2.4534957E-3,7.393982E-3,3.8E1,6E0,1.1160929E0,1.5714285E0,1.3948464E7,-4.9959817E-3,-7.2039864E-3,8.93E2,-2.5398547E-3,-6.0263867E-3,2.5105919E-3,-2.1292784E-3,1.9379817E-3,-3.5720675E-3,3.2622274E-4,4.740939E-3,7.936965E-3,1.2135454E-3,-4.4776765E-3,-3.9910397E-4,-5.0906762E-3,-1.2772983E-2,-5.6116004E-3,9.2568574E-4,-1.6621893E-3,-6.8645673E-3,3.219395E-5,-3.30684E-3,8.539613E-3,1.9488811E-3,-1.5764602E-2,3.3231578E-3,7.254814E-3,1.6678136E-3,-3.0771457E-3,9.0940914E-4],"split_indices":[37,2,61,55,5,65,37,4,0,2,64,47,37,38,8,37,8,0,0,65,3,47,7,54,54,40,54,0,0,0,0,7,54,42,40,0,0,3,54,37,0,3,0,8,56,0,0,61,62,1,9,0,0,9,47,42,65,0,0,1,0,0,38,0,0,0,0,0,0,42,67,0,65,9,42,0,0,2,4,2,0,0,1,42,37,0,0,0,8,63,62,9,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.59E2,3.88E2,5.71E2,1.63E2,2.25E2,1.56E2,4.15E2,1.58E2,5E0,1.1E1,2.14E2,9.6E1,6E1,3.97E2,1.8E1,1.3E1,1.45E2,5E0,6E0,2.01E2,1.3E1,3.3E1,6.3E1,3.2E1,2.8E1,1.4E1,3.83E2,1.3E1,5E0,5E0,8E0,1.33E2,1.2E1,1.07E2,9.4E1,8E0,5E0,1.3E1,2E1,5.3E1,1E1,2.1E1,1.1E1,1.2E1,1.6E1,5E0,9E0,3.58E2,2.5E1,1.21E2,1.2E1,5E0,7E0,8.4E1,2.3E1,5.9E1,3.5E1,6E0,7E0,1.5E1,5E0,8E0,4.5E1,9E0,1.2E1,6E0,6E0,1.1E1,5E0,3.04E2,5.4E1,6E0,1.9E1,4.5E1,7.6E1,5E0,7E0,6.5E1,1.9E1,1.6E1,7E0,5E0,5.4E1,1.4E1,2.1E1,5E0,1E1,1.8E1,2.7E1,2.94E2,1E1,4.9E1,5E0,9E0,1E1,3E1,1.5E1,1.6E1,6E1,1.7E1,4.8E1,1.2E1,7E0,9E0,7E0,2.8E1,2.6E1,6E0,8E0,1E1,1.1E1,1E1,8E0,2.1E1,6E0,1.2E1,2.82E2,5E0,5E0,4E1,9E0,5E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[2.1160794E-3,-3.7910372E-2,4.295013E-2,-4.0345054E-2,6.5129865E-3,5.4701682E-2,-8.130334E-2,-3.7449166E-2,-1.25781195E-2,2.6814848E-2,1.1909535E-1,-1.0612859E-2,-1.22800924E-1,-9.7363025E-2,-3.0299574E-2,9.865255E-3,2.1917684E-2,1.2954615E-1,-2.9839843E-3,-3.0736928E-3,1.6218225E-3,-1.8201905E-1,-4.5026094E-2,-7.47106E-2,-1.6664523E-1,-1.1220783E-2,-5.9511166E-2,5.256058E-2,-8.866618E-3,1.1700152E-1,1.537986E-2,-5.7861637E-3,-1.0658308E-2,1.8077924E-4,-4.767604E-3,-9.524291E-2,-2.3160767E-2,-9.61705E-3,-4.2445865E-3,-2.145832E-3,-1.17857315E-1,-8.0179706E-2,-2.3618074E-2,6.342911E-2,-9.548463E-2,-1.8913884E-1,4.8703547E-3,1.3760278E-1,5.4116607E-2,-1.2808497E-1,-6.0553435E-2,1.2568501E-3,-2.688937E-3,1.573177E-2,-6.098163E-2,-6.568224E-2,-9.874919E-3,-9.470889E-2,-2.4170598E-2,4.294987E-2,-6.6296786E-2,2.1586584E-2,1.0440274E-1,2.7318231E-3,-1.0854278E-2,4.9220613E-4,-1.658141E-2,1.3192348E-1,-1.0804418E-2,1.22419216E-1,2.1345262E-1,-3.5248857E-3,8.7058075E-2,-2.4880038E-3,-7.5861006E-3,-1.025876E-3,-5.611609E-3,-5.4470036E-4,2.6658513E-3,1.2870678E-3,-5.200734E-3,-6.776596E-3,-5.054681E-4,3.7943615E-4,-4.882669E-3,-3.176033E-3,3.2295333E-4,3.4423862E-3,-3.4126756E-3,2.4410712E-3,-3.934048E-3,3.1291924E-3,-1.272005E-3,5.398569E-3,-1.1536751E-3,1.0530199E-3,8.329147E-3,3.99441E-3,-1.404114E-3,5.988594E-3,9.296954E-4,4.5039584E-3,1.3391168E-2,8.031456E-3,2.4402658E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":56,"left_children":[1,3,5,7,-1,9,11,13,-1,15,17,19,21,23,25,-1,27,29,-1,-1,-1,31,33,35,37,39,41,43,45,47,-1,-1,-1,-1,-1,49,51,-1,-1,53,55,57,59,61,63,65,67,69,71,73,75,-1,-1,77,79,81,-1,83,85,87,89,91,93,-1,-1,-1,-1,95,97,99,101,-1,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6278982E0,2.239908E-1,7.243116E-1,3.2194012E-1,0E0,8.066832E-1,1.24836594E-1,2.0780283E-1,0E0,2.828679E-1,2.6900804E-1,4.540911E-2,1.1615777E-1,6.915915E-2,2.4511117E-1,0E0,2.91952E-1,2.8797793E-1,0E0,0E0,0E0,1.2354374E-2,3.7113007E-2,4.1190043E-2,1.7679542E-2,2.5909638E-1,1.2718958E-1,2.5545666E-1,3.8476235E-1,1.5190077E-1,0E0,0E0,0E0,0E0,0E0,2.5282979E-2,2.429641E-2,0E0,0E0,2.6181644E-1,9.00743E-2,8.722514E-2,1.8751466E-1,2.4634016E-1,2.4885145E-1,3.7341467E-1,2.90197E-1,7.982302E-2,1.437397E-1,2.8148353E-2,3.447067E-2,0E0,0E0,2.1959287E-1,2.6608264E-1,6.44566E-2,0E0,9.504783E-2,3.3559307E-2,1.0008388E-1,9.240323E-2,1.658706E-1,1.1563265E-1,0E0,0E0,0E0,0E0,7.9090774E-2,2.476852E-1,5.1175952E-2,9.4438314E-2,0E0,6.799346E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,35,35,36,36,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,53,53,54,54,55,55,57,57,58,58,59,59,60,60,61,61,62,62,67,67,68,68,69,69,70,70,72,72],"right_children":[2,4,6,8,-1,10,12,14,-1,16,18,20,22,24,26,-1,28,30,-1,-1,-1,32,34,36,38,40,42,44,46,48,-1,-1,-1,-1,-1,50,52,-1,-1,54,56,58,60,62,64,66,68,70,72,74,76,-1,-1,78,80,82,-1,84,86,88,90,92,94,-1,-1,-1,-1,96,98,100,102,-1,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.158E3,1.75E2,3.5056704E7,7.704695E1,6.5129865E-3,1.0879832E3,3.53562E5,3.175E1,-1.25781195E-2,1.0280637E0,4.1325716E7,1.59E2,1.3655363E2,1.8416666E1,1E0,9.865255E-3,1.9355932E1,1.0534078E4,-2.9839843E-3,-3.0736928E-3,1.6218225E-3,1.5154321E0,2.342E4,1.3151775E2,7.336111E4,1.2E1,1E0,1.2784314E1,5.1942E4,3.7E1,1.537986E-2,-5.7861637E-3,-1.0658308E-2,1.8077924E-4,-4.767604E-3,1.4806053E4,1.3743396E-2,-9.61705E-3,-4.2445865E-3,4.911E3,3.429012E7,6.459696E6,1.3402174E1,4.4149057E2,3.3924444E5,1.623E3,6E0,1.8E1,1.9125667E3,6.1E1,6.0921145E4,1.2568501E-3,-2.688937E-3,5.95E2,1E0,1.08712E5,-9.874919E-3,1.9047619E0,3.426009E0,1.45752E8,1.5487804E0,9.970589E0,2.3821254E7,2.7318231E-3,-1.0854278E-2,4.9220613E-4,-1.658141E-2,1.65E2,2.617E3,2.1665942E6,1.6896813E0,-3.5248857E-3,8.598131E1,-2.4880038E-3,-7.5861006E-3,-1.025876E-3,-5.611609E-3,-5.4470036E-4,2.6658513E-3,1.2870678E-3,-5.200734E-3,-6.776596E-3,-5.054681E-4,3.7943615E-4,-4.882669E-3,-3.176033E-3,3.2295333E-4,3.4423862E-3,-3.4126756E-3,2.4410712E-3,-3.934048E-3,3.1291924E-3,-1.272005E-3,5.398569E-3,-1.1536751E-3,1.0530199E-3,8.329147E-3,3.99441E-3,-1.404114E-3,5.988594E-3,9.296954E-4,4.5039584E-3,1.3391168E-2,8.031456E-3,2.4402658E-3],"split_indices":[2,6,54,55,0,61,1,64,0,63,60,0,65,61,23,0,67,4,0,0,0,62,9,42,54,65,24,62,1,8,0,0,0,0,0,37,47,0,0,38,7,9,67,4,37,38,3,8,4,38,54,0,0,2,62,7,0,63,63,7,62,67,9,0,0,0,0,0,38,52,62,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.94E2,5.02E2,4.92E2,4.96E2,6E0,4.5E2,4.2E1,4.91E2,5E0,3.15E2,1.35E2,1.6E1,2.6E1,5.1E1,4.4E2,7E0,3.08E2,1.28E2,7E0,7E0,9E0,1.4E1,1.2E1,4E1,1.1E1,2.67E2,1.73E2,1.54E2,1.54E2,1.22E2,6E0,8E0,6E0,7E0,5E0,2.8E1,1.2E1,6E0,5E0,2.47E2,2E1,1.09E2,6.4E1,1.44E2,1E1,1E1,1.44E2,9.1E1,3.1E1,1.3E1,1.5E1,5E0,7E0,1.9E2,5.7E1,1.4E1,6E0,8.6E1,2.3E1,2.5E1,3.9E1,7.2E1,7.2E1,5E0,5E0,5E0,5E0,1.5E1,1.29E2,7.8E1,1.3E1,6E0,2.5E1,5E0,8E0,1E1,5E0,1.15E2,7.5E1,2.1E1,3.6E1,5E0,9E0,8E0,7.8E1,9E0,1.4E1,2E1,5E0,5E0,3.4E1,3.7E1,3.5E1,6.6E1,6E0,5E0,1E1,2.1E1,1.08E2,7.3E1,5E0,6E0,7E0,6E0,1.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[7.603989E-4,-2.5416503E-2,5.6728248E-2,-1.4922482E-2,-1.1215378E-1,7.671902E-2,-5.7061747E-2,-2.2336006E-2,8.590384E-2,-2.3315564E-1,-6.578477E-2,2.6891407E-1,6.463884E-2,-1.7609178E-1,1.2290663E-2,-3.0223288E-2,5.7636507E-2,6.0922995E-2,8.268923E-3,-2.3208894E-2,-2.40781E-2,1.1804893E-3,-1.4396009E-1,7.1460265E-3,1.4428564E-2,7.141904E-2,-6.7349565E-3,-1.3066004E-2,-1.0866408E-1,-9.342299E-2,7.84103E-2,-9.109765E-2,-2.1916382E-2,8.851665E-2,-4.150137E-3,4.079107E-2,6.714836E-3,-3.8105669E-3,2.2919036E-3,-8.621185E-2,5.1713925E-2,-1.3294517E-3,-1.8975432E-1,1.4623298E-1,5.3697146E-2,-7.7289143E-3,-2.018557E-3,1.9966575E-4,-8.902485E-3,-1.9440752E-3,1.3716051E-1,-7.5794947E-3,-6.9173805E-2,-1.25137875E-2,-8.063074E-2,1.297983E-1,2.2601489E-2,1.577699E-2,3.3791952E-3,-1.8182254E-3,-5.5154352E-3,6.6328133E-3,-1.3914016E-3,-6.3134E-3,-1.1461268E-2,1.7599156E-1,4.9730882E-2,4.319546E-2,1.8563344E-1,1.35094E-3,9.328565E-3,-1.0522235E-1,-1.5247153E-2,-2.3780232E-2,3.8121644E-2,-1.7563938E-1,-4.789683E-2,1.7368874E-1,-3.6542036E-4,-1.2313552E-3,3.7676722E-3,2.413797E-3,-2.0152908E-3,1.4792606E-3,-1.599158E-3,2.3128423E-1,8.048175E-2,1.3107814E-4,4.462042E-3,2.1628456E-2,8.135152E-2,4.454051E-3,1.0429069E-2,-7.147061E-3,-1.7192416E-3,2.060268E-3,-2.0407895E-3,-1.2914138E-3,6.132486E-3,2.5222132E-3,-2.432898E-3,-1.6925575E-3,-1.3587872E-2,2.657312E-3,-3.6516697E-3,4.7307345E-3,1.0049583E-2,1.1798828E-2,6.0965354E-3,7.331857E-3,1.1965939E-4,1.4229483E-3,-4.1873683E-3,5.2817618E-3,-1.5209294E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":57,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,33,35,-1,37,-1,39,41,-1,-1,43,-1,-1,45,47,49,51,53,55,-1,57,-1,-1,-1,59,61,-1,63,65,67,-1,-1,-1,-1,-1,69,-1,71,73,75,77,79,81,-1,-1,-1,-1,83,-1,-1,85,87,89,91,-1,-1,93,95,97,99,101,103,105,-1,-1,-1,-1,-1,-1,-1,107,109,-1,-1,111,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3976487E0,5.899712E-1,6.959019E-1,4.362037E-1,3.7741357E-1,5.839516E-1,3.8189515E-1,3.43617E-1,8.138004E-2,1.1279117E0,2.723915E-1,2.4583578E-2,3.5885966E-1,9.033972E-2,2.1701775E-1,2.4780819E-1,2.3046836E-1,4.976642E-2,0E0,5.538743E-2,0E0,1.3252917E-1,1.21293396E-1,0E0,0E0,3.076986E-1,0E0,0E0,3.803408E-2,1.162716E-1,1.4004228E-1,8.216336E-2,2.3969235E-1,1.08630896E-1,0E0,2.1289758E-2,0E0,0E0,0E0,1.223468E-2,9.2256606E-2,0E0,1.8649697E-2,1.2047732E-1,2.6077884E-1,0E0,0E0,0E0,0E0,0E0,8.388357E-2,0E0,8.97626E-2,2.1597798E-1,1.781877E-1,1.5243202E-1,5.110734E-2,3.8743135E-2,0E0,0E0,0E0,0E0,1.5404512E-2,0E0,0E0,1.609397E-1,2.586526E-2,1.4718756E-1,2.7535021E-2,0E0,0E0,8.746028E-2,3.623499E-2,1.989509E-1,1.0430489E-1,2.3397669E-1,1.535273E-1,3.6385894E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.4844656E-2,8.415465E-2,0E0,0E0,1.2108116E-1,1.7904836E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,19,19,21,21,22,22,25,25,28,28,29,29,30,30,31,31,32,32,33,33,35,35,39,39,40,40,42,42,43,43,44,44,50,50,52,52,53,53,54,54,55,55,56,56,57,57,62,62,65,65,66,66,67,67,68,68,71,71,72,72,73,73,74,74,75,75,76,76,77,77,85,85,86,86,89,89,90,90],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,34,36,-1,38,-1,40,42,-1,-1,44,-1,-1,46,48,50,52,54,56,-1,58,-1,-1,-1,60,62,-1,64,66,68,-1,-1,-1,-1,-1,70,-1,72,74,76,78,80,82,-1,-1,-1,-1,84,-1,-1,86,88,90,92,-1,-1,94,96,98,100,102,104,106,-1,-1,-1,-1,-1,-1,-1,108,110,-1,-1,112,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.194E3,4.163636E0,2E0,4.5866325E6,1E0,2.2968E4,1.7743998E5,1.5074378E5,4.087912E0,3.95E2,1.6670103E1,5.4444447E0,4.1325716E7,3.77E2,1.4598765E0,7.7E1,1.0657745E7,3.0714285E0,8.268923E-3,1.073125E1,-2.40781E-2,5.2405543E0,3.14403E5,7.1460265E-3,1.4428564E-2,2.7662E4,-6.7349565E-3,-1.3066004E-2,1.342376E0,1.3728223E0,2.19724E5,2.8636363E0,3.38E2,1.3745962E0,-4.150137E-3,1.4E1,6.714836E-3,-3.8105669E-3,2.2919036E-3,1E0,1.5123151E6,-1.3294517E-3,1.609E3,5.501002E9,4.907764E3,-7.7289143E-3,-2.018557E-3,1.9966575E-4,-8.902485E-3,-1.9440752E-3,1.625E0,-7.5794947E-3,1.1634076E4,3.7943903E2,7.54E2,8E0,5.9754E4,2.0625E0,3.3791952E-3,-1.8182254E-3,-5.5154352E-3,6.6328133E-3,1.15E2,-6.3134E-3,-1.1461268E-2,2.8366232E0,9.132011E6,6.941074E6,4.362075E8,1.35094E-3,9.328565E-3,1.0479E4,5.04E2,3.4E1,8.009E3,5.81E2,1.6386554E0,9.7199225E-1,-3.6542036E-4,-1.2313552E-3,3.7676722E-3,2.413797E-3,-2.0152908E-3,1.4792606E-3,-1.599158E-3,4.5709968E0,1.4875445E0,1.3107814E-4,4.462042E-3,6.426937E6,5.6E1,4.454051E-3,1.0429069E-2,-7.147061E-3,-1.7192416E-3,2.060268E-3,-2.0407895E-3,-1.2914138E-3,6.132486E-3,2.5222132E-3,-2.432898E-3,-1.6925575E-3,-1.3587872E-2,2.657312E-3,-3.6516697E-3,4.7307345E-3,1.0049583E-2,1.1798828E-2,6.0965354E-3,7.331857E-3,1.1965939E-4,1.4229483E-3,-4.1873683E-3,5.2817618E-3,-1.5209294E-4],"split_indices":[2,62,6,37,109,12,42,42,65,2,67,63,60,10,63,38,54,62,0,63,0,62,9,0,0,9,0,0,63,62,1,63,0,47,0,3,0,0,0,23,54,0,2,5,4,0,0,0,0,0,62,0,37,61,2,8,1,65,0,0,0,0,0,0,0,47,1,54,7,0,0,9,7,3,38,2,63,47,0,0,0,0,0,0,0,63,63,0,0,54,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.52E2,6.49E2,3.03E2,5.8E2,6.9E1,2.58E2,4.5E1,5.41E2,3.9E1,1.8E1,5.1E1,1.4E1,2.44E2,1.6E1,2.9E1,4.93E2,4.8E1,3.2E1,7E0,1.1E1,7E0,2.8E1,2.3E1,5E0,9E0,2.37E2,7E0,5E0,1.1E1,1.1E1,1.8E1,5.8E1,4.35E2,4E1,8E0,2.7E1,5E0,6E0,5E0,1E1,1.8E1,7E0,1.6E1,4.4E1,1.93E2,5E0,6E0,6E0,5E0,6E0,1.2E1,1.2E1,4.6E1,3.76E2,5.9E1,2.4E1,1.6E1,1.6E1,1.1E1,5E0,5E0,6E0,1.2E1,1E1,6E0,3.3E1,1.1E1,1.8E2,1.3E1,5E0,7E0,2.7E1,1.9E1,3.08E2,6.8E1,1.4E1,4.5E1,1.8E1,6E0,9E0,7E0,1E1,6E0,6E0,6E0,2E1,1.3E1,6E0,5E0,1.16E2,6.4E1,5E0,8E0,1.5E1,1.2E1,6E0,1.3E1,3.01E2,7E0,5.8E1,1E1,7E0,7E0,1E1,3.5E1,8E0,1E1,1.5E1,5E0,6E0,7E0,1.08E2,8E0,4.6E1,1.8E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[1.3775129E-3,-5.1161118E-2,3.0373873E-2,-4.3185703E-2,-1.4581862E-1,-9.629186E-2,4.1005824E-2,-3.5863943E-2,-1.4938846E-1,-1.9490673E-1,-6.1161496E-2,-1.1885675E-2,-6.2749475E-2,1.3195882E-2,7.8529194E-2,-6.8391874E-2,-1.4521544E-2,-4.108689E-2,-1.83391E-2,-3.8469655E-3,-1.10559175E-2,-4.093547E-3,-1.1091906E-3,1.275296E-3,-8.213559E-2,4.6203323E-2,-1.102112E-2,1.1280125E-2,7.038734E-2,3.1237234E-3,-8.032476E-2,-7.067618E-3,-7.20865E-3,-4.9199867E-3,2.240539E-3,-3.378192E-2,-1.1853154E-1,2.5334457E-2,1.0990617E-1,-1.7240964E-2,8.472264E-3,1.07899375E-1,4.1653577E-2,-1.4182343E-1,-5.9095636E-2,3.4121405E-2,-2.422805E-2,-4.6882858E-3,1.380768E-3,-2.617638E-3,-6.3421596E-3,6.279573E-2,1.6792068E-3,1.4323859E-1,7.2311086E-4,-2.6232503E-2,5.422786E-2,8.016164E-2,1.8790613E-1,6.16959E-2,-5.688344E-2,-1.0252812E-1,-1.1110284E-2,-2.925519E-2,-1.0251766E-1,5.496158E-2,-1.9559115E-2,2.9009825E-2,-4.114104E-2,1.3777131E-1,3.416582E-2,-4.640007E-3,1.9488458E-2,1.6721281E-1,1.8305205E-3,-1.0742621E-2,-8.786651E-2,4.7715614E-3,9.2015015E-3,1.0539665E-1,-3.9121754E-2,2.4315323E-1,9.548681E-2,4.0395483E-2,1.0168936E-2,1.2122303E-2,-8.713088E-3,-7.974242E-3,-1.6799739E-3,-3.6884565E-3,6.058332E-4,-6.0652993E-3,-1.1405846E-3,6.2157833E-3,1.7831916E-3,-3.7378413E-3,1.3946575E-3,-2.2023986E-3,3.2303769E-3,-6.430007E-3,-1.5789716E-3,3.1988139E-3,8.520782E-3,3.0438397E-3,-1.7820995E-3,-1.413468E-3,2.6327004E-3,4.4092084E-3,9.624052E-3,-1.6331541E-3,1.6125616E-3,-6.3698394E-3,4.6641717E-4,-1.3085977E-3,2.3805858E-3,6.8280315E-3,2.9218954E-3,-6.4048427E-3,3.6893676E-3,6.752418E-3,1.2871797E-2,6.300776E-3,1.8218651E-3,2.322517E-3,-5.4597165E-3,2.7480281E-3,-1.9665963E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":58,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,-1,-1,-1,-1,-1,35,37,39,-1,41,-1,43,45,-1,-1,-1,47,49,51,53,55,-1,57,59,61,63,65,67,-1,-1,-1,-1,69,71,73,-1,75,77,79,81,83,85,87,-1,89,91,93,95,97,99,101,103,-1,105,107,-1,109,111,-1,113,115,117,119,121,123,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.4564288E0,2.5037992E-1,8.3351827E-1,2.3958355E-1,9.612626E-2,2.4131945E-1,5.927283E-1,2.0449716E-1,5.162921E-1,5.796218E-2,8.981746E-3,0E0,7.397877E-2,2.6269314E-1,3.071648E-1,1.9488508E-1,1.8786696E-1,9.2192054E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.5831626E-2,1.8200302E-1,2.31957E-1,0E0,2.455697E-1,0E0,1.3261962E-1,1.22399986E-1,0E0,0E0,0E0,7.1386315E-2,1.3715088E-2,9.382705E-2,1.0466266E-1,1.20306276E-1,0E0,2.068578E-1,2.661726E-1,7.902777E-2,1.0325086E-1,5.8129277E-2,1.1104372E-1,0E0,0E0,0E0,0E0,8.330126E-2,1.21465385E-1,5.560875E-2,0E0,1.5658218E-1,4.5016006E-2,2.328923E-1,1.0442954E-1,3.634487E-1,2.1003364E-1,9.034048E-2,0E0,1.0732264E-1,6.931803E-2,4.317013E-2,4.7819868E-2,9.647815E-2,5.9722856E-2,2.206844E-2,7.324153E-2,0E0,1.0779232E-1,3.8333714E-2,0E0,1.4920218E-1,1.610597E-1,0E0,2.0345885E-2,1.0234833E-1,1.7526342E-1,9.7978115E-3,1.9595906E-2,1.5704285E-1,0E0,4.358903E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,14,14,15,15,16,16,17,17,24,24,25,25,26,26,28,28,30,30,31,31,35,35,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,51,51,52,52,53,53,55,55,56,56,57,57,58,58,59,59,60,60,61,61,63,63,64,64,65,65,66,66,67,67,68,68,69,69,70,70,72,72,73,73,75,75,76,76,78,78,79,79,80,80,81,81,82,82,83,83,85,85],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,-1,-1,-1,-1,-1,36,38,40,-1,42,-1,44,46,-1,-1,-1,48,50,52,54,56,-1,58,60,62,64,66,68,-1,-1,-1,-1,70,72,74,-1,76,78,80,82,84,86,88,-1,90,92,94,96,98,100,102,104,-1,106,108,-1,110,112,-1,114,116,118,120,122,124,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.09E2,2.7E1,4.8465605E3,4.909091E0,2.871287E0,1.0479E4,5.8222644E2,3.06E2,1.6507974E8,1.7754011E0,2.01E2,-1.1885675E-2,1.1544592E1,3.962617E0,2.82E2,1.957E3,6.83E2,1.317484E1,-1.83391E-2,-3.8469655E-3,-1.10559175E-2,-4.093547E-3,-1.1091906E-3,1.275296E-3,1E0,8.788122E4,4.1E1,1.1280125E-2,2.0980392E0,3.1237234E-3,7.692308E-2,2.0487332E-3,-7.20865E-3,-4.9199867E-3,2.240539E-3,7E0,1.5889913E-2,2.3786941E4,1.2856612E0,2.8530578E6,8.472264E-3,1.5146261E3,3.1790426E8,3.66025E5,3.1E1,2.7976523E2,1.7754011E0,-4.6882858E-3,1.380768E-3,-2.617638E-3,-6.3421596E-3,7E0,1.5626E5,8E0,7.2311086E-4,2.2968E4,9.912121E0,1.853776E7,8.847283E8,4.907764E3,1.4993486E2,3.9E1,-1.1110284E-2,4.54E2,1.2463E4,5.719388E1,1.6736842E0,1.8224286E2,1.4110284E2,3.3527696E-1,7.579E3,-4.640007E-3,3.056087E2,7.9403794E-1,1.8305205E-3,8.172E3,2.067E3,4.7715614E-3,4.577732E2,9.527559E0,1.4320667E3,4.868063E1,1.752E3,5E0,1.0168936E-2,1.0046621E7,-8.713088E-3,-7.974242E-3,-1.6799739E-3,-3.6884565E-3,6.058332E-4,-6.0652993E-3,-1.1405846E-3,6.2157833E-3,1.7831916E-3,-3.7378413E-3,1.3946575E-3,-2.2023986E-3,3.2303769E-3,-6.430007E-3,-1.5789716E-3,3.1988139E-3,8.520782E-3,3.0438397E-3,-1.7820995E-3,-1.413468E-3,2.6327004E-3,4.4092084E-3,9.624052E-3,-1.6331541E-3,1.6125616E-3,-6.3698394E-3,4.6641717E-4,-1.3085977E-3,2.3805858E-3,6.8280315E-3,2.9218954E-3,-6.4048427E-3,3.6893676E-3,6.752418E-3,1.2871797E-2,6.300776E-3,1.8218651E-3,2.322517E-3,-5.4597165E-3,2.7480281E-3,-1.9665963E-3],"split_indices":[2,8,37,62,62,9,61,2,40,62,0,0,65,65,38,9,2,63,0,0,0,0,0,0,8,42,3,0,62,0,67,47,0,0,0,6,47,56,47,37,0,61,41,9,10,4,62,0,0,0,0,3,12,8,0,38,67,56,7,4,67,2,0,1,1,56,62,4,4,65,38,0,4,51,0,38,2,0,4,65,4,67,0,6,0,56,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.54E2,3.39E2,6.15E2,3.14E2,2.5E1,4.7E1,5.68E2,2.95E2,1.9E1,1.5E1,1E1,7E0,4E1,3.27E2,2.41E2,1.16E2,1.79E2,1.4E1,5E0,5E0,1E1,5E0,5E0,7E0,3.3E1,1.38E2,1.89E2,1E1,2.31E2,9E0,1.07E2,1.71E2,8E0,8E0,6E0,1.5E1,1.8E1,1.05E2,3.3E1,1.84E2,5E0,9.9E1,1.32E2,2.6E1,8.1E1,5E1,1.21E2,7E0,8E0,5E0,1.3E1,4E1,6.5E1,2.4E1,9E0,1.64E2,2E1,7.5E1,2.4E1,1.1E2,2.2E1,2E1,6E0,4.9E1,3.2E1,3.6E1,1.4E1,2.9E1,9.2E1,1E1,3E1,9E0,5.6E1,1.9E1,5E0,1.32E2,3.2E1,9E0,1.1E1,6.2E1,1.3E1,1.4E1,1E1,9.8E1,1.2E1,1.5E1,7E0,9E0,1.1E1,2.2E1,2.7E1,2.3E1,9E0,5E0,3.1E1,6E0,8E0,1E1,1.9E1,5E0,8.7E1,5E0,5E0,2.1E1,9E0,2.4E1,3.2E1,8E0,1.1E1,8.6E1,4.6E1,2.1E1,1.1E1,6E0,5E0,3E1,3.2E1,7E0,6E0,5E0,9E0,5E0,5E0,9.3E1,5E0,8E0,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[-7.5367643E-3,-6.6168234E-2,1.4589538E-2,-5.2612502E-2,-2.384065E-1,-7.307188E-4,8.546592E-2,-4.0540263E-2,-1.2626721E-1,-2.4022585E-2,-8.1713304E-2,3.1692404E-3,-1.2073751E-2,1.3616965E-2,7.3133804E-2,-5.535619E-2,6.961136E-2,-1.0875617E-2,-1.0126682E-1,-1.8504511E-4,-8.569146E-3,6.0152747E-3,-1.2950833E-2,9.456041E-3,6.1239306E-2,-6.2284093E-2,4.014513E-2,1.2777074E-1,-2.6599616E-2,-4.3595433E-2,-1.2665342E-1,-2.882473E-2,2.4449788E-2,5.1538706E-2,8.618925E-3,-6.632587E-2,3.2177956E-3,-2.0517482E-4,3.4392704E-3,9.066438E-3,7.31185E-2,1.4617089E-3,-3.7244207E-3,1.9158404E-4,-3.9000544E-3,-7.5172433E-3,-7.978125E-2,-1.3565191E-2,-9.815708E-2,-7.910215E-2,3.0872798E-2,6.930095E-2,-2.6193608E-2,2.6185547E-3,-7.088335E-2,4.033334E-4,5.8165337E-3,-5.1206416E-3,-1.6659963E-3,4.051938E-2,-3.219634E-2,-1.2225325E-1,2.167565E-3,-3.2088637E-2,-9.602775E-3,8.578747E-2,1.9860853E-2,5.3200662E-2,1.3943347E-1,-3.737495E-3,2.1286601E-2,-3.934285E-3,-1.4614452E-3,-4.7174908E-5,4.9437764E-3,-1.1364406E-2,-9.1052323E-4,-8.276902E-3,-3.1869868E-3,1.1702229E-3,-7.002164E-3,2.39882E-3,6.0173925E-3,2.5719085E-3,1.2365085E-4,4.3191533E-3,1.1084928E-3,8.103793E-3,3.2108799E-3,-1.109854E-3,3.2702007E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":59,"left_children":[1,3,5,7,9,11,13,15,17,-1,19,21,-1,-1,23,25,27,-1,29,-1,-1,31,-1,-1,33,35,37,39,41,43,45,47,49,51,-1,53,-1,-1,-1,-1,55,-1,-1,-1,-1,-1,57,59,61,63,65,67,69,-1,71,-1,-1,-1,-1,73,75,77,-1,79,-1,81,83,85,87,-1,89,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.3164251E0,6.365354E-1,8.003298E-1,2.25178E-1,8.330711E-1,6.1509186E-1,3.1457388E-1,3.6942112E-1,7.3577166E-2,0E0,1.1130942E-1,4.8291224E-1,0E0,0E0,1.8179077E-1,1.3350195E-1,1.5447907E-1,0E0,3.9864957E-2,0E0,0E0,3.8220632E-1,0E0,0E0,1.3054022E-1,1.0302627E-1,2.1527981E-2,4.8546076E-2,3.6830004E-2,2.158654E-2,2.1878153E-2,2.1661228E-1,2.611719E-1,1.5231156E-1,0E0,1.04706764E-1,0E0,0E0,0E0,0E0,3.5931587E-2,0E0,0E0,0E0,0E0,0E0,1.0795794E-2,1.7260638E-1,1.3487336E-1,1.3161197E-1,2.2022745E-1,9.3076766E-2,5.5913053E-2,0E0,9.26463E-2,0E0,0E0,0E0,0E0,1.2078194E-1,3.348668E-1,8.396393E-2,0E0,1.2637894E-1,0E0,8.4578544E-2,1.8762597E-1,8.428718E-2,2.7490735E-2,0E0,2.837164E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,10,10,11,11,14,14,15,15,16,16,18,18,21,21,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,35,35,40,40,46,46,47,47,48,48,49,49,50,50,51,51,52,52,54,54,59,59,60,60,61,61,63,63,65,65,66,66,67,67,68,68,70,70],"right_children":[2,4,6,8,10,12,14,16,18,-1,20,22,-1,-1,24,26,28,-1,30,-1,-1,32,-1,-1,34,36,38,40,42,44,46,48,50,52,-1,54,-1,-1,-1,-1,56,-1,-1,-1,-1,-1,58,60,62,64,66,68,70,-1,72,-1,-1,-1,-1,74,76,78,-1,80,-1,82,84,86,88,-1,90,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.972052E5,6.72043E0,5.306E3,3.63E2,9.264256E-4,1.853776E7,1.273782E4,1.839E3,6.15E2,-2.4022585E-2,6.97E3,4.9E1,-1.2073751E-2,1.3616965E-2,7.0285716E0,1.3631483E8,4.88451E5,-1.0875617E-2,2.8689062E9,-1.8504511E-4,-8.569146E-3,2.6801266E2,-1.2950833E-2,9.456041E-3,6.7473444E2,1.6365927E5,2.6138635E2,1.9184E4,2.670362E7,1.138871E0,2.9541414E1,4.7272725E0,1.727279E-4,1.3358269E2,8.618925E-3,1.0204082E0,3.2177956E-3,-2.0517482E-4,3.4392704E-3,9.066438E-3,1.915148E7,1.4617089E-3,-3.7244207E-3,1.9158404E-4,-3.9000544E-3,-7.5172433E-3,4.0663E4,7.059674E2,6.459696E6,6.485856E7,3.3382E4,1.9705E4,2.506329E0,2.6185547E-3,1.295313E6,4.033334E-4,5.8165337E-3,-5.1206416E-3,-1.6659963E-3,3.3820656E7,7.5456814E-4,1.8898131E2,2.167565E-3,8E0,-9.602775E-3,3.8737908E-1,1.9052632E0,5.6516E4,9.2732E4,-3.737495E-3,1E1,-3.934285E-3,-1.4614452E-3,-4.7174908E-5,4.9437764E-3,-1.1364406E-2,-9.1052323E-4,-8.276902E-3,-3.1869868E-3,1.1702229E-3,-7.002164E-3,2.39882E-3,6.0173925E-3,2.5719085E-3,1.2365085E-4,4.3191533E-3,1.1084928E-3,8.103793E-3,3.2108799E-3,-1.109854E-3,3.2702007E-3],"split_indices":[37,62,2,0,47,56,42,2,2,0,12,3,0,0,65,7,9,0,40,0,0,64,0,0,65,42,4,1,5,47,65,63,47,67,0,62,0,0,0,0,12,0,0,0,0,0,9,42,9,7,9,2,62,0,9,0,0,0,0,7,47,64,0,8,0,47,62,9,38,0,3,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[1.013E3,2.77E2,7.36E2,2.58E2,1.9E1,6.06E2,1.3E2,2.23E2,3.5E1,6E0,1.3E1,5.98E2,8E0,6E0,1.24E2,1.97E2,2.6E1,5E0,3E1,8E0,5E0,5.93E2,5E0,9E0,1.15E2,1.84E2,1.3E1,1.6E1,1E1,1E1,2E1,2.05E2,3.88E2,1.08E2,7E0,1.79E2,5E0,6E0,7E0,6E0,1E1,5E0,5E0,5E0,5E0,1E1,1E1,1.69E2,3.6E1,2.2E1,3.66E2,8.8E1,2E1,6E0,1.73E2,5E0,5E0,5E0,5E0,4.3E1,1.26E2,3.1E1,5E0,1.7E1,5E0,6E1,3.06E2,7.3E1,1.5E1,9E0,1.1E1,1.27E2,4.6E1,2.7E1,1.6E1,6E0,1.2E2,1.4E1,1.7E1,1.2E1,5E0,3.5E1,2.5E1,9.9E1,2.07E2,3E1,4.3E1,9E0,6E0,6E0,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"91","size_leaf_vector":"1"}},{"base_weights":[-1.2927763E-3,-2.0466879E-2,4.411338E-2,-1.7969824E-2,-1.3497564E-2,6.506536E-2,-5.1240925E-2,-3.6735397E-2,1.1453251E-2,7.480881E-2,-9.060571E-2,-1.8463112E-1,-1.420353E-2,-2.9733544E-2,-1.07094966E-1,-9.500582E-2,1.6599819E-2,6.7820065E-2,1.435025E-2,2.4870888E-3,-8.197694E-3,-1.1369126E-2,-4.3366263E-3,5.4573964E-2,-5.5981852E-2,-5.499607E-2,-6.922136E-3,-1.4109719E-1,-1.6990166E-2,-9.099253E-3,1.7993689E-3,2.3264648E-2,-5.9875995E-2,9.069758E-2,3.3247218E-2,1.1492053E-1,-3.0232486E-3,-9.537412E-2,-9.25311E-3,-4.5390192E-2,-1.9321132E-1,1.03300154E-1,-1.3473892E-2,-1.4312773E-3,-1.6587836E-1,-2.5126683E-3,1.0673901E-3,1.8383415E-2,8.075753E-3,-5.6075784E-3,-1.7744895E-2,2.5084049E-2,1.1162399E-1,7.982544E-3,1.8771423E-2,2.425785E-4,9.533202E-3,-2.1633238E-3,-7.08072E-3,-3.5692465E-3,1.9792595E-3,-3.553227E-2,-1.1082594E-1,-1.4987481E-2,-1.4481494E-3,2.3130465E-3,6.47424E-3,-7.831226E-3,-5.1444485E-3,-4.6849963E-3,-9.189603E-3,2.892759E-2,-4.046036E-2,1.2812115E-3,-3.4922124E-3,7.6299444E-2,-5.563841E-2,7.927598E-2,1.549293E-1,-4.355962E-2,3.923558E-2,-4.036627E-3,-6.2582776E-4,-6.79067E-3,-3.0263679E-3,-4.0232283E-3,-2.0126148E-5,3.9435015E-3,7.833485E-4,-5.7503E-3,-2.5305778E-4,5.026272E-4,7.1970494E-3,3.5960237E-3,-6.8921E-3,6.0680746E-3,1.1618022E-3,7.938113E-3,2.5213226E-3,2.1390861E-3,-3.5504883E-3,4.8420154E-3,8.8174985E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":60,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,33,-1,-1,-1,-1,-1,35,37,39,41,43,45,-1,-1,47,49,51,53,55,-1,57,59,61,63,65,67,-1,69,-1,-1,71,-1,-1,73,75,77,-1,79,-1,-1,-1,-1,-1,-1,81,83,-1,-1,-1,-1,85,-1,-1,-1,87,89,-1,-1,91,93,95,97,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.2102907E-1,4.4248676E-1,5.6273526E-1,3.6372945E-1,0E0,3.5563618E-1,2.4933928E-1,1.9485992E-1,1.4238706E-1,3.3460915E-1,1.8323931E-1,3.7667423E-2,1.208649E-1,2.1111181E-1,1.07898206E-1,1.699161E-1,1.267788E-1,1.6466397E-1,0E0,0E0,0E0,0E0,0E0,1.23093575E-1,4.697702E-2,2.2224617E-1,1.4148887E-1,6.5918565E-2,1.7595194E-2,0E0,0E0,1.638751E-1,4.9237303E-2,1.7157817E-1,1.6955599E-1,1.08171806E-1,0E0,3.194432E-2,4.8948087E-2,1.0185206E-1,2.212215E-1,1.4346942E-2,1.0025658E-1,0E0,2.3859799E-2,0E0,0E0,1.3829187E-1,0E0,0E0,3.5994384E-2,1.3654064E-1,1.2219846E-1,0E0,1.0247772E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.6219732E-1,2.4475753E-2,0E0,0E0,0E0,0E0,1.02073655E-1,0E0,0E0,0E0,1.2551552E-1,9.867185E-2,0E0,0E0,1.0151035E-1,1.7230527E-1,1.5594736E-1,5.569613E-2,6.263328E-2,7.706722E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,17,17,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,35,35,37,37,38,38,39,39,40,40,41,41,42,42,44,44,47,47,50,50,51,51,52,52,54,54,61,61,62,62,67,67,71,71,72,72,75,75,76,76,77,77,78,78,79,79,80,80],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,34,-1,-1,-1,-1,-1,36,38,40,42,44,46,-1,-1,48,50,52,54,56,-1,58,60,62,64,66,68,-1,70,-1,-1,72,-1,-1,74,76,78,-1,80,-1,-1,-1,-1,-1,-1,82,84,-1,-1,-1,-1,86,-1,-1,-1,88,90,-1,-1,92,94,96,98,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.8222644E2,5.6E1,2.856934E7,9.860918E5,-1.3497564E-2,1.8258025E2,7.937392E-3,2.2968E4,5.8182236E1,6.5026445E3,2.4526315E0,1.2380953E1,1E0,5.41E2,1.862E3,3.7789655E2,6.346204E-7,3.925926E0,1.435025E-2,2.4870888E-3,-8.197694E-3,-1.1369126E-2,-4.3366263E-3,5.2224144E5,2.9925186E-2,1.919676E-5,7.784E3,1.0041E5,3.7144188E4,-9.099253E-3,1.7993689E-3,3.5E1,3.4289816E2,1E0,1.9184E4,4.179E3,-3.0232486E-3,4.797448E8,8E0,1.88E2,1.75E2,2.0131578E0,2.0312219E5,-1.4312773E-3,2.4361508E6,-2.5126683E-3,1.0673901E-3,1E0,8.075753E-3,-5.6075784E-3,2.0734694E1,1.4E1,7.098E3,7.982544E-3,4.05E2,2.425785E-4,9.533202E-3,-2.1633238E-3,-7.08072E-3,-3.5692465E-3,1.9792595E-3,3.89E2,3.39E2,-1.4987481E-2,-1.4481494E-3,2.3130465E-3,6.47424E-3,1.84E2,-5.1444485E-3,-4.6849963E-3,-9.189603E-3,1.3543621E6,2.0217392E0,1.2812115E-3,-3.4922124E-3,1.7E1,1.7234043E1,1.2434699E8,1.1450447E9,1.4195632E6,1.5382514E1,-4.036627E-3,-6.2582776E-4,-6.79067E-3,-3.0263679E-3,-4.0232283E-3,-2.0126148E-5,3.9435015E-3,7.833485E-4,-5.7503E-3,-2.5305778E-4,5.026272E-4,7.1970494E-3,3.5960237E-3,-6.8921E-3,6.0680746E-3,1.1618022E-3,7.938113E-3,2.5213226E-3,2.1390861E-3,-3.5504883E-3,4.8420154E-3,8.8174985E-4],"split_indices":[61,3,54,37,0,67,51,38,56,4,62,67,15,2,2,4,46,63,0,0,0,0,0,37,66,46,1,1,42,0,0,3,4,109,1,2,0,7,3,10,6,62,42,0,54,0,0,112,0,0,67,8,2,0,10,0,0,0,0,0,0,1,2,0,0,0,0,38,0,0,0,37,65,0,0,3,65,12,7,54,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.41E2,6.62E2,2.79E2,6.57E2,5E0,2.29E2,5E1,4.01E2,2.56E2,2.16E2,1.3E1,1E1,4E1,3.66E2,3.5E1,1.1E1,2.45E2,2.11E2,5E0,5E0,8E0,5E0,5E0,1.5E1,2.5E1,1.73E2,1.93E2,2.5E1,1E1,6E0,5E0,2.26E2,1.9E1,1.26E2,8.5E1,1E1,5E0,1.3E1,1.2E1,1.63E2,1E1,1E1,1.83E2,5E0,2E1,5E0,5E0,2.2E2,6E0,7E0,1.2E1,3.1E1,9.5E1,7E0,7.8E1,5E0,5E0,8E0,5E0,5E0,7E0,1.43E2,2E1,5E0,5E0,5E0,5E0,1.74E2,9E0,8E0,1.2E1,1.87E2,3.3E1,7E0,5E0,1.9E1,1.2E1,5.6E1,3.9E1,1.9E1,5.9E1,4.2E1,1.01E2,1E1,1E1,1.4E1,1.6E2,3.2E1,1.55E2,9E0,2.4E1,1.1E1,8E0,5E0,7E0,2.8E1,2.8E1,3.3E1,6E0,5E0,1.4E1,1.3E1,4.6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[2.6446474E-3,-2.9053126E-2,3.5118043E-2,-2.3536047E-2,-2.3349646E-1,4.762676E-2,-5.8224484E-2,-1.8992499E-2,-1.033662E-1,-2.0411076E-2,-2.2971001E-3,1.6114274E-1,3.6211584E-2,-1.1742518E-2,-3.5067387E-2,-4.728873E-2,-3.1131473E-3,-1.5557066E-1,2.8097616E-3,2.2083983E-1,9.338853E-2,-1.4169773E-2,5.7437696E-2,5.055733E-2,-5.6155864E-2,-3.7895847E-2,-1.5492065E-1,5.1959883E-3,-6.753958E-2,-1.8610187E-1,-3.3539282E-3,6.8199323E-3,1.24306725E-2,-3.217138E-4,8.170961E-3,1.3641659E-6,-1.2753898E-1,8.876037E-2,1.4744962E-2,4.733578E-3,-4.3288947E-4,-9.337659E-2,1.9547924E-2,-2.2035746E-2,-7.353405E-2,-3.3949695E-3,-1.1047863E-2,2.4253516E-2,-1.822918E-2,-3.7685353E-2,-7.4171824E-3,-4.2334893E-3,-1.060992E-2,-2.5358139E-2,9.682849E-2,-1.110929E-2,-1.2843481E-3,7.399157E-2,1.5521471E-1,-1.2029193E-2,8.1246674E-2,-6.432554E-2,-7.4979067E-3,-2.9137186E-3,4.614366E-3,-3.4171958E-2,6.1891112E-2,-9.661429E-2,-5.738479E-3,4.0391423E-2,-7.372885E-2,-3.5916086E-2,1.1698806E-1,-5.760012E-2,1.9309733E-3,-5.8254104E-2,2.724636E-2,-9.2154345E-4,6.139442E-3,8.135718E-2,-3.2089727E-3,1.9464383E-1,8.6422235E-2,3.0696929E-2,-4.560343E-2,9.7665E-3,3.3384547E-2,-3.98141E-3,-1.625977E-3,3.496208E-3,-1.9674327E-3,5.715782E-3,6.5658434E-4,-2.0529923E-3,-7.239984E-3,1.7397889E-3,-2.2342755E-3,-2.6230895E-4,2.9454841E-3,-9.00884E-3,-1.1117145E-3,-5.358227E-4,-3.4459296E-3,1.0060695E-2,6.962006E-4,-3.899921E-3,-3.7328522E-5,6.1889755E-4,-5.801668E-3,4.172151E-4,5.006178E-3,3.3853317E-3,9.914386E-3,1.0436783E-2,4.7152806E-3,6.603623E-3,1.213935E-3,-2.7553786E-3,2.1411031E-3,-6.0901213E-3,-1.0427037E-3,3.0808125E-4,5.6341486E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":61,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,-1,23,25,27,29,-1,31,33,35,37,39,41,43,45,47,49,51,-1,-1,-1,-1,-1,53,55,57,59,-1,-1,61,63,65,67,-1,-1,69,71,73,-1,-1,-1,75,77,-1,-1,79,81,83,85,87,-1,-1,-1,89,91,93,95,97,99,101,103,105,-1,107,109,-1,-1,111,-1,113,115,117,119,-1,121,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0149146E0,5.573413E-1,5.7139206E-1,1.746777E-1,4.719094E-1,5.51361E-1,2.52831E-1,2.0744127E-1,2.3191497E-1,0E0,0E0,1.3712049E-1,4.2001885E-1,0E0,9.854092E-2,1.6195086E-1,1.5934335E-1,3.7404776E-2,0E0,3.0145109E-2,1.6717793E-1,1.8796891E-1,3.6777228E-1,3.48519E-2,1.2298857E-1,8.542436E-2,7.098341E-2,1.1829852E-1,8.76801E-2,3.5663128E-2,0E0,0E0,0E0,0E0,0E0,2.602816E-1,1.3862206E-1,1.4456606E-1,2.1107952E-1,0E0,0E0,4.7077894E-2,1.0493589E-1,1.12118796E-1,7.298827E-2,0E0,0E0,2.3383763E-1,2.8879917E-1,4.503752E-2,0E0,0E0,0E0,1.4694718E-1,9.373373E-2,0E0,0E0,1.4480716E-1,5.9126437E-2,1.2333213E-1,2.0328838E-1,1.0831945E-2,0E0,0E0,0E0,8.8826574E-2,3.9224576E-2,1.02009535E-1,2.5644578E-2,1.3431656E-1,1.2121543E-1,9.820239E-2,1.3929518E-1,3.2982983E-2,0E0,2.5029022E-1,4.761146E-2,0E0,0E0,1.2439698E-1,0E0,2.5039792E-2,3.6697872E-2,5.562033E-2,9.214788E-2,0E0,6.0415454E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,14,14,15,15,16,16,17,17,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,29,29,35,35,36,36,37,37,38,38,41,41,42,42,43,43,44,44,47,47,48,48,49,49,53,53,54,54,57,57,58,58,59,59,60,60,61,61,65,65,66,66,67,67,68,68,69,69,70,70,71,71,72,72,73,73,75,75,76,76,79,79,81,81,82,82,83,83,84,84,86,86],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,-1,24,26,28,30,-1,32,34,36,38,40,42,44,46,48,50,52,-1,-1,-1,-1,-1,54,56,58,60,-1,-1,62,64,66,68,-1,-1,70,72,74,-1,-1,-1,76,78,-1,-1,80,82,84,86,88,-1,-1,-1,90,92,94,96,98,100,102,104,106,-1,108,110,-1,-1,112,-1,114,116,118,120,-1,122,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.194E3,6.72043E0,3.0923106E-10,3.0977E4,9.3E1,9.42812E3,3.909839E-5,1.5961905E2,1.4386049E6,-2.0411076E-2,-2.2971001E-3,1.03339244E6,1E0,-1.1742518E-2,8.2103E4,1E0,1E0,1.0433326E8,2.8097616E-3,5.0406504E0,6.763312E7,4.88E2,8E0,1.2615725E5,1.4497429E7,5.7676364E-2,1.3938298E2,1E0,6.763314E7,7.0285716E0,-3.3539282E-3,6.8199323E-3,1.24306725E-2,-3.217138E-4,8.170961E-3,2.68334E5,6.648E3,6.5178085E1,1.9616238E2,4.733578E-3,-4.3288947E-4,6.3333454E-7,1.4598765E0,1.1451E4,1.605076E6,-3.3949695E-3,-1.1047863E-2,1.1544592E1,1.5209424E1,1.7587205E6,-7.4171824E-3,-4.2334893E-3,-1.060992E-2,1.5484655E6,5.603014E6,-1.110929E-2,-1.2843481E-3,1.762852E7,1.557718E7,2.0132653E8,1.777125E9,5E0,-7.4979067E-3,-2.9137186E-3,4.614366E-3,1.0204082E0,2.8024E4,3.4E1,2.08E3,3.3875784E7,1.2E1,1.6005026E1,1.1948844E4,2.7528079E3,1.9309733E-3,4.3827028E5,1.4327235E7,-9.2154345E-4,6.139442E-3,2.5045E4,-3.2089727E-3,1.6622167E5,1.4388715E0,1.515E3,7.1E1,9.7665E-3,1.2090626E0,-3.98141E-3,-1.625977E-3,3.496208E-3,-1.9674327E-3,5.715782E-3,6.5658434E-4,-2.0529923E-3,-7.239984E-3,1.7397889E-3,-2.2342755E-3,-2.6230895E-4,2.9454841E-3,-9.00884E-3,-1.1117145E-3,-5.358227E-4,-3.4459296E-3,1.0060695E-2,6.962006E-4,-3.899921E-3,-3.7328522E-5,6.1889755E-4,-5.801668E-3,4.172151E-4,5.006178E-3,3.3853317E-3,9.914386E-3,1.0436783E-2,4.7152806E-3,6.603623E-3,1.213935E-3,-2.7553786E-3,2.1411031E-3,-6.0901213E-3,-1.0427037E-3,3.0808125E-4,5.6341486E-3],"split_indices":[2,62,46,38,12,56,47,64,37,0,0,37,24,0,1,112,85,7,0,63,7,0,26,42,56,47,4,23,7,65,0,0,0,0,0,1,38,67,65,0,0,46,63,12,12,0,0,65,65,37,0,0,0,37,9,0,0,56,60,7,5,6,0,0,0,62,12,0,38,7,3,67,42,42,0,37,12,0,0,2,0,42,63,2,8,0,43,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.84E2,4.98E2,4.86E2,4.86E2,1.2E1,4.29E2,5.7E1,4.61E2,2.5E1,5E0,7E0,3.8E1,3.91E2,5E0,5.2E1,1.65E2,2.96E2,1.9E1,6E0,1.9E1,1.9E1,1.16E2,2.75E2,1E1,4.2E1,1.53E2,1.2E1,2.63E2,3.3E1,1.3E1,6E0,9E0,1E1,9E0,1E1,1.04E2,1.2E1,1.58E2,1.17E2,5E0,5E0,2.8E1,1.4E1,1.07E2,4.6E1,7E0,5E0,1.45E2,1.18E2,2.6E1,7E0,5E0,8E0,8.3E1,2.1E1,5E0,7E0,1.31E2,2.7E1,8.4E1,3.3E1,2.1E1,7E0,7E0,7E0,9.4E1,1.3E1,3.4E1,1.2E1,1.25E2,2E1,1.05E2,1.3E1,2.1E1,5E0,5.1E1,3.2E1,5E0,1.6E1,1.25E2,6E0,1.6E1,1.1E1,3.7E1,4.7E1,8E0,2.5E1,1.1E1,1E1,6E0,8.8E1,5E0,8E0,1.9E1,1.5E1,6E0,6E0,4.2E1,8.3E1,5E0,1.5E1,6.5E1,4E1,6E0,7E0,1.4E1,7E0,2.5E1,2.6E1,2.7E1,5E0,1.19E2,6E0,1.1E1,5E0,5E0,6E0,5E0,3.2E1,9E0,3.8E1,2E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"123","size_leaf_vector":"1"}},{"base_weights":[1.4619857E-4,-4.3116298E-2,2.5807634E-2,-3.401605E-2,-2.4226734E-1,3.7377037E-2,-9.606369E-2,7.0993535E-2,-4.2770054E-2,1.6232535E-3,-1.7835543E-2,1.6670904E-1,3.2810718E-2,-1.433587E-1,5.3210884E-2,-6.721502E-3,1.1998553E-1,-4.6648275E-2,5.03407E-3,4.124558E-3,9.196808E-3,3.575874E-2,-1.0426592E-2,-1.06331795E-1,-1.3800777E-2,6.9186483E-3,-1.176937E-3,-2.6762055E-3,2.1044372E-3,9.932896E-3,5.775688E-2,-2.8964E-2,-8.4109865E-2,1.700695E-1,3.0812467E-2,-6.7778245E-2,-7.4439137E-3,8.2069635E-4,4.0924256E-3,-3.5256453E-2,6.716677E-2,-1.1080469E-2,-6.536189E-2,2.0334658E-1,3.2305236E-3,1.0642636E-2,5.6909274E-2,-9.476707E-2,7.186496E-4,-2.927414E-2,-8.857074E-3,4.122212E-3,1.2561294E-3,5.0382856E-2,-8.1048E-2,6.1865384E-3,1.219413E-2,5.0867624E-3,6.252918E-3,6.311107E-2,-7.919077E-3,-5.3357477E-3,-1.9434908E-3,-2.095843E-3,5.1070197E-4,3.0533227E-4,3.980514E-3,-4.8524733E-3,3.7860574E-4,7.0537144E-4,-1.8432189E-3,7.602387E-3,2.3990811E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":62,"left_children":[1,3,5,7,9,11,13,15,17,-1,-1,19,21,23,25,27,29,31,-1,-1,-1,33,-1,35,-1,-1,-1,-1,-1,-1,37,39,41,43,45,47,-1,-1,-1,49,51,-1,53,55,-1,57,59,61,-1,63,-1,-1,-1,65,67,-1,-1,69,-1,71,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0346882E0,6.206352E-1,8.284279E-1,3.0914807E-1,6.045108E-1,3.1048578E-1,3.676854E-1,9.979926E-2,1.8574512E-1,0E0,0E0,2.6503444E-2,4.0129548E-1,1.9086611E-1,1.0174306E-1,3.1805623E-2,8.131556E-2,1.969707E-1,0E0,0E0,0E0,3.355651E-1,0E0,5.7955176E-2,0E0,0E0,0E0,0E0,0E0,0E0,1.1858087E-2,1.2697713E-1,2.628147E-1,4.6007633E-2,2.6047295E-1,4.8660405E-2,0E0,0E0,0E0,1.7503268E-1,8.892618E-3,0E0,1.6203925E-1,1.9080102E-2,0E0,1.9309108E-1,3.1563622E-1,1.2298897E-2,0E0,1.208702E-1,0E0,0E0,0E0,1.6494604E-2,1.6249806E-1,0E0,0E0,1.229921E-1,0E0,2.3396671E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,21,21,23,23,30,30,31,31,32,32,33,33,34,34,35,35,39,39,40,40,42,42,43,43,45,45,46,46,47,47,49,49,53,53,54,54,57,57,59,59],"right_children":[2,4,6,8,10,12,14,16,18,-1,-1,20,22,24,26,28,30,32,-1,-1,-1,34,-1,36,-1,-1,-1,-1,-1,-1,38,40,42,44,46,48,-1,-1,-1,50,52,-1,54,56,-1,58,60,62,-1,64,-1,-1,-1,66,68,-1,-1,70,-1,72,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.540633E2,1.0219829E1,2.856934E7,7E0,1.3E1,1.6E1,1.3655363E2,1.8236995E0,1.8645384E1,1.6232535E-3,-1.7835543E-2,8.771579E2,3.8652172E1,2.6666667E0,1.2083E4,1.1428572E0,1E0,2.6742857E0,5.03407E-3,4.124558E-3,9.196808E-3,1.0918E4,-1.0426592E-2,2.94857E5,-1.3800777E-2,6.9186483E-3,-1.176937E-3,-2.6762055E-3,2.1044372E-3,9.932896E-3,3.3512E4,4.49239E6,1.3909517E-5,7.607E3,1.5859042E6,5E0,-7.4439137E-3,8.2069635E-4,4.0924256E-3,2.4240502E8,1.4E1,-1.1080469E-2,1.6293629E-3,2.78E2,3.2305236E-3,1.6327E4,4.262058E0,1.0600238E3,7.186496E-4,5.51E2,-8.857074E-3,4.122212E-3,1.2561294E-3,1.5E1,7.997723E6,6.1865384E-3,1.219413E-2,2.1129E4,6.252918E-3,1.3333334E0,-7.919077E-3,-5.3357477E-3,-1.9434908E-3,-2.095843E-3,5.1070197E-4,3.0533227E-4,3.980514E-3,-4.8524733E-3,3.7860574E-4,7.0537144E-4,-1.8432189E-3,7.602387E-3,2.3990811E-3],"split_indices":[61,63,54,3,0,0,65,63,65,0,0,42,62,62,9,65,13,65,0,0,0,9,0,1,0,0,0,0,0,0,1,37,47,2,37,8,0,0,0,5,3,0,47,0,0,2,51,64,0,2,0,0,0,3,9,0,0,38,0,62,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.3E2,3.46E2,5.84E2,3.32E2,1.4E1,5.34E2,5E1,2.5E1,3.07E2,5E0,9E0,1.7E1,5.17E2,3.8E1,1.2E1,1E1,1.5E1,3E2,7E0,6E0,1.1E1,5.12E2,5E0,3.2E1,6E0,5E0,7E0,5E0,5E0,5E0,1E1,2.05E2,9.5E1,1.7E1,4.95E2,2E1,1.2E1,5E0,5E0,1.93E2,1.2E1,9E0,8.6E1,1.2E1,5E0,2.8E2,2.15E2,1.5E1,5E0,1.87E2,6E0,7E0,5E0,1E1,7.6E1,7E0,5E0,2.69E2,1.1E1,2.1E2,5E0,1E1,5E0,1.34E2,5.3E1,5E0,5E0,6E1,1.6E1,2.2E2,4.9E1,2E1,1.9E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"73","size_leaf_vector":"1"}},{"base_weights":[1.3233483E-3,-7.1637644E-3,1.0466198E-1,-1.1753857E-3,-1.12060666E-1,1.29026E-1,-2.9940588E-2,5.7675842E-2,-9.821313E-3,-2.3911023E-1,-2.0576114E-2,1.1026906E-1,1.3256889E-2,3.1037685E-3,-5.0419024E-3,7.725164E-2,-1.040588E-2,-4.019458E-2,4.1795173E-3,-1.6726252E-1,-1.7579705E-2,3.2956146E-2,-9.915983E-2,1.267554E-1,-3.0411035E-4,9.5777914E-2,-5.440254E-2,-5.6063095E-3,5.9257805E-2,4.8338518E-2,-4.923502E-2,9.539258E-3,-4.5886543E-3,-1.5283967E-2,-2.4709417E-3,3.9046488E-3,-1.2802079E-3,-7.0528565E-3,-9.7358105E-4,1.5400207E-1,3.7360888E-2,-4.4519263E-3,1.1721136E-1,-9.253999E-3,4.626216E-3,7.6561673E-3,-3.8099505E-3,4.3325415E-3,1.593378E-2,-3.755256E-2,-1.8034352E-1,3.8713685E-3,7.353074E-2,3.5966206E-3,1.8632276E-1,-4.0566668E-4,3.6250665E-3,-3.1934485E-3,2.432776E-3,1.463347E-1,2.6156072E-2,-2.1773006E-3,1.8532051E-3,-6.0552097E-4,2.6333565E-3,-2.6108874E-2,-1.2635478E-1,-1.9908901E-2,-5.663937E-3,-3.8536447E-3,4.6415612E-2,-1.3731684E-2,1.17690094E-1,5.4723304E-3,9.75144E-3,1.0446624E-2,5.4838853E-3,-1.4294594E-3,3.4281882E-3,-5.664348E-3,-8.16702E-4,-2.4096663E-3,-9.765564E-3,1.4281555E-3,-1.9099615E-3,3.3139251E-3,-4.593174E-4,7.571561E-5,6.2159016E-3,-3.1722805E-3,2.3511795E-3,2.9303755E-3,6.859086E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":63,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,-1,-1,25,27,29,31,33,-1,35,37,39,-1,41,43,-1,45,47,49,51,-1,-1,-1,-1,-1,-1,-1,53,55,57,59,-1,-1,-1,61,-1,63,65,67,69,71,-1,73,-1,-1,-1,-1,75,77,-1,-1,-1,-1,79,81,-1,83,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[8.435916E-1,5.5830467E-1,2.4326354E-1,4.2899582E-1,5.529876E-1,1.507101E-1,9.8743975E-2,1.447413E-1,3.1272742E-1,1.3746762E-1,1.2526406E-1,1.10612154E-1,0E0,0E0,0E0,2.1009177E-1,1.9946884E-1,1.8806776E-1,2.7937868E-1,2.5973478E-1,0E0,5.8118723E-2,4.6384312E-2,1.1603141E-1,0E0,1.5949374E-1,2.6575273E-1,0E0,1.0733719E-1,3.0618839E-2,3.1640995E-1,1.7380974E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,7.949841E-2,2.5116477E-2,5.487313E-2,1.5793431E-1,0E0,0E0,0E0,2.262352E-2,0E0,1.7304542E-2,1.9486487E-1,7.3619604E-1,1.454443E-1,1.519007E-1,0E0,1.7600536E-2,0E0,0E0,0E0,0E0,7.542795E-2,4.577671E-2,0E0,0E0,0E0,0E0,1.3991174E-1,1.2678546E-1,0E0,1.54980235E-2,1.7136282E-1,2.6639572E-1,5.272873E-2,3.339511E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,15,15,16,16,17,17,18,18,19,19,21,21,22,22,23,23,25,25,26,26,28,28,29,29,30,30,31,31,39,39,40,40,41,41,42,42,46,46,48,48,49,49,50,50,51,51,52,52,54,54,59,59,60,60,65,65,66,66,68,68,69,69,70,70,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,-1,-1,26,28,30,32,34,-1,36,38,40,-1,42,44,-1,46,48,50,52,-1,-1,-1,-1,-1,-1,-1,54,56,58,60,-1,-1,-1,62,-1,64,66,68,70,72,-1,74,-1,-1,-1,-1,76,78,-1,-1,-1,-1,80,82,-1,84,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.6471942E3,5.869281E0,3.5056704E7,7E0,9.4E1,1.0534078E4,3.6086957E0,1.03984375E1,1.795688E2,3.3924444E5,8.9535345E2,4.6553187E0,1.3256889E-2,3.1037685E-3,-5.0419024E-3,1E0,1E0,1.0204082E0,4.217427E7,3.964269E3,-1.7579705E-2,2.057307E6,2.5503825E6,1.3358269E2,-3.0411035E-4,6.83E2,1.503E3,-5.6063095E-3,3.167E3,5.862757E1,1.1251582E10,4.49239E6,-4.5886543E-3,-1.5283967E-2,-2.4709417E-3,3.9046488E-3,-1.2802079E-3,-7.0528565E-3,-9.7358105E-4,6.880842E-1,9.730783E8,2.327189E-1,4.362075E8,-9.253999E-3,4.626216E-3,7.6561673E-3,9.422379E4,4.3325415E-3,7.8918E4,2.6E1,3.6977E4,1.3439851E7,4.354839E0,3.5966206E-3,1.3359968E10,-4.0566668E-4,3.6250665E-3,-3.1934485E-3,2.432776E-3,2.5818555E2,1.62576E5,-2.1773006E-3,1.8532051E-3,-6.0552097E-4,2.6333565E-3,1.2571428E1,8.7578294E5,-1.9908901E-2,6.387097E0,4.8679228E5,2.2503355E1,1.503E3,8.16E0,5.4723304E-3,9.75144E-3,1.0446624E-2,5.4838853E-3,-1.4294594E-3,3.4281882E-3,-5.664348E-3,-8.16702E-4,-2.4096663E-3,-9.765564E-3,1.4281555E-3,-1.9099615E-3,3.3139251E-3,-4.593174E-4,7.571561E-5,6.2159016E-3,-3.1722805E-3,2.3511795E-3,2.9303755E-3,6.859086E-3],"split_indices":[61,62,54,3,0,4,63,65,61,37,4,47,0,0,0,85,25,62,54,37,0,12,41,67,0,2,2,0,2,56,40,37,0,0,0,0,0,0,0,51,7,67,7,0,0,0,42,0,1,8,12,54,65,0,40,0,0,0,0,4,1,0,0,0,0,64,54,0,67,54,65,2,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.6E2,8.88E2,7.2E1,8.41E2,4.7E1,6.1E1,1.1E1,1.07E2,7.34E2,1.9E1,2.8E1,5.6E1,5E0,5E0,6E0,8.3E1,2.4E1,2.31E2,5.03E2,1.4E1,5E0,1.7E1,1.1E1,4.9E1,7E0,7.3E1,1E1,9E0,1.5E1,2.1E1,2.1E2,4.79E2,2.4E1,5E0,9E0,9E0,8E0,6E0,5E0,3.7E1,1.2E1,1.3E1,6E1,5E0,5E0,5E0,1E1,8E0,1.3E1,1.94E2,1.6E1,4.41E2,3.8E1,1.2E1,2.5E1,6E0,6E0,6E0,7E0,4.5E1,1.5E1,5E0,5E0,8E0,5E0,1.73E2,2.1E1,6E0,1E1,3.74E2,6.7E1,1.3E1,2.5E1,8E0,1.7E1,1E1,3.5E1,7E0,8E0,1.3E1,1.6E2,1.2E1,9E0,5E0,5E0,2.7E1,3.47E2,4.5E1,2.2E1,7E0,6E0,1E1,1.5E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[3.2265678E-3,-3.891943E-2,2.944353E-2,4.084388E-2,-4.658256E-2,1.9949717E-2,9.176524E-2,9.828754E-3,4.601014E-3,-6.1389275E-2,5.070066E-3,1.5682872E-2,2.089805E-1,1.0438453E-1,-3.3805307E-3,-2.0626825E-3,3.296357E-2,-5.3487636E-2,-1.4688909E-1,-3.8830792E-3,1.5592387E-2,1.2709334E-2,7.318877E-3,5.711789E-3,1.2065258E-2,1.5809596E-2,1.2986434E-1,3.0690278E-3,-8.883485E-4,-4.2490866E-2,-1.4025047E-1,-5.871059E-2,-1.3123622E-2,2.645651E-2,-3.147624E-3,2.2501634E-2,-4.9501132E-2,-3.8193201E-3,7.6814644E-2,1.4235021E-1,2.3306855E-3,-4.6986874E-2,5.484616E-3,-9.2867727E-4,-2.0272727E-1,-7.947294E-3,1.0606505E-3,1.4494295E-2,5.3339344E-3,7.043083E-2,1.0949071E-2,-7.517675E-2,7.230388E-2,1.4032479E-3,5.1310337E-3,1.5614624E-1,3.0971933E-3,-9.864465E-2,-3.425616E-2,-2.5535598E-1,-2.954806E-3,6.349437E-2,-2.5909557E-3,4.4530414E-2,1.3008888E-1,-6.9922216E-2,2.0521404E-2,-1.354164E-2,-1.1000991E-1,6.6242204E-3,7.327974E-5,1.0441272E-1,8.132507E-3,-7.924449E-3,-3.6268644E-3,9.318714E-4,-2.6842821E-3,-1.5249563E-2,-5.905574E-3,8.3873473E-4,4.9336473E-3,-7.646858E-4,1.7069352E-3,-3.1587472E-3,3.0903304E-3,3.990663E-3,9.001611E-3,-8.561549E-3,-1.8955243E-3,6.2764506E-3,6.918638E-4,-3.2510106E-3,2.5817323E-3,-2.5554707E-3,-7.971523E-3,1.6591989E-3,7.3380787E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":64,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,-1,27,29,31,-1,33,35,-1,-1,-1,37,39,-1,-1,41,43,45,-1,47,-1,49,51,-1,53,55,-1,57,-1,-1,59,-1,-1,61,-1,63,65,67,69,-1,-1,71,-1,73,75,77,-1,79,81,83,85,87,89,91,93,-1,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.0342649E0,2.213366E-1,3.3956325E-1,5.690424E-2,2.5116748E-1,4.0161842E-1,1.6548896E-1,2.9093944E-2,0E0,1.644184E-1,7.0416614E-2,2.058169E-1,1.621595E-2,1.5818483E-1,0E0,0E0,2.8930234E-2,2.1825665E-1,2.3514172E-1,0E0,6.252313E-2,2.953495E-1,0E0,0E0,0E0,1.0808548E-1,4.7535777E-2,0E0,0E0,1.5727633E-1,1.9125283E-1,1.3246702E-1,0E0,6.185346E-2,0E0,2.3067753E-1,2.1184525E-1,0E0,1.395537E-2,3.859687E-2,0E0,1.314371E-1,0E0,0E0,1.0814059E-1,0E0,0E0,4.537215E-2,0E0,1.1936572E-1,2.630898E-1,1.1659229E-1,5.9574686E-2,0E0,0E0,2.0287156E-2,0E0,4.5745373E-2,2.1400298E-1,7.083166E-2,0E0,2.5315017E-2,2.303141E-2,1.486944E-1,5.093342E-2,1.1238879E-1,1.9304854E-1,8.601943E-2,1.0934633E-1,0E0,0E0,4.2305335E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,16,16,17,17,18,18,20,20,21,21,25,25,26,26,29,29,30,30,31,31,33,33,35,35,36,36,38,38,39,39,41,41,44,44,47,47,49,49,50,50,51,51,52,52,55,55,57,57,58,58,59,59,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,71,71],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,-1,28,30,32,-1,34,36,-1,-1,-1,38,40,-1,-1,42,44,46,-1,48,-1,50,52,-1,54,56,-1,58,-1,-1,60,-1,-1,62,-1,64,66,68,70,-1,-1,72,-1,74,76,78,-1,80,82,84,86,88,90,92,94,-1,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[8.1E2,1.6666666E0,3.8807898E6,9.3411E4,2.3285185E2,7.626E3,2.64538E3,3.8443E4,4.601014E-3,4.909091E0,1.31E2,1.0427401E-5,2.628361E6,1E0,-3.3805307E-3,-2.0626825E-3,1E0,4.064E3,7.90913E5,-3.8830792E-3,7.03E3,3.0923106E-10,7.318877E-3,5.711789E-3,1.2065258E-2,5.941442E6,1E0,3.0690278E-3,-8.883485E-4,3.289E3,1.4E1,3.293375E3,-1.3123622E-2,2.1E1,-3.147624E-3,8.589474E0,1.4595818E7,-3.8193201E-3,1.4523518E8,2.7994856E5,2.3306855E-3,5.4E1,5.484616E-3,-9.2867727E-4,2.7976523E2,-7.947294E-3,1.0606505E-3,2.5427E4,5.3339344E-3,1.535E3,3.74083E5,2E0,3.443E3,1.4032479E-3,5.1310337E-3,2.1432705E10,3.0971933E-3,6.25E0,4.8421054E0,1.53053E5,-2.954806E-3,6.97E3,2.4438E4,2.1525E4,6.763312E7,2E0,1.2877E4,1.839E3,2.511E3,6.6242204E-3,7.327974E-5,1.28738E5,8.132507E-3,-7.924449E-3,-3.6268644E-3,9.318714E-4,-2.6842821E-3,-1.5249563E-2,-5.905574E-3,8.3873473E-4,4.9336473E-3,-7.646858E-4,1.7069352E-3,-3.1587472E-3,3.0903304E-3,3.990663E-3,9.001611E-3,-8.561549E-3,-1.8955243E-3,6.2764506E-3,6.918638E-4,-3.2510106E-3,2.5817323E-3,-2.5554707E-3,-7.971523E-3,1.6591989E-3,7.3380787E-3],"split_indices":[2,63,37,1,61,61,4,9,0,62,38,46,1,109,0,0,8,38,9,0,38,46,0,0,0,54,85,0,0,38,3,56,0,3,0,67,56,0,12,42,0,2,0,0,4,0,0,9,0,2,12,8,2,0,0,40,0,61,67,9,0,12,1,9,7,8,9,2,2,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.34E2,3.58E2,5.76E2,3.1E1,3.27E2,5.01E2,7.5E1,2.1E1,1E1,2.54E2,7.3E1,4.91E2,1E1,7E1,5E0,6E0,1.5E1,2.34E2,2E1,7E0,6.6E1,4.82E2,9E0,5E0,5E0,1.6E1,5.4E1,9E0,6E0,2.09E2,2.5E1,1.3E1,7E0,5.9E1,7E0,4.17E2,6.5E1,6E0,1E1,4.6E1,8E0,2.04E2,5E0,9E0,1.6E1,5E0,8E0,5.3E1,6E0,8E1,3.37E2,5.4E1,1.1E1,5E0,5E0,3.8E1,8E0,3.9E1,1.65E2,1.1E1,5E0,1.3E1,4E1,5.7E1,2.3E1,3.5E1,3.02E2,2E1,3.4E1,5E0,6E0,1.2E1,2.6E1,7E0,3.2E1,5E1,1.15E2,6E0,5E0,7E0,6E0,3E1,1E1,9E0,4.8E1,1.5E1,8E0,6E0,2.9E1,1.3E1,2.89E2,1.1E1,9E0,1.9E1,1.5E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-9.96352E-4,-3.176855E-2,1.8873597E-2,-2.2779811E-2,-1.2230781E-1,2.7977742E-2,-5.2203033E-2,-2.7143361E-2,6.6641956E-3,-2.3564333E-1,-2.0430114E-2,7.993273E-3,5.777277E-2,-1.3526298E-1,-3.1621534E-2,7.163747E-2,-3.3581264E-2,-3.0498512E-4,-1.916661E-2,4.6231397E-3,-7.042854E-2,2.6838E-2,-3.068858E-2,8.369381E-2,7.288389E-3,-1.8136533E-3,-9.842001E-3,4.433311E-3,-5.8579385E-2,-3.3773913E-3,1.3426091E-1,-4.5816675E-2,-3.4868154E-3,-9.760591E-4,-4.8646727E-3,3.3956718E-2,-6.2043633E-2,-1.1672205E-2,-1.1834121E-2,1.21051565E-1,4.345712E-2,-5.4413993E-3,2.038902E-2,-5.9562796E-3,-3.81339E-2,1.0572008E-2,2.1600374E-3,-3.9922375E-2,-1.11710705E-1,-1.6057072E-2,6.859229E-2,1.21874465E-2,6.5369815E-2,-1.8468697E-4,-4.953678E-3,-2.2224613E-2,6.9746654E-3,1.779053E-1,8.6285286E-2,-2.976048E-3,5.8872372E-2,1.1214507E-1,-1.6355855E-2,-9.3190875E-3,-5.8808704E-3,-5.1418614E-2,4.3571233E-3,-6.112757E-3,-2.3056224E-3,3.7603016E-4,-7.697834E-2,5.198076E-3,4.2462422E-4,2.1341642E-2,-7.796587E-2,9.635629E-2,1.831767E-2,-3.9376713E-2,5.475375E-2,1.9646284E-1,4.6152887E-3,1.16912924E-1,7.928346E-3,1.3657594E-1,2.497617E-2,2.761502E-3,7.1250806E-3,7.3903874E-3,-6.3108234E-3,1.2091109E-2,-3.110256E-3,-2.0297149E-3,-5.033885E-3,1.8952106E-3,-2.3486277E-3,3.992192E-3,-7.055505E-4,-6.0325046E-3,-1.2459699E-3,5.370967E-4,3.2009403E-3,-6.18938E-3,-4.4283163E-4,2.7733652E-3,5.688442E-3,4.7023417E-3,-3.6811837E-4,-5.339996E-3,-7.83886E-4,-9.6993725E-4,6.611072E-3,5.768594E-3,1.0866486E-2,6.948648E-3,3.581632E-3,-3.144736E-3,4.329725E-3,3.037663E-3,1.1087829E-2,-2.1427276E-3,2.0958555E-3,-1.8859397E-3,2.7774982E-3,-7.986992E-4,2.2725535E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":65,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,27,29,31,-1,-1,-1,33,35,37,39,41,-1,-1,-1,43,-1,45,47,49,-1,-1,51,53,55,-1,57,59,-1,61,-1,63,-1,-1,65,67,69,71,73,75,-1,-1,77,-1,79,81,-1,83,85,87,89,-1,91,93,-1,-1,95,97,-1,-1,99,101,103,105,107,109,111,-1,113,115,117,119,-1,-1,121,-1,123,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.906411E-1,3.0565685E-1,3.813036E-1,2.551371E-1,3.87762E-1,3.0976132E-1,1.1025913E-1,2.1735397E-1,0E0,6.476078E-1,1.210784E-1,2.2898464E-1,2.7288902E-1,8.779493E-2,1.9414712E-1,1.9970763E-1,1.16933554E-1,0E0,0E0,0E0,2.1592237E-2,1.3561137E-1,4.364747E-1,2.0217437E-1,1.1988308E-1,0E0,0E0,0E0,6.0742065E-2,0E0,1.12517625E-1,8.33185E-2,8.5364304E-2,0E0,0E0,1.3323599E-1,4.111866E-2,1.6744187E-1,0E0,1.2498748E-1,1.1656311E-1,0E0,2.2435749E-1,0E0,9.3290895E-2,0E0,0E0,1.0661501E-1,1.5815914E-2,8.0118984E-2,3.5121486E-2,9.822182E-2,1.1487341E-1,0E0,0E0,1.2230703E-1,0E0,1.8303394E-2,1.0946792E-1,0E0,1.5462023E-1,3.2273546E-2,1.3739032E-1,3.6644343E-2,0E0,6.795868E-2,9.032903E-2,0E0,0E0,8.6736575E-2,4.1700684E-2,0E0,0E0,4.8989374E-2,4.044406E-2,3.8753122E-2,7.273007E-2,1.2571786E-1,1.1581285E-1,2.5321662E-2,0E0,3.14928E-2,9.68208E-2,1.1540809E-1,6.387302E-2,0E0,0E0,1.05772346E-1,0E0,2.4624221E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,20,20,21,21,22,22,23,23,24,24,28,28,30,30,31,31,32,32,35,35,36,36,37,37,39,39,40,40,42,42,44,44,47,47,48,48,49,49,50,50,51,51,52,52,55,55,57,57,58,58,60,60,61,61,62,62,63,63,65,65,66,66,69,69,70,70,73,73,74,74,75,75,76,76,77,77,78,78,79,79,81,81,82,82,83,83,84,84,87,87,89,89],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,28,30,32,-1,-1,-1,34,36,38,40,42,-1,-1,-1,44,-1,46,48,50,-1,-1,52,54,56,-1,58,60,-1,62,-1,64,-1,-1,66,68,70,72,74,76,-1,-1,78,-1,80,82,-1,84,86,88,90,-1,92,94,-1,-1,96,98,-1,-1,100,102,104,106,108,110,112,-1,114,116,118,120,-1,-1,122,-1,124,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.0449125E5,5.119403E0,1.4569893E0,5.53648E8,2.988E4,6.009825E2,1.5031847E0,7E0,6.6641956E-3,1.3E1,1.737736E1,4.9390244E0,2.1298597E0,1.5589E4,2.4E0,5.7018555E4,5.117647E0,-3.0498512E-4,-1.916661E-2,4.6231397E-3,4.9E1,1E0,3.8496505E6,2E0,3.167E3,-1.8136533E-3,-9.842001E-3,4.433311E-3,2.0549193E2,-3.3773913E-3,5E0,4.2E2,7.403968E2,-9.760591E-4,-4.8646727E-3,6.763314E7,1E0,3.5E1,-1.1834121E-2,7.927835E0,1.404E3,-5.4413993E-3,1.6386554E0,-5.9562796E-3,1.01E2,1.0572008E-2,2.1600374E-3,9.558022E7,4.7126865E1,2.4743947E5,1.3177083E1,6.0913334E0,1.3558896E7,-1.8468697E-4,-4.953678E-3,3.9824562E0,6.9746654E-3,2.3017536E5,5.844366E8,-2.976048E-3,2.08E4,1.2825651E0,4.5866325E6,6.1313953E1,-5.8808704E-3,1.2E1,1.387E3,-6.112757E-3,-2.3056224E-3,5.2755904E0,1.17E2,5.198076E-3,4.2462422E-4,1.2600376E7,7.2132964E0,8.57E2,5.457143E0,7E0,3.53274E2,7.7630096E2,4.6152887E-3,9.557399E6,3.84E2,8E0,2.429044E0,2.761502E-3,7.1250806E-3,3.002566E0,-6.3108234E-3,1.5466666E1,-3.110256E-3,-2.0297149E-3,-5.033885E-3,1.8952106E-3,-2.3486277E-3,3.992192E-3,-7.055505E-4,-6.0325046E-3,-1.2459699E-3,5.370967E-4,3.2009403E-3,-6.18938E-3,-4.4283163E-4,2.7733652E-3,5.688442E-3,4.7023417E-3,-3.6811837E-4,-5.339996E-3,-7.83886E-4,-9.6993725E-4,6.611072E-3,5.768594E-3,1.0866486E-2,6.948648E-3,3.581632E-3,-3.144736E-3,4.329725E-3,3.037663E-3,1.1087829E-2,-2.1427276E-3,2.0958555E-3,-1.8859397E-3,2.7774982E-3,-7.986992E-4,2.2725535E-3],"split_indices":[37,62,55,7,1,64,62,3,0,0,67,65,47,9,63,54,63,0,0,0,8,85,56,26,2,0,0,0,4,0,8,0,4,0,0,7,8,3,0,65,2,0,63,0,0,0,0,7,67,37,65,63,9,0,0,62,0,42,7,0,10,63,37,55,0,8,2,0,0,63,0,0,0,54,63,2,67,3,4,4,0,54,0,8,62,0,0,47,0,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.64E2,3.78E2,5.86E2,3.45E2,3.3E1,5.2E2,6.6E1,3.37E2,8E0,1.5E1,1.8E1,3.12E2,2.08E2,1.2E1,5.4E1,2E1,3.17E2,7E0,8E0,5E0,1.3E1,2.1E2,1.02E2,1.37E2,7.1E1,6E0,6E0,9E0,4.5E1,6E0,1.4E1,2.25E2,9.2E1,6E0,7E0,1.95E2,1.5E1,9.5E1,7E0,7E1,6.7E1,6E0,6.5E1,9E0,3.6E1,6E0,8E0,2.08E2,1.7E1,7.9E1,1.3E1,1.16E2,7.9E1,7E0,8E0,9E1,5E0,2.5E1,4.5E1,8E0,5.9E1,1.8E1,4.7E1,2.8E1,8E0,1.65E2,4.3E1,1.2E1,5E0,6.3E1,1.6E1,7E0,6E0,1.06E2,1E1,4.7E1,3.2E1,7.4E1,1.6E1,1.9E1,6E0,3.2E1,1.3E1,1.7E1,4.2E1,9E0,9E0,4E1,7E0,2.1E1,7E0,1.47E2,1.8E1,2.6E1,1.7E1,9E0,5.4E1,7E0,9E0,8.9E1,1.7E1,5E0,5E0,2.1E1,2.6E1,7E0,2.5E1,1.6E1,5.8E1,9E0,7E0,8E0,1.1E1,1.6E1,1.6E1,7E0,6E0,1.1E1,6E0,9E0,3.3E1,2.1E1,1.9E1,1.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"125","size_leaf_vector":"1"}},{"base_weights":[-1.3744421E-3,-1.1059426E-2,5.8610898E-2,-5.372319E-3,-8.725248E-2,7.446804E-2,-1.2691213E-1,-1.84089E-2,1.6433243E-2,-2.0516416E-1,-2.4547707E-2,1.1269331E-2,6.2419422E-2,-7.7512197E-4,-1.0020731E-2,1.8497472E-5,-8.2482316E-2,-1.9790519E-2,3.7491262E-2,-1.825265E-2,-4.6904422E-2,2.4645863E-2,-6.3681185E-2,9.78716E-3,1.00975685E-1,-4.699609E-3,6.7324936E-3,-1.9740802E-1,-4.5928143E-2,-5.7375464E-3,-1.0653645E-1,4.9974537E-3,6.5667726E-2,3.553904E-3,-8.498895E-3,-2.6771491E-3,4.46783E-3,-8.3420426E-2,-4.3919904E-4,-4.29793E-3,2.2933025E-2,6.2254798E-2,1.3786554E-1,6.461834E-2,-1.1699938E-2,-6.664228E-2,-2.2424864E-2,-6.819906E-2,8.095142E-2,3.4240045E-2,-2.1526625E-2,-1.5405503E-3,-6.490511E-3,4.2231694E-2,-6.3626714E-2,8.6188704E-2,-4.617165E-2,-9.146744E-4,-5.1311348E-3,4.651011E-2,-2.889363E-2,5.804533E-4,7.899302E-2,1.6772106E-1,2.2939965E-3,4.299971E-3,6.530017E-3,-1.4554272E-2,4.971559E-3,-7.5137955E-3,-1.0203222E-2,1.5651736E-2,-8.523008E-2,5.2433545E-3,1.1470723E-3,-1.0227004E-2,4.632939E-3,-3.5098795E-2,4.4735074E-2,7.743341E-2,1.0209822E-2,-1.2187626E-2,-1.10545745E-2,1.0243887E-1,-3.7580142E-3,1.136957E-3,-8.0524415E-2,2.332747E-3,8.535292E-2,1.2007819E-3,-5.352921E-3,9.741284E-2,6.301834E-5,4.8840707E-3,9.38861E-3,2.2728667E-3,-2.5871198E-3,-1.5016998E-3,2.6206864E-4,-3.2328472E-3,1.6553224E-3,-2.8298984E-3,3.3025495E-3,-8.469639E-3,-2.8822292E-3,-2.544059E-3,1.6477305E-3,-3.072216E-4,-2.9861236E-3,3.3832544E-3,2.0423136E-4,1.8947711E-3,5.890565E-3,-1.4755076E-3,2.6512833E-3,-2.6204141E-3,1.5329168E-3,1.2057188E-3,5.522271E-3,-5.875029E-3,-9.748267E-4,2.1306581E-3,-1.7014951E-3,2.6238442E-3,5.973231E-3,1.4198788E-3,5.6652464E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":66,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,-1,25,27,29,31,-1,33,35,37,39,41,43,-1,45,47,49,51,53,55,-1,-1,-1,-1,57,-1,-1,59,61,63,65,67,69,-1,71,73,75,77,-1,-1,79,81,83,85,-1,-1,87,89,-1,91,93,-1,95,-1,97,-1,-1,99,101,103,-1,-1,105,-1,107,109,111,113,115,-1,117,-1,-1,119,121,123,-1,-1,125,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.6417656E-1,3.6178517E-1,4.066026E-1,2.2174886E-1,4.2379388E-1,2.384761E-1,1.04350805E-1,5.761862E-1,2.2347975E-1,5.7729226E-1,7.664898E-2,0E0,2.3846951E-1,0E0,0E0,2.6079342E-1,4.4884408E-1,1.3104486E-1,1.6905054E-1,0E0,2.1596947E-1,1.1174687E-1,2.27542E-2,7.0788056E-2,8.854896E-2,1.7981271E-1,0E0,9.3980074E-1,2.4288166E-1,6.0070746E-2,3.2037944E-2,2.2498675E-1,2.3118976E-1,0E0,0E0,0E0,0E0,2.551058E-2,0E0,0E0,5.7552237E-2,2.8154224E-2,8.146721E-2,1.5549663E-1,1.155404E-1,1.0053661E-1,0E0,1.04154676E-1,2.0636901E-2,7.979208E-2,6.272356E-2,0E0,0E0,6.346124E-2,2.7554977E-1,2.3650175E-1,4.0819447E-2,0E0,0E0,5.4712027E-2,7.403083E-2,0E0,3.7048206E-2,3.2763362E-2,0E0,5.640457E-2,0E0,1.1856085E-1,0E0,0E0,3.795563E-2,5.9662774E-2,1.2360114E-1,0E0,0E0,3.6628317E-2,0E0,4.70663E-2,1.3318626E-2,4.3747827E-2,6.293526E-2,5.193845E-2,0E0,9.449512E-2,0E0,0E0,2.7516052E-2,2.888468E-2,1.396288E-2,0E0,0E0,3.08972E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,27,27,28,28,29,29,30,30,31,31,32,32,37,37,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,53,53,54,54,55,55,56,56,59,59,60,60,62,62,63,63,65,65,67,67,70,70,71,71,72,72,75,75,77,77,78,78,79,79,80,80,81,81,83,83,86,86,87,87,88,88,91,91],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,-1,26,28,30,32,-1,34,36,38,40,42,44,-1,46,48,50,52,54,56,-1,-1,-1,-1,58,-1,-1,60,62,64,66,68,70,-1,72,74,76,78,-1,-1,80,82,84,86,-1,-1,88,90,-1,92,94,-1,96,-1,98,-1,-1,100,102,104,-1,-1,106,-1,108,110,112,114,116,-1,118,-1,-1,120,122,124,-1,-1,126,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.97E3,9.639872E0,2.871605E2,1.273801E6,2E0,1.273782E4,5.5766084E3,3.72381E5,3.990487E5,1.753E3,1.2380953E1,1.1269331E-2,1.5484655E6,-7.7512197E-4,-1.0020731E-2,3.13173E0,1.584022E0,4.2436344E1,4.306E4,-1.825265E-2,5.599375E2,1.7915686E2,1E0,1.3163265E0,1.7265024E3,7E0,6.7324936E-3,1.68991E6,2.1931148E6,4.313463E-2,8.16E0,4.087912E0,1.11E2,3.553904E-3,-8.498895E-3,-2.6771491E-3,4.46783E-3,1.06314E5,-4.3919904E-4,-4.29793E-3,3.1790426E8,1.853719E1,3.3765998E0,1E0,1.4127854E7,6.18826E5,-2.2424864E-2,1.9052632E0,1.4E1,2.5055911E2,6.516E3,-1.5405503E-3,-6.490511E-3,1.1644444E1,9E1,6.0450394E8,2.2503355E1,-9.146744E-4,-5.1311348E-3,6.6168125E5,1.7101741E2,5.804533E-4,1.587597E6,8.260895E8,2.2939965E-3,2.9417648E2,6.530017E-3,1E0,4.971559E-3,-7.5137955E-3,9.368421E-1,5.81E2,3.1226995E0,5.2433545E-3,1.1470723E-3,3.7777777E0,4.632939E-3,1.4E1,1.2951E4,1.0988551E8,1.088E3,2.5733176E7,-1.10545745E-2,5.4007E7,-3.7580142E-3,1.136957E-3,3.1797794E1,4.320314E8,1.000501E6,1.2007819E-3,-5.352921E-3,1.2825651E0,6.301834E-5,4.8840707E-3,9.38861E-3,2.2728667E-3,-2.5871198E-3,-1.5016998E-3,2.6206864E-4,-3.2328472E-3,1.6553224E-3,-2.8298984E-3,3.3025495E-3,-8.469639E-3,-2.8822292E-3,-2.544059E-3,1.6477305E-3,-3.072216E-4,-2.9861236E-3,3.3832544E-3,2.0423136E-4,1.8947711E-3,5.890565E-3,-1.4755076E-3,2.6512833E-3,-2.6204141E-3,1.5329168E-3,1.2057188E-3,5.522271E-3,-5.875029E-3,-9.748267E-4,2.1306581E-3,-1.7014951E-3,2.6238442E-3,5.973231E-3,1.4198788E-3,5.6652464E-3],"split_indices":[2,63,67,9,26,42,4,9,37,38,67,0,37,0,0,47,62,67,1,0,61,4,25,62,4,3,0,37,37,47,65,65,8,0,0,0,0,1,0,0,41,67,47,92,56,9,0,62,8,4,38,0,0,67,0,7,65,0,0,37,67,0,1,5,0,4,0,109,0,0,63,38,63,0,0,67,0,3,38,7,38,5,0,7,0,0,65,5,37,0,0,63,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.69E2,8.35E2,1.34E2,7.78E2,5.7E1,1.24E2,1E1,4.87E2,2.91E2,1.9E1,3.8E1,7E0,1.17E2,5E0,5E0,3.79E2,1.08E2,1.07E2,1.84E2,8E0,1.1E1,1.7E1,2.1E1,5E1,6.7E1,3.68E2,1.1E1,2.5E1,8.3E1,9.3E1,1.4E1,8.6E1,9.8E1,6E0,5E0,8E0,9E0,1.5E1,6E0,5E0,4.5E1,3.4E1,3.3E1,3.3E1,3.35E2,1.8E1,7E0,7.1E1,1.2E1,2.6E1,6.7E1,5E0,9E0,5.6E1,3E1,8.3E1,1.5E1,5E0,1E1,3.1E1,1.4E1,9E0,2.5E1,2.4E1,9E0,1.9E1,1.4E1,3.28E2,7E0,6E0,1.2E1,1.2E1,5.9E1,7E0,5E0,1.6E1,1E1,5.6E1,1.1E1,2.6E1,3E1,2.4E1,6E0,7.6E1,7E0,5E0,1E1,1.5E1,1.6E1,9E0,5E0,2E1,5E0,1E1,1.4E1,1.1E1,8E0,1.74E2,1.54E2,5E0,7E0,5E0,7E0,1E1,4.9E1,8E0,8E0,2.9E1,2.7E1,6E0,5E0,1.6E1,1E1,1.6E1,1.4E1,1.2E1,1.2E1,1.4E1,6.2E1,5E0,5E0,7E0,8E0,1.1E1,5E0,6E0,1.4E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"127","size_leaf_vector":"1"}},{"base_weights":[-8.093187E-3,-2.3108354E-2,2.7822828E-2,-2.807509E-2,5.042851E-2,-9.702421E-2,3.8759977E-2,-2.4867022E-2,-1.5003145E-1,9.4678596E-2,3.88622E-3,-1.3428244E-1,1.434725E-3,5.310264E-2,-2.1058042E-2,-4.076567E-2,3.0131363E-3,-1.7457481E-2,-1.3003755E-2,1.292944E-3,6.072155E-3,-2.8388884E-3,5.3641085E-2,-9.448579E-3,-4.2661373E-3,1.00442395E-1,3.4809776E-2,-8.6854305E-3,1.0689192E-3,2.0087905E-2,-4.603238E-2,1.2234967E-2,-9.761488E-2,-2.7173962E-3,1.6112244E-3,3.1553188E-4,3.807837E-3,3.4122977E-3,1.2839748E-1,2.2162456E-2,9.573466E-2,3.42275E-2,-9.573269E-2,-3.98481E-2,9.999477E-2,-1.9571662E-2,-6.22467E-2,1.8303774E-2,-5.7216277E-3,2.493451E-4,-1.3478042E-1,3.56222E-3,-4.973897E-3,1.5964176E-1,6.754561E-2,1.3712961E-2,1.1078771E-1,1.3088366E-1,-8.522384E-4,-1.9854605E-2,5.9842408E-2,-9.163485E-4,-6.829312E-3,4.6734842E-3,-6.360915E-3,1.9102118E-3,6.448036E-3,-7.388924E-2,3.3491503E-2,-1.7124659E-1,-5.416458E-2,5.3178854E-3,5.4911613E-2,-9.6080275E-3,-3.322675E-3,4.264484E-3,8.041731E-3,2.9183632E-2,5.483451E-3,-4.848681E-2,2.7678756E-2,7.316006E-3,2.1082142E-3,3.7628633E-3,8.006698E-3,-3.4186265E-3,2.1459137E-3,4.4595897E-3,2.175852E-2,-1.1245067E-3,2.1927361E-3,-2.7309528E-3,-1.0296632E-2,6.9287927E-3,5.5928313E-4,-1.1850799E-3,-1.1207562E-2,1.8752924E-3,-2.8848085E-3,6.2807574E-4,-3.7615225E-3,5.3086705E-3,-6.461492E-4,3.0087812E-3,-5.262588E-4,8.339617E-4,-4.889023E-3,4.1130665E-3,4.4851942E-4,-7.678467E-4,2.6427852E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":67,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,-1,25,27,29,31,-1,33,-1,-1,-1,35,-1,-1,37,39,-1,41,43,45,47,49,-1,-1,-1,-1,51,53,55,57,59,61,63,65,67,69,71,-1,-1,73,-1,-1,75,77,79,81,83,-1,85,87,-1,-1,89,-1,-1,-1,91,93,95,97,99,101,-1,-1,-1,-1,103,-1,105,107,-1,-1,-1,-1,-1,-1,-1,109,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[5.156601E-1,2.4758631E-1,3.8882336E-1,2.4353546E-1,8.819059E-2,1.1379363E-1,2.2418517E-1,2.7361232E-1,4.910648E-1,4.954633E-2,7.450799E-2,3.413084E-2,0E0,1.7835927E-1,1.8767408E-1,1.268804E-1,2.0998858E-1,0E0,2.5954312E-2,0E0,0E0,0E0,1.6750999E-2,0E0,0E0,1.5688604E-1,1.15770385E-1,0E0,1.5092768E-1,1.5724215E-1,1.5409511E-1,1.7253765E-1,7.344934E-2,0E0,0E0,0E0,0E0,1.21831514E-1,7.4774206E-2,9.4332114E-2,1.0667895E-1,4.967393E-2,4.4007875E-2,8.2898974E-2,2.6375785E-2,4.031092E-1,1.880573E-1,9.430967E-2,0E0,0E0,4.910505E-2,0E0,0E0,1.0929644E-2,2.957207E-2,1.03845E-1,2.6541233E-2,2.7504534E-2,0E0,4.610259E-2,3.1060599E-2,0E0,0E0,1.8380225E-2,0E0,0E0,0E0,1.4046761E-1,1.745152E-1,1.4747477E-1,1.6158378E-1,1.0617098E-1,2.1457931E-1,0E0,0E0,0E0,0E0,1.6632749E-2,0E0,8.5061744E-2,1.0566238E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.847132E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,18,18,22,22,25,25,26,26,28,28,29,29,30,30,31,31,32,32,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,47,47,50,50,53,53,54,54,55,55,56,56,57,57,59,59,60,60,63,63,67,67,68,68,69,69,70,70,71,71,72,72,77,77,79,79,80,80,88,88],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,-1,26,28,30,32,-1,34,-1,-1,-1,36,-1,-1,38,40,-1,42,44,46,48,50,-1,-1,-1,-1,52,54,56,58,60,62,64,66,68,70,72,-1,-1,74,-1,-1,76,78,80,82,84,-1,86,88,-1,-1,90,-1,-1,-1,92,94,96,98,100,102,-1,-1,-1,-1,104,-1,106,108,-1,-1,-1,-1,-1,-1,-1,110,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.867288E2,4.5866325E6,1E0,4.5620965E6,1.162E3,1.7500046E8,1.6553869E7,7.582512E7,2.6886544E7,2.0625E0,7.271717E6,1.0479E4,1.434725E-3,9.318287E0,1.2841364E3,1.6386554E0,1E0,-1.7457481E-2,6.009E4,1.292944E-3,6.072155E-3,-2.8388884E-3,2.95E2,-9.448579E-3,-4.2661373E-3,9.558022E7,1.7379E4,-8.6854305E-3,3.85374E8,1.0677966E0,3.1E1,5.5952E4,1.5215946E0,-2.7173962E-3,1.6112244E-3,3.1553188E-4,3.807837E-3,1.798E3,2.440475E5,1.4650772E7,7.3274844E9,1.0656093E2,3.433001E-2,1E0,3.72381E5,4.157143E1,5.862757E1,2.5E1,-5.7216277E-3,2.493451E-4,6.160415E6,3.56222E-3,-4.973897E-3,1.4293194E0,1.6E1,1.9675E4,3.63E2,3.0612E4,-8.522384E-4,3.1991906E0,3.6086957E0,-9.163485E-4,-6.829312E-3,5.44E2,-6.360915E-3,1.9102118E-3,6.448036E-3,2.4595E5,2.1539131E2,3E0,1.7352941E0,1.9E1,3.395185E1,-9.6080275E-3,-3.322675E-3,4.264484E-3,8.041731E-3,1.4846595E6,5.483451E-3,1.4669823E5,1.36659E5,7.316006E-3,2.1082142E-3,3.7628633E-3,8.006698E-3,-3.4186265E-3,2.1459137E-3,4.4595897E-3,1E0,-1.1245067E-3,2.1927361E-3,-2.7309528E-3,-1.0296632E-2,6.9287927E-3,5.5928313E-4,-1.1850799E-3,-1.1207562E-2,1.8752924E-3,-2.8848085E-3,6.2807574E-4,-3.7615225E-3,5.3086705E-3,-6.461492E-4,3.0087812E-3,-5.262588E-4,8.339617E-4,-4.889023E-3,4.1130665E-3,4.4851942E-4,-7.678467E-4,2.6427852E-3],"split_indices":[61,37,16,56,10,12,60,7,12,65,54,9,0,65,4,63,85,0,1,0,0,0,0,0,0,7,2,0,41,62,0,38,63,0,0,0,0,38,42,60,5,67,66,109,9,61,56,8,0,0,12,0,0,62,3,9,0,2,0,47,63,0,0,12,0,0,0,9,42,8,62,8,67,0,0,0,0,37,0,42,1,0,0,0,0,0,0,0,25,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.54E2,6.73E2,2.81E2,6.31E2,4.2E1,2.2E1,2.59E2,6.16E2,1.5E1,2.1E1,2.1E1,1.7E1,5E0,2.09E2,5E1,3.92E2,2.24E2,5E0,1E1,8E0,1.3E1,9E0,1.2E1,5E0,1.2E1,5.7E1,1.52E2,5E0,4.5E1,3.1E1,3.61E2,2.06E2,1.8E1,5E0,5E0,5E0,7E0,1.3E1,4.4E1,1.27E2,2.5E1,3.4E1,1.1E1,1.8E1,1.3E1,1.38E2,2.23E2,1.98E2,8E0,5E0,1.3E1,8E0,5E0,2.8E1,1.6E1,1.17E2,1E1,1.9E1,6E0,1.1E1,2.3E1,5E0,6E0,1.3E1,5E0,6E0,7E0,6.8E1,7E1,1.4E1,2.09E2,1.47E2,5.1E1,5E0,8E0,6E0,2.2E1,1E1,6E0,2.1E1,9.6E1,5E0,5E0,1E1,9E0,6E0,5E0,1.1E1,1.2E1,8E0,5E0,6.3E1,5E0,1E1,6E1,5E0,9E0,1.6E1,1.93E2,1.35E2,1.2E1,2.7E1,2.4E1,5E0,5E0,1E1,1.1E1,2.1E1,7.5E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"111","size_leaf_vector":"1"}},{"base_weights":[4.5327377E-3,-1.7393766E-2,3.519708E-2,-1.0968917E-2,-1.2466722E-1,4.4102944E-2,-8.6607255E-2,-6.3400534E-3,-7.521115E-2,-1.7076564E-1,1.6227607E-3,4.682815E-2,-4.9838484E-3,-1.1213243E-1,-4.0728325E-4,6.0199995E-2,-1.2704807E-2,-2.7774839E-2,-9.949611E-3,-7.803665E-2,-2.6471898E-1,4.067718E-2,1.4311512E-1,-3.8160172E-3,-8.034026E-3,-2.248543E-3,8.952588E-2,-1.9854018E-2,3.0935857E-2,2.3037102E-3,-4.6764906E-2,-5.6889877E-3,-1.8397813E-3,-1.580232E-2,-6.129566E-3,6.152751E-2,1.0132763E-2,1.0762971E-2,1.03338316E-1,8.645418E-3,4.8851736E-2,-1.1693985E-2,-8.638048E-2,5.414742E-2,-5.96408E-3,-7.236904E-2,5.1259765E-5,7.543086E-2,1.6975341E-2,1.1595348E-1,1.3703696E-3,2.7626872E-3,6.2942263E-3,-1.6620652E-3,7.0717625E-2,-3.491806E-2,4.4743624E-3,-1.1643568E-2,-5.8123883E-2,7.213499E-2,1.0513007E-3,-2.908485E-3,2.2727055E-2,-4.411291E-3,-1.5237976E-3,9.9911205E-2,4.2878553E-2,-5.578671E-2,4.8842493E-2,1.0450181E-2,-5.865313E-4,-1.08274564E-1,1.2056295E-2,6.6405616E-4,4.283865E-3,9.825642E-4,-1.97353E-3,-6.5899054E-3,4.0651692E-4,-4.647677E-3,2.5138406E-3,4.3249927E-3,7.2820554E-4,1.7970054E-3,-7.2994624E-4,6.134183E-3,3.369735E-3,-7.6009585E-3,3.0074136E-3,-5.0667394E-3,2.386436E-3,6.532122E-4,4.7543976E-3,-6.636797E-3,-2.30482E-3,-1.379922E-3,1.2278729E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":68,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,-1,23,-1,25,27,29,-1,31,33,35,37,-1,-1,-1,39,41,43,-1,45,-1,-1,-1,-1,47,49,-1,51,-1,53,55,57,59,61,63,-1,65,67,69,71,-1,-1,-1,73,75,77,-1,79,81,-1,-1,83,-1,-1,85,87,89,91,-1,-1,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.548202E-1,3.901071E-1,4.4423765E-1,1.5921006E-1,2.4146664E-1,1.6152233E-1,5.4640844E-2,2.1336737E-1,2.3179606E-1,1.9714713E-1,0E0,2.1509075E-1,0E0,2.4058938E-2,0E0,1.4454748E-1,1.4382051E-1,4.349011E-2,0E0,1.9091628E-2,7.571888E-2,2.2355163E-1,5.3779602E-2,0E0,0E0,0E0,1.2870187E-1,2.1341884E-1,5.5995777E-2,0E0,2.8270148E-2,0E0,0E0,0E0,0E0,1.2817931E-1,1.3336405E-1,0E0,1.5982956E-2,0E0,5.0670628E-2,1.3270593E-1,1.8536231E-1,2.1022573E-2,4.3735705E-2,1.0338932E-2,0E0,1.2242168E-1,1.2097944E-1,1.57412E-1,1.5830751E-1,0E0,0E0,0E0,2.4274118E-2,6.2842354E-2,1.325025E-1,0E0,1.8484402E-1,2.8277487E-2,0E0,0E0,1.1686819E-2,0E0,0E0,6.970322E-2,3.2801902E-1,9.724443E-2,6.566939E-2,0E0,0E0,1.733978E-2,7.485183E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,13,13,15,15,16,16,17,17,19,19,20,20,21,21,22,22,26,26,27,27,28,28,30,30,35,35,36,36,38,38,40,40,41,41,42,42,43,43,44,44,45,45,47,47,48,48,49,49,50,50,54,54,55,55,56,56,58,58,59,59,62,62,65,65,66,66,67,67,68,68,71,71,72,72],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,-1,24,-1,26,28,30,-1,32,34,36,38,-1,-1,-1,40,42,44,-1,46,-1,-1,-1,-1,48,50,-1,52,-1,54,56,58,60,62,64,-1,66,68,70,72,-1,-1,-1,74,76,78,-1,80,82,-1,-1,84,-1,-1,86,88,90,92,-1,-1,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.797342E2,2E0,7.042942E7,1E0,1.75E2,5.619497E7,2.3017536E5,6E0,2.378914E6,5.3E1,1.6227607E-3,3.8776313E3,-4.9838484E-3,3.426009E0,-4.0728325E-4,4.2384106E-1,2.9652428E6,1.6E0,-9.949611E-3,1.0181818E1,6.23E2,8E0,1.1774061E5,-3.8160172E-3,-8.034026E-3,-2.248543E-3,7.122052E7,3.94E2,2.138772E4,2.3037102E-3,3.8181818E0,-5.6889877E-3,-1.8397813E-3,-1.580232E-2,-6.129566E-3,1E0,1.0918E4,1.0762971E-2,5.428706E3,8.645418E-3,1.9376624E0,1.7667647E2,1.2469576E2,3.5364E4,3.440772E7,3.3253515E6,5.1259765E-5,2.0926375E5,2.6037296E-7,1.6673345E7,3E0,2.7626872E-3,6.2942263E-3,-1.6620652E-3,2.9073685E2,2.416E3,2.0966116E2,-1.1643568E-2,1.6566709E6,8E0,1.0513007E-3,-2.908485E-3,4.9321495E6,-4.411291E-3,-1.5237976E-3,1.302273E6,3.9157894E0,6.926E3,2.652E3,1.0450181E-2,-5.865313E-4,5.217E4,1E0,6.6405616E-4,4.283865E-3,9.825642E-4,-1.97353E-3,-6.5899054E-3,4.0651692E-4,-4.647677E-3,2.5138406E-3,4.3249927E-3,7.2820554E-4,1.7970054E-3,-7.2994624E-4,6.134183E-3,3.369735E-3,-7.6009585E-3,3.0074136E-3,-5.0667394E-3,2.386436E-3,6.532122E-4,4.7543976E-3,-6.636797E-3,-2.30482E-3,-1.379922E-3,1.2278729E-3],"split_indices":[4,6,54,85,6,56,42,3,9,0,0,61,0,63,0,65,37,62,0,67,2,26,42,0,0,0,7,0,56,0,65,0,0,0,0,6,9,0,61,0,63,64,61,9,7,54,0,42,46,41,8,0,0,0,61,9,42,0,37,8,0,0,37,0,0,9,65,10,2,0,0,12,109,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.72E2,5.67E2,4.05E2,5.36E2,3.1E1,3.78E2,2.7E1,5.01E2,3.5E1,2.4E1,7E0,3.72E2,6E0,2E1,7E0,4.3E1,4.58E2,2.7E1,8E0,1.3E1,1.1E1,3.51E2,2.1E1,1.5E1,5E0,9E0,3.4E1,3.94E2,6.4E1,5E0,2.2E1,5E0,8E0,6E0,5E0,2.08E2,1.43E2,5E0,1.6E1,9E0,2.5E1,3.52E2,4.2E1,3.9E1,2.5E1,1.4E1,8E0,1.58E2,5E1,1E1,1.33E2,8E0,8E0,5E0,2E1,1.44E2,2.08E2,5E0,3.7E1,2.4E1,1.5E1,8E0,1.7E1,8E0,6E0,8.9E1,6.9E1,1.5E1,3.5E1,5E0,5E0,1.1E1,1.22E2,6E0,1.4E1,1.7E1,1.27E2,5E0,2.03E2,2.7E1,1E1,1.7E1,7E0,1.2E1,5E0,3.9E1,5E1,6E0,6.3E1,1E1,5E0,2.2E1,1.3E1,6E0,5E0,3.1E1,9.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-3.2629524E-4,5.9016845E-3,-1.00857854E-1,-1.0308241E-2,4.7882557E-2,-2.288358E-1,-4.394578E-2,-4.4171013E-2,1.0464594E-3,6.775809E-2,-2.9864859E-2,9.40335E-4,-3.3431712E-1,5.9028533E-3,-8.410841E-2,-5.7452694E-2,3.1973224E-2,6.814822E-2,-6.733623E-3,7.2897136E-2,-5.1693753E-3,-5.6423448E-2,4.0517612E-3,-2.103913E-2,-8.558656E-3,-1.1644515E-1,6.988324E-4,-4.6720605E-2,-9.673023E-3,-3.8001067E-3,7.2243676E-2,4.8068356E-2,8.699776E-3,-1.6716361E-3,-1.3258943E-1,1.5536462E-1,6.419707E-2,1.9419862E-2,-9.774736E-2,-1.6664366E-1,-4.817644E-2,-7.832673E-3,-3.8533133E-2,4.85066E-3,6.851609E-4,6.4543174E-3,3.04962E-2,6.6682324E-3,-7.572182E-2,-1.5376125E-2,-2.2141889E-2,3.3788742E-3,1.9056772E-1,4.904763E-2,1.0821879E-1,-2.139435E-3,3.5036223E-3,-1.2588471E-3,-1.354716E-1,-1.163438E-2,-3.3512095E-3,1.4678772E-3,-5.0902227E-3,-2.941371E-2,-1.1483789E-1,1.6138857E-2,4.9541467E-3,1.3558708E-2,-6.6614635E-2,-1.1897196E-1,2.2365738E-2,-2.956223E-3,1.3941859E-3,5.0969147E-3,1.1738055E-2,6.432814E-2,-6.57842E-3,8.337767E-3,7.356587E-2,-2.6215147E-3,-9.658808E-3,9.3131827E-4,-1.9301132E-3,-2.0647491E-3,-7.110104E-3,4.0373118E-3,-1.000303E-4,1.7514572E-3,-1.7327888E-4,-6.4985844E-4,-7.3085115E-3,-6.533568E-3,-1.2573168E-3,-1.5563817E-3,4.756005E-3,4.3637455E-3,6.9220137E-4,-4.3671913E-3,1.3703491E-3,4.1047074E-3,1.4820927E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":69,"left_children":[1,3,5,7,9,11,13,15,17,19,21,-1,23,-1,25,27,29,31,33,35,-1,37,-1,-1,-1,39,-1,41,-1,-1,43,45,-1,47,49,51,53,55,57,59,61,-1,63,-1,-1,-1,65,67,69,-1,71,-1,73,75,77,-1,-1,-1,79,-1,-1,-1,-1,81,83,85,-1,87,89,91,93,-1,-1,-1,-1,95,97,-1,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.010333E-1,6.156717E-1,3.9781922E-1,2.5112578E-1,3.914321E-1,4.5346034E-1,2.8567612E-1,1.6769397E-1,2.562669E-1,1.9319326E-1,1.6676979E-1,0E0,1.3057518E-1,0E0,1.0757898E-1,2.1716684E-1,1.2055981E-1,1.13209754E-1,2.8009376E-1,1.3065064E-1,0E0,1.3666637E-1,0E0,0E0,0E0,7.7646405E-2,0E0,1.2544265E-1,0E0,0E0,3.372751E-2,6.7743614E-2,0E0,2.623541E-1,3.6592504E-1,3.842649E-2,1.1406791E-1,6.229023E-2,7.091546E-2,9.3058825E-2,6.239879E-2,0E0,8.360411E-2,0E0,0E0,0E0,4.098629E-2,1.9379558E-1,1.8507978E-1,0E0,2.7948951E-2,0E0,3.2663584E-2,1.15172416E-1,9.824014E-2,0E0,0E0,0E0,9.157547E-2,0E0,0E0,0E0,0E0,6.896668E-2,2.8293148E-2,4.42722E-2,0E0,1.4654611E-1,1.5560205E-1,5.637902E-2,6.616238E-2,0E0,0E0,0E0,0E0,1.5425012E-1,9.762115E-2,0E0,1.7018884E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,14,14,15,15,16,16,17,17,18,18,19,19,21,21,25,25,27,27,30,30,31,31,33,33,34,34,35,35,36,36,37,37,38,38,39,39,40,40,42,42,46,46,47,47,48,48,50,50,52,52,53,53,54,54,58,58,63,63,64,64,65,65,67,67,68,68,69,69,70,70,75,75,76,76,78,78],"right_children":[2,4,6,8,10,12,14,16,18,20,22,-1,24,-1,26,28,30,32,34,36,-1,38,-1,-1,-1,40,-1,42,-1,-1,44,46,-1,48,50,52,54,56,58,60,62,-1,64,-1,-1,-1,66,68,70,-1,72,-1,74,76,78,-1,-1,-1,80,-1,-1,-1,-1,82,84,86,-1,88,90,92,94,-1,-1,-1,-1,96,98,-1,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[9.932432E0,5.8222644E2,1.753E3,1.5961905E2,1E0,1E0,4.911E3,2.331083E6,1.552356E0,5.619497E7,3.2645503E5,9.40335E-4,6.34E2,5.9028533E-3,7.3835544E5,4.3992E4,6.228E1,4.7185095E2,4.22718E6,1.2083E4,-5.1693753E-3,1.84146E5,4.0517612E-3,-2.103913E-2,-8.558656E-3,3.7725E1,6.988324E-4,6.25E0,-9.673023E-3,-3.8001067E-3,5.465992E2,1.862069E0,8.699776E-3,1.9734E4,4.2708197E2,1.0838E4,1.4176E4,1.7743998E5,4E0,1.63E2,7.44488E5,-7.832673E-3,6.4921465E0,4.85066E-3,6.851609E-4,6.4543174E-3,2.2E1,1E0,5.7254577E0,-1.5376125E-2,1.8303784E7,3.3788742E-3,1.2739792E1,5.0741018E8,2.2503355E1,-2.139435E-3,3.5036223E-3,-1.2588471E-3,3.1561086E0,-1.163438E-2,-3.3512095E-3,1.4678772E-3,-5.0902227E-3,3.7289145E4,3.3817584E7,6.69E2,4.9541467E-3,1E0,6.763312E7,2.194E3,3.4779343E1,-2.956223E-3,1.3941859E-3,5.0969147E-3,1.1738055E-2,8E0,5.97E3,8.337767E-3,2.1609572E7,-2.6215147E-3,-9.658808E-3,9.3131827E-4,-1.9301132E-3,-2.0647491E-3,-7.110104E-3,4.0373118E-3,-1.000303E-4,1.7514572E-3,-1.7327888E-4,-6.4985844E-4,-7.3085115E-3,-6.533568E-3,-1.2573168E-3,-1.5563817E-3,4.756005E-3,4.3637455E-3,6.9220137E-4,-4.3671913E-3,1.3703491E-3,4.1047074E-3,1.4820927E-3],"split_indices":[63,61,38,64,6,8,38,37,63,56,42,0,2,0,37,1,56,4,56,9,0,1,0,0,0,65,0,61,0,0,42,65,0,38,61,9,2,42,8,0,9,0,63,0,0,0,8,85,63,0,12,0,65,7,65,0,0,0,55,0,0,0,0,54,7,2,0,23,7,2,65,0,0,0,0,26,2,0,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.58E2,9.03E2,5.5E1,6.52E2,2.51E2,1.6E1,3.9E1,1.63E2,4.89E2,2E2,5.1E1,5E0,1.1E1,7E0,3.2E1,1.39E2,2.4E1,5E1,4.39E2,1.95E2,5E0,4.2E1,9E0,5E0,6E0,2.4E1,8E0,1.31E2,8E0,6E0,1.8E1,4.4E1,6E0,4.23E2,1.6E1,1.7E1,1.78E2,1.5E1,2.7E1,1.3E1,1.1E1,7E0,1.24E2,1.1E1,7E0,6E0,3.8E1,3.81E2,4.2E1,5E0,1.1E1,6E0,1.1E1,1.34E2,4.4E1,7E0,8E0,1E1,1.7E1,6E0,7E0,5E0,6E0,1.12E2,1.2E1,3.3E1,5E0,3.49E2,3.2E1,2.9E1,1.3E1,6E0,5E0,6E0,5E0,1.05E2,2.9E1,1.3E1,3.1E1,9E0,8E0,2.2E1,9E1,5E0,7E0,6E0,2.7E1,1.45E2,2.04E2,2.1E1,1.1E1,2.3E1,6E0,8E0,5E0,6.5E1,4E1,8E0,2.1E1,2.2E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[8.095965E-5,-1.4265704E-2,3.592085E-2,-1.1564438E-2,-1.1076165E-2,-2.558423E-2,5.1344164E-2,-2.9248582E-2,1.4419158E-2,-5.021619E-2,7.7508725E-2,7.1925186E-3,4.3430965E-2,-3.270125E-2,9.388987E-3,3.6856234E-2,-1.653282E-2,2.3478579E-2,-1.04973085E-1,1.549836E-3,5.0434885E-3,5.2032087E-2,-6.7838565E-2,-2.340171E-2,-1.217968E-1,1.4439585E-2,8.79269E-2,-3.9494164E-2,5.9766907E-2,-1.1743159E-3,3.0482728E-3,-2.3452683E-1,5.668116E-3,6.3939705E-2,-1.586996E-2,-1.1241692E-4,-5.1586796E-3,-2.6550926E-2,4.1224104E-3,-7.1151994E-2,-1.3756969E-2,5.916118E-2,-1.2599652E-2,3.307045E-2,1.3707203E-1,-5.1589407E-2,3.5184883E-2,-8.5392484E-4,9.987469E-2,-4.506292E-3,-1.650647E-2,3.383504E-3,-1.6356066E-3,6.873255E-2,-2.3812999E-3,-5.5960886E-2,4.4712243E-3,-2.0147318E-2,-1.677948E-1,-1.2244026E-1,2.882568E-3,-2.3894007E-3,7.903456E-2,-9.335449E-2,1.3056495E-2,5.040512E-3,-1.1500459E-2,4.0188367E-3,7.912533E-3,-6.0080983E-2,8.929486E-4,-1.1064253E-3,5.012442E-3,1.4932468E-3,6.0801283E-3,8.5037155E-3,6.3692756E-2,-9.0515204E-2,7.3420437E-4,2.0605542E-3,-1.1214461E-3,-1.0598361E-2,-3.0569183E-3,-8.874971E-3,2.43059E-3,6.2693325E-3,2.4251912E-3,-3.1742504E-5,-6.6740797E-3,3.1574569E-3,-1.7027566E-3,1.0824574E-3,-1.6115032E-3,-2.044398E-3,-4.9411585E-3,1.3029219E-3,3.6071711E-3,-1.4647549E-3,-5.8604986E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":70,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,-1,-1,33,35,37,39,41,43,45,47,-1,-1,49,51,53,55,-1,-1,57,-1,59,-1,61,63,65,67,69,71,-1,73,-1,-1,-1,-1,75,-1,77,-1,79,81,83,-1,-1,85,87,89,-1,91,-1,-1,93,-1,-1,-1,-1,-1,-1,95,97,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.8744363E-1,4.0769994E-1,2.583875E-1,3.0828518E-1,0E0,1.4355819E-1,1.7128438E-1,3.256728E-1,1.8950571E-1,1.8405294E-1,1.1501133E-2,0E0,1.9780943E-1,3.2359764E-1,0E0,1.7961401E-1,2.0383406E-1,4.2856667E-2,3.7401286E-1,0E0,0E0,1.5383106E-1,4.2258687E-2,1.2890947E-1,3.062548E-1,1.3497081E-1,1.2487149E-1,8.2205385E-2,8.64899E-2,0E0,0E0,1.6532904E-1,4.384922E-2,9.246874E-2,1.3587974E-1,0E0,0E0,3.1227958E-1,0E0,2.1267319E-1,0E0,9.643385E-2,1.4624895E-1,8.207885E-2,2.7436405E-2,4.747133E-2,5.8475733E-2,0E0,3.3755243E-2,0E0,0E0,0E0,0E0,7.7801585E-2,0E0,5.600246E-2,0E0,8.805661E-2,7.762164E-2,2.747613E-1,0E0,0E0,4.69788E-2,7.943593E-2,1.5024881E-1,0E0,1.3756073E-2,0E0,0E0,4.6609864E-2,0E0,0E0,0E0,0E0,0E0,0E0,7.2864294E-2,2.7658567E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,21,21,22,22,23,23,24,24,25,25,26,26,27,27,28,28,31,31,32,32,33,33,34,34,37,37,39,39,41,41,42,42,43,43,44,44,45,45,46,46,48,48,53,53,55,55,57,57,58,58,59,59,62,62,63,63,64,64,66,66,69,69,76,76,77,77],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,-1,-1,34,36,38,40,42,44,46,48,-1,-1,50,52,54,56,-1,-1,58,-1,60,-1,62,64,66,68,70,72,-1,74,-1,-1,-1,-1,76,-1,78,-1,80,82,84,-1,-1,86,88,90,-1,92,-1,-1,94,-1,-1,-1,-1,-1,-1,96,98,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.7831801E6,1.7357699E6,2.4568965E2,9.558022E7,-1.1076165E-2,2.5555556E0,1.1057693E0,9.50283E5,3.63E2,6.15E2,5.5177975E6,7.1925186E-3,2.3821254E7,5.119403E0,9.388987E-3,9.751103E6,2.234478E3,3.8795E4,8.1E2,1.549836E-3,5.0434885E-3,5.1071826E-8,1.6896813E0,9.149E3,4.9E1,1.9452055E0,7.4654E4,2.0094643E0,1E1,-1.1743159E-3,3.0482728E-3,6.03383E2,9.14E2,4.9188764E7,7.84892E5,-1.1241692E-4,-5.1586796E-3,1.70595E5,4.1224104E-3,1.2784314E1,-1.3756969E-2,5.8746633E4,2.2E0,8.315789E0,1.0420099E9,1.1806228E6,2.03E2,-8.5392484E-4,2.4081968E3,-4.506292E-3,-1.650647E-2,3.383504E-3,-1.6356066E-3,2.2404422E2,-2.3812999E-3,2.3447667E5,4.4712243E-3,1.6166983E0,1.8645384E1,5.1586456E7,2.882568E-3,-2.3894007E-3,3.4698796E0,2E0,4.612245E0,5.040512E-3,9E0,4.0188367E-3,7.912533E-3,3.714144E8,8.929486E-4,-1.1064253E-3,5.012442E-3,1.4932468E-3,6.0801283E-3,8.5037155E-3,5.441268E7,1.72797E5,7.3420437E-4,2.0605542E-3,-1.1214461E-3,-1.0598361E-2,-3.0569183E-3,-8.874971E-3,2.43059E-3,6.2693325E-3,2.4251912E-3,-3.1742504E-5,-6.6740797E-3,3.1574569E-3,-1.7027566E-3,1.0824574E-3,-1.6115032E-3,-2.044398E-3,-4.9411585E-3,1.3029219E-3,3.6071711E-3,-1.4647549E-3,-5.8604986E-3],"split_indices":[37,37,64,7,0,62,63,1,0,2,54,0,9,62,0,9,4,1,2,0,0,46,62,10,3,62,1,51,3,0,0,42,2,53,9,0,0,1,0,62,0,42,62,67,40,37,8,0,61,0,0,0,0,61,0,42,0,63,65,7,0,0,65,8,63,0,3,0,0,7,0,0,0,0,0,0,7,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.46E2,6.76E2,2.7E2,6.69E2,7E0,5.4E1,2.16E2,3.98E2,2.71E2,4.4E1,1E1,1.4E1,2.02E2,3.93E2,5E0,1.57E2,1.14E2,1.9E1,2.5E1,5E0,5E0,1.88E2,1.4E1,3.57E2,3.6E1,1.1E2,4.7E1,8.8E1,2.6E1,9E0,1E1,1.1E1,1.4E1,1.6E2,2.8E1,6E0,8E0,3.48E2,9E0,2.9E1,7E0,4.1E1,6.9E1,2.3E1,2.4E1,7.6E1,1.2E1,9E0,1.7E1,6E0,5E0,5E0,9E0,1.54E2,6E0,2.1E1,7E0,3.34E2,1.4E1,2.1E1,8E0,6E0,3.5E1,1.6E1,5.3E1,8E0,1.5E1,1.1E1,1.3E1,6.8E1,8E0,7E0,5E0,6E0,1.1E1,5E0,1.49E2,1.4E1,7E0,1.9E1,3.15E2,8E0,6E0,1.5E1,6E0,1E1,2.5E1,6E0,1E1,2.5E1,2.8E1,6E0,9E0,5.2E1,1.6E1,4.3E1,1.06E2,6E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"99","size_leaf_vector":"1"}},{"base_weights":[1.9716858E-3,-1.5129327E-2,2.6279509E-2,-1.949847E-2,6.5995514E-2,1.4212206E-3,3.8562585E-2,-1.3334307E-2,-8.473631E-2,1.1821832E-1,-1.783832E-2,-9.488269E-2,1.3651199E-2,1.7056614E-1,2.8826186E-2,-1.1248616E-2,-8.338219E-3,-1.8870763E-2,-1.9598514E-1,3.0166532E-3,7.618682E-3,1.8028026E-3,-3.7586445E-3,-2.1596844E-3,-6.6571916E-3,4.310345E-2,-1.55377E-2,4.2336513E-3,1.0435215E-2,4.035431E-2,-3.594384E-2,-1.3952903E-2,1.0655004E-1,4.415076E-2,-1.17082275E-1,-1.6303262E-2,-7.9288356E-2,5.394167E-3,1.0189214E-1,-4.0968355E-2,2.4315296E-2,6.884441E-2,1.3569365E-2,-1.3175484E-2,1.023886E-2,-1.0753013E-2,-1.1136599E-1,-9.459081E-4,1.0009647E-2,-1.0671192E-3,6.977541E-2,-8.521142E-3,-9.2376355E-4,4.201162E-4,-7.164826E-3,5.4249845E-2,-2.0226553E-2,3.6657847E-2,8.492526E-3,3.531177E-2,-7.05281E-2,4.272693E-3,-9.006679E-3,9.430295E-3,5.680206E-2,-2.7330546E-2,4.022648E-2,7.968889E-2,-3.324158E-2,3.524767E-3,-3.86905E-2,-8.143379E-3,-2.8649454E-3,1.2844315E-3,5.0071534E-3,5.011168E-3,-1.2277095E-3,-4.3147344E-2,2.0822445E-3,2.6893602E-3,3.2825026E-4,3.8862797E-3,-8.8246376E-4,-8.827158E-2,-3.3510596E-4,-2.8525984E-3,1.571641E-3,6.5307304E-2,-3.7232533E-3,5.414798E-3,-9.099919E-2,5.5331676E-3,2.427044E-2,5.6158076E-3,1.7966096E-3,-7.587869E-2,2.9172434E-3,5.291398E-4,-4.154215E-3,-2.890409E-3,-5.738691E-4,1.0337745E-3,-3.1600613E-3,-1.6108282E-3,-4.9823285E-3,1.5374839E-3,5.032451E-3,1.3296803E-3,-3.482147E-3,-5.8597554E-3,-7.895122E-4,4.206585E-4,4.84193E-3,-6.243248E-3,-1.53514E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":71,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,-1,-1,-1,-1,-1,-1,37,39,-1,-1,41,43,45,47,49,51,-1,53,55,57,59,61,63,65,-1,67,69,71,-1,-1,-1,73,-1,-1,-1,-1,75,77,79,-1,81,83,-1,85,-1,87,89,91,93,95,97,99,-1,-1,-1,-1,-1,-1,101,-1,-1,-1,-1,-1,103,-1,-1,-1,105,-1,107,109,-1,111,-1,-1,113,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.9862704E-1,2.0083989E-1,1.2085834E-1,2.1401322E-1,1.2907194E-1,1.5676415E-1,3.3567154E-1,1.6800782E-1,3.33365E-1,3.33575E-2,4.6121098E-2,2.8554663E-2,1.02111824E-1,5.7057142E-2,1.866264E-1,1.5598896E-1,0E0,1.9085243E-1,2.8039598E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.3024627E-1,6.1805423E-2,0E0,0E0,1.6008112E-1,4.381581E-1,1.4686528E-1,1.5684286E-1,3.4297615E-2,7.396433E-2,0E0,7.4405245E-2,4.7410842E-2,1.15030214E-1,8.5918844E-2,5.5026732E-2,1.5353829E-1,1.2102951E-1,0E0,1.0229968E-1,1.8402249E-1,3.690347E-2,0E0,0E0,0E0,1.9488141E-2,0E0,0E0,0E0,0E0,5.9254266E-2,3.924166E-2,8.413073E-3,0E0,3.0545685E-2,2.9690698E-2,0E0,4.0331624E-2,0E0,1.15973055E-1,9.222624E-2,8.158709E-2,1.7332397E-2,9.082918E-2,2.2538486E-1,9.6153125E-2,0E0,0E0,0E0,0E0,0E0,0E0,3.2543488E-2,0E0,0E0,0E0,0E0,0E0,1.7761588E-2,0E0,0E0,0E0,1.2134999E-1,0E0,5.825475E-2,3.7014626E-2,0E0,6.845783E-2,0E0,0E0,3.3212222E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,17,17,18,18,25,25,26,26,29,29,30,30,31,31,32,32,33,33,34,34,36,36,37,37,38,38,39,39,40,40,41,41,42,42,44,44,45,45,46,46,50,50,55,55,56,56,57,57,59,59,60,60,62,62,64,64,65,65,66,66,67,67,68,68,69,69,70,70,77,77,83,83,87,87,89,89,90,90,92,92,95,95],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,-1,-1,-1,-1,-1,-1,38,40,-1,-1,42,44,46,48,50,52,-1,54,56,58,60,62,64,66,-1,68,70,72,-1,-1,-1,74,-1,-1,-1,-1,76,78,80,-1,82,84,-1,86,-1,88,90,92,94,96,98,100,-1,-1,-1,-1,-1,-1,102,-1,-1,-1,-1,-1,104,-1,-1,-1,106,-1,108,110,-1,112,-1,-1,114,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1E0,1.3631968E3,3.8913777E-1,9.639872E0,3.1991906E0,1.2391878E6,1.2083E4,4.087E3,3.9401392E6,4.48375E5,6.926E3,2.8524572E-1,1.5310831E7,3.4E3,1.7493458E7,4.1E1,-8.338219E-3,1.312E1,4.75E0,3.0166532E-3,7.618682E-3,1.8028026E-3,-3.7586445E-3,-2.1596844E-3,-6.6571916E-3,1.6100003E8,2.3E1,4.2336513E-3,1.0435215E-2,3.464E0,3.167E3,3.1E1,5E0,5.869281E0,3.2921512E2,-1.6303262E-2,1.3E1,9.3E1,1.137863E6,1E0,4.7E1,3.64E2,3.8223077E2,-1.3175484E-2,2.8367348E0,1E0,2.0549193E2,-9.459081E-4,1.0009647E-2,-1.0671192E-3,5.8114325E5,-8.521142E-3,-9.2376355E-4,4.201162E-4,-7.164826E-3,1.5E1,9.8933E4,8.169029E2,8.492526E-3,4.75356E6,7.0296685E6,4.272693E-3,5.431429E2,9.430295E-3,2.8E1,1.01036356E6,1.0092541E8,1.5583066E0,1.01E3,2.936348E10,1.084437E6,-8.143379E-3,-2.8649454E-3,1.2844315E-3,5.0071534E-3,5.011168E-3,-1.2277095E-3,2.2051822E2,2.0822445E-3,2.6893602E-3,3.2825026E-4,3.8862797E-3,-8.8246376E-4,3.9481482E2,-3.3510596E-4,-2.8525984E-3,1.571641E-3,1.6964285E0,-3.7232533E-3,4.6303625E5,2.847E4,5.5331676E-3,1.0028931E7,5.6158076E-3,1.7966096E-3,6.714162E8,2.9172434E-3,5.291398E-4,-4.154215E-3,-2.890409E-3,-5.738691E-4,1.0337745E-3,-3.1600613E-3,-1.6108282E-3,-4.9823285E-3,1.5374839E-3,5.032451E-3,1.3296803E-3,-3.482147E-3,-5.8597554E-3,-7.895122E-4,4.206585E-4,4.84193E-3,-6.243248E-3,-1.53514E-3],"split_indices":[24,61,51,63,47,54,9,2,54,1,10,51,12,38,60,3,0,67,67,0,0,0,0,0,0,5,8,0,0,63,2,3,8,62,61,0,8,0,9,8,8,10,61,0,63,23,4,0,0,0,54,0,0,0,0,3,1,42,0,56,56,0,61,0,65,60,7,51,8,40,9,0,0,0,0,0,0,4,0,0,0,0,0,61,0,0,0,62,0,37,10,0,54,0,0,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.57E2,5.62E2,3.95E2,5.34E2,2.8E1,1.31E2,2.64E2,4.89E2,4.5E1,1.7E1,1.1E1,1.4E1,1.17E2,1.7E1,2.47E2,4.84E2,5E0,2.9E1,1.6E1,9E0,8E0,6E0,5E0,8E0,6E0,5.8E1,5.9E1,8E0,9E0,2.1E2,3.7E1,4.74E2,1E1,1.8E1,1.1E1,6E0,1E1,3.6E1,2.2E1,3.6E1,2.3E1,1.01E2,1.09E2,5E0,3.2E1,4.6E2,1.4E1,5E0,5E0,5E0,1.3E1,6E0,5E0,5E0,5E0,1.2E1,2.4E1,1.3E1,9E0,1E1,2.6E1,7E0,1.6E1,7E0,9.4E1,4.3E1,6.6E1,1.2E1,2E1,3.05E2,1.55E2,5E0,9E0,7E0,6E0,7E0,5E0,1.8E1,6E0,7E0,6E0,5E0,5E0,2E1,6E0,7E0,9E0,8.9E1,5E0,2.9E1,1.4E1,1E1,5.6E1,5E0,7E0,1.4E1,6E0,2.82E2,2.3E1,8.1E1,7.4E1,5E0,1.3E1,6E0,1.4E1,5.2E1,3.7E1,2.3E1,6E0,9E0,5E0,4.8E1,8E0,5E0,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"115","size_leaf_vector":"1"}},{"base_weights":[3.287716E-4,-2.9062971E-2,2.1857502E-2,-7.906146E-2,-1.9357538E-2,2.9777562E-2,-5.904747E-2,-4.3507796E-2,-2.3542733E-1,3.9286852E-2,-2.9499197E-2,1.3899891E-1,2.5249537E-2,-3.1810973E-2,-1.176132E-2,-7.8481585E-2,7.552149E-2,-2.3068464E-3,-1.9156054E-2,-3.1370064E-3,5.2375697E-2,-3.6080938E-2,6.731544E-2,2.6807045E-3,1.7829092E-1,2.1566946E-2,6.504342E-3,-4.3132287E-2,1.661532E-3,-5.4356214E-2,-9.038128E-3,6.164408E-3,-6.268815E-4,3.2141246E-2,4.69876E-3,-5.7195164E-2,-1.8781143E-3,1.2729955E-1,-2.4916532E-3,4.3012546E-3,1.106213E-2,5.8000865E-3,1.815455E-2,-1.6227815E-2,-9.569293E-2,-8.7854385E-2,3.280804E-2,5.4056663E-2,-2.789163E-3,-4.4400096E-2,-1.3318184E-1,5.3902073E-3,-1.1398365E-2,1.1565199E-3,8.730872E-3,-9.494146E-3,2.99967E-2,3.3707567E-2,-4.7584366E-2,-2.2418103E-3,-5.5337967E-3,-7.1718375E-4,-1.0780339E-1,3.1433825E-3,-3.525479E-4,5.2400795E-3,3.286578E-2,-3.504792E-2,-1.138551E-1,-1.8338616E-1,-4.570317E-4,-4.3911304E-2,2.5507681E-2,3.0180985E-2,-3.2830656E-2,3.254019E-2,-5.0950143E-3,3.324636E-3,-4.572826E-4,1.288769E-5,-3.420389E-3,-6.768886E-3,-2.7804116E-3,2.4132372E-3,-1.2460125E-3,-2.031558E-3,1.4142619E-3,-9.826887E-3,-1.1192651E-3,-3.401729E-3,-1.2670841E-2,1.7461478E-4,-2.9987497E-3,3.368575E-3,-7.1085425E-4,5.5767354E-3,2.7322987E-4,-2.8329974E-3,4.4712378E-4,2.1499328E-3,1.5286761E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":72,"left_children":[1,3,5,7,9,11,13,15,17,19,21,23,25,27,-1,29,31,-1,-1,-1,33,35,37,-1,39,41,-1,43,-1,45,-1,-1,-1,47,-1,49,51,53,-1,-1,-1,-1,55,57,59,61,63,65,-1,67,69,-1,71,-1,-1,73,75,77,79,-1,-1,-1,81,-1,-1,-1,83,85,87,89,-1,91,93,95,97,99,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[6.106225E-1,1.9645444E-1,3.586685E-1,3.5667616E-1,2.051587E-1,2.4772528E-1,2.567196E-1,2.3488683E-1,3.7486738E-1,7.489721E-2,1.8913099E-1,5.3528666E-2,2.039448E-1,3.5994317E-2,0E0,1.1006901E-1,6.8154074E-2,0E0,0E0,0E0,4.2253047E-2,1.984883E-1,1.4475717E-1,0E0,4.479325E-2,1.6538966E-1,0E0,5.359766E-2,0E0,1.1091652E-1,0E0,0E0,0E0,7.198962E-2,0E0,1.5937006E-1,1.2045174E-1,7.6317236E-2,0E0,0E0,0E0,0E0,1.5122539E-1,4.3880586E-2,8.755505E-3,3.7328765E-2,1.6043622E-2,3.1223007E-2,0E0,9.149426E-2,1.467967E-1,0E0,1.1991254E-1,0E0,0E0,1.2968804E-1,1.18429154E-1,1.8889103E-2,2.1282822E-2,0E0,0E0,0E0,2.954647E-2,0E0,0E0,0E0,2.6549177E-2,7.632783E-2,1.442851E-1,1.4782041E-1,0E0,5.2591883E-2,9.147757E-2,1.129915E-1,1.06072195E-1,1.281437E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,15,15,16,16,20,20,21,21,22,22,24,24,25,25,27,27,29,29,33,33,35,35,36,36,37,37,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,52,52,55,55,56,56,57,57,58,58,62,62,66,66,67,67,68,68,69,69,71,71,72,72,73,73,74,74,75,75],"right_children":[2,4,6,8,10,12,14,16,18,20,22,24,26,28,-1,30,32,-1,-1,-1,34,36,38,-1,40,42,-1,44,-1,46,-1,-1,-1,48,-1,50,52,54,-1,-1,-1,-1,56,58,60,62,64,66,-1,68,70,-1,72,-1,-1,74,76,78,80,-1,-1,-1,82,-1,-1,-1,84,86,88,90,-1,92,94,96,98,100,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.700023E2,1.3151775E2,3.5056704E7,2.2E1,4.7097144E2,1.0918E4,4.4E1,2.25E2,5.04E2,1.00037E5,2.4E1,1.7E1,8.2471845E6,1E0,-1.176132E-2,1.02E2,5.539245E6,-2.3068464E-3,-1.9156054E-2,-3.1370064E-3,7.69E2,6.946813E7,3.66025E5,2.6807045E-3,3.299862E1,1.4E1,6.504342E-3,3.714144E8,1.661532E-3,3.23E2,-9.038128E-3,6.164408E-3,-6.268815E-4,2.9E1,4.69876E-3,1.8E1,1.8738E4,6.101E3,-2.4916532E-3,4.3012546E-3,1.106213E-2,5.8000865E-3,3.88E2,1.7E1,1.1E1,1.11E2,2.416E3,2.3603575E5,-2.789163E-3,2.78E2,3.92E2,5.3902073E-3,4.39776E6,1.1565199E-3,8.730872E-3,4.2E1,2.871605E2,3.583621E1,1.3809524E0,-2.2418103E-3,-5.5337967E-3,-7.1718375E-4,8.071428E0,3.1433825E-3,-3.525479E-4,5.2400795E-3,1.0201707E-3,5.6873284E7,1E1,5.4E1,-4.570317E-4,1.31306E5,9.912121E0,1.8949389E6,1.585814E6,4.3E1,-5.0950143E-3,3.324636E-3,-4.572826E-4,1.288769E-5,-3.420389E-3,-6.768886E-3,-2.7804116E-3,2.4132372E-3,-1.2460125E-3,-2.031558E-3,1.4142619E-3,-9.826887E-3,-1.1192651E-3,-3.401729E-3,-1.2670841E-2,1.7461478E-4,-2.9987497E-3,3.368575E-3,-7.1085425E-4,5.5767354E-3,2.7322987E-4,-2.8329974E-3,4.4712378E-4,2.1499328E-3,1.5286761E-4],"split_indices":[61,42,54,3,42,9,3,2,7,5,3,3,37,92,0,2,5,0,0,0,2,7,9,0,65,10,0,7,0,1,0,0,0,8,0,3,9,9,0,0,0,0,10,3,8,1,9,54,0,0,10,0,9,0,0,0,67,65,63,0,0,0,61,0,0,0,47,5,3,0,0,9,67,54,9,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.63E2,4.07E2,5.56E2,6.5E1,3.42E2,5.07E2,4.9E1,5.4E1,1.1E1,5E1,2.92E2,1.9E1,4.88E2,4.4E1,5E0,4.2E1,1.2E1,6E0,5E0,5E0,4.5E1,2.74E2,1.8E1,7E0,1.2E1,4.74E2,1.4E1,3.8E1,6E0,3.6E1,6E0,7E0,5E0,3.3E1,1.2E1,1.69E2,1.05E2,1.2E1,6E0,6E0,6E0,1.4E1,4.6E2,2.6E1,1.2E1,2.6E1,1E1,2.7E1,6E0,1.46E2,2.3E1,7E0,9.8E1,5E0,7E0,1.38E2,3.22E2,1E1,1.6E1,5E0,7E0,6E0,2E1,5E0,5E0,6E0,2.1E1,1.3E2,1.6E1,1.6E1,7E0,5.2E1,4.6E1,5.1E1,8.7E1,3.17E2,5E0,5E0,5E0,6E0,1E1,1E1,1E1,1.6E1,5E0,1.15E2,1.5E1,7E0,9E0,8E0,8E0,1.6E1,3.6E1,2.1E1,2.5E1,1E1,4.1E1,5.2E1,3.5E1,2.15E2,1.02E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"101","size_leaf_vector":"1"}},{"base_weights":[-7.816402E-3,-2.5197875E-2,1.590042E-2,-2.2606913E-2,-8.599706E-3,3.0545583E-2,-4.788681E-2,5.677401E-3,-3.9307427E-2,3.3641875E-2,-4.793845E-3,-6.799612E-2,2.945281E-2,-4.033087E-2,2.6436772E-2,-1.0019405E-1,-3.2144576E-2,2.5666025E-2,1.0150088E-1,-1.02400765E-1,-7.821879E-3,-2.56371E-3,6.108647E-3,-1.581561E-2,-8.167908E-3,8.205104E-3,1.9849295E-2,-1.0502852E-2,-2.4679367E-1,3.2343555E-2,-4.6544023E-2,3.703103E-2,-1.3987534E-2,1.8861817E-1,5.9109244E-2,-8.604104E-2,-8.38742E-3,5.957424E-3,-6.0360327E-2,-4.595859E-2,4.224683E-2,-5.644145E-2,3.0066041E-2,-2.4810487E-3,2.9968489E-2,-1.8789139E-2,-3.5988917E-3,-4.350231E-2,5.472396E-2,-5.0327055E-2,3.0341987E-3,1.8511347E-2,7.072356E-2,2.2878315E-2,-4.940043E-2,4.8145256E-3,1.1230387E-2,5.4582544E-3,2.464731E-2,-9.5956735E-2,-1.5158756E-3,-3.580078E-5,-4.5568994E-3,3.3893336E-2,-7.9746164E-2,8.814876E-3,3.8107214E-3,-6.469971E-3,6.994127E-5,1.2340657E-2,6.742396E-2,3.1349321E-3,-5.525379E-4,-5.4539385E-4,-3.0551227E-3,3.643503E-2,4.8026326E-3,-4.5605306E-2,-9.763467E-3,-7.6353983E-3,2.8415762E-2,8.673251E-2,2.2392727E-2,4.0239785E-3,-1.0232283E-4,-1.1141788E-1,-1.6333349E-2,4.8853945E-2,-1.2933298E-3,-3.1846615E-3,-5.583082E-3,2.9053094E-3,2.0567297E-4,-6.5920306E-3,-9.4722083E-4,-1.5641664E-3,2.0418656E-3,1.1224442E-3,-3.7184672E-3,4.4471268E-3,6.341318E-4,2.785228E-3,-1.3246203E-3,-2.492325E-3,4.1089495E-4,4.2697974E-3,9.2239E-4,5.1649557E-3,1.3707889E-3,-4.3889205E-3,2.8936309E-3,-2.2847468E-3,1.8195681E-3,-7.2728633E-3,-2.6284996E-3,1.3124868E-3,-1.6875577E-3,4.6970806E-4,3.4833257E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":73,"left_children":[1,3,5,7,-1,9,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,-1,-1,39,-1,-1,41,43,45,47,49,51,53,55,57,59,-1,-1,61,63,65,67,69,-1,71,-1,-1,73,75,77,-1,79,81,83,85,-1,-1,-1,87,89,-1,-1,-1,91,93,95,-1,-1,-1,97,99,-1,-1,-1,-1,101,-1,103,-1,-1,105,107,109,-1,111,113,115,117,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.1016057E-1,2.3397675E-1,3.9475012E-1,2.6758042E-1,0E0,1.4564165E-1,1.2486717E-1,2.025929E-1,1.520387E-1,1.7924929E-1,0E0,1.2962627E-1,1.5444337E-1,2.1678574E-1,1.4114004E-1,4.8390973E-1,2.986553E-1,1.3661814E-1,1.181075E-1,3.57278E-2,1.7950815E-1,0E0,0E0,1.0170157E-1,0E0,0E0,1.1159847E-1,4.33385E-2,3.4328377E-1,1.0253081E-1,1.14237905E-1,1.4532492E-1,8.9906E-2,2.474752E-2,4.7524467E-2,1.5387446E-2,0E0,0E0,4.052201E-2,1.05226964E-1,2.5942605E-2,8.197536E-2,8.1940874E-2,0E0,2.1271536E-2,0E0,0E0,9.1563E-3,3.726892E-2,1.824097E-1,0E0,2.82095E-1,6.227708E-2,4.9894787E-2,6.9955945E-2,0E0,0E0,0E0,2.2974873E-2,1.1049658E-2,0E0,0E0,0E0,9.876444E-3,9.694198E-2,1.9442003E-2,0E0,0E0,0E0,9.6188195E-2,5.980237E-2,0E0,0E0,0E0,0E0,5.5372115E-2,0E0,1.100021E-1,0E0,0E0,7.744495E-2,8.387485E-2,1.0876866E-1,0E0,5.216332E-2,2.0910308E-2,2.2485934E-2,1.142568E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,23,23,26,26,27,27,28,28,29,29,30,30,31,31,32,32,33,33,34,34,35,35,38,38,39,39,40,40,41,41,42,42,44,44,47,47,48,48,49,49,51,51,52,52,53,53,54,54,58,58,59,59,63,63,64,64,65,65,69,69,70,70,75,75,77,77,80,80,81,81,82,82,84,84,85,85,86,86,87,87],"right_children":[2,4,6,8,-1,10,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,-1,-1,40,-1,-1,42,44,46,48,50,52,54,56,58,60,-1,-1,62,64,66,68,70,-1,72,-1,-1,74,76,78,-1,80,82,84,86,-1,-1,-1,88,90,-1,-1,-1,92,94,96,-1,-1,-1,98,100,-1,-1,-1,-1,102,-1,104,-1,-1,106,108,110,-1,112,114,116,118,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7943903E2,3.7009756E2,3.0923106E-10,2.0131578E0,-8.599706E-3,5.619497E7,1.4650772E7,1.1742308E2,1.3095523E2,3.4915986E3,-4.793845E-3,9.015703E-7,1.5043378E0,1.0836275E2,3.5909092E1,4.64E2,4.8421054E0,6.1E1,4.8647133E-1,1.4916515E3,7.046396E8,-2.56371E-3,6.108647E-3,1.3E1,-8.167908E-3,8.205104E-3,5.8182236E1,2.506329E0,8E0,3.9125E1,1.75E2,2.78E2,5.994516E4,2.751192E6,9.356961E6,1.33512E3,-8.38742E-3,5.957424E-3,2.5243637E-6,4.004E3,1.5942905E-4,3.0352948E2,2.75E0,-2.4810487E-3,6.7E1,-1.8789139E-2,-3.5988917E-3,2.506329E0,8.23E2,2E1,3.0341987E-3,1.1057693E0,3.0084E4,1.1891E4,2.391191E10,4.8145256E-3,1.1230387E-2,5.4582544E-3,2.6818182E0,8.05E2,-1.5158756E-3,-3.580078E-5,-4.5568994E-3,8E0,4.9069305E1,4.21E2,3.8107214E-3,-6.469971E-3,6.994127E-5,9.07826E0,1.875E0,3.1349321E-3,-5.525379E-4,-5.4539385E-4,-3.0551227E-3,1E0,4.8026326E-3,1.3177083E1,-9.763467E-3,-7.6353983E-3,3.45E2,2.2512733E5,5.8222644E2,4.0239785E-3,5.431429E2,9.750085E6,9.571136E4,2.375E0,-1.2933298E-3,-3.1846615E-3,-5.583082E-3,2.9053094E-3,2.0567297E-4,-6.5920306E-3,-9.4722083E-4,-1.5641664E-3,2.0418656E-3,1.1224442E-3,-3.7184672E-3,4.4471268E-3,6.341318E-4,2.785228E-3,-1.3246203E-3,-2.492325E-3,4.1089495E-4,4.2697974E-3,9.2239E-4,5.1649557E-3,1.3707889E-3,-4.3889205E-3,2.8936309E-3,-2.2847468E-3,1.8195681E-3,-7.2728633E-3,-2.6284996E-3,1.3124868E-3,-1.6875577E-3,4.6970806E-4,3.4833257E-3],"split_indices":[61,61,46,62,0,56,60,61,56,4,0,46,63,61,56,38,67,8,51,4,40,0,0,3,0,0,56,62,26,4,6,0,42,1,54,64,0,0,46,9,47,42,63,0,38,0,0,62,2,6,0,63,38,38,40,0,0,0,62,10,0,0,0,3,56,2,0,0,0,65,62,0,0,0,0,23,0,65,0,0,38,42,61,0,61,9,42,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.93E2,5.73E2,4.2E2,5.65E2,8E0,3.42E2,7.8E1,2.1E2,3.55E2,3.35E2,7E0,6.2E1,1.6E1,6.5E1,1.45E2,3.6E1,3.19E2,3.01E2,3.4E1,3.9E1,2.3E1,9E0,7E0,5.6E1,9E0,5E0,1.4E2,2.3E1,1.3E1,5.8E1,2.61E2,2.34E2,6.7E1,1E1,2.4E1,3.4E1,5E0,6E0,1.7E1,3.7E1,1.9E1,1.6E1,1.24E2,1.1E1,1.2E1,6E0,7E0,1.3E1,4.5E1,2.53E2,8E0,1.52E2,8.2E1,3.3E1,3.4E1,5E0,5E0,8E0,1.6E1,2.8E1,6E0,7E0,1E1,1.1E1,2.6E1,1.1E1,8E0,6E0,1E1,8.5E1,3.9E1,6E0,6E0,6E0,7E0,3.4E1,1.1E1,2.47E2,6E0,7E0,1.45E2,6.1E1,2.1E1,8E0,2.5E1,1.1E1,2.3E1,1.1E1,5E0,1.5E1,1.3E1,5E0,6E0,1.2E1,1.4E1,5E0,6E0,7.6E1,9E0,2.5E1,1.4E1,2.5E1,9E0,2.15E2,3.2E1,1.6E1,1.29E2,4.2E1,1.9E1,5E0,1.6E1,1.1E1,1.4E1,5E0,6E0,7E0,1.6E1,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"119","size_leaf_vector":"1"}},{"base_weights":[-3.3707072E-3,-1.40992785E-2,4.11449E-2,-4.263031E-2,1.0270004E-3,8.073705E-3,3.487205E-2,-3.105976E-2,-1.6796862E-1,-5.5322163E-3,5.7265148E-2,-3.4314033E-2,4.602192E-2,-3.9550506E-2,7.067637E-2,2.3544611E-5,-2.3117885E-1,3.021029E-4,-1.2362211E-1,-3.3274665E-2,7.96145E-2,-4.982688E-3,-5.5864463E-3,-3.3911157E-2,5.502124E-2,-4.4783633E-2,4.1963183E-3,2.1895418E-2,8.014068E-3,-1.569317E-2,-3.798308E-3,3.0909255E-3,-9.067864E-3,-1.7174052E-2,-2.3453329E-2,-4.8368196E-3,2.0062914E-3,1.2493152E-1,2.8348355E-2,2.7973002E-3,-3.341907E-2,1.0172183E-3,-3.2295848E-3,6.082201E-2,-3.793887E-3,1.9045183E-2,-5.2861426E-2,3.3938638E-3,-2.7202822E-3,6.1196764E-3,-5.6608524E-3,2.5191123E-3,-6.086093E-3,8.351072E-3,7.297465E-2,-1.18582E-2,4.415659E-3,-3.0096988E-3,8.964232E-4,9.9682726E-2,4.4637494E-2,4.049118E-2,-1.9434908E-3,-3.9139003E-2,-9.3213305E-2,-1.6379537E-2,2.3135593E-2,5.174974E-3,4.3632585E-4,2.0549078E-3,-2.7609176E-3,1.3488688E-1,5.737926E-2,5.9874464E-2,-1.54948775E-2,3.3639648E-3,4.9280736E-4,-6.289969E-3,-1.4737069E-3,-3.493318E-3,-9.564887E-3,-3.210549E-4,-4.416387E-3,1.4890701E-3,-1.1907483E-3,7.930408E-3,2.759763E-3,3.873607E-3,-6.075984E-4,4.615663E-3,1.4097196E-3,-2.693907E-3,1.9942024E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":74,"left_children":[1,3,5,7,9,-1,11,13,15,17,19,21,23,25,27,-1,29,31,33,35,37,-1,39,41,43,45,-1,47,-1,-1,-1,49,-1,-1,51,-1,-1,53,55,-1,57,-1,-1,59,-1,61,63,-1,-1,65,-1,-1,-1,-1,67,69,-1,-1,-1,71,73,75,-1,77,79,81,83,-1,-1,-1,-1,85,87,89,91,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.499913E-1,3.2760414E-1,1.4644584E-1,3.7619424E-1,1.8364924E-1,0E0,1.3734838E-1,2.1183863E-1,2.3440433E-1,3.0731982E-1,1.0787144E-1,5.188027E-2,1.11206055E-1,1.5642664E-1,9.079544E-2,0E0,2.2804129E-1,2.3314205E-1,5.0987023E-1,6.423853E-2,9.4034046E-2,0E0,3.6850154E-2,3.3412077E-2,1.13678604E-1,1.1237553E-1,0E0,6.2062524E-2,0E0,0E0,0E0,1.6025403E-1,0E0,0E0,1.4110953E-1,0E0,0E0,4.9791694E-2,5.7545394E-2,0E0,2.4471076E-2,0E0,0E0,7.857072E-2,0E0,3.4364328E-2,1.0266703E-1,0E0,0E0,1.5811098E-1,0E0,0E0,0E0,0E0,3.0815065E-2,4.008611E-2,0E0,0E0,0E0,4.8852086E-2,8.797008E-2,1.739616E-2,0E0,9.8843485E-2,8.1645876E-2,1.3203755E-1,1.0353777E-1,0E0,0E0,0E0,0E0,4.4172436E-2,3.6047705E-2,8.574253E-2,5.227326E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,16,16,17,17,18,18,19,19,20,20,22,22,23,23,24,24,25,25,27,27,31,31,34,34,37,37,38,38,40,40,43,43,45,45,46,46,49,49,54,54,55,55,59,59,60,60,61,61,63,63,64,64,65,65,66,66,71,71,72,72,73,73,74,74],"right_children":[2,4,6,8,10,-1,12,14,16,18,20,22,24,26,28,-1,30,32,34,36,38,-1,40,42,44,46,-1,48,-1,-1,-1,50,-1,-1,52,-1,-1,54,56,-1,58,-1,-1,60,-1,62,64,-1,-1,66,-1,-1,-1,-1,68,70,-1,-1,-1,72,74,76,-1,78,80,82,84,-1,-1,-1,-1,86,88,90,92,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.812E3,1E0,1.273782E4,9.639872E0,2.8802464E8,8.073705E-3,9.422379E4,1.559733E6,4.327E3,1.5660409E7,4.5981E4,5.658231E9,8.91E2,1.4433751E0,5.1766735E-1,2.3544611E-5,1.753E3,1.1048219E3,1.4693878E0,3.5304296E7,1.5218779E4,-4.982688E-3,2.1922675E8,7.3E1,2.871605E2,1.0204082E0,4.1963183E-3,3.364486E0,8.014068E-3,-1.569317E-2,-3.798308E-3,1E0,-9.067864E-3,-1.7174052E-2,1.7321428E1,-4.8368196E-3,2.0062914E-3,3.1478872E0,2.8390558E1,2.7973002E-3,5.489183E1,1.0172183E-3,-3.2295848E-3,7.556899E-1,-3.793887E-3,1.766298E2,5.524E3,3.3938638E-3,-2.7202822E-3,2.778995E2,-5.6608524E-3,2.5191123E-3,-6.086093E-3,8.351072E-3,8.611817E6,1.1047E4,4.415659E-3,-3.0096988E-3,8.964232E-4,2.2681375E6,9.3E1,4.31903E6,-1.9434908E-3,1.1649485E0,3.4111922E0,2.6E1,4.383864E7,5.174974E-3,4.3632585E-4,2.0549078E-3,-2.7609176E-3,2E1,5.268442E6,5.6516E4,1.8664102E2,3.3639648E-3,4.9280736E-4,-6.289969E-3,-1.4737069E-3,-3.493318E-3,-9.564887E-3,-3.210549E-4,-4.416387E-3,1.4890701E-3,-1.1907483E-3,7.930408E-3,2.759763E-3,3.873607E-3,-6.075984E-4,4.615663E-3,1.4097196E-3,-2.693907E-3,1.9942024E-3],"split_indices":[2,109,42,63,5,0,42,9,1,54,12,40,38,47,47,0,38,61,62,7,42,0,5,0,67,62,0,65,0,0,0,73,0,0,67,0,0,62,67,0,65,0,0,51,0,4,38,0,0,64,0,0,0,0,54,38,0,0,0,37,8,54,0,62,62,8,12,0,0,0,0,3,52,9,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.4E2,7.58E2,1.82E2,2.62E2,4.96E2,7E0,1.75E2,2.41E2,2.1E1,4.45E2,5.1E1,2.4E1,1.51E2,2.23E2,1.8E1,6E0,1.5E1,4.25E2,2E1,1E1,4.1E1,6E0,1.8E1,1.5E1,1.36E2,2.15E2,8E0,1.3E1,5E0,8E0,7E0,4.2E2,5E0,5E0,1.5E1,5E0,5E0,2.1E1,2E1,5E0,1.3E1,6E0,9E0,1.31E2,5E0,2.4E1,1.91E2,8E0,5E0,4.11E2,9E0,9E0,6E0,9E0,1.2E1,1.3E1,7E0,8E0,5E0,3.7E1,9.4E1,1.8E1,6E0,1.44E2,4.7E1,1.77E2,2.34E2,7E0,5E0,6E0,7E0,1.9E1,1.8E1,7.5E1,1.9E1,8E0,1E1,9E0,1.35E2,4.2E1,5E0,1.59E2,1.8E1,1.98E2,3.6E1,1.2E1,7E0,1.3E1,5E0,3.1E1,4.4E1,1.1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"93","size_leaf_vector":"1"}},{"base_weights":[-4.373269E-3,-2.0315515E-2,1.7837241E-2,-1.41056E-1,-1.6713737E-2,2.7484767E-2,-6.289883E-2,6.368949E-4,-1.3806336E-2,-2.505355E-2,3.0376108E-2,5.4405914E-3,2.3490442E-2,-7.952566E-2,-3.6073363E-6,-2.2601737E-2,-8.390977E-3,1.4661274E-2,1.0565807E-1,4.4805486E-2,9.33827E-3,-1.2669145E-2,-1.1094968E-1,-1.8825652E-2,-1.082621E-1,3.2718305E-2,-3.4579247E-2,5.7123514E-4,7.245243E-3,8.07787E-3,6.582046E-2,-5.1145703E-2,2.9827451E-2,2.1993048E-3,-2.8928453E-3,-3.343283E-3,-8.113931E-3,-5.0153644E-3,-3.851581E-2,-1.0980381E-2,-3.0802745E-2,-5.4032947E-3,6.68043E-2,-7.145234E-2,1.1624347E-3,-4.9177594E-3,4.9352595E-3,7.526148E-2,-1.1304794E-3,-1.18019454E-1,5.637714E-2,9.593525E-2,1.7673958E-2,-4.993844E-2,7.401914E-3,-2.2062866E-2,-1.06047645E-1,-3.6366817E-3,1.7522462E-3,2.3659654E-3,-4.0312205E-2,-8.417565E-4,8.693478E-2,-4.9688197E-3,-1.4244621E-3,-5.444057E-2,4.2746507E-2,5.0952822E-2,1.15415975E-1,-5.521005E-2,-1.1922795E-2,1.0699808E-3,3.686114E-3,1.2393336E-1,-4.043539E-4,3.5386432E-2,-1.4429103E-2,-3.952873E-3,-4.137844E-4,6.7193893E-4,-3.818483E-3,1.8899811E-3,-1.8581591E-3,-1.14696855E-2,-1.0049242E-3,-4.9152602E-3,-3.9621274E-5,2.4534855E-3,4.939002E-3,1.4787582E-3,-3.720599E-3,2.9253536E-3,-7.2206627E-4,5.1521743E-3,1.4686778E-3,6.2281853E-3,2.2852072E-3,-5.1601334E-3,-1.4088993E-3,7.0599723E-3,3.067322E-3,7.0103037E-4,3.909613E-3,7.02081E-5,-3.4874198E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":75,"left_children":[1,3,5,7,9,11,13,-1,-1,15,17,-1,19,21,-1,23,-1,25,27,29,31,33,35,37,39,41,43,-1,-1,45,47,49,51,-1,-1,-1,-1,53,55,-1,57,59,61,63,-1,65,-1,67,-1,69,71,73,75,77,79,81,83,-1,-1,-1,85,-1,87,-1,-1,89,91,93,95,97,-1,-1,-1,99,-1,101,103,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.3886802E-1,2.3987079E-1,3.1329006E-1,3.9165482E-1,2.1361555E-1,1.2561512E-1,4.4907957E-2,0E0,0E0,1.7199853E-1,9.546049E-2,0E0,1.0335064E-1,7.0025E-2,0E0,1.4513654E-1,0E0,6.248188E-2,6.406258E-2,1.0520902E-1,2.5929308E-1,3.8763836E-2,4.458353E-2,1.1863881E-1,1.8166137E-1,6.6631E-2,4.3567613E-2,0E0,0E0,6.566535E-2,7.590842E-2,3.8774776E-1,1.2364119E-1,0E0,0E0,0E0,0E0,1.4428781E-1,1.9765785E-1,0E0,4.5366988E-2,5.1272407E-2,4.774715E-2,1.4139231E-2,0E0,1.1090868E-1,0E0,7.1231306E-2,0E0,2.7379692E-1,1.430627E-2,7.14E-2,7.613768E-2,8.022377E-2,1.3010396E-1,1.6651343E-1,4.1154283E-1,0E0,0E0,0E0,4.133853E-2,0E0,8.989945E-3,0E0,0E0,5.366472E-2,2.9390033E-2,5.5628374E-2,2.948898E-2,3.0016087E-2,0E0,0E0,0E0,2.0950258E-2,0E0,8.395721E-2,4.6587314E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,9,9,10,10,12,12,13,13,15,15,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,37,37,38,38,40,40,41,41,42,42,43,43,45,45,47,47,49,49,50,50,51,51,52,52,53,53,54,54,55,55,56,56,60,60,62,62,65,65,66,66,67,67,68,68,69,69,73,73,75,75,76,76],"right_children":[2,4,6,8,10,12,14,-1,-1,16,18,-1,20,22,-1,24,-1,26,28,30,32,34,36,38,40,42,44,-1,-1,46,48,50,52,-1,-1,-1,-1,54,56,-1,58,60,62,64,-1,66,-1,68,-1,70,72,74,76,78,80,82,84,-1,-1,-1,86,-1,88,-1,-1,90,92,94,96,98,-1,-1,-1,100,-1,102,104,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.6134964E2,4E0,3.0161016E7,4.1618698E2,8.328548E6,2.82E2,7.661857E-2,6.368949E-4,-1.3806336E-2,4.8E1,1E0,5.4405914E-3,3.72381E5,1.5E1,-3.6073363E-6,3.842105E1,-8.390977E-3,7.09E2,3.0516066E5,1E0,1.038946E6,5.388794E6,1.266059E3,2E0,1E0,1.088E3,2.670362E7,5.7123514E-4,7.245243E-3,4.5E1,3.93617E0,4.797448E8,1.6954315E0,2.1993048E-3,-2.8928453E-3,-3.343283E-3,-8.113931E-3,3.175E1,9.244374E6,-1.0980381E-2,1.56049385E1,3.63E2,1.184953E0,2.12E2,1.1624347E-3,6.6091795E6,4.9352595E-3,1.6E1,-1.1304794E-3,1.4283589E6,2.0569372E-3,7.1172465E10,1.3342042E0,1.5827801E4,9.07826E0,2.295E3,7.478261E0,-3.6366817E-3,1.7522462E-3,2.3659654E-3,1.5626E5,-8.417565E-4,1.0489464E7,-4.9688197E-3,-1.4244621E-3,1.753E3,2.4526315E0,7.6825094E0,3.7936268E-3,7.122041E7,-1.1922795E-2,1.0699808E-3,3.686114E-3,1.6063418E7,-4.043539E-4,3.5474228E1,4.990618E6,-3.952873E-3,-4.137844E-4,6.7193893E-4,-3.818483E-3,1.8899811E-3,-1.8581591E-3,-1.14696855E-2,-1.0049242E-3,-4.9152602E-3,-3.9621274E-5,2.4534855E-3,4.939002E-3,1.4787582E-3,-3.720599E-3,2.9253536E-3,-7.2206627E-4,5.1521743E-3,1.4686778E-3,6.2281853E-3,2.2852072E-3,-5.1601334E-3,-1.4088993E-3,7.0599723E-3,3.067322E-3,7.0103037E-4,3.909613E-3,7.02081E-5,-3.4874198E-3],"split_indices":[4,3,54,4,9,38,66,0,0,3,25,0,9,3,0,55,0,10,60,109,9,56,61,8,23,38,5,0,0,3,62,7,63,0,0,0,0,64,54,0,67,38,63,0,0,56,0,3,0,37,43,40,47,37,65,56,67,0,0,0,12,0,5,0,0,38,62,65,66,7,0,0,0,56,0,67,54,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.55E2,5.56E2,3.99E2,1.5E1,5.41E2,3.57E2,4.2E1,8E0,7E0,4.6E2,8.1E1,1.4E1,3.43E2,3.3E1,9E0,4.54E2,6E0,6.8E1,1.3E1,1.36E2,2.07E2,1.1E1,2.2E1,4.36E2,1.8E1,5E1,1.8E1,5E0,8E0,5E1,8.6E1,5.2E1,1.55E2,5E0,6E0,1.5E1,7E0,2.57E2,1.79E2,6E0,1.2E1,2.4E1,2.6E1,1.1E1,7E0,4.5E1,5E0,7.8E1,8E0,3.2E1,2E1,2.3E1,1.32E2,5.5E1,2.02E2,1.45E2,3.4E1,7E0,5E0,9E0,1.5E1,5E0,2.1E1,5E0,6E0,2.2E1,2.3E1,5E1,2.8E1,2.3E1,9E0,9E0,1.1E1,1.8E1,5E0,8.5E1,4.7E1,2.9E1,2.6E1,1.88E2,1.4E1,3.2E1,1.13E2,1.2E1,2.2E1,5E0,1E1,9E0,1.2E1,5E0,1.7E1,1.7E1,6E0,1.1E1,3.9E1,2.1E1,7E0,6E0,1.7E1,1.1E1,7E0,6.1E1,2.4E1,3.8E1,9E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"105","size_leaf_vector":"1"}},{"base_weights":[-1.326992E-3,-1.5176629E-2,3.178164E-2,6.1340414E-2,-1.8525895E-2,4.6300393E-2,-5.9029162E-2,3.6555063E-2,6.21822E-3,-1.0883928E-1,-1.552416E-2,3.85734E-2,1.4902759E-1,-3.7347704E-2,-8.150804E-3,-2.1650344E-3,3.351378E-3,-3.9031483E-2,-1.0087138E-2,-1.6422059E-3,-3.145717E-2,1.147153E-1,3.0050365E-2,4.04566E-3,9.015611E-3,-1.9951744E-2,-5.4799034E-3,1.2208543E-3,-1.2186574E-3,-4.819619E-3,3.9590974E-4,-6.754886E-2,9.087156E-3,-2.689643E-2,-7.4081654E-3,7.6910853E-3,7.006208E-2,7.114968E-4,4.9603965E-2,-4.876433E-2,1.974288E-3,-3.6910314E-2,-1.2651905E-1,5.389906E-2,-3.5615936E-3,-1.0978344E-1,-1.9628394E-2,4.454936E-3,1.4103216E-3,2.617011E-2,-1.10714406E-1,5.7236407E-2,-3.25395E-3,-3.1387277E-3,-6.206249E-5,2.3706865E-3,-5.444995E-2,-2.933755E-3,-9.226998E-3,-1.9587541E-3,6.438889E-2,-2.154546E-2,4.172452E-2,-2.1574958E-1,-1.1674912E-2,1.839179E-3,-4.0473513E-2,3.9838396E-2,-5.076327E-2,-5.445812E-2,-9.066375E-3,6.353934E-2,-1.770562E-3,-1.0284941E-3,-5.1199077E-3,1.7896106E-3,4.2500435E-3,-1.4525426E-3,2.378707E-3,2.333244E-3,-2.425392E-3,-1.5010389E-2,-3.342473E-3,1.8994734E-3,-2.304882E-3,-3.6540083E-4,7.117445E-3,-7.640757E-3,-1.5975253E-3,7.6238654E-4,5.055926E-3,1.0718289E-3,-5.3900355E-3,-5.1183547E-3,4.8584337E-4,5.2390853E-3,2.329978E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":76,"left_children":[1,3,5,7,9,11,13,15,-1,17,19,21,23,25,-1,27,-1,29,-1,31,33,35,37,-1,-1,39,-1,-1,-1,-1,-1,41,43,45,-1,-1,47,49,51,53,-1,55,57,59,61,63,65,-1,-1,67,69,71,-1,-1,-1,-1,73,-1,-1,-1,75,77,79,81,83,85,87,89,91,93,-1,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[4.4618565E-1,1.7694174E-1,3.8113096E-1,4.7337823E-2,1.7669208E-1,1.9110698E-1,9.459366E-2,3.1926155E-2,0E0,1.5065782E-1,1.4106204E-1,1.4711148E-1,3.0653447E-2,4.681213E-2,0E0,8.926404E-3,0E0,4.4063758E-2,0E0,2.4245444E-1,1.6936263E-1,4.0809005E-2,1.2043761E-1,0E0,0E0,5.609989E-2,0E0,0E0,0E0,0E0,0E0,8.2054704E-2,1.6740227E-1,1.7134376E-1,0E0,0E0,1.149302E-2,2.4402106E-1,1.1915979E-1,1.8714808E-2,0E0,5.371905E-2,6.0682893E-2,6.825253E-2,1.8902987E-1,2.365933E-1,1.19107865E-1,0E0,0E0,7.5630635E-2,6.5583184E-2,7.397595E-2,0E0,0E0,0E0,0E0,4.8079632E-2,0E0,0E0,0E0,3.7914068E-2,1.1956778E-1,5.574251E-2,1.4703798E-1,2.8024007E-2,1.9554636E-1,9.566422E-2,9.5294654E-2,5.5822983E-2,4.1046098E-2,0E0,6.800744E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,9,9,10,10,11,11,12,12,13,13,15,15,17,17,19,19,20,20,21,21,22,22,25,25,31,31,32,32,33,33,36,36,37,37,38,38,39,39,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,56,56,60,60,61,61,62,62,63,63,64,64,65,65,66,66,67,67,68,68,69,69,71,71],"right_children":[2,4,6,8,10,12,14,16,-1,18,20,22,24,26,-1,28,-1,30,-1,32,34,36,38,-1,-1,40,-1,-1,-1,-1,-1,42,44,46,-1,-1,48,50,52,54,-1,56,58,60,62,64,66,-1,-1,68,70,72,-1,-1,-1,-1,74,-1,-1,-1,76,78,80,82,84,86,88,90,92,94,-1,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.867288E2,1.0622318E0,3.0161016E7,1.9559006E1,1.342376E0,5.5766084E3,7.671875E0,3.57E2,6.21822E-3,2.2461708E6,3.6245735E0,1.4960126E6,6E0,4.1831533E8,-8.150804E-3,2.07E2,3.351378E-3,2.92E3,-1.0087138E-2,3.89E2,4.5158855E6,3.116E3,1.0901037E10,4.04566E-3,9.015611E-3,6.849626E1,-5.4799034E-3,1.2208543E-3,-1.2186574E-3,-4.819619E-3,3.9590974E-4,6.0921145E4,3.5318289E3,5.862757E1,-7.4081654E-3,7.6910853E-3,4.159558E0,2.16E2,2.1199985E-7,5E0,1.974288E-3,2.081545E0,3.07E2,1.45064E5,1.802361E6,3.6579263E2,1.6955555E1,4.454936E-3,1.4103216E-3,8.216578E9,8.079573E5,2.3876712E7,-3.25395E-3,-3.1387277E-3,-6.206249E-5,2.3706865E-3,6.48334E5,-2.933755E-3,-9.226998E-3,-1.9587541E-3,5.055762E-1,4.251E3,2.923409E7,2.1539131E2,2.810359E2,3.5E1,1.7553097E1,1.0322473E8,1.731E3,5.658231E9,-9.066375E-3,3.6444E4,-1.770562E-3,-1.0284941E-3,-5.1199077E-3,1.7896106E-3,4.2500435E-3,-1.4525426E-3,2.378707E-3,2.333244E-3,-2.425392E-3,-1.5010389E-2,-3.342473E-3,1.8994734E-3,-2.304882E-3,-3.6540083E-4,7.117445E-3,-7.640757E-3,-1.5975253E-3,7.6238654E-4,5.055926E-3,1.0718289E-3,-5.3900355E-3,-5.1183547E-3,4.8584337E-4,5.2390853E-3,2.329978E-3],"split_indices":[61,63,54,67,63,4,55,0,0,37,65,54,8,41,0,12,0,38,0,1,56,2,40,0,0,67,0,0,0,0,0,54,56,56,0,0,63,0,46,6,0,63,1,7,12,64,67,0,0,40,37,54,0,0,0,0,40,0,0,0,65,38,9,42,42,3,67,5,2,40,0,9,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.71E2,6.85E2,2.86E2,2.8E1,6.57E2,2.47E2,3.9E1,2.2E1,6E0,2E1,6.37E2,2.31E2,1.6E1,3.4E1,5E0,1.1E1,1.1E1,1.3E1,7E0,3.41E2,2.96E2,2.2E1,2.09E2,8E0,8E0,2.9E1,5E0,5E0,6E0,5E0,8E0,4.7E1,2.94E2,2.87E2,9E0,9E0,1.3E1,8.4E1,1.25E2,2E1,9E0,3.2E1,1.5E1,6.4E1,2.3E2,2.2E1,2.65E2,7E0,6E0,6.9E1,1.5E1,1.18E2,7E0,1.4E1,6E0,5E0,2.7E1,9E0,6E0,6E0,5.8E1,1.65E2,6.5E1,1E1,1.2E1,1.31E2,1.34E2,5.9E1,1E1,1E1,5E0,1.11E2,7E0,1.8E1,9E0,3.1E1,2.7E1,1.46E2,1.9E1,6E1,5E0,5E0,5E0,5E0,7E0,1.24E2,7E0,5E0,1.29E2,4.5E1,1.4E1,5E0,5E0,5E0,5E0,2.2E1,8.9E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[-1.6754633E-3,-1.9343E-2,1.9881928E-2,-1.3363236E-2,-1.1671001E-1,2.2316745E-2,-7.440684E-3,-1.0662144E-2,-1.1023454E-2,-2.2247858E-1,-4.7413364E-2,1.2801854E-2,6.32371E-2,-3.5715006E-2,4.1503073E-3,-1.4607059E-2,-3.6041774E-3,5.7824897E-3,-6.822185E-3,7.850335E-3,5.777326E-3,7.23261E-2,-2.003377E-3,-4.6065968E-2,2.7330186E-2,8.824012E-3,-9.7632304E-2,-2.5448455E-3,2.1139255E-3,7.044963E-2,1.139568E-3,8.516384E-2,2.9081563E-2,-1.5056884E-3,1.3352699E-3,-3.961507E-2,-6.9883442E-3,3.124582E-3,6.9392635E-3,1.50276525E-2,-7.549302E-2,-1.4591238E-3,-7.3925913E-3,1.0822047E-1,2.3554325E-2,9.293761E-3,-1.0066013E-1,-2.725159E-4,9.438501E-2,-1.4356406E-3,3.7209378E-3,-3.2821964E-2,-6.1931275E-3,4.0040653E-2,-1.59099E-3,6.154842E-3,7.7975504E-2,-1.0983086E-1,-2.7678136E-4,3.12355E-3,7.0031905E-3,-7.8060053E-4,3.0639889E-3,3.3901148E-2,-6.190496E-3,-1.149567E-2,-2.0821583E-2,1.1009326E-1,4.120823E-2,-1.0190464E-3,-3.999825E-3,3.4248936E-3,-1.8812438E-5,5.586162E-4,-3.0633523E-3,6.826778E-3,8.701069E-4,-2.2800893E-3,-8.470734E-3,-2.544265E-3,2.2932077E-3,-5.1680417E-4,4.6270075E-3,-4.412705E-3,2.2048773E-3,7.540729E-3,3.695556E-3,-4.6667547E-4,4.0176017E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":77,"left_children":[1,3,5,7,9,11,-1,13,-1,15,17,19,21,23,25,-1,-1,27,-1,29,-1,31,33,35,37,39,41,-1,-1,43,45,47,49,-1,-1,51,-1,-1,53,55,57,-1,-1,59,61,63,65,-1,67,-1,-1,69,-1,71,-1,73,75,77,-1,-1,-1,-1,-1,79,81,-1,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.7287155E-1,3.1168103E-1,1.971159E-1,3.0512348E-1,2.1666312E-1,1.6894822E-1,0E0,1.8680866E-1,0E0,1.3598526E-1,1.068359E-1,1.9538711E-1,4.876849E-2,1.2329115E-1,1.5165901E-1,0E0,0E0,3.6261592E-2,0E0,1.4317015E-1,0E0,3.7494987E-2,1.1243048E-2,1.0261524E-1,2.1560855E-2,1.6014853E-1,5.215387E-2,0E0,0E0,5.5903032E-2,2.574347E-1,4.6725124E-2,5.770301E-2,0E0,0E0,9.3171224E-2,0E0,0E0,2.730251E-2,1.5814099E-1,4.948096E-2,0E0,0E0,2.1285504E-2,2.855544E-2,1.0956465E-1,2.649491E-1,0E0,3.7228942E-2,0E0,0E0,8.158402E-2,0E0,1.5055623E-2,0E0,1.0697425E-1,1.4058612E-1,5.282536E-2,0E0,0E0,0E0,0E0,0E0,1.5559624E-1,9.386161E-2,0E0,8.574241E-2,4.7709763E-2,3.1114172E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,10,10,11,11,12,12,13,13,14,14,17,17,19,19,21,21,22,22,23,23,24,24,25,25,26,26,29,29,30,30,31,31,32,32,35,35,38,38,39,39,40,40,43,43,44,44,45,45,46,46,48,48,51,51,53,53,55,55,56,56,57,57,63,63,64,64,66,66,67,67,68,68],"right_children":[2,4,6,8,10,12,-1,14,-1,16,18,20,22,24,26,-1,-1,28,-1,30,-1,32,34,36,38,40,42,-1,-1,44,46,48,50,-1,-1,52,-1,-1,54,56,58,-1,-1,60,62,64,66,-1,68,-1,-1,70,-1,72,-1,74,76,78,-1,-1,-1,-1,-1,80,82,-1,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7667856E2,9.932432E0,5.619497E7,4.5620965E6,2E0,3.422592E6,-7.440684E-3,1.7667647E2,-1.1023454E-2,5.75E2,3.9682608E2,4.620305E3,1.1450447E9,1E0,5.1034E4,-1.4607059E-2,-3.6041774E-3,1.00037E5,-6.822185E-3,1.1652361E6,5.777326E-3,9.033567E6,1.016E4,6.0147805E0,7E0,1E0,1.7653975E1,-2.5448455E-3,2.1139255E-3,1.85E2,2.8110507E0,8.3E1,9.356961E6,-1.5056884E-3,1.3352699E-3,5.766206E6,-6.9883442E-3,3.124582E-3,1.96E2,1.4272727E1,4.2711125E5,-1.4591238E-3,-7.3925913E-3,1.296E3,5.989087E7,1.1771428E1,3.1059341E0,-2.725159E-4,2.0130434E1,-1.4356406E-3,3.7209378E-3,1.2915888E1,-6.1931275E-3,4.325E0,-1.59099E-3,1.4178663E0,1.68574E5,6.9003E4,-2.7678136E-4,3.12355E-3,7.0031905E-3,-7.8060053E-4,3.0639889E-3,3.7788504E7,3.2645503E5,-1.149567E-2,1.4E1,2.94857E5,6.1236826E8,-1.0190464E-3,-3.999825E-3,3.4248936E-3,-1.8812438E-5,5.586162E-4,-3.0633523E-3,6.826778E-3,8.701069E-4,-2.2800893E-3,-8.470734E-3,-2.544265E-3,2.2932077E-3,-5.1680417E-4,4.6270075E-3,-4.412705E-3,2.2048773E-3,7.540729E-3,3.695556E-3,-4.6667547E-4,4.0176017E-3],"split_indices":[61,63,56,56,26,37,0,64,0,2,4,61,7,25,38,0,0,5,0,54,0,54,2,63,3,85,65,0,0,0,51,0,54,0,0,54,0,0,0,65,56,0,0,2,5,65,51,0,65,0,0,67,0,63,0,47,9,12,0,0,0,0,0,7,42,0,3,1,7,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.77E2,5.37E2,4.4E2,5.07E2,3E1,4.35E2,5E0,5.02E2,5E0,1.1E1,1.9E1,3.54E2,8.1E1,1.86E2,3.16E2,6E0,5E0,1.3E1,6E0,3.4E2,1.4E1,7.1E1,1E1,1.6E2,2.6E1,3.03E2,1.3E1,5E0,8E0,3.2E1,3.08E2,5.4E1,1.7E1,5E0,5E0,1.52E2,8E0,8E0,1.8E1,2.83E2,2E1,7E0,6E0,1.7E1,1.5E1,2.86E2,2.2E1,5E0,4.9E1,8E0,9E0,1.43E2,9E0,1E1,8E0,2.49E2,3.4E1,1.3E1,7E0,1E1,7E0,8E0,7E0,1.1E2,1.76E2,7E0,1.5E1,3.7E1,1.2E1,1.2E2,2.3E1,5E0,5E0,2.31E2,1.8E1,1.5E1,1.9E1,8E0,5E0,1.6E1,9.4E1,1.69E2,7E0,7E0,8E0,1.2E1,2.5E1,6E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[1.7504661E-3,-4.673873E-3,5.0343916E-2,-1.1766916E-2,3.42658E-2,8.125886E-3,4.2945255E-2,-1.3458205E-2,7.581292E-3,4.0416744E-2,-3.0682022E-3,5.1323038E-2,-2.0549968E-3,-2.4243962E-2,1.2227478E-2,-1.4877837E-2,5.3703357E-2,3.4722935E-2,9.4303936E-2,-9.4659515E-3,-5.313968E-2,1.6749406E-2,-6.800788E-3,1.646192E-2,-5.626134E-2,2.378699E-2,7.4596584E-2,4.4014003E-2,-1.6813903E-3,1.253715E-1,2.2489883E-4,-2.5886795E-2,2.231573E-2,-4.2647082E-2,-1.5524228E-2,2.372655E-2,-6.303655E-2,-2.2944605E-3,2.5224113E-3,-6.7034387E-4,-4.115559E-3,4.1715562E-2,-2.254001E-3,9.988238E-2,1.610637E-2,6.1064765E-2,7.91185E-3,7.4047795E-3,3.0539895E-3,-4.1827317E-2,2.211239E-2,7.7396125E-2,-1.759944E-3,-7.738716E-2,-1.4879423E-3,2.7262133E-2,-4.195443E-3,1.2029406E-3,-1.0226467E-1,1.3375878E-2,6.230153E-2,6.182882E-2,1.2993641E-1,2.5568637E-3,-1.1369445E-3,-6.31641E-4,7.803799E-2,-3.8619544E-2,3.6848052E-3,-2.2205608E-3,2.6741228E-3,2.097775E-3,-4.8041223E-3,-2.1863715E-3,5.989591E-3,1.1980394E-3,-2.6854319E-3,-2.5882085E-3,-9.891345E-3,-6.6609593E-4,5.99668E-3,4.0930777E-4,2.3555905E-3,-7.5274566E-3,-1.6822136E-3,3.1048767E-3,-7.9083047E-4,3.5092616E-3,8.6269184E-4,3.8242312E-3,2.1562616E-6,7.439775E-3,2.6300957E-3,9.655775E-4,4.3451E-3,4.3724084E-4,-3.0789308E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":78,"left_children":[1,3,5,7,9,-1,11,13,-1,15,-1,17,-1,19,21,23,25,27,29,31,33,35,-1,37,39,41,43,45,-1,47,-1,49,51,53,-1,55,57,-1,-1,-1,-1,59,-1,61,63,65,67,-1,-1,69,71,73,75,77,79,81,-1,-1,83,85,87,89,91,-1,-1,-1,93,95,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0211416E-1,2.365549E-1,9.680551E-2,2.1667992E-1,8.418307E-2,0E0,8.120291E-2,1.9921084E-1,0E0,9.263371E-2,0E0,6.736073E-2,0E0,2.155697E-1,1.554046E-1,3.334989E-2,6.135553E-2,4.9723595E-2,7.443099E-2,1.7593004E-1,4.9069762E-1,1.17284715E-1,0E0,4.0160395E-2,1.3638202E-2,5.7814896E-2,8.565059E-2,3.9528847E-2,0E0,3.024894E-2,0E0,1.7078424E-1,1.5263742E-1,2.372435E-1,0E0,7.99515E-2,6.197831E-2,0E0,0E0,0E0,0E0,1.9585647E-2,0E0,3.8856357E-2,3.1441487E-2,5.6833446E-2,7.6263994E-2,0E0,0E0,1.0291886E-1,1.6721499E-1,2.31359E-1,1.2685789E-1,2.521611E-1,1.3138331E-1,8.06091E-2,0E0,0E0,4.1619122E-2,2.7199598E-2,1.0246642E-2,2.5439404E-2,3.991157E-2,0E0,0E0,0E0,3.020604E-2,1.9281648E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,9,9,11,11,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,23,23,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,35,35,36,36,41,41,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,53,53,54,54,55,55,58,58,59,59,60,60,61,61,62,62,66,66,67,67],"right_children":[2,4,6,8,10,-1,12,14,-1,16,-1,18,-1,20,22,24,26,28,30,32,34,36,-1,38,40,42,44,46,-1,48,-1,50,52,54,-1,56,58,-1,-1,-1,-1,60,-1,62,64,66,68,-1,-1,70,72,74,76,78,80,82,-1,-1,84,86,88,90,92,-1,-1,-1,94,96,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[6.624E3,2.9652428E6,1.273782E4,1.03992E3,3.2E1,8.125886E-3,7.134432E7,2.026453E6,7.581292E-3,1.185E3,-3.0682022E-3,2.5045E4,-2.0549968E-3,3.72381E5,1E0,2.6886764E9,2.6138635E2,1.800554E-2,9.7155967E11,4.7837E4,2.2681375E6,2.3E2,-6.800788E-3,1.4253987E9,3.1888113E0,3.723499E-3,8.931E3,2.1298597E0,-1.6813903E-3,5.7217455E6,2.2489883E-4,2.748204E-1,7.0407925E0,1.131645E6,-1.5524228E-2,4.643602E8,1.977157E7,-2.2944605E-3,2.5224113E-3,-6.7034387E-4,-4.115559E-3,7.318515E6,-2.254001E-3,1.84E2,1.8321838E1,1.5203704E1,1.4176E4,7.4047795E-3,3.0539895E-3,5.657353E1,5.306E3,1.654E3,2E0,2.0312219E5,1.5484655E6,2.0480016E7,-4.195443E-3,1.2029406E-3,5.25E0,5.1167645E6,1.3683688E7,1.187E3,4.026E3,2.5568637E-3,-1.1369445E-3,-6.31641E-4,9.422379E4,2.1373269E8,3.6848052E-3,-2.2205608E-3,2.6741228E-3,2.097775E-3,-4.8041223E-3,-2.1863715E-3,5.989591E-3,1.1980394E-3,-2.6854319E-3,-2.5882085E-3,-9.891345E-3,-6.6609593E-4,5.99668E-3,4.0930777E-4,2.3555905E-3,-7.5274566E-3,-1.6822136E-3,3.1048767E-3,-7.9083047E-4,3.5092616E-3,8.6269184E-4,3.8242312E-3,2.1562616E-6,7.439775E-3,2.6300957E-3,9.655775E-4,4.3451E-3,4.3724084E-4,-3.0789308E-3],"split_indices":[2,37,42,55,3,0,53,9,0,38,0,2,0,9,73,40,4,66,40,9,37,8,0,40,65,51,38,47,0,52,0,47,67,9,0,7,5,0,0,0,0,54,0,0,67,67,2,0,0,65,2,1,26,42,37,5,0,0,63,37,54,10,2,0,0,0,42,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.66E2,8.54E2,1.12E2,7.23E2,1.31E2,5E0,1.07E2,7.17E2,6E0,1.24E2,7E0,9.8E1,9E0,5.05E2,2.12E2,2.4E1,1E2,7.2E1,2.6E1,3.35E2,1.7E2,2.07E2,5E0,1.4E1,1E1,4.2E1,5.8E1,6.4E1,8E0,1.9E1,7E0,2.21E2,1.14E2,1.65E2,5E0,1.91E2,1.6E1,5E0,9E0,5E0,5E0,3.4E1,8E0,4E1,1.8E1,4.3E1,2.1E1,1.1E1,8E0,1.66E2,5.5E1,3.4E1,8E1,8.9E1,7.6E1,1.86E2,5E0,5E0,1.1E1,1.5E1,1.9E1,1.9E1,2.1E1,9E0,9E0,8E0,3.5E1,1.3E1,8E0,1.57E2,9E0,4.7E1,8E0,1E1,2.4E1,5.4E1,2.6E1,7.8E1,1.1E1,7E1,6E0,1.05E2,8.1E1,5E0,6E0,5E0,1E1,1.4E1,5E0,1.4E1,5E0,1.4E1,7E0,8E0,2.7E1,5E0,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"97","size_leaf_vector":"1"}},{"base_weights":[1.3591455E-3,-1.8573662E-2,1.1370494E-2,-2.7033256E-2,3.635381E-2,1.5575537E-2,-4.5464873E-2,-3.682134E-3,-4.98577E-2,2.0328863E-2,4.3909494E-3,9.795476E-2,1.2073469E-2,-6.2227817E-3,-2.2039829E-2,-4.457747E-2,1.9334147E-2,-3.94384E-2,-9.146913E-3,3.2949094E-3,9.514628E-3,4.0751714E-2,1.5176183E-1,-9.558606E-3,1.4763506E-2,3.2574985E-2,-4.429841E-2,-1.4518256E-2,-1.07545905E-1,-1.4322722E-2,7.973303E-2,-2.7823512E-2,-1.2190738E-1,-1.463821E-3,2.8615313E-2,4.1186106E-3,-1.394628E-3,3.144981E-3,9.375717E-3,4.4696685E-2,6.381319E-3,-6.912332E-4,3.462243E-3,-6.6981316E-2,1.2452786E-3,-3.6273976E-3,7.239264E-3,-3.9639987E-2,-9.934844E-3,2.8332943E-2,-6.434711E-2,1.03041686E-1,3.2298453E-2,3.686426E-2,-3.697393E-2,-2.590313E-3,-6.8146363E-3,-1.4355049E-3,2.5287292E-3,-4.444383E-3,5.5302262E-2,1.1888326E-2,-4.2344537E-2,-9.373401E-2,-1.8129077E-4,-3.34812E-3,2.8564194E-2,-3.383641E-4,-3.033634E-3,4.507149E-3,8.718752E-3,-7.026336E-3,-3.2254804E-2,5.036949E-2,7.2357156E-3,-1.1068432E-3,3.5178012E-3,3.8221043E-3,-1.2194863E-4,-2.8709643E-2,-7.692788E-3,6.9129154E-2,-1.3831934E-2,7.99801E-3,1.17745E-1,-1.4781335E-2,8.319677E-3,-2.482772E-3,-6.42398E-3,4.9187825E-3,-6.7948713E-4,-2.857033E-3,1.3171603E-3,-2.2179398E-3,6.763939E-4,4.0484667E-3,5.3636736E-4,-1.9925316E-3,4.7835032E-4,5.983804E-3,2.6605842E-3,-3.088264E-3,2.6914778E-3,5.089914E-4,-5.430806E-3,2.2163724E-3,9.423136E-3,-4.874244E-3,1.3029575E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":79,"left_children":[1,3,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,-1,-1,33,35,37,-1,39,41,43,45,47,49,51,53,55,-1,57,-1,-1,-1,-1,59,61,-1,-1,63,-1,-1,65,67,-1,69,71,73,75,77,79,-1,-1,-1,-1,-1,81,83,85,87,-1,-1,89,-1,-1,-1,91,-1,93,95,-1,-1,-1,-1,-1,97,-1,99,101,103,105,-1,107,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.9038643E-1,1.4921063E-1,1.5247008E-1,1.4745083E-1,3.8235333E-2,1.6960798E-1,8.8414386E-2,1.3075554E-1,2.0705792E-1,1.8507782E-2,0E0,6.768571E-2,3.368422E-1,0E0,4.5206353E-2,9.307771E-2,1.820645E-1,1.23046726E-1,0E0,0E0,2.4432648E-2,4.6864234E-2,3.699857E-2,0E0,1.4097458E-1,2.306649E-2,4.3750905E-2,4.904344E-2,1.0730006E-1,1.2558492E-1,3.205228E-2,7.0617676E-2,1.8192664E-2,0E0,3.405696E-2,0E0,0E0,0E0,0E0,1.8855405E-1,1.1883855E-1,0E0,0E0,3.338182E-2,0E0,0E0,4.777636E-2,8.682422E-3,0E0,4.2338908E-2,7.0573635E-2,5.066146E-2,3.1132087E-2,2.7175898E-2,1.0489535E-1,0E0,0E0,0E0,0E0,0E0,1.1088598E-1,1.6254923E-1,6.3240784E-1,1.7773524E-2,0E0,0E0,7.517104E-2,0E0,0E0,0E0,3.769507E-2,0E0,1.5986267E-2,1.6168429E-2,0E0,0E0,0E0,0E0,0E0,5.4869324E-2,0E0,6.0317308E-2,7.961662E-2,1.4397784E-1,7.3908046E-2,0E0,9.03243E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,6,6,7,7,8,8,9,9,11,11,12,12,14,14,15,15,16,16,17,17,20,20,21,21,22,22,24,24,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,34,34,39,39,40,40,43,43,46,46,47,47,49,49,50,50,51,51,52,52,53,53,54,54,60,60,61,61,62,62,63,63,66,66,70,70,72,72,73,73,79,79,81,81,82,82,83,83,84,84,86,86],"right_children":[2,4,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,-1,-1,34,36,38,-1,40,42,44,46,48,50,52,54,56,-1,58,-1,-1,-1,-1,60,62,-1,-1,64,-1,-1,66,68,-1,70,72,74,76,78,80,-1,-1,-1,-1,-1,82,84,86,88,-1,-1,90,-1,-1,-1,92,-1,94,96,-1,-1,-1,-1,-1,98,-1,100,102,104,106,-1,108,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[7.13E2,2.103882E6,3.5056704E7,3.79E2,4.267606E0,1.1015E4,1.11791E5,3.89E2,9.639872E0,4.31903E6,4.3909494E-3,2.1668362E1,1.2083E4,-6.2227817E-3,3.53562E5,5.853E3,5.195011E7,3.1E1,-9.146913E-3,3.2949094E-3,3.8807898E6,3.625E0,2.4827585E0,-9.558606E-3,1.0252778E1,1.839E3,2.696024E2,9E0,6.737099E7,2.8E1,2.16577E5,1.6666666E0,3.2638438E5,-1.463821E-3,7.271717E6,4.1186106E-3,-1.394628E-3,3.144981E-3,9.375717E-3,1.0204082E0,2.855146E5,-6.912332E-4,3.462243E-3,1.20086E5,1.2452786E-3,-3.6273976E-3,7.3569497E3,6.5E1,-9.934844E-3,1.8002975E5,3.470233E7,1.99E2,1.8479175E2,4.252644E7,6.625412E7,-2.590313E-3,-6.8146363E-3,-1.4355049E-3,2.5287292E-3,-4.444383E-3,1E0,2.7125288E5,1.4598765E0,1.3858217E5,-1.8129077E-4,-3.34812E-3,3.5635816E4,-3.383641E-4,-3.033634E-3,4.507149E-3,5.04E2,-7.026336E-3,2.3043478E2,7.701384E3,7.2357156E-3,-1.1068432E-3,3.5178012E-3,3.8221043E-3,-1.2194863E-4,2.893766E5,-7.692788E-3,6E0,7.2E1,4.262058E0,1.3293153E3,-1.4781335E-2,7.080933E2,-2.482772E-3,-6.42398E-3,4.9187825E-3,-6.7948713E-4,-2.857033E-3,1.3171603E-3,-2.2179398E-3,6.763939E-4,4.0484667E-3,5.3636736E-4,-1.9925316E-3,4.7835032E-4,5.983804E-3,2.6605842E-3,-3.088264E-3,2.6914778E-3,5.089914E-4,-5.430806E-3,2.2163724E-3,9.423136E-3,-4.874244E-3,1.3029575E-3],"split_indices":[2,37,54,38,65,9,1,1,63,54,0,65,9,0,1,9,7,8,0,0,37,63,62,0,67,2,65,64,7,10,37,63,54,0,54,0,0,0,0,62,42,0,0,9,0,0,37,2,0,54,7,2,61,7,5,0,0,0,0,0,85,42,63,42,0,0,54,0,0,0,7,0,4,37,0,0,0,0,0,56,0,3,0,51,4,0,4,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.52E2,3.18E2,6.34E2,2.76E2,4.2E1,5.91E2,4.3E1,1.37E2,1.39E2,3.4E1,8E0,2.3E1,5.68E2,8E0,3.5E1,4.9E1,8.8E1,1.31E2,8E0,5E0,2.9E1,1.2E1,1.1E1,6E0,5.62E2,1E1,2.5E1,3.4E1,1.5E1,5.7E1,3.1E1,1.16E2,1.5E1,9E0,2E1,7E0,5E0,5E0,6E0,1.22E2,4.4E2,5E0,5E0,1.9E1,6E0,8E0,2.6E1,1E1,5E0,3.1E1,2.6E1,2E1,1.1E1,1.4E1,1.02E2,5E0,1E1,6E0,1.4E1,8E0,1.14E2,3.96E2,4.4E1,1.3E1,6E0,5E0,2.1E1,5E0,5E0,6E0,2.5E1,6E0,2E1,1.1E1,9E0,5E0,6E0,6E0,8E0,9.7E1,5E0,9.5E1,1.9E1,3.83E2,1.3E1,6E0,3.8E1,8E0,5E0,7E0,1.4E1,5E0,2E1,1.5E1,5E0,5E0,6E0,7.1E1,2.6E1,1.4E1,8.1E1,1.1E1,8E0,3.75E2,8E0,8E0,5E0,5E0,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"109","size_leaf_vector":"1"}},{"base_weights":[-2.8877612E-3,-1.044203E-2,3.6160454E-2,-1.6499192E-2,4.7080096E-2,4.3738466E-2,-7.802412E-3,-1.4656411E-2,-9.841751E-3,5.6731552E-2,-2.861697E-3,1.0485145E-1,3.4051523E-2,-9.974935E-3,-7.050071E-2,6.831351E-2,-2.1974193E-3,3.6823447E-4,6.247283E-3,2.2898516E-2,6.590033E-2,-1.2169894E-2,6.5700053E-3,-1.4865759E-1,-1.86088E-2,5.5342745E-2,6.3433824E-3,3.6313895E-2,-5.822597E-2,8.802045E-2,4.471233E-4,-3.870657E-2,-2.653475E-3,-1.3464262E-2,-6.860439E-2,1.6822305E-2,-1.0066334E-1,6.636612E-2,-1.0137871E-3,-2.189928E-2,4.7472343E-2,-8.059276E-3,7.830903E-4,9.6236524E-4,1.0380329E-1,-2.0901296E-2,-8.1313625E-2,-7.0413426E-2,1.3129807E-3,2.9024172E-3,-8.238954E-3,4.933988E-2,-1.8624628E-3,-7.067456E-3,-1.4955287E-3,7.947371E-2,-1.6857068E-4,-2.8914793E-3,1.5402302E-3,1.3320869E-2,7.1230866E-2,2.3487934E-3,5.482364E-3,-6.505269E-4,-3.035038E-3,-3.9067207E-4,-4.7532963E-3,-4.677981E-3,2.3107507E-4,7.6634153E-3,-1.1751846E-4,2.822416E-5,3.229121E-3,2.1859212E-3,5.156558E-3,1.2305095E-3,-2.2408639E-3,2.6454173E-3,5.9428276E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":80,"left_children":[1,3,5,7,9,11,-1,13,-1,15,-1,17,19,21,23,25,-1,-1,-1,27,29,31,-1,33,35,37,-1,39,41,43,-1,45,47,-1,49,51,53,55,-1,57,59,-1,-1,-1,61,63,65,67,69,-1,-1,71,-1,-1,-1,73,-1,-1,-1,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.9476833E-1,2.92405E-1,2.5818348E-1,2.7012497E-1,8.747077E-2,8.937335E-2,0E0,1.9589493E-1,0E0,9.2447445E-2,0E0,5.9560686E-2,4.715474E-2,2.3294538E-1,2.3246485E-1,5.187553E-2,0E0,0E0,0E0,1.14210926E-1,4.2711556E-2,1.731357E-1,0E0,2.4066997E-1,1.0635558E-1,5.080314E-2,0E0,5.8851227E-2,1.3301107E-1,2.4393603E-2,0E0,1.3561252E-1,1.3608849E-1,0E0,2.3976487E-1,5.01395E-2,3.3973396E-2,4.7082275E-2,0E0,3.5470564E-2,5.9905276E-2,0E0,0E0,0E0,9.284854E-3,3.8592845E-2,8.076835E-2,6.4690664E-2,3.0224591E-1,0E0,0E0,1.673695E-2,0E0,0E0,0E0,3.882292E-2,0E0,0E0,0E0,2.7305538E-2,2.825898E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,9,9,11,11,12,12,13,13,14,14,15,15,19,19,20,20,21,21,23,23,24,24,25,25,27,27,28,28,29,29,31,31,32,32,34,34,35,35,36,36,37,37,39,39,40,40,44,44,45,45,46,46,47,47,48,48,51,51,55,55,59,59,60,60],"right_children":[2,4,6,8,10,12,-1,14,-1,16,-1,18,20,22,24,26,-1,-1,-1,28,30,32,-1,34,36,38,-1,40,42,44,-1,46,48,-1,50,52,54,56,-1,58,60,-1,-1,-1,62,64,66,68,70,-1,-1,72,-1,-1,-1,74,-1,-1,-1,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.9652428E6,1.4039243E3,1E0,1.9269184E7,7.134432E7,1.1057693E0,-7.802412E-3,9.932432E0,-9.841751E-3,2.462277E6,-2.861697E-3,5.8899284E1,2.4342105E0,4.1E1,2E0,4.435876E2,-2.1974193E-3,3.6823447E-4,6.247283E-3,2.2E0,5.4567E5,3.7788504E7,6.5700053E-3,3.4151352E2,3.8E1,1.8258025E2,6.3433824E-3,1E0,3.33E2,1.727279E-4,4.471233E-4,1.26E2,4.157143E1,-1.3464262E-2,3.1E1,1.5757076E9,4.060171E6,1.2820834E7,-1.0137871E-3,9.166947E6,1.2E1,-8.059276E-3,7.830903E-4,9.6236524E-4,3.611956E7,3.470233E7,1E0,1.76421E6,8E0,2.9024172E-3,-8.238954E-3,1.5088889E2,-1.8624628E-3,-7.067456E-3,-1.4955287E-3,1.0046621E7,-1.6857068E-4,-2.8914793E-3,1.5402302E-3,1.4553613E1,1.1E1,2.3487934E-3,5.482364E-3,-6.505269E-4,-3.035038E-3,-3.9067207E-4,-4.7532963E-3,-4.677981E-3,2.3107507E-4,7.6634153E-3,-1.1751846E-4,2.822416E-5,3.229121E-3,2.1859212E-3,5.156558E-3,1.2305095E-3,-2.2408639E-3,2.6454173E-3,5.9428276E-3],"split_indices":[37,61,113,56,53,63,0,63,0,37,0,56,62,3,26,65,0,0,0,62,1,7,0,4,3,67,0,109,10,47,0,10,61,0,0,40,5,9,0,54,3,0,0,0,7,7,8,40,10,0,0,42,0,0,0,56,0,0,0,65,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.97E2,8.36E2,1.61E2,7.57E2,7.9E1,1.56E2,5E0,7.51E2,6E0,7.3E1,6E0,2E1,1.36E2,6.94E2,5.7E1,6.6E1,7E0,5E0,1.5E1,1.02E2,3.4E1,6.85E2,9E0,2.2E1,3.5E1,5.7E1,9E0,8.8E1,1.4E1,2.4E1,1E1,1.8E2,5.05E2,7E0,1.5E1,2.5E1,1E1,5E1,7E0,1.4E1,7.4E1,5E0,9E0,5E0,1.9E1,1.28E2,5.2E1,2.7E1,4.78E2,7E0,8E0,1.6E1,9E0,5E0,5E0,4.2E1,8E0,8E0,6E0,3.1E1,4.3E1,5E0,1.4E1,1.12E2,1.6E1,1.2E1,4E1,1.9E1,8E0,1E1,4.68E2,5E0,1.1E1,2.2E1,2E1,2.6E1,5E0,3.6E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.1850192E-3,-1.72128E-2,1.6280774E-2,-1.4514751E-2,-8.836042E-3,2.64091E-2,-3.694541E-2,9.212633E-3,-2.8896786E-2,1.1179243E-1,1.9996662E-2,5.2153423E-2,-5.4742664E-2,-1.4565629E-3,7.107061E-2,-3.439742E-2,6.1213233E-2,2.499689E-3,1.3587947E-1,1.0193409E-1,1.3950447E-2,-8.257823E-4,4.8561925E-3,-2.3323128E-2,-1.617085E-1,-8.6878456E-2,5.0403946E-3,7.479507E-3,3.50664E-2,-1.07858345E-1,-2.6317185E-2,7.5807828E-3,1.4910841E-3,3.2072826E-3,7.328055E-3,4.3012027E-4,5.900108E-3,1.0629277E-2,1.0476921E-1,-5.4964624E-2,2.8950572E-2,-3.3216835E-3,-1.18279755E-2,-1.058637E-3,-6.82714E-3,5.4955732E-2,-3.4382313E-3,4.8234602E-3,2.7358904E-3,-1.5008183E-2,-2.4588291E-1,9.970846E-3,-4.5927793E-2,-2.773354E-3,2.1924418E-3,1.5061303E-2,-9.884114E-2,1.1926044E-3,7.719648E-3,-3.3015825E-2,-6.1989166E-3,4.262871E-3,-2.3176991E-3,-1.303296E-3,9.304318E-2,-1.270045E-1,7.3765884E-3,-1.3941028E-3,2.8943715E-3,-5.9099086E-3,3.4219045E-2,-1.6308589E-2,-5.5791875E-3,3.3247788E-2,-2.1856692E-2,-3.9461933E-2,-7.0481272E-3,1.6008284E-3,3.376163E-2,-5.036095E-4,-8.584795E-3,-2.2804271E-3,2.5016407E-4,-2.2713023E-3,2.7954E-3,5.662328E-3,1.1314598E-3,-2.5837268E-3,-8.219984E-3,2.9101789E-3,-1.1266174E-4,3.131259E-3,-4.0309512E-4,-1.9846095E-3,2.07619E-3,-3.5220126E-3,6.646323E-4,-2.2555075E-3,1.4867204E-3,3.6046872E-4,-4.5980676E-3,2.0525658E-3,-6.359665E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":81,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,25,27,29,31,-1,33,35,37,-1,-1,39,41,43,45,-1,47,49,51,-1,53,-1,-1,-1,-1,55,57,59,61,-1,-1,-1,-1,63,65,-1,67,69,71,73,75,-1,-1,77,79,-1,-1,81,-1,-1,83,-1,85,87,89,-1,-1,-1,91,-1,-1,93,95,97,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.6529264E-1,2.435348E-1,2.3224624E-1,1.7771521E-1,0E0,1.9541761E-1,1.1226933E-1,1.3011858E-1,1.6270012E-1,2.7557254E-2,1.6553159E-1,4.5937683E-2,1.9115584E-1,9.422993E-2,8.845171E-2,1.7879745E-1,1.15688324E-1,0E0,1.4111996E-2,5.2582726E-2,9.389811E-2,0E0,0E0,7.76761E-2,9.094909E-2,4.2968027E-2,6.705668E-2,0E0,4.8714228E-2,3.8077784E-1,1.9757958E-1,0E0,3.9268408E-2,0E0,0E0,0E0,0E0,1.4946727E-1,4.9303003E-2,4.564114E-2,3.5627723E-2,0E0,0E0,0E0,0E0,7.536127E-2,1.8268006E-1,0E0,3.315764E-2,1.09452784E-1,1.1694789E-1,7.332977E-2,1.17343426E-1,0E0,0E0,7.395024E-2,8.8983096E-2,0E0,0E0,1.5135029E-2,0E0,0E0,4.0877678E-2,0E0,2.8871924E-2,2.948083E-2,6.852315E-2,0E0,0E0,0E0,2.0568797E-2,0E0,0E0,5.1737525E-2,8.322102E-2,1.13420844E-1,0E0,1.0760028E-1,6.1766833E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,15,15,16,16,18,18,19,19,20,20,23,23,24,24,25,25,26,26,28,28,29,29,30,30,32,32,37,37,38,38,39,39,40,40,45,45,46,46,48,48,49,49,50,50,51,51,52,52,55,55,56,56,59,59,62,62,64,64,65,65,66,66,70,70,73,73,74,74,75,75,77,77,78,78],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,26,28,30,32,-1,34,36,38,-1,-1,40,42,44,46,-1,48,50,52,-1,54,-1,-1,-1,-1,56,58,60,62,-1,-1,-1,-1,64,66,-1,68,70,72,74,76,-1,-1,78,80,-1,-1,82,-1,-1,84,-1,86,88,90,-1,-1,-1,92,-1,-1,94,96,98,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.7667856E2,3.6746317E2,2.3876712E7,2.0131578E0,-8.836042E-3,1.3728223E0,1.5209424E1,5.5026376E-1,2.2503355E1,1E0,3.13E2,1.2376862E10,3.6672198E-8,5E0,1.4693878E0,1E0,1.68574E5,2.499689E-3,2.124E3,8.771579E2,4.435876E2,-8.257823E-4,4.8561925E-3,3.416447E-2,8E0,4E0,6.0921145E4,7.479507E-3,7.3351803E0,4.64E2,2.3273222E-2,7.5807828E-3,1E0,3.2072826E-3,7.328055E-3,4.3012027E-4,5.900108E-3,1.8258025E2,4.43837E0,2.9E1,8.58E2,-3.3216835E-3,-1.18279755E-2,-1.058637E-3,-6.82714E-3,1.5857142E1,3.89E2,4.8234602E-3,2.8266037E2,2.8E1,3.0536E4,7.3106743E3,3.0977E4,-2.773354E-3,2.1924418E-3,3.53562E5,1.2031894E5,1.1926044E-3,7.719648E-3,1.6365927E5,-6.1989166E-3,4.262871E-3,2.4131148E0,-1.303296E-3,1.4473684E-1,6.1E1,9.593451E4,-1.3941028E-3,2.8943715E-3,-5.9099086E-3,1E0,-1.6308589E-2,-5.5791875E-3,4.9E2,9.693913E3,8.172E3,-7.0481272E-3,1.3864E4,1.116241E2,-5.036095E-4,-8.584795E-3,-2.2804271E-3,2.5016407E-4,-2.2713023E-3,2.7954E-3,5.662328E-3,1.1314598E-3,-2.5837268E-3,-8.219984E-3,2.9101789E-3,-1.1266174E-4,3.131259E-3,-4.0309512E-4,-1.9846095E-3,2.07619E-3,-3.5220126E-3,6.646323E-4,-2.2555075E-3,1.4867204E-3,3.6046872E-4,-4.5980676E-3,2.0525658E-3,-6.359665E-4],"split_indices":[61,61,54,62,0,62,65,51,65,109,38,40,46,3,62,13,9,0,38,42,65,0,0,66,8,3,54,0,67,38,47,0,23,0,0,0,0,67,44,3,8,0,0,0,0,4,1,0,4,2,1,56,38,0,0,1,42,0,0,42,0,0,62,0,65,38,37,0,0,0,17,0,0,1,56,38,0,10,65,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.54E2,5.26E2,4.28E2,5.19E2,7E0,3.6E2,6.8E1,1.96E2,3.23E2,2.4E1,3.36E2,1.1E1,5.7E1,1.68E2,2.8E1,3.05E2,1.8E1,8E0,1.6E1,2.2E1,3.14E2,5E0,6E0,4.5E1,1.2E1,1.1E1,1.57E2,7E0,2.1E1,2.9E1,2.76E2,6E0,1.2E1,5E0,1.1E1,5E0,1.7E1,3.04E2,1E1,2.8E1,1.7E1,7E0,5E0,6E0,5E0,2.2E1,1.35E2,6E0,1.5E1,1.8E1,1.1E1,9.7E1,1.79E2,5E0,7E0,2.93E2,1.1E1,5E0,5E0,2.3E1,5E0,5E0,1.2E1,7E0,1.5E1,1E1,1.25E2,1E1,5E0,5E0,1.3E1,5E0,6E0,5.6E1,4.1E1,1.7E2,9E0,1.71E2,1.22E2,6E0,5E0,1.6E1,7E0,7E0,5E0,1E1,5E0,5E0,5E0,1.8E1,1.07E2,7E0,6E0,7E0,4.9E1,1.6E1,2.5E1,1.51E2,1.9E1,1.62E2,9E0,1E2,2.2E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[1.4373777E-3,-1.115538E-2,2.5347669E-2,-8.924638E-3,-1.1057859E-2,1.2568443E-1,2.1457197E-2,-2.755076E-2,2.9728442E-4,8.282269E-3,2.0007177E-3,2.7058976E-2,-5.463288E-2,-1.1295351E-2,-6.293298E-2,4.075521E-3,-6.1111446E-2,2.2604404E-2,1.14853986E-1,-7.5049535E-3,6.1367173E-3,1.4119463E-2,-3.878654E-2,-9.491277E-2,7.0690885E-3,1.17282765E-2,-2.3941996E-2,1.1772086E-4,-9.946973E-2,3.0704562E-2,-3.3599578E-2,9.797869E-3,1.7578865E-3,3.9920555E-3,-3.458168E-3,4.691226E-3,4.957399E-3,-1.3172958E-2,-1.5721884E-1,-1.3067204E-1,-1.7066687E-2,-8.105541E-2,9.455094E-2,6.5084742E-3,8.630701E-2,-7.3804475E-2,8.161927E-3,-2.7915868E-3,-6.8856953E-3,6.8727754E-2,1.9962516E-2,-1.03526816E-1,5.677654E-3,-3.3053355E-3,1.7787611E-2,-4.7827806E-2,3.6470283E-2,-4.0821475E-3,-9.006493E-3,-1.1782399E-2,-8.211683E-2,-2.3422774E-3,1.9240855E-3,-1.359252E-3,-5.5357725E-3,6.2246025E-3,1.8184278E-3,1.2047234E-2,-5.7831462E-2,4.5325086E-2,5.6958254E-3,-4.7230978E-2,-8.607712E-3,-1.585826E-2,6.846894E-2,-2.2628438E-4,9.525434E-2,4.588173E-2,-2.2688666E-2,-1.3505857E-3,-7.570353E-3,-1.1619497E-2,2.7610285E-3,4.6620197E-3,-5.2805673E-5,-4.9316036E-3,-1.0454458E-3,3.70925E-3,-1.3497229E-5,1.2835732E-3,-7.1764058E-3,3.0312897E-4,3.8691517E-3,-4.472284E-3,2.0445548E-3,-1.1818416E-6,3.856786E-3,-4.93032E-3,-1.3817294E-3,-2.499748E-3,8.1116863E-4,4.740696E-3,1.6076555E-3,2.2501322E-3,-2.0187832E-3,1.2473703E-3,5.1981844E-3,-4.3629663E-4,2.671936E-3,-8.227726E-3,-4.1357163E-4,1.4737297E-3,-2.939405E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":82,"left_children":[1,3,5,7,-1,9,11,13,15,-1,-1,17,19,21,23,25,27,29,31,-1,33,35,37,39,41,43,45,-1,47,49,51,-1,-1,-1,-1,-1,53,55,57,59,61,63,65,67,69,71,73,-1,-1,75,77,79,81,-1,83,85,87,-1,-1,-1,89,-1,-1,-1,-1,-1,-1,91,93,95,-1,97,-1,99,101,103,105,107,109,-1,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8542113E-1,3.1256074E-1,1.2494999E-1,1.0581769E-1,0E0,4.4631287E-2,1.3617004E-1,1.1662026E-1,9.607196E-2,0E0,0E0,1.1277509E-1,1.4419648E-1,9.906202E-2,1.4394462E-1,8.390117E-2,5.8847167E-2,1.294153E-1,9.5778584E-2,0E0,1.0309204E-1,5.8474552E-2,2.042053E-1,1.202535E-1,1.6955903E-1,1.18949145E-1,1.3506803E-1,0E0,1.992029E-2,9.951441E-2,9.946352E-2,0E0,0E0,0E0,0E0,0E0,6.774291E-2,9.980589E-2,1.3573289E-2,1.5869844E-1,3.1431522E-2,1.828421E-2,1.8858388E-2,1.0332939E-1,2.4311408E-2,8.919841E-2,7.641225E-2,0E0,0E0,9.8793775E-2,2.1550511E-1,5.2939177E-2,2.3037147E-2,0E0,9.2445284E-2,4.7883667E-2,3.8376752E-2,0E0,0E0,0E0,1.8746136E-1,0E0,0E0,0E0,0E0,0E0,0E0,1.0409613E-1,9.460524E-2,1.8857067E-2,0E0,2.5679685E-2,0E0,4.9106304E-2,1.355762E-2,3.5850346E-2,4.17234E-2,7.883382E-2,1.5605117E-1,0E0,0E0,4.465078E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,20,20,21,21,22,22,23,23,24,24,25,25,26,26,28,28,29,29,30,30,36,36,37,37,38,38,39,39,40,40,41,41,42,42,43,43,44,44,45,45,46,46,49,49,50,50,51,51,52,52,54,54,55,55,56,56,60,60,67,67,68,68,69,69,71,71,73,73,74,74,75,75,76,76,77,77,78,78,81,81],"right_children":[2,4,6,8,-1,10,12,14,16,-1,-1,18,20,22,24,26,28,30,32,-1,34,36,38,40,42,44,46,-1,48,50,52,-1,-1,-1,-1,-1,54,56,58,60,62,64,66,68,70,72,74,-1,-1,76,78,80,82,-1,84,86,88,-1,-1,-1,90,-1,-1,-1,-1,-1,-1,92,94,96,-1,98,-1,100,102,104,106,108,110,-1,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.862E3,6.6E1,2.75E0,1E0,-1.1057859E-2,1.1603809E3,7.34E0,2.23607E5,1.3922E4,8.282269E-3,2.0007177E-3,4.435876E2,1.8495639E5,2.14E0,5E0,1E0,4.60349E5,1.2584705E-2,1.9407515E9,-7.5049535E-3,2.855146E5,1.105075E4,1.1906538E9,6.467172E4,4.252644E7,9.812221E9,2.0217392E0,1.1772086E-4,3.9579997E5,3.3382E4,1.3655363E2,9.797869E-3,1.7578865E-3,3.9920555E-3,-3.458168E-3,4.691226E-3,3.9E1,1.4473684E-1,2E1,1.2E1,1.318849E6,2.44E0,8.571706E6,1.4329944E0,2.4346602E2,1.4189452E9,7E0,-2.7915868E-3,-6.8856953E-3,5.1766735E-1,1.5822886E0,3.4052496E8,1.8958656E7,-3.3053355E-3,1.7097713E2,2.392405E0,5.57767E3,-4.0821475E-3,-9.006493E-3,-1.1782399E-2,9.836066E-1,-2.3422774E-3,1.9240855E-3,-1.359252E-3,-5.5357725E-3,6.2246025E-3,1.8184278E-3,1.0503546E0,8.315271E0,3.0714285E0,5.6958254E-3,1.2E1,-8.607712E-3,5.217E4,1.09285E5,1.0988551E8,8.43E2,1.3402174E1,1.3934426E0,-1.3505857E-3,-7.570353E-3,4.38E2,2.7610285E-3,4.6620197E-3,-5.2805673E-5,-4.9316036E-3,-1.0454458E-3,3.70925E-3,-1.3497229E-5,1.2835732E-3,-7.1764058E-3,3.0312897E-4,3.8691517E-3,-4.472284E-3,2.0445548E-3,-1.1818416E-6,3.856786E-3,-4.93032E-3,-1.3817294E-3,-2.499748E-3,8.1116863E-4,4.740696E-3,1.6076555E-3,2.2501322E-3,-2.0187832E-3,1.2473703E-3,5.1981844E-3,-4.3629663E-4,2.671936E-3,-8.227726E-3,-4.1357163E-4,1.4737297E-3,-2.939405E-3],"split_indices":[2,3,67,109,0,61,55,9,38,0,0,65,42,62,8,112,9,66,12,0,42,54,40,42,7,40,65,0,56,9,65,0,0,0,0,0,2,65,3,3,9,62,12,47,4,40,8,0,0,47,51,7,1,0,42,62,42,0,0,0,65,0,0,0,0,0,0,51,65,62,0,3,0,12,12,7,10,67,62,0,0,8,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.46E2,6.2E2,3.26E2,6.15E2,5E0,1.1E1,3.15E2,2.03E2,4.12E2,6E0,5E0,2.94E2,2.1E1,1.4E2,6.3E1,3.89E2,2.3E1,2.81E2,1.3E1,7E0,1.4E1,7.3E1,6.7E1,4.3E1,2E1,3.06E2,8.3E1,9E0,1.4E1,2.46E2,3.5E1,5E0,8E0,7E0,7E0,6E0,6.7E1,5.6E1,1.1E1,2.9E1,1.4E1,1E1,1E1,2.87E2,1.9E1,3.2E1,5.1E1,9E0,5E0,5.3E1,1.93E2,1.2E1,2.3E1,9E0,5.8E1,3.3E1,2.3E1,5E0,6E0,7E0,2.2E1,9E0,5E0,5E0,5E0,5E0,5E0,2.65E2,2.2E1,1E1,9E0,2.7E1,5E0,3.7E1,1.4E1,1.5E1,3.8E1,1.2E2,7.3E1,6E0,6E0,1.8E1,5E0,1E1,4.8E1,9E0,2.4E1,1E1,1.3E1,9E0,1.3E1,2.47E2,1.8E1,1.6E1,6E0,5E0,5E0,5E0,2.2E1,1.7E1,2E1,6E0,8E0,7E0,8E0,8E0,3E1,2.1E1,9.9E1,5E0,6.8E1,1E1,8E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[-2.1859426E-3,-1.2952569E-2,2.4991162E-2,-1.1532723E-2,-7.037257E-3,3.9029717E-2,-2.3314882E-2,-9.447094E-2,-9.433044E-3,5.1464878E-2,-2.8682904E-2,-7.64292E-2,1.8643895E-2,-5.8973795E-3,-1.9429905E-3,4.638493E-2,-1.2456678E-2,4.511597E-2,1.3764808E-1,2.3598056E-2,-8.190878E-3,-5.830862E-4,-1.06018476E-1,-3.5317563E-2,6.357893E-2,6.614428E-3,7.6127304E-3,-1.01556275E-2,-1.0199434E-1,7.324231E-2,2.7608238E-2,8.534562E-3,3.633442E-3,-1.857786E-3,5.046422E-3,-2.3262643E-3,-7.020204E-3,1.2240814E-3,-4.3194294E-3,-5.520852E-4,1.2641492E-1,2.7186465E-3,-1.829997E-2,-1.3996513E-2,4.6655286E-2,-1.3583231E-2,1.13241235E-2,5.4193333E-2,1.281927E-1,5.3363334E-2,2.28772E-3,-3.8675427E-2,3.5867926E-3,8.783015E-3,1.9705428E-3,-3.7787731E-3,2.928377E-2,-1.07900305E-2,-8.590313E-2,6.568408E-2,-7.88767E-4,2.1184427E-3,-1.1551497E-3,1.8749402E-4,7.170525E-2,8.340323E-3,3.6786774E-3,-5.410386E-4,6.351175E-2,-3.091194E-3,1.7218234E-2,-3.2661315E-3,9.572603E-4,-1.1033919E-3,3.5944323E-3,-6.4845337E-4,3.3590128E-3,-1.8907009E-3,-7.5004934E-3,4.398823E-3,5.4185983E-4,7.033018E-4,5.084053E-3,4.215916E-3,7.3867576E-4,-1.3147378E-3,1.5835799E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":83,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,23,-1,-1,25,27,29,31,33,-1,-1,35,37,39,-1,41,43,45,47,49,-1,-1,51,-1,-1,-1,-1,-1,-1,53,-1,55,57,59,-1,61,63,65,67,69,71,-1,-1,-1,-1,73,75,77,79,-1,-1,-1,-1,81,-1,-1,-1,83,-1,85,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.8561428E-1,1.3592109E-1,1.887567E-1,1.1979036E-1,0E0,1.8250993E-1,1.4182751E-1,2.3852408E-2,1.1483718E-1,9.337464E-2,2.669714E-1,5.157776E-2,8.953682E-2,0E0,0E0,1.29419E-1,1.3144189E-1,8.218315E-2,1.8557712E-2,5.656698E-2,0E0,0E0,4.049267E-2,6.305487E-2,9.6358195E-2,0E0,3.5418697E-2,1.3768522E-1,3.4916568E-1,6.1091512E-2,6.965608E-2,0E0,0E0,6.419415E-2,0E0,0E0,0E0,0E0,0E0,0E0,5.0021455E-2,0E0,5.6464873E-2,1.348772E-1,4.956755E-2,0E0,1.4815404E-2,4.3623537E-2,2.6095897E-2,3.571464E-2,5.7733785E-2,2.966684E-2,0E0,0E0,0E0,0E0,2.99671E-2,1.5072301E-1,7.892197E-2,4.7644004E-2,0E0,0E0,0E0,0E0,7.75654E-2,0E0,0E0,0E0,5.842808E-2,0E0,3.6403548E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,12,12,15,15,16,16,17,17,18,18,19,19,22,22,23,23,24,24,26,26,27,27,28,28,29,29,30,30,33,33,40,40,42,42,43,43,44,44,46,46,47,47,48,48,49,49,50,50,51,51,56,56,57,57,58,58,59,59,64,64,68,68,70,70],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,24,-1,-1,26,28,30,32,34,-1,-1,36,38,40,-1,42,44,46,48,50,-1,-1,52,-1,-1,-1,-1,-1,-1,54,-1,56,58,60,-1,62,64,66,68,70,72,-1,-1,-1,-1,74,76,78,80,-1,-1,-1,-1,82,-1,-1,-1,84,-1,86,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[5.867288E2,3.8652172E1,3.0923106E-10,1.2571428E1,-7.037257E-3,2.8110507E0,3.8909092E0,1.4806053E4,1E0,6.5026445E3,6.5718125E6,1.2593857E0,1.332E3,-5.8973795E-3,-1.9429905E-3,6E0,4.75356E6,5.524E3,3.0612E4,1.8E1,-8.190878E-3,-5.830862E-4,2.82E2,1E0,1.6135485E1,6.614428E-3,6.0921145E4,1.5074378E5,2.1027016E5,3.792E3,1.1075305E0,8.534562E-3,3.633442E-3,5.159139E8,5.046422E-3,-2.3262643E-3,-7.020204E-3,1.2240814E-3,-4.3194294E-3,-5.520852E-4,1E0,2.7186465E-3,1.3E1,2.3600838E6,1.2728624E0,-1.3583231E-2,9.47E4,1.53505E0,3.596E3,3E0,1.0901037E10,2.6224005E6,3.5867926E-3,8.783015E-3,1.9705428E-3,-3.7787731E-3,2.2860665E0,3.1797794E1,1.380863E6,2.153533E6,-7.88767E-4,2.1184427E-3,-1.1551497E-3,1.8749402E-4,6.7321223E-1,8.340323E-3,3.6786774E-3,-5.410386E-4,2.53E2,-3.091194E-3,1.3572786E0,-3.2661315E-3,9.572603E-4,-1.1033919E-3,3.5944323E-3,-6.4845337E-4,3.3590128E-3,-1.8907009E-3,-7.5004934E-3,4.398823E-3,5.4185983E-4,7.033018E-4,5.084053E-3,4.215916E-3,7.3867576E-4,-1.3147378E-3,1.5835799E-3],"split_indices":[61,62,46,64,0,51,55,37,19,4,54,62,38,0,0,3,56,38,2,3,0,0,0,8,65,0,54,42,42,38,51,0,0,7,0,0,0,0,0,0,14,0,3,56,47,0,9,62,2,8,40,37,0,0,0,0,62,65,9,37,0,0,0,0,47,0,0,0,8,0,47,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.74E2,6.98E2,2.76E2,6.92E2,6E0,2.14E2,6.2E1,1.6E1,6.76E2,1.81E2,3.3E1,2.7E1,3.5E1,9E0,7E0,3.4E1,6.42E2,1.7E2,1.1E1,2.5E1,8E0,9E0,1.8E1,1.6E1,1.9E1,9E0,2.5E1,6.27E2,1.5E1,6.4E1,1.06E2,5E0,6E0,2E1,5E0,9E0,9E0,8E0,8E0,9E0,1E1,8E0,1.7E1,5.88E2,3.9E1,5E0,1E1,4.9E1,1.5E1,5.2E1,5.4E1,1.4E1,6E0,5E0,5E0,7E0,1E1,5.64E2,2.4E1,3E1,9E0,5E0,5E0,1.3E1,3.6E1,6E0,9E0,7E0,4.5E1,9E0,4.5E1,9E0,5E0,5E0,5E0,5.44E2,2E1,1.6E1,8E0,1.9E1,1.1E1,1.5E1,2.1E1,2.8E1,1.7E1,1.2E1,3.3E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"87","size_leaf_vector":"1"}},{"base_weights":[5.0132256E-4,-1.5046077E-2,2.1180028E-2,-1.2902328E-2,-7.3871673E-3,2.7707797E-2,-6.666226E-2,-8.855995E-3,-7.749335E-2,2.3812722E-2,1.2978283E-1,-3.1458713E-2,-8.018183E-3,-3.1092396E-2,2.615259E-3,-9.305163E-3,-2.06234E-2,3.6275983E-2,-3.1991024E-3,9.450383E-3,2.269145E-3,-2.4937005E-3,4.259112E-4,-1.0149587E-2,-6.469947E-2,-2.0004269E-3,1.2894404E-1,2.2500569E-2,-4.839152E-3,3.9444428E-2,-2.1163821E-3,3.1324632E-2,-2.042098E-2,2.1167784E-3,-7.2280474E-2,-1.1616226E-1,-2.4773302E-2,7.4127354E-3,-4.989803E-3,1.6308145E-3,1.0063237E-2,-1.6039858E-3,5.1583976E-2,2.9034369E-2,6.6420406E-2,-8.939488E-3,6.147374E-2,-5.285701E-2,1.1722001E-2,-5.8364116E-2,1.5617547E-2,-9.140005E-3,-9.164314E-3,-1.4510685E-1,-2.0712646E-4,1.8783614E-2,-5.9284717E-2,-9.62801E-3,6.9972366E-2,-4.819938E-4,4.8700143E-3,7.645489E-2,2.0531846E-2,7.9257876E-2,-1.8452058E-3,-3.2935385E-3,2.7997136E-2,8.9844465E-2,-1.3696802E-4,2.722389E-3,-7.046623E-2,3.536993E-2,-3.5397878E-3,5.793872E-4,-4.126112E-3,1.2954773E-3,-3.172397E-3,-2.4054095E-3,2.2859054E-3,-4.637857E-3,-1.1895769E-2,2.1030318E-3,-1.9419234E-3,-3.8945146E-3,2.5645294E-4,-2.669394E-4,-4.177667E-3,4.4618978E-4,7.041571E-3,4.5546084E-3,1.9241306E-5,-1.07946275E-4,2.1620835E-3,5.72837E-3,2.5373863E-3,2.6546984E-3,-4.9904734E-4,6.504616E-3,2.5156345E-3,8.3314674E-4,-3.981152E-3,2.3678457E-3,-9.257811E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":84,"left_children":[1,3,5,7,-1,9,11,13,15,17,19,21,-1,23,25,-1,27,29,31,-1,-1,-1,-1,33,35,37,39,41,-1,43,-1,45,47,49,51,53,55,-1,57,-1,-1,-1,59,61,63,65,67,69,71,73,75,-1,77,79,-1,81,83,85,87,-1,-1,89,91,93,-1,-1,95,97,-1,-1,99,101,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1088778E-1,1.6854009E-1,2.3981392E-1,1.4175317E-1,0E0,1.5077683E-1,1.0292806E-1,1.3119213E-1,2.1979797E-1,1.2602516E-1,7.370673E-2,2.1323005E-2,0E0,1.2251297E-1,1.985099E-1,0E0,8.533488E-2,6.8220556E-2,7.139274E-2,0E0,0E0,0E0,0E0,8.3224095E-2,1.3478798E-1,1.5988861E-1,9.135765E-2,2.8339699E-2,0E0,6.743446E-2,0E0,4.9106635E-2,8.402686E-2,7.6033056E-2,1.398709E-1,9.311104E-2,5.9738804E-2,0E0,1.1332529E-1,0E0,0E0,0E0,3.7475273E-2,7.1005166E-2,9.584874E-2,4.3237068E-2,4.2304695E-2,8.269267E-2,8.8031255E-2,3.7921198E-2,8.043838E-2,0E0,3.4971166E-2,8.9927405E-2,0E0,3.0864961E-2,3.561116E-2,9.48254E-2,9.233183E-2,0E0,0E0,4.452364E-2,9.186321E-2,5.949992E-2,0E0,0E0,1.4202149E-2,2.109699E-2,0E0,0E0,4.900083E-2,2.961114E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,16,16,17,17,18,18,23,23,24,24,25,25,26,26,27,27,29,29,31,31,32,32,33,33,34,34,35,35,36,36,38,38,42,42,43,43,44,44,45,45,46,46,47,47,48,48,49,49,50,50,52,52,53,53,55,55,56,56,57,57,58,58,61,61,62,62,63,63,66,66,67,67,70,70,71,71],"right_children":[2,4,6,8,-1,10,12,14,16,18,20,22,-1,24,26,-1,28,30,32,-1,-1,-1,-1,34,36,38,40,42,-1,44,-1,46,48,50,52,54,56,-1,58,-1,-1,-1,60,62,64,66,68,70,72,74,76,-1,78,80,-1,82,84,86,88,-1,-1,90,92,94,-1,-1,96,98,-1,-1,100,102,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.855825E2,4.5158855E6,4.217427E7,9.932432E0,-7.3871673E-3,4.435876E2,3.887931E0,1.5961905E2,2E0,8E0,7.969064E8,2.379747E0,-8.018183E-3,4.75E0,3.4E1,-9.305163E-3,3.9682608E2,1E0,2.3625128E6,9.450383E-3,2.269145E-3,-2.4937005E-3,4.259112E-4,3.09627E5,1.8716538E3,4.08E5,9.6603775E0,1.1290322E1,-4.839152E-3,4.5665024E1,-2.1163821E-3,4.6330696E7,2.6776E4,1.2571428E1,2.135961E4,1.1742308E2,4.309091E0,7.4127354E-3,1.5570383E5,1.6308145E-3,1.0063237E-2,-1.6039858E-3,5.7676364E-2,1.104538E7,7.0993075E0,1.5E1,3.7890625E0,5.294E3,7.3274844E9,1.11E2,9E0,-9.140005E-3,1.5E1,2.85E2,-2.0712646E-4,3.7144188E4,8.274E3,5.1034E4,1.2460085E7,-4.819938E-4,4.8700143E-3,1.1057851E1,6.641212E6,1.6063418E7,-1.8452058E-3,-3.2935385E-3,3.14403E5,5.4E2,-1.3696802E-4,2.722389E-3,8.5888403E8,6.3344407E0,-3.5397878E-3,5.793872E-4,-4.126112E-3,1.2954773E-3,-3.172397E-3,-2.4054095E-3,2.2859054E-3,-4.637857E-3,-1.1895769E-2,2.1030318E-3,-1.9419234E-3,-3.8945146E-3,2.5645294E-4,-2.669394E-4,-4.177667E-3,4.4618978E-4,7.041571E-3,4.5546084E-3,1.9241306E-5,-1.07946275E-4,2.1620835E-3,5.72837E-3,2.5373863E-3,2.6546984E-3,-4.9904734E-4,6.504616E-3,2.5156345E-3,8.3314674E-4,-3.981152E-3,2.3678457E-3,-9.257811E-4],"split_indices":[61,56,54,63,0,65,63,64,26,26,12,63,0,67,3,0,4,111,54,0,0,0,0,9,42,5,65,63,0,67,0,5,38,64,37,61,63,0,42,0,0,0,47,5,44,3,62,12,5,1,8,0,3,2,0,42,1,38,12,0,0,65,54,56,0,0,9,0,0,0,40,62,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,5.51E2,4.14E2,5.44E2,7E0,3.86E2,2.8E1,5.13E2,3.1E1,3.73E2,1.3E1,2.2E1,6E0,1.74E2,3.39E2,9E0,2.2E1,2.55E2,1.18E2,6E0,7E0,1.4E1,8E0,1.08E2,6.6E1,3.28E2,1.1E1,1.5E1,7E0,2.46E2,9E0,3.9E1,7.9E1,9.1E1,1.7E1,2.8E1,3.8E1,5E0,3.23E2,6E0,5E0,5E0,1E1,1.79E2,6.7E1,1.7E1,2.2E1,3.9E1,4E1,1.6E1,7.5E1,5E0,1.2E1,2.2E1,6E0,1.7E1,2.1E1,3.05E2,1.8E1,5E0,5E0,2.6E1,1.53E2,6E1,7E0,6E0,1.1E1,1.5E1,7E0,5E0,3.4E1,3.2E1,8E0,5E0,1.1E1,6.6E1,9E0,7E0,5E0,1.7E1,5E0,1.2E1,5E0,1.5E1,6E0,2.92E2,1.3E1,1.1E1,7E0,2E1,6E0,8.2E1,7.1E1,2E1,4E1,6E0,5E0,5E0,1E1,5E0,2.9E1,2.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"103","size_leaf_vector":"1"}},{"base_weights":[2.022241E-3,-3.9318908E-5,1.23785585E-1,2.4789853E-2,-1.009618E-2,9.7444905E-3,1.5490345E-3,-4.864748E-2,3.0787459E-2,-1.5140815E-1,-6.649917E-3,-9.2772916E-2,4.2538028E-4,7.483426E-3,2.750617E-2,-3.1570483E-2,-1.6048431E-2,-2.2817591E-2,8.772098E-3,-6.31603E-5,-7.3251906E-3,2.101882E-2,8.453258E-2,-3.2920847E-3,6.065236E-4,-1.5276542E-2,-1.04644746E-1,1.7389463E-2,-5.7740252E-2,3.3458322E-2,-1.3540131E-2,6.449387E-3,6.0017586E-2,-1.0673903E-2,-1.1465137E-1,-1.4456075E-2,-1.5684655E-1,1.3610563E-1,1.2826473E-2,-7.6128E-3,-2.5007425E-2,2.1294622E-2,9.766833E-2,-3.3566114E-2,7.871799E-2,1.7732116E-3,4.6809767E-3,-6.323428E-3,-8.8270865E-2,-7.695226E-3,-1.2658348E-3,-3.1148274E-3,1.8851134E-3,-9.3346974E-4,-2.121402E-1,8.384937E-3,3.192965E-3,2.6726121E-2,-2.8857771E-2,-6.8750763E-3,-3.999319E-3,5.664188E-4,3.2238292E-3,5.729547E-3,2.0423257E-3,-6.450697E-4,-4.7335997E-3,5.94706E-3,7.4913085E-4,-4.427103E-5,-3.2452918E-3,-1.0862644E-2,3.740127E-4,-1.2321796E-2,-6.312482E-3,1.3861138E-3,-4.150942E-3,-3.9547766E-3,-4.1100054E-4,-2.8834469E-3,8.384772E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":85,"left_children":[1,3,5,7,9,-1,-1,11,13,15,17,19,-1,-1,21,23,-1,25,27,-1,-1,29,31,-1,-1,33,35,37,39,41,43,-1,45,47,49,51,53,55,57,-1,59,61,63,65,67,-1,-1,69,71,-1,-1,-1,-1,-1,73,-1,-1,75,77,79,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4123742E-1,2.3646984E-1,1.1770956E-1,1.2174009E-1,3.2722998E-1,0E0,0E0,5.442413E-2,1.0389651E-1,3.6195344E-1,1.6451398E-1,7.119547E-2,0E0,0E0,8.99709E-2,2.0265862E-2,0E0,1.9709554E-1,1.9473258E-1,0E0,0E0,9.67498E-2,2.503772E-2,0E0,0E0,1.3416259E-1,1.2484893E-1,1.6076443E-1,1.3163683E-1,1.2679039E-1,1.13495834E-1,0E0,1.3072647E-2,9.531559E-2,5.3624645E-2,3.4644276E-2,1.2425578E-1,2.0582095E-2,1.6879617E-1,0E0,3.380018E-2,6.0453363E-2,3.057532E-2,6.5882646E-2,3.1969175E-2,0E0,0E0,9.216256E-2,2.1262333E-1,0E0,0E0,0E0,0E0,0E0,1.2649775E-2,0E0,0E0,8.239257E-2,8.15529E-2,3.559141E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,7,7,8,8,9,9,10,10,11,11,14,14,15,15,17,17,18,18,21,21,22,22,25,25,26,26,27,27,28,28,29,29,30,30,32,32,33,33,34,34,35,35,36,36,37,37,38,38,40,40,41,41,42,42,43,43,44,44,47,47,48,48,54,54,57,57,58,58,59,59],"right_children":[2,4,6,8,10,-1,-1,12,14,16,18,20,-1,-1,22,24,-1,26,28,-1,-1,30,32,-1,-1,34,36,38,40,42,44,-1,46,48,50,52,54,56,58,-1,60,62,64,66,68,-1,-1,70,72,-1,-1,-1,-1,-1,74,-1,-1,76,78,80,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[4.435876E2,2.4344263E0,1.9407515E9,7.7E1,2.5384614E0,9.7444905E-3,1.5490345E-3,2.5807297E-2,1.3050649E4,4.647654E9,2.8713068E2,1.279012E7,4.2538028E-4,7.483426E-3,8.598131E1,9.3994E4,-1.6048431E-2,1.0700264E6,2.4971938E7,-6.31603E-5,-7.3251906E-3,1.1736916E0,5.97E3,-3.2920847E-3,6.065236E-4,3.0977E4,2.884462E0,1.0918E4,4.75E0,1.4892023E5,1.4E1,6.449387E-3,1.61E9,3E1,1.9548485E2,3.72E2,1.2972876E7,2.8096385E0,1.465E4,-7.6128E-3,6.44833E-1,4.2060452E2,1E1,1.0617E4,4.7E3,1.7732116E-3,4.6809767E-3,1.1021739E1,2.1539131E2,-7.695226E-3,-1.2658348E-3,-3.1148274E-3,1.8851134E-3,-9.3346974E-4,6.819212E6,8.384937E-3,3.192965E-3,2.5401869E1,1.862E3,1.839E3,-3.999319E-3,5.664188E-4,3.2238292E-3,5.729547E-3,2.0423257E-3,-6.450697E-4,-4.7335997E-3,5.94706E-3,7.4913085E-4,-4.427103E-5,-3.2452918E-3,-1.0862644E-2,3.740127E-4,-1.2321796E-2,-6.312482E-3,1.3861138E-3,-4.150942E-3,-3.9547766E-3,-4.1100054E-4,-2.8834469E-3,8.384772E-4],"split_indices":[65,63,12,38,63,0,0,47,37,40,61,7,0,0,67,12,0,60,54,0,0,47,2,0,0,38,65,9,67,42,8,0,5,3,61,38,5,62,38,0,44,61,8,10,2,0,0,55,42,0,0,0,0,0,12,0,0,63,2,2,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.6E2,9.45E2,1.5E1,2.72E2,6.73E2,7E0,8E0,2E1,2.52E2,1.5E1,6.58E2,1.1E1,9E0,5E0,2.47E2,1E1,5E0,3.21E2,3.37E2,5E0,6E0,2.23E2,2.4E1,5E0,5E0,2.95E2,2.6E1,2.99E2,3.8E1,1.64E2,5.9E1,6E0,1.8E1,2.83E2,1.2E1,1E1,1.6E1,1E1,2.89E2,8E0,3E1,1.39E2,2.5E1,4.9E1,1E1,1.3E1,5E0,2.69E2,1.4E1,7E0,5E0,5E0,5E0,5E0,1.1E1,5E0,5E0,2.17E2,7.2E1,2.4E1,6E0,1.18E2,2.1E1,1.6E1,9E0,3.9E1,1E1,5E0,5E0,2.49E2,2E1,5E0,9E0,5E0,6E0,2.12E2,5E0,1.8E1,5.4E1,7E0,1.7E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"81","size_leaf_vector":"1"}},{"base_weights":[-2.8033194E-3,-1.4058303E-3,-1.0494025E-2,-1.2999819E-2,1.552914E-2,-1.0310639E-2,-1.1199138E-1,1.8913383E-2,-5.3499374E-3,-1.242282E-2,7.9031624E-2,-1.4345588E-4,-1.27517395E-2,2.8478881E-2,-2.1966314E-2,-1.0635455E-2,-5.460682E-3,5.9632496E-3,-5.7893644E-6,1.3171302E-1,2.1285886E-2,-5.0292946E-2,1.5898839E-2,-6.655267E-3,-4.926938E-2,2.530975E-3,1.6906768E-1,1.4050506E-2,5.505923E-2,-3.0435503E-2,-6.072877E-3,3.42469E-2,-3.2162787E-3,-1.07067E-2,3.3466764E-2,1.9518477E-3,-6.268028E-2,9.489244E-3,4.3443935E-3,3.0518929E-2,-4.5636054E-3,6.800411E-2,7.3213675E-3,-2.9121635E-3,9.0926886E-5,-2.8185206E-3,6.521471E-3,-5.7531433E-4,3.8550058E-3,-1.3555468E-4,4.092958E-3,-5.9234416E-3,-2.1778077E-3,-5.9323157E-3,1.7630048E-3,-7.0669723E-4,2.6539408E-3,3.7556135E-3,1.619756E-3,1.5885445E-3,-7.792545E-4,-1.7087492E-3,2.0973778E-3,-1.3316137E-3,3.093248E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":86,"left_children":[1,3,-1,5,7,9,11,13,-1,15,17,-1,-1,19,21,23,-1,-1,-1,25,27,29,31,33,35,-1,37,39,41,43,-1,45,-1,47,49,-1,51,-1,-1,53,55,57,59,-1,61,63,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.0121765E-1,1.888883E-1,0E0,1.5084901E-1,1.7530142E-1,1.0609162E-1,2.6513895E-1,1.5000081E-1,0E0,1.0128923E-1,5.0941177E-2,0E0,0E0,2.273908E-1,7.9098836E-2,8.224672E-2,0E0,0E0,0E0,4.8427552E-2,7.033603E-2,6.322935E-2,5.2407496E-2,7.968356E-2,6.38814E-2,0E0,1.5611351E-2,7.4016646E-2,3.1018108E-2,3.454517E-2,0E0,1.0678147E-1,0E0,7.150935E-2,8.99005E-2,0E0,3.9023697E-2,0E0,0E0,1.5643615E-1,7.597624E-2,1.4300346E-2,8.363906E-3,0E0,3.330787E-2,3.9677124E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,9,9,10,10,13,13,14,14,15,15,19,19,20,20,21,21,22,22,23,23,24,24,26,26,27,27,28,28,29,29,31,31,33,33,34,34,36,36,39,39,40,40,41,41,42,42,44,44,45,45],"right_children":[2,4,-1,6,8,10,12,14,-1,16,18,-1,-1,20,22,24,-1,-1,-1,26,28,30,32,34,36,-1,38,40,42,44,-1,46,-1,48,50,-1,52,-1,-1,54,56,58,60,-1,62,64,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8652172E1,4.797342E2,-1.0494025E-2,3.7384888E6,2.871605E2,2.670351E6,7.6E0,1E0,-5.3499374E-3,1E0,6.009E4,-1.4345588E-4,-1.27517395E-2,1.2083E4,1.3711089E6,2.3904348E1,-5.460682E-3,5.9632496E-3,-5.7893644E-6,2.78E2,1.3388E4,5.26E2,3.8807898E6,9.750085E6,1.5487804E0,2.530975E-3,1.94E3,1.3705882E1,1.1450447E9,5E0,-6.072877E-3,2.8E1,-3.2162787E-3,2.5176924E1,2.047619E0,1.9518477E-3,9E0,9.489244E-3,4.3443935E-3,1.1160929E0,1.3683688E7,5.2E1,2.88721E5,-2.9121635E-3,8.579633E-2,2.9307975E5,6.521471E-3,-5.7531433E-4,3.8550058E-3,-1.3555468E-4,4.092958E-3,-5.9234416E-3,-2.1778077E-3,-5.9323157E-3,1.7630048E-3,-7.0669723E-4,2.6539408E-3,3.7556135E-3,1.619756E-3,1.5885445E-3,-7.792545E-4,-1.7087492E-3,2.0973778E-3,-1.3316137E-3,3.093248E-3],"split_indices":[62,4,0,60,67,60,65,6,0,73,1,0,0,9,37,67,0,0,0,0,2,38,37,9,62,0,0,65,7,6,0,3,0,65,62,0,3,0,0,63,54,8,9,0,47,42,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.65E2,9.6E2,5E0,5.7E2,3.9E2,5.56E2,1.4E1,3.81E2,9E0,5.44E2,1.2E1,9E0,5E0,3.09E2,7.2E1,5.36E2,8E0,7E0,5E0,1.9E1,2.9E2,4.1E1,3.1E1,4.87E2,4.9E1,7E0,1.2E1,2.4E2,5E1,3.4E1,7E0,2.6E1,5E0,4.43E2,4.4E1,6E0,4.3E1,7E0,5E0,1.27E2,1.13E2,3.9E1,1.1E1,1.6E1,1.8E1,2E1,6E0,4.36E2,7E0,2.7E1,1.7E1,7E0,3.6E1,5E0,1.22E2,9.7E1,1.6E1,2.7E1,1.2E1,5E0,6E0,1E1,8E0,1.5E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"65","size_leaf_vector":"1"}},{"base_weights":[-2.3780165E-3,-7.504069E-4,-7.3416834E-3,-1.479521E-2,1.5573622E-2,-1.7800761E-2,4.7560286E-2,2.1309912E-2,-2.9047398E-2,-1.56072015E-2,-1.04425326E-1,6.4646853E-3,7.87721E-3,8.471169E-2,1.7332625E-2,-4.7684476E-2,1.8860552E-2,-2.4474775E-2,8.591726E-3,-1.4716475E-3,-7.973623E-3,-2.4500438E-3,4.543798E-2,1.5047923E-4,4.9722376E-3,1.4456923E-2,9.5634155E-2,-2.497761E-2,-5.3615794E-3,3.4239264E-3,-1.7830794E-3,-1.8889714E-2,-1.6909258E-1,5.2913226E-2,-2.0690676E-2,3.6819899E-3,4.5822927E-4,1.8219354E-2,-4.714704E-2,1.6426044E-3,6.5982216E-3,1.6947234E-3,-3.931245E-2,-1.5645625E-2,-1.06528014E-1,-8.519508E-4,-1.3718811E-2,1.570097E-2,1.1166779E-1,-4.5998644E-2,2.4529912E-2,3.0452242E-2,2.888858E-3,-7.2681536E-3,-3.139271E-3,-6.1232124E-2,-1.2290093E-2,-9.3273906E-4,1.6292571E-3,-7.379205E-3,-1.2765996E-3,3.1057259E-3,-1.1865721E-3,8.0003083E-4,6.5884124E-3,5.233606E-4,-2.8782529E-3,-1.2212254E-3,2.260964E-3,2.820543E-3,8.290421E-4,2.9246334E-3,-7.89233E-4,8.567169E-4,-1.9590752E-3,-4.095095E-3,-1.1812964E-3,-2.3816442E-3,1.0638327E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":87,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,21,-1,23,25,27,29,31,33,-1,-1,-1,35,-1,-1,37,39,41,-1,-1,-1,43,45,47,49,-1,-1,51,53,-1,-1,-1,55,57,59,-1,-1,61,63,65,67,69,71,-1,73,75,77,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.4542692E-1,2.200982E-1,0E0,9.7551525E-2,1.1426484E-1,9.204815E-2,1.1943692E-1,9.788555E-2,4.643543E-2,1.0364062E-1,5.3375423E-2,4.6286818E-2,0E0,4.171872E-2,8.238135E-2,5.4133587E-2,5.004108E-2,2.8192377E-1,1.6981402E-1,0E0,0E0,0E0,1.3689974E-2,0E0,0E0,8.4300004E-2,3.1431817E-2,2.7003007E-2,0E0,0E0,0E0,9.528178E-2,2.4259627E-1,1.1193898E-1,9.153919E-2,0E0,0E0,6.367435E-2,1.00700624E-1,0E0,0E0,0E0,1.3463166E-2,7.535939E-2,4.5411855E-2,0E0,0E0,7.1494035E-2,5.4822117E-2,4.7536388E-2,3.6966503E-2,7.1068704E-2,1.8297768E-1,0E0,1.4288597E-2,1.0321084E-2,1.7675325E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,11,11,13,13,14,14,15,15,16,16,17,17,18,18,22,22,25,25,26,26,27,27,31,31,32,32,33,33,34,34,37,37,38,38,42,42,43,43,44,44,47,47,48,48,49,49,50,50,51,51,52,52,54,54,55,55,56,56],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,22,-1,24,26,28,30,32,34,-1,-1,-1,36,-1,-1,38,40,42,-1,-1,-1,44,46,48,50,-1,-1,52,54,-1,-1,-1,56,58,60,-1,-1,62,64,66,68,70,72,-1,74,76,78,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8652172E1,3.478493E2,-7.3416834E-3,1.5570383E5,2.7555334E7,2.0282449E6,1E0,2.82E2,3.416447E-2,5.8036E4,5.90352E5,1.365243E7,7.87721E-3,8.771579E2,3.26172E5,2.47E0,7.6E-2,1.6631816E8,3.38E2,-1.4716475E-3,-7.973623E-3,-2.4500438E-3,1.6E0,1.5047923E-4,4.9722376E-3,1.800554E-2,4.43837E0,9.25E0,-5.3615794E-3,3.4239264E-3,-1.7830794E-3,7.836E3,1E0,1.9559006E1,8.8474586E8,3.6819899E-3,4.5822927E-4,1.1736916E0,2.453E4,1.6426044E-3,6.5982216E-3,1.6947234E-3,5E0,2.4E1,3.8575E4,-8.519508E-4,-1.3718811E-2,9.970589E0,2.1E1,1.6386554E0,2.234E3,2.4E0,3.9914E4,-7.2681536E-3,7.44488E5,2.0900433E1,1.3835462E-9,-9.3273906E-4,1.6292571E-3,-7.379205E-3,-1.2765996E-3,3.1057259E-3,-1.1865721E-3,8.0003083E-4,6.5884124E-3,5.233606E-4,-2.8782529E-3,-1.2212254E-3,2.260964E-3,2.820543E-3,8.290421E-4,2.9246334E-3,-7.89233E-4,8.567169E-4,-1.9590752E-3,-4.095095E-3,-1.1812964E-3,-2.3816442E-3,1.0638327E-3],"split_indices":[62,61,0,42,54,60,24,38,66,1,9,5,0,42,10,62,66,5,0,0,0,0,62,0,0,66,44,67,0,0,0,38,8,67,5,0,0,47,38,0,0,0,8,3,9,0,0,67,8,63,12,63,9,0,9,65,46,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.67E2,9.58E2,9E0,5.15E2,4.43E2,4.92E2,2.3E1,3.93E2,5E1,4.81E2,1.1E1,1.8E1,5E0,2.2E1,3.71E2,3.6E1,1.4E1,3.52E2,1.29E2,6E0,5E0,7E0,1.1E1,5E0,1.7E1,3.59E2,1.2E1,2.8E1,8E0,7E0,7E0,3.4E2,1.2E1,5.1E1,7.8E1,5E0,6E0,3.39E2,2E1,6E0,6E0,5E0,2.3E1,3.29E2,1.1E1,6E0,6E0,3.2E1,1.9E1,5E1,2.8E1,1.88E2,1.51E2,5E0,1.5E1,1.2E1,1.1E1,3.03E2,2.6E1,6E0,5E0,1.4E1,1.8E1,5E0,1.4E1,1.1E1,3.9E1,9E0,1.9E1,5.4E1,1.34E2,3.7E1,1.14E2,1E1,5E0,6E0,6E0,5E0,6E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"79","size_leaf_vector":"1"}},{"base_weights":[-2.4142833E-3,-7.68001E-4,-7.5485054E-3,-1.68874E-2,1.5287041E-2,-2.009068E-2,5.173151E-2,5.355696E-2,8.550852E-3,-1.6823325E-2,-8.2291335E-2,6.5372023E-3,7.3165545E-4,4.1044205E-2,5.95225E-3,-4.4488378E-2,1.2925733E-2,-2.61191E-3,-3.0520126E-2,-8.9216E-3,-9.06787E-3,-1.9500013E-3,2.0136544E-3,5.237786E-2,-2.0804857E-3,-7.9658754E-2,2.9896768E-2,-7.3845033E-3,2.5866073E-2,2.864423E-2,-1.3786257E-2,-2.5909908E-2,-1.04063705E-1,-2.8345648E-3,2.75739E-3,6.879804E-2,-2.969705E-3,-3.9053228E-2,-8.418303E-3,-1.1075501E-3,3.6507358E-3,-5.23358E-2,8.19501E-3,1.1960624E-1,1.8962413E-2,-7.966578E-3,7.40101E-2,-1.9286169E-2,3.7980294E-3,-3.1488955E-2,4.059371E-2,-8.362589E-3,-6.04614E-4,2.648638E-4,7.8609355E-2,-2.2642757E-3,3.0748975E-3,2.533265E-4,-3.6760806E-3,-7.824215E-2,5.495539E-3,4.1773967E-2,-9.375314E-3,8.06863E-3,2.6939437E-3,2.4427626E-2,-4.584093E-3,-2.1866667E-3,1.95656E-3,2.4680549E-3,5.2989484E-3,-5.982724E-4,-5.606855E-3,-3.1095652E-3,-1.0371635E-3,2.7001875E-3,3.3112057E-4,6.856422E-3,2.8314777E-3,-6.3281925E-3,-2.3697484E-3,2.5488304E-3,-1.4972645E-3,7.43848E-4,5.327776E-3,-2.7138414E-3,5.6212593E-4,2.322689E-3,3.803519E-4],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":88,"left_children":[1,3,-1,5,7,9,11,13,15,17,19,-1,21,23,-1,25,27,29,31,33,-1,-1,-1,35,-1,37,39,41,43,45,47,49,51,-1,-1,53,55,57,-1,-1,-1,59,61,63,65,67,69,71,-1,73,75,-1,-1,-1,77,-1,-1,-1,-1,79,81,83,85,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.595049E-1,2.525859E-1,0E0,1.0808465E-1,1.2560305E-1,9.332152E-2,9.993224E-2,6.299171E-2,9.733588E-2,8.6345285E-2,1.8917963E-1,0E0,2.9189024E-2,6.464401E-2,0E0,8.609844E-2,1.0164004E-1,7.688436E-2,7.392946E-2,5.711577E-2,0E0,0E0,0E0,5.20069E-2,0E0,8.383013E-2,3.0730538E-2,1.0617711E-1,1.5045944E-1,9.692069E-2,8.6793765E-2,8.0678E-2,8.7789565E-2,0E0,0E0,2.6786938E-2,4.765963E-2,3.0832756E-2,0E0,0E0,0E0,5.8859743E-2,6.7060865E-2,4.3364197E-2,1.446439E-1,6.677565E-2,1.5681088E-2,9.8410316E-2,0E0,6.277031E-2,9.595575E-3,0E0,0E0,0E0,3.719838E-2,0E0,0E0,0E0,0E0,3.5686553E-2,2.6043126E-2,7.149872E-2,7.986804E-2,0E0,0E0,8.778119E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,3,3,4,4,5,5,6,6,7,7,8,8,9,9,10,10,12,12,13,13,15,15,16,16,17,17,18,18,19,19,23,23,25,25,26,26,27,27,28,28,29,29,30,30,31,31,32,32,35,35,36,36,37,37,41,41,42,42,43,43,44,44,45,45,46,46,47,47,49,49,50,50,54,54,59,59,60,60,61,61,62,62,65,65],"right_children":[2,4,-1,6,8,10,12,14,16,18,20,-1,22,24,-1,26,28,30,32,34,-1,-1,-1,36,-1,38,40,42,44,46,48,50,52,-1,-1,54,56,58,-1,-1,-1,60,62,64,66,68,70,72,-1,74,76,-1,-1,-1,78,-1,-1,-1,-1,80,82,84,86,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[3.8652172E1,1.158E3,-7.5485054E-3,1.670046E1,9.25E0,3E1,2.9417648E2,2.083138E6,1.3402174E1,2.1894736E0,1.365243E7,6.5372023E-3,1.7145834E1,2.4946861E4,5.95225E-3,2.4131148E0,8.484036E5,3.6401E4,3.578882E1,4E1,-9.06787E-3,-1.9500013E-3,2.0136544E-3,2.17062E5,-2.0804857E-3,5.2141205E6,4.829553E4,1E0,1.1057693E0,1.5031847E0,1.8365753E5,7.45E0,1E0,-2.8345648E-3,2.75739E-3,1.31E2,3.662857E0,1.1E1,-8.418303E-3,-1.1075501E-3,3.6507358E-3,9.8897E2,3.82E2,8.432552E6,1.1865458E9,3.421543E2,3.21875E0,3.57E2,3.7980294E-3,1.43E2,6.763314E7,-8.362589E-3,-6.04614E-4,2.648638E-4,4.1E1,-2.2642757E-3,3.0748975E-3,2.533265E-4,-3.6760806E-3,2.6363637E0,2.2125713E1,1.5608E4,4.4615383E0,8.06863E-3,2.6939437E-3,9.122396E7,-4.584093E-3,-2.1866667E-3,1.95656E-3,2.4680549E-3,5.2989484E-3,-5.982724E-4,-5.606855E-3,-3.1095652E-3,-1.0371635E-3,2.7001875E-3,3.3112057E-4,6.856422E-3,2.8314777E-3,-6.3281925E-3,-2.3697484E-3,2.5488304E-3,-1.4972645E-3,7.43848E-4,5.327776E-3,-2.7138414E-3,5.6212593E-4,2.322689E-3,3.803519E-4],"split_indices":[62,2,0,65,67,3,4,12,67,62,5,0,67,56,0,62,37,9,55,3,0,0,0,1,0,54,42,24,63,62,42,63,23,0,0,38,67,8,0,0,0,4,0,56,41,42,63,0,0,2,7,0,0,0,0,0,0,0,0,62,65,38,63,0,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.83E2,9.74E2,9E0,4.86E2,4.88E2,4.65E2,2.1E1,7.2E1,4.16E2,4.43E2,2.2E1,7E0,1.4E1,6.3E1,9E0,3.1E1,3.85E2,2.18E2,2.25E2,1.4E1,8E0,7E0,7E0,5.6E1,7E0,2.1E1,1E1,1.5E2,2.35E2,5.7E1,1.61E2,2.13E2,1.2E1,8E0,6E0,4.3E1,1.3E1,1.6E1,5E0,5E0,5E0,3.8E1,1.12E2,1.5E1,2.2E2,3.2E1,2.5E1,1.53E2,8E0,1.97E2,1.6E1,6E0,6E0,6E0,3.7E1,8E0,5E0,8E0,8E0,2.6E1,1.2E1,3.8E1,7.4E1,7E0,8E0,2.11E2,9E0,1.8E1,1.4E1,1.8E1,7E0,1.45E2,8E0,3.9E1,1.58E2,1E1,6E0,6E0,3.1E1,7E0,1.9E1,5E0,7E0,2.9E1,9E0,2.2E1,5.2E1,8.1E1,1.3E2],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-5.7494856E-4,-3.7475096E-3,5.2446116E-2,-9.402724E-3,2.9744226E-2,6.680933E-2,-3.5258124E-3,-7.2104246E-3,-1.2993391E-1,-3.9277535E-2,3.691397E-2,9.16615E-2,2.143675E-2,-4.771521E-3,-6.970049E-2,-1.3822688E-2,-1.6309538E-4,-3.8982246E-3,5.136669E-4,5.4346617E-2,2.5419146E-4,6.7075655E-2,7.882602E-3,3.4968257E-3,-3.2059464E-3,-5.814583E-3,4.926535E-3,-3.9836008E-2,-7.325047E-3,6.2342146E-3,6.976619E-2,-1.066875E-2,2.9662668E-3,1.2305175E-3,7.791631E-2,1.8480794E-3,-1.463448E-3,-5.2561117E-3,-4.781789E-3,-2.0739235E-2,-3.73622E-3,4.083641E-2,-2.0619626E-3,4.623062E-2,1.00575805E-1,-4.3251008E-2,1.9045312E-2,1.9067744E-3,5.133089E-3,-2.6785654E-3,-5.771741E-2,-1.9324002E-3,1.1383172E-3,-7.503175E-4,4.2692036E-3,-1.1710953E-3,5.8186553E-2,6.0279716E-2,7.887569E-3,-3.6960177E-3,3.5552736E-4,1.7196746E-3,-1.2142188E-3,-4.3818617E-4,4.6574525E-4,-5.4150573E-3,-5.88518E-4,2.543702E-4,3.1432617E-3,4.1173194E-3,1.5824015E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":89,"left_children":[1,3,5,7,9,11,-1,13,15,17,19,21,23,25,27,-1,-1,-1,-1,29,31,33,-1,-1,35,37,-1,39,-1,41,43,45,-1,-1,47,-1,-1,-1,49,51,-1,53,-1,55,57,59,61,-1,-1,63,65,-1,-1,-1,-1,-1,67,69,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.6386737E-1,1.7434098E-1,1.06536895E-1,2.0702031E-1,6.7267366E-2,5.441542E-2,0E0,1.1763063E-1,2.9624173E-1,3.021158E-2,7.731448E-2,5.0147593E-2,2.5490765E-2,8.694887E-2,7.007159E-2,0E0,0E0,0E0,0E0,6.0398653E-2,2.8502678E-2,9.364516E-3,0E0,0E0,1.8327668E-2,8.159043E-2,0E0,1.6200796E-2,0E0,3.8636543E-2,4.030785E-2,3.4709025E-2,0E0,0E0,1.9543879E-2,0E0,0E0,0E0,8.156562E-2,1.7113004E-2,0E0,3.941225E-2,0E0,3.3084773E-2,6.2481046E-2,3.2120343E-2,1.6447825E-2,0E0,0E0,6.0917053E-2,7.290012E-2,0E0,0E0,0E0,0E0,0E0,1.5746921E-2,1.038155E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,10,10,11,11,12,12,13,13,14,14,19,19,20,20,21,21,24,24,25,25,27,27,29,29,30,30,31,31,34,34,38,38,39,39,41,41,43,43,44,44,45,45,46,46,49,49,50,50,56,56,57,57],"right_children":[2,4,6,8,10,12,-1,14,16,18,20,22,24,26,28,-1,-1,-1,-1,30,32,34,-1,-1,36,38,-1,40,-1,42,44,46,-1,-1,48,-1,-1,-1,50,52,-1,54,-1,56,58,60,62,-1,-1,64,66,-1,-1,-1,-1,-1,68,70,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.9856038E3,3.2911258E6,5.619497E7,2.0295714E7,5.02E2,2.4508352E9,-3.5258124E-3,4.217427E7,4.771E3,4.49239E6,1.2706E4,1.7307587E2,6.072304E5,9.370294E2,3.481132E0,-1.3822688E-2,-1.6309538E-4,-3.8982246E-3,5.136669E-4,1.411E3,1E0,6.374563E5,7.882602E-3,3.4968257E-3,7.3274844E9,6.25E0,4.926535E-3,2.0926375E5,-7.325047E-3,1.08103E5,4.28649E5,1.7540984E0,2.9662668E-3,1.2305175E-3,1.6E1,1.8480794E-3,-1.463448E-3,-5.2561117E-3,3.842105E1,1.3926982E3,-3.73622E-3,9.475728E0,-2.0619626E-3,1E0,1.29223805E5,6.165E3,1.625E3,1.9067744E-3,5.133089E-3,3.7667856E2,3E0,-1.9324002E-3,1.1383172E-3,-7.503175E-4,4.2692036E-3,-1.1710953E-3,2.7828E4,1.14E3,7.887569E-3,-3.6960177E-3,3.5552736E-4,1.7196746E-3,-1.2142188E-3,-4.3818617E-4,4.6574525E-4,-5.4150573E-3,-5.88518E-4,2.543702E-4,3.1432617E-3,4.1173194E-3,1.5824015E-3],"split_indices":[61,37,56,56,2,12,0,54,2,37,38,65,37,55,63,0,0,0,0,38,74,37,0,0,5,61,0,42,0,9,9,62,0,0,3,0,0,0,55,4,0,67,0,109,42,2,0,0,0,61,8,0,0,0,0,0,12,10,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.72E2,9.18E2,5.4E1,7.86E2,1.32E2,4.9E1,5E0,7.73E2,1.3E1,1.2E1,1.2E2,3.1E1,1.8E1,7.45E2,2.8E1,5E0,8E0,6E0,6E0,8.1E1,3.9E1,2.5E1,6E0,5E0,1.3E1,7.39E2,6E0,2.2E1,6E0,2E1,6.1E1,3.4E1,5E0,6E0,1.9E1,5E0,8E0,6E0,7.33E2,1.6E1,6E0,1.2E1,8E0,3.6E1,2.5E1,1.6E1,1.8E1,1E1,9E0,7.06E2,2.7E1,1.1E1,5E0,6E0,6E0,5E0,3.1E1,1.7E1,8E0,9E0,7E0,1.3E1,5E0,4.61E2,2.45E2,1.1E1,1.6E1,5E0,2.6E1,7E0,1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"71","size_leaf_vector":"1"}},{"base_weights":[-5.0790836E-3,-1.394457E-2,2.0115422E-2,-1.0580024E-2,-1.5501481E-1,-7.852776E-3,2.6449118E-2,-1.9515125E-2,7.863645E-3,-1.4340324E-2,-5.6896437E-2,3.335071E-2,-4.975561E-2,-1.1302419E-1,-1.69439E-2,1.525718E-2,-4.3643832E-2,-6.042748E-4,-4.235653E-3,6.687132E-2,1.8576294E-2,-4.7124256E-3,-6.754365E-3,6.971786E-4,-1.1301885E-2,7.733411E-2,-2.0193264E-2,-7.954836E-3,3.542099E-2,-2.5405629E-2,-4.9314667E-3,5.5117764E-2,7.3340316E-3,2.4218405E-2,-4.7214064E-3,-3.3828386E-3,2.3623214E-3,5.1902393E-3,1.9435752E-3,-1.6985679E-2,-1.0212792E-1,-3.0496987E-4,-4.0499293E-3,6.611171E-2,6.876314E-3,-3.129729E-3,1.2126287E-2,4.483597E-2,6.504326E-3,1.6895931E-2,6.8865563E-3,-2.7600821E-2,1.9557528E-2,-2.188724E-3,-8.623805E-3,-3.4983143E-2,2.5179096E-2,-5.8473506E-6,8.065478E-2,-2.5937714E-2,3.5085928E-2,-1.2444506E-3,1.8476483E-3,1.057385E-2,7.518912E-2,-1.8565452E-2,3.244174E-2,-1.0008909E-3,-3.7848055E-3,2.0173036E-3,-5.1466824E-4,1.4175761E-3,-3.0178404E-3,-2.6328728E-4,3.0896089E-3,5.4450184E-3,2.3093896E-3,-4.558235E-3,1.1781082E-3,2.4408095E-3,-1.3914922E-3,-4.94057E-4,4.019334E-3,5.269663E-3,2.4238275E-3,1.2755198E-3,-2.6063214E-3,5.2869593E-4,2.7715461E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":90,"left_children":[1,3,5,7,9,-1,11,13,15,-1,17,19,21,23,25,27,29,-1,-1,31,33,-1,35,-1,-1,37,39,41,43,45,-1,47,-1,49,-1,-1,-1,-1,-1,51,53,55,-1,57,59,-1,61,63,-1,65,-1,67,69,-1,-1,71,73,-1,75,77,79,-1,-1,81,83,85,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[2.0845345E-1,3.2585967E-1,2.9591566E-1,1.1139079E-1,2.2425428E-1,0E0,1.257798E-1,1.0748188E-1,8.482762E-2,0E0,1.5408073E-2,1.0622646E-1,4.386897E-2,2.0426895E-1,1.3757817E-1,9.108205E-2,2.9287737E-2,0E0,0E0,6.2028557E-2,1.05661206E-1,0E0,4.9489837E-2,0E0,0E0,1.3671599E-2,1.1127457E-1,5.519747E-2,9.065531E-2,3.711374E-2,0E0,4.6191126E-2,0E0,1.302729E-1,0E0,0E0,0E0,0E0,0E0,1.6152301E-1,6.485893E-2,7.511508E-2,0E0,4.817514E-2,5.1821955E-2,0E0,1.5089783E-2,5.640293E-2,0E0,7.722965E-2,0E0,1.0302389E-1,6.939566E-2,0E0,0E0,7.289992E-2,6.297951E-2,0E0,4.0623844E-2,9.8983124E-2,3.577807E-2,0E0,0E0,4.4498574E-2,1.9613668E-2,7.6018475E-2,5.5035546E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,10,10,11,11,12,12,13,13,14,14,15,15,16,16,19,19,20,20,22,22,25,25,26,26,27,27,28,28,29,29,31,31,33,33,39,39,40,40,41,41,43,43,44,44,46,46,47,47,49,49,51,51,52,52,55,55,56,56,58,58,59,59,60,60,63,63,64,64,65,65,66,66],"right_children":[2,4,6,8,10,-1,12,14,16,-1,18,20,22,24,26,28,30,-1,-1,32,34,-1,36,-1,-1,38,40,42,44,46,-1,48,-1,50,-1,-1,-1,-1,-1,52,54,56,-1,58,60,-1,62,64,-1,66,-1,68,70,-1,-1,72,74,-1,76,78,80,-1,-1,82,84,86,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3992583E10,2.1922675E8,5.1942E4,1.0590052E8,1.5209424E1,-7.852776E-3,2.6652174E0,1.1160929E0,2.3821254E7,-1.4340324E-2,4.9269806E9,4.88E2,1.4598765E0,7.13E2,7E0,4.8709216E5,4.9506173E0,-6.042748E-4,-4.235653E-3,1.8E1,4.1325716E7,-4.7124256E-3,1.9069755E5,6.971786E-4,-1.1301885E-2,1.967742E0,2.3876712E7,8.640632E7,7.5052085E0,1.361E3,-4.9314667E-3,1.7265024E3,7.3340316E-3,4.907764E3,-4.7214064E-3,-3.3828386E-3,2.3623214E-3,5.1902393E-3,1.9435752E-3,3.5947333E2,1.651E3,1.23E2,-4.0499293E-3,8.23E2,2.1770449E0,-3.129729E-3,1.124E3,4.68418E5,6.504326E-3,1.3026532E6,6.8865563E-3,5.5586E4,2.3695534E9,-2.188724E-3,-8.623805E-3,9.07021E5,5.7791035E4,-5.8473506E-6,2.3625128E6,2.4076016E5,1E0,-1.2444506E-3,1.8476483E-3,5.5303917E8,1.187E3,6.374563E5,3.5057293E1,-1.0008909E-3,-3.7848055E-3,2.0173036E-3,-5.1466824E-4,1.4175761E-3,-3.0178404E-3,-2.6328728E-4,3.0896089E-3,5.4450184E-3,2.3093896E-3,-4.558235E-3,1.1781082E-3,2.4408095E-3,-1.3914922E-3,-4.94057E-4,4.019334E-3,5.269663E-3,2.4238275E-3,1.2755198E-3,-2.6063214E-3,5.2869593E-4,2.7715461E-3],"split_indices":[40,5,1,7,65,0,55,63,9,0,40,0,63,2,3,37,63,0,0,3,60,0,42,0,0,63,54,12,65,38,0,4,0,4,0,0,0,0,0,61,2,0,0,2,62,0,2,9,0,37,0,1,40,0,0,9,42,0,54,42,6,0,0,5,10,37,67,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.31E2,6.89E2,2.42E2,6.74E2,1.5E1,7E0,2.35E2,4.54E2,2.2E2,5E0,1E1,2.16E2,1.9E1,1.1E1,4.43E2,1.93E2,2.7E1,5E0,5E0,6.5E1,1.51E2,8E0,1.1E1,6E0,5E0,1.4E1,4.29E2,9E1,1.03E2,2.2E1,5E0,5.9E1,6E0,1.45E2,6E0,5E0,6E0,6E0,8E0,4.14E2,1.5E1,8.3E1,7E0,4.9E1,5.4E1,1E1,1.2E1,5.4E1,5E0,1.38E2,7E0,3.21E2,9.3E1,1E1,5E0,3.5E1,4.8E1,9E0,4E1,2.5E1,2.9E1,5E0,7E0,2.6E1,2.8E1,4.2E1,9.6E1,2.9E2,3.1E1,5.2E1,4.1E1,1.1E1,2.4E1,2.8E1,2E1,1.7E1,2.3E1,1E1,1.5E1,2.3E1,6E0,2.1E1,5E0,9E0,1.9E1,1.9E1,2.3E1,5.5E1,4.1E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"89","size_leaf_vector":"1"}},{"base_weights":[-3.698148E-4,-8.537099E-3,1.9557353E-2,6.738842E-3,-4.2404085E-2,-5.0040865E-3,2.2717789E-2,1.0368746E-3,6.8721816E-2,-3.468013E-2,-9.951984E-3,6.445974E-2,1.4523369E-2,4.378895E-3,-8.159167E-2,9.804969E-2,1.8601898E-2,-7.421122E-2,3.3134464E-3,2.240137E-2,9.166897E-2,3.3889703E-2,-7.2637442E-3,6.310484E-2,6.603912E-4,-1.7234652E-3,-7.2795176E-3,7.2418326E-3,5.5655155E-2,2.0712183E-3,-3.0665277E-4,-5.4211207E-2,-1.771728E-2,-1.7055606E-2,6.0664937E-2,3.3245017E-3,-9.247018E-4,7.528751E-4,5.0147343E-3,1.2191733E-2,8.3993316E-2,-4.2499676E-2,9.746352E-3,9.9467054E-2,-1.2341434E-3,-1.2951533E-2,1.3535102E-2,3.715422E-3,7.470262E-5,-3.5892215E-2,-9.6587455E-3,-3.2659758E-2,6.467776E-2,2.5166342E-2,7.589343E-3,-4.369216E-2,2.3507714E-2,5.678547E-3,2.5956474E-2,4.423984E-3,-7.256127E-2,4.555958E-2,-1.9477364E-2,7.3457527E-4,8.414229E-3,-1.7624257E-2,4.515045E-3,3.8420595E-2,-3.5061354E-3,-1.1222379E-1,-2.0966077E-2,-1.1010885E-2,-8.677213E-2,5.0221244E-3,-1.9397875E-4,6.200268E-2,-2.1829798E-3,1.3266221E-3,-4.410634E-3,4.270017E-3,1.36587145E-2,2.5148392E-3,-9.4319315E-5,1.1660861E-3,-1.4303012E-3,-4.7258247E-4,-4.422595E-3,1.5779605E-2,4.936539E-3,-3.6727406E-2,1.8279282E-3,1.0170718E-3,-1.3765986E-3,5.01497E-3,1.3710503E-3,8.61033E-4,-1.5116903E-3,-8.495245E-3,-2.436463E-3,2.5840625E-3,-1.6348722E-3,-2.7748505E-3,8.9316553E-4,-6.0908394E-3,-1.5588929E-3,3.7314912E-4,4.504585E-3,1.5706086E-4,4.010578E-3,-7.8299595E-4,2.312574E-3,-5.8482754E-5,-6.8394667E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":91,"left_children":[1,3,5,7,9,-1,11,13,15,17,-1,19,21,23,25,27,29,31,33,35,37,39,41,43,45,-1,-1,-1,47,-1,-1,49,-1,51,53,-1,-1,-1,-1,55,57,59,61,63,-1,65,67,-1,-1,69,-1,71,73,75,-1,77,79,-1,81,83,85,87,89,-1,-1,91,-1,93,95,97,99,101,103,-1,-1,105,-1,-1,-1,-1,107,-1,-1,-1,-1,-1,-1,109,-1,111,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.5787253E-1,3.5622257E-1,1.159993E-1,1.6780557E-1,2.7699885E-1,0E0,9.346992E-2,1.20757066E-1,5.6972533E-2,3.096429E-1,0E0,4.944341E-2,9.813546E-2,9.167482E-2,5.106719E-2,5.2813485E-2,1.0797668E-2,6.0071594E-1,1.24863416E-1,4.103735E-2,3.1670034E-2,1.3269503E-1,6.6344544E-2,8.419075E-2,6.957413E-2,0E0,0E0,0E0,2.0987276E-2,0E0,0E0,2.6327568E-1,0E0,1.0286736E-1,9.816279E-2,0E0,0E0,0E0,0E0,5.6044262E-2,7.92622E-2,5.1102616E-2,7.9382055E-2,1.20389E-1,0E0,1.011512E-1,8.6646944E-2,0E0,0E0,9.676931E-2,0E0,7.7533185E-2,3.9444797E-2,6.14392E-2,0E0,5.8411382E-2,4.8007105E-2,0E0,1.2758333E-2,1.1723741E-2,3.0254647E-2,5.991546E-2,4.4100396E-2,0E0,0E0,8.923805E-2,0E0,4.810211E-2,7.88626E-2,4.962848E-2,8.290395E-2,7.3499E-2,4.0546685E-2,0E0,0E0,2.7613413E-2,0E0,0E0,0E0,0E0,4.8307568E-2,0E0,0E0,0E0,0E0,0E0,0E0,2.7535133E-2,0E0,1.2935148E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,6,6,7,7,8,8,9,9,11,11,12,12,13,13,14,14,15,15,16,16,17,17,18,18,19,19,20,20,21,21,22,22,23,23,24,24,28,28,31,31,33,33,34,34,39,39,40,40,41,41,42,42,43,43,45,45,46,46,49,49,51,51,52,52,53,53,55,55,56,56,58,58,59,59,60,60,61,61,62,62,65,65,67,67,68,68,69,69,70,70,71,71,72,72,75,75,80,80,87,87,89,89],"right_children":[2,4,6,8,10,-1,12,14,16,18,-1,20,22,24,26,28,30,32,34,36,38,40,42,44,46,-1,-1,-1,48,-1,-1,50,-1,52,54,-1,-1,-1,-1,56,58,60,62,64,-1,66,68,-1,-1,70,-1,72,74,76,-1,78,80,-1,82,84,86,88,90,-1,-1,92,-1,94,96,98,100,102,104,-1,-1,106,-1,-1,-1,-1,108,-1,-1,-1,-1,-1,-1,110,-1,112,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[2.026453E6,3.72381E5,1.424196E-1,1.6293909E3,2.855146E5,-5.0040865E-3,4.9E4,3.9661028E7,3.6444E4,5.5586E4,-9.951984E-3,5.1604336E7,2.2993813E0,6E0,2.47E0,6.072304E5,2.1668362E1,4.6712E4,1.212945E6,1.4492754E0,1.4473684E-1,2.1E1,3.3534848E2,7.6825094E0,4.385E4,-1.7234652E-3,-7.2795176E-3,7.2418326E-3,1.31E2,2.0712183E-3,-3.0665277E-4,9.369565E0,-1.771728E-2,2.2187027E5,1.68991E6,3.3245017E-3,-9.247018E-4,7.528751E-4,5.0147343E-3,8.8E1,1.1057851E1,1.0252778E1,1.390602E8,1.766298E2,-1.2341434E-3,9.374418E0,1.15E2,3.715422E-3,7.470262E-5,3.06E2,-9.6587455E-3,1.3572786E0,1.835821E1,9.0947875E4,7.589343E-3,1.6521739E0,8.589474E0,5.678547E-3,8.943293E-1,1E0,7.450166E4,7.09617E6,9.026191E8,7.3457527E-4,8.414229E-3,6E0,4.515045E-3,9E0,3.6444E4,1.3728223E0,3.6616925E5,1.244E3,1.6E1,5.0221244E-3,-1.9397875E-4,1.594E3,-2.1829798E-3,1.3266221E-3,-4.410634E-3,4.270017E-3,3.6086957E0,2.5148392E-3,-9.4319315E-5,1.1660861E-3,-1.4303012E-3,-4.7258247E-4,-4.422595E-3,7.234E3,4.936539E-3,3.4647146E8,1.8279282E-3,1.0170718E-3,-1.3765986E-3,5.01497E-3,1.3710503E-3,8.61033E-4,-1.5116903E-3,-8.495245E-3,-2.436463E-3,2.5840625E-3,-1.6348722E-3,-2.7748505E-3,8.9316553E-4,-6.0908394E-3,-1.5588929E-3,3.7314912E-4,4.504585E-3,1.5706086E-4,4.010578E-3,-7.8299595E-4,2.312574E-3,-5.8482754E-5,-6.8394667E-3],"split_indices":[9,9,67,64,42,0,12,54,9,1,0,7,62,3,62,37,65,1,9,65,65,67,64,65,12,0,0,0,8,0,0,63,0,42,37,0,0,0,0,10,65,67,7,4,0,62,0,0,0,2,0,47,65,42,0,62,67,0,47,25,37,12,5,0,0,10,0,3,9,62,54,2,8,0,0,2,0,0,0,0,63,0,0,0,0,0,0,38,0,12,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.68E2,6.87E2,2.81E2,4.74E2,2.13E2,6E0,2.75E2,4.35E2,3.9E1,2.05E2,8E0,4.4E1,2.31E2,4.19E2,1.6E1,2.4E1,1.5E1,1E2,1.05E2,1.8E1,2.6E1,1.22E2,1.09E2,2.4E1,3.95E2,1.1E1,5E0,9E0,1.5E1,7E0,8E0,9.5E1,5E0,7.8E1,2.7E1,8E0,1E1,5E0,2.1E1,8.6E1,3.6E1,3.5E1,7.4E1,1.7E1,7E0,1.92E2,2.03E2,1E1,5E0,8.6E1,9E0,6.6E1,1.2E1,2.1E1,6E0,1.4E1,7.2E1,2.1E1,1.5E1,1.4E1,2.1E1,3.3E1,4.1E1,9E0,8E0,1.85E2,7E0,8.2E1,1.21E2,1.3E1,7.3E1,4.8E1,1.8E1,7E0,5E0,1.4E1,7E0,6E0,8E0,8E0,6.4E1,7E0,8E0,9E0,5E0,6E0,1.5E1,2.3E1,1E1,3.2E1,9E0,4.3E1,1.42E2,8E0,7.4E1,6.9E1,5.2E1,5E0,8E0,1.1E1,6.2E1,1.8E1,3E1,9E0,9E0,6E0,8E0,5.7E1,7E0,1.2E1,1.1E1,2.5E1,7E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"113","size_leaf_vector":"1"}},{"base_weights":[5.3193286E-4,-3.873534E-3,4.373913E-2,-5.2762064E-3,6.88799E-3,2.814519E-2,9.970916E-2,-3.9178855E-3,-9.410161E-2,5.382147E-3,1.9732306E-2,1.3607116E-3,1.3832213E-1,-2.2431284E-3,-7.740099E-2,-1.064891E-2,2.5401437E-3,-1.187839E-2,3.807838E-2,3.4325388E-3,8.8315E-3,-3.6387201E-3,9.389073E-2,-8.037495E-3,8.036212E-3,1.2623531E-2,-3.5582166E-3,1.5596449E-4,6.939881E-2,-9.861671E-5,-3.52852E-2,7.984654E-3,6.242521E-4,-1.8158016E-3,2.4994067E-3,-8.8301447E-4,1.734172E-3,-1.2820681E-3,2.101452E-3,1.0412077E-1,1.7714841E-4,3.3748886E-3,-2.738927E-2,-2.2345964E-2,-8.007428E-3,2.786508E-3,6.2718824E-3,-8.554466E-4,4.4068028E-4,-2.9945818E-3,1.3521363E-3,-4.543664E-3,-4.253019E-5],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":92,"left_children":[1,3,5,7,-1,9,11,13,15,-1,17,-1,19,21,23,-1,-1,25,27,-1,-1,29,31,-1,33,35,-1,37,39,41,43,-1,-1,-1,-1,-1,-1,-1,-1,45,-1,47,49,51,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[1.8309373E-1,1.8730848E-1,7.4872285E-2,1.0411721E-1,0E0,5.018574E-2,4.757987E-2,1.0492503E-1,2.745716E-1,0E0,3.8727827E-2,0E0,2.608104E-2,1.12502426E-1,1.5694243E-1,0E0,0E0,4.0666256E-2,4.987452E-2,0E0,0E0,9.242277E-2,7.3161446E-2,0E0,2.588857E-2,1.5597498E-2,0E0,2.6334582E-2,5.1975735E-2,7.052454E-2,1.435881E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,1.2406573E-2,0E0,8.83142E-2,1.7828971E-1,1.24620266E-1,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,5,5,6,6,7,7,8,8,10,10,12,12,13,13,14,14,17,17,18,18,21,21,22,22,24,24,25,25,27,27,28,28,29,29,30,30,39,39,41,41,42,42,43,43],"right_children":[2,4,6,8,-1,10,12,14,16,-1,18,-1,20,22,24,-1,-1,26,28,-1,-1,30,32,-1,34,36,-1,38,40,42,44,-1,-1,-1,-1,-1,-1,-1,-1,46,-1,48,50,52,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.315624E6,1.7971E4,4.907764E3,2.0295714E7,6.88799E-3,3.659E3,3.887931E0,4.9E1,5.59E2,5.382147E-3,1E0,1.3607116E-3,6.5718125E6,4.1E1,9.66E2,-1.064891E-2,2.5401437E-3,7.556899E-1,1.3388E4,3.4325388E-3,8.8315E-3,1.26974E-8,1.6293629E-3,-8.037495E-3,3.907812E7,3.2856784E7,-3.5582166E-3,1.7265024E3,2.59672E3,2.2715E4,2.83475E5,7.984654E-3,6.242521E-4,-1.8158016E-3,2.4994067E-3,-8.8301447E-4,1.734172E-3,-1.2820681E-3,2.101452E-3,5.1985867E1,1.7714841E-4,1.5933333E2,1.1643481E6,1.6423E2,-8.007428E-3,2.786508E-3,6.2718824E-3,-8.554466E-4,4.4068028E-4,-2.9945818E-3,1.3521363E-3,-4.543664E-3,-4.253019E-5],"split_indices":[1,6,4,56,0,2,63,3,0,0,109,0,54,3,12,0,0,51,2,0,0,46,47,0,5,54,0,4,61,38,1,0,0,0,0,0,0,0,0,67,0,4,37,61,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.6E2,8.72E2,8.8E1,8.65E2,7E0,7E1,1.8E1,8.53E2,1.2E1,5E0,6.5E1,7E0,1.1E1,8.35E2,1.8E1,6E0,6E0,2.4E1,4.1E1,6E0,5E0,8.24E2,1.1E1,8E0,1E1,1.8E1,6E0,1.9E1,2.2E1,7.42E2,8.2E1,5E0,6E0,5E0,5E0,8E0,1E1,1.2E1,7E0,1.4E1,8E0,6.59E2,8.3E1,7.6E1,6E0,7E0,7E0,1.44E2,5.15E2,5E1,3.3E1,1.6E1,6E1],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"53","size_leaf_vector":"1"}},{"base_weights":[-1.9196323E-4,-1.097577E-2,3.0922757E-2,-6.2559787E-3,-7.128494E-2,3.4343302E-2,-3.4463739E-3,-3.8914578E-3,-1.2254572E-1,-4.398971E-2,-9.8597845E-3,2.3319734E-2,7.3113315E-2,-7.063001E-2,-9.5991185E-4,-1.0644697E-2,-1.2563054E-3,-1.0782606E-1,4.351844E-3,1.8083356E-2,8.180888E-2,8.957996E-2,3.5939894E-3,-1.0615265E-2,-3.8384288E-3,-1.05866315E-2,1.8112142E-2,-1.2363797E-3,-1.3491546E-1,4.1301576E-3,-4.570492E-2,2.5604919E-2,-4.036076E-2,-1.1529162E-3,6.3855746E-3,6.7876065E-3,7.19347E-2,-2.9783256E-3,3.2840504E-3,-4.8092447E-2,3.9540622E-3,-8.608575E-3,-6.6045388E-3,-3.1731364E-3,3.584656E-2,-8.536101E-3,-1.8025076E-3,-9.07198E-4,-4.1942E-3,-1.8444915E-3,2.9965816E-2,-4.2727203E-3,1.0331046E-2,1.1470374E-1,4.649283E-2,-4.1068452E-3,9.535259E-4,-1.2741975E-2,5.472878E-2,8.948067E-3,-5.9314426E-2,6.4189084E-2,1.233362E-2,5.219404E-2,1.3467154E-2,1.9535874E-3,-1.074771E-3,2.659824E-3,6.8438966E-3,-2.3493389E-4,6.936509E-2,-3.7151514E-4,-3.4580352E-3,6.287953E-4,3.995645E-3,-1.4861192E-3,1.4309798E-3,-1.5951204E-3,-4.607529E-3,3.958618E-3,-9.4620424E-4,-1.699542E-3,2.0138903E-3,4.4653816E-3,9.755022E-4,-2.55389E-4,2.0632662E-3,1.7084391E-3,5.4505253E-3],"categories":[],"categories_nodes":[],"categories_segments":[],"categories_sizes":[],"default_left":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"id":93,"left_children":[1,3,5,7,9,11,-1,13,15,17,-1,19,21,23,25,-1,-1,27,29,31,33,35,37,-1,39,41,43,-1,45,-1,47,49,51,-1,-1,-1,53,-1,-1,55,-1,57,-1,59,61,-1,-1,-1,-1,-1,63,-1,65,67,69,-1,-1,71,73,75,77,79,81,83,85,-1,-1,-1,-1,-1,87,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"loss_changes":[3.1809118E-1,1.9994532E-1,9.035136E-2,1.793532E-1,1.8722898E-1,9.9585E-2,0E0,1.2553765E-1,1.2642409E-1,1.3625097E-1,0E0,5.548708E-2,5.9205085E-2,2.8432557E-1,1.1329016E-1,0E0,0E0,3.8188025E-2,1.1425299E-1,7.6643094E-2,9.271194E-2,3.1502157E-2,5.463621E-2,0E0,8.2920395E-2,1.0562791E-1,7.825252E-2,0E0,5.882916E-2,0E0,1.810117E-2,4.47988E-2,5.3391922E-2,0E0,0E0,0E0,3.166333E-2,0E0,0E0,4.066089E-2,0E0,1.0672578E-1,0E0,6.5178245E-2,7.451333E-2,0E0,0E0,0E0,0E0,0E0,5.210708E-2,0E0,1.268158E-2,1.4126301E-2,2.7472932E-2,0E0,0E0,1.1086852E-1,3.1141862E-2,7.185737E-2,1.35374665E-2,9.2874885E-2,9.742803E-2,8.17204E-2,4.9646594E-2,0E0,0E0,0E0,0E0,0E0,2.0723633E-2,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0,0E0],"parents":[2147483647,0,0,1,1,2,2,3,3,4,4,5,5,7,7,8,8,9,9,11,11,12,12,13,13,14,14,17,17,18,18,19,19,20,20,21,21,22,22,24,24,25,25,26,26,28,28,30,30,31,31,32,32,36,36,39,39,41,41,43,43,44,44,50,50,52,52,53,53,54,54,57,57,58,58,59,59,60,60,61,61,62,62,63,63,64,64,70,70],"right_children":[2,4,6,8,10,12,-1,14,16,18,-1,20,22,24,26,-1,-1,28,30,32,34,36,38,-1,40,42,44,-1,46,-1,48,50,52,-1,-1,-1,54,-1,-1,56,-1,58,-1,60,62,-1,-1,-1,-1,-1,64,-1,66,68,70,-1,-1,72,74,76,78,80,82,84,86,-1,-1,-1,-1,-1,88,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1,-1],"split_conditions":[1.3992583E10,5.035714E0,4.9036694E0,2.697849E8,7.120907E6,1.6978182E3,-3.4463739E-3,4E0,1.5512196E1,8.75E2,-9.8597845E-3,4.1E1,3.116962E7,1.4693878E0,9.558022E7,-1.0644697E-2,-1.2563054E-3,1E0,1.3826087E1,1.39442E5,1.2707317E1,6.072304E5,1.2090626E0,-1.0615265E-2,5.1042255E2,4.179E3,4.7567694E5,-1.2363797E-3,2.8E1,4.1301576E-3,4.826902E6,4.6066E4,5.501002E9,-1.1529162E-3,6.3855746E-3,6.7876065E-3,1.9130435E0,-2.9783256E-3,3.2840504E-3,2.748204E-1,3.9540622E-3,1.731E3,-6.6045388E-3,2.6662E4,2.0158867E8,-8.536101E-3,-1.8025076E-3,-9.07198E-4,-4.1942E-3,-1.8444915E-3,3.4263742E4,-4.2727203E-3,7.6599895E9,6.216657E10,1E1,-4.1068452E-3,9.535259E-4,2.9E1,3.9682608E2,1.2E1,1.159499E6,8.695652E0,3.9149782E6,1.3E1,1.7E1,1.9535874E-3,-1.074771E-3,2.659824E-3,6.8438966E-3,-2.3493389E-4,2.59672E3,-3.7151514E-4,-3.4580352E-3,6.287953E-4,3.995645E-3,-1.4861192E-3,1.4309798E-3,-1.5951204E-3,-4.607529E-3,3.958618E-3,-9.4620424E-4,-1.699542E-3,2.0138903E-3,4.4653816E-3,9.755022E-4,-2.55389E-4,2.0632662E-3,1.7084391E-3,5.4505253E-3],"split_indices":[40,62,47,5,54,61,0,3,65,2,0,3,56,62,7,0,0,21,67,38,65,37,43,0,4,2,37,0,8,0,9,1,5,0,0,0,62,0,0,47,0,2,0,10,7,0,0,0,0,0,42,0,5,40,3,0,0,8,4,3,9,55,54,3,8,0,0,0,0,0,61,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"split_type":[0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0],"sum_hessian":[9.46E2,7.03E2,2.43E2,6.53E2,5E1,2.36E2,7E0,6.41E2,1.2E1,4.3E1,7E0,1.85E2,5.1E1,2.6E1,6.15E2,5E0,7E0,1.8E1,2.5E1,1.71E2,1.4E1,4.1E1,1E1,7E0,1.9E1,4.09E2,2.06E2,5E0,1.3E1,9E0,1.6E1,1.52E2,1.9E1,5E0,9E0,8E0,3.3E1,5E0,5E0,1.3E1,6E0,4.04E2,5E0,9.4E1,1.12E2,8E0,5E0,1.1E1,5E0,9E0,1.43E2,9E0,1E1,1.1E1,2.2E1,8E0,5E0,3.8E2,2.4E1,7.8E1,1.6E1,5E1,6.2E1,6E1,8.3E1,5E0,5E0,5E0,6E0,7E0,1.5E1,3.54E2,2.6E1,1.1E1,1.3E1,2.7E1,5.1E1,1.1E1,5E0,4E1,1E1,2.4E1,3.8E1,2.4E1,3.6E1,5.2E1,3.1E1,1E1,5E0],"tree_param":{"num_deleted":"0","num_feature":"115","num_nodes":"89","size_leaf_vector":"1"}}]},"name":"gbtree"},"learner_model_param":{"base_score":"8.761862E-1","boost_from_average":"1","num_class":"0","num_feature":"115","num_target":"1"},"objective":{"name":"reg:squarederror","reg_loss_param":{"scale_pos_weight":"1"}}},"version":[2,1,3]} \ No newline at end of file diff --git a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics index e964576e8..1cba3c8d8 100644 --- a/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics +++ b/user_tools/src/spark_rapids_pytools/resources/qualx/models/xgboost/onprem.metrics @@ -1,116 +1,116 @@ ,feature,shap_value,mean,std,min,25%,50%,75%,max -0,duration_max,0.31073278,6139.4632063074905,32138.962204191022,15.0,560.0,1136.5,3113.0,515161.0 -1,scan_bw,0.17102478,1593713.4898735192,2294166.855457363,0.0,174981.0170488308,655006.879324951,2101650.4235552456,17385294.707427993 -2,executorCPUTime_mean,0.07895703,1343.870550737643,11697.68382963352,4.25,163.64173913043476,316.2720208661631,687.4622501523162,227948.3076923077 -3,output_recordsWritten_sum,0.025873525,192.27266754270696,2101.059513855952,0.0,0.0,0.0,0.0,28800.0 -4,executorDeserializeTime_mean,0.024394836,6.00370659180027,19.71922884656477,0.04395604395604396,2.2201527924130664,3.6647940074906367,5.147844471947195,475.6 -5,input_bytesRead_mean,0.024062598,8879875.459031755,17843172.786585648,3240.8,947851.6444805195,3648864.1525423727,8947659.156362318,134037865.09764309 -6,executorDeserializeCPUTime_mean,0.023942454,3.480339856961228,9.16075416363637,0.6202143950995406,1.7276889776889777,2.29861370043569,3.147800738859386,199.26666666666668 -7,resultSize_max,0.023571583,3873052.346911958,7059374.819815673,1785.0,25612.5,468118.0,3422351.0,40000929.0 -8,scan_time,0.021254089,46345.203022339025,243193.36454642916,0.0,473.5,2177.5,10481.0,3127801.0 -9,sw_recordsWritten_sum,0.019519525,361907079.3869908,2306099322.4745564,0.0,19991.75,2012328.0,53483121.25,43595020340.0 -10,numExecutors,0.017823713,3.3501971090670173,3.2308043283469328,1.0,1.0,1.0,8.0,8.0 -11,peakExecutionMemory_max,0.0168832,229247556.33114323,530192271.8769413,0.0,36933160.0,73662432.0,236688530.0,11576276592.0 -12,duration_mean,0.016098853,1469.0990844636665,11726.571027982141,13.333333333333334,208.49642857142857,390.50629611411705,838.6961608927381,228041.3076923077 -13,duration_min,0.015893564,16.144546649145862,17.183891508470282,2.0,10.0,13.0,17.0,346.0 -14,sqlOp_SubqueryBroadcast,0.015561328,0.6484888304862023,0.4775991378028838,0.0,0.0,1.0,1.0,1.0 -15,sr_localBytesRead_mean,0.014086169,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 -16,resultSerializationTime_sum,0.0139205605,78.88107752956635,346.69535751687823,0.0,0.0,5.0,19.0,7044.0 -17,sqlOp_ColumnarToRow,0.013868534,0.9500657030223391,0.21788082416070467,0.0,1.0,1.0,1.0,1.0 -18,numTasks_sum,0.013792637,494.50197109067017,1315.5857192434264,3.0,46.0,117.0,382.0,19553.0 -19,sw_writeTime_mean,0.012758898,30.169358107945122,53.46171514110854,0.0,6.257898200965336,15.00956388676358,31.957910089224434,606.2232631578947 -20,executorRunTime_mean,0.011447594,1458.0987197246566,11725.289746335016,6.0,201.34299660441428,378.56819026384244,812.0453680203045,228020.38461538462 -21,duration_sum,0.009790893,1088436.8134034167,7051805.333730643,40.0,14202.0,54338.5,229829.0,191642768.0 -22,input_recordsRead_sum,0.008975233,787220690.6005256,3437802880.1815925,0.0,7106795.0,43264876.0,227006845.0,58530144920.0 -23,data_size,0.0084189065,52257369616.6682,212466894399.25977,0.0,245400479.0,2237123204.5,14651629460.0,2535668591063.0 -24,jvmGCTime_mean,0.008305752,30.5415756714109,114.21214002514562,0.0,1.4274553571428572,5.642595559624628,15.659427859444222,2226.977412731006 -25,sr_localBytesReadRatio,0.008264483,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 -26,shuffle_write_bw,0.0079456,75792.54246937911,99135.47095053854,0.0,1154.757483543684,26859.92379662184,120016.64931211343,473187.3429561201 -27,sr_localBlocksFetched_sum,0.0075378837,46232.10183968463,242956.6186599344,0.0,70.0,449.0,4365.25,3860026.0 -28,output_bytesWrittenRatio,0.006637864,1.1345987204859713e-05,0.00016771251117901053,0.0,0.0,0.0,0.0,0.004454741188373785 -29,sw_bytesWritten_mean,0.0041121156,3740772.892818494,8646461.545663768,0.0,9715.138598022106,422415.42541797285,3103227.323872458,104601898.28278689 -30,sqlOp_SortMergeJoin,0.0040417016,0.4126149802890933,0.4924664490141788,0.0,0.0,0.0,1.0,1.0 -31,sqlOp_Expand,0.0037550933,0.09132720105124836,0.2881685338728861,0.0,0.0,0.0,0.0,1.0 -32,sqlOp_Sort,0.0036450836,0.5860709592641261,0.4926979651899333,0.0,0.0,1.0,1.0,1.0 -33,sw_bytesWrittenRatio,0.002370507,0.7348621806914659,1.3648291225388531,0.0,0.004226587490302376,0.1700493197577822,1.0750713385977424,16.779651636748564 -34,output_bytesWritten_mean,0.0019363238,42.247080700329185,784.3960049968721,0.0,0.0,0.0,0.0,25987.92 -35,shuffle_read_bw,0.0016587302,178564748.4895032,2870761935.402956,0.0,0.0,0.0,0.0,102339190130.0 -36,sqlOp_DeserializeToObject,0.0012919754,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 -37,executorCores,0.0012637067,11.455978975032851,1.7234032412556695,6.0,12.0,12.0,12.0,12.0 -38,sqlOp_Window,0.0011763823,0.11038107752956636,0.31346715319289287,0.0,0.0,0.0,0.0,1.0 -39,memoryBytesSpilledRatio,0.0011455164,0.28201941146050324,1.7605227044105844,0.0,0.0,0.0,0.0,20.350496922781154 -40,diskBytesSpilled_mean,0.0009900176,560139.0351193441,3888224.570120765,0.0,0.0,0.0,0.0,57763330.03389474 -41,memoryBytesSpilled_mean,0.00065111875,2254905.7219324433,13425137.78628761,0.0,0.0,0.0,0.0,292579823.9844638 -42,sr_fetchWaitTime_mean,0.0006130505,0.004314894597195671,0.023989922174385267,0.0,0.0,0.0,0.0,0.38974358974358975 -43,sqlOp_ObjectHashAggregate,0.00052806956,0.09461235216819974,0.2927749541187287,0.0,0.0,0.0,0.0,1.0 -44,sqlOp_HashAggregate,0.00042811377,0.9494086727989488,0.21923371988645474,0.0,1.0,1.0,1.0,1.0 -45,sqlOp_BroadcastNestedLoopJoin,0.00042109675,0.03942181340341656,0.19466029547643685,0.0,0.0,0.0,0.0,1.0 -46,sqlOp_BatchEvalPython,0.00041852758,0.012483574244415242,0.11106682510080343,0.0,0.0,0.0,0.0,1.0 -47,sqlOp_TakeOrderedAndProject,0.00037636794,0.022996057818659658,0.14994002302202078,0.0,0.0,0.0,0.0,1.0 -48,sqlOp_Filter,0.00035233508,0.9697766097240473,0.17125772137831843,0.0,1.0,1.0,1.0,1.0 -49,sqlOp_AQEShuffleRead,0.0002898404,0.8731931668856767,0.3328658271954145,0.0,1.0,1.0,1.0,1.0 -50,sqlOp_Subquery,0.00023014576,0.15703022339027595,0.3639488449522727,0.0,0.0,0.0,0.0,1.0 -51,sqlOp_BroadcastHashJoin,0.00022189623,0.892904073587385,0.3093368062389466,0.0,1.0,1.0,1.0,1.0 -52,diskBytesSpilledRatio,0.00017616934,0.03870446764927755,0.22755637119531616,0.0,0.0,0.0,0.0,2.6257828841329087 -53,sqlOp_Project,0.000107654836,0.9697766097240473,0.1712577213783184,0.0,1.0,1.0,1.0,1.0 -54,sqlOp_BroadcastExchange,8.671821e-05,0.897503285151117,0.3034000972146485,0.0,1.0,1.0,1.0,1.0 -55,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -56,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -57,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -58,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -59,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -60,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -61,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -62,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -63,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -64,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -65,sqlOp_MapElements,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 -66,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -67,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -68,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -69,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -70,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -71,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -72,sqlOp_SerializeFromObject,0.0,0.04204993429697766,0.20076907391845306,0.0,0.0,0.0,0.0,1.0 -73,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -74,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -75,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -76,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -77,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -78,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -79,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -80,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -81,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -82,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -83,maxMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 -84,platform_onprem,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -85,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -86,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -87,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -88,executorMemory,0.0,16262.896189224704,1358.9340452651124,1024.0,16384.0,16384.0,16384.0,16384.0 -89,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -90,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -91,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -92,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -93,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -94,sqlOp_WindowGroupLimit,0.0,0.017082785808147174,0.12962254302936352,0.0,0.0,0.0,0.0,1.0 -95,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -96,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 -97,sqlOp_Scan parquet,0.0,0.9500657030223391,0.21788082416070467,0.0,1.0,1.0,1.0,1.0 -98,sr_remoteBytesReadRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -99,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -100,sr_totalBytesReadRatio,0.0,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 -101,sqlOp_LocalTableScan,0.0,0.0019710906701708277,0.044367767973722515,0.0,0.0,0.0,0.0,1.0 -102,sr_remoteBytesRead_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -103,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -104,sr_totalBytesRead_mean,0.0,3984381.8321779394,8966196.568703005,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 -105,sqlOp_Exchange,0.0,0.9579500657030223,0.20076907391845303,0.0,1.0,1.0,1.0,1.0 -106,sr_remoteBlocksFetched_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -107,maxOnHeapMem,0.0,10042985900.30749,854967375.2687465,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 -108,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -109,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -110,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -111,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -112,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 -113,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +0,duration_max,0.309267,6139.4632063074905,32138.962204191026,15.0,560.0,1136.5,3113.0,515161.0 +1,scan_bw,0.16803238,1593713.489873519,2294166.855457363,0.0,174981.0170488308,655006.879324951,2101650.4235552456,17385294.707427993 +2,executorCPUTime_mean,0.082095,1343.870550737643,11697.68382963352,4.25,163.64173913043476,316.2720208661631,687.4622501523162,227948.3076923077 +3,executorDeserializeTime_mean,0.03126094,6.0037065918002686,19.71922884656477,0.04395604395604396,2.2201527924130664,3.6647940074906367,5.147844471947195,475.6 +4,scan_time,0.02817775,46345.203022339025,243193.3645464292,0.0,473.5,2177.5,10481.0,3127801.0 +5,input_bytesRead_mean,0.024840662,8879875.459031759,17843172.786585648,3240.8,947851.6444805195,3648864.1525423727,8947659.156362318,134037865.09764309 +6,resultSize_max,0.023597391,3873052.346911958,7059374.819815673,1785.0,25612.5,468118.0,3422351.0,40000929.0 +7,executorDeserializeCPUTime_mean,0.021152575,3.480339856961227,9.16075416363637,0.6202143950995406,1.7276889776889777,2.29861370043569,3.147800738859386,199.26666666666668 +8,peakExecutionMemory_max,0.021079384,229247556.33114323,530192271.8769413,0.0,36933160.0,73662432.0,236688530.0,11576276592.0 +9,output_recordsWritten_sum,0.02103182,192.27266754270696,2101.059513855952,0.0,0.0,0.0,0.0,28800.0 +10,sqlOp_SubqueryBroadcast,0.016923685,0.6484888304862023,0.4775991378028839,0.0,0.0,1.0,1.0,1.0 +11,sw_writeTime_mean,0.016114092,30.169358107945122,53.46171514110854,0.0,6.257898200965336,15.00956388676358,31.957910089224434,606.2232631578947 +12,duration_mean,0.014828617,1469.0990844636665,11726.571027982141,13.333333333333334,208.49642857142857,390.50629611411705,838.6961608927381,228041.3076923077 +13,sw_recordsWritten_sum,0.014753952,361907079.3869908,2306099322.474557,0.0,19991.75,2012328.0,53483121.25,43595020340.0 +14,sqlOp_ColumnarToRow,0.014429107,0.9500657030223391,0.21788082416070464,0.0,1.0,1.0,1.0,1.0 +15,duration_min,0.0140327765,16.144546649145862,17.18389150847028,2.0,10.0,13.0,17.0,346.0 +16,sr_localBytesRead_mean,0.013452834,3984381.8321779394,8966196.568703003,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 +17,executorRunTime_mean,0.012751707,1458.0987197246566,11725.289746335016,6.0,201.34299660441428,378.56819026384244,812.0453680203045,228020.38461538462 +18,numTasks_sum,0.012674564,494.50197109067017,1315.5857192434262,3.0,46.0,117.0,382.0,19553.0 +19,numExecutors,0.012506497,3.3501971090670173,3.2308043283469328,1.0,1.0,1.0,8.0,8.0 +20,resultSerializationTime_sum,0.012195481,78.88107752956635,346.69535751687823,0.0,0.0,5.0,19.0,7044.0 +21,jvmGCTime_mean,0.010954352,30.5415756714109,114.21214002514559,0.0,1.4274553571428572,5.642595559624628,15.659427859444222,2226.977412731006 +22,input_recordsRead_sum,0.009083974,787220690.6005256,3437802880.181593,0.0,7106795.0,43264876.0,227006845.0,58530144920.0 +23,duration_sum,0.00883194,1088436.8134034167,7051805.333730645,40.0,14202.0,54338.5,229829.0,191642768.0 +24,data_size,0.0082970755,52257369616.6682,212466894399.25974,0.0,245400479.0,2237123204.5,14651629460.0,2535668591063.0 +25,sr_localBytesReadRatio,0.0075088213,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 +26,shuffle_write_bw,0.0074336957,75792.54246937911,99135.47095053854,0.0,1154.757483543684,26859.92379662184,120016.64931211343,473187.3429561201 +27,sr_localBlocksFetched_sum,0.007119559,46232.10183968463,242956.6186599344,0.0,70.0,449.0,4365.25,3860026.0 +28,sqlOp_Sort,0.0056619043,0.5860709592641261,0.4926979651899333,0.0,0.0,1.0,1.0,1.0 +29,sqlOp_SortMergeJoin,0.0052716853,0.4126149802890933,0.4924664490141788,0.0,0.0,0.0,1.0,1.0 +30,output_bytesWrittenRatio,0.0042833844,1.134598720485971e-05,0.00016771251117901053,0.0,0.0,0.0,0.0,0.004454741188373785 +31,sw_bytesWrittenRatio,0.003755082,0.7348621806914659,1.3648291225388531,0.0,0.004226587490302376,0.1700493197577822,1.0750713385977424,16.779651636748564 +32,sw_bytesWritten_mean,0.003560834,3740772.892818494,8646461.545663768,0.0,9715.138598022106,422415.42541797285,3103227.323872458,104601898.28278689 +33,output_bytesWritten_mean,0.0034547749,42.247080700329185,784.3960049968719,0.0,0.0,0.0,0.0,25987.92 +34,sqlOp_Expand,0.0032825372,0.09132720105124836,0.2881685338728861,0.0,0.0,0.0,0.0,1.0 +35,sqlOp_DeserializeToObject,0.003267622,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 +36,shuffle_read_bw,0.0020968805,178564748.48950323,2870761935.402956,0.0,0.0,0.0,0.0,102339190130.0 +37,diskBytesSpilled_mean,0.0011763439,560139.0351193442,3888224.570120765,0.0,0.0,0.0,0.0,57763330.03389474 +38,executorCores,0.001141772,11.455978975032851,1.7234032412556692,6.0,12.0,12.0,12.0,12.0 +39,memoryBytesSpilled_mean,0.0010527639,2254905.7219324433,13425137.78628761,0.0,0.0,0.0,0.0,292579823.9844638 +40,sqlOp_Subquery,0.00080200384,0.15703022339027595,0.3639488449522727,0.0,0.0,0.0,0.0,1.0 +41,sqlOp_Window,0.000741515,0.11038107752956636,0.31346715319289287,0.0,0.0,0.0,0.0,1.0 +42,sqlOp_ObjectHashAggregate,0.0006764567,0.09461235216819974,0.2927749541187287,0.0,0.0,0.0,0.0,1.0 +43,sr_fetchWaitTime_mean,0.00051488605,0.00431489459719567,0.02398992217438527,0.0,0.0,0.0,0.0,0.38974358974358975 +44,sqlOp_HashAggregate,0.00048551918,0.9494086727989488,0.21923371988645474,0.0,1.0,1.0,1.0,1.0 +45,sqlOp_BroadcastHashJoin,0.00044478048,0.892904073587385,0.3093368062389466,0.0,1.0,1.0,1.0,1.0 +46,sqlOp_BroadcastExchange,0.00044106398,0.897503285151117,0.3034000972146485,0.0,1.0,1.0,1.0,1.0 +47,memoryBytesSpilledRatio,0.0004172633,0.28201941146050324,1.7605227044105847,0.0,0.0,0.0,0.0,20.350496922781154 +48,sqlOp_BatchEvalPython,0.00038339774,0.012483574244415242,0.11106682510080343,0.0,0.0,0.0,0.0,1.0 +49,sqlOp_Project,0.00030256703,0.9697766097240473,0.17125772137831843,0.0,1.0,1.0,1.0,1.0 +50,sqlOp_AQEShuffleRead,0.00029463766,0.8731931668856767,0.3328658271954145,0.0,1.0,1.0,1.0,1.0 +51,sqlOp_TakeOrderedAndProject,0.0002525618,0.022996057818659658,0.14994002302202078,0.0,0.0,0.0,0.0,1.0 +52,diskBytesSpilledRatio,0.00015939545,0.03870446764927755,0.22755637119531616,0.0,0.0,0.0,0.0,2.6257828841329087 +53,sqlOp_WindowGroupLimit,8.789679e-05,0.017082785808147174,0.12962254302936352,0.0,0.0,0.0,0.0,1.0 +54,sqlOp_BroadcastNestedLoopJoin,7.964926e-05,0.03942181340341656,0.19466029547643687,0.0,0.0,0.0,0.0,1.0 +55,sqlOp_Filter,7.669086e-05,0.9697766097240473,0.17125772137831843,0.0,1.0,1.0,1.0,1.0 +56,sqlOp_Exchange,4.6852496e-05,0.9579500657030223,0.20076907391845303,0.0,1.0,1.0,1.0,1.0 +57,sr_totalBytesReadRatio,0.0,0.7902416781930374,1.431541673913999,0.0,0.004238274599405685,0.17196742660015743,1.1388709718278844,16.779651636748564 +58,sqlOp_Scan parquet,0.0,0.9500657030223391,0.21788082416070464,0.0,1.0,1.0,1.0,1.0 +59,sqlOp_LocalTableScan,0.0,0.0019710906701708277,0.044367767973722515,0.0,0.0,0.0,0.0,1.0 +60,maxMem,0.0,10042985900.30749,854967375.2687466,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 +61,maxOnHeapMem,0.0,10042985900.30749,854967375.2687466,455501414.0,10119177830.0,10119177830.0,10119177830.0,10119177830.0 +62,maxOffHeapMem,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +63,executorMemory,0.0,16262.896189224704,1358.9340452651127,1024.0,16384.0,16384.0,16384.0,16384.0 +64,numGpusPerExecutor,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +65,executorOffHeap,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +66,taskCpu,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +67,sr_remoteBlocksFetched_sum,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +68,sr_remoteBytesReadToDiskRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +69,sr_remoteBytesReadRatio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +70,input_bytesReadRatio,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +71,taskGpu,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +72,cache_hit_ratio,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +73,decode_time,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +74,sr_totalBytesRead_mean,0.0,3984381.8321779394,8966196.568703003,0.0,9715.138598022106,441049.2350230415,3399921.6142857145,104601898.28278689 +75,platform_dataproc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +76,sqlOp_CartesianProduct,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +77,sr_remoteBytesRead_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +78,sr_remoteBytesReadToDisk_mean,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +79,platform_databricks-azure,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +80,platform_databricks-aws,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +81,platform_emr,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +82,platform_onprem,0.0,1.0,0.0,1.0,1.0,1.0,1.0,1.0 +83,sqlOp_Execute InsertIntoHadoopFsRelationCommand parquet,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +84,sqlOp_Execute InsertIntoHadoopFsRelationCommand orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +85,sqlOp_Execute InsertIntoHadoopFsRelationCommand json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +86,sqlOp_Execute InsertIntoHadoopFsRelationCommand csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +87,sqlOp_CustomShuffleReader,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +88,sqlOp_CommandResult,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +89,sqlOp_Execute InsertIntoHadoopFsRelationCommand unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +90,sqlOp_Execute InsertIntoHadoopFsRelationCommand text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +91,sqlOp_LocalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +92,sqlOp_MapElements,0.0,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 +93,sqlOp_OutputAdapter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +94,sqlOp_PartialWindow,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +95,sqlOp_GenerateBloomFilter,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +96,sqlOp_GlobalLimit,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +97,sqlOp_HashAggregatePrefixGroupingSets,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +98,sqlOp_Generate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +99,sqlOp_Scan ExistingRDD Delta Table Checkpoint,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +100,sqlOp_Scan ExistingRDD,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +101,sqlOp_RunningWindowFunction,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +102,sqlOp_ReusedSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +103,sqlOp_Scan ExistingRDD Delta Table State,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +104,sqlOp_Scan OneRowRelation,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +105,sqlOp_Scan csv,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +106,sqlOp_Scan jdbc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +107,sqlOp_Scan unknown,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +108,sqlOp_Scan text,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +109,sqlOp_Scan orc,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +110,sqlOp_Scan json,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +111,sqlOp_SubqueryOutputBroadcast,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +112,sqlOp_SortAggregate,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 +113,sqlOp_SerializeFromObject,0.0,0.04204993429697766,0.20076907391845303,0.0,0.0,0.0,0.0,1.0 114,sqlOp_WindowSort,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0 From 5745ea5649cbd4969a53f347db29823e8489ad8f Mon Sep 17 00:00:00 2001 From: Lee Yang Date: Fri, 10 Jan 2025 17:29:54 -0800 Subject: [PATCH 39/52] Fix additional FutureWarning issues (#1499) This PR is a followup to #1494 to resolve additional warnings found during the `qualx evaluate` command. Confirmed that the evaluation results are identical before/after these changes. Signed-off-by: Lee Yang --- user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py index ff8835cc1..8c07d3783 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/preprocess.py @@ -615,7 +615,7 @@ def combine_tables(table_name: str) -> pd.DataFrame: ['input_bytesRead_sum', 'input_bytesRead_cache'] ].max(axis=1) full_tbl = full_tbl.drop(columns=['input_bytesRead_cache']) - full_tbl['cache_hit_ratio'].fillna(0.0, inplace=True) + full_tbl.fillna({'cache_hit_ratio': 0.0}, inplace=True) else: full_tbl['cache_hit_ratio'] = 0.0 @@ -1162,7 +1162,8 @@ def load_qual_csv( qual_csv = [os.path.join(q, csv_filename) for q in qual_dirs] df = None if qual_csv: - df = pd.concat([pd.read_csv(f) for f in qual_csv]) + dfs = [pd.read_csv(f) for f in qual_csv] + df = pd.concat([df for df in dfs if not df.empty]) if cols: df = df[cols] return df From 884553dd74614ec198c973c94806e7fd54eb1b4b Mon Sep 17 00:00:00 2001 From: Lee Yang Date: Mon, 13 Jan 2025 13:22:59 -0800 Subject: [PATCH 40/52] Add compute_precision_recall utility function (#1500) This PR adds a function to compute precision and recall metrics for qualx predictions. This function is often used in the evaluation notebooks, so pulling it into the qualx/util.py module. Tested in an evaluation notebook. --------- Signed-off-by: Lee Yang --- .../spark_rapids_tools/tools/qualx/util.py | 46 ++++++++++++++++++- 1 file changed, 45 insertions(+), 1 deletion(-) diff --git a/user_tools/src/spark_rapids_tools/tools/qualx/util.py b/user_tools/src/spark_rapids_tools/tools/qualx/util.py index 9bb8ee1a5..dcd62b24c 100644 --- a/user_tools/src/spark_rapids_tools/tools/qualx/util.py +++ b/user_tools/src/spark_rapids_tools/tools/qualx/util.py @@ -1,4 +1,4 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. +# Copyright (c) 2024-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -190,6 +190,12 @@ def compute_accuracy( Dictionary of display name of metric -> prediction column name. weight: str Name of column to use as weight, e.g. 'Duration' or 'appDuration'. + + Returns + ------- + scores: Dict[str, Dict[str, float]] + Dictionary of different scoring metrics per prediction column, + e.g. {'QXS': {'MAPE"; 1.31, 'dMAPE': 1.25}} """ scores = {} for name, y_pred in y_preds.items(): @@ -221,6 +227,44 @@ def compute_accuracy( return scores +def compute_precision_recall( + results: pd.DataFrame, y: str, y_preds: Dict[str, str], threshold: float +) -> Tuple[Dict[str, float], Dict[str, float]]: + """Compute precision and recall from a dataframe using a threshold for identifying true positives. + + Parameters + ---------- + results: pd.DataFrame + Pandas dataframe containing the label column and one or more prediction columns. + y: str + Label column name. + y_preds: Dict[str, str] + Dictionary of display name of prediction -> prediction column name. + threshold: float + Threshold separating positives (inclusive of threshold) from negatives. + + Returns + ------- + precision: Dict[str, float] + Dictionary of precision metric per prediction column, + e.g. {'QX': 0.90, 'QXS': 0.92} + recall: Dict[str, float] + Dictionary of recall metric per prediction column, + e.g. {'QX': 0.78, 'QXS': 0.82} + """ + precision = {} + recall = {} + for name, y_pred in y_preds.items(): + tp = sum((results[y_pred] >= threshold) & (results[y] >= threshold)) + fp = sum((results[y_pred] >= threshold) & (results[y] < threshold)) + fn = sum((results[y_pred] < threshold) & (results[y] >= threshold)) + + precision[name] = tp / (tp + fp) if (tp + fp) > 0 else np.nan + recall[name] = tp / (tp + fn) if (tp + fn) > 0 else np.nan + + return precision, recall + + def load_plugin(plugin_path: str) -> types.ModuleType: """Dynamically load plugin modules with helper functions for dataset-specific code. From 42b2a8b13e3765223a984cc1a60c2f97f063bff9 Mon Sep 17 00:00:00 2001 From: spark-rapids automation <70000568+nvauto@users.noreply.github.com> Date: Tue, 14 Jan 2025 03:32:00 +0000 Subject: [PATCH 41/52] Update dev-version by jenkins-spark-rapids-tools-auto-release-105 Signed-off-by: spark-rapids automation <70000568+nvauto@users.noreply.github.com> --- core/pom.xml | 2 +- user_tools/src/spark_rapids_pytools/__init__.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 79cb3830a..a04dce334 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -23,7 +23,7 @@ rapids-4-spark-tools_2.12 RAPIDS Accelerator for Apache Spark tools RAPIDS Accelerator for Apache Spark tools - 24.12.1-SNAPSHOT + 24.12.2-SNAPSHOT jar http://github.com/NVIDIA/spark-rapids-tools diff --git a/user_tools/src/spark_rapids_pytools/__init__.py b/user_tools/src/spark_rapids_pytools/__init__.py index 103ac0793..a9f5d53de 100644 --- a/user_tools/src/spark_rapids_pytools/__init__.py +++ b/user_tools/src/spark_rapids_pytools/__init__.py @@ -16,7 +16,7 @@ from spark_rapids_pytools.build import get_version, get_spark_dep_version -VERSION = '24.12.1' +VERSION = '24.12.2' # defines the default runtime build version for the user tools environment SPARK_DEP_VERSION = '350' __version__ = get_version(VERSION) From 4e83fe48180559a44cd5b79e22f622efd7b22000 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Wed, 15 Jan 2025 11:28:06 -0800 Subject: [PATCH 42/52] Fix Hadoop JAR Download Timeouts in Behave Tests (#1503) Fixes #1502 The behave tests currently downloads Hadoop JARs from Apache CDN (dlcdn.apache.org) for HDFS testing. However, there are test failures due to download timeouts making workflows unreliable. ## Issue - Using CDN (dlcdn.apache.org) instead of the more reliable archive.apache.org - Insufficient timeout settings in download configurations ## Changes 1. Switch download source from dlcdn.apache.org to archive.apache.org 2. Increased timeout settings --------- Signed-off-by: Partho Sarthi --- .../resources/scripts/hdfs/setup_hdfs.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/user_tools/tests/spark_rapids_tools_e2e/resources/scripts/hdfs/setup_hdfs.sh b/user_tools/tests/spark_rapids_tools_e2e/resources/scripts/hdfs/setup_hdfs.sh index 10e979bf9..d8508efcb 100755 --- a/user_tools/tests/spark_rapids_tools_e2e/resources/scripts/hdfs/setup_hdfs.sh +++ b/user_tools/tests/spark_rapids_tools_e2e/resources/scripts/hdfs/setup_hdfs.sh @@ -1,6 +1,6 @@ #!/bin/bash -# Copyright (c) 2024, NVIDIA CORPORATION. +# Copyright (c) 2024-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -67,6 +67,7 @@ readonly CURRENT_FILE_PATH=$(realpath "${0}") readonly HDFS_SCRIPTS_DIR=$(dirname "${CURRENT_FILE_PATH}") readonly VERIFY_HDFS_SERVICES_MAX_RETRY=3 readonly VERIFY_HDFS_SERVICES_SLEEP_SEC=5 +readonly WGET_TIMEOUT_SEC=1800 # 30 minutes load_common_scripts() { local scripts_dir=$(dirname "${HDFS_SCRIPTS_DIR}") @@ -107,19 +108,19 @@ verify_checksum() { # Function to download and extract Hadoop download_and_extract_hadoop() { echo "Downloading and extracting Hadoop..." - local hadoop_url="https://dlcdn.apache.org/hadoop/common/hadoop-${E2E_TEST_HADOOP_VERSION}/hadoop-${E2E_TEST_HADOOP_VERSION}.tar.gz" + local hadoop_url="https://archive.apache.org/dist/hadoop/common/hadoop-${E2E_TEST_HADOOP_VERSION}/hadoop-${E2E_TEST_HADOOP_VERSION}.tar.gz" local hadoop_tar_file="${E2E_TEST_TMP_DIR}/hadoop-${E2E_TEST_HADOOP_VERSION}.tar.gz" local checksum_url="${hadoop_url}.sha512" local checksum_file="${hadoop_tar_file}.sha512" if [ ! -f "${hadoop_tar_file}" ]; then - wget -O "${hadoop_tar_file}" "${hadoop_url}" || err "Failed to download Hadoop tarball." + wget --timeout=${WGET_TIMEOUT_SEC} -O"${hadoop_tar_file}" "${hadoop_url}" || err "Failed to download Hadoop tarball." fi # Verify checksum and re-download if needed - wget -O "${checksum_file}" "${checksum_url}" || err "Failed to download checksum file." + wget --timeout=${WGET_TIMEOUT_SEC} -O"${checksum_file}" "${checksum_url}" || err "Failed to download checksum file." if ! verify_checksum "${hadoop_tar_file}" "${checksum_file}"; then - wget -O "${hadoop_tar_file}" "${hadoop_url}" || err "Failed to download Hadoop tarball." + wget --timeout=${WGET_TIMEOUT_SEC} -O"${hadoop_tar_file}" "${hadoop_url}" || err "Failed to download Hadoop tarball." if ! verify_checksum "${hadoop_tar_file}" "${checksum_file}"; then err "Checksum verification failed after re-downloading. Exiting..." fi From 5ab53d8b7b29c640f486178cd32f8e06d7dd9601 Mon Sep 17 00:00:00 2001 From: Peixin Date: Tue, 21 Jan 2025 13:46:15 +0800 Subject: [PATCH 43/52] Use common add-to-project action [skip ci] (#1505) follow up of https://github.com/NVIDIA/spark-rapids-common/issues/22 to avoid update action details for multiple `spark-rapids*` repos in the future Signed-off-by: Peixin Li --- .github/workflows/add-to-project.yml | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/.github/workflows/add-to-project.yml b/.github/workflows/add-to-project.yml index f8c6a70b8..a0998ecd2 100644 --- a/.github/workflows/add-to-project.yml +++ b/.github/workflows/add-to-project.yml @@ -1,4 +1,4 @@ -# Copyright (c) 2022-2024, NVIDIA CORPORATION. +# Copyright (c) 2022-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -23,13 +23,11 @@ on: - opened jobs: - add-to-project: - if: github.repository == 'NVIDIA/spark-rapids-tools' - name: Add new issues and pull requests to project + Add-to-project: + if: github.repository_owner == 'NVIDIA' # avoid adding issues from forks runs-on: ubuntu-latest steps: - - uses: actions/add-to-project@v0.6.1 + - name: add-to-project + uses: NVIDIA/spark-rapids-common/add-to-project@main with: - project-url: https://github.com/orgs/NVIDIA/projects/4 - github-token: ${{ secrets.PROJECT_TOKEN }} - + token: ${{ secrets.PROJECT_TOKEN }} From 4d34fc77ac7e900fa68fcae4f746e74cf30c9fb8 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Tue, 21 Jan 2025 14:21:01 -0800 Subject: [PATCH 44/52] Revert "Follow Up: Make '--platform' argument mandatory in CLI (#1473)" (#1498) Fixes #1497. In https://github.com/NVIDIA/spark-rapids-tools/issues/1472, the `--platform` argument was made a required parameter in Fire. In our tests, it was found that the short flag -p for platform is no longer functioning. Upon investigation, it seems that Fire does not support short flags for required arguments. This PR reverts the above commit and makes the platform argument optional in Fire. This change will ensure compatibility with the -p flag. The platform argument will still be validated internally to ensure its requirement is enforced. ### Case 1 : Platform not provided ``` spark_rapids qualification --eventlogs "/path/to/dataproc-cpu" --verbose ``` #### STDOUT ``` 2025-01-10 13:35:16,011 INFO spark_rapids_tools.argparser: ...applying argument case: Missing Platform argument 2025-01-10 13:35:16,012 ERROR spark_rapids_tools.argparser: Validation err: 1 validation error for QualifyUserArgModel Cannot run tool cmd without platform argument. Re-run the command providing the platform argument. Error: [type=invalid_argument, input_value=ArgsKwargs((), {'eventlog...ols_config_path': None}), input_type=ArgsKwargs] ``` ### Case 2: Platform provided by `--platform` and `-p` ``` spark_rapids qualification --platform dataproc --eventlogs "/path/to/dataproc-cpu" --verbose spark_rapids qualification -p dataproc --eventlogs "/path/to/dataproc-cpu" --verbose ``` #### STDOUT ``` Report Summary: ---------------------- - Total applications 1 Processed applications 1 Top candidates 0 ---------------------- - Processing Completed! ``` --------- Signed-off-by: Partho Sarthi --- .../src/spark_rapids_tools/cmdli/tools_cli.py | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py b/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py index e50659b46..9fbfc81d1 100644 --- a/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py +++ b/user_tools/src/spark_rapids_tools/cmdli/tools_cli.py @@ -1,4 +1,4 @@ -# Copyright (c) 2023-2024, NVIDIA CORPORATION. +# Copyright (c) 2023-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -35,10 +35,9 @@ class ToolsCLI(object): # pylint: disable=too-few-public-methods """ def qualification(self, - *, # force named arguments - platform: str, eventlogs: str = None, cluster: str = None, + platform: str = None, output_folder: str = None, filter_apps: str = None, custom_model_file: str = None, @@ -56,8 +55,6 @@ def qualification(self, The cmd will process each app individually, but will group apps with the same name into the same output row after averaging duration metrics accordingly. - :param platform: Defines one of the following: "onprem", "emr", "dataproc", "dataproc-gke", - "databricks-aws", and "databricks-azure". :param eventlogs: Event log filenames or CSP storage directories containing event logs (comma separated). @@ -65,6 +62,8 @@ def qualification(self, cluster name on the CSP. :param cluster: The CPU cluster on which the Spark application(s) were executed. Name or ID (for databricks platforms) of cluster or path to cluster-properties. + :param platform: Defines one of the following: "onprem", "emr", "dataproc", "dataproc-gke", + "databricks-aws", and "databricks-azure". :param output_folder: Local path to store the output. :param tools_jar: Path to a bundled jar including Rapids tool. The path is a local filesystem, or remote cloud storage url. If missing, the wrapper downloads the latest rapids-4-spark-tools_*.jar @@ -90,8 +89,8 @@ def qualification(self, For more details on Qualification tool options, please visit https://docs.nvidia.com/spark-rapids/user-guide/latest/qualification/jar-usage.html#running-the-qualification-tool-standalone-on-spark-event-logs """ - platform = Utils.get_value_or_pop(platform, rapids_options, 'p') eventlogs = Utils.get_value_or_pop(eventlogs, rapids_options, 'e') + platform = Utils.get_value_or_pop(platform, rapids_options, 'p') tools_jar = Utils.get_value_or_pop(tools_jar, rapids_options, 't') output_folder = Utils.get_value_or_pop(output_folder, rapids_options, 'o') filter_apps = Utils.get_value_or_pop(filter_apps, rapids_options, 'f') @@ -109,9 +108,9 @@ def qualification(self, if estimation_model_args is None: return None qual_args = AbsToolUserArgModel.create_tool_args('qualification', - platform=platform, eventlogs=eventlogs, cluster=cluster, + platform=platform, output_folder=output_folder, tools_jar=tools_jar, jvm_heap_size=jvm_heap_size, @@ -128,10 +127,9 @@ def qualification(self, return None def profiling(self, - *, # force named arguments - platform: str, eventlogs: str = None, cluster: str = None, + platform: str = None, driverlog: str = None, output_folder: str = None, tools_jar: str = None, @@ -148,14 +146,14 @@ def profiling(self, The tool also will recommend setting for the application assuming that the job will be able to use all the cluster resources (CPU and GPU) when it is running. - :param platform: defines one of the following "onprem", "emr", "dataproc", "databricks-aws", - and "databricks-azure". :param eventlogs: Event log filenames or cloud storage directories containing event logs (comma separated). If missing, the wrapper reads the Spark's property `spark.eventLog.dir` defined in the `cluster`. :param cluster: The cluster on which the Spark applications were executed. The argument can be a cluster name or ID (for databricks platforms) or a valid path to the cluster's properties file (json format) generated by the CSP SDK. + :param platform: defines one of the following "onprem", "emr", "dataproc", "databricks-aws", + and "databricks-azure". :param driverlog: Valid path to the GPU driver log file. :param output_folder: path to store the output. :param tools_jar: Path to a bundled jar including Rapids tool. The path is a local filesystem, @@ -175,9 +173,9 @@ def profiling(self, For more details on Profiling tool options, please visit https://docs.nvidia.com/spark-rapids/user-guide/latest/profiling/jar-usage.html#prof-tool-title-options """ - platform = Utils.get_value_or_pop(platform, rapids_options, 'p') eventlogs = Utils.get_value_or_pop(eventlogs, rapids_options, 'e') cluster = Utils.get_value_or_pop(cluster, rapids_options, 'c') + platform = Utils.get_value_or_pop(platform, rapids_options, 'p') driverlog = Utils.get_value_or_pop(driverlog, rapids_options, 'd') output_folder = Utils.get_value_or_pop(output_folder, rapids_options, 'o') tools_jar = Utils.get_value_or_pop(tools_jar, rapids_options, 't') @@ -186,9 +184,9 @@ def profiling(self, ToolLogging.enable_debug_mode() init_environment('prof') prof_args = AbsToolUserArgModel.create_tool_args('profiling', - platform=platform, eventlogs=eventlogs, cluster=cluster, + platform=platform, driverlog=driverlog, jvm_heap_size=jvm_heap_size, jvm_threads=jvm_threads, From 4a32581a2cd7f8f3deb87ef789301a09e15c8067 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Wed, 22 Jan 2025 09:26:32 -0800 Subject: [PATCH 45/52] Improve AutoTuner cluster configuration recommendations for GPU runs (#1501) Fixes #1121 Fixes #1334 ### Summary Introduces a new strategy-based approach for recommending GPU cluster configurations (num executors, core count and num worker nodes (CSPs only)). ### Key Changes - Standardized executor core count to 16 cores per executor for GPU runs. - Shifted from multi-GPU instances to smaller, single-GPU instances - Implemented two configuration strategies: 1. Cluster Property Strategy: Generates recommendations based on user-specified cluster properties 2. Event Log Strategy: Generates recommendations based on event logs ### Reasoning - Analysis of NDS performance metrics shows 16 cores/executor provides optimal performance - Configurations with 4 or 64 cores/executor showed reduced performance and cost efficiency - Smaller instances with distributed GPUs improve: - System fault tolerance - I/O performance (both disk and network) - Resource utilization ### Code Changes Cluster configuration strategy: * [`core/src/main/scala/com/nvidia/spark/rapids/tool/ClusterConfigurationStrategy.scala`](diffhunk://#diff-1407805efc49b0c67ad7ccfe1648c533688c4a246841a5fdc60329120b97f4f1R1-R213): Added new classes `ClusterConfigurationStrategy`, `ClusterPropertyBasedStrategy`, and `EventLogBasedStrategy` to encapsulate different cluster configuration strategies. * [`core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala`](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L259-L264): Refactored the `Platform` class to use the new strategy classes for recommending cluster configurations and removed redundant methods. [[1]](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L259-L264) [[2]](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L336-L385) [[3]](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L433-R502) Enhancements to instance information handling: * [`core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala`](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L58-R72): Added `getMemoryPerExec` method to `InstanceInfo` and created a companion object for default instance creation. * [`core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala`](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L617-R562): Updated `DatabricksAwsPlatform` to use `getInstanceByResourcesMap` for instance type mapping. Default recommendations and properties: * [`core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala`](diffhunk://#diff-cc81398d095d289f60b6eb03dde5b6995956258b6c176f31cc2df56382cb1a98L134-R154): Added default recommendations for cores per executor and GPUs per node. --------- Signed-off-by: Partho Sarthi --- .../tool/ClusterConfigurationStrategy.scala | 213 +++++++ .../nvidia/spark/rapids/tool/Platform.scala | 378 +++++-------- .../tool/qualification/QualOutputWriter.scala | 9 +- .../spark/rapids/tool/tuning/AutoTuner.scala | 34 +- .../sql/rapids/tool/ClassWarehouse.scala | 6 +- .../tool/tuning/BaseAutoTunerSuite.scala | 17 + .../tool/tuning/ProfilingAutoTunerSuite.scala | 522 ++++++++++-------- .../features/installation_checks.feature | 10 +- 8 files changed, 687 insertions(+), 502 deletions(-) create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/ClusterConfigurationStrategy.scala diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/ClusterConfigurationStrategy.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/ClusterConfigurationStrategy.scala new file mode 100644 index 000000000..3d4a96f39 --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/ClusterConfigurationStrategy.scala @@ -0,0 +1,213 @@ +/* + * Copyright (c) 2025, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool + +import org.apache.spark.sql.rapids.tool.ExistingClusterInfo +import org.apache.spark.sql.rapids.tool.util.StringUtils + +/** + * Configuration for the recommended cluster to run the application with Spark RAPIDS. + */ +case class RecommendedClusterConfig( + numExecutors: Int, + coresPerExec: Int, + numGpusPerNode: Int, + memoryPerNodeMb: Long // For onprem or cases where a matching CSP instance type is unavailable +) { + def execsPerNode: Int = { + numGpusPerNode + } + + def coresPerNode: Int = { + coresPerExec * execsPerNode + } +} + +/** + * Base trait for different cluster configuration strategies. + */ +abstract class ClusterConfigurationStrategy( + platform: Platform, + sparkProperties: Map[String, String]) { + + /** + * Calculates the initial number of executors based on the strategy. + */ + protected def calculateInitialNumExecutors: Int + + private def getInitialNumExecutors: Int = { + val dynamicAllocationEnabled = Platform.isDynamicAllocationEnabled(sparkProperties) + val execInstFromProps = sparkProperties.get("spark.executor.instances") + // If dynamic allocation is disabled, use spark.executor.instances in precedence + if (execInstFromProps.isDefined && !dynamicAllocationEnabled) { + // Spark Properties are in order: + // P0. User defined software properties in Cluster Properties + // P1. Spark Properties defined in the application information + execInstFromProps.get.toInt + } else { + calculateInitialNumExecutors + } + } + + /** + * Calculates the initial number of cores per executor based on the strategy. + */ + protected def calculateInitialCoresPerExec: Int + + private def getInitialCoresPerExec: Int = { + val coresFromProps = sparkProperties.get("spark.executor.cores") + // Use spark.executor.cores in precedence + if (coresFromProps.isDefined) { + coresFromProps.get.toInt + } else { + calculateInitialCoresPerExec + } + } + + protected def getMemoryPerNodeMb: Long + + /** + * Generates the recommended cluster configuration based on the strategy. + * + * Logic: + * 1. Calculate the initial number of executors and cores per executor. + * 2. Calculate the total core count by multiplying initial cores per executor + * by the initial number of executors. + * 3. Retrieve the recommended cores per executor from the platform (default is 16), + * for onprem, limit the recommended cores per executor to the total core count. + * 4. Calculate the recommended number of executors by dividing the total core count + * by the recommended cores per executor. + */ + final def getRecommendedConfig: Option[RecommendedClusterConfig] = { + val initialNumExecutors = getInitialNumExecutors + if (initialNumExecutors <= 0) { + None + } else { + val initialCoresPerExec = getInitialCoresPerExec + val totalCoresCount = initialCoresPerExec * initialNumExecutors + val recommendedCoresPerExec = if (platform.isPlatformCSP) { + platform.recommendedCoresPerExec + } else { + // For onprem, recommended cores per executor should not exceed total core count + math.min(platform.recommendedCoresPerExec, totalCoresCount) + } + val recommendedNumExecutors = + math.ceil(totalCoresCount.toDouble / recommendedCoresPerExec).toInt + Some(RecommendedClusterConfig( + numExecutors = recommendedNumExecutors, + coresPerExec = recommendedCoresPerExec, + numGpusPerNode = platform.recommendedGpusPerNode, + memoryPerNodeMb = getMemoryPerNodeMb)) + } + } +} + +/** + * Strategy for cluster configuration based on user specified cluster properties. + */ +class ClusterPropertyBasedStrategy( + platform: Platform, + sparkProperties: Map[String, String]) + extends ClusterConfigurationStrategy(platform, sparkProperties) { + + private val clusterProperties = platform.clusterProperties.getOrElse( + throw new IllegalArgumentException("Cluster properties must be defined")) + + // Calculate the number of GPUs per node based on the cluster properties + private lazy val numGpusFromProps: Int = { + // User provided num GPUs, fall back to platform default + val userProvidedNumGpus = clusterProperties.getGpu.getCount match { + case count if count > 0 => count + case _ => platform.defaultNumGpus + } + + // Apply platform-specific GPU limits for CSP, no limits for on-prem + if (platform.isPlatformCSP) { + math.min(userProvidedNumGpus, platform.maxGpusSupported) + } else { + userProvidedNumGpus + } + } + + override protected def calculateInitialNumExecutors: Int = { + val numWorkers = math.max(1, clusterProperties.system.numWorkers) + numGpusFromProps * numWorkers + } + + override protected def calculateInitialCoresPerExec: Int = { + val coresPerGpu = clusterProperties.system.getNumCores.toDouble / numGpusFromProps + math.ceil(coresPerGpu).toInt + } + + override protected def getMemoryPerNodeMb: Long = { + StringUtils.convertToMB(clusterProperties.system.getMemory) + } +} + +/** + * Strategy for cluster configuration based on cluster information from event log. + */ +class EventLogBasedStrategy( + platform: Platform, + sparkProperties: Map[String, String] + ) extends ClusterConfigurationStrategy(platform, sparkProperties) { + + private val clusterInfoFromEventLog: ExistingClusterInfo = { + platform.clusterInfoFromEventLog.getOrElse( + throw new IllegalArgumentException("Cluster information from event log must be defined")) + } + + // For onprem or cases where a matching CSP instance type is unavailable, + // Returns the memory per node + override def getMemoryPerNodeMb: Long = { + val heapMemMB = clusterInfoFromEventLog.executorHeapMemory + val overheadMemMB = platform.getExecutorOverheadMemoryMB(sparkProperties) + heapMemMB + overheadMemMB + } + + override def calculateInitialNumExecutors: Int = { + clusterInfoFromEventLog.numExecutors + } + + override def calculateInitialCoresPerExec: Int = { + clusterInfoFromEventLog.coresPerExecutor + } +} + +/** + * Companion object to create appropriate cluster configuration strategy. + * + * Strategy Precedence: + * 1. Cluster Properties based strategy + * 2. Event Log based strategy + */ +object ClusterConfigurationStrategy { + def getStrategy( + platform: Platform, + sparkProperties: Map[String, String]): Option[ClusterConfigurationStrategy] = { + if (platform.clusterProperties.isDefined) { + // Use strategy based on cluster properties + Some(new ClusterPropertyBasedStrategy(platform, sparkProperties)) + } else if (platform.clusterInfoFromEventLog.isDefined) { + // Use strategy based on cluster information from event log + Some(new EventLogBasedStrategy(platform, sparkProperties)) + } else { + // Neither cluster properties are defined nor cluster information from event log is available + None + } + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala index 7473c7698..e875f0f43 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/Platform.scala @@ -55,7 +55,21 @@ case class DynamicAllocationInfo(enabled: Boolean, max: String, min: String, ini // resource information and name of the CSP instance types, or for onprem its // the executor information since we can't recommend node types -case class InstanceInfo(cores: Int, memoryMB: Long, name: String, numGpus: Int) +case class InstanceInfo(cores: Int, memoryMB: Long, name: String, numGpus: Int) { + /** + * Get the memory per executor based on the instance type. + * Note: For GPU instances, num of executors is same as the number of GPUs. + */ + def getMemoryPerExec: Double = { + memoryMB.toDouble / numGpus + } +} + +object InstanceInfo { + def createDefaultInstance(cores: Int, memoryMB: Long, numGpus: Int): InstanceInfo = { + InstanceInfo(cores, memoryMB, "N/A", numGpus) + } +} // This is meant to be temporary mapping to figure out instance type based // on the number of GPUs and cores. Eventually this should be read from files @@ -131,8 +145,13 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], var recommendedNodeInstanceInfo: Option[InstanceInfo] = None // overall final recommended cluster configuration var recommendedClusterInfo: Option[RecommendedClusterInfo] = None - // the number of GPUs to use, this might be updated as we handle different cases - var numGpus: Int = 1 + + // Default recommendation based on NDS benchmarks (note: this could be platform specific) + def recommendedCoresPerExec = 16 + // Default number of GPUs to use, currently we do not support multiple GPUs per node + def recommendedGpusPerNode = 1 + def defaultNumGpus: Int = 1 + // Default runtime for the platform val defaultRuntime: SparkRuntime.SparkRuntime = SparkRuntime.SPARK // Set of supported runtimes for the platform @@ -256,12 +275,6 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], this.gpuDevice = Some(gpuDevice) } - final def setNumGpus(numGpus: Int): Unit = { - if (numGpus > 1) { - this.numGpus = numGpus - } - } - /** * Important system properties that should be retained based on platform. */ @@ -333,56 +346,6 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], } } - def getNumGPUsPerNode(): Int = { - val gpus = if (clusterProperties.isDefined) { - clusterProperties.get.gpu.getCount - } else { - // assume using 1 GPU per node unless specified - recommendedNodeInstanceInfo.map(_.numGpus).getOrElse(1) - } - math.max(1, gpus) - } - - // we want to keep the number of executors used between runs the same - def getNumExecutorInstances(sparkProperties: Map[String, String]): Int = { - val dynamicAllocationEnabled = Platform.isDynamicAllocationEnabled(sparkProperties) - val execInstFromProps = sparkProperties.get("spark.executor.instances") - // If the cluster properties were specified make sure to use those and not - // the eventlog inference. This is broken in my mind but is backwards compatible, - // or maybe use number gpus per node as an improvement. - if (clusterProperties.isDefined) { - val numWorkers = Math.max(1, clusterProperties.get.system.numWorkers) - this.numGpus * numWorkers - } else if (execInstFromProps.isDefined && !dynamicAllocationEnabled) { - execInstFromProps.get.toInt - } else if (clusterInfoFromEventLog.isDefined) { - clusterInfoFromEventLog.get.numExecutors - } else { - // not sure so don't set it - 0 - } - } - - // figure out memory MB per node when we don't have the specific instance information - def getMemoryMBPerNode(sparkProperties: Map[String, String]): Long = { - // To keep backwards compatibility, we first check if cluster properties are defined and - // use those as the source cluster. This is going to be wrong in many - // cases if the eventlogs passed in are not all actually run on the same cluster - // shape. Ideally we change this in the future. - if (clusterProperties.isDefined) { - StringUtils.convertToMB(clusterProperties.get.system.getMemory) - } else if (clusterInfoFromEventLog.isDefined) { - val numExecutorsPerNode = clusterInfoFromEventLog.map(_.numExecsPerNode) - .getOrElse(1).toLong - val heapMemMB = clusterInfoFromEventLog.get.executorHeapMemory - val overheadMemMB = getExecutorOverheadMemoryMB(sparkProperties) - (heapMemMB + overheadMemMB) * numExecutorsPerNode - } else { - // we don't know - 0L - } - } - def createClusterInfo(coresPerExecutor: Int, numExecsPerNode: Int, numExecs: Int, @@ -430,132 +393,113 @@ abstract class Platform(var gpuDevice: Option[GpuDevice], def maxGpusSupported: Int = 1 /** - * Attempts to get the instance type based on the core and gpu requirements. + * Get the mapping of number of GPUs and cores to the instance type. + * Format (numGpus, numCores) -> InstanceInfo about the matching CSP node instance type. */ - def getInstanceByResources(cores: Int, numGpus: Int): Option[InstanceInfo] = None + def getInstanceByResourcesMap: Map[(Int, Int), InstanceInfo] = Map.empty /** - * Recommend a GPU Instance type to use for this application. + * Get the instance type based on the core and gpu requirements. + * Case A: Find the exact instance type based on the required resources. + * Case B: If the exact instance type is not found, find the instance type with the smallest + * combined GPU and core count that meets the requirements. + * Case C: For onprem or cases where a matching CSP instance type is unavailable. */ - def getGPUInstanceTypeRecommendation( - sparkProperties: Map[String, String]): Option[RecommendedClusterInfo] = { - val vendor = clusterInfoFromEventLog.map(_.vendor).getOrElse("") - val numExecs = getNumExecutorInstances(sparkProperties) - // If the cluster properties were specified make sure to use those and not - // the eventlog inference. This is broken in my mind but is backwards compatible, - // or maybe use number gpus per node as an improvement. - val origClusterNumExecsPerNode = clusterInfoFromEventLog.map(_.numExecsPerNode).getOrElse(1) - val numExecsPerNode = if (clusterProperties.isEmpty) { - // numExecsPerNode can be -1 if dynamic allocation so just make it 1 for - // this set of calculations. However if we are on a CSP then we want to recommend - // the best size machine so use the number of GPUs as proxy to be the number of executors - // we could put on a node. - if (origClusterNumExecsPerNode == -1) { - maxGpusSupported - } else { - origClusterNumExecsPerNode + private def getInstanceByResources( + recommendedClusterConfig: RecommendedClusterConfig): InstanceInfo = { + val requiredGpus = recommendedClusterConfig.numGpusPerNode + val requiredCores = recommendedClusterConfig.coresPerNode + + // A. Find the exact instance type based on the required resources + getInstanceByResourcesMap.get((requiredGpus, requiredCores)).orElse { + // B. If the exact instance type is not found, find the instance type with the smallest + // combined GPU and core count that meets the requirements + val suitableInstances = getInstanceByResourcesMap.filterKeys { case (gpus, cores) => + gpus >= requiredGpus && cores >= requiredCores } - } else { - 1 - } - // onprem yarn multi-tenant vs yarn static cluster (dataproc) for just that application - // should be handled automatically unless heterogeneous nodes - val gpusToUse = - Math.max(this.numGpus, Math.min(numExecsPerNode, maxGpusSupported)) - - // update the global numGpus based on the instance type we are using - this.numGpus = gpusToUse - val nodeCores = if (clusterProperties.isDefined) { - logDebug("Using the cluster properties passed in.") - // I guess the assumption here is 1 executor per node - or we need to look this up - // since not in the cluster definition, either way this is number cores per node - clusterProperties.get.system.getNumCores - } else if (clusterInfoFromEventLog.isDefined) { - logDebug("Using the cluster information from the event log.") - val clusterInfo = clusterInfoFromEventLog.get - // this assumes this job filled an entire node, which may not be true on - // a multiple tenant cluster. If the number of executors ran per node would - // require multiple GPUs per node check to see if this platform supports it. - // If it doesn't we need to increase the number of nodes recommended. - clusterInfo.coresPerExecutor * gpusToUse - } else { - // shouldn't ever happen - logError("Cluster properties wasn't specified and cluster information couldn't be " + - "inferred from the event log!") - 0 - } - val instanceInfoOpt = getInstanceByResources(nodeCores, gpusToUse) - val finalInstanceInfo = if (instanceInfoOpt.isEmpty) { - // if the instance info isn't found, like onprem or some platform we don't know about - val execCores = if (clusterInfoFromEventLog.isDefined) { - clusterInfoFromEventLog.get.coresPerExecutor - } else { - logWarning("cluster information from event log is missing, executor cores set to 0!") - 0 - } - val nodeCoresToUse = execCores * gpusToUse - val nodeMemMB = getMemoryMBPerNode(sparkProperties) - // It's possible if a cpu run was used, it could run with multiple executors, but - // if the platform doesn't support multiple GPUs per node then we could recommend - // different number of nodes. We have to take this into account for cores and memory - // calculations. - val ratioExecs = Math.max(1, numExecsPerNode / gpusToUse) - val execMem = nodeMemMB / ratioExecs - logDebug(s"Creating instance info execCores $execCores execMem $execMem ratio " + - s"$ratioExecs numExecsPerNode $numExecsPerNode gpusToUse $gpusToUse") - // here we change instanceInfo to be executor because assumption is it's on prem and we - // don't know how to recommend node type - Some(InstanceInfo(nodeCoresToUse, execMem, "onprem", 1)) - } else if (clusterProperties.isDefined) { - val info = instanceInfoOpt.get - // make sure that instanceInfo matches the cluster properties else change - val clusterPropMemMB = StringUtils.convertToMB(clusterProperties.get.system.getMemory) - if (info.cores == nodeCores && info.memoryMB == clusterPropMemMB) { - instanceInfoOpt - } else { - Some(InstanceInfo(nodeCores, clusterPropMemMB, info.name, info.numGpus)) - } - } else { - instanceInfoOpt - } - // note this is going over as for instance if you have 4 gpus per node but only need - // 10 executors, this would tell you to allocate enough to fit 12. - val numNodes = math.ceil(numExecs.toDouble / finalInstanceInfo.get.numGpus).toInt - val coresPerExec = if (finalInstanceInfo.isDefined) { - // We may not be able to match instance type up exactly, this means the number of - // cores per executor could come out to be more then the original application. - // For now we want the cores per executor to stay the same as original app so if - // that is set, use it first. - if (clusterInfoFromEventLog.isDefined) { - clusterInfoFromEventLog.get.coresPerExecutor + if (suitableInstances.isEmpty) { + None } else { - finalInstanceInfo.get.cores / finalInstanceInfo.get.numGpus + val optimalNumGpusAndCoresPair = suitableInstances.keys.minBy { case (gpus, cores) => + gpus + cores + } + suitableInstances.get(optimalNumGpusAndCoresPair) } - } else { - 1 - } - val finalNumNodes = if (vendor == PlatformNames.ONPREM) { - // if its onprem we really have no idea of the size of the cluster - -1 - } else { - numNodes + }.getOrElse { + // C. For onprem or cases where a matching CSP instance type is unavailable + logDebug(s"Could not find a matching instance with requiredGpus=$requiredGpus," + + s" requiredCores=$requiredCores. Falling back to create a default instance info: \n" + + s"coresPerExec=${recommendedClusterConfig.coresPerExec}, " + + s"memoryPerNodeMb=${recommendedClusterConfig.memoryPerNodeMb}, " + + s"numGpusPerNode=${recommendedClusterConfig.numGpusPerNode}") + InstanceInfo.createDefaultInstance( + cores = recommendedClusterConfig.coresPerExec, + memoryMB = recommendedClusterConfig.memoryPerNodeMb, + numGpus = recommendedClusterConfig.numGpusPerNode) } - if (numExecs > 0) { - val instanceName = finalInstanceInfo.map(_.name).getOrElse("") - val numGpus = finalInstanceInfo.map(_.numGpus).getOrElse(1) - val dynamicAllocSettings = Platform.getDynamicAllocationSettings(sparkProperties) - // Num of executors per node is the number of GPUs - recommendedClusterInfo = Some(RecommendedClusterInfo(vendor, coresPerExec, - finalNumNodes, numGpus, numExecs, gpuDevice = getGpuOrDefault.toString, - dynamicAllocSettings.enabled, dynamicAllocSettings.max, - dynamicAllocSettings.min, dynamicAllocSettings.initial, - workerNodeType = Some(instanceName))) - recommendedNodeInstanceInfo = finalInstanceInfo - recommendedClusterInfo - } else { - logWarning("No executors so the recommended cluster and node instance information" + - " is not set!") - None + } + + /** + * Creates the recommended GPU cluster info based on Spark properties and + * cluster properties (either provided explicitly by the user or + * inferred from the event log). + * + * @param sparkProperties A map of Spark properties (combined from application and + * cluster properties) + * @return Optional `RecommendedClusterInfo` containing the GPU cluster configuration + * recommendation. + */ + def createRecommendedGpuClusterInfo(sparkProperties: Map[String, String]): Unit = { + // Get the appropriate cluster configuration strategy (either + // 'ClusterPropertyBasedStrategy' based on cluster properties or + // 'EventLogBasedStrategy' based on the event log). + val configurationStrategyOpt = ClusterConfigurationStrategy.getStrategy( + platform = this, sparkProperties = sparkProperties) + + configurationStrategyOpt match { + case Some(strategy) => + // Using the strategy, generate the recommended cluster configuration (num executors, + // cores per executor, num executors per node). + strategy.getRecommendedConfig match { + case Some(clusterConfig) => + val recommendedNodeInstance = getInstanceByResources(clusterConfig) + val vendor = clusterInfoFromEventLog.map(_.vendor).getOrElse("") + val numWorkerNodes = if (!isPlatformCSP) { + // For on-prem, we do not have the concept of worker nodes. + -1 + } else { + // Calculate number of worker nodes based on executors and GPUs per instance + math.ceil(clusterConfig.numExecutors.toDouble / + recommendedNodeInstance.numGpus).toInt + } + + val dynamicAllocSettings = Platform.getDynamicAllocationSettings(sparkProperties) + recommendedNodeInstanceInfo = Some(recommendedNodeInstance) + recommendedClusterInfo = Some(RecommendedClusterInfo( + vendor = vendor, + coresPerExecutor = clusterConfig.coresPerExec, + numWorkerNodes = numWorkerNodes, + numGpusPerNode = recommendedNodeInstance.numGpus, + numExecutors = clusterConfig.numExecutors, + gpuDevice = getGpuOrDefault.toString, + dynamicAllocationEnabled = dynamicAllocSettings.enabled, + dynamicAllocationMaxExecutors = dynamicAllocSettings.max, + dynamicAllocationMinExecutors = dynamicAllocSettings.min, + dynamicAllocationInitialExecutors = dynamicAllocSettings.initial, + workerNodeType = Some(recommendedNodeInstance.name) + )) + + case None => + logWarning("Failed to generate a cluster recommendation. " + + "Could not determine number of executors. " + + "Check the Spark properties used for this application or " + + "cluster properties (if provided).") + } + + case None => + logWarning("Failed to generate a cluster recommendation. " + + "Could not determine number of executors. " + + "Cluster properties are missing and event log does not contain cluster information.") } } } @@ -614,24 +558,8 @@ class DatabricksAwsPlatform(gpuDevice: Option[GpuDevice], override val platformName: String = PlatformNames.DATABRICKS_AWS override val defaultGpuDevice: GpuDevice = A10GGpu - override def getInstanceByResources( - cores: Int, numGpus: Int): Option[InstanceInfo] = { - val exactInstance = PlatformInstanceTypes.AWS_BY_GPUS_CORES.get((numGpus, cores)) - if (exactInstance.isEmpty) { - // try to find the closest - val enoughGpus = PlatformInstanceTypes.AWS_BY_GPUS_CORES.filterKeys { gpuCores => - gpuCores._1 >= numGpus && gpuCores._2 >= cores - } - if (enoughGpus.isEmpty) { - None - } else { - // add the gpus and cores to get a minimum value that matched. - val res = enoughGpus.keys.minBy(x => x._1 + x._2) - enoughGpus.get(res) - } - } else { - exactInstance - } + override def getInstanceByResourcesMap: Map[(Int, Int), InstanceInfo] = { + PlatformInstanceTypes.AWS_BY_GPUS_CORES } } @@ -642,24 +570,8 @@ class DatabricksAzurePlatform(gpuDevice: Option[GpuDevice], override def maxGpusSupported: Int = 4 - override def getInstanceByResources( - cores: Int, numGpus: Int): Option[InstanceInfo] = { - val exactInstance = PlatformInstanceTypes.AZURE_NCAS_T4_V3_BY_GPUS_CORES.get((numGpus, cores)) - if (exactInstance.isEmpty) { - // try to find the closest - val enoughGpus = PlatformInstanceTypes.AZURE_NCAS_T4_V3_BY_GPUS_CORES.filterKeys { gpuCores => - gpuCores._1 >= numGpus && gpuCores._2 >= cores - } - if (enoughGpus.isEmpty) { - None - } else { - // add the gpus and cores to get a minimum value that matched. - val res = enoughGpus.keys.minBy(x => x._1 + x._2) - enoughGpus.get(res) - } - } else { - exactInstance - } + override def getInstanceByResourcesMap: Map[(Int, Int), InstanceInfo] = { + PlatformInstanceTypes.AZURE_NCAS_T4_V3_BY_GPUS_CORES } } @@ -670,24 +582,8 @@ class DataprocPlatform(gpuDevice: Option[GpuDevice], override def isPlatformCSP: Boolean = true override def maxGpusSupported: Int = 4 - override def getInstanceByResources( - cores: Int, numGpus: Int): Option[InstanceInfo] = { - val exactInstance = PlatformInstanceTypes.DATAPROC_BY_GPUS_CORES.get((numGpus, cores)) - if (exactInstance.isEmpty) { - // try to find the closest - val enoughGpus = PlatformInstanceTypes.DATAPROC_BY_GPUS_CORES.filterKeys { gpuCores => - gpuCores._1 >= numGpus && gpuCores._2 >= cores - } - if (enoughGpus.isEmpty) { - None - } else { - // add the gpus and cores to get a minimum value that matched. - val res = enoughGpus.keys.minBy(x => x._1 + x._2) - enoughGpus.get(res) - } - } else { - exactInstance - } + override def getInstanceByResourcesMap: Map[(Int, Int), InstanceInfo] = { + PlatformInstanceTypes.DATAPROC_BY_GPUS_CORES } } @@ -732,24 +628,8 @@ class EmrPlatform(gpuDevice: Option[GpuDevice], driverHost = driverHost) } - override def getInstanceByResources( - cores: Int, numGpus: Int): Option[InstanceInfo] = { - val exactInstance = PlatformInstanceTypes.AWS_BY_GPUS_CORES.get((numGpus, cores)) - if (exactInstance.isEmpty) { - // try to find the closest - val enoughGpus = PlatformInstanceTypes.AWS_BY_GPUS_CORES.filterKeys { gpuCores => - gpuCores._1 >= numGpus && gpuCores._2 >= cores - } - if (enoughGpus.isEmpty) { - None - } else { - // add the gpus and cores to get a minimum value that matched. - val res = enoughGpus.keys.minBy(x => x._1 + x._2) - enoughGpus.get(res) - } - } else { - exactInstance - } + override def getInstanceByResourcesMap: Map[(Int, Int), InstanceInfo] = { + PlatformInstanceTypes.AWS_BY_GPUS_CORES } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala index f82124feb..484a9fa6a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualOutputWriter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -507,7 +507,7 @@ object QualOutputWriter { val DRIVER_HOST = "Driver Host" val CLUSTER_ID_STR = "Cluster Id" // Different from ClusterId used for Databricks Tags val CLUSTER_NAME = "Cluster Name" - val RECOMMENDED_NUM_GPUS = "Recommended Num GPUs Per Node" + val RECOMMENDED_NUM_GPUS_PER_NODE = "Recommended Num GPUs Per Node" val RECOMMENDED_GPU_DEVICE = "Recommended GPU Device" val NUM_EXECS_PER_NODE = "Num Executors Per Node" val NUM_EXECS = "Num Executors" @@ -855,7 +855,7 @@ object QualOutputWriter { NUM_EXECS, EXECUTOR_HEAP_MEMORY, DYN_ALLOC_ENABLED, DYN_ALLOC_MAX, DYN_ALLOC_MIN, DYN_ALLOC_INIT, CORES_PER_EXEC, RECOMMENDED_WORKER_NODE_TYPE, RECOMMENDED_NUM_EXECS, RECOMMENDED_NUM_WORKER_NODES, RECOMMENDED_CORES_PER_EXEC, RECOMMENDED_GPU_DEVICE, - RECOMMENDED_NUM_GPUS, RECOMMENDED_VENDOR, RECOMMENDED_DYN_ALLOC_ENABLED, + RECOMMENDED_NUM_GPUS_PER_NODE, RECOMMENDED_VENDOR, RECOMMENDED_DYN_ALLOC_ENABLED, RECOMMENDED_DYN_ALLOC_MAX, RECOMMENDED_DYN_ALLOC_MIN, RECOMMENDED_DYN_ALLOC_INIT).map { key => (key, key.length) } @@ -951,7 +951,8 @@ object QualOutputWriter { refactorCSVFuncWithOption(recClusterInfo.map(_.coresPerExecutor.toString), RECOMMENDED_CORES_PER_EXEC), refactorCSVFuncWithOption(recClusterInfo.map(_.gpuDevice), RECOMMENDED_GPU_DEVICE), - refactorCSVFuncWithOption(recClusterInfo.map(_.numGpus.toString), RECOMMENDED_NUM_GPUS), + refactorCSVFuncWithOption(recClusterInfo.map(_.numGpusPerNode.toString), + RECOMMENDED_NUM_GPUS_PER_NODE), refactorCSVFuncWithOption(recClusterInfo.map(_.vendor), RECOMMENDED_VENDOR), refactorCSVFuncWithOption(recClusterInfo.map(_.dynamicAllocationEnabled.toString), RECOMMENDED_DYN_ALLOC_ENABLED), diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index f7aca2ec4..be4824225 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -358,9 +358,10 @@ class AutoTuner( fromProfile.orElse(Option(clusterProps.softwareProperties.get(key))) } - def getAllProperties: collection.Map[String, String] = { - // the app properties override the cluster properties - clusterProps.getSoftwareProperties.asScala ++ appInfoProvider.getAllProperties + private lazy val getAllProperties: collection.Map[String, String] = { + // the cluster properties override the app properties as + // it is provided by the user. + appInfoProvider.getAllProperties ++ clusterProps.getSoftwareProperties.asScala } def initRecommendations(): Unit = { @@ -422,11 +423,11 @@ class AutoTuner( * Returns None if the platform doesn't support specific instance types. */ private def configureGPURecommendedInstanceType(): Unit = { - val gpuClusterRec = platform.getGPUInstanceTypeRecommendation(getAllProperties.toMap) - if (gpuClusterRec.isDefined) { - appendRecommendation("spark.executor.cores", gpuClusterRec.get.coresPerExecutor) - if (gpuClusterRec.get.numExecutors > 0) { - appendRecommendation("spark.executor.instances", gpuClusterRec.get.numExecutors) + platform.createRecommendedGpuClusterInfo(getAllProperties.toMap) + platform.recommendedClusterInfo.foreach { gpuClusterRec => + appendRecommendation("spark.executor.cores", gpuClusterRec.coresPerExecutor) + if (gpuClusterRec.numExecutors > 0) { + appendRecommendation("spark.executor.instances", gpuClusterRec.numExecutors) } } } @@ -453,17 +454,6 @@ class AutoTuner( Math.min(autoTunerConfigsProvider.MAX_CONC_GPU_TASKS, platform.getGpuOrDefault.getGpuConcTasks) } - /** - * Calculates the available memory for each executor on the worker based on the number of - * executors per node and the memory. - * Assumption - cluster properties were updated to have a default values if missing. - */ - private def calcAvailableMemPerExec(): Double = { - val memMBPerNode = platform.recommendedNodeInstanceInfo.map(_.memoryMB).getOrElse(0L) - val gpusPerExec = platform.getNumGPUsPerNode - Math.max(0, memMBPerNode / gpusPerExec) - } - /** * Recommendation for initial heap size based on certain amount of memory per core. * Note that we will later reduce this if needed for off heap memory. @@ -686,7 +676,8 @@ class AutoTuner( appendRecommendation("spark.task.resource.gpu.amount", calcTaskGPUAmount) appendRecommendation("spark.rapids.sql.concurrentGpuTasks", calcGpuConcTasks().toInt) - val availableMemPerExec = calcAvailableMemPerExec() + val availableMemPerExec = + platform.recommendedNodeInstanceInfo.map(_.getMemoryPerExec).getOrElse(0.0) val shouldSetMaxBytesInFlight = if (availableMemPerExec > 0.0) { val availableMemPerExecExpr = () => availableMemPerExec val executorHeap = calcInitialExecutorHeap(availableMemPerExecExpr, execCores) @@ -1176,12 +1167,11 @@ class AutoTuner( if (platform.gpuDevice.isEmpty && !clusterProps.isEmpty && !clusterProps.gpu.isEmpty) { GpuDevice.createInstance(clusterProps.gpu.getName) .foreach(platform.setGpuDevice) - platform.setNumGpus(clusterProps.gpu.getCount) } // configured GPU recommended instance type NEEDS to happen before any of the other // recommendations as they are based on // the instance type - configureGPURecommendedInstanceType + configureGPURecommendedInstanceType() configureClusterPropDefaults // Makes recommendations based on information extracted from the AppInfoProvider filterByUpdatedPropertiesEnabled = showOnlyUpdatedProps diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ClassWarehouse.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ClassWarehouse.scala index b4dc0bc43..bb70d62fd 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/ClassWarehouse.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/ClassWarehouse.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -72,7 +72,7 @@ case class RecommendedClusterInfo( vendor: String, coresPerExecutor: Int, numWorkerNodes: Int, - numGpus: Int, + numGpusPerNode: Int, numExecutors: Int, gpuDevice: String, dynamicAllocationEnabled: Boolean, @@ -82,7 +82,7 @@ case class RecommendedClusterInfo( driverNodeType: Option[String] = None, workerNodeType: Option[String] = None) extends ClusterInfo { // The number of executors per node is the same as the number of GPUs - def numExecsPerNode: Int = numGpus + def numExecsPerNode: Int = numGpusPerNode } case class ClusterSummary( diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala index fbac09fd6..acb51b826 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/BaseAutoTunerSuite.scala @@ -140,4 +140,21 @@ abstract class BaseAutoTunerSuite extends FunSuite with BeforeAndAfterEach with sparkVersion, rapidsJars, distinctLocationPct, redundantReadSize, meanInput, meanShuffleRead, shuffleStagesWithPosSpilling, shuffleSkewStages) } + + /** + * Helper method to compare the expected results with the actual output from the AutoTuner. + * + * In case of a mismatch, displays the complete output, which is useful for updating the + * expected results. + */ + protected def compareOutput(expectedResults: String, autoTunerOutput: String): Unit = { + val outputsMatch = expectedResults == autoTunerOutput + assert(outputsMatch, + s"""|=== Expected === + |$expectedResults + | + |=== Actual === + |$autoTunerOutput + |""".stripMargin) + } } diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala index 146038a94..5c08f20f4 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala @@ -171,7 +171,7 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { | distribution, this step is not needed. |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with CPU cores 0") { @@ -213,14 +213,14 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { | distribution, this step is not needed. |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with memory to cores ratio to small") { val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(8), Some("14000MiB")) val clusterPropsOpt = ProfilingAutoTunerConfigsProvider .loadClusterPropertiesFromContent(dataprocWorkerInfo) - val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) + val platform = PlatformFactory.createInstance(PlatformNames.ONPREM, clusterPropsOpt) val autoTuner = ProfilingAutoTunerConfigsProvider .buildAutoTunerFromProps(dataprocWorkerInfo, getGpuAppMockInfoProvider, platform) @@ -230,7 +230,7 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { val expectedComment = s"""This node/worker configuration is not ideal for using the Spark Rapids Accelerator because it doesn't have enough memory for the executors. -We recommend using nodes/workers with more memory. Need at least 7796MB memory.""".stripMargin.replaceAll("\n", "") +We recommend using nodes/workers with more memory. Need at least 17496MB memory.""".stripMargin.replaceAll("\n", "") // scalastyle:on line.size.limit assert(autoTunerOutput.replaceAll("\n", "").contains(expectedComment)) } @@ -274,7 +274,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." | distribution, this step is not needed. |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with CPU memory 0") { @@ -293,6 +293,10 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |Spark Properties: |--conf spark.executor.cores=16 |--conf spark.executor.instances=8 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m + |--conf spark.rapids.memory.pinnedPool.size=4096m + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 @@ -310,8 +314,10 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |Comments: |- 'spark.executor.cores' was not set. |- 'spark.executor.instances' was not set. - |- 'spark.executor.memory' should be set to at least 2GB/core. - |- 'spark.rapids.memory.pinnedPool.size' should be set to 2048m. + |- 'spark.executor.memory' was not set. + |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -327,18 +333,11 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.sql.files.maxPartitionBytes' was not set. |- 'spark.sql.shuffle.partitions' was not set. |- 'spark.task.resource.gpu.amount' was not set. - |- RAPIDS Accelerator for Apache Spark plugin jar is missing - | from the classpath entries. - | If the Spark RAPIDS jar is being bundled with your - | Spark distribution, this step is not needed. - |- The RAPIDS Shuffle Manager requires spark.driver.extraClassPath - | and spark.executor.extraClassPath settings to include the - | path to the Spark RAPIDS plugin jar. - | If the Spark RAPIDS jar is being bundled with your Spark - | distribution, this step is not needed. + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with number of workers 0") { @@ -402,7 +401,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with GPU count of 0") { @@ -430,22 +429,19 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=32 |--conf spark.executor.instances=4 - |--conf spark.executor.memory=65536m - |--conf spark.executor.memoryOverhead=20889m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=48 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=48 + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=64 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.03125 | |Comments: |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -465,7 +461,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with GPU memory is missing") { @@ -523,7 +519,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with GPU memory 0") { @@ -579,7 +575,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with GPU name missing") { @@ -637,7 +633,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Load cluster properties with unknown GPU device") { @@ -696,10 +692,12 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } - test("Test executor memory on CSP where executor/cpu ratio is small") { + // This test should ignore executor memory specified in the properties as + // CSP nodes have fixed memory configurations. + test("Test executor memory on CSP where executor memory/cpu ratio is small") { val dataprocWorkerInfo = buildGpuWorkerInfoAsString(None, Some(8), Some("15360MiB"), Some(4), Some(1)) val clusterPropsOpt = ProfilingAutoTunerConfigsProvider @@ -714,22 +712,25 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=4 - |--conf spark.executor.memory=6000m - |--conf spark.executor.memoryOverhead=11776m + |--conf spark.executor.cores=16 + |--conf spark.executor.instances=2 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.cores' was not set. @@ -737,11 +738,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- 'spark.executor.memory' was not set. |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. |- 'spark.rapids.sql.concurrentGpuTasks' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. @@ -754,7 +758,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("test T4 dataproc cluster with dynamicAllocation enabled") { @@ -809,7 +813,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } // This mainly to test that the executorInstances will be calculated when the dynamic allocation @@ -817,7 +821,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." test("test T4 dataproc cluster with missing dynamic allocation") { val customProps = mutable.LinkedHashMap( "spark.dynamicAllocation.enabled" -> "false", - "spark.executor.cores" -> "16", + "spark.executor.cores" -> "32", "spark.executor.memory" -> "32768m", "spark.executor.memoryOverhead" -> "8396m", "spark.rapids.memory.pinnedPool.size" -> "4096m", @@ -834,7 +838,8 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.instances=8 + |--conf spark.executor.cores=16 + |--conf spark.executor.instances=4 |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 @@ -867,7 +872,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("test AutoTuner with empty sparkProperties") { @@ -930,7 +935,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("AutoTuner detects non UTF-8 file-encoding") { @@ -980,25 +985,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=4096m - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. @@ -1008,15 +1017,14 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- file.encoding should be [UTF-8] because GPU only supports the charset when using some expressions. |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } - // Test that the properties from the eventlogs will be used to calculate the recommendations. - // For example, the output should recommend "spark.executor.cores" -> 8. Although the cluster - // "spark.executor.cores" is the same as the recommended value, the property is set to 16 in - // the eventlogs. - test("AutoTuner gives precedence to properties from eventlogs") { + // Test that the properties from the custom props will be used to calculate the recommendations. + // For example, the output should use "spark.executor.cores" -> 4 when calculating the + // recommendations. + test("AutoTuner gives precedence to properties from custom props") { val customProps = mutable.LinkedHashMap( - "spark.executor.cores" -> "8", + "spark.executor.cores" -> "4", "spark.executor.memory" -> "47222m", "spark.rapids.shuffle.multiThreaded.reader.threads" -> "8", "spark.rapids.shuffle.multiThreaded.writer.threads" -> "8", @@ -1056,25 +1064,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=5 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=4096m - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. @@ -1083,7 +1095,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("plugin set to the wrong values") { @@ -1122,29 +1134,33 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=4096m - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. @@ -1155,7 +1171,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } // Why this test to see if it carries over the sql.enabled false???? This is broken @@ -1197,30 +1213,34 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 |--conf spark.rapids.sql.enabled=true - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=4096m - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. @@ -1230,7 +1250,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } // Changing the maxInput of tasks should reflect on the maxPartitions recommendations. @@ -1277,24 +1297,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -1352,24 +1377,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -1378,7 +1408,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } // Ensure handling of integer values for maxPartitionBytes is handled @@ -1422,24 +1452,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -1448,7 +1483,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } def generateRecommendationsForRapidsJars(rapidsJars: Seq[String]): String = { @@ -1513,7 +1548,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val rapidsJarsArr = Seq("rapids-4-spark_2.12-23.06.0-SNAPSHOT.jar", "rapids-4-spark_2.12-23.02.1.jar") val autoTunerOutput = generateRecommendationsForRapidsJars(rapidsJarsArr) - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Recommend upgrading to the latest plugin release") { @@ -1559,7 +1594,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |""".stripMargin val rapidsJarsArr = Seq(s"rapids-4-spark_2.12-$jarVer.jar") val autoTunerOutput = generateRecommendationsForRapidsJars(rapidsJarsArr) - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("No recommendation when the jar pluginJar is up-to-date") { @@ -1599,7 +1634,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |""".stripMargin val rapidsJarsArr = Seq(s"rapids-4-spark_2.12-$latestRelease.jar") val autoTunerOutput = generateRecommendationsForRapidsJars(rapidsJarsArr) - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Recommend file cache if parquet/orc and data thresholds are met") { @@ -1642,26 +1677,31 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.filecache.enabled=true |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.filecache.enabled' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -1671,7 +1711,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Do not recommend file cache if small redundant size") { @@ -1714,24 +1754,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -1740,7 +1785,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("test recommendations for databricks-aws platform argument") { @@ -1806,23 +1851,27 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." // scalastyle:off line.size.limit val expectedResults = s"""| - |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m + |Spark Properties: + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=4096m - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. @@ -1831,7 +1880,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Recommendations generated for unsupported operators from driver logs only") { @@ -1867,7 +1916,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Recommendations generated for unsupported operators from driver and event logs") { @@ -1920,7 +1969,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.commentForExperimentalConfig("spark.rapids.sql.incompatibleDateFormats.enabled")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Recommendations generated for empty unsupported operators from driver logs only") { @@ -1941,7 +1990,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." s"""|Cannot recommend properties. See Comments. |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Recommendations not generated for unsupported operators from driver logs") { @@ -1969,7 +2018,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." s"""|Cannot recommend properties. See Comments. |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("AQE configuration autoBroadcastJoinThreshold should not be GTE 100mb") { @@ -2011,28 +2060,32 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." // scalastyle:off line.size.limit val expectedResults = s"""| - |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |Spark Properties: + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=4096m - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. @@ -2041,7 +2094,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } private def testPartitionConfigurations( @@ -2154,24 +2207,29 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." s"""| |Spark Properties: |--conf spark.databricks.adaptive.autoOptimizeShuffle.enabled=false - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -2180,7 +2238,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } /** @@ -2358,26 +2416,30 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m |--conf spark.sql.shuffle.partitions=400 - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -2388,7 +2450,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Test spilling occurred in shuffle stages with shuffle skew") { @@ -2432,26 +2494,30 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.files.maxPartitionBytes=3669m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.125 | |Comments: |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. |- 'spark.sql.adaptive.enabled' should be enabled for better performance. @@ -2464,7 +2530,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Test Kryo Serializer sets Registrar") { @@ -2494,33 +2560,38 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.kryo.registrator=com.nvidia.spark.rapids.GpuKryoRegistrator |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' was not set. |- 'spark.kryo.registrator' was not set. |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. |- 'spark.rapids.sql.concurrentGpuTasks' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. @@ -2534,7 +2605,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Test Kryo Serializer sets Registrar when already set") { @@ -2565,32 +2636,37 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.kryo.registrator=org.apache.SomeRegistrator,org.apache.SomeOtherRegistrator,com.nvidia.spark.rapids.GpuKryoRegistrator |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. |- 'spark.rapids.sql.concurrentGpuTasks' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. @@ -2604,7 +2680,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Test Kryo Serializer sets Registrar when already set but empty") { @@ -2635,32 +2711,37 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.cores=8 - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=16384m - |--conf spark.executor.memoryOverhead=11878m + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.kryo.registrator=com.nvidia.spark.rapids.GpuKryoRegistrator |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=20 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=20 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=40 + |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 + |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. |- 'spark.rapids.sql.concurrentGpuTasks' was not set. + |- 'spark.rapids.sql.format.parquet.multithreaded.combine.waitTime' was not set. |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.rapids.sql.reader.multithreaded.combine.sizeBytes' was not set. |- 'spark.shuffle.manager' was not set. |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. @@ -2674,7 +2755,7 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } test("Test EMR sets shuffle manager properly and doesn't need Spark RAPIDS jar") { @@ -2703,27 +2784,30 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." val expectedResults = s"""| |Spark Properties: - |--conf spark.executor.instances=20 - |--conf spark.executor.memory=24000m - |--conf spark.executor.memoryOverhead=15564m + |--conf spark.executor.cores=16 + |--conf spark.executor.instances=10 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=17612m |--conf spark.rapids.memory.pinnedPool.size=4096m - |--conf spark.rapids.shuffle.multiThreaded.reader.threads=48 - |--conf spark.rapids.shuffle.multiThreaded.writer.threads=48 + |--conf spark.rapids.shuffle.multiThreaded.maxBytesInFlight=4g + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=28 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=28 |--conf spark.rapids.sql.batchSizeBytes=2147483647 |--conf spark.rapids.sql.concurrentGpuTasks=2 |--conf spark.rapids.sql.format.parquet.multithreaded.combine.waitTime=1000 - |--conf spark.rapids.sql.multiThreadedRead.numThreads=64 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=80 |--conf spark.rapids.sql.reader.multithreaded.combine.sizeBytes=10485760 |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark341.RapidsShuffleManager |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m |--conf spark.sql.files.maxPartitionBytes=512m |--conf spark.sql.shuffle.partitions=200 - |--conf spark.task.resource.gpu.amount=0.03125 + |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -2743,6 +2827,6 @@ We recommend using nodes/workers with more memory. Need at least 7796MB memory." |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(expectedResults == autoTunerOutput) + compareOutput(expectedResults, autoTunerOutput) } } diff --git a/user_tools/tests/spark_rapids_tools_e2e/features/installation_checks.feature b/user_tools/tests/spark_rapids_tools_e2e/features/installation_checks.feature index a05191d46..d4b0c4f63 100644 --- a/user_tools/tests/spark_rapids_tools_e2e/features/installation_checks.feature +++ b/user_tools/tests/spark_rapids_tools_e2e/features/installation_checks.feature @@ -1,4 +1,4 @@ -# Copyright (c) 2024, NVIDIA CORPORATION. +# Copyright (c) 2024-2025, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -28,10 +28,10 @@ Feature: Tool Installation Checks Examples: | platform | cli | expected_stdout | - | dataproc | gcloud | 3 x n1-standard-16 (4 T4 each) | - | emr | aws | 10 x g5.xlarge | - | databricks-aws | aws | 10 x g5.xlarge | - | databricks-azure | az | 3 x Standard_NC64as_T4_v3 | + | dataproc | gcloud | 2 x n1-standard-16 (1 T4 each) | + | emr | aws | 2 x g5.4xlarge | + | databricks-aws | aws | 2 x g5.4xlarge | + | databricks-azure | az | 2 x Standard_NC16as_T4_v3 | @test_id_IC_0002 Scenario: Environment has missing java From a1ed715b283745e0fec894faea968918408b0e0c Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Wed, 22 Jan 2025 10:45:28 -0800 Subject: [PATCH 46/52] [BUG] Remove duplicated executor CPU time and runtime metric from SQLTaskAggMetricsProfileResult (#1504) Fixes https://github.com/NVIDIA/spark-rapids-tools/issues/1467. `SQLTaskAggMetricsProfileResult` has 2 duplicate fields: - `executorRunTime` and `executorRunTimeSum` - `executorCPUTime` and `executorCPUTimeSum` ### Changes - Removed duplicated metric fields from `SQLTaskAggMetricsProfileResult` - Updated existing unit tests ### Affected Output 1. `sql_level_aggregated_task_metrics.csv` Removed columns: `executorCPUTime`, `executorRunTime` --------- Signed-off-by: cindyyuanjiang --- .../analysis/AppSparkMetricsAnalyzer.scala | 4 +- .../profiling/ApplicationSummaryInfo.scala | 4 +- .../profiling/ProfileClassWarehouse.scala | 54 +++++++++++++------ .../tuning/QualAppSummaryInfoProvider.scala | 4 +- .../tool/views/AggMetricsResultSorter.scala | 4 +- ...on_db_13_3_sql_metrics_agg_expectation.csv | 4 +- ...in_eventlog_sqlmetricsagg2_expectation.csv | 4 +- ...oin_eventlog_sqlmetricsagg_expectation.csv | 4 +- ...ventlog_sqlmetricsaggmulti_expectation.csv | 6 +-- 9 files changed, 53 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index b68f18899..1804f1946 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -193,8 +193,6 @@ class AppSparkMetricsAnalyzer(app: AppBase) extends AppAnalysisBase(app) { sqlCase.description, preSqlRec.numTasks, sqlCase.duration, - preSqlRec.executorCPUTimeSum, - preSqlRec.executorRunTimeSum, preSqlRec.executorCpuRatio, preSqlRec.diskBytesSpilledSum, preSqlRec.durationSum, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala index 05c304fd9..93d17f981 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationSummaryInfo.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -128,7 +128,7 @@ class SingleAppSummaryInfoProvider(val app: ApplicationSummaryInfo) override def getJvmGCFractions: Seq[Double] = { app.sqlTaskAggMetrics.map { - taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCpuTime + taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCPUTimeSum } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index ed03cea46..796b97f20 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -773,8 +773,6 @@ case class SQLTaskAggMetricsProfileResult( description: String, numTasks: Int, duration: Option[Long], - executorCpuTime: Long, // milliseconds - executorRunTime: Long, executorCpuRatio: Double, diskBytesSpilledSum: Long, durationSum: Long, @@ -808,17 +806,43 @@ case class SQLTaskAggMetricsProfileResult( swWriteTimeSum: Long // milliseconds ) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "appID", "sqlID", "description", "numTasks", - "Duration", "executorCPUTime", "executorRunTime", "executorCPURatio", - "diskBytesSpilled_sum", "duration_sum", "duration_max", "duration_min", - "duration_avg", "executorCPUTime_sum", "executorDeserializeCPUTime_sum", - "executorDeserializeTime_sum", "executorRunTime_sum", "input_bytesRead_sum", - "input_recordsRead_sum", "jvmGCTime_sum", "memoryBytesSpilled_sum", - "output_bytesWritten_sum", "output_recordsWritten_sum", "peakExecutionMemory_max", - "resultSerializationTime_sum", "resultSize_max", "sr_fetchWaitTime_sum", - "sr_localBlocksFetched_sum", "sr_localBytesRead_sum", "sr_remoteBlocksFetched_sum", - "sr_remoteBytesRead_sum", "sr_remoteBytesReadToDisk_sum", "sr_totalBytesRead_sum", - "sw_bytesWritten_sum", "sw_recordsWritten_sum", "sw_writeTime_sum") + override val outputHeaders = { + Seq("appIndex", + "appID", + "sqlID", + "description", + "numTasks", + "Duration", + "executorCPURatio", + "diskBytesSpilled_sum", + "duration_sum", + "duration_max", + "duration_min", + "duration_avg", + "executorCPUTime_sum", + "executorDeserializeCPUTime_sum", + "executorDeserializeTime_sum", + "executorRunTime_sum", + "input_bytesRead_sum", + "input_recordsRead_sum", + "jvmGCTime_sum", + "memoryBytesSpilled_sum", + "output_bytesWritten_sum", + "output_recordsWritten_sum", + "peakExecutionMemory_max", + "resultSerializationTime_sum", + "resultSize_max", + "sr_fetchWaitTime_sum", + "sr_localBlocksFetched_sum", + "sr_localBytesRead_sum", + "sr_remoteBlocksFetched_sum", + "sr_remoteBytesRead_sum", + "sr_remoteBytesReadToDisk_sum", + "sr_totalBytesRead_sum", + "sw_bytesWritten_sum", + "sw_recordsWritten_sum", + "sw_writeTime_sum") + } val durStr = duration match { case Some(dur) => dur.toString @@ -832,8 +856,6 @@ case class SQLTaskAggMetricsProfileResult( description, numTasks.toString, durStr, - executorCpuTime.toString, - executorRunTime.toString, executorCpuRatio.toString, diskBytesSpilledSum.toString, durationSum.toString, @@ -871,8 +893,6 @@ case class SQLTaskAggMetricsProfileResult( StringUtils.reformatCSVString(description), numTasks.toString, durStr, - executorCpuTime.toString, - executorRunTime.toString, executorCpuRatio.toString, diskBytesSpilledSum.toString, durationSum.toString, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualAppSummaryInfoProvider.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualAppSummaryInfoProvider.scala index dbbf6cd11..0a806f1c0 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualAppSummaryInfoProvider.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/QualAppSummaryInfoProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -68,7 +68,7 @@ class QualAppSummaryInfoProvider( override def getJvmGCFractions: Seq[Double] = { rawAggMetrics.sqlAggs.map { - taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCpuTime + taskMetrics => taskMetrics.jvmGCTimeSum * 1.0 / taskMetrics.executorCPUTimeSum } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala index 440a4ac64..47a52078f 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/AggMetricsResultSorter.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -50,7 +50,7 @@ object AggMetricsResultSorter { } else { rows.sortBy { cols => val sortDur = cols.duration.getOrElse(0L) - (cols.appIndex, -sortDur, cols.sqlId, cols.executorCpuTime) + (cols.appIndex, -sortDur, cols.sqlId, cols.executorCPUTimeSum) } } } diff --git a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv index 5e1116c98..373b4b34d 100644 --- a/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/nds_q88_photon_db_13_3_sql_metrics_agg_expectation.csv @@ -1,2 +1,2 @@ -appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"app-20240919162642-0000",26,"query88",3472,250542,2885555,3818106,75.58,0,3858136,6743,54,1111.2,2885555,13523,18186,3818106,52997115316,69120188398,16100,0,0,0,250840493,181,16203,1394,1759,201596,1750,201200,0,402796,218614,19946,154 +appIndex,appID,sqlID,description,numTasks,Duration,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum +1,"app-20240919162642-0000",26,"query88",3472,250542,75.58,0,3858136,6743,54,1111.2,2885555,13523,18186,3818106,52997115316,69120188398,16100,0,0,0,250840493,181,16203,1394,1759,201596,1750,201200,0,402796,218614,19946,154 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv index b4fa1dff0..084921fac 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg2_expectation.csv @@ -1,2 +1,2 @@ -appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"local-1622821994212",0,"count at :28",213,3041,7151,13522,52.88,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 +appIndex,appID,sqlID,description,numTasks,Duration,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum +1,"local-1622821994212",0,"count at :28",213,3041,52.88,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv index 1cf88cdb6..ca0a2e308 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsagg_expectation.csv @@ -1,2 +1,2 @@ -appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"local-1622814619968",0,"count at :28",213,3087,6608,13414,49.26,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 +appIndex,appID,sqlID,description,numTasks,Duration,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum +1,"local-1622814619968",0,"count at :28",213,3087,49.26,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv index fab4b6a71..d02130039 100644 --- a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_sqlmetricsaggmulti_expectation.csv @@ -1,3 +1,3 @@ -appIndex,appID,sqlID,description,numTasks,Duration,executorCPUTime,executorRunTime,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum -1,"local-1622814619968",0,"count at :28",213,3087,6608,13414,49.26,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 -2,"local-1622821994212",0,"count at :28",213,3041,7151,13522,52.88,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 +appIndex,appID,sqlID,description,numTasks,Duration,executorCPURatio,diskBytesSpilled_sum,duration_sum,duration_max,duration_min,duration_avg,executorCPUTime_sum,executorDeserializeCPUTime_sum,executorDeserializeTime_sum,executorRunTime_sum,input_bytesRead_sum,input_recordsRead_sum,jvmGCTime_sum,memoryBytesSpilled_sum,output_bytesWritten_sum,output_recordsWritten_sum,peakExecutionMemory_max,resultSerializationTime_sum,resultSize_max,sr_fetchWaitTime_sum,sr_localBlocksFetched_sum,sr_localBytesRead_sum,sr_remoteBlocksFetched_sum,sr_remoteBytesRead_sum,sr_remoteBytesReadToDisk_sum,sr_totalBytesRead_sum,sw_bytesWritten_sum,sw_recordsWritten_sum,sw_writeTime_sum +1,"local-1622814619968",0,"count at :28",213,3087,49.26,0,26735,1598,10,125.5,6608,3531,12095,13414,0,0,336,0,0,0,0,8,8075,0,2600,80279908,0,0,0,80279908,80279908,2600,1001 +2,"local-1622821994212",0,"count at :28",213,3041,52.88,0,25761,1624,9,120.9,7151,3134,11178,13522,0,0,424,0,0,0,0,10,8075,0,2600,80279920,0,0,0,80279920,80279920,2600,901 From 9f410290c1bb69380488271882532e4bf3ef91dc Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Wed, 22 Jan 2025 14:48:16 -0600 Subject: [PATCH 47/52] Add stringType and binaryType to the list of dataType map (#1506) Signed-off-by: Ahmed Hussein (amahussein) Fixes #1492 Adds 2 new entries to the dataTypes map defined in the PluginTypeChecker - string: varchar, car - binary: blob --- .../tool/qualification/PluginTypeChecker.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala index b4eeb04af..ac6eb677d 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/PluginTypeChecker.scala @@ -318,15 +318,17 @@ class PluginTypeChecker(platform: Platform = PlatformFactory.createInstance(), (allSupportedReadSources.toMap, allSupportedWriteFormats) } - def getOtherTypes(typeRead: String): Seq[String] = { + private def getOtherTypes(typeRead: String): Seq[String] = { typeRead match { - case "long" => Seq("bigint") - case "short" => Seq("smallint") - case "int" => Seq("integer") + case "binary" => Seq("blob") case "byte" => Seq("tinyint") - case "float" => Seq("real") - case "decimal" => Seq("dec", "numeric") case "calendar" => Seq("interval") + case "decimal" => Seq("dec", "numeric") + case "float" => Seq("real") + case "int" => Seq("integer") + case "long" => Seq("bigint") + case "short" => Seq("smallint") + case "string" => Seq("varchar", "char") case _ => Seq.empty[String] } } From f2a6d626547538d6cb1e2f623b62858ce43b99c0 Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Wed, 22 Jan 2025 14:54:43 -0800 Subject: [PATCH 48/52] Sync plugin support as of 2024-12-31 (#1478) Fixes https://github.com/NVIDIA/spark-rapids-tools/issues/1452 This PR syns plugin support as of 2024-12-31. The changes include: ``` This report documents the differences between the tools existing CSV files and those processed from the plugin. Notes: 1. For new data source/exec/expression from plugin, the first column with supported level will be updated to 'TNEW' for future testing. 2. Rows marked as "is removed" will be preserved in the final output. 3. The "Notes" column for rows with "S" for "Supported" will be updated to "None" in the final output. **supportedDataSource.csv (FROM TOOLS TO PLUGIN)** Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO BOOLEAN: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO BYTE: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO SHORT: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO INT: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO LONG: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO FLOAT: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO DOUBLE: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO DATE: CO -> PS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO TIMESTAMP: CO -> PS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO STRING: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO DECIMAL: CO -> S Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO NULL: CO -> NA Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO BINARY: CO -> NS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO CALENDAR: CO -> NA Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO ARRAY: CO -> PS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO MAP: CO -> NS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO STRUCT: CO -> PS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO UDT: CO -> NS Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO DAYTIME: CO -> NA Row is changed: JSON, read, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO, CO YEARMONTH: CO -> NA **supportedExecs.csv (FROM TOOLS TO PLUGIN)** Row is removed: CustomShuffleReaderExec, S, None, Input/Output, S, S, S, S, S, S, S, S, PS, S, S, S, S, NS, PS, PS, PS, NS, NS, NS Row is removed: RunningWindowFunctionExec, S, None, Input/Output, S, S, S, S, S, S, S, S, PS, S, S, S, NS, NS, PS, PS, PS, NS, NS, NS **supportedExprs.csv (FROM TOOLS TO PLUGIN)** Row is changed: HiveHash, S, `hive-hash`, None, project, input, S, S, S, S, S, S, S, S, PS, S, NS, S, NS, NS, NS, NS, NS, NS, NS, NS ARRAY: NS -> PS Row is changed: HiveHash, S, `hive-hash`, None, project, input, S, S, S, S, S, S, S, S, PS, S, NS, S, NS, NS, NS, NS, NS, NS, NS, NS STRUCT: NS -> PS Row is changed: JsonToStructs, NS, `from_json`, This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case, project, jsonStr, NA, NA, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Supported: NS -> S Row is changed: JsonToStructs, NS, `from_json`, This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case, project, jsonStr, NA, NA, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Notes: This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case -> None Row is changed: JsonToStructs, NS, `from_json`, This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case, project, result, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NS, PS, PS, NA, NA, NA Supported: NS -> S Row is changed: JsonToStructs, NS, `from_json`, This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case, project, result, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NS, PS, PS, NA, NA, NA Notes: This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case -> None Row is added: MonthsBetween, TNEW, `months_between`, None, project, timestamp1, NA, NA, NA, NA, NA, NA, NA, NA, PS, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: MonthsBetween, TNEW, `months_between`, None, project, timestamp2, NA, NA, NA, NA, NA, NA, NA, NA, PS, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: MonthsBetween, TNEW, `months_between`, None, project, round, PS, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: MonthsBetween, TNEW, `months_between`, None, project, result, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: TruncDate, TNEW, `trunc`, None, project, date, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: TruncDate, TNEW, `trunc`, None, project, format, NA, NA, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: TruncDate, TNEW, `trunc`, None, project, result, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: TruncTimestamp, TNEW, `date_trunc`, None, project, format, NA, NA, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: TruncTimestamp, TNEW, `date_trunc`, None, project, date, NA, NA, NA, NA, NA, NA, NA, NA, PS, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is added: TruncTimestamp, TNEW, `date_trunc`, None, project, result, NA, NA, NA, NA, NA, NA, NA, NA, PS, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA Row is changed: XxHash64, S, `xxhash64`, None, project, input, S, S, S, S, S, S, S, S, PS, S, S, S, NS, NS, NS, NS, NS, NS, NS, NS ARRAY: NS -> PS Row is changed: XxHash64, S, `xxhash64`, None, project, input, S, S, S, S, S, S, S, S, PS, S, S, S, NS, NS, NS, NS, NS, NS, NS, NS MAP: NS -> PS Row is changed: XxHash64, S, `xxhash64`, None, project, input, S, S, S, S, S, S, S, S, PS, S, S, S, NS, NS, NS, NS, NS, NS, NS, NS STRUCT: NS -> PS Row is removed: EphemeralSubstring, S, `substr`; `substring`, None, project, str, NA, NA, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NS, NA, NA, NA, NA, NA, NS, NS Row is removed: EphemeralSubstring, S, `substr`; `substring`, None, project, pos, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NS, NS Row is removed: EphemeralSubstring, S, `substr`; `substring`, None, project, len, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NS, NS Row is removed: EphemeralSubstring, S, `substr`; `substring`, None, project, result, NA, NA, NA, NA, NA, NA, NA, NA, NA, S, NA, NA, NS, NA, NA, NA, NA, NA, NS, NS Row is removed: DecimalSum, S, `decimalsum`, None, project, input, NA, NA, NA, NA, S, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, NS, NS Row is removed: DecimalSum, S, `decimalsum`, None, project, result, NA, NA, NA, NA, NA, NA, NA, NA, NA, NA, PS, NA, NA, NA, NA, NA, NA, NA, NS, NS ``` --------- Signed-off-by: cindyyuanjiang --- .../operatorsScore-databricks-aws-a10G.csv | 3 + .../operatorsScore-databricks-aws-t4.csv | 3 + .../operatorsScore-databricks-azure-t4.csv | 3 + .../operatorsScore-dataproc-gke-l4.csv | 3 + .../operatorsScore-dataproc-gke-t4.csv | 3 + .../resources/operatorsScore-dataproc-l4.csv | 3 + .../operatorsScore-dataproc-serverless-l4.csv | 3 + .../resources/operatorsScore-dataproc-t4.csv | 3 + .../main/resources/operatorsScore-emr-a10.csv | 3 + .../resources/operatorsScore-emr-a10G.csv | 3 + .../main/resources/operatorsScore-emr-t4.csv | 3 + .../resources/operatorsScore-onprem-a100.csv | 3 + .../main/resources/supportedDataSource.csv | 2 +- core/src/main/resources/supportedExecs.csv | 4 +- core/src/main/resources/supportedExprs.csv | 18 +- .../qual_test_simple_expectation.csv | 6 +- .../qual_test_simple_expectation_persql.csv | 20 +- .../read_dsv1_expectation.csv | 2 +- .../read_dsv2_expectation.csv | 2 +- .../tool/planparser/SqlPlanParserSuite.scala | 325 ++++++++---------- 20 files changed, 213 insertions(+), 202 deletions(-) diff --git a/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv b/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv index 52eb193f2..a8c6d601e 100644 --- a/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv +++ b/core/src/main/resources/operatorsScore-databricks-aws-a10G.csv @@ -306,3 +306,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-databricks-aws-t4.csv b/core/src/main/resources/operatorsScore-databricks-aws-t4.csv index 52eb193f2..a8c6d601e 100644 --- a/core/src/main/resources/operatorsScore-databricks-aws-t4.csv +++ b/core/src/main/resources/operatorsScore-databricks-aws-t4.csv @@ -306,3 +306,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-databricks-azure-t4.csv b/core/src/main/resources/operatorsScore-databricks-azure-t4.csv index 5ad387036..a52420be4 100644 --- a/core/src/main/resources/operatorsScore-databricks-azure-t4.csv +++ b/core/src/main/resources/operatorsScore-databricks-azure-t4.csv @@ -294,3 +294,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv b/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv index 902e598a1..83c5f2690 100644 --- a/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-gke-l4.csv @@ -288,3 +288,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv b/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv index e30f156f4..dc0e0b421 100644 --- a/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-gke-t4.csv @@ -288,3 +288,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-l4.csv b/core/src/main/resources/operatorsScore-dataproc-l4.csv index 0660dbdee..ffd67f62b 100644 --- a/core/src/main/resources/operatorsScore-dataproc-l4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-l4.csv @@ -294,3 +294,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv b/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv index 8dc9faa90..03d6f403f 100644 --- a/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-serverless-l4.csv @@ -288,3 +288,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-dataproc-t4.csv b/core/src/main/resources/operatorsScore-dataproc-t4.csv index e2eb69f60..6ec40be20 100644 --- a/core/src/main/resources/operatorsScore-dataproc-t4.csv +++ b/core/src/main/resources/operatorsScore-dataproc-t4.csv @@ -294,3 +294,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-emr-a10.csv b/core/src/main/resources/operatorsScore-emr-a10.csv index 0d350be80..93f28d20e 100644 --- a/core/src/main/resources/operatorsScore-emr-a10.csv +++ b/core/src/main/resources/operatorsScore-emr-a10.csv @@ -294,3 +294,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-emr-a10G.csv b/core/src/main/resources/operatorsScore-emr-a10G.csv index 0d350be80..93f28d20e 100644 --- a/core/src/main/resources/operatorsScore-emr-a10G.csv +++ b/core/src/main/resources/operatorsScore-emr-a10G.csv @@ -294,3 +294,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-emr-t4.csv b/core/src/main/resources/operatorsScore-emr-t4.csv index c651cf976..1e3c87af8 100644 --- a/core/src/main/resources/operatorsScore-emr-t4.csv +++ b/core/src/main/resources/operatorsScore-emr-t4.csv @@ -294,3 +294,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/operatorsScore-onprem-a100.csv b/core/src/main/resources/operatorsScore-onprem-a100.csv index 7cdd59978..741ef85fa 100644 --- a/core/src/main/resources/operatorsScore-onprem-a100.csv +++ b/core/src/main/resources/operatorsScore-onprem-a100.csv @@ -306,3 +306,6 @@ MaxBy,1.5 MinBy,1.5 ArrayJoin,1.5 RunningWindowFunctionExec,1.5 +MonthsBetween,1.5 +TruncDate,1.5 +TruncTimestamp,1.5 diff --git a/core/src/main/resources/supportedDataSource.csv b/core/src/main/resources/supportedDataSource.csv index 77f30cbe1..82df521b3 100644 --- a/core/src/main/resources/supportedDataSource.csv +++ b/core/src/main/resources/supportedDataSource.csv @@ -6,7 +6,7 @@ Delta,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA HiveText,read,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS HiveText,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Iceberg,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S -JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO +JSON,read,S,S,S,S,S,S,S,PS,PS,S,S,NA,NS,NA,PS,NS,PS,NS,NA,NA ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS,NA,NA ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Parquet,read,S,S,S,S,S,S,S,S,PS,S,S,NA,S,NA,PS,PS,PS,NS,S,S diff --git a/core/src/main/resources/supportedExecs.csv b/core/src/main/resources/supportedExecs.csv index 50e47f42f..e426af19e 100644 --- a/core/src/main/resources/supportedExecs.csv +++ b/core/src/main/resources/supportedExecs.csv @@ -21,6 +21,7 @@ SortAggregateExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,PS,NS,PS,PS,PS,NS InMemoryTableScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,PS,PS,PS,NS,S,S DataWritingCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,PS,NS,S,NS,PS,PS,PS,NS,S,S ExecutedCommandExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S +WriteFilesExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S AppendDataExecV1,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S AtomicCreateTableAsSelectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S AtomicReplaceTableAsSelectExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,S,NS,PS,PS,PS,NS,S,S @@ -53,8 +54,7 @@ WindowInPandasExec,NS,This is disabled by default because it only supports row b WindowExec,S,None,partitionSpec,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS HiveTableScanExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,NS,NS,NS,NS,NS,NS,NS,NS,NS -WriteFilesExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S,S,S -CustomShuffleReaderExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS WindowGroupLimitExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS MapInArrowExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,NS,NS,NS,NS,PS,NS,PS,NS,NS,NS +CustomShuffleReaderExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,NS,NS RunningWindowFunctionExec,S,None,Input/Output,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS diff --git a/core/src/main/resources/supportedExprs.csv b/core/src/main/resources/supportedExprs.csv index bdc93f62c..a79a4eae5 100644 --- a/core/src/main/resources/supportedExprs.csv +++ b/core/src/main/resources/supportedExprs.csv @@ -269,7 +269,7 @@ GreaterThanOrEqual,S,`>=`,None,AST,rhs,S,S,S,S,S,NS,NS,S,PS,S,NS,NS,NS,NS,NS,NA, GreaterThanOrEqual,S,`>=`,None,AST,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Greatest,S,`greatest`,None,project,param,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA Greatest,S,`greatest`,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NA,NS,NS,NA,NA -HiveHash,S,`hive-hash`,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,NS,NS,NS,NS,NS,NS +HiveHash,S,`hive-hash`,None,project,input,S,S,S,S,S,S,S,S,PS,S,NS,S,NS,NS,PS,NS,PS,NS,NS,NS HiveHash,S,`hive-hash`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Hour,S,`hour`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -299,8 +299,8 @@ IsNotNull,S,`isnotnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,P IsNotNull,S,`isnotnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA IsNull,S,`isnull`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,S,NS,PS,PS,PS,NS,S,NS IsNull,S,`isnull`,None,project,result,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA -JsonToStructs,NS,`from_json`,This is disabled by default because it is currently in beta and undergoes continuous enhancements. Please consult the [compatibility documentation](../compatibility.md#json-supporting-types) to determine whether you can enable this configuration for your use case,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA +JsonToStructs,S,`from_json`,None,project,jsonStr,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +JsonToStructs,S,`from_json`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NS,PS,PS,NA,NA,NA JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,json,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,field,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA JsonTuple,NS,`json_tuple`,This is disabled by default because Experimental feature that could be unstable or have performance issues.,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA @@ -384,6 +384,10 @@ Minute,S,`minute`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,N MonotonicallyIncreasingID,S,`monotonically_increasing_id`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Month,S,`month`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Month,S,`month`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,timestamp1,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,timestamp2,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,round,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +MonthsBetween,S,`months_between`,None,project,result,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Multiply,S,`*`,None,project,lhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA Multiply,S,`*`,None,project,rhs,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA Multiply,S,`*`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA @@ -625,6 +629,12 @@ TransformKeys,S,`transform_keys`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA, TransformValues,S,`transform_values`,None,project,argument,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA TransformValues,S,`transform_values`,None,project,function,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS TransformValues,S,`transform_values`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncDate,S,`trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,format,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,date,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA +TruncTimestamp,S,`date_trunc`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA UnaryMinus,S,`negative`,None,project,input,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,project,result,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NS,NA,NA,NA,NA,S,S UnaryMinus,S,`negative`,None,AST,input,NA,NS,NS,S,S,S,S,NA,NA,NA,NS,NA,NA,NS,NA,NA,NA,NA,NS,NS @@ -650,7 +660,7 @@ WindowExpression,S, ,None,window,result,S,S,S,S,S,S,S,S,PS,S,S,S,S,S,PS,PS,PS,S, WindowSpecDefinition,S, ,None,project,partition,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,value,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS WindowSpecDefinition,S, ,None,project,result,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,PS,NS,NS,NS -XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,NS,NS,NS,NS,NS,NS +XxHash64,S,`xxhash64`,None,project,input,S,S,S,S,S,S,S,S,PS,S,S,S,NS,NS,PS,PS,PS,NS,NS,NS XxHash64,S,`xxhash64`,None,project,result,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,input,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Year,S,`year`,None,project,result,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA diff --git a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv index 62421d9bf..4ba9381e5 100644 --- a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv @@ -1,5 +1,5 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds "Rapids Spark Profiling Tool Unit Tests","local-1622043423018",11600,132257,16319,9868,37.7,"","","JSON","","","",7143,13770,4719,19744,112513,false,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",1,186 -"Spark shell","local-1651187225439",224,180,355637,74,87.88,"","JSON[string:bigint:int]","","","","",498,228,355101,120,60,false,"SerializeFromObject;CollectLimit;DeserializeToObject;Scan json;Filter;MapElements","",1,2834 -"Spark shell","local-1651188809790",347,283,166215,14,81.18,"","JSON[string:bigint:int]","","","","UDF",715,318,165572,271,12,false,"CollectLimit;Scan json;Project","UDF",1,1318 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390",1156,4666,6240,1,46.27,"","JSON[string:bigint:int]","JSON","","","UDF",1209,1130,5809,4661,5,false,"Execute InsertIntoHadoopFsRelationCommand json;LocalTableScan;Project;Scan json;Execute CreateViewCommand","UDF",1,64 +"Spark shell","local-1651187225439",224,180,355637,142,87.88,"","","","","","",498,228,355101,66,114,false,"SerializeFromObject;CollectLimit;DeserializeToObject;Filter;MapElements","",1,2834 +"Spark shell","local-1651188809790",347,283,166215,128,81.18,"","","","","","UDF",715,318,165572,178,105,false,"CollectLimit;Project","UDF",1,1318 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390",1156,4666,6240,122,46.27,"","","JSON","","","UDF",1209,1130,5809,4170,496,false,"Execute InsertIntoHadoopFsRelationCommand json;LocalTableScan;Execute CreateViewCommand;Project","UDF",1,64 diff --git a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv index d78088e22..7d29bc48c 100644 --- a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv +++ b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation_persql.csv @@ -2,17 +2,17 @@ App Name,App ID,Root SQL ID,SQL ID,SQL Description,SQL DF Duration,GPU Opportuni "Rapids Spark Profiling Tool Unit Tests","local-1622043423018","",1,"count at QualificationInfoUtils.scala:94",7143,6719 "Rapids Spark Profiling Tool Unit Tests","local-1622043423018","",3,"count at QualificationInfoUtils.scala:94",2052,1660 "Rapids Spark Profiling Tool Unit Tests","local-1622043423018","",2,"count at QualificationInfoUtils.scala:94",1933,1551 -"Spark shell","local-1651188809790","",1,"show at :26",196,75 -"Spark shell","local-1651187225439","",0,"show at :26",498,168 -"Spark shell","local-1651187225439","",1,"show at :26",262,80 +"Spark shell","local-1651187225439","",0,"show at :26",498,333 +"Spark shell","local-1651188809790","",0,"show at :26",715,242 "Rapids Spark Profiling Tool Unit Tests","local-1622043423018","",0,"json at QualificationInfoUtils.scala:76",1306,164 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",0,"json at QualificationInfoUtils.scala:130",1209,0 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",2,"json at QualificationInfoUtils.scala:136",321,0 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",6,"json at QualificationInfoUtils.scala:130",110,0 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",3,"json at QualificationInfoUtils.scala:130",108,0 +"Spark shell","local-1651188809790","",1,"show at :26",196,135 +"Spark shell","local-1651187225439","",1,"show at :26",262,110 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",2,"json at QualificationInfoUtils.scala:136",321,107 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",5,"json at QualificationInfoUtils.scala:136",129,43 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",8,"json at QualificationInfoUtils.scala:136",127,42 "Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",4,"createOrReplaceTempView at QualificationInfoUtils.scala:133",22,22 "Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",7,"createOrReplaceTempView at QualificationInfoUtils.scala:133",4,4 "Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",1,"createOrReplaceTempView at QualificationInfoUtils.scala:133",2,2 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",5,"json at QualificationInfoUtils.scala:136",129,0 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",8,"json at QualificationInfoUtils.scala:136",127,0 -"Spark shell","local-1651188809790","",0,"show at :26",715,5 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",0,"json at QualificationInfoUtils.scala:130",1209,0 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",6,"json at QualificationInfoUtils.scala:130",110,0 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390","",3,"json at QualificationInfoUtils.scala:130",108,0 diff --git a/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv b/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv index b725defc0..73af0bcca 100644 --- a/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1624371544219",4575,20421,175293,1523,72.15,"","JSON[string:double:date:int:bigint];Text[*]","JSON","","","",1859,5372,176916,13622,6799,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json;Scan json","",30,2096 +"Spark shell","local-1624371544219",4575,20421,175293,4365,72.15,"","Text[*]","JSON","","","",1859,5372,176916,938,19483,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json","",30,2096 diff --git a/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv b/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv index 4c7726207..1af748da3 100644 --- a/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1624371906627",4917,21802,83738,2687,71.3,"","Text[*];json[double]","JSON","","","",1984,5438,83336,9889,11913,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json;BatchScan json","",30,997 +"Spark shell","local-1624371906627",4917,21802,83738,4762,71.3,"","Text[*]","JSON","","","",1984,5438,83336,689,21113,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json","",30,997 diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala index 4d2c93c89..e0a4cc436 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/planparser/SqlPlanParserSuite.scala @@ -26,8 +26,10 @@ import com.nvidia.spark.rapids.tool.planparser.ops.{ExprOpRef, OpRef} import com.nvidia.spark.rapids.tool.qualification._ import org.scalatest.Matchers.{be, contain, convertToAnyShouldWrapper} import org.scalatest.exceptions.TestFailedException +import org.scalatest.prop.TableDrivenPropertyChecks._ +import org.scalatest.prop.TableFor2 -import org.apache.spark.sql.{DataFrame, TrampolineUtil} +import org.apache.spark.sql.{DataFrame, SparkSession, TrampolineUtil} import org.apache.spark.sql.execution.ui.SQLPlanMetric import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ @@ -281,7 +283,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val parquet = allExecInfo.filter(_.exec.contains("Scan parquet")) val text = allExecInfo.filter(_.exec.contains("Scan text")) val csv = allExecInfo.filter(_.exec.contains("Scan csv")) - assertSizeAndNotSupported(2, json) + assertSizeAndSupported(2, json) assertSizeAndNotSupported(1, text) for (t <- Seq(parquet, csv)) { assertSizeAndSupported(1, t) @@ -304,7 +306,7 @@ class SQLPlanParserSuite extends BasePlanParserSuite { val orc = allExecInfo.filter(_.exec.contains("BatchScan orc")) val parquet = allExecInfo.filter(_.exec.contains("BatchScan parquet")) val csv = allExecInfo.filter(_.exec.contains("BatchScan csv")) - assertSizeAndNotSupported(3, json) + assertSizeAndSupported(3, json) assertSizeAndSupported(1, csv) for (t <- Seq(orc, parquet)) { assertSizeAndSupported(2, t) @@ -1018,37 +1020,6 @@ class SQLPlanParserSuite extends BasePlanParserSuite { } } - test("Expressions supported in ProjectExec") { - TrampolineUtil.withTempDir { parquetoutputLoc => - TrampolineUtil.withTempDir { eventLogDir => - val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, - "ProjectExprsSupported") { spark => - import spark.implicits._ - - import org.apache.spark.sql.types.StringType - val df1 = Seq(9.9, 10.2, 11.6, 12.5).toDF("value") - df1.write.parquet(s"$parquetoutputLoc/testtext") - val df2 = spark.read.parquet(s"$parquetoutputLoc/testtext") - df2.select(df2("value").cast(StringType), ceil(df2("value")), df2("value")) - } - val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) - assert(app.sqlPlans.size == 2) - val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => - SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - } - verifyExecToStageMapping(parsedPlans.toSeq, app) - val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) - val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) - assert(wholeStages.size == 1) - assert(wholeStages.forall(_.duration.nonEmpty)) - val allChildren = wholeStages.flatMap(_.children).flatten - val projects = allChildren.filter(_.exec == "Project") - assertSizeAndSupported(1, projects) - } - } - } - test("Expressions not supported in ProjectExec") { TrampolineUtil.withTempDir { parquetoutputLoc => TrampolineUtil.withTempDir { eventLogDir => @@ -1080,95 +1051,152 @@ class SQLPlanParserSuite extends BasePlanParserSuite { } } - test("translate is supported in ProjectExec") { - TrampolineUtil.withTempDir { parquetoutputLoc => - TrampolineUtil.withTempDir { eventLogDir => - val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, - "ProjectExprsSupported") { spark => - import spark.implicits._ - val df1 = Seq("", "abc", "ABC", "AaBbCc").toDF("value") - // write df1 to parquet to transform LocalTableScan to ProjectExec - df1.write.parquet(s"$parquetoutputLoc/testtext") - val df2 = spark.read.parquet(s"$parquetoutputLoc/testtext") - // translate should be part of ProjectExec - df2.select(translate(df2("value"), "ABC", "123")) - } - val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) - assert(app.sqlPlans.size == 2) - val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => - SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - } - verifyExecToStageMapping(parsedPlans.toSeq, app) - val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) - val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) - assert(wholeStages.size == 1) - assert(wholeStages.forall(_.duration.nonEmpty)) - val allChildren = wholeStages.flatMap(_.children).flatten - val projects = allChildren.filter(_.exec == "Project") - assertSizeAndSupported(1, projects) - } - } + /** + * Helper function to write a DataFrame to Parquet and then read it back. + * + * @param spark The SparkSession instance. + * @param df The input DataFrame to be written. + * @param path The file path to store the Parquet file. + * @return A DataFrame read from the Parquet file. + */ + private def writeAndReadParquet(spark: SparkSession, df: DataFrame, path: String): DataFrame = { + df.write.parquet(path) + spark.read.parquet(path) } - test("Timestamp functions supported in ProjectExec") { - TrampolineUtil.withTempDir { parquetoutputLoc => - TrampolineUtil.withTempDir { eventLogDir => - val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, - "ProjectExprsSupported") { spark => - import spark.implicits._ - val init_df = Seq((1230219000123123L, 1230219000123L, 1230219000.123)) - val df1 = init_df.toDF("micro", "millis", "seconds") - df1.write.parquet(s"$parquetoutputLoc/testtext") - val df2 = spark.read.parquet(s"$parquetoutputLoc/testtext") - df2.selectExpr("timestamp_micros(micro)", "timestamp_millis(millis)", - "timestamp_seconds(seconds)") - } - val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) - assert(app.sqlPlans.size == 2) - val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => - SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - } - verifyExecToStageMapping(parsedPlans.toSeq, app) - val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) - val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) - assert(wholeStages.size == 1) - assert(wholeStages.forall(_.duration.nonEmpty)) - val allChildren = wholeStages.flatMap(_.children).flatten - val projects = allChildren.filter(_.exec == "Project") - assertSizeAndSupported(1, projects) + /** + * Table-driven test cases for verifying Spark SQL expressions in ProjectExec. + */ + val projectExecTestCases: TableFor2[String, (File => (SparkSession => DataFrame))] = Table( + ("Expression", "FileToSparkSession"), + // MonthsBetween is supported in ProjectExec + ("MonthsBetween", { parquetOutputLoc => { spark => + import spark.implicits._ + val df1 = Seq(("2024-12-01", "2024-01-01"), + ("2024-12-01", "2023-12-01"), + ("2024-12-01", "2024-12-01")).toDF("date1", "date2") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // months_between should be part of ProjectExec + df2.select(months_between(df2("date1"), df2("date2"))) + }}), + // TruncDate is supported in ProjectExec + ("TruncDate", { parquetOutputLoc => { spark => + import spark.implicits._ + val df1 = Seq("2024-12-15", "2024-01-10", "2023-11-05").toDF("date") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // trunc should be part of ProjectExec + df2.select(trunc(df2("date"), "month")) + }}), + // TruncTimestamp is supported in ProjectExec + ("TruncTimestamp", { parquetOutputLoc => { spark => + import spark.implicits._ + val data = Seq("2024-12-15 14:30:45", + "2024-01-10 08:15:00", + "2023-11-05 20:45:30").toDF("timestamp") + val df1 = data.withColumn("timestamp", to_timestamp(col("timestamp"))) + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // date_trunc should be part of ProjectExec + df2.select(date_trunc("month", df2("timestamp"))) + }}), + // Ceil is supported in ProjectExec + ("Ceil", { parquetOutputLoc => { spark => + import spark.implicits._ + import org.apache.spark.sql.types.StringType + val df1 = Seq(9.9, 10.2, 11.6, 12.5).toDF("value") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // ceil should be part of ProjectExec + df2.select(df2("value").cast(StringType), ceil(df2("value")), df2("value")) + }}), + // Translate is supported in ProjectExec + ("Translate", { parquetOutputLoc => { spark => + import spark.implicits._ + val df1 = Seq("", "abc", "ABC", "AaBbCc").toDF("value") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // translate should be part of ProjectExec + df2.select(translate(df2("value"), "ABC", "123")) + }}), + // Timestamp functions are supported in ProjectExec + ("TimestampFunctions", { parquetOutputLoc => { spark => + import spark.implicits._ + val init_df = Seq((1230219000123123L, 1230219000123L, 1230219000.123)) + val df1 = init_df.toDF("micro", "millis", "seconds") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // timestamp functions should be part of ProjectExec + df2.selectExpr("timestamp_micros(micro)", "timestamp_millis(millis)", + "timestamp_seconds(seconds)") + }}), + // Flatten is supported in ProjectExec + ("Flatten", { parquetOutputLoc => { spark => + import spark.implicits._ + val df1 = Seq(Seq(Seq(1, 2), Seq(3, 4))).toDF("value") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // flatten should be part of ProjectExec + df2.select(flatten(df2("value"))) + }}), + // Xxhash64 is supported in ProjectExec + ("Xxhash64", { parquetOutputLoc => { spark => + import spark.implicits._ + val df1 = Seq("spark", "", "abc").toDF("value") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // xxhash64 should be part of ProjectExec + df2.select(xxhash64(df2("value"))) + }}), + // MapFromArrays is supported in ProjectExec + ("MapFromArrays", { parquetOutputLoc => { spark => + import spark.implicits._ + val df1 = Seq((Array("a", "b", "c"), Array(1, 2, 3)), + (Array("x", "y", "z"), Array(10, 20, 30))).toDF("keys", "values") + // write df1 to parquet to transform LocalTableScan to ProjectExec + val df2 = writeAndReadParquet(spark, df1, s"$parquetOutputLoc/testtext") + // map_from_arrays should be part of ProjectExec + df2.select(map_from_arrays(df2("keys"), df2("values")).as("map")) + }}) + ) + + /** + * Tests whether a given Spark SQL expression is supported in ProjectExec. + * + * @param appName Name of the Spark application. + * @param fileToSparkSession Function that maps a temporary Parquet directory to a function that + * takes a SparkSession and returns a DataFrame. + * @param parquetOutputLoc Temporary directory used for writing and reading Parquet files. + */ + private def testExpressionInProjectExec(appName: String, + fileToSparkSession: File => (SparkSession => DataFrame), + parquetOutputLoc: File): Unit = { + TrampolineUtil.withTempDir { eventLogDir => + val (eventLog, _) = + ToolTestUtils.generateEventLog(eventLogDir, appName)(fileToSparkSession(parquetOutputLoc)) + val pluginTypeChecker = new PluginTypeChecker() + val app = createAppFromEventlog(eventLog) + assert(app.sqlPlans.size == 2) + val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => + SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) } + verifyExecToStageMapping(parsedPlans.toSeq, app) + val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) + val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) + assert(wholeStages.size == 1) + assert(wholeStages.forall(_.duration.nonEmpty)) + val allChildren = wholeStages.flatMap(_.children).flatten + val projects = allChildren.filter(_.exec == "Project") + assertSizeAndSupported(1, projects) } } - test("flatten is supported in ProjectExec") { - TrampolineUtil.withTempDir { parquetoutputLoc => - TrampolineUtil.withTempDir { eventLogDir => - val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, - "ProjectExprsSupported") { spark => - import spark.implicits._ - val df1 = Seq(Seq(Seq(1, 2), Seq(3, 4))).toDF("value") - // write df1 to parquet to transform LocalTableScan to ProjectExec - df1.write.parquet(s"$parquetoutputLoc/testtext") - val df2 = spark.read.parquet(s"$parquetoutputLoc/testtext") - // flatten should be part of ProjectExec - df2.select(flatten(df2("value"))) - } - val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) - assert(app.sqlPlans.size == 2) - val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => - SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - } - verifyExecToStageMapping(parsedPlans.toSeq, app) - val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) - val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) - assert(wholeStages.size == 1) - assert(wholeStages.forall(_.duration.nonEmpty)) - val allChildren = wholeStages.flatMap(_.children).flatten - val projects = allChildren.filter(_.exec == "Project") - assertSizeAndSupported(1, projects) + forAll(projectExecTestCases) { (exprName, fileToSparkSession) => + test(s"$exprName is supported in ProjectExec") { + TrampolineUtil.withTempDir { parquetOutputLoc => + testExpressionInProjectExec(s"{$exprName}SupportedInProjectExec", fileToSparkSession, + parquetOutputLoc) } } } @@ -1235,37 +1263,6 @@ class SQLPlanParserSuite extends BasePlanParserSuite { } } - test("xxhash64 is supported in ProjectExec") { - TrampolineUtil.withTempDir { parquetoutputLoc => - TrampolineUtil.withTempDir { eventLogDir => - val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, - "ProjectExprsSupported") { spark => - import spark.implicits._ - val df1 = Seq("spark", "", "abc").toDF("value") - // write df1 to parquet to transform LocalTableScan to ProjectExec - df1.write.parquet(s"$parquetoutputLoc/testtext") - val df2 = spark.read.parquet(s"$parquetoutputLoc/testtext") - // xxhash64 should be part of ProjectExec - df2.select(xxhash64(df2("value"))) - } - val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) - assert(app.sqlPlans.size == 2) - val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => - SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - } - verifyExecToStageMapping(parsedPlans.toSeq, app) - val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) - val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) - assert(wholeStages.size == 1) - assert(wholeStages.forall(_.duration.nonEmpty)) - val allChildren = wholeStages.flatMap(_.children).flatten - val projects = allChildren.filter(_.exec == "Project") - assertSizeAndSupported(1, projects) - } - } - } - test("Parse SQL function Name in HashAggregateExec") { TrampolineUtil.withTempDir { eventLogDir => val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, "sqlmetric") { spark => @@ -1290,38 +1287,6 @@ class SQLPlanParserSuite extends BasePlanParserSuite { } } - test("map_from_arrays is supported in ProjectExec") { - TrampolineUtil.withTempDir { parquetoutputLoc => - TrampolineUtil.withTempDir { eventLogDir => - val (eventLog, _) = ToolTestUtils.generateEventLog(eventLogDir, - "ProjectExprsSupported") { spark => - import spark.implicits._ - val df1 = Seq((Array("a", "b", "c"), Array(1, 2, 3)), - (Array("x", "y", "z"), Array(10, 20, 30))).toDF("keys", "values") - // write df1 to parquet to transform LocalTableScan to ProjectExec - df1.write.parquet(s"$parquetoutputLoc/testtext") - val df2 = spark.read.parquet(s"$parquetoutputLoc/testtext") - // map_from_arrays should be part of ProjectExec - df2.select(map_from_arrays(df2("keys"), df2("values")).as("map")) - } - val pluginTypeChecker = new PluginTypeChecker() - val app = createAppFromEventlog(eventLog) - assert(app.sqlPlans.size == 2) - val parsedPlans = app.sqlPlans.map { case (sqlID, plan) => - SQLPlanParser.parseSQLPlan(app.appId, plan, sqlID, "", pluginTypeChecker, app) - } - verifyExecToStageMapping(parsedPlans.toSeq, app) - val allExecInfo = getAllExecsFromPlan(parsedPlans.toSeq) - val wholeStages = allExecInfo.filter(_.exec.contains("WholeStageCodegen")) - assert(wholeStages.size == 1) - assert(wholeStages.forall(_.duration.nonEmpty)) - val allChildren = wholeStages.flatMap(_.children).flatten - val projects = allChildren.filter(_.exec == "Project") - assertSizeAndSupported(1, projects) - } - } - } - test("Parsing Conditional Expressions") { // scalastyle:off line.size.limit val expressionsMap: mutable.HashMap[String, Map[String, Int]] = mutable.HashMap( From 73bd1378bbaaf8f5ac428752cbeaa2f23392137d Mon Sep 17 00:00:00 2001 From: Ahmed Hussein <50450311+amahussein@users.noreply.github.com> Date: Thu, 23 Jan 2025 14:56:08 -0600 Subject: [PATCH 49/52] Update core tools rules to allow cross-build between 2.12 and 2.13 (#1510) Signed-off-by: Ahmed Hussein (amahussein) Fixes #1507 Improve source code to allow cross-build between scala-2.12 and scala2.13 --- core/scalastyle-config.xml | 36 +++++++++++++++++++ .../rapids/tool/profiling/ProfileMain.scala | 4 +-- .../qualification/QualificationMain.scala | 4 +-- .../spark/rapids/tool/tuning/AutoTuner.scala | 23 ++++++------ .../rapids/tool/views/DataSourceView.scala | 4 +-- .../tool/profiling/GenerateDotSuite.scala | 10 +++--- 6 files changed, 57 insertions(+), 24 deletions(-) diff --git a/core/scalastyle-config.xml b/core/scalastyle-config.xml index d8f4fd13d..ed99aadd1 100644 --- a/core/scalastyle-config.xml +++ b/core/scalastyle-config.xml @@ -161,6 +161,42 @@ You can also disable only one rule, by specifying its rule id, as specified in: Use UTF8Source.from instead of Source.from + + + + + + + + + + + + def this\((.*)\) \{ + false + + procedure syntax is deprecated for constructors in Scala 2.13: add `=`, as in method definition + + + + procedure syntax is deprecated in Scala 2.13: add return type `: Unit` and `=` + + + + + ArrayBuilder.make\[(.+)\]\(\) + false + + ArrayBuilder.make does not accept parens anymore in Scala 2.13 + + diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileMain.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileMain.scala index 91c2de01c..6deefbdfe 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileMain.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileMain.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -29,7 +29,7 @@ object ProfileMain extends Logging { /** * Entry point from spark-submit running this as the driver. */ - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val (exitCode, _) = mainInternal(new ProfileArgs(args), enablePB = true) if (exitCode != 0) { System.exit(exitCode) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala index 04406ed0d..adfb07b4a 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/qualification/QualificationMain.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -32,7 +32,7 @@ import org.apache.spark.sql.rapids.tool.util.RapidsToolsConfUtil */ object QualificationMain extends Logging { - def main(args: Array[String]) { + def main(args: Array[String]): Unit = { val (exitCode, _) = mainInternal(new QualificationArgs(args), printStdout = true, enablePB = true) if (exitCode != 0) { diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index be4824225..a2d3fadd4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -20,9 +20,8 @@ import java.io.{BufferedReader, InputStreamReader, IOException} import java.util import scala.beans.BeanProperty -import scala.collection.{mutable, Seq} import scala.collection.JavaConverters.mapAsScalaMapConverter -import scala.collection.mutable.ListBuffer +import scala.collection.mutable import scala.util.control.NonFatal import scala.util.matching.Regex @@ -47,7 +46,7 @@ class GpuWorkerProps( @BeanProperty var memory: String, @BeanProperty var count: Int, @BeanProperty var name: String) { - def this() { + def this() = { this("0m", 0, "None") } def isMissingInfo: Boolean = { @@ -107,7 +106,7 @@ class GpuWorkerProps( */ def setMissingFields(platform: Platform, autoTunerConfigsProvider: AutoTunerConfigsProvider): Seq[String] = { - val res = new ListBuffer[String]() + val res = new mutable.ListBuffer[String]() if (setDefaultGpuCountIfMissing(autoTunerConfigsProvider)) { res += s"GPU count is missing. Setting default to $getCount." } @@ -133,7 +132,7 @@ class SystemClusterProps( @BeanProperty var numCores: Int, @BeanProperty var memory: String, @BeanProperty var numWorkers: Int) { - def this() { + def this() = { this(0, "0m", 0) } def isMissingInfo: Boolean = { @@ -158,7 +157,7 @@ class SystemClusterProps( * used to initialize the field. */ def setMissingFields(autoTunerConfigsProvider: AutoTunerConfigsProvider): Seq[String] = { - val res = new ListBuffer[String]() + val res = new mutable.ListBuffer[String]() if (setDefaultNumWorkersIfMissing(autoTunerConfigsProvider)) { res += s"Number of workers is missing. Setting default to $getNumWorkers." } @@ -185,7 +184,7 @@ class ClusterProperties( @BeanProperty var gpu: GpuWorkerProps, @BeanProperty var softwareProperties: util.LinkedHashMap[String, String]) { - def this() { + def this() = { this(new SystemClusterProps(), new GpuWorkerProps(), new util.LinkedHashMap[String, String]()) } def isEmpty: Boolean = { @@ -337,7 +336,7 @@ class AutoTuner( val autoTunerConfigsProvider: AutoTunerConfigsProvider) extends Logging { - var comments = new ListBuffer[String]() + var comments = new mutable.ListBuffer[String]() var recommendations: mutable.LinkedHashMap[String, RecommendationEntry] = mutable.LinkedHashMap[String, RecommendationEntry]() // list of recommendations to be skipped for recommendations @@ -984,11 +983,11 @@ class AutoTuner( /** * Recommendation for 'spark.rapids.file.cache' based on read characteristics of job. */ - private def recommendFileCache() { + private def recommendFileCache(): Unit = { if (appInfoProvider.getDistinctLocationPct < - autoTunerConfigsProvider.DEF_DISTINCT_READ_THRESHOLD && - appInfoProvider.getRedundantReadSize > - autoTunerConfigsProvider.DEF_READ_SIZE_THRESHOLD) { + autoTunerConfigsProvider.DEF_DISTINCT_READ_THRESHOLD && + appInfoProvider.getRedundantReadSize > + autoTunerConfigsProvider.DEF_READ_SIZE_THRESHOLD) { appendRecommendation("spark.rapids.filecache.enabled", "true") appendComment("Enable file cache only if Spark local disks bandwidth is > 1 GB/s" + " and you have sufficient disk space available to fit both cache and normal Spark" + diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/DataSourceView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/DataSourceView.scala index ff9a0699c..61fb78824 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/DataSourceView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/DataSourceView.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,8 +16,6 @@ package com.nvidia.spark.rapids.tool.views -import scala.collection.Seq - import com.nvidia.spark.rapids.tool.analysis.{ProfAppIndexMapperTrait, QualAppIndexMapperTrait} import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper import com.nvidia.spark.rapids.tool.profiling.{DataSourceProfileResult, SQLAccumProfileResults} diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDotSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDotSuite.scala index 265153d9a..1c14c67a3 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDotSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/GenerateDotSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -125,8 +125,8 @@ class GenerateDotSuite extends FunSuite with BeforeAndAfterAll with Logging { test("Long physical plan") { val random = new SecureRandom() - val seed = System.currentTimeMillis(); - random.setSeed(seed); + val seed = System.currentTimeMillis() + random.setSeed(seed) info("Seeding test with: " + seed) val numTests = 100 val lineLengthRange = 50 until 200 @@ -140,7 +140,7 @@ class GenerateDotSuite extends FunSuite with BeforeAndAfterAll with Logging { SparkPlanGraph.htmlLineBreak.length() val sign = if (random.nextBoolean()) 1 else -1 - val planLength = 16 * 1024 + sign * lineLength * (1 + random.nextInt(5)); + val planLength = 16 * 1024 + sign * lineLength * (1 + random.nextInt(5)) val initPlanStr = (0 to planLength / lineLength).map(_ => "a" * lineLength).mkString("\n") // throw some html characters in there to make sure escaped @@ -169,7 +169,7 @@ class GenerateDotSuite extends FunSuite with BeforeAndAfterAll with Logging { info(s"Plan label summary: min=${planLengthSeq.min} max=${planLengthSeq.max}") } - private def planLabelChecks(planLabel: String) { + private def planLabelChecks(planLabel: String): Unit = { assert(planLabel.startsWith("<>")) assert(planLabel.contains("local-12345-1")) From edb306ad5ec051f34a283e749fe78304dee753c0 Mon Sep 17 00:00:00 2001 From: Partho Sarthi Date: Mon, 27 Jan 2025 07:53:06 -0800 Subject: [PATCH 50/52] Fix string comparison for memory overhead in pinned pool size recommendation in AutoTuner (#1508) --- .../spark/rapids/tool/tuning/AutoTuner.scala | 118 +++++---- .../tool/tuning/ProfilingAutoTunerSuite.scala | 245 ++++++++++++++---- 2 files changed, 259 insertions(+), 104 deletions(-) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala index a2d3fadd4..76232e38c 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/tuning/AutoTuner.scala @@ -262,6 +262,27 @@ class RecommendationEntry(val name: String, } } +/** + * Represents different Spark master types. + */ +sealed trait SparkMaster +case object Local extends SparkMaster +case object Yarn extends SparkMaster +case object Kubernetes extends SparkMaster +case object Standalone extends SparkMaster + +object SparkMaster { + def apply(master: Option[String]): Option[SparkMaster] = { + master.flatMap { + case url if url.contains("yarn") => Some(Yarn) + case url if url.contains("k8s") => Some(Kubernetes) + case url if url.contains("local") => Some(Local) + case url if url.contains("spark://") => Some(Standalone) + case _ => None + } + } +} + /** * AutoTuner module that uses event logs and worker's system properties to recommend Spark * RAPIDS configuration based on heuristics. @@ -347,6 +368,10 @@ class AutoTuner( // When enabled, the profiler recommendations should only include updated settings. private var filterByUpdatedPropertiesEnabled: Boolean = true + private lazy val sparkMaster: Option[SparkMaster] = { + SparkMaster(appInfoProvider.getProperty("spark.master")) + } + private def isCalculationEnabled(prop: String) : Boolean = { !limitedLogicRecommendations.contains(prop) } @@ -555,36 +580,6 @@ class AutoTuner( appendComment(msg) } - /** - * Find the label of the memory.overhead based on the spark master configuration and the spark - * version. - * @return "spark.executor.memoryOverhead", "spark.kubernetes.memoryOverheadFactor", - * or "spark.executor.memoryOverheadFactor". - */ - def memoryOverheadLabel: String = { - val sparkMasterConf = getPropertyValue("spark.master") - val defaultLabel = "spark.executor.memoryOverhead" - sparkMasterConf match { - case None => defaultLabel - case Some(sparkMaster) => - if (sparkMaster.contains("yarn")) { - defaultLabel - } else if (sparkMaster.contains("k8s")) { - appInfoProvider.getSparkVersion match { - case Some(version) => - if (ToolUtils.isSpark330OrLater(version)) { - "spark.executor.memoryOverheadFactor" - } else { - "spark.kubernetes.memoryOverheadFactor" - } - case None => defaultLabel - } - } else { - defaultLabel - } - } - } - /** * Flow: * if "spark.master" is standalone => Do Nothing @@ -594,18 +589,17 @@ class AutoTuner( * if version > 3.3.0 recommend "spark.executor.memoryOverheadFactor" and add comment * else recommend "spark.kubernetes.memoryOverheadFactor" and add comment if missing */ - def addRecommendationForMemoryOverhead(recomValue: String): Unit = { - if (autoTunerConfigsProvider - .enableMemoryOverheadRecommendation(getPropertyValue("spark.master"))) { - val memOverheadLookup = memoryOverheadLabel + private def addRecommendationForMemoryOverhead(recomValue: String): Unit = { + if (!sparkMaster.contains(Standalone)) { + val memOverheadLookup = autoTunerConfigsProvider.getMemoryOverheadLabel(sparkMaster, + appInfoProvider.getSparkVersion) + val pinnedPoolSizeLookup = "spark.rapids.memory.pinnedPool.size" appendRecommendationForMemoryMB(memOverheadLookup, recomValue) - getPropertyValue("spark.rapids.memory.pinnedPool.size").foreach { lookup => - if (lookup != "spark.executor.memoryOverhead") { - if (getPropertyValue(memOverheadLookup).isEmpty) { - appendComment(s"'$memOverheadLookup' must be set if using " + - s"'spark.rapids.memory.pinnedPool.size") - } - } + // if using k8s and pinned pool size is set, add a comment if memory overhead is missing + if (sparkMaster.contains(Kubernetes) && + getPropertyValue(pinnedPoolSizeLookup).isDefined && + getPropertyValue(memOverheadLookup).isEmpty) { + appendComment(s"'$memOverheadLookup' must be set if using '$pinnedPoolSizeLookup'.") } } } @@ -684,7 +678,7 @@ class AutoTuner( val (pinnedMemory, memoryOverhead, finalExecutorHeap, setMaxBytesInFlight) = calcOverallMemory(executorHeapExpr, execCores, availableMemPerExecExpr) appendRecommendationForMemoryMB("spark.rapids.memory.pinnedPool.size", s"$pinnedMemory") - addRecommendationForMemoryOverhead(s"$memoryOverhead") + addRecommendationForMemoryOverhead(memoryOverhead.toString) appendRecommendationForMemoryMB("spark.executor.memory", s"$finalExecutorHeap") setMaxBytesInFlight } else { @@ -1439,20 +1433,6 @@ trait AutoTunerConfigsProvider extends Logging { } } - /** - * Given the spark property "spark.master", it checks whether memoryOverhead should be - * enabled/disabled. For Spark Standalone Mode, memoryOverhead property is skipped. - * @param confValue the value of property "spark.master" - * @return False if the value is a spark standalone. True if the value is not defined or - * set for yarn/Mesos - */ - def enableMemoryOverheadRecommendation(confValue: Option[String]): Boolean = { - confValue match { - case Some(sparkMaster) if sparkMaster.startsWith("spark:") => false - case _ => true - } - } - def buildShuffleManagerClassName(smVersion: String): String = { s"com.nvidia.spark.rapids.spark$smVersion.RapidsShuffleManager" } @@ -1473,6 +1453,32 @@ trait AutoTunerConfigsProvider extends Logging { def shuffleManagerCommentForMissingVersion: String = { "Could not recommend RapidsShuffleManager as Spark version cannot be determined." } + + + /** + * Find the label of the memory overhead based on the spark master configuration and the spark + * version. + * @return "spark.executor.memoryOverhead", "spark.kubernetes.memoryOverheadFactor", + * or "spark.executor.memoryOverheadFactor". + */ + def getMemoryOverheadLabel( + sparkMaster: Option[SparkMaster], + sparkVersion: Option[String]) : String = { + val defaultLabel = "spark.executor.memoryOverhead" + sparkMaster match { + case Some(Kubernetes) => + sparkVersion match { + case Some(version) => + if (ToolUtils.isSpark330OrLater(version)) { + "spark.executor.memoryOverheadFactor" + } else { + "spark.kubernetes.memoryOverheadFactor" + } + case None => defaultLabel + } + case _ => defaultLabel + } + } } /** diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala index 5c08f20f4..b67a873e8 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/tuning/ProfilingAutoTunerSuite.scala @@ -129,7 +129,7 @@ class ProfilingAutoTunerSuite extends BaseAutoTunerSuite { | If the Spark RAPIDS jar is being bundled with your Spark | distribution, this step is not needed. |""".stripMargin - assert(autoTunerOutput == expectedResults) + compareOutput(expectedResults, autoTunerOutput) } test("Load non-existing cluster properties") { @@ -977,8 +977,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1002,7 +1001,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.files.maxPartitionBytes=4096m | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -1056,8 +1054,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1081,7 +1078,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.files.maxPartitionBytes=4096m | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -1126,8 +1122,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, Some(testSparkVersion)) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1152,7 +1147,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.files.maxPartitionBytes=4096m | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. @@ -1205,8 +1199,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1232,7 +1225,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.files.maxPartitionBytes=4096m | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. @@ -1314,7 +1306,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -1327,7 +1318,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} |""".stripMargin // scalastyle:on line.size.limit - assert(autoTunerOutput == expectedResults) + compareOutput(expectedResults, autoTunerOutput) } // When GCFraction is higher AutoTuner.MAX_JVM_GCTIME_FRACTION, the output should contain @@ -1369,8 +1360,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1394,7 +1384,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -1444,8 +1433,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1469,7 +1457,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -1669,8 +1656,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1695,7 +1681,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.filecache.enabled' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. @@ -1746,8 +1731,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1771,7 +1755,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -1844,8 +1827,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -1883,6 +1865,184 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. compareOutput(expectedResults, autoTunerOutput) } + // Map of `Spark Version` -> `memoryOverheadLabel` + private val MISSING_MEMORY_OVERHEAD_k8s_TEST_CASES = Seq( + "3.5.0" -> "spark.executor.memoryOverheadFactor", + "3.2.1" -> "spark.kubernetes.memoryOverheadFactor" + ) + + // This UT sets a custom spark-property "spark.master" pointing to a spark + // k8s value. The Autotuner should detect that the spark-master is k8s and + // comments on the missing memoryOverhead value since pinned pool is set. + MISSING_MEMORY_OVERHEAD_k8s_TEST_CASES.foreach { case (sparkVersion, memoryOverheadLabel) => + test(s"missing memoryOverhead comment is included for k8s with pinned pool " + + s"[sparkVersion=$sparkVersion]") { + val logEventsProps: mutable.Map[String, String] = + mutable.LinkedHashMap[String, String]( + "spark.master" -> "k8s://https://my-cluster-endpoint.example.com:6443", + "spark.executor.cores" -> "16", + "spark.executor.instances" -> "1", + "spark.executor.memory" -> "80g", + "spark.executor.resource.gpu.amount" -> "1", + "spark.executor.instances" -> "1", + "spark.sql.shuffle.partitions" -> "200", + "spark.sql.files.maxPartitionBytes" -> "1g", + "spark.task.resource.gpu.amount" -> "0.0625", + "spark.rapids.memory.pinnedPool.size" -> "5g", + "spark.rapids.sql.enabled" -> "true", + "spark.plugins" -> "com.nvidia.spark.SQLPlugin", + "spark.rapids.sql.concurrentGpuTasks" -> "4") + val dataprocWorkerInfo = buildGpuWorkerInfoAsString() + val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, + Some(sparkVersion)) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) + val platform = PlatformFactory.createInstance(PlatformNames.ONPREM, clusterPropsOpt) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) + val (_, comments) = autoTuner.getRecommendedProperties() + val expectedComment = + s"'$memoryOverheadLabel' must be set if using 'spark.rapids.memory.pinnedPool.size'." + assert(comments.exists(_.comment == expectedComment)) + } + } + + // This UT sets a custom spark-property "spark.master" pointing to a spark + // k8s value. The Autotuner should detect that the spark-master is k8s and + // should not comment on the missing memoryOverhead value since pinned pool is not set. + test(s"missing memoryOverhead comment is not included for k8s without pinned pool " + + s"[sparkVersion=$testSparkVersion]") { + val sparkMaster = "k8s://https://my-cluster-endpoint.example.com:6443" + val logEventsProps: mutable.Map[String, String] = + mutable.LinkedHashMap[String, String]( + "spark.master" -> sparkMaster, + "spark.executor.cores" -> "16", + "spark.executor.instances" -> "1", + "spark.executor.memory" -> "80g", + "spark.executor.resource.gpu.amount" -> "1", + "spark.executor.instances" -> "1", + "spark.sql.shuffle.partitions" -> "200", + "spark.sql.files.maxPartitionBytes" -> "1g", + "spark.task.resource.gpu.amount" -> "0.0625", + "spark.rapids.sql.enabled" -> "true", + "spark.plugins" -> "com.nvidia.spark.SQLPlugin", + "spark.rapids.sql.concurrentGpuTasks" -> "4") + val dataprocWorkerInfo = buildGpuWorkerInfoAsString() + val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, + Some(testSparkVersion)) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) + val platform = PlatformFactory.createInstance(PlatformNames.ONPREM, clusterPropsOpt) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + val memoryOverheadLabel = ProfilingAutoTunerConfigsProvider.getMemoryOverheadLabel( + SparkMaster(Some(sparkMaster)), Some(testSparkVersion)) + // scalastyle:off line.size.limit + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.executor.instances=8 + |--conf spark.executor.memory=32768m + |--conf $memoryOverheadLabel=13516m + |--conf spark.rapids.memory.pinnedPool.size=4096m + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=24 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=24 + |--conf spark.rapids.sql.batchSizeBytes=2147483647 + |--conf spark.rapids.sql.concurrentGpuTasks=2 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=32 + |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager + |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m + |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m + |--conf spark.sql.files.maxPartitionBytes=4096m + | + |Comments: + |- '$memoryOverheadLabel' was not set. + |- 'spark.rapids.memory.pinnedPool.size' was not set. + |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. + |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. + |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.shuffle.manager' was not set. + |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. + |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. + |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} + |""".stripMargin + // scalastyle:on line.size.limit + compareOutput(expectedResults, autoTunerOutput) + } + + // This UT sets a custom spark-property "spark.master" pointing to a yarn + // value. The Autotuner should detect that the spark-master is yarn and + // should not comment on the missing memoryOverhead value even though pinned + // pool is set. + test("missing memoryOverhead comment is not included for yarn " + + s"[sparkVersion=$testSparkVersion]") { + val logEventsProps: mutable.Map[String, String] = + mutable.LinkedHashMap[String, String]( + "spark.master" -> "yarn", + "spark.executor.cores" -> "16", + "spark.executor.instances" -> "1", + "spark.executor.memory" -> "80g", + "spark.executor.resource.gpu.amount" -> "1", + "spark.executor.instances" -> "1", + "spark.sql.shuffle.partitions" -> "200", + "spark.sql.files.maxPartitionBytes" -> "1g", + "spark.task.resource.gpu.amount" -> "0.0625", + "spark.rapids.memory.pinnedPool.size" -> "5g", + "spark.rapids.sql.enabled" -> "true", + "spark.plugins" -> "com.nvidia.spark.SQLPlugin", + "spark.rapids.sql.concurrentGpuTasks" -> "4") + val dataprocWorkerInfo = buildGpuWorkerInfoAsString() + val infoProvider = getMockInfoProvider(8126464.0, Seq(0), Seq(0.004), logEventsProps, + Some(testSparkVersion)) + val clusterPropsOpt = ProfilingAutoTunerConfigsProvider + .loadClusterPropertiesFromContent(dataprocWorkerInfo) + val platform = PlatformFactory.createInstance(PlatformNames.ONPREM, clusterPropsOpt) + val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) + val (properties, comments) = autoTuner.getRecommendedProperties() + val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) + // scalastyle:off line.size.limit + val expectedResults = + s"""| + |Spark Properties: + |--conf spark.executor.instances=8 + |--conf spark.executor.memory=32768m + |--conf spark.executor.memoryOverhead=13516m + |--conf spark.rapids.memory.pinnedPool.size=4096m + |--conf spark.rapids.shuffle.multiThreaded.reader.threads=24 + |--conf spark.rapids.shuffle.multiThreaded.writer.threads=24 + |--conf spark.rapids.sql.batchSizeBytes=2147483647 + |--conf spark.rapids.sql.concurrentGpuTasks=2 + |--conf spark.rapids.sql.multiThreadedRead.numThreads=32 + |--conf spark.shuffle.manager=com.nvidia.spark.rapids.spark$testSmVersion.RapidsShuffleManager + |--conf spark.sql.adaptive.advisoryPartitionSizeInBytes=128m + |--conf spark.sql.adaptive.coalescePartitions.minPartitionSize=4m + |--conf spark.sql.files.maxPartitionBytes=4096m + | + |Comments: + |- 'spark.executor.memoryOverhead' was not set. + |- 'spark.rapids.shuffle.multiThreaded.reader.threads' was not set. + |- 'spark.rapids.shuffle.multiThreaded.writer.threads' was not set. + |- 'spark.rapids.sql.batchSizeBytes' was not set. + |- 'spark.rapids.sql.multiThreadedRead.numThreads' was not set. + |- 'spark.shuffle.manager' was not set. + |- 'spark.sql.adaptive.advisoryPartitionSizeInBytes' was not set. + |- 'spark.sql.adaptive.autoBroadcastJoinThreshold' was not set. + |- 'spark.sql.adaptive.coalescePartitions.minPartitionSize' was not set. + |- 'spark.sql.adaptive.enabled' should be enabled for better performance. + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.jars.missing")} + |- ${ProfilingAutoTunerConfigsProvider.classPathComments("rapids.shuffle.jars")} + |""".stripMargin + // scalastyle:on line.size.limit + compareOutput(expectedResults, autoTunerOutput) + } + test("Recommendations generated for unsupported operators from driver logs only") { val customProps = mutable.LinkedHashMap( "spark.executor.cores" -> "8", @@ -2053,8 +2213,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2130,8 +2289,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) assert(expectedLines.forall(line => autoTunerOutput.contains(line)), @@ -2198,8 +2356,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2224,7 +2381,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.task.resource.gpu.amount=0.0625 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -2408,8 +2564,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2433,7 +2588,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.shuffle.partitions=400 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -2486,8 +2640,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2511,7 +2664,6 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. |--conf spark.sql.shuffle.partitions=200 | |Comments: - |- 'spark.executor.memoryOverhead' must be set if using 'spark.rapids.memory.pinnedPool.size |- 'spark.executor.memoryOverhead' was not set. |- 'spark.rapids.shuffle.multiThreaded.maxBytesInFlight' was not set. |- 'spark.rapids.sql.batchSizeBytes' was not set. @@ -2552,8 +2704,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2628,8 +2779,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit @@ -2703,8 +2853,7 @@ We recommend using nodes/workers with more memory. Need at least 17496MB memory. .loadClusterPropertiesFromContent(dataprocWorkerInfo) val platform = PlatformFactory.createInstance(PlatformNames.DATAPROC, clusterPropsOpt) val autoTuner: AutoTuner = ProfilingAutoTunerConfigsProvider - .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, - platform) + .buildAutoTunerFromProps(dataprocWorkerInfo, infoProvider, platform) val (properties, comments) = autoTuner.getRecommendedProperties() val autoTunerOutput = Profiler.getAutoTunerResultsAsString(properties, comments) // scalastyle:off line.size.limit From d15a9995e0924838617469efcff754bf283fcd74 Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Mon, 27 Jan 2025 13:52:10 -0800 Subject: [PATCH 51/52] [BUG] Qual tool should convert time units at stage/job/sql level (#1511) Fixes https://github.com/NVIDIA/spark-rapids-tools/issues/1481 ### Changes - Change qual tool to convert time units (ms to ns) when aggregating at stage/job/sql level instead of at task level - Update unit tests results due to the above change ### Tests - Existing unit tests --------- Signed-off-by: cindyyuanjiang --- .../qualification/QualificationAppInfo.scala | 18 ++++++++++++------ .../QualificationEventProcessor.scala | 12 +++++------- .../complex_dec_expectation.csv | 2 +- .../db_sim_test_expectation.csv | 2 +- .../directory_test_expectation.csv | 2 +- .../jdbc_expectation.csv | 2 +- .../multi_run_freq_test_expectation.csv | 2 +- .../nds_q86_test_expectation.csv | 2 +- .../nested_dsv2_expectation.csv | 2 +- .../nested_type_expectation.csv | 2 +- .../qual_test_missing_sql_end_expectation.csv | 2 +- .../qual_test_simple_expectation.csv | 4 ++-- .../read_dsv1_expectation.csv | 2 +- .../read_dsv2_expectation.csv | 2 +- .../spark2_expectation.csv | 2 +- .../truncated_1_end_expectation.csv | 2 +- .../write_format_expectation.csv | 2 +- .../qualification/QualificationSuite.scala | 6 +++--- 18 files changed, 36 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala index b1842ec23..5d0f3074c 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationAppInfo.scala @@ -16,6 +16,8 @@ package org.apache.spark.sql.rapids.tool.qualification +import java.util.concurrent.TimeUnit.NANOSECONDS + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.mutable @@ -168,8 +170,9 @@ class QualificationAppInfo( } private def calculateCpuTimePercent(perSqlStageSummary: Seq[SQLStageSummary]): Double = { - val totalCpuTime = perSqlStageSummary.map(_.execCPUTime).sum - val totalRunTime = perSqlStageSummary.map(_.execRunTime).sum + val totalCpuTime = + NANOSECONDS.toMillis(perSqlStageSummary.map(_.execCPUTime).sum) // in milliseconds + val totalRunTime = perSqlStageSummary.map(_.execRunTime).sum // in milliseconds ToolUtils.calculateDurationPercent(totalCpuTime, totalRunTime) } @@ -456,8 +459,10 @@ class QualificationAppInfo( val ratio = numSupportedExecs / numExecs val estimateWallclockSupported = (sqlWallClockDuration * ratio).toInt // don't worry about supported execs for these are these are mostly indicator of I/O - val execRunTime = sqlIDToTaskEndSum.get(sqlID).map(_.executorRunTime).getOrElse(0L) - val execCPUTime = sqlIDToTaskEndSum.get(sqlID).map(_.executorCPUTime).getOrElse(0L) + val execRunTime = + sqlIDToTaskEndSum.get(sqlID).map(_.executorRunTime).getOrElse(0L) // in milliseconds + val execCPUTime = + sqlIDToTaskEndSum.get(sqlID).map(_.executorCPUTime).getOrElse(0L) // in nanoseconds SQLStageSummary(stageSum, sqlID, estimateWallclockSupported, execCPUTime, execRunTime) } @@ -906,8 +911,9 @@ case class SQLStageSummary( stageSum: Set[StageQualSummaryInfo], sqlID: Long, estimateWallClockSupported: Long, - execCPUTime: Long, - execRunTime: Long) + execCPUTime: Long, // in nanoseconds + execRunTime: Long // in milliseconds +) case class MLFunctions( appID: Option[String], diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala index 84bc56efb..d72eacce6 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/qualification/QualificationEventProcessor.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,8 +16,6 @@ package org.apache.spark.sql.rapids.tool.qualification -import java.util.concurrent.TimeUnit.NANOSECONDS - import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -42,8 +40,8 @@ class QualificationEventProcessor(app: QualificationAppInfo, perSqlOnly: Boolean val taskSum = app.stageIdToTaskEndSum.getOrElseUpdate(event.stageId, { new StageTaskQualificationSummary(event.stageId, event.stageAttemptId, 0, 0, 0, 0) }) - taskSum.executorRunTime += event.taskMetrics.executorRunTime - taskSum.executorCPUTime += NANOSECONDS.toMillis(event.taskMetrics.executorCpuTime) + taskSum.executorRunTime += event.taskMetrics.executorRunTime // in milliseconds + taskSum.executorCPUTime += event.taskMetrics.executorCpuTime // in nanoseconds taskSum.totalTaskDuration += event.taskInfo.duration // Add the total bytes read from the task if it's available. This is from inputMetrics if // it is reading from datasource, or shuffleReadMetrics if it is reading from shuffle. @@ -61,8 +59,8 @@ class QualificationEventProcessor(app: QualificationAppInfo, perSqlOnly: Boolean val taskSum = app.sqlIDToTaskEndSum.getOrElseUpdate(sqlID, { new StageTaskQualificationSummary(event.stageId, event.stageAttemptId, 0, 0, 0, 0) }) - taskSum.executorRunTime += event.taskMetrics.executorRunTime - taskSum.executorCPUTime += NANOSECONDS.toMillis(event.taskMetrics.executorCpuTime) + taskSum.executorRunTime += event.taskMetrics.executorRunTime // in milliseconds + taskSum.executorCPUTime += event.taskMetrics.executorCpuTime // in nanoseconds taskSum.totalTaskDuration += event.taskInfo.duration } } diff --git a/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv b/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv index 8a9e1f666..b631ba9be 100644 --- a/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/complex_dec_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly), Total Core Seconds -"Spark shell","local-1626104300434",1500,1469,131104,996,88.35,"","","","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map;map>;map>;array>;array","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map>;map>;array>","NESTED COMPLEX TYPE",1260,1388,129598,493,976,false,"CollectLimit","",30,1564 +"Spark shell","local-1626104300434",1500,1469,131104,996,89.7,"","","","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map;map>;map>;array>;array","struct,lastname:string>;struct,previous:struct,city:string>>;array>;map>;map>;array>","NESTED COMPLEX TYPE",1260,1388,129598,493,976,false,"CollectLimit","",30,1564 diff --git a/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv b/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv index 35d5d6f1b..1a22fbfe2 100644 --- a/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/db_sim_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.14,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan unknown;SerializeFromObject","",30,1599 +"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.25,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan unknown;SerializeFromObject","",30,1599 diff --git a/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv b/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv index 35d5d6f1b..1a22fbfe2 100644 --- a/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/directory_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.14,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan unknown;SerializeFromObject","",30,1599 +"Spark shell","local-1623876083964",119353,1417661,133857,92667,91.25,"","","","","","",119903,143821,14504,316964,1100697,false,"Scan unknown;SerializeFromObject","",30,1599 diff --git a/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv b/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv index 50e8986ba..61b194b07 100644 --- a/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/jdbc_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","app-20211019113801-0001",2942,19894,571967,2814,28.41,"","JDBC[*]","","","","",1812,2883,569025,859,19035,false,"CollectLimit;Scan jdbc;Execute CreateViewCommand","",30,9110 +"Spark shell","app-20211019113801-0001",2942,19894,571967,2814,29.76,"","JDBC[*]","","","","",1812,2883,569025,859,19035,false,"CollectLimit;Scan jdbc;Execute CreateViewCommand","",30,9110 diff --git a/core/src/test/resources/QualificationExpectations/multi_run_freq_test_expectation.csv b/core/src/test/resources/QualificationExpectations/multi_run_freq_test_expectation.csv index 91123bb28..260de8cc7 100644 --- a/core/src/test/resources/QualificationExpectations/multi_run_freq_test_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/multi_run_freq_test_expectation.csv @@ -1,3 +1,3 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1629446106683",1910,6475,17698,1910,27.76,"","","","array>;map>","array>;map>","NESTED COMPLEX TYPE",1453,1203,16292,0,6475,false,"","",221851,132 +"Spark shell","local-1629446106683",1910,6475,17698,1910,27.8,"","","","array>;map>","array>;map>","NESTED COMPLEX TYPE",1453,1203,16292,0,6475,false,"","",221851,132 "Spark shell","local-1623263471760",0,0,22937,0,0.0,"","","","","","",0,0,22937,0,0,false,"","",221851,266 diff --git a/core/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv b/core/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv index 1e554e203..ca6f2a220 100644 --- a/core/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/nds_q86_test_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"TPC-DS Like Bench q86","app-20210319163812-1778",9910,4320658,26171,9910,35.34,"","","","","","",9565,9265,3596053,0,4320658,false,"Execute CreateViewCommand","",30,24270 +"TPC-DS Like Bench q86","app-20210319163812-1778",9910,4320658,26171,9910,35.39,"","","","","","",9565,9265,3596053,0,4320658,false,"Execute CreateViewCommand","",30,24270 diff --git a/core/src/test/resources/QualificationExpectations/nested_dsv2_expectation.csv b/core/src/test/resources/QualificationExpectations/nested_dsv2_expectation.csv index 303d9d64a..8aa4689cd 100644 --- a/core/src/test/resources/QualificationExpectations/nested_dsv2_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/nested_dsv2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1630045673160",1363,3757,21200,1363,34.56,"","","","array>;map>;map>;map>;map>;map>","array>;map>","NESTED COMPLEX TYPE",1453,1203,16292,0,6475,false,"","",30,132 +"Spark shell","local-1629446106683",1910,6475,17698,1910,27.8,"","","","array>;map>","array>;map>","NESTED COMPLEX TYPE",1453,1203,16292,0,6475,false,"","",30,132 diff --git a/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv b/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv index 428f89654..a0bbda7e9 100644 --- a/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/qual_test_missing_sql_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622561780883",0,40448,7673,0,55.94,"","","","","","",0,5000,7673,8096,32352,false,"Scan unknown;SerializeFromObject","",30,82 +"Rapids Spark Profiling Tool Unit Tests","local-1622561780883",0,40448,7673,0,56.24,"","","","","","",0,5000,7673,8096,32352,false,"Scan unknown;SerializeFromObject","",30,82 diff --git a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv index 4ba9381e5..534e95ecf 100644 --- a/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/qual_test_simple_expectation.csv @@ -1,5 +1,5 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",11600,132257,16319,9868,37.7,"","","JSON","","","",7143,13770,4719,19744,112513,false,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",1,186 +"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",11600,132257,16319,9868,37.97,"","","JSON","","","",7143,13770,4719,19744,112513,false,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",1,186 "Spark shell","local-1651187225439",224,180,355637,142,87.88,"","","","","","",498,228,355101,66,114,false,"SerializeFromObject;CollectLimit;DeserializeToObject;Filter;MapElements","",1,2834 "Spark shell","local-1651188809790",347,283,166215,128,81.18,"","","","","","UDF",715,318,165572,178,105,false,"CollectLimit;Project","UDF",1,1318 -"Rapids Spark Profiling Tool Unit Tests","local-1623281204390",1156,4666,6240,122,46.27,"","","JSON","","","UDF",1209,1130,5809,4170,496,false,"Execute InsertIntoHadoopFsRelationCommand json;LocalTableScan;Execute CreateViewCommand;Project","UDF",1,64 +"Rapids Spark Profiling Tool Unit Tests","local-1623281204390",1156,4666,6240,122,47.48,"","","JSON","","","UDF",1209,1130,5809,4170,496,false,"Execute InsertIntoHadoopFsRelationCommand json;LocalTableScan;Execute CreateViewCommand;Project","UDF",1,64 diff --git a/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv b/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv index 73af0bcca..b2c5055de 100644 --- a/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/read_dsv1_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1624371544219",4575,20421,175293,4365,72.15,"","Text[*]","JSON","","","",1859,5372,176916,938,19483,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json","",30,2096 +"Spark shell","local-1624371544219",4575,20421,175293,4365,72.2,"","Text[*]","JSON","","","",1859,5372,176916,938,19483,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json","",30,2096 diff --git a/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv b/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv index 1af748da3..640ee7fa1 100644 --- a/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/read_dsv2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1624371906627",4917,21802,83738,4762,71.3,"","Text[*]","JSON","","","",1984,5438,83336,689,21113,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json","",30,997 +"Spark shell","local-1624371906627",4917,21802,83738,4762,71.34,"","Text[*]","JSON","","","",1984,5438,83336,689,21113,false,"CollectLimit;Scan text;Execute InsertIntoHadoopFsRelationCommand json","",30,997 diff --git a/core/src/test/resources/QualificationExpectations/spark2_expectation.csv b/core/src/test/resources/QualificationExpectations/spark2_expectation.csv index 962093b72..44d12c49a 100644 --- a/core/src/test/resources/QualificationExpectations/spark2_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/spark2_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1634253215009",523,359,47063,281,67.64,"","Text[*]","","","","",1068,385,46540,166,193,false,"CollectLimit;Scan text","",30,369 +"Spark shell","local-1634253215009",523,359,47063,281,68.73,"","Text[*]","","","","",1068,385,46540,166,193,false,"CollectLimit;Scan text","",30,369 diff --git a/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv b/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv index 79cb8dede..5d393591e 100644 --- a/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/truncated_1_end_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",395,14353,4872,164,62.67,"","","JSON","","","",1306,794,4477,8376,5977,true,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",30,49 +"Rapids Spark Profiling Tool Unit Tests","local-1622043423018",395,14353,4872,164,62.79,"","","JSON","","","",1306,794,4477,8376,5977,true,"SerializeFromObject;Scan unknown;Execute InsertIntoHadoopFsRelationCommand json;DeserializeToObject;Filter;MapElements","",30,49 diff --git a/core/src/test/resources/QualificationExpectations/write_format_expectation.csv b/core/src/test/resources/QualificationExpectations/write_format_expectation.csv index 50ad4f176..740bfc719 100644 --- a/core/src/test/resources/QualificationExpectations/write_format_expectation.csv +++ b/core/src/test/resources/QualificationExpectations/write_format_expectation.csv @@ -1,2 +1,2 @@ App Name,App ID,SQL DF Duration,SQL Dataframe Task Duration,App Duration,GPU Opportunity,Executor CPU Time Percent,SQL Ids with Failures,Unsupported Read File Formats and Types,Unsupported Write Data Format,Complex Types,Nested Complex Types,Potential Problems,Longest SQL Duration,SQL Stage Durations Sum,NONSQL Task Duration Plus Overhead,Unsupported Task Duration,Supported SQL DF Task Duration,App Duration Estimated,Unsupported Execs,Unsupported Expressions,Estimated Job Frequency (monthly),Total Core Seconds -"Spark shell","local-1629442299891",1151,920,19554,788,91.72,"","","CSV;JSON","","","",1235,1049,18251,290,630,false,"Execute InsertIntoHadoopFsRelationCommand csv;Execute InsertIntoHadoopFsRelationCommand json","",30,147 +"Spark shell","local-1629442299891",1151,920,19554,788,91.98,"","","CSV;JSON","","","",1235,1049,18251,290,630,false,"Execute InsertIntoHadoopFsRelationCommand csv;Execute InsertIntoHadoopFsRelationCommand json","",30,147 diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala index 0b493b0cf..e3ca550d5 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/qualification/QualificationSuite.scala @@ -1039,8 +1039,8 @@ class QualificationSuite extends BaseTestSuite { assert(firstRow(4) == "testing, csv delimiter, replacement") // parse results from listener - val executorCpuTime = listener.executorCpuTime - val executorRunTime = listener.completedStages + val executorCpuTime = NANOSECONDS.toMillis(listener.executorCpuTime) // in milliseconds + val executorRunTime = listener.completedStages // in milliseconds .map(_.stageInfo.taskMetrics.executorRunTime).sum val listenerCpuTimePercent = @@ -1832,7 +1832,7 @@ class ToolTestListener extends SparkListener { var executorCpuTime = 0L override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - executorCpuTime += NANOSECONDS.toMillis(taskEnd.taskMetrics.executorCpuTime) + executorCpuTime += taskEnd.taskMetrics.executorCpuTime // in nanoseconds } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { From de411e35723ef35b37233e08195316bd2c31da76 Mon Sep 17 00:00:00 2001 From: Cindy Jiang <47068112+cindyyuanjiang@users.noreply.github.com> Date: Mon, 27 Jan 2025 13:55:10 -0800 Subject: [PATCH 52/52] [FEA] Add IO diagnostic output for GPU slowness in Profiler tool (#1451) Contributes to https://github.com/NVIDIA/spark-rapids-tools/issues/1374 ### Changes - Added an IO diagnostic view in Profiler output: `io_diagnostic_metrics.csv` - Added class `IOAccumDiagnosticMetrics` to store selected IO related metric names and methods - Added class `IODiagnosticResult` to represent each IO diagnostic result - In `core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala`, cache results from `generateSQLAccums` and use them to compute IO diagnostic metrics in function `generateIODiagnosticAccums` - Added `IODiagnostics` in class `DiagnosticSummaryInfo` - Reorganized `AccumProfileResults` and `SQLAccumProfileResults` presentation for better readability ### Testing - Added unit test "test IO diagnostic metrics" in `core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala` ### Example Output ``` appIndex,appName,appId,sqlId,stageId,stageDurationMs,nodeId,nodeName,outputRowsMin,outputRowsMedian,outputRowsMax,outputRowsTotal,scanTimeMin,scanTimeMedian,scanTimeMax,scanTimeTotal,outputBatchesMin,outputBatchesMedian,outputBatchesMax,outputBatchesTotal,bufferTimeMin,bufferTimeMedian,bufferTimeMax,bufferTimeTotal,shuffleWriteTimeMin,shuffleWriteTimeMedian,shuffleWriteTimeMax,shuffleWriteTimeTotal,fetchWaitTimeMin,fetchWaitTimeMedian,fetchWaitTimeMax,fetchWaitTimeTotal,gpuDecodeTimeMin,gpuDecodeTimeMedian,gpuDecodeTimeMax,gpuDecodeTimeTotal 1,Spark shell,local-1622814619968,0,0,1743,16,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,41434653,60830365,100858775,400284505,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,0,1743,21,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,1,1631,8,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,37444140,92128351,108992798,508750471,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,1,1631,13,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 1,Spark shell,local-1622814619968,0,2,688,3,"GpuColumnarExchange",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,139875,230038,9747416,93193331,0,0,0,0,0,0,0,0 ``` ### Follow-up Issue https://github.com/NVIDIA/spark-rapids-tools/issues/1454 --------- Signed-off-by: cindyyuanjiang --- .../rapids/tool/analysis/AnalysisUtils.scala | 36 --- .../tool/analysis/AppSQLPlanAnalyzer.scala | 181 +++++++++-- .../analysis/AppSparkMetricsAnalyzer.scala | 2 +- .../analysis/util/DiagnosticMetrics.scala | 117 +++++++ .../tool/profiling/CollectInformation.scala | 6 +- .../profiling/DiagnosticSummaryInfo.scala | 5 +- .../profiling/ProfileClassWarehouse.scala | 302 ++++++++++++++++-- .../rapids/tool/profiling/Profiler.scala | 9 +- .../spark/rapids/tool/views/SQLView.scala | 23 +- .../sql/rapids/tool/store/AccumNameRef.scala | 6 +- ...entlog_iodiagnosticmetrics_expectation.csv | 16 + ...og_stagediagnosticmetrics_expectation.csv} | 0 .../rapids/tool/profiling/AnalysisSuite.scala | 163 ++++++++-- .../tool/profiling/ApplicationInfoSuite.scala | 15 +- 14 files changed, 758 insertions(+), 123 deletions(-) delete mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala create mode 100644 core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala create mode 100644 core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv rename core/src/test/resources/ProfilingExpectations/{rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv => rapids_join_eventlog_stagediagnosticmetrics_expectation.csv} (100%) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala deleted file mode 100644 index 01f6989c7..000000000 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AnalysisUtils.scala +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright (c) 2024, NVIDIA CORPORATION. - * - * Licensed 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 com.nvidia.spark.rapids.tool.analysis - -object StageAccumDiagnosticMetrics { - val MEMORY_SPILLED_METRIC = "internal.metrics.memoryBytesSpilled" - val DISK_SPILLED_METRIC = "internal.metrics.diskBytesSpilled" - val INPUT_BYTES_READ_METRIC = "internal.metrics.input.bytesRead" - val OUTPUT_BYTES_WRITTEN_METRIC = "internal.metrics.output.bytesWritten" - val SW_TOTAL_BYTES_METRIC = "internal.metrics.shuffle.write.bytesWritten" - val SR_FETCH_WAIT_TIME_METRIC = "internal.metrics.shuffle.read.fetchWaitTime" - val SW_WRITE_TIME_METRIC = "internal.metrics.shuffle.write.writeTime" - val GPU_SEMAPHORE_WAIT_METRIC = "gpuSemaphoreWait" - - /** - * Get all diagnostic metrics - */ - def getAllDiagnosticMetrics: Set[String] = Set(MEMORY_SPILLED_METRIC, - DISK_SPILLED_METRIC, INPUT_BYTES_READ_METRIC, OUTPUT_BYTES_WRITTEN_METRIC, - SW_TOTAL_BYTES_METRIC, SR_FETCH_WAIT_TIME_METRIC, SW_WRITE_TIME_METRIC, - GPU_SEMAPHORE_WAIT_METRIC) -} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala index 7ca4bbb5b..59cbd4a4c 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSQLPlanAnalyzer.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,7 +19,9 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.breakOut import scala.collection.mutable.{AbstractSet, ArrayBuffer, HashMap, LinkedHashSet} -import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} +import com.nvidia.spark.rapids.tool.analysis.util.IOAccumDiagnosticMetrics._ +import com.nvidia.spark.rapids.tool.analysis.util.StageAccumDiagnosticMetrics._ +import com.nvidia.spark.rapids.tool.profiling.{AccumProfileResults, IODiagnosticResult, SQLAccumProfileResults, SQLMetricInfoCase, SQLStageInfoProfileResult, UnsupportedSQLPlan, WholeStageCodeGenResults} import com.nvidia.spark.rapids.tool.qualification.QualSQLPlanAnalyzer import org.apache.spark.sql.execution.SparkPlanInfo @@ -27,7 +29,7 @@ import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SparkPlanGraphCluster, import org.apache.spark.sql.rapids.tool.{AppBase, RDDCheckHelper, SqlPlanInfoGraphBuffer, SqlPlanInfoGraphEntry} import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo import org.apache.spark.sql.rapids.tool.qualification.QualificationAppInfo -import org.apache.spark.sql.rapids.tool.store.DataSourceRecord +import org.apache.spark.sql.rapids.tool.store.{AccumInfo, DataSourceRecord} import org.apache.spark.sql.rapids.tool.util.ToolsPlanGraph /** @@ -58,21 +60,78 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap var allSQLMetrics: ArrayBuffer[SQLMetricInfoCase] = ArrayBuffer[SQLMetricInfoCase]() // A map between stage ID and a set of node names val stageToNodeNames: HashMap[Long, Seq[String]] = HashMap.empty[Long, Seq[String]] - // A map between stage ID and diagnostic metrics results (stored as a map between metric name - // and AccumProfileResults) + + // A mapping from stage ID to diagnostic metrics results. + // Each stage ID maps to another HashMap, where: + // - The key is the diagnostic metric name (String). + // - The value is an AccumProfileResults object containing the diagnostic data for that metric. val stageToDiagnosticMetrics: HashMap[Long, HashMap[String, AccumProfileResults]] = HashMap.empty[Long, HashMap[String, AccumProfileResults]] + // A mapping from a unique combination of SQL execution identifiers to a list of IO diagnostic + // metrics results. + // The key is a tuple consisting of: + // - sqlID (Long): The unique identifier for the SQL query. + // - nodeID (Long): The unique identifier for the node. + // The value is an ArrayBuffer of SQLAccumProfileResults objects, storing the IO diagnostic + // metrics for the given key. + val IODiagnosticMetricsMap: HashMap[(Long, Long), ArrayBuffer[SQLAccumProfileResults]] = + HashMap.empty[(Long, Long), ArrayBuffer[SQLAccumProfileResults]] + /** - * Given an input diagnostic metric result, update stageToDiagnosticMetrics mapping - * @param accum AccumProfileResults to be analyzed + * Updates the stageToDiagnosticMetrics mapping with the provided AccumProfileResults. + * @param accum AccumProfileResults instance containing diagnostic metrics to be added + * to stageToDiagnosticMetrics mapping. */ private def updateStageDiagnosticMetrics(accum: AccumProfileResults): Unit = { - val stageId = accum.stageId - if (!stageToDiagnosticMetrics.contains(stageId)) { - stageToDiagnosticMetrics(stageId) = HashMap.empty[String, AccumProfileResults] + // Initialize an empty mapping for the stage if it doesn't already exist + if (!stageToDiagnosticMetrics.contains(accum.stageId)) { + stageToDiagnosticMetrics(accum.stageId) = HashMap.empty[String, AccumProfileResults] } - stageToDiagnosticMetrics(stageId)(accum.accMetaRef.getName()) = accum + + stageToDiagnosticMetrics(accum.stageId)(accum.accMetaRef.getName()) = accum + } + + /** + * Updates the IODiagnosticMetricsMap with the provided SQLAccumProfileResults. + * @param accum SQLAccumProfileResults instance containing IO diagnostics metrics + * to be added to IODiagnosticMetricsMap. + */ + private def updateIODiagnosticMetricsMap(accum: SQLAccumProfileResults): Unit = { + val key = (accum.sqlID, accum.nodeID) + + // Initialize an entry if the key does not exist + if (!IODiagnosticMetricsMap.contains(key)) { + IODiagnosticMetricsMap(key) = ArrayBuffer[SQLAccumProfileResults]() + } + + IODiagnosticMetricsMap(key) += accum + } + + /** + * Retrieves the task IDs associated with a specific stage. + * + * @param stageId The ID of the stage. + * @return A seq of task IDs corresponding to the given stage ID. + */ + private def getStageTaskIds(stageId: Int): Seq[Long] = { + app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId)(breakOut).distinct + } + + /** + * Retrieves task update values from the accumulator info for the specified stage ID. + * + * @param accumInfo AccumInfo object containing the task updates map. + * @param stageId The stage ID for which task updates need to be retrived. + * @return An array of task update values (`Long`) corresponding to the tasks + * in the specified stage. + */ + private def filterAccumTaskUpdatesForStage(accumInfo: AccumInfo, stageTaskIds: Seq[Long]) + : Array[Long] = { + stageTaskIds.collect { + case taskId if accumInfo.taskUpdatesMap.contains(taskId) => + accumInfo.taskUpdatesMap(taskId) + }(breakOut) } /** @@ -310,15 +369,100 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap val med = Math.max(taskInfo.med, driverInfo.med) val total = Math.max(taskInfo.total, driverInfo.total) - Some(SQLAccumProfileResults(appIndex, metric.sqlID, + val sqlAccumProileResult = SQLAccumProfileResults(appIndex, metric.sqlID, metric.nodeID, metric.nodeName, metric.accumulatorId, metric.name, - min, med, max, total, metric.metricType, metric.stageIds.mkString(","))) + min, med, max, total, metric.metricType, metric.stageIds) + + if (isIODiagnosticMetricName(metric.name)) { + updateIODiagnosticMetricsMap(sqlAccumProileResult) + } + + Some(sqlAccumProileResult) } else { None } }(breakOut) } + /** + * Generates IO-related diagnostic metrics for the SQL plan. Metrics include: + * - Output rows + * - Scan time + * - Output batches + * - Buffer time + * - Shuffle write time + * - Fetch wait time + * - GPU decode time + * + * This method processes accumulator information for each SQL stage and node and + * computes statistical results (min, median, max, sum) for IO-related metrics. + * + * @return A sequence of `IODiagnosticResult` objects one per SQL stage and node. + */ + def generateIODiagnosticAccums(): Seq[IODiagnosticResult] = { + // Transform the diagnostic metrics map into a sequence of results + IODiagnosticMetricsMap.flatMap { case ((sqlId, nodeId), sqlAccums) => + // Process each stage ID and compute diagnostic results + // TODO: currently if stage IDs is empty, the result is skipped + val stageIds = sqlAccums.head.stageIds + stageIds.flatMap { stageId => + val stageTaskIds = getStageTaskIds(stageId) + val nodeName = sqlAccums.head.nodeName + + // Initialize a map to store statistics for each IO metric + val metricNameToStatistics = HashMap.empty[String, StatisticsMetrics]. + withDefaultValue(StatisticsMetrics.ZERO_RECORD) + + // Process each accumulator for the current SQL stage + sqlAccums.foreach { sqlAccum => + // TODO: check if accumulator ID is in driverAccumMap, currently skipped + val accumInfoOpt = app.accumManager.accumInfoMap.get(sqlAccum.accumulatorId) + + val metricStats: Option[StatisticsMetrics] = accumInfoOpt.flatMap { accumInfo => + if (!accumInfo.stageValuesMap.contains(stageId)) { + None + } else if (stageIds.size == 1) { + // Skip computing statistics when there is only one stage + Some(StatisticsMetrics(sqlAccum.min, sqlAccum.median, sqlAccum.max, sqlAccum.total)) + } else { + // Retrieve task updates which correspond to the current stage + val filteredTaskUpdates = filterAccumTaskUpdatesForStage(accumInfo, stageTaskIds) + StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) + } + } + + // Compute the metric's statistics and store the results if available + metricStats.map { stat => + val metricKey = normalizeToIODiagnosticMetricKey(sqlAccum.name) + metricNameToStatistics(metricKey) = stat + } + } + + if (metricNameToStatistics.isEmpty) { + // No IO metric statistics were computed for this stage + None + } else { + Some(IODiagnosticResult( + appIndex, + app.getAppName, + app.appId, + sqlId, + stageId, + app.stageManager.getDurationById(stageId), + nodeId, + nodeName, + metricNameToStatistics(OUTPUT_ROWS_METRIC_KEY), + metricNameToStatistics(SCAN_TIME_METRIC_KEY), + metricNameToStatistics(OUTPUT_BATCHES_METRIC_KEY), + metricNameToStatistics(BUFFER_TIME_METRIC_KEY), + metricNameToStatistics(SHUFFLE_WRITE_TIME_METRIC_KEY), + metricNameToStatistics(FETCH_WAIT_TIME_METRIC_KEY), + metricNameToStatistics(GPU_DECODE_TIME_METRIC_KEY))) + } + } + }(breakOut) + } + /** * Generate the stage level metrics for the SQL plan including GPU metrics if applicable. * Along with Spark defined metrics, below is the list of GPU metrics that are collected if they @@ -333,11 +477,12 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap app.accumManager.accumInfoMap.flatMap { accumMapEntry => val accumInfo = accumMapEntry._2 accumInfo.stageValuesMap.keys.flatMap( stageId => { - val stageTaskIds: Set[Long] = - app.taskManager.getAllTasksStageAttempt(stageId).map(_.taskId)(breakOut) + // Retrieve task updates correspond to the current stage + val filteredTaskUpdates = + filterAccumTaskUpdatesForStage(accumInfo, getStageTaskIds(stageId)) + // Get the task updates that belong to that stage - StatisticsMetrics.createOptionalFromArr( - accumInfo.taskUpdatesMap.filterKeys(stageTaskIds).map(_._2)(breakOut)) match { + StatisticsMetrics.createOptionalFromArr(filteredTaskUpdates) match { case Some(stat) => // Reuse AccumProfileResults to avoid generating allocating new objects val accumProfileResults = AccumProfileResults( @@ -348,7 +493,7 @@ class AppSQLPlanAnalyzer(app: AppBase, appIndex: Int) extends AppAnalysisBase(ap median = stat.med, max = stat.max, total = stat.total) - if (accumInfo.infoRef.name.isDiagnosticMetrics()) { + if (isDiagnosticMetrics(accumInfo.infoRef.name.value)) { updateStageDiagnosticMetrics(accumProfileResults) } Some(accumProfileResults) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala index 1804f1946..eb6b6a7f0 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/AppSparkMetricsAnalyzer.scala @@ -19,8 +19,8 @@ package com.nvidia.spark.rapids.tool.analysis import scala.collection.breakOut import scala.collection.mutable.{ArrayBuffer, HashMap, LinkedHashMap} -import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.analysis.util.{AggAccumHelper, AggAccumPhotonHelper} +import com.nvidia.spark.rapids.tool.analysis.util.StageAccumDiagnosticMetrics._ import com.nvidia.spark.rapids.tool.planparser.DatabricksParseHelper import com.nvidia.spark.rapids.tool.profiling._ diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala new file mode 100644 index 000000000..272cfb07c --- /dev/null +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/analysis/util/DiagnosticMetrics.scala @@ -0,0 +1,117 @@ +/* + * Copyright (c) 2024-2025, NVIDIA CORPORATION. + * + * Licensed 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 com.nvidia.spark.rapids.tool.analysis.util + +object StageAccumDiagnosticMetrics { + val MEMORY_SPILLED_METRIC = "internal.metrics.memoryBytesSpilled" + val DISK_SPILLED_METRIC = "internal.metrics.diskBytesSpilled" + val INPUT_BYTES_READ_METRIC = "internal.metrics.input.bytesRead" + val OUTPUT_BYTES_WRITTEN_METRIC = "internal.metrics.output.bytesWritten" + val SW_TOTAL_BYTES_METRIC = "internal.metrics.shuffle.write.bytesWritten" + val SR_FETCH_WAIT_TIME_METRIC = "internal.metrics.shuffle.read.fetchWaitTime" + val SW_WRITE_TIME_METRIC = "internal.metrics.shuffle.write.writeTime" + val GPU_SEMAPHORE_WAIT_METRIC = "gpuSemaphoreWait" + + /** + * Set of all diagnostic metrics + */ + lazy val allDiagnosticMetrics: Set[String] = Set(MEMORY_SPILLED_METRIC, + DISK_SPILLED_METRIC, INPUT_BYTES_READ_METRIC, OUTPUT_BYTES_WRITTEN_METRIC, + SW_TOTAL_BYTES_METRIC, SR_FETCH_WAIT_TIME_METRIC, SW_WRITE_TIME_METRIC, + GPU_SEMAPHORE_WAIT_METRIC) + + /** + * Check if a metric is diagnostic + */ + def isDiagnosticMetrics(metric: String): Boolean = { + allDiagnosticMetrics.contains(metric) + } +} + +object IOAccumDiagnosticMetrics { + // Metric keys to support variations in metric naming + val OUTPUT_ROWS_METRIC_KEY = "output rows" + val SCAN_TIME_METRIC_KEY = "scan time" + val OUTPUT_BATCHES_METRIC_KEY = "output batches" + val BUFFER_TIME_METRIC_KEY = "buffer time" + val SHUFFLE_WRITE_TIME_METRIC_KEY = "shuffle write time" + val FETCH_WAIT_TIME_METRIC_KEY = "fetch wait time" + val GPU_DECODE_TIME_METRIC_KEY = "GPU decode time" + + val OUTPUT_ROW_METRIC_NAMES = Set( + "number of output rows", // common across all Spark eventlogs + "output rows", // only in GPU eventlogs + "join output rows" // only in GPU eventlogs + ) + + val SCAN_TIME_METRIC_NAMES = Set( + "scan time" // common across all Spark eventlogs + ) + + val OUTPUT_BATCHES_METRIC_NAMES = Set( + "number of output batches", // only in Photon eventlogs + "output columnar batches" // only in GPU eventlogs + ) + + val BUFFER_TIME_METRIC_NAMES = Set( + "buffer time" // common across all Spark eventlogs + ) + + val SHUFFLE_WRITE_TIME_METRIC_NAMES = Set( + "shuffle write time", // common across all Spark eventlogs + "rs. shuffle write time" // only in GPU eventlogs + ) + + val FETCH_WAIT_TIME_METRIC_NAMES = Set( + "fetch wait time" // common across all Spark eventlogs + ) + + val GPU_DECODE_TIME_METRIC_NAMES = Set( + "GPU decode time" // only in GPU eventlogs + ) + + private val metricNamesToKeyMap: Map[String, String] = ( + OUTPUT_ROW_METRIC_NAMES.map(_ -> OUTPUT_ROWS_METRIC_KEY) ++ + SCAN_TIME_METRIC_NAMES.map(_ -> SCAN_TIME_METRIC_KEY) ++ + OUTPUT_BATCHES_METRIC_NAMES.map(_ -> OUTPUT_BATCHES_METRIC_KEY) ++ + BUFFER_TIME_METRIC_NAMES.map(_ -> BUFFER_TIME_METRIC_KEY) ++ + SHUFFLE_WRITE_TIME_METRIC_NAMES.map(_ -> SHUFFLE_WRITE_TIME_METRIC_KEY) ++ + FETCH_WAIT_TIME_METRIC_NAMES.map(_ -> FETCH_WAIT_TIME_METRIC_KEY) ++ + GPU_DECODE_TIME_METRIC_NAMES.map(_ -> GPU_DECODE_TIME_METRIC_KEY)).toMap + + /** + * Set of all IO diagnostic metrics names + */ + lazy val allIODiagnosticMetrics: Set[String] = metricNamesToKeyMap.keys.toSet + + /** + * Check if a metric name belongs to IO diagnostic metrics + */ + def isIODiagnosticMetricName(metric: String): Boolean = { + allIODiagnosticMetrics.contains(metric) + } + + /** + * Normalize a metric name to its IO diagnostic metric constant because we want to + * support variations in metric naming, e.g. "join output rows", "number of output rows" + * are different names for output rows metric. + */ + def normalizeToIODiagnosticMetricKey(metric: String): String = { + // input metric is already known to be an IO diagnostic metric + metricNamesToKeyMap(metric) + } +} diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala index b9c622334..6d677b072 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/CollectInformation.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -95,6 +95,10 @@ class CollectInformation(apps: Seq[ApplicationInfo]) extends Logging { ProfStageMetricView.getRawView(apps) } + def getIODiagnosticMetrics: Seq[IODiagnosticResult] = { + ProfIODiagnosticMetricsView.getRawView(apps) + } + /** * This function is meant to clean up Delta log execs so that you could align * SQL ids between CPU and GPU eventlogs. It attempts to remove any delta log diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala index 6ba34f7f5..ee5439c6b 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/DiagnosticSummaryInfo.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -21,5 +21,6 @@ package com.nvidia.spark.rapids.tool.profiling * TODO: We plan to add two more fields/views in upcoming PRs. */ case class DiagnosticSummaryInfo( - stageDiagnostics: Seq[StageDiagnosticResult] + stageDiagnostics: Seq[StageDiagnosticResult], + IODiagnostics: Seq[IODiagnosticResult] ) diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala index 796b97f20..27b5a2200 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/ProfileClassWarehouse.scala @@ -18,6 +18,8 @@ package com.nvidia.spark.rapids.tool.profiling import scala.collection.Map +import com.nvidia.spark.rapids.tool.analysis.StatisticsMetrics + import org.apache.spark.resource.{ExecutorResourceRequest, TaskResourceRequest} import org.apache.spark.sql.rapids.tool.store.AccumMetaRef import org.apache.spark.sql.rapids.tool.util.{SparkRuntime, StringUtils} @@ -74,16 +76,33 @@ case class JobInfoProfileResult( sqlID: Option[Long], startTime: Long, endTime: Option[Long]) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "jobID", "stageIds", "sqlID", "startTime", "endTime") + + override val outputHeaders = { + Seq("appIndex", + "jobID", + "stageIds", + "sqlID", + "startTime", + "endTime") + } + override def convertToSeq: Seq[String] = { val stageIdStr = s"[${stageIds.mkString(",")}]" - Seq(appIndex.toString, jobID.toString, stageIdStr, sqlID.map(_.toString).getOrElse(null), - startTime.toString, endTime.map(_.toString).getOrElse(null)) + Seq(appIndex.toString, + jobID.toString, + stageIdStr, + sqlID.map(_.toString).getOrElse(null), + startTime.toString, + endTime.map(_.toString).getOrElse(null)) } + override def convertToCSVSeq: Seq[String] = { val stageIdStr = s"[${stageIds.mkString(",")}]" - Seq(appIndex.toString, jobID.toString, StringUtils.reformatCSVString(stageIdStr), - sqlID.map(_.toString).getOrElse(null), startTime.toString, + Seq(appIndex.toString, + jobID.toString, + StringUtils.reformatCSVString(stageIdStr), + sqlID.map(_.toString).getOrElse(null), + startTime.toString, endTime.map(_.toString).getOrElse(null)) } } @@ -202,38 +221,108 @@ class SQLExecutionInfoClass( } } -case class SQLAccumProfileResults(appIndex: Int, sqlID: Long, nodeID: Long, - nodeName: String, accumulatorId: Long, name: String, min: Long, median: Long, - max: Long, total: Long, metricType: String, stageIds: String) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "sqlID", "nodeID", "nodeName", "accumulatorId", - "name", "min", "median", "max", "total", "metricType", "stageIds") +case class SQLAccumProfileResults( + appIndex: Int, + sqlID: Long, + nodeID: Long, + nodeName: String, + accumulatorId: Long, + name: String, + min: Long, + median: Long, + max: Long, + total: Long, + metricType: String, + stageIds: Set[Int]) extends ProfileResult { + + private val stageIdsStr = stageIds.mkString(",") + + override val outputHeaders = { + Seq("appIndex", + "sqlID", + "nodeID", + "nodeName", + "accumulatorId", + "name", + "min", + "median", + "max", + "total", + "metricType", + "stageIds") + } + override def convertToSeq: Seq[String] = { - Seq(appIndex.toString, sqlID.toString, nodeID.toString, nodeName, accumulatorId.toString, - name, min.toString, median.toString, max.toString, total.toString, metricType, stageIds) + Seq(appIndex.toString, + sqlID.toString, + nodeID.toString, + nodeName, + accumulatorId.toString, + name, + min.toString, + median.toString, + max.toString, + total.toString, + metricType, + stageIdsStr) } + override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, sqlID.toString, nodeID.toString, - StringUtils.reformatCSVString(nodeName), accumulatorId.toString, - StringUtils.reformatCSVString(name), min.toString, median.toString, max.toString, - total.toString, StringUtils.reformatCSVString(metricType), - StringUtils.reformatCSVString(stageIds)) + Seq(appIndex.toString, + sqlID.toString, + nodeID.toString, + StringUtils.reformatCSVString(nodeName), + accumulatorId.toString, + StringUtils.reformatCSVString(name), + min.toString, + median.toString, + max.toString, + total.toString, + StringUtils.reformatCSVString(metricType), + StringUtils.reformatCSVString(stageIdsStr)) } } -case class AccumProfileResults(appIndex: Int, stageId: Int, accMetaRef: AccumMetaRef, - min: Long, median: Long, max: Long, total: Long) extends ProfileResult { - override val outputHeaders = Seq("appIndex", "stageId", "accumulatorId", "name", "min", - "median", "max", "total") +case class AccumProfileResults( + appIndex: Int, + stageId: Int, + accMetaRef: AccumMetaRef, + min: Long, + median: Long, + max: Long, + total: Long) extends ProfileResult { + + override val outputHeaders = { + Seq("appIndex", + "stageId", + "accumulatorId", + "name", + "min", + "median", + "max", + "total") + } override def convertToSeq: Seq[String] = { - Seq(appIndex.toString, stageId.toString, accMetaRef.id.toString, accMetaRef.getName(), - min.toString, median.toString, max.toString, total.toString) + Seq(appIndex.toString, + stageId.toString, + accMetaRef.id.toString, + accMetaRef.getName(), + min.toString, + median.toString, + max.toString, + total.toString) } override def convertToCSVSeq: Seq[String] = { - Seq(appIndex.toString, stageId.toString, accMetaRef.id.toString, - accMetaRef.name.csvValue, min.toString, - median.toString, max.toString, total.toString) + Seq(appIndex.toString, + stageId.toString, + accMetaRef.id.toString, + accMetaRef.name.csvValue, + min.toString, + median.toString, + max.toString, + total.toString) } } @@ -599,6 +688,20 @@ case class StageAggTaskMetricsProfileResult( override def idHeader = "stageId" } +/** + * Represents diagnostic metrics results at task/stage level in a Spark SQL execution plan. + * Output file: stage_level_diagnostic_metrics.csv. + * Collected metrics include: + * - Memory spilled (MB) + * - Disk spilled (MB) + * - Input bytes read + * - Output bytes written + * - Shuffle read total bytes (remote + local) + * - Shuffle write bytes + * - Shuffle read fetch wait time (ms) + * - Shuffle write time (ms) + * - GPU semaphore time (ns) + */ case class StageDiagnosticResult( appIndex: Int, appName: String, @@ -925,6 +1028,153 @@ case class SQLTaskAggMetricsProfileResult( } } +/** + * Represents IO-related diagnostic metrics results in a Spark SQL execution plan. + * Output file: io_diagnostic_metrics.csv. + * Collected metrics include: + * - Output rows + * - Scan time (ns) + * - Output batches + * - Buffer time (ns) + * - Shuffle write time (ns) + * - Fetch wait time (ns) + * - GPU decode time (ns) + */ +case class IODiagnosticResult( + appIndex: Int, + appName: String, + appId: String, + sqlId: Long, + stageId: Long, + duration: Long, + nodeId: Long, + nodeName: String, + outputRows: StatisticsMetrics, + scanTime: StatisticsMetrics, + outputBatches: StatisticsMetrics, + bufferTime: StatisticsMetrics, + shuffleWriteTime: StatisticsMetrics, + fetchWaitTime: StatisticsMetrics, + gpuDecodeTime: StatisticsMetrics) extends ProfileResult { + + override val outputHeaders = { + Seq("appIndex", + "appName", + "appId", + "sqlId", + "stageId", + "stageDurationMs", + "nodeId", + "nodeName", + "outputRowsMin", + "outputRowsMedian", + "outputRowsMax", + "outputRowsTotal", + "scanTimeMin", + "scanTimeMedian", + "scanTimeMax", + "scanTimeTotal", + "outputBatchesMin", + "outputBatchesMedian", + "outputBatchesMax", + "outputBatchesTotal", + "bufferTimeMin", + "bufferTimeMedian", + "bufferTimeMax", + "bufferTimeTotal", + "shuffleWriteTimeMin", + "shuffleWriteTimeMedian", + "shuffleWriteTimeMax", + "shuffleWriteTimeTotal", + "fetchWaitTimeMin", + "fetchWaitTimeMedian", + "fetchWaitTimeMax", + "fetchWaitTimeTotal", + "gpuDecodeTimeMin", + "gpuDecodeTimeMedian", + "gpuDecodeTimeMax", + "gpuDecodeTimeTotal") + } + + override def convertToSeq: Seq[String] = { + Seq(appIndex.toString, + appName, + appId, + sqlId.toString, + stageId.toString, + duration.toString, + nodeId.toString, + nodeName, + outputRows.min.toString, + outputRows.med.toString, + outputRows.max.toString, + outputRows.total.toString, + scanTime.min.toString, + scanTime.med.toString, + scanTime.max.toString, + scanTime.total.toString, + outputBatches.min.toString, + outputBatches.med.toString, + outputBatches.max.toString, + outputBatches.total.toString, + bufferTime.min.toString, + bufferTime.med.toString, + bufferTime.max.toString, + bufferTime.total.toString, + shuffleWriteTime.min.toString, + shuffleWriteTime.med.toString, + shuffleWriteTime.max.toString, + shuffleWriteTime.total.toString, + fetchWaitTime.min.toString, + fetchWaitTime.med.toString, + fetchWaitTime.max.toString, + fetchWaitTime.total.toString, + gpuDecodeTime.min.toString, + gpuDecodeTime.med.toString, + gpuDecodeTime.max.toString, + gpuDecodeTime.total.toString) + } + + override def convertToCSVSeq: Seq[String] = { + Seq(appIndex.toString, + appName, + appId, + sqlId.toString, + stageId.toString, + duration.toString, + nodeId.toString, + StringUtils.reformatCSVString(nodeName), + outputRows.min.toString, + outputRows.med.toString, + outputRows.max.toString, + outputRows.total.toString, + scanTime.min.toString, + scanTime.med.toString, + scanTime.max.toString, + scanTime.total.toString, + outputBatches.min.toString, + outputBatches.med.toString, + outputBatches.max.toString, + outputBatches.total.toString, + bufferTime.min.toString, + bufferTime.med.toString, + bufferTime.max.toString, + bufferTime.total.toString, + shuffleWriteTime.min.toString, + shuffleWriteTime.med.toString, + shuffleWriteTime.max.toString, + shuffleWriteTime.total.toString, + fetchWaitTime.min.toString, + fetchWaitTime.med.toString, + fetchWaitTime.max.toString, + fetchWaitTime.total.toString, + gpuDecodeTime.min.toString, + gpuDecodeTime.med.toString, + gpuDecodeTime.max.toString, + gpuDecodeTime.total.toString) + } +} + case class IOAnalysisProfileResult( appIndex: Int, appId: String, diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala index 592fa2cbb..2a7069296 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/profiling/Profiler.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2024, NVIDIA CORPORATION. + * Copyright (c) 2021-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -405,7 +405,7 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea failedTasks, failedStages, failedJobs, removedBMs, removedExecutors, unsupportedOps, sparkProps, collect.getSQLToStage, wholeStage, maxTaskInputInfo, appLogPath, analysis.ioAggs, systemProps, sqlIdAlign, sparkRapidsBuildInfo), - compareRes, DiagnosticSummaryInfo(analysis.stageDiagnostics)) + compareRes, DiagnosticSummaryInfo(analysis.stageDiagnostics, collect.getIODiagnosticMetrics)) } /** @@ -573,12 +573,15 @@ class Profiler(hadoopConf: Configuration, appArgs: ProfileArgs, enablePB: Boolea } // Write diagnostic related results to CSV files val diagnostics = if (outputCombined) { - Seq(DiagnosticSummaryInfo(diagnosticSum.flatMap(_.stageDiagnostics))) + Seq(DiagnosticSummaryInfo(diagnosticSum.flatMap(_.stageDiagnostics), + diagnosticSum.flatMap(_.IODiagnostics))) } else { diagnosticSum } diagnostics.foreach { diagnostoic => profileOutputWriter.writeCSVTable(STAGE_DIAGNOSTICS_LABEL, diagnostoic.stageDiagnostics) + profileOutputWriter.writeCSVTable(ProfIODiagnosticMetricsView.getLabel, + diagnostoic.IODiagnostics) } } diff --git a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala index e52340dce..d41b9f0c4 100644 --- a/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala +++ b/core/src/main/scala/com/nvidia/spark/rapids/tool/views/SQLView.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,7 +17,7 @@ package com.nvidia.spark.rapids.tool.views import com.nvidia.spark.rapids.tool.analysis.{AppSQLPlanAnalyzer, ProfAppIndexMapperTrait, QualAppIndexMapperTrait} -import com.nvidia.spark.rapids.tool.profiling.{SQLAccumProfileResults, SQLCleanAndAlignIdsProfileResult, SQLPlanClassifier, WholeStageCodeGenResults} +import com.nvidia.spark.rapids.tool.profiling.{IODiagnosticResult, SQLAccumProfileResults, SQLCleanAndAlignIdsProfileResult, SQLPlanClassifier, WholeStageCodeGenResults} import org.apache.spark.sql.rapids.tool.AppBase import org.apache.spark.sql.rapids.tool.profiling.ApplicationInfo @@ -109,6 +109,25 @@ object ProfSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with ProfAppInd } } +object ProfIODiagnosticMetricsView extends ViewableTrait[IODiagnosticResult] + with ProfAppIndexMapperTrait { + override def getLabel: String = "IO Diagnostic Metrics" + override def getDescription: String = "IO Diagnostic Metrics" + + override def sortView( + rows: Seq[IODiagnosticResult]): Seq[IODiagnosticResult] = { + rows.sortBy(cols => (cols.appIndex, -cols.duration, cols.stageId, cols.sqlId, cols.nodeId)) + } + + override def getRawView(app: AppBase, index: Int): Seq[IODiagnosticResult] = { + app match { + case app: ApplicationInfo => + sortView(app.planMetricProcessor.generateIODiagnosticAccums()) + case _ => Seq.empty + } + } +} + object QualSQLPlanMetricsView extends AppSQLPlanMetricsViewTrait with QualAppIndexMapperTrait { override def getRawView(app: AppBase, index: Int): Seq[SQLAccumProfileResults] = { // TODO: Fix this implementation when we have a better way to get bind between App and diff --git a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala index 4ce41e4a5..06353af21 100644 --- a/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala +++ b/core/src/main/scala/org/apache/spark/sql/rapids/tool/store/AccumNameRef.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2024, NVIDIA CORPORATION. + * Copyright (c) 2024-2025, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -18,8 +18,6 @@ package org.apache.spark.sql.rapids.tool.store import java.util.concurrent.ConcurrentHashMap -import com.nvidia.spark.rapids.tool.analysis.StageAccumDiagnosticMetrics.getAllDiagnosticMetrics - import org.apache.spark.sql.rapids.tool.util.EventUtils.normalizeMetricName import org.apache.spark.sql.rapids.tool.util.StringUtils @@ -35,8 +33,6 @@ case class AccumNameRef(value: String) { // We opt to use this optimization because while writing the CSV files, each row is going to // create a new CSV string even though they represent the same AccumulatorName. val csvValue: String = StringUtils.reformatCSVString(value) - - def isDiagnosticMetrics(): Boolean = getAllDiagnosticMetrics.contains(value) } object AccumNameRef { diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv new file mode 100644 index 000000000..8b1d3093e --- /dev/null +++ b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_iodiagnosticmetrics_expectation.csv @@ -0,0 +1,16 @@ +appIndex,appName,appId,sqlId,stageId,stageDurationMs,nodeId,nodeName,outputRowsMin,outputRowsMedian,outputRowsMax,outputRowsTotal,scanTimeMin,scanTimeMedian,scanTimeMax,scanTimeTotal,outputBatchesMin,outputBatchesMedian,outputBatchesMax,outputBatchesTotal,bufferTimeMin,bufferTimeMedian,bufferTimeMax,bufferTimeTotal,shuffleWriteTimeMin,shuffleWriteTimeMedian,shuffleWriteTimeMax,shuffleWriteTimeTotal,fetchWaitTimeMin,fetchWaitTimeMedian,fetchWaitTimeMax,fetchWaitTimeTotal,gpuDecodeTimeMin,gpuDecodeTimeMedian,gpuDecodeTimeMax,gpuDecodeTimeTotal +1,Spark shell,local-1622814619968,0,0,1743,16,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,41434653,60830365,100858775,400284505,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,0,1743,21,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,1,1631,8,"GpuColumnarExchange",1666666,1666667,1666667,10000000,0,0,0,0,200,200,200,1200,0,0,0,0,37444140,92128351,108992798,508750471,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,1,1631,13,"Scan",1666666,1666667,1666667,10000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,3,"GpuColumnarExchange",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,139875,230038,9747416,93193331,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,4,"GpuHashAggregate",1,1,1,200,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,6,"GpuShuffledHashJoin",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,7,"GpuShuffleCoalesce",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,8,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,14,"GpuCoalesceBatches",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,15,"GpuShuffleCoalesce",49480,50017,50559,10000000,0,0,0,0,1,1,1,200,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,2,688,16,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,1,"GpuHashAggregate",1,1,1,1,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,2,"GpuShuffleCoalesce",200,200,200,200,0,0,0,0,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +1,Spark shell,local-1622814619968,0,3,83,3,"GpuColumnarExchange",0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 diff --git a/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv b/core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetrics_expectation.csv similarity index 100% rename from core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv rename to core/src/test/resources/ProfilingExpectations/rapids_join_eventlog_stagediagnosticmetrics_expectation.csv diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala index b33afb21a..83304b9d0 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/AnalysisSuite.scala @@ -67,30 +67,135 @@ case class TestStageDiagnosticResult( gpuSemaphoreWaitSum: Long, nodeNames: Seq[String]) +case class TestIODiagnosticResult( + appIndex: Int, + appName: String, + appId: String, + sqlId: Long, + stageId: Long, + duration: Long, + nodeId: Long, + nodeName: String, + outputRowsMin: Long, + outputRowsMed: Long, + outputRowsMax: Long, + outputRowsSum: Long, + scanTimeMin: Long, + scanTimeMed: Long, + scanTimeMax: Long, + scanTimeSum: Long, + outputBatchesMin: Long, + outputBatchesMed: Long, + outputBatchesMax: Long, + outputBatchesSum: Long, + bufferTimeMin: Long, + bufferTimeMed: Long, + bufferTimeMax: Long, + bufferTimeSum: Long, + shuffleWriteTimeMin: Long, + shuffleWriteTimeMed: Long, + shuffleWriteTimeMax: Long, + shuffleWriteTimeSum: Long, + fetchWaitTimeMin: Long, + fetchWaitTimeMed: Long, + fetchWaitTimeMax: Long, + fetchWaitTimeSum: Long, + gpuDecodeTimeMin: Long, + gpuDecodeTimeMed: Long, + gpuDecodeTimeMax: Long, + gpuDecodeTimeSum: Long) + class AnalysisSuite extends FunSuite { private def createTestStageDiagnosticResult(diagnosticsResults: Seq[StageDiagnosticResult]): Seq[TestStageDiagnosticResult] = { def bytesToMB(numBytes: Long): Long = numBytes / (1024 * 1024) def nanoToMilliSec(numNano: Long): Long = numNano / 1000000 + diagnosticsResults.map { result => + TestStageDiagnosticResult( + result.appIndex, + result.appName, + result.appId, + result.stageId, + result.duration, + result.numTasks, + bytesToMB(result.memoryBytesSpilled.min), + bytesToMB(result.memoryBytesSpilled.median), + bytesToMB(result.memoryBytesSpilled.max), + bytesToMB(result.memoryBytesSpilled.total), + bytesToMB(result.diskBytesSpilled.min), + bytesToMB(result.diskBytesSpilled.median), + bytesToMB(result.diskBytesSpilled.max), + bytesToMB(result.diskBytesSpilled.total), + result.inputBytesRead.min, + result.inputBytesRead.median, + result.inputBytesRead.max, + result.inputBytesRead.total, + result.outputBytesWritten.min, + result.outputBytesWritten.median, + result.outputBytesWritten.max, + result.outputBytesWritten.total, + result.srTotalBytesReadMin, + result.srTotalBytesReadMed, + result.srTotalBytesReadMax, + result.srTotalBytesReadSum, + result.swBytesWritten.min, + result.swBytesWritten.median, + result.swBytesWritten.max, + result.swBytesWritten.total, + nanoToMilliSec(result.srFetchWaitTime.min), + nanoToMilliSec(result.srFetchWaitTime.median), + nanoToMilliSec(result.srFetchWaitTime.max), + nanoToMilliSec(result.srFetchWaitTime.total), + nanoToMilliSec(result.swWriteTime.min), + nanoToMilliSec(result.swWriteTime.median), + nanoToMilliSec(result.swWriteTime.max), + nanoToMilliSec(result.swWriteTime.total), + result.gpuSemaphoreWait.total, + result.nodeNames) + } + } + + private def createTestIODiagnosticResult(diagnosticsResults: Seq[IODiagnosticResult]): + Seq[TestIODiagnosticResult] = { diagnosticsResults.map {result => - TestStageDiagnosticResult(result.appIndex, result.appName, result.appId, result.stageId, - result.duration, result.numTasks, bytesToMB(result.memoryBytesSpilled.min), - bytesToMB(result.memoryBytesSpilled.median), bytesToMB(result.memoryBytesSpilled.max), - bytesToMB(result.memoryBytesSpilled.total), bytesToMB(result.diskBytesSpilled.min), - bytesToMB(result.diskBytesSpilled.median), bytesToMB(result.diskBytesSpilled.max), - bytesToMB(result.diskBytesSpilled.total), result.inputBytesRead.min, - result.inputBytesRead.median, result.inputBytesRead.max, result.inputBytesRead.total, - result.outputBytesWritten.min, result.outputBytesWritten.median, - result.outputBytesWritten.max, result.outputBytesWritten.total, - result.srTotalBytesReadMin, result.srTotalBytesReadMed, result.srTotalBytesReadMax, - result.srTotalBytesReadSum, result.swBytesWritten.min, result.swBytesWritten.median, - result.swBytesWritten.max, result.swBytesWritten.total, - nanoToMilliSec(result.srFetchWaitTime.min), nanoToMilliSec(result.srFetchWaitTime.median), - nanoToMilliSec(result.srFetchWaitTime.max), nanoToMilliSec(result.srFetchWaitTime.total), - nanoToMilliSec(result.swWriteTime.min), nanoToMilliSec(result.swWriteTime.median), - nanoToMilliSec(result.swWriteTime.max), nanoToMilliSec(result.swWriteTime.total), - result.gpuSemaphoreWait.total, result.nodeNames) + TestIODiagnosticResult( + result.appIndex, + result.appName, + result.appId, + result.sqlId, + result.stageId, + result.duration, + result.nodeId, + result.nodeName, + result.outputRows.min, + result.outputRows.med, + result.outputRows.max, + result.outputRows.total, + result.scanTime.min, + result.scanTime.med, + result.scanTime.max, + result.scanTime.total, + result.outputBatches.min, + result.outputBatches.med, + result.outputBatches.max, + result.outputBatches.total, + result.bufferTime.min, + result.bufferTime.med, + result.bufferTime.max, + result.bufferTime.total, + result.shuffleWriteTime.min, + result.shuffleWriteTime.med, + result.shuffleWriteTime.max, + result.shuffleWriteTime.total, + result.fetchWaitTime.min, + result.fetchWaitTime.med, + result.fetchWaitTime.max, + result.fetchWaitTime.total, + result.gpuDecodeTime.min, + result.gpuDecodeTime.med, + result.gpuDecodeTime.max, + result.gpuDecodeTime.total) } } @@ -143,8 +248,8 @@ class AnalysisSuite extends FunSuite { platformName = PlatformNames.DATABRICKS_AWS) } - test("test stage-level diagnostic aggregation simple") { - val expectFile = "rapids_join_eventlog_stagediagnosticmetricsagg_expectation.csv" + test("test stage-level diagnostic metrics") { + val expectFile = "rapids_join_eventlog_stagediagnosticmetrics_expectation.csv" val logs = Array(s"$logDir/rapids_join_eventlog.zstd") val apps = ToolTestUtils.processProfileApps(logs, sparkSession) assert(apps.size == logs.size) @@ -155,14 +260,30 @@ class AnalysisSuite extends FunSuite { collect.getSQLToStage collect.getStageLevelMetrics - val aggResults = RawMetricProfilerView.getAggMetrics(apps) + val diagnosticResults = RawMetricProfilerView.getAggMetrics(apps) import org.apache.spark.sql.functions._ import sparkSession.implicits._ - val actualDf = createTestStageDiagnosticResult(aggResults.stageDiagnostics).toDF. + val actualDf = createTestStageDiagnosticResult(diagnosticResults.stageDiagnostics).toDF. withColumn("nodeNames", concat_ws(",", col("nodeNames"))) compareMetrics(actualDf, expectFile) } + test("test IO diagnostic metrics") { + val expectFile = "rapids_join_eventlog_iodiagnosticmetrics_expectation.csv" + val logs = Array(s"$logDir/rapids_join_eventlog.zstd") + val apps = ToolTestUtils.processProfileApps(logs, sparkSession) + assert(apps.size == logs.size) + + val collect = new CollectInformation(apps) + // Computes IO diagnostic metrics mapping which is later used in getIODiagnosticMetrics + collect.getSQLPlanMetrics + val diagnosticResults = collect.getIODiagnosticMetrics + + import sparkSession.implicits._ + val actualDf = createTestIODiagnosticResult(diagnosticResults).toDF + compareMetrics(actualDf, expectFile) + } + private def testSqlMetricsAggregation(logs: Array[String], expectFileSQL: String, expectFileJob: String, expectFileStage: String, platformName: String = PlatformNames.DEFAULT): Unit = { diff --git a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala index 3791aee89..dd879a19b 100644 --- a/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala +++ b/core/src/test/scala/com/nvidia/spark/rapids/tool/profiling/ApplicationInfoSuite.scala @@ -196,11 +196,10 @@ class ApplicationInfoSuite extends FunSuite with Logging { val resultExpectation = new File(expRoot, "rapids_join_eventlog_sqlmetrics_expectation.csv") assert(sqlMetrics.size == 83) - val sqlMetricsWithDelim = sqlMetrics.map { metrics => - metrics.copy(stageIds = ProfileUtils.replaceDelimiter(metrics.stageIds, ",")) - } + + import org.apache.spark.sql.functions._ import sparkSession.implicits._ - val df = sqlMetricsWithDelim.toDF + val df = sqlMetrics.toDF.withColumn("stageIds", concat_ws(",", col("stageIds"))) val dfExpect = ToolTestUtils.readExpectationCSV(sparkSession, resultExpectation.getPath()) ToolTestUtils.compareDataFrames(df, dfExpect) } @@ -842,7 +841,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 21) + assert(dotDirs.length === 22) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -876,7 +875,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 17) + assert(dotDirs.length === 18) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -913,7 +912,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 21) + assert(dotDirs.length === 22) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly @@ -950,7 +949,7 @@ class ApplicationInfoSuite extends FunSuite with Logging { f.endsWith(".csv") }) // compare the number of files generated - assert(dotDirs.length === 19) + assert(dotDirs.length === 20) for (file <- dotDirs) { assert(file.getAbsolutePath.endsWith(".csv")) // just load each one to make sure formatted properly